From 48edef4cdb1364b91b7e6b19eed89e92ba78d7f7 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Tue, 5 Aug 2025 12:27:41 -0400 Subject: [PATCH 01/11] add catalog-database-table hierarchy --- .../trigger_files/beam_PostCommit_SQL.json | 2 +- ...ommit_XVR_PythonUsingJavaSQL_Dataflow.json | 2 +- .../src/main/codegen/includes/parserImpls.ftl | 24 +- .../beam/sdk/extensions/sql/SqlTransform.java | 14 +- .../beam/sdk/extensions/sql/TableUtils.java | 4 + .../sql/impl/BeamCalciteSchema.java | 53 +-- .../sdk/extensions/sql/impl/BeamSqlEnv.java | 18 +- .../sql/impl/CatalogManagerSchema.java | 237 ++++++++++++ .../extensions/sql/impl/CatalogSchema.java | 247 +++++++++++++ .../extensions/sql/impl/JdbcConnection.java | 6 +- .../sdk/extensions/sql/impl/TableName.java | 30 ++ .../sql/impl/parser/SqlCreateCatalog.java | 43 +-- .../sql/impl/parser/SqlCreateDatabase.java | 80 ++-- .../impl/parser/SqlCreateExternalTable.java | 47 ++- .../sql/impl/parser/SqlDdlNodes.java | 29 +- .../sql/impl/parser/SqlDropCatalog.java | 46 +-- .../sql/impl/parser/SqlDropDatabase.java | 59 +-- .../sql/impl/parser/SqlDropTable.java | 45 +++ .../sql/impl/parser/SqlUseCatalog.java | 38 +- .../sql/impl/parser/SqlUseDatabase.java | 61 ++-- .../beam/sdk/extensions/sql/meta/Table.java | 3 +- .../extensions/sql/meta/catalog/Catalog.java | 14 +- .../sql/meta/catalog/CatalogManager.java | 10 +- .../sql/meta/catalog/EmptyCatalogManager.java | 15 +- .../sql/meta/catalog/InMemoryCatalog.java | 30 +- .../meta/catalog/InMemoryCatalogManager.java | 32 +- .../meta/provider/iceberg/IcebergCatalog.java | 25 +- .../provider/iceberg/IcebergMetastore.java | 149 ++++++++ .../meta/provider/iceberg/IcebergTable.java | 5 +- .../iceberg/IcebergTableProvider.java | 96 ----- .../sql/meta/store/InMemoryMetaStore.java | 54 ++- .../extensions/sql/meta/store/MetaStore.java | 3 + .../extensions/sql/BeamSqlCliCatalogTest.java | 341 ++++++++++++++++++ .../sql/BeamSqlCliDatabaseTest.java | 108 +++++- .../sdk/extensions/sql/BeamSqlCliTest.java | 176 --------- .../extensions/sql/impl/JdbcDriverTest.java | 22 +- .../impl/parser/BeamDDLNestedTypesTest.java | 2 +- .../sql/impl/parser/BeamDDLTest.java | 28 +- .../extensions/sql/impl/rel/BaseRelTest.java | 2 + .../sql/impl/rule/JoinReorderingTest.java | 6 +- .../iceberg/BeamSqlCliIcebergTest.java | 114 +++++- ...derTest.java => IcebergMetastoreTest.java} | 8 +- .../provider/iceberg/IcebergReadWriteIT.java | 7 +- .../sql/meta/store/InMemoryMetaStoreTest.java | 1 + .../sdk/io/iceberg/IcebergCatalogConfig.java | 30 ++ .../beam/sdk/tpcds/BeamSqlEnvRunner.java | 4 +- 46 files changed, 1692 insertions(+), 678 deletions(-) create mode 100644 sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogManagerSchema.java create mode 100644 sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogSchema.java create mode 100644 sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastore.java delete mode 100644 sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProvider.java create mode 100644 sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliCatalogTest.java rename sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/{IcebergTableProviderTest.java => IcebergMetastoreTest.java} (92%) diff --git a/.github/trigger_files/beam_PostCommit_SQL.json b/.github/trigger_files/beam_PostCommit_SQL.json index 833fd9b0d174..6cc79a7a0325 100644 --- a/.github/trigger_files/beam_PostCommit_SQL.json +++ b/.github/trigger_files/beam_PostCommit_SQL.json @@ -1,4 +1,4 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run ", - "modification": 2 + "modification": 1 } diff --git a/.github/trigger_files/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.json b/.github/trigger_files/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.json index bb31ea07c195..ca2897e2eb2b 100644 --- a/.github/trigger_files/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.json +++ b/.github/trigger_files/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.json @@ -1,3 +1,3 @@ { - "modification": 1 + "modification": 2 } \ No newline at end of file diff --git a/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl b/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl index 470cbb443895..9ca50468e84c 100644 --- a/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl +++ b/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl @@ -270,7 +270,7 @@ SqlDrop SqlDropCatalog(Span s, boolean replace) : SqlCreate SqlCreateDatabase(Span s, boolean replace) : { final boolean ifNotExists; - final SqlNode databaseName; + final SqlIdentifier databaseName; } { { @@ -278,11 +278,7 @@ SqlCreate SqlCreateDatabase(Span s, boolean replace) : } ifNotExists = IfNotExistsOpt() - ( - databaseName = StringLiteral() - | - databaseName = SimpleIdentifier() - ) + databaseName = CompoundIdentifier() { return new SqlCreateDatabase( @@ -298,18 +294,14 @@ SqlCreate SqlCreateDatabase(Span s, boolean replace) : */ SqlCall SqlUseDatabase(Span s, String scope) : { - final SqlNode databaseName; + final SqlIdentifier databaseName; } { { s.add(this); } - ( - databaseName = StringLiteral() - | - databaseName = SimpleIdentifier() - ) + databaseName = CompoundIdentifier() { return new SqlUseDatabase( s.end(this), @@ -324,17 +316,13 @@ SqlCall SqlUseDatabase(Span s, String scope) : SqlDrop SqlDropDatabase(Span s, boolean replace) : { final boolean ifExists; - final SqlNode databaseName; + final SqlIdentifier databaseName; final boolean cascade; } { ifExists = IfExistsOpt() - ( - databaseName = StringLiteral() - | - databaseName = SimpleIdentifier() - ) + databaseName = CompoundIdentifier() cascade = CascadeOpt() diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java index f9cc1fd9d482..70ed5cdd6d98 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java @@ -31,6 +31,7 @@ import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils; import org.apache.beam.sdk.extensions.sql.impl.schema.BeamPCollectionTable; import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable; +import org.apache.beam.sdk.extensions.sql.meta.catalog.CatalogManager; import org.apache.beam.sdk.extensions.sql.meta.catalog.InMemoryCatalogManager; import org.apache.beam.sdk.extensions.sql.meta.provider.ReadOnlyTableProvider; import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; @@ -128,6 +129,8 @@ public abstract class SqlTransform extends PTransform> abstract Map tableProviderMap(); + abstract CatalogManager catalogManager(); + abstract @Nullable String defaultTableProvider(); abstract @Nullable String queryPlannerClassName(); @@ -136,8 +139,8 @@ public abstract class SqlTransform extends PTransform> public PCollection expand(PInput input) { TableProvider inputTableProvider = new ReadOnlyTableProvider(PCOLLECTION_NAME, toTableMap(input)); - InMemoryCatalogManager catalogManager = new InMemoryCatalogManager(); - catalogManager.registerTableProvider(PCOLLECTION_NAME, inputTableProvider); + CatalogManager catalogManager = catalogManager(); + catalogManager.registerTableProvider(inputTableProvider); BeamSqlEnvBuilder sqlEnvBuilder = BeamSqlEnv.builder(catalogManager); // TODO: validate duplicate functions. @@ -240,6 +243,10 @@ public SqlTransform withDefaultTableProvider(String name, TableProvider tablePro return withTableProvider(name, tableProvider).toBuilder().setDefaultTableProvider(name).build(); } + public SqlTransform withCatalogManager(CatalogManager catalogManager) { + return toBuilder().setCatalogManager(catalogManager).build(); + } + public SqlTransform withQueryPlannerClass(Class clazz) { return toBuilder().setQueryPlannerClassName(clazz.getName()).build(); } @@ -313,6 +320,7 @@ static Builder builder() { .setUdafDefinitions(Collections.emptyList()) .setUdfDefinitions(Collections.emptyList()) .setTableProviderMap(Collections.emptyMap()) + .setCatalogManager(new InMemoryCatalogManager()) .setAutoLoading(true); } @@ -333,6 +341,8 @@ abstract static class Builder { abstract Builder setTableProviderMap(Map tableProviderMap); + abstract Builder setCatalogManager(CatalogManager catalogManager); + abstract Builder setDefaultTableProvider(@Nullable String defaultTableProvider); abstract Builder setQueryPlannerClassName(@Nullable String queryPlannerClassName); diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/TableUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/TableUtils.java index 2e52a1bbf422..5285999f3292 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/TableUtils.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/TableUtils.java @@ -63,6 +63,10 @@ public static ObjectNode parseProperties(String json) { } } + public static ObjectNode parseProperties(Map map) { + return objectMapper.valueToTree(map); + } + public static Map convertNode2Map(JsonNode jsonNode) { return objectMapper.convertValue(jsonNode, new TypeReference>() {}); } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteSchema.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteSchema.java index 0b015c567cda..963c54285f2c 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteSchema.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteSchema.java @@ -17,8 +17,6 @@ */ package org.apache.beam.sdk.extensions.sql.impl; -import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; - import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -41,32 +39,25 @@ @SuppressWarnings({"keyfor", "nullness"}) // TODO(https://github.com/apache/beam/issues/20497) public class BeamCalciteSchema implements Schema { private JdbcConnection connection; - private @Nullable TableProvider tableProvider; - private @Nullable CatalogManager catalogManager; + private TableProvider tableProvider; private Map subSchemas; + private final String name; - BeamCalciteSchema(JdbcConnection jdbcConnection, TableProvider tableProvider) { + /** Creates a {@link BeamCalciteSchema} representing a {@link TableProvider}. */ + BeamCalciteSchema(String name, JdbcConnection jdbcConnection, TableProvider tableProvider) { + System.out.println("xxx [BeamCalciteSchema] init: " + tableProvider.getTableType()); this.connection = jdbcConnection; this.tableProvider = tableProvider; this.subSchemas = new HashMap<>(); + this.name = name; } - /** - * Creates a {@link BeamCalciteSchema} representing a {@link CatalogManager}. This will typically - * be the root node of a pipeline. - */ - BeamCalciteSchema(JdbcConnection jdbcConnection, CatalogManager catalogManager) { - this.connection = jdbcConnection; - this.catalogManager = catalogManager; - this.subSchemas = new HashMap<>(); + public String name() { + return name; } public TableProvider getTableProvider() { - return resolveMetastore(); - } - - public @Nullable CatalogManager getCatalogManager() { - return catalogManager; + return tableProvider; } public Map getPipelineOptions() { @@ -106,7 +97,7 @@ public Expression getExpression(SchemaPlus parentSchema, String name) { @Override public Set getTableNames() { - return resolveMetastore().getTables().keySet(); + return tableProvider.getTables().keySet(); } @Override @@ -122,13 +113,13 @@ public Set getTypeNames() { @Override public org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Table getTable( String name) { - Table table = resolveMetastore().getTable(name); + Table table = tableProvider.getTable(name); if (table == null) { return null; } return new BeamCalciteTable( - resolveMetastore().buildBeamSqlTable(table), - getPipelineOptions(), + tableProvider.buildBeamSqlTable(table), + connection.getPipelineOptionsMap(), connection.getPipelineOptions()); } @@ -144,7 +135,7 @@ public Collection getFunctions(String name) { @Override public Set getSubSchemaNames() { - return resolveMetastore().getSubProviders(); + return tableProvider.getSubProviders(); } /** @@ -157,23 +148,11 @@ public Set getSubSchemaNames() { public Schema getSubSchema(String name) { if (!subSchemas.containsKey(name)) { BeamCalciteSchema subSchema; - if (tableProvider != null) { - @Nullable TableProvider subProvider = tableProvider.getSubProvider(name); - subSchema = subProvider != null ? new BeamCalciteSchema(connection, subProvider) : null; - } else { - @Nullable Catalog catalog = checkStateNotNull(catalogManager).getCatalog(name); - subSchema = catalog != null ? new BeamCalciteSchema(connection, catalog.metaStore()) : null; - } + @Nullable TableProvider subProvider = tableProvider.getSubProvider(name); + subSchema = subProvider != null ? new BeamCalciteSchema(name, connection, subProvider) : null; subSchemas.put(name, subSchema); } return subSchemas.get(name); } - - public TableProvider resolveMetastore() { - if (tableProvider != null) { - return tableProvider; - } - return checkStateNotNull(catalogManager).currentCatalog().metaStore(); - } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSqlEnv.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSqlEnv.java index 1edb22ac105f..96a2428597d8 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSqlEnv.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSqlEnv.java @@ -41,6 +41,7 @@ import org.apache.beam.sdk.extensions.sql.meta.provider.ReadOnlyTableProvider; import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; import org.apache.beam.sdk.extensions.sql.meta.provider.UdfUdafProvider; +import org.apache.beam.sdk.extensions.sql.meta.store.MetaStore; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.Combine.CombineFn; @@ -51,7 +52,6 @@ import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.RuleSet; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; -import org.checkerframework.checker.nullness.qual.Nullable; /** * Contains the metadata of tables/UDF functions, and exposes APIs to @@ -150,7 +150,6 @@ public static class BeamSqlEnvBuilder { private static final String CALCITE_PLANNER = "org.apache.beam.sdk.extensions.sql.impl.CalciteQueryPlanner"; private String queryPlannerClassName; - private @Nullable TableProvider defaultTableProvider; private CatalogManager catalogManager; private String currentSchemaName; private Map schemaMap; @@ -162,8 +161,12 @@ public static class BeamSqlEnvBuilder { private BeamSqlEnvBuilder(TableProvider tableProvider) { checkNotNull(tableProvider, "Table provider for the default schema must be sets."); - defaultTableProvider = tableProvider; - catalogManager = new InMemoryCatalogManager(); + if (tableProvider instanceof MetaStore) { + catalogManager = new InMemoryCatalogManager((MetaStore) tableProvider); + } else { + catalogManager = new InMemoryCatalogManager(); + catalogManager.registerTableProvider(tableProvider); + } queryPlannerClassName = CALCITE_PLANNER; schemaMap = new HashMap<>(); functionSet = new HashSet<>(); @@ -264,12 +267,7 @@ public BeamSqlEnvBuilder setUseCatalog(String name) { public BeamSqlEnv build() { checkStateNotNull(pipelineOptions); - JdbcConnection jdbcConnection; - if (defaultTableProvider != null) { - jdbcConnection = JdbcDriver.connect(defaultTableProvider, pipelineOptions); - } else { - jdbcConnection = JdbcDriver.connect(catalogManager, pipelineOptions); - } + JdbcConnection jdbcConnection = JdbcDriver.connect(catalogManager, pipelineOptions); configureSchemas(jdbcConnection); diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogManagerSchema.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogManagerSchema.java new file mode 100644 index 000000000000..4b0d2025ce7d --- /dev/null +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogManagerSchema.java @@ -0,0 +1,237 @@ +/* + * 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. + */ +package org.apache.beam.sdk.extensions.sql.impl; + +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; +import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Static.RESOURCE; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.beam.sdk.extensions.sql.impl.parser.SqlDdlNodes; +import org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog; +import org.apache.beam.sdk.extensions.sql.meta.catalog.CatalogManager; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.tree.Expression; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelProtoDataType; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Function; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Schema; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.SchemaPlus; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.SchemaVersion; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Schemas; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Table; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlIdentifier; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlUtil; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class CatalogManagerSchema implements Schema { + private static final Logger LOG = LoggerFactory.getLogger(CatalogManagerSchema.class); + private final JdbcConnection connection; + private final CatalogManager catalogManager; + private final Map catalogSubSchemas = new HashMap<>(); + /** + * Creates a Calcite {@link Schema} representing a {@link CatalogManager}. This will typically be + * the root node of a pipeline. + */ + CatalogManagerSchema(JdbcConnection jdbcConnection, CatalogManager catalogManager) { + this.connection = jdbcConnection; + this.catalogManager = catalogManager; + } + + @VisibleForTesting + public JdbcConnection connection() { + return connection; + } + + public void createCatalog( + SqlIdentifier catalogIdentifier, + String type, + Map properties, + boolean replace, + boolean ifNotExists) { + String name = SqlDdlNodes.name(catalogIdentifier); + if (catalogManager.getCatalog(name) != null) { + if (replace) { + LOG.info("Replacing existing catalog '{}'", name); + catalogManager.dropCatalog(name); + } else if (!ifNotExists) { + throw SqlUtil.newContextException( + catalogIdentifier.getParserPosition(), + RESOURCE.internal(String.format("Catalog '%s' already exists.", name))); + } else { + LOG.info("Catalog '{}' already exists", name); + return; + } + } + + // create the catalog + catalogManager.createCatalog(name, type, properties); + CatalogSchema catalogSchema = + new CatalogSchema(connection, checkStateNotNull(catalogManager.getCatalog(name))); + catalogSubSchemas.put(name, catalogSchema); + } + + public void useCatalog(SqlIdentifier catalogIdentifier) { + String name = catalogIdentifier.toString(); + if (catalogManager.getCatalog(catalogIdentifier.toString()) == null) { + throw SqlUtil.newContextException( + catalogIdentifier.getParserPosition(), + RESOURCE.internal(String.format("Cannot use catalog: '%s' not found.", name))); + } + + if (catalogManager.currentCatalog().name().equals(name)) { + LOG.info("Catalog '{}' is already in use.", name); + return; + } + + catalogManager.useCatalog(name); + LOG.info("Switched to catalog '{}' (type: {})", name, catalogManager.currentCatalog().type()); + } + + public void dropCatalog(SqlIdentifier identifier, boolean ifExists) { + String name = SqlDdlNodes.name(identifier); + if (catalogManager.getCatalog(name) == null) { + if (!ifExists) { + throw SqlUtil.newContextException( + identifier.getParserPosition(), + RESOURCE.internal(String.format("Cannot drop catalog: '%s' not found.", name))); + } + LOG.info("Ignoring 'DROP CATALOG` call for non-existent catalog: {}", name); + return; + } + + if (catalogManager.currentCatalog().name().equals(name)) { + throw SqlUtil.newContextException( + identifier.getParserPosition(), + RESOURCE.internal( + String.format( + "Unable to drop active catalog '%s'. Please switch to another catalog first.", + name))); + } + + catalogManager.dropCatalog(name); + LOG.info("Successfully dropped catalog '{}'", name); + catalogSubSchemas.remove(name); + } + + @Override + public @Nullable Table getTable(String table) { + @Nullable + CatalogSchema catalogSchema = catalogSubSchemas.get(catalogManager.currentCatalog().name()); + return catalogSchema != null ? catalogSchema.getTable(table) : null; + } + + @Override + public Set getTableNames() { + ImmutableSet.Builder names = ImmutableSet.builder(); + // TODO: this might be a heavy operation + for (Catalog catalog : catalogManager.catalogs()) { + for (String db : catalog.listDatabases()) { + names.addAll(catalog.metaStore(db).getTables().keySet()); + } + } + return names.build(); + } + + public CatalogSchema getCatalogSchema(TableName tablePath) { + @Nullable + Schema catalogSchema = tablePath.catalog() != null ? getSubSchema(tablePath.catalog()) : null; + if (catalogSchema == null) { + catalogSchema = getCurrentCatalogSchema(); + } + Preconditions.checkState( + catalogSchema instanceof CatalogSchema, + "Unexpected Schema type for Catalog '%s': %s", + tablePath.catalog(), + catalogSchema.getClass()); + return (CatalogSchema) catalogSchema; + } + + public CatalogSchema getCurrentCatalogSchema() { + return (CatalogSchema) + checkStateNotNull( + getSubSchema(catalogManager.currentCatalog().name()), + "Could not find Calcite Schema for active catalog '%s'.", + catalogManager.currentCatalog().name()); + } + + @Override + public @Nullable Schema getSubSchema(String name) { + @Nullable CatalogSchema catalogSchema = catalogSubSchemas.get(name); + if (catalogSchema == null) { + @Nullable Catalog catalog = catalogManager.getCatalog(name); + if (catalog != null) { + catalogSchema = new CatalogSchema(connection, catalog); + catalogSubSchemas.put(name, catalogSchema); + } + } + if (catalogSchema != null) { + return catalogSchema; + } + // name could be referring to an underlying metastore. + // Attempt to fetch from current catalog + return getCurrentCatalogSchema().getSubSchema(name); + } + + @Override + public Set getSubSchemaNames() { + return catalogManager.catalogs().stream().map(Catalog::name).collect(Collectors.toSet()); + } + + @Override + public Set getTypeNames() { + return Collections.emptySet(); + } + + @Override + public @Nullable RelProtoDataType getType(String s) { + return null; + } + + @Override + public Collection getFunctions(String s) { + return Collections.emptySet(); + } + + @Override + public Set getFunctionNames() { + return Collections.emptySet(); + } + + @Override + public Expression getExpression(@Nullable SchemaPlus schemaPlus, String s) { + return Schemas.subSchemaExpression(checkStateNotNull(schemaPlus), s, getClass()); + } + + @Override + public boolean isMutable() { + return true; + } + + @Override + public Schema snapshot(SchemaVersion schemaVersion) { + return this; + } +} diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogSchema.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogSchema.java new file mode 100644 index 000000000000..06fbf401401f --- /dev/null +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogSchema.java @@ -0,0 +1,247 @@ +/* + * 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. + */ +package org.apache.beam.sdk.extensions.sql.impl; + +import static java.lang.String.format; +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; +import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Static.RESOURCE; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import org.apache.beam.sdk.extensions.sql.impl.parser.SqlDdlNodes; +import org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog; +import org.apache.beam.sdk.extensions.sql.meta.catalog.CatalogManager; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.tree.Expression; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelProtoDataType; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Function; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Schema; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.SchemaPlus; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.SchemaVersion; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Schemas; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Table; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlIdentifier; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlUtil; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +// TODO: CONSOLIDATE THIS CLASS WITH BeamCalciteSchema +public class CatalogSchema implements Schema { + private static final Logger LOG = LoggerFactory.getLogger(CatalogSchema.class); + private final JdbcConnection connection; + private final Catalog catalog; + private final Map subSchemas = new HashMap<>(); + /** + * Creates a Calcite {@link Schema} representing a {@link CatalogManager}. This will typically be + * the root node of a pipeline. + */ + CatalogSchema(JdbcConnection jdbcConnection, Catalog catalog) { + this.connection = jdbcConnection; + this.catalog = catalog; + // try to eagerly populate Calcite sub-schemas with existing databases + try { + catalog + .listDatabases() + .forEach( + database -> + subSchemas.put( + database, + new BeamCalciteSchema(database, connection, catalog.metaStore(database)))); + } catch (Exception ignored) { + } + } + + public Catalog getCatalog() { + return catalog; + } + + public @Nullable BeamCalciteSchema getCurrentDatabaseSchema() { + return getSubSchema(catalog.currentDatabase()); + } + + public BeamCalciteSchema getDatabaseSchema(TableName tablePath) { + @Nullable BeamCalciteSchema beamCalciteSchema = getSubSchema(tablePath.database()); + if (beamCalciteSchema == null) { + beamCalciteSchema = getCurrentDatabaseSchema(); + } + return checkStateNotNull( + beamCalciteSchema, "Could not find BeamCalciteSchema for table: '%s'", tablePath); + } + + public void createDatabase(SqlIdentifier databaseIdentifier, boolean ifNotExists) { + String name = SqlDdlNodes.name(databaseIdentifier); + boolean alreadyExists = subSchemas.containsKey(name); + + if (!alreadyExists) { + try { + LOG.info("Creating database '{}'", name); + if (catalog.createDatabase(name)) { + LOG.info("Successfully created database '{}'", name); + } else { + alreadyExists = true; + } + } catch (Exception e) { + throw SqlUtil.newContextException( + databaseIdentifier.getParserPosition(), + RESOURCE.internal( + format("Encountered an error when creating database '%s': %s", name, e))); + } + } + + if (alreadyExists) { + String message = format("Database '%s' already exists.", name); + if (ifNotExists) { + LOG.info(message); + } else { + throw SqlUtil.newContextException( + databaseIdentifier.getParserPosition(), RESOURCE.internal(message)); + } + } + + subSchemas.put(name, new BeamCalciteSchema(name, connection, catalog.metaStore(name))); + } + + public void useDatabase(SqlIdentifier identifier) { + String name = SqlDdlNodes.name(identifier); + if (!subSchemas.containsKey(name)) { + if (!catalog.listDatabases().contains(name)) { + throw SqlUtil.newContextException( + identifier.getParserPosition(), + RESOURCE.internal(String.format("Cannot use database: '%s' not found.", name))); + } + subSchemas.put(name, new BeamCalciteSchema(name, connection, catalog.metaStore(name))); + } + + if (name.equals(catalog.currentDatabase())) { + LOG.info("Database '{}' is already in use.", name); + return; + } + + catalog.useDatabase(name); + LOG.info("Switched to database '{}'.", name); + } + + public void dropDatabase(SqlIdentifier identifier, boolean cascade, boolean ifExists) { + String name = SqlDdlNodes.name(identifier); + try { + LOG.info("Dropping database '{}'", name); + boolean dropped = catalog.dropDatabase(name, cascade); + + if (dropped) { + LOG.info("Successfully dropped database '{}'", name); + } else if (ifExists) { + LOG.info("Database '{}' does not exist.", name); + } else { + throw SqlUtil.newContextException( + identifier.getParserPosition(), + RESOURCE.internal(String.format("Database '%s' does not exist.", name))); + } + } catch (Exception e) { + throw SqlUtil.newContextException( + identifier.getParserPosition(), + RESOURCE.internal( + format("Encountered an error when dropping database '%s': %s", name, e))); + } + + subSchemas.remove(name); + } + + @Override + public @Nullable Table getTable(String s) { + @Nullable BeamCalciteSchema beamCalciteSchema = currentDatabase(); + return beamCalciteSchema != null ? beamCalciteSchema.getTable(s) : null; + } + + @Override + public Set getTableNames() { + @Nullable BeamCalciteSchema beamCalciteSchema = currentDatabase(); + return beamCalciteSchema != null ? beamCalciteSchema.getTableNames() : Collections.emptySet(); + } + + @Override + public @Nullable BeamCalciteSchema getSubSchema(@Nullable String name) { + if (name == null) { + return null; + } + @Nullable BeamCalciteSchema beamCalciteSchema = subSchemas.get(name); + if (beamCalciteSchema == null) { + Set databases; + try { + databases = catalog.listDatabases(); + } catch (Exception ignored) { + return null; + } + if (databases.contains(name)) { + beamCalciteSchema = new BeamCalciteSchema(name, connection, catalog.metaStore(name)); + subSchemas.put(name, beamCalciteSchema); + } + } + return beamCalciteSchema; + } + + private @Nullable BeamCalciteSchema currentDatabase() { + @Nullable String currentDatabase = catalog.currentDatabase(); + if (currentDatabase != null) { + return subSchemas.get(currentDatabase); + } + return null; + } + + @Override + public Set getSubSchemaNames() { + return catalog.listDatabases(); + } + + @Override + public Set getTypeNames() { + return Collections.emptySet(); + } + + @Override + public @Nullable RelProtoDataType getType(String s) { + return null; + } + + @Override + public Collection getFunctions(String s) { + return Collections.emptySet(); + } + + @Override + public Set getFunctionNames() { + return Collections.emptySet(); + } + + @Override + public Expression getExpression(@Nullable SchemaPlus schemaPlus, String s) { + return Schemas.subSchemaExpression(checkStateNotNull(schemaPlus), s, getClass()); + } + + @Override + public boolean isMutable() { + return true; + } + + @Override + public Schema snapshot(SchemaVersion schemaVersion) { + return this; + } +} diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcConnection.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcConnection.java index 972674df9f91..e3e756a5058d 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcConnection.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcConnection.java @@ -136,13 +136,13 @@ public SchemaPlus getCurrentSchemaPlus() { *

Overrides the schema if it exists. */ void setSchema(String name, TableProvider tableProvider) { - BeamCalciteSchema beamCalciteSchema = new BeamCalciteSchema(this, tableProvider); + BeamCalciteSchema beamCalciteSchema = new BeamCalciteSchema(name, this, tableProvider); getRootSchema().add(name, beamCalciteSchema); } /** Like {@link #setSchema(String, TableProvider)} but using a {@link CatalogManager}. */ void setSchema(String name, CatalogManager catalogManager) { - BeamCalciteSchema beamCalciteSchema = new BeamCalciteSchema(this, catalogManager); - getRootSchema().add(name, beamCalciteSchema); + CatalogManagerSchema catalogManagerSchema = new CatalogManagerSchema(this, catalogManager); + getRootSchema().add(name, catalogManagerSchema); } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/TableName.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/TableName.java index f69918e2c58c..06ca2c7f6694 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/TableName.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/TableName.java @@ -22,9 +22,15 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; +import com.google.api.client.util.Lists; import com.google.auto.value.AutoValue; import java.util.Collections; import java.util.List; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.checkerframework.dataflow.qual.Pure; /* * Licensed to the Apache Software Foundation (ASF) under one @@ -60,6 +66,12 @@ public abstract class TableName { /** Table name, the last element of the fully-specified table name with path. */ public abstract String getTableName(); + /** Splits the input String by "." separator and returns a new {@link TableName}. */ + public static TableName create(String path) { + List components = Lists.newArrayList(Splitter.on(".").split(path)); + return create(components); + } + /** Full table name with path. */ public static TableName create(List fullPath) { checkNotNull(fullPath, "Full table path cannot be null"); @@ -97,4 +109,22 @@ public TableName removePrefix() { List pathPostfix = getPath().stream().skip(1).collect(toList()); return TableName.create(pathPostfix, getTableName()); } + + /** Returns the database name in this table path. */ + @Pure + public @Nullable String database() { + return isCompound() ? Iterables.getLast(getPath()) : null; + } + + @Pure + public @Nullable String catalog() { + return getPath().size() > 1 ? getPath().get(0) : null; + } + + @Override + public final String toString() { + List components = + ImmutableList.builder().addAll(getPath()).add(getTableName()).build(); + return String.join(".", components); + } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateCatalog.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateCatalog.java index c1d96eea7bae..dd5a69035e32 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateCatalog.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateCatalog.java @@ -26,8 +26,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema; -import org.apache.beam.sdk.extensions.sql.meta.catalog.CatalogManager; +import org.apache.beam.sdk.extensions.sql.impl.CatalogManagerSchema; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalcitePrepare; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteSchema; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Schema; @@ -43,12 +42,8 @@ import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParserPos; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Pair; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class SqlCreateCatalog extends SqlCreate implements BeamSqlParser.ExecutableStatement { - private static final Logger LOG = LoggerFactory.getLogger(SqlCreateCatalog.class); private final SqlIdentifier catalogName; private final SqlNode type; private final SqlNodeList properties; @@ -118,42 +113,20 @@ public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { public void execute(CalcitePrepare.Context context) { final Pair pair = SqlDdlNodes.schema(context, true, catalogName); Schema schema = pair.left.schema; - String name = pair.right; String typeStr = checkArgumentNotNull(SqlDdlNodes.getString(type)); - if (!(schema instanceof BeamCalciteSchema)) { - throw SqlUtil.newContextException( - catalogName.getParserPosition(), - RESOURCE.internal("Schema is not of instance BeamCalciteSchema")); - } - - @Nullable CatalogManager catalogManager = ((BeamCalciteSchema) schema).getCatalogManager(); - if (catalogManager == null) { + if (!(schema instanceof CatalogManagerSchema)) { throw SqlUtil.newContextException( catalogName.getParserPosition(), RESOURCE.internal( - String.format( - "Unexpected 'CREATE CATALOG' call for Schema '%s' that is not a Catalog.", - name))); - } - - // check if catalog already exists - if (catalogManager.getCatalog(name) != null) { - if (getReplace()) { - LOG.info("Replacing existing catalog '{}'", name); - catalogManager.dropCatalog(name); - } else if (!ifNotExists) { - throw SqlUtil.newContextException( - catalogName.getParserPosition(), - RESOURCE.internal(String.format("Catalog '%s' already exists.", name))); - } else { - return; - } + "Attempting to create catalog '" + + SqlDdlNodes.name(catalogName) + + "' with unexpected Calcite Schema of type " + + schema.getClass())); } - // create the catalog - catalogManager.createCatalog(name, typeStr, parseProperties()); - LOG.info("Catalog '{}' (type: {}) successfully created", name, typeStr); + ((CatalogManagerSchema) schema) + .createCatalog(catalogName, typeStr, parseProperties(), getReplace(), ifNotExists); } private Map parseProperties() { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateDatabase.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateDatabase.java index 9938ad0e699c..f0d837f6b82c 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateDatabase.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateDatabase.java @@ -17,13 +17,14 @@ */ package org.apache.beam.sdk.extensions.sql.impl.parser; -import static java.lang.String.format; +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Static.RESOURCE; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; +import com.google.api.client.util.Lists; import java.util.List; -import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema; -import org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog; -import org.apache.beam.sdk.extensions.sql.meta.catalog.CatalogManager; +import org.apache.beam.sdk.extensions.sql.impl.CatalogManagerSchema; +import org.apache.beam.sdk.extensions.sql.impl.CatalogSchema; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalcitePrepare; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteSchema; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Schema; @@ -37,21 +38,19 @@ import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlWriter; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParserPos; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Pair; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.checkerframework.checker.nullness.qual.Nullable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class SqlCreateDatabase extends SqlCreate implements BeamSqlParser.ExecutableStatement { - private static final Logger LOG = LoggerFactory.getLogger(SqlCreateDatabase.class); private final SqlIdentifier databaseName; private static final SqlOperator OPERATOR = new SqlSpecialOperator("CREATE DATABASE", SqlKind.OTHER_DDL); public SqlCreateDatabase( - SqlParserPos pos, boolean replace, boolean ifNotExists, SqlNode databaseName) { + SqlParserPos pos, boolean replace, boolean ifNotExists, SqlIdentifier databaseName) { super(OPERATOR, pos, replace, ifNotExists); - this.databaseName = SqlDdlNodes.getIdentifier(databaseName, pos); + this.databaseName = databaseName; } @Override @@ -78,44 +77,45 @@ public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { public void execute(CalcitePrepare.Context context) { final Pair pair = SqlDdlNodes.schema(context, true, databaseName); Schema schema = pair.left.schema; - String name = pair.right; - if (!(schema instanceof BeamCalciteSchema)) { - throw SqlUtil.newContextException( - databaseName.getParserPosition(), - RESOURCE.internal("Schema is not of instance BeamCalciteSchema")); - } + List components = Lists.newArrayList(Splitter.on('.').split(databaseName.toString())); + @Nullable + String catalogName = components.size() > 1 ? components.get(components.size() - 2) : null; - @Nullable CatalogManager catalogManager = ((BeamCalciteSchema) schema).getCatalogManager(); - if (catalogManager == null) { - throw SqlUtil.newContextException( - databaseName.getParserPosition(), - RESOURCE.internal( - format( - "Unexpected 'CREATE DATABASE' call using Schema '%s' that is not a Catalog.", - name))); - } - - // Attempt to create the database. - Catalog catalog = catalogManager.currentCatalog(); - try { - LOG.info("Creating database '{}'", name); - boolean created = catalog.createDatabase(name); - - if (created) { - LOG.info("Successfully created database '{}'", name); - } else if (ifNotExists) { - LOG.info("Database '{}' already exists.", name); + @Nullable CatalogSchema catalogSchema; + if (schema instanceof CatalogManagerSchema) { + CatalogManagerSchema catalogManagerSchema = (CatalogManagerSchema) schema; + // override with catalog name if present. + if (catalogName != null) { + Schema s = + checkStateNotNull( + catalogManagerSchema.getSubSchema(catalogName), + "Could not find Calcite Schema for catalog '%s'.", + catalogName); + checkState( + s instanceof CatalogSchema, + "Catalog '%s' had unexpected Calcite Schema of type %s. Expected type: %s.", + catalogName, + s.getClass(), + CatalogSchema.class.getSimpleName()); + catalogSchema = (CatalogSchema) s; } else { - throw SqlUtil.newContextException( - databaseName.getParserPosition(), - RESOURCE.internal(format("Database '%s' already exists.", name))); + catalogSchema = catalogManagerSchema.getCurrentCatalogSchema(); } - } catch (Exception e) { + } + // else if (schema instanceof CatalogSchema) { + // catalogSchema = (CatalogSchema) schema; + // } + else { throw SqlUtil.newContextException( databaseName.getParserPosition(), RESOURCE.internal( - format("Encountered an error when creating database '%s': %s", name, e))); + "Attempting to create database '" + + databaseName + + "' with unexpected Calcite Schema of type " + + schema.getClass())); } + + catalogSchema.createDatabase(databaseName, ifNotExists); } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateExternalTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateExternalTable.java index 2d98c03574ff..9584337eaa73 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateExternalTable.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateExternalTable.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.extensions.sql.impl.parser; +import static org.apache.beam.sdk.extensions.sql.impl.parser.SqlDdlNodes.name; +import static org.apache.beam.sdk.extensions.sql.impl.parser.SqlDdlNodes.schema; import static org.apache.beam.sdk.schemas.Schema.toSchema; import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Static.RESOURCE; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; @@ -26,11 +28,15 @@ import java.util.stream.Collectors; import org.apache.beam.sdk.extensions.sql.TableUtils; import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema; +import org.apache.beam.sdk.extensions.sql.impl.CatalogManagerSchema; +import org.apache.beam.sdk.extensions.sql.impl.CatalogSchema; +import org.apache.beam.sdk.extensions.sql.impl.TableName; import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils; import org.apache.beam.sdk.extensions.sql.meta.Table; -import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.Schema.Field; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalcitePrepare; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteSchema; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Schema; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlCreate; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlIdentifier; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind; @@ -50,7 +56,7 @@ }) public class SqlCreateExternalTable extends SqlCreate implements BeamSqlParser.ExecutableStatement { private final SqlIdentifier name; - private final List columnList; + private final List columnList; private final SqlNode type; private final SqlNode comment; private final SqlNode location; @@ -66,7 +72,7 @@ public SqlCreateExternalTable( boolean replace, boolean ifNotExists, SqlIdentifier name, - List columnList, + List columnList, SqlNode type, SqlNodeList partitionFields, SqlNode comment, @@ -144,28 +150,42 @@ public void execute(CalcitePrepare.Context context) { } return; } - // Table does not exist. Create it. - if (!(pair.left.schema instanceof BeamCalciteSchema)) { + + Schema schema = pair.left.schema; + + BeamCalciteSchema beamCalciteSchema; + // String catalogName = "default"; + if (schema instanceof CatalogManagerSchema) { + TableName pathOverride = TableName.create(name.toString()); + CatalogSchema catalogSchema = ((CatalogManagerSchema) schema).getCatalogSchema(pathOverride); + beamCalciteSchema = catalogSchema.getDatabaseSchema(pathOverride); + // catalogName = catalogSchema.getCatalog().name(); + } else if (schema instanceof BeamCalciteSchema) { + beamCalciteSchema = (BeamCalciteSchema) schema; + } else { throw SqlUtil.newContextException( name.getParserPosition(), - RESOURCE.internal("Schema is not instanceof BeamCalciteSchema")); + RESOURCE.internal( + "Attempting to create a table with unexpected Calcite Schema of type " + + schema.getClass())); } - - BeamCalciteSchema schema = (BeamCalciteSchema) pair.left.schema; + // String databaseName = beamCalciteSchema.name(); + // String tableName = name(name); Table table = toTable(); + if (partitionFields != null) { checkArgument( - schema.resolveMetastore().supportsPartitioning(table), + beamCalciteSchema.getTableProvider().supportsPartitioning(table), "Invalid use of 'PARTITIONED BY()': Table '%s' of type '%s' " + "does not support partitioning.", - SqlDdlNodes.name(name), + name(name), SqlDdlNodes.getString(type)); } - schema.resolveMetastore().createTable(table); + beamCalciteSchema.getTableProvider().createTable(table); } - private void unparseColumn(SqlWriter writer, Schema.Field column) { + private void unparseColumn(SqlWriter writer, Field column) { writer.sep(","); writer.identifier(column.getName(), false); writer.identifier(CalciteUtils.toSqlTypeName(column.getType()).name(), false); @@ -190,11 +210,12 @@ private void unparseColumn(SqlWriter writer, Schema.Field column) { private Table toTable() { return Table.builder() .type(SqlDdlNodes.getString(type)) - .name(SqlDdlNodes.name(name)) + .name(name(name)) .schema(columnList.stream().collect(toSchema())) .partitionFields(parsePartitionFields()) .comment(SqlDdlNodes.getString(comment)) .location(SqlDdlNodes.getString(location)) + // .path(path) .properties( (tblProperties == null) ? TableUtils.emptyProperties() diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDdlNodes.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDdlNodes.java index e0378d859e2a..340b7214d83b 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDdlNodes.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDdlNodes.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.extensions.sql.impl.parser; import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; import java.util.List; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalcitePrepare; @@ -50,23 +51,33 @@ public static SqlNode column( /** Returns the schema in which to create an object. */ static Pair schema( CalcitePrepare.Context context, boolean mutable, SqlIdentifier id) { - final List path; - if (id.isSimple()) { - path = context.getDefaultSchemaPath(); - } else { + CalciteSchema rootSchema = mutable ? context.getMutableRootSchema() : context.getRootSchema(); + @Nullable CalciteSchema schema = null; + List path = null; + if (!id.isSimple()) { path = Util.skipLast(id.names); + schema = childSchema(rootSchema, path); + } + // if id isSimple or if the above returned a null schema, use default schema path + if (schema == null) { + path = context.getDefaultSchemaPath(); + schema = childSchema(rootSchema, path); } - CalciteSchema schema = mutable ? context.getMutableRootSchema() : context.getRootSchema(); + return Pair.of(checkStateNotNull(schema, "Got null sub-schema for path '%s'", path), name(id)); + } + + private static @Nullable CalciteSchema childSchema(CalciteSchema rootSchema, List path) { + @Nullable CalciteSchema schema = rootSchema; for (String p : path) { - schema = schema.getSubSchema(p, true); if (schema == null) { - throw new AssertionError(String.format("Got null sub-schema for path '%s' in %s", p, path)); + break; } + schema = schema.getSubSchema(p, true); } - return Pair.of(schema, name(id)); + return schema; } - static String name(SqlIdentifier id) { + public static String name(SqlIdentifier id) { if (id.isSimple()) { return id.getSimple(); } else { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropCatalog.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropCatalog.java index ac1dfe5c2a83..484d07096826 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropCatalog.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropCatalog.java @@ -20,8 +20,7 @@ import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Static.RESOURCE; import java.util.List; -import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema; -import org.apache.beam.sdk.extensions.sql.meta.catalog.CatalogManager; +import org.apache.beam.sdk.extensions.sql.impl.CatalogManagerSchema; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalcitePrepare; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteSchema; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Schema; @@ -36,12 +35,8 @@ import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParserPos; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Pair; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class SqlDropCatalog extends SqlDrop implements BeamSqlParser.ExecutableStatement { - private static final Logger LOG = LoggerFactory.getLogger(SqlDropCatalog.class); private static final SqlOperator OPERATOR = new SqlSpecialOperator("DROP CATALOG", SqlKind.OTHER_DDL); private final SqlIdentifier catalogName; @@ -64,45 +59,18 @@ public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { public void execute(CalcitePrepare.Context context) { final Pair pair = SqlDdlNodes.schema(context, true, catalogName); Schema schema = pair.left.schema; - String name = pair.right; - if (!(schema instanceof BeamCalciteSchema)) { - throw SqlUtil.newContextException( - catalogName.getParserPosition(), - RESOURCE.internal("Schema is not of instance BeamCalciteSchema")); - } - - BeamCalciteSchema beamCalciteSchema = (BeamCalciteSchema) schema; - @Nullable CatalogManager catalogManager = beamCalciteSchema.getCatalogManager(); - if (catalogManager == null) { - throw SqlUtil.newContextException( - catalogName.getParserPosition(), - RESOURCE.internal( - String.format( - "Unexpected 'DROP CATALOG' call for Schema '%s' that is not a Catalog.", name))); - } - - if (catalogManager.getCatalog(name) == null) { - if (!ifExists) { - throw SqlUtil.newContextException( - catalogName.getParserPosition(), - RESOURCE.internal(String.format("Cannot drop catalog: '%s' not found.", name))); - } - LOG.info("Ignoring 'DROP CATALOG` call for non-existent catalog: {}", name); - return; - } - - if (catalogManager.currentCatalog().name().equals(name)) { + if (!(schema instanceof CatalogManagerSchema)) { throw SqlUtil.newContextException( catalogName.getParserPosition(), RESOURCE.internal( - String.format( - "Unable to drop active catalog '%s'. Please switch to another catalog first.", - name))); + "Attempting to drop a catalog '" + + SqlDdlNodes.name(catalogName) + + "' with unexpected Calcite Schema of type " + + schema.getClass())); } - catalogManager.dropCatalog(name); - LOG.info("Successfully dropped catalog '{}'", name); + ((CatalogManagerSchema) schema).dropCatalog(catalogName, ifExists); } @Override diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropDatabase.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropDatabase.java index 639edc9ca15d..d71b9589057d 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropDatabase.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropDatabase.java @@ -17,13 +17,13 @@ */ package org.apache.beam.sdk.extensions.sql.impl.parser; -import static java.lang.String.format; import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Static.RESOURCE; +import com.google.api.client.util.Lists; import java.util.List; -import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema; -import org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog; -import org.apache.beam.sdk.extensions.sql.meta.catalog.CatalogManager; +import org.apache.beam.sdk.extensions.sql.impl.CatalogManagerSchema; +import org.apache.beam.sdk.extensions.sql.impl.CatalogSchema; +import org.apache.beam.sdk.extensions.sql.impl.TableName; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalcitePrepare; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteSchema; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Schema; @@ -37,22 +37,19 @@ import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlWriter; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParserPos; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Pair; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class SqlDropDatabase extends SqlDrop implements BeamSqlParser.ExecutableStatement { - private static final Logger LOG = LoggerFactory.getLogger(SqlDropDatabase.class); private static final SqlOperator OPERATOR = new SqlSpecialOperator("DROP DATABASE", SqlKind.OTHER_DDL); private final SqlIdentifier databaseName; private final boolean cascade; public SqlDropDatabase( - SqlParserPos pos, boolean ifExists, SqlNode databaseName, boolean cascade) { + SqlParserPos pos, boolean ifExists, SqlIdentifier databaseName, boolean cascade) { super(OPERATOR, pos, ifExists); - this.databaseName = SqlDdlNodes.getIdentifier(databaseName, pos); + this.databaseName = databaseName; this.cascade = cascade; } @@ -74,45 +71,21 @@ public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { public void execute(CalcitePrepare.Context context) { final Pair pair = SqlDdlNodes.schema(context, true, databaseName); Schema schema = pair.left.schema; - String name = pair.right; - if (!(schema instanceof BeamCalciteSchema)) { - throw SqlUtil.newContextException( - databaseName.getParserPosition(), - RESOURCE.internal("Schema is not of instance BeamCalciteSchema")); - } - - BeamCalciteSchema beamCalciteSchema = (BeamCalciteSchema) schema; - @Nullable CatalogManager catalogManager = beamCalciteSchema.getCatalogManager(); - if (catalogManager == null) { + if (!(schema instanceof CatalogManagerSchema)) { throw SqlUtil.newContextException( databaseName.getParserPosition(), RESOURCE.internal( - String.format( - "Unexpected 'DROP DATABASE' call using Schema '%s' that is not a Catalog.", - name))); + "Attempting to drop database '" + + databaseName + + "' with unexpected Calcite Schema of type " + + schema.getClass())); } - Catalog catalog = catalogManager.currentCatalog(); - try { - LOG.info("Dropping database '{}'", name); - boolean dropped = catalog.dropDatabase(name, cascade); - - if (dropped) { - LOG.info("Successfully dropped database '{}'", name); - } else if (ifExists) { - LOG.info("Database '{}' does not exist.", name); - } else { - throw SqlUtil.newContextException( - databaseName.getParserPosition(), - RESOURCE.internal(String.format("Database '%s' does not exist.", name))); - } - } catch (Exception e) { - throw SqlUtil.newContextException( - databaseName.getParserPosition(), - RESOURCE.internal( - format("Encountered an error when dropping database '%s': %s", name, e))); - } + List components = Lists.newArrayList(Splitter.on(".").split(databaseName.toString())); + TableName pathOverride = TableName.create(components, ""); + CatalogSchema catalogSchema = ((CatalogManagerSchema) schema).getCatalogSchema(pathOverride); + catalogSchema.dropDatabase(databaseName, cascade, ifExists); } @Override diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropTable.java index 5a62b0ee931e..92e62e4f67b6 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropTable.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropTable.java @@ -17,11 +17,23 @@ */ package org.apache.beam.sdk.extensions.sql.impl.parser; +import static org.apache.beam.sdk.extensions.sql.impl.parser.SqlDdlNodes.name; +import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Static.RESOURCE; + +import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema; +import org.apache.beam.sdk.extensions.sql.impl.CatalogManagerSchema; +import org.apache.beam.sdk.extensions.sql.impl.CatalogSchema; +import org.apache.beam.sdk.extensions.sql.impl.TableName; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalcitePrepare; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteSchema; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Schema; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlIdentifier; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlOperator; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlSpecialOperator; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlUtil; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Pair; /** Parse tree for {@code DROP TABLE} statement. */ public class SqlDropTable extends SqlDropObject { @@ -32,6 +44,39 @@ public class SqlDropTable extends SqlDropObject { SqlDropTable(SqlParserPos pos, boolean ifExists, SqlIdentifier name) { super(OPERATOR, pos, ifExists, name); } + + @Override + public void execute(CalcitePrepare.Context context) { + final Pair pair = SqlDdlNodes.schema(context, true, name); + TableName pathOverride = TableName.create(name.toString()); + Schema schema = pair.left.schema; + + BeamCalciteSchema beamCalciteSchema; + if (schema instanceof CatalogManagerSchema) { + CatalogSchema catalogSchema = ((CatalogManagerSchema) schema).getCatalogSchema(pathOverride); + beamCalciteSchema = catalogSchema.getDatabaseSchema(pathOverride); + } else if (schema instanceof BeamCalciteSchema) { + beamCalciteSchema = (BeamCalciteSchema) schema; + } else { + throw SqlUtil.newContextException( + name.getParserPosition(), + RESOURCE.internal( + "Attempting to drop a table using unexpected Calcite Schema of type " + + schema.getClass())); + } + + if (beamCalciteSchema.getTable(pair.right) == null) { + // Table does not exist. + if (!ifExists) { + // They did not specify IF EXISTS, so give error. + throw SqlUtil.newContextException( + name.getParserPosition(), RESOURCE.tableNotFound(name.toString())); + } + return; + } + + beamCalciteSchema.getTableProvider().dropTable(pair.right); + } } // End SqlDropTable.java diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlUseCatalog.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlUseCatalog.java index 7088c7183027..f0a637e05488 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlUseCatalog.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlUseCatalog.java @@ -21,8 +21,7 @@ import java.util.Collections; import java.util.List; -import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema; -import org.apache.beam.sdk.extensions.sql.meta.catalog.CatalogManager; +import org.apache.beam.sdk.extensions.sql.impl.CatalogManagerSchema; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalcitePrepare; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteSchema; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Schema; @@ -35,12 +34,8 @@ import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlUtil; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParserPos; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Pair; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class SqlUseCatalog extends SqlSetOption implements BeamSqlParser.ExecutableStatement { - private static final Logger LOG = LoggerFactory.getLogger(SqlUseCatalog.class); private final SqlIdentifier catalogName; private static final SqlOperator OPERATOR = new SqlSpecialOperator("USE CATALOG", SqlKind.OTHER); @@ -64,36 +59,17 @@ public List getOperandList() { public void execute(CalcitePrepare.Context context) { final Pair pair = SqlDdlNodes.schema(context, true, catalogName); Schema schema = pair.left.schema; - String name = pair.right; - if (!(schema instanceof BeamCalciteSchema)) { - throw SqlUtil.newContextException( - catalogName.getParserPosition(), - RESOURCE.internal("Schema is not of instance BeamCalciteSchema")); - } - - BeamCalciteSchema beamCalciteSchema = (BeamCalciteSchema) schema; - @Nullable CatalogManager catalogManager = beamCalciteSchema.getCatalogManager(); - if (catalogManager == null) { + if (!(schema instanceof CatalogManagerSchema)) { throw SqlUtil.newContextException( catalogName.getParserPosition(), RESOURCE.internal( - String.format( - "Unexpected 'USE CATALOG' call for Schema '%s' that is not a Catalog.", name))); - } - - if (catalogManager.getCatalog(name) == null) { - throw SqlUtil.newContextException( - catalogName.getParserPosition(), - RESOURCE.internal(String.format("Cannot use catalog: '%s' not found.", name))); - } - - if (catalogManager.currentCatalog().name().equals(name)) { - LOG.info("Catalog '{}' is already in use.", name); - return; + "Attempting to 'USE CATALOG' " + + catalogName + + "' with unexpected Calcite Schema of type " + + schema.getClass())); } - catalogManager.useCatalog(name); - LOG.info("Switched to catalog '{}' (type: {})", name, catalogManager.currentCatalog().type()); + ((CatalogManagerSchema) schema).useCatalog(catalogName); } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlUseDatabase.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlUseDatabase.java index 40523e50a63f..2859cbf3ec63 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlUseDatabase.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlUseDatabase.java @@ -17,14 +17,14 @@ */ package org.apache.beam.sdk.extensions.sql.impl.parser; -import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Static.RESOURCE; +import com.google.api.client.util.Lists; import java.util.Collections; import java.util.List; -import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema; -import org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog; -import org.apache.beam.sdk.extensions.sql.meta.catalog.CatalogManager; +import org.apache.beam.sdk.extensions.sql.impl.CatalogManagerSchema; +import org.apache.beam.sdk.extensions.sql.impl.CatalogSchema; +import org.apache.beam.sdk.extensions.sql.impl.TableName; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalcitePrepare; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteSchema; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Schema; @@ -37,19 +37,16 @@ import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlUtil; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParserPos; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Pair; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; public class SqlUseDatabase extends SqlSetOption implements BeamSqlParser.ExecutableStatement { - private static final Logger LOG = LoggerFactory.getLogger(SqlUseDatabase.class); private final SqlIdentifier databaseName; private static final SqlOperator OPERATOR = new SqlSpecialOperator("USE DATABASE", SqlKind.OTHER); - public SqlUseDatabase(SqlParserPos pos, String scope, SqlNode databaseName) { + public SqlUseDatabase(SqlParserPos pos, String scope, SqlIdentifier databaseName) { super(pos, scope, SqlDdlNodes.getIdentifier(databaseName, pos), null); - this.databaseName = SqlDdlNodes.getIdentifier(databaseName, pos); + this.databaseName = databaseName; } @Override @@ -66,38 +63,32 @@ public List getOperandList() { public void execute(CalcitePrepare.Context context) { final Pair pair = SqlDdlNodes.schema(context, true, databaseName); Schema schema = pair.left.schema; - String name = checkStateNotNull(pair.right); + String path = databaseName.toString(); + List components = Lists.newArrayList(Splitter.on(".").split(path)); + TableName pathOverride = TableName.create(components, ""); - if (!(schema instanceof BeamCalciteSchema)) { - throw SqlUtil.newContextException( - databaseName.getParserPosition(), - RESOURCE.internal("Schema is not of instance BeamCalciteSchema")); - } - - BeamCalciteSchema beamCalciteSchema = (BeamCalciteSchema) schema; - @Nullable CatalogManager catalogManager = beamCalciteSchema.getCatalogManager(); - if (catalogManager == null) { + if (!(schema instanceof CatalogManagerSchema)) { throw SqlUtil.newContextException( databaseName.getParserPosition(), RESOURCE.internal( - String.format( - "Unexpected 'USE DATABASE' call using Schema '%s' that is not a Catalog.", - name))); - } - - Catalog catalog = catalogManager.currentCatalog(); - if (!catalog.listDatabases().contains(name)) { - throw SqlUtil.newContextException( - databaseName.getParserPosition(), - RESOURCE.internal(String.format("Cannot use database: '%s' not found.", name))); + "Attempting to create database '" + + path + + "' with unexpected Calcite Schema of type " + + schema.getClass())); } - if (name.equals(catalog.currentDatabase())) { - LOG.info("Database '{}' is already in use.", name); - return; + CatalogManagerSchema catalogManagerSchema = (CatalogManagerSchema) schema; + CatalogSchema catalogSchema = catalogManagerSchema.getCatalogSchema(pathOverride); + // if database exists in a different catalog, we need to also switch to that catalog + if (pathOverride.catalog() != null + && !pathOverride + .catalog() + .equals(catalogManagerSchema.getCurrentCatalogSchema().getCatalog().name())) { + SqlIdentifier catalogIdentifier = + new SqlIdentifier(pathOverride.catalog(), databaseName.getParserPosition()); + catalogManagerSchema.useCatalog(catalogIdentifier); } - catalog.useDatabase(name); - LOG.info("Switched to database '{}'.", name); + catalogSchema.useDatabase(databaseName); } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/Table.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/Table.java index 3b72baa9b38e..5c03a2b20b25 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/Table.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/Table.java @@ -24,6 +24,7 @@ import org.apache.beam.sdk.extensions.sql.TableUtils; import org.apache.beam.sdk.schemas.Schema; import org.checkerframework.checker.nullness.qual.Nullable; +import org.checkerframework.dataflow.qual.Pure; /** Represents the metadata of a {@code BeamSqlTable}. */ @AutoValue @@ -39,7 +40,7 @@ public abstract class Table implements Serializable { public abstract @Nullable String getComment(); - public abstract @Nullable String getLocation(); + public abstract @Pure @Nullable String getLocation(); public abstract ObjectNode getProperties(); diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/Catalog.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/Catalog.java index e347584654cd..825ba8dbd6f6 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/Catalog.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/Catalog.java @@ -20,6 +20,7 @@ import java.util.Map; import java.util.Set; import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; import org.apache.beam.sdk.extensions.sql.meta.store.MetaStore; import org.checkerframework.checker.nullness.qual.Nullable; @@ -36,8 +37,11 @@ public interface Catalog { /** A type that defines this catalog. */ String type(); - /** The underlying {@link MetaStore} that actually manages tables. */ - MetaStore metaStore(); + /** + * Returns the underlying {@link MetaStore} for this database. Creates a new {@link MetaStore} if + * one does not exist yet. + */ + MetaStore metaStore(String database); /** * Produces the currently active database. Can be null if no database is active. @@ -84,4 +88,10 @@ public interface Catalog { /** User-specified configuration properties. */ Map properties(); + + /** Registers this {@link TableProvider} and propagates it to underlying {@link MetaStore}s. */ + void registerTableProvider(TableProvider provider); + + /** Clears registered providers from all underlying {@link MetaStore}s. */ + void clearTableProviders(); } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/CatalogManager.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/CatalogManager.java index 4654f0dd1b0d..c2fe7188aad1 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/CatalogManager.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/CatalogManager.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.extensions.sql.meta.catalog; +import java.util.Collection; import java.util.Map; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema; @@ -55,9 +56,10 @@ public interface CatalogManager { * Registers a {@link TableProvider} and propagates it to all the {@link Catalog} instances * available to this manager. */ - void registerTableProvider(String name, TableProvider tableProvider); + void registerTableProvider(TableProvider tableProvider); - default void registerTableProvider(TableProvider tp) { - registerTableProvider(tp.getTableType(), tp); - } + /** Clears registered providers from all underlying {@link Catalog}s. */ + void clearTableProviders(); + + Collection catalogs(); } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/EmptyCatalogManager.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/EmptyCatalogManager.java index 71bcd0b58af3..fb30691d8eaf 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/EmptyCatalogManager.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/EmptyCatalogManager.java @@ -17,9 +17,11 @@ */ package org.apache.beam.sdk.extensions.sql.meta.catalog; +import java.util.Collection; import java.util.Map; import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.checkerframework.checker.nullness.qual.Nullable; public class EmptyCatalogManager implements CatalogManager { @@ -49,14 +51,25 @@ public void dropCatalog(String name) { } @Override - public void registerTableProvider(String name, TableProvider tableProvider) { + public void registerTableProvider(TableProvider tableProvider) { throw new UnsupportedOperationException( "ReadOnlyCatalogManager does not support registering a table provider"); } + @Override + public void clearTableProviders() { + throw new UnsupportedOperationException( + "ReadOnlyCatalogManager does not support clearing table providers"); + } + @Override public void createCatalog(String name, String type, Map properties) { throw new UnsupportedOperationException( "ReadOnlyCatalogManager does not support catalog creation"); } + + @Override + public Collection catalogs() { + return ImmutableSet.of(EMPTY); + } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalog.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalog.java index 64d2fefe2f63..fbd5b386b6fb 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalog.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalog.java @@ -21,9 +21,11 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.Map; import java.util.Set; +import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; import org.apache.beam.sdk.extensions.sql.meta.store.InMemoryMetaStore; import org.apache.beam.sdk.extensions.sql.meta.store.MetaStore; import org.apache.beam.sdk.util.Preconditions; @@ -32,13 +34,19 @@ public class InMemoryCatalog implements Catalog { private final String name; private final Map properties; - private final InMemoryMetaStore metaStore = new InMemoryMetaStore(); + protected final Set tableProviders = new HashSet<>(); + private final Map metaStores = new HashMap<>(); private final HashSet databases = new HashSet<>(Collections.singleton(DEFAULT)); protected @Nullable String currentDatabase = DEFAULT; public InMemoryCatalog(String name, Map properties) { + this(name, new InMemoryMetaStore(), properties); + } + + public InMemoryCatalog(String name, MetaStore defaultMetastore, Map properties) { this.name = name; this.properties = properties; + metaStores.put(DEFAULT, defaultMetastore); } @Override @@ -53,7 +61,13 @@ public String name() { } @Override - public MetaStore metaStore() { + public MetaStore metaStore(String db) { + @Nullable MetaStore metaStore = metaStores.get(db); + if (metaStore == null) { + metaStore = new InMemoryMetaStore(); + tableProviders.forEach(metaStore::registerProvider); + metaStores.put(db, metaStore); + } return metaStore; } @@ -93,4 +107,16 @@ public boolean dropDatabase(String database, boolean cascade) { public Set listDatabases() { return databases; } + + @Override + public void registerTableProvider(TableProvider provider) { + tableProviders.add(provider); + metaStores.values().forEach(m -> m.registerProvider(provider)); + } + + @Override + public void clearTableProviders() { + tableProviders.clear(); + metaStores.values().forEach(MetaStore::clearProviders); + } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalogManager.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalogManager.java index 84deeb96436a..d45ee4a0384a 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalogManager.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalogManager.java @@ -19,19 +19,23 @@ import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.ServiceLoader; +import java.util.Set; import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; +import org.apache.beam.sdk.extensions.sql.meta.store.MetaStore; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.checkerframework.checker.nullness.qual.Nullable; public class InMemoryCatalogManager implements CatalogManager { private final Map catalogs = new HashMap<>(); - private final Map tableProviderMap = new HashMap<>(); + private final Set tableProviders = new HashSet<>(); private String currentCatalogName; public InMemoryCatalogManager() { @@ -39,13 +43,20 @@ public InMemoryCatalogManager() { this.currentCatalogName = "default"; } + /** To keep backwards compatibility, extends an option to set a default metastore. */ + public InMemoryCatalogManager(MetaStore defaultMetastore) { + this.catalogs.put( + "default", new InMemoryCatalog("default", defaultMetastore, Collections.emptyMap())); + this.currentCatalogName = "default"; + } + @Override public void createCatalog(String name, String type, Map properties) { Preconditions.checkState( !catalogs.containsKey(name), "Catalog with name '%s' already exists.", name); Catalog catalog = findAndCreateCatalog(name, type, properties); - tableProviderMap.values().forEach(catalog.metaStore()::registerProvider); + tableProviders.forEach(catalog::registerTableProvider); catalogs.put(name, catalog); } @@ -73,9 +84,15 @@ public void dropCatalog(String name) { } @Override - public void registerTableProvider(String name, TableProvider tableProvider) { - tableProviderMap.put(name, tableProvider); - catalogs.values().forEach(catalog -> catalog.metaStore().registerProvider(tableProvider)); + public void registerTableProvider(TableProvider tableProvider) { + catalogs.values().forEach(catalog -> catalog.registerTableProvider(tableProvider)); + tableProviders.add(tableProvider); + } + + @Override + public void clearTableProviders() { + catalogs.values().forEach(Catalog::clearTableProviders); + tableProviders.clear(); } private Catalog findAndCreateCatalog(String name, String type, Map properties) { @@ -115,4 +132,9 @@ private Catalog createCatalogInstance( String.format("Encountered an error when constructing Catalog '%s'", name), e); } } + + @Override + public Collection catalogs() { + return catalogs.values(); + } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergCatalog.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergCatalog.java index 1209d2b4663d..6c84339a6a41 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergCatalog.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergCatalog.java @@ -17,10 +17,12 @@ */ package org.apache.beam.sdk.extensions.sql.meta.provider.iceberg; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; + +import java.util.HashMap; import java.util.Map; import java.util.Set; import org.apache.beam.sdk.extensions.sql.meta.catalog.InMemoryCatalog; -import org.apache.beam.sdk.extensions.sql.meta.store.InMemoryMetaStore; import org.apache.beam.sdk.io.iceberg.IcebergCatalogConfig; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; @@ -29,7 +31,7 @@ public class IcebergCatalog extends InMemoryCatalog { // TODO(ahmedabu98): extend this to the IO implementation so // other SDKs can make use of it too private static final String BEAM_HADOOP_PREFIX = "beam.catalog.hadoop"; - private final InMemoryMetaStore metaStore = new InMemoryMetaStore(); + private final Map metaStores = new HashMap<>(); @VisibleForTesting final IcebergCatalogConfig catalogConfig; public IcebergCatalog(String name, Map properties) { @@ -52,12 +54,18 @@ public IcebergCatalog(String name, Map properties) { .setCatalogProperties(catalogProps.build()) .setConfigProperties(hadoopProps.build()) .build(); - metaStore.registerProvider(new IcebergTableProvider(catalogConfig)); } @Override - public InMemoryMetaStore metaStore() { - return metaStore; + public IcebergMetastore metaStore(String db) { + metaStores.putIfAbsent(db, new IcebergMetastore(db, catalogConfig)); + return metaStores.get(db); + // @Nullable IcebergMetastore metaStore = metaStores.get(db); + // if (metaStore == null) { + // metaStore = new IcebergMetastore(db, catalogConfig); + // metaStores.put(db, metaStore); + // } + // return metaStore; } @Override @@ -70,9 +78,16 @@ public boolean createDatabase(String database) { return catalogConfig.createNamespace(database); } + @Override + public void useDatabase(String database) { + checkArgument(listDatabases().contains(database), "Database '%s' does not exist."); + currentDatabase = database; + } + @Override public boolean dropDatabase(String database, boolean cascade) { boolean removed = catalogConfig.dropNamespace(database, cascade); + metaStores.remove(database); if (database.equals(currentDatabase)) { currentDatabase = null; } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastore.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastore.java new file mode 100644 index 000000000000..4450c6f7dfff --- /dev/null +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastore.java @@ -0,0 +1,149 @@ +/* + * 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. + */ +package org.apache.beam.sdk.extensions.sql.meta.provider.iceberg; + +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; + +import java.util.HashMap; +import java.util.Map; +import org.apache.beam.sdk.extensions.sql.TableUtils; +import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable; +import org.apache.beam.sdk.extensions.sql.meta.Table; +import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; +import org.apache.beam.sdk.extensions.sql.meta.store.MetaStore; +import org.apache.beam.sdk.io.iceberg.IcebergCatalogConfig; +import org.apache.beam.sdk.io.iceberg.IcebergCatalogConfig.IcebergTableInfo; +import org.apache.beam.sdk.io.iceberg.TableAlreadyExistsException; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class IcebergMetastore implements MetaStore { + private static final Logger LOG = LoggerFactory.getLogger(IcebergMetastore.class); + @VisibleForTesting final IcebergCatalogConfig catalogConfig; + private final Map cachedTables = new HashMap<>(); + private final String database; + + public IcebergMetastore(String db, IcebergCatalogConfig catalogConfig) { + this.database = db; + this.catalogConfig = catalogConfig; + } + + @Override + public String getTableType() { + return "iceberg"; + } + + @Override + public void createTable(Table table) { + String identifier = getIdentifier(table); + try { + catalogConfig.createTable(identifier, table.getSchema(), table.getPartitionFields()); + } catch (TableAlreadyExistsException e) { + LOG.info("Iceberg table '{}' already exists at location '{}'.", table.getName(), identifier); + } + cachedTables.put(table.getName(), table); + } + + @Override + public void dropTable(String tableName) { + String identifier = getIdentifier(tableName); + if (catalogConfig.dropTable(identifier)) { + LOG.info("Dropped table '{}' (path: '{}').", tableName, identifier); + } else { + LOG.info( + "Ignoring DROP TABLE call for '{}' (path: '{}') because it does not exist.", + tableName, + identifier); + } + cachedTables.remove(tableName); + } + + @Override + public Map getTables() { + for (String id : catalogConfig.listTables(database)) { + String name = Iterables.getLast(Splitter.on(".").split(id)); + if (!cachedTables.containsKey(name)) { + Table table = checkStateNotNull(loadTable(id)); + cachedTables.put(name, table); + } + } + return cachedTables; + } + + @Override + public @Nullable Table getTable(String name) { + if (cachedTables.containsKey(name)) { + return cachedTables.get(name); + } + @Nullable Table table = loadTable(getIdentifier(name)); + if (table != null) { + cachedTables.put(name, table); + } + return table; + } + + private String getIdentifier(String name) { + return database + "." + name; + } + + private String getIdentifier(Table table) { + if (table.getLocation() != null) { + return table.getLocation(); + } + return getIdentifier(table.getName()); + } + + private @Nullable Table loadTable(String identifier) { + @Nullable IcebergTableInfo tableInfo = catalogConfig.loadTable(identifier); + if (tableInfo == null) { + return null; + } + String name = Iterables.getLast(Splitter.on(".").split(tableInfo.getIdentifier())); + return Table.builder() + .type(getTableType()) + .name(name) + .schema(tableInfo.getSchema()) + .location(tableInfo.getIdentifier()) + .properties(TableUtils.parseProperties(tableInfo.getProperties())) + .build(); + } + + @Override + public BeamSqlTable buildBeamSqlTable(Table table) { + return new IcebergTable(getIdentifier(table), table, catalogConfig); + } + + @Override + public boolean supportsPartitioning(Table table) { + return true; + } + + @Override + public void registerProvider(TableProvider provider) { + // no-op + } + + @Override + public void clearProviders() { + // no-op + } +} diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTable.java index 596a1d6d0457..605a9521845d 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTable.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTable.java @@ -17,7 +17,6 @@ */ package org.apache.beam.sdk.extensions.sql.meta.provider.iceberg; -import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; import com.fasterxml.jackson.databind.node.ObjectNode; @@ -66,10 +65,10 @@ class IcebergTable extends SchemaBaseBeamTable { @VisibleForTesting @Nullable Integer triggeringFrequency; @VisibleForTesting final @Nullable List partitionFields; - IcebergTable(Table table, IcebergCatalogConfig catalogConfig) { + IcebergTable(String tableIdentifier, Table table, IcebergCatalogConfig catalogConfig) { super(table.getSchema()); this.schema = table.getSchema(); - this.tableIdentifier = checkArgumentNotNull(table.getLocation()); + this.tableIdentifier = tableIdentifier; this.catalogConfig = catalogConfig; ObjectNode properties = table.getProperties(); if (properties.has(TRIGGERING_FREQUENCY_FIELD)) { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProvider.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProvider.java deleted file mode 100644 index 568893716581..000000000000 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProvider.java +++ /dev/null @@ -1,96 +0,0 @@ -/* - * 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. - */ -package org.apache.beam.sdk.extensions.sql.meta.provider.iceberg; - -import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; -import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; - -import java.util.HashMap; -import java.util.Map; -import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable; -import org.apache.beam.sdk.extensions.sql.meta.Table; -import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; -import org.apache.beam.sdk.io.iceberg.IcebergCatalogConfig; -import org.apache.beam.sdk.io.iceberg.TableAlreadyExistsException; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * A table provider for Iceberg tables. CREATE and DROP operations are performed on real external - * tables. - */ -public class IcebergTableProvider implements TableProvider { - private static final Logger LOG = LoggerFactory.getLogger(IcebergTableProvider.class); - @VisibleForTesting final IcebergCatalogConfig catalogConfig; - private final Map tables = new HashMap<>(); - - public IcebergTableProvider(IcebergCatalogConfig catalogConfig) { - this.catalogConfig = catalogConfig; - } - - @Override - public String getTableType() { - return "iceberg"; - } - - @Override - public void createTable(Table table) { - try { - catalogConfig.createTable( - checkStateNotNull(table.getLocation()), table.getSchema(), table.getPartitionFields()); - } catch (TableAlreadyExistsException e) { - LOG.info( - "Iceberg table '{}' already exists at location '{}'.", - table.getName(), - table.getLocation()); - } - tables.put(table.getName(), table); - } - - @Override - public void dropTable(String tableName) { - Table table = - checkArgumentNotNull(getTable(tableName), "Table '%s' is not registered.", tableName); - String location = checkStateNotNull(table.getLocation()); - if (catalogConfig.dropTable(location)) { - LOG.info("Dropped table '{}' (location: '{}').", tableName, location); - } else { - LOG.info( - "Ignoring DROP TABLE call for '{}' (location: '{}') because it does not exist.", - tableName, - location); - } - tables.remove(tableName); - } - - @Override - public Map getTables() { - return tables; - } - - @Override - public BeamSqlTable buildBeamSqlTable(Table table) { - return new IcebergTable(table, catalogConfig); - } - - @Override - public boolean supportsPartitioning(Table table) { - return true; - } -} diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStore.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStore.java index d3a8f9920c4a..0899d82d928c 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStore.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStore.java @@ -17,14 +17,13 @@ */ package org.apache.beam.sdk.extensions.sql.meta.store; -import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; - import java.util.HashMap; import java.util.Map; import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable; import org.apache.beam.sdk.extensions.sql.meta.Table; import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.checkerframework.checker.nullness.qual.Nullable; /** * A {@link MetaStore} which stores the meta info in memory. @@ -55,7 +54,7 @@ public void createTable(Table table) { } // invoke the provider's create - providers.get(table.getType()).createTable(table); + getProvider(table.getType()).createTable(table); // store to the global metastore tables.put(table.getName(), table); @@ -68,7 +67,7 @@ public void dropTable(String tableName) { } Table table = tables.get(tableName); - providers.get(table.getType()).dropTable(tableName); + getProvider(table.getType()).dropTable(tableName); tables.remove(tableName); } @@ -79,13 +78,21 @@ public Map getTables() { @Override public BeamSqlTable buildBeamSqlTable(Table table) { - TableProvider provider = providers.get(table.getType()); + TableProvider provider = getProvider(table.getType()); return provider.buildBeamSqlTable(table); } - private void validateTableType(Table table) { - if (!providers.containsKey(table.getType())) { + protected void validateTableType(Table table) { + if (providers.containsKey(table.getType())) { + return; + } + // check if there is a nested metastore that supports this table + @Nullable + InMemoryMetaStore nestedMemoryMetastore = (InMemoryMetaStore) providers.get(getTableType()); + if (nestedMemoryMetastore != null) { + nestedMemoryMetastore.validateTableType(table); + } else { throw new IllegalArgumentException("Table type: " + table.getType() + " not supported!"); } } @@ -112,22 +119,39 @@ private void initTablesFromProvider(TableProvider provider) { this.tables.putAll(tables); } + @Override + public void clearProviders() { + providers.clear(); + } + Map getProviders() { return providers; } @Override public boolean supportsPartitioning(Table table) { - TableProvider provider = providers.get(table.getType()); - if (provider == null) { - throw new IllegalArgumentException( - "No TableProvider registered for table type: " + table.getType()); - } - return provider.supportsPartitioning(table); + return getProvider(table.getType()).supportsPartitioning(table); } + /** + * Fetches a {@link TableProvider} for this type. This provider can exist in the current {@link + * InMemoryMetaStore} or a nested {@link InMemoryMetaStore}. + * + * @param type + * @return + */ public TableProvider getProvider(String type) { - return checkArgumentNotNull( - providers.get(type), "No TableProvider registered for table type: " + type); + @Nullable TableProvider provider = providers.get(type); + if (provider != null) { + return provider; + } + + // check nested InMemoryMetaStore + provider = providers.get(getTableType()); + if (provider != null && (provider instanceof InMemoryMetaStore)) { + return ((InMemoryMetaStore) provider).getProvider(type); + } + + throw new IllegalStateException("No TableProvider registered for table type: " + type); } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/MetaStore.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/MetaStore.java index 39ad6d3dfb54..59b855a5a9d8 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/MetaStore.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/MetaStore.java @@ -27,4 +27,7 @@ public interface MetaStore extends TableProvider { * @param provider */ void registerProvider(TableProvider provider); + + /** Clears all registered providers. */ + void clearProviders(); } diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliCatalogTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliCatalogTest.java new file mode 100644 index 000000000000..c1526df84a28 --- /dev/null +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliCatalogTest.java @@ -0,0 +1,341 @@ +/* + * 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. + */ +package org.apache.beam.sdk.extensions.sql; + +import static org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog.DEFAULT; +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; + +import java.util.Map; +import org.apache.beam.sdk.extensions.sql.meta.Table; +import org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog; +import org.apache.beam.sdk.extensions.sql.meta.catalog.InMemoryCatalogManager; +import org.apache.beam.sdk.extensions.sql.meta.provider.test.TestTableProvider; +import org.apache.beam.sdk.extensions.sql.meta.store.MetaStore; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.runtime.CalciteContextException; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +/** UnitTest for {@link BeamSqlCli} using catalogs. */ +public class BeamSqlCliCatalogTest { + @Rule public transient ExpectedException thrown = ExpectedException.none(); + private InMemoryCatalogManager catalogManager; + private BeamSqlCli cli; + + @Before + public void setupCli() { + catalogManager = new InMemoryCatalogManager(); + cli = new BeamSqlCli().catalogManager(catalogManager); + } + + @Test + public void testExecute_createCatalog_invalidTypeError() { + thrown.expect(UnsupportedOperationException.class); + thrown.expectMessage("Could not find type 'abcdef' for catalog 'invalid_catalog'."); + cli.execute("CREATE CATALOG invalid_catalog TYPE abcdef"); + } + + @Test + public void testExecute_createCatalog_duplicateCatalogError() { + cli.execute("CREATE CATALOG my_catalog TYPE 'local'"); + + // this should be fine. + cli.execute("CREATE CATALOG IF NOT EXISTS my_catalog TYPE 'local'"); + + // without "IF NOT EXISTS", Beam will throw an error + thrown.expect(CalciteContextException.class); + thrown.expectMessage("Catalog 'my_catalog' already exists."); + cli.execute("CREATE CATALOG my_catalog TYPE 'local'"); + } + + @Test + public void testExecute_createCatalog() { + assertNull(catalogManager.getCatalog("my_catalog")); + cli.execute( + "CREATE CATALOG my_catalog \n" + + "TYPE 'local' \n" + + "PROPERTIES (\n" + + " 'foo' = 'bar', \n" + + " 'abc' = 'xyz', \n" + + " 'beam.test.prop' = '123'\n" + + ")"); + assertNotNull(catalogManager.getCatalog("my_catalog")); + // we only created the catalog, but have not switched to it + assertNotEquals("my_catalog", catalogManager.currentCatalog().name()); + + Map expectedProps = + ImmutableMap.of( + "foo", "bar", + "abc", "xyz", + "beam.test.prop", "123"); + Catalog catalog = catalogManager.getCatalog("my_catalog"); + + assertEquals("my_catalog", catalog.name()); + assertEquals("local", catalog.type()); + assertEquals(expectedProps, catalog.properties()); + } + + @Test + public void testExecute_setCatalog_doesNotExistError() { + thrown.expect(CalciteContextException.class); + thrown.expectMessage("Cannot use catalog: 'my_catalog' not found."); + cli.execute("USE CATALOG my_catalog"); + } + + @Test + public void testExecute_setCatalog() { + assertNull(catalogManager.getCatalog("catalog_1")); + assertNull(catalogManager.getCatalog("catalog_2")); + Map catalog1Props = + ImmutableMap.of("foo", "bar", "abc", "xyz", "beam.test.prop", "123"); + Map catalog2Props = ImmutableMap.of("a", "b", "c", "d"); + cli.execute( + "CREATE CATALOG catalog_1 \n" + + "TYPE 'local' \n" + + "PROPERTIES (\n" + + " 'foo' = 'bar', \n" + + " 'abc' = 'xyz', \n" + + " 'beam.test.prop' = '123'\n" + + ")"); + cli.execute( + "CREATE CATALOG catalog_2 \n" + + "TYPE 'local' \n" + + "PROPERTIES (\n" + + " 'a' = 'b', \n" + + " 'c' = 'd' \n" + + ")"); + assertNotNull(catalogManager.getCatalog("catalog_1")); + assertNotNull(catalogManager.getCatalog("catalog_2")); + + // catalog manager always starts with a "default" catalog + assertEquals("default", catalogManager.currentCatalog().name()); + cli.execute("USE CATALOG catalog_1"); + assertEquals("catalog_1", catalogManager.currentCatalog().name()); + assertEquals(catalog1Props, catalogManager.currentCatalog().properties()); + cli.execute("USE CATALOG catalog_2"); + assertEquals("catalog_2", catalogManager.currentCatalog().name()); + assertEquals(catalog2Props, catalogManager.currentCatalog().properties()); + + // DEFAULT is a reserved keyword, so need to encapsulate in backticks + cli.execute("USE CATALOG 'default'"); + assertEquals("default", catalogManager.currentCatalog().name()); + } + + @Test + public void testExecute_dropCatalog_doesNotExistError() { + thrown.expect(CalciteContextException.class); + thrown.expectMessage("Cannot drop catalog: 'my_catalog' not found."); + cli.execute("DROP CATALOG 'my_catalog'"); + } + + @Test + public void testExecute_dropCatalog_activelyUsedError() { + thrown.expect(CalciteContextException.class); + thrown.expectMessage( + "Unable to drop active catalog 'default'. Please switch to another catalog first."); + cli.execute("DROP CATALOG 'default'"); + } + + @Test + public void testExecute_dropCatalog() { + assertNull(catalogManager.getCatalog("my_catalog")); + cli.execute( + "CREATE CATALOG my_catalog \n" + + "TYPE 'local' \n" + + "PROPERTIES (\n" + + " 'foo' = 'bar', \n" + + " 'abc' = 'xyz', \n" + + " 'beam.test.prop' = '123'\n" + + ")"); + assertNotNull(catalogManager.getCatalog("my_catalog")); + + assertNotEquals("my_catalog", catalogManager.currentCatalog().name()); + cli.execute("DROP CATALOG my_catalog"); + assertNull(catalogManager.getCatalog("my_catalog")); + } + + @Test + public void testCreateUseDropDatabaseWithSameCatalogScope() { + // create Catalog catalog_1 and create Database db_1 inside of it + cli.execute("CREATE CATALOG catalog_1 TYPE 'local'"); + cli.execute("USE CATALOG catalog_1"); + assertEquals("catalog_1", catalogManager.currentCatalog().name()); + assertEquals(DEFAULT, catalogManager.currentCatalog().currentDatabase()); + cli.execute("CREATE DATABASE db_1"); + cli.execute("USE DATABASE db_1"); + assertEquals("db_1", catalogManager.currentCatalog().currentDatabase()); + assertEquals(ImmutableSet.of(DEFAULT, "db_1"), catalogManager.currentCatalog().listDatabases()); + + // create new Catalog catalog_2 and switch to it + cli.execute("CREATE CATALOG catalog_2 TYPE 'local'"); + assertEquals("catalog_1", catalogManager.currentCatalog().name()); + cli.execute("USE CATALOG catalog_2"); + assertEquals("catalog_2", catalogManager.currentCatalog().name()); + assertEquals(DEFAULT, catalogManager.currentCatalog().currentDatabase()); + + // confirm that database 'db_1' from catalog_1 is not leaked to catalog_2 + assertEquals(ImmutableSet.of(DEFAULT), catalogManager.currentCatalog().listDatabases()); + + // switch back and drop database + cli.execute("USE CATALOG catalog_1"); + assertEquals("catalog_1", catalogManager.currentCatalog().name()); + cli.execute("DROP DATABASE db_1"); + assertEquals(ImmutableSet.of(DEFAULT), catalogManager.currentCatalog().listDatabases()); + } + + @Test + public void testCreateWriteDropTableWithSameCatalogScope() { + // create and use catalog + cli.execute("CREATE CATALOG catalog_1 TYPE 'local'"); + cli.execute("USE CATALOG catalog_1"); + assertEquals("catalog_1", catalogManager.currentCatalog().name()); + assertEquals(DEFAULT, catalogManager.currentCatalog().currentDatabase()); + + // create new database + cli.execute("CREATE DATABASE db_1"); + cli.execute("USE DATABASE db_1"); + assertEquals(ImmutableSet.of(DEFAULT, "db_1"), catalogManager.currentCatalog().listDatabases()); + MetaStore metastoreDb1 = + checkStateNotNull(catalogManager.getCatalog("catalog_1")).metaStore("db_1"); + + // create new table in catalog_1, db_1 + TestTableProvider testTableProvider = new TestTableProvider(); + catalogManager.registerTableProvider(testTableProvider); + cli.execute("CREATE EXTERNAL TABLE person(id int, name varchar, age int) TYPE 'test'"); + Table table = metastoreDb1.getTable("person"); + assertNotNull(table); + + // write to table + cli.execute("INSERT INTO person VALUES(123, 'John', 34)"); + TestTableProvider.TableWithRows tableWithRows = testTableProvider.tables().get(table.getName()); + assertEquals(1, tableWithRows.getRows().size()); + Row row = tableWithRows.getRows().get(0); + Row expectedRow = + Row.withSchema( + Schema.builder() + .addNullableInt32Field("id") + .addNullableStringField("name") + .addNullableInt32Field("age") + .build()) + .addValues(123, "John", 34) + .build(); + assertEquals(expectedRow, row); + + // drop the table + cli.execute("DROP TABLE person"); + assertNull(metastoreDb1.getTable("person")); + } + + @Test + public void testCreateUseDropDatabaseWithOtherCatalogScope() { + // create two catalogs + cli.execute("CREATE CATALOG catalog_1 TYPE 'local'"); + cli.execute("CREATE CATALOG catalog_2 TYPE 'local'"); + // set default catalog_2 + cli.execute("USE CATALOG catalog_2"); + assertEquals("catalog_2", catalogManager.currentCatalog().name()); + assertEquals(DEFAULT, catalogManager.currentCatalog().currentDatabase()); + // while using catalog_2, create new database in catalog_1 + cli.execute("CREATE DATABASE catalog_1.db_1"); + assertEquals( + ImmutableSet.of(DEFAULT, "db_1"), + checkStateNotNull(catalogManager.getCatalog("catalog_1")).listDatabases()); + + // use database in catalog_2. this will override both current database (to 'deb_1') + // and current catalog (to 'catalog_1') + cli.execute("USE DATABASE catalog_1.db_1"); + assertEquals("catalog_1", catalogManager.currentCatalog().name()); + assertEquals("db_1", catalogManager.currentCatalog().currentDatabase()); + assertEquals(ImmutableSet.of(DEFAULT, "db_1"), catalogManager.currentCatalog().listDatabases()); + + // switch back to catalog_2 and drop + cli.execute("USE CATALOG catalog_2"); + assertEquals("catalog_2", catalogManager.currentCatalog().name()); + // confirm that database 'db_1' created in catalog_1 was not leaked to catalog_2 + assertEquals(ImmutableSet.of(DEFAULT), catalogManager.currentCatalog().listDatabases()); + // drop and validate + assertEquals( + ImmutableSet.of(DEFAULT, "db_1"), + checkStateNotNull(catalogManager.getCatalog("catalog_1")).listDatabases()); + cli.execute("DROP DATABASE catalog_1.db_1"); + assertEquals( + ImmutableSet.of(DEFAULT), + checkStateNotNull(catalogManager.getCatalog("catalog_1")).listDatabases()); + } + + @Test + public void testCreateWriteDropTableWithOtherCatalogScope() { + // create two catalogs + cli.execute("CREATE CATALOG catalog_1 TYPE 'local'"); + cli.execute("CREATE CATALOG catalog_2 TYPE 'local'"); + // set default catalog_2 + cli.execute("USE CATALOG catalog_2"); + assertEquals("catalog_2", catalogManager.currentCatalog().name()); + assertEquals(DEFAULT, catalogManager.currentCatalog().currentDatabase()); + + // while using catalog_2, create new database in catalog_1 + cli.execute("CREATE DATABASE catalog_1.db_1"); + assertEquals( + ImmutableSet.of(DEFAULT, "db_1"), + checkStateNotNull(catalogManager.getCatalog("catalog_1")).listDatabases()); + MetaStore metastoreDb1 = + checkStateNotNull(catalogManager.getCatalog("catalog_1")).metaStore("db_1"); + + // while using catalog_2, create new table in catalog_1, db_1 + TestTableProvider testTableProvider = new TestTableProvider(); + catalogManager.registerTableProvider(testTableProvider); + cli.execute( + "CREATE EXTERNAL TABLE catalog_1.db_1.person(id int, name varchar, age int) TYPE 'test'"); + System.out.println("xxx metastoreDb1 tables: " + metastoreDb1.getTables()); + Table table = metastoreDb1.getTable("person"); + assertNotNull(table); + // confirm we are still using catalog_2 + assertEquals("catalog_2", catalogManager.currentCatalog().name()); + + // write to table while using catalog_2 + cli.execute("INSERT INTO catalog_1.db_1.person VALUES(123, 'John', 34)"); + TestTableProvider.TableWithRows tableWithRows = testTableProvider.tables().get(table.getName()); + assertEquals(1, tableWithRows.getRows().size()); + Row row = tableWithRows.getRows().get(0); + Row expectedRow = + Row.withSchema( + Schema.builder() + .addNullableInt32Field("id") + .addNullableStringField("name") + .addNullableInt32Field("age") + .build()) + .addValues(123, "John", 34) + .build(); + assertEquals(expectedRow, row); + // confirm we are still using catalog_2 + assertEquals("catalog_2", catalogManager.currentCatalog().name()); + + // drop the table while using catalog_2 + cli.execute("DROP TABLE catalog_1.db_1.person"); + assertNull(metastoreDb1.getTable("person")); + } +} diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliDatabaseTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliDatabaseTest.java index 1530141c6e22..6000ce2051f3 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliDatabaseTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliDatabaseTest.java @@ -17,9 +17,17 @@ */ package org.apache.beam.sdk.extensions.sql; +import static org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog.DEFAULT; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import org.apache.beam.sdk.extensions.sql.meta.Table; +import org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog; import org.apache.beam.sdk.extensions.sql.meta.catalog.InMemoryCatalogManager; +import org.apache.beam.sdk.extensions.sql.meta.provider.test.TestTableProvider; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.runtime.CalciteContextException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.junit.Before; @@ -43,7 +51,7 @@ public void setupCli() { public void testCreateDatabase() { cli.execute("CREATE DATABASE my_database"); assertEquals( - ImmutableSet.of("default", "my_database"), catalogManager.currentCatalog().listDatabases()); + ImmutableSet.of(DEFAULT, "my_database"), catalogManager.currentCatalog().listDatabases()); } @Test @@ -59,15 +67,15 @@ public void testCreateDuplicateDatabase_ifNotExists() { cli.execute("CREATE DATABASE my_database"); cli.execute("CREATE DATABASE IF NOT EXISTS my_database"); assertEquals( - ImmutableSet.of("default", "my_database"), catalogManager.currentCatalog().listDatabases()); + ImmutableSet.of(DEFAULT, "my_database"), catalogManager.currentCatalog().listDatabases()); } @Test public void testUseDatabase() { - assertEquals("default", catalogManager.currentCatalog().currentDatabase()); + assertEquals(DEFAULT, catalogManager.currentCatalog().currentDatabase()); cli.execute("CREATE DATABASE my_database"); cli.execute("CREATE DATABASE my_database2"); - assertEquals("default", catalogManager.currentCatalog().currentDatabase()); + assertEquals(DEFAULT, catalogManager.currentCatalog().currentDatabase()); cli.execute("USE DATABASE my_database"); assertEquals("my_database", catalogManager.currentCatalog().currentDatabase()); cli.execute("USE DATABASE my_database2"); @@ -76,7 +84,7 @@ public void testUseDatabase() { @Test public void testUseDatabase_doesNotExist() { - assertEquals("default", catalogManager.currentCatalog().currentDatabase()); + assertEquals(DEFAULT, catalogManager.currentCatalog().currentDatabase()); thrown.expect(CalciteContextException.class); thrown.expectMessage("Cannot use database: 'non_existent' not found."); cli.execute("USE DATABASE non_existent"); @@ -86,16 +94,100 @@ public void testUseDatabase_doesNotExist() { public void testDropDatabase() { cli.execute("CREATE DATABASE my_database"); assertEquals( - ImmutableSet.of("default", "my_database"), catalogManager.currentCatalog().listDatabases()); + ImmutableSet.of(DEFAULT, "my_database"), catalogManager.currentCatalog().listDatabases()); cli.execute("DROP DATABASE my_database"); - assertEquals(ImmutableSet.of("default"), catalogManager.currentCatalog().listDatabases()); + assertEquals(ImmutableSet.of(DEFAULT), catalogManager.currentCatalog().listDatabases()); } @Test public void testDropDatabase_nonexistent() { - assertEquals(ImmutableSet.of("default"), catalogManager.currentCatalog().listDatabases()); + assertEquals(ImmutableSet.of(DEFAULT), catalogManager.currentCatalog().listDatabases()); thrown.expect(CalciteContextException.class); thrown.expectMessage("Database 'my_database' does not exist."); cli.execute("DROP DATABASE my_database"); } + + @Test + public void testCreateInsertDropTableUsingDefaultDatabase() { + Catalog catalog = catalogManager.currentCatalog(); + // create new database db_1 + cli.execute("CREATE DATABASE db_1"); + cli.execute("USE DATABASE db_1"); + assertEquals("db_1", catalog.currentDatabase()); + assertEquals(ImmutableSet.of(DEFAULT, "db_1"), catalog.listDatabases()); + + // create new table + TestTableProvider testTableProvider = new TestTableProvider(); + catalogManager.registerTableProvider(testTableProvider); + cli.execute("CREATE EXTERNAL TABLE person(id int, name varchar, age int) TYPE 'test'"); + // table should be inside the currently used database + Table table = catalog.metaStore("db_1").getTable("person"); + assertNotNull(table); + + // write to the table + cli.execute("INSERT INTO person VALUES(123, 'John', 34)"); + TestTableProvider.TableWithRows tableWithRows = testTableProvider.tables().get(table.getName()); + assertEquals(1, tableWithRows.getRows().size()); + Row row = tableWithRows.getRows().get(0); + Row expectedRow = + Row.withSchema( + Schema.builder() + .addNullableInt32Field("id") + .addNullableStringField("name") + .addNullableInt32Field("age") + .build()) + .addValues(123, "John", 34) + .build(); + assertEquals(expectedRow, row); + + // drop table, using the current database + cli.execute("DROP TABLE person"); + assertNull(catalogManager.currentCatalog().metaStore("db_1").getTable("person")); + } + + @Test + public void testCreateInsertDropTableUsingOtherDatabase() { + Catalog catalog = catalogManager.currentCatalog(); + // create database db_1 + cli.execute("CREATE DATABASE db_1"); + cli.execute("USE DATABASE db_1"); + assertEquals("db_1", catalog.currentDatabase()); + assertEquals(ImmutableSet.of(DEFAULT, "db_1"), catalog.listDatabases()); + + // switch to other database db_2 + cli.execute("CREATE DATABASE db_2"); + cli.execute("USE DATABASE db_2"); + assertEquals("db_2", catalog.currentDatabase()); + + // create table from another database + TestTableProvider testTableProvider = new TestTableProvider(); + catalogManager.registerTableProvider(testTableProvider); + cli.execute("CREATE EXTERNAL TABLE db_1.person(id int, name varchar, age int) TYPE 'test'"); + // current database should not have the table + assertNull(catalog.metaStore("db_2").getTable("person")); + + // other database should have the table + Table table = catalog.metaStore("db_1").getTable("person"); + assertNotNull(table); + + // write to table from another database + cli.execute("INSERT INTO db_1.person VALUES(123, 'John', 34)"); + TestTableProvider.TableWithRows tableWithRows = testTableProvider.tables().get(table.getName()); + assertEquals(1, tableWithRows.getRows().size()); + Row row = tableWithRows.getRows().get(0); + Row expectedRow = + Row.withSchema( + Schema.builder() + .addNullableInt32Field("id") + .addNullableStringField("name") + .addNullableInt32Field("age") + .build()) + .addValues(123, "John", 34) + .build(); + assertEquals(expectedRow, row); + + // drop table, overriding the current database + cli.execute("DROP TABLE db_1.person"); + assertNull(catalogManager.currentCatalog().metaStore("db_1").getTable("person")); + } } diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliTest.java index b8aa030649ab..a212edac762c 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliTest.java @@ -25,26 +25,20 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import java.time.LocalDate; import java.time.LocalTime; -import java.util.Map; import java.util.stream.Stream; import org.apache.beam.sdk.extensions.sql.impl.ParseException; import org.apache.beam.sdk.extensions.sql.meta.Table; -import org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog; -import org.apache.beam.sdk.extensions.sql.meta.catalog.InMemoryCatalogManager; import org.apache.beam.sdk.extensions.sql.meta.provider.test.TestTableProvider; import org.apache.beam.sdk.extensions.sql.meta.provider.text.TextTableProvider; import org.apache.beam.sdk.extensions.sql.meta.store.InMemoryMetaStore; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.Schema.Field; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.runtime.CalciteContextException; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -248,176 +242,6 @@ public void testExecute_dropTable_assertTableRemovedFromPlanner() throws Excepti cli.explainQuery("select * from person"); } - @Test - public void testExecute_createCatalog_invalidTypeError() { - InMemoryCatalogManager catalogManager = new InMemoryCatalogManager(); - BeamSqlCli cli = new BeamSqlCli().catalogManager(catalogManager); - - thrown.expect(UnsupportedOperationException.class); - thrown.expectMessage("Could not find type 'abcdef' for catalog 'invalid_catalog'."); - cli.execute("CREATE CATALOG invalid_catalog TYPE abcdef"); - } - - @Test - public void testExecute_createCatalog_duplicateCatalogError() { - InMemoryCatalogManager catalogManager = new InMemoryCatalogManager(); - BeamSqlCli cli = new BeamSqlCli().catalogManager(catalogManager); - - cli.execute("CREATE CATALOG my_catalog TYPE 'local'"); - - // this should be fine. - cli.execute("CREATE CATALOG IF NOT EXISTS my_catalog TYPE 'local'"); - - // without "IF NOT EXISTS", Beam will throw an error - thrown.expect(CalciteContextException.class); - thrown.expectMessage("Catalog 'my_catalog' already exists."); - cli.execute("CREATE CATALOG my_catalog TYPE 'local'"); - } - - @Test - public void testExecute_createCatalog() { - InMemoryCatalogManager catalogManager = new InMemoryCatalogManager(); - BeamSqlCli cli = new BeamSqlCli().catalogManager(catalogManager); - - assertNull(catalogManager.getCatalog("my_catalog")); - cli.execute( - "CREATE CATALOG my_catalog \n" - + "TYPE 'local' \n" - + "PROPERTIES (\n" - + " 'foo' = 'bar', \n" - + " 'abc' = 'xyz', \n" - + " 'beam.test.prop' = '123'\n" - + ")"); - assertNotNull(catalogManager.getCatalog("my_catalog")); - // we only created the catalog, but have not switched to it - assertNotEquals("my_catalog", catalogManager.currentCatalog().name()); - - Map expectedProps = - ImmutableMap.of( - "foo", "bar", - "abc", "xyz", - "beam.test.prop", "123"); - Catalog catalog = catalogManager.getCatalog("my_catalog"); - - assertEquals("my_catalog", catalog.name()); - assertEquals("local", catalog.type()); - assertEquals(expectedProps, catalog.properties()); - } - - @Test - public void testExecute_setCatalog_doesNotExistError() { - InMemoryCatalogManager catalogManager = new InMemoryCatalogManager(); - BeamSqlCli cli = new BeamSqlCli().catalogManager(catalogManager); - - thrown.expect(CalciteContextException.class); - thrown.expectMessage("Cannot use catalog: 'my_catalog' not found."); - cli.execute("USE CATALOG my_catalog"); - } - - @Test - public void testExecute_setCatalog() { - InMemoryCatalogManager catalogManager = new InMemoryCatalogManager(); - BeamSqlCli cli = new BeamSqlCli().catalogManager(catalogManager); - - assertNull(catalogManager.getCatalog("catalog_1")); - assertNull(catalogManager.getCatalog("catalog_2")); - Map catalog1Props = - ImmutableMap.of("foo", "bar", "abc", "xyz", "beam.test.prop", "123"); - Map catalog2Props = ImmutableMap.of("a", "b", "c", "d"); - cli.execute( - "CREATE CATALOG catalog_1 \n" - + "TYPE 'local' \n" - + "PROPERTIES (\n" - + " 'foo' = 'bar', \n" - + " 'abc' = 'xyz', \n" - + " 'beam.test.prop' = '123'\n" - + ")"); - cli.execute( - "CREATE CATALOG catalog_2 \n" - + "TYPE 'local' \n" - + "PROPERTIES (\n" - + " 'a' = 'b', \n" - + " 'c' = 'd' \n" - + ")"); - assertNotNull(catalogManager.getCatalog("catalog_1")); - assertNotNull(catalogManager.getCatalog("catalog_2")); - - // catalog manager always starts with a "default" catalog - assertEquals("default", catalogManager.currentCatalog().name()); - cli.execute("USE CATALOG catalog_1"); - assertEquals("catalog_1", catalogManager.currentCatalog().name()); - assertEquals(catalog1Props, catalogManager.currentCatalog().properties()); - cli.execute("USE CATALOG catalog_2"); - assertEquals("catalog_2", catalogManager.currentCatalog().name()); - assertEquals(catalog2Props, catalogManager.currentCatalog().properties()); - - // DEFAULT is a reserved keyword, so need to encapsulate in backticks - cli.execute("USE CATALOG 'default'"); - assertEquals("default", catalogManager.currentCatalog().name()); - } - - @Test - public void testExecute_dropCatalog_doesNotExistError() { - InMemoryCatalogManager catalogManager = new InMemoryCatalogManager(); - BeamSqlCli cli = new BeamSqlCli().catalogManager(catalogManager); - - thrown.expect(CalciteContextException.class); - thrown.expectMessage("Cannot drop catalog: 'my_catalog' not found."); - cli.execute("DROP CATALOG 'my_catalog'"); - } - - @Test - public void testExecute_dropCatalog_activelyUsedError() { - InMemoryCatalogManager catalogManager = new InMemoryCatalogManager(); - BeamSqlCli cli = new BeamSqlCli().catalogManager(catalogManager); - - thrown.expect(CalciteContextException.class); - thrown.expectMessage( - "Unable to drop active catalog 'default'. Please switch to another catalog first."); - cli.execute("DROP CATALOG 'default'"); - } - - @Test - public void testExecute_dropCatalog() { - InMemoryCatalogManager catalogManager = new InMemoryCatalogManager(); - BeamSqlCli cli = new BeamSqlCli().catalogManager(catalogManager); - - assertNull(catalogManager.getCatalog("my_catalog")); - cli.execute( - "CREATE CATALOG my_catalog \n" - + "TYPE 'local' \n" - + "PROPERTIES (\n" - + " 'foo' = 'bar', \n" - + " 'abc' = 'xyz', \n" - + " 'beam.test.prop' = '123'\n" - + ")"); - assertNotNull(catalogManager.getCatalog("my_catalog")); - - assertNotEquals("my_catalog", catalogManager.currentCatalog().name()); - cli.execute("DROP CATALOG my_catalog"); - assertNull(catalogManager.getCatalog("my_catalog")); - } - - @Test - public void testExecute_tableScopeAcrossCatalogs() throws Exception { - InMemoryCatalogManager catalogManager = new InMemoryCatalogManager(); - catalogManager.registerTableProvider(new TextTableProvider()); - BeamSqlCli cli = new BeamSqlCli().catalogManager(catalogManager); - - cli.execute("CREATE CATALOG my_catalog TYPE 'local'"); - cli.execute("USE CATALOG my_catalog"); - cli.execute( - "CREATE EXTERNAL TABLE person (\n" + "id int, name varchar, age int) \n" + "TYPE 'text'"); - - assertEquals("my_catalog", catalogManager.currentCatalog().name()); - assertNotNull(catalogManager.currentCatalog().metaStore().getTables().get("person")); - - cli.execute("CREATE CATALOG my_other_catalog TYPE 'local'"); - cli.execute("USE CATALOG my_other_catalog"); - assertEquals("my_other_catalog", catalogManager.currentCatalog().name()); - assertNull(catalogManager.currentCatalog().metaStore().getTables().get("person")); - } - @Test public void testExplainQuery() throws Exception { InMemoryMetaStore metaStore = new InMemoryMetaStore(); diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriverTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriverTest.java index e83ee61af4ab..d52e5b5c0355 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriverTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriverTest.java @@ -116,9 +116,9 @@ public void testDriverManager_simple() throws Exception { public void testDriverManager_defaultUserAgent() throws Exception { Connection connection = DriverManager.getConnection(JdbcDriver.CONNECT_STRING_PREFIX); SchemaPlus rootSchema = ((CalciteConnection) connection).getRootSchema(); - BeamCalciteSchema beamSchema = - (BeamCalciteSchema) CalciteSchema.from(rootSchema.getSubSchema("beam")).schema; - Map pipelineOptions = beamSchema.getPipelineOptions(); + CatalogManagerSchema catalogManagerSchema = + (CatalogManagerSchema) CalciteSchema.from(rootSchema.getSubSchema("beam")).schema; + Map pipelineOptions = catalogManagerSchema.connection().getPipelineOptionsMap(); assertThat(pipelineOptions.get("userAgent"), containsString("BeamSQL")); } @@ -127,9 +127,9 @@ public void testDriverManager_defaultUserAgent() throws Exception { public void testDriverManager_hasUserAgent() throws Exception { JdbcConnection connection = (JdbcConnection) DriverManager.getConnection(JdbcDriver.CONNECT_STRING_PREFIX); - BeamCalciteSchema schema = connection.getCurrentBeamSchema(); + CatalogManagerSchema schema = connection.getCurrentBeamSchema(); assertThat( - schema.getPipelineOptions().get("userAgent"), + schema.connection().getPipelineOptionsMap().get("userAgent"), equalTo("BeamSQL/" + ReleaseInfo.getReleaseInfo().getVersion())); } @@ -140,9 +140,9 @@ public void testDriverManager_setUserAgent() throws Exception { DriverManager.getConnection( JdbcDriver.CONNECT_STRING_PREFIX + "beam.userAgent=Secret Agent"); SchemaPlus rootSchema = ((CalciteConnection) connection).getRootSchema(); - BeamCalciteSchema beamSchema = - (BeamCalciteSchema) CalciteSchema.from(rootSchema.getSubSchema("beam")).schema; - Map pipelineOptions = beamSchema.getPipelineOptions(); + CatalogManagerSchema catalogManagerSchema = + (CatalogManagerSchema) CalciteSchema.from(rootSchema.getSubSchema("beam")).schema; + Map pipelineOptions = catalogManagerSchema.connection().getPipelineOptionsMap(); assertThat(pipelineOptions.get("userAgent"), equalTo("Secret Agent")); } @@ -154,9 +154,9 @@ public void testDriverManager_pipelineOptionsPlumbing() throws Exception { JdbcDriver.CONNECT_STRING_PREFIX + "beam.foo=baz;beam.foobizzle=mahshizzle;other=smother"); SchemaPlus rootSchema = ((CalciteConnection) connection).getRootSchema(); - BeamCalciteSchema beamSchema = - (BeamCalciteSchema) CalciteSchema.from(rootSchema.getSubSchema("beam")).schema; - Map pipelineOptions = beamSchema.getPipelineOptions(); + CatalogManagerSchema catalogManagerSchema = + (CatalogManagerSchema) CalciteSchema.from(rootSchema.getSubSchema("beam")).schema; + Map pipelineOptions = catalogManagerSchema.connection().getPipelineOptionsMap(); assertThat(pipelineOptions.get("foo"), equalTo("baz")); assertThat(pipelineOptions.get("foobizzle"), equalTo("mahshizzle")); assertThat(pipelineOptions.get("other"), nullValue()); diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLNestedTypesTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLNestedTypesTest.java index 6876caff3274..fca68e4fd32c 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLNestedTypesTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLNestedTypesTest.java @@ -75,7 +75,7 @@ private Table executeCreateTableWith(String fieldType) throws SqlParseException + "fieldName " + fieldType + " ) " - + "TYPE 'text' " + + "TYPE 'test' " + "LOCATION '/home/admin/person'\n"; System.out.println(createTable); diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLTest.java index 704a9d4586e1..04c0d44f62d7 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLTest.java @@ -61,13 +61,13 @@ public void testParseCreateExternalTable_full() throws Exception { "CREATE EXTERNAL TABLE person (\n" + "id int COMMENT 'id', \n" + "name varchar COMMENT 'name') \n" - + "TYPE 'text' \n" + + "TYPE 'test' \n" + "COMMENT 'person table' \n" + "LOCATION '/home/admin/person'\n" + "TBLPROPERTIES '{\"hello\": [\"james\", \"bond\"]}'"); assertEquals( - mockTable("person", "text", "person table", properties), + mockTable("person", "test", "person table", properties), tableProvider.getTables().get("person")); } @@ -80,7 +80,7 @@ public void testParseCreateExternalTable_WithComplexFields() { "CREATE EXTERNAL TABLE PersonDetails" + " ( personInfo MAP> , " + " additionalInfo ROW )" - + " TYPE 'text'" + + " TYPE 'test'" + " LOCATION '/home/admin/person'"); assertNotNull(tableProvider.getTables().get("PersonDetails")); @@ -105,7 +105,7 @@ public void testParseCreateTable() throws Exception { "CREATE TABLE person (\n" + "id int COMMENT 'id', \n" + "name varchar COMMENT 'name') \n" - + "TYPE 'text' \n" + + "TYPE 'test' \n" + "COMMENT 'person table' \n" + "LOCATION '/home/admin/person'\n" + "TBLPROPERTIES '{\"hello\": [\"james\", \"bond\"]}'"); @@ -126,11 +126,11 @@ public void testParseCreateExternalTable_withoutTableComment() throws Exception "CREATE EXTERNAL TABLE person (\n" + "id int COMMENT 'id', \n" + "name varchar COMMENT 'name') \n" - + "TYPE 'text' \n" + + "TYPE 'test' \n" + "LOCATION '/home/admin/person'\n" + "TBLPROPERTIES '{\"hello\": [\"james\", \"bond\"]}'"); assertEquals( - mockTable("person", "text", null, properties), tableProvider.getTables().get("person")); + mockTable("person", "test", null, properties), tableProvider.getTables().get("person")); } @Test @@ -142,11 +142,11 @@ public void testParseCreateExternalTable_withoutTblProperties() throws Exception "CREATE EXTERNAL TABLE person (\n" + "id int COMMENT 'id', \n" + "name varchar COMMENT 'name') \n" - + "TYPE 'text' \n" + + "TYPE 'test' \n" + "COMMENT 'person table' \n" + "LOCATION '/home/admin/person'\n"); assertEquals( - mockTable("person", "text", "person table", TableUtils.emptyProperties()), + mockTable("person", "test", "person table", TableUtils.emptyProperties()), tableProvider.getTables().get("person")); } @@ -159,11 +159,11 @@ public void testParseCreateExternalTable_withoutLocation() throws Exception { "CREATE EXTERNAL TABLE person (\n" + "id int COMMENT 'id', \n" + "name varchar COMMENT 'name') \n" - + "TYPE 'text' \n" + + "TYPE 'test' \n" + "COMMENT 'person table' \n"); assertEquals( - mockTable("person", "text", "person table", TableUtils.emptyProperties(), null), + mockTable("person", "test", "person table", TableUtils.emptyProperties(), null), tableProvider.getTables().get("person")); } @@ -172,12 +172,12 @@ public void testParseCreateExternalTable_minimal() throws Exception { TestTableProvider tableProvider = new TestTableProvider(); BeamSqlEnv env = BeamSqlEnv.withTableProvider(tableProvider); - env.executeDdl("CREATE EXTERNAL TABLE person (id INT) TYPE text"); + env.executeDdl("CREATE EXTERNAL TABLE person (id INT) TYPE test"); assertEquals( Table.builder() .name("person") - .type("text") + .type("test") .schema( Stream.of(Schema.Field.of("id", CalciteUtils.INTEGER).withNullable(true)) .collect(toSchema())) @@ -197,7 +197,7 @@ public void testParseCreateExternalTable_withDatabase() throws Exception { .setPipelineOptions(PipelineOptionsFactory.create()) .build(); assertNull(testProvider.getTables().get("person")); - env.executeDdl("CREATE EXTERNAL TABLE test.person (id INT) TYPE text"); + env.executeDdl("CREATE EXTERNAL TABLE test.person (id INT) TYPE test"); assertNotNull(testProvider.getTables().get("person")); } @@ -212,7 +212,7 @@ public void testParseDropTable() throws Exception { "CREATE EXTERNAL TABLE person (\n" + "id int COMMENT 'id', \n" + "name varchar COMMENT 'name') \n" - + "TYPE 'text' \n" + + "TYPE 'test' \n" + "COMMENT 'person table' \n"); assertNotNull(tableProvider.getTables().get("person")); diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BaseRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BaseRelTest.java index 5ba74e88acc3..e964ec0a992a 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BaseRelTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BaseRelTest.java @@ -31,11 +31,13 @@ public abstract class BaseRelTest { protected static BeamSqlEnv env = BeamSqlEnv.readOnly("test", tables); protected static PCollection compilePipeline(String sql, Pipeline pipeline) { + env = BeamSqlEnv.readOnly("test", tables); return BeamSqlRelUtils.toPCollection(pipeline, env.parseQuery(sql)); } protected static void registerTable(String tableName, BeamSqlTable table) { tables.put(tableName, table); + env = BeamSqlEnv.readOnly("test", tables); } protected static BeamSqlTable getTable(String tableName) { diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/JoinReorderingTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/JoinReorderingTest.java index 77de4cdec0f9..776f11a95728 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/JoinReorderingTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/JoinReorderingTest.java @@ -327,20 +327,20 @@ private void assertTopTableInJoins(RelNode parsedQuery, String expectedTableName private void createThreeTables(TestTableProvider tableProvider) { BeamSqlEnv env = BeamSqlEnv.withTableProvider(tableProvider); - env.executeDdl("CREATE EXTERNAL TABLE small_table (id INTEGER, medium_key INTEGER) TYPE text"); + env.executeDdl("CREATE EXTERNAL TABLE small_table (id INTEGER, medium_key INTEGER) TYPE test"); env.executeDdl( "CREATE EXTERNAL TABLE medium_table (" + "id INTEGER," + "small_key INTEGER," + "large_key INTEGER" - + ") TYPE text"); + + ") TYPE test"); env.executeDdl( "CREATE EXTERNAL TABLE large_table (" + "id INTEGER," + "medium_key INTEGER" - + ") TYPE text"); + + ") TYPE test"); Row row = Row.withSchema(tableProvider.getTable("small_table").getSchema()).addValues(1, 1).build(); diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/BeamSqlCliIcebergTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/BeamSqlCliIcebergTest.java index cc6e3b426ec3..ea526f19849b 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/BeamSqlCliIcebergTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/BeamSqlCliIcebergTest.java @@ -18,31 +18,43 @@ package org.apache.beam.sdk.extensions.sql.meta.provider.iceberg; import static java.lang.String.format; +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import java.io.File; import java.io.IOException; import java.util.UUID; +import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.extensions.sql.BeamSqlCli; +import org.apache.beam.sdk.extensions.sql.SqlTransform; +import org.apache.beam.sdk.extensions.sql.impl.BeamSqlEnv; +import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode; +import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils; import org.apache.beam.sdk.extensions.sql.meta.catalog.InMemoryCatalogManager; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.runtime.CalciteContextException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; -import org.junit.Assert; +import org.joda.time.DateTime; import org.junit.Before; import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; +import org.testcontainers.shaded.org.checkerframework.checker.nullness.qual.Nullable; /** UnitTest for {@link BeamSqlCli} using Iceberg catalog. */ public class BeamSqlCliIcebergTest { @Rule public transient ExpectedException thrown = ExpectedException.none(); private InMemoryCatalogManager catalogManager; private BeamSqlCli cli; + private BeamSqlEnv sqlEnv; private String warehouse; @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); @@ -50,17 +62,26 @@ public class BeamSqlCliIcebergTest { public void setup() throws IOException { catalogManager = new InMemoryCatalogManager(); cli = new BeamSqlCli().catalogManager(catalogManager); + sqlEnv = + BeamSqlEnv.builder(catalogManager) + .setPipelineOptions(PipelineOptionsFactory.create()) + .build(); File warehouseFile = TEMPORARY_FOLDER.newFolder(); - Assert.assertTrue(warehouseFile.delete()); + assertTrue(warehouseFile.delete()); warehouse = "file:" + warehouseFile + "/" + UUID.randomUUID(); } private String createCatalog(String name) { + return createCatalog(name, null); + } + + private String createCatalog(String name, @Nullable String warehouseOverride) { + String ware = warehouseOverride != null ? warehouseOverride : warehouse; return format("CREATE CATALOG %s \n", name) + "TYPE iceberg \n" + "PROPERTIES (\n" + " 'type' = 'hadoop', \n" - + format(" 'warehouse' = '%s')", warehouse); + + format(" 'warehouse' = '%s')", ware); } @Test @@ -68,7 +89,6 @@ public void testCreateCatalog() { assertEquals("default", catalogManager.currentCatalog().name()); cli.execute(createCatalog("my_catalog")); - assertNotNull(catalogManager.getCatalog("my_catalog")); assertEquals("default", catalogManager.currentCatalog().name()); cli.execute("USE CATALOG my_catalog"); @@ -137,4 +157,88 @@ public void testDropNamespace() { thrown.expectMessage("Database 'new_namespace' does not exist."); cli.execute("DROP DATABASE new_namespace"); } + + @Test + public void testCrossCatalogTableWriteAndRead() throws IOException { + // create and use catalog 1 + sqlEnv.executeDdl(createCatalog("catalog_1")); + sqlEnv.executeDdl("USE CATALOG catalog_1"); + assertEquals("catalog_1", catalogManager.currentCatalog().name()); + // create and use database inside catalog 1 + IcebergCatalog catalog = (IcebergCatalog) catalogManager.currentCatalog(); + sqlEnv.executeDdl("CREATE DATABASE my_namespace"); + sqlEnv.executeDdl("USE DATABASE my_namespace"); + assertEquals("my_namespace", catalog.currentDatabase()); + // create and write to table inside database + String tableIdentifier = "my_namespace.my_table"; + sqlEnv.executeDdl( + format("CREATE EXTERNAL TABLE %s( \n", tableIdentifier) + + " c_integer INTEGER, \n" + + " c_boolean BOOLEAN, \n" + + " c_timestamp TIMESTAMP, \n" + + " c_varchar VARCHAR \n " + + ") \n" + + "TYPE 'iceberg'\n"); + BeamRelNode insertNode = + sqlEnv.parseQuery( + format("INSERT INTO %s VALUES (", tableIdentifier) + + "2147483647, " + + "TRUE, " + + "TIMESTAMP '2025-07-31 20:17:40.123', " + + "'varchar' " + + ")"); + Pipeline p1 = Pipeline.create(); + BeamSqlRelUtils.toPCollection(p1, insertNode); + p1.run().waitUntilFinish(); + + // create and use a new catalog, with a new database + File warehouseFile2 = TEMPORARY_FOLDER.newFolder(); + assertTrue(warehouseFile2.delete()); + String warehouse2 = "file:" + warehouseFile2 + "/" + UUID.randomUUID(); + sqlEnv.executeDdl(createCatalog("catalog_2", warehouse2)); + sqlEnv.executeDdl("USE CATALOG catalog_2"); + sqlEnv.executeDdl("CREATE DATABASE other_namespace"); + sqlEnv.executeDdl("USE DATABASE other_namespace"); + assertEquals("catalog_2", catalogManager.currentCatalog().name()); + assertEquals("other_namespace", catalogManager.currentCatalog().currentDatabase()); + + // insert from old catalog to new table in new catalog + Pipeline p2 = Pipeline.create(); + p2.apply( + SqlTransform.query("INSERT INTO other_table SELECT * FROM catalog_1.my_namespace.my_table") + .withDdlString( + "CREATE EXTERNAL TABLE other_table( \n" + + " c_integer INTEGER, \n" + + " c_boolean BOOLEAN, \n" + + " c_timestamp TIMESTAMP, \n" + + " c_varchar VARCHAR) \n" + + "TYPE 'iceberg'\n") + .withCatalogManager(catalogManager)); + p2.run().waitUntilFinish(); + + // clear PCollection from the above run + catalogManager.clearTableProviders(); + + // switch over to catalog 1 and read table inside catalog 2 + Pipeline p3 = Pipeline.create(); + PCollection output = + p3.apply( + SqlTransform.query("SELECT * FROM catalog_2.other_namespace.other_table") + .withDdlString("USE DATABASE catalog_1.my_namespace") + .withCatalogManager(catalogManager)); + p3.run().waitUntilFinish(); + assertEquals("catalog_1", catalogManager.currentCatalog().name()); + assertEquals("my_namespace", catalogManager.currentCatalog().currentDatabase()); + + // validate read contents + Schema expectedSchema = + checkStateNotNull(catalog.catalogConfig.loadTable(tableIdentifier)).getSchema(); + assertEquals(expectedSchema, output.getSchema()); + PAssert.that(output) + .containsInAnyOrder( + Row.withSchema(expectedSchema) + .addValues(2147483647, true, DateTime.parse("2025-07-31T20:17:40.123Z"), "varchar") + .build()); + p3.run().waitUntilFinish(); + } } diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProviderTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastoreTest.java similarity index 92% rename from sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProviderTest.java rename to sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastoreTest.java index 3e63eb8457e0..e963c616da07 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProviderTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastoreTest.java @@ -32,8 +32,8 @@ import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.junit.Test; -/** UnitTest for {@link IcebergTableProvider}. */ -public class IcebergTableProviderTest { +/** UnitTest for {@link IcebergMetastore}. */ +public class IcebergMetastoreTest { private final IcebergCatalog catalog = new IcebergCatalog( "test_catalog", @@ -46,7 +46,7 @@ public class IcebergTableProviderTest { @Test public void testGetTableType() { - assertNotNull(catalog.metaStore().getProvider("iceberg")); + assertEquals("iceberg", catalog.metaStore(catalog.currentDatabase()).getTableType()); } @Test @@ -59,7 +59,7 @@ public void testBuildBeamSqlTable() throws Exception { fakeTableBuilder("my_table") .properties(TableUtils.parseProperties(propertiesString)) .build(); - BeamSqlTable sqlTable = catalog.metaStore().buildBeamSqlTable(table); + BeamSqlTable sqlTable = catalog.metaStore(catalog.currentDatabase()).buildBeamSqlTable(table); assertNotNull(sqlTable); assertTrue(sqlTable instanceof IcebergTable); diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java index 15fe4769c61b..3badae069726 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java @@ -184,9 +184,8 @@ public void runSqlWriteAndRead(boolean withPartitionFields) // 3) verify a real Iceberg table was created, with the right partition spec IcebergCatalog catalog = (IcebergCatalog) catalogManager.currentCatalog(); - IcebergTableProvider provider = - (IcebergTableProvider) catalog.metaStore().getProvider("iceberg"); - Catalog icebergCatalog = provider.catalogConfig.catalog(); + IcebergMetastore metastore = catalog.metaStore(DATASET); + Catalog icebergCatalog = metastore.catalogConfig.catalog(); PartitionSpec expectedSpec = PartitionSpec.unpartitioned(); if (withPartitionFields) { expectedSpec = @@ -202,7 +201,7 @@ public void runSqlWriteAndRead(boolean withPartitionFields) assertEquals("my_catalog." + tableIdentifier, icebergTable.name()); assertTrue(icebergTable.location().startsWith(warehouse)); assertEquals(expectedSpec, icebergTable.spec()); - Schema expectedSchema = checkStateNotNull(provider.getTable("TEST")).getSchema(); + Schema expectedSchema = checkStateNotNull(metastore.getTable("TEST")).getSchema(); assertEquals(expectedSchema, IcebergUtils.icebergSchemaToBeamSchema(icebergTable.schema())); // 4) write to underlying Iceberg table diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStoreTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStoreTest.java index 825f3ed06485..f90c90588983 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStoreTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStoreTest.java @@ -119,6 +119,7 @@ private static Table mockTable(String name, String type) { .name(name) .comment(name + " table") .location("/home/admin/" + name) + // .path("default.default." + name) .schema( Stream.of( Schema.Field.nullable("id", Schema.FieldType.INT32), diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCatalogConfig.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCatalogConfig.java index 96357b44e54b..48a831a2d902 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCatalogConfig.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCatalogConfig.java @@ -32,11 +32,13 @@ import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.SupportsNamespaces; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.NoSuchTableException; import org.checkerframework.checker.nullness.qual.MonotonicNonNull; import org.checkerframework.checker.nullness.qual.Nullable; import org.checkerframework.dataflow.qual.Pure; @@ -152,6 +154,34 @@ public void createTable( } } + public @Nullable IcebergTableInfo loadTable(String tableIdentifier) { + TableIdentifier icebergIdentifier = TableIdentifier.parse(tableIdentifier); + try { + Table table = catalog().loadTable(icebergIdentifier); + return IcebergTableInfo.create( + tableIdentifier, + IcebergUtils.icebergSchemaToBeamSchema(table.schema()), + table.properties()); + } catch (NoSuchTableException ignored) { + return null; + } + } + + // Helper class to pass information to Beam SQL module without relying on Iceberg deps + @AutoValue + public abstract static class IcebergTableInfo { + public abstract String getIdentifier(); + + public abstract Schema getSchema(); + + public abstract Map getProperties(); + + static IcebergTableInfo create( + String identifier, Schema schema, Map properties) { + return new AutoValue_IcebergCatalogConfig_IcebergTableInfo(identifier, schema, properties); + }; + } + public boolean dropTable(String tableIdentifier) { TableIdentifier icebergIdentifier = TableIdentifier.parse(tableIdentifier); return catalog().dropTable(icebergIdentifier); diff --git a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/BeamSqlEnvRunner.java b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/BeamSqlEnvRunner.java index 9f3b68afc451..c8390ab30f6b 100644 --- a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/BeamSqlEnvRunner.java +++ b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/BeamSqlEnvRunner.java @@ -35,6 +35,7 @@ import org.apache.beam.sdk.extensions.sql.impl.BeamSqlPipelineOptions; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils; import org.apache.beam.sdk.extensions.sql.meta.Table; +import org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog; import org.apache.beam.sdk.extensions.sql.meta.catalog.InMemoryCatalogManager; import org.apache.beam.sdk.extensions.sql.meta.provider.text.TextTableProvider; import org.apache.beam.sdk.io.TextIO; @@ -117,7 +118,8 @@ private static void registerAllTablesByInMemoryMetaStore( .properties(properties) .type("text") .build(); - inMemoryCatalogManager.currentCatalog().metaStore().createTable(table); + Catalog catalog = inMemoryCatalogManager.currentCatalog(); + catalog.metaStore(catalog.currentDatabase()).createTable(table); } } From 07f25c53d662633442c51557eac161ebd2db8910 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Tue, 5 Aug 2025 14:04:07 -0400 Subject: [PATCH 02/11] some java doc --- .../src/main/codegen/includes/parserImpls.ftl | 6 +++--- .../sql/impl/BeamCalciteSchema.java | 7 +++++-- .../sql/impl/CatalogManagerSchema.java | 21 ++++++++++++------- .../extensions/sql/impl/CatalogSchema.java | 5 ++++- 4 files changed, 25 insertions(+), 14 deletions(-) diff --git a/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl b/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl index 9ca50468e84c..46102c7b92fe 100644 --- a/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl +++ b/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl @@ -265,7 +265,7 @@ SqlDrop SqlDropCatalog(Span s, boolean replace) : } /** - * CREATE DATABASE ( IF NOT EXISTS )? database_name + * CREATE DATABASE ( IF NOT EXISTS )? ( catalog_name '.' )? database_name */ SqlCreate SqlCreateDatabase(Span s, boolean replace) : { @@ -290,7 +290,7 @@ SqlCreate SqlCreateDatabase(Span s, boolean replace) : } /** - * USE DATABASE database_name + * USE DATABASE ( catalog_name '.' )? database_name */ SqlCall SqlUseDatabase(Span s, String scope) : { @@ -351,7 +351,7 @@ SqlNodeList PartitionFieldList() : * Note: This example is probably out of sync with the code. * * CREATE EXTERNAL TABLE ( IF NOT EXISTS )? - * ( database_name '.' )? table_name '(' column_def ( ',' column_def )* ')' + * ( catalog_name '.' )? ( database_name '.' )? table_name '(' column_def ( ',' column_def )* ')' * TYPE type_name * ( PARTITIONED BY '(' partition_field ( ',' partition_field )* ')' )? * ( COMMENT comment_string )? diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteSchema.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteSchema.java index 963c54285f2c..1f7d338afd33 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteSchema.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteSchema.java @@ -35,7 +35,11 @@ import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Schemas; import org.checkerframework.checker.nullness.qual.Nullable; -/** Adapter from {@link TableProvider} to {@link Schema}. */ +/** + * A Calcite {@link Schema} that corresponds to a {@link TableProvider} or {@link + * org.apache.beam.sdk.extensions.sql.meta.store.MetaStore}. In Beam SQL, a DATABASE refers to a + * {@link BeamCalciteSchema}. + */ @SuppressWarnings({"keyfor", "nullness"}) // TODO(https://github.com/apache/beam/issues/20497) public class BeamCalciteSchema implements Schema { private JdbcConnection connection; @@ -45,7 +49,6 @@ public class BeamCalciteSchema implements Schema { /** Creates a {@link BeamCalciteSchema} representing a {@link TableProvider}. */ BeamCalciteSchema(String name, JdbcConnection jdbcConnection, TableProvider tableProvider) { - System.out.println("xxx [BeamCalciteSchema] init: " + tableProvider.getTableType()); this.connection = jdbcConnection; this.tableProvider = tableProvider; this.subSchemas = new HashMap<>(); diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogManagerSchema.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogManagerSchema.java index 4b0d2025ce7d..630979efa00f 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogManagerSchema.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogManagerSchema.java @@ -46,15 +46,16 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +/** + * A Calcite {@link Schema} that corresponds to a {@link CatalogManager}. This is typically the root + * node of a pipeline. Child schemas are of type {@link CatalogSchema}. + */ public class CatalogManagerSchema implements Schema { private static final Logger LOG = LoggerFactory.getLogger(CatalogManagerSchema.class); private final JdbcConnection connection; private final CatalogManager catalogManager; private final Map catalogSubSchemas = new HashMap<>(); - /** - * Creates a Calcite {@link Schema} representing a {@link CatalogManager}. This will typically be - * the root node of a pipeline. - */ + CatalogManagerSchema(JdbcConnection jdbcConnection, CatalogManager catalogManager) { this.connection = jdbcConnection; this.catalogManager = catalogManager; @@ -156,8 +157,7 @@ public Set getTableNames() { } public CatalogSchema getCatalogSchema(TableName tablePath) { - @Nullable - Schema catalogSchema = tablePath.catalog() != null ? getSubSchema(tablePath.catalog()) : null; + @Nullable Schema catalogSchema = getSubSchema(tablePath.catalog()); if (catalogSchema == null) { catalogSchema = getCurrentCatalogSchema(); } @@ -178,7 +178,10 @@ public CatalogSchema getCurrentCatalogSchema() { } @Override - public @Nullable Schema getSubSchema(String name) { + public @Nullable Schema getSubSchema(@Nullable String name) { + if (name == null) { + return null; + } @Nullable CatalogSchema catalogSchema = catalogSubSchemas.get(name); if (catalogSchema == null) { @Nullable Catalog catalog = catalogManager.getCatalog(name); @@ -190,7 +193,9 @@ public CatalogSchema getCurrentCatalogSchema() { if (catalogSchema != null) { return catalogSchema; } - // name could be referring to an underlying metastore. + + // ** Backwards compatibility ** + // Name could be referring to a BeamCalciteSchema. // Attempt to fetch from current catalog return getCurrentCatalogSchema().getSubSchema(name); } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogSchema.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogSchema.java index 06fbf401401f..2028c8e59d0d 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogSchema.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogSchema.java @@ -43,7 +43,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -// TODO: CONSOLIDATE THIS CLASS WITH BeamCalciteSchema +/** + * A Calcite {@link Schema} that corresponds to a {@link Catalog}. Child schemas are of type {@link + * BeamCalciteSchema}. + */ public class CatalogSchema implements Schema { private static final Logger LOG = LoggerFactory.getLogger(CatalogSchema.class); private final JdbcConnection connection; From 928624d6864f6e520e41aaee30d0c5c54e953a83 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Tue, 5 Aug 2025 15:40:26 -0400 Subject: [PATCH 03/11] spotless --- .../java/org/apache/beam/sdk/extensions/sql/impl/TableName.java | 2 +- .../beam/sdk/extensions/sql/impl/parser/SqlCreateDatabase.java | 2 +- .../beam/sdk/extensions/sql/impl/parser/SqlDropDatabase.java | 2 +- .../beam/sdk/extensions/sql/impl/parser/SqlUseDatabase.java | 2 +- .../sql/meta/provider/iceberg/BeamSqlCliIcebergTest.java | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/TableName.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/TableName.java index 06ca2c7f6694..53d8debaaf95 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/TableName.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/TableName.java @@ -22,13 +22,13 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; -import com.google.api.client.util.Lists; import com.google.auto.value.AutoValue; import java.util.Collections; import java.util.List; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.checkerframework.checker.nullness.qual.Nullable; import org.checkerframework.dataflow.qual.Pure; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateDatabase.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateDatabase.java index 7f050cd28073..877b6721152c 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateDatabase.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateDatabase.java @@ -21,7 +21,6 @@ import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Static.RESOURCE; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; -import com.google.api.client.util.Lists; import java.util.List; import org.apache.beam.sdk.extensions.sql.impl.CatalogManagerSchema; import org.apache.beam.sdk.extensions.sql.impl.CatalogSchema; @@ -40,6 +39,7 @@ import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Pair; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.checkerframework.checker.nullness.qual.Nullable; public class SqlCreateDatabase extends SqlCreate implements BeamSqlParser.ExecutableStatement { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropDatabase.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropDatabase.java index 6e08ee8e85fe..4b838c9f4182 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropDatabase.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropDatabase.java @@ -19,7 +19,6 @@ import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Static.RESOURCE; -import com.google.api.client.util.Lists; import java.util.List; import org.apache.beam.sdk.extensions.sql.impl.CatalogManagerSchema; import org.apache.beam.sdk.extensions.sql.impl.CatalogSchema; @@ -39,6 +38,7 @@ import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Pair; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; public class SqlDropDatabase extends SqlDrop implements BeamSqlParser.ExecutableStatement { private static final SqlOperator OPERATOR = diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlUseDatabase.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlUseDatabase.java index 04996849220d..9d06e471dbbe 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlUseDatabase.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlUseDatabase.java @@ -19,7 +19,6 @@ import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Static.RESOURCE; -import com.google.api.client.util.Lists; import java.util.Collections; import java.util.List; import org.apache.beam.sdk.extensions.sql.impl.CatalogManagerSchema; @@ -38,6 +37,7 @@ import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.parser.SqlParserPos; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Pair; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; public class SqlUseDatabase extends SqlSetOption implements BeamSqlParser.ExecutableStatement { private final SqlIdentifier databaseName; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/BeamSqlCliIcebergTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/BeamSqlCliIcebergTest.java index 9d1c0ae0b62c..fcd3c769fbde 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/BeamSqlCliIcebergTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/BeamSqlCliIcebergTest.java @@ -40,6 +40,7 @@ import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.runtime.CalciteContextException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; +import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.DateTime; import org.junit.Before; import org.junit.ClassRule; @@ -47,7 +48,6 @@ import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; -import org.testcontainers.shaded.org.checkerframework.checker.nullness.qual.Nullable; /** UnitTest for {@link BeamSqlCli} using Iceberg catalog. */ public class BeamSqlCliIcebergTest { From d2e328c188d2f037578f3854ef0e267a52ce07df Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Tue, 5 Aug 2025 15:51:52 -0400 Subject: [PATCH 04/11] spotless --- .../main/java/org/apache/beam/sdk/tpcds/BeamSqlEnvRunner.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/BeamSqlEnvRunner.java b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/BeamSqlEnvRunner.java index c8390ab30f6b..fe8db05d2be7 100644 --- a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/BeamSqlEnvRunner.java +++ b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/BeamSqlEnvRunner.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.tpcds; import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; import com.fasterxml.jackson.databind.node.ObjectNode; import java.util.ArrayList; @@ -119,7 +120,7 @@ private static void registerAllTablesByInMemoryMetaStore( .type("text") .build(); Catalog catalog = inMemoryCatalogManager.currentCatalog(); - catalog.metaStore(catalog.currentDatabase()).createTable(table); + catalog.metaStore(checkStateNotNull(catalog.currentDatabase())).createTable(table); } } From bca84a43ee87e077392f085bf47371f20f4996f6 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Wed, 6 Aug 2025 13:35:53 -0400 Subject: [PATCH 05/11] cleanup --- .../beam/sdk/extensions/sql/SqlTransform.java | 11 +----- .../extensions/sql/meta/catalog/Catalog.java | 3 -- .../sql/meta/catalog/CatalogManager.java | 3 -- .../sql/meta/catalog/EmptyCatalogManager.java | 6 --- .../sql/meta/catalog/InMemoryCatalog.java | 6 --- .../meta/catalog/InMemoryCatalogManager.java | 6 --- .../provider/iceberg/IcebergMetastore.java | 5 --- .../sql/meta/store/InMemoryMetaStore.java | 5 --- .../extensions/sql/meta/store/MetaStore.java | 3 -- .../iceberg/BeamSqlCliIcebergTest.java | 38 ++++++++----------- 10 files changed, 17 insertions(+), 69 deletions(-) diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java index 70ed5cdd6d98..8365f56e27de 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java @@ -129,8 +129,6 @@ public abstract class SqlTransform extends PTransform> abstract Map tableProviderMap(); - abstract CatalogManager catalogManager(); - abstract @Nullable String defaultTableProvider(); abstract @Nullable String queryPlannerClassName(); @@ -139,7 +137,7 @@ public abstract class SqlTransform extends PTransform> public PCollection expand(PInput input) { TableProvider inputTableProvider = new ReadOnlyTableProvider(PCOLLECTION_NAME, toTableMap(input)); - CatalogManager catalogManager = catalogManager(); + CatalogManager catalogManager = new InMemoryCatalogManager(); catalogManager.registerTableProvider(inputTableProvider); BeamSqlEnvBuilder sqlEnvBuilder = BeamSqlEnv.builder(catalogManager); @@ -243,10 +241,6 @@ public SqlTransform withDefaultTableProvider(String name, TableProvider tablePro return withTableProvider(name, tableProvider).toBuilder().setDefaultTableProvider(name).build(); } - public SqlTransform withCatalogManager(CatalogManager catalogManager) { - return toBuilder().setCatalogManager(catalogManager).build(); - } - public SqlTransform withQueryPlannerClass(Class clazz) { return toBuilder().setQueryPlannerClassName(clazz.getName()).build(); } @@ -320,7 +314,6 @@ static Builder builder() { .setUdafDefinitions(Collections.emptyList()) .setUdfDefinitions(Collections.emptyList()) .setTableProviderMap(Collections.emptyMap()) - .setCatalogManager(new InMemoryCatalogManager()) .setAutoLoading(true); } @@ -341,8 +334,6 @@ abstract static class Builder { abstract Builder setTableProviderMap(Map tableProviderMap); - abstract Builder setCatalogManager(CatalogManager catalogManager); - abstract Builder setDefaultTableProvider(@Nullable String defaultTableProvider); abstract Builder setQueryPlannerClassName(@Nullable String queryPlannerClassName); diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/Catalog.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/Catalog.java index 825ba8dbd6f6..fd13c4619002 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/Catalog.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/Catalog.java @@ -91,7 +91,4 @@ public interface Catalog { /** Registers this {@link TableProvider} and propagates it to underlying {@link MetaStore}s. */ void registerTableProvider(TableProvider provider); - - /** Clears registered providers from all underlying {@link MetaStore}s. */ - void clearTableProviders(); } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/CatalogManager.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/CatalogManager.java index c2fe7188aad1..c7f8891b2285 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/CatalogManager.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/CatalogManager.java @@ -58,8 +58,5 @@ public interface CatalogManager { */ void registerTableProvider(TableProvider tableProvider); - /** Clears registered providers from all underlying {@link Catalog}s. */ - void clearTableProviders(); - Collection catalogs(); } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/EmptyCatalogManager.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/EmptyCatalogManager.java index fb30691d8eaf..3eaf9c00f16f 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/EmptyCatalogManager.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/EmptyCatalogManager.java @@ -56,12 +56,6 @@ public void registerTableProvider(TableProvider tableProvider) { "ReadOnlyCatalogManager does not support registering a table provider"); } - @Override - public void clearTableProviders() { - throw new UnsupportedOperationException( - "ReadOnlyCatalogManager does not support clearing table providers"); - } - @Override public void createCatalog(String name, String type, Map properties) { throw new UnsupportedOperationException( diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalog.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalog.java index fbd5b386b6fb..1afed99c1e42 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalog.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalog.java @@ -113,10 +113,4 @@ public void registerTableProvider(TableProvider provider) { tableProviders.add(provider); metaStores.values().forEach(m -> m.registerProvider(provider)); } - - @Override - public void clearTableProviders() { - tableProviders.clear(); - metaStores.values().forEach(MetaStore::clearProviders); - } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalogManager.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalogManager.java index d45ee4a0384a..6f9b70007972 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalogManager.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalogManager.java @@ -89,12 +89,6 @@ public void registerTableProvider(TableProvider tableProvider) { tableProviders.add(tableProvider); } - @Override - public void clearTableProviders() { - catalogs.values().forEach(Catalog::clearTableProviders); - tableProviders.clear(); - } - private Catalog findAndCreateCatalog(String name, String type, Map properties) { ImmutableList.Builder list = ImmutableList.builder(); for (CatalogRegistrar catalogRegistrar : diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastore.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastore.java index 4450c6f7dfff..1bac1052dc0f 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastore.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastore.java @@ -141,9 +141,4 @@ public boolean supportsPartitioning(Table table) { public void registerProvider(TableProvider provider) { // no-op } - - @Override - public void clearProviders() { - // no-op - } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStore.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStore.java index 0899d82d928c..10fd2da0d4f6 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStore.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStore.java @@ -119,11 +119,6 @@ private void initTablesFromProvider(TableProvider provider) { this.tables.putAll(tables); } - @Override - public void clearProviders() { - providers.clear(); - } - Map getProviders() { return providers; } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/MetaStore.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/MetaStore.java index 59b855a5a9d8..39ad6d3dfb54 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/MetaStore.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/MetaStore.java @@ -27,7 +27,4 @@ public interface MetaStore extends TableProvider { * @param provider */ void registerProvider(TableProvider provider); - - /** Clears all registered providers. */ - void clearProviders(); } diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/BeamSqlCliIcebergTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/BeamSqlCliIcebergTest.java index fcd3c769fbde..dc6f25c38d0b 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/BeamSqlCliIcebergTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/BeamSqlCliIcebergTest.java @@ -28,7 +28,6 @@ import java.util.UUID; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.extensions.sql.BeamSqlCli; -import org.apache.beam.sdk.extensions.sql.SqlTransform; import org.apache.beam.sdk.extensions.sql.impl.BeamSqlEnv; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils; @@ -203,32 +202,25 @@ public void testCrossCatalogTableWriteAndRead() throws IOException { assertEquals("other_namespace", catalogManager.currentCatalog().currentDatabase()); // insert from old catalog to new table in new catalog + sqlEnv.executeDdl( + "CREATE EXTERNAL TABLE other_table( \n" + + " c_integer INTEGER, \n" + + " c_boolean BOOLEAN, \n" + + " c_timestamp TIMESTAMP, \n" + + " c_varchar VARCHAR) \n" + + "TYPE 'iceberg'\n"); + BeamRelNode insertNode2 = + sqlEnv.parseQuery("INSERT INTO other_table SELECT * FROM catalog_1.my_namespace.my_table"); Pipeline p2 = Pipeline.create(); - p2.apply( - SqlTransform.query("INSERT INTO other_table SELECT * FROM catalog_1.my_namespace.my_table") - .withDdlString( - "CREATE EXTERNAL TABLE other_table( \n" - + " c_integer INTEGER, \n" - + " c_boolean BOOLEAN, \n" - + " c_timestamp TIMESTAMP, \n" - + " c_varchar VARCHAR) \n" - + "TYPE 'iceberg'\n") - .withCatalogManager(catalogManager)); + BeamSqlRelUtils.toPCollection(p2, insertNode2); p2.run().waitUntilFinish(); - // clear PCollection from the above run - catalogManager.clearTableProviders(); - // switch over to catalog 1 and read table inside catalog 2 + sqlEnv.executeDdl("USE DATABASE catalog_1.my_namespace"); + BeamRelNode insertNode3 = + sqlEnv.parseQuery("SELECT * FROM catalog_2.other_namespace.other_table"); Pipeline p3 = Pipeline.create(); - PCollection output = - p3.apply( - SqlTransform.query("SELECT * FROM catalog_2.other_namespace.other_table") - .withDdlString("USE DATABASE catalog_1.my_namespace") - .withCatalogManager(catalogManager)); - p3.run().waitUntilFinish(); - assertEquals("catalog_1", catalogManager.currentCatalog().name()); - assertEquals("my_namespace", catalogManager.currentCatalog().currentDatabase()); + PCollection output = BeamSqlRelUtils.toPCollection(p3, insertNode3); // validate read contents Schema expectedSchema = @@ -240,5 +232,7 @@ public void testCrossCatalogTableWriteAndRead() throws IOException { .addValues(2147483647, true, DateTime.parse("2025-07-31T20:17:40.123Z"), "varchar") .build()); p3.run().waitUntilFinish(); + assertEquals("catalog_1", catalogManager.currentCatalog().name()); + assertEquals("my_namespace", catalogManager.currentCatalog().currentDatabase()); } } From bf8bfffa5ce7977749bdd6fa863a27cb1cd3f469 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Sun, 10 Aug 2025 12:24:14 -0400 Subject: [PATCH 06/11] use databaseExists for efficiency; don't use LOCATION for iceberg tables; fix setOption gap; maybe register table providers from top-level CatalogManager cache --- .../sql/impl/CatalogManagerSchema.java | 50 ++++++++++- .../extensions/sql/impl/CatalogSchema.java | 45 +++------- .../impl/parser/SqlCreateExternalTable.java | 9 +- .../sql/impl/parser/SqlSetOptionBeam.java | 35 +++++--- .../extensions/sql/meta/catalog/Catalog.java | 14 +-- .../sql/meta/catalog/CatalogManager.java | 6 ++ .../sql/meta/catalog/EmptyCatalogManager.java | 5 ++ .../sql/meta/catalog/InMemoryCatalog.java | 22 +++-- .../meta/catalog/InMemoryCatalogManager.java | 13 +-- .../meta/provider/iceberg/IcebergCatalog.java | 19 ++-- .../provider/iceberg/IcebergMetastore.java | 52 ++++++----- .../sql/meta/store/InMemoryMetaStore.java | 4 +- .../extensions/sql/meta/store/MetaStore.java | 6 ++ .../extensions/sql/BeamSqlCliCatalogTest.java | 31 +++---- .../sql/BeamSqlCliDatabaseTest.java | 20 ++--- .../iceberg/BeamSqlCliIcebergTest.java | 8 +- .../iceberg/IcebergMetastoreTest.java | 86 +++++++++++-------- .../provider/iceberg/IcebergReadWriteIT.java | 41 ++++----- .../sql/meta/store/InMemoryMetaStoreTest.java | 8 +- .../sdk/io/iceberg/IcebergCatalogConfig.java | 10 ++- 20 files changed, 276 insertions(+), 208 deletions(-) diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogManagerSchema.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogManagerSchema.java index 5ed8f03e8d88..813daada4906 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogManagerSchema.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogManagerSchema.java @@ -29,6 +29,8 @@ import org.apache.beam.sdk.extensions.sql.impl.parser.SqlDdlNodes; import org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog; import org.apache.beam.sdk.extensions.sql.meta.catalog.CatalogManager; +import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; +import org.apache.beam.sdk.extensions.sql.meta.store.MetaStore; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.linq4j.tree.Expression; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.rel.type.RelProtoDataType; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Function; @@ -137,6 +139,32 @@ public void dropCatalog(SqlIdentifier identifier, boolean ifExists) { catalogSubSchemas.remove(name); } + // A BeamCalciteSchema may be used to interact with multiple TableProviders. + // If such a TableProvider is not registered in the BeamCalciteSchema, this method + // will attempt to do so. + public void maybeRegisterProvider(TableName path, String type) { + CatalogSchema catalogSchema = getCatalogSchema(path); + BeamCalciteSchema beamCalciteSchema = catalogSchema.getDatabaseSchema(path); + + if (beamCalciteSchema.getTableProvider() instanceof MetaStore) { + MetaStore metaStore = (MetaStore) beamCalciteSchema.getTableProvider(); + if (metaStore.tableProviders().containsKey(type)) { + return; + } + + // Start with the narrowest scope. + // Attempt to fetch provider from Catalog first, then CatalogManager. + @Nullable TableProvider provider = catalogSchema.getCatalog().tableProviders().get(type); + if (provider == null) { + provider = catalogManager.tableProviders().get(type); + } + // register provider + if (provider != null) { + metaStore.registerProvider(provider); + } + } + } + @Override public @Nullable Table getTable(String table) { @Nullable @@ -148,10 +176,8 @@ public void dropCatalog(SqlIdentifier identifier, boolean ifExists) { public Set getTableNames() { ImmutableSet.Builder names = ImmutableSet.builder(); // TODO: this might be a heavy operation - for (Catalog catalog : catalogManager.catalogs()) { - for (String db : catalog.listDatabases()) { - names.addAll(catalog.metaStore(db).getTables().keySet()); - } + for (CatalogSchema catalogSchema : catalogSubSchemas.values()) { + names.addAll(catalogSchema.getTableNames()); } return names.build(); } @@ -205,6 +231,22 @@ public Set getSubSchemaNames() { return catalogManager.catalogs().stream().map(Catalog::name).collect(Collectors.toSet()); } + public void setPipelineOption(String key, String value) { + Map options = new HashMap<>(connection.getPipelineOptionsMap()); + options.put(key, value); + connection.setPipelineOptionsMap(options); + } + + public void removePipelineOption(String key) { + Map options = new HashMap<>(connection.getPipelineOptionsMap()); + options.remove(key); + connection.setPipelineOptionsMap(options); + } + + public void removeAllPipelineOptions() { + connection.setPipelineOptionsMap(Collections.emptyMap()); + } + @Override public Set getTypeNames() { return Collections.emptySet(); diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogSchema.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogSchema.java index d1eabb83fc57..792e5b98bcd3 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogSchema.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CatalogSchema.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.extensions.sql.impl; import static java.lang.String.format; +import static org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog.DEFAULT; import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Static.RESOURCE; @@ -59,17 +60,8 @@ public class CatalogSchema implements Schema { CatalogSchema(JdbcConnection jdbcConnection, Catalog catalog) { this.connection = jdbcConnection; this.catalog = catalog; - // try to eagerly populate Calcite sub-schemas with existing databases - try { - catalog - .listDatabases() - .forEach( - database -> - subSchemas.put( - database, - new BeamCalciteSchema(database, connection, catalog.metaStore(database)))); - } catch (Exception ignored) { - } + // should always have a "default" sub-schema available + subSchemas.put(DEFAULT, new BeamCalciteSchema(DEFAULT, connection, catalog.metaStore(DEFAULT))); } public Catalog getCatalog() { @@ -93,7 +85,7 @@ public void createDatabase(SqlIdentifier databaseIdentifier, boolean ifNotExists String name = SqlDdlNodes.name(databaseIdentifier); boolean alreadyExists = subSchemas.containsKey(name); - if (!alreadyExists) { + if (!alreadyExists || name.equals(DEFAULT)) { try { LOG.info("Creating database '{}'", name); if (catalog.createDatabase(name)) { @@ -111,7 +103,7 @@ public void createDatabase(SqlIdentifier databaseIdentifier, boolean ifNotExists if (alreadyExists) { String message = format("Database '%s' already exists.", name); - if (ifNotExists) { + if (ifNotExists || name.equals(DEFAULT)) { LOG.info(message); } else { throw SqlUtil.newContextException( @@ -125,7 +117,7 @@ public void createDatabase(SqlIdentifier databaseIdentifier, boolean ifNotExists public void useDatabase(SqlIdentifier identifier) { String name = SqlDdlNodes.name(identifier); if (!subSchemas.containsKey(name)) { - if (!catalog.listDatabases().contains(name)) { + if (!catalog.databaseExists(name)) { throw SqlUtil.newContextException( identifier.getParserPosition(), RESOURCE.internal(String.format("Cannot use database: '%s' not found.", name))); @@ -184,33 +176,20 @@ public Set getTableNames() { if (name == null) { return null; } - @Nullable BeamCalciteSchema beamCalciteSchema = subSchemas.get(name); - if (beamCalciteSchema == null) { - Set databases; - try { - databases = catalog.listDatabases(); - } catch (Exception ignored) { - return null; - } - if (databases.contains(name)) { - beamCalciteSchema = new BeamCalciteSchema(name, connection, catalog.metaStore(name)); - subSchemas.put(name, beamCalciteSchema); - } + + if (!subSchemas.containsKey(name) && catalog.databaseExists(name)) { + subSchemas.put(name, new BeamCalciteSchema(name, connection, catalog.metaStore(name))); } - return beamCalciteSchema; + return subSchemas.get(name); } private @Nullable BeamCalciteSchema currentDatabase() { - @Nullable String currentDatabase = catalog.currentDatabase(); - if (currentDatabase != null) { - return subSchemas.get(currentDatabase); - } - return null; + return getSubSchema(catalog.currentDatabase()); } @Override public Set getSubSchemaNames() { - return catalog.listDatabases(); + return subSchemas.keySet(); } @Override diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateExternalTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateExternalTable.java index 5ec863c0326e..ab644145b4f7 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateExternalTable.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateExternalTable.java @@ -154,12 +154,13 @@ public void execute(CalcitePrepare.Context context) { Schema schema = pair.left.schema; BeamCalciteSchema beamCalciteSchema; - // String catalogName = "default"; if (schema instanceof CatalogManagerSchema) { TableName pathOverride = TableName.create(name.toString()); - CatalogSchema catalogSchema = ((CatalogManagerSchema) schema).getCatalogSchema(pathOverride); + CatalogManagerSchema catalogManagerSchema = (CatalogManagerSchema) schema; + catalogManagerSchema.maybeRegisterProvider(pathOverride, SqlDdlNodes.getString(type)); + + CatalogSchema catalogSchema = catalogManagerSchema.getCatalogSchema(pathOverride); beamCalciteSchema = catalogSchema.getDatabaseSchema(pathOverride); - // catalogName = catalogSchema.getCatalog().name(); } else if (schema instanceof BeamCalciteSchema) { beamCalciteSchema = (BeamCalciteSchema) schema; } else { @@ -169,8 +170,6 @@ public void execute(CalcitePrepare.Context context) { "Attempting to create a table with unexpected Calcite Schema of type " + schema.getClass())); } - // String databaseName = beamCalciteSchema.name(); - // String tableName = name(name); Table table = toTable(); if (partitionFields != null) { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlSetOptionBeam.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlSetOptionBeam.java index f949a1fc9ae7..338ae8baeb6b 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlSetOptionBeam.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlSetOptionBeam.java @@ -20,8 +20,10 @@ import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.util.Static.RESOURCE; import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema; +import org.apache.beam.sdk.extensions.sql.impl.CatalogManagerSchema; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalcitePrepare; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.jdbc.CalciteSchema; +import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.schema.Schema; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlIdentifier; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlNode; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlSetOption; @@ -44,20 +46,29 @@ public void execute(CalcitePrepare.Context context) { final SqlIdentifier name = getName(); final SqlNode value = getValue(); final Pair pair = SqlDdlNodes.schema(context, true, name); - if (!(pair.left.schema instanceof BeamCalciteSchema)) { + Schema schema = pair.left.schema; + if (schema instanceof CatalogManagerSchema) { + CatalogManagerSchema catalogManagerSchema = (CatalogManagerSchema) schema; + if (value != null) { + catalogManagerSchema.setPipelineOption(pair.right, SqlDdlNodes.getString(value)); + } else if ("ALL".equals(pair.right)) { + catalogManagerSchema.removeAllPipelineOptions(); + } else { + catalogManagerSchema.removePipelineOption(pair.right); + } + } else if (schema instanceof BeamCalciteSchema) { + BeamCalciteSchema beamCalciteSchema = (BeamCalciteSchema) schema; + if (value != null) { + beamCalciteSchema.setPipelineOption(pair.right, SqlDdlNodes.getString(value)); + } else if ("ALL".equals(pair.right)) { + beamCalciteSchema.removeAllPipelineOptions(); + } else { + beamCalciteSchema.removePipelineOption(pair.right); + } + } else { throw SqlUtil.newContextException( name.getParserPosition(), - RESOURCE.internal("Schema is not instanceof BeamCalciteSchema")); - } - - BeamCalciteSchema schema = (BeamCalciteSchema) pair.left.schema; - - if (value != null) { - schema.setPipelineOption(pair.right, SqlDdlNodes.getString(value)); - } else if ("ALL".equals(pair.right)) { - schema.removeAllPipelineOptions(); - } else { - schema.removePipelineOption(pair.right); + RESOURCE.internal("Schema is not instanceof CatalogManagerSchema or BeamCalciteSchema")); } } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/Catalog.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/Catalog.java index fd13c4619002..db7724a4809d 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/Catalog.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/Catalog.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.extensions.sql.meta.catalog; import java.util.Map; -import java.util.Set; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; import org.apache.beam.sdk.extensions.sql.meta.store.MetaStore; @@ -59,12 +58,8 @@ public interface Catalog { */ boolean createDatabase(String databaseName); - /** - * Returns a set of existing databases accessible to this catalog. - * - * @return a set of existing database names - */ - Set listDatabases(); + /** Returns true if the database exists. */ + boolean databaseExists(String db); /** * Switches to use the specified database. @@ -91,4 +86,9 @@ public interface Catalog { /** Registers this {@link TableProvider} and propagates it to underlying {@link MetaStore}s. */ void registerTableProvider(TableProvider provider); + + /** + * Returns all the {@link TableProvider}s available to this {@link Catalog}, organized by type. + */ + Map tableProviders(); } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/CatalogManager.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/CatalogManager.java index c7f8891b2285..808449de5d54 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/CatalogManager.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/CatalogManager.java @@ -58,5 +58,11 @@ public interface CatalogManager { */ void registerTableProvider(TableProvider tableProvider); + /** + * Returns all the {@link TableProvider}s available to this {@link CatalogManager}, organized by + * type. + */ + Map tableProviders(); + Collection catalogs(); } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/EmptyCatalogManager.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/EmptyCatalogManager.java index 3eaf9c00f16f..0fa3dd4d01c1 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/EmptyCatalogManager.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/EmptyCatalogManager.java @@ -56,6 +56,11 @@ public void registerTableProvider(TableProvider tableProvider) { "ReadOnlyCatalogManager does not support registering a table provider"); } + @Override + public Map tableProviders() { + return EMPTY.tableProviders; + } + @Override public void createCatalog(String name, String type, Map properties) { throw new UnsupportedOperationException( diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalog.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalog.java index 1afed99c1e42..0f95a7f14657 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalog.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalog.java @@ -24,7 +24,6 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Map; -import java.util.Set; import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; import org.apache.beam.sdk.extensions.sql.meta.store.InMemoryMetaStore; import org.apache.beam.sdk.extensions.sql.meta.store.MetaStore; @@ -34,7 +33,7 @@ public class InMemoryCatalog implements Catalog { private final String name; private final Map properties; - protected final Set tableProviders = new HashSet<>(); + protected final Map tableProviders = new HashMap<>(); private final Map metaStores = new HashMap<>(); private final HashSet databases = new HashSet<>(Collections.singleton(DEFAULT)); protected @Nullable String currentDatabase = DEFAULT; @@ -65,7 +64,7 @@ public MetaStore metaStore(String db) { @Nullable MetaStore metaStore = metaStores.get(db); if (metaStore == null) { metaStore = new InMemoryMetaStore(); - tableProviders.forEach(metaStore::registerProvider); + tableProviders.values().forEach(metaStore::registerProvider); metaStores.put(db, metaStore); } return metaStore; @@ -81,9 +80,14 @@ public boolean createDatabase(String database) { return databases.add(database); } + @Override + public boolean databaseExists(String db) { + return databases.contains(db); + } + @Override public void useDatabase(String database) { - checkArgument(listDatabases().contains(database), "Database '%s' does not exist."); + checkArgument(databaseExists(database), "Database '%s' does not exist."); currentDatabase = database; } @@ -104,13 +108,13 @@ public boolean dropDatabase(String database, boolean cascade) { } @Override - public Set listDatabases() { - return databases; + public void registerTableProvider(TableProvider provider) { + tableProviders.put(provider.getTableType(), provider); + metaStores.values().forEach(m -> m.registerProvider(provider)); } @Override - public void registerTableProvider(TableProvider provider) { - tableProviders.add(provider); - metaStores.values().forEach(m -> m.registerProvider(provider)); + public Map tableProviders() { + return tableProviders; } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalogManager.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalogManager.java index 6f9b70007972..2cbcb56c49ed 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalogManager.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalogManager.java @@ -22,11 +22,9 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.ServiceLoader; -import java.util.Set; import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; import org.apache.beam.sdk.extensions.sql.meta.store.MetaStore; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; @@ -35,7 +33,7 @@ public class InMemoryCatalogManager implements CatalogManager { private final Map catalogs = new HashMap<>(); - private final Set tableProviders = new HashSet<>(); + private final Map tableProviders = new HashMap<>(); private String currentCatalogName; public InMemoryCatalogManager() { @@ -56,7 +54,7 @@ public void createCatalog(String name, String type, Map properti !catalogs.containsKey(name), "Catalog with name '%s' already exists.", name); Catalog catalog = findAndCreateCatalog(name, type, properties); - tableProviders.forEach(catalog::registerTableProvider); + tableProviders.values().forEach(catalog::registerTableProvider); catalogs.put(name, catalog); } @@ -86,7 +84,12 @@ public void dropCatalog(String name) { @Override public void registerTableProvider(TableProvider tableProvider) { catalogs.values().forEach(catalog -> catalog.registerTableProvider(tableProvider)); - tableProviders.add(tableProvider); + tableProviders.put(tableProvider.getTableType(), tableProvider); + } + + @Override + public Map tableProviders() { + return tableProviders; } private Catalog findAndCreateCatalog(String name, String type, Map properties) { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergCatalog.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergCatalog.java index 6c84339a6a41..0ca38824204b 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergCatalog.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergCatalog.java @@ -21,7 +21,6 @@ import java.util.HashMap; import java.util.Map; -import java.util.Set; import org.apache.beam.sdk.extensions.sql.meta.catalog.InMemoryCatalog; import org.apache.beam.sdk.io.iceberg.IcebergCatalogConfig; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; @@ -60,12 +59,6 @@ public IcebergCatalog(String name, Map properties) { public IcebergMetastore metaStore(String db) { metaStores.putIfAbsent(db, new IcebergMetastore(db, catalogConfig)); return metaStores.get(db); - // @Nullable IcebergMetastore metaStore = metaStores.get(db); - // if (metaStore == null) { - // metaStore = new IcebergMetastore(db, catalogConfig); - // metaStores.put(db, metaStore); - // } - // return metaStore; } @Override @@ -80,10 +73,15 @@ public boolean createDatabase(String database) { @Override public void useDatabase(String database) { - checkArgument(listDatabases().contains(database), "Database '%s' does not exist."); + checkArgument(databaseExists(database), "Database '%s' does not exist."); currentDatabase = database; } + @Override + public boolean databaseExists(String db) { + return catalogConfig.namespaceExists(db); + } + @Override public boolean dropDatabase(String database, boolean cascade) { boolean removed = catalogConfig.dropNamespace(database, cascade); @@ -93,9 +91,4 @@ public boolean dropDatabase(String database, boolean cascade) { } return removed; } - - @Override - public Set listDatabases() { - return catalogConfig.listNamespaces(); - } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastore.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastore.java index 1bac1052dc0f..b73aa25c7a2b 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastore.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastore.java @@ -18,25 +18,26 @@ package org.apache.beam.sdk.extensions.sql.meta.provider.iceberg; import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import java.util.HashMap; import java.util.Map; import org.apache.beam.sdk.extensions.sql.TableUtils; +import org.apache.beam.sdk.extensions.sql.impl.TableName; import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable; import org.apache.beam.sdk.extensions.sql.meta.Table; import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; -import org.apache.beam.sdk.extensions.sql.meta.store.MetaStore; +import org.apache.beam.sdk.extensions.sql.meta.store.InMemoryMetaStore; import org.apache.beam.sdk.io.iceberg.IcebergCatalogConfig; import org.apache.beam.sdk.io.iceberg.IcebergCatalogConfig.IcebergTableInfo; import org.apache.beam.sdk.io.iceberg.TableAlreadyExistsException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.checkerframework.checker.nullness.qual.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class IcebergMetastore implements MetaStore { +public class IcebergMetastore extends InMemoryMetaStore { private static final Logger LOG = LoggerFactory.getLogger(IcebergMetastore.class); @VisibleForTesting final IcebergCatalogConfig catalogConfig; private final Map cachedTables = new HashMap<>(); @@ -54,11 +55,16 @@ public String getTableType() { @Override public void createTable(Table table) { - String identifier = getIdentifier(table); - try { - catalogConfig.createTable(identifier, table.getSchema(), table.getPartitionFields()); - } catch (TableAlreadyExistsException e) { - LOG.info("Iceberg table '{}' already exists at location '{}'.", table.getName(), identifier); + if (!table.getType().equals("iceberg")) { + getProvider(table.getType()).createTable(table); + } else { + String identifier = getIdentifier(table); + try { + catalogConfig.createTable(identifier, table.getSchema(), table.getPartitionFields()); + } catch (TableAlreadyExistsException e) { + LOG.info( + "Iceberg table '{}' already exists at location '{}'.", table.getName(), identifier); + } } cachedTables.put(table.getName(), table); } @@ -80,13 +86,14 @@ public void dropTable(String tableName) { @Override public Map getTables() { for (String id : catalogConfig.listTables(database)) { - String name = Iterables.getLast(Splitter.on(".").split(id)); - if (!cachedTables.containsKey(name)) { + String name = TableName.create(id).getTableName(); + @Nullable Table cachedTable = cachedTables.get(name); + if (cachedTable == null) { Table table = checkStateNotNull(loadTable(id)); cachedTables.put(name, table); } } - return cachedTables; + return ImmutableMap.copyOf(cachedTables); } @Override @@ -106,9 +113,8 @@ private String getIdentifier(String name) { } private String getIdentifier(Table table) { - if (table.getLocation() != null) { - return table.getLocation(); - } + checkArgument( + table.getLocation() == null, "Cannot create Iceberg tables using LOCATION property."); return getIdentifier(table.getName()); } @@ -117,28 +123,32 @@ private String getIdentifier(Table table) { if (tableInfo == null) { return null; } - String name = Iterables.getLast(Splitter.on(".").split(tableInfo.getIdentifier())); return Table.builder() .type(getTableType()) - .name(name) + .name(identifier) .schema(tableInfo.getSchema()) - .location(tableInfo.getIdentifier()) .properties(TableUtils.parseProperties(tableInfo.getProperties())) .build(); } @Override public BeamSqlTable buildBeamSqlTable(Table table) { - return new IcebergTable(getIdentifier(table), table, catalogConfig); + if (table.getType().equals("iceberg")) { + return new IcebergTable(getIdentifier(table), table, catalogConfig); + } + return getProvider(table.getType()).buildBeamSqlTable(table); } @Override public boolean supportsPartitioning(Table table) { - return true; + if (table.getType().equals("iceberg")) { + return true; + } + return getProvider(table.getType()).supportsPartitioning(table); } @Override public void registerProvider(TableProvider provider) { - // no-op + super.registerProvider(provider); } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStore.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStore.java index 10fd2da0d4f6..f896cbae5870 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStore.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStore.java @@ -99,6 +99,7 @@ protected void validateTableType(Table table) { @Override public void registerProvider(TableProvider provider) { + System.out.printf("xxx register provider!!! type: '%s'%n", provider.getTableType()); if (providers.containsKey(provider.getTableType())) { throw new IllegalArgumentException( "Provider is already registered for table type: " + provider.getTableType()); @@ -119,7 +120,8 @@ private void initTablesFromProvider(TableProvider provider) { this.tables.putAll(tables); } - Map getProviders() { + @Override + public Map tableProviders() { return providers; } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/MetaStore.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/MetaStore.java index 39ad6d3dfb54..0315d45420be 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/MetaStore.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/MetaStore.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.extensions.sql.meta.store; +import java.util.Map; import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; /** The interface to handle CRUD of {@code BeamSql} table metadata. */ @@ -27,4 +28,9 @@ public interface MetaStore extends TableProvider { * @param provider */ void registerProvider(TableProvider provider); + + /** + * Returns all the registered {@link TableProvider}s in this {@link MetaStore}, organized by type. + */ + Map tableProviders(); } diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliCatalogTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliCatalogTest.java index d4d6c2319785..7461fb1cf4c0 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliCatalogTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliCatalogTest.java @@ -20,9 +20,11 @@ import static org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog.DEFAULT; import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; import java.util.Map; import org.apache.beam.sdk.extensions.sql.meta.Table; @@ -34,7 +36,6 @@ import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.runtime.CalciteContextException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -186,9 +187,9 @@ public void testCreateUseDropDatabaseWithSameCatalogScope() { assertEquals("catalog_1", catalogManager.currentCatalog().name()); assertEquals(DEFAULT, catalogManager.currentCatalog().currentDatabase()); cli.execute("CREATE DATABASE db_1"); + assertTrue(catalogManager.currentCatalog().databaseExists("db_1")); cli.execute("USE DATABASE db_1"); assertEquals("db_1", catalogManager.currentCatalog().currentDatabase()); - assertEquals(ImmutableSet.of(DEFAULT, "db_1"), catalogManager.currentCatalog().listDatabases()); // create new Catalog catalog_2 and switch to it cli.execute("CREATE CATALOG catalog_2 TYPE 'local'"); @@ -198,13 +199,13 @@ public void testCreateUseDropDatabaseWithSameCatalogScope() { assertEquals(DEFAULT, catalogManager.currentCatalog().currentDatabase()); // confirm that database 'db_1' from catalog_1 is not leaked to catalog_2 - assertEquals(ImmutableSet.of(DEFAULT), catalogManager.currentCatalog().listDatabases()); + assertFalse(catalogManager.currentCatalog().databaseExists("db_1")); // switch back and drop database cli.execute("USE CATALOG catalog_1"); assertEquals("catalog_1", catalogManager.currentCatalog().name()); cli.execute("DROP DATABASE db_1"); - assertEquals(ImmutableSet.of(DEFAULT), catalogManager.currentCatalog().listDatabases()); + assertFalse(catalogManager.currentCatalog().databaseExists("db_1")); } @Test @@ -218,7 +219,7 @@ public void testCreateWriteDropTableWithSameCatalogScope() { // create new database cli.execute("CREATE DATABASE db_1"); cli.execute("USE DATABASE db_1"); - assertEquals(ImmutableSet.of(DEFAULT, "db_1"), catalogManager.currentCatalog().listDatabases()); + assertTrue(catalogManager.currentCatalog().databaseExists("db_1")); MetaStore metastoreDb1 = checkStateNotNull(catalogManager.getCatalog("catalog_1")).metaStore("db_1"); @@ -261,30 +262,24 @@ public void testCreateUseDropDatabaseWithOtherCatalogScope() { assertEquals(DEFAULT, catalogManager.currentCatalog().currentDatabase()); // while using catalog_2, create new database in catalog_1 cli.execute("CREATE DATABASE catalog_1.db_1"); - assertEquals( - ImmutableSet.of(DEFAULT, "db_1"), - checkStateNotNull(catalogManager.getCatalog("catalog_1")).listDatabases()); + assertTrue(checkStateNotNull(catalogManager.getCatalog("catalog_1")).databaseExists("db_1")); // use database in catalog_2. this will override both current database (to 'deb_1') // and current catalog (to 'catalog_1') cli.execute("USE DATABASE catalog_1.db_1"); assertEquals("catalog_1", catalogManager.currentCatalog().name()); assertEquals("db_1", catalogManager.currentCatalog().currentDatabase()); - assertEquals(ImmutableSet.of(DEFAULT, "db_1"), catalogManager.currentCatalog().listDatabases()); + assertTrue(catalogManager.currentCatalog().databaseExists("db_1")); // switch back to catalog_2 and drop cli.execute("USE CATALOG catalog_2"); assertEquals("catalog_2", catalogManager.currentCatalog().name()); // confirm that database 'db_1' created in catalog_1 was not leaked to catalog_2 - assertEquals(ImmutableSet.of(DEFAULT), catalogManager.currentCatalog().listDatabases()); + assertFalse(catalogManager.currentCatalog().databaseExists("db_1")); // drop and validate - assertEquals( - ImmutableSet.of(DEFAULT, "db_1"), - checkStateNotNull(catalogManager.getCatalog("catalog_1")).listDatabases()); + assertTrue(checkStateNotNull(catalogManager.getCatalog("catalog_1")).databaseExists("db_1")); cli.execute("DROP DATABASE catalog_1.db_1"); - assertEquals( - ImmutableSet.of(DEFAULT), - checkStateNotNull(catalogManager.getCatalog("catalog_1")).listDatabases()); + assertFalse(checkStateNotNull(catalogManager.getCatalog("catalog_1")).databaseExists("db_1")); } @Test @@ -299,9 +294,7 @@ public void testCreateWriteDropTableWithOtherCatalogScope() { // while using catalog_2, create new database in catalog_1 cli.execute("CREATE DATABASE catalog_1.db_1"); - assertEquals( - ImmutableSet.of(DEFAULT, "db_1"), - checkStateNotNull(catalogManager.getCatalog("catalog_1")).listDatabases()); + assertTrue(checkStateNotNull(catalogManager.getCatalog("catalog_1")).databaseExists("db_1")); MetaStore metastoreDb1 = checkStateNotNull(catalogManager.getCatalog("catalog_1")).metaStore("db_1"); diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliDatabaseTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliDatabaseTest.java index a12b51895210..3cfae6f67461 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliDatabaseTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliDatabaseTest.java @@ -19,8 +19,10 @@ import static org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog.DEFAULT; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; import org.apache.beam.sdk.extensions.sql.meta.Table; import org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog; @@ -29,7 +31,6 @@ import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.runtime.CalciteContextException; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -50,8 +51,7 @@ public void setupCli() { @Test public void testCreateDatabase() { cli.execute("CREATE DATABASE my_database"); - assertEquals( - ImmutableSet.of(DEFAULT, "my_database"), catalogManager.currentCatalog().listDatabases()); + assertTrue(catalogManager.currentCatalog().databaseExists("my_database")); } @Test @@ -66,8 +66,7 @@ public void testCreateDuplicateDatabase_error() { public void testCreateDuplicateDatabase_ifNotExists() { cli.execute("CREATE DATABASE my_database"); cli.execute("CREATE DATABASE IF NOT EXISTS my_database"); - assertEquals( - ImmutableSet.of(DEFAULT, "my_database"), catalogManager.currentCatalog().listDatabases()); + assertTrue(catalogManager.currentCatalog().databaseExists("my_database")); } @Test @@ -93,15 +92,14 @@ public void testUseDatabase_doesNotExist() { @Test public void testDropDatabase() { cli.execute("CREATE DATABASE my_database"); - assertEquals( - ImmutableSet.of(DEFAULT, "my_database"), catalogManager.currentCatalog().listDatabases()); + assertTrue(catalogManager.currentCatalog().databaseExists("my_database")); cli.execute("DROP DATABASE my_database"); - assertEquals(ImmutableSet.of(DEFAULT), catalogManager.currentCatalog().listDatabases()); + assertFalse(catalogManager.currentCatalog().databaseExists("my_database")); } @Test public void testDropDatabase_nonexistent() { - assertEquals(ImmutableSet.of(DEFAULT), catalogManager.currentCatalog().listDatabases()); + assertFalse(catalogManager.currentCatalog().databaseExists("my_database")); thrown.expect(CalciteContextException.class); thrown.expectMessage("Database 'my_database' does not exist."); cli.execute("DROP DATABASE my_database"); @@ -112,9 +110,9 @@ public void testCreateInsertDropTableUsingDefaultDatabase() { Catalog catalog = catalogManager.currentCatalog(); // create new database db_1 cli.execute("CREATE DATABASE db_1"); + assertTrue(catalog.databaseExists("db_1")); cli.execute("USE DATABASE db_1"); assertEquals("db_1", catalog.currentDatabase()); - assertEquals(ImmutableSet.of(DEFAULT, "db_1"), catalog.listDatabases()); // create new table TestTableProvider testTableProvider = new TestTableProvider(); @@ -152,7 +150,7 @@ public void testCreateInsertDropTableUsingOtherDatabase() { cli.execute("CREATE DATABASE db_1"); cli.execute("USE DATABASE db_1"); assertEquals("db_1", catalog.currentDatabase()); - assertEquals(ImmutableSet.of(DEFAULT, "db_1"), catalog.listDatabases()); + assertTrue(catalog.databaseExists("db_1")); // switch to other database db_2 cli.execute("CREATE DATABASE db_2"); diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/BeamSqlCliIcebergTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/BeamSqlCliIcebergTest.java index dc6f25c38d0b..9ac96652d340 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/BeamSqlCliIcebergTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/BeamSqlCliIcebergTest.java @@ -20,6 +20,7 @@ import static java.lang.String.format; import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; @@ -38,7 +39,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.runtime.CalciteContextException; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.DateTime; import org.junit.Before; @@ -102,11 +102,11 @@ public void testCreateNamespace() { IcebergCatalog catalog = (IcebergCatalog) catalogManager.currentCatalog(); assertEquals("default", catalog.currentDatabase()); cli.execute("CREATE DATABASE new_namespace"); - assertEquals("new_namespace", Iterables.getOnlyElement(catalog.listDatabases())); + assertTrue(catalog.databaseExists("new_namespace")); // Specifies IF NOT EXISTS, so should be a no-op cli.execute("CREATE DATABASE IF NOT EXISTS new_namespace"); - assertEquals("new_namespace", Iterables.getOnlyElement(catalog.listDatabases())); + assertTrue(catalog.databaseExists("new_namespace")); // This one doesn't, so it should throw an error. thrown.expect(CalciteContextException.class); @@ -145,7 +145,7 @@ public void testDropNamespace() { cli.execute("USE DATABASE new_namespace"); assertEquals("new_namespace", catalog.currentDatabase()); cli.execute("DROP DATABASE new_namespace"); - assertTrue(catalog.listDatabases().isEmpty()); + assertFalse(catalog.databaseExists("new_namespace")); assertNull(catalog.currentDatabase()); // Drop non-existent namespace with IF EXISTS diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastoreTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastoreTest.java index 6247cefc8072..a7baf1191d15 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastoreTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastoreTest.java @@ -17,67 +17,81 @@ */ package org.apache.beam.sdk.extensions.sql.meta.provider.iceberg; -import static org.apache.beam.sdk.extensions.sql.meta.provider.iceberg.IcebergTable.TRIGGERING_FREQUENCY_FIELD; -import static org.apache.beam.sdk.schemas.Schema.toSchema; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; -import java.util.stream.Stream; -import org.apache.beam.sdk.extensions.sql.TableUtils; +import java.io.File; +import java.io.IOException; +import java.util.UUID; import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable; import org.apache.beam.sdk.extensions.sql.meta.Table; import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.vendor.calcite.v1_40_0.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; +import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; +import org.junit.rules.TemporaryFolder; /** UnitTest for {@link IcebergMetastore}. */ public class IcebergMetastoreTest { - private final IcebergCatalog catalog = - new IcebergCatalog( - "test_catalog", - ImmutableMap.of( - "catalog-impl", "org.apache.iceberg.gcp.bigquery.BigQueryMetastoreCatalog", - "io-impl", "org.apache.iceberg.gcp.gcs.GCSFileIO", - "warehouse", "gs://bucket/warehouse", - "beam.catalog.test_catalog.hadoop.fs.gs.project.id", "apache-beam-testing", - "beam.catalog.test_catalog.hadoop.foo", "bar")); + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + private IcebergCatalog catalog; + + @Before + public void setup() throws IOException { + File warehouseFile = TEMPORARY_FOLDER.newFolder(); + assertTrue(warehouseFile.delete()); + String warehouse = "file:" + warehouseFile + "/" + UUID.randomUUID(); + catalog = + new IcebergCatalog( + "test_catalog", ImmutableMap.of("type", "hadoop", "warehouse", warehouse)); + } + + private IcebergMetastore metastore() { + return catalog.metaStore(catalog.currentDatabase()); + } @Test public void testGetTableType() { - assertEquals("iceberg", catalog.metaStore(catalog.currentDatabase()).getTableType()); + assertEquals("iceberg", metastore().getTableType()); } @Test - public void testBuildBeamSqlTable() throws Exception { - ImmutableMap properties = ImmutableMap.of(TRIGGERING_FREQUENCY_FIELD, 30); - - ObjectMapper mapper = new ObjectMapper(); - String propertiesString = mapper.writeValueAsString(properties); - Table table = - fakeTableBuilder("my_table") - .properties(TableUtils.parseProperties(propertiesString)) - .build(); - BeamSqlTable sqlTable = catalog.metaStore(catalog.currentDatabase()).buildBeamSqlTable(table); + public void testBuildBeamSqlTable() { + Table table = Table.builder().name("my_table").schema(Schema.of()).type("iceberg").build(); + BeamSqlTable sqlTable = metastore().buildBeamSqlTable(table); assertNotNull(sqlTable); assertTrue(sqlTable instanceof IcebergTable); IcebergTable icebergTable = (IcebergTable) sqlTable; - assertEquals("namespace.my_table", icebergTable.tableIdentifier); + assertEquals(catalog.currentDatabase() + ".my_table", icebergTable.tableIdentifier); assertEquals(catalog.catalogConfig, icebergTable.catalogConfig); } - private static Table.Builder fakeTableBuilder(String name) { - return Table.builder() - .name(name) - .location("namespace." + name) - .schema( - Stream.of( - Schema.Field.nullable("id", Schema.FieldType.INT32), - Schema.Field.nullable("name", Schema.FieldType.STRING)) - .collect(toSchema())) - .type("iceberg"); + @Test + public void testCreateTable() { + Table table = Table.builder().name("my_table").schema(Schema.of()).type("iceberg").build(); + metastore().createTable(table); + + assertNotNull(catalog.catalogConfig.loadTable(catalog.currentDatabase() + ".my_table")); + } + + @Test + public void testGetTables() { + Table table1 = Table.builder().name("my_table_1").schema(Schema.of()).type("iceberg").build(); + Table table2 = Table.builder().name("my_table_2").schema(Schema.of()).type("iceberg").build(); + metastore().createTable(table1); + metastore().createTable(table2); + + assertEquals(ImmutableSet.of("my_table_1", "my_table_2"), metastore().getTables().keySet()); + } + + @Test + public void testSupportsPartitioning() { + Table table = Table.builder().name("my_table_1").schema(Schema.of()).type("iceberg").build(); + assertTrue(metastore().supportsPartitioning(table)); } } diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java index 98238336bd14..c0e8c6c7d726 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java @@ -44,6 +44,7 @@ import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.extensions.sql.impl.BeamSqlEnv; +import org.apache.beam.sdk.extensions.sql.impl.TableName; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamPushDownIOSourceRel; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils; @@ -140,6 +141,7 @@ public void runSqlWriteAndRead(boolean withPartitionFields) .setPipelineOptions(PipelineOptionsFactory.create()) .build(); String tableIdentifier = DATASET + "." + testName.getMethodName(); + String tableName = TableName.create(tableIdentifier).getTableName(); // 1) create Iceberg catalog String createCatalog = @@ -153,9 +155,9 @@ public void runSqlWriteAndRead(boolean withPartitionFields) + " 'gcp_region' = 'us-central1')"; sqlEnv.executeDdl(createCatalog); - // 2) use the catalog we just created - String setCatalog = "USE CATALOG my_catalog"; - sqlEnv.executeDdl(setCatalog); + // 2) use the catalog we just created and dataset + sqlEnv.executeDdl("USE CATALOG my_catalog"); + sqlEnv.executeDdl("USE DATABASE " + DATASET); // 3) create beam table String partitionFields = @@ -163,7 +165,7 @@ public void runSqlWriteAndRead(boolean withPartitionFields) ? "PARTITIONED BY ('bucket(c_integer, 5)', 'c_boolean', 'hour(c_timestamp)', 'truncate(c_varchar, 3)') \n" : ""; String createTableStatement = - "CREATE EXTERNAL TABLE TEST( \n" + format("CREATE EXTERNAL TABLE %s( \n", tableName) + " c_bigint BIGINT, \n" + " c_integer INTEGER, \n" + " c_float FLOAT, \n" @@ -176,10 +178,7 @@ public void runSqlWriteAndRead(boolean withPartitionFields) + " c_arr_struct ARRAY, c_arr_struct_integer INTEGER>> \n" + ") \n" + "TYPE 'iceberg' \n" - + partitionFields - + "LOCATION '" - + tableIdentifier - + "'"; + + partitionFields; sqlEnv.executeDdl(createTableStatement); // 3) verify a real Iceberg table was created, with the right partition spec @@ -201,12 +200,12 @@ public void runSqlWriteAndRead(boolean withPartitionFields) assertEquals("my_catalog." + tableIdentifier, icebergTable.name()); assertTrue(icebergTable.location().startsWith(warehouse)); assertEquals(expectedSpec, icebergTable.spec()); - Schema expectedSchema = checkStateNotNull(metastore.getTable("TEST")).getSchema(); + Schema expectedSchema = checkStateNotNull(metastore.getTable(tableName)).getSchema(); assertEquals(expectedSchema, IcebergUtils.icebergSchemaToBeamSchema(icebergTable.schema())); // 4) write to underlying Iceberg table String insertStatement = - "INSERT INTO TEST VALUES (" + format("INSERT INTO %s VALUES (", tableName) + "9223372036854775807, " + "2147483647, " + "1.0, " @@ -249,7 +248,7 @@ public void runSqlWriteAndRead(boolean withPartitionFields) assertEquals(expectedRow, beamRow); // 6) read using Beam SQL and verify - String selectTableStatement = "SELECT * FROM TEST"; + String selectTableStatement = "SELECT * FROM " + tableName; PCollection output = BeamSqlRelUtils.toPCollection(readPipeline, sqlEnv.parseQuery(selectTableStatement)); PAssert.that(output).containsInAnyOrder(expectedRow); @@ -257,7 +256,7 @@ public void runSqlWriteAndRead(boolean withPartitionFields) assertThat(state, equalTo(PipelineResult.State.DONE)); // 7) cleanup - sqlEnv.executeDdl("DROP TABLE TEST"); + sqlEnv.executeDdl("DROP TABLE " + tableName); assertFalse(icebergCatalog.tableExists(TableIdentifier.parse(tableIdentifier))); } @@ -268,6 +267,7 @@ public void testSQLReadWithProjectAndFilterPushDown() { .setPipelineOptions(PipelineOptionsFactory.create()) .build(); String tableIdentifier = DATASET + "." + testName.getMethodName(); + String tableName = TableName.create(tableIdentifier).getTableName(); // 1) create Iceberg catalog String createCatalog = @@ -281,28 +281,25 @@ public void testSQLReadWithProjectAndFilterPushDown() { + " 'gcp_region' = 'us-central1')"; sqlEnv.executeDdl(createCatalog); - // 2) use the catalog we just created - String setCatalog = "USE CATALOG my_catalog"; - sqlEnv.executeDdl(setCatalog); + // 2) use the catalog we just created and the dataset + sqlEnv.executeDdl("USE CATALOG my_catalog"); + sqlEnv.executeDdl("USE DATABASE " + DATASET); // 3) create Beam table String createTableStatement = - "CREATE EXTERNAL TABLE TEST( \n" + format("CREATE EXTERNAL TABLE %s( \n", tableName) + " c_integer INTEGER, \n" + " c_float FLOAT, \n" + " c_boolean BOOLEAN, \n" + " c_timestamp TIMESTAMP, \n" + " c_varchar VARCHAR \n " + ") \n" - + "TYPE 'iceberg' \n" - + "LOCATION '" - + tableIdentifier - + "'"; + + "TYPE 'iceberg'"; sqlEnv.executeDdl(createTableStatement); // 4) insert some data) String insertStatement = - "INSERT INTO TEST VALUES " + format("INSERT INTO %s VALUES ", tableName) + "(123, 1.23, TRUE, TIMESTAMP '2025-05-22 20:17:40.123', 'a'), " + "(456, 4.56, FALSE, TIMESTAMP '2025-05-25 20:17:40.123', 'b'), " + "(789, 7.89, TRUE, TIMESTAMP '2025-05-28 20:17:40.123', 'c')"; @@ -311,7 +308,7 @@ public void testSQLReadWithProjectAndFilterPushDown() { // 5) read with a filter String selectTableStatement = - "SELECT c_integer, c_varchar FROM TEST where " + format("SELECT c_integer, c_varchar FROM %s where ", tableName) + "(c_boolean=TRUE and c_varchar in ('a', 'b')) or c_float > 5"; BeamRelNode relNode = sqlEnv.parseQuery(selectTableStatement); PCollection output = BeamSqlRelUtils.toPCollection(readPipeline, relNode); diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStoreTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStoreTest.java index f90c90588983..ea41490c8d00 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStoreTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStoreTest.java @@ -94,10 +94,10 @@ public void testBuildBeamSqlTable() throws Exception { @Test public void testRegisterProvider() throws Exception { store.registerProvider(new MockTableProvider("mock", "hello", "world")); - assertNotNull(store.getProviders()); - assertEquals(2, store.getProviders().size()); - assertEquals("text", store.getProviders().get("text").getTableType()); - assertEquals("mock", store.getProviders().get("mock").getTableType()); + assertNotNull(store.tableProviders()); + assertEquals(2, store.tableProviders().size()); + assertEquals("text", store.tableProviders().get("text").getTableType()); + assertEquals("mock", store.tableProviders().get("mock").getTableType()); assertEquals(2, store.getTables().size()); } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCatalogConfig.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCatalogConfig.java index 48a831a2d902..7603e2c6259f 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCatalogConfig.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCatalogConfig.java @@ -111,6 +111,11 @@ public boolean createNamespace(String namespace) { } } + public boolean namespaceExists(String namespace) { + checkSupportsNamespaces(); + return ((SupportsNamespaces) catalog()).namespaceExists(Namespace.of(namespace)); + } + public Set listNamespaces() { checkSupportsNamespaces(); @@ -143,12 +148,13 @@ public void createTable( org.apache.iceberg.Schema icebergSchema = IcebergUtils.beamSchemaToIcebergSchema(tableSchema); PartitionSpec icebergSpec = PartitionUtils.toPartitionSpec(partitionFields, tableSchema); try { - catalog().createTable(icebergIdentifier, icebergSchema, icebergSpec); LOG.info( - "Created table '{}' with schema: {}\n, partition spec: {}", + "Attempting to create table '{}', with schema: {}, partition spec: {}.", icebergIdentifier, icebergSchema, icebergSpec); + catalog().createTable(icebergIdentifier, icebergSchema, icebergSpec); + LOG.info("Successfully created table '{}'.", icebergIdentifier); } catch (AlreadyExistsException e) { throw new TableAlreadyExistsException(e); } From 40ee916300fa33975d861f19a24fb297e0f74a67 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Sun, 10 Aug 2025 13:45:56 -0400 Subject: [PATCH 07/11] fix postcommits --- .../sql/meta/store/InMemoryMetaStore.java | 1 - .../sdk/extensions/sql/PubsubToIcebergIT.java | 18 ++++++------------ 2 files changed, 6 insertions(+), 13 deletions(-) diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStore.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStore.java index f896cbae5870..eb293a3c11f7 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStore.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStore.java @@ -99,7 +99,6 @@ protected void validateTableType(Table table) { @Override public void registerProvider(TableProvider provider) { - System.out.printf("xxx register provider!!! type: '%s'%n", provider.getTableType()); if (providers.containsKey(provider.getTableType())) { throw new IllegalArgumentException( "Provider is already registered for table type: " + provider.getTableType()); diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/PubsubToIcebergIT.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/PubsubToIcebergIT.java index 7343c9b9a52f..96aeda2111f6 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/PubsubToIcebergIT.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/PubsubToIcebergIT.java @@ -149,18 +149,15 @@ public void testSimpleInsertWithPartitionedFields() throws Exception { + "' \n" + "TBLPROPERTIES '{ \"timestampAttributeKey\" : \"ts\" }'"; String icebergTableString = - "CREATE EXTERNAL TABLE iceberg_table( \n" + format("CREATE EXTERNAL TABLE %s( \n", tableIdentifier) + " id BIGINT, \n" + " name VARCHAR \n " + ") \n" + "TYPE 'iceberg' \n" + "PARTITIONED BY('id', 'truncate(name, 3)') \n" - + "LOCATION '" - + tableIdentifier - + "' \n" + "TBLPROPERTIES '{ \"triggering_frequency_seconds\" : 10 }'"; String insertStatement = - "INSERT INTO iceberg_table \n" + format("INSERT INTO %s \n", tableIdentifier) + "SELECT \n" + " pubsub_topic.payload.id, \n" + " pubsub_topic.payload.name \n" @@ -207,18 +204,15 @@ public void testSimpleInsertFlat() throws Exception { + pubsub.topicPath() + "' \n" + "TBLPROPERTIES '{ \"timestampAttributeKey\" : \"ts\" }'"; - String bqTableString = - "CREATE EXTERNAL TABLE iceberg_table( \n" + String icebergTableString = + format("CREATE EXTERNAL TABLE %s( \n", tableIdentifier) + " id BIGINT, \n" + " name VARCHAR \n " + ") \n" + "TYPE 'iceberg' \n" - + "LOCATION '" - + tableIdentifier - + "' \n" + "TBLPROPERTIES '{ \"triggering_frequency_seconds\" : 10 }'"; String insertStatement = - "INSERT INTO iceberg_table \n" + format("INSERT INTO %s \n", tableIdentifier) + "SELECT \n" + " id, \n" + " name \n" @@ -229,7 +223,7 @@ public void testSimpleInsertFlat() throws Exception { .withDdlString(createCatalogDdl) .withDdlString(setCatalogDdl) .withDdlString(pubsubTableString) - .withDdlString(bqTableString)); + .withDdlString(icebergTableString)); pipeline.run(); // Block until a subscription for this topic exists From cb2fcc5148121a1900ecc10fbd6cde3c9e479b3f Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Wed, 24 Sep 2025 15:20:03 -0400 Subject: [PATCH 08/11] reset --- .asf.yaml | 2 + .editorconfig | 30 + .../build-push-docker-action/action.yml | 45 + .github/actions/dind-up-action/action.yml | 275 ++ .github/dependabot.yml | 4 - .../IO_Iceberg_Integration_Tests.json | 2 +- ...IO_Iceberg_Integration_Tests_Dataflow.json | 2 +- ...rg_Managed_Integration_Tests_Dataflow.json | 2 +- .../trigger_files/beam_PostCommit_Python.json | 2 +- ...ommit_Python_ValidatesRunner_Dataflow.json | 4 + ..._PostCommit_Python_Xlang_Gcp_Dataflow.json | 2 +- ...am_PostCommit_Python_Xlang_Gcp_Direct.json | 2 +- ...eam_PostCommit_Python_Xlang_IO_Direct.json | 2 +- .../trigger_files/beam_PostCommit_SQL.json | 2 +- .../beam_PostCommit_XVR_Direct.json | 2 +- .../beam_PostCommit_XVR_Flink.json | 2 +- ...m_PostCommit_XVR_GoUsingJava_Dataflow.json | 4 + .../beam_PostCommit_Yaml_Xlang_Direct.json | 2 +- .../beam_IODatastoresCredentialsRotation.yml | 28 +- ...m_Inference_Python_Benchmarks_Dataflow.yml | 26 +- .../beam_Infrastructure_PolicyEnforcer.yml | 83 + .../beam_Infrastructure_SecurityLogging.yml | 77 + ...beam_Infrastructure_ServiceAccountKeys.yml | 68 + ... beam_Infrastructure_UsersPermissions.yml} | 0 .../beam_MetricsCredentialsRotation.yml | 28 +- .github/workflows/beam_Metrics_Report.yml | 26 +- ...m_PostCommit_Java_BigQueryEarlyRollout.yml | 28 +- .github/workflows/beam_PostCommit_Python.yml | 23 +- .../workflows/beam_PostCommit_Python_Arm.yml | 2 +- ...m_PostCommit_Python_Xlang_Gcp_Dataflow.yml | 4 +- ...am_PostCommit_XVR_GoUsingJava_Dataflow.yml | 8 +- .../beam_PreCommit_Java_Kafka_IO_Direct.yml | 2 +- .../beam_PreCommit_Java_Pulsar_IO_Direct.yml | 29 +- .github/workflows/beam_PreCommit_Python.yml | 46 +- .../beam_PreCommit_Python_Coverage.yml | 56 +- .../beam_PreCommit_Python_Examples.yml | 3 +- .../workflows/beam_PreCommit_Python_ML.yml | 22 +- .../beam_PreCommit_Python_Transforms.yml | 2 +- .../workflows/beam_PreCommit_Whitespace.yml | 5 + .../beam_Publish_Beam_SDK_Snapshots.yml | 2 +- ...Python_ValidatesContainer_Dataflow_ARM.yml | 2 +- .../beam_StressTests_Java_KafkaIO.yml | 2 +- .github/workflows/build_release_candidate.yml | 81 +- .github/workflows/build_runner_image.yml | 4 +- .github/workflows/finalize_release.yml | 6 +- ...n_Benchmarks_Dataflow_VLLM_Gemma_Batch.txt | 36 + .github/workflows/pr-bot-new-prs.yml | 2 +- .github/workflows/pr-bot-pr-updates.yml | 2 +- .../pr-bot-prs-needing-attention.yml | 2 +- .github/workflows/refresh_looker_metrics.yml | 2 +- .github/workflows/reportGenerator.yml | 2 +- .../republish_released_docker_containers.yml | 4 +- .github/workflows/typescript_tests.yml | 6 +- .test-infra/tools/refresh_looker_metrics.py | 1 + CHANGES.md | 238 +- build.gradle.kts | 271 ++ buildSrc/build.gradle.kts | 2 +- .../beam/gradle/BeamModulePlugin.groovy | 188 +- contributor-docs/code-change-guide.md | 36 +- contributor-docs/discussion-docs/2025.md | 17 +- contributor-docs/release-guide.md | 19 +- examples/java/build.gradle | 2 - examples/java/iceberg/build.gradle | 89 + .../iceberg/IcebergBatchWriteExample.java | 208 ++ .../IcebergRestCatalogCDCExample.java | 2 +- ...ebergRestCatalogStreamingWriteExample.java | 2 +- .../iceberg}/IcebergTaxiExamples.java | 2 +- .../complete/StreamingWordExtract.java | 12 +- .../datatokenization/utils/SchemasUtils.java | 2 +- .../alloydb_product_catalog_embeddings.ipynb | 2 +- .../anomaly_detection_iforest.ipynb | 2 +- .../anomaly_detection_timesfm.ipynb | 2712 ++++++++++++++++ .../beam-ml/automatic_model_refresh.ipynb | 2 +- ...bigquery_vector_ingestion_and_search.ipynb | 2 +- ...sql_mysql_product_catalog_embeddings.ipynb | 2785 +++++++++++++++++ ..._postgres_product_catalog_embeddings.ipynb | 2 +- .../compute_and_apply_vocab.ipynb | 2 +- .../huggingface_text_embeddings.ipynb | 32 +- .../data_preprocessing/scale_data.ipynb | 2 +- .../vertex_ai_text_embeddings.ipynb | 2 +- .../beam-ml/dataflow_tpu_examples.ipynb | 744 +++++ .../gemma_2_sentiment_and_summarization.ipynb | 4 +- .../beam-ml/image_processing_tensorflow.ipynb | 3 +- .../notebooks/beam-ml/mltransform_basic.ipynb | 2 +- .../notebooks/beam-ml/per_key_models.ipynb | 2 +- .../rag_usecase/beam_rag_notebook.ipynb | 2 +- .../rag_usecase/opensearch_rag_pipeline.ipynb | 2 +- .../beam-ml/run_inference_gemma.ipynb | 2 +- .../beam-ml/run_inference_generative_ai.ipynb | 2 +- .../beam-ml/run_inference_multi_model.ipynb | 2 +- .../beam-ml/run_inference_pytorch.ipynb | 2 +- ...inference_pytorch_tensorflow_sklearn.ipynb | 6 +- .../beam-ml/run_inference_tensorflow.ipynb | 2 +- .../run_inference_tensorflow_with_tfx.ipynb | 2 +- .../beam-ml/run_inference_vertex_ai.ipynb | 2 +- .../run_inference_with_tensorflow_hub.ipynb | 2 +- .../beam-ml/speech_emotion_tensorflow.ipynb | 2 +- .../notebooks/blog/unittests_in_beam.ipynb | 2 +- examples/yaml/README.md | 54 + gradle.properties | 4 +- infra/enforcement/README.md | 224 ++ infra/enforcement/account_keys.py | 523 ++++ infra/enforcement/config.yml | 38 + infra/enforcement/iam.py | 400 +++ infra/enforcement/requirements.txt | 24 + infra/enforcement/sending.py | 179 ++ infra/iam/README.md | 123 + infra/iam/generate.py | 212 -- infra/iam/main.tf | 5 + infra/iam/migrate_roles.py | 340 ++ infra/iam/roles/README.md | 75 + infra/iam/roles/beam_admin.role.yaml | 674 ++++ infra/iam/roles/beam_infra_manager.role.yaml | 848 +++++ infra/iam/roles/beam_viewer.role.yaml | 1113 +++++++ infra/iam/roles/beam_writer.role.yaml | 306 ++ infra/iam/roles/generate_roles.py | 277 ++ infra/iam/roles/roles.tf | 45 + infra/iam/roles/roles_config.yaml | 150 + infra/iam/roles/test_generate_roles.py | 82 + infra/iam/users.tf | 2 +- infra/iam/users.yml | 2 +- infra/keys/README.md | 103 + infra/keys/config.yaml | 34 + infra/keys/keys.py | 383 +++ infra/keys/keys.yaml | 26 + infra/keys/requirements.txt | 23 + infra/keys/secret_manager.py | 787 +++++ infra/keys/service_account.py | 425 +++ infra/keys/test_secret_manager.py | 839 +++++ infra/keys/test_service_account.py | 601 ++++ infra/security/README.md | 84 + infra/security/config.yml | 43 + infra/security/log_analyzer.py | 333 ++ infra/security/requirements.txt | 19 + .../it/common/utils/ResourceManagerUtils.java | 3 +- .../apache/beam/it/gcp/IOLoadTestBase.java | 2 +- .../artifacts/matchers/ArtifactsSubject.java | 1 + .../it/gcp/bigquery/BigQueryStreamingLT.java | 4 +- it/mongodb/build.gradle | 1 + .../pipeline/v1/external_transforms.proto | 4 + .../backend/containers/router/Dockerfile | 19 +- release/build.gradle.kts | 6 +- ...oundedSplittableProcessElementInvoker.java | 45 +- .../beam/runners/core/PaneInfoTracker.java | 2 +- .../beam/runners/core/SimpleDoFnRunner.java | 140 + .../SplittableParDoViaKeyedWorkItems.java | 24 +- .../beam/runners/core/StateMerging.java | 8 +- .../apache/beam/runners/core/StateTags.java | 3 +- .../beam/runners/core/TimerInternals.java | 7 +- .../beam/runners/core/WatermarkHold.java | 7 +- .../core/metrics/MetricsContainerImpl.java | 5 +- .../core/metrics/SimpleExecutionState.java | 2 +- .../ExecutableTriggerStateMachine.java | 8 +- .../metrics/MetricsContainerStepMapTest.java | 2 +- .../core/metrics/MetricsPusherTest.java | 2 +- .../ReshuffleTriggerStateMachineTest.java | 3 +- .../beam/runners/direct/DirectRunnerTest.java | 2 + .../flink/FlinkDetachedRunnerResult.java | 2 +- .../google-cloud-dataflow-java/build.gradle | 4 +- .../runners/dataflow/BatchViewOverrides.java | 16 +- .../runners/dataflow/DataflowPipelineJob.java | 3 +- .../util/RowCoderCloudObjectTranslator.java | 1 - .../SchemaCoderCloudObjectTranslator.java | 42 - .../worker/build.gradle | 6 +- .../runners/dataflow/worker/WindmillSink.java | 72 +- .../worker/WindmillTimerInternals.java | 70 +- .../dataflow/worker/WorkerCustomSources.java | 3 +- .../logging/DataflowWorkerLoggingHandler.java | 30 + .../DataflowWorkerLoggingInitializer.java | 4 + .../worker/util/ValueInEmptyWindows.java | 10 + .../client/AbstractWindmillStream.java | 314 +- .../windmill/client/WindmillStream.java | 4 - .../grpc/GetWorkResponseChunkAssembler.java | 6 +- .../client/grpc/GrpcCommitWorkStream.java | 46 +- .../client/grpc/GrpcDirectGetWorkStream.java | 25 +- .../client/grpc/GrpcDispatcherClient.java | 25 +- .../client/grpc/GrpcGetDataStream.java | 51 +- .../client/grpc/GrpcGetWorkStream.java | 25 +- .../grpc/GrpcGetWorkerMetadataStream.java | 34 +- .../grpc/GrpcWindmillStreamFactory.java | 108 +- .../client/grpc/stubs/ChannelCache.java | 12 +- .../windmill/state/ToIterableFunction.java | 6 +- .../processing/StreamingWorkScheduler.java | 2 +- .../dataflow/worker/FakeWindmillServer.java | 6 +- .../worker/GroupingShuffleReaderTest.java | 2 +- .../worker/IsmSideInputReaderTest.java | 6 +- .../worker/StreamingDataflowWorkerTest.java | 10 +- .../StreamingSideInputDoFnRunnerTest.java | 6 +- .../worker/StreamingSideInputFetcherTest.java | 2 +- .../DataflowWorkerLoggingHandlerTest.java | 61 +- .../util/GroupAlsoByWindowProperties.java | 9 +- .../client/AbstractWindmillStreamTest.java | 26 +- .../TriggeredScheduledExecutorService.java | 140 + .../client/grpc/FakeWindmillGrpcService.java | 47 +- .../client/grpc/GrpcCommitWorkStreamTest.java | 674 ++++ .../grpc/GrpcDirectGetWorkStreamTest.java | 27 +- .../client/grpc/GrpcGetDataStreamTest.java | 831 ++++- .../client/grpc/stubs/ChannelCacheTest.java | 133 + .../state/WindmillStateInternalsTest.java | 16 +- .../environment/DockerCommand.java | 2 +- .../PortablePipelineJarCreatorTest.java | 1 + .../org/apache/beam/runners/jet/Utils.java | 6 +- .../apache/beam/runners/jet/TestStreamP.java | 2 +- .../beam/runners/local/StructuralKey.java | 32 +- .../runners/samza/SamzaPipelineResult.java | 1 + .../beam/runners/samza/SamzaRunner.java | 3 + .../beam/runners/samza/runtime/OpMessage.java | 2 +- .../samza/translation/TranslationContext.java | 4 +- .../adapter/BoundedSourceSystemTest.java | 3 +- .../translation/EvaluationContext.java | 1 + .../GroupNonMergingWindowsFunctions.java | 11 +- .../translation/TransformTranslator.java | 2 +- .../StreamingTransformTranslator.java | 2 +- .../spark/util/SideInputBroadcast.java | 1 + .../beam/runners/spark/util/TimerUtils.java | 10 + .../streaming/utils/EmbeddedKafkaCluster.java | 5 +- .../beam/runners/twister2/Twister2Runner.java | 3 + sdks/go.mod | 115 +- sdks/go.sum | 240 +- sdks/go/pkg/beam/core/core.go | 2 +- sdks/go/pkg/beam/core/metrics/sampler.go | 4 +- sdks/go/pkg/beam/core/metrics/sampler_test.go | 4 +- .../pkg/beam/core/runtime/exec/userstate.go | 67 +- .../pkg/beam/core/runtime/harness/harness.go | 2 +- sdks/go/pkg/beam/options/jobopts/options.go | 10 +- .../prism/internal/engine/elementmanager.go | 162 +- .../runners/prism/internal/engine/strategy.go | 10 +- .../prism/internal/engine/strategy_test.go | 19 + .../runners/prism/internal/environments.go | 45 +- .../beam/runners/prism/internal/execute.go | 17 +- .../runners/prism/internal/handlecombine.go | 45 +- .../prism/internal/handlecombine_test.go | 70 +- .../runners/prism/internal/jobservices/job.go | 20 +- .../pkg/beam/runners/prism/internal/stage.go | 13 + .../prism/internal/unimplemented_test.go | 2 +- .../runners/prism/internal/worker/worker.go | 87 +- .../prism/internal/worker/worker_test.go | 7 +- sdks/go/test/integration/expansions.go | 31 +- sdks/go/test/integration/expansions_test.go | 3 + sdks/go/test/integration/integration.go | 2 + sdks/go/test/integration/primitives/state.go | 40 + .../test/integration/primitives/state_test.go | 5 + sdks/go/test/run_validatesrunner_tests.sh | 27 + sdks/java/container/Dockerfile | 2 +- sdks/java/container/boot.go | 13 +- sdks/java/container/common.gradle | 36 +- .../license_scripts/dep_urls_java.yaml | 2 +- sdks/java/core/build.gradle | 8 + .../beam/sdk/jmh/schemas/RowBundles.java | 1 + .../fn/data/BeamFnDataGrpcMultiplexer.java | 1 + .../fn/data/BeamFnDataInboundObserver.java | 1 + .../GrpcContextHeaderAccessorProvider.java | 2 +- .../java/org/apache/beam/sdk/io/FileIO.java | 16 +- .../java/org/apache/beam/sdk/io/Read.java | 1 + .../org/apache/beam/sdk/io/WriteFiles.java | 2 +- .../sdk/options/PipelineOptionsFactory.java | 7 +- .../beam/sdk/options/SdkHarnessOptions.java | 5 +- .../beam/sdk/schemas/SchemaTranslation.java | 46 + .../sdk/schemas/logicaltypes/OneOfType.java | 4 +- .../beam/sdk/schemas/transforms/Select.java | 1 + .../apache/beam/sdk/testing/TestPipeline.java | 35 +- .../apache/beam/sdk/testing/TestStream.java | 2 +- .../org/apache/beam/sdk/transforms/DoFn.java | 37 + .../sdk/transforms/DoFnOutputReceivers.java | 4 +- .../sdk/transforms/DoFnSchemaInformation.java | 2 +- .../beam/sdk/transforms/DoFnTester.java | 85 +- .../beam/sdk/transforms/MapElements.java | 1 + .../org/apache/beam/sdk/transforms/Reify.java | 2 +- .../org/apache/beam/sdk/transforms/Wait.java | 2 +- .../beam/sdk/transforms/join/CoGbkResult.java | 2 +- .../transforms/reflect/DoFnSignatures.java | 2 +- .../GrowableOffsetRangeTracker.java | 28 +- .../apache/beam/sdk/util/CombineFnUtil.java | 2 +- .../apache/beam/sdk/util/RowJsonUtils.java | 50 +- .../GroupIntoBatchesTranslation.java | 2 +- .../util/construction/ParDoTranslation.java | 2 +- .../util/construction/SplittableParDo.java | 2 +- .../SplittableParDoNaiveBounded.java | 58 + .../util/construction/TransformUpgrader.java | 18 +- .../util/construction/UnconsumedReads.java | 2 +- .../java/org/apache/beam/sdk/values/Row.java | 3 +- .../apache/beam/sdk/values/ShardedKey.java | 8 +- .../beam/sdk/values/ValueInSingleWindow.java | 19 +- .../apache/beam/sdk/values/WindowedValue.java | 7 + .../beam/sdk/values/WindowedValues.java | 125 +- .../apache/beam/sdk/coders/ZstdCoderTest.java | 1 + .../beam/sdk/io/CountingSourceTest.java | 2 +- .../apache/beam/sdk/io/FileSystemsTest.java | 1 + .../apache/beam/sdk/io/TextIOWriteTest.java | 11 +- .../options/PipelineOptionsFactoryTest.java | 2 +- .../beam/sdk/schemas/AutoValueSchemaTest.java | 2 +- .../beam/sdk/schemas/JavaBeanSchemaTest.java | 2 +- .../beam/sdk/schemas/JavaFieldSchemaTest.java | 4 +- .../sdk/schemas/transforms/GroupTest.java | 4 +- .../beam/sdk/schemas/utils/TestJavaBeans.java | 4 +- .../sdk/transforms/GroupIntoBatchesTest.java | 4 +- .../construction/CombineTranslationTest.java | 4 +- .../sdk/values/EncodableThrowableTest.java | 5 +- .../beam/sdk/values/TypeDescriptorsTest.java | 1 + .../expansion-service/container/Dockerfile | 2 + .../expansion-service/container/build.gradle | 4 + .../expansion/service/ExpansionService.java | 2 +- .../sdk/extensions/arrow/ArrowConversion.java | 33 + .../protobuf/ProtoByteBuddyUtils.java | 4 +- .../sketching/ApproximateDistinctTest.java | 2 +- .../sketching/TDigestQuantilesTest.java | 2 +- sdks/java/extensions/sql/build.gradle | 12 +- .../java/extensions/sql/hcatalog/build.gradle | 2 +- sdks/java/extensions/sql/iceberg/build.gradle | 81 + .../meta/provider/iceberg/IcebergCatalog.java | 0 .../iceberg/IcebergCatalogRegistrar.java | 31 + .../meta/provider/iceberg/IcebergFilter.java | 0 .../provider/iceberg/IcebergMetastore.java | 0 .../meta/provider/iceberg/IcebergTable.java | 0 .../iceberg/IcebergTableProvider.java | 0 .../meta/provider/iceberg/package-info.java | 0 .../iceberg/BeamSqlCliIcebergTest.java | 0 .../provider/iceberg/IcebergFilterTest.java | 0 .../iceberg/IcebergMetastoreTest.java | 0 .../provider/iceberg/IcebergReadWriteIT.java | 6 +- .../iceberg/IcebergTableProviderTest.java | 0 .../provider/iceberg}/PubsubToIcebergIT.java | 3 +- .../sql/impl/transform/agg/CovarianceFn.java | 4 +- .../sql/impl/transform/agg/VarianceFn.java | 4 +- .../catalog/InMemoryCatalogRegistrar.java | 6 +- .../provider/iceberg/IcebergMetastore.java | 154 - .../meta/provider/mongodb/MongoDbTable.java | 19 +- .../iceberg/IcebergMetastoreTest.java | 97 - .../provider/mongodb/MongoDbReadWriteIT.java | 7 +- .../beam/fn/harness/FnApiDoFnRunner.java | 203 +- .../org/apache/beam/fn/harness/FnHarness.java | 6 +- ...leTruncateSizedRestrictionsDoFnRunner.java | 2 +- .../control/ExecutionStateSampler.java | 13 +- .../harness/control/ProcessBundleHandler.java | 4 +- .../control/ExecutionStateSamplerTest.java | 8 +- ...MonitoringInfosInstructionHandlerTest.java | 3 + .../control/ProcessBundleHandlerTest.java | 1 + .../logging/BeamFnLoggingClientTest.java | 5 +- .../harness/state/MultimapUserStateTest.java | 15 +- .../beam/sdk/io/aws2/kinesis/KinesisIO.java | 2 +- .../beam/sdk/io/aws2/StaticSupplier.java | 1 + .../cdap/context/FailureCollectorWrapper.java | 2 +- .../context/FailureCollectorWrapperTest.java | 9 +- .../sdk/io/common/DatabaseTestHelper.java | 2 +- .../apache/beam/sdk/io/common/IOITHelper.java | 2 +- .../DebeziumReadSchemaTransformProvider.java | 3 +- sdks/java/io/expansion-service/build.gradle | 20 +- .../org/apache/beam/sdk/io/text/TextIOIT.java | 2 +- ...leWriteSchemaTransformFormatProviders.java | 2 +- .../io/fileschematransform/XmlRowAdapter.java | 1 + .../sdk/io/googleads/GoogleAdsIOTest.java | 1 + .../io/google-cloud-platform/build.gradle | 12 +- .../sdk/io/gcp/bigquery/BigQueryHelpers.java | 17 + .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 3 +- .../io/gcp/bigquery/BigQuerySourceBase.java | 3 +- .../StorageApiWritesShardedRecords.java | 31 +- .../io/gcp/bigquery/TableRowJsonCoder.java | 4 +- .../TwoLevelMessageConverterCache.java | 5 + .../BigtableReadSchemaTransformProvider.java | 134 +- .../BigtableWriteSchemaTransformProvider.java | 31 +- .../sdk/io/gcp/datastore/DatastoreV1.java | 6 +- .../beam/sdk/io/gcp/healthcare/FhirIO.java | 1 - .../io/gcp/pubsub/PreparePubsubWriteDoFn.java | 11 +- .../beam/sdk/io/gcp/pubsub/PubsubIO.java | 7 +- .../sdk/io/gcp/pubsub/PubsubTestClient.java | 4 +- .../pubsublite/internal/CloserReference.java | 2 +- .../internal/MemoryLimiterImpl.java | 1 + .../sdk/io/gcp/spanner/SpannerAccessor.java | 23 +- .../sdk/io/gcp/spanner/SpannerConfig.java | 44 + .../beam/sdk/io/gcp/spanner/SpannerIO.java | 197 +- .../spanner/SpannerTransformRegistrar.java | 139 +- .../MetadataSpannerConfigFactory.java | 10 + .../action/DetectNewPartitionsAction.java | 2 +- .../action/QueryChangeStreamAction.java | 2 +- .../io/gcp/testing/FakeDatasetService.java | 3 +- ...BigtableReadSchemaTransformProviderIT.java | 249 +- ...eSimpleWriteSchemaTransformProviderIT.java | 34 +- .../beam/sdk/io/gcp/datastore/V1TestUtil.java | 5 +- .../pubsub/PreparePubsubWriteDoFnTest.java | 78 +- .../beam/sdk/io/gcp/pubsub/PubsubIOTest.java | 38 +- .../sdk/io/gcp/spanner/SpannerIOReadTest.java | 2 +- .../sdk/io/gcp/spanner/SpannerReadIT.java | 55 - .../SpannerTransformRegistrarTest.java | 145 +- .../changestreams/it/IntegrationTestEnv.java | 1 - sdks/java/io/hbase/build.gradle | 2 +- .../sdk/io/hbase/HBaseRowMutationsCoder.java | 4 +- sdks/java/io/iceberg/build.gradle | 14 +- sdks/java/io/iceberg/hive/build.gradle | 2 +- .../io/iceberg/RecordWriterManagerTest.java | 40 +- sdks/java/io/jdbc/build.gradle | 3 + .../org/apache/beam/sdk/io/jdbc/JdbcIO.java | 39 +- .../jdbc/JdbcReadSchemaTransformProvider.java | 18 + .../JdbcWriteSchemaTransformProvider.java | 18 + .../PostgresSchemaTransformTranslation.java | 93 + ...adFromPostgresSchemaTransformProvider.java | 48 +- ...riteToPostgresSchemaTransformProvider.java | 38 +- .../beam/sdk/io/jdbc/JdbcIOPostgresIT.java | 178 ++ ...ostgresSchemaTransformTranslationTest.java | 233 ++ .../org/apache/beam/sdk/io/jms/JmsIO.java | 1 + sdks/java/io/kafka/build.gradle | 3 + .../io/kafka/jmh/KafkaIOUtilsBenchmark.java | 1 + .../io/kafka/kafka-integration-test.gradle | 1 + .../beam/sdk/io/kafka/KafkaCommitOffset.java | 2 + .../org/apache/beam/sdk/io/kafka/KafkaIO.java | 116 +- .../beam/sdk/io/kafka/KafkaIOUtils.java | 19 +- .../KafkaWriteSchemaTransformProvider.java | 132 +- .../apache/beam/sdk/io/kafka/KafkaWriter.java | 2 +- .../beam/sdk/io/kafka/ReadFromKafkaDoFn.java | 194 +- .../apache/beam/sdk/io/kafka/KafkaIOIT.java | 5 +- .../apache/beam/sdk/io/kafka/KafkaIOTest.java | 71 +- ...KafkaWriteSchemaTransformProviderTest.java | 44 +- sdks/java/io/mongodb/build.gradle | 5 +- .../apache/beam/sdk/io/mongodb/FindQuery.java | 5 +- .../beam/sdk/io/mongodb/MongoDbGridFSIO.java | 153 +- .../apache/beam/sdk/io/mongodb/MongoDbIO.java | 147 +- .../beam/sdk/io/mongodb/FindQueryTest.java | 5 +- .../sdk/io/mongodb/MongoDBGridFSIOTest.java | 114 +- .../beam/sdk/io/mongodb/MongoDbIOTest.java | 5 +- .../org/apache/beam/sdk/io/mqtt/MqttIO.java | 84 +- .../apache/beam/sdk/io/mqtt/MqttIOTest.java | 59 +- sdks/java/io/pulsar/build.gradle | 21 +- ...DoFn.java => NaiveReadFromPulsarDoFn.java} | 210 +- .../apache/beam/sdk/io/pulsar/PulsarIO.java | 194 +- .../beam/sdk/io/pulsar/PulsarIOUtils.java | 31 +- .../beam/sdk/io/pulsar/PulsarMessage.java | 62 +- .../sdk/io/pulsar/PulsarMessageCoder.java | 50 - .../sdk/io/pulsar/PulsarSourceDescriptor.java | 15 +- .../beam/sdk/io/pulsar/WriteToPulsarDoFn.java | 36 +- .../beam/sdk/io/pulsar/package-info.java | 6 +- .../beam/sdk/io/pulsar/FakeMessage.java | 25 +- .../beam/sdk/io/pulsar/FakePulsarClient.java | 82 +- .../beam/sdk/io/pulsar/FakePulsarReader.java | 33 +- .../apache/beam/sdk/io/pulsar/PulsarIOIT.java | 227 ++ .../beam/sdk/io/pulsar/PulsarIOTest.java | 247 +- .../sdk/io/pulsar/ReadFromPulsarDoFnTest.java | 46 +- .../RabbitMqReceiverWithOffset.java | 2 +- .../io/sparkreceiver/SparkReceiverIOIT.java | 2 +- .../beam/sdk/io/splunk/SplunkEventWriter.java | 6 +- .../sdk/io/synthetic/BundleSplitterTest.java | 4 +- sdks/java/javadoc/overview.html | 4 - .../org/apache/beam/sdk/managed/Managed.java | 3 + sdks/java/testing/junit/build.gradle | 50 + .../sdk/testing/TestPipelineExtension.java | 213 ++ .../apache/beam/sdk/testing/package-info.java | 19 + .../TestPipelineExtensionAdvancedTest.java | 88 + .../testing/TestPipelineExtensionTest.java | 56 + .../publishing/InfluxDBPublisher.java | 2 +- sdks/python/apache_beam/coders/coder_impl.pxd | 4 + sdks/python/apache_beam/coders/coder_impl.py | 66 +- sdks/python/apache_beam/coders/coders.py | 83 +- .../apache_beam/coders/coders_test_common.py | 172 +- sdks/python/apache_beam/coders/typecoders.py | 1 + sdks/python/apache_beam/dataframe/io.py | 31 +- .../juliaset/juliaset/juliaset_test_it.py | 6 +- .../complete/juliaset/juliaset_main.py | 9 +- .../complete/juliaset/requirements.txt | 17 + .../examples/complete/juliaset/setup.py | 128 - .../examples/inference/vllm_gemma_batch.py | 130 + .../transforms/elementwise/enrichment.py | 211 ++ .../transforms/elementwise/enrichment_test.py | 263 +- .../internal/cloudpickle_pickler.py | 2 + .../internal/code_object_pickler.py | 461 +++ .../internal/code_object_pickler_test.py | 565 ++++ sdks/python/apache_beam/internal/pickler.py | 30 +- .../apache_beam/internal/pickler_test.py | 24 + .../internal/test_data/__init__.py | 20 + .../internal/test_data/module_1.py | 27 + .../test_data/module_1_class_added.py | 34 + .../test_data/module_1_function_added.py | 33 + .../module_1_global_variable_added.py | 29 + .../module_1_lambda_variable_added.py | 28 + .../module_1_local_variable_added.py | 28 + .../module_1_local_variable_removed.py | 26 + .../module_1_nested_function_2_added.py | 32 + .../module_1_nested_function_added.py | 31 + .../internal/test_data/module_2.py | 82 + .../internal/test_data/module_2_modified.py | 92 + .../internal/test_data/module_3.py | 26 + .../internal/test_data/module_3_modified.py | 26 + .../test_data/module_with_default_argument.py | 24 + sdks/python/apache_beam/io/avroio.py | 60 +- sdks/python/apache_beam/io/avroio_test.py | 274 ++ .../io/components/adaptive_throttler.py | 92 + sdks/python/apache_beam/io/gcp/bigquery.py | 8 +- .../io/gcp/bigquery_file_loads_test.py | 19 +- .../io/gcp/bigquery_read_internal.py | 21 +- .../io/gcp/bigquery_read_internal_test.py | 170 + .../apache_beam/io/gcp/bigquery_test.py | 4 +- .../apache_beam/io/gcp/bigquery_tools.py | 32 +- .../apache_beam/io/gcp/bigquery_tools_test.py | 49 + sdks/python/apache_beam/io/gcp/bigtableio.py | 3 +- .../io/gcp/experimental/spannerio.py | 12 +- sdks/python/apache_beam/io/gcp/pubsub.py | 83 +- .../io/gcp/pubsub_integration_test.py | 85 + sdks/python/apache_beam/io/gcp/pubsub_test.py | 65 +- sdks/python/apache_beam/io/gcp/spanner.py | 103 + sdks/python/apache_beam/io/iobase_it_test.py | 72 + sdks/python/apache_beam/io/jdbc.py | 92 +- sdks/python/apache_beam/io/parquetio.py | 109 +- .../apache_beam/io/parquetio_it_test.py | 41 + sdks/python/apache_beam/io/parquetio_test.py | 365 ++- .../apache_beam/io/requestresponse_it_test.py | 5 +- sdks/python/apache_beam/io/textio.py | 37 +- sdks/python/apache_beam/io/textio_test.py | 25 + sdks/python/apache_beam/io/tfrecordio.py | 46 +- sdks/python/apache_beam/io/tfrecordio_test.py | 259 ++ .../apache_beam/metrics/monitoring_infos.py | 4 +- .../ml/anomaly/specifiable_test.py | 9 +- .../apache_beam/ml/anomaly/transforms.py | 12 +- sdks/python/apache_beam/ml/inference/base.py | 43 +- .../apache_beam/ml/inference/base_test.py | 2 +- .../inference/test_resources/vllm.dockerfile | 63 +- .../test_resources/vllm.dockerfile.old | 47 + .../ml/inference/vllm_tests_requirements.txt | 22 + .../rag/enrichment/milvus_search_it_test.py | 70 +- .../ml/rag/ingestion/bigquery_it_test.py | 2 +- sdks/python/apache_beam/ml/transforms/base.py | 39 + .../apache_beam/ml/transforms/base_test.py | 117 + .../ml/transforms/embeddings/vertex_ai.py | 241 +- .../transforms/embeddings/vertex_ai_test.py | 107 + .../apache_beam/options/pipeline_options.py | 55 +- .../options/pipeline_options_test.py | 10 +- .../options/pipeline_options_validator.py | 28 + .../pipeline_options_validator_test.py | 43 + sdks/python/apache_beam/pipeline.py | 19 +- sdks/python/apache_beam/pipeline_test.py | 2 + sdks/python/apache_beam/pvalue.py | 13 +- .../runners/dataflow/dataflow_runner.py | 27 +- .../runners/dataflow/dataflow_runner_test.py | 3 +- .../runners/dataflow/internal/names.py | 2 +- .../runners/dataflow/ptransform_overrides.py | 65 +- .../runners/direct/direct_runner.py | 118 +- .../yaml_parse_utils.py | 176 ++ .../package.json | 17 +- .../src/SidePanel.ts | 11 +- .../src/index.ts | 35 +- .../src/yaml/CustomStyle.tsx | 179 ++ .../src/yaml/DataType.ts | 37 + .../src/yaml/EditablePanel.tsx | 408 +++ .../src/yaml/EmojiMap.ts | 75 + .../src/yaml/Yaml.tsx | 322 ++ .../src/yaml/YamlEditor.tsx | 338 ++ .../src/yaml/YamlFlow.tsx | 227 ++ .../src/yaml/YamlWidget.tsx | 34 + .../style/index.css | 3 + .../style/mdc-theme.css | 4 +- .../style/yaml/Yaml.css | 40 + .../style/yaml/YamlEditor.css | 36 + .../style/yaml/YamlFlow.css | 168 + .../yarn.lock | 535 +++- .../interactive/pipeline_instrument_test.py | 2 +- .../runners/portability/job_server.py | 7 +- .../runners/portability/prism_runner.py | 54 + .../runners/portability/stager_test.py | 2 + .../runners/worker/data_sampler_test.py | 42 +- .../apache_beam/runners/worker/sdk_worker.py | 11 +- .../runners/worker/sdk_worker_main.py | 20 +- .../runners/worker/worker_status.py | 57 +- .../runners/worker/worker_status_test.py | 75 +- .../inference/vllm_gemma_benchmarks.py | 44 + .../testing/load_tests/build.gradle | 11 +- .../apache_beam/testing/test_pipeline.py | 9 +- .../tools/coders_microbenchmark.py | 10 + .../apache_beam/transforms/async_dofn.py | 19 +- .../transforms/combinefn_lifecycle_test.py | 5 + sdks/python/apache_beam/transforms/core.py | 96 +- .../apache_beam/transforms/core_test.py | 223 ++ .../enrichment_handlers/bigquery_it_test.py | 4 +- .../enrichment_handlers/bigtable_it_test.py | 5 +- .../enrichment_handlers/cloudsql.py | 656 ++++ .../enrichment_handlers/cloudsql_it_test.py | 630 ++++ .../enrichment_handlers/cloudsql_test.py | 569 ++++ .../vertex_ai_feature_store_it_test.py | 7 +- .../python/apache_beam/transforms/external.py | 7 +- .../external_transform_provider_it_test.py | 27 - sdks/python/apache_beam/transforms/managed.py | 7 +- .../apache_beam/transforms/ptransform.py | 32 +- .../apache_beam/transforms/ptransform_test.py | 100 +- .../apache_beam/transforms/sideinputs_test.py | 72 + sdks/python/apache_beam/transforms/util.py | 25 +- .../apache_beam/transforms/util_test.py | 144 +- sdks/python/apache_beam/typehints/schemas.py | 104 +- .../apache_beam/typehints/schemas_test.py | 12 +- .../apache_beam/typehints/typecheck_test.py | 9 + .../python/apache_beam/typehints/typehints.py | 13 +- .../apache_beam/utils/subprocess_server.py | 23 +- sdks/python/apache_beam/version.py | 2 +- sdks/python/apache_beam/yaml/README.md | 4 +- .../apache_beam/yaml/examples/README.md | 42 +- .../yaml/examples/testing/examples_test.py | 262 +- .../yaml/examples/testing/input_data.py | 99 + .../blueprint/iceberg_to_alloydb.yaml | 51 + .../blueprint/pubsub_to_iceberg.yaml | 6 +- .../transforms/jinja/import/README.md | 69 + .../jinja/import/macros/wordCountMacros.yaml | 64 + .../jinja/import/wordCountImport.yaml | 69 + .../transforms/jinja/include/README.md | 69 + .../include/submodules/combineTransform.yaml | 27 + .../include/submodules/explodeTransform.yaml | 26 + .../submodules/mapToFieldsCountConfig.yaml | 24 + .../submodules/mapToFieldsSplitConfig.yaml | 33 + .../submodules/readFromTextTransform.yaml | 26 + .../submodules/writeToTextTransform.yaml | 27 + .../jinja/include/wordCountInclude.yaml | 66 + .../transforms/ml/fraud_detection/README.md | 28 + .../fraud_detection_mlops_beam_yaml_sdk.ipynb | 1329 ++++++++ .../transforms/ml/log_analysis/README.md | 98 + .../ml/log_analysis/anomaly_scoring.yaml | 93 + .../ml/log_analysis/batch_log_analysis.sh | 108 + .../ml/log_analysis/iceberg_migration.yaml | 45 + .../ml/log_analysis/ml_preprocessing.yaml | 124 + .../ml/log_analysis/requirements.txt | 24 + .../transforms/ml/log_analysis/train.py | 89 + .../ml/sentiment_analysis/README.md | 21 +- .../streaming_sentiment_analysis.yaml | 29 +- .../transforms/ml/taxi_fare/README.md | 21 +- ...custom_nyc_taxifare_model_deployment.ipynb | 2 +- .../streaming_taxifare_prediction.yaml | 2 +- .../extended_tests/databases/iceberg.yaml | 63 + .../apache_beam/yaml/integration_tests.py | 33 +- sdks/python/apache_beam/yaml/json_utils.py | 7 +- .../apache_beam/yaml/pipeline.schema.yaml | 3 + sdks/python/apache_beam/yaml/standard_io.yaml | 14 +- .../yaml/tests/assign_timestamps.yaml | 87 + .../apache_beam/yaml/tests/bigtable.yaml | 116 +- .../python/apache_beam/yaml/tests/create.yaml | 23 + sdks/python/apache_beam/yaml/tests/csv.yaml | 22 + .../apache_beam/yaml/tests/runinference.yaml | 2 +- sdks/python/apache_beam/yaml/tests/sql.yaml | 18 + .../yaml/tests/validate_with_schema.yaml | 46 +- sdks/python/apache_beam/yaml/yaml_errors.py | 5 +- sdks/python/apache_beam/yaml/yaml_mapping.py | 5 +- sdks/python/apache_beam/yaml/yaml_ml.py | 57 +- sdks/python/apache_beam/yaml/yaml_ml_test.py | 87 + sdks/python/apache_beam/yaml/yaml_provider.py | 45 +- .../yaml/yaml_provider_unit_test.py | 66 + sdks/python/apache_beam/yaml/yaml_testing.py | 36 +- .../apache_beam/yaml/yaml_testing_test.py | 99 +- .../python/apache_beam/yaml/yaml_transform.py | 271 ++ .../apache_beam/yaml/yaml_transform_test.py | 114 + .../yaml/yaml_transform_unit_test.py | 111 + sdks/python/conftest.py | 132 +- sdks/python/container/Dockerfile | 11 +- .../base_image_requirements_manual.txt | 3 + sdks/python/container/build.gradle | 7 +- sdks/python/container/common.gradle | 2 + sdks/python/container/distroless/build.gradle | 6 +- sdks/python/container/ml/build.gradle | 64 + sdks/python/container/ml/common.gradle | 126 + .../py310/base_image_requirements.txt} | 100 +- sdks/python/container/ml/py310/build.gradle | 28 + .../py311/base_image_requirements.txt} | 102 +- sdks/python/container/ml/py311/build.gradle | 28 + .../py312/base_image_requirements.txt} | 102 +- sdks/python/container/ml/py312/build.gradle | 28 + sdks/python/container/ml/py313/build.gradle | 28 + .../py39/base_image_requirements.txt} | 96 +- sdks/python/container/ml/py39/build.gradle | 28 + .../py310/base_image_requirements.txt | 84 +- .../py311/base_image_requirements.txt | 86 +- .../py312/base_image_requirements.txt | 86 +- .../py313/base_image_requirements.txt | 141 +- .../py39/base_image_requirements.txt | 84 +- .../container/run_generate_requirements.sh | 11 +- sdks/python/gen_managed_doc.py | 3 + sdks/python/pytest.ini | 1 + sdks/python/setup.py | 46 +- .../python/test-suites/dataflow/common.gradle | 2 +- sdks/python/test-suites/direct/common.gradle | 28 - .../python/test-suites/portable/common.gradle | 2 +- sdks/python/test-suites/tox/common.gradle | 3 + sdks/python/tox.ini | 28 +- sdks/typescript/package.json | 2 +- settings.gradle.kts | 13 +- website/Dockerfile | 19 +- website/www/site/config.toml | 2 +- .../www/site/content/en/blog/beam-2.67.0.md | 73 + .../www/site/content/en/blog/gsoc-25-infra.md | 78 + .../content/en/documentation/io/managed-io.md | 3 + .../en/documentation/programming-guide.md | 98 + .../content/en/documentation/runners/nemo.md | 2 + .../content/en/documentation/runners/samza.md | 2 + .../en/documentation/runners/twister2.md | 2 + .../en/documentation/sdks/yaml-errors.md | 10 +- .../en/documentation/sdks/yaml-schema.md | 131 + .../content/en/documentation/sdks/yaml.md | 76 +- .../python/elementwise/enrichment-cloudsql.md | 146 + .../python/elementwise/enrichment.md | 1 + .../site/content/en/get-started/downloads.md | 161 +- .../www/site/content/en/performance/_index.md | 1 + .../performance/vllmgemmabatchtesla/_index.md | 43 + .../site/content/en/roadmap/nemo-runner.md | 4 +- .../site/content/en/roadmap/samza-runner.md | 4 +- .../content/en/roadmap/twister2-runner.md | 4 +- website/www/site/data/authors.yml | 4 + website/www/site/data/capability_matrix.yaml | 158 + website/www/site/data/performance.yaml | 16 + .../section-menu/en/documentation.html | 1 + 698 files changed, 42658 insertions(+), 4705 deletions(-) create mode 100644 .editorconfig create mode 100644 .github/actions/build-push-docker-action/action.yml create mode 100644 .github/actions/dind-up-action/action.yml create mode 100644 .github/trigger_files/beam_PostCommit_Python_ValidatesRunner_Dataflow.json create mode 100644 .github/trigger_files/beam_PostCommit_XVR_GoUsingJava_Dataflow.json create mode 100644 .github/workflows/beam_Infrastructure_PolicyEnforcer.yml create mode 100644 .github/workflows/beam_Infrastructure_SecurityLogging.yml create mode 100644 .github/workflows/beam_Infrastructure_ServiceAccountKeys.yml rename .github/workflows/{beam_UserRoles.yml => beam_Infrastructure_UsersPermissions.yml} (100%) create mode 100644 .github/workflows/load-tests-pipeline-options/beam_Inference_Python_Benchmarks_Dataflow_VLLM_Gemma_Batch.txt create mode 100644 examples/java/iceberg/build.gradle create mode 100644 examples/java/iceberg/src/main/java/org/apache/beam/examples/iceberg/IcebergBatchWriteExample.java rename examples/java/{src/main/java/org/apache/beam/examples/cookbook => iceberg/src/main/java/org/apache/beam/examples/iceberg}/IcebergRestCatalogCDCExample.java (99%) rename examples/java/{src/main/java/org/apache/beam/examples/cookbook => iceberg/src/main/java/org/apache/beam/examples/iceberg}/IcebergRestCatalogStreamingWriteExample.java (99%) rename examples/java/{src/main/java/org/apache/beam/examples/cookbook => iceberg/src/main/java/org/apache/beam/examples/iceberg}/IcebergTaxiExamples.java (99%) create mode 100644 examples/notebooks/beam-ml/anomaly_detection/anomaly_detection_timesfm.ipynb create mode 100644 examples/notebooks/beam-ml/cloudsql_mysql_product_catalog_embeddings.ipynb create mode 100644 examples/notebooks/beam-ml/dataflow_tpu_examples.ipynb create mode 100644 examples/yaml/README.md create mode 100644 infra/enforcement/README.md create mode 100644 infra/enforcement/account_keys.py create mode 100644 infra/enforcement/config.yml create mode 100644 infra/enforcement/iam.py create mode 100644 infra/enforcement/requirements.txt create mode 100644 infra/enforcement/sending.py delete mode 100644 infra/iam/generate.py create mode 100644 infra/iam/migrate_roles.py create mode 100644 infra/iam/roles/README.md create mode 100644 infra/iam/roles/beam_admin.role.yaml create mode 100644 infra/iam/roles/beam_infra_manager.role.yaml create mode 100644 infra/iam/roles/beam_viewer.role.yaml create mode 100644 infra/iam/roles/beam_writer.role.yaml create mode 100644 infra/iam/roles/generate_roles.py create mode 100644 infra/iam/roles/roles.tf create mode 100644 infra/iam/roles/roles_config.yaml create mode 100644 infra/iam/roles/test_generate_roles.py create mode 100644 infra/keys/README.md create mode 100644 infra/keys/config.yaml create mode 100644 infra/keys/keys.py create mode 100644 infra/keys/keys.yaml create mode 100644 infra/keys/requirements.txt create mode 100644 infra/keys/secret_manager.py create mode 100644 infra/keys/service_account.py create mode 100644 infra/keys/test_secret_manager.py create mode 100644 infra/keys/test_service_account.py create mode 100644 infra/security/README.md create mode 100644 infra/security/config.yml create mode 100644 infra/security/log_analyzer.py create mode 100644 infra/security/requirements.txt create mode 100644 runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/TriggeredScheduledExecutorService.java create mode 100644 sdks/java/extensions/sql/iceberg/build.gradle rename sdks/java/extensions/sql/{ => iceberg}/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergCatalog.java (100%) create mode 100644 sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergCatalogRegistrar.java rename sdks/java/extensions/sql/{ => iceberg}/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergFilter.java (100%) create mode 100644 sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastore.java rename sdks/java/extensions/sql/{ => iceberg}/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTable.java (100%) create mode 100644 sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProvider.java rename sdks/java/extensions/sql/{ => iceberg}/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/package-info.java (100%) rename sdks/java/extensions/sql/{ => iceberg}/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/BeamSqlCliIcebergTest.java (100%) rename sdks/java/extensions/sql/{ => iceberg}/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergFilterTest.java (100%) create mode 100644 sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastoreTest.java rename sdks/java/extensions/sql/{ => iceberg}/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java (98%) create mode 100644 sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProviderTest.java rename sdks/java/extensions/sql/{src/test/java/org/apache/beam/sdk/extensions/sql => iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg}/PubsubToIcebergIT.java (98%) delete mode 100644 sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastore.java delete mode 100644 sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastoreTest.java create mode 100644 sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/providers/PostgresSchemaTransformTranslation.java create mode 100644 sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOPostgresIT.java create mode 100644 sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/providers/PostgresSchemaTransformTranslationTest.java rename sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/{ReadFromPulsarDoFn.java => NaiveReadFromPulsarDoFn.java} (51%) delete mode 100644 sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/PulsarMessageCoder.java create mode 100644 sdks/java/io/pulsar/src/test/java/org/apache/beam/sdk/io/pulsar/PulsarIOIT.java create mode 100644 sdks/java/testing/junit/build.gradle create mode 100644 sdks/java/testing/junit/src/main/java/org/apache/beam/sdk/testing/TestPipelineExtension.java create mode 100644 sdks/java/testing/junit/src/main/java/org/apache/beam/sdk/testing/package-info.java create mode 100644 sdks/java/testing/junit/src/test/java/org/apache/beam/sdk/testing/TestPipelineExtensionAdvancedTest.java create mode 100644 sdks/java/testing/junit/src/test/java/org/apache/beam/sdk/testing/TestPipelineExtensionTest.java create mode 100644 sdks/python/apache_beam/examples/complete/juliaset/requirements.txt delete mode 100644 sdks/python/apache_beam/examples/complete/juliaset/setup.py create mode 100644 sdks/python/apache_beam/examples/inference/vllm_gemma_batch.py create mode 100644 sdks/python/apache_beam/internal/code_object_pickler_test.py create mode 100644 sdks/python/apache_beam/internal/test_data/__init__.py create mode 100644 sdks/python/apache_beam/internal/test_data/module_1.py create mode 100644 sdks/python/apache_beam/internal/test_data/module_1_class_added.py create mode 100644 sdks/python/apache_beam/internal/test_data/module_1_function_added.py create mode 100644 sdks/python/apache_beam/internal/test_data/module_1_global_variable_added.py create mode 100644 sdks/python/apache_beam/internal/test_data/module_1_lambda_variable_added.py create mode 100644 sdks/python/apache_beam/internal/test_data/module_1_local_variable_added.py create mode 100644 sdks/python/apache_beam/internal/test_data/module_1_local_variable_removed.py create mode 100644 sdks/python/apache_beam/internal/test_data/module_1_nested_function_2_added.py create mode 100644 sdks/python/apache_beam/internal/test_data/module_1_nested_function_added.py create mode 100644 sdks/python/apache_beam/internal/test_data/module_2.py create mode 100644 sdks/python/apache_beam/internal/test_data/module_2_modified.py create mode 100644 sdks/python/apache_beam/internal/test_data/module_3.py create mode 100644 sdks/python/apache_beam/internal/test_data/module_3_modified.py create mode 100644 sdks/python/apache_beam/internal/test_data/module_with_default_argument.py create mode 100644 sdks/python/apache_beam/io/gcp/bigquery_read_internal_test.py create mode 100644 sdks/python/apache_beam/io/iobase_it_test.py create mode 100644 sdks/python/apache_beam/ml/inference/test_resources/vllm.dockerfile.old create mode 100644 sdks/python/apache_beam/ml/inference/vllm_tests_requirements.txt create mode 100644 sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/apache_beam_jupyterlab_sidepanel/yaml_parse_utils.py create mode 100644 sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/src/yaml/CustomStyle.tsx create mode 100644 sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/src/yaml/DataType.ts create mode 100644 sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/src/yaml/EditablePanel.tsx create mode 100644 sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/src/yaml/EmojiMap.ts create mode 100644 sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/src/yaml/Yaml.tsx create mode 100644 sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/src/yaml/YamlEditor.tsx create mode 100644 sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/src/yaml/YamlFlow.tsx create mode 100644 sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/src/yaml/YamlWidget.tsx create mode 100644 sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/style/yaml/Yaml.css create mode 100644 sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/style/yaml/YamlEditor.css create mode 100644 sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/style/yaml/YamlFlow.css create mode 100644 sdks/python/apache_beam/testing/benchmarks/inference/vllm_gemma_benchmarks.py create mode 100644 sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql.py create mode 100644 sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_it_test.py create mode 100644 sdks/python/apache_beam/transforms/enrichment_handlers/cloudsql_test.py create mode 100644 sdks/python/apache_beam/yaml/examples/transforms/blueprint/iceberg_to_alloydb.yaml create mode 100644 sdks/python/apache_beam/yaml/examples/transforms/jinja/import/README.md create mode 100644 sdks/python/apache_beam/yaml/examples/transforms/jinja/import/macros/wordCountMacros.yaml create mode 100644 sdks/python/apache_beam/yaml/examples/transforms/jinja/import/wordCountImport.yaml create mode 100644 sdks/python/apache_beam/yaml/examples/transforms/jinja/include/README.md create mode 100644 sdks/python/apache_beam/yaml/examples/transforms/jinja/include/submodules/combineTransform.yaml create mode 100644 sdks/python/apache_beam/yaml/examples/transforms/jinja/include/submodules/explodeTransform.yaml create mode 100644 sdks/python/apache_beam/yaml/examples/transforms/jinja/include/submodules/mapToFieldsCountConfig.yaml create mode 100644 sdks/python/apache_beam/yaml/examples/transforms/jinja/include/submodules/mapToFieldsSplitConfig.yaml create mode 100644 sdks/python/apache_beam/yaml/examples/transforms/jinja/include/submodules/readFromTextTransform.yaml create mode 100644 sdks/python/apache_beam/yaml/examples/transforms/jinja/include/submodules/writeToTextTransform.yaml create mode 100644 sdks/python/apache_beam/yaml/examples/transforms/jinja/include/wordCountInclude.yaml create mode 100644 sdks/python/apache_beam/yaml/examples/transforms/ml/fraud_detection/README.md create mode 100644 sdks/python/apache_beam/yaml/examples/transforms/ml/fraud_detection/fraud_detection_mlops_beam_yaml_sdk.ipynb create mode 100644 sdks/python/apache_beam/yaml/examples/transforms/ml/log_analysis/README.md create mode 100644 sdks/python/apache_beam/yaml/examples/transforms/ml/log_analysis/anomaly_scoring.yaml create mode 100755 sdks/python/apache_beam/yaml/examples/transforms/ml/log_analysis/batch_log_analysis.sh create mode 100644 sdks/python/apache_beam/yaml/examples/transforms/ml/log_analysis/iceberg_migration.yaml create mode 100644 sdks/python/apache_beam/yaml/examples/transforms/ml/log_analysis/ml_preprocessing.yaml create mode 100644 sdks/python/apache_beam/yaml/examples/transforms/ml/log_analysis/requirements.txt create mode 100644 sdks/python/apache_beam/yaml/examples/transforms/ml/log_analysis/train.py create mode 100644 sdks/python/apache_beam/yaml/extended_tests/databases/iceberg.yaml create mode 100644 sdks/python/container/ml/build.gradle create mode 100644 sdks/python/container/ml/common.gradle rename sdks/python/container/{py310/ml_image_requirements.txt => ml/py310/base_image_requirements.txt} (82%) create mode 100644 sdks/python/container/ml/py310/build.gradle rename sdks/python/container/{py311/ml_image_requirements.txt => ml/py311/base_image_requirements.txt} (81%) create mode 100644 sdks/python/container/ml/py311/build.gradle rename sdks/python/container/{py312/ml_image_requirements.txt => ml/py312/base_image_requirements.txt} (81%) create mode 100644 sdks/python/container/ml/py312/build.gradle create mode 100644 sdks/python/container/ml/py313/build.gradle rename sdks/python/container/{py39/ml_image_requirements.txt => ml/py39/base_image_requirements.txt} (82%) create mode 100644 sdks/python/container/ml/py39/build.gradle create mode 100644 website/www/site/content/en/blog/beam-2.67.0.md create mode 100644 website/www/site/content/en/blog/gsoc-25-infra.md create mode 100644 website/www/site/content/en/documentation/sdks/yaml-schema.md create mode 100644 website/www/site/content/en/documentation/transforms/python/elementwise/enrichment-cloudsql.md create mode 100644 website/www/site/content/en/performance/vllmgemmabatchtesla/_index.md diff --git a/.asf.yaml b/.asf.yaml index 6023b26998a3..74a92af46b59 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -51,6 +51,8 @@ github: protected_branches: master: {} + release-2.68: {} + release-2.67.0-postrelease: {} release-2.67: {} release-2.66.0-postrelease: {} release-2.66: {} diff --git a/.editorconfig b/.editorconfig new file mode 100644 index 000000000000..7ee8fffb1ba8 --- /dev/null +++ b/.editorconfig @@ -0,0 +1,30 @@ +# Licensed 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. +# EditorConfig is awesome: https://EditorConfig.org + +# top-most EditorConfig file +root = true + +[*] +indent_style = space +indent_size = 2 +end_of_line = lf +charset = utf-8 +trim_trailing_whitespace = true +insert_final_newline = true + +[*.{go,mod,sum}] +indent_style = tab +indent_size = unset + +[Dockerfile] +indent_size = 4 diff --git a/.github/actions/build-push-docker-action/action.yml b/.github/actions/build-push-docker-action/action.yml new file mode 100644 index 000000000000..84c331f09a69 --- /dev/null +++ b/.github/actions/build-push-docker-action/action.yml @@ -0,0 +1,45 @@ +# This is a composite action to build and push a Docker image. +name: 'Docker Build and Push' +description: 'Builds and pushes a Docker image to a container registry.' + +inputs: + dockerfile_path: + description: 'Path to the Dockerfile' + required: true + image_name: + description: 'Base name for the Docker image (e.g., gcr.io/my-project/my-app)' + required: true + image_tag: + description: 'Tag for the Docker image (e.g., latest, or a git sha)' + required: true + build_context: + description: 'The build context for the Docker build command' + required: false + default: '.' + +outputs: + image_url: + description: "The full URL of the pushed image, including the tag" + value: ${{ steps.build-push.outputs.image_url }} # the value is set from a step's output + +runs: + using: "composite" + steps: + - name: Configure Docker to use Google Cloud credentials + shell: bash + run: gcloud auth configure-docker --quiet + + - name: Build and Push Docker Image + id: build-push # give the step an ID to reference its output + shell: bash + run: | + # Construct the full image URL from the inputs + FULL_IMAGE_URL="${{ inputs.image_name }}:${{ inputs.image_tag }}" + echo "Building image: $FULL_IMAGE_URL" + + # Build the image + docker build -t $FULL_IMAGE_URL -f ${{ inputs.dockerfile_path }} ${{ inputs.build_context }} + # Push the image + docker push $FULL_IMAGE_URL + # Set the output value for this action + echo "image_url=$FULL_IMAGE_URL" >> $GITHUB_OUTPUT \ No newline at end of file diff --git a/.github/actions/dind-up-action/action.yml b/.github/actions/dind-up-action/action.yml new file mode 100644 index 000000000000..23cc8613bb67 --- /dev/null +++ b/.github/actions/dind-up-action/action.yml @@ -0,0 +1,275 @@ +# 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. + +name: "Start and Prepare DinD" +description: "Launch, verify, and prepare a Docker-in-Docker environment." +inputs: + # --- Core DinD Config --- + container-name: + description: "Name for the DinD container." + default: dind-daemon + bind-address: + default: 127.0.0.1 + port: + default: "2375" + storage-volume: + default: dind-storage + execroot-volume: + default: dind-execroot + ephemeral-volumes: + description: "Generate unique per-run volume names (recommended)." + default: "true" + auto-prune-dangling: + description: "Prune dangling ephemeral DinD volumes from previous runs." + default: "true" + tmpfs-run-size: + default: 1g + tmpfs-varrun-size: + default: 1g + storage-driver: + default: overlay2 + additional-dockerd-args: + default: "" + use-host-network: + description: "Run DinD with --network host instead of publishing a TCP port." + default: "false" + + # --- Health & Wait Config --- + health-interval: + default: 2s + health-retries: + default: "60" + health-start-period: + default: 10s + wait-timeout: + default: "180" + + # --- NEW: Optional Setup & Verification Steps --- + cleanup-dind-on-start: + description: "Run 'docker system prune' inside DinD immediately after it starts." + default: "true" + smoke-test-port-mapping: + description: "Run a quick test to ensure port mapping from DinD is working." + default: "true" + prime-testcontainers: + description: "Start and stop a small container via the testcontainers library to prime Ryuk." + default: "false" + + # --- Output Config --- + export-gh-env: + description: "Also write DOCKER_HOST and DIND_IP to $GITHUB_ENV for the rest of the job." + default: "false" + +outputs: + docker-host: + description: "The TCP address for the DinD daemon (e.g., tcp://127.0.0.1:2375)." + value: ${{ steps.set-output.outputs.docker-host }} + dind-ip: + description: "The discovered bridge IP address of the DinD container." + value: ${{ steps.discover-ip.outputs.dind-ip }} + container-name: + description: "The name of the running DinD container." + value: ${{ inputs.container-name || 'dind-daemon' }} + storage-volume: + value: ${{ steps.set-output.outputs.storage_volume }} + execroot-volume: + value: ${{ steps.set-output.outputs.execroot_volume }} + +runs: + using: "composite" + steps: + - name: Prune old dangling ephemeral DinD volumes + if: ${{ inputs.auto-prune-dangling == 'true' }} + shell: bash + run: | + docker volume ls -q \ + --filter "label=com.github.dind=1" \ + --filter "label=com.github.repo=${GITHUB_REPOSITORY}" \ + --filter "dangling=true" | xargs -r docker volume rm || true + + - name: Start docker:dind + shell: bash + run: | + # (Your original 'Start docker:dind' script is perfect here - no changes needed) + set -euo pipefail + NAME="${{ inputs.container-name || 'dind-daemon' }}" + BIND="${{ inputs.bind-address || '127.0.0.1' }}" + PORT="${{ inputs.port || '2375' }}" + SD="${{ inputs.storage-driver || 'overlay2' }}" + TRS="${{ inputs.tmpfs-run-size || '1g' }}" + TVRS="${{ inputs.tmpfs-varrun-size || '1g' }}" + HI="${{ inputs.health-interval || '2s' }}" + HR="${{ inputs.health-retries || '60' }}" + HSP="${{ inputs.health-start-period || '10s' }}" + EXTRA="${{ inputs.additional-dockerd-args }}" + USE_HOST_NET="${{ inputs.use-host-network || 'false' }}" + + if [[ "${{ inputs.ephemeral-volumes }}" == "true" ]]; then + SUFFIX="${GITHUB_RUN_ID:-0}-${GITHUB_RUN_ATTEMPT:-0}-${GITHUB_JOB:-job}" + STORAGE_VOL="dind-storage-${SUFFIX}" + EXECROOT_VOL="dind-execroot-${SUFFIX}" + else + STORAGE_VOL="${{ inputs.storage-volume || 'dind-storage' }}" + EXECROOT_VOL="${{ inputs.execroot-volume || 'dind-execroot' }}" + fi + + docker volume create --name "${STORAGE_VOL}" --label "com.github.dind=1" --label "com.github.repo=${GITHUB_REPOSITORY}" >/dev/null + docker volume create --name "${EXECROOT_VOL}" --label "com.github.dind=1" --label "com.github.repo=${GITHUB_REPOSITORY}" >/dev/null + docker rm -f -v "$NAME" 2>/dev/null || true + + NET_ARGS="" + PUBLISH_ARGS="-p ${BIND}:${PORT}:${PORT}" + if [[ "${USE_HOST_NET}" == "true" ]]; then + NET_ARGS="--network host" + PUBLISH_ARGS="" + fi + + docker run -d --privileged --name "$NAME" \ + --cgroupns=host \ + -e DOCKER_TLS_CERTDIR= \ + ${NET_ARGS} \ + ${PUBLISH_ARGS} \ + -v "${STORAGE_VOL}:/var/lib/docker" \ + -v "${EXECROOT_VOL}:/execroot" \ + --tmpfs /run:rw,exec,size=${TRS} \ + --tmpfs /var/run:rw,exec,size=${TVRS} \ + --label "com.github.dind=1" \ + --health-cmd='docker info > /dev/null' \ + --health-interval=${HI} \ + --health-retries=${HR} \ + --health-start-period=${HSP} \ + docker:dind \ + --host=tcp://0.0.0.0:${PORT} \ + --host=unix:///var/run/docker.sock \ + --storage-driver=${SD} \ + --exec-root=/execroot ${EXTRA} + + { + echo "STORAGE_VOL=${STORAGE_VOL}" + echo "EXECROOT_VOL=${EXECROOT_VOL}" + } >> "$GITHUB_ENV" + + - name: Wait for DinD daemon + shell: bash + run: | + set -euo pipefail + NAME="${{ inputs.container-name || 'dind-daemon' }}" + HOST="${{ inputs.bind-address || '127.0.0.1' }}" + PORT="${{ inputs.port || '2375' }}" + TIMEOUT="${{ inputs.wait-timeout || '180' }}" + echo "Waiting for Docker-in-Docker to be ready..." + if ! timeout ${TIMEOUT}s bash -c 'until docker -H "tcp://'"${HOST}"':'"${PORT}"'" info >/dev/null 2>&1; do sleep 2; done'; then + echo "::error::DinD failed to start within ${TIMEOUT}s." + docker logs "$NAME" || true + exit 1 + fi + echo "DinD is ready." + docker -H "tcp://${HOST}:${PORT}" info --format 'Daemon OK → OS={{.OperatingSystem}} Version={{.ServerVersion}}' + + - id: set-output + shell: bash + run: | + HOST="${{ inputs.bind-address || '127.0.0.1' }}" + PORT="${{ inputs.port || '2375' }}" + echo "docker-host=tcp://${HOST}:${PORT}" >> "$GITHUB_OUTPUT" + echo "storage_volume=${STORAGE_VOL:-}" >> "$GITHUB_OUTPUT" + echo "execroot_volume=${EXECROOT_VOL:-}" >> "$GITHUB_OUTPUT" + + # --- NEW: Integrated Setup & Verification Steps --- + + - name: Cleanup DinD Environment + if: ${{ inputs.cleanup-dind-on-start == 'true' }} + shell: bash + run: | + echo "Performing initial cleanup of DinD environment..." + DIND_HOST="${{ steps.set-output.outputs.docker-host }}" + docker -H "${DIND_HOST}" system prune -af --volumes + docker -H "${DIND_HOST}" image prune -af + + - id: discover-ip + name: Discover DinD Container IP + shell: bash + run: | + set -euo pipefail + NAME="${{ inputs.container-name || 'dind-daemon' }}" + + # Use host daemon to inspect the DinD container + nm=$(docker inspect -f '{{.HostConfig.NetworkMode}}' "$NAME") + echo "DinD NetworkMode=${nm}" + + # Try to find the bridge network IP + ip=$(docker inspect -f '{{range .NetworkSettings.Networks}}{{.IPAddress}}{{end}}' "$NAME" || true) + + # If still empty, likely host networking -> use loopback + if [[ -z "${ip}" || "${nm}" == "host" ]]; then + echo "No bridge IP found or using host network. Falling back to 127.0.0.1." + ip="127.0.0.1" + fi + + echo "Discovered DinD IP: ${ip}" + echo "dind-ip=${ip}" >> "$GITHUB_OUTPUT" + + - name: Smoke Test Port Mapping + if: ${{ inputs.smoke-test-port-mapping == 'true' }} + env: + DOCKER_HOST: ${{ steps.set-output.outputs.docker-host }} + DIND_IP: ${{ steps.discover-ip.outputs.dind-ip }} + shell: bash + run: | + set -euo pipefail + echo "Running port mapping smoke test..." + docker pull redis:7.2-alpine + cid=$(docker run -d -p 0:6379 --name redis-smoke redis:7-alpine) + hostport=$(docker port redis-smoke 6379/tcp | sed 's/.*://') + echo "Redis container started, mapped to host port ${hostport}" + echo "Probing connection to ${DIND_IP}:${hostport} ..." + + timeout 5 bash -c 'exec 3<>/dev/tcp/$DIND_IP/'"$hostport" + if [[ $? -eq 0 ]]; then + echo "TCP connection successful. Port mapping is working." + else + echo "::error::Failed to connect to mapped port on ${DIND_IP}:${hostport}" + docker logs redis-smoke + exit 1 + fi + docker rm -f "$cid" + + - name: Prime Testcontainers (Ryuk) + if: ${{ inputs.prime-testcontainers == 'true' }} + env: + DOCKER_HOST: ${{ steps.set-output.outputs.docker-host }} + TESTCONTAINERS_HOST_OVERRIDE: ${{ steps.discover-ip.outputs.dind-ip }} + shell: bash + run: | + echo "Priming Testcontainers/Ryuk..." + python -m pip install -q --upgrade pip testcontainers + # Use a tiny image for a fast and stable prime + docker pull alpine:3.19 + python - <<'PY' + from testcontainers.core.container import DockerContainer + c = DockerContainer("alpine:3.19").with_command("true") + c.start() + c.stop() + print("Ryuk primed and ready.") + PY + + - name: Export Environment Variables + if: ${{ inputs.export-gh-env == 'true' }} + shell: bash + run: | + echo "DOCKER_HOST=${{ steps.set-output.outputs.docker-host }}" >> "$GITHUB_ENV" + echo "DIND_IP=${{ steps.discover-ip.outputs.dind-ip }}" >> "$GITHUB_ENV" \ No newline at end of file diff --git a/.github/dependabot.yml b/.github/dependabot.yml index 248e8d6a69bf..e7a40726ed9b 100644 --- a/.github/dependabot.yml +++ b/.github/dependabot.yml @@ -46,7 +46,3 @@ updates: directory: "/" schedule: interval: "daily" - allow: - # Allow only automatic updates for official github actions - # Other github-actions require approval from INFRA - - dependency-name: "actions/*" diff --git a/.github/trigger_files/IO_Iceberg_Integration_Tests.json b/.github/trigger_files/IO_Iceberg_Integration_Tests.json index 7ab7bcd9a9c6..b73af5e61a43 100644 --- a/.github/trigger_files/IO_Iceberg_Integration_Tests.json +++ b/.github/trigger_files/IO_Iceberg_Integration_Tests.json @@ -1,4 +1,4 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run.", - "modification": 2 + "modification": 1 } diff --git a/.github/trigger_files/IO_Iceberg_Integration_Tests_Dataflow.json b/.github/trigger_files/IO_Iceberg_Integration_Tests_Dataflow.json index 8fab48cc672a..5abe02fc09c7 100644 --- a/.github/trigger_files/IO_Iceberg_Integration_Tests_Dataflow.json +++ b/.github/trigger_files/IO_Iceberg_Integration_Tests_Dataflow.json @@ -1,4 +1,4 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run.", - "modification": 5 + "modification": 1 } diff --git a/.github/trigger_files/IO_Iceberg_Managed_Integration_Tests_Dataflow.json b/.github/trigger_files/IO_Iceberg_Managed_Integration_Tests_Dataflow.json index 8fab48cc672a..3a009261f4f9 100644 --- a/.github/trigger_files/IO_Iceberg_Managed_Integration_Tests_Dataflow.json +++ b/.github/trigger_files/IO_Iceberg_Managed_Integration_Tests_Dataflow.json @@ -1,4 +1,4 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run.", - "modification": 5 + "modification": 2 } diff --git a/.github/trigger_files/beam_PostCommit_Python.json b/.github/trigger_files/beam_PostCommit_Python.json index 4aa5f70b72ee..8675e9535061 100644 --- a/.github/trigger_files/beam_PostCommit_Python.json +++ b/.github/trigger_files/beam_PostCommit_Python.json @@ -1,5 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run.", - "modification": 101 + "modification": 28 } diff --git a/.github/trigger_files/beam_PostCommit_Python_ValidatesRunner_Dataflow.json b/.github/trigger_files/beam_PostCommit_Python_ValidatesRunner_Dataflow.json new file mode 100644 index 000000000000..b26833333238 --- /dev/null +++ b/.github/trigger_files/beam_PostCommit_Python_ValidatesRunner_Dataflow.json @@ -0,0 +1,4 @@ +{ + "comment": "Modify this file in a trivial way to cause this test suite to run", + "modification": 2 +} diff --git a/.github/trigger_files/beam_PostCommit_Python_Xlang_Gcp_Dataflow.json b/.github/trigger_files/beam_PostCommit_Python_Xlang_Gcp_Dataflow.json index 2504db607e46..95fef3e26ca2 100644 --- a/.github/trigger_files/beam_PostCommit_Python_Xlang_Gcp_Dataflow.json +++ b/.github/trigger_files/beam_PostCommit_Python_Xlang_Gcp_Dataflow.json @@ -1,4 +1,4 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "modification": 12 + "modification": 13 } diff --git a/.github/trigger_files/beam_PostCommit_Python_Xlang_Gcp_Direct.json b/.github/trigger_files/beam_PostCommit_Python_Xlang_Gcp_Direct.json index afdc7f7012a8..2504db607e46 100644 --- a/.github/trigger_files/beam_PostCommit_Python_Xlang_Gcp_Direct.json +++ b/.github/trigger_files/beam_PostCommit_Python_Xlang_Gcp_Direct.json @@ -1,4 +1,4 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "modification": 11 + "modification": 12 } diff --git a/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Direct.json b/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Direct.json index 455144f02a35..b26833333238 100644 --- a/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Direct.json +++ b/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Direct.json @@ -1,4 +1,4 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "modification": 6 + "modification": 2 } diff --git a/.github/trigger_files/beam_PostCommit_SQL.json b/.github/trigger_files/beam_PostCommit_SQL.json index 5ac8a7f3f6ee..6cc79a7a0325 100644 --- a/.github/trigger_files/beam_PostCommit_SQL.json +++ b/.github/trigger_files/beam_PostCommit_SQL.json @@ -1,4 +1,4 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run ", - "modification": 4 + "modification": 1 } diff --git a/.github/trigger_files/beam_PostCommit_XVR_Direct.json b/.github/trigger_files/beam_PostCommit_XVR_Direct.json index cccbad0b12df..73867c483554 100644 --- a/.github/trigger_files/beam_PostCommit_XVR_Direct.json +++ b/.github/trigger_files/beam_PostCommit_XVR_Direct.json @@ -1,3 +1,3 @@ { - "modification": 4 + "modification": 5 } diff --git a/.github/trigger_files/beam_PostCommit_XVR_Flink.json b/.github/trigger_files/beam_PostCommit_XVR_Flink.json index 702328d16d4b..2d8ad3760b4b 100644 --- a/.github/trigger_files/beam_PostCommit_XVR_Flink.json +++ b/.github/trigger_files/beam_PostCommit_XVR_Flink.json @@ -1,3 +1,3 @@ { - "modification": 1 + "modification": 2 } diff --git a/.github/trigger_files/beam_PostCommit_XVR_GoUsingJava_Dataflow.json b/.github/trigger_files/beam_PostCommit_XVR_GoUsingJava_Dataflow.json new file mode 100644 index 000000000000..920c8d132e4a --- /dev/null +++ b/.github/trigger_files/beam_PostCommit_XVR_GoUsingJava_Dataflow.json @@ -0,0 +1,4 @@ +{ + "comment": "Modify this file in a trivial way to cause this test suite to run", + "modification": 1 +} \ No newline at end of file diff --git a/.github/trigger_files/beam_PostCommit_Yaml_Xlang_Direct.json b/.github/trigger_files/beam_PostCommit_Yaml_Xlang_Direct.json index 8b2c8c445c1f..b5704c67ef1c 100644 --- a/.github/trigger_files/beam_PostCommit_Yaml_Xlang_Direct.json +++ b/.github/trigger_files/beam_PostCommit_Yaml_Xlang_Direct.json @@ -1,4 +1,4 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "revision": 5 + "revision": 6 } diff --git a/.github/workflows/beam_IODatastoresCredentialsRotation.yml b/.github/workflows/beam_IODatastoresCredentialsRotation.yml index d6b04afdebe3..ee6dcc123a91 100644 --- a/.github/workflows/beam_IODatastoresCredentialsRotation.yml +++ b/.github/workflows/beam_IODatastoresCredentialsRotation.yml @@ -82,17 +82,17 @@ jobs: run: | date=$(date -u +"%Y-%m-%d") echo "date=$date" >> $GITHUB_ENV - - name: Send email - uses: dawidd6/action-send-mail@v3 - if: failure() - with: - server_address: smtp.gmail.com - server_port: 465 - secure: true - username: ${{ secrets.ISSUE_REPORT_SENDER_EMAIL_ADDRESS }} - password: ${{ secrets.ISSUE_REPORT_SENDER_EMAIL_PASSWORD }} - subject: Credentials Rotation Failure on IO-Datastores cluster (${{ env.date }}) - to: dev@beam.apache.org - from: gactions@beam.apache.org - body: | - Something went wrong during the automatic credentials rotation for IO-Datastores Cluster, performed at ${{ env.date }}. It may be necessary to check the state of the cluster certificates. For further details refer to the following links:\n * Failing job: https://github.com/apache/beam/actions/workflows/beam_IODatastoresCredentialsRotation.yml \n * Job configuration: https://github.com/apache/beam/blob/master/.github/workflows/beam_IODatastoresCredentialsRotation.yml \n * Cluster URL: https://pantheon.corp.google.com/kubernetes/clusters/details/us-central1-a/io-datastores/details?mods=dataflow_dev&project=apache-beam-testing \ No newline at end of file +# - name: Send email +# uses: dawidd6/action-send-mail@v3 +# if: failure() +# with: +# server_address: smtp.gmail.com +# server_port: 465 +# secure: true +# username: ${{ secrets.ISSUE_REPORT_SENDER_EMAIL_ADDRESS }} +# password: ${{ secrets.ISSUE_REPORT_SENDER_EMAIL_PASSWORD }} +# subject: Credentials Rotation Failure on IO-Datastores cluster (${{ env.date }}) +# to: dev@beam.apache.org +# from: gactions@beam.apache.org +# body: | +# Something went wrong during the automatic credentials rotation for IO-Datastores Cluster, performed at ${{ env.date }}. It may be necessary to check the state of the cluster certificates. For further details refer to the following links:\n * Failing job: https://github.com/apache/beam/actions/workflows/beam_IODatastoresCredentialsRotation.yml \n * Job configuration: https://github.com/apache/beam/blob/master/.github/workflows/beam_IODatastoresCredentialsRotation.yml \n * Cluster URL: https://pantheon.corp.google.com/kubernetes/clusters/details/us-central1-a/io-datastores/details?mods=dataflow_dev&project=apache-beam-testing diff --git a/.github/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml b/.github/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml index 6b60517a1899..ff7480c320af 100644 --- a/.github/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml +++ b/.github/workflows/beam_Inference_Python_Benchmarks_Dataflow.yml @@ -55,7 +55,7 @@ jobs: (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Inference Benchmarks' runs-on: [self-hosted, ubuntu-20.04, main] - timeout-minutes: 900 + timeout-minutes: 1000 name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) strategy: matrix: @@ -72,7 +72,12 @@ jobs: - name: Setup Python environment uses: ./.github/actions/setup-environment-action with: + java-version: default python-version: '3.10' + - name: Package Python SDK using Gradle + run: ./gradlew :sdks:python:sdist -PpythonVersion=3.10 + - name: Configure Docker for Artifact Registry + run: gcloud auth configure-docker us-docker.pkg.dev - name: Prepare test arguments uses: ./.github/actions/test-arguments-action with: @@ -86,9 +91,28 @@ jobs: ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Imagenet_Classification_Resnet_152_Tesla_T4_GPU.txt ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Sentiment_Streaming_DistilBert_Base_Uncased.txt ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/beam_Inference_Python_Benchmarks_Dataflow_Pytorch_Sentiment_Batch_DistilBert_Base_Uncased.txt + ${{ github.workspace }}/.github/workflows/load-tests-pipeline-options/beam_Inference_Python_Benchmarks_Dataflow_VLLM_Gemma_Batch.txt # The env variables are created and populated in the test-arguments-action as "_test_arguments_" - name: get current time run: echo "NOW_UTC=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_ENV + - name: Build VLLM Development Image + id: build_vllm_image + uses: ./.github/actions/build-push-docker-action + with: + dockerfile_path: 'sdks/python/apache_beam/ml/inference/test_resources/vllm.dockerfile' + image_name: 'us-docker.pkg.dev/apache-beam-testing/beam-temp/beam-vllm-gpu-base' + image_tag: ${{ github.sha }} + - name: Run VLLM Gemma Batch Test + uses: ./.github/actions/gradle-command-self-hosted-action + timeout-minutes: 180 + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.benchmarks.inference.vllm_gemma_benchmarks \ + -Prunner=DataflowRunner \ + -PsdkLocationOverride=false \ + -PpythonVersion=3.10 \ + -PloadTest.requirementsTxtFile=apache_beam/ml/inference/vllm_tests_requirements.txt '-PloadTest.args=${{ env.beam_Inference_Python_Benchmarks_Dataflow_test_arguments_8 }} --mode=batch --job_name=benchmark-tests-vllm-with-gemma-2b-it-batch-${{env.NOW_UTC}} --sdk_container_image=${{ steps.build_vllm_image.outputs.image_url }}' - name: run Pytorch Sentiment Streaming using Hugging Face distilbert-base-uncased model uses: ./.github/actions/gradle-command-self-hosted-action timeout-minutes: 180 diff --git a/.github/workflows/beam_Infrastructure_PolicyEnforcer.yml b/.github/workflows/beam_Infrastructure_PolicyEnforcer.yml new file mode 100644 index 000000000000..22c6f596f5a5 --- /dev/null +++ b/.github/workflows/beam_Infrastructure_PolicyEnforcer.yml @@ -0,0 +1,83 @@ +# 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. + +# This workflow works with the infrastructure policy enforcer to +# generate a report of IAM and Service Account Policies violations + +name: Infrastructure Policy Enforcer + +on: + workflow_dispatch: + schedule: + # Once a week at 9:00 AM on Monday + - cron: '0 9 * * 1' + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' + cancel-in-progress: true + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + contents: read + issues: write + +jobs: + beam_Infrastructure_PolicyEnforcer: + name: Check and Report Infrastructure Policies Violations + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 30 + steps: + - uses: actions/checkout@v4 + + - name: Setup Python + uses: actions/setup-python@v4 + with: + python-version: '3.13' + + - name: Install Python dependencies + working-directory: ./infra/enforcement + run: | + python -m pip install --upgrade pip + pip install -r requirements.txt + + - name: Setup gcloud + uses: google-github-actions/setup-gcloud@v2 + + - name: Run IAM Policy Enforcement + working-directory: ./infra/enforcement + env: + GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + GITHUB_REPOSITORY: ${{ github.repository }} + SMTP_SERVER: smtp.gmail.com + SMTP_PORT: 465 + EMAIL_ADDRESS: ${{ secrets.ISSUE_REPORT_SENDER_EMAIL_ADDRESS }} + EMAIL_PASSWORD: ${{ secrets.ISSUE_REPORT_SENDER_EMAIL_PASSWORD }} + EMAIL_RECIPIENT: "dev@beam.apache.org" + run: python iam.py --action print + + - name: Run Account Keys Policy Enforcement + working-directory: ./infra/enforcement + env: + GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + GITHUB_REPOSITORY: ${{ github.repository }} + SMTP_SERVER: smtp.gmail.com + SMTP_PORT: 465 + EMAIL_ADDRESS: ${{ secrets.ISSUE_REPORT_SENDER_EMAIL_ADDRESS }} + EMAIL_PASSWORD: ${{ secrets.ISSUE_REPORT_SENDER_EMAIL_PASSWORD }} + EMAIL_RECIPIENT: "dev@beam.apache.org" + run: python account_keys.py --action print diff --git a/.github/workflows/beam_Infrastructure_SecurityLogging.yml b/.github/workflows/beam_Infrastructure_SecurityLogging.yml new file mode 100644 index 000000000000..c364056f5683 --- /dev/null +++ b/.github/workflows/beam_Infrastructure_SecurityLogging.yml @@ -0,0 +1,77 @@ +# 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. + +# This workflow works with the GCP security log analyzer to +# generate weekly security reports and initialize log sinks + +name: GCP Security Log Analyzer + +on: + workflow_dispatch: + schedule: + # Once a week at 9:00 AM on Monday + - cron: '0 9 * * 1' + push: + paths: + - 'infra/security/config.yml' + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.sender.login }}' + cancel-in-progress: true + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + contents: read + +jobs: + beam_GCP_Security_LogAnalyzer: + name: GCP Security Log Analysis + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 30 + steps: + - uses: actions/checkout@v4 + + - name: Setup Python + uses: actions/setup-python@v4 + with: + python-version: '3.13' + + - name: Install Python dependencies + working-directory: ./infra/security + run: | + python -m pip install --upgrade pip + pip install -r requirements.txt + + - name: Setup gcloud + uses: google-github-actions/setup-gcloud@v2 + + - name: Initialize Log Sinks + if: github.event_name == 'push' || github.event_name == 'workflow_dispatch' + working-directory: ./infra/security + run: python log_analyzer.py --config config.yml initialize + + - name: Generate Weekly Security Report + if: github.event_name == 'schedule' || github.event_name == 'workflow_dispatch' + working-directory: ./infra/security + env: + SMTP_SERVER: smtp.gmail.com + SMTP_PORT: 465 + EMAIL_ADDRESS: ${{ secrets.ISSUE_REPORT_SENDER_EMAIL_ADDRESS }} + EMAIL_PASSWORD: ${{ secrets.ISSUE_REPORT_SENDER_EMAIL_PASSWORD }} + EMAIL_RECIPIENT: "dev@beam.apache.org" + run: python log_analyzer.py --config config.yml generate-report --dry-run diff --git a/.github/workflows/beam_Infrastructure_ServiceAccountKeys.yml b/.github/workflows/beam_Infrastructure_ServiceAccountKeys.yml new file mode 100644 index 000000000000..cd5eb2a06984 --- /dev/null +++ b/.github/workflows/beam_Infrastructure_ServiceAccountKeys.yml @@ -0,0 +1,68 @@ +# 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. + +# This workflow modifies the GCP Service Account keys and manages the +# storage, saving them onto Google Cloud Secret Manager. It also handles +# the rotation of the keys. + +name: Service Account Keys Management + +on: + workflow_dispatch: + # Trigger when the keys.yaml file is modified on the main branch + push: + branches: + - main + paths: + - 'infra/keys/keys.yaml' + schedule: + # Once a week at 9:00 AM on Monday + - cron: '0 9 * * 1' + +# This ensures that only one workflow run is running at a time, and others are queued. +concurrency: + group: ${{ github.workflow }} + cancel-in-progress: false + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + contents: read + +jobs: + beam_UserRoles: + name: Apply user roles changes + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 30 + steps: + - uses: actions/checkout@v4 + - name: Setup gcloud + uses: google-github-actions/setup-gcloud@v2 + + - name: Setup Python + uses: actions/setup-python@v4 + with: + python-version: '3.13' + + - name: Install Python dependencies + working-directory: ./infra/keys + run: | + python -m pip install --upgrade pip + pip install -r requirements.txt + + - name: Run Service Account Key Management + working-directory: ./infra/keys + run: python keys.py --cron-dry-run diff --git a/.github/workflows/beam_UserRoles.yml b/.github/workflows/beam_Infrastructure_UsersPermissions.yml similarity index 100% rename from .github/workflows/beam_UserRoles.yml rename to .github/workflows/beam_Infrastructure_UsersPermissions.yml diff --git a/.github/workflows/beam_MetricsCredentialsRotation.yml b/.github/workflows/beam_MetricsCredentialsRotation.yml index 0138b9e35571..0eac22a04072 100644 --- a/.github/workflows/beam_MetricsCredentialsRotation.yml +++ b/.github/workflows/beam_MetricsCredentialsRotation.yml @@ -82,17 +82,17 @@ jobs: run: | date=$(date -u +"%Y-%m-%d") echo "date=$date" >> $GITHUB_ENV - - name: Send email - uses: dawidd6/action-send-mail@v3 - if: failure() - with: - server_address: smtp.gmail.com - server_port: 465 - secure: true - username: ${{ secrets.ISSUE_REPORT_SENDER_EMAIL_ADDRESS }} - password: ${{ secrets.ISSUE_REPORT_SENDER_EMAIL_PASSWORD }} - subject: Credentials Rotation Failure on Metrics cluster (${{ env.date }}) - to: dev@beam.apache.org - from: gactions@beam.apache.org - body: | - Something went wrong during the automatic credentials rotation for Metrics Cluster, performed at ${{ env.date }}. It may be necessary to check the state of the cluster certificates. For further details refer to the following links:\n * Failing job: https://github.com/apache/beam/actions/workflows/beam_MetricsCredentialsRotation.yml \n * Job configuration: https://github.com/apache/beam/blob/master/.github/workflows/beam_MetricsCredentialsRotation.yml \n * Cluster URL: https://pantheon.corp.google.com/kubernetes/clusters/details/us-central1-a/metrics/details?mods=dataflow_dev&project=apache-beam-testing +# - name: Send email +# uses: dawidd6/action-send-mail@v3 +# if: failure() +# with: +# server_address: smtp.gmail.com +# server_port: 465 +# secure: true +# username: ${{ secrets.ISSUE_REPORT_SENDER_EMAIL_ADDRESS }} +# password: ${{ secrets.ISSUE_REPORT_SENDER_EMAIL_PASSWORD }} +# subject: Credentials Rotation Failure on Metrics cluster (${{ env.date }}) +# to: dev@beam.apache.org +# from: gactions@beam.apache.org +# body: | +# Something went wrong during the automatic credentials rotation for Metrics Cluster, performed at ${{ env.date }}. It may be necessary to check the state of the cluster certificates. For further details refer to the following links:\n * Failing job: https://github.com/apache/beam/actions/workflows/beam_MetricsCredentialsRotation.yml \n * Job configuration: https://github.com/apache/beam/blob/master/.github/workflows/beam_MetricsCredentialsRotation.yml \n * Cluster URL: https://pantheon.corp.google.com/kubernetes/clusters/details/us-central1-a/metrics/details?mods=dataflow_dev&project=apache-beam-testing diff --git a/.github/workflows/beam_Metrics_Report.yml b/.github/workflows/beam_Metrics_Report.yml index 1d20bd64b7e6..70ed354958b8 100644 --- a/.github/workflows/beam_Metrics_Report.yml +++ b/.github/workflows/beam_Metrics_Report.yml @@ -58,7 +58,7 @@ jobs: (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' ) - + steps: - uses: actions/checkout@v4 - name: Setup environment @@ -82,15 +82,15 @@ jobs: run: | date=$(date -u +"%Y-%m-%d") echo "date=$date" >> $GITHUB_ENV - - name: Send mail - uses: dawidd6/action-send-mail@v3 - with: - server_address: smtp.gmail.com - server_port: 465 - secure: true - username: ${{ secrets.ISSUE_REPORT_SENDER_EMAIL_ADDRESS }} - password: ${{ secrets.ISSUE_REPORT_SENDER_EMAIL_PASSWORD }} - subject: Beam Metrics Report ${{ env.date }} - to: dev@beam.apache.org - from: beamactions@gmail.com - html_body: file://${{ github.workspace }}/.test-infra/jenkins/metrics_report/beam-metrics_report.html +# - name: Send mail +# uses: dawidd6/action-send-mail@v6 +# with: +# server_address: smtp.gmail.com +# server_port: 465 +# secure: true +# username: ${{ secrets.ISSUE_REPORT_SENDER_EMAIL_ADDRESS }} +# password: ${{ secrets.ISSUE_REPORT_SENDER_EMAIL_PASSWORD }} +# subject: Beam Metrics Report ${{ env.date }} +# to: dev@beam.apache.org +# from: beamactions@gmail.com +# html_body: file://${{ github.workspace }}/.test-infra/jenkins/metrics_report/beam-metrics_report.html diff --git a/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml b/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml index 8d780ba46b33..a76c48b8968f 100644 --- a/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml +++ b/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml @@ -83,20 +83,20 @@ jobs: run: | date=$(date -u +"%Y-%m-%d") echo "date=$date" >> $GITHUB_ENV - - name: Send email - uses: dawidd6/action-send-mail@v3 - if: failure() - with: - server_address: smtp.gmail.com - server_port: 465 - secure: true - username: ${{ secrets.ISSUE_REPORT_SENDER_EMAIL_ADDRESS }} - password: ${{ secrets.ISSUE_REPORT_SENDER_EMAIL_PASSWORD }} - subject: BigQueryEarlyRollout Beam Failure (${{ env.date }}) - investigate and escalate quickly - to: datapls-plat-team@google.com # Team at Google responsible for escalating BQ failures - from: gactions@beam.apache.org - body: | - PostCommit Java BigQueryEarlyRollout failed on ${{ env.date }}. This test monitors BigQuery rollouts impacting Beam and should be escalated immediately if a real issue is encountered to pause further rollouts. For further details refer to the following links:\n * Failing job: https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml \n * Job configuration: https://github.com/apache/beam/blob/master/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml +# - name: Send email +# uses: dawidd6/action-send-mail@v3 +# if: failure() +# with: +# server_address: smtp.gmail.com +# server_port: 465 +# secure: true +# username: ${{ secrets.ISSUE_REPORT_SENDER_EMAIL_ADDRESS }} +# password: ${{ secrets.ISSUE_REPORT_SENDER_EMAIL_PASSWORD }} +# subject: BigQueryEarlyRollout Beam Failure (${{ env.date }}) - investigate and escalate quickly +# to: datapls-plat-team@google.com # Team at Google responsible for escalating BQ failures +# from: gactions@beam.apache.org +# body: | +# PostCommit Java BigQueryEarlyRollout failed on ${{ env.date }}. This test monitors BigQuery rollouts impacting Beam and should be escalated immediately if a real issue is encountered to pause further rollouts. For further details refer to the following links:\n * Failing job: https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml \n * Job configuration: https://github.com/apache/beam/blob/master/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml - name: Archive JUnit Test Results uses: actions/upload-artifact@v4 if: ${{ !success() }} diff --git a/.github/workflows/beam_PostCommit_Python.yml b/.github/workflows/beam_PostCommit_Python.yml index fef02dc8f92f..b96067b498e7 100644 --- a/.github/workflows/beam_PostCommit_Python.yml +++ b/.github/workflows/beam_PostCommit_Python.yml @@ -53,21 +53,15 @@ env: jobs: beam_PostCommit_Python: - name: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) (${{ join(matrix.os, ', ') }}) runs-on: ${{ matrix.os }} timeout-minutes: 240 strategy: fail-fast: false matrix: - job_name: [beam_PostCommit_Python] - job_phrase: [Run Python PostCommit] + job_name: ['beam_PostCommit_Python'] + job_phrase: ['Run Python PostCommit'] python_version: ['3.9', '3.10', '3.11', '3.12'] - # Run on both self-hosted and GitHub-hosted runners. - # Some tests (marked require_docker_in_docker) can't run on Beam's - # self-hosted runners due to Docker-in-Docker environment constraint. - # These tests will only execute on ubuntu-latest (GitHub-hosted). - # Context: https://github.com/apache/beam/pull/35585 - # Temporary removed the ubuntu-latest env till resolving deps issues. os: [[self-hosted, ubuntu-20.04, highmem22]] if: | github.event_name == 'workflow_dispatch' || @@ -81,7 +75,7 @@ jobs: with: comment_phrase: ${{ matrix.job_phrase }} ${{ matrix.python_version }} github_token: ${{ secrets.GITHUB_TOKEN }} - github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) (${{ join(matrix.os, ', ') }}) - name: Setup environment uses: ./.github/actions/setup-environment-action with: @@ -106,11 +100,7 @@ jobs: arguments: | -Pjava21Home=$JAVA_HOME_21_X64 \ -PuseWheelDistribution \ - -Pposargs="${{ - contains(matrix.os, 'self-hosted') && - '-m (not require_docker_in_docker)' || - '-m require_docker_in_docker' - }}" \ + -Pposargs="-m (not require_docker_in_docker)" \ -PpythonVersion=${{ matrix.python_version }} \ env: CLOUDSDK_CONFIG: ${{ env.KUBELET_GCLOUD_CONFIG_PATH}} @@ -118,7 +108,7 @@ jobs: uses: actions/upload-artifact@v4 if: failure() with: - name: Python ${{ matrix.python_version }} Test Results + name: Python ${{ matrix.python_version }} Test Results (${{ join(matrix.os, ', ') }}) path: '**/pytest*.xml' - name: Publish Python Test Results uses: EnricoMi/publish-unit-test-result-action@v2 @@ -128,3 +118,4 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/pytest*.xml' large_files: true + check_name: "Python ${{ matrix.python_version }} Test Results (${{ join(matrix.os, ', ') }})" diff --git a/.github/workflows/beam_PostCommit_Python_Arm.yml b/.github/workflows/beam_PostCommit_Python_Arm.yml index 8b990ea01cf5..504ccb659a15 100644 --- a/.github/workflows/beam_PostCommit_Python_Arm.yml +++ b/.github/workflows/beam_PostCommit_Python_Arm.yml @@ -85,7 +85,7 @@ jobs: sudo curl -L https://github.com/docker/compose/releases/download/1.22.0/docker-compose-$(uname -s)-$(uname -m) -o /usr/local/bin/docker-compose sudo chmod +x /usr/local/bin/docker-compose - name: Authenticate on GCP - uses: google-github-actions/auth@v2 + uses: google-github-actions/auth@v3 with: service_account: ${{ secrets.GCP_SA_EMAIL }} credentials_json: ${{ secrets.GCP_SA_KEY }} diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml index 71e032597e6c..ef2768f1efd9 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml @@ -57,7 +57,7 @@ jobs: (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event.comment.body == 'Run Python_Xlang_Gcp_Dataflow PostCommit' runs-on: [self-hosted, ubuntu-20.04, main] - timeout-minutes: 180 + timeout-minutes: 240 name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) strategy: matrix: @@ -95,4 +95,4 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/pytest*.xml' - large_files: true \ No newline at end of file + large_files: true diff --git a/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml index 5f72507bfc20..1ce6d369c216 100644 --- a/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml @@ -16,13 +16,13 @@ # TODO(https://github.com/apache/beam/issues/32492): re-enable the suite # on cron and add release/trigger_all_tests.json to trigger path once fixed. -name: PostCommit XVR GoUsingJava Dataflow (DISABLED) +name: PostCommit XVR GoUsingJava Dataflow on: - # schedule: - # - cron: '45 5/6 * * *' + schedule: + - cron: '45 5/6 * * *' pull_request_target: - paths: ['.github/trigger_files/beam_PostCommit_XVR_GoUsingJava_Dataflow.json'] + paths: ['.github/trigger_files/beam_PostCommit_XVR_GoUsingJava_Dataflow.json', 'release/trigger_all_tests.json'] workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml index 72dcb3f2bd29..1ba0ade06fd0 100644 --- a/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Kafka_IO_Direct.yml @@ -100,7 +100,7 @@ jobs: arguments: | -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - --no-parallel \ + max-workers: 4 - name: Archive JUnit Test Results uses: actions/upload-artifact@v4 if: ${{ !success() }} diff --git a/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml index 1a45436cedf7..c22e0dd4cb07 100644 --- a/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml @@ -21,31 +21,13 @@ on: branches: ['master', 'release-*'] paths: - "sdks/java/io/pulsar/**" - - "sdks/java/io/common/**" - - "sdks/java/core/src/main/**" - - "build.gradle" - - "buildSrc/**" - - "gradle/**" - - "gradle.properties" - - "gradlew" - - "gradle.bat" - - "settings.gradle.kts" - ".github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml" pull_request_target: branches: ['master', 'release-*'] paths: - "sdks/java/io/pulsar/**" - - "sdks/java/io/common/**" - - "sdks/java/core/src/main/**" + - ".github/workflows/beam_PreCommit_Java_Pulsar_IO_Direct.yml" - 'release/trigger_all_tests.json' - - '.github/trigger_files/beam_PreCommit_Java_Pulsar_IO_Direct.json' - - "build.gradle" - - "buildSrc/**" - - "gradle/**" - - "gradle.properties" - - "gradlew" - - "gradle.bat" - - "settings.gradle.kts" issue_comment: types: [created] schedule: @@ -110,6 +92,13 @@ jobs: arguments: | -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ + - name: run Pulsar IO IT script + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:io:pulsar:integrationTest + arguments: | + -PdisableSpotlessCheck=true \ + -PdisableCheckStyle=true \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v4 if: ${{ !success() }} @@ -135,4 +124,4 @@ jobs: if: always() with: name: Publish SpotBugs - path: '**/build/reports/spotbugs/*.html' \ No newline at end of file + path: '**/build/reports/spotbugs/*.html' diff --git a/.github/workflows/beam_PreCommit_Python.yml b/.github/workflows/beam_PreCommit_Python.yml index 3ad9020f17f7..db56f526a02d 100644 --- a/.github/workflows/beam_PreCommit_Python.yml +++ b/.github/workflows/beam_PreCommit_Python.yml @@ -53,6 +53,23 @@ env: DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + # Aggressive stability settings for flaky CI environment + PYTHONHASHSEED: "0" + OMP_NUM_THREADS: "1" + OPENBLAS_NUM_THREADS: "1" + # gRPC stability - more conservative for unstable networks + GRPC_ARG_KEEPALIVE_TIME_MS: "10000" + GRPC_ARG_KEEPALIVE_TIMEOUT_MS: "15000" + GRPC_ARG_KEEPALIVE_PERMIT_WITHOUT_CALLS: "1" + GRPC_ARG_HTTP2_MAX_PINGS_WITHOUT_DATA: "0" + GRPC_ARG_MAX_RECONNECT_BACKOFF_MS: "30000" + # Beam-specific - very generous timeouts + BEAM_RETRY_MAX_ATTEMPTS: "5" + BEAM_RETRY_INITIAL_DELAY_MS: "5000" + BEAM_RETRY_MAX_DELAY_MS: "120000" + # Force stable execution + BEAM_TESTING_FORCE_SINGLE_BUNDLE: "true" + BEAM_TESTING_DETERMINISTIC_ORDER: "true" jobs: beam_PreCommit_Python: @@ -91,6 +108,23 @@ jobs: PY_VER_CLEAN=${PY_VER//.} echo "py_ver_clean=$PY_VER_CLEAN" >> $GITHUB_OUTPUT - name: Run pythonPreCommit + env: + TOX_TESTENV_PASSENV: "DOCKER_*,TESTCONTAINERS_*,TC_*,BEAM_*,GRPC_*,OMP_*,OPENBLAS_*,PYTHONHASHSEED,PYTEST_*" + # Aggressive retry and timeout settings for flaky CI + PYTEST_ADDOPTS: "-v --tb=short --maxfail=5 --durations=30 --reruns=5 --reruns-delay=15 --timeout=600 --disable-warnings" + # Container stability - much more generous timeouts + TC_TIMEOUT: "300" + TC_MAX_TRIES: "15" + TC_SLEEP_TIME: "5" + # Additional gRPC stability for flaky environment + GRPC_ARG_MAX_CONNECTION_IDLE_MS: "60000" + GRPC_ARG_HTTP2_BDP_PROBE: "1" + GRPC_ARG_SO_REUSEPORT: "1" + # Force sequential execution to reduce load + PYTEST_XDIST_WORKER_COUNT: "1" + # Additional gRPC settings + GRPC_ARG_MAX_RECONNECT_BACKOFF_MS: "120000" + GRPC_ARG_INITIAL_RECONNECT_BACKOFF_MS: "2000" uses: ./.github/actions/gradle-command-self-hosted-action with: gradle-command: :sdks:python:test-suites:tox:py${{steps.set_py_ver_clean.outputs.py_ver_clean}}:preCommitPy${{steps.set_py_ver_clean.outputs.py_ver_clean}} @@ -110,4 +144,14 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/pytest*.xml' - large_files: true \ No newline at end of file + large_files: true + - name: Cleanup + if: always() + run: | + # Kill any remaining processes + sudo pkill -f "gradle" || true + sudo pkill -f "java" || true + sudo pkill -f "python.*pytest" || true + # Clean up temp files + sudo rm -rf /tmp/beam-* || true + sudo rm -rf /tmp/gradle-* || true \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Python_Coverage.yml b/.github/workflows/beam_PreCommit_Python_Coverage.yml index 093f7026b13a..7c675c01183b 100644 --- a/.github/workflows/beam_PreCommit_Python_Coverage.yml +++ b/.github/workflows/beam_PreCommit_Python_Coverage.yml @@ -58,36 +58,73 @@ env: jobs: beam_PreCommit_Python_Coverage: - name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) - runs-on: [self-hosted, ubuntu-20.04, main] + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) (${{ join(matrix.os, ', ') }}) + runs-on: ${{ matrix.os }} strategy: + fail-fast: false matrix: job_name: [beam_PreCommit_Python_Coverage] job_phrase: [Run Python_Coverage PreCommit] + python_version: ['3.9'] + # Run on both self-hosted and GitHub-hosted runners. + # Some tests (marked require_docker_in_docker) can't run on Beam's + # self-hosted runners due to Docker-in-Docker environment constraint. + # These tests will only execute on ubuntu-latest (GitHub-hosted). + # Context: https://github.com/apache/beam/pull/35585 + os: [[self-hosted, ubuntu-20.04, highmem], [ubuntu-latest]] timeout-minutes: 180 if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || - github.event.comment.body == 'Run Python_Coverage PreCommit' + startswith(github.event.comment.body, 'Run Python_Coverage PreCommit 3.') steps: - uses: actions/checkout@v4 - name: Setup repository uses: ./.github/actions/setup-action with: - comment_phrase: ${{ matrix.job_phrase }} + comment_phrase: ${{ matrix.job_phrase }} ${{ matrix.python_version }} github_token: ${{ secrets.GITHUB_TOKEN }} - github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) (${{ join(matrix.os, ', ') }}) - name: Setup environment uses: ./.github/actions/setup-environment-action with: java-version: default - python-version: default + python-version: ${{ matrix.python_version }} + - name: Start DinD + uses: ./.github/actions/dind-up-action + id: dind + if: contains(matrix.os, 'self-hosted') + with: + # Enable all the new features + cleanup-dind-on-start: "true" + smoke-test-port-mapping: "true" + prime-testcontainers: "true" + tmpfs-run-size: 2g + tmpfs-varrun-size: 4g + export-gh-env: "true" - name: Run preCommitPyCoverage + env: + DOCKER_HOST: ${{ contains(matrix.os, 'self-hosted') && steps.dind.outputs.docker-host || '' }} + TOX_TESTENV_PASSENV: "DOCKER_*,TESTCONTAINERS_*,TC_*,BEAM_*,GRPC_*,OMP_*,OPENBLAS_*,PYTHONHASHSEED,PYTEST_*" + TESTCONTAINERS_HOST_OVERRIDE: ${{ contains(matrix.os, 'self-hosted') && env.DIND_IP || '' }} + TESTCONTAINERS_DOCKER_SOCKET_OVERRIDE: "/var/run/docker.sock" + TESTCONTAINERS_RYUK_DISABLED: "false" + TESTCONTAINERS_RYUK_CONTAINER_PRIVILEGED: "true" + PYTEST_ADDOPTS: "-v --tb=short --maxfail=3 --durations=20 --reruns=2 --reruns-delay=5" + TC_TIMEOUT: "120" + TC_MAX_TRIES: "120" + TC_SLEEP_TIME: "1" uses: ./.github/actions/gradle-command-self-hosted-action with: gradle-command: :sdks:python:test-suites:tox:py39:preCommitPyCoverage + arguments: | + -Pposargs="${{ + contains(matrix.os, 'self-hosted') && + '-m (not require_docker_in_docker)' || + '-m require_docker_in_docker' + }}" - uses: codecov/codecov-action@v3 with: flags: python @@ -96,13 +133,16 @@ jobs: uses: actions/upload-artifact@v4 if: failure() with: - name: Python Test Results + name: Python ${{ matrix.python_version }} Test Results (${{ join(matrix.os, ', ') }}) path: '**/pytest*.xml' - name: Publish Python Test Results + env: + DOCKER_HOST: "" # Unset DOCKER_HOST to run on host Docker daemon uses: EnricoMi/publish-unit-test-result-action@v2 if: always() with: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/pytest*.xml' - large_files: true \ No newline at end of file + large_files: true + check_name: "Python ${{ matrix.python_version }} Test Results (${{ join(matrix.os, ', ') }})" diff --git a/.github/workflows/beam_PreCommit_Python_Examples.yml b/.github/workflows/beam_PreCommit_Python_Examples.yml index c76d140eadeb..68acb72e0d61 100644 --- a/.github/workflows/beam_PreCommit_Python_Examples.yml +++ b/.github/workflows/beam_PreCommit_Python_Examples.yml @@ -53,6 +53,7 @@ env: DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }} GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + ALLOYDB_PASSWORD: ${{ secrets.ALLOYDB_PASSWORD }} jobs: beam_PreCommit_Python_Examples: @@ -110,4 +111,4 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/pytest*.xml' - large_files: true \ No newline at end of file + large_files: true diff --git a/.github/workflows/beam_PreCommit_Python_ML.yml b/.github/workflows/beam_PreCommit_Python_ML.yml index de920428a24b..471dcf953be5 100644 --- a/.github/workflows/beam_PreCommit_Python_ML.yml +++ b/.github/workflows/beam_PreCommit_Python_ML.yml @@ -57,7 +57,7 @@ env: jobs: beam_PreCommit_Python_ML: - name: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) (${{ join(matrix.os, ', ') }}) runs-on: ${{ matrix.os }} timeout-minutes: 180 strategy: @@ -70,9 +70,18 @@ jobs: # Some tests (marked require_docker_in_docker) can't run on Beam's # self-hosted runners due to Docker-in-Docker environment constraint. # These tests will only execute on ubuntu-latest (GitHub-hosted). - # Context: https://github.com/apache/beam/pull/35585 - # Temporary removed the ubuntu-latest env till resolving deps issues. - os: [[self-hosted, ubuntu-20.04, main]] + # Context: https://github.com/apache/beam/pull/35585. + os: [[self-hosted, ubuntu-20.04, main], [ubuntu-latest]] + exclude: + # Temporary exclude Python 3.9, 3.10, 3.11 from ubuntu-latest. This + # results in pip dependency resolution exceeded maximum depth issue. + # Context: https://github.com/apache/beam/pull/35816. + - python_version: '3.9' + os: [ubuntu-latest] + - python_version: '3.10' + os: [ubuntu-latest] + - python_version: '3.11' + os: [ubuntu-latest] if: | github.event_name == 'push' || github.event_name == 'pull_request_target' || @@ -86,7 +95,7 @@ jobs: with: comment_phrase: ${{ matrix.job_phrase }} ${{ matrix.python_version }} github_token: ${{ secrets.GITHUB_TOKEN }} - github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }} ${{ matrix.python_version }}) (${{ join(matrix.os, ', ') }}) - name: Setup environment uses: ./.github/actions/setup-environment-action with: @@ -113,7 +122,7 @@ jobs: uses: actions/upload-artifact@v4 if: failure() with: - name: Python ${{ matrix.python_version }} Test Results + name: Python ${{ matrix.python_version }} Test Results ${{ matrix.os }} path: '**/pytest*.xml' - name: Publish Python Test Results uses: EnricoMi/publish-unit-test-result-action@v2 @@ -123,3 +132,4 @@ jobs: comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/pytest*.xml' large_files: true + check_name: "Python ${{ matrix.python_version }} Test Results (${{ join(matrix.os, ', ') }})" diff --git a/.github/workflows/beam_PreCommit_Python_Transforms.yml b/.github/workflows/beam_PreCommit_Python_Transforms.yml index 8753777057c6..4982dd2f7263 100644 --- a/.github/workflows/beam_PreCommit_Python_Transforms.yml +++ b/.github/workflows/beam_PreCommit_Python_Transforms.yml @@ -111,4 +111,4 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/pytest*.xml' - large_files: true \ No newline at end of file + large_files: true diff --git a/.github/workflows/beam_PreCommit_Whitespace.yml b/.github/workflows/beam_PreCommit_Whitespace.yml index 8e5b3f0200c2..a378991dcfcb 100644 --- a/.github/workflows/beam_PreCommit_Whitespace.yml +++ b/.github/workflows/beam_PreCommit_Whitespace.yml @@ -86,3 +86,8 @@ jobs: uses: ./.github/actions/gradle-command-self-hosted-action with: gradle-command: :whitespacePreCommit + - name: validate CHANGES.md + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :validateChanges + diff --git a/.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml b/.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml index 49fcff4e91f0..05816350e2da 100644 --- a/.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml +++ b/.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml @@ -94,7 +94,7 @@ jobs: - name: Set up Docker Buildx uses: docker/setup-buildx-action@v1 - name: Authenticate on GCP - uses: google-github-actions/auth@v2 + uses: google-github-actions/auth@v3 with: service_account: ${{ secrets.GCP_SA_EMAIL }} credentials_json: ${{ secrets.GCP_SA_KEY }} diff --git a/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml b/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml index e70ec88d1abd..770f99eb0e13 100644 --- a/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml +++ b/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml @@ -75,7 +75,7 @@ jobs: with: python-version: ${{ matrix.python_version }} - name: Authenticate on GCP - uses: google-github-actions/auth@v2 + uses: google-github-actions/auth@v3 with: service_account: ${{ secrets.GCP_SA_EMAIL }} credentials_json: ${{ secrets.GCP_SA_KEY }} diff --git a/.github/workflows/beam_StressTests_Java_KafkaIO.yml b/.github/workflows/beam_StressTests_Java_KafkaIO.yml index fc4649eee0b3..1230e81324b5 100644 --- a/.github/workflows/beam_StressTests_Java_KafkaIO.yml +++ b/.github/workflows/beam_StressTests_Java_KafkaIO.yml @@ -17,7 +17,7 @@ name: StressTests Java KafkaIO on: schedule: - - cron: '0 10 * * 0' + - cron: '0 14 * * 0' workflow_dispatch: #Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event diff --git a/.github/workflows/build_release_candidate.yml b/.github/workflows/build_release_candidate.yml index b9283665f03c..f1a52000af4a 100644 --- a/.github/workflows/build_release_candidate.yml +++ b/.github/workflows/build_release_candidate.yml @@ -40,6 +40,8 @@ on: beam_site_pr: create the documentation update PR against apache/beam-site. -- prism: build and upload the artifacts to the release for this tag + -- + managed_io_docs_pr: create the managed-io.md update PR against apache/beam. required: true default: | {java_artifacts: "no", @@ -47,7 +49,8 @@ on: docker_artifacts: "no", python_artifacts: "no", beam_site_pr: "no", - prism: "no"} + prism: "no", + managed_io_docs_pr: "no"} env: DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }} @@ -71,7 +74,7 @@ jobs: 11 - name: Import GPG key id: import_gpg - uses: crazy-max/ghaction-import-gpg@111c56156bcc6918c056dbef52164cfa583dc549 + uses: crazy-max/ghaction-import-gpg@e89d40939c28e39f97cf32126055eeae86ba74ec with: gpg_private_key: ${{ secrets.GPG_PRIVATE_KEY }} - name: Auth for nexus @@ -123,7 +126,7 @@ jobs: java-version: '11' - name: Import GPG key id: import_gpg - uses: crazy-max/ghaction-import-gpg@111c56156bcc6918c056dbef52164cfa583dc549 + uses: crazy-max/ghaction-import-gpg@e89d40939c28e39f97cf32126055eeae86ba74ec with: gpg_private_key: ${{ secrets.GPG_PRIVATE_KEY }} - name: stage source @@ -190,7 +193,7 @@ jobs: disable-cache: true - name: Import GPG key id: import_gpg - uses: crazy-max/ghaction-import-gpg@111c56156bcc6918c056dbef52164cfa583dc549 + uses: crazy-max/ghaction-import-gpg@e89d40939c28e39f97cf32126055eeae86ba74ec with: gpg_private_key: ${{ secrets.GPG_PRIVATE_KEY }} - name: Install dependencies @@ -328,7 +331,7 @@ jobs: with: python-version: '3.9' - name: Install node - uses: actions/setup-node@v4 + uses: actions/setup-node@v5 with: node-version: '16' - name: Install Java 21 @@ -451,7 +454,7 @@ jobs: go-version: '1.24' - name: Import GPG key id: import_gpg - uses: crazy-max/ghaction-import-gpg@111c56156bcc6918c056dbef52164cfa583dc549 + uses: crazy-max/ghaction-import-gpg@e89d40939c28e39f97cf32126055eeae86ba74ec with: gpg_private_key: ${{ secrets.GPG_PRIVATE_KEY }} - name: Build prism artifacts @@ -545,3 +548,69 @@ jobs: svn add --force --parents prism svn status svn commit -m "Staging Prism artifacts for Apache Beam ${RELEASE} RC${RC_NUM}" --non-interactive --username "${{ github.event.inputs.APACHE_ID }}" --password "${{ github.event.inputs.APACHE_PASSWORD }}" + + managed_io_docs_pr: + if: ${{ fromJson(github.event.inputs.STAGE).managed_io_docs_pr == 'yes'}} + runs-on: ubuntu-22.04 + env: + BRANCH_NAME: updates_managed_io_docs_${{ github.event.inputs.RELEASE }}_rc${{ github.event.inputs.RC }} + BEAM_ROOT_DIR: ${{ github.workspace }}/beam + MANAGED_IO_DOCS_PATH: website/www/site/content/en/documentation/io/managed-io.md + steps: + - name: Checkout Beam Repo + uses: actions/checkout@v4 + with: + ref: "v${{ github.event.inputs.RELEASE }}-RC${{ github.event.inputs.RC }}" + repository: apache/beam + path: beam + token: ${{ github.event.inputs.REPO_TOKEN }} + persist-credentials: false + - name: Install Python 3.9 + uses: actions/setup-python@v5 + with: + python-version: '3.9' + - name: Install Java 11 + uses: actions/setup-java@v4 + with: + distribution: 'temurin' + java-version: '11' + - name: Remove default github maven configuration + # This step is a workaround to avoid a decryption issue of Beam's + # net.linguica.gradle.maven.settings plugin and github's provided maven + # settings.xml file + run: rm ~/.m2/settings.xml || true + - name: Install SDK + working-directory: beam/sdks/python + run: | + pip install -e. + - name: Build Expansion Service Jar + working-directory: beam + run: | + ./gradlew sdks:java:io:expansion-service:shadowJar + - name: Build GCP Expansion Service Jar + working-directory: beam + run: | + ./gradlew sdks:java:io:google-cloud-platform:expansion-service:shadowJar + - name: Generate Managed IO Docs + working-directory: beam/sdks/python + run: | + python gen_managed_doc.py --output_location ${{ runner.temp }}/managed-io.md + - name: Create commit on beam branch + working-directory: beam + run: | + git fetch origin master + git checkout -b $BRANCH_NAME origin/master + mv ${{ runner.temp }}/managed-io.md ${{ env.MANAGED_IO_DOCS_PATH }} + git config user.name $GITHUB_ACTOR + git config user.email actions@"$RUNNER_NAME".local + git add ${{ env.MANAGED_IO_DOCS_PATH }} + git commit --allow-empty -m "Update managed-io.md for release ${{ github.event.inputs.RELEASE }}-RC${{ github.event.inputs.RC }}." + git push -f --set-upstream origin $BRANCH_NAME + - name: Create beam PR + working-directory: beam + env: + GH_TOKEN: ${{ github.event.inputs.REPO_TOKEN }} + PR_TITLE: "Update managed-io.md for release ${{ github.event.inputs.RELEASE }}-RC${{ github.event.inputs.RC }}" + PR_BODY: "Content generated from release ${{ github.event.inputs.RELEASE }}-RC${{ github.event.inputs.RC }}." + run: | + gh pr create -t "$PR_TITLE" -b "$PR_BODY" --base master --repo apache/beam diff --git a/.github/workflows/build_runner_image.yml b/.github/workflows/build_runner_image.yml index 0f17a9073daf..ddd01d7644e4 100644 --- a/.github/workflows/build_runner_image.yml +++ b/.github/workflows/build_runner_image.yml @@ -47,7 +47,7 @@ jobs: - name: Set up Docker Buildx uses: docker/setup-buildx-action@v1 - name: Build and Load to docker - uses: docker/build-push-action@v4 + uses: docker/build-push-action@v6 with: context: ${{ env.working-directory }} load: true @@ -57,7 +57,7 @@ jobs: - name: Push Docker image if: github.ref == 'refs/heads/master' id: docker_build - uses: docker/build-push-action@v4 + uses: docker/build-push-action@v6 with: context: ${{ env.working-directory }} push: true diff --git a/.github/workflows/finalize_release.yml b/.github/workflows/finalize_release.yml index 5180fa9a4818..01daab24db93 100644 --- a/.github/workflows/finalize_release.yml +++ b/.github/workflows/finalize_release.yml @@ -93,8 +93,10 @@ jobs: echo "::add-mask::$PYPI_PASSWORD" - name: Validate PyPi id/password run: | - echo "::add-mask::${{ github.event.inputs.PYPI_API_TOKEN }}" - if [ "${{ github.event.inputs.PYPI_API_TOKEN }}" == "" ] + # Workaround for Actions bug - https://github.com/actions/runner/issues/643 + PYPI_API_TOKEN=$(jq -r '.inputs.PYPI_API_TOKEN' $GITHUB_EVENT_PATH) + echo "::add-mask::$PYPI_API_TOKEN" + if [ "$PYPI_API_TOKEN" == "" ] then echo "Must provide a PyPi password to publish artifacts to PyPi" exit 1 diff --git a/.github/workflows/load-tests-pipeline-options/beam_Inference_Python_Benchmarks_Dataflow_VLLM_Gemma_Batch.txt b/.github/workflows/load-tests-pipeline-options/beam_Inference_Python_Benchmarks_Dataflow_VLLM_Gemma_Batch.txt new file mode 100644 index 000000000000..6101fe5da457 --- /dev/null +++ b/.github/workflows/load-tests-pipeline-options/beam_Inference_Python_Benchmarks_Dataflow_VLLM_Gemma_Batch.txt @@ -0,0 +1,36 @@ +# 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. + +--runner=DataflowRunner +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--staging_location=gs://temp-storage-for-perf-tests/loadtests +--input=gs://apache-beam-ml/testing/inputs/sentences_50k.txt +--machine_type=n1-standard-8 +--worker_zone=us-central1-b +--disk_size_gb=50 +--input_options={} +--num_workers=8 +--max_num_workers=25 +--autoscaling_algorithm=THROUGHPUT_BASED +--publish_to_big_query=true +--sdk_location=container +--output_table=apache-beam-testing.beam_run_inference.result_gemma_vllm_batch +--metrics_dataset=beam_run_inference +--metrics_table=gemma_vllm_batch +--influx_measurement=gemma_vllm_batch +--model_gcs_path=gs://apache-beam-ml/models/gemma-2b-it +--dataflow_service_options=worker_accelerator=type:nvidia-tesla-t4;count:1;install-nvidia-driver +--experiments=use_runner_v2 \ No newline at end of file diff --git a/.github/workflows/pr-bot-new-prs.yml b/.github/workflows/pr-bot-new-prs.yml index 0f17d662db9c..ac1a599e8539 100644 --- a/.github/workflows/pr-bot-new-prs.yml +++ b/.github/workflows/pr-bot-new-prs.yml @@ -35,7 +35,7 @@ jobs: steps: - uses: actions/checkout@v4 - name: Setup Node - uses: actions/setup-node@v4 + uses: actions/setup-node@v5 with: node-version: 16 - name: Install pr-bot npm dependencies diff --git a/.github/workflows/pr-bot-pr-updates.yml b/.github/workflows/pr-bot-pr-updates.yml index 02c8a2473ff3..962dc5e2d9a9 100644 --- a/.github/workflows/pr-bot-pr-updates.yml +++ b/.github/workflows/pr-bot-pr-updates.yml @@ -40,7 +40,7 @@ jobs: with: ref: 'master' - name: Setup Node - uses: actions/setup-node@v4 + uses: actions/setup-node@v5 with: node-version: 16 - name: Install pr-bot npm dependencies diff --git a/.github/workflows/pr-bot-prs-needing-attention.yml b/.github/workflows/pr-bot-prs-needing-attention.yml index 95be91e8dcb4..dba7a25a94f8 100644 --- a/.github/workflows/pr-bot-prs-needing-attention.yml +++ b/.github/workflows/pr-bot-prs-needing-attention.yml @@ -35,7 +35,7 @@ jobs: steps: - uses: actions/checkout@v4 - name: Setup Node - uses: actions/setup-node@v4 + uses: actions/setup-node@v5 with: node-version: 16 - name: Install pr-bot npm dependencies diff --git a/.github/workflows/refresh_looker_metrics.yml b/.github/workflows/refresh_looker_metrics.yml index 17c993f96a02..7285d77e50a3 100644 --- a/.github/workflows/refresh_looker_metrics.yml +++ b/.github/workflows/refresh_looker_metrics.yml @@ -43,7 +43,7 @@ jobs: python-version: 3.11 - run: pip install requests google-cloud-storage looker-sdk - name: Authenticate on GCP - uses: google-github-actions/auth@v2 + uses: google-github-actions/auth@v3 with: service_account: ${{ secrets.GCP_SA_EMAIL }} credentials_json: ${{ secrets.GCP_SA_KEY }} diff --git a/.github/workflows/reportGenerator.yml b/.github/workflows/reportGenerator.yml index 91890b12ff00..da8c7ca206ac 100644 --- a/.github/workflows/reportGenerator.yml +++ b/.github/workflows/reportGenerator.yml @@ -28,7 +28,7 @@ jobs: steps: - uses: actions/checkout@v4 - name: Setup Node - uses: actions/setup-node@v4 + uses: actions/setup-node@v5 with: node-version: 16 - run: | diff --git a/.github/workflows/republish_released_docker_containers.yml b/.github/workflows/republish_released_docker_containers.yml index 2a1bda2a6eb3..2cf58b4be0e6 100644 --- a/.github/workflows/republish_released_docker_containers.yml +++ b/.github/workflows/republish_released_docker_containers.yml @@ -32,7 +32,7 @@ on: - cron: "0 6 * * 1" env: docker_registry: gcr.io - release: "${{ github.event.inputs.RELEASE || '2.66.0' }}" + release: "${{ github.event.inputs.RELEASE || '2.67.0' }}" rc: "${{ github.event.inputs.RC || '2' }}" jobs: @@ -72,7 +72,7 @@ jobs: with: python-version: '3.9' - name: Authenticate on GCP - uses: google-github-actions/auth@v2 + uses: google-github-actions/auth@v3 with: service_account: ${{ secrets.GCP_SA_EMAIL }} credentials_json: ${{ secrets.GCP_SA_KEY }} diff --git a/.github/workflows/typescript_tests.yml b/.github/workflows/typescript_tests.yml index a3f929817661..55f0ab7898ba 100644 --- a/.github/workflows/typescript_tests.yml +++ b/.github/workflows/typescript_tests.yml @@ -57,7 +57,7 @@ jobs: persist-credentials: false submodules: recursive - name: Install node - uses: actions/setup-node@v4 + uses: actions/setup-node@v5 with: node-version: '16' - run: npm ci @@ -88,7 +88,7 @@ jobs: persist-credentials: false submodules: recursive - name: Install Node - uses: actions/setup-node@v4 + uses: actions/setup-node@v5 with: node-version: '16' - name: Install Python @@ -143,7 +143,7 @@ jobs: persist-credentials: false submodules: recursive - name: Install node - uses: actions/setup-node@v4 + uses: actions/setup-node@v5 with: node-version: '16' - name: Install python diff --git a/.test-infra/tools/refresh_looker_metrics.py b/.test-infra/tools/refresh_looker_metrics.py index 200be34f3fd1..a4c6999be775 100644 --- a/.test-infra/tools/refresh_looker_metrics.py +++ b/.test-infra/tools/refresh_looker_metrics.py @@ -42,6 +42,7 @@ ("80", ["253", "254", "255", "256", "257"]), # PyTorch Resnet 152 Tesla T4 ("82", ["263", "264", "265", "266", "267"]), # PyTorch Sentiment Streaming DistilBERT base uncased ("85", ["268", "269", "270", "271", "272"]), # PyTorch Sentiment Batch DistilBERT base uncased + ("86", ["284", "285", "286", "287", "288"]), # VLLM Batch Gemma ] diff --git a/CHANGES.md b/CHANGES.md index 206ac3ba11ad..6c7c6942dd41 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -50,6 +50,7 @@ * Fixed X (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). ## Security Fixes + * Fixed [CVE-YYYY-NNNN](https://www.cve.org/CVERecord?id=CVE-YYYY-NNNN) (Java/Python/Go) ([#X](https://github.com/apache/beam/issues/X)). ## Known Issues @@ -58,20 +59,72 @@ * ([#X](https://github.com/apache/beam/issues/X)). --> +# [2.69.0] - Unreleased + +## Highlights + +* New highly anticipated feature X added to Python SDK ([#X](https://github.com/apache/beam/issues/X)). +* New highly anticipated feature Y added to Java SDK ([#Y](https://github.com/apache/beam/issues/Y)). +* (Python) Add YAML Editor and Visualization Panel ([#35772](https://github.com/apache/beam/issues/35772)). + +## I/Os + +* Support for X source added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). + +## New Features / Improvements + +* X feature added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). +* Python examples added for CloudSQL enrichment handler on [Beam website](https://beam.apache.org/documentation/transforms/python/elementwise/enrichment-cloudsql/) (Python) ([#35473](https://github.com/apache/beam/issues/36095)). +* Support for batch mode execution in WriteToPubSub transform added (Python) ([#35990](https://github.com/apache/beam/issues/35990)). + +## Breaking Changes + +* X behavior was changed ([#X](https://github.com/apache/beam/issues/X)). + +## Deprecations + +* X behavior is deprecated and will be removed in X versions ([#X](https://github.com/apache/beam/issues/X)). + +## Bugfixes + +* Fixed X (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). +* PulsarIO has now changed support status from incomplete to experimental. Both read and writes should now minimally + function (un-partitioned topics, without schema support, timestamp ordered messages for read) (Java) + ([#36141](https://github.com/apache/beam/issues/36141)). + +## Known Issues + +* ([#X](https://github.com/apache/beam/issues/X)). + # [2.68.0] - Unreleased ## Highlights * New highly anticipated feature X added to Python SDK ([#X](https://github.com/apache/beam/issues/X)). * New highly anticipated feature Y added to Java SDK ([#Y](https://github.com/apache/beam/issues/Y)). +* [Python] Prism runner now enabled by default for most Python pipelines using the direct runner ([#34612](https://github.com/apache/beam/pull/34612)). This may break some tests, see https://github.com/apache/beam/pull/34612 for details on how to handle issues. ## I/Os * Support for X source added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). +* Upgraded Iceberg dependency to 1.9.2 ([#35981](https://github.com/apache/beam/pull/35981)) ## New Features / Improvements * X feature added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). +* BigtableRead Connector for BeamYaml added with new Config Param ([#35696](https://github.com/apache/beam/pull/35696)) +* MongoDB Java driver upgraded from 3.12.11 to 5.5.0 with API refactoring and GridFS implementation updates (Java) ([#35946](https://github.com/apache/beam/pull/35946)). +* Introduced a dedicated module for JUnit-based testing support: `sdks/java/testing/junit`, which provides `TestPipelineExtension` for JUnit 5 while maintaining backward compatibility with existing JUnit 4 `TestRule`-based tests (Java) ([#18733](https://github.com/apache/beam/issues/18733), [#35688](https://github.com/apache/beam/pull/35688)). + - To use JUnit 5 with Beam tests, add a test-scoped dependency on `org.apache.beam:beam-sdks-java-testing-junit`. +* Google CloudSQL enrichment handler added (Python) ([#34398](https://github.com/apache/beam/pull/34398)). + Beam now supports data enrichment capabilities using SQL databases, with built-in support for: + - Managed PostgreSQL, MySQL, and Microsoft SQL Server instances on CloudSQL + - Unmanaged SQL database instances not hosted on CloudSQL (e.g., self-hosted or on-premises databases) +* [Python] Added the `ReactiveThrottler` and `ThrottlingSignaler` classes to streamline throttling behavior in DoFns, expose throttling mechanisms for users ([#35984](https://github.com/apache/beam/pull/35984)) +* Added a pipeline option to specify the processing timeout for a single element by any PTransform (Java/Python/Go) ([#35174](https://github.com/apache/beam/issues/35174)). + - When specified, the SDK harness automatically restarts if an element takes too long to process. Beam runner may then retry processing of the same work item. + - Use the `--element_processing_timeout_minutes` option to reduce the chance of having stalled pipelines due to unexpected cases of slow processing, where slowness might not happen again if processing of the same element is retried. +* (Python) Adding GCP Spanner Change Stream support for Python (apache_beam.io.gcp.spanner) ([#24103](https://github.com/apache/beam/issues/24103)). ## Breaking Changes @@ -81,40 +134,49 @@ * Upgraded Beam vendored Calcite to 1.40.0 for Beam SQL ([#35483](https://github.com/apache/beam/issues/35483)), which improves support for BigQuery and other SQL dialects. Note: Minor behavior changes are observed such as output significant digits related to casting. +* (Python) The deterministic fallback coder for complex types like NamedTuple, Enum, and dataclasses now uses cloudpickle instead of dill. If your pipeline is affected, you may see a warning like: "Using fallback deterministic coder for type X...". You can revert to the previous behavior by using the pipeline option `--update_compatibility_version=2.67.0` ([35725](https://github.com/apache/beam/pull/35725)). Report any pickling related issues to [#34903](https://github.com/apache/beam/issues/34903) +* (Python) Prism runner now enabled by default for most Python pipelines using the direct runner ([#34612](https://github.com/apache/beam/pull/34612)). This may break some tests, see https://github.com/apache/beam/pull/34612 for details on how to handle issues. +* Dropped Java 8 support for [IO expansion-service](https://central.sonatype.com/artifact/org.apache.beam/beam-sdks-java-io-expansion-service). Cross-language pipelines using this expansion service will need a Java11+ runtime ([#35981](https://github.com/apache/beam/pull/35981)). ## Deprecations * X behavior is deprecated and will be removed in X versions ([#X](https://github.com/apache/beam/issues/X)). +* Python SDK native SpannerIO (apache_beam/io/gcp/experimental/spannerio) is deprecated. Use cross-language wrapper + (apache_beam/io/gcp/spanner) instead (Python) ([#35860](https://github.com/apache/beam/issues/35860)). +* Samza runner is deprecated and scheduled for removal in Beam 3.0 ([#35448](https://github.com/apache/beam/issues/35448)). +* Twister2 runner is deprecated and scheduled for removal in Beam 3.0 ([#35905](https://github.com/apache/beam/issues/35905))). ## Bugfixes -* Fixed X (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). +* (Python) Fixed Java YAML provider fails on Windows ([#35617](https://github.com/apache/beam/issues/35617)). +* Fixed BigQueryIO creating temporary datasets in wrong project when temp_dataset is specified with a different project than the pipeline project. For some jobs, temporary datasets will now be created in the correct project (Python) ([#35813](https://github.com/apache/beam/issues/35813)). +* (Go) Fix duplicates due to reads after blind writes to Bag State ([#35869](https://github.com/apache/beam/issues/35869)). + * Earlier Go SDK versions can avoid the issue by not reading in the same call after a blind write. ## Known Issues * ([#X](https://github.com/apache/beam/issues/X)). -# [2.67.0] - Unreleased +# [2.67.0] - 2025-08-12 ## Highlights -* New highly anticipated feature X added to Python SDK ([#X](https://github.com/apache/beam/issues/X)). -* New highly anticipated feature Y added to Java SDK ([#Y](https://github.com/apache/beam/issues/Y)). -* [Python] Prism runner now enabled by default for most Python pipelines using the direct runner ([#34612](https://github.com/apache/beam/pull/34612)). This may break some tests, see https://github.com/apache/beam/pull/34612 for details on how to handle issues. ## I/Os -* Support for X source added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). * Debezium IO upgraded to 3.1.1 requires Java 17 (Java) ([#34747](https://github.com/apache/beam/issues/34747)). * Add support for streaming writes in IOBase (Python) +* Add IT test for streaming writes for IOBase (Python) * Implement support for streaming writes in FileBasedSink (Python) +* Expose support for streaming writes in AvroIO (Python) +* Expose support for streaming writes in ParquetIO (Python) * Expose support for streaming writes in TextIO (Python) +* Expose support for streaming writes in TFRecordsIO (Python) ## New Features / Improvements * Added support for Processing time Timer in the Spark Classic runner ([#33633](https://github.com/apache/beam/issues/33633)). -* X feature added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). -* Add pip-based install support for JupyterLab Sidepanel extension ([#35397](https://github.com/apache/beam/issues/#35397)). +* Add pip-based install support for JupyterLab Sidepanel extension ([#35397](https://github.com/apache/beam/issues/35397)). * [IcebergIO] Create tables with a specified table properties ([#35496](https://github.com/apache/beam/pull/35496)) * Add support for comma-separated options in Python SDK (Python) ([#35580](https://github.com/apache/beam/pull/35580)). Python SDK now supports comma-separated values for experiments and dataflow_service_options, @@ -122,45 +184,37 @@ * Milvus enrichment handler added (Python) ([#35216](https://github.com/apache/beam/pull/35216)). Beam now supports Milvus enrichment handler capabilities for vector, keyword, and hybrid search operations. -* [Beam SQL] Add support for DATABASEs, with an implementation for Iceberg ([]()) +* [Beam SQL] Add support for DATABASEs, with an implementation for Iceberg ([#35637](https://github.com/apache/beam/issues/35637)) * Respect BatchSize and MaxBufferingDuration when using `JdbcIO.WriteWithResults`. Previously, these settings were ignored ([#35669](https://github.com/apache/beam/pull/35669)). +* BigTableWrite Connector for BeamYaml added with mutation feature ([#35435](https://github.com/apache/beam/pull/35435)) ## Breaking Changes -* [Python] Prism runner now enabled by default for most Python pipelines using the direct runner ([#34612](https://github.com/apache/beam/pull/34612)). This may break some tests, see https://github.com/apache/beam/pull/34612 for details on how to handle issues. -* X behavior was changed ([#X](https://github.com/apache/beam/issues/X)). -* Go: The pubsubio.Read transform now accepts ReadOptions as a value type instead of a pointer, and requires exactly one of Topic or Subscription to be set (they are mutually exclusive). Additionally, the ReadOptions struct now includes a Topic field for specifying the topic directly, replacing the previous topic parameter in the Read function signature ([#35369])(https://github.com/apache/beam/pull/35369). -* SQL: The `ParquetTable` external table provider has changed its handling of the `LOCATION` property. To read from a directory, the path must now end with a trailing slash (e.g., `LOCATION '/path/to/data/'`). Previously, a trailing slash was not required. This change was made to enable support for glob patterns and single-file paths ([#35582])(https://github.com/apache/beam/pull/35582). - -## Deprecations - -* X behavior is deprecated and will be removed in X versions ([#X](https://github.com/apache/beam/issues/X)). +* Go: The pubsubio.Read transform now accepts ReadOptions as a value type instead of a pointer, and requires exactly one of Topic or Subscription to be set (they are mutually exclusive). Additionally, the ReadOptions struct now includes a Topic field for specifying the topic directly, replacing the previous topic parameter in the Read function signature ([#35369](https://github.com/apache/beam/pull/35369)). +* SQL: The `ParquetTable` external table provider has changed its handling of the `LOCATION` property. To read from a directory, the path must now end with a trailing slash (e.g., `LOCATION '/path/to/data/'`). Previously, a trailing slash was not required. This change was made to enable support for glob patterns and single-file paths ([#35582](https://github.com/apache/beam/pull/35582)). ## Bugfixes -* Fixed X (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). -* [YAML] Fixed handling of missing optional fields in JSON parsing ([#35179](https://github.com/apache/beam/issues/35179)). -* [Python] Fix WriteToBigQuery transform using CopyJob does not work with WRITE_TRUNCATE write disposition ([#34247](https://github.com/apache/beam/issues/34247)) -* [Python] Fixed dicomio tags mismatch in integration tests ([#30760](https://github.com/apache/beam/issues/30760)). -* [Java] Fixed spammy logging issues that affected versions 2.64.0 to 2.66.0. - +* (YAML) Fixed handling of missing optional fields in JSON parsing ([#35179](https://github.com/apache/beam/issues/35179)). +* (Python) Fix WriteToBigQuery transform using CopyJob does not work with WRITE_TRUNCATE write disposition ([#34247](https://github.com/apache/beam/issues/34247)) +* (Python) Fixed dicomio tags mismatch in integration tests ([#30760](https://github.com/apache/beam/issues/30760)). +* (Java) Fixed spammy logging issues that affected versions 2.64.0 to 2.66.0. ## Known Issues -* ([#X](https://github.com/apache/beam/issues/X)). * ([#35666](https://github.com/apache/beam/issues/35666)). YAML Flatten incorrectly drops fields when input PCollections' schema are different. This issue exists for all versions since 2.52.0. # [2.66.0] - 2025-07-01 ## Beam 3.0.0 Development Highlights -* [Java] Java 8 support is now deprecated. It is still supported until Beam 3. +* (Java) Java 8 support is now deprecated. It is still supported until Beam 3. From now, pipeline submitted by Java 8 client uses Java 11 SDK container for remote pipeline execution ([35064](https://github.com/apache/beam/pull/35064)). ## Highlights -* [Python] Several quality-of-life improvements to the vLLM model handler. If you use Beam RunInference with vLLM model handlers, we strongly recommend updating past this release. +* (Python) Several quality-of-life improvements to the vLLM model handler. If you use Beam RunInference with vLLM model handlers, we strongly recommend updating past this release. ## I/Os @@ -171,10 +225,11 @@ * [IcebergIO] Dynamically create namespaces if needed ([#35228](https://github.com/apache/beam/pull/35228)) ## New Features / Improvements + * [Beam SQL] Introducing Beam Catalogs ([#35223](https://github.com/apache/beam/pull/35223)) * Adding Google Storage Requests Pays feature (Golang)([#30747](https://github.com/apache/beam/issues/30747)). -* [Python] Prism runner now auto-enabled for some Python pipelines using the direct runner ([#34921](https://github.com/apache/beam/pull/34921)). -* [YAML] WriteToTFRecord and ReadFromTFRecord Beam YAML support +* (Python) Prism runner now auto-enabled for some Python pipelines using the direct runner ([#34921](https://github.com/apache/beam/pull/34921)). +* (YAML) WriteToTFRecord and ReadFromTFRecord Beam YAML support * Python: Added JupyterLab 4.x extension compatibility for enhanced notebook integration ([#34495](https://github.com/apache/beam/pull/34495)). ## Breaking Changes @@ -188,15 +243,16 @@ ## Bugfixes * (Java) Fixed CassandraIO ReadAll does not let a pipeline handle or retry exceptions ([#34191](https://github.com/apache/beam/pull/34191)). -* [Python] Fixed vLLM model handlers breaking Beam logging. ([#35053](https://github.com/apache/beam/pull/35053)). -* [Python] Fixed vLLM connection leaks that caused a throughput bottleneck and underutilization of GPU ([#35053](https://github.com/apache/beam/pull/35053)). -* [Python] Fixed vLLM server recovery mechanism in the event of a process termination ([#35234](https://github.com/apache/beam/pull/35234)). +* (Python) Fixed vLLM model handlers breaking Beam logging. ([#35053](https://github.com/apache/beam/pull/35053)). +* (Python) Fixed vLLM connection leaks that caused a throughput bottleneck and underutilization of GPU ([#35053](https://github.com/apache/beam/pull/35053)). +* (Python) Fixed vLLM server recovery mechanism in the event of a process termination ([#35234](https://github.com/apache/beam/pull/35234)). * (Python) Fixed cloudpickle overwriting class states every time loading a same object of dynamic class ([#35062](https://github.com/apache/beam/issues/35062)). -* [Python] Fixed pip install apache-beam[interactive] causes crash on google colab ([#35148](https://github.com/apache/beam/pull/35148)). +* (Python) Fixed pip install apache-beam[interactive] causes crash on google colab ([#35148](https://github.com/apache/beam/pull/35148)). * [IcebergIO] Fixed Beam <-> Iceberg conversion logic for arrays of structs and maps of structs ([#35230](https://github.com/apache/beam/pull/35230)). ## Known Issues -* [Java] Using histogram metrics can cause spammy logs. To mitigate this issue, filter worker startup logs, or upgrade to 2.67.0. + +* (Java) Using histogram metrics can cause spammy logs. To mitigate this issue, filter worker startup logs, or upgrade to 2.67.0. # [2.65.0] - 2025-05-12 @@ -211,24 +267,24 @@ ## Breaking Changes -* [Python] Cloudpickle is set as the default `pickle_library`, where previously +* (Python) Cloudpickle is set as the default `pickle_library`, where previously dill was the default in [#34695](https://github.com/apache/beam/pull/34695). For known issues, reporting new issues, and understanding cloudpickle behavior refer to [#34903](https://github.com/apache/beam/issues/34903). -* [Python] Reshuffle now preserves PaneInfo, where previously PaneInfo was lost +* (Python) Reshuffle now preserves PaneInfo, where previously PaneInfo was lost after reshuffle. To opt out of this change, set the update_compatibility_version to a previous Beam version e.g. "2.64.0". ([#34348](https://github.com/apache/beam/pull/34348)) -* [Python] PaneInfo is encoded by PaneInfoCoder, where previously PaneInfo was +* (Python) PaneInfo is encoded by PaneInfoCoder, where previously PaneInfo was encoded with FastPrimitivesCoder falling back to PickleCoder. This only affects cases where PaneInfo is directly stored as an element. ([#34824](https://github.com/apache/beam/pull/34824)) -* [Python] BigQueryFileLoads now adds a Reshuffle before triggering load jobs. +* (Python) BigQueryFileLoads now adds a Reshuffle before triggering load jobs. This fixes a bug where there can be data loss in a streaming pipeline if there is a pending load job during autoscaling. To opt out of this change, set the update_compatibility_version to a previous Beam version e.g. "2.64.0". ([#34657](https://github.com/apache/beam/pull/34657)) -* [YAML] Kafka source and sink will be automatically replaced with compatible managed transforms. +* (YAML) Kafka source and sink will be automatically replaced with compatible managed transforms. For older Beam versions, streaming update compatiblity can be maintained by specifying the pipeline option `update_compatibility_version` ([#34767](https://github.com/apache/beam/issues/34767)). @@ -241,7 +297,7 @@ * Fixed read Beam rows from cross-lang transform (for example, ReadFromJdbc) involving negative 32-bit integers incorrectly decoded to large integers ([#34089](https://github.com/apache/beam/issues/34089)) * (Java) Fixed SDF-based KafkaIO (ReadFromKafkaViaSDF) to properly handle custom deserializers that extend Deserializer interface([#34505](https://github.com/apache/beam/pull/34505)) -* [Python] `TypedDict` typehints are now compatible with `Mapping` and `Dict` type annotations. +* (Python) `TypedDict` typehints are now compatible with `Mapping` and `Dict` type annotations. ## Security Fixes @@ -249,40 +305,40 @@ ## Known Issues -* [Python] GroupIntoBatches may fail in streaming pipelines. This is caused by cloudpickle. To mitigate this issue specify `pickle_library=dill` in pipeline options ([#35062](https://github.com/apache/beam/issues/35062)) -* [Python] vLLM breaks dataflow logging. To mitigate this issue, set the `VLLM_CONFIGURE_LOGGING=0` environment variable in your custom container. -* [Python] vLLM leaks connections causing a throughput bottleneck and underutilization of GPU. To mitigate this issue increase the number of `number_of_worker_harness_threads`. -* [Java] Using histogram metrics can cause spammy logs. To mitigate this issue, filter worker startup logs, or upgrade to 2.67.0. +* (Python) GroupIntoBatches may fail in streaming pipelines. This is caused by cloudpickle. To mitigate this issue specify `pickle_library=dill` in pipeline options ([#35062](https://github.com/apache/beam/issues/35062)) +* (Python) vLLM breaks dataflow logging. To mitigate this issue, set the `VLLM_CONFIGURE_LOGGING=0` environment variable in your custom container. +* (Python) vLLM leaks connections causing a throughput bottleneck and underutilization of GPU. To mitigate this issue increase the number of `number_of_worker_harness_threads`. +* (Java) Using histogram metrics can cause spammy logs. To mitigate this issue, filter worker startup logs, or upgrade to 2.67.0. # [2.64.0] - 2025-03-31 ## Highlights -* Managed API for [Java](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/managed/Managed.html) and [Python](https://beam.apache.org/releases/pydoc/current/apache_beam.transforms.managed.html#module-apache_beam.transforms.managed) supports [key I/O connectors](https://beam.apache.org/documentation/io/connectors/) Iceberg, Kafka, and BigQuery. +* Managed API for (Java)(https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/managed/Managed.html) and (Python)(https://beam.apache.org/releases/pydoc/current/apache_beam.transforms.managed.html#module-apache_beam.transforms.managed) supports [key I/O connectors](https://beam.apache.org/documentation/io/connectors/) Iceberg, Kafka, and BigQuery. ## I/Os -* [Java] Use API compatible with both com.google.cloud.bigdataoss:util 2.x and 3.x in BatchLoads ([#34105](https://github.com/apache/beam/pull/34105)) +* (Java) Use API compatible with both com.google.cloud.bigdataoss:util 2.x and 3.x in BatchLoads ([#34105](https://github.com/apache/beam/pull/34105)) * [IcebergIO] Added new CDC source for batch and streaming, available as `Managed.ICEBERG_CDC` ([#33504](https://github.com/apache/beam/pull/33504)) * [IcebergIO] Address edge case where bundle retry following a successful data commit results in data duplication ([#34264](https://github.com/apache/beam/pull/34264)) -* [Java&Python] Add explicit schema support to JdbcIO read and xlang transform ([#23029](https://github.com/apache/beam/issues/23029)) +* (Java&Python) Add explicit schema support to JdbcIO read and xlang transform ([#23029](https://github.com/apache/beam/issues/23029)) ## New Features / Improvements -* [Python] Support custom coders in Reshuffle ([#29908](https://github.com/apache/beam/issues/29908), [#33356](https://github.com/apache/beam/issues/33356)). -* [Java] Upgrade SLF4J to 2.0.16. Update default Spark version to 3.5.0. ([#33574](https://github.com/apache/beam/pull/33574)) -* [Java] Support for `--add-modules` JVM option is added through a new pipeline option `JdkAddRootModules`. This allows extending the module graph with optional modules such as SDK incubator modules. Sample usage: ` --jdkAddRootModules=jdk.incubator.vector` ([#30281](https://github.com/apache/beam/issues/30281)). -* Managed API for [Java](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/managed/Managed.html) and [Python](https://beam.apache.org/releases/pydoc/current/apache_beam.transforms.managed.html#module-apache_beam.transforms.managed) supports [key I/O connectors](https://beam.apache.org/documentation/io/connectors/) Iceberg, Kafka, and BigQuery. -* [YAML] Beam YAML UDFs (such as those used in MapToFields) can now have declared dependencies +* (Python) Support custom coders in Reshuffle ([#29908](https://github.com/apache/beam/issues/29908), [#33356](https://github.com/apache/beam/issues/33356)). +* (Java) Upgrade SLF4J to 2.0.16. Update default Spark version to 3.5.0. ([#33574](https://github.com/apache/beam/pull/33574)) +* (Java) Support for `--add-modules` JVM option is added through a new pipeline option `JdkAddRootModules`. This allows extending the module graph with optional modules such as SDK incubator modules. Sample usage: ` --jdkAddRootModules=jdk.incubator.vector` ([#30281](https://github.com/apache/beam/issues/30281)). +* Managed API for (Java)(https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/managed/Managed.html) and (Python)(https://beam.apache.org/releases/pydoc/current/apache_beam.transforms.managed.html#module-apache_beam.transforms.managed) supports [key I/O connectors](https://beam.apache.org/documentation/io/connectors/) Iceberg, Kafka, and BigQuery. +* (YAML) Beam YAML UDFs (such as those used in MapToFields) can now have declared dependencies (e.g. pypi packages for Python, or extra jars for Java). * Prism now supports event time triggers for most common cases. ([#31438](https://github.com/apache/beam/issues/31438)) * Prism does not yet support triggered side inputs, or triggers on merging windows (such as session windows). ## Breaking Changes -* [Python] Reshuffle now correctly respects user-specified type hints, fixing a previous bug where it might use FastPrimitivesCoder wrongly. This change could break pipelines with incorrect type hints in Reshuffle. If you have issues after upgrading, temporarily set update_compatibility_version to a previous Beam version to use the old behavior. The recommended solution is to fix the type hints in your code. ([#33932](https://github.com/apache/beam/pull/33932)) -* [Java] SparkReceiver 2 has been moved to SparkReceiver 3 that supports Spark 3.x. ([#33574](https://github.com/apache/beam/pull/33574)) -* [Python] Correct parsing of `collections.abc.Sequence` type hints was added, which can lead to pipelines failing type hint checks that were previously passing erroneously. These issues will be most commonly seen trying to consume a PCollection with a `Sequence` type hint after a GroupByKey or a CoGroupByKey. ([#33999](https://github.com/apache/beam/pull/33999)). +* (Python) Reshuffle now correctly respects user-specified type hints, fixing a previous bug where it might use FastPrimitivesCoder wrongly. This change could break pipelines with incorrect type hints in Reshuffle. If you have issues after upgrading, temporarily set update_compatibility_version to a previous Beam version to use the old behavior. The recommended solution is to fix the type hints in your code. ([#33932](https://github.com/apache/beam/pull/33932)) +* (Java) SparkReceiver 2 has been moved to SparkReceiver 3 that supports Spark 3.x. ([#33574](https://github.com/apache/beam/pull/33574)) +* (Python) Correct parsing of `collections.abc.Sequence` type hints was added, which can lead to pipelines failing type hint checks that were previously passing erroneously. These issues will be most commonly seen trying to consume a PCollection with a `Sequence` type hint after a GroupByKey or a CoGroupByKey. ([#33999](https://github.com/apache/beam/pull/33999)). ## Bugfixes @@ -297,7 +353,7 @@ * (Java) Current version of protobuf has a [bug](https://github.com/protocolbuffers/protobuf/issues/20599) leading to incompatibilities with clients using older versions of Protobuf ([example issue](https://github.com/GoogleCloudPlatform/DataflowTemplates/issues/2191)). This issue has been seen in SpannerIO in particular. Tracked in [#34452](https://github.com/GoogleCloudPlatform/DataflowTemplates/issues/34452). * (Java) When constructing `SpannerConfig` for `SpannerIO`, calling `withHost` with a null or empty host will now result in a Null Pointer Exception (`java.lang.NullPointerException: Cannot invoke "java.lang.CharSequence.length()" because "this.text" is null`). See https://github.com/GoogleCloudPlatform/DataflowTemplates/issues/34489 for context. -* [Java] Using histogram metrics can cause spammy logs. To mitigate this issue, filter worker startup logs, or upgrade to 2.67.0. +* (Java) Using histogram metrics can cause spammy logs. To mitigate this issue, filter worker startup logs, or upgrade to 2.67.0. # [2.63.0] - 2025-02-18 @@ -332,6 +388,7 @@ * With this change user workers will request batched GetWork responses from backend and backend will send multiple WorkItems in the same response proto. * The feature can be disabled by passing `--windmillRequestBatchedGetWorkResponse=false` * Added supports for staging arbitrary files via `--files_to_stage` flag (Python) ([#34208](https://github.com/apache/beam/pull/34208)) + ## Breaking Changes * AWS V1 I/Os have been removed (Java). As part of this, x-lang Python Kinesis I/O has been updated to consume the V2 IO and it also no longer supports setting producer_properties ([#33430](https://github.com/apache/beam/issues/33430)). @@ -391,7 +448,7 @@ ## Known Issues [comment]: # ( When updating known issues after release, make sure also update website blog in website/www/site/content/blog.) -* [Python] If you are using the official Apache Beam Python containers for version 2.62.0, be aware that they include NumPy version 1.26.4. It is strongly recommended that you explicitly specify numpy==1.26.4 in your project's dependency list. ([#33639](https://github.com/apache/beam/issues/33639)). +* (Python) If you are using the official Apache Beam Python containers for version 2.62.0, be aware that they include NumPy version 1.26.4. It is strongly recommended that you explicitly specify numpy==1.26.4 in your project's dependency list. ([#33639](https://github.com/apache/beam/issues/33639)). * [Dataflow Streaming Appliance] Commits fail with KeyCommitTooLargeException when a key outputs >180MB of results. Bug affects versions 2.60.0 to 2.62.0, * fix will be released with 2.63.0. [#33588](https://github.com/apache/beam/issues/33588). * To resolve this issue, downgrade to 2.59.0 or upgrade to 2.63.0 or enable [Streaming Engine](https://cloud.google.com/dataflow/docs/streaming-engine#use). @@ -400,7 +457,7 @@ ## Highlights -* [Python] Introduce Managed Transforms API ([#31495](https://github.com/apache/beam/pull/31495)) +* (Python) Introduce Managed Transforms API ([#31495](https://github.com/apache/beam/pull/31495)) * Flink 1.19 support added ([#32648](https://github.com/apache/beam/pull/32648)) ## I/Os @@ -438,7 +495,7 @@ [comment]: # ( When updating known issues after release, make sure also update website blog in website/www/site/content/blog.) * [Managed Iceberg] DataFile metadata is assigned incorrect partition values ([#33497](https://github.com/apache/beam/issues/33497)). * Fixed in 2.62.0 -* [Python] If you are using the official Apache Beam Python containers for version 2.61.0, be aware that they include NumPy version 1.26.4. It is strongly recommended that you explicitly specify numpy==1.26.4 in your project's dependency list. ([#33639](https://github.com/apache/beam/issues/33639)). +* (Python) If you are using the official Apache Beam Python containers for version 2.61.0, be aware that they include NumPy version 1.26.4. It is strongly recommended that you explicitly specify numpy==1.26.4 in your project's dependency list. ([#33639](https://github.com/apache/beam/issues/33639)). * [Dataflow Streaming Appliance] Commits fail with KeyCommitTooLargeException when a key outputs >180MB of results. Bug affects versions 2.60.0 to 2.62.0, * fix will be released with 2.63.0. [#33588](https://github.com/apache/beam/issues/33588). * To resolve this issue, downgrade to 2.59.0 or upgrade to 2.63.0 or enable [Streaming Engine](https://cloud.google.com/dataflow/docs/streaming-engine#use). @@ -867,7 +924,6 @@ should handle this. ([#25252](https://github.com/apache/beam/issues/25252)). * Introduced a pipeline option `--max_cache_memory_usage_mb` to configure state and side input cache size. The cache has been enabled to a default of 100 MB. Use `--max_cache_memory_usage_mb=X` to provide cache size for the user state API and side inputs. ([#28770](https://github.com/apache/beam/issues/28770)). * Beam YAML stable release. Beam pipelines can now be written using YAML and leverage the Beam YAML framework which includes a preliminary set of IO's and turnkey transforms. More information can be found in the YAML root folder and in the [README](https://github.com/apache/beam/blob/master/sdks/python/apache_beam/yaml/README.md). - ## Breaking Changes * `org.apache.beam.sdk.io.CountingSource.CounterMark` uses custom `CounterMarkCoder` as a default coder since all Avro-dependent @@ -885,16 +941,10 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a * Fixed a memory leak, which affected some long-running Python pipelines: [#28246](https://github.com/apache/beam/issues/28246). ## Security Fixes + * Fixed [CVE-2023-39325](https://www.cve.org/CVERecord?id=CVE-2023-39325) (Java/Python/Go) ([#29118](https://github.com/apache/beam/issues/29118)). * Mitigated [CVE-2023-47248](https://nvd.nist.gov/vuln/detail/CVE-2023-47248) (Python) [#29392](https://github.com/apache/beam/issues/29392). -## Known issues - -* MLTransform drops the identical elements in the output PCollection. For any duplicate elements, a single element will be emitted downstream. ([#29600](https://github.com/apache/beam/issues/29600)). -* Some Python pipelines that run with 2.52.0-2.54.0 SDKs and use large materialized side inputs might be affected by a performance regression. To restore the prior behavior on these SDK versions, supply the `--max_cache_memory_usage_mb=0` pipeline option. (Python) ([#30360](https://github.com/apache/beam/issues/30360)). -* Users who lauch Python pipelines in an environment without internet access and use the `--setup_file` pipeline option might experience an increase in pipeline submission time. This has been fixed in 2.56.0 ([#31070](https://github.com/apache/beam/pull/31070)). -* Transforms which use `SnappyCoder` are update incompatible with previous versions of the same transform (Java) on some runners. This includes PubSubIO's read ([#28655](https://github.com/apache/beam/pull/28655#issuecomment-2407839769)). - # [2.51.0] - 2023-10-03 ## New Features / Improvements @@ -904,7 +954,6 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a * Added support to run `mypy` on user pipelines ([#27906](https://github.com/apache/beam/issues/27906)) * Python SDK worker start-up logs and crash logs are now captured by a buffer and logged at appropriate levels via Beam logging API. Dataflow Runner users might observe that most `worker-startup` log content is now captured by the `worker` logger. Users who relied on `print()` statements for logging might notice that some logs don't flush before pipeline succeeds - we strongly advise to use `logging` package instead of `print()` statements for logging. ([#28317](https://github.com/apache/beam/pull/28317)) - ## Breaking Changes * Removed fastjson library dependency for Beam SQL. Table property is changed to be based on jackson ObjectNode (Java) ([#24154](https://github.com/apache/beam/issues/24154)). @@ -912,15 +961,14 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a * Removed the parameter `t reflect.Type` from `parquetio.Write`. The element type is derived from the input PCollection (Go) ([#28490](https://github.com/apache/beam/issues/28490)) * Refactor BeamSqlSeekableTable.setUp adding a parameter joinSubsetType. [#28283](https://github.com/apache/beam/issues/28283) - ## Bugfixes * Fixed exception chaining issue in GCS connector (Python) ([#26769](https://github.com/apache/beam/issues/26769#issuecomment-1700422615)). * Fixed streaming inserts exception handling, GoogleAPICallErrors are now retried according to retry strategy and routed to failed rows where appropriate rather than causing a pipeline error (Python) ([#21080](https://github.com/apache/beam/issues/21080)). * Fixed a bug in Python SDK's cross-language Bigtable sink that mishandled records that don't have an explicit timestamp set: [#28632](https://github.com/apache/beam/issues/28632). - ## Security Fixes + * Python containers updated, fixing [CVE-2021-30474](https://nvd.nist.gov/vuln/detail/CVE-2021-30474), [CVE-2021-30475](https://nvd.nist.gov/vuln/detail/CVE-2021-30475), [CVE-2021-30473](https://nvd.nist.gov/vuln/detail/CVE-2021-30473), [CVE-2020-36133](https://nvd.nist.gov/vuln/detail/CVE-2020-36133), [CVE-2020-36131](https://nvd.nist.gov/vuln/detail/CVE-2020-36131), [CVE-2020-36130](https://nvd.nist.gov/vuln/detail/CVE-2020-36130), and [CVE-2020-36135](https://nvd.nist.gov/vuln/detail/CVE-2020-36135) * Used go 1.21.1 to build, fixing [CVE-2023-39320](https://security-tracker.debian.org/tracker/CVE-2023-39320) @@ -931,7 +979,6 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a dependency to 1.8.3 or earlier on some runners that don't use Beam Docker containers: [#28811](https://github.com/apache/beam/issues/28811) * MLTransform drops the identical elements in the output PCollection. For any duplicate elements, a single element will be emitted downstream. ([#29600](https://github.com/apache/beam/issues/29600)). - # [2.50.0] - 2023-08-30 ## Highlights @@ -996,7 +1043,6 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a # [2.49.0] - 2023-07-17 - ## I/Os * Support for Bigtable Change Streams added in Java `BigtableIO.ReadChangeStream` ([#27183](https://github.com/apache/beam/issues/27183)) @@ -1021,7 +1067,6 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a * Long-running Python pipelines might experience a memory leak: [#28246](https://github.com/apache/beam/issues/28246). * Python pipelines using the `--impersonate_service_account` option with BigQuery IOs might fail on Dataflow ([#32030](https://github.com/apache/beam/issues/32030)). This is fixed in 2.59.0 release. - # [2.48.0] - 2023-05-31 ## Highlights @@ -1067,7 +1112,6 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a * Long-running Python pipelines might experience a memory leak: [#28246](https://github.com/apache/beam/issues/28246). * Python SDK's cross-language Bigtable sink mishandles records that don't have an explicit timestamp set: [#28632](https://github.com/apache/beam/issues/28632). To avoid this issue, set explicit timestamps for all records before writing to Bigtable. - # [2.47.0] - 2023-05-10 ## Highlights @@ -1245,7 +1289,6 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a * Fixed Beam SQL CalciteUtils (Java) and Cross-language JdbcIO (Python) did not support JDBC CHAR/VARCHAR, BINARY/VARBINARY logical types ([#23747](https://github.com/apache/beam/issues/23747), [#23526](https://github.com/apache/beam/issues/23526)). * Ensure iterated and emitted types are used with the generic register package are registered with the type and schema registries.(Go) ([#23889](https://github.com/apache/beam/pull/23889)) - # [2.43.0] - 2022-11-17 ## Highlights @@ -1339,7 +1382,6 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a * Fixed a condition where retrying queries would yield an incorrect cursor in the Java SDK Firestore Connector ([#22089](https://github.com/apache/beam/issues/22089)). * Fixed plumbing allowed lateness in Go SDK. It was ignoring the user set value earlier and always used to set to 0. ([#22474](https://github.com/apache/beam/issues/22474)). - # [2.40.0] - 2022-06-25 ## Highlights @@ -1366,6 +1408,7 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a * Default coder updated to compress sources used with `BoundedSourceAsSDFWrapperFn` and `UnboundedSourceAsSDFWrapper`. ## Bugfixes + * Fixed Java expansion service to allow specific files to stage ([BEAM-14160](https://issues.apache.org/jira/browse/BEAM-14160)). * Fixed Elasticsearch connection when using both ssl and username/password (Java) ([BEAM-14000](https://issues.apache.org/jira/browse/BEAM-14000)) @@ -1387,7 +1430,6 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a [BEAM-14283](https://issues.apache.org/jira/browse/BEAM-14283)). * Implemented Apache PulsarIO ([BEAM-8218](https://issues.apache.org/jira/browse/BEAM-8218)). - ## New Features / Improvements * Support for flink scala 2.12, because most of the libraries support version 2.12 onwards. ([beam-14386](https://issues.apache.org/jira/browse/BEAM-14386)) @@ -1404,7 +1446,6 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a * Upgrade to ZetaSQL 2022.04.1 ([BEAM-14348](https://issues.apache.org/jira/browse/BEAM-14348)). * Fixed ReadFromBigQuery cannot be used with the interactive runner ([BEAM-14112](https://issues.apache.org/jira/browse/BEAM-14112)). - ## Breaking Changes * Unused functions `ShallowCloneParDoPayload()`, `ShallowCloneSideInput()`, and `ShallowCloneFunctionSpec()` have been removed from the Go SDK's pipelinex package ([BEAM-13739](https://issues.apache.org/jira/browse/BEAM-13739)). @@ -1428,10 +1469,10 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a * Fixed Java Spanner IO NPE when ProjectID not specified in template executions (Java) ([BEAM-14405](https://issues.apache.org/jira/browse/BEAM-14405)). * Fixed potential NPE in BigQueryServicesImpl.getErrorInfo (Java) ([BEAM-14133](https://issues.apache.org/jira/browse/BEAM-14133)). - # [2.38.0] - 2022-04-20 ## I/Os + * Introduce projection pushdown optimizer to the Java SDK ([BEAM-12976](https://issues.apache.org/jira/browse/BEAM-12976)). The optimizer currently only works on the [BigQuery Storage API](https://beam.apache.org/documentation/io/built-in/google-bigquery/#storage-api), but more I/Os will be added in future releases. If you encounter a bug with the optimizer, please file a JIRA and disable the optimizer using pipeline option `--experiments=disable_projection_pushdown`. * A new IO for Neo4j graph databases was added. ([BEAM-1857](https://issues.apache.org/jira/browse/BEAM-1857)) It has the ability to update nodes and relationships using UNWIND statements and to read data using cypher statements with parameters. * `amazon-web-services2` has reached feature parity and is finally recommended over the earlier `amazon-web-services` and `kinesis` modules (Java). These will be deprecated in one of the next releases ([BEAM-13174](https://issues.apache.org/jira/browse/BEAM-13174)). @@ -1473,6 +1514,7 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a # [2.37.0] - 2022-03-04 ## Highlights + * Java 17 support for Dataflow ([BEAM-12240](https://issues.apache.org/jira/browse/BEAM-12240)). * Users using Dataflow Runner V2 may see issues with state cache due to inaccurate object sizes ([BEAM-13695](https://issues.apache.org/jira/browse/BEAM-13695)). * ZetaSql is currently unsupported ([issue](https://github.com/google/zetasql/issues/89)). @@ -1496,10 +1538,13 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a ## Breaking Changes + ## Deprecations + ## Bugfixes + ## Known Issues * On rare occations, Python Datastore source may swallow some exceptions. Users are adviced to upgrade to Beam 2.38.0 or later ([BEAM-14282](https://issues.apache.org/jira/browse/BEAM-14282)) @@ -1620,7 +1665,7 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a ## Breaking Changes * SQL Rows are no longer flattened ([BEAM-5505](https://issues.apache.org/jira/browse/BEAM-5505)). -* [Go SDK] beam.TryCrossLanguage's signature now matches beam.CrossLanguage. Like other Try functions it returns an error instead of panicking. ([BEAM-9918](https://issues.apache.org/jira/browse/BEAM-9918)). +* (Go SDK) beam.TryCrossLanguage's signature now matches beam.CrossLanguage. Like other Try functions it returns an error instead of panicking. ([BEAM-9918](https://issues.apache.org/jira/browse/BEAM-9918)). * [BEAM-12925](https://jira.apache.org/jira/browse/BEAM-12925) was fixed. It used to silently pass incorrect null data read from JdbcIO. Pipelines affected by this will now start throwing failures instead of silently passing incorrect data. ## Bugfixes @@ -1650,12 +1695,6 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a * Minimum Go version is now Go v1.16 * See the announcement blogpost for full information once published. - - ## New Features / Improvements * Projection pushdown in SchemaIO ([BEAM-12609](https://issues.apache.org/jira/browse/BEAM-12609)). @@ -1675,7 +1714,7 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a * Python GBK will stop supporting unbounded PCollections that have global windowing and a default trigger in Beam 2.34. This can be overriden with `--allow_unsafe_triggers`. ([BEAM-9487](https://issues.apache.org/jira/browse/BEAM-9487)). * Python GBK will start requiring safe triggers or the `--allow_unsafe_triggers` flag starting with Beam 2.34. ([BEAM-9487](https://issues.apache.org/jira/browse/BEAM-9487)). -## Bug fixes +## Bugfixes * Workaround to not delete orphaned files to avoid missing events when using Python WriteToFiles in streaming pipeline ([BEAM-12950](https://issues.apache.org/jira/browse/BEAM-12950))) @@ -1688,6 +1727,7 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a # [2.32.0] - 2021-08-25 ## Highlights + * The [Beam DataFrame API](https://beam.apache.org/documentation/dsls/dataframes/overview/) is no longer experimental! We've spent the time since the [2.26.0 preview @@ -1707,7 +1747,6 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a the API, guided by your [feedback](https://beam.apache.org/community/contact-us/). - ## I/Os * New experimental Firestore connector in Java SDK, providing sources and sinks to Google Cloud Firestore ([BEAM-8376](https://issues.apache.org/jira/browse/BEAM-8376)). @@ -1740,6 +1779,7 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a * Fixed race condition in RabbitMqIO causing duplicate acks (Java) ([BEAM-6516](https://issues.apache.org/jira/browse/BEAM-6516))) ## Known Issues + * On rare occations, Python GCS source may swallow some exceptions. Users are adviced to upgrade to Beam 2.38.0 or later ([BEAM-14282](https://issues.apache.org/jira/browse/BEAM-14282)) # [2.31.0] - 2021-07-08 @@ -1827,6 +1867,7 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a # [2.28.0] - 2021-02-22 ## Highlights + * Many improvements related to Parquet support ([BEAM-11460](https://issues.apache.org/jira/browse/BEAM-11460), [BEAM-8202](https://issues.apache.org/jira/browse/BEAM-8202), and [BEAM-11526](https://issues.apache.org/jira/browse/BEAM-11526)) * Hash Functions in BeamSQL ([BEAM-10074](https://issues.apache.org/jira/browse/BEAM-10074)) * Hash functions in ZetaSQL ([BEAM-11624](https://issues.apache.org/jira/browse/BEAM-11624)) @@ -1874,10 +1915,10 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a on removed APIs. If affected, ensure to use an appropriate Guava version via `dependencyManagement` in Maven and `force` in Gradle. - # [2.27.0] - 2021-01-08 ## I/Os + * ReadFromMongoDB can now be used with MongoDB Atlas (Python) ([BEAM-11266](https://issues.apache.org/jira/browse/BEAM-11266).) * ReadFromMongoDB/WriteToMongoDB will mask password in display_data (Python) ([BEAM-11444](https://issues.apache.org/jira/browse/BEAM-11444).) * Support for X source added (Java/Python) ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)). @@ -1909,6 +1950,7 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a * Added support for Contextual Text IO (Java), a version of text IO that provides metadata about the records ([BEAM-10124](https://issues.apache.org/jira/browse/BEAM-10124)). Support for this IO is currently experimental. Specifically, **there are no update-compatibility guarantees** for streaming jobs with this IO between current future verisons of Apache Beam SDK. ## New Features / Improvements + * Added support for avro payload format in Beam SQL Kafka Table ([BEAM-10885](https://issues.apache.org/jira/browse/BEAM-10885)) * Added support for json payload format in Beam SQL Kafka Table ([BEAM-10893](https://issues.apache.org/jira/browse/BEAM-10893)) * Added support for protobuf payload format in Beam SQL Kafka Table ([BEAM-10892](https://issues.apache.org/jira/browse/BEAM-10892)) @@ -1926,7 +1968,6 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a * Non-idempotent combiners built via `CombineFn.from_callable()` or `CombineFn.maybe_from_callable()` can lead to incorrect behavior. ([BEAM-11522](https://issues.apache.org/jira/browse/BEAM-11522)). - # [2.25.0] - 2020-10-23 ## Highlights @@ -1973,7 +2014,6 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a * Dataflow streaming timers once against not strictly time ordered when set earlier mid-bundle, as the fix for [BEAM-8543](https://issues.apache.org/jira/browse/BEAM-8543) introduced more severe bugs and has been rolled back. * Default compressor change breaks dataflow python streaming job update compatibility. Please use python SDK version <= 2.23.0 or > 2.25.0 if job update is critical.([BEAM-11113](https://issues.apache.org/jira/browse/BEAM-11113)) - # [2.24.0] - 2020-09-18 ## Highlights @@ -2009,11 +2049,6 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a --temp_location, or pass method="STREAMING_INSERTS" to WriteToBigQuery ([BEAM-6928](https://issues.apache.org/jira/browse/BEAM-6928)). * Python SDK now understands `typing.FrozenSet` type hints, which are not interchangeable with `typing.Set`. You may need to update your pipelines if type checking fails. ([BEAM-10197](https://issues.apache.org/jira/browse/BEAM-10197)) -## Known issues - -* When a timer fires but is reset prior to being executed, a watermark hold may be leaked, causing a stuck pipeline [BEAM-10991](https://issues.apache.org/jira/browse/BEAM-10991). -* Default compressor change breaks dataflow python streaming job update compatibility. Please use python SDK version <= 2.23.0 or > 2.25.0 if job update is critical.([BEAM-11113](https://issues.apache.org/jira/browse/BEAM-11113)) - # [2.23.0] - 2020-06-29 ## Highlights @@ -2060,6 +2095,7 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a ## Highlights + ## I/Os * Basic Kafka read/write support for DataflowRunner (Python) ([BEAM-8019](https://issues.apache.org/jira/browse/BEAM-8019)). @@ -2088,6 +2124,7 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a ## Deprecations + ## Known Issues @@ -2095,7 +2132,9 @@ as a workaround, a copy of "old" `CountingSource` class should be placed into a ## Highlights + ## I/Os + * Python: Deprecated module `apache_beam.io.gcp.datastore.v1` has been removed as the client it uses is out of date and does not support Python 3 ([BEAM-9529](https://issues.apache.org/jira/browse/BEAM-9529)). @@ -2107,6 +2146,7 @@ for example usage. * Python SDK: Added integration tests and updated batch write functionality for Google Cloud Spanner transform ([BEAM-8949](https://issues.apache.org/jira/browse/BEAM-8949)). ## New Features / Improvements + * Python SDK will now use Python 3 type annotations as pipeline type hints. ([#10717](https://github.com/apache/beam/pull/10717)) @@ -2117,7 +2157,7 @@ for example usage. for that function. More details will be in - [Ensuring Python Type Safety](https://beam.apache.org/documentation/sdks/python-type-safety/) + (Ensuring Python Type Safety)(https://beam.apache.org/documentation/sdks/python-type-safety/) and an upcoming [blog post](https://beam.apache.org/blog/python-typing/index.html). @@ -2147,7 +2187,6 @@ conversion to beam schema options. *Remark: Schema aware is still experimental.* The files are added to `/opt/apache/beam/third_party_licenses/`. By default, no licenses/notices are added to the docker images. ([BEAM-9136](https://issues.apache.org/jira/browse/BEAM-9136)) - ## Breaking Changes * Dataflow runner now requires the `--region` option to be set, unless a default value is set in the environment ([BEAM-9199](https://issues.apache.org/jira/browse/BEAM-9199)). See [here](https://cloud.google.com/dataflow/docs/concepts/regional-endpoints) for more details. @@ -2157,6 +2196,7 @@ conversion to beam schema options. *Remark: Schema aware is still experimental.* * Go SDK docker images are no longer released until further notice. ## Deprecations + * Java SDK: Beam Schema FieldType.getMetadata is now deprecated and is replaced by the Beam Schema Options, it will be removed in version `2.23.0`. ([BEAM-9704](https://issues.apache.org/jira/browse/BEAM-9704)) * The `--zone` option in the Dataflow runner is now deprecated. Please use `--worker_zone` instead. ([BEAM-9716](https://issues.apache.org/jira/browse/BEAM-9716)) @@ -2177,7 +2217,6 @@ Schema Options, it will be removed in version `2.23.0`. ([BEAM-9704](https://iss * Python SDK: Support for Google Cloud Spanner. This is an experimental module for reading and writing data from Google Cloud Spanner ([BEAM-7246](https://issues.apache.org/jira/browse/BEAM-7246)). * Python SDK: Adds support for standard HDFS URLs (with server name). ([#10223](https://github.com/apache/beam/pull/10223)). - ## New Features / Improvements * New AnnotateVideo & AnnotateVideoWithContext PTransform's that integrates GCP Video Intelligence functionality. (Python) ([BEAM-9146](https://issues.apache.org/jira/browse/BEAM-9146)) @@ -2202,6 +2241,7 @@ Schema Options, it will be removed in version `2.23.0`. ([BEAM-9704](https://iss ## Deprecations + ## Bugfixes * Fixed numpy operators in ApproximateQuantiles (Python) ([BEAM-9579](https://issues.apache.org/jira/browse/BEAM-9579)). @@ -2218,4 +2258,6 @@ Schema Options, it will be removed in version `2.23.0`. ([BEAM-9704](https://iss # [2.19.0] - 2020-01-31 +## Highlights + - For versions 2.19.0 and older release notes are available on [Apache Beam Blog](https://beam.apache.org/blog/). diff --git a/build.gradle.kts b/build.gradle.kts index 316ac4072fa6..33199f5b2ea8 100644 --- a/build.gradle.kts +++ b/build.gradle.kts @@ -1,3 +1,5 @@ +import java.util.TreeMap + /* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -253,6 +255,7 @@ tasks.register("javaPreCommit") { dependsOn(":examples:java:sql:preCommit") dependsOn(":examples:java:twitter:build") dependsOn(":examples:java:twitter:preCommit") + dependsOn(":examples:java:iceberg:build") dependsOn(":examples:multi-language:build") dependsOn(":model:fn-execution:build") dependsOn(":model:job-management:build") @@ -354,6 +357,7 @@ tasks.register("javaioPreCommit") { dependsOn(":sdks:java:io:mqtt:build") dependsOn(":sdks:java:io:neo4j:build") dependsOn(":sdks:java:io:parquet:build") + dependsOn(":sdks:java:io:pulsar:build") dependsOn(":sdks:java:io:rabbitmq:build") dependsOn(":sdks:java:io:redis:build") dependsOn(":sdks:java:io:rrio:build") @@ -380,6 +384,7 @@ tasks.register("sqlPreCommit") { dependsOn(":sdks:java:extensions:sql:datacatalog:build") dependsOn(":sdks:java:extensions:sql:expansion-service:build") dependsOn(":sdks:java:extensions:sql:hcatalog:build") + dependsOn(":sdks:java:extensions:sql:iceberg:build") dependsOn(":sdks:java:extensions:sql:jdbc:build") dependsOn(":sdks:java:extensions:sql:jdbc:preCommit") dependsOn(":sdks:java:extensions:sql:perf-tests:build") @@ -426,6 +431,7 @@ tasks.register("sqlPostCommit") { dependsOn(":sdks:java:extensions:sql:postCommit") dependsOn(":sdks:java:extensions:sql:jdbc:postCommit") dependsOn(":sdks:java:extensions:sql:datacatalog:postCommit") + dependsOn(":sdks:java:extensions:sql:iceberg:integrationTest") dependsOn(":sdks:java:extensions:sql:hadoopVersionsTest") } @@ -510,6 +516,271 @@ tasks.register("pythonFormatterPreCommit") { dependsOn("sdks:python:test-suites:tox:pycommon:formatter") } +tasks.register("formatChanges") { + group = "formatting" + description = "Formats CHANGES.md according to the template structure" + + doLast { + val changesFile = file("CHANGES.md") + if (!changesFile.exists()) { + throw GradleException("CHANGES.md file not found") + } + + val content = changesFile.readText() + val lines = content.lines().toMutableList() + + // Find template end (after --> that follows ) + var templateStartIndex = -1 + var templateEndIndex = -1 + + for (i in lines.indices) { + if (lines[i].trim() == "") { + templateStartIndex = i + } else if (templateStartIndex != -1 && lines[i].trim() == "-->") { + templateEndIndex = i + break + } + } + + if (templateEndIndex == -1) { + throw GradleException("Template end marker not found in CHANGES.md") + } + + // Process each release section + var i = templateEndIndex + 1 + val formattedLines = mutableListOf() + + // Keep header and template exactly as-is (lines 0 to templateEndIndex inclusive) + formattedLines.addAll(lines.subList(0, templateEndIndex + 1)) + + // Always add blank line after template + formattedLines.add("") + + while (i < lines.size) { + val line = lines[i] + + // Check if this is a release header + if (line.startsWith("# [")) { + formattedLines.add(line) + i++ + + // Expected sections in order (following template) + val expectedSections = listOf( + "## Beam 3.0.0 Development Highlights", + "## Highlights", + "## I/Os", + "## New Features / Improvements", + "## Breaking Changes", + "## Deprecations", + "## Bugfixes", + "## Security Fixes", + "## Known Issues" + ) + + val sectionContent = mutableMapOf>() + var currentSection = "" + + // Parse existing sections + while (i < lines.size && !lines[i].startsWith("# [")) { + val currentLine = lines[i] + + if (currentLine.startsWith("## ")) { + currentSection = currentLine + if (!sectionContent.containsKey(currentSection)) { + sectionContent[currentSection] = mutableListOf() + } + } else if (currentSection.isNotEmpty()) { + sectionContent[currentSection]!!.add(currentLine) + } + i++ + } + + // Only add sections that actually exist with content + for (section in expectedSections) { + if (sectionContent.containsKey(section)) { + formattedLines.add("") + formattedLines.add(section) + formattedLines.add("") + + // Remove empty lines at start and end + val content = sectionContent[section]!! + while (content.isNotEmpty() && content.first().trim().isEmpty()) { + content.removeAt(0) + } + while (content.isNotEmpty() && content.last().trim().isEmpty()) { + content.removeAt(content.size - 1) + } + + // Format content according to template rules + val formattedContent = content.map { line -> + // Convert SDK language references from [Language] to (Language) + line.replace(Regex("\\[([^\\]]*(?:Java|Python|Go|Kotlin|TypeScript|YAML)[^\\]]*)\\]")) { matchResult -> + val languages = matchResult.groupValues[1] + // Only convert if it's clearly a language reference (not a link or other content) + if (languages.matches(Regex(".*(?:Java|Python|Go|Kotlin|TypeScript|YAML).*"))) { + "($languages)" + } else { + matchResult.value + } + } + } + + formattedLines.addAll(formattedContent) + } + } + + if (i < lines.size) { + formattedLines.add("") + } + } else { + i++ + } + } + + // Write formatted content back + changesFile.writeText(formattedLines.joinToString("\n")) + println("CHANGES.md has been formatted according to template structure") + } +} + +tasks.register("validateChanges") { + group = "verification" + description = "Validates CHANGES.md follows required formatting rules" + + doLast { + val changesFile = file("CHANGES.md") + if (!changesFile.exists()) { + throw GradleException("CHANGES.md file not found") + } + + val content = changesFile.readText() + val lines = content.lines() + val errors = mutableListOf() + + // Find template section boundaries + var templateStartIndex = -1 + var templateEndIndex = -1 + + for (i in lines.indices) { + if (lines[i].trim() == "") { + templateStartIndex = i + println("Found template start at line ${i+1}") + } else if (templateStartIndex != -1 && lines[i].trim() == "-->") { + templateEndIndex = i + println("Found template end at line ${i+1}") + break + } + } + + if (templateStartIndex == -1 || templateEndIndex == -1) { + throw GradleException("Template section not found in CHANGES.md") + } + + println("Template section: lines ${templateStartIndex+1} to ${templateEndIndex+1}") + + // Find unreleased section after the template section + var unreleasedSectionStart = -1 + for (i in (templateEndIndex + 1) until lines.size) { + if (lines[i].startsWith("# [") && lines[i].contains("Unreleased")) { + unreleasedSectionStart = i + println("Found unreleased section at line ${i+1}: ${lines[i]}") + break + } + } + + if (unreleasedSectionStart == -1) { + throw GradleException("Unreleased section not found in CHANGES.md") + } + + // Check entries in the unreleased section + var i = unreleasedSectionStart + 1 + val items = TreeMap() + var lastline = 0 + var item = "" + while (i < lines.size && !lines[i].startsWith("# [")) { + val line = lines[i].trim() + if (line.isEmpty()) { + // skip + } else if (line.startsWith("* ")) { + items.put(lastline, item) + lastline = i + item = line + } else if (line.startsWith("##")) { + items.put(lastline, item) + lastline = i + item = "" + } else { + item += line + } + i++ + } + items.put(lastline, item) + println("Starting validation from line ${i+1}") + + items.forEach { (i, line) -> + if (line.startsWith("* ")) { + println("Checking line ${i+1}: $line") + + // Skip comment lines + if (line.startsWith("* [comment]:")) { + println(" Skipping comment line") + } else { + // Rule 1: Check if language references use parentheses instead of brackets + val languagePattern = "\\[(Java|Python|Go|Kotlin|TypeScript|YAML)(?:/(?:Java|Python|Go|Kotlin|TypeScript|YAML))*\\]" + val languageRegex = Regex(languagePattern) + + // Check if there's a language reference in brackets + val matches = languageRegex.findAll(line).toList() + if (matches.isNotEmpty()) { + for (match in matches) { + val matchText = match.value + val matchPosition = match.range.first + println(" Found language reference: $matchText at position $matchPosition") + + // Check if this is part of an issue link or URL + val beforeMatch = if (matchPosition > 0) line.substring(0, matchPosition) else "" + val isPartOfLink = beforeMatch.contains("[#") || + beforeMatch.contains("http") || + line.contains("CVE-") + + println(" Is part of link: $isPartOfLink") + + if (!isPartOfLink) { + val error = "Line ${i+1}: Language references should use parentheses () instead of brackets []: $line" + println(" Adding error: $error") + errors.add(error) + } + } + } else { + println(" No bracketed language reference found") + } + + // Rule 2: Check if each entry has an issue link + val issueLinkPattern = "\\(\\[#[0-9a-zA-Z]+\\]\\(https://github\\.com/apache/beam/issues/[0-9a-zA-Z]+\\)\\)" + val issueLinkRegex = Regex(issueLinkPattern) + + val hasIssueLink = issueLinkRegex.containsMatchIn(line) + println(" Has issue link: $hasIssueLink") + + if (!hasIssueLink) { + val error = "Line ${i+1}: Missing or malformed issue link. Each entry should end with ([#X](https://github.com/apache/beam/issues/X)): $line" + println(" Adding error: $error") + errors.add(error) + } + } + } + } + + println("Found ${errors.size} errors") + + if (errors.isNotEmpty()) { + throw GradleException("CHANGES.md validation failed with the following errors:\n${errors.joinToString("\n")}\n\nYou can run ./gradlew formatChanges to correct some issues.") + } + + println("CHANGES.md validation successful") + } +} + tasks.register("python39PostCommit") { dependsOn(":sdks:python:test-suites:dataflow:py39:postCommitIT") dependsOn(":sdks:python:test-suites:direct:py39:postCommitIT") diff --git a/buildSrc/build.gradle.kts b/buildSrc/build.gradle.kts index 006a06271426..9ad1a6a5bf3b 100644 --- a/buildSrc/build.gradle.kts +++ b/buildSrc/build.gradle.kts @@ -53,7 +53,7 @@ dependencies { runtimeOnly("gradle.plugin.com.dorongold.plugins:task-tree:1.5") // Adds a 'taskTree' task to print task dependency tree runtimeOnly("net.linguica.gradle:maven-settings-plugin:0.5") runtimeOnly("gradle.plugin.io.pry.gradle.offline_dependencies:gradle-offline-dependencies-plugin:0.5.0") // Enable creating an offline repository - runtimeOnly("net.ltgt.gradle:gradle-errorprone-plugin:3.1.0") // Enable errorprone Java static analysis + runtimeOnly("net.ltgt.gradle:gradle-errorprone-plugin:4.2.0") // Enable errorprone Java static analysis runtimeOnly("org.ajoberstar.grgit:grgit-gradle:5.3.2") // Enable website git publish to asf-site branch runtimeOnly("com.avast.gradle:gradle-docker-compose-plugin:0.16.12") // Enable docker compose tasks runtimeOnly("ca.cutterslade.gradle:gradle-dependency-analyze:1.8.3") // Enable dep analysis diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index a7f2b99b9a1f..103405a57931 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -604,20 +604,20 @@ class BeamModulePlugin implements Plugin { def checkerframework_version = "3.42.0" def classgraph_version = "4.8.162" def dbcp2_version = "2.9.0" - def errorprone_version = "2.10.0" + def errorprone_version = "2.31.0" // [bomupgrader] determined by: com.google.api:gax, consistent with: google_cloud_platform_libraries_bom - def gax_version = "2.67.0" + def gax_version = "2.68.2" def google_ads_version = "33.0.0" def google_clients_version = "2.0.0" def google_cloud_bigdataoss_version = "2.2.26" - // [bomupgrader] determined by: com.google.cloud:google-cloud-spanner, consistent with: google_cloud_platform_libraries_bom + // [bomupgrader] TODO(#35868): currently pinned, should be determined by: com.google.cloud:google-cloud-spanner, consistent with: google_cloud_platform_libraries_bom def google_cloud_spanner_version = "6.95.1" def google_code_gson_version = "2.10.1" def google_oauth_clients_version = "1.34.1" // [bomupgrader] determined by: io.grpc:grpc-netty, consistent with: google_cloud_platform_libraries_bom def grpc_version = "1.71.0" def guava_version = "33.1.0-jre" - def hadoop_version = "3.4.1" + def hadoop_version = "3.4.2" def hamcrest_version = "2.1" def influxdb_version = "2.19" def httpclient_version = "4.5.13" @@ -648,7 +648,7 @@ class BeamModulePlugin implements Plugin { def spotbugs_version = "4.8.3" def testcontainers_version = "1.19.7" // [bomupgrader] determined by: org.apache.arrow:arrow-memory-core, consistent with: google_cloud_platform_libraries_bom - def arrow_version = "15.0.2" + def arrow_version = "17.0.0" def jmh_version = "1.34" def jupiter_version = "5.7.0" @@ -717,7 +717,7 @@ class BeamModulePlugin implements Plugin { commons_compress : "org.apache.commons:commons-compress:1.26.2", commons_csv : "org.apache.commons:commons-csv:1.8", commons_io : "commons-io:commons-io:2.16.1", - commons_lang3 : "org.apache.commons:commons-lang3:3.14.0", + commons_lang3 : "org.apache.commons:commons-lang3:3.18.0", commons_logging : "commons-logging:commons-logging:1.2", commons_math3 : "org.apache.commons:commons-math3:3.6.1", dbcp2 : "org.apache.commons:commons-dbcp2:$dbcp2_version", @@ -732,12 +732,12 @@ class BeamModulePlugin implements Plugin { google_api_client_gson : "com.google.api-client:google-api-client-gson:$google_clients_version", google_api_client_java6 : "com.google.api-client:google-api-client-java6:$google_clients_version", google_api_common : "com.google.api:api-common", // google_cloud_platform_libraries_bom sets version - google_api_services_bigquery : "com.google.apis:google-api-services-bigquery:v2-rev20250511-2.0.0", // [bomupgrader] sets version + google_api_services_bigquery : "com.google.apis:google-api-services-bigquery:v2-rev20250706-2.0.0", // [bomupgrader] sets version google_api_services_cloudresourcemanager : "com.google.apis:google-api-services-cloudresourcemanager:v1-rev20240310-2.0.0", // [bomupgrader] sets version google_api_services_dataflow : "com.google.apis:google-api-services-dataflow:v1b3-rev20250519-$google_clients_version", google_api_services_healthcare : "com.google.apis:google-api-services-healthcare:v1-rev20240130-$google_clients_version", google_api_services_pubsub : "com.google.apis:google-api-services-pubsub:v1-rev20220904-$google_clients_version", - google_api_services_storage : "com.google.apis:google-api-services-storage:v1-rev20250524-2.0.0", // [bomupgrader] sets version + google_api_services_storage : "com.google.apis:google-api-services-storage:v1-rev20250718-2.0.0", // [bomupgrader] sets version google_auth_library_credentials : "com.google.auth:google-auth-library-credentials", // google_cloud_platform_libraries_bom sets version google_auth_library_oauth2_http : "com.google.auth:google-auth-library-oauth2-http", // google_cloud_platform_libraries_bom sets version google_cloud_bigquery : "com.google.cloud:google-cloud-bigquery", // google_cloud_platform_libraries_bom sets version @@ -749,14 +749,16 @@ class BeamModulePlugin implements Plugin { google_cloud_core_grpc : "com.google.cloud:google-cloud-core-grpc", // google_cloud_platform_libraries_bom sets version google_cloud_datacatalog_v1beta1 : "com.google.cloud:google-cloud-datacatalog", // google_cloud_platform_libraries_bom sets version google_cloud_dataflow_java_proto_library_all: "com.google.cloud.dataflow:google-cloud-dataflow-java-proto-library-all:0.5.160304", - google_cloud_datastore_v1_proto_client : "com.google.cloud.datastore:datastore-v1-proto-client:2.29.1", // [bomupgrader] sets version + google_cloud_datastore_v1_proto_client : "com.google.cloud.datastore:datastore-v1-proto-client:2.31.1", // [bomupgrader] sets version google_cloud_firestore : "com.google.cloud:google-cloud-firestore", // google_cloud_platform_libraries_bom sets version google_cloud_pubsub : "com.google.cloud:google-cloud-pubsub", // google_cloud_platform_libraries_bom sets version google_cloud_pubsublite : "com.google.cloud:google-cloud-pubsublite", // google_cloud_platform_libraries_bom sets version // [bomupgrader] the BOM version is set by scripts/tools/bomupgrader.py. If update manually, also update // libraries-bom version on sdks/java/container/license_scripts/dep_urls_java.yaml - google_cloud_platform_libraries_bom : "com.google.cloud:libraries-bom:26.62.0", + google_cloud_platform_libraries_bom : "com.google.cloud:libraries-bom:26.65.0", google_cloud_secret_manager : "com.google.cloud:google-cloud-secretmanager", // google_cloud_platform_libraries_bom sets version + // TODO(#35868) remove pinned google_cloud_spanner_bom after tests or upstream fixed + google_cloud_spanner_bom : "com.google.cloud:google-cloud-spanner-bom:$google_cloud_spanner_version", google_cloud_spanner : "com.google.cloud:google-cloud-spanner", // google_cloud_platform_libraries_bom sets version google_cloud_spanner_test : "com.google.cloud:google-cloud-spanner:$google_cloud_spanner_version:tests", google_cloud_vertexai : "com.google.cloud:google-cloud-vertexai", // google_cloud_platform_libraries_bom sets version @@ -840,7 +842,9 @@ class BeamModulePlugin implements Plugin { log4j2_log4j12_api : "org.apache.logging.log4j:log4j-1.2-api:$log4j2_version", mockito_core : "org.mockito:mockito-core:4.11.0", mockito_inline : "org.mockito:mockito-inline:4.11.0", - mongo_java_driver : "org.mongodb:mongo-java-driver:3.12.11", + mongo_java_driver : "org.mongodb:mongodb-driver-sync:5.5.0", + mongo_bson : "org.mongodb:bson:5.5.0", + mongodb_driver_core : "org.mongodb:mongodb-driver-core:5.5.0", nemo_compiler_frontend_beam : "org.apache.nemo:nemo-compiler-frontend-beam:$nemo_version", netty_all : "io.netty:netty-all:$netty_version", netty_handler : "io.netty:netty-handler:$netty_version", @@ -1493,7 +1497,7 @@ class BeamModulePlugin implements Plugin { project.dependencies { errorprone("com.google.errorprone:error_prone_core:$errorprone_version") - errorprone("jp.skypencil.errorprone.slf4j:errorprone-slf4j:0.1.2") + errorprone("jp.skypencil.errorprone.slf4j:errorprone-slf4j:0.1.28") } project.configurations.errorprone { resolutionStrategy.force "com.google.errorprone:error_prone_core:$errorprone_version" } @@ -1510,56 +1514,91 @@ class BeamModulePlugin implements Plugin { options.fork = true options.forkOptions.jvmArgs += errorProneAddModuleOpts } - - // TODO(https://github.com/apache/beam/issues/20955): Enable errorprone checks - options.errorprone.errorproneArgs.add("-Xep:AutoValueImmutableFields:OFF") - options.errorprone.errorproneArgs.add("-Xep:AutoValueSubclassLeaked:OFF") - options.errorprone.errorproneArgs.add("-Xep:BadImport:OFF") - options.errorprone.errorproneArgs.add("-Xep:BadInstanceof:OFF") - options.errorprone.errorproneArgs.add("-Xep:BigDecimalEquals:OFF") - options.errorprone.errorproneArgs.add("-Xep:ComparableType:OFF") - options.errorprone.errorproneArgs.add("-Xep:DoNotMockAutoValue:OFF") - options.errorprone.errorproneArgs.add("-Xep:EmptyBlockTag:OFF") - options.errorprone.errorproneArgs.add("-Xep:EmptyCatch:OFF") - options.errorprone.errorproneArgs.add("-Xep:EqualsGetClass:OFF") - options.errorprone.errorproneArgs.add("-Xep:EqualsUnsafeCast:OFF") - options.errorprone.errorproneArgs.add("-Xep:EscapedEntity:OFF") - options.errorprone.errorproneArgs.add("-Xep:ExtendsAutoValue:OFF") - options.errorprone.errorproneArgs.add("-Xep:InlineFormatString:OFF") - options.errorprone.errorproneArgs.add("-Xep:InlineMeSuggester:OFF") - options.errorprone.errorproneArgs.add("-Xep:InvalidBlockTag:OFF") - options.errorprone.errorproneArgs.add("-Xep:InvalidInlineTag:OFF") - options.errorprone.errorproneArgs.add("-Xep:InvalidLink:OFF") - options.errorprone.errorproneArgs.add("-Xep:InvalidParam:OFF") - options.errorprone.errorproneArgs.add("-Xep:InvalidThrows:OFF") - options.errorprone.errorproneArgs.add("-Xep:JavaTimeDefaultTimeZone:OFF") - options.errorprone.errorproneArgs.add("-Xep:JavaUtilDate:OFF") - options.errorprone.errorproneArgs.add("-Xep:JodaConstructors:OFF") - options.errorprone.errorproneArgs.add("-Xep:MalformedInlineTag:OFF") - options.errorprone.errorproneArgs.add("-Xep:MissingSummary:OFF") - options.errorprone.errorproneArgs.add("-Xep:MixedMutabilityReturnType:OFF") - options.errorprone.errorproneArgs.add("-Xep:PreferJavaTimeOverload:OFF") - options.errorprone.errorproneArgs.add("-Xep:MutablePublicArray:OFF") - options.errorprone.errorproneArgs.add("-Xep:NonCanonicalType:OFF") - options.errorprone.errorproneArgs.add("-Xep:ProtectedMembersInFinalClass:OFF") - options.errorprone.errorproneArgs.add("-Xep:Slf4jFormatShouldBeConst:OFF") - options.errorprone.errorproneArgs.add("-Xep:Slf4jSignOnlyFormat:OFF") - options.errorprone.errorproneArgs.add("-Xep:StaticAssignmentInConstructor:OFF") - options.errorprone.errorproneArgs.add("-Xep:ThreadPriorityCheck:OFF") - options.errorprone.errorproneArgs.add("-Xep:TimeUnitConversionChecker:OFF") - options.errorprone.errorproneArgs.add("-Xep:UndefinedEquals:OFF") - options.errorprone.errorproneArgs.add("-Xep:UnescapedEntity:OFF") - options.errorprone.errorproneArgs.add("-Xep:UnnecessaryLambda:OFF") - options.errorprone.errorproneArgs.add("-Xep:UnnecessaryMethodReference:OFF") - options.errorprone.errorproneArgs.add("-Xep:UnnecessaryParentheses:OFF") - options.errorprone.errorproneArgs.add("-Xep:UnrecognisedJavadocTag:OFF") - options.errorprone.errorproneArgs.add("-Xep:UnsafeReflectiveConstructionCast:OFF") - options.errorprone.errorproneArgs.add("-Xep:UseCorrectAssertInTests:OFF") - - // Sometimes a static logger is preferred, which is the convention - // currently used in beam. See docs: - // https://github.com/KengoTODA/findbugs-slf4j#slf4j_logger_should_be_non_static - options.errorprone.errorproneArgs.add("-Xep:Slf4jLoggerShouldBeNonStatic:OFF") + def disabledChecks = [ + // TODO(https://github.com/apache/beam/issues/20955): Enable errorprone checks + "AutoValueImmutableFields", + "AutoValueImmutableFields", + "AutoValueSubclassLeaked", + "BadImport", + "BadInstanceof", + "BigDecimalEquals", + "ComparableType", + "DoNotMockAutoValue", + "EmptyBlockTag", + "EmptyCatch", + "EqualsGetClass", + "EqualsUnsafeCast", + "EscapedEntity", + "ExtendsAutoValue", + "InlineFormatString", + "InlineMeSuggester", + "InvalidBlockTag", + "InvalidInlineTag", + "InvalidLink", + "InvalidParam", + "InvalidThrows", + "JavaTimeDefaultTimeZone", + "JavaUtilDate", + "JodaConstructors", + "MalformedInlineTag", + "MissingSummary", + "MixedMutabilityReturnType", + "PreferJavaTimeOverload", + "MutablePublicArray", + "NonCanonicalType", + "ProtectedMembersInFinalClass", + "Slf4jFormatShouldBeConst", + "Slf4jSignOnlyFormat", + "StaticAssignmentInConstructor", + "ThreadPriorityCheck", + "TimeUnitConversionChecker", + "UndefinedEquals", + "UnescapedEntity", + "UnnecessaryLambda", + "UnnecessaryMethodReference", + "UnnecessaryParentheses", + "UnrecognisedJavadocTag", + "UnsafeReflectiveConstructionCast", + "UseCorrectAssertInTests", + // errorprone 3.2.0+ checks + "DirectInvocationOnMock", + "Finalize", + "JUnitIncompatibleType", + "LongDoubleConversion", + "MockNotUsedInProduction", + "NarrowCalculation", + "NullableTypeParameter", + "NullableWildcard", + "StringCharset", + "SuperCallToObjectMethod", + "UnnecessaryLongToIntConversion", + "UnusedVariable", + // intended suppressions emerged in newer protobuf versions + "AutoValueBoxedValues", + // For backward compatibility. Public method checked in before this check impl + // Possible use in interface subclasses + "ClassInitializationDeadlock", + // for encoding efficiency and backward compatibility + "EnumOrdinal", + // widely used in non-public methods + "NotJavadoc", + // return values used for assignments widely, and for backward compatibility. + "NonApiType", + // Used to test self equal + "SelfAssertion", + // Sometimes a static logger is preferred, which is the convention currently used in beam. See docs: + // https://github.com/KengoTODA/findbugs-slf4j#slf4j_logger_should_be_non_static + "Slf4jLoggerShouldBeNonStatic", + // allow implicit Locale.Default + "StringCaseLocaleUsage", + // DoFn methods are executed reflectively at pipeline runtime + "UnusedMethod", + // Void is a valid element type of DoFn elements + "VoidUsed", + ] + disabledChecks.each { + options.errorprone.errorproneArgs.add("-Xep:${it}:OFF") + } } } @@ -1923,15 +1962,25 @@ class BeamModulePlugin implements Plugin { publications { mavenJava(MavenPublication) { + // only publish test and its sources when test folder is non-empty + def testFolder = project.file("src/test") + boolean testExists = testFolder.exists() && testFolder.list().size() != 0 + if (configuration.shadowClosure) { artifact project.shadowJar - artifact project.shadowTestJar + if (testExists) { + artifact project.shadowTestJar + } } else { artifact project.jar - artifact project.testJar + if (testExists) { + artifact project.testJar + } } artifact project.sourcesJar - artifact project.testSourcesJar + if (testExists) { + artifact project.testSourcesJar + } artifact project.javadocJar artifactId = project.archivesBaseName @@ -2894,8 +2943,9 @@ class BeamModulePlugin implements Plugin { // CrossLanguageValidatesRunnerTask is setup under python sdk but also runs tasks not involving // python versions. set 'skipNonPythonTask' property to avoid duplicated run of these tasks. if (!(project.hasProperty('skipNonPythonTask') && project.skipNonPythonTask == 'true')) { - System.err.println 'GoUsingJava tests have been disabled: https://github.com/apache/beam/issues/30517#issuecomment-2341881604.' - // mainTask.configure { dependsOn goTask } + // Re-enabled GoUsingJava tests after fixing underlying issues + // Previous issues: Docker daemon connectivity, SDK worker communication, timeout configurations + mainTask.configure { dependsOn goTask } } cleanupTask.configure { mustRunAfter goTask } config.cleanupJobServer.configure { mustRunAfter goTask } @@ -3030,6 +3080,10 @@ class BeamModulePlugin implements Plugin { project.ext.pythonVersion = project.hasProperty('pythonVersion') ? project.pythonVersion : '3.9' + // Set min/max python versions used for containers and supported versions. + project.ext.minPythonVersion = 9 + project.ext.maxPythonVersion = 13 + def setupVirtualenv = project.tasks.register('setupVirtualenv') { doLast { def virtualenvCmd = [ diff --git a/contributor-docs/code-change-guide.md b/contributor-docs/code-change-guide.md index 55a1c0beac8e..d21eeb133f99 100644 --- a/contributor-docs/code-change-guide.md +++ b/contributor-docs/code-change-guide.md @@ -444,16 +444,35 @@ If you're using Dataflow Runner v2 and `sdks/java/harness` or its dependencies ( --experiments=use_runner_v2,use_staged_dataflow_worker_jar ``` +#### SDK container image change + +If you have changed codes under `sdks/java/container`, you need to build a custom SDK container to make the change +effective. + +```shell + ./gradlew :sdks:java:container:java11:docker # or java17, java21, etc + # change version number to the actual tag below + docker tag apache/beam_java8_sdk:2.68.0.dev \ + "us-docker.pkg.dev/apache-beam-testing/beam-temp/beam_java11_sdk:2.68.0-custom" # change to your artifact registry + docker push "us-docker.pkg.dev/apache-beam-testing/beam-temp/beam_java11_sdk:2.68.0-custom" +``` + +Then run the pipeline with the following options: +``` + --experiments=use_runner_v2 \ + --sdkContainerImage="us.gcr.io/apache-beam-testing/beam_java11_sdk:2.49.0-custom" +``` + #### Snapshot Version Containers -By default, a Snapshot version for an SDK under development will use the containers published to the [apache-beam-testing project's container registry](https://us.gcr.io/apache-beam-testing/github-actions). For example, the most recent snapshot container for Java 17 can be found [here](https://us.gcr.io/apache-beam-testing/github-actions/beam_java17_sdk). +By default, a Snapshot version for an SDK under development will use the containers published to the apache-beam-testing project's container registry (`https://gcr.io/apache-beam-testing/beam-sdk/...`). For example, the most recent snapshot container for Java 21 can be found [here](https://gcr.io/apache-beam-testing/beam-sdk/beam_java21_sdk). When a version is entering the [release candidate stage](https://github.com/apache/beam/blob/master/contributor-docs/release-guide.md), one final SNAPSHOT version will be published. This SNAPSHOT version will use the final containers published on [DockerHub](https://hub.docker.com/search?q=apache%2Fbeam). **NOTE:** During the release process, there may be some downtime where a container is not available for use for a SNAPSHOT version. To avoid this, it is recommended to either switch to the latest SNAPSHOT version available or to use [custom containers](https://beam.apache.org/documentation/runtime/environments/#custom-containers). You should also only rely on snapshot versions for important workloads if absolutely necessary. -Certain runners may override this snapshot behavior; for example, the Dataflow runner overrides all SNAPSHOT containers into a [single registry](https://console.cloud.google.com/gcr/images/cloud-dataflow/GLOBAL/v1beta3). The same downtime will still be incurred, however, when switching to the final container +Certain runners may override this snapshot behavior; for example, the Dataflow runner overrides all SNAPSHOT containers into a [single registry](gcr.io/cloud-dataflow/). The same downtime will still be incurred, however, when switching to the final container. ## Python development guide @@ -635,6 +654,19 @@ Tips for using the Dataflow runner: ## Appendix +### Formatting CHANGES.md + +When updating the `CHANGES.md` file with your changes, use the following Gradle command to ensure proper formatting: + +```shell +./gradlew formatChanges +``` + +This command: +* Organizes sections in the correct order according to the template +* Ensures all required sections are present +* Preserves existing content while maintaining consistent formatting + ### Common Issues * If you run into some strange errors such as `java.lang.NoClassDefFoundError` or errors related to proto changes, try these: diff --git a/contributor-docs/discussion-docs/2025.md b/contributor-docs/discussion-docs/2025.md index bed283a16172..f5ce610690a1 100644 --- a/contributor-docs/discussion-docs/2025.md +++ b/contributor-docs/discussion-docs/2025.md @@ -30,11 +30,12 @@ limitations under the License. | 13 | Kenneth Knowles | [Beam Element Extended Metadata - CDC Metadata](https://s.apache.org/beam-cdc-metadata) | 2025-05-02 14:05:13 | | 14 | Charles Nguyen | [Beam YAML, Kafka and Iceberg User Accessibility (GSoC 2025)](https://docs.google.com/document/d/1m7AKZYkTf_cuJKU1eCh8oX25lOxwixnVUblEj16aSDU/edit?usp=sharing) | 2025-05-24 11:11:14 | | 15 | Minbo Bae | [Beam Protobuf Schema (Java)](https://docs.google.com/document/d/1euOq_Uu4sycT-AiN6MQxJmsOgInyABiFSwW-U3kpwlk/edit?tab=t.0#heading=h.xzptrog8pyxf) | 2025-05-27 00:43:08 | -| 16 | Ahmed Abualsaud | [Introducing Catalogs to Beam SQL](https://docs.google.com/document/d/16P0JrcJ28KSoMMpLYExWPZaala7CE4Ezen-jC_ly3M4/edit?tab=t.0) | 2025-06-11 08:38:55 | -| 17 | Enrique Calderon | [[GSoC 2025] Git based Privilege Management System](https://summerofcode.withgoogle.com/programs/2025/projects/QRKMhW67) | 2025-06-11 11:14:54 | -| 18 | Chen Canyu | [[GSoC 2025] Proposal: Enable pip-based installation for Beam JupyterLab SidePanel](https://summerofcode.withgoogle.com/media/user/a0dca52853b4/proposal/gAAAAABojNSkDnSn0L_3Y8TRLRvPS99439gBLcsrk5beUZHCj3bGBredej4j0i0A3AppWrm6KBCO2THzaNliJ55wJ3ksKFcqto3En2h74H-UQPo8j846W3k=.pdf) | 2025-06-23 11:04:36 | -| 19 | Jack McCluskey | [The Current State (and Future) of Beam Python Type Hinting](https://s.apache.org/beam-python-type-hinting-overview) | 2025-06-26 10:12:03 | -| 20 | Danny McCormick | [[Proposal] Beam ML containers](https://docs.google.com/document/d/1JcVFJsPbVvtvaYdGi-DzWy9PIIYJhL7LwWGEXt2NZMk/edit?usp=sharing) | 2025-06-30 15:07:04 | -| 21 | Mohamed Awnallah | [[Proposal][GSoC 2025] Milvus Vector Sink I/O Connector for Beam](https://docs.google.com/document/d/1agpFq9dy8_7ptMxTET0X7AmGIbDeY0_hGUq-5GNVDqs/edit?usp=sharing) | 2025-07-16 16:18:57 | -| 22 | Shunping Huang | [Design: Turn-key PTransforms for Time Series Processing in Beam](https://s.apache.org/beam-time-series-processing) | 2025-07-18 20:33:00 | -| 23 | Jack McCluskey | [Evaluating Third-Party Libraries for Use in Beam Python Type Checking](https://s.apache.org/beam-python-third-party-type-checking) | 2025-07-23 10:03:03 | +| 16 | Mohamed Awnallah | [[Proposal][GSoC 2025] Milvus Vector Enrichment Handler for Beam](https://docs.google.com/document/d/1lzoSGSblrFtf7YK9n5p9BEBw8jRhKeOOKqy0FP1urvg/edit?usp=sharing) | 2025-05-29 17:55:11 | +| 17 | Ahmed Abualsaud | [Introducing Catalogs to Beam SQL](https://docs.google.com/document/d/16P0JrcJ28KSoMMpLYExWPZaala7CE4Ezen-jC_ly3M4/edit?tab=t.0) | 2025-06-11 08:38:55 | +| 18 | Enrique Calderon | [[GSoC 2025] Git based Privilege Management System](https://summerofcode.withgoogle.com/programs/2025/projects/QRKMhW67) | 2025-06-11 11:14:54 | +| 19 | Chen Canyu | [[GSoC 2025] Proposal: Enable pip-based installation for Beam JupyterLab SidePanel](https://summerofcode.withgoogle.com/media/user/a0dca52853b4/proposal/gAAAAABojNSkDnSn0L_3Y8TRLRvPS99439gBLcsrk5beUZHCj3bGBredej4j0i0A3AppWrm6KBCO2THzaNliJ55wJ3ksKFcqto3En2h74H-UQPo8j846W3k=.pdf) | 2025-06-23 11:04:36 | +| 20 | Jack McCluskey | [The Current State (and Future) of Beam Python Type Hinting](https://s.apache.org/beam-python-type-hinting-overview) | 2025-06-26 10:12:03 | +| 21 | Danny McCormick | [[Proposal] Beam ML containers](https://docs.google.com/document/d/1JcVFJsPbVvtvaYdGi-DzWy9PIIYJhL7LwWGEXt2NZMk/edit?usp=sharing) | 2025-06-30 15:07:04 | +| 22 | Mohamed Awnallah | [[Proposal][GSoC 2025] Milvus Vector Sink I/O Connector for Beam](https://docs.google.com/document/d/1agpFq9dy8_7ptMxTET0X7AmGIbDeY0_hGUq-5GNVDqs/edit?usp=sharing) | 2025-07-16 16:18:57 | +| 23 | Shunping Huang | [Design: Turn-key PTransforms for Time Series Processing in Beam](https://s.apache.org/beam-time-series-processing) | 2025-07-18 20:33:00 | +| 24 | Jack McCluskey | [Evaluating Third-Party Libraries for Use in Beam Python Type Checking](https://s.apache.org/beam-python-third-party-type-checking) | 2025-07-23 10:03:03 | diff --git a/contributor-docs/release-guide.md b/contributor-docs/release-guide.md index 63293442e0d5..dc3f551b4629 100644 --- a/contributor-docs/release-guide.md +++ b/contributor-docs/release-guide.md @@ -561,7 +561,8 @@ and update all the JSON configuration fields with "yes" if it is the first time 5. Build javadoc, pydoc, typedocs for a PR to update beam-site. - **NOTE**: Do not merge this PR until after an RC has been approved (see "Finalize the Release"). -6. Build Prism binaries for various platforms, and upload them into [dist.apache.org](https://dist.apache.org/repos/dist/dev/beam) +6. Build and create PR to update beam Managed IO documentation. +7. Build Prism binaries for various platforms, and upload them into [dist.apache.org](https://dist.apache.org/repos/dist/dev/beam) and the Github Release with the matching RC tag. ### Verify source and artifact distributions @@ -638,8 +639,9 @@ __Attention:__ Verify that: Beam publishes API reference manuals for each release on the website. For Java and Python SDKs, that’s Javadoc and PyDoc, respectively. The final step of building the candidate is to propose website pull requests that update these -manuals. The first pr will get created by the build_release_candidate action, -you will need to create the second one manually +manuals. `beam-site release-docs` PR and `beam managed-io` update PR will get +created by the build_release_candidate action, you will need to create the +`beam release-docs` update PR manually. Merge the pull requests only after finalizing the release. To avoid invalid redirects for the 'current' version, merge these PRs in the order listed. Once @@ -657,15 +659,20 @@ created by the `build_release_candidate` workflow (see above). **PR 2: apache/beam** +This pull request is against the `apache/beam` repo, to update the `managed-io.md`. +It is created by the `build_release_candidate` workflow and updates the documentation for Managed IOs. + +**PR 3: apache/beam** + This pull request is against the `apache/beam` repo, on the `master` branch ([example](https://github.com/apache/beam/pull/17378)). - Update `CHANGES.md` to update release date and remove template. - Update release version in `website/www/site/config.toml`. - Add new release in `website/www/site/content/en/get-started/downloads.md`. + - For the current release, use `closer.lua` script for download links (e.g., `https://www.apache.org/dyn/closer.lua/beam/{{< param release_latest >}}/apache-beam-{{< param release_latest >}}-source-release.zip`) - Download links will not work until the release is finalized. -- Update links to prior releases to point to https://archive.apache.org (see - example PR). +- Move the previous release to the "Archived releases" section and update its links to point to https://archive.apache.org (see example PR). - Create the Blog post: #### Blog post @@ -846,6 +853,7 @@ template; please adjust as you see fit. * Docker images published to Docker Hub [11]. * PR to run tests against release branch [12]. * Github Release pre-release page for v1.2.3-RC3 [13]. + * pull request to apache/beam updating the managed-io docs[15] The vote will be open for at least 72 hours. It is adopted by majority approval, with at least 3 PMC affirmative votes. @@ -868,6 +876,7 @@ template; please adjust as you see fit. [12] https://github.com/apache/beam/pull/... [13] https://github.com/apache/beam/releases/tag/v1.2.3-RC3 [14] https://github.com/apache/beam/blob/master/contributor-docs/rc-testing-guide.md + [15] https://github.com/apache/beam/pull/... If there are any issues found in the release candidate, reply on the vote thread to cancel the vote. There’s no need to wait 72 hours. Go back to diff --git a/examples/java/build.gradle b/examples/java/build.gradle index 5a1d5b2e8fdc..6f35a109998c 100644 --- a/examples/java/build.gradle +++ b/examples/java/build.gradle @@ -71,8 +71,6 @@ dependencies { implementation project(":sdks:java:extensions:python") implementation project(":sdks:java:io:google-cloud-platform") implementation project(":sdks:java:io:kafka") - runtimeOnly project(":sdks:java:io:iceberg") - implementation project(":sdks:java:managed") implementation project(":sdks:java:extensions:ml") implementation library.java.avro implementation library.java.bigdataoss_util diff --git a/examples/java/iceberg/build.gradle b/examples/java/iceberg/build.gradle new file mode 100644 index 000000000000..09ef64d32ee3 --- /dev/null +++ b/examples/java/iceberg/build.gradle @@ -0,0 +1,89 @@ +/* + * 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. + */ + + +plugins { + id 'java' + id 'org.apache.beam.module' + id 'com.gradleup.shadow' +} + +applyJavaNature( + exportJavadoc: false, + automaticModuleName: 'org.apache.beam.examples.iceberg', + // iceberg requires Java11+ + requireJavaVersion: JavaVersion.VERSION_11 +) + +description = "Apache Beam :: Examples :: Java :: Iceberg" +ext.summary = """Apache Beam Java SDK examples using IcebergIO.""" + +/** Define the list of runners which execute a precommit test. + * Some runners are run from separate projects, see the preCommit task below + * for details. + */ +def preCommitRunners = ["directRunner", "flinkRunner", "sparkRunner"] +// The following runners have configuration created but not added to preCommit +def nonPreCommitRunners = ["dataflowRunner", "prismRunner"] +for (String runner : preCommitRunners) { + configurations.create(runner + "PreCommit") +} +for (String runner: nonPreCommitRunners) { + configurations.create(runner + "PreCommit") +} +configurations.sparkRunnerPreCommit { + // Ban certain dependencies to prevent a StackOverflow within Spark + // because JUL -> SLF4J -> JUL, and similarly JDK14 -> SLF4J -> JDK14 + exclude group: "org.slf4j", module: "jul-to-slf4j" + exclude group: "org.slf4j", module: "slf4j-jdk14" +} + +dependencies { + implementation enforcedPlatform(library.java.google_cloud_platform_libraries_bom) + runtimeOnly project(":sdks:java:io:iceberg") + runtimeOnly project(":sdks:java:io:iceberg:bqms") + implementation project(path: ":sdks:java:core", configuration: "shadow") + implementation project(":sdks:java:extensions:google-cloud-platform-core") + implementation project(":sdks:java:io:google-cloud-platform") + implementation project(":sdks:java:managed") + implementation library.java.google_auth_library_oauth2_http + implementation library.java.joda_time + runtimeOnly project(path: ":runners:direct-java", configuration: "shadow") + implementation library.java.vendored_guava_32_1_2_jre + runtimeOnly library.java.hadoop_client + runtimeOnly library.java.bigdataoss_gcs_connector + + // Add dependencies for the PreCommit configurations + // For each runner a project level dependency on the examples project. + for (String runner : preCommitRunners) { + delegate.add(runner + "PreCommit", project(path: ":examples:java", configuration: "testRuntimeMigration")) + } + directRunnerPreCommit project(path: ":runners:direct-java", configuration: "shadow") + flinkRunnerPreCommit project(":runners:flink:${project.ext.latestFlinkVersion}") + sparkRunnerPreCommit project(":runners:spark:3") + sparkRunnerPreCommit project(":sdks:java:io:hadoop-file-system") + dataflowRunnerPreCommit project(":runners:google-cloud-dataflow-java") + dataflowRunnerPreCommit project(":runners:google-cloud-dataflow-java:worker") // v2 worker + dataflowRunnerPreCommit project(":sdks:java:harness") // v2 worker + prismRunnerPreCommit project(":runners:prism:java") + + // Add dependency if requested on command line for runner + if (project.hasProperty("runnerDependency")) { + runtimeOnly project(path: project.getProperty("runnerDependency")) + } +} diff --git a/examples/java/iceberg/src/main/java/org/apache/beam/examples/iceberg/IcebergBatchWriteExample.java b/examples/java/iceberg/src/main/java/org/apache/beam/examples/iceberg/IcebergBatchWriteExample.java new file mode 100644 index 000000000000..2a5f85e524ed --- /dev/null +++ b/examples/java/iceberg/src/main/java/org/apache/beam/examples/iceberg/IcebergBatchWriteExample.java @@ -0,0 +1,208 @@ +/* + * 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. + */ +package org.apache.beam.examples.iceberg; + +import java.io.IOException; +import java.util.Map; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; +import org.apache.beam.sdk.managed.Managed; +import org.apache.beam.sdk.options.Default; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.options.Validation; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.Sum; +import org.apache.beam.sdk.util.Preconditions; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; + +/** + * This pipeline demonstrates a batch write to an Iceberg table using the BigQuery Metastore + * catalog. + * + *

The pipeline reads from a public BigQuery table containing Google Analytics session data, + * extracts and aggregates the total number of transactions per web browser, and writes the results + * to a new Iceberg table managed by the BigQuery Metastore. + * + *

This example is a demonstration of the Iceberg BigQuery Metastore. For more information, see + * the documentation at https://cloud.google.com/bigquery/docs/blms-use-dataproc. + */ +public class IcebergBatchWriteExample { + + public static final Schema BQ_SCHEMA = + Schema.builder().addStringField("browser").addInt64Field("transactions").build(); + + public static final Schema AGGREGATED_SCHEMA = + Schema.builder().addStringField("browser").addInt64Field("transaction_count").build(); + + public static final String BQ_TABLE = + "bigquery-public-data.google_analytics_sample.ga_sessions_20170801"; + + private static Row flattenAnalyticsRow(Row row) { + Row device = Preconditions.checkStateNotNull(row.getRow("device")); + Row totals = Preconditions.checkStateNotNull(row.getRow("totals")); + return Row.withSchema(BQ_SCHEMA) + .withFieldValue("browser", Preconditions.checkStateNotNull(device.getString("browser"))) + .withFieldValue( + "transactions", Preconditions.checkStateNotNull(totals.getInt64("transactions"))) + .build(); + } + + static class ExtractBrowserTransactionsFn extends DoFn> { + @ProcessElement + public void processElement(ProcessContext c) { + Row row = c.element(); + c.output( + KV.of( + Preconditions.checkStateNotNull(row.getString("browser")), + Preconditions.checkStateNotNull(row.getInt64("transactions")))); + } + } + + static class FormatCountsFn extends DoFn, Row> { + @ProcessElement + public void processElement(ProcessContext c) { + Row row = + Row.withSchema(AGGREGATED_SCHEMA) + .withFieldValue("browser", c.element().getKey()) + .withFieldValue("transaction_count", c.element().getValue()) + .build(); + c.output(row); + } + } + + static class CountTransactions extends PTransform, PCollection> { + @Override + public PCollection expand(PCollection rows) { + PCollection> browserTransactions = + rows.apply(ParDo.of(new ExtractBrowserTransactionsFn())); + PCollection> browserCounts = browserTransactions.apply(Sum.longsPerKey()); + return browserCounts.apply(ParDo.of(new FormatCountsFn())); + } + } + + /** Pipeline options for this example. */ + public interface IcebergPipelineOptions extends GcpOptions { + @Description( + "Warehouse location where the table's data will be written to. " + + "As of 07/14/25 BigLake only supports Single Region buckets") + @Validation.Required + @Default.String("gs://analytics_warehouse") + String getWarehouse(); + + void setWarehouse(String warehouse); + + @Description("The Iceberg table to write to, in the format 'dataset.table'.") + @Validation.Required + @Default.String("analytics_dataset.transactions_by_browser") + String getIcebergTable(); + + void setIcebergTable(String value); + + @Description("The name of the catalog to use.") + @Validation.Required + @Default.String("analytics") + String getCatalogName(); + + void setCatalogName(String catalogName); + + @Description("The implementation of the Iceberg catalog.") + @Default.String("org.apache.iceberg.gcp.bigquery.BigQueryMetastoreCatalog") + String getCatalogImpl(); + + void setCatalogImpl(String catalogImpl); + + @Description("The GCP location for the BigQuery Metastore.") + @Default.String("us-central1") + String getGcpLocation(); + + void setGcpLocation(String gcpLocation); + + @Description("The implementation of the Iceberg FileIO.") + @Default.String("org.apache.iceberg.gcp.gcs.GCSFileIO") + String getIoImpl(); + + void setIoImpl(String ioImpl); + } + + /** + * Main entry point for the pipeline. + * + * @param args Command line arguments + * @throws IOException if there's an issue with the pipeline setup + */ + public static void main(String[] args) throws IOException { + IcebergPipelineOptions options = + PipelineOptionsFactory.fromArgs(args).withValidation().as(IcebergPipelineOptions.class); + + final String tableIdentifier = options.getIcebergTable(); + final String warehouseLocation = options.getWarehouse(); + final String catalogName = options.getCatalogName(); + final String projectName = options.getProject(); + final String catalogImpl = options.getCatalogImpl(); + final String gcpLocation = options.getGcpLocation(); + final String ioImpl = options.getIoImpl(); + + Map catalogProps = + ImmutableMap.builder() + .put("warehouse", warehouseLocation) + .put("catalog-impl", catalogImpl) + .put("gcp_project", projectName) + .put("gcp_location", gcpLocation) + .put("io-impl", ioImpl) + .build(); + + Map icebergWriteConfig = + ImmutableMap.builder() + .put("table", tableIdentifier) + .put("catalog_properties", catalogProps) + .put("catalog_name", catalogName) + .build(); + + Map bigQueryReadConfig = + ImmutableMap.builder() + .put("table", BQ_TABLE) + .put("fields", ImmutableList.of("device.browser", "totals.transactions")) + .put("row_restriction", "totals.transactions is not null") + .build(); + + Pipeline p = Pipeline.create(options); + + p.apply("ReadFromBigQuery", Managed.read(Managed.BIGQUERY).withConfig(bigQueryReadConfig)) + .get("output") + .apply( + "Flatten", + MapElements.into(TypeDescriptors.rows()) + .via(IcebergBatchWriteExample::flattenAnalyticsRow)) + .setRowSchema(BQ_SCHEMA) + .apply("CountTransactions", new CountTransactions()) + .setRowSchema(AGGREGATED_SCHEMA) + .apply("WriteToIceberg", Managed.write(Managed.ICEBERG).withConfig(icebergWriteConfig)); + + p.run().waitUntilFinish(); + } +} diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/IcebergRestCatalogCDCExample.java b/examples/java/iceberg/src/main/java/org/apache/beam/examples/iceberg/IcebergRestCatalogCDCExample.java similarity index 99% rename from examples/java/src/main/java/org/apache/beam/examples/cookbook/IcebergRestCatalogCDCExample.java rename to examples/java/iceberg/src/main/java/org/apache/beam/examples/iceberg/IcebergRestCatalogCDCExample.java index ecc047a56949..4229e401ab94 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/IcebergRestCatalogCDCExample.java +++ b/examples/java/iceberg/src/main/java/org/apache/beam/examples/iceberg/IcebergRestCatalogCDCExample.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.examples.cookbook; +package org.apache.beam.examples.iceberg; import static org.apache.beam.sdk.managed.Managed.ICEBERG_CDC; diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/IcebergRestCatalogStreamingWriteExample.java b/examples/java/iceberg/src/main/java/org/apache/beam/examples/iceberg/IcebergRestCatalogStreamingWriteExample.java similarity index 99% rename from examples/java/src/main/java/org/apache/beam/examples/cookbook/IcebergRestCatalogStreamingWriteExample.java rename to examples/java/iceberg/src/main/java/org/apache/beam/examples/iceberg/IcebergRestCatalogStreamingWriteExample.java index 63dc4ff7056c..0ea73cdf0c87 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/IcebergRestCatalogStreamingWriteExample.java +++ b/examples/java/iceberg/src/main/java/org/apache/beam/examples/iceberg/IcebergRestCatalogStreamingWriteExample.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.examples.cookbook; +package org.apache.beam.examples.iceberg; import com.google.auth.oauth2.GoogleCredentials; import java.io.IOException; diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/IcebergTaxiExamples.java b/examples/java/iceberg/src/main/java/org/apache/beam/examples/iceberg/IcebergTaxiExamples.java similarity index 99% rename from examples/java/src/main/java/org/apache/beam/examples/cookbook/IcebergTaxiExamples.java rename to examples/java/iceberg/src/main/java/org/apache/beam/examples/iceberg/IcebergTaxiExamples.java index 446d11d03be4..5b4fe1b9b913 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/IcebergTaxiExamples.java +++ b/examples/java/iceberg/src/main/java/org/apache/beam/examples/iceberg/IcebergTaxiExamples.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.examples.cookbook; +package org.apache.beam.examples.iceberg; import java.util.Arrays; import java.util.Map; diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java b/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java index 4cfbcd56b983..e4ce5e3eb17e 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java @@ -123,13 +123,11 @@ public static void main(String[] args) throws IOException { Pipeline pipeline = Pipeline.create(options); String tableSpec = - new StringBuilder() - .append(options.getProject()) - .append(":") - .append(options.getBigQueryDataset()) - .append(".") - .append(options.getBigQueryTable()) - .toString(); + options.getProject() + + ":" + + options.getBigQueryDataset() + + "." + + options.getBigQueryTable(); pipeline .apply("ReadLines", TextIO.read().from(options.getInputFile())) .apply(ParDo.of(new ExtractWords())) diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java index 4d1a6cb66add..9171457b2e8f 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/utils/SchemasUtils.java @@ -95,7 +95,7 @@ private void validateSchemaTypes(TableSchema bigQuerySchema) { try { beamSchema = fromTableSchema(bigQuerySchema); } catch (UnsupportedOperationException exception) { - LOG.error("Check json schema, {}", exception.getMessage()); + LOG.error("Check json schema", exception); } catch (Exception e) { LOG.error("Missing schema keywords, please check what all required fields presented"); } diff --git a/examples/notebooks/beam-ml/alloydb_product_catalog_embeddings.ipynb b/examples/notebooks/beam-ml/alloydb_product_catalog_embeddings.ipynb index 4b14f0fea79d..d58d54656d89 100644 --- a/examples/notebooks/beam-ml/alloydb_product_catalog_embeddings.ipynb +++ b/examples/notebooks/beam-ml/alloydb_product_catalog_embeddings.ipynb @@ -151,7 +151,7 @@ "outputs": [], "source": [ "# Apache Beam with GCP support\n", - "!pip install apache_beam[gcp]>=2.66.0\n", + "!pip install apache_beam[interactive,gcp]>=2.66.0\n", "# Huggingface sentence-transformers for embedding models\n", "!pip install sentence-transformers --quiet" ] diff --git a/examples/notebooks/beam-ml/anomaly_detection/anomaly_detection_iforest.ipynb b/examples/notebooks/beam-ml/anomaly_detection/anomaly_detection_iforest.ipynb index db2de68f27c9..92516ce54365 100644 --- a/examples/notebooks/beam-ml/anomaly_detection/anomaly_detection_iforest.ipynb +++ b/examples/notebooks/beam-ml/anomaly_detection/anomaly_detection_iforest.ipynb @@ -489,7 +489,7 @@ "cell_type": "code", "source": [ "# For running with dataflow runner\n", - "!pip install 'apache_beam[gcp, interactive]=={BEAM_VERSION}' --quiet" + "!pip install 'apache_beam[interactive,gcp]=={BEAM_VERSION}' --quiet" ], "metadata": { "id": "0C0qur71DiN3" diff --git a/examples/notebooks/beam-ml/anomaly_detection/anomaly_detection_timesfm.ipynb b/examples/notebooks/beam-ml/anomaly_detection/anomaly_detection_timesfm.ipynb new file mode 100644 index 000000000000..034dca22a42b --- /dev/null +++ b/examples/notebooks/beam-ml/anomaly_detection/anomaly_detection_timesfm.ipynb @@ -0,0 +1,2712 @@ +{ + "nbformat": 4, + "nbformat_minor": 0, + "metadata": { + "colab": { + "provenance": [], + "gpuType": "T4" + }, + "kernelspec": { + "name": "python3", + "display_name": "Python 3" + }, + "language_info": { + "name": "python" + }, + "accelerator": "GPU" + }, + "cells": [ + { + "cell_type": "code", + "source": [ + "# @title ###### Licensed to the Apache Software Foundation (ASF), Version 2.0 (the \"License\")\n", + "\n", + "# Licensed to the Apache Software Foundation (ASF) under one\n", + "# or more contributor license agreements. See the NOTICE file\n", + "# distributed with this work for additional information\n", + "# regarding copyright ownership. The ASF licenses this file\n", + "# to you under the Apache License, Version 2.0 (the\n", + "# \"License\"); you may not use this file except in compliance\n", + "# with the License. You may obtain a copy of the License at\n", + "#\n", + "# http://www.apache.org/licenses/LICENSE-2.0\n", + "#\n", + "# Unless required by applicable law or agreed to in writing,\n", + "# software distributed under the License is distributed on an\n", + "# \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY\n", + "# KIND, either express or implied. See the License for the\n", + "# specific language governing permissions and limitations\n", + "# under the License" + ], + "metadata": { + "id": "eMMlVe_Gukos" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "# TimesFM Anomaly Detection Pipeline Diagram\n", + "Time series data is a sequence of data points indexed by time, where each data point is recorded at a specific interval. TimesFM is a foundation model pretrained on a large corpus of time series data. Its architecture is a decoder-only transformer, similar to LLMs, which learns to predict the next part of a time series from previous data. We can use the follow pipeline to detect anomalies in time series data and periodically learn from incoming data to improve our timesfm predictions.\n", + "\n", + "![Untitled drawing.jpg](data:image/jpeg;base64,/9j/4AAQSkZJRgABAgAAAQABAAD/2wBDAAgGBgcGBQgHBwcJCQgKDBQNDAsLDBkSEw8UHRofHh0aHBwgJC4nICIsIxwcKDcpLDAxNDQ0Hyc5PTgyPC4zNDL/2wBDAQkJCQwLDBgNDRgyIRwhMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjL/wAARCAG0B3oDASIAAhEBAxEB/8QAHwAAAQUBAQEBAQEAAAAAAAAAAAECAwQFBgcICQoL/8QAtRAAAgEDAwIEAwUFBAQAAAF9AQIDAAQRBRIhMUEGE1FhByJxFDKBkaEII0KxwRVS0fAkM2JyggkKFhcYGRolJicoKSo0NTY3ODk6Q0RFRkdISUpTVFVWV1hZWmNkZWZnaGlqc3R1dnd4eXqDhIWGh4iJipKTlJWWl5iZmqKjpKWmp6ipqrKztLW2t7i5usLDxMXGx8jJytLT1NXW19jZ2uHi4+Tl5ufo6erx8vP09fb3+Pn6/8QAHwEAAwEBAQEBAQEBAQAAAAAAAAECAwQFBgcICQoL/8QAtREAAgECBAQDBAcFBAQAAQJ3AAECAxEEBSExBhJBUQdhcRMiMoEIFEKRobHBCSMzUvAVYnLRChYkNOEl8RcYGRomJygpKjU2Nzg5OkNERUZHSElKU1RVVldYWVpjZGVmZ2hpanN0dXZ3eHl6goOEhYaHiImKkpOUlZaXmJmaoqOkpaanqKmqsrO0tba3uLm6wsPExcbHyMnK0tPU1dbX2Nna4uPk5ebn6Onq8vP09fb3+Pn6/9oADAMBAAIRAxEAPwD3+iiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKAOH8YabY6r4w0K31GytryAWF84juIlkUMJLUA4YEZwTz7mqv/AAhvhf8A6FvR/wDwBi/+JrU8Rf8AI8aJ/wBg2+/9G2lT1nLc66KXKYn/AAhvhf8A6FvR/wDwBi/+Jo/4Q3wv/wBC3o//AIAxf/E1t0VNzblRif8ACG+F/wDoW9H/APAGL/4mj/hDfC//AELej/8AgDF/8TW3RRcOVGJ/whvhf/oW9H/8AYv/AImj/hDfC/8A0Lej/wDgDF/8TW3RRcOVGJ/whvhf/oW9H/8AAGL/AOJo/wCEN8L/APQt6P8A+AMX/wATW3RRcOVGJ/whvhf/AKFvR/8AwBi/+Jo/4Q3wv/0Lej/+AMX/AMTW3RRcOVGJ/wAIb4X/AOhb0f8A8AYv/iaP+EN8L/8AQt6P/wCAMX/xNbdFFw5UYn/CG+F/+hb0f/wBi/8AiaP+EN8L/wDQt6P/AOAMX/xNbdFFw5UYn/CG+F/+hb0f/wAAYv8A4mj/AIQ3wv8A9C3o/wD4Axf/ABNbdFFw5UYn/CG+F/8AoW9H/wDAGL/4mj/hDfC//Qt6P/4Axf8AxNbdFFw5UYn/AAhvhf8A6FvR/wDwBi/+Jo/4Q3wv/wBC3o//AIAxf/E1t0UXDlRif8Ib4X/6FvR//AGL/wCJo/4Q3wv/ANC3o/8A4Axf/E1t0UXDlRif8Ib4X/6FvR//AABi/wDiaP8AhDfC/wD0Lej/APgDF/8AE1t0UXDlRif8Ib4X/wChb0f/AMAYv/iaP+EN8L/9C3o//gDF/wDE1t0UXDlRif8ACG+F/wDoW9H/APAGL/4mj/hDfC//AELej/8AgDF/8TW3RRcOVGNoej6ZpHxCsRpum2dkJdKvPMFtAse/EtrjO0DOMn8zXoFcZZ/8lD0z/sFXv/o20rs61jscVX42FFFFMzCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigDkfEX/I8aJ/2Db7/0baVPUHiL/keNE/7Bt9/6NtKnrOW52UPgCiiipNgooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAp2f/ACUPTP8AsFXv/o20rs64yz/5KHpn/YKvf/RtpXZ1pHY4a3xsKKKKozCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigDkfEX/I8aJ/2Db7/0baVPUHiL/keNE/7Bt9/6NtKnrOW52UPgCiiipNgooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAp2f/ACUPTP8AsFXv/o20rs64yz/5KHpn/YKvf/RtpXZ1pHY4a3xsKKKKozCiiigAooooAKKKKACiiigAooooAKKKKACvPbKXXNU+23J8T6lbKuoXcCQwQWuxEjuJI1A3Qs33UHUmvQq4Lw9/x5Xv/YV1H/0smqZOyNaMVKVmS/Y9b/6HDWP+/Fn/API9H2PW/wDocNY/78Wf/wAj1o0VHMzp9nDsZ32PW/8AocNY/wC/Fn/8j0fY9b/6HDWP+/Fn/wDI9aNFHMw9nDsZ32PW/wDocNY/78Wf/wAj0fY9b/6HDWP+/Fn/API9aNFHMw9nDsZ32PW/+hw1j/vxZ/8AyPR9j1v/AKHDWP8AvxZ//I9aNFHMw9nDsZ32PW/+hw1j/vxZ/wDyPR9j1v8A6HDWP+/Fn/8AI9aNFHMw9nDsZ32PW/8AocNY/wC/Fn/8j0fY9b/6HDWP+/Fn/wDI9aNFHMw9nDsZ32PW/wDocNY/78Wf/wAj0fY9b/6HDWP+/Fn/API9aNFHMw9nDsZ32PW/+hw1j/vxZ/8AyPR9j1v/AKHDWP8AvxZ//I9aNFHMw9nDsZ32PW/+hw1j/vxZ/wDyPR9j1v8A6HDWP+/Fn/8AI9aNFHMw9nDsZ32PW/8AocNY/wC/Fn/8j0fY9b/6HDWP+/Fn/wDI9aNFHMw9nDsZ32PW/wDocNY/78Wf/wAj0fY9b/6HDWP+/Fn/API9aNFHMw9nDsZ32PW/+hw1j/vxZ/8AyPR9j1v/AKHDWP8AvxZ//I9aNFHMw9nDsZ32PW/+hw1j/vxZ/wDyPR9j1v8A6HDWP+/Fn/8AI9aNFHMw9nDsZ32PW/8AocNY/wC/Fn/8j0fY9b/6HDWP+/Fn/wDI9aNFHMw9nDsZ32PW/wDocNY/78Wf/wAj1teDb27v/Diy31y9zcJd3cBmdVVnWO4kjUkKAudqjoBVaneBP+RZf/sJah/6WTVUW2Y1oqKVjpaKKKs5wooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKAOR8Rf8jxon/YNvv8A0baVPUHiL/keNE/7Bt9/6NtKnrOW52UPgCsTxX4mtPCXh+bVbtGkCEJHEpwZHPRc9vXPoDW3XOeOPCq+MPDM2l+cIZtwlhkIyFcZxn2IJH41KNJXtocCfiT4+Onf20PCEI0nb5u8h93l9d2d3THOduMc13nhfxrp/ibwtJrkaNAkAYXMTfMYio3EcdRjBBrzBLz4oeCNP+x3Wmx6ppMEflgNGJ0EQGMZXDBcf3u3tW7YX2keOPhhq6aPapoM1upkuYbONVBKqWH3QMqwBHPvVNGUZO52vg/xdbeMtNutQs7eWG3humt080jc4Cq24gdPvdOeldFXifwP0GWWyuNdj1GZRFPLbizOfKY7EO889efTtWbrWn6bDJqNx4m+Iks+r7m8iLT3Z1Q44BAB288bRjGOtFtRqo+VNo9+orxjwl4h1TUvgr4jku72eW4sxLHDOzneF2KQN3Xgk81T8AeEtY8YaNZavfeJr6KC1uCkMCMxLKG3MS24HJJIzzwPphWH7S9rI9M0/wAbWOp+N7zwxawytLZwvJPO3ChlZV2gdT97rx070xdd8QH4gPo50QjRAmRqGxsE+WGxnp97ivH/AAr4RluvitqmkDWryNrAPI1ypO+bbIgw3PQ5/Suvivrw/tGTWf2qf7KIQfJ8w7P+PcH7vTrzTsSptrXuet0V4ZBFrOv/ABg1/RLfXL2ytG3mYxSHcsYK/Kmfukkjkds1Y8MjUfCHxmXwumq3V5p1zGx23Dlv+WRkBx03AjGR1FKxXtPI9rorxOb+0/iT8UNU0eTVrqx0nSy6iO2faTsYJn0JLZOTnAGKgEOs6D8ZtD0W51u8vbRFXyTLIctEQ5AcZwxByMnsB9KOUPaeWh7nXjuvfGW/0Txhd6S2m2rWltc+U0uW37MjJ64zivYq+bNf0r+1fGvj1VXMltbSXKe2yWIt/wCO7qcbdQqtpKx7V498Wnwh4XOqQxxTzPKkUKOTtYnk9P8AZBNZvw18d3njeHUnvLOC2No0aqIiTnduznP+7XnV9q//AAm0fw90HdvLANdr1J2t5ZJ99sbn8a0vhHqKaPo3jTUnXctptmKj+LaJDj8cUW0JU25rse3UV4b4W8M638TLK68Qav4kvbbdM0dtFATtQjHIGcADOMDnjrWr4B8Y6rp0viTQNdna9m0SCaeOZmJZliOGUk8kfdIzzyaVilU7rc9drhvCHjm88R+L9e0ae0gii02R1SRCdz7ZCvOfpXDeCfD2sfESG98Sal4l1G1nW4MduLZyAjAA5Azwo3AYGOh5qP4X3U+k+KfGd3qL+dcWkEstww43sjksfxINOwvaNtdj3as/XtQfSfD2palEivJaWss6o3RiqkgH8q8B0vUbTxd9u1XxR43udMvTIVtbeJmCxjAIOB/DzjAweDk11XhLxVfeIfhV4stNSuDc3On2UyCdjkujRPtye5yrc+mKOUFVTO7+H/im48X+GRql1bxQSee8eyIkjAxzz9a6mvLPhPaS6h8Jb2ygmMEtw9xEkoz8jMoAbj0JzXY+CvD194a0I2Goak2oTGZpPOYtnBA45JPb9aTRcG2kdFRRRSLKdn/yUPTP+wVe/wDo20rs64yz/wCSh6Z/2Cr3/wBG2ldnWkdjhrfGwoooqjMKKKKACiiigAooooAKKKKACiiigAooooAK4Lw9/wAeV7/2FdR/9LJq72uC8Pf8eV7/ANhXUf8A0smqZ7G9D4jWooorM6wooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKd4E/wCRZf8A7CWof+lk1Np3gT/kWX/7CWof+lk1VA58RsjpaKKK0OUKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigDkfEX/I8aJ/2Db7/wBG2lT1B4i/5HjRP+wbff8Ao20qes5bnZQ+AK474k+Ebjxf4Y+y2UgS9gkE0IZsK/BBUn3B/MCuxoqTVpNWZ4za+K/idYaUmkt4TeW6ij8lLsxMw4GATg7SffOK2/h54B1Dw/4P1eHUCkeoarGVMQYERAKwUEjjOWJOPavS6KdyFT1u2eMfDmw8WaTp2peFbjQ57OK6E8g1FyQIpDEFXBHBGVHINUPCGmeKvDlhfaHH4JWTUbh2CapKwCRAgDlsHIHJAB79K92oo5hey21PGfB/hnXbD4YeKtHu9LuIryfcYEIH73KBcL68r+tdl8LNKvtG8C21lqNq9tcrLIzRuOQCxxXaUUN3KjBKx4w+n+KfCvxa1TWbDw/NqNtf5RZE+6FdlYkkdCCuMHFa8fh7Vh8e5tbNhN/ZhiwLnHy58gL/AD4r1Cii4vZr9Ty/wx4e1az+M+vatcWEsenzxSCKdh8rktGRj8j+VF34f1Z/jzZ60thKdNSLDXOPlB8ll/mQK9QoouHs1+NzxvUdE8SeFPiPf6/4VtYNVgvi32i2Ei7kZiGZWGQR8wyCPoax1OuzfHHQ59fjhivpgsn2eE7lgTa4CZ5yeCTz3ruvEXwtXU/Ej+INH1u50fUZOXeJNwJxgkYZSMjryan8LfDVNE19tf1TWLnWNV2lUmmXaEyME8kknHHXGD0p3RHI7/M7uvK/DvhfUl+K/im8v9PlTS762miSZgNsgZ04H1AP5V6pRU3NXG9jw/4WeAdY0bxxNfatYSQwWkMiwSuBh3J25H/AS1a/w08H6jBpfirT9bsZrSLUcRqXAyykOCR9Nwr1minzExpJWPFNB/4T34cQXWiw+HP7XtGlMlvPCSVBOBnjnBwODg9ea3PAXgLUYhrmr+JsJqGtRyRPChBMaOSXzjjJOOOwH5en0UXBU0jxTwxF47+Hr3ug2vhz+1beaYyW9wr7YwxAG4t6EAcHBHrVv4eeD9btfEHidPEFo6R6hA8T3A+7KWY7iv5k17BRRcFTStrseFaPpXifwGt9pMngqHXo3kL212IwwBIAyeCccA7TjHPPNdlpWjeIT8Ndci1eztE1S9s5kit7SBI2x5bBQ2zgsST9M/WvQ6KLgqaXU86+Hem634c+GV7FJYPHqkZnlgt5V++235BgHoSMda6TwZqGv6loRn8SWKWV/wCcyiJEKjZgYOCT7966Gii5SjawUUUUiinZ/wDJQ9M/7BV7/wCjbSuzrjLP/koemf8AYKvf/RtpXZ1pHY4a3xsKKKKozCiiigAooooAKKKKACiiigAooooAKKKKACuC8Pf8eV7/ANhXUf8A0smrva4Lw9/x5Xv/AGFdR/8ASyapnsb0PiNaiiiszrCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAp3gT/kWX/wCwlqH/AKWTU2neBP8AkWX/AOwlqH/pZNVQOfEbI6WiiitDlCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooA4PR/ilpmqfEG/wDB8lrJa3dtJJHFK7grOyHkAdjjJ/A1pr41hb4jv4O+xSectp9q+07xtxxxjGe9eFXnha913xj8QdV0eSRNZ0TUUu7Ty+rYeQsoHr8oI91x3roPAHiqLxl8bYNaRPLkk0XZPGBwsi4DAe2eR7EUAe/VleIvEOneFtDuNY1SRo7SDG4ohZiScAADuSQK8Y8F+Hrj4vW+seJdb17VLeZbx4LOC1n2JagKGB2/8CA7ZwcnJzXPapqd94s+BF5daxe3Fxd6HqIto5hJ8twrFB+8/vEbjg/QnNAH0pp96mo6ba30SssdzCkyq3UBgCAffmrNfPXiqS68K+CvBXhvS77Vzb67Is928L77llKxZii6cHfwvsBnBNbHgdNW0X4j2tro+l+K4fDF5Ay3KazA22GUKxDKeQAcKP8AgR9qAPbaxPEOratpSW50rw9PrBkLeYIriOLysYxneRnOT09Kr+HvCn9ga5rmpf2pdXf9qzCbyZj8sGC5wnt8/wCgro6APONN+JOv6vLex2PgK9mexuGtrgC/gHlyL1XkjPUcjiu10nUbu/a9F1pk1j5E5ijMrA+cuAd647ckfhXI/DP/AJCnjn/sYp/5LXH69f3sXgL4mTRXcyzQa8qwuHOYx5kHA9ByeKAPcKK8h8TeEo9A1/wsumatq0Fxqt4bPUbn7bIz3KFCWJ3EhW4OCoGM8dBVmGKPwJ8QtXs9Ia4/s7/hGpNTNpNcPKvnxybcguSRkdeaAPVaK8C06GfV/CcertonjS58T3UJuIdYglAjWQ8rsXzgPLHAxt6dq6PxLbeItTh8Oalreg6jqemDT8ajpdhcGKRLo4y5RWBcdQFB4/mAejeI9ZTw74c1DWJIWmSzhaYxqcFgO2a5a28b+Kry1huYPh5fPDMiyRt/aNuNykZB5Poa5c3mlTfCbxrbaVf6m8cETltP1NWE1huXiP5iTt4JHJ7+9b3h63+I58P6UYL/AMMi1+yw7A9tOXCbBjJ3YzigD0CzlluLG3muLdraaSNXkgZgxjYjJUkcHB4yPSp6818S6adc+MGm6VcXl3Hp76LJJcQQTtEJgJRhSVIIGcHgg8Y6Gszw74OstT8VeLPDt9ealNoelTwmysvtsqrE0sYZjuDbmxj5QSQMk9TmgD12iuK+FN5dXfgK2F5cy3MlvcT24llbc7KkrKuT3wAB+FZGt6UPEPxpGlXl3eLpg8PLPNbQXDxLMwuHUBipBxzngjOBQB6ZRXjX9qXfgW0+JVjpdxO9ro8drLp6TyGU27TxnOC2SQDggH0+tZ0lje2ui299oHh3xoviiMRy/wBo3MoeO5bILCRfOIKMM8BfSgD3aivHde1OTxB8RNQ0zVdG17UdM0y0tyun6W4VfNlXeXlPmITj7o5I4P415da8R+F/BPjBrez1ew0+HyP7IfVSGmg81hHIu7c2QpOVyTjNAHp/inxHH4X0qO/lt2nV7mK32K20gyMFz+Ga268c8e+A9P8AD/hrT7ywu7/7Qmo2i3TT3ckgu8yKNzqxI3bsEEAdxVnxlp10njS+vvEGga1rmgSQxCxOlXD5syF/eZiRlJJbnd6Y/AA9aormPh9eWd54OtDY6xdarDGzx/aLtSJgQx+RwecrkDnsBXT0AFFFFABRRRQAUUUUAFFFFAHI+Iv+R40T/sG33/o20qeoPEX/ACPGif8AYNvv/RtpU9Zy3Oyh8AUUUVJsFFFFABRRRQBwvi7xN4v0jWVttD8M/wBpWhhVzPtY/MScjg+w/OuP034teMdZmnh07wtBdSW/EqxbyU5I559jXtVeL/BP/kZvFX++v/ob1StYxlfmSvudTc+K/GMOnaDNH4XL3F40gvogjn7MA4C9OmV55rv68k+MV9eWev8AhRLa6ngWSZw4ikKhvmj6469TVT4kXOrn4t6Fp+l6nPZtc2sUYZGO1S0kiltvQkD+QotcfPyt/I9morwDxHput+CPHml6foviPUJJNVVYzLdPvO93KEkdD1BHGRVrW9P1H4e/EDw81nruoXiX8qi4FzIW8z5wrA9iCG4zyPWjlD2ndHutFeJ/EPxLJqXxCj8L3OtyaPotsqm6miJBdim/nHXqoA6Zyeag8LeIo/DnxGstG0nxFNrOg6gAmJmLGKQ5AxnGDkDpgEN6ilyh7VXse50V89+JtXS5+I2q2fjLUtasrCKRls0sWAVUz8rEHPBXnIBJNd/8LlVEvltPF663p2cwwSK3mwDPBO45GR2xjI470NaDjUu7GEnxV8WX+tahp+keGoL02cro3l7yQoYqCefauj8LeKvGeqa9Faaz4W/s+yZWL3G1htIGQOT3NeYeFdU8TaX418Rv4Z0mLUZnnkEyyAnYvmHB4Yd69d8Gaz4w1S6uk8TaJBp0SIDC0YI3tnkcse1NozhJvds7GivBPB2m6t4w8SeJNMm8Qaja6dFOXlWGU73O9wqgnovXI74H4R+DNN1zXdf1jwdN4kv4tKsJZDI0T/vHKvsCgnO0HqR046Ucpaq36Hv9FeP/AAru9R07x14h8LT3893Z2gdojMxJBSQLkZ6ZDcj2r2CpasXGXMrhRRRQUFFFFABRRRQBTs/+Sh6Z/wBgq9/9G2ldnXGWf/JQ9M/7BV7/AOjbSuzrSOxw1vjYUUUVRmFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFcF4e/48r3/sK6j/6WTV3tcF4e/wCPK9/7Cuo/+lk1TPY3ofEa1FFFZnWc/qPjPSNL8T2fh65eYX92EMQWPKncSBk9uQa0da1i00DSLjVL4uLa3ALlF3HkgdPqRXk3jb/kvfhr/ct//Rj13HxT/wCSaa1/1zT/ANGLTtsZ8ztLyN/QtbsvEejQarp7O1rPu2F12n5WKnj6g1o15H4b8UHwh8CLDVI7cTzCSSOJGOF3NM/LY7Dk/pTbPxN8Sp9PtdYtI9F1e2mKlrSzO50DdmwflI6dTg9aLAqisrnr1FcV448eP4S8N2d2LEnVL7Cw2kpzsbALbtvXbkDjqSK5m68a+OvB0ljfeLbGxl0u7kCP9m4eEkZxwcZxk45zjrRYbqJM9brlvE3xA0LwlfxWeqyTrNLF5qiOIsNuSP5g108ciSxrJGwZHAZWHQg14d8X7m2svid4dur2PzLSCKGSZNobcizMWGDweAeKErsKknGN0dh/wurwd/z2vP8AwHNdFqnjTR9H1fTNLu3mFzqRQW4WPIO5toye3Jri9O8ffDrVNUtNPt9AUTXUyQx79OiA3MQBk56ZNWfHWvvpfxC8L6dHp2mzpcyRKZbi2DyR5lC/I38OOo96diOd2vc9MorzHxh4/wBc0H4g2ehabZQXkVxApSEgh3kbcFG7OAMhc8dM1Rbx14z8NeMtN0zxTa6e9rqLqqm1B+QM23IOexIyDmlYp1EmeuUV554z8c6taeJ7Twp4XtYZ9WnUM8k33YgQTjGRzgbiT27GoNB8beIdO8ZQ+FvGFtarcXSb7W6tuFbrgH2JBHY5HTnNFh86vY9Kqleavp9hdWtrdXcUVxdOI4Ii3zyH2HX8elWjLGpwZEBHYtXgPjWXxJ/wuaxaOG0N+pX+zVJ+V4t77C/PX72elCVwnPlR9A0V5p4l8a+IfD+teE9MkjsxNqKxLfDYW2uWVW2EHgcn1q58U/GeqeDdP0+fTFty9xKyP5yFhgAHjBFFg51Zvsd/RXnPxI8d6n4PvdCFlDDLFdlzOjoSzBSnCnPBO4+vasPxB468f+Eriz1LWtO01dOunx9miO5k77S2eGx35H8qLCdRI9iorg/Hfj6bw/Dplno1qt3q2qY+zpJ91QSACRkZJJwBn19OcZPG3i7wpr2nWfjO1snstQbYlza8GJsgHPY4yMjHfgnGKLDdRJ2PVawPEfjHSfC09jDqbyq96zLD5ce7JBUHPp94Vy/jnx5q1h4ls/C3hm1in1W4AZ3lGQmckADIGcDJJ4A/Tz74iXniaTV/Dtl4ntLVLm3lZ47i1YlJlZk7diCvP1HHq0iZ1LJ2PoiiiipNQp3gT/kWX/7CWof+lk1Np3gT/kWX/wCwlqH/AKWTVUDnxGyOlooorQ5QooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKAOW8N+CYfDnibxDrUd7JO+szLK8TIAIsFjgHPP3v0rM0L4WaZ4d8f3nimwupIxcq4+xbBsQvgtg+mQTjtmu8ooA8xv/AIPINTv7nw/4n1PQ7bUWLXdpbcxuTnO3kbep9cZ444rYm+F+if8ACuZvBlq01vaSYc3HDSGQMG3t2JyoGOOOBiu2ooA86k+E8N94PtNE1TX7+7u7GfzrHUQNktthVAVRk/KNo4z+VXfDPw8l0fxE2v6v4iv9b1IReTE84EaRp/ugnJ5P5njJzXcUUAc54e8LSaFrmuai+r3V6uqTCVYJidttgudqcnj5sdugro659PGugyadaX63jG2u74adC/kv805YrtxjI5B5PHvXQUAYXh3wzF4eudamjuXmOqX73zBlA8ssANo9Rx1rGv8A4c299oPiXSm1GVE1y/F68gjBMRDI20DPI/d/rXbUUAYmueHI9b1LRb17hom0u7+1KqrnzDtK4Pp1psvhiCfxn/wkUkxYnTG01rZkBVlMgcsT+GMVu0UAefD4aXsGmS6FY+LtQtvDsm5TYiGNpEjY5aNZj8wXkjoeK09S8DbrvTr7QNWn0W9sLT7DG6RrMjwDBCOj9cEZznNddRQBxSfDyN9F8Q217q1xd6lryBLy/eNVOFXagVFwAFGePfrXV6bZjTtLtLFXLrbQpCGIwWCqBn9KtUUAYkvhyOXxrb+JTcOJYbFrIQbeCGcNuz68Yo0nw5HpXiLXdYS4aR9XeF3jK4EflpsGD3z1rbrPg1qwudbu9GimLX1pGks0ewjar52nOMHoelAFTwr4cj8LaL/ZsVw1wvnyzb2Xacu5bGPbNA8ORjxw3if7Q/mnTRp/kbflwJDJuz684xW3RQBzX/CF2Muq+J7u7driDxBFDFPbsuAixxlOD15zn2rEPw2vrixg0bUfF2oXnh+BkxYtDGjuiHKo8w+ZlGB2HSvQKz59asLfXLXRpZit/dRPLDHsJ3KmNxzjA6jrQBia14Mku9eXXdE1ibRtU8gW8skcKyxzRg5AeNuCR2IINMsPh/Yx6TrNpq13catca1/x/wB1PhWfAwoVV4QL2A6flXXUUAeeXfw01DU7S0stU8Y395Z2M0ctrE1vGpBQgjzGHMhwCMnHXPJrU1Dwbf8A/CQXms6B4juNInvlQXcX2ZLiOQoMKwVvutjjjr6V19FAGL4Y8N23hfSTY2801w8kz3FxcTEb5pXOWc44GfQelbVFFABRRRQAUUUUAFFFFABRRRQByPiL/keNE/7Bt9/6NtKnqDxF/wAjxon/AGDb7/0baVPWctzsofAFFFFSbBRRRQAUUUUAFeV/Cjw7q+ia/wCIp9SsJraK4ZTE0g4f5nPH5ivVKKLkuN2n2PL/AIreHtW1rW/DM2m2EtzHbSuZmjHCAtGefyP5UeLvD+rX3xi8O6ra2EsthbxQiadR8qESSE5/Aj869Qop3JcE3c8v8e+H9W1L4l+FdQsrCWa0tZYTPKg+WMCYMc/hzR8TPD+rav4v8L3Wn2EtxBaygzOg4QeYh5/AGvUKKLjcE7+Z5L438J6xY+PYfGGj6TFrELqFurJwGJIXZ0PUFcYIzgjpVvwtF4m1jxYt9c+GbHQtGhGfJltkMpYDja20NnPOcAAD1r0+ii4vZq9zynxA/i+08QXcer+GIPFWhyZ+yqkKbohnI/hLA44OR7g1F8M/B2qWXi7UPEVzpX9i2UsbRwWBk3N8xB+uBt745PAxXrdFFw9mr3PK/hn4d1fSPGniS71Cwmt7e5ZjDI44f94Tx+FeqUUUm7lRjyqx5f8AC/w/q2j+JvE9xqNhLbQ3MoMLuOHG9zx+BFHgDw/q2mfEbxVf3thLBaXUspglccSAzEjH4c16hRTuSqaVvI8v8G+H9WsPi54l1S6sJYrG5WYQzsPlfMqkY/AGvUKKKTdyox5VYKKKKCgooooAKKKKAKdn/wAlD0z/ALBV7/6NtK7OuMs/+Sh6Z/2Cr3/0baV2daR2OGt8bCiiiqMwooooAKKKKACiiigAooooAKKKKACiiigArgvD3/Hle/8AYV1H/wBLJq72uC8Pf8eV7/2FdR/9LJqmexvQ+I1qKKKzOs8X+LMV1ofjzQPFotnmsrcRpIV6bkkLYJ7ZDcfQ0zx98VdC8QeDrjSdJS6lur3YuHi2+WAwY555PGOM9a9olijniaKaNZI2GGRxkEe4NUbbQNGspxPa6RYQTDpJFbIrD8QM07mTg9bPc8/W1Hhj4I2drrWhS6lEELXVqrbWjDuzhieo2kjOOQfpXmOr2/g600mLWPC+v6hbasdhWwYMWRiRuAkAXGOecnOPevp8jIwelZ0WgaNBdC6h0mwjuAc+alsgfPrkDNNSFKnc8Z+I+na3f+AvCmt6nFM89pGRe8Ydd+0qzehwvJ7Eis7UX8A3i6fbWFx4k1qe6cA2ouCGjPbhkwWycce/NfQ7KrqVZQykYIIyCKpWmi6VYTmez0yzt5m6yQwKjH8QKOYHS1JrG3Wz0+2tkDBIYljUOcnAAHJ7nivHviYiyfGPwijqGRmtgysMgj7QeDXtNV5rCzuLiO4mtIJJ48bJHjBZcHIwTyOaSdi5R5lYjTSdNjkWSPT7VHUgqywqCD6g4ryj4m/8lY8Gf9doP/R4r2Oq81jZ3E8c89rBLNHgxyPGGZOc8E9OaE7BKN1Y8k8UDP7Qnh//AK4x/wDtSj4v/wDI7eDv+u3/ALUSvW3sbOS6W6e1ga4ThZmjBdfoetE9jZ3UkclxawTPHyjSRhiv0J6UXJdO6a7nhvxD0230z4txaprb3sGjX6Lm5tGKshEYQgEA9CASOuDVjw9beENT+IlhBov9u6o9qVnW+kuMxxlTu+YMoO3OB7k4r2y5tLa9gMF1bxTwt1jlQMp/A0yy06x02Mx2Nlb2qMclYIlQE/QCnzaB7PW553r3wY07XtdvNVl1a6ie6kMjIsakL9K5nxy9v4Y+K3hW5u3cWNnZQI020nIRnBOB36HA9a9zqre6bYakipf2VtdKhyoniVwp9sikmOVNdDxr4q6jFJrHg3xRAskmmHbKsgXsHV8exI7H0NZvxe8Y6V4qsdNTRnlube3kZprgxMiKzD5U+YDnAY17zLYWc1mLOW0gktQAPJaMFMDoNvSov7H0sWqWo02z+zodyxeQuxT6gYwKaZMqbd9dzyf4yDOveDQf+ej/APoUVXvj3/yKem/9fw/9FvXp9xYWd20bXNpBM0f3DJGGK/TPToKW6srW9jCXdtDcIpyFljDgH15pXKcL38zxD4p6W8V34V164S4bTEtoYLh7c4ePB3cHsSGOD6iqc9v4F1XXNN0vS5vEWtyXDfeW4I8kkjs6emST0AFe/PBDJAYJIkeEjaY2UFSPTHpVey0jTNNZmsNOtLVn+8YIVQn64FPmJdLU8h8ZTN4P+NOn+KL6GRtMnQAyIudp8sxsPqOGx6Gsb4l+LtO8Wa7oDaT5s1naSlTctEyK7syEqNwB4Cj8699urS2voGgu7eK4hbrHKgdT+B4qD+yNM8iKH+zrTyoTmJPIXah9hjj8KLjdNu6T0ZdoooqTUKd4E/5Fl/8AsJah/wClk1Np3gT/AJFl/wDsJah/6WTVUDnxGyOlooorQ5QooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKAPGj8QW1zUtVlbx7ZeGoLW6e2s7Q28cjyBDjzJC4Jwxz8oxxV+2+Iura/wCFvD8OmPa2+t6tfyWElyqeZFEIsmSVFP3sqFIB4+b2rRtPD3ivwndanbaBaaRqWmXtzJdW/wBsmaJ7V3OWU4U70zyMEHrU974L16fw9osx1eC48T6Tcm8juJY9sMjNkNEQoyEKnGcZ4zQBCLrxVoPxD8OaLe66NT0vUVuWaSS1jjl3Rx52kqAMA7SCADyQc4FangfXdQ1nQ9Yub6cSy22p3dvEwRV2ojYUcDnHrXM3MviSf4u+Cptet7G0Ro78QWtrM0u0iEbmZyoznIwAOMdeav2PhzxloMus6VpP9lPpmpXs11FezyOJbUS8sDGBhyO3IHr6AAh03xL4i1P4W6Bqzazp+nz3bH7fql4EUQxhmGUQ4UscAAHiofC/jC4b4hW2gw+LYvEun3drLIZfs6I1vImDgNGArAjP0/nH/wAK61u08L+DYETTb+68PyStPYXEjC3ud+cENtPzLnIyvUn8da08N+J7rx7o3iPUY9LtLOzt57f7BayMxhVl4YMVAck9RhQABjNAHE2f/JP/AAv/ANjsn/pRJXomleJLuy8XeKNH126UxWca6jZylAv+iMPmHAGQjAjJ55rGt/h9q8XhbRtMaWz8+y8RLqkpEjbTCJWfAO372GHHT3qv8VtJGr674ctdOvFi1LUJJNNuEQ5drNxulJ9l25/4FQB13gLUdV1nwpBq+rP+9v5HuYItgXyYGb92nA5+XByeea5PxX4svbTx1caRf+J28LaekET2Nx9iSVbxyDvLO4KqFOBjjPrXp0EEVtbxwQoEiiUIiDoqgYArkNbsfFia1eSWdvpmuaNdxoBp+oSeV9mdRg7TsYMrdTnnPtQBja34h8Waf4H0Ce21HSrrV73VobMXVqBJb3COWCk8cZwudvTnBq3Ne+I/CHiTQo9U13+2dN1e4+xSebaxwvBMVJRk2AZUkEYOcetcnr/hTUfC3gbRbKO4tYdSu/FcF1GIFJt7WSRjtRAcEouB6d67NdD8TeI/EWkXviSHTrKx0iU3EdvZztM1xPtKq5JVdqjJIHJ9aAMnXtc1G31PURqnxB07w60cxWysLeOGdzGMbXlDAuS390Y/GorPx/rmseBPCTWZtodc8QXL2v2ho90cKxs4eUJ3OEyB0yasaJ4V8V+GrjVLXT7HQ52vLyW4j1q4lbzkDnPzpsy7D03AUzT/AIc61p3gbQLKC7sxrugXstzaysWMMys7ko/GQGV8HGcEUAbKt4p8JTXl7qurLrmhQ2MtzLLLDHBPDJGN2AEADKwB7cfz4MfEW/fw8fEJ8f6UmpeWbkaD9nj8rb18kt9/fjjOevFd4mi+J/Et/I3iRrfTtK+xzWp06xuWl88yrtZ5GKqMAfdGDgnP1y7TQ/iBpvh6LwzaDRNkMYt4NaZ28xIRwCYdpy4Xj72M4oAml8T6t4u13SdI8P3/APZMFxpCatdXfkrLKFkICRKGG0HqSSKg8F2+q2nxX8UQaxeR3twljahLlIhGZUy2CyjgN1BxxxWrqvhrXLDxDp/iPQJba8vYbAafewXrmIXUYO4OGVSFcNk9MYNL4V8P+IrbxlrPiHXpLEHULeGOOC0dmEOwt8uSBngg57kngCgCz471rUPDcGlaxbzBdNhvo49TjKAgwSHZvyRkbWKnjHXmkn1vUbz4lxaLp0oFjp9g11qA2qfMkkOIo9xBK9C3Fb+uaTBruhX2lXQ/c3cDwscdMjGR7jr+Fcp4L8F6ppHhfVYNZv0k1vVMrPeW5J2qIxHHgkAnaBn6k0Achrni3WNK0i71C6+ImmQ65CHkGiWkEM8QK5IiJwXyQPvEjGa6D7c2qfEzwRqDIEa60SecqOillRsfrWdaeCvF0PgWfwhBp/h+wVrZ7eTU45WZrhSD/BsBBboWLHGSQD0rotO8Janb694Rv53ttmk6Q1lchXJJkKoMrxyPlPXH0oAPCHiLU9V+HF5q95cCS9ja8CyCNVx5buF4AxwAK5q9+IuoQ+FvBsE2sWunX2t25mu9VuIlKwIqgkhOFLMSAO3tV+y8LeNNH0nU/DGnnSW0u7lnaHUJZX82GOUkkGMLhmGTg5x/KpR4C1my0HwlcafNYnXfD8BhaOYsYLhGUB0LYyOgIOOtAFfwx45f/hM4vD6eJrbxPbXltJLb3CRpHLDKgyY32AKQRkg4zxWNovjLXdXVJB42srPxEJ9svh7UrRbeFfnx5QcrvJ285BJ7YrutIsPFt7rj6hrEllpVklu0UNhYuJmZz/y0d2QdOwA+vvzWs+EfGXiHQW8O6xZ6BesfkXXZWPmqm7O4RbOHx6MBmgD1WiorWD7NaQ24dnESKm9zktgYyfepaACiiigAooooAKKKKACiiigDkfEX/I8aJ/2Db7/0baVPUHiL/keNE/7Bt9/6NtKnrOW52UPgCiiipNgooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAp2f/ACUPTP8AsFXv/o20rs64yz/5KHpn/YKvf/RtpXZ1pHY4a3xsKKKKozCiiigAooooAKKKKACiiigAooooAKKKKACuC8Pf8eV7/wBhXUf/AEsmrva4Lw9/x5Xv/YV1H/0smqZ7G9D4jWooorM6wooooAKKKKACiiigArC8T+LtK8I2sFxqryrHO5RPLj3HIGa3a8i+Pv8AyL+k/wDX03/oFNK7InJxi2j0fw54k07xTpf9o6XI7weYYzvXaQwxkEfiKwk+KXhmTxCuiJPcNeNdfZBiE7TJu29fTPeuH+H2qjwa/jbSJm+XTle8gU/xBQR+o8r864Lw5p8lv4l8HX8xJk1DUUm3HuBOF/mrU+UzdV2R9UUV53428falp3iG18L+GbKO71icAs0vKx5GQMZHOBkknAHr2yovH3ivwp4gs9P8b2Vr9jvTiO7t+NnIBPBwQMjIwDznmlZmjqJOx6zWB4k8Y6T4Vkso9TeVWvWZYfLj3ZK7c59PvCuY8dePNW07xHZeF/DNrFPqtyAzPKMhM5wAMgZwCSTwB+nnvxHvPE76j4fsvFFrapcQSM8VzasdkysUBGOxBX9Rx6tImdSydj6Iorz7xv451PTvEFl4X8N2sVxrF2AxeblYwc44yOcAk54A9c1U0fxr4k0jxja+GvGVtab74A213a8Ak5AB9ckY6Ag47GlYrnV7HWjxjpJ8XnwuHl/tMLuK+X8uNm/730rfrxxP+TmJf+uP/tsK9joaCEm73CiiikWFFFFABRRRQAUUUUAFO8Cf8iy//YS1D/0smptO8Cf8iy//AGEtQ/8ASyaqgc+I2R0tFFFaHKFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRQc4460AFFefTfEqSD4dT+IX0sHU4LxrBtOWUnNwJdmwNjPT5ulaf8Awmy3A8IrY2qzSeIR5u0yY8mJY98jdOSMgY45NAHQG00zUby11Ly7e4uLMyLBOCGMW4bXAI6Zxg/SrXnw/aPs/mp523f5e4btucZx1xnvXimuavc3Hw0tZtC0u2sLZ9eMU0QunGXFwAP4TkO2S3p71r6nf69bfGSwNppVtc6tceGjG8X2krBCftG4sX25KjbjhckkUAesUVyvhPxRf6vqer6LrOnxWWq6U0XmiCUyRSJIpZGUkA9jwf8A6wd428WTeE7bS5LfTW1CW/v0skhWQISzqxGCeOqgc+uaAOorLt/DejWuu3GuQ6dAmqXChZbrb87DAGM9uAOnpXPad4s1638X2OgeJdHs7VtShllsp7K5MqkxgFkbcoOQDnI4/pWg8Z+JdeF7f+F9AsrrSbWV4kku7wxSXhQ4YxAKQBkEAsef0oA76isjwz4htfFPh601izV0iuFOY5Bho2BKsp9wQRWLqfirWLjxRc+H/DGmWl1cWMSS31ze3DRRRFxlEG1SSxAz6CgDp77TbLUlgW9to5xbzLcRBxnZIv3WHuKtVwmj/EC+v7DxLNd+HLmO70SRYjZWzGeWZyuSFwORnoR25qK78YeKtBl0y68Q6DpsOm391Han7Les81s0hwu8FAGx32mgD0CiuGu/GGv3PjTVPDGhaLaTTWMcMrXd3cskSq65wwVSd2eAB6EnGKgk+Jn2Pwnq+oahpRg1XSblbO5sfOG3zHICMJMYEZ3A7iOBn8QD0CiuS0fXPFJ1m1s9a0SzNpdxs8V/ply00URAztk3KMZ7EcE11tABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQByPiL/keNE/7Bt9/6NtKnqDxF/yPGif9g2+/9G2lT1nLc7KHwBRRRUmwUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAU7P/koemf8AYKvf/RtpXZ1xln/yUPTP+wVe/wDo20rs60jscNb42FFFFUZhRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABXBeHv+PK9/7Cuo/+lk1d7XBeHv8Ajyvf+wrqP/pZNUz2N6HxGtRRRWZ1hRRRQAUUUUAFFFFABXkXx9/5F/Sf+vpv/QK9dqC6srS+VVu7WG4VTlRLGHAPtmhOzJnHmjY+e/jDZz6V4w+02xZI9WsYxLt/jKkAr/44hrW8R6WNF8a/DfTcYa3W1R/97zhuP55r224sLK8aNrm0gnMf3DLGG2/TI46UTWNpcTxzzWsEk0WDHI8YLJzngnkc1XMZ+y1bPHPFcsngr40QeKb63lk0q7QKZUXO0+V5ZH1GAcehqn48162+J+taLoXhlJbkRuzy3DRFVUNgE88gAAkk+2K9znt4bqFobiGOaJuGSRQyn6g1DZaZYaajJYWNtaq33hBEqA/XAouN027q+jPHvGcjeDvjJpfie8hlfTJYwpkUZwfLMbD6gENjvWP8TvF2neLNW0JtI82aztJWU3LRMis7lCVG4A8BR+de+3NpbXsDQXdvFPC3WOVAyn8DUH9kaZ5EUH9nWnkxHMcfkLtQ+wxxRcTpt3SejPFfibpkWnfFGy1fVjeR6NeIqvcWjFXjIUqQD6jg47gml0a28Hap4/0210Y69qzwOk63j3HyRFW3ch0B2jA54yTgV7jcW0F3A0NzBHNE33kkQMp+oNRWWm2GmoyWFlbWqNyywRKgP5CjmH7PW55Mn/JzEv8A1x/9thXsdV/sNn9s+2fZYPtXTzvLG/pj73XpxVik3cuMbXCiiikUFFFFABRRRQAUUUUAFO8Cf8iy/wD2EtQ/9LJqbTvAn/Isv/2EtQ/9LJqqBz4jZHS0UUVocoUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUVXvNQstPi829u7e2j/vzSBB+ZNYj+O/DQ4g1MXn/XhDJdZ/79K1AHR0Vy58apJ/x6aBrdz6H7OkP/AKNdKjPijXJD+48LlB/09X8af+gB6fK+wuZHWUVyLa14rf7ulaND9b+WT/2ktN/tDxe3/LTQ4/byZX/9nFPkkLnidhRXH/a/F5/5iGhj/uHSnH/kelW58XZ+bVtEI9tJlH/tzT5JBzxOvorkPtni4Z/07RH9P9AlX/2sacNT8Vp1i0aX6GWP/wCKo5JBzxOtorkxr3iZP9ZoWmSD1i1NwfyaHH61IPFeoR/6/wAL6g3vbXFu4/8AHpFP6UuV9g5kdRRXNr4105R/pVlq1qf+mmnyuB9WjDAfnVm18YeHLyURRa3YiY9IpJhHJ/3y2D+lTYq5t0UgYMoZSCDyCO9LQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQB5W3hfVB8XPs/2OQ+G5Lsa40u392LjyTEVJ6ZLYfFHw38Matp3im+/tS1kisdEik0/SnkBxLHJO8hkH/Adi/SvVKKAPGX8P61F8JZIF0m7kvLfXTem1WP8AevGLjdlVPXI5FdVp8d5qXxYtNf8A7Mvraxl8NmLNzCUMchuA3lsOzY5xXeUUAcboWnXkHxR8W30ttKlpc29ksMzKQshVGDAHvjIzR4+0681C68JNaWss4ttet55jGpPlxhXyx9AMjmuyooA43xHp15c/EzwTfQ20slraLf8A2iZVJWLfEoXce2SCBWB4Y1DVPh9okvhi68M6xfvaTTGxuLG382K5jd2ddz5whyxB3dPevUaKAOV+Heg3vh3wbbWepBFv5JJLm4RDlUeRy20H2BA+uaxpDqPhDx/ruqR6NfatpuspA5OnqsktvNGmzayEg7WGDntXodctq/giHUNbk1mw1fUtI1CaJYriSykXbOq/d3KysMjOARQB53ZzeKdT0v4kXGlWk1lrUt9BtgikBlRAi7lVhx5nl+nfpziqms+HrS9ttMn8O+C/EAuba+t57u91NJPOCBxkKJGLMSeTtGAAa9g8OeGrHwxp8lrZNPK00rT3FxcSb5Z5W6u7dzwPStigDyuLxBPoPxf8YMNH1DULSWCx81rCHzpImEbbcoOSDluR0IHrTrbTNdbR/FfiKTw5Hcz61dRONGvMFmtY1CAEZwJCuWx2IHeu8svD1pYeJNV1yKSc3WppCkyMw2KIgQu0YyOvOSfwrWoA8e8O6KsPjbSbjwdofiHQtPVpG1WLUFeK3ZCvCqjk7m3d14Few0UUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFAHI+Iv+R40T/sG33/o20qetLV/Dmm65NbzXyXPm26ukUlvdy27BX2lhmNlJBKLwfQVn/wDCCaJ/z01j/wAHd5/8dqXG7N6dVRVrDaKd/wAIJon/AD01j/wd3n/x2j/hBNE/56ax/wCDu8/+O1PIX9YXYbRTv+EE0T/nprH/AIO7z/47R/wgmif89NY/8Hd5/wDHaOQPrC7DaKd/wgmif89NY/8AB3ef/HaP+EE0T/nprH/g7vP/AI7RyB9YXYbRTv8AhBNE/wCemsf+Du8/+O0f8IJon/PTWP8Awd3n/wAdo5A+sLsNop3/AAgmif8APTWP/B3ef/HaP+EE0T/nprH/AIO7z/47RyB9YXYbRTv+EE0T/nprH/g7vP8A47R/wgmif89NY/8AB3ef/HaOQPrC7DaKd/wgmif89NY/8Hd5/wDHax7nwlpkfjLTLBbjWBazafdzSJ/bN58zpJbBTnzc8CR/z9hRyB9YXY1qKd/wgmif89NY/wDB3ef/AB2j/hBNE/56ax/4O7z/AOO0cgfWF2G0U7/hBNE/56ax/wCDu8/+O0f8IJon/PTWP/B3ef8Ax2jkD6wuw2inf8IJon/PTWP/AAd3n/x2j/hBNE/56ax/4O7z/wCO0cgfWF2G0U7/AIQTRP8AnprH/g7vP/jtH/CCaJ/z01j/AMHd5/8AHaOQPrC7DaKd/wAIJon/AD01j/wd3n/x2j/hBNE/56ax/wCDu8/+O0cgfWF2G0U7/hBNE/56ax/4O7z/AOO0f8IJon/PTWP/AAd3n/x2jkD6wuw2inf8IJon/PTWP/B3ef8Ax2j/AIQTRP8AnprH/g7vP/jtHIH1hdijZ/8AJQ9M/wCwVe/+jbSuzrF0zwrpOkah9utUvGufKaESXN/PcbUYqWAEjsBkoucegraq0rIwnLmlcKKKKZAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAVwXh7/jyvf+wrqP8A6WTV3tcF4e/48r3/ALCuo/8ApZNUz2N6HxGtRRRWZ1hRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAU7wJ/wAiy/8A2EtQ/wDSyam07wJ/yLL/APYS1D/0smqoHPiNkdLRRRWhyhRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRVHVdZ07RLYXGo3ccCMdqA8tI391VHLH2AJrnZdc17V+NNtF0m1P8Ay83y752HqsQOF+rnPqtNJvYTaW51lxcQ2sDz3E0cMKDLSSMFVR7k9K52Txxp0p26RbXmrt2azi/df9/XKofwY/Ss1PDllJOtzqLTardKcrNfv5m0+qpgIn/AVFa9aKl3M3U7FB9T8VX33I9M0qM/3t13Jj/xxVP/AH0KgfRri751LXdWu89UW4+zJ9MQhMj6k1rUVahFEObZlW3hnQ7SXzodJsxN3maENIfqxyT+daoGBgUUVRIUUUUwCiiigAooooAKKKKACiiigAqOe3guYzHcQxyoeqyKGH5GpKKAMb/hFdGjYvaWhsHPO6wle1Of+2ZXP41Mlrrtl/x4+IpnUdItQgSdfzXY/wCbGtOipcUylJoqJ4k1yz41DQkuox1l024Bb6mOTbj6BmNaOn+LdE1G4W1jvBBeN0tbtGgmP0RwC31GRUNV7yxtNQt2t721huYW6xzIHU/gah010KVR9TqaK4eLS7/Svm0PVZrdB0tLvNxbn2AY70/4CwA9DV+38YraOsPiKzOmMTgXQfzLVz/10wCn/AwvsTWbg0aKSZ1NFIrK6hlIKkZBB4IpakoKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACufvP+Sh6N/2Cr//ANG2ldBXP3n/ACUPRv8AsFX/AP6NtKAOgooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigArgvD3/Hle/9hXUf/Syau9rgvD3/AB5Xv/YV1H/0smqZ7G9D4jWooorM6wooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKd4E/5Fl/+wlqH/pZNTad4E/5Fl/+wlqH/pZNVQOfEbI6WiiitDlCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAK5jVvE08l7LpWgRxz3sZ23F1LkwWh9Gxy7/wCwD9SvGU8S6pdTXkfh/Spmhupo/NurpOtrASQCv+2xBC+mGbsAUsbG202zjtLSJYoIxhVH6knqSTySeSa0hC+rInO2iKljosNrcm+uJZL7UnGHvLkgvj+6o6Iv+yoA+p5rToorZKxje4UUUUxBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUjKrqVZQykYIIyCKWigDHisL7w+xm8POv2fOZNLmbELf9cj/wAsm+nynuB1rp9F1y01y2eS33xzRNsuLaYbZYH/ALrD+RGQRyCRVCsvUrC4W5TVtKKx6rbrhdxwlwnUxSf7J7Hqp5HcHKUL6o0jO2jO1oqjo+q2+taXDf24ZUkBDRuMNG4OGRh2ZWBB9xV6sTYKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAoqK4uYLS3kuLmaOGCNdzySMFVR6kngCuSbx82pMU8L6JeayoOPtZIt7X8JH5b/AICppNpbgdlRXF+b8QrnnzfDdip52CKe4YfjuQfpTfs3xA/6GDQ//BW//wAdqPaw7lcjO2orifs3xA/6GDQ//BW//wAdo+zfED/oYND/APBW/wD8do9tDuHIztqK4n7N8QP+hg0P/wAFb/8Ax2j7N8QP+hg0P/wVv/8AHaPbQ7hyM7aiuJ+zfED/AKGDQ/8AwVv/APHaPs3xA/6GDQ//AAVv/wDHaPbQ7hyM7aiuJ+zfED/oYND/APBW/wD8do+zfED/AKGDQ/8AwVv/APHaPbQ7hyM7aiuJ+zfED/oYND/8Fb//AB2j7N8QP+hg0P8A8Fb/APx2j20O4cjO2orifs3xA/6GDQ//AAVv/wDHaPs3xA/6GDQ//BW//wAdo9tDuHIztqK4n7N8QP8AoYND/wDBW/8A8do+zfED/oYND/8ABW//AMdo9tDuHIztqK4n7N8QP+hg0P8A8Fb/APx2j7N8QP8AoYND/wDBW/8A8do9tDuHIztqK4n7N8QP+hg0P/wVv/8AHaPs3xA/6GDQ/wDwVv8A/HaPbQ7hyM7aiuJ+zfED/oYND/8ABW//AMdo+zfED/oYND/8Fb//AB2j20O4cjO2orifs3xA/wChg0P/AMFb/wDx2j7N8QP+hg0P/wAFb/8Ax2j20O4cjO2orihB8QEOf7c0GT/ZbTJAP0lpf7a8b6b819oOm6pF3bS7oxSAf9c5Rg/TfQqsH1DlZ2lFYGieMtH125ayhlltdRQZewvYzDOo9djfeHuuR71v1oSFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABXP3n/JQ9G/7BV//AOjbSugrn7z/AJKHo3/YKv8A/wBG2lAHQUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAVwXh7/jyvf+wrqP/pZNXe1xsfhPXrOS6Wx13TUt5bqe5RJ9LeR182VpCCwnUHBcjOB0qZK6NaUlGV2W6Kg/4R3xR/0H9H/8E8v/AMk0f8I74o/6D+j/APgnl/8Akmp5Wb+3gT0VB/wjvij/AKD+j/8Agnl/+SaP+Ed8Uf8AQf0f/wAE8v8A8k0crD28CeioP+Ed8Uf9B/R//BPL/wDJNH/CO+KP+g/o/wD4J5f/AJJo5WHt4E9FQf8ACO+KP+g/o/8A4J5f/kmj/hHfFH/Qf0f/AME8v/yTRysPbwJ6Kg/4R3xR/wBB/R//AATy/wDyTR/wjvij/oP6P/4J5f8A5Jo5WHt4E9FQf8I74o/6D+j/APgnl/8Akmj/AIR3xR/0H9H/APBPL/8AJNHKw9vAnoqD/hHfFH/Qf0f/AME8v/yTR/wjvij/AKD+j/8Agnl/+SaOVh7eBPRUH/CO+KP+g/o//gnl/wDkmj/hHfFH/Qf0f/wTy/8AyTRysPbwJ6Kg/wCEd8Uf9B/R/wDwTy//ACTR/wAI74o/6D+j/wDgnl/+SaOVh7eBPRUH/CO+KP8AoP6P/wCCeX/5Jo/4R3xR/wBB/R//AATy/wDyTRysPbwJ6Kg/4R3xR/0H9H/8E8v/AMk0f8I74o/6D+j/APgnl/8AkmjlYe3gT0VB/wAI74o/6D+j/wDgnl/+SaP+Ed8Uf9B/R/8AwTy//JNHKw9vAnoqD/hHfFH/AEH9H/8ABPL/APJNH/CO+KP+g/o//gnl/wDkmjlYe3gT0VB/wjvij/oP6P8A+CeX/wCSaP8AhHfFH/Qf0f8A8E8v/wAk0crD28CeneBP+RZf/sJah/6WTVW/4R3xR/0H9H/8E8v/AMk1s+HNIk0PRUsZrlLmXzp55JUiMalpZXlOFLNgAvjqelVFNGVWopJWNWiiiqMAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKAOE8NN9ts7jWX5l1Sd7nJ/5552xL9BGqfjk962q5n4e3sd/8PtCljOQtmkR/wB5BsP6qa6aupbHM9wooopiCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigCnoDfYPF+o2K8QX8C3yL6SKRHKfxBiP1ye9dbXC2d7HJ8V7SxU/vIdEuZWHs88AH/oBruq5p/EdEPhCiiipKCiiigAooooAKKKKACiqGo65pGkDOp6pZWQIyPtNwsef++iKx2+I3gtTg+KdJ/C6Q/1oA6eiuX/4WR4K/wCho0r/AMCV/wAaP+FkeCv+ho0r/wACV/xoA6iiuX/4WR4K/wCho0r/AMCV/wAaP+FkeCv+ho0r/wACV/xoA6iiuX/4WR4K/wCho0r/AMCV/wAaP+FkeCv+ho0r/wACV/xoA6iiuX/4WR4K/wCho0r/AMCV/wAaP+FkeCv+ho0r/wACV/xoA6iiuX/4WR4K/wCho0r/AMCV/wAaP+FkeCv+ho0r/wACV/xoA6iiuX/4WR4K/wCho0r/AMCV/wAaP+FkeCv+ho0r/wACV/xoA6iiuX/4WR4K/wCho0r/AMCV/wAaP+FkeCv+ho0r/wACV/xoA6iiuX/4WR4K/wCho0r/AMCV/wAaP+FkeCv+ho0r/wACV/xoA6iiuX/4WR4K/wCho0r/AMCV/wAaP+FkeCv+ho0r/wACV/xoA6iiuX/4WR4K/wCho0r/AMCV/wAaP+FkeCv+ho0r/wACV/xoA6iiuX/4WR4K/wCho0r/AMCV/wAaP+FkeCv+ho0r/wACV/xoA6iiuX/4WR4K/wCho0r/AMCV/wAaP+FkeCv+ho0r/wACV/xoA6iiuX/4WR4K/wCho0r/AMCV/wAaP+FkeCv+ho0r/wACV/xoA6iiuX/4WR4K/wCho0r/AMCV/wAaP+FkeCv+ho0r/wACV/xoA6iiuX/4WR4K/wCho0r/AMCV/wAaP+FkeCv+ho0r/wACV/xoA6iiuX/4WR4K/wCho0r/AMCV/wAacvxF8Fu2B4p0jPvdoP5mgDpqKp2GrabqsZk07ULW8QdWt5lkH5qTVygAooooAKKKKACiiigArM1/XrLw5pUmoXzNsBCRxRjdJNIeFRF/iYnoP6Vp1wWjf8Vf4jl8T3Hz6bZO9to8Z+6cHbJce5Ygqvoo96mc1FXGld2FtvDt74luI9U8YKrqG32ujK263tvQydpZPc/KOcDvXXKoVQqgAAYAHalorglJyd2bpJBRRRUjCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKAMvW/D2meIbZYdRtg7RndDMhKSwt2ZHHKn6VkWGual4V1CDSPEtwbvT7hxFY6wQAdx6RT44Ddg/Ru+DXV1W1HT7TVtPnsL6BJ7WdCkkbjhgf89a0p1HB+RMopmtRXH+DdQu7O7u/CeqzvNeaciyWtxJ965tCcIx9WUjY3uAe9dhXcmmrowCiiimAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABXP3n/JQ9G/7BV//wCjbSugrn7z/koejf8AYKv/AP0baUAdBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQB5D4Mf/AIR/RNCMny6ZqljakP2hufKQYPorgDH+0D3YV3tc74WtYL74daHa3USSwS6Vbo8bjIYGJeKat3d+GP3OpPLdaQOI785aS3H92buVH/PT/vrH3j0rRHO9TpKKbHLHNEksTrJG4DK6nIYHoQadVEhRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABVXUdQt9KsJby6crFGOgGWYngKo7sTgAdyai1TWLTSY0NwzNNKdsNvEu6WZvRV6n69B1JAqjZaZd399HqmtKqyRHNrZK25LbPG5j0aTBxnoOQO5KGVvDFlcQePLO+v1C39/p17NMgOfKHmWgSMH/ZUAH1JY969Hrj7f/koel/9gq+/9G2ldhXPP4jeHwhRRRUlBRRRQAUUUUAUdX1iw0LTJtR1K4WC1hGWY8knsAOpJPAA5NciI/EvjD99eXFz4e0d+Y7O3YLeTL2MsnPl5/urzzyabp6/8Jp4lk124+fR9MmaDSoT92WVTtkuCO+DlU9ACeprsa5qtVp8sTSMOrMHTvBXhrSyWttFs/NJy00sfmysfUu+WP51rCwswMC0gx/1zFWKK5m29zWxX+w2n/PrB/37FH2G0/59YP8Av2KsUUAV/sNp/wA+sH/fsUfYbT/n1g/79irFFAFf7Daf8+sH/fsUfYbT/n1g/wC/YqxRQBX+w2n/AD6wf9+xR9htP+fWD/v2KsUUAV/sNp/z6wf9+xR9htP+fWD/AL9irFFAFf7Daf8APrB/37FH2G0/59YP+/YqxRQBX+w2n/PrB/37FH2G0/59YP8Av2KsUUAV/sNp/wA+sH/fsUfYbT/n1g/79irFFAFf7Daf8+sH/fsUfYbT/n1g/wC/YqxRQBX+w2n/AD6wf9+xR9htP+fWD/v2KsUUAV/sNp/z6wf9+xR9htP+fWD/AL9irFFAFf7Daf8APrB/37FH2G0/59YP+/YqxRQBX+w2n/PrB/37FH2G0/59YP8Av2KsUUAV/sNp/wA+sH/fsUfYbT/n1g/79irFFAFf7Daf8+sH/fsUfYbT/n1g/wC/YqxRQBX+w2n/AD6wf9+xQbCzYEG0gIPYxj/CrFFAHO3/AIF8NahIJm0mC3uRytzZj7PMp9Q6YNUmu/EPgv8Ae3U8/iDQF/1kjKDe2i/3jjAmUd+Aw98V19FXGpKJLimS2N9a6lYw3tlPHPbToHjljOVZT3FWK4O0X/hC/F8NrF8mga5KVjj/AIbS8wThfRZADx2Yds13ldsZKSujFqzsFFFFUIKKKKAOY+IGoT2Hg28Szbbe3pSxtiOokmYRgj3G4n8K0NNsINK0y10+1Xbb20SxRj/ZUYH8qxfHX73UfCNsfuya0jkeuyGVh+oB/CukrlxD1SNaaCiiiuY0CiiigAooooAKK5Tx747sPAeireXMZnuZmKW1srYMjDqSeyjjJ9x6151L8RPitb6cddm8JWi6SF8wgxNuEfXcRv3DjnJXHerjBtXJckj3CiuG0r4i2/iD4b6n4n02BUurG2meS1mO4JKiFgCRjKnjnjj0NWfhp4tvfGnhIatfwW8M/wBoeLbAGC4XGOpJ70nBpXY7o7CiiipGFFFFABRRRQAUV5l8UfiJrPgrVNJstJsrO5a+VuLhWJ3BgABhh61hX/xL+Jfh+H7drngu2SwjI814g4wPdg7AfUitFTbVyXJHtVFY3hXxNYeLvD9vq+nlhFLkNG/3o3HVT7j+WDWzUNW0ZQUVj+KfEdp4T8O3WtXscskFuFykQBZizBQBn3Io8La4fEnhiw1kweR9rj8zyt27aMkYzgZ6UWdriv0NiiiikMKKKKACiiigAooooA5Pxf8A8SvU9A8SJ8psr1ba5bsbeciNs+wYxt+FdzXE/EiPzPhzrxHDR2jSqfQp8wP5gV2cMnmwRyf31DfmK7KDvGxjUWo+iiityAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACuT17SdN1nx1olvqmn2l9AumXzrHdQrKobzbQZAYEZwSM+5rrK5+8/5KHo3/AGCr/wD9G2lAB/wgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E0f8IJ4P/6FTQ//AAXQ/wDxNdBRQBz/APwgng//AKFTQ/8AwXQ//E1B4RsbTTb/AMUWdhawWtrFqqeXBBGERM2dsThRwMkk/Umunrn/AA9/yHPFn/YVj/8ASK1oA6CiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKAOC8Gf8iL4e/wCwZbf+ilrcrD8Gf8iL4e/7Blt/6KWtyupbHM9zCk8PyWMrXGgXIsWYlntXXfbSE9TsyChPqpHqQaRfEosiI9etH0xunnsfMtm+koGF/wCBhTW9SEBgQQCDwQaLBcSORJY1kjdXRhlWU5BHsadWJJ4WsY5Gm0yS40qZjktYuEQn1MZBjJ9yufem58TWPVbHVYh3BNtLj6fMrH8UFAG7RWH/AMJTawcalZahpp7m4ty0Y/7aR7kH4tWlZanYanH5lhe210n96CVXH6Gi4WLVFFFMQUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUVlXfiXRbKXyZtSt/P/wCeEbeZKfoi5Y/lSA1aKwv7dvrvjTNBvJAekt4Rax/iGzJ/45R/ZuuX/wDx/wCrraRHrBpse0/Qyvkn6qFNFx2NDUdX0/SY1e+u4oN5wiscs59FUcsfYA1mm+1rV/l0+0/s21P/AC93yZlI/wBiHt9XIx/dNXtP0LTdLkaW1tVFw4w9xITJK/8AvSMSx/E1o0AZum6JaaZI86mS4vJRiW7uG3yuPTPYf7KgAelaVFFAihb/APJQ9L/7BV9/6NtK7CuPt/8Akoel/wDYKvv/AEbaV2FYT+I6IfCFFFFQUFFFFABXPeOtUm0fwRq95bEi5EBjgI7SOQiH/vphXQ1yHxK58KQofuvqdirfT7THSYGpoulw6JoljpduAIrSBIV99oxn6nrV6iivOOkKKKKQBRRXnHxC+Ka+FNRh0PSLD+09cmC4h5Kx7vugheWY9lGOOc9M1GLk7ITdj0eivEbr4lfE3w3Cmo+IvCNqNMJG9olZSoPqwdtv/AhXq/hnxJp/ivQbfV9NcmCYEFW4aNh1Vh6j/wCvTlBrUE0zXoooqBhRXnc3j/Uo/jRb+CxbWn2CSMuZirebnyWk67sdR6dK9EqnFrcSdwoooqRhRRRQAUUV534p8f6loXxP0Hwxb21o9nqKxNLJIrGRd0jKdpDAdFHUGqUW9hN2PRKKK8c8S/FDxdZ/Ea+8K+H9GsL54SvlK6v5jDy1c5O8DjJojFy2Bux7HRXid58WPHXhl7e48VeD4YNPkk2F4dynPXAJZhnAJwcZxXsljeQajp9tfWzb7e5iWaJsYyrAEH8jTlBx3BNMsUUVieMPEI8K+E9R1sw+cbWMFY843MWCqCfTJFSld2GbdFeWeDPG/jTVNHuvEmu6Zp0fh9LKe6je3yJGMeflALnAOG6jtUfw+8ceOfGuox376VpcPh0zvFLIpPmJhcjGX5OSoztxz+Vum1fyJ5ker0UUVmUFFFch4b1rxXf+K9YstZ0WKz0m3dxZXKowMwD4UkliOV56CmlcR19FFFIYUUUUAFFFFABRRRQBzvjqwfUPBeprCdtzBF9qt2HVZYv3iEfioro9Jv01XRrHUYxhLu3jnUezKG/rVa/UNp9yp6GJgfyNZ3w7Yt8N/DZP/QOgH5IBXXh3o0ZVDpqKKK6DMKKKKAOO8af8jB4N/wCwq/8A6TTV0lc340/5GDwb/wBhV/8A0mmrpK5MR8RrT2Ciiiuc0CiiigAooooA8G+MWLj4t+ELS6GbI+RuDfd+ach/0Ar3eVY3idJQpjZSGDdCO+a4L4pfD1/HGl201hMsGrWJLW7ucK4OMqSOnIBB7H61xd3qPxo1DSH8PyaFAjyR+TJfqVDsp4J379gJHcD6c1tZTirPYjZs9F8SWek2nwx8QjRrezhtH0y4K/Y0VUb903Py8GvKvDHia68K/s9XV9YNsvJL97eGTGdhbGW+oAOPfFdxoPw7uPC/wm1vRIit1q+oWk5k2HCtI0ZVUUnHA4GTjkk8Vj+H/hrql78F7vwxqsIsdSa6a4g3urBWGCpJUng4I9s04uKVm+oO55qw8M2/hNNctvG2pf8ACZ7VnZP3mC5OSm7bnIHfdgkehr1a41i98Z/BWw1GbxJBoEkw2Xl24x5hRmQqCCNu4qDxn0xWDZj4pWPh238MWvhW1iurcLFHqhMTARqeOuVPAxn07ZrY+JfgTxNr/gTRreCWLUNTsHMl1HGqxLMzDkqOB8vQdMgnvxVSabV2SjyjxE/hnw4bK+8D+IdYudSilAuLl1KRk4zkEqp6jocgj9fQfjfqWoLJ4KuLK4eC4m81xsYgb/3JGR3GT3qr4v0P4geOPDFlaJ4RtdKtbGQMtpHOgeVtpG5QSAqgZ4PPPet34leEfEPiSPwVJp+lvI1kpN2nmoDCT5XBy3P3W6Z6U+ZXV/MLOzOV+J3gubwFZ6b4msPEGqz6rJdCOe4nlyWcqW3LgZA+U8Eng19AaZcteaVZ3TgB5oEkYD1Kg/1rgfjR4Z1jxT4UsrPRbI3dxHfLK6B1XC7HGcsQOpFd3pEEltothBMu2WK3jR1znBCgEVlKV4q+5aVmeL/HP/kdPB/++f8A0Yley67FDP4e1KK4AML2sqyBum0qc15h8Y/CXiTxBrmg32gaZ9s+xK7PmRFAbcpAO5hnp2qlqsvxo8T2E2kTaJp2m290pjmmikVcIeCMmRjgjjgZqrc0Y6ivZsp/A1ryXwF4ngt9QTTyJMx3ci7lgYx/M+CQOAAetcbrmneDrHRrmaPxfqWseKkJcTWwYw53d2IzjHcMea9eHwwm034PX/hTTblG1K6UTSzfdWWXcrFfZSFC/qe9cdp2ifEB/h5c+DIPCNrYjy3E19JMimcZLBQAeWP3d2SPp1q1JNtp9SWtLFfxGlz4n/Z50rX9Rv7l7ywLJ97InBnEYL9yQqjn1rp/h14XbTPhXc6jZeIJrG41WzVmuLkho7PaWyyjIxwx5zxwe1Ja+CtfuPgBN4WmsDb6vGzMkLyId+J/NGGBI5GRyetZ+neHvG+u/CfUPCV9pC6c1pFELJ3kANyVkLMrfMccAYPAz+ibTVr9R21ucBr1h4P07RJbiw8V6nqvimMhmntw3k53DPzFc4x3DHmvoT4band6x8O9Fvr6VprmSEh5GOS21iuSe5wBzXlkWi/EC/8AhxJ4Og8I2unxxx/v7ppkVrna24Kqg/eJA+YnH0r1H4aabqGjfD/S9N1S0a1vLYOkkbMrfxsQcqSOQRSqNOIR3OsooornNAooooAKKKKAOa+If/JOvEX/AGD5v/QTXV2X/Hhb/wDXJf5CuU+If/JOvEX/AGD5v/QTXV2X/Hhb/wDXJf5CuvD7Myqbk9FFFdBmFFFFABRRXPXmu3d5dzWOgQwyyQtsnvbjJghbuoAwZHHcAgDuwPFJtJXY0rnQ0Vy/9gT3PzajrurXLnnENwbVB7ARbTj6kn3pP+ET0/8A5/Nc/wDB7e//AB2sfrESvZs6miuV/wCET0//AJ/Nc/8AB7e//HaP+ET0/wD5/Nc/8Ht7/wDHaPrEQ9mzqqK5X/hE9P8A+fzXP/B7e/8Ax2j/AIRPT/8An81z/wAHt7/8do+sRD2bOqorlf8AhE9P/wCfzXP/AAe3v/x2j/hE9P8A+fzXP/B7e/8Ax2j6xEPZs6qiuV/4RPT/APn81z/we3v/AMdo/wCET0//AJ/Nc/8AB7e//HaPrEQ9mzqqK5X/AIRPT/8An81z/wAHt7/8do/4RPT/APn81z/we3v/AMdo+sRD2bOqrzbWL/xXH8a9Gsra2sW017KbZO0LkrCzRGYMd4G8NGgBxgeYMg543/8AhE9P/wCfzXP/AAe3v/x2oz4L0lrhLhptZM6IyJIdbvNyqxBYA+bkAlVJHfaPSj6xEPZs66iuV/4RPT/+fzXP/B7e/wDx2j/hE9P/AOfzXP8Awe3v/wAdo+sRD2bOqorlf+ET0/8A5/Nc/wDB7e//AB2j/hE9P/5/Nc/8Ht7/APHaPrEQ9mzqqK5X/hE9P/5/Nc/8Ht7/APHaP+ET0/8A5/Nc/wDB7e//AB2j6xEPZs6qiuV/4RPT/wDn81z/AMHt7/8AHaP+ET0//n81z/we3v8A8do+sRD2bOqorlf+ET0//n81z/we3v8A8do/4RPT/wDn81z/AMHt7/8AHaPrEQ9mzqqK5X/hE9P/AOfzXP8Awe3v/wAdo/4RPT/+fzXP/B7e/wDx2j6xEPZs6qiuZXw6kPNpq2s27joxv5J8fhMXB/EUf2tqWgsDrTx3em5wdQjTY8HvMg42+rrgDuoGTVRrRk7CcGjpqKAQQCDkHvRWpIUUUUAFFFFABRRRQAUUVh+JbudYLbTLKVorzUZPJWVesMYGZJB7hRgf7TLSbsrsCG61m91K7lsdC8tEhYx3GoSruRGHVI1/jcdznap9SCtQ/wDCLadP82pm41WQ/ea+lMin6R8Rr+CitSztLews4bS1iWKCFAkaL0AFT1xTqykzdRSME+CPCbEk+F9FJPc6fF/8TR/wg/hL/oVtE/8ABfF/8TW9RWfM+47Iwf8AhB/CX/QraJ/4L4v/AImj/hB/CX/QraJ/4L4v/ia3qKOZ9wsjB/4Qfwl/0K2if+C+L/4mj/hB/CX/AEK2if8Agvi/+Jreoo5n3CyMH/hB/CX/AEK2if8Agvi/+Jo/4Qfwl/0K2if+C+L/AOJreoo5n3CyMH/hB/CX/QraJ/4L4v8A4mj/AIQfwl/0K2if+C+L/wCJreoo5n3CyMH/AIQfwl/0K2if+C+L/wCJo/4Qfwl/0K2if+C+L/4mt6ijmfcLIwf+EH8Jf9Cton/gvi/+Jo/4Qfwl/wBCton/AIL4v/ia3qKOZ9wsjB/4Qfwl/wBCton/AIL4v/iaP+EH8Jf9Cton/gvi/wDia3qKOZ9wsjB/4Qfwl/0K2if+C+L/AOJo/wCEH8Jf9Cton/gvi/8Aia3qKOZ9wsjB/wCEH8Jf9Cton/gvi/8AiaP+EH8Jf9Cton/gvi/+Jreoo5n3CyMH/hB/CX/QraJ/4L4v/iaP+EH8Jf8AQraJ/wCC+L/4mt6ijmfcLIwf+EH8Jf8AQraJ/wCC+L/4mj/hB/CX/QraJ/4L4v8A4mt6ijmfcLIwf+EH8Jf9Cton/gvi/wDiaP8AhB/CX/QraJ/4L4v/AImt6ijmfcLIwf8AhB/CX/QraJ/4L4v/AImj/hB/CX/QraJ/4L4v/ia3qKOZ9wsjB/4Qfwl/0K2if+C+L/4mj/hB/CX/AEK2if8Agvi/+Jreoo5n3CyMH/hB/CX/AEK2if8Agvi/+JpV8F+F0/1Xh3Soj/eitERh9CoBFbtFPmfcLIxP7Iv9M/eaJqUwA/5c76Rp4X9gzZdPbBIH901qaPrcWqebBJC9rf2+BcWkpBZM9GBHDIcHDDg4I4IIE9ZOt2E8ixalpwA1Syy0HOBKv8ULH+6wGPY7W7VrTrNOzJlC+x0dFVtOv4NU022v7Ykw3EayJkYOCM4I7H1FWa7DEKKKKACiiigAooooAKKpapqtro9p9ouix3MEiijXdJK56Ii9yf8AEnABNYhj8Qat891e/wBj25+7bWYV5sf7crAqD7IvH949aiU4x3Gk3sdRRXLHwtZvzLf627eo1m6T9FkA/Sk/4RPT/wDn81z/AMHt7/8AHaz+sRK9mzqqK5X/AIRPT/8An81z/wAHt7/8do/4RPT/APn81z/we3v/AMdo+sRD2bOqorlf+ET0/wD5/Nc/8Ht7/wDHaP8AhE9P/wCfzXP/AAe3v/x2j6xEPZs6qiuV/wCET0//AJ/Nc/8AB7e//HaP+ET0/wD5/Nc/8Ht7/wDHaPrEQ9mzqqK5X/hE9P8A+fzXP/B7e/8Ax2j/AIRPT/8An81z/wAHt7/8do+sRD2bOqorlf8AhE9P/wCfzXP/AAe3v/x2j/hE9P8A+fzXP/B7e/8Ax2j6xEPZs6qiuV/4RPT/APn81z/we3v/AMdo/wCET0//AJ/Nc/8AB7e//HaPrEQ9mzqq5/w9/wAhzxZ/2FY//SK1qr/wien/APP5rn/g9vf/AI7UcfgvSYXmeKbWUeZ98rLrd4C7bQuW/e8naqjJ7ADtR9YiHs2ddRXK/wDCJ6f/AM/muf8Ag9vf/jtH/CJ6f/z+a5/4Pb3/AOO0fWIh7NnVUVyv/CJ6f/z+a5/4Pb3/AOO0f8Inp/8Az+a5/wCD29/+O0fWIh7NnVUVyv8Awien/wDP5rn/AIPb3/47R/wien/8/muf+D29/wDjtH1iIezZ1VFcr/wien/8/muf+D29/wDjtH/CJ6f/AM/muf8Ag9vf/jtH1iIezZ1VFcr/AMInp/8Az+a5/wCD29/+O0f8Inp//P5rn/g9vf8A47R9YiHs2dVRXK/8Inp//P5rn/g9vf8A47R/wien/wDP5rn/AIPb3/47R9YiHs2dVRXK/wDCJ6f/AM/muf8Ag9vf/jtH/CJ6f/z+a5/4Pb3/AOO0fWIh7NnVUVzK+HUh5tNW1m3cdGN/JPj8Ji4P4ij+1tS0FgdaeO707ODqEabHg95kHG31dcAd1AyaqNaMnYTg0dNRQCCAQcg96K1JOC8Gf8iL4e/7Blt/6KWtysPwZ/yIvh7/ALBlt/6KWtyupbHM9wooopiCiiigArNvfD+j6lJ5t5plpNL2laIbx9G6j860qKQGIPDMMP8Ax5alqtp6bLxpQPosu9R+VH9na/Cf3GvwzD/p8sQ5P4xsn8q26KLDuYhk8UQ/8uukXQ9RcSQH8tj/AM6P7U1qMfvfDsj/APXveRt/6GUrboosBiDX7kH974c1iP8ACF//AECQ0f8ACTRD7+l6wn/bhI3/AKCDW3RQBijxRYkEm11dceuk3R/lHTf+Er089LXWSfT+xrsfzjrcoo1DQw/+EptD92w1lv8AuF3A/mgpP+Emz/q9E1mT/t02/wDoRFbtFGoaGGdd1Fx+58L6q3vJJboP1lz+lA1DxHL/AKrQLWP/AK+dR2/+gRtW5RRYDC2eKpv+XjRrQe0MtwR/48lB0XVZ/wDj68S3gHdLSCKIH8SrN+TVu0UWC5h/8IjpEn/H4lzf+ovrqSdT/wAAZiv5CtW0sbSwi8qztYLeP+5DGEH5Cp6KLBcKKKKYgooooAKKKKAKFv8A8lD0v/sFX3/o20rsK4+3/wCSh6X/ANgq+/8ARtpXYVzz+I6IfCFFFFQUFFFFABXIfEn/AJFi2/7Clj/6Ux119ch8Sf8AkWLb/sKWP/pTHSewI6KiiivNOkKKKKACvA/BapeftIeIJb0BpoWuTBu6ghlRcf8AACa98rx7x/8AD3xBB4xj8beC2U6gCGntsgFmA2llzwQV4K/lnPGtNrVPqTLueoa9Bb3Ph3UoLsKbeS1lWQN027Tmvn34c3V1F8HPHQjkdEiUPGysQVYrgkH1wFrd1fV/i34y019CHhlNMiuF8u5nCmPcp6jc7HCkdcZPaurj+Hsnhv4Oar4esF+2aldQO8pQY82VgBhc9gAAPpnvVK0VZvcT1Zxnwx8DX3i/SdL8Rar4hvvLsbrFtbBiQUV9zAtnPzMWH0H4CLSNPu/jD8QtcbVdWvbbStNcrBBbSbSo3FUxkEA4UknBJP6emfCXRtQ0H4fWdhqlq9tdpLKzRPjIBckdPauEXQPGPw28e6pqXh7RP7Z0nVGLeWjYK5YsAe6lSSM4IIPr0rmvJ2+QraIydB0m80P9pGw0281CbUDAjrFcTnLtH9mcqGPcgHGfaqeqeINO8b/ELVofFviW60rQ7J3itILckbyrbQeFYZOCxJB64rpNA8L+NJfjXYeKNe0xY45Ud5mhcNHbgwOix5zyR8oOO5pl34U8R+BfHuqavpXhqHxDpGpMz+ScFoizbscgkEEkZwQQfXo7q/nYVh3wj8TyW3jnUfCcGsy6vovltLYXEucrjBwM8gYJyOmV461y3grw3qvxB1rxBplx4gvbXTrabzXVWL7n3MEGCegG4/549U+Hdn4tn1S+1bxFp1jplm+RaWUdtGsqZPdlGcAcc8kntWZ8HvC+teH9a8UTatp8trHdyxmBnI+cBpCcYP8AtD86lySu1voO2xyHjnVk/wCE003wHfa/c6Z4d0u2iiu7lSxeZvKDZbGSScqADkAkmm+FvENl4S+J2maT4a8R3Or+HdSKQyRXBJMUjEqOoAyDtOQBkHFdR4/8Ga7ZfEG28b+H9Lh1cFQt3YyAEkhdmcHqCuOmSCM4q14Wi8Z654yiv7zw7Y+H9CgAJt5LaJpHYZxtbaGyTjngADjmndcvyCzuc5qMV98UvjFqXh661O6tNG0pX/dW74J2FUJHbcWbqQcDisrVfD1z4Y+OHhXTJdTutQtke3a0e6bdIkRkb5Ce+GDfga6bXvDXivwZ8Trnxf4Y0v8AtW01AMJ7dT8wLYLKR15ZQwIz6H3oz+HfHWvfFLw94n1jRRBCssRMUMgYWsSuThjnk8luPXt0DT89LCaPeK+bvEE/iC3/AGiNSk8MWsN1qox5UUxAUj7Ou7qyjpnvX0jXg3iPQ/G+lfGXUPFWgeHzfR/KIWkxscGFUPAYHjn8qypPV+hUzF+Id38Rb3SrRfG2lC00OO5R5WsBGxzyBkh2wcEgZwMmus+KWq2j/BfRLzw7cTQ2LTwR25RijKixuu047jbg+4rN8Rt8W/HWktoV14ZtLK0mdTK6EJkA5GS0h4BAPAzxXS+KPhrev8GrLwvpjrcX2nss4GdomfLFwCenLtjPoK0ula4rPUn8W3VxH+z4tyk8qz/2ZZt5quQ2SYsnPWuI1rSZNY/Z20nWrjUbsS6fHITHvys++5C/PnrjHFWruL4jeI/ht/wiz+FhZw2lvHFJPLJh7hYyuxUQ9ztXJzjg9OldA/hfWz+zt/wjw0+X+1vLA+y5G7P2nf64+7zSXu29Q3+45LRPh4l58GJ9dOualGzWc139mR/3Y8oSjZj+6e9HwssRoPgPVPHa31y0tklxGliW/cs21dpI9ckV6LoGhana/AyTRJ7N01M6ZdQi3JG7e3mbR6c5H51nfDnwXfp8J9S8Oa5ayWU15JOuGwSoZVAbg9iM/hQ56O76hbY8r0t9C8R6Tea34n8d31t4jd3a2QFtsePu5wp4J7KRgYrsdJ8Zan4h+AXiCS9upX1DT8Q/aQxDuuUKkn15Iz3xVTRrDxz4K0i78PDwPbarJvc2mohVdU3d+QdwzyA2Md67K58OeJ5/g3qun6lBbT67doCILOGOPA3LhTtAUtgEk9OcdqqTX4iSZy/wx8DX3ifTdF8V6r4gvWFrcH7Na5JUxxucgnOeWBz7Ve+F97dT/GDxtDNczSRRzXGxHkJVf9II4B6V3Hws0m+0P4c6Xp2pWzW13EZvMifGVzK5HT2INcv8O/C+taR8UfF2p3+nywWV5LM1vMxGJAZiwxg+nNQ5X5ikrWOJ+Huhah461jxHpt74g1O30qCYPNFbzYeRizhBls4UYbjHPHpW78JJb/Qvib4h8Htfz3OnWscjRrK2cFJFAIHYlWOce3pWz8HfDGteHtY8UTatp8trHdyxtAzkfOA0hOMH/aH50eEvDGtWHxv8Ra3dafLFptzFKsNwSNrkvGRjnPQH8qqUk+ZdBJbHI+FdKuvjL4p1rVNZ1i/t7GydRbQWsgUpuLbQuQQMBeTjJJq54OXWNM+P02i6jq91frawOivK5+dPKBQsOmdpGT60aVofjHwD4u1VvBlpY63pV8/CCdD5eCdob5gVK7iM9CP0reCYdWj/AGh7oa3LFLqXkvJcGH7ilolO0eyghfwpvW9trC7H0JRRRXKahRRRQBBe/wDHjcf9cm/lWZ8Of+SbeG/+wdD/AOgitO9/48bj/rk38qzPhz/yTbw3/wBg6H/0EV1YfqZVOh09FFFdJmFFFFAHHeNP+Rg8G/8AYVf/ANJpq6Sub8af8jB4N/7Cr/8ApNNXSVyYj4jWnsFFFFc5oFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQBzXxD/wCSdeIv+wfN/wCgmursv+PC3/65L/IVynxD/wCSdeIv+wfN/wCgmursv+PC3/65L/IV14fZmVTcnoooroMwooooAxPE15cRWVvYWUpivNRmFtHKvWJcFnce4RWx/tbansrO30+yhs7WIRQQqFRB2H9frVHVPm8baMh5VbC8kA/2g9uoP5M351q1x15e9Y1prS4UUUVgaBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABSMquhR1DKwwQRkEUtFAGT4ZdrC5vvD7sWSy2S2hPX7NJu2r/wBldf90L610dc3jZ4+08g/63S7rcPXbLb4/wDQ2/Oukr0KbvFMwkrMKKKKskKKKKACiiigArnJf3/jyXd/y56ZHs9vOlfd/wCiFro65xP+R81X/sGWX/o26rKt8DKhua1FFFcJuFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQBmeFf3I1ixH3LTUpFQegkVJ8fnMa6Cuf8N/8AIU8Tf9hNP/SS2roK9GHwo53uFFFFUIKKKKACiiigDl9OH9sazda1N80UMklpYKeiIrbZHHuzqRn+6q46nO3WJ4O58E6G5+9JYQyMfVmQMT+ZNbdefOV5NnRFWQUUUVAwooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACkZVdCjqGVhggjIIpaKAMnwy7WFzfeH3YsllsltCev2aTdtX/gDK6/7oX1ro65vGzx7p5B4l0u63D12y2+P/Q2/Oukr0KcuaKZhJWZwXgz/kRfD3/YMtv/AEUtblYfgz/kRfD3/YMtv/RS1uV2rY5HuFFFFMQUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAULf/AJKHpf8A2Cr7/wBG2ldhXH2//JQ9L/7BV9/6NtK7CuefxHRD4QoooqCgooooAK5D4k/8ixbf9hSx/wDSmOuvrkPiT/yLFt/2FLH/ANKY6T2BHRUUUV5p0hRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFAHlmpfBn/io7rWvDfie+0Ke6ZmlWFCw+Y5YAhlIBPODn+VbXgf4aWng/ULvVZ9RudV1e6UrJd3AwcE5OBknJIGSSeldzRVucmrC5UFFFFQMKKKKAIL3/AI8bj/rk38qzPhz/AMk28N/9g6H/ANBFad7/AMeNx/1yb+VZnw5/5Jt4b/7B0P8A6CK6sP1MqnQ6eiiiukzCiiigDjvGn/IweDf+wq//AKTTV0lc340/5GDwb/2FX/8ASaaukrkxHxGtPYKKKK5zQKKrahqFrpWnz399OkFrAheSRzwoFcYi694xAuru6utD0Z+YLO2bZdTL2aWQcpn+4vPqauMHImU1Hc7mSaKEAyyIgPQswFRfbrT/AJ+oP+/grkU+HnhJOW0K1mc9XuAZXP1ZySaf/wAIB4Q/6FvTP/Ada09ku5l7fyOr+3Wn/P1B/wB/BR9utP8An6g/7+CuU/4QDwh/0Lemf+A60f8ACAeEP+hb0z/wHWj2S7h7fyOr+3Wn/P1B/wB/BR9utP8An6g/7+CuU/4QDwh/0Lemf+A60f8ACAeEP+hb0z/wHWj2S7h7fyOr+3Wn/P1B/wB/BR9utP8An6g/7+CuU/4QDwh/0Lemf+A60f8ACAeEP+hb0z/wHWj2S7h7fyOr+3Wn/P1B/wB/BR9utP8An6g/7+CuU/4QDwh/0Lemf+A60f8ACAeEP+hb0z/wHWj2S7h7fyOr+3Wn/P1B/wB/BR9utP8An6g/7+CuU/4QDwh/0Lemf+A60f8ACAeEP+hb0z/wHWj2S7h7fyOr+3Wn/P1B/wB/BR9utP8An6g/7+CuU/4QDwh/0Lemf+A60f8ACAeEP+hb0z/wHWj2S7h7fyOr+3Wn/P1B/wB/BR9utP8An6g/7+CuU/4QDwh/0Lemf+A60f8ACAeEP+hb0z/wHWj2S7h7fyOr+3Wn/P1B/wB/BR9utP8An6g/7+CuU/4QDwh/0Lemf+A60f8ACAeEP+hb0z/wHWj2S7h7fyOr+3Wn/P1B/wB/BR9utP8An6g/7+CuU/4QDwh/0Lemf+A60f8ACAeEP+hb0z/wHWj2S7h7fyOr+3Wn/P1B/wB/BU4IYAggg9xXG/8ACAeEP+hb0z/wHWoj8PtAhJk0yG40iftNptw8BH4A7T+INL2S7h7ddjuKK4zT9c1XQNUttJ8STJd2t2/lWWqqgTdJ2imUcBj2YcN6A12dZyi4vU2jJSV0FFFFSM5r4h/8k68Rf9g+b/0E11dl/wAeFv8A9cl/kK5T4h/8k68Rf9g+b/0E11dl/wAeFv8A9cl/kK68PszKpuT0UUV0GYUUUUAc7qX/ACPOkf8AYNvf/RtrWrWVqX/I86R/2Db3/wBG2tatcVf4zaGwUUUViWFFFFABRRRQAUUUUAFFcn8RvFN54O8Hz6xYwwTTxyxoEnBK4Y4PQg/rXn9p8RfipfaVDqlr4OsbiylTzEeJHYsvsBJn9KtQbVyXJI9sorh/h18R7Xx3a3Eb2xstTtcefbFsgg8blPXGeCO1dxUtNOzGncKKa7iONnbooJNcl4E8eW/jtNSuLO0kt7W0mWKMysN8gIzkgcD6ZNFna4XOvooopDCiis3xBrdt4b0G81i8WRre1Te6xAFjyAAM47kU1qBpUVg+DPEf/CW+FLPXBbfZhdGTEW/dtCyMg5wOcLn8a3qGrOwBRRRSAKKKKACiiigAooooAyX/AOR80r/sGXv/AKNta6Oucf8A5HzSv+wZe/8Ao21ro67qPwIwn8QUUUVqSFFFFABRRRQAVzif8j5qv/YMsv8A0bdV0dc4n/I+ar/2DLL/ANG3VZVvgZUPiNaiiiuE3CiiigAooooAKKKyfFGrS6F4W1TVYI0kltLZ5kR87WKjODjtTSuBrUVyHw38W3fjTwmNXvbeCCYzvFshztwuPUn1rd8QajJpHhrVdThRXls7Oa4RX+6xRCwBx24ptNOwr6XNKiuM+GPjG88ceF5dUvreCCVLp4AkGduAqnPJPPzGuj12/k0vw9qeowqjS2tpLOiv90lULAHHbihxadgvpc0KK8Z+HHxj1Txb4vh0bVbKxgjuIXMLW6uGLqN2PmYjGA1XPij8WNR8E6/baXpVpZXDNbiac3CsduWIAG1h6Z/EVXs5c3KLmVrnrVFYfg3WrjxH4Q0zV7qOKOe7h8x0iBCg5I4ySe3rS+LPFFh4P8Pz6vqBYxoQscafelc9FH+eACaizvYd+pt0V4hD8Q/iprFkda0jwlaf2TgtGGQszr7fOGf6qtdx8OfiLb+PNNuC1v8AZNRtCBcQbsjBzhlPocHjt+Rq3TaVxKSZ29FeefCjx9qXjyx1KfUba0ga1kREFsrAEMCTncx9K9DqZRcXZjTvqFFFFSMKKKKACiuQ8Y614r0vU9Jh8O6LFqFtO5F5I6MTCMqARhh2LevSsj4o/EDU/CE2k6bolnBPqOpOVRp8lVwVUAAEckt3OBirUG7WE3Y9GorD8Jy+JJtG3eKYLODURKwC2hyhTAwTyec5rcqWrDCiiikAUUUUAFFFFABRRRQBmeG/+Qp4m/7Caf8ApJbV0Fc/4b/5Cnib/sJp/wCkltXQV6MPhRzy3CiiiqEFFFFABRRRQBy/g3/kRvD/AP2Dbb/0UtbdYng3/kRvD/8A2Dbb/wBFLW3Xmy3OhbBRRRSGFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAZL/8j5pf/YMvf/RtrXR1zj/8j5pf/YMvf/RtrXR13UfgRhP4jgvBn/Ii+Hv+wZbf+ilrcrD8Gf8AIi+Hv+wZbf8Aopa3K71scj3CiiimIKKKKAPEfEt38WvDGkT6te6/ZG0iZQRFDEzfMwA4MQ9aueFZPirrlvpmrnXbJtMndXdGiiVzGGww4j64B710nxl/5JlqH/XSH/0YtaPww/5Jtof/AFwP/oTVnb3rXNL+7exB4fufEcnjzWIdQ1nT7nSUEn2azhkiMsR3rt3BVDDAyOSeTXSNr2jpbzXDatYiGFgsshuU2oT0DHOAeD1rynwB/wAl18Xf7k//AKOSsP4XeEtK8U694i/tiFrm3tJRsgMjKhZ2f5jtI5AXH40KTBxR7mNb0ltNOojVLI2IODc/aE8r/vrOKmsdRstTthcWF5b3cBOPMgkDrn0yK8D+G3g/TNa8Ya/pmpLLcabp0r+VamVghfeUDHBGSFBH41o/DaU+HPGPjqztNxtLGOd44mJIPlSMFz744oUmJxR7Hfa/o2mTrBqGrWFpM3Kxz3KRsfwJq+jpJGrxsrowyrKcgj1FfM/hGXQtUh1PUPFGga/rt9dTEfaLOEuqcAnkOvzZPTnAxXovwTk1W30/VdLv7W+htLeVZLP7XEyHa27IGf8AdBwO5NNTuwcbHoHiXWl8O+G7/V2jMv2WIuEzjc3QD2GSK8n0WX4qeM9K/t/T9esbS3kZ/Jt8Kv3SQRjY3cfxHNes+Im0kaDdx65NDFpsqGKZpW2rhuOvY+lePwfC/X7CIan4D8WeZZTfvI181oi49MjKt+IFKV7hG1jtfhl4u1vxBBe2HiGyeK/syP33lFBKMkHjpkEduORXXX2v6Npk6wX+rWFpM3Kxz3KIx/AmvNvAvxE1y4OvaN4hiV9U0m1lnDhQpby+GVgvBOSOR15rz/wjJoWqW+p6h4n0DX9cvrqYj7RZwl1TgE8hx82T05wMUc2iHy6n0s1xAlsblpoxAE3mUsNoXGc56Yx3qg/iPQ41hZ9Z09VnJWEtdIBIRwQvPP4V5D4Ik1a3+GfjXS7+2vobS2spns/tcTIdrRyZAz/ug4HdjTPhJ4B0fxBoC61qqzTzwXm23XzCERUIbGO+WJzT5m9hcqW56Bo9z4jf4ianDea1p8+jqj+RZRyRGaM5XBYBd3HPU9xXS6hrmkaS6JqWqWVmz8qtxcJGW+m4ivJ/CTbP2gPEzYziCY8f70dcZ4Yv9I17VtX1fxXoWta7cXDjYLGIukWc5BIZSOMADsBS5rD5bn0vDNFcQpNDIksTjKujBlYeoI61mXHijw/a3RtbjXNMhuAcGKS7jVgfQgnNeU/DOLXEtvFeh2dvqVlZywSyaW99E0Zic7lXnoDypOO65ribZNO8NWM+leMfBV35zsw/tBJGSRc9CufkbHYg4PfNDnoLk1PqFWV0V0YMrDIIOQRS1yPw1n0OXwdBH4fvLy5sonZMXhBlibglDgADrnjjmuuq07ol6BRRRTEFFFFAFC3/AOSh6X/2Cr7/ANG2ldhXH2//ACUPS/8AsFX3/o20rsK55/EdEPhCiiioKCiiigArkPiT/wAixbf9hSx/9KY66+uQ+JP/ACLFt/2FLH/0pjpPYEdFRRRXmnSFFFZ+ta3Y+H9Ne/1CUpEpCqqqWeRz0RVHLMewFPcDQorixqfjbVh51pZaZo1s33Fvw9xcEdiyoyqp9stSeR49/wCg9ov/AILX/wDjtX7KRn7WJ2tFcV5Hj3/oPaL/AOC1/wD47R5Hj3/oPaL/AOC1/wD47T9kw9rE7WiuK8jx7/0HtF/8Fr//AB2jyPHv/Qe0X/wWv/8AHaPZMPaxO1orivI8e/8AQe0X/wAFr/8Ax2jyPHv/AEHtF/8ABa//AMdo9kw9rE7WiuK8jx7/ANB7Rf8AwWv/APHaPI8e/wDQe0X/AMFr/wDx2j2TD2sTtaK4ryPHv/Qe0X/wWv8A/HaPI8e/9B7Rf/Ba/wD8do9kw9rE7WiuK8jx7/0HtF/8Fr//AB2jyPHv/Qe0X/wWv/8AHaPZMPaxO1orivI8e/8AQe0X/wAFr/8Ax2jyPHv/AEHtF/8ABa//AMdo9kw9rE7WiuK8jx7/ANB7Rf8AwWv/APHaPI8e/wDQe0X/AMFr/wDx2j2TD2sTtaK4ryPHv/Qe0X/wWv8A/HaPI8e/9B7Rf/Ba/wD8do9kw9rE7WiuK8jx7/0HtF/8Fr//AB2jyPHv/Qe0X/wWv/8AHaPZMPaxO1orivI8e/8AQe0X/wAFr/8Ax2jyPHv/AEHtF/8ABa//AMdo9kw9rE7WiuK8jx7/ANB7Rf8AwWv/APHaPI8e/wDQe0X/AMFr/wDx2j2TD2sTtaK4ryPHv/Qe0X/wWv8A/HaPI8e/9B7Rf/Ba/wD8do9kw9rE7WiuK8jx7/0HtF/8Fr//AB2jyPHv/Qe0X/wWv/8AHaPZMPaxO1orim1bxno486+sdP1m0X7/APZweG4UdyEdmV/oGBrp9I1ix13TYtQ06cTW8mcHGCpHBVgeQQeCDUSg47lRkpbF6iiipKIL3/jxuP8Ark38qzPhz/yTbw3/ANg6H/0EVp3v/Hjcf9cm/lWZ8Of+SbeG/wDsHQ/+giurD9TKp0OnooorpMwooooA47xp/wAjB4N/7Cr/APpNNXSVzfjT/kYPBv8A2FX/APSaaukrkxHxGtPYKKKK5zQ4vxMP7b8Y6RoD82dtGdUu0PSQqwWFT6jdubH+wK6Suci+b4p6uTyU0q0VfYGSY10ddMVaKOSq7yCiiiqMwooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAztd0iHXdDvNMn4WeMqrd0bqrD3BAI+lSeDNWm1zwfpl/cjFy8Wyf8A66oSj/8AjymrtYXw548LyoPupqV6qj0H2iSs6nwm1F6tHW0UUVgdJzXxD/5J14i/7B83/oJrq7L/AI8Lf/rkv8hXKfEP/knXiL/sHzf+gmursv8Ajwt/+uS/yFdeH2ZlU3J6KKK6DMKKKKAOd1L/AJHnSP8AsG3v/o21rVrK1L/kedI/7Bt7/wCjbWtWuKv8ZtDYKKKKxLCiiigAooooAKKKKAPN/jp/yTC7/wCviH/0Ktz4Y/8AJM/D/wD16L/M1U+LOhal4j8BXOnaTam5u3miZYw6rkBsnliBXBaLN8ZtE0K10Wx8NWCwW8flRySPGXA9SfNxn8K2SvC1+pD0kQeFGWy/aR15bb5YCtwZQvTorN/49XKWeueH/HOt6nqnj7xLfWCbsWFrbo7BAc9MIwAAx6EnmvW/hj8Or7w1c6hrviGeO51vUciTadwRWO5snuzHGccccVyum+GPGfw01rVIdE8PW+vaPePuhLOu6PGducnIIBweMHA5FaKUbuxNmQ/DjUZvF2heJ/BV3rN1c2UCiSyvlJWXylftu5AOFIB6AkVQ+BWh27C+8Uy3l0jaU7/6PGwEcgMTZLDuRnivUvh9pvi1NNvZ/F01us9ycQ2sMUYMC85yyDnqMDJwB19OH+F/h3xn4R1i60DUNAjfRryVvtV4ZAQFCMAVw3IJ28Yz1/BOWkkh22OH03VvDvjS/wBR1bx/4pv7K5aTFnb26SMsQ65GEYADgAcHgk9a7T4bfEG+tvBXiqO5u21E6HCZrKeXOZFO4KDnnGVU88gNjtTNH8O+Ofhpe6lYaV4dttf0y5k328zMuVPQEjORxjI6ccGu28K+GvEmo+DdUsfGtzCZ9SjaJYoYo1NuhBHLIAGbJz1OMDnk0TlG3kCTPPfB3w5l+JHhm48T61r2oNqtxLILVxINsZXgFgQTjPZcYHStnxh4Q1u1+DE8PiTWjd3elv50MkDsQ6HChJCwy2Mnn6elUdEsfin4C0y78OaZodvqFs8jNbXiuMIW4yPmGPXDDg+orqrXwT4nk+EWq6Lq+py3+tXsZdFnuDII8YKxhyf9nk9Mn0GaHLW99LgloVvgX4aisfCNvr63lzJLqEbo1u7Axx7JXGVHbOOa9XrzH4QQ+LdJ0kaBruhiysLKNzBcFwXkdpCxBAY/3jyB2r06sanxMqOwUUUVBQUUUUAFFFFABRRRQBkv/wAj5pX/AGDL3/0ba10dc4//ACPmlf8AYMvf/RtrXR13UfgRhP4gooorUkKKKKACiiigArnE/wCR81X/ALBll/6Nuq6OucT/AJHzVf8AsGWX/o26rKt8DKh8RrUUUVwm4UUUUAFFFFABXM/ET/knPiL/AK8Jf/QTXTVheNLC61TwVrVhZRGW6uLOSOKMEDcxXAGTxVR3Qnscd8Bf+Sar/wBfkv8ASux8b/8AIg+I/wDsF3P/AKKavHfCdt8XvBuiDStN8L2b24kaTM8iM2TjPIlHpXa6dN8QPEHhTxTY+JtFtbSaXT5IrFLdlBld0cEE+YwHO3rjrWk4+9zXJT0sUv2fP+SeXP8A2EZP/QI673xh/wAiRr//AGDrj/0W1eL+ELL4ueCtHfTNM8MWbwPM05NxIjNuIAPIlHHyiu30q4+Ieu6B4jsfE+iWloZdOljsxbsuZJGVhtP7xvUdcUTj73NcE9LHimiZ0DR/Cfi9AQttq0sE7Dug2MB+KmStPxW48Uap8RfEYIkt7IQWsDDkf8fEaKR9VjY/8Crr7T4b69L8DbzQrnTSmsR6h9rt4DKh3fdH3gcD5S/ejS/hvr1j8D9b0g6cf7c1C6jkFv5qZ2JJHgbs7eiseveteeN736k2Z6L8Lf8AkmOgf9e3/sxrz79o+WUaZoEQJ8lppmb03BVC/oWr0zwBpl5o3gTR9O1CEw3cEGyWMsDtOT3BIqD4heCofHPhl9NaUQ3Ubia2mIyFcAjB9iCQfz7VjGSVS5bV42Oi06KGHTLSK2AECQosYXoFAGMfhVbTLLRrYXEmk21jGWdhM1qiAl8/NuK985zmvHrO8+M+gaSvh+HRLa6EKeTBfZV2VRwMNvC8DpuGfWut+FPw9uvB1je32rSrLq+oEGUK24RqMnGe7Ekkn6fUpwsm7gnfocf+z+szeGPE629yltMWQJO67libY2GIyMgdcZrkda0rwZb6dqE134y1HXPE4EkkUtmrNHkDI3MQePUhuO3Tntvh/wDDzxJZeAPFmi6jbHTrvUkCW5eVWDfKRyVJwDnB9jVHw7ofxBsPBl94Ng8J2lqbhZY5tTlmQbkfORwTuODgHtxxxmtrrmbTItoixpuu+KNQ/Z4nubC4up9QhmaBpkJaYQBhkg9cgHGeoH0rhdHPga8tLN08Qa5oHiBShmvph5sRb+IjZhgM9CSMd89a9Q8IeF/G+k/Ce40qxb+xtdivHli8zypFmQgfLn5gueeeuVHY1zmu6N448V6OmkX/AMPrBNVOxZNZ3xq5wRlsg98c8kcnA6UJq79QaZ0Pxhu9dh+Hujvp1/NeWTlft97Z/L5y7Rtb5c4Vjk+mcVw3h7/hBpNZ0658OeLNW8N3yuPMGoRiRZT6blIXB5zuOCD0r03UvC3jXQfh7oumeE9VQ3unoVuYSqYucnJCs44wcgDIyD7VxHiDwl4r8fT2ds3gPTvD8qyhrnUVZBuGMHIHJHOcfMeBz1pQata42tTZ+Nt1cQeKvBqwzyRq8z7gjkBvni64rnfjD4Xhh+I+jyC9umOszjeCwxD8yL8np1z9a7D4meCtb1fVPB/9kWb3kGmHbPKZEUqA0eCdxGeFJ4qf4n+E9c1/xn4VvtLsGuLaylDXEgkRdg8xD0JBPAPSiEkuXXuDW5y/j7Qbj4Z3HhTW9Nvr65tbOYw3BmkyZDvaQBscHKs6/RRWl8TNQl8W+PvCvhPSruRYJgtzPJA5GUfnPHpGrN/wKvQfiPosGu/D/WLSZkTZbtPHI5wEeP5gc9hxg+xNeW/ADR59R1PUfE18zSm3hSwtnfnGFUED/dRUH0NKMrx5nuga1se9oixoqKMKowAOwpaKK5zQKKKKACiiigAooooAzPDf/IU8Tf8AYTT/ANJLaugrn/Df/IU8Tf8AYTT/ANJLaugr0YfCjnluFFFFUIKKKKACiiigDl/Bv/IjeH/+wbbf+ilrbrE8G/8AIjeH/wDsG23/AKKWtuvNludC2CiiikMKKKKACiiigAorzL4q/EPWvBN7o9to9rYztfCTcLlHY5BUADay/wB7vWV/wlnxo/6E3S/yP/x+tFTbVyeZHsVFcN4i1vx1ZWXh19G0K1uri5jzqiOpIt2wnC4cdy/977o/Huahqw7hRRRSGFFFFABRXHeMPH9t4U1jR9INpJcXmqTpGnIVI1LhSxPUnngY/EV2NNppXFcKKKKQwoorlvHWqeKdK0u2l8J6VBqN20+2WOZSQqbSc8MvfHemld2BnU0VFavLJaQvcIEmaNTIo6K2OR+dS0gCiiigAooooAKKKKACiiigAooooAyX/wCR80v/ALBl7/6Nta6Oucf/AJHzS/8AsGXv/o21ro67qPwIwn8RwXgz/kRfD3/YMtv/AEUtblYfgz/kRfD3/YMtv/RS1uV3rY5HuFFFFMQUUUUAcx4/8O3nirwfdaRYyQR3ErxsrTsQg2uCckAnt6Vb8HaNceHvCOnaTdvE89tGUdoiSpO4ngkA9/StyilbW476WPP/AAx4G1PRfiTrviO5ntHs79ZREkbsZF3SKw3AqB0U9CaX4ceBtT8H3+uz6hPaSrqEkbRC3dmKhTITuyo/vjpnvXf0UuVD5mcB4F8D6n4Y8UeINTvZ7SSDUZC8KwuxZQXZvmyoA4I6E03wr4D1DRfHPiPWb6WzlstUaXy442YuA8m7DAqB09Ca9Boo5UHMzyG08A+OPB11fQeENVsW0y7fcqXX34j2P3SMgcZHXHIrtvA3hvUvDejNDq+s3Gp3krbnaSVnSMdlTdzjrzxn0rqKKFFIHJszdf0S18R6Fd6Teg+RcptJXqpzkMPcEA/hXlVj4H+J3heB9M0HxBZNp24mPfjK5PXDIdvrgE17PRQ4pgpWPPvh/wDDh/DMl/qOs3aX+q6gpSZlyVCscsMnlixwSSB0rCtPAXjjwdc31v4Q1WxbTLt9ypdffiPQH7pGQOMjrjkV69RRyoOZnDaR4L1ix8A6to9/rcuo6lqFtLGJLiZ2iiLIVCrnJC5OScc+nFW/hx4WvvCHhUaXqEtvLP57ybrdmZcHGOoBzx6V11FFkK7PP9C8DanpfxS1fxPPPaNZXkbrHGjsZAWKEZBUD+E96xl8AeLvCWu3914K1CxFhfNua2ugf3ZySMcEHGTg56dQa9Zoo5UPmZw/hXwbrWn6FqVvrviO9ur7UARvhuH222c8xk4wcnOcDoBj15yLwr8TtM0660W21fS7+wnLgXF6WaRVbrnIP5fNXrdFHKg5mcp8PvBi+CfDpsGuBcXM0pmnkUYXcQBhfYAfz+ldXRRTSsJu4UUUUxBRRRQBiXUmpReOtJbS7S0uZ/7MvQyXVy0ChfNteQyxuSc44x3PPHO59s8Yf9ALQ/8Awczf/ItZ9v8A8lD0v/sFX3/o20rsK55/EdEPhOf+2eMP+gFof/g5m/8AkWj7Z4w/6AWh/wDg5m/+Ra6CioKOf+2eMP8AoBaH/wCDmb/5Fo+2eMP+gFof/g5m/wDkWugooA5/7Z4w/wCgFof/AIOZv/kWvNr/AF/x1rXhm2fxJ4btNPtTqdkVmWVo5P8Aj4jwPKbcfzK/jXtNch8Sf+RYtv8AsKWP/pTHSewI6KiiivNOkK4jYNd+I19LcfPbaDHHDbRnp58qb3k+oQoo9MtXb1xXhv8A5Gbxh/2E0/8ASeKtaW5lWfunS0UUVscoUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABXN6Wo0n4l3dpANltq1h9taMdBPG6ozAdiyuufXbXSVzk3/JUtF/7Bd3/6HDSl8LLp/Ejs6KKK5TsIL3/jxuP+uTfyrkfAV14oX4f6Atro+jywCxiEby6rKjMu0YJUW7AH2yfrXXXv/Hjcf9cm/lWZ8Of+SbeG/wDsHQ/+giurD9TKp0J/tnjD/oBaH/4OZv8A5Fo+2eMP+gFof/g5m/8AkWugorpMzn/tnjD/AKAWh/8Ag5m/+RaPtnjD/oBaH/4OZv8A5FroKKAPH7fWvF+tal4Pn8UaFbaaDqbGJo5GV3Jt5eDE2SuPds+3evU65vxp/wAjB4N/7Cr/APpNNXSVyYj4jWnsFFFFc5ocZD/yVLWv+wXaf+hzV0dc5D/yVLWv+wXaf+hzV0ddUdkcdT4mFFFFMgK8nn+PGkQXc9v/AGLqDtE7ISpTscetesV8z+BvGn/CG+JPEEv9lXOofaZSuIDjZh268H1qoq40rnrHhP4r2HizXo9Jt9KvbeR0ZxJNt2jaM9jWp4Z8eWfifxBq2kW9nPDLprssjyEbXw5XjH0qHwT49/4TK4vIv7GutP8AsyK26c535JGBwPSsH4deKtb1vxz4o0/Ub4z2tnK628ZjRdgErKOQATwO9FgsdD4R+IFl4vXVja2dxB/Zu3zPNKnfnd0x/uH86m8EeN7TxxYXV3aWk9utvKI2WYgkkjORivOPgn/q/Gn1j/8Aa1aH7Pv/ACLmr/8AX2v/AKAKbS1G1uewV5nrvxr0PTNSew02zudWmQlWaAgRkjsp5LfUDHua6D4majPpfw51q6tmKy+SIgw6gO6oSPwY1zvwX0uw0v4eJrIiX7TdNLJNLty21GKhc+ny5x6mkkrXYktLl3wp8XdD8S6omlywXGnX0h2xpcYKu390MO/sQK9BrxTWfGfws17XbbWL6K/a9t8bXjiaPcQcqW2kEkdq7Xx23jaaeys/C3kWtq/N3qEjpmIZ9G5wBySATyOnNDQNHbUV4hpni7xN4c+I2maDqHiSz1+zvXSNzDtPlliVHIGVYHBxkjBq74r8X+Lrb4tL4d0K6jZJ4kWKCaNdisyElycbvl5br26HpRyhynsVFeK3niXxt8OvFWmx+JtUh1XStQbDOsYXbyAxGACCu4HHQj9LHj3xxrEXj9PDNrrcPh+xSNWkvZIt+9mXd1wcDt25ByaOUOU9iorznwPN4yGuSRXus6dr+gFflv4JY9ytjIwF568EH6g16NSasJnN614uj0nxXonh9LNrm41MsSwk2+Si/wARGDnjd6fdrpK8y0z/AInvx51W8PzQaJYrbRn0duv/AKFIPwr02hjYUUUUhBWF8Of+RZuP+wne/wDpRJW7WF8Of+RZuP8AsJ3v/pRJUVPhNqO51tFFFc50nNfEP/knXiL/ALB83/oJq5Z3fi77FBt0PQyvlrgnWJR2/wCvWqfxD/5J14i/7B83/oJrq7L/AI8Lf/rkv8hXXh9mZVNzG+2eMP8AoBaH/wCDmb/5Fo+2eMP+gFof/g5m/wDkWugoroMzn/tnjD/oBaH/AODmb/5FrD8X6p4/tPDNzPpmi6ct4rxeWbS+kuZOZUBAja3UEEEgkkYGT2rvKKAOD0e58QXeu6FL4lsLWx1E6be7oraYuMeZa8nsp9gW+vauvrK1L/kedI/7Bt7/AOjbWtWuKv8AGbQ2CiiisSwooooAKKKq6lqEGk6Xd6jdNtt7WJppD/sqMn+VMC1RXjHw5+M2peKvF6aPrFpY28VzG5tmt1cNvHIUlmIPyhuw5Ar2enKLi7MSaewUUUVIwooooAKKKKACiiigAori/ih4v1DwV4UTVdNhtpZ2ukhK3Ksy7SGJ4Vgc8DvVfW/EnjD/AIQvQdV8NaPa39/fQxS3MToxRA0YYlRvBHJxyTVKLauK53lFRwNI9vG0qhZCgLqOxxyKkqRhRRRQAUUUUAFFFFABRRRQBzuqyalF400htLtLS5m/s69DJdXTQKF8y25DLG5JzjjH4+t/7Z4w/wCgFof/AIOZv/kWmP8A8j5pX/YMvf8A0ba10dd1H4EYT+I5/wC2eMP+gFof/g5m/wDkWj7Z4w/6AWh/+Dmb/wCRa6CitSTn/tnjD/oBaH/4OZv/AJFo+2eMP+gFof8A4OZv/kWugooA5/7Z4w/6AWh/+Dmb/wCRaPtnjD/oBaH/AODmb/5FroKKAPO/GWr/ABGs7LT30XQ9ONy96qMltePc712OcOGhjCpkDLbhggDvWnosupTeJ72TV7WC2v20qyMsUEpkRT5t10Ygf1x6nrXY1zif8j5qv/YMsv8A0bdVlW+BlQ+I1qKKK4TcKKKKACiiigAooooAKKKKACivE/Gnxuv/AA946n0nT7OyuNOtJEjuHkRzIzdXCkMAMdOQeQa9ot7iK7tormBxJDKgkjcdGUjIP5VUoOKTYk0ySiiipGFFFFABRRRQAUUUUAFFQ3crQWU8ygFo42YA9MgZrzr4fePtf8ZeDdc1N7KyOpWe9bSGBGCSOI9yhgXJ5bA6iqUW1cVz0uiuc8E6j4j1PQmuPFGmw6fqAmZRDEpA2ADB5Zu+e/aujpNWdhhRRRSA4jx38PG8bz25OvXthbpH5ctvDykwznJGQM/ga6Dwz4b0/wAJ6Db6RpqsIIsks5yzseSzH1Na9FVzO1hWV7hRRRUjCiiigAooooAKKKKAOY02fXotd8Srpem6bcwf2ihL3WoPAwb7Jb8bVhcYxjnPrx66n2zxh/0AtD/8HM3/AMi0eG/+Qp4m/wCwmn/pJbV0FejD4Uc8tzn/ALZ4w/6AWh/+Dmb/AORaPtnjD/oBaH/4OZv/AJFroKKoRz/2zxh/0AtD/wDBzN/8i0fbPGH/AEAtD/8ABzN/8i10FFAHP/bPGH/QC0P/AMHM3/yLXM6ZrHxHk+IV/Z3GiWX9hK8YMjXB2RAxoT5UhRWk5JJBTrkZGM16NRQBy/g3/kRvD/8A2Dbb/wBFLW3WJ4N/5Ebw/wD9g22/9FLW3Xmy3OhbBRRRSGFFFFABRRRQB4N+0IZV1vwsYFDTDzTGD0Lbo8frW5bav8bGuoRceHdMWAuokYSRZC55/wCW3pUnxf8ACGveJdc8N3Gj6e11FaM5nYSIuzLIR94jPQ9K9ZrZzSgluRbVninx8mlh1TwcIpXQNPPnaxGfmhqt8ZH1A/Ezwpb6dePa3E4SOOVT9xml2hsd8Zrf+MXhLXPE+oeGZNHsGuks5ZWnIkRdgJix94jP3W6elHxC8J65rXxM8J6rp9g09jYywtcyiRF8sLMGPBIJ454FVBpKPzE1ucL8RvDD/DDVtE13RdZ1KW7nlbz5Lmbczsu08kAZByQQc1t/GbxPJJ4w0zwrPqs2maK0aTX88IYswZjwQoJIAXgep56Vv/Gzwnrniux0ePRLBrt7eWRpQJEXaCFx94j0NL8TPA+t3viXS/F/hmGG51CxCpLaSkASqpJBGSAepBGQcYxTjJOze+oNPWx53pviDQ/BnjvRv+EJ8RXuoaPdyLDfWtyrgLlgufmVQTg5BAyCDzg4r13xH8W/DHhfXbjR9RN59qgCl/Kh3L8yhhg59CKw9GHxF1/xZZ3N5pNp4d0WDH2iIxxSNNg5IGQWBPTIwAPU16De+FvD2pXb3d/oOl3Vy+N809nG7tgYGWIyeABUTcbq40n0Pn34laZYa1488OaraXd39n8SeTJ8+AYlJRBtHY45781s/GHS/wDhF9D8GaTaXlzJHbyTr5sj/O/KHnGPWug+K3gvWrjU/DWq+FtKinTSTgWsW1Am1lZMLkfLwRgdKb8Q/Cvirx34H0TUG02O312zkkeawWReFY4G0kkZwqnGe578Vakvd10Ja3Lf7QMskPgCzaKRkY6lGMq2D/q5a5r4wXM8Hw48FyRTOjmOMkhiM/uV61J4z0r4mfEDwzbxXnh62s0tZlf7Oky+ZPJgrv5bCqATwTn5u/bV+JvgzxDr3gXwvp+mac1xd2UaC4jEqLsIiCnkkA8jtSjZcqbG9bnN/EjwLN4a8M2ni4a/qdxrxnjFxPJLgZYE/JgAqAQMDPStH4uapd33wi8JajJMy3F0YJZWQ7cs0BJ6e9dt8WPD2q+I/AC6bpNobm7E8T+WHVeADnliBXNePPBfiHWPhR4V0ew05ptQskgFxCJUGzbCVPJIB544NEZJ2bBrexlfGy6urbRvBMlrM8c21ipB6sFixn15qh8T/A8vgnSdP8UWuv6pcay10sVxcSzcsxVm3LgAqMrjGTwa6n4qeDdf8Q6b4Ti0rTmuXsQ32gCRF8viMfxEZ+6enpW38ZvDereKPBttY6NZm6uUvklZA6rhQjgnLEDqw/OiMrcqBrc5b4r+K9Xn0DwppFjctbTa7EklzJGdpOQgC5HQEuc/QVj/ABF+Hf8AwgXgf7Vo2t6i0MsscV7BNINkhPIdQANpDD34PXrnpPiL4MfVPBvhgLf2dhr2m26LDDc3CRebhE3qpJxuBCkHp+dcn8SJviFqPgWOfxZBYafZW88arFAwaS6kIIDHazAADJ4x16ejg9rCfW57b4EZn+H/AIdZiWY6bbkknJP7ta6Cuf8AAilPh/4dDDB/s234/wC2a10FYS3ZotgoooqRhRRRQAUUUUAZL/8AI+aX/wBgy9/9G2tdHXOP/wAj5pf/AGDL3/0ba10dd1H4EYT+I4LwZ/yIvh7/ALBlt/6KWtysPwZ/yIvh7/sGW3/opa3K71scj3CiiimIKKKKAPIPH/inxda/Eix8O+HtUjtFu4Y9iyQxsu9mYZJKMewrO13Xvip4Gjg1LWb6xv7EyiNgkUe3PJwdqKwzg81U+Jj6knxo0ltHjSTURBCbdJMbS+58ZyQP1q9rHhb4oeOFgsNeNjY2CyCRtrJjI4zhSSTgnAzisXe7NVayO18S/Ea10TwHZ+IYYPMm1CNDa27n+Jlyd2Oy98ew71x9vbfGLVdNXXItVtrcSJ5sVkVRXKnkAKUI5H95s+tZ/wAa9LTRtI8I2EIdrGzjlhyepwI+T7kA/rXusTxywpJEVaNlDIV6EHpiq1bsydldHE/DLxpfeLtJuF1SzaC+tGCPIIyqSg55GehBBBH09cDqL/xBo2lTLDqOrWFpK3IS4uEjY/gTWXceMtKuPDms6hot/b3k2nW0srRo3KsqsRkdcZHWvMvhj4F0fxlol9r3iNZdQvLi5ePc0zLtwAS3ykZYk9+OlO72QWW7Pb4ZoriFJoZEkicZV0YEMPUEda8p+GnjPX/EHhbxJeapf/aLizjzA/kxpsOxj0VQDyB1ql8KZrjQ/H/iLwglw8+nW/mSQ7znYVdV/DIbn3FZ/wAG/wDkSfGH/XL/ANpvS5rtD5bJnbfCLxJq3ijwtd3ms3f2m4jvWiV/LRMKEQ4woA6k11reI9DS/wDsDazp4vN237OblPMz6bc5z7V4f4Q1e60T4EeI7uydo7g3/lJIpwU3rEpIPY4JwfWsvT9L0C68AmIeEvEtzrc0TPHqMNqzRGTJ24IbBToD8uevekpaIHHU+l6yovE2gz3osotb02S6J2iFbpC5PpjOc+1eOa94g8QWfwKsbe/S7tr2W7+wytOjJI0IDMOvPIAX3ANdBb/Bzw3qPgvThEz2uoSQxTPfhmcsWALDaSFwc4Hpx15zXM3sLlS3N/4l+Pm8FaXF9jjt59QuG2pHJJ/q1wfnKjkjjHap/FGraldeDobzwzrmm210ZUD3Ms0Xl/dJZcsCuenHWvPPjX4ft9N0HQruSWW61GMCze7lY7pI1ViMjOM5J561ofEvQNN8O/CKCz0uAwwPexTMpdmy7IcnJJ9BSbeo0loeraPJcHQLCW/uIprn7LG088ZGx32jcwI4wTk8cVFa+JNCvrv7Jaazp1xc5x5MV0jP+QOa8d+IWq3kXw48D6JbzGGHUbOHz2BxuCxxgKfbL5P0FaHjv4XeHtA8CTalpaTQahp4R/tBmYmU7gDkZwDzkYA5xT5n0FyrqevXl/Z6fEJb27gto2baHmkCAn0ye/BqrceIdFtbtLS41ewhuXwVhkuUV2z0wCc814r441i6174FeG9QvWL3L3wSRz1cosybj7nbmrHjHwDomm/CaPW44ZG1YRwTS3byszStIVDZBOMfN6dh70uZ9A5V1PdKK5r4e3Mt38PtDmncvIbVVLMck44H6Culq1qSwooopiCiiigChb/8lD0v/sFX3/o20rsK4+3/AOSh6X/2Cr7/ANG2ldhXPP4joh8IUUUVBQUUUUAFch8Sf+RYtv8AsKWP/pTHXX1yHxJ/5Fi2/wCwpY/+lMdJ7AjoqKKK806QrivDf/IzeMP+wmn/AKTxV2tcV4b/AORm8Yf9hNP/AEnirWluzGt8J0tFFFbHMFICCMggj2rwNH1j4z+Mb+2TUpbLw7YnhI+hXJC5XOCzYJyegz+L/FHw+1L4aWA8SeF9cu9lu6/aIpMA4JABOOGGSAQR3quXoVY96ori7P4j6Yvw/wBP8U6mHiiuCIpFiQttlyQQB6ZU/hipNJ+JvhvW/EUOiafNPPdSpuV1iPljC7iCfUD8M96VmTZnYUVweo/GDwfpurtp0l5NIyPskmhhLxoe/Pf8Aa5L4WXwv/iv4vmiuTPbSPPJCwfKlTPkEe2KOVjse00Vwut/FzwnoWqSafNdTzzxNslNtFvWNu4JyMke2a6K08UaRqHhuXX7O7Fxp8UTyu6A5UIMsCp5BAHQ0WYrM2KK8/Pxm8IbLUpcXUjXDFVjjgJZecZYe56dT7Vp+KPiT4b8I3S2mo3Mj3ZAYwW6b2UHoW6AfTOaLMdmdbRWF4Z8X6L4utHuNIu/N8sgSxOpV4yemQf59Kxde+K/hXw9qz6bdXU0txG22X7PFvWI+hPr6gZxRZiszt6K8Q8J6vDrHx+1C7sbs3FjNbs8TKx2keWnY9Km8CXVxJ8ePFELzytEq3O1GclR++ToKfKOx7TRRXmfh2PRR8Y9be21a9m1IxyebaSQ4ijGUyQ2ee3bvSSEemUVwes/F7wnoupTWEk9zczQsVlNtDuVCOoySAce2a1bTx/4evvC914htrtpLK0/14CHzIzxwVPPf6UWYWZ09BIAJJwB3NeLfCv4kzatr97Z69fTz32oSoLRAn7tAockADhe31rm5WtfHfxG1iz8YeIZdMtraZ47W3LhFG1yoUFvlU4HPGSTT5SuU+jaK8e8N+CfE3g7xrav4evX1HwvPt88yXCbQpJB+XPLL94FRz09RVPxp8UHtPiVY2MN9cQ6Np8yi/jjjwXkVm3D1ZcbeOlHL2Fbse20Vy+q/EHw7oug2OsXt06QX8YktoxGTJICAfu9uCOvFUPD3xX8LeJNTj061nngupTiJLmLbvPoCCRn60rMVmdvRXiPi/4ota/E2ys4764i0TTptt7FHHgvIrMGB7sOnHSvStU8d6Bo3h6y1q+uzHbXsay2ybCZJAyhhhfoRnsKOVjszpKK5Dwx8S/DXiy8+xWFzJHdkErBcJsZwOu3qD9M5rr6VrCCucm/5Klov/YLu/8A0OGujrnJv+SpaL/2C7v/ANDhpS2ZdP4kdnRRRXKdhBe/8eNx/wBcm/lWZ8Of+SbeG/8AsHQ/+gitO9/48bj/AK5N/Ksz4c/8k28N/wDYOh/9BFdWH6mVTodPRRRXSZhRRRQBx3jT/kYPBv8A2FX/APSaaukrm/Gn/IweDf8AsKv/AOk01dJXJiPiNaewUUUVzmhxkP8AyVLWv+wXaf8Aoc1dHXOQ/wDJUta/7Bdp/wChzV0ddUdkcdT4mFFFFMgK8T+CSOvivxWWUgFlxkf7b17ZRTT0C4V4x8JkZfiV4zLKQDNJgkf9Nmr2eihMLniPwUR1j8ablYZMeMj/AK7Vf/Z/Rk8O6sGUr/pa9Rj+AV6/RTcrjbMzxFo0XiHw7f6RM21LqExhsZ2t1VvwIB/CvFfDPjDVPhVFceG/FOi3MtkJGaGaIAgg9QpOFdT16gjJz7e+0hUMMMAR6Gkn0BM+aPEM8vxO1OxtvCfhH7HBblg0scSoH3beXZQFUDHqeprp/i/NLH470ca8l4/hTYpdICQrNk7s4/i+7746V7iAAAAMAdqQqGGGAI9DT5h8x823Euin4j+Fb/QNFn03Qxcwok0sRUTsJAWfJJJA3KMk9q1/GWp3Oj/H631G0s5bySCJHMEQy7p5R34HqF3H8K98rzy68D6pN8ZLXxastr/Z0Ue1kLt5mfKZOm3HUjvTUgTOC8Va5/wt/wAT6JpOg2F2trauzXE0yAbAxXcTgkAAL68k49K6D4lanpsXiiG18W+FPO0UpiHVYGbzRkdMjGMNn5ST6969fChc4AGTk4oIBGCAQexpcwrnzv4F0+CX4qWlz4J/tP8AsSIZu5roYXGDuUkdQeMA8557Zr6GllSCF5ZGCxopZmPYDkmnBQowoAHoKCAQQQCD1BpN3Bu55t8G4nu9I1nxFMpE2r6jJLk/3AeP/HmcV6VTURI0CIqqo6BRgU6hu4nqFFFFIArC+HP/ACLNx/2E73/0okrdrC+HP/Is3H/YTvf/AEokqKnwm1Hc62iiiuc6TmviH/yTrxF/2D5v/QTXV2X/AB4W/wD1yX+QrlPiH/yTrxF/2D5v/QTXV2X/AB4W/wD1yX+Qrrw+zMqm5PRRRXQZhRRRQBzupf8AI86R/wBg29/9G2tatZWpf8jzpH/YNvf/AEba1q1xV/jNobBRRRWJYUUUUAFeSfH3xJ/ZvhGDRYXxPqcvzgHkRJgn822j869brxzWvAus+NPjEt9rulsvhm1Tyoy0y4lVQcDCtuG52J6DitKdua76Ey2sjgPGLaDoWl+DtQ8M6xZXGqaUqx3AgkBLOD5m7HpvL/8AfQFev/EPWItY+Cd/q9k5WO6tYZkIPK5kTj6joaNZ+DnhC60W9g07Rora9eFhbzCVzskx8p5bGM4rk9G8LeMh8Fdc8KX+jyJdq6GxQzRnzEaRWZQQ2Bghjzj71ac0ZWfZk2aMXTfADeJPg+fEup63qMl3a2k01lAJB5USRFsLtxyW2k5yOo9K2vDPjvVNM+AN5q8k7T39pO1nbSyncRkqFJz12hzjPoBXYeHPD2q2PwTbQbm0MeqHT7qH7PvUne/mbRkHHO4d+9YPg/4c6hP8H9S8La7AbC6ubl5YyWV9hGwo3yk8ZXkZ6Zock737hZ9DA8O/COXxb4Si8S6h4h1E67exmeCTzMqnJ27ifmOeDwRjPtRoHjHU/EXwR8VW2pTyS3umRBBcM3zsjfd3HuRtYZ78VZ0sfFzwxoB8LWeg21ykYaK2v1lU+WpJ5BLAcZONwGO4rX0z4ZX/AIe+EOt6REq3et6nHukSNgFzxtQE4HAzye5NNvu+ugW7HE6Z4JbX/gzN4m1HWdQeeyhmksYBIPKiWNmyCMZJJDc5Hb0qXRfCl143+Fd94j13XtSnn06CYWEPmjy0ES7vmBGSScjPXAHJr0Dw94X1my+BNz4euLIpqr2l1Gtv5iklnZyoyDjnI70ngrwtrWlfBrUtCvbExalNDdrHAZFJJdSF5Bxzn1oc99eocpQ+HOpaprnwUu1m17+z7iCSS3TUp/mMMY2nJJI6BiAc8celeWa/YeDdN0J7jSvE2qap4niZXe5hVhDncMncVBAx0IY8/WvQdJ+HXiST4Iaj4cmt/smqPfG4jgeVSJVGw4LAkDO04z3AzjrVI6H8QtU+G7eEIvCdpp0EMa+dO0yB7oowYBVB+8SASxODzyM000m2n1E72HePdSutX/Z60C+vZWluZZ4fMkbq5AkXJ9zio/irNLD8IPBBikdCYYASrEf8u4rV1zwb4k1D4GaNoEWkv/a1rcKZbYyx5Cgyc7t2OhXv3qb4ieC/EOt/DXwrpWnac097ZRwi4iEqLsKwhTySAeeODSTV16sbTM34veKbiG78O+F/7Sl03Trm3inv7mMMW8tm29F5IAVjgdeK5Vdf8PeBvFOj3ngXxJfX1jLJs1G1uEdQVyOeUUHIJxxkEe9ek/EbwFrOqzaD4i8PxxSavpSRq1tKQBIqncMZODg7sgnkH84tO/4WT4i8S2Dz6PZ+HNKgx9qzHFKZhkEgAgnJxgYxjJ5NEWuUGnc5b4uXkkfxJhi8UpqjeFRCvkR2ThQ52/MeflLbs574xW18KE8OxeKJ28M+L7trKSM/8SS9iIfp94MSFJB5+UE46+tb3jKH4gWXiYXul28Gv+HpFw+lSrGuw4wQcgFueQeeuCK53wl4D1u/+JVv4pvfD1r4ZsLUZW0t3XMj7SOAvTrycDgdO9K6cLXC2p7dRRRXOaBRRRQAUUUUAZL/API+aV/2DL3/ANG2tdHXOP8A8j5pX/YMvf8A0ba10dd1H4EYT+IKKKK1JCiiigAooooAK5xP+R81X/sGWX/o26ro65xP+R81X/sGWX/o26rKt8DKh8RrUUUVwm4UUUUAFFFFABRRRQAVk+Jtbi8N+GdR1ibBW1hZwp/iboq/ixA/GtavM/jLo3ibxJoljo2gae1zDJN5t1IJUQAL91TuIzySeP7oqopNpMTdkeSeHYfD2o/DrxRPrOs2aa9qEvnQLNIBJuj+cH2LsWFes/AzxL/bXgVdPmfddaU/kHJ5MR5Q/wA1/wCA1pWXwe8E29hbw3GiRTzRxKskzSSAyMBgscN3PNcx4S8E694J+Ld9Lp2ms3he8DJ5izLiNSNy/KW3Ha3y9OhNbSlGaaISaaOU8C+HZ/GfjDxZpl3q9/baUlw0k8NtLtaZhI4QEkH5Rljjvx6VtfCpbrw18WNf8IR3s8+mQRSNGkrZwVZNrY6A7WIOMZrf+FnhPXPD/izxVearYNb297Lut3MiNvHmOeikkcEdaPDnhPXLH44a54gubBo9LuYpFiuPMQhiSmOAc/wnt2olJO66WBLY5DQtMuvjZ4r1u71nVbyDRrFwsFrbuBgMW24yCAcLknBJJrT8IXWp/D/4uf8ACDy6jNfaPdput/OOTHlCykeh4KnHB64p0fhTxv8ADXxVqd94S06DVtI1BtxgZwGTklQRkHK7iARkEHn21fBPgrxJqPjyXxz4xjitrsKVtbONgdmV2gnBIACk4GScnJx3bas9dAS+843wtoE/jH4neMdIudWvrbTFup5J47aXa0uJmCrk5wvzE49h+EHg3w1eah4813wF/b2ow6BaPLLLHFKFaVUcIozjAzvBOBg46dMd98O/CeuaJ8SPFmqajYNBZX0srW0pkRvMBmLDgEkcc8ijwX4T1zS/jF4m1y9sGi028SYQTmRCHLSow4ByOAeooc99egJbGB8LUuPDPxc1/wAJQXk82lxRSMkcrZwVZNremcMQcYzWf4q0jwi/ijVJvGfji61G6aQi3s9OjO6A5PyYw6jHAAyO+a6/w94S1yy+OOua/cWLR6XcwyLFceYh3E7McA57HqO1cz4R8O+Pfh/rOq21n4XtdVku3Hk6lLMqhcZ+YnOcHOSvBour3v0C2lix8ENWvLrRfFemTXU89pZqrWwnJ3RhhICMdvujjsc1S+EzFfg743ZSQwinII7f6PXQfC/wd4l8NXviyLWbED7cg8q5SRCkzKZOgzkZ35GQKZ8PPBfiHRfhn4r0rUdOaC+vY5lt4jKjby0O0cgkDnjk0Sau/kCT0MDwr4pvvDf7P+palaysb5tQaCGRjuKFgg3c+gyR74rkQfCieFBrKeM9THjXb5xG2XG/Odm/Z1x/Fuxn2r03wx8NdUuvg1qHhfV4PsF/LdtPBvdWCsNhUkqTwSCD3wTWZZRfFPTfDkPhi18LWiXEGIotU3RNtjByOpKnjjJ7ds01JXdu4rMTxV4q8S+IPgZpOrWTziSSUxanLbZDbULLk46KxAJxxzjpXL6H/wAIJNf6fcaF4p1jwxqSODJJfIJUkP8AvKQBz/ewCOor2DxBo3jyPwPYQ6HrUP8AbkAzd7IIkW5zyQuVwuD06ZHWvPPEXhjxf47W1s5PAOm6NdCUNcamrou4YIOcckZOcfMeBSi1YbTPoBDmNTuDZA+YdD706qmlWI0zSLLTxI0gtYEhDt1baoXJ+uKt1zmgUUUUgCiiigAooooAKKKKAMzw3/yFPE3/AGE0/wDSS2roK5/w3/yFPE3/AGE0/wDSS2roK9GHwo55bhRRRVCCiiigAooooA5fwb/yI3h//sG23/opa26xPBv/ACI3h/8A7Btt/wCilrbrzZbnQtgooopDCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigDmfGvgfSvHWlR2WpGWN4XLwzwkBoyRg9eCD3HtXGRfA2C4lt49Z8U6vqdhbkGO0kYhQB25JwMccAceles0VanJKyYnFMZFFHBCkMSKkcahUVRgKBwAKfRRUDCiiigAooooAKKKKAMl/+R80v/sGXv8A6Nta6Oucf/kfNL/7Bl7/AOjbWujruo/AjCfxHBeDP+RF8Pf9gy2/9FLW5WH4M/5EXw9/2DLb/wBFLW5Xetjke4UUUUxBRRRQB55r3gPVNU+KWleJ4J7NbK0WMSRu7CQ7SxOAFI7jvXodFFJKw27mD4v8KWPjHQZNMvS0Z3CSGZRlonHQj16kEdwfxrzZPA/xSsdP/sOy8S2h0vb5aMXIZU9M7C68dgeK9nopOKY1Jo47wR8PrHwj4eudPkcXc96MXkpXAcYI2Af3QCfzP0HIWvw98c+ELi7g8H65aHTbl9wS6HzIemcFSM4wMjrjkV7BRRyoOZnEfD7wAfCIvL7ULwXusXxzPOM7QM5IBPJyeST1444rM8A/D3VvCvh3XtPvriykm1BNsTQO5VfkZfmyo7kdM16VRRyoOZnnHhH4az6d8P8AVfDGuzW8gvp2kD2rMwQbUCn5lHIZM/lWHb+CPiVpujN4asdc09dKyQlwGZZUQnJAIXI+mT9a9joo5UHMzjdT8BJq/gCPw3f6nc3VxGA6387F380ZO7k9OSMZ6d+9cTJ8PviNf6NF4Zvtd08aJHtXcuSxRT8o+6CcYGASOg54r2iihxTBSaPOfGvw2k1jwLpmhaPcIkumspjNyxxINpDZIBwSTn0qrq3gvxj4k+HS6Lq19pz6ot4sqPuKoIVTAUlU+9nJ6d+teoUUcqDmZwXiD4cL4j8BaRotxcJDqOm20SRXCAsm9UCsOxKnHseAfauauvAnxH8Q2cOia94hsv7JjZfMePLPIF6Z+UFj9SOeTk17FRQ4pgpM898bfDubWPAumeG9Be2gSxnR1N07AFQjg8qpyxLZ6etafirwrfa58OD4dtpbdLzyYE3yswjyjKTyAT/CccV19FHKhXZieENHuNA8J6bpV08Tz2sWx2iJKk5PQkA/pW3RRTEFFFFMAooooAoW/wDyUPS/+wVff+jbSuwrj7f/AJKHpf8A2Cr7/wBG2ldhXPP4joh8IUUUVBQUUUUAFch8Sf8AkWLb/sKWP/pTHXX1yHxJ/wCRYtv+wpY/+lMdJ7AjoqKKK806QrivDf8AyM3jD/sJp/6TxV2tcV4b/wCRm8Yf9hNP/SeKtaW7Ma3wnS02RPMidM43KRn0p1FbHMeE/Ay9i0XX9e8PagywX8jIERzgs0ZcMo9+QcexrufjBrFppvw9v7aaVBcXoWGCLPzMdwJOPQAE5+nrTvGHwp0Txbe/2j5k1hqJxungxh8dCynqfcEGsbTPgdpEOoJd6xqt5q2wgiKQbFbHZuSSPbIq7pu5V1e5xmrWM1h+znpYnUq09+J1U9lYvt/MYP416RoGjWelfCW3vNOsYE1E6K0yTpGPMMjxbj83Xqf0FbnjDwfa+L/D6aPLO9pCkqSKYUHG0EAAdMc1r6TpyaTotjpiOZI7S3jtw7Dlgihcn8qTloJs8r+Atjp03hLUpWghku3uzFMXUE+XsXaD/s5LfrWB8ONll4y8df2Rjy4bW5+ybORgSHZj8hXY3HwW05dXnutL1vUtLtLknz7W2bAYHqobPC+xBra8JfDfTfB2uX2o6fczMl1H5YgcDEa5B4PU9O9Ntajujyj4V/8ACTf2NqT6Doekaik03l3Ml7JhyNo+XGRleSfck1u+CtD1LRPBXjgXMli1pcWMkkSWl0syo3lyZHyk44wOfSuhvvg5a/2ldXWha/qOix3efPt7ZjsYHqBgj5eTwc10OifD/SdA8JX+g2LSgX0Tx3Fy+DI5ZSufTgHgf/XockDaOL+BehaZceELq/ubG3nuWviBJLEGKhFQrjPTBJNcr4Zm8RSfFTxNdaVpthf6mk0wZb59vlL5mMryPYfSvZ/BXhGDwXob6Xb3UlyjTtNvkUA5IAxx/u1ieJPhbZa1rx13TdVvdG1N/wDWTWjY38YzwQQcdcHn0o5ldhfU5nwRoWvWfxQ1DVrmHSbQXETrc2dldo2xsAj5ASRlgD+Jqp8BYLa6uPEF1dokmph0DNIAXCtuLdfU9foK7/wZ8PNO8Hz3N6tzPf6ncgiW7uD8xBOSB6ZPJySTWPq/wgsbzxDNrGlaxfaPLcEmdLU4DE/e2kEFc9xyKLoLo5Pwhb2Vr+0Lq8Gnqi2yLMFWPhVOF3AemGyMVN4B/wCS++Kv926/9HJXZ+G/hbpfhfxT/bdhd3B/c+UIJMMOVALFupJIJ/GsnV/gpY6rr19q39uXtvLdzPKyxouF3HOM+lF0F0eo14v4RIH7Q3iUk4Ahl5/4FHW3ovwbttG1uy1JfEGoTNazLKI3A2tg5wea39L8BWul+O9Q8VJezST3qMjQMo2ruKng9f4aWiFojzKx0Dxj4T1TVLjwPfabrenzS7po4pI5WA5KhwSCDgn7p5rqPhVr+m60+raRL4ZstK1CHBu0ghwk2GIIYHkFWJ4JPWrNz8IIINTuL3w74i1LQhcnMsNqx2/QYIIHsc47V0Pg3wJpvgyC4NrLNc3l0Q1xdTnLv149hkk/zNNtWG2jzv8AZ/t4JItdleGNpY5otjsoLLkP0Pary2HgP4u6vf8Al2t/p2rWwHmyHZFJKORnblg2MAE4B5FbOifCW28P+KhrGn63ex2wm802QA2t1wrHPIGTjIpPEvwd0nW9Zk1ewv7nSr2Vi8hgAKsx6sBwQT3waLq9wurnnk+nah8JviFpNnpmsvd2166GS2xtLIz7SGXJGfRq3PHFrb/8L98NRmCLZLHC0i7BhyZJMkjvXU+F/hBpOgaymr3l9c6pfRNvjacAKrdmxySR2yau+N/hra+MtTs9TGpXGn31sgjWWFc5UEsO4IIJPIPejmVwurmN8QdZ0qy8UaLo9p4YtdW14qrWglOxIBuO0e/IJxwBiuL8SL4hHxa8Iy+Io9MhvJbm2KR2AOFTzgMMT1PXua9C1z4T2esW+kyLrF7b6npsKQpfry8gU5BbkfMCTgg9+9Vz8HrR9R0/VZdf1O41W1mWZ7u4YSGUqVKjB6AY/X6UJpAmjnPGFrbn9oLw/GYIvLljiaRdgw5LSZJHeqXxJF4/xk0S0tbO2n8qCEWdrcnbCx3NgHoMZGPwAr0Lxt8NLXxhq1pqq6nc6dfW6CMSwrnKgkjuCCCTzmna78MtM8QaHplld3l0L3TYlig1BSPNIAH3vXpn1z360KSC6OD1zRvGGq+NtD1O+sdD0y/tZUZRBeKrzIHB6FstjkceuK9zrz3QvhRa6fr8Ot6xrN/rd7b48hrtiQhHQ8kk47c4HpXoVKTE2Fc5N/yVLRf+wXd/+hw10dc5N/yVLRf+wXd/+hw1EtmVT+JHZ0UUVynYQXv/AB43H/XJv5VmfDn/AJJt4b/7B0P/AKCK073/AI8bj/rk38qzPhz/AMk28N/9g6H/ANBFdWH6mVTodPRRRXSZhRRRQBx3jT/kYPBv/YVf/wBJpq6Sub8af8jB4N/7Cr/+k01dJXJiPiNaewUUUVzmhxkP/JUta/7Bdp/6HNXR1zkP/JUta/7Bdp/6HNXR11R2Rx1PiYUUUUyDkfGXj+y8GXmm211Z3Fw1+WCGIgBcFRzn/errq8S+O3/Ie8K/70n/AKFHWr8TPGHiPw/420Kw0ScFLpFzasilZnMhUAkjIB4HBFVy3sVY9YrG8Q+KtI8LwwPql0I3uH2QRKNzynjoPbIyegzXk3ijX/iB8PdV03UNW1u31G0u3PmW8cQCDbjco+UEcHgj0596fxwttYHizR5ZL2JrOZiLCMDmFh5Yctxzlip6np2oUdQUT36ivKPFvi7xL4F8IaXp13d2994mv5JFF0qjYqhuCBtAzhlAyMdeuKx9R8SeNfh/Lp+pat4gstcsLiUR3NvHtJQ4zwQAegOD0yORRyi5T2+ivI/ih4317QfEHh9dAugYLuMSGDy1KzksMAkjIBBxwR1p2pSfEDwn4D8RarrWvQzXYFu1oYEVhEWlUP1QdjjHIo5QsetUV4np918UPE3g6LxBZ6zb2scURMUAjXzLrZncx+XGSQcDgcfid7w38U/tfwyv/EOpxK17pp8mVE+UTOcbCPTJYA+mCaOULHp1U9X1FNI0a+1OVGkjtLd52RerBVLED34rx/S3+K3irRW8S2WtW1rG+57ayEajzFBPABU9cEDcef1rW0fxzJ41+E/idryNI9RsrCeO4CDCsDE21gO2cEY9RRyhY7fwd4rtvGWhf2ra28tvH5rRbJSCcjHPH1rfryT4UTanB8ILuTRrWO51IXMv2eKRwqljtGSSRwOvXnFYWu3/AMQ/Deif21qfjKyhvgwLaV+6MmC2MAAYOM5IHbPNHLqFtT2rV9Y0/QdNl1DU7pLa1j+87+vYADkn2FP0rUoNY0q11K13/Z7qJZY94wdpGRkV4t8R9T1nxL8J9E10Tww2UoH222A5kl3YUrwcAFW79+9anh/XNc8D/CZta1a7t76KSCAaTbqMeWGGAr4UZxwTyfunmjl0C2h7BRXhDa18Q7bwunjBvFFhIpUTtphVM+UTxxj0OcA5x3zXr3hTX4/FHhew1mNPL+0x5ZM52uCVYfmDSasDVjZooopCCsL4c/8AIs3H/YTvf/SiSt2sL4c/8izcf9hO9/8ASiSoqfCbUdzraKKK5zpOa+If/JOvEX/YPm/9BNdXZf8AHhb/APXJf5CuU+If/JOvEX/YPm/9BNdXZf8AHhb/APXJf5CuvD7Myqbk9FFFdBmFFFFAHO6l/wAjzpH/AGDb3/0ba1q1lanx430cnodOvV/HzLY4/IH8q1a4q/xm0NgooorEsKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKAMl/wDkfNK/7Bl7/wCjbWujrnG58eaZj+HTLzPtmW2x/I/lXR13UfgRhP4gooorUkKKKKACiiigArnE/wCR81X/ALBll/6Nuq6OucT/AJHzVf8AsGWX/o26rKt8DKh8RrUUUVwm4UUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFAGZ4b/AOQp4m/7Caf+kltXQVz/AIb/AOQp4m/7Caf+kltXQV6MPhRzy3CiiiqEFFFFABRRRQBy/g3/AJEbw/8A9g22/wDRS1t1ieDf+RI0Ed1063U+xEagj862682W50LYKKKKQwooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigDJf/kfNL/7Bl7/6Nta6Oucbnx5pmP4dMvM+2ZbbH8j+VdHXdR+BGE/iOC8Gf8iL4e/7Blt/6KWtysPwZ/yIvh7/ALBlt/6KWtyu9bHI9wooopiCiiigAry/4b+Ltc1/xl4isNTvvPtbNmECeUi7MSEdVUE8DvXqFeKfCH5PiN4uRuG3vx9JjUS3RS2Zt/F/xdrnhebQV0a++zC6aYTfukfdt8vH3lOPvHp616Pc6hZWcsUV1eW8EkxxEksqqXPooJ56jp61498ffmu/C6Dlt9xx+MVHx0jeXWvCkccjRSO8qrIpwVJaPBHuKTlZsaV0j1lfEGjPqP8AZy6tYG+zt+zC4TzM+m3Oc1fkkSGNpJXVI1GWZjgAepNeDfFvwRofhPQdKvtFt3trkXPlPIJWZn+UsGOT1yvUY61ofFXUbzVrvwl4cNw0NvqQikuGXjezsqjPsOTj39qfNa9w5b7HrFj4i0TU7g29hrGn3U46xwXKO35A5q1d6hZWHl/bLy3t/MOE86VU3H0GTzXjnxJ+HeheFvCUetaEk1lfWUsf7wTsTJk4zyeGBwcjHf8ADK+KOrXOseAvBWpzki6mR3dl4y+1ASPTJGaTk1uCinse6W2saZeXs1la6jaT3UOfNgimVnTtyoORVeTxDo5uZbGLV7Fr9Vb/AEdbhDICB025zmuA1zwnp3w48A6zq2iiZdUezS2kuWkJY73VWYf3Tznj0FZfgn4a+H9W+Gkep3UTtqd1HJKt2JWBhZWYLtAOONoPPvTu9gstzoPg/wCKdZ8U6RqU+s3n2qSGdUjbykTAK5x8oFdrL4k0OC/+wy6zp8d3nb5DXKB8+m3Oc14V4D1a50T4Q+L7+zcpcJLGiOvVS+1Mj3G7NZuj6dod34GkSXwj4ivdYuEkZNSgt2eMPk7dpDYKjAzx6+1SpWSG46n0pdXdtZQGe7uIreEYBklcIoz7mqq69o7XkNmuq2JuplDxQi4Qu6kZBAzkgjmvGdRn1aX9nueDWbe6hubW6SBRdRsjtGHUqfm5IAOM+1bPgbwbpOk+CrPxm8Us+sQWct0jvISvCMFXb0wFxVczbFyqx6XfeINF0y4W3v8AV7C1nbpHPcojH8Ca0EdJEV0ZWRhlWU5BHqK+Z/CH9kapaalf+IvDXiDX726nYfarOJnVOATyGHz5Oec4GK6XwlqfiHw98L/FcNxbahaLZqrWD3ULRsokJU7cjtw3HQmkpg4Hsg1/RjqX9nDVrH7dnb9m+0J5mfTbnOa4T4P+K9b8U22sPrN79qa3ljWI+UibQQ2fugZ6DrWF8Ovht4f1/wCH0ep38TyahemUrciVg0BV2VSuDjOVzz6079n3/jw17nP76Hn8Gou20OySZ7PRRRWhmFFFFABRRRQBQt/+Sh6X/wBgq+/9G2ldhXH2/wDyUPS/+wVff+jbSuwrnn8R0Q+EKKKKgoKKKKACuQ+JP/IsW3/YUsf/AEpjrr65D4k8eFoG7LqliT7f6THSewI6KiiivNOkK4rw3/yM3jD/ALCaf+k8VdrXFaD+58ZeL7Zv9YbyC4HujwIAfzRh+Fa0t2ZVvhOlooorY5QooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACucm/5Klov/AGC7v/0OGujrnD+++KunKvW30e4d/YPLEF/Pa35GlLZl0/iR2dFFFcp2EF7/AMeNx/1yb+VZnw5/5Jt4b/7B0P8A6CK074gWFyT0ETfyNZvw6BHw38Ng/wDQOhP/AI4K6sP1MqnQ6aiiiukzCiiigDjvGn/IweDf+wq//pNNXSVzfjT/AJGDwb/2FX/9Jpq6SuTEfEa09gooornNDjIf+Spa1/2C7T/0OaujrnIf+Spa1/2C7T/0OaujrqjsjjqfEwooopkHifx0R313wrtVjhpM4H+1HUvxORm+L/gohSQJrfJA/wCngV7PRVKQ7njX7QaM+laHtUtieXoM/wAK0348wzRnwzqQhd7e2llErKOFJMZAJ7Z2n8q9npCAwwQCD2NClawJnhnxKB8e+GNG8X6Np91PZ2sksc9tIuJNm4fNhSePlOSD0IPrWJ53gPV7ywsfC/gO61C8nP7+Ka8njEf/AAIOffJPAAr6PxgYFIFVSSFAJ64HWjmHzHifxPtBB4/8D28MJWKFoYwoJYKBKoxk8njua7X4v/8AJLNa+kP/AKOjruK5vx7oN34n8Faho9i8SXNx5ewzMQo2yKxyQCeintRfYV9jynwz8Uk8L/Dq20q+0m9+2+S4sZFQeVOrM2DnPYkg4B6VY8L/AA71W5+DesWksLQ6hqMi3FvDJ8pxHgqDnoWw3X1Br1LwXodx4e8HaZpN6YnuLVCrtGSVyWJ4JA9fSugocuw2+x4X4a+LCeEvCkfh/VNFvxrNirRRReWFV+Tt3ZIZeuOAelWPBnhXUdD+FfjDUdTheG51SxlZYXXDBFjfBI7Elzx7CvayoLBiBkdDjpS0cwrngnh7+2Yv2fdQ/sYTrdfbG3+SCJBFld23HPTrjtmuZ8zwrdfDuS10jQL258R+Ur3l26M4g2sGd85wAQCBgDrz7/UNIFC5wAMnJxT5h8x4dfwS6h+zXZC1jeRrdg0qqMlQsrA8e2c/Tmmw3A+IfwZbRNHtrk6hokVuXR1AEzAMCEwTngMe3OBXulIFCjCgAegpcwrnzFa33gSLw7b2kng68uvFCbYZIGmnVZHBwW+V8gkc4C9TjpX0B4N05NL8KWNsmlDSsqZGshO03lFiSRubknnn0PFbu1d27A3dM45paHK4N3CiiipEFYXw5/5Fm4/7Cd7/AOlElbtYXw5/5Fm4/wCwne/+lElRU+E2o7nW0UUVznSc18Q/+SdeIv8AsHzf+gmursv+PC3/AOuS/wAhXKfEP/knXiL/ALB83/oJrq7L/jwt/wDrkv8AIV14fZmVTcnoooroMwooooAwPFMUkMFprEKM76ZN50iIMs8LKVkAHfAO/HcoBV+KWOeFJonWSORQyOpyGB5BB9K0K5l9Kv8AQJXk0aEXenOxZtOLhHhJ5JhY/Lg/3GIA7MBxWFam5aouEraM2aKxf+Eq0iLi+uH058cpqEbW+PxcBT9QSKj/AOE48Jf9DRon/gwi/wDiq5eV9jW6N6isH/hOPCX/AENOif8Agwi/+Ko/4Tjwl/0NOif+DCL/AOKpcr7BdG9RWD/wnHhL/oadE/8ABhF/8VR/wnHhL/oadE/8GEX/AMVRyvsF0b1FYP8AwnHhL/oadE/8GEX/AMVR/wAJx4S/6GnRP/BhF/8AFUcr7BdG9RWD/wAJx4S/6GnRP/BhF/8AFUf8Jx4S/wChp0T/AMGEX/xVHK+wXRvUVg/8Jx4S/wChp0T/AMGEX/xVH/CceEv+hp0T/wAGEX/xVHK+wXRvUVg/8Jx4S/6GnRP/AAYRf/FUf8Jx4S/6GnRP/BhF/wDFUcr7BdG9RWD/AMJx4S/6GnRP/BhF/wDFUf8ACceEv+hp0T/wYRf/ABVHK+wXRvUVg/8ACceEv+hp0T/wYRf/ABVH/CceEv8AoadE/wDBhF/8VRyvsF0b1FYP/CceEv8AoadE/wDBhF/8VR/wnHhL/oadE/8ABhF/8VRyvsF0b1FYP/CceEv+hp0T/wAGEX/xVH/CceEv+hp0T/wYRf8AxVHK+wXRvUVg/wDCceEv+hp0T/wYRf8AxVH/AAnHhL/oadE/8GEX/wAVRyvsF0b1FYP/AAnHhL/oadE/8GEX/wAVR/wnHhL/AKGnRP8AwYRf/FUcr7BdG9RWF/wmOgScWmox379ksFa5Y/hGGpfsmq+Ix5d3BJpWkt9+NnH2m5X+6dpIiU98EsRx8tXGnKTE5JE3h/8A4merX+uDm2ZVs7NuzohJeQezOSB6iMEda6OmRRRwQpDDGscUahURBgKBwAB2FPruiuVWRi3d3CiiimIKKKKACiiigArnG/dePLrd/wAvOmQbPfypZd3/AKOX866Ouf8AE8b2v2PXYUZ205m89VGS1u4AkwO+MI+O/l471FSPNFoqLszTopsciTRJLG6vG4DKynIYHoQadXnm4UUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFZ2s6kdMsd0MYmvJmENpBn/Wyn7o+nUk9lBPamld2QDfDH7yfXrof6ufU22n18uKKI/+PRsK6CqGi6aNI0e2sRIZWiX95KRgySE7nc+7MSfxq/XoxVkkc7d2FFFFMQUUUUAFFFFAHL6J/wAS29vdBl+VoJGuLXP8dvIxYY/3GJTHYBT/ABCtuodY0aPVoonWZ7a9t2L211GAWiY9eDwynoVPB+oBGWNXvtO/da3psybf+XuyiaeB/fCgunuGGB/ePWuSrSd7o1jJWszaorDfxn4XjA83xFpURP8ADLeRofyJBpv/AAnHhL/oadE/8GEX/wAVWHK+xd0b1FYP/CceEv8AoadE/wDBhF/8VR/wnHhL/oadE/8ABhF/8VRyvsF0b1FYP/CceEv+hp0T/wAGEX/xVH/CceEv+hp0T/wYRf8AxVHK+wXRvUVg/wDCceEv+hp0T/wYRf8AxVH/AAnHhL/oadE/8GEX/wAVRyvsF0b1FYP/AAnHhL/oadE/8GEX/wAVR/wnHhL/AKGnRP8AwYRf/FUcr7BdG9RWD/wnHhL/AKGnRP8AwYRf/FUf8Jx4S/6GnRP/AAYRf/FUcr7BdG9RWD/wnHhL/oadE/8ABhF/8VR/wnHhL/oadE/8GEX/AMVRyvsF0b1FYP8AwnHhL/oadE/8GEX/AMVR/wAJx4S/6GnRP/BhF/8AFUcr7BdG9RWD/wAJx4S/6GnRP/BhF/8AFUf8Jx4S/wChp0T/AMGEX/xVHK+wXRvUVg/8Jx4S/wChp0T/AMGEX/xVH/CceEv+hp0T/wAGEX/xVHK+wXRvUVg/8Jx4S/6GnRP/AAYRf/FUf8Jx4S/6GnRP/BhF/wDFUcr7BdG9RWD/AMJx4S/6GnRP/BhF/wDFUf8ACceEv+hp0T/wYRf/ABVHK+wXRvUVg/8ACceEv+hp0T/wYRf/ABVH/CceEv8AoadE/wDBhF/8VRyvsF0b1FYP/CceEv8AoadE/wDBhF/8VR/wnHhL/oadE/8ABhF/8VRyvsF0b1FYP/CceEv+hp0T/wAGEX/xVH/CceEv+hp0T/wYRf8AxVHK+wXRvUVhf8JjoEnFpqMd+/ZLBWuWP4RhqX7JqviMeXdwSaVpLffjZx9puV/unaSIlPfBLEcfLVxpykxOSRN4f/4merX+uDm2ZVs7NuzohJeQezOSB6iMEda6OmRRRwQpDDGscUahURBgKBwAB2FPruirKyMW7nBeDP8AkRfD3/YMtv8A0UtblYfgz/kRfD3/AGDLb/0Utblda2OV7hRRRTEFFFFABXk/iD4Z+ILTxdceJfBerRWlxcszzRSnHzMctjghgTzgjg16xRSauNOx5Bp/wz8U674ms9Z8b6vBcpaMrJBDyWwchcBVVRnrjJP610HxE8Dan4t1bQbuwntI49PkZpRO7AsCyH5cKf7p64rv6KXKrWHzO9zh/if4N1HxroVpY6bNaxSw3ImY3LMqkbWHG1TzyKZ40+HY8V6FpsUd2trqunIBDOMlTwMg45xkAg9R6V3dFDimLmaPH7rwB4/8V/ZtP8Va/af2XA4Zvs4y8hHf7oycdz0znBrZ+IXw6vPEuk6Hp2hvZ20Gm7lC3DsBswoUDCnP3e9ej0UcqHzMoa3pFtr2iXmlXYPkXUZjYr1X0I9wcH8K8v0j4fePtLs5fDkXiKzi8PyMwaRFJlCN94ICvyk5PGcDJwa9foocUxJtHmvgr4Z3Gj+Etb0HXZraaLUm4a1dm2jbgH5lHIOCPpWPa+BfiRo+kS+HNL13Tv7JcsEmJZJI1Y5IHykrnJPBPXg17FRRyofMzgtc8C6nf/DL/hGYtVN5flkd7u/lchiGDHn5iB2ArovDWiyaT4PsNGvvKleC2EE3lklG4wcZAOPwrbop2QrnkNn4B8c+D57618I6vYnTLtyyrdZ3xHpn7pGQMDI646V1nhzwRcWfg680XX9WudUkvkZZneVmWIEYxHu6Y659e1dlRSUUhuTZ4/pHw+8faHDPoNh4is4tCmdsy7SZUVvvbQV+ViOwbHfOa6L4Y+BdQ8EQ6rFfT2sy3UqNEYHZiFXd97KjnkdK76ihRSBybCiiiqJCiiigAooooAoW/wDyUPS/+wVff+jbSuwrj7f/AJKHpf8A2Cr7/wBG2ldhXPP4joh8IUUUVBQUUUUAFcz8Q7Ka/wDAWrx2y7riKIXMSjqXiYSAD6lMV01BAIIIyDQBk6few6lptrf27boLmJZoz6qwBH86s1x/hxv+EW1qfwfdHbbkvc6NI3SSAnLQg/3oySMf3SD2rsK86ceV2OhO6uFct4l0PUP7Tg8RaCI31OCMwzWsjbUvIM52Fv4WByVb3IPBrqaKSbTugaTVmcOnxA0GIiLVZpdHux9631KIwsD7MflYe4JqX/hP/CH/AEMmmf8AgQtdi6LIpV1DKeoIyKh+w2n/AD6wf9+xWvtV2MfYLucp/wAJ/wCEP+hk0z/wIWj/AIT/AMIf9DJpn/gQtdX9htP+fWD/AL9ij7Daf8+sH/fsU/arsHsPM5T/AIT/AMIf9DJpn/gQtH/Cf+EP+hk0z/wIWur+w2n/AD6wf9+xR9htP+fWD/v2KParsHsPM5T/AIT/AMIf9DJpn/gQtH/Cf+EP+hk0z/wIWur+w2n/AD6wf9+xR9htP+fWD/v2KParsHsPM5T/AIT/AMIf9DJpn/gQtH/Cf+EP+hk0z/wIWur+w2n/AD6wf9+xR9htP+fWD/v2KParsHsPM5T/AIT/AMIf9DJpn/gQtH/Cf+EP+hk0z/wIWur+w2n/AD6wf9+xR9htP+fWD/v2KParsHsPM5T/AIT/AMIf9DJpn/gQtH/Cf+EP+hk0z/wIWur+w2n/AD6wf9+xR9htP+fWD/v2KParsHsPM5T/AIT/AMIf9DJpn/gQtH/Cf+EP+hk0z/wIWur+w2n/AD6wf9+xR9htP+fWD/v2KParsHsPM5T/AIT/AMIf9DJpn/gQtH/Cf+EP+hk0z/wIWur+w2n/AD6wf9+xR9htP+fWD/v2KParsHsPM5T/AIT/AMIf9DJpn/gQtH/Cf+EP+hk0z/wIWur+w2n/AD6wf9+xR9htP+fWD/v2KParsHsPM5T/AIT/AMIf9DJpn/gQtH/Cf+EP+hk0z/wIWur+w2n/AD6wf9+xR9htP+fWD/v2KParsHsPM5T/AIT/AMIf9DJpn/gQtH/Cf+EP+hk0z/wIWur+w2n/AD6wf9+xR9htP+fWD/v2KParsHsPM5T/AIT/AMIf9DJpn/gQtH/Cf+EP+hk0z/wIWur+w2n/AD6wf9+xR9htP+fWD/v2KParsHsPM5T/AIT/AMIf9DJpn/gQtH/Cf+EP+hk0z/wIWur+w2n/AD6wf9+xR9htP+fWD/v2KParsHsPM5T/AIT/AMIf9DJpn/gQtH/Cf+EP+hk0z/wIWur+w2n/AD6wf9+xR9htP+fWD/v2KParsHsPM46Tx/okxMOjSS61enhLfTozISf9p/uqPckVq+FdDvLJ73V9YMbavqJUypGcpbxrnZEp7gZJJ7kmujRFjUKihVHQAYFLUSqXVkXCmo6hRRRWZoYPjXUTpfgvV7lQTL9maKFR1aR/kQD6swrc0PThpHh/TdNBB+yWsUGR32KF/pXJSH/hMPGcFnD8+j6DOJ7uQfdmuwP3cQPfZne3vtFd5XbQjaN31MZu7CiiitiAooooA47xp/yMHg3/ALCr/wDpNNXSVzfjj93q3g+c/dXWQh/4FBMB+tdJXJiPiRrT2Ciiiuc0OMm/0T4qy7vlW/0dDGT/ABNDK24D8JVNdHWZ4v0W71G1tdQ0rYNX0yXz7UOcCUEYeInsHXj2OD2pNC16z1+yM9sWSWM7Li2lG2W3k7o69QR+vaumDvE5asbSualFFFUZBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAjusaM7sFVRkk9AKxPhujf8IPaXLKVN5LPdgH+7LM7qf++WWs7XryTxJeSeE9HkJZ/l1S7j+7aQn7yZ6eY4yAOoBJPSu5t7eK0tora3jWOGFBHGi9FUDAA/Cs6r0sdFGPUkooorA3Oa+If/ACTrxF/2D5v/AEE11dl/x4W//XJf5CuR+I8gj+HHiAn+KydB7lhgfqa7G3jMVtFGeqIF/IV14fZmVTckoooroMwooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigDl20+98NyM2m27XmkMSxsoyBLak9fKzwyd9nBH8ORhRZsfEGlahIYre+i88fet5D5cqezRthl/EVv1VvtM0/U4xHf2NtdoOizxLIPyIrGdGMndFqbQyis0+BfCDEk+FdDJPc6fF/8AE0n/AAgng/8A6FTQ/wDwXQ//ABNR9X8x+08jTorM/wCEE8H/APQqaH/4Lof/AImj/hBPB/8A0Kmh/wDguh/+Jo+r+Ye08jTorM/4QTwf/wBCpof/AILof/iaP+EE8H/9Cpof/guh/wDiaPq/mHtPI06KzP8AhBPB/wD0Kmh/+C6H/wCJo/4QTwf/ANCpof8A4Lof/iaPq/mHtPI06KzP+EE8H/8AQqaH/wCC6H/4mj/hBPB//QqaH/4Lof8A4mj6v5h7TyNOisz/AIQTwf8A9Cpof/guh/8AiaP+EE8H/wDQqaH/AOC6H/4mj6v5h7TyNOisz/hBPB//AEKmh/8Aguh/+Jo/4QTwf/0Kmh/+C6H/AOJo+r+Ye08jTorM/wCEE8H/APQqaH/4Lof/AImj/hBPB/8A0Kmh/wDguh/+Jo+r+Ye08jTorM/4QTwf/wBCpof/AILof/iaP+EE8H/9Cpof/guh/wDiaPq/mHtPI06KzP8AhBPB/wD0Kmh/+C6H/wCJo/4QTwf/ANCpof8A4Lof/iaPq/mHtPI06KzP+EE8H/8AQqaH/wCC6H/4mj/hBPB//QqaH/4Lof8A4mj6v5h7TyNOisz/AIQTwf8A9Cpof/guh/8AiaP+EE8H/wDQqaH/AOC6H/4mj6v5h7TyNOisz/hBPB//AEKmh/8Aguh/+Jo/4QTwf/0Kmh/+C6H/AOJo+r+Ye08jTorM/wCEE8H/APQqaH/4Lof/AImj/hBPB/8A0Kmh/wDguh/+Jo+r+Ye08jTorM/4QTwf/wBCpof/AILof/iaP+EE8H/9Cpof/guh/wDiaPq/mHtPI06bJIkSF5HVEHVmOAKzv+EE8H/9Cpof/guh/wDiaki8F+Fbdw8PhnRo2HIKWEQP6LR9X8w9p5FR/E1nNI0GkK+rXQONlmQyIf8Abk+4n0Jz6A1b0nRZ0vDqurSxz6kVKRrHnyrVD1SPPJJwMueTjoBgDajjSKNY40VEUYCqMAD6U6tYUox1JcmwooorQkKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooA4LwZ/yIvh7/sGW3/opa3Kw/Bn/ACIvh7/sGW3/AKKWtyupbHM9wooopiCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigChb/8lD0v/sFX3/o20rsK4+3/AOSh6X/2Cr7/ANG2ldhXPP4joh8IUUUVBQUUUUAFFFFAGT4h8PWXiTTfsl35kbo4lt7iFtstvIPuujdiP16GuYXxHqfhbFr4wgZrdeI9btYi0Eg7GVRkxN6/w+hFd7QQCCCMg1E4Ke41JoybLULLUrZbiwu4LqBukkEgdT+IqzWLffDzwnf3DXMmiwQ3DcmW0ZrdyfUmMqTVU/DTQc8XGtKPQavc4H/j9YPD9mae0Okorm/+FaaF/wA/Wt/+De4/+Lo/4VpoX/P1rf8A4N7j/wCLo+rvuHtDpKK5v/hWmhf8/Wt/+De4/wDi6P8AhWmhf8/Wt/8Ag3uP/i6Pq77h7Q6Siub/AOFaaF/z9a3/AODe4/8Ai6P+FaaF/wA/Wt/+De4/+Lo+rvuHtDpKK5v/AIVpoX/P1rf/AIN7j/4uj/hWmhf8/Wt/+De4/wDi6Pq77h7Q6Siub/4VpoX/AD9a3/4N7j/4uj/hWmhf8/Wt/wDg3uP/AIuj6u+4e0Okorm/+FaaF/z9a3/4N7j/AOLo/wCFaaF/z9a3/wCDe4/+Lo+rvuHtDpKK5v8A4VpoX/P1rf8A4N7j/wCLo/4VpoX/AD9a3/4N7j/4uj6u+4e0Okorm/8AhWmhf8/Wt/8Ag3uP/i6P+FaaF/z9a3/4N7j/AOLo+rvuHtDpKK5v/hWmhf8AP1rf/g3uP/i6P+FaaF/z9a3/AODe4/8Ai6Pq77h7Q6Siub/4VpoX/P1rf/g3uP8A4uj/AIVpoX/P1rf/AIN7j/4uj6u+4e0Okorm/wDhWmhf8/Wt/wDg3uP/AIuj/hWmhf8AP1rf/g3uP/i6Pq77h7Q6Siub/wCFaaF/z9a3/wCDe4/+Lo/4VpoX/P1rf/g3uP8A4uj6u+4e0Okorm/+FaaF/wA/Wt/+De4/+Lo/4VpoX/P1rf8A4N7j/wCLo+rvuHtDpKK5v/hWmhf8/Wt/+De4/wDi6P8AhWmhf8/Wt/8Ag3uP/i6Pq77h7Q6Siub/AOFaaF/z9a3/AODe4/8Ai6P+FaaF/wA/Wt/+De4/+Lo+rvuHtDpKK5v/AIVpoX/P1rf/AIN7j/4ulHw00D+OfWXU9VbV7nH/AKHR9XfcPaGrqmtaZotsbjVNQtrOIfxTyBc/TPU+wrnG1DW/Gn+j6FFc6Vo78S6vcRmOWVfS3jbkZ/vsAB2BNbul+BfC+j3IubPRbUXIORcSgyyj6O5LD866GrhQS1epLm3sUdH0ix0LS4NN06AQ2sK4VRySe5J6kk8knqavUUVuQFFFFABRRRQBynxFtpZPB819bIXudLmi1GJR1PkuHYfigYfjWzbXMV5aQ3UDh4ZkWSNx0ZSMg/ka0XRZEZHUMrDBBGQRXCeFnbw5qlx4Nu2ISANPpMjH/XWpP3Ae7Rk7SPTaa568bq5pTetjr6KKK5DUK5/WvB+m6xeLqCvcWGqKu1b6xk8uUj+63BDj2YGugopptaoTV9zjT4f8ZQDbB4rsp17G60sbvxKSAE/gKT+xvHX/AEMOi/8Agsf/AOO12dFX7SRPs49jjP7G8df9DDov/gsf/wCO0f2N46/6GHRf/BY//wAdrs6KPaSF7OPY4z+xvHX/AEMOi/8Agsf/AOO0f2N46/6GHRf/AAWP/wDHa7Oij2kg9nHscZ/Y3jr/AKGHRf8AwWP/APHaP7G8df8AQw6L/wCCx/8A47XZ0Ue0kHs49jjP7G8df9DDov8A4LH/APjtH9jeOv8AoYdF/wDBY/8A8drs6KPaSD2cexxn9jeOv+hh0X/wWP8A/HaP7G8df9DDov8A4LH/APjtdnRR7SQezj2OM/sbx1/0MOi/+Cx//jtH9jeOv+hh0X/wWP8A/Ha7Oij2kg9nHscZ/Y3jr/oYdF/8Fj//AB2j+xvHX/Qw6L/4LH/+O12dFHtJB7OPY4z+xvHX/Qw6L/4LH/8AjtH9jeOv+hh0X/wWP/8AHa7Oij2kg9nHscZ/Y3jr/oYdF/8ABY//AMdo/sbx1/0MOi/+Cx//AI7XZ0Ue0kHs49jjP7G8df8AQw6N/wCCx/8A47Tj4Q1vUPk1nxbcvbn70GnWy2gb2L5Z8fQiuxoo9pIfs49ilpWkafodgljplpFa2ychIx1Pck9ST6nmrtFFQWFFFNd0ijaSRlRFBZmY4AA6kmkBy3jf/iYDRvDyfM+qahEJFH/PCIiWQ/kgH/AhXcVxHhFH8Ra7deMJkItDGbPSVYYzADl5sH/nowGP9lR6129d9KPLEwk7sKKKK0JCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigDg/CQ8nw3a2LcSafusZB6GImP9QoP0IrbrO12E+HtZk1pQf7Lvdq3+P+XeUAKsx/2SAFY9tqnpuNaIIIyDkGuiDujnkrMKKKKskKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKpapqUemWokMbzTyMI7e3j+/NIeiL7+/QAEngGkMTS1+1ePHkXlbDTjG57bp5FOPriDP4j1rr6x/DekSaRprfanWTULqQ3F3IvQyEAYH+yqhVHso71sVzSd3c6IqysFFFFIYUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAVieJvDkPiKwjQTPaX9s/nWV7EPnt5R3HqD0KngjituigDi9H8TyrfroXiOFNP1sDCc/ubwD+OFj191PzD9a6em6voum6/YNY6rZxXVu3OyQfdPqp6qfcYNcz/wjXifRONA16O9tR92z1pWcqPRZ1+b/AL6DVzTodYmin3Ooorl/7b8W2/y3XgiWUjgvZajDIp/Byh/Smf8ACU67/wBCJrf/AH9t/wD45WPsp9i+dHV0Vyn/AAlOu/8AQia3/wB/bf8A+OUf8JTrv/Qia3/39t//AI5R7KfYOZHV0Vyn/CU67/0Imt/9/bf/AOOUf8JTrv8A0Imt/wDf23/+OUeyn2DmR1dFcp/wlOu/9CJrf/f23/8AjlH/AAlOu/8AQia3/wB/bf8A+OUeyn2DmR1dFcp/wlOu/wDQia3/AN/bf/45R/wlOu/9CJrf/f23/wDjlHsp9g5kdXRXKf8ACU67/wBCJrf/AH9t/wD45R/wlOu/9CJrf/f23/8AjlHsp9g5kdXRXKf8JTrv/Qia3/39t/8A45R/wlOu/wDQia3/AN/bf/45R7KfYOZHV0Vyn/CU67/0Imt/9/bf/wCOUf8ACU67/wBCJrf/AH9t/wD45R7KfYOZHV0Vyn/CU67/ANCJrf8A39t//jlH/CU67/0Imt/9/bf/AOOUeyn2DmR1dFcp/wAJTrv/AEImt/8Af23/APjlH/CU67/0Imt/9/bf/wCOUeyn2DmR1dFcp/wlOu/9CJrf/f23/wDjlH/CU67/ANCJrf8A39t//jlHsp9g5kdXRXKf8JTrv/Qia3/39t//AI5R/wAJTrv/AEImt/8Af23/APjlHsp9g5kdXRXKjxPr78J4E1nd/tT2yj8/Mp32vx1qHyWvh/TdKH/PXUL3ziPokQwT/wADpqjN9A50dHdXVvZW0lzdTxwQRrueWVgqqPUk9K48C6+I0ixxpNa+EgQZZXBSTU/9lR1WH1bq3QcZNadr4DjurmO88T6lNrtxG26OGVBHaxHsVhHBPuxY114AAAAwB2renRUdWZynfYbHGkMSRRIqRooVVUYCgdAB6U6iityAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKAGuiSRtHIqujAhlYZBB7EVxlxpN94XJfTYZb7ReptE+aa0H/TMfxx/7H3h/DkYUdrRTTa2E0nucrY39pqVqtzZTpPC2RuQ9COoPoR3B5FWafqnhSzvrpr+0ll03UmHzXVrgeZ6CRD8sg+oyOxFY8s+t6Pxq2mm7tx/y+6ahcY9Wh5df+A7x7ito1E9zJwa2NWiqlhqdjqkJlsbuG4RThjG4JU+hHUH2NW60MwooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKy7nX7GC6azhaS9vh1tLNDLIP94DhB7sQPepYdH17WMG+mGjWZ6wW7CS5cejSfdT6LuPowqXNIpRbGXusJBdCws4HvtTYZW0hPKg/wATt0jX3P4ZPFaeieHXtLr+1NVmS61VlKgoP3Vsh6pED+rHlsdhgDS0vR7DRbX7Pp9skMZO5yMlpG/vMx5Zvckmr1YSm5G0YpBRRRUlBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQBk6n4Z0fV5hcXdkn2oDC3ULGKZR7SIQw+mayZPDWtWXOma6LiMdINThDn6CRNpH1YOa6yimm1sJpPc4p7vW7L/kIeHbhlHWbT5VuE/I7ZPyQ1GninRTKIZr9LSY9Ib1WtpD/wABkCn9K7mmTQxXETRTRJJG3VHUEH8DVqoyHTRz6OsiB0YMp5BU5Bp1Ok8E+G3YvHpEFq55L2WbZifXdGVNQHwYsZzZ69rVsOy+ek4/8jI5/WqVVE+zZLRVVvDviCI/6P4itpB6Xenbz+aSJ/KmHTvFkXT+xbn6vLB/R6r2kRcjLtFUPL8VIf3miae3/XHUy3/oUS0wz+IV+94Ynb/rneQH+bCjniLkkaVFZovNawd3hPVPbbPaHP8A5GFJ9u1s9PB+rg/7U9nj9JzT549w5WadFZn2vXj08J6gP966tR/KU0eb4mf7nhnb/wBdb+Mf+g7qOePcOSRp0Vn/AGfxdIPl0rR4veTUpGP5CH+tPXR/Fkv37/Rbb/dtZZ//AGolL2kR8ki7RVVfCury/wDH14quUHcWdpDH/wChiQ1KPA2mSf8AH7eare+01/Iqn6rGVU/iKXtEP2bI7zUbHTo/Mvry3tU/vTyqg/MmqCeJLO64023vtSJ6GztXdD/20ICf+PV0dh4W0DS5PNsdGsIJf+eqQLvP1bGT+da1S6r6FKmjjkg8U3/+q0+z0yM/x3s3nSD/ALZx/Kf+/lWY/BiXPzazqt7qPrCr/Z4P++I8Fh7MzV1FFQ5NlKKRWsdPs9MtVtbC0gtbdfuxQRhFH4CrNFFSUFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFAH/2Q==)" + ], + "metadata": { + "id": "cAgnGkn3GFVb" + } + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "collapsed": true, + "id": "oCgmuQtdrSkG" + }, + "outputs": [], + "source": [ + "!pip install timesfm[torch]\n", + "!pip install 'apache_beam[interactive,gcp,test] == 2.67.0'\n", + "!pip install google-generativeai" + ] + }, + { + "cell_type": "markdown", + "source": [ + "# Ordered Sliding Window![preprocessing.jpg](data:image/jpeg;base64,/9j/4AAQSkZJRgABAgAAAQABAAD/2wBDAAgGBgcGBQgHBwcJCQgKDBQNDAsLDBkSEw8UHRofHh0aHBwgJC4nICIsIxwcKDcpLDAxNDQ0Hyc5PTgyPC4zNDL/2wBDAQkJCQwLDBgNDRgyIRwhMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjL/wAARCAIPCGcDASIAAhEBAxEB/8QAHwAAAQUBAQEBAQEAAAAAAAAAAAECAwQFBgcICQoL/8QAtRAAAgEDAwIEAwUFBAQAAAF9AQIDAAQRBRIhMUEGE1FhByJxFDKBkaEII0KxwRVS0fAkM2JyggkKFhcYGRolJicoKSo0NTY3ODk6Q0RFRkdISUpTVFVWV1hZWmNkZWZnaGlqc3R1dnd4eXqDhIWGh4iJipKTlJWWl5iZmqKjpKWmp6ipqrKztLW2t7i5usLDxMXGx8jJytLT1NXW19jZ2uHi4+Tl5ufo6erx8vP09fb3+Pn6/8QAHwEAAwEBAQEBAQEBAQAAAAAAAAECAwQFBgcICQoL/8QAtREAAgECBAQDBAcFBAQAAQJ3AAECAxEEBSExBhJBUQdhcRMiMoEIFEKRobHBCSMzUvAVYnLRChYkNOEl8RcYGRomJygpKjU2Nzg5OkNERUZHSElKU1RVVldYWVpjZGVmZ2hpanN0dXZ3eHl6goOEhYaHiImKkpOUlZaXmJmaoqOkpaanqKmqsrO0tba3uLm6wsPExcbHyMnK0tPU1dbX2Nna4uPk5ebn6Onq8vP09fb3+Pn6/9oADAMBAAIRAxEAPwD3+iiigAooooAKKKKACiiigAooooAKKKKACiqGpa3pWjoG1LUbW0Dfd8+ZULfQE8/hWQ/jW1l40zTNU1A9mjt/JT/vuYoCPdc0WuF7HTUVyT6v4ou/9TZ6Xpyno00j3T/iqhAP++jUDWOr3X/H74kv2U9Y7VI7dPzCl/8Ax6rUJMlzR2bMFUsxAA5JPasW58YeHLSUxS63Y+cP+WMcyvJ/3yuT+lYB8K6LIQ11Z/bmHO6/le5OfrIWrUgtoLWMR28McKDosaBR+QqlSZPtEK3jawf/AI89P1e8P/TOwkjB+jShAfzqNvE+sS/8evhiVPT7bexx/wDovzKsUU/Zon2jKTan4rm+7Fotp9TLcY/9F5phHiab/XeIIYj/ANOmnqn/AKMZ60KKr2cRc8jLOmajJ/x8eJ9Yl9gYYv8A0XGpph8PW8n+u1DWZfZtWuQD+CuB+la9FPlXYXM+5jHwrorf62y87/rvK8mf++iaE8I+GkbcPD+l7/7xtIyfzxWzRTshXZnx6Do8X+r0qxT/AHbdB/SrC2NogwtrAv0jAqxRRYLjFijTG2NVx0wMYp9FFMQVG0ETghokIPXKipKKAK76fZSDD2kDf70YP9KrSaBo0v8ArNIsH/3rZD/StGilYdzFPg/w0W3Dw/pit/eS0RT+YFOHhbR1/wBVbPD/ANcLiSL/ANBYVsUUWQXZkjw/FH/qdT1qL/uK3Dgfg7kU8adqkf8AqPFGrxj+64gkH/j8RP61p0UuVdh8z7meD4oh/wBTr1pL/wBfenbv/QHSpBq3iuH71ro139JpbfP/AI7JVyil7OI+eRXHinVIv+PrwxdP6mzuoZB/4+yH9KlXxvpicXdrqlmf+m1hKyj6ugZR+dPoqfZIftGWbPxX4ev5PKtdbsJJv+eQuFDj6qTkflWx1rl7qytb2Py7u2huE/uyxhx+RrNXwtpEPNlBLp57fYJ5LYD8I2A/Sk6TK9ojuqK4tbTW7X/jz8SXTAdI72COdB+ICufxap01vxNa8XGm6dfoOr2s7QP+COGH/j9S4SRSmmdbRXNJ42sI+NSstS01u5uLYug+skW9B+JFbOn6tp2rQmbTr+1vIx1a3lVwPrg1Fii5RRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRUF5fWmnWr3V7cw21ugy0szhFX6k8Vzkvi64vvl0DS5blD0vLwm3g+qggu/4Lg/3qaTewm0tzqqxtR8V6LplwbWa9WW8H/LrbK0034ogJH1IArBk0zUNT51rWLidT1tbTNtAPbCnew/3nIPpV6ysLPTbcW9jaw20I/ghQIPyFWqb6kOougyTxFrt9xp2jR2cZ6TalL831EUec/i6n2qrJpuo33Oq69fTKesNmfskf4bPn/NzWrRWihFEObZQsdE0vTHZ7Kwt4ZW+9KqDe31bqfxNX6KKogKKKKYBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABWde6DpWoTCe5sIWuB92dV2Sr9HXDD8DWjRSAy47HVrDnTNfugo6Q34+1R/mSJP8Ax+rUfiXWbLjU9D+0RjrPpkof8TG+1h9FLmrVFS4JlqbRa0zxPo2rTfZ7S+T7UBk2swMUwHvG4DD8q165S+02x1OEQ31pDcxg5AlQNg+oz0PuKqRWGqaZzo+szLGOlrf5uYvoGJEi/wDfWB6Vm6b6Fqoup21FctF4weywuv6ZLYgdbuAm4t/qWADJ9WUAetdHa3dvfWyXNpcRXEEgyksThlYeoI4NQ1bctO5NRRRSGFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFUNR1zSdHXdqeqWVkMZ/0idY8/maAL9Fcg3xQ8GgkRayLjHX7Lbyz/wDoCmm/8LM8PnlIdZkHqukXOP8A0CldAdjRXHf8LL0L/n11v/wUXH/xFH/Cy9C/59db/wDBRcf/ABFF0FjsaK47/hZehf8APrrf/gouP/iKP+Fl6F/z663/AOCi4/8AiKLoLHY0Vx3/AAsvQv8An11v/wAFFx/8RR/wsvQv+fXW/wDwUXH/AMRRdBY7GiuO/wCFl6F/z663/wCCi4/+Io/4WXoX/Prrf/gouP8A4ii6Cx2NFcd/wsvQv+fXW/8AwUXH/wARR/wsvQv+fXW//BRcf/EUXQWOxorjv+Fl6F/z663/AOCi4/8AiKP+Fl6F/wA+ut/+Ci4/+IougsdjRXHf8LL0L/n11v8A8FFx/wDEUf8ACy9C/wCfXW//AAUXH/xFF0FjsaK47/hZehf8+ut/+Ci4/wDiKP8AhZehf8+ut/8AgouP/iKLoLHY0Vx3/Cy9C/59db/8FFx/8RR/wsvQv+fXW/8AwUXH/wARRdBY7GiuO/4WXoX/AD663/4KLj/4ij/hZehf8+ut/wDgouP/AIii6Cx2NFcd/wALL0L/AJ9db/8ABRcf/EUf8LL0L/n11v8A8FFx/wDEUXQWOxorjv8AhZehf8+ut/8AgouP/iKP+Fl6F/z663/4KLj/AOIougsdjRXHf8LL0L/n11v/AMFFx/8AEUf8LL0L/n11v/wUXH/xFF0FjsaK47/hZehf8+ut/wDgouP/AIij/hZehf8APrrf/gouP/iKLoLHY0Vx3/Cy9C/59db/APBRcf8AxFH/AAsvQv8An11v/wAFFx/8RRdBY7GiuO/4WXoX/Prrf/gouP8A4ij/AIWXoX/Prrf/AIKLj/4ii6Cx2NFcd/wsvQv+fXW//BRcf/EUf8LL0L/n11v/AMFFx/8AEUXQWOxorjv+Fl6F/wA+ut/+Ci4/+Io/4WXoX/Prrf8A4KLj/wCIougsdjRXHf8ACy9C/wCfXW//AAUXH/xFH/Cy9C/59db/APBRcf8AxFF0FjsaK47/AIWXoX/Prrf/AIKLj/4ij/hZehf8+ut/+Ci4/wDiKLoLHY0Vx3/Cy9C/59db/wDBRcf/ABFH/Cy9C/59db/8FFx/8RRdBY7GiuO/4WXoX/Prrf8A4KLj/wCIo/4WXoX/AD663/4KLj/4ii6Cx2NFcd/wsvQv+fXW/wDwUXH/AMRR/wALL0L/AJ9db/8ABRcf/EUXQWOxorjv+Fl6F/z663/4KLj/AOIo/wCFl6F/z663/wCCi4/+IougsdjRXHf8LL0L/n11v/wUXH/xFH/Cy9C/59db/wDBRcf/ABFF0FjsaK47/hZehf8APrrf/gouP/iKP+Fl6F/z663/AOCi4/8AiKLoLHY0Vx3/AAsvQv8An11v/wAFFx/8RR/wsvQv+fXW/wDwUXH/AMRRdBY7GiuO/wCFl6F/z663/wCCi4/+Io/4WXoX/Prrf/gouP8A4ii6Cx2NFcd/wsvQv+fXW/8AwUXH/wARR/wsvQv+fXW//BRcf/EUXQWOxorjv+Fl6F/z663/AOCi4/8AiKP+Fl6F/wA+ut/+Ci4/+IougsdjRXHf8LL0L/n11v8A8FFx/wDEUf8ACy9C/wCfXW//AAUXH/xFF0FjsaK47/hZehf8+ut/+Ci4/wDiKP8AhZehf8+ut/8AgouP/iKLoLHY0Vx3/Cy9C/59db/8FFx/8RR/wsvQv+fXW/8AwUXH/wARRdBY7GiuO/4WXoX/AD663/4KLj/4ij/hZehf8+ut/wDgouP/AIii6Cx2NFcd/wALL0L/AJ9db/8ABRcf/EUf8LL0L/n11v8A8FFx/wDEUXQWOxorj1+JWhE4NtrQ9zpFz/8AEU8/EzwmhxNqE9v6m5sLiID6lkAougOtorI03xV4f1hlTTdb067duiQ3KM3/AHyDmtemAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFRzzw20TS3EscUa9XkYKB+JoAkorl7r4j+DbOXypPEVhJJ/ct5POb8kzVb/hZ3hliRC+pz/9ctKuTn/yHRcDsaK47/hZehf8+ut/+Ci4/wDiKP8AhZehf8+ut/8AgouP/iKV0FjsaK47/hZehf8APrrf/gouP/iKP+Fl6F/z663/AOCi4/8AiKLoLHY0Vx3/AAsvQv8An11v/wAFFx/8RR/wsvQv+fXW/wDwUXH/AMRRdBY7GiuO/wCFl6F/z663/wCCi4/+Io/4WXoX/Prrf/gouP8A4ii6Cx2NFcd/wsvQv+fXW/8AwUXH/wARR/wsvQv+fXW//BRcf/EUXQWOxorjv+Fl6F/z663/AOCi4/8AiKP+Fl6F/wA+ut/+Ci4/+IougsdjRXHf8LL0L/n11v8A8FFx/wDEUf8ACy9C/wCfXW//AAUXH/xFF0FjsaK47/hZehf8+ut/+Ci4/wDiKP8AhZehf8+ut/8AgouP/iKLoLHY0Vx3/Cy9C/59db/8FFx/8RR/wsvQv+fXW/8AwUXH/wARRdBY7GiuO/4WXoX/AD663/4KLj/4ij/hZehf8+ut/wDgouP/AIii6Cx2NFcd/wALL0L/AJ9db/8ABRcf/EUf8LL0L/n11v8A8FFx/wDEUXQWOxorjv8AhZehf8+ut/8AgouP/iKP+Fl6F/z663/4KLj/AOIougsdjRXHf8LL0L/n11v/AMFFx/8AEUf8LL0L/n11v/wUXH/xFF0FjsaK47/hZehf8+ut/wDgouP/AIij/hZehf8APrrf/gouP/iKLoLHY0Vx3/Cy9C/59db/APBRcf8AxFH/AAsvQv8An11v/wAFFx/8RRdBY7GiuO/4WXoX/Prrf/gouP8A4ij/AIWXoX/Prrf/AIKLj/4ii6Cx2NFcd/wsvQv+fXW//BRcf/EUf8LL0L/n11v/AMFFx/8AEUXQWOxorjv+Fl6F/wA+ut/+Ci4/+Io/4WXoX/Prrf8A4KLj/wCIougsdjRXHf8ACy9C/wCfXW//AAUXH/xFH/Cy9C/59db/APBRcf8AxFF0FjsaK47/AIWXoX/Prrf/AIKLj/4ij/hZehf8+ut/+Ci4/wDiKLoLHY0Vx3/Cy9C/59db/wDBRcf/ABFH/Cy9C/59db/8FFx/8RRdBY7GiuO/4WXoX/Prrf8A4KLj/wCIo/4WXoX/AD663/4KLj/4ii6Cx2NFcd/wsvQv+fXW/wDwUXH/AMRR/wALL0L/AJ9db/8ABRcf/EUXQWOxorjv+Fl6F/z663/4KLj/AOIo/wCFl6F/z663/wCCi4/+IougsdjRXHf8LL0L/n11v/wUXH/xFH/Cy9C/59db/wDBRcf/ABFF0FjsaK47/hZehf8APrrf/gouP/iKP+Fl6F/z663/AOCi4/8AiKLoLHY0Vx3/AAsvQv8An11v/wAFFx/8RR/wsvQv+fXW/wDwUXH/AMRRdBY7GiuO/wCFl6F/z663/wCCi4/+Io/4WXoX/Prrf/gouP8A4ii6Cx2NFcd/wsvQv+fXW/8AwUXH/wARR/wsvQv+fXW//BRcf/EUXQWOxorjv+Fl6F/z663/AOCi4/8AiKP+Fl6F/wA+ut/+Ci4/+IougsdjRXHf8LL0L/n11v8A8FFx/wDEUf8ACy9C/wCfXW//AAUXH/xFF0FjsaK47/hZehf8+ut/+Ci4/wDiKP8AhZehf8+ut/8AgouP/iKLoLHY0Vx3/Cy9C/59db/8FFx/8RR/wsvQv+fXW/8AwUXH/wARRdBY7GiuO/4WXoX/AD663/4KLj/4ij/hZehf8+ut/wDgouP/AIii6Cx2NFcd/wALL0L/AJ9db/8ABRcf/EUf8LL0L/n11v8A8FFx/wDEUXQWOxorjv8AhZehf8+ut/8AgouP/iKcPiZ4bGPObVIM/wDPXSbof+06LoLHX0Vy9r8R/B11II18Q2UMh/guX8hvykCmujt7mC8gWa2njnib7rxOGU/QimBLRRRQAUUUUAFFFFABRRRQAUUUUAFFFc9q3ilba7fTdJt/7Q1Ncb0DbYrfPQyvzt9doyx9Mc0WuGxtXl7a6faSXd7cRW9vGMvLK4VVHuTXMTeJtS1b5NBtBb2x/wCYhfxsAR6xw8M31baPTdVaLRnubuO/1u5Oo3qHdHuXbDbn/plHyAf9o5b37VrVrGn3MpVOxlQaDbi6S91CWbU79eVuLwhih/2FACx/8BA981q0UVqlbYzbuFFFFMQUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAVkyaDDHcveaXPLpd453PLaYCyH/ppGQUf6kZ9CK1qKTSe407FOHxRf6X8niGzDQD/AJiNijNH9ZI+WT6jcvckV09rdW97bR3NpPHPBINySxOGVh6gjg1iVkvo8llcyXuhXP8AZ907bpI9u63nP/TSP1P95cN7npWUqfY0jU7nb0Vz+keKI7y6XTdStzp+qEErC7bo5wOpifgOPbhh3A610FZGoUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUVzviPxbbaFLFY29vJqOs3Izbadbkb2H95j0RB3Y8fWgDdubmCzt5Li5mjggjXc8kjBVUepJ4Fce/jq61l2h8IaQ+ornB1G6Jgs1+jEbpP+AjHvUMHhK61u5j1HxldJfzId8WmxZFnbnt8p/1jD+834AV1qqFUKoAUDAA7Vzzr20iaKHc5P/hF9b1fL+I/FF26N/y56V/okI9iwJkYfVh9Kvad4H8MaUS1podkJDyZZI/NkP1d8t+tb9Fc7nJ7s0UUhFVUUKihVHAAGAKWiioGFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAZWpeGdC1gH+0dHsbon+KWBS34HGRWUPBj6cd/h3X9U0ojpA0xurf/v3LnH/AStdVRVKclsxNJnML4l8R6GceItGW8tB11DRwz7R6vAfnH1UtXS6RrWma9ZC80q+gu4CcF4mztPoR1B9jzTq5zVvCFtd3x1XS7iTSNa/5/bUD957Sp92RfZufQit4V/5iHT7HYUVyGleLrm11GHRfFdvFY6jKdttdxE/Zb0+iMfuP/sNz6E119dKaaujIKKKKYBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFVdS1Ky0fT5r/UbmO2tIV3SSyNgAf4+3egC1XM61440vSr06bbLPqmrf8APhYJ5ki+7n7sY92IrHNxr/jkExPdaB4eb7rAbL28X1H/ADxQ/wDfR9s10Oj6JpugWIs9Ls4raAHJCDlj6sTyx9zk1jOso6IuMGzEaHxvrxzdX9r4dtG/5YWKi4ucejSuNin/AHVP1pYfh34c89bnULabV7of8ttUne5P5OSo/ACuqormlVlLqaKKRXtbG0sU2WlrBbp/dijCD9KsUUVmUFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAQ3NpbXkfl3VvFOn92VAw/I1zs/w+8OPO1xZWkmlXR58/S5ntWz9EIB/EGuoopqTWwrJnKiDxpoQzZahbeILRf8Al31ACC5x6LMg2sf95R9a0tH8b6Xqd6um3Kz6Xqx/5cL9fLkb/cP3ZB7qTWxVDV9F03XrFrPU7OO5gJyA45U+qkcqfcYNbxrtbkOC6G5RXBC517wRhp5LnXvDy/ecjfe2a+px/rkHf+Ie+K7PTtSstXsIb/T7mK5tZl3RyxNkMP8APbtXTGSkrozaa3LVFFFUIKKKKACiiuL1e9k8T30+lWrsmjWzmO+mRsG6cdYVI/gHRz3Py/3qaV3ZCbsrjr3W7vxHK9pok722lqxSbUk+/MRwUg9uxk/75yfmFixsLXTLRbWzhWGFcnavcnqSepJ6knk1PHGkMSRRIqRoAqqowFA6AD0p1dEYqJhKTYUUUVRIUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAMlmigTfLIkaD+J2AFQf2nYf8/1t/wB/V/xqj4m8N2XivRZNK1B5kt3dXJhYK2VORyQa8D8aeAdI8PePNA0Szlu2tdQaITNK6lxul2HaQoA49jUSbRUUmfRiX1nIGKXUDBBubbIDgepqWKaKdN8UiSIf4kYEV5w/w48M+C/DuvXqy6pJbzWEkVwBIjP5fU7PlA3cd+KPCniLw94W+FP9safDqkmlQTsNlx5ZnLNIFPQhcZP5U79wt2PSqK8zuvjdoEMMM8OnapcW7BfNlSJQsTEZ2Ek4LDuOnvUmpfGnw9Z7ZLS11DULfCmW4giwkZYZCksR83t+tHMg5WekUViL4t0Y+FF8StdhNLaPzPMYc9cbcf3s8Y9a461+Nugy3MX2nTtUtLKZ9kd5LCPLPucE/pmhySFytnplISFBJIAHJJpI5EljWSNldHAZWU5BB6EV5D8eL6+gsdGs1kmi0u5lf7U0X8RG3APrwWIB649qG7K4JXdj1WDVdOupvJt7+1ll/uRzKzfkDVuvBJfAPgvXdJjbwT4hX+2UZWiS5utjPyM5UqGBHUEDqK9ZstQn8MeCYbrxZew/aLSLFzOjFg5BwuOASxG3t1NCfcbXY6KivM7X42aDNcxfadO1Ozspn2R3k0Q8s+5wT+ma6Xxb460zwfaWNzexTzxXjlY2t9rDoDkkkcYNHMhcrOmd1RC7sFVRkknAApsU0U6b4ZUkTONyMCP0rzn/AIWRoPi7T/EemR2mpGxt9OnkmuUVBviAKtsyeGOTjI+uKs/DnUPDml/DyW+0+W9g0mCaR5JNRKF1Ixn7nGOmB1o5k2PlPQKK8xj+N+gtOHfTdVj09pPLF60I2Z/A/wD1/auu8QeNNF8N6BFrN3cGW2n2/ZxAAzTZGRt5x05ySBRzIXKzoKK84tvjHpP2+3ttU0nVdKS4OI57qEBPqec49xmvR6aaewNNbhRRRTEFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFAFa/0+11O1NteQiWMkMOSCrDoykcqw7EcioLHXLvQJo7PXJmuNPdgkGptjMZPASft7CToehweToUyWKOeF4Zo1kjdSro4yGB6gjuKmUVIqMmjpKK4zR76Tw3ewaTdyNJpNw4jsZ3bJt3PSByeqn+An/dP8Oezrnas7M3TvqFFFFIYUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUVgeLPEZ8PabGLWD7Vqt5J9nsLTP+tlPc+iqPmY9gKAKvifxNc2t5FoGgxx3OvXKbwH/wBXaR9DNLjt6Dqx4pfDvhu30CGWQzSXmpXR33l/PzLO/v6KOgUcAUnhnw8uhWkslxMbvVbxvOv71h800nt6KOir0A/GtyuKrV5nZbG0Y2CiiisSwooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAoorznxR8afC/hrUZNPAudQuomKyi1VSkbDqCxI5+mapRctEJtLc9Gorz7wn8YvDPiu/j09DcWN7IcRxXSgCQ+isCRn2OM9q9BocXHRgmnsFFFRzyrBbyTMCVjUsQOuAM1IySiuZ8F+OdM8dWFzeaZBdwx28vlOLlFUk4zxtZuOa6am007MAooopAFFFFABRRXK+EfH2leM7zU7XTre8ifTmVZjcIqhiSwG3axz9w9cdqdna4XOqoqtqF7Hpum3V9MrtFbQvM4QAsVUEnGe/FeW/8ND+Ev8AoHa3/wB+Iv8A45TjCUthNpbnrdFcV4Q+KXhvxpetY6fJcQXgUssF1GEZwOu3BIOPTOa7Wk007ME7hRRXI+OPiJpPgSO1W+iuLi5us+TBbqCxAxkkkgAcgUJNuyBux11FcDc/FjR7HwdF4ivrDUbVZp2t4rSWICV2AySBnG33OP5VseDPGcPjOwnuodMvrEQuFIukC78jOVweRTcGlcLo6aiiipGFFZniLXbXwzoF3rF7HNJb2qhnWEAuQSBwCQO/rTPDPiG08V+HrXW7GOeO2ud+xZ1AcbXKnIBI6qe9OztcLmtRRRSAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigCnqml2OtadNYajbR3FrKMPG4/UehHYjkVz2m6tfeDtQg0bXrmS60m4cR6fqspyyMekM59f7r9+h5rrarahp9pqunz2F9Ak9rOhSSNxwwP+etaU6jgyZRua1FcZ4W1K70bVT4R1md55FQyaXeyHm6gHVGPeROAfUYPrXZ13Jpq6MGrBRRRTAKKKKACiiigAooooAKKKKACiiigAooooAKKKZLLHDE8srqkaKWZ2OAoHJJPpQBT1rWbHw/pM+palOIbaEZJ6lj2VR3JPAHeuR0/R7/xNqMPiDxPEY44jv07SGOUtvSSXs0v6L255pmlI/jjWY/Et9Gw0e1c/2NauMCQjg3Tj1P8ABnoOepzXZ1y1qv2Uawj1YUUUVzGgUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUVy/jHx/oXgi3jfVZnaeUExW0ChpHHrjIAHuSKaTbsgvY6iivKdK+Pvhi+vktry1vtPWQgLPMqsgz/ewcge+DXY+MvG2neCtDt9WvYZ7m3nnWFPsoViSyswPJAxhT+YqnCSdmhcyOloqvY3aX+n215GrLHcRLKobqAwBGffmrFQMKKKKACiiigAoorzTxF8bNA8NeILzR7vT9TkntXCO8SRlSSAeMuD39KqMXLYTaW56XRXm2g/HDwjrmoRWLG8sJZWCo15GoQseg3Kxx+OBXpNDi47gmnsFFFZGu+J9H8NxwNqt6kDXDiOCPq8rZAwqjk9Rk9BnmklcZr0UUUgCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigArkb/SL/wANahNr3heHzElbfqGkA4S59ZIuyy/o3fnmuuoqoycXdCauN0XWbHX9Kg1LTpvNt5hxkYZSOCrDswPBHar9cDq8MvgzVpPE+nRu2mTsDrNnGMjHT7Sg/vL/ABAfeXnqM13UE8V1bx3EEiywyqHjkQ5VlIyCD3GK7oTUldGDVmSUUUVYjnfFmpXEMFvpOnymPUNRLIkq9YIlx5kv1AIA/wBpl7ZqGys7fT7KGztYxHBCoREHYD+f1qjZv/afiPWNWblEl/s+2z2SI4fH1lMgPsi+latb01ZXMJu7CiiitCAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACvFPij/AMle8H/79v8A+j69rrmtc8D6T4g8QadrV61yLuwKGERyAL8r7xkY55qZK6Ki7Md4/wD+Sfa//wBeMv8A6DXkkH/JtF1/19D/ANHrXuWraZBrOk3em3RcQXUTRSFDhsEYODWAnw90VPBb+FA11/Zzv5hPmDzM7g3XGOo9KUk2xppI4q5t4o/2aQqRqAbVJDgdWMwJP503TLeJP2aptsajfbTSNx1bzjyfyH5CvQ5PB+mS+DB4WZrj+zhGIshx5mAwYc4x1HpSQ+DtMh8GHwqjXH9nGNo8lx5mGYsecY6n0o5X+Acx4jqwuD+zzoPl7vJGov5uPTdLjPtn9cVt6vp/iTUfhxYw6lrvha30GWOEQSESKVwAVAIU4bgg/jXqdh4N0aw8J/8ACMmF7nTCGBSdtxO5i3UY5BPBHTiuXg+CPhOG7WZ31GeFW3C2luB5f6KG/Wp5WPmR1fgy1ksvBmkWst3BdmK2VBPAxZHUfdKk4yMYrnviH4v0nRLuw0bXtF+26dqGC80hGyMBsMcYJJUEHjB5rvI40hiSKJFSNFCqqjAUDoAKy/EPhrSvFOnfYdWtRNEDuRgdrI3qpHSradrIhPXU8h8d/DrwXp3hq41vSNU+yyIoeCJbkSxzHPCrnLZPqCayvEepavq/wH0W4vXllCagY3lbJLoocKSe/PGfYd67yD4E+EobgSvNqc6A58qSdQp9vlQH9a7ufQNKudBOhy2MR00xiL7OBhQo6Yx0I65655qORl8yPHtY07xJqPw6sYdS13wtb6DLHCLeTEilcAFQCFODgEH8ap/FC1ksvhr4OtZbuG7MQKCeBiyOoQbSpPUYxXcW/wAEfCcF2kzvqM8KNuFtLcDy/wBFB/Wuj8UeB9H8W2VnaagJoobQkxLbMEA4Ax0PGBRyuwcyuM8T2kNl8NNXt4I1SOHSJo0CjGFERwK8aiE5/Z0m8ndsGp5mx/cyOv8AwLbX0DqOnQappN1ptwXEFzC0Emw4bawwcH1waytF8G6PofhuXQIYnuNPmLmSO5YNu3dRwB6VTjcSlZHlMdl4lvfhRbRT6z4Zt/DslvGm6RZAycjgkKfnDdffNW9W8O6TJ8KdA07VvFNhBcwPI9leRszxSjcflHAOACoz2xXRD4IeExdeYX1Iwb932U3A8v8A9B3frmut1nwhoeu6FFo17Yp9jhAECx/IYcDA2kdOKSix8yPINZ1Px/4Fis7nxEdN1zTBMEiecJL8xBPysQHBIB5Oa9zsbpb7T7a7RCizxLKFbqAwBwfzrg7L4L+F7W7hnnl1G+SE5SC7nDRj2wFHHtmvQwAAABgDoBTimtyZNMWiiirJCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAgvbO31CymtLqMSQTKVdT6f0Pv2qfwpqdxPbz6VqEhk1HTiqPK3WeI58uX6sAQf9pW7YorLu5P7N8R6Rqq8JJL9gufdJTiP8RKEA9nb1rOpG6uXB2djtaKKKwNwooooAKKKKACiiigAooooAKKKKACiiigBruscbO7BUUEszHAAHc1wnhhZPEus3PjK6U+TKpt9IjYf6u2B5kx2aQjP+6FFXPiDPLeWlh4XtZGjuNcmMMjqcGO2UbpmH/AcL9XroIIIra3it4EWOGJAiIowFUDAA/CuevOy5UaQXUkooorkNQooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooA5/xzqc+j+Btav7Zik8No5jcdVYjAP4E5rz34A+HrCPwvPr0kEcmoXFy8azMMsiKBwD2ySSfXj0r1LXdJi13QNQ0qZtqXlu8JbGdu4YB/A814L4M8aXvwhuL3wz4q0u6+ymYywywKCc4AJXJAZDgHg8c/htBXg0tyHo7s3v2gdBsoNI07xHbRJBqCXggeWMbWcFWYEkdSCnB9zT/HXxK1/QPDngjUtPlQNqdp593G0at5hCxHAJBx95unrXO+LfE2ofGjVtP0Dw1ptzHp0EvmyzzqOGxje+CQoALYGcnP4Vo/G7T4dMPgPTYc+RbCSBM/3V8hR+gq4r4YyJb3aOnstd8d+FtF1vxJ41ltZLOK2V7SyhVBtldsBCQMjBIByT16nFYOk6z8Vtb8M3HisXOmyaW8cjrprxBWkjGQxQhdw6HGW5x34z6J8TdCuvEfw91XTrJC90yLJEg6uUYNtHuQCB7mvMvB/wAU7Ow+H0fhf+zr6XxDDHJaQWscORKxJ259MZ5B54NTHVXS1KejsJ8Gddg8M/DPxNrNypeO0uA+wHBdtihVz2ySB+NT2niH4sav4Yn8aWl9psOmoHmTT/JUl40J3EZUnsf4gTj6ZxPh3odz4j+C3i7S7Nd1zJcI8SD+NkCOF/Hbj8a5/SJ/h1ZeE5E17Rb+TxLblka382SMStk45BwmBgHIzweDWjSbb63Jvoj1rVPiXe3vwXfxZpOy01FJEhkUqHVH8wKwAPYg5H1FVfBWufEbxHJpfiC+urO18NbD9oXy03yKindJjbkZZT0Ix6Y5OV4h0+2sP2eLt7XRJtGjurmK4+yTXDTMMyINxLAEZCg4xXfeAbMX3wf0qy3bBcae0W703Bhn9azdlHRdSlds4fS/FXxG+I11qN/4UvLHStKtJPLijnjVmlPUAkq3OMZ6AZH1ro/BHxPfVvCWt3mu26w6joSsbtIuBIAGwQOxJVhjpke+B454dsPBvh+fU9K+Iel30epW8mYmiZwHXGNoCkdxkN0IPWvQvA/h2x8Q/DzxK2i+Gp9FOo25ggea8aUXJAJUjcBgZIGeep9KqcYpbCTY3w14h+KvjO3uPEelXWmQadHKwh0+aIYm29VDbS3tksOfSoP2e5muNV8WTOmx5Hgdk/uktKcVV+HHxNsfBnhibw1rGnX41W0nkENtHDlpWY5CeobcSOnTFW/2fvP/ALY8XfaY/Ln8yHzE/utulyPzokrRloC3R674r/5E/W/+vCf/ANFtXz38L/iTYeDPDt3ZXejXl60t0ZhJCqlVBRRg57/L+tfQniv/AJE/W/8Arwn/APRbV4B8Kvilongbw5d6fqVpqE0012Z1NtGjLtKKuCWcHOVPappq8Hpccty54N1Kw8bfHKPXYUttHjgQtHaMwElwwQrxgAE85PsO/Wu61Pxnrdt8c9O8MRXKDSpowzxeUpJPls33sZ6gV59NqknxS+L2i6l4e0q4tYbNoWuLh1AYBH3F3K8A44AzzxWv8Q7w+E/jpo3iW/glOmmFP3iLn+FkYD1IyDj0Iq2ru3kSnodV8RPGet+HvHnhfStOuUjtL+VFuEaJWLAyhTgkZHB7V5/8VbfxP/wtTR1mu7RpJbhP7IIHESmUbfM+Xn5sZ68UvjXxVb+L/iV4Q1DTre5XTUuooobiePYJmEylioPYZUZ9c10Hxa/5K54H/wCu0P8A6PFEVytadGN63OW+MNt40t9O0c+Kr+wuYiziMWq4Ik/iJ+UcY213fiHxr4q8AeCreLWpLG+8R39yyWhgX92kQVeSAFyQTjH+0PTFZ37R3/IH0H/r4l/9BWrHx88P3moaNo+s21u9xDp7OtzGmchH2ndx0Hy4J9xSTUlFPzB6XsQap4m+Jfw8Gn6v4ou7HVNLuZRFPBDGqvCSCcAqq84BweRx9DWj8U/iHrXhfVPDr6FLHJa3qGV4jGG88blwASMjIOOPWvP5pfhldy6da+HPCmp6tf3bBXtTdyxGPjpnkE59OMA8iuj+LlpHZ+KvANnHHsih2RKm7dtAeMAZPXp1p8q5ldBd2N26vfiLofgjxLqvie8sndIYnsokhjdY2LjcCNvOBgc569eKY/jbW7X4AWviaCaGPU2kKl1gQJj7QyfcxjoPSur+LX/JLde/64r/AOjFrzK6/wCTVLP/AK6/+3TVEbSSduoPQu3HiX4p3/gSPxfa3un2VhBAHaERK0twF4eQ7kIAJycAjj9eptPig6/B0eMLu2jN8MweSuQjzb9oPsP4iPqKhh/5NvP/AGBG/wDQTXIaJ4dvPEv7ODWlhG0t3DdvcRxL1k2vyB74JwO5xTtF7rqGqNLTtd+LB8KyeMpbrTrjTjC0409olWQRYPzqQvYfMAWPA6HpXX/CDxRq3i7whcahrE6zXKXrwqyxqg2hEIGAPVjXn2l/FQP8NH8KwaLfy65Bp72jBYx5aIqFTIxzkYXkgjqOtdR+z3/yT27/AOwlJ/6LjonH3XdBF6nrFFFFc5oFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFAGJ4p0N9c0kLayiDU7Vxc2Fx/zymXp/wE8qR3BNaHhfXk8R6BBqHlGGfmK5tz1gmU7XQ/Rgfwwe9W65OzJ8OfEl4Adun+IozIo7JeRL8303xgH3KV00J68pnNdTuaKKK6jIKKKKACiiigAooooAKKKKACiiigAooooAK4fxhLJ4i1m28G2zkWzoLrV5FONtvn5Ys+sjDB/wBkN612N9eQadYXF9dOEt7eJpZXP8KqMk/kK5DwNZznSp9dv0K6jrcv22YHrGhGIo/oqbR9Sazqz5YlRV2dOiJFGscaqiKAqqowAB0AFOoorgNwooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAK8C0O3h8XftGaw+qxrcRad5piikGV/dMsa8fU7vrzXvteEeLrDV/hv8UX8cWNhJe6PeZNysf8G4AOGPbkBgTxnitaXVdSZHqvi3wVo3jTT4rPVYWxE4eOWEhZF9QGIPB7ivOvjnptvo/wALNG0203/Z7a/iijDuWIURSgDJrL8S/F/UfGcNtongXT9UgvppVZ5uFcAHoNpOFzjLEgYHPWtH4229/a/CbQ4NUuvtd+l9CLifaBvfyZckAAcZqoxlFq5LaadjR8XePL/wt4U8LaToccb6xqlrCkTSAERDaqg4PBJJwM8cHNZl34s8dfDrVtLk8V6rY6xpN8/lytAiq0B4yQQq9M55yDg9KzPin4duZ9B8H+IlspL2wtLKGK9hjyCEwrckcgH5gT24rIhX4ea3renab4V8EX2pyXH+v869mh8kcckhm4HJJ6dOtVGMbbA27nrXjqfx9Nqllp/hKKC0spF3XGqTGNhGeeNpyQAB12nJI6YrkPDfjfxVpPxOtPCmu63Ya5bXQx59sqfuyVJHKgc5XBBzway/ipc+T8TtNtfFJv8A/hEBEu2O3Zgr/KcnjqQ2M99vTtWLp02hx/Gfwzd6HpU2maJIyx2zToy/aD8y7xuJJyzAZz2ojFcvyBvU7LWvG/jT/hcWoeE9CntnjdES3S4jUJATCjtIzAbjj5jjPUjr0pNH8W+NvDfxTs/CvinULbU4L4ApJHEqbQwO1lwq/wAS4IP4VBZf8nU6h/1wH/pMlL43/wCTjfCn/XCH/wBDlostrdA13Pbq8J0JVf8Aad1cMoI2ScEf9M1r3avmzUvFdr4M+P8ArWr3kE08KloykON2WjX1IFZ0le6XYcuh3Hx58P6dL4I/tZbWKO9tbiMLMiAMysdpUnuOQfwq7b674wuvhBoV34ZsRe6tPAkcssjp+7VcqXwxG5jgevUmuH8X+ONS+LVpa+HPDGg3iwvOsk082McZxkjIVQTkknsK1fi3Y6r4W+H/AIa0jTpbkaRbDydQltyVLkBdu4joCd5weM49qtRdlF7ivu0Udf8AGHj/AOHt/p82r+JtL1hJnxPZRLHuTGMg4VWHU4PqOlVvjlFrh8baLI11AbOVv+JYoA3RN+73l/l7tg9TXK+Oj4Pm0KxfwZo15HbRS7bu/nR8M5X5Y9zE88MSBgccV33x0kIbwdrcSPLYxM7mRBkc+WyjPuAcfSrStJadxdGdjq83xK0/wRp9tZQ2uo+IpJHW7vIzGI4k3HaVVtoJwVH3TjB471w+q+MvHXgHxDpces+JdL1uC5k2z20KpujAIBzhVZevB6cHik+Kni648SeGvD2rab/aMXhi5mlW9CDY5KuFw2CRyAxXJwe/SuL8ayeEWh0e48IaNdW+nQTFbm+mR8SucEICxOSArH8aUI90DfY+sKKajrIiujBlYZBByCKdXKahRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAjKGUqwBBGCD3rlPDLnwr4jl8JykjTroPdaOx6KOstv/AMBJ3L/sk+ldZXP+MdJudT0TztPwNV0+Vbyxb/pqnO36MMqfZq1pT5ZEyV0dbRWfoWr2+v6FZata5EN3Csqqeq5HKn3ByD9KK7jA5Pwed/hHTJzy1zCLlj6tITIT+JY1t1y/w4vFvfh3oUqHIW1WI/VPkP8A6DXUV1LY5nuFFFFMQUUUUAFFFFABRRRQAUUUUAVn1Cyjco95bqynBVpVBB/OpYbiG4UmGaOQDqUYH+VfPen+E9P8Y/GbxLp2pSXCQo9xMDAwVtwkUdwePmNR+KdCj+F3jfRpvDmpXEkk2GeCRwXxuA2ttAyrZIwR2NZ873sXyrY+jKpxatptxcfZ4dQtJJ/+eaTKW/IHNeU/FPWNS1rxdpfgXTLlreO62G6dT97cTwcdVCjdjvn2p+s/ArTI9EZtDu7watCu6NppF2ysOxwBtz2OeKfM+guVdT1+iua8NNqmg+DI28XXlubi0RjLciQsPLHQsSBlscd847k1yUnx00FZ2MWl6pLZI21rpY1A/In+eDT5ktw5W9j1B5EiQvI6og5LMcAUz7Vb+R5/nxeT/wA9N42+nXpXF+M9YsNe+Emralptws9rNakq44/iGQR2I9K4I/8AJsI/66/+3VJysCie6JIkqB43V0PIZTkGnVwXhTX9O8NfCLR9T1ObyreO2A4GWdiThVHcmqen/GjQ7q9t4r3T9R063uTiG6uYwI29yQeB7jIFPmXUOVnpNFYXinxdpPg/TFvtUlbDttiiiG55T6KP6kgVy2kfGPRNR1W3sLywv9Ma5IEMtygCNngZIPGfXp70OSWgrNno1RS3MEDKss0cbP8AdDsAT9K8O8efEGeD4m6fbodQXTNLmT7RaoAPPkVySyjPzDG0DPcGtzxvqPhfVr/wdqOqw6zHNdYls0t/KULlkOJQ2e+OnvS5yuU9aormvF3jnRvBdtHJqUkjzTZ8q3hUNI+Op5IAHuTWFoPxf0PWNWi0y6tbzTLmYgRfalAVyegyDwT2yMe9PmV7E8r3PQqK5PxP8QtI8JaxZadqaXAN0m8TIoKRrnGW5z+QNYdt8Z9Dl1mCwudP1KyiuGAhubiIKrAnAYjOQp9eaOZD5WekVE9zBHKsUk8aSN91GcAn6Cpa8U+Iv/Jc/CH0tf8A0oehuwkrntMkiRIXkdUQdWY4ApIpY54w8UiSIejIwIrlviZ/yTfXP+vf/wBmFc18Odbt/DnwWi1e6jlkgtnlZ0iALHMpHGSB3ovrYdtLnqNFeZ3Pxs0JLKO6tdN1O6TaGmKRALBk8BmzjP09etdnovinStd8ODXbWfZY7WaRpvlMW37wb0xQpJicWjZory64+OegJcSfZ9N1O5tIm2vdJGoX6gE/zxXYp4y0ifwhP4mtJXubCGJpHEY+cbeqkHGG9jQpJg4tHQUV5rafGnQtQvtNs7PT9RmmvZBGVCIDEzOVAPzYJOM8diPpV3xD8V9H0PXW0aCzvdTvYziVLRAwQ9x15I746etHMh8rO4+0wef5HnR+d/zz3Dd69OtS14Roet2fiL9oS21OxMnkTQEASLtZSLcgqR6ggiu98SfFLSNA1ltIgtLzU79P9bFZoG8vjOCfX2HTvQpIHF7HdUVzvhLxrpPjOzlm05pElhIWa3mULJGT0yASCDg8j0rj5Pjt4cW1MqWOotKJNnklEBwBy2dxGKOZC5WepUVxXiL4maR4cSxjltryfULyJJUsYUBlQMON/OAe2BnpU/hL4h6R4tuprGGK5stRhG57S6QK+O5HPOOPf2o5lewWe511RG5gE4gM0YmPSPcNx/CuJ8SfFTR/D+t/2NFaXmpX6/6yO0QNsOM468nHYVwNhr1n4l/aA0nUrHzRE8RUpKm10YQuCrD1BpOSGos95oooqyQooooAKKKKACiiigAooooAKxfFzeX4T1K4HDW0JuVPo0Z8wH8Corarl/iPeLY/DvXZWOA1q0Q+r/IP/QqT2Gtz0miiiuU6QooooAKKKKACiiigAooooAKKKKACiimu6xxs7kKqgkk9hQBxOmEaz8Rtd1UndDpcaaVbntuIEsx+uWRf+A11dcn8OFMng2DUZARNqk02oSk9zLIzD/x0qPwrrK4KjvJm8VZBRRRWZQUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFRT20F1H5dxBHMnXbIgYfkalooAjgt4baIRQQxxRjosahR+QokgimKmWJHK9Nyg4qSigAqJbW3W4a4WCITsMNIEG4j3PWpaKAGRwxQgiKNEB5IVQKjextJLhbh7WBp16SNGCw/HrU9FMBrxpKhSRFdT2YZFKqKihUUKo6ADAFLRSAguLK1uypubaGYryvmRhsfTNTgAAADAHaiigCI2tubgXBgiM4GBKUG4D0z1pyQxRMzRxohb7xVQM/Wn0UAIQGUqwBB4IPeoVs7VTlbaEH1CCp6KAEVVQYVQB6AUye3huY/LnhjlTrtkUMPyNSUUARG3gOzMMZ8v7nyj5fp6U54YpHV3iRmX7rMoJH0p9FADJIYpgBLEkgHTcoOKfRRQBBBY2ltI0kFrBE7/eaOMKT9SKe8EUrK0kSOy9CygkVJRTAa6LIhR1DKeoYZBpvkQ+V5XlR+X/c2jH5VJRSAZ5UfleVsXy8Y244x9KyPEmkXep+GrvT9Hvm0u8cAwXEJK7GBB5x2OMH2NbVFNOwHij6f8ZW0O50Ca30u5iuUeGTUZJVMpRuDzuHYnB25r0H4eeED4J8IwaTJMk1yXaaeRM7S7Y6Z5wAAPfFdVRVSm2rCSsFFFFQMKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACuW+INvKfCz6nbLm70eVNSg9zEcsPxTePxrqajuII7m2lt5V3RSoUdfUEYIqouzuJq5ctbmK8tIbqBt0MyLIjeqkZB/I1LXJfDSd5PAGm28rbpbLzLF89cwyNGP0UV1teic4UUUUAFFFFABRRRQAUUUUAFFFFABRRRQBxnxFb7dp+meHFY7tavkglA6+QmZJf/AB1Nv/Aq6UAKoVQAAMADtXMXhGofFq2jb5k0rSGlX/Zknk2/+gxH866iuPEO8rG1NaBRRRWBYUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFADI4Yos+XGiZ5O1QM0+iigApkcUcWfLjRMnJ2qBmn0UANkjSVdsiK69cMMinUUUAFFFFABRRRQAUhAZSrAEHgg96WigBqIsaBEUKo6BRgCh0WRCjqGU9QwyDTqKAGlEMfllF2YxtxxilVVRQqKFUcAAYApaKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKAOZ8Fn+ytd8Q+GzxFBOL+0HpDPlioHoJFk/MUUzUWGnfEvw9eghVv7a50+Ynp8oEyflsf86K9Cm7xTMJKzOa+HJ/s7SbDTW4ju9Ntr+29DmJFlUfRsMf8ArrXdVx2k6fPd/Dzw1c2O0ajZWFtNbFjgMfKUFGP91gSp9Mg9QK6XS9Sg1awju4NwDZV43GGjcHDIw7MDkEV2I5WXKKKKokKKKKACiiigAooooAKKKKAPnO18Kx+MPjJ4l06S+ns1SSebzIepxIox9Pm/SvR/Dvwc0HQtWi1Oa5utQuYWDxicgIrDo2AOSO2TXRaX4J0jSPFN74itRP8Ab7wOJd8mU+ZgxwMccgV0dQoLqW5PoeHePn/4Rj416L4hvFYWEwjLS4yFwCj/AJAg/jXpniLxxo2g+HJtWF7a3P7vNvFHMpM7H7oGM8ep7DmtPXNA0vxJprWGrWiXEBOQDwVPqpHIP0rh7b4G+ELe7E7nULhAc+RLONn0+VQf1os1sF09zL8b+I7nxd8E5tXt7CazilnQSRs27MYcDIIxld2Ow6VueFLzw8vwagMslsLBLEpdqSP9Zg7wR/eLZx3ORiu5On2Z07+zjaw/YvL8nyNg2bMY246YxXAS/A/wfJfG4UX8cec/Z0uB5f05Bb9aGne4XVrHB+DUuV+B3i5pNwtmf91npnC7iP8Ax38qvH/k2Ef9df8A26r1648MaVN4Yfw6lv8AZ9NaLyvKhO0hc54Prnv3qh/wgei/8IX/AMIpi4/szdu/1nz537/vY9aXKx8yPGvG6zn4O+CmXd9mG4Sem7B2/pvrc8Y6Z4k1TwtYw674h8KW+lyujWkiCRQSFO3adp425r1M+ENGfwpH4antjPpsaBFSRssMHIO4YII9RXMWHwV8J2V9Hcv9uu0jbclvczBox36BQSPYmk4sFJHEfEKJrLWvh9/a08dxYQ28CzTKd0cm1k8xgT1BXB+lek+NNY8FWbaY3iaK2ui5Y2oMPn4Bxk7Rng8dq2/EPhnSfFOmf2fqtsJYQdyFTtaNvVSOlcxoXwe8K6FqceoRpd3c0Tb4hdSBlRh0IAUZx75qrNPQV09zlvHrKPjt4SYnC+Vb8nj/AJbSUvxjYN4w8GspBBmOCP8ArpHXd+L/AIeaH40mt59S+0xzwLsWW3kCsVznacgjGc9u9MuvhxoV5BosU32oro6hbUiXBwCD83HP3RScXqNSWhwHix4IP2gtHl1oqLARx+U0v3B8rbTzxxJU3x7lsZbTRYIij6r5xKLHy4jI9ucFtuPocV6T4n8H6L4vtEg1a13mPJimRtskeeuD/Q5FYnhz4TeF/DeopfwRXN3cxHdE93IGEZ9QAAM+5BxQ4vVApLRnEfEiBp/iL4Ft75RI7i3SdW5DEzANn9as/tAqos9AkwN4mlAPfGE/wr0bWvBmk69r2m6zeif7Xp7K0GyTC5VtwyMc80eKvBek+MYrWPVRPttmZo/Kk28nGc8e1Di9RKS0OhrxH4qyppnxZ8KardHZaRiEvIRwAk5ZvyDA17dWN4k8LaR4r08WWrW3mop3Rup2vGfVSOn8jVSV0KLszm/ifr2lD4camE1C2ka6iVIFjlVjISw6YPPGT+FclYwPB+zPcb1KmRHcA+huOK3rX4F+Ebe5WaSTUrlAc+TLOu0+x2qD+tdxqfh/T9V8PSaFNEYrB41i8uH5NqqQQB6dBSs3qx3S0RwXgu2hH7P8wEa/vbG9eTj7xzIMn8AB+Ari9DFy37OuvC23Ei/+fb12Zi3fhjr7Zr2zTvDOn6Z4V/4Ry3837B5UkPzPl9rli3P/AAI1H4d8I6V4Z0WbSLGN3s5nZ5EuGD7twCkHjpgdKXKHMcz8OLvQE+E1uZZLVbSOGQX4cjAYk7t49x69RivO/ASTj4U+Onw4smjxDnpuCnd+OCld/dfBDwhcXxuEW+gQnJginHl/qpb9a68+F9JXwxJ4dgtvs+myRGIxxHBwepyec+5o5WPmRyPwStIYvh3FKsa757mR3bHJIOB+WBXNfCB4YPiB4rgviq6m0jBN/wB44kbzAPx2n8K9X8O+HrHwvo8el6d5n2aNmZfMbc2Scnmuf8S/Czw14o1T+0ruO5t7tseY9rIE8zH94EEZ9xg0+V2QuZXZwdhNZT/tLySWLRtF84Zo/ulxbkP/AOPZz75rJ8Cp4mPjzxLFo93pdvqhmfzhqCsWYeY27bgHvjP4V6zpHw18N6Frlrq+nW8sFxbRmNFEmVIKlSSDyScnnNQ+Jfhd4c8T6kdRuFurS9b781nIEL9uQQRnHfGaXKx8yOX+HmkXMHxI1nUp9d0W7uZY5Eu7WwZ8q+9cttKgYyDznqfes/4BWsLt4iuXjVpN0UQYjJCneSPoePyFemeFfBei+DrWSHSoGDykGWaVt0j46ZPp7DApPCvgvSfB6Xa6UJ8XTK0nmybuRnGOPc0KOwnLc8jvl1ofH/UF0+4sbfUXGLZ78EoV8pcBcDrtzj8a19P0bVpPjDZahqviDQDq0C7ZrS0Z1kkTy27FcE7T69AK77xX8P8AQfGLxTajFLHdRDalzbvskC9ccggj6jimeFfh3oHhC4kurCOea8kG03N04dwD1AwABn2FHK7j5lY8++GDwwfFnxXFflV1F5JRF5nDN+9JbH/jp+gqJprKb9pi3ayaNgCyytH0Mgt2Dfj2PuDXfeKPhf4c8V6iNQvI7i3uzgPLayBDJjpuBBH49afpXwy8NaLrVjqun280NxZoUjAkyrZDAls8kncec+npRyvYOZbnYUUUVoZhRRRQAUUUUAFFFFABRRRQAVwvxFP9paTqGmrzFZ6bc39z6DETrEp+rbm/7Z112qalBpNg93OGbBCpGgy8rk4VFHck4ArmtW06az+HfiW4vSrajeWFzNdFTkK3lMAi/wCyoAUeuM9SallI9SooormOgKKKKACiiigAooooAKKKKACiiigArF8YXJsvBWvXSnDQ6fcOD7iNiK2q5n4ikj4b+JMf9A6b/wBANAEnhm2Wz8KaPbIMLDZQoB9EArVqvYKE0+2UdBEoH5CrFea9zpCiiikAUUUUAFFFZ2sa/pXh+1Fxqt9Daxk4UOfmc+iqOWPsAaYGjRXGnxT4g1bjQfDrQQnpd6w5gU/SJcufx20w6N4ovvm1HxfLAG6w6XaRwqPozh2/WrVNmbqxR2tFcR/whUD/AOv17xHN67tWlUH8FIFN/wCEB0r/AJ/da/8ABtcf/F1XsvMn2yO5orhv+EB0r/n91r/wbXH/AMXR/wAIDpX/AD+61/4Nrj/4uj2XmHtl2O5orhv+EB0r/n91r/wbXH/xdH/CA6V/z+61/wCDa4/+Lo9l5h7ZdjuaK4b/AIQHSv8An91r/wAG1x/8XR/wgOlf8/utf+Da4/8Ai6PZeYe2XY7miuG/4QHSv+f3Wv8AwbXH/wAXR/wgOlf8/utf+Da4/wDi6PZeYe2XY7miuG/4QHSv+f3Wv/Btcf8AxdH/AAgOlf8AP7rX/g2uP/i6PZeYe2XY7miuG/4QHSv+f3Wv/Btcf/F0f8IDpX/P7rX/AINrj/4uj2XmHtl2O5orhv8AhAdK/wCf3Wv/AAbXH/xdH/CA6V/z+61/4Nrj/wCLo9l5h7ZdjuaK4b/hAdK/5/da/wDBtcf/ABdH/CA6V/z+61/4Nrj/AOLo9l5h7ZdjuaK4b/hAdK/5/da/8G1x/wDF0f8ACA6V/wA/utf+Da4/+Lo9l5h7ZdjuaK4b/hAdK/5/da/8G1x/8XR/wgOlf8/utf8Ag2uP/i6PZeYe2XY7miuG/wCEB0r/AJ/da/8ABtcf/F0f8IDpX/P7rX/g2uP/AIuj2XmHtl2O5orhv+EB0r/n91r/AMG1x/8AF0f8IDpX/P7rX/g2uP8A4uj2XmHtl2O5orhv+EB0r/n91r/wbXH/AMXR/wAIDpX/AD+61/4Nrj/4uj2XmHtl2O5orhv+EB0r/n91r/wbXH/xdH/CA6V/z+61/wCDa4/+Lo9l5h7ZdjuaK4b/AIQHSv8An91r/wAG1x/8XR/wgOlf8/utf+Da4/8Ai6PZeYe2XY7miuG/4QHSv+f3Wv8AwbXH/wAXR/wgOlf8/utf+Da4/wDi6PZeYe2XY7miuG/4QHSv+f3Wv/Btcf8AxdH/AAgOlf8AP7rX/g2uP/i6PZeYe2XY7miuG/4QHSv+f3Wv/Btcf/F0f8IDpX/P7rX/AINrj/4uj2XmHtl2O5orhv8AhAdK/wCf3Wv/AAbXH/xdH/CA6V/z+61/4Nrj/wCLo9l5h7ZdjuaK4b/hAdK/5/da/wDBtcf/ABdH/CA6V/z+61/4Nrj/AOLo9l5h7ZdjuaK4b/hAdK/5/da/8G1x/wDF0f8ACA6V/wA/utf+Da4/+Lo9l5h7ZdjuaK4b/hAdK/5/da/8G1x/8XR/wgOlf8/utf8Ag2uP/i6PZeYe2XY7miuG/wCEB0r/AJ/da/8ABtcf/F0f8IDpX/P7rX/g2uP/AIuj2XmHtl2O5orhv+EB0r/n91r/AMG1x/8AF0f8IDpX/P7rX/g2uP8A4uj2XmHtl2O5orhv+EB0r/n91r/wbXH/AMXR/wAIDpX/AD+61/4Nrj/4uj2XmHtl2O5orhv+EB0r/n91r/wbXH/xdH/CA6V/z+61/wCDa4/+Lo9l5h7ZdjuaK4b/AIQHSv8An91r/wAG1x/8XR/wgOlf8/utf+Da4/8Ai6PZeYe2XY7miuG/4QHSv+f3Wv8AwbXH/wAXR/wgOlf8/utf+Da4/wDi6PZeYe2XY7miuG/4QHSv+f3Wv/Btcf8AxdH/AAgOlf8AP7rX/g2uP/i6PZeYe2XY7miuG/4QHSv+f3Wv/Btcf/F0f8IDpX/P7rX/AINrj/4uj2XmHtl2O5orhv8AhAdK/wCf3Wv/AAbXH/xdH/CA6V/z+61/4Nrj/wCLo9l5h7ZdjuaK4b/hAdK/5/da/wDBtcf/ABdH/CA6V/z+61/4Nrj/AOLo9l5h7ZdjuaK4b/hAdK/5/da/8G1x/wDF0f8ACA6V/wA/utf+Da4/+Lo9l5h7ZdjuaK4b/hAdK/5/da/8G1x/8XTh4Hs4x+41jxDB/wBc9Xn/AKsaPZeYe2R29FcSPD2vWfOm+MtRH/TO/hiuVP47Vb/x6njXvFmk86rodvqluOs+kSESKPUwyHn/AICx+lS6T6FKrFnZ0Vj6J4p0fxD5iWF2DcRf621lUxzRH/ajbDD64xWxUNW3NAooopAFFFFABRRRQBzXw/xGfFFsOkOvXBA9A6pJ/NzXZVx3gkbde8ZKOn9rKfzt4a7GvRh8KOd7hRRRVCCiiigAooooAKKKKACiiigAooooA4jQyJviJ4yn6mNrO3B9lh34/OQ11Vct4YGfFXjN+51OMfgLaKuprgq/GzeGwUUUVmUFFFFABRRTXdY0Z3YKijJZjgAetADqK5Gfx7a3Ez23h2wutdnQ7WktcLbofRpmwv8A3zuqEp441PmfVNM0aM87LK3NzKPYvJhf/HK0VOTIdSKO0orif+EQnm+a88VeI53PXZe+QPwESrTT4D0xuX1DXHb1bV7gn/0Oq9l5ke2R3FFcN/wgOlf8/utf+Da4/wDi6P8AhAdK/wCf3Wv/AAbXH/xdHsvMPbLsdzRXDf8ACA6V/wA/utf+Da4/+Lo/4QHSv+f3Wv8AwbXH/wAXR7LzD2y7Hc0Vw3/CA6V/z+61/wCDa4/+Lo/4QHSv+f3Wv/Btcf8AxdHsvMPbLsdzRXDf8IDpX/P7rX/g2uP/AIuj/hAdK/5/da/8G1x/8XR7LzD2y7Hc0Vw3/CA6V/z+61/4Nrj/AOLo/wCEB0r/AJ/da/8ABtcf/F0ey8w9sux3NFcN/wAIDpX/AD+61/4Nrj/4uj/hAdK/5/da/wDBtcf/ABdHsvMPbLsdzRXDf8IDpX/P7rX/AINrj/4uj/hAdK/5/da/8G1x/wDF0ey8w9sux3NFcN/wgOlf8/utf+Da4/8Ai6P+EB0r/n91r/wbXH/xdHsvMPbLsdzRXDf8IDpX/P7rX/g2uP8A4uj/AIQHSv8An91r/wAG1x/8XR7LzD2y7Hc0Vw3/AAgOlf8AP7rX/g2uP/i6P+EB0r/n91r/AMG1x/8AF0ey8w9sux3NFcN/wgOlf8/utf8Ag2uP/i6P+EB0r/n91r/wbXH/AMXR7LzD2y7Hc0Vw3/CA6V/z+61/4Nrj/wCLo/4QHSv+f3Wv/Btcf/F0ey8w9sux3NFcN/wgOlf8/utf+Da4/wDi6P8AhAdK/wCf3Wv/AAbXH/xdHsvMPbLsdzRXDf8ACA6V/wA/utf+Da4/+Lo/4QHSv+f3Wv8AwbXH/wAXR7LzD2y7Hc0Vw3/CA6V/z+61/wCDa4/+Lo/4QHSv+f3Wv/Btcf8AxdHsvMPbLsdzRXDf8IDpX/P7rX/g2uP/AIuj/hAdK/5/da/8G1x/8XR7LzD2y7Hc0Vw3/CA6V/z+61/4Nrj/AOLo/wCEB0r/AJ/da/8ABtcf/F0ey8w9sux3NFcN/wAIDpX/AD+61/4Nrj/4uj/hAdK/5/da/wDBtcf/ABdHsvMPbLsdzRXDf8IDpX/P7rX/AINrj/4uj/hAdK/5/da/8G1x/wDF0ey8w9sux3NFcN/wgOlf8/utf+Da4/8Ai6P+EB0r/n91r/wbXH/xdHsvMPbLsdzRXDf8IDpX/P7rX/g2uP8A4uj/AIQHSv8An91r/wAG1x/8XR7LzD2y7Hc0Vw3/AAgOlf8AP7rX/g2uP/i6P+EB0r/n91r/AMG1x/8AF0ey8w9sux3NFcN/wgOlf8/utf8Ag2uP/i6P+EB0r/n91r/wbXH/AMXR7LzD2y7Hc0Vw3/CA6V/z+61/4Nrj/wCLo/4QHSv+f3Wv/Btcf/F0ey8w9sux3NFcN/wgOlf8/utf+Da4/wDi6P8AhAdK/wCf3Wv/AAbXH/xdHsvMPbLsdzRXDf8ACA6V/wA/utf+Da4/+Lo/4QHSv+f3Wv8AwbXH/wAXR7LzD2y7Hc0Vw3/CA6V/z+61/wCDa4/+Lo/4QHSv+f3Wv/Btcf8AxdHsvMPbLsdzRXDf8IDpX/P7rX/g2uP/AIuj/hAdK/5/da/8G1x/8XR7LzD2y7Hc0Vw3/CA6V/z+61/4Nrj/AOLo/wCEB0r/AJ/da/8ABtcf/F0ey8w9sux3NFcN/wAIDpX/AD+61/4Nrj/4uj/hAdK/5/da/wDBtcf/ABdHsvMPbLsdzRXDf8IDpX/P7rX/AINrj/4uj/hAdK/5/da/8G1x/wDF0ey8w9sux3NFcN/wgOlf8/utf+Da4/8Ai6P+EB0r/n91r/wbXH/xdHsvMPbLsdzRXDf8IDpX/P7rX/g2uP8A4uj/AIQHSv8An91r/wAG1x/8XR7LzD2y7Hc0Vw3/AAgOlf8AP7rX/g2uP/i6UeAtLU5W+1xT2I1a4yP/AB+j2XmHtkdxRXE/8IfJFzaeKPEkDjoWvzMPylDCnCHxvpvNtrGnaxH/AM87+28iT8Hi4/NKTpPoNVonaUVyEPj2CzkWDxLpt1ocrHaJp8SWrHtiZflH/AttdbHJHNEssTq8bjKspyCPUGocWtzRNPYdRRRUjCiiigAooooA8++MF02m+ErXVYjtlsr1HVvTcjxn9HopnxwUH4WagT/DLCR/38Wiuyh8BjPc1PBn/Ii+Hv8AsGW3/opabqFpc6VfyazpkTTLJj7dZJ1mAGBIg/56Adv4gMdQKd4M/wCRF8Pf9gy2/wDRS1uV6K2OPqV7G+ttSs47uzmWaCQZV1/kfQjoQeQasVhXmk3VleS6noZRZ5DuubNztiuj65/gk/2uh6MDwRd0vWbXVVkWPfFcwkCe1mG2WE/7Q9PQjIPYmi4GhRRRTEFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABVe+v7bTLOS7vJligjHLH8gAOpJPAA5Jqtqms22l+XGwknu5s+RaQDdLKfYdgO7HAHc1UstJubq8j1TW2R7mM7re1jOYbX3H998dXP0AHOUMbp1nc6nfprOqRND5efsNm3WBSMF3/AOmhH/fIOBySS7xn/wAiL4h/7Blz/wCimrcrD8Z/8iL4h/7Blz/6Kah7AtzvaKKK5TpCiiigAooooAKKKKACiiigAooooAK5j4jf8k28Sf8AYOm/9BNdPXMfEb/km3iT/sHTf+gmgDTsv+PG3/65L/Kp6gsv+PG3/wCuS/yqevNOkKKKKQBRRXB317ceOr2bTdPnkg8NwMY7y7iOGvXHWKNu0Y6Mw69B3NVGLkyZSUVdlm88VX2uXcmneERE6xsUudXmXdBCe6xj/lq49jtHc9qm0jwpp+mXRv5jJqGqv/rNQvDvlPsvZF/2VAFa9ra29jaRWtrCkNvEoSOONcKoHYCpq6ElHY5ZTcgooopkBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFAGRrPhrS9d2SXduVuov8AU3cDGOeE+quOR9OntWbHrus+EWWPxEx1LRui6tFHiWAdvPQdR/tr+IGc11NIQGBBAIPBBpNJ6MqM3HYuwTw3VvHPbypLDIoZJI2DKwPQgjqKkrz+WC58BTyahpcbzeHXYve6cgybTPWWEf3e7J9SK7q1ure+tIrq1mSa3mQPHIhyrKehBrCUOU64TUloTUUUVBQUUUUAc34L/wCRg8Zf9hVP/SaGuxrjvBf/ACMHjL/sKp/6TQ12NejD4Uc73CiiiqEFFFFABRRRQAUUUUAFFFFABRRRQBxPhf8A5Gfxn/2FE/8ASaGuprlvC/8AyM/jP/sKJ/6TQ11NcFX42bx2CiiisygoorB8T+IxoVvDBawfa9WvGMdlZg4Mjd2b0RerHsPrTSbdkJuw/wAQ+J7Lw/HEkiSXV/cEra2NuN0s7ew7Ad2PArnx4d1DxI63Xi+dZIc7o9Ht2Ito/TzD1lYe/wAvoKv6D4eOmyTajqFx9u1q6A+03jDHHaOMfwxjsPxPNbldEYKJzTqt6IZDDFbwpDBEkUSDCoihVUegA6U+iiqMgooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigBskaTRtHKivG4wysMgj0Irl28MXehStd+ELpLIk7pNMny1pN64HWJvdePUGuqooGm1sUPD/iu21qWSxuIJNP1iAZuNPnI3qP7yno6ejD8cVv1zGveHrfXIYn8x7W/tm32l7DxJA/qD3B7qeCKPDXiO5urqbQ9cSODXLZdx2cR3UXQSx57diP4TWM4W1R006nNozp6KKKyNQooooA88+N3/JK9S/66wf+jFoo+N3/ACSvUv8ArrB/6MWiuzD/AAmNTc0/Bn/Ii+Hv+wZbf+ilrcrD8Gf8iL4e/wCwZbf+ilrcr0Vscb3Cs7U9FttTaOYtJb3kIIhu4G2yx+wPQj1Ugg9xWjRQI58axf6L8mvQiS2HTUrZCY8f9NU5MZ9xle+V6VuwzRXEKTQSJLE43I6MGVh6gjrT6w5vDiQTPc6Lctpdwx3Osa7oJT/txdPqV2t70DNyisH+3rvTfl13T2hjH/L7aZmgPuwA3p+I2j+9Wxa3Vve26XFpPFPA4yskThlb6EcUXCxNRRRTEFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFQXl7a6fbNc3tzDbwL96SVwqj8TWR/bV/qfy6JpzGI/wDL7fBoovqqfff8lB7NSuOxs3FxBaW73FzNHDDGNzySMFVR6knpWIdV1HWvk0SE29qeuo3UZAI/6ZRnBb/ebC9xuqW38ORPcJd6tcPql2h3IZgBFEfWOIfKv1OW9626AM7TNGtdL8ySPzJrqbHnXU7b5ZcerenoowB2ArRoooEFYfjP/kRfEP8A2DLn/wBFNW5WH4z/AORF8Q/9gy5/9FNQ9hrc72iiiuU6QooooAKKKKACiiigAooooAKKKKACuY+I3/JNvEn/AGDpv/QTXT1zHxG/5Jt4k/7B03/oJoA07L/jxt/+uS/yqeoLL/jxt/8Arkv8qnrzTpCiiqmp6jbaRpd1qN4+y3tYmlkb2Azx70Ac14tv7nU7+HwlpkzQzXUfm6hcofmtrbOMA9nc5Ueg3HtWzZWdvp1lDZ2kKw28CBI41HCqOgrE8H2FxFp02raim3VdWk+1XIPWMH/VxfREwuPXPrXRV0xXKrHHUlzMKKKKZAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABXJ2Df8ACE+Io7H7vh7VpsW47WV03OweiSdvRuO9dZVHWdJttc0i5027B8mdNpK9VPUMPQggEe4pNXVmVGXK7m9RXOeC9YudU0MwaiwOq6fK1ne/7UidH+jKVb/gVdHXM1Z2OxO+oUUUUhnN+C/+Rg8Zf9hVP/SaGuxrjvBf/IweMv8AsKp/6TQ12NejD4Uc73CiiiqEFFFFABRRRQAUUUUAFFFFABRRRQBxPhf/AJGfxn/2FE/9Joa6muW8L/8AIz+M/wDsKJ/6TQ11NcFX42bx2CiiisyipqepWuj6Xc6jeyCO2tozJIx7AenqfQVyvhmwu7mebxNrEe3VL9AI4Tz9jt+qRD37t6sfajxG3/CQ+LrDw8PmsbELqGojs7ZxBGfqwLkHsgrpa6KcbK5zVp68oUUUVZiFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAVi+JNCbWLSKa0lFtqtm/nWN1j/AFcnofVGHDDuDW1RQCdit4Y19fEOkC4eE295C7QXlsTkwzLwy+47g9wQa2a4a+b/AIRrxpZ6ynyWGrMtjqA7LL/ywlP4/IT6MvpXc1zzjZnZCXMrhRRRUFnnnxu/5JXqX/XWD/0YtFHxu/5JXqX/AF1g/wDRi0V2Yf4TGpuafgz/AJEXw9/2DLb/ANFLW5WH4M/5EXw9/wBgy2/9FLW5XorY43uFFFFMQUUUUAFY914asJbh7u1MunXjnLXFk3ls59XXBV/+BA1sUUgMLzPEem/6yK31iAfxREW84H+6x2Mffcn0qS38UaXLMtvcSvYXTHAgvkMLMfRS3D/8BJrZqO4t4LqFobiGOaJhhkkUMp+oNAySisL/AIRa1tudJurzSj2S1kzF/wB+nDIPwAoz4msuo0/VIx6brWXH/j6sf++RQBu0Vh/8JPDb8anp+o6ce7TW5kjH/bSPcoH1IrRsdU0/U4/MsL62ukHUwSq4H5Gi4WLdFFFMQUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUVSv9Y0zSwDf6ha22egmlVSfoCeaof8JKLjjTNK1K+9HEHkx/XdKVBHuuaVx2NygkAEk4A6k1hbPE1796XT9LjPaMNcyY/3jtVT/wABalHhWwnIbVJbrVX6/wCmybo/+/QAj/8AHaAHS+KdN81oLJpdSuFODFYJ5u0+jMPkX/gTCo/+Kj1L/n20eA/S4uCP/QEP/fYrbiijgiWKGNI41GFRFAAHsBT6AMiz8N6fa3K3cqyXt6vS6vH82Rf93PCfRQBWvRRQIKKKKYBRRRQAVh+M/wDkRfEP/YMuf/RTVuVh+M/+RF8Q/wDYMuf/AEU1J7DW53tFFFcp0hRRRQAUUUUAFFFFABRRRQAUUUUAFcx8Rv8Akm3iT/sHTf8AoJrp65j4jf8AJNvEn/YOm/8AQTQBp2X/AB42/wD1yX+VT1BZf8eNv/1yX+VT15p0hXHeOT/aNzofhwfc1G7865HrBBiRgfq3lr+NdjXFg/bvilfyHldN0uGBfZ5XZ2/REq6a94io7RZ0lFFFbnGFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFZPibxBbeF/Dt5rF2C0dumQgOC7E4VR9SRQBrUV4do7fE/4h276za61FpGnu5ECKSgbBwdu1SSAeMsetXvD/jjxR4W8aQeFfGzx3CXJVbe8UDgscKcgDcpPHIyD9KrlHynsdFULjW9JtL37Fc6pZQ3ZQyeRJcIr7QCS20nOMAnPsabp+vaRqtrNdWGp2lzBDkSyRTKyp3+Y5449akRo0Vw3i7xrpz+DNam8P67aS6hbQ71NtMrsvzAZA5yOetWvhnq15rHw807UtTumnuZPOMkz4GQJXAz24AFO2lx2OvorGtPFvh2/vhZWmuafPdE4EUdwpZj6Dnn8Ku6hq2m6Skb6lqFpZrIdqG5mWMMfQbiM0hFyis621/R73U5tNtdTtJr6HPmW8cys6465APbv6VWn8XeHLbUDYT67p0d0DtMTXKAg+h54PtRYDaopNw27sjGM5rltY8Y6NLomsx6VrtjJqMFlPJGkM6s4ZUY5AzzgigDqqK8h8LeLNduvgvret3GoySalbySiKdlUlcKmOMY7ntXU/CvXNS8Q+B4dQ1W5NzdNNIpkKquQDxwoAptWG0drRWb4hNwvhzUjaXkVncC2k8q5mcIkTbThmJ6Adc1j+A5r8eFBLrGuWWqzrI5a8tZ1kj2jtuAA470WEdVRWDF428LT3QtovEOmPMTtCi5Tk+gOcGs34ieNV8F+HjdQfZpdQlYLBBNJgkd229WA4zj160WYWOworg28fxwfCo+JI7i1vtQgs4WnjidSEnk2jDhT8vzNyOOhritEsPid4u0RPENr4thiE25orZW2jgkYIVdo6d8+9PlHY9xorz34c+L9Z1HRr8eLrd7Kaw2lry5i8hJEOeSSAuQRyRxyKg+H/wAS/wDhLNb1eDUJrGzRHij0+280B5c79xBJy54XoOOOKVmFj0misnVPFOg6LOINT1iytZiM+XLMobHrjrip4dc0m40xtTh1OzewT71ys6+Wv1bOB+NIRforzb4efEr/AIS7VdUi1Caxs8SRpYWnmgO4+fdjJy54HQfhXa6p4l0PRJUi1PVrO0kcZVJplViPXBOce9NprQLGpRUNrdW97bJc2k8U8EgyksThlYexHBqakAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAc1Gf7H+JkLji3120MTj/p4gG5T+MZYf8AABXa1xHjg/ZtO03VRw2nanbTk/7DOI3/APHZDXb1jVWtzqou8QooorI1Ob8F/wDIweMv+wqn/pNDXY1x3gv/AJGDxl/2FU/9Joa7GvRh8KOd7hRRRVCCiiigAooooAKKKKACiiigAooooA4nwv8A8jP4z/7Cif8ApNDXU1y3hf8A5Gfxn/2FE/8ASaGuprgq/GzeOwUEgAknAHU0Vz/jm/fTfAmuXUZIlWzkWMjqHYbV/UioSu7FGP4Hzf2F94hk/wBbrN29yueohX5Il/74UH/gRrqKp6VYppmj2VhGAEtoEhUD0VQP6VcrqOFu7uFFFFAgooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKK5T4lane6N8PtV1DT7hre7hEflyrjK5lQHr7E15z4a0/4o+KPD9rrNn4vgigud21JiQw2sVOcRkdVPemlpcaR7jRXA22neL9I8CawmseJbI6mx3W19JLsjgX5R8zFRjnPY9azNX1jXdI+CMuonXre81WN1H9oWUqyowM4HDYwcKdp49aLBY9RoryDXvFWuWvwP0jW4NRlTUpnjElwANzAls9sdhXpHhS6nvvB+i3d1IZbiexgklc9WZkBJ/M0NWCxr0VyHxF8bp4I8Pi6SNZr64Yx20TfdzjJZvYf1A964Wy0D4t69Ypq0niSOxeZfMitWYoQDyAVVcD8cn1oSCx7TRXm3w08V+J9Svr7Q/FFhOLqzztvPI2qxU4KMQNpPOQR1Ga7PVPFGg6JMIdT1eytJSMiOWZQ2PXHXFDXQVjWoqrp+p2GrWoudOvLe7gJx5kEgdc+mR3qi3izw4ti97/b2mG1RgjTLdIyhuu3IPX2pAbFFVdP1Ox1a1F1p15BdwE4EkEgdc+mR3rPv/F/hzS7s2t9rmn29wpw0TzqGX6jPH40AbVFQ2t1b31slzaTxXEEgyksThlYexHBqagAorz/4c6xqPiHWPFOp3F5LLpy3xtrKIt8iKuSSB7gpV34p6vf6H4Bvb/Tbl7a6SSILImMgFwD19qdtbDtrY7OiuR8HeIoz8PNI1bX9TgieaP8AeXN1KsYZtzYGTgZwP0rVu/Fvh2xnigutc06GWVQyK9ygJUjIPXoexosKxs0Vy/jrxhD4Q8LzalGbaa6YAW0EkoXzSSASB1YANk47enWl8B+Jk8T+F7S6lvbWfUfL3XUUDrmIknAKg5XgcZ9KLaXC3U6eisU+L/DY1D7Add04XW7Z5RuUzu9OvX2rRvtRsdLtvtOoXlvaQZC+bcSrGuT0GSQKQFmisuPxLoUuox6fHrFi95KgeOFZ1LMpG4EDPORz9OajbxZ4cWxe9/t7TDao+xplukZQ2M7cg9cdqLAbFFVdP1Kx1a0F1p15Bd25OBJBIHXPpkd6q6p4k0TRHVNU1Wzs3cZVJplViPXHXHvQBqUVXsb+z1O1W6sLqC6t26SwyB1P4ivNPh34l1jV/iH4q0+/v5J7S0lkEETAYjAlKjGB6cU7BY9TorF1Hxf4c0m6Nrf63YW9wPvRSTqGX6jt+NS3viHTLTw/NrS31pJZpGXSUTqI5D2UN0yTx9aVgNWivP8A4Z/EA+MbW8bUbiygvjdMLeyjkAcRBFOQpO5ud2T/ACrqdR8VaBpF2LXUdZsLW4P/ACzlnVWHuQTx+NOz2Cxr0UyKaK4hSaGRJInG5XRgQw9QR1p9IAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKAM3xDpKa74fvtMc7ftEJRX/uP1VvwYA/hVjwjrD694U07UZhtuJIts6+kqkq4/wC+larVYHgg/ZdR8T6V/DbamZ4x6JMiyf8AoReoqK8Tai9bHX0UUVznSeefG7/klepf9dYP/Ri0UfG7/klepf8AXWD/ANGLRXZh/hMam5p+DP8AkRfD3/YMtv8A0UtblYfgz/kRfD3/AGDLb/0Utbleitjje4UUUUxBRRRQAUUUUAFFFFABRRRQAVnX2gaRqUnm3mm2s0o6StGPMH0Ycj8DWjRSAw/+EbaD/kH61qtoOyGcXC/lMHOPoRR5Xie2+5daXfqOiyxPbt+LKXH/AI6K3KKLDuYf9satb/8AH34cuWHd7K4jmUf99FGP4LR/wlmlx8XQvLM9/tVlLGo/4EV2/ka3KKAM+z13SNQwLLVLK5J7Q3COf0NaFU7zSdN1AH7bp9pc5/57Qq/8xWf/AMIhoaf8e9o9p/153Elvj/v2y0ahoblFYf8Awjjx/wDHrrus2/p/pCzf+jVej+y9dj/1PiMv/wBfNlG//oGygDcorD8vxTH/AMvWj3H/AG7Sw/8As70fafFEf3tK0qUeqahIp/Iw/wBaLhY3KKw/7U1xPv8Ahxm/643sbf8AoW2j+3b9f9Z4W1ce6vbMP0mz+lFwsblFYf8Awkbr/rNC1lP+3dW/9BY0f8JPCPvaXrI/7h8h/kDRcLG5RWH/AMJTZf8APlrP/gpuf/iKP+Er07/n21n/AME13/8AGqLoLM3KKw/+Er089LXWSfT+xrsfzio/4Smz7WOsk+n9lXA/mlF0FmblFYf/AAk8J+7pest/24SD+YFH/CRSH7mgay//AGxRf/QnFFwsblFYf9uai33PC2q/V5bVR/6Oz+lH9p68/wDq/DoX/rtfIv8A6CGouFjcorD+0eKJOmm6RCPVr+SQ/kIh/OjyfFMnW+0e3HtZySn/ANGL/Ki4WNyisP8AsnWpP9d4kmT/AK9rOJP/AEMPR/wjQk/4+tZ1m49f9LMP/ooJQBuVm3fiLRLFtt3q9jC3TbJcICfoM5qr/wAIhoLf6/T1u/8Ar8ke4/8ARhatO006xsF22dlb249IYlT+Qo1DQzP+Er0+Ti0h1C8PY29jKyn/AIGVC/rR/a2tXH/Hp4dkjHZr+7jiH/kPzD+lblFAGH9n8TXP+t1DTrJT1W3t2mcfR3YD/wAco/4RlJ/+Qhquq33qr3Pkqf8AgMQQEexzW5RRYLlCw0TStLJax061t3PV44lDN9T1P41foooEFFFFMAooooAKKKKACiiigAooooAKw/Gf/Ii+If8AsGXP/opq3Kw/Gf8AyIviH/sGXP8A6Kak9hrc72iiiuU6QooooAKKKKACiiigAooooAKKKKACuY+I3/JNvEn/AGDpv/QTXT1zHxG/5Jt4k/7B03/oJoA07L/jxt/+uS/yqeoLL/jxt/8Arkv8qnrzTpCuK0L95408Yzf9PlvEP+A20f8AUmu1rivDf/IzeMP+wmn/AKTxVpS3ZlW+E6WiiitjlCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACvNfjnHK/w6LRg7UvImkx2X5h/MrXpVU9V0y01rS7nTb6IS2twhSRfb1HoR1B9RTTswW5g/Dae3uPhzoTW5UotqqNjs68N+oNea/G91ufGPhmztSDfAdF6/NIoT9Q1XYPhn478KTzxeEfE0K2ErbvLnOCPcqVZc+4xnFbHg/wCFd1YeIf8AhJPFOpjU9VVt8YUlkRuzFmwSR2GAB+WK0TuVonc5T4haTFrnx00rTJ2dYbmGFJdhwSnzlhn3GR+NXPizpVj4K8CQaRoEDWlpqF95lwokZt21PuksScZAOM9q67VvAmo3/wAVtN8Vx3NqtlaxqrxMzeYSAw4GMfxDvW3458IW/jXw4+mSy+TMriWCbGdjgEcjuCCQfrRfYL7Hn3jH4b+HNJ+Fcl7ZWwjv7SCOT7WHO6UkqG3c4IOTx24xXN6lqt1p/wCzvoNtbO0a3t3LFMynGUEkrFfxIH5V07/Dnx/qvh1tB1jxJZnToI9tvFECTKy/cDtsB2g4Pc8fQjeh+GJuvhVa+EtTuYhd27vLHcwAsqOXdgcEAkYYg/U079x3PMdU0eyuvCFpbaL4B8S2+rxLG63/ANkkIkPG4kjPB5IwOOOla3xVutQuvhv4Rn1SKaHUCSJ1mQo+8IASQeQTjP41ujwL8SrywtNDvPE9nBpdsy7Z7dmExVeFGQoJwOxI+prc+Ifw9v8AxV4f0jTNOvow1gfmlvpXZpBtAyWAJJ4yaLq4XMzVvC2mfD34d6lrmjRSR6w1gkEl2ZWZmMjoGYZOFOTnjHSvPdC07SbrwE8MngfxBf6ncpIyanb2zugfJClSDjAwM8cnNfRGqaPb6zoFxpF6C0FxD5T7eo46j3B5H0ryy3+H/wARdH0mbw7pXiLTxo0hYLI+5ZEVj8wHykrnJ4B7nkZpJiTMa81TxDpnwBks9SgvbO4F6LEfaY2jkMBAf+IA7eq/QYrXj+G/hz/hTf8AaQtgNSOl/b/tm87g/l78dcbf4cY6e/Ndq/gGK7+HreF9R1O6vJHXcb2Zi7iTOQQCThQeNuen1zXHWfw58fHRJPDV54ls00IKVVYgTI69QmSmQuevJwOORxRcLmT4O/5N58Sf9dJv/QY67P4I/wDJNrf/AK+Jf/Qqj0H4eappXwt1bwvNdWbXl48jJIjN5Y3BQMkrn+E9q57Sfhv8StCsFsdM8U6fbWysWEaM+AT16x0OzB2Z6T47/wCRA8Qf9g+b/wBANeWeGNG1PX/2f7rT9JJN0907CMNt81Qykrn3x/Sux0zwp4zk8N+IdN8Q67a38l/amG0YFtsTFWBLfIODlfXpUej/AA3v7X4aSeGbjVjbXvntNHdWUjgKT0B+6SOuRQtEGx5P9s0Gw0y30bxZ4FutPli2hr+1JimfHUkOMNnvzjnjHFdr8ULTQ9Q+FGkaxpxkuUgMcNpcyuxcRnIIb1OVAORnirN14I+JWpaCPDV9rekPpWFRpiGaUopBA+5kkYHf8a6TWvhwt18MIfCOnXQR7ba8U0wOHcMWYtjOASzdM4z3p3Q7lPw/o/hHRfhTbPqaRWllq1lAb2R5GxJIyA55Jwc5IxXJxfCfW7KFNV8C+LGa1nUSxAyPCXU9OV4b8QK67QfAuuXXgm98M+ML62ntTHHFZfZfvW4TODnaMkEL1z0Nc9Z+Afif4etTpeieJ7Mabk7N5IKgnsCjFfX5TSv5iv5l74deJ7/xh/bXg/xfCtzPbIyyt90uobY6ttxyDjkda574IeH9Lvtb1u9ubQSXGmzxG0few8rJkB4Bweg656V6B8Ovh3/whaXd5e3gvdVvP9bKM7VGc4BPJyeST1449cHwt8PPFvhLxjcXGnarYjRLq5Elwhz5kkYLELgocEbj0IoutbBdamHdL4Kl8X6wbDQta8X6lLITN/FDE2TnDAAgZ4yQRxwaZ8HNPg1G+8X6JfWjJYSbFlsmlJ2Ydxt3DBJHTPfFbOk/Dnxr4S1fUV8M6zpsenXzAl7lC0iAE4O3aQWG498Hvitn4e/D7VfBniHV7q6v7e8tb1flkBYSlg2csCMc5OeTTbVht6HFfA3QtMvL/V9SubQSXWnSxm1fewMeQ+eAcHoOua5/wjeWeuavrOseIPC2r+I7i4cY+xwtIkOc5BwRg4wB6AcV6F4R+Hni3wf4smksdVsf7CuJw9whz5kkY3bRgocEbuxob4deK/C+v3994H1ayhs75t0ltdg/IckgD5SCBk4PBwcc0XVwuQfB231fS9d1uwl0nVrHRZcz2i39u6bCGwBkjG4qRnHXbXsFcd4C8J6r4bt7qbW9bn1K+u33OvmsYYuSTtB7knk4HYY9exqJO7Je4UUUUhBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFAHMfERC/wAPddx1W1Zx7Ffm/pXaxuJIkcdGUEVx/j7/AJJ/4g/68Jf/AEE11dl/x42//XJf5VlV2R0UOpPRRRWJuc34L/5GDxl/2FU/9Joa7GuO8F/8jB4y/wCwqn/pNDXY16MPhRzvcKKKKoQUUUUAFFFFABRRRQAUUUUAFFFFAHE+F/8AkZ/Gf/YUT/0mhrqa5bwv/wAjP4z/AOwon/pNDXU1wVfjZvHYK5L4k/N4Mlh/57XlnH+dzHn9K62uS+I3/Is2/wD2E7L/ANKI6mHxIJbM3aKKK6TiCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooA4n4u/8kt1r6Q/+jkrzjwP4W8faj4OsLvRfFUVjp8nmeVbszZTEjA9FPUgn8a9c8c6Bc+J/BuoaNaSxRT3IQK8xIUbZFY5wCeintSeBdAufC/g3T9GvJYpZ7bzNzwklTukZhjIB6MO1UnZFJ2Rz3iey1TTvgpqVprV8t9qMdu/m3Ck4fMmR1A6KQPwri/+bXT/ANdP/buvXPF+jT+IfCepaTbSRxz3UWxGlJCg5B5wCe1cj/wrzU/+FO/8Id9qs/t2/d5u5vK/1/mdduenHTrTTBM5HxN/yblof/XSL+b1614J/wCRD8Pf9g23/wDRa1yer/DzU9Q+FOneFIrqzW9tWRnlZm8s4LdDtz39K7fw9p0ukeGtL02dkeW0tIoHZM7SyoFJGe3FJvQTeh5D8fUdNR8N3MqlrRTKGHbOUJH4j+Vem+K7rxC3h6K48HxW91eSSIy+YV2GIgnIJIH93vU/izwrYeMNCk0u/DKpIeKVPvROOjD8yMdwa81tvAPxP0O2/szRvFdqdOHyx+YxBRfYFGK/RTQrND3RY8GeO/F198Rj4Z8Qx2cTRo5lSJBkMF3D5gSO4rKvl8Fy+NNXay0PWfF2qSyEzL96GFsnOGABA7ZIIGODXV+BfhdL4a1t9f1fV5NQ1V1YZGdoLdSWblj78Vk6b8OvGnhTXdSfwvrGmx6ffvlmulLOgBJHy7SCRuPfB7indXDQzfgcZIPFXiizEDWkSkZtS+/yiHYBc9yMkZ71kfBfwfo3idNVn1mD7WlqyLFAzsFUuDlsAjn5QPwr0DwB8PdW8HeJ9VvrrULe9tb2PHmfMJWfdncwxjnLdz2rzL4U6N4nu7TVNQ8K6vDZXkTpFJDcJujmUgkdjggjjjueR3d73sPudH4Etz4Y+NGueGtNmcafJE+1CdwQhVdT7ldxX8a5WGxuvBFxfweLvBH9rxTSFjfMz9D1KyAEc9ex559vV/Anw4utCvdR1nX78Xus6grI7wsQEVjliGwDuPHIAxjisy28GfEXw1cXcPh/xHZ3djcSbwdSLNIh6ZOVPOMDg846UXC5pfCG78My6BdQeHJL9Qkoe4t71wzRswwCMcYO3t6V1vivVP7E8J6rqQba9vauyH/bxhf1IrA+HPgF/BdrezXd2l1qN84ad41wigZwF/Fic4HbjirfxG0DVPE/hKTSNKeFJLiaPzWmcqBGp3HoDzkLxUu1yepW+E2lf2V8ONLVlxJcq10/vvOVP/fO2qvxo/5JlqH/AF1h/wDRi13Npax2VlBaQjEUEaxoPRVGB/Kue8f+G7rxZ4QutIspYYp5XjZXmJCjawJzgE9vSknrcL63PK/E3/Juegf9do/5yVZ1jwHoVv8ABJNZFqW1ZrOC7a7Z2Lln2kjrjGGxj+tdPq/w61TUPhTpnhWK6s1vbSRXeVmbyyAW6Hbn+Idq3dU8KXl98Lk8LxzQLeCxgtvMYny9yBcnOM4+U9qq47nmWqWdvqf7OGm6leRmW8sQUt5WY5jBuNhGM4PygDn0qzHb2/hb4CSaxpEJttR1OCKK5uEdssDIRnk4HBYcY612MXw+u2+Dw8G3F3ALwKxEybjHu84yr1AOOgPHrVPwz4B8Qf8ACI6h4Y8VX9pNpkkKxWaW3LQkMW3ZKDPO0jJPSi6C55pY6Xpd18PltofAniG51WWIvHqkVq7I0mcggg4KdB06e9b/AIil1Z/2frWLWre6gvLe8SHF1GyOUBO0kNz0IGfatWHwB8SLLRj4btPEenro+cLKNyyqmc4GFyOe2fbOK6bxN4BvtV+HNv4ZtdTNxdRSI73d/I5MhBJPPzEdeB2FNtDuU/ht4C0Sx0DRvERt3k1eS2843Dysf9YvTbnHCnHTNcH8F/B2jeJodUuNZg+1rbMiRQM7BVLA7mwCOTtA/CvcfD2my6N4W03TJ3R5rS0jhdkztJVQCRntxXgPwq0bxRd2Oqah4V1eGyu43SKWG4TMcqkEg9Dgg5xx36juk7piT3Og8DA+EPi94h0GwaR9NEEkghLZwVAdfxAJXPvVf4W+GdO8f3Wt6/4nQ6hcmcKI3kYAEjJPBHsAOgA/Luvh98Pbvw3qN9rmuXyX2tXuQ7x5KoCctyQMkkDsMYwKxJPhn4p8Ma5eX3gTW7a1tbs5e2uRwnJIA+VgQMnB4IHHPd3QXMvwjCfBvxzvfDGmyyHSrlCTCzEhP3Xmr+I5XPoan+FjBfij42ZmCqJZiWPb983NdT4E+Hdx4f1a78Qa7qC6jrl0CGkXOyMHrgkDJOAOgwOAKi8LfDy+0fxP4n1C/ubaS01jzVRIGbequ5bnKgA4PbNJtA2jhLS38GXF5qcOheGNb8XXEshMl3O21EJz0kAGMnJyRmnfCuxt9Y+G3iuw1CMz2sbCZIyxADhCQeD6qv5VvaB8O/HnhlLzSNK1/ToNJupCzTmMtMuRgsq7eGwB/FjjrW38Pvh7f+ErDXNOv7q2ntb/AAInhLbwMMp3AgDOCOhPem2rDbON+D2nWFp4M1rxSLRX1WwacQzFm4UQq23GcHknt3rnPBI0zUNP1K91vwhrviK7up2DXdrC0ipwCfmB4fJyfbFei+Afh/4r8JajLY3WpafP4dlaRp4EBLykptB5TI7ZG7HHeqtr8O/G/hK4vrfwdrtkumXblgl2DviPQH7rDIHGR1wOKLoLlr4Jxa3Yafq2mapp+o2lpDKklmL2Bozht24DcPZTgdyfWvVa5fwL4XvfC2itb6jq9xqV3K293klZkT0VAx4Hv3/KuoqG7sl7hRRRSEFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABXPaB+7+I/iZP+etnYy/l5yn+Qroa57R/+Sn65/2DLT/0Oapn8LNKXxHY0UUVzHWeefG7/klepf8AXWD/ANGLRR8bv+SV6l/11g/9GLRXZh/hMam5p+DP+RF8Pf8AYMtv/RS1uVh+DP8AkRfD3/YMtv8A0Utbleitjje4UUUUxBRRRQAUVkXfiCGO6eysLebUb5OHhtsYiP8A00c4VPoTn0BqMafr9/8ANe6pFp0Z/wCWOnxh3HsZZAc/gi/Ws5VYx3HZm3RWMPCOkvzdLdXrHqbu7llB/wCAlto/ACl/4QvwsfveHNJc+r2cbH8yKy+sLsPlNiisf/hC/Cv/AELOjf8AgBF/8TR/whfhX/oWdG/8AIv/AImj6x5BymxRWP8A8IX4V/6FnRv/AAAi/wDiaP8AhC/Cv/Qs6N/4ARf/ABNH1jyDlNiisf8A4Qvwr/0LOjf+AEX/AMTR/wAIX4V/6FnRv/ACL/4mj6x5BymxRWP/AMIX4V/6FnRv/ACL/wCJo/4Qvwr/ANCzo3/gBF/8TR9Y8g5TYorH/wCEL8K/9Czo3/gBF/8AE0f8IX4V/wChZ0b/AMAIv/iaPrHkHKbFFY//AAhfhX/oWdG/8AIv/iaP+EL8K/8AQs6N/wCAEX/xNH1jyDlNiisf/hC/Cv8A0LOjf+AEX/xNH/CF+Ff+hZ0b/wAAIv8A4mj6x5BymxRWP/whfhX/AKFnRv8AwAi/+Jo/4Qvwr/0LOjf+AEX/AMTR9Y8g5TYorH/4Qvwr/wBCzo3/AIARf/E0f8IX4V/6FnRv/ACL/wCJo+seQcpsUVj/APCF+Ff+hZ0b/wAAIv8A4mj/AIQvwr/0LOjf+AEX/wATR9Y8g5TYorH/AOEL8K/9Czo3/gBF/wDE0f8ACF+Ff+hZ0b/wAi/+Jo+seQcpsUVj/wDCF+Ff+hZ0b/wAi/8AiaP+EL8K/wDQs6N/4ARf/E0fWPIOU2KKx/8AhC/Cv/Qs6N/4ARf/ABNH/CF+Ff8AoWdG/wDACL/4mj6x5BymxRWP/wAIX4V/6FnRv/ACL/4mj/hC/Cv/AELOjf8AgBF/8TR9Y8g5TYorH/4Qvwr/ANCzo3/gBF/8TR/whfhX/oWdG/8AACL/AOJo+seQcpsUVj/8IX4V/wChZ0b/AMAIv/iaP+EL8K/9Czo3/gBF/wDE0fWPIOU2KKx/+EL8K/8AQs6N/wCAEX/xNH/CF+Ff+hZ0b/wAi/8AiaPrHkHKbFFY/wDwhfhX/oWdG/8AACL/AOJo/wCEL8K/9Czo3/gBF/8AE0fWPIOU2KKx/wDhC/Cv/Qs6N/4ARf8AxNH/AAhfhX/oWdG/8AIv/iaPrHkHKbFFY/8AwhfhX/oWdG/8AIv/AImj/hC/Cv8A0LOjf+AEX/xNH1jyDlNiisf/AIQvwsOnhvSF91so1P6Ck/4RDR05tY7mzbsbS7lhA/BWAP4ij6wuwcps0ViHTddsPmsNWS9Qf8sNSjGT7CWMAj6lWp9r4giN0llqVtLpt7IcRxzkFJj/ANM5B8rfThvUCtY1Yy2FY2KKKK0EFFFFABWH4z/5EXxD/wBgy5/9FNW5WH4z/wCRF8Q/9gy5/wDRTUnsNbne0UUVynSFFFFABRRRQAUUUUAFFFFABRRRQAVzHxG/5Jt4k/7B03/oJrp65j4jf8k28Sf9g6b/ANBNAGnZf8eNv/1yX+VT1BZf8eNv/wBcl/lU9eadIVxXhv8A5Gbxh/2E0/8ASeKu1rivDf8AyM3jD/sJp/6TxVpS3ZjW+E6WiiitjmCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigDnfH3/JP/ABB/14S/+gmursv+PG3/AOuS/wAq5Tx9/wAk/wDEH/XhL/6Ca6uy/wCPG3/65L/Ks6uyOih1J6KKKwNzm/Bf/IweMv8AsKp/6TQ12Ncd4L/5GDxl/wBhVP8A0mhrsa9GHwo53uFFFFUIKKKKACiiigAooooAKKKKACiiigDifC//ACM/jP8A7Cif+k0NdTXLeF/+Rn8Z/wDYUT/0mhrqa4Kvxs3jsFcl8Rv+RZt/+wnZf+lEddbXJfEb/kWbf/sJ2X/pRHSh8SCXws3aKKK6DiCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigArC8NeD9G8IxXEWj27wrcMrSBpWfJGcdT7mt2igAooooAKKKKACiiigAooooAKKKKACiiigAIyCDWF4a8IaN4RhuItHt3hS4YNIGlZ8kcDqfet2igAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAK57R/+Sn65/wBgy0/9Dmroa57R/wDkp+uf9gy0/wDQ5qmfws0pfEdjRRRXMdZ558bv+SV6l/11g/8ARi0UfG7/AJJXqX/XWD/0YtFdmH+Exqbmn4M/5EXw9/2DLb/0UtblYfgz/kRfD3/YMtv/AEUtbleitjje4UUUUxDXdY0Z3YKiglmY4AHqa59HvPFPMEs1loh6SplJrwf7J6pH/tfebtgYJJU/4SjUpbU5Oi2cmy49LuYdY/dF/i9W+XoGB6YDAwOlctWr9mJaRXsrG1061S1sreOCBPuxxrgCrFFFcwwooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKgvLK21C0ktbyCOeCQYaORcg1PRQBzLS3XhVgLqaW60MnAuJDuls/wDroerR/wC0eV/iyOR0IIIBByD0IqRlV0ZHUMrDBBGQRXOWAbw9qkejuSdNucnT3Y58pgMtAT6AAsnsCP4Rnqo1b+6xNHQUUUV0kBWH4z/5EXxD/wBgy5/9FNW5WH4z/wCRF8Q/9gy5/wDRTUnsNbne0UUVynSFFFFABRRRQAUUUUAFFFFABRRRQAVzHxG/5Jt4k/7B03/oJrp65j4jf8k28Sf9g6b/ANBNAGnZf8eNv/1yX+VT1BZf8eNv/wBcl/lU9eadIVxXhv8A5Gbxh/2E0/8ASeKu1rivDf8AyM3jD/sJp/6TxVpS3ZjW+E6WiiitjmCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigDnfH3/JP/ABB/14S/+gmursv+PG3/AOuS/wAq5Tx9/wAk/wDEH/XhL/6Ca6uy/wCPG3/65L/Ks6uyOih1J6KKKwNzm/Bf/IweMv8AsKp/6TQ12Ncd4L/5GDxl/wBhVP8A0mhrsa9GHwo53uFFFFUIKKKKACiiigAooooAKKKKACiiigDifC//ACM/jP8A7Cif+k0NdTXLeF/+Rn8Z/wDYUT/0mhrqa4Kvxs3jsFcl8Rv+RZt/+wnZf+lEddbXJfEb/kWbf/sJ2X/pRHSh8SCXws3aKKK6DiCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigArntH/wCSn65/2DLT/wBDmroa57R/+Sn65/2DLT/0Oapn8LNKXxHY0UUVzHWeefG7/klepf8AXWD/ANGLRR8bv+SV6l/11g/9GLRXZh/hMam5p+DP+RF8Pf8AYMtv/RS1uVh+DP8AkRfD3/YMtv8A0Utbleitjje4Vk+IL2e2sEtrJtt/fSC2tjjOxiCS+PRVDN/wHHetasSEfb/Gk0h5i0y1WNPTzZTlvxCIn/fZqakuWLYLc1tPsLfTNPgsrVSsMKBFyck+5Pck8k9yas0UV55YUUUUAfPPxG0+/wBW+NMelWt+9s92kKxuHYCP5Pb6H861f+FLeKv+hv8A/Hpf8aPEv/Jx+j/9sf8A0E17jVttJFN2PDv+FLeKv+hv/wDHpf8AGj/hS3ir/ob/APx6X/GvcaKXMxczPDv+FLeKv+hv/wDHpf8AGj/hS3ir/ob/APx6X/GvcaKOZhzM8O/4Ut4q/wChv/8AHpf8aP8AhS3ir/ob/wDx6X/GvcaKOZhzM8O/4Ut4q/6G/wD8el/xo/4Ut4q/6G//AMel/wAa9xoo5mHMzw7/AIUt4q/6G/8A8el/xo/4Ut4q/wChv/8AHpf8a9xoo5mHMzw7/hS3ir/ob/8Ax6X/ABo/4Ut4q/6G/wD8el/xr3GijmYczPDv+FLeKv8Aob//AB6X/Gj/AIUt4q/6G/8A8el/xr3GijmYczPDv+FLeKv+hv8A/Hpf8aP+FLeKv+hv/wDHpf8AGvcaKOZhzM8O/wCFLeKv+hv/APHpf8aP+FLeKv8Aob//AB6X/GvcaKOZhzM8O/4Ut4q/6G//AMel/wAa4rxj4R1rwtrml6dd6691Le42SK7/ACfMF7n3r6mrw740/wDI9+Ffw/8ARoqoybZSbbD/AIUt4q/6G/8A8el/xo/4Ut4q/wChv/8AHpf8a9xoqeZk8zPDv+FLeKv+hv8A/Hpf8aP+FLeKv+hv/wDHpf8AGvcaKOZhzM8O/wCFLeKv+hv/APHpf8aP+FLeKv8Aob//AB6X/GvcaKOZhzM8O/4Ut4q/6G//AMel/wAaP+FLeKv+hv8A/Hpf8a9xoo5mHMzw7/hS3ir/AKG//wAel/xo/wCFLeKv+hv/APHpf8a9xoo5mHMzw7/hS3ir/ob/APx6X/Gj/hS3ir/ob/8Ax6X/ABr3GijmYczPDv8AhS3ir/ob/wDx6X/Gj/hS3ir/AKG//wAel/xr3GijmYczPDv+FLeKv+hv/wDHpf8AGj/hS3ir/ob/APx6X/GvcaKOZhzM8O/4Ut4q/wChv/8AHpf8aP8AhS3ir/ob/wDx6X/GvcaKOZhzM8O/4Ut4q/6G/wD8el/xo/4Ut4q/6G//AMel/wAa9xoo5mHMz5t8afDvxB4T8PPql54ka7iWRU8pXkzk9+TXs3wy8z/hW+hmWQu7QFixOTyxIrF+N/8AyTib/r5i/ma3Phr/AMk40L/r1H8zTbvEbd0dVRRRUEhVDWtMXVtLltd5il4eCYdYpVOUcfQgH36VfooAy9E1E6ppMNzJH5U/Mc8X/POVSVdfwYEVoVh2I+w+LNUshxFdxx30Y/2/9XIB/wB8xn6ua3K9CEuaKZD3CsPxn/yIviH/ALBlz/6KatysPxn/AMiL4h/7Blz/AOimqnsC3O9ooorlOkKKKKACiiigAooooAKKKKACiiigArmPiN/yTbxJ/wBg6b/0E109cx8Rv+SbeJP+wdN/6CaANOy/48bf/rkv8qnqCy/48bf/AK5L/Kp6806QrivDf/IzeMP+wmn/AKTxV2tcV4b/AORm8Yf9hNP/AEnirSluzGt8J0tFFFbHMFFFFABRRRQAUUUUAFFFFABRVO+1fTdLAOoahaWgPQ3Eyx5/Mis//hNPCv8A0M2jf+B8X/xVFgNyisP/AITTwr/0M2jf+B8X/wAVR/wmnhX/AKGbRv8AwPi/+KosFjcorD/4TTwr/wBDNo3/AIHxf/FUf8Jp4V/6GbRv/A+L/wCKosFjcorD/wCE08K/9DNo3/gfF/8AFUf8Jp4V/wChm0b/AMD4v/iqLBY3KKw/+E08K/8AQzaN/wCB8X/xVH/CaeFf+hm0b/wPi/8AiqLBY3KKw/8AhNPCv/QzaN/4Hxf/ABVH/CaeFf8AoZtG/wDA+L/4qiwWNyisP/hNPCv/AEM2jf8AgfF/8VR/wmnhX/oZtG/8D4v/AIqiwWNyisP/AITTwr/0M2jf+B8X/wAVR/wmnhX/AKGbRv8AwPi/+KosFjcorD/4TTwr/wBDNo3/AIHxf/FUf8Jp4V/6GbRv/A+L/wCKosFjcorD/wCE08K/9DNo3/gfF/8AFUf8Jp4V/wChm0b/AMD4v/iqLBY3KKw/+E08K/8AQzaN/wCB8X/xVH/CaeFf+hm0b/wPi/8AiqLBY3KKw/8AhNPCv/QzaN/4Hxf/ABVH/CaeFf8AoZtG/wDA+L/4qiwWNyisP/hNPCv/AEM2jf8AgfF/8VR/wmnhX/oZtG/8D4v/AIqiwWNyisP/AITTwr/0M2jf+B8X/wAVR/wmnhX/AKGbRv8AwPi/+KosFjcorD/4TTwr/wBDNo3/AIHxf/FUf8Jp4V/6GbRv/A+L/wCKosFjcorD/wCE08K/9DNo3/gfF/8AFUf8Jp4V/wChm0b/AMD4v/iqLBY3KKw/+E08K/8AQzaN/wCB8X/xVH/CaeFf+hm0b/wPi/8AiqLBY3KKw/8AhNPCv/QzaN/4Hxf/ABVH/CaeFf8AoZtG/wDA+L/4qiwWNyisP/hNPCv/AEM2jf8AgfF/8VR/wmnhX/oZtG/8D4v/AIqiwWNyisP/AITTwr/0M2jf+B8X/wAVR/wmnhX/AKGbRv8AwPi/+KosFjcorD/4TTwr/wBDNo3/AIHxf/FUf8Jp4V/6GbRv/A+L/wCKosFjcorD/wCE08K/9DNo3/gfF/8AFUf8Jp4V/wChm0b/AMD4v/iqLBY3KKw/+E08K/8AQzaN/wCB8X/xVH/CaeFf+hm0b/wPi/8AiqLBY3KKw/8AhNPCv/QzaN/4Hxf/ABVH/CaeFf8AoZtG/wDA+L/4qiwWNyisP/hNPCv/AEM2jf8AgfF/8VR/wmnhX/oZtG/8D4v/AIqiwWNyisP/AITTwr/0M2jf+B8X/wAVTk8YeGJG2x+I9IdvRb6In/0KiwWNqimRSxzxLJFIskbDKshyD9DT6ACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKAOd8ff8k/8AEH/XhL/6Ca6uy/48bf8A65L/ACrlPH3/ACT/AMQf9eEv/oJrq7L/AI8bf/rkv8qzq7I6KHUnooorA3Ob8F/8jB4y/wCwqn/pNDXY1x3gv/kYPGX/AGFU/wDSaGuxr0YfCjne4UUUVQgooooAKKKKACiiigAooooAKKKKAOJ8L/8AIz+M/wDsKJ/6TQ11Nct4X/5Gfxn/ANhRP/SaGuprgq/GzeOwVyXxG/5Fm3/7Cdl/6UR11tcl8Rv+RZt/+wnZf+lEdKHxIJfCzdoooroOIKKKKACiiigAoorMvPEGl2NwbaW633IGTBAjTSAepRAWH5UJXA06Kwv+Et03/n31n/wS3n/xqj/hLdN/599Z/wDBLef/ABqq5ZdgN2isL/hLdN/599Z/8Et5/wDGqP8AhLdN/wCffWf/AAS3n/xqjll2A3aKwv8AhLdN/wCffWf/AAS3n/xqj/hLdN/599Z/8Et5/wDGqOWXYDdorC/4S3Tf+ffWf/BLef8Axqj/AIS3Tf8An31n/wAEt5/8ao5ZdgN2isL/AIS3Tf8An31n/wAEt5/8ao/4S3Tf+ffWf/BLef8Axqjll2A3aKwv+Et03/n31n/wS3n/AMao/wCEt03/AJ99Z/8ABLef/GqOWXYDdorC/wCEt03/AJ99Z/8ABLef/GqP+Et03/n31n/wS3n/AMao5ZdgN2isL/hLdN/599Z/8Et5/wDGqP8AhLdN/wCffWf/AAS3n/xqjll2A3aKwv8AhLdN/wCffWf/AAS3n/xqj/hLdN/599Z/8Et5/wDGqOWXYDdorC/4S3Tf+ffWf/BLef8Axqj/AIS3Tf8An31n/wAEt5/8ao5ZdgN2isL/AIS3Tf8An31n/wAEt5/8ao/4S3Tf+ffWf/BLef8Axqjll2A3aKwv+Et03/n31n/wS3n/AMao/wCEt03/AJ99Z/8ABLef/GqOWXYDdorC/wCEt03/AJ99Z/8ABLef/GqP+Et03/n31n/wS3n/AMao5ZdgN2isL/hLdN/599Z/8Et5/wDGqP8AhLdN/wCffWf/AAS3n/xqjll2A3aKwv8AhLdN/wCffWf/AAS3n/xqj/hLdN/599Z/8Et5/wDGqOWXYDdorC/4S3Tf+ffWf/BLef8Axqj/AIS3Tf8An31n/wAEt5/8ao5ZdgN2isL/AIS3Tf8An31n/wAEt5/8ao/4S3Tf+ffWf/BLef8Axqjll2A3aKwv+Et03/n31n/wS3n/AMao/wCEt03/AJ99Z/8ABLef/GqOWXYDdorC/wCEt03/AJ99Z/8ABLef/GqP+Et03/n31n/wS3n/AMao5ZdgN2isL/hLdN/599Z/8Et5/wDGqP8AhLdN/wCffWf/AAS3n/xqjll2A3aKwv8AhLdN/wCffWf/AAS3n/xqj/hLdN/599Z/8Et5/wDGqOWXYDdorC/4S3Tf+ffWf/BLef8Axqj/AIS3Tf8An31n/wAEt5/8ao5ZdgN2isL/AIS3Tf8An31n/wAEt5/8ao/4S3Tf+ffWf/BLef8Axqjll2A3aKwv+Et03/n31n/wS3n/AMao/wCEt03/AJ99Z/8ABLef/GqOWXYDdorC/wCEt03/AJ99Z/8ABLef/GqP+Et03/n31n/wS3n/AMao5ZdgN2isMeLNNY4+z6uPro12B+sVaGn6rYaormyuo5TGcSIOGQ+jKeVP1FJprcC5RRRSAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACue0f/AJKfrn/YMtP/AEOauhrntH/5Kfrn/YMtP/Q5qmfws0pfEdjRRRXMdZ558bv+SV6l/wBdYP8A0YtFHxu/5JXqX/XWD/0YtFdmH+Exqbmn4M/5EXw9/wBgy2/9FLW5WH4M/wCRF8Pf9gy2/wDRS1uV6K2ON7hWN4Z/ef2xdHrPqc2f+2e2EfpHWzWN4S/5A1we51K//wDSuUVhiPhHE3aKKK5CgooooA8O8S/8nH6P/wBsf/QTXq3ittfTQJT4aSJ9T3rsEuNu3PzdSB0rynxL/wAnH6P/ANsf/QTXuNU+hT6HguseOfihoOs2Ok6gmnx3l6VECCNGDbm2jkHA5rs9HuPietrqz6zb2COtk5stjR4M+RtB+bpjPXiuZ+K3/JWvB/8Avwf+j69K8ff8k+1//rwl/wDQTTfQGO8IXety+G0n8T/Z4r/zGD+U6FAufl5UkfrW1JfWkJjEt1Ahl/1YaQDf9PWvBrX/AJNlvv8Ar6H/AKPSoLjwJp0vwVHie5muZtWWBHjkeUlUjEgQIF6Y2/rRyhY+gJ7y1tWRbi5hhLnCCRwu4+2etT189xeDLTWPg3N4s1O6vLrVxAzQySTEiJI3KKgHphf1qzd+LNUsP2fNKaK6lFzdXDWRnDHesQZzgH6KF+lLlDlPc4760luGt47qB5l+9GsgLD6jrU7usaF3YKqjJJOABXzBe2Xhe18KWlxoQ1xPE0Ijk+0eS6q78bsHoAMkgj0HWuw+IM3iHxJ8O/C+pi0up7Xbv1S3hBUs42gEgDIHD84wMj2p8ocp7RbX9neFha3cE5XqIpA2PyNch42+IC+FdW0jSoLVJ7nUJlRnd8LCm5QSQOScHjpXk+gX3w8uPEWmXVlLq3he8gcZ/eebG7ZGAXJJA6g5AGDzWl8YPDth/wALF0WX97u1iRFuvm7ApGNvp8tCirhbU9686LyjL5ieWBnfuGMfWobbULK8Zltby3nZeoilViPyNeaePfDnhvSPB2k6Nf8AiK50vSbWRisCr5r3XO4jA5OMnnGBu57V5nqcmi6V4r8PX/hHT9Y02MzgGa8DKs5DL9zJORg4P1HFJRuCVz6dM0ayCMyIJD0UsMn8KZDeW1xI8cFzDK8Zw6o4Yr9QOleI/EixfU/jfoVglzLb/abWKJ5YW2uEZ5A2D2JXI/Gsnxb4QtfDHxN0PSfD11d2EWpxxxSOkxLqJJDG+D15Hb1o5QsfQcV5azzPDDcwySx/fRHBK/UDpUhmiEoiMiCQ9F3DJ/CvA/Enhuy8A/FDwmdAaeBLmWMSK0hbdmQK3J7EHBFWPiDpz6t8c9K09LqW1+020cTyxNhwh8wMAfUrkfjRyhY9xhvLa5d0guYZXjOHVHDFfrjpU1fP/j/wzbfC3VdE17wzNcQFpGSSJ5Cwbbg4z6MMgivf0YOisOjDPNJoTQteHfGn/ke/Cv4f+jRXuNeHfGn/AJHvwr+H/o0U4bhHc9ru1neynW1kWO4MbCJ2GQr44JHcZxXnfwr8d6n4ouNW03XfKXUbNgyqibPlyVYEezAf99V6VXhuqj/hBfj7a34/d2GsEbz0H7z5Wz9HAb8aFroCOm+JPjjWNE8QaLoHh4w/b745fzY9+AzBU+nO78q9CmvbWxjjF7ewRMQBuldU3H15NeO+EV/4TT43av4gb57LS8pAeoyMxp+YDt9apa/Y+Bm8Z6pLrGo6t4k1GVyBZ2MLEwEE/KGBAOOAADxg55p26Dt0PdfOSS3M0Lq6FSVZTkH8a89+E3jPVvF+mancaxJCWtpVVDHGEABUk5rl/gbcTR3fifTR9oS0iKvHBP8AeiOXByOzYAB+lUvhJ/yTfxp/1wf/ANEtRa1wse8RyxyrujdXXOMqciiOaKbPlyI+Ou1gcV5j8BP+RBuf+wjJ/wCgR1yHws1aXQ/BfjjU4ADNbojx5GRuw+0/TOKXKKx7xJfWkM6wS3UCTN92NpAGP0HWrFfLuiQeFdV8PXl54ibWrnXrpnZbmKJ3VD/CcjhueufpxXr3wa1fU9S8Gvb6r55mspzDG8wO5o9oK5J64yR9AKHGwNWPRKKKKkQUUUUAFFFFABRRRQAUUUUAec/G/wD5JxN/18xfzNbnw1/5JxoX/XqP5msP43/8k4m/6+Yv5mtz4a/8k40L/r1H8zVfZH0OqoooqRBRRRQBg6r+58VaDcDrILi1P0ZBJ/7RrarG1/jWPDJ/6iTj/wAlLj/Ctmuyh8BMgrD8Z/8AIi+If+wZc/8Aopq3Kw/Gf/Ii+If+wZc/+imrZ7CW53tFFFcp0hRRRQAUUUUAFFFFABRRRQAUUUUAFcx8Rv8Akm3iT/sHTf8AoJrp65j4jf8AJNvEn/YOm/8AQTQBp2X/AB42/wD1yX+VT1BZf8eNv/1yX+VT15p0hXFeG/8AkZvGH/YTT/0nirta4rw3/wAjN4w/7Caf+k8VaUt2Y1vhOlooorY5gooooAKKKKACiiigArH1m7uWubbSrCTyrm6DO84AJghXG5gDxuJZVGfUnnaQdisSH5vGuoZ52ada7fbMlxn/ANBH5VpSipSSYixp+jafpmTbWyiVv9ZO/wA8sh9Wc/Mx+pq/RRXcQFFFFMAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACkZQylWAIPUGlooA5/UtMTR45dY0eEQTQAyz28I2pdIOWUqON+Put1zgZxkV0MUqTwpLGwaN1DKw7g8g01lDKVYZBGCKzPCTFvBmhMxyTp9uSf+2a1yYiKVmi0bFFFFc4wooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKAOd8ff8AJP8AxB/14S/+gmursv8Ajxt/+uS/yrlPH3/JP/EH/XhL/wCgmursv+PG3/65L/Ks6uyOih1J6KKKwNzm/Bf/ACMHjL/sKp/6TQ12Ncd4L/5GDxl/2FU/9Joa7GvRh8KOd7hRRRVCCiiigAooooAKKKKACiiigAooooA4nwv/AMjP4z/7Cif+k0NdTXLeF/8AkZ/Gf/YUT/0mhrqa4Kvxs3jsFcl8Rv8AkWbf/sJ2X/pRHXW1yXxG/wCRZt/+wnZf+lEdKHxIJfCzdoooroOIKKKKACiiigDF1ie4uL230azmaB50aW4nT70UKkA7T2ZiQAewDHqBV2xsLTTbYW9nAkMQOcKOp7knqSe5PJqjDz411LPbTrTHtmS4z/IVr120YpRuTIKKKK2JCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACsvV9Le5C3tiVi1S3BNvL0DdzG/qjdCO3UcgVqUUmk1ZjIdNv49T0y2volZUnjD7W6qT1U+4PB9xVqsXwt/yB5R2F/egewF1KBW1XnNWdiwooopAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFc9o/wDyU/XP+wZaf+hzV0Nc9o//ACU/XP8AsGWn/oc1TP4WaUviOxooormOs88+N3/JK9S/66wf+jFoo+N3/JK9S/66wf8AoxaK7MP8JjU3NPwZ/wAiL4e/7Blt/wCilrcrD8Gf8iL4e/7Blt/6KWtyvRWxxvcKxvCX/IFn/wCwnqH/AKVzVs1jeEv+QLP/ANhPUP8A0rmrDEfChxN2iiiuQoKKKKAPDvEv/Jx+j/8AbH/0E17jXh3iX/k4/R/+2P8A6Ca9xqpdBvoeaeO/BOs+IPH3h7WLBITaWLRGYvJtI2y7jgd+K7PxXp1xq/hPVtOtApuLm1kijDHA3EYGTWxRSuK55PB4A12P4K3PhZo4P7TknDqvmjZjzVb730BrYm8I6q/wYHhcJF/aX2ZYtvmfLuEgb730r0CijmY7nn1j4Q1W3+C7+F5Ei/tM28se0SfLlpGYfN9CKzrX4Z3d98IYPC+ovFb6jBM88Lq29FfexGSOxViD6Z9q9Soo5mFzx2LSvi5NpdnoHmWNjb2xVP7Sim/eMi8AHBJPH+yCcDPeup8XeG/FdzpGmHw74hnS/sVAmEr7FuyMfM2BjOR0PBBxXc0UcwXPFdY8E+NvH2oWSeINM0fSYIHzLc22GlkB69GYn2BwK3fij4J1vXbrQ9S8PpFLcaax/dSuFJ5UqQTweV5ya9Nop8zC55B4t8F+MvEll4e1xksjr+nsxntNw8s4k3IVycHgDIJ59aq+JvB3xC8XXWl6rqMOlxS2Uv7uxhlICDIJYsSQSSoGAT0Fe00UczDmPOPEPg3WNT+LeieJLeOL+z7SKNZS0gDAhnJwO/3hR4x8GavrXxK8Oa5ZpCbKwMRnLSYYbZSxwO/Br0eilzMLnnHj3wZq/iHxr4b1SwSFrbT5UacvJtIAkDHA78CuO+IlpfX3xx0m30y6W1vjbxNBMwyFdd7DI9MjB+veveK5y98E6Vf+MLTxRM1z/aFqoWMK4CYGeox/tHvTUgTPPrrwV448c+IdPfxiLG10uxbOy2YHzeRnAyTlsAZOMDoK9koopN3E3cK8O+NP/I9+Ffw/9Givca8O+NP/ACPfhX8P/Ropw3HHc9xrz/4reCLvxlo1n/Zgj/tC0mLIZG2gow+YZ+oU/hXoFFJOwk7HE/C/wdP4N8LtbXwj+33EzSz7G3ADooz34Gf+BGuO0Lwd4+8Ea7qqaDb6XeWl+4xd3T/cAJKkgENn5jkYIr2eijmHc8w+HPgjX/Cuv69NqjQTxXy5W5jfl3DE5K9s7ifwpfhp4C1Tw94c1zS9aWKP+0PkBikD/KUKn+denUUczC54v4W8JfErwpFeaDpzaWmnXEpf7fI24xkgKWRQQc4A4IIyK1vh58OL7RNF8SaTrixm21ICJGikDEphwT7H5ga9Sop8zDmZ4vpnhv4n+ENNuvD+ijTrrT5WYw3jSBXh3dSoLAg98YbB6V6X4R0vVtI8PxW2t6o+pagSXklY5C5/hU4yQPU8nJrdopN3Bu4UUUUhBRRRQAUUUUAFFFFABRRRQB5z8b/+ScTf9fMX8zW58Nf+ScaF/wBeo/maw/jf/wAk4m/6+Yv5mtz4a/8AJONC/wCvUfzNV9kfQ6qiiipEFFFFAGFr/wDyF/DH/YTf/wBJLmtmsbX/APkL+GP+wm//AKSXNbNdlD4SZBWH4z/5EXxD/wBgy5/9FNW5WH4z/wCRF8Q/9gy5/wDRTVs9hLc72iiiuU6QooooAKKKKACiiigAooooAKKwvFviux8GaJ/a2pRzvaiVIm8lQzDdxnBI4pvh3xr4c8VRhtG1a3uXxkw7tsq/VDhvxxigDfrmPiN/yTbxJ/2Dpv8A0E109cx8Rv8Akm3iT/sHTf8AoJoA07L/AI8bf/rkv8qnqCy/48bf/rkv8qnrzTpCuK8N/wDIzeMP+wmn/pPFXa1xXhv/AJGbxh/2E0/9J4q0pbsxrfCdLRRRWxzBRRRQAUUUUAFFFFABWJB/yOup/wDYOs//AEZc1t1iQf8AI66n/wBg6z/9GXNa0fjQnsa9FVNUmubfSbyeyRHuo4HeFHBKs4UkA45xnFcn8M/G9x420S7ur6G3gubafYywAhdhUEHknvu/Ku2+tibaXO3orxzw18ZL3W/HkGkTWlkmm3Fw8UUqBvMxz5eSWxknaDx3rovGnjvUtC8ZaJ4e0q2tZpb8p5hnViUDPtBGCPRjzS5la4+V3seg0Vyml6x4nuPG+oaffaPHDocSMbe8CkNIQVwCd2OQW7dq6umncTQUUUUxBRXAjxzqJ+L58IfZ7X7CI9/m4bzM+Tv65x19uld9STuNqwUUUUxBRRRQAUVHcXENpbvPczRwwxjLySMFVR6knpXmEHxaOr/Eqx0DRo7abSZW8uS5dG3uwDElOQAvAHIPek2kNJs9TooopiCiuR8eeJdT8OwaQmkw2clzqF+lmPtYbYu4HB+Ug9cetXb7Xbrw34Ol1bxCtqbyBCXjsi3lyOWwipu55yo5757UrjsdDRWXpt3dW3h6C78QTW0FyIvMumHyRxE84yT2zjOecVy3h/4jR+J/H0+jabEjaXFZtMLllYNK4dVyvP3OSOmSRRdBY72iiimIKKyPEPibSvC+nteapdLEuD5cY5klP91V6k/5NZfw/wDFV14w0CfU7q2jt2W7khSNAflQYxuyeW55PH0FK6vYdtLnV0Vzlvr1zB42uPD+oiMJPCLnTpVGN6Dh0b/aU88dQa6Oi4BRRXOajr1yfF+neHtMEZlKG7vpHGRFAOAB/tM2APQc4ouB0dFcRqXivXdQ8T3mgeE7GxllsEU3l5fuwhjZhkIAnJOP5H0qbw74s1ObxJP4Z8SWFva6qkH2iGS1ctDcR5wSueRg+voemKXMgszsaKKiubmKztJrqdwkMKNJIx/hUDJP5CqES0V53YeJ/HXiSxOsaFo+kQ6W5Y28V/LJ586gkZG35Vzjv+o5rY0fx5Y6h4LuvEV1DJaCxLpeW5OWjkTGUHTOcjH1qeZDszrKK83bxd45i0QeJpdB0w6Ns882izP9rWHruJ+793nGM+1dFrXjWw0rwdF4ihR7qO6VPskK8NM7/dX2PXPpg0cyCzOmorzy68W+MPDUdvqXijStLGkSyKkzWEjmW03HAL7uG54+Wug8V69c+Hbay1ZFjk0pJ1S/+XLJG/CyKR/dYjIwcg9qLoLHR0UisrqGUhlIyCDkEUtUIKKxfFXiBPDWgTX5j86ckRW0A6zTMcIv5/oDWX4g8UX/AIZ0XSoJbaPUPEOoOsEUEXyRtKQNx56KM/y6dQm7DsddRXBHxX4n8ParpsHiyw0v7FqM4torrTZHxDI33VcP1z6j0JrvaE7g1YKKK4rVfFWt3nim48OeFbKymubONZLy6v3YQxFhlVwvJJH+eKG7Alc7WiuS8MeKr+/1u/8AD2u2UNprFkiyk27lopozj51zyOSOD61a0vXrk+LdT8PamI1njUXVlIgwJrdjjkf3lbIPr1xRcLHR0Vwtp4l8TW3xCtPDusQaR9nuoJLiOSz8wsFXOAdxxnjniu6oTuDVgrK8If8AIlaD/wBg63/9FrWrWV4Q/wCRK0H/ALB1v/6LWufEbIcTZooorlKCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooA53x9/yT/xB/wBeEv8A6Ca6uy/48bf/AK5L/KuU8ff8k/8AEH/XhL/6Ca6uy/48bf8A65L/ACrOrsjoodSeiiisDc5vwX/yMHjL/sKp/wCk0NdjXHeC/wDkYPGX/YVT/wBJoa7GvRh8KOd7hRRRVCCiiigAooooAKKKKACiiuE0j4v+DtUv5bCXUf7Pu4pWiMd6PLBIOOH+727kH2oA7uimxyJLGskbq6MMqynII9jTqAOJ8L/8jP4z/wCwon/pNDXU1y3hf/kZ/Gf/AGFE/wDSaGuprgq/GzeOwVyXxG/5Fm3/AOwnZf8ApRHXW1yXxG/5Fm3/AOwnZf8ApRHSh8SCXws3aKKK6DiCiiigAooooAxIP+R11P8A7B1n/wCjLmtesiD/AJHXU/8AsHWf/oy5rWcFkYBipIwGHb3rupfAiJbi0V5h8KvF2ua1q2u6R4hvBcXdi42HykTGGZXGFA7hfzrj7j4s6+nxFeNL/Hh9NSEBj8iPHlBtpO7bu5ALdarnVrj5Hex7/RXmXxX8Xa3oN3oemeH7sW97fSMGPlI+RlVUYYHqSfyrWE/iVPiZDanXLBtEEXz2ReIXDN5R527d33vm4PT2p82tg5dLnb0VTOraas80B1C1E0Cl5ozMu6NRjJYZ4HI5PrTdO1rStX3/ANm6nZ3nl/f+zzrJt+uCcUxF6iqWo6xpmkRq+pahaWaOcKbiZYwx9snmvPPEHjPVIPix4d0nTNTjbR76GJ5EjSN1k3O4JD4J6KOh7UN2BK56hRVa+1Cy0y3Nxf3lvawg48yeQIufqaZp+r6bq8TSabqFreIvDNbzLIB9cHigRcoqtb6jY3k0sNteW88sJxIkUqsyHpyAeKSDUrC5upbW3vbaW4iz5kUcqs6YODkA5HPFAFqiq0eo2M15JZxXtu91GMvAsql1HuucjqPzp15eW+n2U15dyrFbwIZJJG6Ko5JoAnorw/VPi5fax470ay8PTSW+kNeRwSO8S5utzqG+8CVGDgdDzz7e13NzBZ27T3U8cEKY3SSuFUZOOSeOtJST2KcWiWio7e4gu4Entpo5oX5WSNgyt9CK5H4na3qXh/wa99pVz9muvtEaCTy1fAJweGBFNuyuJK7sdlRXNy3t14P8I3uoa/qx1R7YNJ5xgSAtnAVAq8cnjPvUmgXF5p/hFdS8R3f79o2vLpmGFgUjdsAHZV479DRcLHQUV5fYfEPUNf8AiTotlYW89roFwk5R5ogpvNsbHcMjIUEDGOvOfQeoUJ3BqwUUVieJvFOn+FbBLm+815Jm8u3t4ULyTP8A3VA7/WgDborhPhv4l1nxJNr76wnkSW94I47XaB9nGPuZxkn1J71f1XV7vQPG2mi5nZ9H1cfZVVsYt7kcrg+jjjHqM0r6XC2tjrKKKKoQUVyfiPV7ubxLpPhjS52huLj/AEu8mTG6K2Q8gehdvlz25qprup+JNU8Xjw3oE66ZDDbi4utSlt/NPJ4RA3yk8jP9Mcq47Hb0V5/Y6v4i8N+NdO8Pa9qMWrWmqRyG1vBAsMkboMlWVeCOnPuPevQKE7g1YKKCQASegrzjSpvGvjSO41W11oeHrDzmjtLY2CyyOqnG59/TPoP/ANY2CR6PRXDeF/Gd2+meII/EQi+2+H3YXUsAwsyAEhgOxO08fTp0rN05viD4i0IeJLTXLWxM6ma00n7Gro0f8IaQ/Nlh3Hr27LmHynpdFcMvxFg/4VefFrW4EypsNtnjz92zb64zz64rNvj8QtC0FvEtxrdreNDGLi60k2aoix9WVZB82VGfy79zmDlPS6K5TXdVvb3wRD4i8PzMJIo0v1hOMTxAbnjb6rnpzkCt/SdTt9a0i01K0bdBcxLKmeoBHQ+46U7isXKKKxvFfiCLwx4bu9UdPMeNdsMXeSRjhV/EkfhmgRs0VxupX3iTw74FtTFBLq3iCdkjZhGWSORySWYKOEXp27ZrE1y+8Y+BNPg13UvEEGs2YmRLy0NmkO0McZjZeTj3+tLmKsem0UA5AI70VRIUVx3iK88UX/ia30HQS2m23kGe51aS28xRzgRpu+Ut0z9e2Kq6NrOvaP45Twrrt/FqiXVqbm1vVgETjBOVdV47Hn/Hibjsd3RXJz6vd6L8QINPvJ2k0zWYj9kLY/cToPmTPowwRnvwKyNavvEmifELQLY+IDcaZq11KDZmziXykUAhd+Cx+914PFFwseh0UUVQjJ8Lf8gib/sI33/pVLW1WL4W/wCQRN/2Eb7/ANKpa2q86XxM0CiiipAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAK57R/+Sn65/2DLT/0OauhrntH/wCSn65/2DLT/wBDmqZ/CzSl8R2NFFFcx1nnnxu/5JXqX/XWD/0YtFHxu/5JXqX/AF1g/wDRi0V2Yf4TGpuafgz/AJEXw9/2DLb/ANFLW5WH4M/5EXw9/wBgy2/9FLW5XorY43uFY3hL/kCz/wDYT1D/ANK5q2axvCX/ACBZ/wDsJ6h/6VzVhiPhQ4m7RRRXIUFFFFAHzz8Qtdi0L45Q6pLC8qWKwsyKQC3yZ4/Ouj/4aD0r/oB3v/f1a6PWrK0uPjHoizWsMiyadOzh4wQxBGCc9SO1dl/Yulf9Ayy/78L/AIVba0uVdHlX/DQelf8AQDvf+/q0f8NB6V/0A73/AL+rXqv9i6V/0DLL/vwv+FH9i6V/0DLL/vwv+FK8ewro8q/4aD0r/oB3v/f1aP8AhoPSv+gHe/8Af1a9V/sXSv8AoGWX/fhf8KP7F0r/AKBll/34X/Ci8ewXR5V/w0HpX/QDvf8Av6tH/DQelf8AQDvf+/q16r/Yulf9Ayy/78L/AIUf2LpX/QMsv+/C/wCFF49gujyr/hoPSv8AoB3v/f1aP+Gg9K/6Ad7/AN/Vr1X+xdK/6Bll/wB+F/wo/sXSv+gZZf8Afhf8KLx7BdHlX/DQelf9AO9/7+rR/wANB6V/0A73/v6teq/2LpX/AEDLL/vwv+FH9i6V/wBAyy/78L/hRePYLo8q/wCGg9K/6Ad7/wB/Vo/4aD0r/oB3v/f1a9V/sXSv+gZZf9+F/wAKP7F0r/oGWX/fhf8ACi8ewXR5V/w0HpX/AEA73/v6tH/DQelf9AO9/wC/q16r/Yulf9Ayy/78L/hR/Yulf9Ayy/78L/hRePYLo8q/4aD0r/oB3v8A39Wj/hoPSv8AoB3v/f1a9V/sXSv+gZZf9+F/wo/sXSv+gZZf9+F/wovHsF0eVf8ADQelf9AO9/7+rR/w0HpX/QDvf+/q16r/AGLpX/QMsv8Avwv+FH9i6V/0DLL/AL8L/hRePYLo8q/4aD0r/oB3v/f1a4Dx38QrTxd4h0jUoLGa3SwxuSRgS3zhuMfSvpT+xdK/6Bll/wB+F/wrifG2ladH4n8GolhaqsuoskgWFQHXYTg8cimmr7DTRgf8NB6V/wBAO9/7+rR/w0HpX/QDvf8Av6teq/2LpX/QMsv+/C/4Uf2LpX/QMsv+/C/4Urx7Cujyr/hoPSv+gHe/9/Vo/wCGg9K/6Ad7/wB/Vr1X+xdK/wCgZZf9+F/wo/sXSv8AoGWX/fhf8KLx7BdHlX/DQelf9AO9/wC/q0f8NB6V/wBAO9/7+rXqv9i6V/0DLL/vwv8AhR/Yulf9Ayy/78L/AIUXj2C6PKv+Gg9K/wCgHe/9/Vo/4aD0r/oB3v8A39WvVf7F0r/oGWX/AH4X/Cj+xdK/6Bll/wB+F/wovHsF0eVf8NB6V/0A73/v6tH/AA0HpX/QDvf+/q16r/Yulf8AQMsv+/C/4Uf2LpX/AEDLL/vwv+FF49gujyr/AIaD0r/oB3v/AH9Wj/hoPSv+gHe/9/Vr1X+xdK/6Bll/34X/AAo/sXSv+gZZf9+F/wAKLx7BdHlX/DQelf8AQDvf+/q0f8NB6V/0A73/AL+rXqv9i6V/0DLL/vwv+FH9i6V/0DLL/vwv+FF49gujyr/hoPSv+gHe/wDf1aP+Gg9K/wCgHe/9/Vr1X+xdK/6Bll/34X/Cj+xdK/6Bll/34X/Ci8ewXR5V/wANB6V/0A73/v6tH/DQelf9AO9/7+rXqv8AYulf9Ayy/wC/C/4Uf2LpX/QMsv8Avwv+FF49gujyr/hoPSv+gHe/9/Vo/wCGg9K/6Ad7/wB/Vr1X+xdK/wCgZZf9+F/wo/sXSv8AoGWX/fhf8KLx7BdHgXj/AOLVh4w8LvpNvpdzbyNKknmSOpHyn2r174YSCX4a6GwGMQFfyYj+lVviNpOnQfD3WpotPtY5UtyyOkKgqcjkHHFdLoEUcPh7To4o1RBbR4VRgD5QelDatoNvQ0aKKKkkKKKKAMLX/wDkL+GP+wm//pJc1s1ja/8A8hfwx/2E3/8ASS5rZrsofCTIKw/Gf/Ii+If+wZc/+imrcrD8Z/8AIi+If+wZc/8Aopq2ewlucb/w0po//Qv33/f5KP8AhpTR/wDoX77/AL/JXr/9haP/ANAqx/8AAdP8KP7C0f8A6BVj/wCA6f4VynSeQf8ADSmj/wDQv33/AH+Sj/hpTR/+hfvv+/yV6/8A2Fo//QKsf/AdP8KP7C0f/oFWP/gOn+FAHkH/AA0po/8A0L99/wB/ko/4aU0f/oX77/v8lev/ANhaP/0CrH/wHT/Cj+wtH/6BVj/4Dp/hQB5B/wANKaP/ANC/ff8Af5KP+GlNH/6F++/7/JXr/wDYWj/9Aqx/8B0/wo/sLR/+gVY/+A6f4UAeQf8ADSmj/wDQv33/AH+Sj/hpTR/+hfvv+/yV6/8A2Fo//QKsf/AdP8KP7C0f/oFWP/gOn+FAHzn8R/jNp/jfwlJo1tpN1bSNMknmSSKRhT04rx6OR4ZFkidkdTlWU4IPqDX1d8Y/CwvvAL2+iaIkt891FtW0thvxk5+6OnrXlfh/9nvxRqeyTVp7bSYT1Vj50v8A3ypx+bUAZXhn42+MPD2yKe7XVbVePKvcswHtJ978yR7V6XefGHSfG3w/8Q2K6df2l+dOmygiM0X3D/Go4HuwUVv+H/gR4N0bZJdwTarcDnddv8mfZFwMexzXQeOrK00/4W+IbaytYbaBdOm2xQxhFHyHoBxQBtWX/Hjb/wDXJf5VPUFl/wAeNv8A9cl/lU9eadIVxXhv/kZvGH/YTT/0nirta4rw3/yM3jD/ALCaf+k8VaUt2Y1vhOlooorY5gooooAKKKKACiiigArEg/5HXU/+wdZ/+jLmtusSD/kddT/7B1n/AOjLmtaPxoT2NevnLTdV/wCED1X4g6SG8sNA4tv97ftjx77Zc/hX0bXhvxS+G+va743/ALR0aw8+C5hj86TzEUK4+U5BIP3Qp4rrnfdBC2zOYvvD8nh34c+EvFMEe26W+aeQjqdxDR/hiIf99V12jzR+L/2gJdRhPmWen2++M9sBAo/8fcn8K77xh4V/tT4cXOgWMW+SG2RbVcgZaPG0ZPAztx+Ncz8HPBWq+F49VutZszbXNwyRxqXVjsXJJypPUkflU8rTSHzXVyj4RuZ3+PviOF5pGiWGXahclR80fQVg+F9Gu/F3xA8V6Tc6xf22mLcyvPHby4aTErBVyc4Xkkj2H4dj4a8La1Y/GTXNcubEx6bcxSLDP5ikMSyEcA57HqO1L8PvC2taL488UajqFkYLS9ldreQyK28GUsOASRwe9FguYHw7jufC/wAW9W8KQ3s8+mrExVJWzggKytjpnBwSMZqjomlzfFrxxrlxrGpXcWn6e+2CCBwpUFmCAZBA4UknGSa63S/C2tW/xv1LX5bErpc0RWO48xcE7EHTOeoPasZfDnjHwB401K/8NaXHqumakxYxlwuzJJAPIIKkkA8gg+vQt9w7mX4b0u50b9oJNOub2a9MEbKk8xy7R+RlQx7kAgfhU0VtefFr4g6xZ32p3VtommsyJbwNjcAxUcHIycEkkH0rT8OeE/GCfFuHxLrtnFsmjd5pYZFKRExFVQDOTj5Rn9T1ovfC3i3wR42v9f8ACllFqVjqBZprVmAK7juIxkHg5wRng4IpW0C5U0ltQ+GvxTsvDC6jPe6LqSqY452yU3llUjsCGXBxgEdumDwzrUvgbxV430i7ndoreB7y0ErE8LyijPcq6/lWp4c8JeJfEPjyPxh4ut4rL7KoFrZowJ4zt6E4AJJ5OSe2Kq/FzwDreva9aapoFm07SW5gugsqpjaeCdxGcg4/4DTs0roLq9mc58KvEt3oGqak2sTTNBdaY99H5rk7/LJIxn1G/wDKu0+CdreT6HqOvX80ks2oXJCF2JG1c5Iz0yzMP+A1k/Ej4b6td2nh8eHbQ3EtlZfYZtkiodigbT8xGc5fP1r1Lwzo66B4Z07SlAzbQKjkd3xlj+LEn8acU76ik1bQd4g0Kz8S6JPpN+ZBazlC/lNtY7XDAZx6qK8gvdIsND+PXh3T9MtY7a1jgTaif7snJPUn3Ne5V5lrfhbWrv416Tr0FiX0yCJVkn8xQFID9s57jtTkhRZ1fjTwxL4t0IabDqT6c4mWXzkQueAeMBl9fXtWtpNi2maNY2DzGdrW3jhMpGDIVUDdjJxnGeprJ8aT+JLfQg/hW3SfUfOUFH242YOT8xA9K1tJa9fRrF9RQJfNbxm5UYwsm0bhxx1z0qupPQ4r4of8fPg7/sP2/wDOj4hubvxR4K0ZuYLnUTcSL2bygpAPt8xo+KH/AB8+Dv8AsP2/86X4mRmxuvDPiQj9zpWoqLg/3IpCFZv0A/GpfUpdDtdS0yz1iwksdQt1uLWQqXibOGwQwzj3ArhbaCG2+OvkW8SRQx+HwqRxqFVR5o4AHSvRQQRkciuNXRdRHxffW/sx/s46P9m8/cv+s8wHbjOentimxI7Kiue8T67qGiXGjC0sY7i3vL6O1uJHfBiDsFBA7nk/lXQ0xFG60XTb3UrbULqyhmu7UEQSSLuMecEkA8A8DnrXGfB//kVdQ/7Ctx/7LXoNcb8NtF1HQvD95balbGCaTUJplUsrZRsYPBPpStqPoUPie50+48Ka1HxNa6xHFkdTHIDvH4hcV6DXn/xBjOs+JPCXh6L5me/+3zAfwxRDkn65IHvW/ca7qFv47stENjH/AGfdWjzLdb/m3ofmXb6DK8+9C3YdDoa8++H7nUPF3jbVZfmkOo/Y1J7JECoA/SvQa8/8GxnRviH4u0aX5TdSrqdv/to+d5H0YgUPdAtmdXquoaP4ZsbrVr5oLSM4MsoUBpW7Djlj1wK5PwhZaj4i8Vz+OdUtns4Xt/sumWr/AHxDnO9vQnnH1PbBODr1h4v1Dx/Lql34RbV9MsmKadbPfRRxDB/1pUk5JxnkenpXaaDr3ivUNUW31bwj/ZloVJNx9vSXBHQbQM80r3Y7WR1dNkjSWNo5EV0YFWVhkEHqCKwND13UNQ8Sa9pd7Yx28envF5Dq+4yo+/BPpwo4961dWmvLfSLubTrb7TepExghLAB3x8oJJHGaq5NjnvFPiy08KWsOmaZarc6xcDZY6dbr37MQPuqP1x9SOH8UeHLrwr8D9Rtrqbzb+8ukub516eY8i5A9htUfWjwtb+NfD1xdahc+Bm1LWbty09/NqUQYjsqjnavTgfyAA7Waz1Xxx4K1TTNe0gaNPPmOJDcLP0wyvlf9odPao3K2OgvYov8AhHbiHA8n7Iy47bdmP5V4tpzPN4G+GMc2TE2t8574nYL/AFFdZLf+PJ/DJ8NHwzs1B4fsj6kblPI2Y2mT1zjt69u1aWs+ApG+H+maNpMyLqGkPHc2kr8B5kyTn03Et+OKHqC0L3xOVH+G2uB8Y8gHn1DAj9cU9LX+1fhVHbXA3G40ZVbP94wjn8+a5zWm8X+OdMj8Oz+HH0aCd0/tC8luEdQikEiMDk5I/p711ni69g8PeA9TnXCRwWbRQr/tFdiD8yBT8xeRT+GWoSan8ONFuJWLOITCSepEbMg/RRXW155by6n8P/hBp8lrYJc3VpCslxDI+0IHJd/xBbGK9CBJAJ4NNbWE9zz/AMXOb/4oeC9Jf/URtNesvYuiEofwKn869AZVbG5QcHIyOhrgPHUZ0vxn4Q8SHi3gunsrhj0USrtUn2GW/StXxpp3iCf+zdT8O3Dm60+YySWJmMcd2hxlW5xnjjPqaO4+xx3jJtestc0nV/F8drN4bs70Okems2YpM/u3lDjLY9jj88H1sEMAQQQeQRXmevReKPiFbQaHL4fl0TTWlR725uZ1ZiqnO1FHXnv7dq6rXNavtF1nQLK1sI5bG+nNtLMz4MR25UAdTkBvypIGdHVLVF1AafO2jrZ/2iQBGbvd5fX+Lbz0zj3q7XCX48VeGvFt9qdlaXOu6NfopNqtxiS0dR/ArHG088D19uabEjK8HyXmnfEvUrbxUgbxFf24a3uYWzbvAv8AAgwCp+XPPXb+d/x450/xv4I1SL5ZGvmsnI/iSXC4P05p+i6Preu+OU8Wa7YDTIbS3a3sbIyB5PmzudyOBwTx9PTlvi2M618SvCWkRfN9hZ9TuSP4FXAQ/iwx+NR0K6iat/yXLQP+wXN/Nq9Brz7Vv+S5aB/2C5v5tXoNUuon0Csrwh/yJWg/9g63/wDRa1q1leEP+RK0H/sHW/8A6LWsMRsgibNFFFcpQUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFAHO+Pv+Sf+IP8Arwl/9BNdXZf8eNv/ANcl/lXKePv+Sf8AiD/rwl/9BNdXZf8AHjb/APXJf5VnV2R0UOpPRRRWBueMXfxYsfAHjfxTp91plxdvPfJOHidVAHkRrjn6VL/w0po//Qv33/f5K7Xwpptje+I/GL3VlbzuNUQBpYlYgfZ4uORXVf2Fo/8A0CrH/wAB0/wr0YfCjne55B/w0po//Qv33/f5KP8AhpTR/wDoX77/AL/JXr/9haP/ANAqx/8AAdP8KP7C0f8A6BVj/wCA6f4VQjyD/hpTR/8AoX77/v8AJR/w0po//Qv33/f5K9f/ALC0f/oFWP8A4Dp/hR/YWj/9Aqx/8B0/woA8g/4aU0f/AKF++/7/ACUf8NKaP/0L99/3+SvX/wCwtH/6BVj/AOA6f4Uf2Fo//QKsf/AdP8KAPIP+GlNH/wChfvv+/wAlH/DSmj/9C/ff9/kr1/8AsLR/+gVY/wDgOn+FH9haP/0CrH/wHT/CgDyD/hpTR/8AoX77/v8AJXzxqNyt7qd3dKpVZ5nkCnqAzE4/Wvub+wtH/wCgVY/+A6f4V8tRfB3xl4g12+kt9J+xWjXMhSa8byl27jghfvEY9FoA5Pw/408R+FpA2javc2yZyYg26M/VDlT+VeueG/2j502Q+JNIWUdDcWJ2t+KMcE/Rh9K0tA/Zv06DbLr+sz3TdTDaKI0+hY5JH4CvUdB8B+FvDO06TolpBKvSZk3y/wDfbZb9aAOd8Aarb63qXirUbRZlguNRR0E0RjYD7PEOQenSu3rlvC//ACM/jP8A7Cif+k0NdTXBV+Nm8dgrkviN/wAizb/9hOy/9KI662uS+I3/ACLNv/2E7L/0ojpQ+JBL4WbtFFFdBxBRRRQAUUUUAYkH/I66n/2DrP8A9GXNa9ZEH/I66n/2DrP/ANGXNa9d1L4ERLc8B1zUz4A+Meu3q/JHe2MskZ9XePcPzlTFZq+Ey3wFfWCmbk3/ANs3fxeWD5WPpnLV6F8UfhrqPjXUrC+0uezhlhhaGb7S7LkZyuNqn1auwj8Mwr4EHhkldn2D7IW7Z2bS3580cruy+bRHj2jakfH3xa8NTsd6WFhDJMewkRN5P/fxgK2Zf+TnIP8Arif/AEmatf4X/DTUfBeqX9/qlxZTSTQiGH7M7NgZy2dyj0X9avv4G1NvjFH4vE9p/Z6x7THvbzc+SU6bcdT69KEnb5g2rnCxaDaeJPj5rVhqG97PDSSxK5UShVTCtjnG7B/AVNbaTaeFP2g7Cx0dGt7SeIloQ5Iw0TZHPOMqD9a7XSPA2p2HxX1PxTLPaGxuo2RI1dvNBIXqNuP4T3o1HwNqd38W7DxXHPaCwt4wrxs7eaTsZeBtx1Yd6OX8w5vyOKsdLtvHvxv1yDXd89rp6yCK3LkAhHVAOOccluOpqrqfhyw8MfHXQLPTVaO0kaKZIS5YRZZgVGecZBP411/iT4e69D40bxV4O1G3tbucYuIbjhSSACRwQQcAkHvyD6VLf4aeKJPHWleJtU1iyvJonWS7+8m3BOFjAXBAGOuOc/Uln2HfzMi5sk+IPxwvtL1mSRtN02NvLtg5UMF2gjjplmySOcDFQ+K9Itvhr8RfD994cMlvDetsmtfMLAqGUMOSTghu+cEZFdb4t+HWqz+K18V+E9SisdVIHmpMPkc4xnOD1HBBGD1qDRvh3r+p+K7bxF421O3u5rTBt7a3Hygg5XPAAAPOADk9TRZiujJuLqLwF8cL+6m+TT9VspLhvTO0ufx3xt/31XI+ANSudA8a6b4j1R9lnra3W+T1wSW/8fC16p8Ufh/d+N7fT306a1hvLV2Ba4ZlVo2AyMqpOQQO3c1Q8U/C251fwFoGiafNaR32lgBpJmYI2V/eYIUnlsHpQ4u+g01YqfBa0l1K48QeLbpf32oXTRoT2Gd7Y9sso/4DXqd9ZW+pWM9ldxCW3nQpJGSQGU9RxWX4P0AeGPCen6QWRpII/wB6ydGkJLMRnnGSce2K3KuKsiJO7PEviPZWunfEbwPaWVvFb28c0QSKJQqr+/XsK9a8Q6FaeJdCudIvmlW2uNu8xMAw2sGGCQe4HauT8aeBtT8R+M/D2sWc9olvpsiPMszsHYCQMdoCkHgdyK6fxVY6tqPhu8tNDvFs9SkCeTOzlQmHBPIBPKgjp3pJbjb2J9B0W28O6Ja6TZtK1vbKVQykFiCSeSAPX0rj/jN/yT9/+vuH/wBCrq/DNnqen+HbO11m7W71GNSJp1YsHO4kckA9Mdq5T4zf8k/f/r7h/wDQqb+ES+IT4tMZdM0DT2/1N7rNvFN6FeeD+OD+Fd/LFHNE8UqLJG4KsjjIYHsR3rjvilpdxqPgyS5s13XemTpfxKB1Mec/+Olj+FdVpmoQatpdrqFs26C5iWVD7MM/nQt2D2OI18AfGfweAAALW6wB/wBc2r0GuW1Tw3eX3xB0HX4pYBaafDNHKjMd7F1IG0Yx37kUvj3UtW0nw6t3o1xbw3QuY02zLuMoY7diDByxJGPoaNrsN7HUUxoYnkSR40Z487GKglc9cHtT6Kok8++HP/IxeOP+ww/9af8AGNdvw9nulO2a0uYJom7q28DI/BjWr4T8N3mhat4jurqWB49Tv2uYRExJVTnhsgYPPbNZXxNjOsponhSHJl1S9VpQP4YI/mdj+lR9kv7R3UEnnW8UpGC6BsemRUlcvrmo6rZ+MPDlnZXFuLG7eRLi3K5lIVC24ccKMAE56kV1FUSee+F2N58X/GVzJ9+2itreMHshXJx+K5/Gut8QeItN8MaU+o6pOIoV4VRy8jdlUdzXKpGdA+M0kr/La+IbIBG7efCB8v8A3wM/U1S8T+DvGGq+Oo9dsp9DktbRQtjb3zSsIjgZcqFxu3Z5ye3oKm7SKtdlzwxpGq+IfEy+NPEMBtPLjMel6eesEbdXf/aIJ/P2FegVx2kQ/EVdVtzrFz4dbTw374Wol8wjB+7kY64qzZalqz/EfU9MluLeXS47KOaKONfnhckDDnHU4cgZPGKa0E9TqK5Lxb40GjTR6PpNudR8Q3Qxb2achM/xyH+FR19/Yc11M/m/Z5PI2edtPl7yQu7HGcds15RoPgz4i+H7q+vLe78Mz3t7IZJ7q5MzyN7Z2DC+w/wok30BJdS5f+FJ/Dfwn8SNdXH2rV7+J7m+uB0ZzyQPYDP5np0rtfB5Q+CdBKY2/wBnW+Mf9c1qpotj4lvLDUbPxg+kzw3EflRrp+8ZVgwcNuA7EYx71zlp4e8f6JpDeHdKvtJl09QY7a+uC4nhjPYqBgkZ4/zhbD3ODcH/AIVBdSf8u/8Awkpf28vgflmvbfFRVfCGtl8bBYT7s+nltWUngHTl+Hh8IGRjAYsGfb83mbt2/H+9zj04rCuvD3xA1fRl8Oajf6RHp7KIrjUId5nliHbaRgMR1pJNA2mbnw1jJ+GuiJMuQ1tyD3UscfpWX8HJG/4QmW0LEx2d/PBGT/dBDfzY11Go3Nn4Q8HzTooS1020xEp/2Vwq/UnA/GuR0Kw1Pwl8GWlguILTVPJa9eS6HyqzNuIYYPO3C49ae1g3PR689+JLGfXfBWnP/qJ9XWSRezFMYB/76Ndpo01zc6Fp896MXUltG8w24w5UFuO3Oa5P4o2cw0Sw122QvNod9HelV6tGD84/kfoDTlsKO53LOqY3MFycDJxk15d8RdA1qCNvEdxqi6rpunzi5OjTxeXFsB4+ZT8xXP8AFnv9D1fivw9D418O2xs7zyJ43jvbC7UZCuBlT9CD/I9qwdS0Lx54psBo2tXGj2OnOVF3PZF2lnUEHChhhc4/yOCpahHQ7rS7+LVdJs9RhBEV1Ak6BuoDKCM/nVuuR8U3GoeHtM0K30GW3giW8t7MwSDc0kRITagx1A5PTgE111UhFe8WWe0uLe1uhbXTxMI5QocxEjAbaeuDzg8V5jaWt94M+J1g+t3Q1ptcRraDUJFKS25XB2BASoUkjpjr9Qei1/wvrEfiqPxT4YuLVL9oPs91a3m7yrhM8HK8hhgfkPfMOn+F9e1fxTZ+IPFc9kv9nq32KxstxRGbq7M3U9OnoPTmXqNaFf4vMbbQNI1CPiez1aCWNh1z83H+fSl8c/8AJQ/AP/X1cf8AoKU7x9Gde8R+GPDMXzbrv+0LrH8EMXHP+8SQPem+Of8AkofgH/r6uP8A0FKH1Guh6DRRRVkGT4W/5BE3/YRvv/SqWtqsXwt/yCJv+wjff+lUtbVedL4maBRRRUgFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFc9o//JT9c/7Blp/6HNXQ1z2j/wDJT9c/7Blp/wChzVM/hZpS+I7GiiiuY6zzz43f8kr1L/rrB/6MWij43f8AJK9S/wCusH/oxaK7MP8ACY1NzT8Gf8iL4e/7Blt/6KWtysPwZ/yIvh7/ALBlt/6KWtyvRWxxvcKxvCX/ACBZ/wDsJ6h/6VzVs1jeEv8AkCz/APYT1D/0rmrDEfChxN2iiiuQoKKKKAOH1T/ksugf9gy4/mK7iuH1T/ksugf9gy4/mK6/URdnS7sWBUXhhfyC2MCTaduc++KbGzI8ReNNF8L3VlbalO6z3jbYo403HqBk+gyetdBXzR428K6p4f13w9fa7qzajquo3Badh9xAjR4VSf8AePYDpgV9L02rA1YKKKKkQUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFcP45/5GrwT/2E2/8AQDXcVw/jn/kavBP/AGE2/wDQDTW40dxWX4g8Q6d4Y0iTU9Um8q3RgvAyzMTwAO5/oDWpXg3xT8Ka/Lp2r+I9e1fzLe3utmm2Uf3UiaTALcAA7cdMk9zxihK7BK57bpWp22s6Va6lZljb3MYkjLDBwfUVcrmfh3/yTvQf+vNP5V01JiYUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFAHK/Er/knGu/9ex/mK3NF/5AOnf9e0f/AKCKw/iV/wAk413/AK9j/MVuaL/yAdO/69o//QRT6D6F6iiikIKKKKAMLX/+Qv4Y/wCwm/8A6SXNbNY2v/8AIX8Mf9hN/wD0kua2a7KHwkyCsPxn/wAiL4h/7Blz/wCimrcrD8Z/8iL4h/7Blz/6KatnsJbne0UUVynSFFFFABRRRQAUUUUAFFFFABRRRQAVzHxG/wCSbeJP+wdN/wCgmunrmPiN/wAk28Sf9g6b/wBBNAGnZf8AHjb/APXJf5VPUFl/x42//XJf5VPXmnSFcV4b/wCRm8Yf9hNP/SeKu1rivDf/ACM3jD/sJp/6TxVpS3ZjW+E6WiiitjmCiiigAooooAKKKKACsSD/AJHXU/8AsHWf/oy5rbrEh48a6l76daY/CS5/xFa0fjQnsa9FFFdxAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFAHPeLfCcXiu3sY3vrizks7lbmKWADcHAOOv1zVuPQo5fDT6LqlzNqUckbRzTXGN8gYnrjuM8fQVrUUrIdyhomny6Vo1pp812129vGIxO67WZRwuffGBnvjNX6KKBFa50+0vLi2nuIFlktXMkJbnY2MZA6ZwTVmiimAUUUUAY1l4fjtvEmoa7PO1xd3KLDFlcCCFedi/VssT3NaA0+0GpnUfIX7YYhB5x6hM52j0Geffj0FWaKVh3CsbVPD8d/rel6xDO1te2Dkb1XIliYYaNh6HqD2PNbNFAgooopgVrXT7SyluZbeBY5LqTzZnHV2xjJP0AFWaKKACiiigAooooAKxtd8Px6/cacLqdvsdpP9oe2C8Tuo+TcfQHnHfitmikBWvtPtNSgWG9gWeJZFkCP03Kcgkd+exqzRRTAoa3o9pr+jXWl3qFre4Ta2OqnqGHuDgj6VNp1vPaabbW1zdG6mijVHnK7TIQMbiMnk1ZopAFVptPtLi+tr2aBXuLUMIXb/AJZ7hhiPcgYz9fU1ZopgFFFFABWNpXh+PTtY1TVpZ2ub3UJBmRlx5cSjCRqPQevc81s0UgORsPA5tfFsXiG61y+vriJHjjjnC7UVs8DAzgZrrqKKErDbuFZXhD/kStB/7B1v/wCi1rUJCgknAHJrL8Igr4L0IEYI0+3BH/bNa58RshxNmiiiuUoKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigDnfH3/JP/EH/XhL/wCgmursv+PG3/65L/KuU8ff8k/8Qf8AXhL/AOgmursv+PG3/wCuS/yrOrsjoodSeiiisDc5vwX/AMjB4y/7Cqf+k0NdjXHeC/8AkYPGX/YVT/0mhrsa9GHwo53uFFFFUIKKKKACiiigAooooAKKKKACiiigDifC/wDyM/jP/sKJ/wCk0NdTXLeF/wDkZ/Gf/YUT/wBJoa6muCr8bN47BXJfEb/kWbf/ALCdl/6UR11tcl8Rv+RZt/8AsJ2X/pRHSh8SCXws3aKKK6DiCiiigAooooAxIP8AkddT/wCwdZ/+jLmtesiL5fGuoZ48zTrXb77ZJ8/luX86167qPwIiW4UUUVqIKKKKACiiigAooooAKKKKACiiigAooooAKKKKACszXtA0/wAS6W2napE0tszq5VXKnI6cjmtOikBm6LoVjoGnmxsVl8guXIllaQ5PXliT2pdF0W00GwNjYh1tvMeRI2bIj3HJVfRck4HvWjRRYdwprxRybd6K2xty7hnB9R706imIKKKKACs6PRLNNfm1sh3vZIVtwztkRxg52qO2Tya0aKQDTHGZRKUUyKCofHIBxkZ9OB+VOoopgZ2saJZ65bwxXYcNbzpcQyxtteKRTkMp/T6GtGiikAU1Y40d3VFV3ILsBgtgYGfWnUUwCiiigAooooAKKKKAM7WNEs9dht4L4O8EM6z+UGwsjLyAw7jPOPYVfkijmQpKiuh6qwyDTqKQBTZI0mieKVFeN1KsrDIIPUGnUUwKGjaTbaFpcOnWbSm2hyI1kfcVUknaD6DOB7VfoopANaON2RnRWZDuQkZKnGMj04JH406iimAUUUUAZ1rolna61e6uod727VEeR2ztRRwq+g7+5rL/AOEE0H/hI114wztqCStMrtcOVVj1wucD6V0tFKyHdhRRRTEZPhb/AJBE3/YRvv8A0qlrarF8Lc6NIw6PfXjqfUG5lIP5EVtV50viZoFFFFSAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAVz2j/wDJT9c/7Blp/wChzV0Nc9o//JT9c/7Blp/6HNUz+FmlL4jsaKKK5jrPPPjd/wAkr1L/AK6wf+jFoo+N3/JK9S/66wf+jForsw/wmNTc0/Bn/Ii+Hv8AsGW3/opa3Kw/Bn/Ii+Hv+wZbf+ilrcr0Vscb3Csbwl/yBZ/+wnqH/pXNWzWN4S/5As//AGE9Q/8ASuasMR8KHE3aKKK5CgooooA4fVP+Sy6B/wBgy4/mK7iuH1T/AJLLoH/YMuP5iu4psbPGPjh/yMHg7/rvJ/6HFXs9V7mws7xo2urSCdozlDLGGK/TPToKsUN6BfQKKKKQgooooAKKKKACiiigAooooAKKKKACiiigAooooAK4fxz/AMjV4J/7Cbf+gGu4rh/HP/I1eCf+wm3/AKAaa3GjuK8/+NP/ACTK+/66w/8AoYr0Cori2gu4TDcwRzRHkpIgZT+BoWjEjnvh3/yTvQf+vNP5V01MihjgiWKGNI40GFRFAAHoAKfSYBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAcr8Sv+Sca7/17H+Yrc0X/kA6d/17R/8AoIrD+JX/ACTjXf8Ar2P8xW5ov/IB07/r2j/9BFPoPoXqKKKQgooooAwtf/5C/hj/ALCb/wDpJc1s1ja//wAhfwx/2E3/APSS5rZrsofCTIKw/Gf/ACIviH/sGXP/AKKatysPxn/yIviH/sGXP/opq2ewlud7RRRXKdIUUUUAFFFFABRRRQAUUUUAFFFFABXMfEb/AJJt4k/7B03/AKCa6euY+I3/ACTbxJ/2Dpv/AEE0Aadl/wAeNv8A9cl/lU9QWX/Hjb/9cl/lU9eadIVxXhv/AJGbxh/2E0/9J4q7WuK8N/8AIzeMP+wmn/pPFWlLdmNb4TpaKKK2OYKKKKACiiigAooooAKxNZjmsr+21qCJ5liRoLuKNdzNESCGUDklCM4HUM2MnArbopxk4u6Aq2t3b31slzaTxzwSDKyRsGUj6ipqzrnw3pVzcPcG3eCeQ5kltJ5Ld3PqxjZST9ar/wDCJab/AM/Gs/8Ag6vP/jtdSxC7E8ps0Vjf8Ilpv/PxrP8A4Orz/wCO1Fc+GtHs7WW5uLzV44IULyO2tXmFUDJJ/e0fWI9g5TeorirHwnJqUiX09/rmn2p5hs01a5LsvYys0hwT/dXGO5PbZ/4RPTv+fnWf/B1d/wDx2tk21sKyNyisP/hE9O/5+dZ/8HV5/wDHaP8AhE9O/wCfnWf/AAdXn/x2nqGhuUVh/wDCJ6d/z86z/wCDq8/+O0f8Inp3/PzrP/g6vP8A47RqGhuUVh/8Inp3/PzrP/g6vP8A47R/wienf8/Os/8Ag6vP/jtGoaG5RWH/AMInp3/PzrP/AIOrz/47R/wienf8/Os/+Dq8/wDjtGoaG5RWH/wienf8/Os/+Dq8/wDjtH/CJ6d/z86z/wCDq8/+O0ahoblFYf8Awienf8/Os/8Ag6vP/jtH/CJ6d/z86z/4Orz/AOO0ahoblFYf/CJ6d/z86z/4Orz/AOO0f8Inp3/PzrP/AIOrz/47RqGhuUVh/wDCJ6d/z86z/wCDq8/+O0f8Inp3/PzrP/g6vP8A47RqGhuUVh/8Inp3/PzrP/g6vP8A47R/wienf8/Os/8Ag6vP/jtGoaG5RWH/AMInp3/PzrP/AIOrz/47R/wienf8/Os/+Dq8/wDjtGoaG5RWH/wienf8/Os/+Dq8/wDjtH/CJ6d/z86z/wCDq8/+O0ahoblFYf8Awienf8/Os/8Ag6vP/jtH/CJ6d/z86z/4Orz/AOO0ahoblFYf/CJ6d/z86z/4Orz/AOO0f8Inp3/PzrP/AIOrz/47RqGhuUVh/wDCJ6d/z86z/wCDq8/+O0f8Inp3/PzrP/g6vP8A47RqGhuUVh/8Inp3/PzrP/g6vP8A47R/wienf8/Os/8Ag6vP/jtGoaG5RWH/AMInp3/PzrP/AIOrz/47R/wienf8/Os/+Dq8/wDjtGoaG5RWH/wienf8/Os/+Dq8/wDjtH/CJ6d/z86z/wCDq8/+O0ahoblFYf8Awienf8/Os/8Ag6vP/jtH/CJ6d/z86z/4Orz/AOO0ahoblFYf/CJ6d/z86z/4Orz/AOO0f8Inp3/PzrP/AIOrz/47RqGhuUVh/wDCJ6d/z86z/wCDq8/+O0f8Inp3/PzrP/g6vP8A47RqGhuUVh/8Inp3/PzrP/g6vP8A47R/wienf8/Os/8Ag6vP/jtGoaG5RWH/AMInp3/PzrP/AIOrz/47R/wienf8/Os/+Dq8/wDjtGoaG5RWH/wienf8/Os/+Dq8/wDjtNfwhpciFJJtXZTwVbWbsg/+RaNQ0Ha1eG9MmhafJuvbhdkzpz9lib7zsezYztHUnHYEjdhijt4I4YlCxxqEVR0AAwBXOWngfRLCHybP+07eLJbZDq10i5PU4EnWp/8AhFNP/wCfnWf/AAdXn/x2sKlOU2NNI36KwP8AhFNP/wCfnWf/AAdXn/x2geFdPByLrWc++s3Z/wDatZ/V5dx3Rv0Vz8ltqujgz2N1NqNsvL2dywaTH/TOTg59nznpla2bO8g1Czhu7Z98Myh0bGOD6jsfbtWc4OG4yeiiioAKKKKACiiigAooooAKKKKACiiigAooooA53x9/yT/xB/14S/8AoJrq7L/jxt/+uS/yrlPH3/JP/EH/AF4S/wDoJrq7L/jxt/8Arkv8qzq7I6KHUnooorA3Ob8F/wDIweMv+wqn/pNDXY1x3gv/AJGDxl/2FU/9Joa7GvRh8KOd7hRRRVCCiiigAooooAKKKKACiiigAooooA4nwv8A8jP4z/7Cif8ApNDXU1y3hf8A5Gfxn/2FE/8ASaGuprgq/GzeOwVyXxG/5Fm3/wCwnZf+lEddbXJfEb/kWbf/ALCdl/6UR0ofEgl8LN2iiiug4gooooAKKKKAMnWLG5ea21LT1Vr203DymbaJ4mxujz2PyqQexUdiaLHXdPv5DCs3k3a/ftJ/3cyfVDzj3GQexNa1VrzTrHUYxHfWdvdRjos8SuPyIrWnVcNBNXJaKx/+EL8K/wDQs6N/4ARf/E0f8IX4V/6FnRv/AAAi/wDia1+seQuU2KKwLzwt4OsLKa7ufDmjJBChd2+wRHAHtt5+lZ1n8OtD1NkvtV0DT7ZSd0NhBbJGIx28xlALv6jO0dMHG4p4lLoXCk5vQ7Cis0eCfCgAH/CM6McetjET/wCg0f8ACE+FP+hY0X/wAi/+Jqfra7Gv1Z9zSorN/wCEJ8Kf9Cxov/gBF/8AE0f8IT4U/wChY0X/AMAIv/iaPra7B9Wfc0qKzf8AhCfCn/QsaL/4ARf/ABNH/CE+FP8AoWNF/wDACL/4mj62uwfVn3NKis3/AIQnwp/0LGi/+AEX/wATR/whPhT/AKFjRf8AwAi/+Jo+trsH1Z9zSorN/wCEJ8Kf9Cxov/gBF/8AE0f8IT4U/wChY0X/AMAIv/iaPra7B9Wfc0qKzf8AhCfCn/QsaL/4ARf/ABNH/CE+FP8AoWNF/wDACL/4mj62uwfVn3NKis3/AIQnwp/0LGi/+AEX/wATR/whPhT/AKFjRf8AwAi/+Jo+trsH1Z9zSorN/wCEJ8Kf9Cxov/gBF/8AE0f8IT4U/wChY0X/AMAIv/iaPra7B9Wfc0qKzf8AhCfCn/QsaL/4ARf/ABNH/CE+FP8AoWNF/wDACL/4mj62uwfVn3NKis3/AIQnwp/0LGi/+AEX/wATR/whPhT/AKFjRf8AwAi/+Jo+trsH1Z9zSorN/wCEJ8Kf9Cxov/gBF/8AE0f8IT4U/wChY0X/AMAIv/iaPra7B9Wfc0qKzf8AhCfCn/QsaL/4ARf/ABNH/CE+FP8AoWNF/wDACL/4mj62uwfVn3NKis3/AIQnwp/0LGi/+AEX/wATR/whPhT/AKFjRf8AwAi/+Jo+trsH1Z9zSorN/wCEJ8Kf9Cxov/gBF/8AE0f8IT4U/wChY0X/AMAIv/iaPra7B9Wfc0qKzf8AhCfCn/QsaL/4ARf/ABNH/CE+FP8AoWNF/wDACL/4mj62uwfVn3NKis3/AIQnwp/0LGi/+AEX/wATR/whPhT/AKFjRf8AwAi/+Jo+trsH1Z9zSorN/wCEJ8Kf9Cxov/gBF/8AE0f8IT4U/wChY0X/AMAIv/iaPra7B9Wfc0qKzf8AhCfCn/QsaL/4ARf/ABNH/CE+FP8AoWNF/wDACL/4mj62uwfVn3NKis3/AIQnwp/0LGi/+AEX/wATR/whPhT/AKFjRf8AwAi/+Jo+trsH1Z9zSorN/wCEJ8Kf9Cxov/gBF/8AE0f8IT4U/wChY0X/AMAIv/iaPra7B9Wfc0qKzf8AhCfCn/QsaL/4ARf/ABNH/CE+FP8AoWNF/wDACL/4mj62uwfVn3NKis3/AIQnwp/0LGi/+AEX/wATR/whPhT/AKFjRf8AwAi/+Jo+trsH1Z9zSorN/wCEJ8Kf9Cxov/gBF/8AE0f8IT4U/wChY0X/AMAIv/iaPra7B9Wfc0SQASSAB3NYl5rX20vp+hSJcXrfI86fPFa+rO3TcOoTqTjoMkWv+EJ8Kf8AQsaL/wCAEX/xNH/CE+FP+hY0X/wAi/8AiaTxV1ohrDeZesLKHTtPt7K3BEMEaxpuOTgDHJ7n3qxWT/whPhT/AKFjRf8AwAi/+Jo/4Qnwp/0LGi/+AEX/AMTXPzof1d9zWorJ/wCEJ8Kf9Cxov/gBF/8AE0f8IT4U/wChY0X/AMAIv/iaOdB9Xfc1qKyf+EJ8Kf8AQsaL/wCAEX/xNH/CE+FP+hY0X/wAi/8AiaOdB9Xfc1qKyf8AhCfCn/QsaL/4ARf/ABND+EdJiTdpduukzr9yXT1EWD/tIPlcezAijmQfV33NaiszS7+eaWew1BEj1C2AMmz7kqHO2RM9jgjHYgjngnTqjBpp2YUUUUCCiiigAooooAKKKKACiiigAooooAK57R/+Sn65/wBgy0/9Dmroa57R/wDkp+uf9gy0/wDQ5qmfws0pfEdjRRRXMdZ558bv+SV6l/11g/8ARi0UfG7/AJJXqX/XWD/0YtFdmH+Exqbmn4M/5EXw9/2DLb/0UtblYfgz/kRfD3/YMtv/AEUtbleitjje4VjeEv8AkCz/APYT1D/0rmrZrG8Jf8gWf/sJ6h/6VzVhiPhQ4m7RRRXIUFFFFAHD6p/yWXQP+wZcfzFdxXmfi/QrXX/ivodpdyXKRvpsxJgmMbfK2RyPrWp/wqrQf+fzWf8AwYPVOwzuKK4f/hVWg/8AP5rP/gwej/hVWg/8/ms/+DB6WgaHcUVw/wDwqrQf+fzWf/Bg9H/CqtB/5/NZ/wDBg9GgaHcUVw//AAqrQf8An81n/wAGD0f8Kq0H/n81n/wYPRoGh3FFcP8A8Kq0H/n81n/wYPR/wqrQf+fzWf8AwYPRoGh3FFcP/wAKq0H/AJ/NZ/8ABg9H/CqtB/5/NZ/8GD0aBodxRXD/APCqtB/5/NZ/8GD0f8Kq0H/n81n/AMGD0aBodxRXD/8ACqtB/wCfzWf/AAYPR/wqrQf+fzWf/Bg9GgaHcUVw/wDwqrQf+fzWf/Bg9H/CqtB/5/NZ/wDBg9GgaHcUVw//AAqrQf8An81n/wAGD0f8Kq0H/n81n/wYPRoGh3FcP45/5GrwT/2E2/8AQDR/wqrQf+fzWf8AwYPXKeK/h/pOn6/4Wt4bnU2S8vjFIZLx2IGwn5Seh96atcFY9jorh/8AhVWg/wDP5rP/AIMHo/4VVoP/AD+az/4MHpaBodxRXD/8Kq0H/n81n/wYPR/wqrQf+fzWf/Bg9GgaHcUVw/8AwqrQf+fzWf8AwYPR/wAKq0H/AJ/NZ/8ABg9GgaHcUVw//CqtB/5/NZ/8GD0f8Kq0H/n81n/wYPRoGh3FFcP/AMKq0H/n81n/AMGD0f8ACqtB/wCfzWf/AAYPRoGh3FFcP/wqrQf+fzWf/Bg9H/CqtB/5/NZ/8GD0aBodxRXD/wDCqtB/5/NZ/wDBg9H/AAqrQf8An81n/wAGD0aBodxRXD/8Kq0H/n81n/wYPR/wqrQf+fzWf/Bg9GgaHcUVw/8AwqrQf+fzWf8AwYPR/wAKq0H/AJ/NZ/8ABg9GgaHcUVw//CqtB/5/NZ/8GD0f8Kq0H/n81n/wYPRoGhe+JX/JONd/69j/ADFbmi/8gHTv+vaP/wBBFea+OPh1o+leCtWvoLrVGlggLKst67qTkdQetekaEgj8PaYgzhbSIDJ/2BQ9g6GhRRRSEFFFFAGFr/8AyF/DH/YTf/0kua2axtf/AOQv4Y/7Cb/+klzWzXZQ+EmQVh+M/wDkRfEP/YMuf/RTVuVh+M/+RF8Q/wDYMuf/AEU1bPYS3O9ooorlOkKKKKACiiigAooooAKKKKACiiigArmPiN/yTbxJ/wBg6b/0E109cx8Rv+SbeJP+wdN/6CaANOy/48bf/rkv8qnqCy/48bf/AK5L/Kp6806QrivDf/IzeMP+wmn/AKTxV2tcV4b/AORm8Yf9hNP/AEnirSluzGt8J0tFFFbHMFFFFABRRRQAUUUUAFFFFABRRRQAVieIx550mxb/AFd3fosg9VRHmx9CYgD7GtusXXP+Qr4c/wCwi/8A6S3FXT+JAa1FFFegZhRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAVkeHv3U+tWi/6u31Ftg9PMjjmP/j0rVr1k6H/AMhXxH/2EU/9JbesK/wlRNqiiiuMoKKKKACiiigAooooAKKKKACiiigAooooA53x9/yT/wAQf9eEv/oJrq7L/jxt/wDrkv8AKuU8ff8AJP8AxB/14S/+gmursv8Ajxt/+uS/yrOrsjoodSeiiisDc5vwX/yMHjL/ALCqf+k0NdjXHeC/+Rg8Zf8AYVT/ANJoa7GvRh8KOd7hRRRVCCiiigAooooAKKKKACiiigAooooA4nwv/wAjP4z/AOwon/pNDXU1y3hf/kZ/Gf8A2FE/9Joa6muCr8bN47BXJfEb/kWbf/sJ2X/pRHXW1yXxG/5Fm3/7Cdl/6UR0ofEgl8LN2iiiug4gooooAKKKKACiiigAooooAx9dAmutEs35iuNRXePXy45Jl/8AHolrfrA1j/kM+G/+wi//AKS3Fb9RM66HwhRRRUG4UUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQBhayoh8QaDcpxJLNLaMfVGieT/wBChWtasrXv+Qp4c/7CL/8ApLcVq1rHY46/xhRRRTMQooooAKKKKACiiigAooooAKKKKACue0f/AJKfrn/YMtP/AEOauhrntH/5Kfrn/YMtP/Q5qmfws0pfEdjRRRXMdZ558bv+SV6l/wBdYP8A0YtFHxu/5JXqX/XWD/0YtFdmH+Exqbmn4M/5EXw9/wBgy2/9FLW5WH4M/wCRF8Pf9gy2/wDRS1uV6K2ON7hWN4S/5As//YT1D/0rmrZrG8Jf8gWf/sJ6h/6VzVhiPhQ4m7RRRXIUFFFFAHD6p/yWXQP+wZcfzFdxXD6p/wAll0D/ALBlx/MV1esSX8Wj3j6XAs9+Im+zxuwCs+OMk8YzTY2XaK8C1rw/420TwlN4u1jxdfWurRyg/YfO+XBfaFBVtvvtAxivXvBOsXev+DNL1S+jCXVxDukAGASCRuA7Zxn8aGrA0b9FFFIQUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFcP45/5GrwT/wBhNv8A0A13FcP45/5GrwT/ANhNv/QDTW40dxRRXj974U8deJtV1e/1rX7jw/Z25JsooZwY9vJBOxxjAAyTzzQlcSR7BRXm3wZ8T6t4j8NXS6rI9w9nMIo7l/vSAjOCe5Hr7ivSaGrMGrBRRRSAKKKKACiiigAooooAKKKKACiiigAooooAKKKKAOV+JX/JONd/69j/ADFbmi/8gHTv+vaP/wBBFYfxK/5Jxrv/AF7H+Yrc0X/kA6d/17R/+gin0H0L1FFFIQUUUUAYWv8A/IX8Mf8AYTf/ANJLmtmsbX/+Qv4Y/wCwm/8A6SXNbNdlD4SZBWH4z/5EXxD/ANgy5/8ARTVuVh+M/wDkRfEP/YMuf/RTVs9hLc72iiiuU6QooooAKKKKACiiigAooooAKKKKACuY+I3/ACTbxJ/2Dpv/AEE109cx8Rv+SbeJP+wdN/6CaANOy/48bf8A65L/ACqeoLL/AI8bf/rkv8qnrzTpCuK8N/8AIzeMP+wmn/pPFXa1xXhv/kZvGH/YTT/0nirSluzGt8J0tFFFbHMFFFFABRRRQAUUUUAFFFFABRRRQAVi65/yFfDn/YRf/wBJbitqsXXP+Qr4c/7CL/8ApLcVdP40BrVzuk+NdH1rxFfaDavML+y3+akkZUfKwU4PfkiuirxW9X/hG/2jLWb7kGqovHrvQp/6GoNd0nYlK53viL4keHfC+rrpmpzzLclFkIjiLBQScZP4VseIfEWneGNIbVNSkZbZWVcou4kk8YFfPHiWxm8Za3428RRMzQ6YyCPHRlDhB+GxGatzx34gk8QfDbwVYQtvuL5hvAPJeIeVz9WY1HO9SuRaHqknxF0CKXRY3e5VtYCtaAwnkM20Z9Oa6yvKPF2qf8Iz4t8GaFb6dptxCRBAJbm2DyRgSKg2N/Dxz9aseNPiDrvh/wCIVjoWm2UF5FcwIVhYEO8jF1Ubs4AyFzx0zVc1txct9j0+ivIJPHfjbwv4x0zTvFNtpz2mouqj7KD8gZtvynPVSRkHP171r+PfHuraX4isfC/hm0in1a6AZnlGQgOcADIGcAkk8Afo+ZC5WdR4m8ZaR4SazXVHmU3jMsXlx7skYzn0+8K6CvnL4mXnilrrQbHxTa2izwyPJDc2rHZMrFAQR2IIH5j8fTviD4/u/Dt/Y6Fodkl5rd9gxq/KoCdo4BGSSD3AGMmkpb3G47Hf0V4/P488b+CtRsz4z0+0l0y6baZrUDdGe+CDjIHOCOccGt+98balp3xU07Qp/szaLqUKvbyqh35KkD5s4PzL6dGFPmQuVnoNFeTTfFO9j+Lg8Ohbb+yPtS2hbYfM3ldv3s4/1hx06Ctm38Z6rqXxdn8M2K250uziL3MhQl8hRnBzgfMyjp2NHMg5WegVwuofF7wdp1+9m+oSTMjbXeCFnRT9e/4Zrsr61F9p9zaGWSITxNEZIzhk3AjIz3Ga85v/AAz4S+G/gLU0vEju/tAcI11GrSyuVwqKcduvHTk0Sb6ArdT0PTNUstZ0+K/065S4tZhlJEPB/wAD7GrdeVfB+y1XTfhjfXSREzXEks9lE4+9hAFOPQsv+c12Hgi+8SahoksviiySzvxcMqRou0GPauD1Pct+VCdwasbWqajb6Rpd1qN2WFvbRmWQqMnaOuBWf4c8VWHiiGWbT4rtI4wp3XEDRhw2SCueo47e1V/H/wDyT7X/APryk/8AQax01pvD3wSs9UjOJYNHg8o+jsiqp/76Iob1BLQ6jTtes9UutShtt5TT5fJmnYARlwMsqnPJXjPpmucl+K3haOWTbPdy20bbHu4rR2hU/wC8B/KrHh3wxCfhna6HNLNELuz/ANJliYCQtINznJB5yxHParN/NofgPwOIbnYNPtLfyFicDdOcY247s3OeO5PSjUNDftLu3v7SK7tJkmt5lDxyIchge4qauM+Fel3ukfD+wt79GjlcvMsTdY1ZiQPbjnHvXZ007oT3KGs63p3h/TX1DVLpLa2Q4Ltk5J6AAck+wrA034keHdR1GCx826tJ7g4t/tls0SzH/ZJGK09d8MWXiG90q5vZJiNNn+0RwqR5cjcY3gg5xjjGOprkPizcQ6ra6f4VsVE+uXl1HJCq8tbqDzIT/CMZ/DPpSbaGkmdrr/iPS/DNit3qlyIUdtkahSzyN6Ko5JqnoHjXRfEd3LZWcs0V7Eu97W5haKTb6gHqPpXOXijUvjnp1tP+8i0zSGuYlbkCRn2lseuCPypfiEBYeK/BerwgLc/2mtm7jq0cnBB9R1/M0rvcLI6bxD4v0fwy8EV/NI1zcZ8m2gjMkrgdwo7e5qTw94p0nxPDM+mXBd4G2zQyIUkiPoynkVyvhUDUvix4x1CcBpLJbe0gJ52IVJYD0yVz+JovwNN+OWkyW4Cf2pp0sVwB/GUywY+/ygZ9BRd7hZbHY6brlpqd9qNjEJI7rT5RHPFKAGwRlXGCcqw6GtKuB8TynQfiX4Z1aL5Y9T36ZdAfxZIMZ+oY/kK76qTE0FZt7rlpZavYaUwklvL0sY44gDtRRlnbJ4UcD3J4rSrgfA8p13xf4q8Qy/MI7n+zLUn+COPlsf7xIahsEja1zxzoegagun3Ms898V3m2tIWldV9WA6fjWhoPiLS/EtibzSroTxq2xwVKtG3oynkGs2az0jwbJqviAW17PcahKpn8pDNIx5ACgcge1c18M7mPWvEvirxDDsto72aNBYFh5sewEb5FH3SxyfrupXd7DsrHpdBIAJJwB1Joqvf2UWo6fc2U5cRXETROUba21hg4Pbg1RJycvxT8KxXEii6uZLeN/Le8jtXaBW9N4H6jiui1HW7XT9Bl1kCS6s44hMWtgHJj67hzyAOfoK47xVq+k+A/B7eH7bSry5j+xNHGFh3RANlcyv0GTkmtvwBpy2Xw+0mykuob1fIO6SNg6MGYkqD3Azt/CpTd7FNK1zT1HxBZadoserET3NpIFKG0iMpYNyCAO1J4d8R6f4o0w6hpplMAkaI+amwhhjPH41zPwyuHtYtc8NOxYaLfvFAScnyGJKD9GqL4Pf8AIoXn/YTuP5ihMGj0GiiiqJCiiigAooooAKKKKACiiigArJ0P/kK+I/8AsIp/6S29a1ZOh/8AIV8R/wDYRT/0lt6wr/AVE2qKKK4ygooooAKKKKACiiigAooooAKKKKACiiigDnfH3/JP/EH/AF4S/wDoJrq7L/jxt/8Arkv8q5Tx9/yT/wAQf9eEv/oJrq7L/jxt/wDrkv8AKs6uyOih1J6KKKwNzm/Bf/IweMv+wqn/AKTQ12Ncd4L/AORg8Zf9hVP/AEmhrsa9GHwo53uFFFFUIKKKKACiiigAooooAKKKKACiiigDifC//Iz+M/8AsKJ/6TQ11Nct4X/5Gfxn/wBhRP8A0mhrqa4Kvxs3jsFcl8Rv+RZt/wDsJ2X/AKUR11tcl8Rv+RZt/wDsJ2X/AKUR0ofEgl8LN2iiiug4gooooAKKKKACiiigAooooAx9Y/5DPhv/ALCL/wDpLcVv1gax/wAhnw3/ANhF/wD0luK36ie510PhOMT4hW//AAslvBk2nyxT4JS5MgKv+78wcY9M/iKqePPilZeBdStbCawlvJp4fOIjkC7FyQOo74P5Vx/xWX/hHvin4V8Sj5YnZElI7hHG780fH4Vi67pTfEP4k+MXTLxaVYSJb4/56xgKq/iwc1SjHR9CnJ6o9m1PxhZ6b4E/4SsxtJam2juFjDAFt+3aufXLAVzF18W0tfC+j65/YF28epySqkSyAlAjbck47nP5V5bqXig3fwF0nSA5a4GotbOO5RMuP/Q4x+Fdl8RdR1jwD4S8H6bo99JZssZhn8sD5yqpnOR6lj+NHItg5nue00V5J8WfFXiDw/4p8OW+iXZT7QctbkDZM29QFbvg5xx61heM7/x98O7zTdYu/FA1KO6kIlthFtiBGCUC9MEZ5GDxUqDZTnY94rlvHfjWHwLo0GpT2Ul2s1wIAiOFIJVmzk/7v61yvxS8aazY6hovhrw3IINQ1UqTOQMqrNtUDOcZOcntiuF+KujeLdB8M2FtrWvrrNhLdBxI8e2SGYI3y5zkqQW6/wB3oO7jC9rilK17H0NY3QvdPtrtVKieJZApOcbgDj9anryfxv441Pw74c8NaLoIX+19Tt4lSQqG8sYVRgHjJJwM9MGsbXl+I/w6sYPEFz4kXVrYSKt1bSAlVz257Z4yMHJHFJQuPmPcaK8f8VeONSsda8GeI7O9li8PaqqC5tjgqpyN2Tjrh/8Axysnx98SdZ0f4oJb2N7Kmkae9ul3Eigo+TufJx1IOPwoVNsHNI92ory3VvEmsan8bNN8OaTqEkOnWsSzXyx4IfguQTjoQUX8a9RIDKQehGOtS1YpO55hq/xr0yz1W5sdK0e+1YWpInng4RccEjg5HucCuv8AB/jLS/GuktfaaZFMbbJoJQA8be+Ox7Ef41i3n/CI/CDQLy8tLb7P9pbKW4lZ3ncDhRuJOBnk9BmsH4FaDe2Oi6nrd5F5I1WVGhixj5F3fMB6Eucew96tqPLdEJu9mes1DeXAs7G4uSpYQxtIVHfAziuV8Bab4v06C+Hi3UY72R3Q25R920YOf4R7V0Wtf8gHUf8Ar1k/9BNQ1Z2LvoYXgvxjdeL7YXh0G4sbB4y8NzJMjrKQ20gAcjoevpV/TPEaarr2q2Ntb5s9NIimvTJwZ+rRgY/hGMnPU4x3rk/AupHRvgRb6moBa0srmZQe7K8hA/PFanw20qJPhlp8NwDIdQhe4uWJOZDKSSSRznBAz7VTS1JTehUX4lz6iZ5/D/hXVNX06ByjXkRVFcjr5YPL11HhrxJp/ivRk1PTXcxFijpIu14nHVWHYjj86oajqPh/4b+FEG0W1lbgpbWqMWeRiSdq5JJJJP0rN+F+iahpeg3t9qkP2e81e9kv3tunkh8YUjse/wCIHak0rXBXvY7isvxD4g0/wvos2q6lIyW8WBhBlnY9FUdya1KztX0LTdeit4tTthcR2863ESl2ULIucHgjPU8HipVr6lM5CT4mXGnxxXmueEtW0zSpWC/bJNr+XnoXQcqK6PxJ4s07w1pEV/ceZcG4dY7WC2G97h2+6qDvn1/+tXL/ABO1sX9k3gnSIxea3qgVDEvIt48gl3Pbgf19M09UsRa/Ez4d6JI5lgsLOYqW/iZIsBvr8gNXZPUm7N3TfH7Sa3a6Trug32iXN7n7I1wyvHMR/DuXo3t/9bOh4n8YW3hyezso7O51HVL0kW1lbAF2A6sSeFUev+Bxz3xo/c+DrO+Tiez1KCaJh1DAkcfnSWH+lfH3Vnl+Y2ejxxw5/hDMrHH5n86LJq4Xext+HvG6avrMuiajpd1o+rpH5wtrkhhKn95GHDY/z0OL8PiJT4uuPD11bfZ5hAtzaS+ZuFzH0bHAwynqOeOa5bx8fs3xC8BXkXyzG8lgLDqUYKCP1P5074qSHSV8OeJIvln07VEViO8MgIkX8cAUWTt5hdo9DoooqCzE1rxEul6npWl29t9rv9Rm2pCH2BI1GXkY4PCjt3PFZ/iDxvHpOtR6Hpul3esau8XnNbWxCiJPV2PC5/z1GcfwvIda+LPizUpeV02OHTrYH+EHLP8A+PL+tdXe2UGlPf63p2ji61WaNVdYiEknxgAFjwMD19KqyTsTqyl4Z8ZW/iC9u9MnsbrTdWswGnsroDcFPRlI4ZeRz7j1FdLXlXg6+m1L4sapfeILdtJ1o2Qt7XTGGcwBgxfzOjnI7e/px6rRJWYRd0FcCfiXJeSXM2heGNT1fTbVykt7AVVWI6+WDy/4V3joskbI6hlYEEHuK5PW57/wdocNn4R8K/bkCyMI4ZFjSA5zkqeWyWJwPSlGwM1NM8Rwa/4XGtaGn2sPEzRQyN5ZZxn5GODtORjv+Iqbw7rtt4k0K21S1DIsoIeJvvRODhkb3BBFct8IFsY/AyLaXhuZzcSPeAoUMUxwWTaemBj69ah8IyHS/ih4w0BOLaUx6lCvZWdR5n5kj8qbS1Qk9j0OiisTxffPp3hPUrmM4cRbFI7FiFB/Wobsrm9Gk6tSNOO7aX3iaD4jXX7vUFt7bbZ2snlJcb8+c3fAxwPx7ijxP4ntfDFgk86GWWRtscKtgt6nPYD/AArO8EC20fwDb3U7rHGVe4lc/U/0AFcNr0d14j0jUfFd4Hjt1dIbGE9l3gE/z/En0FYSqSUPM9zD5dQq46UXpSjJR9Xtb5vV9l2PWtKvxqmlWt+sZjFxGJAhOdue2auVjeEv+RR0n/r2T+VbNbRd0meLiIKFacY7Jv8AMxfE/iKPwzpkd7LbtOrzCLarYIyCc/8AjtWdD1m217SYdQtshX4ZCeUYdQa5b4r/APIq2/8A1+p/6A9YPhi7n8F+I4tPvXP9nalGkkch6AsOD+fyn8DWLqONSz2PYo5ZTr5d7WH8S7fqluvXqdx4e8UxeIL3ULaO1eE2ThSzMDuyWH/stSp4g8zxe+gx2u7yrfzpZ/M+70wNuPcd+9cp8Nv+Q94k/wCuy/8AoT1o+Cf+JjrniLWzys1z5ETf7K//AFtv5U4Tk1HzFi8FQo1a9l7sYxtq/ilb/gs1fFPiu28LQW7zQtPJOxCxqwBwByf1H51f0PWINe0iDUYFKrKDlCclSDgg1wF0kXjH4mvbzENp+nxsjDPB28H/AMfP5Cp/hzdvpWtap4auXyUkZ4j2JXhsfUbT+FJVG5+WxdbLKUcFeP8AFilJ+jvp8tGzrW8RrD4tGhXNt5Rli823n35EvqMY4PB7np71k+J/HN14X1SCC48P3MthPPFbx3yzoELv229eOfyqj8Tt1kNG1eHie2ucA+v8Q/8AQf1qH4uuJfDugSL91tbtWH0IetqTvNxZ5mMoQjh6VenpzJp+qev36M9GoooqjiMPXv8AkKeHP+wi/wD6S3FatZWvf8hTw5/2EX/9JbitWtI7HHX+MKKKKoxCiiigAooooAKKKKACiiigAooooAK57R/+Sn65/wBgy0/9Dmroa57R/wDkp+uf9gy0/wDQ5qmfws0pfEdjRRRXMdZ558bv+SV6l/11g/8ARi0UfG7/AJJXqX/XWD/0YtFdmH+Exqbmn4M/5EXw9/2DLb/0UtblYfgz/kRfD3/YMtv/AEUtbleitjje4VjeEv8AkCz/APYT1D/0rmrZrG8Jf8gWf/sJ6h/6VzVhiPhQ4m7RRRXIUFFFFAHD6p/yWXQP+wZcfzFdff39ppdjLe31wlvbRDMkshwqjOOfxNchqn/JZdA/7Blx/MV1+oWFtqmn3FheRCW2uIzHIh7qRTfQbPM/iL4Y0fxl4dn8ZWeryubOyZrbaVa3cRlmIKkZJJyOvpxXR/C/xFc+JvA9reXkaJPE7W7GNAqttxghRwOCOBxkGuTb4ITKj2EHi/UI9EeTe1iUJB+vzBSffb+FenaHotj4e0e20rTovLtrdcKCcknqST3JOTTbVrA7WNCiiipEFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABXD+Of+Rq8E/wDYTb/0A13FcP45/wCRq8E/9hNv/QDTW40dxXFa1YeHPinpLWFvrUkkNpMHc2Ug4cqQu7IORyen512teX6r8Hi2s3WoeHPEd5oYvCftEMKkqcnJA2svHsc0IEVfgxrN0JdY8KzeTLBpUhEE8UYTcN5U5x1zjIJ565Jr1quY8FeB9N8EaZJbWTPNPOwae5kGGkI6DHYDJwPc109Dd2D3CiiikIKKKKACiiigAooooAKKKKACiiigAooooAKKKKAOV+JX/JONd/69j/MVuaL/AMgHTv8Ar2j/APQRWH8Sv+Sca7/17H+Yrc0X/kA6d/17R/8AoIp9B9C9RRRSEFFFFAGFr/8AyF/DH/YTf/0kua2axtf/AOQv4Y/7Cb/+klzWzXZQ+EmQVh+M/wDkRfEP/YMuf/RTVuVh+M/+RF8Q/wDYMuf/AEU1bPYS3O9ooorlOkKKKKACiiigAooooAKKKKACiiigArmPiN/yTbxJ/wBg6b/0E109cx8Rv+SbeJP+wdN/6CaANOy/48bf/rkv8qnqCy/48bf/AK5L/Kp6806QrivDf/IzeMP+wmn/AKTxV2tcV4b/AORm8Yf9hNP/AEnirSluzGt8J0tFFFbHMFFFFABRRRQAUUUUAFFFFABRRRQAVi65/wAhXw5/2EX/APSW4rarF1z/AJCvhz/sIv8A+ktxV0/jQGtXjfx3tJrT+wPEFqTHPazmLzB1DcOn5FWr2Sop7aC6j8u4gjmTOdsiBhn6Gu6SurEJ2dzzD4R+G45fhfdrcr/yGWmDkjnZjyx/Jj+NeZfDvTbvUfiPpGlXeTHpU0khQ9IyhLH/AMfAr6eiijhjWOKNY41GFVBgD6Co47G0hnaeK1gSZ87pFjAY55OT1pcmxXPueQfFP/kqngv/AK7Q/wDo8UeLRn9oXw5/1xi/9Ckr2CWztp5UlmtoZJI+Ud0BK9+CelD2dtJcLcPbQtOvCyMgLD6HrQ4gpHkHxl/5HHwd/wBdj/6MjqDx3I/hD4y6T4ru4JH0yRArSIudp2NGw+oBDY717NNZ21y6PPbwysnKM6Biv0z0pbm1t72BoLqCKeFvvRyoGU/UGhxBSPnz4qeMdO8XX+iHRxLNZ2krBrpomRWdynyjcAcgLz9a6D4kpceGPihovjGW2kn0xVRJWQZ2MNwI9jtbIz1Oa9dXStOW3jt1sLUQxnKRiFdqn2GMCrEsUc0TRyoskbDDKwyCPcUcocx4Z8RvGFj8RINK8OeF4p724kuRMzGIoFIVlA5/3iSegx1ra+LOiS6X4U8PavanddaDJFH5n+zhQCf+BIv516hZaVp2mljY6fa2pf73kQqmfrgVZmhiuImimjSSNuqOoIP4Gjlve4c1tj5uGh3M/wAKLnxkRjUTrAvPNA52Btn/AKMYn8K774KWU15b634pvF/0nVLtgD7AlmI9izY/4DXqAtLYW32YW8Qt8Y8rYNuOvTpToYIreIRQRJFGvREUKB+AoULO4OV1Yh1PUINJ0u61C6YrBbRNLIR1wozx718zt4y03xR4zOseM/tkunw/8e1haqGUDPCnLDjuT1J9BX0/JFHNG0cqK8bDDKwyCPcVU/sXSv8AoGWf/fhf8KJRbFFpGF4d8daPr3hy+1XTbe5js9OVg8bxqhwqbsKASOn0q34Q8XWXjPSJNSsILiGJJ2gKzhQ2QFOeCePmFbMNla28TxQ20Mcb/eRIwA31A6063tre0jMdtBFChOSsaBRn1wKpXFoYPj//AJJ9r/8A15Sf+g1ymvWkl7+zxDFECWXSbWXA9E8tz+imu08X6fc6r4P1aws4/Mubi1eOJCwXLEcDJ4FZvgqDWR4Zj0bxDokdnHa2sVqh+0rMLhQm1iQv3eg496lrUaehaTxFZaZ4At/EE7E2sdjHNherZUYUe5JA/GvMdC8UeGte1ZPEvjTXYDdRsTY6WEdorQZ4J+XDPwDn/wCsB6L4P0S70fSrvw9qVus9haTEWU0m1xNAx3KCOu5TxyPTFbv9i6V/0DLP/vwv+FFmwukUU8X6FL4dn1+K/EmmQEiSdI2OMEA8Yz3HatWxvINRsLa+tX329zEs0TYI3KwBBweRwaqajpKXOh3em2Qt7Tz42QN5AZUJ/i28An+uKs6dYxaXpdpp8GfJtYUhTd12qoUZ/AVWotDkfiT44/4Q/TLeG1MQ1K+YpA0v3IVGN0jewyOP8MVz/hLxF4A8NrLdz+JY7/Wbrm7v5opC7n0X5eF9vYe2PT7ixtLsqbm1gmK8AyRhsfnUP9i6V/0DLP8A78L/AIUmne401axw3iW5i8M/E/SfFF2Smk3ti1hNcYJWJ929S3oDwPwNQ6/qll418c+GdL0S4jvYdOuv7QvJ4DujjCY2jcOCScj8RXpM0ENxA0E8UcsLDDRuoZSPQg1FZadY6bEYrGzt7WMnJWCJUBP0Ao5Quee29/a+C/iprn9rTLaWGuxxT21zLxHvQEMpboDliefb1FOsLuDxh8XYtV0x/P0vRbJojdL9x5nyNqnv8p6+30r0K7srS/gMF7aw3MJOfLmjDr+Rpba1t7KBYLW3ighX7scSBVH0A4osFzg/iKpvPFHgfTo+Zm1UXOB1CRYZj+Rrr4/EOmS+I5dAS4J1KKD7Q8WxsBMgZ3Yx3HGax7DRby98eX3iLU4fKjtY/sWmxFgTs6vLweCxOB3x1rUXQx/wlj67JKpYWn2WKNY8EAsGZmbPzE7VA6YA96FfcHY1689+FSmzTxRpUnE9rrUzMP8AZYDafx2mvQq5R9FvNM+IK63p8Pm2WpwiDUUVgDG6D93Lgnnj5Tj1zQ97iW1ira+PoLPxNq2i+JWttKe3k3Wcsr7EuIT0bc3GemR+HY1j+GZofEHxd1XX9GBOkxWItZrlVwlxNkHj1wB19h6ivQ73TLDU0VL+xtrtFOVWeJZAPpkVNBBDbQrDbxJFEgwqRqFUfQCizHdFDSfEOma5Pfw6fcGWSwnNvcAoy7HBII5HPQ8iptXnvbbR7yfToFuLyOJnhhY4EjAZC/j0qro2hjS73VbxpVkn1C585tkexUUAKq4ycnAyT3JJrXpiOF074n+GrvwuL3Vr62trpYit3YP/AKwOBhlCHlhnOP1pPhHp15p/gdTdQvAtzcyXFvC/WOJsbR+hP411sui6VPeC8l0yykugcid4FL/99YzTdbnv7fRbuTS7U3N+IyIItwUFzwCSSBgZyfYUrPdjv0Rxvw9BufGHjrUk5gl1BLdG7Fogwb/0IUvwe/5FC8/7Cdx/MVqabpOoeC/BNvZaPp66tqKNvnVpxD5sjHLvub36ewFRfDPRNT0HwxNbatbC2uZLyWbyxIr4VsY5UketJLVDb0Z2VFFFWQFFFFABRRRQAUUUUAFFFFABWTof/IV8R/8AYRT/ANJbetasnQ/+Qr4j/wCwin/pLb1hX+AqJtUUUVxlBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFAHO+Pv+Sf+IP+vCX/ANBNdXZf8eNv/wBcl/lXKePv+Sf+IP8Arwl/9BNdXZf8eNv/ANcl/lWdXZHRQ6k9FFFYG5zfgv8A5GDxl/2FU/8ASaGuxrjvBf8AyMHjL/sKp/6TQ12NejD4Uc73CiiiqEFFFFABRRRQAUUUUAFFFFABRRRQBxPhf/kZ/Gf/AGFE/wDSaGuprlvC/wDyM/jP/sKJ/wCk0NdTXBV+Nm8dgrkviN/yLNv/ANhOy/8ASiOutrkviN/yLNv/ANhOy/8ASiOlD4kEvhZu0UUV0HEFFFFABRRRQAUUUUAFFFFAGPrH/IZ8N/8AYRf/ANJbit+sDWP+Qz4b/wCwi/8A6S3Fb9RPc66Hwnmfxw0KbWPA8c9rbyTXFndJIFjQsxVsqQAPcqfwqt8C9ButO8OalqOoQTRXd/dYImUhiiDgnPPLM9eq0Uuf3eU15feufMOn+BtRj+LkGjNZXX9lwao0quYm8rygd/XGOVVRXefHqwvL6Dw8LO0nuCk0pbyYy+3hOuBxXsVFV7R3TJ5NGjx34s2F5d+P/B0tvaTzRxTKZHjjLBP3qHkgcVL8fLC8v9B0hLO0nuGW6YsIYy5A298V67RSU7W8huN7+Z458VdB1eDVfDnjHSrN7xtMWMTwopLAI29SQOccsD6cVzvxH8T6v8QfDdp/ZvhnU4NPtrgPLJLESXlKsAqgdQBuyfcdO/0LRQp2toDhc8Z+InhXWrjTfCniXRrR7m80mCHzbdVJf5drKQvU4IIIHPP1rM8V+NdY+JWjxeGdF8L38M88qG6eYfLHtOcZxwM4OTjp05r3mihT8gcDzDxx4HYfBmHRrdTcXekxRyxlFJLsvD4HXkM5x9K4jw/4P1LW/hj4x1PUrW4Oq3cqyxCWMiRjF85IBGfm3Mv4V9DUUKbSsDgmzxv4H6HqP2nWfEesQzpdzbLWMzoUYqACxwe33B+Br2CeZbe3lncMVjQuQoySAM8DvUlFTKXM7jSsrHy1eazrOt+OD4g8ReFdS1S2jJ8iwKvHGig/Kp+RsgdSMcnr6V7l4C8a3vi43yXXhy40ZbQR7PNcsJN27gZRcY2j1612dFVKaa2FGLT3OP8AAXjDUfF0F9JqOhyaWbZ0VFct+8BByfmUdMfrXRa1/wAgHUf+vWT/ANBNXqr6hbtd6bdWyEK00LxgnoCQRU31K6HnXg3TpNX+AEenxDMtxYXMcY9WLybf1xWt4D1uNfhJp2o+XJL9ismR4oxly0WV2gep28fUU/4feHfEnhbTY9J1S802fTreMi3Fsr+YGLljuJ4I5NaHh/w7ceHtc1g20kJ0a/l+1xw5IeGc8SADGNhwD14PGO9U2tSUnoeUaF4nhuNfk8S+L9C8QX2qq5FnbxWBa3s07bASMt7ke/XmvV9J8ZW+s6HqGqQaXqkK2Ssxhubfy5JMLuwgyc56fWukqOeMzW8kSyvEzqVEiY3ISOoyCMj3FJyT6DSaM/w5rkPiTQLXV4IZYIrgMRHMAHXDFSDj3FYHxL8S6n4c8M7tFtJrjUbp/KiaKIyeSMZLkAdu2e59q6fSdLttF0q202zVhBbpsXc2WPqSe5JyT9auUrpO49bHjXhHxV4e8J2b+X4e8U3WpXB33l/Np+ZJ36nJLcDPQfzPNb/jKO8kufCvjvS9Ourj+z8vcWgT9/5EyDPy/wB5QTke/tXo1FPm1uTy6WPKNf1hPidd6RoWiWd6bCO8S61G7nt2iSNEz8nzdWOf0HvjQ8SC58JfEeDxeLK4utKu7L7FfG2jLvAwYFZCo5xwB+B9gfR6KOYfKeYw3T/EL4g6LqVjZ3UehaGskpuriIxieZwAFQHk4wDn6+2bPxdU6hpeh6DFzcalqsSBR/cXJZvoMivRa5seHbi78dnxBqEkLQWdv5GmwoSShb/WSNkDDH7oxnimpa37Ca0LF54mhsvF+neHXs7lpb6J5I7gKPKXaGJUnPXC/qK3Kz59Ht7nXLXVZnlaW0ieOCPcNiF/vPjGSxAx16dua0Kh2KPOvBqHS/il410yXhrpob+D/aQg7iPozAfhVjVfFereE/GN0dcguJ/DNzEhtLm2tt/2ZwPmV9ozzycnPbHfGzrnh24uvEekeINMkhjv7JjDOJSQs9s33kJAPIPzL2zXSVTa3Ek9jzDTnuPHHxQ03xJZ2FzbaNpFvJGlzcxGM3LuGGFB52jdnPsfWuz07xNDqPifVNCWzuYZtPVHaWRQElVuhXnJHBH4VuVn2mj29prF9qgeWS6vAiMZGBCIgO1FAHAyWPrlj7UNpglYuXCyvbSpBII5mQhHIztbHBx35rzjSfiTcaLYNp3jOx1FNdgdlzBZlkuhk7TGVG3pgdulel0Uk11G12OD+GWj6jaQ63rWp2jWU2tXzXaWbfeiQkkbh2Y5P6fSq3hdDqXxg8X6tHzb2sUNgrdi4ALj8Cv613179q+wz/YRCbvy28kTEhN+ONxAJxn0FZPhHw6vhjQI7FpfPupHae7uP+e0znLt/QewFPm3YrbIXwr4mh8VabPeQ2dzaeTcNbvFcqFcMoBPAP8AtCm+NbR73wdqcKDLCLzMD/YIb/2Wr2j6Pb6LaSQQPLK0szzzSzMC8kjnJY4AHtwBwBV9lDKVYAqRgg96iaTTRth6ro1Y1FvFp/ccVoNhF4m+GNtpzTmIOvls6jJUpJnp+A/Oud8R/D59I0GW4ttQu7tkKhbcR5ByQOgPau28L6Bc+HX1C182J9Okm821AY70z1DDGOw79veuirH2SlFcy1PYea1cLiZPDyvTcua2mt9bbXXY4TQLW58JeCJdTRJ7yeWOKX7K4IKHIBUdTxnPTtXU6Bqja1odrqLRCIzqW2Bs45I6/hVjUbRr/T5rVbmW3Mo2mWLG4DvjPqMj8adY2cGnWMNnbJshhQIg9hVxi4uy2OLE4qGIhKpNfvHK9/K23b9e5yXxQt57nwxbpbwySuLxSVjUscbH54qbxD4a/t/wZaRImL62gR4cjBztGV/H+YFdfRQ6abbfUdPMalKnTjBWcG2n6nkvgZr3R9J8SXtxb3Eci26shkQgs/z469eSK6bQRL4d+Ga3EcLvdNC0yoqkku5+Xj8V/Ku0oqY0uVbm+KzX6zOUpQ+KUW9d+VWt+Z5R4Y+HUOs6R9u1aW8gnlkbai4U7Rxk7gTknNR6v4Ym8Fa7pepaQt3dxB8yDbuYY6j5QOCpI/OvW6KXsI2st+5r/b+JdZynrB393pZ9Dz74nyfbNP0exgy0l3c7o1xyeMDj/gYqH4uRiLw54fjXomtWqj8A9dJceHri/wDGkGr3kkJs7OLbbRKSW392YYwOvYnoKxfGvhXxN4p1K0hivdMi0a2u4bpEdX87cg5yRkY5P6VrSVpuTOLG1oPDUaEHflTb9ZPb5JI72iiiqOAw9e/5Cnhz/sIv/wCktxWrWVr3/IU8Of8AYRf/ANJbitWtI7HHX+MKKKKoxCiiigAooooAKKKKACiiigAooooAK57R/wDkp+uf9gy0/wDQ5q6Gue0f/kp+uf8AYMtP/Q5qmfws0pfEdjRRRXMdZ558bv8Aklepf9dYP/Ri0UfG7/klepf9dYP/AEYtFdmH+Exqbmn4M/5EXw9/2DLb/wBFLW5WH4M/5EXw9/2DLb/0Utbleitjje4VjeEv+QLP/wBhPUP/AErmrZrG8Jf8gWf/ALCeof8ApXNWGI+FDibtFFFchQUUUUAcPqn/ACWXQP8AsGXH8xXcV5n4vn1mD4r6G+i2dtdXQ02bbHPLsUjd83PtxWp/anxI/wChc0b/AMDj/hVNDsdxRXD/ANqfEj/oXNG/8Dj/AIUf2p8SP+hc0b/wOP8AhSsFjuKK4f8AtT4kf9C5o3/gcf8ACj+1PiR/0Lmjf+Bx/wAKLBY7iiuH/tT4kf8AQuaN/wCBx/wo/tT4kf8AQuaN/wCBx/wosFjuKK4f+1PiR/0Lmjf+Bx/wo/tT4kf9C5o3/gcf8KLBY7iiuH/tT4kf9C5o3/gcf8KP7U+JH/QuaN/4HH/CiwWO4orh/wC1PiR/0Lmjf+Bx/wAKP7U+JH/QuaN/4HH/AAosFjuKK4f+1PiR/wBC5o3/AIHH/Cj+1PiR/wBC5o3/AIHH/CiwWO4orh/7U+JH/QuaN/4HH/Cj+1PiR/0Lmjf+Bx/wosFjuKK4f+1PiR/0Lmjf+Bx/wo/tT4kf9C5o3/gcf8KLBY7iuH8c/wDI1eCf+wm3/oBo/tT4kf8AQuaN/wCBx/wrlPFd/wCNpNf8LNfaLpkU6XxNqkd0WEj7DwxxwMd6aWoJHsdFcP8A2p8SP+hc0b/wOP8AhR/anxI/6FzRv/A4/wCFKwWO4orh/wC1PiR/0Lmjf+Bx/wAKP7U+JH/QuaN/4HH/AAosFjuKK4f+1PiR/wBC5o3/AIHH/Cj+1PiR/wBC5o3/AIHH/CiwWO4orh/7U+JH/QuaN/4HH/Cj+1PiR/0Lmjf+Bx/wosFjuKK4f+1PiR/0Lmjf+Bx/wo/tT4kf9C5o3/gcf8KLBY7iiuH/ALU+JH/QuaN/4HH/AAo/tT4kf9C5o3/gcf8ACiwWO4orh/7U+JH/AELmjf8Agcf8KP7U+JH/AELmjf8Agcf8KLBY7iiuH/tT4kf9C5o3/gcf8KP7U+JH/QuaN/4HH/CiwWO4orh/7U+JH/QuaN/4HH/Cj+1PiR/0Lmjf+Bx/wosFjuKK4f8AtT4kf9C5o3/gcf8ACj+1PiR/0Lmjf+Bx/wAKLBYvfEr/AJJxrv8A17H+Yrc0X/kA6d/17R/+givNfHGoeOpfBWrJqWh6XBZtARLJFdlmVcjkDHNekaEWPh7TC4Af7JFuA7HYKHsHQ0KKKKQgooooAwtf/wCQv4Y/7Cb/APpJc1s1ja//AMhfwx/2E3/9JLmtmuyh8JMgrD8Z/wDIi+If+wZc/wDopq3Kw/Gf/Ii+If8AsGXP/opq2ewlud7RRRXKdIUUUUAFFFFABRRRQAUUUUAFFFFABXMfEb/km3iT/sHTf+gmunrmPiN/yTbxJ/2Dpv8A0E0Aadl/x42//XJf5VPUFl/x42//AFyX+VT15p0hXFeG/wDkZvGH/YTT/wBJ4q7WuK8N/wDIzeMP+wmn/pPFWlLdmNb4TpaKKK2OYKKKKACiiigAooooAKKKKACiiigArF1z/kK+HP8AsIv/AOktxW1WN4lR0sbfUI0Z2064W6KqMkoAVkwO52O5A7kCqg7SQGrRTIpY54UlidXjdQyOpyGB5BB9KfXomYUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFZOh/8hXxH/2EU/8ASW3rTlljgheWV1SNFLO7HAUDkkn0rN8NI72NxqEiMjajcNdBWGCEICx5HY7EQkdiSKwxD92xUTZooorjKCiiigAooooAKKKKACiiigAooooAKKKKAOd8ff8AJP8AxB/14S/+gmursv8Ajxt/+uS/yrlPH3/JP/EH/XhL/wCgmursv+PG3/65L/Ks6uyOih1J6KKKwNzm/Bf/ACMHjL/sKp/6TQ12Ncd4L/5GDxl/2FU/9Joa7GvRh8KOd7hRRRVCCiiigAooooAKKKKACiiigAooooA4nwv/AMjP4z/7Cif+k0NdTXLeF/8AkZ/Gf/YUT/0mhrqa4Kvxs3jsFcl8Rv8AkWbf/sJ2X/pRHXW1yXxG/wCRZt/+wnZf+lEdKHxIJfCzdoooroOIKKKKACiiigAooooAKKKKAMfWP+Qx4b/7CL/+ktxW/WLrtrPcWCTWab7y0lW5gTON5Xqme25Sy57bq0NP1C21Sxju7V98TjuMFSOCrDqGByCDyCKifc6qD0sWqKKKg6AooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigDD17/kKeHP+wi//pLcVq1jPKNW8Ux+Ud1rpSuHcdDcOMbR7qhbP/XQehrZrWOxxVneYUUUUzIKKKKACiiigAooooAKKKKACiiigArntH/5Kfrn/YMtP/Q5q6Gue0f/AJKfrn/YMtP/AEOapn8LNKXxHY0UUVzHWeefG7/klepf9dYP/Ri0UfG7/klepf8AXWD/ANGLRXZh/hMam5p+DP8AkRfD3/YMtv8A0UtblYfgz/kRfD3/AGDLb/0Utbleitjje4VjeEv+QLP/ANhPUP8A0rmrZrG8Jf8AIFn/AOwnqH/pXNWGI+FDibtFFFchQUUUUAcPqn/JZdA/7Blx/MV3FcPqn/JZdA/7Blx/MV1+oX9tpenXF/eSCO2t42kkc9lAyabGyzRXkLfG66eKXUbTwbfzaLE+1r0yEAfXCFQeRxu/GvS9A12y8S6Jbatp7s1vOuQGGGUg4KkeoIIoaaBpo06KKKQgooooAKKKKACiiigAooooAKKKKACiiigAooooAK4fxz/yNXgn/sJt/wCgGu4rh/HP/I1eCf8AsJt/6Aaa3GjuKKK8v1T4vyf21d6b4b8NXmt/YyRPNCxAGDg4CqxxnucZoSuJK56hRXKeCPHmneNtMnuII3tbi1IFzbysCY85wc9wcHnjoeK5C8+N3+k3cuk+GbzUNJtHCzXwcqoGcZwEIAPbJGfajlY7M9aorDsPFuk3/hIeJY5ymnCFpXZx8ybc7lI9QQRjv2rz1fjk2VvpPCl8mhvN5Qvt+efpt259t340JNhZnr1FQ2l3Bf2UF5ayCS3njWSNx0ZSMg/lU1IQUUUUAFFFFABRRRQAUUUUAFFFFABRRRQByvxK/wCSca7/ANex/mK3NF/5AOnf9e0f/oIrD+JX/JONd/69j/MVuaL/AMgHTv8Ar2j/APQRT6D6F6iiikIKKKKAMLX/APkL+GP+wm//AKSXNbNY2v8A/IX8Mf8AYTf/ANJLmtmuyh8JMgrD8Z/8iL4h/wCwZc/+imrcrD8Z/wDIi+If+wZc/wDopq2ewlud7RRRXKdIUUUUAFFFFABRRRQAUUUUAFFFFABXMfEb/km3iT/sHTf+gmunrmPiN/yTbxJ/2Dpv/QTQBp2X/Hjb/wDXJf5VPUFl/wAeNv8A9cl/lU9eadIVxXhv/kZvGH/YTT/0nirta4rw3/yM3jD/ALCaf+k8VaUt2Y1vhOlooorY5gooooAKKKKACiiigAooooAKKKKACiiigDC/se+0x2OiXEK2zEsbG5U+WpPXy2HKD2ww9AKQ3HinPGj6MR6nVZR/7b1vUVoqs0rXFZGD9o8Vf9AbRv8AwbS//I1H2jxV/wBAbRv/AAbS/wDyNW9RT9tPuFkYP2jxV/0BtG/8G0v/AMjVT1HXPEGlwpJc6TpG6RtkUUeqTPJK391VFtkn+Q5OBXVVg6Ugv9W1DVpfmKTPZ22f+WccZ2vj3aQNk9wqjtWlOc5u1w0ILXUfFdzbLJJ4c0+2kPWKfVWyPxSFhUv2rxT/ANAbRv8AwbS//I1blFdNibmH9q8U/wDQG0b/AMG0v/yNR9q8U/8AQG0b/wAG0v8A8jVuUUWAw/tXin/oDaN/4Npf/kaj7V4p/wCgNo3/AINpf/katyiiwGH9q8U/9AbRv/BtL/8AI1H2rxT/ANAbRv8AwbS//I1blFFgMP7V4p/6A2jf+DaX/wCRqPtXin/oDaN/4Npf/katyiiwGH9q8U/9AbRv/BtL/wDI1H2rxT/0BtG/8G0v/wAjVuUUWAw/tXin/oDaN/4Npf8A5Go+1eKf+gNo3/g2l/8AkatyiiwGH9q8U/8AQG0b/wAG0v8A8jUfavFP/QG0b/wbS/8AyNW5RRYDD+1eKf8AoDaN/wCDaX/5Go+1eKf+gNo3/g2l/wDkatyiiwGH9q8U/wDQG0b/AMG0v/yNR9q8U/8AQG0b/wAG0v8A8jVuUUWAw/tXin/oDaN/4Npf/kaj7V4p/wCgNo3/AINpf/katyiiwGH9q8U/9AbRv/BtL/8AI1H2rxT/ANAbRv8AwbS//I1blFFgMP7V4p/6A2jf+DaX/wCRqPtXin/oDaN/4Npf/katyiiwGH9q8U/9AbRv/BtL/wDI1H2rxT/0BtG/8G0v/wAjVuUUWAw/tXin/oDaN/4Npf8A5Go+1eKf+gNo3/g2l/8AkatyiiwGH9q8U/8AQG0b/wAG0v8A8jUfavFP/QG0b/wbS/8AyNW5RRYDD+1eKf8AoDaN/wCDaX/5Go+1eKf+gNo3/g2l/wDkatyiiwGH9q8U/wDQG0b/AMG0v/yNR9q8U/8AQG0b/wAG0v8A8jVuUUWAw/tXin/oDaN/4Npf/kaj7V4p/wCgNo3/AINpf/katyiiwGH9q8U/9AbRv/BtL/8AI1H2rxT/ANAbRv8AwbS//I1blFFgMP7V4p/6A2jf+DaX/wCRqPtXin/oDaN/4Npf/katyiiwGH9q8U/9AbRv/BtL/wDI1H2rxT/0BtG/8G0v/wAjVuUUWAw/tXin/oDaN/4Npf8A5Go+1eKv+gNo3/g2l/8AketyiiwHG3Fl4w1G6H9pWWizWKsGWzi1GVFcjp5hMBLgenA9Qa2Ptvin/oC6N/4Npf8A5GraoqJUoyd2PmMX7b4p/wCgLo3/AINpf/kaj7b4p/6Aujf+DaX/AORq2qKn2EA5jF+2+Kf+gLo3/g2l/wDkanC/8RoN02iWDAdVttSZ2/APEg/Wtiij2EA5ippur2+pNLEqSwXUOPOtp12yR56E9QQcHDAkHB54NX6wdbUWt5pmqJxJFdJbSEfxxzMI9p9t7I31Wt6uapDkdhhRRRWYwooooAKKKKACiiigDnfH3/JP/EH/AF4S/wDoJrq7L/jxt/8Arkv8q5Tx9/yT/wAQf9eEv/oJrq7L/jxt/wDrkv8AKs6uyOih1J6KKKwNzm/Bf/IweMv+wqn/AKTQ12Ncd4L/AORg8Zf9hVP/AEmhrsa9GHwo53uFFFFUIKKKKACiiigAooooAKKKKACiiigDifC//Iz+M/8AsKJ/6TQ11Nct4X/5Gfxn/wBhRP8A0mhrqa4Kvxs3jsFcl8Rv+RZt/wDsJ2X/AKUR11tcl8Rv+RZt/wDsJ2X/AKUR0ofEgl8LN2iiiug4gooooAKKKKACiiigAooooAKyrnRAbt73T7ubT7uTmVoQGSY9BvRgQT/tDDYGM4rVooBNrVGMYPFWfl1nR8f7WlSE/pcD+VJ5Piv/AKDOi/8Agpl/+Sa2qKVkX7WfcxfJ8V/9BnRf/BTL/wDJNHk+K/8AoM6L/wCCmX/5Jraoosg9rPucvqV94m04RKdV0qe5nJWC2h0eUySkdcZuQAB3JIA7mrVjH42mts315oNtMegjspZRj3/fLg/Qn61PoEYu73UtXkG6SS4ktISf+WcUTFCo+rq7H1yPQVvVEn0R1U07XbML7J4r/wCg1ov/AIKJf/kmj7J4r/6DWi/+CiX/AOSa3aKVzSxhfZPFf/Qa0X/wUS//ACTR9k8V/wDQa0X/AMFEv/yTW7RRcLGF9k8V/wDQa0X/AMFEv/yTR9k8V/8AQa0X/wAFEv8A8k1u0UXCxhfZPFf/AEGtF/8ABRL/APJNH2TxX/0GtF/8FEv/AMk1u0UXCxhfZPFf/Qa0X/wUS/8AyTR9k8V/9BrRf/BRL/8AJNbtFFwsYX2TxX/0GtF/8FEv/wAk0fZPFf8A0GtF/wDBRL/8k1u0UXCxhfZPFf8A0GtF/wDBRL/8k0fZPFf/AEGtF/8ABRL/APJNbtFFwsYX2TxX/wBBrRf/AAUS/wDyTR9k8V/9BrRf/BRL/wDJNbtFFwsYX2TxX/0GtF/8FEv/AMk0fZPFf/Qa0X/wUS//ACTW7RRcLGF9k8V/9BrRf/BRL/8AJNH2TxX/ANBrRf8AwUS//JNbtFFwsYX2TxX/ANBrRf8AwUS//JNH2TxX/wBBrRf/AAUS/wDyTW7RRcLGF9k8V/8AQa0X/wAFEv8A8k0fZPFf/Qa0X/wUS/8AyTW7RRcLGF9k8V/9BrRf/BRL/wDJNH2TxX/0GtF/8FEv/wAk1u0UXCxhfZPFf/Qa0X/wUS//ACTR9k8V/wDQa0X/AMFEv/yTW7RRcLGF9k8V/wDQa0X/AMFEv/yTR9k8V/8AQa0X/wAFEv8A8k1u0UXCxhfZPFf/AEGtF/8ABRL/APJNH2TxX/0GtF/8FEv/AMk1u0UXCxhfZPFf/Qa0X/wUS/8AyTR9k8V/9BrRf/BRL/8AJNbtFFwsYX2TxX/0GtF/8FEv/wAk0fZPFf8A0GtF/wDBRL/8k1u0UXCxhfZPFf8A0GtF/wDBRL/8k0fZPFf/AEGtF/8ABRL/APJNbtFFwsYX2TxX/wBBrRf/AAUS/wDyTR9k8V/9BrRf/BRL/wDJNbtFFwsYX2TxX/0GtF/8FEv/AMk0fZPFf/Qa0X/wUS//ACTW7RRcLGF9k8V/9BrRf/BRL/8AJNH2TxX/ANBrRf8AwUS//JNbtFFwsYX2TxX/ANBrRf8AwUS//JNH2TxX/wBBrRf/AAUS/wDyTW7RRcLGF9k8V/8AQa0X/wAFEv8A8k0fZPFf/Qa0X/wUS/8AyTW7RRcLGF9k8V/9BrRf/BRL/wDJNUtQ0XxfqEYjHiqxtYz98W2lOpcem4zlh9VIPvXVUUXYWOZs9F8RWFpHa2uq6JFBGMKi6RLgf+TPJ7571N9g8Vf9BvRv/BRL/wDJNdBRT52R7KHY5/7B4q/6Dejf+CiX/wCSaPsHir/oN6N/4KJf/kmugoo52HsodjAFj4pB+bWdHI9BpMo/9uKF1O9sLqK21q3hjSZhHDeW7ExM56KwPKE9uSD0zkgVv1XvrKDUbGezuk3wTIUdenB9D2PvQpvqS6MWtBaKzPD1zNdaJA1y++4iZ7eZ/wC+8TtGzfiUJ/GtOtDjegUUUUAFFFFABRRRQAVz2j/8lP1z/sGWn/oc1dDXPaP/AMlP1z/sGWn/AKHNUz+FmlL4jsaKKK5jrPPPjd/ySvUv+usH/oxaKPjd/wAkr1L/AK6wf+jForsw/wAJjU3NPwZ/yIvh7/sGW3/opa3Kw/Bn/Ii+Hv8AsGW3/opa3K9FbHG9wrG8Jf8AIFn/AOwnqH/pXNWzWN4S/wCQLP8A9hPUP/SuasMR8KHE3aKKK5CgooooA4fVP+Sy6B/2DLj+Yrr9Q0+01Wwmsb6BZ7WYbZI26MM5/pXIap/yWXQP+wZcfzFb3izUr/SPC9/e6XZTXl+keIIYYjIxckAHaOSBnJ9hTfQZwXxF1/RvCXhR/BWhWqte3kZhjtIct5KyEkk9Tk5OB15zXV/Dfw5ceF/BFjp93xdHdNMuc7GY52/gMD65rxTwpL4r8Oa1c61deBtV1XVJmLLc3NtNmMn7xHydT6+nA717z4P1rUte0Fb3VtJl0u6MjIbaVWUgDocMAeaqWiG9Eb1FFFQSFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABXD+Of+Rq8E/wDYTb/0A13FcP45/wCRq8E/9hNv/QDTW40dxXE6vd+FPhXo93fW9nFbS3Z+W3iY77hxnA5JwBnk9Bn1wK7R3EcbOQxCgnCjJP0FfNF8/i3V/G58Raz4L1bUY0YmCyltJhGij7q/d5A6kdz1pxVwSudP4K0bUtP+HnjLxTexmCfVLOaSBMbSF2ud+OwJbj2Ge9X/AIX20L/BPXFZQRN9q35HX90B/IV0nhnXdb8bWWr6Vr3hm50a3e1MSvLHIu8OCpA3KOgrzrTZvGXgnw7rPgz/AIRe8vHvGkWC7gRmQB1CMRhSCMDI5GD1qtyjP0i8nX9nvXoQx2f2mifRT5RI/MfrXY39tCP2Zo1CjAtopBx0Yzgk/mT+daWkfDa6i+Dd34buNkepXpNywJyElypVSR7IoJ+tcRJdeNLrwJF8Pf8AhE75bhZBG10UYIYw+8DONo5wN27GBRuG56p8JZnn+F+iPISWCSIM+iyuo/QCu0rH8KaIPDnhbTdI3B2toQrsvQueWI9sk1sVD3Ie4UUUUgCiiigAooooAKKKKACiiigAooooA5X4lf8AJONd/wCvY/zFbmi/8gHTv+vaP/0EVh/Er/knGu/9ex/mK3NF/wCQDp3/AF7R/wDoIp9B9C9RRRSEFFFFAGFr/wDyF/DH/YTf/wBJLmtmsbX/APkL+GP+wm//AKSXNbNdlD4SZBWH4z/5EXxD/wBgy5/9FNW5WH4z/wCRF8Q/9gy5/wDRTVs9hLc72iiiuU6QooooAKKKKACiiigAooooAKKKKACuY+I3/JNvEn/YOm/9BNdPXMfEb/km3iT/ALB03/oJoA07L/jxt/8Arkv8qnqCy/48bf8A65L/ACqevNOkK4rw3/yM3jD/ALCaf+k8VdrXFeG/+Rm8Yf8AYTT/ANJ4q0pbsxrfCdLRRRWxzBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFYfhn/kFT/8AYRvv/SqWtysPwz/yCp/+wjff+lUtdGH+JiexsUUjMqDLMAPc4oyMZyMdc11kC0U1XV/usGx6HNG9N23cu70zzQA6ikV1cZVgw9jmloAKKKKACimq6vnawOOuDTqACiiigAooooAKKKRWVhlSCPUGgBaKKKACimtIiHDOoPuaHkSJC8jqiDksxwBQA6ikyMZyMdc0AhhkEEeooAWiiigAopCwXGSBngZNBIUEkgAckmgBaKajrIgdGDKwyGU5BFOoAKKKb5iCQRl18wjIXPOPXFADqKKKACiiigAooprSIjKrOqs/CgnBb6UAOoprOiY3MFz6nFKCCMg5B7igBaKKKACiiigAooooAKKKKACiiigAooooAKKKKAMfxN/yCoP+wjY/+lUVblYfib/kFQf9hGx/9Koq3K5MR8SLWwUUUVzjCiiigAooooAKKKKAOd8ff8k/8Qf9eEv/AKCa6uy/48bf/rkv8q5Tx9/yT/xB/wBeEv8A6Ca6uy/48bf/AK5L/Ks6uyOih1J6KKKwNzm/Bf8AyMHjL/sKp/6TQ12Ncd4L/wCRg8Zf9hVP/SaGuxr0YfCjne4UUUVQgooooAKKKKACiiigAooooAKKKKAOJ8L/APIz+M/+won/AKTQ11Nct4X/AORn8Z/9hRP/AEmhrqa4Kvxs3jsFcl8Rv+RZt/8AsJ2X/pRHXW1yXxG/5Fm3/wCwnZf+lEdKHxIJfCzdoooroOIKKKKACiiigAooooAKKKKACiiigAooooAKKKKAMvwp/wAgab/sI33/AKVS1t1ieFP+QNN/2Eb7/wBKpa0dQ1Ky0mye81C6itbZCA0srbVXJwMn6kVk9z0I/Ci1RVaz1Cz1CxS+s7mKe1kBZJo2BVgOCQfwNU9K8TaHrk7waVq1neSou9kglDELnGSB2pWHc1aKzZ/EGkWurRaVPqVtHqEuDHbNIBI2emB17Gl0zXtJ1mS4j03Uba7e3IEywyBjGTnGcdOh/KizC5o0UUUDCiis6z17SdQ1G40+z1G2nvLYkTwRyAvHg4OR2weKBGjRRRQMKKKKACiio7i4gtLd7i5mjhhjG55JGCqo9STwKAJKKytM8T6FrU7QaZrFjeTKMmOGdWYD1wDnHvWrQIKKKy9V8R6LoTxJquqWlk0oJjE8oTcB1xmgDUoqrJqVlDpbanJcxLYiLzjOW+TZjO7Ppikk1Swh0xdSnu4oLIoJPPmby1CnoSWxjr3oAt0Vm6V4h0bXN/8AZWqWd6Y/viCZXK/UA5FaVABRRWbqviDRtD2f2rqlpZGT7gnmVC30BPNAGlRUFne2uoWqXVlcw3Nu/KywuHVvoRxUen6lZ6rbtcWNws0ayNExX+F1OGUg8gg9jQBbooooGFFVJNTs4tRi097hRdyqXSLqSozz7Dg1ZkkSKNpJGCIgLMzHAAHUmlcpwkrXW+w6iqthqNnqcBnsbmO4iDbS8ZyM+n6irVCdxSjKL5ZKzCio554rWB555FjijG53Y4Cj1NR2V/aajB59lcxXEQbaXjYMM+lF+g+SXLzW07liiqn9qWP9o/2f9rh+2Yz5G8b8Yz0+nNF5qdlp8lvHdXCxPcP5cSnOXb0H5ii6GqU20uV3euxboqK4uIbS3ee4lSKFBlnc4AHuajsdRs9ShM1jdRXEattLRMGAPp+oouthcknHmtp3LNFVbjUrO1vbeznuFjuLnPko3G/HXB6VaouJxkkm1uFFFIzKiF3YKqjJJOABTELRVCLW9Mm0ddXW+gXTmXcLmRtke3OM5bHGe9UR418KE4HifRST/wBP8X/xVFmK6N2iiigYUVQ1LW9J0cx/2pqllY+bny/tVwkW/GM43EZxkfnUVj4l0HVLkW2n63pt3OQWEVvdJI2B1OASaLMVzUoqpaalZ3811DbXCyS2kvlToODG2M4IPsc56Gqdt4q0C81X+y7bWLKW/wBzJ9mSYF9y5LDHqMH8qLMLmvRRRQMKKKKACiiigDA8M/8AILuP+wjff+lUtbFY/hn/AJBdx/2Eb7/0qlrYrY86W7CiiigQUUUUAFFFFABXPaP/AMlP1z/sGWn/AKHNXQ1z2j/8lP1z/sGWn/oc1TP4WaUviOxooormOs88+N3/ACSvUv8ArrB/6MWij43f8kr1L/rrB/6MWiuzD/CY1NzT8Gf8iL4e/wCwZbf+ilrcrD8Gf8iL4e/7Blt/6KWtyvRWxxvcKxvCX/IFn/7Ceof+lc1bNY3hL/kCz/8AYT1D/wBK5qwxHwocTdooorkKCiiigDh9U/5LLoH/AGDLj+YruK848Ua1p2ifFrQrrUryO2gXTZwXkPAJbj88Gt3/AIWV4M/6GKy/76P+FU0Ox1VFcr/wsrwZ/wBDFZf99H/Cj/hZXgz/AKGKy/76P+FKzFZnVUVyv/CyvBn/AEMVl/30f8KP+FleDP8AoYrL/vo/4UWYWZ1VFcr/AMLK8Gf9DFZf99H/AAo/4WV4M/6GKy/76P8AhRZhZnVUVyv/AAsrwZ/0MVl/30f8KP8AhZXgz/oYrL/vo/4UWYWZ1VFcr/wsrwZ/0MVl/wB9H/Cj/hZXgz/oYrL/AL6P+FFmFmdVRXK/8LK8Gf8AQxWX/fR/wo/4WV4M/wChisv++j/hRZhZnVUVyv8AwsrwZ/0MVl/30f8ACj/hZXgz/oYrL/vo/wCFFmFmdVRXK/8ACyvBn/QxWX/fR/wo/wCFleDP+hisv++j/hRZhZnVUVyv/CyvBn/QxWX/AH0f8KP+FleDP+hisv8Avo/4UWYWZ1VcP45/5GrwT/2E2/8AQDV7/hZXgz/oYrL/AL6P+Fcj4u8ceGb7xF4TntdZtpYrS/aSdlJxGuwjJ4ppO40mesUVyv8AwsrwZ/0MVl/30f8ACj/hZXgz/oYrL/vo/wCFKzFZnVUVyv8AwsrwZ/0MVl/30f8ACj/hZXgz/oYrL/vo/wCFFmFmdVRXK/8ACyvBn/QxWX/fR/wo/wCFleDP+hisv++j/hRZhZnVUVyv/CyvBn/QxWX/AH0f8KP+FleDP+hisv8Avo/4UWYWZ1VFcr/wsrwZ/wBDFZf99H/Cj/hZXgz/AKGKy/76P+FFmFmdVRXK/wDCyvBn/QxWX/fR/wAKP+FleDP+hisv++j/AIUWYWZ1VFcr/wALK8Gf9DFZf99H/Cj/AIWV4M/6GKy/76P+FFmFmdVRXK/8LK8Gf9DFZf8AfR/wo/4WV4M/6GKy/wC+j/hRZhZnVUVyv/CyvBn/AEMVl/30f8KP+FleDP8AoYrL/vo/4UWYWZ1VFcr/AMLK8Gf9DFZf99H/AAo/4WV4M/6GKy/76P8AhRZhZh8Sv+Sca7/17H+Yrc0X/kA6d/17R/8AoIrgPHnjzwtqXgXWLKy1u1muZrcrHGhOWORwOK77QmV/D+mupyrWsRB9RsFD2H0L9FFFIQUUUUAYWv8A/IX8Mf8AYTf/ANJLmtmsbX/+Qv4Y/wCwm/8A6SXNbNdlD4SZBWH4z/5EXxD/ANgy5/8ARTVuVh+M/wDkRfEP/YMuf/RTVs9hLc72iiiuU6QooooAKKKKACiiigAooooAKKKKACuY+I3/ACTbxJ/2Dpv/AEE109cx8Rv+SbeJP+wdN/6CaANOy/48bf8A65L/ACqeoLL/AI8bf/rkv8qnrzTpCuK8N/8AIzeMP+wmn/pPFXa1xXhz/kZvGA/6iaf+k8VaUt2Y1vhOlooorY5gooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACsPwz/AMgqf/sI33/pVLW5WH4Z/wCQVP8A9hG+/wDSqWujD/ExPY5z4waX/afw5v2Vd0loyXKf8BOG/wDHWauVm8U7P2dIphJ/pEkP9mgZ/wBooR/37BNet6lYx6npd3YS/wCruYXhb6MpB/nXyrpj3uqLpvgd1ZVOrl5OfusQsZ/IBz+NbT0Y46o7n4KS3OheM77RL1fLa9sY7hVPc7Q6/wDjkh/KrXg1Y/EXxV8X69Nua1tYpkV15IB+RSP+AI1SfFRpPB/j/QfFFlHtU27QlV6ZUFf/AEFwPwrQ+Dulm1+G2rai4/eXzykMe6Iu0f8Aj2+kt+Ub2uXfhje+G9H8Fa3qGjyanLY2sjz3H2tEEnyxhiFCnB4Hc9aszfGzwumnx3UUOozu+S0EcKl4lBxl/mwAe3Oa4/4af8ka8af9crn/ANJ66T4J2Vu3w5u2aFCbm5lWUkffG0DB9sZ49zTTeiQmlq2dx4b8WaT4p0U6rp0xECErKJhtaIgZIbsOCDnOK5Q/Grwr/aT2yJfyW6Nte9SDMS84yed2PfFef/D0zn4SeNxb7t/l9vTYd36ZruPg5/ZX/CsZPM+z7fNm+3b8Yx/t+2zHXtTUm7A4pXMb4DOr3Xip1IKtLAQR3GZa6HVvjV4W0zUJLSJb2/8AKOHltY1KDHXBZhn6jj3rgPhgZV8HfEE6fu8wWi+Tj72Ns2Me+K7H4GLph8EXWwQm6Nw4u843bcDbn/Zxn26+9KLdkkOSV22dbb+PdDvfCd14jspJrq0tVzPFEg82PpkFSR0Bz1xgcZpV8d6O3gg+LR5/9nAZK7R5md+zGM4zn3rzP4X2ltfePPGlnZKG0GeOWLan3CrSEIB7bS+PauOW6vU0eT4cNu+0trqpuxwV+4R9NwVqOd2Fyo99svHejX/gyfxTH562EAcurqBICpxjGcZPGOe4rR8Oa/a+J9Eg1ayhuIraYsEE6hWOCQTgE8ZBr501O6udC03Xvh9AsjPNrCeSD/HHzj8SVhP519I6Jpcei6HY6ZDjZawJECO5AwT+J5/GqjJsUkkcr8QvCuv+LX06wsNQjtNI35vlEjLJICR0AGCAM8E9T7V57o1pD4S+MGm6R4Q1WbULC4RftsfmiRVHzbtxUAZAAbPbOK9b1rxjomiaxZ6PqNy8N1fYEXyHbhjtBLdBz715T4o0+1+Gvj7QD4Tllhe9IS6svNLh13qBkHJ+bJ+hXIpSte447WPUdd8b6X4e1/TNGvIrprnUWVYWiRSgLNtG4kgjk+hrpazL/wDsP+0LX+0f7O+25H2b7Rs8zOeNm7nr6d606sg8s1/QNL8R/G2Kx1a1FzbDRBIELsvzCVgDlSD3NbPieb+3/G2keEVObNE/tLUVH8caHEaH2L4JH0qB/wDkvkf/AGAP/appui5/4Xj4l8z739nweVn+5hM/rUFGn4m8JXXinxDZLqNyv/CNW8JaWzSVkaebPG/AHygY754PrXPaRbWXh34s22j+F5W/s2ezeTUbRJjJHAwztbknaxO0Y9/eu4udY0i81mfwtcSFrua0aV4GVlDxH5ThvxPQ54PpXB6npNp4C8d+Gj4a32sWrXBt7yxWRmSRBj58EnBXcef/AK+W+4Lseq0UUVZB56vw+tNRu9T1bxzLFfTSzN9nAuXWG2g/hC/dweufp9TT/hXd3FxpWr2v2mW80q01CSDT7mY7jJCOgz3A459/bFbAXwx8R9JJkiF/aW1wyFJN8ZSVRg5HBzhv1rA+H7zaX4v8S+Fbe5kudI00xNamRtxg3jJjz6AkjH+yfeo2aL6F3wdOdE8U614Ocn7Pb4vtOB/hgc/Mg9lY4H1rua88v8/8L60vy+v9iv5uP7u98Z/HFeh04iY13WONndgqKCWJ6ACuJ+Hzvr76l4xuQS+ozNDZhv8AllaxsQqj0ywYn1NdD4q8z/hD9b8rPmfYJ9mPXy2xWP8ADmSGL4YaPLyIktSzbQSeC27pz1zR1DoeaafF4I1zUdc1fxXqoimudUmW1jFyykQrgKSF7dsn0r1zwjomiaJo23QJPNsbl/PEnnmUMSAMhs9PlFZvg/RPBVx4e+0eH9OtZtPuWfc0sRdnwSCG8zLY46H+tY/wujWz1rxjptkxOkWuoAWqg5VGO7eoPthalKw27npFct8R9R/sz4ea3cBtrNbmFSOuZCE/9mrqa5vxw3hz/hHxF4p3f2bNOicbx8/JXJTkDjvxVvYlbnnOjeF/hhcR2Vhc6wsmqtEiyhb5gGlwNwBztznPANeg+LtB8/wYYdOLR3mlxrcWEgOWSSIfLz3yAVP1ql458NeHU+H2qK+nWdvFa2jyW7xxKpjdV+TaR6nA98471qeBri6uvAWjT6gS0z2aF2fqwxwT9Rg1KXQpvqcz4kj0fx38Kj4jurNZJ4NNmntzvYeTLt+bGDzhk756V03gL/kQNA/68Yv/AEEVwHg/d/wz9rOc+X5F75ef7uG/rmu/8Bf8iBoH/XjF/wCgiiO9we1joqKKKsgKKKKACiiigAooooAKKKKACiiigAooooAx/E3/ACCoP+wjY/8ApVFW5WH4m/5BUH/YRsf/AEqircrkxHxItbBRRRXOMKKKKACiiigAooooA53x9/yT/wAQf9eEv/oJrq7L/jxt/wDrkv8AKuT8ff8AJPtf/wCvCX/0E11lnxY2/wD1zX+VZ1dkdFDqT0UUVgbnN+C/+Rg8Zf8AYVT/ANJoa7GuO8F/8jB4y/7Cqf8ApNDXY16MPhRzvcKKKKoQUUUUAFFFFABRRRQAUUUUAFFFFAHE+F/+Rn8Z/wDYUT/0mhrqa5bwv/yM/jP/ALCif+k0NdTXBV+Nm8dgrkviN/yLNv8A9hOy/wDSiOutrkviN/yLMB9NTsv/AEojpQ+JBLZm7RRRXQcQUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAZfhT/kDTf8AYRvv/SqWo/G+lf234I1nTwu55bVzGPV1G5f/AB4CpPCn/IGm/wCwjff+lUtbdZN2kd8fhR4L4O8V/YPgBrg8zE9k8lrEM8jzsbSPxdj+FYPwxiuvCPxD8PNdnbFrliSP91y2wf8AfSJ+dc5q9lfaf4m1bwVa8Q3WrRqi+uGdY/wIkB/AV6r8Z9K/sPS/C2t6em06NOkCY7KAGTPsDH/49W7tt3M137EHh9f+El/aM1fUD80OlK6r6BkUQ4/MsfwrY+FWq6Jc33iptI8PnTJLd0M7G9efzzmXHDAbeh6evtVT4CWUkum65r9wMzX93s3HvtG4kfUv+lZ3wU/4/vHP+8n85qmXVdrDXTzNXS/jTfa/YS/2P4Ou7vUImJeCGYuiR4GGZ9g5JyAuO3Wun+H3xEtvHNneFrNrG8siPPhZ9y4OcMDgf3TkEcVx/wCzwqjw/rL4G43SAn2Cf/XNZnw0tpbnxF8RbW24llSWOPHHzF5AKUox1XYab0OiX4xXuq6vdQ+GvCd5q9haH97cRyEEj1C7T1wcAnJ9BWB8IL+LVfit4r1GBXWK6WWZFkGGAaYEAj15qb4HeJNF0bw5qunapfW1heR3bTMLmQRkpsVeM9cFW47Zqj8LJ49X+I3jO4007I7uG4e3bGMB5cqfbqKbSSaSFe9mdVqXxgnn1u50zwp4butcNqSJp42IXg44CqcjPc4z2rX8K/E238VadqS2+mywa3YQvI+myyYLlewbHrgHK5BPSuF+CGv6P4etda0nWbu303UBcBm+1OItwUbSuTxlSDx7/WpPBk0fiH4/axrejgtpccbb5VGFfKKn/jzAsPXGaTitVbYak9NTttA+J1rrfgHVPFD2P2c6f5gktfP3ElVBUbto+9kDpSeEvida+JvCmr65NY/Yv7M3NLB5/mZUJuBztHXBGMdq8Y8VC98Ma94o8GWcZ8jV7yB4VHQLu3qB/wB9Bf8AgNL4vhn8A6t4g8K2KMbXVra12EdTtIJ/M+YPxqvZpi52j3vwH4tm8a+Hjq8mmfYIzM0caef5u8LjLZ2rjnI/CofH/gqXxxptrYDVnsbeKbzZUWLf53oDyMY59evtWr4T0VfDvhPTNJAAa2t1WTHdzy5/FiTWX448d23gaOwmvLC4uILuQxmWIgLERjr7kEkD/ZNZfa900+z7x5Fq2laH/wALO0DT/h3FKt7ZzYvXjLlEKOASS3oA27HByBzXrXjzxvceDF04waLJqf2t3VtkpTy9u3k4Vs53e3SvK/ivdeFzNod34Oksv7be4Mgk0vbuOehbZ/GWxjPPWvZ9W8V6T4YtbE+IL5LSW5G1f3bMGcAbsbQe5FXLWxC6m7XmvjC1t734xeCre6gingeG83RyoGVsREjIPHUCvSq878Tf8lq8D/8AXG8/9FNUQ3LlsP8AGrJqnibwz4JiVUs7hzd3kajCmCEZWPH91mXH4CtvxL4Og8UarpE1/cltO0+RpX08x5S4YjCljnt6YOeR3rn77MP7QGmPJ92fRHjiJ/vB3Ygfh/Oul1jxdZ6J4k0nRr6GSNdTDiK6YgRBh/AT6ngfiKeulhaa3OCuZ9D1f4o+G28ErbNcWpkbUp7JAsQt8AbWIGGPUD6ivXq8o+J1hpul3/hmbQ7eC18RNqMaW62qBHeLkMGC9VztHPqexNenyahZRX0VjJdwJdzKWjgaQCRwM5IXOSOD+VEtkEd2WK4ifwxoOi61rHivxNeWt0LplWN76Jdlqg4CLkkHPHOAfzNdvXM6X4k0vxPq+saFc2QW50yfY9vdKreYvaRR6f4j1qVcbscz8Lkim1/xVf6LE0Phi4uI/sS7SqNIFxIyKei5/oO3F2Gb/hG/i+9gny2PiK2NwE7LdRj5iPqgyfU4rM8MxWtl8a9atPD4RNK+wK99FB/qUudwxgDgNjPA/wBqrnjLM/xZ8BQQ/wCtjN1K+OybF/8AiWq3v8iVsejUE4GT0qvFqFlPeTWcN3BJcwYMsKSAvHnpuUHI/Gn3KNJazIn3mRgPriszRauxwvgaQ654k1vxDLzlhBBn+FOuPyC/maPHer3GozN4a0s5k8szXsgPCRqM7T/X6gd6j+FzOPDGpJCP9IW4YqD67Fx+orEtNL8baNbanN/ZUDG7RmuZ5ZY2fbg5wQ/uT0rku/Zpd9z672NP+0Kk3KK9nyqKk0uis/lv62Ol+FX/ACKk/wD1+P8A+gpXc15l8LZ9UjtZ1MMY0cM7tNkbhLheMZzjHtXfaVrWna3C82nXInjjbaxCsuDjPcCtaMlyJHk51QmsbVmtVfda2v0fZ+RU8X/8ihqv/Xs38q8w8B65N4e1OBLvKadqQwGPQMCVDfmMH2Oa9P8AF/8AyKGq/wDXs38q4vS/Di+IvhbbRRqPtkDyyW7e+45X6H+eKiqm6icd0juyupRjl84V/gnNRfldaP5NIsD/AJLa3/XH/wBo1o6t/wATT4m6RZfeisIGuXHox6fqE/OuN8D3lzffEC0luyTOsLRMWHPyxlRn3wK7Lwj/AMTHxb4j1g8r5wtYm9VXr/6CtTB8y9Wb46k8NPmlvCkl823H8rlf4najJ9hs9EtstPfSgso7qCMD8WI/Ks3wUZfDHjS+8OXMm5JhmNugZgNwP4qT+IFZtw2r+K/Hl3f6L5bGwYCF5CNqqpwDz6nLVD4ms/FWnXtp4h1byDLDIiJJFjqCWGQB9f5VMpPm9olt+R1UMLCOGjgJSinKLbV/e53ZrTyO3+JFmZfDQv4iVuLCZJo3XqOQDj8wfwrodE1EatollfgAGeJWYDs3cfnmsXxTqEF98O7u+jP7qe2Vlz23EYH1yab4NubfSvh/p0+oXMNtCFYmWeQIoDSMRyeO4roT/eadUfPVIP8As1Oe8ZtfJq7X3o6uuF+Jt/M9hpfhu1kaOfXrxbR3U8rBkGUj8CB9Ca7lWV0V0YMrDIIOQRXnXjnMPxN8AXMn+p8+4iyegdlUD8/6VvHc8eWxH8YpbXS/hvDpaMttbXNzBZjaOI41O7p6AJWdZan8LJtStrKbwyliZyFt573TfKjlPbDH19Tiuy8Z65ouiNpDa5piXNtPeLHHcSxIyWsnZyW+73OR6Gsb4yT6e3w6urecxyXNy8S2UY5d5N45QfTPTsfeqjskS+rPQqKyrG7j03StJtdVvYIr6WGOILLKqtLKFAIXJ+Y59PWtWsyzx3x1qeiSfF2xttctXvbOx0xmW0S3MzSzO33Qg/2cNz6V0ngi+8Cahqsw0HSYdN1e3U+ZBLaCCdVPU47jp0PpmtS017RH+It/o7adHa60luji7eNFa6jIBwrfeYDjg+h9K5zxAYbz44+GE0wq17aW8zag0fOyIqQofHuTwf7w9a03ViNncveIZv8AhGfiboWqxfLba2Dpt6o6NIOYW+uSVz6VS12xs7P43eDXtbWCBporx5TFGFLt5Tctjqee9TfFjM0nhC0i/wCPiXXoGTHXAzk/qKd4l/5LX4I/64Xn/opqFt8mDPRKKKKzNAooooAKKKKAMDwz/wAgu4/7CN9/6VS1sVj+Gf8AkF3H/YRvv/SqWtitjzpbsKKKKBBRRRQAUUUUAFc9o/8AyU/XP+wZaf8Aoc1dDXPaP/yU7Xf+wZZ/+hzVM/hZpS+I7GiiiuY6zzz43f8AJK9S/wCusH/oxaKPjd/ySvUv+usH/oxaK7MP8JjU3NPwZ/yIvh7/ALBlt/6KWtysPwZ/yIvh7/sGW3/opa3K9FbHG9wrG8Jf8gWf/sJ6h/6VzVs1jeEv+QLP/wBhPUP/AErmrDEfChxN2iiiuQoKKKKAOC1u2gufjDoMdxBHKh0y4ysiBhww9a6/+xdK/wCgZZf9+F/wrldU/wCSy6B/2DLj+YruKbGyj/Yulf8AQMsv+/C/4Uf2LpX/AEDLL/vwv+FQv4l0KN2R9b01WU4Km6QEH86vWt3bXsCz2lxFcQtkCSJw6nHXkcUaiK/9i6V/0DLL/vwv+FH9i6V/0DLL/vwv+FXqKQFH+xdK/wCgZZf9+F/wo/sXSv8AoGWX/fhf8KvUUAUf7F0r/oGWX/fhf8KP7F0r/oGWX/fhf8KvUUAUf7F0r/oGWX/fhf8ACj+xdK/6Bll/34X/AAq9RQBR/sXSv+gZZf8Afhf8KP7F0r/oGWX/AH4X/Cr1FAFH+xdK/wCgZZf9+F/wo/sXSv8AoGWX/fhf8KvUUAUf7F0r/oGWX/fhf8KP7F0r/oGWX/fhf8KvUUAUf7F0r/oGWX/fhf8ACj+xdK/6Bll/34X/AAq9RQBR/sXSv+gZZf8Afhf8K4rxrpenxeJ/BiR2NsiyakyuFhUBhsPB45r0OuH8c/8AI1eCf+wm3/oBprcaOq/sXSv+gZZf9+F/wo/sXSv+gZZf9+F/wq9VC61zSbGcwXeqWVvMACY5bhEYZ9iaQhf7F0r/AKBll/34X/Cj+xdK/wCgZZf9+F/wp9lqmn6lv+w31rdeXjf5Eyvtz0zg8dD+VW6AKP8AYulf9Ayy/wC/C/4Uf2LpX/QMsv8Avwv+FXqKAKP9i6V/0DLL/vwv+FH9i6V/0DLL/vwv+FXqKAKP9i6V/wBAyy/78L/hR/Yulf8AQMsv+/C/4VeooAo/2LpX/QMsv+/C/wCFH9i6V/0DLL/vwv8AhV6igCj/AGLpX/QMsv8Avwv+FH9i6V/0DLL/AL8L/hV6igCj/Yulf9Ayy/78L/hR/Yulf9Ayy/78L/hV6igCj/Yulf8AQMsv+/C/4Uf2LpX/AEDLL/vwv+FXqKAKP9i6V/0DLL/vwv8AhR/Yulf9Ayy/78L/AIVeooA4n4jaVp0Pw81uSKwtY5FtiVZIVBHI6HFdPogA0HTgBgC1i/8AQRWJ8Sv+Sca7/wBex/mK3NF/5AOnf9e0f/oIp9B9C9RRRSEFFFFAGFr/APyF/DH/AGE3/wDSS5rZrG1//kL+GP8AsJv/AOklzWzXZQ+EmQVh+M/+RF8Q/wDYMuf/AEU1blYfjP8A5EXxD/2DLn/0U1bPYS3O9ooorlOkKKKKACiiigAooooAKKKKACiiigArnPH8Zl+HXiRAMn+zLg/lGx/pXR1T1WzGo6PfWJxi5t5Ief8AaUj+tAFLSZfO0axl/v28bfmoq5XO+Abw33gDQp3zv+xRxvn+8g2t+qmuirzWrM6EFcVo37rx34whPG6a1nA9mgVc/nGa7WuMvR/Z/wAUYJDxFq2mGMe8sD7v/QZT/wB81pS3Iqq8To6KKK2OQKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigArD8M/wDIKn/7CN9/6VS1uVh+Gf8AkFT/APYRvv8A0qlrow/xMT2NiuMsfhloWn+M38UQyXhvWmkm8pnQxBnByQNuf4jjmuzorqaTJTsc94v8G6b4106Gy1J7iNIZfNR7dlVgcEYywIxz6dhVzSfD9no3huHQrUy/ZIomiDOQXIOckkDGcknpWrRRZbhd7HJaH8PdJ0Dwzqeg2lxeva6irrM8roXXemw7SFAHHqDWj4W8K2PhHRDpVhLcSQGRpN07KWy2M8gAdvStyiiyC7OY8I+BdK8G2V3aWEt1PFdsGkF0yt0GMcKOOa59vgp4X/tR7pJL+O2kbc9kkwETc5x03bfbP416PRS5UHMzmfCngbSvB0uovpslywv2VpEmZSqbd2AoCjA+Y9c9q53VPgp4Z1C/kurea+08Sk+ZDayKEOeuAVOPp09q9Ioo5VsHM9zG8N+F9K8J6Z9g0q38uMnc7sdzyN6se/8AKsp/hzoknjhfFpe6+3BxJ5QdfJLBNucbc579etddRTsguzkr34daJf8AjWHxVM119ujZH8tXXymZBhSRtzkYHfsK62iii1guc/4s8GaP4ysY7bVIn3REmKeJtskZPXBwRg8cEEVieGfhP4f8Naomph7q+vI+YnunBEZ9QABz7nOO1d3RRZXuF3axzOveBtM8ReIdL1q7nu0udNZWhWF1CMVfeNwKknkdiK6aiiiwXPLdd13TPD3xuivdWu1trc6GIxIykjcZWIHAPoa1/FEP9geOdG8XDizkT+zdQbsiOcxufYPgE/Su3kt4JW3SQxu2MZZQTSyxRzRtHLGskbDDK4yD9RSsO5g+JPB2n+JZba6llurPULXP2e9s5PLlQHqM9x7fX1NV9E8C2Gk6udYub2+1XVNpRLq/l3mJfRAAAv8A9c+tdSBgYHSinZbiuzlI9EnHxQl1lFuVtv7N8mRnceWzll2hB1yArZJ9RXV0UUJBc46++HdnNqt1qOl6tqujTXZ3XK2E4RJW/vFSDhvce/rWx4d8M6Z4V097bT0f945kmnmfdJK395m7n9K2aRlDKVYAgjBB70WQXZwvg2A654r1vxiw/wBHnIsdOP8AegjPzOPZmGR9DWh4R0SfS9Z8TXTLcpb31+ZIVuHBY9SzADohZjtHXAFdTHGkUaxxoqIowqqMAD2FOpJBca6LJG0bqGRgQwPQg1xHw8R9BfU/B1ySH06ZprQt/wAtbWQkqw9cNuB9DXc0wwxmYTGNPNC7Q+0bgPTPpTsFzi5fhnYpdXMmlazrOkW90xee1sbkJESepAIO0n2rd07w3pmheG5NI06B4rUxuG2HMjlhgtk9WP8Ah2raoosguzm/AWk3OieDbGxullSVTI+yZgzqrSMyhiONwUjOO+a1tX0ix13S59N1GAT2s64dDx7gg9iDzmr1FFtLBfW5wifC6wdYba/1vW9Q02BgY7C5usxcdAQACQO1avjjU20fwlPDZR5vbsCysYUGC0j/ACgAewyfwrpqY8Mcjo7xozxnKMVBKn29KVuwX7nnfiFtI8C/Cg+HLq8SK4m02aGAbSfOl2/PjA4yz9/Wul8Bf8iBoH/XjF/6CK35IYpsebEj46blBxTlVUUKoCqOAAMAUJahfQWiiiqEFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAY/ib/kFQf9hGx/8ASqKtysPxN/yCoP8AsI2P/pVFW5XJiPiRa2CiiiucYUUUUAFFFFABRRRQBy/xFJ/4V/q8a/emiEC+7OyoB+bV24AVQB0AwK4nxcPtt54e0ZeWvNTikdf+mUOZm/VFH4129ZVeiOmitLhRRRWJsc14EzJqfjCY9G1tkH/AYIRXZVx3w1zN4dvdSPTUdVvLpf8Ad81kX9EFdjXpRVkjne4UUUUxBRRRQAUUUUAFFFFABRRRQAUUUUAcR4czH408awntfQSD/gVtH/hXVVy1nmz+KuvW5+7fafa3a/VDJG38l/SuprhrL32bw2CuS+Jfy+BL2b/nhNbTk+gSeNifyBrrayvE2mHWvC2q6Yo+e6tJYk9mKkD9cVEXZpjeqJaKyfDGpDWPC2l6h/FPbIzj0bHzD8DkVrV0nCFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFAGX4U/5A03/YRvv/SqWtusTwp/yBpv+wjff+lUtbdZS3Z6EPhRgT+CfDlz4kXxDNpiNqquri48x87lACnbnbwAO3atDWdE07xBpkmnarbLc2khBaMsVyQcjkEEcir9FK7HZGfo2iad4e01NO0q1W2tEJZYwxbBJyeSSaq6N4R0Lw+96+l2C27XxBuCJHbzMZx94nH3m6etbVFF2FkZGgeF9G8LW81votkLWKZ98iiRmycYz8xNN0jwpomg395faZYrb3N6264cSO285J6EkDknpWzRTuwsjldS+G/hDV9WOp32iQyXbNudg7qHPqyqQCfqOe9aGl+E9C0XVbnU9O06O2vLldsrozYIyDgLnAHA6AVtUUXYWRzGu/Dzwp4kvPteqaPFLcn70qO0TN/vFCM/jWtoug6V4dsRZaRYxWlvnJWMcsfUk8k+5rRoou9gsjD1DwfoGq67a63e6ck2o2uzyZy7ArtbcvAODgnuKXVfCGga3q9pqupaclxe2m3yZWdht2tuHAODzzyDW3RRdhZBVPU9KsNasJLHUrSK6tZPvRyLkfX2PuKuUUgOV0T4b+EvD1+L7TdHjjulOUkkkeQp/u7icH3HNaOv+FNE8UC2Gs2K3QtmLRZkddpOM/dI9B1rZop3e4WWwV5p47mudM+JPhTW10rUr60s4rkTfYbZpWBZCo6cdT3Nel0UJ2YNXOG8aWVze2GieLtLtJzf6TIt2LZkxK8DgebHt/vbe3sR3rpNS0nSPFejpBqVnHd2cyrKqyqQRkcEdCpwfY1q0UXCxzeg+AvDPhq8N5pmlpHdEbRNJI8rqOmAXJxx6Umo+HDfePNG10QxIunwyhpi53uWUqEC9ABuZs9egrpaKLsLIK53X/A3hzxNcpc6rpqS3KDaJkdo3x6FlIJH1roqKSbWwWuZmh+HtJ8N2P2PR7GK0gJ3MEySx9WY5JP1Nc14esZ9a8ear4ruoZI7eBP7O01ZFKlkU5kkwexbIB9M13FFO4WOa0rw4bLxvreveTFCl7FDEoRyzSlQS0jZ+6eVUAf3c966Wiik3cEjidBsZvD3jrU7IQyfYNRX7RBIFJVWBJKk9B1b8hXZyxJPC8Mi7o5FKsPUHg0+ipjHlVjqxOJliJqo1aVkm+9tLmfYaNYaRYS2mn2ojhcljHvJ3MRjqSfQVm+D9CfRNPuRLAlu9zOZhbo5cQrgALu7kY610VFHKrp9hfWqrhODd+azd+tiG7tIL60ltbmPzIJVKumSMj8Kj07TbTSbJLOxhENuhJVAxOMnJ5PNWqKdle5j7SfJyX03t0uZcPhzSbfV21WKyVL5iSZQzckjBOM45+lSabothpNjJZ2UJjgkZncb2JJIwTknPatCikopdC5YitJWlJtadX02+7oZ2k6FpmhpKmm2qwCUgvhmbdjp1J9an1DTrTVbJ7O9hE1u+NyEkZwcjkc1aop2VrCdapKftHJ83e+v3nDeLtLKaNp3hjRraRI7q4GcbmWJAdxJJzjk5/A1p+KvDK6v4Gm8P2lvDJuiSGHzpCix4IAfI6lfvY74xXTUUox5ZcyNa2KnVpRpPo235t9WRW0AtrWG3UkrEioCe4AxXNeP/D9xr3hzdp+BqunzJe2J/wCmsZyB+IyPriuqoqk7O5ytX0MeAWHi7wxC1/p++2vIg0lrcx4KHupB5BBHX2yKzdI+HHhLQ9QS+sNIRbmP/VvLK8vl/wC6HYgfUc11VFO76BY5rxP4cOvar4fnWGL/AIl16Lpp3chkVcHaqjgliFGT0ANdLRRSuFjE8QeENB8UrF/bGnR3Dxf6uQMyOn0ZSDj2zinaB4U0PwvDJHo2nx23mnMjglnf6sxJP51s0U7u1gstzhzYz+JPicmoTQyLpnh+Jo7cupAmupB87LnqFXAz69Kxrq+uNd+M3hyaDRdXgttN+1wzXFzaMkRJjcAq3IIJHB4zkV6jRT5hcoUUUVJQUUUUAFFFFAGB4Z/5Bdx/2Eb7/wBKpa2Kx/DP/ILuP+wjff8ApVLWxWx50t2FFFFAgooooAKKKKACue8OfvfiD4rl6iOGygz6ELI5H5OtdDWB8Px9qs9Y1nqup6nNJE3rEmIU/SPP41M/hZrRXvHX0UUVzHUecfHJyPhldRqCTJcQrgf72f6UVo/EGyOt3Xh3w+Mf6beySNnptjgkPP4laK7aC9wxnuTeDOPA+gr3XToFP1Eag/yrcrG8NL9msLnTG4fTrua2weyBy0f5xtGfxrZr0Fscb3CsbwnxpNyvddSvs/jdSn+tbNYvh4+Tfa7ZHgxX5lUeqyIj5/76Lj8DWOIXujib1FFFcZQUUUUAcPqn/JZdA/7Blx/MV12p3q6bpN5fPjbbQPMc+iqT/SuR1T/ksugf9gy4/mK6XxFdWdl4d1C51C0ku7KOBmngjQMXjx8wwSAeM96b6DPmPwsngW4spZfFt5qS30lwdv2UfKqYHzNweSS3T0r6R8G6bo+leFrO30G5a500hpIZWcMWDEk84Hcn6Vw39tfD3UPhbqslhbWNlbmGYfZJVRZxLg7OMkkk4IIJ/Ti38C4rqP4es1wGET3kjW+7+5hQce24P+tVLVDep6ZRRRUEhRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAVw/jn/AJGrwT/2E2/9ANdxXD+Of+Rq8E/9hNv/AEA01uNHcV8za/eeGdW+KviK48UT3i2UbmGEWgy7OhVB2xjCtX0zXm3hjxF8P73W9Yig0y30q/jYi5OoRxxmXk78ZY9D16dRTi7Ai18LtK8I2dhe33hO9uLiO5KLOtw+WjK7sArgEfeP17V39eJfB1Ef4g+K59KXGibnEW0YTmUmPH/Ad2PavbaJbg9woooqRBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQByvxK/5Jxrv/Xsf5itzRf+QDp3/XtH/wCgisP4lf8AJONd/wCvY/zFbmi/8gHTv+vaP/0EU+g+heooopCCiiigDC17nWfDK9xqEjfh9lnH9RWzWLqB+0eMdIgHIt7e4uW9j8ka/mHf8jW1XbQXuEyCsTxlz4H15R1bT51H1MbAfzrbrH8Rr9psrbTV5fULuG22+qFg0n5RrIfwrV7CW53NFFFcp0hRRRQAUUUUAFFFFABRRRQAUUUUAFFFFAHDeCQdPn8QaA3H9nanI0K+kM375P8A0Nh+FdZXK64v9h/EXStW+7aaxCdMuT0AmXLwMfc/vE/EV1VcNaNpm8HdBXJfEC3ki0a3122RmudFuFvcKOWiGVmX8Y2Y/gK62kdFkRkdQysMFSMgj0rOLs7jaurGbDNHcwRzwurxSKHR1OQykZBFPrlfDBbQNSuvCFyTi1Bn012P+ttGPC59Yydp9tprqq6jias7BRRRQIKKKKACiiigAooooAKKKKACiiigAooooAKKKKACsLw4fLt9QtW/1sGo3O8enmSNKv8A47IprdrH1DT7uHUP7V0ry2uGQR3FtI21LhR05/hcZODjBBwexGtKajLUT1NSisU+JIYRi703V7eTug0+WfH/AAKJXU/nSf8ACV6f/wA+2s/+CW8/+NV2c8e5NmbdFYn/AAlen/8APtrP/glvP/jVH/CV6f8A8+2s/wDglvP/AI1Rzx7hZm3RWJ/wlen/APPtrP8A4Jbz/wCNUf8ACV6f/wA+2s/+CW8/+NUc8e4WZt0Vif8ACV6f/wA+2s/+CW8/+NUf8JXp/wDz7az/AOCW8/8AjVHPHuFmbdFYn/CV6f8A8+2s/wDglvP/AI1R/wAJXp//AD7az/4Jbz/41Rzx7hZm3RWJ/wAJXp//AD7az/4Jbz/41R/wlen/APPtrP8A4Jbz/wCNUc8e4WZt0Vif8JXp/wDz7az/AOCW8/8AjVH/AAlen/8APtrP/glvP/jVHPHuFmbdFYn/AAlen/8APtrP/glvP/jVH/CV6f8A8+2s/wDglvP/AI1Rzx7hZm3RWJ/wlen/APPtrP8A4Jbz/wCNUf8ACV6f/wA+2s/+CW8/+NUc8e4WZt0Vif8ACV6f/wA+2s/+CW8/+NUf8JXp/wDz7az/AOCW8/8AjVHPHuFmbdFYn/CV6f8A8+2s/wDglvP/AI1R/wAJXp//AD7az/4Jbz/41Rzx7hZm3RWJ/wAJXp//AD7az/4Jbz/41R/wlen/APPtrP8A4Jbz/wCNUc8e4WZt0Vif8JXp/wDz7az/AOCW8/8AjVH/AAlen/8APtrP/glvP/jVHPHuFmbdFYn/AAlen/8APtrP/glvP/jVH/CV6f8A8+2s/wDglvP/AI1Rzx7hZm3RWJ/wlen/APPtrP8A4Jbz/wCNUf8ACV6f/wA+2s/+CW8/+NUc8e4WZt0Vif8ACV6f/wA+2s/+CW8/+NUf8JXp/wDz7az/AOCW8/8AjVHPHuFmbdFYn/CV6f8A8+2s/wDglvP/AI1R/wAJXp//AD7az/4Jbz/41Rzx7hZm3RWJ/wAJXp//AD7az/4Jbz/41R/wlen/APPtrP8A4Jbz/wCNUc8e4WZt0Vif8JXp/wDz7az/AOCW8/8AjVH/AAlen/8APtrP/glvP/jVHPHuFmbdFYn/AAlen/8APtrP/glvP/jVH/CV6f8A8+2s/wDglvP/AI1Rzx7hZm3RWJ/wlen/APPtrP8A4Jbz/wCNUf8ACV6f/wA+2s/+CW8/+NUc8e4WZt0Vif8ACV6f/wA+2s/+CW8/+NUf8JXp/wDz7az/AOCW8/8AjVHPHuFmbdFYn/CV6f8A8+2s/wDglvP/AI1R/wAJXp//AD7az/4Jbz/41Rzx7hZm3RWJ/wAJXp//AD7az/4Jbz/41R/wlen/APPtrP8A4Jbz/wCNUc8e4WZt0Vif8JXp/wDz7az/AOCW8/8AjVH/AAlen/8APtrP/glvP/jVHPHuFmbdFYn/AAlen/8APtrP/glvP/jVKPEkMwxaabq9xJ2Q6fLBn/gUqoo/Ojnj3CzF8RnzLfT7Vf8AWz6jbbB6+XIsrf8AjsbGt2sbT9Pu5tQ/tXVfLW4VDHb20bbkt1PXn+JzgZOMADA7k7NcdWalLQpKwUUUVkMKKKKACiiigAoorG8T60+i6VutY/P1G5cW1jB/z0mbhR9ByxPoDQBU0Yf238QdR1L71ro8P2CA9QZ3w8xHuAI1/Ou0rJ8M6Inh3w/a6cJPNlQFp5j1llY7nc/ViTWtXPOV2dsI8qsFY/ivVTofhPVdSU/vILZ2i95MYQfixArYrkvEy/234o0Dw0nzRed/ad8PSGEjYpHo0pX/AL5NEI80khydkdH4V0j+wfCelaUR89raxxv7uFG4/icmteiivQOcKKKKACiiigAooooAKKKKACiiigAooooA4vxSDpvjnwvq4+WK4abS529fMUPH/wCPx4/4FXT1meNtGm13wlfWlodt8ii4tGHVZ4yHTH/AlA/Gl0DV4df0Cx1WDhLqFZCv9xv4lPuDkfhXLiI6pmtN9DSooormNDh/DA/sjWtc8Nt8qW9wbyzB4zbzEtgeyyeYv5V1Fc/42tprCSy8V2cTSTaXuW7jQfNLaNjzB7lcBx/un1rct54rq2iuIJFkhlQPG6nIZSMgj8K6ou6uclSNpElFFFMzCiiigAooooAKKKKACiiigAooooAKKKKAMnw24gOp6a/E1teyy4/vJM7Sqw9suy/VD6Vu1jajpbXM8V7Z3H2TUIQVSbbuVlPJR143LnnGQQeQRUQ1fWbZdt14eluHH8dhcxMre+JWjI+nP1NRKOt0ddOrG1mb1FYH/CRah/0KWtf9/bP/AOSKP+Ei1D/oUta/7+2f/wAkVPKzT2ke5v0Vgf8ACRah/wBClrX/AH9s/wD5Io/4SLUP+hS1r/v7Z/8AyRRysPaR7m/RWB/wkWof9ClrX/f2z/8Akij/AISLUP8AoUta/wC/tn/8kUcrD2ke5v0Vgf8ACRah/wBClrX/AH9s/wD5Io/4SLUP+hS1r/v7Z/8AyRRysPaR7m/RWB/wkWof9ClrX/f2z/8Akij/AISLUP8AoUta/wC/tn/8kUcrD2ke5v0Vgf8ACRah/wBClrX/AH9s/wD5Io/4SLUP+hS1r/v7Z/8AyRRysPaR7m/RWB/wkWof9ClrX/f2z/8Akij/AISLUP8AoUta/wC/tn/8kUcrD2ke5v0Vgf8ACRah/wBClrX/AH9s/wD5Io/4SLUP+hS1r/v7Z/8AyRRysPaR7m/RWB/wkWof9ClrX/f2z/8Akij/AISLUP8AoUta/wC/tn/8kUcrD2ke5v0Vgf8ACRah/wBClrX/AH9s/wD5Io/4SLUP+hS1r/v7Z/8AyRRysPaR7m/RWB/wkWof9ClrX/f2z/8Akij/AISLUP8AoUta/wC/tn/8kUcrD2ke5v0Vgf8ACRah/wBClrX/AH9s/wD5Io/4SLUP+hS1r/v7Z/8AyRRysPaR7m/RWB/wkWof9ClrX/f2z/8Akij/AISLUP8AoUta/wC/tn/8kUcrD2ke5v0Vgf8ACRah/wBClrX/AH9s/wD5Io/4SLUP+hS1r/v7Z/8AyRRysPaR7m/RWB/wkWof9ClrX/f2z/8Akij/AISLUP8AoUta/wC/tn/8kUcrD2ke5v0Vgf8ACRah/wBClrX/AH9s/wD5Io/4SLUP+hS1r/v7Z/8AyRRysPaR7m/RWB/wkWof9ClrX/f2z/8Akij/AISLUP8AoUta/wC/tn/8kUcrD2ke5v0Vgf8ACRah/wBClrX/AH9s/wD5Io/4SLUP+hS1r/v7Z/8AyRRysPaR7m/RWB/wkWof9ClrX/f2z/8Akij/AISLUP8AoUta/wC/tn/8kUcrD2ke5v0Vgf8ACRah/wBClrX/AH9s/wD5Io/4SLUP+hS1r/v7Z/8AyRRysPaR7m/RWB/wkWof9ClrX/f2z/8Akij/AISLUP8AoUta/wC/tn/8kUcrD2ke5v0Vgf8ACRah/wBClrX/AH9s/wD5Io/4SLUP+hS1r/v7Z/8AyRRysPaR7m/RWB/wkWof9ClrX/f2z/8Akij/AISLUP8AoUta/wC/tn/8kUcrD2ke5v0Vgf8ACRah/wBClrX/AH9s/wD5Io/4SLUP+hS1r/v7Z/8AyRRysPaR7m/RWB/wkWof9ClrX/f2z/8Akij/AISLUP8AoUta/wC/tn/8kUcrD2ke5v0Vgf8ACRah/wBClrX/AH9s/wD5Io/4SLUP+hS1r/v7Z/8AyRRysPaR7m/RWB/wkWof9ClrX/f2z/8Akij/AISLUP8AoUta/wC/tn/8kUcrD2ke5v02SRIYnlldUjRSzMxwAB1JrDHiDUGOP+EU1hfdpbTH6TmoprPUtdxHqiRWmnZBazjfzHn9pGwAF9VXOe7YyC1FkurFLcl8MK/9gwzOhQ3Us12FYYKiWVpQCOxw4rXoorQ4m7u4UUUUAFFFFABRRRQBheL9Um0rw3cPac39wVtbNR1aeQ7Ux9Cc/QGt7RNKh0PQrHSrfmK0gWIH+9gYJPuTz+Nctpq/8JT41a/xu0nQmeGA9prsjDsPUIpKj/aZvSu4rGq+h1Uo2VwooqO4uIrS2lubiRY4YUMkjt0VQMkn8KyNTmrYHVPixI/3oNF0wL/uzXD5/wDQIx/31RU/w6tZm0GfXLtCl3rly9+yt1SNsCJfwjVPzNFejCPLFI527u43Wov7G8WQ3/Sz1ZVt5j2S4Ufuyf8AfXK/VUHetCtXVdMttZ0u40+7UtBOm1tpwy9wwPYg4IPYgVyul3dzFcy6NqrD+07UZ3gbRdRdFmUe/Rh/C2R0wT0U5dDGpHqatYdwf7O8YWlyeIdSh+yOfSWPc8f5qZR+CityqGs6aNV0ua1EhilOHhmAyYpFIZHH0YA1U480bGaepp0Vm6Jqh1XT/MljEN3ExhuoM58qVfvD6dCD3BB71pV57ViwooooA4fVP+Sy6B/2DLj+Yrt2UMpVgCpGCD3riNU/5LLoH/YMuP5iu4psbOHn+EPgme/N22kbSW3GJJnWMn/dB4HsOK7O3t4bS2jtraJIYIlCRxouFUDoAO1S0UXYrhRRRSAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACuH8c/8AI1eCf+wm3/oBruK4fxz/AMjV4J/7Cbf+gGmtxo7iuS174aeFfEd+b+/03/Sm+/LDI0Zf/eAOCffrXW0Ur2EZ2i6Fpnh3TlsNJs47W2U7tqZJY+pJ5J9zWjRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFAHK/Er/knGu/9ex/mK3NF/wCQDp3/AF7R/wDoIrD+JX/JONd/69j/ADFbmi/8gHTv+vaP/wBBFPoPoXqKKKQgoorH8Q6hNa2kdpYkf2lfN5FrnnYcZaQj+6gyx9cAdxTSu7ICvox+361q+rdYzItlAfVIS24/9/GkH/ARW5VbT7GHTNOt7G3BEMEYjXJySAOpPcnqTVmvQjHlViG7hVDRYv7Y8Uzah1s9LDW0B7PcN/rGH+6uEz6s47VDqV1cz3UWi6UwGo3K7mlxkWsXQysP0Ud29gSOr0zTbbSNMt9PtEKwQIEXJyT6knuScknuSTUVJdDSnHqW6KKKxNQooooAKKKKACiiigAooooAKKKKACiiigDH8U6CniTw5eaYZPKlkUNBMOsUqncjj6MAaz/CmuNr2hRzXCeVqEDG2voO8U6cOv0zyPYiuorhPEkTeEPEJ8WQIx0u7CQ6zGgz5eOI7kD/AGfut/s4Pasq0OZaFwlZnXUU1HSWNZI2V0YBlZTkEHuDTq4TY5/xX4fk1qzhuLCVbfWLBzNY3B6Bsco3qjDgj8e1QeHtei12ydmia2vrdvKvLOT79vIOoPqD1B6Ec109cx4i8M3F1eprmhzJaa5CmzLj91dR9fLlA7ejDla1hO2jMqlPm1Rr0ViaH4lt9Xkks5oZLHVrcf6TYT8SR+47Oh7MODW3WxytWCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooqjq+s2Gh2LXmo3Cwwg7RnlnY9FUDlmPoOaAJr6+tdMsZr29nSC2gQvJI5wFArE8M6fda1qv/CV6tA8PymPSrOUYa3hPWRh2kf8ARcD1qPT9Fv8AxVfQat4ht3tdNgcSWOkP94sOks/q3cJ0Xvk129ZTn0R0U6dtWFFFFYm5FdXMFlaTXVzKsUEKGSSRjgKoGST+Fc/4DtJrxb/xXexslzrLq8EbjmG0XiFfYkEufdvaqOr/APFaeIf+EYgy2kWLpLrEo6SEfMlsD3J4ZvQDHeu/ACqAAABwAO1ddCFlzMynK+gtFFFdBmFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFcHpijwt42vdDf5dP1dnv9OJ6LL1niH44cD0ZvSu8rC8WeHv8AhI9FMEMv2fULeRbmxucf6mdeVP0PII7gmpnHmjYadnc0KKxPDHiAa/prtPCbXUbWQwX1ox5gmHUe6nqD3BFbdee007M6NxCAylWAIPBB71wVoD4H1pdGnJGgX0n/ABLJm6W0h5Nux7A9UJ917Cu+qpqemWes6bPp+oQLPazrtkjbuP6EdQexqoS5WTOKkrEVFcjHqF94LnTT/EM0lzpDEJaaw4zs7CO4I6HsH6Hvg11qsGUMpBUjIIPBFdG+qOOUXF2YtFFFAgooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAK5nxBqd3fXy+GdClK6lOoa5uVGRYwHq5/2z0UevPQUmo+ILrUb+TQ/C6pcagp23N4w3W9iO5Y/wAT+iD8cCug8PeHrXw7YNBA8k1xM5lubqY5kuJD1Zj/ACHQDiplLl9TWnTvqy1pOlWmiaVbabYReXbW6bEXOT7knuScknuTV2iiuc6grkPF5OvahY+Dbdj/AKcRcaiyn/V2aH5gT2LthB9Wrd17W7Tw9o8+pXhYpGAEjQZeVzwqKO7E4AFVvBmh3Vhb3Wr6uq/23qria6AORCoGI4VPog49ySa2ow5ndkTlZWOnRFjRURQqqMBQMACilorsMQrI1/QYtbtoysrW19bN5lrdoMtC/fj+JSOGU8Ee+CNeigDitP1OVrt9L1SFbXVol3NEDlJl/wCekRP3l/VTwffUrQ1nRLLXbRYLtGDRtvhnibbLC/Zkbsf0PQ5HFctNeX3hxxD4gKyWhOI9VjTbGfQTL/yzb/a+4fVfu1vGd9GYyhbYZqlpc2N//bemRGWYIEvLVeDcxjoV/wCmi849QSp7Ea9hf2up2Ud3ZzLLBIMqw/UEdQQeCDyDUisGUMpBBGQR3rGvNIuba8k1PRJI4bqQ5ntpOIbr3bHKvjjePxDYGIq0ubVbkpm9RWTp3iC1vbj7FOkljqIGTZ3OFc+pQ9HX3Un3weK1q5GmtyjkfEvgqfXfENnrFrr11pk9tbtAv2eNSSCck5PT/wCtVX/hBte/6H7Wf++E/wAK7iii7Hc4f/hBte/6H7Wf++E/wo/4QbXv+h+1n/vhP8K7iii7C5w//CDa9/0P2s/98J/hR/wg2vf9D9rP/fCf4V3FFF2Fzh/+EG17/oftZ/74T/Cj/hBte/6H7Wf++E/wruKKLsLnD/8ACDa9/wBD9rP/AHwn+FH/AAg2vf8AQ/az/wB8J/hXcUUXYXOH/wCEG17/AKH7Wf8AvhP8KP8AhBte/wCh+1n/AL4T/Cu4oouwucP/AMINr3/Q/az/AN8J/hR/wg2vf9D9rP8A3wn+FdxRRdhc4f8A4QbXv+h+1n/vhP8ACj/hBte/6H7Wf++E/wAK7iii7C5w/wDwg2vf9D9rP/fCf4Uf8INr3/Q/az/3wn+FdxRRdhc4f/hBte/6H7Wf++E/wo/4QbXv+h+1n/vhP8K7iii7C5w//CDa9/0P2s/98J/hVK8+GeqX9zZ3Fx431SSS0k8yFniQlGxjI9D+dei0UXYXZw//AAg2vf8AQ/az/wB8J/hR/wAINr3/AEP2s/8AfCf4V3FFF2Fzh/8AhBte/wCh+1n/AL4T/Cj/AIQbXv8AoftZ/wC+E/wruKKLsLnD/wDCDa9/0P2s/wDfCf4Uf8INr3/Q/az/AN8J/hXcUUXYXOH/AOEG17/oftZ/74T/AAo/4QbXv+h+1n/vhP8ACu4oouwucP8A8INr3/Q/az/3wn+FH/CDa9/0P2s/98J/hXcUUXYXOH/4QbXv+h+1n/vhP8KP+EG17/oftZ/74T/Cu4oouwucP/wg2vf9D9rP/fCf4Uf8INr3/Q/az/3wn+FdxRRdhc4f/hBte/6H7Wf++E/wo/4QbXv+h+1n/vhP8K7iii7C5w//AAg2vf8AQ/az/wB8J/hR/wAINr3/AEP2s/8AfCf4V3FFF2Fzh/8AhBte/wCh+1n/AL4T/Cj/AIQbXv8AoftZ/wC+E/wruKKLsLnnuofDjVtT0+ayuvHOrSwTLtdJIkKsPccV3Vha/YdOtbTzDJ5ESRb2GC20AZP5VYoouK4UUVj6h4ht7W5NjaRvf6ljP2W3IJT0MjdI19269gelCTeiAu6nqdrpNk93duVQEKqqMtIx6Ko6lieABWbpFhcvdS6xqiBb+ddiQg7haxdRGD3JPLEdTjsoosNIne9XVNYlS4v1BEUaD9zag9RGDySehc8n2HFbDusaM7sFVRksTgAV10qXLq9yWxazL7UpjeLpelRLc6pIu7Yx/d26H/lpKR0X0HVjwO5EUFzf+JG8vQj5Fh0fVZEypHpAp++f9s/IP9rpXU6Po1lodmbazjI3Nvlldt0kznq7seWY/wD1hgDFVKpbRFRhfVkWhaFDolrIPNa4vLhvMurqQYeZ/X2UDgKOAPzrVoorE2CiiigAooooAKKKKACiiigAooooAKKKKACiiigApksUc8LwyoskbqVdGGQwPBBHcU+igDzy3eX4e6jHpd7IzeGLl9un3bnP2Jz0gkP9z+4x6fdPau2qa9srbUbKazvII57aZCkkUi5VlPYiuEP9pfDseXMLjU/Cq/cnGZLjTl/uuOskQ7MOVHXIArmq0b+9E0jPoztaKhtLy21C0iu7OeOe3lXdHLGwZWHqCKmrlNTG17wxp3iFIjdLJFdQHNveW7+XPAfVWH8jkHuK59rrxR4a+TU7Jtd09el9YIBcKP8AppD/ABfVD/wGu5oq4zcSJQUtzmtI8S6NroI03UYJ5F+/DnbIn+8hww/EVq1BrHhbQtfKtqml29xIv3Ziu2Rfo4ww/A1jnwLLa/8AII8Ua5ZKPuxSTLcxr9BKrH9a1VSLMXRfQ36K57/hHvGMf+r8XWco9JtIGfxKyD+VN/sbx1/0MOi/+Cx//jtPnj3J9lI6Oiuc/sbx1/0MOi/+Cx//AI7R/Y3jr/oYdF/8Fj//AB2jnj3D2Ujo6K5z+xvHX/Qw6L/4LH/+O0f2N46/6GHRf/BY/wD8do549w9lI6Oiuc/sbx1/0MOi/wDgsf8A+O0f2N46/wChh0X/AMFj/wDx2jnj3D2Ujo6K5z+xvHX/AEMOi/8Agsf/AOO0f2N46/6GHRf/AAWP/wDHaOePcPZSOjornP7G8df9DDov/gsf/wCO0f2N46/6GHRf/BY//wAdo549w9lI6Oiuc/sbx1/0MOi/+Cx//jtH9jeOv+hh0X/wWP8A/HaOePcPZSOjornP7G8df9DDov8A4LH/APjtH9jeOv8AoYdF/wDBY/8A8do549w9lI6Oiuc/sbx1/wBDDov/AILH/wDjtH9jeOv+hh0X/wAFj/8Ax2jnj3D2Ujo6K5z+xvHX/Qw6L/4LH/8AjtH9jeOv+hh0X/wWP/8AHaOePcPZSOjornP7G8df9DDov/gsf/47R/Y3jr/oYdF/8Fj/APx2jnj3D2Ujo6K5z+xvHX/Qw6L/AOCx/wD47R/Y3jr/AKGHRf8AwWP/APHaOePcPZSOjornP7G8df8AQw6L/wCCx/8A47R/Y3jr/oYdF/8ABY//AMdo549w9lI6Oiuc/sbx1/0MOi/+Cx//AI7R/Y3jr/oYdF/8Fj//AB2jnj3D2Ujo6K5z+xvHX/Qw6L/4LH/+O0f2N46/6GHRf/BY/wD8do549w9lI6Oiuc/sbx1/0MOi/wDgsf8A+O0f2N46/wChh0X/AMFj/wDx2jnj3D2Ujo6K5z+xvHX/AEMOi/8Agsf/AOO0f2N46/6GHRf/AAWP/wDHaOePcPZSOjornP7G8df9DDov/gsf/wCO0f2N46/6GHRf/BY//wAdo549w9lI6Oiuc/sbx1/0MOi/+Cx//jtH9jeOv+hh0X/wWP8A/HaOePcPZSOjornP7G8df9DDov8A4LH/APjtH9jeOv8AoYdF/wDBY/8A8do549w9lI6Oiuc/sbx1/wBDDov/AILH/wDjtH9jeOv+hh0X/wAFj/8Ax2jnj3D2Ujo6K5z+xvHX/Qw6L/4LH/8AjtH9jeOv+hh0X/wWP/8AHaOePcPZSOjornP7G8df9DDov/gsf/47R/Y3jr/oYdF/8Fj/APx2jnj3D2Ujo6K5z+xvHX/Qw6L/AOCx/wD47R/Y3jr/AKGHRf8AwWP/APHaOePcPZSOjornP7G8df8AQw6L/wCCx/8A47R/Y3jr/oYdF/8ABY//AMdo549w9lI6Oiuc/sbx1/0MOi/+Cx//AI7R/Y3jr/oYdF/8Fj//AB2jnj3D2Ujo6K5z+xvHX/Qw6L/4LH/+O0f2N46/6GHRf/BY/wD8do549w9lI6Oiuc/sbx1/0MOi/wDgsf8A+O0f2N46/wChh0X/AMFj/wDx2jnj3D2Ujo6K5z+xvHX/AEMOi/8Agsf/AOO0f2N46/6GHRf/AAWP/wDHaOePcPZSOjornP7G8df9DDov/gsf/wCO0f2N46/6GHRf/BY//wAdo549w9lI6Oiuc/sbx1/0MOi/+Cx//jtH9jeOv+hh0X/wWP8A/HaOePcPZSOjornP7G8df9DDov8A4LH/APjtH9jeOv8AoYdF/wDBY/8A8do549w9lI6Oiuc/sbx1/wBDDov/AILH/wDjtH9jeOv+hh0X/wAFj/8Ax2jnj3D2Ujo6K5z+xvHX/Qw6L/4LH/8AjtH9jeOv+hh0X/wWP/8AHaOePcPZSOjornP7G8df9DDov/gsf/47ThoXjWQfP4p0yL/rnpJb+ctHPHuHspHQ1V1DU7DSrY3OoXkFpCP455Ag/WskeDdWuhjUvGeqyL/cs4obYfmFLfrV3TvAnhvTbkXa6ctzeD/l5vXa4lz6hpCcH6YpOpFFKi+pkL4n1HXf3fhTSXuYzx/aV8GgtV91yN8n/ARj3rU0bwdDaX6atq93Jq+sKDtuJ1ASDPURRjhB78k+tdNRWcqjZrGmohRRRWZoFcv4h1y9mv18N+HCj61Ou6acjclhEf8Alq/+1/dXufaoLvxHfeIL2bRvCASR4zsu9Xdd1vaeoX/nrJ/sjgcZPaul8O+G7Hwzp5trMPJJIxkuLmZt0txIeru3c/y7V0UqN9ZGcp9ESeH9Cs/DmkRadZBiqkvJLIcvNI3LSOe7E8k1p0UV1mQUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFAHH+KNDv7XUV8UeHYw+pxIEu7POFv4R/CfSRedrfgeDWlomt2PiDTI7+wl3xNlWVhh43HVHX+FgeCK3q4/XPC95a6rJ4i8LtHDqbgfa7ORtsF+o7N/dkHZ/wORWNWlzarcuMrHR0VieH/E9lr4mgVJbTUbbi6sLkbZoD7jup7MODW3XG007M2GTQxXMDwTxpLFIpV0dQysD1BB6iuMfwtq3htjL4UuEmss5bR72Q+WP+uMnJj/3Tlee1dtRTjJx2JlFS3OKtvG2mi5Wy1hJtE1A8fZ9QAjDH/Yk+44+h/CukVgyhlIIIyCO9W7uztb+2a3vLaG4gf70UyB1P1B4rl3+HWjQsW0i41LRmJyRp92yJ/wB+23J/47WqqLqYuj2N6iuf/wCEX8UQnFt40Zk7C80yKQ/mhSmnRfHC8L4j0dh6tpbg/pLVc8e5HspHRUVzn9jeOv8AoYdF/wDBY/8A8do/sbx1/wBDDov/AILH/wDjtHPHuHspHR0Vzn9jeOv+hh0X/wAFj/8Ax2j+xvHX/Qw6L/4LH/8AjtHPHuHspHR0Vzn9jeOv+hh0X/wWP/8AHaP7G8df9DDov/gsf/47Rzx7h7KR0dFc5/Y3jr/oYdF/8Fj/APx2j+xvHX/Qw6L/AOCx/wD47Rzx7h7KR0dFc5/Y3jr/AKGHRf8AwWP/APHaP7G8df8AQw6L/wCCx/8A47Rzx7h7KR0dFc5/Y3jr/oYdF/8ABY//AMdo/sbx1/0MOi/+Cx//AI7Rzx7h7KR0dFc5/Y3jr/oYdF/8Fj//AB2j+xvHX/Qw6L/4LH/+O0c8e4eykdHRXOf2N46/6GHRf/BY/wD8do/sbx1/0MOi/wDgsf8A+O0c8e4eykdHRXOf2N46/wChh0X/AMFj/wDx2j+xvHX/AEMOi/8Agsf/AOO0c8e4eykdHRXOf2N46/6GHRf/AAWP/wDHaP7G8df9DDov/gsf/wCO0c8e4eykdHRXOf2N46/6GHRf/BY//wAdo/sbx1/0MOi/+Cx//jtHPHuHspHR0Vzn9jeOv+hh0X/wWP8A/HaP7G8df9DDov8A4LH/APjtHPHuHspHR0Vzn9jeOv8AoYdF/wDBY/8A8do/sbx1/wBDDov/AILH/wDjtHPHuHspHR0Vzn9jeOv+hh0X/wAFj/8Ax2j+xvHX/Qw6L/4LH/8AjtHPHuHspHR0Vzn9jeOv+hh0X/wWP/8AHaP7G8df9DDov/gsf/47Rzx7h7KR0dFc5/Y3jr/oYdF/8Fj/APx2j+xvHX/Qw6L/AOCx/wD47Rzx7h7KR0dFc5/Y3jr/AKGHRf8AwWP/APHaP7G8df8AQw6L/wCCx/8A47Rzx7h7KR0dFc5/Y3jr/oYdF/8ABY//AMdo/sbx1/0MOi/+Cx//AI7Rzx7h7KR0dFc5/Y3jr/oYdF/8Fj//AB2j+xvHX/Qw6L/4LH/+O0c8e4eykdHRXOf2N46/6GHRf/BY/wD8do/sbx1/0MOi/wDgsf8A+O0c8e4eykdHRXOf2N46/wChh0X/AMFj/wDx2j+xvHX/AEMOi/8Agsf/AOO0c8e4eykdHRXOf2N46/6GHRf/AAWP/wDHaP7G8df9DDov/gsf/wCO0c8e4eykdHRXOf2N46/6GHRf/BY//wAdo/sbx1/0MOi/+Cx//jtHPHuHspHR0Vzn9jeOv+hh0X/wWP8A/HaP7G8df9DDov8A4LH/APjtHPHuHspHR0Vzn9jeOv8AoYdF/wDBY/8A8do/sbx1/wBDDov/AILH/wDjtHPHuHspHR0Vzn9jeOv+hh0X/wAFj/8Ax2j+xvHX/Qw6L/4LH/8AjtHPHuHspHR0Vzn9jeOv+hh0X/wWP/8AHaP7G8df9DDov/gsf/47Rzx7h7KR0dFc5/Y3jr/oYdF/8Fj/APx2j+xvHX/Qw6L/AOCx/wD47Rzx7h7KR0dFc5/Y3jr/AKGHRf8AwWP/APHaP7G8df8AQw6L/wCCx/8A47Rzx7h7KR0dFc5/Y3jr/oYdF/8ABY//AMdo/sbx1/0MOi/+Cx//AI7Rzx7h7KR0dFc5/Y3jr/oYdF/8Fj//AB2j+xvHX/Qw6L/4LH/+O0c8e4eykdHRXOf2N46/6GHRf/BY/wD8do/sbx1/0MOi/wDgsf8A+O0c8e4eykdHRXOf2N46/wChh0X/AMFj/wDx2j+xvHX/AEMOi/8Agsf/AOO0c8e4eykdHRXOf2N46/6GHRf/AAWP/wDHaUaN45JwfEWjAeo0xyf/AEbRzx7h7KR0VFc9/wAI14sl4n8ZpGvf7LpUaH83Z/5U4fDzT7nnWdU1jVwesd1eFI/++I9q/mDRzxGqMhNQ8Z6LY3Js4p31DUOgsrBDPMT6ELwv/AiBUC6R4l8Tn/ibSnQtLPWztJQ11MPR5Rwg9kyf9qur03SdO0e1Ftptjb2cA/ggjCA+5x1NXKzdXsaxpJblTTNLsdG0+Kw061jtrWIYSOMYA9/c+55NW6KKzNQqtf39ppdhNfX1xHb2sCl5JZDgKKqa74h07w7ZC4v5SDI2yGCNd8s79kRByxNZGneHdR8TahBrPiqEQW0DCSx0bduWJu0kx6PJ6L0X3NaU6bn6EykkJoGnXnirV4fE+swPBY253aRp8owVyMfaJR/fI+6P4QfU13NFFdqSSsjFu4UUUUxBRRRQAUjKroyOoZWGCCMgilooA5K48J3Gls03hmeOGPOW0y4J+zt/1zIyYj9Mr/s96rW2uQvdrYX8Eum6i3S2ugB5n/XNh8sg/wB0k+oFdtVa/wBOstVs3tL+1hubd+scqBhnsee/vVxm0RKCZzt/p1lqlv8AZ761iuIs5CyLnB7Eeh9xzWaNH1Ow/wCQVrUvljpb6gn2lB9Gysn5s30rQm8ManpnzaFqPnQD/lx1J2dQPRJuXX/gW/8ACqZ8QRWciw63az6RMx2qbrHkuf8AZlGUP0JDe1XeE9zNxkho1DxHBxPolrcAfxWl7yf+Auqgf99H60v9vakOG8KasT/sTWhH6zCtgEMoZSCCMgjvS0vYQFzGN/b+of8AQp6z/wB/bP8A+P0f2/qH/Qp6z/39s/8A4/WzRS9hEOYxv7f1D/oU9Z/7+2f/AMfo/t/UP+hT1n/v7Z//AB+tmij2EQ5jG/t/UP8AoU9Z/wC/tn/8fo/t/UP+hT1n/v7Z/wDx+tmij2EQ5jG/t/UP+hT1n/v7Z/8Ax+j+39Q/6FPWf+/tn/8AH62aKPYRDmMb+39Q/wChT1n/AL+2f/x+j+39Q/6FPWf+/tn/APH62aKPYRDmMb+39Q/6FPWf+/tn/wDH6P7f1D/oU9Z/7+2f/wAfrZoo9hEOYxv7f1D/AKFPWf8Av7Z//H6P7f1D/oU9Z/7+2f8A8frZoo9hEOYxv7f1D/oU9Z/7+2f/AMfo/t/UP+hT1n/v7Z//AB+tmij2EQ5jG/t/UP8AoU9Z/wC/tn/8fo/t/UP+hT1n/v7Z/wDx+tmij2EQ5jG/t/UP+hT1n/v7Z/8Ax+j+39Q/6FPWf+/tn/8AH62aKPYRDmMb+39Q/wChT1n/AL+2f/x+j+39Q/6FPWf+/tn/APH62aKPYRDmMb+39Q/6FPWf+/tn/wDH6P7f1D/oU9Z/7+2f/wAfrZoo9hEOYxv7f1D/AKFPWf8Av7Z//H6P7f1D/oU9Z/7+2f8A8frZoo9hEOYxv7f1D/oU9Z/7+2f/AMfo/t/UP+hT1n/v7Z//AB+tmij2EQ5jG/t/UP8AoU9Z/wC/tn/8fo/t/UP+hT1n/v7Z/wDx+tmij2EQ5jG/t/UP+hT1n/v7Z/8Ax+j+39Q/6FPWf+/tn/8AH62aKPYRDmMb+39Q/wChT1n/AL+2f/x+j+39Q/6FPWf+/tn/APH62aKPYRDmMb+39Q/6FPWf+/tn/wDH6P7f1D/oU9Z/7+2f/wAfrZoo9hEOYxv7f1D/AKFPWf8Av7Z//H6P7f1D/oU9Z/7+2f8A8frZoo9hEOYxv7f1D/oU9Z/7+2f/AMfo/t/UP+hT1n/v7Z//AB+tmij2EQ5jG/t7UjwvhTVgf9ua0A/SY0h1DxHPxBotpbA/xXd7kj/gKKwP/fQ+tbVNd1jRndgqqMlmOABT9hAOYxTo2pX/APyFtalMZ62+nobZD9WyZD+DD6Vp2On2emWwt7G2it4Qc7I1CgnuT6n3qgmvDUHMehWc2rODgyw4W3U+8zfKfou4+1XYfCt/qXz6/qJER/5cNPZo4/o8nDv+GwHupp3hDYajJlWfXI2u3sdMt5dTv0OHhtsbYj/00kPyp9Cc+gNW7bwlNqDrceJZ47oAhl0+HItkP+1nmU/72F/2RXR2VjaabaR2ljbQ21vGMJFCgVV+gFWKiU2zWMEhAAoAAAA4AHalooqCgooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooA4y98ETafeSal4QvU0u5kbfNYyKWs7k+rIOUY/3kx7g1BF43TT51s/FenzaDck7VmmO+0lP+xOPlH0baa7qmTQxXELwzxJLE4wyOoZWHoQetZzpRluUpNFKKaOeJZYZEkjcZV0YEEexFPrnJvhxpEMrz6Fc32gTuct/Zs22Jj7xMDGfwUVF/Znj7Thtt9W0XWE/6fbZ7aT/AL6jLKf++RWDw8lsWqiOoorlf7Y8aQDE/glJiP4rTVYiD+DhDR/wk/iBfv8AgPWQf9me2Yf+jKz9lPsVzo6qiuU/4SnXf+hE1v8A7+2//wAco/4SnXf+hE1v/v7b/wDxyj2U+wcyOrorlP8AhKdd/wChE1v/AL+2/wD8co/4SnXf+hE1v/v7b/8Axyj2U+wcyOrorlP+Ep13/oRNb/7+2/8A8co/4SnXf+hE1v8A7+2//wAco9lPsHMjq6K5T/hKdd/6ETW/+/tv/wDHKP8AhKdd/wChE1v/AL+2/wD8co9lPsHMjq6K5T/hKdd/6ETW/wDv7b//AByj/hKdd/6ETW/+/tv/APHKPZT7BzI6uiuU/wCEp13/AKETW/8Av7b/APxyj/hKdd/6ETW/+/tv/wDHKPZT7BzI6uiuU/4SnXf+hE1v/v7b/wDxyj/hKdd/6ETW/wDv7b//AByj2U+wcyOrorlP+Ep13/oRNb/7+2//AMco/wCEp13/AKETW/8Av7b/APxyj2U+wcyOrorlP+Ep13/oRNb/AO/tv/8AHKP+Ep13/oRNb/7+2/8A8co9lPsHMjq6K5T/AISnXf8AoRNb/wC/tv8A/HKP+Ep13/oRNb/7+2//AMco9lPsHMjq6K5T/hKdd/6ETW/+/tv/APHKP+Ep13/oRNb/AO/tv/8AHKPZT7BzI6uiuU/4SnXf+hE1v/v7b/8Axyj/AISnXf8AoRNb/wC/tv8A/HKPZT7BzI6uiuU/4SnXf+hE1v8A7+2//wAco/4SnXf+hE1v/v7b/wDxyj2U+wcyOrorlP8AhKdd/wChE1v/AL+2/wD8co/4SnXf+hE1v/v7b/8Axyj2U+wcyOrorlP+Ep13/oRNb/7+2/8A8co/4SnXf+hE1v8A7+2//wAco9lPsHMjq6K5T/hKdd/6ETW/+/tv/wDHKP8AhKdd/wChE1v/AL+2/wD8co9lPsHMjq6K5T/hKdd/6ETW/wDv7b//AByj/hKdd/6ETW/+/tv/APHKPZT7BzI6uiuU/wCEp13/AKETW/8Av7b/APxyj/hKdd/6ETW/+/tv/wDHKPZT7BzI6uiuU/4SnXf+hE1v/v7b/wDxyj/hKdd/6ETW/wDv7b//AByj2U+wcyOrorlP+Ep13/oRNb/7+2//AMco/wCEp13/AKETW/8Av7b/APxyj2U+wcyOrorlP+Ep13/oRNb/AO/tv/8AHKP+Ep13/oRNb/7+2/8A8co9lPsHMjq6K5T/AISnXf8AoRNb/wC/tv8A/HKP+Ep13/oRNb/7+2//AMco9lPsHMjq6K5T/hKdd/6ETW/+/tv/APHKP+Ep13/oRNb/AO/tv/8AHKPZT7BzI6uiuU/4SnXf+hE1v/v7b/8Axyj/AISnXf8AoRNb/wC/tv8A/HKPZT7BzI6uiuU/4SnXf+hE1v8A7+2//wAco/4SnXf+hE1v/v7b/wDxyj2U+wcyOrorlP8AhKdd/wChE1v/AL+2/wD8co/4SnXf+hE1v/v7b/8Axyj2U+wcyOrorlP+Ep13/oRNb/7+2/8A8co/4SnXf+hE1v8A7+2//wAco9lPsHMjq6K5T/hKdd/6ETW/+/tv/wDHKP8AhKdd/wChE1v/AL+2/wD8co9lPsHMjq6K5T/hKdd/6ETW/wDv7b//AByj/hKdd/6ETW/+/tv/APHKPZT7BzI6uiuU/wCEp13/AKETW/8Av7b/APxyj/hKdd/6ETW/+/tv/wDHKPZT7BzI6uiuU/4SnXf+hE1v/v7b/wDxyj/hKdd/6ETW/wDv7b//AByj2U+wcyOrorlP+Ep13/oRNb/7+2//AMco/wCEp13/AKETW/8Av7b/APxyj2U+wcyOrorlP+Ep13/oRNb/AO/tv/8AHKP+Ep13/oRNb/7+2/8A8co9lPsHMjq6K5T/AISnXf8AoRNb/wC/tv8A/HKP+Ep13/oRNb/7+2//AMco9lPsHMjq6K5T/hKdd/6ETW/+/tv/APHKP+Ep13/oRNb/AO/tv/8AHKPZT7BzI6uiuVHinXTwPAmtZ7Zmth/7UoGveLpf9T4EuF9DcalboP8Ax0sf0o9lPsHMjqqK5YRfEK/GNnh/SEbuXlu5F/DCLn8acPh+2oj/AIqXxFqmrqfvW6uLW3b6pFgn8WNWqEnuJzQ/U/G+kWN0bCzMuran2sdOXzpB/vY+VB7sRVYeHNf8VkP4nuP7N0s/8wiwmJeUek0wxkf7KYHua63S9H03RLQWml2NvZwD/lnBGEB9zjqfertbwoxiZubZXsrG102yis7G3it7aJdscUShVUewFWKKK1JCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigDC8Q+E9N8R+VNOJbbUIP+Pe/tX8ueH6MOo/2TkH0rAbU/E/hYbNcsH1rT14Gp6bF+9UessHX6lMj2Fd5RUyhGW402tjA0jxBpGv2/n6VqFvdoPvCN/mT2Zeqn2IFaVZms+CfD+u3Au7vT1jvhyt7bMYJ1Pr5iEE/jmss+FvFOmnOkeLmuYl+7b6xarNn/tomxvzzXPLDvozRVO509Fct9s8e2ZxceHNJ1Af37LUjGf8AvmRP60n/AAkniaMnzvAWpjHeK8tn/wDagrN0Z9iudHVUVyn/AAlOu/8AQia3/wB/bf8A+OUf8JTrv/Qia3/39t//AI5S9lPsHMjq6K5T/hKdd/6ETW/+/tv/APHKP+Ep13/oRNb/AO/tv/8AHKPZT7BzI6uiuU/4SnXf+hE1v/v7b/8Axyj/AISnXf8AoRNb/wC/tv8A/HKPZT7BzI6uiuU/4SnXf+hE1v8A7+2//wAco/4SnXf+hE1v/v7b/wDxyj2U+wcyOrorlP8AhKdd/wChE1v/AL+2/wD8co/4SnXf+hE1v/v7b/8Axyj2U+wcyOrorlP+Ep13/oRNb/7+2/8A8co/4SnXf+hE1v8A7+2//wAco9lPsHMjq6K5T/hKdd/6ETW/+/tv/wDHKP8AhKdd/wChE1v/AL+2/wD8co9lPsHMjq6K5T/hKdd/6ETW/wDv7b//AByj/hKdd/6ETW/+/tv/APHKPZT7BzI6uiuU/wCEp13/AKETW/8Av7b/APxyj/hKdd/6ETW/+/tv/wDHKPZT7BzI6uiuU/4SnXf+hE1v/v7b/wDxyj/hKdd/6ETW/wDv7b//AByj2U+wcyOrorlP+Ep13/oRNb/7+2//AMco/wCEp13/AKETW/8Av7b/APxyj2U+wcyOrorlP+Ep13/oRNb/AO/tv/8AHKP+Ep13/oRNb/7+2/8A8co9lPsHMjq6K5T/AISnXf8AoRNb/wC/tv8A/HKP+Ep13/oRNb/7+2//AMco9lPsHMjq6K5T/hKdd/6ETW/+/tv/APHKP+Ep13/oRNb/AO/tv/8AHKPZT7BzI6uiuU/4SnXf+hE1v/v7b/8Axyj/AISnXf8AoRNb/wC/tv8A/HKPZT7BzI6uiuU/4SnXf+hE1v8A7+2//wAco/4SnXf+hE1v/v7b/wDxyj2U+wcyOrorlP8AhKdd/wChE1v/AL+2/wD8co/4SnXf+hE1v/v7b/8Axyj2U+wcyOrorlP+Ep13/oRNb/7+2/8A8co/4SnXf+hE1v8A7+2//wAco9lPsHMjq6K5T/hKdd/6ETW/+/tv/wDHKP8AhKdd/wChE1v/AL+2/wD8co9lPsHMjq6K5T/hKdd/6ETW/wDv7b//AByj/hKdd/6ETW/+/tv/APHKPZT7BzI6uiuU/wCEp13/AKETW/8Av7b/APxyj/hKdd/6ETW/+/tv/wDHKPZT7BzI6uiuU/4SnXf+hE1v/v7b/wDxyj/hKdd/6ETW/wDv7b//AByj2U+wcyOrorlP+Ep13/oRNb/7+2//AMco/wCEp13/AKETW/8Av7b/APxyj2U+wcyOrorlP+Ep13/oRNb/AO/tv/8AHKP+Ep13/oRNb/7+2/8A8co9lPsHMjq6K5T/AISnXf8AoRNb/wC/tv8A/HKP+Ep13/oRNb/7+2//AMco9lPsHMjq6K5T/hKdd/6ETW/+/tv/APHKP+Ep13/oRNb/AO/tv/8AHKPZT7BzI6uiuU/4SnXf+hE1v/v7b/8Axyj/AISnXf8AoRNb/wC/tv8A/HKPZT7BzI6uiuU/4SnXf+hE1v8A7+2//wAco/4SnXf+hE1v/v7b/wDxyj2U+wcyOrorlP8AhKdd/wChE1v/AL+2/wD8co/4SnXf+hE1v/v7b/8Axyj2U+wcyOrorlP+Ep13/oRNb/7+2/8A8co/4SnXf+hE1v8A7+2//wAco9lPsHMjq6K5T/hKdd/6ETW/+/tv/wDHKP8AhKdd/wChE1v/AL+2/wD8co9lPsHMjq6K5T/hKdd/6ETW/wDv7b//AByj/hKdd/6ETW/+/tv/APHKPZT7BzI6uiuU/wCEp13/AKETW/8Av7b/APxyj/hKdd/6ETW/+/tv/wDHKPZT7BzI6uiuU/4SnXf+hE1v/v7b/wDxyj/hKdd/6ETW/wDv7b//AByj2U+wcyOrorlP+Ep13/oRNb/7+2//AMco/wCEp13/AKETW/8Av7b/APxyj2U+wcyOrorlP+Ep13/oRNb/AO/tv/8AHKX/AISXxG/+p8BaqT/00u7ZP/alHsp9g5kdVRXLf2h46ujttvCun2Wf+Wl9qgbH/AY0b+dL/wAI74y1I51LxTb6fEeGh0izG7H/AF1lLH8lFUqE2JzRualqun6Pam61K9t7SAf8tJ5Ag/XqfaucXxJrXiTMfhTS2jtW4/tfU0aKHHrHH9+T2+6PetXTPAHh7TbwXz2r6hqA/wCX3UZTcS59QWyF/wCAgV09bRoJbkOb6HN6B4Ms9Hu21O7uJtU1pxh9Qu8FwO6xqOI19l/EmukoorcgKKKKACiiigAooooAKKKKACiiigApskaTRtHKiujDDKwyCPQinUUAczN4JsYmMmjXFxo8hOdloQYCfeFgUH/AQp96pyReJtM/4+LC31WEf8tbBvKl/GKQ4/JyfauyoqlJolxTOJg8R6XLcLayztZ3bdLa9jaCQ/RXA3fUZFatbV3Z2t/btb3ltDcQN96OZA6n6g8VgSeB9Ni50q5vtJbstnN+6H/bJw0Y/BRVqr3IdPsS0VRfSfFNl/qrjTNUjH8MqvayY/3hvUn8FFV31i6tP+QnoWq2gHWSOH7Sn1zCWIH1Aq1OLIcGjWorOstf0jUZPKtNStZZQcGISjep9CvUfiK0aokKKKKYBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUVmXXiLRrKbyJ9SthP2gRw8h+iLlj+VIDTorKTVL+8/5Bvh/U5weklxGLVB9fNIf8lNWE0bxPe83F9p+mRnqltG1zJ+DvtUf98Gpc4opQbLvQZNZEniXTfPa3tJJNQuVOGhsI2nZT6MVyE/4ERWnH4H0l8NqbXWrv3+3zF4z/ANshiP8A8drobe3gtYFgt4Y4YkGFjjUKqj2AqHV7Fqn3ORjtfE+pf6u1tdHgP8d23nzf9+0IUfXefpV6DwTprOs2rST6xMDkG+YNGp9ogBGPrtJ966Siocm9y1FIRVVFCqoVQMAAYAFLRRUlBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQBT1DSNN1aPy9R0+0vE/u3EKyD9Qax38D6QnNi99px7C0vJFQf8AbMkp/wCO10lFAHKN4Z1u3/48/EnnAdtQskkz+MRj/kaia28V2339O0y8UfxQXjxuf+AsmP8Ax6uwoqlOSJ5EcS2q38H/AB+eG9Xhx1aOOOcfh5Tsf0qNvFejxf8AH1cS2X/X7bSW+P8Av4q13VFV7Rk+zRyFrrek32Psep2Vxnp5Nwr5/I1erSvND0jUc/btLsrrPXz7dHz+YrNPgXwx/wAstHgtv+vUtBj/AL4Ip+18hezCioz4I05f+Pe91iD6anM//obNTD4PuE/1HijWU/2WFvIP/Hos/rT9qhezZPRVY+GdaX/VeJi3/XexRv8A0ErTf7B8UL93XNJcej6XID+Yn/pT9pEXs2W6KpnSvFa9LnRZP+2Uqf8AsxppsPFo6Q6I3/bxKv8A7IaftIhySL1FUPsni/8A6B+hn/uIy/8Axij7P4v/AOgToZ9/7WmH/ttRzxDkkX6KofZvF54/srQx7/2rKf0+zCj7J4vPH2DQx7/2hKf08mjniHJIv0VRGn+LT1j0RP8AtvK3/sgpw0nxW3W80WP/ALYSv/7OtHtIhySLlFVBoHidvv69pSD0j0qTP5mf+lOHhjWG/wBb4nkX/rjZRr/6Fupe0QezZZoqEeDpW/1/ibWpfYGCMf8AjkQP608eCNLb/X3Wrz/7+qTqD9QrgH8qXtUP2bH9Bk1nXXiDRbHP2vV7CAjtLcov8zWivgXwv/y00S0uP+vpPP8A/Q81q2ek6bp4AstPtLYDp5MKp/IUva+Q/ZnIr4o0uX/j1e6vfT7HZzTj80UipV1LU5/+PTwzqsgPR5vKgUfXe4b/AMdNdrRS9ox+zRx62fiy46Wmk2SnvJcyTt+Kqij/AMeqZfC+rz/8fviaVB3XT7SOEfnJ5h/WuqoqXOT6lKKObTwNojc3sdzqJ7/brqSZD/wAnZ/47W3ZadY6bD5NhZW9rF/cgiVF/ICrNFSUFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFAH//2Q==)" + ], + "metadata": { + "id": "VgyZHICtuRMz" + } + }, + { + "cell_type": "code", + "source": [ + "import logging\n", + "\n", + "import apache_beam as beam\n", + "from apache_beam.coders import BooleanCoder\n", + "from apache_beam.coders import PickleCoder\n", + "from apache_beam.coders import TimestampCoder\n", + "from apache_beam.transforms.timeutil import TimeDomain\n", + "from apache_beam.transforms.userstate import OrderedListStateSpec\n", + "from apache_beam.transforms.userstate import ReadModifyWriteStateSpec\n", + "from apache_beam.transforms.userstate import TimerSpec\n", + "from apache_beam.transforms.userstate import on_timer\n", + "from apache_beam.utils.timestamp import MAX_TIMESTAMP\n", + "from apache_beam.utils.timestamp import Timestamp\n", + "\n", + "_LOGGER = logging.getLogger(__name__)\n", + "logging.basicConfig(level=logging.INFO)\n", + "_LOGGER.setLevel(logging.INFO)\n", + "\n", + "\n", + "class OrderedSlidingWindowFn(beam.DoFn):\n", + "\n", + " ORDERED_BUFFER_STATE = OrderedListStateSpec('ordered_buffer', PickleCoder())\n", + " WINDOW_TIMER = TimerSpec('window_timer', TimeDomain.WATERMARK)\n", + " TIMER_STATE = ReadModifyWriteStateSpec('timer_state', BooleanCoder())\n", + " EARLIEST_TS_STATE = ReadModifyWriteStateSpec('earliest_ts', TimestampCoder())\n", + "\n", + " def __init__(self, window_size, slide_interval):\n", + " self.window_size = window_size\n", + " self.slide_interval = slide_interval\n", + "\n", + " def start_bundle(self):\n", + " _LOGGER.debug(\"start bundle\")\n", + "\n", + " def finish_bundle(self):\n", + " _LOGGER.debug(\"finish bundle\")\n", + "\n", + " def process(\n", + " self,\n", + " element,\n", + " timestamp=beam.DoFn.TimestampParam,\n", + " ordered_buffer=beam.DoFn.StateParam(ORDERED_BUFFER_STATE),\n", + " window_timer=beam.DoFn.TimerParam(WINDOW_TIMER),\n", + " timer_state=beam.DoFn.StateParam(TIMER_STATE),\n", + " earliest_ts_state=beam.DoFn.StateParam(EARLIEST_TS_STATE)):\n", + "\n", + " _, value = element\n", + " ordered_buffer.add((timestamp, value))\n", + "\n", + " _LOGGER.debug(\"receive %s at %s\", element, timestamp)\n", + " timer_started = timer_state.read()\n", + "\n", + " earliest = earliest_ts_state.read()\n", + " if not earliest or earliest > timestamp:\n", + " earliest_ts_state.write(timestamp)\n", + "\n", + " if not timer_started:\n", + " earliest_ts_state.write(timestamp)\n", + "\n", + " first_slide_start = int(\n", + " timestamp.micros / 1e6 // self.slide_interval) * self.slide_interval\n", + " first_slide_start_ts = Timestamp.of(first_slide_start)\n", + "\n", + " first_window_end_ts = first_slide_start_ts + self.window_size\n", + " _LOGGER.debug(\"set timer to %s\", first_window_end_ts)\n", + " window_timer.set(first_window_end_ts)\n", + "\n", + " timer_state.write(True)\n", + "\n", + " return []\n", + "\n", + " @on_timer(WINDOW_TIMER)\n", + " def on_timer(\n", + " self,\n", + " key=beam.DoFn.KeyParam,\n", + " fire_ts=beam.DoFn.TimestampParam,\n", + " ordered_buffer=beam.DoFn.StateParam(ORDERED_BUFFER_STATE),\n", + " window_timer=beam.DoFn.TimerParam(WINDOW_TIMER),\n", + " timer_state=beam.DoFn.StateParam(TIMER_STATE),\n", + " earliest_ts_state=beam.DoFn.StateParam(EARLIEST_TS_STATE)):\n", + " _LOGGER.debug(\"timer fire at %s\", fire_ts)\n", + " window_end_ts = fire_ts\n", + " window_start_ts = window_end_ts - self.window_size\n", + "\n", + " window_values = list(\n", + " ordered_buffer.read_range(window_start_ts, window_end_ts))\n", + "\n", + " _LOGGER.debug(\n", + " \"window start: %s, window end: %s\", window_start_ts, window_end_ts)\n", + " _LOGGER.debug(\"windowed data in buffer %s\", str(window_values))\n", + " if window_values:\n", + " yield (key, (window_start_ts, window_end_ts, window_values))\n", + "\n", + " next_window_end_ts = fire_ts + self.slide_interval\n", + " next_window_start_ts = window_start_ts + self.slide_interval\n", + "\n", + " earliest_ts = earliest_ts_state.read()\n", + " ordered_buffer.clear_range(earliest_ts, next_window_start_ts)\n", + "\n", + " remaining_data = list(\n", + " ordered_buffer.read_range(next_window_start_ts, MAX_TIMESTAMP))\n", + "\n", + " if not remaining_data:\n", + " timer_state.clear()\n", + " earliest_ts_state.write(next_window_start_ts)\n", + " return\n", + "\n", + " _LOGGER.debug(\"set timer to %s\", next_window_end_ts)\n", + " window_timer.set(next_window_end_ts)\n", + "\n", + "\n", + "class FillGapsFn(beam.DoFn):\n", + " def __init__(self, expected_interval: float):\n", + " \"\"\"\n", + " Args:\n", + " expected_interval: The expected time delta between elements, in seconds.\n", + " \"\"\"\n", + " self.expected_interval = expected_interval\n", + "\n", + " def process(self, element):\n", + " key, (window_start_ts, window_end_ts, window_elements) = element\n", + "\n", + " received_data = {\n", + " round(float(ts.micros / 1e6), 5): val\n", + " for ts, val in window_elements\n", + " }\n", + "\n", + " start_sec = float(window_start_ts.micros / 1e6)\n", + " end_sec = float(window_end_ts.micros / 1e6)\n", + "\n", + " filled_values = []\n", + " current_ts_sec = start_sec\n", + "\n", + " while current_ts_sec < end_sec:\n", + " lookup_ts = round(current_ts_sec, 5)\n", + "\n", + " if lookup_ts in received_data:\n", + " filled_values.append(float(received_data[lookup_ts]))\n", + " else:\n", + " filled_values.append('NaN')\n", + "\n", + " current_ts_sec += self.expected_interval\n", + "\n", + " yield (key, (window_start_ts, window_end_ts, filled_values))\n" + ], + "metadata": { + "id": "E1fHKPrkuLFW" + }, + "execution_count": 2, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "# Model Handler![detection.jpg](data:image/jpeg;base64,/9j/4AAQSkZJRgABAgAAAQABAAD/2wBDAAgGBgcGBQgHBwcJCQgKDBQNDAsLDBkSEw8UHRofHh0aHBwgJC4nICIsIxwcKDcpLDAxNDQ0Hyc5PTgyPC4zNDL/2wBDAQkJCQwLDBgNDRgyIRwhMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjL/wAARCAJlBaYDASIAAhEBAxEB/8QAHwAAAQUBAQEBAQEAAAAAAAAAAAECAwQFBgcICQoL/8QAtRAAAgEDAwIEAwUFBAQAAAF9AQIDAAQRBRIhMUEGE1FhByJxFDKBkaEII0KxwRVS0fAkM2JyggkKFhcYGRolJicoKSo0NTY3ODk6Q0RFRkdISUpTVFVWV1hZWmNkZWZnaGlqc3R1dnd4eXqDhIWGh4iJipKTlJWWl5iZmqKjpKWmp6ipqrKztLW2t7i5usLDxMXGx8jJytLT1NXW19jZ2uHi4+Tl5ufo6erx8vP09fb3+Pn6/8QAHwEAAwEBAQEBAQEBAQAAAAAAAAECAwQFBgcICQoL/8QAtREAAgECBAQDBAcFBAQAAQJ3AAECAxEEBSExBhJBUQdhcRMiMoEIFEKRobHBCSMzUvAVYnLRChYkNOEl8RcYGRomJygpKjU2Nzg5OkNERUZHSElKU1RVVldYWVpjZGVmZ2hpanN0dXZ3eHl6goOEhYaHiImKkpOUlZaXmJmaoqOkpaanqKmqsrO0tba3uLm6wsPExcbHyMnK0tPU1dbX2Nna4uPk5ebn6Onq8vP09fb3+Pn6/9oADAMBAAIRAxEAPwD3+iiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAoooJwMnpQAU2SRIo2kkdURRlmY4AHqTXPy+JJdQdoPD1ul5glWvpSVtkPfDDmQ+y8diwNRL4dju5Fn1u5k1WYHcEmG2BD/ALMI+XjsW3MP71ZTqxiUotk7+LbOclNItrrV26b7RB5P/f1iEP8AwEk+1RmXxRe9ZNN0uM/worXUmP8AeOxVP/AWrWAAAAAAHQClrCVeT2NFBGMdAln/AOP7XdYufZbgW4/8ghD+tN/4RHQ2/wBdZfaT63Uzzk/UuxrborJzk92VZGGfBfhVlCt4a0dgP71jEf8A2Wm/8IP4S/6FbRP/AAXxf/E1vUUuZhZGD/wg/hL/AKFbRP8AwXxf/E0f8IP4S/6FbRP/AAXxf/E1vUUcz7hZGD/wg/hL/oVtE/8ABfF/8TR/wg/hL/oVtE/8F8X/AMTW9RRzPuFkYP8Awg/hL/oVtE/8F8X/AMTR/wAIP4S/6FbRP/BfF/8AE1vUUcz7hZGD/wAIP4S/6FbRP/BfF/8AE0f8IP4S/wChW0T/AMF8X/xNb1FHM+4WRg/8IP4S/wChW0T/AMF8X/xNH/CD+Ev+hW0T/wAF8X/xNb1FHM+4WRg/8IP4S/6FbRP/AAXxf/E0f8IP4S/6FbRP/BfF/wDE1vUUcz7hZGD/AMIP4S/6FbRP/BfF/wDE0f8ACD+Ev+hW0T/wXxf/ABNb1FHM+4WRg/8ACD+Ev+hW0T/wXxf/ABNH/CD+Ev8AoVtE/wDBfF/8TW9RRzPuFkYP/CD+Ev8AoVtE/wDBfF/8TR/wg/hL/oVtE/8ABfF/8TW9RRzPuFkYP/CD+Ev+hW0T/wAF8X/xNH/CD+Ev+hW0T/wXxf8AxNb1FHM+4WRg/wDCD+Ev+hW0T/wXxf8AxNH/AAg/hL/oVtE/8F8X/wATW9RRzPuFkYP/AAg/hL/oVtE/8F8X/wATR/wg/hL/AKFbRP8AwXxf/E1vUUcz7hZGD/wg/hL/AKFbRP8AwXxf/E0f8IP4S/6FbRP/AAXxf/E1vUUcz7hZGD/wg/hL/oVtE/8ABfF/8TR/wg/hL/oVtE/8F8X/AMTW9RRzPuFkYP8Awg/hL/oVtE/8F8X/AMTR/wAIP4S/6FbRP/BfF/8AE1vUUcz7hZGD/wAIP4S/6FbRP/BfF/8AE0f8IP4S/wChW0T/AMF8X/xNb1FHM+4WRg/8IP4S/wChW0T/AMF8X/xNH/CD+Ev+hW0T/wAF8X/xNb1FHM+4WRg/8IP4S/6FbRP/AAXxf/E0f8IP4S/6FbRP/BfF/wDE1vUUcz7hZGD/AMIP4S/6FbRP/BfF/wDE0f8ACD+Ev+hW0T/wXxf/ABNb1FHM+4WRg/8ACD+Ev+hW0T/wXxf/ABNH/CD+Ev8AoVtE/wDBfF/8TW9RRzPuFkYP/CD+Ev8AoVtE/wDBfF/8TR/wg/hL/oVtE/8ABfF/8TW9RRzPuFkYP/CD+Ev+hW0T/wAF8X/xNH/CD+Ev+hW0T/wXxf8AxNb1FHM+4WRg/wDCD+Ev+hW0T/wXxf8AxNH/AAg/hL/oVtE/8F8X/wATW9RRzPuFkYP/AAg/hL/oVtE/8F8X/wATR/wg/hL/AKFbRP8AwXxf/E1vUUcz7hZGGvgvwqqkL4a0dQf7tjEP/ZaX/hEtDT/U2Rtj62s0kBH4owrbop8z7hZGMNBmg/48de1e29muBcD8fODn9acJfFFl0fTdUjHZle1k/wC+hvVj+CiteiqVWa6icUZieLrKAhNXt7nSHPG68UeUf+2qkoPoSD7VvI6SxrJGyujDKspyCPUGqZAYEEAg8EGsY+HY7R2m0O5fSpidxSIbrdz/ALUJ+XnuV2sfWto4j+Yh0+x09Fc7F4klsJFg8Q262ZJ2pexMWtZD2yx5jJ9H47Bmros5GRXQmmrohqwUUUUxBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFZ+r6vBo9oJpVeWWRhHBbxjMk0h6Ko9eCc9AAScAE0ASanqlppFmbm8k2JuCIqqWeRj0VVHLMewFYLWN74iPma2pt9PPKaWrZ3j1nYfeP8AsD5R33dptP0yeS8GrauyTakQRGiHMdop/gjz39X6t7DAGvXJUrX0iaxh1Y1EWNFRFCoowqqMAD0FOoornNAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKK5v4gEj4eeISCQRp83I/3DTSu7AdJRXzd8OPhTaeNfCh1e41m+tZxcPEFiwVG3GDzz3rc8GaprngD4ojwPq+pSahpt1gW8khJ2krlGXJJXJBUrnGefro6a1SeqIUu57rRXH+LfiZ4b8G3SWeozzS3jgN9mtY97qD0JyQB9M5qbw/8AEPw74m0i81HTrl2FlGZbiB02yooBOdvfp1BIqOWVr2KutjqqK8E8G/Fs33xMv21W9u5NPvZPsulwLGAsYaUBMqDwcYyeTXo/iv4peGfB9+LC+mnnvcBmt7WMOyA9N2SAPpnNU6ck7CUk9TtKK5/wp400TxnZSXOj3RkMRAlhkXbJGT0yPf1GRWD4j+MXhPw1qkmnXE1zdXMTbZVtIg4jb0JJAz7DOKlRk3aw7o76iuIuPif4fm8C33iTT7p5IrceWU8v545W4UMp7ZI56da5D4MfER9bmu9K1q9urrWbu5e4jZl+RYwi8DsoyDwBin7OVm+wuZXsezUUV82/Fu91Dxj48vtO0otJbaBZPJLtY4G3DSt9QSq/8BohDmdhydj6Sorjfhb4l/4SjwDp91I+66t1+y3GTzvTAyfcrtb8aZqfxV8LaPreo6TqF1NBcWCb5S0WVb7uFXHLE7hxj19KXK72C6tc7WiuJ8KfFTw14w1RtNsJLmG8wWSK6iCGQDrtIJHA5x1q34u+Inh3wU0cWq3LtdSLuW2gTfIV9SOAB9SM0csr2sF1udXRXHeEvib4a8ZXLWmnXMsV4AWFtcpsdgOpXkg/QHNM1f4peF9B1++0bUrmaC5s4xJITFlTlVYBcckkMOMevpRySvawXR2lFeeab8a/BeoQ3cj3k9kbZN5S6i2tIM4+QAnJyRx19uDV3wr8VfDXi/VW0ywkuYbzBZI7mIJ5gHJ24J7c4ODim4SXQOZHbUVxXiv4qeGfCGpf2dfS3E96AGeC1jDsgPI3EkAEjnGc1b8H/EPw/wCN/OTSZpRcQqGkt502OF6Z6kEZ44P86XLK17BdbHVUV4l8HXdviR42DMxAnfAJ6fvnrZ+DUfh+FNeOh6jf3Y82Mzm8iEew/PjbgnPf8qqULXEpXPVKK83vvjj4MsdSezE93cqjbWuLeDdED9SQSPcA+1di3ijRF8NHxEdRhOkiPzPtIyRjOOnXOeMYznjGalxkt0O6NeivMovjv4Pe4RHXU4YXbaLmS2/d/Xhif0r0m3uIbu2iuLeRZYZUDxyIchlIyCD6YocWtwTTJKKKKkYUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUANdEkjaORVdGBDKwyCPQisRbK+8OnzNFU3Gnjl9LZsbB6wMfu/wC4fl9NvfdoqozcXdCaT3F0zVLTV7MXVnLvTJVlZSrRsOqsp5Vh3B5q5XN6hpk8d4dW0dki1EACSNziK7UfwSY6H+6/VfcZB1dI1aDWLQzRK8UsbGOeCQYkhkHVWHryDnoQQQSCDXbTqKaMZRsX6KKK0JCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigCrqOoW2lWE17dvshiGWIGST0AA6kkkAAckkCsTS7K5uLtta1VNt9Ku2GDORaRHnYPVjwWbucDoBUav/wkWtm7POl6dKyWy9p7gZDSe4TlV99x7Ka3K5a1S/uo1hHqFFFFcxoFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAVzfxB/5J34i/7B83/oBrpKoa3pUWuaHfaVPI8cV5A8LumNyhhgkZ7007MTPPfgIQPhsSTgC9lyfwWuT1q4i8T/ALSGkrpkizx2HlCWSM5UeXuduR6Z2/Xitsfs6+HcjOr6rj6x/wDxNdz4P+HugeCI5TpUEjXEo2yXM7bpGX0yAAB7ACtnKKbknuTZ2seNaNJ4jf45eJ5tItdMudVjlnCDUmYBYhIFBTBBzt2j6E10nhDQNaj+MN3q2ozeH4ZZ4njv7CwugWGUHWM5PJCsc+pNdf4s+FekeJ9ZXWory80vVQAGubN9pfAwCR644yCOPWrXgz4b6R4MuLi9gmub3UrkES3l0wZyCckDHTJwT1J9aHUTQKLueefC22gb4y+NkaCMrFPcGMFBhMXBxj0rC8ISeJ2+K/iqfRrTS7nVlnm3jUmYFF80g7MEew+mK9LvPg9pVx4zfxJb6pqFpLLcC5lghZdrvuDHnGcEjJBzU/ij4T6T4h1w63bX19pOpt/rJ7N9u/jGSOxxxkEe9Pnjf1Fys5TwNoeuWXxO1nVprjQY557eVbmw067DGOT5SMx9R8wyc/3j61X/AGeY7WaLxDNOqPqRlQSNIMvsIbPXnBbOfwr0TwX8O9H8E/aJrR57q+ueJru5YF2Gc4GOgzz6nuTgVha/8FdE1fWZ9Vsb++0m4uCTOtqw2OT1IHUZ7849qTmndMdmtSTTIPAI0/xVa+FktzdLDN9ujQSEbl3Y+9xgNnG3j04rE/Z3hiPg7UJjEhlXUGAcqNwHlx8Zru/CXgLRPB2jz6fp8byC5/4+ZpyC8vBGDgAAAE4A9TWH4T+Eem+EPEY1Wx1bUHjXfstZGXZ8ylfmwOcA8dKXMrNXCzumdd4m1uHw34a1HWJsFbWFnCn+Juir+LED8a+cvh7rHijR11PVrTwbea5/a5IkuvKkKsNzbwCqkHLE5+lfQHjPwlB400MaTdXtza2/mrK5t9uXxnAOQeMnP1ArS0PR7bw/odlpNmD9ntIhEpbq2OpPuTkn60oyUY2G02zwf4J6tdeHPHN/4Y1O1nsft6b47e4Uq0cijcowcdUJ574FaWk2dveftPaobiJJPJQyoGGQGESAH6jNeh+Ivhxp3iDxbYeJTe3dnqFns2mDbh9jbhuyD64+lTWXgCwsviBd+MEu7lry5Qo0LbfLAKqOOM/wjvVOom2+6FyvY861+GO3/ac0V4kVGliR3KjG47JBk/gBUPhuO2vv2kddGtqkk8YlNmkwB+ZdgTAPfy8kfnXpd/4AsNQ8fWXi+S7uVvLRAiQrt8sgBhzxn+I96q+MvhdofjK8TUJnuLHUkAAurVgGYDpuBHOPXg+9CmtvKwcrPPvifDbWfxk8JyaMiR6pJLEZ1hGC37wBS2PUbgc9vani0gu/2qbkXESyLGiyKrDIDC1TB/A8/Wu58IfCjRPCmqHVjPdalqZyFubtgdmRglQO+OMnJq/H4AsI/iLL40F3cm9lTYYDt8sDyxH6Z6DPWjnS08g5Weaapp9rc/tR2UM0EbxuqzMpUYLrbswJ98qD+FWPEEaRftOaI0ahWkhRnIGNx8uQZP4ACvQ5fAFhL8RYfGhu7kXsSbBANvlkeWY/TPRs9aL/AMAWF/4+s/F8l3creWqBFhXb5ZADDnjP8R70udfgHKcRN4judU+J2q2HgfwzpA1e1Dx3eq32RnDBW4XB+9gZ5Jx6VjfDGO+h+O3iCPUZraW9FrN9oe1XbGz7484HHeu41r4QaZqXiafXrDV9T0i6uSTcfYpQu8n7xB6jPfqM9qt+G/hXo3hTxR/bml3d6rmEwvBI4dXBAySSM5JAbr19uKfPG1l2CzucV8G/+Sk+N/8Aru//AKOeua+HMlxF8OPiO9qWEogXleoG2TcfyzXs/hbwBYeFNe1bVrW7uZptTcvKku3ahLFuMAdz3pvgz4eaZ4Lh1KG1uLi7j1AqZVuQpGBuGMADg7j1odRa/IFF6Hkvw8i8VyfDa7h0nTfDUukTmYXct87LJ0+bfg4GBgj0GDWnoPhBP+FN6roms+JNIgtpb0S2t5HepJCj4U7S2QOSp4685robr4FaM1xcDT9a1bT7G5OZrOGUFGHpz2+ua66P4f8Ah2PwafCoss6YRlgW+cvnO/d/ez3/AA6cU5VF0Eos8aupviB4U8KQJq2n6R4i8KQJGY2dUmh8vjYQRhscjBIPWvbvBesWmv8Ag7TNTsbNbK2mi+S2QALFtJUqMADAIOOBxXDr8DLAxLZS+J9dk0lW3CxMw2dc+mP0r0zTdOtNI0230+xhWG1t0EcUa9gP5/WoqSi1oOKaLVFFFZFhRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFY+qWVzbXY1rSk3X0S7ZoM4F3EOdh/2hyVbscjoxrYoqoycXdCauSafqFtqlhDe2km+CVcqSMEdiCDyCDkEHkEEVZrlmf/AIR3XBdDjS9SlCXA7QXB4WT2D8K3+1tPdjXU13wkpK6MGrOwUUUVQgooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACsPxLezx20Gm2MhjvtRcwxyL1hQDMkv/AAFen+0VHetyuY01v7U1zUdYbmKNjY2nsiN+8Yf70gI9xGpqKkuWNyoq7NS0tILGzhtLaMRwQoI40HRVAwBU1FFeebhRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQBDeWkF/ZzWlzGJIJ0MciH+JSMEVX8NXs8lrNpt9IZL/TnEMsjdZkxmOX/AIEuM/7QYdqvVi6k39l67p+sLxFKwsLv3R2/dMf92QgewkY1tRnaVu5E1dHT0UUV2mIUUUUAFFFFABRRUF5fWmnWr3V9cw21vGMvLM4RV+pPFAE9Fca3j8aiSvhjRL/Wh2ucfZ7X/v7Jjd/wFWphh8ealzPq2k6NGeQllatcyD2LyEL/AOOVEqkY7sai2drRXEf8IddT/NfeMPEk7nr5d0tuv4CNFx+dI3gDTHA8zUtfkPq+sXBP/odR7eBXIzuKK4b/AIV5o/8Az+65/wCDe4/+Lo/4V5o//P7rn/g3uP8A4ul9YiHs2dzRXDf8K80f/n91z/wb3H/xdH/CvNH/AOf3XP8Awb3H/wAXR9YiHs2dzRXDf8K80f8A5/dc/wDBvcf/ABdH/CvNH/5/dc/8G9x/8XR9YiHs2dzRXDf8K80f/n91z/wb3H/xdH/CvNH/AOf3XP8Awb3H/wAXR9YiHs2dzRXDf8K80f8A5/dc/wDBvcf/ABdH/CvNH/5/dc/8G9x/8XR9YiHs2dzRXDf8K80f/n91z/wb3H/xdH/CvNH/AOf3XP8Awb3H/wAXR9YiHs2dzRXDf8K80f8A5/dc/wDBvcf/ABdH/CvNH/5/dc/8G9x/8XR9YiHs2dzRXDf8K80f/n91z/wb3H/xdH/CvNH/AOf3XP8Awb3H/wAXR9YiHs2dzRXDf8K80f8A5/dc/wDBvcf/ABdH/CvNH/5/dc/8G9x/8XR9YiHs2dzRXDf8K80f/n91z/wb3H/xdH/CvNH/AOf3XP8Awb3H/wAXR9YiHs2dzRXDf8K80f8A5/dc/wDBvcf/ABdH/CvNH/5/dc/8G9x/8XR9YiHs2dzRXDf8K80f/n91z/wb3H/xdH/CvNH/AOf3XP8Awb3H/wAXR9YiHs2dzRXDf8K80f8A5/dc/wDBvcf/ABdH/CvNH/5/dc/8G9x/8XR9YiHs2dzRXDf8K80f/n91z/wb3H/xdH/CvNH/AOf3XP8Awb3H/wAXR9YiHs2dzRXDf8K80f8A5/dc/wDBvcf/ABdH/CvNH/5/dc/8G9x/8XR9YiHs2dzRXDf8K80f/n91z/wb3H/xdH/CvNH/AOf3XP8Awb3H/wAXR9YiHs2dzRXDf8K80f8A5/dc/wDBvcf/ABdH/CvNH/5/dc/8G9x/8XR9YiHs2dzRXDf8K80f/n91z/wb3H/xdH/CvNH/AOf3XP8Awb3H/wAXR9YiHs2dzRXDf8K80f8A5/dc/wDBvcf/ABdH/CvNH/5/dc/8G9x/8XR9YiHs2dzRXDf8K80f/n91z/wb3H/xdH/CvNH/AOf3XP8Awb3H/wAXR9YiHs2dzRXDf8K80f8A5/dc/wDBvcf/ABdH/CvNH/5/dc/8G9x/8XR9YiHs2dzRXDf8K80f/n91z/wb3H/xdH/CvNH/AOf3XP8Awb3H/wAXR9YiHs2dzRXDf8K80f8A5/dc/wDBvcf/ABdH/CvNH/5/dc/8G9x/8XR9YiHs2dzRXDf8K80f/n91z/wb3H/xdH/CvNH/AOf3XP8Awb3H/wAXR9YiHs2dzRXDf8K80f8A5/dc/wDBvcf/ABdH/CvNH/5/dc/8G9x/8XR9YiHs2dzRXDf8K80f/n91z/wb3H/xdH/CvNH/AOf3XP8Awb3H/wAXR9YiHs2dzRXDf8K80f8A5/dc/wDBvcf/ABdH/CvNH/5/dc/8G9x/8XR9YiHs2dzRXDf8K80f/n91z/wb3H/xdH/CvNH/AOf3XP8Awb3H/wAXR9YiHs2dzRXDf8K80f8A5/dc/wDBvcf/ABdH/CvNH/5/dc/8G9x/8XR9YiHs2dzRXDf8K80f/n91z/wb3H/xdH/CvNH/AOf3XP8Awb3H/wAXR9YiHs2dzRXDf8K80f8A5/dc/wDBvcf/ABdH/CvNH/5/dc/8G9x/8XR9YiHs2dzRXDf8K80f/n91z/wb3H/xdH/CvNH/AOf3XP8Awb3H/wAXR9YiHs2dzRXDf8K80f8A5/dc/wDBvcf/ABdH/CvNH/5/dc/8G9x/8XR9YiHs2dzRXDf8K80f/n91z/wb3H/xdH/CvNH/AOf3XP8Awb3H/wAXR9YiHs2dzRXDf8K80f8A5/dc/wDBvcf/ABdH/CvNH/5/dc/8G9x/8XR9YiHs2dzRXDD4e6QCCL7XAR/1F7j/AOLp3/CDRJ/qPEfieHHTGrSuB+Dkin7eAezZ29FcT/YXiqy50/xpPMq8rFqVlFMD7FkCN+tOHiLxbpXOr+G4dQgH3rjRp9zgf9cZMH8mNUqsH1E4tHaUVhaJ4w0LxBM1vY3yi8QZks51MM6euY3Ab8cYrdrQkKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKAMzxFqEml6BeXUABuQgjt1PRpnISMfi7KKj0ywj0vS7WwhJMdvEsYY9WwMZPuepqr4hP2nWNC07qpne8kX1SJePykkiP4VqVy4iWqRrTXUKKKK5jQKKKKACiiigAooooAKKKKACiiqGpazp+kKn2y4CPIcRRKpeSQ+iooLN+Ap2uBfornjr2q3HNl4flC9nvrlYAffCh2H4gH2ph1TxTk40PRiPfV5f/kaq9nIj2ke50lFc3/anir/oB6N/4N5f/kaj+1PFX/QD0b/wby//ACNR7OQe0j3Okorm/wC1PFX/AEA9G/8ABvL/API1H9qeKv8AoB6N/wCDeX/5Go9nIPaR7nSUVzf9qeKv+gHo3/g3l/8Akaj+1PFX/QD0b/wby/8AyNR7OQe0j3Okorm/7U8Vf9APRv8Awby//I1H9qeKv+gHo3/g3l/+RqPZyD2ke50lFc3/AGp4q/6Aejf+DeX/AORqP7U8Vf8AQD0b/wAG8v8A8jUezkHtI9zpKK5v+1PFX/QD0b/wby//ACNR/anir/oB6N/4N5f/AJGo9nIPaR7nSUVzf9qeKv8AoB6N/wCDeX/5Go/tTxV/0A9G/wDBvL/8jUezkHtI9zpKK5v+1PFX/QD0b/wby/8AyNR/anir/oB6N/4N5f8A5Go9nIPaR7nSUVzf9qeKv+gHo3/g3l/+RqP7U8Vf9APRv/BvL/8AI1Hs5B7SPc6Siub/ALU8Vf8AQD0b/wAG8v8A8jUf2p4q/wCgHo3/AIN5f/kaj2cg9pHudJRXN/2p4q/6Aejf+DeX/wCRqP7U8Vf9APRv/BvL/wDI1Hs5B7SPc6Siub/tTxV/0A9G/wDBvL/8jUf2p4q/6Aejf+DeX/5Go9nIPaR7nSUVzf8Aanir/oB6N/4N5f8A5Go/tTxV/wBAPRv/AAby/wDyNR7OQe0j3Okorm/7U8Vf9APRv/BvL/8AI1H9qeKv+gHo3/g3l/8Akaj2cg9pHudJRXN/2p4q/wCgHo3/AIN5f/kaj+1PFX/QD0b/AMG8v/yNR7OQe0j3Okorm/7U8Vf9APRv/BvL/wDI1H9qeKv+gHo3/g3l/wDkaj2cg9pHudJRXN/2p4q/6Aejf+DeX/5Go/tTxV/0A9G/8G8v/wAjUezkHtI9zpKK5v8AtTxV/wBAPRv/AAby/wDyNR/anir/AKAejf8Ag3l/+RqPZyD2ke50lFc3/anir/oB6N/4N5f/AJGo/tTxV/0A9G/8G8v/AMjUezkHtI9zpKK5v+1PFX/QD0b/AMG8v/yNR/anir/oB6N/4N5f/kaj2cg9pHudJRXN/wBqeKv+gHo3/g3l/wDkaj+1PFX/AEA9G/8ABvL/API1Hs5B7SPc6SiucXVPFGfn0TSAP9nVpCf/AEnFPHiS4tv+Qnol7bp3mt8XMY/BPn/8cxR7OQKpF9ToKKr2d9a6japc2VxFcQP92SNgwP5VYqCwooooAKKKKACiiigAooooAKKKKACqmp2EeqaXdWEpKpcRNGWHVcjqPcdR9Kt0UwIfD2oSapoFndzgC5KbLhR0WZCUkH4OrD8KKo+Hj9m1fXNO6Ktwl5Evokq8/nIkp/GivRi7pM52rM6GiiimIKKjnnhtbeS4uJUihiUu8jsAqqOSST0FcE81/wDERjteew8J9BtzHPqY9c9UhP8A30w9AamUlFXY0my/e+MrnU7uXTfB9rFqFxG2yfUJiRZ2x7jcOZGH91fxIplp4JtZbtNQ8Q3Uuvaipysl2B5MJ/6Zwj5V+vJ966Gzs7bT7SK0s4I7e3iXbHFEoVVHoAKnrknWlLY1UEgAAGBwKKKKxLCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAy9a8OaT4giRNTskmaM5ilGVkiPqjjDKfoaxlfxR4RO5Hn8S6MOsb4+3QD/ZPAmHscN7mutorSFSUdiXFMj0XXdN8Q6eL3TLlZ4clWGCrRsOqsp5Vh6GtGuN1jwxK2oNrnh65XTtcAwz4zDdgdEmQfeHYMPmHY9q0/DXimHXlmtbiBrHWLTAvLCU5aM9mU/xIezDg11wqKZlKLRv0UUVoSFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQBzjnz/HVwT0tNOiVfrLI5YflCla1ZGnHzPE3iGT+5PDD+UKP/wC1K164azvNm8NgooorIoKKKKACiiigAooooAKKKQkKpZiAAMkntQBj61q09vLFpumqj6lcKWUuMpBGODI+O3YDqx44AJEGnaRb6ezzbnuL2UfvruY7pZfqew9FGAOwqt4eBu7aXWpQfO1NhOM9Vh/5ZJ7YTBI/vMx71s10xjyo5Kk3JhRRRVGYUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRTI5ops+VKj467WBxQA+iiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigDJvNMmt7l9U0cpDf9ZYycRXYH8Mg9fR+o9xkHa0vUodW0+O8gDKGyGjcYaNwcMjDsQQQfpUdZlif7P8AF01uvEGpQG4A7CaMqrH6srJ/3wazqRurm1KevKzo6KKKwOkKKKKACiiigAooooAKKKKACiiigDHMqWfjeB3YKl3psgdj0zFIm0f+RnormPipfnSNKsNQUlSJ2gyP9pd3/slFd1F+4jCe56ZR0GTRXF+MLyfWdRg8HadK0bXUfnancRnBgtc42g9nkOVHoNxrRtJXZKVylIx+Imptkn/hErKXCqOmpzKeSfWFSOB0Yj0FdmAFUAAADgAdqitbWCxtIbW1iSG3hQRxxoMBVAwAKmrgnNzdzeKsgoooqCgooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigApkkscS7pHVF9WOBWb4m1uPw54Z1HWJF3i0gaQJnG5v4R+JwPxrxHwf4BvPivbT+KPFms3vlyyslvFAQMAdSNwIVc8AAdjVxjdXb0Jb6I+ggQQCDkHoRS15D4K8E+MPAvjp7OyuGvfCcg+dppVGMjghM5Dg9SBgj9M6z/5Opv/APrgP/SVKfItbMOY9vorhvFPxY8M+E9TOm3TXV3eqMyQ2cYcx8Z+YkgZxzjOa2/CvjHRfGWnveaPcmQRkLLE67ZIiegYe/qMjr6VLi0r2HdbG9RXmmqfHLwjpuoTWka6jfeSxWSa0hVowQcHlmGRnuOK6jTfHGiav4SuvEtjNJLY2kckk6hMSJsXcylT3x+HI5ocJLVoLo6OivNJ/jp4Oh06G7U38zS5Jt4oAZIwDjLZYAZ7c59q7Hwx4p0rxfo41PSJmkh3FHV12vGwwSrD1wR+dDhJK7QJpm1RXLfEn/km/iH/AK8pP5V4x8OfhDpfjHwimsXGp3ttctM8YEO3aNpxnkZ/WqjBON2xN62R9IUV4X4G1TWvAnxQfwJquoSX2n3A/wBGkkJO0ldyMuScA4Klemf17zxV8V/DfhLVDpl0bu7vlALwWcQcpkZGSSBnHOM5odN3stQUlY7iiuL8PfFHw34l0zUbyye5RtPga4uLeaMLKI1BJIAJB6Y4Pp615t4F+LDX3xIvv7Xu72e21GZbTTIVQbIg0uFyoIA4IyeT9aFTk7+QcyPfaK4PxP8AFzwz4X1Z9Ln+23l7H/rY7KIP5ZxnBLMBnHpmtXwf4+0LxxBO+kyyiWDHmwTptdQeh4JBHB6E1PJK17Dutjp6KKKkYUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAVgeJPDz6oYNR02cWeu2WWtLvHB9Y5B/FG3Qjt1HNb9FNNp3QmrlDwv4jTxDp8hkgNpqNq/kX1m5y0Eo7e6nqrdCDW5XC+KLafQ9Qj8Y6ZEzzWqeXqVvGObq1zknHd05ZfbI712lpdQX1nDd2sqy286LJHIpyGUjII/Cu+nPnVzCSsyaiiirEFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFAHNaR/yHPE//YST/wBJLetisfSP+Q54n/7CSf8ApJb1sVwVfjZvHYKKKKzKCiiigAooooAKKKKACsfxZM9v4O1yaM4ePT53U+hEbEVsVieMv+RG8Qf9g25/9FNTW4nsTwRJbwRwxjCRqEUegAxUlFFdRwhRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFADX/wBW30NeIfAW5Sy0TxNdyBjHB5UjBepCq5OPfivb3/1bfQ14L8GP+RO8Z/8AXAf+i5KpbMa2OxPxv8Mvpoure01OaXLZtlhUuijHzthiAvPrnjpXT+GvHGkeKtDuNU08zBLbPnwyKBIhAz0zjkdDmuI+AFtD/wAIdqU/lr5sl+Y2bHJURoQPp8x/Osb4NqI7Px1GowiqgAHbiam0tRtI6j/hePhh9O+0wWmpyzb2X7MIV8wKACXOGIC84654PFdV4V8a6P4w0yW90ySTMHE0EqhZIzjIyMkc4OCDivOv2fXsf7J1hAYxf+epfONxi2/L+Gd1VfhoI2+K/jH+zADphjnC+X9zPmjbjHbG/HtQ0tQaR6Z4O8b6Z43tLm502C7iS3cRuLlFUkkZ42saoWvxO0W7Ov8Al2uoD+w1ZrndGnzhWKnZ8/PIPXFcb+z5LGNF1tC6hlnjdgT0G08/oai+Dc8dz488XzxMHjlkLow6EGViDQ0tQtuVPhh8SZLvxjqFvrM99d3GrzRR2ZwpSEL5hwRkbR8w+6D3r3avF/gUQNX8XgkA+dD/AOhTV7RSluKW4UUUVIgooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACsrUvl1/wAOOOrXskZ/3TbTN/NRWrWVqn/Ic8Nf9hF//SS4pPZlQ+JHS0UUVynaFFFFABRRRQAUUUUAFFFFABRRRQB5d8eOPA9l/wBhJP8A0XLRR8ef+RHsv+wlH/6KlortofAYz3PU9U1K20fSrvUrx9ltaxNLI3oqjJ/GuZ8GadcwadNq2pJt1bV5Ptd0D1jBH7uL6ImF+ufWk8akatquh+GBzHdTm9vV65t4CG2kejSGMfnXS1GIl9kdNdQooorlNQooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooA5P4m6dPqvw31y0tlLzG38xVXq2xg5A98Ka5n4Ea3ZXngCLS0mT7ZYyyCSHPzbWYsGx6fNjPtXqVeY+IPgb4Y1nUZL61lu9MlkJZ0tSvlknqQpHH4HHtWkWuXlZLTvdGuvxO0qX4hr4Qtrae7mb5Tc25VkV8FmDc9FA5IzzkY4rh7P/k6m/wD+uA/9JUrv/Bfw10DwOZJtPSWe8kXY91cMGfb/AHRgAAfQfUmpYvAGmxfEKbxmLm7N/Kmwwll8oDyxH0256D1601KKvbsFm9zxf4eP4sbx14om0ODRpdV85/tB1QvvUGRt2zaR3xn/AIDXUeANKvdE8ceJdb1O/wBAML20zX1jplxvMTqwJzGeQBhhyf4q6vxJ8I9F1/XH1q3vb7StQk/1stlIFEh7kjHBPfBGe9anhH4d6F4Osbq3s45LmS8G25nuSGeVf7vAAA5PH55qpVE0Sos838N6vr/iXRtXuPBui+HfDug+Y63EsyFnfCAn5QNv3SOCMc9etZ3wuz/wpbx1zx5Nxx/27mu1tvgboFreTeVqmrrpsz7pNPW42xyeisQMkfr71taD8MtJ8PaDrmjWl5fNZ6vG0ciyMhMQZWUlCF9G756Cm5xtoCTOZ+BunWj/AAxvWe3jZru4mSYlQS6hQAp9R149zVH9nIn+wNbGeBdIcf8AAa9K8JeELLwd4fOj2M9xNAZGk3zlS2WxnoAO3pVXwR4D07wHaXdtp1zdTpcyCRzcMpIIGONoFTKafN5jS2F+JP8AyTfxD/15SfyrmfgXNFH8M4d8iLi6mJ3MB3Fd/rmkQa/od7pNy8kcN3EYnaMgMAfTIIzXmI/Z38L551TWMf8AXSL/AOIpRceXlbG073RgXlxF4q/aUsX0uRbiCxCeZNGdy4jUljkdskL9a2D4kvdY+JWsWHgXw7osGpWu+K81W9UhmwwVjhefvDHfOM16B4R8B6D4KgkTSLZvOlAEtxM2+RwO2egHsABXP6z8H9I1PxLPrlpqmqaVc3JLXAspQgcn7xBxkZ79RntVc8W/kKzOH+FsV2nxw8SxajPb3N2LScXEkC7Y3fzYt2B9c1b+EUMR+KfjUNGh2TyFAVHy4nbp6V3fhv4W6H4U8TNrekz3sbtB5DQPIGRgQMk5G7JKhuvX24qjN8HtFfxofEsGoahbTNdC7eCJ12NIHDnqM4LDOKHOLv6CUWc5ZeI9R1/x5rdv4B8PaHZ3EDNHeapeKQ0vzkFsLzywJ6HOMms74NpcR/FbxZHdzwz3CrKJpIF2o7+cMlR2Gc12OpfBrRrzxFc6xZ6pqumPdsWuIrKYIH3HLYOMgE8kcitTwt8MtF8H+IbjVtInvI/Ph8hrZ3DRgfKc8jdnK569zQ5x5WkOzudpRRRWBYUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUABAIweRXK+D3/4R/xBqPg+TItlBv8ASs/88Gb54x/uOeP9lhXVVyXjsNpttp3iiFSZtEuRNLtHLWz/ACTL/wB8nd/wAVrRlyyImro7qikR1dFdGDKwyCDkEUtdxiFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFAHNaR/yHPE/wD2Ek/9JLetisfSP+Q54n/7CSf+klvWxXBV+Nm8dgooorMoKKKKACiiigAooooAKxPGX/IjeIP+wbc/+imrbrE8Zf8AIjeIP+wbc/8AopqcdxPYt0UUV1HCFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAIRkEetcp4a+Hmj+FdN1KxsJrx4tRTZMZpFYgYYfLhRj7x9a6yii4GB4R8Iad4M0uXT9MkuHhlnM7G4cM24qq9gOMKKzdK8D6X4O0zXpdNluna/hLS+e6tgqHxjAGPvmuxop3C588/C74f6N4z8J3kt+1zBcw3hRJ7aQK2wouVOQQR+Fe0eFPB+keDdOez0qJx5jBpZpW3SSEdMnjp6AAfnW6qKgwqhR7DFLTcmxt3POrj4LeFrjWJb9Xv4I5mLSWkM4WJsnJHTcB7A/TFb3hrwHo3hPVNQv9L89Gvj88TspjjG4kBAAMAZx1PFdPRSuwuzg7f4S+H7PxaviG2mvop1n+0CBZF8rfnPTbnHtmu8ooobuK4UUUUgCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKytU/5Dnhr/ALCL/wDpJcVq1lap/wAhzw1/2EX/APSS4pPZlQ+JHS0UUVynaFFFFABRRRQAUUUUAFFFFABRRRQB5d8ef+RHsv8AsJR/+ipaKPjz/wAiPZf9hKP/ANFS0V20PgMZ7nU6QRqfj/xJqn3ksxDpcLem1fNk/wDHpAP+A11Fcr8PyJ/Dc2o/xajqF3dn6NM4X/x0LXVVzVHebNIqyCiiisygooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAqC9tIr+xuLOdd0NxE0Ug9VYYP6Gp6KAML4cXk114G0+G6bN3Y77CfPXfCxj5+oUH8a6quM8FE2viLxhpv8Meopdp9JoUJ/8AHlb9a7OvSi7pM53owooopiCiiigAooooAKKKKACiiigAooooAKKKKAOa0j/kOeJ/+wkn/pJb1sVj6R/yHPE//YST/wBJLetiuCr8bN47BRRRWZQUUUUAFFFFABRRRQAVieMv+RG8Qf8AYNuf/RTVt1ieMv8AkRvEH/YNuf8A0U1OO4nsW6KKK6jhM/U9Xi05ooFikubybPk20ON746k5ICqMjLEgcgdSAaRHiecbxcaRZE/8szbyXOP+Bb4/5U3w4ou7aTW5Pmn1E+YrH+GDJ8pB6Dbgn/aZj3rbrrhRja8iWzF+z+Kv+gzo3/gpl/8Akmj7P4q/6DOjf+CmX/5JraorT2UOwuZmL9n8Vf8AQZ0b/wAFMv8A8k0fZ/FX/QZ0b/wUy/8AyTW1RR7KHYOZmL9n8Vf9BnRv/BTL/wDJNH2fxV/0GdG/8FMv/wAk1tUUeyh2DmZi/Z/FX/QZ0b/wUy//ACTR9n8Vf9BnRv8AwUy//JNbVFHsodg5mYv2fxV/0GdG/wDBTL/8k0fZ/FX/AEGdG/8ABTL/APJNbVFHsodg5mYv2fxV/wBBnRv/AAUy/wDyTR9n8Vf9BnRv/BTL/wDJNbVFHsodg5mYv2fxV/0GdG/8FMv/AMk0fZ/FX/QZ0b/wUy//ACTW1RR7KHYOZmL9n8Vf9BnRv/BTL/8AJNH2fxV/0GdG/wDBTL/8k1tUUeyh2DmZjCDxSDltW0dh6DS5Vz+P2g/yp0WtXNpcR22tWqWxlYJFdQuXgdj0UkgFGPYEYPADE8Vr1Dd2sF9aTWtzGJIJkKOh7g1MqMWtB8xaorK8N3M1zoNubhzJPCXt5ZD1d4naNmP1KE/jWrXE1bQoKKKKACiiszxBeTWOh3MtsQty22GBiMgSSMEQn/gTChagQT6xc3F3LZ6NapcPC2ya5mcpDE3dcgEuw7gcDuQeKYYPFJOV1bR1HodLlbH4/aB/KtCwsYNNsIbO3UiKFdoyck+pJ7knJJ7kmrNdsaMUtSeYxfs/ir/oM6N/4KZf/kmj7P4q/wCgzo3/AIKZf/kmtqiq9lDsLmZi/Z/FX/QZ0b/wUy//ACTR9n8Vf9BnRv8AwUy//JNbVFHsodg5mYv2fxV/0GdG/wDBTL/8k0fZ/FX/AEGdG/8ABTL/APJNbVFHsodg5mYv2fxV/wBBnRv/AAUy/wDyTR9n8Vf9BnRv/BTL/wDJNbVFHsodg5mYv2fxV/0GdG/8FMv/AMk0fZ/FX/QZ0b/wUy//ACTW1RR7KHYOZmL9n8Vf9BnRv/BTL/8AJNH2fxV/0GdG/wDBTL/8k1tUUeyh2DmZi/Z/FX/QZ0b/AMFMv/yTQLfxSDzrGjEeg0qUf+3FbVFL2UOwczMiLWLuxuY7bWoIollYJFeQMTC7HorA8xse2cg9N2SBW3Va6toL21ltrmJZYJVKOjDIYHqKo+HZ5msJrO5kaW4sJ2tXkY5LgAMjH3KMhPuTWFWmo6opO5r0UUVgMKytU/5Dnhr/ALCL/wDpJcVq1lap/wAhzw1/2EX/APSS4pPZlQ+JHS0UUVynaFFFFABRRRQAUUUUAFFFFABRRRQB5d8ef+RHsv8AsJR/+ipaKPjz/wAiPZf9hKP/ANFS0V20PgMZ7nU/DhNnw58P+rWaOfqRn+tdRXNfD3/knXh3/sHw/wDoIrpa5Jbs1WwUUUVIwooooAKKparq2n6JYPfandxWtsnWSQ457ADqT7Dk1yraz4m8S5/si3Gh6a3S9vYt9zIPVITwg935/wBmqjFy2JlJR3Owu7y1sLdri8uYbeFesk0gRR+J4rm5PiP4bLtHY3Nzqci8bdNtJbgf99IpX9arW3gfRkuFu9QSbWL0f8vGpyGcj/dU/Kv/AAECuiREjQJGqqijAVRgCtVSXUxdbsjC/wCE6nk/1Hg/xI/oWgijH/j0gP6U3/hNdV/6EjWv+/lv/wDHK6Giq5Ik+2kc9/wmuq/9CRrX/fy3/wDjlH/Ca6r/ANCRrX/fy3/+OV0NFHJHsHtpHPf8Jrqv/Qka1/38t/8A45R/wmuq/wDQka1/38t//jldDRRyR7B7aRz3/Ca6r/0JGtf9/Lf/AOOUf8Jrqv8A0JGtf9/Lf/45XQ0Uckewe2kc9/wmuq/9CRrX/fy3/wDjlH/Ca6r/ANCRrX/fy3/+OV0NFHJHsHtpHPf8Jrqv/Qka1/38t/8A45R/wmuq/wDQka1/38t//jldDRRyR7B7aRz3/Ca6r/0JGtf9/Lf/AOOUf8Jrqv8A0JGtf9/Lf/45XQ0Uckewe2kc9/wmuq/9CRrX/fy3/wDjlH/Ca6r/ANCRrX/fy3/+OV0NFHJHsHtpHPf8Jrqv/Qka1/38t/8A45R/wmuq/wDQka1/38t//jldDRRyR7B7aRz3/Ca6r/0JGtf9/Lf/AOOUf8Jrqv8A0JGtf9/Lf/45XQ0Uckewe2kc9/wmuq/9CRrX/fy3/wDjlH/Ca6r/ANCRrX/fy3/+OV0NFHJHsHtpHPf8Jrqv/Qka1/38t/8A45R/wmuq/wDQka1/38t//jldDRRyR7B7aRz3/Ca6r/0JGtf9/Lf/AOOUf8Jrqv8A0JGtf9/Lf/45XQ0Uckewe2kc9/wmuq/9CRrX/fy3/wDjlH/Ca6r/ANCRrX/fy3/+OV0NFHJHsHtpHPf8Jrqv/Qka1/38t/8A45R/wmuq/wDQka1/38t//jldDRRyR7B7aRz3/Ca6r/0JGtf9/Lf/AOOUf8Jrqv8A0JGtf9/Lf/45XQ0Uckewe2kc9/wmuq/9CRrX/fy3/wDjlH/Ca6r/ANCRrX/fy3/+OV0NFHJHsHtpHPf8Jrqv/Qka1/38t/8A45R/wmuq/wDQka1/38t//jldDRRyR7B7aRz3/Ca6r/0JGtf9/Lf/AOOUf8Jrqv8A0JGtf9/Lf/45XQ0Uckewe2kc9/wmuq/9CRrX/fy3/wDjlH/Ca6r/ANCRrX/fy3/+OV0NFHJHsHtpHPf8Jrqv/Qka1/38t/8A45R/wmuq/wDQka1/38t//jldDRRyR7B7aRz3/Ca6r/0JGtf9/Lf/AOOUf8Jrqv8A0JGtf9/Lf/45XQ0Uckewe2kc9/wmuq/9CRrX/fy3/wDjlH/Ca6r/ANCRrX/fy3/+OV0NFHJHsHtpHPf8Jrqv/Qka1/38t/8A45R/wmuq/wDQka1/38t//jldDRRyR7B7aRz3/Ca6r/0JGtf9/Lf/AOOUf8Jrqv8A0JGtf9/Lf/45XQ0Uckewe2kc9/wmuq/9CRrX/fy3/wDjlH/Ca6r/ANCRrX/fy3/+OV0NFHJHsHtpHPf8Jrqv/Qka1/38t/8A45R/wmuq/wDQka1/38t//jldDRRyR7B7aRz3/Ca6r/0JGtf9/Lf/AOOUf8Jrqv8A0JGtf9/Lf/45XQ0Uckewe2kc9/wmuq/9CRrX/fy3/wDjlH/Ca6r/ANCRrX/fy3/+OV0NFHJHsHtpHPf8Jrqv/Qka1/38t/8A45R/wmuq/wDQka1/38t//jldDRRyR7B7aRz3/Ca6r/0JGtf9/Lf/AOOUf8Jrqv8A0JGtf9/Lf/45XQ0Uckewe2kc9/wmuq/9CRrX/fy3/wDjlH/Ca6r/ANCRrX/fy3/+OV0NFHJHsHtpHPf8Jrqv/Qka1/38t/8A45R/wmuq/wDQka1/38t//jldDRRyR7B7aRz3/Ca6r/0JGtf9/Lf/AOOUf8Jrqv8A0JGtf9/Lf/45XQ0Uckewe2kc9/wmuq/9CRrX/fy3/wDjlKPG1+ozN4K19R/0z+zv/wC1a6Cijkj2D20jBHxD02HnUdL13TlHV7jTZCo/4FGGH61saT4n0LXeNL1azu3xkxxTAuPqvUfiKmrJ1Xwxoet86jpltNJnIl2bZFPqHGGH4GpdOI1WfU6aiuHXSfEmgnfoWsNqFsOun6u5fj0ScfOvsG3CtXRPGNlql5/Zt3DLpesAZawu8BmHcxsPlkX3U/UCs5U2jaNRSOjoooqCwooooAKKKKAOY0HMfxP8Tp2lsbGT8jMv9K7SuN0j/kqmvf8AYLs//Rk9dlXoU/gRhLcKKKKskKKKKACiiigAooooAKKKKACiiigAooooA5rSP+Q54n/7CSf+klvWxWPpH/Ic8T/9hJP/AEkt62K4Kvxs3jsFFFFZlBRRRQAUUUUAFFFFABWJ4y/5EbxB/wBg25/9FNW3WJ4y/wCRG8Qf9g25/wDRTU47iexbooorqOEwvCP/ACJehf8AYOt//Ra1s1jeEf8AkS9C/wCwdb/+i1rZr0lsQ9woqrd6nYaeVF7fW1sW+750qpn6ZNTxTRzxLLDIkkbDKujAg/QimIfRTZJEhjaSV1RFGWZjgAe5qra6tpt9IY7PULS4cdVhmVyPwBoAuUU2SRIkLyOqIOrMcAUkUsc8YkikSRD0ZGyKAH0VELiAzmATRmYdYww3D8Ki/tOw/wCf62/7+r/jSAtUVWXUbFmCre25YnAAlXJ/Wlkv7OKQxyXcCOOqtIARQBYoqr/adh/z/W3/AH9X/GpGu7ZJhC1xEspwAhcBjnpxQBNRTZJI4YzJK6og6sxwB+NQHULIIrm7twrZAYyDBx170wLNFVf7TsP+f62/7+r/AI1NDPDcIXhlSVQcEowYZ/CkBJRRRTAyfC3/ACCJv+wjff8ApVLW1WL4W/5BE3/YRvv/AEqlrarzpfEzQKKKKkArF8U/8giH/sI2P/pVFW1WL4p/5BEP/YRsf/SqKqj8SA1qKKK9EzCiohdW7TmBZ4jKOsYcbvyqWgAoqvd39nYRiS8u4LZDwGmkCA/iafb3NvdwiW2nimiPR43DA/iKAJaKrPqFlG5R7y3VlOCplUEH86T+07D/AJ/rb/v6v+NIC1RUTXVusqxNPEJG+6hcZP0FPkljhjMkrqiDqzHAH40wHUU2OWOaMSROroejKcg/jTJbq3gdUmnijZugdwCfzoAloopizRNIY1lQuOqhhkfhQA+iiigArJ0P/kK+I/8AsIp/6S29a1ZOh/8AIV8R/wDYRT/0lt6wr/AVE2qKKK4ygrK1T/kOeGv+wi//AKSXFatZWqf8hzw1/wBhF/8A0kuKT2ZUPiR0tFFFcp2hRRRQAUUUUAFFFFABRRRQAUUUUAeXfHn/AJEey/7CUf8A6Kloo+PP/Ij2X/YSj/8ARUtFdtD4DGe51nw9/wCSdeHf+wfD/wCgiulrmvh7/wAk68O/9g+H/wBBFdLXJLdmq2CiiipGFYniPxJb+H7eJfJku9QumKWdjD/rJ3/oo6ljwBVnXtbtfD2jzajd7mVMKkSDLyueFRR3YnAFc/4d0a6S4l17WysmuXigOAcrax9RDH6Adz/EefStIQvqzOpPlQzTPDlxcX8et+JZkvdVXmGJf+PeyB/hiU9/Vzyfauloorc5W29WFFFFAgooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigArO1nQ9P16z+zX8O8K2+ORSVkicdGRhyrD1FaNFAHM6frt/4ZvodJ8S3BuLKZhHZawygZJ6Rz44V/Rujexrtqx76xtdSsZrK9gSe2nQpJG4yGBrD8OahdaDq6+FdWuHnjdC+k3sp+aaNesTnvIg7/xLz1BrKcOqOmnUvoztKKKKxNgooooA5rSP+Sqa9/2C7P8A9GT12VcbpH/JVNe/7Bdn/wCjJ67KvQp/AjCW4UUUVZIUUUUAFFFFABRRRQAUUUUAFFFFABRRRQBzWkf8hzxP/wBhJP8A0kt62Kx9I/5Dnif/ALCSf+klvWxXBV+Nm8dgooorMoKKKKACiiigAooooAKxPGX/ACI3iD/sG3P/AKKatusTxl/yI3iD/sG3P/opqcdxPYt0UUV1HCYXhH/kS9C/7B1v/wCi1qHxrr7eGPCGo6sihpYYwIg3TexCrn2yQfwqbwj/AMiXoX/YOt//AEWtY/xS0ufVvh1qsFsheZFWYKOpCMGP6A16P2Sep5/4H+Gdv410Y+JvFF9e3NzfOxjCyYIUErkkg85BwOgGK6LwT4D8Q+C/GN0lpfJN4ZkB+SaQ72yMghQMbgeCeMj8BU3wi8UaVceBLPT5L2CG7sd6SxSSBTjcSGGeowRz6g1e0n4mWut+PpvDenWRureMEi/ilBT5VyxIx0z8oIPJIqUo6MpuWqOF1s33xS+KVz4b+2SW2i6YXEixnrsIVmx0LFjgZ6D8c3/Fnwc03R9AuNX8O3l9Bf2EZuBvlzvCjJwQAQ2ASCPpWboN/D4G+OGtRaw4t7bUGlEc8nCgSOJEYn042k9j9DXovj7xfo+keDtRJv7eW4ubZ4beKOQMzsykA4HYZyTSSTTbBtppI49fFNx4r+AmsXN62+9tl+zzP03kMhDfiGGfcGsD4L+LZdG1RPD+olkstS/eWjPwBJkrx7NtI/3gPU1Y8O6VPYfs+eILmdCn21zLGCOqAooP4kH8MUo8HN4h+CGiapYIRqumrNJGU+88Ymcso9x1HuCO9LW6Y9NUa2lf8nKat/17/wDtGOs/4i/C3w/4b8H3usWL3puo3jwJZQy/M4B42j1rN+F2tXHiH4utql2B9omsiJCP4mWNFLfjtz+NekfGT/kmOpf9dIf/AEYtNWcWw1UkjmPh98LfD+o+HdF8QTvei9OJyFlATcrnHG3pwO9czrumaLq/x01Wz1+8Wz09jlpWmWIAiJcDc3HWvXPhf/yTXQ/+uJ/9DavKtR0rR9Z+PmqWeusi2LcsXm8sZEKkfNkd6GlZAnqzfg+HXwsluI44fE0ckrsFRE1SElmJ4AAHJzWt8YPB0mq6ZH4j0tWXVNMG9jHwzxA5491PzD8fapbPwH8NLG9gvLee1WaCRZYydSJwynI43eoq98UfGyeE/DRS1kU6nfAx2wBzsH8Un4Z49yPenZWdxXd1Y8y1fxlqvxSg0HwtYRNFPLhtQfHys6/xcfwADfj1IHYZ9R1L4VeHNU0bSdLnFzHBpkbJF5LhSxbBZm4OSSufxNePaTaa58KNV0PxFewE2moRFZ4wOVUnJQ+jY2sPcY7GvpK0u4L+zhu7WVZbeZBJHIvRlIyDSjruEtNj5z0jwHo998XNR8MStdf2fbq5QrIA/AUjJx7+le6+FPCWm+DtMlsNMacwyzGZvOcMdxAHYDjCivMfDn/Jxms/7kv/AKCte104JCm2FFFFaEGT4W/5BE3/AGEb7/0qlrarF8Lf8gib/sI33/pVLW1XnS+JmhzPjvxHdeFvDo1GzihllNzFDtmBK4Y4PQjmumrxf4j/AA40bSfDx1CxF+9y95GpDTs4wzc8V6D4b8B6N4Wv5L3TmvDLJEYm8+4Mg2kg9D34FDSsPSx09Yvin/kEQ/8AYRsf/SqKtqsXxT/yCIf+wjY/+lUVEfiQjWqlrGpwaLo15qdyf3NrC0rD1wM4Huen41dryf45a3JHo1j4ctMtc6lMGdF6lFIwPxcj/vk13ydlchK7seV2F1rGkahp3xDnBaO51OTeR1boXH0YM4H+6a+mtS1m3sPDdzrakS28Nq1ypU/fULuGPrx+deHahpfxBuvAkPheXwfbpYWqqySowMoZTkt/rCMnLZ4/iNdF4D1CXxf8G9V0BW339nbyWyLnllKkxf1X/gNRF20NJK+pj+CPBrfE977xP4pvrmVWmMUMUT7RwATjOcKMgAD3rodE+G2ueD/HsV14dvlbQZApuY7qXBKnIZcAfMR1U4Hp65g+CHiPT4fDtzod3cxW17BctII5mCFkIHTPUgg5HbiujufifZDx9aeGNOtP7RE21JLm3mBETknIxjBCqMkg8c+lCUbJibd2jE8d/C3w+NL1/wAR7737d5c13jzRs38t029M+9c18NfhjoHi3wn/AGnqT3guPtDx4hlCrgYxwVPrXrfjz/kQNf8A+vCb/wBBNcv8Dv8Aknn/AG+S/wAlpuK5gTfKc14qjWH4/wDhuJc7Uit1GfQF67n4t/8AJL9a/wB2L/0clcL8Qp4tI+N/h3U7xvKtPLhLSn7qgO4JP0yCa6n4ua7pZ+G9/bx39tJNdmJYUjlDF8SKxIwemATmjow6o0fhJ/yS/Rf92X/0c9eN+OnvvHHivxHqtid9jokKopH9xX2nH1Jd/oDXoWl60/hH9n61vCTHdNbutv2O+SRipH0DbvoK4zwRb+OtJ8M3MWleFLa9sdWXc81weZEK7QPvjjBPbual6pIa0bZ7N4C8QjxP4M0/UWfdcbPKuPXzF4b8+v415z4I/wCS/wDib/cuP/RiVW+C+o3egeJtT8IarG1vNIPOSJzysij5hx6rg/8AAad4Ru7ay+PXiaW6uIoIytwoeVwoz5icZNO90hWtc9xoqjFrekzyrFDqdlJI5wqJcIST6AA1erQzCsnQ/wDkK+I/+win/pLb1rVk6H/yFfEf/YRT/wBJbesa/wABUTaooorjKCsrVP8AkOeGv+wi/wD6SXFatZWqf8hzw1/2EX/9JLik9mVD4kdLRRRXKdoUUUUAFFFFABRRRQAUUUUAFFFFAHl3x5/5Eey/7CUf/oqWij48/wDIj2X/AGEo/wD0VLRXbQ+AxnudZ8Pf+SdeHf8AsHw/+giulrmvh7/yTrw7/wBg+H/0EV0tckt2arYKKKxPF+sSaF4Wvr6AbroJ5dsn96ZyEjHv8zCkld2GYKN/wlXjSW8f5tK0KRoLZe0t3jEkn/AAdg9yxrqazfD+kR6FoNnpsZ3eRGA7nq7nl2PuWJP41pV1JWVkcUpczuFFFFBIUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABWT4j0Rde0h7ZZPIuo2E1pcDrBMvKOPoevqCR3rWooAqeFNcbX9CjuZ4hDewu1veQf88p0OHH07j2IrbrioD/YXxHUD5bPX4DkdhdQjIPtujz9fLrta55xszthLmVwoooqCjmtI/5Kpr3/AGC7P/0ZPXZVxukf8lU17/sF2f8A6Mnrsq9Cn8CMJbhRRRVkhRRRQAUUUUAFFFFABRRRQAUUUUAFFFFAHNaR/wAhzxP/ANhJP/SS3rYrH0j/AJDnif8A7CSf+klvWxXBV+Nm8dgooorMoKKKKACiiigAooooAKxPGX/IjeIP+wbc/wDopq26xPGX/IjeIP8AsG3P/opqcdxPYt0UUV1HCYXhH/kS9C/7B1v/AOi1rZrG8I/8iXoX/YOt/wD0WtbNektiHucBrPwc8JazfPd+Tc2Ukh3OtpIFVj67SCB+GK6Dwx4M0TwhbyRaTalHlx5k0jbpHx0yfT2GBW/RRyrcLswPE3gzQ/F0CR6taeY8fEcyNtkT6EdvY5Fcxp/wS8IWN2s8iXt4FORFczAp+IVRn8a9GopOKYczRnatollrOhT6NcIUs5oxGUhO3aoIwB6dBRoOiWnhzRbfSbDzPs1vuCeY25uWLHJ+pNaNFOwXOV0j4eaBoXiSbXdPimiupd+U8z92u45OFxxWt4h0Cy8TaNNpWoeZ9mlKlvLba3ykEc/UVqUUWQXZQ0XSLXQNHttLst/2a3Xam9stjJPJ/GuV1z4T+GvEGs3Oq3wvPtNwQX8ubavAA4GPQV3NFDSYXaPNv+FHeD/S/wD/AAI/+tW7f/DrQtU8Q22tXwubi4twgijklzGoToNuOmecdyTXWUUuVD5mZfiDw/p/ifSJdM1OIyW8hDfKcMrA8EHsf8TTfDnh608MaSumWMtw9sjFkWeTeUzyQDjpnJ/E1rUU7dRX6HOWfgnSLLxdceJoRP8A2jcBg+6TKcgA4GPYV0dFFFrBcKKKKYjJ8Lf8gib/ALCN9/6VS1tVi+Fv+QRN/wBhG+/9Kpa2q86XxM0CiiipAKxfFP8AyCIf+wjY/wDpVFW1WL4p/wCQRD/2EbH/ANKoqqPxIDWrmtQ8DaPqniu18R3n2iS+tdnlKZP3a7eR8uPU5+tdLRXoWuRewVzPh/wJo3hjWLzUtM+0RyXYIkiMmY+W3cDHGO3sa6aiiwXOJ8R/Cnwv4lvnvri3mtbqQ5kltHCbz6kEEZ98ZNXvCvw+8P8Ag93m022drp12tcTvvkx6DoAPoBXUUUcqvcLvYqapp0Gr6VdaddbvIuYmik2HB2kYODVLw14a0/wppP8AZum+b9n8xpP3r7jk4zz+FbFFFguYnibwlo/i2yS11e28wRkmKRG2vGT12n+nSuRsvgf4RtLpZ5Pt90qnPlTzjYfrtUH9a9JopOKYJtHO+JvBek+K9PtbC/E8dratujit3CKDjA4x2GQPrW5a20VlZw2tugSGGNY41HZQMAfkKmop2C5zN74E0a+8WweJn+0R6lCUIaKTarbeBuGOeOD7Vkar8IfC+s6rdaldi9+0XMhkk2T4GT1wMV3tFLlQXZwWlfCDwvo+q2upWovftFtIJY98+RkdMjFd7RRTSS2BtvcKydD/AOQr4j/7CKf+ktvWtWTof/IV8R/9hFP/AElt6xr/AADibVFFFcZQVlap/wAhzw1/2EX/APSS4rVrK1T/AJDnhr/sIv8A+klxSezKh8SOlooorlO0KKKKACiiigAooooAKKKKACiiigDy748/8iPZf9hKP/0VLRR8ef8AkR7L/sJR/wDoqWiu2h8BjPc6z4e/8k68O/8AYPh/9BFdLXNfD3/knXh3/sHw/wDoIrpa5Jbs1WwVx3i4/bvFHhbSOqfaZNQl+kKYX/x+RT+FdjXG3H774sNnkW2iLt9jJO2f/RQqqfxE1HaLOiooorc4wooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKAOH8afE/TPBOqwafe2V3PJNAJw0O3ABYrjkjn5TXN/8NA+H/8AoE6n+Uf/AMVWF8YNQTSfiv4b1GSNpEtIYJ2ROrBZ3bA9+K6nSvjZpmq6xZadHouoxvd3EcCu+3ClmC5PPTmrtpsVbQ6PXfiBp+ga3o2l3FpcyS6rs8p49u1NzBRuyfftT9W8d2GkeNdO8LzWty91fKjJKm3Yu5mUZyc/wntXM/ELxRq2jeP/AAvp1lNGlteSxrMrQI5YGUKcFgSOPSsfxt/ycD4W/wCuMH/oyShIEjvpvHdhD4/h8Hta3JvJV3CYbfLHyF/XPQeldVXi99/yc5p//XH/ANt3r2ipasJo5bxf8QNC8Foi6jLJJdSDdHawANIR6nJAA9yfpmuQsvj3oE1ykd7puoWkTniUhXAHqQDnH0zWB4KtYfF3xs1+/wBWjW4Fm0rQxSDIBVwicH+6v64Ndl8QfEngSbzvDfieaVZU2P8Au4GLRk4YFWAOOP0OKqy2HZbHf2t1BfWkV1aypNBMgeORDkMp5BFTVw2iahbRfDZ2+H1tJqAty0VrFcOV+ctlslyOBuJx36CuD13WPiX4Y0aPW9T8T6ekzFS2mMkXmAE9Nu3nGecHgd+KXLcVj3WivK/Gvj3WLT4WaH4k0qVLO8vpollxGrgAxuWADA8blHvWVqerfFJPDSeMEvrGCxESz/2ekSs4iOPmbK85HJwRjPGOlHKFj2mivKde+Kl3H8K7DxHptvHHfXs32Z9w3JA4Dbjjv93gH156VmabqnxBuDZ3ui+LdH8RmRgZ7FTFGUGMnIIVgO3GD7UcrDlPaayvEuvW/hnw9eaxco0kVsoJRTgsSQoA/EitKJneJGkTY5UFkznae4z3rzf4ySPeaVonhyFiJtX1GOLj+4Dz+rIaSV2JbnfaRqH9q6NZaj5DQC6gSYROclQwyAffBrA1Tx9YaV43sfC0tpcvdXgQpKu3Yu4kDPOe3pXUxRJBCkUahY0UKqjsBwBXivjH/k4bw7/uQf8AoT00rjSue20V5D4j8aeLrf4sv4Z0SW3kSaNVghnjXYjGPcXZgNxA5bGe34VXt/FPjbwn8SNM0HxJqVvqdrqTIFMcSqF3sVBBCgghh0ORijlDlPT9Q8TaRperWWlXN2o1C9cLDbry5z3I7Dg8n0rXr538a2vic/Gyzjjv7QajKwbTZSvyQxF32K/yckc54P1Ndn4w8Z+JdKudB8I6ZNat4jvIIzd3hUGNGPBKgjHJDHkcDHHNHKPlPVaK8htvFfi7wX4x0zSPFeoWmq2GpsEjuYUCmJiQvYDoSM5B4PB7VH4x8beL7L4of8I5oM0TrPEiQwyRJhXZPvlsZ4+91xx+FHKxcp7FRXjnirXPHngnwBBc6lrEMmrS6n5QljhjZfJ8tiByg/iXOcZqHV9W+KVv4dTxgt9YwWIjSY6ckasyxHGC2V54IJw3Ge3QHKFj2mivN9Q+KkVr8L7TxRFbIb27PkR27E7VmGQ2e+0bSfyHeubmvfi3ZeHR4rl1Kye2EYuHsDEm5YuuSAg7cnDZx70coWPU/FXiO28J+H59Yu4ZZoYSgKRY3HcwUdSB3qxoOsQ+INCs9Wt45I4rqPzFSTG4D3xXmXjHxTD4w+BNzq0cflSNJFHNFnOyRZVyM+nQj2IrT0i78RWnwY0VvC9gt3qTQKo3MoEa85bDEbj0AHv7UW0C2h6XWT4g8S6T4X083ur3aQRHhF6tIfRVHJNeO+IfEnxC8CrZX2peJtNv3lkCy6eFjLLxnBAUHHGMgjnHrUHxzfU7tNFv3ni/si4iD28AH7xJCoLEnHTBXuenQUKOo1E9+ByAfWlry3xH4v8AEngDwJAdZuLO/wDEF3O0cEkS/u1TAO4jauSPTHUjrWNqev8AxG8C21lrmt6pY6pZTSKlxZoiBoyRnGVUYOARkZGfWjlFY9roqCzu4r+xt7yBt0NxGssZ9VYZH6Gp6kQUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFAHL+PQbfw6mrIMyaTdQ3649Ecbx+KFxXbqwZQynIIyCO9c74jtlvfDGq2rDKzWc0Z/FCKt+E7pr3wdol25y01hBI31Mak1lVWiZ0UHo0bFFFFYm5zWkf8lU17/sF2f8A6Mnrsq43SP8Akqmvf9guz/8ARk9dlXoU/gRhLcKKKKskKKKKACiiigAooooAKKKKACiiigAooooA5rSP+Q54n/7CSf8ApJb1sVj6R/yHPE//AGEk/wDSS3rYrgq/GzeOwUUUVmUFFFFABRRRQAUUUUAFYnjL/kRvEH/YNuf/AEU1bdYvjEFvBGvgDJOm3AA/7ZtTjuJ7FqikBDKCDkHkUtdRwmD4UPl+HbWyPEtgv2KRe4aL5P1ADD2YGtqsq/027gv21TSTH9ocAXFtKxVLgDgHIB2uBwGwcjg9AREfEkUIxeaZq9vIOqCwln/8ehDqfzrtp1YtaktG1RWJ/wAJXp//AD7az/4Jbz/41R/wlen/APPtrP8A4Jbz/wCNVpzx7iszborE/wCEr0//AJ9tZ/8ABLef/GqP+Er0/wD59tZ/8Et5/wDGqOePcLM26KxP+Er0/wD59tZ/8Et5/wDGqP8AhK9P/wCfbWf/AAS3n/xqjnj3CzNuisT/AISvT/8An21n/wAEt5/8ao/4SvT/APn21n/wS3n/AMao549wszborAh8Y6TcIXhi1aRAzIWTR7sgMpKsOIuoIII7EEVJ/wAJXp//AD7az/4Jbz/41RzR7hZm3RWJ/wAJXp//AD7az/4Jbz/41R/wlen/APPtrP8A4Jbz/wCNUc8e4WZt0Vif8JXp/wDz7az/AOCW8/8AjVH/AAlen/8APtrP/glvP/jVHPHuFmbdFYn/AAlen/8APtrP/glvP/jVH/CV6f8A8+2s/wDglvP/AI1Rzx7hZm3TXdY0Z3YKiglmJwAPWsceKbBjhbXWCfQ6Pdj9TGBUcsN94ixBcWsljpJP71JiPOuR/c2gnYh75O4jIwOtTKpFK9wsyz4WVv8AhHoJmUr9qkmuwpGCBLK0gB98OK2aAABgcCiuFu7uWFFFFIArG8Uq3/CPTzKpb7LJDdlQMkiKVZCB74Q1s0EAjB5FNOzuBCjrIiujBkYAqwOQR606sGKG+8O5gt7WS+0kH90kJHnWw/ubSRvQdsHcBgYPWpD4psFOGtdYB9Bo92f1EZFd0akWr3IszaorE/4SvT/+fbWf/BLef/GqP+Er0/8A59tZ/wDBLef/ABqq549wszborE/4SvT/APn21n/wS3n/AMao/wCEr0//AJ9tZ/8ABLef/GqOePcLM26KxP8AhK9P/wCfbWf/AAS3n/xqj/hK9P8A+fbWf/BLef8Axqjnj3CzNuisA+MdJWdIDFqwmdWdYzo93uZVIBIHlZIBZcntketSf8JXp/8Az7az/wCCW8/+NUc0e4WZt0Vif8JXp/8Az7az/wCCW8/+NUf8JXp//PtrP/glvP8A41Rzx7hZm3RWJ/wlen/8+2s/+CW8/wDjVH/CV6f/AM+2s/8AglvP/jVHPHuFmbdFYn/CV6f/AM+2s/8AglvP/jVA8VaeTgWus599Gux/7So549wszbrI8OHzzqt8v+ru792jPqqIkOfoTESPY1DLNqeuKba0trjTrN+JbucbJSvcRJ1BP95sY7A9ty2tobO1itreNY4IUCRovRVAwAK5q9RP3UUlYlooornGFZWqf8hzw1/2EX/9JLitWsrU+dd8NDv/AGhIfw+y3H+IpPZlQ+JHS0UUVynaFFFFABRRRQAUUUUAFFFFABRRRQB5d8ef+RHsv+wlH/6Kloo+PP8AyI9l/wBhKP8A9FS0V20PgMZ7nWfD3/knXh3/ALB8P/oIrpa5r4e/8k68O/8AYPh/9BFdLXJLdmq2CuMi5+KWsE9tKtAP+/k1dnXGQ/8AJUta/wCwXaf+hzVdLczq/CdHRRRWxyhRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAeI/Ev8A5Lf4N/3rT/0pavbqKKbdxtni/wAVv+SpeC/+u0X/AKPWjxt/ycD4W/64wf8AoySvaKKfMFzxe+/5Oc0//rj/AO2717RRRSbuDdzwzxBZav8ADH4k3PiyxsHvNGvy5nEefl3kM6sf4TuG4E8dvWqXi74keGPFVhcW2l+EXudZvI/KFzPaxeahxgFSu5mI7dK+gKijt4IWZooY4y3UqoGafMO54kmm+KfDHwFmiht7mzvXujLMqcSxwNjJ45XoM9wCc1x16/g+4+H3l6Ppd9deIjFHJfXciuwgwQXYnO0AngYHQ819R0yOGKEERRogJydqgZNHMHMeA+MZo5v2e/CrRsGAuo0OPUJMCPzBq3qnxRt2+G48NS6Zex65LZJZmJo8IQVC7wc5IK8gY6n05rvfip4U1Lxh4XttP0vyfPjvFnbzX2jaEdeuOuWFdTpFh9j0jToJ44zcW1tHEWAzghQDg/hTurBdHlUWn6h4O+C9rb33hlNXE0jTXlrKxBtwxyrEAE8ADOMFTXnmvxeC7uC0m8GLrEetSyrtswCwT1AP3sg4xgmvqio0t4Y5GkSGNXbqyqAT+NJSBSKWgJfxeHdNj1Ri1+trGLlickybRuye5zmuD1L/AInvx60y0+9Bolg1w49JH6f+hRn8K9NqNYIlmaZYkErDDOFG4/U0kybkleJeMf8Ak4bw7/uQf+hPXttFCdhp2PF5v+Tn7f8A64n/ANJWo+JX/JaPBf8A10t//Sg17RRT5gueG/E69Xw98Y/D+u3kMpsYoIyWjXOdrvuA7EjIOPeqXxKtba/8QaD42n0+5vPDd7bRfaEXKug54JU/KcMMc4yCM175JDHMu2WNHXOcMoIzTioK7SARjGKOYOY+f9Hh8Daz4ysLDwv4Qu72IFZJruS8miFuQc7iMkEADvjJ4FbGoAH9pvT8/wDPL/23evZooYoF2wxJGuc4RQB+lPo5g5jyj4//APIlaf8A9hFf/RclYGtfFC3n+HX/AAjT6ZfR67NaJaNE0WF5AG8HOSCOQMd/xru/ir4S1Pxj4btLDSvJ86K7WZvNfaNoRx1x6sK63TLEWumWEU0cZuLe3SIsBnBCgHB9KLqw7qx4zq/w71j/AIUnpdrHbSPqdpO15LaqMvtfdlQP7wBU49jT774w2eoeBJNDh0y9OuXFqbJoRGCgYrsLDnJ7kDGc8e9e41GIIhMZhEglIwX2jP50c3cV+54fqPhq78L/ALPN5a36GO7uLiO5kiPWPdIgCn3woz7mofE91rNr8C/DB01547RkAvHhJBC87QSOik5z74r3qkIDAggEHgg0cwcx8ueLT4Om8K2o8I6TemSKVHvb+dXOzKkBGYnGSTnjA+Wuw+MEbXfw78J31upktlRd0ijIG6JdufyNe4xxRwpsiRUUfwqMCnOiyIUdQynggjINPmHzHh3xHnX4jeBbDxHoNtcyw6bcSRTwunzgFVLNgE5AwPzz2rnpLr4dXiafBoHg2+1HU7hgJLQ3cybOOQGDHPPfGMcnFfSKIkaBEVVUdAowBTUghjdnjijR2+8yqAT9aSkHMQaXaR2Gk2dnFCII7eBIliDl9gVQAu48nGMZ71booqSQooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKAILwA2NwD0Mbfyql8PiT8O/Duf+gfD/AOgCr13/AMeU/wD1zb+VUPh7/wAk68O/9g+H/wBBFZ1djehuzpaKKKwOg5rSP+Sqa9/2C7P/ANGT12VcbpH/ACVTXv8AsF2f/oyeuyr0KfwIwluFFFFWSFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAc1pH/Ic8T/9hJP/AEkt62Kx9I/5Dnif/sJJ/wCklvWxXBV+Nm8dgooorMoKKKKACiiigAooooAKiubeO7tZraUZjlRo3HqCMGpaKAOa8OTyS6HbxTnN1ag2tx/10j+Vj9DjcPYitWsrV4n0TU5NaiRnsZwBqCIMmMgYWcDvgYVvYKf4TnTjkSaJJYnV43AZWU5DA9CDXUndXOKceVjqKKKZIUUUUAFFFFABRRRQAUUUUAYfhP8A5A9x/wBhPUP/AErmrcrD8J/8ge4/7Ceof+lc1blDAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigDDuv+R70n/sGXv/o21rcrDuv+R70n/sGXv/o21rcoAKKKKACiiigAooooAKKKKACiiigArLQfbfGdui8x6davLJ7SSnan/jqy5+oqfU9Sj0y2DsjSzSN5cFvH9+aQ9FX+p6AAk4ANWdB0yTTrJ2unWS/upDPdOvQuQBtX/ZUAKPZR3zUTdka0o3dzUooornOoKKKKACiiigAooooAKKKKACiiigDy748/8iPZf9hKP/0VLRR8ef8AkR7L/sJR/wDoqWiu2h8BjPc6z4e/8k68O/8AYPh/9BFdLXNfD3/knXh3/sHw/wDoIrpa5Jbs1WwVxkP/ACVLWv8AsF2n/oc1dnXGQ/8AJUta/wCwXaf+hzVdLczq/CdHRRRWxyhRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQBDd/8eU//AFzb+VUPh7/yTrw7/wBg+H/0EVfu/wDjyn/65t/KqHw9/wCSdeHf+wfD/wCgis6uxvQ3Z0tFFFYHQc1pH/JVNe/7Bdn/AOjJ67KuN0j/AJKpr3/YLs//AEZPXZV6FP4EYS3CiiirJCiiigAooooAKKKKACiiigAooooAKKKKAOa0j/kOeJ/+wkn/AKSW9bFY+kf8hzxP/wBhJP8A0kt62K4Kvxs3jsFFFFZlBRRRQAUUUUAFFFFABRRRQAVzsvh+506R5tAnihRiWewnz5DE9ShHMRPsCv8As5Oa6KiqUmthOKejOZOsXVtxqGh6jAR1eCP7Sh+nl5bH1UH2qM+KLEEg2mtfhol4f/aVdVRV+1fYy9jE5X/hKbH/AJ89b/8ABJef/GqP+Epsf+fPW/8AwSXn/wAarqqKPa+QexXc5X/hKbH/AJ89b/8ABJef/GqP+Epsf+fPW/8AwSXn/wAarqqKPa+QexXc5X/hKbH/AJ89b/8ABJef/GqP+Epsf+fPW/8AwSXn/wAarqqKPa+QexXc5X/hKbH/AJ89b/8ABJef/GqP+Epsf+fPW/8AwSXn/wAarqqKPa+QexXc888MeJLKHSZ1a11gk6jfNlNHu3GDdSkciMjPPI6g5BwQRWz/AMJTY/8APnrf/gkvP/jVXPCH/IFuP+wpqP8A6WTVvU3U12F7FHK/8JTY/wDPnrf/AIJLz/41R/wlNj/z563/AOCS8/8AjVdVRS9r5D9iu5yv/CU2P/Pnrf8A4JLz/wCNUf8ACU2P/Pnrf/gkvP8A41XVUUe18g9iu5yv/CU2P/Pnrf8A4JLz/wCNUf8ACU2P/Pnrf/gkvP8A41XVUUe18g9iu5yv/CU2P/Pnrf8A4JLz/wCNUf8ACU2P/Pnrf/gkvP8A41XVUUe18g9iu5yv/CU2P/Pnrf8A4JLz/wCNUf8ACU2P/Pnrf/gkvP8A41XVUUe18g9iu5yv/CU2P/Pnrf8A4JLz/wCNUf8ACU2P/Pnrf/gkvP8A41XVUUe18g9iu5yv/CU2P/Pnrf8A4JLz/wCNUf8ACU2P/Pnrf/gkvP8A41XVUUe18g9iu5yv/CU2P/Pnrf8A4JLz/wCNUf8ACU2P/Pnrf/gkvP8A41XVUUe18g9iu5yv/CU2P/Pnrf8A4JLz/wCNUf8ACU2P/Pnrf/gkvP8A41XVUUe18g9iu5yv/CU2P/Pnrf8A4JLz/wCNUf8ACU2P/Pnrf/gkvP8A41XVUUe18g9iu5yv/CU2P/Pnrf8A4JLz/wCNUf8ACU2P/Pnrf/gkvP8A41XVUUe18g9iu555c+JLI+NNLmFrrG1dOvFIOj3YYkyWxGF8vJHynJAwOM4yM7P/AAlNj/z563/4JLz/AONVcu/+R/0f/sF33/o20repup5C9ijlf+Epsf8Anz1v/wAEl5/8ao/4Smx/589b/wDBJef/ABquqope18h+xXc5X/hKbH/nz1v/AMEl5/8AGqP+Epsf+fPW/wDwSXn/AMarqqKPa+QexXc5X/hKbH/nz1v/AMEl5/8AGqP+Epsf+fPW/wDwSXn/AMarqqKPa+QexXc5X/hKbH/nz1v/AMEl5/8AGqP+Epsf+fPW/wDwSXn/AMarqqKPa+QexXc5ZfE1k/C2etE+h0a7H84qeL3V775NP0aSEH/l41BhEg9wikuT7ELn1rpqKHVYKjEyNL0JLK5N9d3DXupMu03DrtCKeqxr0RfzJwMk4Fa9FFZtt7mqSWwUUUUhhRRRQAUUUUAFFFFABRRRQAUUUUAeXfHn/kR7L/sJR/8AoqWij48/8iPZf9hKP/0VLRXbQ+AxnudZ8Pf+SdeHf+wfD/6CK6Wua+Hv/JOvDv8A2D4f/QRXS1yS3ZqtgrjIf+Spa1/2C7T/ANDmrs64yH/kqWtf9gu0/wDQ5quluZ1fhOjooorY5QooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAhu/8Ajyn/AOubfyqh8Pf+SdeHf+wfD/6CKv3f/HlP/wBc2/lVD4e/8k68O/8AYPh/9BFZ1djehuzpaKKKwOg5rSP+Sqa9/wBguz/9GT12VcbpH/JVNe/7Bdn/AOjJ67KvQp/AjCW4UUUVZIUUUUAFFFFABRRRQAUUUUAFFFFABRRRQBzWkf8AIc8T/wDYST/0kt62Kx9I/wCQ54n/AOwkn/pJb1sVwVfjZvHYKKKKzKCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigDB8If8gW4/wCwpqP/AKWTVvVg+EP+QLcf9hTUf/Syat6nLcS2CiiikMKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKAMG7/5H/R/+wXff+jbSt6sG7/5H/R/+wXff+jbSt6m+ggooopDCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooA8u+PP/Ij2X/YSj/8ARUtFHx5/5Eey/wCwlH/6KlortofAYz3Os+Hv/JOvDv8A2D4f/QRXS1zXw9/5J14d/wCwfD/6CK6WuSW7NVsFcZD/AMlS1r/sF2n/AKHNXZ1xkP8AyVLWv+wXaf8Aoc1XS3M6vwnR0UUVscoUUUUAFFFFABRWJd6he3t/Lp2kmOPyMC6vJF3LESMhEX+J8EHnhcjOc4ph8NwSjN1qWr3EndxqMsOf+AxMij8BWsKMpK4r2N6isD/hFNP/AOfnWf8AwdXn/wAdo/4RTT/+fnWf/B1ef/Har6vLuF0b9FYH/CKaf/z86z/4Orz/AOO0f8Ipp/8Az86z/wCDq8/+O0fV5dwujforA/4RTT/+fnWf/B1ef/HaP+EU0/8A5+dZ/wDB1ef/AB2j6vLuF0b9FYH/AAimn/8APzrP/g6vP/jtH/CKaf8A8/Os/wDg6vP/AI7R9Xl3C6N+isD/AIRTT/8An51n/wAHV5/8do/4RTT/APn51n/wdXn/AMdo+ry7hdG/RWB/wimn/wDPzrP/AIOrz/47R/wimn/8/Os/+Dq8/wDjtH1eXcLo36KwP+EU0/8A5+dZ/wDB1ef/AB2j/hFNP/5+dZ/8HV5/8do+ry7hdG/RWB/wimn/APPzrP8A4Orz/wCO0f8ACKaf/wA/Os/+Dq8/+O0fV5dwujforA/4RTT/APn51n/wdXn/AMdo/wCEU0//AJ+dZ/8AB1ef/HaPq8u4XRv0Vgf8Ipp//PzrP/g6vP8A47R/wimn/wDPzrP/AIOrz/47R9Xl3C6N+isD/hFNP/5+dZ/8HV5/8do/4RTT/wDn51n/AMHV5/8AHaPq8u4XRv0Vgf8ACKaf/wA/Os/+Dq8/+O0f8Ipp/wDz86z/AODq8/8AjtH1eXcLo36KwP8AhFNP/wCfnWf/AAdXn/x2j/hFNP8A+fnWf/B1ef8Ax2j6vLuF0b9FYH/CKaf/AM/Os/8Ag6vP/jtH/CKaf/z86z/4Orz/AOO0fV5dwujforA/4RTT/wDn51n/AMHV5/8AHaP+EU0//n51n/wdXn/x2j6vLuF0b9FYH/CKaf8A8/Os/wDg6vP/AI7R/wAIpp//AD86z/4Orz/47R9Xl3C6N+isD/hFNP8A+fnWf/B1ef8Ax2j/AIRTT/8An51n/wAHV5/8do+ry7hdG/RWB/wimn/8/Os/+Dq8/wDjtH/CKaf/AM/Os/8Ag6vP/jtH1eXcLo36KwP+EU0//n51n/wdXn/x2j/hFNP/AOfnWf8AwdXn/wAdo+ry7hdG/RWB/wAIpp//AD86z/4Orz/47R/wimn/APPzrP8A4Orz/wCO0fV5dwujforA/wCEU0//AJ+dZ/8AB1ef/HaP+EU0/wD5+dZ/8HV5/wDHaPq8u4XRv0Vgf8Ipp/8Az86z/wCDq8/+O0f8Ipp//PzrP/g6vP8A47R9Xl3C6N+isD/hFNP/AOfnWf8AwdXn/wAdo/4RTT/+fnWf/B1ef/HaPq8u4XRv0Vgf8Ipp/wDz86z/AODq8/8AjtH/AAimn/8APzrP/g6vP/jtH1eXcLo36KwP+EU0/wD5+dZ/8HV5/wDHaP8AhFNP/wCfnWf/AAdXn/x2j6vLuF0b9FYI8NW8S5ttR1iCQdJDqU02P+Aysyn8RU2m6hdR37aVqhRrkIZILhF2rcRggE4/hdcjI6cgjqQJnSlFXC9zYooorIYUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAQ3f/AB5T/wDXNv5VQ+Hv/JOvDv8A2D4f/QRV+7/48p/+ubfyqh8Pf+SdeHf+wfD/AOgis6uxvQ3Z0tFFFYHQc1pH/JVNe/7Bdn/6Mnrsq43SP+Sqa9/2C7P/ANGT12VehT+BGEtwoooqyQooooAKKKKACiiigAooooAKKKKACiiigDmtI/5Dnif/ALCSf+klvWxWPpH/ACHPE/8A2Ek/9JLetiuCr8bN47BRRRWZQUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUVxWraPper+OL0alptneiLTbXy/tMCybMyXGcbgcZwPyFH/AAhvhf8A6FvR/wDwBi/+JrZUrq9ylBtXO1oriv8AhDfC/wD0Lej/APgDF/8AE0f8Ib4X/wChb0f/AMAYv/iafsfMfs2drRXFf8Ib4X/6FvR//AGL/wCJo/4Q3wv/ANC3o/8A4Axf/E0ex8w9mztaa4YowQgPj5SRkA1xn/CG+F/+hb0f/wAAYv8A4mj/AIQ3wv8A9C3o/wD4Axf/ABNHsfMPZsk+Gt5qN94du5dRsls3/tO82xBtxyZ3dufZmZf+AZ78djXFf8Ib4X/6FvR//AGL/wCJo/4Q3wv/ANC3o/8A4Axf/E03STd7gqbO1oriv+EN8L/9C3o//gDF/wDE0f8ACG+F/wDoW9H/APAGL/4ml7HzD2bO1oriv+EN8L/9C3o//gDF/wDE0f8ACG+F/wDoW9H/APAGL/4mj2PmHs2drRXFf8Ib4X/6FvR//AGL/wCJqjqnhrQdOjsrqy0TTba4TUrLZLBaojrm5iBwQMjIJH40ex8wcGlc9DooorAgKKKKACiiigAooooAKKKKACiiigAooooAKKKKAMG7/wCR/wBH/wCwXff+jbSt6sG7/wCR/wBH/wCwXff+jbSt6m+ggooopDCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooA8u+PP8AyI9l/wBhKP8A9FS0UfHn/kR7L/sJR/8AoqWiu2h8BjPc6z4e/wDJOvDv/YPh/wDQRXS1zXw9/wCSdeHf+wfD/wCgiulrkluzVbBXGQ/8lS1r/sF2n/oc1dnXGQ/8lS1r/sF2n/oc1XS3M6vwnR0UUVscoUUUUAFFFFAGF4TAbwtp1wf9Zdwi7kPq8v7xj+bGtmsbwj/yJehf9g63/wDRa1s16S2Ie4UUUUxBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABWN4iHlRadeLxLb6hbhD7SSCFh/3zIa2ax/E3/IKg/wCwjY/+lUVTP4WNbm5RRRXnFhRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQBDd/8AHlP/ANc2/lVD4e/8k68O/wDYPh/9BFX7v/jyn/65t/KqHw9/5J14d/7B8P8A6CKzq7G9DdnS0UUVgdBzWkf8lU17/sF2f/oyeuyrjdI/5Kpr3/YLs/8A0ZPXZV6FP4EYS3CiiirJCiiigAooooAKKKKACiiigAooooAKKKKAOa0j/kOeJ/8AsJJ/6SW9bFY+kf8AIc8T/wDYST/0kt62K4Kvxs3jsFFFFZlBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFAHLzf8jzqP/YNs/wD0bc1eqjN/yPOo/wDYNs//AEbc1errh8KNofCFFFFUWFFFFABRRRQAUUUUAFFFFABRRRQAVleIP+PG0/7CVj/6VRVq1leIP+PG0/7CVj/6VRUmTL4WdfRRRXGYBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQBm6poNhq88E919qWaBXSOS2vJbdgrlSwzGykglF4PoKp/8ACIaZ/wA/Wt/+Dy9/+O1vUU7sVjB/4RDTP+frW/8AweXv/wAdo/4RDTP+frW//B5e/wDx2t6ijmfcLI4vxT4dtdM8I61f2l7rUdza2E80L/21eNtdY2KnBlIPIHBGK7SsHxx/yIHiT/sF3X/opq3qbd0AUUUVIwooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigDy748/8iPZf9hKP/wBFS0UfHn/kR7L/ALCUf/oqWiu2h8BjPc6z4e/8k68O/wDYPh/9BFdLXNfD3/knXh3/ALB8P/oIrpa5Jbs1WwVxkP8AyVLWv+wXaf8Aoc1dnXGQ/wDJUta/7Bdp/wChzVdLczq/CdHRRRWxyhRRRQAUUUUAYXhH/kS9C/7B1v8A+i1qLxZ4ssPB2lx6jqMNzJA8wh/0dAxDEEjOSOODUvhH/kS9C/7B1v8A+i1rJ+KGl/2t8OtXiC5khi+0J7GMhj+gI/GvR+zoT1Ni58S6fa+FP+EkkaT7B9mW5GANxVgCBjOMnIHXrWf4Q8d6R42W7OlpcobUr5i3CBT82cEYJ/umvHdT8TNcfAbR9KjYtcz3ZtHUdSkbbwP1irV8E2o+H/xau9DuJMW1xp4YOTwWVBIW/ArIKnn1RXLoz0eP4iaPN4j1PRIobx7nTYpJrhxGuwKgG7B3Zzk46dav+EvFun+MtKl1HTY7hIY5zAwnQK24KrdieMMK8r+FXnXNt418Wso8+VZPLZhkbiGkYHPXkpWz4O8Yave/CXxBrcskC31nJN5LRwIijbEjDKgYPJPWhS7icT1mivFdH8SfEvxV4PfVdNurC2SzEnmSvGvmXTLliFUqVAAwO2SDz6dF4N+JEupfDnUtf1eNDcaYWSTyhtE2FBXjsSWA9KpSTE4s9Irm/CHjfS/GsF1LpkV0i2zKr/aEVSSQSMYJ9K4Dw/r/AMT/ABDp0vie0l006crMY9OeMAzKp5CkKW7EAluoqj8EdSt9I8L+J9SuiRBbFJXx1wFY4HvS5tUPl0Z7jRXjGl+IPif44t7jWdBmsNO05HZYYJFUmXHYFlOT2zlRmtHRviHrXiPwRrwiRLPxLpEfmOFjBVwpycK2ecKwI9ceuAc6Fys9WoryS5+Jl+vwbtteimjGsyzi0LeWCBIGJJ29OUGfxpLf4mX5+DU+vSzxnWY7g2gfywAZCwIO3pwhz+FHOg5WeuU13SONpJGVEUEszHAAHcmsXwfPqd34R0261iUSX9xCJpCECY3cqMDphSBWV448Iaj4wlsbNNYksdJXcb2KLO6fkbRjpjg9fXoaq+lxW1LXhbx3pHjC91G20tbk/YWAeWRFCSAlgCmCSR8p6gdRXT14r8DYEtdf8V28efLieNFycnAaQCu615/Gi+NtJXR4kbw+Qn25iYsg7zu+8d33cdKmMtLsbWtjsK5XVfHdjpfiM6ENM1e9vREszCytvNCoTjJwcgZ6nHeuqrz6y/5LzqX/AGA1/wDRiU2JHWa5r1voUVsZY5J7i7nS3t7eLG+R2PbPQAZJJ6AVpySJDE8srqkaAszMcBQOpJrg7eU638aLsSfNBoFgqRL2E02CW+u04/Ct3xP4Ti8VPaQ3t/dR6dCWaezgfYtyeNocjnAwePfqMUXHYn8NeKNO8V2dzd6YZWt7e5e23uoAcqAdy8n5SGGM4PtW1Xnfweijg0DW4YkCRx63cKijoAFQAV6JTTuhNWYVk+I/EmneFdHl1PU5SsKcKiYLyN2VQSMmtavKfih4Otv+Ed17xFfXlze3aqv2SOV8RWql0BCL64zkn8qUm0tASTep6dFdi401LyCJ3EkIlSM4DHIyB6A1V0HW7XxDpEOo2m9UfKvHIMPE4OGRh2INP0T/AJAGnf8AXrF/6CK5DQJTo/xZ8Q6InFrqFumqRJ2WThJCP948/hRcLHfUUVQ1zUho+gahqRAP2S2kmAPcqpIH6UxENlr1vqGu6hpltHI/2AIJ5+PLEjc+WD3YDk+mRXPT/FDR1nuFsdO1nVLe3YrNd2FmZIUI65bI6eop3w70sH4c2/2l3M+qpJdXUoOGdpSTuz67SPyrSQaH8O/CARpfs+nWanbvILuSScD+8xJpa2K0uaeja1p/iDS4tS0y4We1l+6wGCCOoIPII9Kv1wnwq027tPDl7f3dubUapfSXsNt08qNsbRjt0/LFd3TTuhPRlLV9XsdC0ybUdSuFgtYRl3bn6AAckn0rlIvilo3mwG907WtNtLhgsV7e2Rjgcnphsnj3xXRa/wCHLDxLbW1vqIkaG3uEuVRGwGZc4Deo5PFcr8VNVgk0A+F7aIXesasVjt7VeSo3Al29AMdT/IHCbaGrHWa94h03w3pZ1DUp9kO4IgUbmkY9FUDqTWRo/j/S9W1aPS5rPU9LvplLQQ6lbGEzAddvJBrnNUs2X4h/D/Qrp/OjsbSSViejyJHgN+aA1c+L3+jeH9L1WMYurDU4ZYnHUdcj6Hj8qTb3GktjpvEfizTfDCW4vBPNc3LFbe0tY/MmmI67VqHw9400zxFeT2MUV5ZahAoeSyv4fKmC/wB7HORyO/cetc9ZAX3x21J5vm/s/SkjgB/hLlSSPf5mH40vjT/QfiV4I1CEbZppprWUj+NCFwD9NxP40Xe4rLY622163n8RXmhvHJDeW8SzpvxiaJuN6ewPB961a4H4jynRdR8M+JYvle11AWsxH8UEoIYH1+7x7mu+qkxNBWVq2vW+lXmnWTRyT3eoTeVDDFjOAMs5z0VRyTWrXAeHpTrnxV8SajJzHpMUen2wP8O7LSH67l6+hobBI3Nf8a6Z4fv4dOaC9v8AUZl3rZWEHmy7f7xGQAPxqbw74t0zxN9ojtPtEF1akLcWl1EYpoiem5T/AEpLvT9N0G81LxPDpt1c6hPGiTLbBpZZAMABEJwO2cY6ZrjfBWoLr/xQ1vWbiNtNultFtU0y4UrOUBU+Y4IA7AcZxn6ZV3cdlY9QoopskazRPG+drqVOCQcH3HIqiTjLn4n6NFc3KWtjq+oW1qxW4vbKzMkERHXLZHT1ANdHDrdrfeH/AO2dL3X9u0JliWH70mM/KAcfNwRg454rntTvbX4c+HobDRfDmo30ISV0jtYzIkfOT5jnJGd3vwD6VF8JbaC28AWvkX0N35sryuYc7Y2Y52YOCMccEd6hN3sU0rXN9PEtpceFk8QWUF3fWzxiRIbSLzJmycEBfUHOR2wai8MeLLLxXDeSWlteW7Wk5gmiu4hG6uByMZOMe9c/4JlOl+N/FnhocW8U6X1svZRKAXA9ACRx9aZ8NP8AkK+Nv+w9P/M0JvQLHoNY/ib/AJBUH/YRsf8A0qirYrH8Tf8AIKg/7CNj/wClUVOXwsS3NyiiivOLCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigCG7/wCPKf8A65t/KqHw9/5J14d/7B8P/oIq/d/8eU//AFzb+VUPh7/yTrw7/wBg+H/0EVnV2N6G7OlooorA6DmtI/5Kpr3/AGC7P/0ZPXZVwtjfWll8U9bN3dQQBtLs9vmyBc/PN0zXVf27o/8A0FbH/wACE/xr0KfwIwluaFFZ/wDbuj/9BWx/8CE/xo/t3R/+grY/+BCf41ZJoUVn/wBu6P8A9BWx/wDAhP8AGj+3dH/6Ctj/AOBCf40AaFFZ/wDbuj/9BWx/8CE/xo/t3R/+grY/+BCf40AaFFZ/9u6P/wBBWx/8CE/xo/t3R/8AoK2P/gQn+NAGhRWf/buj/wDQVsf/AAIT/Gj+3dH/AOgrY/8AgQn+NAGhSAhgCCCD0Iqh/buj/wDQVsf/AAIT/Gvj6Lxx4j8MeI9RfRNZuIIjdyt5QbfE3znnYcqfrigD7Ror578N/tISpsh8S6QHHQ3NicH8Y2OD+DD6V694c+IPhbxUFXStYt5J2/5d5D5cv/fDYJ/DIoAXSP8AkOeJ/wDsJJ/6SW9bFY+kf8hzxP8A9hJP/SS3rYrgq/GzeOwUUUVmUFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAcvN/wAjzqP/AGDbP/0bc1meOp5rXwLrU9vLJDNHauySRsVZT6gjpWnN/wAjzqP/AGDbP/0bc1kfEH/kn2u/9ej/AMq64bI1XwHlHwh8T6sfGUVlquo3dzDqNq5gFxOzjcpJyMnj7jiq/wAVvFGsP43vbfS9SvLa106GOOQW87RqWOCSQD1y4H4VmwZ0Pwz4F8VIDi2u5opiO6iUsB+I8yoZon1PwJ4n8TzqfMv9WjRSewyzsP8Ax9fyrbrc57vl5T1LW/G8/hX4UaHfI3n6pe2UCRNKd3zGIFnbPXH6kisLTPh34y8Q6bFq+qeMr20u7hBLHAC7bQeRnDKF+gHFZHxMtpn+GXgW6UHyYrNI3PYM0MZH/oLV7jpGoW2q6PaX1m6vbzxK6FT0GOn1HT8KnZGqXNKzPMvAfi3XtM8ZT+CfFM5ubhci3uWOWJC7gC38QK8gnnt9MC8g17xN8XdY0O08S6hp0SM7psmcqoAXgKGGOtWr2ZNa/aKtPsDCRbVlWV05GUQlufb7v1qfw1/ycNrf+5L/ACSmTvp5lXXU8c/C+a01WTxDNrOmvKI5EuHZhnGdpDE7cgHBB7fntfFvxDcN4E0TVdIvbm1W8nSQPDK0bFWjZsEg/p7Vd+OF3BD4ES3d182e6QRr3OMkn8P6iuR8fW0tp8F/CMMwKyBoyQeozGxA/I0LWzCXu8yRs2Hwz8UX2nWt2PiDqSefEku3Mp27gDjPme9dBcaTq3g/4X67HPr91f3qpJPHeMzK8fyqAASxIxgnr3rndM+Df2zSrO5/4SjUY/OgSTYq8LlQcD5veul1rQf+Ea+D+raX9slvPKtZT50o+ZtxJ9ffFK5SjZXt+JJ8JNRvdU8CQ3OoXc11OZ5AZJnLsQDwMmu5rzz4K/8AJOoP+viX+deh0nuaQ+FBWV4g/wCPG0/7CVj/AOlUVatZXiD/AI8bT/sJWP8A6VRVLHL4WdfRRRXGYBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFAGD44/5EDxJ/2C7r/wBFNW9WD44/5EDxJ/2C7r/0U1b1V0F1CiiipGFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAeXfHn/AJEey/7CUf8A6Kloo+PP/Ij2X/YSj/8ARUtFdtD4DGe51nw9/wCSdeHf+wfD/wCgiulrmvh7/wAk68O/9g+H/wBBFdLXJLdmq2CuMh/5KlrX/YLtP/Q5q7OuMh/5KlrX/YLtP/Q5quluZ1fhOjooorY5QooooAKKKKAMLwj/AMiXoX/YOt//AEWtas8EdzbywSrujlQo49QRg1leEf8AkS9C/wCwdb/+i1rZr0lsQ9z5a8FaFc3PxNsNAuCzQ6ffySyIfugx/eP4+Wo/Ku2+PNjLZ32j69bEo7RyWcjr2GCQPxDPXtoRA24KoY98c0rKrjDKCPcZqOTSxXPrc828CaV/ZfwTk3LiS7s7i5f33Kdv/joWuS8Bf8kF8Wf9dbj/ANEx17tgYxgY6YpAiBSoVQp6gCnyi5jzX4T/APJIj9bn+ZrkPhvo8+v/AAi8U6ZbDNxNN+7XONzKqMB+JGPxr3oKqjCqAPQChUVBhVCj2GKOXYOY8P8AAvxLt/DnhFPDVzpt8+t2jyRW9skOTKzMWAPcEFiDx0Heqfws0abWvAfjLSIyEuJwsaZOBvCtgH2yBXvPkQ+d53lJ5uMb9o3Y+tPVFXO1QM+go5e4+bseF+B/iTZ+BfDsvh7xBp19BfWUjlEWMfOGJODkjByTz0xitr4QaHfzXuueKdStTbx6s7eTC4+8rMWY4P8ADyAPXmvVpbaCZlaWGORl+6WQEj6VLQog5HzNpehXafE238EuSdPtdXa68s/xIAGyfrGo/OjUtBu5PibP4JTI0+41YXXljshXdkfSNj+VfS+xd27aN3rjml2Lu3bRu9cc0vZj5wVVRQqgKqjAA6AUtFFaGZ418Fv+Rp8Y/wDXdP8A0OWu51zx3baH4y0rw3JZTSzagEKTKwCpucryPwrqlRVJKqBnrgUFFLBioJHQkVKVlYpu7uOrz6y/5LzqX/YDX/0Yleg1wOo6H4mtPiTN4j0i1sbi3nsktGW4nKEfMGJAA9qJAiPQVOn/ABo8UW0nH2+zgu4c/wASqAh/8eJ/KvQq5fxPoV3Pq+keINJjV9R0+TY8RYL59u/Dpk8ZGdwz3zXUULQTPPvhJ/yB9f8A+w7c/wAkrqrfxJp1z4ou/D0bSf2hawieUFPl2nbjB/4EK1goXO0AZ54FZsWjIniSfWpJ2eV7dbaKPaAsSA7m+pJxyfQChKysDdzTrjfit/yTLWv+ucf/AKMSuypCAwwQCPQ02roE7Mo6J/yANO/69Yv/AEEVxlmpv/jrqM8fMenaQlvIR2d2DgfkT+Vd/KzxwO0URkdVJWMEDcccDJ4Fc74M0C40awurvUijavqc7XV4UOQrHogPoo4/Ok1sNMvaF4k07xGL06e0jfY7hraXem3Djrj1FM8X2MmpeDdZs4QWlls5VQDu204H51Po+jJpT38xnaee+uWuJZGUL2CqoA7BVUe/J71p0+mouuhx3grW7WD4UaZqsz4trTT/AN6QOQIgVb8flNebaZ428M+ItcOveNNTP7iQ/YNJWCR4oB2dsLhmP+ewHqXhjQbrw5qer2CRq2iXEv2u0bcMxM/34ivXGRkduT3rpfJi/wCeSf8AfIqbNlXSMLTPGuhazol/q+nXTz2dirNOwiZSNq7jgMBnitHRdYtNf0e31SxLm2uAWjLrtOASOn1FWZoS1tLHAywyOhCvsB2kjg4749KraLpMOh6Na6bA7vHbpt3v95znJY+5JJ/Gq1J0Oe+I/jP/AIQvw59pgVXvrl/JtlcZVTjJY+wHb1IrivCXi/wD4fMuo32vy6hr10M3V9Laykk/3U+X5VH64+gHsbIr/eUN9Rmk8mL/AJ5J/wB8ik073GmrWPOvGdwttq/hTx7axy3GmWwYXLRoSywTJ8sm3rgZJP1FU/Fmu6d8QbrRvDfh6c3yteR3V7NGh2QwpnO4kDk54HtjvXqeBjGBjpimQ28NuCIIY4gTkhFC5/KjlC553rlyng34op4iv1dNH1OyFpNchSywzKwK7sdAQAPxPpUUmo23j34k6HJo7NcaXoYknuLsKRG0rgBUBPUjAP5+leluiyIUdQysMFWGQaSKKOCMRxRpGg6KigAfhRYLnAfFtTeaRoukRcz3+rQxoo9BnJ/DIrq7jxJp1t4otPD0jSf2hdQmeJQny7Ruzk/8BNZzaFd6l4+XWdQjVLHTITFp0e4EvI4/eSkduMKB+NatxoyXXiGy1aWdj9jidIYQoADvwzk9T8vAHuaNb3DTY06898DqbD4heN9Ol4kkuYryPP8AEjhjkfTIFehVy+s6Fdr4v0rxJpUavOgNnfRFgvm27HOQT3Vucd6H3Eihe+OJPDvjK707xKsVppM0avp16sbbWOPnV2yRuz9OnuKx7G8h8ZfFyx1vRFeTTNKtJIri92FUmdgwCKT1xuz+B9s+mSwxTxmOaNJEPVXUEH8DSpGkSBI0VEHRVGAKLMdzK07xJp2qa3qWkWzSG704qLgMmAN3TB71pXPn/ZZvsvl/aNjeV5mdu7HGcds1RstGS11zUdVedpZ7xY4wCoAijQHCj15ZiSfX2rTpoR57o3xT0yLS5I/Fcy6XrVqzJdWpicZIJwUHOQRjv+mDT/hZYXUdlrWrTWslnb6tqEl1a20g2lIz0JHbOf0HbFd1Ja280iySwRO6fdZkBI+hpl9Jcw2FxJZwC4uVjYxRFgod8cAk9BmlbuO/Y4Xwypvvi94v1GPmC3igs93q+0Fh+BU0vw0/5Cvjb/sPT/zNauhaFqXhjwfLHZpBea7cSG6uGmcqks7sC+SOwHA+nvUPgDQNX0X+3LjWI7eOfUtQe8CQSF1XdyRkgd6SWqG3udlWP4m/5BUH/YRsf/SqKtisfxN/yCoP+wjY/wDpVFTl8LJW5uUUUV5xYUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAQ3f/HlP/1zb+VUPh7/AMk68O/9g+H/ANBFX7v/AI8p/wDrm38qofD3/knXh3/sHw/+gis6uxvQ3Z0tFFFYHQeS+K/hzafEL4m30V3fz2gs9NtmUxIG3bnl65/3ar/8M16P/wBDBff9+UrvdI/5Kpr3/YLs/wD0ZPXZV6FP4EYS3PEP+Ga9H/6GC+/78pR/wzXo/wD0MF9/35Svb6Ksk8Q/4Zr0f/oYL7/vylH/AAzXo/8A0MF9/wB+Ur2+igDxD/hmvR/+hgvv+/KUf8M16P8A9DBff9+Ur2+igDxD/hmvR/8AoYL7/vylH/DNej/9DBff9+Ur2+igDxD/AIZr0f8A6GC+/wC/KUf8M16P/wBDBff9+Ur2+igDxD/hmvR/+hgvv+/KV88ajaiy1O7tFYssMzxhj1OGIz+lfe1YGk+CPDOiXL3Wn6LaR3TuZGuGTfJuJyTubJHPoaAPkzQPhl4w8SbWsNDuVgbpPcDyY8eoLYz+Ga9R0D9m58pL4g1wLjkw2CZP/fxh/wCy19A0UAcX4O0iDQp9f062luZYoNQRVe5maVyPstueWP1/LA7V09Y+kf8AIc8T/wDYST/0kt62K4Kvxs3jsFFFFZlBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFAHLzf8jzqP/YNs/wD0bc1H4h0n+3vD1/pXn+R9rhaLzdm7bnvjIz+dSTf8jzqP/YNs/wD0bc1errh8KNoK8Tz+X4YRzfDiHwk+qZaGfz0u/s/RtxP3N3oxHWib4YRv8OYfCUeqbCk/nPd/Z8723E/c3ccEDr2r0Ciruw5ImF/witjc+Dbbw1qIF3bQ2sduz42ElFADjk7TxnrXn4+DWq2Pm2+j+NL20sJDloQrDj32uA35CvXaKLsHCL3OR8E/D3S/BUcklu73V9MNst1IADj+6o/hGee/16VzmtfCK61PxTfa5Z+KZtPlunLbYbY7lBA43CQZ6V6jRRdg4RtY8w034M2o1OK91/XbzWjEQVjlUqpx2bLMSPbIro/HvgkeN9ItbAX/ANhEE/m7/J8zPykYxuGOtdZRRdhyRtY8mT4P65GionxA1FUUYVVhcAD0H72uj07wHeWvg7VtAvfEU9+9/uC3U0RJiBUDABc56Z6jrXbUUXYKnFHkNr8FtVsYRDaeOry3iByEit2Rc/QS12Xgrwhf+FTffbvEVzq/2ny9nnqw8rbuzjLt13D06V1lFF2CpxTugrK8Qf8AHjaf9hKx/wDSqKtWsrxB/wAeNp/2ErH/ANKoqljl8LOvooorjMAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigDB8cf8iB4k/wCwXdf+imrerB8cf8iB4k/7Bd1/6Kat6q6C6hRRRUjCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKAPLvjz/wAiPZf9hKP/ANFS0UfHn/kR7L/sJR/+ipaK7aHwGM9zrPh7/wAk68O/9g+H/wBBFdLXNfD3/knXh3/sHw/+giulrkluzVbBXGQ/8lS1r/sF2n/oc1dnXGQ/8lS1r/sF2n/oc1XS3M6vwnR0UUVscoUUUUAFFFFAGF4R/wCRL0L/ALB1v/6LWtmsTw84son0Kb5Z7D5Ygf8Alpb5/duPUYwp9GU+1bdejF3SaIe4UUUVQgooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigArH8Tf8gqD/ALCNj/6VRVsVi6q4v9W0/SYvmKTJeXOP+WccZ3Jn3aQLgdwrelRN2ixrc3qKKK88sKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKAIbv/jyn/65t/KqHw9/5J14d/7B8P8A6CKv3f8Ax5T/APXNv5VQ+Hv/ACTrw7/2D4f/AEEVnV2N6G7OlooorA6DmtI/5Kpr3/YLs/8A0ZPXZVxukf8AJVNe/wCwXZ/+jJ67KvQp/AjCW4UUUVZIUUUUAFFFFABRRRQAUUUUAFFFFABRRRQBzWkf8hzxP/2Ek/8ASS3rYrH0j/kOeJ/+wkn/AKSW9bFcFX42bx2CiiisygooooAKKKKACiiigAooooAKKKKACiiigDl5v+R51H/sG2f/AKNuavVBqehapPrkmpabqdnbebbRW8kdzZNP9xpGBBWVMf6w8c9BUH9i+KP+g7o//gol/wDkmuiM4pbmkZpKxeoqj/Yvij/oO6P/AOCiX/5Jo/sXxR/0HdH/APBRL/8AJNV7SPcftEXqKo/2L4o/6Duj/wDgol/+SaP7F8Uf9B3R/wDwUS//ACTR7SPcPaIvUVR/sXxR/wBB3R//AAUS/wDyTR/Yvij/AKDuj/8Agol/+SaPaR7h7RF6isLR4PFGq2Ulx/a+jxbLq4t9v9lStnypniz/AMfA67M47Zxz1q//AGL4o/6Duj/+CiX/AOSaOePcPaIvUVR/sXxR/wBB3R//AAUS/wDyTR/Yvij/AKDuj/8Agol/+SaPaR7h7RF6iqP9i+KP+g7o/wD4KJf/AJJo/sXxR/0HdH/8FEv/AMk0e0j3D2iL1ZXiD/jxtP8AsJWP/pVFU/8AYvij/oO6P/4KJf8A5JpknhvXrt7dL3WtNe3juYbh0g0x43by5FkADGdgMlQM4NDqR7ic01Y6uiiiuUzCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKAMHxx/yIHiT/sF3X/opq3q57x3IkfgDxDvdV3aZcquTjJ8puK345ElQPG6uh6MpyKroLqOoooqRhRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFAHl3x5/5Eey/7CUf/oqWij48/wDIj2X/AGEo/wD0VLRXbQ+AxnudZ8Pf+SdeHf8AsHw/+giulrmvh7/yTrw7/wBg+H/0EV0tckt2arYK4yH/AJKlrX/YLtP/AEOauzrjIf8AkqWtf9gu0/8AQ5quluZ1fhOjooorY5QooooAKKKKAKOo6Ta6osfnq6zREtDPExSSInqVYcj3HQ9waonTvEMQ2W+u2kijo15p/mP+JjkjH6VuUVUZyjswMH7F4p/6DWjf+CmX/wCSaPsXin/oNaN/4KZf/kmt6iq9rPuI5m+bxDpto9zda9oyRLgcaRKSxJwFAFxkkngAck1Hp0Xjq7JkurnRLOA8xiSwkaXH+0on2r9AxrRt4xqfiq4mlG6DSgsUKHoJ3Xc7/UIyAem5/WugqZV5rRM6aVGLV5HPf2d4p/6Dejf+CiX/AOSaP7O8U/8AQb0b/wAFEv8A8k10NFR7ep3NfYw7HPf2d4p/6Dejf+CiX/5Jo/s7xT/0G9G/8FEv/wAk10NFHt6ncPYw7HPf2d4p/wCg3o3/AIKJf/kmj+zvFP8A0G9G/wDBRL/8k10NFHt6ncPYw7HPf2d4p/6Dejf+CiX/AOSaP7O8U/8AQb0b/wAFEv8A8k10NFHt6ncPYw7HPf2d4p/6Dejf+CiX/wCSaP7O8U/9BvRv/BRL/wDJNdDRR7ep3D2MOxz39neKf+g3o3/gol/+SaP7O8U/9BvRv/BRL/8AJNdDRR7ep3D2MOxz39neKf8AoN6N/wCCiX/5Jo/s7xT/ANBvRv8AwUS//JNdDRR7ep3D2MOxz39neKf+g3o3/gol/wDkmj+zvFP/AEG9G/8ABRL/APJNdDRR7ep3D2MOxz39neKf+g3o3/gol/8Akmj+zvFP/Qb0b/wUS/8AyTXQ0Ue3qdw9jDsc9/Z3in/oN6N/4KJf/kmj+zvFP/Qb0b/wUS//ACTXQ0Ue3qdw9jDsc9/Z3in/AKDejf8Agol/+SaP7O8U/wDQb0b/AMFEv/yTXQ0Ue3qdw9jDsc9/Z3in/oN6N/4KJf8A5Jo/s7xT/wBBvRv/AAUS/wDyTXQ0Ue3qdw9jDsc9/Z3in/oN6N/4KJf/AJJo/s7xT/0G9G/8FEv/AMk10NFHt6ncPYw7HPf2d4p/6Dejf+CiX/5Jo/s7xT/0G9G/8FEv/wAk10NFHt6ncPYw7HPf2d4p/wCg3o3/AIKJf/kmj+zvFP8A0G9G/wDBRL/8k10NFHt6ncPYw7HPf2d4p/6Dejf+CiX/AOSaP7O8U/8AQb0b/wAFEv8A8k10NFHt6ncPYw7HPf2d4p/6Dejf+CiX/wCSaP7O8U/9BvRv/BRL/wDJNdDRR7ep3D2MOxz39neKf+g3o3/gol/+SaP7O8U/9BvRv/BRL/8AJNdDRR7ep3D2MOxz39neKf8AoN6N/wCCiX/5Jo/s7xT/ANBvRv8AwUS//JNdDRR7ep3D2MOxz39neKf+g3o3/gol/wDkmj+zvFP/AEG9G/8ABRL/APJNdDRR7ep3D2MOxz39neKf+g3o3/gol/8Akmj+zvFP/Qb0b/wUS/8AyTXQ0Ue3qdw9jDsc9/Z3in/oN6N/4KJf/kmj+zvFP/Qb0b/wUS//ACTXQ0Ue3qdw9jDsc9/Z3in/AKDejf8Agol/+SaP7O8U/wDQb0b/AMFEv/yTXQ0Ue3qdw9jDsctd6N4uuYDHF4m021Y/8tIdHYsP++p2H6UzTvDviDS4XjttY0jdI2+WWTS5nklb+8zG5yT/ACHAwK6yik6s5bsPZQ7HP/YPFX/Qb0b/AMFEv/yTR9g8Vf8AQb0b/wAFEv8A8k10FFTzsfsodjn/ALB4q/6Dejf+CiX/AOSaPsHir/oN6N/4KJf/AJJroKKOdh7KHY5/7B4q/wCg3o3/AIKJf/kmj7B4q/6Dejf+CiX/AOSa6CijnYeyh2MAWPikHLaxo7D0Glyrn8ftB/lQmrXVjcR2+t2iW/msEiu4HLwOx6KSQCjHsCMHgBieK36hurWC9tJbW5iWWCVSkiMMhgeooU31JdGLWglFZPh+ab7FNZXMjSz2E7WryMclwAGRj7lGQn3JrWrQ42rOwUUUUAFFFFABRRRQAUUUUAFFFFABRRRQBDd/8eU//XNv5VQ+Hv8AyTrw7/2D4f8A0EVfu/8Ajyn/AOubfyqh8Pf+SdeHf+wfD/6CKzq7G9DdnS0UUVgdBzWkf8lU17/sF2f/AKMnrsq43SP+Sqa9/wBguz/9GT12VehT+BGEtwoooqyQooooAKKKKACiiigAooooAKKKKACiiigDmtI/5Dnif/sJJ/6SW9bFY+kf8hzxP/2Ek/8ASS3rYrgq/GzeOwUUUVmUFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFVNU1CHSNIvdSuA5gs4HuJAgyxVFLHA9cCmBborB/4SDU/+hP1v/v9Zf8AyRR/wkGp/wDQn63/AN/rL/5IosK5vUVg/wDCQan/ANCfrf8A3+sv/kij/hINT/6E/W/+/wBZf/JFFgub1FYP/CQan/0J+t/9/rL/AOSKP+Eg1P8A6E/W/wDv9Zf/ACRRYLh4Q/5Atx/2FNR/9LJq3qxfCttdWuiMt5bPbTS3l3ceS7KzIslxJIoJUlc7WGcE1tUPcFsFFFFIYUUUUAFFFFABRRRQAUUUUAFFFctfzSeIr+ewjkZNItXMdyyMQbqQdYwR0RejY6nK9AwOlOm6kuVEykoq7Lk/im2aZ4NMtbjVJUJVzbBREhHUGRiFyO4BJHpUDav4lPMWh6Xj0l1V1P6QEfrV2KKOCJIoY0jjQBVRBgKB2AHSn16EcJTS11OV15PYzf7W8U/9ALRv/BvL/wDI1H9reKf+gFo3/g3l/wDkatKiq+q0uxPtpmb/AGt4p/6AWjf+DeX/AORqP7W8U/8AQC0b/wAG8v8A8jVpUUfVaXYPbTM3+1vFP/QC0b/wby//ACNR/a3in/oBaN/4N5f/AJGrSoo+q0uwe2mZv9reKf8AoBaN/wCDeX/5Go/tbxT/ANALRv8Awby//I1aVFH1Wl2D20zN/tbxT/0AtG/8G8v/AMjUf2t4p/6AWjf+DeX/AORq0qKPqtLsHtpmb/a3in/oBaN/4N5f/kaj+1vFP/QC0b/wby//ACNWlRR9Vpdg9tM5XxTa+I/FPhi/0W40fR4kuo9okGrSnYwIKtj7OM4IBxnmtCwuvEenafbWNvoOjLBbxLFGP7Xl4VQAP+Xb0FbVFH1ana1g9tMzf7W8U/8AQC0b/wAG8v8A8jUf2t4p/wCgFo3/AIN5f/katKij6rS7B7aZm/2t4p/6AWjf+DeX/wCRqP7W8U/9ALRv/BvL/wDI1aVFH1Wl2D20zN/tbxT/ANALRv8Awby//I1H9reKf+gFo3/g3l/+Rq0qKPqtLsHtpmb/AGt4p/6AWjf+DeX/AORqP7W8U/8AQC0b/wAG8v8A8jVpUUfVaXYPbTM3+1vFP/QC0b/wby//ACNR/a3in/oBaN/4N5f/AJGrSoo+q0uwe2mZv9reKf8AoBaN/wCDeX/5Go/tbxT/ANALRv8Awby//I1aVFH1Wl2D20yguta/EN1zoFs691s9R8x/wEkcY/Wr+neILHUpzaqZLe9C7mtblPLkx3IB4Ye6kj3oqpqGm2upwCK5jztO6ORSVeNuzIw5U+4qJ4ODXu6FRryW5v0Vg6LqdzHeNo2qSCS7RDJb3OABdRggEkDgOuQGA45BGM4G9XnTi4OzOuLUldBRRRUjCiiigDy748/8iPZf9hKP/wBFS0UfHn/kR7L/ALCUf/oqWiu2h8BjPc6z4e/8k68O/wDYPh/9BFdLXNfD3/knXh3/ALB8P/oIrpa5Jbs1WwVxkP8AyVLWv+wXaf8Aoc1dnXGQ/wDJUta/7Bdp/wChzVdLczq/CdHRRRWxyhRRRQAUUUUAFFFFABRRRQBlaB/yFPEn/YST/wBJbetysPQP+Qp4k/7CSf8ApLb1tSSRwxtJK6pGoyzMcAD3NZS3O+n8KHUVFBcwXUfmW80c0ecbo2DDP1FRx6hZTXBt4ru3eYZBjWQFhjrxnNIos0VBPe2lq6JcXMMTP91ZJApb6Z60R3lrLcPbx3MLzJ9+NXBZfqOooAnooooGFFFQC8tTdG1FzCbgcmLeN44z069KBE9FFFAwooooAKKKKACiq0Wo2M9w1vDe28k6/ejSVSw+oBzVmgQUUVDcXltZqrXNxDArHAMrhQT+NAE1FMiminiWWGRJI2GQ6MCD+IpltdW95bLc2txFPA4JWWJwynHHBHFAE1FVrfULK8kaO2vLed0+8sUqsV+oBqzQAUUVFcXMFpEZbiaOGMdXkYKB+JoGS0VFBcQXUQlt5o5oz0eNgwP4im3N5a2SB7q5hgVjgNLIFBP40CJ6KRWV0DowZWGQQcgioJ7+ztZUiuLuCGST7iSSBS30B60AWKKjnnhtoJJ7iVIoY1LPJIwVVA6kk9BTkdJY1kjZXRgGVlOQQehBoAdRRRQMKKiW6t3upLVJ4muI1DPEHBdQehI6gHBpj39nHdLavdwLcN0iaQBz+Gc0CLFFFFAwooJwMmq9tf2d4zra3cE5T7wikDbfrjpQIsUVC11bpdR2rzxLcSKWSIuA7AdSB1IGRU1ABRRTJZY4ImllkWONBlnc4AHqTQNK+iH0UyKWOeJZYZEkjcZV0YEEeoIprXMCTrA08azMMrGXAYj2HXtSHyu9rEtFFFMQUVF9pg8/yPOj87/nnvG7pnpTpZooE3yyJGnTc7ACkPle1h9FNV0aMSKylCMhgeCPXNMt7q3vIvNtZ4p4843xOGGfqKA5Xa9iWiovtVv5/kefF53/ADz3jd69KfJJHDE0srqkaAszscBR6k0ByvsOoqJLmCS3Fwk0bQFd3mhgVx656Ypltf2d4WFrdwTleWEUgbH1xTE1Z2ZYooooAwNH/wCQz4k/7CKf+ktvWxWPo/8AyGfEn/YRT/0lt62K2Wx59T4mFFFFBIUUUUAFFFFABRRRQAUUUUAFFFFAEN3/AMeU/wD1zb+VUPh7/wAk68O/9g+H/wBBFX7v/jyn/wCubfyqh8Pf+SdeHf8AsHw/+gis6uxvQ3Z0tFFFYHQc1pH/ACVTXv8AsF2f/oyeuyrjdI/5Kpr3/YLs/wD0ZPXZV6FP4EYS3CiiirJCiiigAooooAKKKKACiiigAooooAKKKKAOa0j/AJDnif8A7CSf+klvWxWPpH/Ic8T/APYST/0kt62K4Kvxs3jsFFFFZlBRQSACScAVhSeMNDVmEV1Ld7ThjY2st0AfTMSsM00m9hNpbm7RXPf8JppP/PHWf/BJef8Axqj/AITTSf8AnjrP/gkvP/jVV7OfZi5o9zoaK57/AITTSf8AnjrP/gkvP/jVH/CaaT/zx1n/AMEl5/8AGqPZz7MOaPc6Giue/wCE00n/AJ46z/4JLz/41R/wmmk/88dZ/wDBJef/ABqj2c+zDmj3Ohornv8AhNNJ/wCeOs/+CS8/+NUf8JppP/PHWf8AwSXn/wAao9nPsw5o9zoaK57/AITTSf8AnjrP/gkvP/jVH/CaaT/zx1n/AMEl5/8AGqPZz7MOaPc6GsHxx/yIHiT/ALBd1/6Kamf8JppP/PHWf/BJef8Axque8da/BrfgnVtP0tdaS9mt2WIDRrsb8jBQ5ix8y5XnpnPaqjTndaA5RtuehUVymmeLNPtdLtYLo61cXCRKJZm0W8zI+Pmb/Vdzk4q1/wAJppP/ADx1n/wSXn/xql7OfZhzR7nQ0Vz3/CaaT/zx1n/wSXn/AMao/wCE00n/AJ46z/4JLz/41S9nPsw5o9zoaK57/hNNJ/546z/4JLz/AONUf8JppP8Azx1n/wAEl5/8ao9nPsw5o9zoaK57/hNNJ/546z/4JLz/AONUf8JppP8Azx1n/wAEl5/8ao9nPsw5o9zoaK57/hNNJ/546z/4JLz/AONUf8JppP8Azx1n/wAEl5/8ao9nPsw5o9zoaK57/hNNJ/546z/4JLz/AONUf8JppP8Azx1n/wAEl5/8ao9nPsw5o9zoaKxrfxXotxMkJu2t5ZDhEvIJLZnPookVcn2FbNS01uNO+wUUUUhhRRRQBm6/qD6XoF9ewgNPHEfJU9GkPCD8WIFUtMsU0zTLeyjYsIUClz1c92PuTkn3NL4w/wCQDGOx1CxB+huoqtV6OCS5WzlxD1SCiiiu05gooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKAMnxDFINMN/bqTd6e32uDHVioO5P+BLuT/gVdPDNHcQRzxMGjkUOrDuCMg1msoZSrDIIwRUXg9i/gjQGY5J063JP/bNa4MbFaM6sO90bVFFFcB0hRRRQB5d8ef8AkR7L/sJR/wDoqWij48/8iPZf9hKP/wBFS0V20PgMZ7nWfD3/AJJ14d/7B8P/AKCK6Wua+Hv/ACTrw7/2D4f/AEEV0tckt2arYK4yH/kqWtf9gu0/9Dmrs64yH/kqWtf9gu0/9Dmq6W5nV+E6OiiitjlCiiigAooooAKKKKACiiigDK0D/kKeJP8AsJJ/6S29aGraemraNfadLjZdQPC2ewZSP61n6B/yFPEn/YST/wBJbetys5bnfT+BHh3wW18aL4Q8U2158p0l2u2Rjz9wgj84/wBa4HwTcX2ieNPDniW8JEGp3skfmH+IEhJCfoZM/hU/j2W58KeOPF2m242wasoJxxhXdJcj8mX8TXcfEHwk2m/BLQTGhS50jypJSOqmT7//AI+y/lW2n3mev3DvFq/8JL+0Joelfeh05I3kXtlczHP1G0VueD38Nn4xeJF0+HU11cRzfaXndDAR5qbtgA3dcYz2zXPfBuSbxP4/8ReLLpPn8sRj2LnjH0WPH41Y8Af8nAeL/wDrlcf+jo6l9uyGu/c6FvjRo32rULKHS9Tnv7WYwx20MQdpyCwJXB4A2859RxWr4I+JWmeNbq5sYrW4stQtl3vbz4yVBwSCPQkAggda4f4ORI3xE8ZTFQZElZVbHIBlbP8AIflTvCihP2jvEQUYBhlJA9/LJ/Wk4x1Q1J6M6bXfi9pel+IZdE0/TL/V7uAlZ/siZCEdQO5I79veuJ8K61aeIv2iJ9VsfMFvcW+VEqbWUiBVYEdiCCPwqx8EJ7ey8UeKrTUHSPVGkUASHDEK7+YBn3K5/D0qPw9c2N3+0pqE+nPG9swkw0ZyrMIgHIPf5g1OyV0uwrt2Z3Pir4saR4c1g6PbWd3qupr9+G1XIQ4zgn1x2APvVvwd8StH8YzzWUEc9nqUILNaXIAYgddp7479D7V578Jbi1sfiX4rg1eSOPVpJXEbTEAt+8YyAE9z8p9wKdqMtvqP7SOmyaG6SNEq/a5ITldwRt+SOPu7VPvxS5VsPme56P4S8f6b4st9TkiguLNtNbbcR3O0FRg88E/3W/KovBHxH0rxy98llb3Fs1mFZhcbRuVs8jBPTHP1FeReOLybwD458WW8CsLfX7AmMDjDSEbm/A+b+dZlwl18L7hGQODrfh7a3+xLIOfxUgfnT9mnsHO0e8eDPHVl43W/k0+0uYoLSQR+bMFxITn7uCewB59RTPiJo+v6/wCFX03w9cw2880gE7ySFMxYOVBAPJO38M1T+Emh/wBhfDrTldNs94Ddy/V/u/8AjgWt/wAU+JbHwloFxq9+SY4+EjX70rnoo9z+gye1ZvSWhe8dTw/4gfD3SPAXhDS9Tsr25h11Zo1Lib/WNtJYqByMEcEfj1r1PUfHS+Ffh/ouu65azzT3UcCTJCFDCR4yxJBIx9015n4bFp8QfEf/AAlXjfXNNgtIXxaaZJdonAPQqTkJn15Y+3X27UdV0O30yG81K9sEsJSvlTTyJ5TkgkbSeDkZxjtVyeyepEerRb0+8TUdNtb6NWVLmFJlVuoDAEA/nXn3xVtYL7VfBdrdRLLbzawkckbjIZTgEGvRoJIZbeKS3ZGgdA0bIQVKkcEY7Yrz/wCJf/IweBf+w3H/ADFRH4i5bE3juRdH8O6X4U0NFs5NYuVsIREMeTETmRh+Bx/wKrnirwhd6voukeH9Lnjs9FhlRb6MOyPJbqANikA9e+cdBWX41yvxW8AvL/qd90q56byi4/pXV+KvFFh4S0V9RvtzksI4II+XmkPRVFPXSwtNbnnnxK8L6B4Q8N22r+H7VdM1q3uYlsntiQ8rE4KEZ+b5cnn09zXrcRdoUaVQshUFlHY9xXA+HvCuqa5rUPizxlt+2R/Np+mKcx2QPc+r9Pp+QX0AsqkAsAT0yetKT6DiuotcPqHgVvEXjm41TxGYL3R4IVj06y3thGx87OuAM5zjk8Yz0FdxXFeLvGF5aanF4Z8NWy3niK5Td83+rtI/+ekh/kP8QCo3voDt1MCwsLPwz8arXSvDimCzu7B5dSs42JjjIzsfH8JJ2j6H3pPC2i6f8QNf8R6/r9ut/FBfvp9jBMSUhjjA5C9MnIP1zXXeDvBsHheCe4nuHvtYvTvvb6X70jeg9FHp/kYHwnIsh4p0aUgXVprUzsh67HC7W+h2mrb0dibdyv4TuF8HeLPFXhlHkfSrK2GpWcbuWMKFcugJ7ZIx9Pc1X8CeDNJ8X+Fm8ReJrRdQ1LV5JJHllY5iQMVVU5+UADIx6+1Pt7Y698VPGzWhDxxaQunFwePNdQcZ9QVIP0ra+EN3HcfDTTIwwEtqZIJkPVGDtwfQ4IP40N2V/QFvYr/DK4ll0vXPDOoSNdro99LZI83zF4Odob14yPpgVN8PLuSwutb8H3Ds7aJcAWrOck20g3Rj3wOPoRVT4W4vNS8Y6xEd1reaxIIHHR1TPzD2O4UaNlvjz4jMf3E0yFZcf3zsI/Sh7sF0PRqp6tqUOj6Pealcf6m1heZ8dSFBOB78VbVlYZVgR7GuR+KSyN8M9dEWd32cE49Ayk/pmoSu7FvYq+BLG/m8FXGstKia5roe8ad13BCwIiGP7qrtwPrWBf8Agbwb4b8GXH/CUz202rSRPLLfyzHz5JTkgx5OTzjAHXHPeu30fUrbTfh3p2pOrta2+lxTMIl3NsWIE4Hfiq02l+FPiBodtrF1ZQ3UE9v+7nkG2SJeSRuB+Ug5zg9RVXdybaC/DiXVJvh9o8ms+Z9tMJ3GXO8puOwtnuV211Nef/CDUry/8GT/AGy6kuobW+lt7W5lOWkhXbtJJ68kj8Mdq78EEZBBB7ipluxx2Ob8aeHb3xTpdvplvf8A2S0e4Rr4LkNNCPvICOmf6VwHi/SvDugatoEfgyOCDxKmoJGILKQlmi53iUAn5eBkt7+9el674n07w7PpsWoGVBqFwLaKRVyiuem454H+BrhvinoGjaD4aufE2mRrpmuQ3CSQ3VsdjzSO43K2PvZBY8+npmqg3ohS7nQ/Em0mHhn+3LH5dS0RxfW7/wCyv+sU/wCyUzkd8Cul0nUodY0ez1K3/wBTdQpMgPUBgDg+/NZetzvJ8PdRuL5AkjaVI86HjaTESw/nWf8AC/dF8MtB887SYCRuOOCzEfpil9kfU7CuF+Il5LcNpfh23cq+oTqJSOy7gB+GTn/gNd1XnfiXK/FfQGf/AFZiQL9dz/4isK3w27nrZPFPE87+ypSXqlodnqF9ZeHNDa4kwltaxhUQdTgYVR79BXleh3GoXvxK02/1JSst3mZF/uoUYKB7YHFdF4/tdav9asYrbTJr3TYFErRoDtd8nIYj2A/M+tc1d6vrTePLK9fRjHqEcYWOzGfmGGH8ifyrGrL3l2TPbyrCqOGnKLTlUjK7urrTRfPdv0PaKKoJq1pG9rb3dxBBezxqwt3kAck9gDyecj8Kv11J3PkpQlH4keJ+Mbi7s/iPfXtluE1sYpQQM4AjTr7etdV4w1iDXfhsmoQcCSWPemeUYHlfzqCKGO4+MuowzIHikt9rq3QgwqCK5PxJaXnhea/0EkvYXLrPCW9AeD9eoP0FcTbipPo7n3FKFPETw0NpwjCS811Xy3PR5r7+zvhVFcg4YaZGin0ZkCj9TVzwxDFoXgezef5EjtjcSn0zlz/OuY11jdeDfCejISHv/s6nH90IM/qwP4VofEzVBYeHYtNg4kvGCbV7Rrgn9do/Otua15dkeN9WdXkw6/5eVJN+i0/+SPPWl1Npj4yA4/tDA/3vvY/3cfLXuVvNb6rpkcygSW9zEGwe6sOh/OvLhpnjP/hFv+Ef/wCEftfsmPv7h5md27OfMxnPtW98L9WNzo0+lzEiayf5QeuxiePwOfzFRRfLKz6/mdmcwWIw/toOP7t291p+69r221GeBnCS634SvQJobd3CK/IaIkqw+hyD/wACNUPBGm2WkfFzxnY6fbR21rFBabIoxhVzGCcD6kn8ataIC3xf1lo/uCFt3/kMfzo8Nf8AJa/G/wD1ws//AEUtdFB+412/zPEzmK+sxqdZRjJ+rWv+Z6JRRRVnmmBo/wDyGfEn/YRT/wBJbetisfR/+Qz4k/7CKf8ApLb1sVstjz6nxMKKKKCQooooAKKKKACiiigAooooAKKKKAIbv/jyn/65t/KqHw9/5J14d/7B8P8A6CKv3f8Ax5T/APXNv5VQ+Hv/ACTrw7/2D4f/AEEVnV2N6G7OlooorA6DmtI/5Kpr3/YLs/8A0ZPXZVxukf8AJVNe/wCwXZ/+jJ67KvQp/AjCW4UUUVZIUUUUAFFFFABRRRQAUUUUAFFFFABRRRQBzWkf8hzxP/2Ek/8ASS3rYrH0j/kOeJ/+wkn/AKSW9bFcFX42bx2Ciiisyjk7n/ip9QuI5yTo1pK0Ig/hu5VOHL+qK2VC9CQxOeK10RY0VEUKqjAUDAArH8JfN4Q0iQ/emtI5nPqzqGY/iSTWzXtUoKEUkefOTk7sKKKK0ICiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAjnghuYHguIklhcbXjkUMrD0IPWqGlySaHq0GlNI76bdhhZ72LGCRRuMWTyVKhmXPTaw6bQNOsnxB8lrYyj78epWe0+m6dEP/jrMPxrGvBTg7mlKTjJHVUUUV453hRRRQBg+MP8AkBRf9hGx/wDSuKrVVfGH/ICi/wCwjY/+lcVWq9LB/A/U5MR8SCiiqd7qunaZt+36ha2u77vnzLHn6ZNdhzlyiore5gu4Vmtp45om6PG4ZT+IqWgAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKg8G/8AIjeH/wDsG23/AKKWp6g8G/8AIjeH/wDsG23/AKKWuHG7I6cPuzbooorzzqCiiigDy748/wDIj2X/AGEo/wD0VLRR8ef+RHsv+wlH/wCipaK7aHwGM9zrPh7/AMk68O/9g+H/ANBFdLXNfD3/AJJ14d/7B8P/AKCK6WuSW7NVsFcZD/yVLWv+wXaf+hzV2dcZD/yVLWv+wXaf+hzVdLczq/CdHRRRWxyhRRRQAUUUUAFFFFABRRRQBlaB/wAhTxJ/2Ek/9JbetysPQP8AkKeJP+wkn/pLb1uVlLc76fwo5PxD8OfDvijWodW1O3le6iRUGyQqrBSSMjv1rf1fSrTXNJutMvkL21yhjkAODj2PY1doouyrIwfCvg/SPB1jNaaRFIkc0nmuZH3EnAHX04pml+CtH0jxRf8AiK0SYahfKyzM0mVIZgxwO3KiuhoouwsjntA8F6P4a1PUdQ05JluNQbdOXk3AncW4Hbkmiy8F6PYeLbrxNAkw1K6UrIxkyuDjOB/wEV0NFF2FkcT4j+FPhbxPqp1K8tporp8ea1tJsEv+8MHn3GDVvSPhz4b0HXotY0y0e3uYovJRVkOwLt29D1J7k966uijme1xcqOP8UfDLwz4tu/tmoWkkd4QA1xbPsZwOm7qD9SM1c8K+BNA8Gxyf2TaFZpRiS4lbfIw9M9h7DFdJRRzO1h2V7nM+KfAOg+MLm1uNXgkeS2BVDHIVyCQcH1HH6ml8VeBND8ZfZP7XhlY2m4RGKQpgNjIOOv3RXS0UczCyGxxpFEkcahUQBVUdAB0FYXirwfpXjK0gtdWE7QwSGRFikKfNjGT68Z/Ot+iknYLXPNv+FGeCv+eF7/4Emuk1XwLoms+F7Hw9eRznT7LZ5KrKQw2KVXJ78E10tFPmfcXKiCztIrCxt7OAEQwRrEgJydqjA/QV558V7mOx1DwdfTh/s9rqyTTMiFyqrgk4AJr0qihOzuNq6scH4yt/+Er8H2PiHw+Gmu9OnXUbIGNlaTYfmTBGeQDxjkgVc1Pw54e+JmlaTql01xJbiMy2/lTFMbsZzjuNuPbBrsKQAKMAAD0FHN2FY8//AOFNeFf7+p/+BrVp+J/DB1XVvCslvbu39l3gmM5kAWKNQCQR1ZmKqBgepOK66ijmYcqCuI1X4U+G9Z1m61a7+3fa7lt0jR3TKDwBj6YArt6KE2thtJ7nHaH8M9A8PaxBqli18biDds826Z1+ZSpyD14JqxrvgHR9d1T+1DLfWGoFPLe50+4MLyL6Njr/AD/Kupoo5nuKy2Mnw94b0vwvpv2HSrfyoixd2ZizyOerMx5JrC1P4Y6DqOo3V4kuo2LXhzdxWN0Yo7g9y69DnvjFdnRRzPcdkZtta6X4W0DybeOOz02xiZiB0RRkkk9T3JPU1zXw6064eHVPFF9E0V5r1x9oWNh80duOIlPvt5/EV2xAYYIBB7GlovoFjlPAHh1/Dei3lu1u1slxfy3EUDyB3jQ4ChiOMkLng8ZxXR39lBqWnXNjcruguYmikX1Vhg/oasUUN3dwSsrHE/DmSa00a58K6lg32iyG3YMOJYGyYnHsV4/4DUc3wm8Ou8ywzapa2UzFpbC2vWS3cnrlPT2BrucDOcDPrS0czvdC5VbUwtT0Gyj8F3eiWemlrT7K0UdnbsELDH3QxOASe5PfNTeFdLl0TwnpWmXDK01raxxSFem4DnHtmteii+lh2M7W9D03xFpcum6rbLcWsmCVOQQR0II5BHqK5yw+GGg2l/b3lzPqWptandbR6hdtNHCe21en55rtKKE2gsmcV8SJp73SLfwvp5/4mGtyiAYGfLhBBlkPsF4/4FTvGXhBNV8H6foNhas4tp7dYCJAqwKny72z1AXPA5JIrssDOcDPrS0KVthWuFcV8Q9PmFtY67apun0yYSMB3TIOfwIH4E12tBGRg9KiceZWOrCYl4atGqle3TutmvmiG0uYr20huoG3RTIHQ+oIyKz5vDun3Gvw626yfbIV2oQ/y4wR0/E1rAADAGBRTaT3M41ZU23TbV7r5PocrrGhvqPjXSb2O1kAtVDy3LMNm0FiqAdS2Tk+1dVRRSUUrvuVWxE6sYRltFWX5mSnh3T4/EUmuKsn22RdrHf8uNoXp9AKNc8Oab4iiij1CJm8piyMjbWGeoz6dPyrWoo5Va1hrFVlOM1J3irJ9l2MYeGLAX2l3W6bOmReVbxlgVxjGTxknp37CjUvDGm6tq1tqV4sjzW23yxvwowcjj61s0UckdrDWLrqSkpu6Vvk9/zCsGDw9pGh6jea7H5kMjq7znf8mD8zcfhmt6ggEYIyKbSZnTrTppqLaT0fmjg/BEfk2ms+LL8NGt27zD5SSsSksSAOT9P9kVl+A9Tttb+KfjDVbEyPZXENr5UjxMm7agU8MAeoNeoAADAGBRRTShGxeMxDxNZ1Wrdl2S0S+4KKKKZgYGj/APIZ8Sf9hFP/AElt62Kx9H/5DPiT/sIp/wCktvWxWy2PPqfEwooooJCiiigAooooAKKKKACiiigAooooAhu/+PKf/rm38qofD3/knXh3/sHw/wDoIq/d/wDHlP8A9c2/lVD4e/8AJOvDv/YPh/8AQRWdXY3obs6WiiisDoOa0j/kqmvf9guz/wDRk9dlXG6R/wAlU17/ALBdn/6Mnrsq9Cn8CMJbhRRRVkhRRRQAUUUUAFFFFABRRRQAUUUUAFFFFAHNaR/yHPE//YST/wBJLetisfSP+Q54n/7CSf8ApJb1sVwVfjZvHYKKKKzKOS8I/wDIl6F/2Drf/wBFrWzWN4R/5EvQv+wdb/8Aota1Zp4raFpp5UiiXlndgqj6k17q2PNe5JRWeuu6O5wuq2JPtcIf61eR0kQOjKysMhlOQaBDqKzv+Eg0X/oL2H/gSn+NWre8tbsE21zDMB1McgbH5UAT0VVutTsLF1S7vra3dhkLLKqEj15NQf8ACQaL/wBBew/8CU/xoGaNFVpNSsYbVLqW9t0t5DhJWlUI30OcHoahj1vSZmCxapZOx7LcIT/OgRfooqva39nfFxaXcFxsxv8AKkD7c9M4PHQ0wLFFVr3UbHTohLfXlvaxk4DTyqgP4k0tnf2eoQ+dZXcFzFnG+GQOPzFICxRVGbWtKt5Wim1OzjkU4ZHnUEH3BNM/4SDRf+gvYf8AgSn+NFx2NGiqr6lYxzRQve2yyzAGNDKoZwehAzzn2qy7rGhd2CqoyWJwAKYhaKzrfxBo13cC3ttXsJpycCOO5Rmz9Ac1fd1jRndgqKMszHAA9TSAdRUFre2t9GZLS5huEB2lopA4B9Mike+tI7tLV7qBblxlYWkAdh7DqehoAsUUUUwCiiigArJ8Rf8AIPtf+wlYf+lUVa1ZPiL/AJB9r/2ErD/0qiqKnwMqPxI6qiiivEPRCiiigDB8Yf8AICi/7CNj/wClcVWqq+MP+QFF/wBhGx/9K4qtV6WD+B+pyYj4kY3izW/+Ec8K6lq4UM9tCWRT0Lnhc+24ivJfAXw7t/HWmS+KPFN5d3U95K4jVZNvCnBYn6ggAYAA/L0z4iaVPrXgDWLG2QvO0IdEXqxRg+B7nbiuS+DXirST4Jh0i4voLe7spJAY5pAhZWYuGGeo+Yj2x9K6XrKzMlpHQf4U+HeueDfHMs2lX6P4ckH7yGeQ72BHHAGNynvxx9a3fF3xL0jwnfppphuL/U3AP2a2AJXPTcexPoMmqtl8TrbVPiF/wjOmWX223xzfQy/KpAyxxjBUcDOevrXHeFp7ax+P3iD+2XSO4kMwtHmOOSylACe5j6e3HeldLRDtfVnceFPifpHifU20p7e507UxnFvdADfgZIB9cc4IFRav8VtH0XxLe6Fc2d891bKu3yYw/nMwUhEGc5+bvjoa43x9Nb6h8Z/DEejukl/FJCLhoTnGJM4bHouSfaptPiST9pjUGdQSkW5c9j5CDP6mjmewcq3Nyy+NWiSNdwahp2o2N5B9y1aPc8rEgBFHZuRwcfWtHwt8UNP8R6+2hzadeabqGCY4rkffwMkeoOOcEdO9cjNDHJ+05FvQHEYfkdxbHB/lT9WAX9pTSiBgmEE+/wC5cUXYWR1Gq/FnRdH8Q3+i3NnfNc2gAXykDec524RBnOfm746Gn+F/ilpniLW20a4srvTNROfLhuhjfgZx7NjnBFcRY3VjaftIag988aBiUheQgASGJccnoSMj8am+IEsF58avCsemukl5HJB55iOSoEucNj0XJPsaXM9w5Udl4n+J+n+H9bOi2unXuq6kqhpIbVc7MjOD1OcYPAqx4O+I2neL72509LS5sdRt1Lvb3AGdoIBwfYkZBA61y0/ibWvEPxJ1LQvDEWlaZNZq6S6hcQB55ArANjjkZxxjtnNYvw9SeP44aslzqa6lOtvKsl2qBBKwKZ4HAwePwp8zuHKrHUXvxs0W0nv7VdNv5Lu1nMIiAX95gsGYHJwBt/Ue+NXWPihpOi6Npd5PaXb3mpwrNb2CAGXa3QtzgA9u59OuOM+ElvG/xJ8XXDIDJHJIqsRyA0rZ/wDQRVHxmmox/Hm0NveW1lM8Uf2Oe7TdEv7sjp7tuA9yKXM7XDlV7Hofhb4l6b4j1ZtHnsrvS9U27ltrtcFxjJwfXHOCBxV3UPHVjp3jiy8KyWtw11dorpKu3YM7uvOf4T2rz670TU5/idocuueLNHbV7WSJlt4YWR3j3k7eBjJG7gnv6V6vPqOiRa1DZ3F1YrqjgGKF3UTEc4wDz2NUm2JpGnRRRVkBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABUHg3/kRvD/8A2Dbb/wBFLU9QeDf+RG8P/wDYNtv/AEUtcON2R04fdm3RRRXnnUFFFFAHl3x5/wCRHsv+wlH/AOipaKPjz/yI9l/2Eo//AEVLRXbQ+AxnudZ8Pf8AknXh3/sHw/8AoIrpa5r4e/8AJOvDv/YPh/8AQRXS1yS3ZqtgrjIf+Spa1/2C7T/0OauzrjIf+Spa1/2C7T/0OarpbmdX4To6KKK2OUKKKKACiiigAooooAKKKKAMjSmFt4l1m0fhrlo72P8A2l8tYmx9DEM/7w9a3qyNU01r4Qz28/2a+tiWt5wu7bnqrD+JD3HsCMEAiBPEM9qNmraTeQyDrJaQtdRP7rsBcf8AAlH41Eot6o6qVRWszeorB/4S/TP+ffWf/BLef/GqP+Ev0z/n31n/AMEt5/8AGqnlfY2549zeorB/4S/TP+ffWf8AwS3n/wAao/4S/TP+ffWf/BLef/GqOV9g549zeorm7rx3oVjEJbttTtoywUPNpN2i5PQZMWKn/wCEu0sqGWHV3UjIZNHu2B/ERUcr7BzJ9TdorC/4S7Tf+fbWv/BJef8Axqj/AIS7Tf8An21r/wAEl5/8aosx3Ru0Vhf8Jdpv/PtrX/gkvP8A41R/wl2m/wDPtrX/AIJLz/41RZhdG7RWF/wl2m/8+2tf+CS8/wDjVH/CXab/AM+2tf8AgkvP/jVFmF0btFYX/CXab/z7a1/4JLz/AONUf8Jdpv8Az7a1/wCCS8/+NUWYXRu0Vhf8Jdpv/PtrX/gkvP8A41R/wl2m/wDPtrX/AIJLz/41RZhdG7RWF/wl2m/8+2tf+CS8/wDjVH/CXab/AM+2tf8AgkvP/jVFmF0btFYX/CXab/z7a1/4JLz/AONUf8Jdpv8Az7a1/wCCS8/+NUWYXRu0Vhf8Jdpv/PtrX/gkvP8A41R/wl2m/wDPtrX/AIJLz/41RZhdG7RWF/wl2m/8+2tf+CS8/wDjVH/CXab/AM+2tf8AgkvP/jVFmF0btFYX/CXab/z7a1/4JLz/AONUf8Jdpv8Az7a1/wCCS8/+NUWYXRu0Vhf8Jdpv/PtrX/gkvP8A41R/wl2m/wDPtrX/AIJLz/41RZhdG7RWF/wl2m/8+2tf+CS8/wDjVH/CXab/AM+2tf8AgkvP/jVFmF0btFYX/CXab/z7a1/4JLz/AONUf8Jdpv8Az7a1/wCCS8/+NUWYXRu0Vhf8Jdpv/PtrX/gkvP8A41R/wl2m/wDPtrX/AIJLz/41RZhdG7RWF/wl2m/8+2tf+CS8/wDjVH/CXab/AM+2tf8AgkvP/jVFmF0btFYX/CXab/z7a1/4JLz/AONUf8Jdpv8Az7a1/wCCS8/+NUWYXRu0Vhf8Jdpv/PtrX/gkvP8A41R/wl2m/wDPtrX/AIJLz/41RZhdG7RWF/wl2m/8+2tf+CS8/wDjVH/CXab/AM+2tf8AgkvP/jVFmF0btFYX/CXab/z7a1/4JLz/AONUf8Jdpv8Az7a1/wCCS8/+NUWYXRu0Vhf8Jdpv/PtrX/gkvP8A41R/wl2m/wDPtrX/AIJLz/41RZhdG7RWF/wl2m/8+2tf+CS8/wDjVH/CXab/AM+2tf8AgkvP/jVFmF0btFYX/CXab/z7a1/4JLz/AONUf8Jdpv8Az7a1/wCCS8/+NUWYXRu0Vhf8Jdpv/PtrX/gkvP8A41R/wl2m/wDPtrX/AIJLz/41RZhdG7RWF/wl2m/8+2tf+CS8/wDjVH/CXab/AM+2tf8AgkvP/jVFmF0btFYJ8X6aASLXWz7DRLz/AONVg3niuTWmez/s3XtN08nbNK2k3RnmXuqbIyEB7sTu64APzAUWxOSSNzw+wuTqmop/qr2+aSI/3kREhDD2PlZHqCK2awIfEul28EcMNjrEcUahERdDvAFUDAAHldKk/wCEr0//AJ9da/8ABJef/Gq1OFqTd7G3RWJ/wlen/wDPrrX/AIJLz/41R/wlen/8+utf+CS8/wDjVAuV9jborF/4SrSlXdMb61j7yXen3ECD6tIgA/E1sRyJLGskbq6OAyspyCPUGgTTW46iiigAooooAKKKKACiiigCG7/48p/+ubfyqh8Pf+SdeHf+wfD/AOgir93/AMeU/wD1zb+VUPh7/wAk68O/9g+H/wBBFZ1djehuzpaKKKwOg5rSP+Sqa9/2C7P/ANGT12VcbpH/ACVTXv8AsF2f/oyeuyr0KfwIwluFFFFWSFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAc1pH/Ic8T/9hJP/AEkt62Kx9I/5Dnif/sJJ/wCklvWxXBV+Nm8dgooorMo5Lwj/AMiXoX/YOt//AEWtZPxR/wCSaa3/ANcV/wDQ1rW8I/8AIl6F/wBg63/9FrWT8Uf+Saa3/wBcV/8AQ1r3H8J5y+I858BfC3w34l8CWurajJdxXEplDyRzBVUK7AHBBHQVN8Er24tfE+u6BDdm60uBWkicHKkrIFDL6bgc/gKo+BPhLpPivwhZ6teajfxSStIDHEU2ja5XjIPpXrnhPwZo/g2ykt9LifdKQZZ5W3SSY6ZOAMDngAVEY7MqT3R4T8MvCfhzxO2rnX7loTbtH5OLgRZ3b93Xr0FWNc0vTvBXxE0MeDdUkuZZZEEkSzCQglwNhK9mHY/1FM+F3gTSfG0WurqTXCSWxiELwvjaW35JBBB+6K0PAi2ngH4kyaD4g062Ny77LXUWBypb7pGTgKw4yOQTgnriUtEU3qy58Z4bW4+InhyC+kEdpJFGk7ltu1DKQxz24zzV7/hB/hJ/0McP/gzjqn8Yoba5+JXhmC9Ki0lSJJtzbRsMxDZPbjPNdJ/wgfwn/wCeth/4NW/+OVVrtivojN+MFhZ6X8KdFstPcvZw3sSwsW3bk8qUg5707TPgt4b1fwpp92J76C8ubOKUyLIGUOyA/dI6ZPTP41J8avsY+GelJp8kclol9EkLRvvXasUgGG5z0rrfD3irw/YeCtIa61vT4vJ0+ESKbhNykRjI25zn2607Lm1Fd20OQ+DWu6jHf6v4Q1OZpn00sYWY5KBX2OuT2zjHpk1lfBG9j0208ZX0uTFbJFM+PRRMT/Kp/g7FLq/jzxP4nSNltJ2lVSw6tJKJMfUBRn6iqHwesH1TRfHenxkB7q3SBSfVlmUfzqV0G+oeCPCzfFW/1LxL4nuriSFJvJigifaAcBio9FAZeB1znPr0Vl8LtW8LeOrTUvCl+selnH2qK6lOSufmTgfMCORnoaz/AIJeIbHSrDU/Dup3EdlepdmZUuGCbvlCsvPcFOnvXU6n8UbS38bWHh3SrVdVNwVSWW3mGInJ6dCDgcnnj8DTSja7E272RU8a/C/w7eWuueIZRd/bvs8tzxNhd6oSOMdOBXE/Cz4c6D4v8MXOoaoLrz47xoV8qXaNoRG6Y65Y17V4r/5E7XP+wfP/AOi2rg/gH/yIt9/2En/9FRU3FcwJvlMLxxbR2Xxq8F2sWfLhhs40ycnAncD+VSfEG91Hxp8SrTwJaXTW1jGVNwV6MdnmMx9cLgAetL8Q/wDku3hL/dtf/Sh6q+IrkeCvj3DrmoKy6deAN5uMgK0flsf+AtyR1x9al9fUa6G/qvwL0BtHkXSpryLUUQmKWSUMHYdAwxxn1GMVU+Hviu91/wCG/iPT9Rlea6020kUSucs0bRvtBPcgqRn0xXfav458PaRokupvqtnMgQtEkUyu0pxwqgHnP6d68r+FOmXKeB/GOszIVju7Z44iRjdtRyxHtlgPwNU7J6CV2tTmfhX4sl8I6/F9sLJo+pt5Mjt91XGMP+G4Z9m+ld14n/5OI8O/9e6f+1ax/BfhCPxj8Gb20RVF9DqEsto57OET5c+jDj8j2rn/AAbq99qvxQ8MpqQb7TYr9jJf7xCB8bvcA4/CoWiSKerbPpuiiitzEKKKKACsnxF/yD7X/sJWH/pVFWtWL4pghutHit7iKOaCXULJJI5FDK6m6iBBB4II7VE/hZUfiR11FYP/AAg/hL/oVtE/8F8X/wATR/wg/hL/AKFbRP8AwXxf/E14uh6GpvUVg/8ACD+Ev+hW0T/wXxf/ABNH/CD+Ev8AoVtE/wDBfF/8TRoGpV8Q6nYap4bin0+9truL+0bD57eVZF/4+4u4NalcY/w98O+DPD8TaXaFro31ij3c7b5WH2uLv0UeygV2delhLcrt3OSvugrhtd+EvhPXr972W1mtZ5Duka1k2Bz6kEEZ+gFdzVe61Cysdn2y7t7ffnb50gTdjrjJ5611NJ7mKbWxj+GPBWheEYpF0m02SSDEk8jF5HHoSeg9hgVB4o8AeHvFzpNqdo32lBtW4hfY+PQnoR9Qa6YEMoZSCCMgjvS0WVrBd7nLeFvh74e8IyvPptq7XTDabid97geg7D8BVmHwbpEHi+bxQiTf2lMu1mMny42hfu/QCugoosguznz4N0g+MR4pKTf2mF2hvM+TGzZ936UT+DdIufF0HieRJv7SgXajCT5cbSv3foTXQUUWQXZ4Yvh2z8Q/HXXrLVbNprKWFjkgjDBI8FWHQivRfC/w38OeErxr3T7eWS7IKrNcPvZAeoXgAfXGa66ikopDcmzi9e+FvhnxDrDapcxXEN1J/rWtpdgkPTJGDzj0xVrSPh34d0HXI9X0u2ktrhIvKCpKShXbjkHqe+fXmuqop8qFdnP6F4N0jw5qmoajp6TLcX7Fpy8m4E7i3A7ck0eKPBeieL4I49Wti7xZ8qaNtrpnqAfT2ORWxBf2dzNJDBdwSyx8SJHIGZOccgdKsUWVrBdnI+GPhr4b8KXpvrC3llvMELPcSb2QHrt4AH1xmr154N0i+8V2viSdJjqNsoWNhJhcDOMj/gRroKr3V/Z2IU3d3Bbh87TNIEz9MmiyC7LFFNR1kRXRgyMMqynII9RVdNSsZLs2kd7bPcqSDCsqlxjr8uc0xFqiiigAooqO4uILS3ee5mjhhjGXkkYKqj1JPAoAkoqn/a+m4gP9o2mLgBof3y/vAehXnnPtVqSWOGJpZXVI0BZnY4CgdyaQDqKit7q3vIRNbTxTxE4DxOGGfqKlpgFFFFABRRRQAVB4N/5Ebw//ANg22/8ARS1PUHg3/kRvD/8A2Dbb/wBFLXDjdkdOH3Zt0UUV551BRRRQB5d8ef8AkR7L/sJR/wDoqWij48/8iPZf9hKP/wBFS0V20PgMZ7nWfD3/AJJ14d/7B8P/AKCK6Wua+Hv/ACTrw7/2D4f/AEEV0tckt2arYK4yH/kqWtf9gu0/9Dmrs64yH/kqWtf9gu0/9Dmq6W5nV+E6OiiitjlCiiigAooooAKKKKACiiigAooooAKKKKACmTSxwQyTSsFjjUszHoABkmn1jeLiV8Ga6QcEafcEH/tm1AC6DYfaxHr2ox7r64XfCjjP2SJuVRR2bGNx6k57AAb9IAFAAGAOAKWsW7nopJKyCiiigYUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABXONbr4e1mAW42aZqMhjaEfdgnwWDKOyvhgR03bSOWNdHWJ4q40iBu41GxwfrdRD+RNVF6mdSKcWalFFFaHCFFFFABRRRQAUUUUAQ3f/AB5T/wDXNv5VQ+Hv/JOvDv8A2D4f/QRV+7/48p/+ubfyqh8Pf+SdeHf+wfD/AOgis6uxvQ3Z0tFFFYHQc1pH/JVNe/7Bdn/6Mnrsq43SP+Sqa9/2C7P/ANGT12VehT+BGEtwoooqyQooooAKKKKACiiigAooooAKKKKACiiigDmtI/5Dnif/ALCSf+klvWxWPpH/ACHPE/8A2Ek/9JLetiuCr8bN47BRRRWZRyXhH/kS9C/7B1v/AOi1q1rOkWmvaRc6XfBzbXChZAjbTjIPX8Kq+Ef+RL0L/sHW/wD6LWtmvdWx5r3MzQNBsfDWjw6VpyutrCWKh23H5iWPP1JrToooEc94X8FaP4P+1/2Sky/aypl8yTdnbnGPT7xpnijwLoXi+W3l1WCQzW4KpJE5RsHnBI6jP9fWukoosrWHd3ucl4g+HOg+J5bSXVRdzS2sAgRxOQSo5y3qeetY/wDwpLwb/wA8Lz/wJNei0UuVD5mctefD7Qb7wrZ+G5o5zp1nJ5sSiUhg3zdT3++1YsfwU8GI4ZrW6cD+FrlsH8sGvQ6KOVC5mU9M0qw0WwjsdNtY7a2j+7HGMD6+59zWV4X8F6P4Qa8bSkmU3hUy+ZJuzt3Yx6feNdDRTsFzkPEvw08M+Kbs3l7aPFdt9+e2fYz/AO91B+uM1N4Y+Hvh3wlM1xptoxumG37RO+9wPQdh+AFdTRRZXuF3sQX1nFqFhc2U4JhuImikAODtYEHn6Gszwx4W0zwjpslhpSyrBJMZmEj7juIA6/RRW1RRYLnPar4L0fWfEtjr92kxv7IIISsmFGxiwyO/JNXde8O6T4msPsWr2aXMIOVzkMh9VYcg/StSiiyC7PObf4JeDoLkSvFezqDnypLj5T+QB/Wu6bS7P+yH0qOBYbJoTB5UQ2hUIxgY6cGrlFCSWwNtmN4a8Mab4T0xtP0tZFt2lMpEj7juIAPP4CqB+H/h/wD4S0eJlt5E1ESeYSkhCFsYJK+p7+9dRRRZBdhRRRTEFFFFABWT4i/5B9r/ANhKw/8ASqKtasnxF/yD7X/sJWH/AKVRVFT4GVH4kdVRRRXiHohRRRQBg+MP+QFF/wBhGx/9K4qtVV8Yf8gKL/sI2P8A6VxVar0sH8D9TkxHxIK+eviP9t8eeP8AUdP0074NDspGIHIJQZfHuWIX/gNe3eKNbj8OeGdQ1aTH+jQlkB/ic8KPxYgV4R4Aj+IFhbXWtaBoVtfJqZy9zdMuW2s2cZdTyxOeOcV0T7GUO56p8JfEX9v+BLVZX3XVh/osuTyQo+Q/984/EGmat8VtI0XxPfaFdWV41xaqu0xKHMzsFIRBnOTu7+hrzz4Y3Wo+DfiTPoOs2v2E6mmPIDAqr8tHggkYwWUcnrWxp8SSftL6gXUMY4ty5HQ+Qgz+ppKTshuKuzqvDPxTsNf8Rf2Fc6Ze6Zftny47kfeIGcHoVOOeR+NXPGHxJ0jwjdxWDwz32pSgFbW2AJAPTce2ew5PtXF+KVC/tD+HyowWhjJx3/1gqtpE9vp/7RGrPrTpG8iuLWSY4AYhNmCf9jIH5U+Z7Byrc7Twt8UtK8R6t/ZFxaXWl6kfuQXQxvOM4B9cc4IFGt/FPSdA8UXWhXdnePNAisrQqH81mUFUUdcndiuL+J09tqPxQ8LQaPJHLqcciCVoSCV/eAoCR6fMfYGpzGkn7TnzqG2xBhnsfs3Wjmewcq3Ny0+NWjGe6ttU0zUdOuoVykEke55TxhAOCGORwePer3hv4p2Ou+I/7ButLvdLvnBMSXS43YGcHoVOOR/OuU1iGOT9pPS1dAR5avyO4icg/mB+VP8AFIA/aI8PEDBMMZPv/rKV2FkdZ4m+J1joOu/2JZ6be6tqSqGkhtFzsyM47knHPAqx4N+I2neL76505bO5sdRt1Lvb3AGdoIBwfYkZBA61y0/ibW/EXxJ1LQfDK6XpctmrpLfzwB55ArANjjkZxx7ZzWJ8PUmj+OGrJcamupTrbyrJdqgUSsCmeBwMHj8KfM7hyqxP8PbtrDx/4+vFgecwGeXyk+8+2Vjge/FemeDPGVj410iXULKKSERTGJ4pSNwOAc8diD/OvPPhX/yVLxp/12l/9HNXOXWqSfDHxF4y0aMMkF/bFrHaPusx+Qj/AHVd/wAUpJ2QNXZ694R8d2fjK91KGws50gsWCm4kI2yEkgbceoBNWPHHhSDxh4ZuNNk2rcD95bSn/lnIOn4HofY1k/CXQP7C8A2ZkTbcX3+ly56/NjaP++Qv4k13NWtVqS9HoeDeF/idN4W8E6pomqqy6zpZMNlHIOWycbT/ALh5+mAK3/hloUHhTwzeeNfEcvl3V4hmaWXlo4ic/Us5wff5R1rC+I+k2Mvxv8PwPbIYr4WzXK9pSZWQ5/4CoH4V2Pxsjnb4cS+QD5aXMRlC9AmSPy3FahX+4t/mVI/jbppZLibQdWi0p5PLW/MYK5/l+AJNdZ4j8c6N4a8P2+sTytcQXWPsq24DNNkZGM9sc5NeWi11y8+FdqLvxVodv4ekt4otrW5LIcjCkqCd4Yc49CauazoWgH4WeHtO1fxRBBNE0j2N9HE7xyAseMYzgAqM8YxRzMXKjoYPjBbQX9tb674e1XRork4jnuoyF+pyAcc9RmrHxkvZIvAw0+35n1O6itUUdTzu/wDZQPxrhdX1Hx/8O0sbvVdRsNa05pQsJmYTFjjOQzAOOB1BIrqvENwPE/xP8F6eEZYbeA6pKh/hyMqD7goB/wACou2rBZJ3OL+MmmPaa54a0uxRneHT0t4VQfMSrFRj34rsdK8ZDxf8Gtda4cHUrPT5orod2Plna/8AwID8wao/Ej/ksXgr/rrB/wCj6574kaNd+AvEV7qWlpjSddt5reWMfdV3U7l9ucOv4joKT0bY90kd18Hru3sPhYt3dSrDbwyzPJI5wFUHJJqrN8cdL82WS00PU7nT4m2vdqoAHvj/ABIrmLGO5k/ZsvBbbiRcFpAvXYJVz/ifbNdt8P8AWPDsPwntfPubRLaG3db2N2Gd2TuDDuTnj1yKab0QmlqzstA1/TvEukRanpk/m28mRyMMjDqrDsRWnXj/AMAIrhdF1mZgwtHuUEWem4Kd36FK9gq4u6uRJWdgoooqhBUHg3/kRvD/AP2Dbb/0UtT1B4N/5Ebw/wD9g22/9FLXDjdkdOH3Zt0UUV551BRRRQB5d8ef+RHsv+wlH/6Kloo+PP8AyI9l/wBhKP8A9FS0V20PgMZ7nWfD3/knXh3/ALB8P/oIrpa5r4e/8k68O/8AYPh/9BFdLXJLdmq2CuMh/wCSpa1/2C7T/wBDmrs64yH/AJKlrX/YLtP/AEOarpbmdX4To6KKK2OUKKKKACiiigAooooAKKKKACiiigAooooAKxvF/wDyJWvf9g64/wDRbVs1jeL/APkSte/7B1x/6LahAtzoKKK8p+Jvi3XfCXjTw7Lb37RaLdFRcQeWhDbZBv8AmIyMqw79qyirux6Ldlc9Wory34y+NNW8MwaRZaFdNBfXcjOxSNXYooAC4YHqW/8AHat+C/HNzqPwju9f1C4E1/YR3AmkKgbnUFl4AA6FBT5Ha4uZXsej0V4r4c1/xprXwvu9XfxLa2t9JfiO3ub3yoY1jUDcM7cZJJHT+GvT9O1WLT/DGl3Gu6vYiaSCMSXbTosU0m3JKtwCDyRjtQ42BSubdFZVx4n0C0vlsbnWtPhu2xiGS5RX56cE9605JI4YmlldUjQFmZjgADuTU2GOorg/G3jezj8Caze+Gtdspb+0WMhraWOVkzIqk45GMEjpWp8PNVvda8A6VqWpT+ddzRu0spULuw7DoAB0Ap8rtcV1ex1FFY0Xi3w5PffYote0x7onaIlukLE+gGeT7VevNV07TpIY76/tbV5jiJZ5lQyHjhcnnqOnrSsx3LdFU7/VtN0oRnUdQtLMSEhDcTLHux1xuIz1FLfapp+lxrJqF9bWkbnarXEyxhj6AkigC3RSAhlDKQQRkEd68s17UviL4g8aXmjeHF/sXTrNci8urchbgjAyGZWyCem0dBkmmlcG7HqlFeafC3xtrGvXur6D4gEcmo6W5VriJQA+GKsCBxkEdR1B9ue/g1bTrq9lsrfULWa7hz5sEcys6YODuUHI545ocWnYE01cuUUV5x8PbzxF4g1PVdTvvEMj2VlqdxZrYfZowrKoG07wARjcPyoSurg2ej0VyGm6vd+IPH2oR2tw0ej6Kv2eRV6XN033sn0QcY9TmsxF8b+K9X1OSDU5fDWmWk5t7VGsVkkucf8ALQ7/AOE54xx27Zo5RXPQqK47wD4k1LWV1fTNZ8l9S0e7NtLPCu1Jhzhsdjwciuxoas7DTuFFZ2v6hcaVoN7fWllLfXMMRaK2hUs0jdAMDnr1x2zXn95H8Q9M8LS+KbjxLAbmGD7XNpL2KCFUA3GPd97IH454z3oUbibseo0Vyt3faj4j+H0Or6HJJaahLbJeW6dQXADeW3qDyv45rU8M69b+J/DljrFsNqXMe5kznY44ZfwIIosO5rUUVmeINXTQ9DutQcAmJPkU/wATnhR+ZqW7K7NKdOVSahBXb0Rp0VheDzfyeG7a51K4ee5uQZiX/hVvuge2MH8a43xp46vIdaXT9GuTFHbttnlVQd7915HQfzz6VEqqjHmZ3YfLKuIxMsPSafLe76af1oen0UUVoecFFeffEXxJq2hX1gunXXkpIjM67FYMQR6iuht/EK6r4KuNXs2Ecy2sjEDny5FUkjn0P6YrNVFzOPY75ZbWjQp4jTlm7enTU6CiuX8A6re6z4cN1qE5mm89k3FQOAB6AetM8F6leaxNrN/PcPJam7MVqh6IoyePwK/lTU07eZNXAVKTqqTX7uyfm27aHV0V5f418ZavaeIZrPR7lo4bOMeeVjVvmJGScg4A3KPrXoWjalHrGjWmoR4xPGGIHZuhH4HIpRqKUnFdCsTltbD0IV52tL8Oqv6ovUVx+i6xeWnjXUvD+oTtMr/v7N367SM7ffj/ANBNdhVRkpI58Th5UJKMtbpNPumFFFchcavd6v8AESLQtPuGhs9KiF1qTp1kdh+7hz2GPmPrjFUlc52zr6K8517U/FWrfEaXw74b1iHTYLTTluLh5LZJcyF8AcjI+Uj8jW/4a0vxbY30sniDxFb6nbNHtjiitFiKvkfNkDnjIx703GyFc6eiigkAZPApFBRXk3hu4+IPjLTpdasPFNrY2MtzKtvC9hG5EasQOcfh+Fdxb6Vr7+Eriwv9c8zV3V/Lv4IhFsbqnygY44z6jNU426kp3OhorjtD1bUPGngkNbag2kaxFKba7kSFZDDNGcONrcYPX2DVB8LtY1XWdA1F9XvmvLi21Oa2WVo1QlFC44UAdz+dLlC53FFFFIoKKKKACsTxX/yB4f8AsI2P/pVFW3WJ4r/5A8P/AGEbH/0qipx3RMvhZqUUUVqeeFFFFABRRRQAUUUUAQ3f/HlP/wBc2/lVD4e/8k68O/8AYPh/9BFX7v8A48p/+ubfyqh8Pf8AknXh3/sHw/8AoIrOrsb0N2dLRRRWB0HNaR/yVTXv+wXZ/wDoyeuyrjdI/wCSqa9/2C7P/wBGT12VehT+BGEtwoooqyQooooAKKKKACiiigAooooAKKKKACiiigDmtI/5Dnif/sJJ/wCklvWxWPpH/Ic8T/8AYST/ANJLetiuCr8bN47BRRRWZRyXhLjwboi90sYUPsQgBH5itmse0/4kurz6RP8ALDcSvcWDno4Ylnj/AN5WLED+6Rj7pxsV7dOSlFNHnSi4uzCiiirJCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKyfEPNlZoPvNqVlgeuLmNj+gNa1ZVt/xPPEULRfNYaW7M8g+7Jc7SoQeoQMxP+0VHUHGVaSjBtl04tyR1NFFFeMegFFFFAGD4w/5AUX/YRsf/AEriq1VXxh/yAov+wjY/+lcVWq9LB/A/U5MR8SMfxJ4a0/xXpY07UvONsJBIVik2EkZxk+nNXdL0210fS7bTrJNltbRiONScnA9T3NW6K67GFzm9e8DaL4i1mz1a9Sdb20CiKWGUoRtbcM/Q5qSHwbpEHi+bxQiTf2lMu12Mny42hfu/QCugoosguzn73wbpGoeKrTxHOkx1G1ULEwkwuBnGR/wI1H4o8C6B4v2NqtoTPGNqXETbJAPTPcexzXSUUWQXZynhj4deHPCVw11p1q7XZBX7RO+91B6gdh+Aq2PBukDxifFOyb+0yu3d5nyY2bPu/Sugoosguzn5vBukT+L4fE7pN/aUK7UYSfLjaV+79CaL3wbpF/4qtPEk6THUbVQsbCTC4GcZH/AjXQUUWQXZxev/AAt8M+ItXbVLqC4iun/1rW8uwSHpkjB5x6Yq1pHw78O6Drser6XayWtwkXlBEkOwrtxyD1PfPrzXVUUcqC7Ob0zwno/hjUdV1yyiuDc3YeW4Bfdu5LnaD05rx/XLyD4v/EDSYtI065js4ECXVxKgDCPdubOCQABkDnkmvoSik430GpW1GoixoqIoVVGAAOAKdRRVEnPar4K0fWfEtjr92kxv7IIISsmFGxiwyO/JNbd3aW99aS2l1Ck1vMpSSNxkMD1BqailYdzzxfgp4NW9Fx9muygbP2c3B8v6f3sfjXV6x4W0XXdGTSb+wjezjAESL8nlYGBtI6YHHFbFFHKguzgNO+DfhHT72O6Nvc3RjO5I7mbcgP0AGfociulg8Mafb+LLnxIpma/uIBbkMwKIg28KMcfdHf1raooskF2YGr+DtJ1vX9O1q8SY3mnlWgKyYUFW3DI781c1/QNP8S6RLpmpw+bbSEHg4ZSDkEHsa06KLILmPoXhnTPDuif2PZRM1kSxKTHfu3dQc9RXK3PwW8G3F8bkWtzCpOTBFORH+vI/A16FRRZBdlXTtNs9IsIbHT7eO3tYRtSNBgD/ABPv3q1RRTEFFFFABUHg3/kRvD//AGDbb/0UtT1B4N/5Ebw//wBg22/9FLXDjdkdOH3Zt0UUV551BRRRQB5d8ef+RHsv+wlH/wCipaKPjz/yI9l/2Eo//RUtFdtD4DGe51nw9/5J14d/7B8P/oIrpa5j4dOH+HHh4j/nxiH5LiunrkluzVbBXGQ/8lS1r/sF2n/oc1dnXGD938VtRX/nro9s/wCUsw/rVUtyKvwnR0UUVucgUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAVjeL/wDkSte/7B1x/wCi2rZrG8X/APIla9/2Drj/ANFtQgW50FeW/HrSvtvgSK/VcvYXSOT6I/yH9Sn5V6lWR4p0UeIvC2p6RuVWurdkRn6K/VSfYMAayi7NM9CSurHi3hnUP+Fh/FzRLqQebb6XpcbygjguqDd/5Fk/8drlrrU28I6L428HMxBlvI0g/wB1XO4/igSvYPhX8NrzwLLqVxqVxaz3FyESM25YhVGSc5A6kj8qyvHPwfvPFPjoa1a3dnFZzCL7SkhYOSvyttwpH3QO45rZSjzW6GfK7X6mb4z0r+xf2eNJsiu1w0Ekg9HfLsPzY1S+JP8AyQ3wX9Lb/wBJ2r0/4jeFLvxf4SOkafLbwS+ckgaYkKAueOAawfFnw51TX/h1oHh22urOO604RebJIzBG2RFDtwpPU9wKmMlpfuNxetjk/HngDQtI+E8GrwWzHVVEEkt28jM8rORv3ZOOS2fwqr8Qdbvv+FLeELbzn23sa+e2eXEaDAPryQfwr1Lxl4SvfEXw9/4R61nt47rZCvmSlgnyFSegJ7elZup/DT+2vhjpvhm8uY47+wjUxXMYLIJACO+CVIJH5HtQprS/cHF62OV+IXwz8OaB8Mpr7TrcxX1ksWbkSMTPudVbdzjndnp2FYuva1d6Z+z34ctLSRoxfyPFM6nBMYZyV/E4/AEd66C5+G/xB1zw4dF1vxLZNZ26AW0UYJ81l4XzH2A4A+pJ/OulHw0S/wDhZZeEtVuIxc2uXjuYAWVJNzEEA4yMMQRx1P1p8yVru+ouVvZHN658JfDFl8L7i8ghZdStbE3X2zzWJkdU3EYztwcYAx6VzmofbfFvwCtdVuWeW90S7KrMTl3iBC9fYMvP+xW+/wAOviTeaQnhu88UWP8AYagISoJkMY6L9wE4x0LY7Zr0nT/CGm6b4LPheBSbNrd4HZuWcuDuY+5JJpc1utx8t+ljwfx/rFz8QrnS/sRDjT9DN9dAfwPjMv8AJRWne6p/wsjxD8P9FDebFHarPe+hYf6wH8Ij/wB9V2Xw2+FVz4Sk1eTWJ7S5N7bi2QW5Y4jOd4O4Dr8v5UfDL4V3ngrXb3UtRurW4Z4fJt/JLEqC2WJyBg8DpnqapyituglFvfqep1heJLO51/SLjStJ1z+zb0lS08J3SRqDzwGBGcY61u15TrXw48Sab4vvPEfgfV7ezlvtxuYLkcZY5bHysCCeeQMHoayjuaSM34STjw14y1vwXeWsDX6sZTfxli0+3BAbJ9GyMY75yea9D0jwPpOieKdR8RWjXJvdQDiYPICnzMGOBjjkDvWF8P8A4dXXhvVL3X9d1Bb/AFy9BDvHnYgJBbBIGSSB2GAMCtLQNB8T2PjbVtS1PWhdaPcCT7JaCV28nLgr8pGBhQRwe9VJ3bsyYqy1Owrzr4Rtt0nxO2M41+6OP+ApXoted/CH/kGeJv8AsYLr/wBBSpWzKe6D4LZl8Am9c7p7y9nnmb1ctgn9BXV69YXut6b9n0rXZNMkEnz3FvGsjEYIK89OSDkc8Vy/w0iOg3niLwlN8r2F6bi2B/it5RlCPXGDn3NMg8K+LPCWo6h/wiU+lXGl305uPsmo+Ypt5G67SnVfr6D6lv4mJbEHwyaTQta13wbeQxNfWbi7a+jLE3gkwd77iTu5Xvjn2yfS65Lwd4RutFvdS1rWb1L3XNTZTcSxLtjjVeFRAecD1PoPTl3gLUdX1LTNRfWLuC7kh1GWCGeBQI3jUKPlwBnDbhn2pS1dxx00OpkkSJC8jqiDqzHAFcH8QvCOteI7C5ks9edLVIw40to8RT7eSrupDYb8v511XiPQbXxN4evdHvCyw3SbSy9VIIKsPoQD+FcUNE+JX9jDw/8A2noYtBH9n/tILJ9o8rGPu9N+O/655oj3uEux0vgLXYPEfgrTdRt7VLRDH5X2dPuxlCVwvtxx7Vz3wmYwx+KtOX/UWeuXCRegXI4H5Z/Gun0zT9N8B+DFtkdhY6bbvI8jfebGWZj7k5498Vyngew1HS/hfqGqG4hsdU1Rp9SM1z9yEuMqW4PGAD0709LOwux6TXAfFiZ00CziBwr3OW/BT/jXS+Eb2+1HwjpN7qXN5cWySynaFySM9B0rL+JGmvqHhKWSNcvaSCfA/ujIb9Dn8KwrJ8jR6mTzjDHUpS2v+YvjLXG8M+F4o7QETzKIIWH8AA5b8B096801efRItB0q0066+0XKStLdyeWy5YgdyBkDGB/9evXdNjsPEnh3Trm8tYLpWhVsSoHCtjDdfcEVh+KfAMOqJaDR7ewsjG5Mp2bN44wPlHPesasJSV0etleNw2FmqNa8ZKTbelm9Ur9dPzOr0zU7TWLFL2xlMlu5IVipXODg8HmrlcX43hm0rw7AuiXKad5c3EMB8syluNqhRycnNdjCGWCMOcuFAY++K3jJt2Z4VehGNNVoP3ZNpJ7q3fp16HnXxCjSbxd4cikUMjyBWU9CDIuRWPcGXwLqur6RKWbTNRtpPIY84JUhT9QflP4Gu18TeGLvWtf0i/t5oEisnDSLITk4YHjA9qt+L/DKeJtI+zqyR3UTb4JG6A9wfYj+lYypyblJbnu4bMqEKVChUd4NNS8nzXT+Ry/hC+/s34Xajdg4aNpdh/2iAF/UitvwyY/Dfw6gu5xgLA1y4/vFvmA+uCorMHgrVofAx0GOe2aWW7Ekrhzt8vg+nXIFbnizQL7WdAi0rTZYIIwy+Z5pIyijgDAPfB/CiKkle2yM8VVw1aq4c65Z1G2/7qWn33djzXw7rejxQa0+ttM91qSshZI920HJJ69dxB/4CK6T4Uaxvgu9HkbJjPnw/Q8MPzwfxNdlpfhzTtP0q1tHs7aV4owrSNEpLN3PI7nNYcvg66tvHEOuaVJaw23y+dAcrnjawAAxyOfrSjTnBpm1fMcHi41qWsebVNvS8dFZW0ujP8Vk2/xN8Ozx8O+yNsehcj+TGvRK4C7iOtfFu3VBmHS4A0hHQNyR+OWX8jWq+o6uvxOh0xbuB9KfTnne3VQZInDKAzHHAbdwM87TWtJXcn5nlZk0qWHg91Bfi21+B1VedfC1jdah4z1CTmebW5Y2z1Cp90fhuIr0WvPfCkR8PfEvxPokvyxamV1WzPZtxIlH1DEcegrdbM8l7oW6+GNrq/i7WNY1DWruSG98sGztXMIXau0ByCS3A46d6o+HkvPBnxPHhKPULm90e/sjd2yXL73tmBIIB9PlP5jvnOhdeE/EeheKdR1vwjc6c8WqEPeWOobwvmD+NGXnJyeD6nrxi54Z8JanB4iuvFHiW8t7rWZ4hbxR2qkQ20Wc7V3cnJ7n39aq+mrJtrsdnVbUIGutNurdJ/IeWF41lIzsJBAbGRnHXrXO+FtR1e88R+J7XULuC5tLO7WO0aFQBGCCxQnAyygoD1wa6HUtPt9V0y60+6Utb3UTQyAHB2sMHHvzUWsy73R55pvwb0yDQoLWfXdVnuYlPkXMFwY0iySf3aAkAZOepySa1Phbreo6r4fvbTVpzc3mlX0li1wesoXGGJ7nnH4Cs/TvD/xF8PaUNB0rUdDuLCIFLa9ulkE8SdgVAKkjt1rqfCXhm18G+HBYrcGZtzXF1dScGWQ/ec+nQfgKqT01dyUtdDmvAzG2+Jfj2wT/AFH2iC4AHQO6kt+f9KX4Pf8AIA1z/sOXP8kqT4YW73p1/wAVSKyjW75ntwwwfIjJVD+rflUfwe/5AGuf9hy5/klOXX5CXQ9EooorM0CiiigArE8V/wDIHh/7CNj/AOlUVbdYniv/AJA8P/YRsf8A0qipx3RMvhZqUUUVqeeFFFFABRRRQAUUUUAQ3f8Ax5T/APXNv5VQ+Hv/ACTrw7/2D4f/AEEVb1KTydKvJP7kDt+SmoPAcflfD7w6h6/2bbn841P9azq7G9DdnQ0UUVgdBzWkf8lU17/sF2f/AKMnrsq4zRTv+KfiHH8Gm2Sn6l5jXZ16FP4EYS3CiiirJCiiigAooooAKKKKACiiigAooooAKKKKAOa0j/kOeJ/+wkn/AKSW9bFY+kf8hzxP/wBhJP8A0kt62K4Kvxs3jsFFFFZlFXUNOtNUtGtb2ESwsQcEkEEcggjkEHoRyKxjoOs242WGvq0f8I1Cz89l9tyPGT+OT6k10dFXCpKHwsmUVLdHM/2T4p/6Dujf+CiX/wCSaP7J8U/9B3Rv/BRL/wDJNdNRWn1mr3J9lDscz/ZPin/oO6N/4KJf/kmj+yfFP/Qd0b/wUS//ACTXTUUfWavcPZQ7HM/2T4p/6Dujf+CiX/5Jo/snxT/0HdG/8FEv/wAk101FH1mr3D2UOxzP9k+Kf+g7o3/gol/+SaP7J8U/9B3Rv/BRL/8AJNdNRR9Zq9w9lDscz/ZPin/oO6N/4KJf/kmj+yfFP/Qd0b/wUS//ACTXTUUfWavcPZQ7HM/2T4p/6Dujf+CiX/5Jo/snxT/0HdG/8FEv/wAk101FH1mr3D2UOxzP9k+Kf+g7o3/gol/+SaP7J8U/9B3Rv/BRL/8AJNdNRR9Zq9w9lDscz/ZPin/oO6N/4KJf/kmj+yfFP/Qd0b/wUS//ACTXTUUfWavcPZQ7HM/2T4p/6Dujf+CiX/5Jo/snxT/0HdG/8FEv/wAk101FH1mr3D2UOxzP9k+Kf+g7o3/gol/+SaP7J8U/9B3Rv/BRL/8AJNdNRR9Zq9w9lDscz/ZPin/oO6N/4KJf/kmj+yfFP/Qd0b/wUS//ACTXTUUfWavcPZQ7HM/2T4p/6Dujf+CiX/5Jo/snxT/0HdG/8FEv/wAk101FH1mr3D2UOxzo8O6hdjZqmtM8P8UNjD9mD+xYs7gf7rLW7a2sFlax21rCkMEShUjjXCqPQCpaKzlOU/iZUYqOwUUUVBQUUUUAYPjD/kBRf9hGx/8ASuKrVReKraW58M3ogQvPCq3ESDqzxMJFA+pQClt7iK6toriBw8MqB0YdGUjIP5V6OCfutHJiFqiSiiiu05wooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACoPBv8AyI3h/wD7Btt/6KWq2uXj2OiXc8Q3T7NkC/35W+VF/Fio/GtrTbJNN0qzsUOUtoEhU+yqB/SuDGvZHVh1uy1RRRXAdIUUUUAeXfHn/kR7L/sJR/8AoqWij48/8iPZf9hKP/0VLRXbQ+AxnudJ8NDt8A6dbdGtWmtWHoY5XT+ldbXJ+EcWWu+LNGPDW+pm7T08u4USD/x7ePwrrK5aitJmsdgrjdXH2P4naPcH7t/ptxa/8CjdJF/Qv+tdlXI/EJGttIstdRSW0a9ju329TDykoH/AHY/hRTdpEzV4s3qKRWDqGUgqRkEdCKWug4wooooAKKKKACiiigAooooAKKKKACiiigArG8X/APIla9/2Drj/ANFtWzWR4rRpPB2uIgyzafOAPUmNqEC3N6imxussayIQVYBgR3Bp1YnpBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFcvpfw/8AD2ja82tWVvNHes7yFjcOVLPncdpOO5rqKKLtCsZ02iWU2vW2tFGS+t4mgEiNjfG3O1h3API9DWjRRQAU1I0iQJGioo6KowBTqKBhRRRQBn61otnr+nGwv1d7VpEeSNW2iTachW9VyBkVdaKN4jEyK0ZXaUIyCPTHpT6KBBTXRZEZHUMjAhlIyCKdRQMoaRpFroll9jsvMEG8uqu27bnqB7VfoopJW0RU5yqSc5u7ZHJBDM0bSxI7RtuQsoJU9Mj0NSUUUybsKKKKACiiigAooooAztN0Wz0qa7mtlYzXcnmzSO25mP8AhyePer4jQSNIEUOwALY5IGcDP4n86dRSSS2KnUlUlzTd2FZ2oaJZalf6ffzKwurCQyQSo21hkYZT6qR1FaNFMgKKKKBjUjSMEIirkljgYyT1NOoooAKp6rpsGsaVc6dctKsFyhjk8p9rFT1GfccfjVyigRk6l4b0vVNBXRJ4GTT1VFWKFzHtC42gFTkAYFJ4c8M6X4V057DSYXit3lMzB5GcliACck57Cteii72CwUUUUDCiiigArE8V/wDIHh/7CNj/AOlUVbdYnirnSrdB95tRssD1xcxsf0BP4U47omfws1KKKK1PPCiiigAooooAKKKKAOf8c3bWXgbWpU/1jWrxR46l3Gxce+WFdPp1oun6XaWSfdt4UiGPRVA/pXI+JR/amv8Ah7QU+YS3Yv7kdhDBhufrIYx+ddvWVV7I6aK0uFFFFYmxzPhbE/j7xlcryqNZ2ufdIi5/9G12dcb8Nv8AStC1DWQPl1bU7m7T18vd5af+Oxg/jXZV6MVaKRzvcKKKKoQUUUUAFFFFABRRRQAUUUUAFFFFABRRRQBzWkf8hzxP/wBhJP8A0kt62Kx9I/5Dnif/ALCSf+klvWxXBV+Nm8dgooorMoKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACuRjH/CM339nz/LpdxITYzH7sTMcmBj25J2diDt6gZ66ori2gu7eS3uYY5oJFKvHIoZWB7EHrWtKq6croicFNWZQoqgfDt/YcaPqm2AdLW+Qzovsr5Dr+JYDsKjZPFa8LpmjSf7R1KVM/h5B/nXoxxNNrc5HRmjTorLx4t/6A+if+DaX/AORqMeLf+gPon/g2l/8Akaq+sUu4vZT7GpRWXjxb/wBAfRP/AAbS/wDyNRjxb/0B9E/8G0v/AMjUfWKXcPZT7GpRWXjxb/0B9E/8G0v/AMjUY8W/9AfRP/BtL/8AI1H1il3D2U+xqUVl48W/9AfRP/BtL/8AI1GPFv8A0B9E/wDBtL/8jUfWKXcPZT7GpRWXjxb/ANAfRP8AwbS//I1GPFv/AEB9E/8ABtL/API1H1il3D2U+xqUVl48W/8AQH0T/wAG0v8A8jUY8W/9AfRP/BtL/wDI1H1il3D2U+xqUVl48W/9AfRP/BtL/wDI1GPFv/QH0T/wbS//ACNR9Ypdw9lPsalFZePFv/QH0T/wbS//ACNRjxb/ANAfRP8AwbS//I1H1il3D2U+xqUVl48W/wDQH0T/AMG0v/yNRjxb/wBAfRP/AAbS/wDyNR9Ypdw9lPsalFZePFv/AEB9E/8ABtL/API1GPFv/QH0T/wbS/8AyNR9Ypdw9lPsalFZePFv/QH0T/wbS/8AyNRjxb/0B9E/8G0v/wAjUfWKXcPZT7GpRWXjxb/0B9E/8G0v/wAjUY8W/wDQH0T/AMG0v/yNR9Ypdw9lPsalFZePFv8A0B9E/wDBtL/8jUY8W/8AQH0T/wAG0v8A8jUfWKXcPZT7GpTZJEijaSR1REBZmY4AA6kms8Q+KZQFNpo1qe7i7lnx/wAB8tM/nU0XhcXEiza3evqTKQywbBHbqR0Plgnd/wADLY7YqJYqmlpqVGjJ7lbTYn8Q6jBqToy6VanfaBhg3MmMebj+4ATtz1J3dApPU0UV51So6kuZnXGKirIKKKKzKCiiigDy748/8iPZf9hKP/0VLRR8ef8AkR7L/sJR/wDoqWiu2h8BjPc6jWv+JL8SdJ1LGLbWLZtNmboBMmZIifcgyL+VdTWd4x0OXX/DNza2rCO/iK3FlJ/cnjO5D+YwfYmo/Detx+IfD9nqaIY3lTEsR6xSA7XQ+4YEfhWWIjrzFU30NWobu1hvrOe0uEDwTxtFIh/iVhgj8jU1Fc5ocT4MuJobK58P3sha+0WT7KzN1khxmGT/AIEmPxU101c94wsrjTL638W6dC0stnGYr+CMfNPak5OB3ZD8w9RuHetu0u4L6zhu7WVZbeZBJHIpyGUjIIrqi+ZXOOpHlZNRRRTICiiigAooooAKKKKACiiigAooooAKRlV1KsAVIwQe4paKAMDTtRTw5Gmk6tJ5NrF8lneyH928f8KO3RXAwvP3sAjkkDpEkSVA8bq6nkMpyDULKrqVYBlIwQRkGseXwh4ZncvL4d0mRz1Z7KMn/wBBqXFM3jXaVmjoKK5z/hC/Cv8A0LOjf+AEX/xNH/CF+Ff+hZ0b/wAAIv8A4mlyeZX1jyOjornP+EL8K/8AQs6N/wCAEX/xNH/CF+Ff+hZ0b/wAi/8AiaOTzD6x5HR0Vzn/AAhfhX/oWdG/8AIv/iaP+EL8K/8AQs6N/wCAEX/xNHJ5h9Y8jo6K5z/hC/Cv/Qs6N/4ARf8AxNH/AAhfhX/oWdG/8AIv/iaOTzD6x5HR0Vzn/CF+Ff8AoWdG/wDACL/4mj/hC/Cv/Qs6N/4ARf8AxNHJ5h9Y8jo6K5z/AIQvwr/0LOjf+AEX/wATR/whfhX/AKFnRv8AwAi/+Jo5PMPrHkdHRXOf8IX4V/6FnRv/AAAi/wDiaP8AhC/Cv/Qs6N/4ARf/ABNHJ5h9Y8jo6K5z/hC/Cv8A0LOjf+AEX/xNH/CF+Ff+hZ0b/wAAIv8A4mjk8w+seR0dFc5/whfhX/oWdG/8AIv/AImj/hC/Cv8A0LOjf+AEX/xNHJ5h9Y8jo6K5z/hC/Cv/AELOjf8AgBF/8TR/whfhX/oWdG/8AIv/AImjk8w+seR0dFc5/wAIX4V/6FnRv/ACL/4mj/hC/Cv/AELOjf8AgBF/8TRyeYfWPI6Oiuc/4Qvwr/0LOjf+AEX/AMTR/wAIX4V/6FnRv/ACL/4mjk8w+seR0dFc5/whfhX/AKFnRv8AwAi/+Jo/4Qvwr/0LOjf+AEX/AMTRyeYfWPI6Oiuc/wCEL8K/9Czo3/gBF/8AE0f8IX4V/wChZ0b/AMAIv/iaOTzD6x5HR0Vzn/CF+Ff+hZ0b/wAAIv8A4mj/AIQvwr/0LOjf+AEX/wATRyeYfWPI6Oiuc/4Qvwr/ANCzo3/gBF/8TR/whfhX/oWdG/8AACL/AOJo5PMPrHkdHRXOf8IX4V/6FnRv/ACL/wCJo/4Qvwr/ANCzo3/gBF/8TRyeYfWPI6Oiuc/4Qvwr/wBCzo3/AIARf/E0f8IX4V/6FnRv/ACL/wCJo5PMPrHkdHRXOf8ACF+Ff+hZ0b/wAi/+Jo/4Qvwr/wBCzo3/AIARf/E0cnmH1jyOjornP+EL8K/9Czo3/gBF/wDE0f8ACF+Ff+hZ0b/wAi/+Jo5PMPrHkdHRXOf8IX4V/wChZ0b/AMAIv/iaP+EL8K/9Czo3/gBF/wDE0cnmH1jyOjornP8AhC/Cv/Qs6N/4ARf/ABNH/CF+Ff8AoWdG/wDACL/4mjk8w+seR0dFc5/whfhX/oWdG/8AACL/AOJo/wCEL8K/9Czo3/gBF/8AE0cnmH1jyOjornP+EL8K/wDQs6N/4ARf/E0f8IX4V/6FnRv/AAAi/wDiaOTzD6x5HR0Vzn/CF+Ff+hZ0b/wAi/8AiaP+EL8K/wDQs6N/4ARf/E0cnmH1jyOjornP+EL8K/8AQs6N/wCAEX/xNH/CF+Ff+hZ0b/wAi/8AiaOTzD6x5HR0Vzn/AAhfhX/oWdG/8AIv/iaP+EL8K/8AQs6N/wCAEX/xNHJ5h9Y8jbvL600+Az3t1DbRDq80gQD8TWJFJJ4g1S3vfKki0uzJe381SrXEpBXftPIRVLYz1Jz0AJs2fhvQtOlEtlounW0g5Dw2qIR+IFadNRSInWclZBRRRVGIUUUUAFFFFABSEhQSSABySe1LXK+I5ptf1FPCOnuymdRJqdwhx9ntieVz2eTlQPTJoGld2RP4LQ6zqWpeLJAfLuz9k0/Pa1jJ+Yf777m+gWuzqO3t4bS2itreNY4YkCRoowFUDAA/CpK5pO7udsVZWCue8b6pNpXhO8e1G6+uQLS0QdWmlOxMfQtn8K6GuSIPiX4jQ2wG7TvDqi4mPZ7yRSI1/wCAIWb2LCqpx5pWFJ2R1eiaXFomhWGlQf6qzt0gU467VAz+OM1foorvMAooooAKKKKACiiigAooooAKKKKACiiigAooooA5rSP+Q54n/wCwkn/pJb1sVj6R/wAhzxP/ANhJP/SS3rYrgq/GzeOwUUUVmUFFFFABRRRQAUUUUAFFFFABRRTJZY4IXmmkWOKNSzu5wFA5JJ7CgB9Fc+uv39+vmaPo/nW5+5cXs/2ZJB6qArvj3KjPbIpv9oeK/wDoB6L/AODeX/5GquVgdFRXO/2h4r/6Aei/+DeX/wCRqP7Q8V/9APRf/BvL/wDI1HKwOiornf7Q8V/9APRf/BvL/wDI1H9oeK/+gHov/g3l/wDkajlYHRUVzv8AaHiv/oB6L/4N5f8A5Go/tDxX/wBAPRf/AAby/wDyNRysDoqK53+0PFf/AEA9F/8ABvL/API1H9oeK/8AoB6L/wCDeX/5Go5WB0VFc7/aHiv/AKAei/8Ag3l/+RqP7Q8V/wDQD0X/AMG8v/yNRysDoqK53+0PFf8A0A9F/wDBvL/8jUf2h4r/AOgHov8A4N5f/kajlYHRUVzv9oeK/wDoB6L/AODeX/5Go/tDxX/0A9F/8G8v/wAjUcrA6Kiud/tDxX/0A9F/8G8v/wAjUf2h4r/6Aei/+DeX/wCRqOVgdFRXO/2h4r/6Aei/+DeX/wCRqP7Q8V/9APRf/BvL/wDI1HKwOiornf7Q8V/9APRf/BvL/wDI1H9oeK/+gHov/g3l/wDkajlYHRUVzv8AaHiv/oB6L/4N5f8A5Go/tDxX/wBAPRf/AAby/wDyNRysDoqK53+0PFf/AEA9F/8ABvL/API1H9oeK/8AoB6L/wCDeX/5Go5WB0VFc7/aHiv/AKAei/8Ag3l/+RqP7Q8V/wDQD0X/AMG8v/yNRysDoqK53+0PFf8A0A9F/wDBvL/8jUf2h4r/AOgHov8A4N5f/kajlYHRUVzv9oeK/wDoB6L/AODeX/5Go/tDxX/0A9F/8G8v/wAjUcrA6Kiud/tDxX/0A9F/8G8v/wAjUf2h4r/6Aei/+DeX/wCRqOVgdFRXO/2h4r/6Aei/+DeX/wCRqP7Q8V/9APRf/BvL/wDI1HKwOiornf7Q8V/9APRf/BvL/wDI1H9oeK/+gHov/g3l/wDkajlYHRUVzv8AaHiv/oB6L/4N5f8A5Go/tDxX/wBAPRf/AAby/wDyNRysDoqK53+0PFf/AEA9F/8ABvL/API1H9oeK/8AoB6L/wCDeX/5Go5WB0VFc7/aHiv/AKAei/8Ag3l/+RqP7Q8V/wDQD0X/AMG8v/yNRysDoqK54ah4qz82iaPj/Z1aUn/0mFWLPXxJex2OoWcun3kmfKWRg0c2BkhHHBOOdpw2ATjAzRysDZoooqQCiiigAooooAKKKKACiiigAooooAKKKKAPLvjz/wAiPZf9hKP/ANFS0UfHn/kR7L/sJR/+ipaK7aHwGM9z16uCuV/4Q3xqZyduha/KN5/htr7HX2EoH/fQ9672qWr6TZ67pFzpmoRCW1uUKSL39iD2IOCD2IrSUVJWZKdncKK5Pw7qt5pmo/8ACJ6/KX1GFC1ldtwL+AdGH/TRRgMv48g11lcEouLszdO4VwF1DJ8P76S4iR38K3Ll5UQbjpshPLAf88WJyR/Ceehrv6R0V0ZHUMrDBUjIIojLlYpRUlZmdFLHNEksTq8bqGV1OQwPQg9xTq5abQdU8IytceG4mvtHYlpdHLgNDnktbseMd/LPHoR0rV0bxBpuvQu9jPmSI7ZreRSksLf3XQ8qa6E09UckoOO5qUUUUyQooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAopksscMTSyuscaDLO5wAPUmuXbX9R8TO1p4SjAt87ZdanT9xH6+Up/wBa3/jvqTQNRb0Rb17X5ra5j0bRolu9duVzHEfuQJ0Msp/hQfmx4Fa/hvw9D4e09ohK9zeTv513dyffuJT1Y+g7AdAABTvD/hyx8O20iWxkmuZ233N3O26W4f8AvM38gOB2Fa9YTnfRHVCHKFFFVdS1Kz0jTp9Qv50gtYELySOeAP6n271BoZ3inXToOkGWCL7RqFw4t7G2HWaduFX6dyewBq74T0D/AIRzQY7OSX7ReSM095cnrPO/Lt+fA9gKxfC+mXetav8A8JfrMDwOUMelWUvW1hPV2HaR+/oMD1rta7aVPlWu5hKV2FFFFakhRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQBzWkf8hzxP8A9hJP/SS3rYrH0j/kOeJ/+wkn/pJb1sVwVfjZvHYKKKKzKCiiigAooooAKKKKACiiigArn9fUX+q6Vo8nNvL5l3Oh6SJFsAQ+2+RCfXbjoa6CsC//AOR30n/sHXv/AKMtaqO4GtRRRVlhRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABVTU9Og1WwktJ9wVsFXQ4aNhyrqezA4IPqKt0UCKvh++m1HRLee52/aVLwzlRgGWNzG5A7DcprTrD8Jf8gWb/ALCN/wD+lctblZvckKKKKQBRRRQAUUUUAFFFFABRRRQAUUUUAeXfHn/kR7L/ALCUf/oqWij48/8AIj2X/YSj/wDRUtFdtD4DGe569RRRWxBj+I/Dln4k04W1wzwzxOJbW6hOJbeUdHQ+vt0I4NYGj+Iryz1JPD3ilY7fVTxbXSjbBqCj+JPR/VOo6jIrt6z9a0TTvEOmyafqlqlxbPztbgqR0ZSOVYdiOaidNTWpUZND6K45n8R+CsJdR3HiHQl+7dRDde2y/wDTRf8AlqB/eX5uuQa6LSda0zXbMXel3sN3AerRNnafRh1U+xwa4p05R3NVJMv1ha34R0jXplup4Xgv4xiK+tHMU6fRx1Hscj2rdoqU2tirXOLNj400X/UT2XiG2HRZ/wDRbkD/AHgCjfiFph8ZrafLrGg63pzD7zNZtPGPo8W4V29FaKq+pk6UWcQvxF8IE7W160jb+7MTGR9QwGKk/wCE/wDCH/QyaZ/4ELXZMquMMoYehGag+w2n/PrB/wB+xT9quxPsF3OU/wCE/wDCH/QyaZ/4ELR/wn/hD/oZNM/8CFrq/sNp/wA+sH/fsUfYbT/n1g/79in7Vdg9h5nKf8J/4Q/6GTTP/AhaP+E/8If9DJpn/gQtdX9htP8An1g/79ij7Daf8+sH/fsUe1XYPYeZyn/Cf+EP+hk0z/wIWj/hP/CH/QyaZ/4ELXV/YbT/AJ9YP+/Yo+w2n/PrB/37FHtV2D2Hmcp/wn/hD/oZNM/8CFo/4T/wh/0Mmmf+BC11f2G0/wCfWD/v2KPsNp/z6wf9+xR7Vdg9h5nKf8J/4Q/6GTTP/AhaP+E/8If9DJpn/gQtdX9htP8An1g/79ij7Daf8+sH/fsUe1XYPYeZyn/Cf+EP+hk0z/wIWj/hP/CH/QyaZ/4ELXV/YbT/AJ9YP+/Yo+w2n/PrB/37FHtV2D2Hmcp/wn/hD/oZNM/8CFo/4T/wh/0Mmmf+BC11f2G0/wCfWD/v2KPsNp/z6wf9+xR7Vdg9h5nKf8J/4Q/6GTTP/AhaP+E/8If9DJpn/gQtdX9htP8An1g/79ij7Daf8+sH/fsUe1XYPYeZyn/Cf+EP+hk0z/wIWj/hP/CH/QyaZ/4ELXV/YbT/AJ9YP+/Yo+w2n/PrB/37FHtV2D2Hmcp/wn/hD/oZNM/8CFo/4T/wh/0Mmmf+BC11f2G0/wCfWD/v2KPsNp/z6wf9+xR7Vdg9h5nKf8J/4Q/6GTTP/AhaP+E/8If9DJpn/gQtdX9htP8An1g/79ij7Daf8+sH/fsUe1XYPYeZyn/Cf+EP+hk0z/wIWj/hP/CH/QyaZ/4ELXV/YbT/AJ9YP+/Yo+w2n/PrB/37FHtV2D2Hmcp/wn/hD/oZNM/8CFo/4T/wh/0Mmmf+BC11f2G0/wCfWD/v2KPsNp/z6wf9+xR7Vdg9h5nKf8J/4Q/6GTTP/AhaP+E/8If9DJpn/gQtdX9htP8An1g/79ij7Daf8+sH/fsUe1XYPYeZyn/Cf+EP+hk0z/wIWj/hP/CH/QyaZ/4ELXV/YbT/AJ9YP+/Yo+w2n/PrB/37FHtV2D2Hmcp/wn/hD/oZNM/8CFo/4T/wh/0Mmmf+BC11f2G0/wCfWD/v2KPsNp/z6wf9+xR7Vdg9h5nKf8J/4Q/6GTTP/AhaP+E/8If9DJpn/gQtdX9htP8An1g/79ij7Daf8+sH/fsUe1XYPYeZyn/Cf+EP+hk0z/wIWj/hP/CH/QyaZ/4ELXV/YbT/AJ9YP+/Yo+w2n/PrB/37FHtV2D2Hmcp/wn/hD/oZNM/8CFo/4T/wh/0Mmmf+BC11f2G0/wCfWD/v2KPsNp/z6wf9+xR7Vdg9h5nKf8J/4Q/6GTTP/AhaP+E/8If9DJpn/gQtdX9htP8An1g/79ij7Daf8+sH/fsUe1XYPYeZyn/Cf+EP+hk0z/wIWj/hP/CH/QyaZ/4ELXV/YbT/AJ9YP+/Yo+w2n/PrB/37FHtV2D2Hmcp/wn/hD/oZNM/8CFo/4T/wh/0Mmmf+BC11f2G0/wCfWD/v2KPsNp/z6wf9+xR7Vdg9h5nKf8J/4Q/6GTTP/AhaP+E/8If9DJpn/gQtdX9htP8An1g/79ij7Daf8+sH/fsUe1XYPYeZyn/Cf+EP+hk0z/wIWj/hP/CH/QyaZ/4ELXV/YbT/AJ9YP+/Yo+w2n/PrB/37FHtV2D2Hmcp/wn/hD/oZNM/8CFo/4T/wh/0Mmmf+BC11f2G0/wCfWD/v2KPsNp/z6wf9+xR7Vdg9h5nKf8J/4Q/6GTTP/AhaP+E/8If9DJpn/gQtdX9htP8An1g/79ij7Daf8+sH/fsUe1XYPYeZyn/Cf+EP+hk0z/wIWj/hP/CH/QyaZ/4ELXV/YbT/AJ9YP+/Yo+w2n/PrB/37FHtV2D2Hmcp/wn/hD/oZNM/8CFo/4T/wh/0Mmmf+BC11f2G0/wCfWD/v2KPsNp/z6wf9+xR7Vdg9h5nKf8J/4Q/6GTTP/AhaP+E/8If9DJpn/gQtdX9htP8An1g/79ij7Daf8+sH/fsUe1XYPYeZyn/Cf+EP+hk0z/wIWj/hP/CH/QyaZ/4ELXV/YbT/AJ9YP+/Yo+w2n/PrB/37FHtV2D2Hmcp/wn/hD/oZNM/8CFo/4T/wh/0Mmmf+BC11f2G0/wCfWD/v2KPsNp/z6wf9+xR7Vdg9h5nKf8J/4Q/6GTTP/AhaP+E/8If9DJpn/gQtdX9htP8An1g/79ij7Daf8+sH/fsUe1XYPYeZyn/Cf+EP+hk0z/wIWj/hP/CH/QyaZ/4ELXV/YbT/AJ9YP+/Yo+w2n/PrB/37FHtV2D2Hmcp/wn/hD/oZNM/8CFpr/EPwegyfEenH/dmB/lXW/YbT/n1g/wC/YqSO3hh/1cMaf7qgUe1XYPYeZxf/AAn+hzcaeNQ1J/7tjYTS5/4EF2/rThqPi7VPl0zw7Hp0bdLnV5wCB/1yjLHPsWFdtRUur2Q1RitzkIfAkd7Ktx4o1KbXJFOVt5FEVqh9oV4b6uWrrY40ijWONFRFACqowAB2Ap1FQ5N7mqSWwUUVy+o+M4ftr6X4etX1zVhw0NswEUHvNL91B7cn2oUXJ2QN2NvVtXsND06S/wBSuUt7aPq7nqewA6knsBya57TNH1DxdqEGteIrZ7XTLdxLp2kSD5iw6TTj+96J0XvzV3R/B0r6lFrnie6TUtWj5gjRSLaz9okPVv8Abbn6V11ddOko6vcylO4UUUVsQFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQBzWl/L4g8SqerX0cn4G2hH/sprYrJh/c+NdWi7S2lrOPc5lQ/kEX861q4Kq99m8NgooorMoKKKKACiiigAooooAKKKKACsC//AOR30n/sHXv/AKMta36wL/8A5HfSf+wde/8Aoy1qo7ga1Z+t6vBoWkz6hchmjiA+VerEnAA/OtCvOfiJcyarrGleGbZvmlkEkuO2Thc/Qbj+VFSXLG6PQy7CrE4iMJfDu/Ras6nwz4ptPFFtPLbRyRNC4V45MZwRweO3X8q3a8vsETwb8TTZJlLC/UKgJ4G77v5MCPoa2/iLruo6FZ2Eun3BhaSVg+FU7gB05BqI1LQblujtxGWKeLhTw3w1FeN/TVP7jtaK8y8Q3vjLRbKDXJtUhCSOoa0jjG2PIyAcjnpgnrW14m8ZS6X4Y067tIl+26lGrRKwyEBUEnHfG4D8aftVrfSxi8prN0/ZyUudtaPZre90dnRXml+fHPh3Tl1m51SG5RSDPbFQQoJ+g7nBwateKvF14PCGlavpM5tmupMONqtjAOV5HYil7ZJO6KWT1Jygqc4yUna6bsna9npc9BorzDXtT8YaNZWevTalCIrh1H2NIxtTILBTxzwDk5zSa3qni+w0m18RvqUMcM7KVs44wQisMrnI5469+aHWSvoy4ZLUnytVI+9otXuum347HqFFczq19r994asbnQIYluLuNJZHdlHlKyg8buD1/SuPv9d17wvqdn5viK21RJHxNAm07RkZB7jrweOlOVVR6GWGyqpiFaM482umt9PRNLyvudxrHim30bWdP02W3lkkvWVUdCMLltvNb1eb+Ov+R88Nf9dI/wD0aK0Jdc1TS/iTHpl5dF9NvF/cKyKNpYccgZ+8COvekqlpNPuayy1VKFOVL4nGUnfrZ9POx3FFcRea5qd78SLfRdPujFZwKGugqKd2BuPJGR1VePWu3rSMlK9uh52Iws6Cg5te8r+ie1zhr74m2FjqV1ZNp91I9vK0TMpXBKnGevtVrRviNo+rXyWbpPaTSHannAbWPYZB4P1rD8Gf8lO8RfWf/wBHCp/ivYwDSbPUVRVuVuBFvAwSpVjyfYqK5+efK532PoJYPAvEwwfI05Je9fq1fb/gnolFcBr/AIuv7XR9CtbBkXUtTgicyyYwm4DnnjJJ6npg1Ul1vXvCWp2J1PV4NUsLltsu3GY+mSO/fPocHpWrrRTPNp5PXnFO6Td7LW7tvbS3pd6nc6hrun6ZeWlncT4ubqRUiiUZY5OM+w960a8j8Z22rf8ACwrBRexiWZ0Nk23iEb8DPHOGye9b3iTxDrOlLpWgW9zCdXuUXzrtgAoycAjIwMkHnHGKlVdXdbGsso5oUfYyTc02/Lz22X33O+orzhdc1vwtrthb6pqsGqWN42wuoG6M5AJ454yPqM1c8SeINXuvFUPhnQpUt5ioaadgCR8u7HIOAF59ear2qsY/2RVc0oyTi05c2trLfpfT0O7rOGu6e+t/2PHPvvQhd0UZCAY6n156Vzujr4w0zxAllqDjUtOdcm5AVfL6/Q9RyOeDXG6dY+IG+IV3bxanCuqKpMtyUG1hhcgDb6Y7VMqrVrLqbYfKac3U5qsbRjzJp6fPS6t1W+x7LRXn2teIdW1Txa/h/R72Kwit1zNcvjJIAz19MgYHepvDniDVLXxU/hzWLuK9LoWguY8c8bsHHsD75Heq9qr2MHlNZUfaXV+Xm5db8vfa3yvc7usH/hKbf/hLh4d+zy+ft3ebkbfubvrXKprHiLxb4lvbTR79bCxsmI37AS3OATxyTg8dMVQ0c6j/AMLdjTVTG14iMjvGMK4ERw34jBqXVvbl7nVRyhQjU9vJOSg5cqbunur9PxPV6K54Q+Iv+EyMvnp/YW3/AFXy5zs+mfve9dDWydzxqtL2fL7yd0np0v0fn3MXWvEUek32n2KQNc3d7KESNW27V7sTzx/9f0qt4h8aad4euEtZUmuLtwCIYQCQD0yT/wDrrA0tzq/xc1CeTlNPhKRA/wAJGF/mzn8a6ibQ9Is9Zl8Rzjy7lUJeaSQ7VG3bnB4HArJSlJNruenOhhcPOnCtFt8t2l1k9l5JLtqVfD3jTTvENw9pGk1tdoCTDMACQOuCPSukrzLQGPib4lz67aRMlhbKR5hGN52bB+Jzn6CvTadKTkrsxzTDU8PWUaateKbV78rfQK4m++JulW13JDbWt1eJEcPLEo2/UZPI9+K7SSNZYnjbO11KnBxwa5W6GhfD/Qrkwx488kpCzbmlbGMc/wAPr6ZPrTqOS2dkTgI0JScakHOTskk7et2bOka9p+t6X/aFpL+5XIk3/KYyOSG9OK5qT4o6Ol0UW1vJLZW2m4VBt+uM5x+vtXP6NY3ek/C3Wr2UNGbwDy1PB2Ehc/jk/hitDS7SFvgxOSi/PHLK3HVlc4P/AI6Ky9pNpW0drnrf2dg6U5yleUedQWtrXV2/O2x3kuq2MOknVHuF+xCMS+aOhU9P/wBVcpa/E/SJ7yOKW2u7eCRtqXEijb+ODwPzrkrq8lPwfsotx2m+MJ/3fnfH54rc8W2kKfCvSwqKDEsDKQO5Xn88mh1ZPVdFcKWVYanJU6ycnKbgne1kuvmzrtc8RRaDcWAuYCbW7k8prgNxEe2R6H+hrarg9Zi/tP4Rwyy/NLHaxShj6rgE/ln866DwffvqXhLTrmQ5cxbGJ6kqSuf0rSM25W8rnl4jCRhhlVjvGTi/Pqn+a+RuUUUVqeaZnhL/AJAs3/YRv/8A0rlrcrD8Jf8AIFm/7CN//wClctblRLdkBRRRUgFFFFABRRRQAUUUUAFFFFABRRRQB5r8a7dr3wpY20Yyxvlkx7BHH/sworp9c05Nb8Radpsh/di0uJ2PoQ8Kr+e5vyoruor3DCe52FFFFakhRRRQAVzOseBtI1S+OpQefpmrf8/+nv5Ujf74+64/3ga6aigDhzH470Q4aOw8SWo6MhFndfiDmNvzWm/8LC0y1yutWGraKw6m+snEZ+kibkP513VFZSowZSm0cvZ+MvDOoKDaeINMl/2RdJu/LOa0U1KwkGY722ceqyqf61JeeHND1Ak3ui6dck9TNao+fzFZr/DvwZIcnwtpA/3bNB/IVn9XXcr2hofbrT/n6g/7+Cj7daf8/UH/AH8FZn/Ct/BX/Qr6V/4DL/hR/wAK38Ff9CvpX/gMv+FH1fzD2hp/brT/AJ+oP+/go+3Wn/P1B/38FZn/AArfwV/0K+lf+Ay/4Uf8K38Ff9CvpX/gMv8AhR9X8w9oaf260/5+oP8Av4KPt1p/z9Qf9/BWZ/wrfwV/0K+lf+Ay/wCFH/Ct/BX/AEK+lf8AgMv+FH1fzD2hp/brT/n6g/7+Cj7daf8AP1B/38FZn/Ct/BX/AEK+lf8AgMv+FH/Ct/BX/Qr6V/4DL/hR9X8w9oaf260/5+oP+/go+3Wn/P1B/wB/BWZ/wrfwV/0K+lf+Ay/4Uf8ACt/BX/Qr6V/4DL/hR9X8w9oaf260/wCfqD/v4KPt1p/z9Qf9/BWZ/wAK38Ff9CvpX/gMv+FH/Ct/BX/Qr6V/4DL/AIUfV/MPaGn9utP+fqD/AL+Cj7daf8/UH/fwVmf8K38Ff9CvpX/gMv8AhR/wrfwV/wBCvpX/AIDL/hR9X8w9oaf260/5+oP+/go+3Wn/AD9Qf9/BWZ/wrfwV/wBCvpX/AIDL/hR/wrfwV/0K+lf+Ay/4UfV/MPaGn9utP+fqD/v4KPt1p/z9Qf8AfwVmf8K38Ff9CvpX/gMv+FH/AArfwV/0K+lf+Ay/4UfV/MPaGn9utP8An6g/7+Cj7daf8/UH/fwVmf8ACt/BX/Qr6V/4DL/hR/wrfwV/0K+lf+Ay/wCFH1fzD2hp/brT/n6g/wC/go+3Wn/P1B/38FZn/Ct/BX/Qr6V/4DL/AIUf8K38Ff8AQr6V/wCAy/4UfV/MPaGn9utP+fqD/v4KPt1p/wA/UH/fwVmf8K38Ff8AQr6V/wCAy/4Uf8K38Ff9CvpX/gMv+FH1fzD2hp/brT/n6g/7+Cj7daf8/UH/AH8FZn/Ct/BX/Qr6V/4DL/hR/wAK38Ff9CvpX/gMv+FH1fzD2hp/brT/AJ+oP+/go+3Wn/P1B/38FZn/AArfwV/0K+lf+Ay/4Uf8K38Ff9CvpX/gMv8AhR9X8w9oaf260/5+oP8Av4KPt1p/z9Qf9/BWZ/wrfwV/0K+lf+Ay/wCFH/Ct/BX/AEK+lf8AgMv+FH1fzD2hp/brT/n6g/7+Cj7daf8AP1B/38FZn/Ct/BX/AEK+lf8AgMv+FH/Ct/BX/Qr6V/4DL/hR9X8w9oaf260/5+oP+/go+3Wn/P1B/wB/BWZ/wrfwV/0K+lf+Ay/4Uf8ACt/BX/Qr6V/4DL/hR9X8w9oaf260/wCfqD/v4KPt1p/z9Qf9/BWZ/wAK38Ff9CvpX/gMv+FH/Ct/BX/Qr6V/4DL/AIUfV/MPaGn9utP+fqD/AL+Cj7daf8/UH/fwVmf8K38Ff9CvpX/gMv8AhR/wrfwV/wBCvpX/AIDL/hR9X8w9oaf260/5+oP+/go+3Wn/AD9Qf9/BWZ/wrfwV/wBCvpX/AIDL/hR/wrfwV/0K+lf+Ay/4UfV/MPaGn9utP+fqD/v4KPt1p/z9Qf8AfwVmf8K38Ff9CvpX/gMv+FH/AArfwV/0K+lf+Ay/4UfV/MPaGn9utP8An6g/7+Cj7daf8/UH/fwVmf8ACt/BX/Qr6V/4DL/hR/wrfwV/0K+lf+Ay/wCFH1fzD2hp/brT/n6g/wC/go+3Wn/P1B/38FZn/Ct/BX/Qr6V/4DL/AIUf8K38Ff8AQr6V/wCAy/4UfV/MPaGn9utP+fqD/v4KPt1p/wA/UH/fwVmf8K38Ff8AQr6V/wCAy/4Uf8K38Ff9CvpX/gMv+FH1fzD2hp/brT/n6g/7+Cj7daf8/UH/AH8FZn/Ct/BX/Qr6V/4DL/hR/wAK38Ff9CvpX/gMv+FH1fzD2hp/brT/AJ+oP+/go+3Wn/P1B/38FZn/AArfwV/0K+lf+Ay/4Uf8K38Ff9CvpX/gMv8AhR9X8w9oaf260/5+oP8Av4KPt1p/z9Qf9/BWZ/wrfwV/0K+lf+Ay/wCFH/Ct/BX/AEK+lf8AgMv+FH1fzD2hp/brT/n6g/7+Cj7daf8AP1B/38FZn/Ct/BX/AEK+lf8AgMv+FH/Ct/BX/Qr6V/4DL/hR9X8w9oaf260/5+oP+/go+3Wn/P1B/wB/BWZ/wrfwV/0K+lf+Ay/4Uf8ACt/BX/Qr6V/4DL/hR9X8w9oaf260/wCfqD/v4KPt1p/z9Qf9/BWZ/wAK38Ff9CvpX/gMv+FH/Ct/BX/Qr6V/4DL/AIUfV/MPaGn9utP+fqD/AL+Cj7daf8/UH/fwVmf8K38Ff9CvpX/gMv8AhR/wrfwV/wBCvpX/AIDL/hR9X8w9oaf260/5+oP+/go+3Wn/AD9Qf9/BWZ/wrfwV/wBCvpX/AIDL/hR/wrfwV/0K+lf+Ay/4UfV/MPaGn9utP+fqD/v4KPt1p/z9Qf8AfwVmf8K38Ff9CvpX/gMv+FH/AArfwV/0K+lf+Ay/4UfV/MPaGn9utP8An6g/7+Cj7daf8/UH/fwVmf8ACt/BX/Qr6V/4DL/hR/wrfwV/0K+lf+Ay/wCFH1fzD2hp/brT/n6g/wC/go+3Wn/P1B/38FZn/Ct/BX/Qr6V/4DL/AIUf8K38Ff8AQr6V/wCAy/4UfV/MPaGn9utP+fqD/v4Krza7pFuCZtVsYwOpe4QfzNVP+FceC/8AoV9K/wDAZf8ACrMPgbwlbtuh8MaMjf3hYx5/PbR9X8w9oZM/xF8JQyGJNbt7qXtHZBrhj9BGGqMeJtf1QFdC8I3wB4FzqzC0jHvt5kI/4CK7O3tbe0j8u2t4oU/uxIFH5CpqtUIrcTmziR4L1bWxnxZrzzwN97TtMBtrc+zPnzHH4ge1dXpul2Gj2SWWm2cFpbJ92KFAqj3wO/vVuitUktiG7hRRRTAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooA53VR9m8X6Rc9EuYJ7Rvd/llT/AMdSX861azvF6MmhHUEUmTTZUvRgZO1D+8A9zGZB+NaCsrqGUgqRkEdCK5MQveua03oLRRRXOaBRRRQAUUUUAFFFFABRRRQAVgX/APyO+k/9g69/9GWtb9YF/wD8jvpP/YOvf/RlrVR3A1WZURnY4VRkn0FeSaX4el8deIdV1K+e4tIQ48shcMc8KOfRRXrlFKdNTavselgsfPBxn7Je9JJX7K+unmeSeKfh8NE0n+0dPurq4likXcrAZCnuMe+Kn8cXc+ueEdCvBBIZXZvNUIcqwGDx9Qa9UoqHQWqWiZ2088q3pzrR5pQbad7aNWtt8zjPiTFJL4OVY0Z28+PhRk9DWR4l0G+1DwV4dvLKF5Liwtoy0QGWwUXkDvgqOPevSqKqVJSbv1OfDZpPDwpxhH4ZN+t1Zr7jy/WvG0viXRW0XT9Iu/t9xtSVSuQmCCcd+3cDFR+L9En0rwBounbGlmimJk2AnDMGY/hk4r1SipdJu/M9Tanm8aMoKhS5YxfNa97u1t7djgfiNDLL4J09I43dhPFkKpJH7tqTxpDLJ8N7CNI3ZwIMqFJP3PSu/oqnSvfXcxo5m6apLl+CTlvvfoeWeLk1BfB/hkCOc6cttH9rSPIOdiY3fhuxnvWL4hGm3dtYzeH9DuYLOJ8SXDRHMjHBC5yScAHv3r22iolQv1OvD557FRXs/hbfxNJ37q2rXR/gec+NY5J/G/hqWKN3jDxksqkgDzBV34mac76Rbavb8XGnyhtw6hSRz+DBf1rua4rxXoHibX7qSzt761i0eQodjcNwBnOBk8jOM05wspdbkYHGqdagpNQVNO7b3Teq23syt8NrWW7Op+Ibofv72Yqp9s5bHtkgf8BrvqpaTpsOj6VbafBzHAgUE/xHufxOT+NXa0px5YpHn5hiVicTKpHbp6LRfgeP2OsSeG/HmuXsmn3Nwkss0aiNfWTOf0q1q95q/wAQrm0sLTSprOyjk3ySyg4B6ZJwBwM8dTmvVqKz9i7Wvoei85hzqtGivaJWTbb2Vr22PNviB4ekibSdQtrNruzskWGaBc58tTkZxzgjIJ7Vm27+H9Y1S1tND8JtMrH/AEh53dBH75DEADnr+Fet0U3RTlcijnM4UVTkm3G9mpNb90t7eqPNPiCZNO8XaHq7wSPawbCxQZ5V9xH1war+MbZtUn0jxTHp01zp7RqtxbkEMFDE846ZBPPtXqdFEqN767io5w6SpWhrBNXvunfTyfmeUWB0HWNctbfQ/ChkgyDNNO7p5fPXIYjj9TV3xJFeeGvH0fiZbSW5sZVAl8sZ2nZsIPpwARnrXpVFHsdN9RvOH7Xm5G48ri05Ntp76vb5I4bSvF2q+I/EkKaVZNFo6qPPkuI+e+cEHAJ4AGT61h3uonwz8Ub2/u7S4khmTEflLktlV5GevIxXqtFDptrV6kU8ypU5y5aKUHHltfX1btv8jybxDplvovje41HV9MlvdHu8vvQH5GOPQjkEHgnoa1PCCadqHiNrrTPDQtrGAEx3kjuGzjGMZIJOT9BXotFCopSuVUziVSh7Nxd+XlvzO1u/Lte3y8jyqwvZfh/4q1NL+yuJbK8bMUsK5JGSVxnAP3iCM07SJr3UPitDqVxYTWqTIzIki8qnlELn0Jxn8a9Topex2V9E7lSziMlKTp+/KPK3fyte39fI54eJZf8AhMjoH9mv5e3d9r3nH3N3Tb+HWuhoorZJrdnk1Z05cvJHlsknre77+V+x574fQ2HxX1u2k4+0RtIh9clW/kT+VY/xC1m91DWTpUcNx/Z1sw8wRqf3rdSc47dB7812uueH7m48RaXrmmmMXNs2yZXOBJEc55x1wT+ftXS1h7NtON7antLMqVKtTxXKpPlSte1mtL/dt6nBeE/Fdu11Z6JZ+H57KBsgOzZAIUnJ+XknFbel+KH1HxTf6I1g0P2VWYTGTPmAMF+7tGM59T0roqzLPR1ttXu9TluJJ7m4AjUuABFGDkIoHv371ajJWVziq4jC1nObp2bWnvN+9ff7u5eurhbS0muXBKRRtIwUZJAGeK8ROsT6h4lOra5pdzexr/qrYZVF54HQ5A9O5617nRRUpudtR5dmEMGp3p8zlpe9ml5W7nHW+pv458NatZrp8lk4jCRiVshmwSOw4yBXGQ+ILqw8Hz+EpdMuhqLM0aDZ/CzZPHUnk9PWvZKKUqTet9TWhmlKlePsrwupJXejXn1PPbzwhef8Kwh01I91/C32kxjqWycr9QrEfUVz9/r134h8N6d4YtdNuft8RRJcrxhBtHuOxOcYxXsVFKVG+zt0Ko5zKN5VYKT5nJa2s3+a8jk/E0SaL8Np7QsD5VrHbg/3jwv/ANerfga1az8GabG4wzRmT8GYsP0IpvizQ7vxCthYo8aWInEl0xYhiB0CjHuf0roURY41RFCooAUDoBVKPv38rHPVxC+pKle8pScn5dF+rHUUUVqeaZnhL/kCzf8AYRv/AP0rlrcrD8Jf8gWb/sI3/wD6Vy1uVEt2QFFFFSAUUUUAFFFFABRRRQAUUUUAFFFIzKilmIVQMknoBQBl6UPtPi/V7nqltBBaL7P80r/+OvF+VFSeEEZ9CGoOpEmpTPenIwdrn92D7iMRj8KK9GCtFI527s3qKKKoQUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAI6LIjI6hlYYIIyCK5nw0WtrKbSJWJm0uU2uWPLRAAxN75jK5PqG9K6eub1xf7K1q01teLeYLZXvspb91If912K/SQntWVWPNEqDszXooorhNwooooAKKKKACiiigAooooAKwNR+XxppDHhWsbxAf8AaL25A/JWP4Vv1ma3pkuoW0UlpIkV9ayCe2d87dwBBVsfwspZT9cjkCqi7MC1RWGvimwtl2aw39kXC/fS9OxM/wCzIfkce4P1APFJ/wAJt4U/6GfRf/A+L/4qtLMq6N2isL/hNvCn/Qz6L/4Hxf8AxVH/AAm3hT/oZ9F/8D4v/iqLMLo3aKwv+E28Kf8AQz6L/wCB8X/xVH/CbeFP+hn0X/wPi/8AiqLMLo3aKwv+E28Kf9DPov8A4Hxf/FUf8Jt4U/6GfRf/AAPi/wDiqLMLo3aKwv8AhNvCn/Qz6L/4Hxf/ABVH/CbeFP8AoZ9F/wDA+L/4qizC6N2isL/hNvCn/Qz6L/4Hxf8AxVH/AAm3hT/oZ9F/8D4v/iqLMLo3aKwv+E28Kf8AQz6L/wCB8X/xVH/CbeFP+hn0X/wPi/8AiqLMLo3aKwv+E28Kf9DPov8A4Hxf/FUf8Jt4U/6GfRf/AAPi/wDiqLMLo3aKwv8AhNvCn/Qz6L/4Hxf/ABVH/CbeFP8AoZ9F/wDA+L/4qizC6N2isL/hNvCn/Qz6L/4Hxf8AxVH/AAm3hT/oZ9F/8D4v/iqLMLo3aKwv+E28Kf8AQz6L/wCB8X/xVH/CbeFP+hn0X/wPi/8AiqLMLo3aKwv+E28Kf9DPov8A4Hxf/FUf8Jt4U/6GfRf/AAPi/wDiqLMLo3aKwv8AhNvCn/Qz6L/4Hxf/ABVH/CbeFP8AoZ9F/wDA+L/4qizC6N2isL/hNvCn/Qz6L/4Hxf8AxVH/AAm3hT/oZ9F/8D4v/iqLMLo3aKwv+E28Kf8AQz6L/wCB8X/xVH/CbeFP+hn0X/wPi/8AiqLMLo3aKwv+E28Kf9DPov8A4Hxf/FUf8Jt4U/6GfRf/AAPi/wDiqLMLo3aKwv8AhNvCn/Qz6L/4Hxf/ABVH/CbeFP8AoZ9F/wDA+L/4qizC6N2isL/hNvCn/Qz6L/4Hxf8AxVH/AAm3hT/oZ9F/8D4v/iqLMLo3aKwv+E28Kf8AQz6L/wCB8X/xVH/CbeFP+hn0X/wPi/8AiqLMLo3aKwv+E28Kf9DPov8A4Hxf/FUf8Jt4U/6GfRf/AAPi/wDiqLMLo3aKwv8AhNvCn/Qz6L/4Hxf/ABVH/CbeFP8AoZ9F/wDA+L/4qizC6N2isL/hNvCn/Qz6L/4Hxf8AxVH/AAm3hT/oZ9F/8D4v/iqLMLo3aKwx408KscL4l0dj6LfRkn8A1Nl1C715DaaKk8MEnyy6lJGY1Re/lBgC746HG0dcnG0q3cLot+EedCZx92S+vJEPqrXMrKfxBBrcqG0tILCzgtLaMRwQRrHGg/hUDAH5VNWbd3ckKKKKQBRRRQAUUUUAFFFFABRRRQAVi+JS1zZQ6RExE2qSi1yp5WIgmVvbEYbB9SvrW1WRoa/2rrV3rbc28IaysvdQ372Qf7zqF+kYPetaUeaRM3ZHRoixoqIoVVGAAMACiloruMAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACobu1gvrOa0uY1lgnRo5EboykYI/KpqKAOa0S4ngebRb+RpL2xA2yt1uIT9yX68FW/2lPYitiqmvaVNepDe2BRNUsyWt2c4WQH70TH+6wA+hCtzik0vUodVsluIgyMCUlikGHikHDIw7EH/ABHBFcVanyu62NoSui5RRRWJYUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRVPVNSh0qya4lVnYkJFDGMvNIeFRR3JP+J4FNK4FPW7ied4dFsJGS9vgd0q9beEffl+vIVf9ph2BrftLWCxs4bS2jWKCBFjjReiqBgD8qzdB0qaySa9vyj6peENcMhysYH3YlP91QT9SWbjNbFd1OHIjCUrsKKKK0JCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACuf1fTLm1vW1rSI99yQBd2gIAu0HQjPAkUdCeCPlPGCvQUUmk1ZgnYydP1C21OzS6tJN8TZHIIZWBwVYHlWB4IPINWaztT0W4jvH1XRWjivmx59vIcRXYAwN2PuuBwHA9iCMYXTNXt9TEkYV4LuHAntJhtlhPuO4PZhkHsTXFUpOHobxlc0KKKKyKCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiis/U9Xt9MEcZV57ubIgtIRulmPsOwHdjgDuRTSb0QE2oahbaZZvdXcmyJcDgEszE4CqByzE8ADkmq2kaZc3V6utavHsuQCLS0JBFoh6k44MjDqRwB8o4yWdpmi3El4mq600ct8ufIt4zmK0BGDtz95yOC5HsABnO9XZSpcur3MZSvsFFFFbEBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFZmq6Ha6sY5XMkF5DnyLuA7ZYs9QD0IPdSCp7itOigDlm1HUdE+TXIPNth01K0QlMf9NY+Wj+oyvclelbEFxDdQJPbzRzQyDckkbBlYeoI4NaNYVz4WtDO91ps02l3bnc72hASQ+rxkFGJ9cbvcVzzoJ6xNFPuaFFY5m8Q6dxd6dDqcQ/5baewjk/GKQ4/JyfaiPxTozSrDPeCynY4EN8jWzk+wkA3fhmueVOUd0WpJmxRSKwZQykEHkEd6WoKCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKRmCqWYgAckntWRJ4p0ZZWhgvBezqcGGxRrlwfcRg7fxxTSb2C5sVHPcQ2sDz3E0cMMY3PJIwVVHqSeBWWJvEOo8WmnQ6ZEf+W2oMJJPwijOPzcH2qxbeFrQTpdalNNql2h3JJdkFIz6pGAEUj1xu9zW0aEnvoQ5pFNdR1HW/k0ODyrY9dSu0ITH/AEyj4aQ+5wvcFula2laHa6SZJUMk95Njz7uc7pZcdMnoAOygBR2FadFdMKcY7GTk2FFFFWIKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigApksMU8TRTRpJGwwyOoIP1BoooAxW8G+H9xaDTks2JyWsZHtST9YitN/wCEVRP+PfWdZgHYfa/Nx/38DUUUnFPdDuxp8NXxUBfFutLjuI7M/wA4Kb/wjGof9Djrn/fqy/8AkeiilyR7BzMP+EY1D/ocdc/79WX/AMj0f8IxqH/Q465/36sv/keiijkj2DmYf8IxqH/Q465/36sv/kej/hGNQ/6HHXP+/Vl/8j0UUckewczD/hGNQ/6HHXP+/Vl/8j0f8IxqH/Q465/36sv/AJHooo5I9g5mH/CMah/0OOuf9+rL/wCR6P8AhGNQ/wChx1z/AL9WX/yPRRRyR7BzMP8AhGNQ/wChx1z/AL9WX/yPR/wjGof9Djrn/fqy/wDkeiijkj2DmYf8IxqH/Q465/36sv8A5Ho/4RjUP+hx1z/v1Zf/ACPRRRyR7BzMP+EY1D/ocdc/79WX/wAj0f8ACMah/wBDjrn/AH6sv/keiijkj2DmYf8ACMah/wBDjrn/AH6sv/kej/hGNQ/6HHXP+/Vl/wDI9FFHJHsHMw/4RjUP+hx1z/v1Zf8AyPR/wjGof9Djrn/fqy/+R6KKOSPYOZh/wjGof9Djrn/fqy/+R6P+EY1D/ocdc/79WX/yPRRRyR7BzMP+EY1D/ocdc/79WX/yPR/wjGof9Djrn/fqy/8Akeiijkj2DmYf8IxqH/Q465/36sv/AJHo/wCEY1D/AKHHXP8Av1Zf/I9FFHJHsHMw/wCEY1D/AKHHXP8Av1Zf/I9H/CMah/0OOuf9+rL/AOR6KKOSPYOZh/wjGof9Djrn/fqy/wDkej/hGNQ/6HHXP+/Vl/8AI9FFHJHsHMw/4RjUP+hx1z/v1Zf/ACPR/wAIxqH/AEOOuf8Afqy/+R6KKOSPYOZh/wAIxqH/AEOOuf8Afqy/+R6P+EY1D/ocdc/79WX/AMj0UUckewczD/hGNQ/6HHXP+/Vl/wDI9H/CMah/0OOuf9+rL/5Hooo5I9g5mH/CMah/0OOuf9+rL/5Ho/4RjUP+hx1z/v1Zf/I9FFHJHsHMw/4RjUP+hx1z/v1Zf/I9H/CMah/0OOuf9+rL/wCR6KKOSPYOZh/wjGof9Djrn/fqy/8Akej/AIRjUP8Aocdc/wC/Vl/8j0UUckewczD/AIRjUP8Aocdc/wC/Vl/8j0f8IxqH/Q465/36sv8A5Hooo5I9g5mH/CMah/0OOuf9+rL/AOR6P+EY1D/ocdc/79WX/wAj0UUckewczD/hGNQ/6HHXP+/Vl/8AI9H/AAjGof8AQ465/wB+rL/5Hooo5I9g5mH/AAjGof8AQ465/wB+rL/5Ho/4RjUP+hx1z/v1Zf8AyPRRRyR7BzMcPDV8FIbxbrTZ7mOzH8oKd/wiqP8A8fGtazOO4+1+V/6LC0UUckewXY5fBvh/cGn05Lxgchr6R7og/WUtW1FDFBEsUMaRxqMKiKAB9AKKKoQ+iiigAooooAKKKKACiiigAooooA//2Q==)" + ], + "metadata": { + "id": "aP8LqLobuViH" + } + }, + { + "cell_type": "code", + "source": [ + "import apache_beam as beam\n", + "from apache_beam.ml.inference.base import ModelHandler\n", + "import timesfm\n", + "import logging\n", + "import numpy as np\n", + "import os\n", + "from google.cloud import storage\n", + "from apache_beam.io.gcp.gcsio import GcsIO\n", + "from apache_beam.utils.timestamp import Timestamp\n", + "\n", + "class LatestModelCheckpointLoader(beam.PTransform):\n", + " \"\"\"A PTransform that finds the latest model checkpoint in a GCS path.\"\"\"\n", + " def __init__(self, gcs_bucket, gcs_prefix):\n", + " self.gcs_bucket = gcs_bucket\n", + " self.gcs_prefix = gcs_prefix\n", + "\n", + " def expand(self, pcoll):\n", + " return pcoll | \"FindLatestModel\" >> beam.Map(self._find_latest_model_path)\n", + "\n", + " def _find_latest_model_path(self, _):\n", + " try:\n", + " storage_client = storage.Client()\n", + " blobs = storage_client.list_blobs(self.gcs_bucket, prefix=self.gcs_prefix)\n", + " # Filter for model files and find the most recent one\n", + " model_blobs = [b for b in blobs if b.name.endswith(\".pth\")]\n", + " latest_blob = max(model_blobs, key=lambda b: b.time_created, default=None)\n", + "\n", + " if latest_blob:\n", + " path = f\"gs://{self.gcs_bucket}/{latest_blob.name}\"\n", + " logging.info(f\"Found latest finetuned model at: {path}\")\n", + " return path\n", + " except Exception as e:\n", + " logging.error(f\"Error finding latest model in GCS: {e}\")\n", + "\n", + " # Return a path to the base model if no finetuned one exists or an error occurs\n", + " base_model = \"google/timesfm-1.0-200m-pytorch\"\n", + " logging.info(f\"No finetuned model found. Using base model: {base_model}\")\n", + " return base_model\n", + "\n", + "class DynamicTimesFmModelHandler(ModelHandler[np.ndarray, np.ndarray, timesfm.TimesFm]):\n", + " \"\"\"\n", + " A model handler that loads a TimesFM model from a dynamic path (GCS or Hugging Face).\n", + " The model path is provided as a side input to RunInference.\n", + " \"\"\"\n", + " def __init__(self, model_uri: str, hparams):\n", + " self._hparams = hparams\n", + " self._model = None\n", + " self._model_uri = model_uri\n", + " self._context_len = hparams.context_len\n", + " self._horizon_len = hparams.horizon_len\n", + "\n", + " def load_model(self) -> timesfm.TimesFm:\n", + " \"\"\"Loads a model from the handler's current model_uri.\"\"\"\n", + " logging.info(f\"Loading TimesFM model from path: {self._model_uri}...\")\n", + "\n", + " checkpoint_config = {}\n", + " if self._model_uri.startswith(\"gs://\"):\n", + " try:\n", + " gcs = GcsIO()\n", + " file_name = os.path.basename(self._model_uri)\n", + " local_path = f\"/tmp/{file_name}\"\n", + " with gcs.open(self._model_uri, 'rb') as f_in, open(local_path, 'wb') as f_out:\n", + " f_out.write(f_in.read())\n", + " checkpoint_config['path'] = local_path\n", + " logging.info(f\"Downloaded model from GCS to {local_path}\")\n", + " except Exception as e:\n", + " logging.error(f\"Failed to download model from GCS: {e}. Check path and permissions.\")\n", + " raise e # Re-raise the exception to fail fast if the model can't be loaded.\n", + " else:\n", + " checkpoint_config['huggingface_repo_id'] = self._model_uri\n", + "\n", + " self._model = timesfm.TimesFm(\n", + " hparams=self._hparams,\n", + " checkpoint=timesfm.TimesFmCheckpoint(**checkpoint_config)\n", + " )\n", + " logging.info(\"TimesFM model loaded successfully.\")\n", + " return self._model\n", + "\n", + " def update_model_path(self, model_path: str):\n", + " \"\"\"\n", + " This method is called by RunInference when a new model metadata is available\n", + " from the side input. It updates the model URI that `load_model` will use.\n", + " \"\"\"\n", + " if not model_path:\n", + " logging.info(\"Received an empty model path update. No action taken.\")\n", + " return\n", + " logging.info(f\"Received model update. New model URI: {model_path}\")\n", + " self._model_uri = model_path\n", + " self._model = self.load_model()\n", + " logging.info(\"Model has been updated in the handler.\")\n", + "\n", + " def run_inference(self, batch, model, inference_args=None):\n", + " \"\"\"\n", + " Runs inference on a batch of data.\n", + "\n", + " Note: While this is a standard method for ModelHandler, we will call the\n", + " model's `forecast` method directly in our DoFn for clarity.\n", + " \"\"\"\n", + " # print(\"Running inference on batch:\", batch)\n", + " # logging.info(f\"Running inference on batch:\", batch)\n", + "\n", + " anomalies_found = []\n", + "\n", + " key, (window_start_ts, _, values_array) = batch[0]\n", + "\n", + " # A window must have enough data for both context and horizon.\n", + " # if len(values_array) < self.context_len + self.horizon_len:\n", + " # return\n", + "\n", + " current_context = np.array(values_array[:self._context_len])\n", + " actual_horizon_values = np.array(\n", + " values_array[self._context_len:self._context_len + self._horizon_len])\n", + "\n", + " print(\"Current context shape:\", current_context.shape)\n", + " print(\"Actual horizon values shape:\", actual_horizon_values.shape)\n", + " point_forecast, experimental_quantile_forecast = model.forecast(\n", + " [current_context],\n", + " freq=[0],\n", + " )\n", + "\n", + " current_predicted_horizon_values = point_forecast[\n", + " 0, :, 0] if point_forecast.ndim == 3 else point_forecast[0]\n", + "\n", + " current_q20_values = experimental_quantile_forecast[0, :, 2]\n", + " current_q30_values = experimental_quantile_forecast[0, :, 3]\n", + " current_q70_values = experimental_quantile_forecast[0, :, 7]\n", + " current_q80_values = experimental_quantile_forecast[0, :, 8]\n", + "\n", + " for j in range(len(actual_horizon_values)):\n", + " current_actual = actual_horizon_values[j]\n", + "\n", + " point_Q1 = np.nanmean([current_q20_values[j], current_q30_values[j]])\n", + " point_Q3 = np.nanmean([current_q70_values[j], current_q80_values[j]])\n", + " point_IQR = point_Q3 - point_Q1\n", + "\n", + " upper_thresh = point_Q3 + 1.5 * point_IQR\n", + " lower_thresh = point_Q1 - 1.5 * point_IQR\n", + "\n", + " if current_actual > upper_thresh or current_actual < lower_thresh:\n", + " score = (current_actual - upper_thresh\n", + " ) / point_IQR if current_actual > upper_thresh else (\n", + " lower_thresh - current_actual) / point_IQR\n", + "\n", + " anomaly_timestamp_seconds = (window_start_ts.micros / 1e6) + (\n", + " self._context_len + j)\n", + "\n", + " index_in_window = self._context_len + j\n", + "\n", + " anomalies_found.append({\n", + " 'key': key,\n", + " 'timestamp': Timestamp(anomaly_timestamp_seconds),\n", + " 'index_in_window': index_in_window,\n", + " 'actual_value': current_actual,\n", + " 'predicted_value': current_predicted_horizon_values[j],\n", + " 'is_anomaly': True,\n", + " 'outlier_score': score,\n", + " 'lower_bound': lower_thresh,\n", + " 'upper_bound': upper_thresh,\n", + " })\n", + " payload = {\n", + " \"start_ts_micros\": window_start_ts.micros,\n", + " \"predicted_values\": current_predicted_horizon_values.tolist(),\n", + " \"q20_values\": current_q20_values.tolist(),\n", + " \"q30_values\": current_q30_values.tolist(),\n", + " \"q70_values\": current_q70_values.tolist(),\n", + " \"q80_values\": current_q80_values.tolist(),\n", + " \"anomalies\": anomalies_found, # Your original list is now inside the dictionary\n", + " \"actual_horizon_values\": actual_horizon_values.tolist()\n", + " }\n", + " result_with_context = (batch[0], payload)\n", + "\n", + " return [result_with_context]" + ], + "metadata": { + "id": "oT9NIaWcuUgb", + "colab": { + "base_uri": "https://localhost:8080/" + }, + "outputId": "a26eae71-067a-4314-b49a-3b028ee75903" + }, + "execution_count": 3, + "outputs": [ + { + "output_type": "stream", + "name": "stdout", + "text": [ + " See https://github.com/google-research/timesfm/blob/master/README.md for updated APIs.\n", + "Loaded PyTorch TimesFM, likely because python version is 3.11.13 (main, Jun 4 2025, 08:57:29) [GCC 11.4.0].\n" + ] + } + ] + }, + { + "cell_type": "markdown", + "source": [ + "# LLM Classifier![classification.jpg](data:image/jpeg;base64,/9j/4AAQSkZJRgABAgAAAQABAAD/2wBDAAgGBgcGBQgHBwcJCQgKDBQNDAsLDBkSEw8UHRofHh0aHBwgJC4nICIsIxwcKDcpLDAxNDQ0Hyc5PTgyPC4zNDL/2wBDAQkJCQwLDBgNDRgyIRwhMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjIyMjL/wAARCAOFBV4DASIAAhEBAxEB/8QAHwAAAQUBAQEBAQEAAAAAAAAAAAECAwQFBgcICQoL/8QAtRAAAgEDAwIEAwUFBAQAAAF9AQIDAAQRBRIhMUEGE1FhByJxFDKBkaEII0KxwRVS0fAkM2JyggkKFhcYGRolJicoKSo0NTY3ODk6Q0RFRkdISUpTVFVWV1hZWmNkZWZnaGlqc3R1dnd4eXqDhIWGh4iJipKTlJWWl5iZmqKjpKWmp6ipqrKztLW2t7i5usLDxMXGx8jJytLT1NXW19jZ2uHi4+Tl5ufo6erx8vP09fb3+Pn6/8QAHwEAAwEBAQEBAQEBAQAAAAAAAAECAwQFBgcICQoL/8QAtREAAgECBAQDBAcFBAQAAQJ3AAECAxEEBSExBhJBUQdhcRMiMoEIFEKRobHBCSMzUvAVYnLRChYkNOEl8RcYGRomJygpKjU2Nzg5OkNERUZHSElKU1RVVldYWVpjZGVmZ2hpanN0dXZ3eHl6goOEhYaHiImKkpOUlZaXmJmaoqOkpaanqKmqsrO0tba3uLm6wsPExcbHyMnK0tPU1dbX2Nna4uPk5ebn6Onq8vP09fb3+Pn6/9oADAMBAAIRAxEAPwD3+iiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiisbUvEMNpdGwsoHv8AUsAm2hIAjB6GRzwg+vJ7A0N23A2awp/Fdh5z2+nRz6rcodrJZKGVD6NISEU+xbPtVRtGutW+fxBd+fGf+XC3JS2Hs38Uv/AvlP8AdFbEMMVvCkMMaRxIMKiKAFHoAOlc8q6Xwmip9zLMnia++9LY6VEf4Y1NzLj/AHm2qp9trD3pp8OQz86hqOqXzd/Nu2jU/VItiH/vmtmisHVm+paikZVv4Y0G1ffBounpJ3cWybj9TjJrSjijhXbFGqL6KMU+iobbKsFFFFIAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACkZQwwwBB7GlooAzrrQNGvRi70iwnH/TW2R/5iqw8LadDzZNeWBHQWd3JGo/4ADsP4qa2qKpSktmKyMgQeIrHm11aDUEH/ACy1CAK59hJEAB+KNTx4qWz+XW9PuNNx1uP9db/XzF+6PdwlalFaRryW+pLgi1DNFcwpNBKksTjcjowZWHqCOtPrmZPD0UEz3WjXD6VdMdzeQoMMp/6aRH5W+ow3+0Klt/EcllMlr4ggSykchY7uNs20pPQbjzGx/ut9AWrphVjIzcWjoaKKK0JCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigApCQoJJAA5JNBIVSzEADkk9q5SSR/F7nlk8PKeB0OoH1P/TH2/j/AN370ykoq7Glckl1O88RsYtIma10vOJNRUfPP6iDPb/pof8AgIP3ho2GnWml2otrOFYo8ljySWY9WYnlmPcnJNWVUKoVQAoGAB0FLXFOo5vU2jFIKKKKzKCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKZLFHPC8M0aSROCro4yGB6gg9RT6KAMJIr7wz81gkt9pA+9ZZ3S249YSfvKP+eZ/wCAngKeisb611KzivLOdJreUZR0PB/wIPBB5BqKsS7srrSbyTVtGj3tId15YggLc/7S9llA79G6HsV6adbpIzlDqjqKKq6fqFrqtjFeWcvmQyDg4IIIOCCDyCCCCDyCCDVquoyCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKxfEWo3FvFBp2nuF1K/JSFsZ8lB9+Uj0UEY9WZR3pN2V2BQ1KQ+JdQl0qIn+ybVtt84P/HxJ18gH+6ON/rwv94DbVQqhVAAAwAO1V9PsbfTLCGytVKwxLtXJyT6knuScknuSTVmuGpNzdzeMbIKKKKzKCiiigAooooAKKKKACiiigAooooAKKK5zx7ql5ovgXV9S0+bybu3g3xSbQ205HYgg/iKaV3YDo6K4v4Va9qXiTwFaanq1z9ovJJZVaTYqZAcgcKAOntWv401C60nwVrOoWMvlXVtaSSRSbQ21gODggg/jTcWnyivpc3aK4X4SeIdV8T+B11HWLr7Tdm5kQyeWqfKMYGFAFb/jG+udM8F61f2cpiubeyllikAB2sFJBweOtDi07BfS5t0V4J8JPiV4j1rxrFpfiDUWuYLy2kNuGiRPnXnIKqOyuKX4ufEnxHonjN9L8P6i1tBaWyNcBYkf52OckspxwyD8av2UublFzq1z3qisLwZf3WqeCtFv72Uy3VxZxySyEAbmKgk4HFZnxH8cxeBPDn21YlnvZ38q1hY8FsZLN/sgf0HGc1HK27Id9LnYUV4dBYfGrVNKXXY9bt4GkTzorDCK5XqBt2Fc47Mc+tdX8O/HeqeK/Ceoy6nZvbajYoQZhEVSX5ThhnjIIOR9PXFU6bSvcSkejUV5l8FfFGs+KvD2o3OtXpu5orvy0YxquF2A4+UDua7Kbxl4Yt7s2s3iLSo7hW2tG95GGB9CM8GpcWnYad1c26KjaeJYDO0qCELvMhYbQvXOemKybXxf4avrtbS08QaXPcMdqxR3cbMx9AAeaVmM2qKhu7y1sLZ7m8uYbaBOWlmcIq/UngVn6d4n0DWLj7PputafeTYz5cFyjtj1wDmizA1qK4Txjc+IYvGGgx6X4h03T7F5E+1WlzNGklwPMGQgZSTkccEc1y3xV8fanaeJtJ8OeHtctLHz3C3d0HRvJYvtw5OdgXknoapQbtYlyseyUVyvg/U4odNttM1Lxbp2tasxZg8U0e6ReSMKpycKM5+tb2oavpukrG2o6haWYkJCG4mWPcR1xuIzUtWdh3LlFFcB8XfGVz4P8IK+nTiLU7uZYrd9oYoByzYIIPAx/wACFOKcnZA3bU7+ivLPg5441XxHHquk+IZ2k1axl3fPGqNsPylSoA5Vhz/vCvSL3VdO01oVvr+1tWmO2ITzKhkPHC5PPUdPWiUXF2BO6uW6Kyz4l0JdU/sw61p4v9237N9pTzN3ptznPtWlJIkUbSSOqIoyzMcAD1JpWGOorIsfFPh7U7v7JYa5pt1c/wDPKG6R2P0APNXL7VdO0sRnUL+1tBIdqG4mWPefQZIzRZgW6KzD4j0Mar/ZZ1iw/tDO37L9pTzM+m3Oc+1JJ4k0OLVBpkms6el+TtFs1ygkz6bc5z7UWYXNSiobu8tbC2e5vLmG3gQZeWZwir9SeBWfp3inw/q9x9n03W9Ou58Z8qC5R2x9Ac0WYGtRXlXjfxbrmk/F3wvotjftDp16IftEIjQh90rKeSMjgAcGtq/ufEK/FWygh8Q6bFopjHmaY80YuHOxuQpXcecHr2NVyMVzu6Ko6nrWlaLEsmqalaWSOcK1zMse4+2TzU1jqFnqdqt1YXcF1bt92WCQOp/EcVNhliisjUPFXh7Sbn7NqOuadaT9TFPdIjD6gnIrStrq3vbdLi1ningkGUkicMrD2I4NFmBLRRRSAKKKKACiiigAooooAKKKKACiiigAooooAwr0P4dv5NatVZrGYg6lboM4GMeeo/vKANw/iUZ6qAeojkSWNZI3V43AZWU5DA9CDVSsbRn/ALC1X+wnOLG4DS6aT0THLwfh95R/dyOiV1UKn2WZTj1OmooorpMwooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooARmCqWYgADJJ7VzGhE6nPceIZQf9NAS0B/gtlJ2fi+S5/wB5R/DVjxXI1xZ2+ixsRJqkvkSEHlYAN0p9sqNgPYutaSqqKFVQqgYAAwAK5sRP7JpTXUWiiiuU1CiiigAooooAKKKKACiiigAooooAKKKKACuR+KP/ACTLX/8Ar1P8xXXVheM9FufEXg/U9ItHiS4uofLRpiQgOQeSAT29KqOjQnsct8Dv+SXWP/Xab/0M10HxF/5Jz4h/68Jf/QTXmejfDn4r+H9NTT9K8U6TbWiEssauxAJOT1hJ611Wn+FPHVz4Q8R6T4l1uxv7m/t/KsnRjsiJDBtxEanByvY9K0klzc1yVe1rEPwG/wCSaJ/1+S/0rq/iB/yTvxF/2Dp//QDXl+ifDb4q+HdPFhpPijSbW1Dl/LV2Iyep5hJrqtM8K+PZ/DHiTTPE2uWN/Lf2ZgsijHbExVgS37tTg5XpnpRJLm5rgr2tY8b07OgeH/A3i9AQtvqE0E7Dugk3AfiDJTtSz4g0Tx74ucZSe+ht7dj/AHDLux+CrHXp5+FGqS/B4eE5rmwOpxXf2mGUO/lA7u525+6WHTrSp8KNUh+DsnhOK5sBqk919pllLv5RO4d9u77qr261p7SO/mTys7b4ef8AJOvDv/XhD/6CK8u/aKVxceGJZFLWqtOG9M5jP8h+lev+FdKn0PwppWlXLxvPaWqQyNESVLKMHBIBx+FVfGfhCw8beH5NKvi0Z3CSGdBlonHRh69SCO4P41jGSU7ltXjY3opI5YkkiZWjZQysvQg9CKwbfxRofiTSNTOj6nBdmCFxKsZ+ZOCOQeccHnvXmC/Dz4rWemnQLPxZZnSNvlqTIwZY+mM7CyjHGA2K7vwT8PbbwX4Wu9Nt5hcX14pNxcsu0M20hQBzhRk/mT3xQ4xS3C7Z578F2s0+GHic6hfSWNn5zCa5iba8amJQSpwefTg1y17Z+EH8FX6eHPCOt6j5Sux126BjSPBznI+UgDjbgZr0jwr8J9Q074da94X1a9tRJqUokjmtWZwmAu3O5V/iUcelZ1j8OPiH/wAInL4Sudd0q30ZUcIYVZ5JMksEJ2jCljyeuPXpWvNHmbuTZ2M2z0rXfFH7N9jZ6UZJ7iK4YtCG+aWFZH+QZ64JUgf7OPSuR/tDwelvZ6V4p8F6jod1bld95YuVkfAwdySDoep5J9K9X074Z63H8LU8Lz6yLHULe4eWC5sJ5NjAkna/Ckg7jx2wD7Vk6l4E+JviTR4PD2t6zox0yNk33CqzyuF6H7oyfyz3NCmrvXqJpknxZ0XU/E3hXw3f+HUm1nTLYebLCGZnuFKrscgYZjgMDjkbj744rTdc8BS+JNNk1Xw7qXhW+tJARJZzfu92RgyBl3ADHYd+c16r4m+HF/c+G9GsfDWvXOnXOkIqRbpWEU2McuF75Gc4PUjHpz2pfDrx144vLCLxjqmlR6faPuIskPmvnGcfKMZA9ePQ0oyVrNjadyr8WyD8WfAxByPOh/8AR61kfE3wvo0fxk8O2y2eIdXnie+XzX/el5trHOflyP7uK7/xt8P9T8R+NfDWsWE9lFaaU8ZlSZ2DkLIG+UBSDwO5FO8ZeAdV8Q/EXw54htLizS00x4mmSV2EjbZd52gKQePUiiM0ra9Aa3OM+KHh628Aat4V8T6FamG0sJVt5I1Zm4DFwCSSTuBkHJ9Kn8RyRfEL44aJpMDCfS9LhW5lI5UggSH8DmJa7v4qjSbj4e6za6ldwQsLfzYQzDcZFOUwOvLDH4muN/Z78PtBot/4huFJkvHFvAzdfLTqR7FuP+AUKXucz3QNa2PaK+cPiR4r0zUPjJYRanI7aNokipKsa797j53GPdgqH/dr6KufP+yzfZRGbjY3lCQkKWxxkgE4z7V5r8OvhbL4dl1W98TjTtTv72QMGVTKqjksfnUcknnjsKim1G7ZUk3oeYxeOtGsvjdF4l0aSZNMvmVLxZU2bd42ucZ6ZAf611n7RG8/8Iv5blHM0+1gcEH93g11/wARvhla+K/DqW2jW2n2OoQzLJHKY/LVl5DKxVScYOenUCsbxZ8NvE/ijw54UtJrzTPt+kqy3UjSybZPuBSp2ZJITJyByatSi2pEtOzRzPxh+H/h/wAKeDdNv9ItXhvEvEgknMrM0oKOxZsnrlQeMdTVj4saxqGo6J4K0L7U8UWrRRS3UmfvsQgGfUAsTj1x6V6D8U/Buo+OPC1vpmmTWsU8d4k5a5ZlXaEdcfKpOcsO1VvFvwzTxZ4N0jTZLpbbVNLgRIblQWTcEUMD0O0lRz1GB9KUZrTmBx3scf8AEv4XeG/DHgOTWdEims7/AE54mEwnctLl1XnJwDls5GOlYHxN1m71z4VeCdUu2JupTJ5j9CzKNu76nbn8a6a6+HPxH8UQ22k+KfE1mdIgcFzAN0kmOhPyLuPux98E1ufEb4Y3fiTw3oejeH5LO1g0wlQt07AbdoAwVVsnjnNUpJNXdwa3scJ8XvA2i+EfCuiXulQPFffaBHLc+axeU7CxY5PXK54x1pPif4G0Xw58NtE1WygcanJcRLcXTSMzTF42dicnruUGvS/ip4H1Pxx4esdP0ye0ilt7kSsbl2VSNjLxtU85NHxF8Dan4u8DadolhPaR3NtPFI73DsEIWNlOCFJ6sO1KNTa7Bx3MH4gy+Gb7wX4Yl8W6nqYd4Y5o7SxILXTlF3FgR74zkY3H1rzXxMLOx8UeGb7RfCV/4ajNwPLe6Zla4KumCFOduM9c87vavU/Gnwy1fWrHw1d6Rf20OsaJBHHiUny3KbSCpweQy9xzntWT4h+Gnj7xVPp2qarrWkPf2cgMdqodIY1yCSGCkliQM8dutOEopbg0yL4kf8l68F/S3/8AR70a5/yc/o3/AFwX/wBFSV1PizwDq2vfEvw94ltp7KO004RedHI7iQ7ZGc7QFIPB7kUuo+AdVvPjHp/jCO4sxp9vGEeNnbzSQjLwNuOrDvUqSt8h2Z5Zdalaa78YNfuPEWiatrttZSS29vZ2MRk8sI+xSwBGFwCfqa6L4ZR32l/FG9j0vQdb03w1fxMVhv7d1EThQwJJyOoYDnoa6HXvhz4isPG9x4r8Eana21zdg/ara6B2sTjdjg5BIBwcYPftWt4I8HeJdO1S/wBY8U+IZby6u1Ki0tpnEEecfNjgbsDAwBjnk9nKa5dBJO54ybC48FarqY8beCpNbjuJix1BpHXOScsrjK85z2Ir1n4M3XhaTQry38NXGpYWVZLi2v3UtExBAK7RjBx1HpzVCHwZ8SfDF/er4d8RWl/YXT7h/aru8ienUHnHGQecdK3fhp8PZvBUWoXeoXkd1qmoOGmaFcRoAScLwM8sTnA7ccck5Jx3CKaZ31FFFc5oFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFZ+tac2pac0cMgiu4mE1rMR/q5V5U/TPBHcEjvWhRTTtqA3RdTXWNIgvRGYncFZYicmKRSVdD7qwI/Cr9c3p7f2X4subTpbapGbqIdlmTasg/wCBL5bAequa6SvQhLmVznas7BRRRVCCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKQkAEk4A6k0Ac3Aft/jDUbo8x6fEljF7OwEspH1BhH1Q1sVi+FMy+Hob1gd9+73pz1xKxdR+Cso/CtqvPqO8mzeKsgoooqCgooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooA5LxR8N/DnjDVINR1i3mknhiEI8uYoCgJbBx7sfzrpNP0+00rT4LCxgSC1gQJHEg4UCrNFNttWFYKKKKQwooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAxPFGbfTI9VQHzNLmW8467FyJR+MTSD6kV04IIBByD0IqhLEk8LwyqGjkUqynoQeCKqeEJnl8LWMcrFprZWtJWPVnhYxMfxKE/jXVh5aNGVRdTbooorpMwooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACsfxXcvaeEdYniOJUs5fL/AN/YQv64rYrB8Y8+G3j/AOet1awn6PcRr/Wk9EBctbdLS0htohiOGNY1HsBgVLRRXmnSFFFFABRRRQAUUUUAFFFFABRWXqmvWumSpbBJbq/kXdHaW43SMPU5wFX/AGmIHvmstrPVdW+bVr021uf+XKwcoMejy8O3/Adg9jVxg2RKajual/4i0rTZ/s9xdq10RkW0KtLMR6+WgLY98VSPiDUrj/jw8Pz7e0l9OkCn8BvcfioqxZadZabB5NjaQ20eclYkCgn1OOp96s1qqcUYus+hmfafFEvIGj2v+ziWfH4/J/KmsPFZ+7q2ir7HSpT+v2gVq0U+WPYj2ku5k48W/wDQY0T/AMFEv/yTRjxb/wBBjRP/AAUS/wDyTWtRRyrsHtJdzJx4t/6DGif+CiX/AOSaMeLf+gxon/gol/8AkmtaijlXYPaS7mTjxb/0GNE/8FEv/wAk0Y8W/wDQY0T/AMFEv/yTWtRRyrsHtJdzJx4t/wCgxon/AIKJf/kmjHi3/oMaJ/4KJf8A5JrWoo5V2D2ku5k48W/9BjRP/BRL/wDJNGPFv/QY0T/wUS//ACTWtRRyrsHtJdzJx4t/6DGif+CiX/5Jox4t/wCgxon/AIKJf/kmtaijlXYPaS7mTjxb/wBBjRP/AAUS/wDyTRjxb/0GNE/8FEv/AMk1rUUcq7B7SXcyceLf+gxon/gol/8AkmjHi3/oMaJ/4KJf/kmtaijlXYPaS7mTjxb/ANBjRP8AwUS//JNGPFv/AEGNE/8ABRL/APJNa1FHKuwe0l3MnHi3/oMaJ/4KJf8A5Jox4t/6DGif+CiX/wCSa1qKOVdg9pLuZOPFv/QY0T/wUS//ACTRjxb/ANBjRP8AwUS//JNa1FHKuwe0l3MnHi3/AKDGif8Agol/+SaMeLf+gxon/gol/wDkmtaijlXYPaS7mTjxb/0GNE/8FEv/AMk0Y8W/9BjRP/BRL/8AJNa1FHKuwe0l3MnHi3/oMaJ/4KJf/kmjHi3/AKDGif8Agol/+Sa1qKOVdg9pLuZOPFv/AEGNE/8ABRL/APJNGPFv/QY0T/wUS/8AyTWtRRyrsHtJdzJx4t/6DGif+CiX/wCSaMeLf+gxon/gol/+Sa1qKOVdg9pLuZOPFv8A0GNE/wDBRL/8k0Y8W/8AQY0T/wAFEv8A8k1rUUcq7B7SXcyceLf+gxon/gol/wDkmjHi3/oMaJ/4KJf/AJJrWoo5V2D2ku5k48W/9BjRP/BRL/8AJNGPFv8A0GNE/wDBRL/8k1rUUcq7B7SXcyceLf8AoMaJ/wCCiX/5Jox4t/6DGif+CiX/AOSa1qKOVdg9pLuZOPFv/QY0T/wUS/8AyTQB4szzrGike2ky/wDyTWtRRyrsHtJdzM83xSnP2nR5vb7NLFn8fMbFOGs67Bzc6DDMn/TjfB2P/AZFjH/j1aNFHJHsNVZFOLxZpe8R3rTabITgC/iMKk+gc/IT7BjW2rBlDKQVIyCOhrPZVdCrqGUjBBGQRWSNBSyJk0W4fS5CcmOIboGP+1Efl+pXax9ah0l0LjW7nT0Vz8PiJ7J0g16BLRmIVLuMk20hPQEnmMn0bjnAZjXQVk01ubpp6oKKKKQwooooAKKKKACiiigArM8NHyrzX7TosWob0H+zJFG5P/fbPWnWXpPyeMNbj7NbWk34kzL/AOyVvQfvEVNjoaKKK7DEKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigArB8Yf8gOH/sJWH/pXDW9WD4w/5AcP/YSsP/SuGlLZjW5o0UUV5p0BRRRQAUUUUAFFFFABXPajrFzeXcml6KyiWM7bq9K7ktv9lR0aT26L1bspdrWoXFxeDRdMlMdy6B7q5UZ+yxHgY7b2wQuemCx6AGxZWVvp9nHaWsYjhjGFXOfckk8kk5JJ5JOa1hDqzGpUtoiLTtLttMidYAzSStvmnkbdJM395mPJP6DoMDirlFFbHMFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFADZI0ljaORFeNwVZWGQQexFYyC68L/AD2qy3WjD79qMtJaj+9F3ZB/c6gfd6ba26KTSejHGTi7ot29xDd20VzbSpLBKoeORDlWU9CDUtcpvPhe9NyhxotzJm4j7WsjH/Wr6IT94dATu/vV1dc8o8rOyMlJXCiiipKCiiigAooooAKytN/5HnV/+wbZf+jbqtWsrTf+R51f/sG2X/o26rah8ZE9joqKKK7TEKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigArB8Yf8gOH/sJWH/pXDW9WD4w/wCQHD/2ErD/ANK4aUtmNbmjRRRXmnQFFFFABRRRQAVn61qg0nTXuBGZpmIjghBwZZWOFX2yep7DJ7VoVzUrf2r4qdjza6SPLQdmuHUFj/wGNgB/10b0q4RuyZy5Vcm0jTjp1mVlk867mYy3U+MebKep9h0AHYADtV+iiug4gooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigBskaSxtHIiujgqysMgg9QRVDw7O9jcTaBcOzG3XzbN3OTJb5xjPcoflPtsJ5NaNZHiAPbW8OsQqWm01/PKr1eLGJU98pkgf3lWplG6sXTlys6eimo6SxrJGwZGAZWB4IPenVzHYFFFFABRRRQAVlab/yPOr/APYNsv8A0bdVq1lab/yPOr/9g2y/9G3VbUPjInsdFRRRXaYhRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFYPjD/AJAcP/YSsP8A0rhrerB8Yf8AIDh/7CVh/wClcNKWzGtzRooorzToCiiigAooooAhu7mKys57qdtsUEbSOfRVGT+grB8O28sGh273IxdXANzce0sh3sPoC2B7AVP4wO/w7Ja/8/k0Fqw9UklVH/8AHS1Xa3pLS5z13sgooorQwCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAoorxHTfid8Q9f1HULbQ9C0m7Fm5D/ACspAyQOsoz0PSmlcaVz26ivKPDfxT1o+MIfDXi7RotPurhgkTwhlAY/dyCWyD0BB6/p6vQ1YTVgoozkZFICCSARx1pALRRRkZx3oAKKMjOM8+lFABRRnnHeuY8c+M7bwRoS6jPayXLyyeTDEhCguQTyewwD2NAHT0VXsLg3enW1yyhWmiSQgdBkA1YoAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACkIDAggEHgg0tFAFLwk5j0ZtOY5bTZ3sxnrsXmPPv5bR1u1z2knyPFurwDhJ7e3ugPV/njY/lHHXQ1zzVpHbB3imFFFFQUFFFFABWVpv8AyPOr/wDYNsv/AEbdVq1lab/yPOr/APYNsv8A0bdVtQ+Miex0VFFFdpiFFFFABRRUF7e2unWkl3e3EdvbxjLySMFUfjQBPVa+1Gy0y2Nzf3cFrCDjzJpAgz6ZPf2rDbVNY1njS4Tptkf+Xy8i/euPWOE/d+r4/wBwipLLw/Y2lyLyQSXl9jH2u7fzJR67SeEHsoA9qxnWjHbUtQbA+Jri840fRrq6U9J7r/RYfzYbz9QhHvTDD4kvP+PjVrWxQ/wWNtvdf+2kuQf++BWvRWEq8nsWoIxj4bil5vNT1i6buWv5Igf+AxFF/Smnwf4eb/W6TbT/APXwvm/+hZrborNzk+pVkYTeCvCjnLeGNFY+psIj/wCy0n/CD+Ev+hW0T/wXxf8AxNb1FLmfcLIwf+EH8Jf9Cton/gvi/wDiaP8AhB/CX/QraJ/4L4v/AImt6ijmfcLIwf8AhB/CX/QraJ/4L4v/AImj/hB/CX/QraJ/4L4v/ia3qKOZ9wsjB/4Qfwl/0K2if+C+L/4mj/hB/CX/AEK2if8Agvi/+Jreoo5n3CyMH/hB/CX/AEK2if8Agvi/+Jo/4Qfwl/0K2if+C+L/AOJreoo5n3CyMH/hB/CX/QraJ/4L4v8A4mj/AIQfwl/0K2if+C+L/wCJreoo5n3CyMH/AIQfwl/0K2if+C+L/wCJo/4Qfwl/0K2if+C+L/4mt6ijmfcLIwf+EH8Jf9Cton/gvi/+Jo/4Qfwl/wBCton/AIL4v/ia3qKOZ9wsjB/4Qfwl/wBCton/AIL4v/iaP+EH8Jf9Cton/gvi/wDia3qKOZ9wsjB/4Qfwl/0K2if+C+L/AOJo/wCEH8Jf9Cton/gvi/8Aia3qKOZ9wsjB/wCEH8Jf9Cton/gvi/8AiaP+EH8Jf9Cton/gvi/+Jreoo5n3CyMH/hB/CX/QraJ/4L4v/iaP+EH8Jf8AQraJ/wCC+L/4mt6ijmfcLIwf+EH8Jf8AQraJ/wCC+L/4mj/hB/CX/QraJ/4L4v8A4mt6ijmfcLIwf+EH8Jf9Cton/gvi/wDiaP8AhB/CX/QraJ/4L4v/AImt6ijmfcLIwf8AhB/CX/QraJ/4L4v/AImj/hB/CX/QraJ/4L4v/ia3qKOZ9wsjB/4Qfwl/0K2if+C+L/4mj/hB/CX/AEK2if8Agvi/+Jreoo5n3CyMH/hB/CX/AEK2if8Agvi/+Jo/4Qfwl/0K2if+C+L/AOJreoo5n3CyMH/hB/CX/QraJ/4L4v8A4mj/AIQfwl/0K2if+C+L/wCJreoo5n3CyMH/AIQfwl/0K2if+C+L/wCJo/4Qfwl/0K2if+C+L/4mt6ijmfcLIwf+EH8Jf9Cton/gvi/+Jo/4Qfwl/wBCton/AIL4v/ia3qKOZ9wsjB/4Qfwl/wBCton/AIL4v/iaP+EH8Jf9Cton/gvi/wDia3qKOZ9wsjB/4Qfwl/0K2if+C+L/AOJo/wCEH8Jf9Cton/gvi/8Aia3qKOZ9wsjB/wCEH8Jf9Cton/gvi/8AiaP+EH8Jf9Cton/gvi/+Jreoo5n3CyMH/hB/CX/QraJ/4L4v/iaP+EH8Jf8AQraJ/wCC+L/4mt6ijmfcLIwf+EH8Jf8AQraJ/wCC+L/4mj/hB/CX/QraJ/4L4v8A4mt6ijmfcLIxP+EO8Nfw6BpkZ9Y7VEP5gCl/4RbTU/493v7U9vs2oTxj/vkPg/iK2qKalJdQsjHGmaxbc2XiO5bHSO+gjnQf98hHP4tThq2v2X/H7pEN7GOsunTYf6mKTGPwdj7VrUVarTQnBFfT/Eel6lcfZobny7wDJtbhDDMB67GAJHuBj3rVrIv9NstUg8i+tYriMHIEig7T6j0PuOaz1tNZ0f5tLvDf2o/5cr+QlgP9iblvwfdn1Wt4109zNwfQ6eisvS9fs9Ule2Aktr6MbpbO5XbKg9cchl/2lJX3rUrfcgKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACsHxh/wAgOH/sJWH/AKVw1vVg+MP+QHD/ANhKw/8ASuGlLZjW5o0UUV5p0BRRRQAUUUUAYHib5rjQou0mpDj/AHYZX/8AZav1Q8R/8hLw3/2En/8ASW4q/XRT+E5a3xBRRRVmQUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFfM/gTxHrvh7X9fbRPD82rvNLiRY1Y+WA7YJ2g9cn8q+mK8t+FXhDXfDeueIbnV7H7NFdspgbzUffhnPRWOOCOtVF6Ma2MPSvD3i7xv8SLDxN4h0v+yrTTzGyow2kiNiyqATkkscknjGfYVna/pl34i+PWp6FHqNxaW10iLO0Tc+WIEYgD3IA/Gvf68xtfCetx/Ha78SPZY0iSLatx5qcnyVX7ud3UEdKakNM5zxxHc+AfDei+CvD99Or6lcSNLcs21ypYALkdB8wBx/d9zUXiz4Zf8IH4bPiXQtb1BdSsmRp3ZwBJuYKSABxyRwSciu4+KHgW68YafZ3OmTJFqmnuXh3naHBxkZ7HKgg/45rk9Xs/il4306Lw9qek2enWhZftV3vX95tOQSAxzyAcKOo7CmmNMzfiX4judc+G/hLWQxhnuJHMnlnA3qNp/DIJqt8R/CMng7TdJ8TW+s6hNrctwFuLmWXJLlS2VwMgAgjGTxXU/EX4eandeDfD+h+G7M3a6cxDkypGSNv3juIGScnj1rW+LfhfWfE/hbT7LR7P7TcQ3SyOnmomFCMM5YgdSKE9gTPPvHvhGTw/4V0nxeNY1CbXriaIzzvL0Z0Z/kwMrgjA56VrfGLU7uXwn4Pv1lZLidPOYqcZYoh/ma6r4leF9Z8QfDzTdL0uz8+9hmhaSLzUXAWNlPLEDqR3rjvjNazWfgzwdaXCbJ4YvKkXIO1hGgIyOOooTvYE72H+OfhzJ4Y8Nt4tt9d1GXXbV43ubiSX75ZgpK4AIwWHc8VmfEw/8JB8P/DPi+4ll+23CrayxggRkgSFmxjqWB79K3/EWmfFDxJpsfhS706yFoHQTakkoCzKp4Y5bI5AJAXOR0rY8e/D69uvhlpWg6GguZtMkRtpYIZQFYMRk4yS2cZ9aL7XC51XgHw3b+GfC8NvbXNzOtyVumNwwYqzIoIGAOPlrp65jwHceIpvDyp4k0yKwuYGEUKRsDvjVVAY/McHO707cV09Q9yGFFFFIAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAzE+Txxaf9NdNnz77ZIf8A4v8AWujrnP8AmedM/wCwbef+jbaujrCpuddL4QooorM0CiiigArK03/kedX/AOwbZf8Ao26rVrK03/kedX/7Btl/6Nuq2ofGRPY6Kiiiu0xCiisTWdZmhuV0rSkSbVJV3/PzHbR9PMkx264XqxHYAkJtJXYbkur67Hp0kdnbwteanMu6G0jODjpvduiIP7x+gBPFULXRZJruPUdamW9v0O6JQuIbb/rmh7/7Zyx9QOBZ0vSYdLjkKu89zM2+4upTmSZvVj6egGABwABV+uOpVctFsbRhYKKKKxLCiiigAooooAKKKKACiiigAooooAKKKKACiiigAoorw/8AaCu7uC78MR2t3PbiU3CsYpCueYuuDz1qoR5nYTdlc9worxDVvhF4l0TTJ9R0PxvqU11bIZVgYvH5mBkgMHPPHQjmuv8AhL43n8ZeFJJNRZTqFjJ5U8gAAkGMq+OgyMg+4NU4aXTuJPWzPQKK4G4+M3ga21I2TaszlW2tNHA7Rg/7wHI9xkVJ8RvHsHhnwT/aWm3cTXd6oGnyBPMRzkEnI4+6SRnjPrS5JXtYd0d1RXnnwo8b23iTwqkN1qElxqtnG0l/JMpAXc7kHceMYHbpipH+NHgZNRNmdWY4baZ1gcxZ/wB7HT36e9DhK9rBzI7+iqsupWMGmNqUt3Ctisfmm4LjZsxndnpiuIh+NXgaa/FqNUkUFtome3dY8/XHA9yMUlFvZBdI9BoryP4zfEa48OWVrpehXwh1C6QTPIibiIG3AFG6Akg+/pXoHhLxLp3inQo7/TJ3niQ+S7uhU7wBnrz3603BqPMF1exuUVR1nVINE0W91S6OILSFpn9wBnA9z0r5i8MeLdf0fxfpXizVrm4OmaneTJJvkJjKkgSEDoApcEf7tOFNyTYnKx9V0VV1DULbS9MudRunK21tE00rAE4RRknA68CuMufjJ4JtrC3uzqjOJ87I0hYyYBwSVx8oyO/XtUqLeyG2kd7RWZoPiDS/E2lJqWkXa3NqxK7gCCrDqpB5B+tctqfxi8E6VqLWUuqNNIjbZGt4WkRT/vAYP4ZoUW3ZILo7yisq18SaRfeH5Nds72O406OJ5WmjycKoy2R1BAHTGa5af4y+CILCG7OqO4lYhY0gcyccElccD69e1Ci3sgujvqK4i++Lngqx021vm1dZUuQTHHDGzSYBwdy4yvPrj2rodE8S6R4i0b+1tMvY5bIZ3yH5fLI5IYHG3Hv25ocWtWgujWorzuf43eBoLw2/9ozSBTgyx2zlPzxkj6Ck+KWsw33wd1DVdIvd8MogaG4gcjIMyA4PUdwR9RT5HdJhzI9Forxm6ktbn4B6DJq2v3elozIWvIo3mctl8KQCDz657V6FY61pHh7wDpeoX2qg6fFYwBbyZSrTDYNrbeTubrjk0OFgTOkoritB+K/hDxFqaadZakyXUh2xJPE0fmH0UnjPt1Nanijxv4f8HRxNrN8IXmyYoUQu7gd8DoPc4FLlle1gujoaK5Hwz8S/C3i29+xaZfn7ZgsIJozGzAdcZ4P0BzXXUmmtGCdwooopDCiiigAooooAKKKKACiiigAooooAKKKKACiiigClqWk2mqxItzGfMjO6GaNikkLf3kYcqfp16HIqpBrN3okqW2vSCW1YhYdUVQq5PAWYDhG/2h8p/wBk4B2KbJGk0TxSorxuCrIwyGB6gjuK0hUcCZRTL9FcnFPJ4RYJK7S+HycB2JLWHsT3h9+qf7v3erBBAIOQehFdsZKSujFqwtFFFUIKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigArB8Yf8gOH/ALCVh/6Vw1vVg+MP+QHD/wBhKw/9K4aUtmNbmjRRRXmnQFFFFABRRRQBgeI/+Ql4b/7CT/8ApLcVfqh4j/5CXhv/ALCT/wDpLcVfrop/Cctb4goooqzIKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigArlfGvgSx8cQ2cV9dXMAtXZ1MG3ncB1yD6V1VFGwBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQBmf8AM86Z/wBg28/9G21dHXOf8zzpn/YNvP8A0bbV0dYVNzrpfCFFFFZmgUUUUAFZWm/8jzq//YNsv/Rt1WrWVpv/ACPOr/8AYNsv/Rt1W1D4yJ7HRUUVBe3tvp1jPe3cgjt4EMkjnsAMmu0xKGu6u+nRQ29nGs+p3ZKWsLHjI+87Y6IoOSfoByRUOk6Wml2zqZGnuZm825uXHzzSHqx9B0AHQAADgVW0W1uJpZta1GMpf3gAWJv+XaEcpF9ecse7E9gMbFcVWpzOy2NoRsFFFFYlhRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFeFftDf8AIR8J/wC/P/OKvda4j4g/De38ftpzT6lLZmy8zb5cYbdv2+p4xt/WtKclGV2TJXR1WsahbaTo15f3kix29vC0jsxwMAdPx6V83+BY76x+DXjvUbcOqziGBGHcA4k/8dkrvF+ANjNIo1DxNql1ADkx4Az+Jzj8q9LsfDWkad4cHh+2so10zymiaA8h1b72T1JOTk1SlGKstRWbZ4R4SsPEF38K3tbLwjot1pt1HMZNQuLpEkzuYbzk/KUxx6bQa059Bv8ASv2ddVtdaitpJ7WcNZukiTBI2kjOVZSccl+lbr/Azyo7ix0/xhqtpo1w+6SwA3K3sSGAPbqvYda7T/hAdGTwFL4QgWSKwkiKFwQZC2d28nudwB9O3SqdRX07iUWeY+Uth+zE15Y28UV1cQiO4njQB3Q3BU7iOTwcc9jWXoWm+Ib74TrY2vhHRJNMuYHb+0ZrtEk3ZP7wkngqfX0xXpPhH4Wx+G9O1HTb/WrjVtNvITB9klQpHGpJJ2jccE56jHrWGfgWUgl0628Y6rDokj73sMZB5zydwUnpzt7UKcddeocrOS8WWmu6D8AdM0vUGTI1Py2MUyyqYcO6jcpIxu/kK9FHhrwRJ8KtL/te3tINKW1t5WuRhGDMF+beOcsTg/WuifwLoL+Cx4TNqf7LEe0AN84bOd+7+9u5z/TiuCT4DRsI7O68W6pNo8b71sduAPodxUHryF70udNb21HZowvjXZ6JF4D8NS6IsElosnk29wjeYTEqnC7zkkA54zXt+jWVpY6TbxWdtDbxsiuUhjCAkgZOB3rnPF3w50vxT4Us9AjkawhsWU2rxLu8vapXBB6jB9c571oeC/DNx4T0NtOudYudVcymQT3AIKjaoCjJPA2+veplJOKVxpWZwPx+8QPbeHrLw7almudTlDSInJMaEYGPd9uP901534h1a91H4b6d4aHgfVLMaWFkW9dXIyAfMYjyxgNlieeOPSvbdT+G9vq/xEtPFt7qMsn2TZ5Nn5Y2LsBK/NnP3iWrtZI0ljeORQyOCrKRkEHqKaqKKSQnFtnlGg+Jf+El/Z91WSR911Z6ZcWk+TzlIjtP4qVP1zVP4L+H9Ku/hjfT3NhbzS3c0ySvJGGJUKAFye3U/jW/4f8AhNb+HdM8QabbazcPZ6xbtA0bxD91kMAwOeSAxHvxXQ+DPB8Pg7wwdEhvHuUMjv5roFPzewolKNml3BJ9TxX4b3l3afBjxvLZswlTJUr1UFAGI9DjP5V2fwW8OeHr34brPPp9ndXFzLKl200auRhiAvPQbdpx75rqfA3w7s/BWk6hpwu2v4L5syCaIKMbdpXGTkEVy1x8C4Iri5Gh+KNT0qwuT+9tEyysP7uQy5H1B/Gqc4yur2Ek1Y5H4bubez+JWn2MjSaRFZzmI5yOBIqkH3UfjgVrfB7Q9NufhTr91cWUE080k8TSSRhjsWJSACegySfxr0nRPh/pHh7whe+HtO8xFvYXjnuXw0jsyldx6DgHgdP1pvhDwJB4S8J3egRX0lxHcvI5ldApXeoXpn2pSqJ3sNRPNvgToem3vgzXrq6soJ5ZZmt2aRAx2CMHAz05Y/pVT4Pahp2nfDHxZc6yjS6bHIPOiH8YZANo5HXgV6n4F8BweB9Du9LgvpLpbmYyl3QKVyoXGAfas7w38KdM0Lwrq/h+e9mvbTUyDIxQIyYGARjPIOD+FDmnf5AovQ82nm1TVPhjqM2g+DNG0jwwYpZPOuZjLM4BOSh67sjAyMDoOKkgJP7K1yCcgTgD2/0pa6yz+CXk6fJpN34u1WfRzuaOxT92isehPzEHB5xgc1sQfC6GL4bXPgttWme3mmEi3HkgMg3q+MZweVP503OP4i5WefeKf+TafD//AF1i/nJWf8TZrh/B/wANbPaGtZNPicxs21Xfy4hye3B69txr1fU/hnbal8O7Dwg2pSpDaMrC4EYLNjd2z/tVa1r4daTr/grTvDd/JKRp8EcVvdIAJFKIF3enIHI/wBoVSKfzYOLPL/G3hzxfr1rpsUnhbQ9FmtZP9Gmgvoo2IA+4OR7H2xWz8Q/Cuu6h4q0TxDol9pz69BapG2nTyx72dQzEorfKw+ZvTGMj20rT4KJLf2cviHxPqOt2ll/qLScEIB/dOWbjgZAxnFbvjP4Z2HizUbbVoL650rV7YBY7u264HTIyOR2IIP6UudJrUOVnnuieLZ7T4i6ZD438FWlprNy6Q2+oRRFGBJ2q2MlW5OCwPH4Yr3qvOdE+E6WviS217X/EN/r17a4Nt9pG1YyOQcEseDyOQM816NUVGm9Cop9QooorMoKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigBGVXUqwDKRggjIIrE06VvDN/DpcrE6RcvssZGP8Ax7SHpAT/AHT/AAHt93+7ncqvfWVvqVjNZ3Ue+CZSrLnB+oPYjqD2NaU5uDuTKN0atFYfh7ULh/P0nUZN+oWOA0hGPtERzslx74IPoyt2xW5XcndXRgFFFFMAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKwfGH/IDh/7CVh/6Vw1vVg+MP8AkBw/9hKw/wDSuGlLZjW5o0UUV5p0BRRRQAUUUUAYHiP/AJCXhv8A7CT/APpLcVfqh4j/AOQl4b/7CT/+ktxV+uin8Jy1viCiiirMgooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAzP+Z50z/sG3n/o22ro65z/AJnnTP8AsG3n/o22ro6wqbnXS+EKKKKzNAooooAKytN/5HnV/wDsG2X/AKNuq1aytN/5HnV/+wbZf+jbqtqHxkT2OirmdUb+2vEMOljmy0/ZdXfo8vWKM/THmH6R9jW7qN9Dpem3V/ckiG2iaV8dcKMnHvWP4fsp7TSw94B9uuna5usc4kfkrnuFGEHsoretPljbuRBXZqUUUVxGwUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAYuvRyWnka7aozXGn7mkRRkzW5x5ie5wAwH95FHc10kM0dxBHPC6yRSKHR1OQykZBFVayvDDfYZb7QW4WycS2o/6d5CSg/4CwdAPRV9a6sPP7JlUXU6KiiiukzCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigArB8Yf8AIDh/7CVh/wClcNb1YPjD/kBw/wDYSsP/AErhpS2Y1uaNFFFeadAUUUUAFFFFAGB4j/5CXhv/ALCT/wDpLcVfqh4j/wCQl4b/AOwk/wD6S3FX66KfwnLW+IKKKKsyCiuV8UeKpNG1vRNItmso59RlYtNeybI0jTGQORl2zhR60lt4pm1Dx/f+H7d7KKGwhXzPNfM80jKGBRc/dUEbj6mnYLHV0Vxeh+LdQfxRcaFrkVvDOOIXiUqGI57k9RyP/r07xJ4svrTxBaaJokME95JjzfNBIXPQcEY45PtXof2XiPbex025r30t3v2On6pU5+Tyv5W7nZUUxNyxL5rKXAG5gMAnvx2p9eczmCiiigAoopCwXqQPrQAtFFBIHU0AFFBIAyTgUUAFFFIWAxkgZ9aAFoopCQBknA96AFooooAKKTIzjIz6UtABRRRQAUUZGcZ59KKACikBB6EGloAKKz9a1eHRNIn1GZHkjhxlUxkkkAD8zRompHV9GtdQaMRGdd2wHO3k9619hU9l7a3u3tfz3L9nLk57aXsaFFUtX1OLRtKuNQnR3jhUEqnU5IA/U1RsNYu9W8JHVbO2UXUkUjQwk5G4EhQTx6D0qo4epKn7W3u3tfzGqUnHn6Xt8zborL8P3Gq3OliTWbZLe73kFE6be3c/zrUJA6nrWdSm6c3Bu9u2qJlHlk4hRSEgdTjNLUEhRSZGcZGR2paACiikDAkgEEjrQAtFFICD0INAC0UVl69capb6U8ujW8dxeBgAj9MZ57j+dXTg6k1BO1++iKjHmkompRVXTZLqXTbaS+jWK6aMGVF6K2OR1NWQQc4IOKUo8smuwmrOwtFFICCMggj2qRC0UUUAFFFFABRRRQAUUUUAZn/M86Z/2Dbz/wBG21dHXOf8zzpn/YNvP/RttXR1hU3Oul8IUUUVmaBRRRQAVlab/wAjzq//AGDbL/0bdVq1lab/AMjzq/8A2DbL/wBG3VbUPjInsL4mP2y70nRxytzcfaJx/wBMYcOfzkMQPsxrUrIhP2zxnqlweUsoIrNP9l2Blk/NWh/KteitK8ggtAooorEsKKKKACiiigAooooAKKKKACiisa88TWNvcvaWyT6heRnDwWabyh9HYkIh9mYU0m9hNpbmzRXOtq/iGQboNBs0HYXWpFG/EJE4/Wm/2p4q/wCgHo3/AIN5f/kaq9nIn2ke50lFc3/anir/AKAejf8Ag3l/+RqP7U8Vf9APRv8Awby//I1Hs5B7SPc6Siub/tTxV/0A9G/8G8v/AMjUf2p4q/6Aejf+DeX/AORqPZyD2ke50lFc3/anir/oB6N/4N5f/kaj+1PFX/QD0b/wby//ACNR7OQe0j3Okorm/wC1PFX/AEA9G/8ABvL/API1H9qeKv8AoB6N/wCDeX/5Go9nIPaR7nSUVzf9qeKv+gHo3/g3l/8Akaj+1PFX/QD0b/wby/8AyNR7OQe0j3Okorm/7U8Vf9APRv8Awby//I1H9qeKv+gHo3/g3l/+RqPZyD2ke50lFc3/AGp4q/6Aejf+DeX/AORqP7U8Vf8AQD0b/wAG8v8A8jUezkHtI9zpKK5v+1PFX/QD0b/wby//ACNR/anir/oB6N/4N5f/AJGo9nIPaR7nSUVzf9qeKv8AoB6N/wCDeX/5Go/tTxV/0A9G/wDBvL/8jUezkHtI9zpKK5v+1PFX/QD0b/wby/8AyNR/anir/oB6N/4N5f8A5Go9nIPaR7nSUVzf9qeKv+gHo3/g3l/+RqP7U8Vf9APRv/BvL/8AI1Hs5B7SPc6Siub/ALU8Vf8AQD0b/wAG8v8A8jUf2p4q/wCgHo3/AIN5f/kaj2cg9pHudJRXN/2p4q/6Aejf+DeX/wCRqP7U8Vf9APRv/BvL/wDI1Hs5B7SPc6Siub/tTxV/0A9G/wDBvL/8jUf2p4q/6Aejf+DeX/5Go9nIPaR7nSUVzf8Aanir/oB6N/4N5f8A5Go/tTxV/wBAPRv/AAby/wDyNR7OQe0j3Okorm/7U8Vf9APRv/BvL/8AI1H9qeKv+gHo3/g3l/8Akaj2cg9pHudJRXN/2p4q/wCgHo3/AIN5f/kaj+1PFX/QD0b/AMG8v/yNR7OQe0j3Okorm/7U8Vf9APRv/BvL/wDI1H9qeKv+gHo3/g3l/wDkaj2cg9pHudJRXN/2p4q/6Aejf+DeX/5Go/tTxV/0A9G/8G8v/wAjUezkHtI9zpKK5v8AtTxV/wBAPRv/AAby/wDyNR/anir/AKAejf8Ag3l/+RqPZyD2ke50lFc3/anir/oB6N/4N5f/AJGo/tTxV/0A9G/8G8v/AMjUezkHtI9zpKK54ax4giG640C1de62mo72/ASRxj9at2HiOxvbpbN/Ns74gkWt2nlu2Ou3+F8eqkik4SQ1OL2ZrUUUVJQUUUUAFFFFABRRRQAUUUUAFFFFABWNqZ+wa9pGqDhGkNjOf9iXGw/XzVjA/wB81s1meIbOS/8AD1/bwf8AHwYS0B9JV+ZD+DBTVwfLJMTV0dBRVbTr2PUtMtL+H/VXMKTJ9GUEfzqzXoHOFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABWD4w/5AcP/AGErD/0rhrerB8Yf8gOH/sJWH/pXDSlsxrc0aKKK806AooooAKKKKAMDxH/yEvDf/YSf/wBJbir9UPEf/IS8N/8AYSf/ANJbir9dFP4TlrfEFFFFWZHO+IPBek+JdT07UNQ8/wA/T23Q+XJtGdwbkY55AotvBek2vjC48UR+f/aM6lXzJ8mMAcDHooroqKd2FzhviNo3mWMevWreVeWJUlxwWXdx+IJz+dM+HektNFP4kvX827vHYIx/hG4hj9SR+Q966PxZY3Op+GL6ztI/MuJVUIm4DPzA9Tx2qPwdp11pXhazsr2LyriMvuTcGxl2I5BI6EV7axr/ALK9lzLm5refLa/ra/8Akegq7+p8l9b287b/AHXOJ0uyk+IWu6jcajdzpY2zARQxtjAJO3GcgcLycc1b0N7rwr47Hh4XUlxp9wuY1kOSmVJB9jkEHHXrUv8AYPiLwrrd3d6Bbw3tldHLQuwBXkkAjI6ZOCO1XPD3hzVrjxI/iPxAI47nGIYEIO3jHYkAAdsk85r0a+IpOnU9+LouFox0upWVtN0073Z1VKsOWXvLktout/T1KXhFmPxI8QgsSAZeM/8ATUURMf8AhdE67jjyxxn/AKYilm0bxHoXjK91XSLGK9gvN2Q0gXG4hiDkjGCOvpUmj+H9fj8ejWdShiKSITJJE67VJTAUDOeOBnFTOdFupW542lSsldXvZaW+RMpQvKpzKzhbfW+h39eWaxZeHzrd5JrmvXN/csxEcFpGcxdfl7jjgYyPevU6800PR/FPhe/vYLTS7e8Fww2XckgAAGeTznHPIrzsokoe0kp8rsrK6jfXX3mna3lqzlwTUeZ81n6pX+bD4fXk76Tr9m0srQQIDEsnVMh8/ToOKy/BnhYeJ9One8vrhLe3m/dRoeN5A3E59gBXReEvDus6Q+ux31uD9qT93MsikSMN3bORndnkCtD4f6LqGiaPcwajb+TK9wXVd6tkbQM/KT6GvTxeOhS+sVMPNXbhazXbW36nXWxEYe1lTkrvltt21OP1fU7DV/GV3ba7fTW2lWbNFFFECcsp29gevJz+FX/BWrRWni6bR9PvZbrSZkLQeaCCrAbuhAx3Hvwa0L/Qdb0TxXca3odrFew3QPmwOwBBOCepHcZBFavhuDxNLqNxfa00dtbMP3VmgU4PrkZIGPfkmjEYmg8G1Bpx5EkuZaS/w8t+a/W9n3CrVpuhaLVrbX6+lr3+Z1JzjjrXkd/YeHPtd4dV1271XUXY7PskZ+X27jr2BxXq91Cbi0mhDlDIjIGHVcjGa868P6T4s0CG70y20q1PnuSL15BhRjGeuSO4GPXivOyeapxqTU+V6aXUb/Np7dluc2CkoqT5rPTql+L/ACQnhC+ubn4ea9FNK7rBDKI9xyVBjPA9v8aq+E/CqeJfDMkt9e3OyOR0t4kYBUOASxGOTk/kPy1vDPhzV9N8M6/p11abZbiNxARIhEhKFex47dcda2PA2k32j+HDaX8Hkz+c7bd6twQMcgkV34vGwoqvLDzV3OLVmtratfqdFavGCqOlJX5la3oZXwtvbi50a7t5pWdIJQI9xztBHQe3H610Hi/VZtG8MXd3bnE4ASNv7pYgZ/DJNZHw90LUtDs76PUrbyGlkUoN6tkAH+6TXQ6/pC65olzp7PsMqja/91gcg/mK87G1MP8A2q56OnzJu2qtpc5a8qf1zm3jdf8ABPIUGgNobXsms3g8QYMgOHxvzwucenfPWu2trV/HXgCBrh8ahFuCTdPnU459iMZ/PtVG1g8cafpK6LBplvlPljvA6nauc9z/ADHTtW7rFv4li8KRWFmPtuozKVuLhGSIID1xnH0Bx6nrXqY3EKUoKE483PdNzTsvkklHybv01OyvVu48slfm0d07L7lZeVzhNPutV8W3Gm+G7i4EcNqW81w3zMF/mQOB+dbvxLiGn2WhwWY8pYmdYwO2AuKjufh/qGlWWmXuisX1aBt043gAk88ZIGB09xSfEmS4uNL0N7u3NtcOZPMi3BtrYXPIJBFdEa1GtjqDw8lyXlp52d3bs+hopwniKbptcuunnrrbzDxR4QOh6R/btvqd2+pQurTTO/3ySBkdxyR36UzxV4pu7rw5ocQna3+3R77qVMg4BCnp2zk4+lW9XsfGutWseiXNrbLbhl8y8VxiQDoTzn3xjOav+IvBc0+i6Wmkuv2vTFAj34HmDgn2ByM+nJrnpYijF0frc4ykpOzVmkraX7a6rsZwqwTh7aSbu9eytp+Jxt1faN4fubO98LapcyzK2J4pVYCRffKgYPTFbXjoz3XjHRI7SdoXuIY1RweV3ORn9a1EPjfVb21je1g0mGM/vpV2Pv8AXAOfw/nU/iPQdSv/ABtouoW1t5lpbGPzZN6jbiQk8E5PHoKr63TjiISnJcyjPVyUvRNpJei1H7aKqRcmrpPW6fpfRL0MPxP4HsdD8LXV1DdXMkkciMN5GDkqpzx7k10Pw90aDT9BS/jllaS+jVnVyNq7S2NvHvWr4s0ufWfDN5Y2oBnkClATgEqwbGfwrP8ABEeuWunfYNWsEtobdAsDBgWfk5zgn29K8+pjauIyySnUvLm1Tau1ZWsuupzSxE6uEalLW+vp/wAOR/EbTo7vwtLdO8ivaEOgUjDEkLz+dYnh/RobL4dajqkcsxmu7GUOrEbV27gMcV2fifTZtX8N3tjb486VBsBOASCDj9K5fQ7DxGvhHUNEvNNSJEtJI7Y713SO2Tg/Njv14p4PE/8ACeqfOlaabTaXu6frqOhV/wBm5ea1pL7v+HMm1dv+FNXrbjnzxzn/AKapVeTw0lx8PV125vLmS8ijBhUv8kaBtoUD6c/Wty38NaunwyutHa0xfvKGWLzE5HmKeucdAe9aP9iah/wrT+yPs/8Ap3kbPK3r13565x0967njoU5N05rWt3Xw2Wvp57G7xEYt8slrP8P8jlRoLax4DfxBqF/dT3kcbGAM+VRUO3GPfB5q4/iS/tPhXaTrO/2qaU2wmz8wUFuc+uFxmt6x0TUYfhm2kSW+L8wyqIt69S7Ec5x0I71StvB15efDyPR7tFt76KVpowzBgGycZK54IJ/OpeMw821WknGNXRafDrsu23kJ16cv4jTSnp6f5HGTHQbPR4b3TdZu/wC3k2u3ysAzHG4Zx29c84969EXxrZ2XhvS9S1JZS92mD5SA/MvDdxjmsaNfHZ0+30qLTre2aLan23eh+UdM8n2zxn2rtl0yGeyt4dSjgvpYlGZJYVILY5IGMDPtWOZYihJQVb3vee0k3a22kUkr6pPUjFVKb5efXV7NN2+S0RxniTxJJ4h8I3E2gC6CxTqlz8uG2EE8YJ4zjP8AhXL2B8LzNata6jqGjXyMN00g8xT9CMY57nA9q9J1zR71NJZPDTwafcbtzLFGqCUY6ZxwfQ1yOq6T4l8RwxWlx4csrWcMDJfblBOOvQ5x+db5fiaHsuSElCN39pKS0+0mrSXbc0w1WnycsXyq/dX+aasy18RNamhXTdLjvDFBcqHuJ4/4kyBnjt1OB14rm7jUNH8PX1le+F9TuZWVsXMMqsA6++QBzz+ldj4j8HXVxpektpcqvfaWiom8geYFxjrwDkZ545NMhPjXVdQtVktoNJt4z++kXY+/1wDn8Pr1p4TEYeGGjGLVlzcyclG/a65W3dbW2ChVpRpJJqyve7tf1Vm35WMrxws91480iC1uGhe4gjjWRTyu53GfyNT+JPDVv4Y8EXsdrPNIZriNi0hGRjjHH4/nWpr+g6le+PdI1K3tt9nbrGJZN6jbh2J4JyeCOgrS8b6Xeav4aktLGHzZzIjBdwXgHnkkCuenjlF4WmppR05tukuvoZxxCXsYKWml/v6nI+ItautP8CeH7O2maI3kA8yRSc7VVcjPvu/SsS9udE0MWl74Y1e6e+jcCVZEYLIMcnkAYzxjnrXZav4QvNU8HaTboFi1KwiXCMwwTgblyOM8DnpxUYbx1qcttbm0t9MEZ/fXI2Nu/wCA5P5CuvDYqhGn7sl8U+ZOSSeul1ZuSttY2pVaajo1u76pX7dHfTsZ/j27ubiXSLqeO5bQ5YVklSFsZY8kE9M4xjPvTPCv/CP/APCRQTaNrF3ZFhtaxuUz5p9N2cfhyc9K6fxHbeJo7q2utFnW4gjUCWzkCqH9TnjII7Z+lc/H4a1bXPEdlfXOjWuj29u6u5iK7pCDnoO/v/OscPXpSwXs5TUVZ7SX3OLWr8195FKpB0OVySVns/zT/NHpNFFFfJnjBRRRQAUUUUAFFFFAGZ/zPOmf9g28/wDRttXR1zn/ADPOmf8AYNvP/RttXR1hU3Oul8IUUUVmaBRRRQAVlab/AMjzq/8A2DbL/wBG3VatZWm/8jzq/wD2DbL/ANG3VbUPjInsReGz5sGo3h+9c6lcsT6hJDEp/wC+Y1rarE8IfN4R0uX/AJ7QCY/8D+f/ANmrbrOTvJlLYKKKKkYUUUUAFFFFABRRRQAUUVjeJrye30tba0kMd5fSrawOOqFslnHuqK7D/dppXdhN2Vyjd3dx4hu5rOznkt9Kgcx3FxE217hxw0cbDlVB4ZhznIGME1o2lpb2NsltaQRwQIMLHGoUD8KLS0gsbOG0toxHBCgREHYAYFTV0pJKyOOUnJ3YUUUUyQooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAqtfWFrqVsbe8gSaIkHDdiOhB6gjsRyKs0UAZen391pF9DpmpTtcW052Wd7J97djPlSHu2B8rfxYwfm5bpKxdSsItT06azlLKsi4Dr95GHKsPQggEH1Aqfw/qEup6Hb3FwALkborgL0EqMUfHtuU49qxqRtqjppT5lZmnRRRWRsFFFFABRRRQAUUUUAFFFFABRRRQBneDTt8NRW3/PpPPaAeixTOi/+OqK3q5/wudsuuQdotTf/AMejjk/9nroK9KLukzne4UUUUxBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABWD4w/5AcP/YSsP/SuGt6sHxh/yA4f+wlYf+lcNKWzGtzRooorzToCiiigAooooAwPEf8AyEvDf/YSf/0luKv1Q8R/8hLw3/2En/8ASW4q/XRT+E5a3xBRRRVmQUUUgYN0IP0oAWiikBBJAIyOtAC0UEgDJ4FICCMggj2oAWiuTs/E19ceP7vQ3SAWsKFlYKd/3VPJzjv6V1lb4jDzoOKn1SfyZpUpSptKXVX+8KKKQsAQCQCe1YGYtFQX13HYWFxeShjHBG0jBRyQBk4rO8Na7/wkWltfiDyVMrIqbsnAxyfetVQqOk6qXup2v5stU5ODnbRGxRRRWRAUUUUAFFFFABRRRQAUUUUAFYfiPwxa+JBai5nmi+zszL5eOc465HtW5RWtGtOjNVKbs0VCcqcuaLswooorIkKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAzP+Z50z/sG3n/o22ro65z/AJnnTP8AsG3n/o22ro6wqbnXS+EKKKKzNAooooAKytN/5HnV/wDsG2X/AKNuq1aytN/5HnV/+wbZf+jbqtqHxkT2IfBv/IjeH/8AsG2//ota26xPBv8AyI3h/wD7Btt/6KWtusnuUtgooopDCiiigAooooAKKKKACue1f954t0WFvupbXVwP95TEg/SVvzroa57U/wDkddJ/7B17/wCjLarp/ERU+FmjRRRXQcYUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQB5f8eJZIvAlq0bsjf2jGMqcH/VyVsweMdK8JeAfDNzq8kwW5soEUohclvKUnNYnx8/5EK1/wCwjH/6Lkrlfin/AMko8Ef9cIf/AEQtWldIpK6O8ufjN4NtdVNibyeQK2xrmOEtED9epHuARXQ+JPGWj+FtMttR1GZzbXLhInhTfuyu4HjtgVwvxc0uys/hNZwwW0UaWssAhCqBtyCDj69/Wud+JJLfBfwWScnZB/6INCSYWR3lx8ZfBttqv2FrydwG2tcxwlolP16ke4BFdD4h8YaP4Z0a31a/mZrO4kWOKSBfM3FlLAjHbCnmuN8Z6foS/A7MMNuLaO0hktXVRneSuCD6nJz65Oa4XxE9xJ+zr4aM+4kaiQpPXYPPC/pj9KLJhZHuWqeJtN0fwz/wkF20gsNkcmVTLYcgLx/wIVw3xD+JUdj4IsrzQrqWG61VS1rIYuRGpAc89DyMUnxBniHwGh+df3trZBOfvHMZ4/AE/hXOeI40b9nLR5Cil18oKxHIy5zg0JIEj0L4Z+KrXxL4Vto4p55ryygiiu3mByZCvJyfvcg812dc18P4Yo/AWhNHGiM9jCWKqAWOwcn1rpal7kvcKKKKQBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFUPDPyXOvwD7kWpHaP8AeghkP6uav1Q8N/8AIS8Sf9hJP/SS3qKnwmtH4jfooornOoKKKKACiiigAooooAKKKKACiiigDM8N/wDIU8Tf9hNP/SS3roK5/wAN/wDIU8Tf9hNP/SS2roK9GHwo55bhRRRVCCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACsHxh/yA4f+wlYf+lcNb1YPjD/kBw/9hKw/9K4aUtmNbmjRRRXmnQFFFFABRRRQBgeI/wDkJeG/+wk//pLcVfqh4j/5CXhv/sJP/wCktxV+uin8Jy1viCiiirMjkPHVtaXMNoNR13+z7INl4FQlpumcYOeB7Ec1xFrPp+neMdJfw7LfLbTTRxyGcYEgLAHHqMHv3rrfGegarda9p+s6bbx3n2ZVVrdyAMhic4J5Bz+lUNS0fxVq+u6Vq9zpsKLbyqfs0Uy7o1Vg2SScEnnp6V9dl9WlDDRhKorNSum4pJ66WtdvzbPaw04RpKLkrNPql+G79WQ+J4Z7v4n2VrBdSW7SwqhkjOGVSGDY98Zqvqekp4P8aaMdMuJwly6iQSPkt84DA+oINdFqOhalP8SLHVo7bdYxIA8u9Rg4btnPcdqPF2halqniPRbuztvNgtnBlbeq7RvB6EgngdqmjjYKVGk5rkdNqWq3s9H5+X+YqdeKcIcy5eXX113IfG9rpk+p2z6zrzW9mq5+wxoSzdfm4z1PcjtXPeF7i3sviDb22jveLp1wrAx3AwW+QnOO4yODW5r2h63beNk8QadZR6hGVA8p2A2ELt7n8QRTINE8Sy+OrHXb+yh8sjEiwyriEbSuDk5JGc8Z61VCrSjg/ZuomnTejcVaXblte/m3qOnOCocrkmnF9Vv2t38yPTlD/F/UlYAqYmBB7/ItHhCU+HfFusaHcyEQ4MsTOey8j80Of+A1pWGg6lD8Sb3VpLbFjIhCS715O1R0znse1YvxStRbXtjqUEmyeWN4HAOCy4/wYg/UVMJ08TWjhOa6nTitNbSirr7hRlGrNUb6Sil81qTeDEu9Wvte8RID50m+O1DdAx5x+A2D8a5S1bR3NzF4kXU49TZz/pGd2w9tynnr9fwr1bw/o0ml+EoNPjkMFyYSWkCglJG5JweuCf0rmJIPF6W81jqGiWes7ifKuZWQ4z9SOPyqsPjoSr1eVpK6S97ldlpo3o11aHSxEXUnZpLRLWzsu3T5E82n2mofDa4RdZl1GK1V5o5gNjAqpIRgcnHPQ84IqL4Y6NbpYNrIkm+0OXgKFhs25U5xjOePWr/h3wjd6Z4Q1LT7iRBd30bjaDlYyU2gZ7+9ReBLLxDpG/Tb/T44rFd7ibeCxc4wOG6de1c9avH6riKVKqn71+iura2Wl9exlOovZVIQnfXyV11O4opku/yn8v7+07fr2rE8LN4gazn/AOEhRUn8z93jZ93H+yfWvno0uanKpzLS2l9XfsuvmeaoXi5XWn3m9RXP+H28Rtf6gNaRVtg3+ikbORk/3TnpjrRp7eIz4pvFvUUaMFb7Ow2ZJyMdDu6Z61tLCcspR54+6r77+S7vyLdGza5lou+/p5nQUUUVyGIUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAZn/M86Z/2Dbz/wBG21dHXOf8zzpn/YNvP/RttXR1hU3Oul8IUUUVmaBRRRQAVlab/wAjzq//AGDbL/0bdVq1lab/AMjzq/8A2DbL/wBG3VbUPjInsQ+Df+RG8P8A/YNtv/RS1t1ieDf+RG8P/wDYNtv/AEUtbdZPcpbBRRRSGFFFFABRRRQAUUUUAFc9qf8AyOuk/wDYOvf/AEZbV0Nc9qf/ACOuk/8AYOvf/RltV0/iIqfCzRoooroOMKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooA57xl4Qs/GujR6Ze3E8EUc6zhocbshWGOQePmNZ3iL4c6d4j8OaTolzeXUUGmIqRvHt3MFQIM5GOg7V2VFO7C5geKvCdp4s8PDRru4mhhDo++LG75enUEV5r8atNj0b4d+HtMhd3itJ0gRnxuIWJgCcd+K9pqG4tLa7QJc28Uyg5AkQMAfxoTsNOx5fH8FtHv4rORtV1NNPKrMbASZRWIydpPQcn35612+t+DtI1zwuPD08BhsUVRCITgxbfulc56e/qa3gAoAAAA4AFLRdhdnmEPwS0o6a9nfazql2oXbbh5AEt+QSVXkZ7fj0rpbjwHpt14Cj8JTTXDWkaBUmyBICG3A9MdfbpXVUUXYXZyngjwRF4KtbqCLU7q9WcpgT4AjC5wFHb71dXRRSbuIKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKoeG/+Ql4k/wCwkn/pJb1fqh4b/wCQl4k/7CSf+klvUVPhNaPxG/RRRXOdQUUUUAFFFFABRRRQAUUUUAFFFFAGZ4b/AOQp4m/7Caf+kltXQVz/AIb/AOQp4m/7Caf+kltXQV6MPhRzy3CiiiqEFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFYPjD/kBw/8AYSsP/SuGt6sHxh/yA4f+wlYf+lcNKWzGtzRooorzToCiiigAooooAwPEf/IS8N/9hJ//AEluKv1Q8R/8hLw3/wBhJ/8A0luKv10U/hOWt8QUUUVZkFFFFABRRRQAUUUUAFchD8P7EawuoXd5dXex96RStlRzkD3A9K6+iuihiq1BSVKVr7mlOtOnfkdrhRRRXOZhRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAZn/M86Z/2Dbz/0bbV0dc5/zPOmf9g28/8ARttXR1hU3Oul8IUUUVmaBRRRQAVlab/yPOr/APYNsv8A0bdVq1lab/yPOr/9g2y/9G3VbUPjInsQ+Df+RG8P/wDYNtv/AEUtbdYng3/kRvD/AP2Dbb/0UtbdZPcpbBRRRSGFFFFABRRRQAUUUUAFc9qf/I66T/2Dr3/0ZbV0Nc9qf/I66T/2Dr3/ANGW1XT+Iip8LNGiisjxDcTpZQ2lrI0VxfTrbJKvWMEFnYe4RXI98V0pXdjjEn19WuZLXTLObUZ4m2ymIhYom9GkbjPqF3EdxUTXvijPy6NpGP8Aa1WQH/0nNadpaW9haRWtrEsUES7URegFTV1qhFLUnmMX7b4p/wCgLo3/AINpf/kaj7b4p/6Aujf+DaX/AORq2qKfsIBzGL9t8U/9AXRv/BtL/wDI1H23xT/0BdG/8G0v/wAjVtUUewgHMYv23xT/ANAXRv8AwbS//I1H23xT/wBAXRv/AAbS/wDyNW1RR7CAcxi/bfFP/QF0b/wbS/8AyNR9t8U/9AXRv/BtL/8AI1bVFHsIBzGL9t8U/wDQF0b/AMG0v/yNR9t8U/8AQF0b/wAG0v8A8jVtUUewgHMYv23xT/0BdG/8G0v/AMjUfbfFP/QF0b/wbS//ACNW1RR7CAcxi/bfFP8A0BdG/wDBtL/8jUfbfFP/AEBdG/8ABtL/API1bVFHsIBzGL9t8U/9AXRv/BtL/wDI1H23xT/0BdG/8G0v/wAjVtUUewgHMY4v/EaDdNolgy91ttSZ2/APEg/Wrum6vb6k0sSpLBdQ4862nXbJHnoT1BBwcMCQcHng1brF1tRa3mmaonEkV0ltIR/HHMwj2n23lG+q1E6KSugTN6iiiuUoKKKKACmyyxwxPLK6xxopZnY4CgdST2FOrB1NBquv2+lyfNZ28Qu7iM9JWLFYlPquVdiPVV96qMeZ2Acuu3d8N2kaU9xCfu3NzJ9nif3XhnI99uD2Jpv23xT/ANAbRv8AwbS//I1bVFdaoQJ5jF+2+Kf+gLo3/g2l/wDkaj7b4p/6Aujf+DaX/wCRq2qKPYQDmMX7b4p/6Aujf+DaX/5Go+2+Kf8AoC6N/wCDaX/5Graoo9hAOYxftvin/oC6N/4Npf8A5Go+2+Kf+gLo3/g2l/8Akatqij2EA5jF+2+Kf+gLo3/g2l/+RqPtvin/AKAujf8Ag2l/+Rq2qKPYQDmMX7b4p/6Aujf+DaX/AORqPtvin/oC6N/4Npf/AJGraoo9hAOYxftvin/oC6N/4Npf/kaj7b4p/wCgLo3/AINpf/katqij2EA5jF+2+Kf+gLo3/g2l/wDkanLfeJV5k0XTCB2i1N2Y/TdAo/Wtiij2EA5ihp+tRXl01nNbz2d8q7zbzgZZem5WBKsM46HjIyBmtOsHxQoh0dtTXibTWF2jjqAnLj6Mm5T9a3q5qsORjTuFFFFZjCqHhv8A5CXiT/sJJ/6SW9X6oeG/+Ql4k/7CSf8ApJb1FT4TWj8Rv0UUVznUFFFFABRRRQAUUUUAFFFFABRRRQBmeG/+Qp4m/wCwmn/pJbV0Fc/4b/5Cnib/ALCaf+kltXQV6MPhRzy3CiiiqEFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFYPjD/AJAcP/YSsP8A0rhrerB8Yf8AIDh/7CVh/wClcNKWzGtzRooorzToCiiigAooooAwPEf/ACEvDf8A2En/APSW4q/VDxH/AMhLw3/2En/9Jbir9dFP4TlrfEFFFFWZBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQBmf8zzpn/YNvP8A0bbV0dc5/wAzzpn/AGDbz/0bbV0dYVNzrpfCFFFFZmgUUUUAFZWm/wDI86v/ANg2y/8ARt1WrWVpv/I86v8A9g2y/wDRt1W1D4yJ7EPg3/kRvD//AGDbb/0UtbdYng3/AJEbw/8A9g22/wDRS1t1k9ylsFFFFIYUUUUAFFFFABRRRQAVz2p/8jrpP/YOvf8A0ZbV0Nc9qf8AyOuk/wDYOvf/AEZbVdP4iKnws0axdc/5Cvhz/sIv/wCktxW1WLrn/IV8Of8AYRf/ANJbiuun8SOM1qKKK9AzCimCWMyGMSIZB1XcM/lT6ACimySRxLukdUX1Y4FKrBlDKQQehFAC0UUUAFFFFABRRRQAUUUUAFFFFABRRRQAVj+Jv+QVB/2EbH/0qirYrH8Tf8gqD/sI2P8A6VRVMvhY1ublFFFecWFFFFABWJB/yOup/wDYOs//AEZc1t1iQf8AI66n/wBg6z/9GXNa0fjQnsa9FFFdxAUUUUAFFMMsayCMyIHPRSwyfwp9ABRRRQAUUUUAFFFFABRRRQAUUUUAY3i7/kS9d/7B1x/6Lat2sLxd/wAiXrv/AGDrj/0W1btcuI3Ra2CiiiuYYVQ8N/8AIS8Sf9hJP/SS3q/VDw3/AMhLxJ/2Ek/9JLeoqfCa0fiN+iiiuc6gooooAKKKKACiiigAooooAKKKKAMzw3/yFPE3/YTT/wBJLaugrn/Df/IU8Tf9hNP/AEktq6CvRh8KOeW4UUUVQgooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigArB8Yf8gOH/ALCVh/6Vw1vVg+MP+QHD/wBhKw/9K4aUtmNbmjRRRXmnQFFFFABRRRQBgeI/+Ql4b/7CT/8ApLcVfqh4j/5CXhv/ALCT/wDpLcVfrop/Cctb4goooqzIKK47xx4o1Hw7Np8dhHbubnfu81SeRtxjBHrWdd+JPHOlQm6v9EtfsycyFPmwPXKucfXFenRyqtVpxqJxXNsm0m7O2h1QwdScVJNa7XZ6FRWXoGuW/iHSo763BXJKyRk5KMOo/l+daRdA4UuoY9BnmuCpSnTm4TVmtznlCUZOMlqh1FUNZ1WLRdIuNRmR3jhAJVOpJIA/UimaDqbazoltqDRCIzgtsBzt+Yjr+FP2FT2Xtre7e1/O1x+zlyc9tL2NKim70D7N67v7ueadWRAUVh+KvEKeG9IN35Xmyu4jiQnALEE5PsADXKy6948trA6rNp9p9kC+YybRkJ1zgNu6V6GGy2rXgqiaSbsru135HTSws6keZNJPa7tf0PRqKyfDeuxeItGjvo4/LbJSSPOdrDqM/iD+Na1cdWlOlN05qzWjMJwcJOMt0FFFIWAIBIBPT3rMkWikJABJOAO5pQcjIoAKKRWVs7WBxwcGloAKKKKACik3Lu25G7rjPNLQAUVgeLtcuND0B7+yELyCRU/eAsvJ56EVWu9W1+Xwxpl/pFnDcXlwiPMhHyqCuSR8w74712U8FUnCNS6Sk7au2q11No4eUoqWiTdjqKKZCXaCMygLIVBYDscc07cu7bkbuuM81yNa2MRaKKRWVhlSCPY0gFoorn/FfihPDVrA32dp5rhisa5woxjJJ/GtaFCpXqKlTV2y6dOVSShFas6Ciua8beILvw5pEF3ZpC0jziMiVSRjax7Eegrds7gz2VtLIVEksSuQPUgE4qpYacaMaz2k2l8hulJQVToyxRRSBlYkBgSOuD0rAzFoornfFniqHwzZxkR+ddzZEUWcDjqT7fzrWhQqV6ipU1dsunTlUkoQV2zoqK87Os/EFLf7e2lQeRjeYtg3Y/3d27+tdL4V8UW/iaxeRU8m5hIE0Wc4z0IPoea68RltajTdW6lFaPld7eptUws4R57ppdnexv0UUV55zBRXP6D4oTX9V1G1ht2jisyqh3PLklgeOw4qnofia+1LxlqukTJALe1D+WUUhjhwoyc+h9K7XgK651JWcEm/R/8ADm7w1Rc11srs6yisLQrvXri/vk1ayigt0b/RnQcuMnr8x7Y9K3CyggFgCegJ61z1qTpT5G0/R3RnODhLl39BaKKQsAQCQCenvWRAtFFFABRRRQAUUUUAFFFFABRRRQBmf8zzpn/YNvP/AEbbV0dc5/zPOmf9g28/9G21dHWFTc66XwhRRRWZoFFFFABWVpv/ACPOr/8AYNsv/Rt1WrWVpv8AyPOr/wDYNsv/AEbdVtQ+MiexD4N/5Ebw/wD9g22/9FLW3WJ4N/5Ebw//ANg22/8ARS1t1k9ylsFFFFIYUUUUAFFFFABRRRQAVz2p/wDI66T/ANg69/8ARltXQ1z2p/8AI66T/wBg69/9GW1XT+Iip8LNGsXXP+Qr4c/7CL/+ktxW1WLrn/IV8Of9hF//AEluK66fxI4zWryr4ueKNUt73S/CehzNBe6mV8yVG2ttZtiqD2yc5Pt7mvVa8Q+K2dF+KfhnxBcK32FfK3NjOPLlLMP++WBrtnsTDcs3vwIht9Ge40/WLx9bjUyKzFVjkcc4HdcnvuNdTpWr+IvCnwyu7/xZEr39gjeXmUO0o4CByO+44J9OetdPqviLTdK8PTa3LdRPZpEZEdXBEvHAU9yegrhtb10fEf4Pa3dadY3MDKB+7lAO4xskjbSD8wwMdBzRZLYd29zlPCXgO8+JtvL4m8U6veGOaRlgjhIBwDgkZBCrnIAA7Gmalban8FvFGnzWeoz3eg3rHzIJPQEbgQONwBBDDHp657P4K69Y3vge30lJkF7YtIskJOGKs5cMB3HzY+ormPjhqcGsanovh3TnW5v1lYukZyVZ8Kqn3PPH09aiyUbrcq75rFX48NI/iPQXtSxka3JiMfUnfxjHeuw0vxgnjD4O65cSMP7Qt9NuIbtR/e8psNj0Yc/XI7Vg/EaD7N8SfAMGd3lyQJn1xMorD8dabdfDjxLqVxp0Z/sXX7OeBohwqM6kFfbaxDD2JHrQ202wWqSN/wCH3hS18Y/CBNLvJ5oIhqDy74cbsjtyD61xfiX4e6fonxI0bw1Bd3UltfrCXlfbvXfIynGBjovpXqXwP/5J2n/X3L/Suc8ff8l68Kf7lt/6Oem0uVME3zNG23w507wR4N8V3FjeXU7XOlzIwn24ACMeMAV594B+Hvh3xR4da/1TWpLO4Fw0YiWWNRtAUg4YZ7mvc/G//Ih+IP8AsHT/APos1458NPhv4e8WeFn1HVJrpLgXLxARTKo2gKRwQfU0SWqSEnpdnf8Agf4d+HvDOr3F9pmrSX7yW7W8kbyRuoVmU8hR/s1xMyyfB34lidQ//CN6p1ABIRc8j6oTx6qfevTPCHgTQ/B1xdSaTLcO9yqq4mlD8AkjGAPWvOvifqtx448XWXgfRFSXyJt08pHAlwc89lRSc++R2ptWQJ3Y3wpZT/FL4i3PijUYm/sfT3C28L9CRyifh95vc46GtDx58JtJjsdf8TC/vTckTXnlfJs3ElsdM459aX4O+IpNLubvwNq0S297bSu0GQAXOcup9T/ED3GfQV3/AI//AOSfa/8A9eUn/oNJJONwbakeO/D34U6V4w8LLqt5f3sMpmePZDt24GPUH1r6DjQRxJGDkKoAz7V518EP+Sdp/wBfcv8ASvR6qCSRM22wrH8Tf8gqD/sI2P8A6VRVsVj+Jv8AkFQf9hGx/wDSqKnL4WJbm5RRWZr2ix6/pjWMt5e2ilw3m2c3lycdt2DxXnFmfD4kml+Idz4aNughi04XgmydxJcLjHTHNdHXitv4Dtn+K93o/wDbmvCNNJW4FwL398T5gG0vj7vfHrXrWi6SmiaTDp8d1d3SRbsTXcvmSNlieWwM9cfTFNpDZoViQf8AI66n/wBg6z/9GXNbdYkH/I66n/2DrP8A9GXNaUfjRL2NevMvjZrzaf4Si0i3JN1qkoj2r18tSC2Pqdo/E16bXzx4r1nUPEPxeN1pWlTavFojqsdvECQSjcscA8eYfxAFdc3oEFqbnwdvLrw54o1jwXqeEmDedEueN6gbsf7y7T9Frpvi/wCMbvwv4ft7bTZDFf6g7Ikq/ejRQNxX3+ZQPqe9eX+J9c8Q2vjbTvGV/wCG7jSHieNG3BtsxXORkgclMr9BXUfHJTead4a8QWn72yBbDjp84R0/MKai9otIq3vJssW/wJhutIFxqGtXn9typvZ8ho1cjODkbmwe+RW14T8MeJ9Q8Eal4a8YvIkEm1bedZ1klCZyQTzwMDGeeSOwruLXxBpd3oKa3HewjTzF5pmLjCDGSD6EdCPWsbwR48tPHEV49pY3NuLVgrNLgo2c4wR3wMkY4yKu0UybyaPEfiV8PdP8FT6PHZXd1OL5pA5m2/LtKYxgD+8a9a8L/CnSvCGrtqtnf3s0ohePZNt24I9gK5T4+/8AH74X/wB+f+cVe1Mu5GX1GKSiuZjcnZHjH7Pf/Hjr3/XWH+T034of8lf8Hf79v/6UVH8ArmG2m17TZpFjuy8TCJjhmC7w2B7HGfrR8RZor741+FLW3kWSWCS3EoQ52Hzi2D6HHP0NL7CH9s9T8Xa6nhrwpqOrMRvgiPlA/wAUh4Qf99EV8++BLy/8H+LtC1nUCVstbV0eRj95GcqS30YK30I9a7T45atcX13pXhTT0ea4lb7RJFGMsxOVRcD/AIEcfSuZ8Zz+JNZ8KWNlc+CbrTrXSFBS52udkYXaQcjpwCT7UpPX0CK09T2f4jsyfDvXGRirC2OCDgjkVkfBqR5fhvZtI7O3nS8scn75rLl8Rf8ACTfAK9vXfdcx2ZguPXzEIBJ+ow3/AAKtD4MSRr8NrMM6g+dNwT/tmrveRNrRPQ6Karq+drBsehzTqsgxvF3/ACJeu/8AYOuP/RbVu1heLv8AkS9d/wCwdcf+i2rdrlxG6LWwUUUVzDCqHhv/AJCXiT/sJJ/6SW9X6oeG/wDkJeJP+wkn/pJb1FT4TWj8Rv0UUVznUFFFFABRRRQAUUUUAFFFFABRRRQBmeG/+Qp4m/7Caf8ApJbV0Fc/4b/5Cnib/sJp/wCkltXQV6MPhRzy3CiiiqEFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFYPjD/kBw/wDYSsP/AErhrerB8Yf8gOH/ALCVh/6Vw0pbMa3NGiiivNOgKKKKACiiigDA8R/8hLw3/wBhJ/8A0luKv1Q8R/8AIS8N/wDYSf8A9Jbir9dFP4TlrfEFFFFWZHm3xR/4/wDQ/wDek/mleh3So9nOsoBjMbBgemMc1x3j/wAP6rrc+myaZbiU2+8vmRVwSVx1I9DVO6h+IeqWz2U0VpbRTDY8iug+U9RkEn8hX0PsoYnB4eKqxi4817uzV5dj0uSNWhSXOla97vzKnw0MraHriLci1X5ds7DIjO1stz6cGsO/svDUWnXPl6lfanqo3P58UZCA56tnt6nJrvofBv2PwRdaJbzj7VcKWeY8Bn44+nAFYNjpPi1fDUvh5dLtbaIq4a5aRcuCScYBPJ6ZPavQpY2lOvVxEKlk5LS6jolvqrtf3UdUK8JVJ1IytqtLpad+79CrdxNq/wAJ4L+8nmeeyYhDu+8DIF+bPXAP6Vr+D9FhtPBsuoQ6jLaTXsB8yaQgpDtLDco4xx6mpLHw3qbfDe50SeBYbwsSil1Ib5w45BI56VV0/RPEl54Nu9AvbSK0SONRbPvBLsH3ENhjx26CsqteE6M6UaqUfa33Xwt9F1V9SJ1IyhKCmkufy2/XU5XUbPw3BpUxtL++1DVUO43EcZEYOepz2x3yea9O8FXc994Q0+e4kaSUqyl2OSdrsoz+AFcpDpPi6bws/h/+y7W1hVDumMi7pcHcFABIyTgZNdb4OsLvS/DFrY30PlTwlwV3BsgsSDkH3rPN60J4Xl51KSnp7ybtZ66JWXktiMbUjKjbmu1Lunpby/ITxbo9nrmj/ZLq6S2cPvhlcgAMOO/UYNca1v478M2xCul9YxLyvEq7R2wcNjHpXWeM/DJ8S6WkcLql1AxeIt0Oeqn0zx+Vc40vxEawOmmxhOU8s3O5N+Omc7sfpmsstqf7Ooc8Gru8Z2VvNMnCy/dKPNFq+ql080Xl8QRa/wDDvVbiO3W2mjidJY06BsZyPY1N4JYn4dkkknbNz+JqTRPBr6d4PvdKmmU3V6jeY68qhIwB7gf41jaNpfjKw0ubQVs7WK2csPtbuG2K3XaAee+MjvVS+rVKVWlQmklNNXdtLW0vv+Y37KUJwpySXMnr2K3hJmPw28QksSR5vOf+mYpvgrwpHrum2+pX15cEW1xiCEEbdoIY5+pJ6VreHfDWq2HgnWdNubYJdXPmeUnmKd2UAHIOBzW14I0q80bw4lpfRCKcSuxUMG4J45Fa4zHxpwruhNXc1a1tra2/IuviFGNR05auS+6xw1ro7eIfH2t6dLdTQ2vmyySiM8sA+AOfc5/Cr/iSKe0utF8F6fcyQ27qvmyZ5fcxHOO3BOPetrw9oGpWHjjWNSuYAlpc+Z5T71O7LgjgHI4FSeMfDF5ql1Z6tpMipqNpjarHG4A5GD0yDnrwc1Tx9N4unCU1yKKttZS5bJv0/AHiYutGLkuVLTte27Oc8SeGR4KtrbWNFvrlJElEciyMDuyCewHHHIPrUnji9N83hS8XKfaF83aD03eWcfrVi+0rxf4ua3s9XtrfT7KJw8jIwJY9M4DHJwTjoOa0fGvhW71HT9NOkIpk0/5UiLAZXC4wTxkbRVUsTTjWofWailNc15JppJppJv1+4qFWKnT9rJOWt35NaXZB8VGK6DZ4JB+1Dof9lqz/AIiySR6X4eaJirgEgj12pTvEGkeMfE+mwG6sbaEwv8tvHINzEg5cktgAdMZzzWj4z8O6pq9jo0VlbiR7YHzQZFXbwvqeehqcHOjh3hoTnH3XO+qsrrTUVCUKbpRlJaOV9fIw/GfhePw/pttq8F9dyagZ1WWZ5OWYgncO45FXvG+r3s+kaFYQzGNtSjV5mXjdkLx9Msc/QVv+PNHvtb0GO10+ESzLcK5UuF4CsOpPuKpeI/Cd5q3hzShbFU1KwiUBSwAb5RkZ6ZyowelRhcbSqLDzxMk5KUt7aaaX8rk0cRCSpyqvVN/Lt8rmB4v8HW3h7w4J7C6uSpkRJ45Hysno2OxB/nTvFrFfhx4d2kjiLof+mZqzrGneNfEuji3vbO2gELKwjR1DTt0yfmIAAJPb/C54i8NarqHgvRtOtrYPdWwTzU8xRtwhB5Jwea6KWJUXQWIqxlJTbbutrdX/AF2NYVUvZ+1mm1J318ih8RZJY7Lw40TlZMMVPvhMVV8aeGI/Dun2urW19dvfm4CSzSSZLMVJ3DuOV/Wt7xl4d1PV7bREsrcSNag+bmRV28J6nnoav+PdGvtc0GG10+ESzLcrIVLhflCsM5J9xWGEx0aSw0FNJXnzbbX0v5GdHEKCpRUla7v9/UwvHGrXtxp2h6dDMYjqKK0zLxnO0AfTJPH0ql4v8JQeG/Don068uQryLHcRu+Vk7hsAcEEfrWv4u0JLrQtIV762stStYwIhPMED4VdwB9QQP8msLxaPFE/huObXXtYYI5VCRw4LSsQfmJBI6Z6evSt8vmrUI0ZqK5nzL+bXS3f9DTDS0pqEkld3XfXT1/Q9G8NnPhfSSev2OL/0AVwPxS02GO+sb8PIZbjMbKSNoC4xjjrzXfeHFK+GNKBGCLOLI/4AKwfiD4fv9csbN9OjEstvIxMe4KSCByM8dq8rLa8aOac0pWV5J/j+tjiwtRU8XduyuzG8d6PBoXgmysbaSaSNb3cGmYFuVc9gKqeIPCkVl4Qh1xry6k1FViZnZ+PmwMD0AyMfStjxFpniPxJ4StorjT4o9QW73tFHIoAQKwByWPr61r+I9IvdQ8Ef2bbRB7ry4V2bgOVK55Jx2Nd1HHSoxoxdRX53zara63fZ/cdMMQ4KmnJX5nfXp/kcz4l8SX8XgLRgk7rcX0eJZQcMQoGeffIz+Nc9ez6FpEFreeHNVuzqUTjzC6kK4xyeg79vQ12moeDrvU/A+m2J2xajZLlVZsgnuuR+HPtTN/j/AFH7NatBDp3lsPNuwytv7dMnPrgDr6VvhsTh4U7U2tJS5k5KKavppZ8ytpoaUqtKMbRa3d9bX7dHdW7HZ6fdfbtMtbzGPPhSXHpuAP8AWvPPFW2T4paOlx/qv3O0HofnP9a9KRSsaqzl2AALEAEn14rmPGXhM+IoYbi1lWG/t/8AVsxwGHXBI6c8g14eV4ilSxTc3yxkmr9r7Hn4SrCFZuWid16XOprzPwThPiNrkdv/AMe+JsAdOJRj+tTmb4jtbfYfskIbG37UGTdj1zux+ma3/B3hQeG7SV55FlvbjBldeigdFGf510xhDA4asp1Iyc0klF367vt5GqjHD0ppyTctFZ3+Zg/Chmaz1PcSf3idT7Gq/he2XUfFfiuymkkEU3nRkoeQDIRxmn6NovivwrqF7babY21zbXDDbNLIAq4zgkZB6HkY+laPg3w5q2j+INTudRVWScHbMrD9427JOAcj8a7sXVpKWJrwqR99R5bNX0t0OitOCdWpGS95K2uvQ53wDoNtd+Ir6V5Zw2nTK0QVhhvmYfNxz07YrV8J/wDJTvEH0l/9GLTtD0bxJ4f8VXZgsYZbC7uAZJ2ccR7icgbgc4J4wa0PD+galYeOdY1K4gCWlyJPKfep3ZcEcA5HAp43FQnKtJ1E1KC5dV5XXre+gYispOo+ZNOKtr6GJ4OvWsr7xZdnL/Z1eUKT12lzj9K5+wn0PVo7q88SapeDUZHPllFJCDHB6Hv29BXbeFvDF9Z3+vjUrcJbX+VUiRTuUls9Dxwao6dp3jHwrFc6dptlb31tI5aKYuBtJGM4JHoODxmtliqHtqrpzXM+Sz5lHRJXtJprfddTT21PnnytX93ql076lfQtZudR+HeuW9zM0r2kRVJGOTsI4GfbBpvgrwpHrOnWeq3t5cMbef8AcQgjaFVskHvy2eldMmla9L4KvrPUroXeo3ETBE+UBMjhd3GT7mrXgvTLvR/DUNnexCOdXclQwbgnI5FcWIx8adCs6ElFufR30tq1otG+tjnqYhRpzdNpNy6emthYfDCxeLptf+2OTIu3yNnA+UL1z7elb9YEJ8Rf8JdMJVT+w9v7s/JnO0f8C65rfrw8VKo3Dnkpe6rW6Lon5rqcFZybjzNPRbfkFFYAPiP/AITAgqn9hbeD8mc7P++vvUakfEX/AAk1kLBUOkbV+0k7M53HPXnpt6ULC3klzx1jff8AD/F5B7HVLmW19/w9fI36KwPER8RC7sP7DVDDuP2nds6ZGPve27pW/WU6XLCM+ZPmvonqrd+3kRKFoqV1r/WoUUUVkQFFFFAGZ/zPOmf9g28/9G21dHXOf8zzpn/YNvP/AEbbV0dYVNzrpfCFFFFZmgUUUUAFZWm/8jzq/wD2DbL/ANG3VatZWm/8jzq//YNsv/Rt1W1D4yJ7EPg3/kRvD/8A2Dbb/wBFLW3WJ4N/5Ebw/wD9g22/9FLW3WT3KWwUUUUhhRRRQAUUUUAFFFFABXPan/yOuk/9g69/9GW1dDXPan/yOuk/9g69/wDRltV0/iIqfCzRrF1z/kK+HP8AsIv/AOktxW1WLrn/ACFfDn/YRf8A9Jbiuun8SOM1qzNe8P6Z4l0t9O1W2WeBjuHOGRuzKexrTorvMzymP4C+HVuQz6lqb24bIhLoPwztr0vTdNs9I06DT7C3SC1gXbHGvQD+p9+9W6KSilsNtvc831z4K+GtWv3vbWS602R23Mlsy+Xk9SFI4/A49q0vCXwu8PeErsXtus13fAYWe5IJjz12gAAfXk+9dtRRyrcfM9jmNf8AA+n+IvEOla1dXF1HcaY6vEkTKFYq4YbsgnqOxFX/ABP4asPFmiS6VqIfynYMrxkB42HRlJB56j6E1sUU7IV2YfhPwvZ+ENEGlWM08sIkaTdOQWyfoAO1UdY8B6brXjDTvEtxc3SXdiIxHHGy+W2xywzkE9WPeuqoostgu9ynqunRavpF5ps7ukV3A8DsmNwDAgkZ7815p/woLw3/ANBLVv8AvuP/AOIr1eik4p7gm1scB4b+E2k+FtSkv9O1LUvtDQPCrO0Z2bh94fJ1HbPFaXhD4eaR4Nu7u7s5bq5urkbWmumVmAzkgEAdTgn6CutooUUg5mcd4h+HGla/4jt9f+1XljqMG3EtoyruKn5WOVPI6fSuj1jS4ta0W70u4kkWK6haJ3jwGAIwSMjGavUU7ILsw/Cnhez8IaINKsZp5YRI0m6cgtk/QAdq3KKKACsfxN/yCoP+wjY/+lUVbFY/ib/kFQf9hGx/9KoqUvhYLc3KKKK84sTaN27A3dM4paKKACsSD/kddT/7B1n/AOjLmtusSD/kddT/AOwdZ/8Aoy5rWj8aE9jVkUvE6K5QspAZeq+4zXMeDvAWmeCjevYz3VxLdlTJLcsrNxngYA9Sa6miu2xNzF8U+GbHxdocmlagZEiZ1cSREB0YHqMgj1HToTUdj4R0618JJ4aufM1DT0Qx4uiCxXOQMgDp2I5GBW9RRZBdnlMvwF8Ovclo9S1OO3LZMIdDj2BK/wA816HoOgab4a0qPTdLtxDbocnnLOx6sx7mtOikopbA5N7nK+MfAem+NZdPkv7m6hNiXMfkMo3btuc5B/uiuqoop2C5554k+Dvh7xBqsmpJLdWFzK2+X7ORsdj1bBHBPsateEvhXoPhLUBqMLXF5fKCEluCP3eeDtAA5x3Oa7milyq9x8z2OTj8Aaavjp/F0t1dz35ztjkZfLT5dgwAueB7109zbxXdrNbToHhmRo5FPRlIwR+VS0U7WFc4fSfhfpej+HdW0ODUNQez1MASCR0JQjuuF6njOc9BWB/woLw3/wBBLVv++4//AIivV6KXKh8zOV8F+AtN8Drerp9zdzC7KF/tDKcbd2MYA/vGuqooppWE3cxvF3/Il67/ANg64/8ARbVu1heLv+RL13/sHXH/AKLat2ubEbopbBRRRXMMKoeG/wDkJeJP+wkn/pJb1fqh4b/5CXiT/sJJ/wCklvUVPhNaPxG/RRRXOdQUUUUAFFFFABRRRQAUUUUAFFFFAGZ4b/5Cnib/ALCaf+kltXQVz/hv/kKeJv8AsJp/6SW1dBXow+FHPLcKKKKoQUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAVg+MP8AkBw/9hKw/wDSuGt6sHxh/wAgOH/sJWH/AKVw0pbMa3NGiiivNOgKKKKACiiigDA8R/8AIS8N/wDYSf8A9Jbir9UPEf8AyEvDf/YSf/0luKv10U/hOWt8QUUUVZkFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFAGTr/AIesfEdmtvehxsO6OSM4ZD7VgRfDaxMsX2zUr67hiPyQu+Fx6fT6YrtaK7KOYYqjD2dObS/rbt8jeGJq048sZWQiqqIEUBVUYAHQCloorjMAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAzP+Z50z/sG3n/o22ro65z/medM/7Bt5/wCjbaujrCpuddL4QooorM0CiiigArK03/kedX/7Btl/6Nuq1aytN/5HnV/+wbZf+jbqtqHxkT2IfBv/ACI3h/8A7Btt/wCilrbrE8G/8iN4f/7Btt/6KWtusnuUtgooopDCiiigAooooAKKKKACue1P/kddJ/7B17/6Mtq6Gue1PjxppB7f2feD8fMtv8DV0/iIqfCzRrE8QkQy6NePxFbagpkPoJI5IQf++pVrbqG6tYL20mtbmMSQTIUkQ9GUjBFdMXZ3OMdRWHHLq2ir5FzbzapaLxHdQYMyr6SISNxH95ck/wB0dac3imwU4a11jPto92f1EeK71Ui1e5FmbVFYn/CV6f8A8+2s/wDglvP/AI1R/wAJXp//AD7az/4Jbz/41T549wszborE/wCEr0//AJ9tZ/8ABLef/GqP+Er0/wD59tZ/8Et5/wDGqOePcLM26KxP+Er0/wD59tZ/8Et5/wDGqP8AhK9P/wCfbWf/AAS3n/xqjnj3CzNuisT/AISvT/8An21n/wAEt5/8aqObxjpNugeaLVo0LKgZ9HuwCzEKo5i6kkADuSBRzx7hZm/RWJ/wlen/APPtrP8A4Jbz/wCNUf8ACV6f/wA+2s/+CW8/+NUc8e4WZt0Vif8ACV6f/wA+2s/+CW8/+NUf8JXp/wDz7az/AOCW8/8AjVHPHuFmbdFYn/CV6f8A8+2s/wDglvP/AI1R/wAJXp//AD7az/4Jbz/41Rzx7hZm3RWJ/wAJXp//AD7az/4Jbz/41R/wlen/APPtrP8A4Jbz/wCNUc8e4WZt1jeIz5lvp9qv+tn1G22D18uRZW/8djY0g8SQzDFppur3EnZDp8sGf+BSqij86l0/T7ubUP7V1Xy1uFQx29tG25LdT15/ic4GTjAAwO5OdSpFRGkbNFFFcRQUUUUAFYeRB41m38fbNPjEXuYZJNw/8jLW5VDVdMXU4IwsrQXMD+bb3CDJifBGcdwQSCO4Jq6cuWVwLdFYq6vfWQ8vVdKuNw/5eLGMzxP7hVy6/Qrx6nrSf8JVp/8Az7az/wCCW8/+NV2qcX1IszborE/4SvT/APn21n/wS3n/AMao/wCEr0//AJ9tZ/8ABLef/GqfPHuFmbdFYn/CV6f/AM+2s/8AglvP/jVH/CV6f/z7az/4Jbz/AONUc8e4WZt0Vif8JXp//PtrP/glvP8A41R/wlen/wDPtrP/AIJbz/41Rzx7hZm3RWBD4x0m4QvDFq0iBmQsmj3ZAZSVYcRdQQQR2IIqT/hK9P8A+fbWf/BLef8Axqjmj3CzNuisT/hK9P8A+fbWf/BLef8Axqj/AISvT/8An21n/wAEt5/8ao549wszborE/wCEr0//AJ9tZ/8ABLef/GqP+Er0/wD59tZ/8Et5/wDGqOePcLM26KxP+Er0/wD59tZ/8Et5/wDGqUeJrWQYgsdYlfsh0u4jz+MiKo/E0c8e4WYviw7vCmpwD791A1rGPV5f3aj82FbtYdtZX2p38N/qkQtoLcl7ayDhmD4xvkI43AEgKMgZzknGNyuStNSehSVgooorEYVQ8N/8hLxJ/wBhJP8A0kt6v1Q8N/8AIR8RnsdSXH/gLbioqfCa0fiN+iiiuc6gooooAKKKKACiiigAooooAKKKKAMzw3/yFPE3/YTT/wBJLaugrn/Df/IU8Tf9hNP/AEktq6CvRh8KOeW4UUUVQgooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigArB8Yf8gOH/ALCVh/6Vw1vVg+MP+QHD/wBhKw/9K4aUtmNbmjRRRXmnQFFFFABRRRQBgeI/+Ql4b/7CT/8ApLcVfqh4j/5CXhv/ALCT/wDpLcVfrop/Cctb4goooqzIKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKAMz/AJnnTP8AsG3n/o22ro65z/medM/7Bt5/6Ntq6OsKm510vhCiiiszQKKKKACsrTf+R51f/sG2X/o26rVrK03/AJHnV/8AsG2X/o26rah8ZE9iHwb/AMiN4f8A+wbbf+ilrbrE8G/8iN4f/wCwbbf+ilrbrJ7lLYKKKKQwooooAKKKKACiiigArn/E4+yyaXqvRLS52Tn0ilBQn6BjGxPoproKiubaG8tJrW5jWSCZDHIjdGUjBB/Cqi7O4pK6sVKKx9NuJtOuhoepSM1wik2lw/8Ay9xDvn/noo4Yf8C6HjYrpOJpp2YUUUUCCiiigAooooAKKKKACsPxZ/yB7f8A7Cen/wDpXDW5WH4s/wCQPb/9hPT/AP0rhprcEblFFFIAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKAMPwn/yB7j/ALCeof8ApXNW5WH4T/5A9x/2E9Q/9K5q3KGAUUUUAFFFFABRRRQAUUUUAFFFFACMyojO7BVUZJJwAKp+EUZtD+3OpV9QmkvMEYOx2/d59/LCA1R1AnXr1tCtiTbjH9pTL0SPr5IP95xwR2Uk8ErnqgAoAAAA4AHasqj6HRRj1FooorE3CiiigAooooAKKKKACiiigAooooAzPDf/ACFPE3/YTT/0ktq6Cuf8N/8AIU8Tf9hNP/SS2roK9GHwo55bhRRRVCCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACsHxh/yA4f+wlYf+lcNb1YPjD/AJAcP/YSsP8A0rhpS2Y1uaNFFFeadAUUUUAFFFFAGB4j/wCQl4b/AOwk/wD6S3FX6oeI/wDkJeG/+wk//pLcVfrop/Cctb4goooqzIKK5jxh4pk8PRW0FpbrPfXTERK2doxgZIHXkgAVgXviHxvoMK32p2No9puAdVx8ue2VPH15Felh8rrV4RmmlzbJuzfodVPB1KkVJNK+13v6Ho1FYVz4otbfwouviKSSFkVhGMZyTjB+hq9omoNqui2t+8YjadN+wHIHtXJPDVYQdSSsk+X59jGVKcY8zWl7fMv0U3zE37N67/7ueazPEOuReHtIkv5YnlCsFVFOMk9OewqKdKdSapwV29iYwlOSjFas1aKxH1HUrvwjFqGm28b6hNBHLHET8uWwSOSOxPermjSahLpNu+qxLFekHzUTGAcnHQntjvVzoShBybWjta+v3dvMqVNxjdvrbzL9FNWRHYqrqxHUA5xWH4r8TJ4Z05JzbtPLKxSNc4GcZyT6VNGhUrVFSpq8mKFOVSShFas3qKx9eudZj0iOXRLeOe8Z1yj4xtIOTyR7Vo2TXD2Fu12gS5MSmVR0D4GR+eaJUnGmql1q7Wvr9wOFo81/8yeimq6MSFdSR1APSlJABJIAHc1lYgWimo6SDKMrD1BzXKN4kvh8Rl0HEP2MpuztO/8A1e7rn19q6KGGnX5uT7Kcn6I0p0pVL26K/wBx1tFFcVr3jK+XWv7D8PWa3V6vEjtyqnuByOnck4FPC4WpiZ8lPpq29El3bHSozqytE7WivOZPF/ibw7eQDxHp8RtZTjfEBkeuCCRkehr0OKVJokljYNG6hlYdCD0NXisDVwyjKVnGWzTumOth50rN6p9Vqh9FNV0YkKykjqAelOrjMQopN6hwpYbj0GeaxvE/iGPw1pQvHgaZnkEaIDgbiCeT2HBrSlRnWmqcFdvYqEJTkoxWrNqioLOc3NjbzsAGljVyB0BIzUodWYqGBYdQDyKhxabT6Cas7DqKKKQgorntd8UJo+rabpq25lmvZUXcThUUsFJ9zXQ1rOhUpwjOSspbFypyjFSa0ewUUUVkQFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAZn/ADPOmf8AYNvP/RttXR1zn/M86Z/2Dbz/ANG21dHWFTc66XwhRRRWZoFFFFABWVpv/I86v/2DbL/0bdVq1lab/wAjzq//AGDbL/0bdVtQ+MiexD4N/wCRG8P/APYNtv8A0UtbdYng3/kRvD//AGDbb/0UtbdZPcpbBRRRSGFFFFABRRRQAUUUUAFFFFAFTUtMtNWtDbXkW9MhlIJVkYdGVhyrDsRzWKbfX9K+VVXWbUfdO5YrkD0OcRuffKfQ9a6WiqjJx2JlBS3OXPiGGIYudN1mFx1UabNL+sSsP1pv/CU2P/Pnrf8A4JLz/wCNV1VFX7XyM/Yo5X/hKbH/AJ89b/8ABJef/GqP+Epsf+fPW/8AwSXn/wAarqqKPa+QexXc5X/hKbH/AJ89b/8ABJef/GqP+Epsf+fPW/8AwSXn/wAarqqKPa+QexXc5X/hKbH/AJ89b/8ABJef/GqP+Epsf+fPW/8AwSXn/wAarqqKPa+QexXc5X/hKbH/AJ89b/8ABJef/GqxvE/iSym0mBVtdYBGo2LZfR7tBgXURPJjAzxwOpOAMkgV6HWD4v8A+QLb/wDYU07/ANLIaaqa7B7FFP8A4Smx/wCfPW//AASXn/xqj/hKbH/nz1v/AMEl5/8AGq6qil7XyD2K7nK/8JTY/wDPnrf/AIJLz/41R/wlNj/z563/AOCS8/8AjVdVRR7XyD2K7nK/8JTY/wDPnrf/AIJLz/41R/wlNj/z563/AOCS8/8AjVdVRR7XyD2K7nK/8JTY/wDPnrf/AIJLz/41R/wlNj/z563/AOCS8/8AjVdVRR7XyD2K7nK/8JTY/wDPnrf/AIJLz/41R/wlNj/z563/AOCS8/8AjVdVRR7XyD2K7nK/8JTY/wDPnrf/AIJLz/41R/wlNj/z563/AOCS8/8AjVdVRR7XyD2K7nK/8JTY/wDPnrf/AIJLz/41R/wlNj/z563/AOCS8/8AjVdVRR7XyD2K7nK/8JTY/wDPnrf/AIJLz/41R/wlNj/z563/AOCS8/8AjVdVRR7XyD2K7nK/8JTY/wDPnrf/AIJLz/41R/wlNj/z563/AOCS8/8AjVdVRR7XyD2K7nK/8JTY/wDPnrf/AIJLz/41R/wlNj/z563/AOCS8/8AjVdVRR7XyD2K7nK/8JTY/wDPnrf/AIJLz/41R/wlNj/z563/AOCS8/8AjVdVRR7XyD2K7nK/8JTY/wDPnrf/AIJLz/41R/wlNj/z563/AOCS8/8AjVdVRR7XyD2K7nnnhjxJZQ6TOrWusEnUb5spo924wbqUjkRkZ55HUHIOCCK2f+Epsf8Anz1v/wAEl5/8aq54Q/5Atx/2FNR/9LJq3qbqa7C9ijlf+Epsf+fPW/8AwSXn/wAao/4Smx/589b/APBJef8Axquqope18h+xXc5X/hKbH/nz1v8A8El5/wDGqP8AhKbH/nz1v/wSXn/xquqoo9r5B7Fdzlf+Epsf+fPW/wDwSXn/AMao/wCEpsf+fPW//BJef/Gq6qij2vkHsV3OV/4Smx/589b/APBJef8Axqj/AISmx/589b/8El5/8arqqKPa+QexXc5ceIYpRi203WZnPRTps0X6yqo/Wni113VvllA0a0P3trrLdMPTIykf1Bc+mDzXS0UOo+g1RiitYafa6XZpaWcIihTJABJJJ5JJPJJPJJ5J61ZoorM1CiiikAUUUUAFFFFABRRRQAUUUUAFFFFAGZ4b/wCQp4m/7Caf+kltXQVz/hv/AJCnib/sJp/6SW1dBXow+FHPLcKKKKoQUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAVg+MP+QHD/2ErD/0rhrerB8Yf8gOH/sJWH/pXDSlsxrc0aKKK806AooooAKKKKAMDxH/AMhLw3/2En/9Jbir9UPEf/IS8N/9hJ//AEluKv10U/hOWt8QUUUVZkct4y8O23iCK2QXsVrfxE+QXbG7PbHXqBgiuXubrxz4Zty9+sd/YJjcZMSrjPc8N+JrovGnhW61mS11HTJVj1C1+7uONwByMHsQf51jX0XxA1uybTLmytoIZMLJKGQbh74Y/oK+oy+pF0KcJThKHVT0cdfs9T1sNJezinKLj1Ut16B4tks/EngK212NXhe3OFiUjaGZgrA8c9OOlXPDGh29h4Inu4NSlspNQt1aa4kIKw4zyvTHBPert34Rlj+Hz6FaOslyAH3NwHfeGP09B+FZdtofiTUfBdzol9bQ2vkrGLU7wTJtYkhsE+2OlEa9KWF9jTqqMFU62vy3VtHvrr+Y1Ug6PJGdoqX4enXXU5TUrTw3Bo8h0+7vr7UoyGa6SMrFndznPQenXnFbHiO3/tP4daRrF1NK93CvlgkjDAnBJ4yT8o71c/sjxdeeFDoLabaWsESAF/MXdNtOQBgkAkgZJrRn8N6nd/DaHSGgWO/hORGXUhsOT1BxyDXbUxlOMqTdRNqpvzJvlatfRJJPstjeVeKcG5aqXdPT/L8jObRbbSvhVd3EDys97BBNIHIIDZXpgDjn3qOddSb4R6eNOEp+Y+eIs7vL3Pnp2zjP+Ga0YtN8SXvgO80e90+KKWOOKK1VZFy4VhksdxHQD0q1DoevweBbKxsbr7FqMG4um4EOCzHG4ZweQc1yvExj8c4uXtb6u6ty6PTp0v0Zi6qXxSTfPffpY4mxfwrM1oYLvUNGvY2BaZv3ik/UHjnvgD2rZ+KGnweXp+qrM0k0qiEkEbGUAsCMDqc+tSapo/ijxJDFaXmhWFrKGBkvty7jj6EnB9Oa1vF3hS8vvC2nWGnATy2O1drMFLqF25GeM9K6Xi6UcXRqOp1aa5lJJP8Avdr9H+Br7aCrU5Ofe+qenr/mZfi7RbbQfh9DaWryvG12khMpBOSp9APSk8Vapd23hLw7ptpK0RvbeMSOpwdoRBjPoS36Vf1nTPEniDwWltd2EUd+t0pEUcigeWFxnJYjOT61Y1/wjc6x4V0qCNli1GwhQKGbgnaAy5HuBz7VzUcTRi6X1mabU5N6p7rR+l+plTqwXJ7WSfvO/Xpo/vK0nw3Fm9pcaJqU1tdxMN8shyGHqAPft0xU3je2sJ5LE6zrptbZBlrSKMkzHuwwSR6cg4qlPaeO9dNtZ3ax6dDE4Z7mGUKzY4z8rEnvwMDNT+JPDmsR+JrHWtLgS/FvGqGKZxnK5GTkjOc5z1zzUQnP28HiMRFySlZq3yXM00r9Owoyl7SPtKibV9rfn5/gc3pM1nYePdOGh/bYrOcqrrcDHmA5Bx6r069wa3H/AOS1J/1z/wDaJpZtE8T3/izS9cvbOACN1DQwyr+5QHvk8nknjNaLaBqR+Jy6z5A+wBMeZvXr5RXpnPX2reviaMpOTmrulJfEnrfa/V/n0LqVYNt8yvyNb31OzrzT4efP4t1x5v8Aj4+br15kO79cV6XXB614U1ex8QPr3hqRPOlJMtuxAyT168EHrg45/TxstqU3Tq4eclFzSs3tdO9n2ucOElHlnTk7cy0Z2V6bFUiN+bcL5gEfn7cb8HGM98Z/WuN+IWp3Ym03RLGYw/bWAkdTjIJCgZHbk5qjceH/ABZ4tvbdddWKysoWyVjZefXABPPuTxW74u8JPq9nZPpjrBd2AxCCcArxgZ7EYGK6cLSw2ExFL2tRSet7axi+mvr9xrRhSo1Yc8k9/NLsc74i8HReFdJXWNJvrqO6tmXezMPmyQOMD1I454pnjnUX1TwhoN+wCySsWYDpu284/EVcvtO8beJ4ItN1K2tbK1DAyzKwO/HcgMc+uBgZq74x8K3l5oOl6do8HmraHBDOqnG3GeSOa9CjiYRq0PrNSMpqTd000o22b232OqFWKnT9rJOSb1v0t3MTxf4Wj0nQ4dZW8upNS81POmeT7xI6j0welHjq2W88K6Hrs0kjXckUUTDI2kMhcnGOufeuu8aaRe6x4ZFnYxCSfzEbaWC8DOeSaz/EHhrUNS8Babp0Eam9tFiLRFwMlUKkA9O/6VlhMwTVCdWa5lNp7aRa/BEUMSmqcpy1Tf3f5GTrC/8ACH+AoIdLnnWTUZFd5GYblygLBSAMdAPxNc5dPoFhplvdaNql4NZiKszFWAYn72OOP89a7WbQNa8R+D/sOq28Nne2zqbXDAhgq4+bBPXn9OKhUeP7i1ttOFvBZ+UVDXgkUlgOORk/oPyrpw+KhGLvNOXM3L3kk106PmVtLI1pVopayV7u+qV/wd16EniHSx4q8G2etpti1CG3E27O0EYyy5+uSP8A69YOhm78f69bf2pIhtdPhUvGDjzD64/2iOT6DHpXVeLtP8Qalp1rpFgvmxOF+13buibsf7Oenc4HoKyrjwZqXh/V7DUPDSef5aBLiN5Au8jqTk9G9OxrDCYmnHDODqRU3zcn91dm+l+l/wDgGVGrBUuXmSk78vl8+l+hmeNtCtR430+MNKF1F4xKAR8vzBPl4449c12Wp+C7XUPD1loyXU0UNo+5HIDMeCOeg71meM9D1jUdQ0nVtLtVkmtcM0LuoKsGDDPODzwcGtLUpfFknh6zksbeGLVWf/SIwUKqvPTcSP7veuepiatShhvZ1UnG61a0etm/K2lzOVWcqdLlmk157P8A4Y6OCIQW8UIORGgXPrgYqSo4PM+zxed/rdg3/wC9jmsrxMdcGmp/YAU3fmjdu242YOfvcdcV4FOm6tVQckrvd6L7zzox558t/wDI2aKx9SOtDwzmwCnV/Lj4O3G7I39eOm6iI61/wipMoX+2fIbA+XHmc4/2fSrWHvFS5lvbfX19PMfs9L3W9v8Ag+hsUVleHjq50lf7cCi93tnbtxt7fd4rVrKrD2c3C6duq2+RM48snG9woooqCQooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigDM/5nnTP+wbef8Ao22ro65z/medM/7Bt5/6Ntq6OsKm510vhCiiiszQKKKKACsrTf8AkedX/wCwbZf+jbqtWsrTf+R51f8A7Btl/wCjbqtqHxkT2IfBv/IjeH/+wbbf+ilrbrE8G/8AIjeH/wDsG23/AKKWtusnuUtgooopDCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACsHxf8A8gW3/wCwpp3/AKWQ1vVg+L/+QLb/APYU07/0shqo7oT2N6iiipGFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQBg+EP8AkC3H/YU1H/0smrerB8If8gW4/wCwpqP/AKWTVvU5biWwUUUUhhRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAZnhv/kKeJv8AsJp/6SW1dBXP+G/+Qp4m/wCwmn/pJbV0FejD4Uc8twoooqhBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABWD4w/wCQHD/2ErD/ANK4a3qwfGH/ACA4f+wlYf8ApXDSlsxrc0aKKK806AooooAKKKKAMDxH/wAhLw3/ANhJ/wD0luKv1Q8R/wDIS8N/9hJ//SW4q/XRT+E5a3xBRRRVmQUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAZn/M86Z/2Dbz/ANG21dHXOf8AM86Z/wBg28/9G21dHWFTc66XwhRRRWZoFFFFABWVpv8AyPOr/wDYNsv/AEbdVq1lab/yPOr/APYNsv8A0bdVtQ+MiexD4N/5Ebw//wBg22/9FLW3WJ4N/wCRG8P/APYNtv8A0UtbdZPcpbBRRRSGFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFeeeH9Gi1Dw3pd7c6hrT3FxZxSysNZu13MyAk4EmByegrR/4Ruz/wCf3W//AAd3n/x2tvY+ZShJnZUVxv8Awjdn/wA/ut/+Du8/+O0f8I3Z/wDP7rf/AIO7z/47R7HzH7OR2VFcb/wjdn/z+63/AODu8/8AjtH/AAjdn/z+63/4O7z/AOO0ex8w9nI7KuN+JV7qNj4dtJdOsheMdTs90RbacidHXn3ZVX/gee3J/wAI3Z/8/ut/+Du8/wDjtI3hixcYe71phkHB1q8PIOQf9b601Ss7idOR2KBgihyC+PmIGATTq43/AIRuz/5/db/8Hd5/8do/4Ruz/wCf3W//AAd3n/x2l7HzH7OR2VFcb/wjdn/z+63/AODu8/8AjtH/AAjdn/z+63/4O7z/AOO0ex8w9nI7KiuN/wCEbs/+f3W//B3ef/HaP+Ebs/8An91v/wAHd5/8do9j5h7OR2VFcb/wjdn/AM/ut/8Ag7vP/jtO0q0Gm+MbS3gu9Qkhn0+5eSO5v5rhSyyQBSBI7YIDt09aTpWV7icWldnYUUUVkSFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFAGD4Q/5Atx/wBhTUf/AEsmrerB8If8gW4/7Cmo/wDpZNW9TluJbBRRRSGFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQBmeG/8AkKeJv+wmn/pJbV0Fc/4b/wCQp4m/7Caf+kltXQV6MPhRzy3CiiiqEFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFYPjD/kBw/9hKw/9K4a3qwfGH/IDh/7CVh/6Vw0pbMa3NGiiivNOgKKKKACiiigDA8R/wDIS8N/9hJ//SW4q/VDxH/yEvDf/YSf/wBJbir9dFP4TlrfEFFFFWZBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQBmf8zzpn/YNvP/RttXR1zn/M86Z/2Dbz/wBG21dHWFTc66XwhRRRWZoFFFFABWVpv/I86v8A9g2y/wDRt1WrWVpv/I86v/2DbL/0bdVtQ+MiexD4N/5Ebw//ANg22/8ARS1t1ieDf+RG8P8A/YNtv/RS1t1k9ylsFFFFIYUUUUAFFFFABRRRQAUUUUAFFFFABRRRQBxvhP8A5E3Q/wDsH2//AKLWtisfwn/yJuh/9g+3/wDRa1sV2nRHYKKKKBhRRRQAUUUUAFFFFABRRRQAUUUUAFUYf+R507/sG3n/AKNtqvVRh/5HnTv+wbef+jbapn8LIqfCdRRRRXIYhRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQBzsfhe4tmmFl4l1e0glnluPIjS1ZUaR2kbBeFmxuY9Sak/4R/U/+hw1v/vzZf8AyPW9RVXYrGD/AMI/qf8A0OGt/wDfmy/+R6P+Ef1P/ocNb/782X/yPW9RSuFjmrVdR03xdZWE2t3uoW1zYXMzJdRwDa8ckAUgxxoekrZzntXS1g3f/I/6P/2C77/0baVvU2CCiiipGFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAZnhv/kKeJv+wmn/AKSW1dBXP+G/+Qp4m/7Caf8ApJbV0FejD4Uc8twoooqhBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABWD4w/5AcP8A2ErD/wBK4a3qwfGH/IDh/wCwlYf+lcNKWzGtzRooorzToCiiigAooooAwPEf/IS8N/8AYSf/ANJbir9UPEf/ACEvDf8A2En/APSW4q/XRT+E5a3xBRRRVmQUVxvi7xHq/hrUbSdIoZtKlYB/kO9SOoznuOR+PpWj4i8UW+keGxqdu6SvOoFqD0ckZB+gHJ/Ku1ZfXkqbirqe1u/Z9jdYao1FrXm2OhorF8L3WqX+ix3mrCJJZ/njSNCu1O2ck8nr+VbCSJJnY6tjrtOcVz1aTpTlB62002MpwcJOL6DqKKKyJCiikDqxIVgSvUA9KAForF8Rza9DawnQLWK4mL4kEhAAXHuw71xLeLvGia2ujtZWQv25EWB/d3dd+OnvXo4XLKmJhzwlHq7OVmkurXY6qOElVjzRa+89QorC8Nz+IZork+ILWG3ZSvleWVORznOGPtW0kscufLkV8ddpziuOtRdKbhdO3VO6+8wnBwk43v6bD6KRmVFLMwUDuTikR1dQyMGU9wcisrdSB1FclY+JL658f3miOIfskKFlIU7uAp5Ofc11tb4jDToOKn1SfyZpUpSptKXVX+8KKKKwMwoorjvB3ii+1y91SO+8hY7UjaUXbxls5yfauilhqlWlOrHaFr/N2NIUpThKa2X6nY0VwbeMdU1vxJ/Z3huKBraPiS5mQsMd24I49PWu6XKRjzGBIHzNjGfeqxODqYblVXRtXt1Xr2Kq0JUrc+76dR1FNR1kXcjBh6g5pWZUUs7BVHUk4FctuhiLRSAhgCpBB6EUpIAyTgCgAopFdXUMjBlPcHIpaACimu6RrudlUerHFOBBAIOQe9FgCikZlRSzEADqSaUEEZByDQAUVwvirxTrem+J7fSdKit5GniQosi8liSMZyB2FVLzxD470i2a8v8ASbQWyY3kYbGT/suSPrXq08orThCXNFc2yckm/kdkcFUkk7rXbXU9ForN0HWI9d0a31CJCnmAhkJztYHBFaAkQuUDqXHVQeRXm1Kc6c3CSs1ozllFxk4vdDqKKbvTfs3LvIztzzUEjqKaZEDhC67yMhc8mnUAFFNWRHYqrqzDqAckU6i1gCiigkAEk4A70AFFNR1kXcjBh6g5p1ABRRRQAUUUUAFFcN428SeIdB8ReH4dOtoH0y+uo7a4lkTJDu4AUcjnbk9D0osfEniF/itd+Hr22gj0sWzXFs4T53UFRnOem4kdKdh2O5ooopCCiiigDM/5nnTP+wbef+jbaujrnP8AmedM/wCwbef+jbaujrCpuddL4QooorM0CiiigArK03/kedX/AOwbZf8Ao26rVrK03/kedX/7Btl/6Nuq2ofGRPYh8G/8iN4f/wCwbbf+ilrbrE8G/wDIjeH/APsG23/opa26ye5S2CiiikMKKKKACiiigAooooAKKKKACiiigAooooA43wn/AMibof8A2D7f/wBFrXnnxj8Q67o2o6JBot/PbNcrIGWIj52yoH869D8J/wDIm6H/ANg+3/8ARa15v8Xv+R08Gf8AXx/7UjruW5pP4DoPAPjGS/8AhjNrGozme509ZhcOx5YoN4/8dKivPPh94z8U6n4+0i11HVrma0ujIxjcja4CP7eq/pWTqupv4Ri8a+E1yq3V0nkAfwpuLH80Kit/SNK/sT4weE9OK7Xh05BIP9swyFv/AB4mqsZczbS7HSeOfG2u3fi2Lwb4RIjvTgT3GBlSRuIBPAAXknr2HTnN1TQPiX4U06TWofFDaiLdfMngZ2cBRySFcYIHOehx0qHwzImn/tCaxFekJJcGZYd3cttdfzUV67r91b2Xh7Ubm6ZVgjtpGct0xtPH49KWxaXNdtnGweOZPEXwm1TXLQ/ZNQtoJEkEZ/1cqgHK57EEEfXHauP8K2XxF8W6Cuq2Xi/ykaRoxHMxByvuFNUfAUMq/CHxpMwPkuhVD2yE5/mK734Kf8k7i/6+Zf5im9BRvNq/YzvA/jXX7bxhN4O8XFZL0Z8i4AAJIXdgkAAgryD17Hrxka9qfizVfi9feGtH1+Wxi+Uxgn5ExCrnoM8nP50a+Rd/tE6VHandJD5Ql29sKWOf+A1la9o93r3x61DT7HU5dNuJApW5izuXFupI4IPOMdaBNu1vM63/AIQr4lf9Dwn/AI9/8TS/FbxJrOha34dg0y/kto7lmEyoB8+GQc5HuadY/DDxLa6hbXEvj7UJo4pVdom8zDgEEqf3nfpWR8bf+Rj8K/77/wDocdJblSuovoe0UUUVJuFUYf8AkedO/wCwbef+jbar1UYf+R507/sG3n/o22qZ/CyKnwnUUUUVyGIUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQBg3f8AyP8Ao/8A2C77/wBG2lb1YN3/AMj/AKP/ANgu+/8ARtpW9TfQQUUUUhhRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFAGZ4b/5Cnib/sJp/wCkltXQVyuianYWeseJo7q9toHOpIwWWVVJH2S35wTW4utaU5wmp2TfSdT/AFr0YfCjnluXqKp/2tpv/QQtP+/y/wCNH9rab/0ELT/v8v8AjVCLlFVxqFkQCLu3IPQ+YP8AGj7fZ/8AP3B/38FAFiioFvLVzhbmFj6CQGn+fD/z1j/76FAElFR+fD/z1j/76FPVgwypBB7g0ALRRRQAUUUUAFFFFABRRRQAUUUUAFFeX/GjxnrXgrT9GvdGnSN5bl0lSSMOsihc4OefyINc/wCGf2jNMutkPiTTZLKQ8G4tcyR/Ur95R9N1AHuFYPjD/kBw/wDYSsP/AErhq3oviPRvEdt9o0fU7a9jxk+TICV/3l6r+IFVPGH/ACA4f+wlYf8ApXDSlsxrc0aKKK806AooooAKKKKAMDxN8txoMvZNSHP+9DKn/s1X6peMBs8OyXXazngumPokcqu//joartdFP4TmrL3goooqzEo6xpdvrWlT2FyPklXAbHKnsR9DXlHhjRbjWPEiaRqE/mWmktITFnKnD4IHsW6+1ey1xfhjw9qWmeL9Y1C6hVLa5MhiYOCTmTcOB04r28sxroYatHms7Xj67O3nY78JXdOlUV7dvXbT5FHxzd3V/wCIdM8M287W8FwFaZl43bmIwfUAAnHvWf4m8NJ4Lt7XWNEvLiOVJRG4dgd2QTngDjjke9dJ4w8L3mq3dpq2lSrHqNpjaGOAwByMH1Bz14OayLzRvF/i2S3tdaht7Cxife5jYEuemcBm5xn0HNehgsTTjSo8tSMYJPni92/TrfodNCrFQhaSUVfmXf5dSr43u/t8/hS827fPUS7fTcYzj9a0fit/yArL/r5/9lNWvGnhW71Ky01tIVfN0/KpEWAyvy4wTxkbR1rK1/RfGHifToDeWtpC0L/LbRuMtkHLkkkegAz3NGDq0JfVqnOoqDldN6q7dtP1ChOm/ZS5klG97vu9CH4jmRbfw0YjtkCuUI7HEeK63w54RtPDtxNcx3E89xOgWR5SDznJI47nHXPSsrxh4c1PV10MWcKv9kDedlwMZ2evX7prt68/FYxrAUqNOW/NzJf4rq5zVq9sPCnF97/foFebXH/JaoP90f8Aok16TXFzeHtSf4mRa0IV+wqMF94z/qivTr1rDKqsKbrc7teEkvXsZ4OcYufM7Xiyfx3b2dzY2y6hrZ0+0D5eJULGfpxgHPH0I5rgpJdN07xHpNx4dF/DG0qqzzjCyfMPu+owefwrtfG3h3U9S1HTtU0yOO4ktCM28hABw24HnAI7EZrM1nRPFmu3un6jc2VrGbaQbbWOUZUZBJJJwc47HtXs5ZWpU8PCMqis1K6bSSbvpa13fu3Y7sJUhGnFOWjvdXS/Dr6srePfNj8UwTatbXNxoqxjYkTbVzjnnpnP6Yq14LOgf2+0mj6ndweapH9n3C/f4/vZIOOvr+FbevWvimHWVvtIlS8syuHsZWCgcYPXGR369aytK8L6re+LINa1HT7TS4oMEQW5GXIzgnaSO/J9ulTHEUpYDknNK0baSWr7ODW/mvvEqsHh+WUktOj/AAcX180c9rGty6D8QNYu7dA87IY489FJVeffGK6n4d6VaPYNrsk32rULh2EkjcmM55H1PUn3/OS08M3g+Il5q1zbRtYyIwjZmVuSoHTr60vhzQNU8N+JbyKCMSaJcHcp8wZjPUcZzx9334NLF4qhUwnsqUkp8kbu/wASS1j6re3UK1anKjyQdnyr5+Rq6b4Wi03xNe60t07vdKymIqAFywPX/gNb9YGmjxGPE18b9kOkbW+zAbM53DHTnpnrW/Xz+LlUc17Sak7Lbt29UebWcnJczvotgrwnS4NX1G/1HSdKBAu3/ft0ARSep7Dn8ele7Vxfgfw9qWiahqs19CsaXBUxkOGzgse31FellOMjhaFebs5e7ZPq7/jbc6sHXVGnUel9LX9f0KPwz1GCGK60WaBYL+Jy7ZGGkA4IPup/T8aqePdUFx4ntNHu7mWDS0VXuPKBJYnJ6DrwAB6ZrQ8T+FNT/wCElt9d8Pon2jO6VS4Ubh356gjgj/Gp/Efh7VrvUdP8Q6VHHHqUCKJbaRxgkdgeh6kHkcV3U62FljI4zmXvp6N/DO3zsuzsdEZ0XXVe/wASfXZ/1szl7K/0/RfFunt4cubp7K4dY7mKVWA5OM8gZ65/CpNe1K01fxxc2ut3U8Wl2eUSKIE5YYHYHqcnPoMV1Omx+MtR1yG61Ex6bYxAbreJlfzcfievrxx0qtqvh/WtL8VSa/4fjiuPPXE1vIwXrjPUjg4B69a1jiqH1j32ufkaUuZPW/WXLZO3XUtVqftNWubl3v19bWv5mR4Q1GGw8aHTdKuLibSLpTtWZSNjBS3cexHvmq+k6KuveOdfs57iaK0E8rypE2DJiQgA+3OfwrsPD8HimbVZr/W5ktrZhhLKMqwzjGc84H48muM0pdYHjvXrjRfJe4immLQzHCyqZOmcjnOD1HSqhW551pU5JSUI63ur33bslfz263HGfNKbi0nyrW99b97LXzL+mQP4R+IsWkWs8j2F4oJjc5xkHB+oI6+lem1wuheHdavfFJ8Q+IEihljGIoEYHBxgdCcAAnvnNd1XiZvVhUqQtJSkopSa2b9evqefjZqU463dldrueV6Fp3/CwdX1DUNVuZ/s8LBYYY2xtBzgewAH4k1c8PtceGPHr+HEuZJ7CdSY1c52HbuB9jwQcdetSr4e8SeFdXurjw7FBd2dycmGRgNvJxnJHTJwQav+G/DOp/2/N4i19o/trgiOFCCEyMduOnAAzXr4nE0nCo/aRdJxtGPVSsrabqzvdnbVqwcZe8uRrRdb+hymkaCviLxjrtlPczQ2izySSCIgF2EhC9QR3Neq6bYRaXplvYwEmOBAgJ6n3P1rl/C3h/UdL8Ua1fXUSpBdO5iYODkFyRwOnFdlXl5zjHWqKnGV4JLba9tfmcmOruclFO8Ul+R5b42a7T4i6c1givdiKPyVfoW3NjPIqtrWu+Jry4/4R/WGtNOFxgM7KQpXPHzAtxkdvxrp9b8O6le+PtN1WCFWs4BGJHLgEYZieOvcVo+MvDS+I9IKxBRewZaBjxn1Un0P88V6VLH4aH1eFRRfu77uLvp9x1QxNKPsoySem/VMxvECyeCvh/DZWM7ec8ghM44OW3MxHp0IHpXGzR6Bb6HDdaff3w11AshbYwBY43AHHGMnnPb3ruE8O6trPgk6Nq6rDeW7A28xcOGx0zj2JX8jzVOKLx//AGdb6RHb29usO1BeiVc7R07n+Wa1wmIhTi06ic+duT5klJd9ndeSLo1YxTXMr813qlf8HdeRmeM9Rl1TwRoN7NxNI538YywBBP5io/GHh1NC0qw1iG7un1GSZRLM8mSzFS2R6YIrpPGfhzVtY0PTrW3YXdzA2ZZHKpuO3GccDrVjxxoV/regWlpYxLJNHMrsC4XACsOp9yKzwuPp0/YRjJRjzTur7JvS/l62JpYiMfZpNJXlf0/yOS8U+HU03w3Y66Lu5k1OaRGmmaTqWUtx6YIGK1PGOuX76HoVjbzNHNqUSNNIpwTkLxn0Jbn6Vs+LNCv9V8I2en2kSvcxtGWUuABhSDyfc1Dr3hC41jw3pUUTrDqVhCiruPBO0Bhke4GD7e9TRxtGfsZ4iSbUpfJW93TtcmGIpy9m6rvZv5dvkYniHwTB4a0P+19MvrpLy0KlnLAbskDjA45P5U3xM1xqvhTRfFcQ23luQJWUf7WA303D/wAeq1e6f448R2selajBa2truHmzhlzJjuQGOfXAA5rsl0K1Xw3/AGIATb+QYcnr0+99c81M8f7BUpVpqdRSd2nf3GrNN+fRBLEezUHUkpST6a+72OJ8YaqfEq6DpNi3N/tuHA52g8DP0+bP0qv41u4Y9b07w9JPLbaPbxJ5oiBJI+g68AY+pNaPgjwbqGk6xJe6qijyYzHbgOG5JOSMdOM/99Vf8V+G9RudYs9e0Qxm+tgA0TnAcDOOvHcg9OK0hiMJQxMMPCScIqVnfTmls767LS/QqNWjTqxpxfupOz82cnZ3+naJ4r09/DlzdPZTusdzFKrAcnHcDPBz7EV69XHadH4y1HXIbrUTHpljEBvgiZX83Hbqevrxx0rsa8rN6sak4apyS1alzfe0krr5nHjZqTj1dtdb/e7IKKwJh4j/AOEuhMTJ/Ye394Pkznaf+BdcUauPER17TzpjINN3L9rB2Zxu+brz930riWFvKMeeOqvvt5Pz8jBUbtLmWqvv+Hqb9FYHiceIiLT/AIR9kB3N5+7Z04x978elb9ZTpctOM+ZO99Oqt37X6EShaKlda/1qZt5oOn3+sWWqXUTS3FkD9nDOdiE/xbem73otNB0+y1i91aKJjfXmBLNI5YhR0Vc/dXvgd60qKyICiiigAooooAzI/n8cWv8A0y02fPtuki/+Iro657SB5/izWLgcpDBb2oPo43yN+kkf6V0NYVPiOukvdQUUUVmaBRRRQAVlab/yPOr/APYNsv8A0bdVq1lab/yPOr/9g2y/9G3VbUPjInsQ+Df+RG8P/wDYNtv/AEUtbdYng3/kRvD/AP2Dbb/0UtbdZPcpbBRRRSGFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAcb4T/AORN0P8A7B9v/wCi1rB8a+B7nxTruh6hBeRQJpsm90dSS/zK3GP92t7wn/yJuh/9g+3/APRa1sV23sb2TjZnmviv4WHxH46t9eW8hjtswm5hZCWk2HnB6cqAK1L3wPc3XxQsvFi3kSwW8ewwFTuPyMvXp/FXbUU7sXJE4Xx18NbXxdcRala3bafq0IAWdRkOByN2CDkdmHP14xzE3wu8aa2qWev+MBNp6sCURnkLY9QQAT7kmvYaKLsHTi3c5mfwdaQeA7nwvpWLeKS3aJZJOSWbqzY6kmuE0/4WeM9JshZ6f4y+y2wYt5cO9Rk9TxXsNFF2DhFnD+CfhvbeFLyfU7q+k1LVpwQ1xIMbQeTjJJJPck/lzWL4h+GWuah44u/Emk69FYSzbdhCtvQCMIeR64P516lRRdh7ONrHlf8AwgfxD/6Hx/8Ax+tHx78PdQ8Xto8lvqcVvNYRsrPIpJdjt5GPda9DoouHs1ax5X/wgfxD/wCh8f8A8fr0jSba6s9Hs7a9uftN1FCqSz/89GA5b8TVyik3ccYpbBVGH/kedO/7Bt5/6NtqvVRh/wCR507/ALBt5/6NtqmfwsVT4TqKKKK5DEKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAwbv/kf9H/7Bd9/6NtK3qwbv/kf9H/7Bd9/6NtK3qb6CCiiikMKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooA8m1r4RWHj3xl4h1W61S5tXiuorcJFGrAgW0LZ5/3v0qo37NWkkfJ4hvQfeBD/AFr1Hw3/AMhTxN/2E0/9JLaugr0YfCjnlueGf8M06b/0Ml3/AOAy/wCNH/DNOm/9DJd/+Ay/417nRVCPCD+zRZZOPE9wB72in/2aj/hmiz/6Gef/AMAx/wDF17vRQB4O37NFrj5fFEwPqbIH/wBnpn/DM0P/AENcn/gAP/jle90UAeCf8MzQ/wDQ1yf+AA/+OUxv2ZVJ+XxaQPfTs/8AtWvfqKAPn/8A4Zl/6m7/AMpv/wBtpf8Ahmdl5Xxfhh0/4l2P/ate/wBFAHz/AP8ADN15/wBDh/5JH/45R/wzdef9Dh/5JH/45X0BRQB4B/wznqcYxF4ywO/+isv/ALUpP+Gd9Y/6HL/yXf8A+Lr6AooA8A/4Z31gHI8Zf+QH/wDi6P8AhQHiT/odP/HZP/iq9/ooA8A/4UB4k/6HT/x2T/4qj/hQnimMfuvGuM9f9av/ALNXv9FAHyR8Svh5rfgvTrG41XXv7SS4lZETc52EDOfmNeb19mfEL4fQ/EG2061udQktILSZpX8uMMz5GMAk4H1wai8P/CLwX4d2PDpCXdwv/Le+PnNn1wflB+gFAHy74V8MeLtXvI7nw1YagZVPy3UBMSqf+umQB+dfQNhpnxB0/wAOW6+LdYsLqH+0LDbEsZedf9LhwDINo/MNn1r1hVVFCqoVQMAAYAFYXjD/AJAcP/YSsP8A0rhpS2Y1uaNFFFeadAUUUUAFFFFAEN3axXtlPaTruhnjaNx6qwwf0NYPh24luNDtluDm6gBtrj/rrGSjn6EqSPYiukrmnX+yvFcsZ4ttWHmxnstwigOv/AkCsP8Acc1rSetjGtG6uatFFFbHMFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFYumeGbLStZvtUgluGnvCzSK7AqNzbjjAB6+9bVFaQrThGUYuylv5lRnKKaT3CiiisyQooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKRmVELswVVGSSeAKWsfX917HBokRIk1JjHIR1SAcyt7fL8oP8AedaBpXdi34SjZtE+3upWTUZnvDnrsc/uwfcRhB+FbtIqqihVACgYAA4Apa5W7u52pWVgooopDCiiigArK03/AJHnV/8AsG2X/o26rVrK03/kedX/AOwbZf8Ao26rah8ZE9iHwb/yI3h//sG23/opa26xPBv/ACI3h/8A7Btt/wCilrbrJ7lLYKKKKQwooooAKKKKACiiigAooooAKKKKACiiigDjfCf/ACJuh/8AYPt//Ra1sVhaVpnizS9HstOGnaLKLW3jgEh1SVd+1QucfZzjOOmTVzyvFv8A0CdE/wDBtL/8jV188e5qpxsaNFZ3leLf+gTon/g2l/8AkajyvFv/AECdE/8ABtL/API1HPHuP2kTRorO8rxb/wBAnRP/AAbS/wDyNR5Xi3/oE6J/4Npf/kajnj3D2kTRorO8rxb/ANAnRP8AwbS//I1U9SvPFGl2qXE+jaOyPcQ24CarKTullWNTzbjjLjPtnr0o5o9w9pE3aKzvK8W/9AnRP/BtL/8AI1HleLf+gTon/g2l/wDkajnj3D2kTRorO8rxb/0CdE/8G0v/AMjUeV4t/wCgTon/AINpf/kajnj3D2kTRorO8rxb/wBAnRP/AAbS/wDyNR5Xi3/oE6J/4Npf/kajnj3D2kTRqjD/AMjzp3/YNvP/AEbbUzyvFv8A0CdE/wDBtL/8jVLpem623iOHUdTttPt4oLSaBVtrt52ZpHibJ3RJgARn161MpKz1JlJNWR01FFFcxmFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAYN3/yP+j/9gu+/9G2lb1YN3/yP+j/9gu+/9G2lb1N9BBRRRSGFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAZnhv/AJCnib/sJp/6SW1dBXP+G/8AkKeJv+wmn/pJbV0FejD4Uc8twoooqhBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABWD4w/5AcP/YSsP/SuGt6sHxh/yA4f+wlYf+lcNKWzGtzRooorzToCiiigAooooAKztb0v+1tNaCOTyblGEttNjPlSryre47EdwSO9aNFNOwPUwdJ1EalYiVo/JuI2MVxATkxSr95T/MHuCD0NXqztZsp7C+OuafC0rbQl9bRj5p4x0dR3kXt/eGR1C4t2t1Be2sV1bSrLBKoZHU5DA10xldXOOcOVk1FFFMgKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigBksscELzTOscUalndjgKByST6VS8OW8l3LNr10jJJdqEtY3GDFbg5XI7Mx+Y/VQfu1UVP8AhKL7y1GdEtZP3r9ruVT9wesakfMe5G3oGz1dZVJdEdFKH2mFFFFYm4UUUUAFFFFABWVpv/I86v8A9g2y/wDRt1WrWVpv/I86v/2DbL/0bdVtQ+MiexD4N/5Ebw//ANg22/8ARS1t1ieDf+RG8P8A/YNtv/RS1t1k9ylsFFFFIYUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUVk6rrh02/tbGHTL3ULm5ilmWO1MQ2pGUDEmR0HWRehJppXA1qKwf+Eg1P/oT9b/7/WX/AMkUf8JBqf8A0J+t/wDf6y/+SKLCub1FYP8AwkGp/wDQn63/AN/rL/5Io/4SDU/+hP1v/v8AWX/yRRYLm9WD4v8A+QLb/wDYU07/ANLIaP8AhINT/wChP1v/AL/WX/yRVDVrrVtat7WyTwxqlt/p9pM0081rsRI7iORidszN91D0B5ppagzraKKKkYUUUUAFFFFABRRRQAUUUUAFFFFABTZJEhjaSV1SNBlmY4AHqTUd3dwWFnNd3UqxQQoXkduiqBkmuXS0m8QSLf6zEy2+d1tpr/djHZpR0aTvg8L0HILHalRlUdkROooLUvnxdYzf8g62vtTHZ7SH92fpI5VG/BjUZ8UXoJH/AAietH3Eln/8fq/RXasHT6nM8RIof8JRe/8AQpa3/wB/bP8A+SKP+Eovf+hS1v8A7+2f/wAkVfop/VKYe3mUP+Eovf8AoUtb/wC/tn/8kUf8JRe/9Clrf/f2z/8Akir9FH1SmHt5lD/hKL3/AKFLW/8Av7Z//JFH/CUXv/Qpa3/39s//AJIq/RR9Uph7eZQ/4Si9/wChS1v/AL+2f/yRR/wlF7/0KWt/9/bP/wCSKv0UfVKYe3mUP+Eovf8AoUtb/wC/tn/8kUf8JRe/9Clrf/f2z/8Akir9FH1SmHt5lD/hKL3/AKFLW/8Av7Z//JFH/CUXv/Qpa3/39s//AJIq/RR9Uph7eZyV09zdeNtP8SHwxrwe0tJbfyvOtMEsQQf+PjsN+fXK+lb3/CUXv/Qpa3/39s//AJIq/RR9UpsXt5FD/hKL3/oUtb/7+2f/AMkUf8JRe/8AQpa3/wB/bP8A+SKv0UfVKY/bzKH/AAlF7/0KWt/9/bP/AOSKP+Eovf8AoUtb/wC/tn/8kVfoo+qUw9vMof8ACUXv/Qpa3/39s/8A5Io/4Si9/wChS1v/AL+2f/yRV+ij6pTD28yh/wAJRe/9Clrf/f2z/wDkij/hKL3/AKFLW/8Av7Z//JFX6KPqlMPbzKH/AAlF7/0KWt/9/bP/AOSKP+Eovf8AoUtb/wC/tn/8kVfoo+qUw9vMof8ACUXv/Qpa3/39s/8A5IoHii9JA/4RPWh7mSz/APj9X6KPqlMPbzKi+LrCIgalb3umf7d5DiMfWRSyD8WFbqOkkayRsrowyrKcgj1FZpAIweRWKbK40GRrzQ4yYM7rjTFOEkHdoh0R++BhW74J3DKpg7K8GXCv0kddRVewvrfUrGG9tJBJBMu5Gxj8COxHQg8g1YrhOkKKKKQBRRRQBmeG/wDkKeJv+wmn/pJbV0Fc/wCG/wDkKeJv+wmn/pJbV0FejD4Uc8twoooqhBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABWD4w/5AcP/AGErD/0rhrerB8Yf8gOH/sJWH/pXDSlsxrc0aKKK806AooooAKKKKACiiigArm77S7rSLqXUdHiM0ErF7vT1IG4nrJFngP3K9G68Hk9JRVRk07oUoqSszGsNQtdTtRcWkokjyVPBDKw6qwPKsO4PIqzVbUvD6XN01/YXDWGokAGeNdyygdBKnRx78MOxFZ7a1Npn7vX7Q2YHH2yMl7Zvcv1j/wCBgDsCa3jJSOWdNxNmimo6SorxsrowyrKcginVRmFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUVlT69bi5ezsIpdRvUOGgtQG8s/wC25IVPoxB9AaASvsabukUbSSMqIoJZmOAAO5NYyfaPFJ2WzS22ifx3K5SS7H92Puqer9T/AA/3qsQ+H7jUnWfxBLHKoIZNPhJ8hD23k4MpHuAv+zkZro6ylU6I6IUusiOCCG1t47e3iSKGJQiRoMKqjgADsKkoorE3CiiigAooooAKKKKACsrTf+R51f8A7Btl/wCjbqtWsrTf+R51f/sG2X/o26rah8ZE9iHwb/yI3h//ALBtt/6KWtusTwb/AMiN4f8A+wbbf+ilrbrJ7lLYKKKKQwopHdY0LuwVVGSScACsWXxl4WgcpL4l0eNh1V76IH/0Kmk2Bt0Vg/8ACceEv+hp0T/wYRf/ABVH/CceEv8AoadE/wDBhF/8VRyvsK6N6isH/hOPCX/Q06J/4MIv/iqP+E48Jf8AQ06J/wCDCL/4qjlfYLo3qKwf+E48Jf8AQ06J/wCDCL/4qj/hOPCX/Q06J/4MIv8A4qjlfYLo3qKwf+E48Jf9DTon/gwi/wDiqP8AhOPCX/Q06J/4MIv/AIqjlfYLo3qKwf8AhOPCX/Q06J/4MIv/AIqj/hOPCX/Q06J/4MIv/iqOV9gujerBu/8Akf8AR/8AsF33/o20o/4Tjwl/0NOif+DCL/4quR1DXdIn+Juk6rD4z0hdJisZ0niGoQ43lk4zuyN2EOB/zyPqc1GLBs9MorB/4Tjwl/0NOif+DCL/AOKo/wCE48Jf9DTon/gwi/8AiqnlfYLo3qKwf+E48Jf9DTon/gwi/wDiqP8AhOPCX/Q06J/4MIv/AIqjlfYLo3qKwf8AhOPCX/Q06J/4MIv/AIqj/hOPCX/Q06J/4MIv/iqOV9gujeorB/4Tjwl/0NOif+DCL/4qj/hOPCX/AENOif8Agwi/+Ko5X2C6N6isH/hOPCX/AENOif8Agwi/+Ko/4Tjwl/0NOif+DCL/AOKo5X2C6N6isH/hOPCX/Q06J/4MIv8A4qlTxr4UkYKnifRWY9AL+In/ANCo5X2C6N2io4LiG6hWa3mjliYZV42DKfoRUlIYUUUUAFFFFAHOeID9u1jTNJPMOWvbgdmEZUIp+rsG/wC2dX6zn+bxzf5/g0212+2ZbjP/AKCPyrRr1sNFKmjhrO8woooroMgooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKAM3Sj/Znia4sBxa6hG13EvZJVIEoH+9uRseu8966auX1D5fEHhxhwWvZIyf9k20xx+aj8q6ivKxUUqmh3UXeAUUUVzGoUUUUAZnhv/kKeJv+wmn/AKSW1dBXP+G/+Qp4m/7Caf8ApJbV0FejD4Uc8twoooqhBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABWD4w/5AcP8A2ErD/wBK4a3qwfGH/IDh/wCwlYf+lcNKWzGtzRooorzToCiiigAooooAKKKKACiiigAoIBBBGQaKKAMKXwnpwkaXT2n0qZjkvYOEUn1MZBjY+5Umojp/iS1/1N7p9+nZbiJoH/F03A/ggroqKtTkiHCL3RzRutfi4m8PiQ+tpeo4/wDHwh/Smtquqqcf8Inq7e6zWmP1nFdPRVe1ZPsYnL/2vq3/AEKGs/8Af6z/APkij+19W/6FDWf+/wBZ/wDyRXUUUe1fYPYxOX/tfVv+hQ1n/v8AWf8A8kUf2vq3/Qoaz/3+s/8A5IrqKKPavsHsYnL/ANr6t/0KGs/9/rP/AOSKP7X1b/oUNZ/7/Wf/AMkV1FFHtX2D2MTl/wC19W/6FDWf+/1n/wDJFH9r6t/0KGs/9/rP/wCSK6iij2r7B7GJy/8Aa+rf9ChrP/f6z/8Akij+19W/6FDWf+/1n/8AJFdRRR7V9g9jE5f+19W/6FDWf+/1n/8AJFH9r6t/0KGs/wDf6z/+SK6iij2r7B7GJy/9r6t/0KGs/wDf6z/+SKP7X1b/AKFDWf8Av9Z//JFdRRR7V9g9jE5f+19W/wChQ1n/AL/Wf/yRR/a+rf8AQoaz/wB/rP8A+SK6iij2r7B7GJy/9r6t/wBChrP/AH+s/wD5Io/tfVv+hQ1n/v8AWf8A8kV1FFHtX2D2MTl/7X1b/oUNZ/7/AFn/APJFH9r6t/0KGs/9/rP/AOSK6iij2r7B7GJx9x4kvrWe1hm8Kaysl3KYYR5lod7hGcjifj5UY8+nrirH9r6t/wBChrP/AH+s/wD5Iq14g/5DXhX/ALCj/wDpHc1vU3UYvYxOX/tfVv8AoUNZ/wC/1n/8kUf2vq3/AEKGs/8Af6z/APkiuoope1fYfsYnL/2vq3/Qoaz/AN/rP/5Io/tfVv8AoUNZ/wC/1n/8kV1FFHtX2D2MTl/7X1b/AKFDWf8Av9Z//JFH9r6t/wBChrP/AH+s/wD5IrqKKPavsHsYnL/2vq3/AEKGs/8Af6z/APkij+19W/6FDWf+/wBZ/wDyRXUUUe1fYPYxOX/tfVv+hQ1n/v8AWf8A8kUf2vq3/Qoaz/3+s/8A5IrqKKPavsHsYnL/ANr6t/0KGs/9/rP/AOSKP7X1b/oUNZ/7/Wf/AMkV1FFHtX2D2MTl/wC19W/6FDWf+/1n/wDJFH9r6t/0KGs/9/rP/wCSK6iij2r7B7GJy/8Aa+rf9ChrP/f6z/8Akij+19W/6FDWf+/1n/8AJFdRRR7V9g9jE5f+19W/6FDWf+/1n/8AJFH9r6t/0KGs/wDf6z/+SK6iij2r7B7GJy/9r6t/0KGs/wDf6z/+SKBq2qkgf8IjrI9zNZ//AB+uooo9qw9jE5r7drT8R+G7lCehnuoFH47XanC38T3PBXSbAf3t8l0fyxGP1P410dFL2jBUonPr4WS551bUr3UR/wA8WcRQ/wDfEYG4ezlq2rW0trK3S3tLeK3gQYWOJAir9AOBU1FS5N7miSWwUUUVIwooooAKKKKACiiigAooooAKytN/5HnV/wDsG2X/AKNuq1aytN/5HnV/+wbZf+jbqtqHxkT2IfBv/IjeH/8AsG23/opa26xPBv8AyI3h/wD7Btt/6KWtusnuUtgqrqN/BpenT31ySIYULMFGWPoAO5JwAO5Iq1XP+KP3kmiWrf6ufUl3j18uKSVf/Ho1P4VUI80lEUnZNlGPSZNXZb3xCizyk7o7Fjugth2G3o7ju5zz0wOK2ERI0CIqqo6BRgCnUV7MYqKsjz3Jt3YUUUVQgooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKQgEYIyD2NLRQBkT6FFFK15pBXTtQ6+ZEuI5T6SoMBx7nkdiK2tF1Qatp4maLybiN2huISc+XIpwwz3HcHuCD3plUdF/deK9agXhHt7W5I/wBtvNjJ/wC+Yk/KuPF01y863OihN35ToqKKK806wooooA5pv+R51L/sG2f/AKNua0azm/5HnUv+wbZ/+jbmtGvYw/8ACRwVfjYUUUzzovN8rzE8z+7uGfyrYzH0UUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFAGXqP/Id8Nf9hF//AEkuK6iuX1H/AJDvhr/sIv8A+klxXUV5mM/ifI7KHwBRRRXIbhRRRQBmeG/+Qp4m/wCwmn/pJbV0Fc/4b/5Cnib/ALCaf+kltXQV6MPhRzy3CiiiqEFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFYPjD/AJAcP/YSsP8A0rhrerB8Yf8AIDh/7CVh/wClcNKWzGtzRooorzToCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAwfEH/ACGvCv8A2FH/APSO5reri/FniPT7HxV4X0+ZpRd/2j5iRCMnzFe3njBU9D87qDzxnJwOa7Sqa0QkFFFFSMKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACsrTf+R51f/sG2X/o26rVrK03/kedX/7Btl/6Nuq2ofGRPYh8G/8AIjeH/wDsG23/AKKWtusTwb/yI3h//sG23/opa26ye5S2Cuf8S/8AIQ8N/wDYSb/0luK6Cuf8S/8AIQ8N/wDYSb/0luK0o/xIk1PgZcooor2TzwooooAKKKKACiiigAooooAKKKKACimNNEjhGkRXbopYAmn0AFFFFABRRTXdI1LOyqo6ljgUAOopqOkihkZWU91ORTqACiiigAooooAKKKKACqGlf8jpq/8A2DrP/wBGXNX6oaV/yOmr/wDYOs//AEZc1z4r+Ezah8Z0dFFFeSdoUUUUAc03/I86l/2DbP8A9G3NaNZzf8jzqX/YNs//AEbc1o17GH/hI4Kvxs8v+MHi3UtKh0/QNFlaK/1NsNKhwypkKAp7Fievt71lTfAa3XR2lj1q7fWwm8SHHlNJ1xjG4c98++O1QfGiOTTPGXhnX3RmtYWVWIHQxyb8fiD+hr1m58Q6VbaA+ttewtp6xeaJlcEMMZAHqT0x1zVWTbuF2krGH8PIfE9j4Z+z+LABcQt+6keZXcx4/jIJGRzznpWRqHxt8I2F+1qhvbsKdpmtoVMefYswJ+oFVr/xufGfwl8RahptjdWjwwmJ95BBHBfaR1AUnPA61J8HbfR3+Gany7ZmkeUX+8A5O44D57bNvWi/RCt1Z2Fp4r0jUPDM/iCyuftFhBE8shQfMuxdzKVOMNjsfauUm+NnhOKxiuUF/Mz5zDFCC8YBxlssAM/XNcH8PiRoXxHSzLHSxZTeV6fdl24/4D/St/4V2dufg/rshhQvOblZGI5YCIAA+3X8zRzNj5Ujo7z4y+EbWwt7qOa6u2mUuYLeIGSIA4O8EgLz7+/TFb+n+NNG1TwnceI7OWSSyto3eZdmJEKDcykeuPfuK88+Cllbv8PtemaFGkmnkhkJH3kESkA+3zN+dZHw4J/4U541Hby5v/RFCkwcUdrP8bPCUNlBcL9umaXJMMUKl4wDjLZYAZx65rrdD8U6R4h0Q6vYXam0QHzWk+UxEDJDA9MDmuC+B0WmP4DvPkgaZrl1u94BJXaMBs/w4z7dfeuX+Geo2OkWXjq7mg+06HCqnySAyygtIFXB45BxQpPS4OK1sdpP8cfCkVw6RwapPEjbTcRW6+X+rA/pXUXXjbRbbwf/AMJSssk+mYU7oky3LBMYOOQTg15Y174l1r4aale6bYaDofhhoZn8hFLSygEggdgSRgHAPTHaq1tn/hmi95/5eh/6PSlzMOVHomm/Fjw5q+u2OkWK3s1xeAFWWIbEJGcMd2cgdcAgVHrXxf8AC+i6pLp5N5eSwkrM1pEGSMjqCSwzj2zUfw+0yG0+E9lcWVrF9ua1mmSUIN5kO7Bz1z0H04rzz4THxV/ZOpnw5baDMHmAuTqHmeb93gfKR8vXr3zT5noFke5aH4g0zxHpKanpdyJ7ZsgnGGVh1Vgehqh4S8aaV4zt7mfS1uAlu4R/PQKckZGME1x3we0a40k62z6jpN1bXDxuI9OufNWFvnyCP4eCBz/dr0LSbbRbeOUaNDYRISPMFmqKCe2dtUm2S0kaNFFFUSFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAZeo/8AId8Nf9hF/wD0kuK6iuX1H/kO+Gv+wi//AKSXFdRXmYz+J8jsofAFFFFchuFFFFAGZ4b/AOQp4m/7Caf+kltXQVz/AIb/AOQp4m/7Caf+kltXQV6MPhRzy3CiiiqEFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFYPjD/kBw/8AYSsP/SuGt6sHxh/yA4f+wlYf+lcNKWzGtzRooorzToCiiigAooooAKKKKACiiigAooooAKKKKACiiigArmfEM2oSa9pmn2eqXOnxS2tzPI1vHEzMUaFVH7xGAH7xugHaumrmtZ/5HLSP+wfef+jLarpq8hrcrf2drH/Q3ax/35s//jFH9nax/wBDdrH/AH5s/wD4xWpRXTyrsbckTL/s7WP+hu1j/vzZ/wDxij+ztY/6G7WP+/Nn/wDGK1KKOVdg5ImX/Z2sf9DdrH/fmz/+MUf2drH/AEN2sf8Afmz/APjFalFHKuwckTL/ALO1j/obtY/782f/AMYo/s7WP+hu1j/vzZ//ABitSijlXYOSJl/2drH/AEN2sf8Afmz/APjFH9nax/0N2sf9+bP/AOMVqUUcq7ByRMv+ztY/6G7WP+/Nn/8AGKP7O1j/AKG7WP8AvzZ//GK1KKOVdg5ImX/Z2sf9DdrH/fmz/wDjFH9nax/0N2sf9+bP/wCMVqUUcq7ByRMv+ztY/wChu1j/AL82f/xij+ztY/6G7WP+/Nn/APGK1KKOVdg5ImX/AGdrH/Q3ax/35s//AIxR/Z2sf9DdrH/fmz/+MVqUUcq7ByROeu/DFzf3tleXXiXVZbixdpLaRoLPMbFSpx+47g9D6A9QKu/2drH/AEN2sf8Afmz/APjFalFFl2FyRMv+ztY/6G7WP+/Nn/8AGKP7O1j/AKG7WP8AvzZ//GK1KKOVdh8kTL/s7WP+hu1j/vzZ/wDxij+ztY/6G7WP+/Nn/wDGK1KKOVdg5ImX/Z2sf9DdrH/fmz/+MUf2drH/AEN2sf8Afmz/APjFalFHKuwckTL/ALO1j/obtY/782f/AMYo/s7WP+hu1j/vzZ//ABitSijlXYOSJl/2drH/AEN2sf8Afmz/APjFH9nax/0N2sf9+bP/AOMVqUUcq7ByRMv+ztY/6G7WP+/Nn/8AGKP7O1j/AKG7WP8AvzZ//GK1KKOVdg5ImX/Z2sf9DdrH/fmz/wDjFH9nax/0N2sf9+bP/wCMVqUUcq7ByRMv+ztY/wChu1j/AL82f/xij+ztY/6G7WP+/Nn/APGK1KKOVdg5ImX/AGdrH/Q3ax/35s//AIxTIjq2na9oySeIL+9gu7p4JYbiK3CkCCVwQY4lYHci9/Wtes2+/wCQ/wCG/wDsIP8A+ktxUyirPQmUUkdZRRRXKZBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAVlab/wAjzq//AGDbL/0bdVq1lab/AMjzq/8A2DbL/wBG3VbUPjInsQ+Df+RG8P8A/YNtv/RS1t1ieDf+RG8P/wDYNtv/AEUtbdZPcpbBXP8AiX/kIeG/+wk3/pLcV0Fc/wCJf+Qh4b/7CTf+ktxWlH+JEmp8DLleGfGqzTUPHfhqykZlS4RYmZeoDS4JH517nXhnxqtmvfHfhq1SZoWnRYxKvVC0uNw+mc1609jihuVfG3wttvBHh2TXtI1y8Se3kT5ZGClssB8pXBBGc/QGrXjbWbrX/gNoepXx3XMt2qyNjG4r5qbvx25/Gtf/AIUd9rmjOreLNQvoEOfLKYP4FmbH5VL8Z7C10v4YWNhZRCK2t7yKONB2AR6mzSZV7tGToXwT0XVPDOnarJq1/DLdWcdw2Nm1GZAx7dBn1qf4KatqP9sa3oMt699YWg3QyliyqQ+35SegYc49vrXN6t8JyPh3YeI9Ku7m4nNnFdXFrLgjayBmKYA6Z6HPHf19D+Dd94fu/C0i6RYpZXkbAX0e8uzNj5W3HkqecDtyPckVqgb0Z5V4N8IaP4t17XU1fU3sVt5cxlZEXeSzZ+8PYV6R4b+FXhfSfENlqFhr81zc20nmJF50TbiB6AZrhPh74N0bxf4g8QR6w0wW2kDR+XLs5Z3znj2Fer+Hfhl4Y8Na1DqmnPc/aogyp5k4YfMCp4x6GlFeQSfmea+KvD9v4p+P8ujXUssUNwi7nixuG223DGeOq1Y8WfD29+HGmr4k8Na5eAW0iiZHwCASADxwwyQCCO9XpWC/tPxliANnU/8AXoa6f4ya3Y2vw/vbFrmI3V40aRRBgWOHVicegCnn3FOys2F3dI5/4ka6PEvwV0vV9gRrm5iLqOgcB1YD23A1Y8ReL7nwp8G/Dg09/Lv76ygijk7xqIwWYe/QD657VgeI7CbTf2dtDguFKyNdLLtPYOZHX9GFM+JGnzzfCfwTfopMVtaxxyEdt8SYJ/74x+NJt6vyBJbGxp3wOg1LRUvdY1m+/tm5jErMCGVGIzhsgsxHc5Fbfg3wv4pfwtq/hrxZI32GVDFbTrOskiqcggdfl6EZ6fy7TR/EOm6n4ag1mK7hFoYQ8jlwBEcfMG9CO9ZHgv4gWfjae/js7G5hWzbBlfBRwSQuD1yQM4xx61aUUS3Jni/xO+HeneB7XTpbG8upzdO6sJ9vG0DpgD1r1Twl8JtJ8L61ba1a397LNGjAJLs2ncpB6DPeub/aD/5B+hf9dZv5LXscH/HvH/uD+VJRXMxuT5UeIfBC9i0218Y30+RDbJFM+P7qiYn9BVTw34f1L4xalfa3r+pXEGmQy+XFBAfunGdqZyAACMnBJzSfCaxl1Pw74+sIBma5tlijHqzLMB+tbnwI1yzj0i+0CeVYb9Lpp0ic7WdSqqcA9SCvI9xUrWyY3pdoxfE/hjUfhDdWeveHNUuZbCSYRTQTkHJwSA2MBgQDzgEYq98bdQi1fwj4Y1GAERXRMyA9QGRTj9a0vjtrtkvh610OOZJL+a5WVolOSiKDyfTJIx681z/xS0+bSvhx4MsbgFZ4U2yKf4W8tcj8DxQ9LpAtbNna/Cbxg+veHZNJv2I1XTF8tw/DPGOFY+4+6foPWsb9n7/kBax/18p/6DVX4iaZdeBPGlp450eM/ZriTZexLwCx+9n2cfkwz3FWv2fv+QFrH/Xyn/oNNP3kmJ7No9hooorUzCiiigAqhpX/ACOmr/8AYOs//RlzV+sFNHttV8aal9olvk8vTrTb9lvprfOZLnr5bru6d845x1Nc+K/hM1o/GdlRWD/wiGmf8/Wt/wDg8vf/AI7R/wAIhpn/AD9a3/4PL3/47XlaHbqb1Q3F5bWhhFzcRQmaQRReY4Xe5BIUZ6nAPHtWP/wiGmf8/Wt/+Dy9/wDjtYfij4X6b4ksrW0OparDHHcCWQyahPcblCsu1RLIyqfmHzYJ4I7mmlHqw1NVv+R51L/sG2f/AKNua0awdL0uDRfEt3p9s87wwaXZqrTzNK5/e3PVmOfw6Dtit6vWw/8ADRw1fjZn61omn+INMl07VLZbi2k6q3BB7EHqD7ivOR8BPDv2nd/aWpm33Z8ren5Z2/0r1aitHFPchSa2KGm6Jp2kaPHpVlaRx2KIU8rGQQeuc9c989a4C9+B/h6e7lktL7UbGCU/vLeGQFMegyCcfXNenUU2kwTaOfsfBukaX4UufDthE0FpcwyRSupzIxddpYk9Wx+Htio/D3guw8N+GLjQbW4uZLacyFnlKlxvXBxgAdvSukoosguzm/Cnguw8I6Hc6TZXFzLDcStKzTlSwLKFOMADoorAm8F2Hgn4aeKLTT7i5mSeznlY3BUkHyiOMAeleh0josiFHUMrDBUjIIpWQXZ4N8Pvhtpfi3wRFfSXd7ZXTTSRSvbSYEqAjAYHr/nrXq2k+BtC0fwvP4egti9lcqRcGRsvKSMEkjHPTGMYxxXQxQxQJshjSNOu1FAH6U+hRSG5Nnmtn8FtDt1kt5tU1a4sCSy2bzhYwx/iIAGSP6d61oPhppcHge58Ji+vmsZ5hKZGZPMQhlbAO3GMr6dzXaUUcqDmZnaDo8Hh/Q7TSbaSSSG1TYjSkFiM55wAO9cXqvwd0S+1O4vrG/1DS2uc+dFayAI2evGOAfTp7V6LRTaTEm0YfhjwnpXhLSTp+lxOqOd0ksh3PI2MZY/0HFVPBngfT/BFtdQafcXUy3Lq7m4ZSQQMcYArp6KLILsKKKKYgooooAKKKjeeKORI3lRXf7qswBb6DvQBJRRRQAUUUUAFFFFABRRRQBl6j/yHfDX/AGEX/wDSS4rqK5fUf+Q74a/7CL/+klxXUV5mM/ifI7KHwBRRRXIbhRRRQBmeG/8AkKeJv+wmn/pJbV0Fc/4b/wCQp4m/7Caf+kltXQV6MPhRzy3CiiiqEFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFYPjD/kBw/9hKw/9K4a3qwfGH/IDh/7CVh/6Vw0pbMa3NGiiivNOgKKKKACiiigAooooAKKKKACiiigAooooAKKKKACua1n/kctI/7B95/6Mtq6Wua1n/kctI/7B95/6Mtq0p/EOO5booorpOgKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigArNvv8AkP8Ahv8A7CD/APpLcVpVm33/ACH/AA3/ANhB/wD0luKUtmTP4TrKKKK4zAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACsrTf+R51f/sG2X/o26rVrK03/kedX/7Btl/6Nuq2ofGRPYh8G/8AIjeH/wDsG23/AKKWtusTwb/yI3h//sG23/opa26ye5S2Cuf8S/8AIQ8N/wDYSb/0luK6Cuf8S/8AIQ8N/wDYSb/0luK0o/xIk1PgZcrlfEfgPTfE2vabq93c3Uc+nlTGkTKFbDbucgnr6V1VFew1c4E7BWD4u8J2XjLR00y/nuIYVmWYNAQGyARjkHj5jW9RQ9QKel6dFpOj2emws7w2kCQI0mCxVVCjOO/FczoXw20rw34nn1vS7u9gM28PaBl8kq3O3G3OAcEc8YrsqKLILs8tuPgR4cubmWdtS1UNI5cgPHgEnP8AcqfSfgnoGj6xZ6lBqGpvLaTJMiu8e0lSCAcJ04r0uilyIfMzgPE/wj0TxVr9xrF5fahFPOFDJCyBRtUKMZUnoPWq+k/BPwnpl5Hcy/bL9kO4R3UilM+4VRn6HIr0eijlQczMHxb4TsvGGirpd7NPDAsqyhoCobIBAHIIxzVqDw/YR+GYfD88X2qwitltis4BLqoAGcY54zkY5rUop2QrnlVx8BfDkt0Xh1DUoIGOTCro2PYErn8813/h7w5pfhbS107SrfyoQdzEnLO3dmPc1rUUlFLYHJvc5fxn4F07xvDaRahc3UItWZkNuyjO4DOcg+ldOihEVR0AxS0U7Bc5Xwd4D07wS9+2n3N1Mb0oZPtDKdu3djGAP7xrM8T/AAj8OeJb99QPn2N5Id0j2zALIfUqQRn3GK72ilyq1gu73PP/AA18IPDnh3UI9QY3F/dRtuja5YFUbsQoA5+ua2fGXgfT/G1taQahcXUK2zs6G3ZQSSMc5B9K6eijlVrBzO9yjrGk2mu6RdaXfR77a5Qo4HUehHuDgj3FY/g3wRp/gi0urbT7i5mS4kEjG4KkggY4wBXTUU7K9wv0CiiimIKKKKACqGlf8jpq/wD2DrP/ANGXNX6oaV/yOmr/APYOs/8A0Zc1z4r+Ezah8Z0dFFFeSdoUUUUAc03/ACPOpf8AYNs//RtzWjWc3/I86l/2DbP/ANG3NaNexh/4SOCr8bKOs6pBoui3mp3J/dWsLSsM9cDoPcnj8a+ZNE1LVdC8Q6X47vQTb319MJXH8YyPM/8AQzj3U+leofHPXJItGsPDlpue51GUM8ackopG0Y93Ix/u1xWsHxZe+AbXw3L4IuYLWwCyJciJ9ylQdzHjHOWz9ac3qOC0PoO+1G3sNIudTlJe2t4GuGMfJKKpY49eBXCzfGvwnFYRXSC/naTJMEUIMiAHGWywAH45rM8JeIv7d+BurwyvuutP064tZMnkqIm2H/vnA+qmpPghYWr/AA+vGeBGa5upElJXO9Qqjafbk8e5qrt2sTZLc7vw14o0vxZpP9o6XKzRBijrIu1o2HOGH0I9q5PU/jX4S02/e0Q3t7sO1prWJWjB9izDP1GRXAfDiS5T4V+NzaFvNEWRt6gbG3Efhmu0+CcOlN8PpSqQNO80i3u4Ak+gbP8ADtx7dfekpN2G4pXO10zxZo+s+Hptb0+58+0hRnkCjDptGSpU9DiuUl+NnhNLBLpBfzMxIMEcAMigfxNlsAc+ua4j4ebVvPiEmnHOji2n8rb93GX8vH/Ac1tfBezt2+G2uytEhkmuJYpGI5ZBCuFPt8zfnQpNhypHRXPxl8JQabb3kct3ctMCTbQQgyxgdd4JAH589q6Tw94u0fxNob6vYXBW2i3CbzhsaEgZO7sOOc5xXnHwEs7d/DetTPEjPLOIXJGcoEzj6fMayvhHqVnpPgLxXe6jALmzhKmSBlDCQFSNuDxzwOaFJ6XBxWtjrp/jj4VindI7fVJ4UbabiK3Xyz+bA/pU/wARNYsde+DOpanp0wmtZ1iKPgjpOgIIPQggiuOa98S618M9SvtOsdB0Pwy0MzeRGpaWUAkEDsCSMA4B6Y7VHYZ/4Zn1Hn/lv/7cJS5mx8qOm8MeNdL8G/DDwzJqkd0YrlJEV4YwwUhzweRjr+hru/EPiXTvDOhPrF+7m1UqB5QDM5Y4GBnn1+grziLw/wD8JF+zzZ26JuubeBrmDjncjuSB7ldw/GuNuNem8eaJ4K8IRSMZxJsuyOcBDsRvwj3E0+ZpC5Uz6E0fVIda0i11O3jljguYxJGsq7W2noSMnqOfxryX4q6DdeGvEFn4+0NdkkUqi7VRxu6Bj7MPlP4etexwQR21vFbwoEiiQIijoqgYAqO+sbbUrCexu4hLbzxmORD3UjBqmrolOzPGfH/jn/hM9N0bw34ZJkn1cJJcKDygzxGx7YIJb2UdjXaf2j4c+EfhKxsLuZi2CQkSbpbiT+NsfX1OAMCuC+A2l2j67rt68e6ezWOKB2/hDl934/IBn6+tHj86k3xx0pbZLN51ii+xrf58kn5sZxz97OPfFQm7cxdlflPRvC/xL8P+K786fatc2t7glbe8jCM4HJ24JB45xnNJ4k+JugeGdV/suZby8vlALwWUQdkyMjOSB05xXC6to/ijUPiHoN/rN54XsdStpImSK3uWjknj39NrZLfxAY9TUmveHdVX4iajq/gXxHYPqsik3Vj5yGVMFQwIIKkZA4OCDT5nYXKrnoHhPx9ofjF5odOeaK6hG6S2uU2SAZxngkEZ44PFc3c/8Tz492sP3odD08yMOwkf+uJFP/AazfAniq//AOE/n0HxH4fsLXXJUZnvLeFVkYhQ2HIyCCozkH04rT+Fn/E21jxZ4nPzLe6gYYWP/PNMkfoy/lRe9hWtc8k8O+Kbjwf8Q7zUwrtYteSQXiqMgozn9RgkfQ+9d/8AGueK6ufBtxBIskMs0jo6nIZSYiCPwrJ8A+HLXxW/j3SbrA82ZDFJjJjkDy7WH4/mCR3rir/UNUjl0fwvqsZWbRb50UsckKzJ8vuAVJB9GFRtEvdn0z4k8U6R4T08XurXPlIx2xoo3PIfRR3/AJVy+ifGTwprWox2Ia7spJWCxtdxqqMT0GVY4/HFcp8UPKk+L3heLVtv9k7IsiT/AFeTK27PtwmfatL48RaavhOwZ1iW+FyFt8ABtm07gP8AZ+7+OKtyevkQorQ9aorL8NG5bwrpDXu77UbKEzbuu/YN2ffOa1KsgKKKKYGXqP8AyHfDX/YRf/0kuK6iuX1H/kO+Gv8AsIv/AOklxXUV5mM/ifI7KHwBRRRXIbhRRRQBmeG/+Qp4m/7Caf8ApJbV0Fc/4b/5Cnib/sJp/wCkltXQV6MPhRzy3CiiiqEFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFYPjD/kBw/wDYSsP/AErhrerB8Yf8gOH/ALCVh/6Vw0pbMa3NGiiivNOgKKKKACiiigAooooAKKKKACiiigAooooAKKKKACua1n/kctI/7B95/wCjLaulrmtZ/wCRy0j/ALB95/6Mtq0p/EOO5ZkkSKJ5JGCogLMT2AqhpGvaVr8Mk2lX0V3HG212jPCnGcVLq3/IHvv+veT/ANBNeX/AL/kXdW/6+1/9AFdVtDVytJI9bZgqlicADJNZuj+IdJ1+OV9Kv4btYiBIYz90npV65/49Zf8AcP8AKvIfgB/yDNb/AOu0X/oLUW0BytJI9jJwMnpXP6Z448M6zqEdhp+rwXF1JnZGoYFsAk9R6A1vP/q2+hr5N8MSvol7pXiUEiK31FYpvZcAkfiu/wDKhK5NSbi0fUGteI9I8OxwyatfxWizEiMvn5iOvQe9T6Vq1hrdgt9ptylxbOSFkTOCQcHrXinxhmbXfGUWlxNmLTNNluZCP4W2l/1Cxj8a7n4Nf8k3s/8ArtN/6GaLaXBTbm4nf1m6Pr+ma/HcSaXdC5jt5jDI6owUOOoBIAbtyMjmsL4na5JoPgLUJ4GK3E4FtEw6gvwSPcLuP4VgwzP4E+BcdzaDZdm0SQNjkSzEc/Vd/wD47RYpys7HV6z478MaBcm21LWIIZx96JQ0jL9QoJH41b0TxTofiMP/AGRqcF0yDLIpIdR6lTg498V5t8M/AOh3nhRPEOvW6XtzeGSQtcsSsaBiMnnBJwSSfWrEfgvwxZeN7PXNA8U6fpscTKWtI5kkEhzhlB38Bhxjn2osiVKejPWKK4bxn8QH0DVrXQtI01tT1q6AZYQ2FQHOM+p4Jxxgck1B4c+IV9ceJl8NeJ9G/srU5U3wFX3JJwTjv6HBBIJBHBosyudXsegUV5nf/FDULfxrqPhqy0A31zF8lsIpcGR8KfmyMKoBYk+w+tUbL4t6wNTutC1Dwu511W2W9tbycM3XDE5wAPmyMjHp1osxe0ietVl2/iPSLvW59Ggvo31GAEyW4B3KBjPbHcVx3hT4i6lqPi+Xwx4h0ddP1EKWTy3yMhd2D1/h5BBqDQNV06b4y6zYRaHBBexxOXv1lcvIPk4KngdR+VFg507WPTKK8tT4qard+ItX0LTfDhvb+1nkit1jlwrKjlS7k8KOB+JxmtHwV8RbzXvEV34e1rShp+p26s2EYlTgjKkHoecg5II/UsxqpFux1tl4j0jUdXudKtL6OW+tcmaFQcpggHPHqRWpXmfg/VdOuvit4jsrfQ4LW7hWbzbxJXZpsSqDlTwMk549K9MpMcXdGdrOu6Z4esheareJa25cIHcE5Y9gAM9jV6GaO4gjnibdHIodGHcEZBr53+JemeLZtLi17xRcRxq10Le1sIukSlWbdwSAflHcn1xjFe96F/yL2m/9esX/AKAKbVkTGbk2rF53WNGd2CqoySegFc5/wsLwh/0MNh/39qTx1f8A9meBdaugcMLR0U+jMNo/VhXgnhe88AW2jWy+I9A1C6uXkYS3iO6xrycABXGcDGeM9etCVxTm4uyPpWzvLbULOK7tJkmt5V3RyIchh6iotU1Ww0WwkvtSuo7a2j+9JIePYD1PsKi0GLTIdBsk0YqdNEQ+z7WLDYeRyefzryj476hqJtLPTjYEaYJUmF5u4aXa42Y+nNCV2VKXLG56Bb/EXwndWE19FrMRtoHWOSRo3XazZ2jBUHna35V0drcw3tpDdW8gkgmRZI3HRlIyD+RrwfxOJv8AhT6tceHINFf7bAi+UgBuEEZw7EDk8nrXez+LH8JfDPw7cwadPf3M1jbxxRRg4B8peWIBwOn1zQ0TGp3PQKK8ol+J/iXQNSsY/FfhiOys7x9qyRS5ZRxk9SDjI4ODW74x8e3Wia/ZeHdF0sahq92m9UkkCIoOcDJ6n5T3GKLMr2kTuqyz4j0hdfXQjfRjU2G4W2DuI27vTHQZrlfD/jTxHN4lTQ/EfheWylkTctxbZkjXg43EZABwRnPB61xPiTVJNH+P63sNlLezLEqRW0X3pHaAqoz2GSMnsM0JClUsro90rNttf0y71u70aG6B1G0UPNAyMpVTjBBIww5HTPUVwuk/ErWI/GVt4d8T6CunTXePJaOTdjOdueoIJGMg9ap/EiY+GPiB4Y8UQnasjG1usfxICM59fldv++RRYHUVro9YrNvv+Q/4b/7CD/8ApLcVpVm33/If8N/9hB//AEluKmWzKn8J1lFFFcZgFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABWVpv8AyPOr/wDYNsv/AEbdVq1lab/yPOr/APYNsv8A0bdVtQ+MiexD4N/5Ebw//wBg22/9FLW3WJ4N/wCRG8P/APYNtv8A0UtbdZPcpbBXP+Jf+P8A8OHsNSbP/gLcD+tdBWT4i0+fUNJItApvbeRLi2DHAZ0OdpPYMMqT6MaulJRmmxTV4tDqKq6dqEGp2a3NuTgkq6OMNG44ZGHZgeCKtV7J5wUUUUwCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKoaTz4z1c9v7Psx+PmXP8AiKs3V1BY2st1dSrFBEpZ3c4CimeGrWfyrvU7uJop9QlEgicYaKJQFjU+hwCxHYuR2rlxckqdu5vQTcrm7RRRXlnYFFFFAHNN/wAjzqX/AGDbP/0bc1o1nN/yPOpf9g2z/wDRtzWjXsYf+Ejgq/Gzk7/wBpup+NrbxRd3V3Jc22zyoCy+Uu3pxtz1Jbr1rrCAQQRkGiitbEXOI0b4YaToUetQ2V7fC31a3e3mhZk2orZAK/L1AYgZz1rZ8J+FLPwfop0uxmnlhMrS7pyC2SB6ADtW9RQkkF2zl/B3gTTfBdpeW1jPc3Ed0waT7SVboCMcAcc1zmofBHw7dXss9nd6hp8Ux/eW9vINhHoMgkD25Fel0UuVbBzMwNJ8HaRofhufQ9OiaG3uI2SWTOZHLLtLE+uPw9qi8LeC7DwloFzo9lcXMsFxK8rPMVLAsoU4wAOijtXSUU7ILs5vwd4LsPBWnXFlYXFzNHPL5rGcqSDgDjAHpVLw98NtF8PaLqekpJc3dpqIAnW4Zc4wRwVAx1rsaKLILs81tPgroVuskEupatcWJJZbR5wIwxH3iABkj/8AXmte2+G+mW3ge78Ji+vnsbmUSGRmTzE+ZWwDtxjK9x3NdnRS5UPmZyM2oaF8L/CdhaXlxcmyjJhidk3u7Hc2DtAHr6V538FdBS+8S6r4oW0aCxRniskbsXOTg99q/L/wKvbLmztr2MR3dtDPGDkLKgYA+uDT4oo4Y1jiRY41GFVRgAewo5dQ5tB9FFFUScr4P8B6d4Lm1CSwubqY3xQyeeynbt3YxgD+8ak8X+BdH8ZwwjUFliuIM+VcwMFdQe3III+o+mK6ailZWsO7vc4bw58LdH0DWF1ea6vdT1BP9XLeSBvL4xkDHXHrnHaneI/hfo+vawdYhur3TNSb789lIF3nGMkY647jHvXb0UuVbBzM4zw98NdJ8OteXMF1eXOp3UTRNfXLh5EDDkrxj065PHWtnwp4atfCXh+DSLSR5Y4mZjJJjc5ZicnH5fhW1RTSSBts5jwt4H0/wlf6pd2VxdSvqLh5RMykKQWPy4A/vHrmqXib4ZaJ4o16DWbmS6t7uMKGNuygSbTkFsqee30rtKKLK1gu9zB8VeD9I8YWCWuqwsfLJMU0bbZIyeuD7+hBFcvo/wAGfD2m6lFe3dze6k0BHlRXTgxrjpkAc49OntXo1FJxT1C7QUUUVQgooooAy9R/5Dvhr/sIv/6SXFdRXL6j/wAh3w1/2EX/APSS4rqK8zGfxPkdlD4AooorkNwooooAzPDf/IU8Tf8AYTT/ANJLaugrn/Df/IU8Tf8AYTT/ANJLaugr0YfCjnluFFFFUIKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKwfGH/ACA4f+wlYf8ApXDW9WD4w/5AcP8A2ErD/wBK4aUtmNbmjRRRXmnQFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABXNaz/yOWkf9g+8/wDRltXS1zWs/wDI5aR/2D7z/wBGW1aU/iHHcTVv+QPff9e8n/oJrxT4PeL9B8OaLqMGr6ilrLLcB0VkY5G0DPANe6yRpLG0bqGRwVYHoQa5z/hXvhD/AKF6w/79V1J6GkottND9N8Z+HvETXFppOpx3U6QNKyKjDCjAzyB3I/OvJPg54s0Lw3YarHq+oJavNLG0YZWO4AHPQH1r2XTPCug6LcPPpulWtrLJGY2eJMEqSCR9MgflVL/hXvhD/oXrD/v1RdCcZNpkmleNfDniC6ey0rVI7m4EbSFFRgdo6nkD1FeE+HdL/tP4P+Jiq5ktLuK5T/gIw3/jpavftM8J6Bo10bnTdJtbWcoUMkSYO09R+gqWx8N6Lptjc2VlpltBa3QKzxImFkBGCCO/HFNOwSg5bng3hCO51jQPG3iK9O+ZNK+yq57/ACc/iBGv516d8Gv+Sb2f/Xab/wBDNdXaeG9FsdMuNNtdMtobK5z50KJhXyMHI+gqzp2mWOkWa2en2sdtbKSVjjGACeTQ3cIU3F3PP/jnBJL4CidAdsV9G7/Ta6/zYVr61o7eKvhKtjaYMs+nwywDPVlCuo/HGPxre8TaJF4j8N3+kykKLmIqrH+Fxyp/BgDWL8M21KLwdBp2rWVxa3enu1sRNGVDqD8rKTwwwcZHpSvoNr3nfqcN8PfHeiWfhGXwp4mlewltxLAfNRgHRicgkD5WG4jn2ridftvCreLdItfBi3NwomUTMQ7B23DAXPJwM9q+g9X8HeHdem8/U9ItbiboZSu1z9WGCam0jwxoeg5Ol6Xa2rkYLxxjeR6FuuPxp8yIdOTVmeN+OLeSw+Nkd3d6pPpFtdxoYr+MZ8seXsP4bgQfQNmrdlZaNf8AxL0mI+MNS1vUbZ0kilSFZIsKSxQuG4GAcnGOa9f1bQ9L162FvqtjBdxKcqJVyVPqD1H4VBovhbQ/Dpc6Tplvau4wzoMsR6bjk49qLj9nqeZ+HAD+0PrpI6QuR+UdJaKD+0ve5HSEEf8AgMlepw6DpVvq8urQ2ECahKCJLhV+dgcdT+A/KhdB0pNabWVsIBqTjDXO35yMBev0AFK4/Zv8bnls3/JzFv8A9cT/AOkzUeF/+ThPEH/XGT/2nXqR0HSjrQ1k2EB1IDAudvzgbdvX6cUQaDpVtq82qwWEEeoTAiS4VfnYHGcn8B+VFw9m7/O55b8MgP8AhbPjNsciacZ/7bmjS+P2kdTx/wA8T/6JSvUrLQdK02/ub6ysIILq5JM8qLhpCTk5P15oTQdKj1l9YSwgXUXGGuQvzkYA6/QAUXBU3ZLzPLfAP/JcPF3+7P8A+jkr2Os600HSrHU7jUrWwghvbjPnTouGfJycn6gGtGhu5UI8qseWfHn/AJEqw/7CKf8AouSu6sr+30vwba3925S3t7BJZGAJIUICeB1q3qujabrlsltqllDdwo/mKkq5AbBGfyJ/OppbC1n09rCWBHtGj8poSPlKYxtx6YovoHK+Zs5jUvGWk3ngS68QWdi2r6dGcSQNHt3AMASQw6Dr07ZrjvE3jvwprHwnuLe2NtBPNEqRaagG6FwwPAAAAGM54H8q9T0/SdP0qx+xWFnDb2uSfKRcLz14rIj+H/hKK/F6mgWQmDbh8nyg+y/d/ShNClGTKXwssruw+HOlQ3iskjK8io3VUZ2Zf0IP41zfx6/5E/T/APr/AF/9FvXqtUNV0XTdct0t9UsobuFH3qkq5AbBGfyJovrcbh7nKjzf4sf8kk0z/rpbf+izVLxX4u1bw54G8GWemXAsxfWMQlvCm4xqscYwM5/vZPfjivVb/RdM1TT0sL6yhuLRCCsMi5UYGBx7Co77w9o+paVFpd5p0E1lCqrFCy8IFGBt7jA44oTJcHdtM+fPiJDp1tLpYh8XXPiG8Zi08j3AkjiXjG0DIXPPGe1d18Trfwjqvimx0/WLi90zU2iAj1BUXyNnJG8kjgHIyMYJ5OK7uPwJ4WjsBYroVl9n3iTaY8ksAQCSeTwT371e1Tw9pGtWkdrqWnW91DFxGJFyU+h6j8KdxezevmeLaJqmt+FvH+k6Hp/ilPENhdSIkiRyGVUQtg9ztIGW4PbmtmcA/tL22RnEJP8A5LNXpOjeEfD/AIflaXStKt7aUjBkVcvj03HJxVk6DpR1oaybCA6kowLnb84G3b1+nFFwVN2+Z5b8Qv8AktXg/wCsH/o5qn+O4M+m6FZxjM0122wdzwB/NhXpd3oOlX+pW+o3dhBNeW2PJmdcsmDkYP15ritW0a98UfFvT3ns7iPR9DiEomljKpNMTuAQkYbnZnH9w0JhKLs13PRVG1QM5wMZrOvv+Q/4b/7CD/8ApLcVpVm33/If8N/9hB//AEluKiWzNJ/CdZRRRXGYBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAVlab/yPOr/APYNsv8A0bdVq1lab/yPOr/9g2y/9G3VbUPjInsQ+Df+RG8P/wDYNtv/AEUtbdYng3/kRvD/AP2Dbb/0UtbdZPcpbBRRRSGY+oeHbe7u2vrW4n0++YAPPbEfvAOm9GBVvqRkDoRVNtI8TDiPXdL2+smlSMfzFwB+ldJRWsa1SKsmQ6cXq0cz/ZPin/oO6N/4KJf/AJJo/snxT/0HdG/8FEv/AMk101FV9Zq9xeyh2OZ/snxT/wBB3Rv/AAUS/wDyTR/ZPin/AKDujf8Agol/+Sa6aij6zV7h7KHY5n+yfFP/AEHdG/8ABRL/APJNH9k+Kf8AoO6N/wCCiX/5JrpqKPrNXuHsodjmf7J8U/8AQd0b/wAFEv8A8k0f2T4p/wCg7o3/AIKJf/kmumoo+s1e4eyh2OZ/snxT/wBB3Rv/AAUS/wDyTR/ZPin/AKDujf8Agol/+Sa6aij6zV7h7KHY5n+yfFP/AEHdG/8ABRL/APJNH9k+Kf8AoO6N/wCCiX/5JrpqKPrNXuHsodjmf7J8U/8AQd0b/wAFEv8A8k0f2T4p/wCg7o3/AIKJf/kmumoo+s1e4eyh2OZ/snxT/wBB3Rv/AAUS/wDyTR/ZPin/AKDujf8Agol/+Sa6aij6zV7h7KHY5n+yfFP/AEHdG/8ABRL/APJNH9k+Kf8AoO6N/wCCiX/5JrpqKPrNXuHsodjmf7J8U/8AQd0b/wAFEv8A8k0f2T4p/wCg7o3/AIKJf/kmumoo+s1e4eyh2OZ/snxT/wBB3Rv/AAUS/wDyTR/ZPin/AKDujf8Agol/+Sa6aij6zV7h7KHY5n+yfFP/AEHdG/8ABRL/APJNKuk+J8/NrmkY/wBnSZAf1uDXS0UfWavcPZQ7GFa+GY/tMd3qd5NqU8TB4llASKJvVY1GM+hbcR2IrdoorKUnJ3bLSS0QUUUVIwooooA5pv8AkedS/wCwbZ/+jbmtGs/UB9l8ZW0p4S+s2gz/ALcTb1H1KySH/gJrQr18M70kcNb42FFFFbmQUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFAGXqP8AyHfDX/YQf/0kuK6iuYYfbPGGnQLytlDJdyH+6zDyox+IaX/vk109eXi3eodtBe4FFFFcpsFFFFAGZ4b/AOQp4m/7Caf+kltXQVz/AIb/AOQp4m/7Caf+kltXQV6MPhRzy3CiiiqEFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFYPjD/kBw/8AYSsP/SuGt6sHxh/yA4f+wlYf+lcNKWzGtzRooorzToCiiigAooooAKKKKACiiigAooooAKKKKACiiigArmtZ/wCRy0j/ALB95/6Mtq6Wua1n/kctI/7B95/6Mtq0p/EOO5booorpOgKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigArNvv+Q/4b/7CD/+ktxWlWbff8h/w3/2EH/9JbilLZkz+E6yiiiuMwCiiigAooooAKKKKACiiigAooooAKKKKACiiigArK03/kedX/7Btl/6Nuq1aytN/wCR51f/ALBtl/6Nuq2ofGRPYh8G/wDIjeH/APsG23/opa26xPBv/IjeH/8AsG23/opa26ye5S2CiiikMKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKAMzXdLfVdO2QSLFeQuJ7WVhwkq9M+xBKn2Y1Q0zUk1GBiY2guYW8u4tnPzQv3U+o7g9CCCODXRVk6poMOoTreQzSWeoIu1LqHGSvXa4PDr7Hpk4IPNdOHr+zdnsY1afPqtx1FZhPiKz+WfTLfUFHSWynEbN9Y5CAv8A32aYdU1YEj/hEdYPuJrP/wCP16Cr031OZ0proa1FZH9q6t/0KGs/9/rP/wCSKP7V1b/oUNZ/7/Wf/wAkUe2p/wAyF7OfY16KyP7V1b/oUNZ/7/Wf/wAkUf2rq3/Qoaz/AN/rP/5Io9tT/mQezn2Neisj+1dW/wChQ1n/AL/Wf/yRR/aurf8AQoaz/wB/rP8A+SKPbU/5kHs59jXorI/tXVv+hQ1n/v8AWf8A8kUf2rq3/Qoaz/3+s/8A5Io9tT/mQezn2Neisj+1dW/6FDWf+/1n/wDJFH9q6t/0KGs/9/rP/wCSKPbU/wCZB7OfY16KyP7V1b/oUNZ/7/Wf/wAkUf2rq3/Qoaz/AN/rP/5Io9tT/mQezn2Neisj+1dW/wChQ1n/AL/Wf/yRR/aurf8AQoaz/wB/rP8A+SKPbU/5kHs59jXorI/tXVv+hQ1n/v8AWf8A8kUf2rq3/Qoaz/3+s/8A5Io9tT/mQezn2Neisj+1dW/6FDWf+/1n/wDJFH9q6t/0KGs/9/rP/wCSKPbU/wCZB7OfY16KyP7V1b/oUNZ/7/Wf/wAkUf2rq3/Qoaz/AN/rP/5Io9tT/mQezn2Neisj+1dW/wChQ1n/AL/Wf/yRR/aurf8AQoaz/wB/rP8A+SKPbU/5kHs59jXorI/tXVv+hQ1n/v8AWf8A8kUf2rq3/Qoaz/3+s/8A5Io9tT/mQezn2Neisj+1dW/6FDWf+/1n/wDJFKNU1YkD/hEdYHuZrP8A+P0e2p/zIPZz7GtVTUdRg0y286bc7MwSKGMZeZz0RR3J/wDrnABNVw3iK8+WDS7ewU9ZL6cOy/8AbOPIb/vsVf0zw/DZXP266nkvtRKlftMwA2A9VjUcIPpyeMk4rKpioRXu6suFGTeoeH9Mnsrea6vtp1G8fzbjacqnGFjU+ijjPc7j3rYoorzZScndnYlZWQUUUVIwooooAzPDf/IU8Tf9hNP/AEktq6Cuf8N/8hTxN/2E0/8ASS2roK9GHwo55bhRRRVCCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACsHxh/yA4f8AsJWH/pXDW9WD4w/5AcP/AGErD/0rhpS2Y1uaNFFFeadAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFc1rP8AyOWkf9g+8/8ARltXS1z2vadq02safqOlw2U5gt54JI7q5eH/AFjRMCCsb5x5Z4wOtXTdpajTsyeis7yvFv8A0CdE/wDBtL/8jUeV4t/6BOif+DaX/wCRq6OePc19pE0aKzvK8W/9AnRP/BtL/wDI1HleLf8AoE6J/wCDaX/5Go549w9pE0aKzvK8W/8AQJ0T/wAG0v8A8jUeV4t/6BOif+DaX/5Go549w9pE0aKzvK8W/wDQJ0T/AMG0v/yNR5Xi3/oE6J/4Npf/AJGo549w9pE0aKzvK8W/9AnRP/BtL/8AI1HleLf+gTon/g2l/wDkajnj3D2kTRorO8rxb/0CdE/8G0v/AMjUeV4t/wCgTon/AINpf/kajnj3D2kTRorO8rxb/wBAnRP/AAbS/wDyNR5Xi3/oE6J/4Npf/kajnj3D2kTRorO8rxb/ANAnRP8AwbS//I1HleLf+gTon/g2l/8Akajnj3D2kTRorO8rxb/0CdE/8G0v/wAjUeV4t/6BOif+DaX/AORqOePcPaRNGisK8vPFFldafbyaNo5e+uDbxFdVlwGETyZb/R+mI26Z5I+tXPK8W/8AQJ0T/wAG0v8A8jUc0e4e0iaNFZ3leLf+gTon/g2l/wDkajyvFv8A0CdE/wDBtL/8jUc8e4e0iaNFZ3leLf8AoE6J/wCDaX/5Go8rxb/0CdE/8G0v/wAjUc8e4e0iaNFZ3leLf+gTon/g2l/+RqPK8W/9AnRP/BtL/wDI1HPHuHtImjRWd5Xi3/oE6J/4Npf/AJGo8rxb/wBAnRP/AAbS/wDyNRzx7h7SJo0VneV4t/6BOif+DaX/AORqPK8W/wDQJ0T/AMG0v/yNRzx7h7SJo0VneV4t/wCgTon/AINpf/kajyvFv/QJ0T/wbS//ACNRzx7h7SJo0VneV4t/6BOif+DaX/5Go8rxb/0CdE/8G0v/AMjUc8e4e0iaNFZ3leLf+gTon/g2l/8AkajyvFv/AECdE/8ABtL/API1HPHuHtImjWbff8h/w3/2EH/9Jbil8rxb/wBAnRP/AAbS/wDyNSQab4huta0q4v7PS7a2srh52aC+kmdswyRgBTCg6yZzntSlKNnqTKaasdZRRRXKZhRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAVlab/AMjzq/8A2DbL/wBG3VatZWm/8jzq/wD2DbL/ANG3VbUPjInsQ+Df+RG8P/8AYNtv/RS1t1ieDf8AkRvD/wD2Dbb/ANFLW3WT3KWwUUUUhhRRRQAUUUUAFFFFABRRRQAUUVV1DUrPS7U3N7OsUWQoyCSzHoqgcsT2ABJpgWqKwD4luWG638M61PGejgW8Wf8AgMkqsPxFJ/wkmof9Chrf/f2z/wDkinysDoKK5/8A4STUP+hQ1v8A7+2f/wAkUf8ACSah/wBChrf/AH9s/wD5Io5WB0FFc/8A8JJqH/Qoa3/39s//AJIo/wCEk1D/AKFDW/8Av7Z//JFHKwOgorn/APhJNQ/6FDW/+/tn/wDJFH/CSah/0KGt/wDf2z/+SKOVgdBRXP8A/CSah/0KGt/9/bP/AOSKP+Ek1D/oUNb/AO/tn/8AJFHKwOgorn/+Ek1D/oUNb/7+2f8A8kUf8JJqH/Qoa3/39s//AJIo5WB0FFc//wAJJqH/AEKGt/8Af2z/APkij/hJNQ/6FDW/+/tn/wDJFHKwOgorn/8AhJNQ/wChQ1v/AL+2f/yRR/wkmof9Chrf/f2z/wDkijlYHQUVz/8Awkmof9Chrf8A39s//kij/hJNQ/6FDW/+/tn/APJFHKwOgorn/wDhJNQ/6FDW/wDv7Z//ACRR/wAJJqH/AEKGt/8Af2z/APkijlYHQUVz/wDwkmof9Chrf/f2z/8Akij/AISTUP8AoUNb/wC/tn/8kUcrA6Ciuf8A+Ek1D/oUNb/7+2f/AMkUf8JJqH/Qoa3/AN/bP/5Io5WB0FFc/wD8JJqH/Qoa3/39s/8A5Io/4STUP+hQ1v8A7+2f/wAkUcrA6Ciuf/4STUP+hQ1v/v7Z/wDyRR/wkmof9Chrf/f2z/8AkijlYHQUVz//AAkmof8AQoa3/wB/bP8A+SKP+Ek1D/oUNb/7+2f/AMkUcrA6Ciuf/wCEk1D/AKFDW/8Av7Z//JFH/CSah/0KGt/9/bP/AOSKOVgdBRXP/wDCSah/0KGt/wDf2z/+SKP+Ek1D/oUNb/7+2f8A8kUcrA6Ciuf/AOEk1D/oUNb/AO/tn/8AJFH/AAkmof8AQoa3/wB/bP8A+SKOVgdBRXP/APCSah/0KGt/9/bP/wCSKP8AhJNQ/wChQ1v/AL+2f/yRRysDoKK5/wD4STUP+hQ1v/v7Z/8AyRR/wkmof9Chrf8A39s//kijlYHQUVz/APwkmof9Chrf/f2z/wDkij/hJNQ/6FDW/wDv7Z//ACRRysDoKK5//hJNQ/6FDW/+/tn/APJFH/CSah/0KGt/9/bP/wCSKOVgdBRWAviS7A3T+GNZhQdWP2eTH/AY5mY/gK1NO1Sz1W3M1nMJFVtjqVKvG391lIBU+xANJpoC3RRRSAKKKKACiiigAooooAKKKKACiiigAooooAzPDf8AyFPE3/YTT/0ktq6Cuf8ADf8AyFPE3/YTT/0ktq6CvRh8KOeW4UUUVQgooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigArB8Yf8gOH/sJWH/pXDW9WD4w/5AcP/YSsP/SuGlLZjW5o0UUV5p0BRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAYPiD/kNeFf+wo//pHc1vVg+IP+Q14V/wCwo/8A6R3Nb1N9BBRRRSGFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABWVpv/I86v8A9g2y/wDRt1WrWVpv/I86v/2DbL/0bdVtQ+MiexD4N/5Ebw//ANg22/8ARS1t1ieDf+RG8P8A/YNtv/RS1t1k9ylsFFFFIYUUUUAFFFFABRRRQAUUUUAFc1pyDVdavdXnG8W80lnZKekSodsjD/aZwwJ9FUeuelrnfDH/ACC7n/sJX/8A6Vy1cQW5s0UUVRYUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAViaog03V7DWIRsaSeOzu8f8tY5DtTPqVkZcHsCw71t1jeJ/8AkF23/YSsP/SuKgT2OiooorIkKKKKACiiigAooooAKKKKACiiigAooooAzPDf/IU8Tf8AYTT/ANJLaugrn/Df/IU8Tf8AYTT/ANJLaugr0YfCjnluFFFFUIKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKwfGH/ACA4f+wlYf8ApXDW9WD4w/5AcP8A2ErD/wBK4aUtmNbmjRRRXmnQFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRWTquuHTb+1sYdMvdQubmKWZY7UxDakZQMSZHQdZF6Emq//CQan/0J+t/9/rL/AOSKdhXN6isH/hINT/6E/W/+/wBZf/JFH/CQan/0J+t/9/rL/wCSKLBc3qKwf+Eg1P8A6E/W/wDv9Zf/ACRR/wAJBqf/AEJ+t/8Af6y/+SKLBc3qKwf+Eg1P/oT9b/7/AFl/8kUf8JBqf/Qn63/3+sv/AJIosFzeorB/4SDU/wDoT9b/AO/1l/8AJFH/AAkGp/8AQn63/wB/rL/5IosFzeorB/4SDU/+hP1v/v8AWX/yRR/wkGp/9Cfrf/f6y/8AkiiwXN6isH/hINT/AOhP1v8A7/WX/wAkUf8ACQan/wBCfrf/AH+sv/kiiwXDxB/yGvCv/YUf/wBI7mt6uO1S/wBYvdQ0W4j8I6uEsb1riUNNZ5Km3mjwv7/rmRTzjgH6Vp/8JBqf/Qn63/3+sv8A5IptBc3qKwf+Eg1P/oT9b/7/AFl/8kUf8JBqf/Qn63/3+sv/AJIpWC5vUVg/8JBqf/Qn63/3+sv/AJIo/wCEg1P/AKE/W/8Av9Zf/JFFgub1FYP/AAkGp/8AQn63/wB/rL/5Io/4SDU/+hP1v/v9Zf8AyRRYLm9RWD/wkGp/9Cfrf/f6y/8Akij/AISDU/8AoT9b/wC/1l/8kUWC5vUVg/8ACQan/wBCfrf/AH+sv/kij/hINT/6E/W/+/1l/wDJFFgub1FYP/CQan/0J+t/9/rL/wCSKP8AhINT/wChP1v/AL/WX/yRRYLm9RWLY+IHutXj0260bUNOnlgkuIzctAyuqMitgxyPggyL1x1raoasMKKKKQBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABWVpv/ACPOr/8AYNsv/Rt1WrWVpv8AyPOr/wDYNsv/AEbdVtQ+MiexD4N48E6EvdbCBD9QgB/lW3WL4UGzQEg/59ri4tv+/czp/wCy1tVnLdlLYKKKKkYUUUUAFFFFABRRRQAUUUUAFc74Y/5Bdz/2Er//ANK5a6Kud8Mf8gu5/wCwlf8A/pXLVxBbmzWBr/jHS/DlzFb3vnNLIm8LEgbAzjnke/5Vvk4GT0rx/wDs+Tx5rviC/QsYreEra47sPuD8QpP41FWbiko7s9fKsJRrzlPEO1OK19W7I9dhmjuII5omDRyKHVh3BGQafXF/DTWP7Q8N/Y5GzNYt5Zz12Hlf6j8KozeKvEU3jO/0PTo7aQrlYTKMCLGCWYjrxnj3FHtVyqXcTyqr9Yq0U0uTVt6ad/u1PQqK4HQ/EuvW3jL/AIR7XjBM0ikpLEuMHbuBGMZBAPUZp+seKdZvvEr6B4bih82EHzriUZAI6+wAyB0PNHtY2uDymuqvs7q3Lzc1/d5e9zu6K4PS/FOtaZ4kh0LxNHCWuMeTcRDAJPA6cEE8dAQajvfFPiA+OLzQtNjtpOAsPmjAjO1WLMR1A54+lHtY2uNZRXc3FNW5ea99Gu9z0CivNIPFXi208QT+HbiKzu9QfCwyAbUQkbt3AGV25PTP8qvaL4k1618Zjw9rrW87SqSskK4AO0sMcDI4I5FCrRZVTJq8IuXNF2jzaPePdeX3He0Vx+rXXjG71yW00mCCysolyLq5AIk/HnHsMdqh8G+KNT1HWb7RtVMEk9spYTQ9GwwU9OD1GOlP2i5rGP8AZtT2LrKUXZJtJ3aT/D8Td0bxNYa5eXltaCYSWjbZPMUAdSOOfY1s15b4J+1fb/FhsWUXYyYtwyCwZ8DFdJ4O8Vvq/h68u9SZFnsmYzbVx8mMg4/Aj8KmnVulc6Mwyx0pzdHWMeX195J/dc66qeq6nBo+mTX9yHMMIBbYMnkgcfnXPeBdb1XxBaXd9fmMQeb5cComOnJ579QPzq149/5EjU/9xP8A0NarnvDmRzLBOnjY4ar/ADJO3nYzB8UvDxP3L0f9sh/8VXS6Pruna9bGfT7gSqpw6kYZD7g1geAbCyufA9g09pBKW80MXjDZ/eN1yK57SoU8P/F2XTrMbLS4BBjHQAx78fgR+VZqc1yuWzPRq4LB1ZV6VFOM6d3q7pqLs+mnkeo0VwEviTxD4h8QXmneGzbQW9mSHnnGdxzj0PBIOOO1WtC8Taxqaaro9xFBFrtmjeW38DkHGSPqR9c1aqxbscM8qrwhzNq6s2r6pPZtf1Y6m11Sxvru5tbW5SaW22+cEOQhOcDPTPB4q5Xkfw5Gs/25ffZTB9nEqfbt3UjL42/jmtseI/EnibWL238Nm1t7SzO0zTjO88gdj1wccfU1Ma14ptHTismdOvKnTmuWKTbb2vbf1ey7HoNFcl4W8WTaguo2msRpBfadkzFPusoyCce2P1FYtn4g8YeKWurvQ1tbWyhcoiygFnOM4yQecEeg5qvaxsrdTmWVV+ecZtRUbXbemu1n1ueiu6xozuwVFGWZjgAepqvYajaapbtPZTrNCHKb16Ejrj1+tcTe6t4h1bwBfPPYfY7qMMtx50ZRZIdpyVB7/wCe9Vfh5d6hp/h+4vbpoRokEcsgC/6zzAQT+GM0va+8l0Nv7JccNOpKS5oyStff/gvoelUV5va69411yxuNZ05bKGyiLbLdlyzheuDjn8x7VtWnjiGfwTNr0kIE0B8t4QeDJwAAfQ5B9ufSmqsWZVcpxFPRWk7qLSd2m9k/6sdVcTpbW0s8mdkSF2x1wBms/Qdfs/Edi93YiURJIYj5qgHIAPqfUVxEeq+Nb7w9c6zKllJp8kLk22Nr+XggsvHbk8n8KtfDMzL4K1E2wDTi5kMYPdvLTH60lVvJJG9TKlRw05zknJSS0ei7pnoNFYvhiXXJtMkbX4Viu/OIVV242YGDwT33Va12/OmaFfXq/fhhZk/3scfritFLS55cqElW9imm720d195FYa/a6lq99p9skrGywJZsDZuP8IOc54Pbsaxrv4keHbS7a386abacGSGPKZ+uRn8Kr/D7TFk8ESGR3Dai0pkkU/PjlOD68E/jS3um6B4J8J3sTqJhcBgonCtJKxGAvAHA/SsuabipHrLDYOOJnQalJpqKS6vZu+vXodZp+oWuqWUd5ZTLNBIPlYfyI7GrNcX8MbG4s/CzSThlW5mMsan+7gDP44/lXaVpCTlFNnm42hChiJ0oO6TtcrahqFppdlJeXsywwR/eZv5AdzXNWnxI8O3d2tv500O44EkseEz9c8fjWv4h8O2viWyjtLua4jjR/MHksBk4I5yD6muI+Ih00WuneHtNt43v1kXakSjci4ICnHc5HHtn0qKkpx1Wx35ZhcJiXGlNSc3e9tFFLrtqegatrFjoli15fzCOEHA4yWPYAdzWVonjfRdeu/slrLJHcEErHMm0vjrjkj8K5DxtDIdS8K6NcPvRUjSTnhiSqk/p+tS/EKKLS/Enh++tY0ikDYOxcZCMuOn1IqZVZJt9FY6MPleHnCEJNudRSafRcu2nW9jt9d8Sab4dgSTUJipkyEjQbmbHXA/rUWg+K9K8R+YtjK3mxjLRSLtYD19x9K4/UwupfGWytbhQ8MCDarDI4jaTp9aS9VNM+MdmbZRGtwi+YqjAO5Sp4/AH60OrK9+l7CjllB0VB39o4c9+npb063O307X7XUdUv9NVJYrqybDpIANw7MuDyOn5itWvPvETnRvibo2oR/Kt6ogmH97nbk/gV/KvQa0hJu6fQ83G4eNONOpT+Gcb/PZ/iFY3if8A5Bdt/wBhKw/9K4q2axvE/wDyC7b/ALCVh/6VxVotzgex0VFFFZEhRRRQAUUUUAFFFFABRRRQAUUUUAFFFFAGZ4a51HxI/ZtTGPwtYB/MV0FYHhP57G/uf+e+pXR/74kMX/tOt+vRh8KOd7hRRRVCCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACsHxh/yA4f8AsJWH/pXDW9WD4w/5AcP/AGErD/0rhpS2Y1uaNFFFeadAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAYN3/wAj/o//AGC77/0baVvVg3f/ACP+j/8AYLvv/RtpW9TfQQUUUUhhRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFAGDd/8j/o//YLvv/RtpW9WDd/8j/o//YLvv/RtpW9TfQQUUUUhhRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABWVpv/I86v8A9g2y/wDRt1WrWVpv/I86v/2DbL/0bdVtQ+MiexHo4+z6xr1ieAl4LiMf7EsasT/32JPyrZrJ1AfYvGdlcdI9RtWtXPrJGTJGP++WnP4VrUqqtNjg7oKKKKyKCiiigAooooAKKKKACiiigArnfDH/ACC7n/sJX/8A6Vy10Vc74Y/5Bdz/ANhK/wD/AErlq4gtyn471f8AsjwrcujYmuB5Efrlup/AZrkPD/w+1GfRre8j1+4sDcoJTDFG3APTJDjPGO1eoyQxTACWNHA6BlBxTgAAABgDoBUSpKUryPXw+aTw2G9jQVm3dvR37KzR5PosE3gf4hx6dcXBmt7xAnmldu/d904yejDHX1rR0L/kr+sf9cn/APZK9EeCGVg0kSOw6FlBIpRDEshkWNBIerBRk/jUqjbZ6Xub1c49rzSlD3pQ5W+7vva34HnV9/yWmx/65f8AtNqq/aj4J+IeoXmowSmxv95SZFz95g3H0PBH416eYYjKJTGhkHR9oz+dEsUc8ZjljSRD1V1BB/Cn7Lqnre5Mc2jaMJwvHkUGr72d7p20PMJ7v/hOvHenTabDKLGwKtJO646NuP0zwAOtWtP/AOS0X/8A1yP/AKLWvRYoYoIxHDGkaDoqKAPyFAhiEplEaCQ9X2jP50ey6t63uEs2jZ04QtHkcEr66u927annX/Nbv+2X/tGi+/5LTY/9cv8A2m1ei+TF5vm+Wnmf39oz+dBhiMolMaGQdH2jP50ey8+txLNVdPl/5d+z3/H/AIB5Je31pdeNNTg8X3V3HZwswt4FLbMA/LwvPK857+tWPADWyeP9RW2t3toHtn8iKQEMELIV688rzXqMtrbzSLJLBFI6fdZ0BI+hp3kxeb5vlp5n9/aM/nSVF817m085hKhKkoNXjy2vordUrdep518N/wDkYvEf/XQf+hvXPeKlufDXiDWbC1Ui31aNWUD0LZOPx3rj0NewzRGG2nazhiFwUJTK4DNjjP41wtp4b8Ra74qtdW8SRW8ENpgpDEwIJByAACe/JyaidNqKitzqweY054mpiqllCy91vVtWtZddV8jrvDuljRvD9lYYAaOMeZju55b9SaoePf8AkSNT/wBxP/Q1ro6a6LIhR1DKeoYZBrocfd5UeBTxMo4lYier5uZ+etzy7wr8QNJ0Lwxa6fcw3bzw78+WilTl2I5LDsas+EbS88Q+NLjxVc2zwWwz5Ab+I7dgA9QF6n1r0QWdspyttCD6hBU1Zqk9OZ6I9GtmlH97KhT5ZVL3bd9G7tJWW54rHp+l6L4o1Gz8Si7hiZi8E0JYBhkkHjqCD+BFdT4Bg0u51W8vtN028hijUxLczT7hKCRxjHXgHqcV3k9tBcqFngjlUdBIgYfrT0RI0CIqqo4CqMAUo0eV3KxWdOvRcWnzNJP3vd06pefm7HlfgLVrbR/Emq6feCRLi7uVijXb/EGcEH06isew0/SNE1nUNP8AE/2yDacwSxFgGGTzx1yMYP1r2k2tuZ/PMERm/wCemwbvz60T2tvcgC4gilA6CRA2Pzpewdkr7Gv9uR9rOag1zpJ2lreOzTt+B574P0/TtUttYl03Trq1SeCS1W4nn3iQN7Y68AnriqHhDxTbeELG90nWYLiG4jnMiqqZzkAY/Tr05r1VVVFCooVQMAAYAqKa0trhlae3ilZehdAxH50/ZNWaeqMHmsKrqQrQbhK32tVbzd7/ANWOOs9Y1XxN4H1q5urBIFe3kW3EYOZBtOeD+A9+awvCF5b6t4LvvDMLN/aDxSyKCMKemOfqRXqgAAwOBUUVrbwyM8UEUbt95kQAn60/Zu6bZEczpxhOEadryUo2ezXfv+B4npL+G7TSbiLWo9Qj1OBmAgR2UP6D/ZPY5rpk0Jb/AOGl5/Zmm3Fq80guEgll8xpAuORwOozgY5x716JJZ2s0olltoXkHR2QEj8anqY0LaM6MRnjqNThFp8ylrJtadEtNPU8w0zxvZjwUNHWCd9UW3a2SFY8hjggNn0xyfoavfDK4W08F6jcspZYrmSQgdSBGhrvFtbdZWlWCISPwzhBlvqackMUaFEiRUPVVUAGqjTkmm3sYV8yoTpTp06bXPJSet/ltt2Mnw14ig8TadJe28MkKJKYishGcgA54/wB6k8XW73XhLVIkGW+zswA77ef6VrxxRwrtijVFznCjApzKGUqwBBGCD3rSzcbM8/20IYhVaUbJNNK99vM4/wAE6nBafDmG7lP7u0SUyY68Mxx9cEV52PElnq/iRtU8SR3FxAn+ptYQCo54ByRx/M9a9L8K6Fd6Ddarp0sKvpUkvnWshYHr1Ujr0x27Guj+xWn/AD6w/wDfsVj7OUopXtY9pZhhsLia01Hm53dNOzSettnZ62fU5+x8aWF74dvdXt7W4WCyO1o2CgngdMEjvWtoWsRa9o8OowxPFHKWAV8ZGGI7fSpbuzZtPngsvJt5ZFwrNEGUH1K96bo2lxaLpFvp8LM6QqRubqxJJJ/Ek1qlK+p5VaWFlSk6cbS5tNb+7bX8TG8ceJH8OaKHt8fbLhjHCTzt45b8P5kV574U8S6Hockl/qFve3eqyMS021SEz6ZbOT3P4fX2WSCKbHmxI+Om5QcUz7Faf8+sP/fsVE6cpS5kzswmY4ehhnQnTbct2pWv5bbHnvj5JbiLQvE9rA7wRBZHBHKglXXOOncflVLVtSi8eeLNGt9Mima3tzvmd1xtBILfkAB9TXquxdmzaNuMbccY9KjhtoLYEQQRxAnJEaBc/lSlSbe+jHQzaNKnFcnvQUlF32Uu6trY858V7/DvxDsPEUsLvZOoWR0GcHaUI+uCD70zS5h4t+Jq6taRyfYLNB+8dcZwpA/Esc49BXpkkaSxlJEV0PVWGQaSKGK3jEcMSRoOiooA/IU/Ze9vpe4o5slRUeT31HkvfTl9O/zPP/G6m88ceGrJOXEgkYDspcZP5Ka9DrlbDQ7yfxxe67qMQjjiTybJdwY7cYLcdO/H+0a6qqgndt9Tnx9WLp0aMHfkjr6t3f3BWN4n/wCQXbf9hKw/9K4q2axvE/8AyC7b/sJWH/pXFWi3PNex0VFFFZEhRRRQAUUUUAFFFFABRRRQAUUUUAFRXNxHaWs1zM22KFGkc+gAyalrF8UD7RpSaWv3tTnSzx6oxzL+USyH8KqKu7Cbsi/4UtpLXwppkc67Z2t1kmHpI43P/wCPMa2KKK9E5wooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKwfGH/IDh/wCwlYf+lcNb1YPjD/kBw/8AYSsP/SuGlLZjW5o0UUV5p0BRRRQAUUUUAFFFFABRRRQAUVXvb+z022a5vrqG2gXgyTOEXPpk1jN4utpP+PHTtUvR/ejtvLU/QylAR7g4qowlLZCbSOhormz4ruwSP+ET1o+4ks//AI/Sf8JXef8AQp63/wB/LP8A+SKv2NTsLmR0tFc1/wAJXef9Cnrf/fyz/wDkij/hK7z/AKFPW/8Av5Z//JFHsanYOZHS0VzX/CV3n/Qp63/38s//AJIo/wCErvP+hT1v/v5Z/wDyRR7Gp2DmR0tFc1/wld5/0Ket/wDfyz/+SKP+ErvP+hT1v/v5Z/8AyRR7Gp2DmR0tFc1/wld5/wBCnrf/AH8s/wD5Io/4Su8/6FPW/wDv5Z//ACRR7Gp2DmRBf6raxfEbS4GfEy2N3Ds7sXktCCB6Yz/3y3pXWV5nfLcXnxF0vxSfCus5s7KWAqZLTJYn5SP3+MYeTPPp+HT/APCV3n/Qp63/AN/LP/5IpujPSyBSR0tFc1/wld5/0Ket/wDfyz/+SKP+ErvP+hT1v/v5Z/8AyRS9jU7BzI6Wiua/4Su8/wChT1v/AL+Wf/yRR/wld5/0Ket/9/LP/wCSKPY1OwcyOlormv8AhK7z/oU9b/7+Wf8A8kUf8JXef9Cnrf8A38s//kij2NTsHMjpaK5r/hK7z/oU9b/7+Wf/AMkUf8JXef8AQp63/wB/LP8A+SKPY1OwcyOlormv+ErvP+hT1v8A7+Wf/wAfqQeLI4z/AKXo2r2q/wB4wLN+kLOf0o9jUXQfMjoaKpadq+n6tGz2F5DcBDhwjfMh9GXqp9jirtZjCiiikAUVXvb+z022a5vrqG2gXgyTOEXPpk1jN4utZP8Ajx07VL0f3o7by1P0MpQEe4OKqMJS2Qm0joaK5s+K7sEj/hE9aPuJLP8A+P0n/CV3n/Qp63/38s//AJIq/Y1OwuZHS0VzX/CV3n/Qp63/AN/LP/5Io/4Su8/6FPW/+/ln/wDJFHsanYOZHS0VzX/CV3n/AEKet/8Afyz/APkij/hK7z/oU9b/AO/ln/8AJFHsanYOZHS0VzX/AAld5/0Ket/9/LP/AOSKP+ErvP8AoU9b/wC/ln/8kUexqdg5kdLRXNf8JXef9Cnrf/fyz/8Akij/AISu8/6FPW/+/ln/APJFHsanYOZEF/qtrF8RtLgZ8TLY3cOzuxeS0IIHpjP/AHy3pXWV5lerc3fxH0zxSfC2s/6HZS25QyWmdxPykfv+mHkzz6e+Oo/4Su8/6FPW/wDv5Z//ACRTdGelkCkjpaK5r/hK7z/oU9b/AO/ln/8AJFH/AAld5/0Ket/9/LP/AOSKXsanYOZHS0VzX/CV3n/Qp63/AN/LP/5Io/4Su8/6FPW/+/ln/wDJFHsanYOZHS0VzX/CV3n/AEKet/8Afyz/APkij/hK7z/oU9b/AO/ln/8AJFHsanYOZHS0VzX/AAld5/0Ket/9/LP/AOSKP+ErvP8AoU9b/wC/ln/8kUexqdg5kdLRXNjxXdk4/wCET1oe5ks//j9PHi62j/4/dM1WyHdntvNA+piLgD3PFJ0ai6D5kdDRVax1Cy1O2FxY3UNzCTjfC4YZ7jjv7VZqBhRRRSAKKKKACiiigAooooAKytN/5HnV/wDsG2X/AKNuq1aytN/5HnV/+wbZf+jbqtqHxkT2LPimzmutDkltEL3lm63duo6s8Z3bB/vDcn/AqmtLqG+s4Lu3cPBPGskbj+JWGQfyNadcvpK/2Tq17oTcRAm7sveF2+ZB/uOSMdlZK1rxuuYmm+ht0UUVyGoUUUUAFFFFABRRRQAUUUUAFc74Y/5BlyO41K+z/wCBUp/rXRVzNw58N6pdXMyMdIvX855lGRay4AbcB0RsA7ugO7OARi49gW5u0VFb3MF3Cs1tPHNEwyrxuGU/QipaosKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKxvE3OmWw7nUrHH/gVEf5CtS4uYLSFprmeOGJRlnkcKo+pNY9u58Sapa3MKMNIsn85JmGBdS4IXaD1Rck7uhO3GQDRtqSzpqKKKyEFFFFABRRRQAUUUUAFFFFABRRRQAVkWy/2l4xeTrBpMOwehuJQCfxWMD8Jat6pqEelaZPeyqziJfljX70jE4VB7sxAHuRUvh/TZNL0iOK5ZXvJWae6dejSudzY9gTgewArooRu+YzqPSxqUUUV1mQUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFYPjD/kBw/8AYSsP/SuGt6sHxh/yA4f+wlYf+lcNKWzGtzRooorzToCiiigAooooAKKKKACsbWtaexkjsbGJbjU51LJGxwkSdDJIR0UHoOrHgdyNG/vYdN0+5vrlisFvE0shAydqjJrmtItp0ikvr4f8TG9IluOc7P7sY/2UHHvyepNdGHpe0lrsiZOwW2kRpci+vpWvtR/5+Zh9z2jXpGvsOvck81pUUV6aSSsjIKKKKYBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQBRvtJtb6RJ2DQ3cYxFdQNslj+jDt7HIPcGrOk6zcperpWrlPtTAm2uUXal0ByRj+GQDkr0I5HGQstVNTsF1GyaDzDFKCHhmX70Ug5Vx7g/n06Gsa1FVF5jTsdNWNretvYyR2FhEtxqc6lo42OEiToZJCOig9B1Y8DuQlj4gik8LtrF8BCbaOQ3aLzsePIkA9eVOPUY9azNHtZ0hkvr5R/aN6RLcc52f3Yx/soOB6nJ6k1w0KPPLXZFylZaBbaPGl0L6+la+1H/n5mH3PaNeka+w69yTzWlRRXppJKyMwooopgFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFAGdd6PDLcm9tJHstQA4uoMBmx0Djo6+zZ9sHmtHRdbku5n07UY0g1OFd5VM7J06eZHnnGcAg8qTg5BBJVDVbGW6gSa0dY9QtW821lPQOP4T/ALLDKkeh9cVhWoqovMalY6qiqWk6lHq+lW99ErIJVy0bdY2BwyH3VgQfcVdry3obBRRRSAKKKKACiiigArK03/kedX/7Btl/6Nuq1aytN/5HnV/+wbZf+jbqtqHxkT2OirG8RadPdWsN7YKDqVi/nW4Jx5gxh4ifR1yPY7T2rZorsaurMxMnT7+DU9PhvbZiYpl3DcMEeoI7EHII7EEVZrDvk/4RrVX1BeNIvpAbodraY8CX2RuA3ocN3Y1uVwVIODsbxd0FFFFQUFFFFABRRRQAUUUUAFFFFAGPc+E/Dd7KZbvw/pU8hOS8tnGxP4kVD/wg/hL/AKFbRP8AwXxf/E1vUU7sVjB/4Qfwl/0K2if+C+L/AOJo/wCEH8Jf9Cton/gvi/8Aia3qKOZ9wsjB/wCEH8Jf9Cton/gvi/8AiaP+EH8Jf9Cton/gvi/+Jreoo5n3CyMH/hB/CX/QraJ/4L4v/iaP+EH8Jf8AQraJ/wCC+L/4mt6ijmfcLIwf+EH8Jf8AQraJ/wCC+L/4mj/hB/CX/QraJ/4L4v8A4mt6ijmfcLIwf+EH8Jf9Cton/gvi/wDiaP8AhB/CX/QraJ/4L4v/AImt6ijmfcLIwf8AhB/CX/QraJ/4L4v/AImj/hB/CX/QraJ/4L4v/ia3qKOZ9wsjB/4Qfwl/0K2if+C+L/4mj/hB/CX/AEK2if8Agvi/+Jreoo5n3CyMH/hB/CX/AEK2if8Agvi/+Jo/4Qfwl/0K2if+C+L/AOJreoo5n3CyMH/hB/CX/QraJ/4L4v8A4mj/AIQfwl/0K2if+C+L/wCJreoo5n3CyMH/AIQfwl/0K2if+C+L/wCJo/4Qfwl/0K2if+C+L/4mt6ijmfcLIwf+EH8Jf9Cton/gvi/+Jo/4Qfwl/wBCton/AIL4v/ia3qKOZ9wsjB/4Qfwl/wBCton/AIL4v/iaP+EH8Jf9Cton/gvi/wDia3qKOZ9wsjB/4Qfwl/0K2if+C+L/AOJo/wCEH8Jf9Cton/gvi/8Aia3qKOZ9wsjB/wCEH8Jf9Cton/gvi/8AiaP+EH8Jf9Cton/gvi/+Jreoo5n3CyMH/hB/CX/QraJ/4L4v/iaP+EH8Jf8AQraJ/wCC+L/4mt6ijmfcLIwf+EH8Jf8AQraJ/wCC+L/4mj/hB/CX/QraJ/4L4v8A4mt6ijmfcLIwf+EH8Jf9Cton/gvi/wDiaP8AhB/CX/QraJ/4L4v/AImt6ijmfcLIwf8AhB/CX/QraJ/4L4v/AImj/hB/CX/QraJ/4L4v/ia3qKOZ9wsjB/4Qfwl/0K2if+C+L/4mj/hB/CX/AEK2if8Agvi/+Jreoo5n3CyMH/hB/CX/AEK2if8Agvi/+Jo/4Qfwl/0K2if+C+L/AOJreoo5n3CyMH/hB/CX/QraJ/4L4v8A4mj/AIQfwl/0K2if+C+L/wCJreoo5n3CyMe28J+G7KUS2nh/SoJAch4rONSPxArYooovcYUUUUgCiiigAooooAKKKKACiiigAoorI1e9uHni0fTHA1G6UkyYyLaLo0pHr2Ud29gxFRi5OyE3YZCv9veIg3XTtJk69prrH6iMH/vpvVK6eq2nWFvpenw2NomyCFdqgnJPqSe5JyST1JJqzXfCKirIwbu7hRRRVCCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigArB8Yf8gOH/sJWH/pXDW9WD4w/wCQHD/2ErD/ANK4aUtmNbmjRRRXmnQFFFFABRRRQAUUUUAc94ubzLKwse15fwxt7qmZSPoRER9DU1QeKP8Aj+8Of9hJv/SW4qevSwi/dmU9wooorqJCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKAOVvX2397o38F9qVncEf7JVmYfQ/ZWz/vH1rqq5K//AOSgWf8AvWv/AKJ1GutrOCSbt3AKKKK0AKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAr+G2+z6rrVgPuCWO7Qf3RKpBH4vG7fVjXR1zWjf8jlq/wD2D7P/ANGXNdLXk11aozWOwUUUViUFFFFABRRRQAVlab/yPOr/APYNsv8A0bdVq1lab/yPOr/9g2y/9G3VbUPjInsdFRRRXaYjJYo54XilRZI3Uq6MMhgeCCO4rlomk8L3Men3bs+kSsEsrpzkwE9IZD6dkY9funnBbrKiubaC8tpba5iSaCVSkkci5VlPUEd6icFJWY07FeisB2uvCny3LS3ehj7lycvLZj0k7tGP7/UfxZGWrdjkSWNZI3V0cBlZTkEHoQa4pwcXZm6aY6iiioGFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFZN/q8guzpmlQrd6mQCyE4jt1PR5WHQei/ebsMZIaTk7ITdiTVdVNkYrW1h+06lc5FvbA4zjq7H+FFyMt9AMkgG3oujjSoJHml+0X9y3mXVyVwZG7AD+FQOFXsPUkkmj6LHpYlnlma61C4wbi7cYZ8dFA/hQZOFHTJ6kknUrtp01BeZjKVwooorUkKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigArB8Yf8AIDh/7CVh/wClcNb1YPjD/kBw/wDYSsP/AErhpS2Y1uaNFFFeadAUUUUAFFFFABRRRQBzvij/AI/vDn/YSb/0luKnqDxR/wAf3hz/ALCTf+ktxU9enhP4ZlPcKKK8h8Xa5r/gLx/aajPf3V14cvXO6F23LHn76j3Gdy+3HY10N2JPXqK8t+JnjW8RtL0Dwvcs2pagUlEtu3Ijb7gB7buufQe9dlBe2vhDw3aL4h1tWmVcSXNzJzK/U7R1I9AO1FwOgorN0fxDpGvxPJpWo292qffEbZK/UdRVbV/F/h7QbgW+p6tbW05GfKZssB6kDJFFwNuiqOlazput2v2nTL2C7hBwWicHafQ+h+tTXt9a6bZy3l5OkFvEMvI5wFHTmmBYorPXXNLfRv7YW/gOnbS32kP8mAcE5+vFFvrml3ekvqtvfwSWCKzNcK/yKF+9k+2KANCis2DxBpFzpL6rFqNs2npndc+YAgx15NVtK8YeHdcuja6bq9rcXGCRErYY+uAev4UrgbdFFZWseJNF8PhDq2pW9oZPuLI3zMPUAckUwNWisvR/EmjeIFc6VqVvd+X99Y2+ZfqOorzvTfFV5b/GTXbTUtXaPR7WF3Ec0m2KPGzB56dT+dJsD1iisjR/FOh+IHkj0rU7e6kjGXRG+YD1wece9S6vr+k6DCsuq6hb2iv93zXwW+g6n8KLgaVFZmj+ItH1+N30rUbe7CffEb/Mv1HUU4a9pR1o6ML+A6kBk2275wNu7p9OadwNGis6517SrPVYNLub+CK+nAMUDNh3BJAwPqD+Vef3mvaqnx6stHW/nGnPFlrYN8hPksen1ANJuwHqNFZOs+J9E8PmMatqdvatIMojt8xHrgc496881bxZdXHxf8P2ul6w0mkXUMbMkEmY5MmQHp9B+VDdgPWaKKKYHJX/APyUCz/3rX/0TqNdbXJX/wDyUCz/AN61/wDROo11tRHd+oBRRRVgFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAVdG/5HLV/wDsH2f/AKMua6Wua0b/AJHLV/8AsH2f/oy5rpa8rEfxWax2CiiisCgooooAKKKKACsrTf8AkedX/wCwbZf+jbqtWsrTf+R51f8A7Btl/wCjbqtqHxkT2OiooortMQooooAK5ubQLnS5HufDrxxoxLSabMSIHPcxkZMTH2BU91yc10lFJxTVmNOxz9hrlte3Bs5Uls9QUZazuQFkx6ryQ6/7Skj3rTpdR0qx1a3EF/axzxg7l3DlG7MpHKn3GDWO2k63pnOm36ahbjpbaixDgeizKCf++1Yn+8K5p0H9k0VTua9FYh8S29p8usWl1pTDq90mYfr5qkoPxIPtWtb3MF3As1tPHNE33XjcMp+hFYOLW5aaZLRRRUjCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKybnxLpNvO1st2Lm6XrbWimeUfVEBI+pwKaTewXNaq97f2mm2rXN7cR28K8F5GwM9h7k+lZ4fxDqfFtZxaVAf+W16RLNj2iQ7R9Wfjuvarth4asrO6W9naa/1Bel3dsHdfXYAAsY/wB0DPfNbRoSe+hDmuhnq2r6/wAWyzaTpx63EqYuZR/sIf8AVj/acbv9kcGt3TdMs9JtBbWUAijyWbklnY9WZjyzHuSSTVuiuqMFFWRk23uFFFFUIKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKwfGH/IDh/7CVh/6Vw1vVg+MP8AkBw/9hKw/wDSuGlLZjW5o0UUV5p0BRRRQAUUUUAFFFFAHO+KP+P7w5/2Em/9Jbip6g8Uf8f3hz/sJN/6S3FT16eE/hmU9wrB8ZaDaeI/Ct9YXY48syRvjmN1BKsP89CRW9UVzEZ7SaEEAyIygntkYrpJPE/gXolveXd/r1yTLc2u23gDc7AV5P1xhR6DNVvGF1JqPxse2u9Jn1i3sY1WLT4zjePKD5x6bmyfXFei/DnwRc+CdPvba5vIrk3EqyAxqRjAx3qDxh8PZ9Z1228RaHqf9mazAAvmFcq4HAJ9Dg46EEcYrPlfKI43SNN1eD4oadrGk+Eb7RdOkAhu4Sv7vByGbA4A+6ceq5qtr2geIPDXjfVtak8L2/iPT72VpFM0Hn7FJyAAMlSBxkjGBXb+GvAesW3ic+IvEuvNqF8q7UjhBSMcYBIGAcZPGAM81Wvfh5r+neJLzWfCXiJbI3hJlguk3qMnOASDxnOOOPWi2gFL4U6t4XudX1GDTNJudJ1WRTJNbyTM8ZUN0UcAYJ6EAgH616XqmnxatpN5p8/+quYXib2DAjNcj4M8Az6DrV7r+sal/aOsXalXkVNqKCQTgdycDsMDjFdzVRWmoHzTHrlzb/Da98FNn+0Bqy24izzsJJIH/bRP/Hqe+szeGvAnibwbLJm7XU1hTHVkOdxA9D5Q/wC+69KufhYJ/iYnidbuJbP7Qty1tsO7eAO/TlhmjWvhYNW+IsXiQ3cS2nmxSzWxQ5dkA79MHaP1qOVgcR8Q7KfQvD/gzwv5cjQBDJcQxnBllJXI+uWbH+9S+JNH1C8OnXHhvwBqGi3tlIGE0YHzAdM46kEDk+9eq+OvBFt410yGFpza3lsxe3uAudpPUEehwPyFcxH8OfFOrajYv4o8VG6s7I5jjtgUdx6FgB1xyeTTcdQPTbd3ktopJEKOyAsp/hJHIrx34geGtftvHp8T2miQ69YtEq/ZZYvOCYXaQU6nuwIBGTXswGBgV5/4l8AandeKF8S+G9cOn6jt2uky7424xx1xkAZGCO9VJXQHL+ANc8MT+OsP4cn0LXJlZEjSRhCfl5XZhduQM4IIyPXFVLPRLDXvj9q9tqMCz28W6YxP91yFQDI7jnOPauw0D4e6mni1PE/ijWE1HUYV2wpDFsReCATwOgJ4wOeauaX4GudP+Jeo+KXvInguo2RYAp3Lnb1PT+GpswOKGnWmgftDWFtpcCWtvLEWaKIYXmJ8gD0yAcUzS9PtvGvxt1yPXk+0QWCyLDbuTtIR1RRj05LY7k13F74Gubr4oWfixbyJYIIwhgKncfkZevT+Kqfij4bXN/4j/wCEk8Oas2laqf8AWZBKOcYzx0yOowQf5lmByXibTbXwT8W/Dsvh+P7ML1o1mtoydpDSbGGPQjt6jNXYv+Tl5/8AriP/AEmWt7w38Nru38SL4j8UaudV1OP/AFICkJGR0PPXHYAAA80eLPh1qWpeLofE/h7V00/UVUK/mJkZA25B5/h4IIxRZgYPjUg/HrwuAekMAP8A38kpb7/k5TT/APrj/wC0HrTh+F+qf8JhpfiK+1/7ddQOsl00se3eQeAgHCqBj9a2LjwNczfFO28XC8iFvCm0wbTuP7tk69O+aLMDh/D2m2ni342eIzrkK3SWZlEMEvK4SQRrkdwB29TmodQ0TT9B+Pej2+mwpBBKY5jCn3UYhgQB2HGce9dd4k+G+oT+Kj4l8L6wNM1CT/XK6kqxxgkYz1xyCCCear2Xwt1K38Z6b4ju9e+3XETCS7aVCGkfkYXHAUDaAPbt0oswPT6KKK0Gclf/APJQLP8A3rX/ANE6jXW1yV//AMlAs/8Aetf/AETqNdbUR3fqAUUV4T468ea54Z+Kky297M+n27Qs1mW+R1Malhj3yefXmqbsB7tRXA/EHxJJ/wAKxOuaFfSRecYXimiOG2swyPY9iK6DwjfSXPgjSL6+uC8j2aSTTSN1O3JJNF9bAb1Fc5F4+8Jz3otI9fsTMTtH7zCk+zdD+ddHTuAUVFdXMVnaTXU7hIYUaSRj/CoGSfyFfPejfErxCnjC01jULu5GhXd7JF5Lt+6VeMgf7gdTUuVhH0TRUc88VtbyXE0ipDEhd3Y8KoGST+FYc3jnwvb2Md7JrlmLeViqOJM7iOuAOTiquM6CiqunalY6vZJeafdRXNu/3ZIm3D6fX2rK1Lxt4Z0i8Npfa1aQ3CnDR79xU++M4/GlcDfoqtDqFnc2Av4LqGW0KFxNG4ZNo6nIrHl8deFobFb19csxbu5jVxJncwxkADk4yPzouB0NFYdz4x8OWmmwajPrNmtpPnypBJnfjrgDk47+laOm6nY6xZJeaddRXVu/AkibIz6ex9qdwLdFc1c/EHwlaXTW02vWYlU7WAYsAfTI4qv491iS3+HGo6rpF7hhHG8NzbuDwZFGQR7E0rgdbRXCeDvF1pbeANEvfEWrxx3F2suJbl8GQrIw6+wxXZX+o2el2Ml7fXMdvbR43yyNhRk4HP1IoTAs0VXsb+11OyjvLKdJ7aUEpIhyGwccfiK43x3JfJrWhi18UW+kIZD5kEsrKbn5l4AAOe4/GhsDuqKzNY8Q6PoEaSarqNvaB/uCRuW+g6mmaP4n0PxAXXStTt7p0GWRG+YD12nnHvRcDWooopgVdG/5HLV/+wfZ/wDoy5rpa5rRv+Ry1f8A7B9n/wCjLmulrysR/FZrHYKKKKwKCiiigAooooAKytN/5HnV/wDsG2X/AKNuq1aytN/5HnV/+wbZf+jbqtqHxkT2OiooortMQooooAKKKKACiiigArGufCmhXU7TnTYobhvvT2pMEp+rxlW/WtmigDnz4Yli/wCPPxBq9uvZGkjnH5yozfrSHSPEKf6rxBbN/wBfGnbv/QZFroaKh04voPmZzjab4rJ+TXNFA99IlP8A7c03+zPFv/Qe0T/wTS//ACVXS0UvZQ7D5mc1/Zni3/oPaJ/4Jpf/AJKo/szxb/0HtE/8E0v/AMlV0tFHsodg5mc1/Zni3/oPaJ/4Jpf/AJKo/szxb/0HtE/8E0v/AMlV0tFHsodg5mc1/Zni3/oPaJ/4Jpf/AJKo/szxb/0HtE/8E0v/AMlV0tFHsodg5mc1/Zni3/oPaJ/4Jpf/AJKo/szxb/0HtE/8E0v/AMlV0tFHsodg5mc1/Zni3/oPaJ/4Jpf/AJKo/szxb/0HtE/8E0v/AMlV0tFHsodg5mc1/Zni3/oPaJ/4Jpf/AJKo/szxb/0HtE/8E0v/AMlV0tFHsodg5mc1/Zni3/oPaJ/4Jpf/AJKo/szxb/0HtE/8E0v/AMlV0tFHsodg5mc1/Zni3/oPaJ/4Jpf/AJKo/szxb/0HtE/8E0v/AMlV0tFHsodg5mc1/Zni3/oPaJ/4Jpf/AJKo/szxb/0HtE/8E0v/AMlV0tFHsodg5mc1/Zni3/oPaJ/4Jpf/AJKo/szxb/0HtE/8E0v/AMlV0tFHsodg5mc1/Zni3/oPaJ/4Jpf/AJKo/szxb/0HtE/8E0v/AMlV0tFHsodg5mc1/Zni3/oPaJ/4Jpf/AJKo/szxb/0HtE/8E0v/AMlV0tFHsodg5mc1/Zni3/oPaJ/4Jpf/AJKo/szxb/0HtE/8E0v/AMlV0tFHsodg5mc1/Zni3/oPaJ/4Jpf/AJKo/szxb/0HtE/8E0v/AMlV0tFHsodg5mc1/Zni3/oPaJ/4Jpf/AJKo/szxb/0HtE/8E0v/AMlV0tFHsodg5mc1/Zni3/oPaJ/4Jpf/AJKo/szxb/0HtE/8E0v/AMlV0tFHsodg5mc1/Zni3/oPaJ/4Jpf/AJKo/szxb/0HtE/8E0v/AMlV0tFHsodg5mc1/Zni3/oPaJ/4Jpf/AJKo/szxb/0HtE/8E0v/AMlV0tFHsodg5mc1/Zni3/oPaJ/4Jpf/AJKo/szxb/0HtE/8E0v/AMlV0tFHsodg5mc1/Zni3/oPaJ/4Jpf/AJKo/szxb/0HtE/8E0v/AMlV0tFHsodg5mc1/Zni3/oPaJ/4Jpf/AJKo/szxb/0HtE/8E0v/AMlV0tFHsodg5mc1/Zni3/oPaJ/4Jpf/AJKo/szxb/0HtE/8E0v/AMlV0tFHsodg5mc1/Zni3/oPaJ/4Jpf/AJKo/szxb/0HtE/8E0v/AMlV0tFHsodg5mc1/Zni3/oPaJ/4Jpf/AJKo/szxb/0HtE/8E0v/AMlV0tFHsodg5mc9/ZfiVvva5pqj/pnpbg/rOaX/AIR/Upf+PjxNfj1W2ggjB/76Rj+RroKKapwXQXMzBHg7SJOb1bnUM9VvrqSZD/2zY7P/AB2ti1tLaxgWC0t4reFfuxxIEUfQDipqKtKwgooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigArB8Yf8AIDh/7CVh/wClcNb1YPjD/kBw/wDYSsP/AErhpS2Y1uaNFFFeadAUUUUAFFFFABRRRQBzvij/AI/vDn/YSb/0luKnqDxR/wAf3hz/ALCTf+ktxU9enhP4ZlPcKKKK6SQooooAKK4L4u6tqGi+DEutNu5bWc3caeZE2DtIbI/QV0Wi6nHF4K0vUtTu0jU2EMs9xO4AyUUkkn1JpX1sBt0Vh6V4y8O65dfZdN1e1uLjkiINhj9AcZ/Crmra5pehWwuNUvoLSJjhTK2Nx9AOp/Ci4GhRWJo/i/w/r85g0vVra5mAz5athseoBwTUl34p0Kwvbizu9VtYbm3TzJY3kAKLgHJ/Aj86Lga9FYI8a+Gjpkeo/wBs2gs5JDEsrPgFwMlee+DmkufG3hm01M6dca3Zx3QbayGT7rehPQH6mi6A36KOoyK8y+L/AIzm0HR4rDSdRFvqU0q+aIz+8SLDcj0yQOaG7K4HptFed+CtSsdC8PS6jq3jf+1YJ5I4vNuJDtgl2FjGCWOTg57dK6y18VaFe6udJtdUtpr8Akwo2TwMn2yB2oTA2KKwNT8beGdHvDaX+tWkNwpw0e7cVPvjOPxrWg1GyurAX9vdQzWhUuJo3DJgdTkUXAs0Vz0vjrwtDYLevrln9nZzGriTO5gASABycZH51LceMvDdppkGpTazZraT5EUgkzvI64A5OO/pRdAblFVNN1Sx1iyW8067iurduBJE2Rn09j7VbpgFFFFABRRRQByV/wD8lAs/961/9E6jXW1yV/8A8lAs/wDetf8A0TqNdbUR3fqAV4jquk2uu/H2/wBMvU329zabG9R/o4wR7g4I+le3VxK+B7lfio/i77ZF9naPZ5G07v8AVBOvTtmnJXEeL6rf3/hjw/rngPVAzCOdJbV8cfeBOP8AZYfMPQ59a6vxvqV1afBTwtaQMyRXccSzFf4lVMhT7E4P4V2/xF+HMfjYWtxbXEdrfwZQyupIePrtOPQ8j6mr1z4EtdU8AWXhjUpdzW0KKlxEOUkUYDAH8Rj0JqeV6gcz4g+HPhiz+GNzNBZxrdW1l9oS8BO93C7uT3DdMdOeK2vhFqV1qXw+tGu3Z2gkeBHbksinj8s4/CucPws8WXNjHol54w36HGQBGqNuKjoMenoCxA/CvTtF0ez0DR7bS7BCltbrtXJyT3JPuSST9aaWtwOH+NHiD+yfBv8AZ8T4uNSfyuDyI15c/wDoI/4FXk2p+JvDt38NNP8ADtvZ3qajaSCbz2RNjSEnfzuzjnjj+Fa9j8SeALvxL4707Wbu+h/s2x2bbTaSzbTuOe3Jxn2FdxJa28sbRvDGyMCrKVGCDQ4tsDz7w34h/wCEi+DF7NI+65ttPntp8nncsZwT9V2n8TXOfCDwfoet+Eb281Swiu5pLh4FaUZ8tAqn5f7pyxORzXQeGPhtqHhu38Q2MepwyWGqW8kUSFG3RsQQrH8GOce1b3w+8Jz+DfDr6ZcXUdw7XDTb41IGCFGOfpQk9LgeY/DDUrrSvh/4zntnbzLVPMi/2X2MN36A/hWt8KfBXh/XPCEuparZx313czyK7ysSUA7Dng989ea6fwF8PpPCljq9pqFzBew6gVDKikDbhgQc+oasJPhb4k0Ka5h8K+K2s9PuWJaGUHcv0Izk47jBpJNWAxvh9JLpmqeOfDsEzy6bbQ3DR7jkAoxQH6kdf92k+EnhPRta8Iaxe6lYxXU3mvAjSjPlqIw2V9Dluo54r0Dwt8PbXwv4e1CyhuDcX9/Gyz3brjJIIAA7AZJ9Tn8l8A+C7jwf4cvNLuLuK4e4naUPGpAAKKuOf92hR2uB538GPCuj67perXOq2MV4yusMYlGRGCuSV9CcjnrxU/wduLaDwl4rXULh4tPiwZXViCqlGDEY5BwB056V3fw58E3PgnTb21ubyK5a4mEgaNSAMDGOazPCvwwfR/D2vaRqN8k8WqgLvhUgx4Dc898kH8KEnoBwbRaPdeD9UHhrwJcXNksUrnV9QkRXi2gksp77cdAe3Oav6U7v+zhqgZiQkxVQew81D/MmtzTvhj4ot9Hl8PTeLETQm3fuoIPnYHnGSMqCeSAT39a09P8Ah1f2nwz1HwnLqFu0txLvjnVG2qNyNgj/AICfzoSYHNSaB/bn7PFg8abrixWS6jx1wsj7x/3ySfqBWV4i8UzeK/h/4T0C1ffqF7KIZxnnMZ2Lu/3iQ34V6voNnZeCfBdjpWs6lZokKvG0sziNJMszYG488Hp7V5P8JPD9rf8AxCv9Std0mmaY0htncfeLErHn325P1Aoa2QHuul6fDpOlWmn24xDbRLEnuFGM15P8ZP8AkbPCH/XZv/Rkdex1xHjjwNc+K9Z0S+gvIoF09yzq6kl/mU8Y/wB2qktAOW+JHhnX/wDhNbbxNYaRDrlnHCEazmj80KRkEGPOSOcjGeevvS8F674Zk8e2/wBr8MTaDrcn7uJYnZIdxBGDHhduenQjNdh4r8A3+p+I4fEega02napEu0iRd8bcYzjnHHBGCD/OvpHw81afxZbeJPFmtx6jeWigQRwRBEGMkZOBwCScY696mzuB6LRRRWgyro3/ACOWr/8AYPs//RlzXS1zWjf8jlq//YPs/wD0Zc10teViP4rNY7BRRRWBQUUUUAFFFFABWVpv/I86v/2DbL/0bdVq1lab/wAjzq//AGDbL/0bdVtQ+Miex0VFFFdpiFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAVg+MP+QHD/2ErD/0rhrerB8Yf8gOH/sJWH/pXDSlsxrc0aKKK806AooooAKKKKACiiigDnfFH/H94c/7CTf+ktxU9QeKP+P7w5/2Em/9Jbip69PCfwzKe4UUUV0khRRRQB5r8cf+RBj/AOv2P/0F64z4i3lw/hbwFpYWR7WayikkijODKwSNQPqATj/er1Xx94Ul8ZeHV0uG6S2YTrLvdCw4BGMD61Q8Q/Dq38Q+D9K0ia7MN7pkCRwXaJkZVApyM9DtHfPAqGm7iPN/Emj6hejTp/Dnw/1DRb6ykDLNGB8wHTOByQQDk+9dX4/u/DUuu6Kuq6Vf6prnkq0Wm27fJyScOD6nPTnjninx/DfxPq2oWL+KPFRurOyOY47dSruOOCwA5OOTya0PGPw9v9X8TWfiTQdVSw1O3QJ+8TcpxnBHXsSCCCCKVmB54I7i1+MXh2RvDkXh4ytGVtIZFbcpZlLHbgAnkYwOlX9Z0W08QftDSaffIZLR1R5Y8kbwtuGA47ZArph8Mtam8UaX4j1DxIt5qFtKrz74NqFVOQiAdBjP4nOK2V8Czr8U38Yfbo/JZNv2fYd3+qCdc+2aOVgcX8ZdG0/QfB2kWOmWy21st67iNSTyVJPXmnfEjwboejfDO1ubOwiju4Xh3XAH7yTcPm3N3yeea7b4ieCZvG+mWdpDex2pt5jIWdC2eMY4NWvGfhSXxT4SGixXSW77o281kLD5fam47gW/BUrzeB9CkkYs5sIck9T8grzn486bZppWn6ktugvZLgQtN/EUCsQv0zXqWgaY2jeHtO0x5BK1pbpCXAwGKgDOKwPiH4Lm8baTa2UN5HatBP5pZ0LZ+UjHB96bV42A4b4p6Pp2jfC3S4tOtI7aOW/ildYx95zC4J+vArf0/wAN6X4R+Hj+INMswuspo7SG6LMWLsgYnBOOv6Ctjxx4Km8W+FrHR4r2O3e2mSUyOhYNtRlxjP8AtV0trp0cWhwaZcBZo1tlt5ARw4C7Tx6GlbUDyj4UeC9A1zwlPqerWcd/d3Nw6O8xJKAY4HPBOc5681S+HryaXrnjfw5bzPJptvDcNHuOQCjFAfqQefXbWyvwt8R6FcXMfhTxU1np9y2WhlB3J9CM5PuMGul8KfD618LaDf2kdybi/v42We7dcZJBAAHYDJPXJ/kknoB5/wDCLwpo2t+FdXvNTsYrqXzWgQyjPlqEByvoct168VH8F/C2j67p+rXOq2MV4UdYY1mGQgIJJA7E8c9eK9G8A+C5vBugXmmzXkdy1xO0odEKgZVVxgn2pvw78ET+CNPvbae9jujcSiQMiFduBjHJpqOwHI/ARmWz1+DcTGk8RUHsSGBP6D8q9hrifh74Fn8FLqYmvo7r7Y6ONiFdu3d6n/artqcVZAFFFFUMKKKKAOSv/wDkoFn/AL1r/wCidRrra5K//wCSgWf+9a/+idRrraiO79QCiiuZufiH4RtLtrWbXrQSqdpCsWAPuQCP1q7gdNRUVtcwXltHcW00c0Ei7kkjYMrD1BHWpGZUQu7BVUZJJwAKAForm1+IHhJ737Iuv2Jlzt/1ny5/3vu/rWzqGp2WlWLXt/dRW9quN0sjYUZOBz9TSuBbornrzx14XsDALnXLNDOiyRjfnKkZBOOgI9a3IrmCa1W6inje3ZN6yqwKleuc9Me9O4EtFc7b+PPCt1qAsYddsnuC21V8zhj6Buh/A10VABRSMwVSzEAAZJPauc/4WB4S+2/ZP+EgsfNzt/1ny5/3vu/rRcDpKK5nxz4mi8N+FLu7ju4YrySJhZ7yDvfHGB3xnNeaeB/EUltpk3iLVvHMk9ysEx/sq4kYgPyEJy3cjgAdxUuVnYR7jRXiPww1HU/EutpqGpeNbn7Stw3/ABKSxxMgXOQNwCjJ7L/DXt1NO4wornZ/HnhW21A2M2u2SXAbaymThT6Fug/E1oanr+k6NBDPqWoQW0UxxG8jYVuM8H6UXQDde8O6T4msUs9YtBc26SCVV8xkwwBGcqQehNSaNoemeH7EWWlWcdrb53FUycn1JPJPuasXV/aWVi99c3EcVqi72mZsKF9c1Xh13SrjSDq0eoW508Ak3JcCMYODyffijQDQorC0rxn4c1u7+yadrFrPcdog2Gb6A4z+FXb3XNL02+trK9v4ILm6IEEUjYaQ5xx+NFwNCisC28b+GLvUxptvrdnJdltiosnDN6A9CfYGuV+JfxDm8M3dhpmlz2wuZnP2p2+ZrdPlxx05BPJz0obS1A9Joqrp+pWerWSXmn3MdzbPkLLGcqcHB/UVapgVdG/5HLV/+wfZ/wDoy5rpa5rRv+Ry1f8A7B9n/wCjLmulrysR/FZrHYKKKKwKCiiigAooooAKytN/5HnV/wDsG2X/AKNuq1aytN/5HnV/+wbZf+jbqtqHxkT2OiooortMQooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACsHxh/wAgOH/sJWH/AKVw1vVg+MP+QHD/ANhKw/8ASuGlLZjW5o0UUV5p0BRRRQAUUUUAFFFFAHO+KP8Aj+8Of9hJv/SW4qeoPFH/AB/eHP8AsJN/6S3FT16eE/hmU9wooorpJCiiigAooooAKK8I+JXjfXz4vvbPw/fXMFrpcK/aDA3G7cAzH6F1X8K9f8La5H4j8MafqyYzcRAyAfwuOGH4MDUqV3YRsUVjy+K9AhmvIZdXtEksxm4VpADHzjn8SBS6N4o0PxCZF0nU4Lp4+XRDhgPXB5x707jNeisvWfEmjeH40fVtRt7Tf9xZG+Zvoo5P5U7R/EOka/C8ulahBdqn3xG3K/UdR+NFwNKisebxVoNvdXdtPq1pHPaJvuEeQAxjIGT+LAfiKisvGfhvUbO5u7XWbR4LYZmcvt2DsSDg4ougN2isjRfFOh+IXkTSdTguniGXRDhgPXB5x71HrPi/w94fmEOqarb20xGfKJLPj12jJFF0Bt0Vn6Rrul69bNcaVfwXcanDGJslT6EdR+NebfDDxZeXEviq41/VpHtLKSLa9w+ViBaQH6dF/Ki4HrNFVdO1Ky1ayS80+5jubZyQssbZBwcH9RUGl6/pOtmYaZfwXZgIEvlNu2E5xn8j+VMDRooooAKKKKACiiigDkr/AP5KBZ/71r/6J1Gutrkr/wD5KBZ/71r/AOidRrraiO79QKOtS2cGh30moTNDZiB/OkViCqYOcEc5x0xzXhbRaRd+DtT/AOEa8Bz3FisUrnV9QkRXjCgksp6nbjoD27817nrekwa7ol5pdwzLFdRGNmXquehH0PNeb6d8MPE9vo8vh+bxYqaE279zBAN7Z5xk8qCeSASOvrRJNiNT4LO7/DuEMxIS5lCg9hnP8yag+N2o3Vj4Hjgt2ZEu7pYZmHdNrNt/EqPyNdF4C8LXPg/w82lXF3HdYnaRHjQrgEDgg+4P51o+JvDll4q0ObSr8MI5MMrp96Nx0Yf56E07PlsBwV58OvDEfwskmSziF1Hpxuhe5+cyCPfnPoT26YNcvFqV1qH7Ot6t07P9lvEt4mbklA8ZA/DcR+Fb/wDwqvxY9guhy+Mc6GpA8sRtu2g9MZ6e27FdXq/gC2uPh4fCelzC1iGwiWQbiSHDEtjGSSDU2YHE23g7Q/8AhR0mpPYRPqDWTXP2phmQMMkAHsAABjpWNNql3a/s8WkcUjhZ79rZyD0j3OxH0JGPxr1mPwpKnw4Phb7Uhl+xtbefsO3JB5xWdpnw5gi+Hb+E9SuROrOzieJdpRi25SAfT9afKB5tNoLal4CtbHT/AIfXa3rQxyR6mrKWkY4JYnqVYZ47ZHpXcTeKvEnhTwDoDzaHPfak6mG4ibcWQJkKx2g8kAVnf8Kv8WzadFoNx4vU6FEwKosJ34ByB68dhuIH4V6jpthFpem21jAXaKCMRq0jbmOO5Pc0JMDxbxf491zW/hzem50mXSjJexWzZ3AvGyux6gd0A/Gt9fh14YPwrExs4vtR037X9tz8/meXvzn+7nt0xXf+I/D9l4n0O40q/DeTKAQ6feRhyGHuK83/AOFV+LBp/wDYX/CY/wDEizjy/LbdtznGM9Pbdihp+oGFo23XPgPqcmpRC4k0qWSOylk5MYwh4+m4j6YrX8GeFdCvfg9JqVzpkEl49tcs0zA7iUZ9p/DA/Ku4m8D2sPw+uPCumSeRHJEU86QbiWJBLNjqTTvD3hOXRPAB8NvdpLJ5M8fnKhA/eFiDj23UKIHF/AvR9Ol8PT6u9pG2oRXskSXBHzKnlpx9OT+ddr8Rb+40z4f6xdWjMkwhCKy9VDMFJH4Maj+H3hCbwXoE+mzXaXTSXLT70QqACqrjB/3f1rotS0+21bTbnT7xN9vcRmORfYjt700tLAeA+HNLjvPh+bePwFc6hcXSSbdUR1yHyQpUnkBcAY74PrXT2nhXWdV+Cl1pGr2c0WoWEry2SSj5iqgMAPqGdRUyfC7xXYWFxoemeLUj0OcndFJF84U9R+PfBAPpzXovhrQYfDWhW+lwzyziIEtLKcs7HqfYeg7CpUe4Hh+peMJ9f+FugeG7ZjJqVzcizlUHlljK7Afruj5/2TWn8V7VtD0bwl4Xtg7WSg+YiHBmddoz9csx+rV1Oh/CWDR/Hr+IBdxvaJJJLb2ojwY2bOOc9FycfQV0XjnwVa+NdJjtpZmtrqBy9vcKu7YT1BHcHj8hRyuwHlfiXQdQv4dPk8O+AL3Rr6zlDLPGy/MAOM46tkA5PPWrvxYgbU/Fng+3ulaJrpESVQcFSzqGH6mt1Pht4n1e9sf+En8Vm6srJsxx26lXce7YHJ9Tk1ueLfAk3iTxNoeqxXscEemurNGyEl8OG4OeOlFnYDgfjL4b0jw9Y6LdaPYQ2MolaMtANpYAAgn1IPfrUvxq0+zGteGrsW6Ce8Z1uHA5kC+UAD9AT+dd38RfA8/jexsreC9jtTbSs5LoW3ZGOxqP4h+ApvGVlp32S+S1u7BmMbOpKsG256cg5UUOO4HW6bpllo9illp9slvbRklY06DJyf1Jq3WR4asdV03RIrbWdRGoXysxecLjIJyB+HStetBlXRv+Ry1f/sH2f/oy5rpa5rRv+Ry1f/sH2f8A6Mua6WvKxH8VmsdgooorAoKKKKACiiigArK03/kedX/7Btl/6Nuq1aytN/5HnV/+wbZf+jbqtqHxkT2OiooortMQooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACsHxh/yAof+wlYf+lcNb1YPjLjwtdS/wDPB4bg+wjlR/8A2Wk9hrc0aKKK806AooooAKKKKACiiigDnfFH/H94c/7CTf8ApLcVPUHij/j+8Of9hJv/AEluKnr08J/DMp7hRRRXSSFFFFABWb4g1iLQPD99qs2CttCzgH+Jv4V/E4H41pVynj/wtfeMNCj0u0v47OMzCSYuhbeB0Xg+vP4Ck9gPD/B/i/Q9LsvEA1+1vbu71hWjkkhRDhGySeWHJZs/gK7D4EeIf+Qh4elfp/pVuD+CuP8A0E4+tesaHoltomh2WmRIjLbQrHu2/eIHLficn8a5W++Hs5+JFv4t0y/itgpUzwGMnzONrYIPGV/XmoUWrCOI8P6HYa98cvEMGpW6XFvAZpxFJyrMHVRkd/vGrGnafa6H+0UtlpsK21rJExMMfCjMBYgD03DOK7bQPAs+jfEHV/Ez30csV+kirAEIKbnVuTnn7v60P4Fnb4pR+L/t0fkqmz7PsO7/AFRTrn3zRygcL4a0208afGDxFJr8YuVsmkWG2kPygLJsXI7gDt0yc0usafbeC/jZoKaAn2eK/wDKWe3jPygO5RuPTAzjsRXVeJPhpdXPiRvEfhnWG0rUpP8AWgqSjk9Tx0z3BBB61J4W+G8+n+IT4j8Ras2q6sOIzghI+MZ56nHA4AH8izA46x0Wx1z4/axb6jAtxbxB5vKcZViFQDI7jnOPaq0HhfSJvj5daObOMabH+9+ygYQnyVfGPTcc46V6LpfgWfT/AIk6j4qa+jeK7RkEAQhlzt75/wBmiDwLPD8Ubjxeb6MwypsFvsO4fu1Trn2zRygcVYWFrov7RaWemwpbWrxMTDENqjNuWIA9MjNS6hP4ZufHeq/2P4Uu/E2rbmW58+QG3jbODgsDjBGOeOOK7B/As7fFJPF/26PyVTZ9n2Hd/qinXPvmsY/DXX9J8T3+p+GPEcdhBfsWmSWAOy5OSACCDgk4PBoswMD4QJNbfEXxJbSWq2WI332kb7liYSj5Qe+MkU74P2MOpv43sLld0FyY4nHsxmBrrfB3w6ufCfiu91U6t9thu4SjiVCJGclWLE5x1B/OpvBHgxvAsmu317qUEkF4VlZyvlrEqbySxJxjDdfahJ6Aed+GPEs/g3wj4x0C7k23tizC2Hcs58skewO1vxr0H4RaD/YvgS2mkTbcagTdPkc7Twg/75AP4mvNfG2n6Z4u+L1pZ6JcpcreiIXcsDBkBGdxBHBwig/WvoOKJIIUiiUJGihVUdABwBTitQH0UUVYwooooAKKKKAOSv8A/koFn/vWv/onUa62uSv/APkoFn/vWv8A6J1GutqI7v1AKKKKsAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigCro3/I5av8A9g+z/wDRlzXS1zWjf8jlq/8A2D7P/wBGXNdLXlYj+KzWOwUUUVgUFFFFABRRRQAVlab/AMjxq/8A2DbL/wBGXVatZejfvPFeuzdkS2g/FVd//ag/OtqHxkT2OhooortMQooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACs7X7FtT8OanYL965tJYV+rIQP51o0UAY2k3o1LR7G/X7tzbxzD/gSg/1q5WL4ZH2exutMPDafdy24Homd8Y/79ulbVedJWbR0J3QUUUVIwooooAKKKKAOd8Uf8f3hz/sJN/6S3FT1B4p4u/DzHoupHJ+ttOB+pFT16eE/hmU9wooorpJCiiigAooooAKKKKACiiigAooooAKKKKACiiigAqK6tob20mtbhBJBPG0ciE/eVhgj8jUtFAGDoXgvw74amefSNLjt5nG0yFmdsegLEkD6VvUUUAFFFFABRRRQAUUUUAclf8A/JQLP/etf/ROo11tclf/API+Wsn8KyWik+h8m/8A/il/OutqI7v1AKKKKsAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigCro3/I5av/ANg+z/8ARlzXS1zWi8+MNYYdBY2afiHuD/IiulrycR/EZrHYKKKKxKCiiigAooooAKzPCw83+2b3tc6lLg+0SrB/OI1bv7yLTtOur6c4itonmc/7Kgk/yo8M2UuneGtPtrgYuRCHn/66t8z/APjxaunDrVszqM1aKKK6jIKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooA5ucf2d40DdIdVtsewnh/qyN+UVa9VPE1jPe6QZLNd19ZyLdWo6bpE/gz23KWQ+zmn2N7BqNhb3ts26CeNZEPTgjPPvXHXjaVzaD0sWKKKKwLCiiigAooooAwvF0Lv4eluYkLS2MiXiqoyWEbBmA9ygYfjRHIk0SSRsHRwGVgcgg9DW7XHWkZ0DUP7El4tXy+nSHoU6mH/eTsO6464bHbhKiV4MzmuprUUUV3kBRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFZepSz3k6aLp7st5crmWVf8Al2h6NIfQnkKO7ewOJlJRV2BnvbPdeHNX16JC7rfLd24XktDblVIHruCSkf74roo5EmiSWNg6OAysDkEHoa17W1gsrOG0t41jghjWONB0VQMAflXKWkZ0DUP7El4tXy+myHoU6mH/AHk7DumOuGxyYetzSafUuUbGtRRRXaQFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUVlag8up3f9h2Lsssig3c6f8ALtCepz2duQv4t/DzMpKKuwL3hNftEN/qv8N9cloT/wBMkAjU/Q7Wcez10VRwQRWtvFbwRrHDEgREUYCqBgAfhUlePOXNJs2SsgoooqRhRRRQAUUUUAYviAfbX07Rl5N9cqZR6QR/vJM+xwqf9tBXT1zmgr/aWtahrR5hjzY2Z9VRv3rj/ekG36RA966Ou6jHliYTd2FFFFakhRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABXL2q/2J4gm0tuLO+Z7qyPZXJzLF+Z3j2Zuy11FZ2t6Uur6cYBKYbhGEttOBkwyryre/oR3BI71E4c0bDi7MkorO0fUm1G1cTxCC+t38m7t858uQDnHqpBDKe4INaNcDVnZnQFFFFIAooooAKqalptrq1k9peR742IIIJDIw5DKRyGB5BHSrdFMDkpBq2ifJewy6lZj7t5bR7pVH/TSJeSf9pAc/3VqSz1nTL8lbW/t5XBwyLIN6n0K9QfY11NU73StO1IYv9PtboDjE8Kv/ADFdUMXJK0lchw7FCimHwR4TJJPhfRST3Onxf/E0f8IP4S/6FbRP/BfF/wDE1p9cXYXIx9FM/wCEH8Jf9Cton/gvi/8AiaP+EH8Jf9Cton/gvi/+Jo+uLsHIx9FM/wCEH8Jf9Cton/gvi/8AiaP+EH8Jf9Cton/gvi/+Jo+uLsHIx9FM/wCEH8Jf9Cton/gvi/8AiaP+EH8Jf9Cton/gvi/+Jo+uLsHIx9FM/wCEH8Jf9Cton/gvi/8AiaP+EH8Jf9Cton/gvi/+Jo+uLsHIx9FM/wCEH8Jf9Cton/gvi/8AiaP+EH8Jf9Cton/gvi/+Jo+uLsHIx9FM/wCEH8Jf9Cton/gvi/8AiaP+EH8Jf9Cton/gvi/+Jo+uLsHIx9FM/wCEH8Jf9Cton/gvi/8AiaP+EH8Jf9Cton/gvi/+Jo+uLsHIx9FM/wCEH8Jf9Cton/gvi/8AiaP+EH8Jf9Cton/gvi/+Jo+uLsHIx9FM/wCEH8Jf9Cton/gvi/8AiaP+EH8Jf9Cton/gvi/+Jo+uLsHIx9VLzVNP09d15fW1uP8AprKq/wAzVj/hB/CX/Qr6J/4L4v8A4mr9lomk6a26w0uytW9YLdEP6Ch4xdEHIc+l1qOr/u9ItHihPW+vImRFHqkZwzn8l/2j0rf0nR7bR7d0hLyzStvnuJTmSZ/Vj+gAwAOAAK0KK5qlaVTcpRSCqmpaba6tZPaXke+NiGBBKsjDkMpHKsDyCKt0VlsUcjINW0T5L2GXUrMfdvLaPdKo/wCmkS8k/wC0gOf7q1LZ6zpl+Strf28rg4ZFkG9T6FeoPsa6mqd7pWnakMX+n2t0BxieFX/mK6oYuS0krkOHYoUUw+CPCZJJ8L6KSe50+L/4mj/hB/CX/QraJ/4L4v8A4mtPri7C5GPopn/CD+Ev+hW0T/wXxf8AxNH/AAg/hL/oVtE/8F8X/wATR9cXYORj6KZ/wg/hL/oVtE/8F8X/AMTR/wAIP4S/6FbRP/BfF/8AE0fXF2DkY+imf8IP4S/6FbRP/BfF/wDE0f8ACD+Ev+hW0T/wXxf/ABNH1xdg5GPopn/CD+Ev+hW0T/wXxf8AxNH/AAg/hL/oVtE/8F8X/wATR9cXYORj6KZ/wg/hL/oVtE/8F8X/AMTR/wAIP4S/6FbRP/BfF/8AE0fXF2DkY+imf8IP4S/6FbRP/BfF/wDE0f8ACD+Ev+hW0T/wXxf/ABNH1xdg5GPopn/CD+Ev+hW0T/wXxf8AxNH/AAg/hL/oVtE/8F8X/wATR9cXYORj6KZ/wg/hL/oVtE/8F8X/AMTR/wAIP4S/6FbRP/BfF/8AE0fXF2DkY+imf8IP4S/6FbRP/BfF/wDE0f8ACD+Ev+hW0T/wXxf/ABNH1xdg5GPqjeazplgQt1f28Tk4CNINzH0C9SfYVbHgjwmCCPC+igjuNPi/+JrSstJ07TRiw0+1tQeMQQqn8hQ8Z2QchzsbatrPyafbyWFqfvXt5Ftcj/pnEec+7gAejdK6HS9KtdHtPs9qrfMxeSR23PK56szdyf8AADAAFXaK5alaVTcpRSCiiisigooooAKKKKACsnXrudYYdOsH26hqDGGFhz5S4y8v0Refdto71oXd1BY2k11dSrFBCheR26Ko6mqfh6ynmmm13UImiu7pQkEDjm2gByqn0ZvvN74HO0VrShzPyJlKyNexsoNNsLeytU2QW8axxr6KBgVYooruMAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKAMHW9NuY7pNa0qPffRJsntwQBdwg52c8BxklSe5IOAxIsaff22p2Ud3avvifPUYKkHBUg8ggggg8gjFa1c7qelXVheyavo0YeSTm8schRc443KTwsoHfowAB7EY1aXNqty4ytozVoqrp2o2uqWgubSTcmSrKwKsjDqrKeVYdweatVxmwUUUUgCiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACgkAEk4A702SRIo2kkdURAWZmOAAOpJrCjil8XkZDxeH885BVr/+oh/V/wDd+9cIObshNpDrOM+Kb6O8cH+w7Zw9up/5fJVPEh/6ZqeV/vH5ugUnqqRVVECIoVVGAAMAClrujFRVkYN3CiiiqEFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAYmp6C0t2dT0qdbPU8AOxXMVwB0WVR19mHzD1IyDBY62s10NPv4GsNTwT9nkbIkA6tE/R1+nI7gV0VVdQ02y1W1NtfWyTxEhgGHKsOjKeqkdiORWc6Sn6lRk0MorHaw1zRv+POX+17Mf8sLhwlyg9FkPyyfR8H1c1JZ+INPvLkWhke1vcZ+yXSGKX3wrfeHuuR71ySpyjuaqSZqUUUVmUFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFUtQ1fT9KVDe3UcJc4jQnLyH0VRyx9gCadrgXao6lq1ppSRm4djLKdsMEal5Zm9EUcn+Q6nAqos2uaxxY2v9l2h/5er1MzMP9iHt9XII/umtPS9BstKd54xJPeyjEt5cNvmkHoW7D/ZUBR2AraFBv4jNzS2MyHRrzW5EudfRYrRSGi0tWDLnsZmHDn/ZHyj/AGuCOmoorrjFRVkZt3CiiimIKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAooooAKKKKACiiigAqtfadZanbG3v7SC6hJz5c0Ydc+uD3oooAyD4Ye150nV76zA6Qyv8AaYvykywHsrKK47VPiJfeHfEcWiX1lb30jnH2iAtAB/wA7/50UVnKnF7opSaO+tLj7VaRT7du9c7c5xU1FFcL3NwooopAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFFFFABRRRQAUUUUAFeea58TJdM16HSIdLRpJm2id5zhf+AhefzFFFbUYqT1Im2lodZDomp6hEkmo69MI3AbydPhFupB7FiXf8VZa09N0HS9JZpLKzjjmcYedsvK/+9I2Wb8SaKK7FFLYybbNGiiimIKKKKACiiigAooooA//2Q==)" + ], + "metadata": { + "id": "CU9zuwUUu7tX" + } + }, + { + "cell_type": "code", + "source": [ + "import apache_beam as beam\n", + "import google.generativeai as genai\n", + "import logging\n", + "import os\n", + "import re\n", + "import json\n", + "import numpy as np\n", + "from apache_beam.utils.timestamp import Timestamp\n", + "from dotenv import load_dotenv\n", + "from apache_beam.transforms.userstate import BagStateSpec\n", + "\n", + "import apache_beam as beam\n", + "import json\n", + "import numpy as np\n", + "\n", + "from apache_beam.coders.coders import PickleCoder\n", + "\n", + "from apache_beam.transforms.userstate import BagStateSpec, ReadModifyWriteStateSpec, TimerSpec, on_timer\n", + "\n", + "\n", + "class CustomJsonEncoderForLLM(json.JSONEncoder):\n", + " \"\"\"Encodes special types like Timestamp and numpy objects into JSON.\"\"\"\n", + " def default(self, obj):\n", + " if isinstance(obj, Timestamp):\n", + " # Store as a dict with a special key for easy decoding\n", + " return {'__timestamp__': True, 'micros': obj.micros}\n", + " if isinstance(obj, np.integer):\n", + " return int(obj)\n", + " if isinstance(obj, np.floating):\n", + " return float(obj)\n", + " if isinstance(obj, np.ndarray):\n", + " return obj.tolist()\n", + " return super().default(obj)\n", + "\n", + "def custom_json_decoder(dct):\n", + " \"\"\"Decodes a Timestamp object from our custom dict format.\"\"\"\n", + " if '__timestamp__' in dct:\n", + " return Timestamp(micros=dct['micros'])\n", + " return dct\n", + "\n", + "class JsonCoderWithNumpyAndTimestamp(beam.coders.Coder):\n", + " \"\"\"A custom Beam Coder that handles JSON serialization for Timestamps and numpy types.\"\"\"\n", + " def encode(self, value):\n", + " return json.dumps(value, cls=CustomJsonEncoderForLLM).encode('utf-8')\n", + "\n", + " def decode(self, encoded):\n", + " return json.loads(encoded.decode('utf-8'), object_hook=custom_json_decoder)\n", + "\n", + " def is_deterministic(self):\n", + " return True\n", + "\n", + "\n", + "# It's highly recommended to manage API keys via GCP Secret Manager\n", + "# and access them as environment variables in your Dataflow job.\n", + "# genai.configure(api_key=os.environ[\"GEMINI_API_KEY\"])\n", + "\n", + "class LLMClassifierFn(beam.DoFn):\n", + " \"\"\"\n", + " Takes an anomaly, formats a detailed prompt with surrounding context,\n", + " calls the Gemini model to classify it, and routes the original data\n", + " based on the model's decision.\n", + "\n", + " This DoFn is stateful, deferring anomalies that occur too close to\n", + " the end of a window until a subsequent window provides enough context.\n", + " \"\"\"\n", + "\n", + " DEFERRED_ANOMALIES_STATE = BagStateSpec(\n", + " 'deferred_anomalies', coder=JsonCoderWithNumpyAndTimestamp())\n", + " YIELD_BUFFER_STATE = ReadModifyWriteStateSpec('yield_buffer', PickleCoder())\n", + "\n", + " # <<< CHANGE: Define a timer and a state to track if it's set\n", + " EXPIRY_TIMER = TimerSpec('expiry', beam.TimeDomain.WATERMARK)\n", + " # <<< CHANGE: Add state to track the last yielded timestamp\n", + " LAST_YIELDED_TIMESTAMP_STATE = ReadModifyWriteStateSpec('last_yielded_ts', PickleCoder())\n", + "\n", + "\n", + "\n", + "\n", + " def __init__(self, secret, context_points=25, slide_interval=128, expected_interval_secs=1):\n", + " self.context_points = context_points\n", + " self._model = None\n", + " self.secret = secret\n", + " self.slide_interval = slide_interval\n", + " self.expected_interval_micros = expected_interval_secs * 1_000_000\n", + "\n", + " self._last_window_data = None\n", + "\n", + "\n", + " def setup(self):\n", + " # Configure the generative model\n", + "\n", + " genai.configure(api_key=self.secret)\n", + " logging.getLogger().setLevel(logging.INFO)\n", + "\n", + "\n", + " generation_config = {\n", + " \"temperature\": 0.2,\n", + " \"top_p\": 1,\n", + " \"top_k\": 1,\n", + " \"max_output_tokens\": 256,\n", + " \"response_mime_type\": \"application/json\",\n", + " }\n", + " # For a full list of safety settings, see the Gemini API documentation\n", + " safety_settings = [\n", + " {\"category\": \"HARM_CATEGORY_HARASSMENT\", \"threshold\": \"BLOCK_NONE\"},\n", + " {\"category\": \"HARM_CATEGORY_HATE_SPEECH\", \"threshold\": \"BLOCK_NONE\"},\n", + " ]\n", + " self._model = genai.GenerativeModel(\n", + " model_name=\"gemini-1.5-flash-latest\",\n", + " generation_config=generation_config,\n", + " safety_settings=safety_settings\n", + " )\n", + " logging.info(\"Gemini Model has been successfully initialized.\")\n", + "\n", + " def _build_prompt(self, anomaly_data, context_before, context_after):\n", + " mean_before = np.mean(context_before) if context_before.size > 0 else 0\n", + " mean_after = np.mean(context_after) if context_after.size > 0 else 0\n", + " std_before = np.std(context_before) if context_before.size > 0 else 0\n", + " std_after = np.std(context_after) if context_after.size > 0 else 0\n", + "\n", + " return f\"\"\"\n", + " You are an expert time-series analyst classifying an outlier from NYC taxi pickup data.\n", + " Normal behavior includes daily and weekly cyclical patterns.\n", + "\n", + " **1. Outlier Context:**\n", + " * **--> The Outlier:**\n", + " * **Timestamp:** {Timestamp(micros=anomaly_data['timestamp'].micros)}\n", + " * **Actual Value:** {anomaly_data['actual_value']:.2f}\n", + " * **Predicted Value:** {anomaly_data['predicted_value']:.2f}\n", + " * **Anomaly Upper Bound:** {anomaly_data['upper_bound']:.2f}\n", + " * **Anomaly Lower Bound:** {anomaly_data['lower_bound']:.2f}\n", + "\n", + " **2. Data Surrounding the Outlier:**\n", + " * **Data Before ({len(context_before)} points):** {np.round(context_before, 2).tolist()}\n", + " * **Data After ({len(context_after)} points):** {np.round(context_after, 2).tolist()}\n", + "\n", + " **3. Statistical Context:**\n", + " * **Mean Before:** {mean_before:.2f}\n", + " * **Mean After:** {mean_after:.2f}\n", + " * **Std. Dev. Before:** {std_before:.2f}\n", + " * **Std. Dev. After:** {std_after:.2f}\n", + "\n", + " **4. Your Task:**\n", + "\n", + " **Step 1: Analyze the Evidence.** In a few sentences, describe the behavior of the data *after* the outlier. Does it quickly revert to the \"Predicted Value\" or the \"Mean Before\"? Or does it establish a new level, closer to the \"Mean After\"?\n", + "\n", + " **Step 2: Make a Decision.** Classify the outlier.\n", + " * **REMOVE:** If it's a transient, one-off event. This is likely if the data after the outlier rapidly returns to the established pattern.\n", + " * **KEEP:** If it signifies a sustained shift in the pattern that the model should learn from. This is likely if the `Mean After` has shifted significantly.\n", + "\n", + " **Step 3: Provide Final Output.** Respond with a single JSON object. Do not add any text outside the JSON block.\n", + "\n", + " {{\n", + " \"reasoning_steps\": \"Your analysis from Step 1 goes here.\",\n", + " \"decision\": \"KEEP or REMOVE\",\n", + " \"confidence_score\": \n", + " }}\n", + " \"\"\"\n", + "\n", + " def process(self, element,\n", + " deferred_anomalies=beam.DoFn.StateParam(DEFERRED_ANOMALIES_STATE),\n", + " yield_buffer=beam.DoFn.StateParam(YIELD_BUFFER_STATE),\n", + " expiry_timer=beam.DoFn.TimerParam(EXPIRY_TIMER)):\n", + "\n", + " key, data = element\n", + " window_start_ts = data['window_start_ts']\n", + "\n", + " # Set a timer to fire based on the event time of the current element.\n", + " # Each new element will push the timer forward. The timer will only\n", + " # fire when a gap in the input stream occurs, allowing the buffer\n", + " # to contain data from multiple consecutive sliding windows.\n", + " # We set it far enough ahead to allow the next window's data to arrive.\n", + " grace_period_secs = self.slide_interval * 2\n", + " expiry_timer.set(window_start_ts + grace_period_secs)\n", + " anomalies_in_window = data.get('anomalies', [])\n", + " values_in_element = data.get('values_array', [])\n", + "\n", + " for anomaly in anomalies_in_window:\n", + " deferred_anomalies.add(anomaly)\n", + "\n", + " buffer = yield_buffer.read() or {}\n", + " for i, value in enumerate(values_in_element):\n", + " point_timestamp = Timestamp(micros=window_start_ts.micros + (i * self.expected_interval_micros))\n", + " buffer[point_timestamp] = value\n", + " yield_buffer.write(buffer)\n", + "\n", + " @on_timer(EXPIRY_TIMER)\n", + " def on_expiry_timer(\n", + " self,\n", + " deferred_anomalies=beam.DoFn.StateParam(DEFERRED_ANOMALIES_STATE),\n", + " yield_buffer=beam.DoFn.StateParam(YIELD_BUFFER_STATE),\n", + " # <<< CHANGE: Add the new state parameter here\n", + " last_yielded_ts_state=beam.DoFn.StateParam(LAST_YIELDED_TIMESTAMP_STATE)):\n", + "\n", + " all_anomalies_to_consider = list(deferred_anomalies.read())\n", + " buffered_points_map = yield_buffer.read() or {}\n", + "\n", + " if not buffered_points_map:\n", + " return\n", + "\n", + " sorted_points = sorted(buffered_points_map.items())\n", + " all_timestamps = [ts for ts, val in sorted_points]\n", + " all_values = [val for ts, val in sorted_points]\n", + "\n", + " anomalies_to_process_now = []\n", + " prompts_to_batch = []\n", + " final_deferred = []\n", + "\n", + " for anomaly_data in all_anomalies_to_consider:\n", + " anomaly_ts = anomaly_data['timestamp']\n", + " try:\n", + " idx_in_full_data = all_timestamps.index(anomaly_ts)\n", + "\n", + " if (idx_in_full_data + self.context_points) < len(all_values):\n", + " start_ctx = max(0, idx_in_full_data - self.context_points)\n", + " end_ctx = idx_in_full_data + self.context_points + 1\n", + "\n", + " context_before = np.array(all_values[start_ctx:idx_in_full_data])\n", + " context_after = np.array(all_values[idx_in_full_data + 1:end_ctx])\n", + "\n", + " anomaly_data['index_in_window'] = idx_in_full_data\n", + " prompt = self._build_prompt(anomaly_data, context_before, context_after)\n", + " prompts_to_batch.append(prompt)\n", + " anomalies_to_process_now.append(anomaly_data)\n", + " else:\n", + " final_deferred.append(anomaly_data)\n", + " except ValueError:\n", + " final_deferred.append(anomaly_data)\n", + "\n", + " if prompts_to_batch:\n", + " try:\n", + " logging.info(f\"Sending a batch of {len(prompts_to_batch)} prompts to the LLM.\")\n", + " responses = self._model.generate_content(prompts_to_batch)\n", + " for anomaly_data, response in zip(anomalies_to_process_now, responses):\n", + " try:\n", + " response_data = json.loads(response.text)\n", + " decision = response_data.get('decision', 'KEEP').strip().upper()\n", + " idx = anomaly_data['index_in_window']\n", + "\n", + " if decision == 'REMOVE':\n", + " logging.warning(f\"LLM decided to REMOVE anomaly at {anomaly_data['timestamp']}. Imputing value.\")\n", + " all_values[idx] = anomaly_data['predicted_value']\n", + " except (json.JSONDecodeError, AttributeError) as e:\n", + " logging.error(f\"Error processing LLM response for {anomaly_data['timestamp']}: {e}. Defaulting to KEEP.\")\n", + " except Exception as e:\n", + " logging.error(f\"Error calling LLM with a batch: {e}. Defaulting to KEEP for all.\")\n", + "\n", + " # <<< CHANGE: New logic to yield only new data\n", + " last_yielded_ts = last_yielded_ts_state.read()\n", + " latest_ts_in_batch = None\n", + "\n", + " for i, (ts, original_val) in enumerate(sorted_points):\n", + " # Only yield points that are newer than the last batch we yielded\n", + " if last_yielded_ts is None or ts > last_yielded_ts:\n", + " yield {\n", + " 'timestamp': ts,\n", + " 'value': all_values[i]\n", + " }\n", + " latest_ts_in_batch = ts\n", + "\n", + " # After yielding, update the state with the latest timestamp from this batch\n", + " if latest_ts_in_batch:\n", + " last_yielded_ts_state.write(latest_ts_in_batch)\n", + "\n", + " # Prune the buffer. We need to keep enough historical data to serve\n", + " # as `context_before` for the anomalies that we are re-deferring.\n", + " if latest_ts_in_batch:\n", + " all_buffered_points = yield_buffer.read() or {}\n", + "\n", + " # Find the earliest timestamp we need to keep. This will be\n", + " # `context_points` before the last yielded point, ensuring\n", + " # context is available for the next batch.\n", + " try:\n", + " last_yielded_index = all_timestamps.index(latest_ts_in_batch)\n", + " context_start_index = max(0, last_yielded_index - self.context_points)\n", + " context_start_ts = all_timestamps[context_start_index]\n", + "\n", + " pruned_buffer = {\n", + " ts: val\n", + " for ts, val in all_buffered_points.items()\n", + " if ts >= context_start_ts\n", + " }\n", + " yield_buffer.write(pruned_buffer)\n", + " except ValueError:\n", + " # This can happen if the buffer is in an inconsistent state.\n", + " # As a fallback, we clear it if we aren't deferring anything.\n", + " logging.warning(\n", + " f\"Could not find last yielded timestamp \"\n", + " f\"{latest_ts_in_batch} in buffer for pruning.\"\n", + " )\n", + " if not final_deferred:\n", + " yield_buffer.clear()\n", + " elif not final_deferred:\n", + " # If we didn't yield anything and we're not deferring anything,\n", + " # the buffer is fully processed and can be cleared.\n", + " yield_buffer.clear()\n", + "\n", + " # Re-add anomalies that couldn't be processed to the state so they can\n", + " # be considered in the next firing.\n", + " deferred_anomalies.clear()\n", + " if final_deferred:\n", + " logging.info(f\"Re-deferring {len(final_deferred)} anomalies due to insufficient context.\")\n", + " for anomaly in final_deferred:\n", + " deferred_anomalies.add(anomaly)\n" + ], + "metadata": { + "id": "c55ou9f5vADf" + }, + "execution_count": 4, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "# Finetuning Component" + ], + "metadata": { + "id": "WSl5lV_9ugQY" + } + }, + { + "cell_type": "code", + "source": [ + "\"\"\"\n", + "TimesFM Finetuner: A flexible framework for finetuning TimesFM models on custom datasets.\n", + "\"\"\"\n", + "\n", + "import logging\n", + "import os\n", + "from abc import ABC, abstractmethod\n", + "from dataclasses import dataclass, field\n", + "from typing import Any, Callable, Dict, List, Optional\n", + "\n", + "import torch\n", + "import torch.distributed as dist\n", + "import torch.nn as nn\n", + "from torch.nn.parallel import DistributedDataParallel as DDP\n", + "from torch.utils.data import DataLoader, Dataset\n", + "from timesfm.pytorch_patched_decoder import create_quantiles\n", + "\n", + "import wandb\n", + "\n", + "\n", + "class MetricsLogger(ABC):\n", + " \"\"\"Abstract base class for logging metrics during training.\n", + "\n", + " This class defines the interface for logging metrics during model training.\n", + " Concrete implementations can log to different backends (e.g., WandB, TensorBoard).\n", + " \"\"\"\n", + "\n", + " @abstractmethod\n", + " def log_metrics(self,\n", + " metrics: Dict[str, Any],\n", + " step: Optional[int] = None) -> None:\n", + " \"\"\"Log metrics to the specified backend.\n", + "\n", + " Args:\n", + " metrics: Dictionary containing metric names and values.\n", + " step: Optional step number or epoch for the metrics.\n", + " \"\"\"\n", + " pass\n", + "\n", + " @abstractmethod\n", + " def close(self) -> None:\n", + " \"\"\"Clean up any resources used by the logger.\"\"\"\n", + " pass\n", + "\n", + "\n", + "class WandBLogger(MetricsLogger):\n", + " \"\"\"Weights & Biases implementation of metrics logging.\n", + "\n", + " Args:\n", + " project: Name of the W&B project.\n", + " config: Configuration dictionary to log.\n", + " rank: Process rank in distributed training.\n", + " \"\"\"\n", + "\n", + " def __init__(self, project: str, config: Dict[str, Any], rank: int = 0):\n", + " self.rank = rank\n", + " if rank == 0:\n", + " wandb.init(project=project, config=config)\n", + "\n", + " def log_metrics(self,\n", + " metrics: Dict[str, Any],\n", + " step: Optional[int] = None) -> None:\n", + " \"\"\"Log metrics to W&B if on the main process.\n", + "\n", + " Args:\n", + " metrics: Dictionary of metrics to log.\n", + " step: Current training step or epoch.\n", + " \"\"\"\n", + " if self.rank == 0:\n", + " wandb.log(metrics, step=step)\n", + "\n", + " def close(self) -> None:\n", + " \"\"\"Finish the W&B run if on the main process.\"\"\"\n", + " if self.rank == 0:\n", + " wandb.finish()\n", + "\n", + "\n", + "class DistributedManager:\n", + " \"\"\"Manages distributed training setup and cleanup.\n", + "\n", + " Args:\n", + " world_size: Total number of processes.\n", + " rank: Process rank.\n", + " master_addr: Address of the master process.\n", + " master_port: Port for distributed communication.\n", + " backend: PyTorch distributed backend to use.\n", + " \"\"\"\n", + "\n", + " def __init__(\n", + " self,\n", + " world_size: int,\n", + " rank: int,\n", + " master_addr: str = \"localhost\",\n", + " master_port: str = \"12358\",\n", + " backend: str = \"nccl\",\n", + " ):\n", + " self.world_size = world_size\n", + " self.rank = rank\n", + " self.master_addr = master_addr\n", + " self.master_port = master_port\n", + " self.backend = backend\n", + "\n", + " def setup(self) -> None:\n", + " \"\"\"Initialize the distributed environment.\"\"\"\n", + " os.environ[\"MASTER_ADDR\"] = self.master_addr\n", + " os.environ[\"MASTER_PORT\"] = self.master_port\n", + "\n", + " if not dist.is_initialized():\n", + " dist.init_process_group(backend=self.backend,\n", + " world_size=self.world_size,\n", + " rank=self.rank)\n", + "\n", + " def cleanup(self) -> None:\n", + " \"\"\"Clean up the distributed environment.\"\"\"\n", + " if dist.is_initialized():\n", + " dist.destroy_process_group()\n", + "\n", + "\n", + "@dataclass\n", + "class FinetuningConfig:\n", + " \"\"\"Configuration for model training.\n", + "\n", + " Args:\n", + " batch_size: Number of samples per batch.\n", + " num_epochs: Number of training epochs.\n", + " learning_rate: Initial learning rate.\n", + " weight_decay: L2 regularization factor.\n", + " freq_type: Frequency, can be [0, 1, 2].\n", + " use_quantile_loss: bool = False # Flag to enable/disable quantile loss\n", + " quantiles: Optional[List[float]] = None\n", + " device: Device to train on ('cuda' or 'cpu').\n", + " distributed: Whether to use distributed training.\n", + " gpu_ids: List of GPU IDs to use.\n", + " master_port: Port for distributed training.\n", + " master_addr: Address for distributed training.\n", + " use_wandb: Whether to use Weights & Biases logging.\n", + " wandb_project: W&B project name.\n", + " log_every_n_steps: Log metrics every N steps (batches), this is inspired from Pytorch Lightning\n", + " val_check_interval: How often within one training epoch to check val metrics. (also from Pytorch Lightning)\n", + " Can be: float (0.0-1.0): fraction of epoch (e.g., 0.5 = validate twice per epoch)\n", + " int: validate every N batches\n", + " \"\"\"\n", + "\n", + " batch_size: int = 32\n", + " num_epochs: int = 20\n", + " learning_rate: float = 1e-4\n", + " weight_decay: float = 0.01\n", + " freq_type: int = 0\n", + " use_quantile_loss: bool = False\n", + " quantiles: Optional[List[float]] = None\n", + " device: str = \"cuda\" if torch.cuda.is_available() else \"cpu\"\n", + " distributed: bool = False\n", + " gpu_ids: List[int] = field(default_factory=lambda: [0])\n", + " master_port: str = \"12358\"\n", + " master_addr: str = \"localhost\"\n", + " use_wandb: bool = False\n", + " wandb_project: str = \"timesfm-finetuning\"\n", + " log_every_n_steps: int = 50\n", + " val_check_interval: float = 0.5\n", + "\n", + "\n", + "class TimesFMFinetuner:\n", + " \"\"\"Handles model training and validation.\n", + "\n", + " Args:\n", + " model: PyTorch model to train.\n", + " config: Training configuration.\n", + " rank: Process rank for distributed training.\n", + " loss_fn: Loss function (defaults to MSE).\n", + " logger: Optional logging.Logger instance.\n", + " \"\"\"\n", + "\n", + " def __init__(\n", + " self,\n", + " model: nn.Module,\n", + " config: FinetuningConfig,\n", + " rank: int = 0,\n", + " loss_fn: Optional[Callable] = None,\n", + " logger: Optional[logging.Logger] = None,\n", + " ):\n", + " self.model = model\n", + " self.config = config\n", + " self.rank = rank\n", + " self.logger = logger or logging.getLogger(__name__)\n", + " self.device = torch.device(\n", + " f\"cuda:{rank}\" if torch.cuda.is_available() else \"cpu\")\n", + " self.loss_fn = loss_fn or (lambda x, y: torch.mean((x - y.squeeze(-1))**2))\n", + "\n", + " if config.use_wandb:\n", + " self.metrics_logger = WandBLogger(config.wandb_project, config.__dict__,\n", + " rank)\n", + "\n", + " if config.distributed:\n", + " self.dist_manager = DistributedManager(\n", + " world_size=len(config.gpu_ids),\n", + " rank=rank,\n", + " master_addr=config.master_addr,\n", + " master_port=config.master_port,\n", + " )\n", + " self.dist_manager.setup()\n", + " self.model = self._setup_distributed_model()\n", + "\n", + " def _setup_distributed_model(self) -> nn.Module:\n", + " \"\"\"Configure model for distributed training.\"\"\"\n", + " self.model = self.model.to(self.device)\n", + " return DDP(self.model,\n", + " device_ids=[self.config.gpu_ids[self.rank]],\n", + " output_device=self.config.gpu_ids[self.rank])\n", + "\n", + " def _create_dataloader(self, dataset: Dataset, is_train: bool) -> DataLoader:\n", + " \"\"\"Create appropriate DataLoader based on training configuration.\n", + "\n", + " Args:\n", + " dataset: Dataset to create loader for.\n", + " is_train: Whether this is for training (affects shuffling).\n", + "\n", + " Returns:\n", + " DataLoader instance.\n", + " \"\"\"\n", + " if self.config.distributed:\n", + " sampler = torch.utils.data.distributed.DistributedSampler(\n", + " dataset,\n", + " num_replicas=len(self.config.gpu_ids),\n", + " rank=dist.get_rank(),\n", + " shuffle=is_train)\n", + " else:\n", + " sampler = None\n", + "\n", + " return DataLoader(\n", + " dataset,\n", + " batch_size=self.config.batch_size,\n", + " shuffle=(is_train and not self.config.distributed),\n", + " sampler=sampler,\n", + " )\n", + "\n", + " def _quantile_loss(self, pred: torch.Tensor, actual: torch.Tensor,\n", + " quantile: float) -> torch.Tensor:\n", + " \"\"\"Calculates quantile loss.\n", + " Args:\n", + " pred: Predicted values\n", + " actual: Actual values\n", + " quantile: Quantile at which loss is computed\n", + " Returns:\n", + " Quantile loss\n", + " \"\"\"\n", + " dev = actual - pred\n", + " loss_first = dev * quantile\n", + " loss_second = -dev * (1.0 - quantile)\n", + " return 2 * torch.where(loss_first >= 0, loss_first, loss_second)\n", + "\n", + " def _process_batch(self, batch: List[torch.Tensor]) -> tuple:\n", + " \"\"\"Process a single batch of data.\n", + "\n", + " Args:\n", + " batch: List of input tensors.\n", + "\n", + " Returns:\n", + " Tuple of (loss, predictions).\n", + " \"\"\"\n", + " x_context, x_padding, freq, x_future = [\n", + " t.to(self.device, non_blocking=True) for t in batch\n", + " ]\n", + "\n", + " predictions = self.model(x_context, x_padding.float(), freq)\n", + " predictions_mean = predictions[..., 0]\n", + " last_patch_pred = predictions_mean[:, -1, :]\n", + "\n", + " loss = self.loss_fn(last_patch_pred, x_future.squeeze(-1))\n", + " if self.config.use_quantile_loss:\n", + " quantiles = self.config.quantiles or create_quantiles()\n", + " for i, quantile in enumerate(quantiles):\n", + " last_patch_quantile = predictions[:, -1, :, i + 1]\n", + " loss += torch.mean(\n", + " self._quantile_loss(last_patch_quantile, x_future.squeeze(-1),\n", + " quantile))\n", + "\n", + " return loss, predictions\n", + "\n", + " def _train_epoch(self, train_loader: DataLoader,\n", + " optimizer: torch.optim.Optimizer) -> float:\n", + " \"\"\"Train for one epoch in a distributed setting.\n", + "\n", + " Args:\n", + " train_loader: DataLoader for training data.\n", + " optimizer: Optimizer instance.\n", + "\n", + " Returns:\n", + " Average training loss for the epoch.\n", + " \"\"\"\n", + " self.model.train()\n", + " total_loss = 0.0\n", + " num_batches = len(train_loader)\n", + "\n", + " for batch in train_loader:\n", + " loss, _ = self._process_batch(batch)\n", + "\n", + " optimizer.zero_grad()\n", + " loss.backward()\n", + " optimizer.step()\n", + "\n", + " total_loss += loss.item()\n", + "\n", + " avg_loss = total_loss / num_batches\n", + "\n", + " if self.config.distributed:\n", + " avg_loss_tensor = torch.tensor(avg_loss, device=self.device)\n", + " dist.all_reduce(avg_loss_tensor, op=dist.ReduceOp.SUM)\n", + " avg_loss = (avg_loss_tensor / dist.get_world_size()).item()\n", + "\n", + " return avg_loss\n", + "\n", + " def _validate(self, val_loader: DataLoader) -> float:\n", + " \"\"\"Perform validation.\n", + "\n", + " Args:\n", + " val_loader: DataLoader for validation data.\n", + "\n", + " Returns:\n", + " Average validation loss.\n", + " \"\"\"\n", + " self.model.eval()\n", + " total_loss = 0.0\n", + " num_batches = len(val_loader)\n", + "\n", + " with torch.no_grad():\n", + " for batch in val_loader:\n", + " loss, _ = self._process_batch(batch)\n", + " total_loss += loss.item()\n", + "\n", + " avg_loss = total_loss / num_batches\n", + "\n", + " if self.config.distributed:\n", + " avg_loss_tensor = torch.tensor(avg_loss, device=self.device)\n", + " dist.all_reduce(avg_loss_tensor, op=dist.ReduceOp.SUM)\n", + " avg_loss = (avg_loss_tensor / dist.get_world_size()).item()\n", + "\n", + " return avg_loss\n", + "\n", + " def finetune(self, train_dataset: Dataset,\n", + " val_dataset: Dataset) -> Dict[str, Any]:\n", + " \"\"\"Train the model.\n", + "\n", + " Args:\n", + " train_dataset: Training dataset.\n", + " val_dataset: Validation dataset.\n", + "\n", + " Returns:\n", + " Dictionary containing training history.\n", + " \"\"\"\n", + " self.model = self.model.to(self.device)\n", + " train_loader = self._create_dataloader(train_dataset, is_train=True)\n", + " val_loader = self._create_dataloader(val_dataset, is_train=False)\n", + "\n", + " optimizer = torch.optim.Adam(self.model.parameters(),\n", + " lr=self.config.learning_rate,\n", + " weight_decay=self.config.weight_decay)\n", + "\n", + " history = {\"train_loss\": [], \"val_loss\": [], \"learning_rate\": []}\n", + "\n", + " self.logger.info(\n", + " f\"Starting training for {self.config.num_epochs} epochs...\")\n", + " self.logger.info(f\"Training samples: {len(train_dataset)}\")\n", + " self.logger.info(f\"Validation samples: {len(val_dataset)}\")\n", + "\n", + " try:\n", + " for epoch in range(self.config.num_epochs):\n", + " train_loss = self._train_epoch(train_loader, optimizer)\n", + " val_loss = self._validate(val_loader)\n", + " current_lr = optimizer.param_groups[0][\"lr\"]\n", + "\n", + " metrics = {\n", + " \"train_loss\": train_loss,\n", + " \"val_loss\": val_loss,\n", + " \"learning_rate\": current_lr,\n", + " \"epoch\": epoch + 1,\n", + " }\n", + "\n", + " if self.config.use_wandb:\n", + " self.metrics_logger.log_metrics(metrics)\n", + "\n", + " history[\"train_loss\"].append(train_loss)\n", + " history[\"val_loss\"].append(val_loss)\n", + " history[\"learning_rate\"].append(current_lr)\n", + "\n", + " if self.rank == 0:\n", + " self.logger.info(\n", + " f\"[Epoch {epoch+1}] Train Loss: {train_loss:.4f} | Val Loss: {val_loss:.4f}\"\n", + " )\n", + "\n", + " except KeyboardInterrupt:\n", + " self.logger.info(\"Training interrupted by user\")\n", + "\n", + " if self.config.distributed:\n", + " self.dist_manager.cleanup()\n", + "\n", + " if self.config.use_wandb:\n", + " self.metrics_logger.close()\n", + "\n", + " return {\"history\": history}\n", + "\n", + "import apache_beam as beam\n", + "import logging\n", + "import torch\n", + "import numpy as np\n", + "import timesfm\n", + "from os import path\n", + "from timesfm import TimesFm, TimesFmCheckpoint, TimesFmHparams\n", + "from timesfm.pytorch_patched_decoder import PatchedTimeSeriesDecoder\n", + "from huggingface_hub import snapshot_download\n", + "from apache_beam.io.gcp.gcsio import GcsIO # Add this import\n", + "\n", + "from torch.utils.data import Dataset\n", + "from google.cloud import storage\n", + "from typing import Tuple\n", + "\n", + "\n", + "class TimeSeriesDataset(Dataset):\n", + " \"\"\"Dataset for time series data compatible with TimesFM.\"\"\"\n", + " def __init__(\n", + " self,\n", + " series: np.ndarray,\n", + " context_length: int,\n", + " horizon_length: int,\n", + " freq_type: int = 0):\n", + " \"\"\"\n", + " Initialize dataset.\n", + "\n", + " Args:\n", + " series: Time series data\n", + " context_length: Number of past timesteps to use as input\n", + " horizon_length: Number of future timesteps to predict\n", + " freq_type: Frequency type (0, 1, or 2)\n", + " \"\"\"\n", + " if freq_type not in [0, 1, 2]:\n", + " raise ValueError(\"freq_type must be 0, 1, or 2\")\n", + "\n", + " self.series = series\n", + " self.context_length = context_length\n", + " self.horizon_length = horizon_length\n", + " self.freq_type = freq_type\n", + " self._prepare_samples()\n", + "\n", + " def _prepare_samples(self) -> None:\n", + " \"\"\"Prepare sliding window samples from the time series.\"\"\"\n", + " self.samples = []\n", + " total_length = self.context_length + self.horizon_length\n", + "\n", + " for start_idx in range(0, len(self.series) - total_length + 1):\n", + " end_idx = start_idx + self.context_length\n", + " x_context = self.series[start_idx:end_idx]\n", + " x_future = self.series[end_idx:end_idx + self.horizon_length]\n", + " self.samples.append((x_context, x_future))\n", + "\n", + " def __len__(self) -> int:\n", + " return len(self.samples)\n", + "\n", + " def __getitem__(\n", + " self, index: int\n", + " ) -> Tuple[torch.Tensor, torch.Tensor, torch.Tensor, torch.Tensor]:\n", + " x_context, x_future = self.samples[index]\n", + "\n", + " x_context = torch.tensor(x_context, dtype=torch.float32)\n", + " x_future = torch.tensor(x_future, dtype=torch.float32)\n", + "\n", + " input_padding = torch.zeros_like(x_context)\n", + " freq = torch.tensor([self.freq_type], dtype=torch.long)\n", + "\n", + " return x_context, input_padding, freq, x_future\n", + "\n", + "\n", + "def prepare_datasets(\n", + " series: np.ndarray,\n", + " context_length: int,\n", + " horizon_length: int,\n", + " freq_type: int = 0,\n", + " train_split: float = 0.8) -> Tuple[Dataset, Dataset]:\n", + " \"\"\"\n", + " Prepare training and validation datasets from time series data.\n", + "\n", + " Args:\n", + " series: Input time series data\n", + " context_length: Number of past timesteps to use\n", + " horizon_length: Number of future timesteps to predict\n", + " freq_type: Frequency type (0, 1, or 2)\n", + " train_split: Fraction of data to use for training\n", + "\n", + " Returns:\n", + " Tuple of (train_dataset, val_dataset)\n", + " \"\"\"\n", + " train_size = int(len(series) * train_split)\n", + " train_data = series[:train_size]\n", + " val_data = series[train_size:]\n", + "\n", + " # Create datasets with specified frequency type\n", + " train_dataset = TimeSeriesDataset(\n", + " train_data,\n", + " context_length=context_length,\n", + " horizon_length=horizon_length,\n", + " freq_type=freq_type)\n", + "\n", + " val_dataset = TimeSeriesDataset(\n", + " val_data,\n", + " context_length=context_length,\n", + " horizon_length=horizon_length,\n", + " freq_type=freq_type)\n", + "\n", + " return train_dataset, val_dataset\n", + "\n", + "\n", + "class BatchContinuousAndOrderedFn(beam.DoFn):\n", + " \"\"\"\n", + " A stateful DoFn that buffers elements, keeps them sorted, and emits\n", + " a batch only when a full, continuous sequence of points is available.\n", + " Includes detailed logging for debugging.\n", + " \"\"\"\n", + " BUFFER_STATE = ReadModifyWriteStateSpec('buffer', PickleCoder())\n", + "\n", + " def __init__(self, batch_size, expected_interval_seconds=1):\n", + " self.batch_size = batch_size\n", + " self.interval = expected_interval_seconds\n", + " # NEW LOGGING: Counter to avoid logging on every single element\n", + " self.counter = 0\n", + "\n", + " def process(self, element, buffer=beam.DoFn.StateParam(BUFFER_STATE)):\n", + " key, data = element\n", + " timestamp = data['timestamp']\n", + " value = data['value']\n", + "\n", + " # Increment the counter\n", + " self.counter += 1\n", + "\n", + " current_buffer = buffer.read() or []\n", + " current_buffer.append((timestamp, value))\n", + " current_buffer.sort(key=lambda x: x[0])\n", + "\n", + " # NEW LOGGING: Periodically log the buffer status\n", + " if self.counter % 100 == 0 and current_buffer:\n", + " logging.info(\n", + " f\"Batching buffer now contains {len(current_buffer)} points. \"\n", + " f\"Timestamps range from {current_buffer[0][0]} to {current_buffer[-1][0]}.\"\n", + " )\n", + "\n", + " start_index = 0\n", + " while start_index + self.batch_size <= len(current_buffer):\n", + " is_continuous = True\n", + " # Check for continuity in the slice of the buffer we are considering\n", + " for i in range(start_index, start_index + self.batch_size - 1):\n", + " ts1_seconds = current_buffer[i][0].seconds()\n", + " ts2_seconds = current_buffer[i + 1][0].seconds()\n", + "\n", + " if ts2_seconds - ts1_seconds != self.interval:\n", + " is_continuous = False\n", + " # If a gap is found, we should stop and wait for more data.\n", + " # We can't proceed past this point because the buffer is sorted.\n", + " logging.info(\n", + " f\"Gap detected at index {i}. \"\n", + " f\"Timestamp {current_buffer[i][0]} is followed by {current_buffer[i+1][0]}. \"\n", + " f\"Actual interval: {ts2_seconds - ts1_seconds}s, Expected: {self.interval}s. \"\n", + " f\"Waiting for missing data.\"\n", + " )\n", + " break\n", + "\n", + " if not is_continuous:\n", + " # Since the buffer is sorted, a gap at this point means we can't form any more continuous batches.\n", + " break\n", + "\n", + " # If we are here, the batch from start_index is continuous.\n", + " logging.info(f\"Continuous sequence found! Emitting batch of size {self.batch_size} starting at index {start_index}.\")\n", + "\n", + " batch_to_yield = current_buffer[start_index : start_index + self.batch_size]\n", + "\n", + " formatted_batch = [{'timestamp': ts, 'value': val} for ts, val in batch_to_yield]\n", + " yield formatted_batch\n", + "\n", + " # Move the start_index to the next position after the yielded batch\n", + " start_index += self.batch_size\n", + "\n", + " # After the loop, remove all the yielded elements from the buffer.\n", + " if start_index > 0:\n", + " current_buffer = current_buffer[start_index:]\n", + "\n", + " buffer.write(current_buffer)\n", + "\n", + "class RunFinetuningFn(beam.DoFn):\n", + " \"\"\"\n", + " Takes a batch of data, loads the LATEST model, runs fine-tuning,\n", + " and uploads the new model to GCS.\n", + " \"\"\"\n", + " def __init__(\n", + " self,\n", + " initial_model_path, # Renamed from base_model_path\n", + " finetuned_model_bucket,\n", + " finetuned_model_prefix,\n", + " hparams,\n", + " config):\n", + " # This is now a fallback for the very first run\n", + " self.initial_model_path = initial_model_path\n", + " self.finetuned_model_bucket = finetuned_model_bucket\n", + " self.finetuned_model_prefix = finetuned_model_prefix\n", + " self.hparams = hparams\n", + " self.config = config\n", + " self._storage_client = None\n", + "\n", + " def setup(self):\n", + " self._storage_client = storage.Client()\n", + "\n", + " def _get_latest_model_from_gcs(self):\n", + " \"\"\"Directly queries GCS for the most recently created model checkpoint.\"\"\"\n", + " try:\n", + " bucket = self._storage_client.get_bucket(self.finetuned_model_bucket)\n", + " blobs = list(bucket.list_blobs(prefix=self.finetuned_model_prefix))\n", + "\n", + " # Filter for actual model files and exclude the initial model if present\n", + " model_blobs = [b for b in blobs if b.name.endswith(\".pth\") and \"initial\" not in b.name]\n", + "\n", + " if not model_blobs:\n", + " return None\n", + "\n", + " # Find the blob with the latest creation time\n", + " latest_blob = max(model_blobs, key=lambda b: b.time_created)\n", + " latest_model_path = f\"gs://{self.finetuned_model_bucket}/{latest_blob.name}\"\n", + " return latest_model_path\n", + " except Exception as e:\n", + " logging.error(f\"Error querying GCS for the latest model: {e}\")\n", + " return None\n", + "\n", + " # Add the side input parameter to the process method\n", + " def process(self, batch_of_data):\n", + " logging.info(\n", + " f\"Received a batch of {len(batch_of_data)} points for finetuning.\")\n", + "\n", + " # If a finetuned model exists, use it. Otherwise, use the initial base model.\n", + " latest_model_path = self._get_latest_model_from_gcs()\n", + "\n", + " if latest_model_path:\n", + " model_to_load = latest_model_path\n", + " logging.info(f\"Continuously finetuning from latest model: {model_to_load}\")\n", + " else:\n", + " model_to_load = self.initial_model_path\n", + " logging.info(f\"No finetuned model found. Starting from initial model: {model_to_load}\")\n", + "\n", + " # batch_of_data.sort(key=lambda x: x[1]['timestamp'])\n", + " time_series_values = np.array([d['value'] for d in batch_of_data],\n", + " dtype=np.float32)\n", + " train_dataset, val_dataset = prepare_datasets(\n", + " series=time_series_values,\n", + " context_length=self.hparams.context_len,\n", + " horizon_length=self.hparams.horizon_len,\n", + " freq_type=self.config.freq_type,\n", + " train_split=0.8\n", + " )\n", + "\n", + " logging.info(f\"Training dataset size: {train_dataset.series.tolist()}\")\n", + " logging.info(f\"Validation dataset size: {val_dataset.series.tolist()}\")\n", + "\n", + " # Load the model (base or latest finetuned)\n", + " # The updated get_model function can handle both GCS and Hugging Face paths\n", + " model = get_model(\n", + " model_path=model_to_load, # Use the path we just determined\n", + " hparams=self.hparams,\n", + " load_weights=True\n", + " )\n", + "\n", + " # 4. Run fine-tuning (same as before)\n", + " finetuner = TimesFMFinetuner(model, self.config)\n", + " finetuner.finetune(train_dataset=train_dataset, val_dataset=val_dataset)\n", + "\n", + " # 5. Save and upload the new model (same as before)\n", + " from datetime import datetime\n", + " timestamp_str = datetime.utcnow().strftime('%Y%m%d%H%M%S')\n", + " model_filename = f\"timesfm_finetuned_{timestamp_str}.pth\"\n", + " local_path = f\"/tmp/{model_filename}\"\n", + " torch.save(model.state_dict(), local_path)\n", + " bucket = self._storage_client.bucket(self.finetuned_model_bucket)\n", + " blob_path = f\"{self.finetuned_model_prefix}/{model_filename}\"\n", + " blob = bucket.blob(blob_path)\n", + " blob.upload_from_filename(local_path)\n", + " logging.info(\n", + " f\"Successfully uploaded new model to gs://{self.finetuned_model_bucket}/{blob_path}\"\n", + " )\n", + " yield blob_path\n", + "\n", + "\n", + "def get_model(model_path: str, hparams: TimesFmHparams, load_weights: bool = False):\n", + " \"\"\"\n", + " Loads a TimesFM model from either a Hugging Face repo ID or a GCS path.\n", + " The `load_weights` argument is kept for signature consistency but is\n", + " effectively always True, as TimesFm handles loading.\n", + " \"\"\"\n", + " checkpoint_config = {}\n", + "\n", + " # Case 1: The model path is a GCS URI.\n", + " # We download it to a local file and tell TimesFmCheckpoint to load from that path.\n", + " if model_path.startswith(\"gs://\"):\n", + " logging.info(f\"Preparing to load model from GCS path: {model_path}\")\n", + " local_temp_path = f\"/tmp/{path.basename(model_path)}\"\n", + " with GcsIO().open(model_path, 'rb') as f_in, open(local_temp_path, 'wb') as f_out:\n", + " f_out.write(f_in.read())\n", + " # The key for a local file is 'path'\n", + " checkpoint_config['path'] = local_temp_path\n", + "\n", + " # Case 2: The model path is a Hugging Face repository ID.\n", + " else:\n", + " logging.info(f\"Preparing to load model from Hugging Face repo: {model_path}\")\n", + " # The key for a Hugging Face repo is 'huggingface_repo_id'\n", + " checkpoint_config['huggingface_repo_id'] = model_path\n", + "\n", + " # Initialize the TimesFm object correctly with the dynamically created checkpoint config.\n", + " # This single call handles model configuration and weight loading.\n", + " tfm = TimesFm(\n", + " hparams=hparams,\n", + " checkpoint=TimesFmCheckpoint(**checkpoint_config)\n", + " )\n", + "\n", + " logging.info(\"Model loaded successfully inside get_model.\")\n", + "\n", + " # The `TimesFm` object holds the configured model instance.\n", + " # The model returned here will be a PatchedTimeSeriesDecoder instance with weights loaded.\n", + " return tfm._model" + ], + "metadata": { + "id": "IzEE_R3SuwAR" + }, + "execution_count": 11, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "# Load Time Series Data\n", + "\n", + "https://www.kaggle.com/datasets/julienjta/nyc-taxi-traffic/data" + ], + "metadata": { + "id": "Lz1OROouy9IV" + } + }, + { + "cell_type": "code", + "source": [ + "import pandas as pd\n", + "from google.colab import auth\n", + "auth.authenticate_user()\n", + "\n", + "auth.authenticate_user()\n", + "\n", + "# Define the path to your file in the GCS bucket\n", + "gcs_path = 'gs://apache-beam-samples/anomaly_detection/timesfm-dataset-example/nyc_taxi_timeseries.csv'\n", + "\n", + "# Read the CSV directly from GCS into a DataFrame\n", + "# All the gspread code is replaced by this single line\n", + "df = pd.read_csv(gcs_path)\n", + "\n", + "# --- The rest of your processing code remains the same ---\n", + "\n", + "# Convert 'value' column to a numpy array of integers\n", + "values_array = pd.to_numeric(df['value'], errors='coerce').astype(int).to_numpy()\n", + "\n", + "# Create the list of (timestamp, value) tuples\n", + "input_data = []\n", + "for i in range(len(values_array)):\n", + " input_data.append((Timestamp(i + 1), values_array[i])) # Assuming Timestamp comes from pandas\n", + "\n", + "print(\"DataFrame loaded from GCS:\")\n", + "print(df.head())\n", + "print(\"\\nInput data created successfully (first 5 entries):\")\n", + "print(input_data[:5])" + ], + "metadata": { + "colab": { + "base_uri": "https://localhost:8080/" + }, + "id": "N7fAXDjXuDF4", + "outputId": "0ea32fa5-221d-44fb-9f83-4f524fd8f3c2" + }, + "execution_count": 6, + "outputs": [ + { + "output_type": "stream", + "name": "stdout", + "text": [ + "DataFrame loaded from GCS:\n", + " Unnamed: 0 timestamp value\n", + "0 0 2014-07-01 0:00:00 10844\n", + "1 1 2014-07-01 0:30:00 8127\n", + "2 2 2014-07-01 1:00:00 6210\n", + "3 3 2014-07-01 1:30:00 4656\n", + "4 4 2014-07-01 2:00:00 3820\n", + "\n", + "Input data created successfully (first 5 entries):\n", + "[(Timestamp(1), np.int64(10844)), (Timestamp(2), np.int64(8127)), (Timestamp(3), np.int64(6210)), (Timestamp(4), np.int64(4656)), (Timestamp(5), np.int64(3820))]\n" + ] + } + ] + }, + { + "cell_type": "markdown", + "source": [ + "# Beam Pipeline Setup" + ], + "metadata": { + "id": "LiQQF_IxquCK" + } + }, + { + "cell_type": "code", + "source": [ + "import apache_beam as beam\n", + "from apache_beam.options.pipeline_options import PipelineOptions\n", + "from apache_beam.pvalue import AsDict, AsSingleton\n", + "from apache_beam.transforms.periodicsequence import PeriodicImpulse\n", + "import logging\n", + "import os\n", + "import json\n", + "import timesfm\n", + "from apache_beam.utils.timestamp import Timestamp\n", + "import csv\n", + "from apache_beam.ml.inference.base import RunInference\n", + "from apache_beam.ml.inference.utils import WatchFilePattern\n", + "import typing\n", + "from google.colab import userdata\n", + "import apache_beam.transforms.window as window\n", + "\n", + "logging.getLogger().setLevel(logging.INFO)\n", + "\n", + "# --- Pipeline Configuration ---\n", + "PROJECT_ID = os.environ.get(\"GCP_PROJECT\", \"apache-beam-testing\")\n", + "REGION = os.environ.get(\"GCP_REGION\", \"us-central1\")\n", + "TEMP_LOCATION = \"gs://apache-beam-testing-temp/timesfm_anomaly_detection/temp\"\n", + "STAGING_LOCATION = \"gs://apache-beam-testing-temp/timesfm_anomaly_detection/staging\"\n", + "FINETUNED_MODEL_BUCKET = \"apache-beam-testing-temp\"\n", + "FINETUNED_MODEL_PREFIX = \"timesfm_anomaly_detection/finetuned-models/timesfm/checkpoints\"\n", + "\n", + "# --- Model & Window Parameters ---\n", + "CONTEXT_LEN = 512\n", + "HORIZON_LEN = 128\n", + "WINDOW_SIZE = CONTEXT_LEN + HORIZON_LEN\n", + "SLIDE_INTERVAL = HORIZON_LEN\n", + "EXPECTED_INTERVAL = 1\n", + "INITIAL_MODEL = \"google/timesfm-1.0-200m-pytorch\"\n", + "\n", + "MODEL_CHECK_INTERVAL_SECONDS = 10 # Check for a new model every 5 seconds\n", + "FINETUNING_BATCH_SIZE = 7680 # 9600 # make larger later. minimum is WINDOW_SIZE for validation and training\n", + "FINETUNE_CONFIG = FinetuningConfig(\n", + " batch_size=128,\n", + " num_epochs=5,\n", + " learning_rate=1e-4,\n", + " use_wandb=False,\n", + " freq_type=0, # should change based on your data\n", + " log_every_n_steps=10,\n", + " val_check_interval=0.5,\n", + " use_quantile_loss=True\n", + " )\n", + "\n", + "#Change to Dataflow if needed\n", + "options = PipelineOptions([\n", + " \"--streaming\",\n", + " \"--environment_type=LOOPBACK\",\n", + " \"--runner=PrismRunner\",\n", + " \"--logging_level=INFO\",\n", + " \"--job_server_timeout=3600\"\n", + "])\n", + "\n", + "\n", + "\n", + "# HParams for the model\n", + "hparams = timesfm.TimesFmHparams(\n", + " backend=\"gpu\",\n", + " per_core_batch_size=32,\n", + " horizon_len=HORIZON_LEN,\n", + " context_len=CONTEXT_LEN,\n", + ")\n", + "model_handler = DynamicTimesFmModelHandler(model_uri=INITIAL_MODEL, hparams=hparams)\n", + "\n", + "def print_and_pass_through(label):\n", + " def logger(element):\n", + " print(f\"--- {label} --- \\nELEMENT: %s\", element)\n", + " return element\n", + " return logger\n", + "\n", + "\n", + "class CustomJsonEncoder(json.JSONEncoder):\n", + " \"\"\"A custom JSON encoder that knows how to handle Beam's Timestamp objects.\"\"\"\n", + " def default(self, obj):\n", + " if isinstance(obj, Timestamp):\n", + " # Convert Timestamp to a standard, readable ISO 8601 string format\n", + " return obj.micros // 1e6\n", + " # For all other types, fall back to the default behavior\n", + " if isinstance(obj, np.integer):\n", + " return int(obj)\n", + "\n", + " # 3. Handle NumPy float types (this will fix your float32 error)\n", + " if isinstance(obj, np.floating):\n", + " return float(obj)\n", + "\n", + " # 4. Handle NumPy arrays\n", + " if isinstance(obj, np.ndarray):\n", + " return obj.tolist()\n", + "\n", + " # For all other types, fall back to the default behavior\n", + " return super().default(obj)\n", + " return json.JSONEncoder.default(self, obj)\n", + "\n", + "class WritePlotDataAndPassThrough(beam.DoFn):\n", + " \"\"\"\n", + " A DoFn that writes plotting data to a file as a side effect\n", + " and then passes the original, unmodified element downstream.\n", + " \"\"\"\n", + " def __init__(self, output_path):\n", + " self._output_path = output_path\n", + " self._file_handle = None\n", + "\n", + " def setup(self):\n", + " self._file_handle = open(self._output_path, 'a')\n", + "\n", + " def process(self, element):\n", + " _original_window, payload_dict = element\n", + "\n", + " # ✅ FIX: Use the custom encoder to handle Timestamp objects\n", + " json_record = json.dumps(payload_dict, cls=CustomJsonEncoder)\n", + " self._file_handle.write(json_record + '\\n')\n", + "\n", + " # Pass the original element through, with the Timestamp object intact\n", + " yield element\n", + "\n", + " def teardown(self):\n", + " if self._file_handle:\n", + " self._file_handle.close()\n", + "\n", + "\n", + "# =================================================================\n", + "# 1. Get Latest Model Path (Side Input) - WatchFilePattern is not\n", + "# currently supported on Prism. Uncomment the following to run\n", + "# on Dataflow\n", + "# =================================================================\n", + "# model_pattern = os.path.join(\n", + "# f\"gs://{FINETUNED_MODEL_BUCKET}\", FINETUNED_MODEL_PREFIX, \"*.pth\"\n", + "# )\n", + "\n", + "# model_metadata_pcoll = (\n", + "# \"WatchForNewModels\" >> WatchFilePattern(\n", + "# file_pattern=model_pattern,\n", + "# interval=MODEL_CHECK_INTERVAL_SECONDS\n", + "# )\n", + "# | \"PrintModelLocation\" >> beam.Map(print_and_pass_through(\"Model Location\"))\n", + "\n", + "# )\n", + "\n", + "# =================================================================\n", + "# Ingest and Window Raw Data\n", + "# =================================================================\n", + "\n", + "\n", + "windowed_data = (\n", + " PeriodicImpulse(data=input_data, fire_interval=0.01)\n", + " | \"AddKey\" >> beam.WithKeys(lambda x: 0)\n", + " | \"ApplySlidingWindow\" >> beam.ParDo(\n", + " OrderedSlidingWindowFn(window_size=WINDOW_SIZE, slide_interval=SLIDE_INTERVAL))\n", + " | \"FillGaps\" >> beam.ParDo(FillGapsFn(expected_interval=EXPECTED_INTERVAL)).with_output_types(\n", + " typing.Tuple[int, typing.Tuple[Timestamp, Timestamp, typing.List[float]]])\n", + " | \"Skip NaN Values for now\" >> beam.Filter(\n", + " lambda batch: 'NaN' not in batch[1][2])\n", + " | \"PrintWindowedData\" >> beam.Map(print_and_pass_through(\"Windowed Data\"))\n", + "\n", + ")\n", + "\n", + "# =================================================================\n", + "# Detect Anomalies using the Latest Model\n", + "# =================================================================\n", + "\n", + "inference_results = (\n", + " \"DetectAnomalies\" >> RunInference(\n", + " model_handler=model_handler,\n", + " # model_metadata_pcoll=model_metadata_pcoll\n", + " )\n", + " | \"PrintInference\" >> beam.Map(print_and_pass_through(\"Inference Results\"))\n", + ")\n", + "\n", + "\n", + "# NEW BRANCH: For plotting. It takes the payload dictionary, converts\n", + "# it to JSON, and writes it to a file.\n", + "plotting_data_output = (\n", + " \"WritePlotDataAsSideEffect\" >> beam.ParDo(\n", + " WritePlotDataAndPassThrough('plot_data_original.jsonl'))\n", + ")\n", + "\n", + "def format_for_llm(result_tuple):\n", + " \"\"\"\n", + " Takes the output of RunInference (a PredictionResult) and formats it\n", + " into the dictionary structure needed by the LLMClassifierFn.\n", + " \"\"\"\n", + " original_window_data, result_dict = result_tuple\n", + "\n", + " list_of_anomalies = result_dict['anomalies']\n", + "\n", + " key, (window_start_ts, _, values_array) = original_window_data\n", + "\n", + " return (key, {\n", + " 'key': key,\n", + " 'window_start_ts': window_start_ts,\n", + " 'values_array': values_array,\n", + " 'anomalies': list_of_anomalies if list_of_anomalies else []\n", + " })\n", + "\n", + "\n", + "data_for_llm = (\n", + " \"FormatForLLM\" >> beam.Map(format_for_llm)\n", + " | \"PrintDataForLLM\" >> beam.Map(print_and_pass_through(\"Data for LLM\"))\n", + ")\n", + "\n", + "\n", + "# =================================================================\n", + "# Classify with LLM and Create Clean Data for Finetuning\n", + "# =================================================================\n", + "api_key = \"AIzaSyCB_g6tq3eBFtB3BsshdGotLkUkTsCyApY\" #userdata.get('GEMINI_API_KEY')\n", + "\n", + "llm_classifier = (\n", + " \"LLMClassifierAndImputer\" >> beam.ParDo(\n", + " LLMClassifierFn(\n", + " secret=api_key,\n", + " slide_interval=SLIDE_INTERVAL,\n", + " expected_interval_secs=EXPECTED_INTERVAL\n", + " )\n", + " )\n", + " # | \"PrintLLMResults\" >> beam.Map(print_and_pass_through(\"LLM Results\"))\n", + ")\n", + "\n", + "\n", + "# # =================================================================\n", + "# # Batch Clean Data and Trigger Finetuning\n", + "# # =================================================================\n", + "finetuning_job_input = (\n", + " \"KeyForBatching\" >> beam.WithKeys(lambda _: \"finetune_batch\")\n", + " # | \"BatchAndTrigger\" >> beam.ParDo(BatchAndTriggerFinetuningFn(FINETUNING_BATCH_SIZE))\n", + " | \"BatchAndTrigger\" >> beam.ParDo(\n", + " BatchContinuousAndOrderedFn(\n", + " FINETUNING_BATCH_SIZE,\n", + " expected_interval_seconds=1\n", + " )\n", + " )\n", + " | \"PrintFinetuningJobInput\" >> beam.Map(print_and_pass_through(\"Finetuning Job Input\"))\n", + ")\n", + "\n", + "# # =================================================================\n", + "# # Run Finetuning and Save New Model to GCS\n", + "# # =================================================================\n", + "finetuning = (\n", + " \"RunFinetuning\" >> beam.ParDo(\n", + " RunFinetuningFn(\n", + " initial_model_path=\"google/timesfm-1.0-200m-pytorch\",\n", + " finetuned_model_bucket=FINETUNED_MODEL_BUCKET,\n", + " finetuned_model_prefix=FINETUNED_MODEL_PREFIX,\n", + " hparams=hparams,\n", + " config=FINETUNE_CONFIG\n", + " ),\n", + " )\n", + ")\n" + ], + "metadata": { + "id": "Oud4wLTjqy2j", + "colab": { + "base_uri": "https://localhost:8080/" + }, + "outputId": "1e0cdb8c-16e5-42ce-ef5a-b870677e954d" + }, + "execution_count": 16, + "outputs": [ + { + "output_type": "stream", + "name": "stderr", + "text": [ + "WARNING:apache_beam.transforms.core:('No iterator is returned by the process method in %s.', )\n" + ] + } + ] + }, + { + "cell_type": "markdown", + "source": [ + "# Beam Pipeline" + ], + "metadata": { + "id": "ZMx8KhRyvj3Q" + } + }, + { + "cell_type": "code", + "source": [ + "with beam.Pipeline(options=options) as p:\n", + " (p\n", + " | windowed_data\n", + " | inference_results\n", + " | plotting_data_output # comment this line if you dont want to save plot data\n", + " | data_for_llm\n", + " | llm_classifier\n", + " | finetuning_job_input\n", + " | finetuning\n", + " )\n" + ], + "metadata": { + "id": "mKa6Qb_1vnNX" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "# Plot Data (Original)" + ], + "metadata": { + "id": "O7egp_5Alzz7" + } + }, + { + "cell_type": "code", + "source": [ + "import json\n", + "import numpy as np\n", + "import matplotlib.pyplot as plt\n", + "\n", + "CONTEXT_LEN = 512\n", + "HORIZON_LEN = 128\n", + "\n", + "def plot_anomalies_and_forecast(\n", + " values_array,\n", + " all_anomalies,\n", + " all_predicted_values,\n", + " all_q20_values,\n", + " all_q30_values,\n", + " all_q70_values,\n", + " all_q80_values,\n", + " title_suffix=\"\",\n", + " x_lims=None,\n", + " min_outlier_score_for_plot=0,\n", + " context_len=512,\n", + " output_filename=\"plot.png\"\n", + "):\n", + " print(len(all_anomalies))\n", + " # The key from your file is 'outlier_score'\n", + " filtered_anomalies = [a for a in all_anomalies if a['outlier_score'] >= min_outlier_score_for_plot]\n", + " # The key from your file is 'timestamp'\n", + " anomaly_indices = [(a['timestamp'] - HORIZON_LEN) for a in filtered_anomalies]\n", + " anomaly_values = [a['actual_value'] for a in filtered_anomalies]\n", + "\n", + " Q1 = np.nanmean([all_q20_values, all_q30_values], axis=0)\n", + " Q3 = np.nanmean([all_q70_values, all_q80_values], axis=0)\n", + " IQR = Q3 - Q1\n", + " upper_thresh = Q3 + 1.5 * IQR\n", + " lower_thresh = Q1 - 1.5 * IQR\n", + "\n", + " plt.figure(figsize=(18, 9))\n", + " # This now plots the correct original data for the horizon\n", + " plt.plot(values_array[context_len:], label='Original Time Series', color='blue', alpha=0.7, linewidth=1.5)\n", + "\n", + " plt.plot(all_predicted_values, label='Predicted Mean', color='green', linestyle='--', linewidth=1.5)\n", + " plt.plot(lower_thresh, label='Lower Threshold', color='orange', linestyle=':', linewidth=1.2)\n", + " plt.plot(upper_thresh, label='Upper Threshold', color='purple', linestyle=':', linewidth=1.2)\n", + "\n", + " plt.scatter([i - context_len for i in anomaly_indices], anomaly_values,\n", + " color='red', s=70, zorder=5,\n", + " label=f'Detected Anomalies (Score >= {min_outlier_score_for_plot:.1f})',\n", + " marker='o', edgecolors='black', linewidths=0.8)\n", + "\n", + " plt.title(f'Time Series Anomaly Detection {title_suffix}')\n", + " plt.xlabel('Time Index')\n", + " plt.ylabel('Value')\n", + " if x_lims:\n", + " plt.xlim(x_lims[0], x_lims[1])\n", + " plt.legend()\n", + " plt.grid(True, linestyle='--', alpha=0.6)\n", + " plt.tight_layout()\n", + " # plt.savefig(output_filename) # Save the plot to a file\n", + " plt.show()\n", + " plt.close() # Close the figure to free memory\n", + "\n", + "# --- Main Script Logic ---\n", + "\n", + "# 1. Read and parse the data from the Beam output file\n", + "all_window_data = []\n", + "# Make sure 'plot_data.jsonl' is in the same directory as this script\n", + "try:\n", + " with open('plot_data_original.jsonl', 'r') as f:\n", + " for line in f:\n", + " # Check for empty lines that might have been added\n", + " if line.strip():\n", + " all_window_data.append(json.loads(line))\n", + "except FileNotFoundError:\n", + " print(\"Error: 'plot_data.jsonl' not found. Please make sure the file is in the correct directory.\")\n", + " exit()\n", + "\n", + "\n", + "# 2. Sort data by timestamp to ensure the correct order\n", + "all_window_data.sort(key=lambda x: x['start_ts_micros'])\n", + "\n", + "# 3. Reconstruct the full data arrays\n", + "all_anomalies = []\n", + "all_predicted_values = []\n", + "all_q20_values = []\n", + "all_q30_values = []\n", + "all_q70_values = []\n", + "all_q80_values = []\n", + "all_actual_horizon_values = [] # This will hold the real \"blue line\" data\n", + "\n", + "for window_data in all_window_data:\n", + " all_predicted_values.extend(window_data['predicted_values'])\n", + " all_q20_values.extend(window_data['q20_values'])\n", + " all_q30_values.extend(window_data['q30_values'])\n", + " all_q70_values.extend(window_data['q70_values'])\n", + " all_q80_values.extend(window_data['q80_values'])\n", + " # Populate the list with the actual values from the file\n", + " all_actual_horizon_values.extend(window_data.get('actual_horizon_values', []))\n", + " all_anomalies.extend(window_data.get('anomalies', []))\n", + "\n", + "# 4. Convert lists to NumPy arrays\n", + "all_predicted_values = np.array(all_predicted_values)\n", + "all_q20_values = np.array(all_q20_values)\n", + "all_q30_values = np.array(all_q30_values)\n", + "all_q70_values = np.array(all_q70_values)\n", + "all_q80_values = np.array(all_q80_values)\n", + "\n", + "# 5. Construct the `values_array` using the REAL data from your file\n", + "context_len = 512\n", + "# Create a dummy context so the array has the right shape for the plotting function.\n", + "# The first real value is used to make the context visually seamless.\n", + "if all_actual_horizon_values:\n", + " dummy_context = [all_actual_horizon_values[0]] * context_len\n", + " values_array = np.array(dummy_context + all_actual_horizon_values)\n", + "else:\n", + " # Fallback in case the file is empty or missing the actual_horizon_values key\n", + " print(\"Warning: 'actual_horizon_values' not found. The original time series plot will be empty.\")\n", + " total_len = context_len + len(all_predicted_values)\n", + " values_array = np.zeros(total_len)\n", + "\n", + "# 6. Call the plotting functions\n", + "if values_array.any():\n", + " # Plotting function for full graph\n", + " plot_anomalies_and_forecast(\n", + " values_array, all_anomalies, all_predicted_values,\n", + " all_q20_values, all_q30_values, all_q70_values, all_q80_values,\n", + " title_suffix=\"(Full Graph with Correct Data)\",\n", + " min_outlier_score_for_plot=1, # Set a score threshold\n", + " context_len=context_len,\n", + " output_filename=\"full_graph_correct.png\"\n", + " )\n", + "\n", + " # Plotting function for zoomed-in graphs - feel free to change\n", + " zoom_ranges = [(2000, 2500), (8300, 9000), (9000, 9600)]\n", + " for i, (start_idx, end_idx) in enumerate(zoom_ranges):\n", + " # Adjust x_lims for the fact that the plotted array is sliced by context_len\n", + " plot_x_start = max(0, start_idx)\n", + " plot_x_end = end_idx\n", + "\n", + " plot_anomalies_and_forecast(\n", + " values_array, all_anomalies, all_predicted_values,\n", + " all_q20_values, all_q30_values, all_q70_values, all_q80_values,\n", + " title_suffix=f\"(Zoomed In: {start_idx} to {end_idx})\",\n", + " x_lims=(plot_x_start, plot_x_end),\n", + " min_outlier_score_for_plot=1,\n", + " context_len=context_len,\n", + " output_filename=f\"zoomed_graph_correct_{i}.png\"\n", + " )\n", + " print(\"Plots have been generated and saved with the corrected original data.\")\n", + "else:\n", + " print(\"No data found to plot.\")" + ], + "metadata": { + "colab": { + "base_uri": "https://localhost:8080/", + "height": 1000 + }, + "id": "_HzqIoZbl25b", + "outputId": "7f85032d-e2cf-4e7c-b9b1-cae9994dee37" + }, + "execution_count": 18, + "outputs": [ + { + "output_type": "stream", + "name": "stdout", + "text": [ + "948\n" + ] + }, + { + "output_type": "display_data", + "data": { + "text/plain": [ + "

" + ], + "image/png": "iVBORw0KGgoAAAANSUhEUgAABvsAAAN5CAYAAAAmV4erAAAAOnRFWHRTb2Z0d2FyZQBNYXRwbG90bGliIHZlcnNpb24zLjEwLjAsIGh0dHBzOi8vbWF0cGxvdGxpYi5vcmcvlHJYcgAAAAlwSFlzAAAPYQAAD2EBqD+naQABAABJREFUeJzs3XdYFMf/B/D3cVSpIiiiVBuoiL0rdozla9eoUYk1KrEbTTQKsccaY+y9GzWx94K9K4gNEcGu2EB6uZvfH/y4eFLVgwXu/Xoennizs7Of2d37BG5uZ2RCCAEiIiIiIiIiIiIiIiIiynd0pA6AiIiIiIiIiIiIiIiIiL4MB/uIiIiIiIiIiIiIiIiI8ikO9hERERERERERERERERHlUxzsIyIiIiIiIiIiIiIiIsqnONhHRERERERERERERERElE9xsI+IiIiIiIiIiIiIiIgon+JgHxEREREREREREREREVE+xcE+IiIiIiIiIiIiIiIionyKg31ERERERERERERERERE+RQH+4iIiIgoz/Py8oKjo6PUYUhG2/ufk9auXQuZTIawsDCpQ8nzwsLCIJPJsHbtWsliaNWqFQYMGJAjbafXPx8fH8hkshw5Xm5Kvc+vXr2aq8d1dHSEl5dXtuu2adMmZwMiSY0fPx61atWSOgwiIiKiAomDfUREREQkCZlMlq0fPz8/qUNNV1hYGL7//nuUKlUKhoaGsLGxQcOGDTF58mSpQ9O4mjVrQiaTYcmSJVKHkud5eXmp3b8mJiZwdnZG586dsXPnTiiVyi9u+8CBA/Dx8dFcsBnYvHkzFixYkOPH+Vznzp3DkSNHMG7cOFWZn59fhrnj22+/zfUYz5w5g65du6JEiRLQ19eHubk5atWqhd9++w2vXr3K9Xjymjt37sDHxyfHBtfj4+Mxf/581KpVC+bm5jA0NETZsmXh7e2N+/fv58gxc9r06dOxa9eubNVNHbBO/dHT04OVlRXq1q2LX375BY8fP/7iOJ4/fw4fHx/4+/t/cRsjRoxAQEAA9uzZ88VtEBEREVH6dKUOgIiIiIi004YNG9Rer1+/HkePHk1T7urqihUrVnzVIImmPXjwADVq1ICRkRH69u0LR0dHvHjxAtevX8esWbPg6+ur0eNJ2f/g4GBcuXIFjo6O2LRpEwYPHixJHPmJgYEBVq5cCQCIi4vDo0ePsHfvXnTu3BmNGjXC7t27YWZm9tntHjhwAH/99VeOD/ht3rwZt27dwogRI9TKHRwcEBcXBz09vRw9fkZmz56Npk2bonTp0mm2DRs2DDVq1FAry+2nYSdNmoQpU6bA2dkZXl5ecHZ2Rnx8PK5du4a5c+di3bp1CAkJydWYpBYUFAQdnf++Y3znzh34+vqiUaNGGr8+b968QcuWLXHt2jW0adMGPXr0gImJCYKCgrB161YsX74ciYmJGj1mbpg+fTo6d+6M9u3bZ3uf7t27o1WrVlAqlXj//j2uXLmCBQsW4I8//sCqVau+aCD8+fPn8PX1haOjIypXrvzZ+wOAjY0N2rVrhzlz5uB///vfF7VBREREROnjYB8RERERSeK7775Te33x4kUcPXo0TXleNH/+fERHR8Pf3x8ODg5q28LDwzV2nJiYGBgbG0s2uAIAGzduRNGiRTF37lx07twZYWFhnFI0C7q6umnu46lTp2LmzJn4+eefMWDAAGzbtk2i6L6cTCaDoaGhJMcODw/H/v37sXTp0nS3N2jQAJ07d87lqP6zbds2TJkyBV27dsWGDRugr6+vtn3+/PmYP39+pm0IIRAfHw8jI6OcDDVXGRgY5NqxvLy8cOPGDezYsQOdOnVS2zZlyhRMmDBBI8dJzcufykvXr2rVqmly0KNHj9CiRQv06dMHrq6ucHd3lyS2rl27okuXLnj48CGcnZ0liYGIiIioIOI0nkRERESU5326Zl3qVGVz5szBX3/9BWdnZxQqVAgtWrTAkydPIITAlClTULJkSRgZGaFdu3Z49+5dmnYPHjyIBg0awNjYGKampmjdujVu376dZTwhISEoWbJkmoE+AChatOgXHcfLywsmJiYICQlBq1atYGpqip49e6bbfwBQKpVYsGABKlSoAENDQxQrVgyDBg3C+/fv1epdvXoVnp6esLKygpGREZycnNC3b98s+5hq8+bN6Ny5M9q0aQNzc3Ns3rw5TZ3Udc0ePHgALy8vWFhYwNzcHN9//z1iY2PV6iYnJ2PKlCkoVaoUDAwM4OjoiF9++QUJCQlq9VLX7/Lz80P16tVhZGQENzc31bSu//zzD9zc3GBoaIhq1arhxo0bavvfvHlT9XRV6jSrffv2xdu3bzPtb58+fWBlZYWkpKQ021q0aIFy5cpl57Sla/z48WjRogW2b9+eZkrBrO4RLy8v/PXXXwDUp8BNld37IfVYHh4eMDU1hZmZGWrUqKG6ro0aNcL+/fvx6NEj1TFS772M1uw7ceKEKnYLCwu0a9cOd+/eVavzOfdIevbv34/k5GQ0a9Ysy7qfymjduEaNGqFRo0af3V56Jk2aBCsrK6xatSrNQB8AmJubp3kiM/UeP3z4sOoeX7ZsGQBgzZo1aNKkCYoWLQoDAwOUL18+3Wl0U9s4cuQIKleuDENDQ5QvXx7//PNPunEmJCRg1KhRsLa2hrGxMTp06IDXr19n2rc9e/ZAJpPh5s2bqrKdO3dCJpOhY8eOanVdXV3RrVs3tfhSz/3atWvRpUsXAEDjxo0znKr57NmzqFmzJgwNDeHs7Iz169dnGh8AXLp0Cfv370e/fv3SDPQBKYOOc+bMUSv7nPv2zp076NGjBwoXLoz69eur+pbR9YuIiMCIESNgZ2cHAwMDlC5dGrNmzUrzhLZSqcQff/yhymXW1tZo2bKlam1FmUyGmJgYrFu3TnW+srsG4qccHBywdu1aJCYm4vfff1eVv3v3DmPGjIGbmxtMTExgZmaGb775BgEBAao6fn5+qidnv//+e1UsqbngzJkz6NKlC+zt7WFgYAA7OzuMHDkScXFxaeJIfQ/v3r37i/pBREREROnjYB8RERER5VubNm3C4sWL8eOPP2L06NE4deoUunbtiokTJ+LQoUMYN24cBg4ciL1792LMmDFq+27YsAGtW7eGiYkJZs2ahV9//RV37txB/fr1s1xPysHBAU+ePMGJEyeyjPFzjpOcnAxPT08ULVoUc+bMSfdD61SDBg3C2LFjUa9ePfzxxx/4/vvvsWnTJnh6eqoGqsLDw9GiRQuEhYVh/Pjx+PPPP9GzZ09cvHgxy7iBlA/QHzx4gO7du0NfXx8dO3bEpk2bMqzftWtXREVFYcaMGejatSvWrl2bZkrT/v37Y9KkSahatSrmz58PDw8PzJgxI91p5R48eIAePXqgbdu2mDFjBt6/f4+2bdti06ZNGDlyJL777jv4+voiJCQEXbt2Vfsg/ejRo3j48CG+//57/Pnnn/j222+xdetWtGrVCkKIDPvQq1cvvH37FocPH1Yrf/nyJU6cOPHVT5726tULQggcPXpUVZade2TQoEFo3ry5qn7qT6rs3A9AyoBL69at8e7dO/z888+YOXMmKleujEOHDgEAJkyYgMqVK8PKykp1jMzW7zt27Bg8PT0RHh4OHx8fjBo1CufPn0e9evXSfR9l5x5Jz/nz51GkSJF0B9gBICoqCm/evFH7ya2pb+/fv4/79++jffv2MDEx+ax9g4KC0L17dzRv3hx//PGHanrEJUuWwMHBAb/88gvmzp0LOzs7DBkyRDXg+7Hg4GB069YN33zzDWbMmAFdXV106dJF7R5L9eOPPyIgIACTJ0/G4MGDsXfvXnh7e2caY/369SGTyXD69GlV2ZkzZ6Cjo4OzZ8+qyl6/fo179+6hYcOG6bbTsGFDDBs2DADwyy+/qO4vV1dXVZ0HDx6gc+fOaN68OebOnYvChQvDy8sryy9hpK4B16tXr0zrpfrc+7ZLly6IjY3F9OnTMWDAAFV5etcvNjYWHh4e2LhxI3r37o2FCxeiXr16+PnnnzFq1Ci1dvv166caFJw1axbGjx8PQ0NDVY7esGEDDAwM0KBBA9X5GjRoULb6mJ46deqgVKlSavfGw4cPsWvXLrRp0wbz5s3D2LFjERgYCA8PDzx//hxAyiDub7/9BgAYOHCgKpbUa719+3bExsZi8ODB+PPPP+Hp6Yk///wTvXv3ThODubk5SpUqhXPnzn1xP4iIiIgoHYKIiIiIKA8YOnSoyOjX0z59+ggHBwfV69DQUAFAWFtbi4iICFX5zz//LAAId3d3kZSUpCrv3r270NfXF/Hx8UIIIaKiooSFhYUYMGCA2nFevnwpzM3N05R/6tatW8LIyEgAEJUrVxbDhw8Xu3btEjExMWr1Puc4ffr0EQDE+PHjs+z/mTNnBACxadMmtXqHDh1SK//3338FAHHlypVM+5MRb29vYWdnJ5RKpRBCiCNHjggA4saNG2r1Jk+eLACIvn37qpV36NBBFClSRPXa399fABD9+/dXqzdmzBgBQJw4cUJV5uDgIACI8+fPq8oOHz4sAAgjIyPx6NEjVfmyZcsEAHHy5ElVWWxsbJr+bNmyRQAQp0+fVpWtWbNGABChoaFCCCEUCoUoWbKk6Natm9q+8+bNEzKZTDx8+DC9U6XSp08fYWxsnOH2GzduCABi5MiRQojPu0cyeo9k936IiIgQpqamolatWiIuLk6tbuo1FkKI1q1bq91vqVLfd2vWrFGVVa5cWRQtWlS8fftWVRYQECB0dHRE7969VWXZvUcyUr9+fVGtWrU05SdPnhQA0v1JvaYODg6iT58+afb18PAQHh4emfYvNe7M7N69WwAQCxYsUCtXKpXi9evXaj8f56XUe/zQoUNp2kzv/vX09BTOzs5qZalt7Ny5U1UWGRkpihcvLqpUqaIqS73PmzVrpnatR44cKeRyuVoeTU+FChVE165dVa+rVq0qunTpIgCIu3fvCiGE+OeffwQAERAQoBbfx+d++/btad6rn/bl4/dneHi4MDAwEKNHj840vg4dOggA4v3795nWS/W592337t0zjPfT6zdlyhRhbGws7t+/r1Y+fvx4IZfLxePHj4UQQpw4cUIAEMOGDUvT9sfXyNjYON37Nz2p9/Ds2bMzrNOuXTsBQERGRgohhIiPjxcKhSJNOwYGBuK3335TlV25ciXN+yNVevfrjBkzhEwmU8vVqVq0aCFcXV2z1SciIiIiyh4+2UdERERE+VaXLl1gbm6uel2rVi0AKesB6urqqpUnJibi2bNnAFKe+oqIiED37t3VngSSy+WoVasWTp48melxK1SoAH9/f3z33XcICwvDH3/8gfbt26NYsWJYsWKFqt6XHGfw4MFZ9nv79u0wNzdH8+bN1dqtVq0aTExMVO1aWFgAAPbt25futJSZSU5OxrZt29CtWzfVdJGp0wpm9HTfDz/8oPa6QYMGePv2LT58+AAAOHDgAACkebpl9OjRAFKmavxY+fLlUadOHdXr1OvbpEkT2Nvbpyl/+PChquzjdbPi4+Px5s0b1K5dGwBw/fr1DPuto6ODnj17Ys+ePYiKilKVb9q0CXXr1oWTk1OG+2ZH6pNfqW1/7b0IZP9+OHr0KKKiolRPD33s4ylBs+vFixfw9/eHl5cXLC0tVeWVKlVC8+bNVdf7Y1ndIxl5+/YtChcunOH2SZMm4ejRo2o/NjY2n9mjL5Ma+6dP9UVGRsLa2lrtx9/fX62Ok5MTPD0907T58f0bGRmJN2/ewMPDAw8fPkRkZKRaXVtbW3To0EH12szMDL1798aNGzfw8uVLtboDBw5Uu9YNGjSAQqHAo0ePMu1jgwYNcObMGQAp925AQAAGDhwIKysrVfmZM2dgYWGBihUrZtpWZsqXL48GDRqoXltbW6NcuXJq7+30pF4DU1PTLI+hifs2VXrXb/v27WjQoAEKFy6s9n5s1qwZFAqF6gnJ1KlQJ0+enKbdL3k/ZtenOcjAwAA6OikfDSkUCrx9+xYmJiYoV65cprnyYx/frzExMXjz5g3q1q0LIUSaKZYBqM4NEREREWmObtZViIiIiIjypo8HfACoBv7s7OzSLU9dvyw4OBhAyqBReszMzLI8dtmyZbFhwwYoFArcuXMH+/btw++//46BAwfCyckJzZo1++zj6OrqomTJklkeOzg4GJGRkemuDwikTN8JAB4eHujUqRN8fX0xf/58NGrUCO3bt0ePHj1gYGCQ6TGOHDmC169fo2bNmnjw4IGqvHHjxtiyZQtmzZql+oA41afXI3Vw5v379zAzM8OjR4+go6OD0qVLq9WzsbGBhYVFmgGHL72+QMo6VL6+vti6davqfKT6dLDkU71798asWbPw77//onfv3ggKCsK1a9ewdOnSTPfLjujoaAD/DUpo4l7M7v0QEhICAF81GPOx1OuV3jqGrq6uOHz4MGJiYmBsbKwqz+oeyYzIZPpVNze3L1rPTxNSr2XqtU1lYmKimi7xyJEjmD17dpp9Mxo8PnfuHCZPnowLFy6kWdMwMjJS7UsOpUuXTjM4VLZsWQAp6yx+POiZ2fnPTIMGDbB06VI8ePAAISEhkMlkqFOnjmoQcMCAAThz5gzq1auXJi98jk/jS40xq/hS752oqCjVlxwy8iX3bUbXKb3y4OBg3Lx5E9bW1unu8/H70dbWVm3AMTd8moNS1w1cvHgxQkNDoVAoVHWLFCmSrTYfP36MSZMmYc+ePWmuVXr5VgiRowOaRERERNqIg31ERERElG/J5fLPKk8dLEhdy2vDhg3pPv3z8VOB2YnBzc0Nbm5uqFOnDho3boxNmzahWbNmn32cj5+wyIxSqcz0CbvUD5llMhl27NiBixcvYu/evTh8+DD69u2LuXPn4uLFi5muL5badteuXdPdfurUKTRu3FitLKvzniq7H/J+6fUFUuI+f/48xo4di8qVK8PExARKpRItW7bMci238uXLo1q1aqo1tzZu3Ah9ff0Mz8XnuHXrFgCoBjw1cS9m937IC7J7j3yqSJEiWQ74ZCSj+02hUGQYz+dwcXEB8N+1TaWrq6sagHz69Gm6+378RFSqkJAQNG3aFC4uLpg3bx7s7Oygr6+PAwcOYP78+V+1FuGXnv/69esDAE6fPo2HDx+iatWqMDY2RoMGDbBw4UJER0fjxo0bmDZt2hfH9jXxpV6DwMBAtScDNSW965RRuVKpRPPmzfHTTz+lu0/qQKxUbt26haJFi6oGSKdPn45ff/0Vffv2xZQpU2BpaQkdHR2MGDEiW/eaQqFA8+bN8e7dO4wbNw4uLi4wNjbGs2fP4OXllW4b79+/h5WVlcb7RkRERKTNONhHRERERFqnVKlSAICiRYtq9Gmg6tWrA0iZJi4nj1OqVCkcO3YM9erVy/BD6I/Vrl0btWvXxrRp07B582b07NkTW7duRf/+/dOtHxMTg927d6Nbt27o3Llzmu3Dhg3Dpk2b0gz2ZcXBwQFKpRLBwcFwdXVVlb969QoRERFwcHD4rPYy8v79exw/fhy+vr6YNGmSqjz1Kbrs6N27N0aNGoUXL15g8+bNaN26dabTSGbXhg0bIJPJ0Lx5cwCfd49kNGiV3fsh9Vi3bt1K83Rldo7zqdTrFRQUlGbbvXv3YGVlpfZ01NdwcXHBzp07v2jfwoULIyIiIk35o0eP4Ozs/JWRpTwhVqZMGezatQsLFiz46j7v3bsXCQkJ2LNnj9qTbhlN6frgwYM0T0rdv38fAODo6PhVsaSyt7eHvb09zpw5g4cPH6oG1Bo2bIhRo0Zh+/btUCgUaNiwYabt5NTTXG3btsWMGTOwcePGLAf7cvq+LVWqFKKjo7N8P5cqVQqHDx/Gu3fvMn26T5Pn7MKFCwgJCcF3332nKtuxYwcaN26MVatWqdWNiIhQG5DLKI7AwEDcv38f69atQ+/evVXlqU+1pic0NBTu7u5f2g0iIiIiSgfX7CMiIiIirePp6QkzMzNMnz493bXsXr9+nen+Z86cSXe/1LWeUqeH+9rjZKRr165QKBSYMmVKmm3JycmqgY3379+neSKmcuXKAICEhIQM2//3338RExODoUOHonPnzml+2rRpg507d2baRnpatWoFAFiwYIFa+bx58wAArVu3/qz2MpL6dNCnff/0uJnp3r07ZDIZhg8fjocPH6p9OP6lZs6ciSNHjqBbt24oU6YMgM+7R1IHID4duMru/dCiRQuYmppixowZiI+PV6v38bkyNjbOcqpTAChevDgqV66MdevWqcV069YtHDlyRHW9NaFOnTp4//59lmu3padUqVK4ePEiEhMTVWX79u3DkydPNBafj48P3rx5gwEDBqR7HbN6Mu1j6d2/kZGRWLNmTbr1nz9/jn///Vf1+sOHD1i/fj0qV66s0XULGzRogBMnTuDy5cuqAbXKlSvD1NQUM2fOhJGREapVq5ZpGxndw1+rTp06aNmyJVauXIldu3al2Z6YmIgxY8YAyPn7tmvXrrhw4QIOHz6cZltERASSk5MBAJ06dYIQAr6+vmnqffp+1MT5evToEby8vKCvr4+xY8eqyuVyeZr7c/v27ao1bj+OI7UPH0vvfhVC4I8//kg3jsjISISEhKBu3bpf3BciIiIiSotP9hERERGR1jEzM8OSJUvQq1cvVK1aFd9++y2sra3x+PFj7N+/H/Xq1cOiRYsy3H/WrFm4du0aOnbsiEqVKgEArl+/jvXr18PS0hIjRozQyHEy4uHhgUGDBmHGjBnw9/dHixYtoKenh+DgYGzfvh1//PEHOnfujHXr1mHx4sXo0KEDSpUqhaioKKxYsQJmZmaZfqC9adMmFClSJMMPY//3v/9hxYoV2L9/Pzp27JjtuN3d3dGnTx8sX74cERER8PDwwOXLl7Fu3Tq0b9/+s58UzIiZmRkaNmyI33//HUlJSShRogSOHDmC0NDQbLdhbW2Nli1bYvv27bCwsPisgcjk5GRs3LgRABAfH49Hjx5hz549uHnzJho3bozly5erxZrdeyR1IGXYsGHw9PSEXC7Ht99+m+37wczMDPPnz0f//v1Ro0YN9OjRA4ULF0ZAQABiY2Oxbt061XG2bduGUaNGoUaNGjAxMUHbtm3T7evs2bPxzTffoE6dOujXrx/i4uLw559/wtzcHD4+Ptk+Z1lp3bo1dHV1cezYMQwcOPCz9u3fvz927NiBli1bomvXrggJCcHGjRtVTzpqQo8ePXDr1i3MmDEDly9fxrfffgsnJyfExMTg1q1b2LJlC0xNTbP1dGiLFi2gr6+Ptm3bYtCgQYiOjsaKFStQtGhR1VPDHytbtiz69euHK1euoFixYli9ejVevXqV4eDgl2rQoAE2bdoEmUymmtZTLpejbt26OHz4MBo1agR9ff1M26hcuTLkcjlmzZqFyMhIGBgYoEmTJhmuN/k51q9fjxYtWqBjx45o27YtmjZtCmNjYwQHB2Pr1q148eIF5syZAyBn79uxY8diz549aNOmDby8vFCtWjXExMQgMDAQO3bsQFhYGKysrNC4cWP06tULCxcuRHBwsGqK4TNnzqBx48bw9vYGkPJ+PHbsGObNmwdbW1s4OTmhVq1amcZw/fp1bNy4EUqlEhEREbhy5Qp27twJmUyGDRs2qP6/BQBt2rTBb7/9hu+//x5169ZFYGAgNm3alOap11KlSsHCwgJLly6FqakpjI2NUatWLbi4uKBUqVIYM2YMnj17BjMzM+zcuTPDaXePHTsGIQTatWv3VeeZiIiIiD4hiIiIiIjygKFDh4qMfj3t06ePcHBwUL0ODQ0VAMTs2bPV6p08eVIAENu3b1crX7NmjQAgrly5kqa+p6enMDc3F4aGhqJUqVLCy8tLXL16NdNYz507J4YOHSoqVqwozM3NhZ6enrC3txdeXl4iJCQkTf3sHKdPnz7C2Ng4W/1PtXz5clGtWjVhZGQkTE1NhZubm/jpp5/E8+fPhRBCXL9+XXTv3l3Y29sLAwMDUbRoUdGmTZtM+/fq1Suhq6srevXqlWGd2NhYUahQIdGhQwchhBCTJ08WAMTr16/V6qWe99DQUFVZUlKS8PX1FU5OTkJPT0/Y2dmJn3/+WcTHx6vt6+DgIFq3bp3m2ADE0KFD1crSux+ePn0qOnToICwsLIS5ubno0qWLeP78uQAgJk+enGmMqf7++28BQAwcODDDc/GpPn36CACqn0KFCglHR0fRqVMnsWPHDqFQKNLdLzv3SHJysvjxxx+FtbW1kMlkad4vWd0Pqfbs2SPq1q0rjIyMhJmZmahZs6bYsmWLant0dLTo0aOHsLCwEABU917qeV6zZo1ae8eOHRP16tVTtde2bVtx584dtTqfc49k5H//+59o2rRpmvOW3nv+U3PnzhUlSpQQBgYGol69euLq1avCw8NDeHh4qOqk17/UuLPLz89PdO7cWRQvXlzo6ekJMzMzUb16dTF58mTx4sULtboZ3eNCpFyjSpUqCUNDQ+Ho6ChmzZolVq9eneZcpbZx+PBhUalSJWFgYCBcXFw+KwcCECdPnsyyb7dv3xYAhKurq1r51KlTBQDx66+/ptnHwcFB9OnTR61sxYoVwtnZWcjlcrVjZ3Q+Pr1OmYmNjRVz5swRNWrUECYmJkJfX1+UKVNG/Pjjj+LBgwdqdb/mvs0sXiGEiIqKEj///LMoXbq00NfXF1ZWVqJu3bpizpw5IjExUVUvOTlZzJ49W7i4uAh9fX1hbW0tvvnmG3Ht2jVVnXv37omGDRsKIyMjASDN+fxY6j2c+qOrqyssLS1FrVq1xM8//ywePXqUZp/4+HgxevRoUbx4cWFkZCTq1asnLly4kO553717tyhfvrzQ1dVVe6/cuXNHNGvWTJiYmAgrKysxYMAAERAQkG6+6Natm6hfv36GfSAiIiKiLyMT4jPmEyEiIiIiItISu3fvRvv27XH69Oks1wGjnHfmzBk0atQI9+7dU02Dqu0cHR1RsWJF7Nu3T+pQiLL08uVLODk5YevWrXyyj4iIiEjDuGYfERERERFROlasWAFnZ2fVlIUkrQYNGqBFixb4/fffpQ6FiL7AggUL4ObmxoE+IiIiohzANfuIiIiIiIg+snXrVty8eRP79+/HH3/8AZlMJnVI9P8OHjwodQhE9IVmzpwpdQhEREREBRYH+4iIiIiIiD7SvXt3mJiYoF+/fhgyZIjU4RARERERERFlimv2EREREREREREREREREeVTXLOPiIiIiIiIiIiIiIiIKJ/iNJ4SUSqVeP78OUxNTbkGCBERERERERERERERkZYQQiAqKgq2trbQ0fn65/I42CeR58+fw87OTuowiIiIiIiIiIiIiIiISAJPnjxByZIlv7odDvZJxNTUFEDKhTQzM5M4mrxPoVDg9u3bqFChAuRyudThEBHlKOY8ItImzHlEpE2Y84hImzDnEZE2+dyc9+HDB9jZ2anGir4WB/skkjp1p5mZGQf7skGhUMDExARmZmb85YCICjzmPCLSJsx5RKRNmPOISJsw5xGRNvnSnKepZd6+fiJQIiIiIiIiIiIiIiIiIpIEB/soX5DJZLC0tNTYKDcRUV7GnEdE2oQ5j4i0CXMeEWkT5jwi0iZS5zyZEEJIcmQt9+HDB5ibmyMyMpLTeBIREREREREREREREWkJTY8Rcc2+PE6hUCApKUnqMCSnVCrx8uVL2NjYQEeHD6QSSUlPT49z7ecwpVKJp0+fomTJksx5RFTgMecRkTZhziMibcKcR0TaROqcx8G+PEoIgZcvXyIiIkLqUPIEIQSSkpIQHx/PR/+J8gALCwvY2Njw/ZhDhBB49+4dSpQoIXUoREQ5jjmPiLQJcx4RaRPmPCLSJlLnPA725VGpA31FixZFoUKFtP4DdSEE4uPjYWhoqPXngkhKQgjExsYiPDwcAFC8eHGJIyIiIiIiIiIiIiLSbhzsy4MUCoVqoK9IkSJSh5MnCCEghOBgH1EeYGRkBAAIDw9H0aJFOaUnERERERERERERkYQ4WXIelLpGX6FChSSOJG/R09OTOgQi+n+p+YlriuYMmUzGaVKJSGsw5xGRNmHOIyJtwpxHRNpE6pzHJ/vyMP6P8D8ymYyDfUR5CPNTztLR0YGNjY3UYRAR5QrmPCLSJsx5RKRNmPOISJtInfP4ZB/lC6lr9gkhpA6FiCjHKRQKhISEQKFQSB0KEVGOY84jIm3CnEdE2oQ5j4i0idQ5j4N9lG8olUqpQyAiyjVRUVFSh0BElGuY84hImzDnEZE2Yc4jIm0iZc7jYB/lGWFhYZDJZPD398/2PmvXroWFhYXkcXyqUaNGGDFihMZiygt8fHxQuXJlqcMgIiIiIiIiIiIiIqKPcLCPNOrJkyfo27cvbG1toa+vDwcHBwwfPhxv377Ncl87Ozu8ePECFStWzPbxunXrhvv3739NyJ8ldSAws5+1a9fin3/+wZQpU3ItrlQKhQIzZ86Ei4sLjIyMYGlpiVq1amHlypVf3faYMWNw/PhxDURJRERERERERERERESaoit1AFRwPHz4EHXq1EHZsmWxZcsWODk54fbt2xg7diwOHjyIixcvwtLSMt19ExMToa+vn+kClvr6+mnKjIyMYGRkpLE+ZCV1QDLVnDlzcOjQIRw7dkxVZm5unqsxfczX1xfLli3DokWLUL16dXz48AFXr17F+/fvv7hNIQQUCgVMTExgYmKiwWiJKCMymQx2dnaQyWRSh0JElOOY84hImzDnEZE2Yc4jIm0idc7jk335gBBAfLw0P0JkP86hQ4dCX18fR44cgYeHB+zt7fHNN9/g2LFjePbsGSZMmKCq6+joiClTpqB3794wMzPDwIED050+c8+ePShTpgyMjIzQvHlzrF+/HjKZDBEREQDSTuOZOtXkhg0b4OjoCHNzc3z77bdqc+UeOnQI9evXh4WFBYoUKYI2bdogJCQkW32Uy+WwsbFR/ZiYmEBXV1etzMjIKM00no6Ojpg6dSp69+4NExMTODg4YM+ePXj9+jXatWsHExMTVKpUCVevXlU73tmzZ9GgQQMYGRnBzs4Ow4YNQ0xMTIbx7dmzB0OGDEGXLl3g5OQEd3d39OvXD2PGjFHVUSqVmDFjBpycnGBkZAR3d3fs2LFDtd3Pzw8ymQwHDx5EtWrVYGBggLNnz6Y7jefKlSvh6uoKQ0NDuLi4YPHixaptiYmJ8Pb2RvHixWFoaAgHBwfMmDEjW+eZSNvp6OigSJEi0NHh/6aJqOBjziMibcKcR0TahDmPiLSJ1DmPT/blAwkJQJcu0hx7+3bA0DDreu/evcPhw4cxbdq0NE+12djYoGfPnti2bRsWL16sGtmeM2cOJk2ahMmTJ6fbZmhoKDp37ozhw4ejX79+uHz5Mn755ZcsYwkJCcGuXbuwb98+vH//Hl27dsXMmTMxbdo0AEBMTAxGjRqFSpUqITo6GpMmTUKHDh3g7++fo2/E+fPnY/r06fj1118xf/589OrVC3Xr1kXfvn0xe/ZsjBs3Dr1798bt27chk8kQEhKCli1bYurUqVi9ejVev34Nb29veHt7Y82aNekew8bGBidOnMCQIUNgbW2dbp0ZM2Zg48aNWLp0KcqUKYPTp0/ju+++g7W1NTw8PFT1xo8fjzlz5sDZ2RmFCxeGn5+fWjubNm3CpEmTsGjRIlSpUgU3btzAgAEDYGxsjD59+mDhwoXYs2cP/v77b9jb2+PJkyd48uSJxs4nUUGmUCgQHByMMmXKQC6XSx0OEVGOYs4jIm3CnEdE2oQ5j4i0idQ5j4N9pBHBwcEQQsDV1TXd7a6urnj//j1ev36NokWLAgCaNGmC0aNHq+qEhYWp7bNs2TKUK1cOs2fPhhAC9vb2CAoKwvTp0zONRalUYu3atTA1NQUA9OrVC8ePH1cN9nXq1Emt/urVq2FtbY07d+581nqBn6tVq1YYNGgQAGDSpElYsmQJatSogS7/P5I7btw41KlTB69evYKNjQ1mzJiBnj17qp4QLFOmDBYuXAgPDw8sWbIEhumMws6bNw+dO3eGjY0NKlSogLp166Jdu3b45ptvAAAJCQmYPn06jh07hjp16gAAnJ2dcfbsWSxbtkxtsO+3335D8+bNM+zP5MmTMXfuXHTs2BEA4OTkhDt37mDZsmXo06cPHj9+jDJlyqB+/fqQyWRwcHD4+pNIpEXi4+OlDoGIKNcw5xGRNmHOIyJtwpxHRNpEypzHwb58wMAg5Qk7qY79OcRnzPtZvXr1TLcHBQWhRo0aamU1a9bMsl1HR0fVQB8AFC9eHOHh4arXwcHBmDRpEi5duoQ3b95AqVQCAB4/fpyjg32VKlVS/btYsWIAADc3tzRl4eHhsLGxQUBAAG7evIlNmzap6gghoFQqERoamu7Aavny5XHr1i1cu3YN586dw+nTp9G2bVt4eXlh5cqVePDgAWJjY9MM4iUmJqJKlSpqZZldn5iYGISEhKBfv34YMGCAqjw5ORnm5uYAAC8vLzRv3hzlypVDy5Yt0aZNG7Ro0SLL80RERERERERERERERNnHwb58QCbL3lSaUipdujRkMhnu3r2LDh06pNl+9+5dFC5cWG1qSWNj4xyJRU9PT+21TCZTDegBQNu2beHg4IAVK1bA1tYWSqUSFStWRGJiYo7Ek15cqVOZpleWGmt0dDQGDRqEYcOGpWnL3t4+w+Po6OigRo0aqFGjBkaMGIGNGzeiV69emDBhAqKjowEA+/fvR4kSJdT2M/hkZDez65PazooVK1CrVi21bamPKFetWhWhoaE4ePAgjh07hq5du6JZs2Zq6wMSEREREREREREREdHX4WAfaUSRIkXQvHlzLF68GCNHjlRbt+/ly5fYtGkTevfurRrQyo5y5crhwIEDqtcGBga4cuXKV8X59u1bBAUFYcWKFWjQoAEA4OzZs1/VZk6pWrUq7ty5g9KlS39VO+XLlweQ8jRe+fLlYWBggMePH6tN2fm5ihUrBltbWzx8+BA9e/bMsJ6ZmRm6deuGbt26oXPnzmjZsiXevXsHS0vLLz42kTbQ0dGBs7MzFzEnIq3AnEdE2oQ5j4i0CXMeEWkTqXMeB/tIYxYtWoS6devC09MTU6dOhZOTE27fvo2xY8eiRIkSqjXzsmvQoEGYN28exo0bh379+sHf3x/r1q0DgM8aNPxY4cKFUaRIESxfvhzFixfH48ePMX78+C9qK6eNGzcOtWvXhre3N/r37w9jY2PcuXMHR48exaJFi9Ldp3PnzqhXrx7q1q0LGxsbhIaG4ueff0bZsmXh4uICXV1djBkzBiNHjoRSqUT9+vURGRmJc+fOwczMDH369Ml2fL6+vhg2bBjMzc3RsmVLJCQk4OrVq3j//j1GjRqFefPmoXjx4qhSpQp0dHSwfft22NjYwMLCQkNniKjgkslkMDMzkzoMIqJcwZxHRNqEOY+ItAlzHhFpE6lzHr9WQRpTpkwZXL16Fc7OzujatStKlSqFgQMHonHjxrhw4cJnP83l5OSEHTt24J9//kGlSpXw119/4ZdffgGQdsrJ7NLR0cHWrVtx7do1VKxYESNHjsTs2bO/qK2cVqlSJZw6dQr3799HgwYNUKVKFUyaNAm2trYZ7uPp6Ym9e/eibdu2KFu2LPr06QMXFxccOXIEuropY/tTpkzBr7/+ihkzZsDV1RUtW7bE/v374eTk9Fnx9e/fHytXrsSaNWvg5uYGDw8PrF27VtWOqakpfv/9d1SvXh01atRAWFgYDhw4wG9zEWWDQqFAYGAgFAqF1KEQEeU45jwi0ibMeUSkTZjziEibSJ3zZEIIIcmRtdyHDx9gbm6OyMjINKO98fHxCA0NhZOTEwzz+mJ9uUQIgbi4OMybNw/Lli3DkydPpA6JSKsxT+Ws1F8O3NzcVOtgEhEVVMx5RKRNmPOISJsw5xGRNvncnJfZGNGX4DSelKctXrwYNWrUgKWlJU6ePIk5c+bA29tb6rCIiIiIiIiIiIiIiIjyBA72UZ4WHByMqVOn4t27d7Czs8OoUaNUU3kSERERERERERERERFpO07jKRFO4/l5hBAQQkAmk0Emk0kdDpHWY57KWUIIxMfHw9DQkDmPiAo85jwi0ibMeUSkTZjziEibfG7O0/Q0njpf3QJRLuEvBUSkTfT19aUOgYgo1zDnEZE2Yc4jIm3CnEdE2kTKnMfBPso34uLipA6BiChXKJVKBAYGQqlUSh0KEVGOY84jIm3CnEdE2oQ5j4i0idQ5j4N9RERERERERERERERERPkUB/uIiIiIiIiIiIiIiIiI8ikO9hERERERERERERERERHlUxzso3zDyMhI6hCIiHKFjo4O3NzcoKPD/00TUcHHnEdE2oQ5j4i0CXMeEWkTqXMeMy3lG0II1b+9vLzQvn171etGjRphxIgRuR6Tn58fZDIZIiIicv3YRFSwJSYmSh0CEVGuYc4jIm3CnEdE2oQ5j4i0iZQ5j4N9pDFeXl6QyWSQyWTQ19dH6dKl8dtvvyE5OVkj7cfHx2e47Z9//sGUKVOy1U5uD9A5OjpCJpNh69atabZVqFABMpkMa9euzZVYiCh/UCqVCAoKglKplDoUIqIcx5xHRNqEOY8KgofHHuL3Ir8jIixC6lAoj2POIyJtInXO42AfaVTLli3x4sULBAcHY/To0fDx8cHs2bPTravJUW5LS0uYmppqrD1Ns7Ozw5o1a9TKLl68iJcvX8LY2FiiqIiIiIiIiIiIPs/TS08R9y4OrwJfSR0KERER/T8O9uUjMYkxGf7EJ8dnu25cUly26n4JAwMD2NjYwMHBAYMHD0azZs2wZ88eAP9NvTlt2jTY2tqiXLlyAIAnT56ga9eusLCwgKWlJdq1a4ewsDBVmwqFAqNGjYKtrS2srKzw008/qU3pCaSdxjMhIQHjxo2DnZ0dDAwMULp0aaxatQphYWFo3LgxAKBw4cKQyWTw8vICkDLyPmPGDDg5OcHIyAju7u7YsWOH2nEOHDiAsmXLwsjICI0bN1aLMzM9e/bEqVOn8OTJE1XZ6tWr0bNnT+jq6qrVjYiIQP/+/WFtbQ0zMzM0adIEAQEBqu0hISFo164dihUrBhMTE9SoUQPHjh1Ta8PR0RHTp09H3759YWpqCnt7eyxfvjxbsRIRERERERERZcS9lzt6HuwJu7p2UodCRERE/0836yqUV5jMMMlwW6syrbC/x37V66JziiI2KTbduh4OHvDz8lO9dvzDEW9i36SpJyaLNGWfy8jICG/fvlW9Pn78OMzMzHD06FEAQFJSEjw9PVGnTh2cOXMGurq6mDp1Klq2bImbN29CX18fc+fOxbp167BkyRK4u7tj3rx5+Pfff9GkSZMMj9u7d29cuHABCxcuhLu7O0JDQ/HmzRvY2dlh586d6NSpE4KCgmBmZgYjIyMAwIwZM7Bx40YsXboUZcqUwenTp/Hdd9/B2toaHh4eePLkCTp27IihQ4di4MCBuHr1KkaPHp2t81CsWDF4enpi3bp1mDhxImJjY7Ft2zacOnUK69evV6vbpUsXGBkZ4eDBgzA3N8eyZcvQtGlT3L9/H5aWloiOjkarVq0wbdo0GBgYYP369Wjbti2CgoJgb2+vamfu3LmYMmUKfvnlF+zYsQODBw+Gh4eHapCViPI2uVwudQhERLmGOY+ItAlzHuV3hawKwbaGLQzNDaUOhfIB5jwi0iZS5jwO9lGOEELg+PHjOHz4MH788UdVubGxMVauXAl9fX0AwMaNG6FUKrFy5UrIZDIAwJo1a2BhYQE/Pz+0aNECCxYswM8//4zu3bsDAJYuXYrDhw9neOz79+/j77//xtGjR9GsWTMAgLOzs2q7paUlAKBo0aKwsLAAkPIk4PTp03Hs2DHUqVNHtc/Zs2exbNkyeHh4YMmSJShVqhTmzp0LAChXrhwCAwMxa9asbJ2Tvn37YvTo0ZgwYQJ27NiBUqVKoXLlymp1zp49i8uXLyM8PBwGBgYAgDlz5mDXrl3YsWMHBg4cCHd3d7i7u6v2mTJlCv7991/s2bMH3t7eqvJWrVphyJAhAIBx48Zh/vz5OHnyJAf7iPIBuVwONzc3qcMgIsoVzHlEpE2Y86gguPzXZRz76Rj6X+qPEjVLSB0O5WHMeUSkTaTOeRzsy0eif47OcJtcR33EOHxMeIZ1dWTqs7eGDQ/7qrg+tm/fPpiYmCApKQlKpRI9evSAj4+Parubm5tqoA8AAgIC8ODBgzTr7cXHxyMkJASRkZF48eIFatasCYVCAR0dHejq6qJ69epppvJM5e/vD7lcDg8Pj2zH/eDBA8TGxqJ58+Zq5YmJiahSpQoA4O7du6hVq5ba9tSBwexo3bo1Bg0ahNOnT2P16tXo27dvmjoBAQGIjo5GkSJF1Mrj4uIQEhICAIiOjoaPjw/279+PFy9eIDk5GXFxcXj8+LHaPpUqVVL9WyaTwcbGBuHhGd8XRJR3CCEQFRUFU1NT1RchiIgKKuY8ItImzHlUELwPeQ8AeHT6EQf7KFPMeUSkTaTOeRzsy0eM9Y0lr5uVxo0bY8mSJdDX14etrW2a9eiMjdWPFR0djWrVqmHTpk1p2rK2tlZ7nZCQoJpyMzPZqfOp6OiUgdT9+/ejRAn1X1RTn7D7Wrq6uujVqxcmT56MS5cu4d9//003juLFi8PPzy/NttSnEMeMGYOjR49izpw5KF26NIyMjNC5c2ckJiaq1dfT01N7LZPJoFQqNdIXIspZSqUSDx8+hJubG6c8IaICjzmPiLQJcx4VBO693RH1LIoDfZQl5jwi0iZS5zwO9pFGGRsbo3Tp0tmuX7VqVWzbtg1FixaFmZlZunWKFy+OS5cuoUaNGgCA5ORkXLt2DVWrVk23vpubG5RKJU6dOqWaxvNjqU8WKhQKVVn58uVhYGCAx48fZ/hEoKurK/bs2aNWdvHixaw7+ZG+fftizpw56NatGwoXLpxme9WqVfHy5Uvo6urC0dEx3TbOnTsHLy8vdOjQAUDKAGFYWNhnxUFERERERERE9CXs6tqh+97uUodBREREH9HJugpRzunZsyesrKzQrl07nDlzBqGhofDz88OwYcPw9OlTAMDw4cMxa9Ys7N27F/fu3cOQIUMQERGRYZuOjo7o06cP+vbti127dqna/PvvvwEADg4OkMlk2LdvH16/fo3o6GiYmppizJgxGDlyJNatW4eQkBBcv34df/75J9atWwcA+OGHHxAcHIyxY8ciKCgImzdvxtq1az+rv66urnjz5g3WrFmT7vZmzZqhTp06aN++PY4cOYKwsDCcP38eEyZMwNWrVwEAZcqUwT///AN/f38EBASgR48efGKPiIiIiIiIiHLFnR13sOPbHYh8Eil1KERERPT/ONhHkipUqBBOnz4Ne3t7dOzYEa6urujXrx/i4+NVT/qNHj0a3333HQYOHIi6devC1NRU9VRbRpYsWYLOnTtjyJAhcHFxwYABAxATEwMAKFGiBHx9fTF+/HgUK1YM3t7eAIApU6bg119/xYwZM+Dq6oqWLVti//79cHJyAgDY29tj586d2LVrF9zd3bF06VJMnz79s/tcpEiRDKcalclkOHDgABo2bIjvv/8eZcuWxbfffotHjx6hWLFiAIB58+ahcOHCqFu3Ltq2bQtPT88Mn3IkovzL0NBQ6hCIiHINcx4RaRPmPMrv7u26h9vbbuPBwQdSh0L5AHMeEWkTKXOeTAghJDu6Fvvw4QPMzc0RGRmZZvrK+Ph4hIaGwsnJif9DJKI8iXmKiIiIiIhIO4WeDMX6JuvxzZ/foKZ3TanDISIiypcyGyP6Enyyj/IFIQSSk5PBsWki0gZKpRJv377lFL1EpBWY84hImzDnUUFgV8cOIx6PQGWvylKHQnkccx4RaROpcx4H+yjfSExMlDoEIqJcIYTAkydP+AUHItIKzHlEpE2Y86ggiHoehYiwCMjkMqlDoTyOOY+ItInUOY+DfURERERERERERJQtFxdcxNqGa/Hm7hupQyEiIqL/x8E+IiIiIiIiIiIiyhYLJwsAwN1/70obCBEREanoSh0AUXbp6HBsmoi0h6mpqdQhEBHlGuY8ItImzHmU31XoUgExr2Lg3MxZ6lAoH2DOIyJtImXOkwlOmiyJDx8+wNzcHJGRkTAzM1PbFh8fj9DQUDg5OcHQ0FCiCImIMsY8RURERERERERERPRlMhsj+hJ8VIryBSEEkpKSuKAvEWkFpVKJly9fQqlUSh0KEVGOY84jIm3CnEcFwelpp7HEbQli38am2XZmxhkEbAiQICrKi5jziEibSJ3zONhH+UZSUpLUIRAR5QohBF6+fMkvOBCRVmDOIyJtwpxHBUFEaATCb4Xj5sababadnXEWgRsDJYiK8iLmPCLSJlLnPK7ZR0RERERERERERNlSd2xd3N52G4nRiWm29TnZB3pGehJERUREpN34ZB9RFvz8/CCTyRAREZGrx127di0sLCy+qo2wsDDIZDL4+/tnWEeq/hERERERERFR/mNVzgo/R/2MhhMaptm2qeUm7B+yX4KoiIiItBsH+0hjvLy80L59+xxrXy6Xa7xNmUyW6Y+Pj4/Gj0lElBWZTAZLS0vIZDKpQyEiynHMeUSkTZjzqCAIPRmKq0uvQpGoUCsXQsCpiRPsG9hLFBnlNcx5RKRNpM55nMaT8gWZTAYDA4OvaiMxMRH6+vpqZS9evFD9e9u2bZg0aRKCgoJUZSYmJrh69apGjkVElF06Ojqwt+cfyESkHZjziEibMOdRQXBt6TXc/vs2ilctjhI1S6jKlclK3P77Nip+W1HC6CgvYc4jIm0idc7jk32Ua06dOoWaNWvCwMAAxYsXx/jx45GcnAwA2LdvHywsLKBQpHwrzN/fHzKZDOPHjweQ8u2w77//Ht99952qvbNnz6JBgwYwMjKCnZ0dhg0bhpiYGNV2R0dHTJkyBb1794aZmRkGDhyYJiYbGxvVj7m5OWQymVqZiYmJqu61a9dQvXp1FCpUCHXr1lUbFPTx8UHlypWxcuVKODk5wdDQEAAQERGB/v37w9raGmZmZmjSpAkCAgJU+wUEBKBx48YwNTWFmZkZqlWrlmZw8fDhw3B1dYWJiQlatmypNkCpVCrx22+/oWTJkjAwMEDlypVx6NChTK/DgQMHULZsWRgZGaFx48YICwvLtD4R5T6lUonHjx9DqVRKHQoRUY5jziMibcKcRwVB5e8rAwACtwSm2VbDuwacWzjnckSUVzHnEZE2kTrncbAvP9lfAbg28r/Xt6amlCW8S3kd9SDl9f2//qtzwQs4Wv+/188Pp9R5fvi/sqP1U+qluv9XSh0NevbsGVq1aoUaNWogICAAS5YswapVqzB16lQAQIMGDRAVFYUbN24ASBkYtLKygp+fn6qNM2fOwMPDAwAQEhKCli1bolOnTrh58ya2bduGs2fPwtvbW+24c+bMgbu7O27cuIFff/31q/owYcIEzJ07F1evXoWuri769u2rtv3BgwfYuXMn/vnnH9UaeV26dEF4eDgOHjyIa9euoWrVqmjatCnevUu5Zj179kTJkiVx5coVXLt2DePHj4ee3n8LWcfGxmLOnDnYsGEDTp8+jcePH2PMmDGq7X/88Qfmzp2LOXPm4ObNm/D09MT//vc/BAcHp9uHJ0+eoGPHjmjbti38/f3Rv39/1YAqEeUdQgi8e/cOQgipQyEiynHMeUSkTZjzqCCwb2CP9uvbo9J3ldTK5XpyCKVA1PMoiSKjvIY5j4i0idQ5j9N4Uq5YvHgx7OzssGjRIshkMri4uOD58+cYN24cJk2aBHNzc1SuXBl+fn6oXr06/Pz8MHLkSPj6+iI6OhoREREICQlRDfbNmDEDPXv2xIgRIwAAZcqUwcKFC+Hh4YElS5aonqxr0qQJRo8erZE+TJs2TXX88ePHo3Xr1oiPj1cdKzExEevXr4e1tTWAlCcPL1++jPDwcNUUpHPmzMGuXbuwY8cODBw4EI8fP8bYsWPh4uKi6sfHkpKSsHTpUpQqVQoA4O3tjd9++021fc6cORg3bhy+/fZbAMCsWbNw8uRJLFiwAH/99Rc+tWTJEpQqVQpz584FAJQrVw6BgYGYNWuWRs4RERERERERERVs+sb6cO/lnu62O9vvwLq8NRpOaJjLUREREWk3PtmXn7S+DVSb/9/rihNTygwsU16blk55XXbof3XqrAWan/3vta1nSh1bz//Kmp9NqZeq7NCUOhp09+5d1KlTR21xynr16iE6OhpPnz4FAHh4eMDPzw9CCJw5cwYdO3aEq6srzp49i1OnTqF48eKqwbCAgACsXbsWJiYmqh9PT08olUqEhoaqjlG9enWN9aFSpf++sVa8eHEAQHh4uKrMwcFBNdCXGmN0dDSKFCmiFmdoaChCQkIAAKNGjUL//v3RrFkzzJw5U1WeqlChQqqBvtTjph7zw4cPeP78OerVq6e2T7169XD37t10+3D37l3UqlVLraxOnTrZPgdEREREREREpN22d90OXx1fCKX6kwtx7+NQokYJrtlHREQkAT7ZR3lGo0aNsHr1agQEBEBPTw8uLi5o1KgR/Pz88O7dOzRs+N+3wqKjozFo0CAMGzYsTTsfL4JpbGyssfg+nl4zddDy4/l3Pz1WdHQ0ihcvrjYVaSoLCwsAKWv99ejRA/v378fBgwcxefJkbN26FR06dEhzzNTjcuoDooIvdf3Qj78gQURUUDHnEZE2Yc6jgsCyjCUggOMTjqPZjGaq8qSYJAQfDIZlWUsJo6O8hDmPiLSJ1DmPg32UK1xdXbFz504IIVQ3+7lz52BqaoqSJUsC+G/dvvnz56umy2zUqBFmzpyJ9+/fY/To0ap9q1atijt37qB06dLSdCgbqlatipcvX0JXVxeOjo4Z1itbtizKli2LkSNHonv37lizZo1qsC8zZmZmsLW1xblz51TnC0g5rzVr1kx3H1dXV+zZs0et7OLFi9nrEBHlGh0dHdjY2EgdBhFRrmDOIyJtwpxHBUH9cfURejwUhhaGauVmJc0w4PIA6Bry40ZKwZxHRNpE6pzHaTxJoyIjI+Hv76/28+TJEwwZMgRPnjzBjz/+iHv37mH37t2YPHkyRo0aBR2dlNuwcOHCqFSpEjZt2oRGjRoBABo2bIjr16/j/v37qF27tuqptnHjxuH8+fPw9vaGv78/goODsXv3bnh7e0vV9TSaNWuGOnXqoH379jhy5AjCwsJw/vx5TJgwAVevXkVcXBy8vb3h5+eHR48e4dy5c7hy5QpcXV2zfYyxY8di1qxZ2LZtG4KCgjB+/Hj4+/tj+PDh6db/4YcfEBwcjLFjxyIoKAibN2/G2rVrNdRjItIUhUKBkJAQKBQKqUMhIspxzHlEpE2Y86ggMDAzQP+L/VF/XP002za12oQDQw9IEBXlRcx5RKRNpM55/KoNaZSfnx+qVKmiVtavXz+sXLkSBw4cwNixY+Hu7g5LS0v069cPEydOVKvr4eEBf39/1WCfpaUlypcvj1evXqk9xVepUiWcOnUKEyZMQIMGDSCEQKlSpdCtW7cc72N2yWQyHDhwABMmTMD333+P169fw8bGBg0bNkSxYsUgl8vx9u1b9O7dG69evYKVlRU6duwIX1/fbB9j2LBhiIyMxOjRoxEeHo7y5ctjz549qrUNP2Vvb4+dO3di5MiR+PPPP1GzZk1Mnz4dffv21VS3iUhDoqKipA6BiCjXMOcRkTZhzqP87sqSK3gX/A6e8zzVyhOjE+HQ0AElapWQKDLKi5jziEibSJnzZIILgEniw4cPMDc3R2RkJMzMzNS2xcfHIzQ0FE5OTjA0NMygBe0ihEBcXByMjIw4zzdRHsA8lbMUCgUCAwPh5uYGuVwudThERDmKOY+ItAlzHhUEG1pswMOjD/Ht7m9R7n/lVOUv/V9iWZVlaDylMRpObChhhJRXMOcRkTb53JyX2RjRl+A0nkRERERERERERJQtHdZ3AADc/vu2WvmLGy8AAIVLFc71mIiIiLQdp/GkfENfX1/qEIiIcoVMJoOdnR2fZCYircCcR0TahDmPCgLjosYYeG0gjCyN1Df8/9xhJ345AbfubrkfGOU5zHlEpE2kznkc7KN8QSaTQVeXtysRaQcdHR0UKVJE6jCIiHIFcx4RaRPmPMrvkhOSER4YDl1DXVg4WqhtM7ExAQCUqMk1+ygFcx4RaROpcx6n8aR8IXXNPi4xSUTaQKFQ4N69e1AoFFKHQkSU45jziEibMOdRfndo+CGsqLECS9yWIDEmUW3b9RXXIdORodPWThJFR3kNcx4RaROpcx4H+yjf4EAfEWmT+Ph4qUMgIso1zHlEpE2Y8yg/C78Vrvr3v73+VdvWyLcRehzowSkbSQ1zHhFpEylzHgf7iIiIiIiIiIiIKEsd1neA1ykvuPd2h1NTJ7VtxSoVQ9zbONzfd1+i6Igorwk+GIwd3Xbg9Z3XUodCVOBxETQiIiIiIiIiIiLKUmHnwijsXBgODR3SbBNC4PCowyhStgjKtikrQXRElNc8OvUIt/++jZJ1SsK6vLXU4RAVaHyyj/INAwMDqUMgIsoVOjo6cHZ2ho4O/zdNRAUfcx4RaRPmPMrvnl99Dl+ZL+bYzEmzbWmlpYh5FYPms5tLEBnlRcx59OT8EwBAcnyyxJEQ5Typcx4zLeULMpkMcrmc874TkVaQyWQwMzNjziMircCcR0TahDmP8rttHbYBAGJexeDsrLNq2xw8HFDDuwZK1iopRWiUBzHn0bPLzwAAlb6rJHEkRDlP6pzHwT7KF4QQiI2NhRBCkuM3atQII0aMyPXjOjo6YsGCBV/VhpeXF9q3b59pHan6R0TpUygUCAwMhEKhkDoUIqIcx5xHRNqEOY/yuwYTGqB8l/LQM9bDgwMP1La1WtQKrf5sJVFklBcx59HE+ImYLCbDrKSZ1KEQ5Tipcx4H+0hjMhowWrt2LSwsLHI9nuzw8fGBTCbL9IeISAr8Y4iItAlzHhFpE+Y8ys+q/1AdXf7ugjGvxqDPyT5pti9wWID1TddLEBnlVcx52k0IAUWiAopE3gekHaTMeRzsI62hUCigVCrVysaMGYMXL16ofkqWLInffvtNrexLJSUlfW3IRERERERERER5RsKHBBz/5Thubb0FmY76F6SPjT+GyMeRsK1hK1F0RJTXXF50GVMNpuL4hONSh0JU4HGwj3Jd6rSSvr6+sLa2hpmZGX744QckJiaq6jRq1Aje3t7w9vaGubk5rK2t8dtvv6lN45mQkIAxY8agRIkSMDY2Rq1ateDn56fanvpE4Z49e1C+fHkYGBjg8ePHarGYmJjAxsZG9SOXy2FqaqpWlkqpVOKnn36CpaUlbGxs4OPjo9aWTCbDkiVL8L///Q/GxsaYNm0aAGD37t2oWrUqDA0N4ezsDF9fXyQnpyxKK4SAj48P7O3tYWBgAFtbWwwbNkyt3djYWPTt2xempqawt7fH8uXL1bYHBgaiSZMmMDIyQpEiRTBw4EBER0dneP5jYmLQu3dvmJiYoHjx4pg7d24mV4uIiIiIiIiIKMWhkYdwdsZZ7O2/F8+uPFPbdmXxFTg3d0azmc0kio6I8prrK64DAGwq22RRk4i+Fgf78pHFFRbj0MhDqtenp57G4gqLEfcuDgDw7sE7LK6wGJf/uqyqs8trF1bXX616/eDwAyyusBgPDv83r/rq+quxy2uX6vXlvy5jcYXFOdgT4Pjx47h79y78/PywZcsW/PPPP/D19VWrs27dOujq6uLy5ctYsGAB/vzzT6xcuVK13dvbGxcuXMDWrVtx8+ZNdOnSBS1btkRwcLCqTmxsLGbNmoWVK1fi9u3bKFq06BfHvG7dOhgbG+PSpUv4/fff8dtvv+Ho0aNqdXx8fNChQwcEBgaib9++OHPmDHr37o3hw4fjzp07WLZsGdauXasaCNy5cyfmz5+PZcuWITg4GLt27YKbm5tam3PnzkX16tVx48YNDBkyBIMHD0ZQUBCAlIE7T09PFC5cGFeuXMH27dtx7NgxeHt7Z9iPsWPH4tSpU9i9ezeOHDkCPz8/XL9+/YvPCxFpno6ODsqVKwcdHf5vmogKPuY8ItImzHmU371/8F71763ttqpt+zH4R3Tc1DG3Q6I8jDmP2q1phz4n+6BSz0pSh0KU46TOebqSHJW0nr6+PlavXo1ChQqhQoUK+O233zB27FhMmTJF9Waws7PD/PnzIZPJULZsWQQGBmLBggUYOHAgHj9+jDVr1uDx48ewtU2ZHmLMmDE4dOgQ1qxZg+nTpwNImUpz8eLFcHd3/+qYK1WqhMmTJwMAypQpg0WLFuH48eNo3ry5qk6PHj3w/fffq1737dsX48ePR58+KfPYOzs7Y8qUKfjpp58wefJkPH78GDY2NmjWrBn09PRgb2+PmjVrqh23VatWGDJkCABg3LhxmD9/Pk6ePIly5cph8+bNiI+Px/r162FsbAwAWLRoEdq2bYtZs2ahWLFiam1FR0dj1apV2LhxI5o2bQogZRCzZMmSX31+iEiz9PX1pQ6BiCjXMOcRkTZhzqP8rMf+HkiOT8bD4w+REJmgts2kmAn8fPxgVMQItX6sJVGElNcw52k322qc1pe0i5Q5j4N9+ciQ20PUXjec2BANJzZUvbYsbZmmTvu17dVel/YsjdK3S6uV9T3bV+11zaE1UXOo+oCTprm7u6NQoUKq13Xq1EF0dDSePHkCBwcHAEDt2rUhk/03/3u1atUwb948KBQKBAYGQqFQoGzZsmrtJiQkoEiRIqrX+vr6qFRJM98c+bSd4sWLIzw8XK2sevXqaq8DAgJw7tw51ZN8QMragfHx8YiNjUWXLl2wYMECODs7o2XLlmjVqhXatm0LXd3/3pofH1cmk8HGxkZ13Lt378Ld3V010AcA9erVg1KpRFBQUJrBvpCQECQmJqJWrf9+6ba0tES5cuU+93QQUQ5SKpUIDAyEm5sb5HK51OEQEeUo5jwi0ibMeZTf6ZvoQ99EHxW7VVQrF0LgbdBbnPI9Bbt6dhzsIwDMeQQkxSXhgPcB2Ne3R5Xvq0gdDlGOkjrncbCPNMbMzAyRkZFpyiMiImBubq7RY0VHR0Mul+PatWtp3jgmJiaqfxsZGakNGH4NPT09tdcymQxKpVKt7ONBt9Q4fX190bFj2mksDA0NYWdnh6CgIBw7dgxHjx7FkCFDMHv2bJw6dUp1vOwcl4iIiIiIiIgopz08/hAbmm0AAEwWk1XlymQl/nL9Cw4NHdD5785ShUdEecyK6ivw+s5rQICDfUQ5jIN9pDHlypXDkSNH0pRfv349zRN4AQEBiIuLg5GREQDg4sWLMDExgZ2dnarOpUuX1Pa5fPkyypQpA7lcjipVqkChUCA8PBwNGjTIgd5oRtWqVREUFITSpUtnWMfIyAht27ZF27ZtMXToULi4uCAwMBBVq1bNsn1XV1esXbsWMTExqoHGc+fOqeYH/lSpUqWgp6eHS5cuwd7eHgDw/v173L9/Hx4eHl/YSyIiIiIiIiLSBoeGHVL9e2Wtleh/qb/qdYOJDVDMrRhMipmktysRaaFSLUuhWKViaPVXK6lDISrwONhHGjN48GAsWrQIw4YNQ//+/WFgYID9+/djy5Yt2Lt3r1rdxMRE9OvXDxMnTkRYWBgmT54Mb29vtcUrHz9+jFGjRmHQoEG4du0ali5dijlz5gAAypYti549e6J3796YO3cuqlSpgtevX+P48eOoVKkSWrdunat9z8ikSZPQpk0b2Nvbo3PnztDR0UFAQABu3bqFqVOnYu3atVAoFKhVqxYKFSqEjRs3wsjISDWVaVZ69uyJyZMno0+fPvDx8cHr16/x448/olevXmmm8ARSnnrs168fxo4diyJFiqBo0aKYMGECF0omIiIiIiIioiw1mNgAkY8i8T70PZ6cfaIql+vJ0WRKE0SEReDDsw8wK2EmYZRElFd4zvWUOgQircHBPtIYZ2dnnD59GhMmTECzZs2QmJgIFxcXbN++HS1btlSr27RpU5QpUwYNGzZEQkICunfvDh8fH7U6vXv3RlxcHGrWrAm5XI5hw4Zh4MCBqu1r1qzB1KlTMXr0aDx79gxWVlaoXbs22rRpkxvdzRZPT0/s27cPv/32G2bNmgU9PT24uLigf/+Ub75ZWFhg5syZGDVqFBQKBdzc3LB37161dQczU6hQIRw+fBjDhw9HjRo1UKhQIXTq1Anz5s3LcJ/Zs2cjOjoabdu2hampKUaPHp3u9KtEJB0dHR24ublxIJ6ItAJzHhFpE+Y8yu/curtlun1V3VUo7FwYfc/2zaWIKC9jziOhFHjp/xKGhQ1R2Kmw1OEQ5Sipc55MCCEkObKW+/DhA8zNzREZGQkzM/VvO8XHxyM0NBROTk4wNDSUKMKc4+XlhYiICOzatSvDOo0aNULlypWxYMECACkLPQshIJPJNLYGHxF9uYKep6QmhEB8fDwMDQ2Z84iowGPOIyJtwpxH+V1SbBIWll4Il/YuaL34v1mV4t7FYXuX7XgT9AYNJjRAjcE1JIyS8grmPDo56SROTzkNtx5u6Lipo9ThEOWoz815mY0RfQl+rYLyjfj4eKlDICLKFUqlEkFBQVAqlVKHQkSU45jziEibMOdRfrerzy5Ev4jG1SVXcW35NSiTU+7l5PhkvLjxAu693TnQRyrMeRS4KRAAUKFbBYkjIcp5Uuc8TuNJREREREREREREWUqMSQQAmBQ3wb5B++DW0w36uvowtTXFuHfjJI6OiPKafhf7AQIwLmosdShEBR4H+yjXrV27Nss6fn5+OR4HERERERERERFlX88DPQEArwJfIfplNHQN1D9a3NljJ0xsTOA5z1OK8IgojzG25iAfUW7hNJ5ERER5kFwulzoEIqJcw5xHRNqEOY8KgmJuxVCqeSno6KZ8tBgfGY87O+/g1pZbCA8Mlzg6ykuY87TbuwfvMK/kPByfcFzqUIhyhZQ5j0/2Ub4gk8lQqFAhqcMgIsoVcrkcbm5uUodBpCbiUQQAwMLBQtI4qOBhziMibcKcR/ld8MFgCIVA2TZl1cojQiOwvfN2NJ7aGA0nNJQoOsprmPNoTcM1iH4Rjdg3sVKHQpTjpM55HOyjfEEIAaVSCR0dHchkMqnDISLKUUIIREVFwdTUlDmP8owV1VdAkajA+MjxUodCBQxzHhFpwtNLT2FgZgBrV2upQ8kUcx7ld8fHH0dSXBLCToXh0oJLGB46HGYlzWBub46OmzuimFsxqUOkPIQ5j2oNrwWjwkaoNrCa1KEQ5Tipcx6n8aR8IyEhQeoQiIhyhVKpxMOHD6FUKqUOhUjFzM4MJsVNpA6DCiDmPCLShFW1V2FPvz1Sh5El5jzK71rMa4Fv/vwG1q7WKNumLOT6KdOVGVkawa27G6KeR+HRmUcSR0l5BXMe1R9XnwN9pDWkznl8so+IiIiIsjTw2kB+G5eIiPIsBw8HFHYqLHUYRAWec1Nn1b+r9K2SZvvu73fD3MEc/c73y82wiCgPC9wcCJPiJnBq7CR1KEQFGgf7iIiIiChLZ2eehVxfjrqj60odChERURpefl5Sh0Ck1R6dfoQd3XagZO2SqDaIT/EQUYotbbfg/r77KNu2LAf7iHIYp/EsoF6+fImpU6aghqsrypYogRqurpg6ZQpevnwpdWhfLK89TeDo6IgFCxZIHUaOatSoEUaMGKF6nRt9Pn78OFxdXaFQKHL0OJSx8ePH48cff5Q6DK1naGgodQiUi2LCY/Dw2EOpw8jUiV9O4OiYo1KHQQUUcx4Rfa2A9QHY2n4rYl7HSB1KlpjzKD9bXm051jVeh5AjITg4/CCinkcBAHQNdVG4VGFU/r4ySrcsLXGUlJcw52m3lwEvoWuoiwYTGkgdClGukDLncbCvAJo/dy4cSpbE2enTMeLePSx5/hwj7t3DmenT4VCyJObPm5cjx/Xy8oJMJoNMJoOenh6KFSuG5s2bY/Xq1Z89T62Pjw8qV66sei2TyWBkZPTVA35SDNA9ffoU+vr6qFixYq4eNydcuXIFAwcOzNFj/PTTT5g4cSLk8pR5/xUKBWbOnAkXFxcYGRnB0tIStWrVwsqVK3M0jvzg9u3b6NSpExwdHSGTybJ9b9+8eRMNGjSAoaEh7Ozs8Pvvv6ttHzNmDNatW4eHD/P2wENBJpfL4eLionofUMG3u+9ubGi+AVEvoqQOJUO21W2lDoEKKOY8ItKEg8MOImh3EBI+5O213pnzKL+zcrVCkXJF8OzKM1xeeBkx4SkD7CVqlkDfs31R7n/lJI6Q8hLmPBr5eCQmxE1AyVolpQ6FKMdJnfPy7WDfzJkzIZPJ1J46io+Px9ChQ1GkSBGYmJigU6dOePXqldp+jx8/RuvWrVGoUCEULVoUY8eORXJyslodPz8/VK1aFQYGBihdujTWrl2b5vh//fUXHB0dYWhoiFq1auHy5cs50c3PNn/uXMyYMAGnFQocio9HTwBNAfQEcDg+HqcVCsz45ZccG/Br2bIlXrx4gbCwMBw8eBCNGzfG8OHD0aZNmzTn+XMIIZCcnAwhhAajzR1r165F165d8eHDB1y6dEnqcL6KtbU1ChUqlGPtnz17FiEhIejUqZOqzNfXF/Pnz8eUKVNw584dnDx5EgMHDkRERESOxZGYmPjF+z5+/FiDkWQuNjYWzs7OmDlzJmxsbLK1z4cPH9CiRQs4ODjg2rVrmD17Nnx8fLB8+XJVHSsrK3h6emLJkiU5FTplQalU4u3bt1zEXIuUalEKjo0doVdIT+pQMtR4amO0Wd5G6jCoAGLOIyKNEIBzc2dYlrKUOpJMMedRftdxY0e0WdoGNb1rYuTTkbCuYK22fUWNFdjafqtE0VFew5xHRNpE6pyXLwf7rly5gmXLlqFSpUpq5SNHjsTevXuxfft2nDp1Cs+fP0fHjh1V2xUKBVq3bo3ExEScP38e69atw9q1azFp0iRVndDQULRu3RqNGzeGv78/RowYgf79++Pw4cOqOtu2bcOoUaMwefJkXL9+He7u7vD09ER4eHjOdz4TL1++xPhx47A3IQG1MqhTC8DehASM/+mnHJnS08DAADY2NihRogSqVq2KX375Bbt378bBgwfVBk0jIiLQv39/WFtbw8zMDE2aNEFAQACAlMExX19fBAQEqJ4UXLt2LRITEzPdL9XevXtRo0YNGBoawsrKCh06dACQMiXlo0ePMHLkSFW7qc6ePYsGDRrAyMgIdnZ2GDZsGGJi/pv+JTw8HG3btoWRkRGcnJywadOmbJ0PIQTWrFmDXr16oUePHli1apXa9rCwMMhkMvzzzz9o3LgxChUqBHd3d1y4cEGt3s6dO1GhQgUYGBjA0dERc+fOVdvu6OiIqVOnonfv3jAxMYGDgwP27NmD169fo127djAxMUGlSpVw9epV1T5v375F9+7dUaJECRQqVAhubm7YsmVLpv359MnIrK5HQEAAGjduDFNTU5iZmaFatWpqMXxq69ataN68udrjznv27MGQIUPQpUsXODk5wd3dHf369cOYMWNUdZRKJX7//XeULl0aBgYGsLe3x7Rp01TbAwMD0aRJExgZGaFIkSIYOHAgoqOjVdu9vLzQvn17TJs2Dba2tihXLuWbiE+ePEHXrl1hYWEBS0tLtGvXDmFhYZmeIycnJzRr1gwbNmxAbGxspnW/Vo0aNTB79mx8++23MDAwyNY+mzZtQmJiIlavXo0KFSrg22+/xbBhwzDvky8AtG3bFlu38g80qQgh8OTJk3z5BQf6MrWG1UKfE31gaJ53p7ixcrFC2dZlpQ6DCiDmPCLShJ6HeqLF3BZSh5El5jwqKAzNDWFWwgxyvZSnF8JvhePUlFOIj4xHIeuc+5Iw5S/MeXRn5x34ynyxreM2qUMhynFS5zxdSY76FaKjo9GzZ0+sWLECU6dOVZVHRkZi1apV2Lx5M5o0aQIAWLNmDVxdXXHx4kXUrl0bR44cwZ07d3Ds2DEUK1YMlStXxpQpUzBu3Dj4+PhAX18fS5cuhZOTk2owxdXVFWfPnsX8+fPh6ekJAJg3bx4GDBiA77//HgCwdOlS7N+/H6tXr8b48ePTjTshIQEJCf9NJ/LhwwcAKQOQqWuTyWQy6OjoQKlUQgih+kndlt5N8nH5iuXL0VhPD7WyWOusFoBG+vpYuWIFJkycmGG97BwzPZ9ua9KkCdzd3fHPP/+gX79+AIAuXbrAyMgIBw4cgLm5OZYtW4amTZsiKCgI3bp1Q2BgIA4fPoyjR1PWBjI3N1ftV6hQoXT3s7S0xP79+9GhQwf88ssvWLduHRITE3HgwAEIIbBz505UrlwZAwYMwIABA1SxhoSEoGXLlpgyZQpWrVqF169f48cff4S3tzfWrFkDIQS8vLzw/PlznDhxAnp6ehg+fDjCw8PVrlF65+bEiROIjY1Fs2bNYGtri3r16mHevHkwNjZWO0cTJkzA7NmzUaZMGUycOBHdu3dHcHAwdHV1ce3aNXTt2hWTJ09Gt27dcP78eQwdOhSWlpaqqVMBYP78+Zg2bRomTpyI+fPno1evXqhbty6+//57/P777xg/fjx69+6N27dvAwDi4uJQtWpV/PTTTzAzM8OBAwfQq1cvODs7o2bNmhle09Q+y2SyTK+jpaUlevbsiSpVqmDx4sWQy+Xw9/eHrq5uhvfPmTNn0L17d7Vj2tjY4MSJExg8eDCKFi2a7r4///wzVqxYgXnz5qF+/fp48eIF7t27ByAlZ3h6eqJOnTq4fPkywsPDMWDAANX1TXX8+HGYmZnhyJEjAFKe7vP09ETt2rVx5swZyOVyTJs2DS1btkRAQAD09fXT7cPt27exbt06TJw4EUOGDEHnzp3Rp08feHh4pKm7adMm/PDDD+m2k+rAgQNo0OC/ec0ze/99ej+mV//ChQto2LAh9PX1VeUtWrTArFmz8O7dOxQuXBhAykDi06dPERoaCkdHx3SPlfrfT9dXlMvlqjyWVfmnee/T8k/bzqhcR0cHMpks3XIAab5Rk1G5XC6HECLd8tzsk0KhUO1bUPqUGmNBuk6a7NOrG69wc+NNVO5bWe3b0XmpT2vqr4FpCVN8f+77bPWpIF4n9iln+pSa85j32Cf2iX36mj4ZFzfG23tvYVzcGEaFjfJsn1JznkKh0MrrxD7l/z6dm3UOhawLoXzn8oh7HQcTWxPoGuni+Y3n8Jvkh847OqNCpwr5qk+fxlgQrlNe6tPHnxsUlD59GiP7lHH59s7bAQCWZS1V7eXnPhXE68Q+aa5PQNrPSjPrU2ZjLF8i3w32DR06FK1bt0azZs3UBvuuXbuGpKQkNGvWTFXm4uICe3t7XLhwAbVr18aFCxfg5uaGYsWKqep4enpi8ODBuH37NqpUqYILFy6otZFaJ3W60MTERFy7dg0///yzaruOjg6aNWuW5mmsj82YMQO+vr5pym/fvg0TExMAgKWlJezt7fHy5UskJSUhPj4eQgjo6elBT08PCQkJajeovr4+dHV1VfV2bd6MUfHx2TqPvePiMG/TJowaPRpGRkYQQiD+k30LFSoEpVKpNkiZunaeQqFQm+rw4zdWXFycqlwul8PAwABly5ZFYGAg4uLicP78edWgS+qbbsqUKdi1axf+/vtvDB48GIaGhtDR0VEN8unr68PPzw9XrlxBWFiY6imm2bNnY/fu3diyZQv69u2LqVOnonPnzvD19VX1qWzZsoiLi4ORkRHkcjlMTExU7cbFxWHq1Kno2bMnfvzxRyQmJqJkyZL4/fffVdMYhoSE4ODBgzh9+jTc3d0hl8uxatUquLq6IikpSdXf9K7T8uXL0bVrV8jlcpQuXRqOjo7YtGkTevXqpfYk1rBhw1SD1JMnT4abmxtu3bqFcuXKYfbs2WjUqBF+/fVXCCFgZ2eHmzdvYvbs2ejWrZtqWs0WLVqgd+/eAFLWvVu6dCmqVauGNm1SplwbPnw4GjdujFevXqFIkSKwtLTE0KFDVdfpxx9/xMGDB7F582a4ubkB+G9AJ7VPQggkJSVBoVDg4sWLuHz5str1+P3339Wux+PHjzFs2DCUK1cOMpkMJUuWVJ13AGnuvUePHsHKykp1LyUkJGDatGn47rvvULx4cVSoUAG1a9fGN998oxp8j4mJwR9//IE//vgDXbt2BQDY2tqqBizXr1+P+Ph4LF26FMbGxnBxccGiRYvQtm1bTJ48GcWKFUNycjKMjY2xcuVK1S+iGzZsgEKhwPLly6Grq4vY2Fj89ddfsLW1xeHDh9GmTRvIZDK1+x0AypUrpxp0PX36NDZv3oz//e9/sLKywnfffYdvv/1WNXjWokUL+Pv7Izk5GUlJSWrvJwMDAyQlJaFo0aKqY6S+nxITE9X+B6GnlzLtX3Jyslo8n+YIAHj27BlKlSqldh1S3w8vXrxA4cKFERcXpxr0u3//PhwdHdPkiNS8EBMTg6CgIFW5oaEhXFxc8P79ezx58kRVbmpqilKlSiE8PFztqeLUvPf06VO8e/dOVW5jYwMbGxuEhYUhKuq/Nczs7OxQpEgRBAcHq8Xj7OwMMzMz3LlzR+3clCtXDvr6+ggMDFS7Tm5ubkhMTFSLXS6Xw83NDVFRUWrrFUrRp9T3GoAC0yegYF2nyPuRiHsVh7p96mqkT7d8b+H+vvtQ2ilhq7TNk9fpw9MP+PD0A/yv+ENu+N/883n5OmXVp/x47xXEPgkh8O7dO0RHR8PCwqJA9KkgXif2iX3K6306sfAEbs+/jfqr66NIlSJ5tk9xcXF49+4dbt++jVKlSmnddWKf8kefAq4H4P7K+zB3MUfxxsXV+nRu9jmYljLF86fPceO3G+i4qyMUDgokOiSi0dZGUNintJfX+hQYGIj41/GIfhQNq+pWBeI65Yd778GDB6qcJ5PJCkSfCuJ1yqk+VaxYEd+s+gZRsigUqVIEgYGBeaJPof+GIjkmGc7dnXmd2CeN9snCwgKRkZGqnJdVnz4djPxqIh/ZsmWLqFixooiLixNCCOHh4SGGDx8uhBBi06ZNQl9fP80+NWrUED/99JMQQogBAwaIFi1aqG2PiYkRAMSBAweEEEKUKVNGTJ8+Xa3O/v37BQARGxsrnj17JgCI8+fPq9UZO3asqFmzZoaxx8fHi8jISNXPkydPBADx7t07kZycLJKTk4VCoVDFdPv2bREbGyuUSqVQKpVCCKH698c/H5eXsbUVxwAhsvFzFBBlbG3TbfNzjvnxT58+fUS7du3Srd+1a1dRvnx5oVQqxZ9//il0dHSEsbGx2o+Ojo4YO3asEEKISZMmCXd3d7V25s+fn+l+SqVSGBkZiVWrVmXYJwcHBzFv3jy1surVqwt9fX21NgsVKiQAiDt37oh///1X6OrqiuTkZLU+WVhYpGnr43Pz7t07YWhoKK5cuaIq//3330X9+vVVdUJDQwUAcenSJVXZ27dvBQDh5+cnlEqlqFKlipg8ebLacf7991+hp6cnkpKShBBCODg4iFmzZqm2KxQKAUBs27ZNVRYSEiIAiICAAKFUKkVSUpLw9fUVFStWFIULFxbGxsZCV1dXdOnSRbVP6nssvfO3aNGiLK/HpEmThK6urmjatKmYPn26CA4OzvSe09fXF9u2bUtzjyUnJ4tLly6J+fPniw4dOgi5XC769u0rlEqluHjxogAgQkJC0r33RowYIRo1aqRWHhERoXaO+/TpI5o1a6Z23NGjRwu5XJ6mfzKZTPz111+Z9uPTn4iICNGjRw8BIM175HPfZxmVp3dvp1e/efPmYsCAAWrlt27dEgDE7du3VWUJCQkCgNi/f3+6x4uNjRV37twRsbGxqhyW+iOEEAqFIlvlqXkvo/JP28ioPLWv6ZWn3kPZKU89L1L3KSEhQQQHB6epm5/7VNCu04paK4QPfIQiOeO+fk6fNrbcKHzgI+Kj4/PsdTo+4bhYUmmJiI2MzVaf8sJ1Koj3XkHsU0JCgrh//77q95qC0KeCeJ3YJ/Ypr/fJBz7CBz7i/ZP3ebpPqTkvISFBK68T+5Q/+hQbESt84CP+6f1Pmj6F3wsXb0LeiMcXH4tjvxwTr4Neq7VxceFF4b/OP8/1KTk5WcwuOlv4wEdEPo8sENcpP9x7H+e8gtKngnidtK1Pqb8zFKQ+FcTrlB/7lJycrJbzsupT6mfUkZGRQhPyzZN9T548wfDhw3H06FG19bzyCwMDg3TX1JLL5ZDL5WplqY92frqu3Mf//lhqubmZGV4+f56teF4h5WmejNrM7jGzu+3u3btwcnKCTCZDTEwMihcvDj8/vzT1LCws1Nr4uK2EhIRM90t96vDT85ZefB9vj46OxqBBgzBs2LA0de3t7XH//v1098usDAC2bNmC+Ph41K5dW7VN/P+TjMHBwShb9r91j/T19VX7pT4lKYRQOw/p3Qsfl33cRup/02tXqVRCJpNhzpw5WLhwIRYsWAA3NzcYGxtjxIgRSExMzLBPH8cSHR2d5fXw9fVFz549sX//fhw8eBA+Pj7YunWrah3FT1lZWSEiIiLNMeVyOWrWrImaNWtixIgR2LhxI3r16oWJEyeqnmzM6Lqnd64+7QsA1dSqqa9jYmJQrVq1dNdntLa2zvK9AwDXr1/H+vXrsWXLFshkMowaNQr9+/dX7btp0yYMGjQo0zYOHjyoNo1nRn35tD8Z1bexsVGtL5panvq6ePHiqrL3798DAIoWLZrlef00hwH/3W9fW55e2zldnhf6lPo0cGbyW580XS51n1w6uMDY2hjJccnQN9H/6j79b/X/EB8RDwPjtL8r5JXr1GRqEzSZ2iTduhm1I/V1yoly9knzfZLL5ShTpky262enXOo+5UQ5+8Q+aSrGzy3PL31y7+0O6wrWsChpkWV9Kfv0ac7Lqn52y/PLdWKf8kefDEwMUGdMHRhZGqX5O8W6XMqU80Wci8Culp1qW3J8MpJik3Bm6hkUdi4M997unxV7blynDhs64PXd1zAyN1L9TZufr1N+uPf09fXT5LzM6ueHPhXE65TTfTo7/SxMbExQbUC1LGPMjT6V+185vLjxIk1+y247BfU6sU+a6VN6OS+zGDUp3wz2Xbt2DeHh4ahataqqTKFQ4PTp01i0aBEOHz6MxMREREREqAaMAODVq1ewsbEBkPIh9+XLl9XaffXqlWpb6n9Tyz6uY2ZmppoGUi6Xp1sntQ2ptOvRAxumT0fPbEzlud7ICO179MiFqFLWrQsMDMTIkSMBAFWrVsXLly+hq6ub7lpgQMovA4qPHmsVQqBSpUpZ7lepUiUcP35ctZ5iVu2mxnPnzp0MP1h3cXFBcnIyrl27hho1agAAgoKCVINSGVm1ahVGjx4NLy8vtfIhQ4Zg9erVmDlzZqb7p3J1dcW5c+fUys6dO4eyZctmmFSy49y5c2jXrh2+++47ACmDgPfv30f58uWztX92riMAlC1bFmXLlsXIkSPRvXt3rFmzJsPBvipVquDOnTtZHjs1xpiYGJQpUwZGRkY4fvw4+vfvn6auq6sr1q5di5iYGNWA3rlz56Cjo4Ny5cpl2r9t27ahaNGiMDMzyzKmVE+fPsXGjRuxYcMGhISEoG3btli1ahVatmwJXV31lPu///0PtWrVyrS9EiVKZPvY2VGnTh1MmDABSUlJquk/jx49inLlyqmm7gSAW7duQU9PDxUqVNDo8Sl7lEolwsPDUbRo0Qx/qSBp1R9XHxinufaMChshKSYJce/jYFTYSHMNa9CObjtg7mCO5r83lzoUKmCY84g+38uXL7FyxQrs3rwZkR8+wNzMDO169ED/AQMk/7tUKu3XtZc6hGxhzqP8QEeug3v/3IOBuQEa/Kz+5dOoF1HQNdRN8zvrzY03sXfAXjSf0xxl25RFXlSqRSmUalFK6jC0CnOedkuMTsTC0gsR8yoGjo0c1Qb7pPTt7m+lDoEKKKlzXr7Jsk2bNkVgYCD8/f1VP9WrV0fPnj1V/9bT08Px48dV+wQFBeHx48eoU6cOgJQPuQMDA1VPsQApH3KbmZmpBg/q1Kmj1kZqndQ29PX1Ua1aNbU6SqUSx48fV9WRSv8BA3AyKQmXsqh3CYBfYiL6pTMw8rUSEhLw8uVLPHv2DNevX8f06dPRrl07tGnTRrWeXLNmzVCnTh20b98eR44cQVhYGM6fP48JEybg6tWrAABHR0eEhobC398fb968QUJCAho2bJjlfpMnT8aWLVswefJk3L17F4GBgZg1a5YqPkdHR5w+fRrPnj3DmzdvAADjxo3D+fPn4e3tDX9/fwQHB2P37t3w9vYGkDJ/bsuWLTFo0CBcunQJ165dQ//+/WFklPGHsf7+/rh+/Tr69++PihUrqv10794d69atQ3JycrbO6ejRo3H8+HFMmTIF9+/fx7p167Bo0SKMGTPm8y/QR8qUKYOjR4/i/PnzuHv3LgYNGpRmEDszWV3HuLg4eHt7w8/PD48ePcK5c+dw5coVuLq6Ztimp6cnzp49q1bWuXNnzJ8/H5cuXcKjR4/g5+eHoUOHomzZsnBxcYGhoSHGjRuHn376CevXr0dISAguXryIVatWAQB69uwJQ0ND9OnTB7du3cLJkyfx448/olevXmrrd36qZ8+esLKyQrt27XDmzBmEhobCz88Pw4YNw9OnTzPcz8HBAf/++y+GDh2KFy9eYPv27WjTpk2agT4gZU7o0qVLZ/qT2X2WmJioyoeJiYl49uwZ/P398eDBA1WdRYsWoWnTpqrXPXr0gL6+Pvr164fbt29j27Zt+OOPPzBq1Ci1ts+cOYMGDRpkenzKOUIIvHz5UuML9ZLmHBlzBDu779RYe/f338efZf7EzY03Ndampt3++zbOzz6PxJjErCsTfQbmPKLPM3/uXDiULImz06djxL17WPL8OUbcu4cz06fDoWRJzJ83T+oQJfHg8AP8WfZP3N93X+pQMsWcR/mBUqHEN4u+QevFrdNsW+i8EP/2+hdPzj/BhuYbEHI0BABg5WKFqgOrosw3ZWBVziq3Q86WiEcReOn/EkLJ919uYc7TcjLAsrQlqg6oig4b0//ivxQSohKQ8CFB6jCoAJI852lkMlCJeHy0Zp8QQvzwww/C3t5enDhxQly9elXUqVNH1KlTR7U9OTlZVKxYUbRo0UL4+/uLQ4cOCWtra/Hzzz+r6jx8+FAUKlRIjB07Vty9e1f89ddfQi6Xi0OHDqnqbN26VRgYGIi1a9eKO3fuiIEDBwoLCwvx8uXLbMceGRmZ4XyscXFx4s6dO6q1CT/HvDlzhLWBgbiYwVp9FwFhbWAg5s2d+9ltZ6VPnz4CgAAgdHV1hbW1tWjWrJlYvXq1ai7bVB8+fBA//vijsLW1FXp6esLOzk707NlTPH78WAiRssZhp06dhIWFhQAgVq9eLWJiYkRkZGSm+wkhxM6dO0XlypWFvr6+sLKyEh07dlRtu3DhgqhUqZIwMDAQH9/+ly9fFs2bNxcmJibC2NhYVKpUSUybNk21/cWLF6J169bCwMBA2Nvbi/Xr1wsHBwcxf/78dM+Ft7e3KF++fLrbXrx4IXR0dMTu3btVa/bduHFDtf39+/cCgDh58qSqbMeOHaJ8+fJCT09P2Nvbi9mzZ6u1mV4sAMS///6rev3psd6+fSvatWsnTExMRNGiRcXEiRNF7969Rbt27VT7fPoe+/Q4mV3HhIQE8e233wo7Ozuhr68vbG1thbe3d6b39du3b4WhoaG4d++eqmz58uWicePGwtraWujr6wt7e3vh5eUlwsLCVHUUCoWYOnWqcHBwUJ2jj9fevHnzpmjcuLEwNDQUlpaWYsCAASIqKkq1PXW9yU+9ePFC9O7dW1hZWQkDAwPh7OwsBgwYkOk8ynfv3s1wm6alXtNPfzw8PFR1Jk+eLBwcHNT2CwgIEPXr1xcGBgaiRIkSYubMmWnaLleunNiyZUuGx/6aPEVZS05OFjdu3FDNG055j6/cV/jAR0S9iMq6cjYscFogfOAjHp19pJH2csKBYQeED3zEh2cfpA6FCpiCmvOeX38u9g3eJ55dfSZ1KFSASPn3Xl63sMxC4QMfcX//falDyVRBzXlUsES9iBI+8BH7vfen2XZo1CFxZekV8fDEQzHddLq4sfaG2vaEqASREJ2QS5F+ntR1usJOhWVdmTSCOY+kEB0eLY5POC7iP8Snuz01FxBp2ufmvMzGiL6ETIj8+9WKRo0aoXLlyliwYAEAID4+HqNHj8aWLVuQkJAAT09PLF68WG0ak0ePHmHw4MHw8/ODsbEx+vTpg5kzZ6o9dePn54eRI0fizp07KFmyJH799dc0UzEuWrQIs2fPxsuXL1G5cmUsXLgwy+n4PvbhwweYm5sjMjIyzRSB8fHxCA0NhZOT0xetTzh/3jyM/+knNNLTQ+/4eBQD8BLAekNDnEpKwszff8fIT57iyeuEEIiLi1OtyUcF19ixY/HhwwcsW7ZM6lC01sGDBzF69GjcvHkz3ScSga/PU5Q5hUKBwMBAuLm5fdV0vZRzTv12Cn6T/fDDzR9QzC3jp4Sz69LCS4h+GY2m05tmXVki70LeIfpFNGxr2ELXIN/MBE/5QEHNebv77Yb/an80mdYEDX5pkPUORFl4+fIlHEqWxGmFApn95XkJQEO5HI+ePtWqKT19dXzh2sEVXXd2lTqUTBXUnEcFS3xkPI6OPQr7+vYZrr0nhEj385n5dvNhWsIU/S9qfjapr/XPd/8gcFMg+l/qjxI1NbtkBqWPOY8AIOp5FGRyGUyKmeTK8c7PPY+jY47if6v+hyp9q6TZ7ivzBQBMUkyCTIefM5PmfG7Oy2yM6Evk68G+/CwnB/uAlD8EV61ciV2bNyMyMhLm5uZo36MH+vXvny//4BNCIDExEfr6+hzsK+AiIiKwePFijB8/nvO5S2THjh2ws7PL9AsMHOzLWUqlEk+fPkXJkiX5Psij4t7FIT4yHmYlzCDX144/WoMPBsOosBFK1i4pdShUwBTUnHd03FGc//08Bl4fiOJViksdDhUAU6dMwdnp03EoG2u0exoZoeEvv2DCxIm5EFneEPcuDjp6OjAwNZA6lEwV1JxHBc/y6ssR9y4Owx8Oz1b94IPBuLn+JhKjE1Gidgk0nNAwhyOk/IA5T7slRCXg4oKL8JvkB7u6duh7rm+uHPfppae4sfoGqv9QPd3fwyOfRCIxKhFWrlb8nJk06nNzHgf7CoicHuwjIspJzFOk7UJPhgIAnBo7aaS96FfR2NV7F1w6uKD6D9U10qamTTeZjhI1S6D38d78g4goGxRJCiTHJUPPWA86cn64RV+vhqsrRty7h57ZqLsJwAJXV1y5cyfNNmWyElP0puTqh265IepFFG7/fRv29e1hW81W6nCI8r2Tk08iMSoRnvM8VWVCKfB3p79h39Ae1QdVx4PDD1DYuTBs3G1wccFFHB55GH3P9YVdXTsJIyeivOLD0w+YbzcfAOC5wBO1h9eWOCKivEXTg338q5PyBSEEEhISuKAvEWkFpVKJx48fQ6lUSh0KZWBP3z3Y3HozkuOTNdLewR8PIuRICCIeRWikvZxgYmOCsJNheBv0VupQ8r3EmETc33cfb+69kTqUPKGg5jxlkhKRTyIR/z7rp7AKiqS4JMS+iYUyuWBdy7wi8sMHZHeOlmIAIiMj092WFJsEAHhy/olmAvsMMa9jEHoiVOPtCiFwf999HB5xOEfa16T8kPPiI+Nx99+7iAiLkDoUksj5uecRejwUttXVB86VCiWC9gbhxbUXiHsfh787/g3/Nf4AgFrDauHXpF/z9CwQf3f+G74yX7y9z99nc0t+yHmUc4yLGeOHgB8w8unIXB3ou7/vPtZ6rMWzy8/S3X7A+wB29dkFoeTnzKRZUuc8DvblYRzYUqdQKKQOgYj+H/NTzhJC4N27dzzPeZhlaUskxyXj2ZX0/3j4XG/uvUFRt6JoNqOZRtrLCU1nNEXF7hWhb6IvdSj5XtTzKGxpuwUB6wOkDiVPKKg57/6++1hScQmurbgmdSi5ZtM3mzDbejbeh76XOpQCydzMDC+zWfcVgKTniXhyIe2AnlxfDs/5nvju8HcajS87dnvtxvqm6xHzOkaj7QqFwL6B+2BW0gxuPdw02ram5Yec9zbobcogzjp/qUMhCTy/9hxHxxzFk3NPEH47XG2bXE+OScmT0HFjRxhZGqHj5o6qNf1kOjLo6Org8KjD8PPxkyDyrIUcDgGANP2inJMfch7lHLmeHMUqFYNZia9/YulzPD73GI9OP8K93ffS3X7lrysIWB+AuPdxuRoXFXxS5zxdSY5KmdLT0wMAxMbGwsjISOJoiIjSio2NBfBfviLSNh6TPVDMvRhMbU010t7gm4M10k5OqtClAip0qSB1GAWCSTETtFvbDkUrFpU6FMpBt7fdBgBYlbOSOJLc8+jUIwDI82um5VftevTAhunT0TMba/atkevDWeEGfeO0X9DQNdRF7RHSTKNVonYJRDyKgK6Bhj+KkAGNpzSGdXnrXP9AsSAysTEBANzfcx+NJjeSNhjKdUUrFEXPgz1xb9c9xL3L+INwPSM9uHX/b3A98kkkIh9FImB9AIqULYJGPo1yIdrPMzx0ON4/fA/LMpZSh0JUYCREJeD1ndcwtzeHaXH1v4+VCiXiI+JxeMRhFLIupDYtcE4q7Vka52aey/Dv9fbr2+PJuSecap8KHA725UFyuRwWFhYID0/5plGhQoW0fm2c1Gk8ZTKZ1p8LIikJIRAbG4vw8HBYWFhALpdLHRKRJOzr28O+vr3G2hNCIGhPEEyKmeTZqY8Wll4IBw8HtFvVTupQ8j2lQomgXUFQJCi4rlQBVn1wddjWtEWpFqWkDiXX9PHrg5hXMaqBAtKs/gMGYIqvLy4BqJVJvUsATiuTMQxVYVk6/Q+0T0w8Acsylqjcp3IORJoxj1894PGrh8bb1ZHroMGEBhAKAUWSAnI9/o76NUyKm8C9jzvXXdNSuoa6KN2yNC4uuIgn556gzZI2qm3KZCVCT4bCrKQZrF2t1fa7ueEmTkw4gX4X+sGmSnYnHc5dhawKoZBVIanDICpQHhx8gB3ddqDxlMZoOLGh2rZ3we/wl+tfAJCrf+c6NHTAr0m/QiZP/zNk917ucO/lnmvxEOUWDvblUTY2Kb8YpQ74aTshBJRKJXR0dDjYR5QHWFhYqPIUaZ5MJoONjQ3zXR62ut5qGFoYosf+Hhpp79GpR9jWfhtc2rug27/dNNKmpr0PeY/3Ie/RcGJDFHYqLHU4+ZoyWYmgPUEwLamZJ0Pzu4Ka85ybOcO5mbPUYaQRERaBB4cfwKmJE4qUKaLRth09HDXaHqmzsbHBzFmz0HbCBOxNSEh3wO8SgLYGBpg04hf0atUHcoO0g14fnn3AmWlnACDXB/uCDwYj9EQo6o+rr/EP3D88/YAF9gtQe2TtXHty4Evkh5wn15Oj/dr2UodBEkmISsDjs4/h0t4lzXspISoBG1tsRNUBVdHyj5ZY6LwQrp1c0WpRKzg3d4aesR4Klyqs+ad3NST0ZCjeP3wP997u/FJALskPOY++Tuq0uDp6aZ+SMzA3QNWBVVGmVRm4tHPJtZjeP3yP59eew6mxE4yLGqfZ/uL6C8jkMti483Mt0iypc17e/L8vQSaToXjx4ihatCiSkpKkDoeISEVPT49P9OUwHR0dDqbmceG3wpHwIQHPrjxDiRolvrq9vzv/DQCoPVKaadWyo/mc5jg65iiiX0ZzsO8rxb+Ph1lJM1iW4hRSQMHNee8fvsfu73fD3csdVb6vInU4KkfGHMHdnXdRvnN5dNneRaNt399/HzfX30TjKY1RpKxmBxIpxcjRowGZDA1/+gmN9PTQOz4exQC8BLDe0BCnkpIwc/p0dG3QDYGbAmFkaZRmymBlshKQAbWGZ/Z8YM449tMxhN8KR7WB1TQ62Jccn4xNLTcBAGwq5+18kh9y3tNLT7G9y3Y0/q0xKntVljocymVv7r3B5labAQBNpjeBdfn/nuDTM9JD66WtYeViBR1dHRQuVRjGxVI+SC9RowRK1CiBF9dfIOZVTJ6crnxD8w0QCgEduQ7v7VySH3IefZ3yncrD0NwQ5buUT7PNtLgp2i5rq/FjKpOV2OW1C9UHV4d9vbQz7tz99y6O/XQM7da0S/e9vrzacgDA4MDBeTJXUf4ldc7jYF8eJ5fL+aE6AIVCgbCwMDg6OvJ8EFGBx5yX97X6qxX2DtiL2NexGmmvxdwWMCpsBIeGDhppLyfUGFIDVfpW4VpcGiCTy2BczBh6xlz3FCi4Oe/01NN4dPoR7OrlrWnwKnaviNDjoajUu5LG297SZgsAoOaPNTnYl4NGjhqF7j16YNXKlRg9azOSkiJhbGyOQSN7YH3//rCxsYH/On9c+uMSHBs5pvkQy8LBApOVkyWJ3cAs5f8hFo4WGm1XKAXi3sWh5rCacO+dt6flyg85T64vx4cnH3Bj9Y0vGhDZ0W0HkuKS0H1Pd80HRznO3M5c9e9CRQpBCKF6QkHXUBfVB1VXbe97tm+a/be23wrT4qbof6l/zgf7mVovbo19g/bh0alHqPRdJejocr2unJYfch59nWKViqFYpWKZ1nl89jEgQ7oDc1/ipf9LBG4KxOvbrzHoxqA021OnGX568Wm6/x8rUbMEnl1+huhX0RzsI42SOudxsI/yjaioKKlDICLKNcx5eVul7yqh0nea+6A8t6dR+xIXF1yEtas1XNrn3vQrBZWFowWaTm+a7pQy2qog5jyT4iYwtzdHndF1pA5FTflO5VG+U9pvXmtCzR9r4m3QW42uaUrps7GxwYSJE3Hx0kQAQOnSwISJ/22/u/MuAMC5efpTyX549gFyPXmu56G2K9oi7l2cxqfP0yukh9EvRmu0zZyU13Ne8SrFIdORwdj6y+6P23/f/uJjL6m0BOGB4ZgspBmQJsDExgSTxWTs7L4T+wbtQ5W+VSDTzXo6sksLL+H6yuuo1KsSilbImx+eVxtYDednn4f/Wn988+c30DfRlzokrZDXcx59nbj3cdjQfANc2rukWbPv/cP3OP7LcdzedhtF3Ypi8M3BGjlmMfdiGB42PMMvotrXt4fnAk+UqJn+LDz9Lvbj1LKUY6TMeRzsIyIiIvpMl/+6DANTA40+PbC82nKY2pqi+94v/xZ8Ykwi4t7GwdzePOvKn0EIgRO/nIBlGUs4NXFSPZlBX0YoBDY03wC3Hm7ouKmj1OFQDmk6rSmaTmsqdRi56puF30gdgtYSQv21czNn6BXSg55R2ieI497FYX7J+TAqYoSf3vyUSxGmMCtpBr1CekhOSNb4ml5CKbBv8D4Ur1Ic1X+onvUOlKnxH8Z/8aDsN4u+gSJB8UX7hgeGf9F+pHkVu1eETRX1qciiXkRhrcdaVOlXBfXH1cfZmWdh7mAOt+5uUCqUUCQoUG1gNVg4WEgTdDYkxaYslZPemqZE9Pn81/jjxbUXKOycdqmH2DexuL3tNmxr2KL++PoaO6ZcT55pnjG0METt4RkvkcGBPiqo+Lw6ERER0Wc6NPwQdvXZhagXX/+NLSEE/izzJ15cfwGT4iZf1dbSSkuxwGEBhFJkXfkzVRtUDe+C3+Hpxacab1vbxL6NRWHnwrCuYJ11Zcq3EmMScfvv23gZ8FLqUNRcmH8BvjJf7Om/R+Ntx4TH4NHpR4h9q5kpjunLVelXBY1/a4yEqIQ02yKfRAJImY4vt11edBl/OP2h8QEdRaICt7bewvXl1/Hw6EONtp3XJcYk4uExzfY5IiwCl/+8jNd3Xn/R/jWH1kSdUV/2VPMPAT+g7/m0U0NS7nly4Ql8Zb7Y2m4rzEqaqU91KVLW7UstOz3lNG5uuAkAqDOyDryDvPP0QN/iiosR9TwK1QZV0/gTxkTaSq4vR+FShdF4SuM022xr2OLX5F/R70I/uHZ01dgxFYkKrKy9Ekd/Opru9itLruB3q9/x7PKzdLcvr74cc23nIiY8RmMxEeUFHOyjfEEmk8HOzo7fvCAircCcl/fV/znlW4mPzz7+6raEUkDfRB91xtRB2+Vft3h5ufblcmTNAZlMhmqDqqHxlMYoXCrtNzbp80Q9j8L7h+8h0+F7HCi4OS9oTxB2dNuBG6tuSB2KmqhnKV9SEArNfylgZ/edWOuxFk8v8EsBue3TJ/vCToZhUblFuLfrXpq6hZ0Lo+ehnvA65ZU7wX3kxIQTAKDx6UMTYxLxT89/4NrJFR02dtBo25qm6Zx3/Ofj2NB8A25tu6WR9gDgbfBbHP/5OALWB3zR/okxiUiKS/qifYtVKga7OnlrrVNtk9n/H0xtTfFDwA+oO7ougJSp8Fovbq1WZ3Przfi31785GuOXSp2dIqv1xUhzCurvefSfmt41MezBMFiVs0qzTSaTQUeuAx25Zocg3gb/H3vnHdDU9b7xJ2HvvWXIEAQEBNw4UNzWvbd1ts466qjVL1Zrp9Zd96x7T5yoOED2lL333jvJ74/8iMZ7LpIQCtj7+Utzck5OSHLuued93+cpQIZfBt78/obYLiUrhaqCKtr2rMAslGeVI+VlikTnxcDQ2mseI+PJ0C5gs9nQ0tJq7WkwMDAw/Cswa17rU5xcjPc33sNisAUxeNb1667Q6KgBQ1fDZr8WW4pNNBUXh6F/DpXIOJ/C4/GgaaGJ3mt7t0olyJeGto02FvovhIqRSmtPpU3wpa55cXfjAAC2E1vGH09cBv82GB6/eLRIsDknLAcA/lNVqzVlNWCxWG3O9ynpWRIAEK9hcipysBxq+W9PCQAwwHMAStNLJS43Laski8nXJ0PdVJ0oXdqWkPSa1+3bbpBTlYO2DfWQVVw69OgAAGIfgu5U3gkA2FyzGVKyolVPZQZmor66Hsa9meBAa2HiZoK1OWvhu8cXvn/5wnKYJRQ0FYjP1evyIWiW+joVuRG5KMsqE/lz/7eY93IeHn//GA9WPIDVCCuom6m39pS+eL7UfR6DMOHnwyGrLAvr0dZCj1cXVyMnLAdeq7ygqKWIWY9nSeT1FLUU0Xtdb5h7kL2J7afa4/Wvr6FppUlsXx63HAmPEyhSxQwMzaW11zzmtIahXcDhcBAXFwcrKytISbXNTSMDAwODpGDWvNYnPyYfj1Y/wldHvyIelCpqK8J+qj1kFCV3oBh0LAicWg66fdtN7DFqK2pRX1UPBS0FiR6QcWo4+EXtFzjMdMC4s227YqI9IKski1vzbqFDrw7Nrub8EvhS17w+6/vAfpo9OvTq0NpTEYLFZrVYVel83/mor66HRsf/TgXwL6q/QLWDKr5L+65V5/FpZV9xcjHAAgxdyEkpN+fchJK+Egb/OrjlJ/cR/bf0b5FxpWSl0HlcZ+RF5SE/Jp9YXdBWkPSap22jjYHbB0pgZh+QU5XD0N1DodpBtVnjcDlcSEG093jU9SgAYH3xesiryTfr9RnER0lXCSw2CxU5FeDWcwWPVxZUIuxsGDr06oAOPTqgurgaAN8fK/JyJN7tfYfVGauhYtg2E5rY0mz47vYFgDYbkPzS+FL3eQwfSPdLx/UZ16HbRZcS7MsKysKZQWcAADZjbST2msr6yhj8G/0eRlZJFstjl9O2a1pqQtOSHAhkYGgOrb3mMTKeDO2G6urq1p4CAwMDw78Gs+a1LuXZ5QBAW6mxz2ofzg09J5HXqquqw9vdb3Fn4R3B4YO47FTeid91fkdtWa1E5tYAi82Cmqkaws6FCapFGMSHy+GCy+EyMp4f8SWueXoOeug0qlOb8wTKDs3GGY8zRHnH5qJpoQldO8lLCbdldO11JVLlLWnGnh6LdXnriG1ZQVkIPROKN7+Rpa1akniveJwfeR5ZwVktMv7ZIWdxa96tFhlbkkhyzSuIK0BmQCa4HO7nn9xEuPVcdJ3fFdZjrD//ZAJbeVuxlbdVrCpL036mACDsE8fwr5ITloMrk6+gMK4QMx/OFJLdLc8qx8PvHiLeKx4A3wPvn+H/AOB7Nc56PAsKWuQqwLaA314/APzqxbYakPwS+RL3eV8SGe8y8Hb3W7H7R12JAkBO6NG00sSwPcOwJHQJptyYIvZrkEjxSaHdT6T4pMB7qzfKMsuI7WHnwhBzJ0ai82FgaKA11zxm98TAwMDAwMDA8AkKGgrQtdeFsoEysV1JVwmpr1IRdS2q2a9VU1qDR6sfoeOgjph2Z1qzxwMkf0AmJSuF4fuGQ1lfGbXlkg0k/hfJeJeB/Pf50OrESBp9yVTmV+J3nd/xaN2j1p6KEAGHApD0NElw4ClJIi9HYrfxbsQ/jJf42G2Vb8K/kfjhlSQoSijCtanXiAdZcqpy6PRVJ0y6OonYtyK3Amlv0wQVO5LEa5UX4u7HoaqgSqLjVuRW4Bf1X1BVUAWXxS4SHbutc3nCZRztdhShp8Xz1yMR/zAev6j+grBzYRIbs6nMfTEXW3lbIavUtqRx/0skPk1E1JUoRF6KRMa7DKE29Y7qmOczD05znQAADrMcBEFhrU5aMPcwR/TNaETfknxCiSR4uvEpAMBmnOQqjBgY2jp1VXUIPhGMxCeJxPZjPY7h0epHqCmrEWt8l0UumOE1A1YjrShtasZq6LGih8R9MgvjC3Gq3ykccT5CbE99lYqX214i6HgQsf3GrBu4OPoiIi9HSnReDAytDRPsY2BgYGBgYGD4hE5fdcLCgIUw7m1MbB9xYAQMnA0kIv+joKGAea/mYdTfo5rtt7OxbCO+L/xeovKiDVh/ZY01WWso0iwMoqOkowSnuU7/KV+z/yIPlj9AZX4l5FTlWnsqQjjNc4LzImcM3S15j8+rU66iNL0UPC7v809mkCifynjmhOcg8UkiKnIqKM/VtNTEtNvTYDuB7Cfpf9AfJ3qfaJGgrby6PFSMVGg9dsSFLc2GoashPH7zgNMcJ4mO3daxn2YPAKivqZfYmGrGfE/FN7+LV/150P4gPFmeqMilfv8Y2j62E2xh3McYWp20UJpeypcF/n9klWRh4mYCdVN1AIDHTg+4bXAT6v9k/RP47PD5F2fcdOY8nwMtay1E34wWel8MDF8yeVF5uD3/Nnx+Jv8u1+auxbdR30JORbw9q1YnLVgOtWy0mjvsnzAEHSMH3sShITDZUA3+KS4L+Yk/yc+Sie2DfhkksbkwMLQlGM8+hnYBm82Gubk52GwmPs3AwPDlw6x5rU9OaA4Odz0M9+3u6PdDP0q7aV9TLApcJJHXkpKVgkkfE9SW16KqqAoKGuJLH9HJjjaXuso63F18F6YDTOE837lFXuO/hKalJox6GklcbrW98qWueYbdDFFdXI0+6/q09lSE6NCD77PUEgzaOQgFMQX/GQ8UTh0H22W3A+DLFrYlIi9FQkpOCs4LyGt28otkSMlIEZNaGiSvODUcic9r1qNZLRIMVtBUwOwnsyU+bksg6TWv78a+6Luxr0TGakDPQQ+GrobQMBfPfzMvMg8AUJ5TLiQB2RQ8WZ4AgPVF6yGvznj2tQZqJmr4+tXXiLgYgWvTrkHLWgvqZuoAAB6XBy6HC7Y0m+IPfWfxHQQfD8bkq5OhpCfa5/5vYdTNCNw6LlJ9UlGaXip4Xwwtx5e6z2tPZAXypS5tJ5KTfJR0lKCk07zf7LGex1BdVI1lMcuEHk/yToLXCi/kRuRC3Uyddl8iKgZdDRrdeylqK2L2s9lQ0CTfW7utd4PbejdiGwNDc2jtNY9ZaRnaBSwWC6qqqpTNJAMDA8OXCLPmtT4lqSUAQHtzcO/be/DZKV7GMu+T8gselwdOLQfnR53HPqt9Yo3ZgPdWbxywPYCqIsnKo3FqOQg7FwavlV4oTCiU6Nj/VZ798Az+B/xbexptgi91zeu1uhdmPJgBaXnR8ysLEwqR+DQRtRWSDwhzOVxU5FaILdXUGG4b3JAZmIn9nfZLfOzmkBmYicAjgagqlOzayK2TnEdac/m0ss9hpgP6b6V65wBAcXIxTg84jRN9ThDbuy/rjuH7h0u8+g4Aaitqkfo6FWVZZA+d5nJr/i1cGnepRcaWFO1lzVvovxATL00Uq+93ad/hm4hvoG0tvmJBXVWd2H0ZmgePxwOPx4NpP1NMvz8dJn1MBG0Z7zKwXXY73u7i+3s9Xv8YXqu8AAD6jvqwHm0NiyEWMO5FVsdobSryKlCUWAQlXSUYdmt7fqtfIu1lzfuSsRxmien3psN2EjnYF/cgDs89n4t9D3l/+X1k+GUQVUt4HP69rvtP7ph8bbJY44tDbXktdGx1/nNe0gytT2uveUywj6FdwOFwEB4eDg5H8tmlDAwMDG0NZs1rfQpiCwDwMwZJBBwKwLNNz5Adki3y2Le/vo0dijtQXcL3QsqPzsd2ue3ICsyCwywH8ScN4OW2l8h/n4+6CskekMmpyWHqramoq6hDsneyRMf+L5IfnQ9FLUVYj2UkUYEvd83j1HHgvdVbLG/PkJMhOOtxFqVppRKf14ttL/CH3h+4OuWqxMcuTi5GbniuxMdtLrF3Y3F38V1BxZqkkFGUwVbe1jZX1QcAFkMsoNNZBwVxBZS2hr8DnfSVnoMeui/tDhVDFYnPK/h4MC6MuoBUn1SJjltdUo1nm58h5ERIiwUSxSX4ZDAern4o+L+k17zzo87Dk+WJzIBMiYwH8APkN2bfQNqbNGJ7WWYZ0n3TaX18VTuoQtdOVyy58x/rfsTm2s1Q1if7JjO0PE82PME29jbsMtqF+qp6oc9CQVMB9lPtBdLziY8SEXsnFgDQ7dtumHJ9SovIyUuKQ/aHAPAlraXlGLGzf4MvdZ/XnlAzUYPlcEvaRNbzI87jxf9eoCxDvOunjq0OrEdbY+zpsZQ2cw9zLItZhn6b+8HAmXxvLQ5VRVXwZHnieK/jxPY3f77Bn/p/Ij86n9juyfKU+LWTgQFo/TWPCfYxtBuYjQEDA8N/CWbNa13sp9lj7ou50LEle6pNuzsNAJDumy7y2DG3Y1BfVS+QuJJVkUWX6V0w9vRYDNs9TPxJAxh3bhwG7RwE1Q6qzRrnU1gsFox6GGHy9cktUunxX6MsqwwFsQVgS/93tuJh58LgyfKE/yFyNeOXuOaF/xOOl9te4v3V9yL3NR9sDpfFLi0wKwgOYTv0lLyU5/mR5wEAY06NkfjYzcF+ij0mXJgAZYMvN3jwaWVfQVwBLo27hJhbMZTnGrgY4Lu07wTXsk+J94rH4a6Hkfg0UeLzbKgGknRFTU1JDXx2+KDn6p5Y4LtAomM3l7h7cfDd7StU2S/JNa/BdyziYoTExixJLUHY2TD47fUjtvsf9MfxXscRfTOa2B57NxY7FHYgxSdF5NdmS7MhJSPFVAG1Ih9Xwny6p9TqpIUJFyag08hOAPgVoMvjlws9Z7/1ftrK4dbGZrwNAKC+up42WC0K2SHZeLDygVj3BP8lvsR9XnvjkP0h/O34N7HN9VtXmPYzhZqpmlhjd/umG6bemtqo5x+njiNRb9n89/wgHt1vz6ibEQDg7OCzxHYtay0AQMKjBInNqaUpTCjEoS6HaO+nGNoOrbnmMWksDAwMDAwMDAyfIKcih9CzoShJLYHDTGq1nYmbCeb7zhfL52PcuXGIvBQp8DJRM1bD+H/GN3fKAACHGc2rDKSDU8dBUWIR9J30GW8TCWDazxQbSjb8p4J9uRH8aq+YmzHo9k23Vp7Nv0PiY36gZODPA0XuKyUrhcDDgdC00hRUT0iKlvQoqS7mVyzH3o6F42zHNnNYXxBbgGvTrmHsmbFwnOUosXE5tRzc+voWpGSkMOZk6wY4Pw32BRwKAABYjbCiPFdaTrrRpJDAw4HIDslGdnA2zAdJNsFj8G+DUV9dD42O4nnB0aFsoIyl0Uub5XvbUnSZ3gXmg1suUWbhu4VIfJIoUa9M69HWUNZXpq2IaNgLpPumE/dJF766AACIuBAB077kClI63u5+i8r8SvT/sb9YMsgMzcdxtiMcZzsi5WUKTvU/haG7h8J1iSvxuR/vZUJOhSArKAsdenWgrSBqbUYdGgUdWx14rfCCaX9TdB7XuVnj+e72ReiZUBTGFmLGgxkSmiUDg2TxP+iPvKg82kSbkQdGNvs13u1/h/Kccgz8SXjfWxBXgKSnSXix7QUAYE3mmma/FsD3QB9zagwMXcnvyWqEFVy/daWt4F3ovxBZQVkS34+0JBnvMpAbkYv7S+//Z+6nGESH2TkxMDAwMDAwMHxCfXU9go8Fgy3FJh5iFcQWQE5VDsp6oleJWA23gtVw6uGr7x5fZPhlwH2bO2SVZcWSrypOLkZxSjGMuhlJVEKpqrAKJ3qfQLel3TBi/wiJjftfhS3FxkH7g1AxUMECv7ZVgdJSuG1wQ4eeHYheHl8qQ3cPhft2d7EC5BodNTBwx0AY926bnkd0rEpdhd+1f8f76+/BredCSkZ0Cb+WQN1MHc6LnKFlpSXRcasKqxD+TzgAiBXse7f/HR5//xhzns2ReKVlfXU9NC01iRXqdVV1OGh3EMp6ypj/dj6l3WqUFQpiC2A53FKicwIA5wXOEh8TAKRkpKBtrY2ExwmoKamB7USyL1FrEHAoAJkBmXBdTA6WNBcZRRlYj5asLDRbio3x58fTynDaTrQFS4pFe8g6dPdQ5L3Pw4gDou8ZHq1+BID/XWlPh7DtkcL4QgSfCIb1GGt06EFdgxS0FNBxUEeh5IC893nw3e0Lh1kOMO1ripzwHNSU1sCkjwkSHiYg4mIEfqz7sU0nNHmt4HsMSiJA7vqtKwriCtBrba9mj8XA0FKodlCF1QgrePzq0ejzeDyeWIlaoWdD8WD5A8iry1OCfem+6bj3zT0YOBugQy/J7XWUdJXgNMep0ec0FsSUU5GDWX8zic3n38DEzQRum9yIiVwMDA0wwT6GdgGbzYa1tTXY7La7YWRgYGCQFMya1/oE/M2viOj6dVdi+2n30zBwNsDsp7Mph9mcOg6ygrKgpKtEPKQK+ycMueG5GLRzEFgsFvJj8vHmjzeIuRmDyvxKRFzgy3CJ4wG1p+MeAMDS6KXQtpZcNZCssizsptjB/4A/tG200X1Zd4mN/V+kuqQaKgYq0O+qL1b/zMBMSMlKQa+LnoRn1nLIq8vDZqwNse1LXfOUdJWAXL68oLy6vEh9g08GI/R0KJzmOUl8XolPE3HW4yw8fvNAn3V9JDo2W4oNHpcH1Q6qbeqgN/JKJEJOhKD3mt4SHVdOVQ59f+iLgtgCvNz+Ej1W9mhUwupTVI1V0dG9I2RVZEV+bS6HC249F+BJAYSDuYkXJ9L2zXiXgeKkYhQnFRPbnec7w3l+ywTlUnxScHP2TQz8eSC6TOsisXG59VxU5FbgyfonKE0rbVPBPpclLjCLNQOnjgMpGSmJr3kRFyMkHvCrLKiEjIIMtDqRA+Ty6vLoOo+8RwKAnqt6iv3a/bb0Q25YbputDPuSSH2Vilc7X0FWRVYo2Bd6NhQ3Z9+EvIY8FvgtEEqUKE0rRdDRIBh2M4RpX1N4rfRCdkg21heux8hDIzFk1xCwpNpGVTeJO4vuAACc5jpJZB/VoUcHzH9DTZpg+MCXus9rT1iPtm70GrHPah8K4wvx1bGvxLr+xz+IBwAsCVtCaTP3MMesx7Og56gHJR0lkcdujJDTIZCWk4b9VHtKW/j5cLy//h7D9w4nehA//v4xNCw04LzAGWyp9vHdVDNWw6Adg1p7GgyfobXXvPbxbWZgACArK/pNMAMDA0N7hVnzWhfdLrqwm2InkNr8FOuvrJHqk4p3+99R2qqLqnG853G8+eMNsa/XCi+8/vW1wGegNL0UwceC0W9LP/xY/2Oz5q1ixL+RkfSNlKySLPpv7Y+OgzrS/k0Ymk66bzoy3mXQVkR8jqOuR/G3A9lzo62SG5mLkFMhqMitILa3xzWPU8fBTzI/4dq0a8T2irwK/KH3B27NuyXy2GxpNqTkpMBiS/7A9P01vodg3N04iY/9+rfXqCmtgYGzQZuR8AQAbWttWI20gqyyZL9nMooyGLh9IBR1FOH9ozfqKutE6s+p4UDZQJkfGBaR1Fep2CG/A7rJ1OsQAORH52OX0S6BR97HqJmooc+GPvj69dfEvsUpxQg6FoSCuAKR5/U57iy8g+LkYlpZLXEpSirCLqNdkFWSxdgzYyU6dnNJe52GZ5ueoTy7XPCYJNe82wtu4+KYi3iw8oHExkx8kojjvY4j6VkSsT3oeBAOOx9GfgxZ5rMirwKHux7Gq19eifza7p7umHJjCuTVREuSYBCdosQiAFQP18RHfBnq6qJqVOQIX7fN3M3wfeH3Aknk7su7w+MXfrWQvLo8VAxU4LPDh7j2tAXeX+dfA5UNv1wP17ZIe9znfWm83f0Wj79/TGwrjC8EAKibqos19ogDI7AqdRVRIlzFQAXmHuYSvz9Ne5OGW3Nv0e7DcyNy8f7ae3it9CK2v/n9De4tuYenG59KdF4tCY/HQ0laCSryyPdTDG2H1lzzmGAfQ7uAy+UiPDwcXC63tafCwMDA0OIwa17r03lcZ7htcAO3jvwZuP/kDodZDtDpTJVHk5KTgssSF3Qc2JHYt/sKflVcWVYZAMCsvxnWF62Hy0IXsKXY+L7we2wo2SDWvFenr8ZW3tYWyYbX6ayD2U9mw26SncTH/q+hba2Nwb8PhlF3I7HHoKu2aKuEng7FrXm3cHnCZUpbe17zuPVcRFyMILY9WMY/eLcYaiHyuI6zHCGrJEs7dnPovbY3Zj6aien3pkt87CfrnwAA9LvqCwU2Whv9rvqQlpdGbmSuxMeuKa1BdRHfq5BTwxGpb9rbNAQfDxa5HwAoaCpAz0EPXDY/aPapZ1/qq1SUZZYRq0o1OmrAY6cHrUxs0NEg3Fl4B3H3JB8QVtZXhombCTqPb55P1qfIq8uj+4ru6P19b6JUdmti7mEOt01ukFXiH/xIes2beHEi5FTlJOrZp+/Irzy/Oecmsb2+uh7Zwdm4Pv06sf30gNPIDskGp1b07zbDv4fNWBuMODiCUuE24sAIjDk5Bu4/uSPyciTi7n9YC6RkpKCgoSDwU+w8rjNcFrkA4CewFcQWIPh4MMLPh/97b+QjeDwerky+gkfrHhHbV6WsQu91vRFxIQIpPinNfr2XO17Ck+WJ0wNPN3usL5X2vM/7Ukh+noxHqx/hze/kZNStvK3YytsKcw/x/GUVNBTAlmILgoYkXv/+Gl6ryIE3cWDL8EMabpvIPtSDfh4E0/6mSPImJ60sCVsCM3cz6HbRldicWprEJ4n4y+Qv7DLc1dpTYWiE1l7zGBlPBgYGBgYGBgYCh50Pw2asDaZcn0Jp07TUxLgz44j9SlJLEPh3IFQMVIAJ1Pa+G/ui99regkMStjQb8uryKEwoRFlmGYx7Gbcp+TsAKM8ux/mR59FlZhf0+o7xJGku6mbqqCmtQdS1KOg5iC4hJY7Ea2tjN8UOb35/A4thoge+2ipSMlLot6UfNC3IB+ydvuoEVWNVwSGoqJSklaCmpKY5UySiYa4BDfOW8cEae2YsAv8OxAvPFzBzNxPLe7QlqMitQOSlSJj2M4XFYMl9B4uSirDXfC8AQNdeV3Dw1FRsJ9iitrwWteW1Ir+2iqEKcsJyoGSSiXy4UIJ94f+EQ0FLgVaOOvRsKDi1HKJcV2VBJQC0iMfm3OdzJT4mwK9oH75neIuM3VwCDweiMKGwxaS3Oo3qJHaSEB3aNtpwWeIC8Mjt3Zd2R9SVKBg4G9CO4TjHEf239Bf5tT1ZngCAFQkrWmytYuCj56AHJV0lyr5TTlUOTnOdUBhfiH1W+yCjJCPwiKourkZxcjHUzdQpyQT3l95H3IM4rEpe1azK9JrSGtxdchcev3pAzVhNpL4sFguZAZm066qskiyMehjB/6A/KvMrxZ6jYLz/rxgXp0KbgeHfIsM/AwAw/b7kE70AoDSjFGcGnkFpeik2VWwSags4HID7394Hj8uDgqYChv01TCKvadTN6LP3RJOvTgbv0w3S/6PXRQ9zns2RyFz+LQRJFhJOmGL4smCCfQwMDAwMDAwMnxBzJwbgAR0Hkavzjvc+DsNuhsSDRUUtRfTZ0AcmbibEvtUl1eDUcAQyJ1VFVShOLsbrX18j8lIkFLUVYTXSCmNPjRV53lenXkXkpUh8E/ENdO0kl6XI5XBREFuAR6sfQddeV+TD8gujL6AipwIL/BZIbE7/FuHnw5Efkw93T3eJjht1NQosFkvi47ZVDF0M22WQ8nM09vk5zHSAw0wHscbNjcxFn+/7wHmheL5p5TnlkFORg4yiDKWNU8dB2us0KOsrQ9tGct6eAL8i8ebsmwD4Qe3mkvgkEcHHgzH2zFiKP6ooVBdXo9uybrCb0nKVyXNfzhXZmzE/Oh/Bx4JhO9FW5M9CVlkW7j+548htcrBlgOcAVJdUE9uygrMEnxMp2Oe23g32U+xh2E08qeHGKMssQ9g/YTDrbyZydXNNWQ0KYgqg3lEdilqKxOdcmXQFKT4pWJu9VhLTlQiqxlRZs/bAqEOjGm1vLHD7beS3zX59LoepAmppipKKsL/TfvTZ0AceOz0Ej5eklqAktQQySjJYGr1UKHEj6VkSLk+4jPHnx6PLtC54uOYhEh8l4pvwb2A72RZ6TnpEfyxRCDoWhIgLEaivqseUG9Sku8YozymHto027Xqf9jYNynrK2FS+idguKj1X9kTPleJ7VDK0TQIOB0BeTZ7oBdcecV3sCrvJdrRJWKFnQ+G3xw8ev3rAfJDo1X13F99FQWwB+m3pR2lTN1WH9Whr9FnfR+JJRDweDzwuj+i5VxhfiJK0Ehj3IqsYFMQVQFZZlp+g204w7Wv6Rd5PMUiWtpU2zsDAwMDAwMDQBsgMyAQAdHQnB/tyw3Pxbu87xN6NpbQpaCog8XEikl8kE/teHHMRu413o6qoCgCQ8CgBR5yPQF5DHu7b3VGZX4nQ06FizTvyUiQAgMelScUXE1UjVSwKXASALwPzKZxaDp5ueoqS1BJi/9K0UuRGSF4+r6XhcXl4vvU5Xm57ifIcyUkSpvikgFvHRd/NfcXq78nyFFQ+MLQenDoO/Pb5IfFJIu1zbs65iRfbXog8duSlSHit9EJ9db1Yc9tntQ+3598mtj1c/RCn3U/jzqI7Yo3dGA3+T4bdDMX2ffmYOwvvIOJiBPKjyb5gH1OeU04bHEh9lQr//f6oqxDNU+9zaHTUwFbeVnSZ3gW/af5GK/1Mh1EPI0y+Npk2OaQx8t/nozi5GPWyZNlm3S66yAnLIXquNVSMNshKf4q6mTrMBpgJZCclif8hfzz5/gntNbLRvgf9cbTbUSQ+pv7mStJKcGncJWSHZMOom/gSyS2B+WBz1FXWIScsp0XG/03rN3iyPBF+QTTZxDuL7+D8qPPEqofYe7E44nKE9nPKCspC0PEgwV6GRPCJYMR7xYs0J+CDnJyWVfuSq26PNEhtflqVdnfxXZzsexJHnI8AgJB/oo6tDtx/chcoE3DrueDWc8Hj8eAwwwHunu4oTilGaXqp2PPquaontnC2iBzoA4Da8lokP0+mlRO8t+Qebn0tupcuw3+LF/97QfRmb6/IqcoBPND+Lm7OvomswCxkBWWJNb7tJFv029KPmARnOcwSU25MQYeeHSCnIifW+CSKEouwjb0NP0n/RGz3P+SPMwPPIOUlWa53f6f92GW4C6FnxLvvZmBoqzDBPoZ2AZvNRpcuXcBmM19ZBgaGLx9mzWt93Na74fuC76FlTT5omuczDwCQHZpNbWQBVQVVtIfKUrL86pSGGw9de124b3dHz5U90e+Hfhh/fjwG/SKe1NfcF3Mx/f50iveKJFA3U8fS6KVEGc/w8+F4tfMVbs69SeyrpKfULDmn1qK+uh6F8YUwH2xOW0UiDpwaDrj1XEjL/XdENnx2+sCT5YmnPzyltLXXNY9Tw4HXCi+cHXyW2O7zsw9Cz4Qi9VWqyGP3Wt0LzgudkRMqenCAx+OhtryW1iurQXbPZbF48qKNcdj5MABILBN+ZdJKbOVt/eyaVpFXgT/1/6T1D+v1XS9MuDBB4okQDTSs63VVogUTfXf58n0sxZhWUVIRgo8HQ66Cf3D3abymuqga3pu9Ef+QGmwxG2CGrbyttLKXSc+SsF1uOwKPBIo+sc8QfDwYAL8KVFRSffi/JXkNagVlTWkNYu7EwGGWA6bdmda8SRIIOhaEqKtRYvWtzKtEhn8Gqov5lZaSXvNM+5sCAGLvUBOQGiPoSBDi7sWBx6F+ATm1HGQFZeGFJzlZIfpmNO4suINXv7witoecDsHt+be/qMPyLxElHSXo2OnA+itrocftp9nDdpIthu0dBikZKaGEJ20bbfTb3E+gIDF8z3Asfb8ULNaHfd5p99O4OPai2PNisVli7xs1LTQx59kcGLqSK5P7bemHbt92Q+zdWJRllok9xwbiH8Zjj/keBPwd0OyxvlTa4z6v7w99YT/ty6jqA/hKMueGnhME8D9l7JmxGHFwBLovJScBAUBmYCYK4gqIbU5znD6rVlIYX4jsEMK9s5jkx/ATwUh7AoAvlQ4A54aeI7Y3JDwleydLbE4tTUFcATxZnrg8keqBztB2aO01779zwsDQ7qmtrYW8vGjSOAwMDAztFWbNa11kFGVwftR56DnqYdhuqq+AbhddbCzbKDjg/Zj89/nQstaCmbsZcewRB0Yg7l4cdGz5Mia6drpCkptdpnURe96m/UzF7tsY1SXViLoaBX1HfWhbU6Xm7CbbwaiHEa1fSY+VPdqlDA5bmo2BOwZCt4uuRH0UzT3MMfPRTJGrgBpoj/ItDd+N1JfkwFd7XPOkFfi3UipGZPmf3IhcyCjKYNajWSKPrW6mjqCjQairrIPZADOR+y98txCyKuSqrK7zuqLrPLKPW3PRc9BDqk8qws6FwczdDAZd6f28PiUzMBNZQVmwnWALBU1ytRod3Dou2NJsKBuS5anKMstwbdo1ePzmgT7r+og0dmOU55TjRO8TgorGmpIaoQqYz2E53BJsWTa49aKvBZ1GdYKBswFyCjNQ2MGBEuy7s5BfuUknL8ep5YDH5Qn8Xz7m9W+vwanlgFNHDhg3h7GnxoLFZonl59hrTS+Y9DUhSnLp2uliS/0WSUyRQm15Le4svAOWFEvs1xjgOUDoGi3JNW/ytckoSiwSuWJi1JFRKE4uBggxlc7jOsOouxGKEoqIfR1mOeDlTy8RdzcOg38dTGm/s/AOZBRlMHD7QEpbfXU9StNLoaClAAUN6m/98oTLKE4pxowHM6Ckw/igtSTdvu2Gbt92ozzuONsRjrMdUVNWg19Uf4HTPCeMOTHms+M9WvcIhXGFcF3iKrhGikNeVB4erX0E54XO6DxOdG8qnx0+SHicgM3VmyltthNsEXk5Ehe+uoCJlyfCblLz5J2jb0SjOKkYr397DdclrmKPU1Nag8r8SigbKENGgSrB3d5pb/u8wCOB4NRwGg1+tSfe7X+HgtgC2vuxpiTgHHU9Cm0bbSx9v5TY/nL7S6S8SMGsx8L73sSniYi5FYPEx4koTS/FxrKNTZ535JVIlKSUoPfa3pQ2417GWPBuAa1svHFvY4w7Ow6FCeRqxmF/DUP/H/s3a636t8kN5yvlvL/2vpVnwvA5WnPNaz/faIb/NFwuFzExMejSpQukpMT362BgYGBoDzBrXutTnl2OZO9kKGqTq7lCz4RCzVgN5h5UT4Pq4mqkvU6D7SRbYt9Pg3sN+O31g/8Bf3Rb1g1m/c0E8kiikBuRixSfFFiPtoaqkeQ8gipyKnBnwR04zXPCkD+HUA7nZBRloNOZ3oOhPKu8RQ6NWxopWSm4fuOKitwK1FXVSfTw5dq0a6gqqMLKpJUSG7Mt03VeV5j2MyUGQtrrmseWYjcaeJ1wfoLYY7OkWJh0dRI0OmqI3pfFQsSlCKiZqBGD8y3JvJfzEHwyGM82PUN9lWgSpLF3Y/Hify9g3NtYEOx7u/stQk+HYvw/4xv1IVUxVMGPdT/StksrSKPbUv7aKklKUkoEgb4ZXjNoEx7o4NZzEXQkCJbDLEU+0JaSkUJ1cTWk6si+fDr2OlDQUiB6Z5XnlONP/T8BkJMHuszoAjkVuWYfgpMw9zBHXVUd6mvqRa5u7ujekVZeu4Hw8+HICc8R8h9rLg1SleIE3gH+oXFFToUg8CrpNY/FYkHTQlPkfi4LG6/unXJzCm01rJaVFpaELaH9DKfemgolHSXoO+lT2rKCs3Ci9wkM2TWEqBbw/jr/ELMktYQJ9rUwVUVVuDX3FixHWMJ1MTVQJS0vjd7f9xaqkot7EAefHT4Y/NtgGPc2RpJ3EvKj8+Gy0AVF8UXICcvB1JtTmzWv5BfJiH8Qjwy/DJHXxqqiKlTkVYBTw0HSsyR0HEhdM4y6G2HMqTESkfx12+gGw26G6NCjQ7PGCTsXhvtL72P2s9mfXedEpTK/EnKqcsQkxX+D9rbP4/F46LO+T7O8gtsaxr2N0Xtdb/T8jpwAVF9Tj6LEIihqKdLuZbQ6aUHNVI3Y5vWdF/z+8oOcqhx4PJ5QpW9mQCbe7XuHXmt6QdlAtESfq5OvAgAx2CevLv/Z33Bj3tksFov2Xr+tYjbADPPfzoempejXfIZ/j9Ze85hgHwMDAwMDAwPDJ/jt9QMADNxBzUgHgNtf34aKkQoW+C6AagfhoJrZADMM2zsMaibkmyHvrd4ojC3EhAv8QEDU1Si83fUWmpaaqMitgNcKLwDAFu4WoRulpnCoyyEAfIklSQb7VIxUMPbMWNycfRPS8tIYeXCkUHt5TjkiL0XCuI8xDF2oskl+e/1QV1FHPEhqLqmvUsGt54p9CPs5go4F4cn3T7DAbwGMukvGB6o4uRgyCjIwHS9eJebr316Dy+Gi70bxPP9aAxab9cX5L/F4PL78Igu0v9XU16mQlpcm/i4a49a8Wwj/Jxw/VP8g+ry4PPju9oVJHxP0WN6D0h5xKQLXpl5D1wVdMfroaLHGf/XrKzjOcqSsf4D4lYMNCQP57/MFgb1Hqx8BANLepDUa7OPWcxF7LxaqHVSJf+vAw4HwP+CPfpv7iTyvxtDurI15r+Yh4kIEIi5EwHKopcj9e63tJdahTWVBJSZdmYRvNpEDwsN2D0NNaQ3qKusgoyicqJAVyPfkkVUmV386znIUS2azKeRH5+Og3UH0+7Ef3Lc1Lvn1KWVZZbg84TLsJtuh5yrhA8ua0hqk+6XDd7cvMgMyMejnQSJfQ+lQ0lHC7Gezid/3pjB091CkvExBcUqxRLwsP8XrOy8oaivCZaGLSAHn6uJqgAViEkZ+TD5SX6U2+p1uTF7XargVbZussizsptjR9p/9bDZKUkq+mGtGbkQu6irrJLaHkCSpPqmIuR0DeXV5oT3a3SV3UVtWi/H/jKdUbtaW1aIooQh1lXzZ4tDToQg9HQrHWY5ieeyRsB5tjcx3mbAea/35J39C/vt8ZPhlAOBL5n8a7Ds37BzYUmxMvzddInNVN1WH83znZo+j31Ufvb/vTXv/0Bz+0P8DbGk2sdKRgQqnloMbM2+gy/QusJss+aSX1sB8kDnMB1GTVBvYZbgLVYVV6Lu5Lwb+RL7/XRazjLZ//vt8aHXSIj6n+7LucJztCEUtRZEDzuvy19EmnXDrufDe6g0ehwePX6gJPi+3v0T4P+GY+3IuJXGEx+Xh3NBzUNJVwsAdA2mrA9saCpoK6NCzeYkFDF8+7UcwmYGBgaGJhJ8Px5+GfyLJO6m1p8LAwNBOMR9sjgHbBtBKyQ3bMwxlGWW0/hy359+G/wF/YtvbP98i4mIE4r34Pko1pTUoTi5Gt6XdsDpzNWSU+IeyJA+dz2E92hoqhirETPrmIKski87jO6P7iu7EoFrYuTB4rfTCpbGXiP1VO6iiML6Q1kOsOZzsexKn3U9LfFyAnx3+5PsnkFaQFktyjo7MgEykvkoV++DvyfoneLbpWYv5j7UEiU8TcW36NeSEi+5B11apLqrGNqlt2Gu+l9ieGZCJk24n8WD5A5HH7tCrA+yn2oslH1ueUw4eh0d7YJj2Og0AUFcumr9cAzlhOXi26Rn8D1LXuLBzYYi+GS3WuIraipBTlUNtea3gsTneczBo5yCKn9SnVJdU49LYS7R/a4uhFnDb6CZYXyWFnIocTPqYoCCmAFFXo/gBYBGIvhGNVJ9UaNuIXoEZezcWR1yOQLmQLI1bnlOOXzV+xZMNTyhteo56GHd2HBYHLyb2LU4uxsPVD1tkL31qwCkAEOs69XbXW6S/TedLT35CYXwhzg05B6OeRlget7yZsxRGWl4aZv3NxKq0Bfjyrs82PUPSM8n/PXlcHvz+8oP3Zm/cXnBbpL6/avyKX9V/RVVhFaUt1ScVdxbcQew9sg/gs83P4MnyRGZAJu34BzofwBEXqj9Upn8mIi9F0u4JOrp3hNNcJ8ipiiZL2lY51OUQjvU41trTIJLulw4AcNvkJvR4YVwh8t7nEfvYTbbDmqw1AnWLPt/3weyns4UkgW/OvYn7y+6LPS9VI1WMOTkGNmNsRO6rYaGBbkv50qSkwIKMgoxEZfvqq+tRmV+JmrKaZo2jbqoOBU0FVBVQf4/NhcfhgVPT/hQ2Wg0e4DjHESb9TFp7JhLl6Q9PcXog+Z6pIdjVWPLk9ZnX8epXsk/rTK+ZtMFAWSVZqBioiFVZqqilSFvhHXMnBq9+foXXv74md2bxE4zODDpDuWficXnIeJeB8PPhePjdQ5Hn1Vo0JLelvUlr7akwtGGYyj6GdkN7KPdnaBtk+GegPKsc8V7xEpfAYGD4t2DWvNalo3tHVOZXIiswCxZDLCjtDjMdUFdVR/QNqsitgOsSV1gMpfYDgCk3puDOgjuoq+IftHf9uiu6fv2hCmZT+Sax5z31lviySTwuDyw2fRWErJIshu8ZTmwz7WsKJT0l9PuRXDWj31UfZZll4HK4kEL7+W5z67hQMVSB0zwniWZaG/cxxvR708UOyjrMdACXw+XL5JDMltogqT6piLgQgeyQbCyNonp9tMc1jy3DD8SRAg8A8GLbCwBk6aHP0X1pd7z6+RWuTLwicqWEtJw0nOY6oaMHeQ80cPtAuG10E1seT05VDiZuJtCyplbdPFn/BPU19airqkPHgR2hrNf0IHnHgR2xoWSD0GNmA8yaVLVbXcSXsmRLkYOjOrY6eLbpGVSMVCTqv8PlcJH/Ph+VBZWoq6hDSWqJSJVbVYVVKE0vFeu1DboawLCbIVKqigGA4tkXdjYMAD9w/CmqRqqNSlv57fOD725fyCrLSnwvbe5hDnl1eXQeL7oPl5SMFFSNVdF/a39Km5qJGkb+PRJG3Y1aRN5qr8VeKGgpYFHAIpH7qpupY/j+4UKefRJb81jAd2nfwXuLN4z7UPckjcGWYdN6xzbsYe4tuUesyte04v+Nj3Y7ih/rfhRKTODxePhZ8WfUV9ejxypqdbGuvS56f9+b9nMqTS+FlJzUFyPhOebUGEEVXFvDeYEzLIZYQM1YeI8z++lswb8vjb8EbRttDPp5EHEMHVsdgQ91kncS6qvqkRWYBXkN8b2KuBwuOLUcSMlK0a7rdCjrKQv22KSqrCk3piD2Xiz+Mv0LIw6MQKdRncSeJ8D3pww7FwYVIxWsTl8t9jil6aV4uuEpPH7zkHgV6Lhz41CSUiLRMUWlPe3zpOWlUV9Vj+DjwS2iStIahJ4JxaufyYE6AFgU+PlrW/g/4ZBRlIHbejdie2ZgJkrTSmE92lronrIirwJVBVUI+DsAWYFZmOczr8nz9t3ji9K0Ugz5YwilTc1EDTZjbeC8iFxZ2++HfihOLkb0jWjKfS5bmo0NJRvwcsdLsWSwW4uoq1G4Nu0agPbp4f5fojXXPCbYx9AukJKSQpcuXVp7GgztBNsJtkh5ngLj3qLd8DIwtBWYNa9tcH/pfeh01iEG+xQ0FRq90Qk4FCDkb/IxFoMtsCplFeXxnLAcpL1Jg804G5EOyCXFsR7HkBmQiTEnx8BprpNQW25kLo64HEG/H/uh3w/UgJ5RdyOszV5LO7bbeje4bXCTqOddA5sqNzUapGwOSrpKWJ0h/sENHSoGKni5/SX8D/iLJSM17uw4ic+ppemxsgeyQ7KJh2rtdc2TU5HD9HvTaf0oe6zogc4TOosV1AAAox5G0O4sesWXgqYC3Da60VYFyqnKiV0xw6nlIDMgE4N/H0yUEZp4aSKCjgXh+vTrmOM9R6S1zOs7L/gf8Meq5FUCnzkuhx+I4HF4jWaEK+kqYdThUdBzpJcV5NRyxKqYbozk58k463EWAD9Y+alc5udwmOmAosQipPqkEn2lGkPPQQ9lmWXQqghDnll3SrAv6koU1EzV0GUa+bf1+vfXKE0rxfC91CSOmlJ+hYrjHMlLeTbHy3LQz4NoAw6K2opwXeyKqiJ+AFXFUEVi14b8mHwUJxfTBvY/x9WpV1FfXS8INEtyzWOxWFDtoIoxJ8aI3PfHWnqfSzVjfvC0PLuc2O40xwngAVlBWcSK1o6DOsLEzQRuG6h7JQVNBWQHZyMzMBNanahJA7uNdwMA5r2aB5M+7b+yxmmOU2tPgRY1YzUUxhWiMKGQVlY1wy9DqCqmILYAmQGZMPcwp8jGPt34FKXppc0KegFAyMkQ3Fl4h1YW8HPUlvIrxGVVyFLF0vJ8xYaPqxHFxaSvCcLOhcFuSvPkHgtiC6CorSiy7HdTcJhBn9zxb9Ae93myKrJEieP2SlFSEWQUZbA4hFzR3xS0OmmBJUW+riY+TcSNmTdQnl2OH6p/EPJz9dvrB5/tPujQswPtNYWOh6v4VXeDfx9MkeY2dDH8bELc6KOjG5WsJ93XtmXUzdTBkmLRJtgytA1ae81jgn0M7QIej4eysjKoqKhIzHuB4cvFxM2EVpaIgaE9wKx5rY//QX9U5lWi+0FyBch2ue3oPKEz8dBSz0EPY0+PpU04yHiXgdryWsHBbnZoNnIjcpEfnQ+f7T6498096NjqYHHwYpHlTo52P4pM/0xMuzNN5ExlFSMVIADEw1FZJVl0HNgR3pu9ISUrhT7r+og2r25HwZJi4duIb0Xq1xRaIoD4MQmPEvD6t9cYuH2gSB4JdZV14NRxaA8KKrIrUJLaulnW/yYKGgqYepNcedqe1zyrEfS+VA0SZ+Lgf9AfSnpKGLSDHNj4HKfdT0PdTB1fv/6a0laeUw6fn31gMdhC5HUiJzwHV6dcBQBMvDwRdpOEDzdN3EygYqgC6zHW0LHTEWlsFUMVcOu4KM0oFQT7ftf+HdXF1ej5XU8M3TWUtq+cqhxcFrnQtsfeiYWaiRqc5jmJNKfPoaynDBUjvjRV/639Ra5EqiqsQvLz5Ear7Bpj9PHR+NHz/6WfP4m3TLo6Cdx6ctVWwqMEPPmeL+9JCvYN2z0Mg3YMgqK2oljzaoyqwio8XP0QFkMs0GW66Ach4efDoaijCIvB5Or5h989ROjpUGws3whZJfJBv6jIKsvCdqItOk8QL3BvN9lOKClAkmsel8NFUWIR5NXlJV4J97mKFqe5TpTkIIAfgJx+lz6Rpa6yDhl+GcRkKgDQ7aKL3PBcKGiQpdTbG/kx+eDWcaFrT+872lrUVdXh3NBzcJzjiLGnxgoej/eKB1uGDfNB5pSkp/iH8fBa4YW5L+ZCSVcJD1Y8wLt977A2Zy3cf3JHXUXzqxjVO6oDgFhVuhnvMhB1NQoAv+r706ST4JPBkJaTxgK/Bc2eJwC4LHJp9PrTVKpLqlFXRfVYlQS35t+CVict2kTFlqa97fMqcivArefCZXHzP9e2woCtAzBg6wDa9pfbX8L7R28M2zMMPVZQK7KB/w+40STReK30Qnl2OcadG0dJNjPrbwbw+Il/ol6n3Da6ISsoS+CR/SmFCXybiAbf549J8k5CXmQenBc4UwL7XA5XsDc0cDYQaU6tSYeeHbClfktrT4PhM7T2msd49jG0C7hcLhITE8Hlkm9YGRg+JjMgEy93vERxSnFrT4WBQSyYNa/1qSyohJKeEq2Mjp6DHiIuRMD3L19Km6qRKh6tfUSr/3995nWcGXQG+dH5APhyHDdm3oBJHxOMPT0WAJAXlYfailpi/8bI9Od75yjpiX7gN/XmVGzlbYXjbGolh7qZOqbfmw6WFAvZwdmU9oiLEfBkeeLc0HPEseXV5VGcVCyQVZIke8z3YKfKTomPCwA1ZTXw2+uHpKdJqC6uFqnvXou9+FX9V2JbxKUIxN2Pw8AdA0WeE5fDhSfLE54sz3YVLKwurkZxcjHRo+lza15ZVhn89vkhKzirpacpEjWlNbi75C4iLkXQPuef4f/g4tiLIo+d8CgBwceDxZpXeU45yjLLaKv3Xv70Eu/2vqP3OGmEjw9NM95lEJ+jYa6BzuM6i3yg0yDXmeydLHjMdpItAP6a+zn2mO/BzTk3iW1FSUXIeJch8co+XXtdrE5fDSUdJVyfeV3k/jJKMvjqyFdiBb2ib0bjxf9egCtFzt9V0lHC8y3PEXgkkNImLS8NPQc9TLoyidiXLcNGTVmNyOseAAQcDoAnyxO3vr5FbH+x7QVCT4cKvMJEIfVVKq7PuE787maHZmNfp30oyyhDz+96iiz91xiqRqqYdGUS7Kfai9XfbrIdgo8F4/Xv/HlLcp9XW1aL/Z324w/dP/B001OR+h7qcgjb5bajqojqERZ4JBDb5bcj+XkysW/M7Rg8XP2Q2LcBv71+eLb5GeXxktQSdOjVAeaDyQkR34R9g628rQJpyPbOAZsDONTlUGtPg8jzrc8BAKb9TYUev7/sPh6teUTsYzXCCpOvTRZ8PnoOeug8oTPYMmxYDLaAzVgbJD5NRPKLZLHnZT7IHFt5W8VSP/i48ujNH28o7S9/eom3f74Ve24tRbdvumFt9lqJ+24DQMiJEDzd8FRkX1lJ0d7ubauLqxF6OrTN7TubS9rbNERciiB+D7x/9AYAlGaQpcV5XB7kVOUEEs6f4vGrB6bemgqHGQ6U66+5hzkGbh8oVkLKoJ8HYabXTGKQMd03Hfss9+Gg7UFi34gLEXiw/AFCz4ZSkp/qq+txadwlHHE5godr2o9nH0P7oLXXPCbYx8DA8MURczsG3pu9EXk5srWnwsDA0E7p/2N/rM1eS+vTNvXWVKgaq6KmrIbYbjbADPpdyTfrDZU0Sd5JAIAu07pgyo0pMHEzgeNsRywKXITJ1yaLVZGwMnklViathFE3yXp9APxM/S31W4jVjNUl/ANhumCenpMe6irriNne1SXViLoahbz3eWLNqzipGLXltS1ygFGeXY64e3Hos6EPLIdZitTXpK8JNCw0iG3KesroOKgjFHXEqJr56G02BIzbA293v8Wejntwc+5NkfsWJxXDa4WXUBCoLVBbUYvAw4G4NvUasf3W17cQ7xUvlpTg+HPj0Wt1L/juoSYUfI4GCUa6NchiqAWMextjyJ9U/5PPodpBFWuy1gDgH5R8yh96f+D0wNMijwvwg4Rjz4yF1cgP1ZJfHfkKW3lbidVDH1OYUIjipGKEnw8ntg/cPhAev3ogL0q8deZzqHdUR0VOBSoLKkXq927vO9yYfUOs16wpq0FJagnYHOrnAPDXr/Dz4cSgmmk/UywJXQLbibbEvsnPk7HPch/l+3dj9g3cX34f4efDEXQsiNi3IaCaF0n+W0dciIC+kz6G/C769y/wMD9w2X8L1bOPxWJBRkEGDrMcMHTXUInI833Mi59ewHurt1h9pRWkoWaiJrYkHI/Ho5ULlpKVQp8N/Gr7rEDRDqZzI3LBqeWgMo/6vVXtoApODQe3598m9k1+ngzf3b44N+QcynOEZdm49Vw8XPMQXiu9iMHmsswypL5KRU0JeQ/F8O9h4GIAu8l26DRSuMp72J5h8PjFAwB/v9qwZwUATQtNdB7fWVD567zAGZOvThaqxLy76C5t0ltLY+hiiPVF62E3xQ4mblQZ2ElXJqH3973x6tdXYu89P8Zvrx88WZ64s/hOs8bJj8nHTpWd8PnZp9lz+hTH2Y6QV5cXkmNloEfTUhPjzo6DnIp4kudtkbz3eTg7+CyuTb1G/B4seLcA30R8g8G/Dib2r6+px2n30/DZQf1+VhZUQl5NHtajrRudQ8rLFISeDRVp3tx6rkDS/VOKkooAgPg7B/ie2RrmGri76C7FN1VaThrjzvFtEUrTxPNObg3S3qTBk+WJv8z+au2pMLRhGBlPBgaGLw6TvvyLvTjZyAwMDAwNHLQ/CEUtRcx9MZfSpmKogu9SvyP2S3iUgPS36bQVAH039UXHgR2h24Uv56RjqyOUvW7gbCC2nIi6qbpY/QB+MObVz68w+9lsim9LSVoJfP/yRaeRnYi+Uq6LXRuV++qzrg8cZztCTo1601ycXIwrk67A9RtXjDw4UuR5T709FYVxhbTyLs1BxUAFU25OEUtGatJlcsUM8KGCqSS1BMa9qHKvD9c8hP8Bf6xIWAFVI1WhNrY0G4sCF+GIyxFkvKOXQaODW88Fp5bTIjJRjdEga1uSIno1oqalJqTlpdtcxb6ynjIcZzuiIK6A2C6rIgvjPsaYcr1xPxFiX2VZRF2NgqyyLHqu7ClSX00LTWws3wgpGbIMsPVX1rD+qvEDmcZQ1FbEkrAlxAxty+GWSHyciF81fsXU21Nh2teUMAKZuHtxyArKEqt6isViQc9Rj1ZKrbq4Gve+uYee3/WUqKdzZmAmjroeFfy/tqwWilpND+I7L3SGoo4iilOKRV6/HWc5wm+PHzq8f4Jot/kUGc9L4y5BzVQNXx35iti/ML4QNWU1MOhKvd682/cOAChtYWfDAAD++/3581/gTOnrusQVXed3pf3+ff3ma7Cl2SLLVANArzW9YDvZFsZ9qJ+hnoMeloQuEXnMBkJOhSDdNx2j/h5FaStJK8HzLc8BAO6e7iKP/XbXWxi7GYst9Zcdko2rU67yJcI/uWbIKMrAY6cHPHZ6iDzuyuSVKM8uJyY2WY2wQufxnZHyMoXYt9+P/VCZV4mwc2GoKakRqvrlcrjw3eULiyEWRI9Z+2n2UNJTok14+dPgT5Rnl2Np9FJoW4vuW9rW2Mrb2tpToMVhhgPRz+3j4N+9b+5BSlYK34R989nxTg88DU4tB4P/GCzWb7yB1FepONn3JAbtHET0ffwc8urymHhxIrHN0MUQxUnFeLrhKdTN1Inyf6KQ9IwfCI28GImvDpPX26ZQVVAFRW1FgYSpJGlQDmFoGiw2C5GXIpHwOEGs719bJPhEMOoq6jD4D6r3HYDPJoo2SHOGnQ3DuDPC6/r9b+8j8nIk7KfZI/l5MpbHLoes8oek1XcH3iH6ejS4HC5SXqTAYaZDk6UNf5L5CQCwoXQDJfhqN8kONmNsaNcarU5amHBhAoqSiigJQGxpNu3619o8WPEA8urycN9G3W803HM0JPZJirODzyL1dSp+qPxBouMytA5MsI+h3SAv/+WY4zK0LMa9jfH1m69pK3IYGNoDzJrXumQFZaEgpgAd5lA92nhcHh6tfQQDZwOiz5K0vDRUjFRogymK2opEn6yQ0yG4NfcWjPsYY9DOQSIdkjeQ8Jgv/dd7bW8YuhqK1Pfd3neozK9E3P04SrCvIqcCvrt8kf8+H3KqcqKPvf8dZJRkiIGthoPxzIBMkcZsoDlBi88hqywLQ1dDZLzLgLyaPFQ7qH6+0/+TE5YDTi2H9m/1+rfXSH6eDPsp1MBGdWE1ODUcVORWUIJ9AD/4Nfr46CZJG37KIQe+lNjSqKUi920OlkMtGz3wbGzNk1OVA1uaTawka01YbFajB2jD91C90JpKYUIhhu8dDj1H0T9jFpuFu4vvQkpOCmOOjyE+h8fjATyyR2djlKSV4C+TvwAAq1JXUdrHnhqLmNsx8NvrJ3JGfMKjBIT/E86vJvn/5fPpD0/x6udXGHFwBLp90422r4a5BpaE0Ad6StNK0W1ZNzgvpAanmkNRYpHg34tDFkPVuOlrBMCXyww8HAh9J324LmncH42EnoMewvLIYj320+3BqeUQ5Szzo/NxoPMBAORAhPNCZxi6GsJyuHBFs2E3Q1TkVOCrY1+BU0OuNGOxWZCWoz9mUDdTR1ZQFngcHjTMydXPdOg76X9W4i7gcADi7sVh3NlxIlXSPVrzCFWFVcRgX0VuBQAIKuhEJe5eHP+g8fcPj4myz+PWcVFbXou8yDzidVRc1E3VGw0yT742mbZNQUMBY0+PJQbzpGSlsDJ5JWQUZYhJAWUZZbg09hL6/diPeJhZns2vFCxJKfkign1tnX2d9sF8sDlGHiAnXHn86iG0jrzd/RYvt73EnOdzoO+oj5jbMUh4lIAB/xsANRM1cGo56DxOPH/LBvJj+MoFPj/7iBxsKU4pRtSVKJgPNoe+I3W94NRxYNrfFAv9F4q8BpGYeGkiastrhYIb4pD6OhWV+ZWUPbgkSH2dCkVtxVb9PbWne9ua0hrYTbGD6zeiX5fbKvZT7KFrrwvbCbbEvV9tRS1i78ZCo6MG0cZCSkYKXRd0JVaCd+jVAZGXIxFxIQIdenWgJF+WZZQhKzgLY0+PRa/VvcSaP8kGgC3NpvgDfgyPx4NhN0NaW462SvTNaKgYqBCvj12md0Hn8Z3FUi8oyyzDLqNdkNeQx/rC9UJtiU8SxZ4vA5nWXPOYYB9Du0BKSgo2NjaC/2cFZUHHVkfi8iwMXwZsKTa0bbS/KNkFhv8Wn655DP8+oWf42v4D/jeA0sblcOG7my9tpmmpiQ49hQOCpv1MYTXCCgVxBUTpx8sTLqM8pxxfv/oaAOC9xRt+e/wwbM8waHXSQtrrNJzqdwrfpX0nUnAJAM4N4Xvmdfqqk8gBuVmPZyHeK544Zz0HPaxKXYU9ZnvAqeVg9pPZQu0FsQW4Pf82nBc5w3EW1fMv6moU5FTl4L7NnZLJqWygjN7f9yZWlzSFoGNBkJKVInoNSoL0t+m4MukKJlyYIFLF0WHnw+BxeNhUuQkyCsKB3wz/DFQXV2OA5wBiXwMXA4ScCiFKnHHqOAg4HABde12RP2MAsBlrQyuH01zCL4RDTlWOIgf2OT635knJSWFDyQax5DCbyx7zPShOKsa6vHUCybIGOLUc5MfkQ1FbESoGKsT+EZciwKnhiPz9vLfkHjL8M7CheIPIc66rqkP4P+G0Falvd73FozWPYNrfFHOfzxVp7I8PeQpiC6BmTE2ssh5t/VkpJxK91vRC+PlwvN31Fn039QUAhJ7myz0lPkpsNNgHAH77/CCvJk/8WwceDkTomVBaeSpxsRlrg/XF63Fl0hX4bPeh9cCjQ8dWB2NOjqFcR5pCfkw+VJxV8OyFNwqffw8utxTd3qpizPTpWLBwIfqs64O0t2nIj86Hto3wAW9uRC4AfvCO+L7G2MBmDPU32X9rf9RX18NiMH1FcXZINl5sewGXRS7E60lRYhGOdT8Gp3lOGHOCHIxujKPdj4Jbx8Xi4MVCj5dmlCLqahSib0Qj/W06bTCSjhkPZqAirwI8Ho9yndJz0MPanLViV0QvCVmCN3+8QezdWHQa1UnkfZ5RdyOsiF9BlKuuyK3AjVk3oKSrBJclLjDp80HOrKqwCrmRudDqpCVUeddAik8K2FJsGPUwogSFU16mIPpWNLov6w6NjtSASEVeBWpKa6Buqk45bGWxWFA3VUdpRikKYgug1UlLqL2+uh52k+1g5m5GfL8byzaiuqSasua2ZXhcHh6seACneU4wdBH+XXmyPQFe61T4cTlcPF73GBZDLWA5lPp7jLwSicK4QooM3kG7g9Cy1sKU61Moa4GyvjIMnA0EcvNpb9Pgf8AfPVf1xNhTYyUyb8dZjlAxUIGaqejJuwWxBXi87jEAfkXbp9eEPR33QM1YDfPfzpfIXKXlpBtNcGgqnUZ2grK+MtTN1Js/qY/g8Xg46XYSALC5djNt1XVL0t7ubbOCsnBj1g0M3y9+0lZbw9DVsNH7hp3KfP/zLjO6YPy58cTnjD46mvh4z1U9kfw8GYraihh9jPqcQT8PwqCfB4kx68bXzYq8ClwaewlgQXBf/TG35t5C6JlQbCzfSLHHqC6uxgHbA5BTkcOIgyNgPojsIdsazHw4k1bmWkpGSvzf8P9vbaqLqApo83zmiSxFz0BPa695jGcfQ7uAy+WioKAAXC4XxSnFOOJyBO8OvGvtaTG0Ud7tf4ffNH9D2pu01p4KA4NYfLzmMbQOTvOcMPnaZChoKVDaGmQUAX41HonAw4GIuBBBbMt7n4e012nw/YsfMFQ1VoVRdyNYDrPE4pDFcFniAh1bHbBlqNu0+pp6vPnzDe1mvO/mvuixqgdsJ5B9mBpD01IT3Zd1JwYIpGSloGashtEnRqPnd1RJwdh7sUh9lYqnG58Sx+40qhPyIvNQmk71RGBLsTH418FiSfcBwJ2Fd3Bzzs0WkW7OCcvBlUlXYOBswM9UFYGGQ0rSvHIjcpH+Nh26drrEvg4zHbAicQUxAMCp5eDJ908QeUk8X9p4r3jkhueK1fdzXJ9+HRdGXSC2hZ4JxQ7FHYi+GU1p+9yal+ydjG1S2+B/yF+i820KxUnFAPiH5p9SllWGvx3+xi7DXcS+kVcicW3qNTz/33ORX9dxriOcFzoTfzOfo6HazH46+TfVIIcqjpylnoMevo38FgDgf4D6eTxa9wj+B8X7nJT1lNGhRwfIq3/IhP0m/Bssj1uO8f+QD54aqMirgNcKL1p/MZclLhh3blyzZOVISMlIQV5NHhU5FcgKyiJmnjeG10ovPF73mBKMa8Bvrx/e/PmG2LZ9wzYMW+4B/aTf8Wd5NC5UZmJVdDR8fv4Zph06YPeff+Kk20m88HxB6WvuYY5FQYsw8+FM4tjl2eU4N/QcxZePxWZBxUAFF8depPWVSn2Viugb0bg55yax/YjzEQCA7STRr1PXZ1xHpn8mMUhZGF+Ih6sewnaiLTbXbIaSLrWirDGMuhuh08hORGkxKRkpgUeiOHDqOHi+9blg/RNnn/eL2i+4PuM65fG6qjqk+6Yj7FwYvFZ6CbXF3InBqX6nBAlKn3Kq3ymc6HMCWUFUr7+soCz47vIleu4BwAvPF9hnuQ+RVyIpcmI8Lg9FiUW4PP4y/nb6m9I3yTsJkZcjaX+PssqyUDVSlUgA5d8iOzQb/gf88XAVwaeuFW3Sqgqq4Lvbl3jtBfgJTQAoyUcaFhq0CWddpnXB7KezBfvFvhv7Yk32GqEg1Qm3E/hn+D+Nzi35eTLRUxTg7zutRljR7pMaw6i7EWY+4q9tyS+SKe0242xgNtAMteW1tF6YopAfnY/Iy5FIe9u8swcVQxUkPU1CvFd8s+dER31V6ygktLd7W/WO6nD/yR36Tvot4gneWtxfdh+/af1G9Fh3nO0IQ1dDdF/Wndi3Iq8CB+0O4tUvryhtoWdCoW2jTSsb3lLE3YtD2ps0pL0m//YMu/P3CruMdgkq9BtgsVlQN1NHQWwBcZ/Umjz5/glODThFbKvIq8C9pffgt9dP5HGV9ZWxLn8dNpZtpLSZuJkQk7wYxKO117z2s3ti+E/D4/GQlpYGdXV11FXwL0yS1ihm+HKQVuAvbQGHAmDaT3QZPAaG1ubjNY+hddB31EfS0yQE/B2AXt8Jy42wWCzo2uti6u2pxOzb/Oh82IyzoT3InHpzKrxWeQkOUVwWusBl4Qcfn1GHqBJiDfgf9MfjtY+R4ZtBrCAZ+NPAprw9IqUZpagtq4WWtRblsLOuqg5lmWWwGWtDlEWzGmGF4qRi2E22I45t3NsYNaU1lAo3gH9AdnbwWfT7sR96LO9Bac+LysPt+bcx5cYUKOtTKxOsR1sj5nYMuPWS30zLKMrAfLA5nOY5ieynNevxLJSklEBBgxowtptkh44DOxLlzQAg+lY0ckJzMPh3ahWStLw0pt+fjvMjzkO1gyoGbhftM9foqAEFbeqcJMH3hd/TenAUJhSivqoeXiu9YDNW+Gbyc2ueoo4iZBRlUFVADbi1NBtKNoBTy4G8BvV7//FvgVQNFHSEHyiZcGGCyK/rMMMB7/a9Q+TFSHyXRvYHpUNZTxkev3rQBvOG7xkutsQoW5oN7c7acF7kjA49qMHowL8DwZZhoyyzDF2/7iqyRNrY02OhafUh4UBBQ4H4G/qUsowyAPyMdBLa1to46XYSKS9TmuWp9CmV+ZV4f/09csJyAPDXUVIVFB3aNtqNHr42BG96r+kt9PjuP//E+fuH4QMeevCE74lmVFfDD8DIDZvgCndMnz2dMq68unyj1dTPNj9DwqMEGLh8eE52SDbOjzgPk74mSPVJRcytGEEF5sfYTrRF6qtU2IwjHxp1W9oNqh1UYTXcivb16dDuzA+K1lfVo76mXigYpO+kj3k+88SW5bs59yayQ7KJcrB1lXU44nIEnFoO0c8m3iseD5Y/wLA9w2A1gvq+8qLyMObkGJh78CsHRN3nlWaUQkpWiujlpW6qjg0lG/jBMznh4FlDtR5dMkzn8Z2R8DgBCprU31jX+V0RcSECr395TfQDNB9sDv8D/rg+/ToWvFsg5PdUW1GLvRZ7wZZho+8P1O+I5VBLKJ5WhJaVFqUN4CdKgMffWzRXGvHfQquTFmY8mEHcF85/O7/VFIkUNBWwOHgx5FTJijfuP7mj76a+lGvctNvTBP++OuUq8qLy8E042bNPTlVOMP7bXW+hoKUAZX1lSiXNp1yfcR1qpmqY/4ZaYVeZX4mixCJoWmoSv5+NIa8mD3MPc6zLX0f8u4/YNwJR16KwU2UnJlycQJRT/xTvLd4oTi6meJUBfH+txMd8CbwtnC1iqxDUV9cj7GwY5NXliTYB4sJisbDg3QKUpJS02vewvd3bqpuqI/99Prx/9MYP1T+0q8QDOry3esP/gL+QT/zHfM7XkcfhIS8qD29+f0OR1vXe4o2SlBJomGugIq8Cvb7rJVQJnxmYieLkYsTejsX76+/5v80m/k2vTLqC3MhcLApYRKmuN3Q1hMdvHrRJrt2Xdge3nov3195TfpdyqnKY/2Y+Yu/GtrnrjI6dDjHhGABibsUg4GAAAKDHCup9c2OwWCxaX+mixCLUVdWJlWDBQKW117z2v2Ix/OfQsNDAt5Hf0hp6tyTvr79HSWoJeq6iVjU0h6rCKvym9RuU9ZWxJmuNRMf+L2I70Rbpb9PFkpBiYGBgaCDoWBBYLBYl2Mfj8lCSVoIOPToQKwdywnPgf8CfNtlAq5MWZtyfQewXcCgANuNsYNrPlHgT5DTXCUbdjaBpQZbnq62oRUlqCZT1lZt0QP4x50eeR05oDty3u6PfD/2E2rICs3Cy70kM/Wsoeq6kXgO1rbUxfC998EC7szb0u+oTJbmk5aRRmVcJ392+xGBf4tNEpPumI8k7CV2mUQ/yp96a2pS3JxaalpqY9WiWWH31uujR+q7IKsvi/rL7iL4RjfXF6ylBovdX3yP2biwG/G8A5YCOLcWGSR8TGDgbiCUpZ9rfVCDhJ2ka+865e7pD01KT9iazMfS66IFTy0Gmv3i+js2B7oAU4AdMViatRHUJ+SB9+L7hqC2vFUtuFeD7cohTsaqorQj9rvqoKZN8YlxdZR3uL7+PmuIaOM11orQvj1uOoGNB8P7RGx0HdRQp8OK1ygtRV6KwhbtF8FhuZC5K00qh3lG9Ua8hTStNzHs1j9avWUpOCtZjrMWWC6bj/Y33uLv4LgB+pdrHVYlNwWacDULPhCLwSCBcFrlQ2kkyc9nZ2diwfj1ecjigO+bpAeAetw598QQqXakSs5xaDh6seICsoCwsfLeQ0s6SYkFRWxHuP33wi2mo2Ez1SUWPVT1QlFBE6QfwM8cnXpxIMzPwPRnFpN/mfkh6loSwc2EYcWCE0HVSXk0eJm4mKIgtQPzDeJj2MyUmmNDRIBlbnl0uSCypLKjE79ofjPZIwTYAAItfWXhn0R2sTl9Nab427RqUdJWIv5mmkBWUhbrKuka/v6RkG8thlph8bTKt92djnnxyKnLw+NUDJaklxGQGmzE2WBS0CBl+GZRkGCkZKfT+nu8dbDeJOi9ZFVk83fQUxSnF6P9jf0r71clXAfCv7+3lfk5WSZYoWwtALJleScGWZuPi2ItQ1FbEooBFlHZZJVmEng6Fsr4yOo8n++wp6ihCxfDDOpLwOAEpL1LQc1VPKGorojK/EhV5FdDoqIFXv7yCto025r2c99m5dZ7YmfaQPfpWNO4suAMZJRlsKt/UxHfLh8vh8pN01OWJnqUAP5jjNNepSZKZZVllePnTSwAgBvsavGAth1nyfyufGpY1kcgrkVDUVoTzAsl6ywKAUTcjZAdn42/HvzH52mTagA/DB0z6moAtzaZNYmtvNPiQz/CaIdK1sQFlfWVYDLUgVtF1nd8Vz7c8F+yHXBe7Ct2j+B/0R8iJELhtcoNxb2OR/qZRV6MA8K/HaorCezxde13o2jcenOq5sifx3rWBTqNEsx74Nwg5FUKbDGPYzRD2U+1hN5WcYNsY5dnluL3gNjqP74yuX3cVattrsRdA68hNM0geJtjH0O4oiC3A0w1P4fqNK3FhDj0bivz3+WJrQjfG5QmXAUDiwb4GaYAGM3KG5qGsp0yrM87AwMDQFB5//xj57/OxMIB6CFpfXY99lvvgMMuBeNNv7mGOKTem0B5yh54NBbeOK9hkx96LRdrrNOjY6SDgUAACDgWALcPGd2nfUTx2FDQUhPx4PuWQ/SEUJxdj5N8j4bpYNFP5LjO6ICc0R+hApwEVIxX0Wd8HT75/gpTnKZhyY4pQO5fDRWV+JWQUZIjBkQtfXYBGRw18/Zrqp6Btow0lPSXoO+kT52U51BLD9w8nVhEB/KpDFpvVYlm3Sd5JuD79OobsGkIMNtJxdepV1JbVYvo9alVNTWkNWGwW9Bz0+PJen9zzdpnRBbF3YxFxKUKo6hPg7xnY0mzMfztfLEnCBokoSUvtcOu58P3LFypGKrR/J5KfY1MZc2oMVI1E87CUBEHHglAYX4j+W/oTg6uNHRRq22iDx+OBx+WJnOl/d8ldBB7my+d1X95d5OD947WPUVdZB6s4apVRhn8Grk29hm5Lu6HX6l6E3vRkh2Yj5EQIAKCqqIoSvFXWV0b3Zd1hO8mW6OfXGJbDLBF1NQrxD+IF1VHHex1HbVktjLobYYHfAtq+skqy0HfUp/WjfLT2EdJ90zHpsmieep/D0NUQXRd0RWFsITqP7yzy5yQlIwUlHSWBKsWnkIIEx44ehbuMDHpwGpef6wHAXUEOx48dww+bNwu1RV2NEny/SHx1+CtKBWTD91+/qz6G7hpKe1jHqeOAU8uBtLw07UH70e5HYeZuJpaHos04GyR7JyMrKAtmA8wEj/O4/N9a4NFAvP3jLVYkrhCpynLkoZGIuR0DltSH98XjfpBv67elH201ouVQS1gOs6T9HAf8bwDqa+pRW14rVgWBoashJl+fTPGCA4DqkmokP0+GTmcdijeeorYibQDnc1QVVUHVWBUdenag/awNuhoQA5DS8tKNfrZsaTbUTNRofy/dlnVDpn+mUJVve+DSuEvQddCFu6e70ONx9+NQV1UnlsR6c+FyuDBxM6GVtS3LLMP9pfdh5m4m9F15uf0lNC01YT/VHiP2jxDqk/QsCa9/eQ2nuU5Q1FZEwN8B8P7RG4tDFmPey3kUD0c64h/EQ1peGoN2UM9tGpJkPj2QbgoJDxNwfuR59FjVAz1X9aQEox+ufghlA2WMOdk0z1BOLQeqHVThsoSakAHw1RpIQW1RkVORg4a5htjeoHTUVdbheK/jggp0kiw5gzDxD+MRfCIYHr96SFz+u7Xos64P+qzrQ2zj8Xi4veA2Qk6EwHG2I22V34j9I1BXSZUAdVnkgudbnkO9ozomX5tMSXxyXewKcw9z2E60Fdlvbs7zOch/n09b4ZvyMgXl2eXEhJeg40HIDMjEiP0jKPuR2opa+P7lC6NuRrAYQu9D3BoM8BxAmzym76gvlmIIwD9vjrsXh7h7cZS1VcNcQ2ADwND+YTz7GNoNKir8w8fKvErE3Y9DfnQ+8Xk3Z9/Eq53COtI5YTk4YHuA1nOgqSyLXYYlYVRpl+aiqKWIVSmrsC5vncTH/i8SfSsa16ZfQ1ESc7FiaL80rHkMLcejdY9wafwlYptqB1V0HEiuSmFLs9F9eXeEnQ3DncV3KO0KGgp4vO4xbsy6QRz7yfdPcHv+bSQ8SgAAJD5OxKudr9ChRwfMe8XPhObWcSneAgD/8C3hcQKtb1CDF5eBs+jVK33W9cFW3lZ0nUc9WNHoqAGPXzxgPticKAEUcioEf+r/iWM9jhHH1nPQQ3ZINspzyEkta7PXYsr1KcQ2ZQN+tjmdd8zPij9jh/yOFvHsK04uhs92H5TnlIt8QBt5KZJ2vxJ+PhwhJ0PQ/3/9iUEgAxcD9Fzdk5itWl1UjZ+Vfsb9ZfdFmk8Dht0MMXT3ULH6NkZdZR0er3uM69OpnlIAX9427n4c8ZAAaHzNywnPQeqrVIpE3b/BnYV38PrX18gKpnpalaaX4ojLEbz65RXR06W+uh5Hux3Fn4Z/ivy6DbKUAEQ+aCqILUBOWA5tAP3d3ncoSiwSZEuLgmoHVUFAn+QDlh+Tj9qKWmhba4t8YGkzzgbg8avlGui7qS+MuhvRHrI2wOPycNj5MO0apKSrBHUzdbHl1egw6GqA0UdHIyc8B0FHgz7f4RPS/dJhNdKKNhD+csdLnHA7gfqaD1Kft86fx6zqpq13s6uqcG7XEcrjGhYa6Dq/q8B/9lPqa+oRcztGcDgMACWpJWBJsbAoYBESHiYItX1M8Ilg7FTeiQOdDxDbr02/hkz/TLEsGYJPBiP+fjwG/jyQEmhPeJyAn2R+QnVxNb46+pXIVcSuS1wx4/4MIXllJR0lgRyu6xLXRiu0ZjyYQXsd6zKjC+4tuYc7iz7sGUTZ56kYqCDlRQqCTwZT2oqTinFp7CXst96PM4POCLUlPk3EoS6HEHsvljiuJ8sTnixPYrV34JFA7LPch5xw8ufsu8cX54ada7SC+MVPL3Bm0BnK+hh9Mxql6aUw6UtOXhqxbwQW+C6ATud/t/oo5k4MPFmeeLv7rch9M/wzEH0zGi+3vaS0nR95HlcmXpHEFEWmPKsc4f+ECwWuP8ZnJ997s99mYUWHF54vEHGR7D3de01vLI1eKqikNulrgn4/9oOSrhK0bbShaamJgL8DEHyC+n39mAH/G0B53Qb0HfWxlbe1UdUIOlSMVGA/zR5+f/nhyfonlPaws2GIuxfX5PHUTdXxXdp3FNULSeM4xxHD9g6jJIA1Fx6XJ7Rek5L6/g3a071tfVU9yrPLRfbhbevE3ImB9xZvqmcfD4JELhUj8udUVViFhEcJ5L8JD5h+fzpmPZoFg64GlIC/UXcjdJnWhTbQF/8wnvZaYtbfDK5LXImywFHXonCq/ylcnXKV2DfxcSIC/w7Ei20vKEHu2rJaeG/2xrmh53B28Fli/9aiNL0Ub/98S7tu19fUE30XP4d2Z204zXPCiIMjKG0rElYwVX0SpjXXPCbYx9AukJKSgoWFBaSkpKBhwT94pauC28rbSlmkaspqkP8+X1BWLi5aVlq0klzNRc1EjShvxiA6aa/TEHEhAgGHAlp7KgwMYvHxmsfQcrz94y2ib0QT23qs6IHx58cTvZSkZKUw+PfBsBxmScmiB/hVDfbT7OEwi+y3MfQvfqAl9VUqAKDvD32xLHYZ1M3UYdLHBOvy1mFp9FLi2G93vcW5IedwftR54thbufxr4MfeOZJk+t3pxGzChkAY3cGdoash6irriAG5krQSPPd8jnTfdGLfwCOB2GW4C5mBjUs4lmWVNdouDsUpxUh6loRhe4bB+ivRpMTGnR0Hy+GWxOConqMeeq/rTesDKKski27fdoNxL6rnmpSsFBxmOiDtdRpi7sSINCcA8Nvrh8jLkSL3+xzS8tIw6WtC6zMTdDwI50eex8m+Jyltn1vzipOLEfh3IArjCiU656YwfP9wmHuYE4PNnFoOsoKy8HTjU9RXU9eKf4b/g6zALHT6SnSJoGl3pmFDyQZ8G/mt0AEHj8vDs83P+J5WNHDqOFAzVYNpf7KUsOu3rhi2Zxim36VWnX4ONWM1bCjZAHl1eeJv9m+Hv3Fzzk2UZZUJBaiagpyqHBYFLkL/LR9k/dw2uGGB3wJiEsLHZPhnoDCuECUp5ESI/j/2h4GLAULPhIo0p6ZiNdwK6b7pKMsUXodqy2sR9yBOkIjxKZEXI+H7FzVo2oD3Zm+kvU4T8iQtKS0FOYxLRQ9ARQ01caRDjw4YfWw0bWJI3P04XBxzEQGHP+ylh+8bDh6Hh5tzbuKf4f/gb8e/iX0b5FbpqrZi78bCaoRVo/60dLy/9h7Jz5PRd2NfSrBPxYB/wN9lWhc4L3BuVIKXRMSlCAT8Tb13GLZnGEYfH41nPzzDuaHniH3zo/Ph+5cvChPIaxRbig3nhc4wczcDIN4+7/2194i/H095XM1EDePOjoOOnQ7k1ITfc25ELnIjcvH6l9eNjl2aXkp5rKGa/liPY8T1rTC+EAkPE7DPch9lLagprcG5YefwfMtzZAVnUQ4sZZVloWKg0uYqZlJepAAAkr2TRe4rpyKHTqM6YfJ1qjSqmqma4LP/t5FRkkG3pd1o92dWI6wwwHMAjLoL7xsXhyzGsL+GAQBibsfg5Y6Xgs9RUVsR2tbags/PrL8Z3Le5Q8VABRV5FaitqMWbP97g3b53jc7Nd7cvYm6Lvo/5HPqO+hj/z3j0/19/on/28vjlGPC/Abg55ybt3vNTciNzEf8wnnj4fuvrW/BkeeKw8+FmJZ7VVdbheM/j8NnuI/YYJGSVZTFs7zDo2Opg9rPZtMlzLUl7u7e1GWuDQT8PQsSFCLESU9oi8Q/jcXH0Rbz86SX1/pbF96jeWL6RViGtoQo47J8wStvVKVdxecJlKOooorq4mjZIFXU1Ck82PBHaH6a+SsU/w/7B041PiX3yY/JREFtAbGu4NvVaS1ap+OroV+i6oCtebnuJyoJKoTYFLQUsCloEEzeTNldFXl1UjYqcCuLfMepaFHbI78DPij+LPK60nDTGnBiDbt90k8Q0GRqhtdc8RsaToV3A5XKRm5sLXV1dyKnIwfVbV0GWZVMw7m0M20m2tP5JTeX+8vuQV5PHwO0DmzXOp9SW8zfEGhYazZK4YuDTc1VPvPn9DW31AANDW+fjNY/NZvJyWgolXSV0HNSRtv38iPOoKqrCysSVlDZpOWnMeED13QOAyMuRePnTS+KBDwB0HtcZKxJXCA5ClXSUoKSjhLqqOhQmFEJJR4nWm8rEjX9Y03W+6LJGn+Phmofw3eWLaXenodNI4eBEdkg2nmx4gu7Lu1PaAMB+ij3sp9jTjt33h77o830fyChRK31KUkvw4n8vkBuWS/QPagiWxdyOIQa/lsUuQ1FikciSgU3BqLsRlsUuEysZx2GmA23gy7iXMcoyyxB8IhgD/jeAcuB5Z+EdJHkn4YfKHyh9ZZVlMfb0WGyT2obwf8JFDkKOOTkGtWW1RA+m5iAlK9WoR4/dZDu8/eMt1Eypn9Pn1jzLoZZQMVRB1JUoOM7+d/dJ3Zd2R/el3YltGuYaGH1iNNJ904lyheZDzKFtq42RB0aK9+IsoDK/EiWpJYIKCi6HC58dPrCfZk8rG6Zrp4tVyatohzXuZUz8LYnCuvx1xPfsttENiU8SsctwF2Y/nY2OA+nX2E954fkCGe8yMNNrpsjzUdRS5Ad6ptNL7frv94f9VHuxfdNIhJ4Nxc3ZNwX//zTjvTC+EOdHnIfLEhdicGvIn0Pgf8gfqa9TiRLNjnMcoWaqJhTwVVNVRXZm0/wrcwDomZITFVN8UlCUWATHWY6UiscGiU+nOU6Cx6Jv8pNjws5RD/o+xmyAWaPZ4RtLNzZh5mRGHxtNm82u56CHCefFk7YCgGtTrwHgH/I2ePblx+Qj+lY01IzVEHIyhLZvZkAmHn73EMneyUQf2TODzkDNRE0gyyzqPi/iYgQqciswfD+1yklBU4H2emMzxgaxd2Jp183NtZvBreMSq6bNBpih/9b+iH9ADTAC/Oo7EzcTYmCCU8dB6qtUdF/RHcP3UOfceVxnlKaVojStlFK9x6njYLvsdgDAHO85AqlWv71+CDkZgml3p7WYpLPrElfo2OnA3MNc5L7aNtqYdmcasa2x9bilUdBQgJKuEtLfphNlRK2GW8Hcw5xyqKxr90FZIPJyJML/CUfvtb0hLSeN6uJq1FXWQUlPSeg6wOPx8IfuH+g8oTOm3pwqJIlLQlZZllb69v3197g84TKtZP7nYLFYGLB1ALFNXk0eVYVVCD0TCsvhlp/1VKzMr8Qh+0MAgE0VmyhV6zUl/GBQTWmNUGLGp9TX1ONA5wMw9zAnSqln+mdCQUsBHT2aft1sKj2W94Bxb2Okv02Hrp0uraxrS9Ee723TfdMRejoUHr94iJw80hZ5f42vmvD1668p74fFYn32PaqbqUNRWxF+f/lRpLzl1eVRX1WP3zR/A4/Lw+qM1UIVpNdnXEfa2zSYDTBDyMkQ9P2hr8B+QUFLAdZjrGE+mLzuHrDhqwQsj19O8ax3mOEAhxnkey2An4Qx8KeB6LmyJ8XTWUpGCgZdDTDP5/P+oqLwYMUDaHfWblZALTc8FxbDLIiSyHUV/D2QcR/R9/H11fWoyK2AgqYCRbHGk+UJgPHskxStveYxwT6GdgGPx0N2djZ0dHRQX12PTqM60RqUP9n4BKkvUzHz4UzBAsZisSTi0eG/n29M7/6Tu0QPyKqKqvDC8wWA5vnZMPBR1lfGhpINbS5blIGhqXy85jG0HGtz1tK2Rd+KRn50Pnqs6kFpq6usw6Vxl2AxzAK9vqNmEmpaaMJpnhNt1RaPx4NqB1WBlElVURXqKupQlFiEU/1PAeBXhdlNtqOsY5ZDLRvdhAf8HYDH3z/G6GOjid4FjZEXmQcAiPeKpwT0qourkfY6DfXV9ajMrxQ6AG4K50eeh6aVJsUDCoBAapBOEs7cwxw6tjq0f08tKy1aE/PmIqMgA2k5afgf9IfFYAtK5ntjRF6JBHig/Rzi7sYh5FQI+m7qS/mcTQeYIu5+HAKPBlI8+wCAxWZh/tv5YgUhw8+FI+JiBGwn2oIlLWGtqEYw6mZE+9393JonJSsFPQc9qJr8+9non6PrvK60VWd9N/YVe9y4+3FI903Hy59ewuM3D4HPCluKDcc5jgI/IzrOjzyPkrQSfBP2DbG9ILYAbGk2rbcoHVnBWfBa6YWe3/VE53FUL7AB/xsAcw9zRF6JpFQPVBVV4fKEy7AZa4MeK/hrK5fDxaO1j+Cy0AWF8YVIfp4s8DXjcXk41f8UUl+lwnq0NTGI0oCmpWajgZ7om9HovqI73Le50z5HHBoy/rWstTDt9jRKtZmiDv83muqTSuzPYrMQ+HcglPWVicG+safGUh4bM306zv78M2Y0QcrzlKwcxk6nVnCm+KTgVL9TAIAu07tAii28BvX7sR+cFzgLrXkfBwPm+86HnIr4h5/h58OhoKUAy6GWIvVT1ldG4pNEXJ9+HQN/HoiO7tRDcb99fvD7yw8zHswgVsjTYTfZDpGXI1GRVyEI9uVF5cFnuw+c5jlBQUsBw/YMI/a1GMr3+4l7QJYG5NZzhYIAou7zFDQVYNrPlOLj+znUzdQx+8ls2nYpGalG/ZMG/G8ABvxvAG07XaKPopYiNpVvou1XW1GLZz88Q7dl3Rr1SvpYdq2+uh41ZY0HU5qLpqUmNC3Fr/BIeJwAWSVZkZKS/w3i7sWhLKsMQ/4YQmz/y/QvIV9lHo8n8GGWVZaF+zZ39F7bW/BdefbjM/jv98earDVQ1ldG1LUo+O/3x7C9w+CyxAUGXQ2IMuSfoqitiOLEYmJbbUUtACDmluiVf1nBWXi37x2cFzoTE1tywnKga6+LTZWbmuQf1lCxq22jTTx8JyWqkaivrkdxUjGCjgYRg31J3kmoKqgienM2h7rKOoSdC0P4+XCkvEiBUQ+jfz3Y197ubfOj86HbRRfL45ZDSe/f/Vu1FH039YXzQmeizCaPy0N2SDYCjwbCpA9ZpUNWWRbu292RGZDJ96L+KJg/+PfBSHicAKPuRtC00qQExMPPhwMABr0dhL6b+golMOl01sHUm/T7O/2u+sgOzm6yF+jHlKSWoKashrge8bg81JTWQFpemqjCIi6BRwJh7mHerGBfbUUtUV0IABxnO4qd+Jjum47T7qfBkmJhS/0WsefH8Hlae81jgn0M7Y4UnxRcm3oNEy9PJGY0N8iUVBdXC4J9RYlFiH8YD4vBFs3awA/cMRDZIdkADxLVUlfSUYL9NHuJ+4j8VylNL0VJaolYnlUMDAz/HdL90lGeXQ6bMTaUtgZvsd5relPauPVcpL5ORcKjBCjpKFFuiDr07ICwc2F4t/8dxpwYQ+l/0PYgKvMrsSJxBRS1FPFs8zMEHAzAwoCFMOphhAy/DNyYdQMa5hoiHxjd++YeABD9/j7H9LvTURBXQDRANxtgho1lG3HQ/iBe//KaEuzLDMjE0W5H0Wd9H3j84kHpXxBbgPKsctRV1lFuAGWVZDHpyiRi1R/Al4T7NvJb2nk/WPEAKoYqcNvg1oR3KRp1VXVIe5MG783ekFWWFSnYd3Uy3z/CqLsRJQAQfSsa2SHZmHx9MtHXzGGmA7x/9EZhPFUSrrqkGl4rvWDuYf7ZbHQSFkMtoGGpIXFPmKqiKvym+RsAyWeFVhdXw+M3D0pW7r9BQ6brAr8FlM+/uqQa8Q/ioWuvS3uw+W7/OxTGFwrk0JrKkw1PUJpWimF7hgll77LYLISeDkVdRR26LyNXHFbmVyLJO4m22vXO4jsIOhKEDr06YP6b+SLNq66iDgUxBUh4lAA1EzXioaSJm4mgCvljuHVcZIdkCwUqi5OK4feXH9LfpGO+73xsY2/DrXm3MOnKJHA5XBTE8aWbStLI8pwf8/x/z1GZX4kR+6leJOH/hCPmdgyxyqg5NFR+bpPaBu8fvTHx0kShdhUDFbhtcoO+I1l4U1ZZFlNuTKFdW7KCspAfnQ+bcTaQUeCvFQsWLsRPnp7wA0BNR/mAH4CX9fX4nxk1wFmUyPe1dlnsQqzQJAUeGzyuFgUtgq6dLm1SXeKTRFwYfQH9NvdD303UgHfSsyRcn3Ed5h7mIgf7ynPKURBbgJK0EoqCR3ZoNvwP+INbz+XLWYq4xo08NBKDdg4SClLzOPwAZ/SNaMx4MIN2zVXSUcKioEXg1JD9neb5zMON2Tfw5o836L2Wuq/4HBZDLAQByE9J903H5YmXoWunC+ux1kIHjIXxhUh8kgiLIRbEwP6rX1+hvqoe3Zd3p3gchl8IR+jpUIw8OJLYNys4C6VppbAaYUV7CJsVlIXi5GJYj7YWek5+dD7sJtsRq2ylZKSI15Dk58moq6gDp4aDA7YH0GNFD7gucSW+rrjUlNYg8nIkdLvoCmRMm0re+zycG3IOMooy2FTxIdDJ4/KwTWobgNapmGio4qBThHjz5xuUZ5XDbsqHc5X66nr8ofuHoKru08/frL8ZWCyWYN9WmV+JnPAc1FXUCSqYa8v5wbrG/I5ZUiza747jLEd0GtlJrOTdkpQShJwMQcjJEPT+vjcG/zpYqP1k35Mw6m6EWY9nNWk8rU5amHprKnRsdZqVTCynIocRB0dAxYDs4+S6xBU2Y2woe8bmUl1cLWRn01KVsV8SiU8T8WDZA8x+NrtZ54dtiYbKvMqCSihqKwpd++tr6nHEhe/vW5xUTAz28Xg8uCxygeti6rqr1UkLG0s30v6epWSlYDHUgp+w8onYQGV+JYKOBcG4jzFM+1KV2BYHLaZ9T3lReThodxAAeX31WuWF6BvRWJe3DvLq8kLzK0krwR6zPWBLszF091DaPbUoVBZUYsifQ5qtnjF873AUxBWAU8dpUkJCU2nY33zsTdzAxrKNtPKrDO2P9lE/zcDwEQ1SH1lBWcT2cefGod+WfkLZ7pmBmbj/7X1cmdQ8Y+y+m/pi0uVJEg/KSctLo9NXnRD+TziKk4slOvZ/kcAjgTjZ9ySSnye39lQYGBjaKNx6Lo73PI5LYy8R29093bH0/VKK/w3A95XaWMaXIaPzGskOyUZmAFlmzdDVEDWlNfDZwZe+6ujeET1W9oC2tTbmv5mPmQ9nwvVbV+LBXsTFCHiyPHFr3i3i2OPOjsOUm1PEumFhS7Oh01mn0cqBcWfGYfw/4ymPp77mV62Enib7YbkucUV+dL7g4P5jOLUcmA0wI1ZoAPwDx0vjLhE9hQDg3b53eLrxqWAOkiTZOxnXpl1Dj5U9GpWJIdFwcEYKUpRllqEgrgDKesrEPYWKgQo2V2+mHFAB/Azt0NOhSH6RTPRR+hwPVjxAhl8G8YC/OXxO8eDFthfwZHni1a+vRB474XEC/nb4GwmPEsSdntgoG/B/Dw0Hlx9Tml6Ka9Ou4VCXQ8R2n50+eLD8AYKOBYn8ukN3D8XESxPRY0UPoYAaj8vDtLvT0HczfdVgUWIR6qvq0XUB+XC34TfemBxw9K1ogWzjx5i4mWBtzlokPEzA3UVUL+wzg87gxbYXxDGVdJXQ78d+QtVWGhYaGHNyDIbu5ktCdV3QFWYDzQDwD/3XZq/FVt7WRg97AKAgrgAvPF/A/4A/sX3IriEY/8/4JgUN6Qg8GoiMdxlCj53xOIOHax5Cw1wDRYlFlO8Bi83CoB2DaCt8b8y+gTuL7gjJXX3MEZcjuD7julCFk76+Pn759VeMkpKBH81c/QB8JSeHwTJDkXCK+rtxmOmAH6p/wIgDI4hrUF1lHXYb78bD1Q8FjzVIbBUlFGFXh124s/gOeDzqwVBeVB7qq+rx7IdnxLmdG3oO0vLSGLhDdFuEGzNv4Ommp1idvppSgV6czK+YMXM3w+KgxSJXfMsoyUDFSNhH7vWv/CRStjQbaW/SEH4hnNiXy+FC11630YSQyEuRSHmZItKcPubqlKu4Nv0a5XEpWSmomagh4VECXm57KdSW+CQR9765B5+fyR5gTzc8xQvPF0SPutK0UiQ8TMDzrc+J0ql+e/xwccxFvPnjjSB43EB9dT1i7sTg7pK7uDzhMqV/yssURF6OFKlSw7CbIUzcTCQuQf0xvn/54s7CO3i89rHIfRvUCT4OmgFo0UrEplBXWYf4h/HE5CEAqMipgJKuEvr/+MErlcVmwWWJi0CGmVPLQVVhleC92E60xfC9wwXVva6LXfF9/vdCwfDDXQ/jhNuJRudm4mYCl8VU9YIGSFJzTaHTqE7YULoBKkYqKM8sp7T3+7EfrMdYI90vHZX5lYQRPpmHhgKsR1vTBn2ib0bj2eZnCDoWhOoS+oprFpuFbt90g81YapIhwA/C3fvmHp79SF47xUVBUwFGPfhrk5m7mViqEP81Oo3shFGHR4FTwxFrr90Wqa+uh9cqL/yp/ycqcoSTQtnSbAz8eSAG/TIIo/4m++mmvkrFNvY2+O2j7jwern6I2/Nv0772uHPj0Httb1QVVaEkrQRczod1Me5+HJ5ufAq/v+h2NPQkPObvb6TkpIj7EbspdmDLsPG7zu/IjcwVapNVlkXXBV3Breci8EigyK9NoiC2AA+WPWj2OWTQ8SDcmnuLWN2XF5WH/db7cWPWDZHH1bTUxFbeVqzJWkNpk1WW/SLkahn4MJV9DO0CFosFTU1NsFgsQTbCxze9H0M6jDPrbwYAzfZwy4vKg7SCNEuW0OIAAQAASURBVDQ6iiZ51BQaDg+Kkookns31X8NsgBl8dvgg8HAgrEZYtfZ0GBhE5uM1j6FlaDiQpasmU9ZXhu9fvijLLCN6hbBYLCyPX06UMUvxSYFGRw30WkM2Cx91ZBRUjVUFPrK2E21hO/GDj4rFEAtaWavilGIAQE44WfKSziOuKaS+SkVuZC7sJtsJ/AQbKM8pR8a7DOg76ROrheyn2EPfSZ+2usnM3QwDfx5IDGCm+KTgrMdZjDg4gih5kh2cjeib0bAYZkHMJp1wcQKCjgRBXk2+qW+1yaibqaPHqh7oMq2LyIcjow6PwqCdg4gZ3N2+6YYu07ugtrwWXA6XEnh7svEJCqILMOXGFEpfZT1lbCzbiF81fkVZehmtdyQdXb/uCvWO6iL1aQry6vKNVi00/NZ8d/nCbb1wFebn1jwdWx0o6ysj3Ted1qeuMXz/8gVYQM+VPUXuuyaTekPcgJqxGj9LOr8S9TX1lEPJuHtxkJaXxqqUVSK9ZsSlCGQFZsH9J2pFVlVRFS6MugDnRc5EWVyAn2E97e40aNuQvT/dt7l/Vs7y0ZpHkFGUoT2UHPC/ARTZIx6Ph6zgLOS9z0N2cDYGeA6AnoNwCverna+gbaMN5wXOAPiffUN1T+prvlynqD6UAD+ADvCDeiQ0Ompgn9U+WH9lTfxdfQ5OHUcQ3Gz4ntdW1CLpaRKSniYJnleeUw5N5Q8HwpkBmbg27Rr6/diPKLtkO8kWJSklqK+pF/jXfAqLzaKsyd+tWYO6qnr03bIJbjwpzEcd9ABkAzgjL48XdXUYbzQWzmUuGPwbNWmALcVuNOB/dcpVlKaXQl7jw7ra4CXrs8MHlXmVCDoShOF7h1Pm7fqNK7SstWgP6UccHAE1YzWRKqUbsJ1kKzi0/hSrEVb4vvB7YrV0U/jb8W8UxBTgh+ofBO9pzMkxqMirgJqxGvZZ7QMAdJlG9YUMORWCOwvuYNqdaeg0iuprG3wiGBMuTID1GP53W9R9XtqbNFQXVxMDIwbOBpj/Zj5yI3MplYUNVbR0cmAjD41E6qtU6HahXrv7fN8HRYlFCDwciKF/DRVUljbgvNAZ5VnleLrxKbRttIWqv6oKq3Bx9EUYdTfC+PPjKWuF83xnmPQxoUj9Avzf2tNNT1FbVgu3jW4CCW/bCbbIMM6AgqYClPWVBdKKkqThHtz1W9ErBm0n2MI4xRjy6sJ7EbYMG+vy10m0OkMU1M3UsSxmmcBX7lMG/zaYskZIy0kLeYy+2PYCPjt8sPT9UtrrCsCvdr+z8A4sh1ui88TOtGtaA083PoVxH2N0Hk+VhM6LykPI6RDYjLURuUqGLc2GnIocVqevJrb3Xtsb76+/x/GexzH+/Hjib/pT3u5+C//9/pj9dDblrMb7R2/kRvADCSZuJrT7UU4dB8XJxVDQVKBU0gL86qqS1BKBPLSkkJaXxtRbU1FVUAUNcw2wZf79mo/2dm+rbqaO4uRi3F18F8til7WYXcC/yZ1FdxB2NgxO85wo10kpGanPSs83KL883/oczvOdhcbICsxCWWYZws+HI+5eHIb+NVSoeqxh7353yV0EHg7Eurx1gvuqhmCq+RCyZ9+fhn+iPKtcIBv8MTZjbKBlpYUOPTsQv1v2U+whpyqHmNsxFOUaRS1FjD46Gm4b3CRWzKFlpYWBOwaKJCFOQk5FDt2WdiN6msY/jEdBbAEKYgsw7qzofqZ0BB4JRFVRFeUejUE8WnvNY4J9DO0CNpsNExO+nIy8RuOHSQAEWR0NPywlXSUsj1ve7MOthhLxjeUbhXSmm0tRYhH89vihz4Y+tJUNrQWPx0NNSQ3lxqUtY9rfFAN/Hkg5YGJgaC98vOYxtAzSCtKYcGEC7XWhurgasXdiiTfcnFoO4r3ioW6mTjEKB/iyWWHnwmA/zV7gRwf8v98RC5BXkyf6ppSkluDimIswcDHAoJ2DiBIbbuvd0Of7PrTvq7qkGolPEqHTWQc6tqJpxL/Y9gKJjxOR+jKVUr2XGZCJi6Mv4qujX8FxtiNFxkhZX5lWYgzg/00MnA2IVYMNckJeK72Iwb4+3/fBm9/f8GW0CdB5BkkCHVsdDPl9CGrKalBXVUc57GwMeTX5RgOQTzc9RcDBAKzNWUvxTsl/n4+YWzFI90unSImx2CzIKsvCaZ4TNK1ElxZS1FFE+D/hcJ7vLHLGfHFKMcLPh8Ntveg3xr3X9EZH945CHh8NfG7N07XThYySDJKfJYv0mg08/I5fnSROsK8x5FTlBNmxpPc16/Es8Lg8kfeMcffiEHY2DN2Xd8f5kedhN9kO/Tb3A8A/hHWc49iohKu8ujyqCqsQdSVKLHlbHo8HxzmOROm+sqwypLxMgYmbCaWdxWJhfeF6BBwOwP2l99F9uXCFcV1lHXqs6AE9xw/7s+QXyXjz+xv03dQXL396ibTXaYLKaU4tB++vv0d5TjkUtRQbTWYw6WOC9UXraX1Xastr0X15dxh1Ez3ABPAPwpbFLBOq9pZVkoXDLAeEnQ0DwJfE/HSNY7FZKIwvxPOtz4nBvk6jOmGv+V6w2CxiUG5N9hpIy0kTA1jfb14PVR81XHp8DWuUMsDhlMDMTA1jp0/HmQUL8Hj2YyjpKBFl7ctzynFu6DnkhOZgY9lGylqgZa0FK66VULVPSSq/KjI7JBu91/VGbXktcR2QkpFqVJ6T5EPaVFwWuaAsqwxh58Jg1MNI6ABWSkYKChoKyAzIRMrLFNhPs6eVyyNRlsEPGGcFZQmCCw0JLFFXowDQe7A27AUufHWBeJ/6aM0jGLgYCIIaou7zMt5loDy7vFF/O107asBOz1EPS6OXUoLFDbgucW1UCtNtgxucFzgTs/1N+phg7JmxKIgpoOw35NXlMebUGOh01iEGdWUUZXBm0Bl0nd8Vo4+NFmrj1HDw9o+3APiy6A3JAPFe8Xiy/gn0HPVQlFREWw3bHJrjhcSWZhOlplksFjGw82/itcILKS9ThORFP+bZ5mdgS7Np/RmNuhvBeeGHPYPfXj8keydjwoUJkJaXRlFSEbICs6BhoYGoK1FQ1lfG8L2fl0zuvrw7bTJ2wqMEvPntDXx3++LH2h+b9kb/n6qiKhQlFEHDXIMoTQ8Aul10Mfj3wUL7dDriveLxaPUjACDK3I04OAIFMQVQNVYlBrAbKE0vxf5O+wGQJQe9f/RGRW5Fixy2K+spI+RUCJ5ueIqF/gs/6/sradrjva3lMEvIq8vTfofaGyZ9TcBiszDq71G0crRVhVVgsVnEsz9dO104L3RG0NEg1FfXC+1L5r6YCwB4tO4Rws+H8yv3Cbeh5h7mkFGUEdqrOcx0gNVIK9q/c3kWvzq3PLuccq+pbqb+2UIJq+FWsBpOXwBAupcXF0VtRQSfCIassiwx8aepBB0Ngmk/U2KSiM0YG6gYqhDl1j9HxrsMHOtxDH029IHHTmHbjQapXybYJxlae81jgn0M7QIul4v09HR06NABMbdi8Gj1I3x19CuYe1CzPxq8VZbFLBNkVPB4PGhYaEgsql5fXS/RYJ+Mogw6j+tMvElrbY71OAa2FBvz34rm6dKaNCUzieHLoL66XqKGym2Fj9c8NptR3G4JpOWkYT+VPkD0cPVD5EXlYUPpBkpbTVkNLo65CKsRVhhzcgwlUOM42xHqpupCsi+V+ZXYbbIbow6PQk5oDhQ0FQReRu8OvEPio0QM2DYA2SHZyA7JRvDxYMx7NY+4kW/sWvanwZ+or6pH/639aQ9t6Oj7Q19kh2QLpNo+Rt9JH2PPjIXXSi+83P4Sq5JXCbVz67lIfpEMFQMVYpDx/tL7MO1rSjwA1rbRht1kO2KwBADk1OQwx3sOVIzIB3tZQVmQVZZtdhYlHTnhOTjifAQDdwwk+k/Rsdt4N+qr67EqdRUlSFiYUIi6ijo4zXUirmFTb07Fz8o/48X/XlAq9+pr6pETloN+P/aj9WRrjJqSGlTkVAhJ6DSVJ+ufIPJSJEz7mVK+m7Xltbgy6QqUDZUx5jjVqxIArZduU9a8qTeniiT59jFLwpbQ/m6ygrPw5rc3cFniIlCC+Ji739xF2us0zLg/g3iI19icZBRkUJJagqLEIuh1aXoCUkOVzV8mf0HbRlvos5JVlkXomVDUltei6zx6Gc6wM2HI8M8gBvsiLkXg2tRr6LqgK0YfHU1p59Rw8HzLczjMcqAoZmSHZOPa1GsYfXw0MRgI8CXdSFW4FXkV8P7RG73W9BJU71XkViDuXhzi7sVh9rPZuDzhMl7ueIl+P/RDTWkNrk37IFvYZXoX2iAzW5qN0vRS1FbUEr22rk65iiTvJAzbLZp34seQ1pjUV3z5YKuRVjDtZ0oJmhk4G0DFSIW2Gk1ORQ6Ocxxh4EL+bTQmq1xTWoPpx6fj/uKF6MaWgqYmcPr0h/ZZj+j9qGJuxyAnNIfva0f4k5ISUhoqyu0m26HHyh60vk9lWWUoTSuFjq0ObULBfuv9UNZXFhwQikJuRC5uzLqBUYdHCQX7qourUZxSjOhb0fDZ7gPjPsYiBfum3Z2G2DuxQn0ern6I0DOhcJjlgP5b+6Pr1+TfnNkAMwz+YzCyAsk2E1NvTUVeVB5ywnOg10VP5H2e01wnWI2wgqox9W9eEFuAqKtRsBxmCW0bbaEDWCkZKUFFpqg0SG+bDTCjrUpT1lMmfkdlFGUo3r6fYjfFTsiPtAFpBWlMvjYZuRG5AgUEgL9WGPUwgryaPHqt6SWyp3FLU1lQiYN2B6HXRU/IC45bz8WDlQ/AYrMwYh/VT7Slqausg3pHdb5XL4GMdxnw2eEDLWstwb6xurgadxbxK/S6zuvKr7ge/aHiOjs0GzF3YgQJ1klPk3Bn4R3M8JqBLZwtRDk9utfOCswietpaj7ZG9M1o2IwjV5cD/OC6roMuBu0YJPR44pNEXJ18FXZT7OC80Bnmg4T3tWcGnYF+V33iOkdCXl0eFkMs0Htdb+J1z7SvKdFr7FMaguYkiwCAX4nrNNcJsiqSO2cC+DLXDUFGAEIV2/8W7e3e9vXvr/HC8wW+fvV1qwfrJYXLQhfaZJua0hqcHXIWGX4ZMO1virnP5xKfN/LgSIzYP4K2OtRjpwcG7Rgk1F5fXY/dJrthN8UOI/aNEFKzAfiVp3KqcrT79LU5a1GRWwENC/IaFnUtCtkh2RiwdQBlT/54/WNkB2cT90Pl2eV4tOYRjN2MYTvRlphkKw49VvZodiX31NtTaYstNMw1aPffn6M8hx84ff3La0qwz26yHWrKJFtV/F+mtde8tr/KMjCAH6wrLCwEj8cDW5qN4uRiWt35hmyPj6XZ3u17h23sbXi7622TN58kNtduxubazRLP7lHWV4bbJjeEnw9HvFe8RMduLvpO+jDqKV4WdGvh+5cv/nb8u1m+LAxtn3f732Gf1T5UF9N7I7RXPl7zGFqOh6sf4k+DP4lZxeYe5ui+vDsx81FWWRZjT49F3P04oj8BW4qN51ufC/nqVRVWob6qHjdn38TbP9/i2Q/P4LeX702QG5GLuPtxUDNRw7LYZTBzNwMAlKRQ17CipCLcXnib9lqh1UkLsiqylBuppmDW3wzrctcJsug/RtVIFY6zHNFtaTehA58G3h14h7MeZ3F64GlKG8C/gUh8kiiQOfqUiZcmYsL5CcS2osQicOo4tIGtIy5HsN96v8AvR5KkvkrF7a9vQ8dOp0nZ3x9TllWGyvxKQTDgY95ff4/Q06Fw/caV1h9h1N+jiFKw5dnlONb9GPz2iO5tAfDlDrst6yaW7KnzAmf0XtebmAVbU1aDeK94hJwIIfZNfJIIn50+RBn2z6158V7xuPftPVoJ98+h10WPVmI2/Hw4Ii5G0Ho0Bf4diNzwXKT7pVPacsJz4MnyxOUJl1FfQ/bVOD3wNA47HSaOnRWchbyoPMrjDUElQ1dDLH2/FAO2DhBq7/p1V4GXEom0t2lIfJJIWwkX/g/fd6w4sZjYzmKzMOTPIcSECIOuBph0dRJ8//LFhdEXhNp4XB6irkUhwz+D0g/gZzyPPDQSlsM+BP3tJvEPY23G2qCje0ewpdhIfJwIAJBVkcWkq5Mw8tBITL01FamvUvHuwDuiB1ZNWQ1O9T+F4z2PE1/bYqgFnOY5EduaQmV+JXz/8hXyBq0tr4VGRw3oOeoh7l4c8TsCAKvTV2PixYnEtle/vAKLzSJWJ/N4PNyYfQOeLE/kx+RT2v32+mG38W7Il/G/Q6Rzsr9M/8LJvicpj3fo2QFDdg3BysSVxARGHo8Hn50+eH/9veCxnJAcSMtLY9ThUYi4GEHrSRP+TziO9TiGnSo7iePu67QPBbEF0HcWbU0FgCcbnuDtH28x5eYUSpVbwqMEHHY6DEUtRXz95mtK4kncgzh4sjzxdNNT4thm/c0w5I8hQlUCteW1qCqogt9fftCw0CBWbjXQe01v2uuYaT9TPN34VFBlLOo+T15dHj47fHB18lVKW15UHp798AxHXI7gD70/hNqCTwbDk+WJsH/CiON6sjzhyfJE7L1YSlvYP2G4OPoi8qPzidVM95fd5ye11NQT2wHg6Q9P8afBn6jIE/aHevPnG0ReiiRWKLOl2Og8vjP6b+kvdKCZ6Z+JosQiVBVV4cGyB0LyuZIi4XECPFmeuDjmosh9Aw4FoCKnAhW5wu+1uqQaAQcD4L+f7Cfa0pRmlCLgUABRdh4AAg4HAADmPJsjeKyuqg5RV6KQHUxWVBhzfAy21G8RJDJ1HNgREy5OgJ6DHlhsFthSbDxa9whPfyD/1hoY9PMgTL01ldimYa6Buc/n0lbk83g8ZAVloTCOeiak10UPg3YOQuSlSLz86SWlvTilmOJZ1hgdenbAzIcziYnmoqCopYitvK3YUExNJAT4e+WOgzrSJg2Iy6eV4ZKsZGoq7e3eVs1YDaZ9TSUeeG1twv4Jw+WJlwVBnwa4HK6gup10nwcAmYGZeLDyAXIjcymBuRSfFCS/SAZbmg0pWSmhdh6PB20bbVoFmJywHPyi+gserX1EbFfSVYKuvS5RXeXtrre4MvEKfLb7EJMYS9NKkfg4EVenXKXso6pLqhF+Phz3v72PE30a9xdtKg9WPIDXCq9m35PGe8XjVP9TlM+pgfyYfGQFi7ZOFKcUo7q4GiMPjcTsZ7Mp7RMvTcSM+6JZQzDQ09pr3pdXDsHwxdNwY5f6KpUoO0IyG224UXi05hFf+/gz+vF0tKTWfk1pDZKfJ6PzBKpmfWvBqeVA20YbOnaiScG1Ntx6LnLCcvB041OMPzf+8x0Y2iVx9+NQV1n3RVb2MbQ8FbkV8N3tCwDETViX6V2g1UkLqT6plJt7aTlpOM52RF5UHlFGsTynHB0HdYTjnA9SUB/LTenY6iAvKk9w0zHq0CiBN4qChgJmP5kNsMgVfFFXohB8LBiJjxKJPmBLQpY04d2Lz8CfBhIfb/Bw6bOeLDHaoWcHJDxKIAZEsoKz8GrnK7h+40qUsg44FAC/PX5YlbKKeNBq1MMIGX4ZKIgrkLh8c1lmGXIjcjHx8kSRPWC/fvU1Iq9EEudsNdwKSrpKtJmZqa9SoaitSDxYkleXh8evHoi4GIEX216g/5b+hBHoibwcidqyWmCdSN0A8IPgdIdditqKGL6PXrbr/Y33CDgYgFc/vxLINDaV6pJqZAVloSyrTKR+DdxZdAeyyrIYumsopc15gTOK4ovg9LUTse+ioEVI96XKqQIQXH/eX3+PyrxKSuXfra9voSihCAM8BxDHPutxFnqOekKHrAD/90L3fa/IrUBVQVWjEqxyqnKwHG5JrNIFgOF7h2Pgdnq5cylZKRTEFeDpxqdYl7dOKCCtrK8M2wm2CD0VCnyydHLqOLgy8QpM+5ui3+Z+0O+qL5QRL6skC789fmCxWfg28lvB418d+eA9+F3ad4LMbGk5adhO+JC40KDeYT/VnpJpn+6bjqrCKuh3JQeQeqzogQtfXcDj9Y8x+FeqXObnyI3MFQRqGuTXKvIqkPQsCV0XdIWCpgLe7X2Hnit7Cv2uS1JLEHUtCuaDzIl/7wy/DNrvNbeeK5AIJd27GPUwQsdBHZGUn4UqNer7DjoWhOqSatj3oQYS9broNVptGnwiGM82PUPnCZ0F0pO91/XG5QmX8ebPN/DZ7gMA2FSxiXKQbNrfFPIa8tDpTL5/kJaXRq81vZpcVfMxWUFZyI/Oh80YarWPrr0u3H9yh8UQC2KFeW44P9kk/J9wDPp5EKXde4s3uByuUJXQV0e+gscvHiiILcDbXW/htcILqzNXUw4dk58nI+yfMPRe05vW02zQzkEUJYCmUlNag7z3eUSfRbMBZlgUuAhRV6MoQbWG367vbl+ir72cmhxqSmqElAgasJ1gi4CDATjW/Ri+S/uOsr7Ja8ijNL0UO+R3YNzZcULJBaXppTje+zjqq+qhZa1Fqcg1cDaA0zwnYuUCj8dDfXU9/9D4o3vvSVcmgVPLgZyqHHqs6tEiFhNpb9L4702MsY37GKPPhj6U4JS0vDScFzm3mr2Eko4SRhwcAYOu5Orhbt92g9VwKyjqfFhTlfWVsYWzRfD/5OfJCDwSCLcNbsT30VBpUltei+QXydDoqIGYWzGQUZShVN19zIk+J2A1wkqogrOpsFgsuG1yA49D3cNr22jDbYMbdOx0iIlNK+JXIMk7CUdcj2Dg9oFCCSh0ZIdkI8k7CXaT7ShVzQftDiIvKg9SslKY83yOyB6DH3N/2X0Yuho2aU5NRdVIFTMfzkTM7Rj0XttbYuN+ydhPtQenjoNr065h7KmxIlsjtEX8D/rj/tL7AACPXz2Aj37KChoK+C7tu0b7F8QWIOBgAGQUZKBrpyuUEPtg+QPUVdT9H3tfGRjF9X59duPunhB3JQlEIIEAwd3dtTiUClqoUWhpcYq7uwRICBGSAHGBuLu7bdbeD/PfJcvcDYTSX0tfzifYm9mdnZ157r3P85xzMDt4NhqKGqDnqiecJ6XkpDAvfB4AIO5YHDLvZmLsmbFCAkXWwywAIO4RAUqNAnzAdoItLR+r0kMFuj110X97f2KudsLFCTDobYDHax9T0vKd6pgaVhrYzNqMyF2RYhtWugvBdzAfIl5y+32gbq4OY19jonrI89+fC2WF32Vv1RklL0twe/ZtTLwy8V9nHfUZHx+fM6Sf8cmBKcnE8IPDuyXXZTXSClPvTkV1evVfkvI8aHsQsmqymPN0zkctMDSXNyPtRhomXZsEqxEfru38scFhcRC4PhAu81y69N74t8H7S2+83PuS6G/yGf8dTLo2CVwWFxIy/4zh/Wd82mAwGTDqYwTbCbZiZZmfbnqKoudF+LaRXJgYtHMQ8fX02+kI2x6GmYEzha91TsCNOT2G6BvFYXGQFZAFVRNVsUkZi6EWyAnMoXlhfQxcm3wNqddS0e+7fjQmUdbDLNxffB/D9g+DzVh6ktViiEWXGw63xW5wW0yWjmkua0bqtVQ0FDZg4YuFtHH7yfZ4ufcl7i68S5RhmR08G221bUKj948J+8n2Yj2a3gVDT0OxvmraDtoojSvF3QV3MfLoSJoM2tNNT1GVWoUNVfSKnKyKLPp81QfRB6KReT+z28W+eeHzUJVaBXYru9vzZMGzAiQcT4DXei9awk9CSgK9V4i/L/ts6IParFpoWHdfbtVhigMid0biyVdPRIo/74v4Y/EAQCz2aVprYsqtKWKP1eupJ/Z51LDUwNywuShLKCPKcvVa3gvOs53R6wu6FyWPy4PbEjfi9bi/5D4y72diU9smxB+Ph4aVhlCGrKO5Axl3M6BuJb4zX9teGxMvTyQm8AG8098EoJoSzIeYEyUeAWDavWm015iSTIy/OB45j3Jwzv8cZgXNohWHHaY5iDRY1OXVoTq9Goaehri36B6q06pFCoGdoeemh7K4MmKhU91cHX4/+HXpy1KRXPHBawaBD6rpwDcJEkUdRaiaqiLheILwtbffvyazBoHrAmE/xZ7I7pv+YDoernqIpLNJNK8wBpMB3y2+0HbUJv5m5v7mSDieANPgu6gxdMHbP9bTTU+h46jT5VxV9LwI/bb2o82DiacSAQD+u98URgWFo5d7X4IpxQSPzQOXzYUUROOIQS8DfF37NfEzGQwGliUvI469D7qSJtWyo34jHpcHDosDCSkJkSKT5xpP2IyzEavQImAA2U2wE5EcllOXo4q211LFfnbl60okHE9AYXghliQsocXWnxR+gukAU+Jz8z54vuc5SmNKsewV/drJqspCz1WPKJNsOcwSvVf1Fit5KY5hBFDqLsMODENuUC4k5ej73gHfD4DNWBtE7YqiNSYwmAwo6SnBcYYjPFZ50I61GmmFtOtpSL+VTouPbbVt2K25GwAw9sxY4XOReT8TD5Y9wPQH05F6LRUlL0rgutBV7Pl/CNwWu8FymOUHJfZN/UyJCVRpBWmM+nMU4YiPh7L4MpTFl8Fuoh2tUCmrKouWihYErAjA/Mj5tMYBfTd9SCtKoyajRsiAZzAYIuGkLrcOry69gtNMJ+g46aA8qRytVa20+F6bU4sz/c/A73s/LEt593PuON0RWg7kay1IaOu56WFx7GLi38QfjQe7jU28xwAI5aJJ4HZw0VbTJrbA0BmlsaU43f802C1s6Lvp04p9Bp4GYLeyoe2oLVatAaCu4z7zfVC3VMfKzJW08eQLyZCSl4L3ho9fkBM0zEf8EoG+3/SFqrHqR/+M/xrYLWy0VLSA28H9p0/lo0DAxt/K3dpt320AsB1vC/sp9nj+23N4rfcSkbzuv70/eGweYv+kmvo6Wyp1RnV6NXICc8BuY0MO1FxsNsgMHU0dYtUXbkyl5NznR82nFdLtJtq9U83GY5UHPFZ50L4zg8GAhLSE0BP7Y8BhigPij8YjbHsYsSnpfRG4PhAWQy2IErKC+NNrOX1v0RW0HbRhN8kOVWlVaK5opu09BQ113Skgfsa/F5+LfZ/xSYDBYEBXVxcMBgNVr6gElTjN5jMDziA/JB9rCteISH5Zj7LucsH3PqhOp1gYjcWNULf4ePIHbbVtiDkYAzkNuX9VsU9KXgp2k+yIHXP/dryrM+kzPn1w2jkojS2Flq1Wl7JKnyI6x7zP+HsgrymP+RHzxY6/2PsCBc8KiEyllqoWHHU9Cpd5LvDb4UcbN/Iygv+v/iKbnLrcOuG/WQ0s1GTVCL2GKl9Voqm0CTpOOrg6/ioAwHeLL1zmutCYXzpOOhTzTwyCvg5C1K4o9N/ev9tFIAH7sPg5XYpOUlYSSgZKiD0ci4LwAmLRpCv8rPQzLIZaYNK1SbQxy+GW0HHSQWt1K/FYI28jOE53hI4LuSteWkH6L/no8rg8MBgMsRvfprImBK4PhPUYa6LUnjiEbAuBrKosvNbSpTgBak2R9TAL7Ba6jKzPZh+cH3weJ/ucxPxI8n26PHX5B3nYpVxKQfS+aJgPMe92sS/suzDkPc2DobchrdjH5/HR0dwBCRkJIgtJ1URVbKL+fWKezTgbouTu+8B3iy/aG8iSz+xWNjqaOyCrJkvsCq7NqQWDyYCaKXndaexrLJaV4DzLmfg6QLFGI36OgMcaD0CU2Ad1C3XwuXw8XPUQcUfi4DTTSVjsUzNTg+MMRxG2MAk3Z95EzuMcbGZtpo2xGlnICsiCrKoskT3QWNKIxFOJcJnnQpN+y3yQiduzb2Pk0ZG0witTggnHaY7QddaFobchjeFUl1uHmEMx8Fj9JjGbfjsdgesCYeZvBiV9JbTVtaEsoQx6PfVQn1+Po25HwWVzwefxsfDlQrH+1mpmavDdJD5h8+ynZzAbbEb0KBSAy+aC084hyt0paCvQipBS8lLQddZFfV49PNd6wmu9Fy0JrO+uD4BKFpPAYXGQdCYJUvJStGIfU4JJnGM6w3WxKx6kU/fG2yT1qXemEos0APD62muhJKT3l960GDr+/HhwWByR+15aSRp2k+yQei0Vc8PnvpdHlThE/RYFPo+PPhvobHAum4uGggaxe62yhDKc6X8G/b7rR4yvcX/GIWB5AGY/nS1SfJGQlhDx+Hsbww4Mw8MVD0Xmoox7Geho7oCUnBQMvQwx+NfBRCkx14WuKAwvxOurr8FuozdSWI+2FilqdHedZ+BhgN6rehMTf9wOLthtbEgrSNPmBCl5KQzbK55x/S44TnOE4zRHseP6bvqYeIVexFbSV8LCl/TmHSH4lIQyAFqxT0JaQqiAIFLYZwCatpqQVZPFqKOjRCw7PhaU9JS65fP4NhJPJ4LL5tJ8sVqqWsBj894Ztz8Ut2bdQlVqldiiL6uRhbaaNrF7+pvTb6Ktrg2rc1cDoPZYJdElUDZShpqpGpxnO8NpppPQhytsexjSb6djG49KCqffTse9xffgv8sfQ/cNhaGn4XspKbU3tCP422BiHBDEUlYD2UOKw+LAcYYjUWEjJzAHIVtCMOiXQTDpb0IbT7mYAiUDJazOW/3OcwQoj0J2Cxu+W3yh40xfi4rzKX4bHS0dAECUHgUoqdrmsmaxsuMfisbiRkTujkTyuWS017XDdYFrt4p9od+FIu7POCyKWUT0Ln4ffGp728wHmajLq8Pi+MX/Gc++cefGYcypMcQmLnYrG6k3UhG9Pxpm/mZERq6kjCTcl7rD0NOQtk4SFLYU9RQhrykv0oDZ0dyBqN+iYOhpiCG/DcGQ30T3kAa9DIhNsAJ4rfdC1oMsscz49oZ2sFvZUNRRpO3l8kPz0VbXBttxdPW0jpYOlCeWQ6WHygd5oJNgOsAUdhPtUJ5IlkB+X1gMtRDr5fyhjaitNa1U09I1aj/3uaj39+KfjnmfPfs+45MAk8mErq4umEwmSqJLELQhCDUZNcS/zQ/JBwC0Vr3ZqKXfTkfAygCxicT3xbxn8zDkjyGQ0/i4nn0aVhpYnbcaWnZaqEj5+J5DHwqmBOWPKM6T49+KoqgixB2NQ0dzxz99Kp/xNyLjTgYuDL1A9Bn51NE55n3G34fsx9kI/yFcuPnujPKEcnBZXKLsFYPJgJKBEsK/Dyf6keg46aAyuRLnB58XvmboaQjfrVQi+pz/ORywOoCyeEprP+rXKJwfch4SMhLC7sTw78ORciml298palcUAHwQy23oH0OxlbcVMx/NpI2Z+pli4YuF4HZwkXQ2iTae9zQP2xnbcXHkReJ7KxspoyK5As3lZO+BCZcmYOptsm8LAIy/MJ6YDAKAo+5HcXnMZbHFnHchYmcEdkjsIG7MGosbkXYzDa8uvUJFUvfm5/Ad4QhcF4jiF/Ti6cv9L5FyIQVLEpYQpTzN/c3hPMeZ+Ds2FDXgWO9jSDqX9EEqA5o2mhi0a9AH+Q8beBoADBDZCw1FDdipshM/yv5I7JTndnDBbmUTZXPfFfNqs2uhYaUBz7Vk7553wW+Hn9iEd+h3ofhV51fcX3yfOL7fYj/2me3D66uvaWONxY0I2Roi4uP2NsJ/DMcp31O0znApeSn0Wt4LPfr2oB1j0JtKesQdicPs4Nno+21fkfGUCykoDBf/mbXZtcgPyYe2IzlheG3SNdyYdgOP1z0mjvPYPLAaWeC00X9HGWUZ6DjrIDcoF8nnyT5gWnZacF/iTksKMiWZ0LDUEEmcmQ82h7yWPHKDcuG7xRfSitK4Pee28O91nHVg0MsAxj7G75TTD1gZgOOex4ljecF5SLueRhwTIGxHGPZb7AeX/f5d/KNPjsbi+MUYsmcIrdAHUKyayTcnY+SRkcTjazJqMD1gOob+MZQ4XvCsAE83P0VjSSNtLOVSCl7ufYlGLXOiYZ+hpyEy72UiclckbYzbwYWMigzGXxhPjAWqJqpQ0lcSmR8fLHuA4hfF2Ni6Efpu+kTvROCNT9z5Iedpf8Pj8vBi7wsEfRkk1t8zdFso9lvuJ37noqgilMaUwsDDgFY4yX6cjZN9T6KlqgWOMxxpicHm8mYkX0hGTSZ5D9l7eW9s428T8QKM3BmJm9Nv4sq4K7AeYy2WIScpI4lRx0ZhQ/UGyKnRr+eESxNQEFqAmzNvAuj+Os9ymCUshlog90kubezV5Vf4RfUXXBp9ScQrGKAkoQNWBhA9H3lcHq5OvIrT/U8LG1o749nPz/C70e9oKCR7oKfdSkPELxFdnnfuk1yE/xBOm5/TbqXBfrK9CHNUABklGXzx+gts428TSsgClKR3e107Opo7cHHExb/Fp5fTzsG9xfeQcCrh3X/8FuKPx+POvDtCeTUBmkqb8Kv2r9hjsOdjnSYNAjaZkgG9mFibU4usgCx4rvMkNvg8+OIByuLLRFgizeXNON3vNGIPU35+b/twuS91F4lpMsoy0LTRhKqJKjxWesCglwHKE8vf+Rv18OkhNmltP9ke2/jbsDKLzoADqCLgk6+fCD1eO4PdykZTWRPO+J3B2YFnaeO359wWSvm/D+wn22Nl1kr0+brPB/kdC6DjqIM1hWuwPG05cdx/tz82tmz8SwVnEprLmxG9LxrtddRzSCpYdoWCsAI0lzcT48/74lPb2+aH5iNqVxQle/8fgaSMJFqrWpEfmk/b+7bVteH27NsojSlF3hOyH2prdSsUtBXgtthNrJR8jz494LXWS2Rd0d7QjrDvwpD9kOw3317fjluzbxH3lwAw+NfBWJ62nOg1mROYg19Uf8Ee/T1gNdEbA8J/CMfV8VeRH0YV/TqjLrcOp/qewh89/sC9JfeIn/0hmHRtkti49b7oubAn2uvb0Vrz1/LXnaFhqQHPddQ+qrNKhQDb+Ns+FwA/Iv7pmPdpRNrP+P8eXC4XOTk54HK5QsNYcQmGldkrsTRpqUjHecGzAsQciMEh+0Mf7PcCAD369oDnak/iJu6vgCnJhISMBG5MvSGUmvo3gM/nY9K1SZgfJZ798k/hdP/TiNxNT14AlB/R/SX3kXpDvNzOZ3z6SDhJbcRN+pn8syfyN6BzzPuMvwesRhYuDL2AkC0hxMaAsafHYit3K7GYIq8hj4UvFkJRTxHlCeTOPVl1WSjqvZHHkFaQprFgonZThTmXeS4YeXQkZJRkMOnaJGyo3oARR0YQPeKSziZhO2M7DjsdJn7u/Kj5WPZqGVE2kNXEwnbGdpzyPUU8FiD7BHbG1LtThZ3fnVH5ivJCaqlooY0BQO8VvVGTWYOq1CraWGt1K1qqWsR2bIZuD8V+q/1oq20jjpfFlSHjbgZeX6EXY94HSnpKkFGWoW0CAcog/eGKh5h2fxoG/ED2KxSHIX9QnastlfRrIq0gDUVdRRG/i7cx9vRYTL1DL4DyODy0VLSg5GVJtxuE+Hw+Ar4IQGlMKZGd8i4M/HEgtvG2EaV5uvKQAygfmp8UfsLd+XdpY++KeTlBObgx7QaqXtPvn/dB8YtisdeKxAroDPspVCKSJNPZWNyI8O/DcarvKWJC/Pac2wjZHIKazBrwuKJFD3kNebTXtQs92TrDfIg5pt6ZimUpy2A6wFREUo7VyMLUO1MpvxUxaChsQEdzB3rO70kctxhuAQ0rDbEMXQkZSpK1oaiB9twZ+xhjztM5KIkuQei2UJExdisbv+r+ikdrHhHfV6WHCkwHmqIkukT4mra9NpYmLsXS5KVQNlSGz0Yf9PmKKuwrGypjztM5mB08GzMezsDtubexnbEdla8rae9dGFmImAMxKHlZQhsDgJmPZ2L0ydHID8snjgNAbWYtWipbiCzf1Bup2M7Yjt97/C58rSqtCr8b/Y7sh9nICcpByNYQWmKGz+PDaoQVTAeQ/VECVgTg0apHxFjAYXFw2vc0nv34DE0l9P1LbXYtsh9lg8mh5rC3wzePw0Pi6USkXKA3jjjNcMI39d/Acboj0QeurbYNO5V34tqka8LXfDb5YNDOQWgobMBPCj/hdL/TRHkzQZE4JzCHXuzj8PB4zWOY9DfB7GAyS13wXUm+aY/XPUb49+GYFTiLxrTuaOpAXW4dDD0NMf78eBoLNOViCm7NvIUzA84QP7fyVSWq0kRjzOA9gzH65Gg4zXRCQ0EDgjcFg9VITyi2VLagLrcOkrKSYlniHBZHeL0+ZJ0XtTsKD1c9pL2ubqEOl/kuqMutQ+LpRJGGirKEMsQciEHg+kDacXwuH2k30lAQVkBs7JRRkkFjcSPuzL9DbNRJOZ+C4G+CcX/pfdpz1VbbhohfIhC8MRghW0LQXi9a7CuLLxOyIN8XzrOd4b7MHQpaCrCbZPdRVXYEiNgZgfhj8Xj+6/NuH1tfUA8AmHB5gsjrwmfkb2zun3p3KpanLxe7jmJKMsWu75SNlGHiZyIiwy2nLodh+4fBegyVd2mra0NhZKFwPWM+2FxEmt10gCnmhc8TYdFdm3wNt2bdEnvOfD4fValVMO73YQxhGWUZDD80nLhWthlrg7WFa+EwzYEolT35xmQ4TndEwqkE1OaQWXYin6UkAzUzNUhISdDmcoBap95ZcAeP1z0mzk+doWKkItbXU0JKAoccDomNUR8KHScd9N1INQ25zHN5Z+PM2/DZ5ANjX2OxPtPvg09tb+uz0QczHs1A7pNcNJV+eP7w34SazBoELA/A2QFn0VAgumaVU5fD1DtTsTRpKRa8WEA8PvlCMg7ZHxI2q3bG2YFncWHYBeJxCtoKWJayDN4bvFGdUY3sR9kisf/11ddIPpf8QXnQoueUz6pJfxPifd1vaz9YDLPAmf5naAoLirqKGLxnMMAAch7ndPuzSUg8nYjzQ86jsZjerNQdFIQXIGp3lLBA3xm5wbnYztiO/Vb7u/We2Y+zEX80HrOCZnWpEvQZHwf/dMz7LOP5GZ8MmpqoSVZBWwE6zjpiO9JJHR9+O/zA4/CQfjP9L2luv776Gkr6SsQu7L8CTjsHzeXNGLRr0L+qcNFW24a9Jnvhutj1b/ca6C4KwgvEMlfcl7nj5d6XSDyZCJc5Lv/bE/uM/xk813rCfIj5f8IwmwRBzPuMvweCxJPpQFOx88n9ZfeRF5xH9NUAgPWl64mvv7ryCgVhBRh17E3cZLexccT5CABgadJSlMaVCjfNJv1MRGK/vIY83Je4E99bINElbsP9tpdBZwgY74XPyIygxNOJyHqQBa/1XjSvuarUKmTczYDNOBtoWtOTFK6LXOE43VGsV4nFMAtMuT2FyDQqeFaAq+OvYuSfI4m+fhJSEqjNqkXk7kgM+ple4FiZtRLxx+O73aksgLaDNliNLJTGltIYaz369sDIoyOh66zbbX8Lz9We8FxNZqL1nN8TJv1NUJtdC0VdRZoczuUxl9FY0kj0qVEzVcOagjU47HgY1yZdw4r0Fd06r6F7h76zwCUObXVtqMmsgbq5Om0OlteQ77Ij1NDLEPHH4pF6PZWSEXoLXcU8iyEW0O2pi5iDMWKLJl3hhNcJAGQfCrdFbjTJtc6YeHki0WsNoLrjXRe5Iu1mGjEB2FrTCqM+RmIlg9vr24nNBhE7I9CQ34AJlybQxmqyanBp1CUM+GkAfL71Ib6vkbcRVuetFhvburo3AWCP/hsGytsd2gKM/HMkkdmlYamB/NB8HLA+gDGnxtCYUAVhBahIqcDo42/kNJX0laCkr4TEM4lob2innVtbbRuifosSSgyTZHvZrWwo6imKZcgxJZl4sPQBNKw1MC98HvFv+m7sC8eZ5OKXgM3RWPQmgSMlJwVjH2Oo9FBBfkg+In6OgNMsJxHmYvajbFwccRGjT4wmFl99t/ii+EUx6nLraHGdz6OKNqYDTImxs9+WfuC0cdC69xJSfZcCEL0uu7V3Q9lAGbOekOVz2+vbwWpkQUlfiSb/eM7/HACIsNzsJtihpapFOIcURRWhvaEdClqiBYZeX/SCw1QHcNo5NA9DCSkJzA2fCwUtBbGShmPPjMXIoyOJMoC+m32JbHzg3d49xr7GYEoyxTKEj3scB7uVjXWl64TMGkMPQxh6GELbQRvHe1OsUfel7rS5LvlCMgLXBWLE4RFwmedCO/eHqx7CebaziMdcd9Z5ccfiUJ1eTWSIGnkbwcjbiGJO80SZ0y5zXPBo1SPiM8OUYmLZq2WoSKog7m17r+iN5vJmPPvxGdob2qGoK+rv47/bH4ZehgjaEAQdJx2RdUxLVQuCvwmGy1wXjDo2isZUGvDDAFgMsSDuy9sb2nFt0jU0lTRh9MnRMPSg1iM9fHog8XQiOCwOmsuaEbIlpEuPzg+BjrMOTAeYYuhechzpCgO+H4AB39ObglRNVP92toSClgLtORRA3Vwdo4+PRlVaFSUx+1ajj8+3PrS5REZZRqT4VxRZhEujLmHM6TFd7q2Loopwb9E99N/eH32/6du11DgfSDyZCHYzm7jmzQ3OxaPVj+C51hOuC+jejJKyknh95TWqXpP9jQFgwkX6HApQsrrpt9Nxd8pdjDs/jpg/6gxWIwsv979EyOYQTL07lWYNE3MwRrjGNu5nLFZuuq2uDZn3MqFmpgajPka0AmxdXh0kpCWg66Lb5fl0FxLSEhjwwwB4rvFES0VLtz2bzQaZ0fwZPwSf0t5WTk0O9fn1eLD0AWYGzvzbJHj/lwjaEISsgCwMPzicFs+l5KSEpApxMOhtAGklaVwecxnLUpaJXBNZVVlISEsg9s9YxB6OxeTrk4UNGRJSEkJp2qCvghC1OwqrclcJZcL5fD4kpCXg9z1ZtlzgJbc0eSl0HEX3ep5rPOEy1wVKekrEpiljX2PIqcvBYpgFbf+qoKUAr7VeYu0WPgQNRQ3ICcxBwqkE+Hzr80F2C3w+H03FTfDd6gsVY7q8aFEUVeAUJwcsDgraCjDyNoKsGpmdfGn0JXDaOJgVJN4b+TO6h38y5n0u9n3GJwcum4sFUQtom0cByhLK0FLZgh59ewg3NtIK0hi2d9hf8iwAgOtTKG+LZSnLPqqWenV6NY66HoXfD35C6aZ/AySkJWDiZ/Kv1Cnv+01faNmTizya1pqY/mD6B0mUfcanA7sJdgB5D/cZn/FOKOoqYkniEshryhM7AStSKlAcVUwsXrGaWIg+EA2D3gZCH63O4LK4aK9vF0mEp99KF/5bRkUGPefRk748Lg/fS34PMIB1JeuIMj6O0x3hOF28f05bXRte/PEChh6GtG5nRT1FzA2fK3bDGn88HkWRRajNqcWS+CUiY+VJ5Qj+NhhMKSY47RzoOosmIqTkpMQyxfh8PqL3R0PXRZeYjBK81/0l94nFPp+NPog9HCuU6X4b6hbqGLRTPMvpXVAzV8Oo46OI86+mjSZUTVVRGlMKDovzzoRQZ9Rm10JSVlKsv0ny+WSEbgulqREAgIKOAjLvZ+LZT8/Q95u+xEKj5zpPsTJ64sBgMNBW14ZrE69hadLSbjMjYo/E4unGp2ILs12h57yeXSbiu4KamRpkVWT/dbLiUnJSGHV0FEYdJTdETb8/Xeyx9fn1kJSVRM8F9FhQGl2KvKd58P/VH38Y/wGbMTaYfGMyAIod5zLXBaomqmLfW1JWEul30lEUUYQJlycQi1ddwcTPBPkh+Rj480CaJ25FSgVSr6XCYaoDrdlGSl4K857NQ+r1VIR/H067b1sqW2Dczxj9t/cXvvbs52eIORCDafenIeZgDBqLG4XFvqbSJsQcjoG6hToifoqASX8T+Gz2IX53c39zsQ0YgvPuubBnl8XimowaFIQXwNTPlMZUNRtkRkvYq5qoYsbDGcLv5jTLieaDJIi3dxfeJRb7LIZa4NrEa2itasXY02NFxqTkpN5ZJJCQkQBPUgYAXR7XZpwN1EzViHG38lUlTnifQEdTB83nHADsJtvBcqQlPNeIFsYy72Xi7oK78P7KG1G7ovDq0it4rPLA2xC3BmcwGTD2MUZTWRMaChuInsuNxY14uOohrEZa0a6Z1UgrtNW2IXhjMHr49CAWe4qiihB/Ih4eqzxE5ip9d31sYW8hnhdA+eLlh+SjMKIQ9pNEpQVLYyhGwIgjI2hJUgDCovaDZQ9gOtCU5g0YvT8atuNtRYp93UHV6yrq+RHjDwqAmLyXUZYRew8xGAxo22uLLUwAFCvCZ5MPsfCqZqYGj1UecJnrQnteVI1VsShmERR1FYlzoISUBM4OOgvbcbbC2CYAu4UtlGYsiiwSFvvqcurw8o+XMOhtAE07TXBZH7dbns/nw3acLdHb6V2oy6vDk6+eYNiBYZSP8DuY7h8bYd+HIXRrKOY9m0cs3CadS0LsoVhYDrckrtUeLH+A+tx6YTx7G1r2Whi8Z7DQg/TOvDuoSK7A4jiqIakmswYpl1IgryEvLDiLY5YLwaCaj0pjSsHt4NIS9UWRRah6XYV7C+8Ri30AYDveltacBlDrr6KoIpgOMBW7BjP0MsT0gOm09SwJCacSELI5BCrGKsQ19JL4JcLmhq5k9GuzaoUy1Vs4W8CQEJ0jo3ZHoSajBvOekRtSPhTlieVoKmtCUWQRnv34DAujF3bpkfY20u+koyK5An029Pkg+fhPEa3VrdCy08KU21M+evH1n4LLPBcY9zMmqr8IUBJdAgaTIXzWO8PIywg+m3yQcDyB1uAmiOMv978Eu4UtMs7j8NBe3w4pBSnYTbSDhpWGyHPivsRdbJNrZ7RUtABvbYFlVWTfKa2r7aD90X0wxaHfln6oz69H6NZQeK72FNsE2yX41B7RbpIdMUfRe0VvOM1wIso2dwUtOy3kBOYgJzAHthNsMfm66Nybee+/Z43z/zM+y3h+xieH+GPx+EnhJxSEFRDHj7oexYWhF0S8dxqKGlCdXg0+j0/0ihEg424GdmvvRvqddOK4xTALAOh2cu1dUNBRgO8W34/OGPyrkFGSgYSUBCJ+juiWf8n/AkZ9jIgJAgEsh1sSF//vAp/Hx3bGdpzx+7jyGZ/x8VGdXo39lvsRfSD6nz6Vz/gEISEtAQ0rDUjJSxEZOc9+fIaK5AosfLmQNtZe346nG58icH2gsLuuM5xnO2PIniHIe/rG80DdQh3uy9xhOtAUe032IvDLN5Jaj9c9xj6LfeBz/29+4lPMmmc/Pev299qlvgvhO8KJXpYCFoq4gtXYM2Phs9kHPpvobCGLIRZYFLsIaTfScNL7JG28paoFD5Y/wOtrBClNPvDi9xfIepBF/Fw1MzX4fe8Hry/Fd1YuSVyCWYHkTsPEM4l/qQiUH5KP0thSsUXQtpo2nPI5hZf7Xoq8zufxwePwxK4r9lvux+9GvxPlfwqeFaAqtQoDfx5I3KyNOjoKjtMd8XTTU3DaRX3T2urakHgmEXquel0y0sRB1UQVxv2MPyhhI5BIVDaiJ8+aK5qxU3Un9lnsE8u8kVGSobEY3xczHs3A+jLxxZyusLF1Iza2biSOpVxMwQ7JHUi/TV77nexzEtsZ21ESQ5eH5HZw0VzeDHareCm60thSvLr8iuZj2FbbhrSbaUQvLe+vvAEAvxv+DttxtpRX4v9BQUsBSeeSkHpVvFQ5u5WN15dfI/V66pu40glh34dhO2M7jrofJR7fb1s/AJQ069tdyZWvKhH+fTjxvAWwm2iHpUlLaeuw5opmhG0PE3lepeSl0FTahKOuR+GxygNK+kpCVlljSSOe/fAMjcWNWJG5ApOuTSI2WAiQ+yQXccfiiGMBXwQg4URCl8cXhBcg5mBMl7+nOChoK0DLVouWsNZ10YXjDEc4zaT7vwKUh86gXYPE+lYJvKdIPpg1WTUw9DREuve8/yv4iWLMiTFwmuWEutw62ljuk1x0NHVQnfqEwkTfr/vCb7tol/2FYRfw7MdnGPzbYOH3ebSaLtlanliOiF8iUPyimDa/8vlU3Lwy9gpO9qHPJQDFkku/lY67C+iSvwDFson4OYLWAFKbXYuUiykoeFaAxJOJYr3mxGH0idEYdXyUSJLzsNNh/N7jd+QG5aLftn5wmuFETL4ZehhidvBs9N/Rn1joXF++HmaDzchzJKj5ZIfkDiEL+W0M/WMotnK2EguNmQ8ycWvWLeQG56IkpkTkmrfVtqE6vZp4T/N5fFSlVYm1uMgNzsXzPc/BY/OIDSfsVjY4LA7kNeVp84mkrCT03fUhpyGHtto22n1QX1AP+8n2cJguKsUKUM1Y68vWY8HzBXCa9ea5ybiTAYuhlAQxU4IJ10UfVjglIe1mGnYwd6Aurw612bVEv8iukP0wG6nXU/Gb7m/4w/gPkbG2ujbsNd2LY72OfbTzfRuhW0MBkOWmW6tbwefy0e+7fkRp3NfXXiP2UKzI81KXW4dDDoeEeyw1UzV4rfUSFoaZ0kyR37wmswZh34VBWlEay9OWv1djD4PBQFl8GZLPJxPzDL1X9saIwyMwPYDcNFOXV4eInyOI37kwohC359zG3QV36Ws3Ph87VXbiyddPYDnM8r0YW3o99eC51hNzQuZA341eBFE2VIa6hTpUjFS6lEhXM1OD6UBTsU0ntuNt4b/bn8jE/VCwmlj4s+efuDj8Ip79SO0rustSi9oVhdCtoSiNK333H/9HELwxGKd9T8PYx1gsa/ZTg81YG3itI++16nLr8LPyzzjucRz3Fon3r+v7dV+szFpJaxASwGOlB1ZmrRRh0dVk1mC31m5E7IyAQW8DuC50FdkL8Dg8FL8oFiupu5W7FVu5W4ns0rbaNgRvCsalUZdoctEAcGnUJRy0O0h836rUKhxxOYKAlQEoCCfnlj8EvZb1wuQbkz+8MM4Avkj9An2/6Utkv8upyUHNTI3YhNPl23aax9Nu0D2sh+0fJqJK9BmfNj4X+z7jkwCDwYCRESV1INBdL35RTPxbm3E2kNOQE+nievL1Exy0PYjALwOxg7kDodtDicfKqMhAy1ZLbDfejIAZ2Mbf9tG7e5T0lOC3ww8Plj3AzRk3aeN8Ph/sVvY/UnCzGWeDUcdG/SX5078Dt2bdQth3YcSxgBUB2CG5A63V3Te0FWxG/23sgXeBy+Yi6Kug9/Id+K8gZGsIarNrwZT6701lnWPeZ/w94HF5CFgRgF3qu0Rk2QRwXeiKEYdHEH8DRR1FLIpdhIqkCpGiXWekXEzB041PhYUgg94GGHFohHAz8fy357gz/w4AipkhrSgNMIA1BWsw7tw4KBspE+/tyleV2Km6E5G7yJ6lDlMd4DLXBb6bfGljfD4feSF5SLtFX+ADlNTTgO8H0LwFAYqloe+mD4/VHsJCQGcknaG6xh+uoPsJgQEM3DkQr6++RvZjsjm772ZfDN49mDiW+yQXyeeSxfrb3Zl7B2f8ziDlEt2X6n1QFFWEuCNxxGRowqkEHHU7CqdZTrAdL9rtn3w+Gd9LfS+2E1LPTQ8AiB5hOYE5eH3lNWzG2ohlz/f7rh8Wxy+mKRk0Fjfiztw7yLiT8T5fTwQcFgfPf30OXRddsd3uXcF3sy/GnB5DZJd0NHWA1cBCXU4dUZoy/kQ8TvmcIq7fGAwGFFoVxHryxRyOwR6DPShPIntkvgtdMU8LIwrB5/Lxcu9L4rigoC9g9nRG8Yti/Kb3G471OkbzagOoJrKLIy7ixrQb6GgSvSZ6rnqY9WQWMXkvaGZynOGISdcmoc+GPiLj/rv94TLPhXi+AFW0Kn5RjFHHRhGfm7wnVCMCyXsRAHQcdTD+4nho2WnRikyWwy3xReoXCN0Wij0Ge0TG2G1shO0IIzYbAFTTw4RLE0Q8lDxXe2JW0Cy4LXGDoZch1MzUhM+itoM2lqcth/sSd2hYaqAqtQoPVz0krnUaihpwzv8c7i++T/zs3qt6w28HWSJKgILwAsipyxGZGaVxpbgw/ALijr4pJjYUNSDo6yAUhBego7kDzRXNxIbA8efHY9zZccTPvDD8AvKC84i+U1w2FxdHXMQe/T1CCdPOSDyViAtDL0CqXbxM0OXRl3F+6Hna6zbjbDDj0QzMeDRDrBf5g+UPEP5DuPD/TEkmdJx1YDvBFhE/RQAABvxEly1Mv52O4G+CccLrBC351tHUge+lvkdJdAnclpKbFWIPxQIAca65NOoSbs64iRWZK9Dna9HnIi8kDzdn3IS2vTa+rv+axvp7deUVtjO248q4K8TPVTNVg+sCV6G0GEAx9nQcdZB+Ox2t1a1ob2gXuycyHWCKflv6EWO6oo4iXux5gfAd1PV8e53H4/DA5/LF7m8B4Nqka9ipspP2emFEIZLPUzKix3sfB7vlzVyWdC4JB20PCn2uO4PdysYhu0PYo7+HVhABqC7/4G+DkROUQ2zguD7lOnYq70RFSgXNm5bH5YHVxELIlhDs0tiFuhzRgvOLP17g9ZXXREY9g8mAoq4iDD0NRZLs1RnVaKlsQWtVK2IPx6I6XXzDQXchuO5JZ5Ow33I/rk642q3j9Vz1hP8WNAcLUJ9Xj/r8eppf1MfEyKMj4f2VN03iDqAY0nF/xoHBZBCTwwLliSWJbxQd+Hw++Fw+TRZWgFF/jsL8yDfy1Ma+xliavBS12bXYpbELla8qcWXcFRx2PIzDToeJDQd8Hh8u81ywOG4xkZUqpyYH96XuYqVamRJMqFuoE58304GmmHp3KkrjShF7JJY2btDbAGrm7+8/Z+xrjCF7hojEhs6oy6ujCsU5tYj6NQrhP4TTGrUAQF5THrOfzMbs4NlExr3ZIDNw2VzEHI5573N7F+rz60X+L60kDWWD7q3/nGZTRfeajJoPPo9PbW9r5m8Gz3WekJT7bzEZ447F4aDdQZqHuoSMBIy8jGDS34Q2twoQ+2cszvidITZDxP4Zi6RzScTjZFVl4bbUTSybtCK5Aie8TiBoQxBxnMFkiLVSiPo1ChE/RSDzfibRA1bVTFXYvJAbnCsyxuPywG5hI+ZADK5O7F7MF4fK15VoKm2CxVCLLn3ZuwKDwUDq9VQcdTtKnOfa69uReCYR8ce753FYn1ePPt/0wdywuViZTbco6b2i9werD3wGHf90zPtvRa7P+M+CyWRCQ4NKDJgNNIOsqizygvPgs5HOPphycwrtNdvxtsi8n4kXv78AALy69Ar9t/Wn/Z26hTqG/D7kH/MAU9RVhJwGfcP9ct9LPF7zGDMezoDFUAvCkX8P2hvaURJTArNBZh+1w+x9kHAyAc9+fIZJ1yaJbKCA/5MCqGunSV8IoO2gDT6XjwvDLmBRzKJufS5TgvlJ6rIXhFEmvnW5dTRK/n8V0grSUDJQei/Zh08NnWPeZ/w9qM+vR+LJRAAgNniYDTIDt4OL578/h+caT5FNuYS0BPTd9DHh0gTisZWvKqFuqU6UQ1qTvwaSspK4NPqSsBt50M+DhF50Kj1U4DTTSSwLpCC8AKwGFiJ2RqDPV/TNGMnjS4DiF8U4O+AsALJ3mSApK6MiQ1uYcju44LRzYDfBjug/YORtBNOBpvDe4E0bYzAY0OupB/PB5sSu8uxH2Qj8MhBD9gwR8YcS4NXlV0g4kQBDL0OhnFdnDPx5IIK/Df5gM3SvdV6IPx6PmEMxNE9ACSkJKOgowGu9F03qSZDwrXxdSfS5mHxjMvKC84jn7L7UHTZjbIheDABVZOS0cYhSO6rGqph2bxoif4lEaUwppt2bJjKeFZCF61OuY+TRkXCc9pbeDZ8qIJDYrO8DfXd9orQPQMmhrilYg46WDmLyrSiiCIURhTjhdYJ2/zGZTFwZcAVcFpd4b8prykNSRhIVyRXdkp4SYKfqTmjZamHB8wW0scG/DkaPvj2gaUP3ogSAb5u+RX1BPVFWV7BWqEqtQmNRI+17h2wJQUtlCyZdnwRpJXqseLn3JbIfZdNkfdtq2jAnZA5M+pvQjskPy0fs4ViYDzZHUVQRvL/0phWnVE1V4fWlF00eVoDZT2cDfIj1Egn/MRwv9lBr5sXxi6HX8806TCCZZD3ampbg57RxELotFEbeRqjNroXtOFsRFQYpOSmkXk9F2o002E+2F8YZgR8Qq5GF0cdHC2WPpOSkoGmjCR6Hh8aSRpzudxoAYDvBlsZQLommmJcu812I38l+kj2O9TqG1OupYlnCpgNN0dHUQUwqlcaWIvthNrIfZgslbJtKmxC1KwoK2grIfpyNiJ8isCJjhUgRtSyhDDGHYuC60JUYC2SUZMQmsditbGHzGWldajPOBg0FDSiNSUa5eR+87dn3cPVDcNo5cF9KXyepGqvSJEffPjb2UCx6+PSA72aqeUQQa0rjKLYqAKIkteMMRzSVNkFSTpLW2c6UZMJhqgOM+xkTzwsABvw4QPh3b0NGmbpeb8tkApSU6+Sbk6HfS58o6yWQfaxKIzcVXBp9CQo6Chh97I2f5MgjI8Hn8dFS1YKw7WH43fB3LElcQpsPks4lIe5IHEafGE2LJXw+H9Vp1fD7wU/YZPH2Oo8pycTs4NlElhIAYUHNuJ9oowWHxUHkzkhYDLNAry96ofJ1pUiCUVBMSziRIOLBBlCefXYT7ZB6PZXI1vZa74WGggZcm3iN+J3N/M3QWNKII05HMHDnQPT9uq9wrCq1CkecjkDXRReui11p38t6tDWUDZSJawIOi4OKpApIyUtBzVxN2KgxJ2QOmkqaIKMig4E7B6I+r554rT4EHqs8wOPwhI2kliO65wWoZa+F6QHT0VrVCvspoixdNTM1TL4xGZq25DnmY6Arlr+amRrmhs0Vu94Y/Otg+G7xFZkP1M3VsTxtufD/ZfFluDP/Dvp83Ye+rgD1XOo46qA8sRw6zjqQkpcCq4mFyleVACi2br8tosV7DouDM/3PwGmWE7EZQuApqqinSGTTNpc3ozCikGoceEtWXMVIBSpGKlgUvQh4K7wyGAzMCpqFjHsZ+FHuR4w+OZr4nd5GeWI5og9EU7H8Ldb6YYfDtIYx7y/p6+H3QeLJREgpSMF7/Ycd/zaU9JUw8s+RkFaURnt9O7Gx5F1wnO6IHn16fFCTmACf2t7WfpI9eBwedmvtxsQrE2E1wuqfPqW/jICVAYg5EAMtey2aKomygTJmPp7Z5fHN5c3ID81H7JFYeK/3FonfET9FQEFHATqOOiiJKYHtOFvh2lRJXwkjD1N+s89/f46YAzGYFTRL6FMsaAQV570esCIAnHYOBv0yiLbO7uHTA94cb/T5qg+xUWvY3mEwG2SGoA1BtOYFHUcdrMxaifQ76Wivo7MCPwSJpxLx/LfnWJ2/uss1Vlfg8/mQ15CH4wxHYm446rcoPPuBYul2pzhX/LIYkTsjYTuOvoYGKNUELpv7r7Rw+hTxT8e8z8W+z/gkwOVykZWVBUtLS0hISGDa/WndkoGym2gHPVc9lMSU4PHax7RFuADRB6IRuTMSK7NW0nxs+Hw+djB3AABWZq/slm/Pu1CdXo0HXzyA2xI3OEyhb2yzA6gJsLmi+aN95vuA3cJG4slESCtIv9ci+GNCQloCMioyxEQUj8uDjLKMWBlP96XuKIwoJE6O7wKDyYC5Pz3Z/G+Hnpsexl8YL3aR9F/EmFNj/ulT+Nvwdsz7jI8PGSUZuC5yhcVQC7H+GikXUpByMQUeKz2ATj8Dn8dHR3MHbMbZELukBUlfqygrYTI7414Gks8lY8APA6Ckr4T5EfNpxwFUh6KirqLYYp/1GGswJZno4UOWfebz+WirbaNi6FvzpLyGPBS0FWA9hmzAfs7/HEpjSyGnIYd1JetEvlvyhWTcnX8X0+5PI254jbyNMPvJbPI58fhQM1PDuHPjoKAtKoUTczgGAV8EAAAClgdgZRa907DvN32RcCIBt2bdwspM8njfb/rSXn9fyCjLQMNKg+ir0FXhVduRkrMiJaEAKpkuzrNG2UAZecF5uD71OiZenkgroMUcjEFTaROcZjpBWlFapBggoywDq5FWiD8ej+Zy+rqA1URt1khMNklZSYw4NALZj7LRVNrU7caW4pfFCFgeAJ9NPjRfIwaD0aW89ogjI2DU14gos8PlcmE8yhhSbDL7zn6SPVLOp+DBsgdivXu6AquBJZYxIyUv1aUPprSitFhPK3ULdazKWYW63DoiS2DY/mHgcXkw9aNLdrXVtqGHTw/YjLOhjT1a/QhtdW1YnbsagV8GQtVUFb2Xv0nUS8pKIuYg1flvP9meFsM0rTXhutAVzWUU0+zttdS7PPwELLI+X/ehPbPsNjZYjSz4bvGlxT8ZZRksTVqK1OupeLzmMbTstETuCR6HB5txNjD0MqQs5hiUBGF1WjU8VnvgzIAzaKtpw+q81QCoZHB7XTvaattwyP4QAKrAYORlRDtnIy8jTLk1RWwxGqCKoOKKKQDg860PWqtbif5RNmNtwGpkicQJXRddrMpZBVk1WRRGFKLXil604kV9fj0SjiegMrmSKAs9+cZkHOt1DEFfB8H/F3+RMSk5KUy4PAFqpmpEFqZBLwPIacpBPyscVT3c8HaxL+F4Aox9jcXKdgnkYEf+OZIWC3IDc6FuqY7ZwaJxPe9pHh6vewx9d33UZNagtaaVxk7VsNQQ62MpJS/VZVMKALExFwDGXxgPHpeHxuJGSMlLiUhmqpqoQtVEFe317SiLL4OqiarIuPNsZzjPdhb73gKWtvlgcxHPPgaTgeRzyYg9HAsDDwOiTCergYWiqCIc9ziOxfGLRfaJPA4Ph+wPwXG6I8ZfGA+Avs5jMBld+klG/hKJgvACbGrfJPK6oICpqKcIq5FWsBopOj8beRlh5NGRxNgsKSOJSdcmif1MFSMV9Pm6D4z6GhGbHTxWecBukh3ij8XTrCjk1OTgMtcF1mOtYTOGHuNM/UzxdNNT1BfUY/j+4SJjzWXNOO5xHABV+BU09xY/L8bZgWcx/NBw5AbmojyxHAN+oDNLPwTSStLCpuBvm7/tUoqRhCffPEHsoVisL19Pi4uyqrI0ZYCPjbKEMiSfS4bLXBdak4e0ojRyAnPwfM9zLE9dTvM8VdJXQnVGNcoTysXOhYJ1L49NNQq9uvwKHS0dwvmYy+aC1cCC3UQ7OM+inrHZT2aDy+aC28ElytkxJZnovbI3FHQUwGVzaWupyN2RiPgpAtKK0vi26Vva8fUF9dRnd3DB5/OJ7AlBMYEEBS0FmA40JbLr38brq69xfcp1ABTL7+1in+daT3DZXDCYDBh6GEJOXY7YVFUSU4Ljval7e2PrRtp9FvZ9GDqaO8R6J34I5DXkhcXQlEspuD3nNkYcGQEt2/dvbufz+FDSVyIW598Xn+LeVlFHEcY+xv+Z4kdHYwe07LTwxasvPuj4/tv6g8fm4dkPz+A000nkfph6ZyqYkkxk3qcY4XquesT9taSMJLUO6/S42oy1gUoPFVgMIZMaBOtdm7E2tDnOcpilWPavANajrGE9irz3BUCcoz4UDtMckBuUiwNWB7A0aanYRsKuwO3gImB5ABxnOBJZuD369ICxrzGc54pfz5DgPMsZ6bfS8eCLBxhzcozIXMHj8ITKAe/yiv6M98M/HfM+F/s+45NBezuVHMoJzMGrS6/QdyM5sfez0s/oaO6gGQ+rmalBzUyNWEwTQNteGxIyEqjJrKEV+zr73pdEl3zUYl9HSwcqkirQWkWWney9sjeUjZS73AT+HVDUVcTcsLm4OfMmtB21P8gb6EOh7aANu4l2RNaMpIwkvmn4psvjx58f/0Gfy+fzUfW6Ci1VLcTk3L8VcmpyXSYr/4voaO5A3LE46Drr/s+fjf8FBDHvU0Dxy2LkBObAZa6LWA3/fxsUtBXEJiMB4OGqh3h15RUWPF9AS5TX59djn/k+eG/whv8uf9qxdhPsoG6uDiV9JWECoiajBqnXUmEzzgavr72GsY+xUAox80EmajJr4LnGUyhhkhecB/sp9jQ2t7KBMq2DWQA+n4+b02/i1eVXcJnrQiuIa1hp4MuKL8V+Z5vxNiiNLYW6uTrYLWyRhJWGpQZc5rsg/mg8nv34DAui6AwpPp8P8EFjqXBYHOy33E/s3hYU+uQ15dHeQL7n1S3UMfzQcJF5+GOivaEdY0+PFdt1z2pi4VTfU7AZZ4P+3/UXvm7QywCb2jaJZUddHnMZmraaGLRzEG2Mw+KAw+JQrHkCqWf8hfF4uOIhdqrsxNqitcRu6qm3pxI/tyajBlwWV6xEVfHLYkTtjoLDNIduF/uCvw1GWVwZUfqW3cpGYUQhZJRloO+uT7sukjKSXRbqHDc5wtFR/DzWc0FPmAww6db5CjDp2iSxzM+63Do0FDbAoLcBUUos424GOO0cWI6wJKocCNaXJJDkTgUoiy/Dk6+eYMThEbQxx5mOCFwXiFO+p1CeWA4jLyNhsc+kn4lI4lTdkrwejdwVicSTifi6/msay6k8sRwxh2OgbqFOkwgFQHy+BXh1+RXuzr+L6Q+m0xgCTEkmdJx0oKCtAPPB5tCyF00mlieW49bMWxi4c6AwTry69AopF1JQFFlEzR98St2h5/yeKHxWiHP+5zDgxwHwWO0B0wGmRBYtQCWsbcaKT9jcmnULrEYWJl0VX9yI3B2J57/SO7JjDsUgYHkAFsctFlGbkJSRFP724pJJViOtoGGtQZNS64zW6laazCtANb51tW8BAM81njgT5wCOND2ptrZ4rdjC7vM9zxG4npKhJkk0Lk9bTkugJ5xKQFFUETqaOtB3Y1/cW3gPhx0Of1Bi6Mk3T1CfX4+JlycSx2/OuAkOi0NUq2iva8fvRr/DbambkC3QGblPcnFt0jVMuDzhndevM+ZHzcfDFQ9FCr1PtzyFgpYCeBweLIZZYMShEcR1Tu8VvcFlcxH5SyRNvo/BYMB3iy+tCNN5ncfn8VGeWA6mFJMoxdhzQU8Y+xrTCiLSStJYFLsI7XXtxCI10DXrqytw2jnQsteCQW8DsexTJT0l9NtKl1tVNlR+Z1NeR3MHOG10qUNZNVm4f+GOnMc5Ik2MtTm1MPQyhLq5Ory+9EJ1ejXKE8u7ba8RfTAaMkoyIoXf2qw30sAfomYjaAK4OuEqtOy0aOvLkpgSsFvYRLb2x8CNaTdQk1EDQy9D2n3G5/OhaqIKk/4mNFlwgGLQPfn6CSqSKoR7ydbqVry+9hqGHobQc9WDvrs+VmWvEh7z4o8XaCppEs7phRGFODvgLIYdGCbSmCIhJSG2IUpCSgJ8Ph8hm0PgusCVVnQTFJDfLiQLIGANfvH6C1qhL+FUAh6ufIgxp8bApJ+JSNMKj8tDyJYQ6DjrYPp9sh/g2xDIkM4Nm0sr9AEQKTonnknElXFXMPXuVNqc0Pn55HbQm7LYLWxIykpCQefjesQd9zguZL8DID53XeHWzFvIvJ+JjS0bieuk98WntLeN2BmBwohCTH8wXWz8+9Qw9sxYsWPN5c14sfcFYg/FwtjXmKYaIoDLXBcY+xrT9iWCOCylIAU9Vz0R9n1NZg1CtoTAeY4zen3Ri6ZaomWr1WXxeerdqSiMKKStKQWoza5FfUE9jLyNaM9U3NE4sJpYRKZsc0UzMu5mwNDTkDjvfgj03fThtd4LSWeSurSa4fP4yA3OhdlAM9r9xWAy0PfbvmLnNouhFh+k9iYhLSFs5jvifERk7SbOf/4z/hr+yZj33zM6+oz/PCpfVyLxdCKxmx2A0Cems19M8MZgnBt8DpWvKxGwIkBsd7e6pTq4LC4qX1fSxhhMBtaXr8fMwJkfvQik76aPr2q+goS0BBJO0T0VrEZaYfTx0f/zJDqDyYCMsgzk1OT+0sLuQ1AaV4qnm56iJpOsDX/C+wTODjpLHEu5mCKULuou2C1sHHY8jLMDztJ8av7NSLmUgmuTrqGpVLxvy38NWQFZCFwXKNZ/7DP+dyiKLELo1tAPllH8p5B6PRU3pt8gPjdMSSa07bVh6GlIW4RLK0nDZZ4LonZH4eKIi7RjVU1UkXEnA3tN9gq9c7y/9MZW7lY8WPoAIZtDcLrfaaHHXNLpJASuo5Kuvlt9oW6pjsTTiUi5QPeg4/P5aCprQktVC32MxxdKqxn3F19kEAefb32wjb8NC18upDEXevTtgTEnxkBBVwENBQ00KZSks0nYwdyBfRb7aO/LlGDCYpgFCiMKhckZAcacGgOXuS5YmbUSa4vWEs+Lx+HBdaEr3JfRJd+4HVxsZ27Hbq3dqEip6O5XBgCEfheKw46HRXyOBCh+UYwXf7xAQ2EDzaepMKIQD1c9FDtPZdzNQOQvkXh99TVtLGRrCO4vvk8xkdzoTCRNa030XNATLvNdaH4h5Ynl+EnhJ0T9FkX8XIthFhh2YBixkCf4DmNOjfkgyXKLYRbQd9cnMm/q8+txfsh5nPA6gYbCBtp4XW4dSmJKxHoAZRzPQNRu8nfKD8tHaWwp7CeTlRluzriJ7YztyLhL9jG0m2gHzzWexLGo36Jwxu8Mrk+9Thy/POYyrk+5jtjDdN+fipQKXJ1wFTmBOcRjAeq33qWxi+ZxomGtgeEHhxNZuoJkYuGzQqzKXoWJV0QLIqxGFswGmWFexDxiU1RJdAleXXwF2/G2REbFg2UPEH80XijV2R1o2Wqh14peyLibgcANop6lfB4frTWtkFaSRo++PWhecPJa8nBb4iaSwPDb4Yfeq3oj/XY6nGY5QUlfCQ++eACAKhj0WtELxr7GGPrHUBj3M0Z1RjXRExIAbs2+hR2SO4ieauw29jsTnGVxZZBSkKKxfAUe0NUZ1SL+2Zx2DuoL6sFqYol9TwkpCUy7Nw1zw+YSx+NPxMPvez+MOEQv+gIUk+7KuCtEv6+IXyJwzv8cOmSVwZeg/85yanJ4uuUp8d5WNVWF9WhrrMhcQZTEBIDyhHIRn8zQraGoelWFpUlLhfcdaX8Q+l0otjO24w+TP2iqJB3NHbi3+B4if4kU+mF2RltdG/Zb7UfKxRSi93bCqQS8uvIKHms8aI1eccfi8IvaL2hvaMeAHwfQkndVqVW4POay2Dhh5GWExXGLRbr8Yw/FIv54PIK/DYaKsQqNFdUZXmu98GX5lzQmMFOSCb8dfojcFYn9VvuJx7bWtOKo21EccTpCHDf1M4WUghQCNwSKMIYYDAaujLuCc/7nELItBMd6HxNZH7y68gpnBpwh+v6wmlg4YHMAOyR2IPdJLm388brH2Km8U6wHeuTuSNxddJc4JkDazTTcmHYDDUWic0LELxGQkJLAoF/ozTCyKrIYcXAEVmWvElESiPg5Apx2DtitbFwcfhGB6wLF+iZ3hYifIxBzSNQT7fG6xwCAafen4fcev+PRmkfdek+zQWYAqPVw8rlkkbHCyEIc730cZ/zOdPtc3xeKOlShjKS8UBZfhnuL7sHY15jI0Ly36B5KY0ox+cabwnpDUQMCvghA9iOyz/Koo6NE/l7ZQBluS9zQVNKEkK0haK1pRer1VBxyOITtjO2oSCavz6xGWKH/jv6QUqDHEcthltjG3yaW5WbqZ4r+O/oTmXmKuoro0bcHbky7gWO9jomM8bl8RPwc0S3f477f9MU2/jaq4P4OHy55TXnYTrAl+vvpOutiG38btvG3EaWGB+0chHnP5n00SUGA8kPrXOhjSDBoFinvgkCGvLseYZ8yql5XIS84j9iQ9ylDIEfbVCa6922pakHkzkiwGlli1+mlsaUoSyiDSX8TWlMEu5UNDosDNVM1mA82F1lHtVS24PXV16jJIu+XuGwujrodRfCmYOK49Shr+P/iT3ym4o7FYb/lfpwbdA4tFfS9ccKJBAR9GYSYwzG0/Vptdi3uL76PI05HcNTtKPGzPwTOs50xO3h2l+SQtFtpOD/4PDLu0eOQhJQEDHobIOlMkljZ8Q8Bn8fHoF2DoNtTF71WiBZcJaQkhLHpM/4b+Mzs+4xPDr1X9Eb4jnA8XvsYi2MX08ZJAaqhoAG5Qbk47HAYANW5QuoU1XHUwbKUZVDUI8s5KOooQtH/3VIPH4qo3VGQUpCi+V9UpFTgxZ4XcJnvAmOf7idwu0J9QT1qs2thNtCMNsbt4ILP42N6wPRuGzn/VSjpKcFpphOdYQkq8VsWXyaWFp8blIvE04mQVZGF3w6/bn0uU5IJBW0FyGt9WnINOY9ykHo9FUwpJiZc7Foe6b8CwYZDnDzVZ/zvYD7YHL5bfIkb138rmkqbcG3SNQCA7xZfWmFkyJ4h4LK5YLdRXbadu4YVtBQw5uQYtNe1i22E6OHTA2BCxFuUwWTAe4M3yuLKkH47HXFH4uA4zRF+P/jBY40HGAwG/Lb7of+2/iiKKiJ29r744wUC1wVCSl4KG1s2iowxmAx8Xfc1mJJMYgGgJrMGB6wPAAA2szZ/kHH4qD/JbMi22jYAIHY8S0hLwHmOM25MvYHq9GpoO7xJhLrMdYGhlyFSLqbAcrglMZF6Z/4dJJ9LJncUMygmeGVKJRJOJGDoH0O7/Z1sxtgg7XoasgKyaB5R2Y+yEbY9DMtSlomcN0Al0OKPxcPQy5BYOJsfOR8PVz4kXmd9d324zHUh+rgBVBLWaqQV0bNKSkGKKnqkVSPlUgpNZltOTQ4NhQ2ozaqlSQ+xW9l4uvEpei7sCZe5LsTP7gp9NvQhMsEAQEFHASb9TdDR3EH8XqHbQpF8Phl6rnpYHEdfv6UfTEc60uG9zpt2zQqfFSL8+3DYTrAlJitzg6lENYkdyufzEfdnHNQt1IUJ2c6wHWeL2EOxtN9XgKH7hiJ6XzSxw7alsgVpN9OQdjMNC54voN3/R92PoiyuDCZ+JjSmo4qRCmqyavB47WOsK1knIndk6GmIrbytAEBjLJTElKDXF71gNcoKElIS4LLoEmmt1a0Ag2LrkqSGPdd6oiK5QqykYWFkIdJupKEqtYrysu7UcW3oaQhDT0OcHXgWZQllGLx7sMj1+E3vN7gvc8ewfcPAYDJEmiVUjVWhpK+Ex2sfw/C5IWRVZKFmpoYhe4ZgyJ4hYEowoWqiCo81HgAATRtNEYm/076nUZFcQZQTTr2eiuRzydR1JiTnJl+fjJjDMYg5HINey+hemADQWNQIdXN1WpGy39Z+kFaSxs3pN3Fz+k3hXqPoeRHODqBkBWVVZZF2PQ1D9w0VWTO31bWho7mDmKQCgGc/PoOcmhyxgN5W24azA6nGtp4L6JLAkjKSkJKXAoPPAwgydhXJFch+mE1k99mOs6VJ8XZG0fMinPQ+CT03PeF+a/KNyZCQlkBZQhluTL0BdUt1ojSpqqkqAGr/9XbjHaedg/hj8TRWXlFUESX5bK4GCSkJmgecAM9/fQ4+n4/lqctpYwpaCtBz04NBLwOiX2VBeAEy7magMKIQX9V8RRt/deUV5NTlROT8l6UsA5fNFRYfr4y7gn7f9aP511UkV6AipQJWI6zESt318OlBbCoBqN+SwWQI5aFJyA7IRsrFFMpb9v9CJLeDK8K0bqloAY/DE/l/fkg+rk+9jqWJS0Xej8fmoSaDSn4WvygWxsfQ70IhoywDI28jJJxIwMm+JzH5+mTaNc17koecwBzUpNfAea6zCHO7Pr8eIVtDUJ1ejdKYUvTd2FekaZXdyhauG94Xg38dDAlpCahbqMNjtQeqUquIc+S7MO7cOFrh33q0NTSsNPDq0is0FjWKbQwWB9MBpvDZ5IOsgCwaM1qQFLebaNftc31fiGsmAKg1gf1ke7H7ZvOh5lDQURCRx1O3UMfsp7OFcau1uhVpt9Kg764PvZ56tHtBw0oDI4+MRNDXQYj4OQIuc13wdPNT4f2VcTeDdkxHcwdCvwuF9RjrbtmzCGDS3wSKeoooiy+DcT9R1qtA2i/q1yhaMwZTiokVmSvQUNiAwC8D4TDNgdZ09eznZ2BKMEW8sTuaO1CbXQtFPUWkXkul/LTU5MDn83Fx+EXIa8qjpbIFOYE5kJCRIOaa3gcXhl8Al8XFiowVH3Q8CTpOOtBx0kHy+eQPkr71+94PsqqyXcan/xrGnRsH76+88eTrJ3Cc4UiL+Z8ish5mIey7MJREl0DPVU9kPa1hpYGV2SshqyorVrY09kgsEk4k4Ov6r2nr9L2me6FhrYF54fNox/Xo2wNbuVvB5/NR/KIYRVFFcJrlBAUtap8bfSAaZfFlRIlsAGgqa4KkrCRtbQYAdTkU69Znkw/x+AmXJyDtRhoCvgjAuHPjRCTRte21MfPxTLz4/QXa6ro3H4nD0y1PkXgqEUvil9Bk8DuD084Bg8kQq77QUNSAvJA8sBrozWSJpxNxZ94dAN3bz7/Y+wLPfniGL1K/6JaM72d8mvhc7PuMTwJMJhNmZmZgMplgSDDgMM0Bsmrvn1Qef2E8ei7siWc/PEPe0zyiXwxAFQ9eXXqFKbem0MZ4HB5Ct4dC10UXtuNsPyqdv7W6Ffmh+fDf7U9ciL/c+xKJpxOhaqr60Yt9R5yOgNXIwtizY4Ua+wI0lTXhqNtReK33wuBfB4t5h78H5YnlSD6fDK/19EIOu5UNLosLPTdyV9qQ34cg62EWks8ld7vYJykr2aXM3b8V/r/6ozabkrf5r4DH4SHq1yi4LXYjLt76f9cfvZb3EptA+5TROeZ9CqjLq0P49+FQM1f7ILbQPwFBUaDPN33EJkAeLHsg3NSQCpmkuQKgut1DtoRgQdQCoZxIbXYtGosb4bXeC1JyUiiLLxMWrjStNQHrN+fF5/HFyhYJGBiui+hyiAwGo0s/jc7MIlLXZuCGQJRGl8JypCWcZjiJFEDznuYh8VQi+nzTh+hf5rnGUyxzCgAsh1tiZdZKWgc2n8dH4bNCBCwPgO9WX/htp8dsQ09DJJ9LxtWJVzEjQLTDW0JKAgtfLETq9dQu773qjGoo6SsRE0rWo63R0dKBV5df0RKHrgtdYeZvRpRpNO5Hzced5b86w8jbiFjUAigPOm17bSSfT4bNGBva+5/scxKcdg7Ro1DDUgMzAmbgjN8ZpN9OpxX7GgobELUrCqomqrTik5SCFKbcmgJJWUlw2jm0IlFVahUif4mE8xxnojxyc0UzsgKyYNDbgHYfyGvIY07IHOL3BQD7KfZIPp9MVGZgMpkwHWqKvEd5YLeyaRtX96XuKE8sx/Up17E0aSmtgLW2cC14HB6NBQlQ99iDZRRTjNQQZjbIrMtOVo+VHpRvJwFmA80w9c5UJJ5KJD57em560HPTE1sk17TWhPkQc9qa8ozfGUgrSGP6g+moyaoBU5IpnOtyg3LxdNNTVCRXIOFEAmY8nEGT87EcbolNraLeXp1hP9leLEsSAB4sfYDKV5WQVpQmyksCwKTrk0SKCgAgKScJ18WuqMmowfdS32Pm45kwHyzqgyyQhuRzqRjEbmWDwWRAUlYSQV8Hgc/l09acLZUtuLf4npAdQuqWllOXg9kgM/T7rp9Y2bjo/dFgMBhii31zw+eKlQsWJPrkNOSE30HZUBme6zyh66yLrIdZSLuVhgE/iSZScwJzcGPqDYw4MgLuS+js5Mk3JuPV5VfICcyhXSsGk4EePj1gPcaa5lMDUHG3pbIFlT/vQ0r/FYCWKEPvqPtRWA6zxNQ7ZMnf2pxa1GbVwqiPES023ph6g/qMtW/iukFvAzSVNgkVFWqzatFW00ZLwLnMcYHLHBfiZ8qpy1FqJm/JCT5Y9gCKuoqY+Xgmvngt3k9owqUJYuWmbMbadCnlajnCEo7THWE7gVzkFHznRbGLhMl/wTyoYamBpxufAgDcv6D/jmm30hD2XRh8NvnAc62nSLKU3UYx0SyGWQiLB2+v82SUZbCVu5V4Xs0VzTg/5DxaKlqwpmCNUBqMz+cLfct0e+pi0M+DqEJgJ3is8kDqtVSiTLWcuhw2szajqaxJGL+4HVyEbQ8DQMVLDouDZz8+E2G0CjDt3jRUp1fj3OBzaKsRTZS2VrdS+7Af/LAgaoFI4xMA+G33g7qFOopfFNMK9w2FDTjZ5yQaixsx8cpEYazScdZBwBcBcJ7jjMaiRuQG5aKxuFGsN644hGwJQV1OHdaXrRe+5v2lN+7Mv4OUCynw2ewDn299uvWe0fujUZNZg4UvF9Lij66L7nuxJditbMqX/gMKX+w2NppKmyCvKU9br6qZqaHnwp7ICcyBoachbR1GkteWUZIRUTJqLG7E/cX34feDH/R66gn96d5OVHt/6Q2XuS5QNlTGqGOj0FDYgKbSJlpsAyD03hRX9E04mYC7C+7Cc50nhvw2hPg3z/c8R/zReHxV8xVxr+j9JV26j8FgQMNSA9Vp1Xj+23PouujSin2CZ13wvNZk1iDxdCIifo6ArosuyhPLkf0wG9MfTAf4QMGzAqj0UEFjUSN0e+rCYhhZYq++oB6h20KhZacFj9UetLVMflg+AMBlngvx+A+BjpMOliZRhX6fTT4oTypHa01rt3zo1M3VxbLP3xef2t4WoGTxo3ZHQc9N7z9R7IvaFYWS6BLMCZ1Dk8SUlJEkrq3aatsgqyoLBpMB96XuKI0pxS+qv2Bp8lIR9rz9FHso6irixd4XCPoyCPOezRPZgzCYDDDAQE5QDkK3hsJskJmw2CenLgdDL0MMPzic9vkAsEd/DwAQ17uDdg4i2iUIoGaqBuc5ztBz1aPtE2VVZWE+2JwYnz4UsqqyaK1uRej2UHiu9iT6LQOA0wwnSjaZsJxht7KReTcTft/7EZtoa7OpfWd3/QCNvI3g/oU7JGUlaTLtnHYOflL8CZIykrRm4s/4MPzTMe9zse8zPgkwGAwoK1Ndsk1lTeg5v6dYb53YI7GoyayB1zovES1pUz9TmPqZoiC8AHIa5K6Rtto2lCeWo72hnbYQ5rK5ePbDMwDUQuljGYIDQFVaFa5Nuoahe4cSN6mCbu+/w9x78s3JODfoHB6veUwr9smqysJzrSdqMmpQ/LIYhh7/u0KS4wxH6LnqQcWYYCgvK4lx58aJ9SOSVZXFouhF/znZha6goKWA+ZHz/+nT+KjICcpB8LfBYEoxiTrrhl6G4HF4Yj1K/inUZNbgWO9jmPloJnGB9j7oHPM+BWhaa2L4oeEfvRnh74SmjSa2sLcADDpzBgAy7mWgLK4MTrOcaIyc5vJmBG0IgtVoK9hPoifMVU1UYe4vKmES+2csnv/6HKtyV0HNVE1EQkeQRJOQksAvqr8AANaXr4ecmhzt3rYaaSU2aSTw/AlYHgC7yXbwWivaLNGjbw9sqN4AKXkporRf9sNsVL2uQkF4AYy8jUSKfdUZ1Ug+nwwNaw2UJ5TDYaqDWK+6t9HR0oFrE6/BYrgFPFeLFgR3SOwQ/jt8Rzix2Nfri17IeZyDurw64vtLyUuJZSgBFEvuoM3BLhNGC6IWEJuIlA2VoaSvhPjj8VAzUxNhhqn0UMHg3waLfc7T76RDUUdR7HhpbCkC1wVC3UKdVuyzHW+LV5df4dasWxi0axCRzTbgpwHEQkxJdAkkZCRg5G1EG5OQkkB9QT0er3mMec/m0YrKTaVNSDqbBENvQ2KxL/pANJ798AwDdw4kFn27Qlf3LoPBwJg/x6C9vp3ISpXXlIeqqSoqkiqIhWoJaQmx8wCDwYC6pTpqs2ppm9uPAevR1mJ95MQV+QCKNfpy70sM+WMIMUmZFZCFnMAc3J5zG2rmapgfQc3xzrOdUZtdi4QTlOy7Sg+yxPuzn54h+Xwy5obNFSZUOqO9vh1cNpc41n97f/B5fCITJftxNuKPxqPfd/1oMomyKrIY9eco5D3Ng4K2Am0tXZ1Rjeq0avj/6i/8zpfHXEbuk1zMDZuLhOMJkNeSFxb7ip4X4fmvz2E7wRZZD7LgPNsZA34aQFSbMB1g2qV/b+qNVDhMc+jSwy3ncQ6SziRhzKkxIizjx+se48XvLzD2zFiRWKNhqSGMKUbeRhjwPX1vILhGD5Y+IBb79Hrq4ZTPKVQkVdASTrKqssQu+c7Q7amLGgNH8CSl8XYNrP93/cWulXOCcnB+8HkAwNKkpTTWjd/3fpCUkxSZ43gcHqpSqxC+Ixyui1wRfyweQRuCxDa+kMBgMiCnLoeSmBK0VrfCchjl+9hU2iRyzXMCc9Ba00prZtBx0gGHxcHlMZdh4mdCbDIpel6EwHWB6Luxr4hnloqRCsZfEO/p7bvFF+Hfh6MgvECY/K/NroW0krRQgnrZq2XE2OcwxQG5Qbl49uMzWI2yEkmk8zg8VCRXQMvhTaKxO+u8P3v+ieayZui56Yk87y/3vcTjNZT8JEkOU4B5z8TfQxLSEiL+lAKJXMGe03WBq1ivVQlpCeg46eDLcnqTpJ6rHja2bgRTkil2rfB4zWPoOOvQin08Dk8oC1+eVC4s9nHaOCh6XgTTgaawHGGJtJtpH8TssxppBVYjnS0hkFT2+dan2/YVecF5yLibAefZzpDXlP+g9f9+y/1QMVbp0jNVHF788QJPNz4V21RQ/LwYz399DufZzkTZyzsL7iD1aiq+rv+ayDRRM1fDzMczhR6xf/b8E3wuH8vTKIZtbU4tnm58CofpDkIZ3HftCWRVZOG72RcpF1LQVNpEU9gQSP692POCuHYL2xGG+KPxGPL7EFqjT9HzIqTfTofbYjdaAYPP46OptAl6bnpYnb+aWPTa3LFZhFGTcCoBkTsj4TLXBc5znSEpIwllI+r5ZTAZ2Nj8JjleFl+GkK0hyH6cDYshooWJupw6JJ1JAgBKtvqttV3C8QRUplTCYzW5wehDkXIxBem306HSQwXPf3uOBS8WdKvY9+ynZ6jNqn2nD2dX+NT2tqWxpQADWJW7irhO+hQx8OeBYDWxYNLPhDbGYXHQUtmC3Ce5YEoy4TzLGVWpVThkfwh9v+2LgT8NhL67Pnw2+SDlYgptrT5s3zAA1N7HcoSlSANca00rKl9VUhYF83rCbJCZyNqkqwYhgNqzV6dXf5CvXGtNKxhMRpfrw48J7/XeUNBSwO05t2Ex1EJssa80rhTh34ej94reNNURHpeHkpgSqFuRZUAH/DDgg3LR5oPNcXH4RcQeioWOs44I25/dygafywe7law+8Bndxz8d8z4X+z7jkwCXy0Vqairs7OyQdDYJwd8EY1HMIui7031uBN3bPXx6CIt9+aH5YDWyYNzPGPKa8sLF2dvouaAnwraHIe7POAzZI7qolJSRxNywuQjaEESUlvwr0LTRxITLE6BpowlWI4vmFTLgxwHw2+H33onV7sBsoBnGnh1LTBbJqsjCeY4z/nT5EwYeBv/TYl9pbCmuTbyGCZcm0DZyEtIS4LK5KI0phZEXPZnZVNYETjtHuCHpDjqaO3DA+gCaSpswPWC6MAnxb0dtTi0aixph6GlITOJ/ilDpoQL7yfZiDZP5XD5+kP4BDtMc/lXSpdmPssFqYCH3Se4HF/s6xzwJiX9PIVMcYv+MxYs9L7A6f/U/fSrvDQaDgabyJtTn10PXRZe2aYk/Fo/K15VYkrCEdmx7QzuSzydT8hqNLFoizH6SPThtHIRtD8OwA8MgqyIL23G2UDVWxb1F95AXnAeXuS4YfnA4pOSlcG3SNWQ9yKKKj/+H33R/g+VwS6pr+D3BYXGEngNKBkq0Yp+ElESXm/tlycvQVNqExpJGWvej22I39JzXE/cW3UPIlhDYjLOBtOSba1aRUoFLoy7BeY4zrWDHY/OQF5IHKXkpuC50FfF5UDZURmNxI/p910+sfAsAscwUDouDkC0h0O+lTyy8Am+Slzw2jzj+dPNTFEYUYm7oXNoYt4MLPp+P+0vuw36yvciGjM/jw3Sgqdg1wZWxVwAAi+MXQ6+nKBM9+UIyUi6mYOrdqejRh87i7P9df/A4PDz78Rl8NvuIJITqC+oRczAGVqOsiN2oknKSUDNTEyurq+eqB6/1XkRPP2NfY8wMnEkcA4DqNMr36e0EFkAl5g5YUTKxq/NXiySQ3wUul4u012mwsrUirnVYTSx4f+mNQT8Poo0L/Pzq8+vRa1kv2r3LYDKIDEkBwn8IR8iWELFJ0p2qO8FqYIkwTARorWlFRVIFNG01iQVZgGLF5ofmo/fK3rSEEUOCQSw+Dj84HIcdDiNkawg813mKMKeUDZWRfJ7yhHJd5EpktDaXUwzM5rJmYtfw9SnX8frqa8iqyuLruq9p4101l9Xl1iHtVhqsx1iDKckkSgGJK7y1VLbg9ZXXMPQyFK6vTAeZoi6vDqf7nYblcEvIKMvge6nvsaltE5pKKBaZ/RR7kfgoDq8uv0LR8yL4/+JPWwuFfx8Odgsb/bb0E3t8SXQJ8kPzadKTRZGUt9yHyDxp2WlhyB9DUJ9XTxzntHMw4dIEYgIeoFhdlSmV0HbUFvpyCZAfmo/GokYUOI4AT5LOBvLZ6IPCiEJk3s+kMQMFHk69VvQSaY4UgNRA8avOr1A2VMaMhzPAamQh/lg80m+n0/4u414G4o/Gw3qsNewn2Yvsa7hsLqpSq3Bv0T3UZtUKO8g3VG0AQF3jlIspePLVE0jISNCKfew2NrgdXGQ/yqY1bxZGFCLrYRY0rTXRUNQgVjJTHLw3eIvISfP5fOy33A/rMdZoLm+GzyYfsQxyTRtNjDk5BhUpFbQ5QUZJBl/VfIWX+17i6eanGPDDANo6r7m8Gb/p/QaAzkD22eiD4ufF8N/tj9qcWqiaqIIpwURL5Rt/opDNIZDXlEdFUgVsxtkI439VWhWqUqtgNsiMNidw2jlIu5UGSVlJmPQzgZy6HGRVZfFl5ZeQlJFE/Il4FIQWYMSRETR/JoBigrNb2cT9OIPJgJScFBqLG1GXVwddF10RxlphRCEsh1vCbakb7Vg1MzVs5W4Fj8MTife3596G2SAz6Lvr48k3TwBQ90N38XTTU1iPEW3QCPs+DG01bRhzcgyyH2dDUkYSJv1N3vs9B+4ciIy7Gbg06hIAYCtvqzC2V6dX46DtQTClmNjSIT6OdVe5qDNiD1GesiT2UV1eHapSqzDq2ChKSeItRO6KROLJRGo98X/zRfGLYpzwOoHBvw2G1zovyCjJiKw3LIdbijTetNe14/XV11A1VYWhhyHkteTBlGAiNzgXdbl16DmvJ3Fub61uRU1WDdFnddDPg+C+1J3GIBegpaoFCjoK6LW8F41NWRZXRrGYXpRA1UQVY8+MFY6x29j43eh3OM92Fnm9Mx6vfYyYgzFYV7IOSvpKsJtgB3VzddhPtqflad5GW20bcp/kEudR/V76GLpvKFgN9HwPAHh96QWHaQ4ftYm1sbgRN2fcFHmtu7mspDNJqMmsgeUIS5EmoOaKZrzc9xIDfxz4zvf41Pa2YdvDkBOYg82szf/0qXw0CPISgqJZ5/VnRXIFjvc+DoBqWnWe5QyGBANS8lIinprvUoawGWMj4nsLUPHk0shLQp920poj4VQCVIxUiHL7gqYCEipSKhD0ZRBqMmswO3g2rXny7MCz6GjuwKrsVbRj80Pzcc7/HCyGWsB8qDl6L+8t9nO6A+vR1lietpz4PQUoiS5Bxp0MmA40pX1nGSUZLEtehspXlWitbhWR+f8r6KwiUpEk6qMqoyKDFZkriA2Xn/Fh+Kdj3qfDof6M/+/B5VKLQJP+JpCSl0LCyQTi3w3cORCui11h0NtA+FrY9jBcHnMZhx0O45D9IbFm04JF14vfX9DGGEwGjH2NsfDlwg/yuekKCloKcJjigPtL7uOg3UHaOLuVjYrkChHD9c549tMzvPiDfs7vg8aSRui768PYl9x9p26hjnkR8+A8Rzxr4u+AipEKnGY6Ef2bACDuzzhE7Y4ijgWuD8QB6wNiEytdgdPOQVMpZVgsLjH8b8TLvS9xxu8M4o7G/dOn8tGgba+NiVcmipVWuDaZ8lsjsVf+SXis8sBm1mYRn4cPgSDmfQow9DSE62LXT8qzr6OlA/cW38Mpn1NEw/BBOwdhzlOyJKGGpQY2tW2CjJIMonaR41BJTAmSzycLO+SMvI3Qe0VvNBQ0AKD09k/5ngJAyUE6TKOaGr5p+AZri9bCaZYTevjSi0A5gTnYztiO61Ou08aYkkz4/eCHyTcnY9K1SbRxVhMLz/c8R8CKAKKcNYNJydIZehjSZKSYEkxIykrCY40HptyeQpMeen31NRoKGhBzMIb2vrKqsph8fTLSb6cj/ZZoYnht0Vps429D/239xUolxh+Px73F98Dj0mMyl8VF1O4oXJ98HQ9XPSQer6ijiA1VG8R2QTaVNKEiqYKYNLy/7D5+lP0RE69OhM9mUVmvV5de4U+XP1GeUE58X/dlVOGoMKKQNlYSXYKcxznQc9UTW+Tst7UfNrVtEkq3dj7fqN1RVNcx6XOXumPm45nE922uaMbpfqfB7eASpUk5LA7ODz6P578+J7734F8HY2H0QqK/nUCWEQDRYyJ4UzC2M7Yj8XQi8bcMmRGC8/7niWMROyPwm+5vKAgvoI3fmXsHQV8GIeZADNHo/l0QJCzTrqcRxwWb7MrXlbSxkpclODvwLA5YH0BNJj2ORO6OxOWxlxH+fTjlo9cJFkMtMGzfMGQ/yhaR2AWo+W9u2FzMfDwTfTb0gevCNw0FPA4PU29PxeK4xRh1lMwcLIwsRFFkEYYfGk70DBEkVnsuFC9/V59fj4x7GbTz7rWsF7bxtuH1ldfC5gIBWmtacX3qdSSdSyK+Z4++PTAvYh4YTIZQSrnv130x+QblbZQVkAU9dz1YjrAEn08xC7fxtsFuEpVcTDqXhAM2B1DwrID23mXxZbgx7Qai90WDw+LQxkceGYkhfwzpsmCXfisdRt5GNGmkGQ9nYODPAxH8TTACVgQIr19JdAkujryIvKd5qMutQ97TPHS00Nm2nqs9xfqJ7jPfh4ifImDQy4A21tHcgXODzuGc/znkh+bTz/d2OgLXB0KCTT1vJNJq6HehuDXrFu11j1UeWFu0FkN+IzNLAeDapGu4Mv6K8P8242xgO9EWsmqyCPoqCOaDzbE4ni5VnB+aj8z7mbi38B4aChtExlqrWvGny5/gcXgYefSNZ9/lsZexx2APWipa8HDFQxh5G2Hi5Ym09z7idASn+53GZtZmjDkpyjIpflGMiJ8ioGGtgXXF62jNgtEHo7GdsR37LPYRv6+MkoyozCEf6PttX9hOsEVTSRNqMmuQ+yQXTWVNxOM1rDRgN8FObFNN6vVUxB+LF/6/8zrvbV+xzui9ojfGXxiP0O9Csd9ivzC+9vmqD75I/QKKuoooiS5B2s003Jl3hyryCz7zWiquTbxG3DO11bXh5vSbuDr+Kp7voWI+g8kAp42D1upWFD8vRvL5ZMQfixfujTojYHkATvY9iVeXX6E8UXQeZDWxUPyyGFG/ReG072mhd5sAiWcSkXw+WSxDnMFkQEJaQiQ5qaijCHlNebQ3tKPqdRUAfBALwWW+Cy25mvUgC6nXUxG6PRQXhl7Aw5Xk9YQ4dG72GPzbYJEmi+oMqknmXfvKhoIGYtHrfTDi8AiMPTuW2GTYUkE1WbTXtxOLSMXPi8GUYmLm45nCgpysmixsJ9iKbZz13+UvIres56aHrdytYDWx8Jveb2itbsVhp8M4N+gc7i++j8QzibT34LRzoGKsgmn3pond76saqxKlBQFg+P7h+LL8S6Jss/NsZ6zKWQUeh4esh1kiY0xJJjxWe8CorxHq8uqEDWGdEfcntZ8WSObqu+vDdaErZJRl0FTWhJ+Vfsb9ZfcBUHNyxr0MFD0vQsa9DET9GgW7CXZEeVkZJRl4rPSA72ZfYgFd11kXOUE5uDP3DvE7vw/2W+3H5TGXhf9/e4+j46zTLVYfAOGe4u11UG5QLvKe5KEgnD4nk/Ap7W09Vntg6N6hqC+oJ94jnypiDsVgB3MH8kPyRV5X0FaA51pPDPplEMZfpBjwmtaa2NiyEb6bfAEAd+bdwW7t3USG3f1l9xH1G32e4bRz8HTTU5gOMIWBhwF4HB447RyRZoGC8ALcnX8Xod+Fdvv7xB6JRU5gjtjGD4epDlDUVcR2xnZankxOXQ6WIyyRE5iDx2sfd/uzSch6mIWYwzFQMVaBtKI0msubiTnc1ipqbS2uqSTrYRYujriIsoQy2lhVWhXuLLiDh6se0prTukL8sXhYjbLCVzVf0ZqKmBJMaFhqiG1c/IwPwz8Z8/4b9I/P+P8Khh6GUDFWQUEYeVFBMnPvt60f1C3VhRuse4vuERkAb29EOoPP46OlsgVS8lLv7Oj6UFiNsqJ5HgBU8uH2nNvw3+1P1J6PPx4POXW5Lv2SxOF3w98BAG5L3TDy8EiRsYaiBlyffB3Oc5yJzIO/E5n3M5F8Phl9N9J/z9aaVpS8LBHbeW49xhqvLr3CEZcj+Lbx2259rpyGHDa2bARDgkFLZv8bwOPyiPIq1qOtEb0/Gvkh+fBY9XGlP/4p8Lg8nO53GiZ+JkRpLHVLdViNskLvFR+nC+tj4t8kK/q/gKqJKjJuZ0DPVY/IjPk3oi6nDtkPKWYASRJOy04LKRdTcGf+HQzZM0REjkTgLzXxykTihicnMAfgU9Ivb7M1VmZRDKPADYHCjX5nBp6MsgxklGUw7uw44nmXxlEFnoy79KKGhJSEcENGQlFkEQLXBwKgfJje9hgriSkBg8GAlp0WJKQlRLqwm8qa0FDQAC17LZq3CUBtppQNlWlyXAKomavBc60n0V8g7lgc7i++T/T3AoDcJ7l4feU1TPqbUP4GnSCtKI0VmStwYegFsQlrBpOBsO/DIKcmh/7f9aeNjzk1Bmk303Bl7BXMfDxTZMzQwxCcNg5sxtjQnmuBzHTazTSix+KgnYPgPMeZyHAbvHswBv40EEwJJjGuB30VBAaTQfSh0HPVw6rcVbi36B4ifooQMmIEKE8oxwmvE/D/1Z8mgSwhLQGbsTY0yT4BOO0cOExzgNlgemctQD3r4pJymjaaXfoSCWTT7sy7A/sp9mDKiX7njvoOdNR1oD6vntZ1Lkhgnh14FksSlkDX5Q17YeDOgUi7noamsibYjqOvC1hNLOxU3gmGBAObWZtp19p3iy+sx1jTfMcEEDyzJAaelr0WdJx0UJFcgYaiBppcT/yxeEgrSGNR9CKomqrSji+NLcWL31/AYZqDSByKPhgNeQ15YiNW0FdBePH7C0y4NAFh28PgvcGbdv/pu+tjzKkxYhUZ3iW7eHbgWeQ9zQMAzAmZQ0xGOExzoPkEs1vYeH3lNerz6lEYUQjP1Z4iTCgGg4HM+5mI3BkJy2GWwgYRbQdtzAqaheaKZmjZadHuW047B4URhbg9+zYAikXyNsqTqGLDqGOjiJ5Xhp6GOOp2FG11bVidS2ah91reixhH5NTlwO3ggtPOQczBGAz4cQAkpCXQWt2K3Ce5cJjmgLyQPDz74RmWpy0XiXPZj7IRtiMMg34ZRJS1s51gK9ZegNXEQuWrSjClmMS467HaA9KK0qg5HYwSmwEARFU6TvmcAp/Hx6hj9KKwjJJMl95gJ/ueRFFkEcz838SC0cdHA6CKeQ0FDWgoaMDgPXRPb99NvjD1M0VbXRuts11aSRp+3/tBv5e+kCHM4/KoBj0GxUyeGTiTkjcmeDJbj7UWuz7vuaAnrEdbi1VwEcRicayW3dq7YdDLAFNuTREWmgb+RDFWjLyNEL0/GucHn8f4C+Npc1HYjjA83/Mci+MW04oTHBYHmfcy4bvZFzrO5NirYamBbfxtxDVFUVQR4v6MQ0tVC/p83UfodyirQnkDyajIwH2ZO+yn2MPQwxBKBm+SdbbjbRG6LRRpN9JoDS+yqrIYtn8Y0m+lC2MIt4OLfeb7ICknia9qvoK2gzYer30MbQdtGuPbfZk79Nz0cGPaDXh/5S0SlytTKnGyz0k4TnfE0L1Dab+J52pP2I63FWGMCNBe347UG6ngdnBhPthceD0nXZuE3Ce5UDVVhf9ufwQsD0BTCbnw2hWU9JSQ+yQXvb544905N3Quks4m4f4SqoDj9333PN8bChsw5tQY6Lnq0eZXs4FmWJ62HFWpVQj6Oghe67xoLF2Aeq4+tNhH8vQUQM9ND6tyVoHdxiZ69ZLmA01rTUy+Pln4/9qcWhy0OQjvr7yJLC4GgwEwAJN+JmAwGJBWkIZRHyM0lTShrbaN+L3aG9pxe/Zt9Frei9jsUJVahbKEMvTo00PsuuP5788RdyQOMwNniqy1BGtpksWFpIwkhv4xFBn3MrDPbB/Gnh1LszKRlJOEhJSECBu/JrMGt+feRl0uVSCMOxKHkYdHgt3GxuXRl2HS30TYlCFOPpnL5oLdyoaUnJTYvWJFUsUH3dcCcNo4aK54U/DXc9XDvGfzIKsqi5aqlm5LsAOAtqO2CFtVAFUTVWjZaXXpF/6pwmyQGTjtHOw12Ysxp8d0KTP5qeCM3xnkh+bDbpId5LVEC76qxqo0ZTNWIwt1uXVQ6aECOXU5KBspo7WqFY/XPobXOi8RZbCUCykw8jKCQW8DpN1IQ+8VvaFuoY7yxHJUJFXAzN8MWrZaiPglgqbSlvskF8CbJsnO4HF5ONbrGDqaOzDt3jQaO9l5ljMMehnAYaoDUd2q7zd9YTncEmHbw2hxRMdJB1NvT0Xl60riuvJDkHYzDQnHE2A7zhbKhso443cGDAkGvngl6kVsOcISirqKxBwEt4OL9rp29F7Vm6gmEHc0DoknEwH8n+T6e6p6VaVVIedxDtGDl8fhIfdJLngcXpfzyWd8Ovj3ZbI/4zPeA/Mj5ot0+r0LJv1NYNDbAH2/7Yuw78KgZEjuWBB0na0pXEMbYzWyhPIqJGnJv4KC8AJcm3QNg3YNIiZqsx9STERSogigNq2dO+q7g+EHhyNgeQBKXpbQxnhsHhoKG9BS1UKTUvm7oeOkA6eZTkSmEIPBgKGXIS3JJIDDFAe017fTuonfBwwGo9s+Df8rHHE5AhUjqgPybZgOMMWSxCVdyuB9aqjJqEFRZBHa69uJxT7/X/z/gbN6NzIfZOLSyEtii/P/RUjJSUHDWkNssvzfCCV9JQz5YwgMPQ2JzBcel4eiqCIknkrEgB8GiGxk2a1slCeVQ8VIhSjRUfS8CDEHY+C2xE24MQ7eFIyM2xlY+HIhpBWlMXg3PUEKAPcW30NDQQOmB0wnFvY9V3vCdYFrl1JPhRGFkFGRoUngqluow32ZO3r49ICKEV26+eaMm6jNoky/p92bJrLYf331NeXzFjGP2Pyhba8tNoFQFFWExuJGuMxzETmnV1de4f7i+8Ii0OWxl7GpdRPt+NEnRqOppAl35t2hJVgZTAY0LDWwKocuzyIAu42N6H3R0LTRJBb7AMonlsR0c1vsBrfFdJkxADDoZQA1MzXahlkAGWUZscUWCWkJpN9Jx/XJ1zHh8gSaj1jG3Qy0VLbAZqwNtB21RTrAJWUloWaqBqM+RkS5lfr8ekjISNAYgQAgpyYH96XueLn3JXR76tIKCCUvS/Dq0itYDKXLdALUvX3S+yQG7RqEPhu6x14evn84XOa6oKmkidiJ3+vXXuBmcCGtRP9O1qOsMen6JFybeA2h34Vi6u03sq41mTUw8TMRSdx2hmC9yOfywe3g0oqMDAaDKH3WeVwcVI1VMT9yPliNLGLBZnbwbMqzgRAnKlIqwO3gYn7kfNrnh30XBi17LThMdcDNmTchKSeJ0ceoQotuT104THVAyNYQ1GbVwnGmI+29VY1VoWGtgayHWWLXUl1BWlEaui66xAJ9Q2EDarNrYTXKiva+ykbK2MLZgoQTCbi/5D7sJtiJJCra69uhaa2J8RfHCwsHIVtDwGFx4P+LP/Zb7YeUnBSWJi0V+Sx5TXnKX45BMUpIHol2E+zQo28PKOkrid0j2E6wJTLvhO8xyQ6VKZVoq2sTmc8qUirQ0dyBiVcnQqWHivB5tBxuic3tlMRXSXQJlPSVoKAjOp90NHeg+HkxLo++TJRMHbZvGPaa7kXhs0LMfjJbZExeUx5Lk5dCXkOeKK2rZqoGKQUpaJQko8LME28X+2qyamA1yorovdha04rIXyKRciEF0wOmE58BksxdaVwpzg85D/1e+mipbEHV6ypa7JdTlxObLJJRkoHvZtG9DpfFRUUyJSm112QvZj6eSSz0ARDOndmPs6GgrSAikSynJgc5NTk0VzQj7UYa9N31Re7frnxDAUBaQRpZAVmI+CWCJvcaeyQWL/e+RJ+v+0C/F73wqmyoDFYDC/st9tP8UDuaOnBt0jW4LXUTq1YhACnenOxzEgAgISOBGQEzhK/X59fj3KBzqM+vh4yyDDStNWmJUG0HbSx8uZCYDJSSk0LvFb1FmuaaSpvA4/DAbmVDQkoCtuNtoWGtIVLIE8B+sj0sR1jC0MuQ1uigYqwC/93+MOlvQpT51HbQxqVRl6BspEzzpWwoasC9hfcAAO5fuGPEwREAqHv28pjL8FzrKWRxpN0gM7K7QmVKJQojRRn3krKSqEql2IJri9cSE7BdIfjbYOSH5gtlaTtDWlEamjaaiDsaRzV3THEgFvt6r+wtVsXnXajNrkXAigC4zHOhrSckpCSQdjMNQRuCsODFAuK6JO1WGkpelqDftn6QkqPvg6UVpGE53FJ4fz355glkVGTg8y2leMBuZaM0thQGvQ2EEn8jD4+kNRJ3hqyKLAb8OAAdzR1ob2inzSexR2IRvT8aAF3aFqAa6zLvZVLF77fSIKxGFtrr26GgoyC2OUDdXB3eX3kTk+lD9w4VuQ5hO8IQui0UTEkmrEZaIf12OvTc9MBhcSApI4kxp8dAUVcRLvNcoGSgBG4HF3V5dbQ4lvc0DxeGXgAArClYQ7NRuTXrFoqfF+Pb5u41K3fG+IvjRdaGsiqywngUfTAa1yZdw8xHM4nPpThUJFeA08ahMUf13fUhpyH30WQG/9eI/TMWD5Y+wJcVXxL3gmpmavBa7yVWvvlTg8D3VZx87dsojCjExREX4TDVARMuTYDfDj9IK0rjyddP4DjdUeT+/bL8S4ABJJxIwMu9L2Ez1gbqFupIPJMIz7WewkKitr02HGc4iuSs3Je5w2acDVEWns/jCxVUip8X0+Y4Q0/Dd9qm6DjpCBUkSPiQAjhAqQm83YzVb0s/aFpr4qDtQQw/NBzWY63BaaOYjJ3XpmpmamirbQOriUXL37EaWQj+NhhuS92Ic5HDVAco6SnBcoQlcR/YWtMKbgeXxtIb+vtQJJ5MxG+6v2HqnamwGmklPKemsiZcGEbFpq7WSZ/x6eBzse8zPgkwmUxYW1uDyWQi4WQCYo/EYvyF8cQuou2M7QCAiVcnirD3pOSloGaq1uXk5r7EHeaDzYmTvYS0BBynOyLlYgrynuZ91GKflDyVKBfXFeX+hTv0e+vD3J+8QeR2cD9YcrLXF72QH5JPM7YGqElobthc7LfcD3YrG4N+Fm/+/rGhZaeF9FvpqHxdSUtwyKnLYVbgLKJMkwBdsYuujLuC6oxqLE1aSks4cju4yA3ORWFEIWzH2XZrIfx3oqOlAxVJFWJ9lBjMrpOVnyIaiqhibVcm5Q9XP4SKkcq/qqgmkIKry6v74PfoHPM+BeQ+yYWygTJsJ4j3efq3QV5THp6rxbOhL428hJzAHHzT+A1Naqcurw4nvU+i77d94bvFl5YY8VzjCbNBZmivaxfpoubz+cgKyELkL5Hw3eILm7GUp8HLfS9RnV6NEYdGvGGgL74HdQt1YSJFACl5KbENCR3NHTg3+ByKnxfDcoQlpt8X9ftTt1DHiEMjxH5n3y2+KHlZgo6mDlqsMfYxhv+v/og5EIOLwy9iVc4q2ua+JrMGDCZDhDVRGFGIUz6UXGmv5b0w/MBw4djDlQ+FhT55LXmxskLSCtIYuHOgUPKkM3hcHhqLGiGjLCO22aE6nZLQsp9C9pjIe5oHM38zIitMgJ+VfobpAFMR70AlfSWxRUY+n4/f9H6D+WBzjD4xmjbX1OXVobm8GXaT7IieucuSlyHs+zCc8DqBJYlLROI7p53q2vb+0pvIzCmNKwWXxRXL3msqbULe0zxibNW01YT/bn/ouekRjgSCvgwCAGKisqWqBdH7oyGjLAPXha7ENY2+mz5AqJ0ymUz4LfODrKys2OKa3QQ7GHkb0YqrsYdioW6hLrbYJ60gjSUJS9BU1kTspM8NzkXxi2K4L3EnJqzCfwhHXV4dfL71ITKCpBWlxXpcqBipoKOlA83lzZDTkBO5DwqfFSJkSwiM+hjRzmtm4ExhgrIms0YkBjnPckbA8gB0NHVAQVtBrFdlxp0MRP4SCYshFrQk6uurrxH5SyQ0rDWInrfiPDIBSoowcH0g5kfNp/kmMxgMMCQYcJ7jDLtJdrTrUhpXijvz7mDE4RHCOJZxJwPsNjb8f/GHzyYfpF1Pw8PVD+H/iz/Sbqbh8drHmPFwBkYeHQkdRx2xSR0ZZRlwWBw0lTRBzUyN1qB22PEwGEyGsJBIQvrtdGHHuZz7m3hyc8ZNVKZUwmezj9jrbdDbQMRCQAC7iXawm2SH1GuptGSPAEZ9jIjrOwkpCbG+xQDFEun1RS/8FtYLXElpvH33fln+pdhjo/dHI2p3FCRkJIiNJfMj6IyYgBUBkJCWgPkQc6j0UEHMwRhcn3Kd6GXJamRBxUhFbKPg/aX3kf0oG2vy14ApxcSY02NQFFWEwnCqWeX61OvIfpiNbxq+IR5/YegF2E6wFWEfsZpY6GjqQFVaFW7NuoWhe4cS2eTiMO/ZPARvDBZec1YTC1cnXIXVKCtw2jjQc9VD5C+R4HF5tIadnvN7Qk5DDi/3vqTd99KK0phweQIkZSVRk1UDDUsN2jqP1chC7J+xkJSRpCl06PfSR2lMKcacEpUtjfo1CvX59TD2Ne5S3YV0X4qDjIoM+n3XD0ZeRqjNrkVrdSvMB5sT7xGAiq92E+jFZGUD5XeuzU38TIjNMqrGqph4ZSLyQvJECtXJ55JhOdwSRn2oc/Nc59ltaw0ehwcpeSla409FcgWcZjlh4M8DP8j73GmWE7ICsrCdsR26LrqYHjBdmGhlt7GRcCJBaBMiTiEoNygXNVk1GL5/OHG8K1wccRE1mTVEv6uOlg5IyknCZb4L0Ru0PKkcwd8EoyazBn2+6gMpOSnUZNbg5b6XsJ9iD2MfYyjqKorMCykXU6CkpyRco9YX1ON0v9Pw2exDbNAkQVJWElw2FxE/R8BxuiNtnnKc7ojCiEL08CGrC4XtCENdTh3Wl62njcUfj0fg+kCMODwCGlYaIh6yHS0duDn9Jsz8zcQ2jspryuPBsgeQVpKG1QgrSMpKQrenLmYFzqLGvniA2MOxYDWwoKCtIML6KooqwrlB54R+h53ROXFPYvFqOWjBtM60yyYjDosDThsH0krSIs9l+p10PF7zGKYDTUXWQ3weH7+o/QJWIwu9V/WGjqNOtxub7y26h9KYUqzMWimyDqrJrMER5yPov6N/l364wL9zbyvYV1SlVtHyf7dm3UJFckWXa4ZPDZ2ld99G5etKBH8bjMx7mVDUVcT6svXCJo7OdgTuy9zhOMOR5kEtuKdc5rrAZpyNcD0dd4SSzswJzMGwfcNgNdKK1gykpKckVj5SQkoCG6o2oCKZ8scmoTS2FEXPi+Aw1YF2Xo/WPAIYVKHrbVSlVSHuzzjYT7GHQS8DsT7a4nDC6wQA0eKYSg8V2Iy1QX1BPbTttVEaXYrE01TjcOe1QeWrSpwffB5D/hhCy0dIK0pj3PlxxKZNgFKdEddMCgAnvU+iJrMGWzhbaHO3sMF2zGVsatsknO+kFaWhYaUhVK35jL+OfzrmfS72fcYnA2lpKjh2NHegNKYUVa+rxAZAACILxnODz6GppAluS92QdDoJo46Ngp4rPZGl46SDlqoWpF5LhdNMJ5ExKXkpjDs3Dv6/+n8Q8+tXnV/RUtkiElQF0HfXx7zweXjxxwvcmXeHtpkz9jEmSv8IMPbMWGKX/PuC5O0kgIyyDBymOfzPC0ktlS1IuZgCQ29DYpHz2qRrKHhWgI3N9A7K0O2hKIoswqzAWcT3bqtrQ3VaNe4tuoexp8eKjLVWt+Li8IsAKE3/f0uxr7mckuMQl7x99vMzxByIwdywud023f63wtTPFBuqNhAlfgAg/kQ8ovdFw6S/yb+q2NdrRS/ouup2aV79PhDEvE8BhRGFSLuZJtT4/1SQfD4ZUbujMPbMWFrXunE/YyhoKxCLKYo6ivD/1R9BXwYh+Xwy1hauFRmXVZFF5r1MRP4SiZXZK6Furo6BPw7EwB8HYrfWbrRWt+LKuCvo83UfDNo5CNmPspEfmo8Rh0Zg8o3JKIkuQfzxeGKxr6WyBel30qFmpgazgaJJHR6Xh/r8ekgpSMFtCZmNxm5jUywkNTlagcF5ljNNykgAPVc96Lnq4eX+l2itaQVDQnQzFLo9FGHfhUFeU15EVlJRTxHG/YxRm1WL1OupsB5tLWQ2DNkzBDmBOZCUlcTwg8PFzmN1eXWQVpQmMgrbatuw13QvAHqTj/AcdBTR77t+Ismezgj/PhwVKRXEYl/k7kg0lTTBcrgltJ1EOz+TLyQj9nAsRh0bRexGbattQ/K5ZGg7aNM8PCN3RSLuSBw2VG0gFpgkpCVgNcIKsqqytA1w0fMinB1wFsMPDicWuNyXusNyuCUxqddW24bSuFJMuDSByDBRN1fHq0uv0FDYgGH7htHGnec6Q91SXegx2Rn1+fUI/z4cAFW8eLsQlBeSh4bCBrFSSPdm3YOKsQqG/DaENpZ4OhGZ9zMx9c5U2vWSUpBCYUQhtjO2o993/dB/W3/a8bouukRmCgCknE9B4ulE5D/Nx+zg2bTxkC0hACjW0NuJktzgXNyadQte673gvsSdluRvLGlE5K5IRO+LxqLYRSLdvzZjbaBirAIZJRl0NHeIHNuZrbQoehHtnMz9zaFuRY8PAmTcpQp93l95E4tIod+FojqtWshk6Q5MB5pi+KHhSDydiKcbn2JOyBtvU3YbG6UxpVA2UiYyszSsNDB031AYeRuBz+eDwWBg4cuF4PP42KmyU1j0it4XDb8dfjAdQH2WrosuLIZaoKm0CWk3KcbW20XyjuYO3F98H+m304ld+sb9jIF35HCSTidBw0qDxvL1XOOJwPWBUDZQRlttG2RVZcFgMtBU1oTyxHLo9dQjPm8CTLg0ARMvTyQW+h6ufggdJx2izy+Py0PG3Qw8WvUIQ/4YQiuqPFrzCLGHYsHwXw8wyA2Dt+feRn5IPtYUrBF5vYdPD/T5pg8813gSi/cAdR91tHTAcRrFHo0/Fg/zIeYYfmA4og9QjBurUXQGX+h3oYg5QPm3Lo5bLLLnaiprwrVJ11CRVCFkS0hISUBGSQbxR6lmlxOeJ6ii4lsxKjc4F8lnk6FmoYZRx0bRFE9e7nuJkM0hmPVkFqbcnkIrlOY9zcM5/3Pw+8GP+OwoGyqLSGhzWVyUvCyBlp0WEk8nQstOC9ajrYnxHgBsxtjAZowN7XVJWUk4THHA+aHnURRZhG+bKNZO53VeTWYNnnz1BADl0df5XpkTMgc8Ng8pl1Jwfuh5jD8/HvKa8rAaZYWO5g7020ol2SN3RyJsexgWRC0Q7hee/fwML/a8wLyIeTRGRENRA/aa7AWfx8fIP0fCbbEbJXf9fzH01uxbSD6XjI2tG2mMaAC4M/8OmkqaaPLXnZF8PhmRuyIx7uw4kRh8d9FdpFxMwVc1X9GOkVGWgf1ke9o6OnJXJAx6G6CjqQPB3wQDoBgZXRXE3waPw0PKxRQwmAyR+fPG9Bvgc/kYcXgEzvidgUl/E5HY9i4ImrcAKpnKZb2RSUu7kSb0APRc5ymWBeW13guKeuLjSFcw9DSEqokqcT9UnlCOhyseYsgfQ4iy4k83PkVNZg1WZq+EjAq15m0sbkTMwRho2WsRcxBvS9Ip6ipi6L6hKHlRgqsTrmL8hfF4sfeF8Hd6Ow4IYD/ZHjqOOkTpXUNPQyyJXyL2Ow/aOUis16Wemx4813ki5mAMarJqhAxsgFIuygrIgoIuvblbACl5KSgbKgtzNn2+6iMSo13mucDY15ioRNBU2gTH6Y7ENZa2g3aXrJm+X/dFkW8RcgJzxCoshO0IQ8RPEVievlzkmWYwGWA1sZBwIgGScpLC6531MEuY4B/y25APUmpymOaA0phSxB6JFVkHCeS+344t4vBv29t6rfdC75W9iQ1b0krS/ynFJAGKnhch+Vwyen3RS8R/u622DTmPc4Q+dgDVJDrt/jTh+ibxdCKaK5qJtkmlsaWQUZGBhqWGyPUceXQk7i++j6rXVcQCN0CtHXcq74T5YHNMfzCdNi6vKS92Dxf0VZDQk7ZH3x60Yl9OYA7q8+vB4/BgO94Wpn5v3qc+vx4v977Ey70vAQBbuVvfuU7sjNlPZ9M8ytmtbMhryWPYvmFgMBhoKGqAtJI0bd+ceT8TAIhrZUlZSUjJS+H6lOsYdXwUbb8PUL8Xq5EFZSNlWkFP4CHOaeOI/BYVyRUYfXI0GAwGOO0ckVggpyaHFRkr3v/Lf8Z74Z+Mef+etorP+IwuwOPxkJKSAh6PB49VHtB316e6NAjYxt+GbfxtIgssVVNVtDe049GqRyiLL+vSgPXFnhe4t+gecYzBZEBJT6nbckgAlaAFujZgzw/NR9K5JNrrVWlVOO5xHEln6WMAtSm+PuV6t88JoOSTzvmfI46117cj+3E2+n7T96MyGd8HzeXNcJrpREzcshpZKE8sJ8rQAZQXV25QLi6OuAgel854dJnnAgBCz4vOkFaSRr9t/dDnmz7d7hb9O6GgrYAxp8fAbhK9exYAwKc2Fzdn3vzfntjfCAlpCcQcjhH7vCafSwZTkklcFP6T0HfTh8dKD7HJs/dB55j3KWD4geHw2eQj0vn3b0dVapWwa5MUJ/p+0xcDfhqA0rhSGotYXlMe3uu90X97fyKbsa22DZq2mhj480DaRnH0idEYum8oFPUUkXaTYoFOujYJ60upzmTb8bYYtHMQVuWsokm6AZSx+v3F93FuED1uy6rIYn3pemxs3gjrUXSZu8KIQvwk/xN+0/0NJTF06eb3gcdKD8wKnEWTbBVsBHuvEvXQVDdXx9zQuRhzegza69rRWNwoHHOa6YRxZ8fBcYYjHq99jJossm9u0JdB+LPnn8QxKTkpeK2nOqeTzyUT/0ZJXwnWo63FdtP339EfsiqyCP8hnDaWdT8LKRdTMPHKRJrMdnN5M4oii2gm9wDFcFqRsQJOs5ygZU9PDNuOt8WgXYPENjOUxZdBSl4K3uu9aUULFSMVeK33QkVKBYK+CqIdy5Rg4sXvL5B+O502xmpkIeZATJfPKofFIfo5AIDbIjeMPU1uMNKw0qDkPb/pQ5REjTkQgztz7+Bn5Z/RVivqT8zj8ZB2Iw0v9rwQblI7oyK5Amk30ohG9J3llUjrAlYjCw9XPUTqjVTid+rzdR8qiT+W/swAwIIXC+C5zlOsT3BzWTOCvgwSeo50xqm+pxC9Lxq9VvSiJXiV9JXQXteOY72O0Y7taOkQq14Qfzwechpy8FjlgeIXxSLPlAASMhLQd6c80UgNaqNPjMb8yPnY1EaXzQUoH81Loy/hd6PfkR+WLzKm66yLXst6gdfBQ+XrSpGxppImnO53GhE/R6A8qVyYYBRAxUgF7BY2jjgfQVl8GYA3SQ2rUVYw9DKE/25/rCtZBxklGeg46aDXsl7C+HLE+QiuTriKgnC6b3fy+WSk306HoachUa1i+IHhkNeQx70l5DU+QMVuLXstWtzuOb8nXBe74sGyB9ilsUuoPFAQVoCLwy+iMLIQL/e9xEHbg6jNqRU5tja7Fq8uvxI2bb2NlAspQrn+t9FU0oSr46+isbiRWCg06GUA88HmkGuqBpMjKk/K5/GRcikFdbl1RBsAs4FmGPTzILFrldg/Y3F5zGVhAQoA1petx/jz41ESU4Ko3VEYf3E8Jl6eSH/vQWZQNlKGjpMOramEx+ahLqcO3l95Y+HLhcLXsx9T18BlvgtMB5hi/MXxIrL19QWUZGXS2SRk3c+C60JXWgLMoJcBeq/qDU0bTdiMsaHFoZKYEvB5fITvoMd6AHjy7RMkn38zj8hryuObhm8w9I+hmHR1EobtH4apd6ai5/yetGOzArIQvDEY7fXifX8cpjnAewNVjHl7nafSQwUO0xzguY7O0JNWkIaknCTqcupQFFkkjA0WQyzgv8sfRc+LUJ1RDRUjFRj7GIvMK/Ia8mitbsVJ75O092VKMoUeRp3v26Cvg3B57GWq2bOnLnYq70Tmg0za8R3NHajOqMYujV148u0TkbHSuFIcdTuKV5degcvigs8TTfJq2WnBbJBZtwoPc0PnYviB4TDpb4I+3/SBUR+jbiVm8f/Ye+vwKO79e/zsbmTj7u4Q4iGBGBACJLi7O8WKFS/cemlLKZQiRYq7BSchBAgEiLu7u8tu1n5/zN0hk5kN0Nv7ue33x3me+zy9O2x2dnbmLa9zXueAGBsXRS9Cn0lUUtZnow+cZjqR+yhZTiqyIMeVQ8DOAFgOsURLeQvyHr59pnXsdWAZaImQAyEYsHYA4/5TIpHg8sTLZNH6QzHh9ASZpKuGuQaGfDFEZle01yovjPl9DLRttMmisbm/OTZWbSTFOfxWPh6ue4iMa8Q8Ks3Ek0JJSwkD1gwAW56NzJuZkEgkSD71tm6ReZNut9pS1oLjA46jIr7iT9VWzP3NoWqoitf7X9PGV8vBlgjeG4xhPwzDyF+pwiWuJhefCz6H+yJ3nB95nnHujvomCrp9dCljjKBTgLRLaXjw6QPc++QedPvoQl5JHh11Hdhnvg+PNj7Cz6Y/4+rUqyh9VUohUj4ET3Y8wfVZ12Ue5zXywFHk0Na0loMtSYtzqdgCILqmnGY4Yf7T+X86ksVzqSdG/joSfSZSnxtpZIqsaJXu+DvubbvauiAW0HOzAWD0odEI3heMC2MuIPdB7v/g7P56vNr3CqELQhF3OA5NxU2UYxYBFtjJ34nN9ZvJfF4AsB9tTwrVkk8nI3JnJOpz60krZSlO+J5A+KZw8Jp5qM+pJy3Tu69BrYOskR+ej9DFoWjIezvnPP3XU4iFYkZyVSwUIz88X6Y4TVFdESoGKpj9YDZjI8iqjFWYGz4XsQdjUZVURTlmFWiF9WXrMfSboTLdX3qDVaAVRegBEK4FezT3kPuVgrACKKgp0FyA1E2JNZKsHF85RTmaI4gUkbsj8YPOD9hvtZ/R9Wb+0/kYe2wsbf0V/VM0bi+6jX7T+6H/iv7/pxFN/3/E/3rM+9jZ9xH/SHgu9+w1c6Mnxh4di6biJsT8GoNXe1/RJjcpHqx9gNz7uZh8iW5pxGvmIXRhKOHdvcHngzcBm2o2QdgpZJzEGvIakHohFf7b/DH1Cr3L7sW3L1AeUw7bUcwKLx0HnT9lOQIQipaCxwWIPRRL6xBoKWvBrXm3MHj3YJkdZf8tVMRVIOVcCmNAOq+Jh7aqNjjNYiYgJ56ZCBabhfQr6YwLuLaqNpqiUwpFNUWZmU7vg0cbH0HXQVdmxtOfhaKaIsQCMYqfF9MylgAgYHsAWitaZdob/BPBb+HjzS9vwG/hY8wReubDxDMTIeQL/3YZi7xmHhKOJ8DY0xiWQyz/16fzf4K26jZEfR0FOa5cr13IfydIxBLo2OvAZ6MP4zMFEDZnL/e8xNr8tYzkhVRN3xOvfn6FqG+isCprFUmK5dzNQUt5C2kx7LHYgyxSdbfoq0kj8qJkXUfLIZbIuJYh07KwN0jFJioGKox21dIueBNvE7gvdqdkDiWdTsKb/W8w8exExmyD/sv7M9onC3lCdNR1wMzXDDv5OynHzo86j9aKVmiYayDnTg66WrsYrbadZjoh80Ym9mjtwcbKjZT5TkFVASN+GgHn2c4yyTwAuDrlKhQ1FBlV4hYBFhB0CpD3II+WJTX74WyKQr87XOe5InxTOKpTqxmPa1lpUbpEusNmOGEZ/mDtA7gvcqd1wV2ZcgVcTS7j+WrbamPETyNwZcoVxB+Jx/AfqDZUXe1dyA/Lh8kAunWbupk6lsUvQ31OPdqq2mjdSPnhRKelTGKrug3RP0bDepg1TXXO1eD2muPnv90fXW1dAAs0hSsAOH3mhLQf09BW3UbLfwr+ORjBPwcj6XQSOPIcSnbjqIOjMO74OMZ7GiCKOTG/xiDm1xhsrt9MW4fp9tHFyvSVjO8FerfLsQ6yxorkFUi7nAZtO3pXvcdSD8gryzNa7EnEEuj104PfFj9aR/6Puj/CKsgKs+7OQsHjAogEItiNJJTWhU8KkXYxDTr2Ogj/LBxjj42FxxIPyvttg21hG2wLiURCdtB1R8/7rSeef/kcLWUtMBlgIjPvqKcLBUCQIyH7Q1CVWIWjbkcx++Fs2AZT7xPdPrpwnOpIFnerU6rB4rAw6dwknB95HvVZ9Zh0ntol3lrZilODTqGjjihqmPnRz1/fWR8DPh0Ar5VejB3ZAFAYUYj63HqMPTqW8fimatm2lw7jHFD6ohQq+irkusPE2wTjToyDsacxcipzwJZj06510dMi3Fl6B/7b/BH4ZSCtuPJp4ad4+cNLvNr3Cj7rqZZvCqoK8FzhCesgZpthtwVu4Lfw4fDpaWT5LAAM384bYqEYN2bdgPMsZ9r1lKIysZKIJpjuRMmV3KO1B7wmHriaXEw893YMU9JWQnttOzKvZcJjmQcsBlkwrsGYOtwa8hoQsT0CzrOcabZ7bdVtSD6dDA1zDfCb+BixdwStE1dBRQEeyzxg5mMmU/hmM8Km10w8aScDk3UyALz8/iUAgC3PpuWeVSZUIu1SGhY8W8BoOV3wuACv971Ga0UrArYHUMaw1spWHHE9As/lnjItDlX0VRgtdQHgmPcxVMRWQMNCg+wKBIjOz5RzKQjbGEYQSWsG0MSZnss8UZ9Tz1jUUzOiW1E35Dcg+geCcJpxawbkFOXw6udXjN1oU69MRUd9B65OvUrLFRILxOhs6MSAdQMYHQN81vtA0C7AmwNvaPNGVVIVTvicgJAnhO9mX9JqUdtOG7/1+Q19p/SFgqoCSl+WovRlKdwX0slXWWCxWHi19xUyr2dil3gX+by6L3THk51P0Freymi/+C5cn3UdbZVtGPXbKIRvDoeayds6gYm3CeY/mY+4o3FEJmXYHEbXGp9NPjLtzN+FrvYuFD4phLaNNi1fTMNcA8b9jfHsi2cI+i6I5tZjN8qOyLovbYaakRrYcmxwFDgUIYCwU4g3+99AJBDBcYojqlOqIa8iD20b6tw14dQE0jVn3pN5ELQLUJddx+g6w1HgwMzHjLHbECCyrl98+wLWw61luvUURRYhbGMYTLxNGLurpfMmEzobOlEcVQyXuS60YyI+NSIl71EeSqJKEPVNFNhybKibqkMsfHtcKgbXMNMgBFmbmO+fysRKXJt+DYauhhh9eDTtuUo6lYTO+s5eLfebiprA5rAZXR/Oh5yHhrkGZf9p5GGEyReJsaU0uhR5j/LgscRDpmiaCQqqCpRsTylG/DQCQ78e+qfrUP9rnAk6g5rUGmxt3sq4h+hs7ERhRCGN0Pmz+ErhK4gFYsrY81dBLBS/04Yy5WwKutq68FntZ73umaRIOp2EhGMJGHdiHHQddDH+1Hi8+P4FDtofpGXTBn0bBC1rLaRfScfdZXcx59Ec2IywQcoZQkCz6CVhDV6bXoukk0lwX+ROjgsm3iZwX+KOoG+DaOcg6BQQmc0ARh0aBa9PqPvfQTsH0fZuPWHiZYKNVRtpa0M5rhzUTdQRsJ3ZJeNdqM2oBUeRQxkHLQZboK2yDdemXYPLXBcUPC6AjgOdhBywZgAGrGGOqmmrbsPtJbfhvtgdFoPo9QDSMnWRG+MazHKwJcx8zRD1TRT6r+hPjo0eSzzAa+Ih504OrXO+pawF+8z2AfiY2ff/Cv6Zo/JH/P8alQmVYMuzGbuuJGIJ7iy/g8q4Sow9NpZiwahpoYkRP41AwI4AmZOgppUmrIdbM4bKC3lCZN0kVPJtlW0yN86y0LOlvDvqc+rxdPdTqJupMxaUtO20oWWjRZvcpBDxRYxK+PfBxDMTUZtei+gfo2nFY2mAb8LxBOjY61CKa38FhDwhiqOKoWmpSVPiBGwPIHIHGH4qVUNVLEtY1us17b7Z6Ak5RTkM/3H4O61JmQpk78Lrn4k8hr+a7AMINX9reSutGCRF9yys/xdQ/LwYnQ2dGHucuSinYa6B2sxa1GbU/q2Cs6O+jUL0D9FwGOfw/xuyT8gTYszRMTTF598Z+k76vdpVvP7lNfIf5SPouyBwtaiK47rsOlyfeR1eq7wI0q4HzAPM4bfFj0IqxB2OQ2FkITwWe6C1ohWK6opkpllDXgMEHQIYuBjgsPNhAMCKlBVgc9i0e9tyiCXNPkkKIU+IjOsZuLPkDpxmONGK8XYj7XpdwIsFYjQVN6E2oxbmg8wpmzghTwheEw+593KRej4Vvp/50rr7mJD3KA+XJ1wGAAR+FUjZkEm7WaqTCbIs+UwyI9nnOMURQd8FoSC8QKYFTHfbw54ofVWKxoJGOM+mz2G8Zh4EHQKsyV3DqOyVV5KHvJI8IrZHQNVIlbIx42pwMfvBbMbCsVgoxovvX0DfWZ/R1g0gFNGJxxNh7m9OI18G7RyEjKsZ+N3zd4z/Yzyj4GbUb6Mw4qcRtLkq/vd4iPgixu4TNoeNuuw63Jh1A1OvTSVtATNvZKLkRQnMfM3QWd8JURczwfn86+eIPRgLOa6cTIspWTD2NO7V7s16ujXGfT0OcvKytyfPv3oOBVUFynrksMthmHibyCwGqhioIGBHAIqeFjEWvCViCUFAypjvO+o7CJtBGYURAxcDmYKo3goIUd9FIXJnJKZenUojP53nOJPPftimMHS1dcEujyhajjs+DvLK8gj/jOjq7P6cdsezL5/h6e6nWJGygtHmria9Bu3V7YzWSNNvTSfy4hi+V9zROLz55Q2mXZ9GG5+4mlwMWDsAlYmV0LLRohV4i6OKEfl5JAK/CiSPXZlyBfJK8liRvAJCnhDNpc2oz6mHlrUW3vz6BvFH4jHh9ATIq8hj+I/DZdp2m/uZM1r9ktfjq2ewH2vfq+13wokERGyLwIzQGZRn8taCW0g+nUzLo9Ky1iKFIN6rvRmLodbDraGko4QX372A3xY/WgeLopoi0i6mQVFdkba+U9JWwpjDdLFTd5j5mqGsz1B0KWui+19my7Ex7fo0ConXHYl/JOL2otsAiEKY9N9JxBLwmojOkQXPFpD3gEQiQU1qDTrqO5B2KQ1+W/wQsTUCjYWNjPl+PSHoFCDjagYM3QzRZ0If5NzNQU16DXw3+YLFYsHAxQBdbV3IvJEJj6UeSL2QisrESgzfMxwsNgvKusoUkvaw82EYexlj/Ek66VyVVIVTg09h8O7BFNJGUU0R9qPpezwp5kXMw5mgM6iIrYDTdCd0tXWhIKIAug66kEgk6GrtwqP1j2A32o5GBvpv8wdHkYOX379Ev+n9KGQfm8OGbh9dmYKEd6EitgIAaITamwNvELYxDABB2MhCbzlNPSHsFEJZTxn+2wibNquhVjLt0wCic3D+E7rdpelAU3xa+Gmvn5V0KgkKqgo0sk9eRR4WgyyQH5ZPmYsEHQIo6ShBWUcZ1sOsUZVQxUj89waxUAxDd0PGjnvbkbbganHhOp/ZzvxdKHpahLqsOlpeMkDkaxY9KQIAxjxbFouF6uRqlMeUw38r3SLvXXi19xWe7n5K2sP3REt5C4qeFtG66qV49uUzvN73GutK1hFZs21dJPnH1eRCSUcJ68vWk5Zwp4acgpG7EWl93VbVhjPDzsBlrgtp8Sclo2TFS6joq8BujB0eb30M4/7GNJtPaRG7IJzeeQcQJHhbZRsWPF9AE6FlXMtA0qkkBO8LptUYRF0iFD0tgqalJmMcCABMPDsRaZfTUJVcBUNXQ6RfTkfSH0kYd3IcjPsbo/h5MS6MvoA5YXNg6GpIdikP3jUY9Tn1OBdyDi2lLTShT0tpCxpyG9CQ24BBnw+ikX3ZodmoSa/p1VEp8MtAUvjSHcJOIVQNVTHq0CiZ41zZ6zI8//I5bENsP4jsuzbjGniNPNoarqmoCbeX3IbbAje4zvtzz83/EtKO47LXZTShSOqFVPCaeTIdEP4MpASyRCQBS+6vJftefP8CkZ9H0vLRu2PWvVmQiCSM4o2Oug5UxFegKLKIzKXtqOtA6ctSHPM6hm0t26BpoQnXua5Q0lKirS2ka6vy2HL4b/OHppUmJGIJ8sPyARB7MU1LTfT/pD/cF7lTSCrHyY6M2a8AUbfzXuON1AupjMJbgFifSDORe67lS1+Vgi3HhokXXQDZ1daF1spWqBqqyhSJyYKgQ4BD/Q4BAHbyd5JddB6LPWA6wBRHXI/A2MsYQd8FIetWFjrqOijXvTazFpfGX8LAdQNpNVgWmwVNS02ZNrL+W/wZrVSliNwdibjDceio7YCaiRo8lxJ1SYtBFjg1+BRy7uTg2vRrGPnrSHLd2pv73Ef8M/Gxb/Mj/nHIuJ6B0AWhpC1md4iFYiQeT0RVUhUq4ivI12N+i0HsoVjUZdWhMKJQZrHQZ70P5obNpXRZSKFqoIrPhZ8jYEcAHMYzWz31huQzyXjy+RNayztAFIZXpKyAgYsBSqNLaZZyg3cNxtq8tTI3iUO/HYqxx5gJkffBxHMTMfPuTNrriuqKsAm2QUlUCUpe/vX2fG3VbTg34hzijsTRjjWXNOP8yPMUOx0pOAoclEaXImxTGEVVJ0VpdCmyb2fL/FypCpbJUqS1ohWHnA7hC9YXuDX/1od9IQArklfQclH+CjTkNaD8TTnpod4TRc+KEPNbDOP99U+Fjr0OhnwxRGZuYkddB074nMC1GR9uYduQ34CkU0n/2QnKgPQ5HfAps1rr/0VEbItAxLaIXgn4vyNqM2oR/3s8WsrpNni593LRWNgI/63+NFJLIpKA38zHsy+e4f6a+7T32gbbQttOG+eCz5FCjKDvgzD7wWwccTuCXyx+wY05N1CZSNjY3fvkHv4I+APA2/vnyuQruDr16gd9H14zDzfn3ISQJ6RZZr0P5kfOx5bGLdjWuo2mxO+/vD8+LfgU7bXtePHdC1pGQe79XHzB+gKXJ12mvN69OFn2uoy0/AGA8afeFmknnJ6A2Q9my5yf/bf6Y17EPJoNSkddBy6MvoBXP78iusYYwG/hQ0VfBQ7j6HP3zTk38bPxzzg1+BSjMrk6tRp12XVIOJaAzOvUOaMuuw6dDZ2Mnf5ioRiRn0fi8oTLyAql22k+++oZIndGYn3ZejIPqzvcF7mj75S+4DXxaPNcZUIlLoy+gKqkKmhaatI2toauhug7uS/j9xHyheBqcDFszzBKR2tBBNGVYjHIAkO+HMI4twJAa1krADASJmWvy/AF6wt8wfoCpdGljO/vDdUvq1ERU8FIutVl16HwSSEmnplII7HlleVREF6AvcZ7yQyM7mBz2Bj69VAserGIkbC7NuMavmR/iWdfPWM8rx91f8T3Gt/j6RdPacfqc+vx5tc3FDuinsi5l4Mrk6+gNpNqQSTtcLuz7A4ujr1IOTbu2DiS+Bm2ZxhC9odQvm/q+VQAwMhfR0K3D72jvya9Brn3cmHobshYwDjocBCHnQ7jTNAZxmfO2NNYJoHJYrPAlmOjPLYc6VfSGccaI3cjDNo5iNb1IRFLIBJQLf18P/PFgHUD8PRfT2ETYgNDN0McdDiIjroOsDn/7jAxUsWKpBXvzOeN/z0eZ4efZdwjJP2RhPTL6Yx2mABRLCqJKoGwU0gbZ5qLiXHrQ7LBpNC00MSUy1MQciCEsUuyKqkK446Pk2lJ3lLegjcH3tDspwAg+Wwy4o7EocZyALqUqIVbFpuFvpP6oqW8Bc+/fk77nVpKiXlv4jlqlpqUXHGc4ki9BySEjWrswVisK1kH/23+YHFYjCKJN7++wV6jvQjfHI7GgkYAxPhg6GYImxE2SL2YiojtEYjYGgGxUAwVfRXMi5gHM1/CljHxRCIernuIVz+9opx30qkkPPvyGeqy66CorkhTtKdeSMXliZfBa+LBZIAJbWzuqO9AVVKVTMLDcogltrZsJbsLWspbcHnCZaScS0HquVQo6Sgh5WwKo+2fip4K/Lf4Y1XmKlpnvoq+ChY+X4jOhk6Z0Qmlr0rJ8ZPXTLUCXRKzhBDL7BqE7DvZpLuNdH0x/o/x0LbRRtGzIoRvDqfMs/nh+Xj25TNGe9HOhk482fkEYZvCUJ1CiG70nfTxWc1n8Fnvg8fbHuN3z98Zrc4BItMx7XIa4zEpatJrEHc0jrbOSjmXAtMBppj9YDbtPTp2OpjzaA52S3YjZN/bsW+f6T5omGtAzUQNJ3xOIPd+LiNp1xv4LXxE7owkXF66zTUXx15EwrEE+G70RfbtbMT8FvNBf1falXll8hXasdwHufha4WukX0knukBkkF+eyz0x9Gvmzs93IfUCMScw5YVXJlYi5WwKJp2fRMmrkiJ0cShe73uNgesHkmRecVQxDjkeIvfKbA4b6ibqpFghYEcA3Ba5vf0jLGLdU51cTea4NZc0I3JXJB5tlB2foqKvAiN3I0br5UE7BpHxLEzQd9aHTYgNLAIsaHO71LnoztI7OOhwkHIP85p5OBd8DjG/yv6N63Pr8WT7E5JoD9gRgEXRi+A6zxUGzgZQ0laCjr0Oo4CILceGkpYS43eyH2OP1TmrMT9yPiNxMeb3MbRu255oq25DytkUcmyVormkGW1Vbbg45iKlflIRX4E7y+6gIq4CrvNdsTJj5TsFzz3Ba+QhPywfkbsiKa+Xx5SjKLIIbdXMNtV/d0w8OxFDvxkKLRsG6/nfYskc6r8KS2OXYtHLRf8V+0Qp+VaTViPz36gZqUFBTQE1aTW0eaYirgLnQ87j5Z6XSDhG5Of6bvSF53JPUizZUd8B3T66CPo2iLSAlr4uXQebeJkg6Nsg6NjpQCwSw2GcA5T1lBG+KRwNeQ2QU5SDoroi7RpE7o4ks4C7g6PAwcgDI7G5bjPNKQIgnHP2Gu3FV3Jfoex1Ge34zTk3cWv+LdRm1tJI8oLHBThofxBXJl3BneV3ZAodmXDS/601ds91tL6TPnaJdmHwrsGoSatBdmg2bY9aGl2KhlxC7NsTKnoqGP/HeEhEEtRl1b33OUnx/Mvn6KjtgIaFBm0d3l0gI82SBQC9vnq9jrcf8c/Dx86+j/hHgM1mw9nZGWw2G65zXZF+mWgP7xmczZZjY8GzBeC38okN478RezAWrRWtZHbIhDMTGC1FAODm3JvIuJ6B7e3baQUnacHoz0BKHNmNtKOcG0AoTQ2cDRC6OBRJJ5NoVgK8Jh6yb2fDwMWAZmsj/X46fXR6VavKQsmLEnTUdci0J1DRU8GWxi2M+QL/KZS0lGA3yo5ikyGFRCyByxwXxs4osUiMgrACZN/OxviT42mLhahvopB7Pxez7s+C1VArWnFl3IlxOOp2FK/2vqLZIomFYnIyZgoSfxf+W3anLA4LJt4mMs8p7WIa4o/Goza9tlfrj38SdOx14DjVUWZR5tL4S+A38zFg7YeTase8joHXyIPlEEvKYvWvgO9GX/hu7L0g+S50H/P+CfBa5UX487d1MYac/x3RVt2G67Ouozq5GnMfz6Up5qfdmCaT8NBz1MPa/LU4HXgaaRfTMOpXuoJSxBeB38yHWChGW3UbjrgcgelAU+j11UNtei1y7+WiMqESGys2wm2RG2xCCEWp1EYu/vd4xiJ8/O/xuLv8LjQsNLCuaB3lGFeDi+k3p0PTSpNxI99c0ozovdEoe1WGkF9CaHMRAHDkOYz5AFL4bfZD/+X9KRZVwNu8pZpU6ibTuL8xdkt2o+hpEU4Hnkb2nWx4ryJUhG7z3eA23w0d9R1or26HphWduAKI/KCKmAranA8QHYeFTwqRez8XufdzGXMObYNtZdrzSckhsVCMmrQaWsbKpfGXoKSlhBUpK2jZB1k3s/B091N8kvoJTanPUeBg0oVJuDHrBqoSq2jdfV1tXeC38MHV4NL+rhQeiz0YO0c76jtQGFkIy6GW0HXQhYaFBuW6Oc1wgpqxGtqr22lEdVNREy6MvoCAHQGUsa/8TTm4WoR6P/yzcOjY6TB2JE6+NBlioZhRFKWo8XbdwkSoXJ12FRlXM+C3xQ/+W/0p14zNZuPNp2+QoJKAdUXraKrj1/teI/5oPD6r+4xmcyYt+LdVtqHgcQHNnUHUJUJjQSOUdZUZ1cymA02RcTUDxU+Lgc9ph+E63xXJp5MZ783KhEo8XPsQD/EQky9NpnX73FpwC6nnUyEWiTFgHXWukiqZ43+P77XA0LO40ZDXgOk3p8O4vzHj9wGAyvhKlMeUY8btGYxznOVQS7Dl2TLtgIV8Ieoy65D3MA99J/eldEZ4LvWE51JP3Jh9A6kXUimK5saCRlwafwmeKzzJ55zyuYMtMe74OKReSIWeox50++iSiuMDNgegqKGIIV8MIfJNlOUxYO0Ayhz/ZOcTRH0ThYnnJsJlNtV6LftONu4uv0ucP0O24+JXi1GTVoOSlyWMHYBioRjJp5PhNMOJttaeHzkfz79+jmvTryHlbApG/TYKGuYaSL+SjsdbHmP8qfGQ48qhPKYcLrNdaGps6yBrWr6cFH8E/AEzPzPMeUjvem2rbsOx/sfQVtWGkP0htPMqiSpB4olEsEYMp71XiowrGUi7lAb/rf6U53LwrsGMVtRseTZm3Z8FVQNVnAs+h6rkKmyqIsbPwK8Coeeoh5IXJbi77C6mXJ7C2FHWkNuAtqp/W/4Ot4aWtRZEfBF4TTy0lLfgxqwbcJrphPEnx5PzzZv9b5B4IpE452vEONF/RX+y67U6tRqhC0MBEJ0+Ukuw7qjLrkP2nWwEfR/E2OkbeygWT3c9BVuOjc8F1IddIpGgpawF8iryJEGuaqhKWFc76aO5uBlKOkqYeXsm4zqH18RDV3sXtO20GQlQAGgqaCKzKntd5/WY+k28TGDiZYKI7RF48d0LrM5ZDR07HYw5PIay5q+IrUD0j9HoO7kv2XGe9zAPr39+jYbcBkw8S7WV7qjrQNQ3UQCIOUlqmV8eW46W0hYI2gWoTKhE2KYwuM13o91/z796jqbiJtSk1cC4vzFlzmgpb0H+o3xUp1Tjzf430InQoayzcu/lIu1SGmM3vyy4zneFvpM+lLSVwJZnE/OfDEtWWZBTkkPwvmDaXN/Z2AkWh4XLEy8j61YWNC01GcewtEtp4DXx0H8F3bZ88sXJ4GpxccTtCPpO7ovBnxPPl5SU0e2ji1WZqxjPS9AhQPimcNnZ7O/A1CtTIRKIGPeJgg4B6rPrZeZJsjlsmAwwQfDPweRr2jbaGLRrEPmbS8QSNOQ1QEFNAWpGarR9jqqBKlZnrcaVyVdwZtgZ7BbvxrmQc6jLJArVHHkOreOwo44g3303+0Kv74c7tIw/MR4SiQRikRgsFosyvvlu8oXvJl+EbwlH/sN8QjTw76WWgqoCxvw+Bso6ysi5mwMDVwNal1tdVh2GfDmErI9o22iTRfPqlGoIeUJMvzkdyrrK4DXxEHc0DsaexiiPLceT7U9gGWjJaG3PYrOgY6fDmC0GEHWX0IWhKI4qxto8ZtKvNqMWqRdS4b3Wm0IYOk5xhEggQvimcER+HgmnmU5gc9hozG9EwrEEWA+3JtYNf8Iqdtr1aTjmfYz2etyROCjpKPVq4S7F33Fva+RuJNMZZOyxsWirakPug1zo9tGFlhVzV9mHQJaA+a+A63xX9JnQh5FklqKlrAVpl9IQ/lk4pl2fRrHs13fSx9hjY6HvpE8hP7tHqZwddhb8Vj7t3jxgfQD8Fj78t/sj6Ju3VpwceQ5mhM7A19yvARB1svbadrSUtkDbTpuca5PPJuP5l8+h76TP6JDQG9IupqG9uh19JvZhvLeHfDEENWk1OOR4CEO/HYqAbW8dN3TsdeC/zR+593NR8LgAwT8Hy9yTdYeQJ0RVYhXM/c2xMGoh5dibX9+g/E05GStkPdwazrOdaWtx6djoOIV5zK9JrUH4Z+FQN1OnifryHuUhbEMY1IzVMPHcRFr28vAfhyP1fCpm3Z9Fsfu9s/wOOuo6sFuyG+lX09HV+v6xWB/x4fhfj3kfyb6P+Megq6sLXC4Xun10YehqiJp0umqFxWYx+hrPvDMT9Tn1uDD6AgDg1rxbjGRf+tV0pJxLgfVwa6IVvJutkpAnRMnLEmiYa8hcoPWGkb+ORNqlNEY/eSFfiK7WLvSZ0Af6Tvq0SSb9SjruLr8L382+jGRf6atSiATvr0Tpjqhvo5D3IA9bmrbQrIWqkqvwR8AfGLx78H9MXjBBUV0RufeJwOORB6jh2Yl/JCLlXAqG/0gvYDTmNyL7djb8tvgx+lT7fuaL8phyXBh1ARsrN9KuuaGrIRY8X8CodtcwpxfQPwS/Of4GE28TmRaifxZaVlqY92Qeulq7GO1FfTb4IP5oPMrflP+ln/u/xuPNj5Efno+dvJ20Y45THWE51JKWVfQ+mBs2F/W59VA1oj+P/ykkEgmEPCFYbJbMrKP3gXTM+yfAwNkAV6dcRVdbF2WR/3dGY0EjqpOrYRtiy7jRU1RTRMKJBCQcS8Ck85Noyjjg33YoDIRcwvEEZN/OxqKXi6BqqEqq78tel5GKuZTzKWR3HFNnlywrYGnHChMhJ8eV6zVXoux1GWIOEKrJ+N/jaWSfNA9Nx14HaiZqlA1CRXwFqpKq0HdSX0ZVesC2APRf3p/MLSt5WYKsW1nw+sQLWtZa0HHQwdhjYymfmXE9A6oGqih7XYbwz8Kx+PViRivr9qp2FD0twpOdT+C3xY8ydqubqmNH5w48WPug16LftRnXIOQJMePWDMrrPpt80FLagoq4CtycexPLE99m5EkkEngu9wRHgUPLRgEIhfbT3U/x7ItnmHqVmrfLYrPQb1o/OIx1YNw4Dt8zHIN2DkJ9bj00zDRopM35keehrKfMmPlnM9wGOzp24ObcmwjfFI4dnTsoXXxtVW04P/I8fD/zpeX5KWkrIWBnACwHW1JeN/YyhraNNhoLGmEbYivzPmJz2GitaIVQXUjbVEtVobKg108PuAq83PMSXiu9aASp7Rhb5N3Nw496P1IIJABwnuUMA1cDyCvJQyQQUe7/qVenojKhEoIOAeOz1FzSjN/6/gYA2NK4hfa5Pht84L3GW2aBvjdbcKuhVgjeF4xH6x+hvZreTVb6shRmvmZY8GwB4/sB+rMukUgQuiAUJgNMGMm4+6vuozS6FPOezMMJnxMI2BkAt/lulH9jN8oOS94sYcwRBPBOa8h9pvtIBbSeox7jutd7jTccxjtQ1soSsQSiLhGyQ7ORdjENI34aAdOB1Ge6NqMWcYfj0GdiH0oBY/GrxUi7lAZIQJtH+K18JJ5IJIkJpg7NpqImUvTHNBaoGqji5tybKH9Tjq3NW2nHWWwWJpyZIDM/SoqcuzkY8sUQaJhrgKPIgZK2EjgKHOTez8XzL5/DOsiaQvYlnEhA2IYwzLwzk3GPMuSLIQCLKML1tMXqautCW1UbTH1MGQtCIftDiHyY1edR2i8Y0H07vna1d2Gf6T6Y+ZlhedJyxpxMsUgMUZcIHAUOef+zOWxUJ1fjwjZiz+Q004m8PoN2DiJcVE4mQttOW6bF1MgDIzF412C0lLeQxej+K/pD30kf8srymHZ9GnQcdEjrvY76DhQ/J2z9B+8eDNOBplA3U6eIClrK3naGea5gnh8DvwhE4Bf0rG8ppCSv+xK6xbGoS4RfLH6BjoMO5jycA01LTXA1uHCZQ5DKrgtcUZ9dDxV9FcYxPWJHBOIOxWF50nLoOuhSxuTOxk7E/BoDl3kuFHKr+zrPzMeMNn7WpNWQ1t6K6oqwHWmLCacnUAp7eQ/zcH3mdYw5OgYeSzzQd3Jfynzlu8kXMb/GoOhpEe2cNSw0sDR2KQoeF8BiMHFvtlW34bj3cQDA58LPYTHIAlenXoXpQFPaHjTkQAjaqtpwZdIVeCzzoJB9tem1uL34NgbtGoS54XNp7x3560j0X9kfLWUttCJoU3ETYg/ForO+E30m9iHFrGN/H4sXe15A1CXC512fI3xLOK7PvI5PUj55rwItQFhzt5S14NXPr7A6azW5j1z0YhGSTichdEEowILMTtuYgzFoyGugkX2pF1IBFjEnVCdXg9/MJ8k+71Xe8F7ljfKYclwcexED1w+kWaNKJBIo6ShBXoU5hzx8czjyHuZhWdwyxu/am9jU3M8cyxKWoSqpCm3VbbTC8Njf6e5AOvY6lGdJyBPioMNBuM5z7ZWg9VjmAathxHfzWOKB+tx6lL0qY+yc6mzoxMvvX4LFZjF27OQ9zEPapTQ4TnWUKWhOPp2M0IWhmHl3JuO/Gb5nOJn5KIW8kjw8l3oi524OLo69iPGnxtPm0Ke7n8LE24T8DUVdIkgkEuy32o+2SqKLbWncUijrKqO9th0RWyPgvcab7BaU5XTSUd+B+ux6aNloQUVfhbafb60kxOlMa04psm9lQ8NcAybeVFvCrvYuZFzNAEDMh6IuEdhKbPSZ0AebGzZDXlkegg4B+K18KGkpvfczAxAE6aoMKlEtkUjgNNOJ0VJUFv5ue9tbC25B3UydMUtVz1EPnY2dOBN0BsH7ghmzlz8UuQ9yyXzwP2vrLAssFovRIrg7zo44i7rMOgz+12AagaRuqk6rqVTEVaAuuw59J/aFvLI85FXk0VjYiLPDz2LYD8PI/bPfFj882fEE/GY+8sPyEf1jNAK/DiT3dD4bfMBR4ICryUXsoVjcX3Uf8yLmkeNgRRzRQdtTkAIQ48RPBj9BLBRj7uO5NOFU4FeBGLBuAIw9jRmFhi5zXNBa2QqOAocm9NJz1EPQt0Hw3+YPUZeIsa7IhOZSonu++7pEipLnJci4lgG3BW7QsNDAhVEXYDfajrb/9FrlhT4T+jDWhnnNPJS+KsXQb4cSsUY9kH4lHbUZRJQNU2eg7yZf2I2ywy/mvyBgZwCG7B4CgMg+ltZ7+02ldoHzmnl4+q+nkOPKYdh3dCvoj/hz+F+OeR/Jvo/4R0AsFiM7OxvOzs7gcDiYdn0a47+TSCTobOgkPP21lchNoratNrRttbFbshsp51IYLdsAIOcOofCfETqDVvRpr23H2WGE9cqgzwch8EvZm0kmSLM8OurpC6Lc+7m4MukKJp6byJjHVvysGABoXWhSmPmakTYTHwr/rf7Ie5CHAzYHsLluM+WYgooCTLxN0F7TjqbipncWQP4MFr5YSHZcdofDWAeoGRNWAz2hqK4I98XutEWuFJZDLDHj9gzUZdUxFoREAhHMfMz+KxYKdZl1qMus+8vJPgC4u+wuTUUvhY69DnZ27ZRZrPwnojiqGDl3cxit9wD8R4tu4/7G/zV1XfbtbFyecBkuc1wYF63vg55j3t8dXE0uPFd4ynwm/47Qc9TD3PC5MguWDfmEdW5rRSut+7i9th3Zt7Nh4m3CaOvWVNyE0pelEPKJ7hINcw3aBqRnV8r74n2CyJPPJkNeSZ5WHDbzNUPg14GI3BlJyyEECHK9s7ETYoEYw38aThF5ZN3KQtTXUTBwNoCasRpUDFQohIuqoSplw1L+phyvfnoFBRUFPPviGVSNVDHl0hRKxteN2TegpK1EFk5ODDzBOL5NOD0BXC0uor6JgtdKL0ahRk/BSHc05Dcg/XI6TckPACN+JLKM4o/F0whUXiMPEVsj4L7EHY0FjWSGghT6TvqwH2sPU186QSmREFavclw5mQWVstdlODfiHMYcHUMjfKR5OQknEmA30o7RKtR2lC1UDFVo91bB4wJwFDmwHUkvnqnoqcBysCXuLL2DEXtHkKre4XuGo722HQXhBUg+nSxTaVoYWYhzI87BZ5MPee3eF0N2D4HXSi/wW/g0oYVYLIbFEgvk3SU6RDsbOykFSYtBFrAYZEFk9FS1YX3JevLYs389g1F/I7Io1xPdix/8Fj5jMaS3btbeoKKngoHrBsqcj9bkrpH53oxrGSh8UoihXw+ljEESsQTJZ5Ih5AvhtdILF8deRFNREz5J/QQAQTpoWmviwugL6KjtYDx3ZV1lVCVX4cmOJ/D9zPeDFemOUx0hx5WD82xnmtCh7HUZymOJDraeRJ62rTZWZ69G6oVUhG0Mo1mLNxY0oqutC4teLiLn4AujL0DFQAXjT45H1LdR0LHTIef9khclqEyohPUwazxa/wjqZuoYfXg0Y1F3wJoBlEzNnmgsaITDOAf0ncy8lmZz2NB30kf27WyoGatRRA2FTwqRfCYZ406MQ79p/UjlfJ/xfUiCQ81IDdZB1tCwoBKNKnoq4LfwcWrwKWyo2EAr4vps8MFhl8OIPxJPy5HVstbCLtEuAMzdsvJK8mDLsaHAawFbRO1mZLFZMPIwgsUgC8ZO75r0Gjze/Bi593Mx59EcSl6R1AKOKSOxs6ETd5ffRf+V/dFe247YQ7GMpHTPTlpBpwCnA0/DbaEbxp8cTwgd27ogryKPtqo2FIQXQNNSE6ELQzHr3iyiiF7TDk0LTbDYLNiNtMPOrp1gsQgb2bgjcVDSVmK0LWyvbcern1/BKtCK8r2shlrJFCSw2Cy4LXBD0qkkvPj+BaWbAQBSzqQg+UwyzPzMoKKvQvsdrQKtkHE1A0fdjtI6JjobOsk8NWmh8n3WeXXZb+27+C18pF9Ox5RLU8jXqpKrcGHUBfJ6czW5tPFNzUgNn9V+xtiZLKcoR1sPS7vzpc4qtiNt8WnRp4xdxGY+ZhCLxFiZsZLWRW7kYYRZ92dBv58+I/murKuMe5/cg0QkoXW7tZa3IvqHaACEWKP78x6xNQL6zvroO6kveS98qG25oroiVA1VaWItEV8ETStNzLg1Q2YWuJmfGaNF9vOvnoPFYcF5pjO4mlxGZ4jCJ4XIuZvD2L2noKIA8wBz5N3PIwuz3dFW2Yaa1BqZ37W1shVH3Y/CfbE7o+iuMKIQt+bfwozQGYz7qtQLqYRTxaFRjFluHAUO/Lf5k52DfwwiOpKlRWEhT4jEPxKh11cPXp8Q40H3vEwmaFpqYuK5iSh5UYLGwkbaPJV2KQ3Jp5ORfDoZu0S7aGNg9N5oVMRUoN/0frSCeUNeA+pz62Huby4zi8vA1QDjT41n7PSefmM6WGwWhDwh5LhyuDnvJtIvp8N+rD3kleWh76SPtItpUNJWgoYZQZor6ynDbYEb2PJs5D/KR2FkIc02tehpEa5OISz6F79aTJtD7yy5g8LIQuzokJ0TZzLQBNr12rRnWsQXoamoCYP/NRheK73I+5SjwIGSAvF8vtr3CmEbwrAoehEtK7o3pF0i7Hq7ZwmWx5Tj3op7AIgalaxnRoq/4942+XQyAMJ1oCepIugUQMNMA+NOjvvL9rfS8dpzmedfTvZlhWYh5kAMhn4zlHZfSSHN8u7NEr25tBldbV3Q66uH5LPJiDkQg5IVJRh1cBRKXxIW/eWx5ZQ4h4DtAWRGderFVJTHloPXxIOQJ8TjbY9hOtCUdL4wHWiKod8OpXSlBu8NxvAfhsvcL0nH06qkKhrZ1z07WRbUjNR6rd1+aF6fmrEaZj+cjfMh5/EF6wuKK9ukC5Mw5MshOOR4CL6f+cJvqx9EfBHN/UhJSwnZodmQSCQ0629eIw+vf34Nn00+jN2Kg3YMgudSTxh7GTPW/srelCHlbArEQjFlbpZ2b5dGl+Kk30lKxmt9dj3e/PIGAD6SfX8R/tdj3key7yP+cXi87TFifo3BqsxVtMUor4mHH3V/BACM2DuCXGR21HWALccGV/OtQpMJw38cjsG7BjN243A1uRh1aBTur7yPxvxGhnf3DolYgkvjLyHnbg528HZQPkPLSgueKzxldgwG7AiAyzwXGHky2wxo2RD2OH8GFoMsMPzH4aRCpTu0bbUx49YMfKf2Hdqr2/9yAqsmrQZ/+P8BHQcdWARYUCZAfSd9PN7yGHKKcrTNgqqhKgK2B6ClrIVUJPeEmY+ZzEXs756/Q8gTYlXmKtoEyWvmIfNGJrJuZsE8wPy9bCm6Y8TeEWitbP2g97wP2mvaUZlQydjZKcWfLVb+XdHV2gUVfRW4zpcd+H13xV20lLbIVODKQn1OPWozamE11IqREH4XymPKoWWtxVj8yHtIFKuZVKz/ryJsUxhaSlve2S3ydwJXg8uolpPi9uLbqE2vxWe1n9GONRY04s6SO/BY5gHPZZ6U7DMAGPrVUDjNcEJBeAHsx9pD1UAVflv8wGvi4cX3LxCxLQKTzk+C8yyiC+neqntoKmiiZNfcWXYH7dXtmBFKVQL2hpayFhzqdwj8Fj6MvYxphI1UtSnqEtE2FgAw9dpU8Jv5qEmvoR13necKM18zpF1Ow+ufX2Nl+krKxp7fwkdWaBa4mlw4jHUgFaP5j4hgdqnFYvfOlskXJqMhrwGv972Gsq4yNK00ARl59f5b/dH/k/5Q1qM+c4IOAYqeFkHLRgu6DvTsMgBkvlLwvmDasfhj8bi77C6G/TCMNt4raijCbYEbzP3NcTrwNFQMVLA0Zil5nKPAwczb9Lxb6fX4QecHGLgYYObdmbT1StGzIhQ/L8aQL4fA2IsuPFgYtRBxR+JwZ8kdzH08l0L2tVW3oTK+EhaDLBg72SriKiDii2RmsnAUOVDSUaLYc2feyETowlCEHAjBjNszcHHsRTiMd6ApUaX5JT07AwGCVH2w5gHkleUxbM8wxm5YFT0VmYp3VQtVDP9pOPjNzIQcANgE24DXSLUiyw8jLLpaylogryxPI++VdZWxuX4z+K18RtI05rcYosh5cBTjHHttxjWUvSrD2ONjYTPchnKMLLyywFhMBwg7u/rsehi6GVLOreRFCeIOxxHEbnEz2f3HYrOwrW0bWdhUN1eniJOcZzrj9uLbEHYKYepjSo4j3SESiFAaXYq4w3FwnedKK6JG7orEi+9fQNtGG8uTltPWvb3Zgefez8Xzr57Depi1zM4u51nOjOdVlVSFsI1hmHRhEtnl21HfAY4CB01FTRj6zVBUxFbg7PCzmHBmArJCs/Dqp1dYnb0aC18shJqxWq/EZW1mLeqz62EVZEUr3vwx6A9omGlg8avFMt9flViFZ/96BhNvEwrZ92jDIzTkNqDsTRlZLOsJTUtNRstUh3EOCPo+CMmnkiERMRfq3Ra4MZIDLBYLEkggEUsgEUloa9am4ibo2OkgJWgdwGKj+zeWV5LHvIh5EHQI0NnQCa4Wl3KPJp1KQu79XJgHmFPI9476DtSm18JpphOlICjoEODsiLOwDLSE3xY/mA40RfzReBRGFNLIvqrkKjQWNMLczxxcTcKquDq5GkaeRiQJdn/VfSSeSMSWpi3QstbCkpglyLiWgbQLaRAJRLi95DayQ7OxtWUr+Vty5DkkQRO5KxJ6ffUoZF99bj2aipqgoqeCl9+/BJvDppB9vYEjz8How6Oh46BDjgMVcRW4NOEShvxrCNgKxN866nYUTjOdyIw2KRynOELTShNJp5JohUd1U3WsSFmB5pJmZN7IZMx+b8hrQPjmcHA1uAj+JRhcDS4sh1hi7uO5yHuQh1d7X9E6qp59QeSMui92h3WQNfitfHQ2EEIJaaG/q62LVvCTQiwUg9fMIzo2/p1Tqe+sj6lXp8LQ3RBFkUVoLmmWec8DBEne3YIxKzQL91fex9hjYxktXqVoq2qD9TA6OQ4QROGavDWoTqkmhaYSsYSMxKhJrcG9lfcwaMcgmjXku9BW3YbMG5lwmeNC6RxNOZ8CDQsNfFrwKbrausBr4jHOQw05DaQItzsmnJlAzgfea7wpGXFNRU24v+o+6WbTnTDpDkG7gJajBRBOCVICvqWshdFd4cTAE2ivbqe59ADEtW4ubUbAzgDGOS7zZiYitkWgraqNtHUrjirG011PEbAzANZB1mDLscksS+l36n4eQp4Q91feh/sSd0oER0tZC4Q8IeM5cxQ44DXxEH8kHk7TnWhj+6DPB0HbVhtKOszzTOxvsVDSUsKy+GW0Y6kXUvF091OM+HkEWGwWvD7xIusFbdVtOOl3Em4L3GSK5+SU5HDS9yRZTzIPMAdbjo2JZyeCxWIh7kgcIj+PhO1IW2hZaZGEuaaFJppLmxH+WTgGrBtAI/sMXAxgFWSFzoZOxvmz75S+MHTvPU/Pb7MfGvMbafdo5s1MtFe3oyqhCu017eRaq7OhE82lzdCy0oKhqyG8VnnRujvfhedfPUdtRi0U1RXJ57o85q2bUPrVdEaS+u+OWfdn4fbi24yZ33/4/4HOxk58WvDpX/Z5jlMcoaCmwJi1/J+i6GkRCp8U4oTPCUZnD6B3ki8/PB/3VtxDY0Ej5Lhy2NG5Ax5LPBBzIAbpl9Ix5vAYIrPS1YBWm+ts6AS/hQ8Ncw04z3Qm9ya8Jh7e/PIGb/AG12dex/LE5TDyMKLZDbPlCNcQFptFczhQ0lbCTv5OdDZ2MlpoCzoFKIosQvbtbAxcP5C2FzwdeBqqhqqYfHEy7b1Fz4oQ9XUU3Ba5Qb+fPvQc9d6rGUBBRYFy/3e1dZH1JI48B+qmhDDNwNUABeEFiPo6Ci5zXSgC4fbadjxa/wi+n/nS9tyqRqpYGrcUiuqKNDcT4N0E54mBJ8j/th5Or3VIIyxe7nmJQTsHQUFVgbCu7mEN/BH/bHwk+z7iHwfpQjDnTg5tc8mR58AqyArlb8ph4Pp2MD3qcRRKWoQ1RtmrMqwtWMtYLFA1UEX683TEHY3D0K+HUoofimqK8PrEi1SrfSiOehxFdXI1EUjbY7Nu6GaIMYfH4PX+17i7/C5m3p1JyTXQc9TrVS1lEWAhU4X/Puht4ucochD4deB/LYsOIJQknQ30CZzfyic7Y3ri9f7XiDkQgw3lG2jFu2vTr6E6pVpmLoLdKDu83PMSP+n/hM311G7Gtso23F50GwDQUdvxwWTfu1SMvSHhRAISjydi8qXJtC7K1spW1GXVIfCrQEZyU5r3yGRv8E+F3Sg7mTlbAGFpE380vlcCVBYOOhwEACyJWQITrw9T6/Fb+Dg+4DhUjVSxsWIj7bjvRl84TnFkzEP7fxUSkURmvt3fGclnkhG6KBQzQmfQOkXcF7uTlpk9oeugi9kPZuPR+kdIPZ+K7W3baf+mILwAj9Y/wsKohVDWVYb/Nn/IKcphv/V+AERXW/zReCx4tgAdNR20jvP26nZGEUZVUhVe/vAShu6GtPGJLc+GibcJVI1UZWZZquipoN+0fowbJulmo2fmGQAyY4TFYkEiltA6A8M+C0PC7wnganGxpWELdPvowmuVF/pO7ovq5GqUvS5D9I/R0LTUJAuH0qKv3+bex9nCyEJ01HXQLEcAYmyUWnQP3j0YQ/41hPZvbIbbYMqVKdB3pnf23V1GZHzFHIihXc+2yja017SDxWLBe4037Zo9+/IZXu97jaVxS2nElnQ9UhhRiJiDMTQbqYRjCUg9n0oT/3SH3Sg7zLo/izbGlb8px6Xxl2TmD4/YOwKBXwUyFiobCxrx7F/PMHDdQMo9XxFXAQU1BVgEWODa9GsAgOzQbNr7B64fiD4T+jDeIy2lLch7QIgdXOa60K5J4slEVCVXwXeTL1QNVWmb17CRYbAfaY8JJyfQ/vbjrY+RHZqNZfHLaDY76qbqyLmbg5y7OfBc7knryAGIYoEsYir3bi5KX5bi3if3GImg9MvpAIjOnp5kX8r5FMIafp4rfDf7kraEUhREFCDlbAqSTyfT5ueh3wxFwI4AhG0MQ0N+A2nRzWKxKEXo0b/RiTefjT7QttGG2wI3xu+UfDoZT3c9xYQzExiFYoknEiEWiAlrRxlNMWKhGIJOAdGd2u23cl/kDuth1nj9y2tk3cjC2oK1JBnTUd+BjKsZMjOGzf3NMe3GNHA1uOA188DV4GLJ6yUAiHUyv5kPl3kuSL+cDmGnEF6feMFhrANhoW+vg6biJjz/+jlsgm1oc3djYSNuzb+FitgKrMpaBUUHKtnnvcabsRAuRWdjJx6seQD7sfYw96d2egR9G4S4I3HganBRGl0KfSd9KKorojKhEoWRhXCa7kTkmErASPz6b/GH/xZ/xs/9I+APmAw0YeyUFXQIEHckDmEbwxD0XRD8t1L/xrN/PUPSqSSwQ7ZCLMcsXLq/+j6S/kiikGYAQRobuhqiz4Q+lLGNzWHDbrQdzP3NkXAiAXVZdRjx4wiIRWI05jfC2MsYTtOdkHUrCx5LPGjnBABvfnmDpFNJAIj8274T+yL9ajoq4ysR/3s8Lo69CN2+ukQXjBwb8kry4LfwyW6uyxMuw2KQBXw2+ZD3Xkt5C/aZ7gMA9F/ZH7PuzaKNBQnHEhD9YzRWZ6/Gmtw1tGc+6VQSQheGot/0fmSHXEd9B3Lv5cLQzRAGLgaU78OWZ0PDTAMKqgooe1UGsUAM/23+lD1mdxh7GtPEPwDRQWfgbIDoH6ORcjYF2zu2g61A3QtWJlQi62YWAGDQrkHganChrKNM5j2O+GkEXu9/jX3m+zA3bC6Rd7ncEzYjbEhLyaRTSXi49iEWPFtACmseb32M2N9iKZZpUtRm1uKIyxEAxNge/HMwVA1U4TjFEcVRxTg7nHC0sRxiCWU9ZRqBfsz7GATtAsyLmAc5rhy4mlyUvSpDa0Ur2qqJjv3EPxJxZ8kdzLo3C7Yhb7vN76+6j8ybmdgtpndaynHlKBlpAEH2SQkvAEj4PQFu890+OLNPIpZA0C6g5aTeX3Ufxv2N0VjQiPsr70NFX4VxDxK8L5gxYqL7eNSzgyT1YipJ9I09PpaxmNzVRuQ9MhGB0m5LE28TmUIYp1lOEAvEjOup6pRqRO6MxJijYxivV8yBGHS1d2En/21kAr+Zj+rUapk5f9276wHC5nFexDxE7orE7/1/x7K4Zbiz7A4SjiUAIMRkjpOptQqRgBCeLYlZQps7gX/nBvbiZDH/yXyZAjGHcQ5QN1VH7r1cZN7IhNsCN3L/zGKxoKCq0KuNpaqBKtwWuRH243hrxSqFdJ+nZa0FsUgMQYcAcopyEPKESL+cDo9lHowd/zp2OozZ0lK4L3RHVmgWXnz/An5b/BhFRFm3snBvxT3MDZ9LES1Ka0XZt7Nh5GkEHXsdcOQ5yL6TjdAFb61Oe44D74NxJ8fhxMATSDiWQJJ9b/YTXUCBXwX2GiHwd4bdSDvGvTwA2I2xg6CdbpH4n8Bnkw/ODjsLIw+jD86mexcCvwiEoEOAhN8TkHUri5HsAwgiP3JnJAZ9PgjWw6zRVt1GNEZw2FDUUITzbGfoOBANCAbOBthQsYEUMsiy8P1B5wcABHlqN/KtyENRXRHrS9fjpP9JNBc3yxy/GvIb8KvtrzB0M6TEKUjBUeDIJKhvzr2JzOuZAIB+0/rRyD4Wm4Xm0macHnoaLnNd4L7wrXils6ET5bHlqE6tRnt1OzbXb5a5V+iJ/Ef5kOPK0Trf67LqwGt+m+vKkecQbgA96pXRPxLrnT4T6c+OnKIc2iqJzGYmm2EhX4j67Ho0lzbDcrAlbX9q6GaIqqQqjDs5jlLzTruchs6GTngs9UDZ6zI05DaQv62KvgpG7pftkvMR/zx8JPs+4h8Daeur5zJP5NzJwaP1j2hkn4KqAuMCymWuC4Q8IV7//BoAES67Jm8NbQHVUd+BhGMJKAgvkNnh92dhPdwa5gHmCN4bLFMxIuKLwG/l04rmTcVNONTvEAasHUBR1UlxeeJlRi/o98HVqVdRlVyFNTl0q6nm0mbE/hYLu1F2jDkj/yn0nfSxtmAt2qvboWJAVfonnU6Csq4yYzh6c2kzsm5mwWqoFaPNp6qRKtKvpGO/1X7G7JZh3w8jPMcZOjTVTdUx694sKKorvtOOoickEgkSTyRCy1rrTy2kO2o7UJddx6gu07HTwdLYpYxdCQBRYEg6SRQxem7A/s5oLm1GR20HY1EQACK2RyDnbg6WxS+jFYaLnxdD00qTcVH4LjhOcUTGtQxGFf67IMeVg/tid5lkntQ2+D/F38Xi5H0QciAE91feR9LpJNqC9O+K4qhiUiXOZKXiOtcVNek1SD6TDLtRdpSFPFeTC9sQW/Bb+WgppdtC16TVgMVmYfrN6dDrp4e6rDocdjqMwbsHY8GzBRB2ChH9YzSZUdAz7w2AzI6+5DPJSLuYhrSLaTRyStVAFXPD58r8zmmX03B32V3wW/hwmuHEqHJ8F2xDbCkFOymsAq1Q/KwYQ78hci+UdJQIUsdIFVaBVqhOqUZ9Tj1jcSXlXAoSTyRizO9jGDvcX37/Evnh+TD3N4eyrjJlLFDRU8G4k+Nwe9Ft5D3MYyT7tKy1UJtRi4LHBTT7VCNPI1TGV0LDXANXplzBtGtvbcKlXYPG3saMimU1EzXwmnhIv5xO2tdIIa8sj2nXpiHuaBxjDqHvZ75wnuUssyM75XwKuBpcRlLN0M0Q406OQ2V8JdIupmHqlamUjZ6IL8JJ35NwnuNMy68SdAqI3J6qNsrrRU+LoG6iDkM3Q9gE26A+px6jD9NJJll24gBg6mOKZfHL0F7bziiiyLiagbyHeYg5QLgz9FQ2KxspQ0VXBUK+EGw5NkUYxeKwiP8xWBk6jHfAq72vAABWQfS5tyG/AVenXkXfSX3ht9mPdg9OvjQZcUfiZHaM7ezaieLnxYzzr6aFJgzdDJF8Jhlqpmo0+7Qbs29A1CXCqEOjaAUIBRUFKKgo0HIZxSIxKuMroaynzHhOoQtD0dXWBe9V3nj9y2uY+5vTbKm17bThOt8Vhm6GjPfY8sTlAEt2rtC9VfdQlVCFstdlmHJ5CqV7StrBVvamDHr99Chr6ZayFtz75B7cFrrBerg1LAdbUq6bir4KRF0inAk6Qyv+eiz1gJAnhM96H8rzpmWtBYlEApFAhP2WhFiCq8ml3WPpV9JREVuBgesHMmag+G/xx4O1D3DU/ajMdYO+sz6j7ZvdKDukX0lH9I/RiP4xmiROSl6UIHxTOEwHmCLheAKeffEMn6R9Qilcl0aXkkQ0kx1+a0UrOhs6Gc+nuaQZYRvDAAC6femdAH0n9wVbgY2i1+lo0bUG8PbvCzoFiPomCrwmHjyWedD2HkzqeoC4tiN/HYm7y+4SlsAKHIz4cQQU1RSxsZIoisYejsXLPS8xL2IejRgFCLGMnLIchJ1C8jv3X9EfqgaqeLLjCQDAf5s/KVYQC8VoLCDW5I5THaHbVxf2o+0p1mkZ1zLI/+Y18hjHGMcpjtCy0YKqkSqjLZdUQNPdDrOltAW35t+C/3Z/RGyPgE2wDWkHa+hqSAoAzHzNGLsOpIj5LQYF4QWYdH4ShayXfj9eEw+u811hO9KW6FCEhLLOswy0xKTzk6Cko0R5ZpLPJIMtx4aGuQY66jqgZqxG5i/aBtuiMqESEdsj4DrPFUbuRvDZ6EMQz/+Gub85Yn+LxYUxF2jWgEraSnBf7I7EE4lQ1Hh7vSK2R+DFdy8AELlCB2wOYPwf42niAtOBpuA18bDXaC+Z5SYt5raWt+IbpW/gNNMJDuMdaOusPhP7yMwUFfKEaK9pB1eLS/6OLA4LOzp34KTfScgpyWHs72PRVNyEN7++gedST0ZrTSaoGalh9OHRqIivQFd7F/lbTb06Fa0VrURmHwDn2fTOZICwHZZIJPT8MrGEcX4CiHWTVADUVtWGxsJGmhiG18zD482PMeDTAbQ9pPQ5aK9pZ3QUAXq3XTNwNcDUq1Nl7rWCfwmmkRr2Y+xp8R7nQs7BdKAp4zqLLcdGwrEElL4sJS0EqxKryOMF4QU0sq8uqw5HXI9g8L8GMz7Poi4RxCIx5LhyjKSXpqUmqlOqcWf5HbgtcKM4+hi6GcLQzRDm/uYYuH4g5ZlU0VfBiqQVKHxSiF8sfsHwn4bTxGSHnQ/Depg1LUsw6tsoxB2OQ0ddB5bGLYWCqgKqkqtw1O0o/Lf5k8+NvrP+n44/SbuQhvQr6QRxy3BLlTwvgbqpOu35sQ22xajfRuHZF8/wdPdTuC10g4aZBozcjRD4VSB07JkdpN4HpgNMMT9yPsVdw22BG/If5cN/q/97R6P8nfa2IoEIhU8KoWmpyegMEvhFIKpTq7Hfaj98N/v+acF/d3A1uTDzNaNZ2f8VUFRXxNijYzH2KD2DU4rQxaEoflaMrtYusos4dEEoip4VYUfHDixPoK+NpJbV7bXteLr7KfT66UHdRB2mA01pa63kU8kwcDZARVwFTLxNoGasRtyrttrgt/ChaaGJ7NvZiPo2CiG/hJBjhZT4cphA73rvautC7KFYKGooou+kvrR1q2WgJdhybIz6bRSjW9O8iHmoz6nHCd8TtL1P34l90XdiX+Tcy0FlQuV7zyMFEQUofVmKUb+NopGQj7c8Rt7DPKIbsaET16Zfg8s8F5odp5mfGbpauxi7PMUiMRTUFNBvWj/GTrvbi28j9XwqAGBlxkpKdz1A1BDqsuqQcS0DCqoK5PgWdygOtZm18PrEixBLfMR/Hf/LMe8j2fcR/whwOBw4O79ddPtt9YPbIrf3fn/QN0EQ8oTQd9LHm1/eoL2mnViU9wirv7/yPgrCC7A2fy1to9Zc0oxzweegZqyGwf8azGiB1htG/DgCXe1daKtqg7KeMmmXAhCFgDf732Dg+oGMirwHqx9A0C6gEWJS2ATboLOeuVDwLqiZqiHjWgZuzb9FU+u0lrfi5Z6XUNZT/q+QfQDRqclUzOI18hhDb8ljpS3wWOrBuJkP+SUEWjZahA+7DMWfrGwfBVWFXm1neoUEuLP0DgDIzATpDU3FTeA18hiVS/LK8qhMrMSL719g/MnxtMWM10ovIiNC9M/qrtpvuR8SsYTxejWXNCPlbAo66jsI66semcnSboA/AyZy5X3BUeBg3PFxMo8LOgW4v/I+zAPMe7U+6vUzeox5f3cIeUKkX0mHiqHKP4bsU9JWQp8JfeC+xF1md2fu/Vw83vwYi18vZiywMHWaAUDC8QS82f8G68vWQ0lLCYIOAdwWuMHQzZAs8MhSR74LrvNcoaiuSLFJel8oaSnByMMIHAUO+k6hkzY/6PwAZV1lKGoQuaj9l/cnj4VvCceb/W+wJncNY56L0wwniiI98WQiwjaEYerVqZBXkYeOvQ4Wv1pMimjEQjF+MvwJVkOt0FTUhIrYCtxfeR8z786kCW0G7x6M9pp2/Gz8M80+VFFdEe4L3eE8y1lmoQ0AXnz/Ao35jTSyb+btmRALxbi7/C5tHtWx18H2dnrXphQeiz1wZ+kdVCZUMh7nanJldvMYuhoCEuCk/0kMXD+Qdi+FbQiDjoMOI9mnYa4B94XuuL/mPoqeFtE6FNqq2tBS3sIoWNLvp4/licuRdjkNVUlVZNfgjNAZgIS4dxOOJWDek3k0+ymAsOS5Pus67Ebb0cQ4copyMouJADD+1Hhk3cxCfW49rTOUw+FgdcJqRP8UjW+439CybIK+CULQN0F49fMrNJc2I2RfCABCZGMZaAnn2c4wcmf+bEGHAFWJVahKrCKIhB5FHa6G7N8JIFS5sjrmLQZZYMHzBci5m8PYmRDySwjF9qo7moqbyFyU7veuoF2A4wOOw22RG8afGI+UcymEBds2glBurWgFv4WP2EOxeP7VcwzbM4xG9lkOtoSBiwFaK1ppOSEAs8ChO5JPJ0OOKwe3RW6EvW43SFXAvpt8aYIDLWstzH08F0VPi3Bj1g3Muj+LRpLqO+lj8O7BZIEj9UIqlHSU4PWJF37U/xH5D/Mx59Ec8rMkEgnaqtrIri6TASaM3S+2IbbkuC6rg08sEkMkYLa9V9JSwoJnC9DV2gUhX0h5fiQSCexG20FRQxGqBqpk8aXftH4w8TaBnqMeWitb0W9aP9r6rDymHC++e4HmkmaE7A+hFXzW5q/Fo42PcHPuTVrOr4qBCkIOhMC4vzGjNb39GHsIOgSw/P0a8j2mAOaa5DFhpxBR30TBY6mHzOJfRVwFYg/Fov+K/hRi7cqkK6hKqoJtiC3GHqe+V9QlQsqZFHgs9YDJABNGksPc35xGAur11cO5EecAANNvTSezDgEiX+busruwHGKJ2vRaeK30omUkWQ62RPC+YPSb1g9qxmoQdYkgkUgov5OJtwlMvE0gFooJW18VeYp95eDPB8N3ky/lfLWstTAjdAbUTdXx8vuXyL2Xi/aadgz9aijl88M2hiHjWgZ2iXcxEg81qTXIDs3GpfGXMHjXYMq+qT6nHof6HcKgXYMoAozu6zwVPRWa9W3qxVRSlAQQYrKeOaBPdj5B3oM8mPmZwX40vSvVaYYTRF0iRkJZ3UQd446Po6xpk88kk4SFjr0OvFd7Q15FnjHzduSBkRAJRFDUUCR/r4HrBsJ9sTvqc+pR9roMzrOdGcdPlzkuuL/6PuN9X/a6DKcDTwMgMraXxi4Fi8WCHFeOYtn45tc3hAXkDKf3LtIChNVo7MFYuM13I/f7NsNtEHsoFgAYiU0pTH1N0dXSRbvvfzb9GVrWWlj0YhGef/0crZWtZFe2kbsRjNyNUB5bjgujLiDo+yDanKOip4Kg74IYBZ/S/X9TUZPM7yQWihH9UzT0HPVomXyqBqqQV5HH6cDTGPnrSNpxQ1dDNBY2Iu9hHkx9TGWOnxWxFVDWVYZYJEbapTRoWWlRhI+NhY2wGmqFeRGE8Hpp7FKIRWK0lLYwinOVdZXhucITmpaaZDZed9yafwtpl9KgYaGBT1I/oe35WytbUZlYiYTfE2TGd+jY68gkuTgKHKgaqlJqMlLYj7GndPC+3v8adZl1yL6djbZKwoJW1CWCqEsEJS0luC10g3F/Y7gvdoeCmgJcZrvQ5hGAiHo4P/I8tGy0MPPOTFqR/vG2x6jPrSfucxlL2rI3ZYT1fQ8ysbOxE/dX3Yeeox76r+xPXi8DFwPSoSnnbg6STiVh6DdDZVrfy0LPfcf7ZIh3x99tb8tv5uN8yHkoqCpgadxSxuvBkSds75nukT+DhGMJEHQKehUbSyQSJJ1KQt9JfXt1I+iJ1spWQmBjpSnTVr4ipgKqBqpYlLeIfE3q6iEWiWnuY3c/uYv4I/H4JO0Twr72cBzUzdTJCBW7UXYQdAqwMGohmkuaoeeoh+KoYtyYdQPTrk+DwzgHNBU1YcKpCeQ4JugQoLW8lZLp7DjVEToOOoyEamdjJx5veQyAEMb3vOd6dt0yQcdehyZe6A770faMWdCyoKSlBPux9sgPywdbjg3X+a7ks+62yA0GbgY46HAQVsOsoKihyLgfc53nSuxdGX6rhrwGnB5yGv7b/Rlrzub+5qhKqsKATwfQ8oMBYp8okUiQcCwBLDaL3GOOPjwagg4BBB0CFEcVQ8deh6zDlr0uwwkfwv7zz9QxP4KO//WY95Hs+4h/BCQSCVpbW6GmpoasW1koe13GSNa017TjypQrKIkqweRLk8kgWODf3TgL3Smt2z1hO8oWqsaqMidJFoeFwieFAEAuZLuj6GkRSl+VkgWZnkg5l8Jou9Bc0oz0K+mMgd3Av5XqLJDt4D0hx5VD7r1cmd+rN4TsC0FTYRNSL6TSis+GboZYk7sGF8ddRGd9J2NX4X+CpqImZN7MhMUgCxi6GVIWGEO/HgptO21UJlbSsoH0nfWxk7+z18LugDUDSGVuT0TuigSLzWJUJkrRkN8AFov1Yb7VLELZ+GftDE0HmkLYQd/sAESBqiSqBJnXM2Uql5jycd4HFXEVCN8cjoHrB8JhLF1R9d/EhNMTUPamjLQv647SV6VoKWvBlCtTZG7ii54VoaW0pdcszr8aYpEYrRWtUFRTZLSjuL/yPpJOJaGtuu1Pk33dx7zu1+X+mvtIu5CGTdWb3ltF+VdAyBfi9qLb8Nnkw1hQL3tVhgmnJ8Bl7v/d7/CfQr+fPqbfnC7z+L2V91ASVYIZt2fQCgUlL0twZugZjNg7gtGGxXm2M4w8jMgio7qJOsb/MR4AUagveloE04Gm5PiS9ygPgnYBaWsJEDlk/FY+TQktVSur1EvgAAEAAElEQVQzoaO+A69+foUX376AXj89rExbSTluM8Km1/wiu1F2AAuE6rNHwUnHTgc2w22Q9yAPufdyMWLviF47WKUF7edfPUd1SjUAIPDrQAzaQWzSJGIJDJwNIBaIURFLdDgWPC6AiC+ibYrMfM0w9JuhyHuYJ9MCprdO/Bffv0Dpy1LGTjWRQITi58UYfWR0ryrs0MWhgAQYf3I85fV1Resgr0IvAvBb+Li7/C6sR1jLXHcI+UI0FzeTGTndMfnSZGTdzMLX3K8x/cZ0RrJo1K+jMOrXUbTXo3+KhogvwoBPmefAhvwGPN78GAqHFMh7SdQlwsO1D6Fupg5jL2NE/xANQbuARjbeW3kPeQ/ySIuf7hB0CNBS3gIlLSVwtbi0ooGqgarMdYx0zNN11IXzbGcaGShF7r1cVCZWviX7RBJcHHOx105VA2cDzH44G4VPChlzqxoLGiESiGQWvoqjiqGsowzdPrqM6w5FNUXG7ERAdjYTADze/BjpV9KxJGYJ6rPr4TjFkbDMVOBg2A/DyOJ6ytkUytpyzqM5uDT+EpmfKKsDJe1SGu6vvM9o31efSxTi+c18eCz1oD0/Gys3gsVm0YRvAGELGP1jNNbmr6WtkRTVFGEdZA1tG22ic6zHfJF2KQ3XZ13HtGvTSHL0/ur7MHI3gm2wLWyDbSHqEiHzZiashlohclckYg7E4JPUT+A8yxk2wTZwncec42voaigzpxIArky5Ag0LDdq42B2ZNzLJIlX38fjC6AvIe5CHvpP7UmyOVA1VSWV7v6n9GAUgbgvcUJlQiZSzKfDb7Ecj+wCiA4apS1xJS0nmWlYKMz8z5PWfjnYNY3SXYShqKGJV5ipKx1Z3RO6OxPMviXvIerg1SdZ0NnSiPrceeo56mHJ5CrneFPKEyLyZCS0rLdSk1UCvnx5uzr2J7NvZ2CXc1es5SqFqpAq70XboM74PUs6lID8sHyN/HQkVPRW4LXCDnJIcqbyPPRSLnLs5mHxxMrgaXNrc96vdr7QcVSlaK1vxi/kvGLhhIIL3UrNaexZtG/IbkH45HZ4rPLGzayf2W+0nrQtLXpSg4HEBPJZ6QNNKEyYDTHBxzEUYehjSyMAxR8bAdqQtLk+8DLeFbpRjXC0uPFd4UoRFPdd5YpGYJA8UVBXA5rChbqIOriaXPJ+e89/VqVeR9yAPKgYqjDmqUsh6ZpjA1eTCYrAFkZdmow11M3WaDXV3cOQ5lHmorboNqedTYRNsg1l36XnaJS9LAAlRrKxNr0VHXQft36ibqWPghoHIf5QP80EEeSkWiVEeU47iZ8XIe5iHoO+CMGDNADjNcPqggrjUPnj04dE0uza70XaYZTWLRjR3h0QoQfqVdIw7MY4iorAfY08WswvCC1CbWUuxYG7Ia8CN2TeIzxlJn885ChxUJlQi42oGAr8KpKz9JSIJXOa6IOVsCrLvZDPu1SK2RyD6x2g4z3KmkXkAUStQ0VehZPWSf18sQcbVDDze8hhL3iyBibcJmkuaUR5TDlMfUzJaRBp9IegU4Oacm3CZ60Ih+5hEmGwOWyaxoWakBhMvE4QuCIWKvgrtupgHmCPtUhoU1RTf5uN2wzGvY1DRV8Hm+s00Qcurn18h5mAM5obNhaalJuEM8O9rym/hI+FEAow9jbHkDbNwdMTPIxCxNQIZ1zLgOMURBeEFKHxSiJXpK8FR4KAwohC/e/yOGbdnwGGsA7k27DupL3hNPOzR2gP3xe40cai0RtCY34jW8lYa2VebXov26vZehVMLny8ks0u7oyiyCAqqChi8ezAcxjswrovrc+uReT3zg2NHDtgeAFeTi6UxSynroLqsOpwafAo+m3zeGX0ia2/7v4KckhycZjoh7WIaCh4X0NaAEdsjwGKzsCyOngn5ZxHzawxEXSK82vuKlvsqRfHzYtxedBut5a0fRKY+Wv+ItJ13GOfA6BDzSeontNcKHhfA0M0QDbkNyL2fi8r4SrRWtGJ+5HxyjXfE5Qh28ndiU/UmtFW1oTq1miTDL469iMKIQow8OBKGbobganIx8dxEGPc3RnttO361+xXqpupwmOCAwC8CaeJQAKRdNRNU9FUw9epUZN3KgnkAVczSVt2G1vJWqBqpoqOuA5oWmrQaWcKJBChpKzE6kzSXNqMyvhImA0wYSTNZMPIwwrA9w3DI8RCyQ7Nh7m9OClH7jO8Dy8GWyLyeCQVVBbgtdCOd0rrHInXUduAng5/Qf2V/ml0/V4ML9yXuNCGfFP1X9Je5nwKAs8PPknn1g3e/rZlLzzF0YShptb6tdRsUVBV6rat+xJ/D/3rM+0j2fcQ/AmKxGAUFBXB2dkZBeAHiDsfBd5MvPeOthY+SqBIAQG1GLfl66KJQaFpqwsjDCKXRpWgsaITfZj/aQsptvhs6x3aiqagJGuYalEKVhjlRIMi4nsFYLAKIhWXJixL4b/WnPdBP//UUlQmVGLhhINTNqBYw/ab3Q79p/VCdUo2E4wnoM6EPpYtk4LqBjL7vUoT8EoKhXw9lJEzeB1OvMHc5yXHliLb7Zj7aa5mzq/4T1GbUImwDYU+0NG4pJedCyBPi7rK7cFvkRtvAslgsJP6RiNiDsZh2YxrN9i3pdBJ4jTyZ1yz9Sjrqs+vRVt2GkQdGUiyuqlOqcXzgcQg7hTJ9w2WBxWJhB2/Hn54sLQcTNgRMAcTSQlHQ90GMnX9Jp5NQHlOOkF9CZNrCyULqxVQURRahq63r/5zsc5njIpOoM/M1w9SrUyndHd1R9qYM16ZdQ0d9xweTfZfGX0L27WzMj5z/wR1SnQ2d+MX8FwDMyiczPzO0lLeQ5M6fQfcxr3v7f+xBQnUs5AkZM9f+W6iMr0TqhVTwW/mYeXsm7fjLPS/RXtv+QQWlvwMqEyuRdCoJ7gvdaQRaS1kLWGwW4zPB1eTCZoQN4g7HIeVsCq1YYOJlgpq0GuzR2oOZd2dScr6OeR9Da3krbIJt4DTDCW4L3PDsi2doKmqiFJeff/Uc9bn1NLIPIIoFYqGYRoJ3NnTixbeEIp8pkPtd6Kmu7w6PJR7wWOKB6J+ikfsgFwE7qaKWmIMxeLDmAbTttLEmZw2xCWQBhu6GJNlXGFEIm+E2MPE2AUeBg/mR88Fr4sFlngvUjNSgZqIm8762G2XHSHg15DXg7PCzsBhkgYHrBzISoSoGKrAYbMH4/lc/v0LMgRhwNbnY0riFcozfwkdxVDF0HXRRl1FHKzbl3MtBaXQpY4FD0ClA2qU0pF1KQ1tlG83m88bsG8i8kYntHdsZ522rQCtAAjQVNtG6SnPv5+L+qvsYeXAkoxLVYbwDNCw0GOeCzoZONBU2YcKZCRTit6WsBQWPCxByIARK2kp4tfcV8h7m0cY4jjwHeo56JNnWHSUvS8jOnUkXJtEIsLaqNoi6RIyZQSKhCA+2P4DzUGdMOjeJdjz3QS4a8how7cY0SsYci82CmrEaea2D9wUzzv1SIokJt+bfQsmLEvSZ2AfTb9AFAKcGnwIkRBFv2vVplGOl0aVIPpMM7zXejJ19AJB5MxNPtj/B6COjyfVMQ34D0q+kQ0FNASlnUxDzawyshlpBzVgNclw5yj01+shoCHnU/OKyN2VQ0lbClCtTKBnPUhQ+KUTswVj0m9aPsdB6wucE2cnqPNuZVhRkck2QwsDVAP2m9UPx82IknU6C32Y/GikotfrsCTUTNfSd2JfS7TflEkEohW0KA7+FDyNPI1yZdAUrklfAyMMI/aYTnVyTzk+CWChGZ0Mn5JTkGJX2r/e/JuwlH8+jWbFXJVbRrmN38Jp4yLyeCd2+urRuRun16DedSuaJBCJiLFaUk7n242pyMeRfQ+C2gN4lCRCdVEP+NUSmg0ZbVRsid0fCbpQdpRsOAJ5+8RT5D/PRrDcfEg71N2Rz2NDto4ukU0nIf5SP0UdGU0gRriYXRp5GmHplKuWZ7GzsJIj+sUTXYFt1G7RttdHZ2Ikbs27Aa7UXNpRvAK+Zh5SzKYwF5QdrHyDm1xj0ndwXAz4dAIsAC9xaQOQpCtoFODXkFPjNfFQlVWHYnmHQsdfBsD3DsM9sH0RdIjzd/RStla1oLW+FiP+2E/P1/td4tfcVxp8cjz4T+9CKeo+3PUbWjSzMi5iHAesG0BTxNWk1yLmbAzNfM/J6N5c0I/VCKmxCbGARYEGxwi+OKsazL57BYZwDSl6UoLmkGU2FTTLnqT7j+zBm0KkZqWHM4TF4vPUx7iy9g5XpK6GgoUBZ58UdicOD1Q8AgLSCtRhkgS2NW9DZ2Imip0UQdAjw4vsXcJ3nCjVjNbDliX3q9BvToaCqgLxHeYj5NQaBXwWSRHvc0TgURRYR5FSP57SxoBER2yPAb+Gj37R+cFvgBodxDnAY54A7y+8g4Xcib21FygqIhWIaef9izwuw2CzKeFX2qgxPdjyBnJIcjD2NURFXgeQzyfBY4gEDFwP84f8HAKIAqaCmgPmRdCsxbRttgqTd+/Y1YacQJ31Pkv//pO9JLItf1ivJyYS2yjZEfROFgB0BFEvn79S/g91IO4z4eQSeffEMXW1dNHEPANiOtIWyvjLNHWjs7287YOeEzaHM69F7oxG+KRwA0X3aveArhUQswcB1A+E6n76OTjyZiJSzRF6hrAwxacau1Eq9O/LD83F9xnWM/HUk4zz4x6A/UPqyFGOPjyXH7ZIXJbgx+wam3ZgG9YnUOYYjz8H0m9MZLW3Tr6ajq7UL7ovcURhZiPBN4ZCIJZgTNofRNlrPUQ9eq70YxVZeK71okS3d4b7YHVxNLmPGlpySHJS0lBC2MQzZt7OxoXwDOe90NnQibEMYfDb6yBx3u1q7kHIuBRoWGnCc4oipV6dCLBCTY46OvQ48lnkwrmfEQjEcpzjC1Ie+f7UfY4/tHdsh6BAwCndn3p4JIU8IfgsfCmoKjOvD6pRqPN76GCP2jqAQJJUJlehq68K16dcoGcGZNzLxau8rhBwIgc96H/is/zCiDwC55zk99DQWPF0AgLCbL4osgo6DDqOIpSdk7W3/V1BQUcDIAyPRb1o/xhzWrJtZYHFYGPo1/Zn6s/i08FPkPshltG6UgqvBhZmfWa//hgkO4x1Isk/anAAQWa7Pv36OMUfHwDrIGq2VrSh+XgwTLxNoWWth3PFxUNJRQmVCJcI2hkFRQxFsOTYkEgn8PvODsq4yGnIbwJZjQ0VfBSr6KpQxzNDdEIURhXiw+gEcpzhS1n/8Fj78tvrh5Z6XiD0YC7/NfjIz8W4vvQ02h03L3ZZTlIPjFEc4TqHvh497H0dzydt8e2m3YXdEbI2ApqUmJCIJdOx1KOde9LQIt+bdglWQFYQ8IWbcmiHTKrknHq59CIBoylDSoX4nriYXC58vxA86P4CjwGHsLs68QeQMMv3Oqoaq8Nngg+gfosHV4H5wPJCU6NO21abUDYU8IcACLAZbkGRfZwNRdzTxNvnY0fcX43895n0k+z7iH4fALwPRkEuEuG6u30zpbtGy0cKOToJs6b6Az7qZBX0nfTzd/ZR8rbW8FQujFtL+fuTuSMQejMXGyo2MmR9MhVcpjL2MCVsPhoVZ6vlUyCvLMxbKWSwWwCKKWU+2P4GhuyFloulq68Kzr57BdKApTZUiFopxwucEHCY4MCr834WkU0noqO+A70Zf2jEhn8hMWJ29+r9CLJgONIXvZl9UJVTR1KqViZVwmePCWLTrautCSVQJmoqbCHvHHkg8noiSFyUQC8Vwne9K21wsjVmKW/NvIf5IPIL3BlMKogqqCrAKtEJnQ+ef6lLq2cnwISh9VYqbc25i2vVpNJs8ZV1luC10g4EzfTEKECrS1POpqIipwNJYusq5N7gvdIegXcC4wfxvo+RlCTgKHEYbRQ0zDTQXNyPnbg6h+O5BbNxdfhftNe2YcfvDsyqzbxObYiaF67sgPQ+mbh7gLSny34D3Wm/wm/n/p0QfQFjATLsxTWaOZciBEBQ/K0Z9Tv1/lAnxf4n6nHpcGHUBbVVtMPc3p5FETGO1FPr99DHzzkzcXXEXeQ/zGP+Nuok6bEbYQElbCU3FTYj8PBKOUx3hudwTUV9HoTG/EaELQ+G2wA3Dvh8GQQe1gDP026GMRZ3wzeFktkFPOzFNC02szl4NrhaXsahSEV+B7NBsZN3MgsVgC4w6+OFzhu8mX/huos8XlfGElaX09zd0NSSLnhNOTUBFfAWO9T+GgogCimqeq8mFjp0O6rLqoOeox1gwvzj2ImrSa/Bpwae0Yyw2C0o6Skg+k4zKxEp8kkJXrfbW1R9zIAYAsSFKu5xGcQRoyG/AxTEXEfh1IJnb1B0Fjwvw5pc36DO+D60TQKo2P+R0iMzF6A59Z33wmni9CnSshloxbvA4CoStUGN+I3Lu5cA6yJoyPtqPtkdjQSMKIgpoSvmGvAbcXnwbw34YRtkA3vvkHgzdDDFgzQDSxoWpe683+1lNS01YBVmRgqmeuDHnBgojCmE/1h7B+4IpmUVikRhZv2WBU8+B4yT6OivpZBIyrmXAe7U35Zqx2CzKeCgllrujo74DRU+LYOhqyNiN6rbIDSUvSmh2qFKMPDASEdsiGAt3tZm1iD8aj/ij8fDf5k9zQDjqcRT12fU0xwixgFD3yyvLw32xO6yGWsnsZuxpdZ55MxMTz0yE9XBriAUE8d+z07suqw61GbUY+etIRocCn40+4DXxYDfKjnE+qUyoRGNBI7JDs+G53JNiDegy2wUuswkLvtjfYuG10oskESriKnDM+xiG7RnGSIJbBFhATlEO4ZvD4bfZD3aj7EjS+eUPL1GTXoPArwOhbUt0FBm4GFCsoW/Nv4XUC6kY/uNw2jgUczAGj9Y9gpK2EkmCdMfa/LUoeFyA2MOEbSUtt7uuA5nXM+G31Y9Gaky5PAVCvhDXpl3DK+9XCPwqEDYjbPBq7ytEbIvAsvhlaK1sRc7dHAz+fDCFzOS38sFR5MDc35yWFymRSHBr/i04TnFkLDrX5xL2j2KBGCp6KjSyr7O+kyh0MZhQiEVitNe0oyiyiCTCu7f+ySr6aphrYG3+WrSUteDqtKsoiSrB58LPwdXkYtqNadCy0kL8sXiEbwrH4leLaUIGAGQ3Yeb1TDiMc4BFgAXZqSoVYw75YggWvVwEOSVi7CqPLSefwdrMWgTvC6bYLj//+jkiP48EQIg0Zt+fTftcjjwHHEVifGQSJCSdSiLzPaVFLYexDpBXkUfahTTSGk+61vFc5gmHcQ7QstaC00wndDZ0Mua3AkB1ajXaKttgFWQlcy+grKcMbVttxnlO20Ybclw5GHka0fZEXE0u+k7si+i90YjYFgHLQCILc/KFyRhzdAy6Wrsg6hKhtaIVhRGFlL1TeUw50i+nozG/kbY/6GzoJIvD7dXtpHVl0bMikuhTNVTFEZcjlGsmRfKpZLDYLJS/Loepryl8N/qiIIIoMirrKuPRxkdgsVikmMHAxQBseTbEAjFqUmtQGl1Kuw5tVW3Ya7SX9nkcBQ5G7B1BrhOyb2dDWY+wlGSxWe8tdlU3VcfMOzOhZqJGseK0CrSCvrM+Lk+4jIq4Cshx5RjJvtLoUpnWj1L0JMCl98OAdQMY70sAqMuuw0m/kxj0+SCaiEfaPTfsh2Eyu8WXxhG/LZNQQ1FdEQauBjKL7FZBVtDrpwePxW/3LmZ+Zph8aTJlf5Z7PxeKGoow9zNHnwl9aH8nPzwf16Zdg5qxGtwXuePB6gfk8x6+KZy2fmjIa0DUt1Fwne8qc3/RGwK/CERXexeqU6qhaqRKWfd6feIFr0+8kHQ6CfLK8pT9nqqhKha+WAghT4gXe17AbqQdjYCNORiDUYdGwXOZJ4B/dwQrEfdnzr0c1GXWIeibICjrKqM+tx7Pv3qOftP6Ie1SGlLPp8LI04hyPbtDXkm+1/vn7oq7SD6djG1t2xi76xvyG9CQ10CuI6QI+i4IVkOtcGvBLdyccxPzn86HroMu+C18NBY09ip2eRdGHxoNUZeIkgeadiENhZGFtCzQfxKUdZUZ72UAWJawDB21HYj+KRqmPqYw96Pn034o1IzVkHc/D4URhTL/nqGbIRa9WMR4rDc4z3SG1VArtJa3UtbvCccT0JjfiKxbWYCEILiivonC2ONjoWWtBbFQjKe7niLwq0Asil4EHXsqeSvdPwk6BWgqbIKKvgqlThmwLQBZN7LQWNCIu8vvYsatt3UZRXVFDPtuGOqz6pF1KwtK2kpoLGhEycsSWA6xJOtdkbsjkXg8kdEuWiKRQCIixuqec2f3vc+QL4cw5sDOCJ2B9pp2XJ54GT6bfDDixxEQCUSI/S0W1sOsMfHcRBQ+LkTug1yZNu89Ufa6DHXZdfBa7UWrv95bdQ/VydWYdXcWVPRVIJFIMO7kONoYU5lI7JtlOXO1VbYh6VQSjL2MaXvB2EOxuL/qPjQsNDDr7izadZv9cDaSTyfDaYYT2mvayZr2CZ8T4LfysTZvLSFOL2t5p63/R/xz8ZHs+4h/HJR1lWExmPCV7mlhIPXz74ktjVsgEoiw33I/dPvqwsDFAIN30W1An3/zHLG/xcL3M1/IK1MXYZ0NnUg5nwITbxOYDqAXfAQdAiiqKaK9uh3Npc00skaWTQRAtKDXZdXBepg1jDyMaMWouKNxiP4hGp7LPWlkn0QigYKaAoSdwj/V2Zd4MhElUSUw8zGD6UBTyiRa/qYcpwafQsj+EAxY27uN0J+BkraSTGuYhGMJSD2fSssKAYgiQOr5VAzbM4xRDTP+j/F4suMJwj8Lh80IG1rRW1FdERNOT4CoS0QWGaTQstbCrHt025n3gaBDgG9VvoW2nTZWZ6/+4N+iPrseiuqKjJ0pWlZaGLRzEJqKmyDoFNA2CaMPjUZnfSdjJse7oO+kT1NR/V9BqvDdULGB0T4h9WIq4g7Foe+kvrRne9DOQRAJRH+qG/HTwk/Bb+XLJE97g6KaYq/KJ4lYgsqESihpK32YDex7oLt92P8lFFQVGO0vpNDto4uTvifRWNBIUTf/ndFc0oy2qjYM2jWIMRcNAOKPxSNsQxhm3Z/F6Jkv67l5vO0xUs+lYkXyCihpK5Gdubp9dDH488EY/PlgVMRXoL26HRKJhLHIKysXVhrqbj/Gnuhw6jbMyFIPSlGZUEla/zHdmw/XP4S6qTqUdZWh56hHKfKkX01H2esyDP1qKG1+BIhxt3s3a8b1DCSdTMKIn0dA10EXug66WBS9iCSBcu/n4vqs6xj/x3hUxFXgxbcvMOveLNgE29AKpToOOsh7lIfLky4j6LsgitWOlrUWlsUtQ+Ifib2KLS6OvYjS6FLShkqKYT8Mg0QsQfatbIRtCKOQfRrmGhj/x3gYujNbAwZsC8CbX97gyY4nmBs+l3KMxWJBSVsJGys2Mr7Xf6s/OpZ0IOV8CozcjWiFrp8Mf4KhmyHmPJxDe6/1MGssi1uGh+se4uGnD7G+bD2lu0skECF8Uzhc57vSyD4tGy1MOj8Jun10IRKISLGL82xnyHHlUPy8GDoOOgjZH8JoZSYWipH3MA/qZuo0y0QdOx3Me0y3OJfCcaojSl+WEha1zXzKMTaHjSGXhkBTqInbS25jwNoBlI3xkC+GwGu1F5qLm8Fv4VOOLYtfhvaadrDl2Yx5knVZdbg65SoAYE3eGgrJCPROBgOA92pvRrtegCC+LAZZ4MasG4xqYCUtJbjMc8GYw9SxQrePLjQsNGDc35hmP9lR14HzI8+j34x+8N3oC34rH0KekFzLPFjzABpmGkQR3vUIhnwxhLam9VjqQWSIyLDAlmU3L8XxAceJvDyxBNbDrWk5YAAQsD0A/T/pTykgK2ooos+EPqiIqcAerT2YcmUKpbMZIARbtRm16Gykrlem35iOsM/CUBpdSskLBYiuu7DPwpB6IRUAQZb3BEeBAwNXA0y9OpXm+CBF4olEpF1Kg+dST7DkqGs0dVN1LItfRit8dLV1QdAhgJmvGUqiStBa2UqKM/T66cFlrguUdJSQfScb8Ufi4b3Km0L2xR6KRcTWCCyJWcIobJpxewaaS5qR9yiP3nUjIUh0l7kujCKLkQdGwn6MPZpG/4hip1GA7lvRVkdtB342/hmeKzzxueBzWieS9LvVZddB3USdLAZx5Dl4secFEn5PgKK6Ivy3Edli+Y/ykXwqGcN+GIa2qjaY+piS81FPDP1qKAK/CCQygP5NRPtu8gVHgYOHnz6ExSALeK3yIueSquQqvP75Ncz8zGDsZYwBawfQSG6xUAyuFheBXwXKtLYK/DIQgV8GMh4DiKJaQ24DzPzeWg+y2CzYDLeBiqEKLoy6AK4WF1OvToV1kDWUdZTJoiebw6aNW90R/UM0Us6lYObdmTB0M6SMyY2FjYjYGgGnmU6k0FQkohYVbUNssaOTWjTPe5SH8yHniWv6zVC4L3KH5RBLyhyYeDIRj9Y9wsKohYxj2bhj4yBoF5CCnO4w8jTC58LPUZNWQxLR+WH5OBd87t8XB5hyZQpODTrF+J3nPZkHSIi8OinB7jjFEeom6pCIJHj982uMOzEOGyo2kK44m+s2g8VmQV5FHqUvS5H7IJcyT3Uvtp4LPgdTX1MM2T0EHAUOxXrQZoQNHq57SOYjM3U4M0FBVQElL0rwcs9LynwwI3QGUi+koiKuAsZexlgcTRf4AIR4V6+fHvy3UjP3Hnz6ALp9dOH1iRfqsurAa+KRziRSh57mkmYcsDkAj6UetPcrqivCdZ4r4x5Q+nceb34s0ypRUU0R/FY+utq7aASR6QBTTDw7kcxH7TkOdc+RlELTQpPWbXd91nUY9zeWOc8/2f4EAMgO+KDvgiASiFD2uozi3iMFr5mH/Ef5MrvrYn6LQezBWHgu94TbQjdGu9aqxCr8EfAHgn8JxsBP6QJht/lutCxxOa4czP3MkXs/FxFbI2idShKJBLG/xcJxiiOZIVaXXQcRX4TLky6jMb8RANF5qKyrjPbqdqScTYFePz2knifmKPuxzPuK+lyC9NDrqweLQRa07r7S6FIIeUK4LXKTuaZ9uvspTLxNYBtCnS8kIgluzLmBrtYuKOspk4JBtwVuJJHfXNqMhrwGGHkYUa6noFOA5DPJ0O2jK7Nbtqcl6ejDoxkzJv8paCpqwtnhZ9H/k/6MtqbySvLoautC+GfhGLRr0F9C9tWk16A6pRpaNrLrA4JOAS5PuAzrEdaMYvzGwkYIOgSMbhKqBqo0B6gxR8Yg71EeHKc44qTvSXAUOZh6dSqMPAlRU0d9B2ozaiHHlaOt+5PPJqM+px5DvxqK2vRaHPM6BtOBpmguacbY42NhN9IOStpKWBq3FDfn3ITNCBvk3s/F7SW3MfrQaJJI9VjmAYvBFuDIc1DysgS35t3C9FvTyTW7kCeElo0WY720Ia8BB+0PAgDGHhtLEVSvSFmBrrYuyHHlGIlxgBBLdLV3YfKlyeTc+eL7F3i66ylptdszz/1dqEoirNeZiGJBuwC8Rh4idkSgvYZwRiuMKKSNuyH7QjDs+2GM41pzaTPij8ZjwukJcJpJF3gURhCdm8q69C5zgHAzsQiwwLcq38J1vismnJoAgLCqlpL+OnY6lLVyQ34DLo69CHUTddqe9iP+mfhI9n3EPwZcLjEQdtR1wGWuCwK2B9DUHV3tXSiPKUdrRStMvE0oAxhHnoMN5Rt6/YyqxCrIK8lj+A90Aqq1spVs17YdaUtTlHY2duLR+kcAgF/Mf6GRAUraSih5WYKX37+E72ZfSiE3/1E+bs2/Rfq+90TZqzIAYNzAcuSJrqjEE4kI2R8ic6KThQmnJuDqtKs46XcSOzp3UApDaiZqGLh+IOpz61H4pPCDW8jfBbFIDEhA8dGXwmeDD/pN68doQ6ZhpoERP49gLD4BRIfGiL0jMHDDQMbFVG1mLdgc9l/egSS9HxtyGwgFktyHkX1NRU3gt/DJzIeekJK+jDk56oqY/YCucn4fSCQSQh3IwgdbgP6n0HHQQX12PeNmIf1qOuIOxWHghoGMmTNMdg7vi96Csf9TJJ9NRuiCUPSZ0KfXTLh3QTrmdUfRsyIUPS2C9yrv97aZ+CtQ8LgAj9Y/wrAfhjFmjcgry2PkryOh1+/Dlbn/K5gHmGNDxQYoqisyKmzzw/JREVsBMz8z2ma8PrcecYfj4DjFkZJXIgVXkws1YzVyAW7oZojt7dsp3SZMhY/3QW8WPCKBCO017Yg7EgcWi0WbM/pN7Qdzf3NomGkwdvPEHY6DroMuqlOq4bPRh7IxyQ/LR+LxRLgtcENbZRuMvYxlWloDBMGSez+X6Fr/d6bs7AezyYLchdEXAAAvvntBZvZdGH0Bm2o20QQaI34aAXVTdYRtDGMseAPolazJCs1Czt0cxuKLtHBm7m9OUz0r6yi/7XR4WgQhT0gprqjoq2DQrkG0ojRAdMZXxFZAzUSN8ThAWKjdnHMTw34YRiP7rIOswW/l49HGR0QBkCGLzGmmE5mP0R3RP0WDo8ghi/Q9v5OWjRZ+9/yd0h3lv8UfF8ZcIIt1tiNtodtXl9YlkHkzE9emXYPbArcPtiruv7w/jcSRgsVmwcDVAB2vOpB4IhF9JvShFN/0HPWgBz1cHHcR+Y/ysZO/EwCx7jvkdAj9pvVjXLsBRLepcX/CTo4pH/E/gRxXDjp2OjI76pnynaVYV7SO8XWxUIz22nbyXG/Nv4Xs0GzsEhG5aONOjEPhk0IccSW6bZiU0Bx5DgqfFOLFty8Q9F0Q4zglkUhIwUDPNVjg14HQstKCwzgHWpdc2uU0FEUWYdj3wyikFkAUD6bfmI68h3nobOykFTHK3pQh82Ym5kfOJwtVP+j8AMshlph2fRpif4uF5RBL8j5JvZiKoqdF8Nngg8TjibAYZIExv49hzFf0XOZJdmEwIeduDiwDLeGx1IOxs0qOKwexSIxHGx7Be7U3WYA+1O8QaRPlscwDY4++FbQ4jHUg1+1+n/nBe7U37Tsb9zcGi83Cce/jWBi1kLJ2ZbEIq+gzw86gIq4CW5u2Uq+nvQ7W5KyR+Z0AIg+uTdMUQgUVdB/V5ZXl4b3GG+YB5owZv3mP8hBzIAa593Mp9rdCnhAlz4lIhEnnJ5FimNhDsSgIL4CBqwFifo0hMlwf5qE6uRqjD9HzUFlsFjhs6poy5tcY6PXTw/yn89FS1oLy/HIYuBqgubgZhU8KYTvSFm9+eQPLIZYQ8oRoLGiE1VAryCvJY8i/hlCytp98ToxVPbPzAKKb8uacm7AJtqHYABp5GNFyjNqq2+D7mS80LTVh4mWC24tv4/XPr4kxuIUPUZcIStpKyLmTg+KoYlgPt4aimiJtD+K2yA2CTgEujrlIFiQbCxrx/Ovn0LLWQvqVdBi4GVCKg1wuF4IOAUQsEeRV5GnPoXT/BxDFvYDtARTXmYTjCXi0jth7du+46fk7TLk0hfkYiwUWh0WZX3Lu5QAAgr4nuoRMvEww++FsRqcBqUivu3WpzXAbmA40hVggxqdFn0JZV5myN+2+norYFoH6nHpsqt5EvqZhpoE5YXNwcexF5Ifloy67jtJN2VTchJq0GpgONIWhuyH6TesnU9QgC8b9jeG2wI22DlI1UoX9GHsE7AyQmYs96PNByLiagfaadoowIP5IPGxG2MDrEy+EfxaO/PB87OTtpLw3+3Y2GgsaaUJTgHCDYHFYuDbjGrY2b6WsSxXUFEgr2dqMWsYuuJr0Ghx2Ogz7sfaMzhQtZS14vPkxhv84nFF0kHohFU92PsGkc5MY5wuAKLQr6yijo66DJC1H/DSCPB70fRBEXSKSnJRmB8pyRTL2NMa8iHl49fMrlL0uo8U2FEYUoi6rDo/WP4L9GHva2Hpr/i1IJBIM+XII7b3Fz4tRHlMOjyUetDWSdN9rOtAUS2OXMu4JPy36FLXptWgsbISWlRburbiH6lRibSzoEMDQ1RDhm8Phs8EHZr5m2N6xHWw5NgauGwgWi4W7y+8i9lAszYa0NLoUjzc/BgDGus/jrY9Rn019JnrCe403o0C2tbIV8sryCNgRAP8t9PUfQHRbP1r/CAtfLKSQV3WZdbi34h4Aakdt2esySCQSZN3MgpqxGqVrWMNcA+217Xj25TNYDLJ4r1gMpr3t/xKNBY0I2xgGsVAMv81UIr0quQpioRhL3iyhrXX+LI64HoHDWIde6wPVKdXID8uHWChmJPsern3I6HbyaMMjxP8eDzNfM/ht8SNtXPWd9Ml1YvAvwVBQUSCtLiUSCZ7uegqH8Q4w8jCCkCdEVVIVmkub4TjZEZnXMpF9OxuVcZUYfXg0Bu8eDIlEgoLwAnKMivo2Cq9/eY0lr5dAy1oLRU+LoGmpCXkVebRWtuLW/FtwmuFE3juWQywxI3QGhVgcvme4zAYAqcVkeUw5mekrhZyiXK+Z7QAhwpZXlqeIOQesHQDbYNs/XQ9ynecKmxE2OGBzAABBOkoF5BNOTYCQJ8Q3St8AIMSUnQ2daClvoYhS2HJsPF73GCYDTGhdwLwmHtKvpEOvnx5jbW7S+UkQCUQyLffjf4/Hm/1vAIAiaJZa0t5beQ9xh+Mw7sQ4uC10A4vFQk1aDeoy61CXWfenrslHMON/OeZ9JPs+4h8BDoeDPn2IzVH45nAk/ZFEholKwWvi4cGaB0g5R3jadw9lL3pWBGUdZaibqUPQIUDiiUTo9tWlLT6nXaNmsXSHpqUmFjxbgEfrHzF61He3YGNSQ9Tn1KP8TTnyw/PhMo+qHjHxNkHIgRDGgg1A5ACE/BICZT3m4r71cGvCX5tBAdZa2Yq8B3kw82X2/tay1kLgV4Goy6qjKUO0bbQR/HMwvlP7Dg05DX+K7Es6nYSK2AoEfRdEm5BSz6fi1vxbUNZVxrwn8yhdVkYeRjjhewL2Y+xpv4uqoSpshtug5EUJ1E3UaQswqc89UwcmQBSUhTwh5jyaA72+epQNXXNJM94ceEOEAPuZ92pZ1hNyXDnMfjgbDbkNlG6b98WYo2Mw6uAoxq4ZaaC2/Vh7RhuW5lKi20HPUe+DOwpf7X2F8M/CYehuiOUJ759R+FdgddZqmceUtJRgGWhJdPXJWMiFLg5FxpUMbG7Y/EFEZUV8BdIvp8N5tjNjEb03dNR14Ee9HxnzmwCg+GkxAKDvFNmdcO9C9zGvOy6OvYiu1i70m9rv/5TsE/KEqEmrwa35t/BZzWe048e9j0PIE2J1tuzf8+8GOUU5KKopoqWsBaqGqrQiwpOdT9BW1UbJ75GiuaQZr/e9RntNO+pz6uEy14Uy/vpv8YfNCBs8++IZ3Ba4wcDFgHyupaHY3dWJpwNPo6OugxKcfn7UedTn1GNt3tr3/k712fU47HwYAKBhoUEj+7iaXFrRozvWFa+DWChGVWIVTVAw/IfhGPz5YGTdysLDTx9i/tP5FPVvTXoNXn7/ErqOugjYFoB+0/ohcmckatPf5udWJVXBepg12HJs2I+xR87dHITsD0Hk55GQV5aHnqOeTGsjWdm17TXtSD6TDPMAc5ljvtS2lykLLvNmJq5MugL7MfaYeUe2dWv45nDa/cBr4qH/8v6MAo326nb8EfAHlPWUMfniZFrwfOIfiSh7XYbpt6YzFu4mnZ+E9KvpuDbtGkwHmFLGqdqMWuTcy0GfCX1IMrI7WitaoaCqQOtgk0JFTwUuc10on9tR14GiyCIAwMhfR+L6jOuMRUPp5tF9CZ1cLY8px+VJl8HmsDHx3ESZ3alMYLPZsDCwAGsiC44THWk2yWKRGCwWC86znal2mhLiHih/U474Y/Ew8Tahjekqeiq92lvfXnIbKedSMP7keJqdjlgkxhHXI6hNr0XIgRAMWEN1Oeio70BrRSu0rLRk2ivX59Qj71Ee7EbZkb9JY2EjUi+kwm6UHQrCCxD1bRQWRy+GnqMeVA1VKUSgTbANpcBvG2yLK5OvACDIp+5Zn1K0VbWhILwA5THl4LfSu5EujLlAkvBMnfWyCoUAkeUUfzQevpt8IRIQREjP9adtiC2t6wAg7t3Yg7GwDbElyT7zAHPou+ij7HUZhu0ZBn4LH3uN92LW3VkoiSpBwu8JCPwiEFsat4CjwGFcI0lR9roMBREFcFvgRuv0uT7rOsx8zDDnEb1bFiB+6/rsemRczYD9GHuS7LMJsSEtDVvLW2V+tryyPOO5WQdZY8btGUg6mSTTRs9vsx866jsYj4m6RKjLqoOSjhLtOxVHFaO5uBm5A+YALBa6r/IV1RUx8sBINBU1oeRFCYz7G1NIkezQbOTez4XPRh9Kp1tVchXqsuow/MfhlCKR7yZfFIQXQN1UHeNPjoeqkSpiD8Yi82YmjezLCs1CeUw57EbaQctGC2pGaog7GgcWhwXHKY6oSa1B/O/xiP0tFmsL1sJhnAO2tmzFkx1PoGqoitbyVqScTUHm9UysK15HdoS3VrSC18SDnqMeMq9ngsViUci+rFtZaClvgfMsZ+Q9zIOa6bsLtDl3c3BnyR04zXRCwI4AzLw7kxScRO6KJDvHJl+aDBFfhN8cf4OBswEta84q0ArattqwHmZNXs/Oxk4k/ZGEod8OxS7RLhQ8LsCzr55hwNoB4Gpw0adPH7I7LfiXYLz66RX0+ulh9OHR0LLSgt8WP3gu84SihiI48hyIBCIIeULIK8mDLcdG5C7C1tRzuSe0rLTQUt6C6uRqGPc3Jomouqw6tNe0w8zPjHxOhTwhOAocdLV1oSqpCqpGqlAzIjJzA78MhPtCd1QmVOK493Gy84EJzSXNYLFZlH1xXXYdDjkewqDPByHwy0DwW/moy66DmpEaFNUV8ZPBT1DSVsLYY2OJLs51dOcYm+E22MnbSXRF/HtL09nYiTNBZ9BR14GW0hZwtbj4JOUTWtfWu1CVVIXrs65j6NdDyQ4YsVCM8C3hMPEywcw7M9GQ14DiqGKY+ZrRxra26jZUJVXRrNfXl60n95Pui91hNeztnrkwshBngs4AEiICgKkDDSA6lm1DbGlihBffvSDXMXmP8hjXDJfGXQJAdP73RF1WHdIvp2PM0TFwGE8XFb/Y8wKJJxLB1eSStYDUi6l4tO4RJl2YRK5f+k0lMks7Gzth4m1Cs+u2DrKGWCim2KMWPy9GV3sXo1AQAFrKW5Admg3n2XQru0nnJ6EhtwH8Vj4jmZ0fng/dPrqYeIaeN51zLwfRP0SDxWGhLrMOgV8Fkr+3tFPIb6sfhn03jPZeFosFjjwHF0ZfgPdab4zcPxIeSz3QXtNOrkMzb2Qi83om+k7uCzNfM8raVSKRIOlUErraumhkn81wGwz/cTiaipsYRccBOwLAa6Jbv3eH+yJ35D3MQ112HUX48uK7F2ivbkdhRCEcpziS643azFpUJVXBZrgNrIZaYdShUYyiYQDwWv32fAWdAtLWXQp+Kx+DPyecBDobOlGfXY+nu59i0K5B7yT7ZO1t/1fQtNTEmrw1uLv8LiOZd3nCZSjrKn9wPEpvGPT5oHeKzetz6gEAAz5ldtUSdBCdYz1Rm1ELQbsABeEFaClrwaqMVQCINYSQJwRbnk0+wyQkhEOBmokakk4n4c6SO1DSVkJnQyd28HZg9JHRZH65sp4yKbjp3g2srKcMjgIHBY8L0G96P1gOsSQ7oxsLGlH+ppwiINAw02B04Sh+XgyRQETbL6kZqcl0SJPmxduPtQevkYfArwNpnal7tPbAZIAJ5oa9rc9yNbgw8TZB6sVUhG8Kh+9mX6gaEGKP94lKkVeWx6t9r8j/31O0zlHgYMHzBXiy/QmR91vcDLtRdpQ1XFNRExKOJaCrrYtG9uk76WNzw2bUZ9ejraqNFi0lx5XrVeRyd/ld8r+ZxPEuc1wQdzgOtxffhulAU+g56sEiwAKz7s9irHN/xJ/D/3rM+0j2fcQ/AmKxGI2NjdDS0oJNsA1Kokrw7KtnCPwikBzoOhs7SaLPdZ4rpfhxJugM7MfYk+HVAOA0w4lRaZZ0KgnpV9Ix7sQ4SvFDQUUBFoMssCx+GeM5Sj3pAeaF9tkRZ6Gip0JT+QGEnZNuH13E/BaDsA1hmBcxj6IY5WpywW/lo7O+k1bgF/KFaClrge1IW8qgz2/lozG/EQ35RD7PqEOjGMk+qc2IrEU4AEy+OFlmgeJdCP8sHB21HdDtqwvvVVQbLA0LYqLvqOtAc0kzzVLRbpSdTKueomdFeLCasEzpuUA7E3QGrZWtWJO7BhwFDm2j5rPRB+GbwnHE5Qgtm7G1spXM85BFpkjtY5lINdtgWyCY8W3vREVsBS6OvYhhe4bR1OntNe1ozG+E1yovxmJ92MYwZFzNwOgjo2V2TsiClESWpeT8X8F6mDXjsyTFhdEXkHs/FzbBNkR3wgfgWP9jAIhn70PJPukGXxqs3BMh+0MQ9F3Qf+SB3n3MY7Pf3r9Sm6W/2h70XbAfYw+bETYwD2DuprUKsnpvn/u/C7rauhB7KBaPtzzGhDMT4DqXmlk5bM8wmfkW5n7mWFeyDg/WPEDowlA4zXCijTP1OfV4ve81zHzNoGWthZr0GmiYa6A8thwAoap7sOYBNjdshp6THvhN1IK8bl9dxg1HVmgWHq17BDNfM4w9PpZSYFDSUcKATwfA2MuYbgcHoguqq7ULFXEVkIglpOpaCmkhhMkOS0lLCUpaSrAKssKY38fQNqtRX0ch7VIaFDUUEbAtACr6Kgg5EAJDV0Ow5dmoz65H6MJQCPlCDNk9hEKs9Wb9CBAKxc6GTprtFUAUisI/+//Y+8vAqK61/R//zEzclbgHCSGQAAGS4O4uxYoVqpQKBVoqp0JLS11pC8Xd3T0hWCAQEkKMeELcfWZ+L/aZnWz2hNOe55zv85z//1xvSmdnZvaatfa97nXLdZ0BaLWifdCqQYS/Ga739zyy8AiAOC8tkRuTy+5Ju+n7bl8GfDxAth7Ovn2WmLUxersRTWxNGPbtME69doob39+QHV4TDySScjJF0iX0JNqObMtrGa/J9qK823mcXXYW+3b2eukKR68dzYjvRkgo9HTIvp7NznE7GfjJQImI/fmV58V/3/5dSGwkHUmSffbQr4bSUNmgl86otrRWTIYUJRbJkn1nV5wlMzKTvu/2xa2nm6QztK6iji/svqDL3C6M3zBe9tk7x+0k7WyazIcysjDC2tOajMsZZFzOoOOUjkzZPUX2/qehNK0Udb2a8yvP69XO0DQJujjJR5Nlyb743fEcf+k43gO86fVaL9kzdf3766RfTCfxQCJW7lZi8K0kuYQL717AvI05lq6WuIW6tXpw17enj103FvM25q0WYd3feZ/or6JlnWQ66LQNO07p2GoxTV1ZHUWJRdj42EhooQavHkz/D/pzedVlrn97nSXpS0S6t9K0UqK/iabjpI56A3+dnumEazdX8mPzKUkpwc7fTtR32T15Nw/2PWDi9onY+tiiMlIxZM0QBnw8AFNbUxRKBQXxBZxfeV7m54OgAbzvmX1UZFfgO9hXZsfGrhv71CKZx3cfc2D2AQauGkiXZ5v3gzG/jqH78925/t11VMYqYjfF4tLVBacgJx4ceEDSkSQGrhqIUqWk6nEV9m3tZXPZblQ7mQ4XCEG4z+0+p9MznfQmVKoLq7n2zTUiP4vUq/d188ebxO+Oh1GdaK3K7OYvNwVWiLRXJV3Gvd/uTdeFXXEKcpLYCQsnC8KXhePWw43ob6JJOZHCMwefwW+oHy/Fv4RDgAPxu+O59s01Bq4ayOjf5HTWukRd5KeRjPx5JKEvhvJg7wOKHxZz6cNLXPrwEh3Gd2DImiGiHSh+WMyNHwQN1eMvH6fnkp6M2zBOPH8kHkpk1/hdAAROC+S568/J/PCY32LIuJRBj5d7iN2/LXH+vfNc+eQKVh5WYuGGrkjk/o77WLpZMnRNc6eSR4QHTfVNGFsai8Hw0WtHt1o0Y+1hTfcXmp/XptomOs/uTNuRbVEoFaSeTiX6q2i6PNsFI0sjYvfFigUUaaeFAG1FdgW1xbXY+tjKtL1u/HSDE6+cYM6FOWJ3hMpQJVI/Pjr3SMYUc2XVFe5tvcfY9WMJmS8UaRx+7jBx2+Lo/XZvIj+LBISA70v3X8LE2gSHAAfxvnTUt7a+tjJN3E0DNmFsbczQr4ZiameKcxdn7qy/AwhdcqVppWRcyeDQ3ENM2jmJ9mPaU11QTXVBNcdeOkZZehnDvpYemqryqzg49yDuYe6Sjj6tWkt9eT0e4R5kXc2iIqtCbyHDP4KRpdDV0tKPVjeoufb1Nbo824X0i+nE/BoDwIqKFbJC1YAJAXRd0FXWEdLSB3iS2i1+V7x4TplzXpok1qEktYSCewWEvhwqs8mNtcK5I2J5hEy3U4ewpWGojFR6deKKHhZx7ZtrTNgyQUbxB5C4P5Gm2iZJF7GJtQkOHfT7oaa2pq3SvP3a9Vfhs5IXs3nwZpFubuCqgTJ9z9rSWkxsTHg17VUZZSgINIpP6ly1RGs06QA9Xu5BwIQA4vfEc/v324QvDRfHbmxpTNCMINp0akNDVQMqY5WkYFSr0VKZV0nYm2G0HSX4SS19g5qiGpw6O7GybiUqI5VkryxNK+XKJ1cIfytcLyOTpatlqwwVIMQRYjfFcmD2AUb+NFLGLAJQcL+Ao4uOMmbdGEmyr8OEDsTvjiftTBq3192m56s9sXSxJOloEmeXnWXBtQW493TX+5va+dvJWKm0Gi2OgY5o1VqmH5nOH73/IH5nvJjs2zF2B4UJhbxw74U/deZt7Wz7vwlbH1tJEqglIpZHYGBiQGNNI0oDpUxz959B/w/6k3Iqhbjtca1qtbUf057nY59vtets2oFpeqkbZ52cxYlXT3DjhxsM/ap5H9s1YRfJx5Pp8mwXxm8aT8aVDHZN2MWQNUMImRfC+E3jybmeg6ZJQ9DMILz6emFiY4LSQImli6WssOVJdFvYjZLkEo4+f5Sq/CoJrbytry0rylc85d0CHp0XCiJsvG1Y8kiuz94adKxqSUeSMHcylyTd1vVah3kbcwImBmBkZcSXTl8SPD+YwZ8Npiy9jMKEQppqm7D2subhwYekX0wXqJ0t9BdKtkRlXiVFCUXYtbWTsS8kHUuirrSOzrM6k3srF2tPa+ZcnCOLN1359AoAw7+V67gqFAqKEov4I/wPhn41VEYzW5xcTPrFdOrK6giaESTzd3ss7sGNH24weZe0q//M8jMYWRjR771+TNo5iQd7H4g23tTO9Knx4P/ir+N/2+b9N9n3X/xHQKvVkpWVhY2NDZ2mdeLRuUdc/eIqvZf3Fg/UKiOVcPifFCALtoz4YQQ2XjZisu/5O8/rPfDnxuRy+/fbeiv2nkZ3BEKiZOL2iVg4WVBfWS87HPR6rddTq5FB4Mf3G+YnO0SWpAoVaPpos5rqmjj39jlCFoRIDHRmZCbbR27HI8KDoV8P1au9A7B1+FYqsit4I1tOcZp/N58L710g9KXQfzoRNOvkLE6+dlJvcsK7nzcr61airlfLfpsDsw+QdiZNb+dU3p08Ln5wkeD5wTh1kTusHSZ04OIHF/nU7FMWRC+QUXv0eLkHtj625NzMkXUPOAc7syR9idABo4dD+86GOxyef5iZJ2bKqtYbaxo5/+55PMI9/imKyepCQb9LXwenR1gzRYg+dJzSkYQ9CZxYfOIvJ/v06Rn8FWzoswFbP1uRD/yv4Fvvb6nKr2JJ2hK9VXXnVp4janUUryS9IutSURmpaD+uvUQI+s+iz7t9yLme06oo99Ng7WnN7DOzWw3MGlsZ6z2c/RW0tHkt8cLdF0DLX6Ys+legtW4IEHQ5fuv2G2dXnGXwanmV7P9FZFzJ4Ozys1g4W+jtqvYZ4EPurVzOLD9DyLwQSbGEgYkB1h7W9PugH10XdpXQ3F36+BIX379In3f7sDhlMZYuljyOe8z6XusZsmYIL8S+AAhJgEfnhSDIk+LegNiZ/iQeHn5IWXoZZelljPxppCQQaOliqffQoMOdP+6IdNQ2PjayvbLwQSHGVsZ6k31V+VU0VDfg2NFRr05E8PxgrDytxGBiTWENOddzOPnqSabum4rvYF+CZgaJRR0atYbqx9UYWxkT9UUUMb/GsOj2Ir3ffXfzXbKisnANdcWlq4skSWTfzp7nrj/H8ZePt6odYuFsQcrJFG6vuy0LLJo7mmPfzh5TO1O+cv1KEjxSGaowb2OOoZmh3uSpzwAfYtbGcOG9CzL9RmNLY3ot6YWdv53e4NrY9WNlfkZLnFl+BltfW7323H+4P89df07sspx7ca4sKLDKdJXexJeOdvJJPyPtXBrQTMPVUNXAoNWDZN/dWvckCEGqdxvepaGqQa+/k38nn6yoLLaN2CanM1Qp8JrghVNnJ/Lu5GHjbSOZZ88+nq3Sxoa9EUbMbzFUF1RLAu06pF9MZ9OATXR6phMjfxopK16ac34O2dezxaReSyhVSl5JfIXqwmq9gR63UDfC3wrn6pqr2HjJn6nz757Hzs+OuZfmSrox3Hu5s+j2Iqw9rDFzMKPzrGbGh/qKehL2JeDcxRmXri6y7/zC/gv8h/sz7JthHF54mLYj2soSX569PRn8+eBWg0X6OpZ10Gq0rA9fD1qhW/NJjRQjcyOMzI3w7O2Jul4tCQZX5lVy88eb1JfXk3Y2jc6zO0uCkYamhtSW1HJwzkFG/DiCHv7NRWC9Xu+Fc7AzXn29CJoeJPk+daOawnuF/BryKyCwPzw55uTjyVRkVzB512S9Nj1waiCHFhxi86DNsiB+Y20jx14SKMye1MxqqG7AwtmCwoRCcm8J54TBnw/GKciJvJg8YjfE0ntFb+7vvM/FDy7yYtyLku+P3x1P3LY4hn83XO98ePb2bLXSvyKrgsjPIjEwNdAbhAl/Kxz3cHfyfjhPmVN7cGj21WuKajiy8Aim9qYM/264bN23Vl1v422DZ29PNvbbiEKpwMhC+P2r0qqoyK7Ayt2K0tRSUk+nMuizQaKmXUsM/nwwQTODKM8sFwsDJm6fSH5sPluHCnpwPV7tgc8AIVldnllO3u08LN0s6TyrMy7dXHDr4SZJAFz75pr4b30UwyAUI+jreNBBZ0MaKhtEnfOWcxX9ZTRm9mZiYUnglEBZJ0Rr/v2JJSdIOZ4iaHb/vaupMKGQe1vu0X5ce+rL6wmcFkiXZ7tg6WKJVito2ukwaPUgxqwbg4m1iejjVeZVClTXt3IZ98c4HAMcCZ4XLBYpuoW6CevubxcZ/t1wPMI9GPvHWElQsfPsztzbeo/DCw6L+7PO1jXVNzH8++GknkwVtWmrC6q5s+EOsRtjAZiydwp7Ju+hIK5ANuZuz3fDwMSAbcO34TfMj+mHp6PVCOcYpUrJ937f0/+j/vRZ2Ufo3DczZNapWeTG5OLQwYGmWnlBVUlqCamnUkk9lUrbEW0xsTHBvp09Zg5mvJoqZTpIPJjI9e+vM+CjAbKCm9Zg52dHyPwQEvYmiN1pBiYGLElfQmlaKZsHbgaEwj19jCHreq6j/VgpDZ9Wq6UwvhATWxO9Pkz4W+EEzQjCpZsLt9beorakVnaGrMqvInZjLG06t5H5GwM/HsiVT64Q9XlUqz62TltOH3wG+vBS/EuY2JjQVNckO0PMOj0LrVpaNdl2ZFtJQRDA76G/Y97GvFVt+92TdlMQVyB2JJm3McfU3pTa4loJy4MOhQmFbBu+TUIj3BLFycWiRq9SpdRLv1yUWMSRhUfo9kI3ie6Wtac11p7WOHV2ou+7fSXnMgtnCyZum0jyiWQ+s/yMsX+MlVDBqxvU/N79dzrP6izaKB1OLDnBje+FooQ3ct7A0tWSnBs5bB22lRE/jODE4hOAkMj8Z89qOTdyuLf1HkO/Gqr3PHnxg4t49fWSraH2Y9rT67VeZEVlEbU6Cs/enliOsqTD+A7Y+du1qmXbEjq7CMLe+9L9l0g8lEhdeR0vxb8kmYMOEzrg2cdTVqz9tM/Wd7b930J1QTVx2+PwiPDQS23b/YXulKSW8Kn5p4QvC2+VZvKv4uoXV8m7k9dqss/Y2pjyjHI0jRq9Re87xuzANdRVQqGrw4jvRzDi+xGS13wG+ZB8PBmPCA/WdlmLukGNU5CT6BNkRmZyYNYBph2YJmNAqS6oRqPWiGfZE4tP4DtEKF4MmBAg+rU6xo3yzHKKk4uJ3x1P+zHt9SaWH+x/wMG5Bxm/aTwBEwQ/TldkOPgLuX2ryKng3IpzKFQKer3WS+KjTdk7hcsfXabL3C6yYjwLJwtM7U0Z98c4aktryb+TLyalzyw7Q8KeBAImBbDg6gIexz2mJLkECycLNGoNN364QZtObVotOj//znkenX/E21Vvk3Q0icq8SrotFAr1r665SmFCIQ4BDjTVNVGcVEx5Zrms4zBgUgB2/nZ6i4caqhqoyK4g9JVQvQV7p14/JbJzeIR7yPadIWuG0P/D/uyeuJuih0Vigj5+Vzzmjub0e68fnaZ1klCbatQayjPKaapvwjHgP0eW5f8y/rdt3n+Tff/FfyT6f9ifsDfCJDpe1Y8FKi/7DvayYIvOAV54cyHGVsatHqovvHuB/Lv5vFP9juxafmw+v3X9DSsPK/q800cWUDJ3NCdoehAfG31M2xFtZZoQvV7rRU1xDcknknHs6Cg5wD7Y/4AL719g9K+j9XYl6PQY/Ib5ya4ZmRvR7YVu5FzLoa6sTtwwdImTrKgssqKyGPjpQL0aUQETAzi74ixrg9cy5/wcSTCgtriW1FOpst/zr8ClqwvzLs9r9XprXNv27e3xiPDQm1jVNGkwsTHBq6+X3uBfn3f64NLVhfs77mNqrz84qO8Qo7sffdWFOth42RAwKUBvBVtjbSPXvrnGtW+usbxsud5k4dOQsDuB+vJ6vbqNCqWCe1vuce3ba0zbP03WpRk4JRD7WHtqS2r/0nf+K5AZmUlmZOY/lewrzxB0cPLu5MmSfXm387i//T4OHRz0Uvv9T/Tw9Gm8/BU8reOwIqeCX4J+IWR+iF5H/H8Cc0dz6ivqaapv+occ9f9KFCcJup3+w/Xz2yuUCrRqLVrNX2yx/F+EnZ8dfVb2ocOEDriEyAPqIFTPXv3iKt79vCXPXEN1A1X5Vdj528nem3oyFRA63fq80wdDU0OsPawZsmYIXv28xIR92BthegXh/xEGfDSA7s93x76dvV4tSxA0bZrqmsQge+KhRExtTXHq7EToK6E4dXbSS9Pxc8ef8errRW5MLl0XdpV0kZx64xT3d9zn3YZ39QbA/Ib44TekeY/SBbhBCABN2TuFoV8NFRNfHxt8DIBXPy/y7+RTX1HPjjE7mH16tqwYZ9KOSVz75hpbBm/h2fPPSoIvRuaClsPTaHZqimuI3RhLxqUMWbLv5QcCzc25ledkCSqnzk4svNH65wZODeTc2+dEO6YP+rp5QHiWC+4XsNp6NX1W9pFphdz88Sau3V3xHeyLuaO5JOBj5mCGmYMZOTdzMHc0lwXAsqOzMTAx0NuJ69zFmUk7JnHhvQsoDZRiwcObuW9SkV3B6aWnid8Vz/N3npclPUA4DH7v+z1tR7dl1E9ynS6VoarVpNysU7MoiC8g+1o2dv7S4g1DU0OC3w9G9VDFb11/Y/LuyWKQvSK7Avde7oS+GMrFDy+SdjqNuZfnolQpBTqj8joGrx7cql3W6SPd33mf7i9110sv+rQkJtBqINm1u5CADpoZpJcBYc75ORiaG8oOzsZWxq3anar8Kg7PP0zvd3rj0tWFGz/eIONyBhO3TkRpqMSthxt27ey48P4F7qy7g5mDmSzx5drdFQtnC7KuZuHey11Gt/Y0aDVaKnMqcerixJAvh8iKxSqyK6gtraXDuA4yhgy3UDfeyH2DhL0JnHz1pEAh3yLZV1tai0KpYNyGcWIA4/TS0zh1caLL7C5s7LeRvJg8cX+vyK6gtqQWU3tTMdE3+IvBenVsui3qRtuRbXHp6tIqHbBLVxfqy+tlndhNtU3kXM+h42SBQrYyt1L0ST6z+AyA52Ofx8jciMIHzZpZ/d7vR8TyCAzNDPHq50X/D/vL/MPi5GIeHn5IXVkdI38aKUkuqYxUzDo5iyOLjvC93/eSZEZVfhXFycVM2DoBzwhPvXuva3dXGmsbcUndSKOJJRqak32NNY08PPyQnq/1pOercjqwxppGChMKOfbSMbo9303SERT5qdDt1fe9viJt182fbnJ+5XkW3V4kUJ2OaoexpTGP4x7Lgr12fnayAi1zR3Muf3QZgBnHZkjs+O11t7n88WX6vNuHK59cYer+qTJffPi3w6krrxMDZkWJRagb1ZLv1j1nGrWGtLNpWLpYSoKNre27i5MXc3DuQbKisjj39jnSL6Yz62TrBU76oDJSUZJSwk8BP9H3vb50ntWZoBlBVBdWs2fyHgBJhb5arabwmpAAWVayDBNrE5ktv/zJZW79fAsQnpMZR2dIunnVjWpOLjlJdUE1gz4dhGNHR5lt9Rvqx4JrC6jKqxID+SN+GEG3Rd1E2uCWQdKd43eKWoFtR7XFu593q0Vsun3LwMQAKw/Bpxj29TCGfT2M/Nh8wpeFEzAhQLLm/Yb64TdU8BUOzD7AxQ8usji5uTOiTac2uHR1EWhEe67D2Mq41c6Q7GvZxKyNIfzN8D+V7KsprqE4SeiIuLflHr1e74W1pzUKpQIbLxvy7+QDMGHLBEkBRkuELAhB06ihrrxOPOfVV9TzS9AvGJgasLJmJWdXnCX1VCqLbi9CoVCIz0N1YTVnlp4heF6wLFHj3sudsX+MJfdmLo01jTJ/ZOCnA2Vz+ySOvXwMS1dL+q7sK3ld15n6tdvX9FzSU1YUZmJtQmFCIbEbY2k/rn2rFOBW7laYOphSU1zD1S+v4jPAR5xLABtfG9qOait+/qTtk556v3Z+dgz6bBAohMT2k3TSLQPai2IWyYpfHp1/RFlGGSUpJbIkv7pRjaZJg4GpQavF1lZuVgTPDZb9rgqVgj7v9pEkzQ/NP0RtSS2VuQJ7QchzIeTdyUOhVGDfzp4ha4YIBdZfDRW64Z0tKE4uliXY7vxxh8MLDgMw++xsGevDznE7Kc8qZ2XtSlTGcl9bq9VS9KCI9uPa600sX/3yKvbt7Jm8e7L4e9m3bWaBuLv5LpGfRTJh6wRJXKi6sJov23yJmYMZbxVK5Rp2jd+FeRuBDr2lLxH+ZjgatYaa4hqhC/kfFLX/byA/P591v//Ooe3bKa+owNrKinEzZvDcwoVocjRiZ9iMYzP0xoWMrYzp8mwXvYVXfxXqRjVbhmzB2NJY1nHVEo01jewct7NVuZDs69nUFNcw+PPBEl8m7WwaCpVClqBuue/F747Hvp29SL3dVN/Eo7OP6DyrM66h8jjhnql7yLiUwTOHn8HYypjC+EIMjA1IPZ2KnZ8djh0dST6RTN7tPF6MexFLV0uyrmZx4d0LWLlbYeNtw6MLjwQWs7/7gmYOZniEeUgKhXq/3Zsuc7tI7IkOtcXN7G0mtiYMD262Xx3GddDb7azVagl9JVR8RkxtTSXxyE7TOwl+zN+13J2CnERfoqGqgVOvnyJ4bnCr54q2o9piYmvC0UVHidsunHV1yb4hXwyhoapBZJECoYiqJb0xCMl5a09rKnIqZNruRQ+L2Dt1L4M+G6S3YSN4bjAOHRwInBqIQ4A0Jvhbt99wDHSkx+IepF9MF5nUAF6Kf6lVNqz4XfHsn7kfQNbl+1/8Z+K/yb7/4j8OUV9EkXE5g2cOPiPZ4Oz87ej3t36cf+c85985T9/3+0r4pAGxOqauvA5Nk0ZWjdrztZ4EPvMEl/XfYWJjQuDUQOJ3x3Pjhxt6q8dB4FF3DNRfDfH43mO2j9zOyJ9GSjjcNWqNUFHXivENfCYQ5xBnOj3TSXZNRyvw+N5jGmsbxWSfU5ATs07NEji0z6W12pkXsSyCxppG7m29h0YtrWr3GejDu/Xv8mP7H0nYm/CXD74g6NKoG9SSYI8OqadTufPHHTpOFuieWgZ4+77bl3Mrz3H9h+uySh23ULd/qGPVml4MCHQGZo5mjPlNTp/WWNtIVX4Vj+8+RmWkkjl+5m3M6TKni169BBMbE4LnB5N8NFmS9IjfE09RYnNVTWvoNL0TTsFOeisX6yvqyb6eTXVBtUgj+iT+Kh2lDpmRmWzos4H+H/aXUC/8WQxaPehPV9Q+iVdTXyX7erbeoHJ+bD5l6WXMODajVWHsB/sfkHMjh/5/6/+XKig1ao1A3WKkajUo2BoaqhvIjMzE1tdWb5XkmbfOUFdaR02Rfv2d/wkuf3KZyx9fZtapWXod4n8Xsq9lc+zFY3j399ZL5xH1eRSdZ3eWUfT8X4Z9O3tRqFofNvTdQPXjarE7ryUyLmewfeR2CS2WDoM+G0RWdBYe4R6CHqCThYS259H5R1z//jr93u8nHh6v/3AddYNaEsCO+T2G6sfV9H1XGrSxcrPSe8AHoSL+9JuneXjoIeZtzFn6eCmASH1m7SV0pbZW3dv3vb5YulliaGYoC/a0G9MOa09roTjl3QuMXT9W1LTSoa6sjqb6JiycLHDrKT2c7Jm8R6RzawmPcA8yLgk6lyUpJTTVyyv9rT2sCVkQgn17e1mxjkatQV2vFiibW+l8PvjsQTIuZfB61uuSqmUQEtmn3zxN8PxgBq2Sd7KJ9z9lDwXxBaIGhg5PdhroUJ5Zzo4xOwhZEKI30F6ZV0l1YTVuPd30atG8kvQKqadT+cH/B8ZtHCfpvlY3qtE0auj+QncZPTYIwR51g1q2d+pQV15H7MZYrL2sxWSfulHNNx7fAELQ8fCCwwxZM0RGE7l50GbKM8v1VqJW5VeRfjEdI0sjIdmkp6OxTWAbvZ2hLa8PXDVQ8jf3d93nzNIzLLi2gOrH1ZQ+KhX2WJVAWX522VlCXw5t9VDu3tOdlxJeIudGjt5ir+TjySgNlHptqqZJQ8zvAqWb/zB/vSwFCqWi1f3XtbsrWm1zIYRuf1c3qIWArrkhuTdzSTqWRLdF3bDxssHS1ZIZx2Zg6yd8V86NHBL2JDB+43hURipmnpjJjx1+pPhhMXb+dhKqu5bIvpbN3ml7mbRjkizZ9+DAA7KvZdNQ2cCAjwZI/C+lgZLXs1rv/Lv00SVu/36bZSXLZIldlZEKSxdLujwrBG2etFXxu+I59uIxZp+djZ2/HVqtluivogmcGkiX2V3o/XZvGqsbubzqMp1ndubyqsvcWXeH17NfZ8AnA3Dtrp+iGIRiLAtnC5pqmwTf+ImihO98vsOrn5defW5TO1PmX51PyokUNvXfpJcS/dgLx1gQvUASGFYZqcSOT+9+3rLKbYC+K/ti5W7FobmHKH1Uqrfr0MTGRJYkzLqaxb5n9jFx28RWOzRBWGNx/V+hydiclqvT2tOa99Xvt/q+E6+e4M76OxhZGkm6orOisyi4X0DXhV3p90GzT+g/wh8zRzOsPayxdLXErI2ZQCG/N0EWGGqsbUSr0WJk3tz1WZ5Vjl07OxwCHGg7si23fr3F7d9vM2X3FPyH+4s2pU1QGyxdLbn65VVu/HiDOefnYOtrK/MT98/aT21JLUvSmim/1A1qFCoFmiYN24ZvI2hmkF6t1pa4vf42N3+8ycTtE3EMcGTfjH1iMUz019FkX8tm0o5J4plzXa91mNqaMvPETMnnDF0zlKDpQRycc1BkVFU3qrnwrqCrZ+dvh3uYO7UltbJCnW3DtzFp5yRKUkowMjeiTac2GFsZi92cBsYGsv0WhES0ukGNhYuF3nOJDk8WM5g7tk4B3HFKR3wH+xI8N5i0c2lU5FT8Qxq2ltIDDdUNHH3+KF59vWSdMHVlddxae4vyzHLUDWrM2pjJbJOJtQnTDkwj9UwqpWmlYmFbQ1UD93fdpySlhKjVUfRZ2YfwpeGELw3HxPbPFVemnUlj3/R9mLcxZ+aJmaLd1nU0uHRzYUn6kqfS/dr42HD+nfP0WNxD9OFURirajWkn6s411jRSX1kvBHdVCtSNajIuZbBlyBYAmV8HQhd59rVsbv92m4GfDJQkTtLOpVFdUM2VVVcoTS3VS2m+a8IuQW++t6cs2ddU10RDdQOB0wL1Bo5LUkqI2x7HlVVXsPMXEpM5N3NE6j9dIFpXhFH0sIio1VEoVdJ9syUF7p+BhbMFDgEO7Bq/C1M7Uxmdfpc5XaivqMfK3UrvHO+asAu3nm68mSen8zyz7AzXv73OophFKFRCQk533qvMq+TCexdoN6adjDEJhKKlPu/04Y+IP3h87zEDPhpATWEN1YXVYgFYyskUto3Yxph1Y+i6oKvo4+uKeFaZrsJnkA8zjkq7IK3crVCohALJ+go5Da2pgylarbbVM61CoeDtyrf1Xjv79lnU9WoilkfQblQ7vck3hVKBQqWQFVPrKNhb7gUF9wtEHfDqgmo2D97MM4eekRQvlSQ3Fzjooy3938Q3X33FiuXLGWBoyGt1dTgD+bm5bP70Uz7+8EM++fhTJm6byP6Z+3l0/pEs5rNtxDasPKwYv2n8v+R+tGqhAzhgcgCmtqYkHU3Cd7CvbK7v/CFQIeu6rZ9E0MwgYv8QNCFbFpYff/k4KmMVCoUC8zbmeql2n5RNaKptInZjLCHPhVCUWMSF9y9QlVdF7s1cXk19lY6TO5JxKYOdY3fiPcCbBdELMHMwoyy9TEwiJR9L5uZPN+nxSg9M7Uzx7OPJ83eex9rTmrL0MnaN30X/j/qLcTCvvl6y/cyps1OrlL2OHR15Pet1Us+kyvyA6sJqUk+nYt7GnNriWtzD3LHxskHTpGHr0K10eqYTpg6mOAY4SmKvARMC+ED7AbkxuaK/qfOzDEwMmH1m9lNZmjpO7ohCpWD3REE/u6V90tnYoBlBKFQKjK2N2dh3o15WsN+6/kaH8R1kSV0rdyuGfTNMoqf85Pd3nNyR02+dRtOkkRTn5t3OI+92Hve23MPS1VJSjN/SJ0s7l8aDfQ+IWBaBjbcN1l7WKFSK/1J5/v8Q/pvs+y/+Y2BpKQTDCuMLST6WLIiVulg0J/wUcOlvl8S/1zkx9ZX1rOuxjo5TO6JQKnh89zF5MXmYO5nLKvb9h/lTklJC0rEkvPp4SYy8rY8tk3dNpv+H/fUmY+L3xLN32l4m75osF79F0CLRarSMWTdGlnjT0cRkX8/m5OsnhWrLFo7Uk85vS2g1WgZ+PJAhXwyRdPoolAr8hvoJAtVrrj61Hbv/3/qLlbv64BzijIWLPGj3Z7BjtMDnPnXfVFnled6dPOJ3xRO/K14WzAS4/dttHDs66g1Y3t91n/0z9jNl7xSRAkCHsyvOom5Ut0qDV55VTuLBRIoSi5i4baKExij3Zi4b+20EhEDJaxmvSd5769db3PzxpkzrD4SD2rj18kPDg70PiN8dT9+VffWuHR1curmQfT2bjCsZsoBRbkwusX/EMuLHEXrn8vx758m+ms2z556ufaUP6ZfSAbi39Z7eZF9NUQ3FycU4BjjqDfD2Xi4/dP5Z2Pratqo/FzAxAPcwd700UyBUJx578RiNNY30XtH7LyX7Ng/aTMalDAZ9NkjvoflpqMypZNvwbYD+yqcOEzpg62crKzb4q9DZvJa4/LFQFd9a8vPfBR1lh040/Enc23IPlZHqPyrZB8JzdXb5WXq93kvWgeXQwQFbH1u9Fc62PrZELI/g9rrbnFh8gjdy3sDExoSG6gb2z9qPazdXHNo7sLHvRpltO7LwCKVppVTmVuLV14v+f+vPnfV3aKxulCT74rbFURhfKAsK1VfUU5pWiqZJg1NnJwm1YH15PcnHkrFvb0/wvObvHLdxHHHb4qgteTqtkO6gro86Mmh6EEHTg0g8mIiRhZFML0KnE2Fobsg7Ve/g3MUZx0BHPPt4knggkerH1dxZfwcLJwsCJgbwWuZroBXWcvDcYL3Bbh2qC6sxczDTe1+5N3NZH7Ye7/7ehDwXIqFx0qHtqLbYt7fX2814ZdUVko4mkXo6VabxVJZRRvyuePyG+mHhYiGj3bz+w3XSz6czcftEWdGAVqOlsbaRk0tOkno6VRbwOTTvEFlRWa0GbazcrPAI8yBieYQsOXB3812OPHeEGcdn6Kf2WxpOp+ny4iAQNNUS9iQw99JcCcV19rVsyd/l3c5j86DNMhvn2t0VGy8bvYnR7GvZ7Ju+DxAoZJ7UpcmKzqKptklvkLm+op67799FPU4tsyM6P+/Sh5eYeXymWJEMQpLG1teWmz/d5OZPN+m6sKveQh7HAMdW/aATr56gNLUUcydzluYvlVxTN6o5/tJxQNDvmh85X3I9fk88N76/ISSiurnK9I2ikqN4cPABOctymLh9okhPmXgokb1T9zJp5ySq8qqI/DSStiPaYuNlg5GFkSToNH7jeMZvGi8Jzlm5WeEY4Mjw74frfabvbr7L2eVn6fdBP73FXnun7UXTKBR4hS8Nf2pw+0m0H9seSzdLHh56SPLxZIZ/N1wsiGisaaQ8sxxzJ3O9RV4u3Vzo824fSRXzkvQlGJoacuqNU9zbco+wpWGcW3EOjzAP2o9tL1Cd2pvRd2VfGmsbyYzMFCvGW0LTpOHmzzc5/cZp5lycI/OjrL2sn6or9Ef4H4Cgt9VSk3JZyTIOPnsQ74HesveUZ5VTXVCNU5DTU/V8AqcE4j/MXxawbqpv4sL7F3Dr4caQL6SJEQsXC9zD3FEoFfzR+w/ajWkn87f2z9pP0tEk6vu0noy5tfYWN3++ybT90ySJSl3n74jvR0iCwjVFNWi1WnwG+VBwv4DS1FL8h/vjEuIiBrKHfj2UskdlVD2uok2QPHm5d9peko4k4dTZibA3w+jybBd2jtspdk6lnk7Fob0DtcW1aJo0eIR7YN/OnjWOawDh7OAa6irpXNZqtER/E82ZpWcYtXYUoS+Hymgg/4j4g+rCapakLWHM72NkHTvJJ5K5+eNN/Ef4021RN1RGKtQNavJj8/m5489MPzpd0o2UfS2bhD0Jki4MK3erVrvqXbq68GJcs65dU20TnWd1pv249nQY34FLH13ij/A/eCnhJeza2TFw/UDOLzhPzo0cjr90nJSTKQDMi5yHZ4Qn7Ua3o93oZt8k43IGt3+/TdjSMJy7OBM4NRBTB1OxayPpaBInFp9g5E8jRRty6s1TpJ1O44V7L4g25N7We2RGZRI4NZCTr56ksbaRDhM6MHTNUMJeFz4r8vNIzq04BwhUaQqFQnaOOzD7gEAT26JTLDMyk7htcSgNlHRb1I2saKHLI2JFBLY+tpx7+5z4t06dnQR6+idg7Wkt056rKarhyHNHxKTmlVVXCFkQIuuIeBp0e3h1QTXFScVi4LWmqIbv/b6n+0vd6TipI+fePkfB/QKmH5ku6zBtN6odlq6Wkm6JvNt5JB1JEve2J6n0dk/aLWrgdnuhm95zT115HQETA+j+QnfZ+or5NYaEPQlCAYWH/mKvipwKfAb5yJLQIKyLPVP2MGnHJL2Fw3um7qEkuYT5V+eLdjv3Zi6X/nYJ737est/Y1seWV5Je0XsmPP/ueQzNDOnzTh9ur7st6iI/d/05vYlG5y7OjPpllN7uen00ui0x/PvhT+267zyrM3Hb44j+KppFtxeJ9quutI476+9g6WrZqgaiQqmgrqxO1EtsqTMNApXw8O+Gy6RCQDhbBM0M0lt07DfUj/ebWi/CGLd+HGUZZWRfy8Y5xFkvi0vW1Sw29t/I0C+HisVkSUeTiFodBcCBWQfwSPUQ11nUF1FEfhbJ/Kj5dJ7VWW/Xqp2/HfOuzJMk36seV4n/bjemHUlHktg8cLOYXD35+kmqH1fT7YVuervC9EHf2fbfgW+++orPVq7kslpNT7VUU35mXR3XgTEfvo/yYyWvJL2il5GiurBar2blPwsDEwOxa/LQgkPE/hFL95e6y1gy2gS2oeuirgRO07/2O07qKGgDP7E2hqwZgkKpIG5bHCZ2zc/mzV9ucvyl42JneVlGGXfW36HtyLa4hrry4v0XMbEx4cH+B8T+EUvbUW1x7OiIQqmgxys9cOvhxrqe60i/kE7a2TS6v9BdknTrs7IPhmaGfOX8FdOPTKfd6HbN1xVCp/TTtDd12DVhF+VZ5Sy6tUjyutJAiZW7lYRqV4dtw7eRdztP/P+J2yaKNtuzjyeNtY08+O0BXv28qCmqwTnYWcJYln0tmwvvXiArMovs69ksuLoAhw4OHJp/CEsXS567/lyr93t/+31AYJvoOLkj6ka1pMhs4raJ/NDuB0qSSwicFoi5k9RWXfzwIlqNloDJ0n0VBArStiPbcmD2Abq90E3v2DVNAt2ohbOFJNn3gfYDPrf7nLrSOlkRUN6dPIzMjbBvZ09+bD63frlFl2e7CPTtEZ5PtU3/xT+H/1c2Tx/+m+z7L/4joFKp8PMTKsfGbxqPiZ0J33h8w2uZr4mJAANjA4JmBOHZ11MSENSqm1umdcnAvu/31UtDBhC3I46L71/k+djn9VZqP0mfqIOVmxUBEwJa7bgoTirGyt1Kr3C2DoUJhVz/9jp+Q/wkQSmtRsuOsTvwiPCgz9vSIFh1QTVfuXxF6CuhEt2nGz/dIH5nPN1e6IbvYN9Wk3VnV5ylobpBr2ZUdWE1eTF5DP1qaKvj0t1fa0msjlM6cunDS2RFZ8kOib2W9MJngA8pp1Jk9AixG2NpN7adLPgBwoEm9WQqDh0cZAk3ECqMCu4XUFdSR/iycFmAb9GtRVz57ArXvrlGU500UGDlbkX4W+EYmhni0k1O2eDU2QkjCyOKk4tl361p0lCWXoaJjYkkcOYz2AezNmaoG9RPDbRXP64m8tNIjCyMZEEqOz87gSKklQ7NiswKHp1/xIeKD3lf875e+tPW0PW5rrj3ctfbXQfw6MIj9k7dKwlUtoROz/JpiczWEPN7jEBBNkHu6JjYmJB7K5dbv9wi/K1wWdLv8seXMbYy5oV7L2Bk+dec8bwYwTH8M87nkzB3MsfA1KDVAMM/Opz+GbS0eS0x6LNBqIxVejsD/p2wdLFkSfoSGfWZDvOj5nNr7S2STyT/x1SEZUVnsaHPBrRqrd6D75jfxlBf1ciFg+V0CTPDzqk5GOrQwYHBqwdz/YfrmNiYiImvxppGKrIqqMiqoOuirmKiJu9OHkeeO0L4W+EM/WooCXsTqC2uJfqraCKWRYg6Ny0xcetEvV1ux146JtJjttwDQQg0vtf4nuw9wXOC8eztSeRnkRycc5CK7AoUCgWvPHzlL/1mCw8vRKvUsu7OOtk13fOvq/A3czBj8OrBJB5MZOGNhTTVN/Fjux/JuZlDwMQAyX0XJxWTG5NLxFsReiuRt4/aTlV+lV6dMTMHM7o824W7m+9S+KBQb7Iv9KVQIbiXXIydn53EVt3dfBcQEtpPak6WpJRwdvlZjK2NZRoYIHQfJx5M5NH5R7JksY23DYuTFrNnyh7s2soTxh2ndHxqcKQkpQRja2O9+jz2be3p8mwXKrIqiFoTRfDcYEnAyyPcg6MvHqUyp1Kmg1OSWsKlDy9h6WopqazdOXYnlq6WPB/7PBf/dpHsa9l6K0qfRk3s0s2FwZ8Ppuhhkd73nn7zNNnR2dj62jJ+03iJDoWmQUP6kXQcPeUJOV3AKeVEiuyaylCFtac1pWmlAHrXT1FiEfF74mV0cjqM/HEkF/92UW8yUGWkYvbZ2cRti9Orm1FXWkdBfAGb+m8iaEYQE7dJu4hOdzpNrUEt1oOsJUE0G28bQhaEYOdnh98QP9qObNtqELfleq0rryPys0jClobhO8iXgvgCFAqFLJHdWNOIylglPGt6KDwnbJmAsZUxbqFusgSUpklD3PY46srruLvxLhHLIwic2ryn6RIQ51aeI2FPgtAh/Xd3Kft6NpsHbmbImiECtamtqcTvcQt1Q6FQsK7nOvp/2J/Ql0LFwIyFs9CdpEuM2frZYmxpLKE23zJ4C1lXs+j9Tm9ZwvnsirNEfxWN7xBfvb7h3Itzid8Tz/FXjjPos0EyzT77dvYUJxXjN8xPsk5MbU3FQG9xUjG/dv2ViOUR9Huvn0jd/nrW66SdS+PO+juM2zBOUiRSklJCaVopHhEesuCcul7N1S+uEjw/WKYF11TbRHZ0tkipqC+obN/OHtdurhjWVqA2MEahkI4pKyqLoodFqOvVMvaO1vSa249pz5u5b5J8IplTr53i0flHojaVDnf+uMPVL66yOGUxdnPk9s1nkA9VeVXUV9SL+nDB84K5o77D43uPKc8oJ2JZhKTSvrqwWvx3m05tiFgeIfGFv/H8Ruw+SdyfKNMRfhz3mNxbuXj19UKhVEh0JnVIPJBI8vFkko8nC+vTyJTQF0PFhP6O0TvERBvAlN1T0Kg1Ep9aX2co/F1PN6NMUqAYuymWe1sFukiFUoFrqCtdF3XFxNoElUpFn/l9qE+uJ2p1FO7h7hiaGeLR20OSyK7MraS+oh6HDg6UpJZwb+s9Ok3vhHMXZyZsmUBBfAFFiUVYOFmgMlJham8qSTxr1VoK7hfwuc3nvJn3JoZmhkR9EUVBXAFlaWWoG9WUppby8OBDsTOrPLNcTPT5DvEVaUifLP7IuZmDpaslv4f+TpvObRi3fhx31gtdKZ2e6cS2EdtwD3Mn704etcW1ePXxotMznbi/8z4mNiZ6n9O0s2liB1zL79N149l421BbUkt+bD5ajTA2O3+7P1XwV1sqSB2M/Gkk9u3sqa+sx9jSGENTQ8LfCsc5xJnNgwTNPvt29jId14aqBg4/d5gO4ztI2IGs3KyIWBGhV18LhKBz0YMihn07DJ+BPnqp+LOjs9k2Yhtj14+VUTz3WdmHkPkhrTLWAE+lHLf1syX05VC9vggI2mRNdU14hDXv252e6YRXXy/JWoz5LQatVitQyethiIjfHc+VVVeEoo53+nDt22ZNyot/u8jM49JEZFZ0FgdmHaD/R/1blVh5GoLnBFNXVkfcjjjadGojofTtPLMznWd2JjMyU7bW7Nvbs7x0OcVJxRycc5CQBSESf6ihqoEdY3cQ+nKojDI6+3o2UaujaKprYvKuyRhbGZNxJYNTr52i73t9ufTRJfLv5OPVz6tVfc9/hBs/3CD6q2iWpC+RJZvVjWoyozKxcrOSJJzvbhJ82THrxnB3411+CviJcRvHETQ9CAsXC1y6uTyVZtPIwkjm4/gO8uUD7Qds7LcRA2MDxq4fKz5DAKmnUlEoFAI14J9Aa2fbfzXy8/NZsXy5kOhr5W96Akfq6+n79gpmzJyJvYN8/S26tYi6sjoOzTuEz2AfveeLfwZHFh0h9o9YAG79fEuW7PMZ6NNq57W6QU11YTXeA7xlNk+XxGpZIAKQdlrQ5E47I/y3PKuc699ex9TeFPde7hhbGvNryK8EzwtmRfkKDM0MJUwpbj3cRDmFhqoGih4K+42BqSDFY+liKSZFb/1yC//h/jRUNWBgaoCpransjF0QX8C9LfcInBYo2ro9U/eQeDBRL5VqY00jJSklWLhYyJL7LZ/rKXuniAUFKkMVmiYNuTdzeSP3DRprGvnW81uC5wfTfmx7jr54lJi1MQz4eADP33mezKhM1A1qVMYqtBqteJZ/kg1Gh7MrzpJ9PZuZJ2dSmlbK977fM/fyXLz6eLGh7wYqcyp59vyzlCSXAEJjxZMx5OqCaiycLVq1E5omDVX5VXr16I8vPs7NH28CSIrTdHju+nMk7EkABDurs+2bB23GJcSFZ889S/cXutNldpc/3Rn/X/x1/L+yea3hv8m+/+I/AhqNhoKCAtq0aYNSqcSrjxeaJo2kFTkrOov7O+/j0VsaYDKxMRGdEHWDGoVS0WrHzf5Z+8UusycTC6WPSon6Igq/IX54D/CWVQB5hHvgEe7BqTdPkXQ0iUGfSoMQL957kdZQcL+AjCsZ+A7y5bXM12Qb2fn3zpN8LFmvBpyBiUDt0ljdKOH4ry2uFbXUfAb6UJRYJOPwBqG6Nv9OPk6dnQicGij5jtybuWwftV0mXN0StaW1rO2ylpknZuql5ur7Xl96vNIDQ3O5g2lgYoBbDze9lX4JexN4dO6R3k45na7A6F9HSw4mOjwf+zy3frnFicUnCJoVpDeA1+ftPrLEKQidZvoSjDqYO5rTUNVAeaZco6nqcRU/tP0BlZGKt4reEgNJQdOD6PJsl3+osRa7KRZTO1O9FLHWntb4DfUj43KGSAvYEuM3jadN5zZkX80WaWP+LCycLPTSrelg39aewKmBMipDHT5SfgT8c/zeRxcdBQQH7Untn8aaRpKOJXHjhxt0ebaL7JmctGMSCqWiVW2Jp2FF+Qq0Gm2rtH9Pg4m1CStrVj71b+K2x2HmaCbRMPsreNLm6fBXuxD/lXialqWZgxkXP7hI+7Ht/2OSfXVldRhbGjNq7SiJQHVL/LE8kYKf93N56hQ+2CV3xnsu7ikJ7LW03dtHbhcT7zk3cqgtqaWprolOz3Siw/gO1BTV0FDdgKm9qd4kqr4uNBD0InNv5hI0K+ipFCM6NNY0cuGDC+RczyHzSiYgCKk/+TxrmjRsGboFzz6eNNU24drdVRLgP/7Ocar3VbNjxg4+H/w59mbSg/Hwb4dLqvuvfXdN1Jt16+lGyPwQVpSvEDVIor6I4uzyswz4eAA513NIOpqEjZcNHad0lOztIASdYjfE8mP7Hxnz+xhJUMbO347xm8bT/aXusi6Pljjz1hnubr7LO9XvSAIeE7dPxNjKmNu/3yb662hJcs0t1I3nrj8nCai0xLj143h07hEXP7jYqjbflD1T9L7edUFXqgurOffOObz7e8soJH/q+JNe/V9opsC59PElLr5/Ef9h/jK/4d7me3j185Il+9x7ubPw5kJqimsozyoX7eqQL4dg5mBG8vFkytLKZInklrj65VVsfGxkNtvaw1qmPdgSfVb24d6WexQnFcv2KFMHU57LfI6GlAbWh62n73t9xUN/z8U98R/mj307e3Ju5FCSUkLHKR1RGQqH8hnHZqA0lNM26lCYUMjF9y9y8f2LzL00V0Yf9DTab6VKie8gX5mujg7dFnUjZEEIx146pt+X6ZxAjl0OxjOMWRnRvG+4hbrhFtr89y31/vLv5rM+bD39P+xPxFsRFCcXU5lTiUeEB7XFtUR9HoVWo8XM3ox1PdfR6/VeMi3K7i90b5VuHmjV5oHQbXZwzkEcOzrSVNekt+gAYOAnAxn4yUBJIMTGy4Y+7/ahNK2Ur12/ZvrR6bJnw9DMEMdAR0ztTdFqtJRllGFsaUz40nBST6cStz1O1tFcW1LLjjE7yLqaBejXwnQOcSZoRhDDvxveaqdixqUMbv50k77v9ZUk+8rSyyhOKiZsaZgsSXTz55vE/BrDhC0TMLUzxauPl7gfthvdDnMnc4ytjaktrqX4YTHqemkXQezGWK6susK0A9PwHuAt+tkH5x7EIcCBxSmLybiUQeTnkYQvDRf3A+dgZ0b+NJK7m+7S+53eejtQ+r3fj7zbeTzq9g3ZHQbS5NLs11bmVrJlyBYilke0WthRX1lP4oFEHAMdJfpNe6buIfVUKl59vZiwdQImNiZc//4617+/zswTMwW6rF7urXZK9lrSi15LpLan5+KeGJgYcHTRUUIWhEgSfVc+u8K9zffo/1F/8m7l0e9v/WQJD//h/pi3Mafrwq56g9bpF9MB4TzYGvq+2xfPPp6Y2plK9pmhXw/l9BunASEYNnj1YNF2tlbk9CTurL/Dw8MPAeFM6NrNFc8ITxwDHfmt22+A4LeO+VXoPFY3qfnM8jMsXSwZ/PlgQhaEyOQlIldHip1w4cvCGbRqEJ2e6SRJ5l147wKJBxJZVrJMooenw/Bvh2PlbkXq6VRRBmDi1okcfeEooa+E0n5Me6Hw8e+P8dEXjqJQKrBwsRDoAEe34wf/HySf2VQvUOW+kiisq/Xh68XfKeyNMDpO7khNcQ0ZVzLo8WoPlpcsF987acckJu0Quicf7H/A5VWXJbSTZo7Nv8H68PUYmhoy6/QsDM0MJXbaI9yDCx9c4PJHl3kp/qWn0pjqYOtjy6KYRaScTGHLkC3Mj5qPR7gHxlbGDPliCGlnhWB421FtZd34ABv6bCA/Nh/vAd6S1228bRj8WbPvkH4pneKkYkLmhaA0UBLxVgQRb0WgblDzifEndJreSaZnZ9/OnvC3BEpSTZNGcj5x7uIMrZP8iNBpyD1ZrOwS4oLjl44cff4oHhEeEtpVEPYxjVpDU10TSkMlSpUSUztTmQ7t9e+uo9VoCZ4bTE1RDSY2JpLn6MH+BwDMPi3QB07ZLfg/ebfz9CZ2lSolxtbGrXZF6wLaXRcKxXP6znsV2RXsn7Gffn/rJ9MOBfDs7SlLYilVSkxsTKgtqeXu5rt49ZPSCmqaNOTH5ku6lhMPJaJQKIj6PIqsq1kYmhmKBRTqBrXg49c3UV9Rj8pIRfcX9e+/mZGZ3PjxBo6BjgTNCJKNKXZjLJW5lQz9aqj+zsmV57m65ipdF3WV7AndX+yOV38vTr1+CnW9Gvcwd3Gv6TK7i8gSlX83n8wrmQRMDJDEErQaLedWnsPUzpSIt5r9uNrSWjIuZ2DtZS3zZ19OePkvabW3drb9V2Pd778zwNBQ1tH3JHoCEWoF70x7m58P/aJXd7mxtpHYjbEYWRr9j5N96gY1iYcSaaptwszRjJrCGnosllPxA3zv9z1efb1kNLP1lfUcmnuIbi90+4da0zpM3DaRmuIaLJwsWG2zWqC2j38JcydzNGoNjTWNWHtaY+NlIztXnlhyAnW9mtFrR2PmaMaWwVvIuJRB0tEkkbmmvqKe9uPaU19Rj/9wfx6df8TWYVsZ/dtoUceuJUpSSoj6PErQZ/37Pu/S1QVNo0akCm6JgvgC1vUQikzD3gyTFB3OOCa30zqMXTcWjVqQbdLYaHjh3gviHDfVCH5tRU4FzsHOOAc7i7II+Xfzxc/QNGn0ni1KUkrQqrU4dXbixCsnAMT9262nG7XFtfwR8Yf499UF1bJk36ifRunVPgdB+unAswcY8MkAvQxvtcVC0t2rrxf+I5v3Rd0eY+Njw5K0JawyXUXW1SxxP+v3fj+xw9DI3Ehiv1PPpLJ16Fa9zGb/xT+H/1c2rzX8v//GfxK//PILnTt3xsrKCisrK8LCwjhx4oR4va6ujpdffhl7e3ssLCyYNGkSjx8/lnxGZmYmo0aNwszMjDZt2vDWW2/R1CQ9wF68eJGuXbtibGyMv78/GzdulN3LTz/9hLe3NyYmJvTs2ZMbN278W8b8XzRDq9WSn5+PVqslNyYXQzNDRv4wUrIpm9qZ4tTZibQzaZx645RIhdISgz8b/FRNHkNzQzwiPAieEyzb8Kvyq4hZG8PuSbv5NeTXVj8j6XASyceT9V4rfFDI937fc/2H65LX0y+lc/yl41TmVmLtYS2r1ClOFLrInqwYByGZ6R7mTuyGWFE0GgRjPvjzwfgM8iHvTp5YDfskFsUsInxZOEcXHaW6oFpyrU2nNoz5fQw513Nk96xDZU4lFVkVJOxN0HtdqVJi5mCmVxetMreSx/ceo26QO2Njfh/DjGMzKEsvk11z6erCrFOz8B/RenCu26JurKhYgXd/b9n1B/sfiAe6vwr/4f68mf+mLMgJzTzY6ga1uAkDfOv1rUj7+DQ01TZhbG2sl0oChHVyYvEJipP1UymGvxnO1H1T/3RgQvzeuiYyIzMpelik97qBqQHxu+P1/mat6Qf+WQxZMwRjK2O9NBl3t9zl+rfXGb9pPE5d5Ac41+6u2PraUp5VLqtY/0dQKBX/VKLvz+DC+xfYP3M/0V9F/9Of0dLmtUR+bD57p+3l0flH/9Pb/Eu4ve42Hyo+JHZjrN7rJSkljFk3hgEf/9/Sa3ga2o5oy/LS5a0Gva99d43EY2nk+/YiPl+a2MqKzmLrsK2knZM/EytrVzLsm2EEzwsWg+BuPdxY8mgJwXODUTeqaaprwtTOFBsvG5QqJVX5VRK6HBC6HCrz5HY7eG4wrzx8hX7v9ZMVgNSV1ZF6OpV90/exbaRgcxqqG4j+MprMK5k4BjryZt6bLLi6QKYPoNVoeXz3MaWppVxdc1W2j5U8LME92x2HQgfu/HCHkpQSvb+bDi2fr6OLjvKxwcekX0wXix7OLj8LCJXBSUcFeqtD8w7p3avC3gij/0f9hURhK3UM7j3d9dp7QPAJTqXQ++3esiRT0PQg2o1qx+hfR7M4ebHkmrGVMW493LBwsiBuRxyRn0fKPnvYN8P06pTUltQS81sMuTG5+m/4738T+VkkZ1eclV2LWBaBrb8tW4dvJelYkt73h8wP4bkbz8mo6vbP3I/KSCWbYwBjS2NM7U3ZNnwbN35o9l+7LuhKWXoZh+YeIuVkCokHE/XqjkZ/Ey0kTv9eRf5X0G5UOybvnMzzt5/XW6hTWFxIU0MT5ZnlkkrW/Nh8jr98nIL4AmJ+j2H/zP00VgvUWuWZ5Xxq/inn3jkn+zwdfAb50P+j/ti1tUNp+K+3+0qVkjG/jtHL3LB39F6iw6O5nHG51fc31jRSkVMhMg0YmRvhM8BHTChFfx3NpgGbqC+vx8rDisUpi1GohO44oFVfKO1sGmsc1xC3PU7v9eqCagruF8gYDgyMDXjm0DNM2DqBV1NflQUaLq+6zIa+G4QCuicqnm19bRn48UC6zOlCj1d7yKg2H+x/wPFXjjPihxF0mtaJpromvvf9ntNLT6PVaMmLyaP4YTG1pbWoG9Rc+fQKG/psoKG6gbL0Mnq90YuVdSv1shx0ntmZidsmtprou7zqMi7dXHgz/03J31z/4To/d/yZjlM6Yutjy9rgtWLHLwg6OI/vPebKp1ewcrdi5omZdHlW+E18BvrQe3lvjC2NCXsjjKWPl8qSDh3Gd8B/hD+7Juwi+VizXY3fHU/m5Uzs/Ox4dP4R51ack3QSPdj/gKgvohj1y6hWqeZA6Hh67NOTamtpstnMwUx473j977316y22DtvKwTkHid8VL75enlUunicGfzGYzjM7Y2hmiMpIhbGlMUqVkozLGTTWNBL9VTQb+m74077go3OC7zL619E8vveYOxvuUFNUQ1Ndk6Dh5mDGw8MPKU0rJftaNlFrosSzydh1Yxn06SBsfWyxcLLg6ItH2T5qu/jZARMDcOvpJsoS/Nj+R46+eFTy/dae1nSZ3YV2o9qJCYasq1kYWxqztGApC6IXgBbR18mPzSczMlPyGTG/x3Br7S3Z2AatHsTw74Zz8tWT4ji9+npRGF8o/k3LwLhGoxEYQR6VYe5kLkv0AZLCwsaaRpQGSgxNDVGqlGjUGo4sOkLigUSAp9LNhS8NZ/bp2eI5xamzEwuuLhC7Zg1MDMS9+dG5R+TeyuXN3DfptaQXdn52LE5ZLNJtarVaVpmsYtuI5nPNgqsLRPpk917uWHta4xLiwjtV70gKwDRqDfUV9Zx64xTrw9cTty2OC+9ekKwf5y7OzD47G5euLlTmVJIfmy95LoqTiolaEyUWsfb/sP+foiEuzyxn96Td/NbtNzIjM+n7Xl9ZIY+Ntw3hy8L1aurVV9aTHysEgeN3xsvWRUvEbojl6KKjskKJqC8EmkUdhW5L2Pra0lTfxO6JuyVdrjrUltZydsXZVv2BhL0J/Nb1Nw7NO6T3ular5e7mu63e94N9D1hlukpM2DXWNFJTVIO6sfmMPnXfVKYdnEburVy+cf+GmN9iJJ8x9MuhvPzgZbFTxLGjI44dHek8q7NeTV23Hm6M/HEkF969wP1d9+Vj/vtZ+vbvt6nKr5Jd/yXoFy5/cpmp+6ZKitNAoKo9NP+QXl28xtpG8u7k4dTZiXdq3hHtuQ4mNiYsL1mOcxdnUs+kAnDu7XOce/scvd/uzcRtE5l7eS4b+20kbnscvoN8WfJoCZ2mdeLVlFd5t/5d7qy7w7GXjsm+O/9uPvG74rn4/kXSL6TLrl///jo513MIeyNMb5H31TVXAeEs0BI+A31waO+AjbcNQ78aytyLc/V2SWVcyuDE4hOUpEp9+JwbOUStjuLssmZ/tDyznMufCL5LeUY55989L/u8qvwq9k7by/2d8vl7Eq2dbf/VOLR9O7Pr6v7U3y6giQuRx/T6kHHb4yiIK2BFxQqGrGm9EPzPor6ynr1T92JgZsBbBW+xOHkxQ7+Ss2U82P+A0rRSvR1duoKS2A2xMjvxc+DP7J22VyjMaRG3MzQzxNrDGpWRivmR8xn962gcOzpiZm9GZW4lPwX8hFMXJ6K/iub699dJ2JvAxb9dpKG6geyr2cT8GsMnxp+QH5vPmHVjCJ4fTMhzIWJ374lXT/Br8K/0/7A/voN9BcrN50JwaO9A3u08vnb/WmIrfAf5sjhlMe3HNbM29F7RW2+iD4TO6fBl4ShUCr1FA/oQuymWzYM3U1dWx+O4x1TmVOIU5CQylY3bOE44r381jLqyOskeY2xljJGlkcAS0ApT1tS9U1l4cyFfu34tnod1vt/QNUMZ98c48Tw7ftN4YjfFkncnT/Y5mwdv5tQbp2Sva9QaGqsbRbr9JzFp+yQ+0H6AmYMZB2YfEF//qeNPAJQ9KuNDxYcYmBpIJBV6vdZLZOmqLa2lIL5AXGcFcQUAehsa/ot/Dv+vbF5r+I/p7HN3d2f16tW0bdsWrVbLpk2bGDduHHfu3CEwMJDXX3+dY8eOsWfPHqytrXnllVeYOHEiUVGCU6VWqxk1ahTOzs5cvXqVvLw8nn32WQwNDfn0008BePToEaNGjeKFF15g27ZtnDt3jueeew4XFxeGDRMqZnft2sUbb7zB2rVr6dmzJ99++y3Dhg3j4cOHtGnz/5ZW7f9fceWTKzw88lDGKewU5ERFdoXoiFfmVOI/3J+6sjrubb2Ha6irWAFz6o1TKJQKGR2VrtpSH1y7u/JW4Vtc++6ajNYDBCqTu5vvMmnHJL2JCV3gzNjaWJbM6zCuA44Bjtj52wlt6s4WkkObvoCd5P3jO2DjbSM77EQsi8B3iC/3tt5rlfdboVDQdUFXfAb6yLrFrD2t6fpcV6K+iCI7Oluvdp6Ntw3Pnnu21c6HQ/MPkXsrl+mHp8uCPte+vcbVNVcxdzJn5I8jJW3sli6WbOy7EUNzQ16Ileo5mNkLgu5RX0QRND1IFvTJjclFoVTIqoJ1OPai4HwP/344fkP8JIndvDt5nF95nkfnH2HnZyejprj4t4vEbY/jxXsvyubRxMaEV5JeoSKrQsLLbe5kTvrFdPLu5LV6TwCjfh5F1eMq6srrZA5+9rVsrqy6Qo9Xe+ill82MzKS6sFovHeY/QtQXUVz84CJWHlZ6afLM25gzaPUgvcE1hULxT3X06RC+NFym66RDm8A2hL4cilsPN71VVZomDeffO8/NH2/yRu4brXYe6kPSsSTOLj9L7xW99VI4Pg1FiUX8FPATpnamLCteJrte9qgMgP4f9v9Ln/tnsG3kNqryqvAZ1DrFx78Duuf70LxDsgMmwJYhWzC1NyXklv4O4P+rqCmuIeNyBk5BTrKEya1fbmFaqiKtr7wru7a4lqyrWcT+EUvKyRT6vdcPYytj0i+mk3U1i26LulGRU8HWYVvFynkdNvbbSHZ0Nn3f70vw3GCs3K3YNHATIFTJ6rB32l4e332sd421hsIHhWwdthVApATUiaVnXsmkML6w1aCYykjFsuJlaDVaBnw8QNL1AjB9z3RmfzyboHtBRP0UhXt7d8lv9vDIQ04uOYlrd1em7J5C8JxgsdpRh+xr2bh0dcHK3Yq+7/dFXa+mz8o+nH/3PCpDFc4hzjJNAx0CJgTotW9FD4uIWh1F0KygVruv0i+kY2JjIuu4bwl93c2aJo1It3Vv8z0yrmRINLMexz3GyMJI7/dW5lZy9PmjKA2UDP9uuKSLBQQKmOKHQuGGTseqJQZ+MpD0S+nc/u22jOLl0flH3N18l4hlEZLuMB2sPKxwDXXVW2SjblCDVrBPTwYcqwuqMXcyx7u/NydfPUny0WQZVd69LfcA9AYoko4msXvybtT1aoZ+3az99GfQVN9EcWwxHn08eCPnDem1uiYKEwopjC8k9MVQ2o5sK3b2GJobYmpnSvzOeMrTywmcFij7vUysTej3Xj/6vSfXpAXhcFz0oEhWMQyCluBqm9WgheD5wTK2gcIHhTy++xjfIb56g/UAbR63oceNHmQNaabSif4mmgd7HzDtwDQeHnko0WC087eTVCt3mtaJNp3aYGhuiMpQhZ2fHbd+FpINw74dpreDPD82n4R9CVi6Wcq6A7RaLV+5fEX1YyFQ9GRXjNJAKdE0eRJV+VUUPyymPENICrl2l2sVuvd011t1Xl1YzeO7j8VkrUKlIPytcFy6ufDowiN6LumJqZ0pX9h9wfQj06l6XEVxcjFm9maydaEP6RfTifkthojlETJf6cK7FwiYFCBjqsi9ISTkMy5nkLAnATMHM0lSZuCnAzn/znm9DBV/Bq7dXRn61VBcurngGCj8zhq1hmXFy1AaKCnPLKfXa73o9Vovia9jYGKAmYMZCpWCuB1x2HjZyPywu5vvUpxcTFag0FXd0mqbWJvQ/YXu5N/N59avtwiYECDpxMu+KlCETt41WdLB8ujcIwriBL2ylnPYslt02oFpKFVKor+OFuyYFkkhxtUvr5J9LRvfIb64hbrh0tWF/bP2Y2hmyOLkxTw6/4ikIwJzw8JbCxnw4QD6/60/O0bvwMzBjJwbOTzMe8i9LffwG+on3nf+3XzSzqQROC2QmoIaKnIqxO+0crPiuWvN+jpGlkYyG9hQ3YCmUYOxtbEYxLu/6z43vr9B5pVMBn8xmMXJi8WikFNvnCI/Nl/SmXbj+xuoG9SyzlnHAEes3K1wDnbGxseGJ7GsZBmZVzLZ+8xeBn06CCsvKzq/3Zm7H9/l0NxDZFzK4O7muzgFOTFqraBhNurnURKN0priGkqSS7Bra4eRhRG3f78NQMSKCFSGKkpSSkQqNN0znXwimYrsCkmHRVl6GQ3VDRhbGZN8LBkzBzNsfW1x6eoiFr3E7Yhj/4z92HjbsOTREvG9uuCjgbEBKadSBPq/J2jENvTdIHamVz2uIisqC+cQZ+pK68QuR4Aer/bg+TvPy34r30G+LIqR6jYVJRaxtstaHAIceHz3MWeXnWVpwdJWi3yeRMtnOuVECgETm6U3KvMqOTz/MIHTAhny+RAK4gu49est2o9pL56NNU0a7PztUBmpMLIwkhSO5NzI4cJ7Fwh7Mwy/oX6EvRlG0IwgMYGq68IHQYsz9EWpPyCOe7AvhmaGss7VbSO2iUXM6ka13s7m4y8LVLQ6/caWyIzM5No315h9djZefbxk1w/NO0RmZCadZ3UWaZ9v/nKTM0vPMP/qfHHf0nWmlBiU0PO1njIJDit3KwoTCil7VIatry0NVQ3cXn8bpYFS7Jp5EgqlApWRSq8UxKTtkxj3xzgaaxtlPikISQzzNuYymRAQkvixG2Jx6erC/Z33Gb12tEjPXJxUzG9df2PAxwP0JnZ1OPr8UdqObIvfED9G/jQStIhnr9yYXIEeWU9MCAR/ytReXrwbPCcY917uFD8sxj1MvkdO3DpRbwGrukHNw8MPmXZwGppGDfG744VOqmFCsc/68PUi5bOOEUfXpZVyKoXMK5mEvRFGwMQAnLo4ybogTe1M8RnoI9H7Tj6RzLWvm6lYr6y6Qs6NHLFzM+tqFuWZ5cTvjm+VIvZ/A+UVFegXJpHDCdCaCxq1T+LoC0fxCPOQ+cH/CA1VDdSW1mLhbCHZ040sjJi8ezK2PrZ86/Ut5m3MGfHjCNx6uEmSSgX3haRL3/fka/Pw/MOAwCTzZCLIyt0Kcydzbv1yC4VKIcbtipOLyb2VS5vANrh0dUHdqKbqcRVGFkJnV/cXu1NXVkdpWqlwjgt1JfdmLt1f7M7Cmwu5vf42R547wpmlZ0S2hpbnMe8B3hiYGohjdezoyNjfxwLCOcna01oit2JkYYSdhXy93N95n8q8StnZwdLVkiGfD2HI5/KE6/UfrnPy1ZPi/5vam7KsaBnGlsZYulqi1WhZ23ktARMDGP3raKFoyUrY/w1MDIj+OprTb55m0OpBNFQ1EPpiKLY+trxdoV9PvSVOvd6cpBu7fqzs+rLiZRx/+TgH5xwEoO3ItpI4YMaVDB6de6T3/OkS4sLCWwtJPJhI3u08ma3VQVecoYPfUD+yorJ4fE9oemqqa2p1j4zfHc+xF44x6/Qs/Ib40eOVHoQsCJGx6/wX/7n4j0n2jRkjTcKsWrWKX375hWvXruHu7s769evZvn07AwcOBGDDhg0EBARw7do1evXqxenTp0lISODs2bM4OTkRHBzMxx9/zPLly/nb3/6GkZERa9euxcfHh6+++gqAgIAAIiMj+eabb8Rk39dff83ChQuZN28eAGvXruXYsWP88ccfrFjRujj6f/GvQ+groTTWNrJr4i5GfD9CQnVWU1QDCsFR0umPVT2u4sTiE/R+p7d4aE05kYJCJU/2gdDBcvnjy0zdP1VCaaMyVGHmYMbAjwfK3lNfWU/OjRxu/ngT38G+ejn7Dz93mDad2vD8bfmhxsrdCit3K+7vus++Z/YxZc8UWbAq5VQKpnamssBedWE1N3+6SdvRbSUBnbjtcRhZGtFQ2cC1r6/h2dtTL71F7q1ctFqt6Czqw+zTs1vVYzOyMKKmuAaNWqOXXqP4YTEFcQWcf/c8E7dKOxP9hvlRGF9IxpUMGqql1UvVBdW0H9e+1SRiRXYFN3+8iVOQkyz4sX/GfhqqG3jm4DPY+dvJAl3jNozj1i+32PfMPplgeG1JLRmXMlAaKLH1k2uyKQ2U1JXWCQk5PfQa9m3tZToG4W+Fc3j+YSpzK5+a7CtNK+XH9j/S++3esqC0ulEtaHmEeej93sjPIkk+nkzoy6EM+3pYq5Qo+qB7Vnq93kvv9dgNscTvjJdVPv4roGnSoFAq9K4vz96euHRzoTyjnJriGlkg9Xv/7ynPKKfXG72eqkOgDzvG7ACtsP5bJvu0Wi3n3z1Ph/Ed9AbRATGgVVtSq/fyhC0TmLBlwl+6nz8LXYL5z1Qx/yvhN8SPIWuGtHqo7fFqDwxNDVvltv+/iIqcCqK/iubaN9cY9u0wGeXYzBMzebEV9uV2o9vxduXbnHztJNFfRtPrtV4YWxmTeiaVyE8jufHjDYZ8MYSsq1kETA6gMreSlJMpeIR7YONlQ3Z0NrEbYrn80WVeTXuVLs92kVV9BUwK0BsIuPrVVc4sPYNbT0FDoaV2pK2vLaN/HY17L3dRj1KpUjLv8jxK00rJjckl/VI6RQ+KqCur0xvkUCgVMj1KdaOaqM+jsKyzJNUvldGnR+PZVRrcu/XLLcoelVGRJQRfDUwNGL9pPAYmBqJ+0y9Bv9BU38Swr4ZJ6LQHfjwQdYMaU3tTvevn1BunaKxtZPQvo2XXqvKqiN0YS+zG2FYT8M/feb7VqrpfOv+CS4gLQ74cQkNVAzbeNuI9pJxMYceYHYzbOI6RP42ksaZR8t7ITyO5v/M+7zW+J7NhNt42zD4zmy1DthC3PU6W7Ht87zEF9wsY8/uYVrWGvPt583al/MBZlFjE3U13CZoRhLWXwAbQsqN78OrB5N/NpyixSEYbk3o6VRxTS2rvX0N+xT3MnaX5S9kzdY/4t09ixrEZaNVavTSzBqYGWLlbUZpaqrcrcOe4nWRGZtJnZR/aj2sv8Rmq8quInBdJ0+tNMkpKXVf5mWVneK/hPcmh19zRHAtnCwoTCknYm0BdWZ1c+6y+icbqRowsjfQWjjgGOFL0oIibP9+U+YUKlYKACQE82P9ALORoiaQjQuGIThPpySDF2ENjsaqwwj/Vn+KHxWLQtLa4ltK0UhRKBW0C2whdcK1QJXv39xYP6011TVQXVjPzxEzMHM306iaBUGAWszaGF++/KKdY1/7dV2lnj88gn1b3k+qCah4eeYhbDzeJ/zjyh5GM/GEklz66xMUPLvLCvRfE67m3cjmz7Axhb4bpDUh3f7477ce2J+bXGNQNarz6eonU6WeWneHqmqtM3D6RkOdCsPKwYsR3IxjxnaCXqdVqeXjoIfum72PAxwNkhUJxO+I4MOsAWo2WzrM7y5J9z995HnWDmsIHhdj52Yl+koWrEGipKRTW7LBvhkn8gt4retNzcU8USgX1FfVcXnUZzwhP2o9tz5XPrvDw4ENmnZ5FRXYFBfcL8B/mL/PTHAMcJeeHzMhMNg3YxICPB3Dh3Qt0f7G7JKkDQtI+LyaPX4MFNpFO0zvJ/N2EPQnCczpU+Gx922/q6VTOLjuLc7CzJNk3dv1YRv8mBL9a2l3P3p6M2zgO52BnIj+P5Mb3N5gXOU+yLyTsSeDelnssiG7u6GqJtDNppJ5O5cG+Bwz4ZAAuXV3IvZWLpaulUDS3LY6IFRHMOD5DLBqpyq8SO8qvfnGV0b+OpsfiHuIa160PELS551yYI/lOdaNQyKA0VKJQKFh0S5osAqHYT1ew8Graq9j62BL6Uig3vr/B3c138ejtIVIvAvRc0lNcFzpM2TNF736iblSL0go63PjpBt79vZm0cxKmtqaCfuiueMLeDMPKywqvCV6UXiwl80ommVcy0aq11FfWt1rNn3YmjX3T9zF131Q6TOjAshIhYaxLhBTcL+DMW2ewcLEQk323frlF0pEkyh6VEbEsAhMbE069cYrEA4l0GN+BxINCZ+CTBX+6faksXegQAIGuX2WkEov8vnT6Evv2goasiY0JHSd3ZO8ze9E0avAd4svDIw+pr6jnwKwDjP5ttKwwIWFPgvh86/A47jG/df0Nv6F+kqIHQ3ND/Ib54TvEl0fnHpF9Lfsv+ZvWntb0WNyDGz/cIHBqoKQwt7GmkYwrGbj2cOXXkF/FwmH7dvZiss/U1pRFtxfxYN8DnDo7Sfah2hKhAKzzs4LdcAqSJlMe7BUCsr6DfVvtXkk9k8rljy8zePVgWcGnS3cXjCyNGPTZIL1nQBC6RwzNDfUm8ypyKnhw4AGdZnTSez4sSy/DzNFMcnZx7uJMtxe6SQLR9ZX1aNVa7PzsGP7NcNnn1FfU83Pgz3j29mTelXmsD1svJi4yLmWItJ46lD4qJetqFpN2TGpVi9zAxKBVH+m568/pfR1g2NfDGPz5YOK2x1GaWkpjbbP/ZuFswYBPBuAc7Ez29WxsfWwltrGpromkY0kM/HSg2JHY0l/Kj82nKq+Kl+JfQmmgpCy9jEfnH+HVz4v0C+kcWXhEr00HIW7i2s1VEmNqCceOjtxae4uDcw4ydd9U0e6eWHKCmLUxjFo7Co9wD/ZM2YOprakYv9EVj6EQ4mGXPr5E+7Ht8erjxaNzj7i65iohC0Kw9bHV67/Zt7Pn2XPPSl7z7udN/w/7k3w8mbYj21L8sJiMKxni9T1T9mDtac376vf5vwRrKyvyc1tn1WiJx0AbzzYETpEXxU/ZMwUjCyNyb+Vi5mAmK1rXB61Wy+EFh4nfHc/LiS/j0L7ZBzcwNiBwSiApJ1MozyynPLOc9b3Ws7JupUTqpe97fenzTp+nSrK8niUvztYlJUtSSiRnkuMvHSftbBptgtowP2o++bH5bOy7kcGfDyZiWQSjfh5F5OeRGJoZYutni99QPwyMDcRi+K4LunL7t9vk3MjRW1jdmv4vCLZwwdUFktd0fqypralYKBbzWwxHnz+KiY2JzI9uqmtqtSAg+kspi5KuGzhgYgB3/rjDtuHb6Pe3ftj527HGcY2orR2/J56MyxnYt7Mn5LkQSlJKuLPuDh3Gd5A1P+hD2rk0GmsbaTemHT1f7UnC3gTcerjRplMbrn55lab6Jvqu7Ct2JL+R84bM177+ndB9qY+5DaD6cTWH5h6i9zu9Zcm+lJMppJwSCkBadjva+dvhGuqKlZsVe6bsYejXzWea2pJaNg/eTOdZnQl7IwzXbq70fb+vaGNURqq/FD/8L/7v4z8m2dcSarWaPXv2UF1dTVhYGDExMTQ2NjJ4cDNXeocOHfD09CQ6OppevXoRHR1NUFAQTk7NjtewYcN48cUXiY+PJyQkhOjoaMln6P7mtddeA6ChoYGYmBjefrs58KJUKhk8eDDR0U+na6uvr6e+vplGoKKiQhyL+u980gqFAqVSiUajkRwidK+rn+Cdbu11pVI45Oh7HQTakD/zukqlQqvV6n39yXts7fV/1ZgA7Ozs0Gg0ePX3Iv1iOpGfRjLg4wFYulkKfLhxBQTNDiJkfogo6K5Wq7FwtWD2udlil4NWq+X5e8+DAvF7dPd+f9d97m65Kzh8f99PdPfeUN1AWVoZlm6WWDhaSMa0a8IuMq9k8mbem5SklZB9M1s0yroxjVk3BiNLI/E79c2TXTs7wpeFY9fOTvJ6VlQW24Zvw3+kP88clur3NFQ2cG/rPcydzQmY3Fxlc/T5ozRUNdBxakcGrBqAkZXw3U/O08F5B9E0angl8RXZPD06+4jjLx1n8JeDaT+2vWytqtVqtFotR58/ilMXJ3wH+8rG9OzFZ4lcFYlTFyfZvOoEiHVrrOV87Juxj7yYPJYWLZXNU9LRJHaN28Wg1YMInBEomSeA8BXhJOxM4PfQ35l2cBptRzfTWCgUCtqObIuVlxVBs4Ow9rFGrVaLY/Lq78XyiuYKXl3yQncPXv29uLLqCklHkwh9KVTyfFTmVnJ04VECpwbSeU5zoChodpDYDfXkb9ByPrJvZmNqb4pbTzfZmNzD3QXqLj3PtkKhIGJFBBlXBC2aAZ8OwEhl9KdthP8of95tehelUqn3mW+qa6K+sh51k1ryWSqViqb6JlaZrMLS1ZIlmUtkY2ptrDp8YvgJAPOj5+Ma2nzw0d172tk0do7dyeh1owmeGywZU7sx7TCyMGLAJwNEO/Zn7d6Y38egMlIR+Eyg5D21hbVEfhpJWXoZ4zeP13vvtv62LMlaIury/DvsnkajwdbWVjbfL8a/yO/dfufetnv4DvOVHID/3bY8fGk4Go1Gcj+6e+/xag++sP2CBwceMOP4jP+I/SkzSqh0bju6Lb5DfGX2zcrTCsO6HPzvnCDfNwyNxks2poi3I+jxag8snCxQq9V0f6k7pWmlxO+M58ZPN1hRsQKNRkPa2TQOLzjMiJ9GMHH7RCZun8jDIw/JvJyJgbkBYW+FycbU7YXmKvyW967TrKrIrqCxtlFyP2aOZoL2Sot5Ujeqqc6vJuloEieXnBT/Tl2vpvc7vcUxaZo0PDz0EGsvayycLDAyM8LEXggqJR1L4sJ7F5hvN5/vXv0O6xBrjG2MJfvZ4M8HE74sHM8+nqjVajIjM8VqxqXFS7Fws6D/x/3F57zlmI69fIy4rXG8U/OOII7+xNrLuJxBXkwe9u3t8R/pj52fnThPbuFuLCtbRuRnkZSllaHRaPSuvVs/3SLpaBITd06UBNJMbEwwNDfk+OLjJOxK4J26d1AaKFEqlVi4WtD52c7Y+Nhg5dUcHNHNU5d5Xbi/8z6bBm5i3uV5kjGpTFV4DfBiSfoSFAYK2XMz8/hMmhqaqMyvxMjcSLIPqdVqto/YjkdvD/qs7CN7nrrM70KnmZ249bNAwzcvap64b+jmY8vgLTh0dODZ889KXrfysKLroq7Y+tlK9lYUf6cbicunTec2WLpZ0nZEW9lzY9bGDHWdmvqqegxMm+2PQqHAd5AvryS9Iv79k2MysjaitqSW02+exsbXBju/Zl/H2NqY0PdCce3oyr2t93Du5ox9OyHI79rTFX4RAiD6bMfwH4aTdDgJlZGKoNlBoq+jm6e4nXEcnnsY9zB3xm8cj62/LVqtlksfXuLRmUfMPDOT0b+Npqm+STZPRuZGTN4zudUxtR3dFmtva/ZN20dmZCZhr4dJPiM4NphUv1S+eOsLVkxZIV7r92E/kfbYxdYFl1AX8fPLM8q5+dNN2o1tJ9EaUqlUZEZlsmXwFob/MJxO0zuxPmw9HSZ2IOzNMMk8dZzWEddQVyzdLfXa7DmX5kj+vuVzU1tSyw++P+DY0ZGcGzkM+XIIDh2lSWOlUon/CH9MbE2ENfH376grryM/Np/4PfHc/v02EW9H4NrdVWLLK3IruPThJRQqBV59vcR56v5ydwzNDbHysGLs72MldqypromEvQkcnitUtbecK92YMiMz0Wq0LH60GCs3K5ktcAxy5Oyys1z7+hpLHi3B0kMI6Az8dCAdp3ZkQ/gGHAIccAqR+qv15fXk3c7D1t8WpYGSq19cpfEVIchTV1ZHZV4larWahH0JXPrgEi/cfQFja2PRll9dc5U76+8w5/wcLN0s0Wq1GJgaYO1ljbpRTa83euHS1YXa8loMzQ1RKBQoFApBM8velNriWjx6e9Dj1R6yMY34eQQlKSUUTd9BqUtHtA5BqNXCvJamlbJz9E7ajm7LjJMzsG1rK7HZCpXwPS33UaVSKHSL3xPPNx7fYO1ljbWntUDdeSWDnBs5dJnbBXWTGhRCMlzfnvvM4WeoKa7hcdxj0da8lPASSqVSTBwFzQzCIcABhUJB+qV0cm7m4D/SHysPKzrP7kybTm0wtTYVz8oxv8VgZGmEiY0JQbOCZLbg7PKzXPvmGovTFovdSbox6daeZ19P7u+4j/dAbwyMDUSfzinYicexjzm66CgFcQWELQ0TbODfzw8tz0+2bW3F11r6e9tHbyf7ajZLi5eiVCpRGaiozK0k/VI620dtx9jamGHfDiPszTA0Wg311fWk/JyCulGNtac1M47PwMbXRrxvtVpNzvUcTr12itybuUw/Oh3HIEcGfzFY7BA1sjIS/xYE2tD50fOx8bGR2JrK3EoiP4skeEEwhpaGdJzakcQDiSQeTKTfh/0wMDbA2FrY07MiBT2y9uPa827Tu5SklvBz+58l96XDsO+GYWxlzOF5h7FrZ0f7Ce0FjfHhfigNlewcu5OxG8Yyfut4wR55WPJK0isUJhRiYmeCQqmgqqgKExsTYR8CilOK0TRpSD6ezP3d91GqlLQd1RYLVwumHhCYbrq/1B2VSkXs5lhu/36b0b+NFveL1nzYtNNp3PhB0ErzG+HHsRePMeKHEXiEeWDtbc3y8uUUPyzm8kcCbeHsM7Nx7OQoGe+ttbc4u+ws4cvCadOlOTnlN8xP9Pda/r1u7T176Vk0TRpMbU05svAIrl1dCX1Zeoasr6qnKr9K5teBUBDVckxPrj2tVovPEB9xnT/pAwVMDuC9Ke+RF5NHQUIB9u2bi0SUSiVzLsyRxKRAoL/2GeQjGdPmQZupyqvitczX9Prfp5cKupftJwgxg3bj2uHQwYHs69mY2JrIzpCFCYWcfvM0o38fjX2AvWxMqadSKUsvwz3MHXt/ewzMpKFLhUJBeXo5v3X/jR6v9hA7oZRKJQYmBigMFYQ8F0LIc0I3t+73M3UwJWJFBKknU1nfaz1jfh9DyIIQcUzVxdXsmbyH0FdCcQ52ltz34QWHubdJKBhYXrEcQzNDsm9mc3jBYSZsncCRhUcAqKuo0ztPtaW11BbVYuVhhbG5sWyuG2saqSmuoa68TrLPhb4UimOAIydfPSl0Fr/UndBXQ8UxDft+GFc+uYKJlQnqRjXRX0ZjamuKe7g7Pd/oSZe5XTB3Nker1Yp7bstkuUqloji1WOxgBbBta0u/9/uReDCR2E2xvJL0imSNDfh4AIaWhmi0wu+m0Dz9nKvRaLCxsRF/53/XmXDMM8+wZfVqZv4JKs8NKiP6hY8Qx9TyOfMZ7ENdWR1fOnxJyHMhjForJG/VjWoMjAz0jrWxupH43QIttoG5gWTP1d17RqSQMB30+SAqMptjwi1x44cbmLcxJ/AZaRJyXuQ8NvTewOVVl4lY0ayt2NLuWftYi9+lVCrpPKczFTkV9HytJ6utV+PSzYXQl0Np07mN+L3Xv7uOjY8NY9eNldxL2vk0DM0MmXNxDomHEtkyZAtdF3WlLKOMkPkh+A7ylc1HcWIxV1ZdIWh2kERDVjdPqWdT2TlmJ6N/H03wvGBUKpVIRz373GzJ2UStVvO57ec01TXh2MmR/n/rT8dJHdFoNOyftR+3Xm50mdsFr35epJ1JIy8mT3y/z2CBtaz/B/1pbGgk43IGLt1dUKvVXP3iKrm3cgmYHMCknZOoKaqh52s9sfO1oyC+gF86/QLAmwVvYmpnKlt7p988TWNNIy/Gvci5d84R82sMbUe1xT7Anntb79FYLdAg67SE6yrrMHMyEz9Dq9XS641eBE4P1Lv2KvMqeXj0IYNWD6Lt6LYyP/7KqisiJfPIX0aiVqtRKpVEro6kprCGZ448w1slb7G281pSTqYwZfcU6mvqBdvyd9vk0s0Fl24uoo2vyKkg6XASZnZmdJre6f9EDuD/UtzonxmTQqGQ2Lx/NKZ/Nd3nf1SyLy4ujrCwMOrq6rCwsODAgQN07NiR2NhYjIyMsLGxkfy9k5MT+flCZVZ+fr4k0ae7rrv2tL+pqKigtraW0tJS1Gq13r9JTEx86r1/9tlnfPjhh7LX4+PjsbAQsvF2dnZ4enqSnZ1NSUkzj7azszPOzs6kp6dTWdmsZePh4YG9vT3JycnUtdjMfH19sbKyIiEhQbKI2rdvj5GREXFxUu2OoKAgGhoaePjwofiaSqUiKCiIyspK0tKaNYlMTEzo0KEDpaWlZGU1C6BbWlri5+dHQUGB+Hv+q8fk6elJXFwcarUau0l2jJ00FktfIdEXFxdH7rlc4rbEYRRghEdvDxrqG0hK/juvvR1U11XjgMNTx3Tpi0tUZ1cz/Mxwai2FyhDdmIpuFRG1MAoTBxM6TuhIl3e78DjjMSeHnMTSx5KA2QFYOFuwvv96GusaGXxwsGRMig4KSopLuLPsDtYdrOk5rac4T0mbk7j/5X36bOhD77/1ls1T1JsCHW3XF7tKXlepVHTq1Il+n/Xj5g83MQ0zxdLXEhMTE2Ycn8Gxl4+RsDuBhN0JuI9yp793f9k8ec7wJGV9Cp8Yf8KYY2PQOjQbGXWVGmNrY7JTsqm4UIGJg4lsnrIuZVFXWofGUkNlZaXetddzec+/vPZ8Rvlg6GMovqflPOVV5OHcz5la21pyHufgZ+FHdlo2e4fvxWWACwEvBxDkEYT3QG8abRol36tbe1UmVTS1ayIlJwVy/vzzVF1fTbuF7XAIchDXng7VWdWknkol9VQqlZaVWLW1QqvVEvt+LJ1GdKL9rPZPfZ6if46moaYBw06GkrWnQ/G5YiKXRjL4j8GYhzRXIDo7O+PVx4vxl8ZTXlJOYkoiCqXiX2Yjur7QlYT9CZz+4jTtFrSTzFNZURkgJDrj4uL+so2w9LSkMrOS+FPxFJs0axE6OztTeaeSk2+dxGWgC5Wmwue3HJPrAiFpEBcX95fH1GVuFxoaGrgf36wxoFKp8Hfzx3uINxpHDTfO3sDUyfSpY7LHXjamxtRGjk8+jls/N7p/20zz9M/YPaVSKdo9EHQdix4UUfSgiOszrmPl15yA+Hfa8obkBoouF+Ex2QO1dfPv23JMbfq2wcjLSDZP/879KeVhCrEfxeI92RvXnq5/aUxGnkZM3D4RlZeK3IZccuNyJWN6cOsB2oJyLAvTyLbzp7TUShxTaWYpZQll2HS0IbBXIEqD5nnyX+5PUU4Rj28+Fm1EvVE9vX7oRZNPExqNhoaGBuo862gzqw2puamoHv/5eWr/Xnu6fd4Nf39/8vPzJb+Zbs+9feg2j1Me4zbEjcr0Ss5PaNbZ8JrohfcUgXKltLRUHFNlcSXHpx3HY4wHWUeyCJwaSPt3hYBNnYUwh7Yltig0Ch7cf0ClZaVYdRoUFISVvxV56jzxfhK3N/tFX9p/iddEL9rObYvaS5j3fXP2UaetI2hpEHFbhfd85/0dA34egMqvubLQzs6ORbcWceH7C5xecppSTSmuA13FecrMyqSyshLHZxxxxFEyJt3aK4opInNvJgV3C3gQ9wAD62b395lTz2BkZMTxz4/jb+JPXFwcSgMlQUFB2AbY4vO6D2WUUXS9CNTQvU/35nlqA+4j3NHy9+B6K2svPz+fR3GPJGPy9PTk4c2H7O29F6WhkjE3xkiep8zoTGrqa6izrqNdRDt8u/vKnierdlZ0mdOFnLIcSuKafaymGKEK1nG4o2R9BAUFYd3OGqcZThx88SCug1zpsKADQUFBTL84ndjTsWwatIm6wjpGnxuN30A/iouLJWNSVik50OcAbn3d6P5ds337M/6e7xu+uM11ozqrGqtOgu1qOSbXia4oshTsH7ufLu90wXuKNwB1znUM+WYIPgN82LVoF+l70hm4byAmjiZ423tzZ8MdTENMcR7oTL46n8KEQsnzVKooxcDcgOzobAriC8AesrKyyEnJoSi1iKzsLNp2aEt+fj5pcWl/aUyVDZXQDsbcHIOnl6dsTKtXrEaj1KA10pKUlvSn7J5VqRXXvr5GtUE15dblpGxOIfNwJgsvL0Rpo8Rvph/lynI2jthI0c0ivPp56V17bQLbcPajs5gHm2MXZPenx1SSW4J1kDWWwZaMeXMMPqE+kjEVxxZjpbGi+7zulJiUkJqXCn+XImkf3p6lhUs59ukxkj5MwmaQDcXGxaItv3XqFmUJZfT+ozfmfubUV9Szc/JOLLtZ4jnWk0t/u0TGnQzm9J5DaWkpd4/cpSavBoceDpyaK1AmDds0DJPOJuLvphtTwJIAHKc5klGSgaJMIRlTRUUFjZWNqN3UhL4ZirG1sTimiuQKYj+ORd2gpvPszlzbcQ3rjtbYdLQB4Fj4MZpqm2j/fHvaPdeOwUcGY2hhiEajofffeuM43ZHkzGRUnVT0/Kon1p7WElteUFxAaUopW4ZsoefKnhgGGYIRNDY0cn/ffRbHLWbXrF0cmHWAEZdGYGRlhJ2dHVRAwPwArHtaY+ZqRrFJMYYFhjI/oqqyCuuCFGqtnCiqqCAuLh2A6uxqytRlfJPxDQvNF1Kd2azv0759eyrTK4k5GUPMOzF4TfIiaGmQOE/3jwq+kftEd9rPExI4Ud9GcfObm2jaaojbH0dTXRPFD4vJScjBuKuxWHWvs3s1yhqqnaqprqomOy5bnKeRe0dSVVdFXlMeeXF5ODs7E/tHLHc33yXkwxBuf3AbpZ+SUL9QVAYqUtJTaGhsYMD+AShUCjqEdMDKyopz685RW1SL+3ChA94+yJ7A6YGkZKZgVG5E2q40TOxNGPb6MNGPUHVXMebmGFQqFZaullRUVJCWlkbPdT1J2pBE2lYhIZR4JJEBB5q7z1v6EVmpWaAFQwtDyfNk0sEEs1wzPjX+lG5LuzF6zWi853sT+WkkeTHCA5J8PhnnIGeSEpOoyK8gYV0CZu5mzDw7E/u29hJ/D+DBFw/IvSn4Jhe/vkiPr3pgPsQcW39baqpquHX8FkY2Rpg6mYq+kXVHa1LTmp9JE1MTZp6YSX56PhklGWRVZkEHCFkewp3P71CaW4rPC0KiKC4ujjPPnMHWy5ahu4dSUlKCul6N5zhPtJXCPpdwLYH9vfdjYGHAqCujsPewZ9rBaWQ/ziYuLg7HGcJebKuxZdzGcTS4N2Bkb0ROTQ45cTm0b98e8xJz4qLiyL+QT9bRLEZFjSKkZwgNDQ3UutXiP8eflE0pHFt8jLqCOhYlL+L++ftoNVocQx0xVBoS0CmAkpwSChKEfd263vqp8YiG6gYMzAwwdDMk/nw85XnlFBUU4YGH+DyVJZShMlURtjIM38G+JCYmUpcj2L2GigZRz+zaN9fQ+mhp06uNXlt+b/U90vel83r26xhYGZCULsQjKi5VcHfDXerL6unwbAepvxdgQr8P+hH5VSQlRiWYOJpI1l5+Xj7p99IxMDXAyMZIry2/teIWBhgw9+RcvWeNP8L/wLmvM6FfNTMN6Pzyq/uvkrItBa/xXtgF2+ndn6x7WuNl4kXa5TTOfnAWv1l+OHR3ENeecy9nfCp9MOolvM9tlhuDOgwS/QjdZ+nGZOxnTOiaUJKuJ1FlWYVPmI9kTJHvRlJ8Rzgbzj47G623VjKmkuMlWNlbYdXeigpNhfj5vr6+aEo03Lt2Dwt/C7FT98kxVVNN4OuBuHR3oa6uTjxrqOvUhH4eSvCgYNGWn5t4DhN7E2za2QAQ8WEEJz47gW1nW0wcTBiwfgDe/b0ZuX2kYPsyqzn781kCxgVIxpS6NZX7Xwn2dfSvo7EZZCMZ05VnrqBUKBl2bBg5dcIzoxtT6MuhYtFevVU9SelJ4pi0gVp67+jN5WcvU19Sz/N3n8fITjp/j1MeU3OxhpOvnqT3+t7YdxWSviYmJjgoHPipg6D3Ne7OOMk8ha8OJ25nHJe2X8IuyE5ce/aD7SnIKeDCxguYOpvi183vqefc1NRU6urqKCsrk6y9f3XMMjwiglWNjVwH5AI0zbgOXFY3ErxewUnnk4z8ZKTMN3KwcWDQ6kFoHbXExcVRcq+EK3OuMOjHQfR+ubfML3dzcCNiRQRNDk2kF6VDUfOY6grrWNtlLW7D3BhyfAhZh7NIXJuI7yhf6lyaP6Mur47Tb5zGpacLmsDmJIGJiQmG1UKMKHZXLKZ9TTG0NMTS0hJfX19Ovn0ShbMC+672aNVaXPxd8PT0xLavLRGdhMSg51hPPHp4MPidwaSmpnLjzA3uf3Ufux52ZB7K5Pji43jO86S6vBojKyPOzDxD7eNaBn02iEqDSsx8zMjPySd1RypuYW54D/Dm4IqDlNwtofvq7iiUCiyLLbm/4z4KTwVF2iKyT2Tj3NuZvtP6UllZSUFDgeDHGpeTnJxMhw4d6Pd9P3wf+VKgLKAgrkBiy3W60oX3C7m9/TYdJ3UkOzub+B3x4no1czZj8EBhTHFxcRTeLKQkuYSIN4Vxp6al4vGiwIwQFxdHvzX9iP0xFpv+NpJ1ZoUVZell4v/Hx8VjbGcsW3s+C32oL6hnjeMaGiobsA6wptq5mri4OHr92As/Pz++dv0agDYRbbgXdQ+Xahc6du3YvD+ZQe6jXCp2VBA2M0yy9opuFRH1ZhQjfhhBlUkVeXHNen8eHh4M/nwwyfeSybuWx5n3z1DrXot/O39cu7mScjKFnWN20u65dpSkluDZzxONRkN6UToDDg0Qf4Mnx5TwYwLJ6wV2Ba/RXv9ncgD/6XmNuro64uObdbGfNqYnk5H/Uyi0/1tqgf8EGhoayMzMpLy8nL1797Ju3TouXbpEbGws8+bNk3TOAfTo0YMBAwbw+eefs2jRIjIyMjh1qplbt6amBnNzc44fP86IESNo164d8+bNk3TuHT9+nFGjRlFTU0NpaSlubm5cvXqVsLDm9uJly5Zx6dIlrl+/TmvQ19nn4eFBSUkJVlZCwOO/GfCnd/bl5OTg4uLC0YVHybuVx/N3n5fcu7pBzZHnjnB/u+BA6brgdN+rUCjEMWVfFzjNdR0Gunt/HPeYptom2etarZay9DLubblH9JfRGJgYsLRgKQX3C/i1y6/Y+tniGOhIxLII8u/mY2BiQJc5XWRjqsip4Huv7wldHMqI70aI85R0JIl7m+8x4JMBOAY4yuYj7WwaDeUNdJzSUe98xPwew40fbjBx+0QcAhzE12vLa6kuqCZ+Zzzu4e549/fWO0/xu+JJ2JXA0G+HSqphdfe+NmQtDZUNvPzwZdmY8mPziVwVSddFXfEf5i9be8lHkzE0N9Rb+RP5aST3t9+nx5IeuIe5ixQeunk6vvg4tSW1Iq1Iy3tvqmsSaEcczLB0sqS+qp7fu/9Oh4kdGPDxgKeuvR/8fqBN5zZM3T9V8rpSqaQ8t5y8m3kUJhSi1Wrp83YfyZiSjyVTGF9I2JthqAxVsjEl7kskfnc8gz4fhI23DZomDZ+afIqFswWL7izCzFHawt9yTDk3cqgtrsV/hL9snsozy7n0wSUK7hcwau0oXLo1t/P/T21E/O54Dsw4QNvRbZl+eLpsTI3VjaztvJbAqYEM/KyZhkqlUqFRayh6WITSUImtr61sTK2NteVnl6eXY+5sLtE2UCgU3Pr5FidePSFoTPTz0jume1vu8WDvA0b/NhorV6s/bffQQs71HExsTSQVrrq190fvP8i+ms3yyuUYmhpK7r0yr5KbP97EtbsrgZMDZWPaPmI7aWfSCFsaxqDVzXSsf7WzLzc3Fw8PD8ln3/j+BqffECpnl2QtkdBp/Dtt+a2fb3Hy1ZPY+tryclKzrpzu3ndP2Y25kznDvxv+1LH+q/en3Jhc1oWuw7W7K/Ovzf+X7k/f+nxLfpkZ9yMWAnD4sFYcU8KeBPZP38/IX0biP8xfoMRRCjoNWo0WAxMD6krryL2Zi317e4mWhb4xnXjlBAbGBgz7Zpj4+sUPLlKaVsrk7ZP/9Jhyb+ZycPZBUSx8ZeNKqguq+dbtW/Fv3m16VzZWtVqNulFN4v5EbLxteLD3Ac4hznSa0Un827N7zvLmvjexNrNm6OahTNg2QdSCVSqVaDVaHt9/TPXjarwHelORXcEPPj9I7ln3HpVKxY8dfsTE2oR5V+fxqemnqAxVOHV2YvCawRL6Ut09VuZXCrSUAQ6CjtXfX68uqaYkqQRrb2vMHc3F1x/ffyzYFVtTvnb+GsdAR+ZenPtPr73dE3eTfCyZ95ve/1PPU15MHn+E/0GPV3oQ9laYhGZFoVCQHZ1NRXYF+57ZJ87Lk89NdnQ2G/tsZMiXQwh/M1x8vepxFRWZFbQJbIOxpbwyfNvwbeTezOWt4rdkY2qqb6Kuoo51oesImhnEgI8HiM/T+ffOE7kqkrClYfRe2RtjM2MUBgrJGlsbtJaiB0UM/mIwvd5opr5VKBQUPSgiYV+CQOE21BePcI8/bfc0Gg05OTnYm9qTcy0Hh0AHkVrm0flHbBu6jZE/jUSr1ZJ4IJHxW8dj7mhO0YMi1gatpc97fej9tqCnqDJSyexeTXENZWllOLR3wNjaWKiwL6mlIruC+B3xmDmY0XZ0W8l+oFAoaKpt4syyMzTWNhIwIQD/kf5/yZaP2zWOM/fPEGEUweFXDmNqK+xzaWfTqC+tJ3BaIBmRGQId8Bu98OztiaZBQ1V+FUbWRphYmxD9ZTR3N91l9unZIpvFGvs11JfX4z3Qmznn5ui1EdlR2Wzst5Fh3w0j9OVQ8R7RwOVVlymIK6C6oJpRv47Cob3Dnx7Tvmf28WDvA97XvP+nbbbu9cjPIzn/9nmxm7+upI7vvL8jeEEw/T/qz90Nd1E3qnkc+5jQV0K58cMN4nfGs6xsGY/OCVrKbYLatGrLqwqqKEkqwb6DPeYO5s32rUHNZ2afEfhMIBO3TZSM6eqaq5x/+zxdn+tKU30T97bco/9H/en9jrCevvP8jsrcSmadmYX3AO9/OFZ9e25WVBa7J+xm0OpBBM8PBiB2Y6xwXpjZhTsb7pB+KZ3h3w3HyEJgZrj04SUuf3SZ5+89L1IyPjkf1YXVaBo1zHrRGoVCgb29hvXrm38bg0+EwoZ+Xv04N/uc5N73zdhH/M54vAd6035ce0JfDkWpVBK7IZYji44w8peRdJ7VWXyeStNLKc8qxznEmWtfX6OhsoGKrArid8XzTv07EirhkqQSGqobcA5pplJNPpqMpaslLt1c0Gq13PpZ6JKadWYWJtYmlGeVk3Exg6zoLPp90I+c6zlcePcCcyPn4t7LXXLvCoWCLcO2kH01m2Vly/TOxxq7Nbj2cBX1pVqbp/2z9/NgzwOWlS1DZaTixvc3aKxtJGJ5BAefPUh5ZjlzLs4Rbfxv3X4Tz0RP+hEV2RWcXXaWgEkBdJraCbVazdahW0k/n87UA1OxcrNCXa/GvqPQpXR+9XlufHgDELrDHDo44NDRAd8hvlh7WpMdnU15ejkm9iZYuliK2opKpZLaklrWOKwhZGEIo34ROk4UKGiqa0JhoBATHLqx6vMX/gj7g97v9Kb/R/3F1+9vv4+xpTF27e04u+ws/sP96fZCN3Gs2TeyWd9zPaB/3wJhD7J0sWTC5gmS14seFBH1eZRY4DPkyyEUPihk2LfDMDY3FueppqiGkpQS6ivqaShvoN3Ydnxm/hkAYW+GEf1VNM7BzsyLmofKWEo79jR/ryChgF8ChY6Nmadn4jvIV/QjMi9n4hDggJ2/nd4xFdwv4Lfg3yTfpfOnqnKryLudh3M3ZyxdLIlZG0PqqVTGbxqPiY0J+XfzSdibQOSqSEztTHkj5w1UxvIz5OUPL3Pt22ssvL1QQpt7/ZvrVOZWcu2ba2KH0ZNr7we/HyjPKMd7gDdzzs+R3HtVfhVFCUWkX0jHuZszHcZ3EK8plUpSTqTw8MhDYtbGMHbDWDrP7szDgw+J3x3PgFUDJPeiVCpJPJjI3ql7GbdpnOgHtnbWaKoWitxaana1nKfMyEw29dvEkK+G0Ou1XpIxZV3NEuhtNVo6z+yMlaeUfvITw0/oML4Dk/dMlryuVCo5sugId9bdYf61+ZSklOA9wBsrF+H9BQkFHH3uKN1e7EbQzKBWbfm3Ht/iGOjIzJMzOTT3ECa2Jgz7ZhgqlYqcWzmsC13HgE8GiB1WLdfed97fYeVqxfzo+ZIxZUZmEr0mmqLEIoZ8MYR246R01zptx5bPJEBdaR3FicXc/OUmjdWNDFo9CGMbYyydhHNg1BdRnFsh2Phh3w0T9do0Gg11ZXU01jZi3sac9HPpxG6Mpe97fcWYEUDmpUw2D9qMfXt7XowXNAxu/36bq19cZfKuyazrsQ4rTysGrR5Ep2mdxDEVPSzil46/ELEigoGrBj7Vj2hsbCQnJwc3NzeUSuW/NWY5cdw4Lh07xin0J/yuA8OAiKEj6HG6J51nd2bC5gkS27HGfg1+Q/2YsnuKeI+Pzj1i27BtDFo9iN7Le/+lc27V4yr2TtuL/3B/6ivqqciuIDs6m0k7J0n2Sh0DwcxTM2Vagp8YfCL++4W4F5rnUAOfGH1Cx2kdKUkuoa6sjsXJi/9hHLYgroDfQn7D0tWSylwhUdFubDuSDifxSuor5FzLIfaPWB6dEwoWF9xYgHOwM5omDSpDFUqVkr3T9pKwJ4GVDSsFaRYUaJo0aBVaMi5msG3YNkb8NIIeL/V46vmpoebvmrp/15oUmdd23hf0p61NMLUzxcbTRhyTukHNpQ8vYWJjQmN1I/4j/HHr6caFdy8QtTqK5+89z4mXT+A90FuigaibpwcHHxC/O57eK3pjam+KuYPA7lZbVivop/5d91ff2rv6xVXOvyMU0y6MWSihhlapVER/G/3/sffW4VGda9v3bybu7p4QJwkRJEGS4O4U99JCoQLUaKm7A4VCS2lxd7fgmgAhJEgE4u7ume+PtWeRlZn02ft53me/3/6+nsexj11GVtbMrHXfl5zXeZJ/L5+6ojqennnKmK1j6Dazm+T3+NHuRyx9LZl7WaoOU19WT/7dfKz9rDF0kHr6tf/9fnL4idrCWlbUrUBLV4vfe/4uEoRAmG7sLAbKuJjB1c+v0u+jfrj0cyHzspCLeI3yImxh2P8regD/6X0NgKysLHHN+68+U1VVFaamplRWVoo9ov8J/qMm+7S1tenSRdClDg0NJS4ujtWrVzN58mSampqoqKiQTPcVFhZiayssnLa2tsTGxkqOV1hYKD6n/H/lY+1fY2xsjJ6ewJjT0NBQ+xrlMTqDjo4OOjqqxsLKY7aH8kJQ99p/9+PKBllHdHaO/+rj/+y5tLa2UlZWhoODA7rGujRVN5F5KROH7g7oGOsI36Oehtjos+5qjXMfZzQ0NMi+kc0fvf9g8I+DCV8ajkwm49j8YygUChY/el60lsvl2AXZUZBQwK0fbuH/gj+mrqbiuVt4WBD9cTSRKyNFjzEzNzOmHptK3C9xpBxNIeVoCtNOTsNzmCcdsc5rHdYB1sy/OR9DW0NxAdDQ0MB3rC++Y33JuZ3DjqE7CH8zXOJr4Dnk+fE6fjfNdc3YBNgw/eR0iQa7QqFAz0SPR3sfcenDS8w4M0N8b8ffI3BaIIHTAukMAdMCBL1sDdWEyiHUgckHJ6s8rsTp105TmVVJ98XdGb52uOR1dcV1lDwp4eSikwz4egB2QdIGVumTUmoKaiTHU5574eNCNnbfKAZbDeUN+L/gj+dwT3aN2IW+hb5Y0OkI2262ZN/M5gebH5h8cLLEX6PwXiF7xu4BQNtIm37v9ZN8psf7H/Ng2wN6vdELmbbq/eH/gj/+LzyXXJDL5YzfMZ6D0w+SeSmTrlO60hHKc7TyseLI3CPU5NUQ+lKo5NyLk4p5sO0BYzaPwbGHqo/Xvkn7SDudxouxL2LhZSEpuvxX91lhvLCmVaRXqL3ncx/k0vf9vkLRp+N6pSHH2k/VZ+GfXQs0jDXQDVTvP9FjSQ96LFFv6K6hocG5d85xe9VtkEFLXYvkM6l7fXts6LaBwoRCAqYHSLwk60rq+MH+BxRtCuxC7Wgsb0TXUFdy7jU5Ndz4RvCN8Vf4q3ymvu/3JfTlUPwmSH2jOjuXzh4vLy/H0dFR8vjZ5UKjb/i64Zg6mv7Tx5fJZLQ1tfH4wGMCpgWIDPx/9ncKmR9C7M+xatcBEAo4zTXNateIf/Yc/9XHZTIZNgE2vHj7RYwcjDpd35RQPp60J4mEzQm8cPAFilOL2T50O33f66tyrXWb3Y09h7TEtVouf75mO4U7MXbrWNJOpnFy0UleTXsVcw9zzrx+hoQtCYzeNBpDO0P2jN0jrlEd8WiP4Hk0Yv0IQc5TV1Ny/+XfyacgvkDlMxUmFpJ9Q5DZ6jK0CwZWz6d85RpytAy06PFqD3EPNLE3YfGTxVx4/4K4X9aX19Pa2Co2oJSxiHIfcI6Q+vFl38gma28W5ZblaBtoEzAtADNXM8nvsm3wNtIvCMngB80fYO5qLvjidLOhqbqJZ+eeceG9C9Tk1xCxPIIlT5aI7/2g6QO1v1n7v48M3KLcVJ4ruFPA9iHbcYpwwqGXA0N+GEJLQwsbAjYAgsfQoO8HiT43Ha+lmz/exMzDDJ8xPirHzo3NJe6XOLq/0p0uQ7uIPg7K3+nE4hNkXMhg8ePFKr+Tvrk+3qO8ub36NtnXs1kQt0By7HNvnaMiowKbIBvkmnLJeSn/29LbkrFbxoo+wOI+tPcxZ5aeYe61uTj3dlb5TKM3jqa5TvV+rCms4QfbHwCYd32exP8rbl0c+ub6TNo/CY9BHhyefZjko8kqXizB8wU5rIjlUr80EHxsLn90GUAoOvZ1lXym+1vu09rYStCsIGSaMpA//0wVGRUcGn6IiCURYpFKibxYIXFN3JnIvGvz6LH4+b1q7WeNlb8VVz+7ytXPruLU24l51+YB0hjWyNoII2sjyXEf73vMycUnxX+ff+c8H7Z9KJG2aqlv4c4vdwDIupKF9yhvyWe6+eNNrn5xlf5f9se5jzPW/taS731w82CK7xcTeSqSVOdUgucJ39/tH2+TfSObrlO6Ul9cT+qJVAKmBYjxbPviap93+kjWEA0NDQJnBmLmZkb4MoF82HHdu/DBBa5+fpUpR6eoXCPNjc1c/lj4nfSt9GlrahPf/8+swRHLI+g6uSvxf8Rz68dbTNwzUSRrVedXk30jG/swe7UehD5jfDBxNEHXRJfW+lYMrAx4r/Y9AK58foWLH1ykz3t9SNqVhM9YH8KXhuM30Q9dI138xvvRXN/MvY33sPKzUvGmqs6t5vaa29z8/iaTD00WC9oaGhqinJtjT0eVz1pwT1hn0y+kU/6sHP/J/gTNDBKfX5a7TPwbijYFldmVaBtoo2+pT+bVTCrSK4SCcbu4q2Ms5drPVeInmhuXS2F8Ib4TfNk5cic+Y30Yt1nq9es+0J3kw8lkxGSwMXQj/pP8RW8X5bGPzj5K9o1sZFEr/vF35bT/CWcEzuDJpidEfxZNtmu26P0IEDQzCIceDvR6vZfEC0fLQAv7UHscezhSGF9IxqUMwevJ1QwzV+G6tPa3pjq3Gr+JfvhN8kNTU1PFIyjjUgYyDRl93+9L9CfR7Ju4D88RnvR8rSfHFx7HuY8zbgPc0DPTw9rfGktvS3aN2AXA9kHb6bWsF0GzgzCyEfb34sfF3PzhJvGb4um+uDtRH0fRUN7Qadwx89xMtI20JfnW7Z9vc+3La/hP9id8eTgmTibYBdvRVNWElo4WMrlM4hckl8uRazxfn+VyOb7jfVViIeU1ZeZixqQ9z33J8u/k031Rd2aemYlcU87tn29z+rXTzLk8B8fejpj0eE6wVCqDAEw9NhVzN3Nc+rhAu/Ah41IGZ5efJfrzaFwjXYn+LBrbYFvxXFJPpbJz+E5G/T6KkPkhwnc5dDvPzj/jjYw3MLA2QENb2LtSjqbgPsida19e4+b3N3GJdGHm2ZkEzRTIqg/3PST1eCqpx1OpyqlC10SXPu/2wS7YjmW5y9A21Oa3br9hHWDN+O3jkSFDriHn3qZ7pJ9PF71T82Lz2Dt+L/2/6I+pq6nY6ANBOq69TyBAc00zpU9K0bfQx6X383s88uNILn98WfRQLLhfQFtTG9r60iZS+9+jI0xdTOm+uDtx6+JorGgUX1eVWcXecXuJ/jwaLT0tCuILSNyVyPjt48W8zdzdnOmnp6NnpsemiE14DPYQv/fMK5kcmnGIFw6+gOk4U3os7iHZp3YM3UFtoTBZG/15tCjB3/4aKkkuwbyLOUuSl6j4Yj3Y/oC6kjp6Le2FYy9Htdeefag9/pP9GfTNIJVjZ1zI4PCsw0w5MgXv0d50xJllZ0AB71a+i6aeJhoaGgJ5Z+8j+q7oq3KP+Y7z5YNm1bhNJpOxY8gO7ELsGPTtIGLei+HaV9cAGLN5jKqvl0LIfWdfmo15F3NJMRTAta+rJI7oiLlX56Jnrqd23/Id74u5hzm5N3M5s/QMM8/PxMReuN8ULQoqsyppqm5SG3sp4dbfDRNnEzQ0NBi/TeqrZeltycxzMzHvYq7ye9zfcp+AaQF4j/ZW+UxukW64RarGskoM+GIAhQ8KSdicgM8YH9Hna2v/rRQlFjHs52F4DvdkjccaIt6OEH9vZZMQ4NzycwRMCUBTVxMdYx0uf3yZuLVxLM9fjucwT7V1Kue+zrxd+jZa+lri59HS00LXVBdNHU3Gbh3LhfcucGjaIQImBwDwe4/fMXM3Y+iaoRK57s7iCLlcTkVFBU5OTn/5vf93Hm+/5xYUFHD69GkWAf2AKGAWYAMUAFuBy8AiYH3MWTbl/yHWctuvHS59XSREdIAug7uInqWdnUv6hXRur75Nvw/6ifE7gJGtEXMvz6UoqYj1AesJWxTGa2mvqbzfd5wvpi6mOIU7qRzfzMOMivQKRv0+ClMXU/F5hUzBy/dfRsdYh0f7HtFc1yx+luMvHSd+Uzxu/d2YFTOLiowKzr55lq5TuuI30Y8P2z6kIL6AHcN2UF9ej2NPR4wdjNEz0SNwWiDuA9zF3EHRokBTSxO0np/TxD0TYTeCtLcyZ/5HPOQU7sRLd18S7ZQ6qy3X5NWwfch2yp6WsbLhOSm1KruKvNg8fMb6YNfteZ1Q+dk2hGyg5EmJ+PjVz6/ykeIjer3ei4aKBi59cImipCKs/Kw4PPMwdqF2RCyPEJs1JY9LeLjrIRqaGjzY9kDMqzrmC0q0P/fqXKE5uiR5ieANXt2MrqkuFZlCPS38jXAOzTzE0zNP6bOij3j+ynM/MPUAdcV19D/QX/I4gKGVITZdbVjrs5YeS3ow8Gup1RgI0+a1hbWYupqirSfsg+O2jaO5tpknR54Q+3Msg74dJP4mLbUtPD70GJsAwXO2qaqJ4ofFtNQK8Yx7f3fc+7ur/awdv/f/6eP/f+lrtLa2ql3z/uoc/0/iP6rZ1xFtbW00NjYSGhqKlpYWMTExTJgwAYDk5GSysrLECbzw8HC++OILioqKsLYWFu1z585hbGyMn5+f+JqTJ09K/sa5c+fEY2hraxMaGkpMTAxjx44VzyEmJoYlS5bwN/49GLZmGMaOxmwbuI0XY1/EobvgU1NbVEuP13pg7W9N6Euh4ut1zXTxm+QnavkDIlu3I1qbBJ+h8+8IRvbqjHjbMyV1jHTQNtSmJr8G2262eI7wFHXOO8LSR2ALtmendkRDRQPZN7MJLFJtvP0a8iuW3pZM2DVB8nh5ejmbwjcR+VEkUR9HAUIx4lONT3Hu60zgzEB6vtGzU1PhXaN2IdeSqzTsQEg8Uk+k4jPGR/z+WhpaKLhfgJGDESZOQtC8sftGZBoyXrylapQ9Yv0Ido7YKW6I7TFszTCiP40m80qmhF0GcO6dcxg7GDPr/CyV9xXcL+DGdzfwHOGJU7hQrKzMrOTKZ1fQM9ejpaGFtNNpbOyxkf6f95fohQNMPjSZ5GPJ3PrxFlr6WpLnbIJsGLN5DMYOxmoNegNnBvLk8BMe7XtE0KwgyXONVY1kXMrA0sdS/L5kMhmOvRwZtnaYirluR7Q2tZJ6IhUTFxOV5xx7OjLt5DRsAm3UvBOMnY1pqmniF79feKvkLfQt9NW+Th36f9GfqE+iJObQ7fFgxwPubriL7zhfkeGkRFtLG89inqGhpYFb/86Tl85wfNFxjB2MCV8WrvJbAEIzePFJQhaEqDRKE7cnYhdqp2L8/M9AuZkGzpDea43VjbQ1t+E9xpsph6eoeytW/laELAihubZZ7fNKz9D/DYzfMV4wZB/a5V9+7/3N9zm5+CQWXhY49HBQeX7fpH08OvCI156+Jik0A2jpa/FqyqudHvuVpFeIeS+Gmz/eFIvP/w5o6miq/Sx/hYxLGaSdTuPiBxe5+YPgt6tjokrGifo4io33azHOT6PeyBowprWplfL0cnRNdAmaGYS+hT5GjkaiB5xLPxcStiRwdP5R+n3Qj7FbxmIXYkfamTT2jN3DyF9HiutGZVYlmVczaapp4pWHr6j8/RmnZ6g9/wvvXSDluCAJNe/6PEmzz6G7AwvvL1R5j6W3Jf0+6MeOoTv4TPszLDwtKEsrY2WjkFA1VjXytcnX+E7w5YX9L6i8v+RJCXmH8hhpN5Izb59h/E+qZuIuUS6UppQK+6tMWM+09LXIuJDBzHMzCV8WzoZuG6gvq1d5780fb5Ibm8vE3RNVngM4+qLg07UocZGggd+uqGzuac7AbwZy97e73F51myE/DBE9d5XoNrsbRUlFPDn8BI8hHpJ17Pw75/Ea6UVFRgVPzzxl0r5JaBsISVN5ejkJWxLwGulF90Xd6QgNbQ1KnpQQuy5WUtgDwSR98qHJXPniini89uj3QT+aaprwn+Sv8pyiTUHy0WSMHY1V9hkA5z7ODPxmIOVPy7m38R593u2Dpc/zPdTU1ZQ94/agY6LD2M1jxceVxvUAldmVOCHsnwqFglOvnsJvkp94/zr3c5Z48imhrsmnhOcwT16MfZHih8Vq96pbP92iMKGQ4y8fp+/7fen/+fNJcUWbgtYGYQKrI4oSiwBoa1aVN5HJZTj1dqL4YTGAyn4PwhTd3V/vEr48HLtQOzS0hASrtrgWmyAbBnwlFNeaappAgeT60TPXY3n+crKuZaFloLpHGVgbYBNkw4mFJ/Aa5cXUo1OffyaFgvI55UzrOg3399wlzO2oT6KoKxGK1r7jffmw5XlTtbmumZrCGgysDNA2lF472Teyufb1NXq/3RuHng482PEAM3czMRZSwszdDM/hnjh0d0DPXE/ynKaOJgsTFqJrqitRc1CisaqRq19eRdtQmyufXyHqkyhJs1EZx97bdE/0/1Ei704e+ybuY8BXA7APs8cmyEayRln5WiGTyVjnu46+K/vS/7Pn14BdqB1hi8IIXRBK1MdRyDXkyOQycY1vaWxhrddaqnKqCFsUptLsu/LZFe5tvEe/D/uJBTol5JpyRm8cTfyf8Wzpv4Uxf44Rm5GhL4cKPnV7BJmdoFlBncbMTbVNwiTi3G6M+WMMdzfcJXFnIl2ndOXmaqHxO+fyHMnfz7icQf69fEIXhIq/Z8mTEuLWxWHlb8Wz889UYuDS1FI2R24GBZxZKqjStPdzU8JnvA+WfpYk3UijSd8UmdXz49SV1GFxwgKDWgPkEXKJegKA53BPPIerFn27Tu6KTYANsetiqcqqIuV4Cp7DPdG30EfRpkCuJefexntkX89mRfUKtWMToQtDcejlQGlyKZbewjmN2TwGI3sjtvbfCsDgHwZLm+faGgz6fhCKVgUVmRWELgiVrGu/dvtVXB8q0itUrvn4P+PJvp7NsJ+HoaWnpTY2yLqaRU1BDbdX3yZgegAmTiZELI8gYnkEazzWUP6snBlnZojriFJZpD0iP4hU/cDAje9vUJ1XzZAfh4iPxa2L48G2ByyIW4ChnSEu/VwYunooZu5mtDYLvufdX+1O3M9xANiF2NH/y/7YBT/PFwofFFKaWorncE9am1tFso6Wvhb9VvaTnIOxg7BntM9FnSKceHrmKT85/cTL8S9j282WquwqqnKE/3kM9qAgoYCGCqm/lVt/N/p90A+5llxsJES8GYGGloaYHxnaGqJnoccq11UY2hqyIHYB9/+8D8Cj/Y/4yugrBn0/CDMPM3RMdHCNcmXKkSnc/OEmGtoa2HRT3SfSTqVxYKow8f5Bywdi0bjvir4EzwtGx1iHPu/1oSa/hqrcKtLOpGEfZk9rUysmTiZCXl5QQ2lKKdYB1uI0dUlyCbmxufRb2Y9eS3uReTmTiswKTF1MMXE2YczmMZh3MefPPn8C4DPWR5IH6hjpoG+hz+bIzQz+fjC93ng+2e4UIRDA7EOfF/bbo8+7fWiub8b/BX8aKxspTS3FwtNC8pr0C+mcfOUkM8/PVGn2zTw7k7aWNrV5qRIvHFCN3ZRw7OnI8F+Gi5OhHTF09VBkMpk4UQPQb2U/+r7fV/K6q19epfxZOaN/H632OA+2PyA9Jl3MsZSxKkDc2jiVZl/KsRT2jNvDuO3j/lu5k3NvZxoqG7jwwQWcwp0k65myqVX+rBwTZxNsAp5fa7ZBtizLXUb2zWx+C/uNfiv7SaYdK7Mr2dh9I2GLwoj6KEryN5OPJbN79G6MnYx5I+MNZHIZD/c+5PDsw4zfMZ4DUw/Q2tSK53BPsRH3ryL1ZCoxK2KwDbIVm312IXYUJRaRF5dH8pFkXKNdcYt+nncPXzecYwuO0WtZL+xD7fne5nt6LevFkB+G4BrpikwuUxvDKKGhpaESK3Sb3Y1us7uxdeBW8u/mM+3kNEk9RyaToWumq0LQ+r+N3zduJFpLi1WtrbwLbAJWAZWACTAWoeFnCzzS1OLXnzfw0Rcfqxxn6rGpNNc3s7HHRjxHeBL1URQNFQ3Ul9VjZG8kNu3bI/1iOlsHCHtcyEshas9PSYx06u2k9nnnPs7YBNmoLfyraw6CEAvbBgkxZu+3e0ueqymoAYSY99CsQ2gbavP4wGNxEk0mk3Fh5QUaqxpZ2bhS5e8a2hjywoEXeLT/EU/PPEXRJsR8Zm5mGNkbCa9v95bGqkYKEwsxdTXF2MFYpf5V+KCQCysvELYoTGw8H559mOJHxSrfWXVutUCavCHs7Va+VugY69Bc18xq99UiicJnrA9PDgv2EQqFAkNbQ5prm0k+ksx7te+hpa/Fp5qf0lzXTMTyCNZ6r6UsVVDBmXttLq2NreiaChP09eX1PDn8hIbyBoLnB0t83pXYELQBXVNdlmYvJfNqJgenHWTi3on4T/Jnx9AdyDRkDPlxCA+2P0DLQIvoT6MlNWQAsy5mdBnaRSWWVUJDRwOXvi6iclZ7bI7cTOYVwfvRa9Tz6eBN4ZuwC7ZjVswsoj+J5vaa2xTcLyB4bjA1BTUcmXOEPu/1wS7EDt/xvviO91X7t//G/zfwH9PsW7FiBcOGDcPZ2Znq6mp27tzJpUuXOHPmDCYmJsyfP59ly5Zhbm6OsbExr776KuHh4fTqJQRigwcPxs/Pj5kzZ/Ltt99SUFDAypUrWbx4sThxt3DhQtauXcvbb7/NvHnzuHDhAnv37uXEiRPieSxbtozZs2cTFhZGjx49WLVqFbW1tcydO/f/yvfy/1e4D3JnhPEICWv47m93iV0Ty8vxL0tea+VrxaS9kySPBU5XP8X2R+8/KHpYxIu3XxSlnCoyK7jx3Q0MbQ15evYpniM8ce7tjHMfYfLh9OunKXxQiLGTMf0/78+hWYeoLapVKdROOzENhUJBc32zwBLVft7Nf3ruKclHkun9dm9WVK2gI3aP2U1BfIHaxNXQxpDAmYGUPC6hKrcKYwfBJ85voh+P9j8i62oWck05dUV1IiO4PVqbW8m4lMHppacJXxYuNvBAaKqdXX4WYydjsXlVlVPFpnBBvuUjxUfkxuaSdyev0yam53BPCQOqI3RNddWyDPPi8ih/Vq72PaUppTzc+5CJeyaKU3l2IXb0fL0neuZ6zLk0h7TTaRx76RjN9eobMt6jvCVFBiVMnExUmYftYGRvhIa2BrXFtSrPVWRWsHvMbgChYfKPZL62uBavkV6Yupjy6MAj3Pq7iclnexyZcwRDO0OGrR6m8pyBtQFGdkZc/eIq3eZ2E5vcSgz5YQiuUa7kxeWpDT7/ChpaGmLxUx16LOlBY2Uj+fH5kolTEALHHUN3APzl79wZ7m64CwiBWccCSmVWJSknUki/mI7XaC+V977y8BWVov4/i47rhBJiMVkhSPZ0LBiCkPCP+m3UXx7/6ldCofT/dAIUMDWA+D/j+bPfn4zdMlalKfdXaKpuQq4pVykeK/Fo/yNAaAh0PG5daR11xXWYuJioNHyViP8jHgtPi39rsy/7RjZn3zxLnxV91N7P9eX15N/Lx33Ac6baiHUjGLl+JM/OPyPnVg79v+jfaZFBtzwP59idpAeOBoIpf1YuFKn/0ajoWCwNnheMcx9n1nqv5cpnV8R7oqWhBbcBbhjaPZfi6PNuH/q8q5548lcIXRiKhY8FLn1dJIXQzlDypIRzb52jraVNTPi8RnuJHgiAeB89PvCYc2+foyy1DNtgWyI/FO7JoNlBHJ1/FPt8ezIrM0koSCDIVtqEivwgUnIPn195nvub7wMCa3zw94NZlvN8QibjkuA949jTkatfXKW+rB7rAGtc+rmoJD5Rn0TxeP9jPtP8jPE7xxMwNYDsm9kk7UoibGEYvd/uTfC8YFoahc+kqaOJcx9nsq5lieSQgvuCl9vS7KVoOT6/huddn4eOiQ6319wm80qmpKHkN8EPrxovyX7dHkN/GkrOjRxufHtDpdmnRL/3pcXYpN1JuPV3w3OYJ/Xl9azzW4ffJD+iP3nuD9XW0saecXvoOrUrE3ZO6HhI7MPssQ+zJ+6XOBK2JBA0O0hyLdSX11P8qFgiHwtg4WXB8vzlyDXlpBxPIeNShjjpM/vSbAysDIh5P4bE7Ym8HP+yZMKlPU69fgp9S32VNVvPXA8HcweV/UmJ8dvHk30zm+MvHRem4do1+8zczRh4ZCAWDRZ8bfI10Z9Hi+vn+O3jGbt5LBo6Gjw995SMSxmELw1H31KfptomQl4MIerjKJUCqRKlqaU82v+IR/sfMW77ODEGTD2eSkN5Q6dMdxAKKIa2huKUSkfURdXxUclHRDlFMWWiKkGky7tdUDgp6Dm7J9YGz/eTvyJ+ZVzOYOfwnYzeNJrgecHkxuWSG5tLwNQAqvOreXrmKUGzgkg7lcahGYfoNqebSuMjeG4wgTMCaahooKm2SdJwlsllnRKHQCC9XP/mOt6jvfEY5NHpPhMyP0ScHlLCLsSO8TvHU5pcyrZB25h6bCpeI6X7t66ZLmGvhOHQw4Hm+mbSY9KFhmWEE0fnHUXLQIvB3w2WvKeutI4fHX6ktbEVm0Aber6murf6jPXBxEVo3nQWB9UW1lL4oFCy/jmFO1EQX0BFegWjN41WaYA9PvRYmDLq54KGtga9lvUS4/HwN8PxnyL4tiobvx3/9pNDT7i9+jZyDTkegwWpRv9J/iRuTyRpZxIrG1aSuCuRQ7MOMXTVUPTM9YT9WAHdl3RHQ0sD/8n+OPZUvWbCXg6jKqeK26t+otCtJw1uQ8XnqnKrMNtkhiJageIzhaTYrURLYwsXP7woyJK3Ix4ce+kY2dez8Zvkx8vxL2PhZcG5t89x66dbvJH5BsPWDKO+XJW4UZZWhoG1AV0nd6XrZClBS3nf9Xi1BzK5TLJnH5xxkId7HzLj9Awuf3qZyA8jVfa3vu/3xczDrNMcLvNyJglbEkQ58dYmoZnWnsg2YecEhq8dTktDi0RaGYRGLgg5l884H7Vr718h5XgKmZczaa5rxmOIB77jfOmxpAeOvRzZ2H0jAOO2jxOv3ZK0Es6PPI/XaC+mHpuKU4QTuqa6EjLL/in7xSa0kozz+rPXOz0Hm0Abxm4ZK3ks8sNIHMMdST2RKjYOph6dyieyTwBBFrM9SWJTxCZMXU2ZsHMC0Z9G01DZIDb72lrbaGls4cL7FzB2MmbmOUEeDMVz0tT4HePZOWInxQ+LaappwqG7A2EvP/d39R7tLeZ9CVsTODDtACN+GSFO31v5W4mv3TpgKyVPSph7dS6VmZUUJhbS6/Ve6JroYultyc0fb3J2+VlCFoRwb+M9ph6fitcIL5KPJXP8peOSxu2z8884teQUsy7Mojq3mqPzjzL50GRMXUzRt9Cn2+xuFD8SSCP+k/3Vko8OzjhIc10zZ5aeQddMV8wXzdzMJOvkkyNPyLiUQeSHkeiZ6Ukag1/of4FbtBvTTkyTHNtzuCeDvh9EZWYljdWN6Bg9b7wZWBvwzyDleAoNlQ0q94iFlwUWXhZsjtqMlb8VI9aNkDzfZYggK5h/Lx8TZxPxOulY9M+4mEFubC6RH0VS8rgEu1A7Cbm06GERMrmM4b8ISj5KRYOSJyVq80xjR2Oh6e5sgqJNIbn2AbZEbyHjUgZOEU4M+m6QRI1AoVDQ1ixIVF79/Co93+iplrxg5m6mtlgOQs5XX1ovjYcRyCEWnhbi9wBw+dPLYi0IhEa3MnY2sDHAbYAbBjYGeAzxoLmumahPotT+zYd7H3L508tY+VnRY0kPiboQwNm3zlKdW830U9MlMdzYzWMZumoo35h9AwhxcXvip884H8w8zLD0tkRLX4uwRWHi9+U30U+MYTIuZZC0J4nwZeGShnNjVSN7xu2hrbWNOZfmSM7Jrb8bJi4mKnFGR8WK/7fgyM6dvPEP3y9b4P1//E8dZjc28Pkvm3hpwstqSdkymYyaghoaKwU7pvub73Nm6RlRMawjaotq0TLQYuLuiXiNkMY+dSV1xK6NRVNXk4i3I1C0KfjW8lv6f9FfskYCrPNZh4mLyT9NaFa0KWioaEDLQEuFvD3tuLDWtDS08IXeFwRMDxDz04bKBvLv5QufXSHYB7S/z34N/hUrfytGbhhJa3MrB6YcoLaoljvr7zB09VB6vtaToqQi6svrxdyt6GERf/b5k8E/DKbHqz1ormsWJkb/sQbUl9Xz9MxTSWzYdVpXXKJcVPIK5WRkXlwem3ptwm+iH5P2TaKlsUVs9Cmbee3RWN3IoG8HMeq3UaLU88rGleIa4z7IXWz23fnlDuN3jBdJ6w+2P+DoPIFk2trUqjZXN+9iLhA+k0vQNdEl8uNIUW49ZEEIyGD7kO2AcJ90bPQBErJbRzw7/4wdw3YwYv0IQl5UbRrbBNnQ0tiCsYMxuqa6tDa3ikSc9AvpfCL7hA/bPuTGdzcwcjAieG4wRvZGTD0+FXMP9YMpV7+6yoX3LqClryUqbvyN/2z8xzT7ioqKmDVrFvn5+ZiYmBAYGMiZM2cYNEhgzPz000/I5XImTJhAY2MjQ4YM4ZdffhHfr6GhwfHjx1m0aBHh4eEYGBgwe/ZsPv30U/E1bm5unDhxgqVLl7J69WocHR35/fffGTLkOUNv8uTJFBcX8+GHH1JQUEC3bt04ffo0NjadJ81/438OmUyGra0tMpmMpD1J1JXUqRTWnPs44xThxI3vb9DW0oZ9d/u/ZKCrQ5dhXXDq4yRpqjXVNBG3Lg6Xfi6UJpeSdVWQW3u//n1ybudQ+KAQS19LRm0cJYxw1zV3OvHTUNHAt+bfErIgRNIsyL+bT9y6OLrN7aaWZW1gY4DXKC9Gbhip8py+pT723e05/dppui/ujrGDMXINOZP2TeLp2afkxuaScjwFI0f1RbAZp2dw/dvrnH/nPAHTAiTNPvcB7sy7MY/7m++TejyVsVvGisxgJRtImRj1XdlX9eD/BdIvpFNXWqd2smH2hdlk38gm+ViyShHfa6QXS7OXShhoWvpa3P31Lm4D3AiaFUSXoV1UpGGUuPTJJUxdTf+yqddY1UhLY4uEkQ5C0vB6+utqGybGjgJ7Kf9ePtX51Zi5m9FQ3sCmXpsIWxSG3yQ/9k3cJ5EZaw+bIBsJq7IjKjIruLP+Do7hjmqLqZ01MP8rlKeXc//P+1h3tZZIkCph6W1J0q4kFG0KlWaftoE2Ft4WKmzAfxaLEhdx9YuragPsR/sfce7Ncypyc0romupS9rRMCEx7O0kS4/8uTN1Mee3Za6xxX0Py0WQWP1ksstKVULQpaG1uRa4pl8h2KbF77G6SjyRj393+v9Xsa2lsIf1COnLkyAJVu5n1pfWUppTS2qg6AfNX6P12byLe6nxNfOneSzTVSD12lLi/+T7n3jxH9GfRKkxygOSjyQxfO1zFW+B/GzUFNeTczGHvhL1qpSCPv3ycR/sesSRliZjUioH+QHfcB7qrvEeJI/OPYJLVRnrgaGrMBXLHvd/vAUIj+iennxj1+yi6DJFOWVp4WTBp3ySxsQZCsqJMttQh80omGtoakgZA3t08GisbVSZmvUZ4qSSQhQ8Kub/lPg7dHagpqCH5aDJFSUW8kfkG9WX1IrtaU0+T5fnLVViKOkY69Frai1s/3cLAxoB7v9+TFF3kGnLsxtpxPuU81oXW3H7jNmavmuHcWyr32R7t7+lbP93i1k+3JBJSB2ccxMTJhPk359Pa3IqBjQEXV14k6tMolWaf/yR/TF1Mkcll4h5ZEF9A7M+x+Iz1wcrPSlKUAbDpZkN+fD4F9wtI3CnIho3bNg5dM+lnV+73I9aNUCmAyTXlaGsKa/3tNQKzdMLOCZLvZuyWsbS1qE6clT8r59bqW/iO8xUbasWPizkw9QBuA9yYdX4Wbc1tlDwu4cqnVyTNPpmGTJBiqWvmJ+efiHgzQm1zI+TFEIJmB6k0F3aP2U19eb3KNSfXlFNwvwAdEx2OzD2C3yQ/XKNckclkuEa6EvdLHNe+vCaegzocmX+E+3/cxynCSSUpb21qpaWhBS19LTGxzY3N5feev2PpY8nix4ux8rOiKLEI227SdUYZ58nL5Dj0dJA07ooeFvFrt1+J/iyaloYWrn15jaBZQehb6lPypITfe/zeqVwuQNjCMKE5ui5OUpAdsmqIOEWnqauplr3b1tJGaWopeuZ6GNoYqjxf01RDZmUm93rdU9l/ZTIZq91XE5cXx7HcY4z0Uo3hQGhk5d/Lx8rXCmNHY0Ge881wcQoj9UQqlz+5jGuUK34T/PBr9OPYy8e495uwHvV4VX2jueB+Ab/3+J2B3wyUML0VbQrqy+upyq6iNKUUt/5ukvvH0MaQV9NeRcdYRyUGAjg4/SDpF9NZnrdc5TljB2MCpgZQklyCoZ2hyiTJzR9vcv3b68y+MBsrPysqsyrZNWoXvZb1IurjKMw8zJBryMm6noWFpwXn3znPkyNPeD39dVz6uuA1ykvtvQCdT6qBUOQ6MvcILlEuvF3ytuS5gvsFnHvrHF4jvTBzN+MT2SeSydPDsw/j2MuRmWdnCt6qPzzPC+2C7cQprMAZgSpqAQA9X++JeRdzTr16iv5f9Kfve32F681Ml+Y6IV/Iu5PHg20P6P95f/TM9fAZ48OVz64QtzaONzLfUJsbKKFrpktG4Cjqjaxpvwqae5jzx9w/qDSpxDhP1fvj7FtnST6cTFlaGd3mdBNj8azrWTTXNWPpY0nkh5Ei6cku1I6AaQFoGWiJ53PspWM8PvCY5QXLaWtp42fPn7HwsmBJcueKN8PWCIS2nFs5PNr/iNCXQrEJshH82WoEWamGigbSL6STsDWB3u/0xsrXSiSfKHFwxkFSjqfwbsW7AIzcMJKhq4eKRb9VLqsw8zCTxNvKpmx7xK6LpTq3mmW5y2iua2b36N1kXcsCBCnL5rpmSb4Q834MzbXNDF01VHKcacenUZpSym+hv6FjooPvOF8cejhgHWBNyvEU0mPSJTmWrokutj1tSTmagoGVgUpTHIS8Qq4lp625TWUKpKawhr3j99J1WtdOySZKeAzyUInf51yew+bIzdxec1vS7NPU1URTRxNFm4KWRmE9X9m0Uvz+Mi5lcHv1beD5pHd7mwZTF1OiP42mpbGFgKkBkr9ZX1Yv7INTDlCVU0XIghCSdiUx+IfBYrPPJsCGOZfncG/jPXRMdVC0KtDQ0mDboG2AsGff23iPgOkBeI30Eq/FupI6sYHrFOHE0NVDJR6snsM9ka+Xs7X/VjS0NRi9abTEHxjAyMGIsVvGqiVDZN/MpjS5VPz3xQ8uCpLUaqZvMq9kcnvVbcKXhovkzrqSOvZO3EtLfQtBc1Qn9k1dTGmsauTcm+dw7OUoFo5ByNNkchmnlpzCua+zyuQOwKWPL3H5k8sYWBt02hCvL6unqbpJ7XPZN7PZMXQHozaOIuTFEMrTy6nMqsShu4N4T007OQ2ZTMa9Tfc4sfAEM87OkFxXA74cwMCvnsvNKZsOyomjjrAPsyd8WTi/hf5G/y/703eFtJZgE2RD9s1sytPLxbVSCUWrgs91Psd3gi+LHy9WyUMvf3qZ1BOpzL40W0JSbG1qpTSllPx7+dh3t+f1dNXmuZGdEXOvzuXyZ5dFxZLbq29jE2TDiPUjiHgzAl1TXT7T/Iw+7/Wh/2fPSYPOR4W4eEO3DWjqaPLibanqUXVeNcUPiyl+WIxtsK1Ksy/5cDJaBlpqFVy09LXQ0teiua5ZZb1oqW/hyeEn2ATYEPJiCCN+GaHyfhCIrHc33CVgaoCk2Zd1LUucOBMfu55Fzq0cAmcG8lvob5x6/ZQKIbkis4JdI3cR/GIwvV7vxV+hfT3vfxOVVVX8tcHSc9gAFRWVXPr4klSZoU3BuXfOYdvNVlJPUk79Pzv7TG2zTx3RRYna4louf3KZiLci6PVGL6rzq7EJsFHJXy59conqvGoCZ0rv45bGFmJWxJBzK4eixCLmXJ4j5lrV+dX85PgTPV7rgY6RDtV51Yz5Y4zk/Ro6GrxX954khyl+WMzW/lsZ/MNg0k6nkXY6jZG/jeTZuWcMXSV4CCcfSSZxRyJuA9zEyWMLLwvx2r2w8gKpJ1JFaV9TV1OG/DQEl34uPD3zlF2jdklkfF2jXEVlGSU6EsfEc9bWwD7MnprCGqz8rHAfJOTtemZ6DPpuEOfeOkfMezFEvBkhEpq1DbU5s/QM8Zviebv0bZ4cfoKZu5mkvjti3QiGrhpKzs0cwdPyHw1JEEgzgTMDebDtATErYoT6aIcY7IUDL/As5hnbBm6TqKsBIun5/p/3aaxqpL60nj/7/knE2xEqecL6gPXoW+kz+8JsyeP6lvp4jvDsNPZTxlFH5h3hymdXiHhLmLpvX3uIXRvL9NPTRcKqtqG2pH5QnV9N1rUs7MPsMXMzE6f72++bf+N/hn/XmtcZ/mOafZs2bfrL53V1dVm3bh3r1q3r9DUuLi4qMp0dERUVRXx8/F++ZsmSJX/Ldv6bIZfLRS3tO+vvUPywWCWxcY1yJftGtuDtgzCBFrE8goKEAu5suEPw3GBxkd8zfg9tzW1MPTZVcozoT6Npa2mjqbYJTR1N5JpyKjIq6PdhP/q82wctPS2SjyZTmV0JCIumxxAPylLL+LPPn5h5mDFp7yS1jYsrn1/B0FaYwuvI6O7xag+6ze2Gok3BkyNPsAm0kRSk/qspIt/xvth2s1VJTjwGC+yy7OvZf9nYClsYRsC0AJUEWN9SH31LfS68d4HSVCHB0TPTk0xwdZ3SlS5Du4hJWkes8VhDVU4VU45MUQlcr397nadnnnLU6Ci93+mtMgFx+dPLz2WC2kFLX4uGygZ2jdqF3wt+hL0cRktDC2O3jMXUzZSEbQnomqifGATBUFfPXI/yp+V0ndJVklClnkrlxMITVGZXIteQq3gSHJlzhId7Hwo+Rh3WbT0zPeZcmUNTTZOYdGgZaOE30Y876+9gHWDN6D9Gdyrz0v/z/qRfSKcwsVCFgf3k8BP2jNvDsJ+HqW3I3d98n9riWnq/pZoA/le4+cNN4tbFoW2orfbYMg0ZU49PVSsfo2OsI/He6ojG6kaufH5FMD5WM81o3dVaRZpWCbcBbgz/ZXinjMyih0Xc/P4m9zff5+X7L3eaTHaEQqHgzvo7nFx8ktCFoYxc/7wA29rUKhYxAMpSy1SafZlXMtkSvQUQvMk6srWMHY2xDbbtVAYUBGZlRWaF2vuyJr+GncN3ErYojMBh0kB/nd86HHs68mbBm//UZ22PrGtZHJl3hOjPotUmIu1lozpCuWZd/OCi2mbfkXlHsAmw6XT65X8LPmN9CJoVhGOE+ikZ90HuNJQ3SIr4h2YdIvNKJnOvzkVTV5O7v97Fua+zynTf0zNP0Woyo7RrsPiYTENoNvmM86E0pZTkI8nErokVPBRdTYl5L4bmOqEIWJpayk9OQuKl7r4sf1ZO8aNinHo7cWjWIQysDCQs2Zh3Y8iNyxULmX+FC+8L0p76lvrUldRh5m6Gla+wrjlFODH58GT2jN1DS32L2oYGwJAfh4jyYx2JMtV51TjOcuRo7FGcs5zJ35NP6ZBSSbPvxvc3OPfWOSy8LXj53sv4T/LniP4RSYHm6dmnGFgb4NjLkUHfDRJJE8PWDEPHWAczdzPJ9GN7OPRwkKwVSonA5KPJnH/3PGGLwnDu44x5F3MaKxupSK9g8A+DCZoVRNLuJHwn+KoU4hUKBS31LYJnnprpvdriWirSK7DwsiD7RjYP9zwUvIn+kSg/OfyE6rxqur+iKvFZnVdN7JpYYtfEEvZKGCPWCWoE3Zd0x2OQBztH7lSRTVNCriEncEYgpamlxP8Rr0IuSdiaQOzaWMZuHivZv5TwGetDTWGNyuNNNU3sGCZMYo/5c4wYMygUClAI0wNu/d0YumYoj/Y/IudWDiPXj5SscXmxedgG2zLrgqrE9oMdz9mwAdMCGL9jvNiILE8vp7m+GZlMJiap7dFQ3kDu0Vzsu9uLZvJKKJPXiswKBnw5QCKzqJTcjnk3hturbhO2KEylKSCTyXDo7oDD5g4kGQVcXHmR3NhcAFyjXZl+crqkeVpbVMsvfgJ50MrPSkVy98nZJ0RfiOZ2z9uow/2C+zjkOHBl7BW8vvUSC3QbgjagY6zD3KtzybuTx46hOxj520hRurD9ZFvQ7CBco1wl0pKP9gqT2NNOTFMbcz458oSknUkETA8QG6u5cbnsHrOb8GXhnHvrnPjaudfm4mz5/F6Wa8o7Zf4CmLgKkmgFCQVkXcvCd7yvylSlpbelyt4JwvSnhaeF+B3rmesxZvMYrPysyLmZg5G9EbqmuvzZR5heN3E1wbabLbomus+niP4Cz84/4+ybZxn49UBJzNna3ErS7iQ09TRVGPQXP7wICBL/pi6m+E7wldxXY7eM/W8TmkCY+gmYHoC+pb54z9WX1RP5USQGVgaknU4jeG4w/T/vL34veXfy0NDSwMLTgraWNq58cQUje4Gd3R4x78dQmVlJibOg3NG+ZKhtqE2WSxYOOQ5Yb7KmKLAIa//nzdeWhhYUbQpee/oa2kbP15hn559REF/AosRFmHua01DRgLahNoHTA1WaCCbOJqLcmIaWBh5DPPAY7MHRF49S+KAQx16OuPV3wyXShfVd1xM0O4jgecGknEihKruKmz/cxGOIB73f6i3uk2O3jOX+n/cxtDMUJpdnBYn7WcblDM6/fZ7oz6Kx8LbAqfR5A0xTV1Ny7/pO8FXJa0qelFCZVYmZhxlGdkZo6WvxeP9jCh8UMuDLAegY6TD74vOi243vblBbWCuJ1Z6efkp9eT1DVw1F0aYQ8iOFYNdgE2TDWyVvSZoLWnpaTD85HRBIbOsD1zPy15E4hTsx6PNBbBu0jfhN8SRsTcB3nC9PDj9hwq4J+I73ZfD3gxn8/fO1oCq3irTTaTj3dkbLQIvy9HKJPHPx42JufHuDwJmBIlHo1upb1BTUSJowKSdSaKppwtDOkLTTaVz98iomziYEzggUi45Pzz4VJxNez3gduaYcYweBjOAzzge/iX7c+P4GJs4mktyhrqSO02+cRkNLg4CpAVTlVBH/ZzwegzyoKahhz7g94ms1tDVY2bRSJY526eei0gRR4u6vd3mw7QEJWxJ4r+49LLwsSNiawJNDT+g2txuECnH65U8v49jLUVQBMnMzo+uUrpxYdILWplb0LJ4TOHJjc9k7cS/Rn0bTbU43ytPLOfXaKXzG+ojfozKvDH4xmJs/3CTzcqZAbtHTImFrAhfev8DEPRNxinCi38p+hC8NF5uP24dsFyfCXCJd1JJcQcjnLTwtVPKtLdFb0LfQpyipqNN8+8b3N9A102XCbtWc6vHBx8S8F8PIX0eqVbL4Lew36krq6P9lf3GSJv6PeK5+fpXFjxeLU7bKyRyXfi6M/G2k2vgjcVei4DcW4UThg0LOLj+LpZ8lA74coFbSXM9cD//J/uI93h5DVw1VaaqLkAlxhmO4o1qVi4aKBqrzq0nalcTZ5WeZuGciHoM9KHpYxG8hvwGClLC6v6tE/KZ49C31CV8Wzkt3X0KuJdwDINyL3mO8O32/fXd7tdOMvd7oRdjCMCoyKtC3UrXcWJS4SO3xYt6P4dqX1xiyagjOfZw599Y5vMd6i6TSjT02UpNfg0s/F5VJoNi1saTHpDN+x3iC5wfTdUpXFfsCc09zoj6NkjQj0k6lcfWLq7j1d8PK14rYNbEk7kjkjYw30DbUJnZtLBraGrS1tInSjn+F9vW8/02YGBtTkJf3T722ELC0s1TxbVe0Kbj5/U26Tukq2fe6DOvC6D9Gq732lagprKEwoRDrAGtJbGTmbsaipEXk38vnR4cfGfnrSMl+o0RbSxvmnub0fU/a/G5paOHWT7fQMdHBqbeTRJZVU1eTkJdCcIpw4s4vdyhNeU5MSD6WTOblTKz8rQieG0xTbRPp19MxdTXF1M2UYWuH4dzbGWNHY6pyqsi4kMGjfY+I/jSauVfnknMrh03hm0iPSSfyQ2GCrf3nD30pVEIYMbIzEqeZZRoygucHq8gWq0Ps2lhybuUwbts4sTnSVNPE+J3jMbQ1VCF1Kwkwt1ffltRuPmj+AJd+LlRlV5F9M5sDUw/gMMmBAa8OQN/keRymoaVB+oV0Ln9ymcE/DCbrWhZRH0dhE2jDuK3jCFkQQsH9ApVmrBJK/9nO5JEXPRDu5YxLGZSmlorToUrc33yfoqQiui9WzSFtu9kyfsd4Ln96maaapk7lNpWy2crYZ/Tvo0nYkkDa6TQaKxslMV9HFMQXsP+F/Yz+YzRmbmYM+mbQf1t6+G+ox79rzesM/zHNvr/x/2+0traSkZGBq6srI9aPIH5TPGu91zJ261iJrI1LPxfKnpaxIHaBuAGWPy3n7oa7uPR1EZt9ijYFba2qTHwQJLYOzTzE5MOT8RnjI8js7E4SGeztG0jW/tbMOD1DlBGpzKwUZWA64uoXV3Eb4KZ2ukPbQBttA23SL6azZ+wehq0dptLMvPPrHbQNtFWKlcWPijk08xA9XushTkM01zVz9s2zOPd1pr60nsyrmTRWSzcYJR7seICmriZ+E1SL9G2tbbQ1tzHz/Ey1E0wgbC7JR5OpLa5VadaBkPyWPyvnwvsXVJp9kR9G4j3am+QjyRK2KwjyfI69HAmeJy1sgDB1V/KkhMyrmTj1ETb6Z+efsX/yfsZsHsP5d86jpa9FfXk9ThFOKgHGwoSFPDn8hHNvncMmyEYSsGjqaGJgY4BdqJ3aorN9D3tST6ZS+KBQZTIBnv+W7f8d/mY4RQ+L0DPTU/Gda4+Wxha2D9lO0Owgic8SgL6VwPCxDrBW6613/8/7ZF7JJO1kGhP3TlTLxu8MPuN8qMys7FRb/uj8ozzY/oB3yt/5p4+pxINtD4j9ORbP4Z5qE8zCB4Wd+gbZBtli7mFO2pk0LLwsVBqgW6K20NrUyqjfR2HsqMpa7wyKNgUnFwvEj9YG6XRcY2Ujt1ffxnO4J0NWDVHb4DSwef7dtrW2qRQp2jOcO8PB6QepzqtW2+xLvygwKxtooLW1VWLgq2+hr9Zf7p9BQUIBFRkVaqdTACoyKqjKqcK2m61Kc8G5tzPTTk5T67cGQnBZW1xL+bPyTpuz/xuQyWUqklXtEboglNAFoZLHHmx7AAhkg/qSemF9/yhS5fpclrOMadOAdnaj7QNh33G+XP3qKvc33xfX17RTaRTcL+DB9gcM/GYgRg5G6BjrUJRURPyf8QRMCxCb/Y8PPebcm+eYf3M+A78eKMqMKNFraS+1csFHFxwl/nehATT58GTcB7gT/GIwdaV19P+8Pxo6Gph7mEskynzG+LCiZgVPzzzl5o83aahooCixiBcOvoBMJqO+rJ6rX10V5SU7ImlPEnHL4rCeZ81Tj6eE3A0hsKt0L3pySPBKkGvKRQ+vN7LeQENbAx0jHRoqGvjG7BtaGltw7OUoYfzbhdjRWNWodk0F2DZoG1r6Wkw58ryB3tLYQllaGRZeFtQW1nJqySma65p5v+F9YTo2Jh1zT3Nkclmn/oeKVgVfGnxJ1yld6buyLxXpFbgPdBeTptSTqRyZc4Spx6cyfsd4odHXbuLt7q93STuTRuCMQJWpbIceDizLXcbmyM3k3haaSVr6Wgz/WVgfHh94jFxTzrzr8zqNRyw8LSReuBUZFYK3XK1gyl5TUENTbRMWnhaSAmD4snCSjybz5PATiQ9N5tVM8b91zXTFJlFLfQtfGnxJ4MxAZsUITbxrX14jcWeiyrRjZ4Uo5fl2m9eNtFNpovyUdYC1UPzW12Jj2EZRDSBoVpDk3i3PKOf4y8eJ+jRKhXhwe5WQxOffycfAykCyvyk9o6rzqoWp1iPJKs2+qpwqKjIqsAmUTs831zWLjT6AwoRClSlNbUNt+q7sS+KORLUFxbq4OiKvRNIjtgfHq45LFBjaWtqYs2EOrRqtKMoUNFQ+b+7aBtuKLGJrf2tGbBghysN3RHuJuMrsSoofFrMoaREGVgadSsymx6TzcO9DluUtE4tNGtoamLmZoW+lT8hLIRjaGGITaKPSlGtraaM6v5qW+hbubLiDx2APSfw24IsBANxadYszS89g281W/BspJ1I4Ov8oI9aPwHecanGi25xueI7w5OQrJ/Ea5UXQrCBxH4xbH8ejfY/wGOzBkJ+GYB9mL/GsbG1qJe6XOM4sPUPw/GAV36grn1/h4gcXMXY0Vvs7rmxaSWmKIOnqNuC5nLqJswlOEU6iRFlH39L2n6OtpY3dY3fjEulC77d6s3/Kfp6eeco75e+QeSWTZ+efEbYoTFLga2tpQ9tAG//J/mIBK3FnIqdePYWmriYtDS30fqc3A79+3oy5+cNNwffmQSFrPNYAgpRax2ZfXmweRQ+LQLrNAUITX94qx6rYCucYZyoyKiSFn+E/D4efVd8X+lIo7gPc0TbU5tZPt4hZEdOpwkK/lf0kBCDlWrt34l4qMyvJi8tDQ1sDl34umLiYoGumy6GZh8i5lcOojaN4PeN1lYacUnZ3jhq20gABAABJREFU1oVZDPhqAJq6migUCrb230rGpQxAIB51nJgpTS2lua5ZJH6pi8XOv3ue5CPJAMw8PxPn3s7omOgQ8XYEdzfeJe9OHv0/7y+uMSM3jFTxEZ1/67msWnNdM+t81uHU24kxf4yhtblV8h0fmHqA7BvZvJH5BiDEjK2NrShaFbS2ttJi34JrtCsZFzMwcTLBNtiWloYWSZzZHsUPizn24jGRGNBxuramoIb7m+9j6m4qyvU+PvCYrKtZlDwuYeA3A7H0tuTs8rOSKbUL719A11RXkmMaOQjXsLahNqtdVwNCEdXCy0L0ef9C/wtco1wpeliEXFNO5AeRbB2wlarsKgB2jthJt3nduPThJbT0tfAZ44NdqB35d/MBoaHYsSGSeiqVncN3omeux9ulz6dwFz9ZzP0/74vT/Lm3c8Xc1CbIhsiPI0XLCX0LfWwCbFQ8ynRNdXnt6Wtk38iWkCTkWnKh+fuP19eV1BH7cyxGDkZis8/Kz4qB3wwk/o94us3tRvSn0eK5axloYeTw3MNLz0xPQnCsyKgAIOKtCAZ9q76YevPHm1z88CJzr8xVaeiFLQpD21D7Lyc4Fz1YhIaOhtiMkkAmxGYdZTKVsPQRpB/bT9Z5jfDC0MZQci2WPCmhrrQO597OaptcFekVHJx2EM/hnkw7MY2DMw5SlFjEs/PPSI9J55UkKVkmNzaXO7/eIeLNCLHJ+M9CriFn/I7xKNoUVOdXo6WvJSGzKQlsT88+xS7UTtz72xM37ELtiP8zHqcIJ8k+WFdaR9wvcQz4coCoAKIk27Q2t/Jwz0NqCmrE+yDvTh6JuxIJnhdM/KZ4bv10C/8X/Jm4R70PtaauZqcy/Jq6msI+t+wMs87PEuOCuHWCr6euqS76Fvrk3MqReF7ah9mTciwF6wBrWptaOTj9IE59nOj1ei8KEgpIPpaMok2BtqG2WjsGC08LFbWG0JdD8RrpRda1LMy9zOkyrAtPzzwFmZBLK72eFz9erPazdET7el773Pb/NMZMm8a2L79keoN6Ql17bNXTY8or81T8nmUaMl5Pfx1NXU3ub7mPmZsZLv1c0NTRVNmH26Mio4LLn1zm/ub7ou2AEpo6mlj7W4sEjc5IRP0/669W4lHHSIel2UvR1NOUyOeCsOaN+lUYDuhI6D331jlxvbfytaK+rJ6dI3aKvsnKdWXGmRmUpZXh1t+N8TvHi/GKYy9H7LvbkxeXh4G1AS2NLZJaVGeKCiDUcjrGavXl9eTezsXS11IkY5xeelqM9cdtHScS6tMvpLN7zG7GbhkriQeb65pprHpe2wxfHs7NH24CQswVNCuIvDt57Bq5i7IFZfzGb+QMycEz0pPpp6Zz7p1z3P/jPnUldZi6mtLS0ELykWS6L+7O6TdOE/dLHMvzl6v102usauTmTzfRMdWh97u90dLX4s9+f9Ln3T54Dvfk8OzDaBtpE/VxFE21TTj3dVZL0k7alYS2kXandaPWxlZufHuDkAUhkmZfaUoph+ccFmMn8y7m4l70+MBjHHo6MPnQZPFcFQoFuia6FCYWsqnXJmGI5Z0+2AbbMnHPROy7/2tr79/45/HvWvM6w9/Nvr/xH4PqaqHiauVrhX13eyHYaIerX11FQ1uDV1NflQQxXiO9eLPoTUnxurOJm4sfXSTrahZBs4PE5lPkR5E49XEi+0Y21XnVOPR0kEzdXf/uOuffPi9Ke6acSCFha4JkQwJBIk8dow2E6YG6kjrMPcwZu3Wsijdf/B/xnFh4AoceDirNvraWNurL62mubRb17pvrm7mz/g73fr+HQw8HgucFU/ywWK3fx8WVF5FryrENssXIwUjy3SXuTOTwrMNi4xOEQH+d7zqMHIxYlrOMupI6Ln54kebaZrXNvmknppF2Jk1tQcopwgmnCCe1UxGnXj1FfXm9Wm+K5KPJHJp5SCIJZ+JiQvfF3bH0tmTS3klkXsnkyJwjjNo4SqXZZ97FnJAFIXgO91RpErn1d2NBbOca9H4T/Tj35jkebH+gUpguSCjgzz5/EjQniIFfDRSvOceejix+tJjW5lYqsyrR1NVU679w7/d7mDibCMzUDnDu7cykvZMoTy+nrqROhWU0ZvMYzr11joyLGf+yvKP7AHeJp1lH2ATa4DnMUy1DsTK7klXOqwB4v/59FTk5HRMd7EPtO/VS2hC0AUAliFOitqiWfRP30fud3th8LW329X63N/qW+n85taoOMpmMWRdmYWBloOLJV5UrFCisA6w7ZaFZ+Vr9l/6E1XnCeqWuWQiC10LZ07JOj99tfjdMI01Vnpt7dS5FSUUk7U76y4ladciLy6Otua1Tr6bVbkIxZ87lOWoZ1Z15WoEwSbTafTV3f73LS3de+qfP6X+KyuxKwTO0p6Pa7zrvbh7JR5MJmhUkTqoszVlK6olUKjIqiFsbR/Rn0Wr18AH0S7NxvXaQXK9oIJDWpla2D92O2wA3+r3fj74r+kqKIwvuLCD1ZCq7R+/m7LKzvFspTOU9OfyEWz/ewi7YTmz2dRnaBX1LfczczdT6d/1V8gTC+qksbPiM8RHX6I6oLaol/14+BQkFxLwbAwiNqLw7eaAAZEJh4+b3N7n5/U2mn5qOTEOGjrGOuGe4RroS9W0U68vW06bRhsJAIVnTm+ubeeHAC2hoa4jJa9KeJNJOpTHou0HoGOmgZaDF5MOTJV67Spx/5zwZlzNYlrsMTR1NFe8FTV1NSlNLOfbyMbrN7oZThBOmLqYsiF2Apa8lMpmMh3sfUnC/QPBYszFk4YOFlKWWUZFRQXV+NU8OPSFxZyLzrs2TTEgFzQrCoaeD6Pu7LHeZeC3Zh9oz8NuBWPpYqiW8DPp+EOkX0lnjsYa3it+SPKehLXgnvJr6qvhYeXo5D7Y9wHOEJ2O3jBU8cy9miEVV8fusa+Znz5/xe8GPoT89Z7Vf+vgSCVsSWJa3jO6LuhP/ZzxH5x1l2slpKvdnzIoYFG0KSbPPzN0Mv0l+aBtoS8gTMg0ZfpP8cOjpQMqJFIofFjNs7TCGrR2GXEv1cxckCN9zRwKGcx9nlYaVTCYTixLeY7xRtCkoeVJCwtYESbPPzN2M8PXhuPi7EPNeDO4D3cUia8iCEOSacnq+3pOmmiYaqxrRt9QXr8Fpx6dRmFiIvoW+WoZt0p4kzr15DgsvC2Gy4R+yqofnHBZfM+j7QeJEa/bNbEoel9B1ald0jHU6LbwA6L2gx3d63zFp3ySVpoBCocCg1oBk72T+mP8H303/TnyuPaHH2NFYMm2WdS2Lq19epc+7fcS1WNlATzuVxvGXjzPj7AwMogz42uRrfCf4qsg1RX0cRfiycElj1LyLOQHTAzB1UZUxP7PsDAXxBcy+OJuKzAp+7vKzQMY6moy2obZaSTHfCYKqRPt9VNtQGwtPC9IvpBPzbgwj1o9QkSJurmvm0f5HmLhIST6BMwKpK67D2NFY5e811zVz7u1zYsFTHTGgtrgWQztDlqQsUSlmKifPUo6lELMihgV3FqAXKqxVY/4Yw8N9D/lE/gmT9k766wl1mdBIVV7TVn5WonR/1rUsrnx2RWXS8eSrJ7m74S5vFr0p/h72YfZYeFuIBTjHXo7k3M7BJtAGLT0t3Ae7U/igUDxG35V9VSYSAWaem0ldaR0lzmsocwig2eq5HHDenTw+/OxDLkVeIjIuEveuqnGeQqGgMqsSLX0tDKwMaKhsYPfo3cL+ADj0dCD4xWD0rfRJ3JlI+oV0QXJRzYS4ckJYJpfxwv4XaGtpo7a4Fm0DbXSMdUTvobu/Cl7N3mO8xe8j/s94ihKLCF8ejpG9EW9kvoGxk7FYbGxrbaMwsRC3AW7YBNmolaw//dppMq9k/qXXTPC8YLQNtDF2MsbE2YSmmiaSjyRjaGco7o+KVgW+E3zxHOapdmq2fSxckFAg/vfh2Ycpf1bOzPMz0TPXw9jBGGNnY8wKzPjK6Cuaapp46d5LosRpSWoJMS/FYGphSpdhXRi+briKP2b8H/HiJJpyUn36qekSX7v2cOnrworqFaRfTGdL9BZGbRzFlMNTuPH9DW58f0P0Hhq2Zpg4tbcsbxllqWXi+nVnwx30rfTxm+DHR4qPaKhs4BtTwSesua4ZHWMdmmqaaG1uZd71eWgbarNr5C40tDWI/CAS/yn+4rWbejKVXst6sTBhIYZ2hhhYGTDn0hxqCmrQNdOlvrSerOtZ2IXYifesklRWX1bP3d/uUlNYQ8iLIVh4WRD9WTRyDbkYM8X/Ec+5t88R+lIoOTdzRIkyHWMdaotrqcqpEpuvt1bd4uKHF5l9cTY6xjqs9VrLmM1j6Dq5K3bBdpLmsW2QLctyl6nE2Fe/uMqD7Q8YsWGEJE72m+AnIc3Wl9VTX1aPibMJGtoaElnbbYO2YeFtISnw1hTUcHb5WfHcO6Izier2+Cuine84X3zH+ZK0O4mUEykqUvDjt49XeY9jL0eV2DRmRQzJR5MFdRs1SNqdBIDnCCEeiXgzgpbGFkpTStXWAcqflXP/j/t4DvdU2+y7tfoWZallOEU44dzHWS0xtK60jh/tf1RRalHCY7CHpJFj6mLK2K1j0TbUprWxlaPzjjLyt5FYeluSsC2B3Nhcur/SnUsfXqLvyr4ETBOaNZXZlWjpaXFm2RmRNBjxphAzFD0s4taPt3CLduPWT7cAsA5UP1FTklwiesIb2RupTCvlx+fTWNWIx2APyfU37/o8ih8Vs/+F/QAM+3mYZBpt6KqhaGhpCHYKMmGKWEkiHL1xNKM3jha/r5r8GkzdTFXqUtk3sqnOrxavZRMnE0ycTLi95jZJu5JY2bRSXEMUCgWzL83udOqpMyjref+beHHBAj775BNuA39lpnEbuFjfyFfBqs0WmUyGqaspbS1tHJlzhK5TuuLSz4W49XGkHE1h7NaxaonVqSdTub/5Pl2ndlWxXGltbqWhogH7MHteTX0VbUNtLn54Ebf+bmJcqsSZ5WfQ0NaQTGTL5LJ/idysxPB1w0U/zk3hm/Ad78vwdcOxC5Xubx0n9kBYN808zFgQu4DEnYms810n+KgffEyvpb3U1mCKHxVzYNoBer7WUy1pv/hhMTuG7WDIqiGi9GvJ4xJM3UxZ/GixRDnLvIuQux+efZij848S/Vk0fd7tQ3VeNTuGCkolSssVtwFuVGZVinmLz1gfTF1NiSqJokmnieMDjrNjgfCee7/dE9VVjJ2M6fteX3GScttAQTL6O8vvmLR/Er7jfSUyjPXl9Vz++DK9lvVi4FcDSb+QTsmTEvF4+fH56Jnp8XvP3yl/Vi6J/dpj9KbRNNert34qeljEnQ13mLBrgihdqsSjA4/IuZkDCGS1AV8NeP7c/kco9iqQacjosaQHv4X9hpaeFgsTFqKlr4VLpIu4jhrZGUmm8jMuZ3Dzh5vYhdoR9VGU2vP6G/86/h1rXmf4u9n3N/7j0FzfjPdobxXWSmVmJc/OP6MyqxJFqwK5lmCurKGt8U9POT05+ITWplaJbrKljyV/9v2TupI6QJjqMbAyYGHCQkqSS7jx3Q0A+qwQgp+b39+kMLFQpXFh5WtFa3MrR+YdwbGXI6EvPafhxv0Sx+WPL7PwwUKCZqo2PJTFjdF/jFZ5zibQhn4r+3F0/lFMXU3xHO6Jnpkebxa+SezaWO5tvEf29Wwcw9VPyY3fMZ476+/ws+fPzL06F+c+zigUCmQymSg9VFtUS/qFdNz6u9HaLCSD1bnVKBQK7m26R21hLRP3qmevASqeViAwTdJOp2HhbaFWfrHfB/1oaWxReRwE8/Y+K/pIjKttAmxIOZ5CXlweL95+EUsfS7XSpiA0YjR1Nf9SgqEzGFgZMGH3BLWMPA0tDcHjcW0cXiO96DKkCxUZFeydsJeQBSHomulyYMoBALXNooe7H9LW2qZ2Ag4EX5k/ev+h1hTazM1MhY3+fwq93ujF5U8vc2DaAZW/0d7UXJ1vVeD0QAKmBXQq7xH6cih3f71L/r18lXvm3u/3uPzJZfq+3xe/SarFt3/Vk1MJmVyGW7R6bzlDW0PCl4dze81tcm7lMO3EtE6b9J3h3u/3OLbgGNZdrTudgumsuQRCgm3X3Y7ExES1zz8++JhLH11iUeKif6nZF/pyKK5Rrmr9JgE8hnjw9MxTlQIsCI3/+E3xjNgwotMmaM/XeqKp9+8NK7KuZnFw+kHsQu3UNhljVsTw7NwzHLo7iM0+YwdjQl8Kpb68Hv/J/pi6mqotXGZczkC7opQWLT0UciFhTjudRsbFDFoaWpDJZfi/4C+Ru5NryPEe5Y3fRD+qcqrExz0Ge/B6xusSJqa1v/Vfymt0hvZJe3soFApBSqdVwbPzz8i7m0fvt3qTfTObPWOfy2YNWTWEsJfDJI359s2vtNNpJGxNwC7YTpzysguxwy7EDs8/PSlPLacqqYpa81pxb905Yif1ZfUsvL9QPE5eXB4JWxKI+jiKG1tvcP2b68y9MldcOzf22IiZuxkTd08k7XQaAN+af0v4m+ESCUOAqcemknoylZ0jduLYyxGnCCfOvnmWR/seiZLK/i/4S5KWhK0JXP38Kv0+6MeVz67gPsgdEycTyWSeXFMuNpxyY3OxC7GTTM5ad7UWGxmlKaXUFNbgFOEkNv6s/a3p/0V/yW+tRHNdM5XZlRjaGIr3aVlqGZc+ukTs2ljeKnqL5tpmseDacU8wcTZBJpNx7etrOIY74hrpin2YPS31LeLUpn2YvTip0drcyndW3xHxVoRI/hn+i7SAoSTDgOBnZeVnxfRT09HU0WTS3knk38vnt1BB3qrnaz1VyBsgTNFvG7QN22BbXr73ssrzHVGUVMS5t84RNDuIAV8OoPfbvfnG7BsVFrWOsQ7WvazRQYdrX11DU09TbBK59Xcj/UI69eX13P75Nhfeu8BL917CLtiOgvsFHFtwjPDl4SoFTCXcot3wGulFyvEUSlNKxaKKc29nmmqayI3LlTSHYtfEioQKdQz49pCbySmxKmH9K+tZtXKV5DkNLQ1WLV2l9n1/hdriWjIuZRA8X4jZlKSyF2+/iEs/F0ZtHEVzbTOfa38OPC+GtIeeuR5NtU0cmHoAv0l++L/gT0t9iyBf/Y+9oD2UBUoQ5Op6Le2Fc19nhq0dplJ8vr/lPtW51fR9r6+KKoNrpCtzr84l+VgyWVeyVLwf0y+mk3U1i7eK30LfUp/yZ+Vsid5Cj9d6EDg9kEsfXaL7ku5isy9xZyKlKaX0eLUHceviMHE2YerxqSqNZoBhq4ep+Anlx+cT90scgTMCkWvIBenw3RMkTZXKrEoOTjsICmH69uSSk7hGC/6ICoWC762/x3uMN6N/H41cQ8779e+L720/RRr6Uig+43xUGjYO3R0oTChklcsq+r7Xl34r+4meXKXJpTj2ciTvbp5EMm/wd4Ox8LIg50YOI9aPUHsvKiGTy2jT1KFNLn2Nnrke6cHp5Nvlo2+nr3KMp2efUpFRwfGXj9N1Slcm7JpAY1Wj2OgDoaGrlNe8teoW8Zvi6f9Ff/jHz/7owCMKEwqJ/DCSiowKfvYURgU/UnyEXFMuNj0bKhooTS3FzN1MaKooBBJHeXo5BtYGpBxNIfloMs59ndn/wn7G7xiPx2APyp+VY+ElTC539FpM2pNEQXwB/T/vj1xTTtCcIIlvcMx7MWhoa0i8dLxHe0sUWhQKBQ49HGiqamLK0Sk82PaAtFNpPNj+gJUNK8Ume/tCX0FCAW3NbdiH2WPsaEy/D/vRZWgXSlNKKXlcwobADfR4rQfDVg9j0DeDaGls4QvdLwBBbUY5udxU3UTpvVICPgrotJkTvylebMLl3MohfFm4eG80VDSQciIFm0Ab8X5QehYp//30zFNCXgxhwJcDGPClUBAsSiri4IyDmLqZUpFeQcalDMn0Scx7Mdh2sxUL/jrGOkw5MoWS5BI0dDTEPUD5OwPCb/qPr6jvir449nIk5XgKXiO9cOjuIIk7tQ21KX5UTGFiIQ3lDdz84Savpr4qrmOB0wNpqmnixMITPNz7kPSYdDyHe/Kj/Y+AoM7S2tSKkb0RehZ62ATa0NLQIky91zxX1+koLWhkb4RdsB2XPrqEmYcZbv3dOlW6UJJ12qP4UTEPtgtNni5Du4jHVjctd/PHm1z94qqK7/eTw094dv6Zinxje9WMzVGbmX5quto17unZp+iZ66ltjFXlVLF/yn6KHxZ3qsRy+o3TWHhaqN0r8+/lc+KVE0S8GdEp4SF4fjCu0YK/r9KHuX0u3WVYF/TM9cQpFHUkzvbwneDLkuQlXPvmGnJNuQpp7f6f9ylMKCRuXRyT9k2SNPtam1o58coJbIJs6L6kuwrRKPNKJuXp5QTNDFL5jZS1ltriWqadnCbGeRdXXqQyq5L+n/dnYcJCWhpaqMyqxMTZhLVea3Ef6I7nSE/KUsvou7Ivp147JXrpuka5YmBlwBtZb6Clr8WD7Q+4/u11FX/FR/secfEDQTpaOVnVHnvG7sHAxkCFeGztby3JIZR1KSVMXU0Z9P0gDKwMkGvK+bDtQ7U+UUm7kjj16ilmX5otqTdkXM5gS5RgUaG8r1ubWlG0KRjy0xAMbQ15du6ZSERUej03VDYQtz4Ou2A7tcTF/xuwtbXl62++YdT773OssVFtw+82MFJTi2G6Qzg28hgZ0zIYv+N501vRJhBhtA21mXrsuZVJ+bNy0k6nsX/yfhWfNRAsSMZsHkOXIV0kCisARYlF/Bb6GwO+GkCfd/tQmFjIlc+uINeUS2Kyp+eecuvHW1h3tZY0+1qbW6nMrKQ0pZSMyxmEvhQq5qAlySVc/vgygbMCMbI3oqG8QTymktTd0tDCoO8GYR1gLdbmsq5lceylY0R/Fk3K0RQqMioYu3UsFRkV2Ifac2bZGczczHi07xG23WwJWxSGqZspzbXNtDULdZ9jLx+j4F6BaEWhUChorm2mtbmVnNs5xP4cS8/XeorDDOae5oz6fZRkCKEzBZb260tbSxuZVzLp824fyRqqVFhqT3q8+tVVylLKGPPnGJo+EfaFu2F3xf1/ScoSrn19jby4PGGSsBPsm7iPgd8MpNcbvUTCgpGdEYsSF1H+rJy13mvpu7IvbxU9J3wq5Ts/kX0CCPlg0u4k7ELsxOlzEIh+t1bdIlWWquJ3WZEuEJLtw+wl972y/qCpp8n0U9NJPprM0ReP4hrtir6FPu/Vvsf+Kfs5//Z5HLo74D/ZXyQomXuYi5Li6hC7JpaUYymkHEv5u9n3/xH83ez7G/9x2BK9haqcKpblLBMfa2lsoceSHqSeTGWdj+DbaBNkw8L7C6kvq6c6rxpTV1Mx2Xi49yFtLW0iW0uJ+Tfni4mdEoo2Bb3f6S3oZ3cxJ2lXEkUPi1C0KShLLaOuuA777vbcWnULXTNder/TW0WODQT9bplcxv0/79Pa2Cpp9rlGuqL4QKF22guEhlxba1unhWEzDzP8J/uLQYVMLsPA2oDoT6MJnBHIkyNPVIzZlVDK8hg7GYtsoW0Dt1Fwv4C3S9/GuY8z2wZtE72jbAJsWJqzVJwec4pwot8H/TqVn7r40UWsu1qr+BM8OfKEw7MOA9D7nd4493GWmE37jPXhxCsnOPPGGZbmLJVMVdgF22Hd1ZqUYyk01zaLSU+XYV3Eor2BtXqzexCKnO4D3RmzeQw6xjqSYl7xo2JSjqdQmVVJbVEtE3dPlCQJd3+7S+GDQrWeZ1Z+Vrx09yWeHHkiMixbm1vFpnCvZcJGrizgdcQLB18QA6eOyI/P5/aa2/iM81EbTNeX14Oic0mIv4LS1F3XTJd3ytQniC59XdROTFh4WvzllFtuXC6/9/idgd8MVGsoP2L9CIavHa5SEARAJkwGdp3atdNr/96me1z78hqT9k/6S8+59qjKraKppon8u/loaGtIklojeyNCXw4V/TieHH6i4lFTmFjIsReP4TbAjahPolQmHu/+JrDWg+Z0nuTun7IfLX0tlWkMJX7x/QV9T30CjkrXqBvf30BDW4NJ+ydh7PSvsfscejjwYNsD2lra1DYbOwu0QWAdp19IZ63XWrW/9xqPNdh3t2fi7s6b/v8bUK5fHZNeJZSJbnsGdsblDDS0NXAKd0LHWIfGqkaJMbcSe8fvxUjLhkd9nzcRb626hZ65HiEvhnB0/lFKKcU/yF+YsjPVJe9OHnJNOZP2TaKxqpFrX1/Dvrs97gPc1U60KfFH7z8wdjSWSP4cmXuE/Ph8SQOtMyjl65TQMtCiubaZ7q90x7abLc59nMm6lgWgklCA0JRY2bSSkicl6FvoY9/dXkUiBkBLroVDrgNPv3rK4/WPCVsoTJo01zZTmFDI3d/u0tLYQsiLIfT/oj9Rn0ShpafFuTcFf7Dy9HKx2adjrCPuye4D3dHQ0cDQzlDtBLryNW+Xvi3KbHUZ1oWK9Aqenn1KTWENrpHPPc2Uvru93+2NW383dE118Rrl9ZeeEQ49HFSm6tvj6pdXSdiSwIqaFSIBoLW5lV5v9FKR8gWhMLt1wFYCpgUQsiAE1yhX8Xo1czcj/o/4ThmdWvpazL85n9LUUtZ6raXPij64RrrSY0kPeizpQUFCAQXxBXiP9hbX1azrWTRWNnJx5UWheF3bpJbU8FvYb5i6mOIY7qjS2FdOqUR9GkVNYQ0N5Q1YB1hL9l/lNFyvparXUfrFdOI3xdPW3Ib7YHdC5odQklxC2uk00k6n0XVKV3RNdf9yzzD3MmdJyhLJ9ddc3yyQnmRCsb7Haz1ENnlrUyuFiYUUJRWRfy9fLZHHLsSOyYcm09rUKpkuGPnrSJ6efcqg7wfR1tzGowOP8BrpJWne1hbVsmX8FqpqqoiYGKHiWdpa04phtSF1+nW0trWCGpUWw2pDXDIFmXllYebSx5cwcTYheF4wBfcL2PfCPvq824fgecH4jvPl/brnDSVLb0v8Jvqha6qLhZcFlj6WfGMuTNr0eLWHiq8LCGtiUWIRj/Y/wtLPEn/8Re/DZ+eesX/Kfmryayh6WMTkg5PxGOxB9k3B81rfUl/071SHxO2J5Mbl0ve9vmqbIQDeo7zVTl89O/+Ma19eo+uUruhb6gvxqo0B2gbatLW2MXT1UDR0NNgQtIG+K/vycM9DUk+l0u+DfixJWYKuiW6nsTIIjbu002m4RLpg6W3Jw70Pif89nqdnnlKVXUXoy6ESqVUQpsraWtro/2V/bIJs2DliJ3ItudDsUAgEs47Tt+qg9LnuiOB5wXQZ1oX9k/dLClSRH0XiMdgD79HelD8rR9dEV/J81pUsYYpo/Qhqi2pBhgp5Med2Di31LTzqJzTe21/55h7mXJx1kYKiAuqe1tFo1Chp3MaujSXtVBq9lvUS46eW+ucErteevoapm6n478HfD2bAFwMkx3hy6AmJOxLpt7KfhOyTfCyZppom7ELs0DXV5eaPN7nx7Q38J/szYecEFG0KQT1k9mGmHJ3CqN9HMbRuKDKZjIHfDqS1qZUHOx5w5o0zksnllsYWGquEz5F6PJUH2x8Q9XEUck25Slz+aN8jtAy0JM2+jihLLSM3Npfc2FzG7xiP9yhvnp57Sk2+cK/8GvwrzbXNkgntYwuOUVtYyxuZb2DqYkr0J8I0pVO4E001TWjpa0n2Ek0dQfmlobKB2sJa7vx6hy5Du2ATZMPIGyMJCAgg51YOSXuScIoQlGQCpwdiH2bPuG3jaK5rRtdMV4V4VpldyaEZh4j8OFJsDDXVNFHypARjR2P0LfVpa2mjIqOCttY2ce1pbWqlrrgOr1Fe3E+/z60fb+E10kucMJp+crroB7jGfQ2+E3x5Yf8LeCPcz8p4yybIhsaqRuRachXimVu0m7j/tLW0UV9Wj6aeJukx6Vz88CIN5Q1UZFQw6PtBDPlpCHoW0twleF4wAdMCaKpuEpRv2hEaqvOqRe/ZD9s+xGeMD2ln0si/my9OdGjqauI/2V8ymen/gj9dhnXha+OvAZhzZY4o0Vb2tIxH+x7hOcITmwAb2lrbKH9ajrahtljkV6pxDPx2IA/3PhQIGLEv4tDdgeyb2WRezqTbnG4Y2hri0s8FRZtClPJM2p1E/B/xPDv3DOsAaybslPrqWflZoWOiQ2NlIyZOJip5xe4xu7Hyt+LmjzfxGOTB1GNT6YjV7qtpa25T26grSioi/WI6A78eqJaseuOHGxQ/LKY6t1psmN744Qb3/7zPjDMzRGlQZV59d+NdtPS1VGIfu2A7lVxMGRerm8bW0NJAoVBw/4/7GNkZqTT7ph2fRmVWJTWFNSrxWWtzK/Gb4lUkwZW4s/4OSbuTRCnbrlO6Yt3VmsLEQnYO30lVThUjfx0pqcdUZlUCQvzWZUgX1vqsRUNLg0WJi+j5ek/MPc0JmR9C2MthVOVUsWvkLmQaMnzH+4rxrIm+EFclbE6gqaZJJff1HuONlr4W+ffy1ca7fd/vq5KPgHDdJx9NZvi64ZQ8KUFTT5O8u3miWsjWAYLMsbo9rvBBIbVFtbj1d8OhhwORH0eq5CTK3719bezsm2eJ/TmWl+Nf5tZPt4SceEYgvd/uLcTiCqgrruPkKyfp/W7v/9c0+wCWLl8OMhn93n6bvhpazG1qwAYoALbq6nK5uZmvv/mapcuWsSV6C5r60rJ4c10zq91W021uN0muPvi7wdz8/iYZFzPIuZWj8pmVfsU1hTU82P4A+zB78Z7Tt9Qn7JUwcV2y8LTglUevqOzr9zbeA+Clu1ISa3VuNT97/izmeF2GdBHX9cKEQpJ2J+E9xpvbq2+TdS2L92qkU+6aupriNKoS7eueCVsTALj+zXXurL/Dy/dfZt71eeTezuXMUmGiVan+IxlOUEiPY+1vLe6ZSXuSSNyRiM9YHxx6OFDTVMPFiosMnj0YHU0pkazsaRm1RbU49HCQ5B2LnyxGriHH1M1UrMnpmugy+IfBnF1+ltrCWmRyGUfnH8Vvkh9hC8PIuJBBbmwug38czIrvVnA38C6Z055bGRhYGTDkhyHc/vk2u8fuZuSGkTTVNmEbZCvmKH/2/ZOsa1mcf+c8ldmVog2DhrYG1l2tuff7PUpTSkU1p46YFTOL1iahQXtg6gGGrh4qafbVldRx+dPLGNkbqeTm7gPdWZqzlMaqRmoKa0Riyhd6X+DYy1HMEfZN3EdjZaP4fcnkMrpO6cqjfY/IuJTRqSoJCOpHO4btIOqTKLov6i7kmtGuuESq98v9G/95+LvZ9zf+IyCTyXByckImk+E53JPsG9ncXnMbr1FemLmZUZpSyobADeLroz6JEosBjw484vhLx5lxZobIJL/65VWaa5tVmn3ahtrk3ckj9udYQheG4hTuxB99/qAys5JluUJzsX0Q7T7Qndeevib4PMTlcfq10+J0XEf8aP8j3qO9ebfqXZUg3jXKFdcoVwoSCtgYtpE+7/Wh+6Ln0paWPpa0NLTQXN+swjKvKaihLK1MNMwFIZErTS3FwMqApD1JXPrwEh731Tf74LmcphJK+ZHUU6l4DvNU8Y5q7wfg0tcFa39rCZtSiZInJVz59AoAN8JuiIwfQMJYvP7NdVoaW1SacwbWBph5mAnsyQ4FtJaGFvZO2Eu3ed0Ys2kMGZcyqCuuI2R+CN9ZfYddiB0zzqhvXvR4tQelyaX8YPsD47aPkzRz8uPzOf/OedFHpW1HGxry5388/UI6aafTOm3SKCdglLDwtGDulbn8FvobjVWNf1nkNLIzYq2PwBzsqN9d8qSEh3seMmn/JNFXpj12Dt9Jzq0cJu2fRJchXTqd3lKH5jqh4NxZkK70/+jog/TPoCBeKB6rMyEH4d6Waar3kAiZH0LI/M4n4HYM20HenTxhAqYTHwp12BK1hbI0IWm3CbKR3NONVY2s9VqLU4QT9t3tVbyMAKqyq8TiUO93eqNhIr04J+ycQGtz61+avj/c8xCAIT8NUZkqe3zwMWWpZZh6m6oUUM+9fQ7fcb6ijEp7lKWVcXD6QUJfDlU7xSuTybiz/g5+k/z+crJQHcKXhVORWUH+nXy1z5t7mv+3pEX+pzB1NeW92vfUSg0CjN06loaKBklxUslcHfCV4MWxsftGIt6OEL34Ln54kYd7HzL4h8Fs2CItQA38eiANFQ3YhdhxccVFLp++TOR7kcy/NR/Hno4cmHqAiswK+qzog2MvR2JWxNDzjZ449nSktqhWLGqD4JcasyKGCTsnoKWvpTJ1oamnqfY+fnzoMZlXMtHU0SRgegA2ATYq3n6jN43Guqs1emZ6GFgZMPPcTO5suINcS46iTUHx42Jq8mtwjXZFriGnpbFFkPVxNUXHSEelwX33t7tcX3udB5EPaDVrxfI1S4nG/6iNo/g15FeOv3wcgICpAULR+x9f+6wLs8iLy2PXqF0Ezw9m1K+jmHV+lvj+medmqv39lIj/Ix4tfS2J56lzb2eOzjsqFv2cIgSz+hcOvEBNQY3Y/Lz+9XXeq3tP7YRWS0MLJxadwLmfs1oPjkcHHnH96+sM/2U4AdMDsAuxkzSLtg/eTuGDQomvkBImLib0fb8vV7+4Sm5sLq+mvoqGtoYo57I+YD2tza1EvBVBXbH6ZrWJkwkL7iwQiTyFDwq5t+kelRmVJB9NZuGDhSLBQ8lMdwx3ZP4NVeISCI3u/Lv55N/Nl8QqTbVNnH3zLDaBNrzy6BVMnEw49dop7v95n3er3pXITA35aQha+lpqJwPKUstI3CFMJNeVCPux53BPer7RE0tvS86vOE9LQwuuUa6YOJtIioK5t3M5FnmMum/qiFgmLUacf/u8cPyUMjwGeUjISw49HLD0tuTqF1e5+sVVrPyseOWh1BcIBOZvx8JkeXo5B6YewNjRGO8x3sSti2N5wXJ6vt4T+x726JrqUldaR+a9TPTr9bmcdlml2afYpuDNzW9yud9lkoOS6Tahm/hcW0sb3eK7YVZuRuSVSNIHp4uFmZs/3MQpwongecHI5DK09FQLp0p0nEYC6P95f/St9FWIVErEvBfDvY33eLvsbbHgXPxY8Essf1ZO+bNyTJxNRF+rKUenoGiVXjNK6XFdE11JE0spPXRnwx1OLDrBrJhZ4hSmksn8atqrkqlnJXq+2hPf8b6Up5cDYOFlIU4wJO5M5PTrpwl/M5zm+mbaWtoY/stwmmqakGsIah1trW2ceu0UdqF2KvJRWdeyuLP+Dok7Exnz5xgsvS0JnhdMXlwe3eZ0o66kTr3M606hINx3RV/aWttYXrBcLLbK5DLmXJojeX3qqVQMrA2wD7Xn7sa7VOVUEf1JNE21TTTVNKFnroeGlgYZlzNorGzEa5QXRnZGzL0yVzxGwrYE7v56F98Jvqz1Xkv0Z9GSItz1b6+LU0RH5h4haXcSDj0dJB6eAGeWnhH8wEKl3m3id1KZhX+qP5e/uozVXivJ9RL1SRQ9lvSQTNmaOJvgPdqbZzHP0DXVJeNSBvG/x9P73d7CtdJhHR34zUD6vt8XuaYcYwdjMcbd2H0jhYmFtDa20n1Jd0oelQCQeTmT699dJ+bdGPqu7EvEWxGYe5gLzX1lDUwhfObRf4wWZJS9LWlraePJ4SfE/xFP2qk0XKNdmbBrApEfR3bqW/nSvZdUYsMTi09w55c7eAzxoN8H/ahIrwDAqbcTbS1ttDa3StYX90HuKtL4vZb2EuNmEOTFGqsbmXxwMtqG2pJY+e7GuzTXNtPrjV4Y6hry9MxTTiw8wZSjUzB2MsbJyYmMCxnsf2E/DRUNtDa1cueXO9iF2GEfZq8izZh1PYutA7Yy+IfBBM4I5IWDL0iaN4UPCkUFEKW09O+9fif3di4zzszAPsxebF7d/+M+0Z9Hc3HlRb42/hpdU13eKX9HzAWUE56PDzxmtdtqbIJsmHJ4Cl2ndBX34k81P8VrpBeDvhuEolWBpY8lJ145gVxLTvdF3dEy0KK2sJaN3TfS/4v+WHhb0FDRQOTHkRyZc4T4TfGCbFs7lKWVkbA1ATN3MwJnBornu+DOAurL6iVqMMoYubGqkeLHxTRVC7loxqUMrnx6Bf9J/pLpPW0DbYb9PIy4dXEqRM+YFTEY2hliE2BDfVk9a73XSqQh3Qe688qjV4QY6mGxQMD4R/yecTGDC+9fwGOIB4a2hirSkYdmHRLJnMPWSKePlXCNdKX4UTFTj01VIQ3kxuUi05Ax8teRndoD9HytJ5Y+lmrj++wb2Zx+7TQzzs5Qm+vdXn0bQ1tDlmYvFR9TtClQtCrUToep88MGaKxuZJXzKgJnBTJs9TD2T9kv5jvhy8MZ/L1UtaGutI7GykZee/qayhQUCJMvneUWWvpavFP+Tqfxf8RbEXSd1pXSlFKufnFVlJyuya8RCT0diSMegz2E6UkzPerL6gmaFSTuBe09VZVytsvylqFtoE1jVSN1JXUY2hlSlVPF5Y8v4z/ZX8V6BYT6h7qpTSVCXwol53YOZ5adIfSlUCx9LFEoFPzoIEy2Rn8WzcCvB/KlwZcUzywWp5IyLmcAYGhnKP5b11QX2yBbrn55lYd7HvJBywcqxLa002lc+ugSsy7MUqlR2He3J3BGIDINGWO3juXRvkdc+ewK3V/pjqJNwdcmX+M/2Z9ZMbPUqsJ0RPt63r8DS5ctY+q0aby5/HeWH91Jc3Ml7m4mjJ0+ja0vvoitrbCWzL6oOqEn15QT/ma4ihQnQMD0ABJ3JPL44GO191PW9Sz+7PMnAEPXDBXXaBNnE4kPtqauptpaQb8P+hGyIERlb9M20hb24kAbHHo4SNYC/xf88Z0gTNRqG2pLahtrfdZSmlyKho4Gy/OX01zbzLZB2wieH0zEmxHiGjz8l+EkH07Ge4w3Vn5WGNkbYWBlgEN3B84sPQOgVlFo1G+jVB5Twm+iHz4NPmJ8u+L8CtbGreXTqE/5IPID8XUZlzI4MPUANQU1Ql6vL7w+7XQaN3+8Sf8v+quoWHiO8MTQzhALbwuqc6vJuJhBxsUMAqYFMOPsDGLXxrIxbCM6RjpUG1Uzfv14jtw7wpg/xlD8qBgtAy0e7nlIYUIhaafTuPzJZaYen8qz88+w8rNi7tW5VGZVkn4xXTIFq5Rj1THWQdtIG6cIJ5L2JGEXLEzupZ1JQ0tfS4yL60rqmLR/koqS2d6Je2kob+DleFWVFE1dTYzsjfjJ8SeRcKNQKNDQ0SDnVg5rPNYwaf8k6krq0DXTFX+XzCuZGNoZMvnQZCx9pTWsyqxKYtfG4jXSC5d+Lmjpa2HlayXmkfZh9v+yd+rf+Gv8u9e8jvi72fc3/iMgl8uxsBCywMgPI0nYmsDh2YcxdTXFzM1M4msx7cQ0ideRXYgdkR9FSjaIUb+Nkkh75MbmoqmniUwu4+nZpyRsTcBjiAdO4U64Rrt2OjWipa9FbXEt17++jqGdIUN+HIKFt/rJgbBXwrAJsFHRZm8PDW0NjByMVAq8jdWNrPNdh56ZnoosYElyCcdePMawn4eJzb6q3CrWd12P92hvLH0t6ft+306bPz85/YSJswnzrs8THxv47UAurrxI3p08Hu1/RK83eonfaXV+NYk7E7EJtMF9gDsyuYxjC47x+NBjPmqTBol7xu0BmSDj2dFs3rabwJxpa2mjIrNCJXjYPnQ7KGDetXl0ROqpVG58d4Pw5eGibFtFZgWPDzym29xueAzxoCa/hi8Nv2TwD4NVfE6G/DiEgoQCTN1MJQwbEKRZFsQtwNTNFD1zPZXFecwfY1jttprTb5xm6KqhkucqsytJ3JGI+0B3yWZp7mnO7Iuz/9JPAYTfraG8QW3hwnu0N29kvqHCflXCZ7wPObdy2DdxH0uSl6h8rr/CoG8HdWoYD5ByPIXHBx5LvKOUqC+v58b3N9DUEdhiHdmIJclCgUedJ2BbSxt/9v0TbSMh+VbXWKspqGFDtw2ELAhRYSeVp5fjGu0qytL9s7DytxKbfSN/lbIfSx4L52sdYK3y+yrhMdiDaSemUZldqVZaS52kWkf0WtaLO+vvCIztDs0+5XTNkK+HIJdLk9gXb7/YqXRnS2ML1XnVahvvAKdeP4WmnqbKZ1ZCmQzMuzFPbUO5ozxae8w4PYMj845wePZhtQzb/02oY8AqobyHm2qaxDVw0PeDOPfmOZJ2JxGzIgZjJ2NJ4pu4M5Hyp+V0m9ONphNV2Dy8QbWFK2CPQw8HHh96TPGjYhJMEtD212bxmMXivd3vg36kHEvhyqdX8BzhKTRqbAxJPZnK/sn7mbhnorhmaelrYWhriIa2htpm14hfRqg8BoLcXtZVYUrPLsQOmwAbhq0ehmuUK3vH72X8jvGSCYdtg7bx7Pwzxm4dy+FZhzm15BRuA9xIj0nnvbr3kOvJKYgvYFP4Jhx7OTIrZpbKd9ra1Ep9lSAz5ejtyOJF0uLcs5hnWHe1xrmvM/6T/NEx1hHZ2LZBtrhFu+Hc25ln55+pbQAovY7mXp2r8hzA5U8uo2WgJSSfDkboW+hjaGeI3yQ/Hu17hN8kP2QyGWln0mhtasXA2oDRm0Zz+ZPLVGZV0tLQQt6dPLKuZhGyIERk0bY2tXJ/832QC+zkuHVxzLk8R5wQbKlvoba4lrbmNpUmEwhyPRmXMjgy9whj/pQSQMw9zOn/eX/sw+xpaxWKfGffEqRHl2YvZcJuYbpFXdGnpaGFmz/exDbYViJLc3T+UfLu5NH3/b6ELAihNKWUo/OOMvCbgbj1d5MUarb230pNQQ2LHz//rRrKG8T/bt8MbKlv4e6Gu4QuDKXbnG6CZ9V4X8w8zFT2o86aSyBMrQfNChK8bf5xDWnpaYl7x89eP1OWWsbtVbcBqXSpvrk+niM8MXUxpSq3Cm1DbXFtVMpFd+aBEzgzkHNvncPYyZieb6gKNt3ZcIfr31wn+vNo3Ae4Y2hrSFNtEze/vwmAkYMR3eZ0w7mPMzrGOvhN9BMLJSb6Jnz7zreYVJiwMEB1yrb/qP68+/BdIq9Ecpe7YrOvNLUUIzsjxh4ZS5pHGudnn+e1ga+J71uUuEgsUtoE2rAw4fmxK7MrKXxQiEN3B5Vi5N3f7nL92+tMPjgZm0AbLqy8gKWPpUpR0WOwBzomOpI9ymOQB3Muz8HExUQsoCrZwOkX0qkprCFgagCVWZWcXHISh54OXFx5UUKGAEQptcrMSvwm+olknsbqRvE1qSdTUbQp8B7lLYl9DG0NQQYbwzbSfXF3CbHJNtiWyI8j8Z/kryLlq2hTUPigkF+DfwWg65SuKs2+2LWxPNzzUJDA+4eEu4WnhYRYcHfjXb6f9j1Tjk4RpyrG7xhP9o1sTr56ku6vdP9Log7ArlG78B7lzeRDk3m09xF5d/KI/iSa26tvc+H95xKz+ybto664TvDLic9n2JphYoOhtrCW7OvZZF8Xpik7xhLJR5LF/07anYTCWqFWzrzPij7UldTx+Pub1BtZI7N+vkaVPS1j5LGRFFsVkzo8VdIUqiutIzc2V9IYULQpaK5rJvmo8LfX+a7DqbcTTw49IXBWILomQvPbJsBGLNwZOxiDmoHoIauGUF9aT85tYfIh5MUQErYmEDgjkE3hmwAhrlUWUqvzqmlpaMHM3Qz3Qe4MNxyO1wgv8fpvqmli36R94t+18rcSmjjtls/9U/Yjk8vEqSl1+ZZyLcm+kU1dSR02QTb0+7AfXSd35VuLb2msamTq8akimaHjdQhIJC8zLmeI0xBH5h6hqaaJSfuex6T3frsn+Bb+I/5wiXRh5rmZ2ATZ0FDeQOnNUp4cfkJDRQNTj03FKcKJviv6isW3+vJ6Mi9n8vjAY3q/0xs9Mz08Bnlg4mSCrokuvuN8Jedm4mLCwG8G0lLfwvqA9Qz6fhDB84KpLapl+5DtzLowC+fezmjoaOAa5YrXSC9aGlooSy0Tm1Gtza3I5DJsg22Zf3M+Mg0Zv/f4nYqMCurL6iUKIt3mdsMm0IZ9k/bRXNfMqymv8vTMU8qflRO7JhaAyYcmE/JSCLbdbPEc7inKgypaFRJvpur8au79fg9tA22ufCYQRd36u6FtqI22kbY4wQTwbuW7tDS2CL7MR5JxinBixC8jRA8q7zHe1JfVi9YTIEhgpl9IJ3xZOO6D3Lm38R5trW049nTEpa+LkPv9Y+/X0tei78q+kkK/lp4WeXF5HJ59mDGbx0h+5+B5wXQZ2kVtHgMwad8kDKwNcAp34uHehyKZVom60jqKHhZR/rSchsoGlWbf8jzVZn5ZWhmlqaXCvmWko9JIAyHGaWtpw2uUF3N85qglOwDMvjBbRWGl91u9RQlfJS58cIHkI8ksiF2gNv95uPchDRUNVOcIky7K5isITeOO55hyPIUjc44w9dhUtTlydV41yFDr+y6TydA11aW5vpk94/fgGu1Kz1ef7/9K8m1jVSOLnywWGyPug9wF/+BWBdpG2nyh9wXD1g4jZH6ISBIuuF/AtxbfEv1ZNL3ekE7cJGxLEJSJZIh1j7sb74rEcqU0u/8L/mqJmf8MipKKuPXTLTwGe4jrts9YH54cfkL503JybuUwbts4yXf2UdtHFCUVYeYhPLZj2A7cot2YdmIaIS+G4BrtqpYYm3Mrh9zYXMqflqtYnwTNDCJoZhAnl5wkbl0cL8e/zMBvBqJvqU9LYwt+k/xwiXRR8eXtDO3ref8u2NraMnnKSiqrVgJw5AgoU+uS5BLy7+bjPtBdJdbS1NUU1/+vTb7Gc7gnE3ZNIPtGNmbuggxwRzl6EKx5Ti4+Kf5bXc6jRFtrG3UldWjpaUlIqTYBNqAQpHXbE8j1LfQlsp4doYznOpLn7YLtKH9aTmtjK9+af4vXKC/hfu9wOXRf1P35sMEQIU9oaWhBrilnVswsnp1/xpXPrhDyUggV6RXYh9mr9bVtqmkiaXcSVn5WEusDAE25JjYFNjRNaeLuJ3fFydqzb56lpqCGwT8MlhDfaotqeXbuGRUZFYQvD8exlyO2QbbE/xHPta+vMfLXkehb6KNvoY+ljyUlT0qQa8qRyWQigc1/iz9Pa59i9LGRQGJQKPjF/xfxb3gM8aDr1K4Y2hpi3dWaXSN3AYK3YI8lPVRiTaUca/8v+7OiagWZVzI5MOX55N6RuUcwczPDc4QwoDL12FSJp6sSfhP9cOrtpFb5p6mmieq8akJfDsUxXIhVZTLBD3nXqF2UPyvntxDBdqG9d+fmqM14j/Jmwq4JaOpqcn7FeVobWxny4xCqcqu48d0NcfrcyteKOZfnSP5uY3UjrU2tatV9/sa/jv8ba157/N3s+xv/EWhtbSU1NRVPT080NDTwGCIULJQG5e1ZX+0bfQD2ofaSBAFQkYL4vefvgMAwlmnIWFGzAk0d4fYY8MUAjr10jM91PkfbUJvAmYFYeFsQMj+E1qZWHu1/BAhJctcpXdk9ZjcZlzJ4t/Jdyd9Qjn7n3c1D10RX0hC4t+keidsTGbt1rAprF2BTr01U51YTNFtVFtAmwIboz6NJOZ6CY7gj9qH26JroEvlRJHlxeVz/5jog+AS8kqTKdvcY6kH+nXy2Dd7GwK8HYhdiJxY17226x6UPL+E30U8sSJY8LhEl2d6peIcH2x7w+OBjAmcEil5/SnRf0h1Fq4Ker3VujyzXlKsNhHSMdNROJoCw8effzaffyn7ib9ltdjdqC2vJuZnD+O3jKUws5Pzb59UmCCCYr3ecngPEgKEzaOlrYRtsKzLn2qMio4KYFTHErIhh9sXZuEa5UltUS8rxFBx6OlCRWcEql1VYB1iLet7tsWvkLrSNtNXKZ2kbaFNfWs/O4TsJnBWoMvHW+63edBnSheLHxWrZkf8TjPptFLqmutzZcEeU7VOiLK2Ma19eA4SkXylf0v68lLI2HaFoU1CYWEhzbTPXv76uUizPjcsl9USqOJ3UEUueLFF5TImKzApOLDxB12ldVXwwJ+yaQEN5A7pmuios9cbqRjT1NKnOrVZrYg/CNdtxnWmPhooGTr9+GrmWnNG/q3qrAQz5YQhDflAvkxb1cRQR70TwLOMZFq0WaGg8L7Y7dHfgxvc3uPbVNeZenSuRq7P2txakfTohD+ma6GLtb63iv6REaXIpgISxrkTxo2KKHxfjMcij0/eXpZap9W38nyJpd9JzSbUOSNyZyMHpBxn8w2DCl4WrPP/k0BP2T97P0DVDxSJAxPIIIpZHkHUti4RtCfR8radEJra2qFY0w9asqcTh8Tmy/QYDwj5yaMYhHMMd0e2nS5FlEf6z/DGwFK7PoFlB+Iz1IfloMrm3c8W9x9LHkoi3IyTFVt9xviqFun8GQ1cPpbawFvMu5pL7yrabLSN/HSlKODbXNaNnrsez888AOPXqKfG13V/pjt8kPzGhUh4n51YOlz6+ROaVTIzsjZh8cDIgJBJVw6so3VaKPaqsv9ifY5Fryhm2Zpi4B9z88Sa3V99mac5SNHU1xWKmcm+NXReLoY0hfhP9xGLpgWkHcOvvpsJMn7R/EllXs9gQtIER60cQtjCMix9eJPVEKuHLw+n5ek8V7zBFm1BIjPw4kq0Dtgr+VTHpeI/xFtcTbSNtVlSvQK4pJ/6PeAxsDCTFkMAZgWqZ2Ur0W9lPkF/bnaSyfinhM/a5LFV6TDo1+TVkX8/GpZ8LzXXNfCL7BNcoVwmzuLm+mQvvXyD4xWCc+zgj15SjpadFY5WwPvVb2Q9NXU2SjyVTW1wrFrAPTD1AyvEUZl2YRUtDiwpD02esj9hgu/H9DXJv5xLxZgR65nriFMi2QdvIvp7NR4qPOpXC3j9lP20tbSoernINuTiBpUR9eT0N5Q0Y2Rux8P5C8u7msbnfZpVjmnma0e3rblhgwU+OP9FnRR/RY2rinomUJJegbahNyokU7m28x4AvB2DlZ0V1XjX6lvp/SXJpaxWk7A7NOMSQVUPo9XovGisbSdqdRNiiMAZ/PxgtfS3x+zqz/Ax3frnDa89eE2OIStNKmh1U10bfsb48zHpIWc8yNk8VPlfB/QJ+Df6VFw68gMd6Dzbc34BLmIvEy629T2ZHPDv/jKPzjopNh8wrmSTtSaLX64JniI6RDnJNObdW3eLqF1fxGeujcp36TfTDc4QnubdzMXY0xryLObqmuhJJ4/Y4+uJRKtIrCJgaIHoqm3ua0/ud3irvqSutQ9GmUJleqS0SJoy7DOuCXEPOyddOYuZmJilINtU20drUytA1Q7ENsqWmsIaELQm49HNB10yXyx9fpq25jf6fC+Se+rJ6UX5V2ejr90E/tTKy4cvD8Z/sj8cQD3EdamttExQE/nFf65npYeljKdn7rbtac/7d86I3GAj7pZG9Ea1NrVz98iq2wbaixNzo30eLMtrjto0T5XjtQu3ovqS7WKg39zCnrriOxwcek34hHV0TXdz6u+E53JOINyMoiC8gcacwCaupp8lvYb8xbts4rHytmHZiGnl382hpaGH85vHkOuQSOV5VXcF7lDcNlQ04zfuGYqdg6vye/x41BTWE3Q3j9JDTaL2kJSEWnHzlJA/3PkTfQp8H2x5g311gdCsnpZW/p884HybsmoCGlgbHXj5G/O/xkmnRhsoGWhpaMLA2oCy1jJ0jd+IU4cTYzWOF82s3kWr7/7D3l/FRnV3fP/ydiUzc3T0hIUaQQAju7t5iRUtbaEuhpUbdCy1WSoXi7u5Bg4ZAEkLc3V0m87zYnU02M5yX/K/7up/z/nS9gkxmMrNn72OvY/3se4HVPmLTCGryayRAyrF5x8i4mMGkg5M4PPMww9YPkwxfdRQ6jN89HgsPCxGkbahooLmmGTMXM2RymWDlpSPjXNo5iuuKGaQ3CJVSJaqeAUb/NprRv0nXavVxce3hSurpVI4vOI5cV87rqa+/UG2rrsaKRgytDRn12yjubrxLdW41f0T/gd9IP6LeiWLC3gkom5VixMSUo1NEe9u0C2nsGrmLPp/0EdYaJ1N0FboSMG1r360UPRQIrU3VTUw5MkWrhaO6zJzNiHonipRTKVxcdZGc6zn0/aSvYOV9PRsbfxsq0ivo+W5POozrgMJUQddXu0p6iW+svsG1hyszzggqsDZlm2gbp2pTUZZSxqEZhzBxNGHK4SmAYFWqBtZeT3ud6txqtg/ZTktdi3DtjJHaMzbXNRMwNoCkA0lsCt3EhL0TOLn4JBkXM8TIA0DM5p11ZRa2gbboKIT1T2GmQIGC5KPJxHwqkKtSTqQw89JMjPsYU19Sz+2fb2MXbCcqyLNisrj+9XXce7lTX1rPze9vYu1vjUs3FwwsDCT3S31jfQ2CodryXa4rl7jhgNBDtT+Gj3Y9Iu6POIZvHI6Vt5XEnvL0G6ex9LaUgH2lT0qpSKvAJdKFRzseET4n/D90y4jfEc+Vj6+w6PEisYd9uO0hBfcLGPzDYGQyGb90+oWShBI+Un2EqaOpQBy2MpQQfEEgKdaV1PFw20OcIpxemGmvbFLS2thKxsUMyp6WET43XAKqK8wU2HW0o/vbQj8+7cQ0QOhptCkbHDs50vvj3iibldTk12ioFrf220pZchkKcwUjfhkhIbK1Kdsoii9Cz1CP5CPJLyRCKswUkn2LTCYTz9uCBwV49vcUHYvKU8tJOpiEfYg9EQsiRCBD1aZi16hduPd2F9XAHcZ2oDS5FIWZAvsQe3q80wMLDwuiVkShMFfgP9Kf6txqje/x+rfXufT+Jaz9ren+ZnfCZoVJHt8+ZDsG5gYszVoqEmlkMhmTDk6i+HExm0I2Ye5hrjVTq/1aN2TNENF+2muAF14DhH1N0qEkYtfEMnjNYBzDHfHo6wEyYe/3veP31BbWajg6+A7zRa4nx8TRRFTK6uvq/5fJts/P8/43qrCwkL17fuXO5Z20tFTTNciMMdOm8cq8eWSeyuTMsjOEzQrDc4CnhquJutyi3bAJFPZvlz+6TPr5dIatHyYe0/alZ6SHUxcnXLq7kHI8RdL/lCSVcPnDy4TNCcN3qC/VOdWs9VyrVfV66KVDtDQIBIrnq7mumYayBoxsjERiXXlaOTX5NTh1dtKYbYzfNZ7xu8aLzgv6JvriTLA8tZzUM6n4DPHRmMep2lR8bvg5fiP9RKvuxP2JOHRy4OJ7F+n7aV8cOzny5MgTGsoaRFehpuomjs07RtfXumLla0V5Sjk2ATYYWhlioGtAm7wNjJHEHvX9tC/KJqXGvSL05VAOzzxMeUo5JxaewKOvBzMvzqS+tB4jGyOJPXB7cmP+3Xzcot2IXBpJUW0RYc1hWN+yxtLQEpVKRa8Pe4nuY53mdRLtV9tX7NpYYtfGMv/efEEE8HffY2hlSMTCCAwtDbm19haOnRyZuG+i+F6G/jwUfWN9sZ96kaqr65Ku1JXUUVtYqzEnSzubxt7xexm3c5yEYKSeO4PgSGAfYk9FeoVobz70p6E8+O0BXxh/wZKnS0g9lUpLXQuDfxCu99fTXxc/x/O1d/xekg4mAZo58v/Uf6/+b6x57esfsO+f+repxkaBkX7po0som5QSOwWP3h6sqFwhMjiOzjtKc03zfzo/auK+iajaVDRUNCDXkWvkIjzc+lAYNih0eHrsKRVrKwieGkzy0WRufneTwAmBuPdxp6mmCccIR62Zfera0nUL/qP9xSEqCBvyovgiMYT9+eowoQMBbQFafZcNrQyx8bfh0vuXCJsdhlOEE4ZWhvT5uA/lqeWEzw0n83Km1hwRgFG/jiLuzzhOv3GaxkrhGF/7+hqtDa1EvxdN0MQgTi87zbm3z7E4YTH2ofaEzQqjqboJHT0dqvOqMbA0YOB3AzVuZkkHksi8lEn4nHANZWFWTBZ/9v4TQAyzbg+yTdw3keRjyVz59Ao9V/SUqAvCZoZpsGzUfy/vdh51JXWM/GUk3oO9tdp77B6zG9coVw3GIgiMVmWTYF1VV1InDFrbMZKqsqsY/MNgrTJ3x3BHsXlQH8uylDKOzj0qWJ/9nTVV/KhY47kAIS+HaNgFqaupponS5FLy7+XjPUS7Jat9iL0GK+8/UxkXM/ir/1+AoEBtbzEBwjn25NATKtIrNMA+K28rer7XEz1DPa0gqY5Ch7ub7uLZ11MjR0JHX4eVlSu5vf621o2l2lJhQdwCDesDdRXFF5F2Lo2giUHS4PYmJVlXs3CN0lSo6RnqiY1wTUGNBBD2GezDqvpVrJat5unxp0w/PV0MsVZXQ0UDlRmVWHhYaM1I/KPXHxQ/KhbJCNoqfns8lt6WWhV0APc23aO0thS/Vc+G7SqVSsxusQ+118qq3TVyF6bOplpJA/0+60fw9GAqMyu1AuyrGleBSlNhAILnfswnMRrZBeqK+TyGLq92kdgs/k+USqXiwNQDgPbGU99UWFfOvnVWK9h39u2zgHQDrC63nm5aLZeHbxyOkbURv4T/gkWlNUlRc2k2NKdN2cY31t/QUt+Cib0JCz9ayNaXt6Ijk673CjMFMy/NRNmiFPJsdOXYh9gzMOTF6tm7v9zFyNpIco0kHUqiJq9GwtgDtGZTZl/L5sC0A1TnCOuxfYg9WVeyeCPzDd6tfZfdo3eTcSEDgA9aP5CsaSAAD6N+H8WpJadwiXShJKFEQ3GqJl+Y55vzW4/f6PZ6N/H7NrE3IfdWLm2tbcjkwmDdf5Q/pk6mKMwUYkZOe7u/i6su4tTZicAJgVj7WwtM0F2P0TfR1wD7nLs4Y2xrTHNds7j2VmVV0VLfws3vb9JxakcNsE/XUBdjW2NUShVN1U30WN6Dgd8OlGy8ZTKZeG/qsrgLXRZ34UV15ZMrJO5LZPa12ZJj0/5e3r7ybudx4b0L9Hi7Bz5DhDVk8ePF/OTzE6deO8WcG3PEniXzcqbkuQpTBfPvz6e1oZWvzL6i29JuDPlxCPPvzadN2SZeo+1z0fJu5/F492NAGJy31LUw9+Zcjfd1b/M95Hpy4v6IQ6VS0ePtHsjkMoxsjEjcn0jO9Zx/qZbdFLaJoodFePbXZHLXFtZSkVGBoZWhQNKwMybuzzjOvnkWQ2tD3il9B6fOTgzfOFzrmt/Y2IiBowHdlnaTDFPbWtvYELiB4GnBwgDlRIp4vRc9KuLI7COM/HXkC8G+rq92JWBMAI92PhKveWN7Y97IfAM9Iz2Nz5t2Oo3WxlZUShXNtc0EJgRSY1qD0lfz/qwr1yVzaabkZ1U5Qu5PU00Tw2cOx3+kP6aKZ/cZlUpFVXYVClMFhlaG1JfVE789Hueuzrh2d8U92p3Rf4wW73vFCcXc3XCXwPGBhM0KI2xWGFv7biXzcia2QbYMW69JXAJBPba171a6v92dQd8OorWxlYqMCs4sO4NMLqPTvE4UxhUSMj2EuiIBqEvcn8i+ifsYv2s8ltEduXkTej7Xeu4Zs4eyp2W8XfS25OemjqbMODsDC3cLDCyEdeh5G6HLH13m5vc3eSPjDSw8LETr9H5f9CNsZhgRCyIwdzPn3uZ7uHR34fw758mKyWJ5yXK6vdENl+4uWjOTQVgnnrfcivkshisfX8ExwpG2ljaCpwdLmMytja3sHrNbJAI4hDrwg/MPYu5Ra1MrV1ZfIWx2mDiwbz+cbT+g8RnsI+kXRv46kor0Clx7uNJQ0cA6v3W01LeIgKJ7H3ce7XxEj3d60NrYSkN5g6iuMrAw4PpX18m7k0fCMsEGr7H1mTK3fekb65PUYw6tCmPaj5VdurlgdMqIu9fu4qeSAvc93ulBwt4E9k3ch56xHjoKHQLGSodsrz55VTL86jC2A5aelpJB4qklp4jfHs+qhlVUZVdRnlJOeUq5CPZpK/Xxy76WzdXPr9Lz3Z4ETgrEMcIRhakC20BbDMwNSD2TyqX3LzHgmwF49vXU+N7PrzzP/c3PrGrVhAnZamEv8uO+H9Fp1tE6LNVW009NBwT1UtrpNFRtKq58ckUgjXzwDGg99NIh6orrmHFmBgFjAsThZMDoABqrGlnnt47aIiHzTw3wR6+KJvdWroQIYRNgQ/jqcALGBmBoaUh1TjWGVoJ9oJGtEQbmBgRODBR6WxVELJDaJubdyWPbwG30+7wfXV+V9gm+QwV7s/Tz6fT9pC+2gbbimpt7K5crH1/B2s+awzMP4zvUlylHpojPDZwYiLWfNa2NreTczMHMxUyS/RS3NY6823mSv9c+9wwEEq42gmlxQjG5t3JJ2p9E6ulUBn47kJaGFlRKlWBJVlKPmasZQZOCyLyUibm7OYZWhhjbG/Ot7bcETQpiwDcD+LXLrwRNCqLvp31x7eFK2dMydBW6KMwFgMLaz5pxO8bhFv2sx+v+ZneCpwezIVBQdMy+NltcY9V7P10D3RcCvMlHkzmx6ASTD0+mqaqJE6+eoNMrnXAMd6S1sRVlsxJ9E31kchk1eTXkxeZJnDYKHhSISoyJ+6UgiY2/DciE7yb3Vi6e/TxFkKi1sZU17msYsnYI17+5jm2gLeO2jxP3kmqC3u4xu0VF8KDvBiHTkWHbwZaK9ApaG1vR0dfBpbuLVsJeXXEd+XfzOfzyYQZ+NxDbQFuyrmZR/LiYsJlh4j1S7QRz6vVT3P75NkGTgySgUNDEIK3qf2026iAA7QbmBqxxX6MV8AibHSYcx5pmjT1mU3UTmzttJvyVcD5Ufij+vK21jebaZk4uOUny0WSWFy+noaIBA3MD0ZHpyaEn5N/Np/PCzkw7Pk08Bj/7/gzA6D9HM2LTCE4sPkHivkRG/jqSjAsZGFoaMmLzCAZ8PQBlk5Jvbb+l0/xOjPxlpEhCUM+nfuvxG1XZVbyZ+6bkfRuYG2Bsb0zxo2KKH2vOAxorGlGYKiR72jZlm2A7aKTHwviFWnNrU06lYGJvIgKUz1+T7V+/9EmpeN6Y2JvgM9iHsqdlYq6vuu7/dp+c6zlErYgidngsrQ2t9FndByNrI/E6qUiv4Cefn4haEfUvVWfi32/Ufh/7P1E/fv89K1esoJeOHt83N+IAFD7J568vvuDT1av58J2PRYJJ0aMiCdjXWNXIgakH8BvhJ54jAFErokg/n87JV09iF2xHaVIpQZOCRLBZ3aOBpiNOfUk9ifsTxUw0hbmCLq920ZhV7Bi6g6L4Ikb9LiUMl6eWc/LVk8j15KScSJGQR+7/ep/rX1/ntdTXiF0bS/r5dA2L5BUVK2hpaJHMPfLv5XNqySkm7psozgYe/PGAm9/dZNyOcQRPD0ZHoSMKI15LeQ1je2OcOjuJvx+7Jpbix8Ui2Gdobcj009Ox8LAg9VSqmM3rP9IfA10DSuxKKF1TStiIMPF9tCc/PF/Rq6IpTSoVVHB/5whHvRMlycNsrmsm91YuFh4WWHlbcWTOEZRNSubdncdXr3zFCd0TzF00lxU9VyCTyei7ui99V/cl91YuRY+KJA5A009Np66kTlDwApsjNmPhacEb6W8Awp55xMYR3F53mzNLzzDwu4H0eOuZDbuanBz5ZiSVGZUUPChg24Bt9P6ot4YAYvvg7aKVcfuy9rOm57s9JXulivQKHm4VyLGTD0/GvZc7x145RtLBJAZ8PQCFmYKuS7pSkVFBYZxAJpt3Z544n9U10JWQDuvL6rm97jZuUW54DfAS+/Xne41/6v9b/W+uec/XP2DfP/VvV8mHk2lpaKHf5/0kbN32w7eqzCoaq4QL6+G2hzzY8kAyCNrceTONFY3iwtp+wNpY1Uje7TwsPCwwtjMm5rMYEYQLGBNA19e6UltQi56xHjYdbIh8M5LylHJOLTnFqSWnmH9/vka2mapNxb5J+3CNcqX/l/01bP56repFr1W9qC+t5+qXV3GLcpMwqdXh7y8qv5F+LC9drmFZY+VjReL+RJ4cfvJCezSQNiYAN765QUN5A/6j/HHsJGy+1TdAI2sjiYJhwJcDXtjc2QXbkXkpk6/Mv6LfF/3oueKZnYU6rwUZrHFbQ6d5nTQ8v58cekLcH3FELo3Uam35g/MPeA3wYszWMVTlVOEW7Ube7Tzub76PU4QTZ5adwdrfWtJAqFQqUk6kUJZcRl5sHp0XdZZYTMZvi+fo3KMY2xlTV1zHuzXvSoDK00tPk3Mjh3er39V4P/om+mLzoC7bQFuG/DSE06+fpvtb3VlRuUJrcw/QfVl37my8Q9LBJDqMkyp+kg4mcWTWEUlj177OrThHS12LVrXif1R3N90V/61NmVWdV83E/RO1DmcNrQzp/3n/F752XVEddzfcRa4r1xoaL9eVa4QSqytsZhgevT1eaIGRsDeBJ4ee8Hj3YxzCHCQbo5qCGoKnBYv+9e1rz9g9PDn8BHN3c6qyqiQMxubaZvaM2yOymKuyqjSen3kpk73j9wLwdtHbGpuuoElBBIwNeOF1q2xRcuilQyjMFbyV/5bGoPkT+ScAmHcwh1XPft7W0saXJl8SPC2YmRc18wVq8muQyWV0XtRZ4zF1/dHzD+xD7LXmE6gVV9oqcEIgMZ/EEPdHHKN+G6UB7Md8EiPYYPwPg32tDYJi6UVDfP+R/ozZOkZi1dS+ot6JoraoFs++z4CJDR03UJJQwrST03CJdOHkqyfxHuQtroHmbuaCnasM6gxrqLF0QI4uVVkVNFUJNnWd5nVia9JWvNK92BO8h1kHZ+ES6cLWflsxcTBhyJohKFuUfGX+lda1DYSh19PjT+kwrgOX3r+EbZCt5Bq5v/k+2deyNcA+lUqFsklJXUkdhpaG6Jvoc/3r61TnCBkkjRWNmDiY0HlRZ8H+ylif4RuHs85vHWauZhpAn7rCZ4eLuXXPrz+5sbkUni9Ev0mf9NJ0CtMKaap+ZtkX8lIIubdy+Uz/M0AgcHj28xSBPf9R/iQfTeb2utvUFtYSPC2Yl869JJ7743aMQyaXYRdkp3XQpmxRYuZqJhm4jt02lq/Mv0KlVPFr51/p/2V/bAJs8B7sTUV6BZfev0RlZiX5d/NfmNmnbFFSGFeIib2JZP1o/x3l38nHd5ivkOfUrJRYgF/76hoNFQ0Si0N1NVY1UnCvgB1Dd+Daw1Vk0g/4egByXTm/hP+CroEubj3dKEkskTxXrivHMdyRhooGwmaHiQp2HYUONRk1nF95nvi/4lnwYIF4bega6OLRx4MuS7rQ1tomggbtqzKzUsxVnHtrrjhMVLYoKUsuQ99Uny5LuhA2M4wb390g6WAS009OlzDmje2MJTlG7Stxf6KoIHWNcmXOtTkiOCuTyyiKL0KuK9cgjcDfeX/fPsJ8vrmGZXTSIYFpqmeiR9clXSXXhGMnR6x8rDg27xjH5h0jYGyAVgDWzNlMQvCR68gl1jkxn8Vwe91tXol9BbdoNyrSK1CYKagpqGHSvr8VjH8C0yQvy6+f/8rt/bcxXmHMmilrAIGANv/efMzdzHlc/pj8q/kkv5pM4w+NhM0Mo621jbUea+k4tSPjd46nrqiOM0vP0OuDXrh2d8XKx0rSJ4bNDCNwfKDke2iqacLSy5JJ+ydpzXC68d0Nko8kM+iHQeIQ8v5v9zm15JnCN+92Hg1lDTh3dWb0H6OpLarFzNWMwImBmLmasfBvZ9HKSnilHX+kw/gO1BXXkXMzh/jt8XRZ3AW7IDv0jPQofFBIVkwWfT7qo5VM4dLdhYgFESJRw9rPmlduv4KZsxnlaeVkX81G30Sfm9/fZPCPg/Eb5YdNoA16RnqitXbZ0zL0jPUk+dHqyriUwfZB2xm2fhgR8yNES862ljZaGlo0SHU6Ch3SzqQBQkyAgaUBvT7oJZ63ekZ6LE5YLIIIz1dNQQ2qNpXW92LX0U4kmhhYGLAwfqE4sC58WIihlSFvZLxB4oFEHMIcxEESCEM9z/6e1BTXYFVmReStSGqKamCD9G9sG7WNtOw0inymYtIkHRjKdeXomOvgUOiAw4cOpBqmimCkU4QTfVb3oSShhAl7nhEjX0t9jZ99hIG3qaMp9aX1lCaXYhNgg+8wXw1XA6+BXigsFMh0ZLj3dmf66em0NrSypdsW6kvrsfC0wHeYL3pGesR8GsPUY1NpaWjh4daHmLuZk3U1i/C54RICX9DkIE4vPU3Y7DCJchmeqflHbB6B1wAv9E30JfuD9o4gFlMt6Ggt7UmeHn9KSVIJNv42OHZypCSxhKtfXKXf5/1wixLO136f9hPJlY93PUYml0nuPeoMdXWlnUujoayBjlM6YmBuIAHB60vrkevKRaVqwt4ENgRtYMLeCfiP8cdtlBsmTiZc+eQKt368RfT70Vz97Cojfx1Jp1c60WuVNCO0KqeKW2tuCRanfw9c22fSlT4p5dDLh+jyahcG/zgYhamCo/OOomxUMnabkO2lJmocnHYQh3AHko8mc3zRcfSN9Rn03SCR0FWeWs5f/QQi4PRT01GYK3Dt7orfcD+MrI3wGerDwRkHcYl0kazJWVez0DfWF0GH6txqzi0/R8C4AGryaziz9Axhc4TvO/dWrgSMVSsOXLq5iOtW8NRgVG0qQWkV4YiqTSXabg9bNwxDS0MK7hUILjMvh+AY7sjtdbdFUoH42laGGFgYoKOvg7JZsCpV7+8S9ydycNpBJu6fSOD4QJprm9k9eje+w31FYoltkC1RK6Ow9rOm+JFAwPDo7YFjuCNXPrnCtS+vseTpEqx9renxdg8xg1PVpuJrq6/F/rHra13Fc01dRjZGuHZ3pTihmJcvvCwB2dPPp1NXXMeBqQfw6OMhvmffYb6Cav7vwX3qqVQ8+3sy5s8x4mxEbTV6e91tTr12SnSdeb42hW3CwMKAKUemiHu9hD0J3Fl/hw7jOkj2KSqVisaKRgZ+O1ArAHft62u49XTDd6gvSYeS2DtuL06dnRjxywitln8KcwV9PumjNfus/dzg+dI10KXvZ30lyh4QlPm3f7pN50WdcevpRuqZVPaM2cOYv8YQ+lIoF9+/yP3N9wFBsaV+T8pmJRaeFjh3cRZdXYofFdNUI5D2VjWskvydFt0Wot+P1iD/qlQqKtIr8BrgJSEc3/zxJmffOsvKypUvBOJAiGtorm2mPK0cE3sT9E30aa5tZlPIJuBZ9uGPbj/iPchbdJDZNWIX/qMEe+nn6/jC42Rfy2bx48WEzwmX5LrHfBrDo52PWPJ0CZMOTMIl0kXcE2ddySJ+Wzx9VvchYmEEGeczuPfLPV5LfQ0jGyPOrzyPXZAdHcZ2eKEa9P9W/fj993y5ahUxSiXdlNL7/vTGRmKBkT98htEXRsy8NFNjzqRsVpJ1JUtj/+k1wIthG4ZxcvFJ0aXCwNJAAnLXFNRw/evrxK6NJXpVtLj+u/dy58O2Z8C0oaWh1rmNSw8XDK0MNbLEm6qbyLmZQ8DoACLfjJSQF/1H+2PqbIqxnTEyHZlkL3Xpo0uUJZdh6mxK9LvRNNc2E/dnHPah9nj29eTliy9LHG7g73upDMZtH0d9aT1xv8cBAundysdKEmsw6IdBEmKDrkJX7DVUShWDfhgkvv7HVz4GICY7RuNzX/3yKon7Epl5aSYG5gbUFdcR+3MsekZ6jNwy8oWKNBDuM9sGbAMEm+eeK3tS+LCQW2tuYbHXAu9u3jQcbyCpWDpjS9ibwK0fb6FSqri15hZD1gzB3N0chzAHlpcuJ3FfIicWndA632iuEz6zNvcpQHRwKk0uxT7UXlTqquv8yvMUPiikz+o+Gs+162hH/y/6c3D6QcxczRjw1QDJjK6xshFDS0OeHH6Cka2RZBYVNjOM4kfFOIQ5oKP37LxuaWihrrgOQytDFKYKGisaufLxFaJWRuE1wIt+n/UTz9V/6v+N+gfs+6f+7Wruzbkk7k/kU91PRWlz5pVMUk6k0OXVLli4W0jyj+pL6ylJLJEMgz37eYoLdHNtM4UPC7HyFizRcm/lsmPIDkZuGUmnuZ14vEtgynsP8qbDuA7YdrAVhwfOXZyxD7HnxKIT6BnpYRNgo1UVo1KpSDqQhFxHLtlUP191JXVcfO8ivT7opWGbdOXTKzRWNoo3jqbqJn7v+bswXIkrZMSmEeKwujy1nKOvHKXTK53QN9XHxMHkhUzF8++ex8TBRAK4jPptlACIHHkCCNL6f5U1ePPHm2ReymTS/kmSZqngboHw+dtU3P7ptqRpD5sZRujLobQ2tHLx/YsaTf79Lfex8rFiQdwCDTCkIr2CokdFGFobihYVSQeSuPn9TfF3bq8TsiKeb9JkMhkftHxA6plUdgzZISou1GXla0XYrDDsQuzQ0dPRaP46zetEwb0CHu95rMEyVrYoaW1oRddQV7y5GloaEjQpiOyYbBzCHLjx3Q2ufnZVQ5avrgvvXsCps5PGsN2uox093unxwjy49HPpFD4oJHFfotam7cJ7F9DR16HPx300nhv9XjQhM0LEbIzn6/j842RezuS9uvc0HvuP6tqXgsVn9LvRGo+1tbaRsC8BSy9LcVPfvszdzDGyMeLGdzdwjHDUsNQ8MvsI1v7WzLkxR2OTUZJQwv1f79NxakeN/J3s60LeWW2BADq0b57KUspIP5dOyIwQcTDyfNkGCWHGLfUtWi0ze73fS/OH7Uoml4m+8jUFNRIws/06FfZh2HNPhNCZoaJ3+/NVFF9EVXbVC600Ew8koqOvQ+gsTTtgEAbq+XfyiVgQoeEfbx9sz5LkJTRUNGh97vz788m7nfdC69P/buka6LI0a+m/VBqFvqz98wAaQBkgnuM7h+3E2s+asqdlGFobiuvnw60PefDbAxoPN7Lx/td4Ft4mKHedOHDs+V5PPPp4cHT0UQITAmkpahHXV2WTkvTz6Xxn/x2DfhhE2JwwXKNcybiYwY1vbwjD/L8VS4VxhVxYeQGbABumnZymAUYN+n6QBFBT12+Rv4ms+tF/jiZsZhhRK6KoK6kjLzaPSQcn4TfcT1y7Yj6PwcjGiHfK3+HSB5fYO34v5h7mPD32lPl356MwU5B3O48rq68Q+Wak1nzNxH2JpH+fjuFSQ/Kd8zE9akpEt2fDioCxAdzdeJfix8UETQrS+CxTjkyhTdnGp7qfoqOnQ/C0YIkCpyJdyJLQploEgRBi6WUpsZ4SMxkQhrvJR5O58O4FlmYtRSaXYWhlSGVmJSAMZ2VyGa2Nregb64vfV2NFI1u6bqHz4s6EzAghKyaL8Nnh4qYp9XQq594+x5wbc+j7SV/6fiIF8J8cfkJebB4BowM0bL28B3qzomIFh2cdpr5EyP1NO5uGpZcljuGO5N7KRa4rp+uSrhrXrKpNUCPqGuhKlLQpJ1PYM2YPJo4muPd2p76snpw/c3CLdtMA8e/9eo/Lqy9LrJ4yLmWI/25taBUBirqiOjYGbyRyWaRoOZ64P5HKzEoxb1BdL53VzJdUl2uUKwO/G0hJQok4gHOPdhdVuWs911KTXyMoIEz1JaSZqpwq0nem49/DH8/eUtXg5Y8uA2gllhjbGksU1s8z00EYKBc+LCR4arCoamhpaKG2oBYjWyMUpgqMbIyw9rNGriNnxKYRjNj0N5gpEyyQsq9la/3MlYmVuMS5wFT4fd3vzLk2B5VKxY1vb+DU1YmTK09SYFdAsH+w5H12f6u7eIwsPCyYfXW2VsAZkKgPs69lk3cnj5fOvYShpSE1BTWinVH7KkkqIf9ePi+df0kkctgG2mLpZUlFegV2He3o90U/rLytMHczF9fF1NOp1BXXkX0tG6NKHZyTL5FSHwCvPANo1RlGD/54wN0Nd/Eb7if2HOdXnAcEu3zfob6i0lddgeMD8R7kzZZuW+gwvgP9Pu0nrgXlaeWoVCpsOtgw4+wMrP2sJfeixqpGrqwWQJGAMQEaQ82DMw7yaMcj/Ef7i8cyaFKQmJMKQgbO5dWXCZkRgpW3FTKZjNfTXpeAgO2vc7mOXKO/2BC0AacuToz5cwx7x++lKquKN/PeJOlgEo92PBIVxGs81uDc1ZmB3wykoaIBh1AH8Vg83PqQWz/eEl+z3+f9JH3Q3vF7xTzy1+MFYmJ5aznPV2JNItn12VwNCGbYgwZk7ZTmzbXNfPPXN9hV2KGXpkdD2bP7d1lKGWauZpKhLwhuDe83v09bSxu6hro83vWYg9MPMungJK2206Evh0ruweoBX/r5dOpL60k7k4ZtkC3OXZ0xdzdH10CXXaN2UZNXw7ST0ySKMXXJZDJkMhkBowNEgL6+rJ71HdaLw00TexP8R/lLBqzxO+IxdTIlzCGMuMI47CfZ09lXSiy4+f1NUUk9dvtYdPR0KH5cTHNNMxffv8i1r66xJHmJ2Jc9b3cISLLaylLK2D5IyOhSX0Pt7Y+3dNuCwlzBgvsLACEf1H+0v0RZcW/jPSGfa7A37r3caVvZJnHpULsE6JvoU1dcx60fbmFsa0zPlT0leZTq360rEqIObANtce3uyvWvr5MVk0XurVxGbhkpAFQRjgSMDcDS05JTr53i0Y5HtNS1SJRV6jXFZ6iPaEf2bs272ATYYBNgg6pNxaMdj2ioVXLyz2IczOpZeHESe8ftRd9EX7z/zjgzg8e7H2Pla0X43HBs/G1w7uosZkiJx/JpGfl38yl9Uipm9i0vXY6RtREyuezZmoxAEKrIEBRrjVWNOHd1ZuK+ieJa4hrlSktDi0TF1lDRQHNtM29kvkFlZiX6Jvo0VjViYG6ApaclYbPCRHvlNmUbBQ8KJBl3juGOYo9i4WHB28Vvi0Rjp85OhM0J03BEAGEG4NzFmdamVsJmh2kM8NVl7mZOzo0c7EPsJQNa2yBbIT5kXieJeuR50Ob9pvc1XrMqpwodfR2sfKwImhSkVREGgoLOyNpIYrvb7fVudBjXQTJgz76eTcqJFOK3x8N2REBTXXm387j2xTUCxgTgO9SXK6uvAIKl3o5hO3i7UKoGf3LkCde/us7gNYO17gNfVG3KNlJPpRK5NBI9Qz3SL6RjbGeMfbA9RtZGKMwURC6NxNrPmqJHRXR9ratoo68mxgHYdrDl7NtnCZwQiEuki0i4KEsp4+SSk0StiNKwM6/Oq+bUa6cwsjVi5C8CkS/5WDJ3N9xlwNcDOP3GaTIvZ2rsI1NPpoJKUBBqU1f+5PMTXgO9GLFxBMlHkyXrrq5Cl66vd+X2T7cxcTChtbFV2CO3Aw9Gbhkp6SG2RG7BNtCW0b+PRkdf54XqSmN74ZyoK6rTmD2M3DySYeuGkbAvgaKHRXR5tQvlqeUoTBU01zRzb9M9ur3RjUkHJml76f9rVVhYyMoVKwSg7wW/0w041tREr3feYWruNOwdpM5IxrbG4tzjwqoLWHlbidfb871a5qVM8V6UeTmT2J9ieXJImKE9H/vyIkvH9tWeXNK+HDs5aiWbA7h2dxXdgp4nzN345oa4h731wy2GbxzOqddO0e2NbgxZMwRPO2nPHT47nLBZ4ajfqpGNET5DfKgvq6e1qZWGigbJuvCivRsgUZWry6jOCIfbDuR2yhXnf3/1/4uMixlYelmKricV6RVc/ewqILhXeQ3wEues9zbfo7m2WSRjtCe9tNS3EDwtmNqiWs6+eZYTo0+Q5JNE0PogbsTcwGugF7tH7RZ7ASNbI0wcTNDR16G1sZUNgRvoML4D/b/sT8iMEA2CYnFCMTGfxODe252x28ZiZGPENzbfEL0qmu7LurMxeCO2gbbizNfG30YrSTvneg7mbuYaIpH2lRWTJR4/az9rxm4by6GXDnFk1hHu/3ofVZsKI2sjdPR0aGttY1PYJnyH+4p7taL4IlobW3Hu6kzuzVz+6v8XwzYMo8uiLpi5mrHo0SINEPKf+n+n/gH7/ql/i5LL5Xh5eSGXy9Ex0sEmwIagSUGibVfurVxufHuDwImBGkPq7su6032Z1N5t4DfPWPglSSX80fMPQFDuNVY1MuCbAeKGYfa12WRfyybneo6QY+RsKgEPEvcnEvdHHADz780n7s847v1yT2QggzAs+Ej10Qsz6Iriiyh7WoZHX4EN/nxjcObNM9z68RZmLmYi2FfwoEC08LALtqO+rF4MTm+pb6EkoYTbP9+mLKUMKx8rYj6L0crEf7DlAQozhZiDYeJggmd/T+bdnUfKiRQ2R2yWWK89Pf6UXSOF4Nrlpcspe1rGlY+v0NLQQpuyDR2ebVBqC2sJmhzEgK8HaAUgZDIZekZ6WjPqrn15DQMLA6Lf0wSJUk6mCAzFyzPx6O0BgEdfDwZ+N5DEfYk4hDvgP9KfncN3vjCfxnuQt1ZbQPdod9yjtT8HhEHV/kn7SdiToAH2ZV7KZPvg7XgP8mb4puGiVN7E3oSJ+yZSW1RL9vVsZHKZ1rye4wuPY+VtxZC1QzQec4pwwibAhpQTKSiblBpMxgX3FxC3NY4Hvz3QAChVKpUIumkD+xzCHDReT/KZJwVi7GCsNU8h+3q2eP0sTlysAax59vdE11BXq91la2MrB6cdBKDr61017C5UbSqa65q5/NFlIhZEaIBI43eNx9DaUKsVZtisMPRN9LWCRMuLl1OcUIy1n7VkQw2CpSkIdqYvKht/G94pe+eFjwPk3Myhpb5FK3Ai15EzdttYih8Xa7DBko8mP/sMg8OQy5+B9Dp6Ooz5cwyZVzK59OEluizuIrEQc+zkyJSjU15o5Rr3exyNVY0SBn1RfBF7J+yl58qeHJ17FBDOB21h0S9S14GQF3hk9hFqC2vxy/6fA/tkchnmbuYvXDvz7uSRdjaN0JdCtQ7KGyoaOL7gOF4DvYiYJ4BTc2/OZceQHaSdTaPsaRkzzswQme4NFQ08+O0Bzt2cWXBvAQ5FDry82YHcgJs0lAtM7GtfXKPwfiFEQmJQIo3zGkVm75zrcyhNLmV9wHrOvnlWXGMe73lM1tUsGsqfDVt9Bvvwyu1XsPKx0spWfBFL1rO/J7oGujhGOIrXm1tPN7q90Y3rX13Hwt1CsgbE/RGHwlSBmbMZd9bfAaDb0m4SYkpdSR0pJ1NIOZnChD0TRPtRNQAaMT+CssAy6tIFq7/WtmdKC4Bfwn/BysdKsqZe//Y6iXsTmX56ujisey3lNXEg2tIggKQ6ejpc/fwqdcV1WHhaiCHr7StgbAB1RXX8Ef0HXV/rKlh8Xc5k4LcDqS2sJWBsAHJdOaVPSjG0MsTczZzRf47m4V8PMXc1J2FvAsWPi7mz7g7z780Xmdx6xnr0/6o/juGOZFzM4NL7l/AZ7CMOxDqM7SAONrXVS+deYu/4vfwe9Tsftn2odRPf3s5u74S9uHYXspD6f9Gf2LxY2izaMFNIVUF1JXV87/C9hq3ysVeOATD6j9H4DPbhyZEnHJl9hDF/jcHK24qkg0nc/OEmPd7uweNdjyl6WCQB+zz6eDDw24EY2xtj4mBCZVYlFu4W6Jvq0/vj3rhEunDjuxuUPS1j5OaRErv09pV8LJnWhlYJiALSYai2iloZRdblLB7vfkxzTbPkMfdod+YlzcPI0Ijdo3fjN9JPtHNVM9eNbIyoya+hOKEYx3BHcRj9Suwr4iBDG+Eq8UAiN7+7ycVVFxn4zUDC54RTcK+AP6L/YPCPg4lcGknnhZ3FDf2Tw0/IupolKL3MDeg4peMLVcsmS014x+8dRh0ZRVi3MEAgYz3e/RhkkO2ezVOvpzQvaOaNkcIgUUdPRzJU1zPSk6jg4rbGcfnDy0zYMwGXSBcaqxqpzq3G3M2cpyeecv2r6/iP9EfPSI+1nmvx7OspWhCqS1s2mWdfT9HRQtmsRK4nF8/ZfZP2kXk5k36f9SPrSpbAaPfshkl5Ds2V2u8pHad0xG+4n6h6y7vzzNqvobyBT/U+ZcqRKXj09SBhTwJWPlZ49PGQXCdtyjZaG1rRUejgHu2uYTulrua6Zv7q/xcF9wQSWeBETacAKx8rvAd7i3lM2ir/Xj5XPr4iKCi9rShPLee37r+hUqloKGtgzo05kp5C1aaivrQePSM9ce0ytDIUh7XB04JFEkzZ0zKSDiURvUroW6uyqqjKqqIkoYSSxBKW5S4T8qNMFYS+HEr6uXTR6cGpixMPfn+A7zBfTBxMiFgYQW1hLeV55cTuiuXguIM82fpE4/NcGHeXB0WX+eDz9ylzPkqdw7Phct6dPBZtWsSJYSc4+cNJPpv2mfjYgSkHKLhfwLD1w7DyFfIcDSwMWOe3Dr8RfmIunGMnRwb9MAj7YHtiPosh5UQKL198WevguKGigZr8GszdzLWqFdQWae7R7iTuT5T0R+ffPU/ZkzIGfD2A2sJaxu0cJ+k5ZHLBktB/tD9zb87VWGvzbudxaMYhAEx+F/qiupY6jfcw4JsBlCSWIJPLcO3uiqWXpbiO7R69G5VSxZWPr2AXbEfX17pq7V3bV8rJFECwGTu/4jzKFiXNtc3CnirSheDpwegZ6YlZSfPuzBOVx6lnUrk47yIBowLo/lZ3IpdGYuZiJlFMqO0SAcLmhDF8w3CWZi19YT6ZfbA9S7OW8uD3Bxyde5RpJ6Yx68osnhx+wuWPLyPXkdNc18yM0zMwsDBArisneFqwoKD820nhxKsnsAmwodtr3cR7uvr9N5Q3oG+iL1r+rWpYxVdfy6i+sZv6plrxGNcV13Fh5QXh+3A04YOWD0TgX91fzro8i+LHxdzbfA//Uf7sHLGT8pRyiQV+xoUMkg4m0fPdnhI7/+BpAlkyfns8h146hO8wX/Lu5DHt+DRMnUwxdTLl3qZ7WHpaioDUldVXhBzhnGWolCo2hWxiyNohdHu9Gy6RLhLSqYG5ASvKV7zwe28fBwCCG0J7oKQ8rZzix8W4R7tjaGUoISFvCNqAoZWhxHEn83Imj3c/xsDSgNaGVmSyZ8ocCw8LXol9RWM/E/tzLLd+vMX0k9PFHqWupI7GykYsvSyR68hZ47YGgHfK3sFniA/Xv73O0xNPRRA9+3o2BhYGWok01n7WGn3/412PubP+jpgnp1KpJNei2r5W3VMPWz8MlVJFdV61pP9VV3NtM1U5VS+MsDj6ylFqC2uxD7UnYEyAOJtJOZHC3vF76f1xb3qt6sW2AdvoOKUj43eNp/eHvSXDc/tge4b+9Gx/OXLzSFJOpmAbZEtjZaOQ3ehnjUukC6o2FS0NLdTk15CwJwGnLk74jfCjrbWNtHNpmLuaE/dnnAjkqMG+mvwasq5mCe5Qf98HfYdLVdDqcyDjUgbxO+KxC7LDNkggkKsH9gpTBfE74qnOqSZqRZRo+6troMvQtUPpOLkjv0f9jkwuY841KRHheRBZJpeJhNT2n788tZz8e/l49PHAxN4Et55uxG+Pp7GqkfQL6WTFZBG5NBJDS0N0DXTRNdClqaqJqqwqvAd5i2SfNmUbbxe/rbXnelG1n+f9n6wtv/5KXz09DUXf89UN6KmU8ePqH/h8zRcvdLi59eMtPPt6Ej4nnO1DtlNbWMvChwvJuZlDZWalxB0r9XSqeH4AEkC+obyBovgibAJsMHEwobm2mcMzD+M5wJMui6QxAodnHqa2sJYZZ2b81w/Ac7Xo8SJaG1rZGLwREICymZdnYuZsJu6v21/HN27AunXw9jIlOZuO49rdlemnpnNn4x12DtvJsPXDOP3GaXq+11Ori1FrUytfmnxJx6kdGfuXJnHatMaUXid7kd49XVx3TZ1MxWtYXe33surZo9oR4/6W+9QV1Ylgn4GFASurV9La2CoqjtWK07XX1lKrW8veSXu5tfgWbS1tEpet0JmhoiV3a1MrvT7oRVF8Eev81hE4MZDxO8dLRBN1RXUk7E3AvY87ITNCKEspwz7YXtw7mjqZ/qcAtH/lupZ5OZOrnwvigPZzyfb70ZzrObhGudJxSkfaWttQqVSolCpST6ZSV1RHr/d7cXzBcapyBDthM1czgWj4971UV6EriTm5t/keF967gNcAL8bvGv+fAqb/qX9d/1tr3ovqH7Dvn/q3KJlMhpmZMAwrelSEnrGeRCHXZVEXgiYGiUBExqUMGisbtbJQny8zZzMU5graWtqoyqnCxN5EYvVkaGnI5Q8vUxhXCMDDvx6KHux5t/M49/Y5QGBUgdCAJu5PlIB97T/H1r5bsQ+1lzz+8K+Hot2INouL5CPJWPtZs+DBAvFnt3+6Lf5bYabg3NvnMLAwoNPcTtiH2LO8ZDlpZ9O4/vV1Mi5mUJNXAxs1P//ixMU82vGIPWP2iBaRdUV1GFkb4TfCD7meMEAtjCuk+5vdJezxlvoWbv1wi6bqJq2Dzt4f9Sb7ejamjqYaAFRlViUV6RU4dXbSqhqcuH8iymYlDeUNGFgYSJjhXgO9GLt9rOSG5xDqgEOoA75DfaktrMW1hyv6JvpkXMiQMA9VbSrSz6dj4miCffB/Pd8OYEHcAq0bbVNnU0wcTUg7myawk+Za8vTEU/ZN2MfILSOpL63n3qZ7AFqtKfNi80QrO23VUNbA/sn7iXwzUis496IsQ/UGXpuNZvt6vEewK3o+byFsZhiX3r9ESUKJRhZc+82nNqvV4OnBOEY4UldcpxFSrqPQYeL+ieybsE8rGHz548vEfBrD+N3jtdqB+Y/yFwaFja3I9eQSe0KVSsXhmYcJfTlUK1v0RcfYZ4gPhlaG/N7jdwAxV+i/UqeXniZ2bSymTqa8mfem1t9x6uykNfdRzfZ26uyEgdxAa6OVfTWbmE9jCJwQKAH7jO2Myb2Vy7nl51jwYIHGMG7wmsEaSrHWplbKU8rJjc0l+v1oylPKteYcxnwew60fbjH72mxsAmw03lddSR093+2pAZ7+f6221jbWeKyhobyBVfWrNB7PvZnLpfcvcen9S1rB++Pzj5O4P1EY2M0TfiaTyUQm65QjU/Aa4CWuLyqlCueuzgSMDaDzFWHw/zDkIUaKKTiEO7Dw4UJOvX6KsqdljM0fS0wvTRsSG38bIhZGSM7HjpM7apADjGyMxPW0tUlQnrU/fm2tbajaVBrXVf8vtNvmukW50bigkeq8avLv5pN+Pp0ha4Yw8+JMih8Xs3P4TgBmXp6Je7S7ZE31G+6H92Bv0s6kUfy4mMe7H6NnpCduYK39rDHDjNacVgzrDWk63USxabG4SdAz0iPneo4YEA7QXNNMXXEdfwvvkMlkEmXyN9bf4NHHg+knp2MbaIuRjRF/9PxD3OwVPSoifns8IdNDGL5hOPl389k1cpeYh3r+nfOi8iVyWSQm9iY4RTy7pvLv5HPzu5uihWjkm5GEzw3H0PrZ8FbfWF9UnNt1tMNnsI9kuGXpZSkqSbKvZ1P8qJiQl0LEXF+FqYJur3fDZ6hUIQ7C8Cc3Nhenzk4iMWnoz0M5ufgkW7ptweQ3E6YemEq4XjgXplyQ2PHoGeoRPjcc+xB7Dr10CI9+HoTPDqfvp32R68pF9YxLpAtTj0/FIdSBsqdlor3wnut7cOrsxPz78yXvydLTki6vdkGlVLFj2A5Kn5SyvHg5BuYG9PmoD0XxRewYIig4tFnPAtxac4szy85g7mauAfZpqzsb7nDh3Qv0/qg3kcsi8R/lT3FCMSEzQiS/p2egh1OAEzUFNaScSsHS+9nxaL/+p5xK4dgrx5h2chq+Q315evwp+yfvZ9Rvo14IyHVe0BlzV3OufnFVtN8zdTKlx/IeOHXRXIdj18aSeTmTyDciMTA3YOX5lTQrm3m/1/tYGUrv3Wpl6dHRR1nz+hpAsAMHQb034bjQq96Kv8VfY//S+v5UKpWQd/n3gFfPUA9jO2Px2n9y+AlHZh1h6rGpdFnUBf9R/mRcyuD4fMGS1X+MprU3CGqoP3v9SdCUIAlT/MjsI8T9Gcfer/byU/+f6N2xN4n7EsVjHTghEH0TfXakh5ETNAQPD+nrnltxjqaqJkZsGiG5zxiYG+A/2p/QmaEY2RjRcUpHTJ1Mqcys5Ng8Aage9fso0s6kMffGXAwsDMi5kcPvUb8z6IdBGsQ8gMurL1ORWsHQn4dScK8A32G+jN0+VitBQhuZKX57POdXnKfPJ31QKVWCPWn8QvG+LteTYxNgIyo3je2M2dp3K54DPEV7/e/sv5MA7+0HNO3V4z1X9qTnymcOFhP2TuDBbw/w7O9JY2UjP7r8SOjMUMb8OQaHMAc6Tu3IxVUXAShLLhNt9kwcTMTBX2l9KRNchXOovKEcO2Np76Jz813afC6SGJiFsXyYxGzA0tOSi30vkuech5uutIdyi3aj4H4BTw4/IedGDu7R7vR8V3jvT48/JedmDq7dXSVkh/qyeqqypdbmsT/HknU5i3E7x/H0+FMx4+bN/Dcl6rX2NX7XeMbvGk9tUS35d/KxD7WnNKmUgnsFAoFw9RUsvSwxtDQk42IGbtFuWHlbSbIWQbClTT6czOg/Rotgf+DEQG7qCS4f2R9ks6t5lwhc5t/Np7awVmufDLA4YTF5d/J4vPMx51ecJ3xOONU51ch0ZJKe/cmRJzTXNBMyI0TIxnMwwau/Fx59PWipb2H3qN2EvBSCS6SLqBK98K4AfD1vdWbqYIrfCD8NlxF1pZxIefb7BrroKnRFYlN5ajkP/nggAUHU5TPEB2RCNq7vMF9JtuCttbc4s/QMs2JmcWf9HczdzCUk2Ae/PcBrgBfdXnumiXn1yatUZVdhZGtEzOcxXHr/EgAftHxAeZWclG7PyAZq0KHHWz1QtijRVehK+g0QlOeZlzKxD7UXHA462NDt9W6ceu0UjRWNjP5jNLZBthTFF5GwJ4Hg6cFc++IabtFuhM0OI3F/IrYdBMVbl1e7YGBhQFNNE7qGwmjLyNqIgDEBWPk++948+nggk8vYPXo35u7mRL4ZqXXP/aJ6cuQJdzfcFZW7lVmVmDqZal2Lko8mc/bNs8y5Pkei+o/5LIaSxBK6LJEO9vWM9HCLdhMyjs2/4qXzL4lg+K01tzj75llmX5tN8eNiFKYKgqcFkx2TTWVGJU3VTahUKtLOpnHpg0vk38lnRcUKDCwM0DXUpbWhlcqsSgytDIn7Iw6ZTCbOObb22YrvcF+mHJ5C/r18tvbdSt9P+oqAzvPVdYmQf3vmzTMkHUji/ab3JT2q9yBvZl2ZJWY/PW9X+nyFTA/Bd5gvu0ftxn+Mv0S5CIKSvSy5jJQTKVh4WIjnuYmDCbaBtlz5+IpoVf+8803c1jjKnpZpAJlmLmYETw+msaIRPWM93sh8A0MrQ0qSSriy+goJexIY8M0APmj9gOxr2aSeScUl0oWdw3YSNieMkOkhKMwUhM0K46/+f+HZ35Po96LpvEDYM6jVyhdWXeDJ4SdM2C11dEo9ncqNb24AiHl/MrmMgLEB6Brqcuq1UziEOWhVAhnbCzbqLyJ1tq+5NzQzm0FwmDj56klevvgyJvYmIlD9cNtDcf0OmRGCoaUh5anlNFU30XFKR/Ji8yh+XCwSEeU6coxtjakvq+fkayfxGuAl5tq+qNrP8/5P1pGdO1n6n8zJmkMr7//yJ26X3Xk16RnZqKWhRVSaLU5YLPY6amtV+xB7rd9D19e6EjghEJlcxrWvrrF33F4mHZyEnqEeubG57By2k1G/jSJ8TjhtyjaSDiZhYCWdKd34/gYP/3qo4RhSW1hLbmwuymYl8X/F0/3t7iLp/fii4+TF5rHg/gLSzqVRmlRKl8VdkOvKxftXlyVduL/5PsM3DBdBtvtb7nNs3jGmn54u7i3WvFuESXk2677wwf1wHG3NbQRNDhLuFd8OxDHCEd/hvqLd8PP5lHIdOb7DfXEIcyDpYBIxn8YwbMMwkUhVblXO7oW7WTZrmfjZtLkpxf4UK/m/kY2RGKUwYc8EjdmRwlQhzhT/iP4DKz8rRm4eiUGMAfpN+uS45Yj7xOUly2mpb6G5rllCvNZV6NL3k74UPCgg+UgyifsSSdyXKIkv8OjrIckJtfa1ljirPA/Q1hbVcmf9HTz6ekhiRUAgOtUV1kkikkDot/Lu5NFY8ew8rsqpIu9OHiO3jOTp0af0WN6D5GPJnHrtFD5DfbDytuLVpFeJ/SmW02+cxqOvB92WdhPJlda+1hKioapNRUN5Azr6OijMFIJSv6yBhD0JEtD1n/rv1//Wmvei+gfs+6f+LUqpVJKYmEhgYCD7JuxDR6HDovhF4uMKM4XEEuHKx1coTiimw9gOFNwvoDy1HL8RfiIwcf3b6zTXNNP3k76YOpmysnLlC/92aXIptoG2FMYVMnzjcFqbWsmOyUZHoUNVdhUNFQ14D/bm2CvHyI7JpvfHvRm8ZrCE7aZqU5F5JRMTBxNqC4VMlPYVMiMEpy5OGFobCooKPblkgPLqk1dBJQVTCh4I7Obg6cF49vfEe5C3xqbFe5A3yMBzgKdk06ZsUQpD1BkhGNsa4z/KHyMbI5GRv23gNuR6chY+XIhjJ0f2TdpH4r5EIpdF4hLpIhmqRy6LpMP4DlpBifjt8aSfS8fa15rg6cGSjf/jXY+58O4F5t6cy9UvruLey10CyjmGO3J80XHubbrH8pLlEpDRxt8GG38bbv5wEwNLA3Fj2drUyp6xeyh7Wsay3GU01zaTdjZN8p6ULUq2D96O7zBfur/dHWtfawkIlXExg4dbH6JsUVL0sIjZV2dLmL3H5h+jIq2Cly9IrXNAAJDm35tP1pUsMWPJyNoI527OXP38Kt6DvfEb6YdTZyf0jDVZ0QseLBBC4puVGgP+1NOpXP3iKlEroug4VXOgmXcnD1SIf7d96Rnp8ZHqIw07NnXtGbuHzCuZGFkboWespzVcPWJhhFbbEcdwR60gi7paG1vZ2HEjYbPCNBoZHT0dAscHvvD5dh3tCJoUhFtPN615OCCcY+ohaHurlYbyBrq/1V2DWQmQfiGdtpY2TBxNuLP+DmGzw8QG1NrPmvKUZ1ZZiQcSNTafaWfT2D54O34j/Bi7bawG8Ktmeg/6QRo0r67q3Go2BG2g6+tdxVwYdbn2cKXP6j5c/ugy60PXszRtKTo6wrnQ0tDCiYUnsA2y5dWkV7WCkDKZABipLTDal7WvNYdnHkbVphKbajVo3FTVJDJUtZWRtREN5Q1sCNygNf/sO7vv6DC+A5P2/89audQW1gpEhRdU8LRgHu14RE3+s99RNiu5t/ke+qb6lCSWCFkh7eyfbnx/AwtPC4asHYJrlCs5N3IwtjfG2tcaIxsjXokVAO3B7w0mxTeF3VN3M+LuQfSNBRuRWZdncWvtLSqWVpDjmoPJIRMqvCuw9LLk/m/3BfVRXg1u0W4ceukQLt1d6LK4i8Z7b2sVgGodhQ7f2X2Hc1dnCft716hdZF3J+k/Z524fsl3MnQKwDrKmLKGMvp/0xSbARgDd/i71xrB93V53G30TfT5o/QCZTEaH8R00rJ+j3KIItgumNK6U5s3NJBski2Cf73Bf7qy7w5HZRzCyMxKyS7XYXlbnVqNroCsAAe0sdsfvHI9KJeQlqDePxY+LufHNDRorGxn5y0icOjvxVsFb4msN/G6gaJ8W82kMXZe0s2nKraY0uRRrf2t0DXWZdnIazl2dNfJl2pdaDaAu9XW+8OFC7EPsSdibwO2fbuM/yl8E+0qTS7H0ttSweQIh53DvuL34jRSA1K6vdiVsZhhZMVnU5tdye9Vt+lT1wTnPmY1fbBQHRCqVCoWZglFbRlFfVs+JhSfQM9YjfHa4qDx7euIpWVey6Lmyp6h4fnr8qeTvmziYaFXo/t7jd5pqmuj+VnfqiqTKl/rSevHY5t8T7NQ6jJXm9Vz64BL6pvqM/lO6noNgo3fz+5vIdeW4Rbsx+PvBlD0to6m6ibNvnaV1ZCtncs4QuiuUqKAoyXMbqhp4EPOAsJ5hfND8gdbvCITh4fBNw0XChqG1IXI9OY92PBIy55zNNIgtVj5WdHu9G91ef9YHWXpZSgbcWTFZZF7OJGJBhHh/NrQypOxpGcoxStCBH0J+YPWF1ZL7c0tmC57pnmS7ZaNUCSxy+xB7wuaEiT2PZbklYXFh5A3Lw7mLMw3lDRx6+RD+o/2JmBdBdW41a9zW0G1pN4b8OETDetIhzIFeH/bCylew3DR3MxfB+6E/DxWHjO0r/14+pUmlElV04v5E9k3cJ/4/ek0011Zew3GfI0GTgyhPKac6t5rE/YmM+GUEDaXC/aHtudYh81Im9SX1zxSHruYozBRY+1lLVHVqNnJDRQPuvd3RNdCl4F4BCXsSRAcDI1sjwmaFSZTMBQ8K2NJ1C30/60vujVzy7uQxdtvYf9lrqKsio4LYtbEEjAnAo48HGZcyqMmv4fj846jaVPRY3kPyvVu4WzArZhZ/9fsL9z7umLmYUfSoCOuAZ4qKzos7/0vHhxdV0MQgsZ9StihpKGuQ2HB3XdKV0JmhKMwU1BXXMe3ENIltIIC1oTVhDmGU3C3hwpELTJ02VfK4c3oZIfUhHB8VQ7/Hv2Pb7tZv4WFB3KA4Wstb2aa7jYr0CpFUMGTNEIofFZMVm0Xnbzrj6+OLaw9Xhv48lFOvndJ63x3y4xANe7CCewUkHUqirbUN+2B7TJ1NUZgpuPXjLYxsjLDyscLM1YzWxlayr2bTaZ6g1q3Jr6E0qZQDUw8w+s/R4nnTVNPEqN9GUZVdRfqFdI7OOcrY7WO1EuTKkstIO5dGS30LTl2ceC31NepM6pi0SehDmkqbqKl99jmOzj1KUXyR5DyqyKgg/24+7tHuYlamS6QLvT7ohcJcoWHDCYIlWmVmJSEzQiTqJ6/+XrQp25i4f6KG04D6bxY9KuLU66cIeSkE12hXIn+LxDPQk4S9CTz47QGd5nXi/q/36by4MwGjA3gt9TVUbSqRPKRsVlKdW42BhQEV6RVc++KaBARpqGgg+WgyDqEOBE0MEkEgZIiKQbuOdkQsiMDM2YyEPQmAYMeqa6iLpZclbxe+jUxHRkNFA99YfYORjRHLS5aL92b1nqjHOz3IupFFds0tWuTD0WuT9ulyXTlyXTktDS3k383HzMWMx7sfc2X1FexD7Mm/k49dsB0zzs7ArqMd7tHuuEa5srnTZmryawibFYZjJ0fCZoahbFGye9Ru5HpyOozrwJFZR8Tj6tzFmcT9idzddJfqnGrsg+1Jv5Au7H/bDVbVgOdq2WoK7hewIG6BqHDIvJzJg98f0GN5D+yD7VG2KEk+moyFu4VIzKstrCXnZg5NNU1kXc2SDO3jt8eTdiaNwWsGC3mGQ3wwtjXGyscKZbOSy6svc3/zfepL63GNchXtstXl3NWZ2TGzybycibG9sXjvUDYrOfvmWUAg2MZ8GoO5mznB04Ix9xDOMWM7Y9pa29gxZAc6Ch2iVkaJ96jFCYupSKugMrOSe7/cY8QvI8RrKfdWLm2tbRhaGnJk9hFaG1tx7e4q9kFH5x3lwZYHInAIiIqkpANJ4nfcvgzMDbDysZL0DPHb41GYK7TmzQOgEmwz2/ep6lryZAnNdc3U5NWIJD318Zp5eSa/hP1CQ0WDZH6wMXgjxY+LMbIxoqmmiaBJQVz5+AqdF3XGe5A3DeUNxO+I5/Trp5l8eLIIUKmBZX0TfSE/WkfOhZUXqMqp4vW01xn12yisfK1wj3bHs58nymYlBQ8KsPSx1HjfAIUPCil8UCj+/+D0gzza+YiXzr1Eh7EdyI3NlTgh3P/1PgpzBRP3ThQzbdVVV1LHjqE7KLhXwLgd4/Aa4MWN725g6mRK8LRglM1K1vmvw3+Mv8YaDcJetr6kns4LO+M92JuJ+yaKPdTReUcpf1pO1IoobINs6bG8h7h+nVt+juSjySx6tEhwyEgoJmFvAoO+H4SJgwnlKeU01zVzZ90d9Iz0/kOwr/08T723/T9RVdXVvNivSFr2QJtxG34jpX18XXEdu0bsEjMS1TVm6xham1qpKajBwNxAQ3Fr5myGqZMpyialSKJSqx9tA20Zum6oCLQpzBRa+5rEvYmYuZhp2Ejn3cljz5g9dBjfgbSzaXSc9mwe1H4OF78tnvht8XSa1wmZjozq3GpUbSoil0Yy6NtB6Ch0RNDMwsNCgzxsUpaFe8IpCl1nsqJyBTp6Ojzc9pATC0/w0vmXcOnmIun1PPp6iPsHENYF9eOP9zymqaZJBOY8LTzJqMxg/TvrNUjgubG5FD8qpuPUjugb6xM0OQjbQFsil0ZKnCgA0T1LXdV51fzo8iOW3pa8mvgqyhYlqlYVVdlVLPhiAdd7XOfcoHOS5+gZ6XF2+Vnu/3qfOdfmkHsrF+9B3tgE2OAY7kjYnDAxqzD5SLII9slkMq0xLi+q+tJ6Yj6NQSaXSe5JubdyufndTcnapq7A8YEEjg8k42IGOTdycO3hSvHjYk4uPsmwDcOYfGgyMrmM7UOEfXB7C+mIBRE01zUTMCZAq7W0uhoqGvjW9lsRyBy/azytja3/ZZL7P/Xi+t9a815U/4B9/9S/TSn/luJ3f7s7ZU/LOL7wOCEzQnDr6UZ9aT3Ndc2YuZgh15HTZ3UfUcEStzWO2z/dZlnuMvGGnLg3kbqSOo1BJAhDhuPzj9P9re50nNKRbQO3if7uqjYVkW9Eivl26uFR3J9xpJ1J4+FfD4l8M1IDmFA2K/mr31+EzAiRsIbUpbZRrMyqZK3HWo3GQkdPh7IUYWCmZqQM+GoA17++jm2gLcWPiwmZHiI2jXXFdWRezsSxkyPJR5K5s/4O4XPCRQuiK6uvcPXzq1RlV9FxSkf0DPUkLHvvId7c23SPq59fpd9n/ej76YtZfq49XFGYK0g9LQSDq5UpKpUKmwAb0s+lc275OWI+i5GAqt6DvEX70LQzaRrMyNbGVly6uaBSqtBRaF8cb3x7A0tvS8Jnh4vh4+pSs8jVViLqkuvKGfHLCKrzqvmr318M/XmohJld+qSUh389xL23O6o2lYZ9YHNNMxXpFRp+5eoydTSVqAtcIl14+cLLfGnyJWVPyuj/ZX+ufHKFnOs5Ahj7XK31WIu5u7kGI6+2sJaC+wX0/rC3xMpGXUfnHKX4cTF9VvchfE64RgNVnlZOQ1kDNgE2GqCdQ7gDMh0ZkcsitWY7nn37LJWZlf8tIEedSeM/+gWbvH9RQZOCCJwYSEN5A001TRoK0O8cvkNXoUvIjBAxu1Fddzfe5eb3N4lYoBmCfn7FeRrKGhi6bij3f72PS3cXEexrbRAAY7+RfviN9NOqClTnfT49/pT6snoNsG/JkyWoVCqJsqt9yXRkNFU3cfWzq4TMCBEHKOpS51M5DpCC98pmJQ//eihknmixFUw8kEjs2ljGbhurNXuxpb6FksQSieWKjp4OE/ZO0Gian6/OCzsjk8vIvpatwdAGAQz+V/Z9/93SM9ajxzs9tCo7AQk4p6621jZOvXaK0JdDWZywWPKYSqUS1dhWvlZYeFqwe9RuOs3rJCqZLrx3gRvf3qDg1QIuKC/gWSgwD5XNSppqhNycrq92ZYvzFpTblLRtbKN0RCmWXkL2jVpJ+/TYU/H9+I3wo/BhIS7dXESbj+Sjyewdv5dxO8cROClQouwCYfPUPoNAXVe/uEr+nXzalG10XtQZ36G+ksa81q6Wn/v+zL1r97CxsKEmvwYzVzNeOv8SjRWNFD0qor60nqL4IiLmRaBnpMejnY8oelgknrPPrzEnXj3B/V/v0+OPHmy13IrsMxmB458BKgO/GUjS/iSSDgoDoO7LumvNhlnjsQa/EX5MOTxFw2JQJpNJlD3qoUHBvQJOLzuNmYuZBHhvb3V2Z/0dShJLKEsuY1bMLKpzqkXmdFlyGcM3DNdqx1ZbVMvuUbsJnh5M50WdUTYp0TXURa4jpySxBIBNoZtY+HAhXRYLiqr2ysADUw7QVNPE66mva7y2U2cnxu0Yx4V3L5B7K5eurwr3GfXnznLKwlfmS8XgCrq7Cp+79Ekp55afY8TmEZg6mlLTaojBZ+/hO1b4Xprrmtk9ejcZFwS74S6Lu4ify6OPB4sTFmPqZIqBhQF1JXVUZVdJhs4Pfn8guhQ4hDmI6151bjUHpx8keEawOHg4vew0sWti8cjxkAwzZl2Zhb6JvlZbX2WTkqaqJirSK8RzKfmIYE085cgU9s3cR2pFKvWu9bgOc5X0FXfW3+HSqksotiiImCtdt390+xErHytmXpypYauqtuJ7evwpT48/xcTBRAL2VedVszliM51e6fQvg+czr2Ry+aPLBIwJIOqdKDqM64CugS5yPTll1mW4Z7ujui4o8NqDfQ17Gph5YCZ3Ot/hsfwxPqt98B3ui4GFATX5NQw8OxC9Fj263ulKwb0CnLs4C4zxCxni59A31qfjlI4vVJmonQtAWIdUKhUT901EYarQSvABwW446WCSZJCkVsKqq9iumFKvUuZ7zxfVB8UJxYzdPhafwT7IjigxK05FbmAO7UZn824LMulHOx/9yzw3dRlaGjLr8ixA6KMH/zCYe7/ew9rPGu+B3hpEIANzA3yH+2LlbSXJela2KKnKruLQS4dw7+WuYTMb92ccj3Y+Iv1cOuZu5nj08WDYz8PwGuCFoaUhMh0ZJg4m1JfWozBTiN+jTCaTsLLfKX1m060wU0isdAFur7+NkbWgXDy55CTKZiUjN48UHCvSBMcKPSM9Lq++jGMnRzqM7YCOno6EdKJ2AFCDOwO/HSgZWO+fvJ/KrEryYvMYajYUnTodMs9kwjTpsfV8koNjak+eBB3WeuzbVG3YFdtx9durmK43lRBPnL5zYsn2JdSU1KBaLPS6XZd0JWJ+hHifz7ycyemlp+n3eT+tebxj/hwjWhU7hDmIzP51Aetoa22jIq2CkBkhWHhZEPNJDAFjArj21TXit8Uz5q8xjP5ztET9ozBVUJEh5PTMvDxTUC6HOdBY1citNbdw7e4q9s4DvxkoAre1RbVU51RT7/Js4Oi30Y/xgc/Y6W7RbhTFF7FathozFzMGfjuQ5rpmUSlsH2JPbUEt9iH2Yk/U9fWuGjZ1g38cLCoJQciKTD6SzLs17yLXkUvujYdePoS1n7WY5fzoSgW3f76NWQdnHDo5oFQqSTqQxIHJB4T3PNKP3Fu5BIwT7n8ymQyZzrOeqzy1nA1BG+j9UW+i3oliSfISyb22Jr+GI7OO0Puj3qIDzvoO6ylPLWfgtwPxG+GHV38vUTU2bMMwTi4+ydZ+W6kvqeeDlg/Enlatoq8vrWe1bDVGtkYsL15Ol0Vd6LKoC62NrXxu+Dmq4HiqPO7gUzAP6MCmsE04hDng3M0ZXQNdnLs482evP4laEUV1bjUWHhaM3zWe33v8TvKRZEmmt0OYA1OPTyXzciZ1xXXiZ5PpyEQlhbJZia6BruQ7MLAwwD7YXtzfFMYVknQgSatCfeqxqTw5/ERC/ilNLiV+WzwhL4VgH2xPa2Mr+ybskxAVOy/oLBIrylPL6f1Rb5Ewl383n/jt8fT/UlCR2XawFclMzbXNXPtCiFIwtjeWOBM9Xx59PPDo4yH+P+tqlvhvp85OTD06VdwTRy2PotPcTpg6myLXkTPy15FYelmKsRsgDMQtPS258f0N7v1yj9CZoSKYp2xRYu1njc8wH04tOYVtoK1kHXTs5EjjuEYN8ml7+7vn9wNFj4r4tfOv9P2sL71W9eLXrr+SfycfEK6/2TFS27rSJ6Xk3clj4cOFYr/b2tRKzKcx2AbZEjw1GH1jzX6jMquSuxvvMn73eA0iRn3Z36Slbwfi3tudirQKko8l4zNMUC39HvU7pU9Kxd+vyKjAyNoIr/5ezLk+R8ju1ZWTdTWLTvM7oTBToKvQldgxFj8upq21TbR6rUivoDihGLcoN9LPp3Ni0Qmi349m+slnitdHOx8Bwr7FIcxBYhvbWNmIz1Af3Hu54zXAi5wbOWwfvJ2oFVEisFiSIPSk6RfSCZ4WTMynMTh3dSZ4WjAqlQojGyPJ3u/BH0JMS+D4QO5uuEvBgwI6L+yMlbeVhDxRk1dDVkwWI34ZweLH0j1TyEshuHR3Qa4rZ8xfY8i9lcvdjXfp9UEv0RYyamUUbxW+9S+z1duX8j+w1vyfKHMzMwrz8/9Tv1sE2LnaSUhAIPQto34fhV1HOyH7Uk9HBMQyL2WKOaYA7r3dxR6npaGF4sfFbOm6RXxcDVJZuFuIe4H2P3++pp2cptXxyCHMgbHbxuLaw1VjJjN8w7M+Jfq9aDrN64SOvg5tLW2inS/A3FtzsfSy5Du770SQx2uANHKk0iGARlNbjKwcRLDIPsSe7m931xqZoc0GWF3PO9vIZDJkbTL0WvVQtijFueGdjXc4v+I8zTXNeA/2Rt9YH9+hvuTfyWet11peufWKZLalvs7V67h6LapIq6C2sJZXbr3Cgz8e8JPXTwDkuObw+trX+fX4r8w4O4P0c+k4hDmQcSGDtpY2SpNLOf3GaQb/OJgTi08QNCmI0b+NZtSvo6jMqpSAmQ3lDRQ/FuJg1D1l7E+xuEW74T3Qmxvf3cDM1Uz83FY+guLu+Wztk6+exMzFjDcy3njh8Tv08iHMXc2Ze3Muzl2dGf3naI7MOsLJxSeZfGgyLXUtyHRkGNkYoWxRcv/X+9h0sJHcV9WVdzuPK6uvEPlmJF79vdAz1KPzos64RQs92PNRPf/U/0z9b6x5L6p/wL5/6t+uIuZFkHYuje2DtuPYyRG3nm5cXn2ZO+vuiLYx7Rvl8DnhuPV0kwz6pp+eLjJanhx+wp6xe/Aa6EXHqR2pyauhrqROtHrq/mZ3dPR18B3ui6pNRWtTq8TTu/BhIUdmCwzDhfELUZgqKHsq5OSpbzxyXTnD1g/7l5lXIGxyw+aEadhK5d/NZ9eoXTTXNovBvEGTgtg/eT8F9wWFn3MXZ3FQVJxQzP7J++nyaheaa5rp9kY3ShJLxEY6fG44toG2uPdyZ0PQBpy7OYtBrgB9V/dF2aSkPLWcLZFbGLFphGglkH8vn5hPY3Dv5U7Eggj0jfWJ/SmW+5vv807ZO+JxVilV3P75NkGTBWVWVY7U+sexk6P4frWFiv/k/ZNW0Avg7qa7xHwaw8DvBkosDqz9rTF3FfK97ILtSD2VqmGVKNeREzE/gpr8GsGrPloKIkTMjyBsVhg6Ch2tYM34XeP52upr9k3Yp6HuK7hfwK0fbxGxMEIyvJDryFmWuwxdA13yYvNIPpKM92BNoC/jUgbGdsZ4DdTMeQubFSbxhH++oldFc+PbG1z+6DLeg7wlDVFrUys/+/ws/v+9uvckTfm/CgYGYTNWmlSq/bG/myNdA12GrB2ioSTJuZEjsP3GaLL9Gioa+Mn7J2QyGaN+H6WVEdhS38K3Nt+K1lftyyHUAaeuThrqOBCsXnUNdTUaK4D+X/antbEVrwFevFP2joRBGfO5YMsYMT9Cq1oHBA/4pVlLqSmo0ao4lMllyP4F5cvU0ZQRm0dwf/N9jY1y5uVMXHu4EjonFJ0I6QZbYabg3dp3aW1spbawFkNrQ4nto5GNEa5Rrlo/Mwjqr8qsSpYXLxd/VpJYQm1BLc5dnMVMlrHbx4rZOu0rYn4EEfM1wVOAERtHsHvMbm6tufXCzKX/ThlaGjLw64EvfLyloYXWxlYUpgoRqH56/Ck2HWzovKgzKpWKzEuZGNsZiyq0ubfm8lvkb5SnlLN71G46jO8gqmhq8mtIPpKMsb0xX73zFW0rP8Yg5hbV9k9IOydn14hdwuZ6ejC6Xrok+yfDBsR1aNqJaegZ6vFb998InhbMqN9GIdeV8/Cvh2J+ju8wQW1q4WFB6MxQLDwsGPXrKI3P1t5Oun0l7EmgKL4IHX0d8RwdsWkEBhYGXP/6Or9N+Y0KqwoOJh1kWfdlbIncgomDCUN/Hsq2AdsACH8lnAdbHtBxckf0jPTos7oPd9bdYc/YPQz8bqCQ49CmEtcJu452+A71pVBRSKNhI21RbZL72ZFZR7DysWL4xuG4RbsJ9nw3cyhLLqPj1I7iPTNyaeQLswifLz0jPaafno7CTMGuEbswcxHYsg6hDuJrTD40mT1j99D1ta5YelmSsDcBua4cx06OzLkxhzvr7lCVXYWOQoeUUyk82vGI3h/1FnNQ2lrbqC2spam6ids/3+bsW2fFzK6OUztSmVlJ/t189E30sfSy1ADmu7zahQvvXeCX8F+Yc2OORPFq7iqw7x3CHMSh5LqAdXj09WDExhEEng7ko70fEeERQd9X+orn39PjT7m/5T5lT8q4WerDE/0Qbt2FQ4dg96jdZFzMIHhaMH0/60tpcikbgjYw9OehhM8Jx9TZlIayBvSM9Dgy+wgZFzJY1fDM/ladrWbpZSnpiVqbWilJKqG+pJ7ihGKUTUrCZ4fj3stdAm4C/9L2LHxOuGQgBn+D6h4W+I/yR/aeDI9MDwyfGHLm3BkR7KvJr+HSKsEWzibAhrSzaZg6m4rMc89+nhpZxu2r94e9ufn9TRw6OYjXl7oyLmRQV1TH1c+vom+ij9dAL5winMi5mUPMpzH0WN4Dz76eRMyPEGzffKxoqW8R76HmHub8MecPUMGy7ss0Blr6A/U50HiAPpf7kN2cDcIyyh/Rf+DRx4OoG1E8DHnIulfX8c5kAUQyczaTfC+GVoYSy5zc2FyyYrIIfSlUwrYGOPXGKe5tusfbxW9jbGvMoZcPYeZqpjFs6bKkiwZTvc/Hfej1fi/kunJePfUqh+4e4uPeH+MY7kjc1jjKU8uRyWXEfCLcB+06DMQ16Rx1dRGAZuazXbAdvT/qLYKWJUkl3Pvlnug+8OD3B3RZ1AWHMAeULUpa6lvE43dqySmCpwdLQHt1WXpZauTuKVuUPPjtAScWnRCPobq2RG5B30QfHT0dMi9l8mbemyjMFKjaVHxh/AU2ATYi0e7hXw/ZFLJJtK2vzq3m3q/3qC2spSK1glG/jfoPWc2XP7qMfbC9YG12O0+8vtWOFfPuzsPG34arn10FBGvJ6txq5lybI97z1d/r/V/vA4i2d+oqSymjvkQYLimqFTwOesyQmZqKjYuD8iiwPMCgU12wbr2LzO6ZyjM3Npcpv0zhXug9ojdH49X7WW957p1z3Pj2Bt6jvYkLjwMEVV389nhsAmxE9nlbaxvNtc20tbSRfS2bysxKgqcFayX9qKs0uZSB3w7EIdSB7GvZWHhaYOFhQcCYACw8LcThv3MXZ/Hcyb6WTXNtM579PPEf5S/sG4ydMXDQw8xZGO5f+fgK3d/qrpUol34+nUMzDtFrqwCqmeibSIA+gGHrhqFnrMfTY09RmCnQM9LDuasz43ePxyHMgZ99fqalvoUxf42hw7gO6BnqieTO9tUeYL/21TUe7XiElY8Vv4T/gr6JvsTm9cnhJ7j2cOX4ouNUZVZxoGUEDHiT1iwDvBNKSPkzBYNGwW5xzrU5OHZylBAQc27moGeoJ4JKRjZGRK2IwjXKFT0jPY19pbmbOVOPTSXtXBqr5auZdXkWg74fxONdjwX7ZXdzid2h90BvwVK1sJbawlqQCftaA3MDLDwsWNUorFOfG3xOfUk9ZSll4v1TrivnzKAzlNqUEnXdDOeC40AH5DpyGsoaOLn4JACv3H6FIWuHYBtkK/YgVt5WjN89XrKfPjjjII92PBL/HzYrDGTCHsS2g63Y2+oa6IrrZ87NHG5+dxOX7i6493HHwtMCgD4f9UGuKyfnRo7YJ8X+FEv6+XTG7xqPTYANx+YfI3RmKB0nd6TT3E6EvhQqAlt6hnqM3zVefL3ny8rHSmIbPOi7QQz4aoBWcqqekR5LkpegMFNg4mDCje9vIJNLyU0lSSUk7E0gcHygJENJz0gPS29LOozrgL6JvuT+W5FeQfb1bEJfCsXYzljMuG1fu0buwrmbM93f6k7YrDAhI+9vEpB7tDtLkpcAaHV0UYO67evCqgtc++Ia1n7W9FjeQ+M56plE8SMBEHSJdKG5phmZXEZDmWZmX9q5NE6/fppZV2aJM4qm6iaufi6snSb2JoINoY8VCjOFCGapY0oayhtwj3ZnfeB6PPp6MHz9cN7Kf0vyNyw9Lfmw9ZnlXuCkQJIOJOHcxZmm6iZ+8vqJ4ZuG03lBZ/Ecy7ycyda+Wxm6bqhGFt69X++JFtpqMs2TI09E29b9k/cDSFxiABY+XEhdcR3W/tY0VjWiMFWI62hdSR131t9B31SfoElBNFQ0kHMjR1Q7mjmb8V7de6SeSSV2bSwZFzOYFTNLPB66Cl3m3Zkn+XsX37uIpbclgeMDGbJ2CE010hgHdfX+sDepp1J5euIpFh4WNFU3YWBpgI6ejphDeWfDHdH6U01+qSupo/fHvXGPdtdKTPy/WaOnTWPbF18w/T9h5fmnrj7Dx0yU/OzOxjtc+uASi+IXYepkyteWX+MY4cjL51/m9vrbVGVX0WFcB56eeIqySSm5Ls++dZa7G++K/++zug8lSSUi+N++1HtUI1sjSbSMkbURKSdTqC+tJ/TlUPHn5q7mGhb42qo9KU7ZpiRyWSTNtc3c//U+v0X+hk2ADYETA7X2819/DS2GZrQYmmGk30bB/QJhvtDdlfLUcvaM2cOQtUN4evypeL/WVqeXnsY+xF5jT6BsU2JdZs1Fv4u0vN8iznBu/XALZZNSyHlvN8dQE6B/dP2RqJVRggX65I781f8vmmubRbKlqaMpneZ3Iu1Mmkj6UDY/A1medHiCd5o3I/qOoOxpGfsn72fANwNoqWvBuZszvkN9hWvKWJ8rn1wBFUJe7MqenF9xXshdzlmGoZUhubdy2Tl8J6N+H0X47HAaKhqI+TSGnu/2xHugN5c/uox7b3cR7NNV6GolaUevipaQVtpXZVYlBfcLiFoRJZKPjKyNJPPFPWP3AMI5BoJQ4uSrJwmbHSb2cMfmH6M0qZTZV2dTX1pPxqUMsUfXM9KTgMS1hbUUJxSjo6eDW7TbP5l9/w/UP2DfP/VvWR69PVheuhx9E32ULUpcurkg15FrzX5rz4hWV3s2n7JFuBGkn0sn/Vw6rj1cWZq5VHxcPYx68McDjs45KmQx6crp/UFvGisbefjXQ0AYTtgHC/k68dvjWdWwSmSDynXlIps2YW8CCjOFJJ/hwqoLPNjygIUPF2qoHQB2j9lNbUEtg38cLD0OfT3IvJSJQ7gD17+5jkxHyFuzC7Jjwt4JVOdUc/Ytwf7jwe8PRKBQzfQDiFoRReGDQn5w+YERm0bgN8IPYztjRv8+mge/P+DSh5eEgXpTKzr6OpQklJB8JJnkI8n4DPXh/DvneXr8KSO3jJQMwWRyGcM2DEOuIyd0ZugLQ49fVB3Gd3ghaKEwU2Dubo5DmMMz9u2SrpINck1+DbFrYl9oAWjqZKrVWk9tO/OvKnxuuFaQpyqnivjt8cRvj2fELyOImB9B3p08EvYkED43nLzYPO5uvMurSa9qBX6PzjmKobWh1rBjENi16zusJ2x2mAYDreOUjnj296S2sFbD7qh9KLrCTPFfCtMGmHZ8GkdfOcrJJScZtk5qPVP6pFS0D+z+VncNsG/6yekvtA+V68hxDHck42IGt368pQH2PTn8hORjyQSMDZAA+Or6V6HVnn09OT7/OBWpFYzZOkbyWPvh4vPnpaGVIY4RjqSfT6emoIaIedrBLbWVmrbKjc3l6JyjghVu3EKtvxMxL0Lra5cklZBzI4deH/eillrJYzKZDH1jfW6tucWl9y9JLIhAWBd1Fbrk3cnDLthOw7rBZ6iPRtZjzo0cTr9xWnI+arvu0s6mkRubS+QbkVrtXAHM3c3FfLb/ybq/5T7FCcVarWlif4rlwsoLRCyMYNg6Yb0pelREaVIptYW1ZFzMYNuAbQRNDmLC7gnIZDJcurlgaG1IQ1kDfiP8GPrTUJHNVpFRQUliCYN+GISFvQWGTaXY5jym1cAKc9dAOs3vxP3N9yl+XMyJqSdotWul2a9ZZMKrm2tzN3PKUsrEa829lzuj/xwtyXdw7OSoAWD/Z2rm5Zmo2lQalpSdF3XGb4Qfn53+DPNKc1rut9Ac3CzYpqkEKyUQSBE93u5B6MuhIuDjPdCb8yvOU/igEGt/a1JOpqBsUoqDoC6LumAxyYKp66QWcupK2JuAiaMJPkN8xM8c92cc9zffx3+Uv3idtVesH5t/DPsQe8m6vSVyC44RjgxfP5zGqkZsO9hibG/M62mvUxhXyNa+W+n/ZX9sA23ZGLKR4kfFdBjfgaE/DQWQKMVq8mt4tPMRbj3d+MbqG/xH+ZO4P5Eui7uIw0ozZzOWZi0FhHM8bFaYeP7rG+sTPD2YqBVRL8ye6vRKJ8pTy0k6kCQSiJ6v9uCmvok+aafTuPHdDfRH69Oq1ypaP4LAfH+r8C3RMrfOwxBjZyta9Y0AKxoqGggYE8C4HeMAgQ3u3tsdE0cTWhpauPDuBXHI4D3Ym66vd5W8l/a2yZc/vkxRfBGTD07GytuK5cXLaapu4ivzrwBhiGUfYk9DRQNJB5MInioM+Jtqmtg1YhcV6RUsy1nGf1TtiUQ7Fu2g6mEV83+dLwELdA106TitI3r+elj7W/O97fcStW376yTlZAqnl55m6E9D8RniQ8GDArKvZzN843DJkFRdDmEO6Ch0UDYpufDuBZpqBIeEuuI6sq5kETpTGKSY2JuIA6uL71/k1g+3WJq9FBPnv4dYMjSU/gBW3ay4W3iXpA5JXJlzBRAylkCwOV8aupRm/Wbqjeu1ugFoq8xLmVx494IAcjqYkBWTxaUPL9H7w944d3WmuaYZHT0djsw9Qvy2eA0HAxDWImWLktvrb2PjbyNmk+ro6/Dk8BNMNplgHGyMXCb0O0dmHwGVkEeXdTmLggcFtOobkhE2BlOPZ/eHjEsZxP0eR/gr4Xj09pAMqCrSBQtNhzAH5Lpy7m++j+8wX/Lu5InD0RGbR2DtZ82MszMwczGjOEGw6w2dFaqRZQICwaylvgX7EHsR6FPnNaorLzYPQFivZM/YyepBT3sVh7WfNRELI8RepTq3WgQ31c9J3J+IqZMprj1cqSuu4+SrJ/Ef7S8O2WacmSH2u2qVI/ztWGGuwNzNHF1DXaJWRHH96+uiXd6BaQdw7upM9ze703NlT8He7+9BVm1BLZ8bfs7kQ5PxGeIj2kYWFBYQ9kkY1WbVjB+qmaFS5FBMgVUBnR50otwxg1qegX0tdS2Yl5nTqtvKNrtt9PN7RoxKPZ0KgGGDIUt+XcKBJwcInxsuAjQrq1aiMFPgNcBLHKYdmHqAx7sf03FKR/H6LUkSCEPuvdx5vPsxh146JP6Nj1QfETwtWPy/eh0d/cdoKrMqJbmcF1ddpCSphDF/jmHn8J1EfDKKX24Lx/jQIeG5ix4tkpAPylLKqEirwK2nG85dnBm6bih6QXqQAXKZnPQL6dSX1osDtxvf3yD5SDIzL86UMNjVqvrwV8K5/dNt7v1yj8MvH+b19Nf/Q9cDIxsj3KLdGLd9HJc+uISOQoevrb4mYGwAo38bzbvV76JqU/GJzicAKLpXUmvtTlG5cG4nrk1k8uHJjNyk3UZ9z9g9WPlYMeeaYOdmbGcsKlpbGlpoKG/A0MpQJJooTBUCuCUTst0T9iUw7OdhOHVxovOizlj7W5N4IJGEPQkM/GYgVdlVmDiaSBxJNnfajP8ofyYfmizeu8duG0vi/kT0jPRIPprM/in7MXMxQ/aqjJTKFKxrx9BiJ+xf59+bT3NdM6eXnqa5phkrHyucuzjTWPVs8F6eWi7sKRNL+KPXH0w7Po2K9Arx8Y5TO2LlbUXSoSQOTjvIhD0TRHvL9v1qTX4NSYeShMHo35meZs5mNFY1cmfdHRRmChHsK04oJu1MGk+PPaWxspHUU6mknkql4+SOGns/ua5cIwP2zz5/knUlixWVKzT66+ef/3DbQy68e4EJuyfg1tNNfM9VOVVc++Iapk6mUrAvoYQrH1+hvrSexopGur/VHcdwR1wiXZh3ex4qlQodPR1am1pBJdwzkw4mcePbG3j08RAVkHd/uUvSgSTG7RiHkY2RqHh37eGKZz9Ptg/aTkNFA2+kC0qS7UO2Y2BuwIQ9E6jOq+b2utv4DPbRuucCAUQIGBOApbelhlIcwKWbC0GTg0S7XnVv9qLyG+6HqZMpRY8EAptLpItk335wxkFqC4S90JCfhjyLJPn7VlyVXUX+vXx0Fboae/faolqUzUoxM1ldfVf3FffZ5anl9HinBw6hDrS1tlFXXMf5leex62jHkLVDRFeRuuI6tkRuIWxWmAiwdX+7O0mHkjBxMMFnsA+Gfxpi5WPF2O1jMXM2Q99Un9TTqeK8R93/q1XdesZ6dFnchYHfDMTcVQDpDSwN+Nb2W1y6u/BuzbuS961ec9POpBE4MVCi4NRW6pw4QNIbPfjjAeeWn2PyocmCfbGvFRP2TEDZrORry69pbWxl7s25EuWhS3cXui3tJiGJGdsa0+ejPoLqMKkEQyvD/78B/V6ZN49PV68mFuj2L34vFrjS2sK0NA/Ov3ueAV8Ka6uyWYmFu4U4v+j6WlfRQej619cxcTCR3Pvbl0t3F1obWtE10uXp0adc/ugypk6m2HawJe1sGqeXnmbQ94OEHkYFf/X/i6BJQRLFb0V6BadeO0VDeYME7FNXfWk9+XfzsQ2yFc/vxAOJNNc0EzYrDJVKhapNJWbBD/5BmB+qyUVeg7wYula4NjMuZpC4P5Hub3XH0suKa9ee/R25soXNEZsJmxMmzCdVQh/cVN1E7NpYdPR18B/lz9Uvr6JsUkoIELFrY+kwrgPO3ZzJvZmL73BfTB1NyarKwlRhSkV0hTjDKH5cTPT70bh0c5GAYieXnJR87utfXcc+xF5QC07pKKyH7UodR6JqU3Fnwx0MLA1w7+2Ojq9Awrg35R4D3xrI5Y8vA0Kf7Dfcj0+vf8rBMwc5Pu04cpmcFeUreHr8KbtG7iL9XLr4+vd/u0/U8ihsA20Z8tMQ0f3Jwt2CxYmLxf3jnBtzxLgHEEhT9aX16BnpSWYoHcYJcVMZFzM0ruf08+mSfHIQjr2FpwXTT09nx5AdePT1wGugF+ln0wl9ORRTR1NevvCyhCSobFKKgKLvMF9W1a/iRbVr1C6RjPVh24cv/L1/6t+n/gH7/ql/i5LL5fj7+yOXy9k1aheGVobi8EcNwk09PlVswA5OP0jmZYHdq60qMipoa23D2teaoIlBuOW50VDeQGVWpdZBdktDC6o2FR2ndKSusI4nh5/QdUlX4rfFc+uHWwxeI+TSlD4pxXe4L2ZuZi9kvZ567RTW/tYSsM/Yzhhrf2vketpBpt4f9UbfRJ/gqc82zfun7BcbYlMnU/Lv5ouWK8Z2xgRNDKKppgnPfp7k3sqVeEsrW5Tk3MjB0MqQ6PeiSb+QTnVetRhsvnv0biy9LRn8w2DC54RLAMwO4wV2YXVuNSYOJpi5mREwNoBOc6WMQplcRvGjYu5uvIuVr5XGECfm8xge73zMzEszKYwrxNDaULQoBWGDEL89ngNTDzBs/TBJgxk8LZjgacE0VDRo2Gle++oacX/GMevyLJTNStLPp0v+bnNdM7+E/UKH8R00LKBAYLVUZlXSWNlIRVoFoS+HSmwxnh5/ilNnJ4klgbr8Rvgx7848Tr9xGgNLYSNY/LiYm9/fxNzdnJLEEmrya7RaM4BgOfIiy9LKzEqyr2Vj4mDyQoWDsa2xJGRYXQozhZhxpM0O8c6GO9QU1JB+Lp36knpeT9O0pMuLzdNqDenZz5NZMbOQ68ixD9UeGL598HacuzrT/wup8kBhpuDlCy+TFZOl9bVzb+US93scr6W8phG8rq6cGznc/+0+3V7rpgFkmTqbaqhS2ldrUyulT0oxtjUWBz89V/Sk54qeospN2aSUgBEgbA6zrmbhHu2OpZelxrW+f9J+qrKr8OjrofXvVuVUcXfjXXyH+Wp8H51e6UTIjBCuf32d2pJaQjc9a/KVLUoKHxRibGdMlyVdtOaPPTnyhOtfXcd3qK/GMCL63WgyLmbw5PATUWnpN9IPuZ6co68c/Zd5SCmnUohdE0v21WwmH5ws+b7UQ0y3aDdx8/A/VZVZlaIlrzawT71RuLfpHoN/GIzcUE6/T/tx/avr7Bm7R2x41cxClUpFa2Mr/iP9qcyppOvWrlTpVmGKqfh68+7Mw9TJlM/0P6PZt5VDL9ngVhqNfYguI38ZSY+3e3D568u88tsr/DbnN5aFL6OttU0yaJh7cy4yuYy823kozBTYBNi88BwGwSbT2s9aoiKI/TmW0qRSCesO0AoY3PjuBoZWhhyde5T3eZ+nvk9pSGmgvGc5vT/oTemTUtZ3WA8INrOAhkpt2Lph3P75Nv6j/ZHrykUijLoeFQuM+4G6AzEYbcDVFVeJfk+wCYleFc3Vz6+SdycPAwsDrH2t6bqkq5Bvp+XaBoj7Iw7f4b6S66u1oVUc0N/fcp9zb58Thw72ofZMOjAJ2yABPLMJsKH4UTHZV7OpzKrExN5EBBrbWtswtjMWlFp+VhhYGtDrw16M3T5WooZtX96DvCWKkYIHBfzZ608GfDOAqOVRnFtxjpvf3eS11NckA+ABXw3Qei95cuQJJxaeYNj6YfgM8UHPSI/5d+eze8xuzq88T7oqHdtiW4wvG7Pj4A6mn5qOXEfOn73/xHuwN6saVzFhdCvhZ76m1CUUGCMCAPVl9TTXNGMfbC/aRD3a+UjCJq4rrtOqio35PIbaglpq8gXrpvZVllIGICEynFh4QiApmSrwH+XPN9bf0NbSpjU7tii+iLzbeRjZGGFsZywqXs+vPM+d9XcoW1JGhW0Fv8/+nbgP4sTnGVoZMm77OBobG9GV6TJi8witLFgQhqp6hnriuluaVMrN727i2sNVA+yrya+h6FER7ze+T3NdM1lXskSAIWB0gCQPs7Wxlea6ZhRmCpIPC9ajekZ61BTV0Ptyb6rNqgXQsL9Scg7NCpvFuxfepZBCdC2F8y/zUqY4BJnABP648wcW9Raisq2pponsq9lY+1tj5S0oCU8vPY1rlCthM8MInRmKZz9P8Rg0VTdR/LiYxspGwmeHEz47nG9tv6W+tJ6QGSFiBuvzpWxWcmrJKeH1+ntS+KCQjEsZnHv7HMYYE1UdheqOiqxPsrDwsKAyoxKZXEZWTBaDvh/EjUvCumn0N/lbpVLxV7+/AIHx/HyeqFd/L5blLsPA3AC5nhzf4b7oG+uTfEw4ns5dnXm49SE513PEHKi0c4IFvmuUq9gn1hbWcu2ra3gP8ub619cpSSrhzdw3hfPC3wb3XlLbtoHfDhTUoE6myHXllD0tw9DaEENLQ7q/1Z2b39/kM8VnqNpU9Hyvp5i7AsIAdnHCYjYEbcCjrwfWftasC1hHh7EdcO3hSnNtM4n7E7HwshCf075XbV/tHSvgmZWWqaMpekZ6rJavRtmspPubwpDfd7ivQDKMcsXMxQzP/p4aPUurYSvF9oJKprVNkwFu0GiLvE3O7lmN+BWNof1V49nPk+/f/h4AWYG0T5mwewIbgjYQfTWaCpcK9M30JeBb8eNiiXMGCISuoMlBElvJG9/cIO7PON6teZe6EmnmVk1+jdhbtTS0oGxWojBV4BLpgkukC1lXszg04xD9v+xP9KpoGqsaMXczx2VMBNtP22CmTMP7zm5ubhxO9GthGtf3w78ecvWzqyxOXIxtB1us/ax5WibYZ8uQcf2r6+TdzhP79bjf4yhLLnthDz507VCGrh3K4z2PsfKxQt9Yn30T92FoYyg5Z7b23UpZShlv5r5Jp1c6iYqqMVvHoFKpKE8tlxDfZHIZU49PpfhxMfFnzVHUliJvNsF/hD8WMRY4dnSkNLmU0ielOIQ6UPiwELsgO6x8rOj/RX9Rkf18pZ5KFa3A2+8PQQBRXHu48uTgE4b9PEyyPyhJLCFxXyLRq6I5Nu8YCjMFCx48yyTs80kfrP2sUbWpOLH4BE6dncTeFARgtrWhlZb6FlyMXaASaq09qWp7piTWN9bXcCwwMDdgypEpZF7J5PzK8yQdSGLivon4jfDDyNaIuTfmknIqhZ3DduI1wAtdA12cIpwYsnYIVr5WbO60mbDZYQz+cTBfW3yNmYsZy3KW8VHbR2RczOD2utvifaG1sZVxO8ZJHDpG/jKSkb+MFPv79lVTUENVdhV2QXZaexaVSkXWFeF+qWxWUhhXyOmlp+n2Rjc6jO1ARXoFtUW1OHV2QkdPB4WpAnNXc3QUOrQp20Ty2eGXD2Pla8X009Mlr+810IuF8QsF29zXTxM0OUi0xv/G5hsCxgQw+eBkNgRtQGEqfF8VaRXicW2pb2Fj8EYq0itQmClobWxFJpMx/fR07v1yD1WbiqJHRQROChSBsrRzaaSdScN3mC9ZMVmUpZRx/avrGJgb4NHHg7itceTfyWfwj4PFe17oS6GEvhTKnnF7RFvJ9mUfYi9aQv9nytLLEmN7Y740+ZLQmaG4RLpg6W3JS+dewtLLkrzbedQW1VKRXiEhNwZODMTUyZRdI3dhG2grOX/vbb5H0oEkavJrqMysZFnOMvLv5mPtZ61B0rTysRL7pAe/P+Do3KMAYn7uvkn7KE0q5eULL2NoaYiuoS7d3+wuruGf6H6C/0gBGFcTu9TOKLtG7iLtXBrvNwruReWp5WReycTUyZTQl0PJuJgh7ht0DXS5+oUQr+Le213sc9XVVN3E0xNP0dHXYVnOMkydTMW4GvX3HftzLA6hDmIfq7ZpB2H/KJPJkOvKMbQyxD7YXgQjMi5mgEwgj7U2tmLX0Q4jW2F/eXzRcSozKum5siexawRwp6GsAccIR3GPWZNfw4bADXR/uzuDvtWeVa+u9vO8/5Pl4ODAl19+xYj33uN4a4tWwC8WGKmvz4fL3qfxRCMJdxIY8OUAlC1KHu96jJmLGX9E/0HkskhJ7I+6V39Rqa8RgL6f9KUwrlCrqg/+JsavH6axP9w5YicV6RUaDlLxO+K5sPICYbPDiPk0RlSkgqCMq8ioIGxWGCcWneDeL/ckjluA1n1+4cNC7m68K9htewrvwyb7Ph7xxygbNIV+X/TDPsSe1NOpJOxJYNyOcdgH27Msd5m4VibuTaSppkkC9i3LXYaeoRATceq1U8y8PBNTR1P2TNjD5P2TiZsTJ1rubwzeqPX9NVU1YdfRjkHfD6IspQyHUAdRfd1zZU/J77bUt3B/y33sQ+xxiXTh5KsnCZkRQlN1E4VbCpn92mx0woXnVmZWAsK1kxubS8pvKZwbeI6bWTfp4dYDmVym1d1J3ftZeFg8Ix4AOvo6ku/4eZFJZVYlP/v8TI93emjsy86vPE9ebB4rq1ZKfu7Rx4Ox28dKXuvB7w84Nu8YUw5PYVnOMgwsDdg+eDs513Noqm7C3NVcAzR8nvDevlRtKvZP2Y9zN2d6vNWD4GnB6OjpEDRFU+n9T/336n9rzXtR/QP2/VP/NqWvL9xQ6kvraalvoehRESYOJlj5WBE6M1RiW2juYS5uCk+9fopHOx+xNHOpeFM6MOUANQU1LMsWWOmmTqaYOpli19GO+tJ64v6Mw7GTI/Yh9uwcvpPGqkZyrucw8NuBRMyPoKW+BSMbIwLGCpY0hQ8KubvhLnc33GVx4mINNmBzXTO/dv6VDuM7MOavMRoKRHUOYFNNEwdnHMSzn6dE9q5NAdRY2Yipsylzb86loaIBEwcTCYsEBIZn7NpYcSOurssfXebal9dE//z2+Q0g2IYmH03GJsCGiPkRuPRwERjtMmEDp7Z1ADSyTNqXusHYNXIXPoN9mHTgmb+4Whou05Gxc8ROfIb4MPWoVDVScL+Ax7sfM+CbAVrzln7t/Ku4Sc2KyaLoURFN1U2UJZdx8f2LAFrVV3pGepQll7E+cD3d3+ouASof7xasbpy7OpN3Ow+/kX6Sjd/Vz69SnVutFeyT68hx6uwkCVQOmhSEV38vtg3chqpNxRuZb5B2No3WplaNjLPACYFcfP8iBfcKJA0TwNMTTzm15BQvX3xZK/t95/CdqNpUjN0+VsiiaTeM1DfWp/OCzjTXNVOdW42RjZFE3ZewN4GShBLCZodJmLfqyrycyaDvB2m1TlKYKjTyEtpX/r18Mi5kvFAZA2gM7tTV64Ne9Hi7hwicPl8XPxCOVeqpVALGBEjAvsurL6NSqsT8jPa1pdsWFGYKhm0Yxi9hv9Dz3Z4iENlc28w6/3XYhwrZLdoA+MwrmRx7RQCgFj1apDGA6rO6D7qGulrPEYDqnGqufSmA0vPvzZccGx09HcGK7HKmsJHf9Ox5DeUNbOm2hS6vdtFQWIJw3VakVTDilxEaGYbqurL6CiWJJSLYZ2JvQsT8CBor/rXVSdTyKPJv55N+Lp3WxlYp2KdsI3FfonDOvfYvX+a/XM21Qj7i8woldfkM8WFB3AJqC2oluSIGlga0NrYy6PtB6Jvqi+duU1UTX1t+Tfgr4UTujsT/R38WblnI2Dlj6f95f/RN9GmsaiRuaxzVftXEe13noXcsrhXP/PStfa0JnBnI9znf45znzHrr9Uw6MImAsQF8qvspgRMDxWHHGo81eA/0ZuoxTUVc0aMibn53k7DZYcT9EYdHHw8J2Jd2Jo2MCxkaYJ86v7W+tB4bfxssPCy48d0NUa0GkOmRybiZ48RBqzr8O+QlYQjRnvUpk8nY2m8reoZ6TDshBEK1HxKAMFTNO5eH3FOOykiFczdnyT3Xe5A3Vz+/yp+9/gQES2v7YHuJ6gfg8MzDmDiaMOCrAbxT9o4GE3vhw2dKWPWmas+4Pcw4M0Pj3jNx70SufnGVi6sustZjLX0/7YtMLqPTK53Ijc1l96jd4u+qVSrPV1N1E0mHkrAPsddYjxP3JQIQ80kMYbPCsAuyw3+0v0TBfnD6QZDBuO3jNF5b31gYnu+fsh9dA11RWT/wm4H0er8X6weuZ6TpSCotKsnNyQWE+3pFukBG0lXo0qYjI6fDQBpMn60xT448IeaTGAruF4jW5SDYZXZe3Bn7YHuUzUpcumvmjebcyOHS+4Jd5tLspSIjuLGykdQzqZg4mDBswzBcIl24/PFlbq25JViQ+1uLIEboy6HYh9jT7XXN8UnKqRQurLwACBaPi+IXEb8jnvLUclyjXPFL8KPQrJBU31QJYKpqU3Hl0yvc3XiXcTvHafQ8Z5efFRRZ8yI0QFn/Uf7YBduxd9xeAMxczcTe7s/ef1KeWi6yip+3+GxfdzbcEW1cgyYHiVnCZQ/K6Hv570HPUWh6pUmifr704SWmb57OqWmnaMhuAAchn7C1oZW21jZetn+Z60+vM3nvZOI84uiyqAuVGZXsHL6TPp/0ofcHvWlrbeP+r/dpa20jbGYYpo6mkvuC3wg/SY4cCMpNhbnihcO1I7OPkH09m5mXZmLiYIKyWcnmiM2S33HLdkOVp6Iyq5LxO8dTnVvN/V/v4xrlKpw/l6Svee3LZ5TvQT8MEnMOe3/YG89+nuga6EpcD9SqILWitDCukAPTDghuD4a6qNpUePbz5L269yT32sbKRmLXxqJvqk/3t7rTUNGAjr6OeF48PfFU6D3+7h16vN2DHm/3oK6kjrzb/z/23js6yrJr+/7NJJNJ771XEtJDChBa6J2g9F5FqSqiKCqIXbGjoiJFVJAmSIfQOwklAZJASCG99z7JzHx/XMyVDBN9nuf93vdd3/0t91r3ujHTr3Kee+/j2MdRyK4xuxj04SD6vdGPYZ8No625jcqHlRiYGuiQHGTGMuwC7ViavlS8tyf+MVHcQ628rQSGc6ehTo3Mv8xIxuMLj5HqS7WklUAgHLQ1tWHqaCq+79ttb4tgRMqOFBpKGxj781iufHIFPQM9ph/pMORLP5COUqHk0c1H+Bf4E5YSRsGBAkjWPifj9pjSIn+e0+P1UenpeiZtj9/OG5veYMxXY7jJTaJeEBqDdoF2eL7ryZbzW7jU/xI/rROujUW3FvH4wmNRvrCusI5Hxx7h3tcd5yhnHWn80FmhOMc4oyfXo/fLvYmYH4GiXsGW3lvYOWYnDcUNdBvbDbmFnGufXePFnBfJvZTLrR9vYRoXRQMmSAz0te7pwgo53a9uJavHJOptvZCam6JqV9FS04LMWCYez4DxAVh5WWmx2FVqAcirba0lY1wGs1d2NEs1IOWF9y5g7WON91Bvyu6XkfBqApP3TxZz2c4eQ5WPKjGt1ybYufRyEZQMnkTa/jQKbxQy8N2B6BvqM+es4HumkWgzdTKl2+hudBvdjU1/ZtIt8XdqBozF2DYCj1gPah/XcnDOQQpvFDJ281gOP3dY9HB8WgKtoaSBg3MPEjgpEOcoZ2JWxGhNulVlVfFD6A/0XtWbsZvHomxTsjlmM4YWhkw/Oh2JnoT+b/UXPQRt/G3IPJ7J3sl7aa1tZeqhqfR/U3hMpVRx68dbpPySoiURqfFxt/a1pjSklAGeA6jyEo6tWqUm8dtEgdQ63Ff8zr8O+5XQmaGcXn0agHFbxpG+P50b39zQ8nGz627HwPcGimCjTTcbbLrZoFQoGfHNCOy624n1TV1BndZxeXDggXA9Rjpz9dOr3Nt5r0ubCAsPC2pzawXJ1OVCbnlv5z0tclF7SztfeX5F4MRARn07CrVKTdicMJwinTCxM6HqURXlqeWicsqVDVe49cMtXi1/VegPjA8Q8+zG8ka+dP1S/PzeK3vrqMMYWhhiGGKIXXc7IhdFir8x+3Q2qOHBgQcABDwTIK6rwdOCMXU2FYEZubmc/mv7a6nD+A73xXe4L5c+usTZNWdZdGuRuJenbBeUiSbtm8TnTp/jEOrA8szlYr2dfSqbezvvMexz7T0m/1o+Dw48ED/3n+Lq51dJWJWAVF/wFOzKjkLfUJ9J+yaJALncTC56iD3tZQ0CcHVo/iGQQPz2eB0pwiPPHxH/7Tfaj7L7Zfw69FeGfzWcXi/24vDzhzFzMtOps628rQieFkzMshjRTsXE3gRFvQITexMW3Vqk812e/e1ZnXpLqVCSeTITz0GedJ/YkbNu9BPsNKYcmEKf17Rl+lVKFf7x/rQ3txP3ThyN5Y08PPQQhzAHLD0sqSus48/pfwLCeZ+wcwLfdf9OtDtRNCo4/dppIp+PFNdStVoNagFQ2j5gO7V5tawsWElAfICWks6Fdy6gVquZf2U+3kO9sXCzENfY5spmQTHI15qRG0dSdLOIHYN3MP/KfIysjTi86DDBU4Pp/3Z/3Ppo1w1/F5p+3v/pWPr8ck69lkA/EugjMWChuhUHoATYYWjIhbY2Pv7oI15euRLFWwrxdYoGBS3VLVi4WWBib6Ij3W4fZE9JSglbem9BrVJj5mKGS08XLX/h5upmEl5LwHuwt1ZP0GeYj47VRVcqU1EvRIEEHeBGZizDzMUM5yhnxv48VivvGPntSNEz3jnKmdbaVqQyKS01LRyYfQB9uT6KBgVx6+OwD7Hn5MqTuMW6EfV8FKEzQjG0NERjLdZqbEmVUyBKUwv6vSHIdN7Zeoecczn0quqFVF+qtYbNPjNbR91E87j/OH+s/azFelBfKqxfnVVNph6aSn1RPSdfOUnSt0ksvr8YZauS7DOCr17RzSJiV8Xq+Id2DkWDghMvngDgxZwXmX12No2ljaJX5nj5eIxvG3Pp/iUGvT+I3q/0xsLDglOrTtHrRi8eez7mtNdp9L/Wx8zFDFt/WzFvzb2YS+DEwL/1tFMpn0zuGQmTey01gtep5toxtDAkelm0DoFq55idZCdk6/hWPzr+iJMvnWT4V8PZP20/BmYGTD8yHQt3C7o/0x0LDwvMXc2F4Y0r+ThGOIq2B/8U9cX1lNwpwTHCUagxJEKtqzl3vV7qpaWQ82/874n/W2teV/Ev2Pdv/EeESqXi3r17hISEsODqAvKv5vND6A8M+3wYYbPDSNuXxoODD3i9RmBFdPYuMXMxwyHEQYuJGrEgQtQuL71XyuPzjwUWtb8tNbk1/DXvL5HJUvGwAlMHUwasG4Brb1fk5nKxaegc6YyRtRHn3jqHXZAdLj1dupZpepJsIUEsfrr8nW0q7v1+DwNTA50C78yaM2QnZLMwcSESiYSZJ2ayXrKeT20+BWDKwSli8vbo+CMOzj7IiK9HYOVrReCkQK1GigYA6/FcD34f+TuOEY5aU1fLHy3nQ5MPub35NvpG+gRNChJ1+1tqW2iubMbUyVSUh7j4wUWStyYz/8p8sehuqW3h0geX0DPQw8LNgowjGVq/p9+afuJUyJgfx2g1jgGOLRdMa99ue1s8dydXniT7dDbDPh9GUVIRXkO8xOIgdW8qSd8m8WLOi9zcdJPaXMEjsHNBDkID9oWUFyi9V8r+aft19Kjd+7kzdMNQnKOcUavVIhM24bUErH2tGfbFMA4/d5iE1xJ0pDSbKpqoyqzC2s9anLoyMDHAwMSA3qt6o2pToVaq2Tlqp5isPx1p+wTQ5OkixHuIN+N/Gf+37DAQZJk22G5gzrk5OhIsZ986K3ogzDk/B88BHY9P2jMJpUKpcw40kfBaAo2ljaLcXedob2mnsbwRiUSCsZ2xjizm0cVHMXcz73LyoLW+lSufXMG1tyvdRusyqAxMDJAZydg/fT+uvV11vFMuvX8J/3H+vF77ujiVqomW6hbK08q7lGS19LLEwNQAY1tjBr4/UCv5enjoIfVF9UQtiRIbHk+H9xBvwueF01Dc0CXj+p+8FQEcIxzp8VwPbm++TW1erVZTt6G0gbqCOiYfnExGTgYqlQo9PSG5NTA1YMinQ3QmGDVReq+UtL1phM8L1/IO08S5tedob2nXSiqVCiUjvh6BVE/KuXXnyDyeyYivR+iAPWbOZsw6PQu1Uo3MRPu99Qz0WKtcy93f73Lpo0tdmkL/r4Z9kP0/ThzCEwZdJ5WT0nulTD04VSepBmEqKHRWKMY2xhzte5S3H74NdDRvlQoluRdySfo2ifJu5dyMuolTkROy5jryLteQ+G0i9sH23Pj2Bndn3cWq2orQ2aGYu5mDGvzj/bWaoUM+GYKlh6XAxnzjDBN2ThCnORuKG4SJlr5uvFL8ig5TdOLuiV1OIOwau4vKh8IE1rAvhtH75d7MuzgPiZ4Eax9rnD93prihmO8WfIeJnQkH5xzExNGEV4pfYXP0Zj61+ZSwuWFc/+I6L+a8iKWnpSDvK9cX5ZqfXhczj2dS9UcVvA2ttq1a0oyaaUkNEzlocpAoKfV0PD7/WGze/J0crCY84zwJnx+OvlyfrbFbsQh2o2DgTFasAMcnt0DYnDDOvikQO6oyq0j5JYVuY7ph7mJO6KxQ7v56VzyvDSUCO9wuyK6DkVxcz19z/xKmSmpaSNmeQuxrsdgH2eM9xJvEjYlC4V/TQtjsMB05nbqCOgquF+AxwIPQmaFa9533EG+8h3hzZcMVqjKrUKvUXP/qOrYBtviN8kM1U0VicSLVsdXsX7kfEJrGqjYVPsN8BL/TFjNKfTp8ce7/cZ/904Tn9n6lN03lTVz/8rroceUS3eEldfWzq1x45wJTD00Vr63cSx2TfE0VTSLYV5Nbw/6p+xn43kCxCVx8qxiXaBc84zy1PDzH/azrL6mJoElBOIY5UldQJ5I0ji4+iltvN2aenMld07tU2FTQ43YPfjr0k9g80/hxgTC1+HQkfpOIz3CfLolPBqYGOIQ4iB5BnY/B1L+mkrQpiaRvk0j6Lok3W94U96javFpK75XiEuOCiZ0JDqEORD4fiYm9CYM/HCzmRFbdrPh+8ffYVtgyym6UzvonkUrwtvVmwe4F3Em4Q697vZBIJPw24jcMTAQZr8UDFmM611Qkhpi7mjNuyzgcI4QL2cDMgNeqXhO/W3urABR2nmDUxM0fb1KUVMT4X8Yj1ZOSezEXPbmeKCekCbml4A+lyQXaW9uFxoMEph2exm+3f2Nf/j7CI8JZN2sdTRVNtLe2k7w9GRDkUk2qwefWPtr9A4HhNJQ2YGJvwqrSVYCgdFCeWi6qShxdepSG4gae+fUZlAoltXm1WHpaivfblU+uUJEuKGB8YPgB3Sd0Z/K+yUiNtdc+Kx8rVhauxMDMQIscV5tXy+FFh8k6mYVnnCdzzgmSxuffOY9jhCOJGxN5fO4xfdf0xa23m9CAKW9i6CdDRYJK4v58Vnvvo8+qXvg51aNqU+EZ50lTZRONaY2YOJiIHq7iOZZItNQxNvptxMLNggXXFnBowSFx0ub2z7e59vk1Ju2dhMxYxjc+3xA8NZjQWaG01LZoTV8lb0um9G4pzlHOJG9LxjnKWauxp/ECb61tZRrTaJG3gPbwFgB5Xs3UmjzCvMoIeWMV0DEhUFdQR9PJJkwaTWixa9HKV6qyqih6UESuh/Z079PTiRXpFRxZdITRm0Z3mX96DfLS+t6GFoak7kmlrqCOqCVRPDryCBMH4f6KWBiBgZkBR184SltTGw+Vg6gPfY5IE9CiRjk4UOUUSL2NB9XOgTj2hapMYTq935v9GPS+IEfqHOkskkLu/n6XSx9cIvzrcPFtan1rteReJ+6eyA9hP3Dn5zuAcN9ZuFlg191Ohyypia6k2DVSbyDk3Xsn7gUEMNHC3UKsl1RtKnYM3kHwtGBGbhRUD1qNrSjy7YeBjSNtLW3cTb5L07kmCm8UErMiRmDz//rM33qjtre2k38lH+doZ3os6KGjpiAzkuExwIOK9Apu/nCTvq/3xcrLiuzT2bwvf19nCjB8XjiKeoEIWHavTMg5nlwmUj0pC28sxMLdgqxTWTw69ojYV2OxcLPAws1CIGM1gqxNRsiVZDzyvkTVvoITL57AZ7gPv4/4HYBFtxfR3tIuTiIB4mRT57oj61QWSd8n8fAvYRJ4ZeFKsbmqZ6CnNUkx++xsFA0KmiqaKL1bim13W+ZemIt9iLDGhs4KxcjWiKaKJvE9iu8U01jayIrMFahVaqT6UnF99ejvwZBPh4g1o0QqwcbPRiRKSfWkWnLSbrFuvFre4X8dOCEQa19rnf0BBDArZkUMxrbGeA/xxtJTAG86N8vbmtpoa24TczFNFFwXiECaqbHO5I6A8QH4jfZDX66PRCrRmm6DDtDJytsKj34exK2PQ2VsSnY2eHvDgHcGEDYnDJmRjOFfDMfYzljLBmL0ptEM+2KYVoM951wOp187jYGpQZekw6fj3m9Co13VruKv+X/p1Ec3vrnB2bfOMufsHK3cOfdiLhUPK4iYH6GTGzdXNfPgoAB+OoQ5ED4nnKRNSZg5mxEQH8D0o9PZOXonAMO/HI7cTFC30dQEDw48ENayd6AkuYQzb5wRPG5HdxP3y7wreRxdfJS4d+J0fmdZahmbgjfhMcCDuefnAgJZ6PLHl5l+ZDpb+25FUa8g8oVIrYngoClBFCUVdUnEKkws5MzrZxj6mdBXKLlTwh/xf4iWIOYu5kzcM5HEbxIxsjaisbyR4OnBIjArN5ez8MZCLSLS115fY2xrzKKbi/Ae6t2lZyIIdcTvI37n3NpzjNqo/Vsn7RG87NL/TOfsW2fpvbI3o74fhaWXJQ0lDVQ8qECtUmtNvv1TdO7naWrb/1MhN5fzxrtrCF4bwiXLCl5pS6StrRYPdwsmzJ7OjoULsTS0pL6oHhMHE/E6M7IyYvKfk8k8kcmQT4dg5WXFnzP/xMzZjKGfDqWlpoXavFrx3gTBo1ID9iVtSiJ9fzo5Z3JI2Z5C0BQhn/if+J91RaYD6P5Md7o/073LxzoTFjtPnDeUNJB1MguZiYyW6hYyT2QiiY5EnXQLZauSsFlhIijV/mSJrrf1pt7WG5dOW37E/Agcwx3Jv5aPXXc7FA2ChYWJvUmXhPyqzCr0jfR1bE/0JHoYNxoTsDWAVNNUgiYF4T/Wny/dv6Quv45uY7sJChi1rTQUN5BZLMgtO4Q64BnnKeZyJ14+gVqlZuTXI2lvhy83dRDDVe0qvAZ6cfmTy6hVaqp6VxFvGs97297DxswGYztjjiw6wvRj0ylJLqFN1kaNZQ0u8S7IjGXsnbiX8Pnh+I/zxzHMUed8ZJ/O5sRLJxj66VD8RvnRWNrIFy5fELMihpFfj+RT20/FiV8Q5L6fvrdA2Bv84/111kWlQkllRiU1j2sEme4n+4rPUB98hvqgVChZL1kv1hSaa7epsokNthuIWhwlEoUzT2RSV1BHj4U9yLuUx74p+5i4eyJBk4OQSCSsVa79Rw/mf+P/XfzfXPO6in/Bvn/jPzIs3C2IWx+HW6wbLbUttDW2/e1zNbJ8nSNyUUfDKOtUFgmrEsT/du/nzrQj00T2r8arIutUFte/uI6qTYWJgwmWnpbIjGQo6hU0lDTgP96fMZvGcP3r6zw68ojJf04WmxQGpgYsSV3yt98x40gGRbcE1sqbLW/qJLa/DPqFx+ced3hiPbUmewzwoOhmEdY+1tgHCxIkdoF2lKWWcfnDy+jJ9fgx4keWpC5BIpEQ9UKUyO4988YZGssbMTA1oNvYbjiEOKBvpM+apjXc+PoGB2cf1Eog7/52l+PLjgPC9EbO2Rzu7rirA3pIJBI8B3riN9qP2Fd0Tbw7x9Pm1yAUCE49nLTOXXZCNmX3yzj83GFqc2u1PMt6vdiLwAkC62Z19WoqH1Xybbdv/xbAcghxYMl93XPSuXnQOa5uuIqpoymvFL+CslVJ1qksHbAv+0w2+6fuxz7YnlHfjcKjvwctNS2C5vqsMMrTyrm36x7DvxquM0UCsDlmMxbuFloTkJrQTPBc+vASThFOImNUE9OPTufxhcfc33VfR+azJrdGBPoAHWbS3zXmAT53/pz2lnbC54bz+MJjLZAQBHBS49Ey8+RMnem/2Fdj//b+VDQoxO/lGOEoStRpor64nqbyJlJ3pwrX/Yvar1/6YCkGpgZdggYjvhJkf+7+dpfwOeFaj3WWmHka0EvblwYSCJ0ZSmNZY5fHxtLDkvitut6anSP9T8FDpLMnhyZkRjLi1sfRY2EPLT8vENjFp1aeYs7FOUjNtNcBAxMD+rzah7T9aeyZsIehG4ZqsV4DJwbi0d8DE3sT1Gq1WFhUZVVhYGrAw0MPaWts05KmOPvWWa5uuMqKrBWib1HxrWIdsE/RoKC9tR0jKyOdgkXTCL33+z3yr+T/bwX7QABAW6pbsOlmo5OQpu1LI/3PdIZuGCo2ThJeTSD/Sr44zfxjjx9xjnZm7I9jMTA14Jkdz5D+ZzpXPrkCwKYXNjHpDaGgTd2TysX3hOOgp9bDotaC5396nmKfRKpz7EjdnSpcj4BVtRUFbgWoJqlwCRQAhil/TtH6fhqAOv3PdMyczLRkejUTLXoGel2C0n/XfIxeGi0CNS49hc/tzOwf4j2E9Ip0Wp9UbRqArSqzSmTCW/g74fdMsFjgTT04lR1DdvCB4QcMfE8o2purmkWfh1Hfj6JlYQuqSyou5V3ix5s/8nyUcL+qVWp+G/6b6IuoiaNLjpLySwqvFL8i3qOdCQNFt4owsjLSuobv/nYXA1MDcX3TeNhaeFjy6xYF9p99wsYrffjgQl8uvHcBcxdzZp+ZjbmbOQamBvR8sSfWftbIjGT0faMvUj0pZq5mXPzgIvpyfa58ckVrOtrcxZxph6dh7WtN7sVcUnakEDIzRJjiG+fP6urV4hR9VzH3wlyOLj3KkUVH8B/n3yXI3udVgcGtbFNy6pVTBE4S7lPTSabcP3efhb4Lxec6RTjR761+yIxkbO2zFTufWAq7d+wzhxYeEv89YO0ASu+VcnXDVcxczHDv6y5KoIXNDgOJ0JhTtavEnCJifoTw2Q6mNFU0kXE0g26ju2HpYcmkvZOwC7Tjz5l/0lrXyrRD0+ixsAcVDyq4+/tdfIb5iOSX2z/fpjq7Wkea2crbSoeJP2nvJBHsOTL5CA2SBsJSwrQkp5sqBIDPrpcdfqP8+Mb3GwLGB4gejy/mvCgSlqpzqnn410N8hvlgF2hHa30rgz4YJPgGP3X87QLttMgRHxh+QOyrsQz9dChZCVkcXniYGSdm4DvcVwRnNb8vfX868dvjMbY3ZvKYyRx5dITQuFCdzyh8tpDtLtsZlTKKvgF9UavVpP+ZTlFSET7DffB5wQeLKAsGzRqEiYHwm42sjbQIXRKJRIsodvH9i1x6/xJL05diG2BLQ0kDuZdycY505vHZx6TuSWXMD2Movl3M9gHbcenpwsLrC7W+l0b2WOMzqC/Xxz7YnvqiehxCHbAusaapsglLQ0sA/pr/FxmHM0RVg0vvX8Ik0BiFoRkSQ+H8Rb0QRdCkDiCs25huWg3vm98LMrIX9l3gds5tWta3MOXAFAytDLn2+TUyDgukL/9x/tTl1eEc7UxLTQuVGZVYelmK14SeTE8nT1E0KPjK4ytAyBf6vSXsM23NbeKaHbc+DtderiLhr764ni+cvyBmeYzoHbX7p1rMc1LZt6U7Prf38XR0Bl1BUOYoSirC0tMSS0/hWAVODBSbW0M+HiKu3xKpIJOGRMj7LdwtuP/Hfe7/cR+pvhQLdwvkZnIcQh2I3x4vSITuTcPY1hgLDwtOvnKSsFlhOIY7Mn77ePTkemQ9yGLb5m1c7H+RU+tP6XzfywMyKbU6zdvv9qbO7hT1XlPFx4puFVH2ehlGzxqR8laKFsizNXYrijIFTiOcsCu3497Oe1h5W7Gl9xYMzAzESWTHCEdmnJghrg2Pjj5idfVqne8BAjBR8aBC9GeMeiFKKx/QXDuDPhhE0u4kCpwfYqgyobbWjl3jdtFQ3MDYzWNRP3hIuUcU7XLhelCpwNDSkMgXInGJcdH9YOiYvOyUIiiUCu2nKNX0Wd0Hr0FeGFoYYu5mjomdCf7j/LWe9/DQQ9L3pzPw/YE6fl9Px6NjjwDBo2fPhD1YuFvQWNqITTcbgiYHMep7QaJtg+0GAPT6LaIoYBBuDnD7p+ucfOkkI74Zwbit4wTPdDsTLUm3zTGbcYxwFL2ILD0sdby8OoeZsxkzjs3g3NpzXHzvIqGzQpm4eyKFSYXc+OoGlh6W1DyuoSa3BucoZ4ImBWnd02q1mq19tuI5yJNB7w0Sj/fu1bcp2HGD8LnhyM3kFCYWUpVZRdHnRZzOPs2QBLCv6Y9UX8rcC3ORyqSin7eJvQnLM5ZT+aiSSx9cIva1WKT6Up797VmK7xRz/MXjRMyLIOG1BEpTOnzg5BZycs7lcGLFCQZ/NFgrd9Xs4Q8PP+SPcX9g7WdN1aMqnr/zPEZWRjiEOfD7yN9Rq9Rik/XyR5dJ25vGWuVapPpSzqw5g6mjKT1X9MQl2kWLKKJnoMe8Sx0Th/9VdN4/QCCdZRzOIHBSIDZ+Nlqg7A/hP9DW2MbyRx1SGMnbkzm29BiT9k7CzMUMGz8bjG2N6b1SmJbtyqLh6mdXOfPGGa1pvYoHghys12Av2praRIWD5+88z4C1A1jscwpZSz0rr0zA08+G8rRykn9JJmJ+hEDQyK/F0MJQJDY/XV+VppRSdLOI+Vfmd0moezpmHJ+BWq2mvqiexrIOmd+SlBJOvnSS4OnBuPV201ILyT6dza9DfwXg2ufXMLQwxMzZjNBZoXR/trtWbqyRuk54NQGPfh4ExAfgN8qPNY1raG9pF9dpjdQhwKrSVeIkS0tNC7mXcuk+oQNAKb5dTNn9MpQKpfi85upm7my5g2svV/Kv5gu/4U6J+BoDUwPMnMyQyqTYBthSm1fbZe2ZtCmJzx0/J3BSIA5hDjiGOQoytrbGhM0Jwz7YnoTVCcjN5YzfMV4k8cjN5eJ9um/yPjz6e2gBv3oyPZ21sduYbhiYCceq87RnYVIhGUcyCJ8TjpW3FV4DvfDo74GRlRH3d9/n/q779H+rvxb4amRjhEu0Cx4DPMQ+gJmTGa+Wvcr/V6M8vZzHZx9jhhmh9vHY+20HYNs2sH0y4H9s2TGSvkti7oW5GFoZitNnRTeLSFiVgK2/LVZeVuRdzhPX5S/dv8Qt1o01TWuoeVyDUqHUuj/zLuaRcyYHEPatd/XeZeC7ApGu4kEF2Weyhbz7ST7xU9RPOIY76hDpdj+7m+yE7H9c7zuHWiUocD1do5s6mvJW61u01rfysbngyZ3XaEPl0FW8/rU+TRVNNJQ2YOVthYqnyGytLfzc6zcCngmg7+q+ZJ3K4swbZ7Bws+CP+D9EcKsqswokaJEFvg/+vktlm9kHZyNvleN62ZWi3kXY+NmgaFBg42cj5AxPBgE6T+yDQHQFQcVkecZyshOyxfvzyhW4cVMPxqzj8OEn6jVqtdhra/IQagzrn62ZHT6bn6N/BgRQMHJRJF9c/IISpxIi1kYQZBaEgakBtXm17B6/G4lUwivFr3BowSG8hnjR68Vewvsr1WJ+LTOREb00WiTzRsyP+Ftrm84xeZ9uvw86SIvHlhzT8c+EDhnSsvuCb72FhwWt9a3oyfTo/mx3rbon6bskshKy6LGwB87RzozbOk6cXAa0+iqa+8E+2J4XUl74FwT8/0H8C/b9G/9xcX/3fQwtDBmwdgAggHC2AbYiaxKEBnBJSgkD1w/8LxeqwAmB1ObW0lTeRE1uDa69up4yqs6pJv1AOi21LeScyeG5m8/x4OAD7u+8z6yEWVh6WqJsU1KTU0NhUiGqtq49Ib4P+h4jGyMt2ZKHhx9y+6fbxCyL6dKHS9GgIGR6CM/+3iEVdvf3u6IXiVql5tL7whSdfbA9Hv08mHthLlWZVSjqFeRfzaetSfAd7DzhCLCyYCUPDz3kj/g/MHczxyHEQWhMO5sRODEQu0A78q/lc/PHm4z8ZiROEU6YOJjQ3iz4AVz/8rrIHOsccnM5fVb34eJ7F3Ht6arlTaZWqznzxhnMXMyIWRbTJdvp5fyXqc2rJe9yHk6RTsiMZEzcM5FjS4/hH++PR38PLdk6a19rrH2tKb5TTP6VfAInBWLhYcHj84+13re9pZ2bP9zEPsReS7r0v4pR348Sz82yh8u6vK4cwx0JnxtO8vZkHp9/jEd/D+5svSNKgz386yFXPrnytx50RtZGmLua68i8akLVphIl5Z4G+wA8B3jqgHGacAh1wCnKiZ7Le2odNxA85EBocNTm1mqBmBpj9KyTWWSeyNSRn7DpZoNDmANyM3mXwGrQpCAeHnrYpfmwiZ0JS9OXcu7tczoTmCAUste/uM6KrBWipFTnsPW3pbmqmeLbxVh5W2Fo2cHqyjiSwZk3zmAbYKtTcP1TDN0wlL6v9+Vrz68BmHtx7j/KlHYV2+O2k3shF31D/S7BvpyzOZxfd17LZ1QT7n3dGfzRYNoa26jKqeqSyV/5sJL0A+lis1MTGtnec2+f0wLCv/X/Ft8Rviy8sVBHZsMh1AGZsYyLH1zk2d+fRSKVaPmJauLUq6e49cMtph+bjvdgby2mr0qpIu9yHr1X9mb4l8P/28fpvxPNVc18H/Q9zZXNvFH/hs7xKr1byv1dQnH6asWrGNsYE7U4ivrCevZN2UfZ/TLK08pFUEwTngM9iT0aS/yFeJqNm0UvJEsvS8LnhguTKgehm3c3Tg8+jVvjHEJnehAyLYRDCw6RsiOFyXsms2P2Dq4XXGdCoO6Ubufo/mx3LQlKEKYMNU3ikuQS5BZyLWnD6uxqWutaxaT95o83aWtq07mm1Go1DSUNyM3kJG1KwvCQIYZyQ859eg7PPz1FgK25upkZx2dg292WWcssgVDiO/WPHCMcyTmTg56BHg8OPKA2v1YE+4ysjNBz0AMJGDUZkfVpFpnzM/Ed4SsSO1J3p9LvzX5isWzTzQaPAR5dAplqtZrNUZsJnBQoModBaNhYeVsRMD6Au7/dJXV3KqO+H8VFI19SQvcRW21Hu1ooAC99cEmnkNSaki1pIHl7sijZNfbnsQz7YphWMWpgaiA2EK18rAidGSoCsrV5teyZuIewOWHELI3h0bFHZJ7IpN+b/TB16ChCY5bGEBAfoLX+gHBtPjz0kKApQdj42SDVkzL/6nxOv3aaj8w+AqGXhWG2IQ/+eoD/OGEqtPh2MXUFdQz7Yhjf7HWi+6WfqLP1BoYwcfdEdj+zG1WbitJ7pThHOvNizosYWRuReTKTnaMENnvKjhRce7nqGLGb2JnQXNVMU0UTVz+7KjQ+VWsxtDQkcGIg1TnV3Pv9ntZrDsw6QNHNIuK3xRM2J4yDcw5y99e7yExkOmBfV+E73FfwiVudgF8PPw5VHmLMs2OYMbrDs8iphxOrKlfxIOMBEqkEA1MDLanpzsBPeVo5J18+Sfy2eOwC7UjdkyqAdsdn6KxdjeWNWLhbsDR9KUqFktOvn8baTzj/Hv09iN8eryPDDMK0Qdm9Mtpb2lG3q/kk5hO+GPhFl2BucUMxSUVJBAwOIOaZGBSNCvZO3EvY7DDG/zIety/dKCgoIKkiiSjnKJ3Xg3A/lKaUimuAY7gjYXPCRBJV6d1S9k3ex5gfxxC/PZ4xP43h1KunuPHVDfQN9en7Rt8u31elVPGe/nsETQ5i4u6J4oTZvZ33qFhRgfcUb4z1jKnvVy8CcYWJhQB0m96D5PJgyrx74fVk61ar1Gzrtw0QmqxL07XzAdsAWyoeVDA9aTpSlZSXnnsJ2+62pO5OFd/fzNmMY0uOsTBxITIjGRlHM9g1Zhfx2+JFdrMmlza0NOT4iuPUF9Uz49gMHMIcaIxp5N74eywaJOSc+ob6dJ/QnfT96QRNDtLyepQZCQ2YyoxKjrxwBD25HgqpHzdHr9X63lGLo0S/SwNTA772/hqXGBcm/jGRmpwafhn4CwPWDRAVF0Z81eEfq/GcAaHB0xnEnXF8Brd/vo2VjxXmrub8Nvw33Pu6M/PETFGV4tGxR1i4W1CTU8P1L67j2tMVx3DHjhwvBvbU7cFcbk6Ig25CIEFYX88OOotX1WQ675DOUc7snrybIuciZC3a127U4igurL+Ad7Y3TsVOJFYmMugDYWJOUa+gMqMSm242GNsYi4okNt1sRGBeE2ffOkv6/nSeu/kcNzfdJGVHivhYl0onCFJRL5u9TFpqGiPPfUZrzmisHEyQSCQ0lDRAahryUB8UjRbY5d6kJCkAv+nuWlMyADd/uMmVT68w9a+phM4MFT3lPjb8mNfPvI7Zp2Z8MvsTVletFo/1pfcv4Rnn+beTcyDcbyk7Uui1shfFt4uRm8u15PMTv02koaSBQe8Pot+afkQuisTG34b5V+Yj1ZPyjc83+I32I2R6iDjlZ+NvI/gFSjvWNRt/G9zGuuE1xAv77l1LcKmVap28TRM553JI/CaRvmv6agFVAH1W9+HhXw/Z1m8bb7W8hUu0i1hDaoDAJalLSHgtgZaaFuZffmI/oBbqXbuyDiLao0dwqLQneoPDsQsyYfczu3l0VAA5VyWu4rfs3zg7+BJyqxNIpBJRXvdpVQYTexPG/jwWh1AHLr5/kbS9aUQviybxm0Tc+wiqKrvG7kLZqmT22dkYmBigVqlRKpRCLhj8PUGTg+j3Zj/OvHEGhzAH3Pu4M+r7UbTWtlKUVIS+kT7Hlh3DPsSefm/102qSRr0QhfdQbz62+Bi5uRxFgwLHcN2pja6ipbaFEytO4D3Mm9AZoTRXNZN9JhuHEIcu/WVL7pRw9s2z2IfYizWXsk3Jngl7qM6u1lmz7QLtiHw+ksqMSvZO2itOXmjUYTRxbu05lG1Khnw0hPwrAui0+5ndDPtiGLW5taTtS6PgWgFLUpdg6WmJ32g/Hh19JPhdhTtiXFeKQVM1qang5qLk6oarVGVWET4nnLLUMn7q8ZMogVybX4uiXoFtd1uxTo9YEEHw1OB/9EPvHJqG/dNWDhfeucDj848JmxPGjOMzRF/C+K3xGNsaC3ngMwEUJhbS1tRGxpEMXGOFc2lgasCzvz9Lyo4UDs4+SEV6BbNOzRL3S5VShb6hPhffv0jN4xodFR2JRCL2QjzjPFnTIHjPZp/O5v4f97mz5Q5+o/xY9mAZ93ff58qGKwTEB5DwagJ91/QlYl4Ett1t8ejnwUa/jThFOjHxj4mi8sNzic8BgkLRvqn7eDH7RbFnoNkH0/amkbY3jYBnAug2phs2fjbY+Ntw4+sbZJ/Oxm+kX5fqMo7hjoz4eoSoCqAJtUpQ2JDKpKIE7N9NXhbfKubiuxfxHOCJlbcVegZ6zL0wlysbrrB/qqAcET43HOcoZx4eeoharSYgPuBv+wsNpQ3sn7afoMlBIpH8/wvRUt3C4/OPUcqNKPPq2grCM84TiZ6EY0uP0VzdzMqCldQX11NXUEfcu3HkX83H1NGUlx6/JL4mdFaoMMVrJOty2n38jvGM2zKOtqY2qnOqufzRZWz8hTWg4EYBx5cdx9Kjgzwk1ZPq9OZ2jt7Jo2OPdOrG3Eu55F3OwznSmaOLj9L3jb7iBN+mkE2o1WqWpi0ldW8q2aezGfJRhw2Ovlyfibsn8vEmC2pMn4DIZnD5+8ucef0MzyU9h0X3J/K11YXYFCSjFxZEXUEdLdUtAhHA0pCJeybiFOlE3zf6ioD/rnG7kEgkWoMNfVb3wdrXmutfXSfhtQTmXZyHay9X6lrrkFhK+GH9D8wbNI8fI34UXzPlYAdhVqonxS7QDrVajV13O9L/TMdvtJ/YD+pM2K+uBtRqQI1KKdilfO31NbGvxgo2IN+A/WJ7ll5ZyrwB8xjyyRByL+XiNdBLING1PAYEgpDcTE7w1GCqc6o5vfo0apWazxw/AzWYOgvrmc9QH63819DCUOt+G/vTWK3z1lzVzOHnDuM70ldLEhsEX8zc87na72dpiEd/D3Iv5mrVQuffOU9TRRNx6+Po83ofvAZ6UXSziLNvniV0ZihOEU46AwNx78bR8yVhj7PystLqM4AwxWxgYiBMbT4hMijblPwb//+If8G+f+M/KkqSS9g/dT8e/T3Epk51TjUVDyq0nvfgwAPu7bzHgLcH8OCvB5TdL2PA2gEiu/3YsmM0ljYyae8kLD0tRcZvV3F/931U7Sru77pP/NZ4rLytcO3tKvpwGFoa8vDQQxI3JuIQ6sDMUzO1GgEgAEy3f76NfbA9ztHOItNKE3Hr4ohZFoPcTE7+tXyMbY21ABlN4qiJtH1pHJgpTFMN/3I4ll6W9Hyxp45es7WvNTbdbGipaWH8L+PFZD3jSAa7xu4SAUSP/h4sur1IbD7sm7xPYIeemIH/OH8OLzpM8tZkhm0YhlusG6tKVomfMSthFn8X1dnV5F/J58HBB7Q1t+E92BuJVCIU6E+manou78mPPX7ErrudFphpYGrA3d/ucvmjyyI4ZtfdTvTCADj5yklQw/AvhouMpswTmZxdcxZFo4La3Fod2QpFo4KTL5+k+7PdqUivwKWnNpsz+ZdkEjcmYuFmwYODD1iWsUzQ8Z8ajNxczs+9fkZRr+hyUtPW35bRm0bTa2UvsRnsHOVM7KuxXPv8GoWJhUw9NFVn8k4TM0/MpOJhBWWpZTrn8vbPtzn/znnit8drsUc1cfe3u+jJ9bQYupqw9LDU8sJ6OnY/s5vmymZsutmQfy1fC+zr9XIvrLyFRpWGwdQ5XGJcupQ5aqltoTSlFI/+Hhx+7jA2/jY6YJ9UX2BATto7Sef1IPixGVoYYmSjO02mVqtpKG4gbX8aJ1ac0JKxrcqqYtfYXegZ6HXppXXl0yuYOZvRbUw3do7eif94f3H6xsLNgtTdqRiYGqBoUJBxOEMH7Lv10y2OPH9EMJE/NUtnkkXVrsI52lnHa00TOwbvAOD06tOM+m6U1iSvhl28Y8gO8m/kM2DGAPGxhpIGdo3dReisUNapdKUtW+tbMTAzwH+cv8jGVavVIqikL9dn39R95F/J5+V8wdMqdGYoRbeKuPHVDeLWxXXpcQng0c+DWz/cYueonVpeXyDID/0S9wvhc8N1dOf/30Z5WjnNlc0YmBl0CRrFvROHkY0R2QnZ4uMB8QGk70/n7q93cerhRNSSKNFbtKmiiYTXEvAb5YdxjDHNSc1E3oyk6GwRodNCce/jjkOIg+it4VTsxIFnDzDmpicSCUj0JYzeNJp2o3Zubb+FU7ETBhsMyDfNxynSiVOrTuEW6yZOUPw69FfM3cy7nARtrW+lIr0CC3cLNkdvxm+0H1MPdkxlnHjpBFmnsnirRfCcufrpVQwtDen9cm9UahVSifB7lQolXzh/QejMUO7+dhcXXKgIraC1RY26E+fEyMoI3xG+dL6N8/MhIECQnXGOdGadeh0NigZilmkXxnWFdbTmCZOC8lY5JntNyPHKEcA+qYTgacE8PveYbf224RDqwNifxnap/Z9/NR9Vuwr3fgKo/XSTbOLuiaIkb1VmFRlHMjCyMeInkwRyQ76n2vodYtuEpuQLyS+gb6TPg4MPuL35NoM+GCRI9JoYUHSriORtyYz+YTSGloa4RAseg//k96Dxy9RE6p5UYaLHw5Ko56MouF5A4sZEohZHiev7w0MPkcqkWlJxmii+U8y5t8+RezEXCw8Lxm0eh1tvN8LnhZN3OQ9mwZjIMdSoath9Z7cgWa0v4cyaMziEODDn3Bw+PKNGv60Z6ZMJlW6ju/G24m0Sv03k8oeXmbBrgtgwqM6q1vp8z4GeOt9J0ajgt+G/UZtby8yTM3U87KR6UuyD7enxXA+yz2STczYHK28rim4WicoCd3+9i8cADy1p5qTvk7j43kVCFoZw9ZOrqC3V+Ib7inKvJcklXP30KjbvCXmNv42/zue21rVSdq2MJtcmrT1FrVZT9agKuYUcUwdT3Hq7Me/SPBG0sw+2x9TJlAOzDuA32g9zV3NR5u8z+88AofkSOjOUGcc6AEYbPxutPCv7dDa3N9+m31v9sPGzoex+GZYelhQmFfJzzM8Y2RhhYmfCrNOztOXXEtuIuRHD78rfeaX3KwTbBPPMb8+I+0J1czX2pfbsm7gPi7ct8BvpR/GdYvZM2EO/Nf3Ewv/HiB9F8DtwQiCBEzpAJIcwBybunohzlDMyIxkyIxnJ25IBwW+pK4La/T/u01jWSODEQFx6udBU2cTx5YIqw+CPBZB2wIUBOO124pHHI3qt7EVjSSN1BXXkXszlzB1r2n2E61yz/Wrk9UBoqNYX15N3OQ+XaBcsPS3FZsU7698BoKJXBbb+tkQvicY/3h9TB1MOLzpMwfUCcU+3627HkE+H4BTZAb7UF9Xzrf+39F7VG6m+FD0DPQxMDXgh+QUk6yWQBKP9RjPSbyRSPSmjN41m8EeDhaml3BoOLzxMyMwQwueEM+rbUewYsoNbP94CwCDSGJyE+mFN0xpQCz44Jg4m6Mv10ZPpYetvK0gzA6ZOpoz6bpSOV91/J+wC7UTCBMCwz4aJwHVTRRNSmZTI5yLxGuiFiYMJ3cZ205oQULWraK1tRb9NH78sP5K+T9Lx9+l12Zkq8/5cHHARi1vd6NwCNnU2JT0wHZMGE7wPe5PbPVfMZ+LeieNOwh2sM6y5tfwWq6evxj7InpEbR3Lj6xtd/h4NybJziNOMaohZHoOZixnZCdkU3Sri5MqTmNibYOlliapNRebxTEZuHEltfi0uiS5UNlTSPbmI5vB85mwTphrUajXqiRMxTsqh3cAYx+xrVKVaAu46n61vqC+oDTxFvjPQE2qsFucWXE07wJ7Trwt+cUZWRrTWtaIn16MkuYTM45mEzwsXa6DYV2Pp9XIvZEYyfhv2m5C/dgL70valUZ5WzqD3B2HqaIq+oT71RfVYuFkg1Zey+N5iHWLSsgfLAJjcv4Ru13agUvbEe6gvjY7CFODlTy6TuDGRMT+M4cgLR4Tv8GIvHZ+w9pZ2shKyBCnI/DoeHn5IxMIOgLmlpoVLH17CvZ87YXPCqHhYwbUvr2FoaSiqqPiN9sPE3gRTR1MqH1ZSlVlF5olMlAolviN9eaXoFa3PLC0FlUyOSiZHTybk5qp2FTHLY8g9VYFbnhv57vko9CtRq+20/IBBkB5N+SUFG38b0SM943AGpXdLqSuo4+WClwWpMiMZz+x4hnNrz4l7tfdgb5Y9XEZzdTOXPryESikkNVc3XAUEQDF6cTT3dt3jzBtn8I/3J+m7JEAgsw16b5D4OzSSs0cWHUHRoOC5m8+J++fNH29ye/NtJu6eiLWPNSqliksfXsI+yJ7uz3anrbGNlB0pyC3lhM4Ipe3WTroAAQAASURBVCqzin2T9zHk0yHYBthybt05Mg5nMP/yfGTGMvxG+7Ho9iKsvK1oqmji0MJD4vrpH++vo4DhGeeJZ5wnlRmV6Bvqi1MhdYV1tDW1Ye1jjUQq4eFfD2lrbmPIR0PENao2r1bwqn/zLC49XXjm12cwcxaUJCbtmURLbYsA/PbfxuOQ8ShMrOgnge+6f0d1VjXzLs3jxx4/YuZsRq+Xe4k18cmXT5K+P10LtJWbyWmubObWj7fwGuz1j9YSIJBJT792Go8BHlpAkEbeTlNXOoY7iufCMdyRFVkr/vF9fUf4YtvdltOvncbczVxryvB9g/e1APJ+a/qxZ8Ieeq/qTeRzkeReysXYxlhHWaUstYw7W+7QfUJ3QmYIOXzKLynkXswlZlkMC64vwMzJDAt3C5Gwa+Vt9bc+XnIzOSZ2T5RWkLA5ZrNgQTLYi2mHp1F0s0hLdrM8tZy8S3ksf7Rch1hUfKeYHYN30FLdQv+3+2PjZ8OpV08h1ZMy5OMh1BXW8ZX7V/R6uZfWnqOJ8+vPo2pTMej9QQRNCcJjgIdY733u9DlW3lZEL4sm8oVI+q7uK56Ls2+eRdmmFOvr1D2pnF93nnFbxmHlY8Xj848xdTSlNKVUi9T9fzOUbUpqc2t1SNRusW5YuFtQm1eLWtKJoNqpLgqcGEjgxEBSdqTQUivIkVc+rOTcW+dw6uFE8e1iTBxMtPb/0d+Nprm6WZA6tzcRz6HmmGlqCZmxDGNbYy3FF9/hvsw+O1vLz3zhDW1VBhAIJ0+TIUHIUy++e5GJuycit5BrqcX4DO9QViq4XsDtn27T/83+6Mn1yLuch9xMjr6RPo16ZqBWY1hfTm2+HPc+7gxYNwAzZzPx2MgbK7HPvUlNdx9WFqwEhD7U0cVHmXFiBlnSLDxXeeJjLXxmV76DmmnSBwcf4DfST5T2B1BL1SgMFTqqYNkJ2eRdziN2VSwNpQ1YeFgQNDnov7RHaWkBvfZWIk5+wnv6sDR9KYETA7EPsSd9fzoAoXdDuRV5i9KMUoImB4ly7QdmHyD+YDwFSwtIfi0Zp+ed8IzzxMLdgjE/juHc2+eQ6EmYlTDrv+WL11UoFUrSD6Rj5qq9ViT/kkz6/nSd3kvFgwoCng1gwq4JZJ7MpPh2MRHzI8g6mUVtXi2jvh0lyon/Nvw3gL9VIehKSaxz7Bq7C8dwR+acnaOjsPVv/OfHv2Dfv/EfEVKplJCQELbGbgWgx6IebOu3jfD54fRY0ANDC0PaW9rF5w/5ZAj93uqHVF9K+v50UnenahWp1VnV4jTT01HxsIKfevxEn9V9GLB2gCjL2FDSQENpA+Fzw0XWYr83+tHvjX5knsgkcWMipXdLu5QsVDQoOL78OJHPR2pp/mvCzNkMM2cz2lva2Rq7ldCZoToeZ3mX86jKrBIMzp8UrA5hDiITbNAHg8TGaUlyCQ8OPhAasOcfk74/nWd2dLzfw0NCwaFWqUnbn4aZs5mWbN+EXRPYN2Ufp1aeYuZJAbwcumFolxNnNn425F3JI+NwBuHzwkUmb0ttC/WF9fR5vQ91eXX8Nuw33mx+E31Dfdz7uOMW64bfGKHRaGhhiMz0qcT2djG2AbYM/2q4yCC8s+0O+ob6BIwPELwIjmUCAth3cM5B7v52l/lXhUawhnX6tKyk3FzOvMvzaCxrZM+ze4h7N04L7Gtvbqe5shnvod50nyDIhdTk1LDRbyN9VvfBpacL+VfyKbheoDNWD0LzoXMS59HfA4/+Huyfth+ZkQz/sf5sCt2EQ4iDFripif3T9tNa26pT6MhMZJi7mOMU4aRj7A4C27Mmp4b0yen0eb2PzuZecL2AG9/coKmiiQHrBmiZOofNCaOtqY0eC3voeIQN/2I4vwz8BbVKzdwLc3U+9+/i+pfXubD+ArYBtrTUtnSp569SqmgsEyRku7q2NNrkeZfzMLQy1Eqy1Eo1X7h8gX2IPYM/GqxVcJrYmeDUw4mgqUFa8oaauPThJVxiXPAb7Ud1drUWU72lpoWzb54leGow0cuiu5xW1DBVq7OqaSxv1AH7RIb038Tci3PZ3n87t368RfDUYB1/RRAYyP7j/ZFKOwAuVbuKxvJGwSeli7j7611OvnSS2Wdni8wtiUTCm81vompXCbr0TW2ip4kmopdE4zfKD6lMSl1hHXIzXemekOkhmDiYUHSzSOcxfbk+IzeOxMzFjMbyxi7lhv5Xw9rPmvG/jMc5ylmL3aYJiVRCrxd76fg5xm+NZ/z28TpNwJbaFpK3JWNkbURVYxMxN2IYdXwUj1WP4cmAWPL2ZBrLGkkankRCjw6J5/qieqqzq2mpaSH1x1TOjTiHwkCBwWkDqmZXYRdkR9K3SbQ3t4tgn6JBQVtjG0W3isg+nU3I9BAxIS9NKWVbv20M/2o4A98bqFOohs4M1TK89x3li8xYxvfjvif5bjIeSg+GvTGMiPkRRC+LxjnKGacoJ9754x0u9bvEo4rBxJY50Hw2BwMzA1yiXci9lEtdYQMWJfpYlmVQUzAQAky58M4FbLrZcMDpAGvPr+X4jOOM8O0grRx94Si5p3JhDdSZ11G8qZg+E/uIj0c+F8m5t87RWNZI5cNK0QPx6Tjy/BGUbUqWPVhG39d1p5E0eysIzeisU1lkn85mpNqGLM+hZIbVoHriKWEbYEtLTQu7n9kNgJ5cmEhceGMhVZlV3P31Ltmns2kobmBV2aougb7Se6Vs6b2F/m/3p8eCHlRlVWHTzQYjKyPqi+sBobE7/MvhxK6KJWpxlNb1ffLlk8jN5V2Cff5j/Xk++XlOvHiClF9SGPvTWMEfY3YYSoWSo4uPIm+VczH6IqteXoVEKiF1byptTW1EvvBEalwi4d4gbf3in6J+ovhWMfqG+rQ1tdFc1YyhpSHRS6KJWhz15GUSytPKeXTsEV6DvURZwiufXhH9bK28rcT9sehWEb8N/4249XEsvrcYgHPrznH5w8ssSV3CxN2CPKtarWZN4xokehItqcObm27SUNKARC4hwysD0wZT/I38aa1r5TOHzwgYH8CKrBV8Gf4ls1xmoZKpuHbtmjil2lrXykYvAWD39PTE0tVSfG9lq5Jv/b8V8yIjayOtppJrT1fMXcwpullEyi8p6Mn1RLDPf5w/Dw895ODsgxx+7rAInHcV1dnVpO5JJWJhBIM/Hkzsa4L8uMxahnSclJaiFkyaTTrkAp9E+8l2Rp0YhWuBK9dSrxG2P4zQGaHc/+M+xbeLid8eT7NRM4Z3DanOFgBZqZ4UI2sjcU2TSCQMfG8gtt11J0QATB1MxcZE5aNK2praWJq2FKm+9G9luG/9dIuSuyW8XPoyBnoG4mcD+I30Y+MvG8n0zURPqceS2CX0WNiD0nul/BD6AzIzOWWePZGolDhkX8NAYgsEcGeL4HX2WtVrGFkZkXEkg32T9xG/PZ5wT2Eiur6ongk+E9iftR9fawFUM7Y1xtjWmCufXiHjcAZDPhkiTlnGLI0RyTaaMLQwpO+avrj3dde6t1pqW+j2sBvP/vksKQkpjLw8UmAgq4VGx7l158i7mEfp/TJsojwJf/K68b+Mp62pjZSaFF7b/AfeaTa0GXsiM+ogXsWtixP/PeN4ByhsbGOs08g6/fpp9OR6DFw/kJ97/YyFuwWT9kyi6FYR+VfzCZocJOQSa84SPi9cnB7u3Oz+udfPGJgaMPDdgfwR/wftq9tZuW6l2OTdYL8BMyczSu+W8hZv0WjcSMLBBJ3v0j3Nm0pLBRcHXESNNhtb9YTtYdpgSvCRYHJ7doB9SoUS7w+8WXNiDYEegWJ+FbMsRovsUXC9gG39tzH006E65A2Age8OFPM75yhnnKOciZgfwUa/jRTeKKToVhE+w3yw8rbi3s57DPl0CAmrEgg5HcLNuTe5MMqTRX06SGoSiQTy8rDPvUWpdyx3B73ImNFG1Dyu4fTq0wRNDRI9i8LnhotNwMLEQopuFhE0JUgE+0omlDBjUse57L1SUEP5KfInAIZ+NhQ9mR4X1l/Aa1CHB7i+XF/0rRu7eawOQXPyvslirtxa18qJl4Q1fsG1BVj7Wou1WFNFE9v6bSN0Vij91vRDoQA9pQLjumKaWxrF2jZ5azJnXj+DvqE+MhMZFm4Wf+tp21LTwh/j/iB6WTSjNo7S8ZFtrW/l6oar1ObVErc+jqKbRZxaeQq5uZz8q/mEzwnHva+7KA/Y47kenF59mjNrzlByp0S8tzuHSgV6imb0FU20NZuL10hTRRO7xuwiLDICyxpLXB5eoqV6FJ/afCoqJEikEkZ9N4ojzx8hYkGEOB008N2B3PjmBvlX8rUkBjs3YTuHkZWRCJiq1Wq8h3prrcU2fjZEL40WG7uAzrSMJuacm0PxHaHG1EzNaWo/zXuqVWrOrz1P8LRguj/bHVMnU95oeEPMKa19rZm4ZyIOoQ7i61trOzwJjW2MRTWYmtwaHh19hFRfSvC0YK16/Omw6WZD75Ud6g1n3jjD3V/vsqZpDTIjGfOvzBelakd/L5Ac0vamYRdox/yr8zGxNxHVCy59dInza8/zXNJzKBoUVD6sRBIorBFSqQD6yoxkuPd1R6onxdzNXAsoCpwUKHoxa0LRoCBtfxoJqxKI3x7/X4J9f8T/QcmdEu7/cZ+TL59kdc1qrdzh8fnHZJ3K4tnfnv1b0OzpqMqsEgl58dvjCZ8TLoDAaoFE6jvCV5TXnXV6FhKpBD25nkis3DF4B34j/Zj611SaKpoouF6AfbA90YujCZ8bjoGJAbX5tdz84SYxy2MY/NFgZEYyLV/ctH1pSPWlzDw5ExCmY/Iu5xExL4IrG65w7bNr9F/bn+eSOsjaRUlFgCCviRotImnx7WKkesL7aUhGP0X+xIB3BtBjQQ9kRjIcwx15fO6xuJdnHM5AT6bHkI+HIDOSETY7TAuUOr/+PBKphAFvDyB9fzrKViWD3h+EkZWR1j3uM8yHlB0p5F/NF5R0noBWACO/Hak9WSzpuK9K75ayf+p+xvw4htcqX/tvnTvNmte5tv1/G+fePseVT66wLGOZjnLR7LOz+Wz4aayL7lHpJpAduuAta62jDqEOTPhjgtD/C7EneEowOecE1RNN76Q0pVT0hQRBmUlzDCoeVtBa16ozbQ1C/+Bpecqu4oW7L3Rp5RCzLIZuY7phH2yvs052vncHvjuQ/m/2R24hp+pRFb+P+B3nKGeKbhZhGTyKCrcIgi9u4swbITz727NiTl1TI7y+2qk7yXY+OLl37EMe/T0Yt2UceEKPnwTShnqdWvxemriYe5GErAQinCJ4tvuzBIwP0FGj0mvXw7HQEUNLQ0Kmh6BvpI9UJmX/9P1inVSYWEjm8UwtIljnKLhegERPgku0C2o1qKQdNZ6yTSkSyX2H+3Lc6jipBqlM2T2Fg0cP8lLmS+Jzy9PLkaqlmNWbUfZHGQ8cH7Bv6j56vdyLvqv7alk/aaLyUSXZp7PxHe6LlbcVbU1tHJh9AM+BnkQvjubArAO4xroSs1Q4LqaOpl2StC99cAkzJzMddbScszkkvJqAa09Xrn12DbVaTcT8COZdmidOu1//+rpIPNUAyy01LZx96yyeAz21CIOaSNufxqlXTjFuyzhR3WzohqFapIN/439v/J9Y8/4n8S/Y92/8x0TKzhTaGtoYvWk0zlHOnMo4JTBj9aXsnybIDmjYZ2Yu5pg/SYaHfT6MAWsHaDV9Oxfy59ef58I7F7APsccu0A77YHs84zzFMfFJeydh5myGuZs5VY+qaKlp0ZHr8h3hy1rVWlALzLAHBx/gM9xHLNrlFnLmnJvztxNdrfWtKFuVGFoaMuzzYToNn+TtyZxefVpgaU8KZPz28Tw6/og+r/Yh/UA6+VfytYqMkuQSLqy/gLGtMe0t7cSsiOHq51eJXBSJ3ExO3Po4eq7oiW13W943eB+/0X488+sz6Bvqoy/XJ2hyEI1ljdzbeY+PzD9i/uX5YkFzf7eQsPvH+9NvTT/05fqk7k4lcWMi3cZ06wD7alq49MEler/Sm4gFEVpybg6hDkLB8iTmnOuY1tPE1r5b8RroxfSj08W/nXzpJK11rWLCsiR1iVhUO8c4097ajkOoA8/ufBapvpS8S3k68o96MiFZUzQqmHN+jlZSC2j5GWqirlDwubryyRXWqdfxU9RP7J+2nxdztJuwDw8/5Nxb5xiwbgA+w3y0mL0TdnXIiJjYmeiwmUCQDjOxM9EZ8QcImRZC8NRgmsqburwGJ+ycQMqvKdz8/iZhc8Kgkw1iY3kjW3pvEf/76cSls+l951Cr1SR+m0jpvVId0FQTWaeyODjnIHJzOc/8+kyHZ4BE+F9LbQvdxnTrEtCqK6gT5TL7r+2vVfB3jl8G/SIWZmJIBBa5U6STjkyn3FzOoluLaGtqo7WuVadpsuDaAvTlAiP8lWJtBvPuZwXgYMgnQ/52ys1/nD9vtryJqk0lTiE9He2t7ajaVV0m6x79PJh1ehZ3d9zFxEG7UZu8PZmbP9wkfns8XqO1JyHNXc156fFL1DyuIedsDs7RzlogqUOYA71W9tJhd9366RbFt4rJOZOD3FzO83c6mFs3vrnB3d/uikWo5rf3eU27+QoCw7or6Vs9Az1ilsXwR/wf7Ju8j7fb3u7ymPyvhKmDqU4zq3NU51RTX1SPc2QHGLh/+n6aypvEqeM7W+8gN5cTODEQKy8rXq97nUdHH3Ft2n5GMYpTQ0/x9grhO6f/mc7p1QKZwrbQFlmIjNk7ZqOyuE76AanoVwrg6+vLLqtdVP1ZRcjYECRSCSuLVmqBkguuLQCE43zm9TMC0/TJ+bH0smTIp0Nw7+vepU/o04Vc/7f601zZzIEhBzBtNCXfOB89uR76hvpaxt8na04C0FJXSEIC3NmyF4cwB4ZuGMr2/tsBMPHrj13ebRoregGmzDg+g+RfksmLy8Nnpg/v/PoOYfFhotRZ4KRAzILMGOE7ghOZJ2hxbBGLg9a6Vn6K/AnPOE+GfDJEXFPv/3GfoltFDP5gsAi0xa2P+1tJsq5i0p5JqFVq3g/6GrtyO4ySVMjNs1CrvWksbQQJdJ/QHa/BXlh5W2FkbYSxnTFOPZzwH+dP8a1iChMLaShp4PG5x1z+6DIjvx0pFuwGJgZ49PPAws2CzBOZHJh1gKl/TcV/nD+DPxyMU4QTxXeKMTAz6NK/ZvSm0Zxbe473DN5jecZyrf3EyNoII2sjwbNGqUZRr+Bji48Jnx9OeWo5Ch8F+ycKuUvYLOEabypvQmYkQ6on5YfwH7BT9qDcs6OI/tjyY3GvX3hjIXWFdWyO2szgjwbT9/W+SCQS2lvaUSlVJH2fRNJ3SbxS8orWpLmVtxUecR5aMlwyIxkOIQ4Y2xqTtj8NqZ6Unst7EjojVEtiWSKRIDOW0VDaQFtjm0h00ACEjYpGhikFH5kv3/gSWoQJQ9dYV8EbJs4LmaWM1kOtXEi/IIJ9eVfyxM9winTi2pfXsPGzoduYbkikEvqu6SvKEqvValTtKmGi6EnzbtzWcWQczsAhzEGr+Tj1r6kcX3GcxI2JKFuVnFp1CtdergRODOTmDze5/NFlph6aimOYIz0W9iBiQQQSqYSKBxU0ljWiaFRg5GrE2h5roQc81+M5lrpqS1fqzdJji+MWBp8ZTOOjDk+i8++cp6GkAa9GL270vMH6d9bzztJ3ACEHerq47/9Wh2TXvZ33yDqVxfAvhosSTJo4vvy4lifpjiE7kBnJdHxRem7oyaCfBrFt0zYeLHuApZclb7YIsq76cn32TNkjPtcu0I6LH1wUJcba6luJPP4Bj0PH4vrgDAp1MBBAxMIIqjKrxDzPqYeTMHH4xP/jE6tPALB9yRbvSm8qF1aSuzEXx3BHmiqbuPzxZUAgcZx/5zyecZ5iM6RzGFoair57mlAqlJx48QTTdwk5ocpWAFtK75ayOWoz0UujeXD1AUU5RfyyMIVe91+nV249p185LkxqTAth9Aej8azzJOiKI9nhz/Cu3kHUKjWBEwPp/UrvLglcXUX6/nRkJjIGrh+IoaWhuC7knMnh9OrTuPZypb2lnfQ/00n/Mx09Az2MbIy0JqY0Da6W2hYajRs5UHCAwu2FbJq5CQNTA+yD7LHwsKAstQy1Us3V2KsEhXcQxK5+fhVlq5K7A+eREObA7F9mYyQ5TVvcWJoqmrjw3gWaa5pZvWc1R8YcYfqN6Th5dDTNjiw+QvLWZGIGxpBrk0t7SztNFU0cW3YMl54u4uSRoZUhfiP9sPS0JPmXZKqzqrskb3WOGxuFycCJeybSXNmM3FyOuZs5Qz8dilQmZeD7A9lfsZ98t3ykRsnIrOZwZ+sd2lvbCZ4SjMrLhweGPWg1tgSJFD1jaK6qInVPKg7hXfvgZBzN4OK7F5GGS1mWIIBCGt9aTQz7bBhmLmZUZ1XT1twmqK1EOQtAX6daoaG0gdrcWmy72+r4+QFajTENmaPHoh5s7bMVmx7uzD48EX0DPVRKldAYl8Dmnj9TlFhIy5BXSB6+Gk9PSN5+h+yz2bj1csMp0on4bfE4hDiIeQMIU/dW3lailKrcQs74X8Z3KR0JglziiqwVJLyWwHcB34l/H/TBII4vP45rL21bhfB54fiO8KWpoom6gjr0DfW5t/Melp6W4rSUUgm2+XdwS0+g9O4CXHu6Ct6Mzx8hLbiMGz2vMfjMYNwzUpHKxhA0JQjHCEdOvHgCgBFfj2BWwiyaq5r5xucbYl+LJer5KJbcX4KyTUllRiUmDiYcmHmAjCMZ9FrZi+GfC43ruoI6Mo5k4NHfQ5zEkkgk4tR4/tV8Di04JJLYytPKAWGaO3lbMlc+vSLms38t+IuixCIW31uMZ5wnNY9raCxrxMrLSkeJQKov5YWUF8RaSyKRaOXzRtZGWkoqQz8dqqWKomxT0tbUhsxIhqWHpVZefPKVkzRXNovk34bSBm58c4Oa7BpGbhypdX35j/PHzMVMBMg615WPjj/i8keXybuUR5/VfRjy8RCta6G9RahDLr53kXFbxhG7KpapfQuQl2Wip+fL0Cc+gmq1WguU0kTwlGCdv2WeyCRhVQJRS6LEa/KfInxuONUDqtE31KfsXhkAjWWNfOPzDeZu5lj7WJNxJIPgKcEi2Fd0s4jN0ZsB6LWyFw6hDpg5m2HlbYW1j7WWxFzVoypAmOJ3CBUUEaYfnc7Vz69SV1CHW283ZMYyFt9dLL5mxFcjxPqu9G4pu8buYtT3o4heHC3mqqUppRxdfJRnfnsGv5F+gkxmazt6Mj3ubL3DkecFX1JN3ynnjGDP4DfSj2ufXQMQvYo1sap0FTe+ucHJl0+ialfRc0VPcTK6JreGlB0p+Az3wcTBhNbaVsxczMTzbRtgy5yzc0g/kM75tefJu5zH4ruLxd6Wsa0x438Zr/V5qX+kItETwL75V+aLuXdbUxut9a0YWhqiL9dn/C/jRfnl9pZ2Hp9/jG2ALaaOpjrSnZ09PhtKG5i0d9I/SiJ3FQqFAkNDw//6if/N0KjXaDzINVFXWEdFegUGjzOwN60Rwb7Ok303vrlBztkcJuyaIPbsjKyN8BvlJ+Ssg70wdTRlc8xmzJzMmH91PgdnH8TSy5JeLwsyzxZuFlh6W4rvmbAqQVBlaX2L1rpWzrx5Bve+7gRPCRYnWTurFd3ZdgcDUwPxuF7/WvByjV4ajVRfyvAvh4vPN7Ez+W8Rag1MDODJ00wdTRn781hMHU3ZP20/7QZGqKVSCrsNYPKz2vup5tio9WS068mQ0EbKrynYBtjiEu3CvV33OBRwCLf5bsSdj+OG1Q0dGeSr+Vd5/9L7vHbsNcwHmmutSxoCkkmjCZO/mcw1yTWR+L7BfgN23e0EBSwHU8xdzTn6wlGyTmSJE9pXP79Kc1Uzgz8YzIFZB5AZywRvOYnwnbPDxzN9UIlW/8bE3oTSwaUUZxVzK/IW8f3iSfo+iezT2Tyz4xnMXcwpzCkkzz2PRQWLMGg0oPhmMdVZ1Xxo8iFDNwwlekk0j44/wsROmPIsSiri2JJjTP5zsuB1qFSRvj8duYUclVLFvZ33UKvUXea3nWPCrgld2gJ1n9CdhFcT2NJ7CzNPzhQBYk0ftamyidOvCb0KfSN9er4onIPW+laSvktCKpOKYN/+6ftJ3ZPK221voy/Xx9jWWItwoZm2B3h84TG5F3IxsTch8vnILm2W/o3/efzvXvP+J/Ev2Pdv/EeESqXi8JzDoIaji4/ydvvbrCoVpCQbShoY/uVwsdnwxReQnd7Ke2vbsHQxEaag/sbPHRBZQGX3yii7V0ZrbasWGKjx49Bsvj1f7Enp3VJGbhxJdXY1lRmV9H65t7BYS+DOljvc+PoGL+e/jMxVSBz0ZHoi2HHtCyEJ7MzcO7HiBMnbk3m97nWtv2vi+PLjyExkTDsyDX25Pu59hck4lVLFja9u4D3Umz0T99BzRU/i3okjYHwAztHOtNa2cuLFE0hlUpStSrwGeeEU4YSZk5monz9h1wTK7pfxieUnjPlpDJHPCUBQzLIYjGyMSNmeQktNC8W3i7EPsacuv46G4gZu/XCLwImB7BqzC2tfa+ZfnS9KioAwrbg0fSmGVoaYOphqSTg2lDZQnV2NfZD937JXB747UAdsGbdlHHsn7aWltoWIhREYWXfI9/Rc3lMErYKnBKNoUGBsZ0xJconWe6jVagGkMdTXSmKPvHCEWz/eYtnDZTrTYOYu5gSMDxB8vIA+r/XRmiTVhFKhpPReqSgXMmzDME68fIL6wnqe/f1Zzqw5w60fbrEsY5mOfwHA+XXncYp06lIOAYRpts8cPiN4arAWeAjg2ssVlxgXoen/1N5cV1An/nvSvkldsn1AmFxqqW0Rm6VqlZoTK4RivbW2lW39t2l5TYLAsGwoaaChpIG6wjpcntxsA94eIIDs/5AoyM3kRC+LJunbJFK2p+iAfde/vk76vnT6vdlPBwyR6kn/UX4XYEvsFhQNClZkak9J/hMT1W+0H06RThxdchTnKGf6vdlPS9pPE52Z30/H7S23Obb0GMpWpY5nCQjGyvbB9jqFGQiJWn1RPUqFkocPHxISEoKenvbn39t1j7NrzrLo1iKtIsu9jzvKViVXP79K7KpYrH2sUTQoOLvmLM1VzYz+YbSONMyjo48oSiqi7xt9ufyR0IjtikV384ebPD7/mPit8ToFlSZ8R/pqrQH/u+LmjzdJ3Z3KhF0TtLzSQJAPvPbZNbo/252xP4/FyMoItVJNVWYVjy88Jv9qPmfXnMUt1o3AiYFIpBLkZnJs/G2QdLPjjMtubkbdxMhfaF4X3SqivaUd/3h/SspKsKi1wKrGhRqTFjz6dWfE1yPEBpZPvQ8yUxkquUpMwLu6rwGCpwbjFusm+jaAsK48PdHyT3FmzRmStyazd91eKiSCdPUPC37o8rn2pfb4pUpQB9Uz4usRGFkbCZK41kZ4DfXldl1fSr16MdJTuIidejhx6pVTAAQ8CCDqfhS/fvuryFQNmx1GGGE8vPqQkxknUZeraa5uxsjKiMayRqoyq6hrrePshbOMaRvDKL9RPDr2iLu/3mXguwPRQ7iGNf4TyjYl2/puw2eEj9Z9/43vNxjbGLPwxkLK08qpya3Ba6AXxydbUmR6nBe/eZHK7jm0t7jzudPnhM4K1TI479x8kpvJ8YzzJPdiLj+E/kD/t/vTWt+qNb1s5W0l7vflaeUM+XRIh7STWpg+DpwQiKGFIU2VTTRVNGHpYSkCuj7DfKgvqufBgQc6k6ftLe20NbVhYGaAnpEebU1thM4Mpb6gnsIbhXgMF1jdtsYdTdvoJdFEPh9J1aMqFA0KpHIlNvnJtBmaoVZ749rTlaxTWYAg3eUY4UjvVb1xiXGh5nEN1TnVYiMoaEoQ47aO0yIE+I/1x3+s0LxOeC2B1N2pLElbgl2gHXPOzUHRqOAj048AoYllbGvM8RePk/hNIjOOz8BnmA+FiYUceeEIZffKWKvU9j6rbq4R/61QKjAxN2HGsRm0t7RTm1eL0/tOzD8wn1ifWPb/sF98ro2fDXHvxYGfID14auUpAicG0m1MN/QM9LSAn8fnH7Nj0A5GfTeK6CXRPDz8kFs/3GLwx4O1puo1ETw1WJzEv/b5NXos6kHgxEBButHeRGzuSaQSJE82z1s/3uLG1zdYkbUCPZeO9bcrKWuJm4Rcz1y2LtjKZ2EnaShp4MceP4prvNsJN53X/FdRcKOAlF9SGPTBIIwwouJBBdv6b6P3K70JnxeO91BvSpJL2DV2F3UFdV1OwtwzvEexczHhv4ZzTX2T3sujxAL/8fnHLNy2kDMDzpDnIwCt5946BwgTK1WZVTSZ2dNiYkNa3+dwe7I+9nujH2ffOst6yXqGfzmcXi/16vKzc0JykJ+UoyxQomhUcH79ea5/cV18vDavlhdzXkTfUJ/HFx5z6pVTDHx3oI6sLAikgfaWdrpP6E7KLymoJCp+fP5HVkwX9vWy+0IDOem7JLr/1p1XMl/BvCkcgPrKVtL3p2Nsa4xzpDOhzVHkW2SREmOFnrGz2PxM25dG8LSOxvbVz65iZGNExLwISu+Vsm/KPmJXxYp+fM8nPy/mnjNPzBRfFzwtGNdertgG2CI3kzP/6ny299+OvqE+zlHObI/bjoW7Bc/seEYEzC68ewGTJhOcip3wWOLBA5MHhM0OE0lw8dvi6fb6DDKNrlCdP1H8rDs/36GpoglV7xgUcgXmdeaojZpoQ/CHSfwmEYdwB4qdimkwbcA+1B5TQ2H/VKvVogysgcKA+I/i2bJrC3HvxPHwr4c8/OshMUtjkJvLsfW3FYlWO8fsJOdMjhbYl30mm/K0ciIXRXJg5gHS9qWJjxlaGGorUTy5lVx7uvLX+L+QqCToKySgVJL4bSKtta1Y+1ijt3sXJiFjaDGxQdZaR0utIU4DnERSpSaK7xSTczaH4CnBRMyLwDPOk1L7UvHx7ve6czrlNIM+GIRUT0rqnlROrTzFpL2TtLwWn/ZKv79LIDXOuzSvS2m6uoI62lvasfa1xnOgJ3qGeoz8eiSX7kKSJJlS16/pNtSTmcdniL5CZ9ecBUCmaKDd0BSJRFBsuf/bfUZ8PaLLpqBarebYkmN0n9Bd3Ns0Ezwg3EcVDytwjnQWSQFSfSlW3laEzgrl8fnHNFc2M/jjwUQ+H0nozFBkxjKuf3Wd25tvM/3YdPKv5tNU3iQ2bNtb2/lzxp+ETA/BLdaNmpYanr81GEefiei39cXEwYyzb53l0geXhFMaIKHcvpxjo45R4TMXuZmciX9MRK1SY2hpKNZi3q7elKWWITeXoy/XJ+9yHuVp5RjZGLF34l7GbR0nAq7Zp7JpqmzC2MaYigcVHF18lGFfDOPxhcc4hDjg3ted2rxaQcLuyaRReWq5CPQBXFh/gZbqFi1PVgMTgbRza/Mt1Co11z67hoGpgRYBThMSiUQkuYKwn5fdL8PUybRLdZWn48GBB+ybso+JeyZqgYKZJzK5/dNtsXmrVqn5qcdP1BcJSgJBU4K4sP4Cce/G4T/WX5QZ1ETFgwqUCiUOoQ6k7k4l71IeLjEu+I70FSxDttzm8oeXhX7Bk5wk/c900SvK5eE5jGsKkUpfB4RG8P1d93m7/W1qc2s58dIJQmeGdrmugwA69XuzH0FTgv5bE0pd+SGmnUpD0aCgIr2C6KXRDP9yOHe23qH0Xilhs8IoT+84j9e/uI65q7ngobY+jgFrB2DX3Y5uY7qRcSQDtUpN9ulsAp4N0OoZxL4SS/41QYXnaQuJzjW2bXdb4rfH49rLlYaSBuqL60ncmEj+1XxmnZ6FlZcVLTUt1Dyu4ceIHxn4/kAcQh3QN9Jn2GfDuPbFNYxsjOixsAd+o/yw9rVm0e1FyM2FiaorG67Q66Ve6Mn0MLE34cY3AhEi4dUEEl5NwGe4DzNPzMR/nD8LExdSm1fLRt+NBE4MZOF1XXlHtVItgp3/JEsPaE2Bds4DU35N4egLR5l1epZI4FynXkfqnlQ2hW5CrVQzfsd4kYT2d2HqYErgxEDamtpI25+GjZ+N1j3TVahUqr+tbf9XY9D7gxj4ni4BJfdiLn9O/5PmmAFkG+v63QKUpJSQcSSDo4uP0ljaKNYDcjM5yx4uE4m7wz4bhr6RPqp2AcgJmRHSpU0IQPj8cDwHeQJCrZP0bRKqNhXBU4K5+cNNji05xuwzs8Xr8tzb5zB3NRfXiYJrBYAwCVqYWMiwz4eJ05Rp+9MwMDXAOdKZ5F+ScevtJpIyTr9+GmM7Y2JfESQwG0sbsfG3wdDSUAR03qh7g7FP7OSKu8XR/VnB4/vOljuM2zIOlaUwFSxta8WgpQ79BgkHZx8kZnkMLtEu2AXa4TXeC4WBApdCoe4AYR0xMDNg7I9jxTxaP0ufShdhAi7jaAaxr8Ri5PRkIMCwhd7re2PtYM2lDy9x9k1hf1yavlTcj41tjbEPsdcagEjfn05tbi2DPxjMgHcG6Nh7VLmGMfzzMKqzqzn75lkCJwYilUlxPO6I3F1OUkwSvot8SX89nQcHH6BqVzH56GSMPzRGLVVjbGOMhYsF8y7NoySlhNubhX5OXUEdlz+6TMD4AKYcmILXIC9mn5ktkkwMTA3Efo9arebNlje1QDxlm5Lci7mYOZtp9aCcI515dPwR1768Rq8Xe4mv6byWGdsai/dV8Z1iQJDmHPXdKDz6e1CYVMjljy4TOCkQh1AHXil5RavX49TDifbmdtQqNd3GdBOVJrqKv+b+JZ7TyEWROv3Ef+N/Hv8n1rz/SfwL9v0b/zER83kMiSsTAfh9xO/i1EbqnlROvnxSlBg8dw7c7ybwzc+3eLX8VSR6ElRtKi2po8fnH6NoUNBtTDdCpgsTUyA04buSilM0KmiqaCLy+UjkFnJKU0pRNCi4++td0vamYepgysX3L9L/rf4ETQ7COdpZW5u6E5NHI4PUGdTzHOiJnlzvbxO3qX9NRW4hFwGPhtIGPnf8HBAM3qX6UgHEe8KKM7Q0FNmIS1KXUJZaJkh7PJGJa6lt4daPt7BwtyB4ajB2QXbUF9Vj62+LolHB155fEzIjhBFfjSBkWojIjF9ZtJLYVbGEzgylOrsauyA7eizqgW2ArZYMKAgA598xT298fUMEFtap13Fr8y1kxjJCZ4SKz4ldFcvNH2/yQ9gPTN4/GWtfa/zj/Vn2cJkwvWFrTPHtYmrzajumyRAY8X/O+JMpB6ZgbGNMQ0mD1mc3lDTwhfMXRC+LFsAitdDk0/i55F/NJ+9KHvqG+uRfyaffmn6YOZuJ/gKXP7mMRCrpskkfOCGQ1ype48ybZ8Tj0VzZzONzjym8UUjBtQLc+rh1eY0BzD4z+2+BlILrBeScyyHgmQBBvqaL6IodBEIjdd6leVi4W3Q5rfbX/L/QN9KnqbyJtL1prFWuFRqfT97PzMUMI2ujLq/P6CXRwiSkRJupduG9CxRcK2DGsRls7bsViVSiI29pZG3EqI2jtOSzOkdTeROVGZVM2jdJB+TRRObJTM6vO8+QT4aI4G3G0QwOzjmIuYs5gZN0gc3G8kbB98hYxv0/7mPhbiE2dPq+3pfW2lY+sfqER0cf0VTRpGNynnsxl+Rfkuk2phveg711AOtTK0+JMhRqtVoH8Nw5ZieqdpUoSdQ5NKD18RePk3E6g5C7HQVKS20LDw48wMTehNGbRncJrJXeK+XWD7cInRGKtY/QtG2uaha8KhZFkronlbNvnxVZclMOTqGtqQ1jG2MGfzhY5/00UZhYSOruVErvljLzxEyt66i5upltfbcRNDXobw3h/1cj52wOR184CtAlwO4/zp8bX90g/c90hn0xDCMrIybunkjC6gR+ifsFEIgHo74TvpeiUWgwmLua47tuCZ+l/IBCrqBNJchOaqRP9kzYg+M1R2bUbOD+4PHoqYxxCBUmcmKWxXBmzRmufHIF60XWSB9KaYxqFKRBM6swsjYS95u0/WmgFkgjfye3B7Cl9xY8B3lqgRoJqxMouFbAvIvzaCxvFCeXpoVNY+PdjYwPGA8IjcfjK44TPDWYfVP28Q7vkBKaQthdA5pcBOlnTbxW+RqNjbBjKqAnQ/pkKWpraiN+ezwfb/2Yy3WXCTYKZqSjLpiuL9XHtMEU70XenL9/npFfjxTllNrz20nflo5TjhMDlg5g2OfDiFsfp8UgFEMtkAuaq7Q9Vb0Ge4lNiaRNSSR9m8SC6wtIdX8RvXY9Ni7byPB8QfIyelk0LtEuVGVVUfmwErc+biKTurGskdq8Wmy62eDe150+r/chekn0P06k2AXaaXm4lKeV89fcvwifH45brBs3f7jJubfO8fyd53EM73DG6iwl1zmStydzdPFRxvw4BsdwR1xiBC+ds2+fJetUFsUniwk2DcarzIsvvv6CZQ+XYWBiwPeB36NvqE+vC7345r0DzNhsQp2dDxKJDzNPCr6u934XWKOmDqYM2yBM0l366BJn15wVvGYuC3udxqOpc5x+/TRZp7IIGB+gM1msqBckgh3CHFA0KGitayXnTA4gAAjtre3ilHivVzumIA4tPMSdLXdweKMbw88Np9ailpStd4lbKsin55zLYeeonfh9JIA5tXa1WsfQ2teavm/05d69e0gkEuZdnve3sjKmjqYETw0W85m6gjoyT2bSa2UvmiqbaGtsw8LdgvaWdg4tOMS9nffo92Y/Br47kN6v9Bavr7DZYVpTw41ljVQ8qMAu0E4ELIztjCnOKGbi3olU2FZgUmRCc99mrWk7lUqFRCVBLVGzZ6+SmaESrLys8BvtJ8h+nQDDZkOcip2oK6jD3NWc2rxaUvek4j3UW5xY/HXYr9gH2zP8i+EM/3w4gz8YLOYDMmMZtgG2mNiZiNMW78neQ9Wuos/rfUTvDhCkwO//cR/jbsa49XUj6mY0p+sziVoQRklKCRZugoeNa64rIfdCmH9pPne97+IU6URzZTMNpULOVGfnQ4PNE5kxS+H/UnakiE3+ky+f1JF11DQ7JOslEAj3J97Hb6QfbU1t1OXXYRtgS8qOFIE4t1yYalC2KmmqaKK9tWN9b65uZt/kfXQb142UX1JormwmdFYo8y7PI7klmV88fqGfhzB5lnsxV3zdwxVZyF6Q4fPIDsO6Mmz97VmnXsfuZ3fzrf+3jGIICUPUZHe3wK3SViTAhM4K1fodVzdcxcbfRrh/1Ojs4V1N7IMgJdp5st6ttxtvt72NUqXk0yufYvSKEeYq7X07Yn4EM/JnoJKqcNN305HNk0gkIJEiUUuQqDsmWoysBRDYsORLbCps+Hb5twy7sx5XhBwIoO/6viy+sxiJSkJrdSttFm3IjGWCvHfTm3zq9ClBqUGkBaYxfvB4kTQJwv3+dG4z9qextDVpSzTf33WfO1vuEDY7TPBMNTMQ15HSe6XYBtgi1ZPSVNFEfXE9LtEuYi7pXOTMjC0WVLZdZ8LOCaiUKuRmctR9+6JsMsS4tpjuV7eSGzSEXgP6PDkWHZ+dfyWfhFUJOEc54znAE0tPSyoKBSKMh4UHPW/25MrvV4R1X0+YtgW05ALbmtpQNCowtDQUiV3ufd0FdQcPC77x/QbHMEcm7+8glRyYfYDSlFJeq3xNax3Z3nsT5RbHoX06ISHDtI7TvLRVvPJCI0qZIZbF6ehZ2zNq8yhcFrggN5eTfSabwsRCgqcEc3/3fdxi3fDo78Hss7P/di18eOghx5cfZ865OSKhVKVUUVdQh3tfd3yG+XB/133C54bT3twu1oVqtVoEum98dYPCxEJhikRPmJCZ/OdkUcL+s6uf8ajxNo+ibmOBGhNnAbDSRKOpcK7rLOposOi4LyRSiY4qg32QvQisHV1ylJubbjJ5/2T6vtEXh1AHIuYJJM4L6y/QUNKAsY0xjhGOzEqYhaGlIZujNxOzPAb3vu585fEVIKw5S+4vIe9yHte+uEZ1djWlKaU0VzYzdMNQLaUUDUFwvWQ9AEM+HSI2RXPO5VCbW0vIjBD0ZHqo1Wqaq5rRM9BDbianrqCOzdGb6ftGXwZ/OJjy9HK2999O7Gux9Hm1Dzlnc6jOriZivjAdbuVtRcSCCCw9LVE0Ksi/mk/miUyuf3FdbBiDsNZpgL63Wt8i73IerXWtKBXakryaODjnIPXF9byc9zKqNoG4VJhYSPGtYn6J+wUjayNsutmgJ9PDe7A390PuY2hpiEQq4d6u+5R5RCF1C0cigeMrjlOYWEjgxECSvktC0agg62QWrr2FCecza85QfLtYi9BgH2zfJbDyT1H5qBIDEwPxvrPpZoNTDycCngkgenE0EqmE619ex2uwF2GzwgicGIj3YG8MrQwpu1+Gql1F2b0yLSLitMPTKLtfxqaQTUhlUsZtHic+dmz5MZK+FXwbDS0NWZ65nLu/3cW1p6vO9LaZk5moDqMBHRzCHNCX6+M92Jvdz+zm4aGHrMheQeisUBxCHPAf68+bTcKU/GcOn2Hjb0P4nHDx92lsNC69f4nk7clELxburccXHotr47DPh1F8qxi7YGG9l+pJufvrXRI3JuI52ZM/Lf/ENseWgV7Csa7OqebOVqF/NHbzWNz7ulN6txRlmxLnSGcaSho488YZuo3rJsocd85TytPLkepJselmg32QPdFLo0XQ+viK4zj1EPITtVJN5PORogTlN77fYOlhyewzswFBojLrVBbdRncTlR2aKprYO3Gv1kTu/80oSS4h92IugRMDtdZ2156uxG+LZ9s5T1qrLMW/d57si98ST/yWePZO2iveh6l7Uvlz5p+EzQ4TQTAN0UetVvOW4i0dOffOoTn+gKggpJnQtPK20qkHJ++brNUX0qh3TD82nfaWdq3ezokXT2Duas64LeNIWJXAgHUDRLAveXsyNn42xL4SS+K3iVx6/xJLHyzF1r/rXpwmmiqaKE8rR6lQij7vFuVZ+NzeS/2Q8Uw5OAUrLytS96RyevVpBnw/gNLEUj5+42M+XPshAJUZlWL9te68kAd+uPxD1OvUXPrwEje+ukHY7DAMHIXjoJAr6L26N8Wni8UaH6C+sJ6Wmhbh2pTAvEvztPKtaYenice+c89QIgFUKtzvH+POVhes/axJ+i4JG38b0ven43XBi54De3JxwEWUKiVjfxrL2J8E1LOlvQW1VI1MIWP7/u0sHLYQEzsTHMMcGfDOAC68c4HLH10m9rVYkXTztBxr5xxRIpHo1L1tjW38OuRXIhZGaK1VyjYlyduSSdubJubEAKhhZeFKGkobsPS0FKbEjWUcnH1QtMKIXhJNXWsdSXOSqHpUJaqdPN0vi10VC6v+/vz/Pup39A31mfLnFIZ8MoTKR5VCD/NfoO//F/Ev2Pdv/MeE00AnPOI8yD2fS7OZHcm/3cchSGjkDfl0iJZvVr2tJ5ExUvTkehyYJciCrG3vYKCffess1VnVooSfZnG18raiubqZU6+ewqOfB95DvPnS7UvcYt3IOJJB71d6M3D9QHESYfCHg+n1ci9yzuRQ+bCSA7MO8HL+y1pG0QANxQ184fIFMctjmHFihk7joHOxuDl6M2593RjxZYdf0tOsNI35uNxczoxjMyhLLcMt1k1kwygaFbQ1tWFoacjJlSfJvZArehmM+GoERxYdIXVPKs5RzgRPDcY+yJ5xP48TX2sXaMeNr29QX1jPpL2T6DamG6aOpmKy0nmTG/n1SFRKlShvoTmWyjahgSM3k3NlwxXu77zPwsSFGFkZ4THAQwT7QEiITRxMtDZuAFWbirbmNtGQHbXQFNR8xuHnDtNS08KKrBUkfpdIZUalCLLd/vm2UFQ+tVnpG+oTPC0Yubmcd6XvioXbvMvzqMyopCa3hgvvXCBochCpe1KJXBRJY3kjVzcIMqgp21OQ6HUN9oGQXI/ZNEb8b403w97Je8m/ks9a1VrubL2DRCrRacQ69RAmympza7XkS0EYrT+75iwLExd2qQP/pduXtNS0oGhQED43nPht8eJjBqYGuPd1p7mqmcyTmdh1txPBmtr8WpK3JeMQ5kDfN/pqAbQSiYS1qrUkfZ+EzEgmJrtPR1cgY/6VfLITsrn88WUqH1Z2KYWkib9rZAx6f5DovfR0KBVK9k3ZJ3gEFdbT3tzRKJSbybEPtqf/2/11ZCfVajWf2X9G4MRAJuyawJ8z/iRoSpAI9rU3t/OJ1ScEjA9AZizT8j/QRMGNApK3JpO8NVmryaKJaYenYWBm8LemyOFzw8m7nMfBuQfpvbJ3l0zI9tZ2rd8Ewjry17y/6PdWPxGs6xzpB9LJOJTBvMvzRADczNlMkD6OdkYikZC+P520vWkMfHegIMlnJNOZ9usqxv08Dht/G5K+TdKS0AFALbCT/yfyjP/d0DPQwyXGhZjlMV2CZR79PFhVtgpFg0Jrqs5/nD8WbhZY+1pj6WUpFjsVDyrYHC3IHhr49sWxZjzLv3yGrCtZDDk1hMqHlSCByBciKUkuodGnB3oq7euzsbyRlF9TyOyWiXmdOdYrrXlg+gC/UX581/07er7YkxFfCev3ubfPIZFI8B/nT3trOzJjmSg9WPGwggMzDxC1OIravFot70gQwG6Nv1pzVTNN5QLhpCS9hMDUQHzyfchzFUzXHxx8gGOEI/pG+rQ3t3M39C4lHr70tde+tpJLkpn8x0ya3JyITf0NVas1IGOD3QY8+ntguNqQ2gu11EbW0nNMByP7+IvHKc4v5sfBP6IwUFAxqkIE172HeqNUKMm9kEvchThUF1RUDK7Ao5+HjtzMnzP+pOJhBYtuLuLl/Jd1zufYH8eK/w6dEUrSt0ls7bOVkPgQai1qyfPIQ1Figr4cRm0chapdxXuy9wBBpqw8tZz+a/vz+Nxjji4+intfd+qL61l4fWGX60xtfi2J3ybiN8pPR67o+pfCJNKDPx8w/IvhuPd1p99b/bSuw++Dv8fU0ZTZp2frvLdtd1t6PNeDxG8ThSk41VokEgm9X+6NokHBja9uMODCADJ6ZIgAQVlqGe2t7XgO9KTnzz3BCS4N/Qi7xg7PYVt/W2rzarn43kXeaHgDA31hX/Yd7ouhhSGBkwIZ8dWILv2KUn5N4conVwB4Luk50cu4Oqea5O3J+AzzYe6FuZi5mHHjmxucffMs43eMR0+mh+8IX6T6Ui6OvEi+VT7qUWqGIzR2NCSmxmtV9L7eG4VMwc20m/Se3UuYXLIzodfKXhz/9TjhfuHcD76vI7H8x9g/yDyeidlpM3wGd8hGa4Af//H+xCyNwa67ndZke/TiaG5vvs2vQ34V/7ZWtZbmqmbu7bwHCH6OEqlE9OTqKrJOZXFg1gGmHJxC7KpYSu+VIjeT01jeSHBqMK0GrcgvyGl4sUFbWvM9WHdmHTtm7cC2uIp2uYkoU97W1IZfhh9m9WaMOzyOB9EPiFkWQ+WjShJeTWD0D6NFsK82t1a8RqX6Ui25Ngt3C52p+gHvDMDY1pio57Vlx4tuCpLMTRlNLMhYwA9LEuiZ9QZVWVVsjd1K3Po4+rzWh42XNpLSnkLMxRiQCPtW5cNKzq87j9TClNRaISeVtrciVUoBmSj1ZWxrzDO/PcPjC4/ZO3Evwz4f1qXcskayKXBCIIETAqnOqebWj7cInRVKZUYl+ob6+Azz4aXHL2m9Tq1UU5hUiEOYA+N+HoeyTYlUT4p7H3fccSfxu0TSZelELooUAVHnaEFaKfh+MPGH+lDsk4JKJcjq+Y/zR0+mx46ckxS4FmClEtbgp4+dJuacnyM23xxCHViSukTr8bL7ZajVahxCHLjxzQ3MnM0InBgoEPvUgEQAoSszKnGKcGJP3h7WnF0Dozs8bvZO2otrb1fsAu2YfHYyf/b/k/gv4nEOckapUHLx/YsY2xqTvC8Zn7YK+hbPwT2vAQTrckJmhJB/NZ+wpFZsip7h5+d+RqIWvrNtgC1hc8Kw6mYFd8Cm0oZNzpu0cgd9Q33iNsexYN8CcoJzOPWWMNm97OEyah7XiESA+qJ6rmy4gu8I3y5l+/q81kdo4pkYaEkZbrDfwJnXz1CSXIJLT6H5dvP7m1qywk3GTWT6t+HtYa9NDmxuxuf2PlL7LaLEO5boABdaalsoSS7B2sdaBKICJwbiHOWMXaCdkH9IOq45iUTCsM8EwolUJpxvjdfVxfcvYmxjTPeJ3anKrCJhVQLzr84X6weN9yAIShAWntokuZAZIVp1WeqeVFJ+SUHpI5AGj4zeST+j32mqbOLeznu49nLF1MeFFnMTTKrz8b21hyajYUj1YtA31qf4VrG4fjlHOXN2zVn6vdkPzwGeeA3Urv9a61r5Pvh7gqYEET4nnDE/jtFSI2mubOZrz6+JWhJF/NZ4xv40lk9tP8Uh1IGxP43FzMWM3i/3FiWULdwtKEwsZP+0/RRcK2DR7UVazeqalhqtz1cq4bnE52gsb+Rzx8/pcdOFVv3hZIa4YFPlS3NVM6ffOI3PMB98h/si0RPyzPcM3sN/rL8ImsYsj+HmppskfZ+ktX+GzQnDvZ+7KIltbGOM9xBv2lvbmX1mtnjuY5bH4BjRQRipK6jjwYEHTNo3iYRXE6jJqeH066e7tEXwGuRFztkcrTru9ubb3N91n5DpHQS7DbYbCJwUyKQ9kzCyNmLIJ0Nw6Snk1vqG+tgF2Yk5zq2fbpG6O1Wsk5yjnMW6uuJBBb8N+0183+hlHZNlUn0ptt1tqUivIPdSLt6DvbU8248sPoKiTiFK3UUvjRZ9u3s81wNzN3MsPS1xDHfEaXgIV0t9MI8OY253SPzmGn6j/Bjy8RDOrDnD5Y8u0xy3jFZTG6RS4Td7DfJi0t5JvGfwHuoYNQsrFuJsJlz7NTk14uR050jfn87eSXuZsGuCSJb+uzi69Cg3v78JIPRQ3h1I7sVcim8XM+r7UbTWt9Ja18pzSc+JYHTnukRT73YmFNcX1ZP+ZzqKRgVTDk7R8qAHRKAPwHWQK+UF5Zx86SQD1g3AIdSBzdGb6T6huw75yzPOk0EfDCJiQYS4RnkN8cLI1ghLD8suvRZnn52NzEhGe2s7SoUSAxMD6ovqufXTLdz6uBG1OAp9I31a61vZMWgHAH3X9KX3yt6o1U9ACgSykbWfNSO+GcGc1jkUlhRS8FwBO9/biVtvN2pyarj0vkC0KbxRyKxTs/hr3l80VzfzYvaLNFc3k7w9GXM3c/H+rcqsQq1SY9PNhj/G/YHMRMYLyS/g3tddrHlVShWJGxMxsjbCc6Ani+8t1pqGde3pqkUKK7pZxIkVAuCUfy2fEy+eIH5rPBP+mPBfgkr/p+Lwc4cpulmEY4SjFthn5W0lSCzeaEevpBmlgZC3deXZp/F3A6HH5TfSj/LUcoEo79fhpy6RSNCT6WlJzfqNEmRepx+brtPfk0glWqCQ73DdffRpAHrktyNprW3VmThXq9SM3z4eqUyKlZcVC64tENdDgKVpS8Wel/dgb/QM9DC2Mab0Xim7n9lNn9V9RPUu1Gr8r23n3DpPBq4fKA4gFAuDYzSb2VLk1x9jeycC4oXroeZxDca2xsgMO3oGSrUSfYm+liy9TE9Gq7JDQjtmeQxhs8MwsTehWdVB8Hzv4nu81+899A31aW9pZ/hXw7nyyRXu7bzHgmsLKLpZJJBZzs8R67Onj4nWsUaNfd4tDi24xcIbC1lVugqZiQwTOxNyL+TSYKpk9JHRXEm+gv+Bjp6UZiDDvsyemg9ruPraVQxMDPCM8yRuXRxhs8KozqnGtZfr3xK91Go1j889xtjOGLtAOwpvFGoNWciMZYz6fpSOstRG3400Vzcz/8p8rV7a2bfPcuVjwYdyx+AdtNa2siJrBbGvxYq9luSSZKK/jeat229h7maOY5gjSoWSqqwqjG2Mu+yZlCSXkJWQRdDkoC7rob+b6P43/nPjX7Dv3/iPibbqNnLP59JmaMbJut4Uz/qKXi/3YvgXw3X0wqudg/F/KRi5mbAJPz3NFLc+TkyW0/9MFyR8pgZj4WEhSnuolWq8h3jjEOqAY4QjobNCtSTYQACerH2tcevthscAD0qSS3S81EAoCgInBeIY4ajjp/V0tNa36jT5QWi23t58m5fzX0aiJ7AGF95YyN3f73LyJW25mdubb3Py5ZPMOj0LS09LLL0subrhKiBo1XsP9abyUSVjfhrDBvsNBIwPEBkuBiYGzL0wl/WS9aTtS+PUqlPEvhoreraVp5dTmVGJRz8P5BZy1Co1Vz65wrm3z/F88vNi46ryYSWbQjYxYN0A9GR6WhNrPsN8eLPlTZH5MuXAFJ2psS29t+AxwIPlGcvFv22w34Cpoykxy2MouFpA+LxwsVn46Mgjci/m0nd1X9z6uOE3yq/LCTojKyMm7JxAQ0kD1ZnV4gi+ex933Pu4U/moEpdoFxwjHBnxzQiMbYzJPJnJvd/vUVdQx+yzs/lr3l9s6b1Fy1dDc2zyr+bjO8JXR+YldlWsWERe3XAVPZmeDtinaFBQm1urJbupifA54XgP9tbxc9SE32g/7v9xHxCkfTqHBogpSS7h9xG/M/LbkaJkkEZaqPfK3jr+DBr279UNVzG2Ne4S7KvNq6UwsVAAZXq6iAVSyIwQshOyOfvmWXyG+2iBj5qoL67n2NJjuMS4EPBMwN8WClt6b8E+1F4LCFApVaQfSqe4VzHPnHsGX9+O5Nmjvwdzz8/t8r1QC0COUw8nJHoSJv85WeuePPPmGQC6je32t+Bm5HORWHpaUp5WrkUy6Pz5/xSxq2IxsjHi0PxDBE0O0gL7iu8UU3avjEEfDsJzqafW68xdzZl2ZJqYPD4dtbm15F3JQ0+mJzLUTexN6P5sd+oK69jWbxtyczlL0zs8pyoeVFCZUcnVz66Sd0mQc3v292e1Gh4gFCx9V/el7+q+Op9rZG3E0vSlXP/6Onsn7WXi7ol/O2X6Pw33vu4svLGQ7wK/49KHl4Ri5unPf8ps/t6ue0j1paJhuFqtRqVUIdWTYuZsxuCPB2NoYUjqpPXMQI9GMxsKFd1Rq+H7oO8BeLv9beyy7Bi3biNmSfpYts/l1uZSEjcmYh9kT0NRA+W9yil1KKXP+31w6uHE1titGFkbaU3eatbVG9/cIOHVBBZcWyAWdmqlmqaKJtqa21hZuFLnd8Vv7bhnrH2tWV29GqlMykWPi4xpGoNxszGZTpkMem+QIKeoFuRzJeuFY++UNgmpsSGbQjfhHOnMoA8H8ZfzX8xgEmcGnSHo4g805D4PfRwJnhaMsa0xr+56FQMbAy7kXtD6LhVpFRRnFpMels6woGF8/eHXSCVCA3Xy/snc2XKHy1mXKXQpJNcjl1UBq2gsbxSmrDwsxAJYZir7W+nmp8O1lytjN4+lKqsK1ccqHvo/pMq6ClpbAN19tr21nXs77wms5J4uDP5oMDW5NcJeLxEaH4/PP8Z7iLfYSKwvqufqp1cxsTNBrVKT8GoCgz4YhO9wX4KmBnH3t7soGhSolWo8B3jqAIIOIQ5C8+OlE/RZ3UcLcNY8/8FfD6h8WElrbSsnXzmJ92BvZEYymvs1cyDgAEUuRZxaJzTaU/ekCtMF00OYtm0a5w3OU2vvhUm1AACcW3cOGz8bnKOccQh1oLGskX3L9hE8LRjPOE8iFkSI++q1L66RuDGR2Wdmi+uURtZ6yoEpIugMQhF/8d2LmDqaEr1YaEQ6RzsTsyIGzzhPrTXybE9hzziZdZJ34t4B4Pk7z9NQ0sB1/WzejViDaYMp+yb9RVtjG9c+u0b0smhGbRzFla+vEKAMoHt6dzZs2MBbrW8BwoR25nFhWkRmJBOnGvQM9FC2KilMKtRq7D4dviN9KblTgl2QnZCrqMHEwYSVhSv5Y/wf5F/OR61SU5Nbg4GJASb2Jjy+8JjCxEJ6LOyBkZURztHOjPh6BPbB9gTEB4jv7djLkfVr12NZY0lQ2RTmyYVjcfvn2+RdymPQ6EFsq/6FIaeHYFmXi0JILdnWfxt5l/KYwQySw5I5MvoILwx4QTi2Uc4suLZAyyds2cOOSe/KjEpa61pxinTSaR79Nf8vmquamXpQkFZM+TUFtVItTpe21LZ0nFeLGhRGeihlhpg6Khn62VDcYgV1gfJB5eRm5GK3zJ7QyFCK/x/23jI8qrPd+/5NJplM3N3dhSSEQNAAwd2KS5FCS1tKKdSoU7nrLRXcChR3dwsBAiFCEhLi7u4z835YzUqGpPveez/7ed9jv0fPTy1jK0uu65S/PCoi91auyJSzdNCizCkMn1tbUTzzgu9m0N7Ujr6Nvug9VxhXiIWfBXITOUqFkuyr2WKzzbjKmHll86jJqxHvnx2DdtBQ2oCFrwVb+23FppeNyBjoGrrmuqyrXqf2byqVisIHheyfsJ/6onqsgqwIXRoq5loduceEExM4PPkw3mVLaa1vJet+AQ79BE+0hTsXYlppyoDTCVQ5p3Dn6ypqC2rxnuCNVZCVuI/8R1LfILAnO9DVV969gkOkIBN95+s7XHrrEotjF5N7O5cLb1xgwHsDyEjPYGDpQG4MugGAsl3Jk8NPkEglGDkaoVOqg1abFnJNYV1rb27nxic3sAmxofhhMX3pS4ZbBrmunc+s+0h3Bq4fyI2PbxAXGkf09S8waywDc0PcR7hj4mpCfnI+UZejeOb2jIzIDGaECWyitqY2Cu4VEP9lPCamJjxTPRO/18zTTG141FjRSOz3seiY6GATYoOyTanWUH3+/SBIsDeWNeL3keDdZuxsjFWgFabupmpyclWmVaT0DmVqqActdS2CL5q+DImXJ4W1+rTom5PvOxyjIChLzmPn4J2ifCyoAw87FEgG3RcADBoSjW4yg2/Xvs3vvX4n91YurXWtmHmZYR9hT+hLoX+rHvG8Fyaoe9zc+eYOF9+8iKZck1mxM0nyf8y1Ide4eRPmj6zj3KvnGPzxYExD2jEubqLexJ7MXpMxd7WhNKmUuqd1NOYJQJ/RG0dj38eepQ+X/u3xSKQSDGwMkBvLsfS3VGvMg9BU7Lu6L+mn09k3fh+ZFzPFc/V7r99F+eOOCJwbiI6ZDtqG2tTk1nRbbzpkpr1SvfC8u5ui+Ghc+1qhUqoY/ctodn5wgHLzcqIuDMcuZxvN1a/ycNNDtA21OTj1IABrytfgNd4LQ3tDYr6NwXmwMzYhNgx4d4BaDt2hzDJl/5RuTVVNbU21AWsHS6+2oJaUIykYORox/9p8cm/l0ljeiOtwVyz9LUUJZxAkgZurm3tcc/qv60/ArIBOWWeJhJClIaKijo6pjpqXtYmLiVqd0f/t/sIa3EPqq2elJwxlvcywDramPLWczMuZAnvNSE7UZ1EcmHyAkoSSbgDFitQK0UICUANVmPuYIzOQsXf0Xsy9zTlpuADsoSILSkvh6cmnlDwuYdgXwwiYFYBZgA0PdwnProaGwBzpAAU9XvuYG5U3OLn/JPeXCMOy5+0iQFCZObbgGNpG2n/ra941Sh4LsrqaOprEfBPDgHcGYBNqQ+iyUGrzaonbFEf8tnhWPFkhDi6aKptIPphM8aNiIl6PwMTVRK1HUJpcytmVgn92+MpwvCd4c2ndJeTGcvqv688bBW/wrd23ALxp8SbKk0ru37yPkaMRSoVSrAmWnFhC1u0shu0ZxqjPRxE0LwiHfg4UPSxi+4DtVKZXMu3QNDUmDnR6CupZ6fFmsUCZ6ZC2fTnlZTb6CF6Zfd/sKw6bpVpS+r/dn5LHJTy7mMWmG4U0Gtvy8sswciSkHk/l3KvnmHN+DtIMKfr1+gy7PIzsodk49HXAro8dK56s4P7G+zSWNVJfXE/EGxEoWoRcydzLnLfr31aTNTww9QBtDW2sTF9JxKoIEfTQNSQaElYXrSZ+RzyX375M6LJQtTWlY8jcEa7DXJl/bT6Wfpbkx+ZjFWCFvrW+msLSv4v/SSk7lVLIC/St9f9WQtTgzJ8EFuTwaNQ7wme6DPuKHxfTUNqA6zBXce1zGuiE00AnPjf4XGBYD3Bi55CdaOpo8sLxF0Qf8I54dvEZmtqaqJQqJFIJJ5eepCqzSgQxlKeVo6Wj9bfPi0qlQqVQidfO1E2o8RtKG2iuacbUTQC5Z13J4vCsw4zbPA4tXa1uQ8KuIDTnwc4iELm+uB5tA231PptEgry+nLoC9b27g/XYbGBJoZclrl3aMl7jvVCpVFzechkDawPsCuwoeFCAU7h6z0PzL8kYj6ce5NzMEfqFf+37eio9Ls29xLjfx9G4pJEHLz9gedJyVAoVm3tvRtGqIHJdJPo2+lj4WtDrxV5qdVVNbg1tTW2Ye5mze/hu5CZyph2YJnj2STRQSLWQKtrQNtQWh13+L/jz9h/+FDunEp5kQ0NLA2VPyqgrqsN5sLMIEKo1rKVpbhOm7qacWnqKwR8Nxmmgkzg07hqPtj/ixIsnmHV6Fh6jPEAFu4buwneaLxO2T2Bb5DY14L1UJhVrq67hNcELbUPtbiSRjnv52m9pmPbxwEhP6At3ldb99Man6DT9lbP+pUpTk1vDL76/qCl+JPyRQEFsAdFfR5MXk8elty5hG2orDvtmn+m0rvon/u/E/xfynR3xz7Dvn/hfEVKplFvzBSZYiXNvVBINmsdPI3iB2d9+pmPDCl0a2u21rsl01tUskvYlkbRPGJR4jPbgtazX0DbURktXS/TNeLTtEYdmHGLYl8NQtiuFzctQGyRCEeI0wAmnAU7c+uIW9366x8JbCzFxETYHHVMdph0QUEP1xfUoFUq1YVDsT7Hk3cpj0p5JPUr7bdDfQFtDGx6jPdDQ1GDwB4NF6cOGkgbcot14vOsxTZVNeI33wjLAkrDlYUhlUh78+kBEzHZsOiGLQwhZHIJSocTc25zWulZRsqBjqOc82Jnsa9nEfBNDr0WdKLfHOzuZAeErw7n30z2MnIwInBOoZkZ9YvEJAOzC7fAY7cHA9waKrz1Pce/J3LmusI6mKnWJN9dhrqQcTuHsK0KSH7UhStysph6YSntzO3oWeqJcZElCCUqFskeGlb61PlP/7PQ/KU8tJ/d2Lh6jPLr5xniM8sB5sDP5d/NFv8MOmYWukXM9h9PLT6NjqsPQz4cSujSU9LPpSGWCnErywWQurbvEpF2T0NLrzqb6l8W/cIly4aXHL/V4vB3mylbBVt2GLmN/G0vfN/rys9fPohRFR+TdyWP7gO1Ero1k7KaxaijI6G+iCVse1q1hAMLw8QvDL5BqSwmcG8j9X+538xNM2p/EpbWCSfCkPZNEdqZDXwdG/zIamb6sRx+lju9PO55G6tFUbn95m7VVa9VeL3xQSGVGZTefLRAawje33+RK1hUGNw3u8ftvfXGLqswqceACQmHTlXnZFcEMwtDYMsASPSs9iuOL1aTmOkJuLFfz4Ogp4jbHkXc7j3Gbx/Xo+Rc4OxC/aX5o6qhvwymHU4SCMfVlAgLUB24yfRmeYzyJ2xTH/gn7mX5outp1i3g9gqD5QdTm19JS1yIm1/c23uPGxzfQlGviNcFLDUF/cNpBSpNK1YrAiqcV3Y63PLWchtIGHPs7/u0gr+BuAU8OPUGpUCLV+J9LbMrTykEFbiPcur12c8NNHvwmSEFZB1kjlUm5/uF1pNpSfKf6omhRsNF3IzJ9GcsTlmNgY0D/tf15euqp+B37Zh4muPAVyss7v1elVPHJjU+Il2zmretvUer0GJXCCkWLQhyqXx90HYmBhGHvDhNlI3VMdfAc06mH3wHA6GDcdkXaWfha8FrWa/+pc6Ah1eDp6ac82vII2SwZeyv20senD6/Pfx1ATd5MQylHqdGMrN0SiURggGvqatJU0bme5jjlkO81EJmJsK6P3zKerf22sixmGTERMehq6bJt3zZxLZ17cS5bHm6BkyCTysRBn7JdSeGDQs6vOk9e3zwujBCGVnoWeqIXzHut74nPQMfAXtGmIGlfkgCW6VLk3NxwUxgsrxPWt45GycqslUhUEt785k3KXK/RWDGQU0tP4TnOk9XFq9GUayLTkzHw3YFoaGog0ZCI6378jniOzDqCc5Qzl9ddZsbRGeKwzzrYmlczX0XHRIf82HxBTvAvuVjPMZ49em52jSn7pnDto2tc//A6IYtDevRs7Bgc1RXVEb8tHg2pBg83P0RiK6FwaKHae/2m+QmDBgl47fCiYEgB9fbCuVa0Krj9xW10THVoqmxi0u5JKFoVZF3NwsDOgKNzj+Iz2QfHAY6kHE7BcYCjwBTr8rhGfx3N8K+GI9GQkHkpk6KHRfR+uTd24XasSF6BvrU+W/ttFWWX3Ya7kX83nwe/PiBkcYhY8Pa73Y9ehb1QzFMg1ZKK6+S9u9nUGtVSa1SLro8Buha6vJr5KtoG2px97SzVFtWcGnsKv2Q/Xhj4gnhcHeCroZ8PxbGfI5/KP8VtuBszT85E31pfbfBTnV3NzQ03RR+rymeVOEY6srZqrRrYKuVQCqdXnKatsQ234W4oWhX86PojgXMCmbR7Eumn07nzrzv4TvVFx0SHrFpzSl3MCXcV5Lrvb7wvsOksQaWhosq0imxlE9v3yPjgAwFNDrDu57dZzns0PijFs6A/9SWN3NgrMBl8p/liOcSSTY82UWhXKO6FciN5t0ZN17j8zmVSDqcIHmUIw59bX97COsiaxvJGGkobRJ+cY/OOCTJifw37xm8Zj5aeFl8VHuT3zO8xLzNH1liNnoUx/Vb3E3/j5NNTACw7tZQFAUs4tfQUhQ8KGbh+IA82PcIiL55aC3fKHYIxdhau7+xzs1G2K3lc+JjjT4/jbuYuNrxbG1rZPXw3QfOCMPQxxCHPgfgf4vG09eTpyac8PfmU2jwByNR/XX+QCHlNdXY1WVezcB7sLObMXUPZrgSJwLrfEr4FgOsDr7NsnSAHGLFKGPxc/1gAKFwYfoHEwEScHthQnV3KnqG7CJgVQOLeRN7kTbYt3IaKFkAY7oMgZdjVQ6epsgkkApCktqCW1GOpOA10Eq9fxKoIEV09/+p8MZ8z9zLHb7qfAPoY5krYijBufnoTTTTpK+9LlUkVyQeS8Zvux3rFelQKFfd/vY+sXsZUg6kUrSgifXU6biPcWJG8Am1DbR48ecA7371Dqncq1vVviedlS58tWPhasGnlUwrNHvH25xNp0b9FtZ8bSoWSjd5Cw3kgA8m3zydhToK4FlWkVYgy1+Yh5gy4OIDLTZex62PHnxP/BDolWc29zVmVtwptQ20OTDlAwb0C1tV0Po9d2YwPtzykKrOK218INcKM3Bm8MesNXgkX6pquwxqpQheFtBGjxmAUCtgasRWVUiXUKRcv0eQwCKVU66/fAGNXY0b9NErNQ0+pUKJoVSCVSbEKshJ8eXWEBS+zKpORv47kzzF/YmhviERDQlVmFY4DHfGfIXgramgKspWuw9QHLGkn07jx8Q1G/jCyWxPu+ci7nYdEQ8K7Te+yTvcd7ArsmHh0IpaNVzFxjWT+tfkYOxnzg8sPuAOJg1+m0i4AMzM4/+oOyp6UsTxxOU4DnDD3EbweO/autqY2NuhuIGhekOjxLNOTsTi2u5dXR8j0ZUR/HU1dYZ1Y1wKM+mmU4O3dx468O3kUxxcTOCcQt2g3nAc7iyCc1oZWvjD+goBZAYz5ZQzDXIfx0fWPMK12Qbe6gKbqFgruFXD94+ukn06nxb6duLA47CqXoCnxxdDBkDXla9DU1iTma8GjXiqTMv3QdPLu5LEtchvR30ZjE2JD1KdRlCSUsHv4bnq/3FtkYqefThfBh3kxQv0S/XV0N9lggKpnVZx79Rxaelq0NQgSs57jPJlxdAafaH5CXUGdWOvd+/kelRmVgvUAAuusuaqZKXunYBVo1W1Y0BVg+O/COsgaupCbix4VEfN1DKHLQnEa6KTWi7j01iXKUspYU7oGEOqQFckrMPcxp66wjvQz6ThEOmDhYyH2IHqKhD0JXHzzIuY+5tj3s4ck0K0pwrA0g5qcAGYcmUFJQgm7hu0iZEkIbmP9cVh7HIOKHDTWvoqOqQ4pR1PIvJhJwMgATt49Sb+iflQ+qxQHDs9HS20LUi0po34e9W+fDUDMITsktrWNtNHQ1CDu9zjifo/DZagLEasiULYrRZ/GfeP2kXcnDxD8cy2DrKkpa6XPilCGvNsPmxAbZhydQcH9Amx62dBS10LCngQM7Q3pv64/BrYGfKD6gNj8WD7c+iEowCrCCm1N4R7vAA0O+2gYtlW2BMuD1QDJJQklVKZXIpVJRWBDVWYVNz69ge80XxpKGgBhf+gYxNqE2BC8IBhtI22CFwWjbFPSe3lv2hrb0NTRREtXi6EbhrLRZyPlqeX4spkS53AOvWHIyJGR2IXbMexLQaVq1gezuKR9iawvsug1XwCdyvRkAsvYyYj7G+8T8UaEuoyhhqTbgDxseZgInOpauz+78Iy4TXEMWi8wHfWt9YlcG4nPFB+eHHzCqaWnGPr50B5Zm/pW+mI/yHOMp1q9858JqVTarbb9P40XTryAnqWeWg8KBK/5W5/fQqnrQoV958PZVcbzxic3SDmcwtxLc2kobSBgZuexzTg2QxygyU3kaMo1aaltYXv/7YS+FKrm0dYVJNFa16o2oP/F7xe8xnkx4+gMMi9lknI0hX5v9hPznV/8fkEikYgKAh3Al0trLxG/I563695Gpi9DQ1MDU3dTtHS1UKlUKFoUSDQk4hCvNLkUbYPuQ3hLf8sefUkfR6/h0y2CUkHZkzLcR7rT2qoOxtSoq+Fzw18IXxnO0M+Gkh+TT/necvSW6fHCny8QpxOHU7gTT089RUtXC5coF3HYN/HYRK4WXGX64ek0VzVj5GiEVCZFR0s4p7rVujSUNqBrpkvc5jjs+tgRujRU9Cg1djLupmx2fOFxiuOLBT/5LvYxwrAPHo16h2PHoK2+mbInZRjaG6JtqI1SWUVg9kasFxqw8n0tjs0/xuNdj3mn4R2UUuGGqDOso35oPUFRQSKjNedmDgWxBfR6sRe7hu7C3MucKfumYOxkjM9kn05AkUQA65i6myLVkhL9bbSaLcTfxagfR4l5jIaWhvj3BMwM4MisIyR9e5HEwS9z8Ep3MHxtSy3N8mZiImLwN/WnNr8WuYmcge8PxHFAZ56UdSmL+B3xDN0wFN8pvtiG2nYjr3TElj5bKLhXABJ4v+19NVDoP/Hfi/8ba95/Jf4Z9v0T/ytCpVLhM9OH+N/isU+9goaiHcXEIVgFCsnu3e/vMmH7BLEQNy1IJOG9FHr9OfrfGkhHfRqF9wRvSpNKyb2Vi/sod7ERqBYSAZmVeTGT+xvvM/fSXG5/cZuC+wUsub+EyvRKHAc4IjeWY+xs/LcL5N6xe2mqaFJr8ubH5JN8IJlJu7tLRIBA+7cOsWbguwO7vfZo2yPkxnKeXXiGhqYGXuO9cB3qiutQV1obWhnz6xhUKhU513IwcupMADrYXgtvLCThRA5HJ+zANMAO+wh7Yr6Nwb6fPR5jPPCe6E3ayTROLD7B9EPT8ZvuR1NlEw0lDVQ9EyRx9Cz0uh17RVoFPpN9ug3OAO7/cp/47fFMOzgNY2djwatFhRoT7/Wc1ymOLyZuUxzek7zRs9Bj8h+TOWd+DpehLnhP8FYbUGgbaKNtoE11TjW3vrglIv06UG0d0VTZxMW1F3GLdlMb2HQg8eZenIuBrQGNFUJDzdjZGC0dLYZ/PZyWmhZqcmsY8umQHg3aPUZ7MGnPJI7OOUrhg0JCl4ZyYbXQ/H75ycukn07n8c7HDHp/kJpEV0cEzAn42+SgQyIk+WByN8+UjjDzNCNgdgAmbuoNM11zXfxf8MdpkJOAQOoSepZ6FD0sojSplPzYfJ6efCp4RBpoi/ewokVB1qUsavJqug37XIe70re0LzIDmZq8aFVWFebe5rgMceHhlodkXsoUdMO7Hq+HGesV64nfEU9tQXc248MtD4n7XfDe7EmCr6mtCeMqY5rONFFjVCMmuMXxxaQcTeHxzsc0VzWrDfv+XSxPWk57cztfGn8JoKbV3xHNNc201Lagb6Xfo4/hZ7qfiezcMb+OURv2qVQq/hj1B06DnBjw9oBunw2cE4hdHzvK08pJPZdKv1f6dVtLJFJBRqQnBHHK4RROLjnJ7HOzcR/hTsb5DG58fAPH/o5MOzQNLR0t6orq0LfWRyKREP5quODRsDSU2oJatHS0ejzXNz65QeLeRMZtHofPZB81BGFrQyuPdz6m98u9mbx3cjdk+P9JlD0pY2vfrYLkn5G22NjrCEWbgtq8WraEb2HFkxVY+FgwcedEEvcm8pPHT+Ia1ffNvmrf6z7SHXb683HGdPRafFEWtVFVpcX77e+jbFNy7cNrmGwwYWCfgex7YR+BhS8R9pIVYS+F0VzdzOZBm1n9zWo2viusG3qWev/hYMh9pLvwmz2ESqkicW8ixi7GaoP47OvZ1BXUETArgNb6VjIvZpJ9LRvL8Zakm6Xj5uKGkYMRDWUNlCaWYuFnwflV53n74Gpu9b/FwGs7aJ0xR2y2KFoVpK2z4Ahv0CJvIb+iNzKzzv3Rd5ov+TH55DnkEVUaRUOx0NBIOZrCoRmHMPpYeL6kzVL2T9iPa7QrXuO9RGmifjH9KLArINk/GRDYxgZ2Bj0Oh9ub2zk2/xjBC4LVGkaPdz1GQ1OD/uv6c2z+MYoeFrE8cTmVNu7UyC+S5gsyPUfaGttIPZaKkbNRj355XaMmt4a8mDwGvDeAOefnqA3wNbU1xYLbfYS7mpygSqXic4PP8RrnxZR9U0g+mEzcb3GM+nmUGvMn4vUIQhaHdJNMST+TTsKeBAZ/NBgzDzP0rfRZV7OO2vxaHm5+iFazFjqNOgRVB3Fzw00iXo/A0t+S9DPpZF/L5uEbD0lsT+SlTWk0Ge5FS3cmK9NXUldUR871HKwCrTD3MufdxnepK6zj4eaHpBxJoTKjkpKEErR0tVhyf4naMUkkEhHJn3o8lfs/3ydoXhD61gKKVtGmoDy1XPCa+ysOTjtIbX4tJq4m6FrosvT3pdgW2dKu3Y5KoQItQUKtPK2c1qomTCpNqNNvwEbmgIZUkBuqzKjk3o/3MMOMeoN6YiNiGf9BJ1red4ov65XrqaurQ6VSETAzQPSyeT4ayxt5uPkhJm4muI9w58mhJ1xed5kl95eoyS5rG2pj4WNB1IYoHCMdUbYr6bemnwgu6vNqH3ynCR4vN2/CV18Jn9PRgZQtj6jNq0UildBc3YxTtpPAKlWpaGwQJhvek7ypya3h9n0tHK4IjHAMoL60jKvvXWXg+oEM+WgILe0tbM7ajFQiFZtCXYcUHet7ytEUtHS1cB/hTtC8IOz62IlrqbJdyfUPrxO8IBjX4a64j3Tnj1F/kHMjB0mQhJBvO5lG+jb6XFt/DX3ddmReMpZuWkqT8WlAHb3rk/c1OSbvExI/lDdmFmHxl/xnxGsRxHwbS5lDL6qtvamy9cPwrzq17EkZWyO2it8x+8PZzPCbgVRD8Jwe89sYzDzMUMWqyHHKYeDOgdj3sefKu1fIOJeBnqUeDaUNfKr9qcjqfHLoCScWnWDK/inis6hoU5B+Oh0jRyOOLThGe3M7yxOWEzArgA21G4gLi6OvrC8zmMHhmYdJPZoKQLWrgjuRd8Tj07HUZ/TG0aSfSQegjjqa7jWyQ/IzsqTDGGu244I/qz9+U6358XvI7+hZ6LHk/hIqnlZw9pWzjPp5lFhjdM0JOu45lUqFxzgPNcnyoZ8NxSbEhnyNfBbEL2Dh3oXcyryF33Q/4VnUlGDubY7uaF3y2/LRidGhrrAODamGmAsaSAyIjYgFwLqhs0upZ6lHzo0cIqssODgF7g7WwrJuCHogno8hnw6hpk8NG65swFPV2ZTVs9IjakMU983uc7LoJCs2riC1LBX3UZ17VFtTG1o6Wki1pCLjxnuyNzZh6uC5U8tO8XDzQ96qfItHWx9REFsgvpYlz2Ll2ZXisK9rqCRKjKqN8HpwmyrXFnym+gh1iUqFMD1UIWuswinxDEUXAun/cYDI1u+Ix7sec2KRgK4PeTGEkBdDuJXbaRFgsceC71d8z9qqtWgbavPk8BPifovDKsCqm/x612hvbqehrEFgWH56A2NnYzXv24trL1L5tJIZR2cw48gMcfB7bronpUYZLPg9lBaDPGT6MpEN7jU9iJRDySi0OnOnsOVhVBZUomelJ+Y3SoWS1vpWpDJBethnik+3c94Rj7Y94tYXt5h2YFo3YNqUvVPQNtQWcujyNeia6TLgHSHnvLDmAjFfCxKP5147R/qZdN5vex8Qhga2YbZiHayBkG89Dn6KmeZZXusNe8K+FyXG7fOF992O+JPAnE1ItaSi1NoHqg9oV7aL4CDLAEsW3VmEsZMxZ189y4NfHzDt4DQK7hfgU+LD6I2j2XJ/Cwm7Exjx3Qh0zXTRMdXBa7wXGloafCT5iL6r+xL9dacfolWgFQuuL6A0uZSsS1n4TPUR1/jJf0xWq+fH/j5WTTmnIrWChjIhz+nJX7trVD6r5PDMw4QuDSVkcQhNlU3E/hiLQ6QDbsO7A9HqCutI3JuI63BXUe2jIr2Cnz1/FuQafxCk3mtya7i07hJ+0/2w8LWgLKWMk0tOMubXMT0yjE+vOE1lRiVzL8zl6QkBtFaeWk7UZ1E4/XEC7cYqDCuyqcm2Rx7hgsxARlFcEfXj6mlrA4WWDm3a+qhUKoofl/Dk4BOS9iWhPUgb00pTTH83JVGSyKD3B5FzI4fm6ma1Nc0t2q0bMPM/E7pmutBDf3no50Ox623Hl6ZfikzvjkHfojuLuP/zfe6n6qOsz+RxkpQhf32X90RvjJyM2BSyiSGfDmF54nJxLy1LKaOtsY0Hbz5g0bNFbFu0jYr6Cqz1rdWYbwCFdoXEvBvD11O/5s43d0g5nMLEnRNZ8WQFRo5GxO+I5/7P9+n9cm/it8dj7mNO2LIwHAc4YmBjwEafjehZ6bHk3hJ8Jgvg0QlbJ4jX6sGvD3iz9E1R7rWjfm/WNcGsIIEmA0sgEusga3Jv5ooe2EbBRjS4Nog9rPbmdpqrm/Ge4I1NiE23e6O1oVUcbnSAzv5Oproqq4qUwymEvRSGsl0peI2b6RC/PZ5bn99CQ0tDlDa8/O5lzDzNRF9DQFzvOnL72oJaNoVsInRZ6H/oiw3Cc1ZXV4eBgcH/SK0o0ZDgOdYTlULoaXWtN7R0tdCz0qPU1p07KUeovvYmbW21TBhqyNT5s1i8ZAmhy0Jxi3bjzld3yIvJI2BmANnXskk9nopduB0JuxMIfyWcGUc6WfHR30arAQOe/zueZ8QOfH+gyH4velTEg18eEDw/uLP2GKVeG+6J3kNNXg0jvx+Jvo2+eM86D3bmxTuColR1djU/uPygth5uDtuMy1AXZp2aRcKeBOJ+j2P81vHdmPfPR9KfSYK/X8rLNCr+YuBV5mGXdgVpaDAOfR0wdjKmNKkUTbkmrvtcKU8q5/jU4yxYsgAQ8gAjJyNevPOiOOw7PeY0Ly19idgfY7nx8Q1WJK/AwtcCCRKadJs4/PFhZpvM5kuTL8Vjif4mutvxdY3AuYGies7cC3PFfy9uTediUD+CU1+jKusVSh4VcGj6Iabsn0LZkzLCz9zg3IhzpLAamErQ/CDs+9ojlUnRl+rzavir/HjvRyRI0JRritc39odYbn1+i4trLmLqYYq2kXB+XKJc1AaREolErT/WIZPdEYpWBb8G/ornOE/RZ70jTi45Sfz2eNbVrkPbQJuMcxnk383HbXIgj69UoNCSi5K/f4z6Ax1THSb/MRldLV3aZG3U69eTtC+J0KWhOA927vYMRn8bTdSGKLR0tZDpy7rVqk8OPREByR2gn+eB8P/Efz/+p9e8/2r8M+z7J/5XhFKpxHqmNTNGzWDXkF1oN1SJNPy2pjYayxuFptNfoVNXSvnjVFpqh/Jwy0Mq0irUhlH7J+6nMr2SFckrkBvJcR3miuswVyJej0CpUFKdXY1MX4ZUW8qdf93BzNOMS29dImhBkJDk9XfE0s8Sl2EumLibkLg3kesfCqjiNeVrRNRgRzRVNnH53cu4DnMlYHZAt2HN5D8mM3nPZJAIw0u5iVwNLdXVGB6EJnBleiVB84OYdmAa7WgS81Ab7yh1VJNMT0ZhXCE12TWCBNh7V3G57cKtL25x+e3LmHmZ8UrqK/x21pH80e/RXiuhV00d1z+6TuDcQHyn+qJvrU9bYxs1uTUo2hTYhNiIw5PKjEqCFwX3KFs4/9p8EVGWczOHorgiei3qhbahNq0NrTSUNojv/b3X72qopo7IOJ/B5XWXsQmxQc9CD01tTUZ8N4K042loaGlwdO5R6grrmH9lvshGaqxoJO63OFRKFX1X9xUNzDuipa6FR1seoVKoyLqShUuUC37T/HCJciEvJo/U46nsG7ePwLmBPNz8kMX3FmPsbIyiRYFVkKD1r2Oiw9K4pTwfRo5GBMwKIGBmgJhwRn8TDSpBhvXxzscsfbgUDU0NWutbuw38xm8eLzBN9yd1Q9Td/uo219ZfU5NK7RqX1l1CQ1MDQ3tDNW8DEJDZPcmxgMCe+2PUH4CAVi95XCJeNy1dLcZtEZhp5t7mPQ62bHrZ9MicPPvKWbR0tVgWv4xH2x5Rk1PTbdjXEX/XrA9dGorrMFdkBt0Ho8p2JSXXS+iV04u8H/Io9CgUh31Fj4q48fENph+ZjudYddRhW1Mbx+YfwyXKhbCXwvjW/lvs+9iLz5hEIuHUslNY+ltSmlRKzvWcbsO+2B9jubb+GlJtKfOvzO+GbvUa74WZlxmRb0V2k5JVKVXk3c5DKpNScL8AUzdTtcGZubc55t7mHJp5iOT9yYQvC0cmFf7+koQStvbdyqAPB7E8cXm3c1KZUUlDaQN9V/ftROWqwMLPgoHvD0TfSp9jC47xeKeAZtPS1erU74f/UGI4aH4QmZczObnkJDahNmrH3FzVzJmXzxCxKkINdf8/EUWPimipFVi0BbEFAmuQzvtw8AeDcejnQNaVLPGY7CPsqcyo5Ompp5h5mhHxRoRY9BbcL+DsK2eJXBeJVE+CUqok/I4Vus07aGtbgoZUAw2phvgM6DTpkOadhkd9Z2GnbaSNVEdKu2Y7egV6fBHwBePWj8M+wp5j848RODdQlOjd2m8rjeWNanLE4nmraSblSArm3uYcnXsU/5n+asO+2O9jSTuZRsCsAAofFPJ4p+CXVbO1BqYKQ3hFm4Ksy1kcnnmYyX9MJmlfElpoIVVZU+xgip5mZ1IvlUnRtNakpVo4n0pJCyqV0PQ5segEtuG2HNh4gCdlT/j5y58Z4iIUDK11rSjblLQmtIIXnE87T6/TvdC30VdDwQJMOzSNMafHkDUwi8DZgd18WJ+eekp5ajkRr0fwwvEXMHRQB00svLFQHGLLDGWUJpWyd8xetFx0aDFuIbGPOZ41vhg5CFKrKqWK0ytOE/d7HH1+nc+DhxosfdeCoutPuffTPQLnBSIzkLG6aHWPXgvtze3UlwgebF3l5QAuvnWRtgZh71MpVTSUNlD0qEhkD4DAHtYx01F7jkBI7stSykjal0RTZRNNFU0svLUQbUNtjJyMMPMyoyKtgpc3vow8Ss6VTVdE/7GnJ59S8bSCEysEdny2Uz4GykgkGhJROvzJoSfsG7+PlekrkWoJ6PPob6N5tPUR0w9PJ35HfI/IzYr0CuqL67GPsKff6n4EzQtCx1RHaCTVNCM3kvN6zuvI9GSkHk8lfns87qPceXryKY4DHFEpVdhjz9ORT4n4KEJc375z+A4AbX99Xkt6jXq9JnZu/ZXV+auoza9Vk6bWbdClUa/Tn1KlVPHn5D8xsDegrq2O4WuHq8k+N1Y0knMjB+sga0xcTbAOtuatyrdETx+P0R48+OUBm3tvxq6PHXoWekw/PB23aDdRqQAEb6QOPzEQZJE7hhhbtoBBeRYOyedIchiKuZsJdUV1GDkYUXa5jIU7FlKnX49BvT4Nc0sAa2YcmUH29Wy2D/0YmX8+qaF+uJa+jszKhGWPlqFSqvhI8hG20bZ4r/bm8WuPsX7HmqC5QaSfSWf/+P1M3DlRlGU7u/IshvaCBKPXeC+86GywaulqsTxxOQ2lDTza9ghFi0L0H9vYbyMeFh70Q9hfix4WkXEuAwNgscVibg64iUvlDMpTyzk86zDhr4TTa1EvdFqcMa42ZuilEPJ8Mhnx3QjqS+rJvp6NtpkezXoWqDQ6JO2E4+gYmgHkOOZgVG3EpfWX8Bvnh30fe3GdrbtZB0bwwO0BQ2yHMGH7BMZvG097czs7Bu1QQ707RDow6/QstWFFe3M7f076k+CFwbiPckfRqkBTrsnkPyYz5aMpaDdro6wTcjvbMFuBtfWsiub8YvTq9Xjpt5eos7mCtnEUvVf0piShhBjucIWL9EuBJUoV1hRSDOyS32Hi++P4QvsrVr0hyCkHzg0U1wubXjbMvzpfzbena7Q1tiGRShj0xyDaFG3cXXxXHG7IjeWEvBiCslBJVW4V12df543Zb9DW2EZeTB6mbqa4DXcjty2XLfe38O6pdwmJElQ3GssbkUgkVNyrwLDGELdnboQkWaJsVwq5XpgnZU/K8Es0QamawuGp7zAkYQYuCPKqIKDD4yXxaJ3XInxrOAmGCQTODsTAxoABbw8gPjYeimDzks3Uv1OPTE9G9DfRZF/NFp9tRZuCmpwa5MbyHmWobMNsaakR2D5T/5xKY3kjWZezuLT2EpMPT6ZFu4VrXENDqsGTQ09YcH0BciM5Rm1e6DY0YpX7jKZsB4Yc7GxQSUOCsLz/gHwfE/Qrc2gq7jmvMHUzJXBOoJqsaIckF0C+Zz5zIuYglUlpb27n6ntXkUglGNobCkodtgYUPSoi8Y9Ehn3R6f3uN81PBAPuHbsXlyEuasO+irQKih8Vi/9flVVF6rFU3HPG4aC/gMTh4OzcyXqUaEgY+vNEPtMrQNr0FmOOuqJs6I/Pv8JITEykraWNsyvOEr8jnsWxi9nSZ4vYxJ1+SL3+U7YrufP1HSx8LdDQ1EBLR0utwd3a0Mqh6YdwjXbFZ4oPpu6mnH7pNDahNiJjPmxZGB6jPdC31qc2v1aUla1MryRseZiah157QRADkxLRUAn7Y1sbDP5wsCCLnF3NhX9dxyvVizTvq1Tr3aO92ZmKpxXoW+ujY6GDzmc6tCvbOdd0DmNbY/q8KngBW/oL93LOjRw15vbgjwbz7MIzcX039zJnxpEZNJQ1kH0lW7Re6Ai5sRyngU7UFQk+bv4z/Uk7nsaltZcwcjTi9ZzXxfda+ql/tquc56bQTagUKjVllcOzDmMTakO/1f1QtilpKGkQWejN1c1c/+g6kWsjcRvuxoGpB8g4l8E79YJUoPtId95tfheplpSylDL2TxD6DiD4InXkTo3ljSTtS8IywBLvid5YB1sz+9xscdifeiwVXQtdMTdsqmgSmWX2fe3JuZGDpZ8lihYFFnmPKLcP4smApczyN6UqswqZvoy1VWu58v4VNrl/RUnIMtp8o2lvUfB78O94jvXkzdI3+drya0Y4j6DwpUKxbrr+0XWKHhWxtlJ9uJcfm0/8jnjCXgrrsR7tGtnXs9k5ZCd9Xu1Dn1f7YORoRNmTMjzGeDBx50RxMBy6NBQDO+E5nnNhDjJ9GQ59HXDo68D2cYAtZLSC9+vneHLoCSvTV6JnoUef1/pgF26nJuX/i+8v4n/bSm3Rr9dns8lmwl8NJ+rTKBJ2J6jJgte3CpKMzVXNVGdXo2OqIx5X3u08kvYlMW7LONbVrhPyaG1NsSkeMDugW42qUqpobWjFJsSGoPlBSGVSqrKqOLHoBEqFEu81Y9mTEoq0tQmVRpfh41+PsdRPyonxJ5hQP4H25nY05Zo8u/CM/RP2A4LKyZxzczj+4nGqM6uZf3U+FWkVbAnfwtDPh4rPedfYP3E/+tb6jP1tLGHLwsS9ur6kno0+G/Gd5otNqA1DPhlC+MpwkSUX83UM7iPdxWFf6rFU/pz0J5P2TMLA1oDsa9n4TvXFzMusR4+w50OpVJKZmUlAQMD/mLRd/I54Tiw6wcxTM9V6Z0HzgrhSdonda4IYINHifWUz1kDxs0J2bdjAJx99xBdfCfu+ZYAlzVVCXlIYV0js97H0fqU3j3c+xmuClwjq0dLRou+qvtQV1XFp3SV0LXSFPbC2BfcR7j32STqUuADCXw4neH4wcpPOft2Ib0aovd9znCdNlU34TvUVWW7Ph8xARuCcQDVlrAHvDRAHiE1VTVRmVNLW1CawvPcnifLJHWFS+IScm3r4z/DHKsBKuJ5/id5I21vQqSulVSplzpk54nm+/tF1nH51ol2rHb3Rejj1E/p/Y34bI7Jje9v25uTTkwyYN0BkzUeui0TXXJfKpkrmHRPWXaVKqcYgtgm1IfdmLhffvMiwr4ZRX1RPwu4EBn0wSBxY/l2/6GDZR7RqlTP6sIKNh39g1ulZDFw/EEs/S6wCrDj93QUy3DPwzWzk6amneI71VBvWmeuaY1phisEaAx4VP8JrghdaulqELQ+jqaqJmpwaJu6c2M2H/j8dfxFGngdvX3jzArk3cwmYFSC+lrA7gcS9iRi5mfE0YilWWXc5t6qR0T+MFJnCAGG2YRxPO87wS8PRNtLuUYkKUFsbFW0KlG1KNOWaYt5w5+s71ObXErwgmLkX5/b4Hf/Efz/+b6x5/5X4Z9j3T/yvCvu+9iQMfR2USgJ+/pKzrYGM+mFUt0ZbgVcU034bipmnkKh1oMQ6wsjRSACOIshq1hXVUfxIkOwzcjTiB5cfCH0plIHvDeTGJzcIXhCMhZ8FBjYGYrMNEKUUi+OLebzz8d8yrlpqW4j7LU4opL/ujlrpKsF27rVzWAVadZNGSDuZRs6NHAa+O5D4bfE83vUY/5n+lCSWcOzN22SEz+LoTX0OHxY8XJ6efMqon0bRWivICRQ+6JQL6/Dmcot249pH16i7b4XKxof4BEFWaXnSchL/SGRTyCYm7ZlE/7X9e2QVdpyLZxeekfRnEgPeGSBu3Fv7bqW9qZ0Xjr9Azs0cYr6OwXuiN9qG2kSuiVQzRu/QAe8IlVLF492PMbA1YO7FuWKT5eSyk5TEl1Bwr4CqrCram9vFIe+hFw7RXN3Mq89eFeQnZvr3aD5r5GDE2qq1NJQ28LPXz2jKNfGb5icWfY93PaY6qxrXYa7o2+hjYGNA5qVMjsw6wvTD0wldFkr21Wwe/Pag21D3+WsJiEy68rRyXIe5Yupmyq5hu6jMqBS1/rvGnX/dIed6Trdhn3WwNcELg9WGLF0jfnu8OEDNOJvRI4uoPK2cvWP2ikUXwKEZhwBwHODIiG9HMOJb9aQz5MUQvrb+GjMPMxbeXNjjb/cUAbMDuP7RdT7W+BiHSAfmX+suRdNa30p+bL6gh96DfJdNiA02ITbE74xHbixX81Fqa2xj3u55ZLhlcHXxVd7o2+l55jPJB7twO0Ey4jkJTUWrgicHn4iFmm2oLaaenclmfmw+yX8mEzgnkIU3FyLV7r4x2/exx9DekPrielHyr2tM3T+12791hIZUg7fr3iZpfxJbwrcw7dA0fKd0T+b7vdUPw0GGaomhlp6W4J/1N7r/ifsSubb+Gi89fklco7oyyuJ3xgtsvpXhIrsHhKZF5qVMkAgDJNsw225IQLdoNxbeXEhFWkU37Xpdc13mXZlHW0Mb2deycRzg+D8m/eAxyoP51+Zz6a1L+Ezx6dGH0224WzdkdeCcQLXmXEe0N7dTV1RHXWEdimvFTEmdQkBSAOVWNSiVsG/8PvSt9Rm3aRzL7JeRXZ0tfrbgXgFFD4vwf8Gf8vvlZHsKr9UX11NfVM/3jt8DqEmC2UfY01LbQtrJNJ4cfMKQj4eISPOGkgZOLDpB5NpIZhydoWa0DkJxFLwwGABjZ2OGfDqE8pRyEv9IZFnFMmyKbbiccpmguUGM2zwOl6EuLHmwhGkffMW94MPkeXmzKtiauz/cxdjJGI8xHhifqmV443CKbIrwu3WQ2ozJqMLMid8RT1tjG8qBQqO0o1kNnbKuTeebwAuadZq5se0GH8wTmIyri1ajVCj5yvkrNNs1KbEq6ZG5DIIM9JNDT+j7Rl81tHhHdG0WjP5pNG2NbSTuSWRRuw+Pg8aTGSyhozfSwVJ78KsgmXj0q6fYPLvDbs25DPRvpa6wjicHnpBzI4eAmQE9DvvyY/PZOXgno34ahfckb55deIZjpCNmnmaClB+I+UP4y+Gi12lH3P3+LnoWeqKvnaZck4dbHnJyyUlWZqykz6t9uLjmIqlHU2lvbqcsuQw9Kz2mH5rOrwG/8sT3CT6Tg3jtvRVoG2lz9YOrFD4oZGncUtYcFCS+jk++QFTibygVSuoK60QWu665Lu3N7WRezMTYxZi+q/qKaNKoT6MojCvk9r9uC839v5rhMd/GEPebwJY2djYW78W0E2nsn6A+fKrOqib9dDoLbixQY0e/X/h+t/PoPsqdjLMZGIy15LLROXSbbegbOIC6ojp+cPmBPq/3Yb1iPR9LP2b0mdE4DHDk6NyjTNw1keaaZtKOp4nfFf5COGaunetPeUo5ByYfYMT3I4h4LQINTQ214tUqwEpsthXECv6xz4MCeoqWuhY0pBpo6WrR0gIaqJAoFSjaVcw4OkOUjrb0tmTkjyPZ9lMTbfVlSKXC3lFXWMfOwTvRAAIf22NcU4p5xW8o17+EdbA151adA6DwQiHnis4xQDJARHUa2hkSOCdQbS0dt3lcj/coCMMCS39LytOE57+tsY1Veauw+caG8ppySmtKAQHMcvql0+LnLMssuTXgDkZxgSgVpTRXN4t7lgQNyizKODTjHh7lrxPxuhFZV7PYFbWLgFcHc++RM9K2ZuyfXEDaZA70I2mvIAtodcOKD698iGOOI3e/v4uRuZEaG7QjrmRfYU2kcB9LJBJaalsovF9I75d7c2DKAfSs9RizcUw3+VtNuSaT9kzC1M1UTe60Jq+G/jf7M+zyMJT2SpiIyFaK+S6G4jeKef371ym3aEahpSOCAtM90niofZ2bLSr6qOO/mN3cTCww7h2hSb/qjTeI+iRKfF1uLO/GAju24BjNVc28cPwFPjf4HPdx7tzpdQfXZ64cuXeEoe8MFZi2h1NY8mAJ+Tn56NfpE2cSh11vO8pSytg9bDcD3htA1CdRFDcIg6OGViGHqy+q5zuH73Ae4kz21Wze4A0qTWswrSyiqUGJnpEGB8oG42xXh1lBAqaVwn6vkggglahPoxi0fhC1zbUs+WkJ8mY5jnGOgnRjFwBGh0dgm6wNTV1hb+37Rl/6vtGJSq/Nq+Unj5+IXBvJsC+GdbvGoUtDRYlCmb5MLe/WUGrgk+JDljwLp8FOAlDir2ti1hrCM5sdHFygzVsDn2tO19Ujry+jycCcR6PeYchMwSvsxOITDFo/CNVgFV/c+oIPB38oAjnjd8ZTEFvAiO9G8GDJA8I2h5EekM6IN4ScVtGqIPrbaC68cYFj84/RXN3MgPcGINOT8eTgEyLXRmJC9zx02cNlahJ/gOiXCYLn05kVZ8i7k0cAF0kesJQmI6GhmnM9h51DdjJ642icpvYmyekVTCtMqTd8Bx15Z7M37tc4EaChZ6lHrxd7/a0Hlkql4vLbl/Gf6c+UvVPUPNxAqJ+yrmRR8bSC/u/0J/bHWGrzasm/m0/aiTTCV4YTMDNAzBHt+9lT9LCI1KOpJP6RiP9MfzU5vC0PdhLj9T4euTOwz1lJfbG92HCtzq7mxic38E4fhFGNEcFJBZQklrAlfAuDPxrM7a9v817de3z67qfc23QPez97QpeGoqGlQejSUC6tvaRmO9BS14JdH8HD+/lz3gHi6Ol8oBKG8qN+HoWZpxmX3+703u4a7c3tarJ3XcOhn4NaHQqQ/GeysF6uFoB4XQeHhvaGLE9aLiphWAdbq4GOO4BjHaGlo4X7SHccBziKaxaAdS9rkVUJAmvNfURn/XZ41mFchrgw6/QsADX7iWFfDGPYF8OI+TaG0uRSHo1Yi1JDikqqhVQXDs/aTV1BHavyVmFga4CZrxWqv1g3ShUM+WQIFn4W6FnokROSw1Pzp8RYx/B7r98BgZH0vJ1GQ1kDt7+8TerRVDxGefzbYd+FNy6ASmDIxP4Qy6r8VVQ8rSDzUibKdiWPtj0i9Vgq47eMF/M/t+FutNS1UJFe8dfeIOyLEgnk3sqlrqCODbobAKFWchrixtOYcszMJJh5mtFvTT/u/EtgeJ8ZfYY2rTYcJjpg08uGxvJGzrx8hn5r+oEeGFUb4f7InaLwIqI+jSLq0ygUrQpSj6eSdiKNyLcimbhjoprEXltjG6dXnMbS35KhG4YCAjsq/XQ60V9Hc2DKAXJv5dL/nf5M3DEREBicJYklBM0LosFYD63mOtrknfveo22PeLT1Ed4Tvfmu+TvMKswI/jiYG5k3iPokCmNnY3q/0pu042kiSE7RrBD7Tga2Bgz7cpga+PrUS6doLGtk+uHp1OTWiGtv19DS0WLAewOozqzm8rrLTD88Xe35fyXtFbU62MDOAN9pvhg5GJF5KZMbn9zA/wV/Aaj3/0FUZQpDVKCbktd333zD5+++yy2Vgj4qhdprPe37HRG2LAz/Gf7sjNqJsYsxnmM9ebT9Eag6Wf3V2dWipY2xszHV2dWiClHq8VQUrYoe7T60dLW6rW3PR9f1oWtkXc0iPyafkCUhPSpqde3T9VnZhz4rhT5P7u1cLqy+wIjvRqgN+1weHSbmaw9eOP6CaAnS4Tlda+nO4+g1eHQRg/IY48GMYzO4W3sXDYUGUknnfeE1rrOe05cJ18HHXGCHdRAqACpqKsiozEDaLsX1niuECwpIpcmlFMQWcHHNRVFRoyyljMS9ifR+uXeP7MRH2x+hbaCN71RfZBrqvTGZgYwhH3UCiLYtu0GVfjmcuM6RM4WsK1H3hJZJZUgVUjQLNcm+ls2JF08QtSGKAW8PULN+6YicGzkkH0imz6t9xGP7vdfv2IbbMvyr4ewYtAPfqb6ihZFUS9ojSDthTwJmHmZq3pie4z1J3JtIzbMKJO5KjErTSTlYzegfRrLg+gLxfc7Gzoyw6sxv5MZyqrOrOb7oOEHzg8QBfUNpA01VTZi6mXLvp3tcWH2BRXcWicSA0RtHdyNG/BP//4l/hn3/xP+auP/WfWoH1dKqMxCN9lYUFtZ/z0SRSERN7tlnZ3dL4DuMvUFgRHUwJgAGfTCIvm/2xT7CHj1LPVY8WYHcWE5tvlAsNZQ1dEN2WAdb81qmIMuZeyuX7OvZhC4JRc9Sj8aKRooeFvFizIs9+qKBII3UUNaA00Anph2cJg4iQChkzq86T8LuBJoqm+jzah/6remH/0x/NOWatEtktMkNkLY10VbfCOhS8pc0x5CPh1D8uBirACsqMyrxmiBsxrNOz0KlVNHe0s4G3Q2YWXnTpq1Pq44RUi1DLP0s8Znsw63Pb3F0zlHs+9iLhWGHCbXXBC/6ru6L0wAnSpNLid8WT8iLIZi6mVKTJ6CA65vqqXxWSe8VvfGe6P23kqrPNxCU7UqOLzhOwOwAgfH4Vzzc9FD87yvvXGF18WpROz7spTChkJNIekS1dYREQ4LcWI7MQMabpW+K6NGOCJoXJBbPfgiJWgdz7cCUA7zf9j55t/I49/q5bsO+2J9iubD6AkM+HoLXeC81Sc6+q/oSOCeQ5upmPMZ4dPPVA6GBpGOiw5T93Vl4XuO88BrnRc7NHJoqmroheFYkryDzUiaHZx4WJTY6ojS5lNgfYnEc4CjIc3aRP41cF0nG2QyxIOkaLbUtXHzrouBpMlyQhX2+GRn7YyznXjuHVFvKpF2T8JsunDPvSd5c/0hgu/pM8cHMo3uiVp1dze5huwFBnuLvvCkurrmIhY+F2rBPU67JybEnqTCrQOIvUWsWyo3lyI3lVKRXUJJQgkM/B7FIkxvJBR+kv07RC8dfUPutfeP24djfkQnbJ3STe+kIt2g3VuWt6vE18W/LqaahtAGbXjY9fo9VkBXDvhzWbU24/M5l7v10j2UJy7CMsFRDbJu6mTLr9CwBdfhTLH7T/cT7H4R7xNDOsNvQKPNSJlfXXyU/Jh/PsZ5qrJk/J/9JdVY1xfGdKPV+a/qpMWA6wszDrMfrqCnXxGWIC8cXHlfzF/ifCB1THcy9zClPK6e+uL6bBEL2tWwqnlYQvDBYHOx+Y/MNph6mYvF564tbtNS1MPSzoTgNcGJV7iqBofJKGQEEsH3Bdqwbv2CxUmjgix4ASJC1yFj9zWpqbE6TelzOrQ23cB7iTPj34fzy5BdKrEvI/j0br1Avzr0mNPe7Fmodw/ObG26SsDuBiFUR4oDFwM6AWadnYeJmgrlXdz3+rs1zY2dj/Gf4s6WP4FlVallKqWUpk0Mmq3nN6FvpU+iopF2rHRVCU//SW5dwi3bDKsgKs0sNRBLJqTGnkNfXis2vt+veRqIh4b1tgrRe5d1K4q/FE7wgGL/pfjSUNpAZnAllwvEolJ2Fc2NFI7/6/4r2Am3WO69nsPNg7HrbceuLW+TezGXmyZnifRz1WRR9V/ftUYIWhOdGpVSJw/8JWycwbtM4XnJ+m0bdRoJvJaJr+ICGMh9KEkqw9Ldk0IeDMPMw45PfLWnX1se01ZSwl1wJeymMqswqMi9nknMzB0WLguOLjjP5j8liEW5ob0jf1X2xDramNKmUE4tOMHbTWMw8zZiwdQJB84Koyqz6W5/KRbcXcff7u/zs+TOLYxdjF25HSWIJIDAipVpSRn4/kpHfj6Qmt4ZNoZuIeCOCEd+MoOhkEWfiznAzJZ/Whxv4MVxoaNqG2VKTW8P7H7/P5aGXieudKZzn8ka+d/weh0gHUAm+Z00VTewdsxeXKBeR3fri3RdpqWkh52YOl966hEM/B3HY1yEVJtOXoWhToGhRoKWrhbGzMWErwtT8PCNejyDi9QhaG1qpLahF10xXHLaXJJRQnVON+0h3pFpS0eA9JrmAd5TfgqqCz17+Am0Dbfq+KeQJ+XfzaTJqJd0jHftb4SQkJTBhxwS0DbUJWRLCw83C/m4TYsPZV89i5GREv9X9MPMUCuGOJkV7czvlaeXoWeqJ6/6LMS9Sk1uDvpW+2HzNj80n41wGwQuCxQHE3jF7se9rz8D3BrJ/wn6K44tF1kKduSvJQ15hqC8U3s8hLyaPkMUhGNoZ0mdlHz4VFLnx+GsesK3/NgByRio54P8Ng68NRrvNhvZWJU1VLVQ+FVgcSomSXMdcil4rwneor/g3Pt+o6SqxfXLZSaozq9VQto0VjSKzroNVVFxfzNItSyn7rQxKQaVQ4TzYGbsIOwq1dHirbAoq/hpa+lmKeSpAgekeWrVbSfG8jnODEaeWnyJxTyJjfh3Ds7gqfG9uIj3sBUwLk5DInIB+RK6NpL2lnce6Qs5cZFPEtHvTcHFzobG8ka39thI4Vxgm+TzxIexfYaQZpmHsbExTRRP2EfZM3jsZM08zji883iNYBoTGyPOs4PaWdnYP382wtGE06jQiHdTZaGqqahJBbAqpgqMzUgjP6Et5ShmbF23hg2fvcVOlpE+PvwZ9gJMtLQx86y1mzpqFtXVnftVRP3Tde5oqmmisENipgXMDseplBdVgXWxN8sVkIhZHoKGlQXN1Mz97/Ux9UT0v6b3EN2u+AQRQw+hfRov39KEnh9Bp1CHjdAZVnlXITeT0frk3lv6WVFZWUvu4lgsj7uFVeYmzF6VMmqDA4/4+6kwd2bLiCfkWB3jlp1fQkF6mfqwHUi0pf076k5zbOcytnsvuObvZ+91e1r8qeEAenXeUhN0J2G0SBkrjZOMoSyxDpVKRtD+JwNmBYm4iN5HT/+3+OA104vK7l6nNq2XSrp4tBxrKGpBoSHh24RkAtyNvc3vubfLfzEdDqqE2REWpiUpDRbuWAqVEyvnV59HU1qT3y72RWJiRFjEf5V8eWyqVcB1UCkHmc+iuoZQ2lHIn7w6ZrwnrY9alLBL2JDDi2xHiELNF0emtLZVJ6b28N631rWgbaqNSqLDrY4djpCORazvBh4Caj2TXNRGgsBBu3YKxY0FXFy6+eVENUKpX8BWpNrm4lHyHrrkuvtN8MXU35Vfrj/hQ9SFfrP2Ca2OMiTYPYk/0TpSGSiKXRhI0L4gB7w7A2NmY8VsEieO2pjYuvHkBx0hHAmYJTDANTQ2WPFjyt8A/bQNt3m16l02hm8QmOMC0g9M4MOWAqJTQEaN+HMXI70dSV1hHxKoI5MZyLr97GbvednhP9KZV1UibVgXW+eD+4E9KE2bj0UsYRhnaG/J46Os89FhAyF0bbPIbkRvLGfzxYJwHOXPtg2vCMSs1cD3iSkhVCJ/pfMaon0YR/ko4a6vW0lzdTOqxVCx8Lbj6/lWSDyTjPclblM2rza/l9le38RjjoTYE64iCewVsjdiKnpUeOqY6nH3lLI79HVlwYwFbI7bSXN0s1pFb+26lpa6FVzNeBYQmeFNlE17jvBj98+hu3/1ey3t/m6tIZVI1pmBXX3oQlBvKksswcTXBwsdCjTHYVaK8Q87372LSrknoWnSX1gcB4Bf7fSylSaXITeQoekUhUSqQtjXT1qxFrxd7UV9UT8rRFBz7O7LV6iCNG6txzAhFJYlk4HvCMK/sSRl7Ru6hTaYOWO5J6rY0qZTUo6kM/mhwjz7az8fgjwcLflSaGuTeykWmJyPkxRDufnuXb6yF9RAJanvBL36/UPZESDZ1zXWxsohEJdGgzcwacy9ziuKKcBzgSO5N4fu++AIaPt+LlaM2r6csY+iGoaLM5lDLoahUKkJtQwXZu6Y25l2ZJ/SP/gCTKhMCjweSNzxPXI9zb+WK3qU+k32wDramtb6V8tRyDB0MybmeI/aNOhRriuKKSNidwOAPB5N7S9iLug5WrAKseKv8LZL2J3F45p84W7iR5zsClUQDMKOtsY2miiaKHxUzilHsnrObgv4F4ndYBVox+ifhHn3wywOGfDxEbUCgb61P5Fvq61hNbo3Ya1j2sNOzra6wjvK0cqyDrNEx1SHqkygq0iuw72ePvo0+9zbew6GvAzYhNt1sbex62zHtwDRA8B33m+HXDQD6/2Z0ePWO3zYe29BOGffi4mLWvbWWG0rFv933B6xeo7bvy/RlyPRluI9yR6VQIdWSEvNNDCqFCo8xHuwZsQevCV6M/X0s1TnVWAdZ097cLnr3Xv/wOi21LWKdcWjGIXTMdBjzyxgaShtoLG/ExNVEzKdvf3Wbpsqmbr2wez/fI+14GtMOThPtem5/cRu/6X7/JYaZVYAVi24v6nYtM0Om8OJb/7HVkUZzI5fW3cF5sDPuI93JuZ5Dznc5GL1qRNjKMI7fO86EXyaofSbcLpxWRSutS1rZ57yPmSdniq911I5abVoM3DuQR7qPGLdpHCqlit8CfyNgToDolerY35HeK3qrgbYvv3OZorgi5pyfw5V3r2DsbCz4bmsItcCPK39kl/dDtXsBEPPgq0OuMm7gOM6vPk/6qXRWJK+gQdHAusvrwBIKdhSwutdqtHS1sOllIzDh2pVoamvycOtDpFpSghcEU5JQwv2N9wVW61/DPg1NAeChUqporW8VLIr+TbyS9kq3vobLEBcmbJ/A8YXH6XX+Sx4PXcV7+7vv93MC5zDVaSq7Nu/Ca4IXKqWKtqY2Sh6XUF9cL77v2ofXRDlhC18LguYFqd0/Xc9V7E+xZF7MxMjJiJHfj/zHs+//B/HPsO+f+F8TJTdLKLxYSBhXeRYyjcZp8+n3ptB0KowrxGu8lyi7IGuqoSqugpY+dmgbav+HGrmeYz0pfFBIWbKQWFamVzJpz2RRtqhDF/3R1kdcff8q9cX1PNr6iMl7JnP1/atYBVkx9rdO1Ef2tWyuvn8Vr3Fe6FnqUZ5azoEpBxjx3QjsI+y5tO4Stfm1akOsq+uvknI4hQ9UH3Qzo+1AxLkMdWH8lvEY2Bhg5GAkFuKPfotFs60N75idVFr7ANOJ/jqa4V8NFz1hmiqbaK5uxiqoi8a4hgRNbU1eSniJNxdW4HNnG3k+w1C296WpqgkLXwtmnppJ1uUsanJryL+bj+c4T3EQmXY8jaK4IlobWll4YyHrataJiU721Wzqi+qZfng63pO8kUgkasytnBs5VGVWETAroEeEpUQq4YUTL3TzxVtbtZbji46TejSVqM+i1GTXupqfd8Sx+cdoKGsQG5EgNIzKU8rRs9JTGxDtjt5N5sVMpuyfIhrDd4S5tznDvx5O3q08VCqV0LB+zgMMOs2EL799mdLEUib/MZnfe/2Ovo0+s8/M5syKMzw59IT1yvU93pOPdz7GZ4qPGkLq+dg3dh92fezUtMqh05fv0rpL3dhPtXmCR5RdH7tuJs2hS0JFZmxpcinlqeW4j3RHpiejtaGVuN/jAMi6nMW9n+6xpmxNj8elIdVQK4qtg6zp83of7v98HwsfC+pL6tUGUyAgEcf8Oob7G+9TkljSTW//7GtnSdyTyKTdk7oNiqUyKXFhwrHZK9UZBW2NbbQ2tHJ53WVSjqTwftv7asX08+zLrjHkkyHI9GX86P4jZh5mjN00tkfW4X8UZ1ae4cGvD1ApVLxV+ZYaC0XZriT9rOBF9HxhBgi+bf0d0ZRrIm+Rd3sdhOHd5XWXcejroHZOrYOtaaxoZP/E/Qz5ZAjOg5wpelTE7a9uk383n96v9FbzXQChkaIp12TsprEk7UtCU66Jz5TuWu3n3zhP5qVMlid0R6Z1RMCcAGzCbHp8pv+70d4iFFBL7i+hNq+22z0SvyOexzsf83jXYyZsm4CZpxnuo4WG0O2vbpN8IJmiuCKMHI0Y+tlQ8XPm3uaoppqzW+db8hzyMMvWQKGApQ8Eed7c27mM+mEUl/tcJs8hD21tU3ot8sIlygVDe0NcZ7lS8rMw1FGqlBjYGAieU39T9Ea+FUnEqgg0tTufTZmerEdP07+L9DPpItvs2MRjqDRUHJl5RO091TnVKFuu4JDrQFisgjbzEhbcEGTT5MZyaoK1iLE6yYPeD9CX72G0ly0SCeJwNq1CYFjt+2Qffk/8CJofhFWgFeM2jWPjvY1wFiRKCYYJhpQklqCprcnOwTsBkFyW4DDcAQNPYV0tTSol62qW2rPWURQ1VTXxveP3BC8KZtQPneCb/eP309bUxsqnK4nbFIemjiZBc4NIHDCKxw4/seZf/ahxKyE/xkBgou2aKErkNO2DJkMrTI07f8/E1YSKXyqI+SaGyXsn4zHKQ20tMXUzFdn2DWUNzDo9S00mLPtatog2r3xWScnjEhwHOIqFkqmbKX7T/NA20Ba/d9QPoxj1wyhqcmvIvZWLdS9rZHoytI20idoQJTI2qpurhePWekLO4yZUSjnhr4TjMcaDoodFPPN8RpVJFYOuRWJZE4uWTjARq4QhQsLuBJprmtE112XC9gm01LaQdSWLpsomzrx8hsyLmQQvDGZx7GLMfTqb1S5DXHAZIuQYF9de5M5Xd3j12atYBVoxZuOYHu+7Y/OPkXI4hWkHp+ExxoN333kXzSOaaOdqq61vilYFWjmaaDU1UmH2lPzaQvydbEV/iiOzj6BTIyPJPxkNfQvu3Z+DREMiAnTcRrrRZtOGTF9Gwu4EUT5Nz1JPbHSDgOT+Pfh30RPv7vd3ufv9XeacnwNd8F/5d/O5/uF1XIe5isO+nJs5oueG+0h3MY/S6oL5USrh3k/3eHLoCd4TvVEaKFn751ayLKS4lL1ChwpLdZZw/cp6K2mQNnB67GmCsqYxPqWQr/pvZcC7A5iwfQLGvxvTrmznwZ0HrB+0Hk3Zvy+9WmpaxGFSR/zg/IMoITf8X8OJ2xzHhx9+CEDlGGGwmHUlC7mxHK9xXjTXaFByr4SZe2di1HYOGKn2fUWmAqtfodGAs5OKuI3CfhqyOIT4wwdpMLKhWd+c+Og1BAVriOds/8T9FL1SxFS/qRyadggDTwN0jHRoKGtAU66JhqYGnw/4nE2Zm2hza0NuLOfaB9dIO57GesV6mquaSdqXJO4lyQeTOTb/GFP2TsF7ojfPx7UPr4EEItdEUpFWQYllCbvm7WLpMGGtvvv9XVKPpZJzPYeq8ab8HPQ2FrUCyEKlgjs1txmIlD78x8jlPsBgmYx1X67FPteBSIdIRn0/ioyzGewds5fx28aL8sxdm1cTd0wUBoIfw73we+zdvBcbWxvs+9jTZ2Uf/mXxLwASAxJZtXUVP+/9mVfSXukmiWlebk7YtjBSXVPpu6qvOHxoHt3MgO0CgMQ+vpraWmsUrQoMyzPRq8ylj+13FA84S5FNEcbNEYAgK5l+Oh0dCx1iw2OpMarBWGos7p0JuxOEa98sNN68Nnmxf/N+oj6L4vYXt7HrbSc+GzomOiJ75c7XdyhPKVc77sS9ieTczCH662h2DNyBREPCrDOz0AzQJNwhHCdbpx4bRhKVJpptmpiWttNcWkPa8TRk+jJch7vSfvwMBv6jadU1xqAim4YcUxwWOvByyssAlH4kMFlbkls4vug4YcvDGP3LaKK/jUaqLRWlzz3verJv3D4m7JiArpku5anlXFt/jSGfDlFjQTyfjxc+KBR8JPdNQUtXS8jH/mK7rFwJWsV5FFxpYtVvnvR+pTe+03x58OsDiuOLudnvJBOPTUTb4ByW/kvFxngHwEyqkNKuUY9E8hcgRFuKW7QbnqPUWWggKC08+OUBilaFuAZKJBKxOVdwr0D0mXpeQm/o50O58ekNcm/mMv/afCzDLHmjQGCvnFt1jtjvBZZV+ul0cm/mMn7reIwcjGiqauLWhlv0WtwL74ne1CgEYEG8/35kyrWMdDFgc/hmbMNsaSxrxCW+kRh/DY5PPMKzXiP40MOMQe8PAmDh04X0+qkXbVptTO0/ldr0WgJmB2DmZcaTwwLr3meSD39O+pOoDVG4DHMh+UAy2deyxb+jsaKRez/dAwk8PfkUt2g3NVUAXTNd/Kb7UVtQS0Npg/D6BC8MbAww8zQTpSFBYKZ0bcBe//A6BfcKWFejzvLoiK5AvcbyRp6efoptqC2W/paolCoaKxrRlGt2kwDvuId2D9vNmF/HiMDQ9uZ2Yn+KJT82X7xeTZXCsM3MU2B15sXksXPwToZ9NYyI1yK6yfhlXs6kvriewNmBpBxKoeBeAS5DXRjx7QhiXynCsDQD+7QrFN6cyZBloVRmVPKTx0+4vOzC7xb/YmrJVOxTq1EqhPojfkc8F964gPUiaxp1G5n9x2xiTWNFVtDzYeFrweS9k7ENs1XLZ/8uuqoUddS2z3u0j/xhJNc/uY6umS7DvhgmDvr8ZviR/Gcyxso0DCpzKXPvQ9Ruof62DrKmMqOSi2suklaYipZrBDJnqXjdvMZ5Eb8zHt3HuoS82Olpq6WjJeZAAIW2hZxbc443p71Jzs0cqrOrse9jT68Xe+E2wg1jZ2Nyb+eiUqjYMWgHI74bgfMQZ/xm+BG8MJjzb5xHIpUwdMNQBn80GE1tTd5teheJVELKkRSOLThG9NfRIgipg6luVPYM+b0/aJfpkp29lPBXwnHo58DhWYepSKvAP8mfwN8D8QlXr8cc+jpQkVqhBgr/u+ja++gaT0895dSyU8y5MEdURTHzMOPJoSds6yeAmIZ+PhTrXtbU5NQgM5CJ/bWuoWuui665Li21Ldz+8jZ2fex6HMg/H3J5z7Xtfycq0gTlkY69uSO2bN7MQImUPih6+pgYfYABEimrhr9Or6IQ3ip/i8byRpqqmohcE0ldUR1JWUn0+a0P9ob2qBQq2hrakMqkPfadQGBKKdo6f7c8tVysD+7/ep/rH15neeJycY9NPZZKTU6NOOw7v/o8mnJNFC0KCu4ViLYSEa9H4DfND0MHQ1rrWzn76lmcBjkRPD9YAEMN243HGA/6r+tPxdMKCuMKcR3qip6lXje7EYBqG18cI+H6J9e59+M9ltxfgkplDIBmSz0GFTlIteXc/vU2SASGnmN/R1LrU2mWN1NtXI2GsbBG/uz1MxZ+Fsw4MoPZAbNpamui1bQVfRt9Hm17RNL+JCbumEi7trD+tspaubDoAkPdhrIrape45s84NkM8vufZ0SAAcUqThf1/+uHpYr/LQS48J+06jvx21gkfz0xufnaTIZ8M+cteRNiAs1yzMB9uTtPTJqFXIYGm9k728tYJgid1R1+3g+DgFu1G8eNiDGwMCF4QTK9FvfCd5ovcuPNe7uqR3gEo6Rr3fr4nMEW7rIlyIzmJexPJupLFiG9HCD1rqaSbOpdULhQp8Tvi0bXQxXOMJ/HF8YzYM4K+Pn0p+qBIfP7eqnhL7bPuo9zRMdUR2eXPf7dKJfhdakg1uPLOFbHOGPmdes3wT/z3439yzfuvxj/Dvn/if0VIpVLea3pP9JozLUgEhAT46amnXHn3CksfLhWTEZPCJFLevkRk1GI0NDVoa2zDsb8jKhVkZEDV1XjaqhuIfCtS1MQuSylDS0eLHceMefll+P570NQQfDM05ZpUZVbR57U+2PSyIdcjFy1dLdpb2lG0KUR2U5/X+hCxKgLPsZ6i9KS5lznjt45H20ib+pJ6iuKKRFm0juj1Yi+cBnX3vQNAAnPjV5GQrImOjS4azz219gOceVKroMbCjSbD54Z5ck2GfDKEtsY2Rnw/QixWi+OLufHJDYycjRjxzQjqTfTI842mztSJ8tRyfg34lUEfDGLwh4PxHOPJ1fVXufHJDVY8WSE0rV+PoK2pjeQDySTvT0ZuIldLPB36OTD5j8nY97VHIpGgUqpQKgSfEYlEwqOtj3i86zF+0/2QyqTc+PQGLbUtIptIQyok6rE/xrI5fDMLri/AMdIRubGc6Yemiw2LtBNpNFY0dkvyOqKlrkVEsSraFJQ9KaOpooldQ3fRb43gVyQ3kWNoZ0jmRQEdXHCvgML7hZj7mJO4J5ER34/AOsiafqv7wWqhYalnpddN7hIEOVLPsZ7c23hPZOqYuJqga6FLxrkM8mLyCF4U/LfD57fr3/5bBsmTw09IO54msFB6dZdNKU8tRyKVMO3gNLGo6AiXoS68VfFWj9IRbY1tbNDbgNd4L8x9zLn95W1WZqzE1M0UfSt9zL3NsfS3xCbUBsvA7szUrpKgHdFQ2sBG340Y2hniNsKN4wuP01LbwjsN76i9T8dUh7CXwnqUQwVBctUqyArnwc5/K3vhm+zLxBMTSXVNFZl/cZvjOP/6eSLXRTLiO/Xr1N7STv7dfAztDTF1M+X2V7eRGcjExlv4y+FkXs6kuUqQtry09lJns+aveLz7MXf+dQf/mf4EzApQk61SqVTc//k+2obaDHh3QLfBa1tjG/vH7yd4YTATtqmj4aBz+HrqpVM82vqItyreQmooFLDVOdXc+/ke1sHWLLi+oEdPrtY6AX3acd9nXc4i82ImL959Efs+9sRtjuPmhptM2D4BuZGcJfeWqP3234WyXUlpYikb9Dcw79I8NWm1ymeV/OT+E/3f7i82Bf+n4uGWh5x95SxGjkbU5NaIXoMd0W9NPxrLGsm+lk1LnfA3T9g6gdxbuWwfsB2AgFkBjN4oNE6rc6rJuZ6D00AnlFPNyUzNxKTSBHlDs8gGB0j8IxHLLEv6KvuyY+EOxj7YjYkrokyxg8yBab7TOHPvDAZXDKjwrsAt2k1othvJxaHWvZ/v0VDawJCPh/wtU7StsY1vbL4hYHYAY37pHLgcnXuUp6eesrZqLc8uPuPGJzcASAp5hkqjk717/5f7xHwbw8wTM/nF7xcWMY3rA6/jnSynvV8V9n06G+jmixZQ+qQCr4KxaLfZoFKBUqGk6GGRWrPwVv9bfPrVp2rHOcZzDPHF8ey4t4PQ70K5U36H8FfCaSwXhhKKPAXLYpdhqWNJea9yNUBLR3SgJDWkGjhEOqj5NQCELAkRi+SbG27SUNKArpku0nYpTbpNfL12LxNzP8IyoIoxv47BPsKenJs55N/NR6M1CIVMD5lMwvk3znP3u7usKVuDzxQfrIKs8Brv1c1jsGvoWeipDV/TTqRx/cPrOEQ68Kvtr5ieNcXgdwPmX5uP3iDhXNUV1WHhZ9Gjb+3jXY+5+v5Vxvw2Bk1tTQJmBTDgbaFpn3srF59pPrzg9QJVJlWE3P2KusJVGNobcu7Vczy7+Iz+N/uz98xexp2ajEI7GW3DPuK+49DPgcr0SmT6MlFSrTS5lIebHuI33Y/Mi5koWhV/KwUHgtdW8IJgEaTzfNTk1VCeWi423mT6MlpqW9D7Xo9Cm0KqF1eLz+Lm3ptFqfCVzOLGgBtkpiXB3s61vc9rfTiaZY93vjHtmi3i2qhsV7Jv3D7kxnJmnZ6FRCXhtezX/vZ50bPUEwr4AYKHkZaelrDn/LSR7Q3b+WLAF0yfN53AOYG4RLmoDeDfrn1b/O+uYAsTE2goqsG45CmqUmdKk4RmQsnjEqruV8Hqp7g45qLb8gxN+w8AYxbHLhZykN9ukDnYlyd+T1ChQNNQl5AlIaSdSCP2h1gMXjKgXdmOT4oPxfHFuIW7kR+bT9zvcfRe0Vv0gvlj1B9UPqtk5dOVPUpBhy4LRddcF5VShcxAxsFpBwE4Me4EHnOERsfJpSdpKGkg9VgqANbLrDGrMENDs4qGsgbSTqRh19tOZAJbFVux/LflNPe9gW1vW5TtSlQqFQWXUmm29adFX1jHVH9divQz6RTFFQHgn+zPkclHqCuto1Xaip6FnjjAu3v7LlmuWRRNKsJpgCDf6DpckHFKO55GXkyeOGTXs9TDZYiLGntF0abge6fv8Z7oTfbVbCQaEgatH8SayjUYf21Mm6xN9GZL/jOZgnsFAJheakDP0wafR60YNj3F2MOTYrNnvF7Ws8T+8zGvqYlle3YxX7KCeP14Rn0/Cn1rfXyn+nZDxXeNxjZhDWzXakfLVEtEonfNxdI90rE1tGWoZ897ZKVpJfEL4lkxQt27urdtb9yaZlDWepZq6fcY6m9Apicjx380TklnCLi2mVQLHw5PPczA5PW4A7c+vwXAtPvTWLtjLdJ2KXcC7lD5rBJTN1NRmu1E9Qk4BllDspjeW/A4fuXpK91kVcXz08XLrSNybuQQ93scQz8biv8sfyQaEiEnOgxOLU4YtRpRXFBMS20L5WnlBM8PRiqTkm78K+Zl5ow4DvWtcax68jIqlYq2xja0Rg2ltPooSXavs/BcNCVu/WFh9/NmXG1M/MF43Ee6Y9fbDm0DbRJKElhyUshrjEqNeHb/Ge1NgjfrH6P/QNtQG6cBnet1dU411VnV2IbZiqAXh0gHZp6ciU2IDd/afYvXOC9mHBWakK2t4JR+ner7efDb22LeGbo0FMdXXiTP4hmXhl2if5u6QseSgvXY/y5Ft1EXv0dtSL2y1fKvh1sekn0tm+H/Gs7FNRdxHe5K0NwgVhev7pZHdkgQP7v4jKvvXcV5kLO4fysVSp6df4aBrQGeYz2pzavlxr0brD20ls8WfcaiXouwCrTCZ4oPWjpapB1PI/1MOiO+H4FUJkWmL+OVtFfEmq5OKYCaKs0qOTdkLe8azkPRqhDXCqXiCTXaZwFQSdSfMz07PSrNBCDCN4e/YXnv5WJecGblGe7/fB/bMFsm7ZmEdZA1lv6WpB5JJedmjvgdFr4WvFH4BvXF9WwK2YTMQKY27DN1N2Xqn1PJOJfB+TfOE7EqAqtAK44vPI7TICc1xl3Up12YpUD/t/uLoIrrn1xHpi8TpbBBAH3JjeVY+llSlVXF8QXHGfaVoMhRX1zPt3bfEv5qOKN+GMWD3x5QnVPNsM+Fhr2ZhxnR30Zj39eeuqI6kg8kk7Q3iYJ7BYQuCxWbyfl389XABDomOsKQqQcbCoCYb2LIvpZN4OxAsV7MupxFbX4tvjf3UmvmTJlDL+SWAlhWZqpPv28nk517lyFXhnBzwE00TH4gtLaZbZH7MLAzYOD7A8n5M4dcy1yUukpxiLd37F5qcmrU5Of0rfRxH+FOXWFdj97zz4dKqSL1eCpGDkbiXtfe1M7EnRMxdTflyntXMPcyJ+brGFGVxC3aDfu+9gz6YBDjNo9jxjQVOnWlqPQNKE0spSRRUHVorm4m/Ww62h5OlLj2xfSv22LH4B3kXBfuIZVUxf2A+9hts8Mm1IaQxcLgr6MOb9Vupcy6DD1LPc6uPEvygWTeb39fZNcef/E48dviWfFkBUM/H4pDpAPWQdbiHn3+dWHYF/2vaHHd15RrcujJIU5tP4XLeRcGfzgYRZuCvNt52Pa2JejXZRzcXo+8oQKlVIvav2afD7c+FIdXD0MeMpxOhZXcW7nc/e4u/db0Exn/GecyqCuso9eiXhTHF3Ny6Un6vdlPVNjpGskHk9E108UlygXH/o6M/mU05t7mVGVWcXLJSYIXBqOhqYGxs7Hwd/ZzoL25nR9cfiBwTqCoRFCWUkbsj7EEzArAOsiatqY2FC0Krn1wjT6v9/m3wz6pVIq3d3dQz383OoZovwX/xrhN48Sc9/jevbyuaP1PfcciVRsbimMYFyFc85jvYri14RYRb0Rw99u77Jy3kyzXLBTrFWhINFiZvlJY+9qVAmDtub7N84O1rqxe50HOqNar1Oqt54eyacfS0DbSZtnDZWrWP/pW+iLAt6mhifjt8WhoaRA8PxhFi4KK9Aqsi4T+UMa5DM69do4F1xega66Lok0hMs+eDz1LPSz8LJBqS0Xpc93aEtweHqLFZBQr01ciN5aT8EcCxxccx3GjI01lTWxdvJX1rwtqAWaeZqLFSGlDKe9ceQfzseaUrSnj2ofXKIgtoL2lnXYtYdinlCo58N0BcrfkqoE7Cu8XUp5Sjl0fO5TtSmpyarAKtBLXmSl7O/fVDhlKAAe5H45lS5i5zZ52ne3UvxBKSWKJ2AtRSToLfIVSwfAvhzP8S+H56lBv0G7WJml/Eha+FmKO3NFve3bhGXMvzRXvt/+MHOvzce61c3hN8FIb9pUklpCwJ4GMsxlEfRqFtqE2p5adEgCWl5ex7hsLtBurKIorwmOQDedeP4dtmC2eYzxRqVSUNpTidk4Y2P+delqHMtjfxb6x+8i+ns079e8w78o8lO1KTFxM1Oxe/on/fvxPr3n/1fhn2PdP/K8IpVJJVVUVfjP92LOliVoLd5xuXyFpvyV+0/2wDrZWa+jUmrvh9LI2Rk5GHJp+iLKUMtaUruHePfj0U/CPe4xJa4las6eDwXflQjvuD/7kiJYrw5Z7sNFnI31X9yV+ezyBcwLVDHM7ZBEebhHkp2J/iGXwR4PVkvS8O3mceFGQUun/dv8ezU+7yjf97PUzchM5i+8uBoSE9O0NhrTUNFOaX82Lqww4Nu8YTw4/4b2W93Ae7knrlQLKHENRyASEfWlSKY0VjTgNcOL6R9dpqWmhKrMKgA9UH3Du9XPkXM9Bx1SHAW8PQKkpo8RVKHDkJrWELQ+juaaZbZHbiPosCv+Z/tiE2oiJuFQmFdCJMfnMuzyPxnJBqtTUwxRtA21M3U259cUtjsw+wqr8VTzc/JDrH11nedJyLP0s6bu6L77TfcUC9unJp9QV1XWTDjRxNcFrnBc6JjooFUpKEkpAJTQhrQKtuPv9XUoSSui1sBd/TvoTfVt9NXZChwwMCAyNPdF7QAJDPh2CmacZvwb8SuhLoYz9dSwvJbyEVEtK3OY4Yr6JYcB7AyhJLKGtoY3kg8nc+vwWY38bS8G9AoycjHqUdgJhyNoVEdnhMXHxrYvUFdQx6P1BxHwbQ3lauShV0BEyPRnHFhwj9WhqN6RpcXwxCbsTWPFkhXivdo2tfQW5GkAtMQdBEkvHVIeW2hbu/3ofmxAbnAc5C8e19iIgNLYHfzQY617WYhIq0ZDwcsrLHH/xOEgQJUT+XbTWt2IdZI37aHcqMyoF43j/7sf87yJ7RDZf6nxJ8u1kvhr+ldprzdXNfLrtUzIkGTR6NapJGlkFWBG2PIyQxSHdhgmN5Y3sHLxTLNDv/XwPAxsDcdinaFOwJ3oPAbMDMPcx7/FcVzytoDSxlCuJVzDzMOtWlK8uXo2mtqYa4qsjNOWaTNw1kbqCOjaHb2bwh4N7ZHdZ+FvgNNwJlaRzqFNXWEfM1zFEfxPdTeIMBNT9/V/uszh2schG9H/BH9vetuLfUZJQQtrxNEEy529UkHuKUT+OwiXKhbjf47ohSrV0tfCZ4oOF33/9Gv+7MPc2J3hBMGbeZrQ3tatJ0IIgTdfhZaL27/6WzL00FwNbA4ydjUW53oJ7BRybf4ypf05FXixD94wOBvclZEhWsuDh5wwJG8rcGfMIfyWcg2kHueJ8BffCdwGBJdLxuOto6TDIaRAx52Ow3WpLunM6+lb6HJt/jKFfDBX9XJ8cfEJZShmhS0OpLajFKsBKLBDqCuv40f1HwZ+nj53aHlZRARJzE7FwbaltEYcf/g/dyLHujU2RDQnuCUhlUrQNBDSga7QrmRcyeRjykDx3J17xdhekOqUSauqkxJ83JrD1Y6RtzejV5NBS4UJrvSZbwrcQvCiY8++cZ8SeEVj0shALkusfX+f+RuG+mug9ka1xW0mYm8D8RfOxC7dj6p9TKbhfQMzXMbQnt1OYXEhFdEU3+TOAk4tP8njXY95peIc55+Z0ez38lU5PvBeOv8C5V8+xd8xefFwdqMKXLPcGpFIwcTERQQI/e/4MQFn0GvrdnIHUbCJF2cJA4tj8Y6SfSeeNgje6SRC3NbVxdf1VHm15xJT9U7o1Ka6+f1W4FhkVbI/fjpmWGad3nFZbE37x+wWbXjai5yvA4ZmHSdqfRP+3+2P8pjFXDlyh6UoTXuO9xHWqww/OPcOdo5OOUuoQhJauFmkn06jKqqL/uv7kaAnNqgOzEgjO6RxSASTsSiDlaIooIQWCnFTAzAAcIh0IWRyCUqGkrVFAIHfcO+dWnSP7SjbL4pfhN81PlBlKP5POg98eMGj9ILEh9/TkU868fIYp+6Zg4WeBuZc5KpWKPbP3UGNUg2OQo9gU7OoJDOCd6k1lfAH1xfUcX3Qc36m+BMwOwDT7FIYGf5LhXkz+3THY9ralKqtKHCh+afIli2IW4RDRWcRnnMvg+KLjjP55ND6TfdA111Vr4IYuCSXm6xgqNlawkIWk7UijdUoruma6PaLRewobG6i4V45T0hmaPcYw4oNBxP4Yy6EZh/Bb4YeGUgObIhvsCo1pC60HjLELt2NLny3oImH6wekcaT+Ce14N8sWmDNs0jvu/3ufMijO89vVrXBh+geiL0eQOycUt3I2qzCrit8fjOdZTPN/mPuai92BPEf11tCi97j3Rm2mHpjH+2HieuT+jNdEfpsOQj4dwatkp8TMv/f4SH374IaPjvqUqs4STi08y7KthQiNDJaFZ3kySXxLOJtNYcnoQ2dez+dX/V/xWDmFvigCi0qkuQFUqAyxEqVXdJbp8qv8p5uXmHPQ8yOCPBjNo/SDxdyubhAZ/61/Ntg4ksaJNQfa1bNxHupN+Jh0NLQ3chruJ+YgYEgnGLgJQ6sWYF1GpVEgkErT0tJCoJAy7OAzdOF0YKbDsFG0Kzq86z90/72J2sJ2snLskS/syeqAlxSUF/MeuUp1hBaha4NdVv7Jx7EZAkCqM2hClNuzLOJdBc00zvlN8ObH4BLq9hPtMt0GX0sRSLIIteHbhGbUFtYzbMo6zn56l1rCW7W7b2fbBNvJi8jix6ASDPxosNmYb9BvI88jDwteCksQS/hj5B71f6c2Vd64w3sKTYhMNvJ7qYDivBdCh3tRRPJ7Rp4eQ7HcP1V8sBt+pvph6mCLVFZrPBs0G7B2yV8x7OsAo9aWC1NNhp8PsfHEn2jLtbiylpqomDs04hNd4L7X1uSOiv44m6rMo5MZykdEFgtyWaaQp9Wn1mNiaYOlvSfz2ePxf8Bf38Qa9BmL7VRPt7S4qAmhqa6IsKab3Q22u9o/nYb95BPT3oTq7mvQz6bgMdcHB0IG82jzSvNJYV7MOTR1NanJraKlroVhezL2CewAUzynmvZuCNHVtQS0yPRn1RfVcWneJ9uZ2guYF0VLXInoddzT4DGwMMBgrDDzDlodhFdAJpAQodotEaqLOvAXgr3ztcfBjgmssKXpYRNzmOGHAYGOLS+kbSJue4veogWZZFkqlI1VVVVTFCQ13EKwkEv9IRNtIm+D5wd0UMQC+NP4SrwlejPxhJM6DnUWbBRD2lr1j9hIwO4AJ2yfgPtKd34J+Y5TjKL4v/J6xH42l18JeIkgy4o0ITD1NiflWaHC/nPqyuqy4Sr3GaWtT8VK80LyueFpBilcKfWP6khCYgEVhE+ln0rn1+S2iNkRRXVmNZ5onTz2fUvVaFX9a/Sk2vod8PIT7P98n7ViamhfflP1TaG/qZN9JtaQY2BigZ6HH6qLVfwtOAaFmQSJYLuTfzccywLKbvGbX6Kqk82jLI/Ss9NSGfbuiduE2wo2ZJ2Zi5mHGzJMzxbxGS0+LsOVhOEYKz2HKkRQKYgvEYZ+Ro5H4XXl38jj/+nnxe7v2Hsx9zBnx/QhRtt3c25yZJzqZw19bf43nWE9x+DTog0Hiczj448F4T/JGU66JplyTfO8oGowdqDN3xsADji86zsMHSh46TWKQIolBNwZxv/d9Go1tUbQ1UJVZhdMgJwatH8SNT25QK6sl6EQQoSEC8M/IyahHpY6UIymcXHKSOefn4Bb9H0t57hm5RwTTuo90Z9yWcdz87CZpJwSm96ifRlGbX8vK9JXib805P0f8naaqJhRaIdSbOqKpCY93XuLJoSdUPRPk1WccmcHq34VctcNvrKstQZJPEh+f/pgPfv8A32m+2PSyYXPvzQz7chjHZhzj2JNjDLYRpEb7vdWPgNkBSDQkNFU10VDSgNc4Lyz9LTFxNVGzCKnKqkJLR0uQYtWQUJ1dTX1JPTa9bHh66ikXVl7gQb8HyP6UYeRoRPqZdA5OOyiwjF2DqbWEWoR9UakUhr6owDLAkuuDr1Mhr6Dw7UISFiUQODuQ2vxaUo6m4D+rU3ko9sdYcq7n0GtRL6oyqyi8X6gm35d1NYuWmha8J3pzcvFJ7MLtcIlywcLXQrQZKUksoehREVp6WlRmVBL9TTQ+kwXwkKJVQd/VfdXkSOsK6oj7LQ7rYGuS/0zm/sb7vJ77Oi8lvPSfyrc6+nkmJiZoaPQM6PqvhMdYD1KPpVLyuISKpxVizVRTW/tf2vfbZO3MOiXUks6DnEEFlRlCHlNpWolmmyaNdY3oGwrrceGDQraEC7YKAbMDyLmRw/yr87v1HJ4P58HO3eRxn+8VvJz6spr/Z0c0lDWgpauFTE+G3ETOupp1op+itqE2bxa/Kb7XY7QHhvaGmPuYk309m11Ruxj18yg13/GAy99zcLogyxq2TKinVBnCa40GVjwLmYqNi424v5i6mRI0L4g+ffugd1aPhrYG2pXCWt1V7aDD871DsnPwh4MZ/OFgAApKBWCWpZ4ldoZ22L5uS0lCCY93PmbAewNIP5NO4h+JzDw1k7LkMi6tvcTie4ux690duKhSqkAi9OG8dPsRmNMPnaaPoCkXfatBvFXeleEm1FzT/5xO7M5YBmV25iodwDGjBiMOzzyM/wv+qFQqAmYH4DvFl3U162htaEXfSl8c7DZWNNJQ2qDWX3i49SFyYznuI91J3JuIpb+l2kBy/rX53Z6RHQN3YO5jztrqtWLu1ZGfHRz6O6qxH+CYeJojk0tYW/EWM47OEN/34eUvkLZLMawwRCqTYh1kTVNVE1mXs7D0t+yxBk87mUbqkVQGfThI7F059HdAbiLcgz2d53/i/yz+p9e8/2r8M+z7J/5XhKJNQdyBOIIGBFHgMxxUKuSnd5Fy2Bf/F/zVCh2AJiNrLEdbo28FEasiRIPpmzeF1596j+e3jcIGde3Da6K3mMxAhm7gPPQrc6jPNkRuHEDvl3vjNNCJoPlByI3lPHyoYuNGCatWgf9fOVfI4hAcIh2oK6xDQ1Pw6ZDpy9DQ1ECqLcXEzQRDe0Nchj4n0dlDOA1yUkPKqZQqFKWV2D+7TdH5R1RPfAWrYCtaG1qRSCRkX8rAPvUOlTZ+4rDvxic3RHSahY8FchM59zfeF7+zz2t98Jrghe9UX/5l8S8cHILJCRJYRoZ2hoz5ZYyASv8+lp1RO3mz+E0sJgiJYdrJNIrji7m2/pr4fU8OPeH08tPMuzxPLJ7y7+YDgjyBZYAl/jP9xSFBV48pQGiUdqknW2pb+Mb2G4LmB4lo2pa6FjaFbMLQwZDaPAEGN//afHHjrcqsEhuoPUVHU9Jvmh8D3x1IW2Mbgz7obGx2FPMD3xtIn1f7YGhnKPqLJPyRQPGjYs68coZlj5Zx4sUTfKzxMesV69WOO/NSJuln0olYFdHNT7Lfm/0IWRKCgZ0BmRcFD6muwz6VUkXhg0IkGhLs+th1kysctH4QA94Z8LeyKZFrBe+9nBs53ZijzdXNVOdUI5FIuPDGBSLeiBCbax2+QguuL8A62FrNC1DRqqA8tZz4bfGYhLvT983+ooRZR+TczOHZ+WfIDGS4j3THOkgYvM+7PI+iR0VcfPMiQz4Z0mPBXfakjD8n/YnTYCdch7niPdFbTZv9XsE9UspTGPHuCLZ4bxEH4B2hoa1Bpn0m8kVyNaS2S5RLdzncv0JuJGfUT6PE+2/+1fligalUKNncezNauloEzgnsJnXQEVGfRNF7eW+KHhapFUEgJJ09NWc6QiqTEjQ3iMxLmTz49UE3z6LUY6nkxeTR/53+aPfXVkOO2fSy4bWs15Aby9WS3I7QlGuiY6KjxogxtDdEx0yH+B3xnFlxhqD5QcJ9+1ckH0ymvqie82+cF4uLwR8PVmvadYT3RO8eZdYMbAyYfmg61z+5zkbfjSy6vUhNuvT/JFyHuuI61PW/9Jlzq85h7m1O2LIwlEpIvl2FpUkbVv6WOEY68sKJF9h2aCvf7fqaAUhZRBPWKijOrmVbdgb9D//Kl1//i0+PfsonGxrg8DOaDRK5vK6EmG/usDJ9JSauJmhINCg3Lyd7bTa2YbbsitoFoIY6n3ZwGsp2JXGb4tQAD9DpdWgVYNVtkL5gAcBgtu0R/t93ii++U3xJP5PO3jF76X2/N5ZlluTdyWPMxjEiWnnG6ZlofybcM/rtClQamnym+xE+k3w4pTkBv9OCpFyO/yicks5Snz0fTW17hn4xFKsAKwqlwtCmpb3T10fPSg9zb3OkMima7YLPUlZklliw2obZ8uziM2rDaznveJ4lU5fgPMSZwrhCGkob1MAsDpEOQuP+bxjMXcM6yJqhXwwV1t6Xz+Ct502zPB6j1jKgc+Am1ZZSJauizqAOqaIFiUrJ7DOzKXxQSHVONXITOZo6mgLzbctDAmYFYNfbDkWLgpivYwBh8Jp3J49dw3Yx/F/DCX85nFE/jeLk0pPIrYUiqMK8AqcpTujpdyJyQ5eFUvyomN3Ruxn+r+FYB1mTtD8JEIr60fdGY+JpwuFlh2muaWbvmL0Ezg0k7KUwFqUtwnuLN416jTi2jELHVJu042mUJZfh/Yc3iV8n4ix1pta8mnZtPeqL6zm59CQuQ12wDLTEfbQ7jeWNbArbhOtwV8qSy9DU1hS9TxN2J3B84XFmnZ4lAgqU7UoBjfwcSGXvmL3C39NFlshpkBNjfx+L00An0fMPIMND6AY40FnEri5ezd3v7nJ3fxz/mryBZnkzO4edp62pjexr2Vj3skbZrsSotAzzcnOsix3Y+stW3q5/u9twQddCl9KkUqQyKWaeZmjqaGLiYvIfMggiVkVw9NWjxIXG0ebYxns67wmeGc3tyI3l4pqYcS4DubEc+wh7zr56Fh0zHQZ/MBhtbWgwsiUtYi6DI83xnqSLmacZ6WfSMRpixDTLaWi1amFbPpa5ZsKan3wgWfz9S0Mv0etRL2yL6kSGcO/lvck4k8HTU0/Rr9dn+4LtLJwo+Ij6TfPDc6ynGmOnq1pA2ok0FK2KbhJuNbk16FvrYxVshe8UX54lPSPyViTWZWbwCWIDb+D7A7nxyQ3y7fKxqh6HhkoLcy9zZp+dLTLCTesjqTS+RVavmdia+pJ2Io347fFItaXU5Vbh9uBPzow8xbhLfkhzPYAFSLWluES5EP1tNG998xZKDSUZAzKYGTqT5upm4nfGY9fbji9uf4FVsRXyB3JKvUvFNU9DUwPb3rZYh1hzduVZtI20RRmxjlCp4L33NdAYvIhFH3YCLFRKFfd+uMc7nwsKAfr2wj3ZwZ67p3mXH/mWATlSFtGOtQKKMyp5AyjmPxclgEQbVBoq9GTCcx7zbYygeJC+Uqw1bn52k4r0CrzGe/F452Ocm5zBF4IeB3H8X8fRvaHL/Y33ybuTx5BPhqCQKOh3px8nJvzloaYS5OqflxPvaDrVFdTRUtciSmYalUl57F1Cg5ETERqagu9cQyVZQRPQKznFlUHHibwViV3FMyQhvXCLdkPZruTeR/eYcXkGdwfdFVQqgq1RtCnYO2YvLbUt9N7eW+23Wxtaaa5qRsdMR2xgKduVFMQWYBVoRX5sPu3N7WrD2Q5/o64R+2MsihYFW/S3oApTsXP6bqyDLPF/wR8tHS1Uf0lqNek2kRRSwwgnRwrjCpHKpFgFWNEQackJtw9o0mmi1MkDPXdbShLTOPPyGSbunMiRGUc4mnKUQKtAsa44v+o8KUdTiHgqSJn2su7Fw2WdPt+GdoasTF/JH6P/QFNbk5KEEtqa2nAf6Y6OiY7aGtc1egK51Zm7oPprmz888zANZQ0MWj+IF392IjZ8FGdHn6VN0kBFehNxv8XhOtSV3JM5jD0RQXbgBpIHVuHgrcfjXY8pbSjFqF2oF+ZcmIOpmynvNr+LhlQDZbuSiqcV6JjqqKH2gxcGYx0s+NZ3rTUyqzI5knSE6N+jubDsAsl/Jot1UZ5DHlOOTCEtNE3NK7ojx0o5kkLwomBk+jJhMKovQ24kR1sugQbBg3PU2VHk6CdDb2ENlJvIuR/+jBynHCLuRhAW10zNhBoq0itob2rn2MRjzGIWH7//MU8jnzLYajDHXzxO0LwgnAY6MXHnRIycOo8/5tsYLqy+QNjyMFHlQNGqoCqrqts56Ija/FpufHYDHRMdQhaH8GjLI2Fws6YfUm0pD7c+FGUcL70t2Bx0SH93jRfvvthtX4z+Nlpkq8iN5XiOVZdf66rEMPmPyShaepYMtPS3ZOGthRg7G2NoZ0jqsVRSjwtyvSYuJkS8FtHj51QqlSBF2uXe7OrlDMLzefe7u+TeyqV4zHpxwVQqBVuSxjwlOEGqVT+Ov/wqLdotSNqbkOjrsrpotfg78b/Fczb7LKPolFXvSdq76GERJ5ecRN9aHxO3f29x0HHNXIe7knMzh7aGNnq/0hvP8Z4cmX2ExL2JAKwpX6PWCC98UMiBKQcAkEW50CY3RKWQUFNUAwjWKiComYAnzo+OopuqAiazrnodTw494cqtK9xuvo1KQ0WfuD5EeUZRV1Qn+Mh7mhHpHUlQSRA7++0k7uc4YRASiih9CjD30ly1ATBAwf0CtoRvQUNLg/db3wcEOenYHwRp3ANTDmCHHebl5uQ2liDRkKBroUvf1X1pa2jjwaxvMXEaQZWtn3itEv5I4MGvD9C31sfyD0s052siuyKjKKiIwNmB+L/gj/8L6hYjUZ9Gif2t6uxqALGnAXD9o+uUPSnDe6I3k/+Y3CMA1SrAirWVa8m5mcP/w95/hkV1tn3f8G+GGXrvvQpIR1ARFRV77zX2EkuaJZYYk6iJiWnGVEtM1Bhb7L13VCyIoIAgIii99zrt/bBk4YjJdZ/Pc93Ptp3vlv1LzMAs1qxZ6zj28i8n5pwQJP1egJl1dHW0mGUAbt3cWFK8BF1jXdKOC7L/+mb6rXoefxcajYbs7GzMzc3/j37/f4qQySFUPq8kZEqIll+9makpBXl5//DOligEzMxazt+rrxdefb3YM2QPUj0pleaVzNk0h992/cbchLkk70vWAr3qmeph4mgi1jUbAjcgN5SLzO3nN55T+bzyb5VFKrIqaKppEtdlHbkOyIUeSenjUjz7eKJrpMueIXuoyq5iUe4iJBLJP0q5WraxFPMVYztjQT7ZR1sNqN7EFnMPc63Xmpl9Sn1jyh0DsDdVUZldiZ6pHq5dXcm9m8vZhWcx7G5I2PUwstyz8JyqXaMP2i2sG22utSHeKl6sTwFxOKin1GOtyVoC3whk2NZhDNkyhDW6a3Dq6MTwHcOxDxGA532/66t1b+XcyqEyu5KAMQF8a/ctNgE2TLsyjfKmIpJd1mLVxhiPJzKtewHApqoflUb3qDSrRN9en/RT6ZRllBHxboSYd1UYV1D/cT2eHp4cm3FMeJaGCN+vnqkeykYlGpUGuaGchO0JnF98npmxM0Wlo4vLL2Ld1hrnTs6cmH2CiPkRWsO+l/tUzRH1URTG9sai8hpAh7c7iOoMPrF/UOISysAxApj/ZQnilFQFOmqhf9Vsv1L+tJz9Y/bTa20vEZwQtzmOlP0pjNo9isLEQhK2JxCxIAJenE6z4gwIINjaolqU9UqsfK3+0Qbr3/g/i//tNe8/jX+Hff/Gf0U01TQR+1Ys2ZHZOBW5Uu4QQNWUt+i/9n/WwH25Od08UG8yssDmhQx6WXpZy9+pbsIm6w73B3xI/2VgbI/om3Hp40vErIkhI2wM+jXFrM7w54OJzzFzMaNN/zbY+Nlg42fDzW9vcn7JeVE2r+RRCeUZ5fgM9sGzlye5d3NpKG/QQsOdeucU2TezmRM/hyG/arO9Mi9nEnTlTxS6hjSER6Jvri+yRgAkMh2qrNxxfHwFhb4x0Id2M9vh2s0VqY6UlIMpoiRP91VCA99vhPDhFfUKwueGczbGCP9rmylyaw8IzT7fob50XtKZ2z/c1vI4SP4rmYe7Hor/f+mjS7Tp34aeX/QUmSl3N9ylOLmY8DnhAorMzwb/UdpNq5fj1UaBRCrBtaur1hBXR65D9JpojO2MufrpVaqyq8hWOBDSUUh2XpZJaI57ex+jo2ggdHIwbt3cWKlZKf5MbigXkUYgMJ7iNscRMDagFco8eGIwObE53P3lLk21TTh1chITYslL3ZrcO7lCw3P9Ldq/1Z5Bvwzi+pfXMXUxJXhiMHd+uUNJSgljD40V/TuaQ6VQ8VvEbwRNDGrlx9f8+XXkOmzrtg0zV7NWEnnBC4NpM7kNm503iyyJ5nh68Sn7R+9n+B/DmXFjhpaHxfhj41E1ql6bNFblVLEpZBONBuY8rHJl46ATvHNmsNbvpOxPEfw0EAaq9iEtw0L7UHuWli39x8a+jq4O8b/GE/9rPMvKl6Fj3jLsa7jUQPTtaDI9MhnYXbvhom+uj9tBN64dukYv9etlsa5/dZ2kPUlMvTRVZNToGutqodNfRuE1a/EHTw6m8IFQoP0datXE0QS1kQlGr/SHNGoNJakl3N96n2dXn/HGqTdea6Tt2duTBc8WtHo941wGcRvjiFjY2i9Dpi/D3N2cG1/feC3areM7HfEb6cezmGeoOwoyDGqlwO459dYpgFbsuxOzT6BnpoepsymVz4Qi+nVI3vRT6VTnVWsl7a1C88p//z+IZhbVuMPjcI1yxdDKkLgNcXj19SJ0WigbPikk/5fDGFLLxzXLMHE04dSeE2zd8wMxaIhAe9g6ESW3NTDkww+pqdGQ/3AB4U/2UurYgJWf4AUjN5JTWlfKT3d+osGggcq2leLwuHkvaI5mlqxXXy90jXW1BsEGlgavZSVqXrp+2dlg8+JwSX8lcXD8QQAOjziMRdObrFinLVOreNHY02vQw7xchqq2geBJwTiEO1B1uuXAt0N/p85wCp3cbJDpy8Q9pSRbaC6H7w7n86Wfs7RkKe3ntKf9nPakFKew86EwfWwu2GoKaoj/LZ77v92nok8FGV4ZpBimoGeiR8yaGNKOp/GJUhguf3zpYwrsC/j1j19pqmni8srLuHR2EWXQtm6Fit2ncLOtZ9TuUTTVNuEQJnhfDUucTqNePu/99B4NbW9yf6srN766wei/RjO/cj6mXwqFnYk8kv5eHpQ/LcIhzAG3bm6UPi5lY9BGoj+N5vb3t7EPtcepgxOKOgUmjiZMuTgF67bWFKcU4xHtIRaJbt3ceCf1HfKq8+A7kEll2BtrY4V7r+1N/O/xnFt0jsbKRvLu5eHew53gKcF49fWiKbaJQvtCrHtaI6mRUJldSUNlAxKpBAt3C+qMmtkhwvfWY3UPIuZH0FjZSNnOMhz6OlBlmom8vpKmWhUZZzOQSCSkHUvDfaE7he0KMbIxouxxGTmxAsDm3OJzxK6Lpc83fQieHIyxQ8s993LjOu1YGo8OPiJ6TbT4mveglsGsbYAttgG2PDr8iNh1sfT/ob/oFdXjcg8iHkRQM6xGlBbq/WVvUnrWUhUrgHE0+lIsPCxYUScwY88vO49S1sTFXhdxzHNk8ZT56Mh1MLY3Zvgfw7m74S7ei7wxczPjB5cfsPCyYMb1Gbh3d2fGjRnieTXVNLFrwC68+nvRbUU3Ms5l0FTTxDdLvkGhq8DWyBYdXR1ivohpNWDfP2Y/Ll1cmHRmEikHUrDwsKDHyh6o1aDSNaDa2hNTJ4j5/LIom9t+tYB4VugqqDdUiOvjgXGC513KW2qu214nOSAZv+cr6ZxeyMFN1wl7M4zRf42m7aK2lBiWUGVWhYGjsA9JZdLX+jw1x5VVV2ioaNAa9jX7bAFEfRhFbXEtvS70Iup6FI16wmDGIdyBt1LewtDKkHPB6fyWvIqO9yZjUvIUfXNPLQBLs6yRBCkSCRyZdoSG8gZWalZycMktDE4kUaufwKWeeXSRC94dM67PQNWk4smZJ0wxnsIOdpA6ORWfQT6UPi7l7IKzdF3eFfTAptgG9xPulEwr4eY3NylNK2Vm7ExG/DkCjVqDUwcnpHIpeffySNqTROj0UGwDbCkthSRhXk5dHdTnlCCRSjB1NuXC0gvCvdtpMsPmuwMCw2Ldl9/w+67vX7umZwJ/AK93L9KOHQYG1AbV45DnQLsybXn4/Pv5Yk7a+6veKOoUyPRlLK9ZTmpJKmwXBiqXe1xGkadg/o/zqcmvYUcvAQTibuFO5M1ILn10iZ5revJWUotcp4OxA6oMFeO2jOOO4g4d3+nIB5UfIJFIcHvfjdHfjSbfIR+HqiCQy6nKLaHNvX3ktYni4Lg4Ci1SWbB+AVJpPhUvTvv47ONU51bjK/ElNjKWPT57+KH7DzTVNolMm94VAgtpyLEhHLp1iJDxIRyddpTxx8aLsk9GNkai0sTvkb9TmV3JopxF4rlX5VZRX1aPjZ8NJ+adwNjOGJcuLuS3zyfHOYdi22KemQQSHID4HDYj/gE0qFCpYN/IfRjZGdHnmz7o/3QZ4wHG6CtcsK7uiUYjSHZNuTgFaz9rTBxMaO8oKJDk3cvD3M2cgHEB2AbbUq8Rmt9SiZTS9FLKM8pxjXLl3OJzSHWkKGoVtBnQRgQSQms0+9MLT9k7bC8Dfh7wtzYBzT6fGrXgd9PcnFPKlEz5YwrW0ljafjFTbIzvH/M5lsCDTp4YygNx9NHl7PyzmPqbMuvSLMJmhiEzkIl+6iBI4m8I2CDWE83RzPJqltJstkgI2xxGZWMl73V8j87DO5N9M5vaoloa2zQSGxlLj4E98BvlJ/jiXc2i95e9xXXIb6QffiP9qC2q5Vu7b8WBm+QFs69Jt4kK8wpBFnHjXazbWtNY2UilWS0ZbTKwrB1AmroDPcvq0TfTFxiYXVwoflaMRqLhQbcHtPVry85+O3GOcMbGz0Zskq5zXEfEexGiHG/z/QmCzOovbQWFnXYz22FkY6Qlj1tXWse9Tfe0vpvQ6aH0+boPm0I30VjZKA77UvaloGemJw77js8+zuPjj1mUt+i10rUvs2D+p3g1zy9ILODotKNEvh9J8KRgXLu4olELMrXxW+J5euFpqyESCM9T7LpY2gxog1cfL6Zfm/63f/PJ6Sece/8c7Wa1w62HO3H3wag8B9usO5QkRjA9ZjrD+9UTfH4d9X6+lAwvYfjh4YQmfo1y2nJAl6yrWVxddRX9zvroqHSo2llFdmO2VpP65ZDpy3Dp4kKHtzv8jywmgBE7RjBixwit16x8rNBoNBybIYAfbINsKXpYhExfhkOYAxc/vEhTbYsEo2vSacyL0il2Cyd0eSgWnhb0/rI3eXECk033UCW6DVVIXwL8+o/257bLbYpPCPWwxFSCrrEuVt5WjD3QwiQ1tjMmdFqoFgumoVJorEvlUpHRnR+fz8l5J4lcHCk+My6dXci/n49aocZ3mC+mzqbom+nT/4f+vHn1TZ67PsflzAAU4xSkHkkldl0sw/8YjqK0Gq/SA5QUpSNRq9FoRtLx7Y549fEieV8yNbtqaB/XnjUr1tC4qgV492q8DDZ1j3an/4/9tRRCeqzuIcoYvjyoTt6fzJVPrjDktyEiM9Utyo2BPw8k5WAK+0fvJ2pFVCvZWxDqw+bnr1kd4mX/ylcVNP5vx5GpR9A10W11Lw574w3+/OILJjY0/I/H2IqMTgE9ubD8gsjMBRi6dSjXnl6DM/DY5zGDwwZTW1jLidkn6PphV62e0svh0sVFC5h9a/0tHh18ROD4QO5tvkfS3iRG7R4lDvFPzDlBdmy2KDOfczsHPVM9EnckalmrtB3eVvRT06g15N/Px8DCAAtPCxoqG3hy+gl2wXYi6Ks5bPxtXmur8KTjG/T5CtFmJnJRJKDdV5VWlPG96wa6fdyN6E+jqcmvoSipCFk3Gb0u9SLLOAumQszaGMxczbRUhyJjI7mbfxenCCfKn5bj1ddLrB3zavNobNuIjq4OyfuTSdyeiFt3N6I/jRZtEUydTVvtzbHfxZKyPwV/tT++w33Feq1CUUim3fdkTdRh0D0lOnrlJO9PxrWrKyYOJrTN/Zwis1Oc7X+WhRMXcn/ZfR4dekTEuxEi+KhJr4mbrjdZO20tIVNCkEgkPDr8iHub7tHlgy6cW3SOhsoG5j+dj3MnZ7p93E27n3Z0PHJDOYZWhkw8M7GV6pNGLfQNX5ZS7fx+Z6rzqsm/n4+Nvw0yPRmmTqas1KxktWQ1pqVZZISPJWhWawB1XR0oTBSc7n8ajyIPChILsPCwYNTeUVpqBFU5VeTH56NsUIpWTDKD14+A1hqvFcDkwMeKj5HI/h32/bfHv8O+f+O/ImT6MkI+CkGVqcIh9iYaiQ6q8J6YOMK1NdeI+SKG2XGzxQ3OIi+Zh3MuE7hn6AtjViGah306igbqStUYWhkyctdIRu4ayfUvrxP7XSylLq8v7IztjfHs60V2TiVOj6/QYGzD6XdO49XPC31zfZ7FPCN0aijWAXa0m9lOTIbOLhSkO+qKhabe5Y8uk30zm+XVLbJcGrXmb1lpf/b+E4DngQOx7haA0Su2acWJeRhV5KKnW45Cv0XvvnlIMeXiFJoaNDw8lqXlC1iRVUFDRQODNw5m38ASjDMfIlELTK7Ln1wmdJpQLPX5ug83193k2qfXmB4znejPomk/rz1pR9OI2xRHzOcxRC6K1LrOemZ6uHR2odOCTq+lke8evJv8+HzezxNQhaWPS1HUKURWma6xLpPOTOL5jeecnn+aDm91wNrXmm4rBHZY2Kww9u+Hr38AT0/44YfW1y01Ffa8fRPjhmJCJ7ckHxVZFRx84yDtZrTTGlxsCtkEvECouZqhUWkoSCjANcoVYztjoj+LJurDKPLv5ePcyfm1RVq7Ge0InhzMzn47xc0y5vMYsaGdfjKdsvQyETFdlVtF5sVMQqaEINWR0vvr3q+VjQQB0V+RU4GqSSUOz2qLaknYnoBLFxc+K/2M3+7/xqfzP6WHWw+t99r42xD9WbTguxegfQPJ9GRiUnrn5ztc++wak85Nwj7EXkQB6tVXYFL6jNLMbEB72BfxXgQePT2QG8q1ZEGS9yUTNCEIG38bbv1wi2dXnzFm3xgt1pmNvw3zHs4j906uIInyiv656V1Tul/rzpoVaziy5kira3Is7RiDdAYxJW4KhaGF4sAlaW8Sjw49Qt9cH0Wt4h8Zn7VFtWjUGoztjdHR1eG9jPeoK6njGxuBAdV5SedW8rIlqSUk36xg3QE3QsLlfPZZy89UTSo2BGwABBmfV/92bXEtv4b9Suj0UKI/jebViP40msj3I3l29RkPjz7E5xcfDC2EtURRr6C2sBYTJxMCxgVoSZc2R969PA6OP8iwbcOw8LDg3JJz3P7+NkO3DsUl0gWpTMrTi09x6eyC3EDOiJ0j0DfT13p+Xxd3f7lL+ql0StNLCX8zXGsQX51XTcwXMXgP9NaScvvfiNQjqaQeTkVuLKc4uZiJpyeKzw8Ize2kvUn8NeIvpl6Zint3dxZmL6Q8s5zP9T8HQKpnzDNPYZhVUFDAB8uWcU2lovU4VYgI4HhjI1Erl/CGbSEnxlrgVN6bwImmhE0PASCzMJNHJY8A+GnATxiaGr628KvOq0ZRr8Cls8trDdJB+F6vfnoVpw5O+I30o7k2tcx9yJPNRYRt7kVxSjGphwUfrqx2dRTaFWJcKPiyZsdmkxObQ8iUEH7v/Dur0ldxYtAJBp90o2liBiN2Co0W1QXYO0ufVGehcWukGYrMVLtB1dwEyTTJZFD0IC3mydknZ9n5YCcDvQbSd2VfTj86jUdPD7HJ6XrelajaKHapdvFLn1/o+G5HfIe1sBzXxAgegO90fAcvvLj59U06vNOBtsPaUlEBhw+DT0IJeva1AKx3Xo9DmCCRWWNbR62kghODTtBVNgWprEqQr5FAY0Mj8iY5SpkSjUSBRCKs575DBZ8lfXN9TJ0F/9CF2QtFuZKChAKB0RebjXVba2z8bcThazM6ssM7HbAcIdzrIQ9C+Nb+W0b/NVoEhDQoGyiLLmNh6UL0ZHocmXqErCtZyA3lNFY1MnfjXPZM2ENycTK+fr5ik7yhsoGvZ31NW5226Kh08Iw/Tk3hQEydTEnYlkDq4VTOfHqG+Pp4Fvy4Go1sH5Zeb/JR40c0VDZw9+Bdxt4ZS9GJIpR3lNQU1HB2wVnqSurE5k7OrRytptarUfigkMQdiXRa1InwOeFk38h+LYrz6DTBc7UwURgsrlizAolGQpVrlbgXZZzP4Pb3tyk/lc6I4BHERMUQf7SSCS0qo5g6m5LvUIhGouGJ9xPBB/dFSKQSgicHo+uji0QioevyruL39GpIdCSUppfiEC40uh4desS9zffwXutNSmMKM0KFwaBje0fC54ZrsQUGbx4sDuAXPFsgrs8vD9jVari3WWgeB4wNQNdKl/an21NoV4ihQhepcROgS/CkYPTM9LDAm3s5T8hw3k1ljYTG4iqS9iTh0csDj2gPxo8ZzxcxXyBRS8QhR11JHRVZFVh6W4qI3uT9yeTF5dHri14M+GkAqiZtlkjmhUwAen/dG6mOlB89fySqJopbEbdQ2QpJ0Jn5Z0jakyQwvwwlsBS6X3JFo3sF0EZf6+urMaoxIvB2LjreqaIMOEDKDxcos7Xhmfsz8p1q6dHUgvwtTilm77C9eOIJq8BILlxPUxdTpsdMF5ofOwSPul4xvfAK8SL9RLp4rxwcf5CaghoR7JK4I5HYdbG4dXPDNsAW1YuPbf8khsdHLIlZeRE9Ez1m3ZlF9Jpofj1iR7W1J8eOw7jxsLH3Rr5++hUxqF+7ps8CPgNuw9+u+bz4+ZWmJgiDIbuHcP7MeYKeBYnI9+YmG2h78uga6aKuFu6jHJccclxycNdzx8bPRksO8dDIQ/Q724+EpIRWTVSZVEaTThN6NnqiTGHzs1hsVEymp/Dda6qEwZixvTFFbh1wfBLD6D0+/PIW3Orti0PFG1ggDNarc6vpfbA3XR8Ke9/+lP38MOAH5AZy/Eb68fjEY/GzGdYZUldUh42fDRHzI7Qaxi9Hlw+6oKjV9mW7tuYa9zbdY0nxEp6ceoKFpwU91/QkNTOVsnwBTPnoEQx5CcfYjKQ3qTJh9G4/Grrcov/H3ZAbyjGyMaIp2JYyyzJMqs0Iuvg92WbtMOzfvZViw/Prz9kzeA/Dtg0TvUtPPj4pXr+E7QmiNOXLA6FmFPzfhYGVAW7d3TC2M2bf6H24dHHRyvk97h9C70YOrH2P0X+NFs/lF52rxHY4z8SdE1FLJFr5dcQn/dh64Hti/D+j9/00VA0ujNo3ipySHHR0dah4WkFdSR2O4Y7kxeVhZGuEsb0x3T7p9rfSWlc/vSr4qSe/hY2/DZWNAmDrUtYlfjj8A3Gb4zg59yRFbxRRK63FqKcR1r7W3P7xNnEb4gRWs4keOx/s5Gb2TX4a8BMyAxkd3+so5oSWcmEgl9Emg4w2GfT2ncupgYJKRFVOFV2vt+V6F5AYDKTaaQB3flpHTUENTTVNzLg+g4SCBD7Y/AGNykY8enqwrGIZMn0ZF5Zd4PYPt5kZOxNzd3P0LQRP9ge7HmjJeBpYGhD1URSG1oZs8N9At0+6Eb26JXe2DbBlSfESsq5m8XDXQ6I+jMI20JbiR8X0+bqPFvts7oO5WmA0IzsjIZfVQHlWOTI92d8yPJ/feM7Ofjvp800fOszrQFNtE0enHcWjlwft57anKqcKlUIlSuhr1BoaqxtRNalE3/qkvUkcmXIEn8E+TI8RhniPDj3i6uqrDNwwENcurtQV13Fr/S0MrAxasZ4BDk06RNrRNJZXLydhWwIgSJACtOcaec4qrHJ1qM3zBZzQSKU0GlrQKK3DuNqY567PcSybhKpewcM9aVRkVVCQWEABBTg4O1C1s4oM0wxcIl249+s9FPUKLeahjb8NM67PaHVe/2lIJBI6L+3Mza9vEvRGEMdnH0duIGd6zHRi18USOiOU4MnBZJzNIMM5GJVMj1pzR0ImBxE4XlAKKs8sZ9+ofZgFaHgcORXLF2VJc/2XNzOPObVz2PD2BgoeFFCuUy5+PwDzT88nrTSNVZ+twtPZk0MTD/H4xGM+qPxAzOdj1saQ/Fcyvdb2ojq/GkWtgoAxAeLPN7fbTGN1I+89eU9k3US8F4FxVTfCzqfQ/VpbKldV4j3AGwNLA3yH+mI/OYobNw9hXPcAowZn1GpBvtW6rTUunV2IiY2hQb8BpVyJ5oUkYnV+NcUpxdiH2GsNu5vDoZ0DDu20lWZayWO/dO0lOhIkEgk1hTU8v/4cx3BHkvclc2/zPdx7uGMTYENDZQOH3jiE92Bv0epCUaeg/Gk5xvbG4nm86l/5/2WUppeiI9eh8EEhJo4m4jnNevNNPlu9+v9o37+uo2F8Qwg3vrxB77W9ubvxLqmHU3Hq6ETy8WRMe5lSM6mGwW8Opqm2iUlnJ7VixL0cr1q0dF7cmaA3BFZfTWENxcnFWr2BkKkhuPd0F/9/V/9dOIQ70OebPtgG2YpAgpdlZFUKFVvabyHojSBG7hpJ5fNKDk44KEqqP9j5gNPvnWbM/jH/ozrOk7NPuP39bcLfDEetFvJRo/Ic2tzdAxERRCyIwDnSmeyb2dQV12G/yZ785Hw2zdnE0VlHAaHX5dbNjeCJweLwbM+EPTya/4i7G+9y+/vbLMxeSKhjKDHTY4jaFsWGsRvoVtlNBLGCoOL1T9FpQScRBDd0y1Dx9csVfwDgl+KLSUkWWZfLOTbzGG+cekMEcpjVtsfS0BwLAwvCPgsjYr5wZzTnIwAqjUp8PgAK7heQcS6DjHMZdF7SWeztuXZxFQflzfEyQOJ13pVrTdbi1t2tlUdj7PpbxH57k/eevoeFhwXnl56nvryenhtHs/O7ItQ6cjEv/tzgc7wHeTP2wFgkEgkaqQaLcgsyb2dSkVWBfYg9geNeYQB/1lNUKoPWgO6kv5LIvJhJv/X9CJ4UTPnTctyj3VspT/wb/53x77Dv3/ivCJm+jKBpQTjZOXH6viMlzqG419ZQVyrF3MMcj2gPdI11xYaNRiJFotOCvq4rqWPKxSnisM/r3j5+8s5nWdky8W90/aArXZZ1ZegQDWaFj6lKNabMVp8Lyy4QPDmYswvO4j3ImxKXdlRZe9JkYMbEMxMxsDDg8cnHXP7oMheWXiB3+ocYmnox5MXeOuS3IVz79BrZN7NJ2J5Ap0WdCJqkTeUftGEQSrUSjUbDja9vgKZlUw8YG8CluwKd3vbF+d/fep+i5CL6retHl6+GcOoDNU2G5i16Ry/FyXknKSyVIcnL48GPgmffgXEHSN6XjFQm5WPFxzSaWJPU8z0AaouekbI/BQsvC+rL63Hq6ERGkSn1Vk4okWHnYYGFhwWx62Jpqm7i7dS30TPTRqgHTwzG0suSzMuZmDiZkBObQ/L+ZKI+jMLCwwL7UHst9NeJOScofFDI0tKlWscpSirizo938Bnso+0hAZw/D963d2F8+jn8sJykvUmYupiKm++ZM5Dj1wfpC8+YZlkOhzAHqvOrqc6vZnv37Xj19yJqeRShM0LRN9OnOreaHz1/JPqzaC5/fJlJZydRY1tDXlwebYe35fDkw9iF2DH10lSt83l+/TnborYxYucI3k55W3x91p1ZyPRlxK6PJf9ePnMS5lD2pIz6snoOTjhI+dNybPxtcGzvSJclXUg5mMKF5ReIXh2ttSHH/BDDve/ukfx9Mvvm73vxXdVyYdkFuq/sjsUxCybUTkBqI8WylzbCzcbPBpuPbNBoNCgblUikElEusyStBEWdAod2Duhb6GPlayX+XQNLA6Zfn86quQXUWLpi0UNbAgK0pSKaozCxkJg1MTi2d8TG34b8uHxSj6SiVqq1hn3N4dTRSdTZfzkaZzSyLWAbKh2V6NnTHIo6BRW7KpDlyHgU94iAqABx2FeUVETK/hTeSXunFVO2IquC3YN2Ez4nnIj3ItjRawcqhYp3Ut8BhAR6e/ftePTyIPNSJrVFta3OK25znJAYA08LRsJnLc+zREdCt4+7YRdi91o2q0QiwcTJhKbaJhK2J+Ac6ax1bxtaG2JobcitH27xdM9Tmr5qEod9Obdy2NFzBwM3DBTN4V+OzMuZPLv6jKFbh4oSi/Yh9viP8cdvpB/6ZvqcX3qem9+0JJUvG0X/U/T8QjCOvvn1Tbz6eGl953Wlddz95S6GNoav9R/8fxMxn8eQF5eHR08PilOKxUS7OTov7ozPYB9y7+aK19HI1giNRoNHTw+S79aSHdCfenuhCP9s3mqiVJJ/LP5AKA67osNN3cOk+z/B5t5CrYFAczHjX+HPHw5/0GN1D5wjndk3ah+9v+wtNh6PzjgqgDuqlrf6G4o6BZdXXsbKx4obX96g3ax2+I30o+aF5YZ5/iMe/foINvci7VgayX8JsoHG9WboNeohq3lIxbMKnpx5wrVPr+HVz4vy9BaPiRyfTnjZvgoeaClq1JJG1GphHdk9eDeBEwLJqRLYYbGdY7m58iYgsDpz7+ai7iq819JQ8GZtRmFbtrEUvS0CkwLpcrMLaR3SRC+qlyM4MZjERYn4bvDl3Sfvioxi5Yve3uPIKXy6V/h34IRAkvcls951PY697Hhml09chzg6FHoQMgVCpgiD1x98fmBF+gq2T92OU0YOOJQTuTgS2wBbkv5KouRRCRNPT2wlLesQ7sCgjYNorGqkOq9aq8mnqFdQV1onSEG+QKIqDZRYtrHUGjav7LmS2zq38Xvfj42DN9L+rfY8u/aM9FPpPL/xHLmOnD7n+5C0OYk+GX3EorOxqhGdAzr0tOlJmm8aNrmPaKzqhdxATklqCTUFNdSb1tOkbiK7jQMGyhaFAn0zfZ4eeMqsc7NY8/EaFGoFJg4mYtMZBObwqzI21XnVHJp0CJfOLvRc05MuS7sQuSgSmb6MqBVRlD0po6GiQQR5JO9P5tKKS4RMDUHZqMSzjydoINMjk0d+j1APUItNiJ19d4p/J+RBCBblFujnJdD0ST+yrmRh2caS9nPbE3f9fUyqTag3rGfzZpgzR9ibD08+DMCkJ4JXT8R7LU9o4YNC0o4L95OVtxVyA7mWN0nX5V1JP5nO2OVjkVvJ0dumR318PT6DfbTQ7IDYcIEWtjwIAz6j8mza3thGmWkf3Lq5kXIgheyb2cg8ZQw+OZh8+3wcChyQvpkPuDHizxFsDtvM8/uXcCWPQisJOXUf87H1RsavHI99P3vubrxLp/ud8FB6MHXbVBL1E7Ffbk/asTSOzTzGxNMTRbZd+ol0Enck0v2T7q2aCCCoF7zMknfp4sLNB5nk+ETiVCWAeCy8BJ/PzEuZSOo0fPTZR+wbu4+QZwfJvpnNH9F/0OfbPkS8G0G+zm0sGy1wTy+hziSHcYfGUZZRxu0fb2Pd0ZPkBuHZlKoNRGnS1COpIushr0ceBnUGtP25LQkkEDotFNeurmLjpFG/Eb8wP/QM9Ri2bZh43nWldah0DZg9Gz78UGAUuXVzE4ewzWuBY9oVUvZ602lhJ3R0he+q8/LOjCkZjrQ+gYBna4BpZPlk0iNbToTi9U0ie+BLYAhwnNc3/m4DQ/T0+GLNGqrCq1EbqIlyFwac4bPDce/uLsqfvhxqpZr8+/lUqau0XpdJZcKwViJI+BUlFWFRbsHuN3ZTsKiA8sxy0k+m49nbU/Do8p/CmeIyknuNYP40N3Ju5ZBzKwefwT4U/lWIQ54Drs9dCUswpaFzLfpmRpQ5+mP77C42RUa8++O7XB0qR9EoPPNNNU2Yu5sTOSKSmPYx9NnQh9GbRnOt/hrdPuomeqRllgtDxL/G/8UvS37BytCqVS6mVqp5fPIxZi5mIgv75fAZ7IOJgwkyAxmLcheJEnDRb0dj52eHjkqHRvk+jj7S48HOB3xQ9QEa3Ree1xoJ+g16qBsVhM1qGSio9SWMPDSSP2ZcRKlrhFRfmy3y9sm32RC3gYUuCxnxxQgthkvz3hyXF8cKxQq+3fgtxnbGGNkaUVtUS9SKKMJmhRH/ezw2fjbkx+cTtzGO8cfGi+wQh3YOTDw1EbVK8L97GYkPoNAzQqJrrvWaa1dXnnlWA7Br0i6GlbxNbVEttUW1WHhZ4D+zE/ulf2FRZkHkyZ001XTB84NopFlSaotq2TNkD6VppSwuXMzWLltpN7MdQ38bqjXYao7js49j6W3ZYpHwcg2mgQEfD+Bk0km6LO3ChBMT2PrGVvoE9mG73XYWdFpA7y970/2T7mJjfPLhyfim+vLjph+Z8csMrYZ9X6s32V+4puXwuhKmXJyCkZ0RZRnlpMY8JjgxGAyEtalZGhIgvzqfdptfeAMe6sShO4fEfcqtmxu3f7jNvc33mHlzpviekMkhWp/V0MqQnp/1pLaolvrS+lYDDKlMiqG1IRq1htTDqQROCCT1aCoxa2JwjnTWOvarrKOXG6C/dfxNlIQTX4v4DfswewZvHIyeqR5u3dwwdRKeMbVSTcqBFPHaH3zjoPj9gXAPvfdEqKszL2WKDF+AwDcCtZ6zptomMbe18bdhYfZC9Ez1UCvVXPv8GvYh9qJKkbWfNfWl9Wg0GoKnBHNh6QVMHE2ozhPuvSS/i1zt15v+3QXZd6MKHdK6zMAp42cWr1vML+8exUEzEkVVKYfmHyJqRRSLchfRYNhAil8KV5deZcHMBYAAeqkvr28lM5pyMIU7P96h/4/9tdRcXhfN3rU+g30InxOOa5QrZxecJedWDrPvzSZ0WijG9saYuZkhlUnRNdZlftZ8cfB/NOkoX/++GIvaLvgWjOLU20dJ2J7AnIQ5GNsZM+CnAXxzQLgnFC9wCMffPI6+uT7KLCVWUmHdzhqXxdmBZ+n1ZS/u/nKXoIlB/HjnRwASChIoWFyAla8VLmXaoDxVo4qmmiacOjix8PnCVp+v89LOqBXaoM6m2iZs8w15YlvEvYgmDK0NsW5rLTKWnvYp4k8vwRN2cJzwvSsblcj0ZBhaG9LnTB9+/PNHfNN78dmiclb9YMHT8085MvWIFuv6wPgDPDr4iI8VH//jdwDwrd23ePT0YNSeUfiP9hcHJk/OPmH/6P30+LQHMgMZA34aQPiccHTkOtQW1ZJ5ORNL75Z6ryChgK1dttL3u77I9GVkXsik/4/9CZ8b/tq85XVhYvL6gfr/k5gdN5vMS5lsCtnEwA0DxaGkvb09KxZ8xJAf13Jc0fSP+/6XX3zBhDcmCM+hRkPl80pyb+diZGNEXUodm3/ejLmDOSCsIV59vah8XsmmkE2YOpsSsSCC/HhBbvXVvBsQGcwAPVb2aCUj/HJeCtDz854Y2Rq9doDbHDpyHXqt7SWCrM1czBhzYIwIGDewMsChnQN6JnoUJRdxa/0tUT65OeyfXCdhuzFdl3UlbFYYRnZGaNKFn6l15DQY26BnZU3/9cK9kvhnIgnbEzDyF3K1AocC9F2EeuHNu29q1UYAxbbFWPlYETwxGIcwBwwsDZBJZVgaCPeTWqPGe6A3d3+5S0NFAz6DBYWI9XPXM3jzYKrzqrn7y11G7RklEgf+DjTbDOIYu38s8AfyhWMYtXeUSCJQoyTo2UY65qfQ+EclNh+21MaiZ1+FGT2+7sGZB2fwG+mHhZcFUSuisGxjSUlaCT1W9dCymml1Cq9h7r0c/qP9sWqrnUf+1GEHuU/qCZrTU6y9Mi9lkn8vH367z94lR+l22Yu/hkaz4O5beA/01pLqRQOdbneixsIZx+7/c/+lKqeKmoIabANtRRuB7BvZxG+Jp8eqHgz/Y/j/eIx/4z+P/8017z+Nf4d9/8Z/Rejo6ODlJSDcCj0FdKX5H19wKNuVSWcmibTx5mZshYMfAav9cO0qMFKaGyTNfmMVtj70GyBsAM+vP6exuhHvAd5ik8H77h6yTPwJDIjk0aFHOHVyov289lj52aA6pU+9XB8XlxbtZCNbI0GuM7eee4VSNMWgUoFMBmEzw3Dt4sq2btuoLa4Vm8AvR6OyEf8N/riauTJmxxg0ao047Bv912j+GAIWuUkYpDyg/K0BpB1L4/GJx/Rb14+4ry8TfDmR+P7LUcuEQubA+APUFtUy9dJUnDs5k3RWgi0t2uXNxXjk4kj2j92P2VMv6hwElptbNzc+UX1CxvkMdvXfRdsRbdnfMApNQCA3UmG4VxNowKufl+Dl5GtN+ul0rq6+St9v+4po0CPTjlCaJiCuGiobuP/bfcJmhmHhYdEK1Rw2O4zawloen3jMniF7eDPuTZL3JWMfYs+i3EUYWBlQW1TL3uF7cYpw4vb3t7E1NqfIMRSl3ACNRsPBCQfxG+WHaxdX7uffZ0f9Wsztv8CoUWikPb/xHIDK7EqWFC1BUafgzk93sA0WmmTDfheaUalHUjGyM8JniI9YTCdsS+DSikvk3cvDf6w/iloFJ986Sb/v+ombZbNPnrJBiapJJQ7MmhMv5whnIuZHYGhlyOn3TpN2NI3Z8bMpSS3RYj+mn0gnYXsCUR9GaQ379Lrqce7hOeLzWzxILLwsmHV7FqYuphz9/iietZ4oHyvJOJshFgMvR31ZPd9Yf0P4nHAGbxIYemfeO0POrRw+qPyA4InBWhIM8AK9lLydCjtfdPqNa3XMZu+45pBIJPgO8+WtlLfEonj4juGM+HNEq/fWFNSQvE+QWXjV+w5Az16PEpsSul3txtUvrtJjRQ/xZ41VjUQeiSQhJIFeN3vhHdSS5PRc05Poz1o3SJrP92VPguDJwSLCTlGnIOGPBIpTivHq58WUC1Nee4zAcYEk3GmiJLUUha42M0pHrvNaxl5zGFobMuvWLJ5eeMqfff5k0KZBWsO+utI6FLUKoldFE/VBFEZ2Lcc3czEjcnHk3xbXzT6bC3MWitc+dFooodNCUdQpOPv+WSRSCYM2DdJiu9QU1nBr/S0U9QqUDUr8R/u3QhTbh9gzePNgen7eU0sWEIR7/P2C9ylIKODOL3cInRr6j/5a/0mUPy3HIdyByecn/60cbDMiFoS1reRRCfrm+ky5OEVkEzRbY12PO8fSV2Te/i5momROwxNMK02RNdWQdiSTnOuZRH8aLd4/Sn0lbt3dkOnL2NZ1G4BW4y1gXADOnZy588sdErYmMPbQWFHaQ9UkeMaFTg/lncfviIOv5mFfduAApvwhSNT6jfLDzNWM+vJ6Tr9zmnH143B/5s5V/av0WNWDNv3aYOFhwdzSxdj/JHw/PpVd8bQ25OiMozhFOIEmnIH7m+hiMpO7He7S72QyFYGhqANsqC2sfeF98uK8bVoGdY9PPiZuQxyGJ4R7RiKRMDehRTZ59r3ZlGWU4f+HP4Z1hoQkhjC7zWyta9l8vVyyXciMy0T1g0pLdudlnMrJkzB2rACCsfaz5s6Pd5iwvT/x7WxJC49A80rmWpFeAQiygUF3MpF45dNnt8DGPbPwDPd/v0+P1T2Q6cmozqvG0MYQfTN9jO2MOTlPYIEYWhvi3t2dWz/cok3/Nnj28mRB1gIA0koEX5JEr0ScPnHCqU1Lk05yV4K/qT+Xd16mpn0NzhHOIrOusbKRC2MvYFhnSEeLjijrlST+mYhdkB32ofbkfZ3HnyV/IsEGPdOjWHgZcmjCQZL3JbO8ejl/ffsX+g36ZPq7Yl0dQWN1I9k3srHyscKysyUPCh6go9Th+tfXcQl10fJGDBgbQF5cHkenHyV8bjjOEc6UZZSRdTmLrMtZ9FzTUxig6Oqg0Wj43vV7QCjYmwtJqUyK3EBOwNgADKwM+N71e7p93I3dEwV/v2Ba9ooOb3cQfYGPDTmGUkdDuMkbVOdWs2fwHrp+2JXO73dm9L4+xIVbUGFeQcbdzTSMm0pdSZ14HGdbZ3ReMYbNj8/n8keXsQ+1x8q79cDF3M0cEycTqnKqUJQqsHJt/TsNDbBkCQQFwezZLzwUbmRj7GCMpZclGg0o5QZU2PtiZ2PJmO8jebj7Ibd/vI1dFzv+mPIHeo16eDyPpIeN4B9S8ayCw/cPcYnzRKHDplIN9lRRkF3Fjq++5PM1nzHAeBDtKsOYP2w+mv4anPyFe8exvSO91vbS8lDp9WUvuq/s3qphIt5rr6x/k85MYuOE6xjWx6FpuAX0p+uyrnRd1pXqvGq+c/oOmUrGY9/HuNc5om+uwquvl7j+9Gr8hUsWb/P9wm1MqF6O9yAdkvclc3jSYbxn96Aq9RZGNUaMPtgfPYOj8O0wTr19CjNXMwyuG/DrhV8xqTLB4Z4DBQkFIntFLRX2U5lCRl1BHaYOplqfqb60niKZOaYHtvLbCViTN0Nrv2jOwx91fZPZa/VwCmxhYjyPf06XS7qEPHiTegNhUHU/6zoL/mbQ1xzNLdoooBMwB7BD8PL7Q1ePayolX37xBQsXLeLDix9y2/s2UV2EYd+VVVfIvZXLxDMtaOw9Q/eQF5fHvIfz+K3jbzhNdAJvCEgKoOOdjnT7vhs/+/6MgaUBM2/NJPVSKtsObaPesB49Kz2ybmdx+t3TDN8xHOu21ugWdqG0Zje3ak5RUzOPJ2efcHXVVUycTMj8KJM5zCG9TTomVTooG4XrW2PlTmbocDwSjlBhXoF+nQZZQw0SjDB1NmXCiQlUPaui6W4Teo166Ch1UCkEllFtcS1yQzlnnpwRP9PL6PaXQ9mg5K/hfxEyNYTh24e3+rnPIB8t0JBEIiH7ZjYAtUa1WJRboJGosA22xXugtyClqRG68lVmVdzvM4Mu4dp+TypzGcW2RRTa3GPznA3s7Hqf5H3JHBh3gDH7xxBbKnitrs9ez3crvwME+eLilGJM1rc0VW4b3cZ7kjf6xvqMPzZlmiQ2AAEAAElEQVSerMtZ+I/xp7G6keOzjtPh7Q6CpOArIKLmkOpIWalZKe5fzZHj3w+XF0S/h7sfIpFK8B/jT7ekIJ6YSpAYT8DWMJLEHS22Dmo7oeHbqNdIUlAeoc5OYm1755c7lKaV4jPEB10TXfqu66vlq/dqJO1JEuWIA8a07NWWBpaU1Zdh62BL3IY4XCIFL3mDKgNcn7sSviCca6XX6LaiWysZYbtCO6ouV1H/Wb3oUwcgQViPrYut8XvkR6NLDR4zBGljtRokGjCtMsUs/xR+SUnUFM4V5cp/Nf2VVaxi1apVmBSbkJ2dTX58PmZuZngP9MYizILD5YfRf6JPvzb9KEouoqG84bVKE0a2Rq/NrZWNSkofl2Ld1prJ5yejbFDy5LTgKxs4IZCq3CpMHAQ/rYLEAuQG8lbeVQAd3+vYym9b1aQSvaztguy0GBl6pnp8ov5EBCEGTQwSFXxed+6BEwKxDbLFxt8G7wHeKOoUyA3lonxqc+jo6ogDA0W9gqurrhIyJUQc9jUr3AB0WdKFLku6kLA9gaPTj3Iz8iY3u9zEocwZDRJOvnUS5wpDqq3csH9aSqZ7JhqpARIkSMxMGHtoLFY+VkhlUlzmu7C3YC+6HrqiR9bov0ajVmmvCzWFNZxbdI7K55VaDMy/i7s/C3nB4/OPeXziMbPjZ6NnpoehtSH1ZfWce194bt9Je0dkwZq7mdNQ0cB3zt9RnVtN7ZIYip3O4l/8Mfnx+QBsDt0MgO8wXxpMOmJa9AS9MhXgy/A/htNY3cjRT45yus1pACTTJQT1DqIis4K7v9wVlQGsSqyIPh7N09CnojJJU20T55eeJ/1kOm+ceEPL8gNe1NI23xA6PVSU1L3+1XVS9qUw+fxkvnP+jnb1Rpj7f0lhm+4YvFRvVT6vRPd4JY7mjuQ55aGWNKHR6Ar7hYUB4XPDeX4jG317fSbsiqLI7Qb8MBinCCcGbhioJc/nEOaAqlEAw9756Q7xW+IZf2y8yF7cP2Y/uXdzWZC1AIdwB62hXXPYBdsxcvdI1Eo1Vz65Qt/v+ooDDSNbI1GGvTnMXM3ovqo7zp2cSdiWwKNDjxi6dSiDNw5udezXxcv9vP+tsPC0oMenPVoxoM0PW9LdZghd8w4ThQ4zUYj7/na5HtcUTXw4dxnz31ugBUTuvbY3vdf2Jud2Dtb+1kR1jSLmixjOlJyh/3pB0ryupI7CB4UUPijE1NWUe5vu4dLZBVNnU66suoKhtaGWZch/Eh3e6tDqtdqiWo7NOkbghECCJgRRUChB2q0rPi/2IX1zfS2AsfcAb7EmeHL2Cfd/v49ThJP2sC/jJok77MSB+8tRb2pHWudpBL1Egg8cH4iZqxn7ru5DqpLiqueK8wvm9+tUqfQa9GiobMCxvaPWgEoqkSJVSYk8HUmlayVTLk7h8YnHXFl5hfLMckycTJDpC4CpxurGVvtvc5xddBZzd3Mi3ovg1V+R6MoIGN6Sl8QEhFJtkEzgzh+IOVtDl2VdBAafVIKLqQtbhmxhwZ4FVLlWUV9az/bu2+n3fT86ze8kAktfjgc7H3B/630Gbx4s1iU/tvkRAwsDxhwYw8++P9N5cWd6fdFiM/O6QVpeSgV1Fs7szY1i4otUd/q16Xxh9IVwDWtzkCncaKoS5MmbgVoAUo0uOqoX4PyqQmS6OuTeyWXPkD30+LQH7ecI+3RxSjElqSV49vHk7sa7XP/iOu+kvSPug9GfRdNjVY/Xenr+G//v4//GmvefxL/Dvn/jvyLKMsv4o9cfRLwdAQjSTw1B7fEdqi0187r9YNSeUeK/m5l9RZ6d6Put8O/LH1+m8GEhYbPCuPHVDZzadCUzZChhwy1wjnAWC73UI6mkn85A3lCNWqqDq6MuzY+QiaMJI3eO5NkzuDQuA6vsRIqSo3AMsaGxupGkv5IYtHHQ3/rWnf7jNGYXzbgSdoVjF45pMZgufHCB9idu0KRnjI66kcbqRob+NhRlg5Bk27Z35fZtCYaV+WikUsBF3MBAGACY1ujRYGRJub2wa4/ZNwYQJAW/tf0WQ2cp1gqoM7UDnJBIJTi0c8DEyYTUw6no9G5AqWNMTY3gkfXkzBM+bvqY+vJ6qvOraappEhpt9UIBn3k5Uyx6TJxMCJ4UTNjMsL8dADSbFp+YdwIQBmc3v75Jh7c7iIinuqY6Kp9VYhskFMDymgoK2nRFI9UBNIw9NFZMViJ/j6RR1YhRmwdEJwvSd569PXnj5Btigi83lLOkaIl4DqomwYTYrZubWFQ1D1UCxweSejiVe5vuIdOXEfhGIHEb4+j9ZW9x2NduRjsqnlVwfNZxSlJLcO3iStvhbaktrkWmL8Orrxd6pnqUPi4lZEoILp1dsA+xF/9GU00T23tsx7mTM++mv9sKgeoT7cPNBzdp/7A9yfuSCRgbQM6tHK59do0eq3rw/cLvQQOzfpuFb472oC/9dDoxn8fQ/ZPuBE0M0kKUtpvZjjYDWssNgDBoOTL1CHUmdtRYumKTmwtoJ9PHZh4jYXsCILC/opZHoWeip5X4/Z3Bb0VWBWfmnxF0/k10mfdgnlbCadBggHG1MSGJISQVJ2kN+wwsDdg6fSs1xjU0ZTZxMvKk1rElEglFyUVk38jGd6iveFwLTwst5mWXpV3Ef9eX1XPqrVN0fK8j0Z9Faw1tXw7nTs6YTXLmxqnXfixAaFAXPizEb6Tfa72Z7EPtmXBiQquGzvnF50nYnsCyymWC7JKkZdhn2caSvt/0JeNcBifmnqDLsi5akjSdFnQicFzgaz0CmxPH8DnhYgII8LXV15g4mVD0sEh8TaPWvFY+qNmk+tWQyqQY2xmTcTaDW+tv4TvU939l2Pf8+nN0dHUIeiOoVaN7U+gmPHt7YuVjRenjUtFEXq1QsyFgA0ETg0SPAqdHFzCtzgOmUK9u4J9xyC1hB2gaYdH6RZQ6nOP5dSvu/XyXLku7oJYJDZDH+o9RfqHE28yb0+8KTYVmT1RA9Pu5+e1N6svqtSSk9Ez1mJ81H11jXa3ha+0LMqlC3wTDFyBGK28rQbrPS0Ai5zvkk+mRzxuj3sDMxUxszsgaJXgULKTM5DplxjGoVX1J2JYgNGU0YZhWgmmlC1e7X6XCUheJvmAs3yypZ/utLUW1RXyr/y3nlpyjx6oedPuoGx3mdWB76XYA/nzwJ3ZGdnzTV5C6zbmVw85+O/Ec4kl8eDx5Tnk4hDlwdOZRsq9n807aOyLb4my/syz5bgv65vpUPhdM3/XN9cVhn3HpM45928TYsUKRGvFuBGGzwpgQ9TZZrlkM2RcBbmfJiwsi+2Y2geMDCf6+L4vvTaLItgjLxnl092hpFPb6ohfOEc4kbE/ALsiOvcP2MnjzYMJnC9607Wa1I/GPRJw7OVNbXEvst7EYWBhoyd54WHjgbu5OVkUWxXXafqgHvz6I8UVjxv81nrxJefgM9iF0eiix64RmdPCDYP4a/xfzh89H1aTiyJQjRH0UhX2oPQofBfV19bTNmYdUaoRaDZ59PNEz1yMvLo+BqwfCALgV0ZPBcRrKM8rZNWAX4XPCebj3ITXBNeiodIj5MAYjWyPUKjWDNw3GwMqAymeV6JrokrA9Aa9+XjhHOGMfak/0mmgRCVxTUEPl80qtBtDL//Yb4YffCD+qcqu496sggWdka8SGiA38+eef9MzuRX15veAj/EFXOr7bkc37TxKvjMeyJgpXZS+M7BoYunUodsF25NzKodqklnTvdBzzHJEpG1Cr1DhFOBE4IZCkPUkUlxTjaOTIgTEH0DMVGGG+Q32Zc3+OlmxSykHBF9KrrxdVOVXMuDGDxKJE7ubdpa11WwwsDbi/9T4ZZzMYtGkQDx4bkJUFBn/+yl9nzRm1eyTboraJEkgaDTQaW5PRfhyhofDo8CNS9qdQ8qiEc0PO4ddlPKf6rKXRYgB9rM1RNiqZ4j+Ju1x8vfdnQ4OAGq87g56PHu+Me5fIcZHIpEK+YBdsJzLRm+Nlz6hNIZvQ0dPhzTtviq8VJBRwZOoR9M31mXZ1mpCzyA4TGVOFbXEF0L/lWI4m6Pzkx5fPBLk1jUSFjb8NE45PEH/HShmERl+DQk8PdZ0uB8fv49GhR0y7No2Y7U+JuNZAspcVMoWaJpng5Tn096HI9GXs1Qj022qTanb/uJuV764kPz6fX8N/pftnQsPUN82XrV5bGXNgDGqlGlWTipDJIUy/Pp157+ni+PgKOoaCbFNdSR3GdsbIDeXisK/ezB6Tl3r+ykYlf3b4kxBCSPNJo8RByEUqq6r+j9b0hQjZyxQZPFdaUycDQyV4NQXxLmFM6ClI+CYWJnIp8xKTggSWaXlGOfnx+Vp5jG2QLTI9GXIDOb2/6k2hXSFkgUG9AValVphKTLEdZouusS5fWXyFWqFmqOVQdkzZQVFqES5dXJh6ZaoI9HlamUqZwUGC75mRff0Z7Wa0E/Y3byts3rDhwaUHnB5wGo/yX+lgZsKzmGcEnz9Ajl8ffll4m2Kz07z7ozm6qnKqrWZz6cNLXPrwkni+Rm8asWPJDtYvXU9tcS3r7NfR/q32aKYJ67LXEy/S9qVR5VzF3Z/v0n1ldxGAJdOXMeLPEVh4WbB32F4aqxtbKVs0R+blTAwsDShJFe6XNN80nng/4VPpn0QuRJTCVCqVmNQH0CDPo1Gej1Jpxp99/sTUxZSoD6Owqnfgjr8wxGqUFaLRCLWE3yg/TJxMMKpuyXGaVR+qsqsoTSvFSKOd/6g0KlQKFcp6JQHjArDwsEDZqGTi6YmYuZph42+jxSQGITe988sdfIf44tbN7bU5bDMm4eqnV5HKpDw5/YS2lxXYOo6kwtYVPbMHuExzofvK7pi5mFH6AktTZ1THpb4ptLP0Y63pWpyinOi1uhe9v+qNzxgfkiuT0YzR4OXiRW1xLfvH7Md/jL+Wf9ziosWvZQ80ry99L/TlpPdJHux8gFOEE0prJacHnKbv5b6YOplSnVdNQ0UDVr5W4nGud73O3G/mYupiyt7he/EZ7EPYrDBMpDZ0eXSTMs1Mel3qhSxKIl5zK18bds8sJcv+Ol2ua2jSNSd5XzIN5Q2tJN13TdrFivAV/Br+K0O3DqXd9HZ8M+kbioqLUC5V4jXHi7u/3CXjbAYfNX4kvq+2uJaD4w/SdmTb13roVWRWsCl4k9ZrHd7uwOy42RydeZQz751hWcUy9M302dl3J1Y+VqKE5uOTj8m/l0+XpV3o/nFrCfo59+e0eq05Xr0nXs6rQQDupR5OxSFcYOeM2t3Si9g/Zj8pB1K0hoXNoWpSUfm8En0LfQwsDJj3cF4r9ZzmiNscx/nF5xm+YzhjT09j6+GHGNQZ4POkKw2ltQz8ZSCrP5Xic2cXAHc63sEnzR+Ph4fRjByI3wg/nt94zhpdgbmZ9kkagY2BogrLq6otIAwdKp9XCmoWLzGW/i7GHBjDkJ1DyCrLYrrxdMxczej/vbBXrZasBsDQxlDM8dUqNVXZVeTeyaU6V2ArdrvWDblCjrX0PhZeFhQ+KKTLB13Iuyt4JHMRnNIuI1fUAb4io/+y52VunxF8bpvGNxHQMwBVk4r3C94X6rJvwLjGmLb32grN8N5C3leVXUXchjjhfS+8A6vzq3l6/inOnZypzqtGo9Zw//f74rBPUasQcnyE3F5Zr+RO6EZsm7xobPTj7voYUg+l0u3jbsj2FzBdNp09E/ZgVpCCRhOK9yBvjGyNSN6bTOGVAhwnOnKlRyIO9ZMBsPa1bqVw9HL9qlFrUClUWveTuYe52Jd5eVBdlFxE1uUsfIf6YuZqRtCEIBoqG5h8YTKG1oacfPsk7j3ctYAEzWHqbCoy01wiXVqp5/xPoVarKSoqwtbWFqn09Qyo/ySS9ydj4mDy2ue385LOROlFIVs3nFuPPuVtTRYqjQpduQxHEy9mlnVDtlHOL6d+YWbsTBS1CkxdTMVr6BzhjHOEMyGbQuj8a2c8mjxoN70dv7b/lW4fd2P2vdnUFtVi5WtF2KwwcXgStzEOKx8rcdh36p1TPNz1kEW5iyhJLaEsowzfIb5i/+jyJ5fJvJTJ9JjpWt/f7R9vc/3L60w+Pxk0gkenYwdhaDb7BZ7y44+h42tmig0NoKcnACk9e3mK8skvx6MuM3h/m4zG6kaU9UqBIa3RXo8klRXsGXqa4EnBBIwNIHFHIvVb65EtlzH1i6mcu3OO8UfHU1sk9Lr0TPUw0RVy2Q92f8DWY1t5K7nFnzilOIV5J+cB0P5Ke9Ic0hi1exTmHuYU3C+gzYA2Yn3m0dNDa/C57uY6itYWYX3LmgXPFwgDzI5OL/Zv4bxXrVrF4DgNi7XdV8QatGyQD4tnh/PX8L/IOCfsNTpSHZxMnKg1riV2YSxfDP4Cxw6OuHVzozqvmuq8amwCbLjz8x0qMisYtGEQtUW1FD4oRNXYIrfvM9gHuaEcuYEc7wHerwWVvBrNqmovh0xfxtwHc9kUvIm3N7zNpx9/yqxhn7T6PQOlPWqpmmx3FXqacOrKGpAZyLDytdJSsnm45yExa2J4K+Ut2vRrg76ZvpYUcLOVgFqp5tisY2ReysTC04LJ5ya/Vo3r3/jP4n97zftP499h37/xXxEatQbkiBrKAHVd+tLhLUFiLPNSJl0/6IreC7k7vdoyCo+lU+rdRmux1Wndtyfqoygayhu4tf4WAGqpjFKXdli22LkhkUjIic3h/pZ7WPhb45pyFrXFCD6VHcFvpB8jd41ErVCjVMjQqy3DKu8hVXntcAyxYUuHLZSmleI3yg//Uf6cmHuCpxeeivIeALnbcukT34f7YfeR28jRlwkLr0qh4sZXNwAod/DHfMIA7F8BmHiNCSPrRhgBl39+Mc18S2vA6TfSj9Mn1ZgXpL0YjLWEobUhn6g+YWL/UnzP/0J+m67UlVpQ9LAI67bWjNk3huKUYh7/VohJ1m2Upe3x7O0pDk5Ov3uah7sesqJhhVZSmHEug/qyetFUGNBKMm58fQOJVELnxZ21zqffd/0wtjdGpifjvQzh+pQ+LsXE0QRTZ1MW5Qp+Rx3f6cjnH1SLWvYSiUSrwd6oElDeww6F4/HsG2AJpk6mmDqZUl9ez4NdD7APsRcHLRqNhi0dt1CYWEj3ld1bofgsPC3o+UVPTr19ik4LOxE8MZjeX/bWGmhoNBqMbIwImhhE7Lex5N3Jw3eYL9/afovfKD/GHhjL+SXnKU0vFSXIKrMrqcmvwS7YTjC4LqlDbiR/bYHVfE90udiFG1k3CBgbQF1JHdk3sqkpeEEFkkC9QT0lj0q03ttY1SjI7EloZdL8stRe3r080k+mEzwpGAtPC5QNSp5ff44RoCg1Rbn5Ivzykdb7HTs4UpRUhImjiciuaqxuRFGnwNDKEKlMSunjUmGQGuWmNTyz8bdh2rVpPD7+mJxbOVrPN4D+j/osvrCYb9//lsR3ErV+pqOrw3O358ib5NyKv4ViuEL0/Ct+VExtYS359/M5t+gcVr5WrVBrrwsDKwMmnp6IVCZlrfFajO2NGfHnCLEAfDn+Zn5JU20Tewbv4fn156iValy7uGoN+xoqGojbFIdzJ+fXSmh69fNC31JfMNuOf4JlX0t0DbQHZwWJBdzbfI92M9ppDfvM3cypyq5incM6en/dm7CZYcRtiqPwQSH+Y/zRt9Cn5+farFqnjk7Yhdgx8OeBZN/MRtdYV5SVezl29N5B9s1s3n70NkY2Rlr+iop6BdM3TedK9RUW/rLwtcPG/yny4vKI+SKGju92FBnTzXIT594/h3OkM1XZVbQd0RYduQ4VmRUUJBRQklpC+sl0kvYmMWLHCFyjXOn2STesvK3YNXAX7V8gvBUGLzT7TU0pyMv72/N4OQoBQ7U+NyNvYlvfl84fhtH9w86CTGhRy9Rud9Jupkycwieq1sl4c3Re3LnVeieRSjB3MxeaGzlV6Brrom+uL0rm6TTVU5uvQNPWRGhm5rZIxV3vep16QwXeA71RK9WoVWp0dHV4euwxHW9acbtdMaFXr6CWteGDyg+QyqRsHyshtjs88tzKc7fnGMjNGezzimTbC3ZHcUwxT7c/peuyrpg4mGDiYILmestnfrDxAan1qTRWNXJkyhG8B3qT56h9XV8uKJqPq5QrqZE2UJNfw/du39NpYSf6fddP/MxOaZfQry0FFnN0xlHsQ+2JeC+CG72fUKj3kN6XxyORSMk4n8GlDy/h2tUV217OPK14CkCVjScSYyN2DdiFW3c3un7QlacXn3L/t/u8efdNui7vKsq6bArZROGDQiafn4yllyXKRiXzs+ajb65PTWENT04/wbmTM9ZtrfGz9qM+tZ6STSWULiwVkZxLOy9l2ZNlZL2VhX2oPSffOkncxjj8R/uja67LzeSbyJuEZ8XEyYRxR8aJOUlz8WlQK0hI1lfZEzYrDPt29qSfSud+l/vkOeYx5NgQXEqOY+rSiyFbhmDmZkZ2SjZVplUo5Ar6neuHQYEBCVsT0DfX5/b3t0k7lka7We1YXr1c3HsTtiVw67tbQuMAgZFy7v1zzLgxgw/rPgR4Lavs4PiDPL8uMOP1zPSQf+NF1NkpyKsKqVtWT8bZDA5OOMj4o+NRlymwbbBFaSgR74HmgfflTy5jUm1EqVUp9ZY9MDKdj+GL9GzgzwPpsrwLBZUFOGgEqW1FndCkMrA0aOVPenLeSWz8bPDq68WuAUIj888FfxKbE0sP9x5cnnqZ/Ph8kvclM+CnARi8eLtS1whdMwOkOlL6ruuLtZ+wZ6la6nXUarjx5Q1y7+SKrzkX9mZwnACYULeBnKwcztSdejHoe31EAMcVTXRJ38++e/vJGZSDk+nrvbdA2DcbyhswdjDGqZNTK7mgswvPUvigUAQ8nXr3FGN2mlJs3UhiuCCtGbc5jvrSeiy8LPAtD6bWqI7Zm2djqDyK4F7XEhq1BKlKilWxCTRU8+iQ4EHqFuXG0fmXKbAroNimmN9n/c6IKqFR2Lw3qK69uGASUL+QBtY31ydwfCBW/lbwUJASvhVxizleczg6/SiN1Y2ETA7h+trrODyRkxU6HGdnSNl/X/RWeVlhQ0fRQGOVhH0zjmJgacCAnwZgaGdIjFMMJwedxLVEAO1IqiUU/O1V1Q4FoG8sQzk+hMisU4SdWQsI8kr65vqo1CrSS9MZfng4BUcLIF47Z6psqOSd0+/Qd2xfJocIz1GXpV0oqi2CbyGuQxyL1i6ib9u+GA4X1r6Yz2MAodE+4vAI9uzcw9LipVpyhDWqcoxqjYi4aUPWxQz8B/QUARzOnzjzts/bmNdE4FAxCpVKyH80OjI8Ew7T1taa4s5wv919nErbYaJngJmbGZXPKrH0t+Rm+5tUmlUiUwvrwHqX9cL33M2NLE2W8BludOH6yev0W9eP9NPpdHinpcEmlUkJniSweOWG8lZMn9j1saQeTmX80fHsHrQbj2gPxhwYw4rqz8gsEdiXarWGlyUg9GR6uJROJd12BZKyn1E//4CmmiYUdQrqSupoiInDKcQHeZOcTrFhVMozcF3lJcrEGWUIOYZDngNbOm2h65KuojykUq2kenk1JmtNaH+3PXuD9jJi6wh29NpB8ORghm0bhkxP9to8pzmq86uJ/TYWYztj1Eo1xg7GWgA2i7wkVFcrYFlXRuwQVCuenH2CRiYjz7sbXnH70JHr4NJ5nig7VpTRsn+qJU2o1eDd2xPswb6dPU7tnXhe+Zx237dDT0ePho8aUDWpKEoqasV0a16nUw6mkHE2g56f9+T0VSP8699kWFgx9sb2zLo1C5mBDAsPCyrGV7Bm3xrSl6cTOi2U47OPE78lnqVlSzGwMCDINoiHRQ+xsbBB1ajS8nM8UfI9cV4/IFGVsWnOJja6j+dT6ad0XtqZktRSph53YNfs5dzoupY87/ZEXs4i7Wga3T/pzrTyabj/INznWQuyMCwypOcXPXEMdyRuUxzRP0dzru85vA97k+6cTvjscLz6aQPNNCoN+ffzMbAy4NGBR7Sb2U68H0GoY3us7kFxcjENFQ04dXISr5dnb090jXXFQVLUiiitvSTtWBrxv8bT8d2OrRrhr0bp41Lif4vHb5QfzhHOaNQanl17hpGtkQigeTmqsqs4/uZxotdEi1J8VblVXFpxiYaKBoEtooGsa1mUPCohZGoIcgM55Znl/NL2F6I+iqLnZz1bAQKT/koi/14+PT/vyaMDj2iqaeLuz3fpv3UM7t9Z4aoOJ/piGUXtntBjTQhVG4X7rklXh6eeT+l9wROr3AcoG/sCelr5ulWpFaM+HcXVsqtEr46mvrweiUSixfiwbGPJ3MS5Wqoj/xQ2fjYY+RpRmFnI86DnWsC25gidHsr+0ftR1CsYsWMEP3j8QPt57Wk3sx33f7+PRCMhPD6cCvdnjIkV1I90dHV4fv05R6cfxcLUmBy/3ujJtH1u3Xa5MTN/Jr8H/k6DUtgjdXR1tBicz12f8+OnP/Lx3I9J+COBsvQyOi/uzMANA3EMd0Sj0RD/WzyG1oYcmXqEQZsG4TvUlyG/DcEl0oXDkw9TnVfNlItTRObp4oLFWH/uistdK8JizlKYbENTTRO1xbU4hDsgNdNHXgk9L/XEpvQianUog34ZBEDe4Dy2tN+Ce5Y7F/ocZnDcof+j69xpQSc6LdCWW33Vb745cmJzOP3uaSy9LUUWr76ZPg/+fEDiH0KtLdWR4jPIR5DxbGPZatD4ctSX13Ny7km8+nuJ+d7fhUajoaCgABub1s/M/5M4PPkwXn28XssGbj+3PevXreNAyjKiJHLWaBTYAwWKJrZVpPM7j5jZdTYdQjtw5r0zPNwt9LNKHpVQkVWB90BvHlU84kHhAx5NekTszFh0Jbp4D/TG2tdaS5Ho5Vp8dvxsLTCGmauZ+Lvxv8UTtzGOxYWLxTWnpqCGsidlQq9TKvhAtunfBms/a8zdzdGR62DlY8VHTR9pef353tzGjZVOdDwtgIAPTjjIwF8G4tQvkDnDC2mrTOadXwOxDbAVBzoAuRZ7ybXahU1VX8zd3uXk2yeJ2yCck+YFWEZeX4V1djxShREZlzNwjnSmrlTwlE20SEQpU1LXpw7PvkJ/ZJ3jOnwG+zD+yHikL3pzLkNd8LT05OKKiyT+kcjchLlkV2Zz7dk1kMLW97eySbGJze020+2TblTlVmHmps3yfzkWn19MVF0Uo71GI5FIeO/pe+Iw6lWyR+KOBG6sPM/YA2NfsBmF6/abxQAGS4/g2s1Va11rPmeVRoWJo4kIALqy6gpXV1/FyM4I20Bb8uLyGLRhEJGLIolcFKn1Nwf82CJ/Pf7o+Fbnf27JOYztjbX8ygGsn8djnX2f8syRWHhY8CzmGXJDObqW+jSVCeuWQiWAls4tPodTRyeCJwZjpHDHvK4Tt9ofYcwBHTIveNBhWgDTr03XOn7AmABs/G0wcTTBxs9Gi90JAoCjprAGMxcz0Qe2qbrpbxmV/8Z/Fv/ba95/Gv8O+/6N/4owdzcn+q9ogoKC4KLwWvMa9CzmGbHrYmk/tz265kISaVBVQPavZ8iPHkXevTw0Kg3Bk4JFZp9D+jWOTC1j+B/DRQS/30g/qmskTJzYcvxmmr61nzVV2VW0m92e5EduFLmG42xlie9QXxw7OHLzm5tcWnEJ6zAXSlynUuIaxqIuwh/zHeZL9o1snCKcKEktQddEt1XjqsOaDnzzxzeY6ZnRmN9Ig6qB0vRS0QenwtaH7MABWLyolavzqlHUK7D0shSbpHm+PXidm2r4J4PYlA3Vlm7isO/h7odcX3sdXRNdpl6aSoOeGY8jJtFoYE5ObA57huxh2PZhhE4NxaWzCwbrY3F4ch1luQ8RXwgb4K3vb/Fw10M6vNOhFcI04r0IAscFYupsikajob6snqrsKiy8LNAz0eP+7/fR0dMRm9/nFp8j93Yub5x8g6urrhI6IxS/kX4k/JHA0WlHmXB8gpb3jl2QHUo3O6yv38Ogugh4vSF0vkM+1rUt8pEpB1IozyznwtILRH8WTVFyEWYuZjh2cKQwsVB4z718dg3Yhf9Yfy5+cJHhO4bTpl8bvPp4ITeQc+qtU+Teym1Fxy9NK+X0u6fx6OXBtGvTxEFg+NxwHNo5kH4qnZxbOXT5oAWJ1yydNuXSFDyiPViQtYCGigaqcqowdjDWuq631t9i9ubZnOl/hgULFgDgO9QXVZOKtKNp+Cn9qDGuYdekXexc2eKfBILs5KuGva+9XvfyubLyCs6Rzlh4WmDjb0PYm2GcuGJMnak91u3dW72nw1sdWklOxK6L5erqq8xLmodtgC23f7rN3Z/v8n7B+1pFlp6pHm5Rbrh2dX0tcrr/G/1ZWb2SWqNaZBbCdlX+tJwzC87gN8oPqUqKz2MfxhwYw+OOj8XBZey6WO7/fp+5D+Yy6dwkLdmT2uJaUvan4NzJGYcwBy6uuEh1TjXD/xiO3ECOV18vbn57E1MXU6qyq7jx1Y1Ww76Diw6yb/3PpMue0CBV0MHPgmFvvMGsN9/E3MCcwgeFePTyoOO7HVtJXtaV1nFx+UW6fNBF9NV7OQLHBxI4PlD01vNN88XGR0gQcu/mcmXlFdrPbc+SkiWtWHaqJhVSuRTbIFuxoH5y+gnpp9P5uEnwdLjz8x3u/HSHCScmYOVtxcTTLWjPVxPAl8OliwuZFzP5wf0HLc8IgOLkYrwXeZPVO4ud7Xfyvv77rz3GhoANGNkZvZYVUJ5ZTurhVDx7e4rDPq++LU2f2HWxPDr4iA+qPkBHrsPgzYMxsjPCPsSelIMp4pqiI9chenU0TTVNog/Yc/9+qDoIRXC3yAFsf/KUicqmVufwavyODF0nC871O0e/+3+ha6KHsbFwzZuTYONqY1x3u/LU7inmbuYk7U2i7fC2ImvnyuoXEnAvXefm0Gg0VGRVoKhVsDFoI+3ntWfQhkHimu728AQne6UQrv6Ev0b+xZMzT7ANsmV76FlqjVu8JC9/cpnrawVZjkvvn8SvVEmytzEWpbqo6xta7hMJPPXV8NxRGNyopY1oNNBQ2UDqkVTsQ+zFoVzQsiCGrByCvoU+jVWNKBuVWjJvEccjuK+6T/CkYBzbO9Lz856saFrB2t/WMvDuQLK6Z9Hvu37i76tfyKSZVJmQePs63duMoNPCTrh1d3vxc+H38r27IX3x3ST+kciDnQ84M/8Mb1h04bv5l/ntzf2MLHuf4En1uHZ1xcrHiru/3GPy7o+J66zGqqY7EtQ8ixGacFW5VQJDYaYwRHtZSqbwgbDmNw98ZHoyUeIwKyGLo9OPMuCnAVi3tUZHqoN9gT2VRyopHdgy7Gu6I0jklXQvwdTZVJTfSjmQglVbKzqndqbYppjK/EpU36i0PK/ULz60fdYT/GK3UvpkHiYdbUnckcidH++Q/EkyOdIcBpwegGFdHoZWhoTNElBItjG22J6xRSPVoPRVEtInRPQ60jXWJe1YGuZu5lqAFEMbQ+yC7ZAbCY1it25u9P66N2ZuZq2GfE1NUJlRQsbZJ6IXUNSKKMzdzcndf4Q65xCygodiYGdK6eOHAOwdJrC93uIt4sOz8MraALQgeju+25FZuWsptyjHLaulCdVQ0UBRUhFmHmYgqGFr+SwpG5QoG5ToGuuKhf2wbcPE5kXbEW1JPZyK3U92tHduj+qWisaRjQz8eaBQfEtA5wUhMz1iIh99C1IZWoW6RiM0N+wzrlPn5YPei/u15xc9MXYw5ej0I+jXliFvrEbp2Y/ff/uNHrq6RDT9s3xkBNBZCnE39Nk+dTtD3xxK0MAgkvclE/tdLIM2DhIbwTe/ucm1z67xVspbDNncGi0fMT+C4MnBtJshNNPitwiS3mf6n6HBQljfHvz5gPKMchEAFDq0HaVWpUhqTCh7UsbdDXdFufMU3T8wrjFmxu9DqAq4id9IP3R0dVAr1dRkFlNvXk+9ocBUUGu0m6jNa4FUJcU6zZqStBKsfa0ZtWcUCpUCHkK+Yz75jvlYBlky9PehqBTCMXJv52JYKXx3EonAkuu8pDMWnkKzrNl3KfjCek5MdUFRKXhnyvRkzMqcxdKvtb2dveUhbJcWMVGt+MfvAl6s6c7m1Bk1oJbp8tbj+RiYyEQwUGldKell6QRqAlsNtQAOPTqEWqPm1/hfxWEfgK2RLW0s2/Ck7Al7k/fSzqEdba2FZ33wr4M5MfsEaqmaxJAkpneaLspGNg8hjlV+hsxcxp9TzjF16kLUSjUatQapXEpllXAeNkVmWOY+RFHri3OEM48jJhN0+Sd6nfNCo+rK9ajrdEhfTpBMF0svS5qqm+h1oRcLNwgipp1udSLrShYe0R44dnQkcFwgl29fFr7HmVJGtx9Nm35t/rFZ+zKQsDlqC2spTStFrVTT/4f+mDgK4BSXne6EOITQpKsByUMSd2jIj8+n77q+Qm6rkaHXqEf3Szao2j9k5l3hmVc1qTAc2Q/dlKfIlDKiL3egQvcR0JIPGMiFGkqqllKVW0VTTct+LpPKMNY1Rl+mj0pHha65Lromwjr44M8HtJvRTiv3yrmVQ358PkETg8Q1xaGdA+89fQ+5oZx19usInhTMiD9HiHWnVc4DFA8yga6iUoZTRyf+yu/OffUHpNnn0rahZQ2DlmfGoM6A/ieCUfsnMnr/aB4+FNbP80vPk3QqCZ0ROkzbNI1ztefou7YvS0u073cQgHkyfRm5t3OJ3xJP1+Vd+eMPIxpMoskwOMb+mP30jOyJ3wg/VAoVZXvLcH/mjspZeAZ9Bvtg7GAsNptlUhnG1caU3ytHb4AenyhbgEs1ynIadQUgT63GBgMrE/xG+mEbYItULuPRyUxMKxTgABpU9Fvfnx6re2h9ZkO5IeqHagqLColaLsjjJmxPwOupF845zpxadoqF77bIz78cxvbGLCtbRlFSEdu7b2/lo21obUj3T7qTciCF/WP2025mOwytDFktWU3k+5Fa/oOvMji7fdSNjm93RM9Ejz1D9mAXaqfl43d/632MbI3wGexDRVYFN7+5iZWPFc4RzqiVav6I/kNkh59+7zQNFQ3i8NfCy4KJZyZi5W1F4cNCUWYR0AKVPtz9kPhf4wkYG4DcQI6hlSFdPuiCW5QbGrVQP8sMZKLaS/rJdB78+YAeq3uIAMn8+/lsdP+anjhyLyyf+AgZHbxf5DoSCY2G5ujVVTDs6DCODjtKg/M6nIsr+criZ6I/i+bD2g+Z5z+PDnc7kBicyJhgQf1nW9dtaDQaLTUUuYEcqVxK1pUsvPp4tepnvBp1pXV0PtWZbJ1sbCOEwWXyvmSq86tZUryEK6uu4NrVlYqnFahVauRGciIXR+LS2QW/EX5Msf2QSt1kLkdfZlRmFfe33efZ1WcM2TIEiY4EuZEcjUSHamsPVC/miN/af4tLpAupR1JxwYWzX54lc1YmZ8+cpfsn3anOq8bURbjXNFINTbImpDIpKftTeHrhKT3X9BS9386+f5Zb393izbg3GX9sPHbBdpg4mBA2U8jF1Eq1uLe9HKXKbDwV9ug0laNsVNHr8170+lyQ9Av4+V1WnfkCswpnakr6ag0qbANt2THtNCUWjzCrDRdfTzmQwqWPLjFkyxDcooS8+cGuB+TeyaXvt33/0UsM4OKHF7ENsiVoQhDeg7yZemUqdkF2JO9P5sx7Zxi+YzhOEU4oahX0W98PfXN9qvMFGfbOSzvT5yshZyvLKOPYjGOEzgjFqYMTNQU1WPlYkbwvWRgAT//H0/hfj1G7R1FfVs+2qG2EzwnXAgKsX7eOtStWcF2jIuKVHGaiWiGoL9zYjs8gH3q1642RnRESqYS7G+8S/2s8S0uXcjDlIAAKXQUSAwkWDhaMPzIeRb2C/Ph8TBxNMLQ2RFGvQKYvQ0eu02od67K0i8jCDJsVhnsPdy227pBfh4gMSZVChaJWgaJeQejUUEKnhoq/J5FItL5neUM1qlohR9M11sUuxA4DSwNiYkC/phid+BhKHtlj6mxKeUY5Zm5mGFoZ8tR+HZVGcdTIL1GaM0uo/TUgM5CJ96K8sRqnx1dROPYWmdYPdj3g1NunUH2qQq1W802nb5g/eT4gDFabgWhRrlE4mDgQNjuMdg7tuPHNDcxcBE9OlaYFKFZuVY5+vT6N1Y00VTdRmV/Jw8cPSbuZRocxHVA2KsmPz8d/tD8GFgYYyAyI6RbDn/P/RKYvey1AY9WqVcBqFD2HYONnI9Y7zeBKvQY9yvNqGLakxUe6sKaQBWcXoNuoS9ujbXns9FjsOXr29uTq6qvUFtYyIXOCljrPfxqJ2xOx8bfRGvZZ5j7ErPAxunWVgs8zcOiNQ1TnVRO0vgejKnvgnulOyYlsVAHB3PnxDiFTQgieGIxP+Xzs8mbR7sznABh7v35Q+jolkZcjblMcV1ZeYW7iXD6o+kC0cPg3/v8j/h32/Rv/tWF86RgXlhvSdVlX2s9tj4mjibgG11i64bN2Gh49rdnReweqRpXWsM+49BmpR3K1jieRSsRNLvDyTySrXLF92489g/cwdOtQMi9nYtHGinoLB54HD8bVFcb9IniYpR0XPH1K4rPRDJKCREKzBUSfr/qQd09AamnUGvp+07fVZzELNOOZ+zMsJBb8Nfwv6svrGbatZSN6FjQI3bpyJOnF1Je7cHz2cTIvZrKifoXYJC13bBnmxP8ej4GFAX4j/fhOsLPAM+Hwi58GcO2za5SklmBgZcDTi0+RNVpRZSMU0jYBUgb8NACZvowvzb+k+yfdKXEOpcLOl34uLQlMsyfHgB8HUPm8kmfXnuEW5Ya5uzkmDiacX3yeh7sfMv7oeKrzqjk57ySTL0zGs5cnU69MFX0QAFEOtDS9lMj3IwmdFkrl80rMXM1Ec9yyjDKenn+KW3c3Ug+nopdli27RE8yL0lDU9eY75+8IeiOIgT8PxETXhOqmai73vIyR6SU0Grjx1XUuLr+IRCph3GGBXbEhYAMB4wIYtWcUKxoEXfpz758jYVsCoTNCBS8VI10ufniR2HWx+A71FWj7TSqKkouw9rUWm4/N94BdiJ2YiAOijv3djXdBAl59vLj8yWXST6UTMDaA5H3JYmJYp6jj0MJDZGzPYHHRYi3EZVZBFgb1BuQ65WIeYg4ICMWQqSE4Rzoz+q3RPGnzhD1v7Gl1fzWHSqHi5LyTOHdyFpvGR6cfpb6snvFHxwuI1UhnLYTawF8Gct1qCxKNGvnYXn93aK1w6uhEh7c7iKyegLEB2AbYtpJ21Gg0TDowiVu5t0h8KxFjXe3BWLvp7fhxxI98+exLDIoMwExoBGVezMTU2ZRPNn5CpnsmsZ1imePTIrcTNDEI+1B7rNtaaw36ACqfVXLq7VP0+rIXDmEO5MTmUJpW2nJOag0Xll0QdPEnBmlJIYBQPHzw/VK6IuVLpVJACaYWsuOLL/hs9Wq+/Pprlpa2bs40h6mTKW/GvUlFZgVfGH1B7696v1bX362HG8VlxVrIs/rSep5dfUbghMDXomNPv3eae5vvsaRkifjzkbtHoqxXolII0kCNVY3o6GkXhWqVmsaqRmT6MqEJJ6FV4Ri9OhrfIb483PNQbMo2h7GDMVe6XyHbJRszlZko8fRqqBQqyjPKW72+tetWsm8I68nLErMGlgLTsvBBIW7d3AgYF4BMX4ayQYlGo0HfXB8DSwPC3wwn/M1wrWPKjeS8n/8+E6fLUcv0sHuR7XRz6M5Pyu+5DX/LyAHBuP2aRIlqiOAPokFNTWENDTkNWLaxbGFl1RvgfNqZzOBMnp5/Sn58PhaeFmJiXfa4jJxbOeTeyaUoqQj/Mf4i01OtUPOj54/4DPEhYn4ELl0EBkDzml5p6037ngIj0bKNJW2Ht2XEnyO4E5SMrKAzfo/8uKwWvMyCJgahZ6rHiIszGbNmJkmBSZQ5BvGmRQB59/Je3A/mWBeqqVM7oZQp8Y+vprZ9MdXucHTaUbp93I0aXWFI0O6vdtQsr0GqI+XM/DMkbE9AdqYlZTw85zDvvvMuNn424pDdNc0V4xpjnBKdqHxe+drrGhkbiSL2KarRKq1hYDOzqnkfAlhevZz9Y/eTfjId03IZ4XHh3Gt/D2VZg5Z06f0PzuGFDjp6w3BN+owGvRGsrBGYamcXneXW+lt4nvfki01fMNh7sCg/+ubdN/k98nfWO69nwvEJgiRkbhX65vrYBtoy4cQEbPxteFbxjBOPT2DgY4D1HmutoXjJeyVEe0ZTEVUBQNSHUVQ8qyD+13jKn5Rzf8F9bG/ZUnqslIYVDVrPbVGxcA899D+LTfVv6FkZcW3NNbKvZzPuyDhO33/OE56wZfYW+iR8jkbTwiguvFxI0MMgTg08RU5hDkpbpVj4OncS5McbKhvIvZuLubu5wDqfECRKZgOij4ZaqSbndg7GdsbCMC8X5s6F7hZ51P55lnGHx+HRy4PT75ymOKWYA5MqadA7jXtpJyRyGeGzw7my8op43EZdBVJlBbr1FZQ8LmVn3z/ptKATQW8EMfxAGPfCmpDp12OdF09TbSD59/PZ0XMH/X/qj17X1nJl97bc48x7Z5h6ZarIhnqZFR39aTT3t94nNDeU0MRQ4RwqG9Ez0WuRBntpbtP4mvmcRgMyRT12WXdpzDRi6Pp+nJx3kksfXqLvHwMocNanUf85TqUq1I0Kdm/YwqfKfx70NcccFaQl6qC8peRpu6cEDQyiqbaJ6txqsbgHAVDRaWEnLQT2y9EsL94ccxPnEvBrAKXWpTiWCqvZ6L2jUTWpqC+rZ0uHLQw/NpRVq1bRK3E9VTlV3Fp/CwsvC1y7uJKsvwU99LjS/QptdScy9mAbEv9M5DP5ZwSvHc+KRmEfC0gKwKRBkHn+3v177EPtUS8QLmi4RTj9P+vPzcabDN0iyJnJdeQcHHuQUfuEwVBOVQ6eYS2AmXaz2hG/Sx/rZ/cwLFPg1KGTltdO04u5TYlLKN17WtBrRQtboa60DudsZyJjI7EpNYUN8Fvy77g5O/8frukqdKNqsalQImusxdTVHL2XbrnmRtThkYfp0b8HAE8vPkWj0uDV1wuVRsWxtGOMfjaai+kXiZgfwZ4hewiaGISOjg5WJVZkJGVw3/k+2cey0TfXJ2RKCJk9TMl2yUajH0G35d1I+iuJg+MPMnrfaFEVQylXUuBQiqGdKZc+vsCNL28w4cQEng5OZC5zOWR4mGN17Tg8S8P8N6WYSNogcV5EYE4ePa5Ecz3qOqBCUlZK5qVMrNtas99xP1Hdo7jZ+SZ9jvYh0TIR126umLubAy3DmIa2DbTp93qmm0ajYb3zerwHeb9Wsq33l73p/WVv4X54sQ9X5VThGGdGB5fJmFRJQHaVJzJHkvYmiZLbEo0O9Qb1nB76nF52LaAAHV0dKtLicHlcRKNeI9um7efLiPnk3Mrhwc4HtJ/bIpeY65xL99vdCXUK5enFpyhqFfgOFYBIUomU+2H3GbR9EE4WTryb/i7J+5Jx6+5GQ2UD39p+S+j0UPTN9bnx1Q28+nqJz55MX2DEqZVqBv4yUGRjN+9T2QH9sQ9pEq+PRCJIW541GkWWwVHUDmqsciZy79d7PNj5gFG7R6FUC7mkTCmjbaozGrM8QKjbHh16xM1vbgoHHwF6jXo01v39+rJrwC5s/GyYeGYiXZd3FQE9FUb3uFX7I1HrV3Lz+k08e3lSkFhAzqc5dHfvTnVuNemO6fgO9RWvE0B1YzXBD4JJWZdC59udtfKwJrXALJCqpLTJmYPESFf0DXp+O5/rn1/D9akOxUYutH2sh66pHuYv2Blxn8QxOHYwV0Zd4foX18m8nCl6gHX9oCu31t/CrtAO/cn6rx30vRy2gbb/nFs7m+I/xh9TZ1NufC0o4lQ+e30e0hwv5xG5d3KRGWi3xc4uPItThBM+g31wjXLlvafviXu4VCZlwM8DxHuj6GERtcUtg0g9Ez3xmXp84jHxW+Kx8LLAyNaIrh90FX8vclEkAWMDxIG0obUhvdcKz1OzN1zY7DARANL3276iB/GY/WNoqm5CZiBjU7tfSa9P40aXGxhqrNB3sWFj0EbaFJlR5hCAQ8YN/B/5c3jEYVQyDRq5HIdwB1EC3LLEklz7XA6PPMyhUQKbLGB8QKvmtlqlJmFbAje/ucmbcW/+j8O+3yJ+Q5YhY4JkAprzGvJO5RG/JZ7cO7l0mt8Jzz6elD8tZ8z+MeJ7mvskT84+oeepcC5F51BpXolUT07C1gRybuVQ/rQcPVM9giYGcfKaoOzTnB/Zh9qLQMugN4Lo69WXn3J/osahhscnH3N40mHGHRb6N7qNutjm21JTUMOQLUNERYGStBKKkooIGBNAm35tsPG3wTFcGKBq1Boen3iMqbOpCIB4fv05pY9LCZoYxKWPLrHq21UcGnGILfNO8oHvaq3rh44O7mWr0OjIqLER8pMT805g42+D/2h/5Co76g3ieXvTaOqsLgPRYl32sqXB03NPSdyRSO+1vcm+ma3FEAVhkFyaXkr3T7pzfe11/Mf44z3GG7WFWsyldI11sfCyoCKzglNvnaLH6h4iaM1IbcTQrUO12KWqJhXFj4qpK6nj+trrPNj5gBX1K/5R2eT/ZviN9KPwYSGXProkyqgCFBQUsGzJUmI06n9WX2hspNvSpTzLyREHhWEzw3Dr5oausS51CsGSxqrEivKH5fCCzFeWXsav4b8CMGjTIE7OPcm4w+NoO7wtZRllyA3lWtLszeEQ5qDFCHw1dOQ6vJv+bqvX60rrKEgowMZPYGiBIAE54gWe3aWziwikTToKlTbeJPV4mw/6mfDs6jMtEL8GYSObt3EeP+3Zw6rsGSIgvLkPWm9iS1L3eXgFtfRl7EPt6f1Vb47bHodnwmvpZek4mTox8OeB4u+t77+eDXc3cCXrCgG2AaK3KICysEXy3rDCkIglEUR/Gk1jdSO/m//Ob3t+Y8LeCTi3cabsSRnXPr0mkA4sDFpAZi9YeEXJRega6Qo1jkRbEUtuasC0q9NaXpAI7x1xeASZXz+Bl2a/5Q3lpJak4ih1pO3ptlwtucrF5ReJWhFF4PhAVmpWtvo+ipKLKHlUIlr0AFxbcw21Uk37ee25/uV1vPp6aXmpv5P2TitZTI+Ew1TY+vKgzyKsX2zJvb/uzeFJh3m48AqsEoBauTsfIl81nEW5i8Rar0GnmItBfnQ69i759vmYtrGmIquCxB2JeA/01gK3Nset72/xcPdDJhyfIILv3Xu402N1D4zsjF5rO/Nv/HfHv8O+f+O/ImoKasg/nI+NyoZmzzD58wyeXzNDf62+2BBvbhIo9YwwCTTCyFZAgL88WAJIj5jEuuPCv3f220ljVSMDfhrAhY+uYlLZCYWuETpGQsNv4IaBuES6MDtuNpWFDRz9RNVKDtN3iC/LypeRnKgk4fM69GtLqS2xwcxMSIINrQ0Z+vtQUc7l1VCphV2nvKEcTX8N7c3b497dnSG/DaH8aTkFe28ha6pDnpNIycIZ+I/xF5MFdWvwMZc/voyFhwV+I/0oLgaJWhvRNPyP4Wg0GkydTVnvvJ7aDvepcV+NaX0IFh4WdHynI2UZZTRWNnLu/XPQrx2NRpYgg5gvYqjKqWL03tEgAPnJvZPLkSlHGLV3FObu5jRWNeLVz4tHhx5h305AFUWviRaHBK8mQMN+Fwablz6+ROy6WDz7eLIraBeR70eKjYHkfcmcnHeS4TuGc2nFJcyB+32XItGoQCIkIc1IvUtTL/HDsUukXRc8ElUqgR2KBIbvGI5RtBH3iu4xet9ozFzMkEgkojH4wJ8HiklLcxOm7EkZqiYVKQdScO/hjoG1ARsDN2ox1XyH+PL03FOexzynOr8aY3tjrYFHh3kdRKTgo8OPqC+tZ/j14aL0kKJewcx3Z1KWVsbYGWNbIZas5lixwHYBaKC+qR5zfXOKHhbhO9QXzz6erElYw/3G+7g8dyHteJoW86r0cSl5cXm4Rrly//f7KOuV4rCvrrRO9Fc0tDLUakbXldbx+MRjDKsLqbLxEAsptVLNoYmHCJ4cTHWe4GNgaGNIm/5t8B3qi/dAb7wHtiQ4blFuWgPQ5si8monPWB/Uvmp2Zuxk6qdTWw3X3M3d2dhtI6pGFe+kvYOltyXLq5dTX17PtphtpHun8yDkgSjNB+AR7SGywwCt4ZOVjxVTLk0R78WXWWYFCQVs6bgFj14edP2gayskVDNK8JpGTQTaD57o0fShMGRYuGhRq88LQhPJMdwRuaEct+5umDhpPws3vr7B85jnjDk0Bv1AfYysWwa+bfq34cPaD6ktrqXwYSGWXpZacprNqMzmexlA10gXXSNdjr15jPu/3SdycSTzHgh6+SqFisufXEZHV4drn14T39OMYn81XjXZbg5TJ1OuRF+h58WedNvWjbI+ZSLz6eWw8rHi6YWnrV5vHiwa2xuLRQwIRaVrV1c8+3gi1ZGK/iDVxdUceuMQDmEOjNk/Bo1GI8oFN1Q0sHf4XtqOaEun+Z1QywSJROP6SjTqAPot6cdKzWoGf7uGE4qm1xaBt4EhenrIOpujMilk7F9jMdbc5tZXGhK33GVR3iK8Lb2ZETqDP+L+4NY3t5jdc7Youew9qOXeH7lLkIA7t/gcseti8ejlISbTEh0JnZd0FoZ1b7QMYZrX9FKXUPwXCg2MZjmeR4cf4f7IBt2KQIxrjKkvrSdgbIA4cFPUl/Ig8Agg+HSpm5Rsab+F4MnBwAj6HdUB3uR6zyYCEpQ05JRi6uLBhOMTsPS2xPqkNXnVeejX61PytASDNga4R7sjM5DhFuiGSl/FyisryXXNbWXI7m/jz/R5M3H+wJGQ4BBSDqRQlVtFp/md0JHIMa0LJd07ne5e41p50LxuH5Mbyum3vh+2gbbc+OoGdoV2dLzdET3FYzTqcDRqDRIdCS4DAkhM0FBvakulrTcSdUtx6j3QG7mhnHOF5yhPKqfwu0JS16XSdnhbHNs7YmhtSE1BDRqNhrInZWwI2EC3T7oRvTpaHCglPUsCoN6wHomXRItRmz4lnYqkCjrM70CaSRq+Q315uFNgaqiVajo/7kzVsir6O/enqaaJtaZriVoRRddlXakoEu71HJcccssC0DU3IutyFuWZ5Zi5mNF5Vg8qet8mrW0yKmktefcUHBx3gM5LO2MaYMoZ6zNI1VLSOqaxJWgLbYe3JWCcAKwAAZCze+Buhv4+VGSDvS4aqxr5vdPvYkPxoABk5kaBFz9cn451W2s0ag2JOxKRyCQktVmPjlKHasPHNDVeRF2voM+3fci9nUtZrS7L/b6h3OQG4U8msUQqFWR/jeQo6hXoN9lgXT2I+tqjuD/IoK7UC8s2lvT/sT+uXV1RWCqQSCQieMZ3iC+2Aba0m9VOa214NYb+NpRlS5eR2jaVBvMGVtisoDilmNriWtyi3FCrhfXf5lkcybtlWM3xZ1vUNnyH+9JjZQ/Uamgwtiah9/v0idbF3KOBHqt7kLQ3iXNTT1Pv8ZQdk3ZgU9mPRbZLqFPW/0fenwqVgq+WfsXtaYJ/ULvp7VoxqNr0ayM2h69+ehUDS4NWQJDUo6momlQEjAnANtCWUutS+pzrg8czZ/gRsUln4WlB2Uem7KoQEL9W1mqcOzmz4PkCLQBJo34jV6Kv4Pq0DXc33uX297cJGBdA0ZUUVlxazdbpm+h9oTe6ugI4zj7UHisfK3Eo1sGjAwN+HoCNvw2l6aXc/OYmgRMCkTnKcH3mSmRsJE9Cn+A+2B2JVIJEIqHrsq58dR38Yragr64DtKXHmpl92YEDCHnJMkutUnPpnUvMOjoLhUxB5QupC3t7e7786isGL//wn9d0uS62bS2RNcqZsXUAuT5x/BqSRFlaCe3ntafbx91QGbdmaJxbdA5FvYJ3H7+LUq2kpqkGizgLYpNi6fBWB2qLaikqLiJNnkan9E70P9ufnD455O/Ox9RZ8KCzeN5Ar5xeXBgkDAPMXMzwH+MvDhoAJGoJcoUOjbVKcU8wcTBhL7tJ5zFRdYIQq70GCjRqtpDK4ZzZ3DXuiu5IQwacGoBd+TM0fsL1bPbN63G1B8kByQw7PgwnTyc2BGwQrqdSjca7pTbSaDRU5VRRnFIsrI0vckGJRCIwAmwMSdyRiEaj0WIbvC6aZX/rrXpS4WpJUKCGgV9Y0+fbPkh0JFQ2VJHsKjACsjyLUevbkPhnIoZWhnj29qQ4Us759n+i1lGT55yLzNqc4keZ3P3lLt4DvcWaCSCzIpMOTh248skVyp+WUxFWwaZ7m8QmbXOD0LKNJVEfCqwyHbkOXn29sAmwwWeQD179vLRyMVWTiuq8avTM9LSUK5r3qUYjS6QvUqH1zusxczWj47sdmfJtCEkBOhwcdRCVSkltscB6VClUSDRSTOtCqTJJ4LslB1ho8Ann3z+LrpsulpaCnJ7NWhtU6Sp+nP8jq5etpqmmifTT6a2Aaz1W98DA0gC5gVxs7GvQ0CQT2OrZa7KRfCDh8OTDWLUVcrHUtqkMODOAzNBMrQYkQHZpKXXudTxvH4KhjSEZ5zMwdzfHytuKJo0w7GvzpA3D9pSQ5RwHPQXPNVMXM84MTOCp202CHgbR6boJpenl4CUwSFJ+TqE97Tkx5ATx3ePp6tqV3zr9Ro/VPfDo6YH1n9bcjLtJz+qeNNU2cW7xOQriC5h1u0VyWNX0QlbU3fy1TfLq/GoOjDuAWqnGxMGEE3NPYO1rzfAdwzGwNOCvEX8RvSYa2wBbtvfYjkOYgwg0UtQrUNYr0TPTY3Hh4lbHHn90vJiryA3kWkBIiVSi5SE49fLrfSxBkOdfWrYUuaEcmZ6Me1vukX0jmyFbhrzWi605dHR1CJ8briVRaGRrhBFCXVCUVETqkVTCZ4czPWE+Fj+8WBOzg1GrBQ/epmpj6syE67Z/9H4M6wwxLitEatmGKRemAPD/Y+8vw6M62zVu/DcSd3d3JQnB3d2LQ4EWLVCgtFCh1HmAKqUttNAWpzjF3T0CIU6cJMTdJyP/D4tZYRr67Gfv/36P99jv0esLZJbMknvu+5LzOs+cKzncmnKLu253MdFrizlepoNWnSN0OFr7WrcD/r3MwmeEc/DgQR67PKZbTjekekKRVFGvEDX7pHpSui7r2u7YgrsFdEjwpsDFn3y3fGR11RhaCetXaWIpinqFQGto1APfB/swaq4CFjPj3Azh8wEQ2xyLNE3K0idCAaU4oZjea3sLshMJ4FjsyKu/v8qjwEf0XC4UYetL6vkx8EcA5sXO02EZAaGTUstkoC0EPNrxiIe/PiRoQhB3vxI0m+VKOaBBpRL0TGvyarAPtSdx2jYcvTzIiuyDscILtRoe/fYIn8E+FD8qZuruKL5dfh09hRqpSlgQgycEEzwhWOc6hnw7hAH/GoDcUN6uQxQg+UAyGWcz6LO2D0vSl6Bvqo/Hdx6U1pdSuboSS0NL/Ib54TfMj5r8GoJfCcYmwIbS5FIMLQwxdzVv56fYBdnxTuk7gACEce3milRP2k5X/e9MIpFgbW39UkDo/9QcwhxEeRStbd+2jd7I2sXqf7UuQG+JjF+3b+eDNUIHm0tnFxHwoJWFGXVyFPf33adjXEfufXdPR6/SIcyBiNkRYg5qa/hWPPp4iDqJT04/oeBuAT3f7flSPfvCmEIqMyoJfiVYR+qkMKaQrAtZdHi1A6WJpewb0d6ffxnTokYDaj0DmvUMMDAD20Bb+n/RX5w/NRJh/XwU8QjPivHtjgXQyPRoNrNHLWkm81wmNgE22IfY8+CHBxiuM0S2VMbIUyPJa8yDDbrfr1KrWHZuGQMuDcAnwofRm0eL25RqodhnZWjFoo8XcSrrFJMOT8LAzAC1XE2eRx6qrSr8hvvRXNWMW3c3kXFFpVHhn+7Pw/UPcfrIiW2dtuHVz4tpp6fhrBeGc8U0HnUxxSPXBYsAXQ9dC9DN9M1En87c//4+lVmVDNs0TPQRFCYKFqcupuJJBRdXXRQZGLRWkVEhUEV3ciHpjyRufn6TxWmLxZgw5VAKrU2thEwK4f539zEwN9BZa18GjMjqOIlWA1NRZkqtUuM33I9XL79KWkMNxH/Mjd43cOg6BYlUohMHSSQaWvUq2DN9D5XWlXyg/o6q7CqufXQNYztjMV9zf/N9bq27xYwLM2ipa6GhpEHn3jx6e+DR2wO1Us2z2GdU51aDRNBt/09/1//Y39v/E3Pef8f+UV38x/5PWG1eLQ8+fcDTG0/FzypeXcFrt1+jsaKRqpwqHR5rALVKg0atwbmjs7hoi78ziUT80Zk5m2HmYsaFlRfIOf8Eq6IU0nu8hsf8IahaVNgGCJzZt9bf4rfoLZhV5OIdfxjVkyzOLD3Do52PAEGvRN/aFIvSDALv/M6zBwUAnF58mk2em3i8+zG2gbY8Of2E21/e1qEIujL0Csu+EwLfW9G3RARM5JxI0o6l4Zh9F8P6ChTDx2DtY03ErAiRG167MHvHHSLgzg4AJh+dzLAfBOoSuRwkKmFxLXOLAARHxrWLK4YWhpwZdoZ8t2z6HDmKa8oF8Zqsfazp91k/HCIckapa0WuqRa1QknU+i5RDKUikEm5vvM3hSYdx7+HOlD+niAHJ2aVnOT7rOO+Uv4OFm8BV3vuD3mKgVJVdRX1Jfbv3HD4jnDE7xmDqYErXt7rqdFB49vVk5qWZ+AzyIXBcIPWWLqj0jVAamKJnpMesK7PouVpw1KOdo3nFaRUhic64plykuVmgR/xI/RHh08Px+d6HYfuGkdshF9euAh1L2p9p/BDwA7E/x7a7rojZEQz9XghuNWoNfsP86PNxH51Ci22gLa7dXClPK+c79+/4peMvtDa1cmjiIWK3CufcN2IfO/vtZOi3Q1mWs0ynoNdS20Lgr4HIlXLODT3XDl2jpQ1c8sMSDnY5CAiovYMTDtJQ2sD8t+bz1OMpo++O5tiMYzrHZl3I4uj0o1RlVbG6ejWjtrehs6eemMrrd9vokxorGlG2COOlMqOSP2f/icLAFI1ERvP32wBobWwl+WAyl969RObZTFIOpxC7JZaC+wXtnt2/M0NbQ5JCkrCstqTkhxKaq5p1tn+97GsWRi+kelA10W8IaO7YrbF8KvuU0sRSVpxeweMOj/G1fjkiPedqDp/pfUbcL3HiZwbmBnj18xKdxxdN31Qf3yG+BI0P4u43d0k+mCxuKy4u5t3VqznZ0vJfogTffWcV20ZvY53JOspSynT2UavUtNS1YO1jzfQz03W0JkEIYLMuZiGTyXB3d3+pmG/CzgS2hm+lJLFE5/PIOZEMWD+A21/eFpNtFU8qKE8vJ/1PIXn+In2VSqHi9vrbFNwp0PnctZsrf7W739zlyporf3Pnz6/dvpSnkU9FuqG/Wve3u+t0LGtt5sWZvN/wPmN3jaWxvFH8/OHvD/mX2b/IvZqrs7+hpSGmTqYUxRexb+Q+Nvtu5ua/blKeVi5q3NQV1vH01lOcnlzHI/E0VpePoFapMbQ05IN1H7L0zffphYR+Eil7gUvAHmCQgSG9ZTLeW7cO/c5CAsYnOwjT6mo8BvjSa00v9E31sTC0YJjfMFRyFY22jVj5WGEbaEvvD3u3K1iDQN0y7cw0TOzbEilSmZRBGwfpFPpAt/D11yKYtjPyzPAzfLPyG4Zt1qUwVj1fV0zqTfDOMENdXUP/df0JnigkCZ50mUF6l5mY6H1IYt/FmIQJxUf/kf7YBthSsKIACRJ63+jNjqAd1BXV0eHVDoz4aQShLqFMC5smXFerQF2kFZAHOHT/Njv311OZKtxP/PZ4rq4RaOI0GpBopOR45yAZ7YSyScn+UfvF9VN7n95xhwm59iMatYbCB4VI5VIGrh9I2pSPODP8LEPPDcXiSTLXP7vOZ3qfUZZcRseNo9k39WcODHLn4sgiWuzdST+ZTvGjYrwHelPztAajGUaYNJjgUujSpnEKLHmyhHdr3sV/pD9GNkZ0f6e7qAulNW1wKlPKaK1qFedHgJqBNaQFpmHgbICesR5JB5KQ6knF4qtJkAlhvmF06tgJA3MD3Lq5iQWZqeY/IlVrKVbVqFQw89JMlucu5/js41iUZ2Nab4lHrgfmZTmABAMLoVst6+csPHM9cTJ3osOrHTC2MebGZzeEzrvJh/lE8gnlqeUMWD9ADPgyz2dyde1VEf2cejSVX6J/oSSxhIEbB7ZLJCkNTHDv4U789ng2WArRfOfFnbGqtGLgpYEs2izoJOVcyeHi2xfx6u9Fw+iuVJkJRW+NRI2FpxXzYubRcV5HUo+kot9cz/2I73nQ+Rp7pu/B0NoYCzcLuiztglOEkzjnXVp9icvvCpztXv29GL1ttA6AYNfAXfwYJCTjziw9Q8G9Ao5OOEpKSAqF7oXIDeTc/OImO/vuRNWqEseXY9ZtUn57gEatoaWuBWWzUhyfGqkMpaEp6Ovz8LeHHJxwkJRDKQDoK4Q5TSGvRK0Ge1e7/1gnrgRQmCmEYrHhfxbsxW6JJWl/ks5n51ac48DYA1xadQmAZ/HPeHXnq3SM64hp3XPtl5J6cXzXe6qosK2g8/3OWCY9QW4ox8LN4qXIXZUKzrxxhoonFbzyxyvIzY2RYIihMojjY4+T3Ufwoaccn8KgjYPE5IjMQEbnxYLOal1hHfHb4oX5t6UOszozAtIDqCyoZLPfZrZFb9P5zqyoV6gZM5OMsxnsHb6XoodCF/WBA237qNWQdjyN7EvZqJVqsv8UwCI/L/iZU69UA0IibULXiaz6+DN6SWT0kUl05vR+yOktlTGu6ywWJC5EppKREepCvbUHlenl4vNurmoW78s/3Z/mY4I/MmD9ALE4oNV9ipsWx5K0JZi7mrM8dzn1k4Vn/sT/CQcmHaDStpLFKYtFfUy1PrTqtRIe08CuYbtw6+7GxIMTdRKGFjUWLP9mEjEbrhA6OZTxe8ez6KP5FPGE28BVYDow4Pm/14DbQHX9LfJuxeD+1B3rsnpazOxE/WTx3CYWRIyMwC7YTqTaKk0qFe9X/aGaDXYbeHLyCXuH7hVp7bX2+p3XGfDFAG5vvM3t9bd1thXFF5F0IAlFvYIfAn7gwjsXsA+zx2ZCH8rsGilyrkDmI9CLmbuYI5FIdIp1GlSoVHBu2Tnufn2X2sJaXLYX45fhBxqQtUpQtSgJmxrGyuKVeA3wws1cAE4aNRpRdKSI0qRS+n3ej5G/jCSnOocdj3YA0Km+E6k/ppJzJYctYVuI2yb4gnrGekw9OZUuS7tg5W2FVz8vHdqq8rRyNnlt4sHmBzr3qp1HpEoFmiZhLPiN8MOzvyfZF4WxWeJQwsKtC+l6JIneH/Tm7ZK3sfKyQiqREZa3BcuGLpg3RaNWw/1N96m8V0nErAiW5y3HJKrNP1CpVdSX1HN40mES9woAksbWRtLL07GebE3o5FBqC2spflRMa4sKlbSRbEeByqXco5xOizvhM9SHiNkRHJp+iIQOCXy/9Ht6ru7J7Y232dlvp9jFpFKreebyjGyPYCQSCXsG72nzm5/HmDUWNcRFxaGRSTiz5AzZl7PJOP0Ep0JTKmwqiOsYx6lxpVxadUGUKOjwdgcK+gtxwddNX2M3wI7aglpaG1spTyunsaARUzNTwueGc/bNsy+lRFQ0KDg44SA3190k5XAK5em6muRqpZqK9AoK7haQejSVhpIGHKMc6TCzA7UFtaSfSKehRCiy1z2r0+n+ubr2KhttNlKbX8vLzLOvp0iz3NrYSnVeNYqG/5oCHgTf+3ODzwVAnZ5M6ExpVVMYU0jq4VQSdiYIBfe/ZOsbShvYM2QPsT/Hom+qz8gtIwmf3kZN2FTZRHVeNRq1htQjqdzZeIesC1kCFXWrDS4FLgw7IqX0egqTj07madgIWowtKXXzpcaihi73uxB95RqtNW1diJffv0yfY30wrTPl89jPST2W+rf3ZWBhQNT8KAZuGPhSX/ev1vfjvmyZvIXbPW+zZ9EeHDs4Yhtgi3NHZ1EiwXO8Jxd/vUj89ngayxvZO3wvcdviCJks+FE2FTYs/Hkh1nGXiF4YzdBNQ3mv7j0WJizEva87ra3XqDPXoLBs60CTyqUkVSex6cEmxh0Yx+knpwFw7OBIv0/6YRtoy8FXDlJtWc2FQRfw6OFBQ1kDtYW1AjBFJsGhg4OYt8i6mMU603Uk7EoQAZURr0WQfjKdh78/pMuyLkw7PQ09Yz1mXprJL/N+Id8tn6AkV+qK64ndEsufc/7E2E4AUbjk5OGY9CmhV75Do4G3S99m/L7xop/T+UFnTkzVUBDcng1Ka0bWRgJtslRCt7e6MfPSTLFDFGDYD8NY9HgREokEG38bzJzNKG0oJfJhJN8bf0/WhSxxXws3C7wHe/PkxBO2hG7h6tqr/+W79R7gTac3OqFRCfqVlZmV/+UxUqn0b2Pb/6411zSzwXoD51aca7ftz337mKP5r6m9AWYrFez/4Vf2j9pPa5PuMa3Pi62x0bF4velFXVEd9zfdp+5ZHW+XvM0HzR/g1t2NMb+PETs/u63sJsZcAJnnMrn5xU0U9QrOLD3Dl/Zf6swjcb/EcXT6URQNClqbWon7JY78O/nk38nn6pqrVOdWYx9qz4gtI3SaBqyeJdGUJLzDgvsFXF17laqcKiHeUinRaxa0r7VAFy1gRPI8/X954GVKfLoTuzWWQxMPoWhQtBUP1WrkLQ1Ii4vZO2yvmA8xdzHHs4MnYY5hhCSHUHW3CrVKzaGJh7i36R4AsucNEb6ZvmSfyebJ6SfcXHcTZbNSXPvDHMLouqIrekZ6xG+PR6PWcPPpTZqNmtnRuANjG2Osfa3xHeIrFkiVaiUB6QE8XP8QZbOS3mt6EzZdiDc7GI4kKmcvSruVZHWaQn1JA9c+uUZ1XrX2hoT32CmWx11tyDidwaPfHglbnvtBErkE20BbAkYHsCRtCWHTwoj/NZ5PJJ9w5+s7XHz7Ir/1+A0QGKsm/DFBp3lh1rVZzHswD2s/a1bkr2in6VeeVk5FRoXOZ9WOgajkBljkPaa2pJHytHI2Wm8k70YeBm4CcPWZyzPim9woLVFT8aRCpLJ+bCeAyxX6CsYeH0vmhVRcOruw8PFCMQYFMLIywsrHCpm+jD4f9mF53vKXdp221LWwrdM2Dk08xKFXDr2Uyv4f++/b/+ac9z+xfzr7/rH/E2YbbMvww8Px6+wHz/PN2gXp5hc3ufftPQGxbC+gZM3Ksnk4djfOW0YQNS9KRBRri31GtSU8vdWCe093MfmcfjIdAwcLEsuFyVmlgjtf3yH+l3iWZizF2NYYaaAfUqUC62fJaMrdiTkRQ9i0MIxtjbn37T2cJnSnwdKFp8FDsPARgm0tyl/b7ZG0P4nEvYkCiu05kiNgUAANmcLk3dQqBCKFMYUk7k2kPFUIbAqCB+Ee6o6JrlZ3GwpHIkMta6Px0ppcDmqZHpkdJ6Iwtnp+jKCdUZ5aTmaAmma9LArdWlGb2vLk1BMurb7E0O+H0ntNb7q/25sHkTF4JJ2hJfxV5l6ciVqlpjytnDtf3UFRp2DioYk6lDAefTyQ6kn/VvD858ifceniwswLQiIk72YeVdlVdJjZge2dt+PSxYUZ52fy+dxclMt2MWdHHzz7eIj6imP2TWbPREFjR6ZsQaOxfSliwrI4HfPyLFSqNn2govgiVn6xkqv9rvKlx5eM8B9BS20LB8YKGaa0Y2mYu5hj4WFB6tFUgl8Jxj7Eni5Lu9BS28KNT2+gbFbS96O+7b6vNKmUqLlRqBQq5EZyVC1CN6CBhQF1RXUUPigUKV9AoOLJPJfJwscLsfG3Ye+0vdRY1DBQNrDduZsSmoiMj6TIt4ihXYXCY9i0MB7vecyDzQ+wftuazi6daZ3SyqggXaol32G+TDoyCbsQu7+lCAOha+jIlCNMPDSR4FeCsfYTuqUkGrXwnBsa0Gg0GJgbMPyn4XSY2QGNWoOiQdA70BY/47bFkXUui9G/jsbQ0pD7398n5qcYpp+ZroMEdQ51pnJVJecen2Nz981iElxrtY9qcYh34P1R71MwR0gYWHpYomesR92zOmS+MtyeutHnUB/yOuSJxeFTC09R+KCQ8XvG4z3QW6SsgraiqXa8FD8qpqGsAZ9BQofJ1JNTqcyq5MziMyTsTODhbw+ZcW4G27dto5+eHl1UuomIv1oXoIdawv5z+5jYa5JOQRigLKWMreFb6ftJX/qsbY+aHbd7HON2j+PRrkfEbIth/I7x2PgIY6a+uJ5ncc+w9LIUqFZeQntUX1TPjU9voGesh3tPd45MO0JTRROLEhchlUupyavh/vf3CZkUgom9CUueCGjPlzl+L1rKoRQK7hVQ/KiYHqt66BTiSxJLmLVjFg86P+DmnJt/24GTfSkbm4D2HX9SuRRVq4o9g/fodBXa+NsQMTsCM2czzi47S9K+JN7MfhMDMwPG7x0vaq49+v0RV96/grWvNSETQ0SNm08kn+ACVDqF0Nq9NxKphKaqJhT1Ctas/xD30AUs3zqcBVkPkTea42jlwsyF09g9dy6Ojo6ceGUkamkTD4d4I1eb4TEIIie30QdKJVKkKin6Zfpo1BoWPFpAa0MrqlaV2K1YFF9ETX4NAaMC2iVgtdZU2cSfc/7Ef5Q/UXOjxISiQ9YdYlY+JfraZO5tuodKoaL7292ZW/0dBfICnCoF2qOUwylknstk4PqBqJ6j9RxKHBh01o3WYXn02iJ0M/CLLk2mSs8Q6V+mA4lEgoHcgFzPXOZ3mt8OhSqTCPf1xudvsO/GPhrLG2mubqbnuz358cZeKs1ucfJXN2b39GLopqEo6tqozrTWqlSjaFCQdTEL+3BhQdPec6uhKTJlM8oWJdu7bCd0aigdF3QkXfMuSDTsmL2DntUnsQ+VEzolFAMLA9LvPUS/LhWJqQS1pAVUKv4Y/QfhM8MZt2scHn08uP3sNnkeeax/bz3/WvgvAPYO30tZShnLc5dzJ/8OW2O38uXaL3EwdSDjTAaHJh5i1PZRtHYWgv2g1CCqP68m40gGQeOFIn2rqpUc7xxcVrjgHeHNkWlHaKkRikjBs4JJ+iaJD1QfMGDpAKxcrZhxfob4HORygcquw6MORFz9k7KUKdj3tWP34N2UJpZS7N2V5LBfmfX7cCzqYnGI6MeC+AVo1BqKZcXsuraLwoZCrk+9zvru66ktqMXc1Zzih0IZquRxCWN3jAWERNXeoXsBiHw9EiNrI5TNShrLG5FIJSLASMc0GlStau59c0/8qL64nmXfL6PQuZDEsCzkZoZknMoA4PQiIZm2VrKWHbN34JpdhUqpQS5/3m0Q6UixdwQNJg1UWVdRZyFHqt82P6rVagoKCnB1dWXY5mH/FgVpF2wnFs7TjqZh4WEBQwSqOZlahkatocNsgeJaKpeK9HuZ0VMYvkKKgbkBb2a++cJ3CwwIBo1VqGqNuPGZ0Okc/UY0mXeySG9MJyglCNMmB1RBdYyZPo0dX2/8j7Q/f5ZDc4gU+xJbmsqawF6YM4sfFuM3wq9NX/VcJjE/xdDvs37Mi5kn6jFpTfs8Rv8qoKS3R2/HG28uDLpAbqA7XwP7R+6nobSB5XnLxQRG97sDMSKT1sa+1BXVYWJngoG5ARZKP1pa85n/23qUrnd1KHzKbqbRYmxJtVULVWZ5+FfqThRzIuYQZh/GxeyL/PDgB5Z0XoJbDzdWVayiQdJA1++7QigkhyZzoO8B/FL9xLlE+y4UJlaoLaG+KI+863ki2CdDGE64J54i5isz0nbHYuVjxezrsxl3eBxLdtzArvld3MvmodHAsRnHcIpy4p3Lq2hsfZXPrnqR8EiJrNkKZ2cbxk+eyK7583i88TGxN2MxaDEgNyQEqwZPurzdEyt3U0KnhGJkZURBg+BndIrpRNPxJliPDjI7szITgBt1N3T8CoVKGAeVNpVU2lQy3nK8iPauyKjAoEZNVscsvLO9Kc0vfek4aTFoITEskz6hAi3c48ePOXXqFLf5e3rSLsB5oEd2FT8v/JmohgP4t4L3IG9Cp4aSeiOVz7w/Q2Xf5reUJpYSNCGIgesHYlssrElldmXYedjh2deTkb+M/Nu1avye8e2KEw9/f0jMDzEsf7ocmb5AM2fjZ0OJXyN3msdSY1nDEOM86oqsUNQrsPa1Rv38HEaNRszfHI067AyTDk8SNc0ru5tS4lCCXqse725YRlHvU8inj8XUUEh+/TjiR3q69yTzbibVC6p5YvBEBPvFJQpFqgFeA/ii+gvOLTuH41ZHVK2qdiwvf2cm9iZ0e7sb9mH2/Bj0I0GvBNH/s/7i2PV6eJSmi5mwdo1IbVpXVMf7er+R5ngbgxYDXKpCdM6p0YBVQ1d6pdzGpKoATWQlb+a+SVlVGWq1mpxLOZTHCPFeZHwkuZdzCesXxoQ/Jojv471L7/H9g++F832k4fbG2zz4/gFLct5CI2lLzEkUEgZ/NViMwawHWJP8NJmlA4RYtraglpLEkjZ0//NjJUgxsjFi5M8jRWYLbWK4xLGEk6NPMtjsDWL+FYOllyXPHpbSIcGXUyOkVFtVU6JqIrBPEE4RwrFjvxzLWMZycP1Baltqse9nz1sFAuvF+bfO0/htI4pFCvRG6+HR2+OlHaP6JvqM3zeepsomDk08xMANA7Fd1TY+LdwseLvkbXKv53J/0316vd8L52iBMtY+xJ4PlR+Kc6e2u0trrl1d6bigI3omeqT9KcR+L2OvAGENPTD2AON2jxPp/raEbcGznyfDvh9GzpUcYWz0F1hF9E318R3qi7WfNc3VzVTnVnPvu3sk7Eyg32f9mHREYHA5MvUIyQeT+VDxobBWKVTk383HqePLqf4uvXuJ+G3xrKpcJQLhYn6IIetKPh8cXkpiaCKyViXqFyiiGy1dKJUOwrTxBzL8MrBtnINtTSNX1sQRODaQPmv7EPd5HN7Z3tQcq6EopIigcUGcWXoGfRN9kaYXwMTO5KWasn9nGrUGwyZDlPI2kFRrUytoYOLhiWyw2sDBzIM43HYgWBOM71Bfcq/lYh9mT/j0cGL6+JDrnoxJqx4WBiHYhdhhbGeMWqmmOq+a6x9ep3jsMS5MS2ZGTisatYara6/iEO5A89xmhgQM4Y+pf7Dvx30EjQrS0WDv7dGbWota7vS4Q5lrGddnXCf/Tj7v1b0n6lZeXXuVW+tvMXbnWNx7umPiYIJ9qD0faT6ivh4ODN9FeVIRqytXi8UU7wHePLv1jC73ujD0XBSVr1fS872edJjVAUsPS+pmWRPz9DCGzYZY1XdFrUaMzcNnhPP1H5t46v6UdJc1+BUJtLcliSXkXsslaFyQGCs3lDbQXN0sgA1f0iGq7RzXaDTUPatDz1gPGyMbqi2rcRvjhqmDKfl38sm+nE3U61FkX8wm5VAKvT/sjVOUE2WpZewdtpfub3cXWQYU9QqenH6CbaAtjh2E7qmmyiZ29NlBpyWdGL55OP/OXvTz/jeS306RThhaGJK4PxG7IDuR5aemtva/xb5QW1dL1oUs1K1qzq4+S+qRVN7MepPW51rASWFJuMx2wdnJmRUFKzAwN/hbykMtGF9rvT/oTeclnTG2NcbM2Qy7YDtBMuO5RS+Mxm+YH/om+jRVNnFqwSk6Le5En7V98OzjibWvNfqm+joU1gAeiWeoq3eAb30ovF/Ijc9u4D3IG43GCvPybPxi9pN8cAwRsyN0jpNodHNzz+KekXI4hVHbRok5RaP6MkJubEUR2ZlR20bh3MmZzPOZpJ9IZ9DGQWx+spkNqzdwYtoJNGoNKYdTBIrKZeC3WfCZts3bRuHKQu6/dZ9Hvz2i85LObeBJiYwh3wzh0MRDnJx3ksjXI3lc8hipSkppaSkqhUqny1Fr1/peY9JawVfQduoDtKiaSXP+nOAECVaNoyiJ1/Bw3XW8+gvgbtu6gTw1FGhXNaiYeGiy2CSi9ZXL68tZunspX476so0x7rke8MW3LzLpyCTcewlgUIcwh3YyMS+CH/6azwLYPXg3Zk5mOp3rAJYl6bimXebWibl07S+cI35bPGGjhbVIrrTAojGKwsxGTvT8gY4LOzJyy0haZQJNtV6rHlZVVtRV1wrajX+5rvAZ4Tpaln+1xP2JxG6JZeh3QxmwfgCtja3YBemO0X/sf27/23Pef9f+Kfb9Y/8nTN9MHz0fPZ0ksl5xAfl3NXj280Qik2BgZiAuUq0Gplj1CsHKx4ofg35EbigXqesAXFMvsneY4NBpLWBUAAceBtASA/Y59ym9akL58yRW4v5EHu5Oor5MRnXvQGJHrKV7V3hraxByQzlxP8eRczmHnMs5NI/8iGYzO0ye19tWFq2kMKaQgxMOom+mT9+P+9LlzS46vM2DNgwi/2E+nAC73+04fPQw7j3dub9JoHzKinqFemt38f4uv3+ZspQyphyfIn6WE9XWiq9WqcUCp1wOSKX4xh16vvUjNnluEjWVHGf2IMUnlqvDGwnJj0KjTkej1tBc3czlDy7j3MWdJnMHSry6YG1hjkxfhgwZ1z+9TlNFE2sUa9q9r8jXInVoBjLPZ3L9k+sM3DAQj14edFneBQv3NvoiLYo9bFoYUfOjsHC3oKUFniQ04VlYRGFmEx6926gYq0takLaCW+oFrJ8l09rwHrc23MI52pnAMYHMPDaTPY/3EDTsHQIL1qBWQ/qJdIF2QwKFXoXUm9bT882eHL10lFHbRjH468G01LZw/ZPrZJ3PYuyusVz/+Dp2QXaUJZeRfzefzDOZqBQqHu95LCZ7tXb9s+ukHUvDoYMDCx8tFD9fq14LGkHYuLG8kVHbRwnc+unlZJ4TEkeKegVyAzkZ/hlExkfivN6Zhl4NOpp9dafrGHNiDHE/xjHijRGAILqrb6pP9qVsGvY38O7wdxnz2xiRz1xr1j7WIs1h8aNi9M30xb9Tj6Yi1ZMSMCoAa19rOi7oKCaxjG2MeavwLd4akEC9lRu+y0aIBXMtJSmgQ2sHUJJQQurRVEb+LOgVSqSSvxUP15fpU29Wj8Rb0s6xa13byr9uCYn54vpiXMxd8BrgRWtjK4/3PKbldAvRSdE4pTvpdOtIpBIkUgl2wXbUF9dz4a0L+CYL3X951/PY2W8nw34YRufFnbn20TUyzmbwoeJDQPjt/BDwAwFjApBIJFh6Cc/iz337WN6s23n4d/Y6Shab3KRofDGLPBfpbDO0NCTitQhM7E24uOoivkN9xeTAi1b3rI6yhDLR0QSBlu/ghIOM3zf+pYXC+O3x5FzJYcHDBSKdSPSiaFobW0W62Ue/P+LiOxdx6eKCqaPpS+k2X2ZTT00l5XAK55efp8OrHXS2tTa04lDigHGjMdWa6r89x631twgYHUCHmbrHaxFzAH4j2xKrL9KxmruYYxdiJ1JKvLitw8wOVOVUtaMU6vpWV87sr6MwsD/2XtZIZYidCUszlmJp6YiD11hM0wwpa86kobGGozt309rUysKlizBt8dc534s5zpL6En57+BuW1Zb0/Kwnd6ruYBdiJ2qBaPW17n59l8R9iaxpWfPSwGX/qP2YuZiRcTYDK1/h+rWFL+OaYsozMtBoNCTsSKC1sZXub3fH/qkZZea2RCR2IvtKHgX3C3j460N6r+mN8jkSr9S+lAvDcuntpju2qkweUGx5FHmrPq6lw1E1d+RZXBk7eu+g3+f96LaiG/oyfZ4EPCF8SThGVkbc/eYuBfcKCNocxMVs4T2lhKYwvP9wIuZEiN0BCmk1ToVO9Lz4jIzTGXSc36ajWN1US41JPB1jO6LalYz0cg/WNLetHS9qIbm5gVSuYuDGgSTsTGBn352MIYinS2x46vGUBo0JwRN8xE60VPdzzGc+WxdsJeJeC3LfYsbsGIOVtxX3N98naX8SNfNrUOXpFurtQ+3FrufX7F+jxrKGOkUdxyYfw8jaCI8+Hpg6mIrBqb2/PVHzozB1NKUqpwojKyNC3gpB6iXlfOh5ZkXMYsg3Q3Dr7sbZpWcBSA9Mp1NMJ370+ZFFiYt0tE+E8SRFrpQjVypQKtQU3CsQ58qHgZtoNsjnar+rBD8dLo4/iVRC8bZi5ubO5X2f9zmfdZ5vh36LXbBAq9rvs36ETQ/ToWvUJiKnnJgi+lFh08LadZW+aFZFKXyuf5iOCzpiFyIkUdRKNbd63CLbO5sSFxk/OVrg2t2VtONp4nFSjZROMZ0ISyqivrie2IOPce/ljlOkE4X2v2Bea06VdRUaiRqNBvJu5HFm8Rn6fNIHhbcCFxcXEdgDQmdX8sFkBqwbQKlZKZ6WQnJVa8tyl7HZdzMff/wxGjRIkFA9qxqfQT74DBKK29rfVJO5AwZtclSiqdWg31RD6LUfaZD1IHphNA82PyD2p1giv+9E9ZtVyFQyfLN8UVsU0zu6D+uV6/4jnbi7SgnBBiFM2DKBXNtcOn3aiSennnDl/SvMi50nFvtqC2rJOp9Fl2VdxCTaizbkmyE6Opdd3urCukfruNPjDj1SvwIgdFqoWGDXFlQujiyjY/VKnt5+yp7Bexj580g6zu+IBjUSjQTT2gbqGuuYeWEm+XfyObngJDYDgrhcHCt2aaoQfgNXP7qKjZ8N4TPCaVY2M+OPGZhsM+Hi7IsM2jAII2sjmpt018nq5mom/TBJ/DtFaJZE1tqMpEnVzl/UmlVRGvnXbRi1fRT6JvpIZVLCJ4RjsSMct8RM9JsyUKk6MmrbKJHaraDiAhbhxrxx6w2q7d2Yd3w+4c9zHIoJSn69t49y23JsywQfqcfaAZi9gE3Ros5vDrnJF5O+aHdNEoT1x7jBmOpcgVIu7VgajXqNOvvJJDJKk0oFfVwNyJvVyJVyDk06xOdLPif/bj4JuxLo9EYnHMIckKFHk3ETpYErKbUPJflgMq8un0Ev/v344vn2nsD186Df1R5pfQ02PW3w7OtJbkYurgWuxDnE8bHsYzpM68Drd1/H0NIQiURCpFMkFgYWXO1/la1LtmJnYyfOIwAPix7yzb1vmJI4BQ9/D0KnhLb7/qjXo/Ds64mJnQmLEgV/p+5ZHfL1MXTr0g2XQheUTQc5e9Wd1COpfKT5SFynVDIVBe41OFrb6vhB+mWtjIwZSamrB+WuHXAMcKOpson64nosPCzQN9FnathUmt2ayXPMwzaoLcn9oq5P8MRgXDq7YBdsR/SCtkSpqlXFpdWXcIpyovhRMXe/vsvyvOViXGLqaMrgLwfTUtfC1TVXxdhDO4/U2nlj56WrMW3mZEa1TSNqmZpLgy7RKWcBRQ+LqEivwH+UPyq1jHqDLPRaWuh49yiq5o6Yuw4jryqP8rRy9gzZI5zoYxhzYgzJkmQ6juooaikBnMo4BcC0vdM4lnmM8JnhmNiZIDXUR0ObDxy+MJz9e/Yz88JMGisaGfDaACw6WuDX1Y/m6maGfT9MZw5VyuoITQwl7Np+KjPm6qzd8BfQgbcJK4tXom+iT86dIh4ejCP8cTjlbl1xLZ9BxMZOGBnpHiOTyBh+ejhHfjzCOwXvINOX4dbDjXvf3iM6NhrTzaZEdIloN7ZAoLIMmxpGY0UjxjbGOHRweOl+jeWNpB1LI2RyCKlHU7n1r1t4DfASqSpfZlpqRI1aw4GxBwiZFCJKKwB8af8lDuEOvHrpVax9rOm2spvOWIM2INO5ZedQtapYkrYEEFiDpvw5BRAAWYcmHhKP8R7kLYIfXDq7oFKoRHCHuas579UKuYmmyiZOLzqN3wg/0e/2HuSNgbkBMn0ZXZZ34fon1+m+qjt/zv5T+K7gFIpC3qJXrzBuf3kbm3wTKtwicMq8SeiNoXz64ac4KdzoUtvAzc03sfS0JHBcII53HfGI8iDgXgC9O/QGION0hpDAXq/73G5tuMWdL+/w2q3X/hYYoLXTb5zm3Z/fJc89D7sBdlTnVnN89nFKk0p5p+wdJjydwGc/f4ZdmR3vv/o+5q7moq5jc3UzxpqBBJS+QouzIZVyuLnuTx799ohxu8dhaGVIp2+68uWzL5FqDJBKhRju5hc3xbmqxUCgYfT83ZMbCTdorm4m5scYhn4/lBb3Nl3M5NJkgicE4xilu/ba+NvgN8wPj94eOprHAPPng4pBfLRXF/RTV1RHZHwkldaVHJtwlyk+q3CKMBK7RBt7GnGjUAAUNaasY5xaQ1F8MUY2Rlh6WHJ61GVaZVVExkdiVp4NePP05lPOvXkO+xB7zF3NOZtxloR3E2g52sI7Ze9gZGPUDiDVXNOMskmJoaUh37p+S/iMcOQd5OR45xC9MBpHB0durb/FtbXXcO3qip6RHiN/GUnU3CgkEgnl6eUY2xrraFnWl9RzZMoRuq/qzqPfH5F1Pot5sfMY+v3QdtIXLzONRkNlZSUuLi9xxP6bZmhhyKuXX6UyU2AP6LG6h1jsszA3p/jZs//oPCWAg4cDa1KEmMTY1hgLDwskUolY7AMw1jNGpi/D3MWcwphCtnfejrmbOVNPTuXu13eJmhulA4TVmqmjKaaOwnrR671e9Hqvl852547OYlegoZUhM87PwNzVXKDsfYEN5q+WHTmern2F/Ev4zHC8B3pj4WGB7BK0GFtS4tkJSz9bcq/ncvndy/T9pC8+g32QaIQYY8SpEbhUXWT0ydGM3jYajUYjxhlKPSNKPaIx8fAlaq4Ql6flptFQ0oCyWYlUIkUlV4FcoMVeq27TbKxrqQNALVOjkWro+1FfOi/ujJ6JnhhPXc29ysfXPmbRh4uInBuJRCJBKpHilePFzD0zeeT1iNqCWm6tv8UbyW9g7SvkrOrM65D7yNsVolrUTWQ6f8GMXz4GziKbPIZFiYvEHE543s+06BXhlVpGaHoZ+mb64u9FO4cbtBhg+6oth4ccJnBsIB69Pei6rCuOEY40lDa0y/v91aqyq1C2KLHytqI2vxYjGyOdAmC3ld3QN9Wnpa6F0qRSLD0tiTj3I03mDmRGT6a7ozVPTiYAYOZihoW+Df6Fn9LtGrhnfYHinbfp+X5PHU1dgCzfLL5++2vGD16MSqGipbYFfVP9lzZc5N/Np+JJBaGTQ9u0/6qaqUgXOg61oKl/7H/P/jfnvP+J/VPs+8f+T5hapUbdqkt3YX71T04+kfBG0huiPpk2F99sbo/X/FfwGSRo9miTrFo/qNSjE9PmCsnYmJ9ikBvKiZgTQew9FUikOD+5RpnUiYWPFnDj8xt0X9kdpZk1D3dLXuACbdOe6/ZWN4xsjEi/X0Psc/YbbfJSz1gPI2sjHMIcxLb0l5mRXFgQ9Av0KcoqYsRPI0g+mExVdhVVTsFYFzzG6MYV8sdOoDytnGcxghPzMr7uL+2+xDHCkVlXZqGn136773Bfap/W4jvcl/r3zmHRrRsZoUKLvla4vTqvmkOvHAIJNAxbQ721O6F2QreARq2hwywBNS+RSsg8n8mhiYcYsWWEDt2I1hT1Cqpzq8WkcL9PdFFP3d/uTti0MC69e4m4n+OYdXUWvwzYSavUifsj5vBqb3tit8RwZskZZl2Zxc5+O4lCKII2m9igbFZy8/ObRM2LInBMIHseC0FzqteX+FRsRKMRKB1B0D37uMvHAETFR1GltkTPSI9ub3Vjg7VAVdZrTS/8R/izKHER5m7mXHj7Ag+3P2RZzjKqsqvIPJfJjj47mLB/gpg4NbEzEQKPxZ1QNivFRVQikYBEQJCO/Hkkjh0cubnuJvHb4lmSvgSJTCIW12yVEdiW22Ly0ER8Vve/v09LXQtmk8zYqrcVf2PdAsSC+AXIDGSsCFnB2eyzaNI0jA/S5WCvyKgQKbx+if6FgFEBojj5pdWX0DPRI2BUgI7DqTW5oRzXtMuUuUchkQhObM7VHA5POkzI5BD8R/pj5iLQiBhZG2HmZCbqHmp/r52XdG6nPQRQkFiA/Fs5PkE+NFU0oVaqdYrg2uNHnBpBbFwsHU90RG4gZ/Rvo5HKpRx/9ThWw6x49cqr+Nm0FYlG/DRC/L9HXw/UrW2IZ2NbY4ImBInFzk6LOxEwRpg/ytPLid0Si4mdCe493en+dnfxuP8uSrBVpSKtPK3dNgs3C8b8OobiR8WcXnQaQ0tDnSTXs7hnNJY30v2d7pgNNdMpDjh1dGLM72Nw7dKeZhOEQmbS/iRG/zpapKSKel3QZsy7mcfxV48TvSia2TdmYxdkh1qlpr6onpbaFk4vOi3oX6jURC+MbocgNLYxJnpBtE7CTGuuXV3ZuHojvhm+dD7UmdoJtS/tOvQZ5EPasbR27/nye5fF/79M2xGgx6oe9FgldB8V3C9g77C9DPjXAKIXRGNkbSRy4SvqFWSey8Q20JYhXw/hhyfC8S3PY3r3nu6o3lLRVNXE1nFDyOEivZDzFq04VkFxFezYsJ5/bfgCX/+NyLqYYFvUgHXjSB5tqaTgSjqTj04mqy6L0xmnMTY0JnBxIJZeliJ97osdH9GLovEd5svF1ReJ2xrHspxlmDqa8uQJXL4M1Tfy8BvqKxaboS2hmBM1nqGLxiOVwfRz09GoNFTnVtNvtztOAQMJeNLI+U/vM//8BHqv6Y2BmQHKFCHpXG9Wj8xsKSojU7Z33f6887oXRVaHyHb8iuDkYHqdklHta4p+pCMevT2wcLNg2N5h1LYIlFbabpX8O/mkHkklaVESn974lDEBY/h55c84mOoG9mX6MbTYmnNqxCkW9Vgkorf1jPVEvQR9hT5UK9tRdf2VrlSmJ6PHOz3ovKQzXzl8haJOQaeYTpwfch6lqoWXmXmtOYFJUG1TTsQsoXiQey1XALaowLDJEI88DyozK7H2tWbQxkFU51aTuC8RowVGqGQq3Ne587DhIZGvRYp6G4npQnd+g3cDo9aN4si0IyLFYqNDI+a15hT8VECJXwkO4Q50XtJZLPalBqVSb1rPcJ/hKFuUXHrvEr5DfPHs68mZxk9QyeqJi44jLjqOBX5vcXTQZvRN9FlZvJIji4diUW2Bg2o/SqsoylLKyLucScCYAALHB2JQKgT5UXuiuFV/i4jZERhaGiI3kGMfYk/RwyIOjj9It5XdCJsahqWnJa5dXdsBLxorGtkzZA8hk0Po8U4P0cVRGFkQOjWUDrM64NbNjc8NPsdniA+XBgk0ksbNviiVAkWtqaMpx189jloC2+b9jEQjocF2ELOK6rm0+hL9Pu+Hib0JHe8Z0iTzwyvHi563elI5tRKNWiN03Wg0nMo/xY9Pf2TryK0iFVBpUilJ+5NQTlYy5dEUXgl+RSdxKtOT4dLZhZqnNeR55OHq44Geia7jI9LvqVqpKdPQ2iTh8Z7H2AXb4d5DAFIp9Yx45tcLX18Phn3vg32oPfl38im5VsLVvldJDknGvSCaUQ4rub18H/0ZyHAucwbV3+rEjZDIGaDpj3GJCVf7XqV7R2FN0WruWXlbc+sW+PsLz1GrpVudW43cUC4miF5mfTf0ZdgXw3DPc8friTCfdlvRRhukfn7TtRYqmhXWWHpq6LG6h5gs16CmybiJ62P746zuir5pFUXxRcRvi8d2UhQ0X8W0zpSF29ejJ6+ETXDrX7fwG+5H+IxwpBIpGkmbA9pS20JVThUqW+G3bVJvgkOJAzVFNTrXrfWNfWP2Y3KjAra318oCeDxwBQu2SHBzb1srMs9nEXTjEoYNFUjUKpqbOxL8ilD0L08vx3VzDm/yJmeGncG19tPnSZUazFzM8OjrxbGx+zCvMSfo9kGU1nok7ZHSWllL4JhArP2sRS3CSpdKMWn3e6/fkepJmXVllpgcGnF6BJu+3MSipEUCNdvrxuAG0THR9L/SH5mDjF1TdmETYMOcG3Mo7mNIg0xg71CqlZSnlRO3NY6AUQHPi336qGhFrjLnyBEwd84nryiFd/727evaPOBRrgxPR0/09co5u+Qsg74aRO6eXIYznKTQJNL80xgSNYRfu/2KzEDGkrQlWHpaiuAwbZHsRYv6RRiPnus8qR5QLSS7JOjMIY4RjjqayYBIX66WCho87tLeBI0PEpkttL6dwkBBfO9wBljr+oe1QYbk2VxDIzUgN2IsPfpD0oEYzrxxhlcvvyr6TIaWhiKryM5+O1E0KDDcJhR+JRIJZk5mGJgb8LXT1wxYN0D0QzVqDfe+vUfYtDC8B3sTNCGoHQsDgIGZAW8kvyH+rZ1Hyjw7E/g8X3t+5XmsvKwIfiWYIYeCiA8rIyEiAZVGxePdjwXmmbzl1CvgWlgA8lY5LjlXcQ5wpjKzkpbKFnLTcwGwnG0JwO4Zuzm3tD013YtJSUW9Ap/BPvgM9qGmBjSSanG/qm5VZF/M5t6meyI1unGjMc+GPuPKkisv7b6RqWToKVTt1mZLmRAT2JfY0+tmLxTSKkzHCvOSgaUR5XZVKPQV9L7RBZ/k85Sv9sato+C3ajXZZBtl1JvWIzOQkbArAe9B3nj08qCqXxW5trmir/Hk1BMayhra6YSB4IO+rNisbFaS9mcaEomEKSemUFtQS3WO8Cwi5kSQfSkb2yBbzJzNSNiVgJWX1UsT8uN2j9MBoYLQ1WvhKXxmH2ovashrTVvcBhjwrwHtNJ60ZhdiR7/P++HWzQ0LdwukcimFMYU4RTrR7a1u7ejetKZoUJB8MBkzFzOx2BcyMUTUkw8aF4RnH09M7E3Qd7ajz58RlNuVY1fzGWr1We5svIOt2o4GS2dMq7JJCklCI9GglrQi83JncdpiTB1MkRvJOT/zPLlGuZw8d5IpXYQi5ZtZb4o0rlprqmwSKbb/jr3nRdN2H2okGox/N6ZiagW+Q32xD7VHIpFwa/ot3n7wNhve3YBtR93C4ZPTTwi5cYzM6MlUOwaiVMLTm4JEwbGZgr9t1dGGhlENdIztgWXhDaTyXixOW4y+qT5PFj3h2FVhv8Q5icx/ZT5lyWWUJJagbFKy4vwKHIscGX1iNJV6lXR8XyhyqxQqNvttpuZpDYtTF/9tV0xdHWDhRKUZnJh7gsS9iayqXMU3zt8whjGcHn6afF9TZKZtCf+6ojrMt1US4h5Ccmjyc21tFb90/IWw6WF49vNkzu+vcWDidsacGEOlUxzgTeDYQOxD7cWC2g8xP5Cln8X81+YjN5RzeNJhUo+m6nSynpx3kpRDKaxpWSOAiKIcKckRklRaSurI1yPxHeaLnpEeCbsSMLQ2FI+3DbBlfux8nXs2dTRl4qGJ2ATY8Oj3R0hkEvRN9Omy9L+Cpfw/Z6ZOpkw6MkmHOWb01Kn89tFnTEf5b44UbKe+AWOnTRP/7rO2jwio1c5NXyR9QdK0JDqc6UBtYa1IpajVd328+zGefT3x6O3BnqF7cO/lTu8PhKJ5S10LinqFAMyQS/l3JjeQixqRKoWK1sZW9Ez0yLmSw7k3zzH4m8GinnitvS9yT+E4I6u2wpKeHjSb2ZMfOhz7SCi8nkFNfo1IHart7PPN9EVfkyt+t0QiaWuaMDLnadgI/F7AiwaOCaS1oZWte7cS6xuLS4ELDekN4IdOoVlbIHUodqA4tpjIfpHi3DoheAL+Nv50/bUr6W+kkz4tnd5rhOcklUipNa8lPjKeWQGzkBvJxY5Hrem36KNXq4dGrWHXwF24dnNlwBcDOFv3L53nKDM30cmfAJg0++OV64RrbhNVWVWoFCrsgu1E30cpV5LSM4UOjh04veg0I38ZiV2wHZ59PNu9p2sfXyN2SyzzYueJHbRHZxylJq+Gqaem8kvUL/T/or9O96FWlzT/Tr5IBypHYNypdgzEwEJgQOn8Zme6LutKmdIK/6IP0ZPdot4mD5VUjwFfDGi7CI1ucb9VqST7UnY7bcfCB8+1H2d1IGFnAnE/x+E/0l+cvzu90UlHm/gf+/+W/VPs+8f+T1j2+WxOjj4JvwIIwWddxz5Mma870f01YQgw9Luh7T6rcQzAT/Blufv1XQwsDHh68ykdzz6ixL0DT7q+yphX9DCxM2HYJgH92FxSjX1uJvXWHhg0VKKptqQsVYGhpSFmTmZ0nNcRTSSYL87ENe0S+dcG4TbTh8KYQm6tu4VTtBPBrwQL+jGtauRGciQSCfUl9RxYfIDdFbuhL8Qsj2HT65sAmHNjDmeWnqHuh09pMJWgMDVHqidl8tHJ4r1oF2bLolT0FA1ANIHjAkVNMolE2EmpZ0i1gxAUj9widFzVFtSiXHMaw2ZDOt6RYaVIAYTEiZmzGW7d3ci/k49ErUIjlaHRwPHZx2mpaeHNrDcxsTPh6a2nGFkZ4dbNTexEu7nuJlXZVYzeLtBNvUxYWqWC+HgICgKnKCecopyoLajFo5cHJY9LqLqTS0mfHRTa90Gh+BhLL0sCxwaKfPcAVc4hVBGCoZWaxamLRSF1rZnUmwjvSmNN34/60nV5V6EIkQKL93xHQq++GOj343UEpO/QTUMxdTDFe6C3WLwCGPzlYPqs7YO5izmWnpY8vf2UstQyHe2m6IXRlKWUcWr+KUoTSzGyNqLnez0puFuAuZs5Nn425F3PI+9GHp2XdiZwbCCWnpZiIbomv4ZF/5pIqWdnBl4diOXzWDTncg7pJ9JxjXEl0y8Tp91OHE85ztiNYzky9Qj5d/NZnrucmHdiuFtwl7EfjOW7uO9Ynrf8+fjQcHD8QUqTSpkfN5/+n/cX6TkBRv4y8m8FePNu5LGjzw7qLV1otHBC8eAhalUH5IZyDC0NifkxhpgfY8T9u67oqtN58F+J0RbnFBMdF01AegAle0qoSK7QQZWTBZ45nlhXWlPfIKCWU4+l8izmGX0/7stPH/5ElaqK1az+2+8Y+q3u798+1J5Jh9u6DF4UXq/KquL+pvsM/3E4zp2cqS1sK1r9d1GC+lI5PS70o+a1GtERfNFsg2x5M/vNdpoXNz69Qea5TN5rfK/dMZYelkTMjiDpjySOzz7OiC0jsA9pc2ZHbRsloFVrW1Ar1ToUIwfGHaCpogmNWiMW1OpL6vnW7Vs8+3mSdyNP3Df7Una7Yl9VThVoaNc9p7Xjk4+T8V0GDbcaaCxvfGmxz7mzMy21Le1owOY9mMejHY9Eyk6tPTn1hJRDKfT5uI+omwGgZ6RHc1Uzpxee5vGux5g5mzH619HIDeXUPavj0MRD9HyvJwPWCU6xW/I5bM4n0PT1MvxH+OM/wp8P31hDDJe5iYYu6Go0TFcpuA8MSV9NjZuS1TdW02x2jboQD4ofFaNWtQFPnN2cmbx0MjX5bQntFxOf7j3dce/pjqJBgdcAL6HTA9i+HVJTIfqtd3nlI93n9OI61vi8YUQLLInZIvzeKmwqODsyg7FeM5EbykWnXVtUk6jlWDRGoVa1UFdYR1OVVqdGOHmldSXJ4bV4O1pjG2DL9LNCYevyZ0ISJzwhnBtTb+C2041JhyehUWv46LpwoW7mbmKhr+ZpDTJ9mViUqLWoJbZTLPYh9uwauIuiuCJWV60WOznudr/LKzN/wczJjMxzmVj7Wgu0bs/v2epZEkal1Qi9KsK7XvlsJQNfmUquSzLv/es9Gu1vkHZcSsbZDPp92g/vPQNYcW8sNRY1HJnRQg/9Nvq0Ph/2wcjKiKJdRdj62TL1j6mkdUwTC/ljfh9Dr/d7cTfpLvdj72OTYSOM9RdMi0SVS4VnHDgukKT9SYzdOZaInAh8M3yZsXcGz+KeYeZiRkttC0HjgwQa6JRg/pj6BxajLZBoJNxefxsDcwM8+3pSpEzW/R6Vmv6f90etUvO149dMYBDHxzSAvjCPFj4o5MLKC9j423Bp9SUchzpCF/CO8ebyvctcfvcyk49NpqWuhZxLOYS/Gk5dUR2KBgW513I5t+wcxjbGBI0XRNcrsyrJu5GHQ5gDjeWNYhex/vO4usHKlQn7XClLLePX7r+iUqgEfYw08HviR4eErlQNr8S5lzUdZnYgYHQAUz/bR5GZoL1mpzDE0teWuffnYuZiRklCCXWmTRS4FmDQYkCWTxZSfTmefT1ZkrYElUrFh58LRW/fT3yxUFqwMGEhvd7vRY9VPRiyT1hbDqcc5vHex5QmlTLwXwMpuF9A30/78q3ckVZ5BS6hXTB1MOX8W+fJOp/FoqRFqNXCMwy6uY37jzR0ezSbU/NPEf1GtFjsU+kb8SygPwH+EP9rPHE/x1EUL9xLT3lPrve9TotpPmp9U/wGeYPQ5EovySW6SaXMVylxAIqBHXoG3FQrWbv6I/qY92Vp+VIemT5iY/RGAHHcX7kC334rnOfkybaxsCV8C27d3HRoX3Ov5bKz304cIx1ZEL8ARbWCng/eJyJWjm3pOUpLO2Nv37bmajv7yk32U1f0JjZ+HXTo2OrkAgDqnv8gxqbUcWbJGTLPZrKqYhXb37hAz9s9eRyWRKOlJ4bPO3XeSHoDPWM97uTf4UjKEVRyFbve2sWm1Zt4cuoJ+0ftp//W/gB45noy8fBEzDuYc+n2JYxtjem+sg1AU+0YiMywgercakqTSnHt5ip2OYKgofhiU5GqVcXeoXswAa71jUWuN5rmZjB9Xg/V0qClBKXwoMsDDJMdKY9/ypk3hcRH6MxIqIPWuwouFO6gqWgX194AL0KJWtOR9zPex8LJgpXdVqIv00ejEdgktMl5gE7OQkIkPSCducPmYuFmwdRTUzlZfxLSoNG4kRKHEvyN/en5fk/Rh3S7qkZq1oUaixpu/fiU194dSMCoAFFfKVQznapnLQTFPEDpXMuAHQNQf6/B8T+UTHFAKNZ5PfqTrEHzmXVVAMVpzajJiANTDrBv2T7ifo6jIr2CRzse4b3Cm6rmKsITwnn06SOaOzZz47MbTNg/AZdOLkQ7RxP7LJbCdYWsGLuCn0J+Qs9Yj4UJC9tdQ2tTK/Hb4nEId0DPRA+FiZJS+1IeRT6ip+NXOl3EqhcWOfeK11BbwgbrDbj3dGfS4UmYJzbR5NkECM9PrRa6H3qv7Y2llyVz/pzDifQTrOq+ilc7vIqTmRNWPlYom5XUaYSuggtZF4jcEslPQT+hqFNwdulZPPp4CMVVfRnL85ajZ6KHsY1xO/rI+uJ6Trx+gsDxgSJgCtoK1dprAoj7OQ73nu4UPyrGI8sGpSYcszoz3EtKCN/dB7fubhjbGqOuEnxYpZ6SY8PfYqHqNj8FrsNtpBtjNo1h6smp3DC+ATchYFgAvp19qcio4Lcev9FtZTd6ru4pru+/v/Y7v330m3gtSiVoJG0XlzM7B9srthTcLSBwbCBFwUXc6HUDExcTJvadSOGDQqrzhOKttkMiISKBRvsxrHE0ZbPfZjou7Ej3ld0ZbfUeBys+wKTBhLCkMFo71lKWWoaZsxmGVkacH3qXfJdkfIvNqHAOJWbzPeJlGsb8Oobob6PZdGkT5Y3l3Ohzg9VTVnNy3kleOfAKIZNC2NRnExK1hHsL7nFv7j2enHxCUVyRTrFPpVDxncd3BI4PZMSPbUA+rSkaFByZckTnsz4f9eEjzUdknstk96DdjNo2iog5Efw5+09CJoeIxb6042kk/ZHEwPUDX1rQGbtzbLvP/s78R+oCMZuqmrj+6XU8+3gSODYQuw/aYpsT807wcPtDVlet1unAB1C2KMk6n4WFhwUO4Q6sVa3l7+zxnsecXnSamZdmYh/pTfmdckzrTAlKtKXBvZQ5N+ewYLEMg4YqDJoayfXMpcfdMXR7kI4yMEKkfTw28xhjLo5hzdw1eJZ6UvesTtCCew5W/ev1AXRe2lkH2PZ3tvDxQoJ/DOZZ0TMO9TmES2cXseMeBKaCCpsKJGoJCoVCoEa/moO1j7UIKlDXb8M91gKVSXfcentQmVHJuN3jqM6rplhaiaRZQnCKN+YFt5BIerfRWWa3XUdxaDEevTzw6OUhxjdHzx7FWe2MUZMRLU1tIDKVQiWyH2l99rLUMh7teETIpJB2oNiSEnAOtsNrgBdSmRSfIT48epTHE/8nGGpcUKlg34h9FMUXMfvGbIweNDHxwURkKhneT9JR05H+6/pjH2LPk9NPcC40Iyg1iH1Tj9KhYAEg5GRejI8M5YakB6ZjNNwIfVN9XLq4iGuW1nyH+mLiIBSYhn43lHpFPfwL3J668WjVIzzf98Q+xB4TOxPUKjWv33sdNLCj7w48+ni0A2aDQKurBdi8GO//v2ENZQ3c//4+Xv282nVdzZ07j88/+ZT76v+afeGaopVveo0l81wmXv29dFhYFnZcyECvgZAk0KYW3C9gV/9dDP9pOB9phJhIrVLzfuP7yPRlaDQa8u/k6zAz3fziJrc33GZp5lKK4oooSymjz0d9xHd1de1VHvzwgEWPF2HiYCJ0ucmlJOxO4OTck0w/Ox2JTGA/+jtqxeaaZpTNSoxtjZG9sE9rqwBa0NInA0Tm7CHRYwGHZ5YTmTOPstQy6ovrn8+LuueXFBfxvc8heq3pReScSJL2J2F0zgjWwPS903ma+BT1MDVPbz7F1NEU20BbUedw1MlRXDpxidCMUFStKgzMDTDWM8bNQtAd9MryovhRm/q1BAll9mWcGHOCP/sIncovMgG50IkOl+xI+lcSQ4qHUJFe0ZZfeV74+vjjjxkaX8fgDgY0lDVgaGkoziPBBV9R56sh1hdOzt9FSUIJqypWtRX79JTcHneb32f9TvjMcGwDbSl5XELRwyICRgVw99u7ZJ3L4vV7r2Nib4JNgI0O8ClqbhTNNc2YOpjS872euHZ7OTD7RVBJuZMth0atJiT/GySSbnj29hSLi6XpEOc9kdrQBMJztzFW/fJ3b1pnileOF3VRNVh4udNxYUedjuv8u/lc/fAqnn096bS4E/4j/V9KQdtQ1sD+kfspfFCIiYMJK56ueCkj0T/2f8v+IWP9x/5PmKmTKb7jfHUSzc2+oYRMCiFmSwx7h++lqapJLHzpN9Xw9KdTIk2i1l6sPXzxnKFn6qmpTNg3AbmxkMiLCfuCTM+byB1tubLmCj8G/UhLXQtNZfUY1ldgVFtM4N0dqB895qfgn7jy/hVBjy2mkNZGBRKNBqmyFeXzbqJfu/5K2vE0Ms8I13L6jdOsM1lHS43gWN795i4FRwrofkdIgmjRVop6BQX3C4j5QUjuxkZdY+v8H3DppNsGrL1n+7wYXNKERO2YX8fo0vxJJDSaO9FibKlzrLmrOb8uOML9Lvfpdq8bphVZlCSWELMlhobSBsbuGsu0y69hWpWPT8wftOY9o9tb3ej5npCIvbjqIn+M+QOXzi7MOD9DLJzkXMkh9cjLBb6VzUr2DNnDr3Pv8umn8O67bduiF0ZjE2jDybknKbEvITY6FrOSOOoLa/Ab5sfko5OxD7Fn2qN3SBjY5rhIZVJsA23b6Y4NPzOcsKubUas1GFoaYulhSW1hLY5/ODLs9hHcK16jukpCbWEtPwX/xPFXj5O4NxGJVIJKoaKxvBFls0CBYeFmQWVWJYUPCunxTg/eKX1HLEBoNBpitsRQk1dD2PQwsi5k8XjPYxpKG9jZbycPfngACKj4e9/ewyHMAb/hfuTdzOPaJ9doqmxCpi+jzsaTFhNrjrwQt4ZND6P32t50cRHc1eCUYNIPpAvvz80ciURC/p18HhU/AuBk6UmRtkB49RJKkwSNGH1TfXq+21On8OrVz0t0LEoel3B4ymGyL2eL+wOYVhdiWvmUpv0nUCvVuHVzY1HSIoZuGopJzwiKfHtS6N8HnyHC+y9JLCHnSo6Icn0W94z7m++LSDitOfV0YsOqDZwddhbf2b7tgl7573Jm7JnB7ld3E31GCM6e3nxK7JZYVAoVTUZN6Cv0Kb5STN2zOvG4pD+SSNidwH/XvPp7sTxvOc7RzmzvvJ3NvpvFdzdm2jR2G/693uGL9ityOtR1p8edHjr0ogCVmZUcm3mMnCs5WHlZtbvnzks7M/LnkVRmVtIU3yTqGL1ozdXNVDypQNmsi1iUG8qpL67na8evufPlHUAo8p1adIpBGwfRf11/eqzuIRar5IZyuizvQqc3OrG6ajVzH8xlcdpixu0a1+47D044yM+RP5NxNoPaglqdbQ2lDfgl+THzjZm8XfK2TgFSaxq1hqjXo5h2elq7ziLHCEeGfjeUUwtPcWRa2+AveVxCwq4EmquaST+RzrWPr6FsVuIQ7sC4PeOw9rOmJLGElMMprLdYT+L+REwdTZnwxwRCp4Tyjes3RJ/6BPPybBrNncjKeq7TWFzMxl/W/21HDjzXQlIrkV2EE/1OUOAXTPdPhrC6cjXGNsZtgt7PMyEWbhZ8pPlIDP7+atELopl2appY3E19Pj0+fiwkSTPPZ1LxRKDR0CYRjWpLqI7LQqPWUJNfQ11RHR69PVAOHkJMpxgedNxLs4sZtQW1FMUXoVKoUD0vqrkWuNLhwpfI0pJZkb+CwV8KiHStrk+xUzG3+xRh4KWbtNDel5/Cj4p7FW1IUKmk7Z4lEi69d4nrn10XxbxfZj6DfQidGqpzXgA3gxBq8mvYO2wvCbsSdO7ZpuAxZnHXaapq4pfoX7i98Tb6pvokRuSS755PoacxzTaBFD4oJP6XeFpqW5A6mVBhW4FST0mDqaDZ8q37t1xcLVRj8m7kYXrDlErrSo6OO4rfcKEL+O63d4ndGotjhCNKtZJqq2p87/vS/7P+FMUXceGdC5Qml4qBa+7dXL4c8iXmLuZ8pPlIRNrnu+WzdcFWAkYHcO2ja3zv/T3DfxpO5KpIlHIlUpUUlVqFXYgdSzOWit2x2mdiV2pHVFwUtUU1hE4JpSavrXCc55FH+O0Ygm7+gvdQf1678xpu3d3wX+DPUYujAPz6ya/MuTmHTos7YeVtRcLOBBJ2JZD0RxIrnq4gck6kWIA+NPGQ2DVecLeAE6+doLagluW5y8XEzotsAEol7Bu+j4K7gpaZXbAd0/dMp8/1PoQm+9BQ3sj1z66zvct2Sh6X4HO5gvCEcKRqQ+xqByMz1MOlswvmLuY0lDVg0mCISqYiISKB/dP2Y+rcBgp4MUml8lThGOkojj2ZvkzU6wJIP57O7fUCxeTugbs5v+I8NvW9cKweS2mWQJUlUBIJyS9tkr7KMRClbyAGFgZMPzedzos764w/7f9vr78tFvoALg+4jH6LPhpUaDQCG4GhlSEfrFvDF2vSKPF+n5WmAUw2cOZjSy80rd25tvsmH3yxhp6re7Jmzhp2jd2Ft1UbPSlAUlLb/+ue1ZF5PpOG0ga6Lu9K8CRdgFTSgSTxHQAcGHaAgWf0aTQxJb5nKJcuwbnl5zi54KTO+Jq9YzZhFy/yV+tR/itSlZSgFG8MywvIPCv4qEbWRjQmFlFhXUGVdQ1Pw0ZQ2VXoBLLxt8Hc1ZxDyYf45t43wrt6Tn1pE2BDv8/6YdNBQNc/c37GiK0jmDh2Igk7Ekg7ptvpXuLdjZpOA8UiYWmirpadcfUzyhKL2ey3mT/G/IFEIsGxtxOXBlziWt9TpAfcRaWCzX6bOTr9KKaOptz8XMXByQeF948aEzcb+nzcB8dIR7779mtk30Cn2GbWaSrZryrnc8qBW3wv+Zbtf2zDzsQOT0tPGt5v4FP5pwCM3zteTPqP9BeAco87PKb/xv4YmBvgP8IftavwrFNCUtg5eyfDBg2j67KudJjZAUWDAqOwUJJCc+l5qyfPPhfYRIxtjZEbCHFHQO0i9FqNsc9Px8egED0jPfTlcor5z6wEARVfEDgApdwIz76eYgdI14+6UmMhzCkqpUqka/Ib4cfuhN0ABKYFkv5DOjI9Gfqm+u2Km+4d3bH2scZ/tD++w311vvvSe5fYaLORyoxKzi07R/KhZNy6uXH1s3QeRT4CQKnWpbnTFqJlShmuKRfQS0/CZ5APTh2dkMgkGBa3Yl5rTonlceT54yg7dxOXzi70+6QfVl5W5FTlUNlUyZcHvuRnh5+58fkNRm8fLWgKvtCKJD0r5UK/C/T6QEDXa+mMtUXcF4vLWsvLg9MnVORczaEspYwHPzwg93qucN1aPd3suyj+EObf5bnLmbB/AgGjA8gNUnB9QAHRD0fjkaXCLtSR4FeC0TPWQ/1C15dCXoIaKd1XdydgdABWnlb4j/RnYveJJL+RzI/DfxSej54MuyA7jG3bXyfA7S9v81uP32iobEYjafMHt43dxlr1WibsE3yhPdP3UOxUzO89fyd4QjD3v7/P4UmHdVgvBBOuUc9Y7wU/TZiXcz1z+fTDT5E4G/FT8E+kHU/j3pc3mfPbaIwbjXkcdo/4Xi5UZ1eR/qcQo7hNdeNip7a5x6qTFZOPT8athxs3Pr/Byq9WYl5rTlhSGPm38xn05SAmH5+sc0USqQRrX2tqn9bylcNXxP0Sp7PdwMyAiYcm0mlxJ7wHejNuzzixsGwXYseILSNw6+Emdv69yNhRllpG8oFkGit0aXhfZnk38tg3Yh9Pbz0VP4v7JY70k+kv3V9Rr+D+d/d1gHSZ5zL52vlrpDIpAzcMRG4oJ+anGC68fUHcp6W2hT/G/EHcz3FIJBJRlkD8zm1x7Bu5j8aKRuJ+Fp7FnY13qEgvY8i5IYQ/Dqf/JV9qE7KxDbRFYWxFnY0nJ6d6ER8Vj2GLBfLWZtTNrdQV1dHa2Iq5uzmWAZZY1Fgw+6fZPPhRiHvy7+ZT9LBtHQShs+vD1g9fCqR+mckN5JS1ltFg2kC1R3U73fiSmyXUmdXx6q5X2e6xnfrievaP3M/D3x/iHO1Mnk8JicEXMWisRq+lgaE/jeYjzUeEzxCS8anvJxCSHMKfY/4kdXx3NGoNdUV1NNc06/idWtDWX+2ZyzM2Ld+EarCKyx9c5si0I8iN5My+Ppsl6UuEeW3FOfJu5HFn4x3KkoWuZW3uxTdmP8nT1tHtrW5MOzUNmb6MGedmUDzyDYbd2sf8H6J5djsbx+cMGtY+1jT761FpVUlIcgheqTkgldHrvV4EjA7QAR5k+WbTZCH4QRq1RgdoeDRVmH+0DDLd3+6uA2QFQVJl+Obh4vjR+rK25bYU7SuiNr8WRb2Cpqomocty3S129NlBWUoZtfm11BbUcvvL2zyL+/dA18aKRraEbeHmupv/dj8Q5l5HR8f/Egz8n1hDSQM3P7+p8xvTmpOzE+s3bmCUgQH3/+b4+8BImR7Lp60g949c9g7bS0ttCylHUri57iYajYZeHr2YEzmHOYfmsDhlMRbuFvR8r6dIyQpCDkrPSA+pTIpEIuG92vdE7XkAt+5udF7aGQNzA5IPJnP9k+s692/mbIZTpJNQ4NuZwGd6n5F1MQvbAFsi50Zi5mKGzyAfFiUuwndo2/obcu1HKr/ZAcCNz2/wtePX1OYL8blhbSk+sQfIPJvR7r6NFZ50yThPZI7AgnV7/W129d+FqkUljmu95jq84w6h9yRJpA2uyqnCpYsLJ2cIPqb1Emu6L+2OslnJzn47ubX+FtD2W7vX9R7hq8M5Of8kGyw3oGoRfEUtm8D699fTUtvCV44CDb2WlcpA9nItxIZ6GXkeeegNt0XPSI+3Ct8SfTPtdRs3GCNraSDnYAxf2X8l+vGXwzw4E2VAjXECSCREzImgx7sCW1CofSiHJx4GBL/Z2MYY7wHemLuYk34inT9n/8lGm4201LbQWN6IRq2h0xudmHNjjg77RuRrkXRb0Q0zZzMGrBsgSo1o7ej0oxyffVykaAXQtKbik9ZK7+OnqEl8qrN/U2sTJZYnaTDMQCmro7Giid2Ddot5Ka2erkOJAxOOTqAmphz7UHtGbhmJW3c38Txh08JYmLAQx0hHHMIc8B/pr1PEq8ioIOlAEvVF9dQV1Qm6y/427YAe/9j/zP4357z/if3T2feP/Z8wl2gXph8VOg/4WndbZUYledfzhKDlOSJdpmik/HIcRb0syLuRh6GlIT1WtdFTuSeehpuFsHE+dkFC4mTEjyMYaS8E8nqKGSibxglOtkxwtJvKGmkys6PJzIGnIUPx8nal15pWnCKdeLz7sUCHaGlMTc93qHHwY85z2uMB6wVqD0WdgsKYQty6uaFWqJHqCZO0e093Uq6m8F3wdwAYPzEm40wGBfcKuPGZwOl+vfd1rva/iplaEIZ9evspLbUt+A3zExe4/OAhSFW6HSovWkHwIPSaatFoNFxafYnHex7T2tiK/ThXst1T2PVaIiGFK8m7nsLZpWexC7LDxMuEE03HqTJoxLdUgaq2o4h+vPTeJXIu5zBia3uk5cwLM1Gr2pzsmvwa8m7k4d7DHUMrQ/Lv5lPnZQnuQmB9ZukZHv3+iLcK3iL7olBoSgxLxKXQhdF/RlPWOQ8Gt6EuZWbGtBqCY+YtLEozUTRMp6W2GX0TfR39uLTANKxb5qNWCwWJ21/epjy1nIzTGRgHm2BakYfSyAyNyku83oRdCTw5/YTRv47mwNgDjN05Fu9B3rQ2tHLolUOUPC4hZHIIr/zRputQFFfEmTfOAAJi//V7rwvXqS9j2OZhOHRwIOdKDlXZVdhP7sucObBhg5CkBPAe6I17D3euDG/CpPYB7vf0UTT4o2+ij88QH/xH+nNs0QXW7PuQHxZvZt+r+wAYtHEQ5anl7Oizg249u1FkX8SlQZe4+JFuYm911WrkRnIxsfR31lDaQPKBZDz7eeI9wBunKCcGfz2YX3Ya0GDhjOeoMDEJJDeQ0+XNLjw2gsITwvG+z0F+Nz+/SfLBZBGRGn8wi7iNV3Dq7K7LPy+HJuMm0kPTmb52ervraR3VyiU7gTJOSwsx4F8DSDuexonXT2DsbYxbvhtXv7qK1T4rUU/h9obbtNS20GFmB5L+SCL3Wi5DvxuK3FBOaVIpMVti6PBqB1y7uHJx1UVSj6SyKGkRekZ6mLuZc/n9y9gG2VKeWs6jHYKo9Nx58/jsk0/+I42mW6hZ9/oous3tpqNhkHwomcOTBIfSLlTQxjEwN9Dp7tOKx9/56g4X37mI2103TGyFZ5Z2PI1zy88x8ueRvF3cnvqsMquS6pxqot+IFjndq3OrUSvVRG4RkNIxP8VwZvEZXr3yKl79vHQ6H/8KJHjRouZFkXYsjX3D9zH6t9E6yOuSxBKBxnfriJfSfIKAcv7O4zs6LujIyK0jdbbV5NfwLOYZhfcLxfkYBOrOrsu7IjOQEbMlhofbH9J1RVfkhnLCp4eLlMElj0u4t+ke1r7WGJgbiBo32uJyQeAgahz8MLWBBz8+4Kvvv6Svnh5dXoTpv8S6AN1lcLM5FUcnJ52CgDah+OzpMzb03MCg2YOw9LQkfns8vT7oRbZlNm9deIuup7ri9sBN6Dw2bx/AGD/LJO14E0enHaXb290Y/OVg8XscM29ReCMJzWcfsrPvTgwsDFgQvwBNJ3tq5HXPr0PJna/iuL/pPisKVmCmZ4V38TtIVQ+ptZRhoK+bWNHQdhNqaQsajVCAjvsljqDxQeJ9/fDHDziaOorPt7mmWUwybH6wGcsdljg7OhP9RrQOgtXpmRPjj44nwTtBpF0FxM4++xJ7qsseI4tyY9T2UWIRSnvPT0OG4WCtQKPSiN1m55afY8rBXmxdEE9qp2g8lX3p/aGgX2hgYUDxm3+yOHYxv8z/BdMaNdCCqaMpBuYGFMUX4TvMl65fdeXj3z7mcYfH2AQJxYiHvz5Eo9LQfWV3cX7RkwqVrtKkUu5+dReP3h4MGTSE7m7dqUqpovFCI9Wzq7HysUIqlzIofhA5Zjlk+mUKAeJAb+RGcvRN9KnJriE0OZS0wDQqrlRQ81uNSGMHwHMKRO9sb4adG0bNxEroZS0mdn+de5Aq6yryHc7hWtIPQ2tjbNyEbTXParDMsIRAUOgrxA5SgFG/jGLnazs5oH+AyJpI3CzcSPpDKBSFTAoRkeqe/TyZempqO5S6tthnVFvM0TkxIuhj5qWZqBQqvPP8udb7OhdGNzMtxIWKq4kUPihkR+8dmAHjH43HP7OUoNQdNNe8hcHzYl2HmR0YkPAa1aZZbWNQLcxbWReydND+pqtMGdtlLCDMYbWFtcha24LTod8PFWjTNBoGrB/Ao98fEZHwBTkBpuhnGVBXNF1nbtOOr2eB/TELALkB+A5pS5poNCBTNOETdwiFNADT55R+0W9EYxJqA2+A8zNn/DKC0YzJwOczP94pfYdrH1+j8Yt9jDFSc3nCFzhVT+CVwbX0DCzDKdKJ8vRyntx/gq+zL/GfxlO0sAjbabbc+PwGtzfcxvzteYCAwM2+lM3xWceZcmIK/T5tj6jvtkKgY3XtKqCFSx4LdFynhn6FnmwIU6UTyL+TL2r2WcmE/ZJCk/Aq7kVhTCFX11yl61td8R3ii31zD+RKOa8cGU+FWxxd3+qKulUtnNdARoVNBa36TSR4zCO4dSYaTS/qi+qFwoU2iaqB8LvhZJ7LxHeoL73X9KakvgTOQ7V1tbgeLEpaJCLSX4x3lUrwGuDFmN/HtNN+8ok7yJ33BComS29LpHIpfQ4MYuHPC55/tRq1Giw8LDBxNBE/A3jt19eQ6SVi9E4kncb25duvv+bLDz/gtga6/CXnOx0l9zUw6vOP0DPWI9UvjWSLZAb30qXsg7buXhASp9q/X9T1AXA2a/tNtdS00HIrFrtoa273uI1r5yk0VjRSX1SPpacl+qb6VGlyeeKxhR+WdWdDyBpq8msI6tCJ7TG3ae8ZtbdtQI2nisfhsXjXhKDRwMKEhZSlllHTWIPylJKhZ4fy0PGhQOcW6YhLJxfU14XndXrEaZbvWE5YZ10dT5lE+M2p6lS01LUw5Ov2nRxmzmY4dHDA2M6Y2Tdmi8kvbXGry70uaNQpXC2u59mDZ0w/Ox2ZRI5ZUwhqdQWO2Xdp1u/AK6ltPrVB9564PwkCySkiEiJo0tNNgmnHX6teK+oQtQ5S/sUEf4VNBc7jnfEZ7INnP08djeKq7Cr0jPXIu5lH7rVcBqwbgKGFIUuWAFgwa9cH9OzTwFf2XxG9KBrPPp4iaMC04imasnRgvDhfB4wOgGNfEF4EOZ2anz+DtmvWdjPqKfSYs30scp/rDLo7SNy+b8Q+Ch8UMiZtDHvd9+LVz4tpf05j9vXZ4j7aNTg4OZjUo6k0ljcKOkGtoH5OV2cgMeXRmkeYOZnR892eNJY3suTuEo6bHic9UChMdV7SGf9R/mIcCmBTboNfqgESmVSnc1Ojgc5PzvHAfygaNNi7+uG2xgrHDo401KholauxLbcly2sDDQaZvPLLIcxMda8XwCvbi5w9OUSsicDIyghFgwKzejP8n/jz2ZrPUHyseGnXilQuZc7NOZQklnBm8RlRn1NrMn0Zwa8EI5FKiPkxhsjXI6nMrOSHgB8InxGuk3TXSn5oTUsPr2xW8o3rN4ROCdWh6tQm/Ht/0JuG0gZyruYQObfN972w8gJuPdwIGBXAbz1/Q99EX+zGNnMyY0X+CvTN9Mk8l8mNz26QfycfEOJ+bSydfiKd/Nv54vcamBswdtdYbPxtaG1q5VnMMyzcLcQuuoonFeRczkHZrMTczZziR8UY2xrzR8+f6EY3Lve/zLlRZSyJDqSxvBFZqxSVniHBDysIO/8qhwJukii9j3TTOr7/l5oR4ybx1qcridkSQ/f67lwbeI1Zg2cBAsjPysuK126/Jt6zRCIh+0o2T28/pevyru3YSf5quddzmfHlDG71vMVK2UomB03m1oZbNJQ2MGzTMDpt7MSuh7twKXRhiOsQjO2MGfP7mOc011acm/GAIkkiieGJjIz9hpufXyf7dBqzb8zGws0C00H2VFtWU2tRS0OzhpbaFr5x/obIuZFoZraNv76f9GX3kd0M+WYI5Wnl+Az2wUBmQMtzWvgmZRNFsUWUPC5BKpPi0duDkhL4/fNk9G7e57Xbr/Fm1psiu5B2Lqi3dMXVsz0F8FWrV3FyM8Y+fwhSY0P6fN5f3Fb2lhW/Fn+AeY05ndJn0u0FshPXrq7cHDODmLDPgLaY4f7m+5xffp45N+eIvl7H2I4YHzOmoWPDv9V2UzQoODrtKPZ9BCBmYlgi69euxyvMi0vvXuLet/dYlrMMhwgHzFzNxA7avBt5XFp1iaGbhop+YkNpAz8G/0jk65F49PKg5mkNIZNDUCvVOnmfvzOpVIqj438qivHvzdrPmsWpi9GoNWyw3kDEnAidNWrFypUgkdDz7VX0lOgxV90ssi/sMjTkemsr69at483lyyiKKcI52hk9Ez2S9iWReixVpGCceGgicc/i+HnkzwzyGSSyxmitpbaFspQyLD0tX0q9rpXHARjy7RD6ftxXZ/uL0hkW7hYETwzG3NUcuyA78V2/zBosXbByEcaIW3c3Wt9oxdDSEI0G5K2NWBanUZXtRXVuNU9vPRXkGl5YK02qCgAJ4cvCce7sjExf1kZ5r1RgXZRCo1MPFjwUfK6kP5K4+uFVmCrsM/n9yXhaeqJqVTHsh2HYBtii0WhEf+jdte/SK6gXacZpyAxktKokTF5xk6f2WwEhjnbp4iKO3XGB47h0/RIT4yaSGZ2JSqEi50oOvT/sjZ6lHtWm96gOBSLs28XT2p/Qqi9XAVuRBA6l48KOmDo890ckCtRSBSa19ZjV5BI2LRzpc51UA7kB/jb+yJQyBu0YxD3ZPbouFyg3QyaH8PC3h5i7mjNs0zCR7e1/YtV51WKR7e2St4nfHs/+5Ydwf+pOla0hchMDEnYnkHo4lRFbRpBTl41a2kJIUgiByfm0DGmlMKZQBEJKNcIzKHYs5o/Jf7C+08iXfq+JnYlOnP5XyzqfxdmlZ5l1bRYrnq74H9/fP/Zy+9+c8/4n9k+x7x/7P2EqlYrc3Fw8PT0BYaK0vHiIHX0bmH1ttkglUP+8iabJzIHQXe/Q9RU9NvttxtzVXCfxiEYjwkCaq5uR6kl1OKHf/P5N8p12M+3Ja+2SLq1G5pR6dcHLBfqvEZLjBfcF1LuyrkncT+sI9ninB9mXs9k9cDfmbuZEL4gWeZRBoOarjqnGw8SDXK9cZsTO4OjvR3XoQ5JCk7AvsScwPZjKrEqufniV4kfFrK5cLRb7mszbigpnl53Fxt9GRK0DOGbdxvpZMhr1h8Rvj6e5qhnnaGeCk1xoljtj2ToRjZ4pgeMCsQuxw8Ldgu89vychPIGz446BRSPDQ9uceo1Kg02ADRGzIqjMqiR+WzwBYwJw6+YmIPGlbYm5orgijs04xrg94wifHs57te/xwQfAY2G7ta81Hr08SDmSgu9QX2ZdncWeFXto1Wvl5OhbDAh7k4wzGWSey6Tnez1RqYQOPv2mWgzrSql9Vs+P/t/TaXEnhv/QpkWRGJ6Ih0Kgrbr07iUe/f4ImYGMjpc6sn/fAV7ZbUWDpRsW7hEsy14m6ks0VTRh5WVFx4UdsfG34fyK8yQfSGbElhGcXnQaE3sTkg8m4zPEB0MLQyw82pynAesH6KCFtRoheTfycOjgwK1sR8wqLrLF5QHRb0STczlHTAA/9pxL/8v9CbpXQm3BYmwDbNkSugUzFzNKHcJ46l5Gg0kDvNDA6DvclyenntD7Wm8SQxNJDdbtqMy7KaBRh3w7hKjXo9g9eDe2gbYM+15wWL5x/QbHCEemnZqGV38vge//BRRpyOQQ3N/eRKlXZwwmD0EqExLh6SfSCZoQhETSXpw9al4UHn08xPP8HheOfjd3biZbM/2FSllLbQsOxQ40Wr0cVWvQ3YA4ZRweuR6UXSqD+QKtn32oPS21LSzauojHYY/5c/SfLO28VDxu9K+jUSmEH2DOlRzit8UzYN0A5IZyqrKriP0pFtcurrh2cUVuKBfoXzXC9TSUNpC4N5GQSSFMPyNQZwA4OjqyfsMGRn3wASdbWv5Wo2mUgQEb1q1jxVtvtduuRZhNOjoJGz8bNnluoteaXvT/rH+7fX2G+tAoa8TCq21syQ3lmNiZ/G3R9vK7l0k5nMJa9VoRQaR10htKGyi4V4DMQEbI5BAdx6+1sZXCB4UY2RihalFhaGUoahpqrdOiTgSMCiDrQla74MM+1J761fXMSprF7H2zWT5uuagZqDWZvgzbIFsK7xfqaFoC/NbjNxGNOPKXNmdVKpeKxeV+n/Sj2wpB3Lr4UTEpR1IIny4gex3CHRjz65h2z+NDxYe88kqbXp9MBuVp5cRkXONzTfuOyZfZAiXEJ4Kmj5rShGdU3q4mYHSAmMSSqWTUP6jnWegzTs4T0I5R86Io0SvlXsE9XDWuhHqEkno0lZLEEvp92k9nrfGMPcz972wZ+ctIsTCsndPL3aOos/FEIpUQNS8KuZGcxopGZFt/pVd4DzrFdELieR6/b3phYm+CgbkBJo0GmDd24JH3l+yfoWFa85sk7Ep4rmXhipbG0+mZE4Muu1FvmkO1jYY7X97ByttKTJRqkZcg6Hrm3chD/WdbIL9x9kaKVhVhZaVL66qRaFBL1e0QbNrz+j/x59nl4zTPmadDj6YNMBUmVrRagbGt0DEBsG/kPhyLrOhxqwdKewlqtTAPaMdY/bl87LDDtcCV8fvNqe6RzrwH8wC48M4F7n51l8VZi3kw7wEyqUzsxpxxbobYfXwj7wbyVjlfbPyCPqv6EDQ+CPdeAjhB31CfaKdoNgdupvVCK0YqI752/Bq7EDt6JPfAIsQCm2hBOzZwbCCBYwU94PB3wlltvRqPPA9aYgVEqNxIoEA2MDMQi31pgWmU25YzyHEhPwT+QPDEYNaq1rJurdDdcWLMMaKypqBsUaLSkyDVk9Ja1YpRk7AeW+dbU55WLhZLrLytWNFvBUaNRjQub2TrW1uF+VgmYeD6gWLHhrmLOeYu5ijqFTz8/SH2ofa4dHJB/vynqd9UQ+qeeMbsGEPErAj2jdhHS10LlXPew+LZWsLzQa2BYd8PY+h3Q/m91++CvmNgKrUWptRbulL6uJifRuxgwPoB+I/0581vx3CtjwVP3Z/im+lL/bQ6Kh8JYJnx+8fTwaYDCRUJeFm2IWJjtsRwZ+Mdeh/tzVGOYqxnrNPF33lxZ65+eBW5UolfcjUg+HUv7vMyalyd8akGiUaNcW0R6honpp2Zxm/dfyPtWBo9IwZR4RKGhXoYrSapSOUGXFx9EUWdgtgtsQAYN8nxTkzCK7sUyfj5+AwSOhaPvXaMzJOZpPunE1kdSVOl4CNauFvg1sONJtO2ucC1myujto3SAYi8aDb+NjrF4neq3yFoxQpKHUpxrhDWO+24B+htPpvfc9dys/dNNCnf0FTZTP6dfMJmCMUcjUaCUq7kyPhTBJfNZcjXLtz99i5bO2zF5v3+nGp6G6lKimtGMjbVMWhUPfnGRUiIq2e2PdBBJwfx0OChiDivb1CL529tFYrHL/pE2nfhlHEDi5QK7LaP0wF52NlBWRk88+vDgDlyBixvKz5VZdQQGR9J5MNI3PLdUK3S8OqlVwEBUNJ7jR7BNkuQSCzRaCzQaIRO7ndXr+aG6t93cp9saaH3qlV0/64Xd3rcYUhfIa64//199Iz1iJobJSaGu93pxsFRBxnwyQB29d+F/zv+gjZzviszm2YiL5ezZ8IerP2tGfzVYIyX9uO+6kfK7MswcbMj+UCyALp5rj/XqmkGCbTqK0Am49a/bhESE8Y2bv+HACNgCKhbmrEoSOJT6VEmHZ3EuVXnqGisgHnQ9X5XslyyMLYwFrvztfNyg2kDDn7tx93dgrvCf6bD/qj9zL42u/2zW9pF1GvS+c09P3dQahCyikKqbM0ojCkEwMbIlsCC9cT6jObXeVcYrdJVJ9Tk5GNdLBSz1723jq8880g+lEzsT7EM+2GYeO5ai1oaPm+gw5AO3Nt0D5m+jKlzpzLCbwSBPwaS55mHz0AfdvTcQdT8KEb9PEr8ju99vidoQhCmTqbE/hRL7zW9dbqOdu6EcaMNmXVtlnhf2rGb1XEiHaOE+bs0qRRDS0NM7E3QKy9Fr8mEViPh93/l/Us8/j2eN1Le0KEulatkoFKLsW31tWoyzmRgbGdM91+7M8B1AAFBuoWpF63/lf5cTbnKG0lvMGjDIHJzQa42wa1sLu6ueiTuTaSpoonAsYEU3C/A6JQRUf5R9EjqwQObB3Re0lkEDWg04FQ5Eae8LDrfVFKSWIaFa1vH9ZGKj0jy2I1BswH2pfaY+tnS/zOhUFVR2IyeUkpE2lJkqp+IeuRE7bwazEMtdcYAwHrL9aR+k0rToiaMrIwInRLK7fW3MWoyQiVX/S09ndYcwhyYc2PO3263DbKlz8d9sA+1F7v/5Eb/Ps314ndauFm0KyQm7ktErVTT+4PeBL8SLNIXam3amWnimDFzMtP5Pqlcivnz59hSJ+iZhs8Ix8jWSKegPmH/BDFeAQFIqaWuq3hSwY4+O+j5fk9Rq2nwl4NFpoapJ6aKx7VqZJyNTSSmUwy2Dd5IrS3Z7LcBH5kTmZ2mkJt/ibsNZ+keo2G+EhwRih47T/9G0Imf6W3fhzr7Bu53vS+yvfT/on87uje1Us21j65RcK+AyDmR/2Wx7/Dkw9hW2DL4wmCM/zQmo0MGmWcyqcquYtimYTSVN+Gd7c3FwRfZuWwnBmYGRMyOAKDoYRFjfuvB9W6NYmz75M9UypNK+Nb1WxzCHVDoKSlwK8CwyRBNg5BM77y0M67dXOkY0RF9mT6rL62m2rkamwAbkv5I4uYXN1mcthi1Ro1JvQl+GX4onBVioVaj0ZB9KZsdn+STahCBZkAEzp1MdFhJtD/nYr9e9J0iUKoWPSxi8JeD+a3Hb8y/F82fo/8kuceXzA5oA3+oFCo88r2wrLek2qqaBgsDFDVNbO+6l+CJwQSOCaSKeRg2G7J081KaLA+g0UzGxs+GkEkhOl2+FjUW6GXqoWxWcv/7+9QW1DJoYxuA4P7398m9lsvIrSPJOJMB1oCnQFc4MGogUokU126uRNVHIZFKuPHpDSJfb8tPOUY6Mvf+XJ0CkVRPikOYA2bOZsT9HEfWxSw6vdGJxamL/+04EO//hXyeTPb/H0Wg3ECObaAtLbUtOEY46khmNJQ1sMlzEx0XdmRivziy4r/mnZZ7tKgbcHSwYNq8aax168vl2ZfpEdODnM45PFzwED0jPQZ9NYhea4RC3538OxxOOYxXthf5zfm8TDWkKL6Inf12MuyHYUQvjCbjdAbmbuY63X9as3CzALf259CaNkZ/0ScCIXbNupCF/0h/keUsN2Isfs9fd9C4IILGCVSmGg3U23gSN/IjXp8JBbeTODbzGBMPTcTC3YI8221Um9xj8nU/pFjgPXCRCDTWWouJNbEj1uLh2RbLefT2YOyusWy9LxTr9GWC/yrTk4n5xhZlGx3uML9hGOsZi3rUt29DWm08KZYCYN07y5uApQEi0FgulWPcaIzjXUfK08qpLazl/qb7dFnWBbVZ21rSpBC+I+mPJMGX7u4mxs3VFtVY1lgiM9Jn5Hdt+QQtq41R0TcEPO6AsuV99F/Q6dXGvF7JXlz/9Drn3zrPmN+F2GdZ9rJ27ynjTAbPYgW2My0g8vzK85SnlDP4m8GcW3aOjgs66jBpvXZLAE2kHU/jxmc30Gg0mDYYopFouDvQlZl+DlScSSbjTAatTa2i7rZfhh8+qaWoDYx4t7qNDs241Q2zxnDqTB+TFpSGzMGQwgeF3P36Lp3f7Ix7DyFXo2xR0trQir6pPmeWnCFxbyLvlL0j6hT7DvVl4qGJujI6/9j/mv1vznn/E/uHxvMf+z9hBfcKuLnqJiUJJeJnGomsHZewCCKUSpGbG6NnpMeix4tEPSKtPQ0fSWpvIQm+2W8z+0bsI/VYKh9//DEdHnUgKSQZwzBdqpqAVztT5tFR/JIXk0euXVz5SPMRA66vRb+xGtu8OKqzKsTt+ib6jP51tCiq/aKZuZhhP8KeCpsK/G38GfzhYEZuHUnAqACWPFnCzIszqbGowanIif5XuvI0vpye7/UUu2PEe9aokTynU4r/JV6kDdVaqUcnciLGIpFIWPpkKasqVzFuzzg6xroTmBaISa0SeUs95i7mePXzwtRJQMN0eNwBmUomdoH8MeYPjkw7wqCNg1iStgS5oZza/Fpub7hN8UOBdKg4oZjCB4Xid7t0cWHS0Uk6ougGL8QPXZd1ZfrZ6Ty98ZS7X9/l5rqbzNg7A/en7qQG52DoZEX+3XwebH5Ac3Vbkv5p2HAShqzCwMqYTks64dFHOL/kL73najX4DvMlZHIIix4vYkniEtLdv+Pn+ZtJ7hwo7jflzymETg1lbtpcNN4aRm4ZiWtXVwLGBNBjdQ+iF0bzkeYjjG2NOTz5sMjpn3Mlh+BXgnHo4MC5ZeeozKqkPL1c5xo8ensw6fzr1Nr60GxijdTXi74f92VJ2hJM7ExoLG9k0IVBKOVKrgzMFNFITVVNFN4vpDnSjl2vbsW03pSGTKFj6f7397m29hoAF0Zd4OKgiwSmBnL9s+tiMULfRB9FnYLTC0/TUNpAXWEdjWVt2U63bm7YhwloP4lUItJQAJSllHHj8xsojCxoNm1zAoofFXPlgyv8GPgjhfM+we/+XgLu7ODOVwJ1pPdAbx2xX4WRBfU2HmQ8NaAqu4pncc/QaDQU3y1m0dZFjDg6gl/H/EpVtq5W1if9PqGzS2f6X+lP9vtCx2dVdhWDNg5i8rHJXOl3hfioeB5GPdQpTjlFOYmJjEFfDuLtkrdFukyfIT6sLF5J8ETBAev3aT8WxC9Az1iPtONpbPbbzPCfhjNiywgsPS11gocVK1fy3rp19JLK6Cc1ZC9wCdgDDDY0pLdMxnvPC30tdS1U51brUG0O/NdA1qrXEjg2EBMHE7q93U10xrR2dMZRtkZsxTbIFtt+tjqBne9QX+bFzMPUyZSkP5JoLNfNWodND2PAvwa8lCrg3LJz/DHmD+qe1TNq5yvYh9pTV1TH/tH7ubnuJjv77WRr+Fa2ddrG+eXn2x0PAu1v5GuROuh4AFMHU74y+grLdEsapzfq0LElH0pmW6dtyPRkoq5Na5NuF4R2rJi7meuIl9c8raHwgVAcNHM2wy7YDqlMSvGjYm5+fpNLqy/x4McYDm/MouT50lCaXMpXDl9x77t7AEilYF6agXP6FVpqWxi+eTh6Tvr8pxgrB8C40pjgezdJ3hHLoVcEGkRtB1ytRS1/bvqTriu6isd49fPiX+d+BeBw5GFev/M6BfcKuPfNvXbUq0/DRtDvs350nNcR1y6uqDVqcW2ps/Wi3KMjEqmEnu/2pOuyrmRfzEZaUYlvpi+FLoVInQV6l17v98LAzACNBiQaYV3UoEbT0sLxWcdJ2CnQZWoDHtN6UzyzLWktq8GthxtvZr8pUm4C1GbX8uT0E5prmum0uBNDvh2C8oVFr8WwBZmx7vrbp/o3ip2K2T7vIB6jwrm1/hZnlgodz9ouj5TgFOoWO7crJmvBMVKlApqadLaN/HkkFbbNqGQqnNKOYJp5irpndRQnFKNSqHBcEE1iRBOWipkU+fZEYdMWYEfMimDA+gHU5dbhUe3Btehr3P9eIPUxdzUXkxjd3bpj1GTEuF3jiN8Wj76pPlZeVmJhViaVoZFqUEqUlCQKg821myvBfwST45XDsBXDiP81Xue6r0y6wshTI7nX7R42N2yQyCR85/4dsVtjn+8hPM8ayxqyfLPQGAnUaRKphJ39d/L+F4sJTA18/t5UfGfzBZ8bfC50pNjo/f/Y++/oqMqujxv/zEwmkz7pvYckBFLpvffeexMQBAsqKigWsCuiqAhiQRCQJr33TgiBAAlJSCO99z59fn+c5IQR7/K87/Nb633Wcq9134YpZ069rmvv/S3k+wgsgck/TWZLxBb2T95PcUIxjeWN9JH3wb7Gnujt0aSfSMfS0ZKcSznkXhXkjpKT4aKg+k1TZRPHFh4T2X/iPeDSjozJqwifHk5tfi2F8YXkXc9DefR3zNQNWFcX0NwiMyyRSlh4cyHqjZO4OLKKhG453BsYjczBjqj5UbhFuGHQGqi3aaLBpgHvAm96xfaisawJ//7+zL0wF99+vkiNwvNf/EexKAkVOjaU4d8MZ0zoCywtNnK6TyPNVc3U5teK89zzd55n/9T9bFyxkc9Xr8cpxImMUxniuW5dJ3mkX8H6xlnqCuv4yPwjzr15ruX+BJ3CmgfDVyEbPpTmymb6rumLYztHziw5jFxVT3lAN9J7zkfv5UvakTRyLufgHu1OzoB6Nq7YSI31TfSSSrGBDOA/yB8QPN48z3sSuFAoqETNi2LOmTlYubcV1Z2Cnei0uBP2fvbsHrmbax9f469x8Z2L4jE10US2xxbGHB/Dgh+DMGhMx1Wdrk3mxyjR4z84iLfr3xYLyZk2OzDIDCRF3qVR6cXFdy5y7vVzDFg3gKYb2bz+zevY19gTmRiJ12Phmey+ojvtRrZDb2x5YCWwa8Eu+q/tz5MLT9jWZxsl1yrwqJpCu8xA3vJ6jY3fbqQ8tVz0wtS1DIE2VfnYFDwr9+7W0nOq9I3Bb0xbUdygN3Bt2jnGHxuPQWogJ7ARvb7tXLfK0zpXOnNiSiM50eOpvJ/H8u7LGCAz+7cNMxAaav3kZlw5dEX4vZZGRezXsaJc3qfXPwXApdyFgmsFmFmY4dffj7COYYwNGYtXoReKPxQ8TnlMXUEdTWVNmCnMaNh6mWHnhiHX2eMiDcGzqyf91/YXJdcvOsxEYpDgWKZFW1ol+F9+PJMRQ0YwHP6tDNlwwNgOplyfwoATlWgt7OgwpQMJPyVQl1mHvEiOslbJhtc3MHzLcB789oDL714m4dcEcR57MfhFHGsdKbpbROw3sdQXC8zxe8XCcd+PvE/w6GDOrzrPtU+evS9BKJA/LTPXKtV5aPJZzNaMZdLuSbxV8VbLZ4V5yig1UmffgEFhxdmVZ7m/7T4A0t49yOo8FYwSNAoNBpmRpvImSpNK0TXr2u4/2iTD7nx/h4SfErCUW+Jm44aTpbBOUZUKY1TCT6bjc+/VvWk/oT2DPx3MypKVJswYiV6HsjSdirQK/Pv7i012UQxAKsUgEfyZtkRs4exrZ7n30z28TvyES/omshwW02w4gZWbLW6RbsjksjY2ormWrS/spblrX3aP2M2DrQ+wdbcVcpTfOlLZXMn+6fsJfS2UhpIGrqy7IsrUOVgKRd47M++YKCTodKDQuRFQugILlNj8IeQPNz6/Ia4TkyKS8ErxFuev1lCrhXEis10m14ZZY+drz+2Ntym4LYBYa/TFNCmycS1zZdG2RdRca5OFc4nyJKXvEgr8rXAvcafjI2dyLmaJ3209Zi9bL0auHknfd/uyOXwz6SfScYtwo+RgCTf63CC4JJiy5DJ2DNwhSrqBYGvRqGnk2ifXRBnjv4ZBb+BL5y/Z3GEzcRvj+Lnbz0jlUpbcW0LMwhg2hW7i/m/3Uder+dTmU069dEr8bn1xPbnXBWWgRbGL6Lemn8m251+ab8Jq+2v49fUTwRlTD0w1kcA36A1UZVbRWNZIx6kdWVm0kok7JzLimxHEfh3Lz11/pqmyCUsHSzHf+2tYuVgx8vuRz/gBtkbm2Uz2jN1D8f1iBnwznvh+VqgsBUlXvR6iF0ZT4xZC3uOPSW48yw2MXNXBbGAwwn/PaVRc0+u5W3KFeN0dk+3HPBfzTINT06Ch4HYBvn18TTzk/lUM2zAM5zXO7Ju+j9K+pdh52zHv4jxeyXoFgMKzhYQ/EtaeNuam56G5shn3fEdcy1zp+KgjthXZWLYonbh0cKG5uhnzBmGNNunQJLruTkJuJWfkdyOJmBmBl50XC2MWcmjaIeYcmMOo70fRcXpHJu+djK2nLUaMOFU6MeHoBGT3Tdezu4btwuzmVSRGA1pLOwxaA7V5taK0/dM1IIlE8Bi//fVtjEajeP/X2dWR77QdnQ5ub7zN9c+u01TZhOGTOBZuf43Q3Dexq3XCoDdQV1CHqlpF0h9JjDvaH988X/L8KmhSumM0QvCoYKbsm2LCgL80+BIFuwtQ+ip5fOQxCb+YjnHF94p5fOQx5jbmvKd9j+5fC7OgRbMFZ86foaCkgI5TOzL2p7FYu1nT7/1+hI4LJfVwKjlXc1DYKvDq5mXCVrN0sGT+5fn0WNGDwZ8NZu75uf/xHvhr1NfX/+cPPRXqejVJe5JES5LW0Kl1NJQ0IJVLmX9pvsjGAqGR79dfkE1Vas0YVRPEkLDD9BpayObfU5jUcTKGAuEitktsR0VTBeWNgkSrQ4CD2Khbc2kNAD1u96DgvQIK4grYOWwnWefaFCocghwYun4oPr180Kl07B2/l7iNbbN2yp8p7Ju4j5qcGlQ1qmdy9+zL2Vx69xINpQ3sn7SfA1MEa4SC2wUcnHWQgrgCCuMLObPiDGXJpufgKfL0376m1YJvX19BPrlF2rHU/hj5Lts4O/QPCkI7m3xe/FsiEf5XV0fs17GUPCgRGrw/3WPOdqEp/uvAX9k1a5fJb4/+Q2CFKmQKLi+7bOIdrNcDT3nLztg7g8vvXxb/HeYSxsbXNjIuZxzdXu5Gv3f78Vr+ayh9lOJcEvUgCv0aM5oqmjg486CY67dS+7Yu3UpSn4k4dg/+61kB4GHUQ+71NOfMq0JNBCC3Jpc1l9agN9Pz/effM/fcXNpPaG/S5AZBVe3+b/fRa/Skn0znygdXxHUnQH1BPVWZVWgaNBTGFdJQbGrj0ho6lY7ihGJK7pdQ4dzIhSEXAAkSCQz6eBDvad/DMchRXEOdG3aO8xOD0f2FoxVe9jG9H98Q/+2q8KGuoI7k/clifRLg4Y6HfOn0JTlXc3AKdRLBn63h2M6RDlM6YKYw4+HvD7n8/mWufngVg+6/NI3+J/5j/E/HvP/N+KfZ90/8n4iqjCpyD+WK7A+A6kGTmHtuLqWJpaSfSEevadOalhj0NOeUUl9Uj6Wj5b9FnkXMiSBkTAgPtj0AICgriLu9VSjH9iPjdAZx38dhNBhJ2XaboISDVJh/SdjVTRhi49gxaAex38SK2zIawbK+FP+kE1Q+LECn1vFz15/5teevXHznIpaOliT9kcTR546KTau47+IoO1lGRFIEUomUwCGBdJwmNAXNbcwx6A3ItXIygjP47bk9OEX7EjQ0SPxM6zEH3d1PzGmhEPFawWtM+mOSyXE2OPlR6R2FRCrBytkKSwdLlD5KDk29x6PwRww+monn43Oiz43cUs7gbweTGJGIVZMVduWZ6GsbUNerRZmoA9MO8Fu/3/Dq5sXLGS8TMVsozJxafop9E/eJv23rYUvYxDCUPkq0TVrSjqUhqSh/5loM2zCMiTsnioukjOAMwIjBAH1W9+H1otdxbOdossgGkNsohEV8SzPVz15o+vW+0Zvg2zvRa/R0nCpIbyr9lFhUWyDXyKlwqaDJRkpTZRMJvySg9FMy+Y/JLL67mMBvA8moFJLaiJkRDPl8iPh7HaZ0YMr+KWIjKPtiNil/puDbxxc7LzuOLjjK9n7bqcmpYXP4ZuK+F47nu6BvCYnbSYVfZ6yfn2XCrmquaqb3LUFWKyWyBLVEaE71eLUHlk6WpKtOAkJCk74wXbx3miqaCJ8RTlZYFnXKOjomd+TK+1fEgqNHJ2HBatAZaK5qZnnycibvmSz+7tQDUxnymXBsmkYNuddzqcmpAYRm370f72GmbcaqtoiK1z4R5GNHBbMkYYm4DWV5JhYN5c/404nRwqQ1Gox8F/QdP3f5GYxgG2TLpYGX0Mq1FBwrEFkPrXHs+WPM2TSHeV/PY9QWgbF5ZsUZtnbaipWTFdf6XyMnIOeZnzPo2gpOFkoBcd3KMjRTmGHjZvMM8wwEf49eb/XCzsuOXcN3cf2z688c02uvv8477xVQ3O4dVtqEMV3hybdhYfRfs4bcggKWzn+BlbNXMr/bfL4N+Nak6W00GmmuaqYgtgCjwciw9cNM9PcBzG3N/yNSNuNkBgdnHqQyo9Lk9fYT2tNpcSf2T9nPg+0PAEg5mEL2pWyxeX89zZUpU6C4WEjaM89koqpR0XmpYOgcNDyIyLmRf/1Jjiw4wtFFR//tfuX75JM4ItGkAJC4M5Giu0U0ljXS791+LL2/9BmkcJ/VffjA+AED1g0gaXeS+Hr85nh+6f4LdYV1qOsF1qXRYCRsUhhDvxpK2rE0Tr90iuRVu/iyxyGeXHiCzFyGU6gTlk6WxG+Op92ZTXhk3sQz4zrqOgENqLSz+x95IcllZujkZrSf3ZlJf0wylbFDKGgpfZT0e68f049MRyKVkFdoWngf/OlgVuSseMajscorQvQ6fefiO7isd6Go6Vn/idawcraizknFzd432TtzL3azI0zeb9I2UWl3Gds6W7rG+iCrKGPG0RmiRExrgykjJINNr1/Ctl80cks5DgEOyG3bnomFry5kz5g9VGdVEzImhM5LOpOd03bMThVOlCSWsG/iPq6svQKArUpAlcr1SoqKIPN0Jsl7kwEwkwoFmSqnKlSRVjSUNPBt4Lei90DrmB7w4DBeuwVvhPu/3afoXhF2XnYU9P2MlKhmQlLtsSvOIPbrWLZGb6W+uJ6IeaNRe3+OV8M6CtsPRuPgxo0vbpB1LgvXcFcKYgv4fdjvTD86HY2LRkTi1xXUiVKvL3d7GZWFiseLHhMxKwJ1nZqaXKFZf6/oHt/c/gbLJkuMD42Ezwhnde1qxmwZg7KLkhz/HHJ65eAS5kLc93HsHrUbdb2a4BHBWEW0SD4Z9Ni429D91e6iHJJEYpqly6zMWZa0jIHrBopNOa1cS3hSOIEpbc+6hdKC6pvVeBV64W/vz6hPRxE4NJDUQ6k0lDRwZN4RhqwZQkB2AE7fONFxakckEgnlKeUcmHYAg0Hwyv1tdRofW35CzuUcZp2cJTItW6+FUSpDZ2bByRdPstF3I03lTTgEOlCjyEZe9glhN3+lMrmE3Gu5YqPQoykA60ZHKuwuUmlzFZWlAxO2TyB4VDA1uTXYNlihVqi50+0O3738Cw7Bzli7WhM4OBAbNxsm+E1gXf91aE5rxHvDt48vPV7twaafFRQWwvr1cPrl02z03YhOpePHqB+59/M9UjqmUONQg0ahRSqTcm/rPU6/clo4FiNkua3Hovo6FtmpmCnMaDeiHc6hzuL7raHXQ8IvCZxYeoK864J8YFZIEyXKY+L5eeHhCyy9v5Sl95eSPqKGGrMazpuf51fZV7zzsicBEhdeX/AanpM9Wbt2LQmdE5hzeA6JpYkm11z6N1mY0Wgk93ouFammgKFL717ixmc3uPGZkNw/3PqQtWvX4l3gTalHExKJhLybeeTdFPZZoxEaKoMvDKb93Uf8VbX4kcOnJv9u3W6/9/phlEpQKVSoLFTsnr2bWxMEGc0RG0cQNS9KHP/6+/Xn17W/4trRVWB9ZlZhabCg85MDONUPQSVVoTKo+K3vbxx9Tpg/mtU64tuN48ike/htfYf72+6z0W8jRXeLTO6/1r+vfXxNmM+MoCppptmimQNTD3B+dBFGpNz4/AbJ+5Ox9bDl2qpGPnnnExHwpW3U8KDwFvM0bWjzfxcLVGqsk4SCknGjEYPOwJwzc5i0e5J4bQA6ru/ImqY1uEe5M/vUbDpO7YhcJicxMpFNL24ixzmH5cnLmbJPkKasjZGS45/DlEOLqXxlG55dPBnwwQDR9xlArpUzf8dE6k5eFyRR1/TjxPnTuHWOpDcwAEwARv2B3oDCbRjKCb3QyrVo5VDn4Ccg+P2F9WmeTx5auZZ6u3pkVjKxgPW0D6vtz7Z8G/AtWeezOPf6OZNCEcDV0Vfp/WZvHu15JPqxtUbG6Qwuv3+Z4oRiPjL7iIvvCCiCwdavEv1kF51yr+JibzpPGY0gwUxYB9aaQ0MDcd/GkXZM2Lbq8EHsc7aCxIhHkQfqogq6Lu/KWxVv4dnFU9xvi2YLLLZZkH4ynTln5jD1z6nib8ikMrzzvcn7PQ/vngL4q7WZCAL4KnJOJApbBTZuNiYsL5lOTXD8HuJ/iMdoMLY1MFvuTYuGCgw5eWCE/h/0p8PUDjSUCmvF1OAz9LxipMvl+0Qv7c78y/OxdLQ0YfYZJXqMOgOFcYU0FTQRNCKI2adm89CuzWtaZxAK2VfXXhWbffum7OP2otsc//w4vn18yY/NJ+mPJDTNwsPdaJHBRfXn7MnZw8jvRxI5J5JOizpR/n4wj9s/5tM1XzN261iOLznOetf1GPQGdDojGrNK6pR1lHsqMegMnH3tLBktXk+tBcca+xoedLNG09zMjkE7KIgrIO1AEj6PTiMzSIjrHsevL9wnaUcC+6e0+mYK35VKpNh62OI/wB///v4C+ORKDp2ud8JcY86MX2dw4a0L+PTxMVkTB3wbgM1nNlxde5Ubn93gyrorIju0NSRSCW4RQsNNVaMiYGAAPj198OjkgZmlGTKFTAQx+vX1w7FdG9Do8eHHbO+3/RlfutawdrUWWcl1BXVknMow8R43tuQ1fxeqahXfB3/P5Q/aitk1uTU82P6A0sRSYT2rN1KTWyP6NYOQh33t/TVnXz+LpYMl3V7qhk/PNipQZXolGacz0DRqePDbA9JPpJN6KBVts44Cu3X45vqycHMQVcdvMnzDcPI8/XmU++l/9Kg+YzQguWhk5taZxG+O/xefFNiSwzcOp//a/s8Arv8uImdH4jDdkVz/XO5OT8MjxgMzCzORVYIOtAFaFmQv4M5bd6h+Us1Gv43c+uoW/gP9uTEymfiu8Yw7Ng73rJt0XT2YeRfnMfPiIgZ8PQ57Tw9CM8eSFppGQQuj9OlwtnJmYthEBgUICipuEW6ETw9vAccZKXcpp8OvHVjw4gIK4grIvpyNRCLBf6A/DcHR6M3MUTRU8vjIYzb6bRTHqNb51DkvgfIdpxi1aRQrclYgkUpYkfcaX775JU1WTbRLTKY2o4yHvz/k/i/3RflBu1ojw892o9epI0itrXi94HUGfTxIVD2xq7Mjs8tkikP6P1PzeDpa57rph6bz0uOXTN6bsGMCHxg+EM91q7xiWGoY8cPjWfVNG03NTGGGuk5N/A/xHJl3hBuf3sCgM6Bp1PzLor9ruCt+ff3QNmuJ3xL/t955/xtRk13DoVmH2Nppq8nr+Tfz2eCxQcx3nw5LR0tmn5pN1+VdafQK4eHg16hxE5rmRiMcmX+ER388wtLLkhL3Evpd7ceJLidoKG1A06BBVSvU6Fp9Dq/2v4rXRi9UNSoK4wppLG8bB5Q+Snq90QuPGA9k5jIm/D6BTs+3KZdUZlSSdjwNVY2Kg7MOstFvo8m+5t3I4/on12ksbaTH6z3o/qrwpNbm1fJozyNq82oJGhbEgmsLTLzY3J7EorokgKxvrr/J/in7W+YrkGrV2FY8oTa3BjsvO9qPby/m5q2A0KTIJMq9Azj3xjl+6PBDS+1E2LZEr8OmMgfzvEzOrTwnepW2G9eOB9HC+c4py6GsugxVjYofOvzAlXVXSCgWGs6jgkeRWZJJc1Mzcd/HcWjOIcCI4Slv2ZOjT6LX6kVg35oznzD54GSOPb6CVCbFQmmBnbcAAm6d891K3VBkpKFp1DD1wFR6vt4TaJunmq2audHlXaof5XJg2gERZNYq8f4k6AkZHWU0FDeK7xXWF3I07SgeNh4UvFaAZxdPph+aTsDAAG6uv8k6yTqOLjzKg98ecGzhMbRNWvqt6ccLiS9g6dhWr5mybwovZ7yMV1cvVteuFpW9WiPzbCbpJ9IJnxGOc5iQe0j1Ouzq7PBJvUZzgWlNp/WYm6ybaLYxQ9OsI/VwqgjYMRgNXA5vj0+eD6s+X0XRoTTaT2zP+/r3RTsTEFjvnZ7vhK2nLb1W9mL2qdl/qxDVUNLAkflHuPbRNa58cOW/kuX9J/6/H//IeP4T/yei4/SOaAO1BHYNhF+E11onpLjv47j/y31WVa/C2OIvIVfVkfbqjzgV9SJ6QTRmFmY4BDiIXiHK0gzkqlqgi+jrEjoulJOlndA7rqJnugX6DvDgtwekHEih20vdsPV34klAHk/cT9CPVRh1OkoelOAe7U7irkQOzz1MwOJBNNp3Ir37HIZ1dcVoMNJY3oi1mzXdXu6GTqWjIK6AB9sfMOiTQVjYW1DzpAYQUJePKx7z4dUPeb//+9z98S4nlwkNHt9pvqR2SKVIUYrU6i8eTC3nocHJD4OZUFB9WjLp7/xAy1PKKU0spTSplBr7JiqcK7je5zrOmiFc//Q6l9+9zJKEJfjN9+NQ9SHCk8LpeOsk2lBr5l8StPyzL2eTciAF/wH+yK3kJglU91e7o679++JKfXE9e8fvRdKnF9gL+gPpJ9LJvZ5L33f6ErtBaJ7m+eRhobJgzvdTKLBIpFu3SJFlIRrx1hajaKxC09weC4u2pOPcnHOs2L4NxwIzrGvy0euELzRVNpFxMoOZH87k5KiTJHdMRqZppja3luPPH2fQJ4MoTC1Esk+CrbctB6YdYPrq6aKX0OF5h0ncmciYn8bQ+fnO4u8N2zCMtGNpxP8Qj0tHF3ou7UlTpWDia9QLyWBdQR2Sxka0tgKTSyqF9a7raSpv4vm7z+MW6cbGFRtRK9RY6XpSVwdKpYDyGfTxIBbP2MCwvGE8iH5A1w4CE2rOGQFd5RDkwB8//cHJnJNcHHyRH3b/YCLFOfizwTRVNoko8n8V1U+q2d5vOwPWDaD/+/0JGh6EmaUZDQonVNbOmIcGIJFJsLSzxNLRklknZ7F/0Vkeew+hxr09H7cAco8sOEJ5SrkoKeaaHYdvylkMLs8hU8hEk2bbQFuu9b+GxCAhYH0AHp1NJS+kMimWCksGTB4gvhYxO4Lq7GpufyuguQKzAhlzYgyPox/TfrzAgtnotxGln5JFtxbRWN5IU3kTju0ckZnLUNWqqC+sx87HDoWtgpwrORQnFNN5SWc8O3vi2dmTx0ceU/KwhKK7RSTtTmL5o+Um+yV58Ihw81RK+o3Avekd/rzchrSsTK/E7g87Kv0rudnrJi97tcmLXlpzSSyojt4y+qkGTFuM2SKgtRN+S+DcW+ewP2iPfz9/QPBoyjyTiXu0O1P/nGrq/9USOpWO1IOpYkHv+PPH8YjxYN7FeQz/ejgz+uTjl3+coz9144UP3HhP896/viGeiorHFRTFF5FzKYcBHw4Q2SEgeE29sf4NTo88zaPRj0TpIoAhnw+h0+JO6DV6zr52lojZEaJP21/j+ifXUdgqRC+T4FHBWDlbYeVsxbmV50j4OYG3qt7C0sGSqHlRuEW68ctXtdRkluP+5DYVad4EDgkUpZ7WSdahAMo6dCY3cgxznKwpvl9Mn67D2ZmTw2zVf5by3GoGRX3qSHPqjHOUFz4+gtxIhGsEr3R7hR9u/YBbkhvVT6qJmhdFQ2kDOpWO1l5gYFYg1z+9TvdXumPvZ/+3v1GTU8PvQ37nStAVqnpVsavgAyzYjnfyWexL09FrlnN43mGUfkqGfjGUo8sTyZIJBQetXsetr26RejCVWSdnUdJUSJ7LT3gVeNEjNpD6HkWEjmuTsPYtX4qVOohm8zyc64ZgMAiMmMayRuSOcsxl5mj0GtJD0hnTa4wJsjGwaRqROe1J9F/MxMMTOXb4GIZmg2jiLWv0JiLzfQIyranrVsTsM7PFQpiTpRMutcMoV57DaDQgkUqwsLcQ/eNaixl1LkGY2VnTUNrAsYXH6L26N5oGDY7ZlUgCZXz/8vd0zt3PK8ODUCgVWNhbYMj/ywnVaLi4+iJdlnchaFgQ0c9Fc9XhKle0V0ienczrcwWJ3Z+7/Sz60cgkMrTmWor6FOHVzYs7m+4IHgaX5xNrJcxJPvk+WH1pRbZzNtHzozEajXx6+VOqHKtwXetKwKAAUg+nkns1F6PeSENJAw4XHXh95uv4pPqgCFGY+Mi1NkA73evEqFOjKHfJhU4CYGVlyUrmzpNS5xJL7yNueBdW83L5u1hI1Fg5WRG0NojtN7ZjY25Dn2V96PN6H3G7ZcllZJ7JRFmrxLq/NY7tHMUClVFvpBVgqFNY4dI1AKWf0sRIvnV+lerUmFdVk3paYF/5D/RnwLoBbO+3nSKHZK73s2aIhwOxG06TcSqD8BnhVL2zgecIY+/09oSnNragXoVCk3uUO1lRg5CaN6CyTEJtUYzEzLRY+Hvm7+ileo7/ehwvG1MP0dqmJjQyNVKjOe1GtsPKVVjnaJu01ObU4uzijFauxabRDm2TloEfDxQZtzWaClJ93iJjiRmzCyuwcrYykUAzGASQmLI0DX2+Pfd2CIymIV8M4WFsKg73b5LvnIiu4gCS8g2YWQhseFWNCumGJ8gqoLcMluhq2+TR9v5I+13fw2CwjLYk8EkgDRkNECCsefJu5GFo1xsQigUZpzM4/dJpRnw3gnca3uGv4Rrhim8fX4ZvFOQlr752FYDrfa9T6RXKUHMzTiw5gdFg5MXUFzlR/SVNFk8IfDIYx5oKagsbqH5UiEcnD3GMluqlvPHVGzQ6H8PGw6ZlnSxBUqwCiR06cxs08jKaG027kq1FiCGBQ+jn1w+j0SjKSFVXA7ug0LeZuE47+LLbl/Ra1UsEN8WWnaXU/jjYg8HwNWaWZli7WYuFYxHAFr+X6y9JKLv9BO8e3kTNj6L32dFMPDeQRptGlGqBBX35/cu0G9GOjtM60kXyMnGXj1CXuJVY7Xayk1xptmrG/b8E1boBRrXgoylNlGLQGUQmxVdn9rMxbiPQ5mUHkFOTw5xDc7iZfxOshGITTy3TdSodygQDLh1caLQyYObuIoDMW8bN1kaSXqbn4uBbTO1oqkTy3deXWPz9W1yP38b9fAlGowSZUYLSLJS+XX8joEHKvbzNHJ3wGx3y+xHYQtwatWkUdsvt6H+oP1KJlMQxiWiLtdQX1RM1P4qY52I4eOkgANquWgZ2Hkjk7Ej8+/uLck5jQ8ZyPP04Xw/7GoDlyaZrIRD8XuK+jSNkbAgdp3XEPUqY34PM+uFdBfLmOrQlleTHNtFU0UTo2FAK6gqICxmGxCBh8U8jaAo8w6s5r4qF7idjZFy3EqS+Fm5biLr9bXjvKZZny/2nUCuw+tOKHL8cQkYLhdzrudfZl7yP9Mp0IqojKD1aSr/3+mHjbmMiud8a9cX1qKpVOIU4icoCejMF2VHjmDhQwYeyD+m+ojsjNo4QC/yejy9hdiUVye8fiP5LHaZ2YKr+DaoU96m3rce/qL1Jg91CZo1z7VAqbS8QndAeqaKYt6rfIilJADjd/vY2mvsaCIDBFwaTJc8i/LVwliUtE1mHIU6mDK97W+/xcMdDxseFYsCIXibc6BaFFgRMD8Clgws6lY7DFe8it5HTbNWATicoyriEuSCRSNDodFTYCd7YntVTsfWx5rnrz2HnI4wRrUXSert6jox6k95Vh6g/UYqmQUNjeQMWDdlkuf6ITq5DbaGl8yu9kRvVJtcpvy6f906/x6zgWcw8MROZXMbJF09StLkIl4UuJE9OZtaMWeIavjVKGgRYVs9TPXEudOboc0exdrE28ZeWSCTMvzyf3Gu5XH7vMj1e64FHJw9OrzhN4OBAliUuEz/7V5Uf3z6+jPhuBLaetlz/9DqeXT1NvGNr82rRqXQ4hTjx5MITjj53lJknZor32lduX+ES5sKCqwu4s+kOCjsFUfOEtbHcSk7vVb3x7uFNVWYVhXcKufTuJWqya5i4c6LIAtwxaAelD0t5q1JgvUpl0r+VFG2NB9sfcOOzG7yU/hJVmVUAVD6u5MiYX3k9+XWu97lOkbcEVycBSJCf9zN9JXK6G/V/u73W6A70kUjIq8yluVYAXv7a61ecw5xNZPLNFGbEPBcjyMzpDf9RflXbrGX/jtPY6+x53CQAWMqSyzDoDLhHuWPnZMdgn8E0VzaTsDeBXm/2EiRxreVIZVJSuuTTJGvi4OSDxOQuQ92gRVtdzcqNPtjmVxH44D7KsVbc6XIPC58VNFU2cXzxcULGhdA0uInYglhi3GOwOmIlSHw+VYA3GA00WzXTd1pf3G3c2dZnG1WZVbxR8gZzzs9nwgRwy7qFT+p5tLPH0uP1HiJAqHXNaleeRU1iClbOo0QlFhsPO5qsmwjKCqJbXAfqc6uYfWo2Bp0Bc2tzHN8Yx7Ub1SjUMrTm3TDo2yaE9hPa897pjWQFZRFULZzbq+uukLwnkcAhgfRe1VsEcjiXOzOmdAzqevUzQMKnQ6/Rk3Mlh0proZlQ7FHM5QGXsXSwJO67OApiC5i4cyINxQ1UZ1czYccErJytyDqXxR+j/2Dcr+NE+xmdWsf1T6/jHu0uPK8SATh6avkpuizvYqLg9L8VrXnIXyXObT1t6fpSV1zCXLj07iXcIt1EIPzTYTSTo30K4Gs0Co1QS0dLcvxyWLVjFb1u9oIWAuOBaQfIvZrLO43voNEL4PYiryIs+1rSLqIdq2tXP/MbrSGTy0zyY4C+b/el79uCLGjw6OBncvcuL3Sh47SO2Pvb0y+yjV0cNjmMNao1ouLHXxnAzvn30ZQbgF4UxhWSdjQNiVSC0QgWjRWE3t5J7snBxAzsg2m0ARQMEk1bM1nStv4y0zTSPnYHTe1jWHhrIfb+9qQcTOH62uvUThbAQNsWbeOPSX+0ebMaIdAhkHvF9zj8+DCHex7m7vN3ebLiCSl/pjBwyURAGIciXKL49YdfuTP8Dg9/f0jfd/qSmGTE3Myc29sM1IXXCTWbGhUOAW0WE+eHnkfueBJ7PzOTnNpN2gG36vGU2h+lzO4Ejx+NwXCgmD5vtxy7CbhSz5SDMzBvUdEXQU8KW0quCx6orfda67334LcHvPDwBcImhSG3lmNhb/FfMZufjourL6KuVxMyJoRBHw/i2KJjWDWp8Mv1o8sdP5oKq6lIM1KbW4tvX18RIGTZZIllgwpVRSP7F+ynx2s9GP71cIxGCWrzIjTmbhR7FCO3txKUnf5S9/Xr64df33/9XCbuTuTMK2cYt20cM47NEEDxHjYmssX/xP/d+KfZ90/8nwgzhRlBHYNMkAgW2anc3dpIzHMx+Pb2RW4lR91CDNLLLXGfOYCgoT5s77cdl44uLLiyQGx8ueTexa4iC2grtjuFOFHrJkzA3innqTHIGPfVALou74pEIqHyUTHeBR6UuZVxe8RQ+oVGsOq8UERt9QfI/uUSujF9qXMJQu4IckvBdyj9ZDp7xuzBzsuOoV8MZfAng0Wk1cSdE/H5woddB3dBFdxfcZ/vGr7Df6C/uG819jVIDBLMNDLUzQb2T/6TqqwqXnjwgjgxlwYK6JaaGtAUlKGwU5gUar2Tz+JYnIK2+SW2dtoqNlxsZpqj99JzcchFBiduw7l9A2GTw7BQWnB41mF6aXuRGpZKXF8Y4tVWpE/8XUCozzw+E71GT2NZIwo7BQo7xTNypdmXs9k3cR/DvxlOhykdmPD7BI7HukBLkTb7Uja3v7lNx2kdiVkcQ012Desl67GvsafR1gyZpYL64nrUtWoc2zmiN0Cyz0r6p/nin1FHdfZLnF50gdBxoUQviCbYKZguFjP5aIaw4Fpi/g5XP7zKlQ+uAJDUN4lSt1KWb16OxuoBDkE9mH1mNjU5NZyYeYLudCdtbhraS1qqp1Zz8fJFylPKaa4UbrCCWwUmzT6FrQIrZysaSxtxDHI08WRs1bHPvS4g3hqc/LAry0B1LBt1i5ym3FKO1ExKjUMNXe90pffNrhQNLMfHx4XD8w7jHuOOTZqGXg968dG7H2EzWUjiHNs5knwgmU3tNxGpj8QYaeTMtDMmC+Kiu0WoalV0WtwJM4UZqYdSMbM0I3hkMEajkfNvncc92p3I2ZHYedkx4tsRogSmwlbBq7mv8ma3a6itHXFa3ANza6F4pVPrCBwSiK6kEkfJI2rc25J0qZlULNwVFIDKxokqz47YW1ryrupd8XMyjRMKrRtqeSlqg06Un9wzbg86lY6aj2o41PMQkgQJizotAiB8Rjh3t9zl7Ktn2dJjC+eazqFWqE3kH4PHBIuL4tivY7n5+U1ezngZx3aOZJzK4NCsQ0w7OI2wSWE8PvKYuG/j6Di9o6i7vm/iPjpM6fBM87E1Kh5fIyItlIhHUNzuIYIgjhBKPyWbl22myaqJBtsGE7nC9BMCIzNgUIDQoB39B+0nCmy8v4bCVoFDgIOJt11hfCEXVl1g9unZJjrwrXF25VmK4ov4wPiB+NrE3ycKnoRAdTVYNFbhkpeAvjIEobTZFuWN5RTUFRDjEcNfY2HsQk4cP0Hh2sJnEGFyazkVzhXCdfiLYIBjsCO2XrZUP6km/UQ6zmHOdJzW0aQZfWjOIZHRt/TBUvF1v35+YuLoP8AfqVwqok8NOgP+A/wxXpRRIDdQGDqY914w/e2ZJ2ay4e1Kyj07YZArQCawV2QHzLgs0/5XXki39BDo+SaW9b4m7B8XaxemdJjCr5d+pd+P/bgru4vMXMbtb27zat6rGI3C8YWkh3Bp5yWCRwejsFVg729vcuztb/zCvkl6zG3M0cuEMVljUGMBGGVmAoBDAvm38lHXqmmuaibkjgsqHy+CsoJQe2fQ7GMjSBoajKLGf5lrGTsX3GCowdQLSdkcjbI5GpmmGavaYrRd6sg6V8S+ifuYuHMi6nfVzDs8j53sRDZUhpWzFYfnHaYmu4bIpc+RWtGVRL8lxHeNZ1qvabi4uBAyVih8FZhfxKZCQafbzZTdfoJ8YptPibBbUrrEd8HxXDaNlxtZmrD0L+9DuV8X9I5g66lh5omZOAQ6cGHVBXwepOMb7EpSxwS0cglBQ4PEopxKpybd43Oq5T8w4fgiFEFzeD7+eaxcrLj8wWXiNsZR/mE51EB5UznN2mbMZeZ0XtpZRGXm1wmTUaNGQOu6x7jTc2VPlL5KdGU68Zw2Lm3EzsuOLZFb8O3jy8AtA3GIcqDXzF4AjPp+FKO+F1jIgRMCqfGswT/Ln6SPkuh8tbNJIWSZ0wGW13hSq6wlKyiLLjIJcd/F4dHJA98+vujMhXN2cvRJOuR+whsyGVYOQjHpyedPGKgdSGy/NmWB1uj9Zm9G60ZTp6rDOtOa52Keo/heMUjgncZ3qK4RPtfo4EO3D2cR8JfHvfU+t6kuIChuF9Gvdid0bCjqejX5t/I5OeokOf451NsrMHdR0uftPkQ/F22yjXaZ7QhLVFGbUcaRn+IFJGt7Z5SVlTRJj6JQKbBqskLbrOPhzmSOzD/C9MPTKVGV0KhtxNLLEgdHYZ679sk1UvancHvADVJjdgNweo5RBAW8nPEyn9p8ykuNL6E2V6PQKKh8sVJsOgDUaoUil06uQ/83dDqjESQGHe3uHcCoj2bghwO5sOoC6cfTUY51wvWsK16FXnROaEeTYzklDwxIzaRs+/MXUsqvcRPobqrQy2y1ijhg1DkZ7sn+TC2cSoV/BQxrW/MEbupKa7NPJpehUCr+ZYIdPj3cBK3bf1N/Xrn0CsnhybhXhyCRwIB1A8TCS6NeKMj8/PzP9EmNZ/qdAo7N3MeE3yeIxSijxEipWynm5kqW3H6F9AOJ/Nb3N1zH9iIhVolT/X0azM5gUadF26Tl4KyDBA4NRO8tjFUyiYyPLT4maGgQM48LzVODwcjTZRUjRnq/2QY4KGkSJM5cS11pbsghYn0EETOfauS0jAVSgw69WsqSu0swszBDIpFg7mNNg20DEYkRhGa6on1Bz/Pxz6OwVfDNhg1888UqesoMLNEZcaeRktwqXpdI/kdMbokCTo06Rfv32yNTyGiubkYqk/Jm3HTxc/WP63nc+BjXMFdiN8aS35QPfiDXyDHXmIMOHu17hLWLNX79/KgPhlK3UuK7X6Sb82NSD6cS920cI78fiWuk0DjWm+mJ6/GAOb7tuP7pddJPpDNx50Tujd/GOMswkkP7MzB3IGXjFrF8di2ffm2BWXMjfkmH8Un2JfF9MKLHtiKbky+m0PO1nmR+ncm8q/O4+epNjvY8KrIDWoFzrUWuDdYbmLRwEkofpUneIMpVb6jjUuAlBn38rL9wn7f70HlpZxzbOYpMRhCYLyp5MYF3jlN1o5Lr8X5kX85mTdMa1DqNeP/d6ZZKkNUkE5BQZZiUfKMwJl8cfJFRvh9RnlpOeXI5gUMCcbcRnu1623qKthbRZ3IfGssbkZpJSSpL4od4we+0omsFfVf3xS3PjYhZESYSeHvG7cEtyo2m8ibubb3Hm+VvisV6o8yMSp8YXLs10WFqBxGg1HpvVvpEYx1mKsEukUjExtiDmAdonBaQeTKNhvQi+qzug42Zksjcn7gXMJ5RpwehCr6PROKPj48PRfFFbfLpa6HL3S4UG4uRvyN4VLeGzqCjuL6YY0OOQaOQP4aOD6VZakat9V0eBAhgzM7fd+bo2aMsvr2YortFLNk8n2t9r5Hrl8uT26UM+GAAtCwTdS3o/YjECLof30T2pWlETmlrKhqfXvgA8r6erPhWaFRVleqQa83wKvSiyaoJxwoLfAe1w923BXgqt6KLZxfuFt3l0jeXMDtvxqLbi/Du7k3AwADubr5LaFEo1kusn2n0Pf27lqGWdOgprMn/VYFVXa+mPLUcdZ2ayx9cJn5TPCX3Swgd96+9D90i3XCLdKO+uJ5Lay7R7eVuJs2+I/OPUJpUylsVb+HT24dx28aZ5Feh40NFP8cbn99A6as0afa1KsLc3XqXky+cFL/3tO911LwoEwURMwszFsUK+U7F4wr2T9lP91e603mJkHOGTQrDMdgRa1drhnwxhMdHHhM9P5qfu/4MCJLRuGynRydvjsw/QkPOLywy/Hce1c8bjHzsnUe/t1saDkZaxSBM4uKai8Rvimdl8UoTice/i51DdhJzKwh7//HU2zZTOLmQUy+doqGkgdfyXmPk9yOpbq5G7i5nlMUolM5Kno9//qktCOvl9NB0Qup9SD9wjIx993ENqqTZzp2Lw7K4F/UnADK9APRJO56GQ5ADN/xv8NG1j3C0dGTlppU4OTlRm1fLwx0PWZ68nEfLH2E0GnG2EsaFPqv7iIyu8kINMo2ORntPitv1IWhYkEmu1joW5EaMoev7I2kobUDbpMUhwIH0E+lMOTCFu13usvWFw+zu/A427m3Npjj/K/xp8zoODb3onn4OvUpL4q40XCNccY9y5263x9RbVTD0WjIKowM5VdlUZ1VzL+seMQtjsPEVznlEUgSJPyTSp28fzCzMMOgMJjYLrb5nzqHO7Bq+i4gXIsAdSjxKKPEoYYjXEPJP5JO8P5mJuybSUNJA+MxwwiYJKh0lD0uImheFY3BbHmvQGrj24TWiF0Rzd/NdCu8U8mb5m8y7OA9br//c/JBIJPj4+Pyt3cS/Cgt7C5O8tjWc2zsz6vtR6LV6dgzcQfjMcLHZ11DawK2vbtFueDskOm8UDfXoFNbo5RYYDIj+dklpQu55q/ctxn8wHhs3G9qNaCf64rWyIUFQ6Pi7KHtUxpEFR+j+Snfx+f9X0ept93RYu1ibKD21hlQmFZvprQxziVQinruMbrPpIZT+mPbnNHHMNBpBY2lPbsRohvT04tG+Rxx//jhT9k4heFSw2PiasWcGrjWn6b9rMSO/GSZ+F4Q6am7HkVgHuYvMYmsXa/z7+3Pg+QMszlnMhScXMBgNWDpY8mKKUOvSbzU9R3qjnkl/TGLironcjpeIzL7Obj3o7NmZ9pfaY2yRY68kA6dKJ3rsbyAxJpGGEoHx/1r+axgcW7ygpUaqbG8iaBu0RVf5AhqyFpDiEMS8nfPQLTKw1vD+U8ckfH/A5QHEPNTQ/KkacxehNtK6DpIg4ffBv+MY7EjQ8CCi5kURPj0cz86e6LV6XMJcxPFfXa9Gr9Zj6Wgp5vR5N/KoK6gjcEggeTfzcIt0M1FvGLp+KAa9gYLbgpLV9CPT2TFgB94F3nz38ncci36bez9d5vbXt1mRs0Lcr8EXB9PlXjbaHhIm7Z6EU6jwjD9wXwFAqXspv8//nVndP6WxrJGKxxW4dHAxsYFpjYc7H1KWVMaQL4aI95GloyVuUW7Yetji1c2LLfFbeJDzgB/Df3zm+//E/zz+n4x5/5vxT7Pvn/g/Ec0VzWiLtOgsdFTY3iTbdSO+h2K4uNqSVdWrRFq7sUUGWy+3wHVqfwKHQO9VvbFyMR3wCtsPolQjzJCH5x3GPdqdzks7Y1VTgcbCDvuSVFQac1zCBokmuR1fHcAnQQLa2iBRmRR+Oy/pTIZVBh8e/5DUDmsZfr8Gvb4tYbb3s6fri11xbu9sUrxvjS6eXXiw9AE2n9nQZNWEjb0N434eJ5jJS+DnFRl0TE5mysEJZHs/xsrVSvR++qtet0oFWyO3EDYxjGkHp4mv6+UWaCyEhVjfNX2xdLAUFmaT4PKAJq4OuIoRPR0md6DD5A4YjUbyT+UzjGHc6n2LrPYShjo6kLg7EQt7C3q91YvIuZGYWZhRdLeIX3v+ytCvhtJrZa9njs9CaYFnF0+sXaxR2CqImhvFiTzEZl/fd/rS7aVunFlxhoxTGSy5t4Sl/V8lvvsTUvrMYmbPUK59dJK7W+7yRtkbXCm6RLbbRtQDPBlsexa9HtKOpomLM4Aybbb4t9GIKInVe1VvtjhvobSxlPsx93FujMRCaUG74e1IPiBIzl0ecJkngU94uOchH8z4gH2T9lEUX8RrBa+hV+u5++Nd1rusZ/aZ2VjYW7B75G76vdePsIlhYsPor+EW6cbjXs+htnLANecO6juxvJT2kojw0qq12FfbY6YzQ2uuQ6MVJoXEnYkk7kyk7E07tg/4ApnBgQPr+zN+K0iaG5GZy4RiUnk9mmgNEcoImquasXCwQCKRUHy/mJuf38S1oytOIU6cfuU0dt52BI8MxqAzEPtVLOEzwomcHYmloyXdX2lrf2gaNVSkVuCWc4cnHr+So30L6Ma9n+9x5pUzDPx4oHB+JVIcipIpuK3Eu4c3434ZJ24jLQ3qXIOpcw2m01+8f6viqlj0ywvc6nEUzEHVW4WF0gIzhRm6Zh3pJelcybmCy5cuVDVV8Wa20LgYs3UMZY/KOPnSSUojSzk86TA/jPxB3O7YrWPFvwMGBSAzl4moR9eOrvR7r59Y9On+Snc6ThdMz1P+TCHhlwTaT2xPvXd79P4RzJjx7LW8N62K5IadBD4JJKTyeZP3zBRmlLmVPfslYPTm0ZQmlRKzMIam8iayL2eLfomtkbg7EW2jls5LOtNxSlvT3Ggw4tPThwVXFzyDbmyN5srmZzTin/b6WLIE1B4dqXENZlyEAnWdmvzYfMwszNgxQNDV15ppSV6QzJyf55hsZ+vdrSy/v5wuy7qwbNoyk/d8evqw/bnttE9tz8BvB1LSt0QstMd9G8f5N8/TeWln2k9sz631t+i+ojt2Xm2FveJ7bRJKTxfon46IWRFEzBIKwlc/usqV96+w9MFSFAp3kEgxyqRIZVBXWEfCLwkEDQ0iZHQI5b8CzRrkqnp0Gmu6LOtC0PAglGm2jFr/0b+UNooDRkjl+EQspU/SBJBmc/uDQvafTmbFkxXIreRIJVLUCjW3Zt1iXKdxHFskyPxZOVlBi/fYrV632LZlG7fW3yJpdxKr61abyJjqzK1wjbJm4m/jWf35alCDpyKYBqCw/WAK2w9GJofX8l4DIOtcFt3PBmDToReBT9oj86tg8O/jGfyJ0HA2ZgkTgtZcS6VzPZTo+cz2M7os7wIMFX/Xqq6Y0LidNIaOxnGILz1X9hTZHGZSYY5q9UISv9MyjUqMMh5GP8QxypGjw48SNT+KCdsnkOX4A02uD7Fr3MGU8RGUp5ajV+vFQqnEKKXetp4KV4UoZdQaT8sUGY2C12grej5qfhTpx9NxK9JR7OqMRb2p30VyzR3SvdbiVeCFc6mcBq8mPLsIz47SR4lXdy9uyW5h1WhF53udyeuZR8SwCKHg2RIrz63EssmSvm/15WLORQZ/OrjN3LxEOA81DjVou2vJvZZLWVIZdQV1FHUrotGskdy3cklfkm7yvFmPsmZZiSulxQABAABJREFU7jKC64PZ/9t+DHoDe8btodPiTmIBUma0IKtdFpby13je3Y0zEw7Q842eqOo0+D94RLqjFemBldQ2yEj7MwmZTkXXZV0J+jSILZe2kFeax6pOq4jyiKLnyp64R7tj6WhJraYWx2pHQuaEcDX1KsO+HoZPbx9KHpZg5t0GYmhuhqqsKrZEbKHXm70YuG6guKZQWTliMXogUfOC8YjxYKP/RmpzaxnNaNauXYuFxhu9HhEcAvDoU2eONL6KQq2gNLAf3VU6Hu54iFukG1bOVjgXPMSt1A2XcheGnR9G+agSlD5KwiaGYedlR+MDodlaVFCES7MLdl526DV6tE1aDMZ/nSh1WdaF2K9iieseh1auZ6X7SpP3m3QC48W53BmzsmKaKmVc+/gaAYMCCB0bitEIBqmcrJjJ+MfY0/stb5E5evGHq9wLvcdV6ytcsz+L4spW/uymp51DFFcqL3HtP8ijnUJP76JUdo/dzedDPgeENVi3l7pxJratOBY4JJClCUsx6A2kHU9D6at8ZjysyqzC3NYcGzcbwuaHkVSRRPvU9oRkuWHooqXDnDYQSGsRP6DsVeybuuAcXsuE3yeI97Vcb49WVkNu57exUYdSfDeXez/doy6/DoV1Nu6ZDVhEmjHq0DScq2vR/6An42QG1q7WvDz5ZUa0G8Gt/Fu4DXDDq5eXyF6pDZBwsksEjpWOdL/dHUOIQdCb/Mt+Dbw8kIq0HbDetIjXOhZkdJ/Dos+hlcxkNBop/SmPtRvX0mhtxLpRInh2RrvzzYYNfLZmDdeN+meartlGIzsQ/Kn+U2y3MKcxQgMWagxKAxKJhM0dNuMQ6IBf9HJyXTcD8OTXJ+xL2MfM4zNJ/yEdz+Ge5Pnl0SmhEyPPjEQVruLYi8fw7eNLwKAAniyWEFsnNOZz1Ak0V3lQ8bgCXbPuL80UQUarqbKJmpwa9Go92pomLGvqSVemkaZ4gP7cj9xJt6feMItgx4l4mcPV/ufpkNyBwEINlvVl3N18l/AZ4TRmNhKYHcgdzR36vN0Hx3aObOu1DYAXEl9geNBwPrvxGSAUMg06Awa9QWQStBY3VddUZKZn0nF6R8wszMSCclUVbNpqw4gRNrj8RRk9ozmWC1G9KDdGMaB5MT3f6ClKhLdKbiGBK4Pu4l4aTmVGpSCp6W6D4akOQ1YHO+TBwaQeTOXye5dZ+mApx2ce53L2Zc5mnaWrZ1esnKzY4LlBYAB8I3x3aoep7J+6n5wrOeyYIqxvPjB+gM6gI64gjoK4AswUZkTNj8LazbpNVvCpkFhZMXV/mzRoK1Ov1i0ETajAKt49cjfBY4Lx7ePLis8mcn6IDTf73ASJnpxz6aTsTKDHqz0wGuVYafzp+/gBad2zcfa2IP14Og6BDpQnC7YG6tcERtzm5Zs5NvcYeq0eVY0Kc2tz5FZy1t9czzuX3mGEbATLei8T1Sji48FI242f3T8byyOWXFl3RSwwyfQy5u6aS3zTLUL6tHnLaVuex3rbempd/FA4WKHX6gXpS6lEvD9dylyYfHAy9RMyYYow4ll72PEgKpUqxyoikiIYeSaSsukluPsKY0y4azjxz8cTuSWSEvcSbCJsuP31bQasG4B3T2+yZ2aTbJvM/CChSRn7TSzNVc0M+mgQEomEAPsAsmuy0dfqMSgNolznX+P6Z9epflJN71W9yY/NR24lx6eXD31W9yHuuzi8unnhHObM9U+v49vHl9Cxpg1AK2crliQseaY4Gjk3UpRndQp2esavetzPbbnOnDNzTECHT0fQsCCmHZqGb29frF2tKbpbRM7VHDpM7kD0gui//Q60yITqTaVCPbt44tlFAFI5hTgRPCoYpZ+S+WlvE7LDDq25mtAqCTodpB5ORaer/x95VDc2tK2vWpuOT4dOrSN+kyDz+XfPzF+jVb2h1K2UHnE9qHpSQ7eXu6FtFJoof07/k4zyDDYu38i5qXEMdTZthnTWrKRJ/xUudcL6teCyIC/rkXULtaUSx2wPrvfSM/XYD/iWqVBuVfK+TijyX78kyANWNVexbdY27i67S+rBVJxCnZCZy+hg14GDBw5yYM4BAlYHMH/dfPF3t/p9RgzwcPBrFLb3R2na2xfHAr25JTIlnFy2j7Sjabyvf58DE/cQTjgZwRlkeDRgNJNTm1+LRCLBwsECn88bGBA0gCsDr1Budw5NVV8Ov3yYvmv64h7ljtZMkBf0zShFY5PC2KNTkaNFoRTWzjqjjjDnMBp7NjJq0ShsPW35Y9QfVGdXs7Kobe0T+3UsCT8n8FrBa4zZOgaXji4MfTKU80/OA8Jaf/LeyUzcNRGpTEp5crlJHcU9yp0JOyaYHLfcSs7y5OVYOFhw/9f72HrZIjOXETAogP8mpFIpTk7PqtL8vwmpmZQXU18Uga0AjaWNxH4Vi8JOgUWpgaAru8mJHEuFbyeTulkrcw/acp6n6yCtMp6TDk4i+/Ns6jLrKIovwrOrp5jHGg2CRYe2WUtzdTM/Rv5I5NxIBn8q5GW1ebVUplfi2cXzbxmYmkYN6jpBtePcm+ewcrKi//v9aa5upiK1AocgBx4feczJF04y9/xcAocI3s8aSyW0pdLiWG80Ch7U5X5dsA0Gq7JmvHt4i2zhVlCKUWLEKJGYMNCNRtDIqlBZ5mMI6IaZiwGdSodULsWvnx+Pjzwm9VAq0s5SwpPCqberh6ecPx6UPBD/Dk4PJn9fPt1e7yZu29jS7DMaZGzttBXXjq5M3CnMR03m2ejsVJwddpbFgxbTWNaImaUZ5rbmNOkFsL19tT1W1Y9R1Xbnu8Dv6DC1A2N+HIPBAJnuX9Igb+BRx0e0d/U2OR+O9f0pdTiKpCWPyD6fhaWNjNBxoeIcl1aZRuWySrrQhfhN8Xj38Marq5eJcllrnHn1DA+2PWB17Woxn439OpbHRx4z/9J89k3Yx8hNI+n2YjfS0sDTE/G6nXn1DHHftnk6poalIpV3Qqowp+PUjji3dxakv7OE61Tg04h3dU9czC3FOghAs1mRyT4VNuby5GKVCaAdhCZk/A/x9FzZk/Rj6aT8mcLQL9vqAcEjgwke2eZx+OrhV5HpZUwKm8TwdsOfOfZ/4n8W//8Y8/4n8U+z75/4PxGJuxM59/o55l2Zh0peQKnDMa4NkXDq+e9NPvd3RrW93ni2+dRs15YwJP2RhLpOTfalbDrcyOBh5EO+eeUyX7mVY9AbRBSNJMCSPL88LJss8XySBQo3Ug424NjOEfcodwoDC0ntkErAkwC6H9tIbvsxdI6KImlPEvd/uY+mUcOoTaOoK6ijoaQB13BXzCzMyDovmPz6D/YH4Nzwc/zxpiBf04oWicnehXV1IRXe8ShclQx+uc2YvfWYnfLuY1Odj147hgFrB5ggWAGKQ/pTHNIfuSX0f68/RqOR2rxaqt3DqLE/xdzf5yKxfAgICzaJRIL3KG/yTgs63SHF72NsL/jxeXXzYu75uRTfKyZpTxJ+/fxE6RQQJBC0TVpmnZgFCL5x8y7M+9trW+Swnxu19gwLGkbY5DCsnK3YGrMVO+QElK+iStERgwHajWyHpaMl5jbm1GqEhXCRVxHVxeHYBcD7+vfFbU7bM48DtTtxL3ZHWatEp9Yzec9k0avOsF6YQC8OuUj39LfE73Wc2pHKnErWbl8r3Ccti4vph9oQ3WYWZth62orXrzK9kqqMKo7MO0KUMQqNBn6adgFFVQmzD08ieV8yXt288OziiWNBImobJ0oCe9FhXmfsA9oKfeWp5bz67atcGniJPfPiWegnXL9Bnwzi0ppLyFR6ap1qmXz4FXxKLlFQMJMT/bfgFOokyha+xEscfe4oX778JW/Xv425jTnhM8I5seQEh+ceJnxGOON+HScmaFIzKStyVvxtAxqEZ+PEkhMkhj+i1DWLzL2b0SyIxqWDCzGLYrj3o8BodSpKwqkoifgfIk0KvyAwTVvDwgL2jN1D+ol05pybQ1NxE7Z1EmLux+B01J7iXmUE9PfFpaMLV9ddxfmCM53GdqJWWYvBXrhmB2ceRF2npu+7fVmQtIC3NgvXT2fQIZM+y4h4mgEEbUje1nAIdBCTm7rCOvJu5DFhzzS+fK+BxoQyRo50RfkX5SeZrYo6WR0PYh7gfddTYIW01KENOgNWjVb45vnSKaETRWOKRI8u3z6+IqLXztuONU1rntnfO9/fob6onuhF0WRkZBAcHIxMJqO5qpnNHYVC4+gto7m05hKzT8/Gq1ubnNGE7RPQa/WiR18rUKE1VCpAZoZeZobEDCozStk9Yjedl7axVOU6OWUFzzYrj+88TkBtAHe5+8x7rWHZbImyRImmoS15uvi24N9zb6twr8QsjjHxS0w+kMy4X8ehU+kEdptGL7JCr396ndRDqcy7MM8kQfLq6oVPLx+2Rm9FCrR38Ca/w3DqCu2oK6jj6tqrmFubC89oowq3zDjcn8RSn/cSXYYHEDAwALc7btxcf4u+nKeHTMJSvQE3WuT3FBZc02lpF/I5gT6v43VxIzpzaxSDQ3EKdkIilVBcX8zeR3vRyXVkds8kcKiwiPfo5IHcSo7EKBxjnbIOjxgPAQHuaSsem6OjUCjN7DaLz7YIxzUudBw7E3diKbWltWXr+5cCQ6t3Rq2ylusjA5nf29SbVd+CBJTqpVg0y0Cnx6eXD/b+9pAC1dax1FkmobBSodOFMDjYF9dwV4Z9NUzchpnUjJiEGFQ/qig6XiTKTf18NJkyuzyMUiH5VXgosPOxE+caqaIZjZmGfL9iLN2VHJ+7TSw+PKl+Qpn9KcrsQWHhjLWLNbc23MKvnx9eXb3EZNMl+w6OSXkY9JNERGvwqGD+mH2OMucUFm5biF6RQ+w3t0k9mMKMIzMwtBTCCr0L+W3ZI0bUrENdr0ZmLqPT4k6UPCih3bftiB0ey+BLg8nrKjT7/hoGqYFG60bMbU0BG63J/sLohXwz4htyrXMxszSj85LOROyOQPVIRe+fe1PWuww7b+EebDeinYjKzLDNIGBGAHUJdWSdzRLvFYMBJMYWf5bmKBQOShZcXYCtly3ft/seZ0CieYRcI0eulpL4Szz1edV0XdaVybMn031sd/w2+lFdWk3G/QwyTmUw59wcCm4XsObjNeyevZt8v3yUfkpqc2s59eIpQseH0n+LgGBQNFQS/8k9HBb5EDAwQBwHW9cUGmsHgub3w6ArZJ1knck5CX0cysArQ6jwyoMeT92kMoE9p5PrqFcasY/w4e2Gt5HJZRQnFNNkbaTRupEqxypu9L7JXNdX8Y9R4j/AH71eDy3khzPzznAr9xarqlcxcN1ABq4byMY32wAIdzbdIetcFhN2TCDlQArtx7dnuI2QlEqNcmzcbNg9aje513J5p+EdsWkx5sQYvEuOoKpZTNzGOMwUZoSODRUaTFIp1V7h+HkKSfulNZd47sZzXEg5w9XKS/QyM7JEp8edYkqADyou0cPwrxt9rdEd6CWD6w0ZmAe0Sa1bOVkhufPs53XNOvaO2ys20VvjycUn7ByyE49OHiy5t4TanFp63upJh5QO+BS4Y9ToaJVMFe6vFgndcjNsqvKw8vDFJ7wNdW4UGyrCc3Z62TFqsqp43/A+3ww/hH1pGkXKQyRGRuJfFoyFvQXvadtknwPsA5j25zToCao3VST9msTJF07S9TehwOVe4s7IMyMx9DSwe9RunMOcGb5hOJZS4T671/ke3TutpPBOITlXcoicE4mtp63JOt5oFHxEZAoZCjsFKRsF+e6CiAU02zgzXSEn+3E2q1et4pr+76/FYuAj+K+Y3Fe1OugEkWaRvOr+Kga9gci5kVg5WyHLThU/+yD6AS8seAHfvr7YHrTlfrzgBVfqVkpctzj8nP2YsneKyBxuHQvap7bHpqKAmDOjRHWI1vFFqpfy8ub3qOt2keFnhjN8g3A/P5pQzOGjP9E708gSHbhTS8njEn6RfsoZ4zou9HdH1buQeTvm4V1oIGlwJ767HI6F0oKwr8N48ZsXSW9M53jP47zf530sHS1prmom73oe/Zf3p71ze5QnlDy4+gBtby3XP77OkntLBCnETMHz8tR7p7i+6Dpfe3+NYztHFt5YCMCXX0LqQzUPbmrZ/quBax9fI2RMCCFjQjjd/BEAD6MfEmnlYCIVbDDxr9NhMMCmkE2CFOb+qeL58ilfRFTuLzi2h7BJVjgGO4rSXQMDBtLftz/1hfU0VzUTvSAaK2cr4o1CI0IqkdJc3WziBV2cUMzXFV/zVexXzNs6jzcnCACyp0EaIPgVhV/ZxI132zP55zbp5adBKXq9MB+Xp5bjHuNO6iHh/gjIDsC23hbfwkY6HR/AgLd7oFAq0Op1NMtLBb8kl0AcrOrZN+FngucGM+G7CQQMCmDdw3XwUJDMlDpIKblfwi/df2HIF0Po/VZvttwVFgtnRp7h9AenxX3R6dqKqACZfTLpl9YPjBAwOIDN57eTEJ2AjX4aQydF82jfIyoeVzDggwFodcLYmBOQw4XR1bykMONj848Z+NFA+r3b76lxQvCVNBQ0kvJnCn79/ZCay8jxz6fCuQL3mmmkR3hz872zJG22Yvaptva6VCLlSdATXDu4kvx2MpHzIgkYFEBJ3xJKSkooWlzE0fCjVDyuoDa3lkEfCQxSS7nw/FzrdY3sLtlMPTAVcxvzZ3KWW+tvoapuY6+N+mEUQ78YSnlKOX+M/oN+7/fD1suWW1/eQq/Ri82+9JPpXFx9kRHfjjBpVhQVweHDMGVKDG5/3198Jp5mYALotXr2jNmDX38/+r7T14Th8WjvI2I3xOI/wP8ZgKjRYOTGFzdwbu9M2MQwUR3m7yLrXBbHFx9n9unZWEa0Q2uuxrrBGv+kqzS7d2V17Wo22f9OSV31f3UMpYAFFhQnFIv5/F9DIpXgFOJEyNiQZ0BbfxfzL81Hsk6CVC8lPSqIVePDsLBsa4p6P+fNtivbUNYoOfxDNb3WNvLgtwf49PbBt7cv7TSTMWZMJvzSd6hs/iBwcgyVCXlcMh+OorGKQttPUKgU2FdqsClME9hPLQnZ097aVQ5V2PvZ0/P1nqLPF0BcTRyFfoWoJH/Pfmy1R8m9nsvNL27SZ3UffPv40twyrMhV9ajyVcSMDxXBu93f6M36uM/IbJeJXU0gmgYNu4btQiKV8Hz881gUGBhQMIAmqyZiHmUjHTiMGUdnPMWgE/Z/y7It9E/O4cj0vTQU1/F6gSBBL0PGrkm7SCxNxOBswEJpQfRz0TRXm/reR8yKwD3GHSsnK2Kej+G9S+9x/sl5gtOD6X2zN80Lm5FIJKKawOQ9k1HXq9kSuQWlj5JZJ2f97fVvBQb2e7ffM+//p9Dr9Sa57X8TrepYrXLKrZF7LZe47+Lo/VZvk1wYBNbfK09ewUJpwc4bAtjbvFlQOzAaBQne4NHB6F4Xxk3fXF+0e7WoIlWirze0NQPLXcqxtrcm/1Y+f077k8l7J4tKC26Rbqx4IrCsVDUqbL1sTXLW1EOpnH3tLM9df46s81nUF9abgKLv/XSPc6+fY+HNhaQcSMEhwIH+7/cn/1Y+e8bsYdy2cTgGOdJxekes3doYgGbqBgw1BsCOgtsFAii4l4/JGspgEJpMrY0mAKNEeC72zdjHwKRPyD6Xgbq0hq7Lu2I0wuWIILRmNfRJice3wIFPLHcxZusYOi/pTNZZoWYp7SJlyIUhVKdWo1mi4cH2B8+A03rc7sHjE4+pGFWBqkaF0eiNsUXGc0fKZgKN67Aqs6I2rxalrxIJUhptGontFStez9a52dAyhnW+15m+N0qoK6jHLdKtTW7aCOmeH2CQqvhz6p985jqfnKs5eHbxRGZhTtesIyT5vsjlQZspaN+P0K9v0lTWQOi4UJNxYovHFr5+82v6v98fhVJB9uVsHu15RI/XenB/233if4jnlaxX8O/vj0wuM/Et7fVmLyLnROLc3pmJOyfi2dWTu3dh3TpwdYVffxU+12VZFxJ+TkDbpKXczZLMdpm4VnmhbdLiPdhbrKO5WHgSVLwKlbMjBR2GEfEXLkHr3KysUdL5XmeabarxHBvKyO9H4hbVNnHV5NbwaO8jOk7vyKgfRjHkiyH8XZQmlbKt1zbRRqB+1n+pf/9P/Nv4fzLm/W/Gvxfa/if+if+PhFcPL9ovb4/ST0mRo2D8/cjvKD49fTi78iwb/Taibda2ec1o1WSu+Y2bX978j9t+V/UuUw9MFQtw6SHpqC1qMEjN2DtuL59afwqAWisgLm3rbelwNw3Jo0QOTDnA/V/vU5ZcRs2RGqwarWiyaiInsAKFixJVrYpjC4+Rfytf8HEyGrmz6Q4/d/2ZusI6AM6+dpbTL59GKml7HPVGPWWPysi5ktMmyeTgRU70BBzCTRc0rWFbmYNL/n30Gj393+//t7rlT0fSH0l86/8t1R5hpHbIxLvAG4vGOpL+SOLUy6dQ16npubUnX771JSFpEYRf+g5NUhrTj0wXGV03PrvBjU8F6ZLhXw8Xk3m9Ro9e8y/kDpLL+MbnGwy342gyzyUhaDrjfhlHblouEbMjRBNgIxIq3BS4Zp2hNrWI0LGhDPp4EHJLOXKJ6Yyn/8tPHX58CIBud7oxc+9MtM1tSXDx/WJGbh9JYFZgy+/oyb2Wy0fmHxG/JR5dsw65Ro5CpUDxSEFdQZ343YzTGVxYfQHPrp7Mvzwf146uNLVIcbYiaPbtg/T4WnLiy6kvrOfUi6fIOJUhSIXk38euLBO9uSVyZ3vivotjc/hm6grrsHK25n7nYgq8C5AZrMWiQtikMCbvnYzBXEA02dTLBZ9CDUQ/F037iYL0TfSP0Zh/ZE5zx2Y6LemERCYkCwpbBbNOzWLIF0OQmklpN7ydqN0tkUiw97MXJS81jRq2RG7h4hqhQdO6oA9ND8ar0IuONwLQNmkJHBzIuF/GMffCXMx93MjoOousTlPo9rKA3krcncjD3x8CsH072FTl4Zt4Akd9uShluWvYLmTOZvy0/DynRp3iVt9qbqcJyfDTsjAjT4/k3PBzOH4nJEE1uTVknMpgW69trPplFcMth3NEeoTa9FrxOyeWnRCldf9TGPQGdGodRoORHit68E7DO0itLAh4eJTwa1u4sOL4M9/pr56CW8nfVwDKU8p5a/1bDDs3jIDsAJNCU2liKZvab2KdZJ14Hv4a434dx4wjMyhNLOXhpodUZQieHFJ52/hg5WyFW5Tb36JqNQ0afh/8O7e/EYqiX3t9zZ8z/hTfl2lVWNaVYmhSofRVMvbnsXR5oQvv6d7j7OazfPjehwR9E/TMdiN2RzDx8ER63exFYXyh6TGnltP/Sn8KvQo58vkRkTkCwuI3cGgggz4ZRP8P+jPggwEmhtZ/TvuTbb23ETAogBNLT/BDWBtDU9OgoamiCSSCDNKBqQdQ16tpN6IdE3dNpP2E9hglEmRaNWE3fyXpjyTcItxYlrSMqPlRfK78nI7nNiIx6Cn174bcpq0w4dXNi7m7ztJ/SBFlQetYaRPGfKUn34aFMeDdNeQWFBAY9DpFDge4MdiC9M59iVjej+euP4eZhRkPSx+yKX4TAHsn70Xpo+QD4wcsubcEAGVTNAABxctQ1aoImxTG0C+HihKoTU+R0xrKmoj9Jhb7LHsAmvUCu8muLBPLB7HotXrSjqdR8rAE757enJqVyO0et2m2VSCxE5K8hzsfolPrxCKqS7kLL/wwDMv0h8w5O4euywSfz1yXrST5L+VuhxUcG/YGcs+2AlWTtonxe8fzc8LPaOVajPZGE6T68eIt3AkZhWflDLYk7+HuK3epy6+j9KFgElUjewxGSPVaiV5npMvyLvRdI/hU6Ezkb4zU5tdy/o3zZF8SGNit451NdT62ucmU3C/hE6tPuPnlTeSWcgp8SqlxqOFm75vkBjvQVNFEVWYVBr3BJFEzSLTImur53O5zLqwSfIi0TVpkKhk19jX8uPRHwhYLY/XeCXu5+tFV8btqCzU319yk79t9ebTvEX+M/oOqrCoR7WuVYcWWyC00ljbS9+2+WDlZ8aT6CcUexXz07kf0eqMXtzfeZs9YQYb46f36/s73+PTy4V31u3R/WWg5HKl7j0aLDDwLPbHN+ZiGvCr8+vnhGORI/0+GkBL2mFy/XF74dTOj9hUz4OuxzDwhSCVKJVIszYTn6OclP/NC0guM+mEUzu2dyTqbhVwnJyQ9BMPnBqLnR4vzcdrRNHG+VDTXUHMqlqaKJmadnEX0fOGe/Wuh4Jduv4j/nnt1LrV2tcTcj0GhUqDT6Dkw7QBfewmeXkHfVjPzj5lYNFtg1aBHrxdYmjJzISlutKpBZ6Yj1z+XC0PPY+P1rIcWQHOvZros+4unqbHtXix7VEbW2Syaq5o5sfQEibsSxfc61wmNSZ/ePrSfIMyPMokchcad5MhKqiIGYO9nz2v5r9FndR+T+6/17xuf3UCn0rHtwC/E1p7lBkau6gR22GCE/zoY9Cz5271/NpbqwDqprfjYVNlEbV4txqd+uPpJNbFfx1KVWcWEHROekXcuSxJAGJ5dBfDI3pi9DD83nPSQDJIGvozMWsHOoTv5Y4wAFmv9ra43LAi6t/+ZdVKrb4t/ShKOhUnUZFWJ7zSVlKExV6NR6LBTHEbl92zB+ek1q8FoENgrB6dhHSpc0xz/HLY9tw1ZJxlVGVXU5QlrKRdZEJ6VM5BYz8C6f1dyruZwYdUFca3Vekrsi1PJPZnERv+NHJ57GIlEQtiC7mRHjafe2R+dhQ1GJCwKW0g/o/RfNvLcgc+BsQgNvb+LOGCsuTmLVj0PthB9JpqtEVtR16oZ+uVQer/VG5mhjfWT75uPzzQfLJQWWPhYoLYQ8oOcgBxOjzqN3EdOyJgQUXWk9VqEpYZhf7FUBG08/Z5RYkSu1mNUt4FlvtmwgYsnfuWm8dn777JBxQ2jHsX1IrgFBWHjyey2EKNMjrWL4IFo7mpOeqiw1vj69tfU1tXSXNWMQ6ADoeNb2MUSGc4VztTE1uAc6kzErAiTORogoSIBqZlUkOxc0gYOKisDz7QrRJ3bQGlSKfe23qPoblHLdWw7Rj2mN5/hqUFmzu4FKB9cpffq3uLz2tZgkhBy+3dqdh3HpYML4dPDTYqoTeVNbPTbyOX3LzP408H0fL2nmDNJJBKyL2azf/J+cX1elVnFV7GCufTvD39/5j4QQyJBZ26NXqvnxLITpB4WGnmtz5B3ynlsdv2Iua05b5a9yYiNI4hZGMOF4fEcG3cM23pbbOqNWLgI3nhSmZSiphwuRnlzJUIoXurlFozbPg6P4R4obBVIpBI0VcK1t6+2x6HeAWtXazq/0Bn3GPeW89J23oxGI8cWH+Nz+8/RqAwYJW3nWGWp4sWUFxmwdgC+fXy5O/Ixlc6V5Ae74NApgOR9yVxdJ8x9rUxbALW8BLmdBeEzw8XmlY1UyAPKXcv5/pXvMTqac2DqASoeV5BzJo0JR4fhUO2AlXQhdX6LsPa0M2GiASIQz364PS9nvIz/AH+ufXSNnst74lDtgFFtRK/WM/P4TJbeXyoeXytIw3WKK2aWZqx3Wc/DnQ+fuVyLYhcx5IshhIwNYcHVBYTPCKe+uB4rZysWXFtAp0WdsPWw5ZWsV+i3pq1BYTQYhXxVq0dVq0KnFub6NWvgzBn44CnSceLuRL4N+Ja8m3nia5feu8SNzwUfbnWdusWjtuUWkkrIj82n6qlx9cLbF1gnWYfB3sDM4zOxdrHm9Cun2T95v8nxXHrnEsl7k585ThCALt8GfEvF4wpOvyQ0fO/+eJf0/Q9YsXGFAP7IzECTlY9EIsEtcBa/Sv+1n9vT8ZNMgk9ZKPf+EPKnez/dI/2kaa4ik8t4Ke0lE4DYfwqlmTMGmQGfquUY/1J69Jzhxb0u9xh2bhhum2/TUNLAhVUXeHLhCdA2HzTZuaOycSLy1UF0/3EBTUoPpHoN/a6G4pvny4M+7Uib/j7aJi1PLjyhKqvK5HmR18lpKG0g50qOwGA2Gvjg8gesL1rPH7P/QN9Lz/7J+/mt728ALMl6k4QRq7GtzCH49k6enH/CkwtPRKZnSoqwXe/UC+St2kz49HBRtrXv2iHU+67EN8+XF36cTMXNNGIWxRC9MBq5lZzaLjIedXyEe4k7bgX1IBMYRq0ATWWjsPapU9ahs1DgECIA5EqTStE2C/fYwZSDPHf0Of5IEub7Tos7iXLZiaWJSNZJCLgaQOTzggLTwZSDfHpDqGcp1AqcK5yRa+SUPCghP1aQWbq67ionl53EKcQJpb+SwjuFHFlwhMI7bTmf0WikobTBpLHYVNHEOuk6Tr7YJlX770L1X/ilPx2taihPs6FAqAekHkylsayRhtIGIV9sCZm5DIcABywdLcWGrUVjZcsxCPUNO287XK2FcS4kPQQ2Cwo51z6+xokXTgBtbL8vd3zJymsr8e7uzcRdE58BNreGhb0Fi28vpvdbbXIGAYMDGP3jaByCHMi9kkv6cdNnyj3anW4vd8PG3YZXc15l7oW5gLCPw74ehlc3LwKHBDJl7xQTdnO7+L2ofhA6SCeXn+TQ7EPi8Skaq+h4eRMZu27/zV4+BbbBQOL2BE69dAqA9HTQmtVg2WRJl3MnsU5LIGJ2BI7BjpQmlmLRx4KjU45yJvMMhyYdwnG1I+paNadfOk3Knykmv3JhyAUit0ZyYdUFfuv3m9B4LHuVvskJAHww4QMqMyrZN3Ffyzf+tXqHvYUDEoMZaaFpPOhuj4WLLfMvzxfH8yPqVzBIhfvKq3I2ttdhx4Ad1OTUiDmNhdYT2+aOmGud6bNuCGO2CsQJkxzSaMDMwgwbdxvklnIK4wpJ+DmBzR0249jOkYCBglpU1Lwoxvw4xgR44tPTh7BJYdi42whNv1BnrrakmGVlsHvkbjaHb8Y51JmYRYJ/gktpM1EPo1j4c08qbmeaHLOHpT9hhZ/TrqQF1F5ZywbPDVz7+Jp49UCoS/e73g/14xqcQpzo9lI3EwuZiJkRvKd9j9DxoVi7Wpuwd0GoTV1+/7LgFdjHl6zALO52vvu3APp/4v9Z/E/HvP/N+IfZ90/8nwivbl6EWoai9FFSb/kIAIlBgkFnQGGrwNrVWvBLaF3XSUBdVElDaQO7RuwSmQutVi1B8ftQlqUD74nFzB4retCzRkB7uZa6om0sxbefL5ZOQtKr1bdIednXcGdgOJGB3Zn+lj+OwY5knc3CbL0Zb/EWa9eu5fe5m1jY5Xss7HXMPDGTuoI6npx/Qn5svshQa02mh301jIT8BHy+EYoCYSlhxH8ZT2NyI6kHU3nf8D51Fo+os07ASh2IwdCHBzseoK5T0/3l7uJCOC9iNPkdR7BI8fePtV1ZJlZ1JWiburN71G6xiaAx3EZjXsVn73xGv+REci7nkPBLAgM/HEh5fDkO1Q4YpGAwU4BEKjb0ji85TnlKOQMODsBjgwcre67kzd4CSnbGEVPtw9q8Wu5uvUvwyGCsXKyw9bKlUaGgyVxYzPe61Yvtn2/nxdQX8eomyEFdHeVMtusIup19jtoUR6DN/0kuERZ+gy4OIipuPTU5z9P4qAp7f3scgxxFv6yETgnk+Ocwy/xd8m/lc2XtFXKv5hKoD+SJt5Ze8QtR6JuxWGVB8MhgZOYyroddZw1r2PbcNmJ+iyHFPQXXjq5IZBLOvHKGqswqVLUqxmwRFgle3b0YvnG4KDn3+DFkdxIYhA6BGuZdmoe9vz2liaVIjQaalB7ItCoMlc2cf0+Q0vjG+xumH52BzmMLXbKLsb2XS9OQegw6axrLG/Hq6oXbx2peSX6FDz/4kP6PHjFZDUM+H0JleiWnXznNsN3DkPeU477JnQH+A8Rz1VzdjFukmwm7rTUMegO1ebVY2Ftg6WCJTC4TmtIt+um+vX2ZdnAa83+dT1ZQFin9cnnPsQ3Z7xzqjPcnL3CrJU/1alFfufXlLbTNWqLmRWFnB+Z5Fbjm3UNd1p45Z+eQeSaT29/c5v6vt2mMiqfaVkOKWS01+hamo0QwKU/MSGRP5z1Am27+oluLKIgr4MoHV4g/FE+jspEHFx8QEBCAawdXjEYj9368R4epHei8pDPJB5K5u/kuIzeNxLWjK+kn0ondEMvQ9UPx7OLJ1Q+vcu3DayxPXi42Ny+/dYY6BxcsmmoouJn7zHmz/KOAZanLaLCRUOt5FYOhP61gHUtHS+K7xJMekk5GSAYfD/1Y/N7vg38XkxFNg4aMUxkofZUmiGDXjsLfCb8l8PiHx0QNjcKlvQsWSgvMbc3R1GtwCnFi/qU2uZnWyDqXRUNpA6M2jxKTRvdodxyC2hZ2duVPCEo4gKrDZKxfDDcpKCusFBhkBpp1pujQhtIGModlUlFawbDzw8gdlotX1zbQQXlKOQOvDGRg34HMmmGKBG2VMck6l8Wu4buw9bI18bsc/9t4Cm4XUBhfSLtR7UyKJYM/HSx+vzihmJQ/Uxj942gAHAIcmH54OmvXQtrNBpwKE/Ef4Ifcqs3jJmh4EFlnsyj37YzKzhVzR0G2ueRBCcsSl3G06lMu9fqIkY0vE9j4Me3fmcTy5W37nuvyI0l+gmRpn5Q7f2GcCP/oZt+Nm71vUjuzFqmZlLQjaSy6vQgbVQc8qqbS42YQX9h/wcuZL4uLb72+hWUJOOfe4/b6Cu5tuo3DfAcmj5pMpG4wtwCnwiRsCxPRNXdi77i9RM6JZOLOiUi8Yqi3PEZDzUUqK7uS8GsK93+5T8iYELGIqpNbU+bXBZzaxk3ABKVvkKgxGARP1dgNsXR6sxPH0gQp0kcRjxi/fDzu0e4k708WGoktCZG1OhSzOjMaSxoZ/9t47APsxW26lrmyfMtyEquvMvm7AeLrrUbjnoWexNyWI31ZyqLYRSIqs3Uey46eSEXfiVjY1xAwMACln5Izr57hrW/n8tXKr4jrEUenrJUM+HAQgz8Z1LJt4Zgtmi3wz7JFpq8jekE0Xt29yDyTiY27DbdX3kZXq6PEowRLN2Huzb6ULc7/TpZOVDZXcmDqAQBqcmrIvpyNtlErJvsyZOjVegx6A/mx+eiadYw9NpY83zweRj9EKpPSaXEn/Af4I1PI0NcLhcqOjzqi36Ind3+uiWffE7VQuHApd6Hn7Z5Ujy3BaHBBIpXQ+eXe/KkegkGqotrNC73CHftgF1yfIg88nYg5d3DGLVwoAsw8PpPvPv2O5Jpk+hqFZuutr24BgseTrgX7Uu/oS+KgFayeaVrYb70WlrUlZL93Co/OHlgoLZh9ejb59/Kpt63nYdRDnrS7wrjIDRjLn4jPrWWRnlBCGXh5IN3vaGgcW03hHRVKPyUenTz4bdFJmiwyTX4r70Yej/Y+apGaFaJuRB2Dxw0Wr1NVZhUSfVuRcNSWUYz5cQx6jZ455+Zw9rWzrE1eS36gLS4NVpQvLjcp6gZZdWJooiAXXB0MUjNMPMKMRuH/oi5sQJIfgrmNOaUVpXz35TdcNxr+tpFUC/8jeTSbChtKfiuBz+HSmkvc23qP4J/fBIQmUnlqOedWnmP8b+P/Vtqt85LORM6NFNkUTh2cyM8oo8KnE8pKRyRSwT+1FWnc+qxf7v8nHbPXkn0ug91vn2DUD6MIHReKTiagdn3SHlLnomHAlyPRVdeTvC8ZqZ+SVOk5rHHBUuuFzEyQb8u9lovSV0mKRQqp5UIDZNjZYcTbxtPv9X44BDhQmSoUspqsm7D2dsDVy5XnMp4TjyPAvCudsoV53RADUXOj8B/gL85ZrfefR8Y1Ujap6bq8K04hAqM6+M1e3N74HQ1mrxJQNBudZhHF9pm8W9M2b/xdvNby375AD2BpyzUpAX63sOCqVsvnn33GyHmj4LYEd3N3+g/ub1LIyfD80GSbOoNOABkWGzFXm6NRaPhX0UExnJMNH3Fx8EWUAyZSnV1NaWIpgUMCkdvL8a6dil5v5OGQpYweZkXezTwyH2T+W8YiCEzFs3ojvc9D1QQ1VvZemGmaKHvUgEOgA109u5L+Ujohm0KY+cdM4svjGbN1DE4hTth52ZFXm0dyeTLJY5JZsXcFkUGRohcmwNBAQfJtkH4QxQnF9FjRw+T3pVJocPSl1KjHq6sXb1a8KTL3W+fIgZcG4llczMH7Bym+V8xLaS+J85Rc54BbkQUqaQVDPmvz+zMajSABCVJkmmYMTabyilMPTCW5LJkPun1A+IvhJh7nrfPU3kd7qTSrZP6q+fSd3peRm0ZiYW+B7LEMvVFPu4x25N/KJ+9mHhknM5h1YpbIsjJKZaT2fZ4Fz1dzoM93KGwVhE0ME+9NiUEHWq1Qa2v1gg9z4WHnDOrN6zgw7QChhVG8WKuiQabD2tVanAN1snpiTr2Pxi+cyJ8mk5SUhKZBw5aILTgpnOAleHXXq8TfjSfiRoSYazx9Trvf7s6VD69gH2CPZxdP9AaJCbOvx089OHTrEJN2Ccz/JkMNALkuW9DrxzNi4wgGfiiANrUtjWf3Ync63ZKhmqli8h+T28618isu1/yK1kzYBpFKJu+ZLEiN3a9GI9dgobJA0tLEGfrLdDxaSGF3Cu8w9cBU8mrzaJfRjidrntDz656YW5uLDe+A7ABWT1qN8QPTBmGzrpm0yjThM+8GEK2ORumrfEaxAoRcpCKkggurLhA+I5yy5DJOLT9F+wntmX64TZnlr0XO0LGhhI4NpSqrii/sv6D3qt4M+XwIFRUtH7h4kV3Di5lzdg5mFmZYOlmaeKom7UrC0smSPqv78G3At7hFujH/srA2l8qkvF33NgDxW+K59M4lVDXCwm/pk6X8OutXQqxCqM6qpvrJU8w7CTx/93ksHS1pKG0g9VAqPr18RNaMmaWwHxKphNDxoSTvSyZgUABnVhzFAQe0ci1bl53gTd/V5F7PJcJjAn88XPdfe1T7jc8jZKaQz5559QwBAwPE/LY14jfHk30pm4m/T/yPUp63v73Na+++xMVBF7EzVlFX0siZ98+hadAw/fB0CouF++BR+CPkspE4BDqwOG6x6P9WJn1Iss92nrj6E1g2jfi1J8nccxfzQStoUnpwo28KlU6VZDiNxVj8M7V549k5dCd93+2LsX/bPTXt52ls+GwDGMHWy5aXsl/iw2umY7q5jbnYTFPYW2EwAzN1I9Y1hQQN6y8+M4Co+FLjFkpQF3skMglf3vySH+J/YKb/AqzqO1HsUs/drhmEeTjT69U2NYnCuXL+rBZAmPkR0+loadqM7ZS9m9MO1ihrlJg111JTXk3JgxJ+jPyRJQlL8IjxQCaV0fVOV5TfKSm9XGqiWKPWqcW/yxrLsG+0J3VEKn1C+3Cj7w0eRTziUcQjpnaYytnXzoq+lJ5dPfHu5c3QLwSJv+T9yTzc8ZDgUcEmzLkN7hvoMLUDXt28aCxrpN+7go3Jv7Jh+H8bfv38mLJ/iokFAghrh9b56jPbz/Dt48ucM4IChLZZS2NZI1ZOVmiULRYFGiG3NRgQn1OAFd1XsL1mO92nd8fW05YnF55QnlzOmB/H8NHAj6hR1RBgL9S+lL5KImdHPr0bNFc38/jwY9xj3PGIeZYR6xbhJjbpFlxd8IwPasDAABPmuwxhjHEIcKDnaz35V1HlFYHSXRhT+q7pi14t5ByCXKYUo1QGEilFd4tIOZhC9IJonEOdic7ewb2gaYSmeuNZUkjPTQPps7I7cXFw6xbQRQAfSY1SNPauTNol+OOl/JlC/k/5FE4phHDI88tDHi7H0smS+ZfnC8/sH237V+xZjH1Pe4Itg/Ef5E+zEcx1zsj0bezEqCVRWNu2/duhyoFp+6dx1+0uBp2BtGNpTNk3BXNbS1xrR1HgcwwHvZUgFfVU6I3C/OdY6cjoEyOxGmjPyO9HYuthK87bwcVriHk0B8v6Mjw+88bRTY5OB1nZbXnxks1LOJh6kH7v98POy44uL3ShPKUcpZ+SLku70GXpX0CI/yGanyqnOLRzwNzWHKPRyMjvRhI+I5yt/X/CvcSd291v08XjBa5+dJXE3xN57sZzGAw2pHi/gceTLCIf9cEQOVdsYEMbQ7PEvYTvX/qetX32/d0uCAp1LUpxDaUN6NV6zD3MUelU2CpsKUsu49pH15h2aBqzT89mzjrhGXrL7q2/3d4/8X8r/mn2/RP/Z2Pmnpl89uVnrGlew4C1AwAwtmifGcwUhP78BsOnwreB34pI0FapvUYHb4xSKUaDkfxb+Vi7WZto8U8+OJkm4wH6lL0kvqZrmS00Cg0VHtboHV0InyEsICwdLDn95WkKFAUABBWvwmAQ/LtCRoeQtCeJ1IOpbFZu5tjmY/j39wfg8dHHPD78GOk8KYXXBeRURFIEdw/cZfap2fgP9EcikXAtPAL3Ynd63u9D+T1fsrfeoza/lu4vdxcL0K3IJaPewG99txM4LJD+77UZ2DqUpOKSl4CmIYbylHJcO7pSX1RP+4dFBEUEkdUuCyN6hn49goEfDcRCacGVSVeYZTOLDW9s4MTUJuaHtGk6u8e4EzYpjIMcRF2g5v7C+zxa94jwGeHPXKv6onpufHoDS0dLeq3sxeLbi/n0U9CmHhbObaANPbv25Ob6mxj0BlbkrODsklEYpAa2LTrIln5vcu2Ta2RfzGbO2Tn0cB1G9JNdODY30mwjp6GkgRMTdtLn7T4M/nQwkhYGQIFPAQU+BUjMZFx85yK5V3ORmct4t/ld7k6S4ZO7EZ15JW6Rbsw4OkNkW+lkOqodqnk4/SFL+y/lzxl/IpPLGPHtCGK/jkXpq+TyB5eJni8snJxD2yRTE9sIBpjbmIsLuNr8Wio9w2m098I15w4VZy/Tf21/cq/kknMlh/KUciSEYluVi0/KORoLvFHVyNjebzudl3bGMroDt5SCjGOV7XXUaoG5WZNbw53v72CNNZ5FngzcMZDGdxqxkgvFkVtf3eLGpzd48fGLOIc683PXn5HIJCy+vZim8ia+C/yOri92ZdSmUcjMZbyc/rLJtXMIcmD0qdFcHHQR2aiOSGVSUg+lknoolcGfDUYqfZaZMX77eDGRb98e4qsjqHFvz+AgBUHDZAQNC+L2N7cpOZXHrORZHJ54GO/K+WIz/tKaS7hFuFH5eSXZ97IFXfjiehDISbiEuZB1NouudOVWz1tMvDiRoIi2ZuZblW+JzEZ1nZqyR2Uiw66psonSpFJRatI9yp2I2RGY25pTllxG9ZNqKh6VkBuawsHJezgy4fIzx2ffrztJ2jLsyrMwmJmbNIHsvO04Oebv0Y0hY0IojC9kacJS1HVq1rusp/MLnU2KOdpmrVDUHB3MgL0D8O3XxpKbsGMCjw89NpHxeDpufH6D0oelvFXZtkD7qwxLs60LhSED8PRp6xqoalXEbYqj+UYzneSdKHMqg6ce46PPHaVDdge+nPglvSb2ImKKkLRqGjTsm7iPyLmRLEtahl6rp+pSFQ19G0R2ZkFcAfGb4kXWxr0f7xEyOgRbTyGRj14QzfElx2kobmDG0b8xSGyJsVvHil6M936+R+bpTMb9Og6j0RKdhQ2lQb3w6ipIJ6lr1ZjbmDPnzBzGjWtjKhkMYG5rjqWDJaoaFb7fa1nYNAe9uQzHokdoi/vxXdAf9HitB91e6sZju3fhCtgkQXLjKMZetaNnh/589PvHbYhhicC0VNequbtFkDiVSCR4107Fs3oqMkUKMYsyubvlLmWPyph5bCYGSVuxyP1JLJkNBhbeXMiIsyNISU2he7AAmigK7oexSxfMLMyY8PsEUb6sc+O7JEv3MOagHY0Ox+h2ZBYhY4QGRSubosG2mbyI0bjaNnNh9YWWJlMwtCQHHkUeTPlzKo3jHtBgJSP7YjbtF7YX96uzR2e87bzF+0pVo8L4Zgtjwiihy+dT8LWp4uI7F58qDktQWah41PERPQPbfI6grUjpWRRIwOMcGssaaTeinfi+yDqSSjFKwLGdo3jv5t8UUMd9bvQhv+NInOsGmzxzrUVUxypHRh0PpbJTDuNPjgfgwuoL3PziJg7rHZAYJJhrzFE3qcEWsRAHAphAppORsSkDeXc5fVb1oc8qgfWluSAUf75t/Jb1v63n7jt3yb2ai523HZ0LOiPXyqlyrKI2r9ZEqre16KyX6THaG1HXq8k8k4lLBxeUvkrxHkrpkEJ2QDY/2kziQ9mHDPxoIB0W90KhMket0FAcFE6N1gdtsw6dSpCSTihO4P3LgnR1h+QOPD72mI4ThDnB0sESqylWVB6uxPl3Z9LkaXRZ1oXgUcFEzIrgSYudrVEmR2Nlj7apgavrrhI0LIigYW3jqNSgQ1tWTddVg4hZGMPOoTt5cuEJze2aSe2QikLrhl4PQ1v8IgFyZliR1LAPlYUez6pQIvMq+GX2XkZ8N6KF0Sgcc0haCN3udKOiTymqjDLif4gneFzb+uJpv8iEXxJ4tOcRrGhD/eoNgqeUzFxG0NAganJqAHApNUeqb35GWcBohGrr2xgleiyMXdBrzKjNqxWBVwYDIJGgsnbCwsqW5y8+z7TQqfSvNaO77u+bOEqEZtF/E6WApYUlQYHC+Q0cEojcWk6VeVuh1KeXDwtvLTTxBzEaBUk5d3fBK+fpwuqLD19k8ORcbFQ3aJCeBsNwE6BV6/31OOwxtmY2SORm2HrZYmYppH6dcn8nwW8eW5Ztpu/jDCIW2XN/w0UOzjyIxReDOdJ8BBujB16p51E01dBUMZxdw3bR7ZVu/N7vd/Y+2itsJ6ETmRaZ9HtdaK623vdyrQOHhiQQ2t4Ura3SailyOExYQikVayVYLlxmwuZvfbbzIkYz9DkDQxe1zYGPrsXhn3QDt1I/ArOLUK9Uo7cy4F7zn6/Ba4AXMFcOj+VWGLU2uLsrGTNuEr+/+yru7kKhcoD/AA6rDuPfzp8BVgM4tviYCaABoP+V/pz/5Tzy9XIqZlYQOTqSu13vEpweTMz9GKJGRvGV21cEDApg8p7JBJr1xqluIJXKyzTL5GSczOD8m+dZFLsI7x7eRBRuJM1xA5nuX2A0ruPu5rts+uN7BijkdH+WkmkSrTKxOZmn8fGeilP+A7ZEtG072CkYc5k5bqVuNOQ1kJiYiEOgA/4D/Pkm9htxO0+j21tjeNBwzj85T9QvURzac+gZScHSUsAjjBqPMKxMnQPE7TlUO2BRrMHa1Rqlr7Be1IusPwkPRqx+Riq9tVGd5/IT36yw5Q2nq1z/9Dpx38ax8NZCHpY8JKMqgxmnZjCh7wQOTz7Mn9P/xKOzB4Y+bcdxXneeBTEL+KnTTwQMDmDehXmEOIWQWpHK5EOTuVx8GdcIV8oelYlstHynHdRZ3adD/jdYeyl5o+wNsYHZeinyw0eCD+jUWtKOpeEc6izMBwn9yPayx735YxwbehO79iyZBx7wdv3bJuc3LTQNT+s2ZlTKnylo6jX4dPLhzuI7lBnKcPP81/qR4Y/CSUhMYGXhSvqt6cf5822efVKjGdFEk7Q7iS7LuuDb25cGg9C9mrEjhPtFpwjfO0rclu6p+bNDkhlVGZXQv61oLwAhZFg2WdIhpQOGrrq2PE8qxVxrjlwrR159kqCH5yh7MAUPDwFk1KxtJq9WYMI5VTpRc6aG4rnF2PvbEzw6mJO/nERl0YZ4r8mpwaA3tIA22yb53r69sZJbMebHtrXy09FU0YR9gD0TdkzA1suW4vsCsEPppxRzEZ1KR2N5I7Yets/IgJrbmBOzKEb0whNfb6qhNFFQLmj1s386no9/Xsw1IuZEmABIng4bNxs8u3gSOj6UZeeWke+Tz4bYDQwJHPLMGl0ikYjS//m38jm1/BTDvh4mNlI6LepEp0UCSG/K3ilM2Ss0yetVclZfX8KDmAfYNLfHIFewa/guPJwCaB/6KSMfv8Ppf+NRPVoqxyFKSVZMFnahwnHMPj37bz3GTr0osIDG/zb+b4/36bj2kcBCCXwSSEDORbLP2dJc1Yy6XmhItT4XqR1SsTXzwNza3KSxdNXqZYqUgvdeYNkKSm8LC5jIS99S7+iHh9GHC4OrcClzQVlYhrWrNaN/HI1HJw/O1pwVt5MYlcgbAW+Q8HMC9YX1oi+ofbU93e50Q2ojNfGnu/7+OcKu55DWY/7/j723DK/q6tt9f8vi7u4JJCQhQQIEC8GKe3EtFG8LpUKF0lKh7i0tRQqlBYq7ayBYiJBAQkLc3W3Z+TCz5soqPO/77H32tc/1Xuf5fyEsmTLmWGP87b5vKr174jXA8L5007PGLQSbSSFc+/AyOUdzyJ+Uj9EULUM0STj0/ZEq7yCMfV07fU+L9U0lfkZ+ZPtno0VNc1Yxn1j9TuyHsYJenFaIERb/thit0Qm6f9kfx0BbbP1tsXS1pKq5ik3XNhFuHI7aWZA+OL70OMpGJZP3TBapJwdeG8iBPQeYdWAWWmMtapnhXhLhEkHU6iixCfX+b/cNEGvBk4NZ37Be1F0EYX72Xtkbl0gXknYkUZ1VzfDPhvP8wef/27nw/8a6TXs2W5WOsrXX8l4GVLkFNwvYPWw3o38ajVbSm/yQkbSZC77VPyV/FkQsYKDXQLo6dEVuImfehXnifjA7XKAjfvDXA5LvJzNs8zBRYqC9Hb78Ero5NpC45BiD3xuMnb8dd368I6DxhvrxLNNd87Os4GYB5k7mT+nEZV/M5uHfD4l+LVpsHC337UNAh1JR57VJq4V2MxseDl7OxMlQmnyfG5tv4DPYB4cuDhirnInOuErwjV+Rtydj2yUGU1NY3fEIZWoLWk0bSRw2Cy8LfUwQMCoAx48cachqwFRuyq5Ju+jp2hO5sRyfGB8A3o95n/eu6CHRGq2GrhOE2PLSJXEEAOh1txcWMyzoPUdI7rjWTKFBtg/jNmNUrSrqC+spul2ERqnpeGa6hj8NylY1l7+4JuQiJwWLPoNFowVuOVlohnsTtUroQG9rg/PhrmikrSzcswW3nHTaGkLAWcFPP8Hpi315OfYi39YORS1Tk30xm9S9qUzdN5Vuz3cTpSw62+3vblNwo4DJeyaLTaMnV5wkdW8qk/dM5vji44z4agQqlT6ZMvp7Yd/9c8yfZJ7KxCXCBYVKTrtRO2dGneF1X2dkFZkozBVIpBJa1c1UWl3Av9EJ+7JGMDFl0Y1F+ufccc8qhYoqhyokZkakH0nn0juXGPPTGLG5tLmqmcpHldh3sefYC8fIuZRD0/EmPon7BBsTG0qWlbDi4Qpx/5rYdSJ1rXViDuA/9j/b/lPs+4/9j7Ck7Unc334frz1eSDo2iVyfXKb2nfovv6PbzHU82p2tNECA16valOwYuIPwueE4dnNk48aNnB1xlvh+8TyvWPyP4+m9A5um3gbOgq2fLao9KnbE7cCnYATDLw2iOjAXOrpNrb2sOTn6JEkuSfye9Dur+wgFldLEUpJ/TyZosr5r7vzw87z282sEDNYnQgGs6q3onhJKQ14NE3ZOEJNZuutQtNQjb29G1WZP1eMqnMIM9QNKAgZS4dUTUztTXit/DWWLkntb7nFu7TlcS1xxqhmCY6MCuakCE0t9QsmyUUjKV1lcQ1O3mE2Kb+jzch9GfDGCHot7UJNYg1G7EYHZgSJdSereVKRyKSFTBefDubszqzJWYeZo2J2rlQhJ1AznekZsHsGe0XsoSynj/LrzjDw3kmPjjpEaXojUwoy6vDqRLs5UZo5H9WxaXeCxC1j6tjLm5zEi1c0/qQC0Whj41kB6Le8lcqwDPIh9WV8BRtDbCvo2iBXZK2iwauDDNz/ExdFFpCsJHB1I4OhA7m25x8nlJ/Ee5G3QKZrVAVgwrynEqKUOrTZEdOosXS0p9+1Du4klaoUxFiOj6bG4B/3W9qM6s5qmhjY8fviccndzGvrPJ8bXCYWZFK8BXiT8koBV3+6k97DDO9cbt7xmNBo4OOug6BCW+ZRxbPwxIhIj2Dr8GEuPTsTE2gTPaE/6vNxHFJ537u4sBqdyUzn91vUTqab+aY8OPeLm5zcp8DWi3kaCf60tGpWG0uRSHux5wMC3Boqf7X7uC44sCGDizokGnW1arZBUVskUqLQCiqMmp4b5l+fz+5Df8cnzYeTZsfgWPqTd0x/wJOb9GE6vOo210hrJUAlhD8JozWmFLQL9rIWrBdGvR/P9he+53+M+H1V/xG9WAtWcRCIxoKDqHBgDRMyPIGJ+BOXlcOgQjBwZLFI8nXvtHPFfxDPp6ktsuCyM66HH+5jQPcZgXNJ9cjhg/RWUZ6F9oiUpzIrx02ewaMELuHu7k/1SNufvnMeqwIrGUqHwpWxW0mVCF4Z9Okyg17IwYvz28Qbamhffvkjcx3HYd7Fn5aOVRI6KxNRSuJfqrGpS/0olfG44Ndk13P3xLj0W98Day5qmiiYKbhQQ+1EsbXVt/FfWaulIieVgjNyh8FYhh+YcInxuOFc3XmUUowCoy6uDjj6H9sZ2ylPLsZPbkbI2BVtTWyzNO9aEzCqyL2RT9biKRTcWkXUmi+NLjjP7zGwCXIT168xLZwwoYErul9BW3yYW+y6+dRH/4f70eeXpFETO5Rxqc2uJmB8hdqWBQGeXfjgdhakCyfVa7O16UOUZAUBhfCE7B+8UiwtarUAHZ1+UQkP+CMb8KCADi+4UYZkJlnjwwbsf4FO+mpckEoytjZEZy/j6yy/RfFfFIBkdOkmVlLZWsq0sH1+fXcxbtwBM4U7NHZb8uoSesp5isU9hpqBJnk+Sz0LkAebs/voYR184Sn5cPlptZ2IheNJzGi9sluLZz5H62/VQD9IO16zNwp42J3tkRkIHq87UapBoZdzvcZ/e1hG4RLjgEiGsffYmrviXvEmt+W3KrE/i1BzNjU+FJgoINBBmV8mVaCUSwmaFETYrjOqWanggnONY72PknM+h1qGWcVvHoW5XcyZRQKlmuXzK16nNrHGbycO/HxokZuqt6zkw7QAbh//N0YVHaa5sZubxmWJyNz34Cd7NR/EaYIGyRYlMIUMql4qdl8aNVZg0N6JRe4oBdb9Xo9l58S/yvPMYfN4EufY6hXd60ZBXTfDkYFGbrMa2hrOjHxIiXy5eT/S6aCxcLRhvNZ7bJbdZ8dUKEpoS8PnOx2CuKdVKpBopSR8mwQJhrYeOItGFpfjLHvPE6CAJxQkMrh2MuZM5I74cwcSsiTTnNvPi1hdJ9EwUG49An8RKD05HuUSFQqpgV+wuUbRdNxOURkqURkq0lsZ0m94NpzAnfgv5ijdq1rJz/k5uB40gqPgDjk6opb2miVfyXqGgroCTmUJTwfDzwznw9wGudL3CxF0TkRnJ+PXHXzG2N8bhmAM5vjlEr4tmz6g9VDyqwHeZoHkiUatQtDVS86SB+C/jMbIwwn+Ev/gsmmw98P3hVbo818z1j69TmiSUtgKzArGttqVb+mCaAmoBG/Gea7orSGhLwK1qJmWtMzDzr2PYp8M6UfvqaWc8CzxprW2l+7zudJ3YFbmVHOKFT1n/bc3+3ft5/uDzDHxrIBELIth1+iPxPKVppbQVt+EzxAe5sZz19esZNbGYKsvbyLU2fBThwu3vblNfWC8K0d/sMoiRZ4finT2B6qwp/NTtJwa9O4ghHwyhWpbG1ZCZlNhsJNZ1COUPcsnWPuDtf1HoA5gA7EagVPzv7HcTE154azVDXxQKoyFTQwiZGsL+TuxtpramePbzpKmiia89vyZiYQSyYUP4/HMYMgQWTqxh+4Dt9F3Tl/6v9Uej1lBreh2Xx38RdTcK1dgBgF4H2JLOHeYanPr4sviW3q91rhMSD5WOlbTnWXJ+5VHS9yYx7e9pHK1shBZolJSgUl3Bpl7QoBn761icujmhKdYXLr576TuerHnC7e9uc2XjFYK3CH5B10eB7Ou2iSl7pyCVSzF3Msd7oDeNbU3c95+OXfYwulRPRNWqQt2uRm4qRyqTGsw/uwj9HWg1WrKX36Qf/bjZ7ybVjsaMkcuwtrKitLj433gKoASMLCWsrH6dCs8IHAuS4EfYtXcXqx+vxtTOlJSyFPal7cPZ3Jn5EfN5dPARrj1dBVhgh7UbtaOwU2DtbY3xbGPKjAXf1LbGluBHwVi2Wgo6zUH65KIEGUZtRqBuJ2hakMBE0aHR1CqtJNvlK3rdG0S7Ipu+a/ry/tXXeL3o36P+WaqC19OP0v3RR2RGLWLwe4MNCg9SiZRv1nxD7su57LTdSf71fHxifNA4d9A+lztSdamKh8kPyTqdRcz7MVi5WyGXCntR9YRqpveezh/P/YFdoB2jvx8tIoTFsW1RUp1ZjYWLBeZO5mg6fuuHphxCajySTW/qtZYUUiPMW4PQoqZVXoqV2pHdI/7Ed6gvA94YQF/NaxilrOJyaBBKRRUqrRIzRzMcujogN5YbFM50jQGPDj1Co9KgHWCYydUVe3Iu5tDe2M6YwDE8qnyE5BUJ/Yf0x3+4v4EOVLLvAgB63w4l72x/vJcGi+91rruq1UKR6eCMg/R5pQ8m1ibEnPGgh/1AioMCUbTl4DLHF1snBVKF1OCa90/fz+KHv/Cdz3eELQqj65iu9Hu1H5EvRKK2VmP/ij3O5s5UpldyecNlIhZEEDg6UNwz9k3fx5PVT/Rj0Emzz0Xbk/kn5/O1x9c8PPAQr/5eRBkv4FLrF7QbtSMxktNa14pGqcHMwQy5VkDLZAZmcnB2C9O7OHFsyTFCpoQQ8FwAWi10z91OnvVixp0Yh0WnQlnApDCmZr1ElVUmxm27iGgZQ/qf95HVVRM6I9SARvFO1B0W91nMoVmHmHNuDv7D/dmydgtt6jZCH4Ty8OBDbmy+QVO5sMd1Hq/zq85j5WLFkPf1yKrOtrX3VrHhA2DsL2N5u/VtGksb2aTYRN81ffEd6ivqX0UuFCjUanNryTqThW+sr4GGls5yekzhu6eZ/EXTxVUAo74d9dT7j08+xtTWlODJ+jjjftV9Bl8ZTK/NvanuU/1UUh+EJkWJTIJjN0fmXZpnQMnW2R4eeMj5188zftt4AqdHok19A6l6Bg6V7qiMmhj59Uj2nbQioHU87ukXGcB5+kulLNGoDZHN7UpmdJvBuWiBbUZHn6prTP6n6VCF/9QbfJbNPD6TQb8Mot6qnuBHocwMXU7fpXp/ViLtRCkoUQrSCq0qgfpbIUOX4O8f1x/HsnsY2ZpDThUVnpFo5EZUml0BhGaw7ilKjCyMRPSN9rx+/l0bcI1Lb18i8oVIIS7vCCksGyyJjo+mNdxwrU36MR5zQKIVxqKhuIHC+4Uc1RwltmcsanWU+FmNBirSKrDJsgFAphGKY+l+e/Fq2IZKBccWH0NuImfU96Nw36dmHvP4ftX3OJQ0QogMj74eYlykQ8neibqDT+V4Su4VIjORMXiD0MBd3yA0T6Z0T6Hfon44dHWgOrNajP/a1G0d46lFIpdg4WqBZJuE+CuCg2VTY8Ob9m+ywn8FlgP1PsPE3ydi5mDGnR/uIJFK6L2i9zOf8egfBN8hZGqIASX1v2NSqRQ/Pz+kuu7ef8Nyr+aSczGHXst6iWMEwu+3JrsGt95ujPjckFbWysOKfq/2wzXSFe11KPfTo9LVSjWX37uGS4RQKIpwiSDcIVzYPzSG8gUA6y+sp/zrcrzueuHaw5Xz684zYccEHjT7c/Mm3FJZ88WpWdgF2NFa28qlty7R79V+YrHv4YGHXHzrIuO3jUdhqkCj0hgUVZ+cf8K9n+4xaMMgtvffLmrXFicUc3zxcfq/0Z+GkgYSfkmg+7zuBuuB5hnD3zk/WV8PMdO64TPYBwtXC4PP5YWNRaLVCNSsjaD7UailQvFXpjFFUV3G0RduEbkwEq8BXlR9XcUw02GUDSujfFo5503Os+TeEkDIv2wYvIFVUatIKE4g9cVUbnx5g8H1g8XrKrY9QK25IFY95PIQkmqTxGKfZ+VC4kJ+Zuuyo3z7yrcAYv6tsqGeMtsj9L7TmwE3K6hcWMG1TdfoPr+7QbGv0KOQT1//lPfDYwzGSCmvQSNtozgghAbnKI7N2otEo+a864vIseTR5QiIFGQRiiYVkf53Oo4hjrTWtdJa24qFiwU5l3LIuZTDgDcGUBhfSNr+NCbvmSyex9rLGtdIV+TGcqy9rTGyMOJZEm1uvd3IPJVJaVIp50cmcKPfaYILPkOjwaDRNOneRerNkjk/Aoq7jGeJlaXBcXR5aalainmTOaqmdrTGWrRqrUHOOu9qHvun7GfKX1PoOqkr2bbZfBL3CQC1rbUCHb21CbV5tVz78Bqz84SoxmumF/+x//f2v7Pm/R89//8nZ/2P/cf+F621tpX6vHqBXrCDovFm/5tM2jWJjGMZ3PhMSGaKa5tWS92NVFH/7V+ZVC5l5DcjCZsdRnmKoIfiXOZMUmQSkp49uPXtLc6sOQPoO1Edy93pd/QHjE8d5guXL7jy/hWgUxdrnQcuOXdpyCyhJqeGj8w+YseAHYw5NQaTVhNK9pWwNWor1VnVoiB4yTdCF6JUIuXkupOE9n8aHZcVkMXmN77BZUQYDl0cREoA3T27Zl2n2/VfUDa3s65sHWN+EpLaulpWu5kNzTZuohOjMFUQtSqKn1ce507UHUac8sAzI5XavDoqHlYAgliz/EVrbGpsCEu0R11eReCYQByChQLF94HfUzG/gnLnctL/Shdpsy5vuCx28x1JP8KsE7Nod23H1NaUuvw6rn9yHW1xMZqOYl+56VWya7KZfWo2awvXUptXCwjBp0mLEaoWJeN+HcfrVa8jU8ie6sqSmZvQa1kvPPoIzpPOUY69GMvaL9fSUtWM/wh/QqeH0lTRxJHdRyi2+JJS22O0KoopTy3nzJozNJQ00HduX96c+ibbxm8jxFEoVgaNDRJFggG6Pd+N5anL8ejrwZ0f77Cl+xaqHldRJdDB45xzC//7Byi6U8T7kveJ+zSO5qpmgm9swzUrjgYHP2yfH46RuRG5l3NpLGtEqW3HrryWYtvd3Av7HomxMQozBYPeFca09VYyinYFA68PpNf1BjQaIblRfK+YPq/04UnUE5CAW7EbtVczULUKgX/QmCCe++Y5zOyFoHT8b+NFhJSJtQkjPh9B8CR9MiNpZxKPDgn0XFWPqyi8VYhroZToxPWEf2hMY1kjgzcM5u3Wt3Ho6iCi8VosnbD27ujaVqrFrjiNRq8T117fyo6BOzgy7wi/D/ldOF/3JIrdy0CrRRfb65L8ZjfMeOfDdzg96jTyrXK0Wi2HZh/i7g93Gbp5KMfHH6fKvopDSYdoaBUoyXTFyIaS/1pY+LXXYMcO+OUX/WuhM0KZsHMC9TnVvPXRW0w4MgH5A0NP7esvv2TL25PolnyVb+uL2N9WzJr0dK5//Al+vl68NGY1CcsSsNthR8aSDApvCWjfyvRK9k3axx/P/UHc5jgaSxuJXBiJZz99oTXuY0H7o8uELmhUGhRqBRplh8ZTZTMP/35IdWY1RXeKuLbpGnUFgk7h3vF72TdpHxbOFvgM8eHH4B85/7oQtF965xIP/nrwzDGQyCQYWxpj423D0G+HcrfXXc6MPINiqb7Y/3P4z9QX1NNa2cr9hfexldqiVgrBr2ukK6/kv0Jdfh1fL/qaZXnLMP/Q3IBOpqmiSfx7yl9TeKf9Hey7CChqrVZL3CdxZJ7KxHeILze/uMmZV86In7/38z2OLToGEqjMqCTncg5qpZpR341iXdk6Hvz5AElBAe7pFwm+/iv3frmHhasFPZf1xCnUSc/T31SFTWkG7fX6YN7Wz5a8GRq2LN2CRqZBKW9E4mDP0vtLuVZ/hU/efpsbWp7SSbqEimtqNYe/+h0EZkTevfwuTt2cWP5gOWsKBcI4taSVKqtLVFkIhP0Ttk3grca3hERlp+CsxcoZmwAHVG0qimqFouijpo4DazRoVeqnKF+E5KaWqzFXaehvSJtjb+yEU90o2rmH/71DGJfnszprtRg86JB9pa6l/LziZ4x666mFOifXiu8Uc+GNC1RlVuHW0w3Pfp7i+xpZC9cu7KfgUQEb1Bvot/ZpmhmNRqC20XUM6/bPRosSGi3llKWU8bHZx9z+/rb4eQDXrDh8Lu2kIq2Cc+vOkX8jH2sva45NuMKjkEeYNLdg1FpH0vb7HJp1CGWTUjx2i1kLRb42qIzNOTzvMKn7UjFzMCPjSAbnXzvPn3P+xHW6K/7R/mjUGjJPZVL2QEjSt6haUMlVnF13lkEbBlHxqILMU5k01igpSvPGpHQwlvWWWJ+zZvIfk1lXto5uz3fj70V/U2Nbw83xN/Ef6c/pl07zjfc3gg5QJ53C1IdqHLo4MG7rOBHtre0YT6M2I+yq7JBZmzJ171S6TugqUjE2WDbQLdma8JsZeA4NJHiKsFZ3TqIeG3+M7ku7I5FJkEgkHF14lKm7p9I1vSvfvPwNgzcMFhI2Wrj9zW1xrM3rigm/9C35cfm8nPMyfV7u03Fdhs/xr3F/centSzRXNtPn5T7Uu3Vl2qkDDDsXQn1GMYk7EkXtQ7M8Je6F7uiSBkYO1vR/vT+uPToKTxLh4Am9EvjkrU9w6eOFwkyBhYsFJmYmvB8j6O1JHkvIOit0zziFOuE/wh/7lmjxuhK3JLJn1B5KE0vZpNjEpXcukeGxmXtBU0jxFjSf0o+kc/83QRvkceN9tFIlxm3GyNrbMHMwo/+b/fEaKASzF+yn0mD2gISAKWi1cPPzm9RU1/6XNJ2Lgcv8ax04nd0GLrW1svCFRf/l57QarUijbelmibGVMbt3C+9dvgyp+9JoLGkkaUcSIFASj9/zBMsGSxJ7FCI1UnB/231S9gj0AgNl67Bt7Mf4o+OJPpv+1Pk0mo59tskMqbKJ9L3CcYMnByMx1e972xZ+zNGZEhRmRvRc0hPPaE+DtaLZvBmFtQJzZ3Nce7giMRGO22DVhMdzoVh7WXNk3hHiPhH2t1v1An3ZheEXMFs/nzs/3OETy0/EZq5/6if+Pe1v4j6NE1/L98znUuwlbvcvQCKXEyDvzu8KQ3qvf2VbpSY0hWvZsnQbpQH66p1TqBNSuZRWVSvVLdX0vtMbyzcsaapo4rXK15hzdg62DXoNnvjoeKKOR+HZzxOjlUYUeAno4zt97vDtujN4D/Jm1slZYnGiUplHrdldnjvzHMGf5WDhYkHI1BDM7M2EBpCO5PCoU4NpvXYHt15utGhbxflXCnyIQG4Q1PHvh+iRpc5Ai7SVXH8VTXaexGyMwcrDiqzqLNZfWE+rStj7Wuv0e6CNr424jkTdiSJzaSa5l3NJ3JZIa43wuZuFwl70qOcjus/rTn1hPc3lwppeIoQt2Bck4X9vP/lx+WzpvoWErYLel7bTHFFrDRElPlb+RGWeotnkCQWmIzCpKKTodhE1TwQ6Q7nWFCO1HSYSK0LSQpCkltJzSU8WXF2AlYeVHj3arsD/c39uf3ebd5XvMu3vaSzpsYTsl7LpYt8Frzwvij4pEtefvGt5jA4czeahmxmxZMQz6e111jWxiIw/EyhPLaepXPBjdHPTsjIHk7QETGxMmLp/qsDy0REX1bp0wTXrBj7JR/GbGM7oH0Z3FCgN93G1VoOVhxVWTlZ4RHkw4osROAY7MnHvRPy/8+dizkWaKpp4+PdDqjJ1WlPCMRotGzF1MSVlTwqX3r2ESgXGSlfcqmbirRqOmYMZ8y7No/cKIYk6UCE0vf764naCXx7B4TmHRZ1VY6k5fqWvojRS0mSpob1JSeJviZQmC7PrUP2bpHu8QZ11Hbvm7sKo3YgvnL+g5H4JGfuSWPrTKGLvbCelewrblsVREp/L9Y8EJJbuOckkMu4uvUv/mP70XdMXay9r0vanMX7/eCzrLYm9FMv1D68T+UIkUS9FGXwXIOt4Fjc/v8mB6QfIvZr71LPqPl8oHtn42tBvXT9cIl2QG8sx7migcYlwESjx1vUz0LwqSynj5PKTFN4u/JfzQGfFCcXc/OIm9UV6HffqrGrx2TzLDkw/IMbCIKB2hp8bjlQjpdCtDrmJnJxLOWSeyjT43mcOn3Fo1iFMrE0EOvMORGxdHSyfUMyy8Js8SW4gbnMctTm1PDn3hObyJpwL23ErdmPBtoG0X7lBr6W9aPcNQi03QTN0JNbz7cgKcOZtHJhj6sy3wcEMfvttjn58Ar8H/rgUurD+4/Xc3XRXvJak0iQ+uPoB+1L1FHFzzs4hZmPMf4lQ0plnP09C+4yh2r6a+71zsQk0bETWNWsNvDaQfufSKbpTxCcWn3Dzc2HtkXTkfPrc7oN9QRL9vn6eQUfXktd9PA123jhUueBW5EZSRBL3BknE+BFgac+lHJim1yzPOpvFubXnUCvV4vwqcS3h61e+RjlSSfLuZO5vE3yGfu+NIDdsLDJVOzal6aTuS2X/uP38/ufv9PlNL6HinB1P2We/M2HHBA5+chCAjRs3snHjRqotrhB0+wB1CVkU3y2mJKFEHLNm02b63O7DgPPlSCzNmXturtggfaOr4OtcH3SdMr8QiuPzybmYI95HZ+YDnZ85/9J8UTNcR+MZNzCOYVeGYe5ozkt9XiLUScgtRVdF07qhVUTA6ix4UjDeA72588Md7v18j+aqZoruFhno83U2E2sTgSazqZ1Tq0+Rujf1mZ8rTS7l+sfXqcmuQSKRYGVl9W/NHZ3lXc3j2qZr/BT6k8HrD/56wK6hu0RZms7m0MWBEV+MwKOvx1MFMY1Sw7UPrpFxJINt97dhs9mGtbPW8pHpR1RmVFKaXCpqz17Pu87ulN3sG7wPs7/NMLExwdrbWtBerOs4ntyYwFGB2AfaY+FiwYsJL4r+NAixttxEjkQi4cSyExycedDgeury6sg4lkFTWRMjvhpB+FyBLUWrERhB1O1qei/vzWuVr4mazQBu6ZdQ7xXm9+4Ru9k7QWBb0GpBqmrDKec2O9/Pw9jKGLsAO4zMhcJtqedPPPR4jVLXShpt3Tk+8y9+8RSa6bRoQaJBppLh+iQDiyfJJG1PoiqzivamdsymmHGv1z3kUjneMd54DfSiNqeWD6QfiHkHO1M7hvsPp+eQngSNCuLkypNsH7AdjQbKrI+R7fI5AHtm76G1prVT3C/tuIanK5jlTRX68dSC1EjOiocrRKkPfXyqocWshepTKfzU7Sdq82pRq/XHPN03msTQvdgHOxlQQkvQ+7z2IfbEborFKdSJhF8S+NbnW/6e+jf51/OJ/yKelpoWJv85mQ3qDWLzPMCANwcw7+I8fGN9eeHmCwSNCaJzfSdxRyLXP75OzMYYeiwRmtCHn+1J96TuDD4rpyYp1+CeDWJ/iQZ1Y4vAcnZZWAtMlK6YtfrjVO7Eq1+9SsvxQoInBbPy0Up8BvtwKvMU711+D8cwR0Z+MxKXSBd6vNCDdwPeNTiPqk1FY1kj1VnV3PzsJmn70kjbl4ZS9V/T4//H/j3731nz/k/af5B9/7H/EdZ3bV+snrPCxs+Gf6K2Hvz5gLR9afR7tZ9Bkqr4u4PcKgqmoaQBS1dLkVoLwL4wGfOaQiSSUaIGhVd/L+a6zxUpRVpUzWSeyKToThHPff2cuJG0G6uodQnGxMEZe7taoQvqxzuYrjJlSugUjkz8nYJVAeyMXY/cuA2fGB9MbEz4rew3NFINl1IvMaNsBmqlGhtfG0EPZ4kJPIYBXgPoZd8LdaOaY+uOUZZSxpI7QseMRqZBadSOBintje1otVoD/vJa5yCUJpZIjZ/Nny9Rq5Bo1Kja5GQcTRfO3cONZrNW2o3bSenbFWP6cO6V0zw5k8m7yncJnRHKWYf7OOx0YMA1T7T2NSJNVOq+VOry6lA8L5zPSKbv/pq4c6KIxFl4dCH1TfWoylXsnbuXmuwaLr11Cdk4Y7R2Qtd8SFoIxyYfY+6vc7EPtGfGkRl4fu6JdZ01i7ctprTmLkzSJ/pyGjI42bMbQY97MPD+R7TUDAQ6U40I5241aaXVzBhtx/8rHlZw58c7JP+UjHriYcocKrGs2khVZn9uf3Mbt15u1J+vJ7QglN7Le/NT6E/0XdNXRIb92vNXSu6XMPyL4US/KlxPS5VAGWZqb4qkVjh7mW8falyCMbUzJXhKMA5dHJAZydBIpLSbCt3WUil8avupeMWLcpbx8dsfI9FIsG3QoFYJYiD+I/wZ8dUIzq09x+S97vw9bS9B+aEM1cI7re+g1WgFB/odCXZVdpwedRql6zY2OD2b6vG/swtvXsA+0J7gycEMeHMAZSllHEy+TWbgDsYGjcXI3AipTCoGVLq963G/eYR2oMG+8/sOS3dLFt9ajEYD1mWP8Us6jNJrOh59PcQCmHmIDVcHX6XGrgaJ7WfM9PNEq9XSdWJXnpx9QvqRdFRhKmpta1F5C8HN9MPTsXCxEJ2gwMxAZv01iwzPDNyWutFc1cwvEb/Qa0Uvxvw4hqaKJnKv5OIS4YJ9oIB6LUspo67UB4zMeHQym2MPUxnw5gDcerrh1tONI69eQyVXEZkUiTJXBR2yDl9/+SWfvP02cRoNff7hi85WtnEbGHv+VzJVmaxatprx28eLaFNdl31pYimliaXYBdph42NjcIz3tHrqiwf7HnBoxiEm751M2PQwPPp6sEG9Aa1WS1tdGwEjA7D2sqalugWZsYyA5wIwsTVBIpEISf+O39/1j64TMjWEsJlCUce8phCPh+doDhiA+/wgliYKifH6tnpO1ghIodm99FgVO387anNqaW9oJzEhkcdmj4n5IEakCDazN8PGz4ba87U02DWwbeA21rmuE7+/6MYitGotRXeKOP/aecwczQyoTYLGBlGSWIJWqxWKL8llYpe9DtEjkUi49c0tErYk8Frla5jZm2Fqb8qawjV89V49uQ9a8Eo9jbJZhX2gvUiLqgvwSgMGUBowgEldBdRxS3ULvVf0prKPhtK6Uuwr7XEuUaAJ0FJaWvZv6SSdVKrofx403SSE3wnnSfATJDIJedfz6LWsF0983gIgINONQ3MOMfSToVh7WhtcF4BEraTwZjE/TtxJ/6H9iRsYx9GKz+nLanySj+JQlELDB2vYFr2N7vO7E7spluMOg2gwFoJqjVbDieUneLDnAa8Wv4pWayQgSNqN6JrhRbW8Ejt/fRf1P4Molbad2rxWKtIqMA7V7yfBk4JxjXTFvos9Wq22QxNXv8H2i+9H0l9JjFs5TnxtRNMurrOYmCsxFFg+eialoKJdQZXJduRmMwifEy4GWzrERLV7KEp7ZxrLGon/Mh4bHxtaa9sYeXYEF2L3cmBmIn5lr7L6BQe6jA7AyMIIO7k7nhUv4GzmgUveRiSyGlL+TsHC1YKA5wKIeikKhZmCgMgAhu4VAsO2hjb+HPMnEQsjmLB9AtGe0dwsuEmGYwa2vrYiyndh0kuALRKtFLtqO1x3upIbnotTqBPqdjX1D+pRKBUkD0zGs58nmacyBQ1hqQQfG19kagvsqkzRnpZSM6DdQCNTNyaBmYFMOzCNevN8mCwU/2clvIrHVhlINMReisXniZoea2Nw7wBRdk6E5vjlEPtmLFbGwhoT+UIkp1edRqlQsn7aekztTDm2RNBiDBoXJCJy2k2sKPXrh0c/D4O1SPeY5e3NNNx8QkW8sF6P2TIGS1dLrL7dS31Xdx5HzSIkyJUHew5RfK+Ywe8OpsuWZrqwhEvDc/FO20Jr1VxAvxf9c/6pNVqaq1pormjGwsOCmgoh2f/t2G9JP21YoPKpXoxzkYBMi1xfgEcPD8ydzAkcE0h9QT2excloWr2wrbGkqbyJ6Yemi40JDUohGXR04lFGPPoacydzhn0yTDx2o0zQZh1wfQAy4wdknsrEBGNK+ddNIy7AZmAccJxn6yHdBkbIQD7AiDOfnCFidAQ9R/Uk/qt4ci/nIpkxDV0olnM5h93DdjPm5zEsvi3c5w+dGN7afQKJeV9Nt+cFOqvUP4U1IL5fPI12vswxN+H6a9cxtTMlfHZ4B/2eLtOgpeZxBQV/pRM8WfBJCu3+AODlb1+mxfoI9qEuuIY5IJFKOtEs6kwjNO505EM6zz/7SntqcmoInR5K6PRQ4lLzcLo8mlyfU3zQbT0qxatM+mOSiILRaPQ+vEqjxjncme7zu4v62LpTd7m5k0vjamivbkSr0Qr01lefY+NlXYe+BrVai3uqBwclyn9Lk+qGVom2J5Sbl9FWaMtzBxcT2N1U7JQ/+PAgWxK2MKRxCLI8Gep2tejrdNZEA3Do4K2USWRYGlnS0C7MlVrzO2RVZxFgp2fnuNLyPSp5HYWeNYRIgwx0v+ra6rgSIiRh900/xItuywBExOLXwJvAEOAVhHlXCuwCNiHMQSeg3bKdm0NaCOlUt8itzWXzjc0ABD4OpPRmKc7dnQmeHIzPYB80p4TB7jG/B+NXjCd4UjCxH8ZiZCn48/vTBOjprcJbAKxI1Yva6tZs04ZybEofYWwVTcz7MSJ1lG59884NxVvpRMqeFJrKmui3VojVJB1UdVMOjKPdIY43694Uj61bgyQSKSPPjsTYWo9iA33xUKKVYP3AmqLbRaTtT8MuwA7XHq5Ym1jjYuFCvjofVaUKv4V+BIwKwC7AjkDfQGJ8YsTrK7pTRG1eLcGTgg0QHVfG2LNpQhQ/h/3MwHcGErspVpybDgWJ2BU9QKboLtLLuUS48MHBULKNDyJtuotTdQ8WdKabVut/M8PPDcdUlcGCmwt4+PAharWaYwuOkaPM4XbwbSYenkh+Rj7j/xzPBs0GHSBaTBb1N+tPQ14Djw48IuN4BmF/x2LT3ItuBV/T1HUrX37zJRO6TMB+iNBYpdHpnUo0qNUQMDoAax9rcawlWhmKdgUWdRosPG15vfp1keqyQvWERlNhLS7ybMbY3gWLmnLkJnKMbc1otnZFLcIXtAz8aiLOHfU03ToR4hhCEEFUSaoY9O4gTG1NufbBNbo96EZieCJHJh7htaWviQ2bnb8L4H/On+DsYPZO2EvA6ADQK1QAELMxBvcodw7POywe40u3LxnywRCR5hJ4Cv3j0c+DuefnYmJjwqE5hwwQeADmNUU8PFBH8JRg8q7lcf6183hGe2LlLuy1f437C4lUwoq0FRx94SiOwY5Er9PHquN/G4+5kzm5V3N5cu4JcR/H0Z/+7Jq7i0q/GVh5WLFv8j4aSxtZk79G/F7EwggDLW+dff01KDNz8Xx0nj9/8kLyQGhS1mq0HBn2I33qWyk3d+ZOn0z6+ug3D61MjmWtioi2CK5PuU5oznuMap+IS+olRkUJOnkH/zqIXCWnwLOAGI8YAD6QfYDxEGPeG/geY4PGMj1U0D+0cLWgqbxJXJeVaiUPKx4S7hz+VEKzuaqZvvf7c092kEYbM9RqQcJEq9ESPClY1Iw0bzLHvEGNibUJ3ed379Q0KBxvx8IdDE7Loux2LqV3C5CqYpGp27GtccK9cjJ3u/+AiZmgfbt/yn6iX4sWmlOsPdk9aTfVX1Vz5JsjtNa2kh+Xj31v4behUqios6kDc7j1wS3a6tvo8UIPwpb24/sEcMi/j0/KceQLR2PxtgUlTUKBTKXWAFIULfW0l5ehUWkMdJQBTFtMsazKpr0qjGXJy8TXnd9fzq7k/ai0rTzUdsXX2LBZpV0uFDasmrsj0RhhE+jI431J7Bi4g4m7JqK2EdY/tyI3LDItqIuoEwvCgEjjCQJrRUtNC+m/p/Oy3cssYQmpbqks+2UZtmG2/DX+L6RyKdMP6bUtZx6fiUQqIet0FofnHmb64el0nain+T846yCO3RwJnR6KRi0ghO/+cBeNUvNMKZeShBIuvX0Jl0gXrLytePjwISEhIcg61o38G/mo29UGunWdLWpVFBnHMrB0NUQ2BYwMEIpvXtacWH4ChZmCkV+OfOr7zc2lVF1eTWHrTZrkkJVkxcRVE+i2sBtnVWepa6vjnOQcayavwdjKmP2T99NQ0sCa/DVM2jeJqpYqsACtq5bA/oEEjupg/sh66lTIjGT65rYOC54ULDZV93+jP6oWQ1+ix+IeYmzQWd7Avbc7L2W9JP7/n/TDZvWlaEuKxfOKes1akCnb8Eo7Q5lvH9obXWmpacHc0Ry5iZw8y7+oMIsj2/kLBqYl4BrhhbG1Cbc7eoHsGgbTaBSPf8o16n3CeaP2DRSmCjKOZ9D4SyOaKRpuF91m1opZRLpEolApCJkagnO4M7/d/40WZQszQmcw6B2hWf3owqO01XfQ9kr17EPF7sVolBoqHwk00wqVLVGP9uOdLaVoYhEyIxm1ubUEjg4UJSru9r6HoywOaz8jHDstkbrYQtGuwK7aDupVaDsa2oSYR9hE1bJG2hRlxHyyHgcH2D4OGo0fk+8osEP1vNeT9MPpdJ8mxM264mrFwwom/j6Rnkt7YulmaYAO/q+sM7LvwR8PKEksYeBbAxnw5gCkCim3t9zBrdgNp+IGWisayLmUQ2lyKb2W9RL3QIcKB5wLW9HY1nH5y8sMWD8A3yG+hOdtRSmr45ZvALf63GJGiCE6fcyfAuij+/PdmfzyZP6VZZ7KZP/k/YzfNp7VWavx+NWDVpNWlqqX4vJvK5P/x/6VqdXqp9a8/5v2n2Lff+x/jKk7IkvH+uHkmTxh1MV3OV50nNiPYun3aj8kUom+2CeRkNVzGi+ss2LHoB0Ejg7Ea4CX+L5lRTYORSmolSPE4M7IwgjL9mE0mx9j/p4PkUqO8vzd58VkkbXCAafa0bQYFZA0IIxIj2AWfi049elHhEAoLDWMg1MPUm9dA0ZGWLoZMfvUbOK/iqfvq31J9U3ldt/bfLblMxy9HanOqmbMz2O4rrgOjwVk3/Elx3nw5wO6z9MnaAGMW41xKnehpbKJnTF7aKlq4eWcl8Xgs94pkHqnQKRyNal7U7H1t8W9t55ezePROZxz71KdvZID0w8QOjOU1L9SmeUay6652aSEJ+FfOoyuk0NxjxIcla39t5LX/ojCoYUcmJHM8976JH7ulVwASiaWoEhVEH8onqpuVdgH2RvQQta21uJU6UTEpghuVd2i75q+vJjwIjsOWqEpThLGts6a+vh6Mk9mUt+9Hu+B3jSZNyHVSLkf+QB/7zkUxBfQUNxAyJQQSpoK0ErUeOXZ4JFxk4pkP36Yc5qoVVFErYrCpN0Npbyam/1vYmd8AyNr2DV0FzmXhG6YK8OvUOhRyMy/ZqJQlxNwMIBX8l+hOrOaw3MOA/Aw+iEtTS1oVBrel7yPT4wPtv62lNwvobWmlabyJoytjRm8YbBIrSHJFu65ydaTJluwD4TnD+h57BsWvkRZiTk2JY8o/+o+CjMFymahc+bu5lvgCr3u9WLMqdHUuOdS7W3N/sn7Rc0Iy0Yz8r3zMZNJxOcukUrYOXgn/Yr7Ie8l5+TYk9wI6Y9EIiRH4j6NozC+kOcPPo9UJuXOD3dQNivp/3p/avNqObn8JOFzwgmbJRSDpv09zYC2Y+Dbgzg55iJNxg9oHr0YExsTarJraCxtFKitOm0jL7wAx44JCWV9Yk9ALxUFxeBtb88Lu18A4H3J+zQ9rMUtxI0auxoy3DcAx9GqtXzp+iXdnu/GBs0GcmtzmVAxAScjJyQSiRhwfGzxMRubNpIYkUhyeDKjvAQKHYWZgpj3Y0QHrTK9kgPPH2DkNyOxf9mejOMZnF93HpP+i2gyMsO4rpzEbYl0n99dpNNJ+eYKRf613Ot5lNVBbwNQWlr6bxWBTqiUDJCe55rVdTYGbeR1b0E/z7xT8dV7/mC2nXTFfe23dJva9ZnBia2vLd6TvA2S8BKpBAkSUWcKhM7ivKt5DN44mP1T9tNQ1MCqDL3W6Mr0lQbPU6pWYtJcjabFkO6zc2LlxZ4vin/PPT+XAzMOkLYvjX3P7aNnQk+mhwtB4eMTj/lr3F8A1FnVUWtTiy22Bsc1sjCiPLWchpIGGssaKb5bjEcfD4wsjJBIJEzeM5kfg3/kwhsXBM2+TsmxzvSUoTNCcQ5zRmGmoCyljKYKQXidykrqHcNJjX2JTYZyk08hgDUauP/TXaoyqui9ojfGJVo8Kz2JuhNFWKoLmrAGNix4lxj5v6+TdDNBQvS1aJLMk3h44CFoodeyXpibC1GAU7kTDy4/wHeoL2YOZgSNDTJIdgff2Maxc+WEzQrjL40wlu0aIeJqsvXA1FiglLFwsRCbO3RFt7HHx2JplIPd5O549PVAIpPQ2N5Arfkdquyr+OyNXUzMe4e6/DqMrY0BEzEgMm80JzQ1lJbQfDJP1nFq5SnGntDr4XT5owszQmfwhfUX/NDlB2TGMjRL9HPk+sDrTF01ldyrudgF2GHlboWTqjfGamN63+tNTbciQJ80M5EK60HX9K6MPVRH1bQaJu3W6yCI+5ijP61G/ngPVPFyzsuY2JpwatUZIu970mw+nfNDt2PfMBjH8IWY9xP2KQ/jbnTP+w1/60rMz39FXbe+vNX0FlK5lCsbrxD3SRx3Nt+hpbSFPyb/gZulG3JjOeO3jxd/89899x29fu2FlcYKZYuS0OmhOIY4IrEwp8LyAjlO36OmjJz1OdTm1fKp7ae01grPaUDvAdyZJtDSxG6KJXZTLAD2xi54VyzHvPwMPW8bUXC/Aq9IvTawDvlS6VDJ9QHXmdw0kBPLTtBzaU9Udq5YN/egzvwehyYfIq1PBAvVQgMI6H+vXR26ErcwDgsjPT1P1MooVhuvJrEgkbfa36K5shmZkQz3KHdmHptJYqKWBL/nkaut6G62DfcoFTU5NZjammJiYyL+boybqqnYcoiBbw8kalUUedfyuL75OvejVTwJOI139ftIreRM3DkRZYthB6hRcyF2pd7UJuex9dMb9H+zf4emiHBw61prQUOsvIHi6w85t/Yc86/NJ9RGnyiKy4+ji0MX9k7YS86lHDQj9RqLntGeop8x48gM3pe8z1CGUuZUhnO5M1WvVhk0eHVe3zRPFbLArWUEBeZHGXRtMFKPFAa+PZCrH11hp/wms/8LKk9denYA0E8CS7UY0qMplbQPVePia0fhD4U0aZvoOaon5WnlPDn/hMCZ+mNZuFgQNjvsKc0+nT0oc+aVDXpUyrDjw5hxZAaFnoXYNQj3Ou3ANNGn1WiEbuVjE47R48kcxj0qI+6tS9gH2WMf5ECal5BESuiZgFvDGCad6U/6Txf51O5TNO/o0boeBR64Fqipyanlr1G76bmsJxoPYQzHBY2jzw99uJp4lYA4IUnlbu5NVNZJkr1fIJ/t1LTWGLAHaNTCHPbJ8UHZcB+/pQMNmBN099xo60FQF1tePKnXhVLWt2FbbUvX9K50exRC+3NtfNr+KW7fuDDu3Xc43tb2L4uu44yMce82nzyLXzFpliNVt2MX5o5dJ3CXDjFxOfYyXV7rgpW7Ffk38oXfRadi31avrVTuqMSopxGOXzqSsDqRCQcOUWT1IfZV9sSlxZF3Og/Hbo6Ezw4X51+dazTy8JkkbL3FtQ+uMe/SPIyD9InerIBsZHZuHJ53GNscDz6UZVKlVnONpwuZs3X3BVhKoCkMtBI1doUp7Bx8n7G/jkUr10+giUcm8jD3IdFrozG2Es6puy5pqJTImMhnjJxgo46MYueFnSy4skB8TbdFFoaMoDBkBBv7YFCs6at5FauCmQy8YIRNYSqJRY2UpZSJSHCJVvAdT4w9TY+qFwzOl85hUrzPUC97wpGJR5ho8R2PTzym5H4J/V7t16n5sp1rv11jsmSyqNn34j3Bf5FJZeT45eC5y5PWQ610ndgV+yB7/k77m+cPPM/anWuJiojCyNKIlN0pvNUs7BkOTYOpNL9Ko60T9t09GLxxMN4DvQ3uudS/P+3B3ZEqDAmSLvr2pdpMwPr6l7zJw+23uPconyl7p4gaYSD4+Sr7x0AUarWanIs5pPyRQruiHd4Gl1IXWhQCmqZzUvHg8wdpVjaTsyCHPZ/tYVnyMoZuHsqlZOH9NkU5N03epefXq7loe1H0l2vVQkK4952elJimM+GN3uK1tKtUtMur8cv2Y/JeS/KiHuP3SucGIf0cCip5D8Vzniz8uicXLsCFDEce9+1Cg80RbGpsCMxywnSaFR59hGKYSDMmkZJxLIOTy08y98Jc/Ib6ETQuiAd/PsC4zZj6iHqDuQOGa3aztpmgcUFsUG/4lwlWYytjnMOcMbExoaW6hcaSRgP692eZuaM5fsP8KE8t58GeB9gF2BkU+5xybvH3tFTeVb5L2MwwvPp7iQhOgN4re4vXk/pXKr5DfA2KfbrCx/VPrhP3cRye/T3JvVuKT/1BFFqh6W3ox0Of2j917CvZF7OFBpAtY+i1tBcJCaBwD6PRzptAG0deebCc+sJ6nLs78/B4Fu31rWT7ZZMeWk+USSBbIrZgpvEC79F45LXjpQzj4rCLNFqqoU6BzFiGRCahpboF9xR3Hjk+4o+5f/DeXKHxMHhKMIk2iQCceHxCvL7rH14nda8wLhKphEXHFvFHyh98OeJL1vZba3Av3/p+i6qhndjAWKwa7HkSkE7KN5fQqDQETwrGSGKOdVNPEvuaoHB6E8cQoWFYtA5kX41dDW1m5uQdTaL8RhZ+TjXUOgdR6RVJuatQddGgQSIRUFRibkdmxJzwOWx9uBWJg4TW2lYuvnmRiDURAMiVco6PPE6AdwCWv1t2UN5DTVYVFlWNNNh5URM7ia4TfcmozKDusADnqmuvAewp7DaSHutHUpVRiNtDN9Kd0hl6YShRd6L4Y84ffP7Gr2ztqm/kBFC4O2F/V4jTml1AVV/LhfX3CBob1EF5LvxuZuxbiUwSh6aHsPBUPa5C3a4W9ymPQg9sTttQs6SGikcVSCQS0n3SWXB0AQCe+Z5k/paJ8yhnzq45i9ViK/CAfLN8xpeMZ2P2RgJUhpIxAPaBHYXQVhXDPh2GYzdHg/czjmbQVtdGxtEMmsqbeDn7ZV4tfVXUNv2nufZ0ZfaZ2SKTjfof8dWOATsAw4bXzmZqZyqu6wbH7eEqFtZyLuZgYq1v+tYxOzwJyeT0iU0M0kr5BCUuaigtLGbXb7l89fMXjFoxBuwFzch7Pe6xxH0JfV7pg7JJ+E3q1iHTZlNURSqxwA36uMWouYaPTH+i79q+DPlgCMomJTJjGXLjp9Ps/0p78L+zpvImGssasQuwE8c5K2oWvTuW8lknOhX3taAyMiO93wLsfKxI3p3MqRWnxLVXFwe+sO0FnCrPEfnjm8hk8OcCkCDBvDWQGvNrXBwjxY+B4rjaB9ljOseUSvNKShtLmX5gOj+M+oGVUSuZ9reg0z7yy5GUNpbSomrB3dKdIb5DRG3Ps2dBI9HnHmQqGdNvTcfFTojzU71XoNTcZvS+V0j725K2ujbub73P61Wvo+pAsCrUNki1RmjUWhpLm1CYKTC2MhbvybHCkRe3vkjz81JRY7iyErQdrDYjz4yk3y1b6pfU4eAgFMibTDLIdvkcXxtfpsZN5ciJIyQNSSLm/Rh8h/gKjdYd9K663Et5ajktNS2ibwBCTqTwdiFdJ3Ql62wWIVNCkEr1+8X47eNRt6u5/sl1GksacQx2RKaRopFq2PLSXXYM2kDa3ydI2JJA97ndRTaAiKQIBtxoQRshZ2niUoN8UnzQEOrNKzkz6gxro05TnlZO3rU8uozrIn6mpqVG/PviWxdZe3UtX434SnzN1s+W3it74xzujI2fDS1mgu+RWZ2Ji+X/erEvOxs2bYI5c2Do0P/+8/9/sH+uef837T80nv+x/xFWcr+EwtOFtDe1o/OsXYqk3N96H1s/oaglIA/036l1DcGjrwdT906l/+sC/Y7u/YLQUSQNX4eyScnP4T8Tt1mgCOr95Chj72mxqfdH29KCsZWxSH9oIbcBJDSYPaDGPN4AndF1YlcexK7mzvAxyFQyrGtMaa9tFt/3jfXFc4MnNbbCgptYIjjP1z++zm9Rv4ldPldyr/DY6zFd53Vl3G/jRFQfgHuROy9sn0fZlUd0mdCFbtOf7TCoWlUcnHmQhF8SDF5vsvGg0jMCYxtTpu6fSp+X+2DlaYVriT0RSRE8cd1MvVkSIc+Hiqid4pvF2N+zptW0lWL3atBoOPPKGZ6cf8KwzcN4teRVmkyasGywZO4fc0ncnvjMa2o2ayZrSBbuUe4YWxoLVE8W5tg2CR3a8dHxhNwPIWV3CscWHSPh1wTe+egdXEtcOT3mEubh/sR/Ec/f0/4G9M5XfL94EoZOxNjBCplCJsLp+2VcwbfsZbwqXqRNXoFGA2aOZlh5WDHut3Hc6XeHKocqrsRcIb27JQpTBdae1pg5muHcyxmPjz14Pet1Plz0oehI5l7JZdr+abynfQ/7IHu+cP6C7PPZz7zff2XmbtZoZXKMWutpfZzHvIvzeLPuTSIWRWDlb03g40BUchWPgguQWZmjUWlESlUQqEEEU6PRaMm5lEPFowr6ru1Lu2s7qaGp2NTY4FCmEmk8S+6VkHs5V+xOT9qZJGqLtdW3kXc1T6RNBfAe6C1q7lVlVpF+NB33QnssGi34q+pVAG59c4vt/bfTUtUiIvtcM69hVyBkHMb8NIYhHwj0VbpiX0nQYLQOhgEDCMnJvvF9CUmxR11VK3R7TgmmpbqFiocV+Nr6kv92Prt8dxkgi3SOeFZAFocnH8a+jxCcGFsKBVhd551jsCNT908VNbiCxgYxZe8U2syFz9cF9uLNujfxjPbkzJozfO31Nb0/GIva4S0CG9N5bt5cAH7bupUhCsV/iRwAISHXT6ql5W4Lb1x4w+C9SX9MYtbJWRysiiGz1JJGtalBIW7X0F28L3mfyxsu49rTlYgNEbj1EoqWzVXN5F7NpbygnJKKEhpLG1G3q3Ho6sCqjFX0WtoL5+7OBhQfIFCZ6LqQARocfEkevg6LfmFUP6km/qt4Mk9l8q3ptyL1zJ6he8TPN5Y2krYvjaauTeR753N48mECxwlj2XluXh5yGbtqOya8OYHM03pKoscnHrM9eruANGpScnH9RSozhC4+rUZLXYFQiDK1M8XY0lhMQv7TfAb70HtFbxSmCq5tusbuYbu598s9JEePYtxcC0BNjUDLdGD6AbIvZHfSM63DsjKbtrpWxm4Zy6xTsyh7UEboZwpe2P4Cyd2TuTmwEk1dA+fPHmFe27+vk2TyUMPelXsJmRZi0H2v05aNGxDHa/XvkHUqi73jBYqVzntHrXMXwpb2Y/KeyWR0zQBArRXmdoVPbyoGTcbCxYIld5bo97GOoMazwBPjvEai10Uz99xcFKYKnjQ85KHnWpCAVqpGVl/LN97fcOOzGwD4lr9ERM7vBObNZdSZUWgyivAZ4sP4beOxDtJ3BpfUllBdU41GpcF/pD9+w/0YZDubbnnfA1DoWYjCVcHvMb/z6KBA+2ustcG2bRH75ykJWDyYjOMZPDz4EABvK3/cq+ZQ4VhBRqgCG19DDRqNBlTSBrRo0WqFDlYbHxtMrE0I66C0katlBD4OxLmw6SmqSRA6W9vNrNEoTFCYKZAZyfDs70n4ynCuVF0h/lE8O6bu4MbWG8iMZEQujMR7oDe5tbmkVaQh1UhZsH4BxxYdw62XG5ELBU2YYrt9NJlm0GraSkNwA/FfxIuFPhBooJb/spz4r+MN7kmpBNvGvrTa9OfaKF8kMgnf+X8nUh3pkqhlLmVcHHYRFDISfkmgNqeW1G23WfP5WMIeL0ctV6OWa0j86goX1l8QvtsxAM7mzhwcdJBNsk3c/v62SF+s0WowbjMmNTaVi29dZMyPerRYUWMBJXYHKHDcjrVDMxVpFXzn9x13f75rMJ6t5vbYr5xO9/ndsXCx4MCMAxTFF9Hjppw0v4+5HTQStVpALeuSQ+afv8euZY1cGH6BPS82obC1oKmiSdyP+qffIqTgK/yy/Zj11ywqU0rw6OPBoHcHIXWS8lGyQCXkXOpMa3wrWq0Wt95uBIwKQKVRopG0P5PiZ8Q3oyh1LuVS7CX+mr4f+64O1OTUiGuNzmfwzfbFISeT6ifV7B6xmwd/ChTH7s1Cs8jh55/QPGQMvZb14o3P3uSaVv3f0nRGAxqg0MWHt3FgnoWrSI92LyMRooWC7g8rf8B8ihCkT9g2gXda30FmpE8GOYY4MfmPybj2dCVucxy5V3MN5rniH3k0x36OFHoW0j2pO2OOhNNe3YhbTzdcugvB+SXNRqotdRRyGtwG+rPk3hJ8Y30N1qBzI89R6hdB5uEH3P3hLi7dXXAyCaB35gnsJH4MvjqY6EtqtFpQmAu/K914jgsaR8zrMfRY1IMn559w6Z1LtFQJvq99hYKxx8eiSTd8Xrpmh7AHYagPXRMbnvTvC/8WBQ/jrtsEg/fKNqXw8ncvE/Q4CIdyc1RtamQKGWtfW8f6jz9mgETGYJmMPcAF4A9gmNyIQTIZK978mLwJv6JQKnj989fxSj3NtZX7hD33vcuo2lQG9Gi6e9w9fDcXXr9Anbnep26Na+XC6xeoelxFzuUcqvMbaDLJomt6V+bvmk97Zjtxn8SRfijd4FggQa0WigzO4c6Cfk+nogYSYf2zC7BjcGQMmWr1v0SN0vH6cSBPC92U3eh7xQR5ezMVDytQtarEY5vITRj+/XBi34vlyPwj7Ju0j5L7JQbFGBB0qUoSS8Tfq472baj7UJTNSlL3popNc8/SCupsgdqxeFe+SLVXLPLpkxn761jmX54PQHrVI651E9b1lPBkGh18SD+SLuqCFklvke/4KyAgl9t9rMg4nsGV964Y3BcINHaPTzwGhKTyxeyLvHH+DS7lXBKeVUYrCb8ksGeU4Nfk1wnyDnXmdZg5mhG1Koqp+6aKqAiJRvhNStTtSE0UxLwXI0ou6O65xcqZRmd/ap7U8JnDZ8R9GkfyrmRe+mwUI87q0GNaqh6UCLT4WrCQ2eFeJbAmfL/6e0p6D+HG5htU3a+iobgBYytj/pgjoG23LN+C5w5PWmpayL2aS0OxsK739ehLrG8sEbMjiFodhaWbJQ5dHATNPtTonJDEgYlUpleK+8XONqF4NPz8cAqO3jd4ThUtJRQ4/kaNbQMl/v0xd7ehNKlUpGAXJSqqbZn4ezG5u46h1cK330JaGlztFsq9gEl453kz6mQ4Vaml4rF1300uS+ac2TlcYl04v+48ZQ/K8Inxof+f/cnzzhOpCI+/eJzDcw8bfBeg+UEzJfdLhIa3Z1BhnX/9PLuH78bExoT4r+JRmCoEqu3p3Ti+9DjJu5J5cu4Jf475U6Tn62yOIY683fo2A98eaPB6uU8UU/ZOAYnQjKGLYXUWtSqKqJUC7egbNW8w/fB0nmW9lvZiVcYqHH925MR0P5xLrfBNvU3x40b8hvkZJGY7m7mjOcFTgrH11TfRKU0sabJ1RyM3Qq1UU5JYQmttKyOOLmLjxo3U2NXgWDdKKExrhRkhUauwq1RT6lKKUZsR5g0q1B7eLE9Zjt9QP2x8bLi86jJJEUmAnqln2v5ppA/VI+y/+krIo+h8GF3R448UYd5+cPWDp+4hbFYY7TYa0rqlYdKioLZSxZifxjD2F6G5LNAikoGP7tH7yeGn9jjBhHOYN5ojb6+nqVDIo1hVZGFX8hCnnNuAlNiLsYz5C6y9rVmespzIhZHE5cfx092fuF14myV3lrA6czVWHlaY2JqI88ul1IXbg25TcbAC53BnMf79e9APdI3fidLEimb/cCzdLMV1EqBNrfcBtVq4/O5l+m0XGhkGxg3EuN0Ys2Yz6PBpsy9mk3ddYBBoVtfTqihGKRMKh8rKem5sviHOTZ1vaFVTj1l9OcHzo1gYt5B1Zetw6OIgIptTQ1MZcFZgpTm75ixn15xlZ/JOaltrAaG57tGGRyjMFQT9GcT7Fu8bjGx1SzWzT81m5rGZBq/rKPidujnR//X+on+ns/WN65l1chY9X+xJ3zV9kUglWDhb/Mv47eKbF9k7Ya+IGP+nTT8yXSwWPcuaq5ppLGt8StKgs61+vJold/U5s9q8Wvac2c3333xMnFbNRZQGkgxnW1u5plZzfctJUZJBt/+HzQwTkXa6sR5xbgTK6Ury4/IFStKcGnE/0MgU+A71xS7AjurMajZbb+bah3r63qrMKm59e+tfUv42ljaSfSGbsgdlbInYIsYSzZXNJGxNoDSplPu/3WdL+BaqMqqeamT9p2m1Apq30d4bYxdbnMOd6fNKHxH9qZtf2X7ZVLg5odEY7ucyjSlaqZYSj1ZUJhaU3C+hpboF5zBnlHeU9E0Scnf94/rTtKPJ4Ny6tfydS+/w5btfcvxlveipVgtqmfD5HtolfPjbhxwf3VkUVUujRSM3h7rSY3EPIl+IZNLuSSjMFR1IWnApdsYx9y41T6r50vVLzq07B4AjITjUD6PBsoFLQy6hDNDPNaUSUcKizLmMXL9W0v64z+3vBO9eVwh0sXBh9snZDNowiNKkUhGNKJFKBJr32lZqcmpQt6u59M4ldsXuMrj3zNOZXP/wOgXxBVx+5zLlaeUGyD4bbxvsA+1J3pnMne/vcHr1abJ9s0nrlka5zQk0GkFnfmHcQoytjcXGxIchD7k9SIHEyhKXCBcD3cp6c33eVa2G/Ov5nFpxisqMSro5Cnliuyw7fg7/mUeHH1GaVIp1pj7WB3Dp7sLoH0bjFOpEdW41rsWueOd6P7Mx8t+x334TCqzffPO/9fX/2P9h+w+y7z/2P8JS96SS8G0C0TOiCSn4muDCzyjvomHUewpB40Sjxcjc6JkbYOcuPd37aoXQpaLVCEkodbthxf1J7xlEREBNdg2qNhWOwY4dVCcKTFpMCL9dhcI1gaubGvEa4IXvEF+UFna0yhtwrHBk2S8LKa9LoGFIBPFfxxPwXADHgo/RlNGEY7kjDccaaPJvotvz3XAIdqDGqoYwpzAelD9gvcl67n5y14DmB6DarpoLQy8R7L2UwS/pOy109+SYexf7ohRaa6cw7e9pT1EEVnuEU+0Rjqm9vrNo7uVFzJ63iEKPQlZ/t5pWq3Q0Gn0bhtMIJxJTE5FoJEhVajQtDdz+9TbmTub4D/fnzHtncGlxocmuiWPjjrF4ipBM/Mb7G6w8rVgUt4hQp1BSSWXBbwsI8AtArVSjalGhVcuxagnHvj6WKqtLKNVKBm8cjKpVxfnXBN7vOus6nGsnoFYLVK4hzwt89jonvdGykUZrS8x9HFn+YLl43UZqezwq51NkMpW2uiza6k8aULkoNylBA6lhqUhyjWmta6W5ohk7fztcd7sycd9EQHD8bHxs6LGkB0HjhM7zutY6Uo1SiVwWibW3NU/OP6E6s5qeL/ZE0hEIeDw8i31hCqVJc0nYmkDIlBB8Y33FZ1Xu24foNX3w6GCjmrBtAgknHzD79dmcGnWK0+MSWOjlhEMXmH1mNruH7SZvkhDwTDo0iaDMdjSTNOwasYuwWWFM3jOZ6Fej+fj9jxlxdgTR8dE0vteIjbcN0/6eJurnAUw/NF2fKA5z5q2mt/in6TrXEn5JIP7LeOIHxKNUKInZEUnjwkaCJwdj42uDsbWxWOxzybpBo50nYIhI/edv8jP7z2ipbsFvuB/Z57Mpc67DodKBXgm+qLqUIzOyIWRaCAdnHOTn0J8pHVWK1Lgnjj170ljRyo9B3xK5KJINmg3UtdWx8dONwNOaMDozczAz6KRz6OIgJEeE2BSNVI6xlbAVmtqaYulqiUOEBzbbk9BKZahUQrfx0T//5JXWf78I9PCuMXPvLycjMoMu47uQezWXqxuvMuxTgTpOK5OjWvQiQzrFXrokWtr+NAa9N8jgmIXxhfw17i/uzLlDbXUtI06NYOH1hXgN8KI2r5Z9E/dh5mDGqoxV3PnhDrZ+tgSMCqCxpBGFmQITGxOD40kkQmfauVfPMeJLPbVRg0UDzt10eqBafokURA3N080JTw4npXsKGq0GGTL6v94fp1AnDs07hE+uDynhKdS71hskQw7NOiT+bWpvyqB3BomUpu2N7fwc+jMgoOFqcmporW0Vg+0/x/5J8d1i1pXpEcUAPV7sge9QX9z7uJOwJQH/hP2U+0Rx828nQgPaSNufhu9QXzw7qJ5sS9PxSjtD3eMFOMYIXXiV6UIRoMitiKzALJrtqhnh5IjUQYZL5b/1mHEGtG1Q4lGCV38vJDIJkS8I6AiprvNTpkErldF7VW8CxwY+1ZRS3GUIvdcbHlfzjGLGs97/ecXPzJYdNnyvwzmXK+V45DshbWuh75q+QrdwCtg3DoLGQUhVM0iPLqVPT2scg6079jgtyneVfH7jc/Z+vRfvTd48PvKYUd8JhZCWI3C3Yih5Tj/RaPoIY09jRn0/StQ9y5GfxETlhWvDFCRGCq68d4W2ujZCpoR0UApKKHUtJamPEchlHF10lPA54fjG+pLecIczPfow/tRrBKX70lAyB1WrCnNHc7wG+ZA48g2SA2Yxcf9wTNprubD2FBl/P2BNwRra1RIqrK5w0XMWpou6MapgHoW3C7FytyJoTBCXPrnEkrQlbHthG6pzKnI8c+i/RK+/NXjnYPLr8pFIJDzq+YjRg0Z3Gk9D+j61TM2CawswdzLHLsCOblu6UZRdhHmyOeo2NelH06nKqCL6tWjK6muQq61walqImSwKE7sCjK2MxaTyS5bX2FgQTY2FENgb9/bitcrXMLIwIuWkkKhWSwV6ZvsyNbnpj5Bq1Qz7ZJgekSOR4tbbjZKEEs68dAa3nm6cW3uOsXvHUrGggvqJ9fgN19PmCjclrHe21bZ4XPqbnNBAol+LfgpZoTYyxTSyK48OxXHxzYvi66ndUumS3oVpf0dTvfQBzbGCBqKFswUNmgoaTR6JM9UixItXcl8Rv+tm6coFz7XYmdlxeOJhxgetwjNaYAMobygnp1FYAwdeH0jllkq0S7UiFdC6lxeSb7cTgOPvHSNnTw5T/pzCgz8f4DUsiC3LtwAg0ShQWJmxb+JeqrOqWVe6Dk3H/hB1J4qgjDsomyIpvFUojo22g1Kw2qEZrdSGtP3x3Hr9Fm+uWs/onzZzSqP6l4ixkTIpmqEainuU4FAbzaq67/CsSOS5Kc+hdhbOq1KoqHSsBOtnHKTDtFphXW6ubObi+osMWD8ArdZHfP+fidCK+5WMPzoe6zprvPMc0LQrBV9WAjKFjEatwAgQlBGEz5NGjKxNcfMXupKVyn86yxqurhbW6rkX5vLXXinOdW7I5ce5GX2Ocs9ArL1tWJYkUJFp/9IXifq/JvyWLr51kbhP4hgXEwGYYVkvp1dCL7RFAlrfd6gvk/+YLDZF3Iy+Sc8+G8m5lMPj448Z8uEQrD2tDZJOpSVasi/kYO5sjnOYMy13hGTZkYlHkEsjGWtrQX5cPubO5qxZu5bcvFlsTehLQkYeiiZbHE1s8Kzx4dLuTXjHRPP+tlfRSDUU+LujMXGj8PxpAK59cI3oddFiss+pzAnjOGPaYtoY8cUILN0tIUl/XdYzrBm/ZDxuEW70WtaLhgZoyHxAq08uJ8acYLnnclY9XiVq5OiSa055qWiu7iTwjRkim0J5o75pZu2Xq2ny/ovBaTO5LrlGzD05fVT69edZ1geIlkipTZcQWGXM/TF92ZYmOJdJWcJFBzsEM3zRcAMEUU1ODRojYbAvvn8ReYacbiO7kbQjiZXpK3Ho4iAmt7t+1ZXh/sP52OJjvAd64xvrK6LcjJuqMWqpoyTRhDMvnaHX8l6EzQpDowGltJ5KJyOU4b7YB+rRx62qFpTyjm5ziQatso19k/YRsSiCCdsmGBZAEXz+mI0xAnOHjQm2JrYUNxQj0UiwSbGhz0t9CJ4STMjUED6K+4jPbn4GgEurC6p8/fiVpZTx1S2ho33btG389p5A3eUepWdBsWvuh1RlwXN/53E15y+Cb87TX0eny9JohAYTp1AnzJ3MO5guhATqpEOT6PLYhL5Jk5h/VECwO5t40j13u9AsqpCiVau4/M5luiztQvSn0QRND2L9Z3pnQK1RU3K/hN3DdjP6p9H0Xi5AOGpaavCa7YWdqR3NVc1olBpUKgsK7XeR7CvogmZ3ySZ4SjBO3XQ8Z4I/8vv839ky6C7XP75OZXolk3ZNQi0izGsoCh4G0gJ+ifyFEV+OEChXO+Zuu1E7mYGZBDa4cnHDVRStPYTCk7GAqsr1yeX4+PtIV8hI/8WZJXeWYG5kTohjCA8rHvJ2ztv81O8nqj6vorWmFRNrE6ol1ZjZmDHvp3n8uPNHzB3NaW8UkNQKmX7BK3ujjH2SfUzaPQnHYEcDZANA7uVcVK0q0o+kozBTEPlCJN6DvGmrb+P+r/fRqDR4D/Qm51IOfV7Rr+Q5l3LYPXw3o38cTa9lvfinNdl5Ejq9A43Urkaj0gjaW89ADD4LxbO191YsXCyYeXwmJrYmPL/+eYztjfFVrsAl25iKvHDcgiye+t7xF49j62fLgDcHGLDDyGSgVmqQatWgkZFxNINLb1/CwtkCeZTQhGfSYsLEA5FoI++wLHkZq1aBNldLlbMzhR5XCHkYwoSjUlqn5CIogELu1Vxiv4+laWwTKz1X0najDTrcoM4FrsuXYfp0CJ0ZikNXh6fGQacN2tnGbhnLOFeh2Jzc/QFTBnyJ9wD9+2o1qKRNmNUWYNHcTMUjT25svkG36d06mjSFc7yw7QWM1X/jt6IvdaXNHC6LxqKmgBLjFVSaXcdX2hfNPyjSDjw8wLe3vyXEMYQPYj6gn7Yfwz4dht9wP0wUJtxZfIfmwmba7Npw6+WGqk2FRCIRfTQApbwFtdqIhwcfUrCmgLkL5uIf64+9WlgzTOtLab5XzeCXojgbeBaA/dP202bcRrFbMf5ZYbTb1nLs3WOYO5mz5O4SzjV+wYXum3ju/HK6JXVFMmAOqx6vEhlxdIWJ62O649AwhLg3tyJRKlmdKdCX6PYpM0czho4Qcja6OG7lnZXitQ99eSix78Ri62tLeV05dY+F4mLog1BGnxqN9h0tjHrqkbF7+G6RPepZpiu46wpiGrWGqsdVmNiYPEW1CdB1clesva1Rt6mRKJ7+7XSd0PWp1zrb2VfOkvJHCrEfxzJwvb4gf+X9K6TtTWNh3EKxMV9nPtN9uPDqea4p/2s2nlNKNf3Pg7eZHzbXbaiLNKRE1RWvsgKyiAyKpCRRoCR1j3JHoxGK8CpjC2YdF5B1DSUNRCyKMKDyLE0s5ewrZ7F0s+Tc2nNIpBKByabDsi9mc3jOYZ779jnaG9tRtwnnrMuv48SLJ4j9OBbvQd7EfBCDhYt+vTCrK0H7uBEI5P5v9zGxMSFkqi7e0gJa2loEmSIBMSqYrmHucuxl2h99zIPfbtNYWAuMRCNRUmgvFLFU0gZMS3P4ted+Ju+ZTNisMBzMHFjcczEmoSbY/WhH2+M2GksbubrpKoGjA8VcjEKmoEtGF0qzSnk85TEt1S1o7LujkjYCEKgdQ/fnjSiMLyT/Rj5e/b1oU5SjUqi43eMXHLoISE5dXKLzCbzzHPBOPUVz+QJ6r+yNZ39hfR4geR2jx69zopeEa4Ov0aN9Ond/ukv43HCUSn3hLykyiWr3PkSeeES+VAsBfcTxkEgkImuITm/50aFHHF14lIm/T6Q8tZzL715mWfIyeizpIcYPhw+DsTEMemcQfVb3wdTeFK/+Xtj42iDdqZ9vLTUtqFpVLLi2gJPLTpJ+JB37Knu0Ei3+Wf60etZjN9ZOpJXXIfuK3YuxVVuiVShoqWlBYapAbiInxVsYI6s6K0adHkV23VWiX43EPsgelwgXFFnCHqpSqVC3qdGqtcw+NZtvbn0DZ3nK8m/ks3vYbpYiyLtI1v/vacwZP7vm/x/7/8j+g+z7j/2PsMgXIpn410QsnCxoMcojxftFHvpsoLjGlEOzD7HZStCk6JxEDbv4Dfun7jc4jqhF09qISUM5pnamrH68WqRhBEj2XsydgHFUarI4uugoOwYK9ALNyiZqLeIxajciIL0ZoyfpXNlwhewL2eTH5WOX/iVFZqtptGjk2oDrmAR6CLpDX8RTeKuQK7lXAEGfR/2JmprsGgKeC6D/a/0Z220sKctT8LURukdVGhUPDzwk6fckABzrRlFrW0uN9xTabA0h1brgU65swai5FrQQMjVERAT90zoHqxbu1tT4LqDAq4B6q3raTGRceec8f4wUKiGR2yLZ+uJWfHJ9WPXtMMjPZ03hGnqv7E1jWSO3v72N7yNf2kzauN/zvnhOv+F+IoWWjjte1ymYdSaLzdab0SQLnYFSrQLrWmvqr9bjGe1Jt2ndiP0wlsTehcgIYPyRIdTeeoRXfy9Cp4d2jE+HM9GuQN7ehLLt6eS4Ul5NYGYgsZcH0VqvR1lmncli2TfLCHzcwbkuUZF+OJ3vA78n53IOzY+bcSlxwbTZlF53hALEuF/H0WVcF+K/iudjn495/dLr3Jh8A+cwZ5J3JnNq5Sk0ao1Y+FIaW9Jq4UhDSQP3fronCt13np9qNRyafYj3Je/zmf1nmPtYcWTCEbL9sgGtmEjx6OPBvEvzMKnQEpEYQaOFmhZLFzRaCaN/HE33BYbFtfSu6Zx+7rRY3NFRH+nM2ssaG2+bZ84NgG3R2/jG5xtAoHAB6JbWDdsaWzxzPVE2K/GJ8aHfmn4YmRuJ9/xowGLywoWA7tTqU9z+/rZ4nxZVeXS5uQNN5hNaqgV6gOzz2Uy+sZbbI/pwJeYKh6cmI/cRnDqHLnraA5fTLvzWbT2nu8jIzJLg1ssNay9rJBIJNiY2BNYGMv7oeMrihcRmfWE9OwbuIGGrIbL1X5mktUWgZq1rZfCGwSy+vZjmsgZcsuMJvrGNQ2O+A6Cuvv7fZi53BmiV0mTepO8Wl0hoKm9i/5T99DrxPlYVT5763ryL85iwYwKLby+m/EE5jz95TP41oQvdsZsjz333HHes71DsWIzldEssXC1QNitpKm9C2aykLr+OrDNZnH7pNInbElG3qfnK/StOrTolnkPe1oRtUSrK0iq8B3qz6MYiwueEs6xhGRs3buTLdV9Su6xWeHZtahpLG8XvDr04lAlHJpB2IE18LXB0IK1VrUQkR1BvVc/VV64a0OcNfGcggzYMYsSXI+j7Sl9CZ4Ri4SwEK531cRpLGzm75iy/Rf0mvuYS6YJPjA8AVz+4yo8hP9JS3YL/cH96LeuFa6Qr0n5RNNj74JNynIZbD/Eb7scG9QZ6LOkh/t4a7H3ICxuDmYcdqlYB9erQ1QHPbe9x8zkdbaAGjIywc7BF35f+X1sZIDEGaZtA7bFBtUGkX9IFw5b1luRdz8e1hysR8yOEcf1HXbo2p5pji4+xUiIE6eZSGwDsCxKxiztGc1UzcZ/GkX1RQBLrAhOviiW4aaNI3ZfKhfUX0Kg0YnBg3mTO7D1jsCxIY+RXIwkcHUi1+U2ynb6h0vIKGrkRjXZeYK0PaiUSCXKpHHMjc6rtqqkdXCvqDEInHSWtjAlHJnBn2B16r+gtIonumm3ikfsrSFse0FTawHPfPse4rePE70p04usSDQ3FjSTtSKLikZDo/qtkIwD1pneRqNVkX8jmO7/veHjgIW317Txwm0i5zXHOPHeGpD6WWHpYCwhxmYT79ae5HTSCFmklrYoi5E11bOu7jXu/CAhmmwgbcn1yabRoZNM7m+i2sRsNxQ186/st1z66JiI9tFItZ6edpffy3lx86yLf+HxDa22LWOzzKPAgYk4ExXeLcejiQF1eHUb1RjRYNdDvaj8GvDmAlN0pXHjjAhKJhISS+9zuMpwHnouRqpU4dndnaeJSsQFJp5XkUeDBkm1vI81ow8zeDLmxnPB1w0kc8TpNVjD46mBGHIPn9i1k6X0hCNMFvpdzL3NizAkWZS5i/pX5OHZzpL6wHgDXEle+iviKbtO68ejwI+5tEcZDp2OhaDfGuLgIjUrD8M+GP4Ve0f2tK/T5xPgwbNtwQtNC8SzwJMcvH5m1Bfun7BeL9ndV2yi3OYldlR2uBRKUzYYUmDqx+mr7apIjkrFy7dQx3imuTOiZgOJNhSFlm6TTD0choLibypu4/e3tp9AaSiVELoqk36tCp70uoX1t0DUe9O+PU5gz6+vXi4UqY5UjbtXTcS9/DlpaOfeq0C28Zt2rDB47iwHAIBkGiLERJiYMkskwHRoK0dBm0kaTRQuahiaK7hahbFLqkVNqKeaN5qiaOhD390vIOmMo+FKVJawDlY8qWXJvCb1X9DZ4Fv8s9l0YcZ4eiT243+M+P6w5g9zBlm/9vhV9Vl2SfnDcLLrdK0XVpqatvg2NSoO6UwPQqt/34fWo47fia2OQwJVKpOT45ZAXYDgv1kWv48/Jf1LcUMyOxB00tDXQb20/VqavJMc4m1ORZtzstYXNb2xGMlCCU6gT1t7CWqJrVqhyqELj50J5WjnJu5JFtKy+ge0OHo8usHv4bq5/eB0Av9cHcHE02Le+QHjuNlStKnYM3CG+b2HhglkfX+RzrAkd+R2ffP+QvWn76DWhl6hDqJarKQidRKVXJD4TuxPzfgzLU5ejMFOISdTQ1FDsP7OnobiB3it603VCV3pmHRTvf97NeaTapyI3kbM7eTcTDoyk3jSZcudy7vW+h9xJjn2gvdiBLaKjqjSYpueJyLnOzwkgzzsPqZuwnh79808W/jeFPp0t1WrI0aay54Uyg9f/idzTFVJCpoUQNCaIUQGCpotSoURqI8V/pD+xH8WKSWddolOnRTV171QGbRCK77p9zDHvHl1u7aI2t5aqx1Wij1ckuc3V0BAuRLhyU/0DbQ1tet2eTs7wsp+X4ZFwlEm7J4nJ487o3ZU/vor9d0lYulriHOaMVCYldUUqKctSWNljJT2/6cm9n+8ROj0UqUwqjvXyXsuJi42j4osKQRvd1oT2pnYRdf+vLMvha8psj1Lu6Y+Flx07B+8keVeywT17PThB0N4PMLY2ZsGVBUQujMSpmxOb3/6VrMAsamxrqLZXP7WWSrVGxKY+4bnbd5BpLFlybwmxr8ViYmVCO/q10jfbl6bbTdgH2QsNNR0+1S/3fsHuMzucvnCisL6Qw3MP853/d6hUoOnUlNJg3sDzB54nfI6AnlQg+FzG8tmYejqRdy1PQByibxDS7c8mztYM2TRETKLq6KabLJrYO3MvKjspNz68grxN5xsKz7LOpo6sLiUETI3EZ4gPIOjQp61IY7jfcACsnrfinbZ38B7kzf1t93k08xGaUg0BMQH4xPiw4OoCXkwQEpg2JjaMCRTmp9USKzz6evB7zO9kX3iaUWXxncXMOjWLkGkhLLq5qIMyGowsjXi96nVGfz+aiAURvN3yNv7D9by9Zo5mdBnfBTNHMwriC6gvqv+X8+Lmlzf52PxjShL18dTxpcc5MOMAAAXxBSKSXGeW7paYu5jTVt9GXVEds/6cxZpv1pAS/Dd/zcvGzNOB32N/57c+vxl8L21/GjkXc566BgsLcMxPoMfpjyE/j0tvC01BpUml3H3nDBs3bsQ3xxeLmhK0NTUdzw+QSCgOnkq9Y0/KncpJ7a4ElZpb397iwbUaknJtye+ej0quIv/nfHIu56BRaTi54iS2V/WowhSvZSiVMOXPKQZ5E0czgbXFuXYCNXrGuGeY9ilEsEoF9/wn0da8Dr97x6nJriF5VzLlaYIeYZfmhZi1+pEXYEaVeyhOQ4KxGxSKVirFtL6MmKt9sa+yR2G7h6zod2mra+PWt7coSykT17+HFQ956ZuXuPDWBQ7NPkRVRhUyqYze7r2RuEhInpDME48n/NrzV7Z0F5qGZma9xcaNG8k164/b3q/IvZyLraMtm4Zv4r2Y9/T6nfmJ1G39G7eebmz7bhtzwufgNsqfJwFPcCp34vn9o2lOymDElyMY/J4wZrr1r848iTbjOiRyBfaB9pjamupGCYDbXYbTYlSAXbALyhYBWd3W0IZMIsPb2htvU29aalpQK9UEjtJryensjeffoPeo3siMZGhVWiQdqPoW0xbKnMuQW8i5v+0+GcczDL4XMi2EsDlhPD7xmK1RW8m/kW/wfn5cvkFeoa2ujZ9CfuLKxivPfOp2/nYk/JJA4vZEpFIpXbp0QSrVx38JvyaQdfYZAngdpiuq3Nh8w+B1qUyKRCagroruFonISfhfY+OJloEmWY3VRStaqlvYP3U/eyd2MLF0rH9poWmol6kJnx3Oi/dfxD3K/ZkAA0tXSyZsm2BAXe4zxIcF1xb8S01C9yh3xvw8hqBxQbyU9RID3hSq4XYBdsw5O4fQGaF4DfBi8LuDsXCxEOee6+NraP8U5B8uvXOJ2992INW0IFO20uvkJiyunnzqfJ33Vy0a8s4/5tEeAfGtkbSiktchU8lY8LM99g/jiP04FufuzhTcLMDSzZJu07ohQcLv83/H7CczWqpbuPfTPYruFIk+g0Kq4OiEo4RfDif+i3jOrhGqS+qOYp9ca8Gob0dR9biKqxuvAtAmF37zdeaG6HNA9N9SQ1O5NNoI22ABjRY2M0y853z77eLnje41c2rlKVprWlGqtNg26qnptRI1I3bOZNZJHfWp8CBvFtxk+bHl1DXXiZ+ty6+jrb6NkytO4tHPg+jXogVJjjFBRK2MoqYGtm+Hn38GU0dLHLo6YO5ojmsPV0xtTQ2QfUfmHeFb32+xcLag59KeAFjXW9M1vStz/5hL3b1MwT9vV6PVasW5Z9MYhW/5ajTlVXxm9xnXPxH8XV2jjUKpIOhxEOU5mVi5W+Eb64upnSlyqdCAYtZLaAQPmSrsiz1cexhoSpcklrB/yn6qHlfR8+WeXI65zKlRp5AbPd3AUlpayoebNtE7OJggd3d6Bwfz4aZNlJbqMycO+hQeublPHeL/d/asNe//pv0H2fcf+x9hTqFOOIQ4IJVKUcqrKLb/C+eKbsgrS/GJ8RELG5033hZLJ2z9bPnM/jO6TOzChG0TxPfdMq/glJdAa+1r+k4q4L7fDIrt9hGQGYC0IY3IRZE0lQuQ88LmbNoUZbRbSjg205y+xhPZsKYRcydzUvel4vO4EZ/HPdm48TiXhl1kSbAnTt0kvJzzsgDH3iIs2o+CHxEzOAb7IENaBAC5VE6PhB4kJSahzdbSWtNKxPwI+mTqk/VaLVx8+yLqNjUjvtAjckoCB1ESOIhF7obH1BVjbItSsS7PpDy1P/tG/U6PF3vQ1qQm/L49ma59+X3B7/TKnEx9Qb2IesnbK1BEVTg1U+kejL2Do0gJeOpt4ZqG/TQMOnIgSo0SI5kR438bL57/17G/UllUSc6qHCynWuLS3YWIRRGkGdnRUp5Prdk9Iu93ofSbUspDy/GJ8SF8TjhGx7bS53EpToW/0JbvQ2f9J90GOCBuAP2vHaJqnBW3b5XiGe2JWy83En3nUGS/BysXK1LCUxhj9ybxX8cT93EczZXN2CN00kzfOx3bhhqcVjjR/83+GFsZ83jQY5axjJ+W/8SYU2O40HSB5opmhn4ylJuf38SszIzgR8HsTtnNp8M/ZeDbA4l8IVKgEe0Y6zL/aMr8o/EbqubV0ldFqkbd/DNqrqU5qVykEGupbiFzbzpJkUl45nsy+FoEjR5l1BeZ8vjEY25svoFzLkxkIjtWlGJstxSk0HuF0Ol7bPExErclsnj0Bn6L+oB873yMrYTfxN4Je+kyoYsovt1U0YS6XY2VuxXNlc0U3i7EKdRJLAD6DvUVCzye/TyZc3kRcz+ZzqPgR9wado+P/D585vxqtdRTdCb+lojvUF/6rO6DVgtSdTsmjVVo29pYnrqcspQyTO1MKUwtwbJWTaN/IwXelWBmRnNVM3d+uMOAtwZwNPcof9n+RbuikhLbQygsRzHvgtBlHf91PMm/J9PVvis9EntQly04Z6pWFVWZVTRXCgXeqsdVbO+/nb5r+zJw/UBufH6D+C/iMQmeT6ulIxZFGfza8ygzT8wkaIzQ5Rr/2iGyfbNpMm/CrdkHAGsrK0qLn6b/eZaVASqLdra+uJVfnxOoqLwHebPk7hJ+6PKDcJ0KUyTxN8g45kCX8QKFjy7ZDtBY0sij/Y/wH+aP7xBfbH1t6bO6D5XvV1LpWEm/Rf2w87QjeVcyR+YfYeTXI0k/nE5xQjHzL8/H3NEciUxC1EtRuPXUF/5NGivxTzxIS8hITO36irpXCrWCALsAsqqzyKsVAiapQsoLt14gaUeSEKhFJhJzNYaShBIipkdQnlpOaXIpbv3dKL5RjGmL6VNjodMvu/7JdS6/e5mAUQFiZ6LCTMGL91/k6sar1ObVEj4nHI++Hk99t6Ghg4roGfRNktGjKLyppM4xgIHjbAz0bXS/txYrZ1qsnDGyhx2DttJY2sia/DVoNGCsdGHU2TkEpfujXtvOuOdnsPuzzcz+N1Cc243lNIeqeG3jyxx5dESgxb2Wx1uNbyHRCk5d2IMw/v5qB4tvLxbRA52THa6Z17iyMoOqB8VEvxvNj7IfkUuE9cKyKg+rwmSaK6O5+OZF+rzSx0BvIfjxCMyblDxOf8yDPx8QszFGfK/FtIVzI64T0L5QPFe5zUmyXD/GtM2H4ILPMG1zpV3Vk3tb0rjx6Q2mH5mOS3cX5FI5hZ6FFIwowLWHKxffuoipnSn5bjZUWVTQpiij2s4WiwgLtFqtmDzVoMS8yZzoM7fIsdIyZN9z4rlTKu5T6PA7HgUeDDzXhmqshjfr3kRmLERCFlJhP7wy5AqWVpdx61lB9GvROIU6cff7W8zb0Y89s7LJDMrEVC6jx+r+DN2go+cWHrRpsyl9bgdjpK5g+OfD8ejrQcofKdRm1nIp9hJIhEQ/HcGXqb2pgLzRx3XisYwsjATfQCJFKxHQMDFOL9N1ggc2PjaiXuVkJmP6nCmn+59mkPcghn06jP5vCNelo53xfWJDj9Mfk3NyCp4rQjudC0CKaYspTqXGqIsbKEspw8bHhna1hHM9Q2gxzkURFoDKLJzZpkYYWQgXPyVkCtUB1dh9ZscvCb/wzqB38AnwAWDG0RmErA2hyl5PGXTvp3sU3S2i17JeKFXCdVU4VVKy8C0GGLINi78bk4ZySl7ejnO4M8FTgxn87mASzz8gMSKRtG5pXB1yj+e6/E6gazmepbrEsF5jYtB1Sxr6FZG0sw7PaE/sg+y5b/2ewblUKkj5I4XE7Yn0+bIT4sIvB3WMGqlMyu3vbtNU3oS2U7EvcFkg494Zh7pdzZrCNXwf+D0bWzbyIPQBQY+DyOuTS+QivQaZLmFd4lZCY3kY/7Q8i/0UW+1j3q8hyOz3IlVI0Sg1PDn3hGjbQfjhyH7ffbyd1UqzQoJ3gAMTZ81i1+LFDP1lFaWkYNZkhm2VKfTqxZvHhXspqi8CBB2R5VuW09jYCAPh2ofXSD+STrf9+vFoLm8mcVsiDsH6PaFzY4CVno0ZAM/nfTj2+DDpXdPpkXMMgK6TuoqaIrq1IDlKSkPxHPJOprL39SPMODoDz+H6QN+6OJdW82Zit8+hPSuXyxsuU+0RSIH9RXKVwnG1Eg1N5U0kHXqAZ7Qng6KEgo/lJ5Y8t+c5jHoZMXvnbMwczNBW5aCRCQUftVzo5J97fq7+WegSTVowUlsxcP1A+r/WH7mJvONZCW/blmZgUVfEmJ/HYOsvJJzDZg7F+t2hOD8sx6SpClWbmpHfjMQxWPA/EiXbqbK6wrKfl2HRnI90ngmOIYIvpK7uTBcqRStT0PujiYQI+Q9UGhV1bcJiYD7SnAkrJhigFKRafbVVqpaibFLSUNJA5qlM7lXG49Y+nQIHodCq0WioL6xHYabA1M5U9FlPjzpNS+xw5qdXcvfHu/Re0RuVqX5DqAjehKJPEEm/J1FeVPG/1GCkaQeZxA2ThgrS9pfjP8JfPG9CSQIbAzfi6eXJwriFWLhYIDeR05gwHof6YcRHX2DlpJWEhocaHDetQmjs+eyLLViFW9FrqR79pNvHalyDUVnZ4jfUzwCFf172Mq3SIkadGkXkwzZ2HdtFXX4d68rWoVbr77nJvAmp1kwsTNXWQlubBuTQV/kmmEZj4i6lta4VjUqDqZ0pEomEMOcwvhv3HYm/JdJQ1EDyrmTCZoWJhf2cbCkuM1x4/uDz2AXY4dDVAZmRDMktYb/qdbcXiTsSKYwvJP1wOq+WvIpULhUpvZ5E9GPRLFOuL/5dT2mpo/G0dKbBtYtBwxLo5/aVIVfIC45kQUYZOSXN+AzxQaVRo5Q14mgvw/PPnWjMLHHt8YJYbGtRtYjHGX1qNJU3KrFeYk3Uqijx9bXnBD202IuxXLx7kfAJ4TiFOpGjMkSgT98ynZ/++IkVaSsAkGrlIAGn2hFolFrmnJkjrtU6JIZTmRXuSZuoSZ/B4Hf0zBL/ZBpoH2jL3Jcn8sZndgb37F/yOlbNkfTaOIZAw5qDoLf9qCt5u/PI35mP1wAvqjKE/cm2xhbfD3wZ4T+Cf5quyGwyxoShs4fiNdBLRFD+8/jqdjUP/35Il/FdxAYkiUQirofPMucwZ6Yfnk7J/RJ+7fnrU8ghj4fn+MI5hdWZq3EOc6b7vO4GCKLa7FqxuL29/3aCJwXz/EE9Em/GEQG9c3nDZa5tuoYXQtG2yr4KjfFDNBIZNr42opSHztaVrUMilVCeWs69LfcInxsuaF0bQauFPZXu4ViZW9BjSQ/ub71PwKgA4oftBqDBsoHbYxfTPcCRuz/fRZFnC5IAAu7uxcPIjG9fLuZ+v1YC8to5+8pZMnvZopEb0SXZizTXNPpd68eA4AFoVBru/XwP8wHmIBCSkO/0C0rVj5xadZa8q3kio85A74EcenQIq+oYDh6ExYv193J47mE2/rGRm/1uojLpQpNfBb+8fAipQsqSO0u4V36dSuvzJPR0QWMxlvUxvqxvXC+yGwW0zMLsySzUzlDuDI+3nKDgcAJWUbNosHXl9HOnqXCsoGu14DdXPq7i7CtnGf3jaDQ++rk76vQocgpzeP7Q8zh01WejT2eeZvONzbzS5xXGjBuDVi38LiqahcaJdqNW2s0tCJkWwugf9KwPhYXCv1WeEQTEeGJiY4JCrmD3pN3cuPqE/PMZ1FnXcWLsRZZ0XUHIVH2MrPNH4qPjqfGaSIiZKfWF9ZjYmHTkC4Rr8M32xbqiiMb2WhqKGjg48yCrHq8iODCYC/MucPHbi3xm9xlzzs0xKGLrzyPwuJanllM3sI5BMYO4GnOVJwFPeBLwhLdD3+bc5HO49XIzoJId8IZQbEr7O42msqan2K/2TdqHY4gjtn62qJVqxv06jkHvDvqXTeaW7pZELIoQx93ISI+00mq1nFgq6EFu0Gx4ZpzXfV533Pu4GzTJgICk0jE/nF51moaSBtbkC0rKB3fsZt3/AhvPsvo8Mn7PwCnMSaCKVnY0B3ViDbI0ssTMwUzMG+r2A6mqjXPrruLZ39OgyKczc0dzzB0FNPLM4zOfet8+0P4pqlQQdEj9RzzruQr/lvv2ptZFyFvMPDZTjKd0NJ7Vbt2QOrrx+MRjEn5NYOgnQwW0d8ce1ze+L4G56Qw9PAtVu5r4d/SUlhqphhz/Zkwswhi4XmAKyDiWQc7lHMLnhSNVSGmyaEJiJ8E+yJ51ZetQmCnQfN/BLiOVU2dWh9xJzrDPhtHe0E5aM6hkQn4pRbKL726nMfzn4dg6Cf6dBAnmjeas+XoNZ2rOIJFKSNuXxqqMVWKxr9a2lnI3CXLzp+FjmW4fYFdlx4SjEzDubsm8S1MwdzKnvEBK//SbZLlsRtu4k+6p1siWGGHrJzwT3T0DWMyyYKvNVkZ/OxrPaE/6vtIXqUKKjbcNfkP98BtqyJTSeYq11rejbWvDxNoEtVKNkbmRQYEncGwQdoF21GTX4BTmxJK7S9jSZxsytYyzz91iVdeVHF96nKTtSbxR+wYuJj74lK2ie4IbIQ++QeMxiR5Leog5Hd11VzlUsWnDJt7xMyzs3isWmvgyq/XSKvk38tlzcQ9Z2iysjK3YMGgDTeVNPDr8iIDRAQz+dDDjNgtNunKFYZno6y+/5M033mCIQsErra24AKXFxez6+GM2vf8+mz/7jDVr12LfaSr/9Res/wd70f8frfOa93/b/lPs+4/9jzCNRsODBw8ICwujwP53APrFBVKZ/As9mt5CYfY00XtW1CyGbdZSfK9YLGToNsg6pyBURuao29Xc++Uerj1cce/tToOpgDYbEDcAu/I0uh/V64XogmatVEuTeTsauTnO4cJGEbEggu9/PUyFjZ47WaXWIDMyEuk0dd27NXY1tPZqxdTWlPOvnyf3ci6L7ywGBFSFR6EHZYllzLs0z4AWxLS+FJ/k4zR49CXzZibKFiUjvhjxVJdcc2Uzn0X9QOTiSIZ/Olx83byuGIeiFJQtfXDs5oiFiwVxL53BEwgMdOVuDyFYHL1tiphUSn8tjUlM4vM3/yQ38nnsPJVUZVZh7mhOwKgATO1MsXazxrzRnBe2vcA1zTWGvT/M4Hoa2hv45OInjD01FrdubkTMF2h6Hm2GfM1GlIoqsv0kWL9rzYllJ7D1s6Xvmr643n1Anl934se9xNzxNhyac4j8uHxeyX1FfBZF7kUU+vnQ3tjOmVfPEPtxLG693KiyvALw/7D3n9FRXNvaNnx1klqplXPOKIEAISSRswgCk7MBg21sjANOOG6HbW8nnBPGxtiYZKJNzjkJIQFCKCMJoZxz5/p+lNStNt7nnOcb433ecca75x9QV1fXqlqr1ppr3nPeN62OrbQ6tiLIJJx9+yyaFjGr+MKQCxSFFzEgcwCCRMAnwQefBB9TJmG7XTvNTs1sWrKJnUN2cvWzq0ikEh4+9TBTnp+CxlrDxN8mci9JpB5wj3a3GF89JrOSmaqYeh93rCui5dRBJn0zCe+B3mwet5m6i9X4Ry/HteE6cdnBaOpaqM/t4OBK8+J9atQpIOaBazWXNAOQrz7NX23apmkW1Tk7Z+2kLreOF2tfpPpmNdumbLOgCOoBWHpM26FlwrEJaK203HEVdVFOvnaSokNFLL+8HCRSbgQ9gnNLMCE1LwE2rKlcY9IINBqh1SOcm+NfICEKPGLAI8aD1vutbEn9jFHA7UBXjEo9RqOYIZj1UxYpL6XQOLOR+ux6BqUPIqrACWG5gZ5I/bE1YuWF81xnphdPJ9xbjC64hLnwQrU54CS3keMebab9sXWzxTXSFWM3RZDa2ZvR743GNcKVnN9zaClvwSHIjUqbSk6MO0F/GxG4njpvPj+/9S4L+e8z7dfLoaX/g7SirhGu/EP4B2lpUOT5IdE/qbnpHGUK7DaXNlN7uxaXMBeCxgSRlp5G3/i+pvN76xX0ZGW59XEj8elEQsaFkPljJrm7ci36cOIXlhwtagd3igbOxj/eC6PBiGAQ0LRp2LNgDzObZpIlz0LfoIdhYtakvac9+X/m0x7fTkZCBpdSLlHzirgJPvv2We7sEjXZts/djlqpJulgErXDa/GIFemjOmo7+HXsrwSOCGTwM4PZNXcXs3bMMjmrMoWM/D/z8Yz3NNFm9LaqKnjsMejbdwTv5YhZsXsW7aGpuIm4RXEY3j+NzcDFtHqEYeUtajvU3q7FrY8bckdLeiSjEcInh6NuVtNe007Fo+uYIpVTH7YUG20uhnv3ad3Qxmmplqv8e50kEOn7zmr1CAPA3VkcXzk7zBWPNoiBMIP9WIa8M55z756j7k4dTxc/bfHuqmqLaJc083Lzy2TVZ8FvoO/W7LsXN5nWlFScguSsSF9hmkt6xkHMtRKE69WMv7OUkW+PRGYlQ9+9IGittag9H0dTp2Jb2jZiF8SaAMgu61LuOzzFk789SXtzA/KFgdi42NCsa+a53c+x/baY0dpT5XLzl5s4Bjqyc/YJLvfZTmDtkyx+ZiUjnK35vt/3DH9jOLFzY+mS1iK1NnB65GmeTVxo8bx6KrxtO23xqhSBg976HnKJmWZWEEQNnXEfietX7h/ivONf7k9hRKFI+9brGfasBzZdNow4O5D6qEpSXhCpjYqPFVN7uharCCukBim+Fb60lrTikOLAYxliBQG9qHRn7J/BFZsrDHt1GMNeHUZhIRi733mjhy3TXpjGR64fWdybQ5sDx748xhPKJ8QKjO69eU9Au86tmMLIZEYJApc/vUzohFA8Yjw42PUajQ7naHSA9157kx9q0/i+3/fMPzAfITCALutSAELbriO4qWi5W0utXpw/rWRWWNlYoZQr6XehH5kfZeL7ji9IQOlsg6thHffsZvPErlVctL3IhM8moGkT1z+dXpybJEgRBJHS9uiaowxYMYCoGVGmZ2uUKZAH+JDyYrwpEF96pAyvai9qPGuQCg4IAgxebX5TevycvD55SGXxBDV38Mfrf5D2YxquEa4UOIi0wJF5kUw6NIl77oXYqzuouVmDplNj8Vx7fuvOzjs0FDTAArOzozOI74jMSobKV4W1kw36rjY6bTtpdLdFYf/XQIAUmcEep45ErPUedDZ2UX6hDI8YD1zCXLr1rqAszAp/pwieOpDK2bfP4hTohPG8wN2+d+kY0UnI3RAijXsYcHMnHhc88Hrdi6n8hKJkCm7FvzHswjCMz7RA9/tvo7BhVvQsjrYe5WriVcZHiQHtxKcSiUiLoHd9t+cAH9a2rEWqkKJp0yC3luPmJqOqO4He4S/sWDEfDGD5r0vxrAkjvDACQ5eWSV/1oqDtDsK3OMto1wRiHyAl/pF4HAMcTYETgILZj9LV7IxTdCfZu65SeKgQ7adGbgYvwxZnHtu0GZeaWlqXtnH0uaOM+ucoU+KCTCLDpdGF9tJ21M1qdF069N1BMmu1NU7NTghtlo6RkyQAz+Y0Rp9IQH97HR2LnrPQle0BOO8OmMmkVCMJK83z+P0jtwm/egODzAqX6lw0TasZ80yS6XiZVNQoyxyQSUBdLIPVOtprNNg425jAd4lRgn/eRxhladw/5YrmaicRkyO41HaJ54+JusTKCCXxs+MB+D7+e4JGBSHpBfaNODuCG+/ewOU9F2SvyQiaEUSbpwNDriwiPsMRTZCGz5Z+RszcGGZtn2UCRASpgFFipOxcGefePUf0rGgM/t3tEmTYa8TA3Z3f7yBtl/4fVZnbGwOIz11OhOQOu+aeYeWtlQjW5mefJ88jyDeIzWM3I7OS8Xz18xw/rkDiKvpUf6XO7G0OZ+HgtwcZsGKAybfr6acOZ3/Ubv5Y/wWM7nl/2+3b6XKBiKkR5urNXtWMZyZaEVc+ARADZ4sXQ5u/AHZiRWJZ3zT8h8Kfy38nd3cubxrfNJ0rkUoYsHwAuxfs5sw/zhA7L5aGBvE+8nKl2LnbIVVI+b7f97j1cWNV7ipTcsqQi0O4VnWNyGmRog8v6fGreyrdJDiEevBi7Yum6/Xcc13QIFoiBqFpaeXSJ5cImxDWnTjVa6xLjOR8e45jx+/wpvFNcluucbR/Cg66UOYE/4pMqaCruYu8ojz6DeiHtcyaVYNW8c21b7B6worYYEvgFcxrsnudO3X5dSQcEMHX9estwb42hzbqcuooPFRI+KRw0/MevecStwvuM3b8w6ages9coFQrca3R01nb3vuSKCXi/sFabc2o06OQDujAfrYnQk/FgkT87YQbUwnPvMD9lHzCwy016KQSKRJBgq5cR+HhQtxj3ImZG8OJnBM0ujSaKk+LjxeDgCnA3VOZoDfqcQl1sVhreluPvtLM7TMx6o2017Rj7ykmI9XcqkHpqMSoF6kG/Yf4o3S0pLV38HVg/LrxFqwUAHorO1wjxchlxJQIIqZEWBzvncQwft14XMIstYh7zC/JjwFPDeCpjqcQJAJdNl24NVmj7tAz7adpD3y/Z+/fVNLEtW+u4TPIB7/Bfshk0OYWQptbCP08YPJbk0n9IhWZlYyhW+ax8tAU7vvfp9YrmtjOOg6tOoRNUBTEhCHTd6C3FftWkBjQB4ay9NxSnvmnO1KDnrNpV7nrf5cJpyZwI/wGfa378kLNCyw5uAR6FXZp9DqufXPNor19XPvg3DYEpc6HlhaLQyYNb6lRyvDTrjTGleAzyMcElLdrxcTMau9qKtVKZEqFRTXMX/fVtRdEfzDs2nZaPIKJrB/H1aSrONVVYd+cj2tkDEvPLcUl1AXhhvnki0Mu8t2Y70xgTJeuiy+vfslPf/zEwiMLUegVjP2XOX5xdvhvvNUs6piXsZCgEXYIgsDCPQspbS5FcfR7VPSl09EbxQBv9i3dSWNhI49nPc6Jkb+xgAVsnb+Vm/E5yH0s9ep7z39gQFNUzmf+vzLpm0kMenIQDl1xaKyqmXxwMkrdNfr/MgdpVQV2HnamBJSjRUf5+N7HTBszDZWvig2DNoiJ770oUpekLSH0YCiLji5CNk5GjYtl5bdcKmf+n/P/rZZezOwYCxmMHhv38TiUzkqufn4Vg9aAwlbBqHce3L/12ImXTuDV36ub/tlgiufJujvaP8WfyoxKjDqjBYVqb+vN+PN3NuTlIRY01Q01Df/HyTJapRapTMqCAwtMx3rWxWP2x6h9r5aOX0TGHis7K4zdLBlSg47L6y6jV+vxHuDN6ddPEz0n2gSgCoKAYBDlUf6OArjH2qvbyd+fj3+Kfy8KZtGub7jOrV9vMf236dj5OAHiXNBjvamoBUH03+8OmIW3NzSXplN0pOgB3ffI/Eh8K6uQ2yuRdQ/JHl9YkAqcSq1ncI3ZvwqfHI7bs268nfk2u2x34djsiL5CjzRZaoqx9ICjcqkch1YHOu924jtRbNvt/TC44Bh6WQsXogex//lKgkcEM/C9gaZr6OV6ikOLmdZnGp0NnSKTk1SCv0MQCr0LOnkjAkbUzWp+f+5PQlNDGfjoQPYIS+iyLsOu1Q23ei9cnUNMlZQ97q7C4IRvaSzRtx1oudtIp68EsIVeCS25Ubl41Xqxc9ZO5u+fj8MUB5Mua2/b//h+am7UMHmfObPh/D/Pkv7ZJYa9Nozz751n4ZGFSKXmxLr+KwYik8E673U4hzoz9aepdKrsqHerJy+mGSsfN/xs/DCoRS3qYLtYYsu/wqUrA71VFjioSPsorVcrLCdHo9HIlS+ucPS5oyw7Z070dWxzJGN9BgFDAjj2/DHci9xhNbw14i2eSxbB8X8YxcTDHr1PsKRw/mzdOv712mucMxgY/Bd6ooVqNVeBKS+vpaGwgfgx75mOBVguqf+ftN4Yhkz29/Pb/5P2H7DvP/a/wo6tOUbmxkzC7obRZC8q6eZG5TI8+DmLzMq/OoUSqYQlp5Y8cLzFM4IWzwg0rfUcXHmQYa8Nw3eQr2kBPDH2BONap1v8Vg/tgtQgxa61A4milc4GuaiH5ajkytgW7rocx7HZkTm7nqB1cjb66f1oq2rD1tXWYiOtMYhBLU2bhq7GLr4/v4u1x16jVVFI/pR8ntr6FEGRQabNWJfiPp12ZwnRNWLU6FiZ8ehf53iUbbXYtlSj7/DHs5+nhYArQEWfMVREjmJNPzlLzywVz/F0Zvf87RiEMpIuJ+Hebin8K8wP41jDN6YgsbGikq8jNjHh8wkkPZOEylfFh4c/RJAIaKw1SKzF9p5YewKVv4rEVYm0a9u5LlxHtVHFW0vesuiLnooJubsPc9bM4fzt8yhsFBx84iAupU0cHPoCNb7WLBRycPB1MG2kejJhCyIL8DDEsrhvIEvPLTXTU3a3V6aXIdfLMegMLD27lNbyVoqPFbOvwQbXVgO3hsTj3jrB1CaXUBeC9wTz5KUn0VprKQ0uJW5ZHAmPmrOZ0wenE3UnioAbAbSWt4qVLd391ONQqOqKsW+8R1tVAuomNfZe9ti42JiebatrELYLptHnoVAcfBx4qf4lCk7dZ9GMJu73eYzrqX2Ii7PDq7+GOXvmcHj1Yc4Pu8+56nO4bE7nStd2CjM9+E4bxrKlj/DwyYfxXudNdXs1g9IHkZCRQMOCJtzCnem32JLmM3p2tKla1T3KnWk/T8MvyQ+D0WDKou2xE6+c4OIHFzkx5gStqlZC80PQtmvRq/Vo2jRIZBJyOs5y3+0X5m98EaNyB7DUREXS08+9/5+9NZuK9ApGvDkCZbQj6jstDL0wlP43+qNZmoPTU9EsPLKQLalbCFYF47LEBb/7foQU2dNZ287JVzMIHh3M9D3TWbppKXmBeXzr9y02Vn+fvevo78jSs0tNf/df1p/+y/qzp9tfUjt6MuxVUaPu8OrDlF8sZ8rZF3j91zVYaazQdcf3FsxYyLtvvfU/AoEu6yU4hqiIuRhD7W0R+Kq7U0fR0SL6TOuDQWpNnv9aNqzwZccjO0znfhH8BYCoP1TxHFKF1LQxyNuXx6nXTxE4KBArrRVHFh4h9f1UfBN9cYty49CTh6jLqWPG1hmmbM6/21TorWxp9o5G4Qm3t91i7+K9TPh8AsXHirHBhhRSMFYb4UtRh6E+r572qnYMEQa0Vlq01lqT5kL/5f1NYN9S66XU2tWiOqSidrF4z81lzWydvJW6nDpqs2sZ88EYUbOuO3O2o66DI88eYcCjA+i/rP8Dbb3y+RUyLmuB4dy6Zf5cZiVDZi0TMyW71Ljfu06Zkw9arVnbcNqmaUR0B2udqu7gl3eKxolpTP7HSEAMDgFIjXrOjr1I8L03SXFsJGVhClZKOVM2f84Brebf6nRNVsgxjNIzIm4ET657ksrrlaR/lW76jpsQjXvrRDQO4cQ+mkx2W7spA7x3gkZB0sM89glYqaTk3s1lRtQMfJpnUYq4WTMoFMitwXeQeSPn1jGcdqsiMhMb6G8/QQQBPbsPdr9wjtoofJpmI9HWUnyqGN/BvhT6mKty1Uo12bHZ9PEcQfySeOKXxFPcWMz2gyLQ513pjf9Jf8oDy1l5ayUSiYSffhEBdvuuKIJs4rBV3Bf7stc7rrXWcnbkWd4aGMaPg39Er9Gz8sZK0/pXEFnA0bmDWTk2gsrrYiKOrZstIcpBnOJH/O/5416Wgaa9L9bdFdExC/vx7oX3KYjIZcmmJTi1tlCQkkNnSU13NWM3JZJjCz8t/53hdWtM7Rn1ziiyWrOIuRXDtUHXWL5xOUWaIkakmCmogpyCKG0uFf9/NYhS91KSugGE3pp9EkGBzEpG7LxY7LzsUDopWZOzhvygfF76+CWyvLLxHuht0hDuCaJW+lZybmwDqxysOfbUHqY6T8UjxoMCvVkHD8Dob8fId8Sq/3a1+aH2BKfPrfodpZ3MQp9WJpHR91Zfso5kkfVeFo9ee5SOuk7CcurJ9pfjlu9CXU6dia4SMFX2yfQC2vvHqEj3pOxsGWGpYb2HEFpbZ5SPPYxbn0q+j/+eYa8OQ9OmQ6lW4l7nTvyNQXRSDvib70Ew37Nc0YFdbDDzD8zHM65bB7Tbx9LL9bQ5tNGFlnFrkklek0xpY6nIkQmMPT4W4w9GtPla5u6bi0Fr4IuPl5uuU3u1lsvFl4mdG4u2Xcvs008y9V9bESQ+CF4LcI2148ATB1A3qpm1YxYD7dOYmNVG0I19uFT+k7o7S9jx0A7GfTyOlBdS0MhENoPLwypxlaXQXJqHXq3HwdcB7dVm3CvdaXRpRHO7kZL2wWRYSUnuHMvY6rFYC15Y67wpDSpFYfAgqb2D3L01BI8KxsXJhZ2zdzKHOexU7WRSkgjG9VRx3zUzQyKVS7GytaYqq4ofBvzA6PdG4+IyzAT2/dW/tWq3Jf7OSiJynYjI3oJ25pNA7+oBASSQ7/saDl3RuPV/iJRu/ammVjOwesZvCgPaDpHx7gnuHSrkiewn+Oz8bWgFldQLicwLvVKLU4gLyy4sw9HfkQv3LlDbUUuLpoUfHv+B3FW5nHztJBnfZhD6x0gAQu6GMPf3udj623L66mnco9yJnRdLpGwCg4omYCW7gbRvEZ31nbRVtOER54HCRmG6T72VLdK/AEjXX9mNI3Bq/D3sI+JJcrTU7upJhEgfnI727moaz9xi3YIDLDq6CCFUnEMlggT/fCVNnne5vPq46dyEbLOfNzzQXNlk0Bo4kneE9KQvTJ9VeVfhNM2JoJFB6J7XcV93HyedFEEiZsRLpBIGPzMYr/5iuLGPJI0rfI9zozOqei39P+xP9MxonEOcqde04NE8CZDgk38GrcyWtA1p3P4om83rP/8fVZmvl0Oscz+iL2zA9oWVTFrthmOAI8OVw7nx+A3i18ezY94OXljxAmWbRRrR2tu1qK2gXZlPQFkAjb80cs71HIUHCpm9azYqXxWDbOdwrfN3riWW8OHnH/Cx+8eEpYYxc+tMi6pTgwE66jopOXkXr/5euEW6md7188PPIx0+mk/fMM+5PUkptuoQ+pX9iEzbxae+n+I3uS8wtpeGjnl/Fzo+FDsPOyQSCZO3TqZF3cLnqZ8T4BjAkJeGEDs/FqlCirqb1r/U8ysm/JbPMmsx6FWfV09XUxeCIM6n2+ZvI+fpHNz6uJkoCQ0GMErFd8Ot8EfufDuGmG/M7e69dhsM0FLewrl3zmGtssY/xd/U7pjbMUTcVRL0aH/6TRf3cj1zY5uimN/TniA1awsfu3xM1Ooo+g3oh8paxVcTv+LrSV+brnHvwj0OPH6AUe+OImpGlAns2zFvB/940lwVLO6nzGDfuRHniMqJouJaRbeOkg4kUBzeRGDfoVTfrEbbriVgSICpXSUhJfz6eBlTorxFNoznkoieFc1i29/Ia72GVFtF0tUkNPI22mo6wagEqdS0/urlNnSqvLnzw0Vs1E0kPZvEqZJTPHXoKXLrcyEanPycuPD+BSImRxA1PYrt+dtp0bTw+qrXkQ6Ukrc3D8EosCp3FWXNZezJ3QNAxysdbLPdxvw/H6yKATj+0nHuHjfTe87fP98EzK2PX0/07Gg8+3ly+vXTPJb5mEkXurm0mcufXiZyWiTJa5If+N3qsCEs2z/kgc//zpKfe/D86xuuIxgFEh5PwG+cH4G/BXL+3nmGXBjCuBPjaBxeCwMerIaqzKhEZi0jdHwoz1c/b0qM6h0nlEjg9vbb7F28l7n75uLcz4eiwiKkBinhBYEgq2LJqSWs+9YGukR6vk6bZuKLxzHiaCySkSKVv8EKDMCkKRs4ez8AVYuKtoY2JF4S7DzseHr00+zeZF6oNHotdh5mEBRgivvTnL0/FYXe5QGa+idznsTnI1/0ch0O0rdITIpg4nxz8Lx3la+Aka4WDS1F9TgGOGLvZU+LtJgir50kXrTGtisMWz9XNHVtlMdMQC83cidU1BdTq/9J9J1IBGIJHBYo/l5Wr2SHqDxC55irpDp0Haw9uRZftS+BZYHo6y0TOa187WnSNSDXOyI12FFzq4afN/zMYdlhmp2bSZaZ+UoFAWxcbCihhIQfEpgeOBtNuY5qr2oERJ3uDYkbUPmpmLtnrime5FfuR/gdHYywYvCzg/HsK/pJSYXHOBrvwvFxx4m99xA3PzuNnR0sPGROopNJZZQFlVGeWi5W2YU6PwDaZdpkMiB1AO4x7lj9w4q8E3kAeFV50SevD9bh1gQuCOSvdubtM7SUtTBt44NANIgJ7vDfa+312L0L9/4tiCeRSHjk4iP/9lyD1sDm8Zuxc7cj7cc0C6A+/898Wu+3MujJQSZ6/B5zdXehuqzxf9S+GkCmkKAoVqDt0Jq0dgE+HPshBqOBrm1dlJ4p5eoXVzn/3nmWnV+GRCIiGXorW54pfQZrB2uaS5u59dst3GPcTWBf4aFCtk3ZxrSfp4EErOysTJSKAHdP3OXQqkNETI3g8ieXSf0iFY8YDxoKGvg29luGvjIUhY2ChsIGjHrjA74ggEEnMmFIpJa68EajmFzWu0J8cNku0n0f5ujkcqLLV7GgrAm9xgC4mar+AORGB2yr7/Jj0inGvD8GvyQ/qv9Vjb6fHqbD3B1zUe9Qo5+ip+luE3YedmYaT5mC1COp5H0qzukSiQSjEew1YmK2TLAiJieGsvYyMGFDEjRKDdsWbGPryq0ADH9N9MUctU54N83CtzCbCcd0VCWUkrsnFzsv0QfUCh0ggXr3ei6lpbFmWi/azu7nEVi3EqnTMm6M02H/zE9kOFtDwEp6b2IPTjnIx9M/xnDHgPcAb5pKmqjPq8c/2Z+S0yVkbshk/LrxCEYBwWjZEV6J/iSuTsR7oDf9l/dH5adCVtqrjwziHJ74dCI2LjY4+Kj4ffYuatxuo7XWYjTCwEcHMvDRgaZ23/X4nIuRu/BrWMICF2csrbuKUicn4F4ARkUXLn1diJoRha2bLTKJDINgoK++LwdWHmDiVxMZ/sZwtvy+BYD3zr/Hs0nPmmKYFekVnH77NCturkChUyBZIn5eXV3N2pdfFoG+B4eeOKaAA3odw77/Fx8qJgMp/+ab/7H/2/YfsO8/9r/CHAMdcY5xFoPf3boC+X3ycfQfTvrXl6nLrWPqhqkWC5xX0UWub1CaJk14MFii8lex+PhiU/Vdz4R/3/8+bW1OHFx1kLaKNubtm2cCmJRqJVN/t0LjuYuP3coZ8dYIkp9LxrZVgtxBTljl83hWKzA2dFCZUcnPw35mwucTTBuqgRkDcf2XK2XHypjy3RQARr2wgVYHMfvNKDNilBtBELO/JBIJJ/uJAZozg5T8GPQyvXTLe1Ee5eGbfxpt2+MWAKfpe1IZILPQwAlOjeDjtT+gtqrgpQ9fosOxifILpVTp1PR5qA/GcT5cvncZ50ZnHIvegOCHGPrKUHwSfBCMAuv7r0efrKdzQifrV67n/efeB+D6+ut4D/QmcVUiO3JEMKNJLTrG5ZfKyfguA63dYIzdYF+EfBw+Dj4mqsnDTx8m/at0mpybcKt2R+vebFGlmOg6gbhSkR7RThOK3NaKwBSzw9ozRkadHsXQi0NpS2jCa6YPXv28cAlzYcBMZ0I9V6CxE8HDO7vukLkhkwmfT8Aq3IqOWx2m3+oN0nY2dPJy9Mt8KHxIy+EWYlNj2T5tO0VHinhd87ppk6OqK8br7mWKjznzx9I/mPj1RBJXJZr6SmPvhmKQG+qWOvY/up/wKeGoIrzRW9khyFQIchWCFGxdbYmaHsWRu4c48NIXDJfCY3o1Xqiprmxio+Qu89Yd5kPvj83aLEYpMoMMo+FvvEGwcPgcfByIXxrPkaIjzPpgFuunrCf8fDiNRY2kfpZqAtKT0yeS0f8EI86NoOW1Fiasm8CEdSJIWqsRKR+z+mcR2CQC5NU3q1E6KXEKdMJgEDUyHRpKEYJ82LNQ3Lxf/eIqIXtHsebETFStKjwbhxGuUokZXN3UkvJWOU9/9TRvv/k2l0dIGdWu5cK/LiCVSxn0+iCybmVhrbam9kYtnqGe2LqK3PFFR4vwivcyUXv9V9Z7Tpj0zSQ0rRpyLpfw7BfPktsnlwZPO3gVQuNC+fCTj0l77TX2a/49CDQJGWMYgzRbxuD0wexfX8nyrzyoSK/g2JpjXPnsCn06bbkS5EyFXwX2weaqBbcoN+pz60l6LglNq4aGGw20ubbh5O+EYBQwdm+KnZucuXvkLp1rxIxYXYeOW7+JaJhgFHjf7n2CRwUz6/dZ7F20l5DxIabKzR6TSMApyIm4BXH4J/sz/eQ8Fnw3k0aXRh4bLVY81dyqYcvELQQOD6TsbBlz7s8h7pU4qAUCICw1jH8I/2Db1G0U/FrA8xnPI18iR+UnRmg1LRpay1vxiPXAI9aDhMcTTPQwALpOHSUnSyg5WUL45HBubr5J0aEilpxeIopP/3aLtvtdMGg49g2lpH9TS/9l/U0bUINOfOGsO5sZcOifFMoG0v+zJCZ+NfEvegoSjFIZvf1yl3AXbqxu54TV9/hpRK08iasLMz6ewQxm8MclD4bkPU+KTKR48QSqgV+VSs7qdMx9Zh6b7X8zZb75DPThH4I5+JYgPIm0UKTQMhoxVan1/G0afzI5Bq2G8owKVu9cTaddJz+GbqQUsOpqQaHrxKj3RK/WiyCnlYx+lV/SYpXLmZQPcTTE0lnfia5Lh8pPZaIws+lUEnPmW5r8Ynld87p4sV4VbK2OreyetZuXlR+a29VrrutLX9zOu9F0twn/ZH+L4xIk1J27g0RfyeM3HreoQO8xg0HAO8HbTIXTfdN26gg8WlOpTC/m4LzfmLJ+CgMfG8goh0epOrCCgNtH8Sw9SP4f1mRvucWw14bhPiCAi0Ovope3YrSORmfrzd1DeRTuuc2IN0dYaHFVe1cjqdLxQ8IPJKxMYMCKAaiPqhlYO5BLKZc4Ov4or056la7GLrI2ZuGX5MfHfY8w+1wfHK0deaX1FSQyCfl/5lOfX4/L5BQMUvE921/+Ah88XMXClQtNlQe57+WiU+vYvGgzQ0c/ye55uyk8XMgb2jcsdNGQGPEc6M8jlx4xia/3BIbt2u3wu+9HR2IHI9aKgeXiG+Y1yLfgKq6VxYSuGohfqBj0u3jvIr/c/IUOXQfb521nu9d2Oq51YONiw+Zxm4lqVlPs05e9nxzhnaffwag3gkSs1nWUiwCEQ5sDIUcuUxc/hFc7Xv3b8anRwIZBGwA4/955ItYO4t7VewSXBJN8JQFtRA0n1ubTUdchUqV3+1ByvYqA+hVIHexM9MjdI168v24aqXl9PzGPGcGs9fHKmFco3F+IIAgm2rTeNJ5VJ6so/rIYha2CgysPMuT9SQQ0mANGRqNIsdZeY65SaVfm0eSiQ6YLxCnYmembp5vAmBo7EXgt9fwK6r/k9Bunqb1dy/A3h3Mt9hYHCzcz/AI8pgcvqqhWw6/XdhDo9xsjR3+Eq/IJEvudJL9diVBayO8zTvJoxqOmCubFfRcz2HcwQ/z/feBY066jobQeg9ZA/LJ4PPt6outVrP9XFofzs4/yUL4XhbFe3E50INjJnqPPH8Xe054hLw0xja+5O+bSJ/8mwhsPmc6VSeQMKN5BZuhclLXl2DaXc++syGzhFuWG8YL4PalESkWUWO0gt4WAIWJw653N73D8rhkoMwpGgkYGIZVJqbcS58Q2RyW+i0cxdVofvosTKypi58UiCFCnOk5NrJqQ2FTu7Ern/D/Psyp3FW593MxZ2F2taO+r+TrydyKmRjD+4/HIQmw573mCcynHcGofzGrhcb7r+x1RM6IY+dZIC8pBQWLAytedQasGofJXUWc0QhsYrxv52O0trJu9UKEnmFgeWbqcdoN5rOSvz+fdCe/y2LnHWHVnFUGfB1nQ/eZF5eE9yxv/GH90HTqaLjXhXC0Dx8fJmlLDy2kJBC42+6SR0kn41y8jOqOSgVc0CB8IJkYIF6krA4v3UOT9Hs73z2DQBeDgk8jTa58h8OuP/0cJRpcM8ELqDO5fCKWPpzsxs8WgsRIlV//shxMBNHOPrmozVaTKT0Wm8yq6rEsJLxxD84VmGpc00nKvxZSQI9xaAGG/0+IqJ3hMCH5JfiYJgp5+8iy+hFt5FsXHR7B34W4mfDbBAuwDM23dg39LcC9NBwFUAY7cLrYBWzM91UXZPxmqvkr1yQUMPLbUdP7Z0rN06DoYtGEQL3zzAiNnjWTyt6K+m74XeJB7OpeC7QWo/FW0lrfSXNKMttkVZPeo9ay1oPMT22X+f0CJlLLmm9we7opXPy/c+riZfHynqlwcGktxDh7GkzlPmqoaVOoY2oVCRlxfiUdJA26JIcQP6K6E7PUM2mxvobWW0G9ZP2zCxUS1dZfX8fqp13ki4Qm+mvRVd3vEgGKPfk/vNdogGDj1xilaSlsQUqYjdFegO2jD2f3iL/T9R18T601Pdfr2OZ8yvf8bHFvzJ1WZVbzc9LJFuwSJAW2nnqaSJiqqK3DvcEcQ3JEIUtrt21m3Zh1P3niPnYM+RjniCdQOZkq40uASanxjcD95mtw9uSQ9m0SrplUE+rrNerA1aRvSTLS8PetF9J1oMjMzxWB4t/Xcq53CDl9bXwoOFPBt7LdM+HTCA9R2iasTaSxsxCnIiZi5MXj2E8e/RCJh5DsjcY92N2lo9tYs76jtIP2rdFR+qgfo2f5qefvyyP8jn9HvjTYl05acLkHToqHPQ38PeqR/lY5BayDh8QRsFDZ80PAB323+jvTEdDIGFTDSwY5r310DwSzLALAtbRsuYS4sO7/Mgh1GKgX7xnu43ctC4pPE3n/uBaDmZg26LinJl5IpiChg9s401FHpBI2chm47UAHHp1mT5/87KXcfRtVQi7qzU/QLBAlIJDhKfImRxjDzs5kUtheS8l0K9Xn1eMo9Le5JrdOy8uZKC/BmV97vXIx6Gp/GuYw3bLf4vr2nPW0qUQuxxk+JlYeTxXF991wz6tQokq/cp3BMIX8+LM4jSc8mccF5JZXKE4xsXIlDl4wBm5+mrg4yNoJb6VkGXh/I7djbnB96hFp/f15SiVWcEpnkgWrl29/fZusHW1l8YjHyPqLfWuFXwfuvvc8zg5/h5GsnkSlkjHxrJGEbhvHisdV4V/bB+fZ5bmzqRPO1BtdFrjQ7N2OUimwV3oXnaD6TztDsFbx9/G2uV17n+rLrDM+5xeDcAQw4f5v2hzJwc7N9gGI7vDCcgZcFGG9N6r9SLdqKICW/Tz5eem+kzfXIe5EVZFVl8cRBMemr5x5nbZ8lHuzl5+fE5uA90xuVrwpjiflZeNZ4MvLsSGynmSlpe1vZ2TJqbtVQn1/P/cv3CR0f+kACeW/TtGnYPm074ZPDSXn+wSD/Y5mPceq1U+TtyyM8zZLj979KTgVouttE2VkxznBn1x3e0L9hqi7P+C6DsnNlFu9Oj81c/jCb33//f5Qs84vSmuF9xtDn1T7Ujq6luaQZmZWMqBlRrEnuTh4cAlO/nsrdE3fFymEve+hJQpVITfOK0knJa+rXLO7HwceBmLkxOAU5sffhvTh4O1iAfRKZBJm1DM++niw8vNBMd2pvRfjEcFzDXem7qK9Jy6/nlvzuHMW5Kg+9ZhUfu32M/xD/bopmwGgk8PZBpP6+wACL+7XThjAk/4Lp7/MvbKK5oBaGvGSq7ANIPTwepfoqrbJWdF061M1qpAuk3JTfJNwlnNglsfha+1KXW2dKUNs/fz96o543Tr8hyhYNHcmmEZtoLW8l6PNnzPeMlO+e+I53NrzDH8v+YNrP0/BumkmJ5xf8nfVonDe4NnAvGByC3Sz23L39Da28nvs/HuTtmRmsqVyDWq/geN9QJEgYeTsPudGe0JnxODrLOHgN3FrHMiT3MhejRIDQLtCOgL6ir3v+/fOceu0UwaODiXwokrJzZWhaNEzdILI+9STjAYRM7EP/eeJa0FNBLL1kPn7l8ytUXS5j9q7Zovbw9tss3zCBC0PsiMqNomnODZgTb/q+Wt9Fk/1lmhwu4to+AmNTC7sXnCB6VrTIwtK99tp12PHw5ofRjq8j4lWxAl0QBDNFd5wHi48vxjXSFUd/R85knAGgoauBmb/PZMv4LVRnVdNQ2EDpyVI8bD3QWmuxtxLXHpP+5V8zOf5ig4GhyDibvhXcxXngfyg7/R/7f9D+A/b9x/5X2OBnBmM72hZrB2t6o1WCgBgsPlXyANjnUXKFrB8dabnXgnd/bwt6KreyDFwrbqNpnUnIWLOD35u32SAYaL3XSnNZM2B2qHQKHaV9onF0DSblIQ98Bvpw45cbzNrkRYXPMs5NknEjdS3zHgIH3yaSX0jGZ6APxjPi+a9Nf41mTTPWjmbPTaszX9e50Zmma018PelrpHIpq+6sMrcPPYIgiqlKJBK84r1o17VwuL8vLv5OjHM4zIzu8v4eU3QDg4quFqzUbbSWO5CxI1MMzDg54lrnSIVvBUemtuDbNJurH52j/lYVfR7qg49VJP71yxC6jhOeJ0fvq2HM+yI92tWvRCHgjggxKPnNpG9wsRGDmKvyVpmAon15+5DpZciuy6jPq6exuJFbv91CNysawV10lr3uKPki+Aum/jSV4NHBTPxyIpNcJ+Fa78rSX+bTOiUTnjdTE1pLbQmsfxSnqlxcKk/QcX8MRoNY6i+RSExgX4VvBZn9Mxlqo0DdokZuLaf8cjkBd45SZOOIbUs1UoOGtkot5ZfL0bZpKX+unIerHuZg2kFWf7maP278QcjYECImR7Bh0AZs6m3Y+NxGhiWKug5+KX7IbSy1ZmpCkqn3j+epZBnD3zTz2P812+rb6G8BMetrxsmHOZ3mjkKdh1+NI9oWJ8BGLJ1/5VUuGI0M/kuwb6GgE0vn175CmEN/lFOVXE26SmVYNOtCXNCr9XwV/hVRs6JI/ewvG4leNvP3mXTqOlm0dxG/XfmNsvNlpH6Wyuh3R6M3yth0oICi8Grs3EstNGzAHEQ4Me4EUzLE4N/6+PVEzYxizq45CALYttUQmrUbo88UBq0axLVvrhE1MwqNSkG9ez2dDioq+z5PZCBo27U0FjUy6dtJHHryEDf63UCQCghSHdbezjyV/xRKZyUatQaHVgf8y/35dfCvpHydwrhV42i628SeBXsY868xuEe5o+3QcunjS3gP9CYyLZLCw4VUpFcg1SVjVFhjVV/JT8mHSXkxxZQZeOStqzQ7NuNf7k94oa0pGyvFaigDNCMYxnGGIGMFehMItElhzTmdltGMIZkUcnzknJ3kjWN+OMuB8EnhzPp9Frvm7MKRVhzaHGi3b0fdad6I9H7XS8+VcmHZBWzW2ZCyRmxbyNQQVr+3mrKgMua/OZ/ApEDy/8wnc0Mmdh526Lp0hIwNIXpWNO7R7hi0BvL352Pnaa5+sGmtIfLyLzSrhhGwJdlEW5R5sZicWJGG0uDeTcOmshb1Mx2sKDtXxgDrAbSuaOXys5dJ/SzVVNWa9GwSCKK2am/gx7OvJ2tb1v7bcecU6MTyK8u59s01HLwdKD1dSkt5iwnUXHBwAWdPGbm0FVwqczj8VAYxs2NMASypXIrx9TcoTtcQfGMfIX5uOAc7mwDt9u7YbbN3FM3eUUzpC6feOIVRb2Tsv8bS6q+nvbOdzsZtqNs7MGo/BcTfDgldQ+7EVzifpSUzGxQdLvh5ejJvuajTdbruNJv3/Mbp0tN8PO1jho8djmAQaK9pZ+y/xqIXdFwLm0loviM7x6Uw+5dppnmgd9Be2V5PwbZ8bnx0gphJMVxLvAaC+Ax9Cs7gVn6Dujsr+b7f96S8mGICDSWCmXrt4JMHubNTpArzVgYTWrUWrO9gkLY9mOHyFzMYjdy7cI/7V+7jME18t52UThz65hD6T/XIreXU59Ujt5GbAnP3XX/j+ul6VBdh2KvDzH0uSLDWWLNg6wLuVV5i2frJFtcBsOmwRlVbiERhxZh/jTFRzwiCBAkSaoMG0ewZiUHXRumZUgY+PpDiI4Us2DKDE2OPURE7E1tNKC+81cb490YgVZgrCyRGCVYaKVKdhq6GLnRdOhqLGpn86mT8k/05cPgAl1Mu4zTUiY7aDo6/eJxhrw/jqzwPiAWjTmEaW9lbssn5PYfZk1KIvr+OYv1HGDUnEfYKVCZU4pfsx+axm1les5zdqbspDivGPtgdu9FB2LqLAZSejHXPak+GnXGhcWQtCTPNAEBPu72qvZi/fT5am0ogvvt5mAdJjWo7zhVBBE/vT9wgMas5vyGfDZkiCNemaiN0WSiRL4pZxBO/mczPT/5CnXsdNS1ixdqvY36l7k4dL9a9iEJija0mmA67SopHxPBor6BDb5Npu9AeNu9Up6yfwp0zFfS71Y+cmBy+XbWN96JfoXz7FlrKWizuaczZR+h//RJdKUGAOaBuSV8l+j61t2upz69HniDHx9YHlY2K0U+NZvTb4prfVNKdQd8L7POZ78OIeSOwdrBm6KtDuXeqiD7XMsjv54xzXScNBVNZdHSR+boCpIdPojO2hCG5V7DztDdRk/a2iYcmYqs/Qe1tkdL7510/cWHfNi4KMPgvm9Ye6prUUy8j9N9GTVs8FRkXuGVsxsPHBuMuAyt9nsDLy4thymF0revCaq4VzIF9S/ZRlVmF59vmKs3GwgY2Jf1A4IhAHj75MFKZFN0x8/X+CvY15dcBcGTka2htvZlqs4Ds37JxjXBlyEtDsBM8QQLl/uXYq+OpvVbGnX9mkvxCMg5hXvg0zSFTmMesXbOQys7jMyYKvxgHUR+xO0hQqc/B1vc1XNqGYzSMx6AzIpVJLYKoYYVhVJ2qYszcMcTMjmHzmYvi/bi24ZkyHI8YePL2k6aqA0GAzJC56ORNDDHeIWxCGNYqaxNgUuT4I0V+PzJ7x2RqTxrxiPVAIpNQ0FCA6qs4jl17SfwdiRGDXkCv1psSP3oCPqmHU/GrqcNmYgCT3hTXuHVrP0L2qYQhMqEbtK3uTuC4wvLN55njOhe63ZtzTedI9Uk1zW2K3hl23daTwW7W9JHi0j4Uax30whNM94wgIzcqFyf7IUjlUtpr2ruBbFEXtNDnXb5bqWKNsoiOug7sBDv+9cEHTFy7lsP/JqP6KjBeBoYx8KHfMoJGPU2UdJjFd/bvB22cjMQbiVToKoidH0vohFAcvB2QdFMwXht0jekrpjNl7hSTdiKY1xlBYsBoNFeWCIKA0di9F5NIECRSnCPcmL1rtqk6pScBLbwgnMAKgeMvH0fXoWPS15NQSK2x0QSiMDjhXXgOja0zy+qWM306IEBkxbuEVb9EVt+xROcEY6fuFVHDMjFFa6NFYWfun95JFhprDfK+chKmJuAU5IRTsBNWTX3BLYvYux+iadNQdLiIutw6Rv5jpMU79uccDd8Oe4jd835mzAdjGNpnqOm4Q2MZniXpGPRDTKAtQHLBaQQB2vu1UhuoZaqk937V3K60P9PwbcgkrXgB2dnZ3e02YBAM1HXWsXHcRoydRlZcXMGqXNEv/OKKOQjqe9+X+0fuU3eljppbNdgmmyv7PLqGIz8k53rjdZOeVZR2CVeV73Rfx0ji04kmhg8HuRMgat4G3LXBKtWOpUVLcfvYDT6BzxwFIive43rYTNpUbTjFxuLv2cHNOhG4sFOH06Es5HrYLBw6Y/nqz6tE9pE80E+u9a6os9QEPxZsArB6qtZ3zN1BzfM1FsBWz7lSiZT5f86n8HAhR587avEceywyLRIEUaM8/pF4C+mCEb0qSnvGJsDly+Cs8mJN5RpaylpY3389SWuSLNhQVHXFnHm7nOTnkqnKrOLGphukvJRiAj7OvnWWujt1hE8OZ8OgDYRPDmfMe2Moay7Dx8GHWTtEAObGphvk/yHqXYW1hLFj7g4qA+S87unItVeuIQiCBWAx4q0RKJ2UaFo1tFa0ilTVKmvkcrDuaMTt/g2ENnMlk723PQfmb2MCE7jvd58/ph1hmFRMful5XB0OEjrsO6iM6iJL+wrRmizeVbyLKnEBrR7hGI1gtDVyOekysYNi0bRq+C72O+JXxPPY2Nf5IU9khqhv0lFWVIOuQ2eq8JJ2yysY0T9Q2Vd2vox52+ZxJekK52LiAYELH1xAbiMn6ZkkE9NAm0Mbza4KHENcGPvhWNPepGdO//6J7xlz8x7xO69SdbYQmdcsNEpbKnwr0Cl0NLk0UaPVcffEXXbNENk9BFfzWJm+Zzrpt9IJHhOMlb2VhRab+JwE7uy8g1wpZ+RbI00MTE7NEjyyT+G0IhXZtzIqykQdXqNEPK63skXq4YpULjWtCQA3gpYQUJGAUT4IFHKLqryeMZzVPwuN01AWOInjddcuuHJF/I6k+5kKGNG1a8g7UcJHrh/x+I3HaTGI/lZgaSABxwKoiqjCe4A3giAwJngMJ0vMrBE6o47m0ma0H2oJdQqlOKyY/Mh8Sr8o5ftl3/O+/fsEjQhiwUEzdeXDJ0Wa34z1GRxceZDFxxdbgH2/jv0VpZOS+GXxSGVS/JL8qLlZYwLZe1tDYQM/D/2ZjtoOXCNdHwD7Ggsb+Trya2TWMp6vet6CGQhEppuhrwylPq8eG1cbCyaRCZ9PoL1a3Ojtf3w/9y/f54lbok+14tFHefftt/9nkgw6PZdfe5eOmx2o/FTsWbgHawdrU0zgh+s/sDFrI7OiZ/HC2BfMccMe/0wQaLrbjJW9FXYedg8kQHr39zaBsbN/n41UYan1Gjwq2NTu3ubg48C8P+b927YbZVborWwQjALRs6JN1bY977z7vUzaZVqaS0Ooz6/Hb7AfSiel6bhVZzNSg5bQmfF01nVwMdccB5UIUjxKM2jxjmTNfRHwzPsjD+NWI/Yz7BkcOJjVq1cD0FbVxoh/jCBgaACBoeIeRyaVUZ9cz9CAoeS/n09naCdd+k7yfT5AZrRHQEBrrcUxwhFbD3HPFFL9POUu3zD2xDhuhd/G2l5B7e1ahrw0hPrORipddqDzaERwMLI2+C+JOt3zhLXami71q9TpniRuZhQKGwUGoxGNVSUAck0nSnUXMY8NobBEDtfAyuCKVYcrtgpbko4kkaXLIuCf4vwTMjaEU6+dwjHIkcGrBz9AJd17vvs7YKt3NXbV9Ury/8zHqBf9aCsHa3RyPVKjFCnuCFIZWRuzKL9UzuRvJ3Ox/gBVLr/jV+5H1G0Jxuh6bm+7jVuUW/fYFDuyy6aL/VP2Mzr2WXO7es1v9q72OI/9a1WgaHvz9lLjVcOvY35l4lcTeV39+gPf+WPrVp79H+pfLkfPGzf2EjPu63/7TP5j/3ftP2Dff+x/hUmlUuLi4kSh024KlpSLKdSWbOTpS3NNJfe99wD5yUt57juBb6K+od/D/SzAPoWmHWV7HQbNX5y97sVi3rZ5hN27xPzO10zHemgX5IIP9WGzkXrAlO/FY/cuiIT2njWe6GVt6GTN6A0qnIOdGf+xqNGiOKdAEAQSZyXi+4gY4Cw6UkRnQ6cpM9Ja68O/mt4kf0E+cQvisPWwNbXZptOGyLwotNIa9nwiVketurMKCTIMsg7qPDto8glFQBQD90/xJyw1zAT2uZddx6foPNXX53DunXMAZP50g0cr5vDF01/gZFyFzCqOQavcsDKKk7qLwofA2ie5FPkLn635hSek5ueRs10EBqwLrJH7ydFs01DcVEzo+FCLDRuIWkppG9JIt04n9fNUYmbH8MNPUox3xMq+XI6S5LCA6+uv01HbQew8Uauiw66DIxOOszjuEW5uvkl7VbuYsd7zTNrrcKnKoeFqIO/OPcS4T8aJWWXdYyQ3Opfc6Fyec1LyodOHRM2IIndPLmobJYWhWSSdrMG2Q8fgp19i8NODaSlvofNSJyGE8GHahzhUOKBp0XD4qcM4BTrRd3Ffrnx2hZiQGGQ3ZbRFtllUKvW0S6d0QKd0wDUCCx2ynuNOVXdof/cYDj4OtFW2AXBy9QEy5jxNzO0YYs7Ppn3gbP54rZCX33+R8wj/dem8TsuwxnRGZI6hLKiMwNonRVofQcDBx0Hk8e+2s++cpeZWDXN2zaHgQAGHVh0ickwkWYFZAMzYOkPM9uw2rwQ/gtdlIJfvosknGqWTqMHQXt1O/2X9kQjmZcTdwwhIGfXuKFNmmtEInSpPigfMJjDIm0lfD2TS15NQt6j5JPRzpvlP49TYO6bvVt+s5uehPzPhswnc2XmHfTn7cGlwwavaGqMgNWWVfyL5hOd5nstJlzkx5gSJoSLA4xzqzOxds0189waNgbNvn2XAowOITIuk6HAR6V+lIxs7EKPCGkFvoOVeC7/P/B2nICeeyH6C8kPZVIdWc3LMSbwaRvOmETqqW6m7U8eclLmEFMdyUraf1yqb6JIa8A31YubDC9j8yHK8vL3IyJCQ8Q7YSUHfjenbedgRmRbJQ788xLvrG6n0+SdrP1hLxsUcBp95sHddQl0Yt24cQSOCTJ9Zy62ZETWDPbl7sJaJP9xY3EjhoUIeufQIv8/8nR0P7RC1FLvtTf2bFr9rlCnoVHkhU5kBQIPOgFDdjF27+Jmmymhqw/TN07l78i4V6RUMe3UYDQUN+AzyQdOm4YvgLxi6dij9n+lPu2M7438bzwz/GTw+7HEUNubgW9bGLFT+KhwDHE10gY4B4ubWb7AffoP9AFFzIPVzMyht72mPbbfoQk1wEs9vjEHppOTmrzeRyMRkB87cxkoaQ1HifMaZ5aqAB3EuQYD8ffkYtAbGvD8Gz6sK+hn64VvhS+I1N4wR1fyxPJO4+XFACNgLMAI6RsDAog18NHkoYX5qvLy8MNaa35H2/e3cabrDvfPiOjD2X2NBkFDjtB9PZV907QPJ3pZN6dlSUp5PsQgo+t8+zI0zd0l5O4X1FaKeWXbnUWAGTV5RGJ1csHG1IX5p/F+SBqTM3T4Xv45Kwt9NxcFX3Ij72AYRUPcop/uG8tkzJ5mc3kDRkSLTe9Njth22jD41GmlkGYXtEi68f4GJQ0V9RwkSkVJQLq6rPyX/JNJTThQb3mx/hfXJ2Xy9/GsufXuToFHBBCeKAU+JIMG1wRVNg6XuT09ww6VeTkT6VhoLpjBurXnubNE10KasoUvhiI19CLELof9SMei2e8kfhJQEEX0nGomvOLfbuDvg5uXQ/dtiu1StKp77/CnqI27xTImYQXrqjVOmqqENaRvo0ncR4x6DPfYsv7IcBx8Hij/q3rAa6xm+dji7l+xm9PujSX4+mSYBVF1xeDRP5mbQVq796xpx2XGs81qHrlOHO+7E3o6l2rsag0EgpZd+WE/Q2aHNgYg8B5qL6tGrfZEqpEhlUlO7az1q2TN9D/M6+/HzsJ8Z/+l4jDJzYOXApJ+4PfBpljv01jU0D277NnvayttMmFr0nFg+LxQr5sLyw6jMqCR4TDCufcxjQCLI0Sg11EZ44RzsTMGBAjxiPXAKcjKNT5leDecvkPxCssmPqa2XcC96An0a12Jss0bSHwttiJ57MsokGOXWtGUV8Z7tetI2pNF3YV96Nqeu9a5E5UbR6ltH9olyLrx/gVV5q7i35p6FoD3Ajuk70HXocBk/mAqHPwBwD3bH31OsOB3z3hjelryNPSBvPk5Ibj9aSpogxRwMyG4/Rad1idhGDP8WA48oiEAhv8vgZwdz4vMTfPX555wz/tfUNUsMer7OyMBReoM1Rr0oWt8Jv37+MR98/D7aMQYCIgN4ZO8jpmCYlYMVSmclvbAAbD3scQp2ouxsGaffOM2Y98egM0vQPAD2zbn2NCs+OEGDWwP2XeLvri5cbcooH238gJLmYi4N2UWr3wJm3hMTrWIXxGIfKk6sEqScHHOSqIqZLF43hLvr9vKpz6cYPzPfsaxlN97FEiquRLFzojg+ewdRU4+kcuv6LcbMFRPBjL00z3ra3BsQuaD9Cp28ici8SJR3r+G6YqaFVlaG96MAZPazJSLuDZ44OIp3z77L+K/HM9Y4nciCSLyrvBl0LQlteBurC1abztW1d8INuH39OlmGy5xb8xOFBQswGAz89NkHXBSEfwvaTv7ydxgFpMCtfreIWhCFS5gL1767hvGa0aTFqTKE4FUkp+r1KtLHp2P3jh3hY8MxOsiwaa3BqbmUppIIjj57lKBRQSQ9m0SjoZx2mzs0ud/Dx0Mg47sMzr17jifvPIlDsLsJVGt1bMUoV7J7/m6qrlexpmINx148LiYYSaWsMBpMCUYbpEouCTokIxwgpRkAjaIKzcnzfPT+ZZZfXk6dUz2F3jvptC4h6UoaleWV9J/Rn846sVoZqdF0XZ9EHwugD6DRXgRuh50O5T2rd1hTuQZbD1tG/TIKbbstrhyiJiSZmpBkPPuCf4JZJam3LlDodTn5VfloO7RM+noSca4DGZNdysGBCjYurSCh+ARGo1i5ZDCAlcEZDM7Y4sK2+duYLtnI6TdP01bVxtQNUy0CWcdeOkbysWQ+8fqEF6pfYE7As5QfWsjtgNXUep7F+nFrLj9/mZHvjMTG2QbP5mnYaANJ3Sfhp4yfcOvjRu6eXEb+YySGXowYeoWAfbA7s3fONmkQ9wT2KiNGUh06BCuVkvaadqxV1hY+j9FehVoPt/51kCu593gi+wlL4KvBFZvWZou9bc897cjZgaZOQ6KrpU7QO+feMf1/2PlhpFA/iRgAAQAASURBVP+Szpta0b/75huw1YTh2TQVR008mT9mUp1VTcLKBGzdbEnsfIuryncYd2wc9YUZ9PltvOm33JU+xNz7gnbjp0w46EtldAHGWPNLsrdzDYU+p5AYJbg0uuAWEc+gVc5sF+UtGXW7gHqHUxS5zGVgZjQdia3YpIjjoGedclY6813od9x57g41STU4B4uBxx76sE67ThPYb3qGvcA+gPCJ4YRPtAQKeptTsBODVg36b7W9AEpK4P33AWTs3+9Ae3U7mjYNBq1lTEBVW8TZt67Q/5H+DH99OENeHmLRzyPfHommVQOCmKio79Jz8u5Jxm4ey5jgMZx4WOSlvrHpBgUHCrj6j6ukV6Wjs9Jh3WXEaIS5e+c+0L6Ex0VK4Tu77rBz9k5mbp9J7NxYpFJo9I2jyTsaJ5mcX3pVtuSerCJvbzY1njVU+bSQUO3H25K3cQqJoTJ6Fj1rb45iE0qnqUR09id2Xiw5FaKvcajtfXK6cshJzeHlsS8jt5Yz/M3hVHpVcq3OXGL+5TdaUs8dorO+0wT2fdINSFe77H4A7Ns0fBN96IPWSsvCLQu5v+IKLccysXa0JumZJFMiWMagDNQej/BhPx9ChvSmNjW/k4LEQNWhG7QXVRNR+iv1vuEYcKDBtQGJUYJEMGLvZU/svFicg51ZFbmKKRFTSNuWhkFmwD3ZnYdPPCy2tV2k8ld2KQm4F4DUX8rjmY+briVH3GOVBJdwbVISLy3uy5GbR1DXivERbz8txhaoC0wgZmQCJadvEng6kKyoLJb/uBz/+/5sengLucN+IOwvDK+BcvHdbnZuplZtQF/TwLappzlQm0ireyjnovujVdSyaPMi3FrO4DY7htorJXj28xS1uTvE99Op2QnXS660VrRScrqE7/K+46SfGegLKwyj5t0aquZXIRwQ8BznSXFYMQMiBrBu/jqsbayJnBqJe4wlC07PexmWGsb8/fMfAPGkctGHPfrcURQ2ClbeXMlLDS/xd1aVWUVHbQeRUyNFSkYJ5ngemNYcg8ZAR03HA2BfxdUKLn54kbQf0x6Qe3CLdDO971KZ1IJ+8+ZHN5nkMJm0ziPs12r/PRuPQoZNqi0nHE/w0iviPTy06SFTZWh6RTp78/ZyN+suFfcr0A/QP7BOSg1avgz9kvhH4kn9LJXanFqcgpweSI4GUb/z/x+ryqyioaCBiLQIjBLxPisjR1EZOQq5EouqaEEAJBKyxr+Ik6ucO7syOP7icZZfWY7fYD+KXL9CLW1g4j5fHBvqCd34Clot8KJZs0+ClJvj1oDcfK9e8V5IXpRwX3OfZIl5UDt4OzDyrZEW7R0fal5jAj4U/bsNOyoo9HkXiVGOHCtcat1J/CSRAckDuq8pQ2aQkXR1MNe2F+OsMpC9JZvk55MpbCxAJ++hZTWiU+spOVWOyl+Fa7irye+0b7dn1u5ZtKS2MWdXd/VrmZkpRdu2ioRLsbSUPsnx45bjXiqREpcdx43zNyjcUMjM7TMJHhVsUUHYY7l7c9G0aHAeFW/6rGBvDvcO3SZ0Qij3zt1jxD9GAOZ91+ivZzB3xwy2TtmKV38vAoYHo9DLaXRpJH38CAYmxVF6fC+3Nt9i4lcT6Zn7gkuCGXhVgTDAlte1r5t8fIXeFWutNxrrKq4nXGeoly3Fx4vJ/yOfhDVmWvrNtzbz9OCnAdgycQtL8pbwy9JfTMddwlyY9O0kAoYGoDfquddyD7lUToCj2G8tra3/R/qXWrnZf+i9h/n/qllgGP8v2H/Avv/Y/wrL+T2HiusVjH1vrCkT0LbTFn1dE1KZ1JSN3zuAo7F3xTVC4Mk7T5r47nuOV0WMpCpiJB11FXwZ9iVjPxzbTT0gfqHGswZPgyUVQQ+NZ0/VWO/AS8DQAH55tpRS1S/I9XIUzftou7cfMAuztr9qGfwEuPTxJaoyq2CpuMlxaR/C+KXjqe5fzcDHBmJlZ2UGiJqdeOjPqXTJChj18hCz/yuY00YEDGhbdZx79xzJzycTlhqGdTfY0OoRhkGhxDvRjydznsTG1Yaq7EYK994m5G4IcpkBqV6LT0oQbr32SlJBgVFmRK3sQlJVz+bxx0lcnUjaj2mcfessu/x3ITPIaP2slZy2HELHh1KfX4+VvZUpE0xjreHcrHMsm7cMqUx0EmVys2ZfdkQG7Y9NpWRWCe017SawT22j5urgqyz39+fmL6epul7FkJeGkNeayZmYxSginBkQtZ2kgTbEzI3ptckzT6hurWPBKKP/8v54D/QGKwnrWj4hPTqdMmcvgqrN4rrWDtYMfHwgwWOCiRkbA2NFweS63Do84zyJmBLBs35vsO3KDuavnkvaj2kMWG6mR+jpK6lBh1Svxai3+VtNSUGqAHs7Zm4dT+CwQDHr81YZQy4Mocm5iTMj0lniu4qTGd8xQipnsPG/Xi0HA0MkEi4rL+FV5YVPcSkdddG4+Nmy4uoKi+9WZVZReqYUELXP7DzsUCjNm9fejnb+/nxub86m0aWO3ID3cZR8APiR/lU6ZWfL6L+sP/3sUkko/IO+V2twVJ4ExpmyiUF8T/TW9jT5RBPgBF2NXWg7tCgdlRgaNPRv6E+NZw0exT+jDxqP4xxHRr83Gv8Uf3Yk7iDJaSqnX97BwMwY1Is7aasUUDopCZ4UTMmhEvL65FEWVIYyUmlqf/RMc6WKtaM1T9x+wkQFN/SVoQx4dACPrhX/bnX0Z03FGvY/vp/MHzL5NuZbRm19lM8vRNPk0oTEqgNBgIIDBVz//jqTv5/MlsZ8CjSFWGmtSC64wK7vB3H6iZ0UbSqiJaaFn58tQRI1DgkCgkQKyEy0Mn0X96V5XyVeLQsojtTg4CxmDraUt/Bl6JcYdUZCJ4Sy8PBC+j/RH6VSvK/qG9WUXy7HxsoGVYuKzvROusK6SH4umeTnklG3qFEFOKN286G5GZyc/n6saOxcKEh+mAFDofRMKbd+u0XMnBiyZh3gRV4EwOAtgDkJlZAxITyW8Rhp29LQ+enYkrwFQSeg8lOh69RR21HLYwceI7wgnIatDeT8nEP80ng6ajsov1zOn8v/BGDKD1M48PgB5h+Yj2OAI5o2DeWXynENd8U55MGss+ayZvSNUkCFxt4V3yGuyKxEWg2ZQsaw14chvXAep0g5XSpPDAYoPl7M0eeOMu7jcXgOEQND1u0NONYV0X4vnBVXV4hVCXojkbvtcLAfzG/Rm7nqcB7F5u9QdhkYXzUdte4deibZtD/T8KspIa9EzaWMu7yuft2iQumDtR+QOyaXzWM3mz47JBGDH7f63WLkewvJX3OM699fJ+X5FIt1qj5gAKMei2DAC2FUfSxWL3xRPZMpCLR4RqCzjUDla7mROxERS5tNDlqr6Rh0EuKXxMMS8/vWOyPYurORLRO3MPo9c2U0gJXWioTrCaht6xn04SQip0bS5C1WUEklUtQtahryG3AKcmLIy0Nw8HHAWLvHdH6HfQeaDgPn1h7m175pbLzhzqimrRx3fYhPXvyEPxLe49InYlVYygspJqqwKu8iLg/X89CgQIv2HGz6hLOxH4jjJEOweEbhaVFc2XOJcv9yQqo34tgQS1fjNNoEPfZe9jjLAvBpmEeMTRR1/vF0OZs3032m9QFBBLRTAlPYmLIR6zRrRr0zCr/BfgiCQIWbud+GfTqMozePsujwIgiFJpHZEAlSjDIjems92VuzsXW3Rdcpzst5ffJ44503uJ3xJyn7zf0U6hCDUuPP3ZC7bFxZwEiVDe/ZvMf0zdPpu6ivaQy1qdq41e8Wc8utaShsQN+lx2D7F9o7iYELT+0gz9eah355yCJwPHvnbA6uO0j1ymqGvzYchZMtNl3OdCmbWLhtEefazllkB7doG+hQirThBsFA7e1atqVtI/XLVAavHtwraUWF9vGnSHneGl2XDrm1nNLDeQTcuUCbWxAGrTWGDhvADEIGMATPpofIiaum03MufcJ8CRoZZEoA6gEA3OvcGXtyLJ0JdcQuiMUzzhN7b3u0Wi17i/Zyas8p4tvieeztxxiwYgAGnYFL6ckE3nsFtaKSaFdP9Bo9MoUMiVTCuM3zePWnx7nZ7yZnh19gWdKrFB8vxqg3Ej4xnAadmI0fczuGiOw87l3w48+HdzHqnVH0f8QcQPpq9Vc8Xqdn4GP1/FG8j5HHFQxW/3vqms+ArcBFYLDREkXqAZHGn4R73GPCvQn4q0SActLXYmbC3r3m79t62LPw8EKOrTlG7p5cgkYGodeb6er+CvYVSoq4FDeHhGsJDLk4AsOrtVirPCy+Y5oLJAYCp/Vj3Jo4pAopzW06Kp33IUgM3Op3C3cCKfzlEjmbb5HwRAInpeYXMPZ2LHG3FVg7KolbECdSbfZKdjiSeoRvJn/D5U8vc+/8PWxWR+LSNgKHhlKOPvM05RURLJ25FLlSjtJRiV4Q3xvfCl+U6SWom9UPBPoBrg88w3LPUwC8eUYENQZ+FMFQdT8uJV+iybkDodfm+bN167j3xWGGy+AxfZ0Iula28P0/3uGyoBf76O86sfvzgzo9Q46DIQ5wEMEGo8HIoScP0T+2P3dD7wLQKrvL2PYZtO1vQ5usRXAUMCqMSAQZko6T+N4qJOeajsLDhSbatpPGN2iyv4StJpgg7WR8hzkwaNUgbJxtKG0u4cgAca1ybHbEIG8makYUPoN8kCqkJJNCLHHsSbnIi9fr0Ro78PR0wqBcwMiAFVwYlIiOZoafHU5MbhCSFKWpyv52bQ75vm8AsHXBVrbM3sK5kWKyX9zCOCTdiYaxpZ8zJXAKFekVNBQ00Gd6H6zsrCj2FufkSu8CQic9xOk3T2OTYMO5SvE3UqWdyI12fzs+e9bPq8kFOPSfy69fzTMFqXrmGIkgp9q7Gk2lNRc/OE+Suw8Xq81jXiqR0qZqQyeRU3KshKZicY3qPf8ZBSMecR4mX9vdxgtVlxc2WnEd0N/Xo25Uc/ipwySuSsS7eTrezdNp9ThFnxmiRs7It0eK3+0F9jk3yOmsbiVxrtmf7LlHg0KJQaGk7HwxO6aY6agFAeodTlLucBTbrv4gcTS1qzdd5qZlm5h0eRV7F+8lcnYk0VOjLe5p3/R9jJk4hqaSJooOFxEyNoTGrkbTceu51qT4Wu5RfZpm49U8jQrfb2ia2w5ZkPlTppiQ2J0A2fdWX9rr71r2Ujc9Wr1bPafG3Sc53JP6P2/i0uBCo2sjFcZMWm1vYtNpw+qvV9M84hzC+GlYmhTXBleSL0dz/+gdmhNEGv+ee4rzjGNi6kQkVyTse3gfy84vw7OvJ88Ofpa3zr6FQ6sDdfl1HFh+AJm1jCWnlpjObdG08OUnXzLCc8QDGuSCIFLSH11zlCufXUHprKRgfwEzt800yQH8OvZXPPt5Yu1gzc1fbrL88nIqKrorCw166vObcfR35Omip/mr1YSm8Paf/bH3tDfRqPe2oJFBpv/3nD99hyhncLLkJNoOLUa9kXEfjiPujTjeXPcmeEBkXiSjzi+nLa4O1+R/LzfgFuXG8DeGm5IlBEGU5BCkMhqb4f7V+1xff53EpxIZ+I8p/KyLQWv9KjKDHIMBomdFc7m8R+9ZHNv2bfZYNR/B4DaamT/PZFO3fvn5rvWm6xoEAwpbBaPeHsX83fPJun3R/MwlOjrrO/9tm/8K9k34bAKn9pdwOWw9SrUdMe7dOurdiS7GXs6enSbsgXmkJ4jve98Xh/pqE4AsSOXYtbbgXOlDeOWrBOVcI+WyM8rnbJm5babp/DjPOH6e9jP6ND2zo2ebf7d7fLnVu7Fg2wI67TqxesQMFoWr+uHQFUObTQ5NzgqsHUQ/SGKUIEgF7FRa2np+ywgZ32YQkRcBUeB/Xxx7OqtO7np8gSA8w9WvrmLrZkvc/DjirWbR/+4WGhyP4dgej0HWQdGRIqyiRHaGTivxHdVa26CztsfOz4Xpv03vTpiCr7eLVTO3+t7C5yEf3pz8Jp8+9ilWCitYDqHOoRQ3FeNd5U37qXYc1zryctPL1J2p49LNS3jIPagsq0QVrGLmVvOz6rGa7Bq6GroIGhlkQXt78u5JtmRv4eM9H+Nq60rR0aJ/S7/ZYzFzYkQKx5FB3XpyAlqt1rS3dQxw5NmyZ/l5+M8UHS16gFrZ1s2WyKmRFu0AMZFA26bFysGK+/cltA2fzMJPzcetVdZMCptC0vShDH39FYZKFKwwqk3JMr9YWXPOoCfx4SQu+F9k2+1tpPinMDRgqCn5qEvXRfJPIqiVej0Vp6tOZHlmUX6hnFH/HAV00xFL5aS8lIJPgg812TViwvLnE0za3w0FDVxad4nYebHic5BYPrOWey3k7s1F26bl7Dtnmb55OrFzY+lqUnP27TMEjgik7FwZVz+/yrP3nkXuYqnNqNebWbzEZwNIJBisbDHIIHRCKEonpSnJ4q7b13QoC1AOjSO2+BkEATq68TArvSv+dY9Q7r6RnOCP6Vv0LDc35xM0MgiZQobwscDgwYO5E3+H7x7+DmW7kmV7xIQ/o2Dkp8yfkEqkLOq7iPSP0yk5WcKiI4uQyqV0dVOly4x2IDMwd8dczp85z4CcnjiahNjyrZxNs2PtiuEk9tOQ/HwyMoXMlAwTcjeKMecGUpdczpllvzL42cHdTENGkIgVwjvm7GBo5KOm52HsNakUhGdgpXfH7/PTOLU7g8s42pQ51DoeoV3bzvrH1/NH0B9UbKmwAI97+qnmVg2+g3258tkVGvIbmHE53nS8saCe/D/ysVZZk701m0FPDaI32LdlCyyf20bhwUJRa/LpFLYuOkalZx59K6QYjTDl+ylM/HIicqXclAhxI/4Gasck0txckSnMa1Bi0X60siZOxQWjl7cQbDWYquslXPvmGpGLIk3fO3/0PM1Dmhn7wViUzkrUNpZVerbetgx6YhBtVW2k70nn8e2PY6e14+Lui8gUMhxVKqorxcrIauBH4A9EZntHYBqwAvBC1L+0dTTf838q+0TrPef937b/gH3/sf8Vlrcvj5ztOYx6ZxQ2mmBa7DIpjxqF99Ln6aitp6mkyaSN0mNyTQedDVjodv01m1vpqCQyLdKkoaMwOKPQO6H1/gBd3CCKjxejV+uJTIvEQ+lPYO0qOhQ3CLq2DiunSHbNUxM7P5Y+0/qQUPst4SXruBocwrwd8+hoyqEyw5bTb54m6dkkQseHcrn8Mn/8+Qde17xYtHYRY/41BnWzmj+PHATEgGLw6GCCRwc/0OZGl0a2zN/GmtjVxC9xMR1v1ol0UwnXEkg4+jUdsxbw5J0nTaBNT1JOu0sA7S4B2HiAs7MIwg15dwIf8gnl/nd5bMN+dDZXEISVpt9u1ddR73AahVaBR40TNDRQcbWCjrkdRKZFMmvHLL755Rt0xTp+XP4jh1cdBuD7ft8TOi6UWftE2gKdlY785HwChgTQXt1Oc2kz+g43jBKt6Vp6o55FxxY9qP/UTTOUtiENfZe4apS059BucwdsoM32NjZ+qUzo4asHrPQudAJRd6IYdWkZjNAz9UeRXzt0YR8WfDwfiVFCtXc1duiou1NHY1EjwaODmfL9FIyCke+ufYdRMPJI/0cIHmXuj7zWDBzdVdyfo8E/2Z8rX1yhraKNcR+NM/WVV9FFfArPcmvLNG5uuknyC8lETI4wHW/xDEcxPZzAYaDX6Om7uC9VG1sZ99k49kzfw+Wh13nE043LN0/w7H8D9PXYCsHItbudhDkMIPp6Ds1lKbj4PcjHP2+fOegbOj6U0PGh3Dlyh/Sr6WKf329F3aLGI8aDzB8yKTxQQFliLhJtFYPObKPuzlJG/3M0XU2i9oujzAuvlql4Vn6BYFP6wPV6v3OCAB+5fgSIAt/unw7kwPZvkRqlBBTdw1hTj2OAL8NeHUZjUSNvS97GdYg/RsflFMQLxBfV8OnU35jw+QTSdqXh8pH5PfgrJUuPSWVSU5UfiBloDt4OCFLL9vUAxS7hLug7tUw5OIsLQ07gpluC0QhBI4IIHh1Mn2l9MP4sOpVaay0Gmei0Fx4qxKg30lDQgGNVFjcjv2T80STK4iYDCaR/k87RZ48y7895ODZ70F/9E20hctrjxOtf+fyKSd+sPLMcpw+deCnmJdZOWotMJqP4eDEnXjqB7Ye29MnrQ/tn7dTE1pgq/xryG6i4fI/7fSKoGHuMtIUODHpyEGXnynAMcPzbbOfanFqyfsoiYkoEylBHOu82kR2XR6BCBKruXbjHjU03SFiZwO+v/061vJqMQRl06btwdXRl5Q1xrihpEitmWhxbyB2SyyORIn1Q9Y1qdjy0w3S9sNQwFh1dZNLJaipuYkvqFka9O4rhrw+nMqOSutw6omdFo7BRsHnsZnQSK4h8HAQBnQ6srGDm1pkIgmBK4rBpq8Uv5ygNp30g1VbUGxLMfWvbWk1AzhHaCu1R2JrHzO9DTpB/6QJDMnu0uNpEWrfTmzmj2ShWPaSIiSVWaj1+0waSMD+se9yYB7bWWot7nDux82NNwR8DGvNxvZ7UL1PRtonzXe9gRpNPDOGLwSDUWvSNgFEEmB4IoIJRKv723hl7SdVNtDjWrmulzeY2Mr2Mgdf7oexoYvIPafgN8sHzlzRqnPab+urDlz5khfIyKj8VKj8VzbXNgMjjP/6V8Yz5bgxTfphi0okwfmxujLJLSZdSSsHgRXQ5eCAI4K02VzEbjEayfspCIpWQ8kIKdnJxzWlTtVEcZaCroZMfk35k+BvDiZgcgdoobgLHnBjDwIvvUHZuCYJag2+iLyGpEeSN/JAulxa8L1bhXq/g3Iv7KT2Uyz+EfxCuTGZASTKJiZDeD+wknWRtzMInwQefBB82ztrI4U2HCd0RirZNi16tRzAK6DV6BJmlU3ByzEnmLJtj+o7BIOeux5fUq44j08uQdEp4Kv8pHHwdsLKzoqmrmXfff5fC8EIiQyZwad0lGgoaSFufhq9tKKE1L5AT8Aw6Kz32fk7ELYzDKdjJ1Me9TZoSwAs/iv15P6MV+64+tNvk4VfuR3ixFR3Vzdhai8+xJ1A1wHsAc5+eS/radK5/f52ElQnsSd3CyznPsHHZRng4jsRF8RbXadLVAKJWYML2y2TL2pj28zR8B/dQqnaPQakMXF3ZNW8TZWfLiJoZhc/yQZwuuYTUuIkRRzV0uE/g/mA/jDojAUMDCJWMw7U9mzv+zyNBijL0N+a/3ou+qns9Lw0qZf1j6/k8QQz6evb1xGAwkJ2dTWZtJpWHKvFP90f7gtZEy/vVAqh03klm6BzeefRNpJukTP5uMrm7cwl5JJGzI88CIDHKkVorOP6iSBsYXhiOsXt9CCsKIyi/HARw9Hc0UUv6tsyiwnEXgkSKgISC/QUc2b+bf/LvqWuqgbXAOf5rEOmYAYYch+TPk3l/2vu8MuyVv/2uIIjr0JCXh7BpxCbq8+sxGv892Fe9pYQ1P66hIKIArZUOT08JNbdqkNvIcQ137aaOlNI/sz8xBR0Iz5mz6Nu1rWSGzjH/mMRIzmciDXfq56mc2GDW4zs55iRlkSE8Fe7OjC0zxLZsEhuzqO8ips6aSmJoIqcfPU3+n/lMfX0WKflnaBRWY9tgRVNDE+u81hEzN4ZZ22eZxv2FoReITP4nhYcL2Tl/J1UvVTFzZK/Ao0TAoDNw9csM4u/Hc8PvBjZq0a89N/wcVkINU5xUZKzP4M/sfXz/4yd/W7lXIuix+S/6qHdfpcjgfCb4hfjhdtuNtsQ2Fp9YTPJuy9KM99a/R8i2EORKOUPXDmXGjXd44w1I7z+dq3FZ+Hn58Yb2DXPfdb+vY089SljhVTyfXWjSstRXVpv0KJf9vAyZ3QEGVZgpvbK/1bKn+lNSFE/h5PQZUoPAstQqfvvDAa3SicTCQ5R6foVNVxl27QoUg+JZulakBDRkm+e3evd6ZE69kgMNgolVJOqmRtT6HRNMyckSnit/ziLQlTkgiynPzeFzp3dQ3lVCd1G2UaLFrrERu5ZKig67cHD5HsZ9PI4ByweQoH2Rsvo6vJpmEBQZiLxXnMOUGCfIkWkVKDTtnH3zFI4TBoMilArn7bTYXeOecBn7NnukBjWPXHjERPHdmy4v/Ew43gu8Gf1PMaGlB/+VCDJULSr0leYB0XS3iRbbEi5FDkMZ68vKGQU4motKkSDFv34Z5W4/M+yEguuZuxk590nT8R4gQ67tRKbtQulqR8KTCabKmPNRA2mxy2T6nun0u2VH5LlXSRrWrZv3lxdYrukke0s2grdAnyl9LO4JxDFTc6uGQ6sOMX3zdNPnEa4R/PTkT8ikMirSK8S1Sgjs7g8dN73WcLfWgV2v7TIBXp2IlL9fP/U1X0QUsG/JPpruNrHs/DK0Bh16WRsd9h0U9enEqDNyY/UNQiaHIPWXIlGLY0YmeJAfZ09sRxsXn9uFXJeKXimCZip7KZluJfy44gArNoC2tILFxxZbVOfZe9kTlhpGQ0EDkm6/+ZmkZ3jr7Fuk7U9jw3cb6PNQHxOg1hv8bPq5iX139qFt1xKWGoZzsDOlpbB2LcydC/bdgKq6SY1LqIsFKNdc0oy9pz0O3g7YuNpYUOgpOxr4ps/3jHx7JCPeNNN99phO6YBHrLjmtpS30F7djmdfz7/VKe4xQy/NyF/H/Ubz3UZeqH4BrcG8340tXoN7dR26NjVdTV1IJBILBpatU7aidFQyY8sMPN4x71/s7UV6bevOJjS2zlz84CJ5+/LwHuiNMUFJjt8TSA1Snvl8JfgeYnbubI49ClSb6fl8K3yZcMCHtrGlCIIZPBUEAdsOW6b9MY0qRRV0y/h26cw6nwq9CzbaAMZ9Ms6UZP1X++s6lfRsEpccPam6V8WOebtZnPIrql7FTdYSO+y7+hCVN4R+OTLuHivi2oenSXkphZjZMdC9Xkw+OBnnlgsEvjaNxnoDBwr74FSdh07pgEI4Q5V3FXfDNMiUCtra4OhRGDkS3NxgafxSQNRYzDiRQeJTiQh24iTU6NJI69xI5i0NpfZ2LVK5FLc+bggCSI02SA1S0LaR/k06imcVBD0cRElICTqj6IfbN5SiPV7G2I/G8tPhnwD49olvMcgMdNl0EXjjdzQu0zm36RzuMe7EzY9DEMC3cQFJ1/3wzDkNLw7jDe0bpHUDrz19VRf1Mp2aUHJ/+Ip7jgr6LuzLzeqb/JEvMhwIUgG9VI9EKmHRyUVErReTR4ubigG4knSFOWvn4NXPC6lcSqsgaieW7C1h95LdBHwdwLJVZmaGHjv12imKjhRZrGEAs3bOolndTF1nHfvn7ydsgrgf6pGMcApyeiAWJ5FISFydSFVmFU0lTagCVOTn5xMXF4esm+PQzsOOlrIWTrx0wgSQ9ZhPgg9z987l5uab7Fm4h0nfThKThbUGPnD8gH4P9+OnxocA6OyER7qlm0e+NVKkY9XAidOL0Z15hNeM1+hQSHByciKgzo+ta1ZxYtQJLly/iO6Ejj0f7SFsd5jIGoPlHJQdl01EYgRtFW1kbxWrzXpMkMoY96EosdBa0cr4dWIid4+1V7eT+UMmHrEe7Jy9E8++niw5tcR0vKGggaPPHqXfw/0ISw3DwduB8+dh/cdawveLkjn9H+lPwNAAbN1sUXcvaQ51d7FtraKrLZEjrxzBe4A3CY8nmNZXZWstMpkCzzhPPON6V2eK95XdNxuljSdX3zxI8/0OUM1BJtjg2JlAORuRdRXiUpHNvodPM++PefgP8UcYKVDtWM2NezcIvhhMqD6U2pxaDq06RP+V/Xks/zGx3xx8KMosov1aO5c+uYRRb0QX5ASATFAi0MXl5MsE3gokd28uUdOjuBY2lRa7DAYVHuDX320YP9XGBPL2AF++lWF4ld5E2xHNpG8nmWnDu+9Ja60lNzqXUWUGDq46yLiPxlkk2pQGl9LlVMmI3VXY2LSCCzTbXSPX/wWi3KI4vPAwPg4+KFaK81v61+kcXn2Y1C9TkVnJOLjyIIuOLSL181S0HVp6F4P3e3oEkz8agdFgZMr6KSI4esd8PP1QPc5/ivFej1gPrB2VlATlYaUWsK/YRmdRMApbX7PWbne721Rt1HvpMBrEuIxTkBMqPxUAWSHzsdLoeOLb1UhGFpH4azLxy+Ix2BvgqHjdU/WnCBsUhnOoMykvpHDz9E1x49JtBqMBuVRO2bkyjs87zlTFVKx0VtypvENcYBzTFizg53feoVavZy0iAcaziOBeNfAr8C7wAXBALic+ZQFd3Vun/0bm7/8TZjQaH5jz/m/a/zv1hP+x/9j/oY37ZBxj9o1BKpeaKvsUBkcEAQ6vPsyvY34FLIGFPhd/YvPYzSI9R6vG4riyrRbnyhzsveyZu3euSVx7WO41JtxowrkjEcEo4dRrpzj8lAhguSh8cG8dS4tdOvYtTVg1VJOzI4f63HoKDhQQuftrvErL6LTtZPeM3SgGh9PZ0EnpmVITp3hWdRZHjx+l5ZcWGosb8U30FYXGexw71THWZ6ynoKGAk6+e5OLHF01t1ig1FEYUIHM3B6t731O7fTvtzp4o7JW4R7mL4sE8CHDqNQY6ajvQdemwcbcnNyafJpcmMvuf515QFwcXbeOb6G8AKOpK507Ac7g2uLL8p3lImhpZ27KWAcsHUHOrhju776Dv1GOUGbnvfx+6fbyha4cSPSfaYnMT7CwCZrl7cvkp+Sd0ZVV4NYmBI9d6V4zbjShsFRZ0Tw6tDjz+/aO0HbuMc7Cz6VjPoq9qUeFc24iu3RxYBxiad5XA2pWEl4zH434h2lZzwO7O1hze+OcbhBeGgwASo8Ct326xfdp22qvbSd+SzonNJ1izew2HnjrEzd9vms79Y/kfvPX2P3BpdOH3uB9xj3Ynf18+WRuzLJ51h5MPtYEJSOVSarJrUDepH+gLoxG+jvya95Tv8S+HfyEoJGybt42isCIERH2U/9PSeUEDt2Nv8+viX3EKcUXXqePCBxcoPl78X577xog3uLz8Mrmrcjny7BG+i/0OgLEfjgUg5K4/OoWOZlUVEpkE7wHeJjH7nnvKHvU05amPYtAa2DhkI2ffPWs6btd0n35HP0KSmWG65q3fbiGLVnF40mHSE9P54pmfkMSZM6irMsUqp4aL5UjsptHq9xBWHs4kPp2Id39vU1WMX7kfKzasoO64GMy4d+EeH7p8SOZPmd3XF+io6zCBk+oWtZiV2t1wqV7Drd9uoW3XEjUziolfTaT9XiOhxb4s+XUJk7flYDCI2YUj3xnJviX7GPqWHYPSB+Fd6Y1NeytGI7za8Spz985lxJsjuBGfTpl/Njf63UBtJ2Y4uUW60eehPmyfup2+p79E2nWbdmUeclsxN/TKp6JQw4wtM3h95evIK+Xcf/Q+13+4DkDs3FhmHprJr02/UhpUCs+Ba7grjUWN3D15FztPO8riJtPiHor01g0K/iygq7GL38b/xrVvrwFiBrRc24lP/inabxUz8NGBvNz0MuGTwwnfNJZ3/vEOB9POUOQtVmZW36wm66csKq5V0HK0haEXhrJiwwpufHrDYvz0OKS1nrVcnHkR/2QxwOQR68H0zdNZeXMlL9a/iKO/I6HjQ7FzFysBVP4q0jakET5ZrGq4+etN9j28D3Wz+L4MeHQAvpPFYIR/zmHWObxDR20H3gO88Rnog72XPboJk2nw7YtXyRVaskQa4VW5qwifFG4am22uQeQlL8WxbxA12TXU3anjs3XrqLxyiYsCnNWLhYxjEP89qlZzQTDgcFKAS7Bj3g7OTQrEbUgkiatE4GGw32A+GCNWPXhXetPR0MHof44meIw4z/VkI7vWu3JvZw5yazneA7z5O6u/cZ/do3YTmWfOxDNKdHgVnifk1E80Fjey9+G93Nkl7hp6g0QCRs7/6zw7poug6s2mC1wLn4JcL2fS4VRUdUXELxuAV7wXNpogAKx0bgTUP0lA02vYyr3Ra/ToOnUWwcZsq2wKZxeaNPUAhjksoc/9D7DSuTP61GhaHr1Hp8oTndJBDPAYrAiqWsnIi8/RnlvF4uOLTbppsS4JBFc/Z2qztktPW0WbqTqup/Kg3q2eZo9I7l8uZ1vaNiquVZirQJCye+Zujk0T8BsVTuJqsS96gks9QV5FRzN/Lv+TwsNi9VrH+A4uxF7gyxtfYve7HVGvR1F9s5r3bd/n0ke91NOBq0lXCXsojF1zd/GB4weoDWruBDxLrdNBXBpdSHo9ieyt2XQ1dJG3L4/FGxdhkBu4Osob35mDKTlRQvaWbFO77NV9CKxaRGhJEkpPFTN+m0HAkIDu52Cew9asW4P+fImpHXZyFSNzcomseJe47DiGnnEk5dsFJl2VnnODnIKY8PwE1ras5cW6F/GI8cC6O2ho326PkBRAyJgQLnx4geMviQBOz8ZXkAholXJcwlzE5Isoc/WA+B8jQmubSY9v4pcTMfpKODv4TbL6fEjGoDKk/l4cWnWIPYvMVZ8IMnwqfIjNUqFttGQ0GJF/jfi7v6G2UVPlU4XU3gwoFDQUsODMAtZdXsfl5Mu0f9puqowCcf6qV4n38HvX78QtjEMwClSkV9BZ045cJ0ehVaBqUaHt0DHuo3GkfiHSAvfMUcfGH+PM1ER8kvxZfnl5dzARApvEgFdYyXDkNRWcePkEajT/5fr7I+Km938KIjmfdMJQKb5jt7ff5tp31yxoPFvutfBN1Dfk7snlxboXSViZ8IDP0NuqPstF1abiVt9b/PLIIWwD3dk0chMHHj8AwCnpWipdtxFyN4SwPC3qunbKL5ejblZbBLEW/raQqMsnkCpkREyJQGYlw1fejwHFOwiw6k+zczON7gb4G/2xhyIfYnbMbJyUTkzfPJ03DW+aREruB2lY98I6JMMkDFo1SPR1sQzKGGzl6NV6aipr+PLyl/T73hx8Hn52OC2f/syRZ44wvEhck648d5uvV32N2kaNgBF9h4btK7fzwbf/Yr9G87d98Qemwuf/1h7Xg102+N33w3OXJ633WwkaHUStp2UyhtJGiZWdFVJZd9VW99TZaa+jLKgMqYPl9rrneUmQgCBYJIvoe4EDGQkZ6PtZUhWOdl5Gv9JfuKT7msP9rVBoO7j1zEY8S0SfQaWOxaljEEdTj/LriltIrHrr14m/rdA78/uQSwTrxfUpaU2SGCzqBt9bnOX0e7gfyWuSmbt3rqiNBMgN3YmBeZfQ6USa2H/2+6fp941SNU41+QTkHEHXqcMr3gtbNzHJrI92ESE1z2Hf5YPQ2UlVVhWV18Xs8Jt1GZyPGoRe1s6qb1YRffUsSy49hsMEsVqtzvEod73EMo05v88h6EfRhxXpjwWLyvr4/fFk/ZRl+vty3REKvN+mweEMDm0O6HfocQ51JmBoADIrGV2K+xhknXQp7vPHH718QrrfsW62lLy+Epz7+fOR20dkrM8wHwc8iy8Rd+Zr5LZWTP5mMgFDAkStPmUBAMWhxeTEtSF0vzM7dkDhDU98GsT5O6AsALmui5da1xI8t9tn6PVO9rvRj64dXQQMCeDhUw9baMuPDxmPTCq28ehzR9k9f3evSsluoExqJGZpjCngvMNpICDuIwUrmfgMu2mm81tukOf3CjaaQAaUbME+zJPEnxIpCitCLpWbfJl2uzK2zXyBJqdmyg/nIDNoMUp0nI3uyzHfEWittTS5tRC9erRJD7Wnn86UnuGrq18hSZXweObjpgQ8mURs7+3Y2yQ+n8jMrTN5aNNDDzwPVsPAxwdy6MlD1NwSk1W+/VasRtm4EcZ9NI4V6SvoM70P4z4eZ6I8B3i6+GlmbJlBygspPJbxGLautqapTGdtx7DXh+ES5kLmj5nU3amjt0kNYhWbUW8k/at0fkz80SS9ALB7/m4+C/gMg9bA9Q3XKTtfhped+dr+YyOJWxhHfV495SfLkenF+232TyVr4ms4xvrzY+KPbBq5yeK6ug4d2nYtfzVBAFV9MdEXNqCqLyFvXx4gstIcH/ktT3yZjLXGmi47BXqFremc7rMBqPKu4tS4+wiCwIHH96NsqzMdlxlkhBaHoivV0VHbwaaRm3A66mS6vq0mFKlgRcrzKQx6wqwx2Nv+uk5pOjTklRxG2aU0Jaq1lLfQViU+x3jVWEbm5NL/zgJ8is7TVtVGR12HKbm3ZwydH3aekri+KFxVCEaxKlNVW4R30QXkgpTsvtmcHd9Cc1kzn43ez84vq3j03Yv8cuMXsmuyyduXx54Fe7jw/gU6ajtM40tto6dNNY9BQwaxLW0bexbuMd2HzKgktDiU8TtvUHW9Cu0YLe327d3PVZzjHRpK0R8/g42zDY0+YvVtrWctDW4N2HXYMf74eNR3ill8fDFTvp8iPhNjJzpZC2pnJxp8YkGp/Euspnu96L6GSz9/am7VcOqNU6h7xTNsO2x5zPoxOus7ce3jSpNLk8Wz11npMDgY0Kv1/LzzZ3Zf3g2Ifnb6oHSanZo58uwR0z6zxxJWJjDxy4lkfJ/B+3bvm6RqmtXNAHQd7uLWllvm63Tq2DlrJxnfZ/BXu3vyLtunbWdjyka+DPmS+tx6i+NVmVUcXXMUiUzyt3qcPVaZUUn21mx0HeKeQSKRELcwzpSkZtd0n5Itl+hq7LI4TxBAqfRilH4wy3iK8QMu89pbN/jgmw9JmpuEvpuRwUprhXWTNXqNns+DPufnYT9bJBFX+FXQNqaNkW+P5HXN6yZA8K+m8lWRvCbZYr/nl+zHSw0vMfDRgUROjSRwhCWzifdAb5aeW8qYD8Yw/8/5BA4P5KOPoMVoz80xzzHy7ZF49vUkema0BY2wc3Uu/rkn6GzSkPlDJiUnSkz3DBB94QfcM4480MYeMBlEf6yjooWOew2mz3rkWWZuC8O5Moe5++bhm+hLdVY15IFOKfbB5oc3Y9xkRNepoy6nzqLi9+2zb/N07NMEnA8g5/ccsjZmoTPoTb+f1rWLdz95l457HeTsyDG1BgHsWrV01LSj7dCiblYjCILJl8nuW8qNcS/gmRLKoCcGmfZTbkIUzu3mandloY6MbzMwaAwYjZbjSpAYmHzsGQLfXWF6BgAhziGoGlU05ZrfI11XN4PL3jyCRgYx7edpeMR44D3Am8BhgX+r2SeVSZFby7srWc3HXQsuU3q6lJcaXmLGbzMQBNDJG1G1qkg574H6TgnNpc1UZVWJDEQ9cSqDFKnOCPV1/DzsZ25uNsclm+3SESQCOoUOQSpS2dp72uNo58jD/R4GoNG1kQ9SPyBqupgM0JtiFaDqZhXfRH1DbXYtSV8msXP2Tn55+BeMCvG5TJ02jdN6Pe8jYoRH+EvMpPvz94Ezej3+Rb20d/8mYfk/9n/X/lPZ9x/7X2H2XvbYB9ojkUjoW/Yjsfe+xqGxg05pNoMWxRGRJmbI9p5UG/z6MWaSjnXe6xjw6ADSfkgzHXepzMGn8Byt95+0AJd6zLX8Bvb36xi7biz67hQakepEjs5Kx/dPHWdqzRV+/ln8wcwfM5FrOgm5+Se66Tqy+2Yj8XElbIInr3W+hsFoIG1bGgcKDmAVbUXEmAiLajErQYWtOpROZTGbXt9EY2kj2gItfoP9GPzcEHPDJCK92faHtiO3lovi3933lBeVh79mCsuC3WivacfKzgorezMNqNu9TNzLMsjbkcCpZ/eT+mUqNcXtLN4zi43LN3B8/HHiSmcxT+aEja3oYPYsgG0ObZwadY7Bno+YmnJg5QHuX75PwIsBYAsygwxNl+jI93B3N3WJC6a12prZ78/mZNFJYmbHMO6TcWR0ueB1dyoh1S+gbDiEfKec8sHlOAc7I+lFfanUOCPoDeKCbxSwcbExOekxOTEknyikPjGf/X+UETsvluBRwfTpA03lyziQmszJUX4c8FjLnkV7CB4TzK31YlCgVdXKkl+W4FNlQ8yVGNyi3LB2tObwIhHcVaxRkHgtkZOPnuTooqMsOrqIvD3ixsqn0oe6sFwA5uyZY6rIMlfuRdDiGUHfhZhoN3qbdXs9hrN5NBSIzpW+S8+1ZafoE9+HLpsuxh+bSPvsImykSqofOPvvrQZQaR2odm6m2bkZhb01mrZ2Tr5yUgy0jRMDbdU3q2mrbCMsNYzqrGrunrhL7LxYkgLEbDrDHAMesR4IgoB7tDtzTq3k8X/Opty/nOy+G/g24lOL61ZpijiQIAanZhU1IhgdaCppoqOmw/RMDDIrulSeKJW2vNL2CppWDVK5lO3vnCe5dBWXB31Dm0MLyBWUnC7h1GunGPz0YJRfKTnefoP27OO4t47DysuFMV+IlS9HnjvCW5+/RdboLDGg0ym+pwo7Bd79vU0BJ4BPPD4hakYUc3bP4cjTR7j5602kE1/FKFMg13Syd/FerOytSP0yFfcod268u55mzwha3UKwa67AYBA49/oprq8XN0RyJCSmJ5J6JJUGrzzATJHoHOLMkYmic13hV8GUDJGnLSw1jODRwbxn8x6CUeBa1BPEZ7jQ0DYN1j7G/APzufHzDfRheoyFRiSCBIVOYRpbjgGO6N30aNI11Cpr0Q/T4+DjwMnXTnLh/Qs8VfAUqrpi5Jp2bo96ijd2ypDIJEz+frJJb8beHrRtXfgUnqf99v+Pvb+Ojups2//hz2gmM3F3T4gbwYIEt+K0QGlxaKkBpe7udtdbaGkpbSktDsXdPViAhAhxdxvd+/1jJzOZtvfzPN+13t/vXc/7vc+1ukpm27X35ed5HMcpolBHWhHQJnOXTr9o63t9HupD8n3JVF2RWuHV5KukXEnB0GzA0Grg0neXCOgdgJBgL6nVbc4Bzlanj2D5+4pP66klfbFNBjd9cTrhI8KtCOeBTw/k9Gngbehw8SdyUjxKByUWowW5Uo6DswOWtAyar4lcGbGCGffZS25090ezg442Bx0KF/h9+ho6lB28nPcCxyzC/4iRkxHzErp2+zNjPGN4qM9DPHvwWRauWciB0gN0VHfQ2djJ8sLl1txlwaXB5H5xkqQkfxw9HO3GZICgG/s4eO9ptAFaVCG2TZwoM6I0daLqaKazoZOr667iGuJK/N3x1nE54XoCPm1tVDlXkb83X7qua0IwqUysnbuW3iWPWBfbGlMQru3pBNctJqz6QRzbanF01rD/6f2c++wcy0qW0fRME1tubWHBtgXcSb6DX4ofP434icD+gWTETqewCuqdD1EUXsTg0FEIrVKZLRYo1m7CvaE3Q/eX02i+gsvTU+zqQoYc/wp/pvzmQHtIM4+XPt6jsiTv25XUKwSbZxI9uQH3YCf8UvzsHJnNbs20dQhEz0gjpIuRIYgCJkUrX7cNYWBhOhrlImb/ORvPXp6c+fQMqp0q8u7Oo6GugcU7FrNr9i4GuA8gZW4KXoleYNs3We8XNjwMRy9HWoxN1t87tB0UjS1ijNsYDr90mCs/XaEPfWjPasfiMxelXMnMLTOtbb2uow6lxZnUm4tIO3aU4t43Sehrc5A9pDzDF3XTMCsv0+zajGtjJ9d/u97F7JcC4u5ucs73OU+jbwSTnBztygjYwE9alRUNOnPnHPze9ELv2MaZtg1ABrnbcmkubmbU+6MQuoK6HboOLt+VyUdP/lWyXPq/0tiJwxcfo4z1YsbmGZKUa63kGGxyb2LnXT8wxnsVgwcOtq6T2oUG9OoSwvPDGXDCl/bY2+x7qpak2UnSuNwZzeFeKcgtcpRmJaJJyjF8cdVF+u/qz83mm9b7G2ONKNQKtszZgspJxaGgT6nSSjmQSpJLmPa8BBZKXJLI++r3eZEXOTXgFJmnMymLL2DIA7HWd+p22nRqO+nUqewCVwAyQRoHx+xOxdW8ld4P9sZjswdVta38O9uGhG79n9iDZnjhshJhjwAz4fyX52nIbyDya1t7aKgTqLtVR92tOqqyq5h/dD5t8jKaHetwMPkhivbOJc+l0fx85zNKQkvQ6qVgwJCXhlglMZu4A8DOCTspTBpC+tE8dr4moZJlSa7W+4gyEVEGQzc/RueRs2yYtgHXcTMIaJyBv2M2Fe3XUBoF6m/VcvLH06TMSbFj7KwdthZju5El55bwTyaTy6yypWBru9p2LfK2Zvp9lMXRAUdpOtxkd517ozvmqlrmHZnH05eehhZojuygrq6O4JJgYvN6IUSZEOZaGLZBTT/DP7Mwm+H/GDR1JeUKoaNC8U/ztzoDe5qh1kD57XJEQaT4WDGG8DjAowtAJsNiFMjfm49LkAs+CT7WgMnpASdx6vcRJSdK2LdyH+M+H2cXuDox+AT9iOfAcwdor2pn8g+TaX3+IIP1jnRmxXDvb/dyJymPkCWjyb1mcyZ2B3lEmQVTbgGHXymh//L+VieXc2cyZQ9eosYBllxYYnUYyrpy9u0b8gTTJ3zLlN5T7N6z+74+5XoOP7WHsW8Nos3ZFsAXZHpqQ3vT7BPNstEBpN0Xb3e9IDMSmLsHze5sft3ghFKjZHnRclqMzTTrJMdwTkIOAU2ZeCf7I+ZYr7Te41rSNRw60ik7W4ZcIcevtx9alZYOk+RY3PPwHgYxiCOvHWHoK0M5VbubvMDPAKjxU6H6VsW4oHEE9g1E66XlZrAkV551tD9y8162XGqgYG8BLxpeRBDAuTMR7+axVIVFETqmL4rGBqsMfLdjr9UrHGQy1C4986hC94bsaspVGgN7U3u6gN2HW/n5XAaQRDq/YHbPYfihvgRUXkKuHtlD5tPmNcy4kIHhpAHtx1q7vSJAq7GVrQ9upfDPQsZ/OR5jq5EjDXAj6GkKfaVcrXdvvJuff/iZZ+qekcrWVS7vGm8Mmnqm/mRjCnYzDr1rPEg+8AkVvQYSudKHptImguRB1nfqtrZJ3iz41yNceAZEDLRqJXCLwqxA26YhfHY/UgZIazGtSkuoayjFzcV889E3NPzRgPJ3pRVkUdhYiIejB5rxGgbNG2T3HJ3aJu1rjjEzdMpQEmYk4JPk01Vuu9Npq2rj1pZbVvDu/8TMDk4Mf2M4d47eYe3QtYz7fJydT8Dv9nE+8D5uBZBpvbR2ABTXUFe8G7wxdZrY+cBO0halEXq/VF9BdfNIe2kgPj6w69FdnP/yPJonNbQ7tXPH+ys82gYjCHeRODvxb/Kg8w5L8ITCA4Ucf/s4w94YZg0odzr7UhY7gk4XH7JezaL6SjUhg0KIeX4Im9d/QadjJ6UZ7+LiJPDnI3+izQ0AtzS8WkZT7rGeFtcb5MbXkXHZwqXVl1D3i0fv7I2AQKtLK9NKp5Hmn0ZzaTO1ObWo3G3rUp0+iv/O/lovazLXEHi1lgFDBqAyqWhUFbLmgW04+zuz+OxiBEHyNVSFp1MXmMiD0zwZ/JhNWrt7vXsz/iY+QhBVf16gescFvBLHcSchhS3jnqHK9yYORj+UFmca8xsQL1xC3TuK08L3bN32A+OixjF61Wjaqtp45NYjuIW6YVaa6Z97GJkog67UaplPZ1rZ79caztPgfAJlhw93opxZ8GBvajW1pN9MZ0bCDMp3zuEcUs6+0LFxOPk74SpK82pIcQjRt6O5nHqZ1YtX81TYY/il2oDa2zue5Xja5/RqWEGE+zuEqizc3p2PqtMHk6OL9Z3DG3IR6mrocJQAV8ffPM7Kh1Za79Ovuh85M3OI3xhPWFYYme6ZnGq0AdheSX2F+NZ4Sk6WUDKjhKSxSZztf5byoHLKg8oZHTma8/PPEz0+mt4P9LZeFz1e2tPnbs8lNCsUB1dJRcXPyY+qtioyDmdw7NwxTr1/Cp2Pjtl/zmb6b9P/MR1D9dVqCvbaAMcKjQJ6YMBqb9Zy4WtpLuj9YO+/Xk7JyRJyNuSQviSdUe+PsvYVhVrBtJ+lNSC7wbU6D2X+cdprYnD0cOTW1lvU364n9WHJh1YXlIJX2RXa3IORO6npM1dae5m3SfN7dno2jpMdeaffO4QPD0frrbUHHCCN0XKFHP5KzhEFfhm3npAhIQx+bvDf3kGhUljHjclrJv/tuKO7ox0T0GoyOSZHFzSu9j9397GqyEzqg1JQOjvydMPT1rmke89UEZ0FHu6c//o8pz88zazts/BJ8LG2rxEHRhCbf5shR56gpASOfQYmeQsVHusBuNS3Eu+Ou4mZ2Au5XGIgBgYFsnnOZj6TfcaX579EEAUC+wTyVO1TtBvb4R3p2SqFjZ12/577ESwCn++VfGgyUUGoeRRjY0TSLqehcZXmUYu8A4VFwdAdV6kPsLB9oZTG6WXLy9Z5qsnlGqW+e0Exze6bjLB8hPctOB7lxtJvlmIYAE9UPYHGTWIRdlvS1ST6nYuhZXQDZnNXn5TZ1rNb5myh9FQpQ18bSvKcZPo91o+AjABcAl3wjPH8m0pSz/GutayZwvx6nPycMLQY8E3xRRRtfon6wGTmPuNH/t58PGM88UgIAFFGvWc93y/8nudiHufAs/vI2ZDDS6aXrH6Efmf7MWafB+LMVsZ+MobggRKg+nLYPEzKRkxK+Oahb/gg8Fnaa9vpqOvAPdwdlfzv7OtL312iptUevCbIpDQz7hHu6CbruN0gAWW7Gfi//PwzMmAn/7WKyU5gIHCp/BBOQSukOv0Ps+//5/afYN9/7H+FddR3YKyXUG5KhYyr/i/Sr8iFpu9diHj1cVyCXKishE8+sV1TGZPFkBeNGFuN1oHRKonpn0Cnsw/GdiO7l+8mcWaiVe4kz/81+t9sQVvoQvjwUdb7tZtaadFelu4jMyOIMuRdA2HGgxm8teEEjcpC6/lCj42bIArszJPQ1kYHI51enai0KtYMXENnQyfxwx/B6fZCzkXdhdLcgtAhsPTyUlyCXBBFcGvrj0J/m4e+eYzWcUfRdEh5AMA+D4Qos9BW0sBHaV8y6PlBjHhrhF0eOaXJJjvjm+zLsbdPElIVSHxOPDcSboBMIPONcfj7d99Pune7UzsnBp2mb607t7bewi/ND79UP8pOl/FS5kucyj/F/Kfmc+nGJXpvsi3WTF0SlKJMRK1Vo1Ar8EuVrr0kSc4jE5UUhRfR+kkr2xdt59aWW8zYNhPnjkRQidzIeoq4se6sn/gD9Xn1PFn9pHVTXBRexNU+fsQq5FaJhPBh4eh0ACKCQqBdW0JjezXXfrmG2llNe1U7pUGlVPlXURRehFkTiHeSVCZju9TGGtwbaHNq48MnPmRf732UbS5D56vjiaon8HrWC48GD+Z/fh9XI67aBfN6OvE97AmYUjvoqirH1hrEiweZtGYS8XfHs3boWiovVRJ2J4ychBxcWpwQ9EbSQgexpqyE+/jvRa+/R4lDoDt0ZREQRdB6OLLkwhK7wNeJt0+Q83sOL5lfouRkCQeeOQDx0iInxDWEhBkJ1nMtJgs1lysYcWgE1b7VNLlLqOx1o9ZRfa2aJ6ueJK9T2lgElgXiUFuKUpPMExVP2H0TvYsPeQPmkRgLaidQO6kRzAJlX55lDN4UBwRgVsixeHZibDPSXNyM2knNtprTnDZsY0SZiZgrFzHOe4xuJ/il1RJzTxWo4pMhn7B13FYA/NP8mXtwrvX5MpnEKuiWeggbHoZSq+RiibQoNmskdu+GqRvYvnA7zcXNBE9KJS9fSX58Jw6mOERRRsSoCBQOCrxivfj+2y2sH76OxOuJ+HQ8idkkULCvCOdAZzvJ0G47+PxBOhs6mfDNBF62vMyA2Qep8H2V+68/jcxfklUJ7BtIZ30nx14+RpxrHDfjb/L5ss/55BFpUNM36bEobGNKtwMydkosriGuOLg44FxfjEWhxqLW4iApPJDxoC1Js1wORq0b14c+wtw5jrRVt9Fa0SptjIZt5QXlC9yKvYWz/n3gaWRySVrIP92fvDfyONd0jkMjDvHIY4/Q2dDJvpX76L+yP2HxYQD4VPswYvMI8hPyrfIuAB94f0BHXQf3/HEPm+7dxPT104m/294pCFjl/Hpad5+qD0lj5LdpaNzgPY/38Ev1k/K6CIBMhsnRFZkW6m/XU3igkKixUYiu7j1uIiJYJEnJNbu+Y1iRin7/zSq0HzBIrkY43Eygup0b7+/kRlUVfb5djIsLeAdKbejgiINkxmZSsNu2oe3eTBVEFtDrXyM49sYxdj6wk2can7EbJ+RmI7pgdx68vQDNZQ3L9iwDJGZfWfxoqlNGE9Bb5Pn253vk/5RuMOTkFLyaTdzT3iMPSdcgIygEiiKK6H3HzFe9PmXw84PwaRqPyuyOzhCB0tRJwrFvoCODwOXBJM9JxtHZEY1Gg7tG+m7dG93aG7U4BzojWomHCm7G30ShcCD9pXcpTJ2CIKRwxn0FJvcGIvvvYsrYRFrKW5DJZZJsrgggR5ALGNUicrX98lMU7YMv7pEeBCVLg2hzM5yMzaTR6TSOHY44dFrsNlgn29axN20+ujYd8TemURtRT/R4yXGoUCsQHUVEmYhMkJGWnUZzeDNuc92YsnaK5DDuEeybu3Yu+8v2M221tJHcdPSm9Vi7UzulQ0rZ+/heku5Lsv5ucDAQd/k6FbtAOSjV+vuZ6oOcjJtFqPMYvGteoq9CzrZF28h4MIPAvoEoREeUFhcq/avYcXc1T1f7sOneTcw/Nh9BmsSQo6DOuw5HWQdVR3O5Xacmely0tW423dzEc7OfI6QohGmrpuEZ7YlSo6TdWXIORXzQxubszdbNNtjWDBpjIB7tmZScKGHzfZsZ8c4Ikmbb3ktQqDD37svQZ0KsiNCe+XW620j3MYCT4ocU+n1MrdaZNvcIlsjknP7wNP5p/hIjWwRB0UnU7Sju/+V+jCsr0IZqJdnJHpKqMkGGYBQQRZHqa9Vo3DQI/U12z+22DlMHFf4VBFQGUBZUxul+Z8hytcmRS+dLfV3brkXb1kFjQQMXd+QQNTYK/3R/ZKKKgPpZ5Cd4EyMbzIRvkrkcmM26t9/mPv3/d4JIzQ7NCP2kck9YNQFzp5mzJbZzRDd35uyfw7pR62gubaa9tp3rrh9yM/hToiqfRxDesrun84xQzh47i3+FP1EFseh7t5L1eA9pqa5xIqr6QwJbH8UjuZpRH47CM8aTWsHGGvGZ8DP5t2UYW6q5tPoShlYDAeOk/ihHzl1/3kXaZTcaB9eR/V02zbpAli94lnhjFdfvVOHsbsDHx4fK7Er0jXouWErYl3I3mg4jidcSEWLtHWbdY2Pvi73xO3SS+gcTmRw7mRcPv4i31pvaDimgvG3KNobN30pYlsQEoQVS9sYz+GAfiiM6iLuuwzS5g+Pn9rDi3wT6QMop8n8CmpI5QIe2k87ATkRR5B3VO0xOmcy2Kdus581/bD5j946l94O9ufjtRa4sXU2l20QSrvqQdfgVOl06+eXJX0hdmMrk7ydL7VUGNW47ud4RQ23OeG5uvsmId0dgUdt/HwELZafLaCyUwHL6oio8kQBq1xKvoXCei/ekcFol8QOKfD6j2PsbgkqDCCvzwNJxm2OrzpIyNwWLYGOIhIyLx9NbxuqM1QC8aHiRGHEiuUhsjxZDy9++R7e8qEtjEzd+vELytAgcOxzp1ErOM0FuwKh1x6h1t5PpBKiSX+Rc9LMUO1SS4XoXDy8aZ3Wk92Rw7B+9nyHXP6CzUY++TQko7ZgHuQlmojRRbJ23FblSzsPXH6b9+XbuNN3hjaNv4Kn1pOrNKmpv1jL0laHWsSG64kWOvvoGdYev8etdv+Ic4MzK8pVWYEn07Wgw3CL6rYG4dcmLtejbuBn0JDLk9C7YiFOkzhp4Aegwt3ExYiH+7ncT0DiD5uIKjj71J30e7UPwoLC/sO5FSjZfoOlKMYywrcMeEC5T4F9EuVszpafLMHXlsIzyiGJUxCj2F+5n65StPNX3qb/VB8DaK2spvVnKeO/x9JrYC5lcxpHPJIBQN1Oz1ruW8Dvh3Nh0oyuHtVSuuesept3zFDxj27d0zweCHIwaZ+QaNaFuks5iWUsZbl0iUEqTknvX34smsRZGyLpyrtnqMexOGHN+nk6Z+3UyBkggrkm9JjGp1yRmb5rNiZITyLxl5GzIIWxoGDpvHRPWT6Chs4Gv7/oaR5UjZz8/i8pRRfridMLcwljWdxmfnfsMIVvg8vnLpC1MsypD9JyKig4VceP3G4z5ZIx1XdudM/7m5ps4uDhgMVqoyamh//L+yGT23nrfJF/u23Pf3/KFtbsF0ntpbxxcHfCK9bLKtFe1VdGsb2bku5L6icVk4f699+Mc4Myt0lY8WrNw0tvmxcR7ExEjRAwNEiC2zvEbXGpuYWwaxsjXhvHvrLOxk+qr1VZ1IkEAvbM3Vc7e+PjA0FeG2t5hXDS/C79b27fFLHLhqwtoIpLALQ339r5EVD3B1fBFiDIznb1SeGl9PHMXS85og0EENdZ8Yq7BrjxV+xSj142GLteGQtDRoS4G7AMTcsEBQW4goH4Wwl/GAUOrVPYWlxYm7pxIW1QZvR/sbZXhB7gStoAyr5+IK/0AFE/aXR/eej/Ninyi9DNxMPlRe0Aah/3zT1IadS+V/jdAlHH3viO41eQRsCyE7NFPIyhU1jlwd/5uLiRe4Pdnfrc665Uo8Woditm4H48/3+C38GhmPWdLc9FukiJSNb41XBzkTHBmCA/IHuCB3pJM4WtSKnLMDk7IfJ34efQqZjfMJvPrTNxedQMgLyaP0pBSRFctxjYjMoUMlaOtXLke/0JvNpJ1ZznrX16PS8pk6oNTrcEH2e1N+DRG0nv7o4R6tKLUKNF62/bzbaFtjPlkDL7Jvnwe/Tm9/XtzaqYt2Df69mj2P7WfgN8DMM43UqYs46/2eOnjKBz+WVqu16Re9JpkUxyZkzyHD059gPCMwLT+0zj14SkcPRxRqBUkzkz8x3v0ebgPqfNSKTlZwtZ5W6m8UIkiwfa8hHsSkMlk7H9qv92+sduqLldx7vNzxE6N/YsUpb3VhaQTOjIa1xApMnblpyvkbssl5SEp2FcaP4by2BFSu+gxdvRUNOn+d3dArr7DxnYbdHwQwV8GU+JVgkwuwy/VD1HsDqbIKD5WjM5HR9nZMv586E+GvDiEuGnSGGDqMNGQ34DOV2fNX93TupnWZafLuLHpBhlLMwAvEAUc2htpq3Lg5AcnufzDZR67/RiiQmoD3XOvKMOauke6X9e3ix6MoyMo1JdwcHGwKhF0zwfOrc44teilgHu3ipiqknqXwwBc6FdGv9vJCILkP/CI9sA/Q3IQymVyQopDQI0V79wTsKKUK/Gv8Kf1YCuOyxyRK+WYRRuzTxRhy/1bECwCd3el4bHIOxHkAicGniDZZRJhwy2S1L4MzF331nRqsJhO0lwwhC9m/Eb64nQyuwCLVW5bMSvNVPpXovNxs31rUY5zRzKt2qvo2nV41jlSdbKATpUeCLD2xx15O4gfEI9vtS9HXjlCyKAQ3MPd/wa4AVjddzUdtR2M3Lnc+lvRnzfIfncfUeOiyN+dz9KrSxFFW5tt9QjFK1bgp+E/odKqeLzueYZdy+dwciQVgQ3g4kLCTAnQIpPLrHOzURdCXVAKHl7e9F9hA7M0d/mkuy1Pf4LzXwVy9NWjPHTtIWuwz6nVibvO38Xtvrc5+d5JDCoDQUuCKGuRxgP3OHcevi7Jld+qu9WjHUnf5Zcffvgfq5hkAVc6T9O9E/lPsE+y/1/Id3bbf4J9/7H/Ffbnkj+5vfs2fY19EWQGSr2/wzjAl4H9D1vRMq+/DmV/WceodWo7RHH3ZKZ38ZH+ayzg3Gfn8En0ITgzmAuR06hy30L5VB1D61bZ3Su39SK5gS8hE2RE5vng2FpA8TEl7hHuuAS5cCntNCUuv+PY4chDXz+EPvMsrRNGUnmpEs8UW7JSlVGFvEaOsd1IQN8ADC0GbnXnmEDBicEnmPf+PKvT22iEQbdOo+psoc1jJ56ebszdnmW9X/fG1r/Cn9CcC+gbkunzSB+C+gfZvfPteAfyEgbx+Jhe9OlK3Drsi+n89sAhnIU+jNkTjFtH619yrHUhsA3hZN48hUxWxYapG5i4eiJj/zWWYa8PQ+2sRvaljKtJV4lPj0cURdaNXEfo0FD6PtuXP+75A5PFxL1v32v3PQUB2h3yadPcpFPbiSHGwJBFQ/BJ9EGGnKwb1+zqrdeUXla2WPc7V/lXURTnjOegOJ6qe8rKbPiDGdyI+wPnFmdcm10R0xW8oH8BmUxGrycziP9BCuweyzpGeHU6t25BQoLEjvA948uru18FGbQ5txFyVwgD7rU50JrdmtF2aK2Lg8aiRgSzgGe0J4IAZnkrHpWl+FzNofhYX9qq2wjsG4hbqFsPWcFwhEWLiZ3sgYOzA4tOL+Kz3l/CdUlK4tMVX/NV0Ie8/uQbRPf6hbOW/3qSPQucVIiMuW8pfme2MWbvGEr7FZMwNvRv+vm9H+xNxOgIZDIZiTMTCeoXxNIrS9l5cSehrqHcWXHHeu5Pw3+i5EQJ2ydux6w0M+BUH5rLWgnsH2iVie2ui8nbJuNsPAzYMxn/KkPWjXBdcmEJvsMTqD6UQ0xeDEOPDkWYdIOYN9IJuxWGSqtCPs2FMPcwjMoazIp6Wm6U8duUbAY8MYBl+cswG8xMODgBSv99zj7AbgxInZdK6rxUvu3KjSAoVMROiWX2n7P59a5fOfraUQb/+RBnfxlEpX8lWssQRPFP4qfHEz89nqorVZh/FHFudebUwFMMuOWJWW/m5zE/kzIvxSpB5FHvwZBjQ9A65JOzIYfGwkYEiyA5o2SSc+eTxz9hlsPPABTuL2TrvK0AzGQmr776KiGuIdYFwvZF2yk4WIDiMQXxN+Jx/8md8vXlBPYNJLCPJGNyddzTiGYLmrY6GvLleETZR5wFQcotoHfyQuUGV9edYv9T+5m5dSYAKrOK2Fux6B2lTW5dbh1mvRnPGE8wgsrUxeISLTj5ObHw5EKc/J2oEiVXqq5dR+TVSBoLJCflzS032f3objrqOtC4a3DydyJ6fLS17ZSdLWPz7M0Sgu7+vzNg/7jnD9od3AHJodItkRE3Lc6KIO1uXw7tDRjqVFScr2DXw7uYsXkGPkOkc1xrbhN9fj01vSYz5qPeLP3kflb8Gwf+X22JxcgLuX+QJQtECIjF1GHhxRdBr6rgsXckNOqZAWeIGx/H+U/OW3X0e2rtO/X1Jdgos8ohiiKYFM00ay8hJo9n6dtynB3g0b6PWoN9YlcbEUWJGdMzP0q3Q7EweQQuPraNuFQ3f2VQiji4alA6KKl3OUROyHLUJi8eTtlIQUUsITEhJN+fZPf95TI5vlW+9PuhH9ccr/FEpRS8f/e7XJq0rZgUXRInnjIa/OIwOrp2Bd8ESdI53AGHSH/WDv0SlU7F0stLOVV5kEK/DwD48+525qaFcOWnK4QMljZTVgdsXjTB13fTWjUYXaRtU9yokyTr7vnjHgLLVVz1Pc754iru+f0eawCqXdfOGy++wbgc2wa9z0N92PLbFvqc78P+UfuZtGMSNZYa6MID/BW569jpiKHZJgvdYmoCQGMI5jmXHB5dYOK65TrBmcEMf2s4ya9nUhGQx5h9Y6g9paLuVhCmDhP+6f7WcrW4tlER3QeVSz7n1lwmcnQkgX0Du9QCpMnfq2UkTklJjJvujXecN/m36ziaMJRWxxxrnV95bw+VMe5Ej4tmfup8psROIenrJO5cuoMmV8M3yd/w8I2HubHtNktWLeGX+36h07ETBxcHO8dad/tRmFR43jpJ4UETOl8dSkdpS9AdSBWUaowjxpFgiyVbmUIe9R6M3zUe/eDrQHqPe0vHW11aqQ5oQxMbx/KiULReWkRRAlOB5PjLTs0mMXA+/ZaF0m9ZP86WnbXeZ+SBkXi97kVTYZM1N+gzz35vPR50K4jti7aTPDeZyopKfp39q5VxdDM+l+gTvhR+vgaAhScWWueHYYeH0edCNpXTIjj0wiEcPR3xT/fnht+LNOhOExA8kyCN1BcWL1nCG6+9xln+ef79Pw0imZxNCL26WDTdkqnFPb8tRIyMsM5FeTvzEOiSrBKVf2NMdBa0kHwlGa86L4Yc70vn5CasFAVs9ezYLqBpqcElyoekadK6sqqs2nqeQi71vevv/YmhxcCLhhf58PtyKtxP0ao/ii60HZ0+jkVDIvD75Gn+2KdCfFECdf2SGElHSiGnFp7i8EOHKTpUhNvmPhhVdURUxHL3prsRY0R+//13QoaESGyzrj5XHFpMZ1ZvmkuaqTxRidqgpk1pg/wrLFosFhHBLNLcIY2dfju61h0eT5AzBBKDPP5bufPJwDokuaH/zr5VQnsSLIxfyOt9X0cwC0RNimKXcRcgAe+anM5QElKC4yOOZCzJYFfULnbXbSDM3AvcDFxPuM5Q3+FMWDXB6ljurouA8gDC7yjJ/C2TEW+PAKCgyNaKlqz9AWePbOZfmG/9TfloKmcv3qAyoJJNd29i3MVf7Bwot/3fwqiqYVj2RHpfikb1dAaLv07HNdgVS4P03HqXw5xOGcOD/afBq9J1pg4T6Yp5ZNcdRtN4Ev1ZPb87/k7hgUIeu/0YOh8dZoUEHjsy6A+WvPcFv034jGnp0/hl0i88EPwvKi7EWsE0jflNnN98mdjJsQRkBLBbO5smRR51iaCIGWSVwAabvGi3ObS38mXwe6iGZoLTKOv3mur4CaYLK4hIgayX7deyYW5hfD/5e8wGM20/9WAbWu8tx9hpQeMuRR9aK1oxtBqswYzvF33P+y4GMpbanDAWQUCUmxGByOw9XFhpoN/J2dbj+5u+otLjDyo9/iDgwgzaKlu5sfEGMRNjCBqINdiWcT6DxJveRDydhaNG4OxGELFIwVO5nFavcJzrivhl2I+M/3I8imEKFqcvZnH6YhZsW8CPl3+EcMj5I4eNMzZieMEAPUD6h0Yc4qn7nkLW1XcFAQSZDRR4tt9ZHh79MCevnsRllIt1XK6ISiE8PYybm29iaDWQOi/V6uAuC8xm68wiBnsN57uA78gcm8mpuFOYsEmzBZYHIqo6qD57B5klCFFlq8dgzSzy4xxRfLQXZ3OjtX2DFEAqDSlFPUXNzdU3JQayt846Ftz+7DbXz12n/nY9Om+dVfGhW65U/aSagxwkbmqcNdjXswld/+06V3++ytWfJWnB0CGh1mDfzqU78Yr1wj3CnStrr9Dn4T50U3NUnS38NGIrCbMS6L3k76yiZr9eTPi6199+9/9IcnhXrKzA39kfhUphlSk+/8YM0k1SAO/y16foKKxhytoptEa3Yv5aqqPo29FM+DON1kHVgL0jubAQdrx3g4EDIWNugpUFCfxlf25fJptUnQzf/OMIbh48Wf0kjz2ugK44vhwlIcUh3LMxCXOfG6hdk7GJeYgoTUpqDtdQl1ZnDXyaeuSNL/X+jiTxfv4a7HPSx9KivUJw/QKEv6jVP5D7ALq3dSjMClq84nl/zDMMmWWLCF5o3kWZ10+oDWocOiw0FzdRubuI4MxgvHp5EdP8ED7FD5FwYyOWxq249etF87USCmOmWvubDDm65nK8Sy5hbB+ARd1NhbI1klqfWrRRWlorWtH56KzguTqXY9S7VHGhvplZ2IJ9PccoQd5J4cFCrv96jYHPDMSrl5e1/cksJixtJqLGR2ExWFg+djlfzfiFs+f3UeNTg8qowmw08HHQ1wT2CWTO/jk22cCCCPpe1CEOVjPuqwm8/3t3MkPpvbZO2UpawY94zv2eq36OdHzZwdZftlrLdfeYu7EEW/AM8SRlYQpeGvtgdXBmMENfG0rwgGCMrkbKT5fjpHbCM8+Tfmf7YQ4249T/78Gn/c/sJ297HkuvLLVjnSrl0hrRGGUkICPAGqT5r0zpoETpoETrpZVknj20RCbZchEffuUw5788j7HVSOmpUrvgIkDagjTipsZh1pspPVWKX6ofKq0KY5sE1pcCMckYtW7IQtzoUq9l1PujGPTsIGs/cWytwbv4ItWRA9CXyPkyfiMDnhiA2U3qkx71HvgX+9N6VyvO/tLY0dOvoNAqEF1Fstdkc/mHyzx842FEsYsFLJPxfLuU5LL4eDGGZgOWHsncam/WsjpjNSPeGUFrZStesV52Mrhlp8tYM3ANHtEeNNxuIGZCDOCFwmwk6cgXHHopTZKOTJPGGqvYgCAgFy0Y9Qqu7a/h8FlH7lnk+rdxIn1ROumL0q1/d4Nptk7dSnrBbMacKqC+3ALE2Et8ysx4ll3hA49dzNw8E52vjovfXMTZ3xl5XznDDw1HsVlB++J27hy5gzbOFoiWo6Tf2X7Urqqlfmw9Kp0KS1fBZSi5qViPz+1inPXOmDpMqLQqZMgQ5SIHRh1gdkQsGXMc4UHpft25UONvxDNphwv1MSVSaqeuOVAUpbVQu66dn+b9xDKHn6m9UYtXrBfOKjeyblyhzOMXzgy4n/wEJ9xXuSO6uUOvhfQcJ95zfo9TW08RIUTgHuFO5aVKCg8WknRvEhUXKtizfA8TV08kbFgYhhaD3bf27hvOXV/fhZOfExEjI3oAXW2Wuz0XkNZfgiCNXd0phSwmCTzZDaD0d4giuHYhnR4J3EmdgvvfSARdezmzgowLGagi2gi9O5QhLw2hQd3ANxe/AaR9bdyxOMqHlTNj0wzu3XQvZS1lJPkkMTFmIg5KaX94Y9MNdq3cxYrmFbg2u2KeI9VXR1MT8//66H9jC4FHOppsJRQkwIe+SY/OR/df5rv9/1dTKBQkJSX99yf+P2T/ydn3H/tfYdETokl9IBVRFLnjJQXhqv2qUadGs2fFHtaNXve3QF/Qjb38+fCfdr/9ddANGRLKyvKVVjR7o+40ICH4O10U/Dj0R1b3kVCwPZ2Ck7b1x+fMdn7M+pGcP3JoKm4isEiLtl2LIBdodm1G1KopOVHC+onrrXrnADF5MaStSCN/dz5jPxnL5O8n2xSlulbdgihQe7OWpuIma5lNji7k952Npl+K3Tt0b2wDKsIIzruEqc3I+C8kxGf3OwsyI8cT0jgRn0Gb0bYp9usXSm7mAipCO+mV2wu/MgM3f77IqY8kZFi39JB3bRSpR/5EXlnG1HVTCRsahkKtQOulRemgRK6Us3n6ZgLnBiIKIpXZlTTdacJR5Ui6fzofnv6QYWulzc+Vn66wqvcqLBU1XAt5mGr3bZLskd7ChG8m0H9F/y5ZDxEsHXgWn8Fwu4TMJzIZ9f4ou3cGaTEiUyjQemqt0kSlSOVPuZLC4u8X01nSgNJBiUKtoO5aDeN2j8O7psvZhsDNzw/wSfAn6Bv1iHIRZBKzwKvWi9Zym4xX6alS0i6l0eDRwOpH15B8XzKb79vMj1k/AtBhbuVgcigVTi/iVHKD/D35bJyxkeKjxXbtz6J2RPAP5Pbu27wme42jbxyl75ujKIgRaHJrQpRJzJWQyBDefe89Jjo4YHOD2ttZYKKDAy4TXNmheJ5G90Zye+WidFEjCiKGVvtFZ/jwcNIXpSOTy9D56AjsG0i+UZIALG4u5sR7J/gx60fMejNhw8MAGHB6AH5Vfow8mEVzSTPD3xjO1HVTu9qqdO9jQ45RmZiCxWjh2q/XKD9X3lVXoNK3EpB7GFlJMee/lPLH/TT8J8JfGs2u2X25knKFo0OOIvj6cPT1o7zr8i5n/nWGUdsjmb92PseHHOenhccQTAJ5O/Noq2xDdBd55cYr+Oh92Om5k3SLtJCtu1XHoRcPUXX5f+qGlczJX9rwDHlpCA0XSrjv1/uYtH0SQ/eHYTLYvl/hgUKCz2kIKZE0BEWZgEyhYOLqiaTOT7Wep2vXkXolFce2WjKfysTBxYHs77L5JPgTxv96FtcmV0xqE0KXA2fPcpuu/rZJ21Ab1GRdyaL0dCkgsZsVDgosSgvIwVnr3IWstplCIQXzYs78xJY5W2gubeaLXl9w6kOpP0g+OQGlvg1Tm56QQSEMf2s4gX0CGbbvWbbMFnj32XfZOEcKlhx+8TDfpn5LW1UbMW/EMPDkQFIup1B9uhqlg5LgzGDcw92tY2NReBGfv/t5lzNFCp67hrgy/+h8nml4hpCBIczaNouQQdK3k8llOLg6WDeTx948xjsu71B/W3p+2dky2goluQn3ihxOPPcn+mY9k76bxODnB1vbF0DS4c+pWbuH8OHhzD04l5BBIdb+ZtQ40xCQgMrHDfg/z4XZrtJzs+9gYp+exMCfpJ1HjetuJv8moT83z9hMcO9gXhFfsUpTdjvXEko+I8gxmoFPDbQGnUURTvfK4kyv4ZR6/UB7ZTNXf75KU1ETaoWEshZkRnSNZbiVXEXfpKfsbBkt5V0em67NWIerK2afAGkzckCCXnfXhcKs4NVXX8W/4BRzTz5IytwU6/xiVNXxL/1Q1t0zi8pIm7zJnaY7LN6+mOcPPY9ckKNuU1Owr4Cvk7/m1tZbrKt/lBPxffBuGcmrxRtx2aihJGk8bZ5hds7GM72G0WZuIm1xmvRcoNNscxiKMoGG6xVsnbeVstNlHC46zGH9v4CuXFl3znF51Xn+Ffovio8XS/XY5dS5FXuL2/Emai+Xk7czr+t7dntdwKK04NhUyVuOb3H2c2nUVNxW4FPjgygTWXf/Onwe8KEhv4FtC7ZRtK/Irr53PLWDsWvHcurDU+x8aKdVxlNt9kLVLLJz9k6c/JwIHx6OW6gbVQHVWBQWPn78Y/wfG8yuR3bx00gph3B3Lq5G55NciJxKwLBePNP0jHUjt8/yPBWev+Hc4oxH8UaE9g7ip8ej9dJiEozWQN9TO/ex8KsoYpcOZeT7UuDbSe1EkEsQ3lpvfrv3N9J/Syfr1Sx03jqOPLefwIpAAssD2bJgL3d9dRf1efXU3qztKpc0niktSvyvH6C5pJkl55bYMfSsdSXC+knr+Tr5a7v2pTaqCS4NxtLYwvpJ61k3ap1du5cJMmRmAVGhwi3MDXVXbr58v/cACS2/bco2nBJsTsOecomV/pV0Du20D3L3cEL4VPuQvSab679dZ9/sffhX2jyM8aUf09QEHpEeeER6dL2HCKKMRl9/KiMzCRwYygOXHrAir0W6ZYwViCLsfGgnReuLePe99xivVP7j/DsZWPsPv/+TdQeRuh3rpk4TZr0ZQRC5EfQ0pZ4/YGgzcmXdFYxtRkZ9MIrAPoEI2JDQf7XqJeeYtmUadT5wc+BCdOE+/DTyJ/as2NP1TlJdJF7qJPHY11j0/8CMFOWocq/jUXaVpqvSXCNTyCgynuNS5AwaLeV4Of9AddwjqHQO7DzoiKjoWRbbmrXPI30kWcqugbc8sJzfZv6GLFHGzc03qTgvKQO4EYZXywic5CtQDZ7AnSN3OPXoKRw7Hek0d+LbOAmV2YOsC6swnsnmDdUbzNwl5VQsSCumNH40nS6+dLr4Ym7TY6ow/pdB18XAYfi3a6huOwucsgDpEHImhO/CvqPifAXj14/nXL9zAGTmHiWi/V4qAitQ3afCL8WPzuhO9I56QE5ZaCsb79mIspea3kt6EzpEat/dY1T07WhS92vt8n51AzTUJm/cGozIKqt7FouzMSfYMeZ5698qQzO3Hv8W38LTXb9I3zvrnqe4nTELmYc7Pgk+KNQKu1w1H115lms5EojOPcIdlU6FXC61ed9qXyy7Lah0KgL7BSJX2bsHiv1/RePtQr8V/bgWKt3jhzJJItL/9jEy/nyDgl25HH/zuDXfck/Wl4A9GKsns2/K9scIv1lMwpw0LL6B1isA5DIZgbcOotj0B0mzk4i8J5IHdjzA3vy9tvfy/4htC7bhFurWdWUXK0aUU3+rll/H/2o919RuskofW5QWHLT2aOue+RMVJiMtuZUcfuUwJWdLqOuoo9Mi7Z+SriYRc2YdnnE+vCK+QvKcbrCM9F5+VX6EFGlxDPbEJ0Na71S77WRXbwd+lEusc4POg2HvjsI92d3uewwKHsSC2AXEOsbiEuhCx4AOfq78mb+aIApsmLqBtcPXYhL1FPt8aT2m1+i5fuc6K9tXEvVZlDW4UB4VidPgNE59eIpDzx+y3qfbmnTnkDs6EJARgNFJYv/Ku3DhZpWZd597F6O/imML16IyttOzjhOdn6cp8mmUOgcrsKnb5DKpPSnGK1h4aqEViNb93p0lnVReqmT2ztlMXz/del13cEF0kM77otcX1ly8PYN9mU9mEn93PJFjIpm1bZadwsbkHyYz/M3hZL2cxeKzi63sUgCZKFCZXWm3z/t3duqjUxKbo0deqqM/HuXwK4ftzqsRb3EgJZBj8alUni7h2nqpv/QMHNwJu8OecedwDPZi16O7OPj8Qeux5cuh8tfDbH/8yN/KIAjgVF9M3PHVaEpvs7rParYtkphuVta9TCAgbz+uxdnofHSIaimw1qEuplF3BovCQrvOgMxspupyFXKzsftj4NjpyOnZp9n47kYMLQZyfs/BodTBrgwGs5G/WqRrLC4dqSgtrnj/JTNKdx1blBaiGr5BobOn/rWZmwBJqm7Kej35266zfeH2v+2bUSoRFErCn5iG91sraPMMtbZrGXKqogaTPfppHNwdcWypQm422EkkywQZeZ/l8XHgxzTkN9Csb6bI+wuu9VrNd0u+wzDcwPpJ69m+ZHvXt+7yf9R4M/CgknOfnePyD5f54eAPjFo3iquiJHXoW3QW4Z0PiJ0ca/VRZL0yi4rUd9F2aHnh7Rcw7DpB0n1JRI6JtPsmvtW+xN7QISoUJC/ojcFJCtbp9FJqmjrvOlrdtbj08kPrreWljS9xrOiY9Z2eOfAMT+6XmJApr6awWL3Y7tv+Jv5Gr5W9cAlysfbzB3s/iGuzK/E34xHqJV+TdW8BNOubpeCDXMado3fY//R+moqlOuowdaBRSuofPVMzGFoNfOj7IXtX7uWv1lrZSunpUtZkrqHkRAkhg0NoaWmxfoOT757E2Cq1qZPvnbQqLHWbSqvCOcCZqz9fZc3ANTQUNADSGurymsuUnpLWLYgigslsLZdHlAeBfQOt44RDZxOeFddQdbZYQZQymYwQV2l8Di8KJ/WHVGpzajn4/EHOfXnOTi7/yPYjvHX7LdIXpzP649E4+Tr9zacIEDo4lGUFy0icZWM6ugS6MPT1oYQMDuHitxfJ3yX5XOrqoLYWtF5aEmYmMPzN4Txe+rg1D50gV1IZNYjocdH0X9GfuQfn2gH3gm4dJH3329TdqmXz6G+5/e0R3njD1meizv2KX/buv5XRnoFu4drHB7nz5S7r3902fmssfvknCOwXjMZNAo26zHdhlXwVP1/9mWNDjmFcZqQ2p5aNMzZy58Ad67WdbSouZFzA4y0PNkzdwC9jfyFIncLAm6dJL1zPUe1yXhn5CnXX6qy+2p7q+j3X+wCxHkk4GsKo9K/kcu8OPNLDePj6wwxYKYHxNyvvtkqDOxgD8P9TyVcJX2HqMFm/h0JwRG3yRml2JX75SFzGDvjHZyndlfgm+6J2UnPh2wscePoA2xZsQ6lRovPVoVArGPXeKCZ8PcHuOudoPzKWZhA7JZYBKwdQLBbTYraBT/3yT3Dus3OkzE1h0ZlF0juLCrxrvXnxradp333U7n7xTgNJKf6eiGpJulesruGblG+4vPayXbmVZiXj9oxDe7GN8GHhDHt9mDWIDVDvWY/lDwsDnhiAb7IvjQESYPeDUR/w1oi3oBnOfn6WqstVqNxUlAWVcTPuplXGU+T/UAq/R8ewWMCsN9NR22Hni/y/yURRtBvz/t+2//vCq/+x/5WWujAVxTUFgiBQ7WxziosyC4ZmA531nWAP+sS5vpgbG5tpr24nanwU6YvSrQO+T9FZ/G8fo/bmHALS/Hrcr4vJ1q5F3aTHI9oDi14anLo7qSgT2THxJOmNn7LonhZCBodwe9dtxmwJY6DuYT586kN2T+1kZVxfAvoombRmEr69fUHyS1LnVUfZ2DI8om0QjRzNd1yI/5IW7WUCygNo29zGV29/RcSoCO7ZPgejopEm3TkUgiOiOITrv11HqVESOyXWuoHIjS0grO1R7uptQ3aDtDmwyDvxqfYhsDyQG1uzOX+4kkHPDqK2Roaq/hD16sN8+UgDcWUfEfNbNvqaFjKfyLQGcuSiHLW+GbNCZWWA7HxoJxe/ucjF9y9S0CHJ11kESc/8mYZnrM83WUxcLrtMVnYW+b75mA1m9E16BItgZa+MzHsA39d9ufrjVVLnpQLQqS7mXHgST+5+Er1HHyDE9k5dC7Ckq0kMO1RKY2IR1d5aXIJccPRwtC5kisKL2Dt6L9FO91J6uhSXQBfK95TQ+1JvSsMV9LucjledDw5TNLgEuSBXyanpW8NK55V8+ciXPPrlo+zatYuCcQX0X96ftcPXMtkwGbVRzY1MybmetigNfaPEEirtyMekbOTIkC34+27i8Uda8U3xJaifPcsSUUDQm9ly/xZA0uEHEHrPILV0AW4XKzBNqqe1Us2cu+Zh0psZ/OJzDELBIsz4IjEKftJoOGoysWLGMrTrdGwTt5Gdnk1erzz+Ff0TjUWNfB71OYnLEnkg8AFcHFwoWm5zbpsNZgSzgLxHnrbW8lZqb9QimAWGvTYMwcWN5w49SG6vXEpCqnky/jm79tXd/q4nXSdEE4Kx3cjm+zaTviTdyl5RGtoIuH0Mo5+Sid9N5MLXF0iek8yJutNciJyAoBA4PPwwmaI3RQelnEzu4e5YFALHBx2z9k23zDheNr8MwK1Ltzj51UkEhcCFbRfwX+1PWGIYdbl1HH/rOF6xXtYE2nse34MoiIz7dBwn3j1BzbUaQJLpkwkW3nX9gOCBwSwrXIZzgDNf9P+RGu8a1EY1Mbd9sBgFjr99irLTZdYAQ3LOaJJyhiJoKpCrFHa55wDKgsp46/m3GHv5RSJGNbMgcwGuIa685y45u5VmJb5VvigdJMfBkJeG0FzSzOgPR/PeW+/h1exF5I+R5LjlENgnkLSFaZi8THAebiXf4qE/HgKkfFc5G3JYdGYRCoUjJhNURwxg+mJpQ6DUKO10/FX6NlIOfkK5rh8Tt461MoAVRdCpk4KJ3Yv9pPuS8M/wt+rptzm1MXXrVArUBQwaPwiLScqd5+fkx/sj3+d67XUeznjY+g2ixkT9oyxLtwX2CeTBSw9a/3YJciF4QLAV+fV4yeMcOwZ7PwCnhhJyf76A6f0h1vJ0vxNAZeRAktJ8pPyuXczBLjU4Ol39KUy/m9GJUuBC3i7/P2LkdOoauR11E0EY0OOI9OBJvSbR8kgLe+P2YmozIVfJmfTdJDyEODxahxB+24ntfd7AYcPdtsCCCC3aLu3I9j1ceUdH1ZFcYj6KYUbCDMSSTOotbniVHsG75BJVV+axduhahr42lCEvZeHRNohal300aA/SaNFy+OVLFB0s4oXOF6zlCmAADX5xtLsFWb9RXuBLf3s/QRTIXpNNyfESfF7w4fvsLvZUABx4+wCDLw2m5loNd47eQfCVxlUnfTzujW7UnLiJfKAJHGyON7lFzpMfPkld/wPMPzTD+pxuh7auTUfvc5EQK2PKxims06/jg58+sJ53KvMULupNjI+vQOej+5vD+ly/c9RVDeLtlbOIju5+B9H67MDyQFT6dsJHhOMa7Erp6VI6xnewyXMTyKAgqgBNLw1t1W1c/vEy3sne9Mvbh4iAT+sodmyX+krBvgLKzpTBOKn9KgUnBMFCZXYloUMlB359Xj1jdg/lSsp5SkNKQaem94O9aauWHMLdyHCfah+mb46nxC+H+B7yaTeRxn+XFhdGHhxJs+sNIMj+nUQlops3Ld6ReA+KIWSg/fze7UR16+/G4JlSAHzGn3P54PFTVPlVARLSdtPsTXTUdbDizgrr/NjiVMCp8b1Y/lKW3T2tiHXBgsOWP8jLz6PX5F525aryr+Kd59/hPbdnUZ60OXjELrnCfmf7MXZvGvrlBTTd8ULrrUWhUVsD1d1WpS8md4eequwqLPfYNoLXk66TmZ6Jk68TeX/moXHV2DkhzvU9x45fdiA2ipAIda/Ucd+v93FslJrkK164TyhmyvdTrOePdHmUxr2PIspEyuNkOPmBs7PtW5rlUp0lnr6Gl76Jq02lhGaF8viuJ9hzJZvBv/zCUDnMM2Odf3fJHDgnGv4t86/bzgKnBRnzr8xDtkUGI+CHQT/Q2dhJ86upVsZrZ8M09szdCsDYz8bik+hDoZPk5M8LfBlBsO+/cp0Sod2MwSWVdkMwCi20lLZYc/Z1B5jO9P6RTt1SMg7e4uzS00xcNRHB2+YY7jz3M37tvoTdPxBfLzPFx4qtaywPZSDuhlQ6TZKUtGNzE0ZHV2rdszErWjEp60m/mM7V81fRtGgY+MxALIIUrGp1aaXSrRnnCGdWiCus5U6R3Y8+734ABF8pV+sh90O06yXlhrSin1EKzoRd24qhbAcOyTGcLldAGmydvpXYm9Op1z2Me0sKyO6jb+ww1hZs5z6TjZHb0/yAd4GJwI5/U1dngdEKsIwAnCEsIgyvh7xwDnTGTePG4/1X8smZj9mT5kx4h8SEt4gWLCYLol6SCO7ObwfwV4J1hGkytxQbuZJyBZlvCg9aBIoOFRE0IAitUodHaxYqsxs3Bz9AaiqUnSnDYrIQOjgUh3L4WAhkqJuRVzb1x1lWjNBhQObQtTfp6s/qMA+a/XqB2UB7jR5HT0eGBo8i8+ZJzsSMZPihgVRcqWDYm8MIHhCMQqWgU16NXl3Gmf5nGPX4GCaPmdxD6gtCG+dS7P4TujYv2ssaGf7mcOZdnccT+54gTicFrTqdvWkISCB4eBQP34iyzr89c8v2zXHnh+M/oHHTcO/2e63zgaehN7F5MYiyZob/6z6ufwZctO3FZDI5ji01yDqkQPHLh19m9aXVrL60mnFR4/hp6k/E3B2DRqOhpazFzqld6vUDL9/MYdScgQwdORS/VD8cPR2tzD6fah9EQwXH3y6g7mYdU9dNtQv27ZySzSe9lnLs8XWsvrOa9RHrGeMkSWu6Nnuhba7A2N7FvJXJJMZE1zvvnLiTS5kuDNdqkMlEJGSW1OfKOMuAO0uIyo8ibcuTFBRcw2IRUCql9rMofRHKJ5TkXstlct1k9s3bR2lVqV17ismNofybctxUbqgcVZw1/mB3fP6P89HX6Wl7tQ0s4NjN7PP4jVZTKNO/GG/Ns9o9Tzl2OJJ+LgBzsIlx28Yx78N5AMxT7uKXxsdodD4JQHuSliH9M7lwRWHnNO6ei0b8uZI+XaSVHbk7eO3oa1ysvIhvlS/Gq0bkz8pR67qkI7tyUi+IWUDNqhq8dbZI0cWKi3xwShobZR/KcH7XGYVKYVMV6dHHPGM8SZmXwvqJ60m8N9EKLgGIuSvG+u+/5hQzat14uv4Zig4W8ob6De766i67tbxb1S3+mHGNEe+MwNBioK2qzc5hV7+nntxdufR5qA+fhHxCn4f7IBO7gYAWhn9zDwGBkpT/lQ1X6Lu8L+caz1HnXYfeqQiVhzPXduSh89VZmZBF3l+gH+iMV+swGosaqbxYScigEOt7ywQLSkMbmKT1Zvf6vmfQdvWS1fQvOEPtjVrkbQ6AC6Ve31Ps8zWBst7cGvgAqVUtrBv4Lbr+c2n1CmfAraPkp49k26RtPDLyEZpLmtk4cyPzVs5jr4ttjv+nYN9Mx+/YlX0HuajGwT42yIXPLvDqq6+yftZ6HNqaMTbL2DJnDzo/HaM/GG0td3lgObkJMiYOjOTenT5WycZ2ZQllHscQxrtiKZpMzLpD1Oy7jLLvUuhilgoyEycShuHdNJ68zSNJOLab3P5zrSAxgJkbZlKQW0DfZX3RuGuo6qgiJ/Qx63ER8S8gDOlan/oUAoprCV2Wzv7Z+1h7Yy1CrcAQMQsXoMPVHzJ6o/XSMnPjTBo7G1ke+hkafRBGtZHLKZfJCBzLXV/26/Esqdzn+5ynJiSexYGBdmztrBvXOBqfhF6Zi8JkxNRqoOBCEU/ufZL3n3qfDp20bwwvDCfzu0xu+99GM/Av+qnA0weepqGzgUddH0W7Ukuv5F6oB6lJuC+BV1Ne5ZPBn/BV/FfETY9jxsYZXKy4SMbqDO4fcD/r3l/HqY9OceqDU8ROjcUt1I3jJcfRm/Xoluv4tPFT+jzSB62XlsRZiXj28kTnq/tbGS58c4Fjr0v7eMEkYDaaKSwqJCkpyU7abtQHoyjYV9ATPwBAW3Ub+iY9YcPCGO082toPtJ5anm1+FrlSzjczJb+e+c+9lA1fQMigEEydJhDBYpEAY/WBSTQEJCLK5KgCZSzqki1MI43h4cOZ2TiTMw+d4cmUJ/ljxh/4p/kTtziOmPLXAJHCQoiIkNiS3Wl/evru8/68jc5bS2DfQP5qTn5OZHWts1eWrUSulGM0wgIJw8SWLZ7/yJIUFUrKY0cQZ5+ezjr2tbsFUBeUgkKrobzXMDqcfWkuhZQuToBDRyMolJSeLqXsdBkpc1PQeGjpd3sX10IeIaBaR2ghJKwYSXW5mVMXsVujezTo0Ou8mLFjJhoN5O3Mo+XHFnKm5FCfWk99ZD3mTDPe8d7cveFuVL1UsFW6Vo6SsuAyHMc6kqhNRBRFCmQuuLf3B6QAvSAXCLs/zCqP3A3Eue/n+2hKPsWZOmcqL1Yydd1UfB0DCaqfx+2A18hWtuIYaE9z65BJG361Qc30HS+h8nCk30tDuvaQ0jn+TdMIK8tC21KF9wfh1JdooQh6NrpBxwdx/PfjxO6PReutJX1ROvm78kmYmUDU2Ciixtr7NOwkYXv04dy6XOK/ikeOkvF0pzSS453gzfC3h+MS6EJrK1wNW4Jepudq0lX6+97DwRcOUnykmIUnFyKKUOH+O0LHH6RcTQXVZEydJgRTdx1J/zeqjaxZsIbRAS9K7cLYTtLXNibZtvu3kRGQgYOTA6YOE3WNdQAs27OM3EdzaatqY8+yPQx9bSjTj09nxScrAHjeQQKZubm5UVVXx//EqgENtrFIEEDnrbMy8v9vNEEQKCz8+5j3/5b9J9j3H/vfZ11yX0OODqH+7Cfce/UBPCI92DXFfqC9OfgBnvuylc/CPsY5SHLqdA/KZrUWvZM3ZoNAXW4dzv7OXWgZaeAcu2csydfucJfhfivrpBstigxuJhQSVRRHv2XST92MsnrPepLufENo3YPo4sE9XApa9GTTVftVk98/H78UP46/cxylg5JOeRUtjpcBiL0VS+vqVlLnp1qZVfvSPHDscGTQiUHor37OwT0HcfR0JHZKLCqZI14tI1FYdBicPDE2NvH73VtJnJVoy4slyoi+Hc2oA6OodaqgeNMt0henc+6Nk6ScuUN2jI474bUICoGsz6bj6y19yO5NYFHwXn5dEM0E5ePW9+h+57N1Z0ELE7dPpExeBj38UU36Jjbf3IxckJO1M4trbteYum4qvZf05pNPQCiXJkAv3WCSZjty9LWjWAwWou9P4GhCAogGfprzE8vDHuHg8wcxtBgY/8V4sjxnceQIBNY3o7I40nqjjG+eOsSEbyfQ+4HeVjRfdxLqh8xm1mSuYeCzA6k8VopZ5Uhw50Yi8zfgUldE1KzejHgqHQdnaZfi0uqCWWnm6JCjzJTN5MJXF0i4J4HRH45m61P7aXZtZvD54VRcrLBKIwAounR21GYvBFGGS6DLP+rYu9YUoP7zV7wTvPGM9uTWVkkjOzJ7K/l9ZhFzaT+mWAeOvFrBpVWXeKLqCWperOWs6iwPup5A3umAn2sE8x6azU+LF3Ny5Umuc51eub3ITs+W2rhFwMlJTeqCVHQpOhpKG2jobGD3M7vJ+yOPpZeXcvrj0xx97ShuL7pZZ4Nxn41j3GfjrGU1tui5a9ddVPlVUetrRKVTc+bTM3Q2dErBwB5oVYtoQa1TM2PzDCvKWhRB7+TN9ayHCYvXkb5Ia/1me2Wv8azqWT564iPC616BADmO7o4oHBTETYvj+5XnqdAdIag0iPDiEMx9DYBUR1tHbWViw0SODj3KrG2z8E2RNobhw8N55OYjVqYewJ1Dd7CYpHKWnCih6GBRtzIkokyOf59A8nfnszpjNQtPLaT1RhWNYY1svHsjPk1jWKpWUnO9hqLDRdz19V3sev0iBs8++BadoQPNP6L7RLmISW1ClCvYdO8m2qvbWVG8gkdzH2XiC7uo96pn5UcrUblVwb+g3zLbRrDjhQ4M7QaOhh8ldWgqpg4TW+dtJfr+aIiSZHXKWsoIcglCrpKjcFAgV8jpXkNURwwgXQKNsfSKLXeVIEjSfNVhfYhMsAXPzXozuVtO4Fh/kGBLMH71UmOInWLThv9yxRnqXI6iUczkwefGUnuzlq8TvybrlSyGvjqUpwY+hb5Zcto3i824BrvyV6u9UcvFVRdJvDfRGgDvaanzU+3YkT2tPHYES34egs7HkT8f+ROvWC/6PdbP+u3L40bSf4D9NX+XOYKO2g6S/fqzzrDz3+bi6rYq4AUZ0AHFvz/B0l1vkZE4Br3i5R6IYhkdtR1UG6qtCNNJ301igP5VvG6/ipOlBO/MU1z67hJFh4sY//l4u2ckXQ2hKjeXjG8ymJ07G62o5Q3XdWwToCa0D82+0XhEeTDy/ZGEDJTYiumFv3E5bD7px64RUN5Bv+33EzddCiRapYfkMgozZqDqbOH8Z6eIHRuOSdlkfa7SpGTuT3NxCs+nuNSJK2uvMPK5kdbjke6RBLkE0VQkXWNoNiD62hgTbnePYMhD8VQ9dJq64FQslgAQJTBMtW81vp720kDdDp2AmkEkXqynJbOG1EWxfPChLdCXXvAbvs0TMWsciZrkTp95knxVbZ1tjBlc/yPyuj72MjVddaEyqVi0ZhE1YQXMPiVJrx159Qgu37vg9ZgXpVqpfkRRJKh/EM+1PodCrcB7utRxFAq4vuE6ap2ae7ffi2AWeP/AZgDaNLlsVj/BhM+DKfi6AJcgFy5+e5GMi8mYVO2069rpqGohYX6PnKddQUq5IMexU0l7dQuVlypxj3DvShovlbvGp4ZvHvyGJcVRvOfxHnMPzLWiymXIMSemkq9JxcGmBs7hosNsz91Obn0u/hX+1B6slfRTgOAh4RQmd9Dq0krmuXSu/nKVjIcyMHVI822QQzwyQYUoN1HrJSBYBE6+f5Lo8dH4JPr0AKWIKArySJmbwpS1U6R6/Iv8nkUQuOd3m85n99zb7GGgPNQbp6JyPg3/hbt/v5teUxPo3lR71Xox+Phg2oVScssbyP4+mwGTpQ4c6hrK5aWXcVB0jfXztuKb5Is4oCt/WEcqK8ZMReuuReOtIdA5EK9HJUR8aPlgXOqOEOqValdOUYROVTkWeTsakz8mgxa9YELlqEKhVtDqeAOAVsdsfDsCeLL6Sas8UN8J4wmL/oWWow48dVyNUVTjoXbAVTOWKI9gxha8yx7B8G+DSBMdHJi7YDYhe0OJ8I4AIPquaEwdJvaYbCwuuZMj4SPCKTpYxOmPTtPvMfs7/nU8G3/0BV58vQ33jrM0OB5BMA/h0dxHe9SFVFe34m5hcW3gfkHE0GxAsAg4q91JKVrDlfCF/DF5HekFvzH5ocEUvfkL68at5+KMGIiU2p/bncv4VRRyZ18UCce3UJg6lSMpy2jWXQQg/dJ0asolBnbslFjEXlJBXdt78074BR7qb19uUYQm7QW8yvORrWtFvHcyYj8R86mu4INcj1Jwpja0NwGDIznZnECzTgJH+N70xKHqXQKAkQd90cc38OnhzwgN2vxfBl0fB4qBgUCmAh602IK2azUaDhv0UqAvEzICMrjodZFe83sxKmQUu5ftlqSqvUGQG0FUEHcjjo4NHewavIvwVeFE3B+BzEFO+rUF+JQOJTYtns+iPiNpdhLDXh9GjP5+8uuLyAt8mWoHPVfXXeX0R6dZXrScULcIMnOPUOTzGbf93iaGB9j16C466ztZXrScxPecKGcWX2Wu4dql07Qpx3Pe1wuZMJtgfZRNyq4r2GLcd4QPXzzDipIVeOi88Wj3xqUzmaj8KFpLW2lwbqCpqInw4eFsERZS67pP+u4Ool2gDyC6/nGK3X8iMSeZwx9+SeC++zneIIHTrrYdxEV7Hvz70OQfj0cv+1zV3e3Pt9qXkJs62mPaJcebKPQY32TkDnsQs1kKltmGF+kfG9uXoV0YwWj5+6wZuIZqczV0TaG783fj/YE3nlGevLvpXb7P/J7HSx63scUcivmzrZiUXtFsnbeVAU8MYPSHo1GKUuqHCTsn0NT6GyX9fSk+XsxUptpJ94kyI85xQTyc8zChayWAx942ab66kWrGzfEZlnq1U3S4CO94b9TuTugMkehVlUwxbKaxLIbLK9fRnl8JWc/YBcbq3Qvxd07AYhHJztbx4otyZGOe5Fj7ap4Z+AxD7xpqXdM6qf8us5eQk0DF+goWdC6gzlzH0o9Trcd8m8cTkxRD4+HGv9VFysk86i9twv/cwr8dc+x0JPNEBA3BBVycYGMBW0SzVWos9lYsgh84TvZnvyIK13ab9GVz4wlCsouoPjEU+khInNqOWi5WSuOEe6M75s1mqsdXE9A7AJlcRlubYJUnNQtmOhs7kSvkOLg4WJne4W7hvPLoK2Ab2qQW0mMqEkURrzgvRn88WprDBNE6fnebWS+xfiS2uMzuPo6ejkSNjcIl2MXuGk1bHTf+uEHmk5kMe20Yw14bZpfjK+ONDCI+jEChVhA9LhrveG+uKqWcbiZlAzKlAoVKyh2sddJyfOFxHD7qjoZJZVhetNx6vxu1N8gJfYycUJhwQaTo4CV2LNnB/Xvvx8nPCUGAVu8Iro18HDc3WLTOlvu6Zz7dKv8q9GUOfJXwFe4RKdTET6F77g1V9MfJ0AuDVwX9nhvK1bNSANTJ0Iu44Dj2m/ajSdbgEuzC3Rvuxjvem660nl31ZGsbACdKTvBR62JqE3NxaU9jknjJ7viRp44AkgRs6vFV1HgMxnixRJJvx7Y2LIwsRKmR4R7vT3i4jal/1H8KjZpsLgMDNadoOJaD0NRK3InVnBs71Xqegcs4tMXjGe9LRXQWeicvejrxi0OLGZ4wnHGfSvtbob4LvNbqRMqVFJTOSh648IDte3Y1sLLgBi6Of5HnHpHx6Y5PEBTS75Yu1n2LdyRi70gurj6I4ZSB/UP3k/joKjJaXGlV/UxlghbHNHswaPcYZVaZMWhMWAru8F3yTtw8RtPkJ+27ZCi475f78K+6QMDT4zG013LQ+SAmle37y0QZcqMcwSxweMlhBjYO5OSgk9bjcTfiUG5UUvZAGfJqOQ4GBxQyBWqFGkelIwqZgiEvDZHqGKzB9Z+v/sy6qetIX5xO3LQ4q6xld125DnQlgABOf3Qa7zhv0hels+DYAv7JwoeHo1Ap8E3x5eI3F6nKrgI32/GFJxeidlLjm+xrzb3W085+epYT75zg0bxHGdAjJ7FMLrNjuemdvJAlJ+HoKY3vG6ZsoPR0KUsKn+u+oNt9+Le1VJxXHO/c/w7eWm903joeznkYuVKOzkFHTKUEMD77RwmV8lLSF6Vb0wf1vM/v0zYQOTqScZ+Po/BAIWHDwqzKEj1N6yUB8ApsKd4xGkGplBR89I36rnHI3kV/Y+MNqq5UkfVSFmIXsKgxIIHGgATU3lAZPQSQ7tNdrpyhjyCXQ8HeIxx97SiRoyNRu2lx6UxhYO4JYq/9gVPxDby+mom+TCYBbbpAdc5iAPmD3rV7T780P9pXtFNmLiMrNIsZCTNI80tD56MjYUYCerOe3h9uBgTk3hJQTRAF+q+QFoH563q+kYzwonB8RvhY9/1BbdO45fEhwaXBmLzqKT7WQO62XKuSVDegSkTA0Kzn7OdXCOwTSFD/IKsvSm1UE3O9GkaGMOx1SVGswVDD4cRByEQVM3d8R/CNfbQWLUCUS34Q36ZJpAT14hfFSHTtOlpvtPJx4MfMPzaf0MGhVsWennbsrWOIgoj7ZBtYsnR3DsdnHMY3xZe8K3k4T3Cm1cUGIEh8IJM5s5L5Mu5Lei/tTd/nRlLrsg9kIpunb2ag8yY6bhVYmbRGi5FWx2u41RcTfjsRUx8dj+X1ACh0BWZFuUhJaAlGVyWXvrvEke+O4JLpQourxNidsm4KP2f+zMTeE/k2/VvGy8ezatEq8urz+PbCt9wfcz/zj87HNdQVNxc3hoUNo7aj1irvOWPBAlZ98MH/TAofSMm43/p3N7PPbDCjdlL/bY35H/t/3v4T7PuP/a+wI68c4c7lOyRtS6JbNqjVuRVFiK+VBaJQ/D0RqM7PmVeEV6x/d09YDYFJNAQm0Vxykx8G/M60X6aRNDvJ6qjKi8nDRTXITiav5yYwseQLu0neK9aL1U9kU+68jeSC/vjfPob+ii/QjYa3XXthyQW8tJJjKvu7bBxcHBDH2gp+OfUyd919F+MWjUOtU9PRpRiiNqoZeGogJk0xM9dOscrseKtC6VSVIigbMQqX6ajx5Obmm1ZWk6QJLeNG/A2qfatZu+JP5q2TGBeX/sin6swdtB1a+uWtwL0uEadgd7y6lLVCHdKIrniR2wFvcsf3C5TZy/k4cDtjPx3LmI/HMOyNYbz2oZSDJ+NKBvIQadF5a9st3MLcqA6o5vlDzyNTyPh58c+ceeqMXV0IcgkhaPB2ZeLLY3hb+zYlx0uImB2LRSG9eGFkIQovLwp/LKS9tp3xX4zHWelBWO3D4Aw3smDQ0EYiwgUCMiR6Z89NNYDcyYHRH48mICMAtb8XP35nRpQrKEqbBqKA7+L1OHuqmLNvDo5POPJd5Xc46Zx48rsnyfTJRGVQoXHTEDY0jBcOBRPYHEnfPbkUZhba5cTrrmfnVmccq4vobPCzLgq76wLA6OiCOTGVu75KJXRwKFVXqvg29VvkgpkWx+vkZ8ykTy8/olMkNIzGVUPIpx9y4NJaHFxuEFIxkkfSP8ev9Bz6XD1jPh7Dk6pnqPGtJLgkmPTLw2gd0UTQID8mr5nMzdqb8JX0bJPKJDmaRRHfFF9S5qWwS7cL/gEUf/iVw9zZdYfy8BAMDiIavSuCRSBnQw7Nxc3SxrdrgXXP7/cQWH8AhfohOzk4UZTQaXpnb0Qt1OTU0FLaQvhwKU+F2qSmd/ZY+l8IQD6sgFnbZiGYuxYwXf0x7mYcA08l0zayioL9ZgJ6B5CyIoVjbx3jRuwNNrhuYKJ6Immk4eDsgEOsPbT0gUsPWBcYs3fORhRE5i+AhgZAJiPj0zl4fiExVX+f/jv99j7Iq4f8EBQCRgczgiBj+q+StJBZb+a7mxVcrT2ONnYgbh0ZtFe38dnQNaQtSmPwc4PJup5DpesafKuDUOlbrfJlVZer8I73plMnvdexIcdI19o2SAX7Cvh5zM94xnqy5OoSfAb44BnjiWgWmfzjZCx+FjgDvlW+vLPyHd559R2yXsqyIgaV/82MLopgUWkoTRyPeyZk/5DNpdWXGPneSEo/OspUhtOubUcuSAHMnpIa8WUfYlK+gMUlGr9UX5pLm0manWTNLQpQm1PL2mFrGf3xaAY8PoDyc+UUHigkeU4yrsGuNBU3cfbTs/gmS2zXhvwGcrfnEjUuyprDqqflbs+ludIZCEBQqlG6qJErIPv7bCJGRtDvsX52Dh9BgMs/Xmbbgm3M/nM2zumSw8mhrQ6/wtM0X0li0elF3FV1F6FBW/5L5/AnwLNAfxHe6gA/mqjqbGLN4dUck63Cu/cUCINtuduY+etM0irT2DBmg11ZroU8ijGihnumvEvzy0e58fsNxn8+HkEA/4YZVHr8TlFsEHMenYjn+CZaqlpwkbtY55ZOVz86Xf1wDoCBT0lJ5rvnOJmooDAiHwfnaGuuGIBgx15EVj6Ng+8dap0PENoQwbEX9uOoHWPfFmQiHg0emH0MTPh2FoYR41i19iKopUDf1fuukrcjD+0yLVHjoggdEoq4VypYjetuNleWIauMxaf4Aq2eIQhCAMhERLnI2vlr+Tb6XbYt3IbaWc24T8dZHSdNbh3cHLiIcZNcMAttdmVy0sei1oPSWI+p0w3cu3LI9Zg/U0/7ISu/QUOOO3cqDIQNDbM5TpRm9ozZQ3DbXdbz0xalYTaYmTdmHnOK5pD1QBZ3Tt1Bvlduh/4Hqb3vW7kP11DXrpwZkOI2hD63d9CsvcilwFfR5g1iSsEU9I16+j7al88qvuZi74s8/NXDVKYfhfkRPdqAVO4q/yrWLj7HSN/ZrOq9ihmbZhA3LU4qtwxMahNV/lUIBif80/xRaVUIrZJDxyIzsls7G4VPPw5OEMhN9+G+XfdxsfIi/zr7LwAGHx/MtVXXaP6xmem/Tqe1WaBG/Rxyi5x+x9M5V3eOxWds8k6Oche8WodR67IPudlIVXYVB545gM5HZxfsExVK2la+zJQPbd8oQBfKgFvHuBg+kojCXpjD6unpuekesxX+I6hMepjExCrC/YtZnrMcN8HDelzXriPlagqGPk0MfW0o/Zb345b2Fu5qdwKdA6neV82tbbcY9vowJnw7QWL2HfoUgNiyd1gRM5jGa41SrlxR4KOVH+HY6UhkwyBuJWfxRN94zn5+FoVKQcbSDACyI2YRnttJ/MV7yf1jBPse3sakNZNIW5BmLf/GezYy/84f9PSvR+nSOFLxKp0jX6VzgDOpN08w5sAWKh2iKOx1Ny1u1xh4YQtZchULLca/Me/fffttHl+50q6tdTsh9q/ZbPtRpWTugbnse3Ifpz8+TeGBQuSi0spn/KuDShShQ30HS+tzjN11F819Q2G0Lf+TRuzpaBIImZjChBckyHddHQTXL+BW0LNUBFYQ1eRG7md7KdmdT3HieLrgzuTrT9O3YzuB5SJOwRmU9xpGp4uv3Rpr0/RNfCR8RMkXJex7Yh/OO6QAQHiBH7VfvMd1j7vQeetw8nfCO84bUYSroQ8Q3SEnMXsqpg4TBrO0AHk283kO52WT7/cebu19SAt/F07AyTjJWXT/L5ITYcOMDWSn3SLKzQk/Pw9J7vyFF9hh+PdB17VyNU7pSRx3vkjxEQ/0WHAUVUTqk5jtHcS6TMkTVdRYxIWKC4yMGMmCiAWc/+I8rsNdoTs9j6hAZVIhNoq4hbrRnNVMq3MrDgY5ftVBhOZdxbXVFbWTGoWD5JwymbpysyA507rzzjp6OtLRpbZ02/9N/CocMRT2Z/ALg63Mq239D3P97FGGnYc3AT8zVJXX8Z38bY7cfg0RFQyCnZ+vpPe5vsgmTCDj4QzUTmrau+cLFKx6YBUbZ2zkSpIUOJ24aqLkLBIlxrXlpsCViivU3qgl66UsVFoVoiCVv9KvmvDZ/dmzbA8V4RXWibNTXYpbh0TjMuvNtFXpcXB1QOWosvb1w8MOU5Hpz413Xkb+upx5r0uMMbXJC7XghqiUo+hsY+8DB6mSdVDt6kZC6af0qniDkrSFFHIag0GSQDKa/y4DJYgCSfcnWXOt/jUPq6iXynHmkzOM+mAUOnxoAKrC00iL6M99P2TY5pAezD7/EiW1x2/h9UYSndpO7E2qy/JTxexe8AfT108nemoiQ3NuAhDuU4eqogWXhCBcQlzBYi8VdmjEIdSax/g163sOmjMgHK7d6qDFp4U2YxvJy5JRyVWUny8neX0ytyNvU+1nAwYcHnaY6S9PR6FWMOu3WbSKtmN+zVOYuX4JA16zOSgjOu7lhvPnOLc6I+rt38Vd7YvC4kSLSwvr7z/Dm8lzKf5oH36iH1X+VVgwE1bzKI26E8zaMAshTcn6Pl9gUFVT47YLjTEQvbqcwyxifsV8in4+RWl/DV4ZXnYsuPyofOST5OxYvIPgAcF4x3tbxxG/Sj8KthVw5okzuIW7sfjMYms9djMG/2o9x8N9T+zjzCe2/eXSK0uta9Nv075F46ZB56sjZ0MOL5lfQiaTYVI0gdDJzc1NeEW7c+/2e/mrVUUOZFXOQLvAodFiY7bFJ8Tj7CAFQmZtm8XnZz+nsdIWbGnMr0e4007WyxIoznpdTjxj9t1Fo9cdZCPCrL/XtzfZPT98eDjTf5tufZf/KmeftyaAkVfKOZASiIPeAUQTg18cwh/HJV9Ad3/UtCjwyz+BGBNBxsosvuwRo2lrtbEENa4aEmZI4KWhx6dxpEaaqwSZPbNv8A+DbWWSmf+WW3bKtinM/nk2d8LuIBfkzIq9m4f+GG67pkdArjuFRU/rbiNxN+IILCy21kWTT4zdPJR+KZ0RhwJRPehARa+hdtcCnM48zUcLP+rx/bpAKc2ujDowirZg+/Vot8/HqGrAqGqlvVaJMk+J2qDG6GBEEG0yeYIAedvzCKsOg6GwbfI2Uq70psxPT1zFO8iB7Uu24xXr1aWaZFORcq9TICpNqHQOiDIb60MmKrgdfRtR05/24nq8J3qzW2svyVgYWcjB1w/ywaQP+H3G74SEhnASW/vT6DXIy+QE9A6g/qd6rp6/yhTZFL7o/wWvuL6CT4QPutdtjBtXBwmkGVYUxrkvzpG+ON0alJW+iTT+hj0RxqjIUfR/vL9dTr9/srCsMMKywig/V87tXbcJGRKC0xjbAqubJfdvrx8WBjJbkKzbzAYzNddruph+LrT4REGfKLy73A8RoyJwi3CztielsQNNWx16Jy9M9WbOfXGL0KxQfJN8CXYNZmmGDRjbHdzstmrXnWw/sJnYA6E05DdQca6Ce3feiyjawAETv5uES6AzlZcq2bFkB9N+nWYN9jWXNLNp9iZS5qYQnClJYur1tmtbylo49PExqq9WU3a6jAezHwT8QBCIPvcLx98Oo+5WHVfXXWXIC0P+cS3YbSrV38eJtEVpRI6O/BuzuTVhANWeCXZAGyurviuvqa6xjCMv3qLfw+m0lLeh+5eOqLFRDJo6CN9PfLl46SIDbklBWI1Sg3+TFJgb4ufMpDMDaRzfiD5Pj8ZVQ5WxgALfrWiMgchEKeff9ZPXGbdQCsAn17zDLY8Pefe5d/kqtJ6773NF7AJNVrVVUuO6k8j8SEYe8qDK/RY5b+1h8AuDCeofZO3rHdoOvnjkC+4O+Nj6nibBRLvmNjJBSZNPDEaNC9rXtkFQEHhOxdEURLQiiBjPGPaO3ctDyx5CcVSBs78zglnAYrKgUCtoLGzk1pZbRN8VzfVfryOKIoMm2YJ9AjIUagUWgwVZ51/yrAAajZSuRN+k5+S7J+nz7EhUFjdMykbiSz7B3Etk4qqJ1vP31/7A7YA3UfgoqAmJZbZrqH29d+0OVCYPLPImRLOFtqo2OnI7kPe3zZsuLS7kjs/F82lP4u+JZ891m0Le0j+XMjpyNOFDwqm6XMWpP07xmvE1WitbiXKWWIyPr1xJyEcfcVYQ/nsVE5mMt+95lWNdasMWC5z97CwHnjnA4nOLCezzd+brf+z/WftPePU/9r/Cys+WU3tGomjLuqA52enZ6FZMoepyFTm/5yD/S2vWNlVQfKzY7re/TpAeMZ4Men4Q3gndu3hpsriedJ28TBcufHOBs5+d7bpWutjblM6ILbeI3P0FXyd9zfXfrkvHu3NMCBYCcw/TeekGOb/n8K+wf1Fy1Jazz+OOByfvO0nhwUIWHF/AvTvvtT7Xtb0Pax5ew5Q5U6wyJ91lbnVu5V/L/4XjmMGEDgkleIBNRsCkrCe8yIvMXdvorGrhFeEVhrw4xPpMuehAs1sHhZFluHv6SFrlchnJj2Xx1dJvKIgsYNif4aScykPf2ElHvRRo81f1Irx6JWqDmuQrycjra3ANdUWlk6CYooNoXag/WP4g036dhqnTxB93/8H5L87TqJeQpaJcpCi4CM8YTyovVXLp+0sI7Z1WGc8bil85XX2a59ufZ+LqiXaB1e5vv/jsYpYVLPvHelT7upP1Uhb+6f52dZF8JZkHv3kQQ00rAx4fQOjgUHyzwqjxNdCkOUSDy01anWoxtuppLpFyTAx4bACT5k3ii/FfMD56PG6ublY5OQCNyR8dU9E+PI+k2UnsXLqTjbM2AjZJy/ACT8KO/MTR14/ygc8HVuae1Ynv4oth3GQC+wRSfLwYjauGiFdTAcgNeIpj/T4BDx2xk2MZ/uZwlBqlFLQVpYTHUzbFYyiu4uI3F6nNqcXJz4nisGI6dB1413qTdimCjop26/fpmSsienk0D156EI2rhripcUz5cQoGN1ukrzK7kpzfczDrzeRsyKHmQglObWomHNvIik/uofRkKbO2zWLJhSV29xatUkX2JoqAIKA0tCHoDXyd+DW/jPuFjTM3onw/gZ137aRTa8SxpRqxXWp3cqWcxqJGHvygP098+AQXe1/k95nHaM+r4OfRP1OZXUnS8iTefe5dqv2qeeXIK1yqlNCkpk4TbVVtmA22jdhfkUQyucwuMKbXw4SvJ+Cb7EvctDjqjuWz7LNlpFxOwatai8Vsa4+1N2vx/ywbp6rNVHisR5AZEQURB2db7jlnfTypuSsZsbMZz7Ir1rw936Z9y/XfrhOc34jKqOJCnwtoMiRH2Y4HdpC3M4+OIR28PeRtNG9rGL9fgrArHZSkzkul95jefDz6Y0JKQvD53sea267bupl94Zc2sW3hNjrqOzjz6RlJjhD7fiMIEluruaQZtZMa9Qg/CiIKODDyAGcHN2CxwO5lu1k7fG3XOyXg0TYIB7PkdHANdmXaL9OImxZHq6GV7bnbef7G8+ie1HUlTJdYlIdeOERLqYQuCx8WzuNlj1tzGVRfq2bfE/skpCdSYvMDzx2gubQZURT5bfJvlPwqbVxV+lYacmuwmCw8WfWkNa9L9ztFnv+Nps2HcA1xJW56HE5+tnwKan0b3iWXaL8jsU/8/Px49733GKuQ/2Mep0+At4FjwFHgPmBE1/8PYuaEaKHj0ja6UoMye/NsXFJcuGfjPdZ+IYpQ47qDSo8/aDTUM+fAHJbfsSG4u3NwNXmBd2YUFpnUj8payripP4RJYZ/zpttsdSjn5KCT3Bhr37bDdUn0qniTG/Lf8cz/BMfmIu7Zs5CEmQl251mUFj586kNqRkejdFDy8+8O5Bd2MdFkclpKW9i2YBtV2VX0X9Ef/3R/67ha5rmO4+VvU9iey7Vhj9HkG9uVZ9XWTyyCSNmZMiovSvmbulHnRodOWjy8qb9axg8BPxB708YeNSrr8Cg5TtKRL8ndeI2T75+kuaTZypAD0NTW4FqTx+WPDrJ+0vquewvWd7qaXo1e68K+J/dRcrIE12BXznx8htMfneb1oa/jkeVBdL9oDC0GKi5UUFteR7HXKkq8vuNk9FAO3XuIvm/2pfZmLVWXq/BUB+DbPAH3toGojCp0lTru33s/A1YOIPn+ZKpTvqbes5mTA0+i6RfO9iXbWTd6nfUbWOtNZsE52o/hbw+3rjes7FBBhoPeAXOSN3MPzsUr1stuDjSXnCP2Uj6OAe5Wp19Ph/a5vufQhGmou1mHxWTh5yGrmPlrb9ya3Fh/73omr5n8Dy1JgVyQM/HXYi59n83ic4utAc6/jhMXV1+0roMc5Fo82wYz6VQpszZMR5ldypV1V7iyrkuWVrS1IQBloB++T/pyUHaQM+WnrGudkpAS3nzhTRxHR+IS6IJvki+DwgZxauopjs0/RmV2JZfXXEbfqCd+ejwRIyPwMfbHq2UkarMXOb9fZ1X6Kv58+E/2996PS4sLNb41nI4byenYIZjNImc/PcvFbyVWybm2TTQ4n8C90R2P2g4cvXSkLUrDM7oHXRLbK1RcrKAhvwGAEG0cC50duRACC0p24WToxYqqZylMl2SXDENqePh1kcCJsTzhFMf9Kk9exIuMB1awO3sfL3S+SORnkX97Ts969GgdjChI3yxsaBhuoW5YTBZkPTCZf133nH5oLcM3bsKisHA9MReVm45r669x58gdAMbo1xBevZzMk5mM3C7H2Gqb523NS2LsIAqUbJCc5XXBaXQzIiyY+WXaa/y4tByf3sFURg+h08XXTh6t0aMRn2lSjiyPaA9rXzc4mFEE+6PUKPlpxE+cel8aMI+Ir9Oiy+ZixkVuLk8h548c3Ke449TqhEqu5kLkNOpdDlHs8xWWDgOhV3eQcN02fjW7NJPbK5e9Y08i1zmye9luxkXexXNvv81AGQxRwi/AAeBnYLhMxSAUBPR6h8F+F3AY7IZsnjOD+29mAY8yhCwclTZQllqhJvROKL0+70VjYSPzC+bzQ981PRqIgqspV5H/Jmfw84OpeLSCGt8aZCioC04je/RThI/rxdLLSxnygrQOb7ZU0qkuZtDxQcx6PxiFg4I+D/fBwdnBWq8yUcG43eNw33WRuKlxJN2bxCcffUTRheMcF2GvyX4uOizoOSFa0B3WwylocK2nMrABZWoid315F47ujtyoy6HQ92ManU4hKAT0VTY2u+Q0lx6eeD0R9WMyTrx9gpPvnsRsMCOKIg2OUr+vCCwjanEWDQUN+FbbAD4WeSfOtQWEXN3J9Z8u8ZH/R3/bE4lyEUEmWuUzu230lVo6lJWc9PdH1r6b25uvkV+7hvPRE1EIWpz0sTgrvAgoD8Alv51FpxZxdPBfg24wcOdADC0G655ndvALDLx5hoD62bg0u6DY18UGEESMbUaSVdOJqHqCFr9++N0lAQG6y9Yz2Jd4xYHcD3fQUtWC0vQXJJXxGm6VN3EKcmPcF+Pw7+1v1z/Vl8/R6+w6/Cb2Ifml7vH3r1EMM4YWPYquOuj+Xu+ceAfnd5x56/hb1OfW47PfhxkeM+wubXJvQhOn4eovV/E8bj+GORpCqa0FjVFST5iTPIf+jZ/hYPTjxwU/YnpsEKv7ruZfYf8CIMIpgbSidZhVZqr9mjB3Gqn6pspaz9stD1Pg9yGiTGTTtE04GDQEPqNB2y4530deLWN88WVKQkt477lVNGTf4ZcXfkH7tpYNORIAKtwtnB3zdzBo2iCS7kvCwbULkNc1H2RcyODgnIPETIohdqq0JugZ7KvMruTzmM9Z1XsVdbckObGe20S/Huk4UualoHG3yYj5JPrgGetJ6JBQUuamIJPLEEWRvWnunIyJYdOM37n0vT0bzWoyGY1NUtsoPVXK5bWXsRgtPNLnEZakL4EOSWaw2z+wbM8yu8svf3aMHwb/YN2PTN0gOcItCgudjgZQyCk9XWrNMe7vIM0Ti75bRNLRL3CPcCdxZqKddKlK34pn6WUUTXWc+/IcuTtype+EAo0pAJmoYOXHK+l1YS+DXhpGW3BX9KNrzL5dt4WgWwdxqC6xA0jn+73L1aITLPt0GQ3fNFh///Xar+S1Zlv/tmDP7OtprdprfwvWxY6PxS1sHh26Dg6MOoAmOcTueHc9x+TGMGQf3Fhzhre0b9nWFF39o8/5PkRdLSDk6ZnIH3uY0sRxyEQVzh2SXF1xaDEX+tej8bIFksJrlrHhbhsIr2xjGRumbpDkWLvuW+NTw6olq9AP15PzR44kJQkoulRkBLGSesWLnPnoFDEvxOBTI811lq5gn0vNbWSbNjL156n8ulzKD1oYWciWaX9Qr9tI2OVtGM5f5dov1yRVGSBY0ReQGLrTfwtFVKmZfexBmn2jERE4mpBIs+6StMbPCKJ891Xq9v2zjF7395tWMo3fZv1mdyw7PZvan2sJzgzm4zEf0/ZcG88Pfp7CA4X8NPwnW767LutmESfkJLD7sd20lLfQdKfJ2n6v1Uj5J1fsXQGAZ7QnbqFumA1mjr5+lNztuf9YRoCiQ0UMe3MYqQtT0Wh6yPxZBEnicX8BB547YJcTEyByVCQj3hpB0cEivk37lvLz5YCUcmR1xmrOfvrP2Xgzn8xkwtcTrG3cua6I2FM/4FxXhKm6nt2P7ab4WDELti0g5vMYfvjgB95xfof8vfnU366ntaIVg9lAi+M16lwOsitpE3lv5uHs74yxzcjhk81sOn2BVo2kCpE4O5nw4eEEDwzmvt33EZYVZntHs0BDfgPtte18nfQ1+57YZ9f3Ouo7ufjtRZQOSgY+M9Da35GBrqmchoIGRr43kkduPoJCrbDONa7VuYRd3krjrWpiT67Bp1Baw/U87lKTj2uwK8GZwRKAR4Qin88o8P2A1gBHGgPiObFoLbee/E66tmsvKpPJca3Ow7fwNOc+PsnVY808/44zlaM6qAioQCFToPPV4RriStGhIj7w/oCrP1+1vlO662j6J/THO8qb7Qu3s/OhnZQZr3Mz+EmKfD8FZBwedhhzm5nsNV3jSw8pdEEUUKgUVkLHxZrTNDmdQ6PX4N6gRKlzYN6ReaQtksB63fOnoBCo866Dk8WsHbYWi8liBV6KcjOHe0/jaP/PcfBwoqfjWBRB0RVsdxvgxoRvJuAR5cH+p/fztvZtjr15jNobtRx45gBV2VXMPzafBccW2M37PkPjeejaQ8zaNou+R/rS6tKKv8Km7qUvKLP6O/o+1lcCVYtKtO1aBh12x3zlhl0b7h6jLEoLJpUZsb2Di6suUn1NAvcoBSeUZheUZg0vv/Ey3r82MOTFISSdTqLJvcl6n07HTo4NPkb48HDGfjKWsyPs+4xZMCMKIiUnStj/1H7OfXOO81+ctwLv/fz8eO/99xmvUPzb3NdngfEKBe9/+CEuLrb52GIBnyQfUhem/l8t5dlzzPt/2/7D7PuP/a+wufvn9virh/yGaOHYm8doLW9FOUhyBlhknVgU7cTm7GHTPbXM3DQD1xBXPKI8rIOyrrEMl9p8VNo0Rrw1wnq/vyatvfjNRURBpN+yfrZEvTI5Bp0nCixYTG0IFoFrv17jgY96czJTz4GRj1D1oC+L/XZIyVy9dbg4uWB6yYQgCtzZc4eiQ0Uk3ptIxIhuFoB0b/e2fkyOlTaGP2b9SGhWKH2fkRDggkKgyb0JmdbmlABpgpKhoN6zntKoSAYF28sGCALIRTXjL0mbfPldzRQVFUksNxdHavykiacuKRVTqwv77l+H0NHJ8sLlXfeWo+3QMm3LNNpSmll0eZH13p1m28bbzdMNlVKFxWRh+vrpuIS6EPpTaPfHRLSIWEwWcrfncvS1o2hWBiF2yVHUNe1n20J/gl8KJnxYOIKhC4HYqeHp95+mdcAOZI9Osup557Sc4lzUO3jVepFyYxr66iGA69/qUWFRoNFrUQgq67HL60/T58A5vl/4PS0uLYRVzWbYmvcIC5WuGRw6mMGhElJx7fC1eMd7k/FQBu7h7qwZtIaM7Co2zNyOt+taXEPCaCxotC5OuxcUxaHFFPbOYGpGAFXZVaid7Rkk3fXyluNbAHj28iR8XX8WGe/BorAgKG5jEj7Bqm8DND/+GuMc4JfZOXQ4uzMnOYanap9CpVXR2dBJaHEIdV41XEu6hqPqWRb3C6W5pJlDLxxCM8I2yXSYOv5aFDvkbPb32Zz/8jxPVD7BknNL+H70r5RUX+Za1GEcW8aj9bHX3k50GEdaoQ8VSSFYDANpq27ji15fkLE0g5HvjkQUQdPRQOKRL9G3DEaulNifHfUdiCE+XOhzAbnggJPybfpHxVF0qAitl1bS2we0nS40eDZg0lbjHBTFhG/VeMV60SxKwZDwwnBm/TaLFqEF0uHmpptsmbOFmVtnEjtZchrU3qylpbSFyNGRVF6q7HJk2dCEt747juWMg1XyctWc7Wg7tAw8ORCfWh8MzxspPV2LqcOET4IPRo1AjU8NjnV7oS0djfd9XUg8mxk1LpTFjqTVM5T5RwdRsL+A8nPlbJ23ld7AzZhpKJRJhPSSgn13Dt9B66Wl/rl6KrMrcexwZPHri9lduNsuCfSFygvcir1FjU8Njyc/Tt6feTQWNNL3sb4oupIpO3Q00Vysoq2qjb0r9pL1SpaEfBNBbjYSfnkLtUERTPy6v1VeQ/NoNOu63j+wLhGzGfSNetqrbUHja8EPc/93XqydGcm8DfOsv+fW5zL5N2nc2ui2kSdTpYTxyXOSCRsWZnWoKzVKXAJtaMawrDAWnlqIR5Q0ZpWfLefkuyeJnRKLS5ALU36aQkGdKxwCv4KTHLnnLOnlK3EOsKEuux0LuqZyTJVywocPt7JGKyRCJW0ewWSPfopQBzXXN1xH46rh8See4OMDvzJw7yVJ1q0rF9ct4GngBP+e9dcP2CuYGbgfLEmQUJxA2cky3DRuVFdXI7aI/KlbRqdDCT7VPtR+mUO1Z4AV0SYt8LvlSCxUHLrJldcPEDs+lltxt/ikYQQDNadIvNiB953zFB6YwcHnDtDnkT4kzUnjUGIkHZpC6f0R2Dp/K1WXq1h6eandvfue70u9fwV+fUbg/Hc1Gel6UaA+rx5ts5F6p25opxz3CHfu3XkvGjcNjYWNuIW72ZxvopLkq8lwtBXZEDOiQonFApmVv3AscApZR7LorMpj5Y1H7J4D0OJ4kVsBy1DqnsGnnw8dWtt4dKbXSKrUYYyyfEtTUQOX/nWCwL6BqOI0qMzumJSNnJzajKxyGK9leeKxUJrzPRSh+DZNpG/AAMQLz+HUeofTH53GLdyNoP5BTFg1AddgV8LDwxl2QJpPCw8Usm7UOgZ8PIhrYQ+CKEeOkj9lRr7J+IY/J/5J7c1aMrdIOZq658EBqwdw2fkyYz4ag6nThKqzE5VJzeHhh5nZ60cMm8/S2SDNi5HaNFw6UjCRR9x1Hyx9LGQ91wMF37WR86nx4aFvHsI4vghWZtp9L4Dg0v8Pe/8dHNXRrX3Dv8kzGuWccxZC5AwiZ5NzMGDANjZO2DjbOOeAs3HGYMCYnHPOGSQQklDOOU+e2d8fW5qR4D7nud+v6n3eOnXurrILzU69u3t3r17rWtcVQucb3iRuGMvw2b4PHM+PyKf7m90ZHinSsE7cPIenVi2i1rOWMcK3+CT6sGPBDixGC9M2TaPKWEyV234QoCBKw5jxsR3Qlu2ddbLCXPZ+vRe3ELcOVMNmtSt5XSaRkOjHmQ+3IpFISJmfQpDQD/+6fNzLdQTc/gVTykS2390OQElTCW0YQ0EqYJFaECRgbDRiNVvReGqIj4/ncM5h/un0Dz1O9sA7wdv+zH5131NZCc2qTPTRlaS+l4pCqaA4o5igkiCcm50pDi4moCwAg1c9c/fPtTvxy8wi4ObIiCPURC7muWExdJke88D3EJsZi3P+dX7ts4eYsTHM2jkLQQA/bT5Ndc5IzuTi5uWKIMTZr9HKzHwdCUTeItb3TUbnzmJoWCYDVw6kQFaA3qKnoqyC7e9vJ6xPGN2GdePoq0eRyCRYQkUnmkSQo6/R8bbkU9xC3Xji9hMonZWEHZpFtvv3D9QToPZKPgAZCRmkpeTzaJgv28f9hDEokjfvhrcOMinOzc641UtoyCzn2t0aYsfH0mKRUOF2EqOinCU/L8G79iAyjYLEKfFcaZB3yEKyyWxYZTZAgq8vVFZ2tJXVejWCRGDqpql4x3uzvSQLt5YeZMQd5JuIJgICLIz9bize8WJfGoUWuxlvFax4xXihT9ZjlVlRyVVYZeKaM/hYb8xnP8IHCPUNIa3zdT566aNWG8kGVhtWnZFL31xCppLx3Kcr+LDgb06nXSL3SgAmnRlXbx8WLZ+D7OQS1GrRASFBSkF4AZENPiR9MJvuYxR0OdDFzm6glClxa3DD57IPTaVNyKJkGNWOQKmbMYlRUaOI9BDtd/veQJBS53Ido6KcopbOuHs6UNgn3ZZSpNmLV+kIaqI8Ubs7bLJblTc4mDIcs6KG40OOM9woIrvLy8t5+aWXOGX9r9HUvYFDVuh/GC6vuEx2opTEdnuEa+WXuBPyPCAyArRcb8E5wJnez4h3bOtHtWwp/VbF02lCJEoXpZ1G8XqICF6xyBuQa9W8YXoDyduOPdiNyHm4F00jrrATUsV4ei7v2S5LX5zf/Co7E1fTneu/XyfxdiJ3khwOLSlSJuyagE91LtMzn+Sd9e+0tqVo+0okUlJPphKTIwKbOhV+g1RQUOb5j/0ecbfiyLZk2wOr/qoIPFoiKDeFYBQUyK7LcA93J3KkSLfYjcVIih1tWH6zHFOzidD+oXYqfoCrvZvowUBWB66m08RO3Oh6w/HMzDiir27Gal5MrydFx30bK3i90xVudLqMUuFHH43K/nv7byY2MxbPinrmnX+SY0vloqKfxOH97Xe2H5ILEhK3JxI5IhKdUsc3X39jPz4taho+gg9nPzpLUlMS2xaLWVeeTYMwyMvZkv8DIdWLiKh4hh9ftvHUHkemhsVmJWJoBMZGcUy32QwSmwSFSYI2LoTovdHcPS3OmeXcAC2EycHULR2ZMRjVTQeAta0nWzuO5Len8Wy+mKlxOPcwACn+KQz1H0pDrwZcJrrYx3945VNkB77D9a7Xee7x5xg0bxBydceM/uzabH458Au27FZ2gKJ6CAWbzbE2pMxPwS3Ujb3L9hI3Ma4DjXwbBRxAzydaxQRbx6ZBbaD/D4Px0bpz4LkDJM9JJqhnEOXuO8kMfJOgigG8uWAln63z5/rv17n+y3VemvgSbw9+G6PVyKEnD5G+Lp3nip7j3OfniCuPIzPeEfAIG59MTGoAuYdzuXHtBjuFnSCFzPhM3KRfMikxlM1TPscj0oNHzj5Cc6P47sXBxWiVE/hXRd1URcTNnZQrxnHo+UNEjYgi7qE4u39DipwbXW7grR+I1doeJCL+o8KvgnUPH2Ns1li2P/QbGs1Y9K7+5Ph/gsIoYFQZEZQCpVdK+WvMXxwfepzSxDz78+/P7Lu/3A9KMVWa6HFrNOmJ79Po1ojFZiPncA4qVxXBvYPt/exR50FQgQSbTQS83K/9eWD0Abrem0JMVSPW26VIlW5oCCL1zi1qYr/kfNgKrM61HF2+nchsJ3K7TcWjpQ8zkvqgN+up/LuSO2/dQSKVMPKLkdha7V2z0kxpUCk2Dxv7Fu3DO96bqJFRdHMfRkLRJzQL3zB+mw/NM5oonFFIg5u497S10niqW2qR3rmNsaEnJpkJHDhTzLIyvItvYMqHl6tfdFCDy5ZSmB2ExHcjad1UJLt5ODKrsNGkuQ1AiDUVldkX7742SnZdY6ZhJtumbMOsFPfHbvVuhGWFUT24Gou7BUF6X+MjZuM1ljSSfyKfkL4haCO1BPUKYuz3Y/FN8mXtkLVEDItg0OuDmJE0gy8ufMGFPhf49rNvSduQxok3T7D0ytIOLEYhh0I4mHOQ/iv7izISciknVp2gyyNdiJsQ1+H55788T8bWDIrOioHFQa8NIt7HAfJ7V/Eu8ZPi8e3ky9mPzpLycMoDWXwAVrNVtBVN4jyp9lAz9IOhoiREBjjVl2DbcZXScT06Mi61tqvO1Z/CxFHoXP2RB2tZcHwBnjGeFB0tIrs2myuSK8R3i0ftrubHlB+JGBJBjz97cCqps/1ezVHNDJ41mMFvDab7w5u5ljgTz6ZU+mWesI97Zz/nB3TdPCI9eKH8BQSbgM1iwzfJF2O7YJ9HrA8v1ryIXCO3S+QAIJFyY/TLvNsqpW6nU219J6fGCryLb6KvTkapq0NuEvdTbXUJTT+ARanBZonEZrW1BgolZAWuwiyvJ4MX6WJah1bmga1evKnWEEVk+Upy/T8l8M56pFJ/Fl1/ire/cEFak0/AZSe8JnpR0FDAkGeG0NulN4p7ChHE6AwlnhuQCEpstmkMeGkAA14awJqua0QQdidxXpYKciRIKQkqwVRjovRqKV0f6YpVoqP7va34F5mQDGyiKkOkng/uE2zX+b0XW8vV0csZN9CX8AGOpmoPntS2aFGV2qgoqgChI8VxrcspbBIDvdf8SFoasBMaNTe5LJwnozoDn0ofyraXoV+gR+OpsfvvWipaCBsUxpJLS/CI9MDJq3WMtrMlLO2/fav4jcpw+P+M6fcoOlfEsrRl+HbypbZWrLfSpCQwLx9bcAD5J7Q0lTWRPDvZ/s5OLU74VCqRWMvY8+0eRn05Cr9kPwbduYFJVsOphM6kdUqjU+RCsX3vy4Y3aAwcG3aM6FHiuFTKOvokG3IaeKf3O3RZ1IWe23sy5+QcQr1DeVPzpv2c555/npMnTtBvzx5CEb2TbcI6JqAIeGjsWPyO+pNz9nMY+Lx9rMaMiSFmzIP7rf8tRSaTER8f/38+8f+l8p9g33/K/4his9moq6vDw8ORgp568VNs3KT/S/1Ru6m52gq8PRc/kBZ1Fp4lJxgxvJE/h/1Jv5X9GPHJiA7BvqCskzQUReIb69gUyGxOWGhg9o4tBFfqmHZwFHKN+JkEqKKJqHgWi2caJ0c7E6abzCetPPYZ20Tqlr4XBnFs2FmR+sRVQ9yEuA6Gz73ae2zUbsTvtB+de3SmPr9e3NjYHStSTn94mhNvnsBmseER5TAAJTYJbg1uWFU6vor8GddgVxadWkSu7gZGRQUV/pDVLRFNoAd5x/LwjPbELdTxbj75lwi5c4jz5hgKDt5lycUlpG/L562P3+KXxb+Q30kPFTFMiU7ERS0uUnWmCuq0N2jRtrBx1kaG2Bw6DwB6syPYN+eTOTw58EmGDhxKp1mdRJ3CQ45z33z3Tbbc2cKIj0YQ3CeYrZddabKJBq1Lkwuux125GXoTj0gPrK3xAKvMSlZsFl0CR1N6tRTBJhDUM4g7TWepdN9DUG5X4q6HU3XQmd9/zmfAqwOIGRODTFCBRES1aZy+ZpDahzVd19D72d6UH84HoN69njH7x5CQ4Yv6LRfc7mOS+Dv9bwpvFJJ/PJ/L313mkXOPYGoWNzlqvUAm2xCEJOYfnm+/pm1hrvKtojQmis7z4uk8z2EotvWluqkK1T4H1Ub3x7qTc6qC2KxYGtwamLd+HsYJ1zhZb6XgZIH9GSojFIYV0uxZjESjppUNlr1P7GXh2ofJjs7mr3l/cb7TfKzyJvRVem6tv0VcYBy02iUlZ0owFhnpvrQ7mbszyT2cy6ihC7mOiKzqsqgLYalhdiqmsJnJHP19D3ejDuNt+xaPaDX1BfUiV3mEB37yWIJqWzNC5GIWXWCPQLv2hc0GFoWayrCeOPkF8YbZIer49tgf6Gfpx7n+5zid1JUkYw7b5m7HI9KDBScWkDftBSoMzfTK6onc5oKytw/dJ4uOpK3jt/LW3rc41/ccRSFF+HiKv3vFetHzyZ6o/T14+mno3RtcT17g2s/XWFGygv1P76cupw5h9PP2euRvOM+9Rj1ytZxui7tR8U8aRaFVnO9znG53nkWikHHkpSPUZNbwQsULHH+yhExtJs9//jxGpzLuL7c8H6cyZw/OpWDLE0jwkJIS2I/VR7+i8HQhty7oSCx7hfAbOzBc2QbPT+GpbJGH/exOcVxY5BZyYnMY32k8dXl1rB28ll5P9cIYaqTJtYkm1yZ+zfqVoN+CyNiWQa+nemFSlpMZ+AP3pqn59OdXMLWYWHR6UYe+kAg23Msz0eV23Ey1ZRK56JLpmv8nVmtHJ0mpxxYKfH+gIHQeMcHx1OXWcfqD0yTNSMKW1M4x3C4Icb8os1lvprG4ESdvJzQeGjSeGnuGMkDXR7raA31twYPqo8AxaPCJpv8wNUpnJeU3y1G7qXEPd+eq+lNuJH9PVsAinu/REf1upyOUyrAqnYiIgf1P7ccn0Yfo0dH4DkigOOkaRdvCeC2vBYNaCnITA3XN9LZZ+O9Kb0Ttp9PXYMylMVzNv0rllUqMPkY+fPLDNmlJ3OvdMe4p4E7CHfKb8vmg/ANcbWEYFeW46JLpf8KXq79uxqWzC3qNYz61SUwYNW60eARjE0Bfo8eit7RmcosZ070u9iKx1Iqin8KuN9pibkKnElF/n77wKQPvZGDSWbC6SvFqHEqN6zH7M5LSk1Ar6zj0/CHiz+ZQNEMMRGXXZdJnQ1/C3EOZu3Yud/65w5u2djqFgoLMuEwGRI9GnmFAajFis6kIap6I1KZk0KlBGBpyKWmM5p2T77C813K0cnEMetZ6smBNINVL8hmyeQiPfuPQSAExcFVY14fkJTK6zYnHO86bRqOKUTdqyQl6l3MBTxFqegy/gT/SqRUwmagYQ897YxgbYyPv1BqavMJ4KvcZtF4aTr9/mhOrTpD+UTqz/GcxNVHMCPWM9mTYR8OQJ7nCedFRL0EKEnEMd3+8O/oaPTkt2RR5nUWnykXnpOPq9Kv0k/WjKqOKKz9eIeXoJcpl3Tjf7zQWa0f9unBNCjGlqyh2XsaIg0mUR2XBTAfisa099Ro917peo5M+jGNvHKPnEz2RIENjDEWvKuR83/MURfkxPNiRwdH2nQ0OH8wP434gxNXxHQX0CObY0LMggSjDaPt310YLWGwQnbhI4MpAN1Ie6Y6p2YRcLUcq77hJ1G5bj0+SD9P/Ed+rxlBFru9fyG3OhCqWIAmAKeun2DMbEoSp5NVa0TV/SlCeN8bLaXge8CS4SzDFIcV2W0dpVOJb6YvZT8eex/aQvimd14yv0dDUQFpFGj+n/0xLcguPyh7ly9Av8RvoR7aXCWfJbM4k9mTk9SbuPXWPKM8o3Be4U+snZiJsnLWR2Ztm06TNw2u5QyenvbZse6c6iPNEYO1MSj3/ps+FPniWHGLoB0PtlEe1xko8tXcxtGhwzb1Ls8yX7L3gVO+Czj0QKzaOVMJHW5ypv/4Zm2w/sTvLQn5ONV3GdQFA06Th1ke3OD/rPN2GdSNjawYylQzbU+J3UeN6nBZBDHI1FDbwbdy3rChZQXjDfLLdv8fJGPFAxoTTW3H8fucd9E56FBbx+8/qMh2LUstnnzmAYIdGHeJeSmd6ZJRx5rODLDqziAofE5djRLrb3MhcUPdn9K8LufvhduIv/kLJBMf64F7njm+ZgoLDWQTuP4MufChtALXwiqd47GIgVz++ylWu0v/l/qQ8NJyBGZdJC11Gge+PFOtH8cgTD2qEedR64JZfQ9wzwyj2Lkafpuf1E6/Yz9M56ZCEBnMzagrHe4obZavMitQmJSI3gp6X+2NdYGBFyQo724TMWQGpMLN0JS5ZTXi99iZPPQUXLorACpO8FqOk1Vkrk+LZO5aILvCh+UOe3CeCE1RyFbc638LjIQ9eH/c6Vy5dQduspcVZ7J/Yuqf5ts88Cs8UcuP6DWYfm82MgV/y+xU/0gNnYpCd4dC1t2je0oWArgGEDw63f6821zkYhi7k9ubT3PrzFgtOLMBkNWFWiMGsrLgs+hh82TBuA9uytjJEoaD3/foE95XeQH+pnFPXLNBdwHzpGpun3eOhnx/q4OR6aPdDVFgrmLV2lj3Y2tYXjV4aUp7oj6+v475miyODJ6QwhEtP/on3hwOI9YolqybLfuxs350ond7h0VlxBAQ55o8uLS9S1dRA37NdCL9zhV1/72KI95AOwT4EKZd6XSKuKBWrTLDrYZV7bKVFdY9a0w0a+kghpheXvr2EX6kXEuHrDsG+X5/9lT+K/2DX0l1M+HmCY8+ElBrvGpq+b2KMfgwxY8Usdp1Qy6GUOEbvTyV31Xh0/lLKrpXxUt1LuCm9iCt5l8ygN6jyN+DXtRt+viZ+MYlZDz7SGKps2WTHZBNd9T2u4Q4Ujc0mcCqxK41OYkaSv8cUJuzpjL6qBRjRIRs2LjOO2LQSTC0mTCYbCoWC9pl/EXkRSIolyNVynNXOGHSOjMwVfVaQ+k8qx+cf5+FjD/PVra+gXjwmFRTcCX6BG1erGKQYiKvVE8ECRqEJq1RHdHY0EmUpw39zAMjMVgtWqRHnZmee/HoUxfkn8XhfDHAPCR/C3fx6yrjOgQAt/lL4dngcKS8Fozv2KjKrs72tVQYVYYWBuEwOosjWMWNIKpFScKqADeM2dKBOjit9m1y/LygJLiF0Sqg90AcdbcmcmBy+q/wOq9HKIycfYeunW5ksvw50sZ8jkYiMNfdnkP6rIpVKkNqUWBQmQiZGwhUdF1dfxL+LP0E9g7DIGmhyukVgYRd8jq+l/OYCej/Vm/iJ8WzL3cbc3aJ60YYuG+jp2hNTs4mLqy8yYsIIMnEE+/z6RxMXF83WOVu5u/EukjckCAgk56/Bu1H0SQx9f6hde6yhXnz/g6MPory5k+u/XefoK0eZuWMmIX1DsNlEevfsnrPBz5eHVz9sD5zWGatJD3kHq8TI/rH7GX7tKzaM/AOP5hgagvvbv3WTykRFQDW2bBmNBXVIY9psXQGdVseax9fw80M/I1PI8EnyQadxALKUZm986xw0cwC9g3pzscSR73F/sO+LoC/wBBJJJConisYht9i67Tx+yX4sOL4AlUSLxhhKaVQS18atYuZ8SPzQIfRq1zb2q6S+Xkv9qTQkp24SEGWgJGGE2Pd2imQbNosN6X1ZtAu6LGDLR1u4zW1eN72OVCaltFJEA0psEnrf3cH82eF4r3OysyqJQGk19e71nBtYxoJnlrLm3g805YkaXG3BvsrwXviO74XKrQLPMk8aPB2sHDonHR+88gEvuNxp1Yp0tJFfw1iio8dyLxCw1HDtu/OommPQuzgAkbNPvU5LbhmmCHFvEH83HqnNMcf6VPnQbUM3ioYVUelaiU+lD1W+VfbjHrUeKE8quVdxj91LdzPxj4l4RHrwe/XvHHc7zmLdYipuVeAR5cHmzRAclcCGKRtwUjgRHh+OYBXo+3xfnP066oaG3g7lxqkbXPv5GsF9gpm7fy6P33ocjUdHMDqArlpHXU4d/l39MevM6Op06G16PDw8kEgkJE1PIrBXICkPp9B5fmfcw9w7XH/q/VMUnChgzt45JM9Otv+u8dAwsA0896UIcpVcv05dThSB3QM59d4pdNU6kp4fDYDR2YtKZ9GGlmjEgDI42Im+F74ncmokT/V+iv4v9u8AcASQWqVgwK4HapKLmZYNTpcB+KnLD/h28mHapmkPtEFbkUglDF41GIDLl9sdkEr/Zdu1L/o6PVajFa2fFqGV4aw8sh8VEX2Y1lvBrRGiT0OD4xssSB6HIJVx/M3jou5h5nKcw7zs39T43eNJTs8l6firlFbI4VdQWr3waRhJrv+nHB55kYSyH3CP9ESmBpPGhew4HbWetay9uZa1N9eyvOdyvhn7DQtPLORe7T2u35yLzOrMvaY4hJwy4rzi7GDoy99ubn0bGQgSWrQtjM4bTc9gEYRxIDKZFmUeb2x8l+Y6K0duSMk5nMPrhtftYHpnQxwqiy9Wq4AgYtCQSCT2d1IZVbzw+QtU94EXq18U+7id8Zx8K5kx+4dRHZaHTSkCg6tcj3BK+gK9g3qz0ryS9BXp/PL9L8zdP5dBrw+i5xM9RQYkrdIOjKzJrkEqlyIIDr90bVY1F6/lINfIKa8qR2aRYZA2IGBFggxZ9y4sei0S9wh3+zWC1ESDWzN/PVLHw2GDOP/FdnIO5pA8OxmhdfzFZcYxcVcEhvFG5h2ah1esYy94O/QZDOpStk7byhDvLZTfLMd4zoi31Jtq24PZwPuW72OVYhVbu2zlRP4JcfhppSTNSCJyRCR1XetouNlAmiWNjJoMEn1EXdgvP/+co/v3IwOigEWAP6JkwW9AKXBk9x5UoWq6Rjt8Mf8H8/V/RWkfw5DeT0P4f6H8J9j3n/I/ohRfKCYrI4shC4cgbUUu+Zc60XLzDB4fPoZ/ir842wAtqmwssibKQ13osiwJjxCtXb/OrtkX3JlGn0gMDQ38OfxPUt9MJWxQGCNuicafR90+NJWZuAa72g3pIHUsYVWPccIvgfLwa4TedjjCE6YkkDt4EQaLHLn1WxQmGzbVg+i3e7X3eOvkW3Tx78LjPR7n514/i7oFA8TZsEF7mdvyBPyG+TH9h+l4RHjQ1Krt6qRz4tmvnkU/4DgJqeFovEXDwCI4NuQCVlryq/nzkT8Z8ekI+r3Qr5VezUqZzzWUgUoGjo4haUIkbmFu2NrpXxzxH02c9T0Slr5GWKSJgvoCbuiOcilWzOTLjM8kNcPGqfdPiSisJN8OmX1xb8dxo9cNhp4WufjbNFgAkMC9rvcY1m8YntGeeEZ7cunqL/a9bUFYATnf5sByUDgp6PyOyP9tVprZNHsTPdx/Z+ei37EarSzPXG43wO5F3+PYWClLQnyoOHrVjlR9Ql/KtsaXMCoqUFq8ESw2ERFmtKIJdSUnP4Mm1ybSO6WjdwnkoeJ6mt3lOPs502BooN5Qz/zt8zE/Y+ZYj2NwCVyDXVme+RTSdyR4V3mz/EOBE/ITdu0d6OhQtPFfr3AKYwvKjJuM+XaMHRV8zOcjpldP55fFv1ASVEKcVk19fgll18tAAkdeKuMKIqWiILFgaTFSmd6Aa4gr0WOiubn2JhkJGagMKpybnamtqSWlsx+v6l7lStkVaBVGLjtYxvXfrpM4NZGSSyXidSVP0ccnCYmgIODNADsqrjK9kpJDWXRO68zJ1JNci5qBIOxi89TN6Gv0PJP3jP2bci/LQGnV4+TdjYePPNzhfS0qZwqTxxIZfl9D7K9kJCMpCinCpckFiVs9wz4ahspVhUwhw6rWIjdo6XJZQ0D2UfR942gTzmkuEvUVqnyqODTqEL36iO0Y1CuIoF5B/PUX5OWJ//30dg+C+wQj18jpu6IvhgYDaScd1Uh4fy5pT/1C6ZVSui3uRuyTI8g65Up8w0/ogkCqhAGvDMBQLzpc9K5mBEFg25RtRFa+ja6qhfPbbxHaP5TNpzdRsm4N/WXwqKXVGDLCr/XbCAveykeffMJF1xEUeb1M5NVYrLqOBn7b2DYrzWyevZk/H/+TpqImtH5aGiWNbM3YCoJIZ7zj7g62frCVPs/1QSKRUK2vIjvyHZRmHyyWV1BqlYQO6EiZY1WouTr+TZYsESlbq26LFC8tU8/yFm+RF1mOX9WPWCyPd7gu3/dbkMD6+evJzVjG0Ipmrv96He8Eb6SJovHiXeXNzM0zuep7FZcgF+RqOYHdA1G6iKLMpZdL+SP1D0Z/PVrU27PYRLShQoZEKkHjqemgcQmOObvRN5qIRdGoXEU61ITJCczYOgODpAa9Kh+johJBEKlD0zelixQZWjEaLrWaUbXUYmtyZsr6KXatNptgAxdgkIR4/6F0dXmJTYdGsZj/PtDXVh6zwLU02DZlG18GfUnllUpUVR31InMjczkxoS++++9Rt66OfY/vEw+04jBaXF3wHRBL9HfeFGx30E4LEgvVYT2oDutB+BB4OvcZJBJRyL1tUxOVE0XwPRh3w6FRd652J8eTRXBAi3ML2sZK1oT8RvLLY9CYxLGgMvsxyv9humx1whBVS8pHfTmaE4YgcczZNyqu02xuIm5CHO7h7kgkEvqqH0FWMIJKt32UBV7B1gDx537nXvcZ2GwJCAIE1cxn2+xylvV4hMfffZybtTf5+/bfnBxTT3zxBxS7fUB2XAs9Yvzs+h8PFImAxluLn58YKBYqW3+WSPAr9yOwVPGAQwlAppAgtRhBEHANcUcuF+lDmgY1cajiEIabBko/LyUuOY6Rz45kwEsDSCsoEoN9SAEJT3z3BHv27uGJS08AsGXTL9yMWIrc4opFbqYmrIZzn57DyceJbou7sf7gfq52v8iita9h7JQOc8Pt9bHZQG0OQkF/jo02keSk4pc+vzDi0xGEDQyjDWnf6NbIrom7SLw4jdPvnSZxWiL+6jCGpRVgSv6JQ+rHaHYxkvPrSTRZ7nRb3M0eXIvxjKH4s2L+/vhvHvr5ITrN7oTNprQH1dx/3sUv+9Qsu7XMMbbaMxhILGTtzmTbTId2cfugkm7IeEa+5GbPyqrQlXAnVNSCKvPYRhfbWjt9NjiyRK53vU5xhDPP1U9GeVKJU6wYPBp45yrNqiwqVS+w5NclGAwFRAyLQOWm4kzRGVbsX8H1mutom7UochSYWkx4RHqwqXQTp2N3E2sosGeuWloD8j5aHzI/LOH83cPUedSxc8JOlkYsRVejQyqTonZX2x063lXe+Bc1UXwmnxObr9HziZ4E9Qkhuuw1Sj3/5vjQK4wqfIf+Kx1ZWRfr9vKb/jhaQYp12Wniy6ayffom/AMSye0+nfqLFYw+BQMkLXxpE/BHR3khrCmFc6duwHBo6NHA7wt/Z/xA0cn+6LVHQYBte5SssTyNWV5LtaSKx248xpoua9BV62gqbULamuFk+xdzkry7KxlkiDq913rRHFRBg58ILrtwAUqTV5Dr+mXbaMR/cDwpw7zxSfShqMLhjD827BimOx9Tc/Ee5QdvgUcI7s19SMn7HUvSnwQfUNL3gj9NneqQNtYhtZrtgQv/+skMnq/hVP4p9LV6Ck8XEt/qC3arlTE0bShCsACpjnq3jb/I3Egi9hRR+VRlBx2stnKu/znGpp7FdAGsrUHo539/GXWZjL1j95KYEY2loQWXQEcAfWj9ei7YvuHoqON4RE/lofwSzq0uQW7qRL1LMSeSxfZ5/rPnsaqPk2Mr4vK0m7g+5WpPTor3jierJgsbNloqWzgw4AADew/kwJgD9jFeeKaQnYt2Ej06mnsH7pGY3BUpcqKy3Uk9/gQWTyOH3j9Er6d6ddAVlSBt1cOVIJGJgYn7aettWJFr5KSVX+BNg4F/pzxqtZB2VUG/llislgIyrmUw9tux2FqphNWmYBLnjaR7uB/rR60HYJWwyuHMFKRYraCv1WPWmXEJdOlA/94zcy31t8+TtiFNzLhpt8yZlWZk0gju11JIallOSRnoPEto7O/N0pVujNw10tG/cYOpVd6ktjM4y70ou1yEa4MrjW6N3IhoZQ6wQk0kRPgp2f/UfrwDkqgM7NfhOXqFnqrbIvMCwIXaveT4ZdDgdA0AWZqMA18f4Pgbx3m5/mWahQpMimpkFjEztNfTqbRUioFcERDlyLrXhPqSunA0T194GqtgxXppKa83eVPtU011cDwV14rYMXkfJ8afYOLsGfZAn3i9jdqzmeiKayid0IDMpsGvbiIVHjs5mXoSq+tMBm1LR1LnC75hHbJpN87eyMq+K9HX6Vm1ZRUbKjfYj61M+ZzyshsIgkBQryBKqkvswT7/usk0qUUQpUlRyZnw3gz8UyAtJAOLtJlhR2egNN0BHMG+89X7uRY1HZ/avlSHdCEs3B99aTlysxy5VI6tLSBMC3teeJaa8HXIuovOPKusmTPxfdFpMvEp82bOholUut0RaRLalQvFFzgaeRTnWGcufX2J2HGxaH3F9b2tvW2CjT2P70Hrp2XI20M6ONqtMqsdOLb1noiyveP0AzGsASBrTxb/zPiHoF5B3N1xl+C+wfYAxcVvLiJYBZBAZVolE36Z0PpcJWDCaDWRMDyS54qes9N/lnqITunbcQcJr9yJZ5QnrsGuZLplMvePufZ6eU30YmTUSGwWG88VPccjhx4RUxwAlckfq1WUN0hdlYowUcCWIb5TSEkXAoovoiuKp+tSh2ZsXqMjUGi2WlC4qPGK87IHigQBLEonGvxi0WggtL+7/fwmcz35fmL2p2/9OGQ2BTUZVci82jqjdQ6ySZCboCmyMzM3duP08rajDqPKWemMXyc/Fp5YyGtfvAatfgiF1ROlxZFRCY51uK3cD0qJHB1J7oFcSgNLGXpsKKbKBsZ8M8Ye3OjrMovctFn/5fV2DTGbBGxmGs+JNMFuVTn2YJ8UKZ3SOjHgXAj9Noxh/9e+YIY67Xn+uV1Mt4BuTPhlAuN+GGfPrmsbX6FFSYzefJ3mRC/6rOoIzpUIcppcm7jVrRK/roFYshzvardBJRIEYMuMLYxpGMP3yxyZ+IJUwKQyYZVByaUS1O5qvGK9sNgsIrWxVAZIEcrLOfvtIZy6uaBzcTCXKAqL8SwtI+r9xSS+78fnRZ/jWepJWbMIMi0NLCV4TTBRo6LYFbyLwYmD+WeGAwgRkReB925vZGtleH7myTemb5iaPZUb5TfYl72PYRHDeLHmRU6dgk8/BXBl925Ru1KwCYQPCbczpbQvhxce5qtlX3H6w9O4h7sjlUnxS/Z74DyAYe8PY9j7w6jLrePrqK858/4ZfOf54u7ujlQqZdrfjuDYv6L5q8+vp+RyyQNAtAfO84vDuvIlEqaI+7vsfdk0FDQQ/6wY7DMYyikq/Jn60g1cO93Ib1+7MnHOHHQujmC2HUTXqq95u/K2/diAMwPo9m43ruivIJFKqVcfB8AqE6/X+jmj8dJwa/0tdi3excztM4kZK2YzmXVm7u68i1esl92/Ym7HhmsxWKi4VcOdLXcouVjClL+mAE6YpY14l5VRdN6JC19c4M6WO6wSVnXQ1RbgAQWV9vtmAL8UnUif7CrShrd9Uw1uDTR4yAFJhz1VWxb4vdhiQvVBNBY1IJdo0bv6czG5jkKfQmQSGfFp8ajz1CAqjtj3dBJBxt8lH1Ky7TJLtEt47ovnUGqVdupbqSBnYOMPDL18hwt/XaB3dhvTgFiJOz07MWxgCl37G4kcEdl679ZgX5MGj9Lb1FxU8s7gDaS+lcrgVYPxsMWiN1rQKW5zsddF/KIeBAyAmM1d69VM6cFbGEKMQDxtzsg47zhSB6bScKKBorNF6Gv1eEZ72tcSm9WGRW9BppTx1+i/ULur6f6TA7Bae7uM618cQK6Wi+DK16HaloNBWYLGFIrg5o4gNLD/6f30ebYPsgA/bBIjglQgO/QrrPL3GfjaQLotEYGKbWCtkqASLvQvo3NAEFEjOuou1mkdQAubDW6uvcntL29zr/Aef5b+ydMHnkaj0zD9n+lc8rpE2oY0AroGcPzL4/h+6kuVrgqFt4Jpf0/DarZyIucELo0uqIwqDCbR/iwvL+fFlStxFwSO8CDr0VxEGs/xCGwt3EKXx7+2y5zYbHBnyx3u7rjLiE9GdGBm+t9SBEGgqKgId3f3/0+e/59g33/K/4hy8NmD1BbWMmThEFJrNlCTq6fF35mAGVK849w7nGvnmxZkyLUqei7r6TjWuphZlE7if/oaSq+Uoq/rqANRmDyWhgFjsVl06Gp0OHk52Q0/gF4XU/AtOsmVH7WEpYbhk+BDs2coJhMgSHn545cxp1yldIQfecfyCJ0cyos3X+RY3jE0Og0+t3yoL6in1/JeaP20KIuKUZn8qXM+zzLdeT547QO7KHJbnU1KE2f7nWVE7Awm/ppgr2vb4upf5k/ns1doTuzP2O/HdtD0s0oNHEhdBqkwd/g9enYSdQFil/ZltnwUBo2BqVum4tlggZUwev1ojucfp7/WQdUHIK+t5vjrx/GI9MA3yReLzYKnxpNafS0nBp9g6ail6Kp1fN/pe6LnRNsd2r2DerPu2roO97po/dH+b5vMhtHVyPBPhuPfxb/DpsOv/iEENxjw8gC7UG9bBl2TaxNlwVa8BnVi/kcOXmybDfzqJ5LtNQlDSwUyt812vb/mPh48tn0eAHeS7tDo15MjM3/hZrwnj5x5hO8vf8+rx16138uju4cdnd+2CTGqjNxOvM3UxKnkHMpBsAlEj44mSCWiz2MzY+m+7yAXvmrCUG8gaUYSPgk+Dv1Fr1BqHn+VHo85+Mm1XT1oOFyOzCpj3cPreNflCyau6GHXWzI6WzBYDfQ7248eV/vRMuoWPzy9j8nrJtN5XmdeWd+Ja93fovPNzkzZPoWKTkVIOvuh0Chw1jpQeerpahY8sgBnf2dGfjaSYR8MY+pMBd5NIp1te1Ds2qFr0VXp2DFxBy5NLnS9rqbwSiVdFnWxO1ZKTRns6ZHIwt8X4l8eCziyKdq3WVtZ03UN5TfKmb5lOq4Pp1Cy/Qbu9e5M3TaV+tRCuvw8CRDFt53Of0hVYBXV3kNRNyVQf+Y233yTxoytM3j82uPk1+ezd9teKH1ww1ndDtAU0C3A7pBOmCJ+O8IJx3F1ZCBvmN+wjy/35FBU+3NQ6uoxObkDdKAgaDMcc6Ny8RA0tJQ3cWjFIapGV7D5+G+cFaD3fb7ZuVi4aIWHXnmF2kGvYU0ysPpJmKoSN4Z5x/JQu6s7ODja/u0e7s7SS0u5U3UHvhf1TcbvHY9itQLvOG9oTR728hbrH1LgSubuLBInxSBYBbtGZ/u+EAS4vfk2Zz86y5KLSxy/S8AmV2KxiJslU4uJpOlJ0CFAYCWoZxAv1b2EXC3nclU7iKIgGjbb5223B0efuP0EPok+uIW6MeCVAfbNzvpR68k7lse4H8fR47EemFpM6Gv1aH20WIwWvk/6HtcBnQGRnrANITbk3SHiewN5qh0AKBoOYsgqpMJWweXvLhM/OR5FrHiOuqmSxDO/YHQZTtQb/e1VdZOImjb5EflU+2tIbohF8AD/Ov6t4gdgFMdB+OhwZA0ylhmXdTjHorBg0Mrp/3J/Ttw58cA9SqKj6fZ8Mo3awx1+b599ZLHACy9AYCA8+yz2QM6WaVvo1rKSt9td1z6Tw7vKG2wWYqd1xujsjbouAGd9PJEVL7Cw+2K2dcsgIMaFxKnBVPwBHs1Ght8spV57iVyPR5j5wUxqXqphxCeiQ6WfegkNpaBX5tOgvYIlVEJ1zCAMzt7YbFDg/DfejcPxb5qAm7cTsW/G4ufmx6+hv7bOfVKaXZo5NaKaR4I8ufb2CXxsHZHIibcTSTi7lar0cXhoPVoz61s3+YKU1FOpJGZ4c9m2nYtVNSy5sMQ+r35eM4TyJ5vodXcLNdl1uAc7ETsuFs1iDf0N/dnjtofO6zpj6GVg5LOi09mu0STIkEikFAcXM6jroHbt6chmBGj0bmTG1hmiDlCsF5tm/IJNZsKv3B3Bs5msPVmYdWaSZiRRa6hGZlMTW/kFTvIwVD53OmTYLTJf5e/mJyjzFB2Lhj6BPP/FGNzD3ckV/dRIkKDWq3GtV5C//iyyiki6Le7mqBcSe1bA7qW7iRoZheCsxCYVA7clcS0MHuJwYLV/J4Aep03c1WeQsiDFnsnWftNv7NSNwJ569HV6NB4a+4bbvc6deb/HI/S9gO2FcSARs7oNQiMmeTVWuRWj2oS8a2eqZ/9E1mUxE8hN142rkdORmhs4OPIgMxNH0m1JN7ot6ca2O9u4XiNmmKfcTCHicARVg6tYeGIhi95eBEBW0FsAxN2N49DoQ9TMrsFsMOPq60l+RD4gZvTj7c6armtwCXBhycUldvBNys0U+pypoHxgCWl/pREzNobA3iFIBHGMlftXoDc4An1iewikauAlTxtPKo9irGlm9JpJrP7Djbycz1FkFXFMgN73RaDnWsQN8Kij4CE3U9OnAJtfa2ZjO01mqU10MButRvxT/Bn73Vj2PbmP6rvVGCWNaIyhuOq6PLCWtvWFV40XXW8kY6xsoL2XvUYqOqr8y/wJz1Gi9NAS3c9dHAPlHesqYCXjPdGJntnnYbQmOdqaSNydqzgT9z0SWSyzpnbhaEtvGjPbUSYJUvo82wfXEFf+mfYPRWeLiGu9tVu9nEGnB0EP+C7hOxJnJDLk7SEiWlkiAiGyJwRSe6+WLie7sCtiF6b7QHI2ixVNYzVOLXKMSlCXiTbT1e5XuZVSwK/xr1GdWY2TlxNO3k5ozeHY5CZueu4mxtAFU0Y4h784gWJQGLg43lkSE4e50o3SbaKjxPKllcKCUkI8AtiesZ0j147gluGGZaCF2Gdj+avyr3bjASKGRTB7z2z8u/ijdldz9IQcTkKln47jg48zLnY6j5x95AEqOlX9NTT3ZCS9MpYBLw9o7QtHx874ewbBdeeYUbaC14Oew7+Zf6v4ARK9lJ6XolG+OoQXr05GEASsaeK93XRd6T19Ot0SDRxfdrxdv7dmF5ve5sfULHzlXlRnVPNK0yvYlI566b16M3BLL06M+4yY/jFcG3HNfsy1MQjnhgaKT5fxxweX0AzuzTNfhrfeX6DFPYCmIH/iJ8oouuEIMte6OBBXTk16zs/eTZchXTiVeuqB97NhY+HJhQz7fQk54dM7HPMq92LibxPxSRSBYMeqNpARsgGZ1Rm5WW6f74wNRgRBYKdMBMNsn7KdidF/ET/RATCy2UBjCsOzaSAj9oVz8fcPGdHyCiv7i3TOi8/vcLRd6xqts+i4VXmLHXscx/qc70Ofi/GEvT+WH+tf5Zp5JeEVT9MzZwd3+wyhwPk6UpuUUy/uxy1uJC2+YQjtAII2mQ2b3Mbd7XfRPqpFM1cDrSboTz/Bm292ocvCLljNVnbd3mV39oZXPcm9gA8BKPD5kXrnS1yvBBniOx4cdZBJ8m84/eFpMssz+SjqI/q5i852s1IgP2UiPTV3yBtVSPzUeOQJ8rYUYQYVw1znerwrfajekUXv0l3YZDJ7hnC9u4kDo48z+geY3ns6/4wRgw4+Tj6UNpWy7OYyVnVbRfOmZoxNRrS+WnTKPBRWD4aeGsWOH3agr9CLALW3wVXlyHCytdjIOZSDTClDYVJgVpqR2xzHS6+WYtFbKDhZQMHJAga9Mcg+FV776RpWsxWvWC9yDuYw4ZcJ1JmqsMqaURlU7Ou2hbJ5XRj1xSj7/epbHagV/hVURvTGVTQVyajK6DD22qjapHIprsGuGDUOwFSX/LVk/HKOU0fTePjow2i7aUWeeMBk3UlIhhJ9oRfgyNLQtbLl9LrYi+Ccs0SMHUmn6Y79fvs1wGYTtcDa7Ps2W0YpuDJt+zLMqgIey1nJsmVAE/g0jqZZfRep/jKP/TSDmq4XsNna8eBJbChMCp66uJXeAx376fZ7K5XZ9wGgVbx3PFfLrtr/FgSR6vmNN+Chh2DkPyPx/1wEZHzw2gesia4gebZvh/MBFPpG1M3VVFyWUbo2k+TZyR20oudsmENE3hU8xvWjqshAlncqLaocLsWMoUWdTS99f1wblFj0FjvuIMfvc2Zs2cqj3R5lXud5BNUHIVwRCBsYRrh7OD2z96JttFEdbMU9tl1aM3Ct7hjpYSLoS8BG+l83GfDpANTz1bwy7xV2fp1KNqKGIgW1dHu0G39d+KvDPaRWKf7l/ggeDfzWf4OdGnyLZRE3uq9Hffsj4s76YBs1gKm7H+Gtb7xov9cyTJ9MVpoWzbOr8U/0Yvfp3QDU6GpYf2s9c5Ln4KP1QRAExnwzhobahraYLh8M/YC6iDoChwQSNSqKL89/ydqba4mvjEd7WcuTvzyJxcMCfR2SB3XaC7x8ZAcB6wOoX1vP07lP231RAHFecWTWZOLl44XGU8PIT0e29qGAvlaPXCW3gynvL2p3NT2f7Elwv2BMPAjuMevM6Ov0OHk5dcjwnfDzBCb8PIG6vDryjuYRMTQCj0gPau/Vsm3eNno83gPoIga+VHKkrZcuOL4Am8VGcQXk5XxOZsZLDELKG5hFEG5DKX+8+z5XrUYYDv4R/nQ91ZXKYZX4dnqQLr/Cr4KqAVXc23ePrD1ZKBd3BHbO3j8fpRJyj+YSNTIKJx8HO4Kh3sC2OdvouqQrtVm1RI+NxtbX8e01FDXyV2/RJ6Z0USLYBCrc9nE5ZhxvvLOKMx/EETcpDueAVnuitY/lxmaU+gZ0la54luSjd/FBUPs/8I0mTEuiU6tuu4gfEt/r9KDT1IXNYvC2q63apD0xyquocN8lPkdiwbvwOr92Oohm2iIglE4F35O1+hvePfUuuX/los3TUv1yNemb0lGkin0vQYYEKbFZsRguGzgaeZeEPm5YBAdtfbB+LAldBbL3ZFN+tw7/eA97vS5138uQ2GHET3QEhtp8nS61RURd20JL75EkzUjCN0nsq3GGDWRmwp4eEvaP3c/s6j7c/uc2V2KucKPCkW2bHZtNRZAzYd8FIAtrhKh4B6AACd5x3iw6vciewdm+5B3LY/3I9YxfM55eT/XqME4BFHGRLDy1EAQwNZv4Ku0r6gx1WKSiIWexCFTdruLGbzdImp6Em78fIEFqleJb6YvNtV6kpW0t7bOab3SvJknRMdB3LXIWOvU9QMzSrA4/wLTXUgjqFUTqtlRu1ovgI6lNSkBZAA1FDawoXmG3F+StH0vbHH/pm0ucef4MC7wW4F3jjfU5sc2//OILEAT28N/Lm+wBBiBw7OiXqDUfi/1mhfIb5aT9lcagNwbhwv++YN//1+X/fi7hf8p/yv8fpf/L/Ul6Rlyo1OYACn3WkBb1DnofWDtkLd8ntUNStW7AYq9eYNOQnzrcp20BlFgtyI0thI+K5eX6l4mfGG8/Vuj9KzfDFlOiOciv/X5l7RAxm6rF0kiLKhuAPud7EHD3BHuX7aXobBENhQ1Iiz6lTL0Co7KMa12vYYn2ovB0IUdePEJ1XjWb0jdR2VKJf7k//b/vz73990h9M5Uej/VggO11RtwqI6RKzKKzmq2kb0qn5HIJggBafRxmpZmixKEokhM6vFObMeKkc8K7tAqb2UrPZT3vy2YU2p3v+LcUKVFV4oSsMCuQWQSufnAQ1Q+iEXO2RXx3bbOWlz98GXVRPo9ee9TO+xzrFUvNizUMDB3I+X7ncRntgkQqwSvGC5mnI5ClkDnoK46+epT31O8RUpdk/01pVKIsVpIyP4WoEVEgSHAyRKLVRTBlxzJslzNJnpNsp8Rsb4AJmO3o5Q5FIhCeH87o/X0oynGgbYs3Z/LIr4/gVu/Wer2NqFk9SZ6T/MC94+7GUXmu0v731Z+uMnf9XCSChH9m/EOnWZ049MIh9j+9X2wnmTtqU3DrfQUqblVw8q2TdkF5BxJQiiBXsH3+dt6WvM2+5fvwnhvO4eFiZoJ4/X1UY619qLR5orB6I/P1InVVKn6dO6LpKn0rOTXwFE6hLhibjBRfKCZCiGBklGiUL7y4kE3CJuRqOTKFDKVWSbM6m0bNTcyyBk69d4pPfT6lLq/O7owadGoQXjVepNxKYv/mJno92Yv+K8WgyVXDFkDUU7g+MAF9nZ5jrx/j3kHRADHIKrHYsom8ugHlnRt2IfozH54hftYkMlPfIj+imn+m/UNLlINnv7G4kcAcG31Pe3G6zw+cGQFyb3cUTgokUgkKmYIYrxg6mTsx/PBwTOni5uHegXtsnroZS0nFg2OiXWm/aTbX62gqa0KmFMds/e1iQjKOEHhnI7Hnvn1AMFxo9w0JEhsuYZ6M3zWenw//xG6j8b81hnabTEiPGFFXig7eNtT0pkmbOPLyEYcT3yZh5P6R3PzTMXbbjN1az1rSk9LxDvIWxadbN/lVXtsBGHloJEdfPETZtTLeVbzL2U/OttZb/J9LdS66/ApS5qcwc8dMfBJ9kP3dnbfeeos/5//I3il1WK1w8p2THHzuoHhtaz2HHh1KYK6IslS7q5GrHc60ap9qvlv+Hd0f7U7SrCSCegfRdUlXO1raPdydYR8MI6SfCETIOybqgLRRUN5af4vVoaspOF0AAriFuCHTitf65F3i6vI/aKlsYdBrg0icltjaKqLVOnn7ZFp2HqXLgi48X/48oQNC7d+bWe1KSexg5BEdsxzHKj6h791Trf1oxapxxsPHnfL/ov/uLxUAKvH5LrEujPt+HEWhHemrFCYFmqZmYsfFopz84Ca4yvUgjVnlNH7byMfRH9t/F7DiWpVDUMYRLu6pouHEda7sE7MX2/oitGYFoZJJ3P7nNpd/EAOu7eevx9Y8RmBuOoO/nowyPpLQqseIKXsDjTEMqRTqAxKwBATTWNyIzKRHKihRmwNQmQMwK8xURFXYA0DQPggkpdvVbqg/MlAXkIjBxQerFS4EzeN61GwESymWJj0HRx3k9MDT9mvbAioCNpruVZL1bRbjnB1ZiQDOzc44N1Ry47uzfOjyISUXS2g2tXA+dih3Al7javernEmtRSKTIlfJEQQ42vw1+7qpyDSepEF7Dee6Qn5M/Jr0TekIgoDZ12zXBvzmqW/wed+HgtMF/NzrZ8qOi9pDEkQaz10Td9H9w+7sXLSTDeM32AMqEmRom7WMe38cpVdKHTQqrRlOl8ZMJ+iFWZx69xSHnhf5q0/X/sOppBTuhKwAwH9IIiuKV4hrHKC2eaE2B+Dc5MyMzQvxytXiHe8tflNtVHQSKT2u9GDu2iSSVk1lwm8TOvTzr9d/5XDfw4w+MZrp/0x/QOvk91FvMPS9oWTtySJrb1aHa91auhOa24Kh3sCkPyYR3Cf4vn4W//37wN/5te+vrX+3Am5kVppcmrCp5Hwb/y0/dRdtrROyV0kLfxyNToN3hRab7sHMJKOigmaXZs73O09oQl/77+0zPQtDCymfVd5B47N9UZqU6Ep1ZO/J5sTrJ9h247cOxwVsdJ7fmYSpor1ka733ncQ7XBngSeL87rxmeI2kmUkIgkibG1gzi7DqRxEE+KX3Lxx5+Yi9vaIU0F0mw7vCF6XBQOLsFKq1ajLvvsRBq/Df63taofQAPHa8H+6vuyMIom1Qfbcamw2RdhzQ6/UceeUIZr2ZOfvm4Jfixx3vd9GrCgmsnf2AA6fNlktLTuOTlatx7RZFl4MfE3FtG4MHO2yGESdfYtARNaYmR1+0USLJbBreOHeK5Mt1WJoMKHzcEKQO200qkZIfkU9a13oUzir7uIwue5XEwtUYFZXszdqL1ygvHr36KEuvLOVc1T6OJodxsesPfLP8G+iLPYsNHGtenWcdNXFaKm5VYNlkYXTI6A7vF5kTibBnH0mnfqT3hS4AGGe4cWJEE4F18+mWtwVzVQPfxX/H+S/Ot7tSisKkQGGwoeiewvD1CzA6edgdOmrBE7cpSylNGIN7z2ikE134adoaUr9YbH/n8Pxwenzag7JrZcSsjCE7NhuF2YuR16tJ81pF8Ppgflf9jmuQK6tvrObTvIepcT5FtZ+Bk4NPIgmRE9IvxBFAb3MmGS7idfseTWVN9tq2z+zTa/RYnMR1ws3V9f/RWmR2MfHrksPQmhkikUiwOrissdmgsaQRgE6zRGd+vEScTxRmBbY6K2GpYfR+pjdShRRLe/4lQYZUo+Shnx/iXrd7HZ6dkBFL0qkfubc7A+PNu1w/2UhxMdTI0kgPfYK9PWQccv+vqc1GHhxJ95NZRK7sKlLKtitaqQcTd82h748NhA4MpTTA8ey8Z/KYmTSThVsXsm3eNkdVW+e32NJVbEsqxuUbh3NJHIMOB57lvuzyoqY80kKfQKfKQ6ZMRaJUsHnaZgrPiOtEnVUU6Em5kUK3gz+gcFax8PpCcqM61tsmtWGV25BqVNyVXgCgRS3Ov0+5HGfirSrUkuGMWjuTWnfRLtEaY+gX0o8Q1xC8qr0gHfy7+JMxNoNaz1r7vddZxyN5W8LaG2v5LuE7HlnTUV6hLVOuXnvJ0SYS0ZmnUi7GJaY3t/++za31t8ipy2Fdnmh3tFEhKvw8iV4aTY1XDfvv7adCcguA1zygx4t/YPCXU/DFOW6GzSDf9yv7M1qcW7jW7RZuncN4YswTCKsEhFUClSsrWd5TTB+zLrDycsPLuIe7A3CiUyJ6VT4hxsV4R3vzbMGzLDghgky7B3bn69FfA6DMUbJ+1HrWDllLYKm4T7DhSI3pv7I/Y74dQ9LMJJZeWdohQDFr1yzm7p/L9H+m82KNSOtWa6yw95M6VItCq6D2Xi3lleVsSt/0L3Xpdi3ZRcXACuRmh4O34u8KNk/bjK5aR012DYZaxxyrsHhhNVkx681YjBYsZkfQ7Fz31WyYdxJtUhi/D/ydnY/sFPvJJo7Nzmk98Cu49S8p0LR1xXTd/wGe9y7xruLddplcbdnDEjzKM3Ctzu2wz/FuGkJM2RvonHTc6pyFTSIjZ0eaQ+cLAaVJievRa2Rsy6Amq4YTb53ArcjNfg8BgVrtOQRBDBCdPAmBLg693+DqBdhssHGjeHzNmo4Zg0CHjGEQbYy7ga9TqB5O3MV1FO9P5/xn56m6I4LAwuoeRm5xwxLuT3VQBO6TBmFOHYZFqcUmMdKizkZt8yLS9Qx3hr5L7d1K3MraqMrFZ/907SceWv0QOx7dwfqR69HV6HBRueDXMBarOpjTw+ooT6jmy9Av2ThhIwANZpFa2bvKm2kb4rj6wyW89F788dAfpIan2tdjt4osNJv+wC/Zj/P/nOeDoR+glYtzjsKs4NGfH0V+LJ3hHw8nZWFK6zuLF2fZ1iM15oBMjm/3ECxKpw4Zvp87uVPivRW35FB0NTp+eDGXC+dseDl58UyfZ/DRigAHiURCr+W9OnjipyVO45OHP2Hpc0tx9nO2t7tUIgUpCBIBWrNu9DklgBjs+/jsx/zW+BuJ0xPJ3JXJhnEbqMsT/RIv9BP12MMN4VSmO3wj5hYzn3p/yv6n9nN/KTxbSNrGNFRuKsZ+O5a4iQ5pG4vRwp7H93Dzz5vc+OMGXwZ/SdH5ogfuAVB2rYzdS3dTcqnEfm1jcaOd0UlqMSGUltFSJWZpy1VylFolv/z0OSWZr3EGK0cxMxcYhpiNdNhs5KwNXI+C5JxAyrEUarJqWDdyHSffOdlhrGbGZ5L9WDbDPxnOyHXzMCs7utDbxkPksEhm757dQQdb46lhxtYZdFnQhaqMKppKmzp82yp3DYPfGcz8I/N5pfEVtL5a0kPFQPP2yVvp+3xfui3uxpivx3R4lldJGolnfqHibA6R17fhWXq7w/HY82tJPLXmAZu+/RgTJFYK/rlE7cErADRp0sjzWw3AnHXDCb1zkC7L+iB1F8e0FDkquQqlTMmB0Qeo/rya6sxqTr59kvqMeqBVA1Yi4cyAM+Q8r+HMqsNseuSwnf2qLXFiwEsDaChs4O1xl2hocNSryOcXWmw1mM3w449w9aoD5F8UUsSR0SV4DUxk2t/T7L4AQYBqlxP29/I5JbBlxhYW7ljI6sxncDK001GU2Ej5dB7OU9qYBsQGWntzLW8ef5PMmkykMukDlNCuQa6kLEjBM8aTPs/2ocfjPTq07d97tIQNDCNsUBgxY2PwchL3ic1qMWPbfOo8e5ftZd7BeUQMi0AQYNSNWjR6DY+veRxOXerwvDbbMKxxNj1ytyAryucd+Ttc/l7c79c4OwBT3a51o+nuNQK6BlDSo8Qe6ANxbV73/jpGfDwChZOC0+WnOZZ3zM6oYagzsGfZHirTKvGe5M21btc4OeikPZv2r99/ZzD/daCvrfQGBgFXT7ZL5rDB4LcH85rhNbxivP6rS/9T/l8s/wn2/af8jyhxE+NImNGakSOIiMVS959orivDL8WP4L7B9owkm1Q0tg3KSprLGvku4Tsufn2xw/28im/Q5fBnFJ8t6PD75egJ3ApfgtG2B3XpdRKnJ9o3xdcbD3MpVsxVXzd/AxkDlzL/yHyix0TTUNRAp8s6HtriBhLYNXEXxt4hdJrdiSUXl+Ce6G5/RpVPFRfmXrDzhYMjm0qCjPC8cKwrrGydvZXNUzcjCDDk9l3GXxGILn8ZgOOrjnPpW3FRaDNGcqNyOT7tIYLGOyhB2toLILgomJEHR3Lvs6t8l/AdVXeqkEpkRFQ+TXzjk2yavYmjD9koP5dL9D1xUfSQicaKxhSHzi0Cs0cwAV0DOlDt1enr7GKwVpsVjaeGRacXEflopP2cecnz2PvEXi5+cxHPGE+iRkTRTbXYfty30pfoV6NJ25AmPlftydD0HIamZeGXfwlLZk6Hd2ozGJLSk5j3k5KKnRdJ25hGfUE9ADvVU7gQM4Ks2CzWzVtHy6Fs3pa8TUVaBTVnSwktCkVhVjDo5CAe2hpI7OIBduH29huTydsnc37qef4Y/AfVmdXsW7aHmHsxROQ5aC1GfDqC0atFJ1UbBVBWXBbnxwxk5GejWJa2jMhhkR36QmoxIa8oIX2TSEdSnVGNKs6FswPOIkgEhh0ZhiW7mIJTBWTuymytV6uDLH4AmQNfQhEXyeC3BncI9nXJW0t5QDnHhh1DE+FOxc0Kfu37K+l/p/PNGJHeRWKTUFhZiGATqMuto+RSCbfCFnEqqQtXoiYj95Tjk+SDTCGj74q+JH44lMs9L3Or8y0+eOUzTMEdKT3aHLTlAeXU+jphqDNw+v3T5B/PB+Ci32LOxfXAoywbWXUF8w/Pp89zfRj15Sh0+ZWomqsxaszc7nQbk6czv/b9lR0LdqD11XIvvoKdE3ZS5XaIu8Ev45bahcdvPo5vJ1/u7rjLupHrcLnqwoCzA+z0KnW5dWRsz0BocQTorv16ja8ivqLsehkbH9rIxoc2djDQMl9fx+rQ1TQUieivnL9OUhJYwr2wXUhMJdgsNtaNWMe6kes69MWT3z5J/4N5yDRK9t/YJ+rr8N+X3sAAZFjSzMTdjcMlTUTyjvlmDH2e7dMhWNPnYh/u7Rcdg2c/OUvNNXHzmRuVy5bpW1BGiw6Qz3w/A0AuEYPqB0cdJPWjUWg8NHSa1UmkChYcwb64C+uo3HoGn0Qf4ifGo3RWEq3tjnejmEFX53wOiwVGfj6SSX9MonXgANDvXD/8iqswNpkouy5urNrXGUTn+J1/7qDQKJjw8wS7qPj9JfahWGRKmT3Q7tfZj97P9sYtxA21u5rF5xcTMF0MOCtMzegKq7BZHuD4AWDvuL1oRg9C4aTA2c/ZHggCMKtdKItNRRoewucBn7Np0ibxUkF0dve4PZhZf41BWlPF4Cnj+Onf5Dz4WaaiJVlg+TfLOTD+wL88J6QohNQ910n/O92OAm9fVA23uPzUn6SvTmeC8wR6BPZofS0bzrWFBOScpfZWMRG3duFW1eboFF8sLn8OAcVBXPn+CsdfF7M12vfF6YGnqQoWNxQ2m0hpdD1yLpejJ1BqyqRZlY3B1syXIV/S9dAnHO0cSlrok0gEKU2uTRxcepD6gnq786PSkkOT+g5WqQ6rzIrNCYRWx6m4JxHr1ePQ31R8vxX9QD1ZcaKDs9suCRkhL6LRaRh41AuLycJT955i0aJFHdrjUu9LXBy3mJipnUlZkILWV4vZaqbGVXy/nOgc7iY3kvLmRBaeXMjHH8PdLAs2qbhpSbmRgrapnh7P9MOvsx+5h3MxBhu53lXMGGtybULiJsFisNBc3oxZL/aJ6OiU2ttQX6dHX6N36Iu19GaBMZ3ITpF2xG57IE299hIGq46RX4xk4h8TW+8jHnOrU9N1/wdkrjnR4V3F71GKyqgi8U44stw6msubsVlslOpzOZ3QiwPKJRSFFHGtRykuCcH2b2l5r+VkLs8k2DWYd0+9S6V/JYnTElE4PUhxahNsHF55mCMvicErqz1bUcaNkS8y4a9/rXUJ4LrpJ6puV9ntoPZZ9b8s/QVT7xhiH4oleoxoM7Stn3GZccxd1x/z9duoLqvQNov0TNkB72OVtXR43tlPzrJt3rYOjpXikGIqJlWg9dNy8t2TxN2N63BNWuc0RlwdwdSNU5l1fpYdJBOWH8bTXz2N7tIdhr0/jP4viqCUtsy+ssAy6oJ6onRVI1fJkcrEPr8UM5ZSr02M2h1G9Kb3aCprsmcm2wSBgRqornaj7+onCM3OB6Co8GcGSv69OT9VCteOq1DdUGGoN7Bh/AZ2LNzBzZYD6FQi6MFkNnL2o7McfuEweUfzREaJVvpOCfIH+rUtqGGVW4moeRupXEaLexAGZ0/c3BzrVFGUKzndplG06wbvqd6j5HKJvR/lNi2yzBy8i28Rs3Ii2i4xeBXdQK8oosJtL7mGthRTG415tcjv3EJhaCK4dh6Rlc+Q6/c5Hy/7mI1TNrJuxDqu/3YdnaUZvaoQk8pEjXcN+MAT6U/YqbHa6i2zOuFt7sqQd4fwTN4zSN07bkljsmMQrlyjKrQbVQGiw/nDxOfAdSk9Mr8lrEAEdAx4dYDdnr7j9CP5fl8zcedEZvypAFdX3FLCscmVtKHG24IaZrULce/MJa2vM4VhheiaxCDbsMhhrH9xPf0+64d/V3876llhc0Vp9cJkNdNsakZv0dNc3sylo5c4W/qPvR9BdNJYjBb7etXWF1e7X+WvJ29h1pm5+M1FzDpzBwBeTeyLGCbP58YfN+idOJR1avX/YXSJZY0cmlIEDC5ahJo68o7niUCgViBcpds+9r32IRvnbGTm9pmkrkoFoJ9kBdGlr5HWOY281XWM/2E8o1ePRq6Sd/gec71fpPr8PcJSwygJE52ti7xFh05hyF2K4vsQOacPV8evoja4Mzod7PEaRoHvj3S+2ZlJv8bxtuRtVv6x8oG626Q2LAop7qMCRF3PdmWh39f4Nz6CVOOLYBXsQSsAjVzDpmmbmPzWZPxT/Ck4Je7n2tZAqaAiI8eDh355iMUXFtv1oiWta1bUvSjw+uDLAAEAAElEQVQaT95ix4Id/D7wdwBMVgsWeQMWaSNlMYMIf3IcGVszuHrjKv1/6892o6itbFQZMWhdkSrlHTLQ2sql3pf4e146VrON6DoRpOhfL64LEgn4ZRyn08nvcYvxwdrKHhFb9gZnHznL3OS5DD8yHKcXnAjoFkDmhExqvRzBPolEXOcqX60kLDWMqFEieERh8cQsa7BTsNna0XLbJOI6F1zzMBqbDwuOLeDXxb92qHOz8jIeWa/SklVCxMK5qAM7zrnPesAyd6ge4ETtazJMShPgAAYMsL1GRNXzpKxeaNelaitSiZSA0gCkL0vJP5lvn3fbvsmauEQeOfMIbqFu7Y6BrBV4YPAxED85npixMfhGipkc7ekkFU4KPCI8uP33bSpuVXTIyPCI8MAjwqND1lFbkNesNJO8rg/+Xfz5JuYbnnjjCWZvnY1FLgbF4+7GkXTye/JP5uMd741zT2dsUoeN1ZLRQsbWDCrTK/k29lsmt8p7uOiScTKGE7N4IE9lPcX+5fu51tmRDavT6igPrEWm1WCz2ESaUUTqToBdky5wZ+BjFJ8r4uQ7J+1BepsNrHIVzZ5hmFQudJrViZD+IejMunbziIT04c+S3XseWTtuo6wptT9XIshpcG9g7/jjgMDZFdtQtbSNLQGD2oDTa6Pp+3xfqu+KDnyPMkfgtM7lrF33+bHH4LPPoLzEYTDHlL0uAlgcw4L9M/bz1ltvEZsZS58rS7AVNfND5x/YPl8EJ55v3My9wPcpCM/j/KAWwmf14Zm8Z4ifJDLlxFW+xugb9SQN+ozizvOo3ngE9eqPcWooxQFkltj9KJfeP4xX9oXWN3L01ZDjQ2i61MTIL0ba5VkAKtx3cjNiEX9n/klQzyB7hnAbIEYiSFCYpSQ/3JXnCp8TZWNw2EnNniEYhozGK050ZL8y8BVW9xIz8CxyC4eHH8aUFEjfFX1JmJzQoV7pyen88dgVrCHhmPWWNqPQXjevai/cq5uxNBupzqim8tN1TNk4gZWHVtJkbGLHxh28o3iHK2uuoDfrmRzvkJdpy9oBuLvzLhHzIki4k4BMKkPXQ8f3T36PtLOUf6b/Q/1B0Z9kkYnj7FbKLaZsmoJZZybveB7mFnOHewavDeanHj+xZeYWznx0BqlcSvfHuxM+JJz7y5Xvr7Btzjb7GAdwaQWkWI1Wrq65Sv6JfPxSxP3f/TR/ecfzyN6XTeiAUOYemGtf632TfFlRvILeT4sWmLa+BMVvP3HnH1ETtiKtgrRjaXz+2UscsP33INxDVqhMr+DP5X8SNTKK0sul1N6r7QB4UUhUyKVyfBJ8CBgQBVJFh/tc/eESN9fd5F8VuVpOwpQEQgeEsrJyJWO+GtPBnlO4akh9I9XuKwKRIh2gJijB/s5lZdDQ4AArN3mGURw/HM/OwdzrPoOaIHEv3XZvg9YLvYsvd/65zc7FO8X9ug3a5t34jHgSr7XQ5YMZhKycKV7b7pup82ikOrgL/d8ZhcSjY1aZVCKlzrMOU7iJiCERLEtfhveQ1uCzIGb2Nbo1cqVeSmGnsaR5p9ppPCWCjHzlXnaX7CY3uT/1fvFcuuR49rLvl2Fcc4I1E/aS8+pvvPWWY11vcG8gJ7YBhbdbh/oIAuT4f4LUKmXmppko0dDl+y52bM/Q9GxmGA7jW+FLnzPxyLRqZIF+D7zz9u+287fP3xx99Sim5o6gD59EHyb9MYmIIQ4/WIdAqk3AZrVx+aKNp55vRisV28ykEMElgqcPnWZ1wj3cHZlC1srsocKkNHF06FGscWFsnrqZL4K+ACBAGUtg7UwSM7uTcPpnJFWVxI6PtYNl2gdtP135KQUznajT1zH6r47AORCpmQFKr5Qy/+v5DPtzGONix/FM72fwkHpw9cerKF2VRH4Sybn+5zg+9DjS1oC2rr6ehQ/c8V+XRwCbpdneMFaryPoiV8kfyJT831RcXP61H+z/RvlPsO8/5X9EkclkREVFIZPJuOv2LRZ5Iz0v98Ty0jY6P9yFuoETqK/veM21/k5M2bvYjkw4knuEGou4GTS4+FIZ1gOrRSBjewZNZU0i9YSriErqfrU7ffYZ6bW8FwNfFQWA22fzVPqW0+IWSOSwSFyDXPHt5EuDm54z/c/Yz7EJNpz9nAnqFYTCxWEUNLs0k90rG+94bzaM38CJt0/Yj0kEqYhabLUXIoZEOAJEVjPRlzZgOHGeqz9e5fbm2/bnADjr43HTd6XudgnfJXzHrb9utdYbQMC/3J9+5/thKNeJE66kXfBJ4si4GLVjGYdWiZkJ0cpWRzuR5PZ6lJbYHhjqDVjNogG0N2svnp94crLgJHPXz6X0Hcemor0Gy+N7H+fizxfJ2JdB10Vdmb17NlKtQ9i+wa2BijEVHHr+EOe/PO/gI5fKuDl8BeopY9n40Eb+nvJ3h3dudG2kINKMqaaJbXO2UXRORITlyfdhk+lFoyA6B2kr9YFFb0HuLm72WrQtODc741qvwNbOCG3vLN85cSd+Y/youl2FxWBh5u65XO5xmSqfKsbtGcedrXeIHhVt52WvMZWgV7UFkK2o3NT4dvK106y1GWiapirc//mFqFFRzN49m3mH5lH+3B1e+vhtnJudGXhmIEJBGafePcX2h7e39mMrarN1R9PewDBbzRzrFO3QOUHcqLiHuzPisxGEDggl0kM0IgedGoTfVD8qb1dy6r1T/NL7F2QW8WY1rscJmxfGwhMLcQ0WHRh1V0oZdWgUCrMCs8KIRCpl96O7HUjUtgxEAQTBikuQK8szl9N3hZixUeqyh0a3Rr5+/jDN/UcROTySUV+MImxgGOlP/EDyie+QmpvtbS+VS5HIJKhcVByeeJvr3a4TdS+KYQd7YaxyUDHcOXCH3MO5mLxNDDw9kLnPiDoaPZ/oySrbKhSxDmNMrpaj8dR0MDTat59TrJjRcv1XMSAgUcup8KvgwJgD/Pr4MdTezqg91HaNif68BIg86vWeKqwWGzv/2sD8f1NfZzEWVBlWhhwfQuBBMdjXZUEXYsbG0DOwJ8MjhyNIBY5/epwxP4yhobCBIy8doeq8g+5w3eR1DAwdSPiQcHv2iqw12JcblYv3ADGrYOrGqcRNaOe0kUgoSB6Hx1AHKMDUYqL6k7OM2TuSxNuJdLruidUKof1DiRzeGqhuNYa/e/I70npFUXqllJ+6/cTtv2/bvxmNTkPPSz0pulDE6K9G029lR22dmuwa1o1YZw9yT90wlWcLn7UfD+kbwugvR9v1wdqX0rih9Ny8Eq2flt/6/8aJt060vRAgUvIq4qPQ1+qpyhD1ex4IetggpF+InaJFEMBd14Pxuk8IKA9Annkb1ddqzlpF+r3/rlwEztiM0FVGRP8IlDIl/0z/h/C88A7n+bW8SFlUP859eo7GFxofuI9HrQpzg56x348lclgkMolDL6gyojdpQ57Cu28M2T3nUOefIGqwthr4EenX8dzyM2O+GcPc/XNb38nx0o3xEzG4hHFyxU6a0vLtTm6bTM/TGcmkHP0O5d5fiH0olvJgdwzKYgp8vyfXT9xo2AQbVber7PPqr03TONkpCY/mviQmfk6np4cRens/2rriVm3YVgqWnqewdHZnUtwkItzFtmkrCrOCuAxXmnOr8IzyZFD8IKYkTLmvVWwEDopm0h+T8Iz27OAMbzve9tPZsx0BGmP3jcWzvIwBb48guHcwZdfLcD/qjkavaW1vD0zlJqJGRPFc4XN4D2+lTBGkKK1uDL04lIxvMpi1YxaLzy+2IyulNg0aZThjPhvDqXdPcfXnqxiNjnduEF6hNO2i+M3YwR3isVLv3RSEFSHYBG6uu0lTqZjVc0L+Cnn+X1LrWcu7r7+LRKXk84DPqb1Xi96io0F7GZXgxvTgm5gDv8RmsdlRy15OXsR6xRLoEkhYfhhlP5VhaDC0PhcGp4kgkcHHB3P4xcOM/np0O1BKmwNDik2upOJGOTsW7KDkcknrcUdLC3IF3ZZ2axeo6dgXNsHG6C9HM/zD4R3euSygjDMDM7DWNeD6liuBpYEs7PQYmUGvA+BZ48mjax6l+uBVCk8XcmfLnQd0R2yCDavJyok3T9A3ty/3F4vNgrO/MxUBFVgUohPBKrOC1Au5XP3AvQAiy1fi0dIHXbWOonNF6Gp0rUhnsU0avbxoDornucLnGPfDOO5W38UmWNnSDD9b9BwffJxaXzd+iv+SltyfWWz79+b8BRbIQQQz3d1xlwGviACjMlOW/RyTxEzXJeK8fOOPG2J7ShwaJ/d/Bm1j0796EInZ07E0G8juPY+y2MEdvscGTyfqApPQhnkTNTIKtZu6XT9KYfYs0oYsx2tIMrrMIgKzTlDlepjLMeMpMWaw8MhGZq3rRc72m2gPbkfT5ED0S5ASVhBGw+EG9LV6pDKp/d5ysxznJmcwdqg2rkIY7s29GXHqAyJ/FJH6bmFuKBWOeUJqU2P0fR6WPUZhp7EEWNajMgUw5NgQIm9dxq0qh/jzf2Auq2bY+8OIGikGPO5qfwZaKULjLVitiPadINjnCb2kmsIjH+GVd4DG9ELU2S0gQIXHTiRvS1h/az1de3RlxPMjUDoruTTzEt2udrOj0SWCjMTbibhPcuefGf+QvCoZ30pfJIKMiBw3Fv22COspI++r3+fwSyI9c1tfmJVm9Fojd7ff5cDTBzA1m+xOZWd9PJ7NA1AKbpx+/zQhd0M5bjb/W2vROSt8PvEiA25txnzyLH8O/RNjo9Ee7BOkZjLrrqBDx/E3jvP3ZNGWFhkaxXXBZO4IpGmfcdgi3cat1zaTuSvTDvCSS8RvrCywjHudQlH6O2z69kAIiSBBYpMQ1CuIIROGIKwS+KjnRvu5d/tpuDh6CF5+DmaHtiKTSimJH0bDuDm8q3iX4Yccsgxtwcgej/Wg6HwR+54U9XBt9udKOX1Ric1s49c+v7Jtblv2nzjR9L7Ym8Y/j7XWt7Wd7PTIYpt49EtgScMSplRP4VzROUedE+6SNmgyciclZ78+i3ud+wN1FyQCeZ9vo9ca0RYt9fibvd0U/NI8BYN3CJVh3VFo1Wi1WkfWqw2S/ZJxmuCE5wrP1ro4JsYueWuRSqQElQSh269j2PvDiHm5VRdKXkt66JPQpoEnve/DAzqdOYjpi2/ReGrQaTsyVkgECVFZKlruFvPhh9DU2BH1lGWCvbv6oSo10RCnwyazUeUm7pmlgoJhNc8yeYMbJTsuc3+RSCQoTUqkN6XkHs7FahL7ru27EGxiexedL6IizcHK0WYP6T31zNw2kzl757DhyQ1cWXqF6IbH7OfpqnXINXLG/TAOJy8n+/0BdDU6miuaqb5bTdk1UevM3A40ZrKa8Y7zpv/L/TkndfRxW5sgCAg2gX4v9CP4h2BsMse13q94s0pYhXeCNwNeGYBnL7HPmpzSONTVmyaTuLcxB4ZCf4dzWmKTIDNLsZptLD6/mElrJ4l1sYhBFb3WiEntTMmFQk6sOkFTiSMT2ODiQ3bvudQHJDD2t6nsdtaj/UDLHzkfAGCU1LO7uwuNqisceGQL7vdEik29oohGpxutdxFoCoyj31czMNj34gJWuZW3VWM40HyA8CHhLEtfRmasQ0cQxCz49mtRXa1jnEhtoiZYe4xC4X4xK1ZmUTJ6TzDmE/dw8nJC5Sbui+stYv5ytU81mUkGNMFeuIe7P0AH2RZAbL4g7h8ibu5sR78nRdVcg2dJGn3eG0dVt44ZOwAZCRkEPhtIryd7odQqqdZVU+T1Oy3SvUzaPgnNVRUzts5g+EeiLWNpnf+qfKs4ONWLzo+IQLw9WXuYtGkSV1WftfaHL8Yuvdn/1H57ALOTex+G3SzGKrdydsBZTLE+mExw65ZIy9/eZhUkVrh9m9+j3se9XMxIbGMIGnF4BAMOFhMyZwDGIW7sGbeHcs99fHb+MxbtXMRjpx8jPz4fZ39nvkn8hrfnOUj9DRYDx387zkeeH1FysQRDtAG9Rm/39wAgh6mbpuI+oqe9HduKxWZh4CsDeU33mn3PVKevQy1XYxtiY9Abg7i78y7FF4qRq+WM/2E8KQ+ncH/p81wfZmydgc1qY/PUzVz57ordn6d0UbKyeiWjV48mtH+ouP+L67j/O/76cXYt3oWznzPRo6Lt1Nj3F6PWE/OgoQT1FkHq2+dt55XpL5Mq//cAWf1kUJFVjtJZyUt1LzH5z8l2UJTGGMpGZRpLDy2lLrdOBHbfp4R1/tMzXPn+CoVnCjm44iC1ObX/4kmO0v4baltqS6+UkrkrE5vVZpcOaNNaP/jaKT7ssZV58xzX6twDKY/uj3O4N/UBCRhcfDvcu7DzePK6Tqb0cgk3frthD161zbtxmXEk3TCi8nVD4dc6D7QLIO2dcIL8LiJIpY0aN9/ne+Ztm8fBnIMoTApoApWrCt8kXyTO4kfaRuMpN8uRm6zU+8fT6BtDkmIcfTKPEFv6FhfdV/D5x58j2Epo9hLZZtrq1eTSBM5KdI0WpJZWIKdnH1x0rcFMrBgqGtjz+B7uHRBBsNucRtvXo7jMONyUwbiNEedcb+JbG1qOf2Uw/c4l0HSvwuH7a/fObSwwZz48Q+29/7oPN03axIFnO4J83SqyeFf+Lvv6vItk3VfcrBbXQrOsHgBrVAyjvxqNqdmEscnY2u9SzEozpwedxhIdjFecFwHdRdmZbs7j6Ja7idDmcchNLdjcPZm1Y5bd59ie8lfvpMeqgJOvneSFT1/AqcXB8iK1ShlcNJiya2X8PvB3huwbAsD7Q99n9ejVxMXGsbJqJcM+GPZA5rXY3uD/wK//uvgBQjtfm9UKdXl1FJ0rwmJ8UHv8f0NpH8P4/6L8R7PvP+V/RPlrzF8YDAYWHV1EvouYXVMWUAaDo7merWWLyCR4H12EgGuYB8vSlrExbSNz1s0hUNKFblyn2TOUZs9QdJU32fnEDmbumEn0uHg75cOtzreQOvdD7eawWNsHgdQGJVKTDkFwQiIBtZuatUvOUetyGoXFg6lb56GNKMP6rBWr2Yr1Pi6Otsm06GwRSq2Sy3FfcjVuG/XaS5S5wfDJwxnROILoUdGi5p7EQIPTBVxqcrBVuPDY9cfsQcy2LDeVQYVrVQ6GcqMYMLkvKHSr8y2yY7L5Z9Z54uOdmbd7Hg/5L6PG2YtK1Tni7sYRUimeb7KJi6uMtiCl+CxNfgYfe2xl+pbpJE5NZMKmCfZ3cm52xlJrwVBv4OI3F/Hv5Y+3kzfVOpHC8tPnP+XWU7fs53sRg9oUjFlWRy/Lz0x5y4v0O+mYWxxO+n3dnbBJTHRWluDXqk3Qvi+KQos4PqqJsT17kzo3iKBeQa0939FpoZmWyPI14iYwYlYPNhnv4GIYQFbXrpTHJxO5cgsNQwJIfTO1Qz9nJGaQ8kgKfUNER6N7HBy6eA5Ni5WeV3pSdLaIxKmJ9vML9aKeg3udO9F3LJRdL8c91AW1m9pOEQlg0rjS1Gc4Q94Jt9dZl9OARi/QI2cvl0coGdA7lIGpNehq2ihWxHrZ6s+jLd2A4UZX/ki9SOpbqQQODESnFrMfQwtCGXxiMBXBRfRa3It+z/fjWN4xIj4QHQ4VfhUIwwXUbmqSZiThHe/N6opd9rqZbY7so98H/U5zvZ4Dow4gt8gJLPXEFqinMr0SW6tjqG0sz/1rLhH5V5CuH+OguGtXGp2uI+hF6say62X0eKwHgkKCxCwQXBzMpB2T0HfLZNFpR6ZP2/ccUBZA92sJNKdkc2G1heQ5ySS+lchyYTkt7i38MOCHB57Xnn2h89zOdJ4roqtnt4qP757jOB4wdyg9RnvbaeyCvx7J/vMrkdgkCBIrggDTNzt0YvyEFOJLPuTyoNuEVfak6kohxZkl/8+MISPsn7qfvtInOhx7ru9zPNf3OWw2G5WVlWg9tKiSVCy9vJQ8WR7sgh5VPXD/3h23Z92IeiPK8c6CY0mvbzYS5OeYvxygXwlVYT3QJsORV45w5fsrLDy5kJpTt4kClC19CSwN6iAeDo4AU51nHaZqOa5hHgx9fyjBfYKxeFh4b8h73Llyh9hPYsntlsvtv29Te6+W5NnJjPpyFFpfLRa9hZLLJfYsIKWz8r/Ud9DX6bny4xXqJSFkB6xHZfbHal0iZqTm1dFc0dz6zo7Nqc0GN9fd5OCzB0XO/SBxk6TU1RNxYzsWdQoztjqymHaanyI96QhOTm+jk7xErKaIBG00sXWDGXXzNAdt1n+5SbwIjJLKsA63Emyax4qdK8jam8XG8Rtxn+huP8+tpQc2dSQuk0fgc7eahuKGB+6VE1vHa8s/J3SInuv11xkROQK/2qk0GRLt2rIKD2jwE417qxXcW3rRrL5DboweIbQzPok+D8yN8UwgoX4FEl0OWX/fID85mLvjXrE/VyYRA9puLnHM3jWbPybVAs8CUOq1EZVBRa/jvUh+Ntk+9tvmIBdDEmG2Ybia0tA0ViCztG5aWtfPQ6MOkRq4mvjXy/g44mPeGPQGmTWiw6jJpYlt8934dt4gMtIz+Lv0b7ZlOCjYPGo90BvfQVf9CYSI2WDtnc4jDo2g0+1ginxvIbluQdQIdRzfNGsTCaV/2DfPvZ7sxT9X/sGlyYUK/wqW/ryUmnM10Ap89HMKpMe9HUgEKfHSh+id/x0lBSXwlnjcaqedkSIIoNAqCOgWgNZH28GhPWnHJCz+6fDREHtdbO2y4LZNP8V4yVJ2PLyDufvn4hLoQrrsD7FdpQJWqRVDqDO9lvdE7aHGltuaeYUajcQNmQ0uzfuc+vExzNoxy/4MqUTUxSg7V8bHX3zM8+XPI0idUVlExGpsVix3C+8y+jMH0rOtH+udL9Ag+YbcA524+edNjhbH8t42B/UQQP3kRTz0nePvtndSGVT0vNwTSUgpEPvAvSv8K9C7ZjLE2gW/YTn09+1PV9+e/MEaAIKkErQGDdW6IpbsmoVgE/gr3aF3k3AngZj1MVTFVPH4zce5q7/LHwf+6DBOirYUETUmCpPBJM7VUgGNbDG5fd8kuRvsfnQ3zv7ODHlnCBO1H6E4/R7+uVfwz/mS9N+7cvWzk0zfMp3ohxLttG2ZyVpMVpEq6aMzH/HqsVfxUgSKcm8aPTnDTxKS9gaqOheMFbr/R3N+naYO9HDjtxv2dc76zVH7OUbBxISfJ6DQKLj0zSUq0ipoUeQDcCV6Mo/eF+3zVPnjVzeBpIwkkk7/RMOgGUAbbamjL65HzcZcUIP/kCfp+WLr8VtiYNcor2S74lE8XF8l54u7mAoryOu7ACRiFqiXIhg3YygKUyW+vSNozPVHZ/Gn3ukyVqkBi7SZ9fPWs8awhrKPy+g0qxPni0XQW1ROFLM3zUYulXMx6yJ+yX6EDw6nj+0FnO6+gKshB4vXGawmK4M+HMRZ41mQQkLRp0RViHRhEn8QWlm9ZIKa1FOpQA67Z2TToHInNKijzlLbO1/rfo36oGEkHD7B9ldOo059gkYnxzyhvVGBp+UWGSsvEgWsdFrJpy9+Kt6jXTtbjVb0GXpee+Rtfsjs3vqrjBZtC6ZOJuInx3My4CQNbg34VEtRmmR41XgiAMlzku1awSFNU6hRXUFhsuBWraT7U91JmpmE2l2NU5kzbi09cTKG4VGajqKpmWl/T0MilaA8Kmf0Sy9ywGr7L9eikTKwDgPJqTJSju5HNmU8IxcEoNQqGRI0nn1/RXErbCnbpm5jzENjKOleYr9eZ6vHLKvHo9YD53wlB28dpC6njulbpqOSaoioeIY8v6+o9awl6onhHH7hML+98RtJzyWxa4dj7RYkFprKW9BWF3Gmy3JWnve398XNLjepjQvmy48+tJ/fFuAMNg1jeNUumsuruDJjGyOiFnK4xx/289oc0zaJlMTpiewy7LEfe3j7wxycdxCbYGPYB8Psa2BbwK7CfS91zhc4b+wFiBrF0/6eZmcDOD7kOF21zzPpC4fOdJuD3yyvp9G0nMKfJ5DQK+YBIILC4onc6krF1QzOrzyP72xf6j3q7ccDSwJJyPLFrVccx2r3AtDgdB1BauGWaTsV/Zvoc/4xNvb8Eue4cTSFdkEikWA0wpzkOcz5WDRQM3dl0u+LfpQPLqcssAzvxhGUSrdxYPReJqyegNZPi6LJASaVCAqkrXagVfIgEKHauw5naSC1ObUs9FnIH1WOtjYpTXz+wnq+clpA+M+/k+ZdQLUrPGT5g0tNO/jSZQdROwfhEWygfMx9NJeCFIkEBImEor9OkVS5kDvxYoZNalgqJwtOQjhYpli49M0l+q3sh1uIm92uvCEfyOaXPiTjkwwihkbw8NGHOZJ7hMf3Pg44shIAoj1F+1FibeR4pzh868dxY+00Dr/g0D1eUbrCngW/YewGmsqa8I73puRiCS83vIypFbAqtSiw/Kmnpk8Nwz8cztJjS3nv9Hv2+9xNuEt0y5NEtC7rBkvHNm3L+nX2c2bYB8M4mX8S1jqO190u5fT5Zrbc60JZcCNuKncajPUkZCQw45+J1MvuwJRO6HTg5ATmNn08cyGCKZ24WV1RdYqhQO9BEA8GCNatg+2XL0EwHCx1rJ9+5R5oWpoY+fM0nj/0J0V+W1BYPcnzW02SYSD9N87DFlxJwNAErK3N1jfzFNciZmJQ5FPRUIHKRXTgb1ywkbEbxjqeKzF3AATtaXoHAKXZG5tE3L+3339NvTyV4b8Op86jibyUiQzo58OCVUGYWoeQrZ39JkismFpM1OW1oPXVotQq0csqqHO5Q9XFcoLT5EiU4nivDO9Fm/2ll1ZSrF/BgOuRqNwXYPEJhHrsx0GkYfSb5Wffi+fX53Mz4hECSwLpcnMcFQkdwXht64B34zCiKl6g6nYFS397hl0eu9A76YmRqXFv1xeGegOXSi7x599/8lz4n2jMQahNQRiUJVgFKx9H/USpMoKUlSPsc5RbvRsxmV6gkBA5MZk7Na7Ibc4Mv1XExZgxXO96HZNrX7w2nwdnG1d6XrHX73bVbSr9Kjn8yGF+GvUTwvOCGJxuLdsytvHbid+YFDCJmHExrOm6hvw7+UglUpSlSnpd7IWQKNBpZifSNwGXgNb9VExWDPuf2M/QVUM7UKl/dPYjDBYDS1cuJdkvmUGvDeL/VAJ7BBLYIxCLwULGtgwUWgXl5eX4+voilUpx8hKDEsZGIxnbMgjqHYRPgo/9+sHvDMbU5JhvBEFAIpHQUtnCvYP3WukyvTFp3LD2HEhg6zLdZ0Ufvnv5Y16u/fcAWY9ZYPlVE6YWk11T2UfjT1TZSyisbuhqK8lde5Py6+VUZVShnRsAjmqS3XMOr74pp+xMDhe+vED85Hg8o8QAmrHJyOqw1SROSyRiWAResV4IQoD9Wn2Njt9n/m2ni37D/AZWqej7mfC3M5urNlNdbsW9Ir+1DTrW/f/098BVQxm8ahAqFxUtOuiTdZTMoNe5MbiM2tBJpBotmBttgKaDhIyAgHtZBnvm3kAaPhLwotb5LH+lbcBP68fk7ZMJ/DAQyxQLxiYjJp3YT22ZfcOPDKfPRQM3hzVh1rjiJg3Eu0kE9UiQkHQ7ibC8Quoi2oKJ4nexfv563vV9AzdpLKdbl/xwlzjCq57AYPyCKdvjKa+4QunfV3ELcyN6dDSN0tbMfpmNHQtCebTHHOr0YvDd2ixm2OnuDGZy0lnSBzXCez+gSLoFEXM6+I5zo3KJ3BRJbEUsLkEds7Fq79Vy6r1TdJrVieqMagSbgEe7udCsdiFpZhJX95RT1U5w2SITwRpWK9z+5zb7l+9nwYkFqGLDuRm+0H5emy3Tvh9rnE9xK2E1qrjOjNW0oyKlY2afb4UvCq0FtY+aGq+aDhnoB6cd5GzKWS5VX2Lo+0PZcGkD4MjUlUglOHk7kbk7k6oNVYzNGYtPlQ/W+eJYcHd3p7y6mn+nVAAajad9EbBa4eLXF7m4+iLPFT1nTyT431Ta/Hltc97/7fKfYN9/yv+IIlPLsBgtCIJg30DmReZh651M4YEM/O8JlEf377BI+RcbufnTRUa83puKlgoCnAOINgzvcF//XqFM+WsKgd3Fxaft3iXBJbgr4NALh1C6KBnx8YgOQaClPy/Fq/ZT3pHBpLWTSJmfQptR2Tn/F2Iz7yBHx9lPznL8jeM8dOoh+7UBpQFM/Xoq15yv8VKdmCH08arl1LqIDhKTCqwuVmQGGVKFFLPVwv7uYlbC6WT4zFnApR0AVi1xxaO5L6HFCcReXI+ht0gduS97Hxm3MxCEGQgImFQmTCoTKKTM2TaHM4VnOJRziDbAy4Szj+BdI6f6RjGqbBX4gNEmGhtVLttQF7+OzNqXlEdSeDHtRWKdY5maMJV/7oh6AWseX8MfE/9AX6fnxJsn6LeyH0p/hzPAoDEgc5KRsS2DwrOFnFKeQZBaiSt9hwjJTIb3guH5Yv9kl1ZyKmE0NokBrxovBGujPXMFYITno2RcDsCoqMC3YSwKT1cSJzqCbm392OV6F8bvGY9+WSGmvkGo3dWEJCXgJvuQXrmONqy5+jGVPq1o2PsoCdujXGxWG/0yjnEqqRurn/2dsrdf4bf+vyGVS1l4cqH92ojCPkSkHePIi3nkH8tj9p7ZxI6L7UArqIvpj8qtmv1P76fTrE4kPTaAc5+fR2UJxuDig8wJwgc7DI1IyVCy9VfwKmwg4Vo2luAwqjKqRPR2h0C0moCyAEwNDgO3xdRiz7S8m3AX60ArbqFuuIWKRpL1JUdkp/h0MXlX8ui5rCd1uXU0lTSRmjMVvUbP5B2TsT1cxOJzDgrWNnq0grACNJbeWExWjLU6lC5KuxabzCIjPD8MqaWaP4f9KZ5/ogDDh1FsSn+NFm0LLa5OyFUdg4RthsylXpe4nVTGW0WpHHzuJGGpYUhCJJQFluFl9aKhqAGNhwals5L6/Hpqc2oRDEFARwHtDvduZwxLnZ3oM7+P/e/63Vm89tNr7B+9H2Q+GJtMgLLDtW2UugAqzypRX6fRgb7970oFIFFBQXgBCTLRIPo27lsihkUw7vtxrc8QuHfxHqquKjxCPQjsEUhxiUhx5VnhydUtV0mcntghC65Nq2X+n/PZ/OOfPHNmLmc/OUvSjCRCUyO5v7iFuhHUOwhnf2c8l/Vj762N3Oy0H5+6Qcy3ws89f0awCTx69VHajPD+Nzfi3dQHl2BXe9YzwGuDXsPU3URpt1LcwtwoPl9MbXYtaRvSGPLeELRo8evsx8v1jnarvluNTCmz6xoVnC7g3Kfn6P9Sf7S+Wo69egynyfFkpryBf5k/VSf6YhsUy/Olz9vv4WQNoo50Fv+ymCbnvwj6bRCD3hiEa4grta2frsRmRd1Sg61F3+H9G4QimjV3MUrrATD5hbBk8xIyVy5hl3kV/W+/Qz+ZuAn0A8qBP5RqTprNePYcBf32IakWDbfYcbG8aXuT8+vPQ+vcMjBDRPWF9oHZn87mg9MfwLGOfWBUWZFp1Ry6spNHTjzCwJiBTFIf47ARpBYjMrMRq0lLG12W2Qz9Mk9yN+hV9g9eRqeWZ7BZRNCFTCmzO04kEilBpzbSonBnbvrLrHhRjkHpcPLKJHI2zd7EmIYZrWPHYQ5KBBn+0iBSTqaQ1znPTqdkzy4WpAhlZegtzdwZ9DhmjStWa8fdpc0moPBQYFQZKWhw0GX7No4ntO5ZKk5c49iHu1k/bz1EwzzrUfIz3TA1PcHw/b6c9tvNbauSltkteAU4dGojbA8hxUzehgukVVdwbOk0PJtS7cfzIvNILLGyY9pfpL7SF99Ovvjs8iGmVwz3Yu5xoc8FpvSfQl1eHQWnClDHh9vp1WxuMP/wfCQSCXd33kWwCva51aAs5bjkKbSrPZj37Dxix8ei0znmqH1j9zE78Et+7fsrxiYjT6Q/cd9aYsO9eyQzd8zEv2srDVRrX8ksMgLKAqjyqWLsR+PFs9tRckky7xJz8QruPaLseieHcg5xquAUZ4vO4tbLjbFeY3GpdhGpYSyOueD3Rb9TsbKjfmm0ui8aYyh6VSH9D9dSVJXPtTGvYpPIuHLlvsw+AfY/vR+5Rs6Ij0fYkc4qo4rhR4ej71vCkVeOoHJRMfDVgfZNs1bwI7rsZYQ4T5a+tpSlLKWoXM8zx5YAcKtXDfT6ig8tc5BIJEhkEnvGUP+Q/rzj+w7XzlzDarIS2D0QJ6MTtIJoA2vmMKYpjPRn00l/VkT5uzznQqNbIxEVz9jrnrkz0w48kUnk2KTNmJQmzGonfFKCGPnFSPw6+yEIYJGLgfhLsWOJuGPj+u83KSpoAAm0WBsYpYU//WFSKVRIYfzOJbyX/DnlzTX8O6UC8PPUIDfK7f0P2GmNQKTxbChqwCvWi5hxMahcVR2AS/c7cJJc+9I9dwsWxUmykyvoFOJD0KYj6F39EIRke18MPj6YPhcr0E9uAESks68mkE4F35IetpyC2t34Ns+g6ohIKdHiHmy/ViqR0ty5Hzl+/VjYC/TnwVIE16L+f+y9dXRUZ9v2/RvLxCfu7iSEBIK7u3tLgXpLvdTdXShVSiktUtoixYu7BwsQ4u6uk4zv/f2xk5lMuR9b6/vebz3rvc+1uhpmz+x97UtPOc7jXESnuhiZoAIZaKZqmDpnKr6JvljKu9qsiqc2YiDPjIrh9+lb6L+iPxGjIxAE6FAX0BzRiHrAZHJ2X2L8L+O58tIV9E56WwadxYRY20zC2b00hvRBnuTED4/+gEyUURNYQ0a8CzNk97Fx/Eb6LO1D6vLUfwC9BGT+fgRNSOSGYN+Xax9ey6hr2xh0TNqjc+Nt2SvFLcW89tlrOH7vyMLVC3mp6SXa2+GXLoCQTFRQGlFK49xGhswYwmsbXqO9pB3q5eh9FnBwYTGr5o1kfB9bXeq4liexdHigaviGaX+noLtHZ6WCSvZJY0R2OpVev+NSsxv1bRmB/V4F4NnU5/jmxJ8MP3CZ4XJ4sMdZtE6m5hxmzOMsMBQcjYHkRToS2TuBgY9LQAk/xxB820Jw7+yLTl2KPk/SC5VOUh9vEidR6Z9OSkYK4bs03PC6gaHdACK4OLgQW/Umxf6r6XTpxGdQIh0XMvH28OZa9TW2Nu4CpNqK/U6cJM9SQa+L6WSG1HC2JsO+LtA/Jq/1LEGGurEa16J02hoaUUbYg3+21b+Nt3o9nnnTmH7kYRZ+YgPrHCs+hvI9JfO2z2Nhr4Us3Cpd6953tY7ZyE311H4UjgrVHRRS1UHVOASF233Wc88OLlNQm3OBiruccO5wtsuEiy+cQUjeJdyWR2D5wkJFjY1+VCYquKv8I9xOFePyaTQHO6V+6t5jAGpdjtLhsoSgEVEUiXLagRvhDxD13QE+m/gp9/S5BwBdkw73SnccjFK/iJhRyBQICoGqRpE3hxylKnQvSCzLyEUVvcu+467lHbxx/M0emVyS/LbwJe4S9/LHzD9IMCTw0KqHuFlazqXGgyADs8qEqaUDr5oynDu7goaYkSHnlgFqH/mFlN1PEnevG6pXVZgcJNtBlJlp9Gnk9oRRjNy5g/iseGuw71Spra6QaaCJif0n2oC0XXMkrCyY7F+zSXs0zVrTVmeS9DUP7SCedlrNOzIpa8lxoyOKaAX5HgIdjnkUB+QRNkyqi6sJ0zDg8QFWNhWApEVJ6Fv1+CX5ETtNAk2Zu+pcO5qCufbZGYx39abX3F53ULImlq2y/n1j0w28rnuBO9bgb2BbIMUnigkdEorSUcnoDaPtft9wppCMXy+jGhWKX9sk1t99m96/BNPi0cLt3iXE+Hjw61PXOHbJlfs+irNSvc/eNZvw8r04eLzKZxukYMjGPtIZoNK14VdyiXbvSMq+yybBxYecEHBSutJhlgJWi/5chExxm+jNE7lQ+SkAfi2SbpHIcIJKq6l3qUEQbLqVR2cagY3DWLpuKZ1ZnZjSTBi1RiaGT2S41zzONu3oGus76egBjKoGTiclM1zosAv2uce6UxtQi1xwoD4sCYdIJb/8An/9BatX24Lz8TnxTD3gRU7lYY7tvsrs3+aTcncSJ2MHoHMoJ/yPRSSU9iLwx9fJzob6AgUiGdbnXE3cS7vjNBbG+FqzkLoBaN1S9XUV3xz5hgcvPWjdg6oDq3n/tfeZF/MAJ946gUeEB33v62vTdXQqfEqvcv6DAnrt68WpR06hc9ZZKbY1tXm4XTjG8N/Hc//5+yEHduW4Ex/wAX2Kf2To4du4965B16hD7mdk/34Q+0vPDi0PZdKBRPSzFIz4ai5bn7S1VSYqyE3IxVHtxuCTLch87fu9mzHDIlhQOiqJ2xvHy1t72KVKNUXRRdyeeJuwYWEIW7v0TZkCdYGaqQemYhlpD0TvpgAOrA7kxokbxE6IxcHFgfCR4Ti4OtgokmX2Dmuz3syu5bsIHxVuLYnyT1GoFbwlvoXFYuHWrVv4+vpiMVloyG7A2deZ2hu17L5vN/Gz4u0Abd1MGXW361jbby0jXhvBqDdH0ZDTwK5lu5i8ejIg2cE9ddfU5akYXjX+jwBZjo1O5OzKQeWsQhOqIah3KL0qPwYg9BkLi74YwLV118jcU4KDpRmAgGaJbrPSEsCFfJhzvzsJsxNw8XOx3luulBPYNxC1u5odi3cw4IkBqGbagn0Ws0hjfiPho8Lpe39fZAoZrc5SYLfRsxzXgIGEvzCV97twCN3HqUdNDoH5Zyje0Y/Ug0eojhtFa+IQ6/Xk46tRd7Yg8pp1zxVF8NaOZGjuaRITIUuASw+tRlQ7Qu9HrOd2sCyNKQfX4pd1kHKhA/mS7kxx6fq4qHEkTU3Co48HObtz2LF4B+N+mUxq0SYUghPyoIOUh5bj0zKEhPPr6dQEIk5ZZOtwUc6pUaeILEnAv+gCMtkQAtpmUuq9TnqKKNjpvN1lDwxqA82eBlwTw3ih/gVUzlLw305XRqB94y4y386EJ6DZ9QLHe8fiZAyjv9Mx9O6O+I7vg9bZDwQkqkzPVE55LadZ34y6t5pBi+6EV3U2dHJjww38kv14IvcJtEYtqd8kYwwfTErpT3R6BDHv9/lsmAmF/p/3aE+Xnl9WSsahDHrfk4JzoCcmAaq8/kBlVLFs4zKUcVeA6bZ5IQjoHEopc9mJj6UNWX0DB56+TNLCJEKHhlnHAuDBdQ/SGalnVOYoNsZtRJ9p8wFO2TGFq5uv4hXjRdDAILLelc7mZl0zZsGMl4MX7fnt5O/Pp+yPMmJiY3Cuc0bVxRa18L77WPvZZ9i8sP+x/AgkJt8FogAyqVZ0wuwEa3mW/xtFFEVqamrw9fX9r7/8/4H8m8bz3/K/QhZsX8CgL6WNtycCwyJaaD+TgXdFBmBPdxKRp+f8Gwe59PUl2q6aqNZW0yZIFB6ujaVEXd2GrklH8t3JuIe4SwdJD8VQRKDwUCElx0sAeyOwITiK5oB44mdJGxhgRydxffIrqO5ZhH8ff1KWpaDysCEvN87fSGiIZBhYpYdR7KhzRLZPxuGVh9k8cbNdVgNIB15zUbOVwz9S3R8Hkx9lwWfJHBCDZ79IBFFg2pZpLNq+iHpRooVwMDjg2eRJ8+Va3P92x6nTye6+JSP9uDUkicuv7mLstrEAlJquEF/5Pkqzkt4ZKjC0EfdxHPtl+1l1cZV1LL6a9BXiWyLLU5fjHuzOozcfZfAzg6lqt9F6+tX60VLQQvHxYi5+eRG9uQ6DQzUWuQ55cyPrBq3j+i8SjaLebKDNRfr7yW+fhN0X7NrqpQwhon4FqTkrSLlYjzbXvs5HTyqAwuhCDPvzWB2xGkOrwU4ZdG6pxLMyk8nHnrdmr3SPc7RnNB9v/ZiLQy5ye9tt9K16PnN9jzE7djP59Fpi6r9G4eiAZ5QnHpEegK1+XVVQC7mDl5H26EDSHk3DI9zDOnY9x/G7hO9I/yad468dJ2BhAqdmDaDOpxyVrg2h0x6N1ld2Lwk5H3PBeJ7V7p+w+uQCtnqvZ/vNbVTXVFu/lxefx2cvfkXU5FTKz5fzY78faTpqT0XQM3sPbBRmADXnazj97mm0NVpWVqwk7r6htPhHUhVYzeHxR5D52gfkujNLz444S8bwMBpzG/ky+EsufGEbMwejA0s3L8L55kVrJmNgWiAWHyWlEaUoXHpTnvYOYsJorvx4hZzdOdaRBDCqjXS6duA6oi/3nbkP71hv69xzz3fnq7Cv2Pm1hODK/DOTTeM3YamzOWFrb9Vy/vPztJS0kLEhg+y/su3Gom7neX6b+htiF+WVPrcRrYuWiJIIpuzvj6G5gys/XuHGRskZWigepcx7PZ6lu/AvuoBblC/3PP/If7u+zs8o6UjuesOu+ebg6mC/JwDH5x/n4JMHEcwCRq0RuSDH3yWQ4/HHSb6QjDxFzql3T3H8DSmC1B3oadeIdDr6o63Wcm3tNepu1dm9b+KpH6j6cR8DVgxg6eGluAa4UjGkkUMTfqEmsIaa4DbrGAX2lwyS3mXfMjDvIGP2lRCdsYt/bEvSO7g4EDE6As9IT/QtehIXJPK64XUrv/w/pSc1L0BHbQcFBwvQVmvRhGp4+OrDKMZL8yXlRgpFn2y3Zrp2S2qzhCrudLEgc3MhdEgoY94dg0e4h7WNBldvbkx4HuWoYRx79RgZGzKkvkfASQvH9j/NxZMatq4PYkCvXvx5aQzVI/ZgWQlnRsAjPjDP2Y33g3sRd/9rjB5XgWeyVAus3v0Ia59by8nPTyKTyfh4gmQYOpkDKfT/jBL3h6n79juy/8rm+aHP8+rwVxkTMBuvdoki2bUzlJKNpymaUERESQRKuY2uz6/kMinHVlG+8wr9/v4A35LLmLtshu5gjoiFDWM3sCpsFYIoEKZOIqL2SdxkASjby1Hpm1G6qBHlCnqmJ3RTvlos0u+D82zUWy76ePYvzufp4qeJmRxjpUmx1g11vUBV5e/c+vww6k7J4LUItgk2a9csLHtuofxcydKYpXZoeJlFjnNLFQonB0IeCKHFo4UEnwQixLF4dKZRElHL9nnbwSLj1qZbvLzzZZ46eY/19+7DnyR3xEp6vzaTH2f/SKe6mAqfX+3mhMqop+piKdpaqf7dzPUz2bhxIzPjZ3J61GmcZjtRmV7J7nt3U59hO6NEUSrA7hbkxsk3T3L05aMM9JhGv8I/CWpaTLZiHYpfFbY6qiLIRcnhkp2YjZDkTdCAIGuGcPce5ahzZOTJPmgLakmYlYCrv6vddVetKw/+/CBO5237eHfmS4eshlvaNTi3FBJ69wj63ifRPJ4oPsEHZz4ApBoWwR8F88D5B3DycsJoNpEZJnmNTA4m5I5y1qat5ddRUj+5ywPwa5UASLfSBJIfGICgUIFcLlGT9tgrHLOvk/5NOkWHpQh2lGsig/IO0eHSwdqH1tI5NJrMLZnk7JL27O49aHLBk4zdq8VSWALA2A1jmbrDnta31izVAKy5UUP5+XLkMjmOCkd8nH0Yu2Isz1c+T8igEDobO3n74NvW38VWv8aSWS/hHmpzzEr1o8AoXkBTvo3O0gqeq3mOe0/dax2rW2Er+G3WOA7MNRA6NpYhzw6RzpIe75t6PRX/K/vYc/9uOn6TTKQkt+EsIJXWFhcCvnmAsLxqBAE8gu7mZ/l/b8/foILQygF4+nky5espbJqwif2P7bfqDDJBSZgqja/CvuLAkweYtGoSHuEedjqt0GONdb+TSdHM4cET+G3ePaiDvAksPIdHTQ6CACpRcjZpXbU0eZtovFbKoZWHaKtow1PtQ0T942gMSUzfN50hh6/hEheE+/ThiHI53U6EUv0tbrt9Ra1mn7X2J9j2AhkK/Gr90BfpCRkUgtpNbXWiNvnoKe89hfAxUSw/uZzBz9oANdkhL3Ku12AKHXYiRseQPigLs9LMgQVn8G2bwOWYmShrX4Xvf8C1pYIin1fQOmVRG1BLTWCNtQ2CwUz11WpOXz3N95e/t7Yr7UoaI46qISGBtI8XYHJyR2XxJKRhGSDVzGrzdiXpp6dJf03Jnlk2doOKtgq23N5Ci76lR9/b+r177+0Gglmzf1Hg1zaZiPoVxHnYAn3dYyVDQVVQFVeGlqIJ0yAKol0QrDDgE7bNXUXegl5YjBZrvb+AUYmMfA08xil53s2VxSpvXseHDq97GD2uArqW1buud7Nnxi/IXG1Oxu52y5ATmxeL7rCOqPFRzPhpRlcfSs8vjiwm77EGHst6jDeMb0jAEcHm/AWQezrxyLVHGLJyCDdqb3C+YzMg6XdO2g7Ugd5kDoigOrCaAKcwuvU3n3ofEq77sO/RfUycMRHZOzJeuybt6Qoccawrwac8g6FrF5DR77b1eSEN9xKsTiCqwIXosy20d/xrMJVLhwsddbY6oN3vJBOVCHIBwUUgdFgoU76d0nVd6pQB+ftwlnmTtSPLqhP0tPPOjywncOk4ji86TnipfVDQp96J4NwTGNr0GJIN1kDguBtlTLtqJnDALHKG3ItD0J0sFyDVSorMaaL/6xMxdNGXGpWN1HRUozPpOPrKUdakrqHP0j7s/nQ3pRESYKbSezNyuRxNi4aTP+oRr2cgv2ADDMhEJT7tYxnhNwOlxbY/BmgnWv+2YKL/Y/0Z88wY1s5Yy5KwV3EyhIMIgZXeqEID+PMBFdmJElOJgJmgpkV84we/9auF5Dja00x2mQOizMI3JHCyd39SflzBsXG2jOVuce5wBi0kLUiSgAw95v61ftcYcmgIk76cRK+5UgDKtq7kZGfbvvv1ma954cgLlLtttX4WMjiEu/++m9ayVpx9nK2ZOQBDVg5hzDtjSFqYxOCnpT3IRuMpZ+6RR4mZHMOmiZtwzLbfz13bRHxLLtNU0ETu7lyurrpqVaO8nbwR94psHLuRklMlbJq4iaRM+3XvMT6WoWuW4lWVSUjWYUzmLhsxuIoD0y7hFBtC0feH8C1OZ9UqQJBsgMzemVTE9ELXZkRp7ARBoLVV2keUpk4CC8/j1lSK6uY1fKslJ7yvOpjRt/KQiypOjTpFaZykO3fXNhTkUvDUGABXp7yKReXIvuGf4NRq0zsscrjR5wbyBDlZ27L43O9zCg4V8Ers7wQ2ddnIMuO/1P9BCvqKoo3yD6DqTDXjj4zHrc2Jv9McqDbmc+7zC3hUZ7Fxow3YJMpELEoR5zAftH2G8OnP3hiNtvW8c85O/l4cRcelTBwP7UZh1Nn5Tpq9mimOaWbnmG8IOCZlOfYMAAw7O4za72sRTAIymcy61kW5iFllRlCIXPj8AllbJUd4tw7mqIOIW/tQOio588gZa33g7mCfRL9kuYP+LjfkNbSO2Th1OqAwwK1xT1OWPE3KgO1qV1lYGftmZWAJDLmjT7spNUWZBedIPxyu6HnimyeQCdIEdFBIPp0hu4ZQcrLkDrter5XmslkwU3OjBr/f/fCt80Uuk/PJS5+w/ORyHln0CJ/7f07dRglJ1c2UcmHIBe4ruY/qa9VsmbaFtgrJJuzWV7K/yOaXEb9QeKRQYvqxCNzeepvK9Er+KdsWbOPLkC+tbFM9pbO+kzUpazj78VlryYiSkyV3fA/AydOJmMkxVnCob5Ivi3cvJm66xCyh1jai3vSz1Y8ESCDcf3m3O6UW8PTywDPKk61zt3Lxq4t0k3I1uZzn1ezZvHH+DQY/PZjh3y1Gox9Kr/LPCWlcDoDcbMSoNeLg6oAmTGNnz6ucVCw7towx741hwbYF9L2/r914q71ceL7mee49eS8py1KQyWRoHaX9d9Oy9Uz91pZZCz3OdYsZpbEDGTI6PYIwqbtsi67t0uAsZRa2lbdRfa0ai9FiD3Q2GVDqtQRMSMZ1UPfe1RXQRYlX8TVE4MGS1yGoKymiy180JGQIr37+Ko/98RheMV70X9GfkPgwQpruIbBlHsM8FvLgcw9SnDoZnZsfJrULZabrlPquocHtJCBD76hHabag7mhELoe+5T/hYPJh4KWBWC4UYbiVh3eF5Hep0lbQ6HaSsvAydi3MQdM/Fmcf538Z7NMKH9HQnoMszDbnOh0LaHfKRCZYUBo6iHp6Ouqxkh3vaogj0jSDvoF98av1o3ZVrR2ddLcEDQjiFe0r1nNk041NFGozKfddZ/1Od/92s4UAeHR0BQ7rG6i+Ws3uomQeeUmDyWTb33yawlF1yrj28zVOv38agN2Nn5ARtQyPZg/ib/uirCgl/et06jIln461prx2KKdGnaKhrwInlRNb5m2xa7eD2oE+S/oQMijEuoYBJmyaQOiqUE5nnWZNnzVYDBZebnuZVdmr+KjjI1IiUwB4duVKLsjlVkr5GuB9YAASr8uArn//DVxARr+LKhQm6VyyWCBiVARDVg75D5mc/i3/38q/M/v+Lf/rpHtzi8mPQf7XJXzvmcSVK11FYVHg2zqRes1hclK9WfBwMgefPsCN/nKYDiqZFOBy0LXgVZ2FtioZkA6wno6U4WeGM+SSwPLby9GEScG8boU0mgm0hz9Dezh8vg2USvt2Icolx6oc4mfGEz8znoImyVnq5uDG+AnjGX9jPIJFIHdPrmT0dyms/s2zmJU/EeXfUm0uv2Q/a7BPJsiIKYjB4lvN2v4bCUgNYPnx5YgiGFTVtLjmURrnjsLFkbNfncW3zpd6v3rKFadQiveSmJXI7N2zqUzKZODtgVx+7DI6Z1umS5zmPSpalSQ8cpMWrXTIJTvMxq36NQr83+frJ79mXP15Enxirb+pbpeMBAeFA9k7s1GqlcROjcU/2Z+i5h6pc8D96+8nPSud5TuXM+ylYfz1k+SMNCrrqXA4QGh2DenfpBOQEoDgZfVQcHrEaQYHj+Dm5psonZQkzku0HqYOulb8Sq9Qt6OdTz4uZfaG2ZLi1xW0LYwppDCmkA8b7qGfgwKloxK9yUCnQxUKx05ii6+juV2IWfcS3ZkzA4MHsqL/CkZHjEZ+Xs7tP2+zfeF2Hr35KBHjoyk5Wohn52hcKtxpLW9jziZbQexuR2et70VqgyBmRgK9FyTY5m5Xu9UdTXjt2GGbb68M58Dm/biWbuB63+s8vuNxOujH5kmtdDZ28vCVhzl35gsyDr/IUIXAw2YIQEdNWzUbP/yQ9955B8Zhdfokla3GRemOxdiEvkWP0mjb6qMKo1CeV9IU0cTVn65ScrwEcZzNWeC32I9RS0dZFWqT1oBbSyuDymvBERS+UHikELlSTuSYSDvniFxwQO3hxMCnBlp58wFMKhN/TznCIOVyHtxjQy0dHfgl83TzODdNCkYKAhx54QihQ0JJmJVAt7PIqdMJj1YNsnBnwoZ3rUed9NxWTSsXBl9AcOsK0k6MRu2u5nyzxvqcyvRKjrxwBL/efhx/9TgeER6IITZEa+PhK9TUNpO1PYukhUno0qupCarh+Njj5CY2MtfjFY59dBZnH2dSlqVwRfYduZG7GL/rXZRCFYIwhAcfeoj33nmHS/Cf1ge4BJyRCQRGLGTGT9H46kR4g67sOZj9x2x250r1EKePnM74meMpPlHM5ombmf7jdAacqiI9ZgbzDs3jZ/XPGHcYMelMjH1vrDXwenk4uDYvJGSwwItNL6J0tK/3JMoVoLBZ46IgImhNqIwqLAoLcrOIIIhMX2MbK88OSbmti1BhcnSnIr2SUy8fZNhLwwicFEh2Qzbbbm8j3jue5anLcfRwxNHDnr7W0G6g+Fgx3nHe+Cb6SuhGZxsQInF+IonzbRm6gf0CCSlzg3rISM1g3tCPcNQ4cvO3m3hGeRI6JBQHixd+LdM5O15B/6jZdn39r2r2nf/8PLFTY0ldnkrTuQIsR2GozMB9Fj0BtFGTU83a/FzOnRVhAjAKOkZBXOVzvN3vafx9Bb6s8LLuMXqHCm7/cJsWpxYi+kfgFOzE1Nip5F73pU5zAEdZHvKyUC5/JwXTPpj7ARnesPDaWzS5ncWrQU7pznOIMmjxaCG3rogkMR2jMooOjyBqIweREuxFm08kRid3bIzQckadHEWfXA0R90fQEdaBx8cerAz/nd7lX3PC35/0lXWMybhK/Y0qFEYPK30ZSOjeoeeGEmBooOZWK2bfniaxgCiT4xHhwd6HpZpagzMGUyNIjtgqz63sdC/mgfiVOJ5pRO/qjdlic5LF5scidjTwxonP+aeojSoSz/5Eo3caquciaDzeSKA8EL3YjlFpoMG3mjr/UsKnprBn9G9Ul1YjtnYF5ESZFTHuEhVAZeGdjoWnv3oWmSKf+wteJTAQTn9wmhOvn+CRjEd4ZtAzzO81n0EhgwgMDOSuvXfR4uVMxfnNKAQXbvltY9rXrXw05iOmfDsF0SKSo48gqDmONqcbGEINHFp+iFRTKlVXq3CPD2LqNR3HkiPQqUsRRYEpX0+x9WJ3IN/owMBLiTSFlQA96md2U1A56Tg08RDDmu7hz7l/Mu2HaXb76s6BqzmdMJrfe711x71Bygw0d5gRBRGZXIZJMFsDoD71PtRm1OKb5Gu31roDFrm9zQSPiMLl10r0rt7I5Y5268blyikC0wJ58JKUkeeq9MC3bSKTM4zIkOMTAo/eTO1BG96drQgO+lbEaxmsithJ0Zgiq8MawPGWWtJJUls4vPEw1dereb7+efrQh+TkZOv3Ouo7+CLgC0qHlMKkrvcVnHHw1TD+k/Gkf5OO75u+6C9JAeUSz8cZvedeOhK8kMlCrI7Z87r1VHn/0TWNLHfsDd0SXRiNV9E1KoYtJDPwewASXIbg35RBp6Idn5p4cAjk5vdnSXIeyx7xv7fnnzDD0/SjPkvS7bS1Wpy8nayZfaEND+AVGUF3OOOH3j/wuuF1O2J6yZloO8t71lsDKeB+a8yTCAoVsSJMa/mb48ZPuTLgJWoikxhV08rNNRfps7QPopcUCJAh50bKDSyu47jnw3vJePgHklqyKVkozY9aYxEy3Sck1Ewn60cN/r+cp2XA3dAjoDL2+FhKfijBtMiEUq3sMTelzpcpFUSMirC285jiOWo8d+Jb50vE1TY0z3hzcuxlzCozCpmSKzFz6HDMJzs2nz58SkZgCqcGvG0/TgXRTDw8CfOyVp5rfA7Veyr4GxwUHtAOztf8OVW1lkOf/8Lvv2gwOdxNaNhDpBo3oPNOR2yqQ92hw8HPA321qSdAGgeFAyWRJRx58wgfjv6Qgz8fZK/uEEV+4UTVPWMNcLr/4s6BYwdI25tG1tQsZKIcdUcTzq1VNGb5kvHuJTLCMtjitYUo1U6MygYqQyoxexdx7pNzXPvpGi82vYggdAHvRDmNPo10OjixOuprcHPjkSsPgiiS5gPPjvBl7aFHqIwbRcScflzKUGPpUeewxliIi7MC48HjfPdLNg+lP0R2Uy6lPhdodDvBxD0z0DXoWNq21DaHul68TdOGJdjFCkQA0JsMtDtJmbMezR5cWfgtjs8MZN+gfXx2/jPr93J65XCa0YycPpCSlhqavJsIlsdZ7x1eGs7wI1Fc5SoxwTEc6S/xBk7MqCcnaQm/Dp1Of48v8dFkUa06Z71vh2MuGlUip0b9hSmiP0kvnqZf01yupf5lNxc2Lt/IiMMj+H3m79y15y4W+72LLP1JajX7KA5YRcO9DcQeibVm+caa56Fu7E/a6Voq/3iP8/0DaatoI3V5qtXBD9Dpose9Ty8Ck5pZXbXa7pmX+m3A03yDu9JCObXDaIVOKwRXZDJQeGnQemuo+nk3T7S8xrezPrD7vXejN8Gl7eibdAhCV8ZFlxP1w7MfEn8yngF1AxAFkcwVmUzcPJGjRUfJCXmVfrKFDLo0iKEXWskcuYKjA4ZZ79td58liAUEmOdt+HL2b4z/M5EjIKMLz2/Fqq2bgiTn8mvEr5y6cI9RxPv0LdnImsR/LNs6kceBxxBSbTXBQ+ShefsNJU4NCr8ar70xuzrmApUxq7/gbVbi4mTkUGEdqRj9MngbaNPaUiP0C+/G049MULy8m3y+fQU8OsgPudrh24JzgjMrJdkZZ9xFRjmuEVBtPEAXefvdtAJocr9o9wzXAlcQFif8huKyndAfdOp2KaAhowq9ZScXFCjZ4b7Cy3QzOPUFUiRvhmfuovDyPad9PY8KnE3h7k/T8idETSRiRgEekBwoHBZWXKgkfG85tbAFrRYALfj5BuDecwEHXylPHP7VeE7tqD+cPvAtLV43Z/u7TUZo1XO93HY2qNzfXXCD18Eluj3wEUQxAFEHv4sPNMU9hcXBCM2UopzrfBKCkI5vehli8xHiu97uO600NvyR+xrjY0RyYvN1aN+po2xqi/aPwaxqMe0KgBPRBykQpCdqIW0wy/Tv74x3nTb8V/TipP0lBSzqu+i6Gh/8gsw9AkBsRzPY0nodmHGQ4wykLK+P+9fdjHHKNoJxC2n0i6exMROyy9/Pi89B51TB33mB+L5Z+W1wM3ZuzUW3E5CBDe/IKqqIKwlqheWgczvpoaxkLUSYQPCaO+kpprUfWPsPr8+bywcVXce50Rumr5OFrD+Po4YioleafyqjCr84PpaecFbdWWIM0ii42l/LA4+ycp2LXK0/y1vlXMTRK66o72NcaEE/zoHg66q4SlxtHXrytDm+lzx/cGneFQYOAS1Ds+y3VXn8S36UxtGnaKIitZkxlC+dfOoFT20BaPdWkx06mxTWd8UfG0zu7FtUoCUBmUdgc9Q4KB1y1rvQ524eKSxU0tDUQURlBSWSJNM773fEz+RF6LZTM7Ez8d/mzZ8se+qT0wVnljFuAG0atEe94bzo9XKHdBlo3OZhQeilJWpCEd5w3roHSudCslwKde8/vJTkvmc0TN5O4IBHPjzyJuh5FUoh9sBvAK9YLs17qq5KTJShdlHRXiHFwdWD0O6MJHhiMXCln4JMD6WywB3T+2PdHXANdWfL3EhbvtmX8OXs729Wjl4kCsvZWDK3S+Pw5509iHVLZoC5hieG/pvL81VHNkmcfJnhgMHN/m4smXIPW0EmHuppW52uUlJ7Ao9aCYbABQVDjqk9ERORG5H1Y5B1M2t5IRZYK3Zzl6Jp1uIe4o1Tbu9lVTiqrjXujR7m3fxU8D69fQWHgJygsUmZvS3YVro0mtN7hVv21Obg3zcG9mT8b8q5LdNSO2GzfvMFLQRTp9/1Brv2QzrMVz2JxcaXY7ztkopyUkyGkXs8gePcL1LY7wx+2c0guk1M5ZC76TtEugN99rvfM7gxKCyIoLQhdD/KcFLcJzBo8gWMdUDBQotKWGT7hVvjLhDbchww5ATUB6JzV1IcPkGr2iQByhp0bhsyrkA6HekKLa2gMSeFk9V6qvLtr/QpY9CZqbzXhFuiGs4+zXbDvns330BLfif4zAc7Y2mRU1XE87x7GHEmgPnkWYmiq9VplJchSFPg0+NC0rYk1P6/hyfwn8Yqx1SKWK+RWIEnWjiyaa5ut1yTwcBP7HrlAwtlahCQTuSEwUP4Ifi1SXXpTYgpPrUnkoccdEM3Q1gbIBEwOFq5OeoDhKUFkbtlI9fVqRr4+0gqICawOZMKhBNrHKVhZuRK1Rm2tGykTlMhEBeeGn8FNOYPcPblUX69GaVJaa5l3mjrp/X1vbq24xfpB6xnnNI5j44+hVnZlwDsilQRKC0LuLKfD3IFKVOHYdS4FBATwyaefMvWll1hqsfA9ENH13gagFfgVeAsYHD0Iz9DJXQBj+EcVq3/L/w/y78y+f8v/CslYn0HLmRZkMpl1Q/do8UCeWYdcKcfk1JN6Q9pg2jwVxC5IJWfIveTHtwBw2bIOEYGm4D5cmfYmpk4TXwR+QfHxYrsMOqODEZ0zdrQv/4q+wP6Alv5xJXYWGX7hlFbtt16J8oyi9eVWyp4ts35mMVj4Y9YfXPjygu2dOgfQ2jqD4a8M555D97Dor0V27VqyZQmmk+cY8ORg6N2bujobYhikQ1pbUs+JlScILQ8FoEmWj0JUE9e4n+roYfR9ZiLpL6TT4tFi18eKrpM8ZGoflBMkBcXcVbsPmZwm7yZULc3sumcXEbURALQbJdeUUq7k0LOHOPnWSQSzQGdjJ0U19sG+0yNPE74oHCdPJzShGsxdxeNbna/yd/xSzr92nprrNWT/ld2DwgyOjzuOOTWM468f59wnkiPglvYEt8KeIL3Pz5yeOQGvCWkE9g3EUeP4j7qNkrgMimXSF5NQOavIb83ieJ8ojsT15o9hL3JoajF16cXWwu0z42fy/bTvWZi0kPl/zOex248x/8/5aMI0zP3rHj5+5WcODJ1M71M/cO6j03bP6YmYaXe8/S+R+N1/yPWdTPpqEm+JbxE9MZq2bZUMvTAUnZOOC4Mv0BnqiCZcg2eUJ6u++IJLR1/lrChwygxLkGJ7S4BDej2nLRbcjwHnwUXrQmBpJ+3lLUSMjuDpoqdxnWBz4PTT90N1WIW2RouhzUBnQyeizGbYP5/+vFQDTCHj5m83qb1ahE5ZwO3QZ63f+fvxvzn0zCEAUlWL6FOyjoX7ttP/nAXXQDemrJ5C7BRbUNisMnO1/w0MgRFYTBbMeomSV3W5neTMZOQdjXgX7kBensOSA0uY8PkEAMbV7mdw7nH63OzDIz/ehb6oCovJItH5dmcPeDdxaPIhxN5dSlHfQAasGIDC3YZuj58RzwMXHiBkcAgLti1g8teT7dau/7JJeEZ74uwrKdWerw7g2Lhj1PvVUxBbgcLJgcW7FzP719nS8HWt13X3f87JyTJaC+pJfyOdFx9+mRlqtRX99E+5BExXqYlL+Iy0+j+Rq0ehCo23+05PJ/6+kfuIXRSLW6AbKctT8OpyVql1krNR6BS47+x9PHT5IQD8ZEmENCzHq30EgwdL1CFOnk6onFR2zu3sEQ/jf99UcnblcOzVY7RVtuH6WBGvffgaC7YtYMU3Q2yFq3tIfuAH7J/8C/m9wNRpprW0FaPWyIWKCwz5eQjrD6ynekA1R148QtSEKDyjPSk/X47FKK2L9qp2/pzzJzd/k2p3Dn56MGkPpd3xHADBLGBoM6AwS1ZhbUAtygHhyBQydt6zk6s/So4eN1MMAwv2klK6DlGEm1sy+ab/BkqvNljfWW424lmZiVBVzZN5TzJ9zXRWffEFhmO5nBbgqEW0W1OnLCLnRKxrasj5IYw6JFL6yR+kP7HRbg4A/Hrvr8Q9FceGMRtoONXAlrlbMMqbaXQ/QWVIJUXvRKGt0XL8NSkDUxRtwRatq5Gwu4dxerIPDb4NlGkL+apjEHXuB2j3iaI8aTKa/rEUDLybVv94TCaRI32CyQ96FyedE05aOSNeG8FT8U/RbmznnXwpQCvvOhM8Gur5e/ZPWDrWIihslphCpmTQpUH45LbyUtNLXB1uK+YtImIxi7SUtJB6XyrTfphmzSIDkItqtG5a0AtE3NyLc1stoiCnX6nUN5+/8DnCM8PwP+xP71u97ca10mcXF4dW4zsigR/WSPMisy6Tz1XuHE71RZBLZ46gEHHzcEOUd9EJCmqQiVSX/4pbxR+YdSYrwhmkuirLAz9H5jkavSbBur7DR4Qz9IWhOHk6MSZyDM4fO1P0XhFuQW7ETY+j0amBjKilZIY9Tp3TaUI/CuXQjEOEjwgnYnREj5bLEeUiRqWRjF8zqMmosc4vmajg/p/vR/wq3e5dYxwH4t02Bq2rljUrtqPycuMj94+syOXus8qoNnJh6AUsTkpyd+di1pntzsDu72a9voWzH0t03z33iZGnR5I3KI8dd+3A1Gmyc1jf/ddK9szew5yNc6zB+xpjIZXe3fPYQtGhfHqdW4drU6mEOu/x6ObxC5i1fhbyLnCA7Z1lyM0GRJMZR42jlTYtWBhGQPMcqhLbODVrGGJCAvWqejsnFYBvZwiz9sxCvFLLwCcHMv6T8chkMry8vLhec52V61fyxdtfoGvS0e/hfuQGSdmU3m1jkAuOVHYWETUvigfOP4BznG3uVgdWs+6BdShSIyk7V0bNDSmIXWy6CEBIeQjxN3TkbLkm1bA5VWJ3Fuyftp/s6SvRhSeQE/M3AL9Vvc2HZ3eyQVfA7akvUB0zghtfHMO9/SxeA8cxUcF/uudPVIBsjCsen97P6O9HU3q6lBU3VzD/j/mYu4J9MpRYLPB0ydME9gu09md40/3We5lFeyfrsdo/OZzqS3hJOA+se4DW6wUYXLwwObpJNft67DMiAuELBvJM2TP4JfnRZmijwe0kLepb3Opzi6KkYOoPXcdUWUebbxT00KPic+MZfSwEi0nA6BOERaG2OgXlopIr/a+gDFDyofOH1NyowVmhwVWXiEN7Jp7ZT/HT+h/tsth0Mgnc49PgQ9QlHbrSOqvjVLQo6XDMB6QgkjBtBiaXO9HAMlGGwiJHFASadTaHi/liG4ovQVV/lPdpYHN7NS+X5BBY8CEnj4VQVPglMlHOkt+W0O/MTXKe+ZGRb8qYesAWAFMrpL4XRIG2ijYuPXiJrO0ZVHtJmUROxjDSTMPQ7NJQlV6FQ6sDMlGGDAWq1rNEX9tB3rmbXP/5OmcOnyG3MZcD0YncDnsaTUd/otvuJ6h/EIkLElE4KLhYfYpjyeG0uVxHaVKCwYQ2NIFcSzQvvwxlFQKjnKDW0x3dzPk0BfVGcHVHUKnpGaVMykxizvaRiC2tUvBdBpfqjnMr4hFMymaOTDiCy8c2vah7XgAMzDvA/WEfU5NRQ8GhAkRRpLK9kvMJUga6WWlG4aLmxoYbbP1zK/+U7oweoUuv7zC3W8e0MhTOzBZ47PZj7FhuA7kJmBEw0apposNNQcexevxqbTUYlRY35HI5zV7NdLqJ3F5/m7Ssp/BuG33H89WeaitFVJhjMj7tY3EySTaQQ6EDZWfK2LlUYn8YYHiJ5LLvMbvEo06KZeq3U63MHg5yG+OJWicDuZyIZRE0ezXbPU/nrKPZWyR/dxZT3vUguiCax5LexMHiiVadywnDp1R5bMFY34pvi/8d7T0/9DwnpwZTsvc2LtruEL80Fiq5ikOTD1G9ppqWkhaytmfxbsqHUptM/oQ59EUzVsPtfkEYnTSoVDZqKLmoos79IHvLNtK3eDOTrjczzH+y9VryrWR8LkmsFx+e+ZCVh1eS03ZVsiFlcGzceRwTo/CvNEuZeF3S5HaWo53w5cqnMG/8nf5eE2xj30Xz6dLhwuRDE6k/cgMHg/2aTfFPYd6UeQT1D+LU26fQNekQRZHwOqlmtcKswNBsYG3aWvatkIo0ddu+zW7n+KXmKamH/lFmoVty9+aytt9asrZlcfSloxjaDNZrJ948wc6lO9nz0B42T5ayUQOVNr2kpKKAmEkxvNL2CrkJNjrfKzEzOdfvPXIHLyN0ZAQufi7kqHKs158Y+AQ+/X0YsGIAkWMiebn1ZXLH2H4PtqBi3qClHJsZwplKKeMxoDqAu7Y9SUd2GVrvCHQaiWzQYukJ4BDQ9AqiLjwNs8pZyuQQQVQoMbp4YlE5Indxotr7oN0z5V2AEEFuwSPGh2Z3CRRllks1pAyGdlpkn9PpFcLAb5dhcJUy/3KCpZrOaVfTMP1qImRwCCO/HMmjWY/yedEyul2G/1lmH9izPAAMenYIbd4R1Pvp0bpqER2V5A69j/LEieh09mPqqu9lZa8AKcDQfdZoWjS4N3ViqZfOD4VZj7u+N2MzC5ii30CfG3144Ps0ouam0tJnFAC+7eOZG3k/n034jDEfj2Hu7bk4eTp1tVO6r0+DDw+tewiPc854RnniFiSVzxjitpj+Bbswqo2URtbjHuFlZ+NbM/uQfEEnXjrBiDO20gYArS5XqfD6jXZzDd5l1+kQvqDJ7SyeYjTDsi8RL5+Cp3YossZGCrdfx0HXiigz0uIq6ZI+pihURjkuCSG0verCD4/9YNWJHZWOVAVV8fNrP5N8dzL1H9aTfEsCSgU2zUfb5IpPgw8ReyLwivHiibwn6DejH84qZ6sNrXJRcd/p+/CeKe3zoQ1SLWGnTifqbtbhHe9N3/v62mh3u2TfjH08V/Mc036YRr8H+/HD1R9YtnsZR4qP8E8Z9+E47tp7FyCxuZx47QReXl7IZDLU7mpGvTmKmMlSLbIpX09h3pZ5dr/3jPK0gu7/M9G7+dLxyEoGPyMBU3VNOob7juCU2fQf6mfdcgk4ZTSROrUvcoWc5LuTCRsWxqXKC5xIjuF2+JP0zuzNoPcHcWjlIa5/dBCDooozSX1pc87getQSmgN74ZoWz+UfLvNNzDc0ZNtqnImCyPHXj5P5R6btsx7LxGywcOv3W6yOXM33SRLIzLdzJADDzk/h+vrr5H59mOgrf97x2//y3zIZUVPiGPvBWNTuagRR4HbYU2SGP8G+qPcp6KUGpcK6pt07U4mrfJsK4Qrn4laCTEbZ8QLkXZla3dm09R31bPtgG7/O/xXBIiCKIgcLDlDhtbkrcw9ubblF5LUdKA1S5r3QTVsvKkCUMX/HfJw6DcgtRuRyiSkqpeQXMkYMxHXpdFznTKAwzZ6eO6imL1MPPUT94eus6bOGjF8zAPsatX9P/ZvqQRbm9bKfSwDZmuNc75tLza5LGM5JVKmtTtfJc9zE2cJrFMQU4H6X+x31+gBMnSYq0ytpq2xjx107CNsfZr1mkXeg0rdzfd01XFsqCS7rzpDsAWCRK8nbn0/8+V9wbKuT9rduXburLvzsjbOtPp3uBILy0HL2zLmOMTQatyA3HFwcEASYlNHIhJvVBLmUEq2UEyubSObOTE69fYoJ4bYzWm6RM/Xlqex/bD+d9Z2oDZKO263riiqR0W+NJqBvAJu3bybpsSSWP7vcyi4B8OxzzzFi6lR+RILxRSEF99Z3/T8aCYp4ozidC843EZSSDiAIcOKtE/zY70eMHf+o9ft/iXTbtv8qu/n/hPw72Pdv+V8hZ94/Q/6mfORyudU4Nfg+hNc3L+IQ6IVKJ6EIRSwYlVJWmlxUg0xKbW/X2JBCFrlOgp3JZChd1LiEu3DP/nsYsdGGzXZy+ZWykc/TUtJC1RUpqyDYoRdRNStRyRxxrlpH7xNfc/iZv6m6Kl1PK9zBmFv5OOujWLLlLjT787m99Ta77t1FZ10n7mp3BFHg032f8smTn1B1tYrZG2aT9nCaFQ0vE+V0eIYy5r0xhI8KlzJyut5XlInsmrULxaA0KmNGsb04jRUrILPzCM2uF/Bu8Gb07mzasqvou34SeXESuuyqwxcAdHoEUdlrPIEjYlj/5npuPHODv8bfsr6z856f6HPkU0TRxgXfLFTQ6pSBRd6Je6s7To0N3NpyC3WzdEC06CQj9Zv0b7h+z3Xcn3WnMa+Rz3w+o/R7CcWf7JeMr7Mv54edJ2RuCNpaLU2FTVgskuKgECSDUueu4+79d9NnaZ87qEsF0cK8LfOY+p1EZXC1fS+lft+RH/opOeHf4No3lmXHlhE+MtzOcAiqDGLJ1s/xqLEpij3vXR1UTVFMM1df2Gp1onbL0aKjbL29FUuYhaSFSThqHBEEEb26HK1zGVkpLkRMjOPS15e48uOVrnZK944simTalqvsXrKdbQu3UXtTmpNWagVXb2oXP22lAgDQPB7JoYmH0LppOTT5EPoYb2asncGIr0fw8ksv8bfF+B9mDwwCDltAcQR8i3wZeqKN8rM2o7S7iPz4qPH8vul3Xm57mZAhIUz/YTpPFz9NXNsK63cfTX6UjroO9M16dt6zE3Wihu8e/wiH5tP0OrMWS0ExEz+fyNgPJarXcMUgwhoeIKBSwK/06h21u5yMksPF0RSEKML7Du/zgdMHXPvpGh33+JKZlIlRrCIyOxNzZTphw8LwT5acI46iJ5qONFx5jPzegXSczeB9h/epv13/L4O6/5G4+LkQMjhEqtk4NNRao9N6vV88TxU8Za2hozKqGHp+PJoWDWH1jyAIEJASgF9vyREl9EDD61xldNa0cX3ddWb3ncNz773NMJmMUQo5vwFHgc3AGJSMQM5dj3xIVLRUX6QicRLi7DkIFoELqy5QcLDgDmeGKBPx6+3H7F9nEzw0AoDofE8e++ExDDcNqN3UVkMsTjYZT+1QQopqaD/+N4Z2AzU3auhs7PyXxkD+gXzOfmQ/72sD5TQEJyGKcObDM1z6RjKTqjy3kRv8OpXem9E5lBI0JJyVlSvps6SPtc1GByO3E2/jleTFyTdPcuzlY6wftt5KvekW6MaczXNIWnAn+hMkPvyCQwW0VbRRfb2ajzUf07j3itVhbbDoUaqVLPxrIQOflJCCVzxf4mRSEorGbzFeuMrlkx3U3qzli49N1ndWmnREX9+BcP0mHhEeaNHy8ksvccgi/LfWlFuVG/6V4DE6laDZA7t70PrdNk0bPqN8mPLtFJbmLMXjEw9qHE9arwtYmP7jdJxfcmbp9gd44Kst1Hrsw02XhH/nM0Q+NJ4Gf3sEaE9q3Z7bodEkYnCQzpyDUw6y+5EabmapCGyXaj1O930CvaoaA+3E5cbhoOsk/PFULid/bHd/hUzJhuUbyJoSKDlpZTbnmN6hnAcOjWF15GqKjxWTfFcyZa02oIpCcEQmyNDFWyjqO5dON38Q5YQ2LcXRKCGQBVFg1IlRpF21D+aaVWauD6hFFET6ncrCv8beEdrnRh+e+/w5cnddxTHfEZVRMpSmXtOTVvstbbk7ic/I5eK9a1j5jVS7cWjOWQbnHWW233PUDJpJbdQQSo/k0lbZhou/C+c/O89rz77GyZKTVF+rpv52vfV51sx55ICMjNQMwpbYDLiSjiyqPLfR5nwdRGjya+KhKw+ROM+WgQpy2t3awd2B85+f59ynEiglwXkEvSo+RVAItHi04hzmQ+iQUNSaLgML+7XeNCmENy1vogn3oKhQjsokBfc1LRqiCrxoSc+nLrOu67fS/JsVP4uVy6T95PbW25h0Jrs9pDNyIiPfGGn3nHzdeUzKFgBGHVJy88cLlPcaj95VqnVTJbtCfuD7VHvswOgbjKnTRPllKbhZr6+i1GctDc5/0u/gx3DwIHW362gqkBxwyaYHiKp5nrPip1yPXIIlNoHyT8upCO1R0wooSi0j7ZnNCEMDSJidQNpDacjlcsLCwihoLuDEnhNo39HSUtzCjB9nkJnc7SQRuRIzi7fXTedzp895R/YOl0dcloIkgMHRID1L48jv03/n8MrDXfNRWk/hpeGkXDHRVtZMY7WBc2ek7IqgJqmWiN5Jj8HBCZWDDJ1DubXNvSK+45kQN4aH/oGgUDJ+60McnLiZuimHaRsHw+QwRondnj9CCcPlMjwmwVMztXxrSOLEEye48KWN4ro7s6/e/QDVhgI8wj1YemQpD1x8gLqOOvxbp/WYq2Y6O2HVKjh7Vvo3gNKsRNOqwawzoNK1oTDq7OgbI4si6XfJA4vRgiZUg8JBQUFrNhfjx9gmhcVMydcSlWVlrwl2c/NW8i32zS6k+mQeFidX9O5+1usRdU/z7NPPMvTzofRZ2gcnLyeGes1i9O3bRFYtILLIk+orVbyreJf9Txy0G4uiqCLOLHNBV9HIgq0TUJqULNo3yW6uinoDXuU5dkGg4Y4wtF8B3z3xAw5RoZTtLcO7wRvZeXA9LnBOhGMmox2A44Sg56xooSr3FTov1JE+MJ1Gfw26YikY3OembT07KBykNXcyCkObgZDPQshKzLKeQ8FNd/Hu/X/wdPHTLD2ylNerX+fLsTn4tk7hWvIf/LnwT3IScnip+SVuTb1FTxl2+RmG/yUSkBrAgq0LcHBxoNPciU4t7a8Lti0gdc11LvtMoSp+DCUl0pw3iYCzDCE+CaOTB5b6RonGDjkD8w4SW/Umns2eRBQHoho1lMduP9blEBLp5QCzXKAjpBpZiIxjrx6j9LSkn3cHYAPKm7kw40PW9l/Lb5N/65pvPWix3LREvjGNpoImovLt6/+6tbnhX1pFzd50xuzOJ6owiqL224R3zCem+hX6VB0mwOUdfBN9aXO2ZXyJMilIhAhubXp0qyuJKrLdu15zmFpDEQqzgg43gefrnqc2agiJFZJN4yRIQYrYvFgS70lkzkaJZcPabFGO0qREppMcLNpqbdc7w6nEPmyeuxjjg4MI6h9E6FBJTw13jSO1SAoIDTofSu5jX6NrsoFkQmWSxiC3yFGYdKi83SiO7kTrqsUkGjiXMIwTvROoObWDmZvz8Zk7EpfXbDXbre8uF3FtN5Lz6yXcTAYJyNp15ivlXcEaUSD/73y2L9xOc05Xv4kKpnu8zNb3t6ILegi1rgX35p4Z2yrygt7m3VvLaXfMQmXxQIFKClKIcnbN3kXOA0HsXLaTXr9I7BZvZy2wzu1Lg6+DUsm4vxVWwGi3vF15P9mhwVg62/DcKKI0uVjHEVFGm3sbv929hfoDV5l8cLLdb2XIcPF1IWpiFEonJaIgopArSC77DlddImlX0zibepbqa9VWB6PBaJt/pc3XOfbqMY69fAxHnaTvqiwaImqfJKHiQ7vgXkd9h92zKy5UUHCoAEOLwZo1JDNo0HRItcXKpl9j1/Jdd9SVNCvaKQ84SItvAM6+rmhrtZy5bEsTGbZ+GN9f/t7uN/+kUqzfmcmu1Hfod/AjnNtstHCJRU8RUtSAvrrJ7vvddalGnxhN8rlT+A2LpSx5OiYnd9u8FgQcdK0oDR3IamtwabcFVrODX6FOnsGiPxbR+1IBgzdP5eJQ6byxKKR+cdW6cv8vd+FZcNk+60Im4mCQgAvycXJEUbTajgAhDcsYnHuC6JqX7NqsltkDCARRsMvsG/vxRPKGLKfT1cBPD/+EODmOTo8gDK4+WCygxBGVyZvkvKcZcmEYFbuvEnvpN1yayrvuI43L+KPjGbuvGudpo+mYMo+CgXfbmo6cTudO6v20KFzUdu/V2QlLU5byyohXcMhw4OLqi5gNNkBVu1s7t/v5MXDsFMoy29iyrpPm5n+AZWQWLn95hsUvLMa3TgquW5DG2qGzGcdblxn0yiAOTTrEP0Vo/5S2wnNE3txj1Ye11ODZMZB7i75iyc/+CDIFc9Jfos03pofzX0bvmT+SPfoVilfvw3GvfXaaWqHGorTQ7taOa4ArPmt8uDBEGmtBZkKGgsLoQs68e0bK0Iv1ttLorXp/Fe/K32XTd5vs72n249dZv/Kx5WN2D99NTca/JsEURRGZTEb/R/sTPTGaQ3mHCC8J58qlK//y+90yefVkBj8zmLCwMOTy/54LeuGOhUxfMx1Du4GDzx4ka7tEs1p4uJCP3D6ylteQ2mX73b2n7uWxo4/x6GOfMFn+n4NwJyrAeYALv9/9O0XHbEB1k8W2B5WFlZEzL4eG7AYKNl/CqLRfu5W9xuM9fwyhQ0OluvP+PdaFDM58cIYLX17gU59POfXeKbu2mnRm/rr7L1pKWnAPkZIX1CppX0+7HMWVH64QvngIFYkT7d7TobMZ7/IMGq+WEJxzDJfmcrvrgXkn6XvgQ9xCPRnx6gjUbmrMPRbHjeQrXB5upmjNERq3SUAEZ2MEQU13YcFEg+oAfsWXOLT0N2QtEuClW/fbmrWVP//4k9IdpZScKGHV5FU8/u0DZEQt5XbYE9Toi7l66ireVZkE5p/Cq/IWlq4AuVxUIkPOztk7KYvxo9M9EJkM/k7RcDl2GqgH4hwQiSo8iHYfyS/TDWIMqk0lIvsGxqZ2Rr09itChoTx36Dk65LaSDFf7X0Whg/b17VY7qlvq/Os4MukSnSW1tOVc5WzCIDLDniQjahnu2gGce+Icy9YtY2XFSrusPoCmgiap5ND668zbMo+hL9hKEwgyI1qvMJ6qeIGb457h7FgpsKUX2zDLJV+pqNNTebEcp/Z6ZKIg7ekyAZkgQ9n8K8ayLNyD3fGK9uoaR+mdtW5aiqPrMCvVNOY1om/V27I3A9/jVnwZ72y7C9+91cQ9F8f3K77nYIUNBCLKRNq823ANcOWxwseY8PUEPpvwGU6qLuBDNyvU95cpW1zGhCMTSPw2kcxyW3C6pqaG/fv24QqcRiqhbpd40PW5kyBw8O/n0etrrHPRrDNjaDXcUTf5/xbptm3/u3ve/9vybxrPf8v/Cpm/bT4NTQ3SZt+FKnExxKLEkfL3fyK2U0XWyEcxKVpodbkGQFJmMhviPkTTb74t3xiwyDtx1IGjth6flGBSN49mxW8r8Gj2YPoVafcUEVGoZGxftBZNmIYHLjxAlGN/elWksT9ATpLeAceOJK7+cJmIUWEEpQXhZJKcnErBjWPjjjHA7TmqrlRxY8MNRr0pocwaOxv5cv+XrFizglL/Uka+LjnhxKPSRqt1zKHaYycHf1Fz+aHLzPt9Ho5Dw6WGyyCjbwYL1WHc6vIfGI2gFyTjReK7l6HSOPPTX/1J0PzGFbdZTOn8DQEzDe7HEGUWDKbx+Dn74efiR02Jzfi4EvEnIepBnH10A9fbr8NCyDLvg6R9KE1KVq5aSWNkA2Pyx/DOJqlQujJvDrGCKy3Of7BRtZG06DQcPRxJeySNtt7tUAP1tUrcvdwxdAX3jr16jIz1GcheMoMS5IITar0an6s+uA5yxSfeh8KCFmu7lmxegjLkBqEvL7R+1u3MVBlVaJoUmNp0NOl0TNg0gbt7380jNQKnzB+T1nCb2Cwttb8cYMtJFQu2Lbgj2w6ZSJ83Z5E6WgoIao1a9GY9j//9OO4H3ens18m9794LwNFn9vH2r2/z9ZNfs3XOC/w06Xl+7fs1ao2a/o/0tx6YHS4dVIW5EyuIZG3Lpt9D/fDH/46AS3fB+b4P9EW52I0LWpsTUES617qffmKMSsWg/yIXfhAwVAHpNVXsmLuD3r2n0VzcTPHxYvSBkpGgkClQqpV3UEukdj5PyJXnAUjUXOTzIZ+z7NgyFu1cxKWGGqgEUabFIpZgMeuJn2mjwOx+p5I+MylNnkZLUTN7Xj9Cn6V96DW3F3HVb5EZ/AgP/jgHZYgN9Vd2pgzDaHe2x2xHYVbw/YrvGW74HcEi1VPoVgpUgjsq5TJaI0Admk1kmBm1xlbvw7/Gn7HHx4IDMAYufnWRK2uuINy1BJCoSAWLgMVoQaFSIFPIkMlk5Hp9QYnHBgbnHUMU7YvmqsvlJGdGE9y0Dk1dPm2zGjBFaZDJZXYBeNd2V9QdRnz6R/BS80sonZSEmcN4ufYVLlxSUHDJFSMqgkO86OXSj9WBowieO4s3Nv6MQnDGt3UyBkGNYHLg8MrD9LmnD8Jkm3Gx+PfFnGs5x5xVkhPrcsU1ziU8iWNbMaIQx13Bd1GfXY9MJsMnwQdBgMKAz0g7MRz9NSVVl3uxcdxGpv0wjcRl/TEpWqh3P0JyRgjtlzTMeG8sQ58filuQG40/h/FN+f34tUzD1WU+Mjlc+eEKbsFuDHpyEHlBEoXgnL/m4GKutjOmu9djp0sn2xZu4/u7vqfpchMtxS1ET462Zqmo3dX0WdLH+rst07fgFePF5K8kx1DVlSp+m/IbM9bNIHJMJKn3p5Ln24EoszD6xGjqvtiIdtRj9Jpjm38dynK0TllEZTdjKrzIzcWPUzRpEOhtc9Ps4ExB2gLiUr1pKWnhhzXf/4/W1GHfwxQNS+ODsYMxOpZz9dgCNJ39ia98j9zgN3BrcwMlDHx8IHUf1oHJ5iRQ69U4F7bjMtuFG9obbL60HqLWW5+hsnjSeCGfIceaqJ3uR51/VzBHZkFTm4dv2VXqz/QlJKuM5sBeGE2Bdm0UEfhxyWkGO8ewcxZcat1Dfcq3AMzYOwOLYzmab4dTe8i216uNAcyMfohjZY04KhLI3JqFa4sOunx7SosHN9rP4T/chfmT5gNY6TwAHE3B9LvWj6B9avIGOWN2dEUQJCerX8s0vOp1OEaa2Xzfn3TI7B1vXTMGfW0rPtVtOOrtEcNS8fVmfI41EnY+jLFvjWXooOlcviI5DM8OP0tZtIanVEPJKL+MwuIqZf1hxfDg0lzOkfu28cdzjVzxv0KfUX04pTqFIlvB+D3jSfJL4ubmm+x7dB+hH6ZKvxUVyIBLgy8R9WAUa1LXoHBQcO6hWq5Fv4FX+whUJhXLP1pOemU6szfMprPZwuWYOXQ6FnB15BwG9ptMxup9iBaRYS8OQxRBZfHAt3kCXi1BuM4MYf6rcXajBxKKetnGZTgkV8FrcPgw5JzoxyQacE77i8KCD5m+L4WED+5h7qvRADQ1S2suzLkX0xdPZ9TUUeib9Th6OGJp01qf0BLWm4GPqzn32TnUbmr6P9rfClYAcDDKcIxwp9ZVooKTy+GC+g1KgiVDsV+5hS0zfqfB4oXjEw8QODSXWxGP4GBwIKLXowSHqKQaqQEW5N/JEcTnkKFA06IhtjACMawFk8XmAE0u/QF3hQZV7N1MH1DApR61g3dl72L1udW0Wdoojizm1pO3CEz7x3yXCchQ0OncaU1qMlYZ6es5jMvaU8gEGQqLAovZwqTPJuDs49w146S1fiPlBoLrFCYvGsH6knFcuAQzBUguXUOV15+4trvi1FJJ2O7NTE+ZzN6ZewGwxDyBwqwkq+p3TEYPNL0m03G16wwaARmz4OcT8MhJcG7xxkkUqRnUzqz7Z/KksIOhSiV+h4dTP76D4S8P5+bmm7gFuTHD9W2ul2dT47mT67rdbBzvi3+KP5O+mMTsP2ZzJm639d0topmNG+H4cem/3ndLHVAYU8iXz33JHwnvkvLp5zQGJXPBYy63w58gy+U7xh4bS9plbzrKGmkq1OMe4m4XEB57bCx9bkvUc+rUXghyJd7akfQu/Rb3lOOc5y9U8laad1XhQhX0mmOdu4HNc3hsQIpUtGNJ1xh1sZiVRxh59613eTHxTRwvJ7L9XCB9ymzOIoOjAa036Ku0hJYFIhfktBhsDrSE7AQUJ38lpraGSlUSjT6dDDe/xaaIFwhVwcY2AVN9C/ue2Efq2FROnjjGYfE/plQdBBwUjAw72sCllQ04OL1BWq8IquXBXLYYACmzRdYZgG+9L2nb0qibVIfLZBfqdtTh3WYLCMocHPCIkBynDjjgIgeFCO0aE9WR2eCpwNHDkcTgRGp76Nkqo4C6pdYuONEzqFYUVYSng31ty/D6FXydPgxd7DMUunoxwLgdvj6Db8J4amKG4dc2CZXFg7Mj3uXawEre9nzX+ltBFFnoCm97w2rT9wxzG8D+j/aj1qgJHxluHQujgwOtmnBmPhaMW7AbMpkMU4/zcfStHNz6B3Hv2XtZtX2VXfuCqoJIuZBNu0qBIJdsEYCklhcQa6VME0GQao07NTmhd5f00UtxE9Gqsulzow/DDlcim+ZGTkyO3b0r9XlMOTCF/ldFZO/JEZQO1jYLZhU4wOiTozl97jRpiyVH+pW2fZT4ltPhmI/SrMRnqxQUDBsugTj0sibanSUjyigY7J7X08FfHt5IWl0Spyed5u3n38Z5gDPm0y/wdocfsbl+jNizC93gBRybUkOtTy3Fbbk0u54HoNG7kdz4GiJcnSkz2vR6985U2pwzcG13xaCWMWnXw3y7RY3YIVrfS6VQEV4SjkutC9FPRNPyYgsP3VoueWxkFhQ2ZnTiLm7EvaQ3hfdLLCMeHQNocjslvYvMQnrMdO462kRGkq0NnWaoPd+Id6PkBLWIZk717o1P6wQGFOxBObiDCyMN1Prb1yx6PGA9vVbCH1+9j/yvy8hfcQA6OB8/ChchCJODieLIInynpFHi+Jfdb7Masvgz80+iH49m5Qcr7a7JRDl1fnUELA5g3tvz8ImXxktvsK0LZYfcCkzzesiLquAqHA2h9C7/GoA+S6R6u1d/vMqAJwZYdU6ApUeW8k9paLCNs9N0b3x6+ZC9KxvXdleJtaBLBLmBw6k+CEIHex7Yg+6IDl613Ue/Xc93D3zHzPUz6ajtwLHOEdTSOLvpknD090auUiCYLPhV2YI1ZvdJXJvcm7uHKei75gPafaMoGHAXl1r2YFI14tPgg1tTI6YescPumqkqQzt9jn1FQ0gK6iM3GJLai52zJb9HYaAE6nLR+6MyyTH2CD5aZFLQWuek48j4EyRXjiZvzXFU+gGYHN0QEVGalYSXhSOUCVz/+Tq399/GO8SbJp8mXIxRuBilYHxPAJpZtF9DZsGMTGabpC2NrbQrszB0AbEtgoDcbESUyREEJRPcnkZ79Gk8qrMJy9pKs1sCro0lKMP729W3u510G7lqGMPjo7FkdSDTmhEVki0rl8kpiC2gxT+TsXtv4ppXS7NvP5pd0jlc3ICzfwqyPBm/z/gdi8FC3/v7YukK5IiqIHRBK0gbCutT3qPFN5YrWYsZMECaIx7NHjz5zRQyA65QG1VrrQ/cbaM7t9WiufI3RSMm2IGaumXW7lnoYyrJH3A35/pJPh8HUcoakrs40qkJBGcXlC6OiHKsJQJkyK3UiZr+sVjOZjLGZwknRv2GTJQT6BrIxyM+xqHRAUObgZQpKTTUSNlkHY65yEQlBkcDrQGtyBQyVu9dTZ45j+UDl5OrzKUpqQk1ak6+cxJtRxBS9S1YnrqcCn0FBW4F3PrtFn8t+Yv7Tt9nl13nW+HLrS236DWvFxW6Cow6I/f9eh+ZNzLhCfv3v/zDZYxaI8NeGEbEmAi0tVqK84sJjw6nvaKdv5b8Rcq9KfR7oB85u3MoO1vGmHfH2FH6Apj1Zi59dQmz3kzi/EQreNfZTwqqyc0GlFm5VF/zJbCfTW9ctPg5jh6VMTz7RUYg5wFM+CPVHdugduSESY9lHMT4+hP5ZyQNOQ3svGcnyUuSMd9lDw4v7F/I+nnrybhs4rcdX9wx1jIZRI6JtAKHbZ/LeDz7cQztBo6+eBRXf1e0PfxBMgcVC3csRBOusYKR21ylwOkfi7dz67nL5DV609iFNehefy4tVUTe2E3VsQEEFlzGpHalLTDUavsanTzQeoXb1Q80/wNMLyLQeCEPmUYDXaZ1d3bxnB1T8S9JZ9jHMzjX7A4VWH2wSrmSnXN2onxNSVtFG+2H23G7y41qqjHL29lTv4oDAd8S++BrLPz1Jk7aRmqsdS0VJDe8x10vNrPjs/E4mpR2WbwKs4BFZ0DhZNvHu3XWOj8DmSNX0Ge2G6PvluyHL9/58o6x8MtQcGb3GWTPKe64JsoEUjY9z323omlxtAW0XfUJpHql0tnYiVE02tV+BXALcmP8p+MJGxZG6NBQyQ91qvueZkS5AgcPZ4xO4GzpD0CGuJHQUBUppetQFOWTe/AKxWkL0WkCMJttk2DwKQP6hot01IUjWATcAt3sGF5EBByqSvk2fgsz1s0g4S6JtrXF5SJfNkNHrQbRW4tjoCN1/nVo1Bpqnq+h13e9KGkpYfuK7ax5cQ0Azwx+BoAd2RLTgklvYnXkalpKWnCd7cppl9M46h2ZKrfVilz15ZcgiuwDwpFq9O1GovDUALOAB4F9wHBEyvI+Ia7PKkQRJnw6gQmf2jIN/28TQRCoqKggJCTk/5eA378z+/4t/yskMC0QAiQ00aDiXYzMvIVvYy8M2UW4D0ygISQVsM+GMDppcI/yJvradlKu2bIHBLkOt8YSEi5soPxMCS+8JjkjnVU2FI5bUzm+uWcZ8twQO+SGDBky5Gyfv51zM+/nsaLn7DjDpUbIuTLgCtoET8Z+MJZXtK/Q4dXBA7sf4Pkjz9Pk1cTmFZvpe39f608UOKK0uFHtuZ1W8UUuPXMJ73hvPCI8sPwj00cQBRx3/UlwzrGuf0vv3OTdxMkZ4QRMTwNRJKBlBqs0Igmmu7EotNSpHiX+4u8cXvoT7yrepSm/yc7JcGLsCU5PaLuDPkzqM4GLgy7SEhyAoBSsNBL+LbOJr3oH/65DzSJYcAtyY/qa6TgPl9BJuk4lZxYUsObcGkrfLiV2SizDXhqGXiEFKRWCM65aV3r/3JusHRJiS+wRkAuuCkfZqOvZHOuhH1IRwqLfYqn97SjfPvotZTfLeP7I8yDK8NQOY/2MrSz54msUXu4UHytGtIiYeyC1xh0dx30/puAzJNaqnL154k18P/MlrzGP/lf6U/peKZ/5fkZTQROZv0oGVbeRLAiw5MASKwVQrOMQQEIOXRkVzIzNC3lTeNNaeNpKK2gy4JR73doOR08b/aiL1oXFvy/G4WoZl765xJ/fr2ep/r/mmwd4xAyaAiN1fW+hCnSh6nIVex/ciyJXQZJvEhEeEXQ2dFJ5uRJ9q56CQwXc/O2mXTF1/xR/+j/WH02YhoTZCahjpXHMi8/jy5UfIYuPsHtmlfkWJ5ISONQ/nBqfs+ia9eTuzbVmegQ33sOEG42oTSGIwBN5T7D06FImfDaBYHUvomqew6K0UOdfh9FRxYcuH/L7TImXPVPzCbfCnqDdUZoX6tRezNsyD02ohhivGK4/cp3BHoOJz4tH/qn0EjK5DIVKgdrJhiC6uD6bD50/JPuvbD5y+4jfpvzGraDnaXe+RX7ge3dQ0lRuu0C7VzitvjFoPUOROyj5OuZrNozd0DWO0lgt+W0Jow+0IldJjj2lWimtKTcwjTdT9VobKdMz+O7ZP3hy+VOsOLyCYm0WNyMe5HrU3TRaJlB16hPkKjn3n7ufkW+MtHPCBlYH0lrQSuXlSnYt30XZxTKaXc9THVTNqdGnUIQo+HPOn2xbsA0Ak2AAmYVds3fR8WwfPCI9GPPeGIIGSFmVl2Nmcy16IYF5B2k+fBkXPxe8Y72RK+TW53Yr+YIA95+/n4U7pCB7t8EdVhaGplFHW2UbNzffpKmw6Q5EtEWwkP1XNma9mSHPDrlDYe6WpoIma+F3AL/efkxfO52w4WF4Rnky6+dZ6JIkg0/rqkUZ7INcZa+2aJUSCnPz3d+gWrKQnktFqvcBgkJFS2Ai+PmxOnI1G1b9z9ZUwG0/fFunIIrwY8nzVHttJy/obWKrX8fJEMbSTUu5eLdEE9hhks4To0JCQXo2exKyqYHb227boaS7Rd3ZSearWwgqN+Lc2ZNK04KDrhX3ugK0+dUEFF3Aua0Wc4/M2ZDyECJuO+FZnU10vkTVq+hBGbJv+j4KesdYa48AOBpD6FXxKQ/EvEFKwQcE1Aby1+JtBJZKVE+e2qH0K/oDs8rMicknyNqaxdq0tVa6D+keYTR6N9I+1GQtCG+yWKjR7MG/dRpT9qYi23qTuqB6Gnwl58OisGfxb56Jwqxgzp9xGBq1/H13b7s6bgC5Cbmsf2A9zov9Gf/peHY8toNlCZL3QNZFJ10T3EHUIxN49suNTLnejkendP58VbqMxqp7cGpKZ/gXs9hk3kTn+U7MhWbqfev5Ov1rZv4xk725e3HxdyF8RDgyt24jWI6VpkoUCB4YTNCAIOv5KhOVCHKBrH5ZeCd4WzPoaj2kYFBM9St4KcJYengpS49KzkWtpQmQ0T9vHXdtiKTklxN273pX6w3C6iW6FrVBjUNNJ0XHivhutW2eyGRySiJKODTlJupQKbuptRVOnpLW44Xz0j7nFuiGb6Kv3VoGKHfeRaepk/Rv0sn4JUN6vx4b3rHpCkausSHku9+1W3x3rkXX0ElN5GCuX7edvUa1kT8WfU1zihvDXhrG7+G/88KRF8hVbsUi1xFUFcT0vSOQXb2Mx04PNC2So8ijYyC5Xt8xvBwSSkApU7Pj7h1smrCJ4pZiTlae5GbdTVo9WqntWwsibF+83UoH2+R2GoOyjtKIUrzvk87hhX8tJNJbyhYOqQjh9Q9ex3wyk74P9CN0QkLXmHYBqty0NPmCqceaMJlA8rLBmBNjiD/+K2afABq8G63fmeECVe1qwnb0I6A0n7aWFjq7KN2cZRDgAbNHOeE9I4yhk28g/8Id4wQTO8p38FEz3C73Rvf3aFwPuxIyOIQ9D+zh4qqLyFGhMksoXqNFqml68UtpL+mu3QrgqR2CKMqo7eF//6euJsigNnIQrX4xdHZCnUrCsl8adIk/lxRQfy6Pb2K+oTGv0U73c29zx7HTQNznD4FKSWDBGdx1fYiof5z+HlO7JwXeaeGYXD1Q6dutDh8Zcg4/f5jven2H0O08/WcWuStcCltAY2gqu3bZzhKFWYFCa6bPC5P4ZUU+RrU9xY9fnR+y2hoqkhPJ7J2JXHSgtl7gk2b4ssCD/lfSEEwm5v42l2xjFsMU/3ntRLABOAZkLsNJHIT/0ok0RQeA0o/J19qZfkXk6o5RVAZXcuTJI8ROjbVSt8l64GPNOhMtJS2Unikl49cMzNp/6KgGgerr1bi2u9p9nptYzfXFs2mvbmffin1UNFTQabYBIi4NvkTR+BBCbh/Cv1DKEPbSDmOBm8hehTuPfPQIAaWlWIYOQ+sZ3OPOXTUzEbDkFZL+bTqCWUAQRbZr4cn8oQjfubF/3H6eyH3CSp/dPRYHR8xnz9xbjH57tPVa93urTN64dUZTsz2d9rp2Wj1a7d6pzdOTgrQFBC8fz9E5XhRHSQevuYtWy6viJnF/vMvX0V8z708brZbWKRuAZs9myiNdsIx2pMWzxe7eCpmS8tByKmJFbl65yem4EM4mDrD7zuGJh/FJ9eHWFimAd7DpWzLDH6PV+QpGByOlL5ey5OASa9DnTxeJaq/3rd7I/rrKD31+YHWkrSafg8UbTUc/asK88L9nPM4eztzb715eGPYCtyw7MKrqafZspjIyHIW3B75tEljpaMUu6z0KYwo5OOMmxvpWjpx73fq5syGSFMcZJOQkMuxoHboGLbXyPKo9/7LL7EvNSMXnBx/cQ93Z5r+NKqV0PhtUtQgCpH+bTq/TP9IUnMzlNBs7Q2DLXGsQ61r0Auo89pPRaAv0OXU6oWwzkj/6QX65/xf7vhZcSDy7Be3RS+Ql6e4YizmuMMwJCqc7EP/H3aSWbKZf4R/o1eU0Ol0CUbIVgx+aQm6yLejhrHLmYsVFnv3mWfYm7eXyD1L9YlEUMShrUQruNAY6kvxpsjXQB6DGFlzQumgZ/8l4Br86mGbPrgwTmZkiv68oCPiUDmMHZr2ZzD8yacix0ef9R9JgKqPFVdpnXZ4KwjvOm21ztxFYbQ8uicmP4Y33XiRj/TV6ze0F8+3vY8GCTCGj7lYdf875k+BsaU2mFm2ib/FmgocOZMJBqQRCWJFN56zy2UG192FEuYKWgAS0nhJYuNEkjfP2Bdu5OP1uSvdnEnV1Oyp9u3VfFZRqaqIG0+oXg3nkGEojpf1Do5L6zl0MRTdkNWWpD5G9+jIx+RJFonf7aIIb78HgaODCkHREZJRsPGOl2EMmoHPS4TRnAJHjImkubqZoVxFOOicUsn8wT/TY4/1VsXbXBNFiZ1P+nPQ9Y3YdxqvJi2FnhyHerCL10CdEXdtuDWACaL3DyRl6H/4PTef61NdoDYi3C/bl9MohJ8UZ3al03Letx6M2jzanG5xO7Ms+9RJkohxEGQW/Xca5UqKCzg16nScuTOPn6z9z9IejWAwWFmxbgMpZRZxXAv0Kt5JU9pX1nWojBtHiH09+Plxp30t63FRMKhPFUbUMfHkMpS9XERQVTsYjGaS2v0pm6NPkR12mfuoy9ub03I9t8sfiP2gcFUyrfyxytQT8Pip/gf39HLgedoy8wcsweQXQmFmN3GSwZdWLcsTiEnxKr2Jq7QABRp6IY/oVkZeNFtbNXMdSzVLaFrZx7adrDA8bzkvDpKxLrVMOzoYIHkp6lHl+88j4JYOWmS0c3nGYD3/MJwMD2xdsRx4p59Tbp9BekxA6ja6neP/0+2T6ZDL6rdH4JPjg7OOMTCGdLwOCpL035VoKfy35iy8CvmDnszuxKCz8PeVvOkbdCe7L+CWDy99Kaz79m3TWD1lPTWGNlDmqN9OY32jNuC08XMiFzy/YgWFOvn2SG79nsXqtE0O3PcP4j8YDEDwwmHsO3UPUBGl+K42dOB/caaXKvPX7LfL/zsdigajolYweX0FV3Js859qLZe5BrO7Vi9Gvv4bHa94wFApiC/jg9Q9IWZaCZ5Qnzj7OVhpe29wWcPJyQu3rbqcPeLeNISjnOPW/Hf6XcwDAJ8GH4AHBLD+xnLSH0+z8DjKFnF5ze9mxDmV6SIH76iCpzn3PNdf9t9YrjPwBdxF51xBujX6CxuA+dtcbQ1PJH7SEnO2Z/DzkZ3TNOjv7IPV6KsOPCPRduwL/1yTKeL2qigb3owDkxRdQEzWYmIV9UbhK9mnfot9pfLGRpX2WIigEBIVA6r2p7P9xv5VRDGTIZFLpA0EhkDv0Pkp7T6WoxJbZF6ydwYK4ZRLrE5Yu35zUtpSTv9G86idaVm+gz1EJXNTtCy0L3EJ+5AEUrk4IAlRX2/olqFGyZR786UHkJpHhB4ZjVNXZjYNvnS+zdo5Cm1eJq3OI3TWZKKfwSCGrQlaxafwm9C32fgJnH2eGvTDMygTw49Ufe/xWidxiovZmLa6NpYTVTKR/8yfSeHSVyjF7+5P89Bj0XdTJ3ZTHokzkj0V/oB0cy9Z5W/kx9ceucZTeOSY/hsdXT0BZU8HwV4fTqAxg0SK4Ej2XFtd0nmuANx/9ntZFAzC0GXDucKZV30rqmlSeGCDZz4IoIJgFrv9yndIzkt2t6AJlCAi0lLRI7+EmoyC2gMzkTDt/y7rvvmM0cB4p2HcWeAb4oev/Z7o+Pw+MBKqqNvJvkUQURZqa7vSV/Z+Sfwf7/i3/K8RitFipPVwMMTS6ncRg+Ia2bzfhkhRBXZREh2gN9olyOj1DGbfuLpoCk3AT7kItSlk+FnknnZpASpOmoNNBePZtPJo9qNJWolOVU6PZg874LUFZx4ieGC0p+Eg1KHQOZVJRbbmIoJDj7OdqRR8V+a0mO/gVOtWS81lAQKFS4ODiQLOhmfUZ69mTuweTg4nK4EqQwZfBX3LqvVNMFX5g8vU2ompXSg5fEaZ+O5WQwSES/YleUqaf+/JtLDv2I6+pxLFdMmh6HtwiZhrP5dJ//7t4Vd6UirR3XXbudCa0PBSVryMO4xyIWBvBhIPBJFR8hFLsRtcL9PpyBtsWbrPrf0EhcHDKQVoC/GhKb7LSqHQX6pbLFDz848NoH7QhIvXGrlo0ogKDAVrLW+mo6SBxfiLjPx6PqyIEF30cDmZv2tzbyJ2Ty9kPz3L247PIkKM2+eOijyVr7Ae43L2cL4K+4JeRv9i9c7NnM5cHVSFTKhA3ing1dae+S8idm5g45tREx72hvKZ7DQdXBzsnV7tbOw2+HdAjtbznZrx14Vb8H/LHN9EXhVrB3VdWsHHpRqqCqrj3l3u59NlpvOO8ran+nsogvNvG2s1FmUxm5Wm20goaO/E6vYfei3vzTNkzTPxsorVdKpOK2PxY5A3tXFp9iaaGZgL474k/oDHB1TBJMQobEcbdf9/NvMXzyHwsk14+vRj5wEjWDVxH+flyLnx+gb8f+5sOdQFadT4d6gKEFIGp3061vlPPGgUgGV7rBq3jt6kS1dNp41d0OOWi7tShbq/ALyWQN81vMuxFKVPEJG9DlMPt0Y/RMmAC3rHeRI2LwjXAlSjHNBIrPsfLmIhzhzOi0Uji/ETCRkgI7GKXPyn1+w6T7jvCL3+GscRG0+CkciI1IJWwkWHc6HMDWYsMU6eJQU8NYsWtFTgFeFi/q/L1oM89fdCEaeg1p5f1/gBGZeMdwT7PAbG0+oRQGuNMxsjBOAV6kLQwidipsdY+AClLpCjOAWObnurr1eiadHdQ0CITuPrNeU68KTn6e1JoJN1OIjRdhlwhJ3RoKN5x3nbredXKVYzaMIqW4hZubLyBruLOANXgZwZbKS1PCR/SqS6mxbMFvY8cz0hPRr4+kqC0IAQBK+L7j7u24bN0MromHW0VbRi1RnwfrOCx7x4jsLgcz7x3MRvMaEI1uAdLwV7kUrtWP7Oa85OCqbtRw86lOyk7U2Zts1qvZur+qeTtyGPelnnW2ovdoq3R8mXwl5x4S+qLJ3KeYOF2W8aue4g7aQ+l2Tl8TF11Q/OSzAS//CBKtZIP3D7my4kH7bIL6/3qwdcXsa4ez6osFCY939szLCEIUBM1hA7B+D9aU/JOOVGFAVT9sJuEjd3Ake59Qk5GagaOgY58n/Q9sXn2jo8WjxbKFrpz4vUTaJ7R8E+pd9uHwtmBwqQYHBzus34uyizURwzg2rQ3CFw4gptjnyYjJZ11WZ9ZvzP63IMMPeBD/oC7WPeQhNCrMRZar+cm5CLKRHIeOEZ0QTRyQU3vsm8Y6l6DWqgk/sKv+JzYzpT180hPkwILSosGZ0OE1F+igKOXIy5+LnaZfTJRTklkCZ2JZuIubsSlqZwOYyeXY2ZxOXYm5b2G4TxhGLJONQqzZEw8EfslAS1ShqpLhwpTux7v9jEMy75EqtudiD+PPoEMe2EYLn4uNOjqSI+ZwWW/FcgEGTKLgMUi8tzaPZyLH86+/jIOpXpysnkTUTlueNTXEjY7kRbPFtQGNZpWDQqL1I6owiiMV41ET4hmyYElqBPcrO8kE+VMOjiJG0/dYMbaGUz7bpotCC4qsSgtFCQWcPzV42T/lW231v1Ls9CevIJbkJt1zRxq+oETyTHkhL5PTdQQ5GoVR18+SnOR5Kx0tYThbIhG56zjmye/R+0cyKbxm1CabIEDOXIafRrJTqyko7CG83/VcM89NhrPHNlffPDJB6ydspbK9EpEQbQDCHne/pWNozdy78l7mb9V8lR2Z/b5t8wgvuptSg/l0OvMWlyay7tqANucGLqQGPxnD6Y5KAmtOh+jxR7FbxEEBj8zmIy+GQDsd1rMxfgxVIRUsGP+YZDJCdwSaD2bBZkJnaqCW7XuLHjzdTQ7WhEtIhazhSNFUua3Sq6yzj9Dm4Hbf97Gv9YG2OoORnou8+SR648QOzWWraXSYu9w6eBGnxvg787PP8PixXDjhi2zT2FWoDCa0RbX4VN2DZWuDZNJxKLQEtS0iBb/aKoShtE8cxkNkeOsz/ywGV7sMLB1wVaqwt04MmI1U/+WAmHFZpDlw7xzYdz/y/2415ba6RGHOmF4m5Z6n3pal0iBkoV/LZTAHUIX5TzSXjf+0/F2ALNu6VOyDieZh53Dp3tuurW50edGHwy1zZQnTaYppMvh0zVHEqt/J6p1I55pkRK9lK+LtX2OFl/6LV3HzQkvoHBzxnD5Fp7VWba+RsaQ80N4+BsJnKbStiATLMRUv0ZCxYcYVfWUt5XTkNPA7vt2017dzqmGbZxMSqLe5VeiCqMQG3qgk0Xb+RlZHMnEnwyU/Z1JWtGfJJbbo7PPjDiD8eU3KE+No96vHrmoQtPRj12FS9l2eT7T90/HXN1I1PwoWnOqefhOLMW/lEfM0HDqOHFXztJy+iYtZz8iIziVYv+vuhopR+esoza2Fl2jjqKxRQy6OMg67yq8NvHoz0NYHbmaHXftYPd9u9mc/TztjlnE5njw4icvwmE9a/utRbtNa/fsnJCXOB94NyXHS7i65irRX0bz5KlFdt8RRAHvypt41OZbP1Nzi1cPd/CL5Xt2lz7Hhtwnud64CZ2+gjKfdVR4/4JHsweRhUGYzqVz4MkDUrBPEPCQwzzfYryiBBLnJ+Id542Tl6TzR1tmWp/Rbbt0S7euIkOBTBQo+/UE59bctquVCtDpaqYlMBGlt4bApgXWz5scblAQ8CmnUmdTHFNP/Jx4bqTc4J9SHlbOlZE+mKLvHCuNypvcxEbKkpw4NPkgcbm2YExKyc+MD59GyqQUai7VcOUHezr9sIYHmZFfzxLnJfw2+TcriKx7/kUXRqM8XUDYiDB8x/gyZsMY1uV+SEbEcuSCE31Kf8RtYC8ey3yM8JESw0qNINUbrgquIntAMro2E0NOuRNclXJn4xFp2HKE8YfGWj9pdj3PU757GOC4l8r4MYjIOBU1kmsx82l0Pw6Aj7NEx1f5RiUqJ5WVMaRbNjU+zkv7X0IUa6mNHMTNlJv/eK6NevCfsnTLq6R+30he3h2XkCFHYTYg6E34t0zHVZdod73EBH/8MoXA80Y+K3iH9Php1Hc5hQE8Wjx48703qdx4jMe1jUy/IjL9ikjHqx28PeptzEozMq2M/P35WIwWOkwdHEkNoNn1PKNyrzI0dChX116l4GABAKODpzA4V7q/QWlk2IvDGPTGIHTO0vlokevICnuWnJCXeOTnRyg5WcKQ54bY+QsAajJqKDpWRN6+PCsFYK2Qbb1uES2EjQhj6i9T78hm1LpqyeqVhWuIB33v74vwoL1+b5lh4bHMx4ifGc+cTXMIHRNKpGUSKouk6wkCGNsNaD1CKEywAUzKPb+gXvUGptYOivvNoyZmRFdbpE0sqGkRzsZwWnJr8aq+jdxsRGc0UKdKx6xyoCJxEs1BvRFHjKQ8QuqPsX6LrSMpk4HCbKT4+5vE5ks6aXTtCyRUdtVelgnUR/Sn/6+PWx3PINHLGi8X0lbexrgPxrG0YSkVIRUoZCq0jjkU+35Ljcduu3NovMZWBiK8boUVONMtfb8azNV+V2l3a2fC0QnIrlXRGNKHdu8ILBZobIRSnx/ZNzCIc/3eRnC0Aa8FAUKa7kFhcWWUy6NoOvujPyXVs/MrvohZoaXNOQNPIYbHblcyd+/LjPx5GZX9u/a1riy5t06+xYsOL+L7hS9x0+OQK+T4OPsS1LwA7wYvYs5/wvktO6lInEhjmHTWNXUFXjtcO9g99xwJi1NRb99P9OEbRDilcMv1S0r8v+ZS4jL0gZH0OrSDZ7/bQHLJGrv3z4/LRxco7bdywcZkIMpN/KG7n1OJKTgU5XB80Vpcm8vpDniIchPX0l8k4tY+Qh+aTK8355M7ZDkA/fpJvgXXAFeGvzocj0gP3pG9Q/IfySgsrkRXv4ybPol+557BMNtAfXY9FRMrJHukVE5VtyntCo/efBSvOSOxyPQ0uB/jjRNv8Oi+R2nRtzDoqUE8fOVhq077+cTPpfk7vJy5v83F2ccZo2hEUAikD0pHNuBOmr6FOxay/ITU7ourpAC70kU6S73jvHm+5nmGvyTVDBz15iieKnoKZ+8uRgaLwKl3TnH4qywuXJTz9QaNtTartR+7GVzUrhT0X0SfZdJefPDpg5z9+KzVvnd0DCA27nUGjM7i8RcruZyVxWuvv47o2jN7SsTB1YH7z93P8JeH22W398rqxfAPh5OzO4f20ibkPUBxMlGJR10+2iu5XPv5Gpsnb6a9up2eoq3RWkta/FO623j1p6tcWdNNhSq1S61Xo2vRc+P1bcRd2GD3fZOjG63+cTgFeWJw9cbi4ITJBOn25cPRN+toLW+V7IOedkutP6ElYNabEEVp7BrcTnArXFrTVwZcpSJxEshkVmpepeCCl5MXzipnvBq9cMpxQrAInLn/TDfmCAezDwgyNC0afKpF9C5eGFy9EWTdfkGlRO9olugvT/ZO5InLY61nc0afM+iSPFCGB9PWRePZ7YuSW+Q0OG9Gm1PB+9G/8NJ0m77a7iSdhRaFBZ2nyLf139p3hCjHudOZhJxIms9lE91mDxyq8djDXy1SVnrFxQprEPqfYjFa+Mz3M3JelwB/iWVf4WDxxqmtlk1D15Bw4VdCbx+y1k/tDvaZPP0IGBmHc2sVCpMewSJj0vUW5KIDOb1yMAZ6kLgwkb4PSVl73bZaoGoiHZ7xGAPDGPfBODYfD+pqr2S3D3eEYY5wXdzI5ecv8+JnL6KwKMhtzKXNIAGq046ncWPTDfbcv4dDqw+RXplupQy3KCxWkKpFe6feUFNTg6mzkwDgI/5zGs+PgEBAbukqryVCwaECrq+/fsd9/y3/Z+R/TbDvo48+YsCAAbi5ueHn58fs2bPJzbUvhKzX63n88cfx9vbG1dWVefPmUVtrr7iVlZUxbdo0nJ2d8fPz44UXXsBstrcQT548Sb9+/VCr1cTExPDrr7/e0Z7vvvuOiIgIHB0dGTRoEOn/3Fn/Lf+vylehX3FhhYROFEUo91nP1eQfaZ0XgVOYjYKvm1YBmUCl1++4RXhTlDafVv94VKKkPFjknRhcvKiPHIixVUfsrWI8m6VA4LleQ7gSO4vjI9ewf74WzyhP673Pt/3OsT6Swefd4I1XdRH1t+swtEvOr3KfdRQGfoxZ2cqyDcvw21lIc3EzlZcrMZt6IPUFGXKDHIvBgiZMg9pd3UNxVnAz5SYNWxus2WAuSnfGZOYx/YqITBGCzMkJVq6kcIDkIOg2BNV6NbG39eiqmqkNCeFyr5U8r5Vx1OlhjIomspKy+OjVjwh9awB/L/nbSlUSU/MyaZ2v0u9qP9IuuGIWbOhqZyTkvFf7CEbevol/qYlbi24RUiGhYeSimnbHLMyijpqAGmSxMpqLmtm+eDvNxySHpkxUoNfDoxmPsminzakx3bSRMZm5+LZNweRgoiKtgtipsWjCNYS7RzPhRg1jMvOsYx47NdZqcHej9Fs8W0gfWorPorHMTZ9LSUQJAEc8F3AtaiHDChx4NdsLs87mPO2p6KQPSmfv7GzOL1/LwWe668nYrjd5NxH0XBD3nroXTagGpxA3iqKL0Dvq8a33pb26nc7GTiv6p5sCyKPZg+RLNWRuzqDkVInddQCzoyt1E+5m1Fuj0IRquuauaH2n9958j+bxkSz5ewn+YX78a9b8O6UWMMqUfP/EczTsKcQt0I3YKbG4BXY5tGUyKoIqqLu7Du84b8Z+MJb5W+ez03cAJ5PjOJEcS8w3MRgtRlpKWvjY42MafpX2WU2LhtTrqQiNzbgFu1mLmHfPv3HHxjF2bzkmvW2uiyKc6B3Lob4edKgLEEU499k5fhrwE61lrdb+cNU68+JnLxKcXsTczXOtin+34tehOodLWyOdl26y56E9dyjRJ0efxLTBhEJto2voGXPz7B3MnE1zCBkcwpxNcxjxqq2Iunf76DsyEQyLPPl71HLO9ErkfPwoRBEmfzXZRrvb1a6LQy6Sk6qi4VoZa/utJXdvrjVz1MHgwITDE/CqLKY5vxGzzszp90/bzb+/5v7FuYcMiKIoOSgswh01+yyChV7zevGK9hUch0tzJT4nnvvW34fxtpH+j/Yn7WEJid/t0HbUOWJute+jnlId1Iw61J9Dzx5iVegqK4rSr96P8NJwovNEBJNAS2kLHXXdSM2eoAIB7+QgFu1cROTYSButrknFwMsDqTlXw6aJm1g3cB1rUtegb5Xmv1wpRxOuwcnTif9Kam/VsnPpToQsKbtGLjhgsUiZm02OQWRVajh+HOgyUFRGFaLBgEtZFtHXtuGgs2UfKIw6Ug99AocPU5E4EYWj9/9oTaktSmIvb8FQUoNzTVcgQm6gzekGBmUt54edJ/zh8H+J2tI76WlKdaDfQ/0wxBruuF4Utp9Bu16hOXIJcdVvkiyf39XHNoVbplLS6arieszjfJP1mvXzTv+l1I5bjFntQosm5457S/1mxlSvQ2VSIcgN5MXN4bshL5JUlkxtWATN8SkETehNg69k1NRrDqB3qEZukbNg4wK8or1YcmAJzqoeWYddIA+Lk4De1QeLQsmyC8nW642hvXFISeDpL1Zw9xYJaTnimIwbkfdhUVrY+OB1QuYPxqe6AZ/mZCKc+lh/61/jz8hTI/Fs9MJkMXH3jru5/9g06jz2ATDlwBQe+TaVgq8PEJReSLOblP3SXYNu09JNXJyQbM2izI/NpySiBE2rBhkyZuydgf5bW9DcFhiTMvsCagJozWi947qzIYr5lYWse2QdA58c2BWYt423X/FFWg+fl+hsqrrqM3Tt6RaVjIrEiTj4ajj3yTkqCipo1bd2nRdyvOSQ0Dofde9BjP16LHkh31DjsYsLcWPYxX1d4wgF72zh6LuSHhRd8wIjbl9DIThz6PAhqg9Ws27QOowdRrvMPUe9I/oGPZ5RnniEe0jt6n5nUY6mJpfSvTdRGjtBMHOt+QQmbAGK1v7j8V82iVKfnziZHMe3uS8CoDQpJUrfq7a6dj2l3b2dvLhiTGmDufX2LaqCJO9Ssf8qOlSljHY30xRbhCrAm/l/zsfzB08OFBwAJHq+Xlm9GPv4WOqz6nnD9AYnxtiyIuWiA751vjRsaqA+q57CC7YAd5N3Ezvn7kSWGEze82sJu7mPX36xnesDLg9g4QYo/e08ETf34qStp11n4GhKMFVef+Ls8Da18eORO6jwah9pvW9uEzSfMVN3MovLZ1eyyWUtVzovQ49tts6vjr+n/A0WM7H/D3t/HV7VtfX9w5+1tmYnO+4uhCRECITg7gXa0lIvpe5up+7ufqq0paW01CjF3SFA0BB3d93ZLuv9YyV7s097n+fc1+/+vdf7vM8zuXKFLJlrzTXnHHPMMb7jOw6nuvMIgmww/OSeT/Dd4cvaS9YycvFIYsbHUGD+jvpw2VFpl6xMeXTK39LdDBtHvNHdQwwL7RFcuu5SLMUeMExcnGedUjlEVFYTQbkJbJlwHP3neq7aLq+BGmcIfmIIoktFyzcyKr0y/xrM6ga69HtptVbKBsKEfjIfnE/d4ruwaf1J6riXEW1PUB3+LrfE3IL/zf6c/f4s1gErBnsPgz4lxDbFsuL7FSgPCMQXbSSwtUSO5HDTPgu0ZcbgsDgIaK9AYfdeFyRRAlFEKfgQbJiKVdXOsdQLuCm2gC8mFdMwdSmaUSkcbSxEsvLfAnCY7YMEdFTR/NE60o7qWb56OYIky3Z/82jO3HGGDVdtQKFWoIxTYvYxu519TtFCo18d7Uvamfv6XA7fdZitfIVF1YJF66Q9oh0pQmTs02OpSfZ2oAX0BZBSFsmUx6fg+N3hpoQbajCXb/yW7C0WimbdR+VQTqqSxvu588PPqNtr5g37AGvtvbzaXUZU1avs25XI2ZZbqQ//jKziLK74eQGK7FFcv+d6FGoFLknCX4Sx+jascw8z4u0RDDTLAB+AabZXyKr/J2EdYUw4GM5XE77i43TZQDZsaLKpOiiJe5G4B+fRtr2MqQdl/ezagE8B6PctxKxqxtzQSWxVFbEtMsPF0fBbKYt9jPrEWjYvPc0lP1+C/5X+f9snoW1m1Jd3klfoneP15qRXmHu2kVjhNtIeHOWWIwCipGXV/I2sv2o9yzcvZ8nnS7zeW+HSoXYEkZAv7xtqdsh9MTwvts/fzsCT81j00SK+n/M9e+v28knZU1jVrdhUHYQ0nKL+1R/44Zt2rn3wHA9sfIw9kociVRIcDFR2EF5fiN6g93rv2MZYZm/PwG96Lnsu8OR5G84T5wwJx6b158AtP5BU66F8mx11CYtGLKIjooPB0YMUfl7IiudWENXicXJW19vYO3EvG65SYNMFsajLQ4/vEmzuyL6/K5bg2bjyJhDQLudWPL+0Bf3OjytOoByVysz1JYw5t9zr/H2dcKYwjYiTDuxVQ0waQ89SuHwY63sjjfF2Ov84gk/JCfd9N94Iba0ineGdmGaZqNxUid1s99KX9F3N/HrVr2y8fSPH/3ncfXx4zg2v574qX0Y2Pw+AWeNhBTiz6wyH3jjEkXdkAKPrPAaXPc/sYc3iNex9fi87HpW/lXSenm14o4nitcWkX53OQICHaQKgLaqNn6/8mbgZ8rf615x8w3/7RfqRszyHVfet4grbVjT2KJyChb7iRvZf9Rl+fU30hpzHWNMcww3fLmXwuLfe5hiqL6zNl8C2MpJumU3h4mex+gbz7PHb2RwzgZoID42uIMhOTwCVMERnjoBfUxn6rlpyfp3LgWkHUNvDCDJOkI3dRh23fHkDwXWn0MaGIimUbl1FdIoIQX5E58vGZJfgAgFEFPT5HqM44V7qwz71WofOBxdlNXyM6NJ65ewLnhbBhos24FA6+PXKMhSLx1I/+mI6kifS3g4/ln5LUeIdOBT9ODFibOknoL0CpdWIwwGZje9zwSkDlxy5mok7TiNGhSMBLWkz3XNZEESUPR1EVx3A2m/BrhheSzwv2h7ZjnaaFkEhIEmSe4/Y7bcTydXKiZZ9Xn1x/l5NEpxUbKkmpnQHSquRtjYYVHjyWEsS2KMTMPlHk9B1Oy9meQOmJZuDsZtfYdruUAKM4xBcQ86urhAmHxyJSxLIuGc2Ft8QN5UhwIHJ+9i7UE3F09/RubeEzkgFJ5Kv4J8tN2Bz2iinHP1delLmpTDykpHE58Uz70wrGc2vAWBUBZJ/Tz651+dSdU0VXWFdCJKC0HY1izcuxlXjIiI7gj9cb7MtN5guf5kxSn1czXczv6PpqDct6XAkUFdsF9nXZHNv5b1Yb/Tsa/5uDxQQF+BlSwMYqBz4y3UAfhF+BCUFISo9TD13nrsTx8y5IElojD3u/X/72XZ2Pr6TtrOyndelUNEXmY4uQWbAWLZmGXPfmOvuZ7Wpj7jirei7anCdK+XbGd/SdrrN7UTSGXWMqBxBb32v+33OHwNKhxKNQcOfN//Jvss/YRhUAZDYcQ+lU24i4c076a/vp/FQI06bt9Pkg+QPWD1/NXue20P9/novG4XT7uKN4DfYeNtGt0M0xC47Le/67HG+nvI1olJEEjzMO+cXh8WByjyA6LTjdMpONAC/7jpiS3aQdcM4Hmp6CF2Izmt/sG3hNn64zYGlrQ9r/VCI3HkgEQkXETVH+G3Ghwi9PV7PFAWR6funk/xsMr01vXzyxSfEC4lyFZICl0sk70QeC//U4GPoQHTakfAEAXRoD7K5+k86ArZg1FZwum+fG6Cya+4uzAuS8L14LnW5S7Eq2/mg9FFAZjy69dOpdKw7jK2lG4XDQqpVti0adHJU5zc3fUPXBWG4+uS2RvZeyoJTfUwrKaQ+oZ43Hv+Knp2nGLnVO9+oSVvFkzVPErIuhOt2Xue2cw2XgaYBvsz/kqMfHiVsVBhGvWwfUTllvcbmE8D4f8xgIDSJ7uhkHKJ8ftjZJ0nQuK2E5NN/DEVrC6icAW52DxdOghZM4LPy2WzYAJHKNML7lqCJzKFq/HIs0TIiqpljmNR1boaNA3HwVXs6AZUuwqeHU5hX6KZTf3G/rLNM2jmJ0t9KmbN2Ds+FPsfMb2eyOFXOCf7snmfl3MVT4jBvN7Ps12U88tYjOIzye3/15Ze4gLXABv49Vf4G4GfAOSQLXC44/slxNt256b+46/+W/7fL/zbOvn379nH33XdTUFDAjh07sNvtzJ8/H6PREzL+4IMPsmHDBn755Rf27dtHS0sLl156qfu80+lk8eLF2Gw2Dh8+zKpVq/j222959tln3dfU1tayePFiZs2axenTp3nggQe45ZZb2LbNk3R37dq1PPTQQzz33HOcPHmS0aNHs2DBAjo6vEOF/2/5nysZl2aQNCcJQRCoCvmQAd9TdId2Y5wUSsfavcQVy46a85GM6sFGDv1jPX7ddQAoz3P2DZeoOaPYtyhFjrQDLGr5t8HfQHe4ne/nfc+669YB3pRJUw5NIbNgN6smfErNzhp3JNlwUTqU4HBy8LWDfDX+K2wGz4Y+uCeYR196lBNfnODmIzcz8f6J7nPCECJOLBX5dcWflO4+Lz4dKJtyM4oLFqA8j1Hj2PEh5EfnbDJPtGDtNnB2ShalaTJKtFj7JTaVx+ktSRICHk1cQkJAJL0snVFnfWnfVEpGSQYqUcV47gZAb8phzP5ifNtbiH0ilq7QLsbHjKck9hH2ZWVSad/Hnxf/ifiwiKnbRPHaYsw18gZEQMkTR29i/vfzKe4o5sBrB1i9cDWSczhiQlaYDKEGrtl0DdlXZ3ttJALay3HWNpBw70V81zSbwkLPZl7ucweij5awjDA3DVStz6/YVJ3c2jyC+z66DzbXULm58i+oJrn9LtSBvu4cD+c7WzQWDbYeT9/1VHQR1xCHS3Tx1j/eIv/5C/g893NWL1wNQJu1hq6AHfgafUkuH+DIq3tZNXMV7UPKqJuGRaHCHJtK7Z5aXhBfoGJjBXG6kaQ1v+x+lktyEDIyhEtvuI7vtd5Itv+qfK6E7hQNbYG94PPXlKwqUUV7ZDttF7cRnBJMzPgYRiwY4VbAAJJqkvjtit/oKO4gJj8GMURGo0a1RrF0/VKkhhau/P1KLvrqoqHvJ8+50oxSSnIDsBls1Oysob9RNpg7lPJvZfen+NSVsvMfO2kpbGHv83sxOwcxqesxaDsozCvEEO6tWA0rMmdyz/D2o++DSs2pr05h7bfSNtjGy/tfhgG4/8P7iV8Vj6gQqdtbx9EPj+Iwe/rt71Kz6azy5t3fnPUXZ595Qy23fXkbmcWZzNgzDlO798Zksv1ZFE7fofa78E0IYdbLs4gaE4VjaOOgdCiZcngKQR0tjL1PjnIseL/AfR5kSjmTvwtLn4WXNS+z6a5N5EfnsyBlAQCzWmfRfbQbUSGi9lW7e0lj1ZBgSCLTP9PrvYYdr5f9ehnhr5TTeKSRr6d+TcWmCq/NgdYahcvhImVBChMemIAuVEfqh/dwcGE6mxdt5pP7tiFqVXw2+jN+u+a3ob6QP1JCXQLBHUa0oX6kL00nID7APWeMvkbeeuQt8p7LY/R1o1H7qd1GRZApMG4+fDMTH5BlXvEvxW46CYDWU618OOJDTn19ioEmmSbUNeRsDGsx0vX7dlxOFxUTV9CePAl5yZVl5o3f3Ij9w8+wJGVQPXYZNh9PFJ0kipj8o8BfVsgDo6/ha+V/Nqe+FLXE5c6ifOwUYl+4hQ9v9USf7M/MReny463pn5F/cT43nrmRypGVXvePrf6JfMVNXPjFhXTe2vk3T5CwtPfh31GFwm5x5zqRBCcq8wD+HVVY2vrBVora6k2HagqIwByXhsbUS0bVVX+p+YFPnye5tIPotRMpy5CNShYJHu2Ee3tC+PaKJZycVIPNBtNKPKi72vD3Abmvu0rlCPLznX1Bg1NYVvwDo6ozaci6AGNQAG0WTz9uGetLtf0Q9WnWvxi75eKic38pI4+twWewE4Xo2TRHtkUye89sat8/zvPpz7N1/1aKugvd51uiW6gc2UP34XLCmv9at9HPSEB3Pxunf87C7oX4Gn3JKcohtCsUpahk24JtqG5R0XikkR3/2IGlQXZsCYgILhVrblrDpL2TOPDqAY5+eNRDOShp8bMmUv1xNcGpwcTkx3itJauWv07vLfGsmr2KHy6Qo56HDYoNYV+wMyeOkDmjCf4tmLH7xxL4RiBHdE9RGvcP9kUpeK63AkmS2DJyH8WJT1E44hK6/feiQc/TlQe545/LCJqeTV1eL6cTr0ftCCPAPAYfeyyHJx8m8KlA8u7IQ6FS4K8KYvbZOgD+uOQPFh1azGC7B1U8LCcERKIr99Fb3ELRnAcoG7mXx0pm06j2RGtIElQ++Q0jTspG0ppBObeb6BIZfXY02qZ+1ixZw6JNntwO7nsFJy4fX475H8OqtfJs/vu0BK9FAawfaeKTZ9bgN0M27m+o2OAZHXYVJp2J3uRefEJ8EBQCLoXnW4uSmuCeYPq+7uP3a3/np1k/eT13RMtTaAX/oTZKMnp4OL+OdAVdcbmEzh1NxfhrMQVEYT3PKCNICiS7A78jO4hqGAIsHYbmN6BuKwR1gr+5F9HYgrq8DfEdiNwxiYaoOHom9HJywjF0vR1M/WUqGqusU/waGoCUCvcFgi3RRnBqsPt5NXYPxZ59KH/Yrl3yz/ll2Nl3vhwflrt2XRZV467ELyOFEcd/JLymYOg6WWYnl7aRu+NtKg93sXr/Ie96EXHWNxJWX0h/gRzt4tD40hK0loK0WVQaCxkZ+z41eddT9OYW9LVnOZ+jbZj2WX+rnscHHid4RDCuoee2R7SzYckGHHES4fUn8OttQpLAX4pHb8oiwvIC5nG30H26kdTjP6JxeBsE5x87iXDyBOO+X8slm19nbPVPSKKD52IryU48giMqDOeAkV05h9C5tP8tAIcQouXsNE/EVcHEAspiH2fjOAGDTzH6Yj1rw9dSs6uG4C+DOTv6rNvxIEgKukO7abi6gZzlOTRnNmPT2BAQaUw0suqGVbjGqkm9P/UvNMXxDfEs/iOfrtIujDoj56niBA1OIqrZF31FJS6lGpdSTW31O3QXfcx+F2y3eyOp97gsHJSc+O8CDsNti7+nLudCFGkjSJyZiCAKuFwSF/iCUZK4v+VzNm3axHux71H4uSxT3YCDnmByT4bRW9OL2k9eZ86PaqiKeQUyAuiMTaMjXN7r+gpyRFBCXQITNn1D0+ebiC/eSlinJ1JouEi40Cg17Fyx8y/nsluWkFVoRPRR0h/gTREqigLBTWcJPL2X+BtSaInxOPskwYHNBqsXrOaXy34hbFTYUJs8YAYcDrc8HmarGJbpZp0ZZ4AaQRRYliHTi/opA9z3aky9GE9XsePNjWwUpvDBiTfdz45qiSLryGmEkGAKLlhBTeI5r/cO6Q4h+2w8yvAQqkd4wAgdgRtZ03cvoghmfTgjb5pMT7DHiCpJkBSUxKLUReRG5uIT7ENveC92lewESm572D3nJFykFnxP6v4+9/11YZ/+W2dfR+Jo7DPnEX9uM3N3zv3L+eaQ1didIi6FCofCm5IvSAFdT36EIVpk2bey0bEhXKYbU7r8WBzyLt3pckI7RYdn79rVBXv2yAN9cNEgV/5xJSbRROug5xqNaYDitcVMuH8Cs16aBciybjjKXGkUeUF4gVeVrzKqajmTyvaT0vqY+/6G+AYCkgOIyIngku8vQaHyfIP8u/NZ9PEiFn6wkItWXsS60nV8Y/esV45Dg9TtrUMQBKYneAAew0Xp1ONywa6ndqF4yfvbuipcHPv4GIPtHoCMJMGunAS25PnQWH0Sp8VOV1wuPSGe+3qCe9g9qwBNcgzxxZsJbTgp1zek5Y8+GcyIwrU4HbiTEf/ZIEf1VEY+zohja4gp3Yli1ddkn5YrPtYj20H6hXqEio+IqNqEKtGHQf0goqTGomrFqJX1Ux+zFqXVSH3TOUqj/kFF9PNyP1g1OIoaKPqhiPr99bQdbENlUyEKCgSXLBdcgt1rzySexwQgIMoy5Ty5NpyXSuUMIdH4I+HhY72+4XBE8YiqEdz28Si6vlxH6vEf8e1r9tq7SUYzalM/untuovOu5zCEJjO8xgkI2BNHcm7m3WhC9UgDw4Ar7/1+13NdvKx+GafNSbuxnZagtVQl7eCDBz6ge5aREcd/JLLq4NC9nsiqRVsu5tgLW4mqPoxFcYZ/FFxLn0peL+Pr44n7+gWcUbE0Z8hzakLIAmYWlaOxRfPw2w8Tu7aS/tB46mIP0e9bSKBRNpcH9QYx6bC8B026YRL7x1xOaeyjbuN/R0QHbbECmugQaoRSdmfH0xr8C3t6V/F54eeM/2o8k1dOxifYh/qH6pneMZ3iuPvdY8karSD/9XxixsecZ9sQ8e9Xk1+Yj9QsYeoysbHtdVwKszvnqNaiZbBqkIoNFRx66xBWgwxK7TJ1oVVqiQ/wsONMi59Gsi6ZO/95JzFr/0plOtg2iLFTliUrdq9gzM1jiE6NRhAETN0mL+pdu8lOf0M/drMs7wRBIDwzHKMqEEFykb3nI3Y8IuuinSWdHHrjEN0V3V7Psw75HpPnJhM3Kc49hpQ2IxG1R/Hta0Eym+ks6cRuthPhF4GP0ofItkiW/7Cckl9L2P/yfqq2VbnXwKDBybx95yle63qNhe8vJOm6KV5y9lz8PQzqGhBVSma9OIsnDE+4wXXDJf+ufKLzo9n/4n4aDjV4nXMhEJkbSeT8HLrnXoHBAApk/bE1PoS0pRlkPbeMyoly9NWwHqjvqmXMllc5+8I6Ru96j6Bm73XIt6+FyJrDDLZ55NPfsQ+Vv7CWvp9kO/f5cyaxLo64ku0MNvQiDDFg1US8y60bbqOwpZCi7CKaVzRTva0aw90GlMMgEBQ47CKVqZUcn2xl1MGvSD7xC/FdNzOucj2x3Ss4HfEwN+24mLKYJ73eZbicD8gYlk8gR1ufHd1I8Pw8zsx/hK74sQQ7vW0hANlr9WTeJx8P71uMyhkgr52CTJsZftVMqjIb/nIfgBgjkjwnGZVONfQuEqWdpThdToydRpx2J5lv38CZObIMUA45++xaPfmPzqRi4goOjf+GkwHPDn3TIWaR5gZqfjlBR0IeNh9/99gUUPDom48S9Ptx3nwTLBb44guYqFvO+KoNjHHeDoCqvYnPZ35OrXQ1u3O880Ju/nYRSXsFEq5OYOOFG732TOOix1H4fCGLPl5E5NxI2qLaUIpKHpwkU0yXd5ez6+ddKNQKBF8Bo6+R7pBud5+vX7MGEZjBf0aVPw1wiR57xawXZ3H1xqv/F3f+/28RBIHIyEg3y9v/t8tfrcH/P1q2bt3q9fe3335LeHg4J06cYPr06fT397Ny5UrWrFnD7NnyBu6bb74hIyODgoICJk6cyPbt2ykpKWHnzp1ERESQm5vLSy+9xGOPPcbzzz+PWq3ms88+IykpiXfekROvZmRkcPDgQd577z0WLJANsO+++y633norN94oI64/++wzNm3axNdff83jjz/+t+9vtVqxWj3ol4EB2XjsdDpxDlOlCAKiKOJyubzQMcPHnf9isf6vjouiiCAIf3scvPO1/LvjCoViCP301+P/+o7/1fH/qTYt+XyJu01uqh1kFJypsgkfs0xL5MKDutOa7FRvPU06p9m0tIM+hazkOkUjfl21JJzbQueMmfSEOv+SK0R0iog2p7wRF4b66bx8S0XZRUT2XsGSC8MIGhGEw+FkWOmMVeXw52UNLNLdSNrkcIJSghC08gRXCAosWgtnRp/hslw5gsPpdLJPepXS1IPYFb349/uj26aj+Mgp/igZyZXPheOSBAy600iCE5tjNIrKGnQ9vhiDYt2LiFnnoHTSDeQu9sO1xRstaBcH8DP4Ed0STVdvNeln0ynML8SsM9OrO0q34hxnl64ntvMaVqw8wWTtZOpy6uR+EGRFOqCjkoGYDIKWB9H3Sx9KUYkkyN9NOaSUOFwOosdF86T1SYqr+pn23GQULh0nIxejK1RQ5ihDLBepO1jHD3kzifd9Db05kynln7PgjXZ2VO9g9muzkSQFILE/Yxx37ZiPsTuap1ruQRAEnn9ewjlLbnNoZyhL12fRbzyFZVwQCocCp9Izdp7J6KLowgMcOOHLmsVreML8BAFiBAkdd6EOrSelMh3/kghy3r6aGRcMOYPPc8Zcs+Yail8vJvS5UMbdM45107/mZm7mq5u/oimuiUGTg+zl2WgCNDidTkqMMjKwLaKTgguu5a371bQWlZO9PhvDHgMXqIoRJZkW1uWS2HzXZnk8/VhE4kNLGdHyJNEdlxHY1UJsbgqDnYNcc/m1vPTCCxzl3y+yR4HDTkicY2Sz79fMnnQnp749xaY7NhH0dhAf8zEdJtlIY3fa3WMPcFMrgEzFU/lnJXk35XHN1mtYufMAFEBTbBM/XPMD9+Xd6L5PoVC4DcdlGWWotXo6yzr4bfH3LHh/AWPuGOeuN6/ABwKPkXllJsVri0lbmsZ7LT+yO+c2ADZe2MCclrvZeOdGwnPCGXf7OC8HuoQLzdwp3P3DBHwCfShsLeSZPc8wrmMcS1hC6+5WSteVUrOzhsJ/FhL+ZgaSpEIQoKukg9LXj5BzXQ7FPxcTPS4a19DYxaXEbpdpAd1tKu+nL6CP0K5QJh8Zi7FtgM33HEQfq2fyo5OJsk8mrf0VRp6pILRXjc8dQYxZIjv0ejvr5PnoY+b9+99nXNUBRt+VSN71o9D4a/jkhMfY6jvoi9oo5/zIvjabqHFRvDzL4/B9N+pdSktLyRibQV9dH6YeOSKpfoSG5W89TWKii7XL1qIJ0HDhlxe6DQXlaeWERI/FarDSW92LzWDD5ZJIbX6WypgXufXzJTSFr+S66lvJvCoTSZJQRgTSH6zHrrZj08jX59+dj3+cP06n070BuHj9xYhCN3a7U460EwQywjJ4ZtoznGo7xY25NxIWEkbV1ioix0SyYs8Kr7F2/vr0+7W/k7IghdjJctJiQSGg1qtBAUlzk3h88HH+8f1v0A6JddGYDx/B1JOHJGkRBBmdOxzZV55WTpT9Elwh4fRYhqK9h9Ycp0JNxaQVjM2xM+LLH/ALHMOvTvt/NKcOShack78nyRjMQmH2X66J6b6WxNXxHPvhGFM/mPqX8yHd4dhf+ZQt/fmc87X95bzohMo3/2BkUT2Fs2YQz2RCxTQsg+Px76wi6ewG2sLHM3HrMdrVmZSNaUBpjsGqaqHP9yhqMYq0k/tJPp5L6aMep8c45fVo9aMYsGmJUCe4j9skeLsPsgJlY4J/s5lfJrxNXMxcGAqwawmR6c5eeeoVTiac5Ng/j/HyZS+ztWwfayNMBKh/4vftT9C390+U43owhfV5tWn598vx0Z5GynmTQ2l/NTzmnspHylRQkpdLeeKPdPd5HD0VIyv49I5Pedn+GUJZDYLkrRyfHnOalqQYrpr6Ki8e/6tR2b/fH43Zik+4HoVGgc1h44+L/6AytRKlqJSdnuOh5UQLh986TO5nSxhTsxqF04/I/ov44w95nH/4+Yf4RfjhvGGYGlzA5XJRvLYYQRAYf8947OexQnSFdeEIVjPm5jGISnFIZxhy/NuVLPt5Nu3tJ+FWcBTJ953VyQh9lUNJ2XcXYRtzmG05e73ao0KHTh1Gj08QIQvy+MaeCICfeRQpbf9AkkScSieaRRouGHuB/Kou0NniUTmCsCt7sdhsfDb2M4JSgrjp4E3uyPi2oHUcnprDs5fcwIE/8IoYGC76osOYyxrwHelNT2tT23j5qZe53LGRvo3VzIiYQd60PD7a+xN9ikpSqlK4/JcrkGaeok3VhqAUkFzDecXksmEQKqXDnF3jh7pQzXCKJlFSU59YT8GkAsJzwqndV8va6Wu5cr+M3hVdGmqTitBM1GAtsHrBFnMqHmPa/nxEsZPSabcBEtF2F5erVhJx7FMUah11o3X4jHLRHya/j8nS574/pP4g0bX1+Ay04ZfmQjgMmh3gkiBZguuQI8jagO8kaAX6DxXwa4LEg4tllbE23Z8dU7/Doh2KIG2/lqbu9QQ/dyuOEAezDs7iVb9XyViWgTPdo7PaXFaMRifvvSfKxlLP8smenBQm2MtQSGmAhCRBoCKCe5UZLMkq4ZvucMQAHf4dVdhVOiwOl9vBWTjqn0j2pTi3lXNhTQa/XnrQnXetT11C1caPSTinw39aNmaFH1bXURyibCwNVceS4JiH0yxgbtuD6GdEkiT6fU4jiC5cgo2kmiQGfh1AeElAQsI1NO77A/s5OfYM0zIXcWreI0gKJSNcElOMbxJZLdMRu4JdxF+ax6eKdzg54gV5zJmymVZ8isjao6hKd+BSqrCr/dwG5kMmDVkqJWd8tpPHfAZ1GhRh/nxZZeVa+/86J8bnSmjNbKM1qoERq5/hnZoVnHIdPe8KF9oQLRnLMghICOCKUVeQ5TeL5570kcFyQ6A8u1OOTnKd71warsEF5iE63ii/KPyrbqY+aBUN8Q1suriAG8JuQFWsQnAJSKKEn3kUOmsSNaMvxrTgIrTVbZgcvZSXPcZByfVvkdTbnTBtJ/hdtouu+Jtx+ThxOBwIgsCU4IvprzhDnfJPRp1KwdnhZNzd44gYHSHnTHOZcQoWapJrWH3DOc6+/x1afy2SJBGiDiWl5XGqo+XcQZKPgoasC4m0jybinIPo6bLeb9VY6Q/SED8lk10RAZSlvuH1jhFtEUw6msaHIz5k5JKRTBs9jQMNcrTbSMcy5ne8jr77BzIfX0y4KYiqIZAhyFPbt78F/9qzmPojvOoti3kCi2U2YZkyzb7VaEWtU3to/n2PcEK/m9yFiWRFZ5F7U+6QLiOPEZ1Rh2QYYN+L+zC3miESTA553Bt9ylm35G2e738BvtlMyPgQr6iv0J5UohraMHYMUJj+D2yaPq93K84spjVWxxvZT+Ns9mYP2mX4GHNPH5OKsgm+aildXZ4cc3ePfIvFIxJRfKTg5PUnSWhM4OfbfqbfKs/X0P65tAX9QWRrJMllTnQDbVgCPTGtgqRA4L929nX57aLXlcNg5kLag/46VzKL0rH5wpEFk6iIf8B9XOHUcU+Aiak+Tm7OFOiKa/2XO2WaN7vKh1E/P8XtxSPpFS4EIKz/Aga1xfgZ/NCUadDkagh5N8Tr7l/nXMPP37/P/LT5bv1wQ/VvHE6XZb50HsBAlAIJNiRjVnkMtQMBA0x+azJblm3BZrINObpdmO1mkubJRtFh+0LyC56IRpU9hLm778GvooEvkr/gm++/4b5fk9iTPRKTtoqgniAmH55OXW41rSdbEYtEmOV5b6lQYsvXWzD3mTnx6Qnmvzsfp3MUgmIoF1JeKNP+eID3P9fhUHuipPqC+iiYfIKbYyMIqz1Ob2Q6nXFjsDtlHXHvtJ8xhrxPbl0Xvl1GjIExiIi4cCEJEvrualm/NfcS33oRsI46U7G7/k2L/ySj4SXGdPSjtqqxqboojruf1uBfyPCbyOFF88g9rqbxht8w3LmK3hA7E8v3UB75IBaNBa1Vy76X9lG7s5av/vyKok4du4fItFyCvD8AGXDyfbecE0ptD8WiasHuDEeSRDctoG2Y2UgSEexWHGZIPL0FY0AUnUkT3DYMg95A5checsYsoUKXi0kfjs3mxCYasKiaMV4cznGfscw8dJyggiKax1yL5Cff2y2UsyX8apRiGvuW+xATnUXt2Es5P7Jv1u5ZmPebiZ0ciyAKnGk9zckUD0jOJbnw76zGKapktpWhdSy2cxmjzmrxvyCCd+eupjX8JZx1HjuBSWfCGJ+ONUjgZMLVKFw6NMKX+FpS0VmTaIhvIDQwmZrQizkyVt4PDTuKGuIb+PK231hsfZTDbdvckXWLj7s4m3gbjWFf4cKBoFbiv0/iyUNP8srTMhVrVU8Vgb2BXLjhQs4knsGeYHfXLUkSfb7Heap/Ml99lcSWlC1kfZBF2aQykATqUoy8+eg3PJ32NG+FvcXc8XPZsmiLu01FOUW8/9b7WD+wcuCVA+gX6qlT1bF07VIAnmp7ild9XyXr6iwyr8rkmenPUPhmIdGbozn93WlyV+S693zfTP8GpY+S20/dTtKsJOKny45CSZLoKu/it6t/Y+6bcwlKDeLUqlNsuWsL12y9hhHzR+ByujB2Gulo1uBCRWvKFJZdGIkkSaRckMJdZXehDPRDWi2hspvJ2v0hux8bzbIv5nv61SV/D5M+gtNzHsCp1BKTq+LRP8bicrkoGV+CwWog/pl41l+0nqtyr+KneT+Rf08+0dclktB+NzprkvxdXU4yr85kIBH45qT7GRZNI6rBOsw1WpzOyL+1w855Yw4um4vx941HG6xlxyEn0pDOIEmwYtcKLrwQqIe1a11uu0zF6HAmPTmdEyc81zud8v7XrtRiCE5AnxbFmRo9Fn24l621M24MPZHpXGIe4PSq04y6YhRqQUt+xSZqol8n0GYmpj6Q6GumM+DSIzVJXqwyfgY/JEFg+idX8N7mQADaAzbx1cndzE6cTbVfNQP7+/j9x2+xYYcfgHjoXXgER/BUGuMbUaqtJDRNxuwXip95FHpLpjwvh9oy6OOh4XR/q51z0IgVGEOURJdbOevxLdMf2M/ueSUsz01BOiG3dbz5KU5r38cqysCZC3a9hGiDlultQ3NCdnfoTTm8ozZxfGsb/tcFsKfz5788O6Itgr5H+qh8opLkefJa8dL+l3hh/wvcP+F+3qt9j9OnXTzzjER55iDo4MSIy5hZVIavZSR2uwtJEr3sai7BIeuKNhsIgkxhrFBjtJg5k3APFnUzzTHNhAePxufgdhKMZupGX4TTKTGgKWOf8S0SGqLRiHPprO5EF6rzeuegaohf8ifxwpV/Sb0TrY+m4KYCt62/uUMOalGICnqMHqBR4W0yACx9dTrPVslOyru0d+F0OukfGMAPuOEvX+vvy03AKVGDJElIkkB4jjwu/9Xf8X+SXyMsLMzrG/y7Nv1P5/b738bZ96+lv19WfIODZYPViRMnsNvtzJ3rQaqlp6cTHx/PkSNHmDhxIkeOHCE7O5uICM+GYcGCBdx5550UFxczZswYjhw54lXH8DUPPPAAADabjRMnTvDEE0+4z4uiyNy5czly5Aj/VXnttdd44YUX/nK8uLgYPz8/d1vi4+Npamqip8czASMjI4mMjKSurg6DwcMZFBcXR0hICJWVlVgsHlqq5ORk/P39KSkp8RpEaWlpqNVqioqKvN4hOzsbm83mRYuqUCjIzs7GYDBQU+NBz2u1WtLT0+nt7aWx0UPfpNfrSUlJoaOjg7Y2D772f6pNiYmJdHd3YzAMup1uqRWpRG4vJfquZWzZGwGGAUwODyq0NcpJ4uOTqXv9MNp+GaE9T3oboXcqJlMdLpeTprompAEHolP0QkFMOjKJeTtDWXh8IYc4RNoHaVQbPOjM2uRairpzWH65RJuzjeazbW4D2tUBz3N022zUI82YotvImJBBrbVW/q6CAqOfkXWXrOP9tPc58NoBSIVG+zG6ArajtkSTVpOA7oiOkrT5tPuE8+oHdZzOm49FJ+cN6T70AiG7jhMeEM/ZrKWoAgPRG7MRLDG0qgJpKCtj5NGz1AVE0Rotb8ga9GuIa4zjyp+vpCuskpjOGDQ5Gsw6M4dHTXK3y2CwIN6cwube97A6rBhsPaCBhohPWXXdY8R2ZBJdJ9OpHW48jEolb9gkp8iEoxOwN9kRZgiUlZfRVK9A6EjEhYz2W7h1IeWnylm0dhErUlcw6BikKWMq084UoW++EN+oHzn2z2Mo0hWoJ8ZzaOQNDPie5M+LLOQ5Hyfi+GbwD6UpZgy5xmtoLctBZ6olrCOEns0FrPq8g8TliV4o2oj4DpzRu/gtZDkLE1IpLilGY0ogu+ETUlQ9iEX7CTh3hpriSnInxhMSEkJPn2eclqWXkdCQwL4X9qGdoCViYSLtW+sw+hrJPJdJY2wdgReHcfKkHxwupqtH3rQHD87FZAyjW9uAfbIdww55jDdrtxFoiEFtM5Kw9RsAgpKDGPPgGI5WVGEwRKK26RldsB/JZuaT9CJ8k3y59777WPjhh2x1Ov/W6HMUmK+AgHlQNcSm93J9HaMCUkiem8xZy1nKjLIDOKkmiZHfjaQmvIYtj2/B3G5GuskjJ06NOcVbT7yFWWumqKgIbVcIc043sis3jkp9JVZtB388fhxNoIYlTy3xon01W0z0CgMs+XwJfhl+nDpz2n3ut2W7mTFwP1Pflxjx+AgsWOgq8iDy4prvxtETyInftxE1K4r4S+IZ3qgF9AUQ0xRHS1gdFkLxVfjS0CBv9HuCeki+OpmaH2voLOkkeHEwU8ZOYX9NNwaDhK+vH4Ot/Zz59gxSlMTZz8/SXt9ORNLFDOhOYbB20NjYRFFRj1vuuVpMdEd1c2TSEYqyahilvpbaH88RnBGMfqGerq4oQmtvILHmN7S2Qbq7eygqkjfyNb2yIiWJEn1BffQ5rPx22bdY2wa5oegGL0Xs4vUXk1zjh/ElI5euvpSysjK3fJYkiZwnc0jOSGbv53s59cIp7A8HgF7+LA6Hi6KiItoq2lD7y3Ld5XKCAo6PP06o9WoGwweZv30+WdlZ1NUNEFx7NWOb59Id2EBAgrzu9Pb2Ul9bz+lNtZgsu/Dv9yes24+uji7CLpfnd1FREdKQfKsaORbNYDqHv9lD7btHmfnuTKbcOoXrE67nkoBLGCgxcK7pHMkXJhOaGOoly51WJz0be0iYkIA1wUre63logjUUFRWRlpZGWGYYE7+Ro/7OFcsIRfuQU/bIpCPEZd5AdVM1YWePYo9MwWKJQzSFgi/sm7mPjK7vEB1gt9uxWDzUvQqFEl9fXwYNg+i76zAq0klIeoEF9c+wzSn92znlnAPowdpnobewhJTeFC8ZY7PZaTvZSH9ZJ5ZQC5f6X8rv9t8BmFx0DD9LKnZdGQUfHSUwxw/mgcLph1MxhLS0OjEUydEfe/KWkGFfzsSeZznb5ku7ugfbyHmk5EdSUNEq08OZAxhTsJ/WuJXo618n+/hEGmKmYfDzNr7ZbXZq82bSW2uhaU0xqcyiMlymQpzuA3H0ErV9HhkV/egTYjBZvJVLH8EftVpg/9v7sfZYmT95Pjd2nCM54DLwaaE5KJqsx2dQtbaYU9Nu8brXJbqw2m30DngjbgEiWq5l8l4VffY6zmV0UzLiccJsicxs/Y6eLl/OZi/DrDNTG9fNsQlH6Wnxpo6ZNfAOXbXJNKS2YfH5aw7LS3+/lMi2ZmafeRRTbQ95a/IQ14t8ftvnGIKG6H/a2xFzReZumEurQU1k51WIosiAYYC935/AaXIw8YuJZIzK4M91LwFgdvVwSPcw+sec2EvtVB+spsLqaZ/aqkYy2cm4JYPGxkaKioro6+8DQJAEEuvi6PSvYOBMixuUMuxAv7rbTvANq8k4eAsXvTmHkusOudvmcDjpCvLl3PgVTM5Qw1B6pm71cSzBz9OrO4KfwY/mvc2ctJ1E5auipUWN05mGIClIrE3kyAdHUE1I5FBFMNnHXYwUZ6Pvz8cQcJzKuFLa+rsIaexFSqv9y/f0aapE0Gk5me1N+YYADpWDnt5eJq+Wc8wFBwezZlcxfYpKlOoAeoJ8CDh3mqe2PcXXN36NI0NurwtQF6uYtWcWo0fVsmtvB4GmQJDBqwT3z+Tnu5/HPFDP0U1H2XftPlLvToUhH35g+3wyM5Pxm+VHZFQk6sfUcgIJIKLqQsKadtJbpgK/NBwOB+3tA/j6WjAPKJGUPfjroKfHgcEgMaA/wRsHPJRbrf5vkWCaS+O8pWyOvg5hFfgBG/krMOBaZDmxBIlH14A96mHu/TKC2iwzNRM8evO7hx7GFTKf8enHiFjuS9HZIoJyg7AH2N30dAADxj7Oni3GYEhAr/8r3WFrVwMxCrlNJpOJkN4M0g0zmZ3wBV/4nsBgH8feafcBENTVizNMrrss/SQdoaHcdGwsEa06mXkCiG98kIa499g4/gdG6Z7ksmtn0vPoZ8QGVrN94tcAGAeNSPWVxBbXEvfsPHb+EYfNMMCR3Jk4VQP4mUcx+WwOhtMG9qfvJzgn2O0ED21fyqjib7lgShOvWB3o9TocDgddXX10W3vRWOpIOtWIPW86J8YecUePS04Bk9HMYGgSpumzqVAl0mP3x2KW873NaLEyojqGa1bX0n7BGXZOj6I9qYOud/mPQVFxoXGENffRXN+EdcAid/JQsSl7eKV8NdqrtcQp4jj1yCkS54xDa4vFYBjAorMR0RbBiF9G8MV7X7CseBmf3PEJZqOZpHMXMKkjkxRNCnue30NuYi6dUzqJqXgYrf8kzuQupiy9gZ2v7kS3WYf4tAgKgZnFxTT7/El5wGdYzROZcHgnu1THmSIKTPgbZoLzywRglgIKtr8M3Izx2x955clqQj68hjNnAzB0vsVJZw+ZtTo4A5HvRWJQG7BYLHznm0O/v7ye9fgZKK8td+8Je+r7iSl7nLrwj3EqB2m6/meSQlM5nb4Ig/4k6ztXgkKmOtw/J4KEEaEU29e4c6oNl6DeIHJPJdBLL02nm8gYt4ADyM4+m9VGh68vxTPvZ/5iPdZVR8ET+MpvDR9TdnEno88tQX3nQZLykqhNluVUv+8J8n8NwRJo5Lm9z/HV7K+4aa8HzDCgLKcndBunjZdj/9pO5IRIenx7cDrtoICZe2cSfPwAx8J1WJQWuM2bNcSsaqI/0pe9U5NojfJeXxujRNYvmsz8AAjsUNPvo8Ku9jjur1F+T3lTPK2bvuFa11V8c+PXXvd3qApwSSNpbGwm2DqdnsD9ANx4cBJ90msEBwQTPiWc8spyL53RaBnAYXORVpXGxAMCZ6cu52yuh1rUaraTYnmMGSOz+LX97b+MFW37W4hH5tIU9wA2aRHwoNf5BdvnYdHv5cy1XzGg91BxhrZfwoyRP9BXH4UxBKrHVOMVNCVBn6mayDOHqfwjmt6RnrWkM0B2IkR1RxH4UyBHQ466ZflwGdRVcK72HPNHzsdqtXKu9Bz3HpAdfcGdF3B/0mZu7R/gXMU5vn/3a0RJjcLmbfBscjSRdkcargj5xV7e+TLPHXmODyd+yLSIaW57xPnFxzgCm8mJRq9BE6ahvqUegyEEpSUMtFX4DfqRXziGop1nyHktixBTCHhYU/Gd7MvkSZOxWCz4xfpx1aarKE+qQxrKEdzc3Y5y5EQcQEjNcl6aHsYzDUMOTFyYzRYKpt6C3aXEYRjAYegDoCGhgeBOPeU/7CGjoJSCSbfxQPoHvFt2L06lkw0Lp+BvyCMvT6K95l8idYCmuCbie5VULNrPrImz2LZwG63B8jo3KeRCalpm0KmpxjqpHZPOhF05SKhhJo0+UygZVcKIsBHEXhNL4rxE8qfn0/lVG1aTTC/ocFmGAJJgMHkoD22qLnaOjiGirYbGRgGDQY7UrqsfcspKAnnbXqezPonQplokqwmSJuB0yutFe2Q7W+dVkhCQTLcxCLPZTFlZFTtHj8WlMLG/GTJC3sS6To2qp4/4E79RM90jLBoVGwmnB93cuylqjsfhcLip7AAG/Adw5bi4aPVFIEJtXZ08Bkw+pJWnoZVUHF/wBEbjIBgGcPT2gAAWrcTJmfcw8opmWqvPeQGKQQZ6ncuZTlL9LqJclZwYd4KLDqxHF7OImOqHOJvZzYTISUgNHvlgEWXgrU1jozu0F9e+MnruO0v0nGhaYlqw2+0IiCzatIi8Ew5s82TWm5JRHsfIQM8ASoeSmOYYmk824/jNQbpPOv16CYNhADNWIlsjmf7bdPaF7SP0bCi60TosJhuuQDD5mmg3tTPmsjHsEGQQqr8plwHdaQAqqirIWZjDzKkzGfvzWK9299h70CfpObXyFEaTkZarW/huxXc8/O7DrL9+Pbkrct32vfB54Ygqkd7eXkJCQigvL6erqwu9Xo/dbmfJd0tIGJ9ASUkJg36DpCxPoc3YRqwlFlOriY9TPkaVkoshZialMflkXO6HxWKhvFa2WZp7BAyGJAJ9NAwGRNMv2CgqKmLLrC2E5YUR8/At2O12eimlK2o9Uc030dOjA4LdNku7y86146/Fd4YvCVMTuHjLxZgFM51tPiQUv4JGo8HU1MOudwrRj9ZT1RqCxeq9/0g7dorGIyc4OGIC/pI/2ZdkU1pR+hc7bEhaCEVFRdTXB2AwyOPX4fDDYrFh7h3AJSiorrZgEOV5UxP+AWfPZnB2dRu+Vf50RGXhcvlht9sxiDo6MpYwenw75WVj8PX1xWa1eAWXqFQ+lP18kNIfTmFPtmOw++DTNIUJfduZaDiA68Bh+p/T0u3UYygdIGu2ibMGmBgxkRDXWvZMC2BMdAsOhwOlUoVjCJBQ/UEFilOQSTe3MASCc8GXdXDwMyhK/x2uApvNwrnEodQpBgP+/v44HA6cDvm7RDXdQmvsV0wKupjKajtdwZsZWTESpaMDk4+d0G4Tka3X0hDuyYXpctmx9Qyir6ugVxfC2sxr3I4+gMyTDmx+Fk5ddAq6oCTuISpinkXfN56ssidJL1hPfeAYEiZMp16Q19704q+oT3wDncmBfb+dNfvXMG/TPHTROl7YL9vvPzj6Aa/NeI2tG5oJLaoiXhPMuSGQrN1lxt7RyA/Tfie/uh/HRA0NC4fOOc0YDAO0hAYyZ+VVHF4Zi81qobK6ksYwWTdYc+0aVtjuJfTTc6hMAxgMAzQ29mOihe7BfSw6ez0No0XG/JHAuQPeEZx9LuhLqUI0DND0RROvHniTlbN+oVUq5tXcVykqKiIhPAGX2cVvmb+xMHchx5YeY/+p/e46NFM0WA9Z6f64m/Qr0+lx9GC1WCkqKkKj0WDkv0eVj6DCYrGg0fj8H+/XCAwM5OjRo/j4+Lij+/5dm/7VGfn/tPxv6exzuVw88MADTJkyhaysLEBOHqlWqwkMDPS6NiIiwt1JbW1tXo6+4fPD5/7dNQMDA5jNZnp7e3E6nX97TVnZ3+fMAXjiiSd46KGH3H8PDAwQFxdHZmYm/kPUYsMDIDY2lpgYTyj88PHExESvOoePp6Z6I66HPdqjRo362+PZ2dl/Oa7Vav9yHOTB/nfHg4KC/vKtAcLDwwkL82jx/1Nt2nzXZuxBdha/sARBMWQQECQkpYCoEN2GEbUy1nOT0sW4q2ew7sRkytNlhKG/FIdVocMZN4qSuFGMVRVw6R9NODXzKc7wRI92hHdQkW7CJ9iHvt4+L0cfQHb9P9ErYhk1yp/AQJkfW/hVfq+QkGASBhoI75YFiyAIVLfK94f6hrLq4lWIgoil1MLuJ3ez+IvFqDXyVFQIKhrjGum4vxN7UQZ+PgEoVWYsOlkwzdozC1VME9orL6Kr2g+9Xo9//xK69fvoCNrEQPwfBNhHk1TXQVBvkNvZ1xb9HWZdAOuWruPBi97nx8HX6e/ypsuZ0vsQtGYz8tI8Wg/L953WfMqI1ieoinqN+uRXGVc9HsWjCmImx9Ac24xdLRs7rw3+hIzmepzVTiz9Fvy6/YgNCHL3i0JUsuHCDby+8HWys7MZXO+hFBCULnxCIrnstcv4Zso31Bd3EDhaQ6+/rHyWZJYwnhii/zyGyRxNe+J4cuNmUn1MhleenQdL85roLl1Df0A/wT7B9JhlATytEfpdsGBcPEvumCF/i3Z5jAUHB2KaOZtzgRNILC+n+2g3IYtC3PNxQswEbnntFiZ+OJFQRyiByYHEZudz251OYvtjuPTXZuq6TzK46Ar27hUoLZUITCuAblA6lYRhIT4kHku40r05dKj60Ov9UdhVmCMTuezJVMbfNR6Xy0WttRdXeA0uu5ba7CUkZoeTO02BPlrPnQ/cyekz0Uzd8ziTFE5ud8iLaRvwlUrBYZeEfY6LyCkw2OfHnzvGMmJ2IqNvGM2YZWNkqsTdMo2nyq5C3afGbrKTMjUFU7fJHREmdwikpqWiH9BT+GkhYUkj0Do8+UJiYuOofXGvTIn01NCclWDS4UmMPWMg+dURxFzgi8vlYsBkhiGwYEtcF/aWCLS2PkJtoYRlhRF0pAA6IN14CfM3zsAep+ShtodQqBRoA7RuY3hCfQKXrrsUzbIBrPV6HH4OYuPkuV6jqmHg/gHmvzafSQmT3Itm1ZcC6qBORKcexYhkHu19FKVWyZz75iAqFbz6TDG9+sMERe0jJuYCsrM98knxVi6/1b+GWWfGru4lLimJpY0P4pJcKLVKBsN2Y3B1cTZkLjp7PDPqajnx/AmmPzed6RNnMuWzAqoj3iK6K5swIYyBc/I88XH6eNF4lmaUQkgcQUHypvh8uTfcjuScZHzD5YTYJyLbUFb5ozF1s3PNz8y+Kpd7Tt/jvsf1u3xPsGEaEyKvJjvbI4v1ej2hmjQgjY7RMG2ZxKE3D9FwsIFla5ex4bsNXE4Ex8dNI78wH78VOi/Zn//hFxjtZhwRMxBCg0hIr8MxsYWoZHlsJCYmcnS/hb2Xv03U4lyC9S7aTrfhE+bD6OtHo1ApsPRbePvltzHfbGbx54u91qh/XZ9M3SZ6KnoQBmW5OqgfRAiMJG1EGomNW2hXgFY7kokta9gQJlOqBgT749h2kPxTRyidfBNWX3kjJbicRJfvRqeO4OQFMmVIOrAn7iumHahjpgKut3vm1JeiikOSXXb0TZYRwWNPhWMQt7FQtZBP7vnE/d7moLP4f3Qx42py2Hb9Np7/6XnOtJ+hurea5ohvaY6QCJh9IQlvljO2eCxCWBMql57mkB8oSrgLSaMi8cGL+LTmF4x+RpRoiIyMpLZWAL0/AxGJ+GYLVIytdVOYqYOsaFQ+9Ab10h5jwpgxmcbg7e53iuu8mbCYCMSoZkKq7LT+8yiRl4RRGQ4hIuyLBajklobpqK0SWV8u4P1PP3ffr3IE8VlqN5NCqzD+w0h4djgR2RGEhQk4JTXxgWfojNvGYLeSEbWV+A/4e0U9rLl2DXf53kTyVxuZ6JhIwaQC9zm1SkPluCtZerk/4uk/5DHuN4YxumupsAhUm5Jx0crI6HTO9Xob89TOQGY1z6ftVC3BUxSEdIXQHertUDybcxZL4EQaDDU8dvQx0v3Teeehd1ixYgXP1zyP+KBIUHAQY4/LVFIlJaBQDFER6v1p/LaBjnMdPNz+MKIoMjXoKhpOZqJ0BnByxGUcO5VP+MZwxt09jtT0kTAUqHv9qusJMhcT9OiFbt1Iv08PfWBX23ntyXf4vGknttvribo5iqa4Jrd8uzTAhXVsFV01figbLN4GKmUdx4LfoDciktLbdVyYdjkb5v6CWV9GZ4RMsZtXOAnlO0oORB/gnup7cKkNlCf8A5uqi/HHZtNW2kzplTcxmCLw0ktw0UXpxPVfQ0nAcZQqEW2Lk6QzmwjNDXXn8gUI67uAtllXMT7fha5QRpyHaWKJP/chJ1OWEdMcQ6hSTXZ2NlYraLWCu01jRt1Kc/vt+KkqOB36J0ZfIysb3nLXrXQomXxkMlaxn2XfLeNcxzk4MzT+pAD0LUGYt9UTPSOa+e/N52zYWaiSz4/sf4jbktXMmmBCG6jlj4Y/3PUaEm3sj7yERTOSCPu2EKvWD+doPRERUTSnP4pk/p3kI9fS+8MkZh44yarrV/Jbl4dqcfec3Qgh35OVEInhSDMif+/oGy4Ths5PBU6UdpHkTER0eW+tdHpftOaLyHzwYjJ6DmA/Zue2fXJE+1tvyOApnSWZ8dpbyMjIRK8XvWjQ3PUEqZD6QalUotf7k5Dgx9fb3+CFzk1c9PNujEolev1MAAICJBSiyn2vQiXgs+AC1ozZ754zVr9awiyTaI49QrglBPvuIpQGO2enn3XfZ1R30Te4iZTaAYIUK/D3D8BqBUEcok+TlByacoh8Sz6HbjnEvbX3IooyDVpCaxCTTnyPpmoG4ZIZh1VEFP0ICQnhiP+ruCzrmHXwRgwN3eAv1zfvVAdIAsXZt1KBSKLPagb7dlAQdh02lWwgzVbDxHAjNRll5CRfhrZvEPTgmgfzd8mRbv8VgGOBKOKc6+KCgxcQ2t2Eec83XEE2Z7Mlfl8mgzQkwcYnpZ8QrY/m3vH3sv3T7QSExSEIyej1/gxo/RjsVVJSWsx29e/0uwZwrDbTE/0bqdK1ZBXXo2/ypaPNijZSi0t04ROgwEczJM8EiQk3TMSib+aZCe+jEDQUHJdojP0Ep/M4aS2v05o6jfraL3nT6fiblvy1rLDDI9u6uVp8Ek1SHikXZ7BmawoIAmOidvBSxlbu0PhgfzKD0WNHu9HGw4AqpV1JoMkH6YxE4YFCxt8znoyMUfJYHIoUU2eFE3y2gpjEfEoTWjmt+MH9fFEpERsTj6pKAgkW+P2DbUaZ9rJqRBVf3LmDfU99y7jvx9Ff7dl7aDRawiMjMDV1c+7DTegdgzC0jcur/I2wWds4FPgnkmYqUmsQaps3lbUTBw7JQfJlycSNjEOr1bojWtWirBf4mWRP7rF3jnFnyZ0Iv8uTS62+HXGGLzeuzGPad/P51xLYqyPEX8QaFfgXYElfyF5OhezlgjM/snzdeL6+8RwNCQ3kVf5OkvVips20YuruR9TX062QQYCh/fPpCpDX6arUKs6M15Gyt5HRAdezZ7JsdDO4Ouk3hnPDizLNZMOBBibtmsSRMUcY1A8yGLkPNRqKsotw+UwlSAqiKtRjBNVqtYR2zWR2pN7L2ae2h2JTdRHZFkl4AzSO8gf8vSI3AX697FdW9PzCmFPbOZ1TRX9gP2PNj4Apny/6f2Dww2XkoKXuIQHO+ySS6ODd3lweaXoE4cAAwSHB9IR4A3U6wzqxpFko/6Qc5VNKHCrP2PYx+RDSEcLLypcZdfkoOh7xpEVRKdX09so6bGRSJNVpct7YIMNk9zUpVSkUPS8b/Gq+r2H84HieO/IcALte3gXtoNar6TZ2w3lsYkqFmtbjJjLmB3JH4R1YraD/UUQcEuEt0S28/fDbFN3RTlyShmyyuazpMn4t/ZUXZ77Ik1M9FHS2223c/qqMWNE4I3AyQIg+kIr3tzF+ayElU24hKjoaGiCyNZKLNizFcXE1vuGZbupBX79AGPq/j68P8VPz2d6WgMo/CLXgyS/lpwtEjz/+/hKi5ElNMrnkEGczLmNQaMWhghFX57G7ux/wpKIRnZDQU4FFH0z1lZ0M9nnWfLVaQ2plKjqrjhnLZ7j18ri4eHQa2cYlKJ2AiEIBfnpvHQ3k3NwJCXHo9fI8s/vJTi6bup1To8+QmpxCecLj7vxjotKz2CmUCnx95RB/rVZLYuIIKJVlVExTDPFVdkTDUH1B0eh8PBGfIytGcsUvs4l8NB2LJhu52qF0IQicGHeCG56+gdBEmREiNi4OTshsNkvXL6VZ0YvxPHuSLsAfBgBRxOkXRGKCFv9TvvQHmLzA4QC6IJGw7fWMMozixLgTDDi7GYj+ngjLbBK6byFa40BzbiOZqkyKs4ox+cnfUuFQ4GvQoHM50Ayo3EAchVqO0OsM66Q1FkYrtZy43si6pHXuZ8ZExtAV1sXrT7zOQ9c8xPGRx4mbHIclR4de74/LJwCdSUfiuUTGfDqGlVevpK65jnEVviRXvccvi54jfVwg+uV6Hn/jZbDLrAkgA2x9S3xJX5pOo6YRZ4m3g3P+Q/PxecSHlqMt9Gh6CAsN45++/6T+y3rybpap2Yfte9kfyns6QRBoPtZM8XvFBC8IJvOaTBQKhTvCJcgZBKOAa4bGkSji9HOSd0ceW8qT0Ov8h47LYyMtKQ1zrxmifdHr1bgEgapJ17H8ThfZ2VA+q5ywzDDMLlCpVJzJmYHfoAKFppEA+8sU/1xM4uxEwob2yWNHj3Wviznzc5AkicpK0OtFDNpSftz5ARlrognPCcepDyQqZja1POP+JnWjghmZms3Anjb2fr6XtM40rz3unzf/SUdoB5MemUT6iHQqK1Xo9Z7xe/SNo0w/vI/BwFiSVtyIuUW2w405rqXd0IR5Rx1JnRLGERNwOuU2qVSyjpeS4odOJ9elVmtQq+V+FB02lDYTo64dQ/ayVOLHxlPW0o1e749WC8qEdCp6A7kobQTqXh/0ekG2RxogMjCS4JBY/DoakA71oJEcOFGhUAoIa6CvoolD/NcguAVl1aR/N5JFPYnU5Tqw+gbT43eIJk0tAaYxKBRDKWIsCxl7/Avuukvgn9sl9mtH88VtX3CzdiPp0hQO7AO10pMeQz+gZ8mmsbSd2U168SnqshaxV1vq9Q4/Xv0VE1X3odYOUZKrurHTjeivJTI2haKM+STs2s1i6+X8c8Z+QgZmkWK6iQHLTmqTf8LnaR8SShLIGpuFb5gvrJfrVdqVlP1URkiflqSGo/SGRrqdfQqlgJ+vDstZWZ76mzz7gAjTXPR6f4KCJPp/3cmYrd9TNOMu4uITZHoQIKd2JYFRI6iZMQG7XcZ1H9O/RmH2GyjtSj67Yx0XGP5BZroePGmAAWhJ9sXsMrK8WsfgmUGsu8yka49y81QHy2fI4+LNoDcJGxVG6JRQurXd6NQ6Hj/tYSNM+yiN4knFCEaBn6N+pqe8h/Ex4xFVIpdffz3/fOEF2v7DiLN2QK0OQavVIkmwe8Vuemt7ua9GBiP+n+bXkCQJHx8fMjNlmfe/apPN9lcWqP8n5X+bnH3nl7vvvptz587x008//a8v/v+RotFo8Pf39/oB2dM8/DPc8aIo/u3x84/9u+PDA+7vjguC8B8fB/7L4//6jv/V8f+pNp359gydRzoRRYXbmFSVWkXTg4koFCJ+fU0IgoDWGUFCh0wBIzpcDFR2IUouHOphegMlMt2I/OOXHEln/FhMvt6KTOXISg7O6ePELzV8+cZvXufytVcxsSCHyX9+yA+TP6P1eOvQew3x0iuVxJbvQnnkELuf3M3KiSvdyEyNQsPkwMkYXjJgaDZw24nbyFia4b43vH8x8f1vcfHypSjtZhQOmxvtDLDq+lVYF01FNyGHwbBkd19bVa24pHqWrq5jsKienZeOpzyt3Ou9+wP7OZN7htCRCTgCHEiip16NK5DZWyYwd3094dp4tl4rOz7DXaNJb36VCHUSydXJBHT3obQqmRQ9yatureDP8o3LuX7n9bSfaee7Wd9RuH4XldEv0Rr8CyIKqkdUEzAxgLYTbUxq9NzvFM20+2+hUFXJzqVp3KW6kcs2eedNkASJc7PupXrs5UN9J7r7UBAEfEbG4fuIL11hXeRE5Ljvq6uMZfTK5YhlvTQaGvn69Nfs6/4Zq7IDk6Ido/8AgwEO6r7ex/7v9/PYjsfIjcrlupzreGLqE9w09iZGpY0iPDMctY8aUVSgQE1C9y3UjL2cEddNoeGTjcRU7KWuTnQbaR2uI2Tv/4x1i7/lj/FbiWyV8TAOwYQgCLjUOtrnXEvu9bkMNA3gsrko7NnDwayx7B4zisOT/qAjxsSCdxYw+eHJKBQKJkx8hPGLDlKbFMNThHKdMpinCaVolD/2B1wwGdqcMKLIl8o/ZuMqNnlC2YdojdQKNRVpFRx97yhpF6ax8L2FTPnnFK8x5jvoS8O+Zj57voPDbxxmoKQVELlq6zmeefFl7PtbWbFzBRd/fbFc95BhxW/QD32/E4ao2kRRdOfukTsRJKeLDxI+YOWElZz99izSUMIahUskuLUETXcr+nA9uiAdoijig4MTcXDZ6AZ+uewXnLWNfJn3JQNNA17c11+/8jUb18pGU2Ho3znLVnaOiaQo4U4++0KJLlCHWqvGP9If3xBfRElWhiXRjrxx9cgx1642rv3hWnxMPugNOiSHhEqnQuOrQaFQUKB9kVMp19Dve1y+3uKgv6Efp8WJj8qXINMExleu4aIfIaX4FNpgeUO8cvxKRuomkVf1Kwkdd9KRMIrOqZlY+638ecuflP5ayrzV8wh8MxDNaxou2XUJpV2lRI+JZupjU5kx+jIWnu5nVPlMul6sYOfunV5yclg2Tj0wE+X3BzC2Gin5uYS++j4kSeBwxhS2jwmlR38QQRBpO91G1dYqVFoVTfdr+fy2zylPK+fA9CIUWg0/L/2ZQ68fQqFQEGlcSFTfJahdwXIehXFJrNixguQ5yfRZ+vhiXTm3r/qCspxg9nbEUfxTMaW/lbL5js0IkryO+AT4cPup25nx3Iy/XYfMPWaOfXiMlqMtNB1q4ttp3zK6bhx6UzZqqxpjuwXfUF+KZt5Da+p0ubclNSEDs1i4/UZUm/eCTofFLwxJofTICCC68gBiRbmX3NDkRyM8BHumwu2hcLVPNG+njqQoLw/nQ8CQLcmusmPVugi4Yg7b58vGupABGWzQoz/A9bvm05Pew4rdK0ielczU+KmEG+bTryukPvxTWqSTqO+6mepxy/FxxKB0+aNwDeV8VDgJmzuGzijZ8GWki3apBJuq0/2eLhdIokf569UfQkBBYX4hWy+rQtD5ciLVk594dP1XlDm2Ul1+O6L1FOmvL6Q4QTbAK4amTZlTz8pbVrLz2iDaTS1UxniiAyRcSJLAjxf+SPkf5USPiebPij85KnzEwY581nemc9Tndg7rdrLu8hLaIzw5Yc+b7gTV1JJSneJ1XEDBQGQ6xsoWFv7aTURbBJIETsGOS7SQVh7HY28+Ru931QTuDURj0Zx3rwjNLYQ3nKTiqdVc8fMVf3nu2THltMcG0fRtMcHdwegsOo6/e5ze473cOOZG0tLSiE6Mxj5ox9BkoNPQTmvgH3Tp91IS/xA7J+5i1MujGGwexNhuJMUnn/jumwg2ymtWV2gXdxbdScq8FPw1gSwqtBNgzKcsvQzTmEA23LyB7Q9tH5qP52+MXPiOiEFzlYZBP3kz6BJkvWSFHq72g44ZYcR/MQer1oPKdWKjrPVH4iuLcAyYsKutQ9/CI//qE2RnlaHFgEJUYHQMUhMp55Y8MO0A2Z9PwGIWhmTm0PgforXJORFE6TtbaU6b5dWPk8r2M6FqMy6lBsuJUvy6Zcois9NAVP8lLDjZzy0rbyHpgIvvnjzD2FumEvhaKEZRjmxWDs1pa2Isf1zyB92h3XSYZWe1CAxmW3jgg/cwXRhD/OR4Zi+ZzafpFUw4WEpW/cf0Vvdy+K3DmLvN5N2bx28ujx4momSwuJEPEz/kzcA3qcipcJ8rGDWb3WNH06toJu7cFlTdn7M2NpbHLQpqI98lpDuElAoVolJBf3gqZh/vKCS5OFAMDiCe7WMm/1mOiunA/oM/UTn3ZgI723jmxWfwMfmgtocROTWRFy4MJsB1huMfH6fouyK33HNI8rxO6rifBGkWgqBw95NO6ev1HAsDQ1SWwpBRSsHteQ+xO9VKW2wTop+WgM5qfAY7cblELmk7Q27NatLK0liwIRVXVx/SedRC7UF/YFHKfa60O+j6eS8gO8yHS7XxFGvHP863txRgbOjFp6tR3rwO50qSFHSFdeF7ry9zXpuDb7CvZ90XBCRRial9kKwDnxNRexQQ2O17J7WR71OfUM/a+wYZrOvigfceQHAJCCg5kbqMlpCfaAlZg2AYJL6whqhmH/c7PR8CX4xt5+zynxEjQkg9105MUwzSZBiYA1MEmK1S8wOwE1gNzBC1TBUUxKS9RUz+THbN2UVtuidHYFCv/P/AwQn4WBMJ6gliyq9TsPZZyducx+bkjbQGrkMQBLqKt9L2czMo9vO8pY0fHSY+7ZFILv2eP0rmsS5ZIOnqCaTvyKVgUgGtg60Ux/wDp8LEyPKR3PnhxZSUKthqnE/h5/dw+Yhbh7QWkRn7ZjDu11Y6kyZgwfrfQlKb7WbM2n5U0yez8OsrEEQRg+4c+sjvqbBo2S2awSg7dKo2Vw31owdQteLrXP684U92PraTsq4ysr6NZUduhDvvsu8VslHCoPiOmiiP496/35/00z20fb6BFV8EkVKdgkLwOOUcKgdGPwv44KajTGq/n/zKP1ELvnyuiUAwrKbll1P0O9e67ws1zCVQE4J/vz8WnYupe5fQFucd8SogMvbEWLShWqY9Pg1BELgy6G3yK/8kyDgJwSWgbZfz83ZXdKNQKEi2LCO2awX9EVmI8+awdmsgFZzlX0tGWSyt765Fa/yrjNCatcTXxyMF6SiYVE1/QD+R2kR8bSmIooi9vI6sA5+jzc+EayYysvlF/M053nVYnfQWNaMbjCBw0JO7/ZuTP6B/Xc+Vv11J46FGpuyeQrYgG9DrI/6JgIKekB5a410EtFeRWOvJ3TOs1w7nZfKxJlCwUGL+mU5iuq7j18t/Zd+jfmQe+JzY069TEyk7BBPbZdBYXVIduCRm7R5NSHcIj0ftwi4MUh/+OQ90wa+TDqO2KJm6MsP9zAkV27i4uhGjr5FP7voEW2MXE456S82w/vmsvHwlWZdmETcnzgvQAnIuy8YbZVR9dH40XSYPtamAiNPu4qsJX7F+4np0Rlmf7tUfdl/jpkwGEmcmetWt0+lQ6VRE50VzKvCU17ke/QHOPfcDW+7dQrupnch3ItmUL9Lnd4SU1sdxKp1yzjutmpajLdTvrWfVJavYcd0OHp/6OKIoIgoioih6RWkrnLKD2VnSRfO64wiShEup4d3Sh4bOK/Ad1CHZHOiMXagtBgRB4JLQp/G1pLJg6wJm/HkA39RYmepSpUVznrPPaf4O/+46hKJiehUfu4/rrZmEk8UN39zAzA31jH9hCY3/EpS/t3ktSWc3ENp0xiu5Xm3Yx7T5f41dZccV4qK+wsH1t/fz/M+/ccq4FQXynJYE+xBFpwDiXyMQXDjdDhybqoPbdi8b7kjWX7IO05QoJJUGlMNglCGgdFcIM3bFYft9M9m73kff0yCn9BhK55B7OpdJ++yoLr+YzqW30DxqPiJqNDZZSnaHdHNqbC3amFD3GprU/gC3xX7EmKgx8jhZZ2Hj7UN7xSFnY09wD6uvXU1vnhn/jmp0hg55fR3Kn90StIaTsVkUv7OD+z+8k4B+Tz5wgMSmy4g7Wohhzmh+v/R3r3MtIT8RXbkfxfECYqrPeunDGnsEcY1x3PvRdYDAlD/upSFBjuY6nDGJ+vBPOT7+OPsWOuneepzAk95ABx+VZ030T/DHtNbEvhn7EFG5378mpYbP3vyM/NvzWX/Veg5fWUPo4ByC2pvYtmgVLftbUCgU9Dpl3cyolXWp2KZYah6soelgEx09HSDJdIDDZc7qOag0KhKmJ/B09dPM/24+PTt7uHrd1Sx8Tw5n+jv73kDTACU/lWBuNf9HNku/cD+WfLoES1ImgiCQWvgTu5/ejSAIlPxcwkdJH9F8qPE8e4CA1Srff+XvVzL7pdly7k9BILBPw30f3UdieRdCeSW/XvkrfbV9TPt2GjNWzaDycCUfp37Mqa9PYe234rK5MDstWFXtmLU1bIn4ib2370Wj1zBY0UawaTKhAx4muMYRSvxm5DPmxjEsXbUUtV7t1Z6a7TUcefcI70a9y9lvz7p1PdkZIHDoDRk52BeZho+PSJ5JZpBLqU6helM5mU9fQu2YS4ao/+Q2qS0G4kp3UvPFHkYU/oxPf6vXHjek5Ryj93yIddBGU04TIe+FsGLbRTSFfktD0PcoEuPoSsyn6vkfMazeIH/HIdmsEBUoHTbiS7Zz5q2dKGyyjLV3DSBWyPCBfweC2wZU1lTQa+ki5dTvhNcfpzHsK04nr6A9cLN7DyMAXQE7ONz9Jy7RIst7pRNUIgp/PXatHlHyjH21Tc3IcnksVo+9HENYClax1+v5zbHNaMu6yPgmw+u4VdXKKuFGds34HFGnQWUcDh6xD40hWSaoFqm48rcr8Y/0d9uFQM5p+udNf2I8U8W5GXdyZPJp9zkJFza/UG6ufZrCJc9RME1+p5mOV0npeETuE5OZ/pIWHCodkkKF67w0FeMLEtCfaeB8G7VzCIjvUDkYCDQg2h34HPUhrGukV7uiFEZ2Pn8b479Vc8WvV3Bi8dMIgsDhQyp6LL2M+HgEp8acIuvqLMZ8O4bj44+jVqrd9tLnZzzPouxF5N6QS0dRB8U/FrP3ub24bC4UCgW33X47XcCq/6K//7WsVGgJjLl2aGxD7MRYkmcn/x/t1/jX5/6v2vQ/Wf63i+y755572LhxI/v37yc21hPFFRkZic1mo6+vz8sz297eTmRkpPuaY8eOedXX3t7uPjf8e/jY+df4+/vj4+Pj7oy/u2a4jv9b/ufLg60PUlpWOpSU1qNcOiUHNW//TqwYQvlkOYfiMBd6QJ+T7Zd9SWxMDsoheb9TfIRM31jCu9IJbC9H4Z9Afc6FDAT+NVeNhIuDz+0kKzKZ4+flfhUEgSEdkd6KLqwGK28ceg2TVoadv1yxjMDLg5kkPEb+gBVTp4m86DxaHmpBEATsvXZOf3sabZCWvNvy3M8CCDCNIb7rFoJOnCTzwBdUjL8Wo69nkWuNbsWJnvPWHnebHUoH9ckKkkbFY+vo+AvVBJLIkhNOgicZ8Gny8aIuFRBxpaXT4QrhwEVv45OqhMXgHMqBKAoi16y5mt6IQe5rvo8dP+4Aj30Ni1lBQLysAAsKgXnvz+PTwdVU6D8loncpAXicTvtf3s+8LfM48rRMe2vS1HAu5np++MfF2P3UMoL2vH3gHZ/egV9QMd0j5riPnTPtoDLyOFqLL8k1iVgaJuFIkRdFheD5OL8H+LCtOQ7H1+X8uK6CJ295UtYscmGHA9SSmvCYqdzx8bfcUJlD3+E+Xp31Bg8mfkdmMhx84yCNhxq5aOVF+AT78OPUjxhX28c/715HPqcIzBYIaP0Js6+MFBwOvTb6Gikeo+bSkQmc3FztcQKKnoT3kgSv+b0GwLSnpuEa72l0ZfTLjHVOANLcx2wuKyqfAKQZkBw1hmnp37E38koK7AXgsYfTHdLNpmVO3r44j/oD9ZT9UYZjnPxt4gLiiPOPY0zkGPf1t2+83WuYJNYlsuvt7dTkXoJx5j2kzmyg2H4f0WIckaFJqIP1hGeFM5yuKk+4BaF+JtaYWZwLzqa3ooEfZq0nevls5j3uQb3c+vlSfIQv3X+be81IQzlDDLpODiy9mSBtMO1n29EGaQmIC+DSgR8Yq53A2NQ+nqUPRUgO0y4JxSfYB8noQXtetOEi2ABdF3dx8LWDnFl1hl3P7gUR8mKC4QABAABJREFUGsK+ILf6I1pOdOEX6Yel14I60IdBrYysdApm/jViXqobJKo1ijGnxjB/x3z6R7XQ4qNE7acmNC3UPV/btPejcxURNvl5FjXLm/jBIZCsJCroSp1Evz6axw5fROMvBVj7rRiUMUT1LSOqT94AJyWBbbCfUytPoQnQYMwwMmiTK5n/3nz2b9lP1i45gt0+xALTFDfI2ivWcm/qvZT+XoparyZlXgoR0hiiei4jsi0AR1Mpzcez+P2a37n0h0uJnBdIr58858Kr19O7zskN65exbI38HoOp0No1FAkcq0KhVVG3tw6Vr7whN5naKGy6A8W5g9hddi4riebyG67hlltvZWfHTu4qvg6S4FwSzDxbyvIJ+Zg6TeTemItCLfOkX7f+OrQKLStzV+JyuHgr/C1GXTaKC7+QI6+NHUZ+e+g3emZ0c7rxED3+vah//gF76iATrRNJLzyO4aFJWP08+V4EScmkit2klX2DwlqF7aal1PqO9epPlyhybubdpGWpCDhagdUnEIt/OAocrE2FvZEKvshXM7emnr4Ll3Gm9k+v+w9OO0hjWhaPjUmlsllGGPpaU+lGpsWMbYxlsHSQhBUJWB1WvrroK6IPLqDP7xgai4boQgckdGLXJuMULAxqSxnwGaJrFRx0bC5k6RofVl8bTlHEGoqcaxgZ9iL5p68irngL7T45jDtYRqdSpjA7McLj2JNwEV+4jsvKLuPXy391HxcFBfnH89HYeylJK6HHIm/mO5wgVMK0qCnAVnT9dtq+LidSJSftlj+qTNF00dcXoQvRYTPa+OzEZ2xnO2Ol2+nuGUFK9QCZvyZQfPnPXvRlANlns9Go2njv4U+wKQa8zjWEfYlJU8tlrrcY0DfhEl0cM6zjmL8W8iC6OZrCPD2Ty0JIPpWM772+bueXVdFD3Rwb1WFzuCYM/mx5wF1v4OBErs1ejvXA3dibTtO8aT2hV4dSE15Dzgs5xE6MJTwxnJnbZgJw+J3D7HhkB8EfZnJixOX4m3IB+MHnNCvmXceX+V8SkhaC7z03Dn1oeQ1b8fUKdvfs5srfr0QQZMeTwqXhwPQDpPvfT9OXTehCZGPkCM0kInuW0Rb0GylVSYj5GrT3a+nb1uf1TXycClbd/gSxo5rZk1rMv5bwjnAmFKQT8+DFbA14CpDXzeHSEtNCcEEw9064F/CAOwDCbC8yauE44l/6A6tPIC1ps2ixldEWJDvPBgIthE9Po0iR5UW/N+wUUhj6MPyxgbjkepgEg0NU6UrJj8aMeUSMCqXm3d8YH5/K92kF2IVaInovwmHtIqRqNfiOhUDv9oiAKEoEhPbj7Ja/q1appa00BY3NgFlfz1rtKkb8HEz6wnQGbYPsqJHD4/2NY5AEB+ZAJ6JSxOXwFt5Ku5KwzjDAROWE69idO9Hr/Kkxp+iPmsZrc+ezFZGOCO+k8WEdYUTVleH/xxE0SP+tHBX3Dlppi3wXoTUSo7ISrS2VEc0vs2z6RYz1s6CxPM/E7e+jV+vZ8Y8dROdH40R29okuDS4XnJ++oeCSXhb/804aw1YCYHL1cV7qKrZ0fElYwGpGBlrYcP1qxoS8xMiPVtEZN4bemItkIxcioV2hjKgKxZnRQIwdGoM9lIMGZQ0XbL6ACcdk/XVwQrRXDmuFoMSmsTHoZ6H4tQ2ExGbTnxuPJ2JCCRIoxiqYMmGKDL4S9fhYEziV9SNN8d1E5eloHjmDwaB4JAkGxaHcVYJsGBZ81Rh9Zd1IlJT06OUop8xzmfj8+i7xQFxQHPWJslN75pCNs84Bjp5+Mk53Ux8RR3NsM0wGZzZoT1/HPw7uwqG0EhIciMvnGmbG34JWG8lpNlE9oppQawBTxk/hoOYIh2lhQvkOQg1zMGiL8Rv0I/NIJj1VPZwZe4aP6l4jLuBmLCdqGSz/iUMSTPiX/IDXOm0cBRbWvczKtS6qGQ1DS1Vd6GfUhX5GdsmFGAN0dPZrqA3/kPbADTxwKJiDmUUM+pRyLmsEOr9ZSIKISqWnzcp/VNqB4AAHK2/9iZf8PnHb8NsC/2BE3GpSNNDS6ws9UP5nOUlzZAfRsC6T0v0l0iQ7S28LJzQ9lMrBSjpMbeAJDkVICeDgRVewJyfba3+hN+jJODuAI2eQmhEmBv0G3c4mgOC+PGI7JlD1cxXx9fE0JDRQG/EBXf47iSUTfZeKrJMDdM4wcS7LQ09lUTchCiLjj41n6iE9kQ/HMK30GAfTJ9HnJ0eLi4JIZnEmZfvKQA4kJEU9mYh+cChkKqWI/TITz8SHZFkwaeAdmmVMApIEZ/f2EBjpT09op9c3rUxtYHbjPGZsLqM6It5tiAeIao3i+u+up39ZFwVTKuj36+fhpO84djAHc0AzJ/x3MJgbwrSYCBbonsDVCuXRz7rvD+wNROGUiPloGpsap3k91+mEhNIEwg+Gk/lhJukXpxPZlMeRo/sIMI5DZxnBtPhp9NUlkHR6HXbTROqShlJFuHT0+hbQamlgaskxfGyJOJcM9d/wPgwXFl0QTVEbkcRhB5XHuORITeXn5ZtoiW7hsGENRdqVoIU/omBBykleqXyKsCYPTZVLsINLlKnrA/sIun4WJcI3Xm3Sm7NZnrkcy0MWWm2tOL8cyofWdQNNod/SEd5ByD9CuOrGqwhND2X/AQ+9WFvQOtbZruTq47IerDPpMPmavOpvjm1m3vZ5VLxcQd7t8r56bvJcdtbsZNFbi1iesxyAH3/7Ec5BKOl0Ie8BivI0zJ4Sx4k3TyAO2t3rVXXU63LEa18ggx2DHHl4O+017ajXqZkSN4X7t95P70+9pK9OZ9Eni+jp7CG0M5SusC6ULtnZZw1TkHTtVLY2ZFKW/AMl/cfd77tn6VgyJ+QwYuVLdEdnUTt2yCkmiZh9zJh1Gq+9yQslV7r/P3urP071BijoxW+qJ4WMJMj0j02xTYjKLAzWQSzn5QoEqHGcY/W1JmYfnUzOV3EUzZWZGYoT7kVj0RDcGwwH4PtRrxMcFsqLMfcQpIwiXZId8S7R5l6nJOGvHMPDedwB+nWevGb+ptEM6M7gcjnw7W3GofbB6hvszs3uN+hH5rlwXIm9ONS+SKJCBroNyagTeScwB41ntq8fysZ2XHotwYNTmXe2lYBxm/khcjEHp5Uy/esdRNhTaU+ZTFTfpSwKhVkpoRRvKcb4vJGTnOTCLy50yyir1kpVahUZwdPJOvYD7XFx7Js3wH26e2k5PIOC1PHc/tmVdARUcGRigduxrHT5MrX4FP59LoIqfmIwPB2jn2ePD9Dne5SQxoNIxiCOLryUfdmykArvW0JWw0f0J9/B2dxOwgLDUaKWc5ALEv2+8ndLUkxBa4sjcFoE7D3NAt8FbFsoR2lWlKlRW9Uk1yTTUdqBPUDOtz4M5BrWWSUk+ur7MJYbCYuOQenSYtaHk3PfDPwi/Pj1yl9Jt6VyJvcMdmUvoiDSGNdI/LvxlPxSQvl15fg+4kt4VDgthqF9xJkOdj+9m6yrsjBajYgukd7XejleepyosVH4BHkckeuuW0dAYgCzX5pN2sVpPGF6guJSWc+t21fH2kvWsuDdBeTekEt3ZTfbH9pOzoocMi/3GN6G1zNdXws9FXL7wrPCmfDABPTxgfJJl4uY8t00b4+EyVnue4fnkNHXyP5p+2mJNSMFpHPZnUEEJAdwZLM8fyojKnHpXAy0DvBm8JtMfnQyBTPq2Zl7KypHMHblABURTdz02U3s3w+H3oIu/53u59jFASIjIXZCLLETzmMZGyoPtzxMd0U3Rz88SkROBPXnmZElCe44cwe33qXC7uOPIECydRnHfJ9j9XV/Un//55RXipiGmjo8/5Q2I5E1h2kzBxLY0U9HUr7XM83+EbQlT8IvJoBeQcDqtNI62Ehv0k0A1FteJyhyOdMrwt0fOV03ldfmvMY7R94hzX4/8/qjiV+cxQmXTCVr3l3DNP4zENxUYKPuOHf36LD4hrjXZFFSymMdsCt7OZF4GUdrYLaqlrTmV/EZaCcyKRJXUD9qk0RD3Bfueh3aRHZfdgl3X55D7/d//2zBJRBSoyK0LxHmeY47RROnbb+hCg0k+Z/fcqTqQ5CgR3+Q1sDfMWor0Jq12I/Z6YvuIzBB/uBXZl7J2uK1TM+azjWbruFAUQCWg+GMbShgX3gUFpeJbv1+Asxj3Lmlh+WacF5clbKnk95T9bRlXoBTrcPh7HOfi6wpRLBEo0sIx+VwMRgc76YVV9qV6Pt90DZX8/Ol65k+4S32TfkQjT2KjP4HeDJsKYR2IjrH0Wfpoz3wMAqXL6GGmbzyqkSduo662XWsvW8t++r3ATL4XyHK9tJQXSivq+QIf+WTSu403skV71/htv9ERkbyzDPP8NKLL/5HVPkHnTYW6RfgRB7bc1+f+2/u+L/l/+3yv01knyRJ3HPPPaxbt47du3eTlJTkdT4vLw+VSsWuXbvcx8rLy2loaGDSJBmRPWnSJIqKiujo8NBD7NixA39/f3cY5aRJk7zqGL5muA61Wk1eXp7XNS6Xi127drmv+b/lf774BPqQlJYECG7lcsqZb0k7HEPEknxaR0wFwClYsSnlDZvJV0VgRiQhzWeJr5dRu31CLQbtObTGLhLObab3TANVka8xoDvNWJXH6HPpzgNc+Ods+ifOZ/fco+7jCR13oBZ0FI+xUnDRQ9zd+Rwp81J4Zt8QNVzTG0T7pFCbXEp/tJIlny7h/tr7USvUROmjCPEJYXXLaoL3BDPlmSlYB6xILsmtzCKJBLRXsPmODQwGxmLxDfGcA1Q2FS67jZ5n38e35gH2jcqhKP5umkK/w6axcWS2ksCJ6fj32tAbvHMTJNUmkLPzPfZd+wXjnhrHvqX7+GyCLPTtgon+7CBqsnIInZJGyCTZQmEQmhjUVGJ09rFxyUYaRsZgspvot3hTgDb6ruOmb2/i+13f4x/jT+SKSFY7PgWGI60U3PrFrRRdUcSkhyZRftN5PMouOYdMzt4s8k7keUVyAPQH9OPQq9AOdqEyy889aPyG8tinaPF/hdGHT9P51Z8MPjZIeHs4u2o9czM/r5KnV76G39R82ZD5L2CJKYemcMsX01H6ahmVI8uA6uOpPPssPPr+cQq3FVKxoYK3w99msG2Qwdo+ABxiIx0Bm7E5bZRf+Ajlk24APFFuJl8T58aqmP3hUg4vmkZ7pKzVOQR5IyA47QQXemj3pvxjittAq7FouO+D+wjd1cT6G9ez/xV50/uLahH7skbRHNvMvhkHcYVHYomzeBnm/EWwj3Lw6oKXsPm5aD3ZSsG7BTg75feakzSHX2f9yjWt19BT1cOhtw4RvjHcfX9Cx91MnDKH4JunYAyKxeoXQpuqhbqIjylO+4Wq8degz01hwy8Wll1oo7AQRgqLSOq4D3+zHBbvkgS6+1Vs2yHQ36tEZ5FhpdUpzZiiknlOeo7npOeY9uQ0tyLToNjPrtw42nRb+Wz0Z+x4VDbsusyj+LnkEd7seJDcmu8QkpOZ/vR0fIJ83AlsXechSwveLyBmfAyZV2SSrl0AyMaEuTntfDnuS45/cpxPsz/lwEv7aA+SeRms6hb+lZlAKDfSGN9Bc0wzJTl21CF6Vs1cxdb75YjX4Tk5d+dcpnovF3SZuijXP8vBzmn80nEHm4rnMzcnkU9Wf0z6Helem/fIqoP4rf8B/xh/Hut7TEYhnmeoH/AfQBOh4ezqs3yZ/yX95fI4MgTYKB1Vihgssvnuzex7QZ7H2Y6biOu6iYNzFfS+chGxE2K54vcriJ8W79XG2ForpkLvpNjDzw3vW8zksoNIEjxheILLfrqM9955hy2bYhh5bj1vOLpZ4xrgHzVl7HvxZRJiY9m8apNXXZIoUfprKYZmAyMWjEBUiLQNtrHm7BrWHFlDV3cXkiQRNSbKDRIAWLX+Gz4S36f56E88XFPJyoEunuuoJL+gjUMn9rMz+gwqnQqVxYBol62gp2Jv40DGeA7PjcN534OcR5UOQFXka2zPDacrpAdJpSb1+I/4t/zO0dQLsGnqWeoHS+pTeezNxwgoK0B3SutGjnu1SXBhlzzo8WG6G4B5O+ZR9UgV5V3l6F7VEfhKBAaNLOM0Vg2JO8FZcJxWv/cpHHExBzLHUhP5NnpTDtn1n1P/ySb8B0QvtLskOJFEEZdSg9NoIaTDTFTHVK93Sq1IJX9/NBpDJ3pDIAD3xqzGLg7gxM6PV/9I4cwkrI6/WoyVopKolijiSyz0fF1FZJsHrOQQjbzTsYBHxUfZ+cROvhj7hTs6/aG0H7kl/gDtEe2Uzq9m0G8Q0elRJbW2WKYenIrvriaCpHwvqq7hcuNn+fQeKeeL27+gM9zbwNoS08LGCzdyMi+IPy85i0Evd+jUkuNM617F18KFFCc9j/8F2ZwYJ+cUuuCEmallR7g46m78T+0nvPYoiV9NpiG+gfSydI6/dpxndjzD6bbT7udE50Uz6eFJECQbDgRJdG8+XZKL/Lvzybo6iwbLOdoDNmPSylTcBRMKCEwMxGqwnjeXhyKCcXF3yd3ceEB2EI72WUJ2wycIksC1a66m/fdDXvN7uHw/KOFKr0ZpdGD9sxWFYwhR6JIjUmqSa/ju+m3osjwQfadCNnbGdq1g3w37uDTD4wB2DL2Y0qknvusW/JWh+HdU4dcrG/3OWP6k238vANWp3eS+fhUW30Cvd6oP+yd7MzPRnZbzSRyfLFM/3pzwGm0hq7CG/k7XiHycKSOpGDeHvTPl+hJbHyGr8SOONX1FUlk1voWFLF23lOBuT24dSVKT1zCO2GNBSP1WPkj+gJdHvcxP+om0J67ErG7ks9pX+arjK7rKuvhh3A9kFQ0ZbwQX5+Lv4Y6BdByT5f6qeCrOXXdwTzC3f3E7wpFqDKGJ7rV3uLgULhxKF8Nes7B+b/q+7KJsRpw+giN5JC5B+m9FVjkBe9Mf9IWEEvJGHJGG65CQuLsD3u6Fe7v+4LmK5/AN9+XwW4ep2lrljuzr9SuggQNe64PBOsiIticINsgU5Gapz4vec/9hCw7RQpIKopSAWqQu50K6Y0e7jUGCJHJoyiE+emgvQncXc7b64zd4XoI65HWmM0JL8mePouqwsHDLQvc5haBEY9Hg368m+4VltCfJDpNhQ0Zs93W8fPpleib1YO6RncVzgm9iTlEd8V230B60nmpnBa0jZ2IIS0aSPOunxqIhrFEgbFEWX972JZIoUXgekEF0ibj8AyibG8GJvBMonHJky+0dcMu5IKZsvACH0ULB/Kle0YgAzuYB1NZBOaJGclEd+Sb7MrNwCmZ5rgNWHwG/qblkRt5JTv3nhBnmuiPsWqJbWPnsSsbcNAZHnwOlXYnLaKO87DG2uuz/FtG+1WVn5Scvod8nMqJ2odd5f+FJ2iY/g6Omnmlb2uj228XWxp8Z9JFpqKpSq6jL9WNkwfcE6wP54j+E5H6ngpunAbhwHD3Bnkc3D52ReLATxjQMpQF4R8WjHY8y4d4JQ2flNcek94f588m9IZfYibF/iywe/PIUMVWlfwEStke0s+2SYCLuWsauha20R7azzfQqIEfwzS9YxaVrwjl651Fm7Znl6SPBiiCIGPQGivK0dOQPegFH9mVl0mvvoDK1krJsM437alCZB0htlQEPAcZ8RBT8sfQPgscGc+Q92Wjr1nUkeU099cwpLl1zKbNe9Dy7MPkyuhyLGPz2QyI3fMk1P14HwN2pbzKlVI4Y64joJuiy2TQn+Lud0eGSrOd2h3SzdcFWWuO73MCIopoOClLnszU5k5Wq+9m84DOsJTWYDhRi0JZSGf2S+/nhHeHkHbZgKv1rXlsRFQn1CURsjUCpVRKeFY5TNRxJK5LUeQ+vd7/OJd910jpiKgeneji9RElFVeTrvFp5Jf26U2gcYW6ZIiAS3h5OULWdqnFL2bTEo7fVRXwEwF2f3IVy7VoaExqw+FgotexxXzPdxxetCH1LppP03jPuaES5/QKCSyC4Jxjt2GS3Y95TBLrKungr7C2qP61m49Ub2XD1BnLrviGsfyG9wb34X+NPaLoMnFQrvKOYqpWbufnIzYz6dCZ9gX1/+WYzEmYQKobSWdqJpU/W04w2uc90Ko8+p1XKa+po6Xqiei4HoHakGp9wPSdfPklgX6BXvaFdodzzyT2UrznFtKen0bC8gUd3PMqRpiN8efJLjnOcpKVJGFoNFDxfQHiHvJ9SuORn2iIUJNw4B3NAJCZ1nbve1JbniO25FgToypxOX2S6/C0luZ/2z9jP0fljaP/jCJn7/onC5u3cPDizg6aMeXDVFRRleWRfefRzVAvb2DlvJ0V5QWy9+0vCGr1ZflwKF1WpVUiCQPg5vRc1rlVrZcOSDdhm2rCERGPyG/52AnpzJvmVG8mp+8o9pqTzUkAoHbIu78LhPu8UPTqzICm4aP1F6PY2k3HoK6LL9wwdVyK6tKiV13Jq/j8wXnw1pdNuxRgUK4NKh9aatqg2GpNcuErKCDqwgW7XYgpHLAVkpz+AKIGpth212WOjWL8e5kRcxawhNo6bDsmODteQoBBdGkbXfsOiuBuoy7mQLbNeoST+QXYNfoiAAqfCyeHJJ8h4ZDHbFmxzg6JUrgD8rKm4NKlUXPQw1uxo/Azea6td2cent79F8aUazL5at5PaJVrQ2RJ5aMxWbLEf41D7cuDXH/A9b++htodyd/2XTN0Xj7VFlhNjTnmAuoXHlPgN+nHV2qtkppw6CZ1R53b2CYhozVqiKqMoeL+A1QtWU7RH1rcrkjby3dhvKXQUUvxzMReYZJ01aHASn87+mdeufo3p108nY1kG6kVqHEoHobpQ97NDOkI48MoBPs3+lMTvE3GJLur/P+z9dXRd1dr+D3/W2h53d0+atqm7u1CkeKFQSnGHgxaHg1PcoS2FUqBo3TX1pk2bpEnj7rKT7GTrWu8fK9k7m/I833PGz8bzPuceg1Gy59I555rztuu671OSi98MeNOtD0p3lVJ7TMmuEFUiap2amNgYBZnmqSUsKwyPIOW97WY7pbtK6ejztbRebOXna39GX6Vkl5+b9SjX/Kx8t9Hjopm7ai7+SUF97ysTXppN82Hl2J2P7SR3Xa5zLtp0nuydsZfuwGSkgCAGXTMIrZ9r7i86tYiHr36YpJuTGL5iODl+OTx3akXfHFGOs1r7/A4SWNQuXzLA0GPNFDy+hv9OAlMCmf/RfGInx7rZ5JKktNkMA5DrkmIHONQ2esx9iW2y6/4AZq9gzk17gFFf3cnpBc/RFejuGzf5R1GTMZvcT4+SPTgbjVVDu8VFrxx/1Id5G3uJfWQxXjcuAiBWP4R7R91LS08LuRHrKR88h9TbJ5GUpswrQ2037lXa/2tZAVjbD3N6/kqOTLXR4K8gX5U5qnyzFyNedB6/d0g8pxOuZvQhK6pv99L99Y+kHnXHk/mbJuBjdqHky0M+vOS+9358L5pekQernsG/2+WbF1Dj1+7HHZ/dTPu2Y2yVH3S2nU5ajKc5mR9H/UjvQ728H/c+lk7Fdt5w9Qbk52W23rKV5PnJaMKDUFtM6GwGPFTKulcQ8xCyo4uin84SUbgX/xZl/OyyDYegrBlm3xAyX76WzpAkALekqP0LIumYP4Go3K3En1XoevvrD4fXh3P3p9egaW/iqu+vQvacjNHjNLWB33Ex6C1ea67htYW/oZuzgpOHT9Ggu5tzcco6dyHfZZ/3dvTS8k4LL+te5oHRDzjBEQMTNKRmiQ7/Dr7o+IKiNtfe8fyLL7LwssuYjRLQ+zs5DixQaZjmvRi9XrGe/kXmz/+/FkEQiI6O/r8dsfevyv+YYN+9997Ld999x/r16/H29qahoYGGhgZ6e5UPyNfXl+XLl/PII4+wb98+Tp8+zbJlyxg3bhxjxyoK6ezZs8nIyODmm28mNzeXHTt2sHLlSu699150OsVxd9ddd1FWVsbjjz9OYWEhn3zyCT/99BMPP+wqYv3II4/w5ZdfsnbtWi5cuMDdd9+NyWRi2bJl/+93zP8SEUWRwMBABEF0OgpCGyOR/zyIPtyfzhCF37fLcI66wB8wWGKI7XiB0a9fSW3KVFoCXYqfLNjp8QmnaOxSZARG7+/E2yjQVZnMvNM9zM3pxq9dQ0j1RfD3d6O2yqz6mHzLdrKTZmDSF1+yiHlYEvBQKdkvA4N0/WKX7Ny3/T4eOPAAOatzeN33daqyq5zHmrU1tPqcxW+4P7Wp07B6+rtd58H3H8Rj3TZU/j50eTXR5XEei8ZVOF4W7HQcL2LithwWHdzFso5Kp+JnV9vp9RAJm57OlBemMCp5FF5qJQjqEMx8pI0hL+YexLtiuCXgFgCsQif7B6fQaW/l7LCz9Bq0fPbCZ5zJV6hQdNZwEhoeocNRj/oZNSU3KNnh35/73u2Z1IKG5tBmDEkG4qbGEXGViw5CJXlg1pvZf80hQuq7GJo71K3PfrjxBxrnR5N+8HNiz2/Gqm6h06EoWZ0+nfy5KBttXDj2Qw48ehSFcU71UdJqXufPzgg+vHgtPotnsePxHfxVaqJqOJtVCSoVdotitJw9K+EQelnf8AyPTHqE4DeDGXn3SLSeWubmLeeVZ16h3b+dCZt3c/qpnxBVgjMjyjGgHpssKAaPLLiCcVIfFZ8gSfhfOErWbVk8Lz+PzkfnDF7JgoxFZ0FSCRT8UkDlgcq++TRgsgkSstXGBM8JqOwqJkROQWeNwFsESRKYIBsoajrNsNuG8WDFg9iTXajHhtwGfr/ldyoPVXL2m7OEHFSM05TaFxlc9RFPXfY+sYtmYtX7oDF3YTcrzhej5ynOxN9Ml72NE0veJ+nkBt54w7WRG4wN+NflEZARRsGUu+gIz0AyezKsXJkLe2edoHmoK8VKEARkqc+BIQnEVsSib+9l6otTybhGCbz22rxYl/sW5N3Fgg2NSPkuOGl/f9RH1PPKM69gu89G5vWZjLpnFFf/eDUWlblvfhnQBvuRcXUGLYUtjLhrBAlzk9zmwV+RfeFXTET2XcHglgp6Yl5BH+7PtJenMWy5Ymj1BymzJ2RzcoIDU20H574/h7HKyLvvvU7ZhpcZfOYo7xjL+clSz7PtVdTl/ERsVBTrNj5DbcAPtHjvQ9fdgKZOGV+9rx6Nh8YtGPDj9T+S+WYmtl4bpmYT+e0nOJY8m0b/353HLPpmETP+6UK9VgetoSDmYUrtB/GO8Cb9ynR8o33d3vGb274l9KnbqM+pp3SXEsToVyqHn0pmyIHfMLcpjpJV77zDa888Q7YsccCu8PHPQPl3l93KQYeDnR9uhCNK/YlbV99KbEEel6++nHkfznPrV1ESefKNJ9l+13ZUGhVL9yxl8srJzvu8+cJzHJQc7DCb3e5z0CGTLUNe4yY++PB9hu5+l4ji/QB0GQowep7EqlUKmycKpYRf3I/Kpox/YdTT2DStNHusRrbbKR+6iF2T3qDZdzudmkZer4RdBdV8rn+b3y5cTs57ezFs18OAoGFceRyDznjQ9dJPPL/+fUI65tPmfZDEeoXv/vDEw3gO9eSnkJ9Iu5CG3aJB7gv8mzxNnLzLhpRfyIiDlTT7uoL83foCPC2JeA2OI29oz18CXw46gxMpmHwnwTfP5cy8pylLcXf4xFXEkXUinKYp17Bu6c8A5HRtYcdwX2odZ2kLbMOnzYT+5kZSihTaD28R5GTY67GZy7bcQtI5G+FfuNM+B3ZNocCyk91lu0m4LIH0xelOiqrjrUMI0XXSHdKCWd3LbatvI7TRVcM4rP1KDs5owXzDCEIavN2CPP1SH9GBPib4kt8HSqtOS0t4MFopnk8m/YlfTxYRWiNeXV4ENhuw9JoVWilZRJSV7ENBAHVXG7reDlQJHpgNZrq8u6iJqOGXsl/YV76P71//nm1vbyNuahyz354NQf3QFREBkSt+u4LcublMeW4Ko+4exY72TziZvIDagB8A6PHo4diqY8T9I45HD6wgJ+F62rwPcPmWp4nYeCndmyjr8DONY/fsAlS+nojPiJf0yRfdEj/c8ANWDxUpGyMRJZG0mteYn9PLndqDmA1mWgONtO/NIaHUnZPLuzeToJNBrA5ZTfab2ciy7CruLYvEntvEN4mvcm7GQ1wcqzi0B64xsuCg8qfjDN/6GlHVrmzk+oCNdBsK2D/qa3yumk5TiOIgKO0+x8Rht7Jz9tUwaBG1nKQtPIrqmOq+69mYFLGd4enVfHDfJ1gCfcnKzXLuzWHtVzK1fR3lplpWvP8gXpvqSJyTSFtqG0XdJylKeJS6wB9Q29To2/RYjBYcVgdCfyBWsDl1moobRU7PX0llySCCdQqKvNurm90zduNI+fv55WHyIKhRxHi8kOj87QwvfNWtPS8zj4Ixk7BedhWyQanh+a9II4BGZMGWhYTXaXl0yJskND6KLNipc8A/WmBfLwhGAVuPjQfKHiDhHwm0SCUEdk6nJmgNO8WH3ZB93138gH2Dk2jzVpI5emWjG7LP7pDYYoJ7Cn2J2zYDa2kjLTHDOZ/+B6eDH+Jw4G3kJF4PghKcc6QNoizrCgJ6lrg9e/bEbLLnhmOpb0VX3uZGUzYx6Eoy8zJZuno8Gl8PTL79upuy/4Z1XM7cBUpAa/XE1Qpiv08n8OoUGXNsDKILEKXoRH3zL7A1kJm/+dC8z5V4MjBT/vyQ83TfcR/1gwz0evQS0D2JxPon2dgNuxq8GXNiDI6mdtTaGYR2KQwFnvsSUL0rYCv5lVdo4fueBp6tKWLckS6kD1rZZvGgWrubG7+/kWmbqmj/4yCO9z/Cp6lkQI8IONQOTJ4m6nPq4XIYnjMcU0ERkwTNv5TRPgkR47n3GVww4S+tIrIM9vYufDv+PpInI6HtNZLoE0O24792rPTLcWCfXWCLbxCjj2XhOHGavNWnlGsJEsEq2BauZe7IRFIfTkWtd923374QZFFBllodOKyOS5ISJhWcxnqwmqjiwkuSYewaO1rVTAQPA5LonlgyqPo9bIZw6pIm0TU+gUOTXIGpHn0JBeIGur27Kc7Q0uthRmtxD/I09VZTGVdJS5iNw09swcNY70z4FGQVAiJdPl00H22mfE85Z+rP8GnLdRRFvIBV3QICBJ4O5Ncbf2XN5DUAWGQT3YZ8DD1mtDWd2DOzOJelTFKNqKc/K1AWHHgMTaZo9Ax87bdxheYT7nCcI6BrEp2+nRwbdwxdg8Q1P4wjqDmInIpiWnx3YVMZnX1r2nuczuzDNPj95vZeNVE1nJioxWG1X9KfoqzmwJQD5Lyfg85Hx21rb+Mf2Tf2PZOyD/vF+9EZnEhzVCo10UoSh1dvGnpbhAtJIEjkRz/M8uMj2TxSoDpoNRMPT2T4zzbg0powEa3XI3lORY6IRpasIINOcAUvKmwmVq+6Bum3D5FlnPvuqaQrOBl2JzqLjrs/uxvjb5fO2F5tJYfaDqH2UVP6eynzk+azMGVh3zgqz2u32dl6/1bOrj3LkiFLGBPiCs5Kgp2osVF8vCfILYEoyKjYFCPl0ey8Zyc+kT60XFAoQPt1FvNxM8feO8aBlw5g+ENBHPXIrdhUHc5xTroikzl75lAf7rKnQdlPDkw+QOCwCFIWpNAxTjlHLaoxqA2UJpUy7qtxjHtkHIv2L6IsoQyVrCWgawph7VfiLYTSdqKEiKK9aCyubyqicSqB1Wex1rbQnjWN9ggFxbS97SNn4B8kxUaSpEv8CaVpZjrC05HSUmgKcz1zZcgnAARbR2Gw+GA72Ommnw2UmrSJbH/nPC3BLW6/z90+F7FHpHruCgpHKfa4gIjWEUCocQGB3ZNcwb4ByL5+qr2BdrA0INjXo6tgyLkhaMo6qUmbSVukksCTXqvoOqkNr+LQGrA4XOuT3f5XX4qElK2gehPKEmj026TcWxCJqI1gzrYsYh+7hupMxfYweuRQ2HuAL9a1MvbBsdxXdB8RIyP6nrMP1VwxkcvX1eKzVaQlZjgVCQprQpntGIKsQlJJ7JueTdCUDLeSFzJ2SkPfpiDmSYxebQR8uZ9Ffy66pJ+NfkYsHgLeRjXhTUriTr/zv5+lyVBbiuW9UoJaghAkl05qL64gtPw4wTfPofxFT75Z4UK8qh3e3D/3flgJESMj8LvLj+Xly/Hpzeo7QiC0MZQrv7gSm8lG0TwdK099iElXQrtXNr9XrKVIVcRzjucQru9HejuYF7uYe0bdQ2JAIkOXDsX0qAmL3sLuMtfeXJpSyvKjy0lekExLVAuyKLPDovhYBNHdof1Yw2PcvEvRPc1GM7XHatFatYiiSMTICG7ZewspCxX7JHRwKCvNK5nwuLJvmppMFPxcgKbLvf7nQHEhqUTOT72PhLtmI8syR985ysVNF506VYhRmRPevZnOcwZS7/aLLkjHZV9cxpM2Vz0zQdYwKG8Qd758Pbse30XrsWKqg752O0/lsGPvNrP1/q2sil7l9Cn1S/WRaqqPVl/y3KDoRd2N3SQf/57QMiVppVGjrKN+bWoKdtZw7KaPSD+kINycyFqVGqunP2ovfd+D/n0wwScugMrISrfkaKXPZGRReZb+b1oUXcHzbu9ummMH4Rkd4PTXyPK/mQQn21HZLdg17oH/hMaHeW74p3j3ZrqdI6l6aYgfiWbGZLRjsmiOHYko653tKoeId0sFbbtOMXTHG6jav7rkvhdTLtKUZMFuF5xsb/33lQUZlSTSdSSPyJpIt/O8zRlMnzgdlVY5x9bTx4DRBcePu/rd1tRB1q63Cb94AL3KtX/bKeXgw38QUXKIobnJqCUPDmme5WjqdOXd9B7IkkxQVQ6Cw47d4RqPw1k30+BTRn3yJGpSFZ9OWYVywy7vLo6NycUUlUrj2EZ2jhqPTd0BQIf+LMt84GZvZRyLXi3i5nU3O5G9/aGehZsWsuPhHeR/lM+o5lHcPepufi5Q/AYPbn+QqYVTlWcslNBatBh6DHSZ3bOnf//zTx55/nkmCgJTENyo8qehZiIiC5e8zvB7f8LkryRhyjIceu0Qh984fMk4/W+R/hiGKP5/E3b7H0Pj+emnCkpo6tSpbr+vXr2aW2+9FYBVq1YhiiKLFy/GYrEwZ84cPvnkE+exKpWKzZs3c/fddzNu3Dg8PT255ZZbeOklV62a+Ph4tmzZwsMPP8z7779PVFQUX331FXPmzHEec91119Hc3Mxzzz1HQ0MDWVlZbN++ndDQv1eo/iP/18XhcFBcXExMTDJji3ciCTZs3sno7k7Bb0Qg9OnGTiOwL9PeOyGE+pQQTH6uxVgW7Ni1WtpDQrCZ6kko0ePR44EoalDJBpChOXYkzTHDSYgSkXAZr5JgRUBFRG0ESWfzyP9WZtTNaQOeVEIURO75+B48xRpKJpfQ29aLbrae5V98iKcYiOgQCWsIQzVMRdayLLwjvBVEgSzS6LeZi5OexzPtWizfKYVM+7PaQHFEDfGYRND1i9havxxQKE4ARIfIqIMOupPrqE2ZitkzkGAC8bUn0aEppDqmmlUPPMSaCceZOnO00hcDDPqJhyaSdjEW82R3h/JACWxspeufXQQsC6Dbu5uExkdJbHwU3ZBnOD94L/7B/lRlV2FeaiZtXBqF6YVIgp1PRhxlsitezqLURaw6plCnqh3eyKJMXUIdZWlRlxhb/l0T0Hv50ZA0kQvxazicdZOzgLlFb6Ewo5zAtAV8nniKisgKZb5gxc80httKBeKK7SzTdKAOv3S5K04ppts3lKylH5I2OY0T009wPvZurBqX010/U8+CYQuU/qrDaWg2BzeTGuqJR3MVkk1Lr08Yg/RzUDk8MPSITNti4UDPn0QX9+ARrdDOSH20qJJaS9nlD/LAS1qeew7Gj3cZHladlc/v+pzFtl/5eP8dzufoN3hSilKYkD0R0W8rvm/6ErM0huGjR+P/x352DdcxO9eLSW8/gH5xGbrFU9F56/Cr8SPOL44gjyAi0iK4/o/rCcsKI2lOEs/vfR5KXUaZ1aroi96tFaSc+J4NHn/AcAhoDWDQqSbaw8tpjRyC1cMPhwPqpXOcSHqa0Yf9GJSTTE/LQ4Av3boi0n9MIzroNmbm1imZcR7w9fivqTlaw4pTK4jUDiK26S7q/L5m2Zpl1CY1M+VXlyPynM9r6LyrgUWYvQLRncrlq/GHCXknhLFDxnJ42WGe3vs0BysPYr/cTtyUOOe5FkmZxyrJA8nTG7vFzsVNF5n97mxi5sTTX6JFRkaSoK0NXnsN5s8Hj5QopAJ3OpaxD411Upf2OzpLkkuQvey0n6/mwHO/0bPUxA8bPlAovv5iQyzBznEHzNvwJu3T7TAejqVCWs9ynrU6aL7QjFeY1yVONqvdypgVYxixYgQv/vIDLW27GJI7hLnb52IJsZB8t6uQsUOSEGSR4KZgvExWZEl2Gl4DDQuzvgf0evY/v5/SXaWsNK8cgCxVoTUbkWwS2d9l8+QTT3DQ4fhvkQxbbHYm7AI5QyakKQRjaDffz1OCvKJG5JneZ7BLdmRB5vTw04yZ4H61hoaGf+k+2x12Jr/+PDdErcURENuHElHW/ErfO+mqjcavtp3Ii0dpjcrCodHj3TOYLo/zLP5Oj5y8ldaU66iOrQBAOALP7obJKhNv2SAMxbn/eQEcyQPHLGA8zK+4lZAjICYb0AX50uS3FUFWMbngLK3eeylOOUHIghAMegNd3l1YNK4EEYfaQWe4He3c2RTaP3Z7p36nXPIrt/DBTnc63YFOlH4jTDXA4AEl2FyZoWeCzcfpZDV1uXjX9L16ZEHGliI6a5OFiyL9FHymiGXUhvUQHWekt64XrS2YsRd3I8pa9memc+9H99I5p5Nrv7uWF1a/AIDZbiDG0IaHoGRanxzZS4+Ha88I67iSoK5phA2HhU8cojhZzfkJ45k6RWTjmR00+m1i8xU5zAl8mnH7xnFuyDk36qOI2gjGHR2HPdnEENtnAEy7BWwJX/PA8Ae4+avpJO0bScPBX3k57Cf2h02kOmgNEW3XsrrydQ5edZrossdI6LUgOkTMejOeJk98jb58fPJjZr03i1B1KPMeUwx/h+xydsuCjFlvRq0b4Azvaxf7AkwlSSV4BXlh9DWyvvAr6IvbJZYmo7Y2Ura7DI2Hhujx0fQ4OnGIvYwp3o5a64026AjtO7s5/9Z5IkZEcOWd+ZwT1/Dr6LeRgefH+9EwqxC72u4yUCUR0SHiZdLR/P1u5o9fxkeJz7rNIrvZjsPiYPcTuxWUeN/42tVGWgJbyIiNpN+B3aW/wIHup5xnR1UHU5l9FKvBB4vuUgRowaAC/GIm4bigtLU1a7miD0iX9F00PRHHqI+63Hm8JFi5ddDbdOtsbLS0YBu2kKf2ziN/cz4/F/5MYNc0KoI38kpIIw0zDmOLvoWFny7kyw1fQl+8udtQQHR1NFd8ewWHph5i9D9H83DBw862gO6JhDSGEPKLDYe+FFkcw6jAWWytW0OPZw+HJx1mvHcAIz98EUPZNfx87c/O50suTmbh7360ZhwntLyC3Ex3TvSm0CaaoqJIMOgwZcEXR5SEg/+TfA44hgmsS1nL+JzhHHthG/sybqPHo4EQlYK6L7NBxNMRrHtnHXefu5sz5WewY6bdZ6/ybkK9W7Dv08IXnP/vaxqBl28I7raizDg9DGvzJvTQJOxhTTgEC/kxDw48CB+jD1E1vkhR/rRFheNpz7r0BRwSTV8pjtPfruzTKSU9owJmczrsT06OLmNssCsIOJCNYsSKEXSUd3DuO2VT7d9r/NpUzNw+D5Ofg9Qjq2mNGoo8briTAcHoa+T4FCORvXZm7lnCwcm7sGrcs+UddiuaHjMaT01fdr3Ech94Z3ItV0e+zXj1clRHelHbtQhHQH2okmxZZozdHX22xK4ExWbvgU4gOnwcQq0F4299VMyWbZwPvQOVrEOQRYW+z+iHSqvCsdBBU2ATHDWyXDLzr8hyyc7j+koiYzVuv4u9hfiV1uF3w1hWDfnJrXY2KHr4qNxGCiY/yuZxDyEcUZ55p+PvqZOOA3NFNbGD/oGtYRcLd09FfdMkbnwyhpNPKiOlE8Bs9yBh6BimXnc51UeqCUoPwjfa11UDPeAqMn+4gVde1qHz0TEiZ4TzHlpbCL49w0l4ypuyl9eTfiHdiaoGCGu9HP+OaExnLjJxbwj7pgS5BRAsngHUpU3HGnqUsmgX/fBACam3MnJdDFVXpnFuqAuppBKVdbchwkzMa5P4uPZymoP6E1MkBEHEw+TBjN0zGDViFM9lP8ep3p8gAhIaFGp1U6iyvzScVUL3G0LjMaua2Te9kPDpDxNpmsNZlDXxvaIHCAlfgGdvKoPyh1L5+Soiky/Hp/dNRsfAUWEtbd6ugKXaJBPUEoTGpqE0woVoiaiNYP7W6YiTB/Fh7LV0+HcACg3xsPChHGANKodE+wd5BN8aTKWnCwknosFsMGMOMJP9djaxr8TieY+BnpBuJ71fxuIMStZkOAOLAN2GQvQ211p/PvbuvgZXP+cMz0GMzCK09ARZqizODjvrbNM4/KjJvIZAywmefu1B1tyyBlWqK9g3vw7mdamJKhdoOnWEFONLdOsvkBd7D00ee7Fj58SoE4zeD1M0MzgwwUV70eS7hav3/czDgx7GftTOS/tfIrc5l2afu1FL3sSXD6fx2kaqy6vJvCGTrFuyuCLuFk7Xncau7kTGTvOFZjw62xEkAVmUESQNYR1X0KutQl+np/Wign5qvdjKjH/OcCL5s9dmU7y/GI8gD7y9vOFWOCq+DX3L2dh93Wwp/41xP47Aesg9CNrt3c2+6fv4YKSSCGOTFBtOI2owaAx0WbvotfdiCDXgO8QX834zGtmTiPZraPM6TFdJAwVv7yQC0IxxrUlmYTfxuRrMybMQ9C52lWqrQu+eVJxE8vkmgp5ZzOauKTgE9725P+hrdVj6iQUI6ZhPk99WQhnCDQc+prfBiPhnLLtOv6DMK0lLVOst1Pp/w1OvPUV7+GbMM5ROCGtbTEPAL6jsKnTBEXg3WGn2r0T27dN//oINGLhPBaniMVtkbLLyjA7Zhezz7s1wHmdTt/HGE29ws+fPNB39axIEiA4bOlMrYrmJsNIGmiMTsdvDnAG2GbtnMOaECHGxtNDGtnnbQBbITpuALNURbPIkoSQMS5PrmzgXeydGz1OYzn3N1UxAa9diL7QTOiSUzMAshpavwa/Di84gC/owP6iC5LqVFEe8QpFlH1KS8r3JOKj+M4ebf7uDbfOzeW/un3zy5zGORC9BdIik1c3CPCSSguhfL3kv705vDB0Whu7+lqDKa1m95ADt3tnsyPIns/RLEk86sEfG0PuoP02qJvS2CHp1lQiyiOfCqRxXT4BnV5E5Kp3W0GykCjuyYEeQ1bwwXw3zwdRsYvJzk4mbEkf5KiVwp3H4kZ40g657uxh952iWfP8UXYZ8fIrmEV6jY8bmW5H8JYRJAipBsS06vE7wZd4qxhxOIf/+fOZ9MA//OP9L3qlX10vU2Chu3Hwjr3z8CrQoPpFjLx7j5zt/vuT4fqk/Xc+3M75l+LPDmf/8fLd6aH8nMRNjeM7xHHfeCTSAT1MJFQfUxE2Jo/CPQs58fYaMB2cBQSAIWLwC0fgrI/ZgxYNoDBq27FeulVH2IFf+PILWiAxEn8O8FniIjrc7nPfy6vJi8PnBNJ5uJGhSkNtziLIGk6eJ2rhGjr5zFLVvLvJi1949N6eL9hQvMu4Ew8l9eEd6I6rcv5nfb/2dtuI20henM+aBMchyrLNNkuCTjE/wbetFUqmBcez3UXx887fOJ2fdesJnD6b8rMZ5fP//aC1ddF7oxrvZgsk/ijafXMpDPiSl7gVCG9SElRwm+LYJbPTYiMPqrhedGHOCxvgRZJ4txSL5ACm02Ws4Un3BeUx0wX5+H5mDvOxhwAcVGhpwL93wX0kjoEbEvzaPnBgXQk+Q1YR1XM71SbBmR84l5xVklhI7aCIhhggaLCALrhIVTV5fk3rMD2NrFD2+gW51t/vXwJ1zdjKtPpldm7fR7nlswH1VGP2MfHn7tzzx5oOM8B2h0L/3iyziFerFP5r/ga3Hhkew4jt+8Il2ilvLuHqOju77/kCTlkhT7EgKkrdQZy51nm7W25ny/pVs+LAZa2A6s42ZbPVf6LTnZRlajxYTXnqW+uRJbsE+ry4vRKuZ9rCBdQaV9g7/DvbNPMLVFbG0954YkBACndoivgmF49vH8GPNFkKui+Rbr7VO/0K/LZlalEqLqkVhm/NWEjMG+pvyO/MZunQoZzPOMvfHuQw/Mxz7skuD4Xdcfif+DZP4dM8hHm34CZvNSGysL2bHNcyMXMrCyxI5edL9nNy1uag0KiY+MfGS6/1vkP4YRnJy8v9xzft/Qv7HIPtkWf7b//oDfQB6vZ6PP/6YtrY2TCYTv/766yV19GJjY9m6dSs9PT00Nzfz9ttvo1a7BwGmTp3KmTNnsFgslJaWut2jX+677z4qKyuxWCwcP36cMWP+T7me/5H/q2I2m5Fl8O0ZjllTQ1HMR9TGNFLw5LfE5yj1Z5wOUlmFptfI3hu/IrjiJGOKtxFtvM55THbaeLYP90I3L5Fv7qqiNrKW4vCXafM8QptXNgXRz1ATvBG/z9/g2h8XOJ/BomnAKveQXJxM7MVyDjy2mYZcV+53adhbtNsaqYiroDNKR/ab2Wy7fxsFtdUctn3I4Y4f0Jv13PHlHTSebeSyLy8jIDGAK3u3sPC0g+iWWwHwOupJ2uGvlWK7sgoPi6K8lQ4dhTTx7xdLQRZIKgJ7l5n6lClYvAL7knzcP3NJlll9ZjXCiwK3Hh1JROv1+Ehx+Lf7E9zoQfc3hYw5dul8vuPzOwipacHwrgdNIU19fals+AIqjo07hukGE7JDBruC4gEl+22gYfDLDb+QvyCfZPMNZFS/iyjpESQBu2Bn/5Q/nQgBlcML757BXLn9HaJzPKlPmcLhcWv+5s0lZBl0Nhda0CoakQUbD/boueyPIdSv/5a4XXF/2292jZ2QWUPJDchVzh0Q6AtqDqL3RK8zo6n694vM3TYX0SHy/U3fE3HnKKIO/kB0gYLWCVOnEdW6FEEW8G2Hiz/nkn66iOmn9/Bk8vckNvdFPAUBm4cff3zeQNG+Wj7+mEuCPA5ZQhAFV7Cmb9MPt6YT2hSD5ONHxu0ZrF2xluWZSq0mGyJHBTOnh59GTPamt72X9rJ2Hhn5COUPljM2aiyRqyO5o+0OfGN88Y7wZlWpEnStC9iAVdXOhf055N31IV5tVVQkh9McrPSHr9GX8UfHYy1vpTpzHo0J43A4YDsP0+S3hZMjd5M9XY/ZaCGk7Bh5Yco8rQ7+Bq09kNjCfALP7aPmqJKFXLqzlDT9VAZXfUq0eiJb522lJtHHrQ96Aj7myNhPeXzsPA7MC0QOC6amqIabNt7EIzseYYK1gNXGcl544QV0dyro7FOfnWLz3ZupdiiZ7WVh72CzQeIcpTC6zkeH1eGuqMoyrF4NhYXw7rvQ9PtRYvO2oGs7Tvypb+goqmftWli8GKqqLs00DciKZfo303nv+3fZbP3vKb622e2oduFEjkmyREdlB18M/4KTH590mwejToyier0rE7A/MNFr6KUxtBHBwz2Lb6vmNuoCNzDp0CTS1prI/ymfN/zfoOjPIrdgn3+7D7b6ZkbfP5oFnyzoew7lI/1j4Tv8tLQNrb8nz968ksmI/xKSYbwKpFyJN594kwsjhpK1LAufKB+S5yUrtUolG7Ios2nRJhJuTsButrP32b0U/lHIV19+yTTNv4aYmKLRcMyj3EV71Lfmz9s2D/8Nu/CYOZ6cGVezY+QICiOfdtYIqEmKwZGgBEaHla1HOAI+ewQOS7DHxl+QhJAtg88eCNt6OSMWPciFCcvxfuAacq+rc95XmQfKOucx3IPBnw52NxxQUKui2YFu6jjOZVW4tUmilarAb+jMLWf0sRGkVj6OT4+SSS3jQGtqJ7TsKKbzZUjd39Kpca8lGN3xFAHCDXjUFZNUrNA2n5W/dbbfuuZW0s800PKEQHVMNTHNKxhyysgL+7fQMPICxtAU2qMGYzNaUNlVaO3B+PQOwcOifCsmTxNaX/ds7bcK7mDIvrfwLotl2K4hFKYVYvRzOVeOpU4HWUZySJyaWkXu0Fw0Dl8WhtxLTPMdfX0n0XW2lDk751xCKejT6cPgvMFk7K5n2NZX0fYaqemuoK4vY707uYyjE8qQLDas7S3sHBbEubjb2T7ch/U1/6RGv4OQqotULThEfHk8KodKySKWRFSiio1Xb8T+kp28DXmsX7gee12/815EkEW2z9tO8rfJ/HT1T+x5Zo/ze+xHk1226TL86v3c6tsBFI+9Gb+XH+G3pb+x8zFlP/i9/UV2D42gOPwVAEKuGM9Ky0oiRih7lZ91ED49Q4hSw4Uf5xBZFUFrYheyKGPSlyAj43CIJJUkseLzy/GdNYqWSe791eq9jwPJBxj9/WjmfzxfmVcDYLw/Xn4b3q/G4dVejcFYT6/WBbOKab6DoUV3Y24wUj50phNZGtnqHt6quuMV5v2pfDvnxLVs6IYDPeDo0SNYJUZv2cDVP1+t9EPEy8iGYtosKnw6vcBuR+upRadV1mdJsGLWlHBPoMRLt+0mabRSS0kesEAJsgajr5GCyQVU7K9gzxJ3rmRBVqE36/E6D/71Cipsa90aBCCoP0YqS7SFZ+DNQ27n1kXUcWJ8JyE3z+H0rNs5PvQFt/aollvQmzVoN65nlvVZsoV/DVl1TICSmx3YU8oxdLdw8dsTOMR2AN4LhuI4eCcIuqZ2Mei6QXRUduBh9qBddjn5TTRi6wtSuSH5gUkXTjFadadbsK8uYAMTDHBLRgMf3f8B4rBohu75lKn7prqde8XRT7jq5xHQV8KgH1XQL682HmL8AW9stc30+EvY1YqBL0paREGkNqqWE2NLObniC2LOK9R/SQ1Pk1T3DDZ1KydqT5DxVAYPVz2Mwd/A3ra1HEofxenBn/LNsm+wZkrou1tRW3v62A76kg28TBQO7cZa2cmEQ8lMvOBeU93H6IPHrm1M/LKBkadGIsga/Exj+ChIha/WQXDbM6hrexi2803C8xoRd8EO6b9PGNnpANUuCL/qakpH3oQ6MoSqjA5+ufx9qoO+6TtSJKQphKWrllJ1uArzA2Yq4iuQrNZ/K6O9V+iiING9dlmj56vE5OfiqGtEEqyXnKexadBYZehbd+Tx0DkDJggwVe2eST1FFJkggH1iEhPnNuKdVcg3y75HTIpH6+/Zd0WZ5wIgRGviQu8NNOQ28N2c77i4SWFJiLEo+7/a5iCwQo1PtA8Z12b8BfGgzEVdVBAlg9OojXTf4/zbDWTtfofmr/9kUF4Q3l3ezrY6/5/YkeXP4bRxWGx/7yD0a/dj9EEzVk+707boFw+1J6NOTGTa1gCERO2AQB+EdlzOYO/J3L7rdvbM3IPGoHHWO1baFzEjt4pB6xTUVPSEvkxzt7rvDkQR7AOCZk1+W/DtGUZS0ys42jqJyd+GtqcDSYIqQQn0hTaEctend2HXQd7M50Hvbpep7Wr8O7yQPXXOQB8oesOK8C94tMeKVrcCz4fSaAlyR1Zp8MLQY0BTpyF4aDAnR550c2iWhL1B6NuhFEY8zagtn3DFb1c42yTsTqTc34mH+gHE9NmEl512owRUpK+mn38gFYkCgT034SW4ENILPWHYAz/QFi3TvXorx1KnKehJ+urGa+wcnHwQVYQ/CzxeYOEpmYWnZBackkireQOAumV1LDm3hBcOvsBvhb9xPGU2jb6bGF3yJx6CF7PenuXUSS+Pu4XpeaV9/SbxScYnjNm5B/92Jfgw9uJufHtGYjIUUR9XT8btGSQvSOaGTUpJjuenPK+8z3USyw4vY8WpFVx8bECx+T4RZBlrtxWz0Ywg/T0yxuaQ+H7+98S/qNji/ci+qOooTj12irrTddAN06KnEW6bRLPPLvJjHiS/Q9EFGuPH0O3l+t5zU99h/fUb0GYmE7lrLWElyrzqp3Uec3opiXlF2Httfe/v/u1ctjGSIbvfRffWB4w6odTpim2+V3kfBAwVhQRXnlK+475XGlz5GYOq30MSJcoSypBUKrIqxpFQexfh7cr+rbarkeqbMVYZCd+xhvDy/v1JwCZ2Uh24hurANc5gg84ewqshZdzQWE5E2/VENy9H6/B32hs+vUNZMynb1Y9aG46/JDgANPpu5lzoSAYd/Byfg5uJKtzNucjxfFx+PxGtN6ByeBAVOYTOAD/E668hd2Gw8l0JMu1eR/AUglgYeZ7T855E0GkxGPv9Msq9iqKeZsi7Q1h31To+y1KSyCK8ooluvQVRk0X2LF86hlgYvPd9RhxV0J/dcjM9egXxff0PV1D89mbiy6OZVJBNiCaBZr1C96s36xn95xE8NcGEsp8RJRvd3m3Z6mUk/95GQ8J42kJdQSSbuoN15Q/h21SALEPPMOj16EWUFV3Jom3gmd4E8hLfxpCZgKO7F31Xs1InWtYjDsBreAZ7Mu3FacRPd9E46m3h3DPka97+6G1CB4c6EyMFWURrVWr50gHl+8tZe9aVIPTyiUf4qugrfNN9Kfy9kMQvErk19VYAHhrzEACSVaKrvgtrtxW7ZEd0iDz9z6eJ3RqLV6i7jlq6s9TpJwtMCWTW27PwGaTY+s0Fzex9di+N55QESYfVQfG2YhrPuxImBdHFnhSX+yd7nlT0wY7yDoq3lvDa864EHI25C0t7D4Ig4Bfrh0Xtybq+um6CZMWnzYiutwfZwxM5Tubzc587z/U1+jJn5xyKfizilxt/ISPfFagWZS0V8RVsuHoHS7YvIeH++c4gdHTz7c4anbIM016cxu3HbkdUu6/D016axuAbB3Phlwt01nS6se1IEgy/YzjVGbMpG3513+sq1z8x+gRhN04n6f551KYrtc/6fWpas5Ehe97j9F1fkXp8HdpeIxcjXqQ2aB05idehtpjwaSmnp6kbtfhfMwnUrd6BJVvxmWR3rmf2dwq9fWBLIOFlfcE4jRJo1ARM5FIs3d/Ll4DgH05s/jZGnIpz/i6gVpJ1Zf5WD8pJvI5Kx3EnUNEhuhJCzXozu2edIuiGmeRNvIzilGJnW7fetb5nbvMl594DA+rSglpS5p1dbSPo3ms4PVJhKetHiLd5H+CjEx9xvvs8XmFezoDtD/5DOJwxkkfLs4ifFg8yWA0+nMlY09eJfexeGki8cgi1aTPo8YtQalrjojvWttTTsP0sjXGjkVRaPMVAZubWorWFcOfndxLza5EbONNZtkr1EFPzivCsKaJoWhGphalu/VVlg5z9wxHPHcd7chCnRp1yIvv6g36rHl7FkoNLkEIlLtouUtZexvqr1juv4ZAcXLH2ChyjHFTEVXBqxKlL5jDAF8O/oP3zwwyKfoBRUwsYP6uW3UcLSEx/EbVPIpbaFtTHDqPvUuxJWYZlB5c50b3/W8Vs/tcSBf+fkP8xwb7/yH8EXBnDdQE/cjHieaqsx1D7eODQKFlJ/cpMj76E6sCvaT9fS2zeVjyM9dCX3SALdrzb6nnhhRfI/elHJLUDWZSRRCtH0idwJG0i9T4fIvXuwhGbQGWM4hyb7HEHe4fE0y21kDs0l/yR6Ux6ayGv1rhooIyep+iwNrF1wVZKZ/gw47UZLP5hMY1NfdzwqLFpbOyauYvqfdV8PfZrt/cSUCi+wjeE4dVRgyg58LDGM/18GQtPyaTWvYReDqB76yHSLgxEFIJD5eDXmzyIve8ymr13k502gQ98DHRoFGeYX7sfEw9NpOWNc5y+Jgd9rx6HbCex8R+MtT3FpkWb+Obuc1j215Na5L6RTAq+HIN9EDaPWOSRYDYoi5aXOY1ebZUzGOaQHMROjqX4vWIKBhX09bcNSYITH53g6LtHMQQZ8Ar3YkbXWuIbH0JARG1Xs/T1JYw84kJgzjvTxZSCc4SWHUU4fepv54OP0Yd7P7iGlq//ILNwBuHNiiKUHX4t52Pu4qGMJq68+1eEll6SNiZdcn56QTqztw0h6qYp5GblXtI++eBkWu5qYcMVGzAbzRQ8c4ixx8eiNyvzze5w0DJsJk1x/UhJxSnc5dPFL0sdXHPoforGLkXUDmNGyI0EdPdlM8oymu52Kl9eR2Th3r6f+uaIQ2Ts0bF4lxipOlxFzdkaLHYLMhJLvGHWhDHkzXwe+4QpTF41mU29m1idpxgvabWvcFloD+3XbkI9JoCTn5zkg8QPaC5QNl2H7KDT0kmXRQmAGKuMzhoDPbpyjqSN55ETV2GtayOi5BBVSZ5OiqDq6GpWPbQK3USXMSFJriBRY1gj1QkCxoo2Ygp2EFHnCr7uHBqCf30BPpV5TFo5ifTF6Qy6ZpBr3qtETow5QWuIjtWTVjvrFPY72kuscCr5cqxTR/L6w6/T5dPF9+e/B5We5mbFeBLLRHY+tpPSHaXkfJlDr+Ry3lgb2zn37TmmvjiVc9+e4+xXrkyymOYVSBL0DvB/dlZV0RJQTXngPfg2l2PpMJH38DdE5O3gu+9czqLLf7+cy37UoQv24UDN/n85YDVeBeQoDqOIUh0GfwPT/zmdhFkJbsG+scfGUruulvqces6sPoPUpRj8xSnFrL11LZpBGt6Pf58fFikUg/3r39mss5Qu0GAINBA+Ihy9v94t2Lf4l8W0vv89ibMTGXab4vAZYlhIdLOSSVgT+C0yAs3hFdzq+HsH3V/lTjt4KknJePpByfYSfGN9uf6P6xFEgSifKArvLST/nnxSAlOw9do49MohircU88f69dz8LypCt5jNdNQpCqpiLCnvfGrkKYyTM1H5epGb+hEmjzJKwl9zGgkXhw3GnpjK4D3vEXuyVnEMO+T/o2O45eRmenVmTP5R/Na7kh/b/+E8plt/AbvKyOW/X86F+RecWd8DRW/WM+01Lyx/7EASLm2vDPmE0ue/JfN0PenVzxLUqRhZsuDA0N1MdMFOOrYdY/T+cmKqYpzniZKWtKrHSGyYj//xHczYM+OSa+cPyqcm3kfJ/kapTWK2e3G6fj4mfSw2VQee9Xk0XXOUjIIMug0FnI+5h07DWQDWLFuDqBXZ88wep/N314ylrMpci9quxqvDw63eS7+kH/4K8+sfkDe2jgsZF0AW2dr4OSeTL1P69kgiks3BV8u/oi3AnZ6nKLWIl559ie6R4+gKiqck/APuPXw5b9tz8CyBTREVnBxbRspHD+C4Y9Yl9w5pDMHHaMFvZoyCstRZyB+UT1NIEypBRUtwC1KsRHt5O+V7ypHMynsJfTSeoKw71dnVNJ1rcq5vGocf0y6cI90/nXBDOADLB7kcJN2GCsyCibnvzWXSM5OUMXQGbEQGHfiEylW/4ZAczmQ1pWaMSIcE+TvH4lNspNvaATJUBX/B8ZRZ7JQex+hr5NyQYrzGDcbmq6zXnr2pjC88hCRaeWD7A5yPPc+oexSnn+MviSM2h4OkkxuIKtyD2uFyxAuySGtUFuM2PoLJN55RxZsZVfynk44LYNr+BWB30BBW5/ztMyNMrYX3HviQijkhdPv7u42jBDSXRfHIqkcwnD5E7Yla5zypCv6cBvUZfu6Cp1qUPXPXE7sR1gW7PVdbYBvHrzzODZtvIGVFCoEtSh3hqecvIshqqmKr6EqX8G+4gL3vG7/TF4q9fbnv69tRHymnbMQ11CS760mtgd3UpGSgiQ6j11N1Sf2x2fvuZuyWbxFLiolvH4k0C2b/NwG/48AcEd64AcL8wCTD0UmNvPPIO1i1ihNji8nVLx1XdzB8+XDej3ufylXuda1kwYHJ2r8RXUoDL8vOUoO0eh2gw+s420xQLdiZqL0MvUcgktqA1hHtdp45PIbKzPmoCs6Tvu8FqvyfdmvXXjASUp2Hz/zxtCXIRFcr59vVnbRa+5IbkPBOj8biqUBZU+teIK3uFcpCV7Hs4WV8s/QbetuVZ2+31WP0PEWvRy9VsVVY02RyZz9GQ9JENxpPD0s8IZ1zSbhzJhcm3u6kBeyX4OZgdPl5OFQCnX5e9OhLOJOwhD1dvpyonIet4HYE0UBJGpwz/8ZEUfUv7r8Chw9+Ra9PKCHP382RayooTSpF6POKG6xRrLv9O5IeTlIQurY+lgCt4d+jddWByeBeL6sytpKvVvyExsdAfLE7faPWFkRnzP0UXbkE36YSYir71vvxMPoZmDBf5GmdDzeqfHlaFcKFjAAcj8CYBYWszvqGvRnd1EU2Qa+Z1sL+pDWZrT2wvjWEkkOf88uKX5j64lRipyioginGLxlR8ivVMdWseXoPD1c9zKIvF7nVcLFqmikO+ydNLWW0hAfQEN6A1hbCdN3jANQG76U2MYWAmxdwZvY/6Apw0QHnJF6HR7fMlT8PY9q3+7hhw5149wx2e29RErGrBUqn1dMQ3oBf92hn2+3pTzC44i28TEHY/4JK0Dh8eTr5Z2657xZixsfQVtJGl9VFO6V2+GKwRTPtleks+HQB1/12nbNPQHFkeha2o/1lA3O2jXO7tllbQ2dIMn5Xz8DQ3YJdLmZL+UZyRcVmE2QBjU2Dvs952J8M0i9VsVV8fP9GHGnhGHoMrgbBgUbUoKuqJnP/p9jNVndke+v1hDtGMyF7AqOeGEVgZiBbFm6h29sVxHSIPRjyDQw75sCu9cTbfp2zTQkK/X3ASpA0JDQ+TJhjDHkTL+f3K353a68M+QSp7VGkugpa0p8jvPdlDIIL1bPCF+73g5JxNvZfdaLvvRW7WkDFJI8VBPQuIeSfy9DPcAU/++tgenV5Eb46nLyDeX95RAGTfyTzt9/PuEfGczpfz868k2T8JLB7iGstC0wLRBJVpNd9Q0DXZI6mTSE7Xfnivyn5Cu8Z3hRvKaatRNmL+uv+9YT1EDMhBr9YP5bNXMbQUPdSEdkzNUTPSuPP5D+JrP0LrVunN7esuYXajUXo/fRYDIouZdAYMGgMBLYGUv1dNYdePcS6mHV8HvY5izq3OeuLVkfVMm7H81QPmoukcjmdzQYzF1OLUAX4oe1oRNvbiYzE2W6lLnjRUA/yJ91Bb1k9wRUnkSX3BA2Lzo5N540tLBCTpwlBUg8YC5GOGVdRMOlO7Oe68Wv3I6xtMdGty0AWkUWZ9UvWY9fo8H2vjmElrxLSqVAx2zQ21JNGOu8TWVbjvKZN3Upu/DLyYu69pPSBJMGgmncZWvkVHjb3emQD0SsZFy/Hp15H+qEviSjaD0BF8KecTL6M1oByjk4opX30HA7NTaIuoo6CriMML1/PvDMmrrvlXSpGPoR06AgzPipz6gSg+FY0gg6N1UHNuz8TVN2PyHMla405Pga5SGbQtYOczwxQ7/8LJ5MX8kv1R9jVemTRnUEDIKIulKBpmeTMX4nN4IvN5rK3rForpYMj0WSkoHH4Et6xmMGVnzrPzRmeQ32WgfLMMVQmmjFYXIiu2qhaXn7uVdrTxmFx9NWaHBCwsHUb0Xcra5r5QgWpR1ZzNu5WzsTfjF3s5vS507wR9gZ7X9iLxW5x+goGjgsoSbDLv1hAcFMwIFKZ0Mvb/3gbIUXg22nfknrKtWYD/C78jrxKRuerI3dtLrq+AGT/vhBVE8W7Ee/ymvdrRJyJQBIlipOLCT4T7MakAPDdnO84+LJi0/tE+TDmoTH4piqI0uYLzRx65ZDTR2E1WVk/fz0nPlTWF1OziYr9FYi9yhpZlTmPqS9OBRS2nZPznsXk73r2wXveo+TtP5Almfaydr5838U6ci7lbV595llOjqvGljmMH27/wS1RsTm4ma+Wf0X07GjyfshjgsOFPhX6dWJBInFWIoETM+jfR/r1Bu/mMlp2n+W/kszrM7nq+6t4Xn6ezBsy3b4RWYaZr82kMWEcsqh8x3pJ0YnVHg8SOGfkJcFBAIdaT0PCOGJvmULF4AXY9N40+SkJWUbPU3SEp5Mz/xkku8ykHZMuSWaIqI0g/byBqAeuQr9AsSHd5o8oYfIJZMhT83FolX0sPeM9DvGvJcEdBgzTR1GVMYfiFJf2JMgqOg25HG3YQ6/2r/Vd4bI/LyPim1K6N2wmLvcPNwS7TWsjZ0QhhtTYSwKF/cH5SQcnYTBpOTZPQfUl1z3HDRePssl7JGEODcPODEbw9KA2Ukmm7tc9m3138tLal9iStIVtD26jpqOGxu5GrKrWvv52cOOWGxF9vYgq3Et8RXT/Cyl9J9ic4yoNqGfaj8R2qDX4DY+nOyBaCWBLKvS2CAzWWE6POE17qh9JJ9aTfGwtRsMZPM0pBHZOI6F9MOlHt+BRW4w6Qu20LfoltgKevuoHjl5ndLKEOZF9/XatSsJmt7HpwCbGvTeOOzbdwQ2Db+DGwQpFeP+37ZAdnBt6js2XbUY0XBomUulUaAK8sOlcQX2HA3SmNrQ9HVhqmtEe3INHp2u8PYI98QrzuuRa/5H/d+Q/wb7/yP8okSQoC11FfcDPJJYmMvz5NgImZ1I1uI9mcQD1WVNwIcNeWIRN64ngsCNIrgyLHo8e8jPyQaMmpF59SZ2ItMI0rvw5FtvQkTQmTSGp/ikMoovCqMO/g9qkCDTzvVldpWTuipKiDKlFTd99JKLGRJE4O5Fdu/spj1TYtDayJ2YjSRLGqr7aDv2bvqwisDUQfaOO8qGXuykxAFEFOxEP7KPrj70MPu9uMEe23URMxz8wnilj1J7TGIwVOASXAz2gLYCZe2bSc7QJz3ov9xo4zqVARv/hSH669icGBQ8i3aRQy2X4jaB0zAPUZF6Frc/puzh+OWfjl7JnSCwtthpm7J6BzzuK4WtxuChHZMHO6orn+eXtXzj04SHmfzifdxa/w2fBWlq99yLISgHsmvhahpxPI7EkkbsS3naeXzx6CY4rriLl2Dqm7XXVcAAlwNka2IHlYiWpx9YR3Bf4cYhmevRlePt30TniHG23JXL6qdP8VQJbA0kqDsfaeWmtJVCCJoDC+251EHvvUKqiq7BqrYzPHk/LtiI6k0c4UUYNtos0+m12vrdHuB9dQfFIah2i6FLQBFkiYdMHAFSnpHA+5j5kWSCp/ikEWWDujrn45RlZN2sdL9/0MjHvxWDHzHdh8FjwC8q4SWC0GHnryFu8n/NPstPG0x73GBvC4VC0QukYPS6a8Y+Pd2baqQQVMZUxzH9wPqe/OM3X47/m+g3XO99XkDVUxlUiPpZOS9RQbAPo7Owau6IY61XEnf2D0FIlW7Ofkou+v4JHxFA4fhlqvauGqV1t5M/FpZTNXMH0l6dzfMVxkn9N5vf2l9h0g0BJ0h48BMXJ3nqxle56xbFRbRcRimFMTgBT9k+BukYMGpfj5LPaCg4OG0Xqdwlo/bUgwJXrruSxhsdIV893PZVdwlhtpLuxm+YLzXQ3Dsj+Nl6GJOGGmJBF6PDrIGdEDq88+wq+w+JQW02obGYkCcaYXkZnDceis2DRy8gy/1bAqj8wNu7oOKb+GobGU8OkpyYRNyWOEeEjmBQziWCPYPYt28eQj4dw4bcL/Hnbn8jtrmDRjYHvMC5qHGHDwghMDXQbi7LEMmpHCiTOSmTp7qXETopFkkBj9wPg1IjTeM5UnFsOycGrB1+lx9FBXPM9hNWHkZEfj81kxSL8e0gG2QIZ+Rl4NrQw6elJjLhzABWYSktqUCoXH7vIkbePoPPRce+Fe5n6wlRaO9r+vfv0NBBQc46eHhdK5ELGBTqGxeHotaKy9jiNmn6kblXwF0qtJK0Hpe1HmaD6e1q0gTIGmCBo+HPnK1hUZegO1DP89HBn+4HMTEAmMWA0thobhSsLCWkMcbtGVsVG7MMysB49zcQD7oEH6Nuz+tZ/h9bgdNjIgoNu/2gujL8N38uncGyK8BdEhUho2RGSN72HaehozozVuV1XiwfWcePpChmMzwaZoOYgKkI/oslnGyPCt3EqezLa2ptIPPE7YZcPJqrzLQAqQz6lNMy1/pZtKqN4c7HTECjvjsVb8iPedyWO2/zIzMvEt8PX7d45mRvpSNWgxoDK4QXI7Gh21bcYeiYeU0ElNdE12LR/QdmKMpJKondIOmtv/oRzSc9x0XiOABXc6wdj9EoQrcVay5qeqy/pz+E5w0m80MnYJ65m9qzZRNZGMunwJLy7vFGLyl4vd8tMemoSz/Q+AzFKvyloCIH0gnQav2rk0fpHuWHTDQOQfRq8egYz/Ibh6PP0CuWK4KLjOB82lsKL6xl07SBSFij1R/qN5tLw12nyP4uls4vHX30cw9MGxJdEDgRfz9mEm8iohE8fXgU2KwseHutExbf47MGKiXsyD9GR+AramDB+61X2ZAGRgO6JGCwxBLQGYNtso7OmE4Boz0Sm5OUjSjrSLqTR9kkpNWkzaIobQ23gd25zSFJrMVW2EF10kKiGiQR1zsYuuhzmhl4DKn9vjo11UeGM0MH6MJigVxxe5ydMZu+Mvc72xAq43dLJ0bFHEdoq+GrMV2hqNPwxswiTvhgHcF2dQM0HN2HafojTX+YQUHBptr9DdiDZJAo/KSSiLgKN3R8vS7LTsd480UZN2kxqAtcA4CuCLIv4dISisyjj2uHpziczouxHYlqWY+/qRdvTgNrm7qS3awx0BcQi33svVYOmK8iqWTBRgKkibsiqyYKCuDLPhJ2Jwxl9OJZ7nnuOhItWuny6nBSN33eBUAyPtihzV+OhYdxj4/AY7R7sAeixKnvIQF22X2TZVZKlv90mg6kngISK2/BQB1Ew+U7Ojmx3O88SFEBz3Cgc/kF0e/e41W0x2CLxvWwyeVPuwe+qGUScERl31BX4yDMe4ZaL33L9T9cTtXQqjQnuQREQiK6Oxv67nZOfnMRhczgTlwRJQHSIbnFLhwM8pQg8zAmMznuHab/U03q8hB6/yEsCJsltu+i+6iaMNz9EfM9O0mpeQxLNLKmI452tX5K5/2McZVUcmWrCVmNkhePSPvs7udMuc3zPx/hX/IG1rhmfhj7nbl8mtEo2MGXENJa8uwSttxbDawbiy+JRJwzj679xBP+dfC2qGdKZ6UTd9ItFb6E5pJX2vWeYtS3Wrc2/ezzhHVfiaY0j4exvjDnu2qHi/OGqGTqW2R5iQeKrJN7+Jc1Xt4A3hPZ1W6nqfuadbsH2+xZ+n6cgFmRB4scuqFDV0tW1ga7GLoYuHUrIIGWfUhLUlHVMGoh4k9z7sijqGVpX7SL9ZB6g0O/7Cop90unbxMHppRiGp+PQeiD9haVIkAVCGv0A8DNfzojSX9zaPQJT2Hl1PJ6LMolqWUpE2w3ONrUo0pQwlrqUqdTfc474sgHJZn11sYcvH07F/gp2P7mbvKY8Z3u/A8/abWXL3Vs4/flpZ58AZOZlEvNjFWJFKXE17rZFv3vEe9YYNl0fyuaJYzke6UI8N4Q38OEDHyLEROLdcAyV7e/KH8jIb/3GDT+43qfLkM9vLf9E8PKkIywdKcg9wC0LDgxSCJpMDXsz97Dwsml4vAOeHwH7IbBuCoIsEl4fzqBzHlRkXUF7vEuHb/U+MMCecxcBkcjCPfiu+QCzh4H2gPZLjokuM6E976JR7e50fZMhKsg5kYa+S6AkoxyA4oiXnNe+2esTrvlhNL1/nrmknr2AiM6iw++oH417G93aHCoTgsPOxfWnWP+PM7zzQhcPfKAg4gbWfLvt3G3kzF+Jv+VytDb3mqyiQ0QdoGb6q9MJTFH0YY1K6VubzYbD5sBYZWSqz1QeGvuQe5/jwDc5hPgl8Xh4xru1qRwqQhtDMTebWLx+MdnLlDl09oQnBrWB/EH5DDs2jNH3jWbY8mH4xvgiy6C1K7ZoEZuwdfagNnc70R2uTpGR1RpqlzxO1eAFVAV9TZesOEjNHnp6fcMxZucRm7cVQXan8dx6RTEXJq2g9YZpFAwqQBbtNPsqgcJ6zrAuNoNG39/QPmdkxOkRIMhY1S2YNa6knbbwBIKWzacg9l0uRD3e90gCTSX70d2mo/XG+8gbm+kcP6Hve5IEmxtbj5LAMaA/ZdffvdoqDjYowQfP3lSu+mkswX82oLaaQDLS4XGaLoOSJdjl08WJcaX0xKTR7R+qrJVtLv3QaaOdUpJ/52912XgCAqKlF09jLcFLZtEaldXX4Ar21UXUIVwtKLWagfruWhp9N2OTTzN7x2y8S3ScmD2RH698lL/K2jsukvj4YgCsqjbeL7uDEn8lydausXNxWCj2Y6cJv6jU1w1vv5pJ+Wfw6cni8KTDVI8w0OmRy9n4W+jVXZrk41t0grA7ey+pITbx8EQWr/fDY2gy3TEShybtoyZoLbWB39HudZQp66ZQrCmmo6mDZ7OeZfS1o6kIUcoFyEi0l1TwQeoHnFt3DqvGhizIfcGpPsYgX5np7053q9ndL3bJzvyP5vNU11NofPqCIT3N6NV6PCI9SFqgJFDPMM8gMSCR42OOo7aqKfi5wKkDyLLMZV9e5mYLDpTE2Ynce+FekuYp19J6aln09SKyblXGr/JgJWunrUVXp6w3HeHpJM5O/NtrATTFj8FvbCrWbisfJH6A6fddzrYWb4Wyu8PzOLKM217hrfVGMkjURNcQPiWcldaVdF7b6Wz36xnBlftLWfrjFzScbVDmfJ/h1uK9jyOpUzA0fE31J5vI/zmf7Dez+ezUZ5ysPXnJXgogCMIlNfsku4RXWxU6k5Ks0L8v+5lGI0lQ+tkup9/FandgF7uwaTXUZMwm9paptMSOdAIf/iqVO4sYvm+40+/n7P/SRMYd8kMV6IcQppSikgeUDmoPaCd32g3EXu3SY5pTt6CPjWcO/4ckOMCRAqpwP5oSxlIZ57JfA7omcyHqcW4/NJNWn/2XnK+1ahHMDhz1TRg6G3Eaya4ew1LbQsyFk4TVX+o9SCtMQ2/35nSWsl5Htd7MzIRiZsb8zmRBx/xtczCdzHceb9K7ajbLggy9cOKDE4x5bwwpH6Ugqdx9hLoERfcZlD/I7XeVtZfVsS8ycvOLCM33s8V/bt81lbXf5hNE2srFdPtHKww4Tp+giv3T9tM8MhhZraHdO49Dg4ZzIfoxEhofY7A8D+/WCqxeAQR9E0R5Qrl7fwnQE9CByc9B+coLLPtm2QAaTzWpAelM7JlI8W/FVMyrYPyR8U6kp6rPju1P6P27+TpQVppXMnjNo0hql9/c4YDUo2tJyNmIV2YcpiUrMAa7ABZtpW1O+/Q/8v++/I+p2fcf+d8toiiSkJCAKIoURipZyZ0+nbQMVpMS5g8VynHygMCDLNiJuXwYX54axsmkRc5CzrJgp9tXzc/X/sxrBdew6Jdo6u8IpD7CVSuuJqqGoxMbGOIfSEq9QgUiJr/i9kyybKfdPCCbva9mUoJ3BhmbxxE3YCePjpGgRVnQRUFEkiXm/zofX0lxkh7WPUN14gU0dn+awqo4/1IB9sP/oNZ/A749w/GypGDSleDbeA5MXgSvvIPdne6bjCCLGDob6Smtx6vThtru/nnXRtby+R2fc1XI96y13eOkhekynEcSKwirDyO21gdpjJJ1qBJVOPqMf1UflD207Cj2GVYil0UixqudmStqdETWRqLt1NJe1o7fTj8CvAJoC2wjrulB8iI/ZfOVp3l/zvvAwKLIKnS2cNJr3id2uYj5hQa8ur3wF6OpAXLib6TVex+y6iPSOhvx6/BzeyeTl4n1N2/hXc2LnPy1GpN3iVt7cBlYZbg2UItnnCfkg17wIqh5MWGRVsyTvfg1QkXw7ycY7DuY80POu51fllhG8Ilg7h6p1L6IXZ7Fvd6PYdfYGX9kPB31JYjDZjuPP9XzM726Cgy9IWSdnUpDTh3N3gWcj72T0ydl4ry+JrBrGrIg0po2nvSpYRwwPEaz707eLPya2bW9RDUv5dT0GhITEpl6dx23HbmNJlMTesHBg80Q0HYNupY/UOVGsfGaHQRlKPVR2r2O4idDV7eeT75cgGNUA/GfzCJ+ejz/PPRPft38K6lBqZj1ZloTWvEK82Lk3SP5MftH5/P3O0ek9l58myrJOiJQFhRJbVQtapsar24v7CYz/nV5qIPiaUyc4PzmJh6ayKTDFnqv66U7IIbQ7rsYH7yTI83bACiJ/ILB3W9Rf6aept+bMPmZ+F3/PISAJAks//hu2oJaeazxsQEj0JeB3RrItP3TsJnyuMlwE595fMaUlCl0nH2OJwOARY/DEpcBoPXSMkKzlF3WV1DbfRGCAomfFs+pT06x/Ohysg2n4fe+7wbBDTEB0PlGHD9UP+P8W5Il8qYpVKmVxwEWkBb4KqfH/0htehQzdp6ntqju3w6M5QzPoTs8zFkMGmDNFWuUN5dlurq68Pb2pi2qjehx0XzVdBC6IOPiNYw2DSNqThRDf3VlKUsDsvH/TmEbW7SXQ4OGUzC4AY+YMfy29DeKTxdz8tmT+NgCEOQQBuUPYtLhSViXmPD18aGhru6S6/ydNAIatRfX/nwtnRF15FSX0dPaQ8PZBua846p5m/djHoltiYx/bDxBaYpDxK6xORETDcBXwB+AEaWsyuXA7Sg19RoBg6RCb2ph61aQMwbUtsOO6c89LN2fwNuPerplw0/e5Yc66CTnJ91BxdF7eeJSKvq/lRWSmUd2reae3mAgnfowP3JGuJChEW03Mu7up0laepY/bv0Dvxv8aAp1UZEZ7INQXx6LWPEeQS3el1xfxk7Su3ezYWsbPfoqfHuGkdDwGAFdk3BoDZgCotHEQWmDzY0OTGP3oynMDqo0DKmDCWoYB7znbJ9kuItR5tepbz5LQK6GgBsCaAluoTdzPi/EAjKsV8+lLQjmPHkVm10AdeoDfkJAYNTxUQx7bBjjbxnPsNZhvPNJB2frdnPzoLcZVfwZad3R+BbY6YxewY5xrnVwz6ydhKnvZMmHC2n2upeywUPZZVrqbN+8uJUnRv0DdeMdOFQORFGNo68GhL5XT0hTCBptL+aAGuc5WTp4Mwj+uW0omXtH0lZ7HI2mFf5i958bcg6VdgELPHVY2i1UR1dzeMJhejx6CBaDWfrtUrRfaZVJBaR4D2NwxRfo7CGEGOdxneMXKt8vgb54p+TMDBeVdUItIqkkJFFCJbj22Dk75hBd1QQDyjIOROmuvu1L3i+4Gp/nyvG7y4/GsEYqvX4C4KMQ0IV2ciIhhJ6Ak4qx2T/OeBCmSca300j57f9k1thZ7Ji7Y0BGq0hMVQz2P+ysen0Vz8vPoxX1eJsz8O4dREJZGKaT9bTPWoJd54lDdBmtlSGfoLYKNO6eTVjZGdoiMjk05BpafF0OkpOTW7jmroeJy7ZRGPUkoARdb/CGoweG0mjvRmVJZVTxFlq991IW9g6/h8OUhHb8A3YQeP5RKlPWUGOsYa7kKtyrF2ViqmNo9T/P4O+H89L+F1wdJygU2rM3zOaw72FGPDSCpMuSWbXa0NesQmfW4dBIdIckIQtKIPLjDjjSdhWB055l8hQNMR9soWaoq76h2u5DYmkC0Re+oLFMz+iCcs7EBbvpfk0xibRGDWJ4sExTZ1/9nfEgDoG6c3D/GZC6ffCz6BBiPXBcVYnDG0wFX9ISczWFaYVYNekEtgTSFtCGLMqMv5BNh9cxCqIfJXBNIDu37+Syzy/j/q33gzsjIma7okfKuK/fewbH02SfzxWa/gmmbFi5Vnjr7HUk/LKPLgfAZNQOdzrso4F34CvMIs3nBX4Z8jImLxPBxjk0++6gV1PLRl6gO2wxbetPIkoCJ0e5AqRBughierKgZTOWjt6+Z5Pp1hf2OQ0l9k3bx3h5PPtW7mPcw+Oc8z6pJIkl65fQucyOf10evd4hGDuDSStZTxpKJrDMBoymZvalR2IbkK08+0wrKtEfR6yAbAda+iYGkOFVz+xx95PTMwr/yGDKwt7Cw8K/R7HZbSQx7ywteWeZTxpp8Us5M8XlCJMkUKmgu74br8NePDtlFaub4zgkj+A4/32iyHHgkOzg+rChnPbaQULDY9hVnbR7ZdOjKcRg0uI/ZzQ/B29xO0/nCMGzvQafpibKhl3F0eGuGjerQ2BHTw8vPvcSI0t+xzCgZs76Lthg1PG1cRkjtr0Kw4Yw+N7JHC2xENF2LeFqT15LeIZnFu2n+eF0/OL8nOfKsuLI1/fqiaoKZPPdm2npUDPsyUXMjL6M3dWbXMdm+OF7sJHL92ylPSyCuNGu8ZJxYDP20CX8RJfOlRgASnLkRw9tZF7JRcxm0Nu68OpNo9tQyHBu53LhSy4UV5KRe5AejxepjXIFofptD7vGgDre4FZXtDz0PRwOBV09842ZBKYEomp26VJWdSv5UY9w3KgENk9+fJJJT09yJmEUphWS4HsDkUlLKbwoomgZirR5HyQ3fCozd72Ch8WGpJJQ2/VIontGvT6vjbBTuewb4e72NPQYSCuMRY4J5ELEBbe27e0fEhmRR0rsNAI/aWfYtPswBmuIa7ofleRFc8e3ZB/ayTSNhjvMZqWWcBd8fhiOHDiEOFpP5bQzGEOzSGqO4BL5i0O3XyTRgklfh10rYOjRMS7/d+qCd1EZ4tq0vl/yPbdf2EpE/loqU1Pp7FBBX/7Sex3g8/tE0trUHB5td16z/56iCB7Gejp+q2Zb4NscH6nMncDOqQioaPVupWNEh6J7D3F/tryw2Zx6YQoAkVFDyY+/VH+1DWCZGIg+B2WtyX5ZcYQbAg083vI4D+9Q9hvHBw5eufYVECB6XDS87wqwepiTGFIwB2NEG0u/W4r9m6X89puy3vbqKujw7+DNJ97k/GKFhrSmsQdUcHifJ4YZBuwaO3YfO/Fj4p0UipKklBsB0Jl1nFn+EVmdveTFejrvG9oQyvKvl9M9/zBCkIKCrA383tleGPEQPl2x+E8dz/HeIThUGvy7x2NVNzO89Cc0Dj8APKVIxhUe4GjaFMpD33OeL5jaER1dHL/yDEUBRTT412BXddLis5tRwVNJ/fgqRG034uRELlQqyt/Q8m+o810NZivNbc0IMYEIPYMZWv4NKsnL2eeyaMPhkAGBHm0F/2y9CnuIP1ktm5RAoOwBKH1cE7iOPWWvK8+ESF3qNBKHenG+cxB7M5Pp0Zfga3IFEmQcyA4HWmtfeQ+xC0mwYVN1UH6skLCLHWBRvsH8QS5nfatczPHWB5hyMgoyF9DjF9F3vb7EE1lNaVIpTFXQZQBH6w5xMvkGwurDGH/0LqqC27B5dFwy72Kb7mFw1ce0lFRQqV9OcdI5zK3u1LsOLNjLyjHHnKUoYi+pdS+itQfhZUmm0+MsYo+d4QfOYOsaqVDs9YnWoiWqJgrBYaM3weFc31JrX0bGQXn8d/j0ZjHy5AXqBndxcNR257m9mmp6vXv5+vaveWj2Q5z57gweUR50+yl6aa+2kidKx3KN8Rpuf+p27r/4Cj26FlJbVHh2qRjUOAgGQeZdmVS/We32Pr4dvnR/3k3DigZebXmVz04rgc0NeRvw1HhS8EoBkkPC3GFGrVeTVZPF8qLl1EyoYeU/VyL0ZSUJgsDw210Jki1FLWy8biODbxuMOFRE561Dl+ZKVFRpVU62GYCwoWHM+2geaw6FgwsArYx5cSs+Te10B0QjqZVr1GTMJmg2iBobE56YwO6CcGecSMZI2oU0un1OItaXMLVlKsfGHsNsMNPwWAMBrwWADRw2B532ThxiH6uHLZCArsl4dpmJKD3J58NO4pkaCZOVfcSmaqdHX0L2JG+mJLxH7toTlO0p49leRffsfaYXlaji52t+pvFcIws+XUDokFBk2ZXsJUnwy42/kHakgOboYcAipL4AUcqZfEpyTuFoM+InBNKYOIF12wrYPnwIWlsIs3Mb3QKHGVWrKIh5GJ+eoaitPXh01DJo5Vg+HPY6EhJfTv+TT7+0UB/2JZVZNWz36uX+sABo6+8n97U3qOYCZ19sRQ6ZC+ioDlyLaVk5QRvimVBYzkRgBYpe1YBC3ZkNBCSnMy7Oj1kfx5A/sYQuD1cSiac1gf8Ob/TL1b9wnepqxnlfz5E9Fjo8XQaqxqrhtq8uo8HwM7HVTYSmhdIQ7s65cHCOD1cF3InJpgTufXS+3HZfDHLBHJaYH+XHESGk7Pud2ebZ7Jyz0w1N2xLcAi9DfHM8TfYm7JYBzgIZdv5jJ5IcQnXqRLbNW+V2X4fKjm9yMMbiZswG134lCw4Koh5FrXUw7r0JDD1YwC+3hHKuK5m86D/p8FJ0JgkHNeOuYWPmtc5zTyYvQK29g7o7NYxtHY/VcfaS/rIkQU+XB1PzQXLISlC3L3lNlLWcvLWAb4a+z6EfDuG11IsqRxVpKiX5uD/o1+8zSgtKY8SpEcSXx+NY8veBv78iu+12aIwfjUOtQ+VlwB5iwOECY7J64jcEJgey7NAy/jfKwBjG/yf3///krv+R/8i/KYIg4OPjgywrzgWA5pBmSq7VYCpvxL+unzLStSgLdgdVm3LxbK/G6KHwMi80/0BM851OBVA1KIRjEzvo9HHPOGgKbSJ3RBOa00cJrjjJ0ZQZ/NntchqNPTqWOT8e4MyIHURVu6Pv7k9/neFnJxKeZ+OrsV/x6eBPCQp2wap1Vh0rvljBxZ8vEpaluCdqVQdp8P8Ni6YRi96CQ+2g2/ECRWHL2T84lUPpI9g3OJl3H3yaulvHoI0Np32A49fDnIjWHkr6oS/oLWtg23WRVMW6avPorZFY9BbqI+oRfNVuWePn4lZwVPMqo0+MZt7mJBylnfh2+KISVFj7KrvvrvsJ2l9H11mEPFrGrDdzsnm/M9tFhZZvb/mWmq9rqD9TT8LqBCLqIvDuGUxA9wQcnXaMZ428es/TxPuH0PZUFewHe08nWkcA8c33sfiqe/jkwb3kZuVS3l3A8eQ51AX+AFI7jt5Ocmc/xm9X/eZ87iEVXxHbdC/J9c+giQrl4PhfKI0eiFqApOpo7nz7ETyONmNwKI7CK/yfJ6tiDSsC1nOl+guS6h6gZU8ui1j0t3PPITkQBAFBEIj0jGN6Xgkaux+rl60m5LGxRG79kpjzCpqv3zEc1XIlmcePsu3KL5n3QzZexm7qzeUURvTNIUGgccgsvMYPpsNToauwSn3UqNY08JiJxi+ezNszuZiqKCEZtqV8XzWVX8+fZ/Cxs6gL87Dts+HR41IYOyQIL1ZhPjUYR6kydpIs8d257zhdf5pmUzNNoU3sv3c/qYtSmfzMZE6McdXJEWQN3p3eyN9UoLH2ENhkctbUiqiL4KH3H6J3fyln5z5J6UiFOsghK99ct1c3HQGAKCJLdhr8fiXD12XE+bV7oTU288XwL0j6LImMAoUPXygGVYmMyuqPtg951i8BajtyMvwysYpP7v4EuddC0Kog9GY9alFNtR06e3RsuncfeRuULL220jaaC5oRpP4sVKuCavNU/v75tRKu//0K5z06DbmXIPv6lR7vTm8SSxLpbbk0Iym6dRljircT23InWh8PvAye/xbFl6BTqJ5KMjtoL2vn63Ffk/uti0pWEAR06LB0WghMCSR5fjJSn38juNmTjj8O0l7qnpndj+ybvmc6oz92UHmokr0r99JS1IIkgaclmfGFhxhdvBlZBo2HBrvOzh9Ff3DU9D0t3vs4N+QcP1+9FZWnnuD6ONaoLqVp/Dv5UjQQFvwEpcOvxpg6GVuPja7aLicdS5WxCuFFgRf+8QIJHycgOSS6G7qxdFmojm7nC2AVEItC/fEQ8Gnfv4f6fl8FfA6oE26jLnU6gNPpMWf7HCK/PYg6IYYLg0zYNDZSal9wPl90pR5VrRI8cljt/5Zj2CJbuZjWQendHmy82r0eh4CI3Q6Z12UyK3cWgxe4ENcZ1e/iYVUojQJefoiN1yroJ50t1HmMLDjQBPuRk/Yse4ckYFW3kFHzFmHGRSDLCA47kl3CIbqcnBla2BLij23RHH657CNEb+Ub9ep1FfYWBAH1gT34tJQxbsP9CAFKf5lkqDamsVWVxZa529l+5aWKO4CIhikHp1D0fRFqvYYxUWOId8wht3YxX5x+j842T7wigjg/9T6yhz5wyfkSEmKvCZXN4sxQBUhoeITE9s/p2Huala+uJLA1kKVe65ll/IGh5d8QVRPFbatvQ7fnN6buHY7OrBjxY/oTVnunEtAejHXDeabt+ysaA+oi6zAbBHJe+ZMRlhGENoYyMXsi/u3+qEU1RalF2KfaaSlqoXhbMcFEEtuygrCOyxFlLZNfns3yY8sp3lZM3ak6ZwBdFiQuhr/E7s49CMsFejx7+Oz8m877nh5xmsZ53nw+7HM2Xq/Mkb/WXdMPj+XPy/68RN+Y5wGjuvwpTqtk8gtPOhFh/SPhqG8i7txmVP7e1EUowXeDNYaykPfpMuRRllAG4eAdqQSTXYa/iv1T9+OzJhN7HwXPXw35kNo6GredoSJzBsVxO9wCfQAthgO0fLuZEadd1Hr9K4L3xgWEnzYSUbSP9PwoMmreJqBrCoEq6JLVJNc9T3ewP9kTs7H6Wp30MhqgJxleXf1P6q7zpVQodKuB6GlO4qXEH4kpjKH2eC2n3zvNdZNvJLzjSgD8TCOZ1r2U5E8M+NdfcL5TrAau1wcRaKhDdsiEVJ4isSzB9dz2YERJQGXrRRMXTnl6At1eroQAgO3DvejW5uC4mE+Vz73O3wN94eIt8PzzELbkCSb8cxUVt1SCtxKCmhmxh+zp0fSMu4yweg/u/+h+Z8LV4vS3yJ/+KPfpZxJSGkL5HiUrdyADgti3uPf/9ldkX6+ugm650blP9ddwetPPh1WjfqQmKQ0h3A//yq1oO4+4j3FxJ3M3diPU1jjP8+nJIrnuOQD2t3yE2HuS7r1KkK8qxqU7CogweDC5Mx/FmFuJX0MhIHMgM4P9menYVJ1YdVbEFSLX/3k9Kp0KqW/ee9qn0xqRzpSoBSTm/ILJ+iTf6AbjEHuwiZ20+XdwZtZVmM7X8/C7t2FVu5LnRFmn0P+1NOO99Ud8Gy/SH+x7Nrye5Vm/UX3D03Rre1i8cTFaUfNv7b9WHwunx7tohdr9252Z0Haxi7c3vs0rw1/B1GTiWfuzTLh5Eb6qoaSmvcFcUfN/oHVVMf+m+zh8hZ0LGRdQSXqGVH7O8NKfiauI46H3lmNtbCNI+BavXsXhMq5wP3Fty9G0/ULYiU10BcSAYYrzuq+0w4a+qWro6iX6dAOhDaED7iygDw+gNXII6knjqFnQzPZhXhg9znCNfwe5nZF83QmqfBUfJn9I3o+KvrQ2LICTyZcR2BrI/B9Gcfqz05T8ep6VKwW+mfUnC0/JTtpNaVQgVr033j2xaOxB7G7YoIxztycTd0m0rlrHtE2VhNeHX9IvMhKqvm1ALXkT0XZj31OLYLUSUnGC7twy1NZetHYXOn5XzS8U+y+izfAz3isTqYt0JR/16ip4+MJIZs2bRe6OXFIXpRLs4UJ82VVdlIetonanElGf0hdM6g/2NYY10jU4DJWX4ZLgEYDW3ETHxj3EXTShtqmdtbQAAtvSmbfneQz+cfx25R46/DqcbaKk5xH9ahb9OQVHTABHxx91u26HvYF88Qe6PGsxBAeS2PwsGTXvci7uDg51DOXsoac46HCw3Wx2ryVsh2xZwnpyN5YcCyZPOyGl27D0uByhgd2TiWm5gzE+V1zyPgDfXb2UncsaGHxwDeP2dhDXdI9be7d3N46eViLKKyiIuJbUupeYfq6CuMb7+bkL9s7fhuiAJe//9foiveoucqddDzJ4F7jW81af/XTrC0GAjhEdRCy7NEBpE111lzpCU93QJWHtV3FjscR7hvcYuflFGj2/oqmPQcX53APW8dH3jXZra4hsIPOGTMY+PBa/hX6sP+QaD9+e4YSXl3HqpW3IkjxAF3Ld37tnMN3WbnK+yuGK4y8xpmgXvqYR7LxpJ8b7jUywT8DcYeZ4zXGC3gziV//x6PtqyPt1+OHoY45Jr/vAWc/NqrVSGVuJ4OOFR0U+Hh11+Jlczz1jzwxmbzyNoNfRFRSPTgplQmE20/Iu4tubRXSZkZGbX8TnjTeJqk3G05zi9s4LNy9k6pYOFtz+gLMUQ4vPbgDmx1xP4oV2/OtL+bTJhVYN6pxJYM8CPnjwAwzHDfj+8BkevT5Ety4jov0aJ7IPwGLvQ4GIPVTbz9CmOceh9FHsGOZHg/6QUwexi65xERBpih+DkKkkKvdT7hn70Pf+bf5c88MYwjd9yYxf9hBdHU2vroKtI7Tsygrhl10vEXUxB/nWW9m53D3prodWToX8yI65x9ClDERMu9dclj6WWDdLKeLWb+c1Bzez6qFVNE7oIupiIYklifiaRjBHv7LvXBHfxmJqVm1l8cbJaKzuKEtBEhizo4yeyRGsW/IWxREvsWdwLMdSZhLfvoKbf3iTYVt1+DcbnfUm+yWgLYCl65aCw0bTo1ol0AAURT4LiJQkl3BqrBFLSTXeNe4oYLXk6UTl+A/y59wX5zg27tgAhLxIU2gTX9//NVm3Zg2gZ1QR0qDlmo3XIBfIbuUHIlqVddnX6IvlGwvnfziP6bzJjf5RFPqCCCoRj0APtJ5aBEEgMy+TrtAuYie7I9b/KjaTDa1aOcfWY6O7ods9oDKwf5ICGH3vaBy+Clo35eha1kxZA8D578+TcuJ7tD1Gt3MkCTQGDTNfn4k5wZUU79HjwfU/Xs+Q3DRU5aVMPTDVSS9vl+z8I+wfrHx1JXlv5LHnzz3sPLqTVJ9hzMltIbr1VjqDkzg3/QEGXTsIbaCPMymvf19oDGvBOyuJue/PZf52F+r0cNVhpde1KlovtvLtjG+pOV5zCbKv6pCid7VFKGham6AwbHTrziBZbAz98j5KRiko8cZ2BUmuN/uQdGI92Qv+Sea+j1DZzHiZXXagh7GelBPraT1fjzHQCAL4av2JaL+aGfU7eDwhnyDpdYqXv0HPN0rid/88uSr9Ku5qriXl9B6q/zyLbOsbo761eeqtT7D05mVUeSVwL2quQuAeEQ7HqRg6ayNDr19Ou387bUFWMo786GQ50NgVKvj+2rI+PVn4mi5FfspIyDIURa6kV+fSSWVBBllAnxbHsTkz/haVatVJiNs2ccuaWwA4H/wMGRtu4sWeCLq1kzAFpaJJjUXr7aKJHlLxFQFdfSUYJspEPRSFXfNXNDYcffsovafyaYgffIktZ9fC5Tvu4dTC5zmf5UKw28VOysLepVb7FR0FlbQFOjgffzc72z+lIlRh+Vq4aSGRe2pQqSCu0WVT+3WP4aj1C6oDv0FrbKLrqy43CuN++f7NJUxd68/k9y7jxMwRJDU85WyTJJjy3BQmvTiT+rlelCSX/JfIvmXDljGobRCZ+Zn8FYxuN9v5MOVDzl7xIh4dLl3MZoPGxAm0xI5EdkjIFgterZXoupX1bNR9Yxh8kzsb3f8m6Y9hCMLfJ2L9Py3/Cfb9R/5HiMnk4MCBAux2h5sTxCE7qP9uH0HVSjDPYHUt3Cq7xJnnfif5xHq0ZmWq+zvS0ToC8O70ZP6W+UhdNtR+qwjrupe/ioyE5lwOnq1nafVxUVRpBB1+agUWoPJTavANFJUocnHszXRctYyIURGYkk2svNC/8Qt8NG0D8XI89jo71m6r816gKKSiQyTgiD+T9ng6FUOjp6LU9he3tpvMzMnpy8SSBRIbHqfVex/nR4r4TRl8iaPIrK1FkAS0Fi26qnZSL0Shsrtz7fSMiOb4ZC8cL1zgxt03cqbhDGWePxDQNYmLnedIzW1BbWkh/dMhtAa1UtHt2mBn+zxE02NNfDjvQ2InxeL3kR/Dp8xhWPk66gvXc+GlQ0w+DM+X1/FVRzPvdJmYdBjO/nklZaXvApD94E+MPKa44Dc0vEizr0LjsuLLFaR8W+/2rN5CKDEtyxla9i5js9Pozb3oLBQ/UH6NsZMQbCTg1xqSb0/m2PJjjPFQlFlRBIxG9N2tDF33COId7sthaPsVvFnzJm3j28j/PZ9jlcf4IVWB56dXvUG46UWCo+IRZBmhT2vrdwzb1QKlw68m7fYJ1MYaMOvNfeN4itLQt2n0VYIt1U9+zIJNrrpT3bqLmDV1CJID2e5wU8Inm9/mbe0Ebsiq5KdrfqJrzkIOrT3kFtQFMHmaeOnZl4i6dzpnvjnDWxPfoqav7oKuL/tNkv/Gu4+C7AtoC+inH+fA/DSK0or6riuRPT4bMc4PjV7l5JXvR3eeHXaWPQsFWnJrGbX1VSzmV/mq5CXntS//Yx5xB1yZ6kZfl4I+SreUkomv0JJ0M+d/yKPmRA1mu5kQtXLtwb4WmkKbMI8dSt3jdZj1ZmewrzY/npx1tfxywy8U/VnE5js38/X4r+mWlPkgqXpx9FoJTA2kNSKTwgsSQc2uIunnY+9xZvH3S3/QNqEsgZu/u5muojoCq8/i3aI4aVu899LouwnP5hOElmYTMDqRpU/ezTr9v0bx9bkaTH16jyxLOCwOjNVGLJ0uI9LhcPDJiE/4cvSXzt/6KbfyMs5gWTkBXZaO/S/s5+zas0p7n9Pbw5KKVuVHw9kGDr16iLaSNmQZ1JIXAd0TyTzZQPvH65n9wULqH1OCxSXWbApiHqY5pJmLqaWIei3T42ZwQLL/S9z82bKV6JjbaY8YRHdIAkFpQaRdmcaDZUrmfWWHQl3jUDswOUx013fzTvg77HtuH9QqmYD/BA4C28HNybWj7/d/ohw3UMYXHcTDnITGpkGw2tANH8ShGVVY9BaafLcS1KnUIdh4i0jPwqsJLc1GK6r/Lcew3cvOvtm1dMZBa1CrW3u3rogL2X+w9+29jBs8jo1LNhLVoqDYmn12URj+OMZf38Ny+BTjivYz9XwR084XM7Lkd2XMBAflL6zl9o8Vo3RgYMy7tYIR216l+f31XPWdnqRihRYjQg0zgi+QqVX2Dq/vPiX56BdkVL/jPNdXFY5UWYhXazmGiAD6gd5lNrhn6wV+047HAQQ1yhx9ejXt4utu7/WSxsLDhx7GMH48yy9roqJCQZtOit1Au2SjpvMA51/6He+2SgTcnRDjjowjaFs9phUPUjVkIQLue43G3IXKx4vKRB2xLc+SIV5Gkvl6oluXYfYKYdvcbahtMOb4UKcxvt0EKxv9iZjxBHkznkF4cAR7Zuz52zHzNtqo35rPZ7s/oyKugj8W/cFn93/GxOiJHJp8CPNyM2fXnGX9/PWYm92DPb7xgUSMiODHK37kwEsHmOB5K5mVHxHYNZmLkc/T/WEXQZuCLrlnYXohLWP1eAR7oPdT1oGB6+zIkyOxVLWQMyLHLbAFCvr8h2fuIWFHGwGaCLc5YKKRg/Xv49tcgs/CSU70uSzYKYh5iA6vY3T6diL8IPBIzSMANPTUUhTxPEbPk/R49iAFqck4+Dkx5zZfohs0BxuJvGEiraGBnEm67ZL3Cq8Px7QvB99GF+XR2pwPGFKm58frfqRygg/hJYfwry/oa5XI0EJT40jm/5GGb5li8NZ31/PoySvxFGDYAMZZWZYJKR6Lj9GFRhMkLYNGTGelZSVxU+OIHu9egy6i/TquTFQyakPLjzmdD/M94faUVYQ3b8ZaW83JOXeyeaHLKdyjL2X9/EGsW34Kv6tnUZQVRZdPl9u1I2siSc05h3b97wzPcWWkrwyARrtAwZeLyci1Y3e4rOBJBnhu1OOEtx8kzjqExihvDk46iEPVlzQS8jsAd/saeDrvaW49cCvfz/8e1R5lnNOr3yK9+h2m9n6Ar9a/b3wvzaiVsDuDfRa10q/7rZ20+3Swd9n1GKOsJJ4/SVqhO12wLMjIooSqspRhJ8MUPdAeQkTb9YSaZjJt3zRm/tkBKpEL03rdqHU7bE3kCd/T6rmVpl+z8WkuZSClkiir8ez2RBIlEmcnIqpEZ02+tiAoH34tCZdnUTRyAaeGbabLkE+L9x5qgtawb0gieVEPoR8awrkh59A4fJl/ysaCUxIF0Y9Q7XEnPus+QVtaSKXfQ5SE/xNQ5o/RAbt7gC4Tg/MG4xEaxBf/IlfN52rozJKoStDi+9jtbHzkDJsWbXIiomyqTlYdfpPOok5MzSZElejkT41PfBT/wXczQYDJasGN1nWKqGOioCIy9U1WXPuBE0UryGocggVBFjH6Gjk94hzaCCUoldjwBIOq3sfDkkB50Mdsn/4KRxYYsGs9yKpY7XzmtZ0wTK3i5ppxRFZaSDnV4qyLHKsGR6qZEeFD+OnqtYgRYaw8fgeyaCc3fhnpQdmE6zpxNAShO6ZDskuI6r6J1PfttPu3s3fRGWb9uNzJYuCkmOpbv6VMH87NfISO8Axs6hbOeyu6u9quJrpCAJ2WIxMK3HQ7ALVNTUpRFP6n3qdGPZmc+Budc0gUBARzDwH1BfgsGEFrRBx6Wzhji/Yy+uI2REEgqL6NMQftWOoupdvvcXTiU+tD6+FWkF0OqyHlXzvXUb1Rx5CbhjBixQilVqqg6FnTz5WTIl2OUHyRRvXtl1y7Lrwe/dAUEossBLQFuAUENZZ2xhwScLS1Uphe7Kxjrrb7ckVeL0OTFlEy4hqsg5SArMrhRUCXK3jr3+ZPcKNM8A1TaUKPLMt02s9hO1DPNoftv60lvENyoNoFnnUWoi/m4NVyGkFSMznvPF7mNIK6pjHdf/l/cQVFv22LHUZHaCre5gzujPzE6eT07vSmKzaE1594ndrIWrT2YDyssQR2TeXdYDg8sZaO1BDaAzqc19MLkKy18lRtCFvHpiE+NJ3TI9zLJgiyGrVNjTnETPAN7hScAE0hTbQ+rkf7+IN0hKczMNgmyCokh8tJdjH8Mewq93W7LrKOYRuGYQg04Bfvx3fnXImfRSOLWLx+MXPemcN7Ye+xp/PTAdcWqU+eTMotY/n1pl+xNCrv1b/+qm1qossdNJyv4PDrhwk+XU9w10y0jkA8tZ7U7K3hsyGfsePRHeTcl4NQJWAR252BMaOvkahH55E3+W5klSvRtj2gne9v+h7VoCQC92wkuPIUmgGI7JqoGqqSAkD994taSJkLPas3tTlZWVJrXyZESCd/UD5Fg/UM95/OsLL1bueKgsj56Q9i8Qxg1ltD8O5UkoQkweZ0wFuSLaiaG0k89ZPrvAHz32q39fVTf81jlTPIJMsOV90qwWXTdBny2ZHlT67lBwKrzxBd5b6vi5KIX4cHdr0XJSk2TJ7uUK4j44/wyw1tSF0mRmen4muMc52Lmi6fLsria6l7ba2TTrPfmBVRk1qYivCzQNnuMgDsfQucQ+3A6GdEUslk5NQwKH8QGoffAFpjkfizv2KtaeePRX847fl+kQUZ3w4LQrvr915dFS0+e5DEXsIafNF3Suy4bji7ZrsnUxl9jWxauInO8DhujFqJd48L8uqkVxYlvKaOIDzXg+s2uOpzGqwx6Gw6Jh2cRNXuKufaJ/aNQ/9YSrJE9dFqRpxIx9BjUIKX8pPEPTubayZdww+jfmD08dF9CbLKO9eH16P5VkNHWQeJTye6MUWpRBXdjd2sX7CeHY/uoPBiISGeIcwsm8nwc8Mp/L3QWaLG0mXhg6QP2LtS8aEFpQZxT+E96CfpcTgc5P+Uzzvh7zjHBOCDpA/YfJd7ML/fR27Te+MdoczX1EWp1Ay7DJvexZgSf+Y3ar512QXWAUDsXkMvv1z1C3mZhVhGTuDbh7+l26ubCdET8NH58Njcxxhy0xA8DB7kX5/PkHND0PbRdps1dZxJvJuc1Oe5+serSXz6WgK6JpJY/wShHZf1jZMy1wMSA/Ac5ELx9tOFXvX9Vdx74V7mfzyfkMyQS6hvH6l7lFMLn6crWElQ69cffrzmXeLevgOVp95J0+kQlWCfWVuFd0sZktmGJKqRRRWBXZOZkVvNuMKD9HqHUD50EX4pITye8jjvzH6HC0fjkZERBBAEGWQJr9EZiLEKcKHfjxDuFY6HTVmTIucNQdJ7uLWHCUP45ttv+HZbKYuG5zI/7WbEB3yZMPoEEbrFhEmjuXLFlZydMAiL3gOVQ5nPka1LMGvq6U/eimu8n8yqAZQoQFJxEt45HdgvliF0uzMg2DV2vrrzZ4JunoesG4KkvrQWXId2Db0t+U7a0jL/L5kg1fCCcTVhwgEcGj3+D95C/Mg+GmpZRUzLcrx7h2DoMSD/LCPm/32I5IGyBxB9vBi1/dNL2MYCuiYPGFfXHhbYpSTbepo8sbeYKEtS/FMnTS5K85SSsfjXivhVnCWp0DV/OryOo7VoSS6Ox7uqAPMXZoKb3ffQF+xXcTwjj4ohPYR5RBHXfA+RA+jQHQ7IujWLX4sHs2NXHyNbX7Cv/18X4xqsObSG5xzPMTZ1rNt9LJ0W2oqVpLy/0nj2i+lCJX6fvE7a0TXE5CtI5AlPTmLknSP53yoOh4Pz58/j+BdLDfzfLf8J9v1H/kfIvfcKPPdcJLW1rsCYvldPwp82AueOpGqQwovsbR7EoColS8Kqs+KTFs7/j72/DpLqXve+4c9arTM97u4Kgw/uDsGJEEIS4kbcdSfZ0R33EDcSYoTg7g6DDOPu7tI9rWs9f6yZbjrknGeft+p97jp176sqNaSX20+u6ytqmxnvnn75p/5BkMbax5gzY+guqwT+RhKkJIkr1ydjvGIRX6y+z23ZNJ/bCPR7kqyFzxO9dRRNYe7+AypRwK7zQvIwcMWHV/DYsMecywRZZFr8Yp6qf4ry7eWsm6dMSAYGlf7GcUw5/S9itkZRG22lLbCN4K55zu1DmkLQNHTQ9OC/SD53dGCnVIZ8TJchi3Oja9HHhhFZ0Yun0d0PJqY6hqdfexrvD8+zeNNUt8Gbv5TMjISPUPk8QuR107jrybtID1LQQSn1LzIiaDzrV64nZ2wUi5OX89yU59z2rcMHW4GN5sPNGEIMPLDmAcalzKeo6lnqip7imCRzyM5lKNWjskR90VPk1t5D3Z4Cxp0Y5Jx4DETe4Dza0/T41ec69eyHaRWEvyDZiSw+RPvHP3PljyH95tOuCI9v4K5/fkX7/FAyrs2gtL2UbNN2RescI5a8s6Se+gFzfQc9smvyGNaxnLiWuxkdORok+H3Z70xbO41fr/6V5piRxLTfQWLdzeiMOhoW30HV0P7BXn+SqyZ4PeXJPQx5aBYnZ/jQ7augfyTRQkH0Y9QGfkf00fXY6luxa1yT2DnTUnlvUSQjt7+MuO1rfh7/M/O3z0clqPBSNTMv+hXsF7uoOpDP+p+GkPPUSTgI9J+6nwhyCthSJaI8Uums6qT3bK9zkKVVafHp8mHon0OpPFjJr1f+yox9My55PzU0hTZhn6QM8AxG17mNqDuNEPwzUUMm4NlWjb4fsePOFBHQh/nSEplyWcLn3MjztKRM5HHT87z24puUJyqD+3t84fGAU3hr2xAdNv64bgP/euJfhL4VSqHUzrgaiCkV8TR6YvXx5bEnHmP58OUUthZytA/uFVupiVeKcFlrsxh+03AmPz2Z7y3Lncd29Pay59E9aKxGIkqOkF7gQr5J/YbKbp59/c+xNqqWzYs2o4sNJD57E8GVCgI1N2YNZ5IX4193huiCvcgy3Hb77Ryw2f6twthxh0hs1BoWbbuLFZ8NxifKh4drH2bMvWMY/cVoQt8KZeTnIzmWeAzDbAO7H93NK56vMF11DQvO2tHKybymns2Wmi0ce+MY+b8qifYBZl/1oJHYb3mAYTcOY03hGuKmxiFJsHdoNHuHxKC2tOBo6+SLL2DPgb/xmhEkBI2aRy88yr/efINFOt1/y2RYqNUSn/Ek7eGHafT7E4uqjd7GXiWfFwx5zXlYHcqsK7I2ku6CbjQGDaPvHY3HYA88q5Rh/1b+a3m0sf3LRaC37hea9e+THXsrXZ7n8DWN5Nj0dqpuVgrnwV2KbGin1+lLQAASmPqILthLoE/Mv50Y/l4DcaP6+x2jDZ1Zh69xFL5GZeBaF/QD+QfWceK5EzTnNGPttWKwpODfMxGzpoaK0Hfwy+mid8t+gpplvCwpqCVvp+eMjAOpz4LUz+ZyqIyYtBVY1M3YdF60RQxBHRFCn6cDe3/x29D/WXacHcSiP4Zj10kUx2/mdIoCKvE1jqTYeoBXb7+Xnct9KSnPokflKtaoBBsegp2UkiRGnoDeQ9W0ebgK8aAM3sNHhFPwwR4Cj23mzg9/Jkf1JUtS32dY9C/sHbyG04vPUxP0M2Z1odu2qUWp+F7o4rDucU4lz6Xd66hzmYxM2rGv6Dl8kZb0Jwk33o8o65wsuB4fM6fGnaJmaTIf3vulM+GTZYGvOnyx4geAPcOPyvjKy57X6FNjSL/QA29EUBVbRUpxCks2L6GzqpNl6ct4a/ZbLEpdxKCrBrH468V06btp8dlNl+c5SsNe4/ZNqziUf4hFXyxi7P1jSdfNIq5lDb4mBXmaPSybI/YjCJKAXuXB3HNdBHUp750kS9yw+wYWrl3Yf62uyd6ko5Po2Zl92fkCvN4BrVOPo+mTqPhwq5t3WR+t/BL6Gm888Rm6yZdqnw2YtykNl3xJ5qDZVO/0UtL36bE19IEgIIuim/oBQHNYO+FXTsShEtwAQIOq3wNcnhR/LHrVucy39TrmVP5Bpn0jkQE3c3LObH5b9jEnUmZi0TRxbwucNs3Ct7mEiDOF3P757QS2BlJlLGKMHk7FwN3NkLItmYAdrTTdv41lG5U+fcSZA6TVvYrD0S/pdOsIak/W8vUnX9Hs45KwEgwGvMemYzYEOu/zGJ0iZaPe307n+YNYPUQsencU/sBzkWXcJE0HIrIukojKWixDU9xkwuZ76BCFoYTnjSC+LArvM11E1CnFlmY7/NwD5dFf4Thai00D+2fuR1Ip53VHM6zKm8v2WoWBY+2xUnmwEqFBeYYqyYP4lnsZbrkPH01/sY+/L/Y5k16qTgDqHZColRimBclTzZkZ4938FaNabyQvI4/vbt2NYDIx4WgCWquW/OhHOJQxiCbDXqpjqilJ1xH6xiOkHvfjrm9+IrJNYXxUGQvY23s3JsdaYp+/ifqUaW6sGwEV406OQ75Fdib4nD6X/e+oxktPdaKDlhDFP/VM8mLyYh5AZ9YxKNsXdZIvm5dsVpL6ooXsuJupDv6cqvBfMSXGkrWsj6MTd2BXKftf2gCLc/y5+tsbEXpMvP3YeuoXNnDM8V97yAyE0v8KaAdrESUHqphI+gy2/vMVnX9bQlp454l3GH7TcD796lPWHnuZXp0CfIpMvQHN3Xpqo5N4ShXJSpUHdwdCXdq9zJxayjDfZXTnVDHqpIHo6miKI1/gVMosZkVvwT+ilR3z99GxJ4uhe94muu0mwjqWIcgaBERag1tpDhcYvuctYi9uYWSZkmwPV8H9BhWJX84lpMmTfQtDKUxT2t35/XmhAKGbJr/N2Lbv5rb1dwEQ2rmIm88+xIycOQzNH4z3Rm+u3XQtg64c1P+eKc/KZDBRNqyVwGFRODR6Wrz3Mm/TMLISl2NRK7LU4kdFxGd9SWnY69QFuAoIXX5dvPjCi/g/fDPHJ+Uq3s6XhM6iY+nG6YSVdTI4Nwazth7vvsEML/8eDQbe0w3n6EwPvk18nR2jPGn12UdQz3RWBDSToWojf1A+Z8e0YdndQ0Sje8JIEET+XPonhhQDB54/wCPjH+Fazy8I6J3sfJ45y3IJGhSE3WIn6t0oJMGGpzmJiLJKNB++Az+tZ+Qpd89bAIu+D+8548kdocVoMCJKLjBXe0A739/wPeah0c4CTkzLHUzNy1WYJd4edIYPwvNAFav2PEdS45N497lYJt493qTmiXSX1mHVtNDmfQjrxeZ/20t4kiAQvqmWwpFD2DN7Dzp7GD7mDOc6A92B2uGaT0W33EZ8eTwJZ0TKh8fw66JbqApey9zAu5lQdBhBVrH0z6VErz+A2cOMpJIQEMiJuZvq4C+Y0W+XrZpwNxtXuZLxS7ru4mxKOff4yKQVpiGppMv8AP17J/Lu6HdJezmNxp8uh1pZdVZ6kgXw81PO3w3sIGO3wz+kf7Dz2omXzS0GQvAUCB0SSqG5kBs23uD8PdTLxYB1yO7tan3gz9RE5qH19SB3fS4f1s1ka6aAWVtHTMvt6M16rvntGho3F7DijxV0TFvqtn3woGCmvjAVhyTSvKkZT5MnoqxBlNVo7H6Iem+8pyZj9lHYquWh77htL6lVdM6+kpbYUTgEV7EoLyOP7HGRNH+/U/F/6meh9egLKAl/mZPTROpSp9M5eSQXB+2lx0Ppq3xMw4kQRnBx2EUqUjQcXvkBsUXu93tf3UaqInZj0etwqF33ozr4C2oDFdZb04NNOMZNoDU8iibf7bR7HXUWFAHneH6gnxJk1SXen5dI64vuhbFrf1pE+O+1xGdvZsT5EW7L2oLa+GzNTupnXM/J6fJlwDqTwUS3vwPh/Dn8Ck8xKsvF2tfRr2qgktCEBmLXKvmP+MaHSat9HU9zMv4d/sgqmRk/3UJdnaufUtlV+HX4IUiwf4EfRyYfQe3wdbaNlaEfsHHxzwQ9No/zI89f5jMd3XYzZ+fei2mQP+H1LlbzZyGwe8aVlE27gdar7nKy6i+NPs8+zmaeBZuZyq/24t/h2r4o6hmGXRjG3M2B2Js7AfDv8EdnDSO+8SH8jOPQ2XTM3D+T4h+L8T3hS2BrIFr1wHMS8DR6MvLESA6/dJhZ+8Yx9cJevM2DkHWRxEwfT0hsCJJdQpAFZNFGq49SKLNpbThiHGTenUntylocKgePjFckETvNnSz4bgEl20s4+c5JHrr7IcZ+OZbR/xhNX2kfvyz7hdJdCmtTlmR03jo3qwrAmfQOSgti9JrRbtLSnoGeaL2VnFzOTzl8MvgTNM0KO7tixHKuXK94J4aPDKcnZSSOftUKAO/WCozF9fTU97Bu3jq0+Recy2xaGzlDc2gIb8bh4UVLcAuSSqK6q5qVG1biG+PLsh+WMXrNaEKeCKE8oRxJljBr6rGqW2n0/YFe1WasvYpaUHDPLNLrXieiQynAzt02nJwVL2PuMtPd6WJ8me2ubyAoLYjR94zGL9bvMmbfX2N4+0uu+yVJ9JY3ozUp7erA2NWqs7L/6llM2P08+VPvQlJpqAh9n+rgz3GoerHpvWmLHkHelyepm1/HW8ff4vmuKErC/0mxsAXzodNkbnsJ31mjUU2ZqDyz/vdeFEQETw8KJ9xM4m3TkAd8HvvHf3K/t7gkQUfEIP68chddfl3OdqAzezKvznyVAGkVWXOXcXK8Mi6tDP2QTs8zLm9ZQaK7X31tIMadHEfE9lb6Nuxg/o75l90bGQmHxY6HSWTOuTqGVXzttnzJpiVoemW+vflb52+9/ff4VPNLZG59ka7vNiH0K2rIggMZCUEWMRgNiJ+IHFp0iCG+l7PR/OP90WUk0xCX7iYrDqByaDn1j20MOvQpEbVKEXK28VuGVyjz67bANgI+nkbWqJy/bGegePIzmFasIbDgKBkX3HPSfp1+rPz5GkSrmTvP34mn+KDb8jb9KL7KPEHepG7KtuQTUOs+15z4YwbpH6dTs+MwS35oJSMnA42otBMvTHuB0vtKuX+si00oiILy3yVMNLMZzubpuXbnalI+WIPZywV6dTgg8czPROXvRhvgjTl9OB1habTEKIDJv3u//2+L/1OFPvhPse8/8b8kWluVBufECVxILUkkPEtGMluxeLkozQOJMofazvAXl1E28mqnxMpx3fM0++zE6OPDm4++iUZnYNChtfh0ujesKocKvVmNrFX/TbJIcBYGLmVeDcSbOQ9yKjaFkr6PL1s2qux3+tUvyLhuCLFzUgFX5+rdl0Gy52oG3TCCi6O6MHuY3SaYV/1+FUGbzqGfMIKegCRS6xQfwQEtbEmw032uhLGHughsc6d5Wz10tEYPx3vZTDYu2+PGSBQE0ZlEirh2CnLGUNqbPfr3aUYUROoj69H2Wdn32E7Epr9kyiU1+57ex69XulCA355/ndYzW9kp/fco1Z2SlZrsT1m/dD0qSUVcZZzbOgdmHKB2koGkcxuYfeBmEhofJlUzh6qgz6gL2cKJuVPwWjQVSfVXA194thVWNoB5aiJTP5zK9Ruv59v229k50of76vz5LPo69s84h63LxNe7lcFCcNd8Mss2ENw9h2kvTOPGfTcy490Z2NV28gfn8911j3IyeQ6DjnxB9mPr3IpEA5Mcu7qLmsBvcTi4xFvJFU1+WzG0VmMYO5i6QUuJbb6LCYGLWe0Dg3XQPjKb5ug+HDYHgizgkB3sOvsQsffBwc3wWSv8ZGzgnXqFIal6BzgOBlHxv6spisFY1cr0F6eT912e0+srQB/AeK/xZOzLoPZkLTXHawhpdkkmibIGs4cZ6ywvqjKuwGBJR9+nvH/NvtvQOgIQUZF4+FsiC5RJwSjHA6TVvsrS/SeZdnAS+lA/CsdMoS7K3Yzo/MhzNCeN44cfwGFzDfw/DoGrvIuIbtyKob2ERT8tZUPKBrot3STVvYUuKwd1WSyPv/k42osFdJm7+DXvV8o6yrjBB55Mb+W3m7/D9JaJ2W/MZuj1Q5n4+ES3ZKnDU0fygmTUFiNlI6+idLALRSj/TbHP3j+xbgtq49yoc6iDvSgZvZLGpEn92yjf66/L/sHW5Z10F9SR90Eezz36DxZp//vC2AK1jtS0NxnS8RGhPasQtO6SVy3GFpqNzeS35rNh/AZ87/YlIDmAuGlxCJp+2QW7gFePF7JV5q7su1j4mVJcCLZlEtJ5BXprNJIEel89xwoDWf7jfXyR9xZmbS1mXQ3bFxfg/8w9nP0ul4Qi98n4pCOTeOTNuzBWKonZhx55hMdfepVJgoppKq0bk2E6aqaIKkbcMI+SK1/hXOIKJm8+TMTRXxh5+0gGrxjMvDfmkfFpBoeqFKTtqh9XUf5SOR7+Hsz7YB5byjcjA9P495JcUwGjpYHA2ovUBH9Nj0cuo8p/YWLhMYJswzEfO8vU3V4EdCiJhIH3ILDJjtDbQ+7k62me1/JvJ4YPOCDaO4lpeyIZ8bKdx9e/zsjyX4g3uBJ3+2fsx/sqb9YOW0vVkSqSG55BFmz0eObiUDnY+FQ52OxEFB9U/K50RU6PAFmwE//2Gj6/V5E7rQr6nP1DEyiMehKzdzAVI5fjtXwOm66pcxa3SmzwWju0N/sTXRlI25wMti10oR8lwa5MpASIrjRSc9M2vJsVdnScGv68VssHps+ZdHE4QS0CfV+lu0uhyAIOB9hMNpoSxtMcN4bjHk+zU38Hn+SvYYRfGUkBRuxdZibvN+DT7e4Rtn7lekrujcU7txpDuzvruiLsXc6OysIw3lW42n60jrWhIlszBSeL3aoTaAusdxZM/EVoSKskoeNGdC2bsXW5M/IAnkz/iunyF1g9I7HrJGRRpj6inhPjTqAOVDMldgqj94+m91+9RIyKYMTNIzhvPsqplLkURD5Jq89+hLesHBx8kCGrhpAwy5VMGiiqBbQHMCx7GLIg45DtaCQftI4A5uyaQ/JH7gnOBPUEolsUttzP1/6MJi6Qt799m4C2ALf1ymxQOecAFoNI886zLh8kFES6LMpoLVq6vt3KoDwlSd+rVwr8gqxicO5gmA4/XvEjFrsFq+Taftm257HcXkRp5rXUZFzhJiELyvfRuOkUY3b/QlCra/JWEq4kGrKHZVN3mzdtgUriTZT0DAk9wFszrmBa+CFUsp5ufxv1YUdp89mPUV9MUN8Q0uJPcujK5TSlR2AwGlwIWxs81wrVjgRW/LKCgANK4rY4RfkePEwJOAQzPxa+wuvvvI7dYidodhAv5v6D3Jh7AaXo1Wc3EvrwIorHr3Ymhtf3wmcOIxtWrsecGYGut/4y8JBvpy+p+V50/nmQibsbmZG1g5BOl+RSbkYux+eNo3vOaLc+LLXawqa+U8hPPU3VsKVE/9nhBI0U2mBlI/xe0Upo5Rl8Otr5a+y09FFhiqX8ZDm9jb08Y3qGmqWKOoMoK32sLLuS9BqHP5PyzxDWsdwpe+SQ7fzV8mGvCWbm+uD91bVI52rpCDH8xdsz0Pmc+0ZO4Kfrjzn9mgeiYFAB58d7YCmsQLRYkLShqB2Kn7RKUBNTHcOSP8dj6TBi1xnc+lZBVisyssBHKR8hyzJWoydaWxAO62EMFQ+xefs3VIR8cNk98ejzYNKhCIwnlPssoEISzNQGKYmRTv9OmhdNoz7d0n8sZZx+0QIdFhXhjWEIRhMOtRV8QJoNc1T/dbt+Cpgn6giYNpxrdlzDgl+66Pl2Aze8NIbph+/C06J87+PC9nKHj8JQqT5aTfNtzezb8hu9eqWNVEmexOkGs7piFbc6budO4WFst+uIS30IT6mV1JPfU/fnIYafDXBKWgq+R3lm9GPsC9cT03oHmmBfWoLbOZY2gX3DYtg7PJzqgO+VZyVI9PpFYfEMcBZr3g+GHM1EfK+dQXPcaDqDlUSxt2kI+0xwQyPMOBrJ4s2LkPKK0DYoL0qT3xY6064mN3MD08YV0PJICz7RrvZ6YO4x/WIZK/uO0lnWikd3E70eBRR2XFTsBbSKXJPQbCawsY4Gr9epCl7rdm+Dumcp7+/fJLT19iHsmVdJ7nR/tl+xHVmwcjbpKi4k3IiVXtq1VTSHmvBq98CrxwuQ6dPU8vD41dza8yEdAR2Y9XZs2+qZkv27U/pUeWdEBFnA2m7F0m1hduJsxmpuw8uS7GyzI86Hs//p/dgsNup7lGsZUv0JohiI7OOLMHsmB6YrmgHxnq7kniw7UMdEkD9MxOhldJPxtOqslCeWI+cU8OA7txDYGkhC00N42KIwC+0c6PiOev9fEApqiatKJLnhGS5F+tdF1nFwjo2q1vPkDh/OybTpGHLhjn/bS1imQJdDa5g/vd69bmzwbo8cbJKVcUX7yKj+yHU9gp30gnSGHNRSHPgUvR4F5MTeTZ+jF5vYDbJIzpAcutKjCK8PR2PV0Oyzk6qQtbT47qTMBh+/dCP+37zLWP1Nzv3W9vnzZ8mT5FhVXLXhKqQDxfw1vM2DuH367XgGedK2pe2y5QC+v1mwPv0ioeUnkC/RD2sI+J0TUVfTYmpBFiUnzuWvYcgwEDYijF8/+dXt9zd1b7L59s18N+M7En9XTH7Ta97EwxIDQIvPLpJvnsSE/AkUB7uSztXBX2DWm/ntqt8ImJ+CKknFhZj1VAd9CcA357/hocKH6FrRxW7NAnauGE1NdA2CrEZAxdwLHTxkb6ZnQx4JZ3+jIPJJOr2U1snQa2Dl7/9CKqnGlpKByS/yMraiLEjo4iPoCEun2yOXPUMjOJQxiKLI58hL/oGGpMlUZNrJTnzQuY1F3exKoiPRZK6mPuBnt/0ebthBWcAdNEVHsum+I052e1n46/R4KkloSZYQ586meIjiGZUTc7ebjKeL2af0BaKsRqBfKhO7i9l3SbEvpPMK9GY9SDIlo1dSnRx3+TM0pyi2CtLlxXetRYtnL4jlZQBu487bPXcSXh/OtT/OwWPiCI6P20Nu9P0E9E4kqfEJotpuZNyD40g6Npo3formrrtA6s9AB7UG8eD7DxJ6xIemcCMdAR00+v9BtvUPZlysBCBvyAXUgy+XJ55YcIK02tfR2H0J+r6ICccnOJeV2uC0yYMmrwN0O0oJrm13AyQPrn6fFEMmfr1jMLTW0/DrMXy73QHLMcZUQhv06Icmc+i+etbevZb45gcYXPsOAgJWg5W1d64lemE0qR+lklKcgt3az+xDxLvHm3nb5+Ed6U3hhJvRqkajkpRCqN0O/gn+XHH8Ck6NU95Lq6YFjahFdIjYW+3ETomldkEtkkoip9lVoDhsOszNFTcz7cVpVCUoqi2X4pO8QpVCh95Xz53n72TqPxRWs7HDwrkvz9F+URkjRY2L4oqPriBksCsPcdup25jzpuIBLkuyAmT7L+Tv/po/vzj7YRKevwFrr5WaYzXQ2Xn5RoKEbLYgdooIkkBNdw0b8jdQ2VlJaXsp2iAtntd6Uh9ZT17XKfYOi6Qo4nniK+K596ObWTd3HS07svDTN3F35t1MCVHuXUtIG15DE/lpwU/sGeticFr6fZgLNhZw5hOXH/KlBRBJgtxf8sjc+iIhFcr+UjsUVZz48ng6956l4NGviL2ogCwGmH3Kv90ZsBWh71ES8RJWdYvzt8gZyYx7cJzzuyyOfIFTCYvZ4PEilQkORC9P57IRvrN5ZcYr7KvYx17DfRgNErYeM7Kjv8jX/6BtkoUjPx8h+93fUVuMlxT+XfnBykOVROfuQGdyfw4CKqeSglXdRk7s3c5l0S23UJU+DuOKaegWzSJrvHu76GlOYMmud2j7YTvD976Nf2MxWru72sr5EecpmNDCk6NcVgtV/V1Kq+MUpYmlWC8WIZW5fASb/LZg1tbS6deJlKRc6/1j7uet2W9h6FNytQnyLDqrOlEnRFObcSUTSl3y08p1yRT/mIVnTzOR1d7990OFiBpPSzwOtQPTwWqm7xv0l+0GQJvQPPlKfr7mM7fl3T7d/LlkEx2xwwkbHoaHNIJhFd8y93wnZ28uYmTPWp73VxFvXcy5jw4RXryDLk+XzHFxRwHDPhpG0oUDaOyKTO8Aoy/UK5TEgET8PRSg4b3b7yXyoUhe/ehVJLvrJf34Y3jzHRW/no5DE+5+v+128OxpQmfqQBcZRO/MJahsFkKqFH/SXxb9yMEXDvKf+D8T/yn2/Sf+V4XZ4mp4Btf9Qs8Tqwi9bgZCv6SSQzBj70c8IwsY4oLpiBiERackN0o1G2j12YskChi9jEgi2MV6agO+Q23vHzjKIqGOX7kw6w7sBg98O33dzsFD5Y25aydpRz6m9fkyt2RSTOstGDTeLPtjOsN/b+Pwy4cZe9KVwvawRfNn8U98+M8PqfWL5L0zE9m+/VIGgIjNN5gZ7y6mLlpJgjX5/+nc/sT4E3SMi4frxvLnvDsoinzW7dzmbA2k93wZJ6b50ennXqRsCC2mctgSvOZNwkv7DmNL9vNsgmKgLmPHvGsrQ/e8hbXDxLPPgtWkDLZNugosUh8qu4qAFiPnPjlFbb77CKvEfIKsCVnoH9Fz8ceL/FP1T8p/PP9vo1QnqKCmuZqSEbOojaq9bB0ZibKRV9IXegeDat/msPlDcuLuIivpWvaPvhLD/Kn8tqrMiRwfiA+qvOn9/SHiawexpWiL2zJRUNEU1kRZci3lr/7CuCyFrj60Skli9OqKOFh5EHGkSP2seicC0qgvptV3DzUJPvhOTsSrMAufZqXDLy1zIQVXfh3C9pnvMPxEL4Zeg9uxJdFM7qJHCbv3KiLbVzKk+lPG+M/n+kb4oxeOX72R+kwvFhxdwPYF2xGOQ9HhnzkswV7b5QzJYzL47IP6wxBcpObbl26har0im/HYBBezdFjYMLY+vZX7Su8j865MHml4hBeOvgAoWv3B3XOZ7nczPpssxOZuJ/3kH4wseAIAi7SP5FM/0nUin/ohc2iNGa7cL/stJDU+RVCzjYiSw5iae5D+hjEBIEk2Gu58kWdffpiYKmWi/VgLfNMNulNleDVuw2eWPw0RSmHCYE5irFcXD6cvpDzVA8+LJexK34WHSSlEx6lhglokrC0QR6yDkAzXhKEbJakzKf8MotpAX3sfnj3N6I1tNAS6kriyYMPhcC/2eakCnD41MS13IKCmKzTFafo+IEHVHthOU3gP3SWNHH3tKCtnrGLVg48zURCZplJfVhibLIjMWPoqCYkPM2ECNCZNonL27Uh2iYvrLtJa2OpEmzpNk2UHmXdmsmr7KlReynUnFfvy6NuPYj1jJTA50Gk4P6bvHyQ3PI+hvZjes79QVWDik8+z2Vn7Fa+fc70HZeGKZGNQ+WmGnHWfRKnEDEy+8cgIHHvjGKU7S3ngwYcZtPxzjk8wcGcQXOnpxWux6cx66Xmq6moZtdglT9US3IJNp2brnVvZsHID016eBsDOUoWVs3/GfoJWKgNFk83E2i/eQQ/c9LdvzOVxM6ASJC4M3QHAhfjVnEiZjrr7ENqcs9gq6whoyEftUNqvHk+F0TZ5rxqPI3vo8Q+mJ7Ln30oMz1XB1csgqSuKtIIA1INS8BmeiVlTR4Hg8u5rD2zHa7kXxkXejNsxjjOJy+j06vfCFBTZEf/Vizg9poCiyGc5OCSN7Pib8eobxKiy3wEBSTUgwaOc96UJdUmCYZXfMDNbKQ5U2uBAH1ycfIyPHtmBOdRdPlYSzQiCQHBzMBqzg9xJRU7AS9QlOI3TMw6ya7GVnj7ZzSdOQOR3eRUvxLyAf8M52qOGOiWKO8xBmB1a1AIUp5Swff52N7klnS0Ui96CTQdTNqeTmZXpNvkDODYxC9Hfi7C8tXRrdtPh5fIYi6yN5O5P7sZwvgi/Th+nV8gjjgUAHD9bw5BT59E/eoqVP7lkSoK75rIo6hbsg4ZSPupqJKMNrUVLZXwlu+bt4uGLD1PRUUH5nnIKfi9w3SsnC0lEkEUq4yrxWuySham25NDqfRCLRmHw75u5j29v+haES8A+sgq1XY1olTn53knyflVQ9mP0NzCs6isESU1jeCOqmAAMfgaSg5Pd7sfLgfBeMOTP92bLC0fdlgmo0Jl1JJRH0Xc811k8sGqUAmpcyxqeW6p4r5XuKCXpwyRWH1Ik2TzNiYgey4m9bgIOp+SKOygmvM6X2nUH6fELcHuONo2SkG0JbsERrFOSdMC03AKWZijjjvq1Rrz+XIe2z+gEhYS3X8MKdRJTVXs5lJFB1VAD7z30ntOLZoQOXgqCbeHlZNy0yXm8U2OVL7HX+yI2dTtbL36I5RELW27bQsuuFtoC25zvUXnYW7xYEMPFNa/0y5kpz3BjL7zQa6MgtQi7n5bMvZuYuW8mGrs/HhbFRyasMYyp+yLoO5uPV0czAT2jGF75Hf69SpLOZDDR4+dJn9Y9gaIV4GrPNGaHPIlN58W6Veu4MPyCc3mECrbVPsWnj72F2qbh4V+eIqhTeQ4jdTAp7DCf+vry2a2f8dvVvwHQZ+v3PJY86Pa4SK3qMJ2W9v7nLuBnyiSzbAMZ1R8CLslsAA+bIrn0UiA8YdAQU5IEzT3IkhnR4erMKkLfI6gliOHnkkCSaQ8GEfcxLYDG5qDzS0VS6OKIDqpCPlHOTVBTG1XLjitOgbcXVf6vsmeYy/E0pHMh165aTdwjcSTOTaS1VUA++jhzsluIq7+C9Dw/mvaWc9Nngxh3wl0SqNerlz+vLEay2Fm0eRHIAucTrndbRzaZCKoQ8enyQdfPhp5vgItjW6l46Q16hiSw6NAB5pzORZ4A3TNhogCztXq3/neyWvm9e6aM/5jBFKcU0+srYjun9A+jzo0isHcKAFck/MhbXjrScxVAR86KHOoj6p0KIN7mQTw77CRVc28je+ZDHFuymvEl2ejsIeQkfcTWBVspm9HEsfnTaI1S2FYmGZ5qhU/3vM3CTaPRp8SyftUfdHi5+7lNOzCNxV+bqc64gq+umcbplPmIQKIGUKvQTJqM0T8ar95QgjqnICBiq72f83kfILcGMPL8SFTzZnL+ZZefSrxGKZCeD2imL6WP7O+yac5T2HoDcw/P7m48P3+fnQs+JvnUOuyqy32KLfOV+x9ZF+ksUAN49EUx48T79G49yLXrZrkltLW2EMaXnsBf/Q11GTo6AjoQJRcTY0BGNLRBZs66SUTVRmHSVrFvWDTXN8I6T6WNL0nrIPGdezD5hhPdqgAoolpXI6IiqDWIpH8mMfcdhdE/kLQcSKB1hXUx87WZOLSuPjU/6lE2zbqJi6t90EyfRFmiMoZ/adBmZl9QmFBBbf403PcaaReVxKrWHoynJd7tnrR61tIS3IpdbSc77ha2ZgoUpE3mx3OPMPu3C2gmj6Nn+Wos6haqQlwFUofagU8neO/sxLunn5Vk4X/mJYwWSQU6s86tjy0PfZu3a6+k05Dldq9lwcaJ8Sc4uNLIiGMqJh9W2qdVed4cGpyBgMiFEReweuu48/M7CWkOdWPlfy19zHF1JipTL3NPLiOm5Q4AzEBawBGG6WX2zdiHeK6e6fv/4qcrC+h8dERPjEZlU/He3Pe4Ou06xhbvJqbldgB8jinvgrav6zIGep3f75z76hwJeTWXWVAkND7CrqtOkRmRSXVWNVG1UW7LOy90cv7L81QeqMS7RrnXXuY0wvsZOSNOqdg44iX6mvv+4perjN3yMvLQJwVQ3VTNacOjznn3qbpT/JjzI9lN2dQ1iDjUCsBIxPUs5K5uar/bR0BDPt2eLrZFdOMSUnP7cFTUO+celwJxBucOZuSRGjzHZFCWeQ12tQWL1mVnIUgyace/YcS7F91UeooinyNX2sDkw5MZdayP3x/c4ubLPhB3f3o3SRcO4pAvB6MC+P7pi3DyxCW5CQEBgZFlvzKq9A90Qv876ybjeTmzzyFcmocQ+OKOLyhfEUBXaAohlheYlV1PePtVAIxu+BezjzyLf95Rxh/kMlDUhOMTWPF9MFJKGnlTI9i8eDMA/r3jEQUBWZCxa+zIgkhR1DNUhn5IfrQibR7euYx3571LQFY6fv2S447+sZ9D40tRhgfJIyYjyd3O/rNFKnYD63b9dpoH3nvAOf8E8O4bjM4egl9TEd2jgsjKzHIWMbYZYZO5B4vhLsxVvzH2YI0bm1FG5p3Q3dz16fWo7AL5L/ZQkOQqzHpa4rnq+l+4OPsZOn/aQeLhgeKma84mqAUawxsJnhhMzl05FKcUO4uuyCId/h18c9M3THl2Kr0BMUgqpWDr6P2cQ8ue56dPfyIpIIk559vQ2BRg0PVJD/BqwquIV4mcfO+kU95vd9nuSx8lmkANU/8xlYo4RVnnbOhZUr9WiiIlO9yLIAAnT8Lqa8xsv2s79XvqL1v+dzH0+qGsyV+DPUSZewfWXODkewpD7MhrRxi09wM0fe59lSxDYEogT/U8RXXCNOfvfrVJTHxlAgFvyfzyVQT+b2hQ71dBjzKWn/jmRB6Z9QhHfzrqxsYDRf2rPaCdE+OyqDleQ8vW06h11VyRvJb0gAvK9Y27QPQTK0ldnErIUlcuYmCecHbtWbav2c7Hgz6m/mw9P3bcR0VI/9hOgk03bVRu7YD8fz9YZeS5kbR9vZvQKyfRFjVMWf+SYl9gQytV3+x3Kg8NgCvzoh/Co7uRIfvex2Fx4PewH41Gd0uc4/E72L2gmK7dp7GfUYpDQ32m8vTkp6nsrCRb/SmRRUc5cdPnYOv/Fvr77HLHEV78+EXa/8wjsDabOTsnY+g1OGW/baouco7mEFx+mtiCbEKaXPdEYSYq11cY9aTbObX67MOXNfgmTkE9KJXKRHeLnqi2G0k/W4S5oIL6hHSqIo8iCVY8zQlOYFz28GzMPiKzSuY6tztngWF13nzRZ2fTkk3IfWbKqt5yLs9KWorOFs6m1Zu47sR1PG1/mhvH3sjD4x9mel4hC7NkbmAPnwz+hO7vNyOgQm+LQCu42oMOw3nmb7qT4jGryBmu5JlamkVFuUZWITpErGdbSCx1B13aVT1YLb9hrc/CHhxOY7g7G9vsYebCiPP0eQXS19FHuG0U0W2r0Th8OVC7jZs9api8fyoZ2xxo7w/gp2s/ISfGVUBVCSqnwldjVDDGYVNYmLKQvwuz3cyow6Ow3Wcjt9qlCHTwICDL5O6uo3HdXqcfHyjvb86MByjLXOEELXq1V+HTWoHO2E7D2Tqn/Od/4v/7+E+x7z/xvyIEAQwGL8wW10A8wDgJH88k8m9+i4RzSuK1yW8ThVFPE9g9jczy39k7+w0Szv7mNnGQBQeiQyK4ORj7iEC+v/U4TWFl2NXdiA4P5uc24tM3FE9rHP7f/8DcXa7OIrX2FfJN+ykRnsKrqxXLgS4SLjHD9TeNxVPtxfkR56keruLsZ2cZna1MegK7FbnE187fT/vz7Zz/WhmAf/qpa8JtU3XQqD3Kttd+Z/Vn8W6dI8D5kefpGhaOWep1MiAuDUOPClGvReP1AFOLm5yotIEwaStwOMDfOIbA3qloUBBeHUIZezSv0RZQz4VnPsLWfqUzCZEbu4bctnPc9+F9RFZ20vBFK9/Kr7jtt8lWwgfCBxxOPYxPlA81Q2qw1Xb92yjVO+2gzbNj9Qi9TP5l1p5ZxO7poSMig96AKBxCHya5fx0Z9CYtDpPlMkRxZOsNrNXEk3nSF8u36zn6mHsSVdU/IDZ5mIlYPYuSJGVwOoBeLAv/F6v/uZp1/1yH3Xj5hay7+hYsC73xP7GD4CrFn2IA5SgLMp0BNswtPSQX2JyJ0kvDLhrp3HECv0ZFikmSZQ70wZUNcNaiJKGL8jygB8Q9sNvx30sc7nYo63UY7eyYt4OWTBNNOU10HehCbXMZ8ar1agISA5y+UqlBqays6GVExToSmh7mRuPL2M8rid661OmYPZTBU7f+EJ4dxXQ0VdGePI6u0H5Wav8n2Rozktxpa7B2W0g6d4zQRvfBzJxdc0g+6vLR8DQp795bnXBLE/xw/Q/kjvTER+tKRCb7FvDG7EncG/Mh7cmPY0kdTndcN5KofC9fdUNjYTzXvX8fnk96Ijkktty5hbULXEkVD2s0sgxXrr8Si4cvkuheePg7Gc8x3suZmn+RVduKue6bwfSddZ+YDHyvKrsKlV0iZNpg7i2+l+iJ0WQunYzjYYlDU+w8ofZlpcqXV8MS8fSdzSeLP2Pw1IU0+W6hRexn40rQWtjKxhs2Ura7zFl80Igapu+fTsOrrsH5qfZtnElcRmnsTk6NOYUYJtJd101fe59zX41+G9C37yB0fyEXvjjDhO2biK1McDv/2MpY+rJyqB+5kH3zXai5rRHwzTW/UjHmVjT+Xux9Yi/5v+cjSTIXh9yKbWYHQ0eOZ0Hc/Xz6fT7PPPssYWFhSJKrXV53wzqKxiQy6s5RBA8O5sR4pR0Z8IvMGp2F92xveup7+OOqP1D3qlHzP0tyySqtsyAM0OZzkLCKYrwObMXzmoUcX3IPRoM7ejwn00DfyAkIjj5Eh4g8AayzYJII09T8bWJYnA0/LoeYK47w5T3n0a9egcfsSZxIm4pFVooBoqRX3oMYFTsDPWgPvHxA61uvQR0exLHM15y+U6C0x36mTGVwPCCd0t/+yIKE1thBbPZmjHuPE156DL9OT5AFItSwOxJu8lG28zlZ7DQ/BwWQkG3ezOw9sxmU3cPRK047JZHOWOCWTZX8kPgZ7dJsZG06QkmvWwIjtvluSoVtZKVk0RquPBkHdgQgr204N595mpbiGFZ+vwK72u70KwKIbLuRuJp5+LV4s3vlBY5NPHaZZx+ChKmsmqiKJnKir71MslBtVxN3qIcH338Q3654vh5cT1DvNJosoWyMK2Hz0pPYkrycbDPlfvUnnQQIqM9F+3QboU2uNqiys5LzjedJ/SaVZbnLOPTPQ7wf/z5Sx0C/IQIiWaOzCHwhkA8SPmDbmm382f0cJ1On0+yrMCev+v0qFm1Z5NxvduytNPttIT8zgfaHJ3HgHwc496V73+zTN4KArkw8J6dx5/k7GTFiBJNiJjmXp2rh0EfXEHvSiNlTSY6NLPuNBVkSK+17CGkOYfHm2egXTeTQ1ENu+/Y2DeHKFddw18W7GPLLEGq7XWAZAZGu0BSS7p5NYF0Ovk3FeFhj3LbXWtWIHjpqUtLp9b6cLam2q4l5rZWZ+2YS1D2TVp99fGAt4uNOqIzqpCPUyvRNp533RBbsZAQfYWebjsTSRLSd7vuss8OxbgXVumJGDlW3BJE9wdvJ4Lw4chH1gT/R7dPN4TmHLzlPrTOJKshqJFHCs1WDX3Mx3n0KS3SRAfqSYIU3SKJAxaBUCtMKESW90zuoNqqWbYsr8L//Bi7MU9izWnsQEwuP4dWXjtqmxrujk7APtjLhmAulP1wHHkIlZ0uy6BHXU5pc6vym0rVQlwCH4iQSAnvx6QnCv9qHkeW/APCIP2yKgM0R0L64nUlPT6J4W7FiCIrShlyIv5ENPlPJbc+67BkMJPIvTaIG9kwDFL+s0Ig2Xn3uZRxjo5j761Fm7Z116cZE1UYxa08mYmM9E/K3MPe8e/JkyMUhzNhiBFmmLdWbi/GPOpcN85tKl18XVbGNNL7yDRElh7GrXTJ6oV0LeWz086x+azXX77ye/HzXfhui7Lz+xOsYpzswe0eR0uSOVrZr7NRHd2FvNDIkZwgg0OLrkmr1MHkQ8ftuJqz3IaU4BV/TSKJbbmNDP8GiyAqa+hbSj39LRK0FX+NomAC6e5JpnjOcB71DuNYjgDV+Ko5OBsfDIE20UuyxjtNjT3NimhbN6GFUDllA+YgrncfdXXEdX9d4c/XvV1N3po6iqUW0Bre6y/2r1diCI7F5+OBlScbLkoooa+gzOMganYUlXoVONZVAk7LfPlmR6y2oHUFAQz7WhjY3Bu9AdPh30BQl4FCpkUV7/3OHiAp4tqWH97tSkHvXM/e3i4w/cx3dntlUhn7AssBKJo/P57UnX4O0JIp7XJ5pLweCHx5Y2r9lkXYRux7aRd0pJRHlTOaLamQPT4JGxtCYNAm76I6mH194mLiMGZyZFkNNdI0b8ya6ZQmDTvxG345D+Hd4o7arCeyeTkjnFYzu96YFUNssaKwaVJcUoC5IP+Bp9GTiARGTV19/UVV533/sgWNyBKNPj+b6r1Kw9/QhqbXO+ybKWgREJhyfQN6teTisDvaXH+BIwy6sqg4GUhxpe9Io21WGTXbNEfp0FXQZztFLE4IgOxlIavTO/mqAATv0nMigsnuJa17DjJxyYpvvIrA1kOf++Rz6dpnvV/9Ml18XHV4nmOkBlRn5jPe20xTWit1LS7tQRLfnBeexPUUf1DY1rSEO6u4MoDVI+R4FHf8jL2Gt2pspWw4zZ/cctz7WxeySKQt70/l7XeCPdAR00BppI604ndiqWOcys66G6NZbmXmxCntoEvun76fbp+eSfUFQ3xpue+B+2iKt9O097myXSsNfI8A7myCVwgS3DwuhOaTZ7Xytmhb2V+wn5dMUnux4ktSgVH4r/IlTKXPo8lTe1fLngMcfo2bwPKJab0ZndR8VnnrgFKnZtXj0eeBrzGRhlszCLJlBtW8h1YzBUefAb4wfG67c4LbdsMeG8UTHEzxnf46T9ymFArvYi0NU+qaOAAf+GZH05PQ450p/DbsksWfCHlZ/txr6i1oeag8iayNx3OcguPIMvq1daKwaNwacLKowpEVRPXieWx5Crcrk4oz70Ywehv+XbxJZsIekxqecy0ObQomq7MZhUt71v85vl2wYjleHAv4aYM0DWLT12OgjvC2BgBYZs8MdtDIQJ8edxKbVk3Y8Fq3FXdlIQMB3ny/s2k1Cfv8x+t+DiI6rCe9chkboZ6MLLhlP0dlP/T2zb4BpNNCPaR3+6G3hjCz/lQVZDoa03ErC+T8IztlPfImMp8mTiLZrndvXRtWSN7QXx5Tp1Ea57FN01ghEQaQxvJH1K7fjaO0gtlJ5t9u9j9DpmYVd7KWrposz9/5A4lkFbDOg4KIS0+mJe5xhGfNZ9ZUHSzff7Nz3wPnf+uWtmDZm0+fRhyy45jtNflspDn+RuOzfUXVaqIqrcvYTY/QKI3vajnmkH7GSn5lJd+hEEhseZ1zRfsI7rkRUC0iiBpvWQLHXReyaS3JVOBAEkAUVuuFphJQYGHZxttMCAOD+0ffz+JDHCfQLJHl5GmrVZDysUc7naNVZqYqrwqTypjz4dYrDXsSibqHN7yK1wVWUtZZxbu05/NrMeJsVcMtgv9HcM+ceMu/JpGhTEUOfH+oGIAIURtzBGpouNuHov49nnj9D0S1FzHx9JoOvVvZl6bZw4t0TVB+t5pVXwKoxUDx2FdMfm44oipRsL+HXK3+lJd8F1D7zyRly1rvLHA4Q+4Krz3HsDYWFrdarQatzY/15tVfTW1CDw+rAYXVgsyrPqqLsbXq+KkNrO8Xb5k5+tLfzMq2MP+lwKiN5mD0YeX4kZevKaF7ZzJCLLpa3KGtpDW5l3+yDrClYg++D0zgxaAw2GZo9T6FyeKKS9cgyTHx8IrFPu9pWm0P5due8PYfMezIRBIF3Tr7D/t6PyIu5n7yoh5EkWPjVUspGXkVTwnhkWaZVp+QIj046SsDjywleNon2KGWseymzb9ShQurWHyGqcK/y3vR/k20+B5AEAbvWA1GrZsFPC/jbkCW692chlSgKDQO5EFEQUTlUhFUrTFqH0O/JOTBmkNQcnXSUP57dhdpqJPPsMCbn7MejHxBTF/AjVxmvYs/CasKqKkksS3QeUu0wEN12E/eluoptzvulUvJ7A23IANByILT2EIrH3UjQfSs5P96Tw5l3Uxb2BtNzi5lzoZWxelhogLTjfpx/5wDRLYp/7SxP+Je/iRg1GA1GrE9cweEph9327d03mPlJ80nySeLz05+je1nH1b9d7bbOuIfGIVutxJ/bgMraR5reBXbu9SjELy2M7pAkdIzG05zE+YRVbMsUMelL8e3yRSrv4Vxm2WXXnZh9AJ/9u1AJjsu+t4Hwrc3ljYA3qBCWUBHyIa+9Bucbz/JhJ+QWRqOtLUeV4kFNTA2XlnhEQeSnVT9xZPnN1A6/h7G9r3LN4GsA2FW6iyf3PukkRDhkB7kZuQqQV3bvP/S9rQw6+iVtm46hN7pytZeyVC3NXQR8/z5ify5L39vG/bWPs/zH5fzfGqIokpqaivhXaZb/r47/f+So/4n/xP8PYdU1ctT8OZklmxhXtA+Vwwu5qQXP1Gh6gpTOxTnwHECWdPcR0JCPZ5+raCZjR9fXwZpP1mDad9Et0aiSPdBLwWhNnQRWn6dvcBq5GS5kg0M0ISCSm5HLwYWpzPvjdtp9XAN7BAeiIHJmzBnKRqm49eStNIy5i5Flv+JhjaEq6AsQBX5a+RP6zm7CSo/1n5PSKLZ7H+aMdikVb+ehtarcpDadYbMjfXeYzDOXm53+vqqSyIdcHZOnNdbpE+Hf7o9v6SM0PPs6w3a9geiw4XC4Bq3HJx7n95XnkTutePZeLolZmFZIXaw3Fn+rm86/j2k4qn6UryRLxE2N48htR7A57M4EfiPwMjAaSOn/+zKuiW0ooO3WEpfnMlYeiJSSIQSU2pFx0Oa9nx2jPKlzXACUROQj79xNxxtfMPiCv5tPobc5natSCrjj5c8QPD0IPuEujyEKajJyMrj+hzno06KpiFXQaWeSlrB1lEhtwDoG5Q/C/q6d4hnlfzsJlGQHbQtuoD5VKeiGd1ypyJCoJLYur2Pe4afYeJ2eDv+Oy7Y1NBXS9tNuvBr3smdoOB+U3cNSA9zuA1dsu4Lwsz1suKeeyG0j/m2G5Ew1vFQjM2P+KRwZBs5+dpZTq0+hsyiJGYPWgMPmoKumC3OnmaItRdSfrUflMDj9dfDwQNAo30+7XxPn0hUpiYaIBl559p845ihPVRKslIW+SZ74E3nRD3J08FJKYrdgrOsksqLSTWIluuVWkmquQ9vXg3zVVRSMynRKf0W2Kt4alfGV9HnCN/Fruf20gvQ97/clzRYv6rRLAOgbNo6Nt290Sut+snQ9Z6xKMVJVpuLbqd/SXdNNa7krmamzhyIbTXya8Sk2nRdhpUcZeXakc/nfyXgODDJlUcSh1iE7YMT2V4i70M9E6Ue2Ldm0hJs+i0XUqglMDkRr0Cqm797AVKh7tov4a3/i7ce38tJbL3Pbn7eR1fcLZ5IX84E5k7yARHoab8cn2ocVf64geUGyq9in0pBUmkTvgV5yf85l92O7qesuocn/T2qja9lxxQ5UCSrei3mPzbduvuS8RU6NPcWRVT2EjomlKW4MDp07aGDisYkYf9yMzT+E1lDXZHyBAeI929GrjYh6Lbeduo3Jz0xGusRcILEskYQSDaaGLsp2l9HX3uc0uh8IhwjFW4vRems5OP0gACszXCwsu2TH0m2hYlsFOkGLD/+zJJdGH3DZ7xsXPEfR0hRkBE6mzMXo4Y4slf3uwuEfxLhta5lyWGFwRE+Bmg9BPcGTJ1V+XKvx575gHVljI3E8DIOmwUedUKm/iUDjUkodh9hjfNNtv8Hds1mwbQG1k2pR9SMfL2VjA0z+OYqOH7c7Jaj/GsrtVZYNJNDqAtdREno3wTXnsZy4QFThPvS9rQiyiuH9eda6Zj9iy4MIOFbJsOxhl+03KzOLC2O8sDpcz9hh86HFFMs1w+9gZPk60orSCX6/lsi6SAzmFKYVHWRs90oMgsDuubvxa20mNnszCA70Avw0exkPJf+KRWehNqqWXi/3Yk5E+wqWbV5F9NeF1KS00hDRgCBrSK95g7B2JfE9bd8IpM4eXn3qVTr9Ol1IZKAuqo4P7/+QkiuCaEwYz5SiArwJZ2vRo9yZu44LPu2UpNST9MwjZEzc7vRCkHHwa/V7HOm8Hk33aSxztE4240B8ff5rJvwwgXey3kFj0OAR6IGjv+8VZIXZB4q8kyHUgM5H52Q0DsjOVMZVUhvjuuaa4K9BFsmo+ZAEcTpTv7mJ00mp/HRqD3W2PGxiNxMLj3Lbl7fQ9uxGZv8wm4rOCvbesJeZ5adIqfsnhVaoL48gsNyKV8OAv7AKAQFR1tDp18meWYfwGjKIG4S/Mv8UydXQIaFoh7sn7Yz6Etq9jtLS10R03k6Cai4Q23KP2zpV8W0M//lJmqOjL3t/AMaeUnoepQ8TkJH4qhvubYEfVr1OzpQuKlJDKElWvrdG/z8o6/OnsMGPG9bdQPpxG6eiTjH/tFKc75YgTKXipDWMaZWeWCI8ScszMuXQFOcxO7xO0efZx4kpJ3jW+izji8YrUmP9SVYBFZ3+nZgCraisfYR2LSCi7Vqu8YKeDi/in3oKw+4yalMnYgqcSljnEmfSzuhlpDq+C8HHmx59PhUhH9LUX8gVEBmSM4QJu86h7bU7PTIB1ofBOTPUf5FJ6gX3ifoAWzbbqKKzPojSIalk/v4kajmImJY7+KRTWX6iD3ozewkdGsr6heu5svtKpttfx7tviFOu3dL/rVrVrVyIu4mLMXehcfjh1ZeOj+wq1A4klH/ugWE6WO0Ddo1AVaKahnB39HZpUinrr9uFrjSfkbteR21192kNbgnGt1PCsHQWJpqIrVDeQb/ecYz0V0BqNq0NzxljqYqtcttWQCTvp2w+H/8tD93UwYEDrmWSSkEk26Nlisevpi1qMH8NGQdBj09l7Zrd+PS5+6NorVp07V3YtFL/cUVA4vMueKVei/VCOrIkUT1oLr3+UeitShXQS5OKh13m3p57mDL3Y7ofdCj6z/0iHFrJl/ujfkAvLsLzuqU0xw2lz8vTOQ5P8svhrpgejl37G+nXpDv9qS4t9jnsMmFHf2fwwY/xbSoB2b0PlK0Smr5uNDZXIvI+X0gZfoSjS29D8NQz9lg6f43s4dkcmwUhladJK3DJVQ6X4tjmn8V1780jtrielugR1IYpheGbfeCl4e/wcYQDi96CVFVD4EWXmsSbHfBR5XSGZmWQc0cOM96Yi5iazK+HLjoLZ4dHXMsft55k8jc30xw/1k0aTG33IbB3MqqKPtQ2Kz0+Pc4xP0BZxFq2LthKzz2j+ODBb2mIaGBwzfuMKd2Gv3E8gsPGsD1vMXNtB1duuBK1wzU2tGNBEiUaIiXOTs2j27cbleSFAHwYDBMc+XT7dNMUbqKw6xiH0wZTFKl4houSDkEWuTD8At6Z3tSdruPG329no2EevfoCZ3t++pozjH98PNO/c7HN7KoewuvDiTxmxPLC61zXzxI/2baDZt/t6C3ROLRBGJZMw6b1JK32n0R0XE1V8FqqQtZi1VopTinGFAgR7SudHpcB/TU3z8Auflr1J81+9ew1T+NMkgKGUDu8mR1wJyHNIczZqkdTY3F6gBkz+Le9hL/V6Ek3RmIy6ClPKHeT8RxgFhVGPUmX4YzbdoIkgEOi4LEhbFmWT2zzXc5lOnswUaVtBJw8x/kR5+n1Mrntd5U3TPeEsws6sa2Z5mQ5LjRAiLaXeodIt283xpsHU5xR7nbcNu8DLP5kMT+P/pmdzxzh45NfOJd1Gc6hsQUi+nkgeyhzOB9zBvHND6KzRjjXi78lkZZwbyw6i/v1okiMdVR0kP92vpP9PhB37bsLvZ8eUSU6CxPnE1dSFfIpAGWpZoLHJ1J6Xyl+nX5u22qsGh5981HqPjxJ+IJwSpNKnYUvvVqP1qqFagitOMWkXaUEtAc4i15nEpewzu8Kgl6dQ3P8WDdp0qaAXdSHZiPpNUgBQdh1Xq45GHBw2kF2rhhLX4Eig/dXO4jqOGWOUz/UE5vWhsphwL93PACJ4nSiZn5L3vRHST2V6PSWBZxszAMzDmDTapi4fQQ6i47kekUdQJS0vChKPHFCUXTxa+lnTcnu6cIB+URBVuFFKDpHEPTnXSTZ4ZxDRbW5vBM7vU4zOHcwAeeNjNjxCtrauzibcE0/Z1BE9vCmdNQ1lM28g/W3WKmLrHMrYpcml3JycgfChXPM/dHhLFY3Bmxgh1lpEzxNOnq3HCS+Qnk37aoejg4aTbvXMU79qqgHNCUoDPNRQZPIqPqYmFblnggaNW2RQ/HmXjR2RdIuK3EpAD3ePWgnJ/LDTTsRhQhmZtcyqPo9zidcR3HkCxyf5qB1spJjGOgnAkQ40gf7phwie4YaU9gCguz/IL3uXxRHvMjR9LFUGArJn3oX7dHDUHXIbqxVWXBgb27Hr7kYqduI1iRwxY4riGm7xbnOM6OewfNKT7KezuKpsS8zumwj/sZx/eehZ7TnVVyddjXHX9rLNd9YaPF8H5u6jcZIK9/d9B1EwfY12/FtLnECOH4q+5A6bR0LPl5A1IQoPHs9ifd3ZzQD7Fi8g7XD1jLonALeqo+op3FQI6PuGEX8DGV9Y4uR3Q/vpnhrv22BSk13cCJhg5U8QntpOwV/FDgBqwCH/nmIrE+Uvq1sTxlZa7MY8L+pGLaElduVd2r8Q+PpW30nNr1LXSvuwibqv9zOuS/P8bLuZbzbKqkoe5u6omc4Jsvsx+GmjHTILjuVkZoKG3ntyddIvT8V2SwjSpcUS/q93RBkgtKC+ES6D7MMp8zQbkxg/nkjy3f/QdP3u7Bb7KQHpxPooTAlB5h9IRkhLPh4Affk3cPbrW87910R9i7NlmrSr8qgI0IZI8lI7ItWxl5NYU1o0qPcipoelzDML46LI/ae+RSPdVdFADD5+FAw+Q6MDV3M+2HeZcujaqKYeDCM0MdvpHlRHAWRT1LTV8j5hvP0WnuRRInmqFhSHpxPh7YEi7qZCYVHmZZTSJCcjkVvwejbR0PyVC7MfhS1KhOVU+5aKTbXx3Rzfso4CtJdiio6eyhhnUu4Nubhy87Jru4mMvtD5Fdewfjh11z7kzsDrSboG/JSD6CJDHG2iwrwR/l2TkbD/QdnY+hUE/p8GjGttynvhhrmeTpIVMO8nfOQyxswerkXswREuqq6eFn3Mm1XtSFIAhcaL2BRN2PUltPAOWa8NANNdBiB9bkUhz3FxT4XQE0WbBgbu9EZ20mreY4xJVvd9o/Kj7AlY6iPcAfcgdLmV07wIfz3j7jj8zVuy/w6/Hjw3QfxrS/EOt9KbXAWeTH3M2SDwI85P3J/Czy48jvKV1+FQ3L3zB64LoDBR3eRufVFVN2unOShqkP869i/2F+hqIHZJTsVCRWcHnsaUef6BrRakNRauoITCb52Bj1Bca7rlsGvoQBDRw12owVVTycAxWOvpys02c2f8v/W0Gq1/+8r/f8p/lPs+0/8rwhZhmOpE9lsvw+TvgxR1lEZ+gGObz5HEEWa45WE1KVm0QDJ986m1y+KqQXlpNS9oKwj2DF5mjg49SCSFgZfDHVKUEmClW5dHp3iB8Rf3IwlLpL8wQpUOUCM62ddKFIRPX5aAjIiCPdweQ+VhryFyd7Tfy4SXhFedPm00qvPpzboW+oCf0QQRYpTi/Hs6cGrXUmcLGs5zfyzZoK6ZzsZT2VDxl7Gcrvhp6eJ+a0Q8ezlUiUD19a+/STDd72BR1fjwI+AwqQadnEYQp+djkA7e4fG8n7tdXj1DSJCHt2/qkTHO1FsXL7xsn3vnL+TsnR/hDrcBqSSYAFZZNHmRaQ9oiQn7JLdiVJ9F4gFjgIPAp/2/z3S//u7KAl8USti6GokudglcebfO56azDdxrH6M4bvfIqD0L+bmokTWqPPIdgeTD8Xg33kJmkrVSYvaxp7ABsrujuT02+5SJipBjSzISKKMJLsmYV2GMyDICKg4PeY0UiqYRMENzQcwd+dcer/Mxx4VR5+PwiDxN44ltvlu57MQ9Vqiu59g/nkr1/t+joclzrl90sm9iB5aNix+DYtWeVafh8DnoTAyezj+5WZic7cjl1X92wzJG23w53F4NUgxdB56w1B6Hu7BorPw5LgXGB0+joxnMngv5j2Ov32cnxf/zLO3PsvmmDTsohFJsGL2UzPki1upTZtFe9Dlx5AcEkl71+JT+Q4F0Y+zTb+KitD36fI4iE3OJ3TGYPYtn+Qs5gHo7CFUD19N7swHUA8bTGfUJFIa32O19/fUjv8BOVlJGMiiRMiQMKQgZbCSqyojtLqXj+vmE3nxY4SyHFSC690bHxiLeWoUrz35GtbZVuKmx7Fq+yosH7sQ80fSR9FFK4ZQAwgifT6h+PlHuq5HsCFJoLo0V9L/qNvCAsmefjVeI+LoDYxzGhLL/YWZ6phqitO6sfXZ6K7rxm6xO4t1l9wxyn86xd4nFdSdo794IKIiuSSJcUeiEFUiaUvSCEgMcJPx/OrWr4jaEkXpjlJOvHXCqZl/aQy7YwxycjJ2O2zxm0tZ+Ou0B7bTHGUhcmIcNRnz6fV1H2QcmXwE3Y2LECQHQys/VBgRwNUN8OqpWMSqV7G09RI5JhL/eH/n4BHglxW/8MvqDlqPFbFu7jqacprclk88OpHwsgaWfLOESa+42Ev37bgPgFXrVtH4ZCNBaUFMKZ1C81gTDgG+uezK/j4+FwQGWdMZlTXK7femsCba4rRILW1E1vg55R8HQpBVSBodrVFDiez9iFGlf1Bjh3OCLz/O92KF//Xcon+eB1ueYql8HI2HP8N1cK8fLNdcTWrZHOSf99K1fbv7Ccki1THVCD4CU3Z1E15/ua/HxamN2MpqWfLnErffJVUflcGfIssQ1rmU2OY1eF3CFM9L/o3XnnyNxtti2bkUjo14CbXkg6cA3Q4Qj65k+e8TqLsill+v+fWyBFhxajEmg8CCH2YR2hjK1Nw8luQ0MD3uBzQnljMk/xvicrbSvtCf1qBWPC3xLAqs4ucrJjLDsx/1bLOicliRsGOX4VDdbIJ0XWijmihNLGXW3ln4dLmSt0cHZdIWORR50iRncVOQRRKbHiOxSZGSDW8IxFbZhFVnVWSv5MsznL2JkdQOmoOk1pLTUEh23C2sTLyO+/0AJCQJerpUqBxKUrvN5yDvFz2Eo6mQsNpO+kaKTi+agVCLaoKbg1EVqpjwyATuyLoDyVt5T4R+Zt/IsyNpfraN6d/fzKzXZrmKfbIKAZFd83bRmfA0WpurcRQQ8W6twH7qLK99E8qfqk9YtXMOr3RksGukL+Wh79IZlorgrce4zsih3ENoVBr8TWPw7x3PmBp45v73kA1BzPlQGctYNPWKBJasXMfZUdlIEqi73VmQnYazfP7bWr674zuaz7uzKQB6rPdw7rFvqR+3nLrU6W4Sy8n1zzE1/yKt+7IZcvQgvp2+aGyBhHS6EMCdfp0YUzXkZOTS6rOXnLg7SNXAhRi41UdhsedlRnJhxAXnNvM7SvhK1c3mRZsRJIkdt+3AYFSht0SjESBA24Cn1zOMe+1NUl6uRtclkVDuYh8PFL4csoOTp1XsPTjAenUx+wAurqyjeohyrvWBP2OSwaJyUB1dg8Nfj4ctmqHVaxlc8wE2decld0XCVt+Cvu0AJaGPUhWsJH2DuuYg6zKoTYin4ea52IIfdCbAXmqHD7pg95zd1MZYefalZ5l6UEH0HjCBXxksLQhlxQf3ElVWTIupGUkw4987nmNmEErg1Q6QZZnfrlKYBVdffzXjHE/gZUl2yvdaHErfZRO7qQ36jrrAHxT/4Lx8lkjfOfumAQZwvR1+bFPTlZuIbLTwzQ3PkjPUHRW/yHGcGNMmLLHJVCX6UR+ww235gts+5viS+9COHEx0kScpxSlA/xhGEEksTeSmr+ejSoilML3QuV1804NYVe1UVpXTcLIKeeduzp5VfEePp06jwedLQhtDEXpkuvW51AX85HZcQ6+BNe+Nw76uggmlZxhbsstteY93DyU3X8GOe1v7JdoF/I3j+WfBKn7KvYYVv67Aq9bCrpkfc3zog9jU/YhwwUF9RD2nxpVi1yjHuVS9Ntw2iUGH4xl6vABrXgl9vbdyYJCf09vqWN1iVndZODUoj+7abua/Op/E0kSnZ2CftprXL47HqzIPj95Wks/8REnY8ziEPvzatKz5aA3B35gZtu9dDE1KIihWDR+EwL/GPcG+YbH0Hs9h/Inh/F2IDpmIsuMklyhjYRUwxbuXc1Y/TGobHUE6qoYtpilCAR0YJeiVNKyu0BNeH460bTdz/pjp3N9eE0zwqyLW/wxRY6PYmhfPs697cddWl8xTj0cedaojNPTXiR2qS5Jg/e14+7YTpJ9T2pgBT0dQJCmzRmexK+ILRpX9xpjiHXhYY7CqOugwnGbf0CSagpW5TnlCOTq7u+qD2cPM3oUWcsYr75bWHoQKpe+9wboD2+ByamJ78XinGO9OF6vUp284UdqhRIyOoCerh+zvs+kyDrD+dPiZMpmWW8CK+pX8vPhnspuy3Y4bWRdJ/CGzosTRDwx5r+IWLsSvJqLjambmVOA1byrZcx7DalCS/41+fyr3y6eHX679BYsB5u++jsyizwHYZITgcuWvLEgE/pDHrL2znO2IhJ0VoS9wa8BJ6hPH0BeiFOBESQej+Le9hA/aLYxkBJ5GM3kZeX+Ryv6vUzsrNr7AVe8GkNY5h8mFZ4lrvs+1lSCiNXeja20noeFuEpoecduXA9i2K5PwC/CyfjQF0Up/HtI/7Hg7yMKjbz6K9w/l/ENlYmGWzBVnLcw710tC4yMAGOoM7Pwin61lf7id19zsVuZab0M6ctQ5d01qfILZF+ucBb+05wdzcmYcNq1N8fdEAYHWBvxIadgbWFOsDP/XcKcc9ZAQBTzQ09RD44VGCv4owKfC3V8YlPYiaFwSEU9EuElZK8tk2gLbEP30DPrnIA5PPewsIqtFNRUJFXSs76Bi+FIKRsSi9UzE16bMg9u9jlEjHqa7sBqvtio3Zl+b1x4KQpdikU30XXszTQnj3Y47vHI9QcZ59OWUElJ55jJmX9a4UrIWPk/RbKXQEdw9x/ksBUFAEEBr7mHWlhkklyTja8wkofFhhlZ9hl6lgDnqklI4e0cngaaVhLcPgIQFZBl8Y3xRv/w8Z6cm9/+qjJOafLdQF/ATXRalrfU3juMpTSNza08Q1D2DiPYVTolpUMa1o0s2O/9/9p7ZhB810hMaQm34eRoCfqM24Ac6Pc+yIS6DnVPvo883TJHjFGXMmjqCumciSlqWRd5DUPd01FnKFzLqrGsOYMfCU6Ya5hx7H69VixU5zUuiLOxfXF9+PfU3GmmKV+51vNcg4lruIag1ntizH1B8YD8VI5bRFj3c6Wtn0ivz2F9X/EroHcuYmVPFjNwyPGyRtHnvd+6/JtZCzB8t3P7D74wtVvqx8XqYqIfzycXUJbmdDhZ1IxZtPWvOj2fv0Gj8Kvay6OVJbr7JZm0dP++9iaSsX9CPG4o8ZQpVQ9zZWaJWxYjbRiCoBD5NeJPAapfnpMbhyx3CFwy+djAVm/PIHVyIVWvtL9z2z4/CYfof0zk1/ldafRWfuXNtR3jv5HtUdFQw//35vNT8EoUPFLodVxZlUm9T1H08+pT3SWfREZYfRneNS1bTJ9KHW47fQuZdLnC6LENOTg6SJDH2/rE8Lz9P9EQX2GzV9lUs+kIBR1z8/iLb7t7GQAdu8QokMN0FXrX/JTdSlzaDoOVT8E/0J+O6IXQLfRQVPsFOyfL/qozUt9OIxWYhYHQAs47PIuOGDOc6gqwhvD6cFeuXk/tLLkXtR6mzw13NcKhdebg+LWW0bzvJ7kd3k/diHndnKv3rALNPlmQcVsdlxwdoNFfS29hLUFUWHt1NOHC/MAkHpc99R2SBkkcI7p6NX69yRbUJ/oQuHIPRP9qZmxgIRz/IrT23gZgSd1UPgMC2QAbnBuIwW/lYO5qy8H9xT85QRn6uAKJlUSZ/7Hj6Zmv5I2IQR9PH4mmNV5QMJA88TB54NRuQBRG7zuCGmhZkEX2fHg+TGrthAsPqjhPf9CBJ9c/ga1L2b5X+XkY4P/k3OhJEBB9vVEIaY4pdBbUuwxnyox+ir6iaUYcriKqJco7NzJpa+iTo6fTGordwY908ziQp8+7N/U37VE8Yc2YMqtOVaC1atLYgQjuUdbo8z/F7uaIUJ9crBd+yjjL2DA/lwNBEPhdGYbab8blmHvuvXEhB/Pv4iuGEdixWnpNgZ/PUdxly4EPCyo5fpmpj8ZAImpZBePedzt8m553H25RBwaACmlN1mGNTKU+sdttunv/jqIQkeoIS6L63m6Ywd7ZjhhbSNHDK+2nabilkzUdr3AAaKkFFen46Ac2KSpVce5Q2U5tzGeCU6x0AxPw1tFqwevhSMvZ6ApZMdkoCg8LsSzz3G2Glx9FEBNG24HqMvhF4dDeBJFG0qYCG8w1/u9//G0KSJGeb938i/lPs+0/8rwmzhzJZbPD/nWbfbeRHP0Lx5G78Ziga6CZtJRcSbgSgxXc3BZFPEXvtBAon3QqC4DKPFuyYDCYOTj+I0CczY286Xr2KR49D1cvJ+CWcHvYmm5flYIlWJqWCpGaY5kqKI1+gxHwSnVmHT3svR8/toqrN1SgbdeX02ruZt2MeI7bZqC+q50ToVIojX1D2I4uI/Y3/4aULKMtc0f+7BpWsQ5Q11EbVsvWDXXTEzHFq+E/Ju8DCLJnArgzUdg0vPfuSU6/+0kgoCaavvIGOQDifdDsnUmZi1igNbH1EPS8/8zJN/8hk7R0vYfJooMvRTHzTg0ziSeLL48k8FYNkupxNeP2g2xhXvJfBF32JfSCCwLZA5zKLpoHcHBGbxobD00H1sWoyf89ETvTgBUHFa8BhYCfuXnO7+n9/DfiHANY0NfURjU40/YyLFUwoPIYsqpARMPn4XsYikVQS2xfswuf+mygcfxOjai5gUPkBUBb+BtFlAndU6eiTu5RB7iUhoiIvI48NV+6l4cvdlxUPBFlNe2A75ncEdl0V7MZmBEgoT8CW3c5fWdkDk+4RZ/wp+/Ek+p4WRAkm6m8non2Fc73izKEkPnUNbUEuhM/7ncrf6n+8Te4SHwrHXY9Rbf4fSRw2GUWeefxO1D82EjU2ioJRBdg1dravj+addyVq5BrOTTpH9IRo5n4xlwPDD9CtLUKQRS7G3sHd1T6cevdLogr3MupwmdM3QWfWkVKUgqOxB9Fuo9PTXU5u5LmRXPO9F12FjeikwQyp+dq5rDT8NUAZ7KpkOxqbN3EtdzNGfwNGqzL5Hv7ck4w7YGTWhkWMvm803yz+zrn9DzkvE17disf5IwzdMtTJssz+5V1uMm5E0lswP2Jmxkv9LATJ9Q53Gc5h1drxjvDGq6OG5vixeIx1TSwSGx9XDOEveY4Hu75lf0Yyu0b4s3tEIO2OZkrGrqIxaWL/GkqHnTU6i4OzG2jem8O7Ue9Se6LWrTMfc2oMYZVttGZV0tfWx3PPPseOvhcBhVl6auwpfr8uRylE9sdAMVArapFUEpIoMeedOTxQ8QB2tbLvuIo4rvnlGuwFdn7tnsePBSPZuBHsQj9jQwbJ4UCWZWoD1tHmcwndAqiJqUHMSCB+35cMPvixU/bl917Ynh9F+nk9lrYerEYrDqsD+yVFRpvWhl0j4TMsjoWfLyQwJdC92HdsIrF59fw4/0d+mfkLV/+qJBEkWWJo6FAmhEwgVKu0a1aHFUZBtawU//+dJNcJQWCQfqxTyvXSkLBh3nuUFT9PdfMxAUg9uRHDga1UDl9GV/hg1A5vzDL82unDucA2/nzwE8qHjsZ30hBsWgNji3dReHYbr574ktr93Qw6+gXhFzRuRQkASezjwogLeN7njWevhMam4a9RMrKZ3jiXLNilURj5FJIE8c33M6T6I/yMLjlOSSVh0VvI8zxIUeIeKiPWEdd8L9/3QEhhOI2O56kcuojuWJ3CiHAYLtu/p8lBbFk0Hn0eDPYrYP2V/jw8/kbE+o3oIyXaIgbRMcWLTv9OQCTEoPSzIiKz9syiPTSE8pFXgeDABpxomM74oFyG9oM3L5WQGoj2qKHIY8ez+uUZ/QVOgfKQ9zidpCQqfrxhO4b7lxFeH47KrnKTvTL0GhiVNYqYOgUtXBewnn+2pxMa+Q0r/FoYd3Eoa95fTNO/vsWj6AIBvVPcvG4K0gvYtcSLyGGDiW692e28VKKKeTvnEfAPFzNUusQfQ5BFYqpjsOw08chDEiUlLskpQVYxofAYD3ufwtMay5zsFrYvyMdHhFGe3YQ2HMC+cSsqmxmt1cPtuIIsUp86A01KKHN3z8XT5IkoiMiyUijcGQEF0TpsiRO5OENBQOfG3suBIYmcUitMUr9Ob9pf/wKvczXO/Y4p3olN1c4nW9+n8otKaq6t4a/h1+mHqsCCNjoUs3ewkx3Xf9GoLUZatp4isKmRoRUfMrTqc6dCAkAIm0h59GkmVLY6E9ZPBChsMr/dU4k8a3aTFAJ4KwjyBvWRm3mO2lGjWfTzSqwePthV3XiLcLA3CpsYjMrLlZi4VHpNJenx6/DjoVceYveUl8n9ZUAat5/ZJ6vRWrT4VOsUhQJBSWjc0QzBTX18f8M6TCOCST3+LaFlx90S4Rn507njo5F0ffA94w+04tXr5Xz/Bte+Q7T5exoH3YhXwDgi21cy/5yRyNYb+KIbwlUQPekC5clmyhLLaA9QzssOdElgNNg4NOUQrT5nePXXqf2eGe7Jl8APwzB3KucblBp0SfFOSVZYpQHZNrvzWgdCklxAlJogpX+N08DuJgNDv74Bw6m/98TwliPR2YLoTInlmxse5HS6uyxRlD4NrSMA43alTz82UVGc6PG4SI+tA5vGRre3CVnt3rZFtF9LReh73GBYTnOiHZ+WUgBMunLavQ8RXRPO3WvvxrBLjW/1WtrUz7htr7clUxOnRdBq8Oy83MNHUklYvbRotMEY+lIxacspiHySZL/zfJ3ciWN6ClJyJg1+G6gPXO98D1t8d1ATU8PxyYVEn23gsbceu9TmCGQRU0ENgfW59H35E1MPJLFy/UoGVno4836+jAlj4/X7EBER7AKCLDjR/RJ26iVX8vPPJX9SHPkKkmhFFhQQms1f4OyoYgoTFeCcrv/4axt1xFRG47t4Kl/e9r3b9fqaM5h99DPGHkukYNI17J+hJJM9BHg7opUaBN68cy35o/wpCX+F+kClOPlrL3iX2ThzMZ47P78ToiMIuNclQeshwtLQPMLH7SBjfQYFLUEgy852M6r1RjxMHow8nsb5Wz4kMesX7Jcw++yqHipCPsAWq0PfJ5FYmoiHNcYNFADg1awnqSiA8Nap2FVd7B4RwLH0sZg8qvl+9S/seExN4RALXubUy551bJnAjf9aTnx5PKKkQ2uO558tWuwecczMy8ff/hjGqd4YDUaCu+Yy/6yJmNZbuT3oOw7deYiVW1cy9v6x2FG+LVHSoZI88TKnkT4ng8gVkZcdM39QPqdv0mN9Zjnbr9jptsykrUK0W+n69k/8ao9iUTdh1JXR4utekA6q0BFRchi1rZ+RK8NiA0xHx9QD4+hL9OH0mNMuj0PBgU70xNMvGlnlSey3RoJbghWAgzdop0YxT6X5772ERTXyDIHiMSbyxi9lfOEhhlZ96VznUkS/k5HSH1r1MoyJQxAaGvDobnIDCBWF/ZN9M7bQOmEis3emMrzoUbflz7bB/pNpDDkd4gTIArzbqfxtdhiw6jWIZ6uQihSmtyhrUUsGRFmH0ctIa1o7hi73ROVAVL+yDtX+fQRXn6Pd6yi7h4VwPHUKNrXSrllsdvyMYxlZ9iue5kS2Z2rYlqnmQsL1FEY9QbGpmPLvypm9ZzYAOc0K6CH2SCyfjfiMX6/8lQXHFxDcNcftuKH1IrXbL2KYYqDP032MZtfY+eaWb0i5cR62ATpbf+pMFJS/DsmB0T8KY+RNLDIfY3zbZ/3PQYXapqbj6b2knfgW6ZJin2+XLw+99xCO/S7m5fl4FwsuuDUNQ3sNftddQfash5H+Uuyjv4++dPwywEwWEFE1K893/epfyR6WzcjynxlU+7bzvK/67SoiykuYkf4uGbWfoZa8nftxOKC3qRehp/sSD0PlWi/G3sX5hFXUm1wMb0lS/ktp+Acjy38m0DTBjckhC66x+t65xdQtiqFs3gSnl+CFhBvJSlpKt6YEk7YSlamHEXl34dOTSJvPQRIbH+eKcxaub1rD1N2uidrZUS6pYrWgxbNHJDb/GI4u42WS5IKspiWkhbYpZmweypzz937b7Tbvbfi2NnGh1AWCGSj2XRqmi+XKeMJh77+uS3y1BQt4+YAqEH/jeFLrXuLeFkivUu6cvttBxr638K/6jXr/XzF6FAHKvHrYuUgEyU7uxBJMniYSG55w7vd8wkkOzail98992CwS3SHufs+N1kYy3shg6K1D8Y4PxqFxtwyRtVoGXzOYYQ/P4M/l2zB6GREQ8WtTM+3ANOQGGctQKxfjP3Hbbt2hdXy+8HOyf1AAEmpR7XzfB2LEayN42v40J8cr0rilSaVcuPICPy38id2PKf5+ar2a6PHR+MX5ASDaLYza/jLnXzjvti/hEtZa+MhwgtKUoueUf0zh+t3XI6jV/dtbMbUp32jZnjI0Z04gOFzfRkfEYAyZ6STNTWLp98spbdvAZEHzbykjTRDB+5AXxjojE6In8MoMl12NKGvRm/XEVcSw4doN3PTtTYSqITcW7oxQ1L9q02eT8NGDVB2qonhzMY9NfIyGRxp4bZaS//j92t95Wfcypz485WabANBgKeO7iV8Ql7MNn9ZyHJcA0G/+8RNabv0Ec1k9GksvMg6ODBpJp5fSS8iCA4fVDrJ0mdepaO8gtOw4qTeN593n3r3suvMG5/HNHRdR+fk4QZt22b2tic2/SOWD21HZVfTpKp1sOlkSGHtqLMvenI1XezWN+ucoCX0OmzhQ7FWUMq75No30k7/hbU5ncM27pNW/DIBRV8rZNvfcxEAcmXKE4hXe6G+4msoRywjpnuu2/NE376H1jW+IqO7Fu8fbCX6rDfyByAq4Z94f/L5GeccEWY2XPY5OCarDruRQH3x+++eoqzqYdnAaDrGPJn9FtanBfwO3H7md3nd6kZ6V/jbPsP7a9fRs2K3kBAXokhpo8u9XVxJsRM1R5q4m3/DLALiirKXi050MP+5qPz2ssRgs/QA7HHRPmMfembvdtosKG0nRpDW0xI91y28NRE4s7JfDCa3UoxphoCyxzE2CWxBEJh11ga9/H3Irrx5RLEXU/dY2A7knh+xg2oFp3P3J3dh6XMcayJEJDhvWli5EuyunKstQPmI5TQnjkGQRj8JsDF31RBfsIbDuIptW/srpjy73jv1P/H8T/yn2/Sf+10WH13GqghQUZflYE70XygiuPENB1GNu67V7HXEbcA4kTS7tDHtGe7Ph6lN0+brQokZdGb3evVTGt+K/5SDLtjxFfPMDboiZpNIkpm6voPrakwiWs0S0r3AmwtSimuCWYHxa4PvR37Ns4zLXOSAiIPLUq0+RfuYsstgveyENnKMKWZQJKPYnsnA/hl4FPevol9opHb2SvpW3MMhjkdPjZiASGx5nysFkLLUt7FlkoTR2K20++7FpFPSGLCqMREnETU5EFmwIiCSXJDPhaBK6Qz1E1UQR2rGU0I6lAIwKG83wLIGApmZaFrdh8nQl92zqTiVhOX8n+S/m05zbTPqRdPxC/KmQHWzhv/ea2wJUydA+voOvbv2eioQK7vT9E70tEgEBj+5GhOYmcmfM4/DUw3+zFwnBz5fewFgktRajo9O5ZElnGE+9/hThm1rRV7kPggfQLIIsY+82XZakd/oPSBIyl3eun979Kdq3RxLyyYsknP2NdsNx9gwNpyL0fQBGngmi6P1dZBz6BI25B4cDYlpvd25fF+uHIT0G4RK/s9+L1hBeDr86LDg0MpXxVdgNpv+RxGGvVk+XRY1sV/ZrtCpJG5XkSWmJCqOXkYMLDpI8P5mUVSlUxlf23wfFa8LQa8Aju9+4XCU4Bzt+nX5ct/46pNP15Cy8hp3z3ZMjTaFNFGSYkUUR/3aIabqWWyNdTMxW1RME1JzA+o9X8Ky6l71DYiiw7OXqPxoZ37CCvJQiWsNE8lvyuGvbXbx29BVSRB1yMlRPr+WN51/E6udD+q50p9HwF7WeVFdE8Pzb/2D8KQWtWbytmIXtC7lJdr0rdtnK8FsUJFlAbTaVZ9PwNCcyJS+b+Ob7LpPx7LW3Y9KX4t/uz/jj4zFXu/u/pda9glefwsCSBQeGhFDGPjAWnygfRdKyBzgIjTtLOXXiftZ5fsEhDlL2jUv+TYWGLr8umkJ7qD5Szeu+r5P9QzbDQocxMnwkkT6RpHSkIOfLeAZ64hfnh9SPsA/oCCO9YBDJQjLd/ePqM2dwspBm75nN1W+FUvj9GUYfrCW42V3CFsAh2zFGJNMZmoqvSSl03yYOJ3zyOT647wM0/t685vUaOx7Ygd3hmtQGtAUQ0CLgERPCqNtH4R3u7VYM/PqWr7kwMYGUhcrgNbndNTmdHjede47dwzUbrsHSY6H1aCu+Dl/kOcpgZCH/dcHvFDBXgBW3307W4pGcH+k+WVy0eRETPihCnZHG3llncKgchLdf41ze5ZUNomvgLcoaBGBTlycRo62kn3Rg8Z9A2JrlWLwC8TONxq9nGk+Pv42k1M1UDB7Nwcfa+OZmdw6iQ+xjtvolxtw0hT9uEKiOdUfkTSw4wWzbR1Td78Puue4DeHBPGABukwO1TU1YQxjaDhmV3a4wAGQNngLkJjWzYsrbtMaMxNsRQVL1LcS13Ot8L4PVCVx79DFSc2Q+ePBDKuMrCdB2o1FZ6egL5XO/G3hh1h1sWXKUdP18xhXtZ7rjVRz97V6tTU1aYRpB9QpYZCDZdLZ5Ko8d/Rb54luMUk9ly6ItdPt2u13DrmFBZLOOhrgOWoJaERCoCfrG2RcJDhnTn8e48/M7MRgNzokaKEmwRVsXEbBlIwH5d5AbtRpQfNgA1tpNNIW1Yy2uwlz9I3UBPxLQ45KANBlM+HXYEZ+vZ2zO60y/6Prm1KKarMwsuq5XJGiPv3UcyTIg06mgnbcs2sKfq0IJqTzN4a+K3WQ8/Y1jCdbE0emZRa++EEmWGKeHE7EOzFc/Re5qkWF732HcAff7URD9OPuHJNA92YMvb/2Sbp9uxBdF9iWOIjvuVjJ0YCmIwu7hw8VpRc7tzNo6OqngFdVhbvr+HoQQPzYmKwUTjT2AkO65eFijKUsswzrHin7O5d6wR6YcoeOLEOSAQJBlLsbd6nr3kPHoaaavrIGqjCvwddxAeOdy5xhpePkPeBsjaXnzW0LLTzjHUIEiWCSQj40iosBO2qntjDsxzrnfVC3kW0CSoTGoiqfaX6Eg5jXs6i7StbDM/wKjzNfyytv3osocQV+cyo2FGdaxFEmUUDlUiJKDgGZlfOZi9qnQ2DQM2hyBf0M+JREvAYp33lchsMKyEIMUgr63GdHahEM0kl7zBokNTzKo4Ud6AlPQTRvHqSleGA1Gt/cPQJgtNb0AAQAASURBVGXtQ3PmOF7t1Rj1JdQF/QDA28GKXKbR28b669Y7GXQ+IrzpFcPFlDbMVxwgsFXHtb9ciyCrqA7+krUhICfDPb5g70+eJsxK4EzdGRo4i0PocxYcB1C1f5Wkdz6zAS+k/oTLE/7wSqRA5dBFeCUN45rf1pCe7y4P6VFSTua2fyKWukv6AXjYorC1dBJccRrryQsAzvGwJFrJ7jrMuIRXyR87HfPRLK7d4mIzmHRlOJPAi8zkTb3H7dzbAtvYP30/Dl+ZUaeDia6Jxsc4wrl9WuOnNGU8ham8iZST7oUvgNnn64m8aGXeOzqW7/mD2NY7sWnaeC+2lLHxW/HPsILZzB2f30xKUcpl24NEW4yB06NP42l2gTTaNBdpvltP9iRXu1GYVuhUwbjQOIcuxzWcHXOWmpMNrHtsHaXJpc6CrICKLr8uXnr+dU4tepwLIy4gizKCLNLtb+fTez6ldbmOfXNO9HunQLFNYXe+kh/Mzd/dRF9JDS0h7khnX8sQ4mrSiCopw65TOyWmzLLCun+9VQGG+bX0Me6AhagaF4NGK8BL07dSkzYN9fhxeI52qVy8Hwx7etVc2b2ZT97/hFHbXyagLsfpYylKHniaPJmybxSW+nZlvPqXAn5ezAM0p3TTGCVg1pvROHwxWPqZh3YV9354L/M+GETi+Q3o+jo50y9/NxAyEmMtzzAl/wKxLXe5LYuQhjL0nCdePZ749k1Caw9kWm4xJ/d3YJxRQXeNHt+WUnqmeDtBKSpZSZZKEhx84SA77t2BX6wfDlzMvoHI/zmHzvOdl70dJoOJ7nCZEK805p/vYcZFlxqFgIggS5hPZROTv5sDg6OoCfrKuVzfp2fejnlIPr5kT7uJPk/leH4ifBUK9+p1TDo2BpuvmuLUYlT9bGVJtLKv/WsEAUw+YbRkqjDrzU5ZVM3oYOIn3stEEaapVW5ewlNFPZMEFZ4jl+KYLJEzrJmI8gJSCnzwN4655MpciXMf0wiCul0enu2RQ2ibuhz1pg2EF+1wG3/IgoPysLeRrB1oTR0IksMtYVhvh9pVf9Dj08fdn7pYoQNhUc+nYfD9yr7q/lq8V/ZTMauaC2O9LtsWwH/+WKTQMFqjh1MZ/AlWTQvt3kecEsz7k7YzeeOXRDctRGcPuWx7m9lG7E2xHJx20O33hsQGpr88nVlvzOLq568mqeFZt+V+7SqqN2Shblfj70jj70KNjryncplwbIKT2ScKKjyNnmgOa9D1KmOb3l7o/0wVH7t+KUBZENxyD2a9mePjj0NEEJrjB/FpLsWsUfx2ExueJKasg/TjX2PvNmHXe13G7BtxJp7MrS8SUK7cG5u63ck0K3HspfLiUyRc2EhdVB2d/p0IsohN7Maqascu2RXZ9q4OJLtETcC3lEQoIERJ1cfP8pW8O/VdbP96F69OY/+19F9zfz81IG0McOqUct0DIcuufqrTM4t2L2Uu5t87gTjTNmIjV+PAHXxr1tbi1ePF/e9dw6Ad75Bx9HvCmpQ2e6CdMtd34NNcSt+Nd/DLGhsVCRXO7QVExJAgisatRpWWdJntxgDYO+4dbwYf/BiA/Vk1tHofoCG0iFefeZXaqV34lj5FnXoSRr27DcCk49fS9uYfRBfsQXTYyI96jGa/bc7lCzYakD09aUiZBkBU62qub3mDojh45M+rmPaLhNHLSGH0m5xLdIF+9WY983fOR3DYOLrgJD0+PW5tV0tIC8Vp7Yj+vpSr9nFgcBq1ga6+cvxX40j8IJH26HY+uO5Dflw4hBafS+YZHh4s/3k5MYuGurzWZBG/DjXTDk2DKsgvvDy/obFp0B/Xc/rD01z49gKyLDuLAgABHi7A3D2Z95AamEpdVB15UwpR6dQ4+sfVA2w2uT/PIYuKjKdvmpLb6qjooGJ/BZYelxqP3WzHalTej8DkQBJnJyKISpuWePZXPk19D4D83/LxPLob0fEXFtzAZcrQWf8Tt0pm/p240w4BWXpyX87l7Odn6St1bedhjSax+zjVt93PpKcmUZxWghawyWCR1JxKns/xwVehDvDljrN3sOT4Ek7WnmRv+V6e2/8cbaY2YiYpzLqd9+90EgwGot5cQvBQBdbdEZbulqOT9MPxGJ5K8tqHqRy+lD5tNd2eShE2s/RPpm9t4OzSV8g4+PFlcr/lQc8QXbCH+kMlf8vYsmvs2LR2mv75Ocv23fS39yW+IB9Vic2pblUU+TSFEc9iktupiq0ie3oBQbXZLPxdTaP3h86xg4DolJT36qghsOaC236LIp/l0Qvz/8vnIckSsgy9umJ6dUWMLdrjZOAVpZbitXgaf9wUScGggkvGZiIdEjQ5YOXnc7jvg/sQZTVLGy7QuCaH4MTrONSn2BIYx0dSHVPtpmIwMMexpdvom9J3CdjBFcVHiuk7V4BvS/Nl3nqyYGfCe1eTtfB5tk+5hf1D3cHBDkctxrIGzIYgxhceZmLBKdSSF4Iscs0v1zD853YEQWmHByKgZwrDvZRip6GlEp83fAhruJwCsO+XmUzdGIHurtD+/Jzr3CINsZy/5jzlg4ez9apeWkJanN+zSnRn9tklO4IsOPNsznsjgK63jSH7P6Di/vfwa3QB3iQJOiIy6A2M5Y8/oNnTxSJ1aPRc8dVyRt42kv/E/5n4T7HvP/G/Iv6KahtIGkqyg64jF/Fuq8LfOMFtHVlQTKIlwcbx1KkURj3p/N2vM4zV364mriyG6rgmp2fCpSHIMuqOLiJaxjOo9i3XQAloDmmmI0iHI13E5GlCkDXO5RGGGDZddZFj14eQflc6eYPzLjkpEVFQURVbhdpqRderzApO+T7K+fgbMOpL0Fq0JO9KILzsGIKj/zoFMxZ1EyZtFVZHL6ubP2DZoWYepf6S5IlI1ZAFBK6cc1kCGRTvgaiaKDwO1zPt4GSnzJ1JV45JaOXE+BP8eMMxfH7vYfyJ8f1JZ+WaVKKIX30BapuZ2muaLkfN9Tclkiwx7IZhfP+P72nsbmCSoPr3EFWCyJRNM7jpm+sACFdlIMoaCiKfwL/kFdRbv4G/IJYGYtmGJfTtOaZ4psgyqkv8n75NNDFi2jkCsnsZ8a5ynwxSGMFd8xjts5ipnTcxLG8B4c9dQ80cd1aEIKuZtWcWXosEhmT1IkgCodoEdP2+MAgKAsaSmI7RP4rjaZOwaBsx6UsxmFM4OWc86U8sIm+4nlOp1/ND98306vPxNCsmxXaMXLj6VZZtdEn7pVR/wPDiXCaf2E5ywwwkwfY/8vH4TgODMr1Zd//HdK0IZNfDuxj38Dg0Vg3nE1fS2j8JGBj0DXTuyuWoEGUNerMrYVw6OJBO/068BPAP6mLLso3Iw4IwaS5nkFTGV3Jqci9dhQ0MPvwZhs46rgh8gEn5CponpriShIt7EIcPoSaqGrOuho+6Z7NjlCcne39h4/KNFA1WU/leKcnFyahENWGafqSqAKIo0zh6GDuf34lVa+XGQXeyyfs7WirCsfeI5D2fx55nD/HnXbvJfuEiMfJktDYlIWC3W2jJa6YzJBlJJdKmeRWTvgwPqyshdqmMp9T/3ge3BDN391zMhU1E5e/Gv175lqParye17iUmnbqKWTvTMCRHMO+9eQQkBbDn5w2o3oEpR+FVuZ2fpG5eMNWh1p7kt8af4bhyDLWgQXSIqOyg9dYSOTYSQ7CBvTfu5ewdZzl28zHuO3wf7Y+009vYS2dlpxPF2xKTTtu9zzN/1XwSz/xMaNlxhfXRf96NYY1UpZkwt/YSXmNxPtPkesXXYvGmxVifWUv7sOnsm32M6qAvGaRZwBeJF/gtqQ+P0HYEvZZhNw4jalyU08geYM7uOSz8I8ANSJHqPYqrzUt4RD2CuL7nQT+G1sJW4qbHYf/e9Y69f+p957/bitsovbmUITlDBl5AOoBJwCxwS3JNBiYC4XIKs2Ysck7+DWZXgrc5pJm2OA/UqYmcHZ2HpJLc/EJ2Layhe+Jcoi+up0W4jdzYNQzRQsfQIoSixwiqygLTMR4sT+Z46mSsqg7M3if4sW4M//Jcx9a5b2PyVmQnL01YR7SvYErOUhqerUFtvXxS6WMaRuC28yR8opzzX9kQMg4kCayqdizqJgyW/4e9v4yO49rC/OFfVXMLW8wMlllmlu3YiSmxHXAShx1mZma+YXLYcRI7cRIzM7MtyZIsZmZ1q7m73g8ldavjzMy9H2b+76y5e60sR30Kzzl1YO9nP086oT3TAQjpCOGur+5i9Jd+3PfRXFIqUlA5Q0jDQJrGhahuwqyuJP77Yyz8azKD6l/HYJKzT6f6LyNLsRCRAC72f4jUxqfoNGWwufQuXtm/kRpNEhKQVKbE9dZaEmvSmT1sJCOueYLFq63kWdV8dedX1KYnE9hSxqiK35hctYJnRj1Hpr6D0LolhG3WMu3kJ0wuOupxlANM2zeStJ8bSLjhE0K0B4nqWoRRn+cpf+KdZbhOFFMfU49Va/XJXmoPbWfFDSswh/sRVxvucZZ90w3z6uF4ahm/XbOH8qcNrFq8nE7/QwzMnlI4FejMLuydJgRJws+egqZvHKivVVI0uAjjPCOFawrZ8fgOMjVjGVzzIfFtt5BduZKzN7UT23kDced3YT5e4OMgASh2budw1mTOxT+A2y0xpW+4VMW20JHqpj12uA+Fcb+NOh6LtKWcuvg6maIKiS79aSyaKgA2f72IgFNbPOhKgHEaWBrUTIImFgQt5AyiKrG8r9/0f1ciDrUD88NmAl+7kKIMwGl3EfHVaySf+RO7stXze1nM6xwf8gKxDyymJ0zelBbEP0xboEwVJCDi11GDvaQa//ZqTzstbARtOXx9+9ecvlRBemmyj07SGA2cbTBwz0cPEFlYxPaWFbQEydoV60zwZJ8ztMKqo3SOk47pMr1qRNd8sk/sJcSYQ09QD28/+Tb50++hPfEiRlR+T1rjM/JzSUpPlqxfV72njRb4wfUaFaN2TCO0Qs2vy1r4+obLKYl5hdTmx8mqfxNzUDRl465Fe9FkqtNFbFqbb2ZpZw2ZR35Au38HonETJq0cfN0ZC+dssPOtm5i+LcDT3+ec7uHRuvU8Fl1DlMKNAsgf2cQfl/8hawvqT9De16SL/aHi2iaW/LGEjrIObn/5dn7UjsGsqfQEMvvn5IGU9J1+x9k3ZCirFZd6nrPfcfxJF9iUamLGdiGEhzK4IJyYBq9GE8Dp8G+oTnGhq6vm8j8uR2VXkVH/suc+h7Z+QGLBFoSgAE5Nb/XRqjGoIhipuI6ozuk4i8oIHBDHNmnPI0gCoW2hhNaIuFQan2fvCO1gf85+eqe4+PWGIs5kn8GpkC8Q13ajR1vImRPC3un7OZIxw0PZN7Z0E3qrgeB9mxHsdiRB8NR5vMrB6Y5sVrVHYHP3oHIoEd0igRYvpX5GcQaX/zaB7lAlm+dvRuXyOidNimreqllMwaB96F9/ms8f3seRSUc869hGYzp+PRsJTdeycq0fUyq3Men8QfT2JE+bAETXRxLSUDhg+PmbrpXgmyWkt0djCu5i89zNKOLD0Zk1PuWd2jO0JI0j/6IHcCgUnuwpfW82+yzwgBImroqldMcLbCv7EPNvrbAXsh1gS4MrAlNpTstBERWJfUC2Q4h5BCpHFOGNUQTvCcYUHItdH4xT0Sc5ILjoDurm52XrGfrdw5yfchujy38jqNdXF9ycpaZy4q0kmH4lpekxwnpmEdt+HZIg4VK46B0cyJHpwZTHbcTpQ5sLkw+OxvDpq0RUyFkgSqfs7J2ovJtHtCcI7nCjnTiSOMsK/G2ZKAWBJP8qFPYmdD3NRFUcQWh39NW/t57dblDpZf1Vuc7leVbh1mJR1VMc8zxN0U3ohvhmUkAffabFiW3bXiKqS1C4vZnxjSG/s3fYFJTjslDbFajtag+9MMh6khOOTUDdbuGv6UlsHis7bxf1+XAPqkx8cff3tE2VM1QGOvD3df3Elpa7Ca5bh5gSh9Y8j5TGJ5h2Lp/R5WuYOvJfbHxzCyrlbB7VJ7EwAO4OFWlOe5bpF9URPvQSArsDSS8OxtBcjNbkq/8j+OirCWTVvtP3vkp6dHkYxVpOX9zA1plvUhO+3OdcQ4cBi17FiVnz6Aiz+NTJvcGwOs1M8ehq8oZ75/OhfcmD8dIaLP4RSC9fw6oJ77FxjOD5ry1gJ0qHEl27ht5/iPUdyZjBifGncN56J+bgmAuc1n83vVZxwW+mUybyn8j3aJT3W1F0EdOencbkxyeTMivFB1BlME3EZbiLlJum0PZmGzfUHmPBScl7jAQ5e3No3l1A07pGUstTPQCMA/sVhLeGY3jHwJD9X5B1YDkdbV4/gYACh8qB31tTyZv5IKKkQnRrGFeyFYvewvZLtiMlRaE9so+g1jJcfRTbNWFfs3PiQ+SPCUdyudGY2lC4/Ak0jyCy8zImnT/IyEKZxSS5JJpxJZsJMU2V9YP77PCYXRyaITEt8hri2m5G4dZzLGM227NDyYm7hMNXjcSuD8f4+Ks0Bn5Ibdj3xLQvJaxnFiWKP6kbLwceB581M6T6kz5K1wEZ6H3jS4ffYQ5nTiMv/h4kJNw4PQApgNykW6mIkkGf/X4cyWoj+uQJH01SAJfCRVNUOx1xw6gbNAtzX+a/S5TZA4KuSuPYZfcjNjcybb0Cf6O3I5mkFtYp7qck7hNM733FhKMTfK4tSkqyCrMIKFSj6MvCbQxZw9HMmdSGy1mxog3Sz2uJbAzwOTe94XkmHc0BQeDX6w5xaPB8qiI+8TmmJ8iBKdDKufgHqQn7Bp0jHp15JCdN0XRprHSHwebLrZwY5wu46fXr5btbvqM+JR17X1Aqsusy5p90Maxaphd3qh2IIcEMOqxh6U/TcCi6POcrXUquWHMFFd9X4HI7+3w/8jzpEHu4o0Ek/s54Tr+/nfkbL0Zr0SIgUh9v59N7P4Vg6FpwnCkHpvg8V1dwF9V/VROUEMS6W9aR9dkA7VpBpP2JdraM3sI7Ae/wcurLLL9UHkcaGiR2DLqfuR/LQZzm/GZe07zGgTcPyG0vKigdey2WYbI2Xf7P+ay4aAWdFV7JmuWjl/P1WFnT0+10Iw3YbHaHpzH4WlmffMrTU2hbuAyX0vu9p578jbpXvufc6nNsvGM9Tkf3f8SM1KPuQRenY+OdGzmz1TvG6ewJaJ1RaF0RXPTGRcRfeSlaEVQCuFx6WoO20qvcj62xA0EQ2FO/h0tWXsINf93Ah8c+5LYNtzH+gfE8WPUg83+/ieDEYJ97r21/k0Wrr+Xkghex64NxSN7gZ1vieMLuvRpR17++8tZHfchKXCp53d8VmYnCrWVagZeuuiJuDUWTbyVkeCzxZRfqcofaowltC8XvovFIgy7MuAdwKpUEj0nxJBhURP2LspjXsUo9VKZUcuqSfNrihrI3Z6+sp+ph4BCpi6/jrxsKaIuOozzuT6rDv/ReeCDNpEtPgNm7fhueO5yk7b04Dh2j13Yne4fJ7BNDaz8GYN2ijejn5HjmCaEvg12QFGyJgWp3PEqzS9ZYlZRohSD0zVvRHbqCX7O2cPHJPMxTUnxo6eXzlWisGnQf6pB2/LPQXPmn5WgGpzJu9ylZs3WASYLTE2x2ixfOYQqXAskt0RLehE3VhNoRRknMizSGrJHZ0VQQcHQH0/ZN95zTEbCfX+sfgbYnCa05jt8BP/xN/sS13eg5Jr4Svs0+xomLmnG5vDr0/bZhXh4nPjiBSz+YmBoR0SWi6qPh9GT29fl5hoQPYc/MPXx6/6coA7x7c5kauge1zYSgUnpkbcAbYAew2yGuaCc9ocmcXPAiXVGDGHTVMOIn/rMu/H/tf7/9N9j3X/u/wkSUjCr7jUSm+vw+bJM/+uGpVI5cRIB5mE9Zf7BPEhx0BMjIsonn95NZ/xqCWyCiJQKp2+qlVxlgqWWp3P1JDtYps6gefik1Yd+yz/ahp7w1opV98+IwvqqkJ6injypFnhjuHfYc0wrPktlzNyOfGemTgSIgIiLy67W/EtbUSmzxXgBqdOupD12JQ9FBaHso4aVh5A+r9OhIHBmUw46RUdToZmEuWIPpo++IKTtCsDKayO4FaOzRqNyB9ESk4XZLDD8pyTolAyykI4Tbvr2NkE215OzP8dCvVUS9zwZuxxhopDWim9bb/Tky8QgiSg/9xu6abey8XODkrKk+ASJvZYtkFGcQtTEKlV5Fj38PugKJ26ULg47/ZHdKbmrbComvi8PQYeBs+2FOpl5OefQ7HJq8h8qJIvEFZ8k8712MpDQ9TlrP7QwpHIJ1407GbHqV3JjxhKq8E0pQWDeX3Lqe6sWhlMyV6dGGlH3HjKpVfBVp5/K2m5lw0A/qzaxc/KvPMwmSkuZImW4m85yN0J5ZXBR8G7PzGgg0jyS6IRpncTe985fQnDKRuPabPOfGtl+PRjGHqHmjODa5nPrITRyz/8CJ9MuIa7+RYVVfMXWLjBysTvSm839wyRi2zBnK+N15RO46xrxfjjKvad6/reOxzwVrFjTTlQpOlxttsBZzoNmDyLKqGtH36lm8YjFHPzrKikErmL19NoKklEXSJRWdhk7qHrbjUOlQ9BHiPxAMRVlWXONycUfrCS0pIaky6YJnkAQ3AYNiqRo8kvrIU5Sb8gg2j0VwKzk86TCF4y5Cfc3lnB3lDYB/HA7FiTDPeglBplSalteQWp5KUVshh2wOAsvBv0BNaGUigstJZ1gnkijhtvrT4oKlah3DH5oOwMGvCsiNnUvnhEtkas7+bAmXnSPvHca/s5aw+kKimuTlv0lbSLf+9AU0nianTBtUG1/LN7d+g2pIJFEVRwhqKfMcE911BWNKbiC1RMTtkPv5B++/z57vfuaQBPucvrS12+1WDrolAncBh2Uaz+l7p3PXJ5PQBGm5YfsNpM3xCjuIokjOkzlMf2U6m+7exMdpHw+g7BFwuWTUZFBLKfqepr6xTn6O/OH5HLy0hTFPzOS3W3We7ILIrksZX7Idf+dixLhokCTaA/Zg0VSRopjC4ZbR2K1qInoCcSsFFv24iJE3jfSh6YwaeQPVWRNo2nKGj9M+pvFMI2MNc1imi+C95DNkNC/G3xRO7NhYIodHolX6jq9Dlgzhm/e/ISghiIjnIiiJLkHcAXslqAMeBc4D9wNXAc8jB/rWA6VCCV3dnSSfbyG6IRq9LQWNXW7LYxOOkXdpCIJadQEKWq4yJ5jNRNaUoDIXY9QV0O2GOgcE1u4kKX8TabmnGL9mMCpzG0XxjzJi7Cyuiz1OjQPATWCNSGRTpJfyCAABV00951fnMqgggvim+ehsMqotvvU2OQPB6URXZmPppjOMrvjNJ1jY77Q/kX4pO0ZG4VB0M6RGdiSY/E3smrmLunF2cocX0BPYQ3nUO8wIkjfGKwtyGb77M1zhUR5hdzxUTgKKqZMpG3st07uvZ2jlC8Tqm5iX/iWphtMkOBuYljeCyXv9kSzyxlIQQOus4snJS1gcYMOhdpCeW0Bk5VEiu+cR172A9JDTRPhVYQmKJuvphWgdBkK7szH0ejVnAowBaDr+GZyR1OzVCNq4YCN2jR1/6yCGV31DesOL2LQ2KlIr2HNlHW88+wY2rfxsxQ7ocARxb/hDpDY/hiXURU9QDzZVC40hazzXHHpuKMNPOkh+cDbGYD1uwe4JBlVX9GX3IzHl6SncevRWUkJGkdLyIJHdl6J0+yM1uvHv6qBo8jI6Fqgpcezpq08FlRGfsN38NgBtQTu4duckjtlgUy/sschgl+oRl3lojQZaankqymNtMuL8b3vIm5vhxIItuOwt3PHrfZ7flwTA69F5GOKdnJv5AK7J3vEhwDqEmrBvsKir5Gs2gmC5kFLV0GFAmWfDnpBGryHOh0oMoCZ6M9qkaJSWKpr9V9EWuMNTdiblOorDZQDOwUmbcSrljK9AEe4IhKxwI5ZAiS8fPOWj8zuhFn6zZ4AYRPoZC8++9izBHQGkNT5DsALeDoOV5kFcu2YYUd8UEvuDjdk7FuBnSyOGaajdBl5OPMiBOw5g9Q9DKWQQ334zkd0LANDbUxiiuIKekSKd0YM9495ELTgdSiwnzbhLy+mvaF/nt2xut9f52D9PHBo0mfyoeeiNLZSMM7Jx1lM0BcvvFSBCnh3MSGgtSp5cv4WlW8pQugOo6xnEitzX2R39KdkfvUt0U2Jf1p8MoHm2HWbsf5OHihYxPGgCR/51hK6qLmyCNwtJ8AT7+sauAZl9bsGKUVdAp1A6ILNPPs7WPpdUvxZiQw7hMuj56t6D7Mvxpdg+kPwVv1zzNZIgMTx/OAHmdAy9E0lpegy34GBd9Dccnwx+d17P6L3hXLTLq/cmCCJ0dmJoKkT3wG2UD/UCLK4d+iozQ8rJLM4k+089Ae3yWqa/PZKbHuaSM53cMfxFOkN7segtngwmp9iLXagn7twaHFYrB6dswaKu8TiT1c5Q3KISY/ZUjPOvpis6i9Gj5f6dWSOxqKqcaet2IzXk8dl9X3I+6zwpTY+jEWTt40skPZFNwSjscn9XuH3nofCWcMKbJFCpcSn616ny9U2OAIzBIpbHmukJi8Vw0E1UyxAPtVu/M2vcsTGkn9rIo+8/CpLsaIptmcO8Hc+QciiGK9bMJqVcDqKHKaB3SCPnh5s4Pv44pk2HeeTDF4htv97zTAnGJdSHb6NHV8Go7T8xb+vNnqwsxyG49UnQnW/mbUcLKzHysdHG1IOQ/yZ8sAm6FfK8Yv/hF8oe8GahfbT7BN+ULmV4SzTBp4JpyJiOKSSBjgDZAVsb/g1OlZPG2BZUobLj8PCgHFSuIOafdHn0aEK/bGHcXysINU3Fz5pBl98xBEmJW+Hmi3u+wHnDbLZNf4hTmXdcEOictn8CgttNQIec+R7ftgwArRCIqFGRP/0+9Jd6+51O2cPn84fgvyOayllPcmqCm4C9PQR3BvsEaz5vu4KbXTfTXNDM3pf2epx/oqTBpmqiNOY1lk/7khtX3kj+3fmcvuO059zE6lRmfujCunUfhvrjnvbvt27/U4g5w9g6vxibxobC7Q0YaqRx/HmdhH3MJPxMfh7w5FkbvNQO4ZH/Is74AXErKpi/cb5PoLDCcooC2xoCOnuJ7k1jbN0vJLfeS17SrZxNuQEHvcTGxTI74UrGTFqF+VGw3R1IesZzaLVRCJKCqQemMnm/vNc5mFNEY7B3/I3uXMJgrbcu+/dxkuikRbwazdFPqBnUSUlmCSqnweedM4szSdlxnOKIOewbMoTY9qXknCsgtfEJFvuBsTOAkvR6ds3a5Tlnad9yyEwMkeV7sBWWcF6z0ee6vdpi1HY1Q3/NIqHyQlBUe+BemuxlnnX4wDVcaM9M5pw2cVfzc5y47iZqIzfQE3z4gmuIESKj7xnNx/d87PP7QB3ts425nqwYkLMmdK5MRL0Gp83pGV/71wwBQjQz9s6gYfdpbii8k7IxSxleJX9b6lMPc+jlQ1z2y2UYQ5OR3K3sGpFMfsK9gDzvSKKEkBCAQxfE5POHmXfaSniPl0ZU8tfQc/3dNKVO9mQwuhS9VCafpXSoGuO6PQw+sJxw4yymFZ5lbPk6QkyTaZ5yG4VTbscycgERPXMJ9snshLr4OroNbsY9EM1FuxexY2QUXf4y8PKKtBtJ6bmZHkMQ+UPz6faXHd5RXQuJ6rwcgLZ5bSgvm0dn7BSSW+8jpnNJ3zv5Zvb1Z192689wKvUKNo9RUWH42vMcrgF0wACD932BavlnxOc1klyZ7FNm0VtYtXQzldmXc2J8CXVRMujodOoStozyY8G+KPLj70V1aC/RtSI5+3JQuuTOZ5OMHHctpybiD5TjR6FUXezzzQmSErfopjfOIdPS4wWlKB1KMoozUPUq+P7OUnbO2sngmg9IbLmn72yRilFXon5pPsXpO+gKOHkBPd+ei9vpCupG2bOZ2rDv2DtkMFNGL2aYroNDQy6idNbFOBSd/N2cKic1iTUEdrSwcOU8QtpjOZm2iINZY7GqGhiWN4wbvh6G5JDnsbawNgaOUwqUDDs3jNo1tQzdlklUY5RnvSMgonQouePrOyj95iijzoxA5VAhSAocaom28DaIAGlcAO2h7YQYpxHW0zduCOCUnEx5egr77txHcXsxKlFu+1CdLOPSXduN0+KkZFOJlzp4AGUrgDZYy7DrhhEx1JuJKwjw668ZiKJI2tw05n0+j8A4L1Atc2EmmZfJvp7lY5bzYconVPe5SVpSJjDj3XkAGJINWMPjfTbvLoUKQaum7mgdud+fQaUI+I+Ykfwj/Rj70Fi6Xuviro7bSG94ngUnJWI6r0ZhN6OpKaGrqou5hodp7klneDWsrZLZu3L2TaP60U8o31GOudg3M35rmcyEVFgfzHM/hVBqKf377XEPYHmyu73zZ1X455hdPXQfzMevoxa36A0EGnonUTr+ekatf5G6wRcjoPABPTlVTnoNcZT9coIlq71MNwBZte/wmnEXV6xKRJUUS9cgXxTGbdm38aBURe5F95H14tUXrKHmBZbwWvYSpoVcS3dEEntn7MWqsw74NuQ+0RZt4sSMeA6OfZXq8K8GXGEAdWvnElSuYM/f6aXpxByz49hziOxTMlOFIKkGBAjduLpMhDWa0ffqPetGEJnjB8e3jUfVa2PrnK0IKGnQ7uTBs5sBGMRWhu/7DcP6cobYlvm8k6F3AkqnEu02Lf7v+l+wVwOICYhBM3oohaOScKh8fQ1uwcGeG75jzMaXSSm7kFHJFGDC/lMWP1/9BKdTl2DWVHhkbv66/C9OX+GHruwco3K9wD6V08C+lq8Zc1SPQ6XkyHdHKE8tx2CaQu4Njbwy/RXqnLA3pZTSoZ0E/KZg8cbXSWi9w/tcblkvPDFvPaOO6xlSMMSb2dev2ddHHfvKjFdIMaQgIPgE2kURTCEJnJ39GImfP4452AsqdNmcjNj+HvHn5DquHzSTliRfsNj/yyaKIsOGDUP8u+7T/6n7/39y1//af+0/NAGByPbLWcIfPr9rTSIukwVJobyAY10SXHJm34B0+ODesejtifQE9fDuE+8iOd088P4lLN7f5IOSMOvN1CS04/TXYVHXYNTle8rCVYmMNr5EoGW4Z8FbG/4dktjnVBBE/Dtq0NaW+qD0+9/kvtRPWX3VaiqHTqU9TkYoebj2JSU9gT2cXZzP2VHnL9CJm3pgKtH769BcdjGdMUNQKCCz4RVGVq6gNXAbBXGPYCmqZthpEb3Ztz6MAUa2XbyNhmsS+Oyez3y4qJVoGSfcQEzbXKRRYXSG+dEQspqWYHnDtqd6BydT7kXfupcxH4wkssUrXKy1xxHVtYgH254maVUSlg4Lt4Tcgtah/48QVXa1/K5hbWGsk26lySBvXguGFtAwWCDhXAGDzg9C5TRw8Zl2Bte9w+T25Zye9wLqpVdwflAdnYFFTAz0UmRc2Qgz66FtuI5n3n+GCcW7CO4dx7z0z0l0fsfwkQcpG70EW30btmKbzzNFdS3mhkduIG3VOIon3MjQmk8ZqrsEh6Kb5OYHuPKvu1B8WuU53i14N7ACIkq7GWuX5YKsVLuyjeDecbjUBkJmZfsg/UyuPiqgiSLmrGwa4wPpSOxAl5PMPFHNZuA1YCyQ0ffva8Bm4BIFzLlMS1QwHN09hqiCMHJeyGHN/WtwquS+KUpqFC4FSeVJtBe3o4vUyUisPkejKKlwK9woW9x0xA1HdDtR2VUY+tZuDglwuhlyvJSRZ0f6aGUlVyQzY2sgbkROjq9nf/ZVrKh6ra8+FNTF19Eel9D3t3eRd38wZKhhxu4ZZJ6PJOn3IRycchAAF2B0g3+HgVt+uIWQ/PMoO5QIbgG3U8lXEbB3wiGqLmllXuE8ckcvoyc8lY3WSo4LH2NV1wPgVDiJGR+HTW8gb9q1MmUXcChrEgeyZI20/jm4R3uOjd1v8loo5GVZaE2og1AdeTMfpC5Ldry1+++jLWA3TeNyOD3nabpzq/jyki956skn2exw/i+FwBU7QDTJFBeFQxpR+Wn+8fi0y9IYfcdoMhdlMumxSR49QNHejLlhFw2NDZye/zyV2Zf3ZfbJ31BC6x1k9dzfl9ls6/vtdoLNYwnvmU1P4s343XUToef2Mu6w3CY7rG8wufsUOb+NYuE7j2AqrvM8h1rUMqjwXSLXXsra3Y+yoexWXltxBTtNO2jrbMPlgsLWKdT3pJO2+ycGHfkebYiWHnsPm37Z5PNO0/dOp+nTJvzC/dBfoaelroXJCrluooC3kIN+HciZfpV9v80DJisUHNuyh+EnOpl68nUyGl5idl4jUZ3yZsslObGs38GtX1+B6BLp1p/yUHmmlPqjamti3+LLODLxCADVTnivC2JCzlO89DSdoQKjzozCr9eP2rDvOWyFNzsg8PQwrvolmynfRDJ3y1yULu+myKqup2FGOMOeGM/wk91MzvucxFbZSdCjz6Uk+mVOLZIzKMJqS1C49Uw+f8iT8SoJrj5UXF9gAi/dk9nPzIFpB6iYbmHtorW0RrSicgWzrRceLMmhHidWjR3LlMk0pA7BoeimvQ/YEqSMxEgjgS25tH34MzpjC+fbJ7C59G7uG3cnd7R/y6T6RFQOEeMjUynIWE+hbQc653kmxK3nKued1C2qozF9Og0ZMwDodRj48OiPzE//jCEZz1F17gxx53ejcFh9nB9rF6/l9L0GpG3bMTSc82l/AQWn5j2H3zsPEux6mEF1b6NxRJPQdiuZDS8RaZbv5Zm3JZFA80gA1sX1clt3ERMPJqMttqAz67Coq32u3x7aTnmmgq3On9k6KpAdI6KI7L6Uqxs/o3biCrYcnUPUY1EYkg3EjY9D+psOSf4XB8k6/B1WvzBeMi7w/B5oHs752GdocHrfp9dpZFMvLGiAxD/nM+1lwYMa/7v9svQXrGODeO715wjqDvIpWxA2gcxxhRhDRMQ2O8GmcX11IFthz8s06V6F9zcz8sxIlM4gRLeGvKTb6fY7TVJVEsHLgjFdZWJ8ZI7PtbOKsgj5xIhlQg4tyeMvyPqXcNG19yxZxzZSFnEPRl2BT3nB0HOcfsJKXtYqz2/XOubwVSQstAeh7nER3Os74k3UwceDnTROfZySMXFUJVXhVElEdM3HIcFrzQYO2OOYemAq+ib5ebSqOxhS+wFutxtRUpEmjcH1s4vQ+nM+2hAAYcbpXKv+jdGPP09T2hRPTX3YBW9WLuXUrBtgxgQSyhVEN0R7MrYaDL9xOCkCReMdmL5YyWW/WAjsDvRmlkoChYML2XDdUM5c1CbrYPatK8fXwgvtw1h18wqOT+lFd+Yofp11uAU7xzUH+aA5juzUpQS3u/A3yUEBQVJ4r912OcklfzGjdi61h2TwRUW8nAEqutXEtl/HmK43GGoY1/cofe004Bruv9GvA2zQbCHpVCwl37bRsm0VNq0Vp8rJiEpvlod8vJvqaWm89uxr2NUS4T2zGVz3LnHdV9EQ20DpEAn7ORkQ1RDjpeAzO41UtW4k7vxuXDUN7BohU0cKwMuhsHzoHxRnytmPaad+67uX/JyRjQEM3f8nHfuq0ZoFVHaVxwHbEryJPcOSiaoqwF3YT5MtklOQx5zTJloDt5OX/BD1ExPZP/Rn9g0ezknnD3I9SDBS76I3tAOHxsuqARAiwu1B8K8FZ/nswfWENqlY9sd9hJimMrzyW89YMn3vdGZvFDE//xb3vzeTnL059DufEoPz+bKpjPvyX0fXVUVi/iZ0xgE6Y31AuWPjjmHVB9Hr1wuCvO6La7mYcYfU6I+YSC1LIrAnkKiOKxiGvFZS9i191CPSaYsfhdbu1ZIrC/6c/Lir6dT8Tk/GaPTK+xlV/huuvDOodsJ+N+zC6QMi2u+Eg254ZTVM+0Ik/sRTSE1NdIR1ea6rV/Wwctw7LJx4nnNvFGIKuRBhLbgFFA6BziNF6FoP0u13grbAXbhEsxepb3OjcNpRWY0gSJTGvEpdmFdb2SANACtJvoj3b25byd47bXx7w3OyDEMfAE0QBARRQOGyYz50GrVZfm6nW0Nuk5y9ND5pF06hDn2hrS9ryztmG91tNJobUY7Us+ec12EsB9D7KBSR8Ff7MzRiKBmh3mD1iNo92IZkU3qplR+XPkVJzAsX1IsbN50hvbiULh9gaGvwTvLSXybwRCmPv/c4gT2BTCk8TsCZUxzZ1cGQkAdIaJqFaHMjCRJqZ6gHBKQU1LSFt3E4pxtXQwsKczduwUGX/3E6/Q8jSQLDlg4j+bVbUdssjD0+FsE9MLggUpJR4vmrIOEBKiO9ekxhxhlcFCCDauS1tuRhQYhpiCHtnB9Kq1z/Gmck6Q0vENQrU7mXppdSM0rODtZYtahdoQRYBxNimspmMyz/ej5XfzPTp45+McKtVelkVDmJKd2HYmsu+l7f/aeAEqvWis3fRnzFhWAsgMRfNShfe5mokm0eDSQApSsApdsPAQUt+r2cTr2aGg5ecL4qSUVbQRslH5X4/J6Wl8a3k77lZeFl3r7iLc4l3uMpE1CgcFgJnZjBnWfuxKkPlDNn+vqniWa+uOsL1DdmEBIcTqh5DqEmGXSscOuJjosl+9psSsdfx5arArBoanAo2z3XBuh95gDpR38acE8Bfa+epT8vRTpxHmdIBHatvycI2a+9LOFCPXpo3xzna4JGgzk4BqdODnYNZEdIUcjH29RugicNxuIf4nOuQhTxa67AHODPH1f+4QFUuQXHgKANiBPG0hOR5nNuPzDG0ZfZ17/nECWlzzzlAaX0BTCv9Ic/Mk7QlLGHpnQ7X967+gK6Vfl6MnVfe8CFZUFdQSRU6RFN8no62vgog+reku8vKIloieDez2+BgAD8lc8z77TFw8QjSEqKBxWT91QzplCZ0aU/CK61aln661KijwRi19hxqpx97+LNkLH6hyGF+snAKrdwAb220uUmYU8PWUVZCJISk66INZZeNjZfTkrLQ0R0jiQr10lCdQKKAfsHJBBdIjpjN4ml8czMO4NbsNHtdxqXaMWutmPxcyDotGxbUiWDqgawaEgqiVeef4WUB1IYuWNIH8OCN9g3MEv/7cffw+RvAkSyal/ikuOF3DPvHpyPRPVRIXrnUoVTgSpXhdpPTc1gGaDx/DSZGSbKX/bsPN76OFf+fiVhF4URpg/jtdE/klX3DqG1ZyneIK8JghODuXzl5URP92ZxRpfsJ7JM1gWOGR3D2LvHog/1jhcXvXERs96S99qpl6RSKvhSdPdjT502Jy6rk4E0M1XZi4l85DpmvTWL++ufJCjuOr4VLwT0/5Ot0Om4845HSE9OxzrC2idPINefUzTRpHuWsG2/8lHyR7T9sY+UjptYHhzFogj5XStSKgiaP4m1N62l40Vf/eTLMi8jb2Uem+9ah+Du/futAdj+4HqZmrcuD7vkDfY5LZ/SuXIzDZ/8SUTVcRRuvSe5QZAUmDRF1H69hYA2L61tdsUvAGhtsQguB8lXj8HvET9uGnyvzz018eHUZ+QgRIeyXrzJpyzZkIxBSMS/u4Otmz7EJXr3NoEiPJDwLpedDSf7s0TUJi9jSP844G8dzAPmx7j9nfHM/eWITxl4GcGyat9jZNX3BFiGeMo2z9vMkQe0qJddz7pF2zzn9p8zfc9Uuj79ielbe8k58SnJLfd7rnljE+ROOcLWK7yafYeC7+Xjkj30ugU0UhPtcSNRFjcw99SDxLcuQ+0I5+Iz7QRYhmLWmzEvMKPMUv4d/0OCNIWxLWNBEKjMCr/AR5vQejvtZ+T1vaHjQt1PAMWL5Vz1m6ybPTBDeEjNR0Rbc2i7+h5OT/fuqRzKTsx6M1/f9jV1GSNxKyXU7jD8bOlE+kXxwPgHKMxIYEuMPH4atzeQVhpGbOc1nmu4XPBDzg+obTIrm1lv9gT7/p7ZB7BrxC6ORx5nbMRYn2eXRAVOjR+C2nd953KB1T8Up0Ye21qTxmEyJBBXuJ2Y87v5NO4t9r3iC0L8f83s9v85W8H/TvtvsO+/9n+FSRL09ppwDxCtBTh0dRPhN85F09uBWeNLnSUhp1MPdG71TzT9HM3tYV10RWQiKfxIaPOiIBpjGtmw6CQWWyXnQyZ5dNgAphtuYnLFMmb+/hdhyySPqG2/KRQiMcV7iDj4J7uu3MXkg5MH3F9kXPB8QleFgmigOzK971nlZ/SzZpJifoZJN83B6N99AR/0jtk7KL8sie/GvsDvs8dx3P0lleGfUhXxGe0B+5i9HkyHcll3tZmOEN/FhtnPzJFJR7BnJdEa0eozicUwiisO3sSSHwUmWBYwo+A8qY0y7Wly80NMiZ9GWGsYEQ0WNGUqRpet9JxrVdehdPsz8YlLufXIrZz94Sx+t/kRpo35jxBVmgAdy29fTm18LZLouxl0qlycmj+dPTP24FB2sm/oYHITb5cDNIKAOHwomxYewRRgQhqwEN7eEEDUz5cTVNjL6JjRSIKLtsDdVJmiqRavp0y7FIteT+vPO2jZ5XXmhHfPIbbjOh6f/DhTJ87FGJbM6dRreKZ6NJ1+R4hvv4Wu+CuJu24Wfns3EVp71mch1By0niF7P+DIwneY91eEz8a3KvITWoO2Uzr+OuLuucznPS8zHeXttgBOZTdSM9jF8RlRHJ14FP2YFALjLuEyYBfwEPBF3787gcuA7lTYPzyOn1wSm3+4HP3eTiRJoqW3xXN9UVJhDDTy5vNvMv/z+Vyy/RL25+z3bOD6UZtRP2uJrDzG8BPtqO1q1AJ0NBtY+NrbGPa5OTQrhsOTDhNgGeo5N7grmKQKLbZ2k4eOprNV23fdvo2R04JjzToG5yd5nml8DbxqnE93rkBwcwliospDExsgylmFDyca2TN/MwG1TVz14lXcNfwu1tR9RKsLgkWJT05dxc/VP3vEyRtCVrFd8aC3/+BEVIr4dTeSdE5Gn3lMcOOW3B4az+oIWc/BKYHRJeDnUuB2u2TaK428CT+ZtpijmRfRq23ArVRjberi9x2/MV35bwqBKyDuXBatsXGcnBpAb5uFnU/vpCm3icxPM0n5KIW7Nt7F5G8m83vB74y8aSSz3prFwrCnmHfKRmJVFMnrDrL699We67rdXvrREfkzSN/TQPOpOiLq7WisGqyqBvYMzeDgIJneRpLAv7qQ1FLZCWlD3kA3RjdyZMIRJJWCtTev5cxPZ/jqsw8p+/0pBuft4LXW8/xma+DFjkps3TsZc3E2P/3yJnaxl9jAUlxDrXRFRXH84+MUfVHEtN+m+bz/hks3oHtadobnJOWQVBHOHf+cBHaB3eF0UXD2EOcn3UJH9FAaQlZREfkv9LZkRp+ZS9Y+O5LFisoRhCRIlMS+REeAvMAccToC/1P7cCrBpXQRLMagdkTwfQ+kN9r5I3M9Z8frePOpN2mOlhG4ey0wUw+XqtT49WqoGNPFsfHHCO4dR6RWdpiWRb/BcvVo8gedo3zUlVj9w9A4ogg0j8QlmiiJfYl85feEPn079X3BYoWk81LICm7cbslDzeIW7NhUviOnhOQZE0W3jvMOCFRZiR1byIplGzke/ztbRwVwOuUazNoykATKbIf5sXYUDtc+uGwYHSGdJAXnMi9dpgaqU8ezc94WVt7aSKniIOcS7+WA6Vv8LHIO8YnmG9hw0wbCq0/TFRpAbegPNAb/RYclmgMtY9gb9B4/hn3A3osdlMeuuADpLLnccPgITsvXdOlPDSgQkEQFroMFzP/DQGbN/T7Bf0O7Hy+8/ALj/9KQUZyB3prIoLo3eD8MIpVOShtmEl12kLSvHSxau8gn2wJkRHvpYAFlpYTOrMOh7KQ27HssfQ4Co8OfICkIp82Jw+KgxVpHh/9BTJoSasK+YWfCNs6PTESQ3Cic3nfyt2X+Y4bYlTWryYsxkJnYQnu6m8S8DYw7Nu6C4xxqB7ZUNfnD833RoJLIRaoKHg6GE3M0xL11n0dfZq0JVnWF81vdn4TVHEGs7UBtVyNI3qx7QVLSaZD7a2/LhU6EsrQy2m7S4g7sz+Dw3ZyGtxhoX3+YbkOgHLTos6Tm+1E6g3ApXOgaJAyd3gyQOyKqADC/+zTT/gxD33HWRxd0mBpiXPkoRAd1aUH8fP3PmP2cILhZ7A/PRXbyVfBOHntmJU4NKEKCaEmZ4FnnSRI4jDZ2Pb2LlDN/YqjdTHPQRnoGgK4G6gL1O6hOWWFz8Lccy0mlgkNM3+FH9plsBi52Akx6sk/F4u7oxOwPbtHtnf+QNZMlwYXgcCO6RE9wSnSrKUys5FzsdKLML9F2w0O0JI+nd+jH5CYvoyfjJgItv9B93ZVozRKPvP8IAgqcyh6u8IdbpswmXF9N/VZvsNhDiyWpie66guE9TzPYMKrvnbw0nv39wceJ2hdQOxIPa5M7sKptSEikVk4mqX4R/tZBXHKmi1OGbN51hDNny1T8mm1oneno7ameZ+h3nuh63dg3y9k6YdIPnoyyFlsNP0bdzsrr1iMplRg6DH3PBzEVMPHwQ7SHtVO8yERD+jSfZ1c5RDTmTlqPVbBs+WhGnxrtqU+3aMWhcvDFvduQnBI3rLgBAQGnwkRR3BOUxL5AdcRnOIxlRJ3qRtPbiFmSnehXBcD6Qb0ID39Hd1Ioo09lE9IeAkio+5q6xnANY8rWEd6gI6EwgkH1b5DQvoxphWcY2vsAZ7LPUB/vBpvs6E6sTvR83zmJq1mk1DB9z3TAxZ65agrTNuHqA3T111l9fB2nZ9/Al3f3UVNJIg5dIOcn3kz4vVfx6guvcjb7LApJR67UzB1Fc5h9Kox7PrsHSSFSN3g2g+rf8FC7uwU7o06PYsEfOrqTR9IVPRijswDrdtjg/J9rX291QcH+U7hNXbgvnsi+G2QQ2dDqzxiU+hZne2L4U7KAE+ILtqHv8tVUC+wJ5OG3bqXqrd9IKPJmbVnU1Z6MFEuq3A8bA77FpPWlvxp3bBzG5R/z5FtPEt4S7kNbCdAc1UJLaCMmv1Icyk7Cuy9mZIWsPfVmzyDcpuVYN+3hTNwouvWnsbn0vLp/A+vjX+bnVjXnBzdT81YyPcFaX702SSS6MZpyrY5C9XCGVX/B4JoPULj1PvICv/wC990vsXSNF9Rp9wun96LLqJtoozOk84JnBnB+sZFpu+XszIFzjeCW6WvtwXqOjz2OXW0nwDKcIPMofpw3mhFNMxix8184k6KonpZJdOcST/aVQujLkGlX4jqTj6W3EcsAavwORQFJHybxXOUkYivamL95PlMLvNwegiRSmlHK8nuPcuSi+L5sR1+u/4E067nJN9OrLSG8+2L2T9vPqvuqmbomiTlb5oAkktnwMlOLTqJzxNAe1o5LlJizbQ5B3YG4BAuFcY9SE/YN1zXByqQKFG6BuZtlmj6NPYbazjF05p/CIilZu2gtqm4Xo0776vMEmIfxzpx3CMoJpyPswgx0AHWX3F51Ebv+ViK/zJ5HNjDt93qfeXmguSU3+jA9+hBfJ2sooXRVdQF4Alv9VhXxCULPao7d+AW/rPiFVSFD2JotOyzDei5CRwjNUc0QqqX+RL1Hm6/fnAPWrv0gh/62ULj1KCU9UpWRoDZf/4TCpZIz21o6ESxmXJLXWa63JzPi7Ahmrm9DERtJY7p3DW1VNVAR8SGNLCeouZhO12lqQ7+n0192pgeYhxGvHMNFOy/ikvVKQm/PoTU+2ufehxv2EHFyNcn5R3x+rw9dSVPwOgAif4nEseoPbMpW2v330aPL7Xu3fhpPu887g8Kzf3QLA+epPvp6BWRr4PyVv1M4x01bSC0WvW/2r9KhZNq+0aQd/4XLfovuG9O9llWUxZyNcdhGjKEs5xaa0mfIAQBJgaIvmHxuSBFEeMeHlOZHSG94Ho1DroPwfXoiy/uzQuV1g1Vr5a9Ff9Ey0khoi4qAngAfZqOS2BfZkxWD9evdPPf6c6gcap/1bkrTo0w8djlNwzUcmHrAM47ssYCkyWWEeTVhebuYeCheZngYEOzTm/W88OoLqOxWMn56HmtAOHaV3Bcqot6leFAx664owHGmgKTzsq9poI6mQlTgVrgJHh/Mits3U5RV5F2nSgJ2tZ13Hn+HGfvuZ3T1HiYVH0XjDCeivoOJm3+jaVsTTkluS4eiywP20tg0JLycwNEPj3rYnARB/m7zW/KZ+eNMqtxVhM0JI3NdJoM/H8xliTcS1bWI+MLtHP3Al93irgEyraF1uRjq8jzg1f+ZzX57NrVD5nj+jqg8xpZb/0CSJNYvW0/iytcRJDcVkf/iaMZsXKIZtxuUGiWqAC3xCXdwQHL8W8xIe+12tH/qOPLBEU82cH87u0UrBWnfsHbhWgCM+SUkd9zA2IBW4rUy40XxoGICl0xj1luzUF/jGwxZmHw9NQdrUOWfZcLG5Rf4DQEqN8sgBYc2ED8xlKyG1wGIr43Hvb+Y8Otm05I8Ab09kZzCPMK751CQ8CDhlTto3XCczKMrsCs6KIl+lZYgGWSrM+sZveUNWo5U8twbz9Fj6/bcrz1gH5rYMJpTJ9OzarOPrxLkjC9BgMS8DQR96futWiXI7ZqJqUuHuayemLLD3Pz9zejMOk8g3NA7jtluL2AysinSd57y+Or6GTi8ZRa9BVughBgVQXuY/D2IksrTHiNyh+Pu7OHs+DDODPoIlTOs70pufjLCawH16NrNTD0wFUFS0q2U69a/XOL2NgP7pmxCEgQUHa2MqP6W2blNqF0hnmzsnjt7eLbwWS6O9EqBAMzhI7re7MK4avM/MgipXSEsOvsCJ+c/z8mx8j3TG3xBREK3E79e2ZfUD2rR2RIZdfYiEnKdoNEgKXznMLfCTX1cPW5R4quEr7n0bA1hxukIAgRpg8iSanCuns+i79KZsuJ2Ssdf73P+3QcWsEsvz6tHp3RQnlbuCfJdP/x6cu/K5a1Zb3mOz/85n013b8Jh9r6jKILCYcXQUEDDGz8Q1OzVlxfVSoon3eIzV6mt3URVHCGm7AD2Hhu6kAup1P9fMbfbTXFx8b815v3vsP8G+/5r/1eYVdVA0eDb+BPfgRenm7qXviW6ZB9nU3zLvDSe3tV4edQ7NBhWE9o1icEFg3FG+lE+9mqcGn8fdDDIbASRm44x7riv405AQHT0OWl6hQv0AZYXvMOW2d9wJsdF2/E2wtq8vMZDaj6huxuOfnCUkMZC77P23dvPlkpW2zMkb4znns+uIcDoyx1fF19Hd4Ieq9SFQ9lBM7kUJN5Ps2EtAO1hJjSZSXQbnBcgTqKc45l3yka0eegFFJ8CIsqkGNriRlDxyKck5G30IHPdgh2FKHLFH1dgaLNzc/FTOHUXIoMDk0OJmxBH3IQ4pj43lbFTlvzbiKqvlNAdYSKtLA2r1jczYuHahYz51YU7KBmD7XpUzmBsqmZqw7+hSbsXoXcPzrpGTwZdt9Mb3FoZpGV4/nDSfu9k1aJVnEu4l9OpS8gYfguJ7pV81pPC9nET0Tw0lYdssrC81hbP+NIthJgmU7KphMPPbkJlNXocgscz5nI26RY6YwZjmDYMTd5JAtsqPG0A0OV/nLzhFYhaFYZ2tQ+6DmTUollVQM3Ha5m57wFi268jrfFpapzwVKeRX4M/oF6920NJ1Jq3i866zRwC9uBLD7kXOAQElUP4iTbUCitlY69Bd9lF5P2ex8XHvZQxYT0XMzJqJNnR2XLWl0skPSQdP5uM3ux/x45xVuzaQGoGX4JFZ+G1Drgt9w4c6mBUej+a4l20RLagdAUyxvwsqY1PEs4OTix4DofRysyNJiKaIxD7nCMCCuZvnM/U9atwnTxLTL33mzhugyP2FIouup3SMVdgbTGjtsl9b5wGPgqHJxPMNEw6TtPITMY9MI5dNbuwu6280gE6Uc4wEBtFtKY2huz9nEvWh3qur3aEI7lFrtt5Cy2JozEFhV3gLHBKjgH0QfJ386sRnj2UwgOvPI95VxlqcycKu0zN0f+97g+aSK3uViIvHUtPZiM32v59IXDb2TomlG5ncN27tJ1v49Bbh2g730ZFZwWVXZUUtRUR+2MsJUu9KGVBUiBKalojHWycvxExSSSwtRytsQVJgjDbGEKM0zC0uQkoOUXRTyeZtVlJaHso7QH76NWW0uV/DLv5A0ybNtM4dxm/3LDVc30pHapmV3N83jYUwVpyf8zlnlfv5r2Xn+Sg5GK32+rT97ZZrex3ufjj2+d54fw9RFTAuwteJneyi7H3jUU3SseWuVt83r02oRaGQv2JejaO34i90/mfaSr0dGMKSaA7uInKyA+pD1nF4Lr3GX/2PuJzO1HPmUHZlFdIaX7Sp632zCqhfdoi/Dvb8TP5oRCVpKgcvBcmZ5baJXArnbIzSPD2D70AQVNP8d2d+wlccAdDEj8jvv0WMgKzuUgHb4VCcm8AktFFV2QGTrWeyK5LUboCMemKAIguD6aibCtHhzxEbegPWJVNWFX1nnvIG8s+vcXgdRzPkB1pAT0B3PrNrQz/xY/Ffy4mrDUMs0bWbHsi8Rh3BMrvl7XewsxdMz2I8H5UekpFCulFEl9HPMD2sdm0meNYX/wAj2w7xorIuzgpOdCbREKOWfE3+iMKIpbgOXxz+n26LJFMeHgCTWmTMWsqyE2+heLkWwnV17O89DpKqiNY8v0VOF37qIr8Dp3dK8gdXxOPocTGLw/sZN2Cb3yCl5VRH3A6djzW9Tvx76pHQMIh9ni0feqD9lGcWUx0pYqlvy5F6VQhCW5KHbDd5Efp4JEcnTOL5nFuSjJKPGMMwMsjVvCUy8zQgknE/agnsMe7oa63RHLprxJX7fydJ8ueZP2t63lD/wYHmn/j8KCplMa8QlvAbr7XfUbRkE5G7niPRVtkuqfILplSRfiHJfOk0AKG+XUSMf0EZ6+1YWgs8tBTBdmGENMuoytD2kMIjkplTe4alk33UscICGQIbfz+yq2M32Ble9cndPrLzqjDVni8aRDV7QFMODYe++xBHB9/HIeq3cM2IEgK2sPaaVndwhPtT2B1+o5BBseDRE+bhv7obkJrz3rmSX/LYAAcKhva9DiqM2Kxa7zIQ0lwIiAS3BVM1kodY056KVletJ7nqzPLaY6bjDVlOIv+GM6kw7JmshK4LQgK7ZNRlR4hsKnTU3cd/gc5YoF32+VNZlRiM0U3+xF8+5U+z+wWHOx0fYf2VR2dkelUxxdxIv1SKiLf8xzjckm4+sBf/WP2CA2cjBNY6gwCt5u9s42cyT4zIEgrUh9bz3e3H0L7wB0UTL2OrKbVnkxcQZJpr1LOt7PkvVSmHJzimROvD3LgwMCpA7MJbShlU+wyjmVej6RvZboOypNAceo+AqM/wuRnpCu4y+OYuCEAboyo4fFJ15C4bCpZl2eRNCsJnVnuu/3OwYEBzEDLSGafbWZq0QmPnp80IFDb72g4aQOTzsJf936FeWgI8//UcfHeBzD0TkTlCiJM04l/TyCjTg8jukzJ/IPbGFcqO3QHhx3krZTdXJqXzeU/y8+6a+YuRCEelTO4r8aU2LQ2rHoLjtV/eTVWgUanSLtD/sbqJ5ppyJzR9z4qRJeOvCEbWXlbMU2zu8kbUU9TVJNMa9xvAlj0ViS3u4+JQsCuaKc64nPPIcN+OsronaHENMR46Ia+7PPl7jaDtrWHBZvmE1sfy/nYZ6lxgl8ZGP3/RWy1jeaxC4j89Hmf/iVIImXpZZyY7EAxZxYbr7Sz7dIetA6ZluizE1+wr0PD9H3T0fc0s2/8M5zJfNSTpSJISgxiHAHueF8wIQosmi6ao130GrxBBYVbT4cbvlZuxdIxC5XTgMVl5FTKVeQl3oFZK4/pDkUP9bH1nBtpw6GVHcJNlWs8me//MxsP5Cjhq3Ff4R6Z4XHgVod/yZLB76FSt7LXqEBfoSO89gwOl68zdur5UiwZwwmcPYy/Fq7w/L5v6FDaA/cC0JGj4bebyjgx5DHaA/b4nH/Jtkvwq7LQFNWEQ+XwyewLtAwlrGMEkeVaQttCAYHcpNs4m3Ijze4C2pylDD0DHWFd1MfWe6i6bC49nfp0QgsGcdO3YxlWNZ6LzlUxvNpLBSYgklmcSfzeVlQOicTWu0hpeajvu/UG+75cd5rPwkTWl/4JQGbd6yj6vjtlr4TGqkF0awnq9Q1SYbISWx/cF/j39l2tVcv9n94PrfVsnr8Zi96CICmYFL+GKP9KOgUTFZkh6FPGMr7zXyS13u0JHigEJWqbmo5QG0fvMLJzzhDykrxg08uif2RUhYPknaF0xI6keMKN6JwpnnKVK5jkgEwiWpOYuKuWYfnDfDKNetUVKFAxtnQj2RU/e5yoEhJOlRO7ph9IIACCrLUmOOgHRTSm+bPq6lV0B5no8jtORdS/aDas490w2Hb5CayhgQwtHUVK02PMzqtnatEJskOO80SwibaUKppm6qlJqPGpRj9bGo9OepSbvnmY3X2yxQHmoUR0eZ3B5+7oRPnaixyc7EsB2mxYz9mkm6nNrySg63+MDLOUWDDajBwYdMDn9y0rt/Bow6O8KL3I4XlygGtY1VcE9Y7GqTBSmrQLw5gU2o+1Y5JKcSnkdX5b4C7MtMmgIaON3+Z8R2j5NzQGy32oLWA3D3x6L68Gv8qgQ98SU9VHh943Z+QUnOO+7l5CLx5N9dB5nE26heNpl2LSlBDX/Sx/3hSEYtI4DF+/i1+9d27T25IR3SJKhxu3U54Q6kJWsntoOidTF1OY8DAJh0+RfmIVQSc3k5u8jLrQH+Q27pu3LToLpgA3d51PZc+wdJ/6+L7wE9bN+wljkJsFGxZ49lutQVtpDZL3A36lfkh554goXc2RQdM5lyCDJy/I7BO8oBTRk9nndRD3g2A39MKNRbPJtYPkdpJZf6Vn/us3hUvB5EOjCW4pRWdRXLBvrkipYN/MBhzpg1HazHTqdlIV8SkxHddwd8if9AT1sC/nIK78QozOFyiKfYr4tmVkNrxCRPcc7o64m7TfQ4gv2uHz7E6Vk9yRuViDnVy5KoMJRydQHvk+nX6Hyap9F4Alq+egyG0ld3hu3/Ak9vWj5aQ1PUVsRRUBDQ66DF0eQEukAq4MLURXlYeh4jSrl57i5JiT+Pdpr6U0PUJm+CjOZ9VgCg7H9Q+qJ9HqdAQiUI0ZTmZeSB+QTP5OL079mtzQcgbXJmBuMNMc2tUHZPWudxAgsjmSwjd3ElcbjsE4BlFSY/Uz0JI4FoVKgeL7VpIrkun2O4VdJftP7Go7tctqOb/2PEufkanc63q8bC97qvZw7vtzlPwh70/7AwcA5aOXMPvd2QB0lHew/rb1iJXlnvKiSbdwdrhMF3vo3UN8OuhTeuq8YsAH3zrI5vs3X1gZgH9nLaV/nUNyS8RPjqcnaRiSIFAY/yhtgTtxGb+gZ8cxuqq6aDrTiFYTQeagt5kjav6HAb9jwKUaDU88+RRSs0TeyjyiF0czPHe4d80viZgCTJzNPsvT1qf57Pr3KRyViEJ00WX1Mhy53C6yl2UjXuzbd9f9mMTF719M14RLaIu5kP4RYPbviyiatAxjWBIBYjgZzc+gdAZxYOoBhNcXYpg7gV6Dlwmgf12bUC4DdQun3IZd1UJJ7AvUh/4MyOv7ttjhGIbJ5+kUXt9fS/AmXqubzvnoJ3E1tV/gb9xXvY9t0mNozRfSz9olCFPXMHbGBhJ/vA2HRkdESwSCJPgEwgcvm4A5VvYvBXcFe8YI8DI8uUUbbsHhGVsCe7PJLv4Xqa0TkCxWFP3M9pIKpSuYMaXrODZ7KgHPP8jRCbvpNnTSo+/L4kNkoR88bYBh54YxtXQGiX10lvcEyev0yq4vKUj/nN6XbqNz4lhcghWj7hzt/gewK9tlloMKBV3VXaQ5Lvd5byU6LnrzIhAEZqyrY+bZQp/y0ujXZEY3QUTqY0/7+1qpc4ma3TN3+/wmIBJefRL/k3sRO+opiJrlW+ESqOwqwurLWDH9e3SdXrBWa28r79lHUuB2oRAD6dX10mo4h0njpYot7s7l54k/s3PJAvJHyu3Z/82G+4UzPHI4cYEya8fCVQt5JPARUr5PQe3vG7QObC0n9fQabMXVqM3ewPFAcFG/Wf3DaE6S/edZS0cy7r4LQbD/tf8z9t9g33/t/wqzK9tpjVpDpeBF/I0v3sHUyptQJ0XTFptxwTn96DJR4V1BFcc9S13oCrQ2NUt+X4L+eDtnk27meNp8j3CsyhnM7JM7WbTxATpHpXB83HGf6+oVATSlmjl82b2U/thFa7gXkWcwTcIlOSkadJCyrA5ua7qNwzPkTbfGHoOfPYX3/tpO3M9x6HrqiSmWJwFvoFFE2dNBwTdHcSqc2DS+KEQA0eZg4edTmLp/KirRN9C4e3YhAUvmoDWLiC5fpKfVWMzg/d+ieftzHnvvcUaX/elJ9UcA5chkKkZcjC4rEYfOS23lULbjlJwcH3ec0iF+uNxulC6vE1XtiKBHl8+bxx7n0/2fEjcxjqFPDSVn0fX/NqLqsAvGOcYz4cRCAk2+gUSFS4HohCBzGCMqPmBw7b88ZVsiZzD00A6cny5n9tYx6Mw69vV4tUouTWnj2R9exZ2aRlVVFb1aefJbIyeO8VPDEB59/05cNjsdoR2oHeHMypc3qRZVHSd2naB45UlG7PyXR+8AoMGwmk6/4/RIXRjveoyaofMueK9DOaWM/esZfrgz35Op1m+tQVvQN66ka28uWWUzya5cidIVwFMGeCUErvn1GoZttTL8WAtZx7IQd8A2t+t/iuze5oLqbV3kdOl49OobcMT4cX7VeSZsmcA7IR3MyC8l2Dyak7ec5Nf4X3n/4Tren97Ab3FbmV4s65gEWIaRrbkK9bVJshCv1UhU91VkqeGBMb/CoiEEzByLQyEvzM3qKuyCEa0jBrU7VM5kMFnxM0oonUoPElqQFHQHddMZHoH62cfYOXu/59lDRSjo+YQt40OoNXxAw4IScvbJVHTlDlhTdSnrdUvxq3oLc9I45n40F7VWXoA4JHiqDVabwO9tPzKO/oQlIIJug/zNDa79FxfnthDjnIwkgTEkiZjKfDJKfMcL14Bgn9IlU+y9EwbL042UZhrRBAYxfPfHxJ2Xx5/+TeKkw5NY/HsCzh4L3T09/1HQymzpJqi5mKSzawnJDOO+kvtIn5fuQRSqRJUs5myT2PnUTtZcs8azoOo22Dk59iSiQSTj2Eqiyo8gSTCx9RuGV31D4Qg3x6+fStbN4zg10Y/uoG5cCm8f1HbuhT0nQKHApr0wQHlt3U9oDeFcfu5yTpQdY5vrf55VsMXpgp1wLFjeBKCQKF5XjMvuukAAG8DpdiIIAgq1AhuW/0xTwT+QHu1J6kLk7OJuvxMcT1tA5eBsaq96GMFPRsRlD5PHxn6UbEtkF5KgZPzOQwzLH4ZCUBCq7uX2IJjba+CFl19g1MFOohuiUfWJbt8QAA1OeL1DHqPDyCLeOQOdPQEJNzvj4MkQGHtqNMKDVspDbmVrdiDVEV94dGIBsk7HEPR7Lc1Bv9Lpd5SdI6M5MGQUakc440u2I0mCR1ezf8MDILgVhHTGEVEXzIi8Ecw4uwdR0nBPkJz1+nNxBGOPDSamAGLrY1G6ZQdxkhJSlK0UDClgy8KePp0PiAs8z2WZHxPpV4UoiAT0BDDsrB8J66wYOg2IggJ7wBhquodQ4v8o7/i/Q1zhVpJPyxRU/upOnp16OQvjd9Dr18uRCUc8lH8xHdeQ0vQYSlcA44+NJ2udGWOwGbOfmb9zobgEucXLh6TRFlyIReOlnzEGGll9zWq2X13D2oVrcSsEyqLe5KfmDN45v5xHmy9hX/Z1lCy2cmrMKU/wQG9LZn78DWga2jE0l9Ewz+TRvAUoin8cu8Kb7R43KYERN47A3a8xIoneTBCFm7a4EbhT5c2v2hmGQ9HNPy2ZQ7RyRviPRtmpfGbu0/xxhewMzKnZwajKX9Hbkpm9YzbS27KjUTNQx1ISOCol0RlgJLDDTfHR3zxFH4RBReZBNCHd/HT9T9hHezUSHEp5wxbTeTVfzPid26bextr6teS2+8646U3PkKIcjebcaQLaqzwAEpNO3qS2h7YS9+KtNKRE+Jxn1lTgFuzE1ckbQKvWiuiWv6mNvfBn00Sa0qZgHDaJrXMOkTtCRv5LwK6Km/ix6jyp+ScZeqyO2dtnozVrKI55ngon3Bns5KgjlvjzKvT1Ttrf/haz8RrKo97pqxIHK3se5inXkxSPv4yG+H4aavn+TUHrecEt8mpdv4az3IavB0Rjt6kIfvlhDDvrqUy30hjT6Gk3QRJxqpwYA82g0eBnSyOyex6Blj5KJBT4m/wZelJ2eDVHNnsct5+ES3SRScnmQGKLD9Fl3kOXbhMgoRfkuegnUzg7K3ZSk+ok706XJ5PnkTZYZYQP3Ud5u/M+9BF6qnZWobX2ZQ1KaiyqWtpUp2k2N/T9pkLjjEDj9KKhB2b26W1yQPnrbpiig1sCwemnoC5zJj1hKXT6HedU1njubNLwTmgVHzy8HKWxi6xD3yG65PqMCSzhyogighUSPUFu1l22jpSKFMSuzz06mApBiegSsaltiAvmUpour6FUwNeRbp6JLea28qeY/ko0fh21SEj4WTMZU/4nocYcasK/pkBzlqrhs9Ap7yWod4COhwQhbX5IV6by1V1fIUgiJbEv+vRDSQBTkIPy1HK0ovw9PtcGN1SridyRg2B3UDjhchJ7t6J2GZCAVBV0HDpD2qnf0LQ3eOrsXPyD5CYtwyLK46ExyI1i2mSiba8ztuIvDL1y1nu0fzkPJ5g4/chn1GYMHfAwcj9Su0J4PayW+7Z+z+S1X3Dzqp+ZUngSAYGi2CfYOzSNQ/Vfc8mB75lw7nvPevmlEHg+IY7yya/h7Opi3H4HTQZfaYKaxBoKRlgxnFpDvTielpaP/u3M92UO8MsHTpxjyIF0Tz+6pRnurYtkcOFgUn5IpnLEQs4N8R0nXCp/umYspuqaXipSK/7h6uDUSXQb7DKQ8G9Zzj/e9CN/3HOQH2/+kS5DlwdpDzCrNJ87vpxPzuooZuyZgSAJAwLYMs1lwfBejk09jV1jR0AgSNPC6zNnMKRzC8aIVsrSW3G2dyO6fJ2kLoeCM9ln6IrV4tdZ61PW7wx1S2669Cd8yqoiPiE/+mLsO95i6uvBLPltCQIKphadYsFJCdElzyvuS0ZSG9+OW3SjcOtQO+Rx0qFysH/qfkrTvDrORXFPUJl+Fa0OJTc1nuKnqx9BMlvQ1JQi4abJ8Gff87jw6/VjxvYwwgvlbOKBWYPXxX3Gde1xTNwxBrdChTEsWRa66rPI7kt5t+ZrFv0mrzirkqp8HKxVkR/zUdt8Ov0Pobcne8sEN/5GfwzNSjbfWsiWeVtwKYxsGiOyebQaSXAyLG8Yo9fXYtFZcGgcPvp2t4TEEaGUaFpwNeFPvcrgOjkQcsPwZ3l+8iyeNPRgULlpnqCkJtE32CcgYuux8VX0y0zfEc+Iyu8YVvMl48o2kFH/ilwvuBBFCDX60oQC1IX9iP69aDZfH3IBkDWj/hUeDNrNSN1I6jbWcfS4byBbHEDV3U9VZuid6NGN7Ai3oIsPo/en3gvAuwAPf/AwvR+dJOzeOA5M/JHSGPl5jbpz/F61CmOcEf/OOoaelufWgRkqbjfE3rWA1qSxtAXsoiV4I06FkdTmx0ltfhy1JgTL0NEUZngDnAIiZ0adYetVwdiP55J6YhV2ZTtmbZlnPVswXO53bXFyv+gPrpt0RZQ6d3F48mEOzLIxfXcOqWXeTO5+KxpchEMFY06NQeG6MFOy/MlyiAj3JN8Iff+T3vAcIyt+IkUn075K9M/NSm9m34B5qh9sfXcQbBy6g9lHJ5K5TeKq75KYtn8aSc33eQLsOe7PyJ9+L2dnPcrKZQWyptwAa4lsoWhoO8qSQpKOrsbofBqjPp9eTZmnjfVmHVJ+ATZpB+XRb3MiXQ4mR/TM5R6DDOqpzZqNU+ylOPY5AIJ6R5FV+w5joy/nyORaytLKsKmaMOrP4VR0I7hVlGSUYJ0RzV+X/4VL6fZ8U4beCaidYZSNuZqCBWoUToVH22yhH2zphU/nf8XWq3ppju7CGGhEKfSBewQXP1xyAFPqt1iCs/hm9yJOJc31vrAk8i//Q1yyexnOKjnQNubkGE+QZFzMBvSCxJJvl1H2VhlaswqFU+FZx/b/m1aWRvPWIgYd+QFlHxisIvEoW+fupdBVhHq7mamnPyG2TQbMD/OfwbOznmXWw7MYe89YGpMbAfjq1EC9NSh6sojdN+9m1KlRON1O9jZuoDVgJ8awJKJHyWtVU5OJM9+eQWv0+smcGn8cqj7fgCigUCkQRO/4Vr6tnHO/ygwIu5/bTVSpN3hfOXIxt9c+hyAKjL1nLA0TLkcaMDYmlfbS/ecu9r60l19yliO6nCSk301M5htMFmCaEn5GZkRaCUxVwjSFgqffeINVUat5/I7HiX8pHstoCz2BPZ52Hgj0E5QCdtFGpwvWVS2ivEPuv9mns2l7/zeMjUYcbt95qrHJidpPTc/gCRROmHZBViuALjWA3j6Kbbdb/k9AgU1rwx2k8WiNSLiRkDwAj2M5IWR8eDfm4FjP+r7fHGo7VdmLaTpYysejP6bb1uVT3lVeQs42EK6ZxNa5W33KituLOcz7nM8eRNl1vkxhAG9W3sWYc8U8XjKe6iETeefJdzD7eWm/XYKFVqmNmpkz2X5FKuWp5Z4xQjb5fYpjn2PP0AzCemahcPmhcgUz+UAK8V/tw/bmv7ju5369UCUKSUNU92WE2BYj6LQ4FT1YNDUeEKPBNIG1MTBl01xC2gJZ8PVDJLXK1KVWCVJUMEEr+9jMm39heWgQW0f5s3/ICI4MmobKFcwv838h/MFwPkr6iBGahZ6njem4GqWkJ+iiINRD0lC43fjZ0j1rBfldnmfDH8uJLD+M3iS/a6/GG+gG6M52XDA/mjWV5I+NoHbeTELW/8rCdQt9ytV2Nc++8SwRtaXMeu8SrH4yoF0UYV3xOh6vPstTOVuxx+dwaPLndLmWUT3oYZKS5PO9/ddNeM/FZNseZHT06AvaFKDT0klZQBk3Vt9IWbd3fSMIYPEPpyE9h6hX7qY12Ru8c5gdRJfsI6jZG2B0KzU0pk+jeMJNjH/SV17iv/Z/1v4b7Puv/V9hf0+XFt0awo2zCD3ehKOxjaZ4lU95QsudzDh3Xl5wir47ZUlwYte4+f3K33HpREYeNdOj2U1DqBz4mlS+nfDONBLKyrAHa6mP82ZgRHVcwXnzYVYZhtMQ+ju3DXqO6edKiOmQdeJCe2agVcgDv4SbSP9IBtfKiL1+OovCuEe5/fjtONQSQt/Gox/x6RLNdKsL0MT7sXPW3+gGgWt/uZZRXxSjsitROpWoBN/MOUlwYtx0gCUr/Fl8sJmL8rx6RoIkoHD2IoWF0Zo4luiuxZ6FTLV0kAdbdRwYMoHC+8ysWngv5VFvA9AQsprNpZs4M+oMrVFK7nntSnYM96aYD635lNTIDRjfqqA9px2X3UX0+9HcXj6Y1CHP/i8RVXNEDa7ZUDm8ku6wWATRV9T2zyv+5MSVbjLWvktc4XYf3QyA7RdvRwo1MPJsBmq7miilHMjR2mM56vDni8YxOK++ji9v/tJzTmFfDDUorJuCwcW4g31pLAFqIj9nmXYZ3QusVCf79iG3wkL4+TcpffI78PPz0Ef6toWsxeUW/5mnOaNQhSoqxBMolJB4wgDPh0KSOQCtSSL1vBlOS/82snu6Ar7dA6MjDnHC9jOz353NbcduQycYPNl7dpOdn+f+TMMv+4gp3MWer0o8HPgxnVdxa8BvzE5fRtnoJbTFjUBtjyRDBdNDauiJfZuGrnMs+dbMgg0LsKtaOat/n4aQ1ShtJgLaKgmemMWaG400xDZ420pScHDqQc7mTAM/P7SuTPwtg4nqXExbKlQnwxPOENQ2N/HXZaAYJvfLKie8U3gHSaGfccmqACLOlXHnhjs51yJvBK7oY0PZ1AvWOVaaUyZQMfpKDk1v7GvPPiepW0Ye2fxCaEgeJNPeDaB7dUkOz+Y0vPsSAApNEahCdYjDHiV6zHiaUibSHZ7a11byd1ucWczhKVXYu3rRKXT/GW2tTofO2EJYXS4Oi4vQ9FDU/mpPsE+tULNu0TrU36ppzmum9nAtR3p+50zy9TSEyvpZklqiatgC2uNGeN6xLXAHJzNvID/yc0KGxCCFPsaQht997r975m5Mj8xD1dtFcLsX6Te4Gp5bfxf6DS6s1a2s+OXHf7vvTVHAh7th8eZ5ZB5tYuF3C/F/2v+CY2/95lasS6zEjIlh2q5ptI+2slx54TX/yb5SQuqQUcxcs4m4cm+AqCV4E06NAqc+EFdJBSE1By7QJgs2jcWpC6BkRBo1CTW0O+o4arfzZBvcEeGgY3AxIa1O7lx+J6HtwQDEK2GiFiJaw8g6F4300cfodmzAKRop7j7NGiN0OEVOJJUjRcKSn1MJb/bHJfhu5M5Mkhf7C9ct9EHgOxXdhPfM7luIe2kZAQzGyUwr66RoxksoH32M03OfwRI8GEFSYnTDEQu0VcQyc/d4jlxl4acbf0LhltvyXCJ8H7YBd0g3Ad0Cd315F1GNUdR2D2Vb2W2khJzhgcZ3efzwZaQX+1O6REVreCsiIpK9h1dmzOHKpLX8eu5X2iODqEo4CUCXG37Jf5FBgeUMj+xB4VJw4083ou/VENEzlzk9N/O+NpvucS2ULYrCv0uHvlePIAn4WTI9790a1orzjWX8dNX1HBo08QIKMoDWWAclWUa0jgS6/I/Sqy3hueQvWROmxM+oxW2T1wT9mmoe2sOebvy6GzBGWX0ou1IDKtm2JJQz/7qdlc9s562dYxCvWIRb9GolIolMOjSJGZvM1GXNpmqMHMirDfueTr+j/0jj+ZItlyFF6UQcuJvRW+KR3E6c/ahdlzyf6G0pFGe1IcQF8u6UdRz7TR5H1AKM0jlZ5x7GZ1f8htlfYMYGOUNuctExohofQyVIzDDYaYxuRKzoIKYp2ef+AZahTLBM5WDOQUq+8NUqCjFOIaQuj/JHPqN34bXUDJ3nmQf6TRKdtP2xl5x1R/E3er/X1qBtuBS9NEY3Uj25l8LBhQRY5MBHlAI2zRnG5dNGU6PbRsGIBqqSq0hveJ4p587x5/nHaPazsOK2b+gyuJl8eDIau5oAyzDsEpw1hXPKEcMVn9xD6ka5jarjT/bpL8rtgCTrxuB2IPU5MPqDjf3fkLuvzYPMo4jpuJq6zlGctOo5MuEI5tQAvKqHguc8hVNBYLcGx9kCokoPIDoHAKokEbPeTO64ULbeUEXxoGIUfXPY4kZ4qKWDI5eXUjjCyYMfP9jneBPZbAZ1GXzUncCcVVcz8kwWS9Q/ENchU+lUOODaJljeA6oeBae+PEVEdgQ9gT2e9yqNeY0NMaP5q9oLVuq3fwr2JbTdBshghOcaldRsn4iqvpem9KmYQhOxqRrRBx9nS3oxt4e66PXrxZQ2ktqs2UiiXH+7K28k+/g9rBpylnXX2DD5m0iuSkayHfLc+z6/Sl7tjufez27ELbn7gqeQpJIzOG9OW830xBsRe10knttMo2ENBYn3czxjLrpegaH5Q1E2CcR0LiG1+TFPH5LfS+Dm72fBb/0OBYGm4LU+737sngmsv7uKXv9eopVDiOpczOud8FeHihl7Z6Bv7sYcNgxUMaicIWgEyEuEBN09HMnRUe+/iQ9OyLTvVZEfUxv2PWX6lVy08yIu/U2D89BR0o+sQNfjpXGv7R7KlS1udge1EtJUzQsvv0BKeYqPw0+SAP8+cMV5GUgFoO9V8OIrL5L2kcTEXdVklGWjcGuZqoUXQ2HpYHk/IJW3kF6ajujWMbZ0I/6WwUT3ymsPhVMiqKGB0PZAJBv/EYhIaVKi2HyIkYdk/Ztuv9OctsEnUWbujMwg5MowzIGRhJt8KeQFFEgSqIUL17IA40q2kq1dLFOe4EXl91tNYg3tEV1ymVtJeM8sQow5jCpfjSBAW0I2piBHHyhAxKKRnV1Fzs1IosSJSd2UZVT2SReIqBQ20kJOMablJ14d2UhnSC+9v29BZenxuW9Pt5zFFFxvRddVzKmUq2kw9AMmvMG+v2tn29RNWBV5RBXJ339xZrHPmrB/jHHMGsqBOeHEtd+Mny2Ni3ObGVmxAqfKye6LdqM2SyxZvQStWU9l1Ad83QMRVU5O2Pr2dVu2oC095KFCBVAJGnr9ejk+sYPeIDtKh9Lnu9jecjX7ss7xyz1r6AiDwrgnKI16w+f5o3O8gLW2sLa/0aN5tQoBuvuyHtoCd5F9JptLVyYQ0N43lg7UejPdQHrTB9j9A3EqnfwdpNNoreOPP6ai3v2RZxzq0p+gQLeLaksoi5oiWfDRPST/4gtuBHCKRk42y+uI4A4b8e23EGKa3PcMcl0HnVfi+PhLhhROv+B8+RpurDq5/w1XXMncU1bmnXSQ0fg8acoZjJ43mpSXU7BpbITr5X2kQlDQVdVF7k+5bL5/M4kF/QwEkqdPSIKLqAVj8P/AH4vuQif86VGnUYwMJ+haWdvMEwCQFHSEdlD5YiVFk2/leB/2RPyH9Yx8nwGBMbeLoOZS3D1GjDnzKBoss0BEdC0Y0H5uXB1d6IwtHh9If9ZY4bAyTi54kdqM/nt6MzDqXbkESYn4mwIYe2IsUU3/PILkjw3k7SfevoBNB0DSSIj33UPRqH6WFPmZInrmEtdxPSGKBE/dyebVlh3or+lfl522wXGjgbQzI4g8L5E7upXK5Er8rVlMOn+QOad7GOy6AZt/GE6tvw8j00BTOiW0eXI/iq+N9DyDKIj4mfwYkTsUllzJsfGy16HT/whGbSEuwUbyRckkvnwLbfEjqYh833PN0K5xLPpZSfRmDcXDdbRHBnuyvN2CHQEFh6YcwrSoD4g8IHupW3+K8sj3aAzdx7gfrFyy7RLPvneEBoZroCm0nfZQJ0lN15NR9wyXB73B2NINJLTe7gGZhtWeZcb32diEAeBywY2ttoW44t2opoxnx5J21l5x0AMe+eDoj3wYfD+m240ERwVz/8dXMeHk1aicBs9zAhyZeISEeyZxLltBWfSnuAQrbYE7qIh6l7zgfCb/9ShdcbPROWRgV5xqBC9Nf4lrhl5Dzgs5bLhpwz+2RdobaWhCNZj1cibsA0cuIy9JXpf0jw9x4+N4sutJWhO9IB+1pRudRc4KmvToJO7Ov5uAGG9W2dJNS3m49mEA8lbmYWgq8pRJogK7U+GhFJUDYgJRHVcAkD82mJCHb2LwVYOZ+NxMGkI2sCU7AGmSm+Rrn+XAVLjPEMS1+hjeTU9lzOMPUl1Xx8OPPOIJGAcMD6D56WaqkqtgQGZfdEM0D33wEBtv30hgk55mF5xpnUCPTQaABHcZcJQ18t2k77C95QvUL4x/FGODEVV7E0j/7BNqOlFHQv5GtMYWXm4Yx4bRAg5lBzqzDldVG6U3vU78uS20BG1i0xiR5mC5XUS3i+5j51Gbu3yCfXprGnq7vF9wqyVOdZ5iU/Vqn3uqHCoMnTocvb5jdqRfpKc+ajLjqBsv0xf3+zoTlPBaxqtcagkhuSAejUXF1ILTMuipbxxoDl7PzB8yMJc+S3xZJ27R7TPfRHUtYlzwggF3FXApenEouuiOSMcxfjJi9kiKM2XA8EAgrKa3A+vOgyz9IVvWwezbBwaZx3DSCsFh3TQk9qAK9voqv+uBSoec4RfdGE3o6V4MHQYfX0GAZQhXjbzK8/edjb57vo/FJG5ceyN+i2Zzbsb9SAqlZy3usddaiS/awcj8qRhMk7CpfSnTHf+D9t8xYR75SZsxjh7D6VGnfcpcChdnR5ylMXkQTxueZNfw+diUMigxrzmPxX4wVw9BxTKo6ao1VyEIgmd8UQgKJh+czKzfNjPsXDbTrR8yI1lm4jjbdJZX9r3CL/myD9wleeujw+IN8goC6I3NqGxGxCDfLFBHr53Ykr0EtfjuQYfs/YyE/I2cX5VH7WFfMNb/a6ZQKP7XB/1vsv8G+/5r/39vksQFGm6jKmRnt33iNMLvvpIe/Vmf8n4xb0mSNzwKl3fAdwtOWoP3UzC0ALu1izEnh6C2exfKakcElpBY8mY+SPdQX657SXAiCgr8TH5k5Jno3lxDoCUJjSOqr9wlU16uuYLLvveneEMxoa3+DKv6knDjLBoMaxBQoLVqOTpriEfDqT+zz6jLZ9OoaXz50FccnXCYv1tTVBMdKX78/OAm9szcg1r0pcHIOB+JLb+E5qSxuJQadPYEAs0jAOgM6eTtxx8n/45Yaob1B5i81KWDigYxb8NYzC0yoq1f13CgJZZZGP3lCPxNXqdgJga+m/gsFw2toWFCA/m/5HP9D9cT0h5CQtp9aMNmMxmYji+iKgeYDGjCLkaYoOJ81nnKsidf4KT3s2agEg10pGXTGR6MW/DdmJwacwrHnTfywcPf0xPYg9Ytb/J09gSua4jnu1NZSF1dHt55gPMOeLE9manKOo5NOInrvcOMPzoeUVJxOHMqW7L9aQ3cgV1jp22pmb2XXFgXkiAhSRKK9hZUViNDqz/zKZ+xI5Pzz/5EaonhHzUmts+vI/qRJdjVvRwaNIniuGeZ0xdXdj/2NUcuFzk7+1FaLd3/NrL7RgcsP6DjqbseJeGLTkJSQ4gZE8NAmmilXsWl31xKU+okCqfcTsDsiT4p+G43iBoVhsZChu7/gprwT5msA2uvluLjHXSdL6AzMhl/6SbSG58FoNP/MO3Co2QeXUF3fo1Hs68frTypeD85587hZ0nCXd9AztkNJLbcTZPhL1bLDBTYPr2RsYcE5r57LWOv9ooCV0a/hsvVgSC5CSo8gulJk0fL8jZNLC+HyIt+2yU2mlMmyu/Qt9n8e7AvqKUUQfAj2vwB2ZU/e+7hlOyeOtA6ooltvw5X7zAi/auJ8q9A0KipG3wxXdEy9V2/w6sitYLT4+roPFZKaG08KzT/AW3tUDsrFl/GF/fvJyg5hJ66HuwW7yJQrfSOSddtvo6Hax6mynKW+tCfGVQ0iPs+uQ9Xrou2xNEYw5I8FHCCpEDfqye4VY3D0ucg+FuA3BRgwhmmI2LXr1y6wcuxXmSH/Nos9M212NuMbPlpzb/d9251wOFjMKQ8jbiSbn6a/RMd13Ww7NtlRHcs8RxXk1ADfUxZfxb9CaPhkIt/OwN46pw5VKa5LtAkPZYykmrXR9gPniApfyd/tL7mKdM4I1i0ZjjRG76iYnAsDbENRKiT8e8dxdm2UfyogK1LV3Nysj9HZp3BqZazEt7ohHtb4bLWS7lk6wiEzk7ahUK69WdptTVwVROMPH4LlQm1OG4UqY/twqFy0BrkS13aGtVFyxg3VUlVDHSg9Tvj5L7nq1vQGXCIwrjH5HJBxK1QgSAiSAp+MsK0OjiSVsKPN/1Fa4y82e4P9jX3DaGhIrhFF1atFbfoRq/q5pK0b7hq8FvoHR3EJiZTnzmDjkyw6qyIgoJktxxs2GvyZ+HahYhuF6XDZKeK2x5Ofst0RoQUMkIjz0eFWYW4lBJO0cj4tA94IGk/S8PmERV3E9d9PJuLdl0ECCS13EdCH50KoguptoURZ0cguhQ+AVCtRcvczXMZUjCSmefKmFC6HUFScoU/TA89gKo4hfs+vZacl4K4a/1OMhpekp9NcLKxbgV/Zb3NxqVDaRpm9GmDIX2fU9nqbsrfPILDLvH113iC6yBn9qkcKtR2CZAYpJ3uOV927sjjzsD+3OJ3lJzADiZXjSXseCeBbVVMP7WWrNp36FaWIuFiQukO4hw7UaRFYD50ljbnOfzcscQq4GQCvK9cR2EinBsvsn3hnr621HK4+jpA3liHdISg/+0s1+Wv9HFKIwnYuq10lHbIQq7gWe84FUZM+lqkQDXo9biVasaV+VKk6cw6enYcR2e2ebJLB1pTdBNF87poim7yjOdfRIBSgMI/Y4ned2TAeKsgL+kORo28jpWxXdiS6qgaPpfkr64nvWUFWkcs8/0gwB1EhMKCOlvOLrSESZSnlSOgJDAwEFFQoDfref6155mw8WOia/q1kFSe+wAeGs/49lsYVbGK1Uc3cmlVFtvmbMOcpGfpt9Es3PgQQeaRfeeJRDZHcss303D+uoa44t00Bq6gW3dWLkeBXWOnIsuf1lj5m1I6g4lpX8puC0zTFDJq8McUZp7g4OSD1MXVIQ6g/HVr3ay5Yg1OlQL77oMARHYuQi/ImbgA6atlJ+LwW4aT1vwS6Q3PIwzQO+pfp5g0xeQn3EtZ1JuIbi1aWzz+kpfCqd+WBYLdqSBz0yXo89vZOySLzaN0dPodps0F+y3gsKkIbzVQGVPM6stu50yK7JzIjtrOmXGfszQARJfEhKNyZtv+abLOqdIVwEVs5s4EE2dHFuAK9Woodbrhpx54/8hPGCZmoc5IRHTaMWm9tEaGdpEr/7iSwF1KUk/+RmBLKQF99LEgr6EOTz6HqFEz6sws9LbkC4JIbsnZN4GDQhSRkBihhtwsC82PfUF9tpcJQuH2I7HPlyQaqjkw6UuSzvVy1crFHgrOfhsTOIfgThH3pm0EtlUSV7TTU/bk5Kt4LEhAMvph16g4P+h8n0NzwNgtgXvsePKn30fhFK+zx6WUODfkHJ1jRaqGLaA3OA5BUtLjhqMWeH3vp4TU7McyP5X3Hn8PUdIQ2T2f6QUFhNrGklGcwcT9enLnTmfn7J0IGv4jEFGwNRhndho/3PiDT1mQAuLCE1n6wTJAQGX1XZOeTroa85FH8F9WyO3Lb/cp09hjiOi5hBBFog8LyUAT3AIRZTpm7J6BnymQ9MbnmFS8l5jOJQgC1A6Zw/q7qyjNKPVp4/45cNA5Pfe/faucSSwJtJnjuXpNN5XB0zlghYrUFloWhXFg2AKKY14acGc5QH94aTQHph6gMeQ3TqfKDsmBmX3/FDxoiWjhz0c72PFqG8fHH0dApDVgJ81Bm1C6/VA7wpAkyGh8npFV3xNoGU555LuUDLh/cIuWwUWDiW33IvEfCYYXgwWuXn0VnRlKVs5/gnMJ93nKJwUvwa6xo7GKDN8UjL/JH40znFm5DVx8pp0fqp/lE5eZ5uhWUs9s58Yv/agO/5yBFpgeyem5z7DuuigsOotPZl9//ZZHv0VN2LceWm+QswBPTW5lWOEoxp2+A3/rIE+Z3h2FPeQiumNiuOGnGzC0e3VaAa5pgg3VBmJq9ZzYWIJJU8rBweN4xXWMscXDOGrTU51QQ2CVmyHnhvic2+V3gim/TKE9sQOF3TegJkoaRJeO5G3BSE3NRDZ4A7r9dMIANqfDM9eoRR0KSYOId40PUPdFHeOOj0OtUJMUnIRLcnHXm3ex9sa1nPj0BOm5csbr/iEjKEx4CICoei1lH29GUkgXZA0CbJ27FcXsBFwu377vAZ1IbnoNcbRG+c5T5+IfYJ3/PMrNsuN2oKafSXWC9BO/4Dx8BLvbWx96WxI6ewKJNZNJL1ShWTyPczMf8DjxvbqSbs+9AXS2ZHQ2WQc6W30lD1XsZmjRFD6+/+ML2IkAbvrhJsbvbcGityCJEsOqlvfVdwhzT5n5KOUj3CVleNalf/ve++tbkBQoXH4oXHqPM34gKGVQvRyk7nXDuIBOjHf+zMFlLo7m1FKSWYIhSIlC0qF0B6ASVKisRgLaKsgsNKC1+O5bhuUN45of5f7aFG3n6AQ5g7Pb7yQ7TR+htqsZdWYkUl0Dkujt8/uGDqFXex67vx13hAZBkjAPYJOI6FmIU+OPQq9laO0nTCs6TVSnTN9XHv2OZ+7QHGpl8Z9L0NpDmVR8kIz6lyiOfZ6i+MdpCfyD5nTkNUFfQHZRI8yrh2hJQUiLwGU/N5Kz/2LG+V9BafTrnExbTGVXIYl5GxCdvay/dD0WnW//CxiZQsm467Cv3cK43DmMqssltkOm1bS7dNw86lne/eI1Zj06i5bEMcSa3iHAOtjTZoMUc8kZlUPkVdmsWfg855KfxC1aCW9SsWT1EigEdYg/bqXas/fY1/UDNe01fJb1Gdsf205icCLJwb4gM4DwJeFcUXzF39hbJIbs/Zzlo+QsQFEpog3SIim8QZ60U78z6tyfiOI/O79VehUqndyX7jp7FyXjvRI9mt4Olj9Xg9vp5tgnx4g8sYHyyHdpCpEz5LsNEsrEWDLmZzDqwalURX4Fgpui+MeJs93AbVUfMi/ha26K/YL7gp7kgzc+JCpK9t8pnUqG5w6n63TXBZp9AiIuhQyayvs+j2lbJqAU4KWxT3Fp+lf8K1DCL3AX8Z8/jT5cT7QqmpxEbzaTW7Ry+L3DRK/7isFH96N0+AICBET2Xb6JiOpTaE1tVNq9mehjT4xF+d5uRJ0auy4Il+gbSAxrstL86x7Ulm7POKG3pjLzXCmTCw+SdHYd+kFhrLhpBX+3msQavr/9EG4lRDd4/Z3Nvc2Y7HIAcOT+M4x/UR5bmoLWMy/+OtJr36LREo89L5M5P88grL6IpHI1BtPIAfO7SEZJBqOOJpKV24HerPfoTwNEdS3kmpjnPH/3gwSs6lrKMzpozUlHvGwOjWmTSG5+CIXbD7fgoC5kJYmnv8S2aRdu0YUkSJ5gH8DYWpiccZh9cwooqT6LRe3NonuiDV7tEFl5/UoUDhiRO8Jn7lQqRBRqBbeX3M7nd/vOuRF9cg6ZH2bS9e43dPod53zsM7QF7vQ5rnepkp7QJAJ5monFe33KYttuYErwVfyTiS4RnC5M2cPJHZnrU+ZSuli7eC1NyWkcrjtEh/8hmbJZkOf7P2Pgo9Z07P7y2LPt4m2IiPTHlwREDwW/f0eNj8/vbNNZXtz7IivzZLYkp9vJgg0LePGlF3F0eX3vggD+HbWE15zGcrbER79WGaCjYNpdNKZN9R7vcqJw2nAr1Rx9cw/V+73JJ/+vmUKhYNiwYf+fBfz+G+z7r/1fYX9PTRfdWqrDv6Qi6ixdf+0lpaDSp7x/sS1JoLPHM/dML6PKZbTnwE1f7UwVHz/wA8YAr2PQrKmkNOpjqqP+IGX5flnEHHnDq3IZEBGJaIlg+CkzZ19ej7/be251+Be4JCcmfxMOtcSqy1YRUncIs6aShpDVNBhWIUgK7vryLkYc9iIg5paf5+Iz7QT00UmlH07l3q8/v2CxWz40g+b5U3H3UWeo/6aJl1oagauxldrBc3BqZMfvQJ0XAIvURU3YN+Ql3sH5uKfZGiNrdU3rjCCrKIGYP+XA30BTCAoWbFhAQrmVkzeepdevl6HVnzK56CiW7qmcah9K1YST5N2WR297L9GN0SidSmpLv8LatoP1wGzgQ+Ae4CPgYmA9YGvbDkfkhVVe0m0+tDExHVezaNcGRufnkD9dy6+XTiU3eRl/N6dCgaCIROtIwNwRDIBd2cYsSyOL1iQhrf6CuLw4z/F/RsOtgY1UO8GmtiJlGugO6saiqaYj4CAuRS8CAvpePYoaPLRXA23donUYH4lG/8MXxBbvxt+a6VMeXx2K8WwFczZlsOB4NyMrf/Qpb4wqpsVYwdHkJI+4+nEbCKXwTqfsCHNq/HE67P8RsttogeKUcnqTFORV5TH1s6ms6JWdxi7BQtbyQVxpupKuiEgao5t5yTqVE6mLPNdwud043U5MhnhqklW4RTc7zPBFk5bpO2ajKO2mcsxNmGKuxM/qRRcXZK7h4PQWJFEkqVREZ9Z5nMMB1sHEVatIyd2J87Oviao47NmkP1S0mEs7H2TPuKOUD5KzIS/Pkjdak7TQmn2M7MZUOq9X0xsRS2qxl5Lm/aLbsLnVvO9ljCK6ZB/pRXIwujj2GQ5n5lCj3NUX7CsjtKGE2M5riO241uMUt2P2LH4CrEPIrlxJaf4PPPnnRmx76ug+66UygL9pJ+HCPzuNtAlPscf+79PWGiZEYtd0YdX2ULrxPB/Ef0DxVu99FKhILUvFvt479vVvQCRBwqVw+QCva2thffQo8pPuYtTpUVyxIpNjL25m6J5PCOjy1SPVWDVI3SaMQydiibiFUeUyeOLtUEhY9DK/3uZEiPGjvbbjP+p7db2w+dGP2XVVAiNvHgmAMcBIY4iXmnDHxTvQP62nq6oLx+8Owi3huGfDxYr/ccDvGHCJKJAZm0VwXDgHZpkpT/OlxphwZBjTV3ahGJ/Niht9NzWCpMQYloQpPotQ41TmnO7h7bTjzC7dTUblN3xhCqLMAZLfVBYsXMuwxrWEGKcA8F6YyI2Taikdey2rHzvJV1c85QGX6AWI1nSgE8CRA2uuPU5beJtHp6TfXKKTiiVuTo055ZMdJokOqsO+wu62Etl1GXFtN6O3J3rKa0I/ol6cTE3RD9Rr76Qw+g4CLcPlsmTYkRKIUnMpgc0S0Q3RKPs0+ybWQmoVGE6OYNruYPbM2ENLRBt+/kqc+GEKnAeLatlof5XG9Gk4VH0UooICUXDTaEzhuMkfjU2Dyu70BBFTmh/hfNtELt35K2tqgsgszqQoqwi7xklbwB4KQuVAYYMxHUlUcHZiKeWp5YBAcut9ZDTIFH0SLhS/72fx2sWoHRoYsNlS29WMPz6eUVtMDNv1IbjduEUbF/ft5R6ROjk5Rs7s1bd2oHTJCEObuoGvS14mV/yJhOI8pjye2KcPJdu6Xrgl/2o+GHeAHUuqiKg6TmLuugGbejnYt2/6PrYtUhNXtJOwk96Au4Doabv0xmcJMMuZGCkq+Dy2nXGXncJ5/0Okn/qNjNwaiuKfYF/aJDaNUVIb9g0AlkVxvPnUm1RHb6RXrKfdDZ+evwmAgt3jiWpOpyxD7tcVke9TZA7mkdxGPq8I5rZvb0OICseWMsiHRrtXW8KxsKOkvzqEE5Pkecrd5wzo0efy58ULOHVfCyiVF2TFpDQ9wrwTdbhNFsoHR9Pr30tQ7yjPu2kckQwqu5yZb0Qz/uh4OgLkAFanGw5aoKc6mrA6kdu+WsCM3TNwKkyYAw4zPjSPXb3QY1PgFvUYolNRS6EIkpIQEQYFVnOV5hzv3LYDaeJEOjP7sjIRcDodMt2m0klXUBcAfsZ+ilt1X1t5M90GWlzAeXZk5HNTgBwPtfvFE2K5lqiuRX1HiJj8TZwYV4biykWsv6qNU5l30Rwsa9gF944l0jgXQ2coc9fN4OIDnxBmnOFZM17hb2e+Hnr9beycvZOq5CpERJKU8G0EnA49w6TJ5/DrMeHcsRe34MDQO5410dCTKmvNtWcZGbZ0GGOXjSW98TkyG17pezf5nVx9wT6LupbqiM+pD/kVP3sKs/JruN50lr/bzYFwTaiDb5d9i3FoMNf+NJPhuYNQuPV0uyFNBbNbcrjzq5v5/7H311FSnGvfP/qpau+ZHnd3BQYZ3F0CJCQQIsTdZcfddtzdIQkJgZCQkKAhuPsAw7i7z/RI97RUnT+qp3s67Oc5+13r/N7znPPse62sCV1d1XfdVbdd11eCSqvpMZzHoq10v9UNvbHMd7zBrG1DSS5Ppi2oze0nI+HgvO5GNvu0cXTMScTftzDquMJga3XCtU3wZ80iGiyVaO+/iHMz7kHn8PjXFCdu48flP2KNlQhsLMDpOI1Pfwpae8jAz3Ngch66ZpF5W+cxpnSTdyIbSN1eylWvp5NelI4oCAT2juXPKB3JeokA7WxC642M3PQcXfa7kYR+Wp3wdF0U1zYqYLayISq2zd3mhSgPsuUw/d6HKB67AlQqWkO62TzzJZr8NwHwR/G9/Nbsw8NvPkxgcx3rlq+jMbLR3f8dYi9vdo7jW90YzAESeZmrKAtXJA1tepn1y9ZTO9vBmeEVVMbsRkBNng3G18K72udIOrMLR74ilTkg+aa8nYqMbFijGtHl/2sZKv7bzPcv1NA3VObo+GO0hXgCMefiwGw38OXJtyjfVkL2vs8IburyOrcx6FdSzioS5k3hCssxvvkuQrvmMbZEkffyAqX8LSn76GuPcsnqmUzdO5VRJR8hI2HQNTAm4TPKspPYnZ3pTuz9nTEGMOKoMqb1+PYMSiwIHI+9ly1n4hh7MIn2sG6agw9j0Xr2e8b+JJYdW8aEHxrQ9Xvvhwy2eCafP8GGi3dfoA4DIIsyksrp9tYBgSPpszmWupCIjiXMyWshCG8JxKrQT+nTl6NyqLjm22uw++o4vvBZMhsV8FiWFt4KhQcCZdKL0nGqZVrCWhBkjTuoODPwRl4MqEVlvIkzs1tdyTo1enskkdo+vh01kpf9VWgtasyBfuRn5TM4XNNq2sklfw6hp/8+Zm5sIao+ygswM/i7NcErveqv1VxNW+ZEEg7YGHdsNsHd093HBp6pUx9MW6hIuNk7KBmnhvjrNtIa2kpEwYccyPTIeOkECFJJHJy8D8moYWHHShYctzP3ZBezTzcT03YtAEeuOM6Zibk0+W92ewolN/2DBaf6SL/yFsSpkyjPVAARels0c093uNU2HJLT7QenEjwsj2a/bey1fMTJ4pOE3xTOzhk7qeuu44lJTwCwzWcbS9cv5YrfrmDfzP0XvAdqu0BfRRNO878wTXMVp8NB7Y3nmfHXDM8aThYx9hoJXRuKqbUcjSMEX0sWersSNG837aNKu8XtIz842bc/ey6/XvwrfSn+BG3/neGnFJZjYM9EMuteZcbRtxlxuAVnc5vrXG9mX86JdHL/eJ5he13JPlsMSY0PEamCYbp2xNICoov34FQ5sWvtjC3exsiyn1h4XCbaN5Zen17UDhWT628huf5BAnsV/Q5R1qCSDex8bC+s/p7gRhfQ1TUudxlP0hjwGw1WZfyKab+G+ad6yC1fj8kylPDORfhY09z7qeSmh8ipWMnpfriqEXZqurGaJLf8Z17QMxRHvoBFU8t3upHk7Hib9MPfMXN7iheoWOMIZETYFBw6A51Lb2LLpa1eY1yHs4b74k9zdM5ypIQETGZv5kl1yJcEvxpMwTPvkXHga689nKl/BIfmDsc+KYiM/V8SWnnMiykJMOHABEzrKsg5k8W0c/n49CfTGPAbVq0irzlnUyzBjSaCxZ9Jr3sRgFQN5MXDLb9exoJf/GiNGY7FPwJBgD5dGb36Em7aPZPQ6pNIcjMnR53EoVXa+VY/JS7TYI7nUM5LiIlxSvDc5RsPsCD1YwJ3RUD9ViKGR1A99CL6AgZ5uiFys2EzG5ZsQOgU3CBZBRAqklWQBTVw/NwGSkP+QVPARgD65E7W5K/BLts5vfI0b1S/Qfl95cxMnOnVJg67A1u/d2xOFiS6g+OJnazsY/rN/TScavCqd2vscBrjRuFwyFTsrODIB0fcwFRQfP5qjyjtqg/Q49QaaPLfTG3wt4RUbEX3/Ur6u/sp315OYFUeBbGPDKqAHanfjizLSBJYdJXuQ6KsJ6qyi+AmKwh4SYeCsve4dMOllLxYQu7PuSzredwtxQ4CzeHNfHHrF8x8byYnck8RoQKHpKatL8adVJFluOXoLdy64VYemvCQ+9pOwUrKfEVVI6ShHlOvx/s4peFxPotyMumluZiDE+gO9k6sVsdVY5+dSuw/b6cpeYJbJWygJBYrYIE+/0jPODHAghMgpPY0XSfr+K+K1iai/e4oI096+9U29igwo7D6NgwdSn+QVBZmxS7CUPcoDlnLTeNK2H75bvxaK0g+ud5LZluQRbe8P8CMYz+RXf2O128MTjq1+ik2KTZNKwczJnNG/RUCIlm1r5Nd8w5qyQdJsHI66Rr2TfwV8eKZrLz5D2riatzJPpuqDYMA/iLM+SmNvfet5HTCDYAST4pXw2Grsg8oXADFacWonJ5xpkdXzLr8daxpWUNzeLNXXU8nKUodNoMNR10jwdX7KY14BV9rBqn1g3yhlwRQPP46pa/L3skdUdYSrIl1x6NB8SgM6p7KlWuuZP4n3p6nRquy/ngwaSXTzhYRVdnJLR/fQmR9JIKsQhAUmep6B+Tty8GnqZHeHyI4NOEQgiC6mX0CIt9e+y0H542lMiuHburptSlj+4AX9gDQ0Ck5aYxopCCzAEEzSO1AgJqsORRMuJGOlb8SXHfG8xwFEYtfOHaDZ28qq9Scmvc4Pl2KGshATOh/Y5FlGbPZjPyvzA3/L5T/JPv+U/7HF6fTe/ADZVI4G38HZ1Rf4Oy1eEsxAWWRr3Em/lY3owc8ASIZJzqrjodff5jov2x0m7q9UGB5sXdQEP4IbfovsPvpMPuZ8bVkEGaeS23wN+zvXEtzWDPF2VqEhQHkRb9Jn1ZBLDjUXTgkO9vmbWPzsg6y3x7ClukPUxapSGIqm1iRkyNPorE58G1XECf2bn+0ziBEWYfOqmPcb6MJrW9E7fAw9xYelxlW/SmhvSPIPJ5IbHXsBZ59e6aX4P/MvVSEf8ix5EsoiH7MjYR8xKDl6aIRJP5cS+rxrVSHfoFFV0mkoEx0/bP28+G9vxB4VkV6UbobJQjwyMRHiG+biUMbQEVuFTadDWN/EoG9Y3FIOu448gqvdCjJiNz7c3ntsddoNjZTev45tkr9LACeBI4Bxa6/TwILgK1SP+IOJ3Qr0mGDS07FN4RVHsVn39Z/uVkHBaGo/XYds49uZFZemTtQ0Ksv4f2EPuZcvQ19m41xm8Z53g87xGms/CGHM/roSGzz4y7wFhNkNSNPjiThsSBmb/LeOAwUp0rCPmEKHeGpHE9Z4nVs1c07SHn9FspGXoak0hHTdi0hZo/prr7XgfXF7Uw8MNH9WapG+S+9MJ2wMgl9dwtale//EbI73Oig9opfaZijZ/e1u5n84GRqHCcG7oqyjjIqOiuQZBuy1E6j9Ty9+iIAqkI+44FOFa/kXUFM4Q7sWhkE2NGnbNJ+vf1zLFM9wTyV5FkktQe3c35oB5aKJqb/GcrIwlcIMc92H/dvLiGk9jTixHF0hqUNknxxIAoih8cfpibezsbrV1LwoyLbYXCtM3ok2Bl7G+dmRPPiMy8iqZQNboFYTavNh0AVGL80knpkNRFlB0kpUhDJdnUn7aa99AqNyDI0pkykzy8Sg1lp0aSmh0itfwaNbHKPE3VBP/BHrsD46bGMTb6ToKpCGs4dJfXI9+ibvqU2aLVbdnfMkTEs+2EE5Y0+NITMJS3jNRao1f9t0mqOCpyzQe/vg2+3L+ENfvhE+jH2vrFowj0LJadNzZijY5DelajaW0XFzgp3YqIoo4hP7vwEVaqKITs/cHt/2tSKx0V1XDXHJtRhjPCnV9/AiZTLveoxY+cMAl/bhjU5i/bYXHfy4JEg+CC6nYik57EbIDA08P/o3dPr4aQNrFqo2FmBnC7w0+UeCdGsmrepur+Khyc+THN+M6qPVETXRSNPAMOohUwUYJogXuCpMFEQiYwdxuW1y5G67RdIcgFUJFZQmOtEjo90ya94iiCraEidQmfKKLL2fU1IXQ0mjT9pwcf5YP5IFslO5m+eT2htvRuxPLTqMy5v+IS9FdfSkfoUXeFp9BuVja3KlVRbFwlHRm3g0oJh6O+XCGxVNkQBg32pgKmbhzL2URU6q46/BzrPJtyOxdlDWsOzCnvAxUIC0PXruHjjHGJ/rOLi9VE4HNtJaL6LiXqIVENDw6Vk1DzLpJ+DmPnXTPeGp9mpSAe2BXZQntLrCqAKGDTdqENH4jv6YTDGIAgQUbqfic86iKyPREREGPI4t20qo8mhY93l6+gI8WXagTuYc7qF3O5L+XJREotit+O0aUguSyaoPQgQqAz7kK/NkFsNTdsEfD97myPT8zmffd4dRPRsfCTEVjMOlQOb1u61Ieo2dfPWg2/RkTESh9bH3VyPt0JCBVSEtPDnnP34vPEsDZOWub3LAHrsXRj6DBh6wRGtIaBvrvuYRYYD7WkcCmmkIakbU2slwbVnsDkH1g6iO6ClckiE1Oahq/IkkwRZxcjydTwcthNjfxJTz5/llZQD3OTqsic6p2H309CeO4uGBG9FAEEQCS87iPqNY6icKiSXUkGwCKNCTgNwdMtkEs634XStdWpDviVoQiIToi9H1NopSitCnDwau28QGpevaHLDY1i1ddy37wpWh26icKjyfv6dHS9Ynfh89xlRJXvwLiI2vR8Rz91GS/w0ciq+JrXhGff5I8p/ZPT559H2iuSee9h91o1NMLkWPr/tc/66yoJdY/d4WgkwRAdlrSb+8eLTZB4/Tl99B0gSDlUnn3XBquYEAP4svwFp1hzKFrr6syzS19eHLIvYdDbefeBdDlxyJ+dzFISxO9nnAooMMPtknMg4mZqwmuG6PqK+uhJTXjtFE6738pXQOkLRCqM5l+sHOTm0hFmQVJIbHJZe/wJTyn5l2saThBQ0EVc3HJM1GxknXUlQbNfy3TO3ceW3g8FQAhMMcKM/SDJUOeDXRd/wyS0fUhz5PLLgZKsrzpWlhboJ7Vz6/aVofbyDiQPzoUP2SMspVx/EHBy0ns2PvR+AqxvBVy2xPD4evX8spm4jeqsevUuS64YmOCs8Sn3qVIKrurnt09sI9lHuNyHgDJE+Ncjl8YTXlNES1kZ+dj6mbmW9I6ksHNTdw7VNLp8YrRb/vgnu+kxpeg7iXuLp7RPI+/lldL3ebOu2oBIKMwvpnujg9cdWs3bx5ZiNZ9z3FtAzjoiOywi6YjYVI5a47td7Wxpa2gmA1ain01lHZehHCIKTA/XXc/74Z6iEcIrSC6mM3ohD1UO7BC/11dNyOp273l+OJDhdIAtPO8qCA1EEc2gywjNP8d0NuynK8Kz/2ixRvBRpI3iuk56gwYG3gbFbotJxBF1VCzHn/6LB93UvthdAn9TFyeRlnEhZ6n62yRqICe1i1XWr6AsXiK2O9WLdx/QtxB58K3svngGSnaC2ILS5hn+b+X5Igv55bVTojmLo8+wddvTBYXMEdttv7H51B12hKewZdbX7+KTzR0GA1x99j71vVrHxYiXoO7T6Q8aWbMGhMlMd8hXVttNuVQNkEUN/nPsa57POc3ZsJeem3YWfYyndhrNUj47i6fG3MyOwgunb07j61TTmbrvMi601UEQZCtNLaQ9uBwQMajMTYtezOCKNWcWbiK2JRNU9ICHqeZZDqz/ksWsfoy80DsE1HiQ1KmOVStbj3zeSrOBh/3LNoLarCawVmfx5HBdtfQD/Pk+ws0evgDFlGeyiGYfYg4xEn15JcgiyQHRdNL6uIOiAp94Slyr60iZ44ZkXyF+iSBKLqN3voE404t/rR/qxrags0K/vd49BTlmNWnQwKW84d718C92BJn66/Cev5KpD1U2ZOR+LtglEP9LrPyDFpbLhqswF9zpQMuteIb7nCvqWXUt92hSvflFo+phG/TOo+sy0pzxAfNergxK0sMwEzwfDmclH2TtlrzvxtNgH8rL3cz6mipFhyZifuBR7zmhE1GgkP3SOUFSSEdEpknIgCav0G8dSL6Iu+DuvuunSE1DPnUlprEtOTLRwLvZe+nRKmzscEuHmheRUrCRAFcUfuQJ/5AocTZvHz9a72bdzHzX/rCG4TQH53PqHoiTQFdBF2iVppC9O56rptxPTeq3X71YmtRA0ORvrc9YLALYAizYuwrihC2eH3eXp52H2GSwGIn+JJP3wd8zZFMa0/HxSGh9XjrsZ6AN2Ha5kH2pklczpEadxhPljLCtibP7zSiKu4wqlzuHpFI9dgWTuwa+51P3+DoADjL3KX79W5fkkJ4kgSPwjEFZF7CRkahAFE27AYDFg6DPg1zecqI6lhPtUEKuSWL9sPf3GYGZ+Hk129csMJPEHJEQDZw4HILqy21Vn5fPSiNc4nnIJJ8ybLmin+NZbGV26kZj2Fe519IwZytxxlUlhvBqbRhJeG8xV31/Bsk3rMUtNFEc/y+nEa2lS5XFk9HEaUiZRMHYh/QbP/nLu6XYeuO1DSic8haaukstXhRHQEeA+rhI0BGni8DfrUP3wAxmF3oBlATXpRen4tuiQRZV7LRFinkVd0I/szxrDxsY30Vq7UTn6L0j2xVcp8ZCT8x5zy2APXmeVpXZiS8/Bpz+FSFMEiU33kahRxt7i+EpKMqzkjx5CXbxAhe2Y23uxxdHIa4+8xokJysAhSgZGlP+Ay3IcH2cbffoK0GjwKz1F+mFPn1meraiXfLywjD/u8VZsGCiyDIffOczRJR/y+CuPuxhlIlXJ/Tz33HNI0TLdd53Bv/GAF9D6sV2PsfHxjUSNjuLwuwqDUqvybpNjQ46xPnq9FyMMZKqHXsSsNxWQfO2RWj4f+TmBDR4pzub4XMrCh+FwSJxbe46t927FYfGstXc8soOvxn2FLMs05zej7eugLmg1pxOvIz87n+qsuaj1apauW8rZeQ8zuIzda6Hl/pfZ9sA2Nixa6VETAc7F3ck/n3yBfbMdVE3Oofzlct49/K77uNPgZM0VawgYH0DH9x1M7bua4B6FqTQYpDn89uEUZhZgEkEtOrA6fBBF0PW00XeqiH6zsqeYnjCdkTUK0E8SrSTPTqbmmsc5PfshJpacYM6pNsYUbyW97p/IMuTcNo7i8dfh1HordoXZP8Z/4UWoA13rsr8l+ypTtAQtGIuk1rqBfoL7/RU4Oe8xTGOiGHnCO5kHirJHfEUwznnD6RwWRVTbVRd8pyUikM54D4D1wYNXUB+4lrz2sez0b6Uop5Sa9BHsnyFSEvXhoDMFZFFm/YqjNKRMQq0eh2+/BxRv1dRT2XfW9U2BbsM597ExR8aQ+4OA48efvTwbB8beY2OOwfjhOMUBVRtl79sYuIH8eDhqTsW3x5czw84gympGOu5inhHeDIUwlUBMbQzdgf3UR9e7wawAjX6buHLtldQvq2f+5kH+mYNK3vV5aIZmMPKQa2yUNV7jhV3yJMD/vp4VZS0+qgCiOpa5bQnMPqcw9idQkVhBXZpMyMY/mL95PoKkxqc/DZMlm1TfEQw5cp6IsmNo7BoEWXCvgwRBILoC7hi1m72zylH/0k1wazAi3sm+Pp8+ugOMnEq5l68CovkmTyEgaFTeqiIOycHx0cdZt3wdKp9BigGCksCzmkLxv2YRnRGeMVZySKhtfRd4Kg9m9PpGXGjr8r+lSJJEeXk50mCZtf+L5T/Jvv+U//FFli9k9g1sEkb9GYvU00dzyqXkVHgjGBsC17ul7cCDnO4wHUASJVpCW0AQCWw3uRFPAD36QiIaI7j+6/lYMlKoTR1KZMcyRb7GJYfS69vLqXFa9i48QH7sKwT1eAIgatfgFtAmYm3rV/zB3EXRej844SD+HRaCBiEjQFlQGPuMqB1qGpInEmhVGE5+fcORkQmuOY1pzyZm/j6BUadm4it6ZFZyW99lZO1ubMWV5O6pxazbTnXo5/QalETOY1ojqjUXE1bsJLMg033etKJh3Fa5l2/6JKz6fk4/0svOGTuJ7LicwB5FFnFs9FjMk++naPK9OGRvqZII3zKWxW9iztnhDFk5BEuvS5LkBEwSNP+W39cEFXDywmMqWUddxky6Z11C1l+lDD813D1BetpMQF1VS/beT1Hb+rBqPOkJvbEf08xDnL7ahx2XeKj2D7fCT91RBHUGkHtyBHLThR4TImpFchCIqNdecDy1OBXtfjOOydMpT2nBofKWjZNEB/qkKDqihrgnvPS6F0lq/AcAy9Yq8olF6UXuc7ZHailOgEV/zmHobpkhez4myFf/byO7v9XAikl2jsWBLEuYJprIy8lDg7IQGvAsuve9e8k4vIFZP+9nyt4pHnk0VxDVJtroCYqnK0B538fo4fc4J8NS65EMIsEln2C1vIJN3eT1+7LgxG98FqWjLiek/za3j01F6Ifsm3yM47MuQ54znd+mz+VMgiIVtTLlCL8FvMPNfoofQ9Nf1Xz5y5eEdy7iL4vCdEwui8D/YC5Cm/dL0hG6kiuP3895/y9JtCei623n/ORbqRm6jEt6tmDqU1hQTtmOJIG6v5eA5mLMmm00+20hvf550uufxyiHuMeJgYSZrwCJqbW8+tgrMDEG37ZKzOrvOZ10DfEtdxDQMw6tTYuxT4MgS3QZT1J30U+EXTKUSSLM1XjL1s5AzWRRxDwTmAAaUcOQc0O4avUURI2aee/OIzAjBF9LJj6WdAS7iUOzDqF6TcXW+7ey8eaNSC7pHn1/LBktjzE8YjiSWnPB5rM6vprj4+oYes801l6T5wqgeUplQiV94+NB7f3c76+N4myTCb/iZCS7navuuZlv9f+eNOkqDVw5EaLqoghosTDylpFkLxtFdtUH7u+EdM2mcXUjOx7fQez4WDpf7aY0RWEz+ufkonsogD3TJB4JSmK5Lop7/cPIHzGBMQv20bxExesPv47klBi/R09cVZzX75eklXBmQj+y0YP4HvBxsGhr2Js1HMHpRG3rRXQqwd6mniTymqbziK+esUfHkn30POa3v0TX04bJmsWykAqWpa6iWTbQr24h+VQoqcWpbtTkANtM1zUdVSlM2J+Kb7cvMW3XklnzBmOKNzOx4BDhpksQZIGRJ0dewIgAj/cYQGiwp7Nb9VbWXLGGg4vL2D57O13+vciCk5mu393110WM2vJPOmJ1FGer8bVmoHIa+SgUyhP0xKZMpzwzkviqeHy6TRg1ZuguAns3O3eswMc+npjCv3DEhpLa+AEz/e8E4JMF6bwTXYsgCyQWNRDUWI3WEYIo+dJpDSdUhDT7g5Rc04xPrw9quwpZcNDhhEo7FCR8SWeEA/7G5BjYnMiiTP/CMay5cg2y4B28lUWZbr9uTk+Bz277mLwEZbPeLkGmFq4wAYLMQdsnHA6/mU6fYwSbZwAKQjG2JpbUQpHY+YvIbP6eaeeUIEOQCEN9m4hQKcGn8pGXseaGDtY1P++qmwLEiaqL4pI1dhqSJ1C7yPNcBFQE90wmWp1Dn66cPm0lNrvE9j54rR2+DLiOgx2P4DMvi78mDkIaA6fjbqY66F2Euh6i66Kxq5Q5ykeEIYHn6I64g/PDr0Nls3LDxyvc5y3wgaUh+3hr+AdUj7iHppHwS8Akt99ViHkmels0eosevzN6fMzK+BDWuZCg7in49eUQ2B5IyF6wpWXTGZ7B7uzB8moyskqNo6uXmIpm4puuJqLzYi9vIYfWB1QqegclPfQCvBsCV/iCXSfx1c0/s2faHkRJT7cEPqXwiNnOmaFnCGyqpfD292ny+YxuQz4CcEdkCU2Gq/h51VUIf+1wSwN6JJMEdz+RcXo8+9zMvgH5NOX55CXczKZcNfFRP9Jl16MuTMO3SYVTsOIQe9yB0sDeMYwr3sHQ6o9wWu2obQ6QwTRozEAQsfiG0JM9hqak8cjINAb9zJ8W2NsHpUlldPn3s+K7FWSezyRClcUeC1zd4sNF5uEMff4hxhzJpCmiSZEF9NvO+53KPDalFjQdKjZcs4HTP52mzy/PzW7xJPtcCcxBLBD30xq0nnWoFJZmhwQBkomMjkWEBk/kw3u/4fD4wwT0jGVm+z184BdDdEQJ9enT6A7SoOvXoVIr7XeueSqrz75CwMSRFI6/nuZFsUzZN4WcPEX6fYgWUi0P80D/88zfcSfytJkYje8T03odCWrYM+k5to57k4jGCJL/CiCk5hTOQTLrbkCPPDhAJWNXK2viEeXfc/HPWfSeKKI7JGngLPf508+U0TBjMe2LrmVo82kmma7HqqslqsSP789/wojtb+BTeZ61V6ylJq6GgDClTX6IgIcjbHQEmrGrnBe0Y5e6mGP71+NX9ztSRxcms971HWUfUN+dRmfY9Ux98jIcOn/mb55PUFvQoMCN8jexIpH44iIefOdB92dqu8DC3xeStsbAve/dS1pRGiHmmaRqoDQBTqXaqUysJGBfB8t+Wua1ng2x5RLfehthrdnk7DxOTl4O/gHhyLNh7v8b5vsiNTx89SimljWw6NvJ3PKFR4rzzhbIk1rxj3iMvtJuWqOHuFkxKuD65HXMN0K/zoKguXCxWRv8DWcSbuaM9Q8Gj+kDSiQAGy/eyMF557D6hjA8V01BzKM0OGBdN9Q4IL1YUYFIbLqV2LYbvK4fokpizU0W9s61E9o1H39BYlrC9zw+aRm6P4cwY9oWGlKnEr3DrjB2/uadGj0mGmNLNUlFyueDvX0Gyr8CC/r2+DLzh2AMVVZSy3OJ6Lx40FGXioIMe7OHsnWkiS6jZ/3p0Dh49fFXqR3hj29rudt//eceWNoAx6wgSiLT/mln6Xol4Tvg332iexOCWoXVNxiLjys4Kyl17rRGUNWbiTaqi9LRtViNA++H554Hxr+CIbUUTr4XH+FS91pbOe69xhiYHwGC6s4S++fX2I0aev1MMCiZ160pwypvIObMaQzdLRe01yU+UF8RSYfYz7kh59yJ6sU+EKR20KC9kvQfbyFwWy0XAthFBFkgbX8KMdVqVz293zWnQ0Lq7UN0yWXa1e1Uhn+AVaOwUuySEz/LUGLbrsf0N293ACFGIPKhSOqiL2Sx2J12bD02bkt9lIy6V7yOOVTd6ONC0Y7XopMiLzg3vSgdW2EjAasz2LJgi3scEGQVnQGdnHntDObgBHoHsaoGjoNHkWOAiS7Iavc77Az2oeHWZ6hLn+F1rs3gjzk0Gftf+4jN3+p+dwZkPPdPO8Lxi55h9yXKe+cUbHT6HGWea234g+lWaoNXc/tntzP89HA6fA9SGfoJb8wdxbsByh6nKSYM+6RpdPmcpyxC8RLt1zZwKuFaDqRsR7r2OkqzBoCu3n3LJnnHZP5eFOVlmWrnETp9jmIUIRQji1bfzKjV3fi2VuM7aMvc5rcLWZTZNv9P6jJmcnTkarp9K73bU1Ck4XwO/YVPr5ope6cMPoogOxGddmzjxrn37p5nocbsZ6Z+RCdVwxaic4QhSGoM/fFYtbVMODAB31o9X92+ix8uuY52X28G6NaFpchvXoek1oEgUB7+Dt1GT/ymYGgzotlMUO0Z5s6F+JbbedmQTrbGyI8jj7J/RhfH0haza0g6GzqeGVxtLEYLiUXt3Pvevfh3BSEj8XQbZFTCm4eTuOODGch25TmfGL6TRn8FjPFt3it8ZY3kfH0F+dvz8S29kTOh6ZgNgxg3MkSOUt5pjUODLMiuec7leRcKHZfY/2WfkWSJS765hDvPKfuCwck+rUqLPknp/5dqLmVynJIUG1jLDcS0g5KDmPHPGfQGRF14fQkmPDSB6/dcj87PMw8Ou3YYs99QgMKrpqwiIW+jWypx/+iXaU4ah8agSH3aRO8xpN80Eu3oYVjaLPQ2dRPSrfQrvS2aHn0xTrWTNr+D9KqreWX/K3x6/FPPo9AIFGUUEX1XNA/WPYgmy6NcJMo65hw7wy1/bKT9RDsSTs7ZYM7PFazLf5Jveq7GanmStg9/ZPdzu9n38z5ON55GIynAvAEpWEmtVVS3BAGtM4gw81wEBJxOsLb3oetpQxikJJVa/yxh5vlo+rS0rvtLSfy7EnpGaxJjijfTFXMpYcumgixjV3UC4BR72Zs1nH1ZY5HUOpp/LWDqnqnoVUZiWz3zb1B7EHO2DsGgCmFc4OeMqFjNsEAPAB3g+PQshj//CuEdl7g/C9F1IQoSq80Ka6kzRMOOKc9QFPOspz1dY0ZjdAd1GTOxGQO8rlsc+QIfVHnWKX6DwDaBHYH4NwjIZ8/hsK6mIPpRZGQvcIqzvpkRJ1Lw6/JzJ/sEWSRRA2VnUtDZdJwZdgZBVjNb9RK3X3ESyRjPT5EOLtp0Eam7lGupB4HWBVmLLMj41fsx9ui/jl7anXYMcyaze55BYYjKavdcJkhqgm62kvvH8zSo5rFteJDXuVVhn3C4Q/FaTG581HMvooWDEw9yYo4DdU8Pvj2+CLKGsSWbmZp/jlS/HEJq87AZTHxw7wfUR9fDIGYfQGtoK5lnw9Ct72XF6hVezD5/TTDJlmTCa9rQW5W+P3CeRnSN6U5lTB/s2Te4CIIio6u1dKKfMII+f89caWnsYvj2N4j6m2wpwPlJtzDvmyuQnP/fSXT9p8C/GUL+T/lP+f9ekeULN2r9GiXJYA7sQ5sci8maiY8tk7xEzyQmC4pufK+mgrPxd9FlPO4+ZtfaWXXDKp7YtJY7P7meNx56g15fjy9bt6mbI2PzyQ69jmHVygbwVII3GlAWnG4Eh4CaEBWUxsN56RCHT89gzu9BlDlL8L3Dlz4fBX0i40RApF/Xz+GZSfg6lMnkbNxdyDiJ7LyUHt8efrpuPYY+NeGaZGoAs/E0raa/SGqqx9BYgfr6q1FX+DPSR8u3bb44VT2IgojKbsWeX0xiuQ9qhxpZbcSOEli5oreXwJtW08IKdsZ6/Kz6NQ3Y5F5MZhPhzYFYMm1027sJ61W5F26iICIIEFW0m6x98/n45gp3giAp8BQPZ3+Kz97FtByMo+FkA5nnM6k5W8DN0oXG3/+q3OaAgnw/Wqd6ZMZESUt52LvUZK1kVNcKpm3pIzIkkuoUE50+R90LnlU3rOIf+Scw53XjVOvIqfyKQ+lTsKs7yaqCbglSYlVUmirBDr6WTAREfmYJpWH52C8vZ0l1IlM117An1oOaE2Q11fHV5H9Uz6mTF8p4jj42GlNFB/0PQXHUC17H/HtHM+L8/XTrKzgX+w5WTT0RXYvR22IJ7p5KecRbnM8uJ9d0kZcc4e7Kq7k+fSWZi9vZU38ZPS07cMQ4OfC7EtD57xKnR4CdTvhmOqxaM40QyYr/m/5s1m8mQZjouicVsiDTr+tHpTVSkxRCQ2QDoqwslAY2cnaVDXNoEr7dSt0W6kzIUg9HujTMU9lILGqm2f8IPQlaRpX+Qm3ISnza8ln0RzLW29rojMz0qltF+Lv06csw2ecgq5z06T1ylfMDFamE1LWXYO3U0L/RwM4TO4ls90gIGa1GFm5aSEt0N2V+SZQnleNrzeKn5PNMSnyeo+EOZmyERYqkOnomEmcDo/AJ3cYzbrZMd0gix2dMY9ukaWgdYczJU8aRwYyJNt+9AGzshasjJRaG9CMFajlwyVXsHKboy0e1L+OJkF5+m36AnzJauLOwiJG71lJwdRmFQ1qZkglTCuDxnSp6en3x69eQzFBmPKNlGwp7VS1qqI6rZs/Us1wXrWwItIKBafmK59F1Y50k3nKWoUOHUpZahqPfwZZqRSYxs+xqRhddwvAH03h7ikeGiUFIUxlJCbIO0vcXJR16WwwFWQVMHppG0Be/0Ol3kgNzDxPXcgs7+oaxu/0rlvx2CdbgJm5+8BZefP75f+vd2+2El6f4oH7peuTONiqtlfR22BgqxJLvIgmLspbCX07ScKqBWa/MoifNQo9VSbQ7VL2oTBqYCjNG/Ib1eBi6vg7MIUnIggaH1kmfsQ9Hi43080aKkkMwxo6hUFzvdc+Olet5ouwJXn7yZRJa7nR7OQzJi8RUeYb9i6ZQGfoeQlMabb2P8vOZpai77+cb/y/o7u3FWCPS7/cpQ9V/sL96Ofn9IltKb2PmpoVo7amEpUD+BGWRPKYGhjTfhyrsXUY9shde34Vvjy+iZCC5ySPhImbYYM8ZwpvCaUy+MNnncDpxiD2AzEWTI9h21oRD1Y1T7aQoowibFEeFeFBhFDYLfN1uYndLAuHaIPpjUxCHTSXMEQNdENw9gzsz/gCs3OV3J3+WdhFfYMZumkRFyGgwRMHexcwACqXpVBrtGJdeg1+xiQQ90HKAaL9ipNZQksuSKRoRgyVEQTi2W6J498hKPpg/nF7LC3Scnk37sXxskUs4mXoX4/WwLxYe8d/M4cpx3Lz2H9QYkmlJUN4eQdYoiWxUWMelUOYoQ+X08fYbkgSFLdjXgjnqNGoXiy1EBR+FgVQfhvzTXITg8/QO/50e/SgGApaioKIltIVjE+xcEh8FteDjkle+NwCeTf6c9Z9cwtmDwzg1F0riPJIy4Z0XE9gznkuNC2jkDP7NZXzW43mGA3U82beBvdk3E965CMu2hzmUAVv6YNb+CUw6EMqme++mL8hbVhzgz9k/kSylcfUPV/P6w6/T59NHjQPO253kNq/iyhkVrC+5kRajx6v39Q64zg+maE+xtWYmQRu+IXZ2P00uS1OHyoyASEhrCJlfRZAJnHkO/CxDyah7mfMxDyH0/Er0Ng19s5Lo1KfQo/cgrMsj3qJPU8n4A0uJKSyiNXYE1WFr6dUPJKBUdIckYnzlKVr3eu5luS/cFwia4/E0VAmUZgzIbCoBm7sD4N4EK5N9f6HpQAzGrgIag00IiKRq4Zk2MAanouvYgVhmIMZXDRNgcEBbbddw2fpLCerdgUNeil/fcLekklvy0i3jqaxTXii6lLSuKylfMILRQy1odz7O2bSNaHW3kuLaWGstXcSf+QO2VnKdI5HXHjEgMCg5oFaTP+0uVDEnaTLvwGRRkqNLGyBXp6Hvoh00nA1g8c+pFKcVM0JzBasfU9gY4z4bR1twGzatzcUQEdxJu9CuubT4byNmfwhn/jxDt6GbnWnXobdFM+tMrXvuHZDxHJAZE2QVNlU7R1Pnc1QvcY+seLkkJjuo6VLu/Ib9LzNmZRvCsN+QF3tknJJb7sYgb6WYXyiMriE4OpUjke+TqRroz5H4+h2hwFlCReIVhJ45AIxwB0rfC4Wp9h2kGAr4svwWHAktYExDpVIS1QAWp4aCzAIW/bGY4Jo8HCoPcCqxPJ5Lf1mAeYlIrBhCY3gnHT5KysrUNwwfWxK6nt/oaO1jz7QH8LVmEdA7mnbTfjJqX8bHlkR/HAh6oNYTlAhW9zMm4XHOZ0XQFeQJpvgRQ6oGrjQBoyu4y/gby3+5lbSTN/PnxYMYkoKdIxu/Y+SpkXDqJDczm6Oj/WlNVcbmK4c+h1MYQp82B9+ug2SfG4tsugnBMJCkUG7+6JijpFVdTX3wb+7xQXQI5J7IxRLkpEtrwynK+PSlMtI+HDhNnms6bpysIa/3L0Q5AIBoUxFXZr/BJztfRtfZQOnQaIrSizCqwhgyMYDrQzqZ/COME5X1cjiKl9/naoGDThlhBowY/wfivSsR1Ab2jj3A4PJebBW/BAi8M2E3Qb97gs2JXQu5KfdN5C44WxzOkJ9C6E4Yw9GxR7GrOtE4A7ySFlktzxLTeIerX8gY+5OpDH8fgJy+a2jRf0Ox7A/6TgTgcpPCdn3swbe5qdFGQ4O3rNUc42Nc5v8KezcWE9V2jvL4aSTHn+fO0XdS0DKezNBDDA3ZwS5hArKgSJiHh4kwyP5FH6CnNW08jVEK20iUlKC0XdVFVejHvH9SdCf8h6ov4azjV+XdNVg4Ob2D8ekPUFrjDR5q89vNwfSppDo/d5+rVl0YMgk/XUdy3ne884DC4Ci0Q5cEYSoIKo2nN0imJrYGQfYkp052b6aYQoKSEkg/HUte5KX49Ke4j796Io9PPhYpzVLx+zvrmLdlHnunFw/6VeX9G1CYuLAMSgwikNrwBG1+OzH0x+OgFnVfBzvU93By1K8ML/dm15WmlFIyKg7f9nzsKSamZmVy3tWlP+oCaeNYLjs6nMInzrtZZk+3wTf1Y3l68nfoKl9DLitnd/YqyjOL6PJR9t2ZNa/jVDnpM1nc4MmBAGmT/yaqQj8m7d2x+OXLjMkJomaQUIpTpeyfnU7JnURQ/wu/LyFUoPnrZuaY5rDqhlXuz1OLU3nT+CayUybpsuEIskflJb73MlKK/oEq04fbVt7GuWcep76/hN1DPYH+t/7xFj9mO6l1/Oyq9yAfL7WT3qheisdfd2F93J5+rnW54BnT1Q4Nd314H4aMk3QNi8GibUcl+aB1BlEY9RQNQetIanqQ8QtmU3tORuv4GaM1hYDe0SQ1/YOoMB12QSDOfAV6cw5DJjjZE3wFr3TARD2c6Cskw9hA6QwZi38qjcHf0hi4gde7ISlsApftvxR9j56e2aNobv+UumDFrymidzZ1Id9xuL2TafEPoiocR0bta24FhQHAjUNS+kRDwC9Uh35OiHmW15pXids4eLd3HITBhh64089B2PBuGi0zKI6YTI/QcEGbyUjIOKkIf9fr8y7DaZqL/AkvK3F/pnhgK8UuW9jU/AjjjvnRP2oUTRHeQFRBVlEfXc+p9CrCd8eRUfcykuCgPOINREnL1Z8/SltmH9aLjtOnL8dk9YCisqrfJan5Puzak5T630NXkAHzoEQfgNoO+uIz9A89ydm+Rnytc/j5zBtIKd8C6/ExC4zZM5q8YQ7UI7wBy2FNYegtNvp1/QiylqiOy4k4vpimgD+o9buP0tRGUjqC2bGohP2j1pFToczfW8tuoy59Jafu/oiVPivJeNhK3rDDDE7gP93nzzPHbYxbeordQwb25SI6i0BCRQJyBrQssVBbUYt/b667vwJE7IygNqiW5fXLUYtqTjZ4wA79T/Vju9+GudaMb4QvRdYihn82HJAJKz/M3uf6mPvaDAKTApn8xGReP+S535jCv9B01ON0Xk1wajDBqR65fYCMiz2MoUmPT+Lz73287mmgtBa1oTd71hv+vbmYYy7CZzYsuRcqK+H3h5Rx0yH24NDWEVMTg1X3G/7lsxneOBz7HFdssfUokYKdM4CgE+g2ddNEE3YxDI3kh4CIvzmE6OO/sHLCSVZMvJPtoxpYd9FoTjQsYEH/rwSOMDEuYzWH3zmA9YCVVxe8SlTAJYBioXLiixNEr91D/5DFmMM84/35mH9Q01mB9fLpDM1rp3j0Fe5jiU33ENBYSPeb25DaOjEly27gVEDvWMLM84ku2EHp/t/xzb2eEPNsJhYcxqqp50TKpYiSDmNXA8HLM3mn/mXUgprQrvnUhKxEdBroCbGwedFZlmbehWRX5gvt3xTDEJRxd7DkeLimj6sSPiP/j+GM/2M2xaOUvuwlJ+1aJ8koYOj+mD+RmjPcoJq/s94GbB46fQ6xbd422qbn0KsqoJjNONVOMupedZ+zYNMChGPfMZdpdPvr3epEoqxlWi2k5nRQFRZGn7EPH7MaP20AoaEjIGo+gfZztMfOIunccXKP5VKS5ZlHRUmLLMqcWX4GVf2/9lcLPx6OrbWcpmgP6GMAJC6LDtRqDTZsSPS5gW2DS2HvAeoDLVi1Ne7PBqxOZCQaVlzGdu1qL6WBNbUvc+BGGzGtnrlrsIynMwWQBW4xKDHpzoBOL2bfR6MOcPqmD2gvOYPdEUxl1KBkn4vZZ3eN6UPChsBfEFsTi/OKQYBXAWIKdhDQWIC0fBBgARB0WprjRtETGMPfi39zMVuv20N8zm1E5Py7xjD/Kf+fLP9h9v2n/I8vkgSh5rlMOFBBtNMbcXJubDWG3GyMXfV/8yzwmETbVZ20+G/BIfbg1zvC6zuOlFj2TTzo9igZKGZ/MwcmnUZdU45/YxGtpt3UhXg2RVF1UVzxpYUZd47At9sXQVaTrAF/FYy3HmDWkRWonUrgIDvfs2iUBQeCrGLxxsWE1ndhNSkbwJqQr6gO+wxJ6MeutdNibKZDt5I833+6zz2SPpuvV9zJwZvCICUFqykUk8afyI7LUDtNqNERWbIHx9lCvruhkG6/bi/T2u0OO2tjSzEHeyeuZgZVcGvEQhYVDGXF6nmYykS0/VpXYkiZBA/XHqaj63N82s7SE9CLJEqIsprMTAUdDjBxxWb+ePcPyjeXs3zdcgSr8H/k96Wx6QcaSXl+oo2SqBfoNp6hw9TIzjvS2TpvK4KsZljllwyr/JIc2+0kNzyCIyGNtRc/wJ6hE9Dbophzuo3s6vdp7zKy+KPbiDlsdVPUxxZvY9eIe/h+oj93R/1CdvXzCLsruKz/bi95IdnpQtobnajEJAIHsTcDuyeyY9YO2m4LQPvDKoaf8L7T+JbbGH4Mql74lsu/8aHd53dOJ17H4fQZ2NRt5FSsoj3pWvoWeSNBV5VexbelP3DG8SCET+X0mD5KR5b+W55mc1RgmBXBVY7ryNs3nMAzDuwuSr3KHcwUkFQSn93+GSUjp5M3PpbCzEIPs08eoPPb8W8uJaFUWWDHE0JLhy9XvvAE+h2NfH9DObum70JAJLJzCWFdC5AFGVmUkWXoNB6n1bSLfnWz61dVaPu1mDo6kD/5hjFHPNJq6hKIqYqnqjqVwA5f94JDkFXEqeGDUPgusYFDN3+Dxurk2u+uBQF6DOfZ0gc9TjWBfZvd19NYu1HZrV73I6Ew+2IKdjBq527XMTVWTQPd+vNY6XQn+wbYcVUO+KFVg7kqEqnTglO0uH9D1Jdza/oqPgwJIqfqK0S1Clnlg+Tyfmk1QNZsuFJzG7eo/sEtfg8zhakE/Oh5vxbqGhiTVs/xsUW05jfz48U/cmjfPjqNiuRqT4+A3sWqS1uYRtZlWW4JgJAWFVF5W2k+560pPxAgzsrP4pK1mZSsOc7QU2HoLXr8e3PJqH2VoVWfMrrkd4LlDFS9ZnDU02E6QFT7lQwnDmtCDRsXbQSjhvUL1jNr3EX/1ruXPXES2cEGAkaH0pQ0ifrj9TSfrCOmpNz93brg1ay6eBWG7w3Ikky/zT5oD+ckSTueK8quIPhoGaE1J0k9/iNqm9Luhl4doc2haIaa+ObmCvJy8gjGIwmSczqHKRt0yP4+XmzZgZJQEUFA2Un6tBXUhH7NcfMmKsre4pVv7uXAHype7KrjR0cnL0vtpJ0/ye6/Yvjz+G5uyvyBZxJ6qItp4ejscv6a+RcCApPCL+K8XUtTxeM4HJBx9Sh+uroMhz4VjTPQ67dltQbjG/dhif+U9LoX0du8+72EgwOZ49g60o9uZwvDKr+84DgofaLD5yD1Uj9L/CxkLxnH59fdSX+oZ4EtCw5m1MJO2xzM9Vr82ttQzZ9NVO9LhBkroeO08tyCF/P5xF18d3Mhsu8gmeIKZZ57pT6GiQcmkni+gdKYbzkbdwftvvvpsESyLv9xChrHETo6k8Lx16N3DiWx6UGWuACSPgLgkBCrqjH0dLnnZo3Tn+n5hUzLz8e4Zj8Pv/4wAgIXzwkku/o94lpuRdev454P72HcL+VMODDBHVh7I0Txx/u5cwHpxUmkHQph8r7JyIKdNj9FxlYtqukI6qDbT6Jr1wlEWw8DiaBDLtxJYWgjtcld6HrbiWjwjNvhXQvQOgNJuXgyZSOXUps5E78uj6wuiNQGfc/uHsW0vSngd46kzcEkgjUZFo+ooGS8mQnfhjJt17QL3j+A1ux+dszc4bXeiFSBSrIwOWQn+6adJe6Gh4htuRFQWJJvNI6m1r6YqOLdiDbJ7S9s6htKp88x+tXNtAcpzF2b9m8MOVlFbUwtZ+7rwZ6coSSeBO+gSY92G/ajRZgDw6gP2UmHr6enH8icwJYRvmzrf9YL7HS7S7Ly3I/zGLfVyCVbXmXevh/wtaahBtI0oBV1RLatoDHOyJ9z/sSuFYhuu4YlvvBCMDzG87z11ltY/CDsrBaNIwi15IPoCuLeYvyNjKIMwmrLSKgIIaHlTgL6FHlcjSOIRMcCMnRKOw+M2e12P+L9z5IYcBosVkYcCSCpPMlLgkktOPBtr0L286MgqwOH2uFOHp5OuI5fMn2oC/iYmt43yI+6jGb/Le5zv4/o5ccI6AzRU375UwRpdg6o90FvDT8mhxG4IoTIxngefPtBFLaBhscC4Uj2IS6XxhLsVNCw0dcpY8DffQgH1ileMp6CRKfvUZrVxy9giRyI8Oe1od/TEp2BMziQ7LOphDWFIcga6rvTsDj15FhOMmlrI1HFPa5rKu3hp2vlkvhfae76mn7HJnKPKevj+qh6pY6u3EZUYj3nZtyFLIOmuwSLKZ98m8JWzN3zKlaDlVNXNVKTPY+Ijkvd7RVhvgmrKZlh9nFc/f3FpBWnudeksmDDIfZwas7NSG29XP/ldHp1JW5JJZ3d5dtVX45pz+/oetpQCxpE4PagXq5PeRfD7Y/RGFrD/M3zCW8MJ4ulaASl39zdDAgSMvIgNoNSRODghIMUDPHM6YPbJcq3hOa2w2x54A1kuZ/Ts/6BRnvxBcy+Xt9ezo0byu+Lf3cfM9mmseaGNhoeSuDTOz6lPKUcjdMfx6n93Pp7MYvzgnnuueeIr00mUtrP2DKFpXH98EeYGvMV6fr3STr9KxafQKx+KSQbR9EuO9FOhBX/TGHfZLjPL4jLdXruDjBydkYIzgfBMQFUBj09gbG0Loxj9/Td7vt6PgiOtg3jpQ6Z4HOBZBz8mtjqWAAq/P/gbL9yRzP+mkZggc7t3VMU9bTX/UqyRFD/SMLM8zDYY0GQ0DnCAJj15yzGvdHB/PWVHOp6CgSJZicsb4DVZpBUA7oESgnrvAiAcHUqoghhlcfwKT+L2mah1RLLj+ee4qtTb3Fq8iGuLg6iJH4TZ6+BHlMPRoOK995TrpMf+wCLnlqElXO0hSiDvChryQ7dy/jYHymMeYI3jj1DfMvtTCw4xGTtfe46ODRqSkb140wMxa7uwvk3ebR2014sUpc7Yb9gjsadSBzV/jrTDz4LPjFsWrAJm9bp3rt8Hg75CRqu/uF6+gIljow74l6LAlRbz7HL8h4WdSE+Vn8yij8m1DxHefdMxVya+Ci0HUWWIaShhVEnRuGVwHMHUZ2YDXlUh3zlxUQKNc8lTuWREpcFiSwtrIyvovXKBXx7ywEWfzWcJb8s4e+Sn1aDFcneQGRFPntCZpKoGcek80dJrX+KV9ph3TAlGbB0/VJElyd3gxMK+/UIooqSMYoMnKaqxCtx0KsvBQFKx5ZRnKH4Rapk5XyLtormgM20uIB/zeHeSithnQuYfbqRuZHX0K3Pp8l/Ey2SBzAoSAOyzk5yrszh0usu9TrfYrAQMSYCU7QJZ4ZaqYurxFgWEN4cRsXr6yn/s1wB/g0CF0ephuFnGUGLpZLQfcHMOvSd25MtouMyts+t5PNJnyM6+imKepbd2VlUhXzmanhPsBtg/kkrFx13oLdHIyNgMViQga7WdRyPH8qJZEWJwqZpJrXAh0u+aUKWoSs8lZm6x3hEX0J6/YuEmmcT41RYbSZrNlEdS0kxKHuq77sVJu+EvVO4ZG0Ik2Y+R4zlGzQOJaEyXAdTQtJIP+VLVOlx3vMxURCrSCEuSltEjno5ACk/RSO+9y7+PZmkND5CVIcCvhx4jweAzn26clr8t2E2nKE04jU2j9RxNu4OF7PPE2uwyFDVr0MYEkZf7mQKw5/nbNyd/L0s+2kJoza9zJMvPUlgu2cdvS97BI9+fxkxxYfoHj2dNddVcSbnjNv/1SHb2Cl9yM7pO3EkpJBb8pvXdQfm16yfo4jN34qMk8owZRCRBAdf3fQVZbNbyciPIrwxnMSm+4hqV9pCQFTmn0/+5Irvc+n0OXxBvCn9fDA1Gf2sv/gViqx7OZQ2k9XBT3B5YB7PH7+e+b/Hk52fSFhz2AX2K7d/ejvGXomvb/8Gq4+KwujHqMqczW3ZbzE8oYVdM/KhtY2w+gEGktJnU4OO8oaxhEl68LvIn1ee+ISCzAIGjxUOLFgdVi8ZY0FWEVGv5fpvrkc45UnchnbNdfs1A6T9mMbeF/dSWVzJyYaTjIxU5gUfjTJHa321hGSEoA/QuxMHCDKBDec5++0p93WKB+MUUNhBvr0tSBJITgnJKf2XXlYTHppAW0yO+9/jD44na8+ndDd088ftf5Bx6Ef3Ma1DeRcGAAEdHQoxIKdiFZMLlLHomu+uYe62uUSdqVWsD0QVyBJsH8drvmqeefUZah+t5cEvH+Slumzqg9YotyVAv08wlbNvRh+gJ1OYxYSqr/HRmJFlEVFQKQnmMQms2L6Czis6lTpJChjJoerG0m5B3ddN2tHv2TUkjSOpis1Ai98O8mwbsLsk3B2DhKSa/Tdj1mxH6u4h5KbFNCWNdwN4B8B1To0BfXI0Do0BrTOQwN6xBPQqiEBZcJJ+cCXtm0ppiGpAJaiJ7FjK5PMnmHu6nW/GtRFtWUnfh1+jPrANSbC55fFfmfkKDxvzCeyZgCThJTleZU7lz9orGRvhhxyiI/nMSR5+/WE0tsE+5665RZDo8DnEUb/HqAv6ftDT9cw9IeZZqCUfsmpfJ6BXURSTkehRt+JUezy93TLDHYHIPgYKx19HcucvGOwDe2qRPRb4zRFA0pEuFmxegIAatRpw2mDk25RE7KUveAyWUD3BjlFesT1fq5Jo1i3TsW7vOhZovVngAHEH47Bu3uWOjwqo3Im5oO5JTP3jIY4vfJbz2SUXnAtQ3HuEk8nLqQnxVqMbfXQ0I/5U4+OMYXLBMSYWeeb0n2rfpjDuZdTWckYdH4VPj48iDS5AalAqogAbP7+YuMoggtbM4uCcBLKEpe5kn9MJo+9U3on6GGUPOdBnBxi7A/HCNZetIachh7FHxyLZPKs2QYC26CE0JY6j7VmPfQyA2t+H6mELLwD6Awiuvu0T5nPBsf9NRf9vqmT9P1H+k+z7T/kfXyQJVOgI0iUwy/aR1zFVv0DnNxvxbdxBQ8DPXsdkweFauCsTl84RSlDPJEBhD0zdPRUcEobAbcw+ayW8Y7HX+WqHgP7oPkytZ7FqPAiMTN8JpLPC67uVYR/SP2itEjxTpDJXWbhrB01+suAkufFxRraPIrhFM+jzgWCu8t1xh8dxzepr0Nu9B8de314sfoJHmlSE4ZWrmHT+BNXG38jL3IB6ynhsWiWIMTjZtzES9sSAT6OD+EqPH989ATDe4GTBCCunc3sY8rU/F21bQov/Vrp8FBT5HyV/4H9uH0ZzO60v9ROkmozGEYRGA+b+UG7efIKqpK2su3YdmcszaX6wGYPR9H/k96XTK/eqcQa4P7erO1j+43IWvWtCFh3Ioowgq4lpX0Fc603MsX/CxCNXoztfjtnnFF0+J0CQERCJb76DbdGQ5tSRstvGC2tfYnTJRrSOEEb6f4XqzMPE2n5ArXMQ+tStGMfnoHJ6gt7BPVOZsf9Zhj+SxJgTS5hYeMC9qYzoXIK/83YiMnIRWlrw6fHWWBdkFS3xuWjCA2kL7kQeFGBt8v+DEPNMYhtn4PPGWa/EV4/YxYmuJZR1KAuHAflaeQKYZ8IkQWCmmn/haQYBc+Gu+TLjTblsWWLHfP0oWj5pYf7m+Vj7PPJoAxO8LDgvkApzJ/tkOxZTKOdGKBP97YULGFY8nNM5p7EnGOn1tWM1WD1ySrJIRVIFa1cUYqlsYtJvv1ISvIw20253e0zdM5Xx248j1DW5fC+U4gSmhdxIz+zXqRr5IPIpOwEdAdQH/8hovcISmeRrwS+jgpLcGHSP6IhvvguA9zuVjVqo+SH2btpLUN1Zcna8TXjBx5zTfIVd3ea+H1mGnsAYWiPD0NqUZPbJpCvZMySbKvV2j+yQK8garoKt5aFM+OA25N01RFeJpFYo1MFKv02sLrmGrafuILjmNNqR2RybPcstl9nRBU+uTeXb3jV8ZX+H1f4r2Z6zjZOpHmTk7cZCHg50GZrXdlGypYS7N1zL/qxcHGI3IHLuuXP8U/tP90ZoIFxWmdBC3rRZGDIMhJcfwtRa6XqmyvG4xksJbguheut5pu5Kx6fXh7T6Z0lqvp/Q7llk5sehW/MnnRddzdrlawE4G38Lq2ddTOGYNrrHnYQAPeZGM53GVuXd+xfSpNO0KiYKyrsZlbWEn7qOsEt1Pw1pU8m4NAvT+GD2zXKxgyU9kmBjV8cuKoQKzq8/T/IV0WTnZxPaNZf0upe4VvMbs0pn4X+qjLbooRSNmkyPsQ1JsDH+yHTu+vgufDt8iDbfw9SOtQyTruWi4xLJDY/hZ/YjuEmNY+IQfl6qzAVO0UJi0/0A/L54B6WzbiOmvI2gtiA699ZTX/Q4+2Un2/qtXA3MBK4G9jpl9stOmkuf4PMjvhSllLL+8t2cG1dLY2QjIPD+2N8pXbSRFydeQ5T4HQWbT5LW9BHjyvZisioLXqumnrLwN2mo/QDn74fQW/1IivFlSv5Zt7wkKIbYspud5mEDaWwaHnvlMVa8MolbP7uV0JZQZEFGo7JxZUg1S32VZx62/mPi85Sgcbc+n10W2OC4mvYOFYFNRchaHZJai8XhD2n3wtyjnA5eQJ4N4ip96N3yDc2Gr6i0HYf0e3lx/1Yyi7dx6zuP0pw9m271DqrCPkU0HWNO8pccb5rMkVaZopfW499cAoJAZOcSGvJXsa45jd1H04k/LmN/4BFqhvxr3wNNfjU+fT6ktz6OXuVDYvO9DKv6jFhpIXsn70VnUTPnzzkEmNSMKFvD5l74uCWW4dM+4/hFT3NkYRknR57EKXoY+aLLaDy8QaBn+xH2ZsZyImk5Ma3X0XJ2LcP+/Iyncw6zc1kpsQXbuWHlDRfUyxAbQnThX6Qd/YGxRzx8Vr0thuLop6l2ePqwJFq53gQ6ETrSyjg3tQNBFrzGt4ES3hiOukvixKgTqGUlkaEC7nappX3w3A3MWy8SJ45zB35NIuzuieX7xlP4tVbQMS6Ioowi/F0b+bLIV+kxnKfPp489HxZxZo7i+5NWrzDNBVT06/vpC3bgs/FHwsoPXlAvm9aGMCGF4mwVR9MXUxXmWWOJkg6nqpcGOY+6kO+IbF9KRu1rvHH2fuaXZfLXtF2cG2ch81QFSaV+aB0hBKjgJn8wG5YwYv0y0s6HuOoiEsck1nfDh31KokGtdXDkFpmw2x9l7uk20hqexdfXF0EQSDHMZcXBO6jOmktPkDfrxrc/nSV9m7ghWGE6DwRTE/TtPDj+OobY1yP1dLPhmmoOjzvMwFan3fcAG8al8P6DG5Duuofds6uwa+2ILjCMJNhwin0MOdbF4u8yGHp2qDuJ81YImGUDX22YwJgDQagkI+oBr4+K1fBbHAnNvzNR709DjMDp4acRZAFR1jDLCIkGM+9EtTH19kd5tONRtNHeSb6gnilktjzFhDClv8iDWCAMYr4OsEQGkn0b6sfTZSzkwN1XUJ5SxSW/XURacZqbKVjaMZTzFhWhLSFElPUx8oRHSjgrdD81lgBO7x7JJeuUwOqmBZuUuR1Y1gCb4p/gnuZt9Ih70OzaQb3hGprVniA+sojarqY9vo/O8DR3sATA5pdJ2dhbSFo+lc3zd1AdV+1ek/YYCtk60sSuoak444wK+wmREeXfM/t0E06xj4LoR4nY/h0+BSc5H3k1Z6yb0ArwbDAU2uDLLjB0Whl7dKwS/EHivA0SK+HHsmAm7B9BZWYEf91aiNGWREbty9zn648zFYKiWskb1Yv98iv59I7NbL5oMwPBpzcP/UBhx2mEQ33oemtx6H29vEcG2k9tV6Ox9iE6RXeCPaZjBakt75PqM8fVPgJWTT0Vfps5I1bQrnJyPvM8cnggasmXK5YogIKPjn3O1Wfv5bPkD9g9qwXZMIMJRXu4M/Zzaunm7hYYHqri+tGJTE+6jfv6HyN8fgLRc9OJC1SkS0eoktl79WZMw2eSrPUAI2/0Ax+1haIuPSFngvDpqCW9RlEpcAKxaohSw6HxhzAsHcGaK10BTjcIzJXsGyQ3DVAW/iZF0cp1hp5VJD2PjDmCTe9ARmKaAT4MhRQtxNTEYDv8FOW+M6kPXOdOqKgE5Td826uxhMfTGxBFmG8VVwx5iZSgEzRau7B1nmTGVh3+ZQPBNZGBuIlD34CzxElMcTcB5niCuieht8Vx04gHeXDUo64+I2GwxxDYO45AIUH5XaeR+ad6mNT1CfuLHuZUVDrlLinDwcUpOz11FdXMOd3KzLxaTgU+yeQ/ZQIaJI6NOYbGIDDl/Gk+cl7PQh/oNC6lNmshVbkeSeSBEqFNQWPTADLW6fPQBYXjq+1iScabJPifZXbsO1T9/D1NH/9CZXoc797/rpf8tzvRIEg0+2/mTMLN1AR7Aoch3dOYqX/M823JSJYUzXKT4hsqCxIVGa3UR9UjIBDZfjk+FgU4JTpFyjNEPr7jY7r9ejGq/AjoG01Q91T+GQznptfSkKywplSu4K9JhNl+zVgrA6ibkc3xpR1ueXZ3nWU1yFA6spy8EQ3uZ6AcU+7n7MI6VC8+S1GWN4hN5whH5whHLRuoDPmcY6kLOWJZPejaLhmyfBvVf1WTkJ7gdX5tbC1L/lzCAzUPcINmIQczJnnOFQT6/MLxG51OyeYSZIcTQdagtYehs0UgI+Hb0UTj2ZOUv7yX9PN+BPcoAFe15INvZwArE1cycuurxJR10GMowK5u93pOA7Joiji1CgEBWaV43lpGxJPzSzXDTw/3+D7JImY/Mw2xIBtcbSTgAZbg+f+BxIbGJWc43wj5cf4EJNRTlNULWmUtMnDtOUYItBbTMf1SOoMN3PjVjagcSj0tDgtqlWts61UjmLuIPb/Vqy3drFnJ21tWRA3ISKINSez3ir0ARKjgs95uXvB9iVOqf6JvOYLDuY+/F2XukqhMqLwAhF2SWsL65WewB4Yx6egKLjrSQELz3QCoBDU2nY2KxAr0u3eSUZiKzqYAa07HwYdZvxDWFEbioVDCK44gu+qqNKZEfXQ9To3E/E2jXTYnHmUjQRYVsFWXlZ3Td6LWgEblneyLqjURXq7BqXaiFtW0+e2k13iOSG0zvhUJmLosvHf/9xyccBCNoLShjwBGAXZN30Vb9BBe97cwueAYZsNZboo9wm0xJ3gxGHpMfTB2NMNORDLrz1nuseChCVczU9POkFPD6dzVpez/BLyALSCQVJbEpI1/MHPHTNcnIib5VopGjGBG0lziXvUlvTAdUdZ4+RAeu/8YXdVd3PHJHQBcNURJ5M9IVKQxdz27iw3XbaCzq5MQYwhP5nxMRu1rlOUuZ8X+2wGo2lfFt1O+wq/ZMx6U517Oudn/QBAUSfMX1S8i2T1Jhbxv8/hsxGe0FrZe8H6Ikogo2ZGdMjk3jKQm3cN+avHfiq7lN3p/+I38dfmsfPg8xVHPkpd4PZ2uWNaOWTs4nnuc0knxrF2+FhE1RUVO5OGvkzXmWVKnpGIrtZH5YCYJlQm4/TlFmfz4ZziU/j53Nd2F6cqFRJpK0ahshCYmu/ceTtlB8uxk+hIVNrLBHk1cyy0kNN/DxEcm0jL1MkqGJNKrL6FPV6nck6tvRn2Qyqm5j9IVHOa+p+Ko59g49z5aXpqMacoIHDofjP0JhHbNwyn2UB38NaWZOsKuno1D44lDuS0TcFKfPp3wuVk8MfYJrky8HwEB/76RqGQ9ogiyINIfoWF3yNPsy8zlVLsi42HUGIlUZ6GWfPm4dRm1Iavc13c4AwjTdrJ47HHGvf4ibZGRtIS2IAmed89gi+eSuJuJ6blkUGLsQtBKav1TDKv63OvzsKYwwopEMipyCGkJGXRfyjnfr/ie/kdvoCc4wcvjUJBFbvGDt+J2YqpvI6Q1BEFWK0mvM0/COiN6Z6VCtLjxUWJjPyW75n0OzJZYeFzGaFUYl777fKncU+kF0gHI6LuZqMejEHx9WLrS4v7NqPblZFW/S1zLrZR3K3v7gXseGIcGSr/kDTIbKMPOLyTpnBptyTlMrd7qMKIgElkfSWxJOYv+WERwe5RbxnNZ9jI+Nq5gb3gtZWnNJDGMjJqnGCoucyf7JAnG3T+O4wufpS7W7L4meBQzBsZ0gI82fsSd1XcyMdOzjhQE6IzMoiF1CoJeh6QaFMP+17l6AOrTpnFzzdOYIk3/9Zf+/7yoVCoyMjJQ/W3e+L9V/pPs+0/5H18UnxIZm81GWbn34i+iyg91dBgnRxzmRMpS7/Pcyb5BCGlXQFWQBabvnk7/+SOuYwKxrTe5jqkI6Ahg0W9TqR8XzurLbuB0kgfVPiVoOaMDX+P4wmf59d1d9Jh6cIp9nO6HrCrojbuPBmEIHZFZpG0Zzp+z//SqU3jXAm7ccQ/+bU3EnN9GbdB37jrq7ZEMLX2CkadGYtVZvQIMAMZeI341VmzPPouj9TLK+g9RE7yKyrAPqDVs5lTOJsTckQS3alA5VG7kJcA0uy9773iK6d/queoHjwnvwy3wTUcSpvRVSCH/xDgtl+TQJ0lsUhCwoV1zGBU5it3TdnNwhpqXxn7BvNatmKxZyDKIgpPmruGYenPRlelIyEngo7c+YubMh/hK/PeQDF+IOoJHDQdA7TSRVvccfiJcY4K66Dqa4+3415sJbA/0MsJVqSCqeA/GTRu44esb0Fl17slZRE1OcB/3vfM+tcNUJOVm4GvNoiNgF4UZz0PcMhIqniZ5z/f0V9QjabRe2t3hnQuJa1qE3GIn+eRPrvdEeR6RHZeRXvccSUETsD/wMHun53vdT6vfDspS6oh7816+X7FJCSi6vFkag37GbDxDVPFuNFW9XoHhhyYtZXxyPAlHXyFqwxvM3Whi3CGFtp8zDWo+lJlxCdwfCsuMgdweAocm6om87joeuQheCWvin7p7KJz9EcmmBfTs7yGzIJN+q2eoVwkqxh0aR1TJaYYfqPLyIBMHJftCqk4QUS+icvoyyqeDvGFljJ05lrixS/DvVGPsNXq8K9xTiYQqwERzuBm7xu7lsVQdV01VWiSWx65l3xTPJm+SHmaoTxJrOoe2rxO/5wS3Z9BBC1xaD1Naovm5B3rCUnjstcdIcCX7AK5tcrK8poAtr24h6dQvNCWMJi97BztNN7sX006UZJ/K0U9oQzP+Xf6IqN2BR5vc62H2uTZ4TwfBY4lmdszcgRSuJ+3YWnJOK0mcuqCfuYbvqDgdSWLeb9DTTVCPSyr1ILS+AXH7Snizp4O19g6eqilEd+YY5X+UgCve3qLKIO/AeK5buZig7Aietj1NZaJS3x3DYtnZ+QW/S7+jmazltfSVfDble5aYXmHOqTa6/VrZMG0SawvWEnt+OwFNha66K+OINewazs1+gtEvLubHq/bQ5d9Fh+9B9maNIC/hRgw9LVBUhOjsB9eCtE9fxsDwOkQLYpgvSduTODjxIJoxAVx/3XNMuRhe8AvjajGQ5wISCUpfxA0f3IJqrC8vjnyJxX4TaPc9gFXTQM3+alpaGzkyTGEqzc5rQECFyWxCrpHxi/XDb1Y4oX33kFn7BirZwK5dsHTtUrovv5G+gGh+mbuCP0cl0aMvwKC+nbq0qaQGDyWyIZrkrgXUiYcoiHkYi7aSE2Ma2HizGlVoGFHty/FTh3AsdSEVLvSuVWdB3dPFiIMNRBZH0vhHOVsk+38pTzoW+MPez2tfnueValzt5KZ/IggCsefnMTbqT9r73uPk4zuVBfogA2aLtoaC2Ifpb9pP/96jGM0NTJ0KWmcgvtYM1C5wgUNyuq8tiLLbj0sSJeqi6+jz7cdgMSA6VYiymmeCIFxr4/a8UBZtnIOmo4XyyE/xm/4FsmBnhgEM/e/wdMgI1l7XRmemHxZ1LbGms1D8PvTVIAoiEQ0RTNgbhelYJRWhj3Oo+0fwzyLUp5ppUduJy0kh5uRWxu9XklNBagvX5jxJdsw69g9dzK5pu6iKPkxN0CoACmuu49HiOfgWpJJySGCf9BSnEq+he5B0pLtkpNMZlkZW6xNeQSxZLbBz5k6+vWUnn9/yOQa9GgSZn3rgn41pIGpBECnMraUiqcJrjlMJKmJqYkgoU3N4/lnMfmYag36mNuQbJMGGf98I7s99hFTHZbTGjmDXdA8y0aKpoyHgF949cx/ICrOgJNWDzDTYoy/wigIYaplKr1PNZ13gVDt594F32TZvm/t4vBpSNYrvT8ZPfviZ/dDbo5h8/gRZWtgYBb3oaMouRGcB6+7DaJwKGvj1ENiU8guPBTzK2w+8Td10Jbkkymp3vx1IBDkHvXcDwSAlUQRin4TY1ozadqE3rU1nQ75sFA2xBi+lpKDuqW7prgGkr0HtQ0THxbTUX0NC/QcMVX2CIeo+SibfQFPSeERZR6+krCla9TMIrTnF2H09XL36anQWDckpDgrtcKuhluPOBML3h6HrksCgPMOBdZ4sy8iyQHBmGM1J42gKraHVtBur2gMf8tpcutrihUwFuNCwVoP54Do6gvvpMfUM8v+TsKs7cajMXvLwA8z3gQBJbIUSBKyOq2biBAG1w8SDgeCrHUfAuRUMPRWCsaYIbW+H8u7qQiFsGkz5lYMbx9FndLqTR61+O5hXB1dVJPLlybdxWh005zdjM7sCja53KqR7OtlNLzIl3KVFPUjGc3CiYCDJN/D3t/pJTDR1cpUJrEEqNi8uoD7OD5VswE/XQknHcL4ytvDGI28giWoW/74YX5fflU5lQSXIdAR2UJFs4ZclvzDu8DgW7HqWnPAcztqgRAjg67aLKA17ko7FM8nPVtY6ASL8Fa3lzwkvMKowi9n/TManLY/NuUpbhndcQnD3NAC0If6cyD1NS1jLBX5qqYUJWLP9+XXJr4CATd1OWcQbnIu/k7LI12lKMlKd3EVRyp/0SZ3YZbi8AW6oF5nw41K0vf289eBbSnLB9R4s0OtZHfMAU/eMI7lqIrdr95PU9AApjY8z3Xw9oMiQWowOnImp9JqU4EtykvKehBhruDykl4I3nqN4WBptmg+oCfB4+ww8szFHxzDttz1cv+p63NtpWcZgbkIqrGXcoXEEtQfTZTzB+ZTLeWj8lezI6mLd8nUIBdWkHfrWfclOazglllia/My0hFtJO/IdoZVHEUTPi36XoYzsr64ktSiQ71Z8R0NkC5dHP8Kd/op0abyqneaQNTRseoehG6OJarsSgJE1cFferUTXRRO/N5bCUSMpST2FqW8YKiChEhbVQ0laMb4Xj6QrQGFeuddvbjlCiUbfrVSHfEWvrtQNpgL49PZP+eGRY2xZsAWr0QbI9Mpw3gadTlj20zKSz+pJLTQRndLG8IrVzDhTjpN+XmoexeHJPfw+/UO2jPSlwemksHUst+few5DK9/GPbeTc0Er8GjUY+yLQi74KUh9AVpGfnU9blIr0ioVMKNpHTPvVVHYNo7gjx9VXPGPjCdsPSu9S9aGyWwn9+VPGfR3GjJ0z3My8wUWSne79mVpUoZZMaJ2BSIKdXy79hdrhSiBSFFT4WYZh75rMmaZpVAW9SltMDgnFkczafx8BvePcHtImdRBqh5oxh6MQSwvpt/UzK+krbhzxML66dpp6o2k8dYCeQ2cx9caTW3mE0SW/u+skDGKMDWYXeBXXayMIUB7+FrG+iidXfk0wSUUBHJxXwpFxRwCBUeVrmZ5fSIh9BGnFacxf1094UziyypNUMthjWOjCNhRNH8Gaq9YgSga3R/D3SQUsNHaR2BVMr7/1Ap9ogy2e56Y+x6WvL2bCvnBgMBh1gDnqAjII3n5wsa0K031wAkkjePZNA2OKJEsIqgufYYA+AEmW3KzowaXY5yvKo9YhGnQU/17MX3ETafbfxMSCQ0R2XI6IyGU/X4a4pozhb1xFc+IY7wvodIy4SWFER9d4Azj+7tk3uAzsmRwmLYVT/ClNKXWDfQVEypPL2TNXxrn+V5KPedhLDrGbqpAvKNArQJcO3WnqA9dSZ1fG5WeCIEvXhXloIeeHdtP2+c/4N5x3t9EP3dBi76cmIh+LwU5gh4c9t6N8B9XibgAKF9fgzMiiM9iPTuNxenVlrrYekPF0AVEHEnqyyn3fA6pK0qBk3/V+8HU4TNibxOg1Vi75Zax7nze4vPT0S+y+7Aq+X/E9VoP3O91j6qExqgtDUR6xBfsIq66+YIwSZAFZciLIMtk1LnlhHcwNLcDUray3y4fNZdfQZK9ra2wabAYHG5bu5uzQs4iy2r0XPBd/NzsnvU3jRUHsn7QfQRC9+tvQppcpG30Ne5a14d/pj9qVeP0iHAptOnbHPcnJ+U9gMfQhqSTUohYBMCfDb1Ei+6bso1+nxrLzEKKjnw7fQ2zvU3w/nysMYt6m4XDmHICSgHL1/3ZLJDWSnsAN82l6r5HYqggMfQa8ZXxFAjsC0fbbmLx/MuOKdgICgjaD7ujFhMdEoa9VY+wzIok2+rSV7nNbU1sZ//R4jo456nrmyvP8vfh3Fq1ZxN4X9nLm2zNMvWEqM76dwbLEO4htuw6HzgdDiDJY2PvsqHq6EJ2e2N3AOs9ul4gZF0P25dmIak+dbb02rF1WnDYnq6atIrrwLzcY8sCkA5ybfg9+MX5kXj6UlkRP4h5A3XcYx7HT7HhsB6b9W9yKPANjzbExxyjMLKQnVEdtbC31tSoeelhDeZUWTd1KVmy9iou/vpiGixtoD2p391NRECgPf51yv3cw95hxSA7aLBGcFN4jZ+4cVIKaUcdHIT+1kYZTDW6WlNYZxLCqz8mse0UBFiQNoWxonOtddUl/D/QprQOnRo8ketpqgE0n2e3Y27sRHTaiOpYztmQLGmcgZxJvoiFkK5pwZc/Q7LeV8vC3Pf6ygkxj0jiaf8wn4p4Ibkx5msHF1thO9u6PsCT6cWzMMS8f1Tj/OPf4X2U/7nXetclrMGlbYIniz1qZmc6qG1YNxqXhZxnGU8O+ILv9CXdf8t5DDcxj3pkiAZERp0Yw9hcjE7+JYOmvDzO+cLf72EBxOp0DAWL3Z7Lg4PNwiPhrCr69vvx28W+eZF/gKFD70FS7grhjT6P+/jMkSdkvqQbmDVf9ot+L5ptp39DkPO9Vt0kdn/PsimfRjh9JfdzAXKOMB736Ek4nXcuxr38kY/+XBLYp48244h1e17DL3goCA6Ul4zEqr3wM352/E1HmLcMuCiLXr7qegNYuLl6/ghF1Z1DJOgRBmd+04Yt4Z8hxWkJ7OX7l28Tlb0EUcSf73i6+namrptLhc5gBtZuBZF9uVC4HbzzImsvWuH/PP9if0NhQ1OpBIDfX7Tq1BgIevY3GFE/fszZ2kXRiPQEN3u01cKKo+t+dbpIkiba2Nrc61//t8r+79f9T/n+iSBK0++7nbNwdlEe87XUssjoAR10zZv/eC08UZBxOyb0YtalbqQz/ULmmKPHxHR9jNcmkHf4W0e4JeguyGo1dQ3RdGE61ssD0uqwguAfQASmLgUROgQ1WCjl8F3wrPmXPUTz/NAGdAe5zBwxZRY2azrB0rL6hXolEvS2apOYn8ZkYxvY520H0DozM3zKf0eu7qEqqpSmsArPUSF7iDVSGfwAoshmO/Ye55OchaG1aQvw8zL4/rfH4JVnoig9j21xPMDLPBq+1DqNLGIqk1uJ/5UVYU4e6E1sZWQ5EQaQ2thZZkDn78V4kq8vI1QmL0t5n45Uq6r7+ji9Gf4G1Q0GFT59xC/tk+38p/zdQjgD75X5OZGxwt78sSDwVBN9GgG7Ofk7N7CZ3QwvT915PiHmW+1yzUMX5MWPoj4khuC0YQRa8FivbpWHcWW+iZK4/k19dSF3QDxxOvoisbxbweJOVZ4ROjkzaR5nmGHdX6OkxFIAsMrpkE36WHFrjR5Hz7MXUZiiIuAGT5YKYR0g6+TPFV78Asuz+fKDUBX/PqcQVyubUhR5UDQpKt/htQWUpQvbTUTbEY1ydpYUrApqJTjtAa6iZwFY1fmY/kEX8VRARAL4zTXTfDemLXsdnWRi6CQFE2pfyaZfn9zuaA3C2dhC3Mo6P7vrai+GZGpjO7B2ziSk5S2yFmeiWJLSOUFfbu3yDZDtlucsxx1zH/FPdTLNNI9ynHsGShMYMy79LZtL+Se6gQ3TbNSw+WMW8vx5CmxjNz8sPYfY3e5DhqCjKKCI/Nx6H0bXAlbTEtdzIvli4Ufcbk52vYTQ30nFTvzvI3uCE/fXzaO8J4unnniX1iJLUGlgcPxkIy01w0AqWJRZKRy2jJns+Z4cr31M7fVE5fRQddhk6w9MoGJmG2c8MssqNLLYLfRfIeH7aBSr/HqTZ+3EMDaIi52K6IsaQ0vA4w3WKP6d12u1surgYqd2Mqe5HtLvVmP6CfRJstePFFtspOzggg99fYNyVRE/8HopKXsOvOxxHvzdi3qHuotFWyrfZ33JoWSON3T7klxvRYETrDELlNCJIAvgLFE64gSaXJ5p/30j8e3NRSUZkGQyRAdTE1uPQOGj33Y/ZeJqakJUcnliL+ZkbUbfUkViuBPRlYHQ1LNieSvwr92I9XU2/w+MN0NjyLH9ynuxnHyB6xYfcM/kLcs4O59HcV3CqevjJ2kGJvYmMc++Ruf9rpr44k8aru9D2azHY4t3ePwv/WEjAXQHEjo9lwYe34yvehp9lqPve/WP9cZoCXM/CwzztDkmgIW0aXSUtZO/7DENNMUXCL5RHvEVw9wxmnCshq/UptAeLWfzHTTyZ/CuukwEwmXVIgopD08MoNZcwWSX8tz6EoCT8pqoF2OXDlF3DufHFqdz37TsE90xBEKAv7h98Wz+C9bGl2FKcJJ3egL7Xg0Yd6B8nRyuBGKHzM451b8Ah9mBTt7qDWpLsdG+GyvpOcjZeQdI61U6+u/Y7frx7D+/d/x4t4Z20+O3guFVhtfR2mhieN4z9c6rYsmALvj6KbOBfMfB64GmG+jhJLLLi9+oPnA+bRJl5gsLsa9zJjUV38GLlUAI69Ry8rpe24DY3OnVp+ossSnsfRJG2zAlUJSqBu0ZrGP/YfphJEft4ILKfoPYgxu11Uhf4BQCB+gY+SN+G/0W72XuDHUvtcWzObdhccr4SDvZkDWV3dhaW5fMoHXMlgqB8/keuwB+5AqU6BVjREdhFfXQ9KjxBl09i85hQE4i6azeGDqVtB4+r92S+wk3azYhyMB2+bUpgxaF4/t2StIF9F4/h2uMOcv+MoDM8ncPjD7vPLYt4jTbTHvo/r0RvsdMZlupOvg8Y2v/dYwLgk4LbGb3nVUJODmPS9xGEd8R7Hf8mHPbFwLkh57Al+3D96qeYmL8NZJFSO/zVo+WYKo0T0/fQY3LSeWAbFm0VaXUv8LtrWXPOIaN2qAnd2050bTQOsZt+HwUpLcpqQptDmXFvJilH13j9dlD3VDIrn2TMq0HYUrOpzB7lPja+cK8bpS/9dJRFa8rRWz3gHEWCy5vRExcncih9BoeyR/HZlFnck/UFOjmaouSDlMZt5LbLkzB2TedMZzr+6jMULFlMSYaKhMoERElFk6Qkfb/qNXDcmcAlv17C+C9VOFo8vhZWqxKcOWJZxZfrP0S0VlMY8wSH06fT4u9Ztwzs85W6KvXb2jSST5vi+G3xb7SO9EPXJygMGtkDSlHb1SSVRCJs3cKSXxZxWf+PRPW7fE9d39s9X88v152jProerUZEJRmJqYAd0pME3HoDRaMXELPvR3odzykb4Ki5MGsXB6QQfE4eILF0IAig/HUAaxwVHKtfROeRQlZOWknjhkbXNzzRkcFxi7Cuhcw4U87witVe33H+jdnX0R/A9aXDOLdqEf5nzYRJP5LVsgmtI4QwnyruG/UIc1zTf0/KVRSPuZqrfb8G4NeiB7jt3DWUJ1Wwa14HBouB4PZgTP1x+Ov9mWuE+ysfocIUxIofrkTV1uFeD6dqYIbRRpShg94QZcxLOPuXu55xrTfTy29o6x9j3+E1DGQfBtbKA+WiTdMxbqt3t5dFV+nFsDo9P4gdy0qRVBJalRYnsL4HzloFhuQPwdRmpduvG4fGwTnWEqqCTbFWpgW/SP6U28DpoPAHJfCVnQ17e97lYb8XCP91Hjd8EoH6j9/4x+uXMTJvGXffFExODqhFOwsaTazthfDqIuZuaKdV97K7TgIiBgJpj1DQ+3E1ce6x3kkH2Xs/RfpsO/O2zSO8KYxeXTlzjbA0qJ2xBpmotivophmz8Tz7e1YBMDJyK48n/EycWkGES2odsqBCJYgkqmFdBFT6307wovFUZA2hLKUMq9GOjJM/euE9ezwJ+aGkb5lF0iFfYkrC3O9fixN+mfAkr2W3cnZpAU2RvVSFfUq38QwZWuhMhpXWJ7mqzEp3Wz0+PQOgAo1Xv5BlieLgdziTcDMdPoewqzvdbWIxWrDolAFLEARkQeKYFTK1sNxo4sCkcxRnt3Bo/CFCQgR0jlCMtkTsWKmyn6QisZJegxIU1op20oKVAHNsy2Y+SDfjUDuIPuxkzqlSlvq/5t6DyZKIKIn4NzvRWj2L4PePfM3je5W5RJIlmv22URb+Jm2SIis+oOQi2G3YjE5OjDqBgIDe5u05o8zNyrurcv3oQED83NBz+NTUcNeHd7lBezvKb+SJnTvY03mOZtMmQk61k108laiOZW5wgUbQ06/rZ/eMCs4GrWfXaF+O2fo4XHsxFR05fH72Sb5My+OfT72CzTeWkNZwfPs9kuWiZCBcH4vOHvEvg6hWTQM6/BhZ9hPXBX2GSlBzqh8+bwlDVZTEzK1JBLS65s1BCU5REOkM6KQqCRxqB4Kk8gLi2GT4cfVsxq9uJb19OcHd05h/qpfW86v4s+I2suqHEPD9EoZt8pYxBzD0x/HkZMWTya9LmXcGJHsHgt7xx4JwPv086eddkpMV3zDjTDl9unLOxt3NkZa/vNrQXe+BhE+GgGGEgR92/uD12+WXl3PyyZN8nPkxw08Nd187sek+mvUHKIj9B+HXz8Oy2kKr6QhWbQ0+tiT8LMOoc55h97TdOC6Kpj9HRUnCX7S6FEvMhrO8X/UE5TdVcHrWP9g/VWE8DuydxxXv4KZaiQl+y3EK/RxPWsrJpCtwClaSGv/Boq3PYqqyU5mjpS6mbtDY6AFPEhyEzeDHBv/pvC3H0ey/lbMJt7LToDDaqgO+42TyFRzqVoAurRL0Sxp29IHosNFVW0BB1DXupEeGFqTOE2zzm8PBSeW89dBbbpk8gLOiIuNuNzlwLFnGmZx89meNpiDmEdez8gBCwTP3CrLKfUwW7Bcw+74xw3FzGCkt4RjNIlsWHKUgs4B0o0dCD5QYyADjbvBe3vVjGHvUGKqU/R2OKhoCfyHEPItrAz9DZ9Uxa8cserLTOZy7jh59ARcdl1i0RmbC73sYNX8sxa8aaI5Pw6Kr9rr0A+88QM73MZSmVdEW0kaL3zb69KXEtt6A2uFHftZBmidokVSSi6GpPKPExgdI6byDoI5EZn8fyuhjo1EJSr27JeiSnHT55tOnqSKyPhjfbl93ovpMbwCSNgxjfyIRte3Yft+Oyt6PU9XD9j7FE7ymJ47MglhITeWb24/w5S1fMjCuP/7XXi7vGc+aK9egH2HgmtWXEVMb48XsExA5OfIkp6YtoDR3OSHd071YwpFj4zn+diWnRp6iMWADksrDOpIkiczrM9k1QwHFtVk84I4/iv8g7KswwpaE0RrSilpU43A9bo21m7YiZU2QMjcFy50Pesn7GTvrMdWcwWGXGHvvWJauXYogeuo0+o7R3Fd+H+HDwmkvaUdjvRCkBsoaUCUbyKlY5f7s2DgzhhceY8m3S2iffDFOUZmjz8Xd/bdzFQsJe7/yrHqqTyE3bKerp56QnBBqLq+hNbTVk5ATIaQllCdffpLVWat591gaa4ZGgNwC5kJUggqb1obko+PzkZ8T9oHCzlMJWk4kLePPYVH8vPlnjKVnUNkGbEY8sRCAfqfy3msdIcw51c6s0w0Isgr/Tn+MOyqpfeBtwiqPem5ikJzzQGkIWsf52H/QZtoz6G4lQsankn5x+mAsKl2G07xTfwOtoZ04fAfUn9RsmFHIqotX0drXylbrs3TrC7DL3jGu27I+IingOJXbN1D/0w70fQPPyBts4V7T/itm3yCZz8ElxDwHc/gkqualwqyZ9EQscrOpAUaU/8Dsg19jWLWJ3E0vUBCYi03V4b7qB53QFV/DkTHHmB55E2FdC5R5LOEKCJ/JbOEgZt9KNLWNqFtrASjrOUOX4RTSILsWgEJpk9e/JaeAw+pAO3EMNTm3MTn/FKkNirrBwPMMWifi21lHdF24+3Ot3ePZ6/Zw/VuR1FokjY6aecP5fd57HEyfNqitRHZN30V5diYRE5NxapT5T6OBlt4WjJWvcrefmuyzCojBvykfm67evU4q6TlJyNshzF67jeTiQK/299f7Mz52vOLVB4z7chwZL2RwcP9BJKe3jGd83kaSj63l7zkre4+FoIZ8DD0XsnHhv2f+/W8osixTU1PzX8oV/z9d/pPs+0/5H19kGczGPBpivqY+2BPMyi35DVXuCoJfuIdeo7LJ1DgCiWn1GGXbnU73ROhQdXsuKkBzeDOy04nOXEpewnUuxANoHaFkW9awe8k8mrP90fYPEtAGjCoT3b35pBz+itHfDUFj06B1hJCohkdCjIi9hczcnkzaeRVhF8USbr7Gc25/PJ3GY2ze8RvVaUm0xo30uraAGqfWSPDzQzg56iQqvAMjxWnFlI/S8utVWzk18hRqlXcyMK0oHinvHCdHKRIYGjxJnst68vkp5wqaZ12B4P8Uo0s2klnzBgDjjS1oD68ma++n2CrrkWXPQtshWNyTQnS1xNl3dyHblIW+0wn13QrtPTfrAwx3Gtj/6n7eiHgDnUNNesZrzBN1/63f1zxRhzjDH0yQW7qB8UW7sava2W+BX3ugxgH9eied4+ehNd5PUvP97vM/12awcdo4Ki7L5c2H38RqsHohx4PTTmD5bjc5ZQv4Lv8rGgIVE9xgEdrR8IOuneKESkJXNTHk3BBCzDO56ITCvrSp2smcko96ZjCbZ3/Cn8M8SbmmgN9oC1OjHZ4A+fkMKVpBeMclXvc29vBQKq57nrlbJiE6RS8GSmX4h3x200uEPP8QI8t/dCcwR1XDfgv0LNvE/oUWvru1nO1zt6OWfNhrAaEE/nlOz80f3MXsn+u585M78e81ISNxzgaRZ0ez1ec0bzzzNLbvfuCqkVcx+1wHo8t+df/2vqvOcuuBWykefyMn5j/BiMxNjCpX2kVvjyBdWECGcRLRRTtJPK34HkyN/4GWzkhqf9diPVXIydxmKhIrGJhCVLIOk9lO4rmdWIpq3Bvywcw+AJXDhup0ERENEQiyBlmQuLcZtvSq+GujDyEVP9E5y0p9dL27vrF9Y3koax7NSU6ii+t5M+1NenVKoHumES51kTHt6XY6I7NAENxJlJzKVcw/1UNG173IMvQFRFOdFk2/vh9RVrs9Q5z0uxcjWpe3RY0D0rUwzjECH0MY5ekSxcl/IUoGnDIka+HxrAoK089hK60i+tQpnHscbHfy37LFtjuhf285AQXzePGmJZydeT+GMD/KtpfhZxvkEeZayPdZZMpyl1MxYom7jilFvjz7wrM4DjnoCYrD5qMs3sYX72R06Ubs0gE0bevobezG1JeJqXcY7SYPmzIv8XoaOYNp12YmHvDINLRL0Nd6BSopAUO/Lx2/dhDREIEk9CvJwvDP2dn7ONUhX+C3aCqPdj4KQcpvb+uDMXrF58nY42TDFetIeC6E61dd7+6TgguF33uZMl7bvYnaHEyfTPzz8bRYdhNd+Bd3v38thj6D16bfJyGE4mFxlEefpgUFRXY24VZ0PW341BbiPJlHSM3JC0zGp+2eQPLB1TRFa1CX2LjZ/u8hrK6zyRiLHYw9ko0siqhDR6G3RynJvow3eaF0Dl0BXXQttXJyZD47c8a5GUgDQfpenx7K5ls5kvszJdYj/JkTwfbhociCg9zSX/Elyp2UVP8N4BHUPYU5/Z8z+3QjU/Lz6NfUs6EX3uwAR0olrz76JkVDlQW2KAqIqPnKFfMMdyp9rCSlhH69HT9ds8LsK/kIszacppF5bFlYQme4DUklKcm+wzcSYqzhmYpxvLvsSfzKj1GSoQSx7E4j5R3DCTM0Ea+BwoxC9kzZg+Rin0yK+4mFoSUsiurBapBYuHYKucdzuegiF9MMkW7jOXoMBfTLPZj15+jWFeHAs5F8P0jknbXXMm5fKhqbBlFQ0eq3gyd9Q1gc0MrmCiPD9+3h8vcmM+vPWe4+DDAzegm+SUMpmngzVVHliE7RHVCcHacwI6r3t6I7dQRR8vaulQU7AiINkYrEWHtUNiZR2bCHds11+TldyBqQBRuPpWxkmt1ESI2eK4886/bDAvimGz5riaU9uB17vA8qhw1R1iILDtqcMNPXxjTnWTZHwebFRXx+1XsUxDwMyKzvgeHnb2NxrUByWTLhB3oJbwqn23gOm6Qk4kPMs7k3R/GVCGgu5njSUpr8lXsN7Z5FSvNzGCcOxxEZS7+mwV2v4J7JaJyBigLAYUUyxqnybEL7NU0MJGgkF7NPQKBfW88R1+3t/SKX2C1fcC7yXs7E30ins57xxTtZ0XcNGW0fUjh6JDvn5PHPp/5Jv1Fiv/UzHguEqw19zFKfZ/w8Zc3VvuZDDqXNoDbYI832u/VBem9pY+SOlcRVKIP8wHzSqyvl40AfHnBtpgfYLV9WLuCZmkxOjTyFzV/k6s9TmLxvsoeBLosY+4xc9vN0VMePElndSKZzCX5Souv+lP7aHgqtLt8oURRQSb7UOWCi8BqLI66gK9iPDZds4OSQtZ5guGTnk533s+6Oj+jxtbLi13vxtWYS13Ir2VqYIAazeaSeddp3MEWZ8Jvi564TKN7SZm0hjRZFMl4t+WC0JWKw/7/Ye8swKa7u7fdX1T7d4+7ugrs7BAsECFHiRtzzJE/c3ZUICQkQCAnuIdhgA8zAMO7uLu11PlRP90zyvEc+nHP+1/U++0OYdHVVb6u9117rvu8VOmT9GwjyhWjj8eoZy1upGxht6CImaxSGMtmxNVCn2q4ETtQsJNKmJjU3FQmJroA45/VgQyk7R3/KWk9QG2Fipux4bYz25ljlMfY50qAYvLo5NucA/SH+MsgEKLPATQ1ww5HvCUq6mp5ULR1+Ps56Nnnuxma7QMYFHW07rvDgx08y68SLuA8CdgDsXHIQu1bB5OMyeKja97sh191ajXi0qkECtUMpIloF3Yk21B+9RP6EMBYfz+faskoEBGIdy6ciYgb9HkH4VucQWvAnAJnmL7HMfpCiniwaguQ1WnklB4DJp1cyImgECoXEExOv5x6DmajyKCxqkaNTj9I9CPQnIPKSWxuLgy5yZuHDHF4xg9GlMlgtL+Jpjkw/QvHVZv5aEIBefAiQKLXAh+3w+O5Mrt5xI40pFtbd/hYF/bJT7u5RD7As4DQ39fhz1R8hFCV388uSBayreZiJ9a+z0h10mjDcFy6iPnqAGaAkWp/G8qnvEzDyDcxGLZNPTqZgYgOf3v+pEwgRq4Iyo55cfSeNKc24t3dh6Dbg3T0JoWs8Rb1+qEU7/llHMT68l5t+usnxfJWzvfLcc4FSRo0auhb6tvgy5nAc1266FrdeLW62IKwSFFjAp/VpAsQDHL+qlqbAJsRBjl3R8eyEfE9u/f5GGbQnCNR1J/DJmW8ojHmUjzsga3QhthvXYFNpEQRZ1QMASaTPrY/TKzQ0RblYVytTXmfHsijiHfOh3nsL+eFPUGaT+7vReweZKcvYc3sH+5+poTihGBCZfamaRVkShv4UYCC4Lrd5AP0vDKIvtKuvIAkSIkqKg15n12iBUZOGM6JpIcNO7sQcHUXtsKsAZDAh0GvrwK6w49eiY8yxWNRmNRdah/Ha8T+o646nuH0YP/VK2JRWoi7tJPnkt0P62qd3Ir9NrmJ88QFn3QY7USv9v+LLvrm0uv9JgJiMiIIzRvijw4vahBIOLCwi+XwIGTkZf3O+ijQEN9DnJnHtr9fi3q13Ogt1pihKU06wt24NGqOZueefJKFeZon8Wb6Gb3K+xIyC0thSvBu0DMseytoSkNNPqL10KAgmteojPPqHOa45JHBz5PUjtjyDyKb7cO9Pw80cTbP7ISoDPqOo66Iz99hgZl9i7avMb93FipQVVP9eTc1fNc5rGoWGzqpOzn58lraiNkJr5UUtsP1qohofBCCgwYeShz9FOC04tni5PtV+8lp0adglbN4qct/4nZyomygNeguAPk0pmyo/4Ne8zVi1Bky6oUEqAREkwcEiN9Pg8xt1PnJQLrbxCUaeEdHmVGFznFdc5yVZsm1kJkjTZ1CVNo9a1V90Uu0c74F/B1irKlG+d3EdjK1eQGJBIim5nnx191dcGHXBGUh8s11et2/+8Wau2ejvGNuhMtkAPgXuiKdPorA6bAJHUDio42rSK79ghHql4+MB5SSlk4H+GfgFAAEAAElEQVT+9xQqALVWGO3RRMgDG9j+YAu5GcXUh9RjULgCw7co97Hsz+NE5eUz58Ac3K0u/wVAfHE8t30j5/O6PNrArnkf0KnPolt7BZWgRZAEWc65r4PikJcoCnnB6bvw7ZnK5/N+Znr3atw6hyYYMfSnUBWjQ5sRPQgc3kaXWw4mZRNKu8zQCdjfye3rbkdARHTMEb/umajtXqhN3dTE9lIcX4zCwey7uRF+7GlFiJ6O2P4Ra9YvJ/1yOipRgwTMz59C+px67l73Crrefg7ekcel2Bec9dII8PGy0+ROuw8uXSL9Qoijr+U2GdRtxPd0YI6roG2cJ3vn/0lTQBN/Z/ZJokSXjy8dQS6JbWvfRuJPvMjuX7ZyX/zbTMvNI6LlDgDcTNFMEh5n6dNL2TZ/m/OedzLfGdJvquEqYt+JpS60DpPVxIZMORAeeWkXv0x1MeH/rlwXVH6G9PzdWAflA8vMhDfegNzcod99sPJRKoYvJalWtnGD64Lxq8yir7WPvXdvJ/bcZsJb15BQ+5LcLrUVSa0hYnIE026PcwZvjOoaEgUDj/+0hut+X83obUU8+PGDCJKSeJ+zBKgv8UxvLCsW3MqGDzdgsw/MawezTwSjzkRORg79jf2OPKqQzodQ+QuioOByxmU67p5E4pJEesPlM64oqTAr2zCp66nfV4//8d+ZvPei49lKx3fkd9fsCPY1exyk2XMfWmsQAiJRFVG4H6lB6e9Fr6fLD+WU6hTs8vsm4cw7q7C7wBDxZzfS39pJ1DNR1Pa4gtwWZRsn2cG2a/YjdhmJKY1BlJREeySyZvgafsj+gf2ml+nRFmB1sNEMxkQmB03G6khp88b6z+nYcpKo/AKmH5mOYHfNPTtWuswdmMVOFyhl8BnKYfuWBL/BX6muYLBf93R8bc+hj7seYcrkf+SBC227jvQLSsTKGnJTc+n0rGLgrGIwpvBgM6z0L6Q1zI3F/fcR1naDy04f8Q5z6z2cqi65/deSH/oUN5wYxvHUkaitAXy/9AeUfyl56fmXUOPOeEduOq0pnD5a2Hj9Rrr/9SYaSwCe/cNxM8sAzwG7oPoxEzVJsyiLGZDiFJ35FcEV7PPunoTWHOq81ySdwNZ+gbK4Kkpis2gbFLAVBZHTE07TFB6McVDcVaORGdnXSxe5syQZrVVPb5TIhuu/4Ij4L+f+LQoKApoc59m2DJL77iDeN57/VKq7qonfEc/BKQe5Uu5SLRMEUJl7URs76f/zFJ6NrmSc2sggshY+T33clP/4zP/dg33/f5f/Bvv+W/7HF7sdrIrOf3we1LmEQONoWl/5gnGZ8iKWUv0+wyq+Y+7FNuZd6ESwK4cYnIOLW68b5VOUvPGvV6gM3kytr5w4Nq7+GQL7ZhHYNoxRXxUx6/As5z3ufWmU9V3kN/NsvFpqiLoYypTLx/HvmkOGBt7y7uO+lrdJTCsjP7WDBZ/eSmzrh6isXnJbBDOFoc+T8/RpIq8c+g+1kujRFFH24Ske/OhBVLahMpiXMy5TOEWDXRgw8Ic6hj069WCzotW9wsKCbpZpXPl3bnCHxdEbUFq9Ce5YRmDnYhR2HVuC4NvQk3iY3wVLBXVffE1u+QrqvbcCcKrmFMerjrNs2zLi8yU2P7CVn+OD6dEUYTDAubrFfFNwO+tDivg2+VvM/mbyNfk8WplOdOxjhCS+ziRBZOrfcs1NVSiYLCgISXwd5Vi5fzSWEHSWcCoCP2VHLyyrB21WBmnH9VhHjPlH7p4BQ8xid+1+gzW2/epfI+rSTny2bWDrO9/Q7SZbkn+GwUtmWV61w6uD1uluNAY2orDpnYZ0re8G1tVM4ZcNr6A0dWJSuxyldtHMz8uWkbeiAfWO3xiWk0Ja1cd/G0l5Qx95IR3RLg5h1wGIdjvG81fQdTU4GWZWYEoNvNYuS/QYjEkOOTUZreMmQLBKAFGiIVTFkelHMGtszrnbJ/Tyad3XHJm4hZJRzdRnN6DrGnqwMZkgbFwYRoMfkkJFV6/LEvfsG8lNit3cEfYhHYGJtIXKSJ8/Ch7l4+JF+NXk0FlbyNlJ9RQnFA9BEBp9QiiYcAuSKDLheALafu2gsVCQmpvKxH15uG3LJOPyJNxMsUhIfNIJ1zS4sW3mIU5P7mJmyFLWBL+LoT+JVQY4P/Ulbul/h4ylgVQm+1PcU8zlyLsBmF0LVRY473A+KyQLsec2M+J8gvz/DoN3wBCGQWwxFIiO61aMzuuxDXKy+szOQG5eX4bivXvxPGWlR1tIZcAXNHj/xmWz7Ow8bwKwoxo/ip9T9zNJ8b8O9A2UccBkUcPPB3IxWmUndvnuPDbM24BHoyvYJ6AgIyeDuB06BMfB41TvT1yKvJvqoJPkpuYi+QrytUGWVLv+NL22dxh2Kp/Mtb9wzyfDmXP+6JA6eLV7oS1pp2vWHP6a/hcAOgH+DIUbx5VSMuV+dKFBdL7USWJhIq0ef9GcPoULEz9kc0sCE49FYzdZON5ynOc2yYfBcossr7Zl5Rb2LXcn49aRWN1sFCQVDJoHIpeGXaLr+i6qT1Wz7+l19Nm3YFTJwV2zsoXp66cT++NJbEoNXR7d2EU7AgrarCvwv3Q759uOs3/2T+xKuIU2wZUHwrshj8jMzYjzZnN5+r3/CPZVRwvUJs7ALtqQTBD0fzFOAyUQEDHx5T1b6brvKfpGuoxZhaWJad7lhCigd4SJHUu20OZTPii4MBDkFmhOMlMZKTukbQoHA8KuJqhjKWrJ3XkYGpAA0piDuTbPzsrtzxNypJGQsjr0fZ5OYMDPQbA93I5JYyTlgj+JBYkoRJE+TTl3NMnAgPzSYMadCqckroQujx56rX4QOBPS/s359M/5xJpOj3cU2k7koA8ieKWT1bCYY4oczrudoT1E41zLlMjSf8v2HuKJUgNzDs7BprA5pVvUDlTwVxUJ6M2+HFp8WpZEEl3BvoHSIhVzLC2dw5ETUQiuNWitt4SpMpgJmek8/ebT6EUfutyyiXFrod+m5i5LBwfmyIcvm8LmfIcBkpJtiCIEF/3FPR/dhr5Xj8KuQwCeb7Pw4okf8XvzafqfvJ6wvAPcsOEG572SYEOQRM6NPcfFsWZisv8g+bwMZLkQu5p+TfnfcqHIRYh7hJsjjrF4fhlnHvDB63QVIy6OcF7XdU3EXXBDawrHuDSGS7Mfxa7UUBL8BlZk5xzAod+mkZbtTrtPOyqrLzGNj8kBXtNbaEwaFu5ZSGe0ioLkyiG/725MYfG420l5/QZ2L62hwec3erWu90JSKPG8bgGSqKDO7ZGhdZdUCA4nXmWMDova4pS07NOW0K+WHQM2R4BpwDF/oA8+7YAiTTcWoZ2n31jL3ANzUSgEPDVNLIj/gq29HnRWBqPvHhhbgRjVRBpsUGVXEyc2ccvsS+TPtlI6qoVWjyP0q11ts0oCZdEyC2dA1mgAAY0kYhX6MElycMurdzQBHYuIFXWUjTjEyz5g1QrkZbRRE1aDy8GgoF/Xz65FxzDeeDvnFzyLoHbZCwPj61dvYfn36bLEtSDQpy0lTgVu9mxam/NoddtMzvAcWv0dMp4tZ+DUzWxQZ3FrUhOBzTHE5IcQ0r6CtKpP2WAYzsnYVrIiTbSFdfJo7aNoIjSOXpH7p9bnFw4mJvNB3tAxkus1iNnnGIs7I95jcsEZUtzriNX08cbTb9A01oMR+94gpOYMd9wB/VYP/HT1TLMYWLl1JZ5VQz1pAfoKavq9uVIUyqJt/nh2eXIp/RJ5IbKCxmtt0KaNYG1fHzVBDQRu2c/ICzJArdUOG7qhq+EWbvH/jMh7nuLyxIwhdW4OsvPJ/Z/QOcqC2u5HcPsqDKYExhcecqLxixNKEdvMjucKNHsOzRE1ZVMLK75JBwF0CgNhxmRKo2THw0WLhH9lN6MOb8a3zogPcRSYYWEtfG4ZRrXX+2ROqWXj6g0UBb/Ecc97CSz9hO32HUyans3uq5uwRMVRmziDugQZOS6K8MnZr7nQq+WW9bfg3dTJkZlH6PaQg30aRR9vzJpGhPqQLA0ueOJpnIp3ryy3LgkiR6cfpX2MBoPiXgJ75fQAuWZ4pAWqWlMJqMzCvdIRpHF4ob46/wnnOhM4YbHRq7di1Bgxq5oxSb10Vj7D4o12Pu2E18onoWlez79f/jdRZZFc6TrJhssb6Gg4ytWh7fzw0MfkTqkFwcV6zgwD7G5sznmHRZ3XMOpkPQFNASjt7kTlnkJv82ZMyG5nDpbcNHmeDDiowlpvZlTJb4xW38yAU83DfdBaKAlct/E6MjLDSCpIQmlREWG6irG2JHLr53O+Sc5f7mTICYOd0XL7R5+Rz3BdHl0giFR0pNPSH0Zj8FJO5EawcvN0pK4uJFExJNgnSArSctOYvMlErdtaDg4LotZ7E2kBR9Eq+52cB7soOyuVgwCQzZ77aFFfQJL+6YgceOdsdhvB7SsIbluFm3JA5l+u/9pP1+LV5sbnaz/HppUoDn6VOW7wUkQuM93ArrRi0avo1/RhE0worbJ9F6lLAwmKEpvZu/wMJo2J8b65vDDtKu4a9RA/LpjEFJsboVUhNIX4kDu8n0r/QTKyuNTCXTm2/pkLCSQEARoNB4lQwp74IlZFt1MZ3U5MfgDR5dFD2jy3/3sm552j3zuJ0+NOY9ZYnU5ShaRFavyWhffeSI+nEUO+nAOrTZ9JbvhDtPp9xFh1HxWjL9AY0YVRa3Seawb6/1LDZVK3L6BqxP1ENz2I3nGuGajvqRVFcM/dtMc+QXrVZ3j2jXSMijwWg5V6JnncwNyLrcy/0E1ky92E9S/Ev98fjztkR66X1guAcM9wQseEsrZgLfdU3MO++fuczxwAoVlU/YhuWqQuSZa9dgSttCqXDaneWoXiSA9Lty911leQFKhNaoZ9OpzwK/vQ9g2kCXGBMwaYLYMlLQeuF0y8HWnYcKZs6GLk+ZGDJClFfFt9Sc5VQGsr5kFyuW6mOMfI2h3PHxrsC1PCaO0OMlojiS/0RNcvrwEDz94TIuc8LUoooldvJf1SOlpzKOlucxhcwjP9UR05RHiZ0VknAO/e8UQ230O0Sg66udQ3Bsl4CnLaBAlXm2usMC9nKR90yHUeuE8jyvXz65pNnGI6cVmbiLmSz6TMScy3vDqEsdrl0UVBUgMNs2/g3KgztHiddtTBhoCAUWfk29u/xeShw7fFFwSJRs/d7LxOYOM1XvS2m6l77XsSsobuMV69Y+iJfpHE0ct5+O01TD+61CmxLwlWkESmHp1K0LF+/Jv9QRCc60OvppQS7y/wrN5KSJkblVGVzvXNTZBzqS7MnMzE437kDrdj9ZzARI9VjCzdSHrz/WA3o+7vwKK0cDLsV1o8Zfm+Wz3AGAcjOhdg0RiQ4uOJKPcjoirVCRxemvgBP4Zc5LWkCRw5k0hZsg5RkTZE7UJAwK3XDUNrGXWGj6n0+wqATv05TKpm8kovYz7Rhm9bEBpLkGOOxTJffIfhK4ZTd7KO5b8tRyEoeGjcQ0P6zVhlpDtfBtHXdtbzYdtssmKvpi0klZEPyhJ/7eXtKPMuoep3+fCaIkeTm7QAuyRw9rOz7HlgL2+8IQf8nnkGmgrauPDtBToqO5xswaCOpYS0Xs/o7OuIurybrpouept7UZrbqfPeSqu7IyDTZ8NWXYelzwFIdzD7AG5QpJMkKPGzemIKSmXU8lFkVHyDSmFCYW1BY5eYfHIyNS/VkPRmMsF1wS5VCAF6DH38cfUfTP11KsenHCdRJZ8N8R3rPMOYkVi9fTW182XZZBGV0/ejnK5EEgQEaWCdGJDxlNcZs82MhER29E1cjLmeXnUZgqSgMrKSlmtjCHz6Vrr9Y8iJvJO9I9ypDPhcnqPYnOuMJAwoSbmhMQfjZglDae6jvbWBxE8Teei8y585sGdojEp8jjQSXR4tv8eONV907tN2Z7BvUvlefr1qH2abPMfenZXP+nt/xq1HYPrR6U7QE0Cr+1/MOujNofDJONNUDHqf/bpnkqSTz9P2/4WftlDaTUnQG7TrTw35vC5+CpaV17B15VaZgeloi0dfBtFKmKCFaQfiaNooA71EEaj4BfLepNyqpNm/mQ23bCU3LReNxRVAde9PZc2wNUyLnoYkSgiIGIwJLHR/jsDORWyK8GedtA6AsPyDQ+o0sE91J1loiJuMUpGKV894FHYdRrULgDKQFkGUNKRVOsZQsGGofp/4nTvJVw/d68FlJ4YVXeT72Bcp9ViJSiXPy8LWQr7phOKCCFR93ZTfqqQqsgoB0WUnIfLlvV+y68Zh2P2eZGrnN0wMl8F9rX2tfHDqAz45IyvE2SU7xfHFHJ16lHZb+6A6QMmY68ifdAfG7Qfwrb3kvCazNwWGSAEMKv8N9v3/W/4b7Ptv+R9f7Haw/C3Y59s1U/7DZkMR4Eun5wCKRoOAiNrmjcrugSQJTtT33/MxPP7u48Tuci1kaquce0FnDke0mXFvKac9XEdlZKXTySQJVgRBpMWvhb/mmEn41zXobaNR2Xw40Q+vqOR6xcw4z/FZ9ZhM8m/G1z9PcNtKerXFCJJCTtJssuHZWDhENsaoqiczIR3lrn582n3+wewDUPRbmb5vAgmFCf8I9p0Zn4Xw7+fo8wpDiRYfRYTzgPmdl4phl/IJ/+Vlkk6sc7THzhkjdNtEPkrIYd2dW9B0COib+zErXXRspaik1beV+lCoDqjCrOpGQOTaa+XrPxTdxpvt8iYRd0cc39/2PSaDbGzFxD7KlDkFhE67laeVodwgevG0OpKm2BeZPquGmNhHBzmzXEivYWqHxn9ZCgkX9Zis3VjFHieyHeQN7O4v7yb9k32kXU5DYVUNkagI73qZG56U2QKD8wNs7Aa9ZCZfGch1u5bQEw61YbUIKDmWPJIDwwLp0l0ipiyGxK9DuX798H+MA4BZY8O8ZAWt4cOdKK2BcnpiFoFvPcIX93yDTWEbYoADRFaG0bvhD7zq82j03gHAYj3M0sHoc6NJO61m7PnlzDq7ibHFewhQwMYgyBnTRPHjn5M5y42j049i0thp9tjHp/7QmZbHVo+vGbvgNDVj7Wy78Xdiz28d8rs//ghWoxWlqRd9ew2tVUMlcCVJfuf8qi6g7jjNmfj5HLNXogz6k/0vv0bN1WoSa19hQsFfBHbK+YXa9Wc5H/sgOSlbMdc0M+FUKhqTxnmYHlbxLSML/oV7lxFh1gw8VF+QVvUJNX7rAQjRxlIaV0q3h5mWWy4z8fx0vHsn0GKDK0YF5sBlFLRM4MroYL6++2un8WwHsruDyDKB209ujNj9Ot6NBQQ0ygjhgRwYg+XRDMYkMirWEV/3ojMYaMXodJ4obZ4Etl/NmUgzD49dg6q/G3NVI7N/K2XiyYl0a2WkU2wFvLFlIg9+cDXW7n766su46z/bq/8od9pM7DrRzbE6E63WZXjG+XNy1UnavV3rUXWlSGxpLMHHdQSWncK79jIbu++hyv9rymIvs3XlViR3GLXnVcLyXcABQVJQFlPGriUXCFuYQWtIGnblUHZyxqUMIjaUYPXxpCZcNkKVAsxwgztif2Fq5CYUvl5o3tE4HX8THDGVivwoRmVFYq5t5p0f32GreTUg5xe7Si8j+S1aqDxShtHPzLFpx1wMgQEHmmSjpaCF3t21VHs9S5vhuOO6kryUPFqGBdEQN5nvbvsJk9aEICnR9ov4tfhiNVqdzNHB5fCEf3Fqlp2WoHa2T/Jk7RWX49m/fwqB9l/pdw9gyS8NaFHT8I8n/OfSCJjd3HFjPmWaw+Rrf6BPXYkggHvWPL5N/5UlRalE3OhNYkGiox0D8mdye4edT2b8Bx5ysvBB65PCkY9BnpvyBFU5gn0mdT2lvl8SXHICzblMoi/tQG3sJLnmbZ4wX88oLXzbpCKoPpAJRyIYnj0c8W+GdqdnJ2cmVFAbWgsIRBouQEcO+I5jduJSZtcdYnLmdGZ+5Y1Hl4fM7Et6hA+yfkcjQObETBRWOxkX5QTjsfp6Nq3w5rr4H5EECZPGhFXpmvS/FzzGNb/2kbz5Ayat66Eiro6a8JohTA7XHBgIuosoB0F/17ZImN98i4sTVLSFjmaN9zoEScXtTTDj3Pt06fs4M/4cvS+sReW/YYis887iP9jZtBa7NYfiuGLMajNKmweeImwMhoyob5m8K4Vt7veisBgJbB1Lilbes5U2LwZM4qgSeQzibYPygUgK53WvHheD5HZHbqLfMg+TId1Pz8ol9AXe4by+3LOPBwNKuG7XNnweP0ud+g6Kg19Fb4xnug6+9JfbXng5jvRLYbj1umFSNZCZOIWpk0bx8QgvXgi0kjn5BKrR05iZ10JC7YuDRlnAbAafsXHURHYN6WOzop0eTSHtHeW4H9jG8BwZjZxY+wogo46tSisdb87l3FTZke0husLgAw5C557rmF/PtMIDzbBh9UvsWVFGbmoh9cH1dNuayY5dSZ+ilap+Hbd8dTeLto/ipfrjpFa+Q7AymR+6wKSTbZ6bGqBsjIXKkT3OuaBQuFgRP675kSMrVlIe0zCkPq4cYnK94hqeYWzJTu6NOIlBYcP06v0EH2knf3wU3f7DuGF+nKNfRCxqC7npxdiCw2jw2Ue+uBWjomnIc2fvtuHXrEdpVaIQReblFlIcBW32CD75cjyzdw6Aixzzuv0CVG7iit2Do61asqaG0nHH484+3p//DABVVmcqLaK8ooitf4qwllscdXMoKTja1K4/S17Yk1T7rgdEVBZfdLYApwzMAGL30f3nGO/ezsuhJsw6qAwvJjPmfrptsv32auaPLO7p4deVv6JvyCJj/7/Z0fc0ACOCDxCma6exwQevdhU1oTWYNCbEHjlY/FwrVE74g/Xd0GHooTsxkjYfVy6uUCVMDPsNW3E2Xet+xbO52XlNawnHqoJWv1a60m1cmXafE6Ht1z3LmeNWwk7nHUlsvGEfGosMrRhcLk/Vc2FSjWP6CdgEK0f74IlmgTNXIrALdurjJmP18MFdCKTDDoc7gvl0428s29CNe0cfZTE5TmWPgX1spv8SRLer+f36g+yb/SNG9wAEQWZytvWH8IJ/Ny3LdlMT4+OcjwARnldICzhGrHYXYkUZqUc+xafG5fQY+J7NZkewWUGSCOhcAMA0pQcTErZxYe6D9PpLDMse5twPLtTP54GCBzjq3sb+RVWozBZCakOcbEYQeM74PHdsC2TkaQ3F8cUYdVY6rc3M6L/Avc1f8XlQD56BbejbFITUhiA6WM/vd8Du2mmkF16PdV0nRQn1NAY2OoESG3Kf5a1mT3bM/ZZLH8OJKSeccxfk3DvBHcsJFtOcQQZBEJiUf4rE2leJbLmbM+POcGJBIS+9+BI9Xv2k9z/A8vZnuC9mH4FuRcSf+Zk73hnBwl0LhzL7HBO5LqyHi8Nz6NP34aVqY3LEVh4evwb33mKsSit9+n5KVIfJTJzO0Z6vhzD7WvxaaAsVsCi6MKkasYsmfHW1VHUlOPMQ2x2Mh0YGjZVdICHHmyXPRzN3/1wESaBLd4kW97+wib0obG5ICAyvWM+oss14qwfk7uUf7/DqwOgm4d+5gBiF7LSc7TDzo8vh25vX8+f4nRyLiKQ88ANmXapmxuViwrWpqM1qlm9JJ6LEH5vSxgS/HEaH7MWglm3A55tiuP272+ny7OGPhS9R4f/FkPfi7zmmhwJRHACOgC/Jtm7EKvYw3jGNCsxgVls4dL+Ni+NVLJ/tOn/6S6kEN4QTUtlJYWLhkGAfQLr5DKvcIXtRPi0TZLZiZvIkliZ+zPYJj/KLXzFjEurZc8tFCpMKHXuqXHq0hYz+LoM7T893SpC5aiv3Z7+HEYKC/pGr3sl+ttmctp9OYUBt83HmWbfboa2kjc6POgmtDcVN5UaibyIlbSUs3raYUn0puiCdU7Jy30gDh4fJ4NGmwHo0kUHoftQ5x/eppyAlUZ7/qzavQlEmn5Pqg+sHySqLKGwKgrIDCSw/w6iz8vwYeG9Kgt7mT9+V5PYeHgI6FlDQoymmIbgDi5cOQZIQJMG5v6mtflRHWtixWotUW4dXvWvODjBBBqS17Y5gr1Yhn4ludIdvA+HWRau4MOdufFt98W7zJqR9JSqrDz+3G2iwKzk+9Tg9Hhau2XYNCpuB5b7PA+BJBHMvtjBxuWzDGLpcdtrg4pJxcwUuXDKeMrNPaXdniceLgJy/7o/07Wgqwog7r+Puz5Yz7vQ4NI56q6w+qBUaqlLnUzZ8GZdmPkSy540YjC42WmNQI/sXXkZbV8ptX6USUic76c2qJjL7fgQgsDGQoL8uOBmcF+IXye1Qd1KdJYOJjHr3IW0ZAGoIahUmn0QiOl7Bo18+PzR77seoqUa0ixh9RT5+8EuUkpZpHeuJaXiUBu+tnA+6j7Nj/uLsTNmOGLDDr3WH33vgYHw+pyc1YQp+jgj7eyTqJ9AW+iYHr5pH8FENeTPWkj0pCoVV4WTORzlegU57qpwaoLsHv2YdK39/lOCOZQAsSfwQgNb7lzLuUDHjivcxqfAEOovrnY5gCjObFzLq6EmWbGrnSsTDANSHWfh87efYvSVyn9+CR3OZc11r8ThEv9TOkm+WEDojlKjaKILdg3lnzjscu+UYSxJlCeTW1a3kLs4lrDoMu3XAFyXRFpbB6MenA1CdWY1yx+/oO1xqPT0+4bQGp4IoUry7mIvfXhgyHmVHa9h5x07qz9fz678vo2+vQWMNYGT5z6j0zyHechPeMd4s3ng9Z+aM5kLsSlo9/gIgIc8Ny2ff8KbXm1Q8+41TvhbA0xrI7S99i3l6Mu3DF3HzupsxmBLJa57C04eOUdmt5NuHPsb/Dn8M5XrUZrXzXRdFF0vSZ4oPjUGNCECvEAW6YNaPKeCawxUEHOmno7KDNcPW8MqMV/AxjXT6KyyhFirXPM+h5TKI2skAdvgXB9ICmVWyHZUTfSsg0uHdQXeGNwpv2ZdnU3RjU/S4fJrCQODfle9UZfVlzqU6llVUUzD5LsRIHdOPTHemapB/X5a1T78URdNMP06PP+3Kb4cr2CcJdiwO9ZUy3y/YV3OIa7d28eqFj3nN3EdjcBOVGVeTO+1evPvGu54/KCef0w87aB0J7FzEdf5v8vdiUjbiVfkrXt+/Q+TbZ4i6fGyILGmD5w6aPY4jFVxBtA08z7VRXYqEb2pT0fWqEadFuCQ+W05B2ff4KyD9cjr+9d50eXahtQQ777WJ/djtcDjyMA9/8DBIAlpLCCt9XyGm8VEALo69iCIhBt/qsxQHv0azx4Eh42mz2RDsNsYXHWRywakh72Ny9dtc4/uiq7GChF4ArQB5KXlkT+zhjo+uZsWWFc6vBOgrSFFZuWHDDcTk1VIUX0ij73HUDreO2m7mria4bvwBLk1bQejvVhmoKIrOsRT+g+TrQGnua+bRA4/ywl8yq9gu2SlMKuTIzCMoBuVNddoCgoDbg7dRmzjTec3aY8TQWoGqv+sfz4f/BvsA3N3d/6+/9P9S+W+w77/lf3yRJLAqhi4gBmMyNT4/0V+4CVttI5WRDiPrb/lAJAn8u+awMMvO1LxsAFQWWaYva3QWRjcbSflJDplF+QBiVrZQp/qGmOzf6QpQkZeah2jXobJ6obR5oBAUWFVWqmIteE9JQxIVSNhotcPzeX/yfvin3NYIMUUGjj/9OxaplH51JfU+W2jy3I0gyVIn+l4bhrZqxhXtJ77uOYaVf4fCrkVhc1DBY0OY2v858XXPgySisBmYd+h+pmy2M/rMMCKqIlD9TcZTEqzYDxwi9ciniDYzguBCftqsSk7vnYjSImFWtVLvtY2SoDd5twOmld7G/j7ode9l++u57LlqzxAZAKWo5Ni0YxydZ0fZrwCHzKeHB9wy41ceT3+P6WfHMe+9efQ0yo47hSP4pVe1kxbQx2LfhdxhvZMJs2sZMbeC+ITn0GqDHBuIvBRdCX+Ycn85r+IaDxndNn56Mo2Lb8f9t/dpN8925g4AEASRshgZ/b/itxW4GV0IHQBRYScvop1Dd8Km1a4k52+2w2btRAp6leibfREcKLAG72106S9iVjUBkvPZdoW8U/l1utCPGTkZBP5uxRKbSHNA7xCJNJCRSoKnB6Nqy5mf3UN8/fNDro8/LXPAts1b5fzsiwA4FAazcoeRek5L7IUt+FVfRGMJZJQGlhigR1JjbbwWn57J/L5qB+OavkaUNBQ7nBpawcpH/rITQLFSx8lJZygOfs35G+80zeBlj5cZfvBdkk9+Syu3UhT8kqveDpusOXI0xUltNHvux939Mi+ElzLN0wLdFkb9dYGEPA+0DkRUn6aUCp91NHnswTB/El/ftZVu925nkNyjP52u2OvJWvQC4vSpGN39AQG1ALlhakpCs9kQCCAhKBVICFjFXv7sh7RqG9mhb9CbmUdghUPizOZBWMvNzOldxq6a+dzdBDYfO93+0eTMfpSdi/cA0Ol2nrNxCznr94CzXVpLCBEttxPSvsKZ/HqwjKCbOYoxpb9T37QYtS9ULrwR93kTMenAorIgiVbClfCoF3j6dlId3gRKJZLJ+v+ILVbTp+PrnCDm7RuOZIeDKQfpNbgCr2aTgr0L9nLu5WaCSk/iVXcQM31DniOpBVpD0ujzDKJNn8neEXqy4pfS7tNOcWI90ddPoHzkNVjVQ1mlxfHF1C4MQeURwtyLbUzKz6TbDsMrYUNBOqW5OXQ2V3NatNHqJ6OKD/TB7Bp4dfRxvr/jGL3n8pj08iQMPQZSqj5glBTFWK03yXnJRBWZmPTCbMrmy/KPA84PhaRl5olZBD4fSNrqNHa8eoSymDJnUFalVJA1Jou6SQ6q5oBEkKTk/Nhy3nnyHaxtFlZsHOnMNThQyiNPkJ9aid1kklG9g4roWIusajd6fKIJ8r6NH9RDHU7/q/KjCpZcdS/Di55Fd+h3Oks/ptPtvMwu6JEDoTmaHkSLwMLdCzF0G5zI8IHDTqvDQZ5xKcOJ0AOwKNup8fmZblsLAZ1XEdy2Ch9toPN6VuB9rLtzD5evD+Tgwk6ax3+FRjAwUSF/51z2Ddz9zZ2Ux7VxeNZhREHAvS+dC+EgxYPWt5Nm/25GnR+Fe7eBfpsXuCeCoIQrbzKh7lWCS09SndyPUWt0Hu5enDSZzNTTKGwKPBv68eyQ+6rb4sn+kjsI1DWS7tdLfnI+8cXxTukWAYkEn7OoQtX0pY/BopIXJYXwn4J9AwFbwTk+IOfKfL4NKpJ9qRwmOxW6dNly/4W+y3t+oBBsaO1+aC0hzoP0ugC49tJqKpt2EVxjpiSuBItadqBZJHi4GWr7Aphu1WNo0VA2cgVFkx6kv8IhJSaJCJLImLNj8G4TaQ1NJ3SqS4JbQHQemJJqX0d0IFt/7IJ7KyNpb/BEUVbMnxGfcS79Fed9/fpsSnqjMWs9sCvBt6GKFsMhNJZgTBI0mjw5EfECX87wp08vsWb9GkRJRZf+IuF6eU95OtBK9vSjWIJ8Ea3mIex1o6qWQ3l/cPHHTQTVOthijvlX7fctf6UnUbF5HZaQSAqTZNDUgK0z9UoOCy72EnhexegTnSisCqJV453rtlfvGHy6p6CR5LyH4qAD9Z4QeMUXTDob25ft5dKwS0iCjStux4ivNvJZj40/Zx5G22dF+uowOqMnSkGud5ytng7dLDw/3MbcX1Kc7AQBJckh5YS4lzBgE0iCy8HrdI4MMAgYiq5I8DtDqVFPo8qITSUQ3/IIo8p+ZYL/AucYDjyT1lY6eZvfuJZ2pQzg0BsTCOibSv4wL2qSfREMd5PoPhqtKZoteU9zyZZBzrAcaiJaeOT9R0i/lCbbL5HXw6IinrCPIO2tBxl9tAbbIPbSiepVDDu4jqvrcQbrEnwTSK590ylLPrBG2hzysl26bMqC3qHB63cEBObltHBjQyN6HA5lx+OtdjW5fZ7UVwZhFW38uOZ7LqdnAZDsd5L1CzN4P/hh/NhEZZyS0thiWmylABwouZPnT3/Ltde+w4lFyzk15RJjssYQUeVSULBLdjw7PFn2+zL6gr0pd0gTjdVATTQ8M2UFYksmlpx8ogrKnfdFNz6IaANDtwFMEl26y3TpLjtl+AYQ83d+fRPBm7sYV1HImNI/hrAYAcrTVJyfIjNMRUFE2TWVFXlzONnyDLd+fxuhZRL75/zBieiXGXD4bA5v44+UTs6PvEiXl83hj3Y4YBtgdlsoyqa3GX6qA6EoC3XnX+waLXC6NpOCAqjuTOGpFgOHx5xDYzJy8/qb8W6Vg34NPTF8cuYbXuk4zfn+59D1thKT/bsjKCv/zq3f3crId+yM2vsaQvcWDKZEHqzdzF/RXTwy/lb2TPQh4KyRaUenITjmiUHdRoi6DSXg3q1k1GlZvkuj0KFXdZAW8Bf5hNDY6kG7bz+/X/MXnb4Kuq1taB2v5X2N7jS3ejDj5xSWbl/qtOHfaocAbRtTIzeiCzCQm3qKXoNsswbqyxgdso8aw3EqPX7G4kDwA/+QXB0MSlEIIt6944mvfxZRUnNu7DkahhsYXraBecI7CAKcrV3CkYrV1Aaupc0gM1aaApqG7IED7+TO5SXsWCqD3qp6ksmsXoa3tokR2Xfw2eg6mv07SPqti17VaVps5c5gn8YaQIBXAD61Er4tcnsFScl7pzbwbuZPTCg4xmejTjjrPZgpJEgCU/ZHOf8GkWOpwzidOAOv3nEsuNhLlM0FJhmo9kCdf77xZ1oiPFl04EVWab5CQODdDkiukGULNSYNi9clMePIDDkXvN0DvSmO0R4LeSkim/qoJKoi5UB2fpvMBj9ScSN7ym6E8Hr2zdtHl6d2yG8CdGlzuTtrHOdil/J/mgsJOGOTmTz+CrBJoDx0J3d9MoWxNbeQUbOFJP2EIe1TmnoJqmogtmohUX0uhyOAp62VP8/HYq5rZ31GHOdjZCnHCVowKOy80R2O+qvrWfHzYm6u6WFGbhFHFtYz81I5AR0LAXk9aXH/k1bDcWyC7IgPaVvNwiwba8XtSHn5YCrBpGx0rhUDDnebze78bLDcebv+DIVuP9AU2IT7g+40BTShFJV8suATYrxjOFh0kMrCSsoPlePZ4cl/KrqR8fSu6GVAxjMtDXTqAVlNAUuiga5vgjg35pyL7SM5mOIf7KYufgp1YQI6UyRKm4ejXiep0G+l0VLmcrJKIgICZxLmkJkwgkavWi6uGk1z5Ch8uqcCEF//HFOKCgjpfhIx8yR+tfI51INQ1zgLkoMpK4+/RtSSUPsi1zr8iVW2eei6W1i5dSUxZTH4WNOYl91KhmkiN7nL9xSmazhydRW+PVOcQWMVOtQ2X4LGpNG79gnyhmuc9QboV1fR7H6IGrNs+ybVvs6iLImUmvfQWkLx7ZqJR38GdjuobF5c7fUCydXvkmeGiyaY8dd0Rv7pSZ++H4vKgkYhz2+b2Mvnxklsn3MfzRFxmN28EJUiI8s24t0zAZ0pigURqwhpnoVXnszoS8tNc45ftTmH1wKvMDr/HZrGZDjP7jYgtAxmH3+NhWfGkDO3jLI0l/ICgM4USUDhLzQePUbllBtpC03/R77Av2b8RetDs5mZ385d1ly8rAnU+P5Im7sMjGj17SIjO4XrdvzOdF9ZMeJxLzlodzCylEujWpEc/hpRhFbtZeezjyWNJjbnHP9+9d/o+uT+eKENhGK47nwBySe/Q9Jq6Q6KozHWlePwo9M/8P65rcTOS0Td30nqX5+jb3cxiABuFHbz0IIvsSqVNPk3/WN/tUaB+UFfclK3061zSfaVSn9S3VnNHX/ewXut71H9SDUqhYopkVNwUzmYamPlPnLvdkcxIFPoSEcwcN6OnBqJdO219Hq7Ah6CIKDX6xEEBSs2r+DeoqFqBl4jo7hh7w0EjQii/N1t+Fdm0eC1nXqv3+jxACk6Bo27BrvdFdwaKLVhPQhzZhG/IB5NZJBzjQEo8dhJj8UdTe9w/l6mRf3CD8FX8Atqw/d2Xzx3ByPolqA3JjjHTN9t4Orfr6Zqu2yPFFjgqOYYRKxCr3bDo70JxZEjbFqyifSCdOZonkPZkuEEevdb5booJB1uxlgn0H9E+U+8pOrnxqS7qfNx+ana3I85x0tZ10PzJ5vxaC51shWd7FMHsw9w5rscAA4YVXX0akoxnmgkOT95yFlLQM53O+5MCqo2E72GXgSU/2D22QWzc30oDniHrcWyrXOyfgHvNitQ96uwagwY3QP+1qsDa5Wd0Nbr+G7KUeIanhryDZv9n1GgKv91FES8Q4dnOwqLgNKqZHAw72LM9aSdOY/6cgG3fXcbXu1ezsCnTdGDQYTWel8kwc47qhspCnlRtpdHvAuhi8kKbWP0+VGMPzUGwS7L9A+UXm0hu4p2AeDV6eVsg1rtAvFa7BY0N6xgx7VGCkOfo9FT/v7AWKXf48moPa/iXetS0hhRupHQlpuIanqQGM04hpV/R2zDE6isPqx3G0V/HNwyqZArY/voCIZW31Yimu8C4J5RazkY1EFg+1jaA4LZdN0mOr26UakAYzP/aniDXSHgrbMQdeEnvLPt3PDLDYgMCvahIL4onpHHq1D0VWGkA4vNIYktutoF8h79n4oggFd9Hh7NpSgiwzHpfZzX+iqaSDq1Hp/6K//x3v/dg30KhYLY2FgUf9c0/v+o/DfY99/yP778JxnPLl022TE3cy7mV7QzxmPUOfKrOAyOS5F3kxN5Oz1WB9IdgeVL5WsDjqM9C/cg2Oys3rwapVVJr7YQgOLg1zkV9Sg7Fu+nMV7epH16JuHXNZtuXR6Hmzcg2kQ0/RLbSt+nOPg1zMoWBOTtSBRlh+HsfeHU/nGJUwkjKQ/8yFFzERDp9Oxk9yp/GuIm425MIbHuFX4ZsY4nRj2FUWvkk8c+pSXuJoLto0mse4mF5y3My24ltnoe2m4r6+77mROTT6AWhzJ23DsNUFaGZG8iK+x2im2HnQZBaqOR3574gJ/uzefjtU+TH/YkJrWMthol3Y1fsx/DLiah7hRBALXVtXHfNuI2ZndsZOqRaB5+/SG0Rq0TARTpdoxl0du4XqnBrdONmr01TD06FVuX3N/jwnbwyYLhTLxPycV5T2FTDWW4KRSDkLKGM9T5/oLOFEm3Y7+50LYYq9YATvSjK6ArouDg3IOUXDOXklErmZZXNuTZq+33srSllSbvMro8hwaMz6sTWaapZcN1G/HItfwjeCCgpN2nnSOf5/DNfUf4e4kuj8Y3UwK7jaOpqRxPGTXkelCdP127juHW1YHCpiOkfSU6U6Tz+plxZ6hZqKXVr5UAUTYo75JjI4y7dwu7bmindOQ1tIYPRxKs5JjgxgYoCj1K5L5HGXGykta7WvGqnogoqfmoA15t3kueLYRXPltM0m8irRN7OD5xA126HOfv9ujyqQ6pdv5/Uewxp+xbl+4Sr0g67sqPoCF+CleGyX32ROwh2s0q9lwJRdVgxK2jFrXR1Z+CJBJeHc4DH0yl9/Rl/MwPkVj3JlqzK3fVgAE5sOmLkhIBUDnyfdV/fh0zDvhS/GoTvyV/63wfAe7ZdyNhpWWkn2lhzNkxKO0Ghles58PQRr5IkZmLvfP7KZ9yMxatO5LD4LWL/TR57aFJN1TGcqD4dk8lpuFRgs1TnIeTouCX2DVa4N/KH0kLOI7/5CAua49z4GoV58bK0kXxKnjPH+6YcYXfVh3BcvYivr0e/4/YYh5uAreMLuPXVfvwSg0hWp869EuSLEtj8rRQOGENmdNc/R1aE8qinYuQWiXKR15Dfko1mcmTsCkGBwNlHf+jKRn8mRY35NH1IfW0jvTE/YfPSDz3JxoHsi3HDK8Wmwn4y52ushI6M1cyquQ3ABpscLgfajT9dHr1oM1IIHtRNkatEa++MXx1/CDBin5WZo1n7HETR5/eR8bXUUz7axoeBtmYjGl8lBsND+JR6YGoEOnV9WJT2v6R29GODc/GIiaeGCejXQfnrOq34dHphto8dO0buM99axa3fn+rM7AAIEoKAstOEZZ/iPIRqwhJeIFjVsv/MpfoQDkD/GUXmRY0iow/PyIpJ5T44ngGDh49cxoYd+JJToVV0riwHY9uDxkN6gxKyO2piailz8cmB9QQSal+3/kb2TE30mwrIa36Y0aVbSbaMDQ/QU3oWQ7FvsfJMR9wWf8TKhV8n/02yzf34976MTVJs7g4rpYW/xZEUWRy/jnCJTkYOFUHAY0GRmSPQGPS0mRMhJZM+Gs+XHqO1IiTdPlFkzOjg363flmKpuwH4r1Ps6c9kPjieGpT3emNfJNZlyqxNV3Hp+e+YV7Edp72Bs9OTwIbXcHJcM983pg9naXz19M7Ygo3fLWQqUen/o3Z50A/SgOBGgGlQmRWThXxdc/hLsLowkSSCufJVwWQRAsjNfBGSAW3mL1Z+9F9iJ98hKGpzDl3wh1TpCOwiR3LC7ClRBDStgqQ6JXgow6YEXCBezKnsfSjsSgtDifAEGep4Ax02BRqrBbXHBMkBanVn3CrZifu/RmMKJcZ42dNkNnrQUBLNm47NmNvr6NTf46oxvsBWR5XFCQ6AxPoTzUwKXMSSqsChV1HphEiNT1MrnqJP0dtoy46nMvpl1HaZIfkLU12bsr7itRKFT6l0Xh+shnvhnxXDkxk2db3su6g98dClv4W66yrfF3+N+CKHrtGR5u/Qz7Wcb/S7o7C7ob5yCmiSsxM7vyQCbpbnQCh4RU/MLHwGCukLcy92MKa4HcBeNYbFugh4GI6SRf1TkeA2uHMWusJpVEtmOceJ2dyKrrrlmJXqOi01zPHDX41eVId8Cl6OlFVlpO610vuZEngwbFreG3mTBSCgqfefIppv/2BR+dQycvB68Tg8uj+c8zKnca6O9fROF5PxOkteNflOQ+8Cps7oR1TuP/jW9B/9ynX/zwLQRKcuY1jGx9nTu1ResMfomfY/cQ0PUSUIQmbpOLHnDfolQwkT7rE6QmFdLt3Y1E6mC9qT/CIxyKoyB6eTW3Qaf4whlMSNAi1/Ddlib+Xv78XrlxIrsPh4APzCwVLOJQeQXjUR3xWdC2GD+4i7uAg9ogg0G32oaBlIu5CNB796eQPd2Pz6s3Ozbjb7MOqmB+ZZyhHSzopuXIOj8pImQHxbx8Y9ddoMkMF3MwaRIuSkFaZyW0GWm06DjeO4JmgtfS7m1GZ5X3XvS8NhaQlsE7B4+89TsAeDabuu8gLmIlJKQPzlHYDwW2r6PXQIaiUqPq70GoHsaIH2owNld0dldUbjULDsMp1zKr4lU+C6tBM8aM1YjQVgZ+S5/4JJqGTqTpY6mEiNaqZ3Yv2MuqUhkfff5SBNbvFBp4B4+gpayO4PI+VW1dy04abuGbrNdglO8befj6YN4qxxomsit+EvsdAUEMQKgdzodvsw+maqzllzuLw6I1cnBDMjsU7KHEAqgRJxKfNB6vSyqX0S1xM+BcAwUr5939sU+Pd5M+Vma38ds1vzsDMo+NvYtuIF3imdzXpVyZxYWwvl9Mvo1FoifS6zBuzZnBFiuCVG9bz5yIdcy7VMyX/PBc6DvFau+wY3tbsxV3vPYpoEzg+5TiCpEJhkx1ZK0JOM238D4T/NJ5Lw2SmUK+2GA9NC7OiNpGihpjSGEY9pGH+3vkAePaOBuTgUq3PRiqt55zv+pA13TGffJv9SS6aQJJdZn31WrwYF7qb2Z7VrF/zBV8/nsWl4TXoFHrnvQPSwKmXfLj7yzvxbvPGT9fIxPDfOVZ5LZLveBKtk3GX7qY2yYxdtKMQFChrNvLFwkSWGhfzx8P76Z61lOqINscYKBAFO1MjfmO46Eu65yTn+5RQ9zx6YwLufWnYRTu7V14hb603l8f4DGER9WlcgWs7ViTsCMJQ+TWA6HwtkVf2Or3bzY4laZEeRuWm0+7X65Ayd9kwbgoPgnXJKO1uzNkzCY1Rw6n6mdy6vYqnJq0m2ec8Uy2tnJ5wmqiiOlZtXjWEuWdT9FHQfZYut2xCWq/n1YwtToaw/NL807XzbgcoS8DS+iT9nlGIPV2ojN3OtRGgULWRi2m/UjRiJKn5kxjX+NKQZ2S5L2D3uXhmHpmJxqRxrk2rG8BwJZrzqiVoWsJwL+1C19yF0q4nUB+EmznKycCTJImz8Qs5lTQVk2qAUS0iINL94x8Iv/6Kpv4VDg4Pckm3OdpjtduIaL6LtMpP0YkGdo0W2DVaIDdiLSf9bmVn3U66v+hmwqkJVHVWMXfDXMraywhsDCR/dj5/LPqDNXlriW4YGlzQ9+gx1bfQN6LfNX9ElxN08+rNeI4ajsedDYRXh7tk2lGAACaNibrEmfiInzLrcgUh7atc15FzNA2Wu8Rx9fF3H8d/ywVCeheSUfkVYW03OuskWs1oetuwrLmd/JHyOUqD59+CNZLzfVSICiTBzr9b4a3W6XR3eCJazWxfsp2ymDKUCrnOv+U/xSvGKCYfn0zq5RBifb4lvuHfzuDdAIDGbgdJqcKqkpxjBFDnvZkziXM4bByav01AIKBrHhOKDpNU+7rrnCfKT/2uCybqYOS8bM4uMFM+9mV8FXuJ08uBtyav3VTYM+kwnMZmq6RL8RsV1r/w6M9gUkEmsy6X83rs18zdm+D8zYKkAld/CSIhinjicy6g6uymZ1CO1TobNBn96fTq5MqMcrp9vJzXxhcepsstG0PLGZovnHbW++9BMQBVacsQFrdLQheQJFQmG27GQHw0Afh3LmBFPTzZJNvzPk1qEk98ilvdMZotlXRjJ7AMMhoCCaruQWW2cCn9EnalncQaeS/R9yeSw0myRhcgVJTTGZI0JO9eZs015PUtYtbXs6hJmoJLN2BItdCG+HBy6Wo+X/u5cxwNXSJjz4yFXol9MX9wMekVegaduTdLK3jk2kc49tqxfzxzINWA7i0dSZeTyE/JR4FDRQc7gaUn2bliPVajFc9wT8SUJCxaF6slpOAwGQfepau+G427BrXXUDCq2teDuPlxeIZ70rVgFU1RY7gccQ/n41bQo83HZpewmqxkf3UGfWfbkHvrQ7sQpkxm9fbV+N+2mNQa11nrky4bE3dEUVxRgO7KJl6+8xXqvLfi71aJl7aRT5ojaG/1wNxm5ta0taRVf4xXn+zXEUXQmLUMzxnOhXsusGS7DESca8yA48sRBGgPTsFy7Q00XmrkxE/lvPqq/LsDgMS+LX1E/fASw84pmZlbwugy2XehkHQo0SIgUhHw6ZD2JNW+xuzMHwj/rJD+iwWo+ztdwT77gNTsoGDfQP5PB/HhaNBy6lQLqZrfzLe3f4uxf9C8lkQsKgvf37aL/vFJxLStxKd7ipMx5gr2mdDh5bztUMXvjA39gwh9EemX03n4jfsxNGZS6/4B1b4/OL83mE2ms4QzJmDqkD3WrGij0SwDzwYDVARJJGtMFoduamLHvyrYtXgXg4N9AiIfPfwR7TPTCKsNQ9+rd87rZvdDxJTDfSOP8crzr9AU2OSSJlVoIGQhG3t0nBt3Bs8uD8adGYcgKbgu9n68esYhCVau/nUxO1bt4Perf3e2Qat1AfJSzqVg2r4Ps3oo0GZgve9IkwFTFp0rLUto+2pGVPyIQtLgp4ghvPVWArrmUxzyEmfV5wHwVsjvz6nlFo7MPOIcw78qb6TB73kqpj1I/vjpzt8SRcCRu32hHoq93LGq5fX74OyDiMJQGc8lO5YQWtlFl/QM64O9+b1AllxVK4bmjLRLdsafGs+aH9Zg6XKpKIkiROTuI6T4KHazVVaucBSlvzdVqfPp8v2bH/Vvfr//XYvdbqehoQG7/T8HUv/fLv8N9v23/I8vkiQfSgeXdncZJVof0oSEwMi8taRXfI1Hnyy9UO33LdX+39FvcTFl1A5pEKvS5TQvHd7Lb8t/w6aUsChlqneftgSzxkxxfClxZ2zMOH4vvt0zsIn92BQ9SEh4t3tz7XoDofd3U+nzKj49U1jtDvZ4mNm2iZF1snFu/y6Sbvdu5+9ZtLUIkoIbfr6B0cc7sKm09GpK6FfVkuKfyfyoLUiiREBdAL7V2c775sZ8z6fzxiBNiqHypucQ/YNRSXGDckjA/J6NXH1kD0J9PTuW7qXcbwNltqOY1HIIok6SKDF00q+TWUwDBsg9nvBN4hh+6o3gqj0ziD7lg0+rDz49kwg3xAAwNnQs6Xnp+NW1kj0sG6vSiiApUSjgr863qeqJYOE1x9j8wmbas9uZeWQmSou8UZW1j+Cb/Dv5vPZ3KgM24FP6GbXKyVyIkZ1GA/IIA0WQVKisPrzTDosO/URtYzSCzcqBW9s4PeG0c8MF1319vl50BKcw5OQKRNnaef6V55ixKRB9j8vB8LE/rDTJB0ldvw73MglDj2HIvU72gORCkw4u++bvI/8xG7qvPmLq0Yn/uB5TFkHfriOkHf3c+Vla1WdkVMgSqvkp+dROlHdAvSgzkfb0wdSKaB6xdtHpY2HftCf5Y+owioNfoc4GP3dDf38wkVf24lt7mZb8ZuwWO0a1rA9/yvcznjeuprzGD32zgMkywIwabOAp+OHWHzi8YjGbb1PRGNTo6lNJxIoRk33AES7f32nWc6rNn+Vf34n+Qjf7rk3h1KRcerQDhy2RPrc+ihIaEQx6YmuuJbbhMbRWmetW572FGt1beFZvx/78S/R1r+R87CpMEgwvDuEuy1sc0XbS5WlhX/UmDnZ+TmTTvYzWwMZA+MvvIs0vvUu7r4m5B+YiSEp6NSX8WDOBFpM3nygWsDTqGmKOrycx8wemXviFUSVbMRjlAJpR2eAM5hmVDTR57KPD7RyBnYtJqXmPSNNiZ/f0aSoAWXLt37VqKk+no6juwSa6Do9ZDvD7HDeHLEJwEAb/cL7+vwna+UoJ8yequS6yi7K4Cip353LDuyudjDQ3YxwgYugxkPyNN6nHviKizPXw8LrxjD4/moi+KLk+sVcPeX5qbir3fDKP7Bf/YMbBOGwMRXuC7Ci3BYVi0vsiSEo0AixzE/GOaWfd7euQDBqm/LGV4eeDcHesqx/4wZs6LYYuDar4SM7NOIdJa0Jhc8Ni11Bh86Tp6n2cmarEPdIbgIwrC5kUNIcExxk95eklPNX+FKYuE/oaLWqT2hmwAQVTjk0hZkcR3vVXmP3nLCIa70Bp1+PT4k5qbiqKZANf3b9TlokaJHM88vxIrvs2EkuAnlMTTuGh9MO9T2Ym1LsdotrrDdR9bQh2G1ptEPesfYv5ouZ/GfA7AyzWaHjjjigmez5LXcwo9lx7nh1LdiAgyEasxhc6pxPV+ADBa5N58YUXafNpc0r66izhjC88zETzRk4+1iZLmgoCMY2PsOC80THOsqzpQFGKQyeRe5c72l5X8EYQwC4puX3EYzw7dwUNsZOweyQT2HYV/ppQFJKGpw+eYpvtXoSm5USXh7P7qt20e3fR1eI6GP+Z8CY3R2ey4cYtxGruYGTpJsZ5LIPaXdglgWer0kgsTCQyuwuNxR+dOQKlXV5D3zn7Ob9duRNFiAdbV2zF4DjTTInYDMAe9QEuuX1Cn96IWW1GoRh8UPunjOdAXxmMKXTFwtKtK0k+uR5N1Z3s7HoZgGe8wVcBk2olkASEjg6krm/o1ZQS4xXHvDqYb59Hu9aIf1Mwi3ctZWTx1yTWuljNP5Qv48/IUi5PrcDQUki/8SUnEKdHU4SAyK5Fu9i3pBuL1kBvTYXzXpXNG7/u6USJU7AqujCpXKH9Y4mXmbn8Fs7OreLq78Yxf+98556xrB4W10mcTBrHpfGFbFy9EZtCwqKUx+KEST5gqbvaaQpx48SUE6gGyZ/lSOe4NWY3S7ffAsC55JcoCXoLgOHKa9FYAun07KTX30hFtGzD/F02d98LhXTNX+msk/j3gEpLG53+saRY1pKmmf+PQJPCLqP9dQrZubDaXZaDy/xzJGOPerNw92xSrqQgCiLuIozSwCWzkmYblMRksTd9A7V+28g17WG2Du5UN5Bemcxbb96KzdsPt46B4LhIU3cQHpoWPEXQGXUo7HZ8WgxD2jOATpYczs4z8fPZM1JLq88vPOffw83SeDz6g/Cqy0Pb0+Q0CwymeOY2H0RQRmMNCefAnCNIgoRScjmbBAE0vW14lGWj6u8cYlIsUO/n8wBoCG5l3Z3rKEwultcBSzd0FvC+mEX6tQfwbvfmuo3XMeCgeHj8GnLm3s6T3i5mX4+5R0ZcOxQBBtYMqwNdO8AEGbx3D5ajbrc0YNRU8/rIV3kkqIr6hLmow4Yz69AswqvCERGp6UrGbNcywvYSw/a/Tfo5R7DXUS+FaCXMIw+puxH/inOk5sYjIdHh1QHAKodJpPL3JG+mP26tVqacf5vppnfJNsGihuU8mnM/NTY4cVMxl8dGDWlLr0Hi9LjTIMGCffMJqQthySK5QzXWAEaVbaYt6R1sLe0knv6RyYNUcwGmXrlEdNfNzGzZyrzsNqb4rEKn7GbNsKcZ5f0DaTMKkZC4ef3NhNSG0E8rHTbY3QtX1cqOk/pQK4WJhQiSwI6rsnll5PV8FaRGOyWbomEux1NNmEtuuKxjOH3t09A/0Ypbj523n3qb5gBHIAmJn6/x54sAOwhQFW/gwqgLgwIsIu89/h77Hy5m2zXbaAiSJblOVq9k8UaJB8p9Wfv5WgKLdLJ0vMMbUt8j7wWR3cOIu9KA0myj26MbrUJHQ08sX5z7jAJRZgL4NUiEX9mHprdtiJPs3eEnqI1No2gpNKRb0Vj98egbxh/BcKrbj5kduaz74EWefuNpwqvCce9Po6RtNFl1c5nnBil5KQDElMl2/wAwst5nKxdjrues+Xtn3j1xSN49kcU7FjNjnZrE0z9Bv2w/qhX9vHx0O2sd6q7j6r9nzqU6pvqsdt6rFtzwVASSeimIoMYgvHrH0mcKYHPus+wqegBh3ik+ydyNygJF43uxqC0yI0GwEuZRhL++ivqLDWgvn0NrHHC6KYn2zmZFytuEuhc5GCByW7SWMGbkFhLb8DQIUB3dxrCQBxlZ/guBnQv5e7HbYc8oDbtHK+i0NTieL7+TY8+MxaxTUTh+DYJS/sxdhEe9YVsITDwxGYvaSlFi0T+YQqII/e4BlMVWYBft+KiMJPvJ59por3xSDWH4KqLQ9Zrwa/FjaE4+hWN87HgY05gVvALP/uGDxmMosMDTlIafAi5HwPzpmykbtQLDoe0EFx0dsr7lqL+iIPwZlP0VeDYVI9htQ2Q867QpXJoi1/G6jdc5ZcpBXosfS/6UthhZ/lHVIvfVYLlVkPd9uzj0DDpQVMPkwEhRkhxodYFaHDn77DYCuuYR1bwWrWLoeQ1kh5ruXh3Zw7OHfN5j6MHjag9iZscw66o7hgQAANz63Oj64ySGXAMqqw9Kux5BcDlBAaxnKuTfEO1DmH2CXcC30Bdtd9M/6uNqs20QqGgAsCJyOf0yJn89HoXn0HU1cJNLSACPljLS//oUqa2dHg/5wKHFg8HzQBJczxUFkR5tHrt64dX2PI6evY/ErJ00+zfT7tNOn7qCeu+N3DfuJkYoeogpiyG4qs15LvqucS1jNfBraCneGVfx5wdvY/jodbxbrEPaMjiA+X9WJAlsgpFaSy492kL67PBzYwIm4RkCg15DbfNFYw1AqxgK/r193e1M3P0rM7fnsr/r4SHXrL0mPJtLaEmezBcPnqUyqtJ57VXv3VytTaVk1CqaU2KdnyuBpXqY5puLAMz9bDRJZzNxM8bi0TcMv+6ZKG2efH7f55y/qQpjzY2UGmY6A9EDJa50MgHrLxCT/bucu0q30ekzAki9HIZdVNIUOQZRhIS6F9Hkb2ZbaBkfXJ7Aik2xaHrruRzwEFldMoO5yQZtkpqr9lyFoUvBtmu2YVZL2BS9xKkgL6mce0P6uTiyEPQGGjWbyEycSqtBBq/GeF/kybH+3P+ZBzkZZ/nowTf4Y9oYp00xMA5y/kQXsxTAu1XJVXuvQlEoOOUj//4+Rp6P5MhzRyj/s3zI5wP56ax2K6EeoaxIWUG4bfrAL6Lu76QltxGbWf5N36FCK1jVevq0ntglibaSNprzmoded0w5USlijk2mzyvUmVbG2vUIwsuvcOXXK5x8bh/BFTJgXbTpGF72I9FND7nymUoQ2XwP8wyyGtTN7rDm6CzSC8vwLi+hfmcdtT4biPc9x9LEjygwuXPLu49SfVc1JbvyUZhd4FlRhJSGvRxfvIiQ2aE0BDXgKwgIWEEfxTdlz3Iu6W4aovp5pPoRfiWUTl02VrHb2a8mT/k9Din+ZwB1oM6+3dOdn/l0T8WvexbexnkIEeH43ruKloiRzmBfRPNdjCzdQljrzf/I2SdKKk4mTaJFe4ZZh2fhv1uHWWOmv2cos08SZRs65MtzzNp3Hwn1zxPgwPkrxIGcgmpeMbSTVCOfO7xF+PfUZXw1fSGvJ7eSMyqbiPxMFm1upzjk5UGtEhwzYqj0/EApDXqHr5rW/KMvnKoeks1pLwvSP/e/rhGRvP3E29QH1zt/S0Ck3ApXzPDwx8+yeuNqV2CspwJ8RvFYuzdVobXkpubS6tuKICl4buQnTC447VQ9uZx2mZzhOUgO1RqdzuUTDK0OxXbhEh4dQ9UCAjsWkV75BaHPX0XWohf4Y/okDmWEOQFutd6buBCzmhPdPw5qi5q320F/OYaftk1i3iYfInqmk1D7MgGdMmgqyfcUhn7ZX+ndUM38vfNx73KX92a3MF4Pl8+4mbsn4tPqzoVn4OTkkwgIzj5XCzoOLj1Ifbg7lVGyWtOADTegEDfA9LPZbXh0eRDYGIj0N+ZlxfCl1CTNpuvf7xCX5WKhKr3daYoeR7/nUH2rgf3/f/dgnyRJNDQ0OOfz/9flv8G+/5b/8cVuB7NSNgY05qELicqoxHTkFGHVAVwdfic6ixxkG5z/pF1/ivMxK/mt+bUh9y7btoz0kilczriM8B+otUqriHd5C4nlS4ltfNypOa0UVBi18mHFLtqxKWwIkpJqC2ztEcnoPsaNd+/iu3srseolJNH1cveoSxCQ0TQaox3R3MuRtEQODwtj0e8lzNtajWAXmH54GhF5+ynVyKwahWBFpTBiE4zYbP3c2LiNmdlniNS7WCAaPOjxjcRy1WKaAmSKmGpQMvrzoQIVBm8Sc7wYcWGE0wBZ7IiB9SeXsHvhIZKOBTHm3BgESeE06EVBRFNVjNrUw+6Fu7GqrIjIzD674MaNh7cRWi4HxqIfjuaLe77A4kC9aBR9tAldbL64DfqPEFPQSkSFHxZFh/xsEcJbbsNLCHOOnShpkABr0Hq0zY8Qsu0zFMaBpLaD2BaITMicQMAZF3PNVSReFLeSNKyU4Co9qzfJDgY/KYnZbjDTlMPn7okkdg+j6oEQjMOGSoAKkpLU3FRmrR3BtD/lfm7xdCXkNWlNGA02bH6+9Bh6htzr0z0Fo99ylGPSKE7s4mL0zfRoC1Bb/fHvkqVAg+uCGf2SkfGnxqMUHMwmu5J0yy5CsjeRUHMjFkUbZlUz7QZ5o5+ghYTeu+ldGM+Zedeg3hJATaBs+F9rgEeMu4hXNPL9bd9z+q5+eKGHZduWDWEHiI6/JVHCphhwKDrkRQakxBwSewMoz8ev3Mj91WkcnnOArgw3KgM+Jzfyfjr0Zx33i7T6tbJ38SVMxVUMO/Q+SrNLPqPK/2tUrduJz8nGplHQ7FvoTFjcr6lkb+O37Fm4h5PTWog9FklwXTAqmzfDNLDaAzxEG25ePRydVc36NesRJCV1Ppt5y/AenRi5IeQ8gRcDMTRX4N5WSUquB8Ed16A1O3I6KNqdm2y74SRnExaQF/6oa6ZILmN7IIB5mwd0d+oZtnk52uPNJF6yEF4lry9ddrg17yoeyRzJ7EMzkELDKb22lZN2/m+xxTJt8K8ZHfzQBZIggVKBm8GD8Ja7mZZ7hZm5xcTXP8snZZ/gmysfhtUWL9fc81nIxblPMHX6PCJzdpBQFDPkN4xaI80BnXSXNDLy4jCUViU6U5Tz+rDsYSR9Vk735JHsm/sLxSGvkN6xmm2hdk6lNhGfUIPkq6XHJxKzzhNRUnGXBzzsDWNOTeCOr2dhbmynzyIfOhSSjjCPAk7b7udY71o6g2cgKkU8RsVRMeZR/JRRTrSgxRE3v7LlCks+mE1obagTwSZICoLrg/Gs6qIuYQbjvr2T1LovUNrdiS8KZuXWldiazU7H3RjhHlefaEx0u1voGRPKyckn0SkMRDc+7Lx+YO5O8scvwr3mZ2rcHsF/ZhiTJ6xgqghzNWp+Bg4BG4CpCiVTRAXPvPIS19++gs81Jfxw3W1UxjVj1BllKSYB1NXfc2bhVdzbMYKIzUFyDhHBtf8o7G74dc8ksiiAkd95ElQf5HK0SxoXQ0myyVIs2DGoh+q6P/DJA9z7/nU8++qz+DZ7IQgQqC9jTOhO/NwqMCvaWfGFkmu33MIIr5kANPZGo/L6nOluX+DZYcC3azKh7TehEKyQ8AAsuEirNpJquki55E7sljwiGhYRoUuFsV/x2LEKBBTsn7efvMleaLvl/TfIUMpjE27Aagqmu+BDhu+KZ8apZ1ii+JzgYDhXt5A3S2eReSaIxMO9qJc+jYd2F0nuo53t0Zkj0JkiXQjqQQeSgX5bds/vWJWQmB9Ao7UIgE874ZYGKHDv4OMHP+G3W/5k/6yvMaprnYhErVI+qHl0KZAqKhEkO+79aaSo4Xw46EUTX0cWcnFuCUFlp0g97wp+evQPI7rxEd4ffomJx+MILTpKxUXZKZhc/TYqB9su27KJI+lx5EauBeANX/BUwFGvZkrSWqiMa6LVtxWFI1/WB35QkFCGT081vtkC7d7tSKJAr7aIBBWcdigIb1m3mrFHZJTrgAzYOC3M9yxgglcSajM0DNeQl/g7FpV8WBMUFkDAqrJy9Llsdi2T0flO9rvjEKrqFvH97h1GnvV19LO8zhcFv0x21Br6FqfTFDUGQRSosV109olCcABeHGvjgON3Sg1MrIZdi3ZxeGEjIy5myHmSlAYCFHCDBxztNzD9i9uJv1TLX+b3aPbcx03eX/JuBxzwcMlhn7nDnczVtY65IFLTGYdGacRHAe88/g5ZMyagVq0mufpt3EwDufdcDgtJsGITjdhFE8O8i7kj8hhTz0whvDWBLx7O4pvrZrCrbp3z+4JGQ8HkO2hYtZDMSUeRRAlPW7zrugCxWZsJObsdsecAXdYWNIo+Xp4+lz5Jz1PrZ7Dql6kD35b/Kf0WdieTLnQQoYIzY89wZMYR53WNg3H9vEv1hgOlBziSHseFmFVD2mR1yHgGBFmHfH42bjG7/KfSIcms/IE9+vVLD2FQtxMzrRFrUAhTTkwhpC7EGUQ6WbWcfcYU2vz6Cao2MfvgbOcadHXSe2iUPfyYqSIyV5a+3rJyCyaN7JSaXAN5005wzFqBX+VK9AXZaPvaUCOvUT32ZgREdH06lEYRtZTO8LIfSayVHT5d3hL7FuyjaZaZ79f8QFlMGUlJgxlh8r8do9w4N+okW/vuJ6jjarn9HYvw6E935tIe+L5O1cWC+K842bGSD4pWI1maCa4PRmPSME68j5aGa3nizNtkVfizfNvVNIZY2b1oNyCQ6juM5zQV+NVsoqblPo5O3MSHD33Ia/96jTPjzyAKImabjh9z3mBq1C8o7Z0o7G6MKd7ByDLZsZHoJ0vI+SgE9D16fBu60fe4kOUDfeuSph16zLaqbBydepT6uG5ZDt/x/W8vvs+dh4zsa3iMvMl3kJfRimAX0Ci0tPWHsKfkPqaRz3ONMQzLEgksP4PK2MXsoBuYqIVL0VputEbiPTWA4Rmv8oCQT2TzPdw54m6menrhIbjR16kn4nIoNoUNk8bE8PL1SIgYbXqMEpycdJJLD3Xx5T1fynPM4VR07VN20ppeYVTJVpLdXbl50hvfILb6GgAujQ2lUn0GQYAZUT/x5uyZxKgg7XIaqb9+g3fdFed6PyJoP0967uSLmBr0/UH0egaT2riPCLdmrk17jWGBh8FmQtnTQVjBYXyq5XdBFESEIHmvS/E7ibXfitjXg8LmYt7dPuIxOo1+/G4rYUv1R9jEAVbzUNY9dhu2U2fxaC7lDleqVToMpzkTfxWVHAens1Ieq4H1ecKpCXg2tPPrnFS+6JsLwCs+cKcnPN4s8Me1v3L0KseaPEg+rdqYz9aWZxF6j9AWqsPQPYUVGcd4ctJ1zu/8JNzNo189RWWCP5+v/ZwhzAYnY2IgfyJ/K4Mco4JATPudRCkhTQNqbR1GdQdZk0s4NOkxznZtH3SXiMaooSZKx8XZt2PWD527aX3HOJjaTfGYLIf8utyHqWpY69vBsNoMhg0/Q87twRwe/inFwa+SuFFm3/XoZLu6w9zqfN7AGbTDLYvzMas4MHwrtn+/QHF8haOdQ0ErVpsLFa8YknvKkSvukkT/x/24dw+1obo9uvF+wZubDt6E14ihzIObvL4ipm0d2lGJxBTHMC+7lZjGR2Umj1LeR5PykzA7ggpNAU2D3mk5Z9/0j6aRdvQLfKsuDnn2wLlLkuz/IbeiyK7Fu+hN9Cfg3B7c2vKwIa+7Ff6fcSLjLvIylEhaHQFt07i5+wrLFN+htBsYXfI715i3I0gKwltuZVj5D4z3WUy9z1ZGa6A5ponE1MOcmFqJh340AR0Ladb/xcXY62lS1BHgEUX5slTa/UMJ3+hi6K1yh1luMCd8L40meQxiiwO5LuR5gjqWyl8alHcboNL/S7Jir6HOe8uQttvtsiTes7XpVPt/yyp3aKWPo5XXQ28vQSUn0bfXMMZn3pD7rEorZrWCkxNPYlNLXAl/mMPpkVT5fYt7tA8X5j2O0cOfmQeiZYloR7FIKoy0UOf9M/F/HCLligxc8BThjxD4ZOTHhDf741fjiV9dDTNzS5ypXQCsKiuqXgVjzsTj3WoaAogaXvYTs0/KaTiOT69kn+I+it3WM7RIWMRuSoPeoLTvPN69YwnvWkSVJZy2fhVdHiY+fvBjskdkoxI1+CtgXwg8aOhh5+KdnJs4ACYRiK/7Ny/1P0SE2sxNHtDu04ktJo5RpwxMPeSL2QEUe3v2JBJ03Xjtm05cXjcmVYMT3D1QPiOFu35OY/pvG7n7y7udc7cpyMr3t3yPJMDC5yeTdjkNvTF+yL07H96JR5wHX634imWblzk/X7dkHdZ/W4n/Op7DkYdZP3M9U2wvOXpBojrtKjKnPImg1ZD9QzZtT7yJe4srYNgYPY4Lw1eh8tSz/bbtbFk0tC/rT1Xwmu41Nl29iTiFfN8AAK3DqwNrfAx+SX7M33ADlfHyHPDsG0lY203ElvgjfbOOjUs20nlYljNf7C7nIvNXwIr7tlG4/GdOLPeWfQkoyG+exMtHd3LR7MOp8aeQJIn9t/6KaCzE5sj1KgjgYUpHJ41i4ZZldE8IZIRxCmrawSOJUy27qPfaSK+lGn2gnnNJt3A8dQRthky05jDc+zLwnedLw9ybKBp7/ZD21vr8wlbbzews+9W5TwEML5eDQv0eQUgqFV275dz2A/uyd+94QtpX4NGf4TxPpVZ9zLjCg/j0TKbdkAnA7oW7qZ/ehV+z399AJ4Lzv/aUVPo85TyXf8/Zh2B3KKx1ALL/4/1TP1LQPoz0pGoOLtlHRXI0WaOzhspkO9dJiXb9GX4p+YQW97/+w3VQqSDG4cYQJAURlREkn/Ig8biXQ3Vr8CYn4tXuhaasEfe+VPSmNOceobC78aov5On9MHQpHb5aBzj3yuuwfwzfjP+D4dWHOTY/l+KEYieIdnCfTD42mamXpqK2ewHg5ubah/ZctQchLIRZewYUR+R2ePWNIbL5HlINMmrNqK5xAvEBut0uUeezmYu9u2j03EO7/ix9GvnM16ctQ9evw61HQcJxE+PPLSWgS1ZY8NdXYe47R0/vq0RdyWL8mfFo+2VACpIdBRJxFfBlaDFZ4wvR1tnxavdCFFwynk+F7yZnQw61wx6l3adnyPgOMNglJGx2G8n+yRyYd4C3n3obldcghSQRuvxj6fGNpC0kje5BLL7/FWHt7wHe/5b/f8p/h+G/5X98kSSYmpvLhGMVTMw/SUTT3Qwr/wEA/0ZPxJBAOoKSiR5kww8syhablT5NOfU+W6k1FSLYXRtdcn4y2loZLSQyNNin79GzZv11dGRMoyZlHn3qcpo998rfFRT0Gnr54d4W3n3xQyxqC6Kk5IQRbmpWYxm3mYMXtjOq6ANspcZBCWRlx5SAyOFZh/Fqt+FbdciJzPt05mL2rwhnmUZJYFMgrT6tZGs+BuCvyhtYXTSOvZ4L6Kt4E/WXnxJYdhqlQsSjbziCpECFG0aDH5LBg4TCaAS7gEpw5aWKF5R88dhDTDsczazDs/DykPvi+gaIKw1heX8XeSmFZC0qIz85HwGFE2Ve2FLIlekmTl41A6vS5YQSRfBWlxNlqCCoPojYc7EIbgKNQY3OVX50yB6eStrMpJOTuHprAkeuCuDA3ANOo2NG5A/coIplvkbWPNCpVXQYTjPDDfalHSJydCYFwzuIuqwhpDbEmd8LYIzyNoblziAop5C0Q89xIXLlkHHUKqxc99hGsiZXczn9MlpzKA+I+STfIsGoj1ja182YHekkF6WwRrGPmIbHnPeKktIZ1B15XkYBOueHMQFDtwFFr522lUvIGiMblW5GGVGYWPsKgf0PorxuHj9f9z61fhsoCH2Wk8njqPfeSnrV5yw49DAKs0CfWx8aUY64FkZbeTz0Pib/1cO4bQe56/OZTD8yHYXdjXlukBkO/ub9VM24gaKoDTx67BYKQ58DZNmg2W7whvZnpHiwY8fea0dj0gwxwHRakSXblzD7111c/42d2JLYQYfqocG+AQM7UtdO2ZgDzFuSSU+kiugSL/ya/f6B9kSwo4qPoio+gjb3HKekhCApKIspI3u8L5X3D+NyxmWQBFTA3Z7whk8/j+mTCWiNZ/KOCURVRCFKGn7tgcQSA5O7R/F9kR8mjZGa8BoESels9+2VkTy6P4uOyg5nG9uD5YOeS+bCSr/F5PxbLgpsYh/9qmp6hUFISEeb7/aEa0J62Lh6I8ZEPaNP65yIdxG42DSFiPJPGJkVjr2zi9T2TwgauZq5iv91wO8MMFcB4hwo8x1J1qlUHn/rDrR+7nx61ydkJ70+JD9p4ORA/JaOJ3v2Y+QPH4SAFXTY1G6Yui34V190JqofKKVxpWxZfZRRn93G20+8T59bHxEtt5NY+wpplZ8S0XwfCqUBq9hDlf831Pr8TGTZt+wvuQPJLpAoKrD76sicE0Bu6mWMqnoqrbCjB3KiVlCZGEt/ZjbXfHYNbr1uKGwG7hm1ltW6LSS3PItPzwTU7lpEvQ5JVKBUglIJjZ47eWHHcl559RWCRwRzdvZF2r3bnQF8D3MSJQ+XkH3vJMxuXninhjg9WSXxDfy68lckm0TSlTA8Oj1wF1wSklfSrrD92jLMQXrHnBSHyuwA2p5mYvPrMQr7uNj+FyOSb+b069FMvDaGZ/HjRtx5TOfGpZGxfP1tDY9MKSSo+G0+6ZQdNl4tOhlR5zg8aLJvA8DSARXrs5l0agE+nWP+gaxXtTXjW6pm1unvWeT3sPPzAQegzW7jcEYUu0craDbVEF/3gvM7Z8eepTqijvLocixqO4IAy5LeI0BfzdTSBiKzngagxyfCaVhfFfc5seZ3UYcFkD/lXjx4nNTKtwj3zIOiT6C3Cr2lnfFVUUz/MxF1TTmCZHeyFSM9cvg0+iIhKomUEx2oWr8mL+xxVIZspkf9QphHPjalhsCb54AqhWBRZn4WtEzku5qJ6Br9CS1RoTHLjDC10nVgmJ6Xy6zLFXjYo+QPHIFTkA/sQjGk+V7hk4e28M4T7zj78mg/7OqFhyJuI6blQWojmmjzbUOQBGq6q+iKhT/s20mqDyH5sh6mTcOm1slrEuAlglp0SCRipyxjAjsX73TWa5wQxji/bHzq9Wj75AP3lVQ5/0B000MANHru4bxNdkwM5OzrssNZI6zrBJPWzN7V5zk39hyebvLeKwqQZxLwavYj8UQQPm0+CAhENN9Jqhre8IN8VRynxh1GY5J4YN86UqrfA+Bf3vB22AkmtUbQsGIFRQtk56jGHEKMVyx2hQmr2I1gF+jt7aDDIVc8EOwbWJfdG1VICiU97kbHdXk8Gr12UeP3I63xdnr0XdSpjlBvzXf2ybnoazgwLJA9iru5FHEPJzo2E9ZyMx12mKr2JDmslaqYHvatTKEj8jbC9DEEFv/Ag+efoDtwFT7twYw9HcHDHzyM0iSSpptLxVO9zLUWUCYFE78nDnX+ZdmWsGsRJBW/ZD/Mmt+rCK1bx+crfsHsMYbA3tuIbXwCN3Oko/5qgoxTSVTPlNl9jn3mmZR19Jo9qDrkhXDxAggSCDglzMCFNLUh36NEgxJ5rIqCX+LXKH/cHCyNA5NuJ6fzCKJgJdHvNN2SB01t7oTX+HPrr/tYfCJLnrveIyDxYf5MX0fuureJrhrF+dHnHXnA4K2TW1h+5Fvm5Y8lTJ3hfOflIg4ZE6eM5wCzzzGG7YZMGrXHMUs9Q753rH4aqQEnGB26A0tQAB8/8DE5w3IQBIFQ90IivXL5zPonX9z9Ir0GibFnxzrfqYbuWM53RvKj7QOyxjWzf+5+Vm1ZxYiLI/hm8Tdsu/EwKaGTkExm1P1dNC+4npaQJPqFZqKUsNbjANlzb+XxrlBmfJdMcE0nYW03EdgpM+VjGh9j+uVClsY+SWV0BT3uPSjEQWhvwY5fxWmM9HF45iYa7XmorXJEVOcY68FsRoUCOo0BrN2dy1OtrcQd+wFVz3neevotymPKUQlaRpZtYo1ezyNKN9Jz0/DodCG+BQGIv4daTTRf1Wsx9Cjpc+vDonYwXoWB70qIynYi326kISqRxIJYgltmO+ojUto+nK86lURWRjL5QAMPf/iw88zh1TuOKafXMuboMJZsX4J/kxys9NHV8flVyfRmNHF+9hHGHk7myfc+5Hp/GS1vlxR0C520uGdhVJSw+sdopp5cjp/GtbePpQi/g9PxaYZv7zrGyeHPszxsLZtGLSJdaaRDMY5esxe2HzZg3yw73+cG3YT3te18nvcUCb2ehBWFcmrCKZoCm9BYgtApu/gj/zFub4R2n3aCEsdgV8hnknrvrc6+A1kNwK93GsEd1xCgdeVgUgkaSsat4fxDk9h21Z3sUspAqurOFI5VXkufHaYem4rSYsa9tcL5Dj45aTWxyo14CCWUjVxBxTA5oNDQm0B2wyxuyHiB3hPXoRlxKwWpJnyrlXJ+dUGBoPVnfc5b/NFn4tau6zlyt5LKqGbHu6TkSPlN7C65m/NhT/BZ6cMk1L3E+MJDTrDdwHt1xwdTsO/ai0/NRUSFnRGlG/HsHYnOFEmz5156qGMgD5XCscEq7HpWGn8kd4GKM7NkO3Ig7/NxI3zWAfqwtzH0PM2UfWlMyJwwxMlaby5mZ8cbhFeYSC2dTXrONnSCvLb/XvoqJ6pW4tH0A3azBez/gdngzIVko9PtAocaNtGlc8kLevdMxBeX414SrSxwgDo/af8NReO/KE2upyixyDnnB54b0BTA1H21NIvXsD1oJINLUu9JLGYlh2Yf4vCsw84+fMUX3g7qQJBM2Cu7MDfmke//HW0OZzNAv7qCv5eBXFYmVT31PlsoshyS3/e/5Z1KqH+emZfKmGN4lBb3w7QZTgzJsz3wPbunnbhr4nhjzRv/+C2LzYLVbqW2t4J+VQ2PWluofLiSGR534d+zBEFU0lHR4XqmCE9NeoolhijmZs4EvZqwP1aTXPctUU2yRLd371g+G55Dwl0yGLRNt4HjyWNodj/kGDOX3LTOHMEr2h7m5AywQ+UxNPm5ceCaZn6fs4xdbbJMtlFVT0XEcXJGNqHZ8jNRVzLxtacQICQjSiqCOq4m3r4EARHv3gmEt64hziCfTz/yB60IZUH1tPvaWPHNROYffQOl5IYdWXZ8iq2U0dITtPq1UxF6mbNxi9CLXhzsg1JTAD92Q+m0Oozjp2HymcXN4S8R3HHNP9oE0Ol2gQbvbfRoC2j2OMj+4T5kJk53sGldMm9X6+Ee/1oic7YT+eu7hBUcos/6IYIooTWHOr+3/pb17F0dw8G5B7FpocFrO/2aKi5F3UHSZhXHU6YRlH2AxAJ/RlwcgcYspyCwoiRI0U5F5I90RkYQ1fY+vl0z6ZXgWD9kt8cjWJX0uRupSZn7t9khENAYAHb46IGPuTDyAo8uXIxf12znWDVHjKT2hjSOTdxAlXBsCIAWIGtsARfGltKm/pyCHvkE+NWieHSCjo3hkzi8LN4pLapSqPAUYZ4ennRvpyS+BH2PwJLtS1BaFCgkLVlVN/BF3n0sLzCw5vuFKPIdMqui3bl/X2mewubmKJSnR5J2vofR50Zi6DYMWSv6aaPbJs/r4IZg55ndpJOojKrEFiDR5d+DUWvEo384aosLXGN0M5L07yR2zdrFudpzzs+VohKFqODSj/KaU5dVN2hNcnn929pAH6BHExuKTelSYRkodjuMuG0EIx+YNORzhcGN6FnRFG4vpOOjHwnNP+hcg4sSi+hZvoDQMaGETI2jw8dhxznO+qLNAt3dFO0sovvUZVrc/6Tccsr57MikKurDajFr7fS79SNIStqNwehU3fwanse4R+2M/HwkWbdks3/cZFo8DsjPFUGw23DrqEVsVHCzagfe+Ts5oz8IkasRBQUZlzJIev8c5z4/h9ooj5EoqYhpephpeTk8PvFxjCExFMcXcCxlBLnhDwLQ6XaeHH7iUvN5rA4FocSa13AzR9JqOEad92YsSgvW1g6Upl5nsG8wE3PARvLsH4F/92w01kDnWFdGVeK7X8XybcuHzFuv3rH8OdXOmh+WI5nMdAQ6wHSOLeHDeR/yjO9ZAjqvwm6H0FY5F+Wo0BkcqbiJh/88yKNuKxlmWEF1fCh7r9rLkKCc5LL1mjz38trFB6n3/vUf1wHCmYjW4a4UkPMVDz/uQ9oRX0Znjf6b+pdIbGksQb9nsejIR0y/nIVCGvB1ijzrA+0XE1BaBbJGZyFIjqBX+HLwSGJi97uMOHmW275cRVjP1WgsAXJKIs/LjPCUVeSmHZrGzG0zWdAgszB1OpfNYFPaEMcNIy/D7PzNwaX0/e2M3vUSgQ2BzvbI9ZbvzzMe4lz8QgpDnqNXW8SN7tAYDdYlh/jtjjrc8rLwbnCdw0raRlFp7CQxqxGF1ch7j75Hu49DgrvpOE9VP8cn/rA9vAysClLWi6z+9QHS9bOdPoEhwbhBbHAYymA328ycuv0UCb4J/J+VyozFNMS53tu+gipSj36OV33ekO/9x9//b/n/vPw32Pff8j++2O1yfg83IQA3czQZVV/i4Ugcb+jWYW9soVV/nDzTQRcKZxBK2u74zKD0xLd7hvO5r//rdfAI5l9vvMO/xFq8u+WFS7TpsIt2evW9WJRKbIIRq8IlxZniMZ5xwlp8eqZgkeTFvtVd3hCMViNS+CquFI8h/uwv6P7VJAdbHEUQBMJabuWBKc9TkZRAS5C8+LuZIojylBf3DDcbxycf5/Csw877pkf+zInRP7Hg4mjSj4tYM4bT4xuJKMLUvIvMv9hFjuYzsmKXI1y6wPz9Mup1MPrx5Q47tlknODeukk2rNzl11zvt0GbTMdwwF3fzFKrG99IQ1k2PtoDGfhnNuLNoJzuDViN27+KOPffg3z4VUVKjUMAMj0f4cdYK/lX3CDM2zkDVoCK+5Vr8O2RUSmm7rMmvSy/i9KRqurwFp5EFcFPa41yX9pIzsOjprmJY+Q+s1MvXSxOLKBzewYhjPiQUJTAs1bUxLdS8SX3as9SlRVMfVEOHIWvI3PlRmM68WigZp2LOjU+QVvm5C2kSdQM5YXuoS5+FJIhY27uHHMg9+ocRlrwM87P+FI+5nql52Xj1yGhmn+7JLNr3IBnvarH3dchja1fhRAA7Dp9WqxXB7jD67PJndd5bCG+/ETUjMHrLeQc0omy0XTFDtNtRJk8/SGvUSNp8uuhz60Nt8adfgnM9HtzZGc2vXQI+zTZSc1NRWmQn4aMtsMUxTSvyokgoSqL9OQ2brtv0D2bfyIuuQ7zCpkB0tNtZb2ewTz6szfG/gtEuctEEynYrV+1MJvVKqssAk0Q8Oj2YcTAZSaXm++vv5FjGJKf0hiApaAxqpDxRj8nNgY62G9CJ8GUA3OhWzcpiH9LyxvLTbRu4knqFFvdDdNuhSOrBIqi4/7P7mXEog2fHvkhIm8zS/DkIPoi6TKFYgX6pN8Xz1nJuwcMUxB2l1nsTStugxMuWbkebHGxVSUmtz0YOD4vgmOddLvaKw0BeVQ9BbhYmji2kL0VP7pQ1aN0+Jrz5DhbqlGRPf4b0qYVcmvkQtLYx7hiM9v2OoOQXmCwIzFMxhC02AyWTBOiaBZaJcFI7jVpdD5WRNag83eizykZ+YcjzrJ0yn0cn3EzS1amE3jIHq9bgZBYDKC0mxL4r2LwEzlx1J8enHB8y79MrviSh7kUEtYp+XR+SKBHechvx9c8R1bwWyfsubLc+gLawnLTLaQiSEpPNjc+zvmTU3tEEPfFv7JmVlAW+x8XY67AputnfB0vrIXjcfTTH34hNtOPR4SHnvLJ5sunK89jRkHHmQ0bvfhm/YSG0hfbQJf5AB5WoVNCvrsZwyIb933b8kvw4O+M8Hd4dznk3ofF7zt91nlDzTASbiZaOGsxiiyzWp0uhKTwYTY2KxTunMq/kTUYr7mDB+X6nvIiEHa+dBazYsgIBAZ05nNQqWaYxsCEQ0WZi9wobpbGlKAQFFxvmsqXoW16cV8Ctq6dxTeCD3Nf/JH5pKfj7B4H/FDpCr8G7LojVmxZz3VfTmHtgLgMHGkv6u3x47lOOKWcTMCuW1NxwpucccI6DVeylwv8LCmIvAKA1u+Ou8nYN1ABKGisDa4dCFJBwOUkOzTnEL2v+4JcbfqHPIQOTWbOcry++Tp9NwNBjoC5tCk3R45xr271j1pJieoJoxUYCKk6TcuIb1MYuTN5zIWQhXHmDKaUv82yHnJ+u8LqJVAbvo85UDH11DPffxQq/GtaNX09T+kTOjdpDWdB75PT4sWyzidkx3/LYjLk0X75EaIFrn1KJRnaP/pJZ1x7mjwdaUBbl497icu4OLjp7IDENjxPfeY/zcKk3xTJCkukVfbpaWbJNEEmp+hCA99zjeMu+gbmn5hFWFuBYW+Wb3R1t1+uNlMbVsdn7To6kJdChP0dr3c0s/+s7MrwL+bV0FPO+GY1NhPIYF9r42piNvDJjLnXf7UZt6qE+Kp6a8BpESeUEQRQHv0SVJDsO/LpnEti+lDfaYVw1xF0YzrUfjySgRmZ9qEX58DlBCxIClxMKaYjpYOWWlYg22Uln/z/Y+6vwOK6t2x/+VTW31GJmtCwwM3OMiTmGJI5jJ9lhZnKYOXE4cew44MSOmZnZsi1LlsXMDN1qrO+iGqTs/V78L75z3uc5e+2LeKu6qhbVWnPNOcaYF65ypmIWWslM0OAc2g12/Eol/IxDAPi4BXa0xnFKmEKtcRO9/2witiSWoPYJdNlMXDZux6QpxbfVl4UvTOPJj2SwiguF7zpcDtgUhd03kOaoEcTWPYh3V6rzuhORur+SAUd2sc1nMoc6P+8xfy2qOsqEo5SFfMsN4ynCmxdwU9NjbIlu5RWDEoUNgkwLCW1fjLfam4CGOzG0TuMJvw6sty7i6iCRTq9OJOcg67FB78fZI4xkxOkRDN7hzS2FrzPjkom4ugd5atyTLOv3EmHGGcRfjCOouvHfNGAUDj3Tao7yeMBBFJLGDSpZU7CEby9+wY1hd+IYN4Hwci98W3zdTnqjupjfQqLobJ9L+M9/ce+396JA7Z5/NkUnZkUD2xYa2THnCu0+7SgUIiabD4s2tvF71zK2zd7GtrnXic05g1+jUb43dBwM+oSJfVYSVqgmvtjlvPQ4M6w1K/C/epZFgW8DUFreU6ZT35VEfM3jTAyV9zaH0PO6x8kqrw1253+7bAbGXpnOhrV22o9/TlNgE126LkRBINirjJt7fYXLZDo1wc6XD33p9ubsK7qbLVUjCIho4OKwOrw65YiAUW8kNSiViQEROAq+J7jzMWKv7cJedYKdQ7VkqlbzbN953CeTXWkKkYMs4XkeeSqV2EVYSxv9TlyCq23ytyqBwhl4NStr2T5Qga5uHX5nXHJoAvG1jzHxahEBHSPJC3+NOv0Rzvo9zulek7jWfhS7pKKiPYVQlQWdQ4HgmhuSDMCI8rnOs8Me5LkhZbz+8uvoTL5M37sEjSVCbnb8HWyKegx1Vm+W/JLBC++8wKuvvkq/y/0QRQGNopPBETt5vMHO/NzP8Kk9Q9LFv1CbZfvhRuNwHtuTyeYOlVM+Clp9W3EdpyObljB+fwSxBx0MzBzoZhn1Cz1AtK/MbnJIYE4Mpy24N3qF3ImzUz7m+3HxNPUeSVH4aprDehPZ8R5TwheTGnSS1TPS2CKO5/txJ9h18xXKIw7RYshEKSqIHvIejN/No81TMJ34A6mgkH+Wjwc9xw8ZjWy/ZxfXMq55+tv/Ch/eNJqF3iDaRRLzEomsiHTOOY+8LnjYitATse126Lgkt5xgmJyG0fQKPMdnwbBlzhYOTa/g75vf51LrXgCWbGrmR5NEh9gbQZLwaq5AtJrRqTroHybvK9qavTybcAi7VEZUnk7O240Ijeewo+GCSUFWw2U6xUq8u1Lw6RyA0u7Hmsvvs69wBd7OT9BgSieofRJaqxw8DW9ewMwLDkwRGdTFd7Jh9gPsalhNZPNixly/iJfZJbHt2YsV7jyFDv7ss4wDsz5k8OlAXn3tVUSHyKjcUxReXcfvx04zN/EpYuuX49egl3MKdTtbiIhIgsSe6Zexag2oFQpudMyAmEXMTXwJo9WAcu43hLzxEA4lJOcl92SIumQ8cVARuI6XLi+hKsAjqRXYMYZJwhvusVA6DBxsiuThKj9stUEMORePX6OXc+x6Bvua/ZvJTe+UzwX2ni4iky6edb9N5/53nsOvxc+9Jn3XCo/X+TCpPoGyzOv03adHaVWisHeTR/6HjKnrfc6eBaD/oSSUb77GyBP9nffI+67aFoTeEo/G4c+lhMWc6j2GBmuZpz9dQIloO50lndgv9QR5ebd7U3dHHW8p3uLTLxZxuE8yBkUgMb5OgJTkwHfJdP6V84inbgKkBaexNbyEZ/+1mbq7gonUpBDVdBuBHTKzW+HQE6VPY+5nt3Jh1ipOjr5Aq9cFrMqWHu1zSHYERIJ8vdxBCUESmbZ7Gj4Xy6mK6aDNtw2V0+nqus+udGDpN5j2wNh/Y2/+T4yJPCtc7PJlcydYVTbs/sHYVFpEZ061WhtY9LEomuupidLyy7JfAAGFoGKvEVZW3sUBIxgDzHQNH0+nf1QP++2fMp7d5aYlHFiVzdgUrXLOv27Bvvvr4LxZwid+Hc1BDtbctYEjo77DaG/xSEw6izvvGAqg25pjF4krCUFplsFYsQ0vuW2dE129ealOwcJfl2JXaRF1U/HrHEqXBOMqYOXZNykNr+bP5w5QHzuYf5a5m+cyYEM0zc79U6lQuNmYSCJm70CMCX5OYIjwb8E+7w4tY/dF0Suvl9veOFU+n2rzAnpXvk1s1SwGXBpAUH0QClQUWuGeWvjRKINb4ktSGZg5kOG5OwHIbxrCb/nLqbeDT6sXDl9/1i8/wbbZ23DZFauO7OWJomH8sfgPFHaYuWsm/s3+9GD2IlAZVcne+bF89shn7non1D7JqJwzzJ9xD7vvP0JBcgGiQ93jnOlV74UmVkN2RrbbX9S9PJDzADO+mkFQahB60Z/U8g/oXfEu2rY6/KpzsHdZSZ6RTPgzd2D084BWvBtLSSw/j6nBSP/l/UlZPqLHc70SQli6Yym3/CjnxTPrekr2Wpy+Irsdt2/OLnZS47eV/JQmbA8/zmOlj6FbOZEzKZP4yAnU2WWE22ug0Ar6ZjN+zX6eXMkOJWoieW3C60ydOJXywZUYvYzuvhRFqPT5iLQTP/Bt78+wX83GZPMh3LoJ8r9GKSppCmiiI9qbvY/tZcg5GWzRXYmq8lwl/pcOojRW0aa/7M4L6wKwWxwWN7NP4UyZkB/xBpcSF1OdYUHqNOHdVIZf51AC2kdjVtZS6f8HLfrz/zGY0l0C/saEYop7+xLe7AHEy6oqAnVxQyhOrmTnYJFL8Uvca01KUArxmiEIDjXfmyeTHf04N+dXsHmufMZVW4P5KPIj5m6cg09ji/OZnrmntgUzKWQJ4e034/mOe85NgLjah7jZ/jOCAHpVK728aigYfImdy0o5uaiSY2OP/SPntUhxfDGWIB+ic/ah62jo1maRx+rhVEwh22dtp9m/GTezL2IahIzDv+1PrAZftJ0iN136BENXOqIIn04byJphsgLctWeucc+Fe9ySshoNiA4tkQ13sDL8bggJ5lq/Lvc7AUyqchoMB2k8cBnA+S3SDWz/nyWQp+ohRAkvaZMJb59F4eLZZA+Ow6KQGbx+2lpilRb23ryDc2P1mHy7QHRKkyrlPXa6F3wUBBFV8jtb4lYyzHduj2Db6rTVDN7xGr2vyQCJf8p4Alid39a2Qds4k3yGQUGDPL0uwoA97xCXuZl/FoddlscS/qFS6do7Skrg0iWorf23W/+fKIIgEBAQ4FZc+T9d/hvs+2/5X18cDvlD0el07g/FtYldGpyF7pn7ODFgBR/W3uRmxQjdGBMuw1ElanCIZs+DBWj0LseiD0CjMLjlg/w7h2HSm1h753ryok5yJj6CwvR73LeNDLyFWV3vc8sv13nqrcdR2BSo7L6M1MJXwSC25xBeeAJ9ex2mWXqsKk++N0FSyEmsK8eAqhd1EfIm7+WQHYR5TQNZm/cI127Kl/OMuCTfRCvtNjXF/a9wfoqVNTPfYuuEGdRanBIH2MlXbcRQfw1V7nVOjjiDJEg9Nt632u28MeYAF4Y3y05M58YTo4Sn/a28UzKcez/vz23Fb7OwqImw5nkARPtEE+8Xj9akJb5AQ+SFUIbnHUDh0COKkG+RpWduHfMJi7ctpuLLCm77MpU+pbIEWmtXCPVmX2wpRVwaXI66y4q+U+8+oKkVRhL8r1DvkOXaFCiJbryTgkvb+K54CvuM0O5jZs+iIi73v+yWvAR587Gr9RQNSWLt8rUgdDfABY6IA2DzTSRdDWR0wAKC2ydTynEuXf0COoro8u2HUR2Aeft+pJLSHtIdAe1jman6gKk3PUBrqMtwk68Ht01HoV+O16Sh6C9kEl8Uj0ahxeFEpLZ4nUdX8xu2Zz7jqQ+fct4r17vF+wwWVQ0VaVM495iARWNB62T2za2GH5p6UZwmkjU6mD9uO8S5YecQJR3HTDAtrx9XasNY9foqZv7tzcKNC1GbNcS33EW9HV53ZHA0UmL16peI39GG1e46iHU3sBR8+eCX5PVLZPsCozOfyD9kPLG6ddaRBBZGnqKoKYPYtz4h8VwIe6dfl5Oju5l9Inqjnr5XYrCUVXlyPrn7U/6d1mgh8GAZsSWxqG1BGB0wuQJ+YyUHDiwipPgchTEFtPm20aG9gbcAKSqYr6ojf+IRoksFvBZpiGyWHaIaASLVZqoyxvJN/ucIooBgq+JywjIuJ9yOgMLtcGi3uIJ9Lmaqwi0faRe6uskKynOo3i4HEAJEsKugyzcOldiXhNonKHdKrc2NXYFF74dUVU1weSai3UqvuFeZPvUGI27W8qo2kDsw8BJBiIxhzIDx4EzvOFu4zuaxpZyYX4Dd5iDjeCpak5YG3/14Be7F3/cST5x5nN3XP0HfWk1s/d3cUWRnTPYlNM1HGXhoI7//8R02tYqIlhV0LylFcxl01p/6U/mE1AY7Ha3/2O4lCZ+z10m5kdItP6WCZu8urva5iiTCsBMhxJbEklL5JkPzdnN3+Z8ECbJ0q3bGeIacGkJa50coHF5cqxtHIOcJTinCaPDjwvuHEX8vJvbaIcocp1Cp5D6/OOgiJQ+XoNQpuS38HTJKv0RnkYNODgfUXq1FXVpA7JVNXJj2I/v7h+AQLGSUf8HwvAOkD59EwaBbCQ24HYMYjELSorWEE1OWweAzMegKWwmrTaCv73guJM4jO0ZmZfXJ6kPy5ZNYlTasaitKZw7XgqZB3GUcwOr0TWyfe4XPHvmMTm+bbKQm3oVdG8b7eh1xJdF0enfJ8955eHD0epJvOzrYNyiGE8NOUJoxHZtax9KlMHUqTLuljWuxD3Ak/nk0K2+jOSyth9Omp4xnt2BfNycJeBDSonN9u1o7icLmAUyOauWTJz5h0/RV7Bqg53ijzMR498SfGCUtX1cup0m/k/rIZExaC+GBzVC1ExrPUO0/klODLrB2xXEuRKzhUuKtXGrfBWfvZnTEGuZtMFGwViIk6xS1Ic7DlKTC5lBTZYzkiu4Qa/q8z7mxGq5JGxEEmJH8Ncle9cSpQGUEzfEDmM2r3ZJQ3YuXPZK0ig/o0/xSjz75NWYLq79/gHu+vZ3wqnDnHijxfQjcGVJAieJxgq+cYt76iUzbM81tE4QWwVs+t5Pl28SpUVmyRLHDQXngGiqC1uEQrAg2P6SWKMJKAxhwdDv+TZ7Aa6PNyQ4YeZxWfy3lCX4AGKSobkxNzzckShqC2icRW3c/m317c1uwmS6dlaGHU0jJTUHjzEXzeD28UO+FRWOhOqGZ4vhi18qKvbMPqUFniLeV82kw/H1HA8alHi25oyZ4tmIcU4vOI5SdIaRQcDqOFThcMoGS0i27qOvSOevWM9fRtcmVGNMHE1t/H33KvsS/c1iP64asFnl4BQmtIDsjVTZ/lJK8L7kCTKIgEtp6Mz+lyg6pI+++xu3fJKFrrUHV1e4+WEZ456Ot+Q1/bQ3ZAwW+v/d77Cqng/LqK3DpCR5gE6/fKTv0FadOyPURBGL8CkkPOY5K6OTQg5vpde5XxI79NHudc6OeoSfjy2Xj7Sibye+ajey/KZps8S/mbEyjT1YfRGfFJMGBSaxm3NH+qFva0Zg1PYJ9rn2yOdBKUbLM7lN086o+7b0dY6IPQbY3yRl9Dy1hnpxvOOxQtgnfB4bRHGDmjnV34HLKDY/azOSEnzCoG3E4JMrL4bffe8o8+nRlkF7xMfMiH5bryj+Dfc412vl3u2RDBLZOWcDK0Hy0rT5o2ySiqsYT1joSESXX6say9cYjLNQpGH94PKHVKkx6FTpBtjf1qlY+7LOGFT5gaFUw+qQsP2QwzCcjJAOOzkQ8dy8zgr/n2KKNtASr0Jq0KCUd901bwy7pIhMPfc1qsZW8IdVUJvUmY/42mr3OcEffl1g1/Cb86vKo23aWVa+vIuNahjtg4mr3hkUbsPoqmbljJiIiansgeks8Td4nyYt8lVr9YRo0F2j0OUSrrR5RsBPqVcymmBNkvPIN2f0cpOak4tUpOwTDvIsAKIq8g2H5B0nMD2TA1WHE1z/kHietw0xzeA1tBg87KL44HlEQuGliO4sz3mSar4mBFwdiVZk5PzaYstAD7t/ePfAxPg620BzQzMa7gvny4S97ONaKB8yj9a6xvPvsu5TFVzJrlhwk/OLcd6RmhzJ/zZ3Y9Woq0qe6596wyG0YlEbuVOpY8Ec6NrWetuBEBEFmGgJMjVvO+/Nu4CXcT0BjAEqr852+aRAxjX3164moECnoX8NXC1+lInCd/M1VbKW4I551VjNKtYaFu37jlhMViJKa2o54chuGk6yCKfun0P7pLhZulBH8Qe0TnS3ysHmqvfZSGfA7TVaPPFWFzyZqlf8i6ffDPPTFQ2i6NO52HSpexsZ2karIKjL7naIm6BjtDo9zLsd0kFdLx2Dv+Iy4rB2cSUyjzSFxqnweH5z6jZKB33NrpcCBySfYfn89oJHPAFU7WNHvMfyd5yuH5KB/yVrGXr9EYMcY0kOO8/PsRJa4FB2Fnt5Qwfm/qtG3cnR5CfUh9T1Zbi7pyO7BPmeg2iHYqLeB2aSm3U8kOy0bAVEGCDbdwbLIc6TnhZNx5lss3mquDwxFb3blEZNQ4EAUILY0iIC8s+i0WpqsvaHfW5R39CO/uQ8zzjzHW/WDybjYxtytd9O35PtudXeuZzh6BCTAw8ro7nOLb11GyKVCIpsWY8jI54/bTjPgVALDzgzr0WZREGX2keRgyoEpqC3qHnuzdsJFjjTJwKqBlwYioCDOO4U9Rug/5hvMdhPZ6VloLBrSctJ6ME/UtiAeHvAMc2PuoW/JDwwq2PRvfe1XJ3/Dmm6smO7F7nC4JQx9VcHd7nexou2YGk1Y2i097lPalNiuy+Oosqpk29+5rF8w/kWDdg01T3zIR/e/w+leE6jx2+K+vqUznB98qnGooPlCwb8BT7o72T1rdk8mh3uv7uHnE0jPTkdf2IjDKd+pEl1nW5GIyggm7DFgTUghN7WEU5pVFEtHkLBTEfArWeIvOAQLrbpMan13UmMuQmnz5a5aeKrmXtKvpTP2UBSdo6bQFpKMKKlQAC0OyAycBcc2MnZPvbMmAl2ODvxESNVvp5cKQrL90O3dApYiyrtyPLllJU8AE7qdp1C6QaOSYHXmD/N8O40OGKWD9CV/s3lZHlWRNbT7tNNgreghPdk/sz9jdldz2/rb5HWkW5BYbVEze7Nsu+SOWE5LzE2ITkZPpLKVVgmqw6uxeMvzKKBD3s/U1iC32kHvk3EoW/ZyLG0A5xPnutt/esRpCkbX4tvii9qsdubOlsctM3EJR9LT8f3rBE+//zQiYo8z9fCGr0kvepLShFZupNxwf1PfXfocQZAYq/+emJzDzN42m7iSOJSiAgn4oQ0+7ork7u/vxq/RSPO/nsWh8nwzST75hAZ08Nttu1A01pOYH9JjDHw09SwPKSBV45mTdSF1PermtllV8UwxrGdA8XoAgmtFpvy5h6YNlW6gtSiI7lzSAJPXTubCMhlA3T3Y9/Pln1m8cTGHHYc5NeAU/mv82aO8j8Tap0ioe4ygisskXfyLrkY5cPXPNFU+TaXEFp+kq15mqD38cM/rrgDL3sf34tU/iZqEDPe1yIpIvA4cJ3NNJr+kvEm/izH0Lf4Rg6kPF5LmkBv5IpIEvjG+OPy6Ae0R0AiQ9OH93Pb1vxi53Sgz3SQFE+LW8djw5Rxr+4AN95Zy9I2jbuUFV1+KItyI/pRTI2S2sr2oBIAg216oPYwoKCiLLaNobhyjnx9NaUyV837PGlZzuQbfqyeYutnJ7nXloHb6hqx2CzaFbOO68qO7ftMRocL3tlkYfcMYULyekTeO067LITNxCWXBHpn6isB1lAZ9i1lZ57bv52+cz6CdfdDo3yOh9vEefS0IUB87GN8SCym5KfwzLCAIMtO6UDpIg+9+dLZw1OYK1syO4vlRc2k5t4OO09dIO5/LnM1zesw9b3Myr6T/Rv+aT/6DjDHud0mCw7029g89wOZhHzAmtIOmMBOJSU+S0PIX4c0LPHWSRJoCm6ib1I/KXuOxajzBYIdo5rMWeFpbi7ZLS1JBkkfVAWDQp7yoeY8L6TIrVd1a7x7fP669woYymfnfnN5MxKAI9/quUoFC0jKgZB1zLs/F/vU6NBYXuEluR7X/Js6kTOb4m4VcHX8PBUkFzuv/BOz1DPY9VA8P1cHefV+Qdi2eMxFvc2DwcDeA52zlbF5oiyQvrgSHJoPNsdVMuVzqURWZLIO9x14aRHCjAdVtC+lwAkRc+9imhtfJ18isRbPWqbjlPJ+oFWr237GfI3ceQe9cf/LX5bPntj1Ipm4fryTR4R+D2TuQqJz9hOcddV/S9Iole8JDNEek9Wib6/1//w2rVsHJk/w/WURRJCYmxn0W/T/+/v8rb/1v+W/5/1AkCa7G3M+lyBV0KZywANfhoMkb0zurGXFaRgZ5NkjZMLE6bO4AnygINBk8yOPosmgqEo3kjlqBqBRxCBZEYKJTasG73YvR27oYdHEQOpVHDlMhigh22RrRWDQIkkBE22xWJg7nfj9Q7k1nwtyXOTfSSPtsLTaVx+AdbJnFwPA9XF9zlvDCE1gVMqLZbvPh3u35vHVsO+nln/CvTQ/z3LvPuR2ZftpaDEoLZVGVlKTZqFffoFOb1y1w4QxuKuxYQ0IoTigBweMcBvglbBYbFQsJb5oLEojOfDw/hcLzwWX09l1Hl08wph9/xzfntPugZbFbEAWRSQcnEVatZOzWx5Gchp8oQqFtEauO7KYs6nVSbk6h1829qI8ZiOREjeQ3DWH6kS9Z1w4as4Lpm5oYf2S8u85/57/Gzw3BHLXJuQMUgvzezJrpfF54C/HnBzJpcxRK7wx00mTCNUnuNploQrIWommSnQdKUYF3NxDYMukYo84OZ/B+FaU/HsCkquQH+1gCMx+h7tAtPHHdwIFRU7gyr4HVkVMpCJflX6YH3YfeEoetuo7iNUfcydebDPKmWh70I62hyejHDcX34nXii+PRiF7uPHTZMY9yIU2WmfQyynPS1+Ax+pq8T9AhbqPXPi/65iwmUedBGt7bWMBj4u8U+X/rDpopnEmDfZVGBnu3UpRxjWv9Ovlz4Z9Y1RIhpnEEKWCATyACElUp43H0TiPwLasTFeyZBz5CBLpEHaW9k2kNVKKy+bkZcN0NUwk7g4o2MOuigw+P7ebbcz+h6QKloONGWg21YbVuIyeobRLjbOc4N30FkkJg0R+LUFlUbiNWkBSkX0tn+l81hJyqJ6oiCq0lhoCWmRw0wRVxJM0Tbidr7GI3A8DH1Ic7fSA3Dp7RFDFo3hGu9w8l4WaPMbGgGoqskBcnrxOJu79k8P4/GXZmGIKTSam0+6Cw69355TwHcqW7fnasnmCf28kDo7K9UT/zPL7bahFtFgSHHXBw2QwfN8PBJn+ywqZiHTOGnYsTqQw5iQMbkjKZlmgT903eyOKQb3jxpYsMiBiMKUTOYxgs6UgwX6WgbTQDcg9QfSSPyXsmou3SEqeU83h9XDSRql8qiHihg7Tj3yE67DQ3ifiaBtDqr+Tw+MPYQwT8Gu0MzPuIhJqn3P2i66gnIv8419/YxP3f/guNWeMeq2avs3TwK5YLu6hdMpq9U/cioiTMu5ANC3z4dGABp2/9G0eInsHnE4moisDHOIDJOjXfT7yVkBMricp8Hxo6WZyxmJg6OQ9EsymM+1oX89r4L9i2sInU5UNoDzRxYfAFFAoRtVo2eiujKmkc0Ujullwi37fSu3ChOy+TJMGJd08QsPtX2gLDyeyfKYMWJAWhRafpt+8DLO1mWsJTkbwNbgM+qukOJp7/mcFn/VDOmkfJ8Pd4NPFrFJLnsHe171WuDx6OaHegtCpRCkqSA86xIO09OlFTbweTzkxzQDOSwoNIUxvLmdCvmHde/JTs5ZMIDnsFX+MA2QnbVcVdUccZIiqx+NpoiuxLYLiGJUvgoYdg2FC5z7t0Rq4lXuJyyltcafcw4Vzfpc05rwDa7c0Uhr/n/s3cv+ey4tvF3LT3JnSdOneb56Z8yZpQGe08YU8y/S+nuh2aJ8sX8oTlPQ7XGEjPCiA3/icODhiEFW9Ivh/G/E1e5O28YzWitApEFOhQ2BQyEzz2VnaXPo/VoSV8bC8ae4/E4ZS/9FJ2kRp0ko8vvsfTlWruXHcnmtZTnLZ/IR8sBDsVJj/ezvPH0Cyy4Y6dHBz3PW02j3P3XOJsjqcOoVGUmeyC4HGCWZQNpGob8AtuIaQ+iGXrliEKIpJgpdUB5zuCyQq7l2sjJlIVUU91eDVatUjZY2UcuOcqD03+klcCSpl4bClLf1+K0qYCBMIVMDfiBDuvvUXcS1vQL5gOwF1r7nLX68fOdn4uH8GGuPVsWlpIU6i85qrw6jZePU3mvIhXSI/7mjkhuQwbfp3di68SVRpEYmEqOoXsZQ5VwD2+DkJtKrT9BtOQ8iGDil25eK28eOgQABt+ms5NmzVcUnxNveEAsXUPAOAjxSIKIgMuD6AspYOc9BwkJGo6PQ4Dk97E5w9/zqePftqjfgZTBvG1j+HTfxjKxloMDUU88IDnuuhsjylSR1GSA0mUUAnyN+MQrB7HHi6HnjxQO/Ie5qesr6k1TKAzMI7049+iq1mHRTIS7ZPDPYMepT3+Rcou+RBU7blXEIDgMRArgzXikqo4N68J68gxgJzEfNO1+wjzLiYg7EO0C3zp0hvITH6Wk6nDaNdlu+v+n4J9Pkor5wdu520/EYsXHJ1QRGFiIUrR5XwVcYgOvr97HVW3r+DUjAnMFn7sNr5Oxozg+R4FBJSimWGRW1ELAjkF0wmsruFc+htcTXhFblP533B0JpxYwODIl0DqQmlTuvtrUfqbPDpsJb/NDwLJQUFBz2De6NGeMXHic+hnu4d7bVn0qnrV+Tunjedi9kk2BKDKGIlSgN8f/orKgVru/m48c098ik6px+bQ4KepZ5hKZPzR8fS5EsHNp/O53U/OydY/7AAVxkC2Z0eydK0cuNw9bTd+wlh8tb6Q8hj29BcZVmOjQXLQ+0AhiYWJcj+pfGgVB9JilZlp527J5+qgVt4tnU1u5Isk+GeS2KeIkmdyaZwpci39Gq2+re7gqetbagpsQjTae+QZdPULuPpJHmilQiTUq5jvbu6FQ4K/OyC80sGiPxcRVhuKIEBW7QRWbismuPfHxJYlUDxgLvkT7pWfKQCZT3Jv8TM8llHDrluyODXiFD/d9ROnxzZiUBuITQ7i+QNHaHEombF3Gn4NDeyc+CCXk2SPZJQhl9kpn3G3jxWFpO3m7HaycQQzDRGRGOMMdOm6kBQSggBV7b3YV3gP1paphNVEY2wvICvmIc62ywCN90/9wZfFS9jQZeN6ei0d/tHuOuc0jObBXTlYdSl4268RVVTLI188Qkidv8eJ1XSRV4Ib+fuNN9k5Zx/1Qo4bGc7JJWgVshMsqCSAyPxz6EwSAgKNpigkSWCJJpwQhywfXZ00npeDL+Fj6uv+bkAOWuSEvkxmwlIKjZfdY9WizSQ904J/vUYGWTjrNCh8NzG+OexvD0bbFY5/o7cM9HNWek7vj3jedwlm6wlSs2SgQrNfOanRFUT5XMfhUGD268+NonAe/fwuxl1czPTLncwKeBqS7qPD4sfz8Ued86RnMG92yicAZDnjPpUBv1MS/DUmdVmP3zkcuBeSnqhrl2POA9Z0BartopGQYnj8hZWEFXXy161/ITjn9bTEb7l30KO0401rrIgUMYQBxesJ7JDXuInx63hVMZvpXnAt/Qrrb/uN3QNC6ev7OxyaxMaSz9lZ8C8qSwoIOgv10X2pTJuDoSvdXQ9BUmJQBKCy+7uZOG6pMEHOUe0thdO/aD2ztTLoUiWaeS7jG5YEWWjz6cTQqsfQbujJ7HOd2Xq3s33Wdmyqnk5Jdclq3lq5AKtOIKmoPwZjH47dmou0SuLO9Mks0FZB31zODz4v563tEewL5uUR7/Fi3++IaVhJeMs8z3uddT+w8AyvvfIae6btcbZTPo/U+ewlJ+oZzhv/cts4Xk5GLHhUU6gE/Sg9Z5N6Cuk3fdLEKmkVSxuWcnL0yR7Bvp8b7uNy0kNIkX40NzTT6HOELlWV+/orzX1Y1QSG/U3kvfwrNX5baNXJuflMqkp+qXiZ96Jfp+/+j/Br1jrbI8+dPqXfsDCvg5t8H8OkqmRt63K3dF90wwq2LxIxDR/Iik96M+jCINSi67wkojFrCK5TIRg7qfPdyRnN65RwFElwcDnhdrYrl2ETOykK+4jzybM41bQFQVISKMJ9/ocYqBZR2iQ2G8aya6AOUVCgFuDjYKi89gE7hnxBY2AH446MA0nEIpmY6QVfh+XwhD8YavRorl/B5PiVlZfSyYtY1aNtLoe11I2B7hovBzaZ2ddNoSJQhBFXJnN7LeBwoLSIiHaRqq78HmM17ugEguosxJbGonAocC8mgFVl5cSYa5T1m4kkitilFlq8zmIw9mWBVyF3+To4OOkgUlcDDar3MCtrWZpVhzmtgfPT5zBNnMbwXRn0yrxMm/4yHVqPVN7VfldpCTPy+KePM/DSQCrM17EqWgloH4dXVxIDzgcSkosTWOVh9gW33kRS53KiS3XEFvnS6dXZ45sa7PU+YU2nCKq8zp6pe8hPzkej0KETYLyUTH+fMBqD2mkN8MKm1LqN4fmp77Fmwm1sCldgNAhYe/dhxMlezoCM/Jvl/Z/lrfiLDPeXQSuFCYWYtWZ6sqdE9J16kq9WklKY7JYxtqm9aApPQ3JI9NufSkhtCO36Kz3GImt4FqYyE8vXLO8R7LtUfYkN2RvIOpqFabcJ0S72sI0bI/tQOGA+ukA9ZSfLaPrrIGqnAhJAXfRALg67C22UP1vv2krM+U3dX4uxvpN9T+1DEAW0yVHYRQ8oJ6ghCO9LORgbjISPTkRUpRDTuIKQVllJyrdZjfrt19i4aCPWNhngq0CFSqFioAaGZpRiTiglv59I5oBMRJS0dIWSXTcWk8OPogNFHHnlCDNfmkJoTagnUCOAVW1n/5T9RO9K4f3xS6gfGozWUQYxC9wqWhYNTHp7EsXOvKOipKI8cA2HM3rxe9DvWIIiKE10rhP/UFQy282Ets4ktu4+DKZ052/k52qru7C3tGHR+XnGtpuN5ApK5Ua8RFbcfZjU5e59uyashpYUE4LUc480qktZlb2QosCXCM61EdQQhOhMzwOwNXcre9rfo0V/0fNOQUAURRqMUYyI3kJG0IMk/fgQFo2S3rm96f69gry/ShLudbvnGcrDlHedA60ODVWdEehrFhPffDMB9mQCO8ags3pkw9MqPqJv8Y/YIuL4dd7d7Bs8qBs7WGKYFu72gTEnRjP+2gz8OofKbcp6HQ6M48/ibzg18G2u3TuelnhZeUwUYUP2y7yfsxKAE2UnmLB2AueCZJCYshuxNelm2VaevDeDkbkniKl32pfOudLlZcZkCHD7fv8J/nCpBbjOB60OWN0KHTkSkWe34FMv58F2ARSW9X2eV31ksEV8bgUXl3+J2tgit0lUUu+Vwi/6aZxr0mNXKjGF6TCqi+l0NLnHsth8kc/mfMam5eEUJsvPcjH7REFkcsJkxsWNQykqyfgqg3cM7zB5zWTU3h5yBYJA/rDbqE4ei39NDn61N9yX/hnQd5X/ynjKxeFwUFZWhuP/Ukf8N9j33/K/vjgcUBG4lqrwNW6tap01mgFFvzKo+VWkiEAaAxtlSrozB0BPGU8niuEf7NnFfywm+WgbmfF3sMP6BBZlA7cb4M9eF3jd/AQzDr9BZZwss6gWPfmydAovOn2MnJtxHx8++SE2pQ2Vw4cVcw/DvHrKDYMxp2VxbvglorzTia5f6b731/TveW38dAyLQbCZCKyUjU2dJZ7qjiSC9aXE+V5GvNKJyuYJvBwsXs4dWQspt4G2VWLxL/Poe6VvN5S0vIkUJRbRuHQRhQmyjI9C8OxQS/W7MP2lYc4vxTz7wRtM1j9FbN39/NIGG5rSKBq8i4LBixBCg5Ekj4yA0WpEEASK44vJ7tuFoNW4O1OhkBfzM3VD+KSpmc/OfEbsnP6U9J3lXuUfGbqSC1PvYMT5Icz+K52SJCWFiYVO9qXE3tInebxcRr32rniHeV5yvqJgfRm/DfmQCVYDAfUaUltWMqjoL/obPHr7H3cOQl/+PEO2ZjLx4ESEbvmfBByMd1zk0U8/xaFUUnk9052w+MVGaBO9OafSsvKH2xE7bDQEVJFQ8yR7x3Zyf+znKNW1tFXnUPLzUTKOfs2hDE8ODIuyDqO6BKOvjco7bqEmfgBjDR7HMcgScZ1v38Grq16V+wrPeF6KWYHF9B5hJzsYevVBpgXIBsWL/nC7QWLckXFM3RLCyJNpJBQmYBM7SFXDN4kX+SoiB++VGzk7uomc9BwcSkC08lUwrBOOMrZS5NEFDyLkXUd7Q+LmvS/Sp/Rb97vvVR/h3KQLDDpezswtfZh1uoRe1fJBTiFpibaPI0M3uQcSuqY9iVtSPydwZRDKm6fjwQ2Lzvt0GIRoREU0troGUm6kIDrEbgwTEYvaQruPgpLpwVwcdBFRUrkdZXtr1vJlVBxFQc+w6vVVDDs7DI01nMtmOKDoxbHAW9lbuJLS1GQCnomnS1XlrtvmTjnw5pAkmpJl1KdDdLjfPflqJTMud9LLT0YHug+ndDucCp5gX7R/mHOMYHWoSG2YCY1JzcA97yA0vOx2OD/ZAM/tSeSuH0ZiqSrkfO87OJcyAwQHz4xcxOsDQjHcMpb8oYux1LVQNaCes8NlJ4S3wobSWkV2/khCrx0joF80P674iQ7vDqKU8LQ//DL4S6I7ZAZGbdxQHN0C981BVo6OP4okSaQf/5agsky8nMjxwLaJFCWWcWLGMOIfmcGJ0Vmo7XHuthaGvU+b8DE+hy4iSDYnklshy5goTMw2mHi+6z6C4wbx9QN/cGngJRQOHYn+shTl3gofAmvbseYWceWXq1hN8hh+MGUEK/VnqLXL8kMVhwuwqW1cT7uOUhTdzD6QnYZdzV2YqxsRHR6HwNmwlXwY8CH5kyKoTkhl65ytIMhjZVXrMRlCEFRKavy2UqLZjlUyue9tCU8le8y9SBGe/B8uOSWAutA6BMnKLX/pnOhaJYkBl7g1/W0SFUYe++QxZm7tjU+rD4LduYjkfSXnviqWnYkRXZNJMN7qlgJTHRzIM8k7GVcbRcaz0dRLc9geNtT9ToXoGbM8+16Kwj4iz3jG/bdeVa/Rv2g9IY7+uPQv1N0kNUBmnITWhjDy9EiW8D2CAIvTX2d45E6WVyjof7k/GVmxcp5L0QMO8RE6aTe0s2XOMbL6ZCEgEKs5CPlfg6hCFJWozWqGn0piwtYY1Ba1DA5JWM6Rqgfxj1/FFuXXeJVfZOhZOcDey6eQ96eMZmzkbhyig6t9rlIRVeGu6+bcp5h85mmCLvZj3KZA/JoNtPq1dpM/gzbdFVq9LmCkAaO6GJOy0r1mt+ovkVYKHy36jY3zN3N4wmFAwC4aWVWewsqsh1l4NpGNU8eyfvlOLg+4jFIpEO0bTZ+2U/iW/oJ/hxc+zVayMrJwKBzYFK2kqeG19F/ICD2CzQaaMUOpiovjRornsNJcfRvvFsziglnOMeGS/1Z0k+HpKSUjOyNvdTJHplWC0ctM07JHUfn/SppyJgAZapFp3kYSS+NI/aiR0Mpqt5TXyOhNfDlDdqiXGlWEV/uR2/IGZcHfIwk2fguD0xmv81tIBycX/s31QTIKtcHXIxXrZxzCAu1X3DHkHVr85fnuWh/9O4eRXv4Jgyzz8M48ga7lOiZFrccmctoNxffGcXKS3F6lM9hnFzu7MUhca6b8/3fkP8yh8vuoTRhB0eAFHB5/lFODPqTd1ojZrudKzSTeyf6VqLO7mLDbwagToxAczmBfzHzoqseEhqQSKEmycDx2HeeSbqbeZx+5dWn8lf0sR/XreC7jcY7MHkhDcJOzv0V6OVNJ/BkTxlP1AZiVtUiC7JB/b4gMsKl4/kmSt3aSNaCK6ojqbs43EQSojqhC8IomvGUuaeLsbsw+1/fqcDNtBUFEr2rjpbFzMBHCof1LSLlwkEavX6n0/1m+t6MIag6whUTevpHDmREX2bOoBp05HoBvLnzJjsrRfN+s5svaRU6JM4+s0X33yUEik6qCOpPMmPISggghA501usdYuRgV0bpe6E3pPH5sLbeFFvBlCBh9oDZ+GBY/OVeIVtnBH9deIabCxuoHVqPukoEjrtIv9CBR+kZKjSpsCgclsSXElcRhqJcR+aQ8jNTnVTLNUBVcR9HQMGpDa+V1wmFFKzTxSNx+nu2cyfzPxxFd0Ozuxz2F93K+NYqLvr9zOeQ0maMTUauWoFU6gXNOB4jWpKXkjhC+v/f7Ho6g7k5lNypcEDBafdiVfz+P1Qyn5mR/1BZvrg8ZxXDF3wCY7XqMVh8arxXT+9QafOsKsKmc5wIB6JSDPVctUBvWwr6p+yiLLWNI8THSQlJBVGJ1qHkmsIGuf63n6hCtu0YAiQGy4+vv6r95NPsGC39q5p1Tq8ko+wKAvIhV7Bnow6WOLxmV+TFD6t5wt2lwxE4+CgmnePpTGLVGYq9nkWuUAWQtXWFsqJzFOclK5qBKTPbvyPMdQVbbCff94QXzqHz7M1IzSzg++jgd3ma5Tefugz2Dud+nnWQVpGbGk5adhoAzV83wn/mzdCnBdcEM2TmAxogMjD6efLffXFzNuyc20x4xggsfW9gwawYnTWu6jYUr6O8Zi3/mejs84TCbFx3k6we+xqq1IwgQYchndMxfLGn/k4e/fpUV389g7LGx7u9x5YCnCFLU008DmYOKOTnyJFa1FX9fgRjf6zw7ehG69usY9UaK48qxOzznDrzjKG/v554x/5QC7B10mis1Ezljkn9REPY212IfoN3p4G/VXeZCwgKEGy+w8NUMmZXQzVH5T8cc0O28Jf83Jy2HK31OOeeW/J4EfzkQ1Kegiq9nP0NbkBpdqycndHqwc7ztMG/TNAZfGIhDYSRYkwudpaT57+ORIQ9yRhHKLdtvwa5Q0hraM3+Ot7kX6/s2MvFaPrjb7WGgVASuZQ3jqPPdSZSyP4IAU5NkZuCmZiWCJLDuwQMcmHKgJ7MPkcCGQEYeDUQSJBzKnh48ZfVGYnxzaJg6j9o+b3jYsufuh79DeN4rn5HxdeyevpvKqEpUdk9ATkDsAdDoXtx9LciAE9cwKO2y/dtkOEZR2AfkWvZ76iJ67KTohpUMLd3AjIAZVP1axb79nv0xWB9CQYEzSCt5bH8XmEshqOSgvLcav3O+7vq4umWJdwVP/347Xlc6abhDyYXkeRj7fs748WBW1fBn7ZvQIKE2d5BQEOy833MuEu1eKAUVVkUzp01r3WyNxNqn6FX7Pnrv3hQlN9Hi14JKdOXaFSlOKObvxbUoC28QUyQ/T4m2B4OYbsxOARGrqpHlPrDI9yIZI7LYMu888YUxBNf5opA0mCV4oxGUgkBxWi5GLzOjT4zG1eGBzkcXWqFkWDVt0xZSHucaMKdt2TGctPKPGCTIfo3u4EnBzeyz/Ruz7+dQ2JN+hJTcFCbsiOTZ9x9m+Jnh3diM8EZILtUZT3Jsegpvv/g2Fi+b2y4GOVfWmZE56FpqSD35Ey3CyzR7n6Zdf5VixzzClJDYFEREbhnNmjUUhL+D2O0cu1B4EwCjwZlP2akA5WPsR0LNU8QFjeX08NNUh1fTJbXR5pVJl6ocpcOHpoAmalIsrFu2Tl5XnftTSMvNqNBSFzeMbQtuYNKZ3ICWewc9zAnjUN5J287WuZlcGnhJtocFJb1UcLhXPr8q9+Od9gMlaSmc7VhGQegbzrkq180Q8hE3H/oJRans0xlyfoh7LKYkrMHoUGL68g4A9k/Z754LriIg4NPmQ0JOPoo/fnP/vTLsMvtnlFDtU8Pgg/0Ye+k7/LsGOsdSwSDH/Qy/fzjBs4Pp0nb1CPa5AluKhxWYXzcTUxaDADR7naHZ6zQm33CaIzNQ6tVUnqukdccJVF2eVDg2rTetGl8khYLWslY0pha6F3O7mdMfnaarpYvGv44QUNPAwEL5u8lJy6H0ntkMe3gYY75a4lZXcM29pHz5G8z+MxvzumPO+qpQiSpCFTBj+S5yZx0grz9kDswESUFmzVS+v/QpFcIKfnzuBfxf96cj0IhNaXPbKdHemaRr7EiihMNfolPXTovDRr36FjAkoxCURFZEErezjMpzlQhKq7NeKqyKVjq1+VRJVdTNuYdz4/TOsXEy+5xgBavD4lTf+Br/zhE9fmMoNNGx8xgKq+fM2x2c5oohdM936rp+cvRJ1IKWiJy/sCia3ffbxHaO1G0kuqiekqEKTo46KX/HzrXvl6u/sKntOZq9nXQsSZD3C68Ynt5/hvv272N2jcRd5UFcmNiXd597t8fck5Cw2C3YsfwbA13uG/nfpSHf8KNCTl3go2kgwquKmGOTGf19Pup33yDu8pYe8yOyaSkZ18cS/d0mYotUdGrzcX0TOksMZ6LhkcIMtF06EicscUv9YmmGxnMM1FgYe3IU9jOfsSFdS5vuCoIAvYNO8XziHtaMe4WlfZZypOQITRrZ7+HO9S2Y8BsejJAYj67TQkDHKPSWOGd75B/pqlQEVGWjNnuY2t3HqzuzL7h1GtFKGKHSU5XsTcGoydzySyozd850K08dL1vEBYsXd65bRsL1CsIXjcamdgJ/W7IwnljC9zf28GD/42SPe4WjXl9zqG8Cm+vfcQfb3IxKwUFw6zTiO5YQYfBI63YvOfU5nPI6xQfaDyjuKP6Pv8keex+5Iz0+T0ttM8El59F0NvX4nev9bjD9P/zw/68USZJoampC+p+iov9/Lv8N9v23/K8v3bXfXRu6yu5LZNNSYqvSEKoaMelMqAW9+xA25voFJl+uJkxIdzP7tAp9j+cemHyAdp9OEq+VkWXbRHDbTTRYdZR0hqNvG0dSdg1dOgcV0RWoBadx6NBQarzOO7ZwriY8x2PJ+xmVewqN5AcKLWiDOBH3JEtrYNCZSFSvFTOg6GuSHLLj77WCe3mucCSvVL0CmNGYnPJb1lACdJW8P2UUywc+jEMHmf0z3e3x09ayMvIit1wZyM0/60koisOnzcd9aO4udeG7bQevvPkSi0uaSVFPdP+9yCZRekOWy2sP7k+aZgoquz9r2+GV2lRmZQ1j+4gkdj2UxaYpL7oTvrdb2mk0NpKTnkNWv3Lu3ziY/X0iWZD2DprtAcwzjOSp4XdRvnk7dePq+Gv7ZnYOFjmeKrPVytvk/EBRFi1as4q8fmFUR+vRm+NQihZ+nqrgmxjZkPXqSiZQJTu3bu71OX19SwmceZRdtxYSuekLwgqO98gfICKSnS4HX8YeHwuC0GMzsejSOFR/N8dW9OWNhc+QF/EqAL+2w3HDeH7vstEUVovRS0aOKRzeeKn1pFg/4Ka+j/BJ3DSao9u40asIo7aAmPp7UdoNgEibeSGVn36OTpVEQtOHLA5+m+j6u/Hr8Dj7bZLVfXAV6enE92vxA6A1JNm9IT4XAOvCJKY0hhNZrmfU8f6k3hhJaOtM+qkFbvKCk1ZfDptwOzgFSUGD/gQ7Oj3PHhqxl8vjyvBOjaay9yQEBJ4ccRsPDL4Pmw2+7fc1AVXZ+NfeQN9W575P4dCzuOsIz0bs76E1r1e1MSl+HaHRX1FVeZgV34xi6NmhPSQZRMmGtq0Ox01DeH3V65i1Zje7Dkkkv1c+e+f709QvGEnhj9oahL+qE1sSXA45RmVkF6KkwXd8PMO8P0aQVJzsgp+Ugxg+YQOpv04kNvskN23pyw0nynS+N5gc8LwzrVPdwGkcXnAr54eexyXl+08nQ/e1xNVGBxb39eVeG0AS+K4VwlV9MYy7g9Bpi8lOLyM3cRP54W8xSAM/hIBfTA2nh1/EQidh1WEorHpESU11RxI+qjp+aFSQG/E0bfvPkrIzDoCk6hd4Z8xZOuY1cvLG7YQVncZqtFIeU4ZNZeO8GVa3yHUxzD3Avk+zKM+YTmnI95xPnE1lwO/udcHuK1DaewQtQUGobIHozLFYlPWcTJ/BqX4PEDS+L1q/jYzNvY7a7u+eL3m98rh8uw9WbyVakxZRUlJvjGXF1jL+uPAuNRsGIxa20uLXKo+jpGBX/v3csm8zL/Y/xadPbcBUVsiWZZtp0cjI+kZTFHYUjL4wmMEnBEa8czNn58qONVEUKCuTv4Lxh8fT74F+pMxOoeODfpRH5dN3gHxA6aSOM8FnKM9QyE4f59wRENg94Vm+uP8xLp7bweS/znC9/jFMgsewtKn1mHzDcZRX4FNXwD+Lb+dgOn0CudbPRItfC0pRybHSJXx8ei3vaLLwDmkiviiQJz55An2nU76q/gTeVVvRmbT0yk0k7I/PMFxxSQ4CgpIGizfb1K0A3LRvBJK53v1OZbdgn4uRI3Yzu0JbZxDVdBsGKdJ9GFIpPGv5v+qN5I37iIrbn+XqxEfxD5IPtmqlier2BPY3q7ll+y3k9i5j+83b3Y671TPSeV/9An19LHh3aLhl2y14dehpcaRA+HToqqN/1XpWXx9HYmEYWUPqsKgtKJwOywHBW1k3/HXK6n7B6GXDrJHXR/+YWBj4CQm+uSwPtKGyqhhxeoSHtYXE7eFXSfCRnzN1x1gZ9dvDMSz3Sa14kUN9E9gbOcZ9TcLBdQtcMsP1tEIuDcpFLegwaoro1N3g5pBz7A6HSEQ3VND97PP3wcWHCTKfwre5nYagBhyigEXZyGUzTK8EBAv2S2/RmnmOvAFp7Jy1EySRwflb+TjxJDuGfsb8jfNZuTqKhOwy53ipu9W9m8ksCVhU9bzaCBNy04nPm8/oE6OQHA7saj2+qkDSS79gpjKSUx1hFHk5sBoUBFZedS9GNR0JXKyaxu/+S3hx6jbavduZvns6gqSkd8XbqGuWy/2os2BOLka0Sni3+6Cwe9iGXl3JDLfdRewnGoacG+LsRztRhlyW9XueRP9LmLYdwBoQwo5JbzD/dBgNhkM92hO8t4bpfzvzhkhGZ2MlcDv5bT37Glg1Oo2np/SmImg/x8aeoCS+RM7/0RnHp2d/pkbSsn3pH7T5mphyYAqivdverDJwQ0hk0PpbmftpCFXiWer8dtClriDcUMrV2gnkdbnsNUcPgMYjzrRKZkUjRqkZh2DDv3MEQ81D6eOfxbbGGDLTsmmJ9+TccbPJnIdxtVmJpug6gWWZzna5WuWSFeoW1EDAZDXw0elfOGMezaVhmRT0svHkx0+Sfq23fG/qU7DExjpFXxL/nMOczbMYV76L8Ja5ANxoHMFL2Xdzb4MFo6NT3o+6Mfs0Gmj1usDBftE8mTXhH3VyDYdLxlMeiw/772Vc9jW8TH35qbo3J06nom2xU54+ja6QGAQB+oQc4etZqcwxQH1IPZVJfaiNH+Z+5prL7zPjxNvkxZXw3cMXyO2dS2puKi2WbdR1ynaB6LARYvTivm/uQ2k00xDcIPdj1W7mOgJZnrKGud5B6OolEq63uMfpRNkilp+9A+P1BNTNSlIr3yW18j23TJBr7i38ayG9Pq90/u0/M/s8EscireYQvr7wFUeMUUzdO5XwciudoZMxaOU83hpFJ8+Pnkdw/gSyR6gpidjN+eABFId8Lvfp2L/5ttc3HMyKYd6f/bll6y3M2j4L0eKUbRVsdFgCuLMqmg2hZYw41MGrr76Kb6sceMisnspT+05R0jnWPUjmffXdnHMiD3/+MDM/iGfK1jYySmVw2ujoP1k1bhazk99Dr2ojPEeW8XN9U3F+VxnhfxkV4NWhIu1aKN5tFhySjVCvImYkr+aKmEyeXzulca0cnHyQdp8uuQq+6RAynrFV0eRdTeSmrWMYeWqk3KMCEHsrE4IyWR2YQNcQkSsZh8hMvBcJiZTAMzwydCWBukrnvHTmne621gW3TqN/0TqGifdDt7FwfzWCSGlcKYXJZe4+EATYnvcIf2a/wLDQ9bQFJQBQlFDkRpQ/s/8EWzpv42QXHB+fzf6bZGd1ky0ZlLLaROy52ygb3EKXtgu/C7KUqigCDhsfZe7ngeyFznni4HSvCRzKSKRFf5EVW0t58/hWDyvRpfDiUixR1VMTsIn46/L1qoiqfzBLPQHOkJZZhLTM9ORUk1TsiIAj9xyl07eNGTtnuOfuVxe+4ebfJVokEZ1JR/ShEwRUXHKvJ0qnM/aiGU6OOktxfDFIIkWd8nff138roiBxNbyCDXf8QlW0mcLQD6jy/5Puxe3cdTEmusl4evYpj2PJZldT0BZLy4047vlmEkk5zrw93cZxivgOA4p+QWPxRmFX/BuTvbPtBrtPpdJ2/WsEm3z+OF19mDOVZ+nQhPNwxwBaP17BY58+BoDK7ue+V8JBUXMRlxqPU+ezhxa9J7+661sPLfNj0cbbiaiUnZAem1WuhyuXOECGzygmXSljyuUawlpvIaL5VsZPHw/3QkNQA14qZ07C2v48+ZidfZ9kk7kqk7DqsB7MPhcg1jopgeLlJfL78LBbnvfLYVafMtoH6WkZ4wr2iigUnnr9/NpaCgfMpyjJlZCoZ7/9T8wWXVstkiCwd3YuhUmFqN3zy7VPgbrgOj7NrmCfhu7MGUmwu5/r+qZc4KMPmnUENfgx/uh4QupCMOjVOJBzQ83XGdELcHDKdb657xsEBPp4T+CPdhhTDj+2glVvx5KU5g62uaUjuzJIqH2C3sxx1qFbWgTntyUJVhwOj/w8wJ8dcNnoy6jsPsQX+HK1TzZ1IXXuNiOJhKjjMfpFgm44GaWrGat8ip6CtGBo0xJcKgMuVJ1X3X83OyXHg+uDyRnqT2FiIXaxkxKxjPGbc5j1u4MWTT/877uV6wNjnHWWx96/cwTTd40laaOOYxOuUxvuQK3QuNsnIJKdkc3ZBW20+LfI9q7TjrWo6ijwXkeTfhczNicz9NxQ9zoyIW49Icp6rgc0UprQRFjbchJb7iHBqy/+xR+7665SWInNyWLW2kRadfL5Ykvuk9z8u0St7l7irm7HHhzO4Uk3ODjpuLveT+w9x73Xn6NrotzP9317H0mF4+WzrWBnZPRGRuosNAU0URfuQ8Mgb6r9ZBZdrd82rkc/w+XIi6R9eR9Wv6moJT953lmDmWr7incnv0vSR0n8seSPnsE+57nGdov8XrPGjEVo52TqCE6mjnT/zuGAAXcNIOzthzD6hrn/nhhrRWk1YTPbWXZwGbmjPIB4AGWgL4+WPIpfnFwfv4YqIpoXEdlwB0Eds5H0gSi1SsxmGSBY67uLLrUMcMvqU0TXnEWkLUxD6iuvI0pBRcUTFQhVD3DaBF2dacS13MnXQ06QWPMMIIM0ng/Kpp8GdDN07H34EI1BjfJehoOXBw/kTGQ7//rmXzS8WoW+U0+xVU2Z97Ogi0AhKvBt9SUop40fhv1ASpYM9BIllSc9SpcdhbEN0dbTn9k97dC/Fed315KkRt07HrPayI5BIvv7hrt9QOD4N5UL0aFGZfP3PKbJQqtmK5fj73D/zbUWxRVH4V9mdwNrXcW1pnT/xsFjmxa19OVAvY7w0hgCmnsxLH8vfUpl1QiNwkhQ2B8sO61hT0oCru+4+/rn1zmEKGU/ACzIweDLNZN59chOqlT96EjqR1ewjoqIPLc/0lUcChWKTiOL/lzkTMcgV8qrK4VKG4iiRGNwG4oguQ9EERj0CfR+kg2BlaQWJNPvaj+0JjU41/NnRi5mQfLPLI4dxc29bnZ2nPxcldN9t3ugF+Gf+9C14iZyxt3fo06u/hn9QTLJF3eQmjceg7FPNyl+V7DPtS4KBLVNYlUAnIozsnDuGI4N+oqswZXkJ+d72MNeZczRtdAaXk1pooqfRnzO5aRn5DZ1FBPbeJBj0TBMC71P/sTIz3Tu8XMH+1Aw6sQo5v5cy4DrDzK2/jcGhA9w1/3nyz+z+txqmk3Nbhtw642tVHd4gEpYzERe349vbR4OpQapGzDZXFxF7LVd6Fs9QHzwBEn/X2f2/d8u/w32/bf8ry8azb8H+1zFHBKNcd4A6kLq3HnPALTWcLS2MARJhcO5IWoVuh73Zg7MRLDbmXh4IiIKQtqmIZzo4OFtVVxrn0XO6HvIGiwfENWiHOyTsMsLqF0k8YaekCx//DuGy9KTXQ3Qcg2NZGXMsTGMOBWPqVg2/sc5XmP/HftpaXiY32pTqQpsYOtSFXXx40mueokk8yDWzonC7lDxhuMErz/7KlvnbHUfIA3qRob6luOtN9MQZuPHe9dyYsyJf8t/4tfsh7qmFpN3MGqHH3rBs9knl0m89Pzb/H1nNGV9Zsla3M5Nx0uA+BovJu8fhqnyBnVeh90syu5l0NlAFn+/AJuiiVt6fYZgbSZMcZopMdt5ObGV2rhKWrfWMn/jfFyb76ny+QC8eG86mVPuJ8C8ks/TL5FS9QauXCSLAusR6Y52BwmBdquOzxsM2JVqOmJS6PIK7JGYXBBEctJz2Ha7wJcPfvkPVJFIzaBs1p94CaXRlfTYMwfKNdE84ejg0NAzZBwKJyk/CVFScvvpNKqa3uT+5D/p5WPm8OMX2HjrDgDiax9j3rUSxliH4W/So69VIra3IdqtKBTQr/R7RuXKzJ3wqnD03+xj+OnhIAlueVJXOTf0HE2Lx2D2CkAQILR5DoPKoNUusuCJP/jxgXzKb74Ptc9XLBl0M5azeTy85yyrmnqT8PdMlqyJ4Zn3nkFj0mATO1jXDh9GvMBfjpt49J1bSTrlRcr7K+j0l+UPon2uE+GTh80GcZNkx0tbUAKdfj3RPd0DYzciVnEhcR6fTs/gQnsgn5wy0V58EZs2lIiWB/HvkA37Dk0+WTxDxrGvcZy47Bkfp7HSq3oVE8oOEdZ4M70ab2LqpWIGFK+n1vsEWzuhyq5lzVdzuWVTEkPfWoYyZiRWpSz9t+n6JgQBdO11pF6VmLpnqjs3xMsB8Hqg/C6Hq+IOTxC0e3EZG/4do0gv+4zohhWeYF83Zp9KJRDUNpkNZVPpF3iC8XHrEQP82DPzAnkpeXh7Q4IKVvrCB0MqODj5GOrcYu779j68jTIy+PdrcjByxY2+jN2vw1IsG0C9bvSiIPxtFh8dSO7JB7hz1nC2PLQZ/2FxiHZ57pol2NAOee0R7O705Pto01+m1n8bnZo8Isq13PXTXQiVEkfH72PjlDRa9ReZlFVCv5IfUdgU6DvUWI1yX0R6yG4IkoIOQwcdwQKRP5xg6t6p7nW1uSuc82WLURdUYCmtwqdVL+fNQIHZ7kVTVyhWZMdNzTAFfyz6g6vx8mHpi3PfM1xVyIzSZFJyFOxbuo5pXw1l+q7pKBQiVqv8bpPOhDG4C0mQeK/qJk73Ho9d2dZjzBzYiMi/zOwts91jbVO0Y1W2YNNDZWQlZo3ZjTCt99nH6eRR3Ah5DNveTQQWfMFjWaPcbfaz9mbBjs/od/wo+b2NNAQ3oBSVGK2+5NSPZrctgvP3rGPnLdmcGXYRh9KZc2jEWlrGbOdp5QQW/TUPZUcLxq5LmJW18vU5FYw78Tp5vs1cnHaF+qD6HlK4rsOSq03QM2DSc27KE1DV7VBdpt0lX1drsej9EJyW87or73DPjnya1GZ+X/w7Z0ZkY1Vb3cG+a3XjAJiglwMbhnYDokNBuz0GqnfD2ZVEWMoZnNhGbdxQrg2swa60y4ydS08xKepjXi8diP1yKhYNXOmXI89BewT0fox+Iad4wBdafVtlVr0TZNEr8Bwv9d7AfWM0tGXIzu/xR8b3YPa5UfzdDjweNrbIAA3cmTmUKUcfZ9rlVu4M/hJBUrLAG56J3c8gScs9r77CM++uYOHugwRanAeWAR/SZkhnr/ff/La8itMjTjv3TweNDthjhD5Rv2PadxjHz7sZv3kXsw9uY/T1M4S13kKgvookrzoinN2vkALobbybBKt86FuU/gZ3B8sHH401lJA2WQq0xAYKfSFzy+aSekKFqrQQhcWEl9qL+PqHaGwZhtJ0C4MrCrBFB+LTUExxyGoA8puGEulzgyXNv2NNhiNTDnBq5ClESUnZgBXc25XNwfRcQstCac6NY8pfCcw5cob08s+79acSu9lGW2YJM3fNJLR5DjprNDN7fcnCtHfpE7UBq82EXW/A5CXvgy7ngwud7n+plaB6+d/Rqv6ENs8mpu5f7uvdc/YBPDNyEck+17l43J+kC4fdc9c1/waF7+Enr1wiBtzg3Phk6pdPJLHuZXmcC74HQeSYOAy/mBpUFpFRW2THCJLIY6OeYXHGG4DIq6++yrTfj+Pb7OW+7tr/xW4sgvFN6zmbcQ6FaOeB/DHsnLWTxiQFd/w0hjEnlxGqj3DPL4CnP3icgP1/EpO9nTxp978x+6RuzD5RFLE6tBwpuR2b/SwDEl7j+JhNZPbPpMW/le5FFEROjD5Bm58S75Is/nORaGwEQ1cqcbUPMSp4FqLosW1dDu0S8QDHhNeo95FZKkqHlyxT3w0hqxTNLEx/m+MtUeg3zSDulIkDfaPYHBeN1WGmuj2JMxWziVIKeHd4kZV+gU03v8reDlkm2GrXsH7o23wSJPuMIyvljeJS8hYq2yrh2BzEP3WUJtuoSy6k3c8pOyUowCuGYvEuVp/7hiy/N0AUMXp7nBoT49fynPYMy35ZRuReLTFZO9C3VnuCmE4nVm7vXMwBKhKK0tGIrgRr3fYCwQb/YJMpRQu/Rh5Ef+8fXB2id/cHwKCI3fQNPUqzVwfbpr1AeqadOZtGuyXzAYZEDmFuyN1ElwczMHMggy8OJuqGjF6PNP/IN7N6M1jbRUedP4Y2+d3O7Zk2SxA2h5pgdTbWsGhuDLuD0WtWdBtekbqQOhpjrVSkTMRmkKU27ZKSBmMkX2WuRSpsJnuylTV3rXEzwu7s9xzvp3/AC34CqdmhHB9zmSv9rqBWqIj3v8L9gx8i26bm1eEb2LB4u9xm19qZ8jBMPkyhXcPC9bfR6WVk58ydIHnW1nivUoaGdTDi1bvIzPiFar9fZJe1aMVL3UJ9yHc0mx9lyBMablt/W4/Aq6ErjaimO4gTR7tt9B7MPue/o4pDGHZmGAqbx/bKCDlCuv8uKlMn8/mTh8lLyXPfe71hFPu7FlBvhzFHevPy6y/j0+qDVuwAWwf0fpKWhPs5aISctFxyhyo4mzyDi51b4dKT/DBZTahaPqc4cGDSlGLUFiEJVgyaJp4euYSlBsk5j8zOeaXqMb9+vmcPfz+dxblh5/4jsxRJZGjBdoYV7kCvlNchpcOLmc4lqXdefwZfGEKksq/73uSAczznZ2XSxSE0BDWxZepKqv1lJ/vq89/yMVvpkiCw0Y/pe6YjSCIlxrEw+Tjh+muUtGSwqMNIo3c7Y7cfZdyOIkqDPeoc0A285mZ2dXfEyv1bFfgHWdatCILMxr7t+DfkGVq42reEqNre9C6YR4Dak/suXOyDVpxObXSIzLqyeFgeAOej/sXvN0JIuRGCvk0+2y7aPZERVzMZWOXNRXswbWozPu0+RJdFo7J7zp9WZSMj/kjkX2fGcq7XdK5HPftvfT3kcB9SryUx4fgKvs3I69Zal9y5J9inVejRWaPR2GSGqtuZ+B30u9IPb7U30T7R1Br2cTVyKaef2EjeV3n0zu2NgCfYJ6JAbVYjniyFThd4VtGj3Uk3H6N6SQCS5HDfo9F46m3Sm2iOzKApsNV5v/zwioD1nAlbzoWOzR7mqSSiEKx0qapJuLAe9a6/3Q56V+BL4fDGvzUOv1YDTXc8zOlRV5y9oJG/S6cDunvORtfe/K86uLdqHq9K5aQX3s/629aTn5xPgkGWBfywGRZ3JDNo3yRu3tKXZv9mkERmhzxBnR3yLWkMuFzKozG/Otl5HgZ69+Kaf1p9d6UUJyBFkGU89eY45gQ/DcCPbTDS0MjURQf5+7YbbJ29m4LkAnebdZZomeUnOQiq19Dv2mT6inNR2QJ6vHfRbzKAuS6k1Q30Bfi7M5jB5V6knRyBf40Jm8qGWV3NibShHM1Ic3a9Eu2AVNr8nQzK7ra6zQImC1OyS5mcVYZBJZ/lTJoy2jUyaDE6U834w+NBEBjU8QqxdffRqcnneOCd5Ea9R0VMG62+re717a6t5fzUfA9BCogtCWH51zFMPn0/WtGLy75/EnhpLO03S6ivF+DVYeXgxIPufcZV1N5qKnpPQlVWSGL1IwyqKuFfk6YSHy+nSNHXv86jd3qS3vWt+BulwwtJEnh+9EJ2R9cSF9CPE1MNfHnzU1yPlsdDYxKYuWMm6qsC2phQ7CqtO/BtVtdgdpjY/chucu/NxVfji4/Gx/0O17nLdJ8J+yE71RHVKLqB4kKKz9Jv34fUX6tF66dFERzYIzCgvZ7J2NPf0ni5nP9UJEGBb4wv8ZPj8Zkxitr44QAMKFnHkLxf8WnQU36qnHOv76FV+JLzyTOp9d0GQKehE2tibxb+uRDbMDmoq0CFn9aPi4WP8OX3b5O4/V5SDh9HvbYDb3My6cHHGBi+l+fqBlOUlYD5mrlbTmwFguDgh5z13FXhT3hNOOa9HYw+MRpBUpDWehecWcFHQ/8mqusoTQuXofZW07t5JonVz6G2Bbm/C69TXoT/9gnDjna6n939vzbJRpeqCouisZuahnytsbcK/0eXUROwBwQJSXC4ZZK7M/tc+5woqZmUVUqwbSCJBYk0J3Ww/ebtPXyorv3up5V/cH0csqpNt7XPtaY4ukmVCwKIlloWpL1DasBF4kriWLFmBTF514grTXXnyZya9B0/T1jKl8G46+t8qmeetE1noXe3s4wAOlU7C9LfJHbIEUrHDWbzfefYMuNZmrw9CgeN3scpis+hfnR/Orw6nMAUt2FJTDH0Dczm9yX76QqRZbfd63nkLN5pC+bkOJn1mXo91d1ms12HzaFk14Wx7vHvzpgXBNlOH356OMrVm2hT7qYo5DOavc45+0fu2/xJ8jkxqfFXxuVcReHMKxpX9xBTL7Wy1OBR15AEG1ucQP0Km6xKdWpyAVf6X3Gv8/sLV/JQSwJ7btrPscltnG3ZRqPhoLyHhU+jVCeDgLdKIahsbTTFec5rHnKEKKsqSKBvq/k3hv2T+57kod0PUdUu+6om75/MUx88hbXZs+cKnZ2EF57CtzYPbXs9urZa9zVVchw3ht9Be2Bcj+f+l9n3v6P8N9j33/K/voRHONzoMpfsgl0wUe33N9XCbtqvHieoIYhWe92/3etwQHL1i0y71MFjvT/7t+snJ+Tyzb++QeF0sPYLPcLyAY/THvgZ5SG7mLBdzbAzw9CIOgSHCkFSISLr6U/YH0LTallaQBCAkl9hVx8GVK1lqFJBs7+RhK/vx6w2EiKkMzlhMoGigQRtGyvWLmfmn1ZC2qaSUvUGhuZZHK9/lPdPr2VrJ4w8OZJlf3zGOPvbAGSEHEMt2MkMj+b87ASqw+V8Pd214UVJSUBTAAqjkbo4WaPahYgHSFbBQA20el2ixm8r5bZL1Pnu5vNguNR7E38qAxh2dhgjdqcTWRGJ2hbMrIT5zOo1izDvMPpn9icx35/jo48DCtZcfh8pag6/ddWwNudhQodnseuujdhqLTJi2XngaTRF8vbxTXR6PYyhw5/kM7/QcVRG5lkdWk5X3waAUoAbkS9yxSLLMa25/AFLNxoZdbSIUfnP8Nu8L1h/czqZnTs8bXYuYUYvaAkwo5K8ex5Os5+iz8mPGPlrLpEVkW7j6EQUzGnaiCCI2JQ2TN5mbEobjYbjlHRe59E6WSrhCYMCVbsChc05/yQ1I6O3sH3IahTTjnHh3jr8/vqBwOI9bmSSgIAgKfBr8UNV3sS0vdP4V1s9sbYpnopJAi1+LRwM+Zi9/QLY1PA6kmAjzwrpRRn0LpXzL1r8QrFqDeh0YDElUdI8lMGh0xlyYQhenUr0iUEk1D2DWpKdZHalDweEseR5teHToOLUex9zIXEe5YE/8+KhQ7xxdAeb7Xez+tYv2bd4Ertn13Ciz2wq/f/wVK1bsK/J+zg1/ps5UDuEg80R3Lx1Nj5FFgpGPIotcBlam4zW61JXcjnwG06OvobdS01CYQKiXe02wHxMfYhsHkra+bN4f/8pQWWZCChotYvMr4aXpGf5I6qAnD4NOBxygm6b2MkcLzgcbkH1l8Cg5y5THWGl35V+CJKKjJZn2OQ9haLOSHZGgOBwkLDnKyb8vRmFTeE2oG9EvMK5pFmcrj7qrkt83SOEts5yH/bs3Zh9CgUMz99HXN5mvrjwF6c3DcF8+bobcDBIvZTNHbDfCP01ABKdieEcnHgQq0p+iN2h5IWS2Vy+1IvYYhXaPrIE7OCrdyJICrwECDYWEOFlpCnkOtc/2scrb7yC0qpEJ0C5Df516RHU1SGMfjuFtGPfYhdNzjmoQ2URCGwMBJPkkeRzowQVJBYm8tAXS7j20DeknPyJQF8PwlZAAZKMWLcPHYxO9TgjbhzFX1vNrelvQtoattwWQFeEigdX3+HsbwU+6ga+G3M3BzV+JBSGY/FTkJuai6RwBgCe13FK249rt25m95wu4ufIzi+lTYlSdCKhUXB22DkuPJ1Fe107GVkZaE1atCq1u26TDkxi5JpaAmpK6ZPVB1d+1ohyf8YfHo8jRsMvy36hMqrSnZPUomwgqriT237ypzPWwJ5pu7FJFnebHYINq8Ybo18EsY2PMCk7n9uSH0Kr7MDq0PKpPZ29RrD6z0IVso2MqvXyOiKq8Cv7jScij1HWfwbr7z7A9wsfpc2V20JyoBbsKIELEzJZ/dBqzF6evlZ2oyE7/qHTD9DsdZpqv820SOUEtd1ESMtMfLV+7uu7DQso1c+hru4Tynye5XTXt+717eGh99CaLNIVNACFXYVvi697P/jkzFo+sT3D35XeDL6QztW+V+kwmFCIzv1AaYCJB/im4wzlGdPp8JWZe6IgQmsOKtHM91UZCPWBBFXbER3yvI4Ik+fYqpPrWVQloLaoyevlccqNjpHZBzvzH6Y1fSRZ/W5QE1bzH5l9dmdQo4eDVVJwKQYGnBrB8GMKks/8AkCj4QgrfUAUrESW2SmLlhkkgfU2lDhl9lKf5OeQe/mp42d6XWvkwdUPItoVPQAkX99YwcfxHgdR4vVm/IwyG+63rNewOhRId27ih4eKaI1ZyJjm7xlqfgGQcy0tCpAPN2llnxLTIKORtQL8GN7FHTd/QNNNt+JzfBeOpgepFmTmws+X3yOrdQEaRSeFM5R8/vDn1Pvucddhb+E9AORdTURviiE3NRdBUtGiy6RJex7f4HZEQWTuZpklZvQN78G6tiqbKVCfpXpkKztm7nDndzXb5b3uR9/3+WXJWprHzkZhdaH05XVvROUGplyuQ6E10BAkz0+FoGRI4Rb6ln2DtyUJ386BbsS3a+6GGwo405zK2WuxpOT6c9eaOxlwaQBKpUigroIxMRs4ZvMnxwI2XRiGsDFobZHy3G2+DOV/84h9Dd8uO4YlKtI9/wREdtz4F4kBl5jo1eFuo4y+pYeT1i0Zhp1Zs6C+U1YE+DnlCE/4gSQ50NiCSah7nOFhcvDbFWC63P8ydb1D+XXpWvY6PLlOtdYIgsyD0VliiKt7gJSKt4j1SnFf76PK5JkAaAqvZeucrVREO4P+7YVQvY9fbFt5eHIW+k4bIVcPue/7dOpALt+0nN/DQIuN9etlNkFG+RfMj72vR7DPJRtYIu7nmOJV6nzkuTI25zIL8zuIdMggBkEAtaKLO/q+zNSAMrbM2UJlbxtzN44lvFSDIAhUtPem0+LHm0EOnvrwaQafNNFkOEqtXZav9VY3Y7b6M8BvBXM3z6HPtT7UBdc5pcgUcruA36xmrg49T0ReK7ElsXIQ2L8/meqf2F14Nw22AJSvP8a5sbKDNCpCycS4dcxOv0j+3O1YDA5CSi8iWa7/G8Pl/NDz6Ex6Fmx5mHt8Pewlt0TVP5h9wfoynh65mD7aZmIjQwmtDSDt0Kvkt77izo238fpj3FwlA2ey+7ZwdthZ3KCC2qMMPD2Dx6fGkjtIdiJaVBYOD3+NnNqrpLbfB8AFkxf3fH8PFhV8+uinGL3lOTp2tJUHh/yL2VFT2WaYRXFCKUf8dlPru9P9jWxYvIGjd1aQPcBBTZDc16cr5nHX1gpeULxF/MXNBGa30qXrcn9TB4ruksdfkOh1I4TgOn8ZgCGK5NSP5vmDh8lsC4MjoPuyiqA3dIjf3WDNT29QU1MDNiOxChv7Zm9j57wDVEdUu51zbI6isSuWh3Zfo2bTVZ78+ElC6uQgSU79GGo743gpaS8KoxygVtqU/yMoxR0E7wZkEQWReZvmsfjXmUzfM51xPIUgyFL8v2a9wajC3pxMGU2XSpZ8du1TGkUnOkEO1iXlR6BwKLApbQiCAtJfgpiFtCY/wrIbBpIK4qkMLabedzdNtjIIGEi9KYGlYdcI04ahchh6sGVTAs8wNHIHY9pv59XEvSidkpLiP4J9HYYO957dPYA5qPAvZl5wkG70oPe7d8mSkru486fxSI4Qrk5+mvmGD+W573OdhWnv8LK/mcS8Xoh2Uc5v7Vx7gvTlzFZs5o/Er7H4zeTEqBMIiETqr0D1Hn4vXc/JMpmtmHLDs/Z03ye7VFW8XDCBs8kzcOcWlbo5JbspbpywfOFsm43TM2bwQ7IPRenxpFzUMf7UvfT2HeQZR3cAzBtELyLMk3qOv6jFe/oxzJouVHXvciPiZfe1us46BAQuDZRlz8bn/EF0wwrOLClg3LVr+HeM6PEssZvT2a9jONMutRM74VHKe7VwePRabnSe8rT93wBCPSXSO7S5VPj+xfmS89jvsZM5IJNBEYP4Y8EfRDQupiJ0ExVzzCQ9lERWnyyQPDKegiAiOkRUV6rxKpb3TFU3u21q3TQySiHkj0aiX7c7+1JEo8G9n6Se7o1fTS4qu6xY4sq11Ox9hlL/tZRaMt12yM+RLWxZrCYzaRYHJv5OU6zA1N0T6J+9knCdDHqJr3uY2ZePMHGPN8riPJp8jrvf270/JMHuPuOLgkhUwzKumsFuT0YoziY++zSiQ6RLa8ZXDEd6rpH8OFimN6K0qvBt8SKu6hZ8jQORcBCthGtxeazs9SM+YogzgNBzflkUjTR7nabGIa8Vc+0beNbWSUzD3SgdPvgbh+JrHIjDAd7mFO6MeN+df/iZGgO/aJqpD+l0+3RcwT6HYGV1wzzizzxH2vHvSD7/O4LVSnLNi/gY+xHYNpHJ0bMQ0NIQ2Zfflp2kMagRgDAFvOf3KNfjDFyaMIGKlBi3hJ63AG8EwuIht/OhScGqutRuqjhOgI3YTvbwQYi3jSHh7O/41ub1YLLZFZ3ElsQycJcP44+O517OYbDFUxryDVWB8tk5IzuRuGI/Ijo/YXSIrOTUZfPmlZBVfHp1JLM39cGm1mHX+2B2GGnxPkOL1xlePPoUQtXvaDvbOT72OJJCXmQyQo6wfYlAdXY0x4Z+iTU0EkkQQZKYPx8eelAizu8K340XCXTIdrJVrUcfKQfYQ7xK3fX/OPUc3sZUBIfgPk8pbQJDLgxBnS2yOf89ioLfw+EEXQI0SLnUFdfRfKiZwyGHOXbXMfc117dnl+xY7XJfKoVuNqlKR5d3IIJCxNxmxtbU5sx37yzBIdRE9kPl703uthv41BfSvVgtEk35TWT+kIm1oZXawIuUBf5Emy4Ln/pCAn79ktMfn6Zy01l8WuT+Uji83fNI98fPrJu0zn3GcAGtJ0YcZmxtKb2KW1G3N2KqkYPzyYHnmZzwMxU2b6b/vojGexuZfXYByWV3o7GG4ZCUJE+5jc1mNe8++y7KpV4UxxcTo5JQSQ0QPAoftS9qKQhHaDzPtz9PcJ+XSa18Rw72Of2WneFyVCeo1oHGEuYGQ8Q03M39HfW8O+JHjqans29AEB3aPOccdYKeJDuSBC3OwJJ/xwj3WHaXsPb4A+TxMNjjiC/pS+yRYLw6vXow91zrV0hdAMM3yIEv8KTCcY2zJ/emM9jXVcGd/V7gs0kz2TOkluPT9xBUfpnEi395Hu0MVJXY5CfENKzkl2k7iW64q8dYu4NATvtMIdgI88mmJvZpzgQ+4AYzxMd59rQrcXdxrtcMCgbb+fDpD2lwrgOu9juQV617vplF5pfPUxj6oby31R2DzjJ+MgVR7t/IoQmHZEa/U+78sT2XWLg1jw82n+ZcpTOA121fdfkwVFYViqoGVG17yYl5jAYnIM81VtnTyrgwaxVm7yCioz1tVUhaVA4fEpVjSC/9gti6B4hoWkz9pf3EnrqX3ZvGseyrZAzN8v7h6yv34QND7uflQB/Glp+gX2Yiy9Yuw6tTL4+TQs2e6McosEDuhVTUnR0UDZd9RUJ3Zp8k8sPdP3B6nJ6msAQcWHtISqqcsthmuxws7vTqpDGwsUeUSFFeAoBF70/85c0knff4DUWDF+1BCdg03j3G1zWXXMHo/1eDfYIgEBYW9h/t6v8T5b/Bvv+W//XF1i2nk0sK0aJs4mLSfK5r3sC/VI3e2FOisyDsPa5FP0ytLQ9RUqGSvNAp9T30opetXcaII4nUhNe4g4jDIrcyv/eneEc+QVHISwgOCUmQCFRH4d85EpXDh6utx7GqrFRFdNIRL1IU+rGskx00DPq8SkTbeR5YeoR1K06yw/QVewf4sUeUkVdfjPHhyMC/IK4cScStbzxzjg+jH/mU0xUyC+6m/TeRkNtMFMMQBChrTed85S2kFP5CUsMSknOTCGwIRKXs6UStiqiibP50Tg7+hpOB9+DARnzN4wCci4bfGnox8mAnhrLP2N/5AUq7D4lOwNWh+Hw2L1tHeGkQvfJ6IUpKfpq2ke1LtnNLyi0Mqb8dSZAPcwJKjpQuQxi7GYsYyrrrT9C/DFodEupn43jzpTdxLS+39PqM5MDz1JQEElSWiW9DEfYmDyL+5+vfE3NpPBYJOrX5XDXLCC2V2MXE+LX0at1L+PG/8G6VK9o9p5UgiAw/PZyZf+iZfbyI2fVneuTsm9/8EeNnnUSUBCYemuhm9lkliLJU8Ju3iuFVMVycmE9JfAkmtaxPnW2Bt3IXsPNCEjevGsfSX+fJ88+hprBpIF8UT6bACl1aK+X9NJwY+CTfVN/tqZdTKrFz9hjK0qehJ5BYh+fgLEpq4kriWPj5ENJykuVcKIKNEAX4iArCqsOIKtWD0Yhot6JUymj2lMDTvKX+kdD7A7g07Xkmff00/l1TUAgaXgqAp8ueZ5pwgpJbZAeh5kQ7Y/Yr8euVxfqF0bwwZi51XON89Tksiibadddo8DmISVPmrtu3XlHcV+KLSVWBJNgZr4MT9UM43hbOwWW/UjFGwz+LIImY9CZOj7qGoqyZZb8sY5Tl5R6/UXZ1EFAtM4SMvmGo1bg1yW2ShYuDL3JpYB5/vfQCOXUv0Kt6FaN9Exipg2ZJxSGftfy29Fc+f+RzBEnJsPb3eH3hPrqsoYzXgQIHmjaZDRhbGus2alu8zlLnt5PKjlL+WXSWKGLq7yWuc5Hb6FzfvoJTKWN4atwkrP7voM/Jo/7YNqbtHExSfhIJypEMu36Gb079weSD8Sz6Yw5WjcjxscexO8+HUxJ+IlrdwvqbdvPX7c343TWHprkr6Ih8AEmws9QAse0XeaNGRXu9D4bEUBojMtBYExmrg+J4ODzuOR7K64d3vRZ9W4072Kdw6ChN6OLDpz/EHgzTNxvIyMpwO7GQRFr8Wrgw6BqitwbBksdPqsHYBZN7bmZcy2DsJx0Qk0BH6Dh01iiC9BXc0fdlPur9GkMGPUKTfwsFqUF4O+5Ba41ievLXpPnnIGT1ZsFfkxFrjO7n6XTQT/8dZapIfrN3UR9moz6zgqawNrbfst0tFRXZtIRZFx2smbiP65uvs2DTAvxa/DzBPkmBVWXFqhXIHj7WmQNAHsfICn/GHx2P1OIJ4nV3fjQGNpKdUU1T/0Dye+X3kCGTsNMU1Zfa+OEkXD1HcIOBAG0QUxO/Z+2cSHqL7Yw8OZKowvaeCLS2GxAwiI5B39EQPZCGYNe6JR9MhGuvkznhaZ61BjD1x4lynqRuTixFtzyL9n+wo0Bmzl5MmkeZcJSBxb8ytGAHMYb4HnN00t54Bu9oY95vem40bkYQZEdARvBxipvGM6joe+5YN5kJhye4pdW6bN6USMlUYac+pJF2Qzsg4K/Mh+j5MPkwKGT2YvS13dz2ZX9UFpV8uJuwi1UXixBQcHTcUXIGKph4YSNP6bN56KY62NWfeJ88Ci0CAzMHElcS5+6PfQX38OqRXWTVjUdfnE1Br2Jy0nP+IY/mYou5gn3djV+RYyZY+vSvdPqGo7SaEAQwaot4pB7GVzqwqmVGzIVpD1HVa7znVskhIxltCrzbfREFDX6dwwGJmV5gSoRehmL2BtTwzWsbsWgNiDYPajG3YST9jrzHl60eNHt1teeQcr5yFlcqZzPHtoGATo9U0U+hEKuCv4pWYQmP5eLwG+T2OoFNIc+VSfFr+VfyFMKtV4ne0khIXYjbcZsWdIIYH3lNPLJrOhP2DnL2kZIaUzm3G4DMpxmvtVAVW0ZtnwQsej/Ptw50avL53j6Zb2/6hAtDLrgdC79nreKmgz9xvgv8avWE//YxAy4lyr3svF9HABpbMPZb5pI/YBCjjO+RrvXkxB1Y+zljrl9kUds5Jl0tZUGYzMZ4fO9FHs15lI0LNvLn0hv4N/uh7dKiEAX8dTUMidzJVbuBOe8+wdDDBTi6LCBJ8jc1ZDUMkaV+rtWNoWnGMi6OL3HOBQUFraNxSEpClA7efPFN9s8f6AY3dXdie/KVyN/Vim1lfHHuOyb7VxKxbSoBhXayxz9IY3T/bmxANcH2vhydXE7hhGSKEoug2/yLaVjJLbXnSa55gejG5Yxsu400RS0Buip+nh1Jl6Rj/reTeODDe/Fu98YdQCr5FQ5PxYFAjQ02zv2G3279kpLgrwHIqZfRzosNoO4WfAanA0HwjIkr2NdduvQ/lbvO9mdPr+Hcu38vaVozd/VR4fALI6kgCZ82H/d3tb9oBcfMP1IfN4iwSljw1wJ3m+/s/xzhXuUodG8RVi4jes8NPYcgCfI6NTMLlkrcU6egtUuFrkMiuGU0MdJYuW4CdAX/ykcNQRw/dTc+zrU5LlZB2tyH8AlsQzH6IkWTmnj/6ff5e8pNdNld+4aSyM6ZhLbcjGPKeCq7f8t0Z1naZSkshxqFqMBL3cLI6M2cbH+djcfPoiCahsByrnuvB6CkpS9rrrxPeXUA9319H5Lg4PzQ856xyloFXbV4dZ2mNi6RV199lbdffJvrqVewI9CqGkFJSwZrIsqIGWaipHeoMxemwB9/wNyMr0gOvMj2DmiwHCGw7Ag/7FlFYdgHzpo7QW7c4HTv8RwMnO+2SXXKNkI1Rv5e8BflCQ34N/m7x+lk+UJuOZDDtoLX2X1rDOeGZ6Ez6lCKIm3mYLafvsg3937LpOMCn3UY+cNu4ovWTorWvk1sVCSf3BvDqYgSksZcpiTBZc85X2xIotkShFG4SO2+azT7NdPp5VHuyKqdwLZ2FcfHHGf7a1dZt2xdj33KqC6lxncbZY6zuFmW3dnaiDT7N9PhYyN3xHLStLK9PDpmA29NnIS/1xnSrsAz797B0LOj3A7+X+cF8WHAbSw1gElrpsW3BaOXkUBlAWS/CXlfEK0zsKz5DP2uZuBfp3F+MyIk3Mm15lt4IP4AByecZnDr2x4GgqTk0WErOF85gx2Xfqa/4SYP8829dsrt828wcNOvAxl9+k5CtbHd2iSzBbrbA91VRcoswSjy4wmvVGDTeFRlFqS+x4joLdxU7cOPy9ew5u7fe7wvNegUifafSVWbSMxtQ2fSIQgivbz3QPZbaJXtpASe46PGBCYdks8Mx8Ye67H22YUuso1HaPI+RnTDSt4bvpaQ1lnOcek27t1K76DTiIJER80yYlqep27cAhoj+/VwvhVIeyn3+wyzTkddykMMNX/Q4xnhHVf4KkyiYPwJLve/5N4nb/aCN/1MHPfdx53D2jn38FA6w0cgSmoS/BMxdKW58++5+7ebnSSiROnwRpXUm8OLcqmKKv8HQEhuu0JSk1b+ManlH9JkrWTHYIEdgwVKgldzKfFWftn8C4rvFYTUhbArfxejfhpFdcBf2JV2aiZ3MubNMQxMXkhU4x3dZDwVdGm7MGeE4F0sj6NO7TnjTAt+lim7v8HvggmHxhPolpl98kPG/z2epAsbmHHoY266Uk9Q+0RnG122jh3XdxOqlP+rFiUuDr5IZ5Q3Sdck+uU9RKrPEPd7LT5BlKZPx6xox9AmgznjhNE9+uOfOfscghVfEX6Mfo/h/X4lZ/hUmv2bUdicDE1RDdHzqUTPvul7aA30ZunaYSTXvECztZrbDRCksJEYcKkbc7RnfvYGn0OcTB3JDulBQJZIVAt6REmFlzmRycVnGVqwvcd3I2HnoyAYroND9ZGEVodw79pvmL//BFFessqMWV1FpnEbuSmXaQ7yIWtYL6rEbMJb5jI25zIj8g6ydtp28se8Qad/FEvXjXLK90G885P2FaqJalIyZH8e0eWylz1YAS8FwO1Jv9FX48COzR28cAWcKwN+51DfeA7Uv4VPfSHqrrYecvoAYTUysLWo/xxEQewBpAaoD2qkKTwNg3k4gVoZSJHof5FccwaZXi0UxzdwbdQcWqICaLU1kK6GtmQLC1rWs3nxT2xalOl8kjzvR0fLgRNvezXFMfuxe/kQm72bpAt/OFlGDr6Y3h+Ak78FyXWIGdRjjWrtCuJ43WOYqlu46c8DrHp9lXvuGL0cvP/0+1gSHMQ+2YXQ/gu2bjnO1+oGct/w+whOD+bUB57AO3jONV4ve6GdoiWiMgIFnm+mMSqNGyPvwicxmLNfnKX22U/QtncD48fFUzFgDt5xoex5dA+RuQd7PN9qsvFlypfy//H2ptrnZ67Gr6TOdycm70DaB4xm5FMj6fvzYxQlymd+Ob2KLCErqVQUHypGf6qZ1PIPmBv4Is8deI7KhPu5/Zn1NCxu5/S8AWSvrKLR+xh7C+7h7m2FFJj92T1jF8pkJb5/KuhX+DYZ4Sn8/DOM65fNUJ1Vzu+5XEN+r3xCRFA76sGQLAeqLCbEqnI66zqxedyWbtuuLb6N2lsfIm/kI0y5Wk1GuUxAUDj0aO1BiCjceea1VllmObb+AfoV/0ycYzIAJrW8t4e0TmfmdE/QXwbiOJBE19xWu/vjxKjdXJh1TbYLe9jQ8r81XWrqennTGhSJwZT2b8w+SbD1kCPFJ5XH9lzk15wnue7bxKURZ8gbOIyrgxTU+G0BYHveo4zfcolnGuQ1xLurNzfFz8DQlep+v03spNVe02PsU4LO4KXsoiw3lolb4pm0LpGMrAxSevX0dfq2+BJxtoaIyogeQJgWr3NsDYc6fRAqq4q6kDrctl/eajhzJwICKquKspgyasJruimGCBCwl5zUibx/6n35gd3ksV2gvONjj2MP9afvRe+ev+mx3svlYQ/p1l1CFb2Jr3+IsNZb0FviCG6fjI+xL20+bTSEdDF5axrDTw9n7hz5uZVtKTg0A0jM9aHvuSoiKyNR2FRymywt3BHTn7EVsDIimy3zjtDp4wKACN2CfQrqQ+opT1JyuP9I1ieoOVJyxF0nl03WZZNBbadHnmbNijWofD1nTSkwiObQFDoCoqmLG0pNkkc96X+S6XTNpf9Lqer+1xRRFAkLC+shm/5/9P3/V9763/Lf8v+hdFo8ycgEu4xady2qBUn57Hu5jqy+WWR4j3P/rjLgV0pCv6TB5nHwyzmWeq5ISqtCzlnldJCdq5Qlu8JRcv8399McZKCsVyQRml5YlPWYVTKy36aysXVhHh/e+Qo5MU85g33Doc8qTia/xewqiKj0weusDcEhyMZR8a/u915Ly8GrA/QNp+jQ5mJTtCMI8OLoW9km7/G0BkUiivIGc7byFvYV3Y3BWEbIuR0s2jCffldHo+oW+Lq7sZEJOc0YKi3E3Sgl3/ADKkFDesXHRDfchZ8CNny8mJQbcfS92hdRUCJKGmZWwYSsT3ik1UpOVDmb7zxE5oBMwEPnHxQxiMCxT3Js3kKaA5oRJCXpISfh5G0MUT5NjKGIuOI4xuwZw+k9XTgUDvcGPCL6b0bFrufQztVE3TjEnnl63hp6D3nhr+OjqWdxrycYa/BILLlKRshRHh++nCEjT1DfbxIDziYQXxSPWulBkIko8O7wxtDaSZ8jq2UHvCvY50QOps4+wpY52zk94jSiM9g3oRIOGyYwSVCTfGgcoUUy6tfF/NOL8HflWI7p5XrFOA8O8uH1MjNDrhBRH4xfiYq8MQrye+WjUWg9cw0ldqUdU58E6uLlgK0oiPQr/om+Jd+jchhIy5GlRUSHiEJQYFE2UhkPu6JzmHtiLDO2xZC46QO6WhfR4ChgbOzvfHjTSBSWchw6HQ6lGqvTV12l2U8/NYhIzGUP1+OguL+M0qsPrkepUNCl6oVW2QmSgoe+eIhZv2Zy95cJRJdF95C87BKaMEltstEs2DgcBR8PfJ8dGftYMaIAi5eEd/m3dDi+weREarvGWhIcKAcNo6TPTEbYX3A/s8FwmNyojVyYMJLj/zLw1+R0bCPfJkBQ8XkwPKH4g8+DQWeyEXBUQ1vbUVR2X6aPKoSlEosdE9h/Ppmwaj8nGl5JuzaHfYX7+DDvcQYdfYcJMbeSN/85MifeRoDtNdLLPnf2rzwuXTYZsWRSl9HofZxOTQFe5iT6ln5Lv+ZVbmOk3JpJk+EEfQKuEO9XLzNxRmjJuJZAcH0wCkFJjDWJYEFPe6s3cSXRCEaTe9wBxsb9xpKQc3R6d9LhY6N5zRYCdqxzO4UOmuBk3FOcyorj0S8eQu3vRcfU+STUPU6JFX6qD6WgI4IbfbK4eFshl6Y9j8PF7HNoCegYQ3T9SmLUyXi3i2i7tG7jXkCkLrSO/dOOE/POEj58+j1qVFe6zU0Frb6t1KSKSN4eB0xZazrvHN9Ii1GNT1UYdpVAS+KD6FV3o7LLcpdrb6zggYBCds48hu/JWp575znUFjVeXkDO2wzpuoZPqw9+TRA5MQVNRir9i34hUT/AbXQaGkq48sVRwsaEseHWDbT4tbgD+IKk4Ni4Y5xaYgBlILOSbyOyaSkA2X0rWf3AahxGOxMPTsS/yb+bQ1JBRXQFB6dexxji3CMEEY1VPnS3K4s52CcOq3QJQ0MxCmsXogiFzQM4VT6Xd4Jg6vGpDD7RQNK531FY5CATV16EzKfoCFyCw27n/8feW4bZUXXtunfVkl6r3d0l3enuJJ2Ou7uThBiEkECAAEGDvDi8eHAPQROIQDyEuLt1kpa0u7t3L63zo5b0Ar7vnH3OPvva17Xf+SfQtapqWs055hjjeZ7YnGCCKoPs9GgZr8nzu2YUkbnhTDoyCZXBvg44KZ0YlLeXsRX7LXor/CPKzSzZDwc9s8oBjkw6wrXUa5wbfg6jRt6/JsdsJMQ9j/VX9pF88GN5/KJEvJzkQ/+agQ9yt2ovyX7juDy8CZ2TDpVehZuiBMp2Qlc16Zkb0XUNJKD4MiqTM0klX5LoJgex4j1P8EefvUwxu9D3somASj3PrUzExUkEYwfuqk761z/B+anXqQmosWVllrclsFfXzLFeo/C4/if9rssHO/Efgn3WoEZPx6QgiYwpB//Oeo7M0fLjyh0cafkcwawkzwDtRg2zXMBdKZEd8SGZES/TqZTXILYoeCT/USbpPEjICUeMnkDfso3EVD9LvUlG4jbofFAgH8bSxk7m+NQGyr3lfXlC1I8sDL4EyIdmk9CFQdFMbqG8bhwpXElRxd1EG2cgmNUYFM0AnOqCH1rhdZ9l/OE9hfNj08nrlYfCkqEd4p5Demc4V7z+jUe+TqZKtGS9BroVMC5qM1Lyy2SELkGUYO6uubZ15H4PGNhykt/8G1DPjuLM+CyuxtxBg4fsHAlyinHQRZL70IoQU6ISJDQC+FS4Y3LSUhVUY/mNhcbT8vmYQyNQOc1lkP4Z4jWj7c+yDI2T2QutPhw3lZft2h2BZ7jfx0SDXycfPP0BF4ZfQBQEipv7sGpvEXuMQVQGV+Jf2Ybzx29T437A7lgu+Ba9Oor1P32Gy/njdsocSeRcwXDu/K2FbY1+GFVG9BoBSSF/H2/+226T2JB9gon783051M+X+1PXcro+gZpLA/DP6xF0t3xSapM3K7pvMjorzZbVKgqig8O753/H+VwhoWwkw0N3UNcZTq05mDLXVkSzwNzdcwktt+jQhMyEwRtYq5pB2sZ5TDw6keLIfFvG9YbrnzLw8A8o8qDZZEUTdNCtrKZbav1HZJ+5B0rJWnoemEs7s2nVZlPY0pt4lxrGSb3xClzGGy+/IVPUCwJJfqdZkvwabURTkjyTen+B4MpgW4DpauUMdmU/iXOQB/mp88nqp2bmHzOJLI50CPYsdHuNfllLqOo7mUDjZqYJH0P9ZQbq7+boxHv4VOXOsAOJROdY6NIFBaqQ8awqv489HWBWSnS6dGJSmmyUvgpJw4Sa/Uw/8iLirWxa/WMd+j68/n5G3L5IXO06plScY/p1HcP9plHc3JdZW8xc7nyBvkc+JDT/Nt+v+p7awDrb/VOjv+Mbf3AyqmyoZFtSQfLLdPZ9m6P6ZXhV3sa/xh+V3rL/CAJpfuf59+m9ZLfNYfD9w3HuUDNv5zycutW4GLMJb3ubpq4APmlyxaPFg0HnRB785sEelFwiw88NZ9qmgazauIqAaplnPMrzBj/NDSYnvhx9/0xSz0Sy8LeFDvuBh6E3/ZpfIrZwOMt/mklcXhyCKFBUsJ6KnBc4bYajJollwARgGXBE381pk5m3NzXw3MGBOGftYu36B7nvuwdRmtzkb3ziSc40x+OWNBFzeRdZiVm0uclj5edcSmNXMO/VazGqjBidDBa9XHu96twPcTVuDqfNbzugLK1FFEROjD/B7vsqafeJQFTK39TNmonsz32YWJUdpdY/91VG+c0B4FSJvLfXmzTsWHSJzSsu4tExkDpjf/BIhuJfUN58joXJG6mJHExghWyfW7VwLzas5rmjp+giwKKLZg+Q/5L+GqfK7pG/JTN/c/BbbdbFmyfikyfRL+sOBnlN4a+lW2rjQKoT+1M06M3dtr9HqRsoe/AnDs1rQ9Nm1+k9Xrycr658wS2DE1KXhoc+X8GYk2Ns829i9A8AFCofRpDMCJKAJInU6mUqLi91KX38T7M0sBODkx69Ghq9G+nprrE+S8KMd8dwFsQtx71L1qwURUcUoLUMCZGTKE/fnEvU9R2YBQVdHoEOwYGr5u8o83qdhGs38Ky+/bcs/Pja3bQ2uHMrpJSsxCzbnH/UEx5x19PuMRfjrudIPClituxxggArU9axf34Q4T3iJ3+V5AAwmyVbUlRPZJX1PQqciK55gpiap2g11fW404I68zGgv1tPRUiF/ZJVe1EwEO4Rzn2BX9G74h0HGk8EMLuq6PYwIJqd0KrsNtwTwjhm62UKuWsr5X1KrxNRq+39fGr6aYxKJ0wqx0RIOyrHbAuMziwOZtYWiS6TJQAQ6cbtmU/RGJzk0N+Sswvdbr74nTrPzBO/sEh3hAAxyaG9csDJqkurQKeq5pSs2sAm4QpVHj/x8JcPk5CTgF5o44+iUxQa4H4xjzAlVIZrqYuQE4y+LHuQi92wtQ1erfbiUMNXsgPXhsQWHNtk3Z/+MkdsNoUZjGI7tYYiutUV9HWCoU5qVv7yGXO3hhJQWoBHkwKtyi6tYcbEkclH+HNeKzumLWWP+Ym/7c0KfRcRGQfwr/WQmT8AU4990c2rlZI4P5meFKg2wZpa2Jh9H7d08jcjSCpcpEBURl/bfdEF0Tg1widP7WXXlIcdgnkKs4bbSfWkP5BCU3CybOuqjzi0uzKkGr3GHaeuZtvf3ps0Ao3YzQeRWey48zh7R8Xwe4w3pV2ZqARwEWG0WIPWp4WELD9WbVyFwqhg+HDYk/MERz02MbVEwXNvPYymUKYSzep1hJu1aYgiHC+6m9dKh7P3vMxWUe3+LYc9+2AWDNR0RHHXrjr2VnyEqOmhO235HiRRoNOlE32ImasDr9Lo3YivqS+xVf8irlKWoTCZTczfMp8F2xY4jrHFNugeKq+HA1IHECTYaYytyEmzGUIGh6AZOwSj2p4QIQhmOjo6MBhMTP1sBuW9Jzo834SC4c/IZ5Gu4xdJuiWv+9mhz7NjXAC3hwuEDg1F5eOB3kk+F7vq4ulT8hUJ5e/QPn42KmcVqlo9MTVPM8X7YbZnbsdJABePDso8iqly289XRU9Q7bmbLqM7oe7ZXIzaRf/RN3F9z5XRvz6AztkLpRJ8mr+HA8mciKxhwYHfCcqTnXQZ3VouBlVC3Bp2FG+gXvUs3ju/Z98D+2hpvUK7Ux4SJttaZzAZMHv5oHf25K/FbIZOo93XaU2O8G0bS1jDPXibE2SVEtHCwiJpWDl2Iqv9fiSy9hGLRqY9eVGU1FyPWkaReh/d2m6Sj8cx4twIHBNB5P8ecW4wPhUCsc07iKx7+G/BPq0+jGd0eqamtco+NpUzBU2p/JzxAlMrockMhXEt7J/0OHlB/7bPA5QWa8GaIGR/c4Lvefr0WcWWjkUO/dDYFczVxgTMDZ5E5XoTUOJKTEEMf9XT9WjxIOJCHdP+nIZa58STT1pbpGC2K1QWhiBKIsWRxXaGg8RnIGQWO70LmXlyHMt/Xm7rB1GEmb0+Y9eYh5jt0rNG9mCfrNMqr3+dk/tyaWST9W4APDoH0LtsPbO/mMDA/a9x23soM/5MsD2pwfU0NyNWcbLzM4c2L0h8m/SJj9BnTBp/zinCt9ad2OKJRLvL91a1x+DbtQvvmhsArH96Pe1uevlcU30E56NDyU7pS4FPPX41XkRdsPtAetJ4Onc449psRGEUHMYXsPmSdRY/2T8VU0g4BYMW0+4dTkNYCnURA23XOs7dYMD+1/CoyXW456/Ivv9Ti8lkoqCgAJPpH3Q5/xeU/wT7/lP+ty/1nXLmjsLohkJyYt06eP45KweHZHPa9He30yTaHCdmE+Xem7gRuYJTtbscnvvzPT/jYoxl3fvrbHRweY2DeP7kbn7tMJOWkobJdTQDC3YyzvtuWwa5WrQLN1tLz4OvzjWG9twEFm4dQtjP8t9EFGDqpFEXzj2ZM7np3sjFiWYupm7gZHJvcto2wem5DAvdjUKAHXfs4OqQOgqEg4gihLtn8cqYmfQWj+JRlok+pjc+fEOAtocYl8EFhdkJ54IsUq+nyu+1Il/MapZUQfni3fy24A82rN6AiMJGN2LGKDuD29yoCaqj2asZUVI6bM6iCDG30lj9zWpElNzd53ko+ZUkcT3zYn7kzoYQhp0ZhvlGmyWAJN/82qk/CHYpJzb1K/bNuUaLRzO1hkL0ygbc1Q2MC/2ah7x1WMOW1k19fJTceV97XeNGrz/onRVBUFWQgzNcFESOTjpKxqAYaixBNWsxSyJfaJczrgJu9rtBflw+CrMzCsubWgNnMMk0kAMzb+Em+RNfMwmtXs7sTQ+Hnwd9QG1ALbvXHeXTRz+1zCsVZknEKCkISk8kdWsgqlbZ+LIKegP0bXiVpKK3EarryfF/iBOateiEJjw7BuPXMhWFpCUvLo/iuGqKI4sRBZGB+Tv5rWQakWodq6dmkdcnimsDblISnoUgSlS2xXG4YBUNQy9zOn0S7uX7Ofn5lzRrztOqLGRhNSy2aOmePZNMu6eeix83c3LcSRSiAqMqhDjvyyCJ+DbYDzndmu6/8Lhbs8XMSJhYZaHlLu52ZVsbqJpNJNysptv0E23aDACcxS48O1xYvHkC5upa6iMGIirtk6fc52cuhjxCfsx1jgU9hUnRyU3dLjTqVh71hBSxiMGXB5FyPYb1T63nZop8eLEOtSiIzN47m7m7JuNX64coqchy+ZLg01PYmPoUsdXPMTpoGpLaCZNzLIFtdxHUbEFjWua4NWOpwvsXLiSMJj/wHVv9elKXmiyIo7nHtjDeo5pXEhpp7qXiw6c2cXnwZRSigkEh+/l6wmz6jyjno6e3oqxtZtXGVah1spH14YVNeCq7+NbFGf8qBbr0PDAYaVfKGZLv+MCAiu8ZEtLEtRHn0Eb60iBm4tM+htgLZnYdrWbt3mImDPuTZXM3YFaqMQly/WO1jdwtmhh7/nGGxkzm9wdKZESPZEfHgXyoNTpYWJbxkBSUhZeROVnEePYPDK3PUuj/MTqTM+fL5zPi4mDc1j+I6koPYWYgvXYc391+hGzfOtL75dAVqqEipAIJBR4ewLhD+KlUPH10ObO3u9JZ04ZnvQe9c4YToAmzfZtuDYVkfHoShYeC24m36dbobeuvNchhxohHk8g6lzfpWyJr1XQ5G6jzr0Mq62L0mdF4NnvajGzbARYT0T/nsOKHFYiCSEjDXbb6e9UbUHe1cWJ2OFf6fM21+rNk1I5lS8arOLdcJGkNVIe649aQTZH/B3J9o+6hI+k1hn0YQNLxl5m5cxgDrw60Hx5G7eTFEyepqN6NanIAXs1eOBnsB2uFKBLQMovgjhk9EDr/hHLr4VhQOHpPMpMz2Td7H0cmH8GklY3nnbef4ZWTf2JEpNUnnPL4cQQbd5DqK2d+T4v7hoHs44fgVIZdvoN5u+cRWjebAsMicAmHUzMJKfmWe01KzIKZipHLCW25j1BtHNScItHrIO5KPWqjkvQBAvWhKXJ7/UfC7DxmxGxkX79djMyazPCLM5iuesc2vhcn3c3Psdc4PaOCkI5B3PnnaSJd7QcelckbtcGvh0Oyp2af3B9O3U7ohXzqPA5TabjNiOyLhLVO5CVpGbsDRBbsmkv89UxqXL+gW2lZoCyBziD3Lq4NqkUXLCMk3bqSuNQNd5aEMif8AGfyhrHmlUUMObiDEt/3KAr4EJA1+V5L+I33ffaQUPEWpX7fcqi/FzcjV1iu/5vXx00hXfyOoylB3IpYTXDDUr5pgZU10Cs7hlUfTSTS4ghQWsaxX8AxuujmUtwBCuMrGH5+uG0OnCy+i4P590Pmm0wd/gW1QTpMCpPNIb2qBt4wJVMS8DXSlXFM+UFA05yGztJmmabI0YwXJAVqRRfjo37mj/GruM8DYm4EgUJBUXSJ5Tfy8/PdvyU9fA0lkuzM/K8YRv6alTk4ZC/PxvzCvd3eBFRpbTpySqWA0exEkGs+X2gzaFy1lSOTr3Kj3w3qPA7Kz2+5DRFLOej+OLT9ilvGJaJu+1jGX2Tr4l7sXypyMPY2Cw9OI/lyMb3KnuPZIa8QH+b7D8E+A+2mBnLiGtAou9lz/UdyRr2MOO0+2juXcTU4guMV+2x1Vygg/NZ+hn99jnXvrXOYfz37oE2TRYZOpND7DW5UT2LdkQtc1o+nZvgV9s3+k9iCWDya3eXfe6dC7P10iVpEs0wLJ7dHwM0iQycqlTYMxtSpUOYrz6Nvqx9waI+Veuqvmi1ZoU9xMnQa5Vy0/M7IF37wzMCnuGt3A3venIT/zh22dqhUAi7qZqK8bpDb/SsFAR9wdGYrnz72qc0uPF2yhFazkYKap7idmItOkIPdHS4dcoJa6Q648Ryvu+eQkGHGvVJ2oosi0FFMmGkzDTpPTrvXU9S7iqbgIawJ/4r7Uu8DXT2uJSLLP3mEwPPO+Nf449Tt5OBgEARwaamCnGyuRE9jf/srtmtafTheHUPQ6iP+Qe9DIErzOVKUmQZ/N9vcAegbcJxHBz/EguhGtj/+GX41Gubummsf58Dx/GYM4rndd9H3wnXWfLWGF956gaEXhqLurmZQbTIbZ0dzo2UpOdcTCC0so9+tfiQXvwcZr6M215BZN4pCg5oGnwZ0ThJpKWkO69mQS0MILfLFo8UDhcWpn+h3Fq2qg7QuNU1muDW4hGsDrtmc6A8PeoBN0wQSfY6jMOrITCygOrCa9qZGcrOf5aBZxxD+uQwB9hnhw1/SUG7LRdOhxD/0TkIb7fvf3VHfsDG+lfVPrefUmFMICDz+OER53WDtkPsYqjXj2uZK/23h9LnVx3GcsNsUiZVvk1L4M9FuibbryR1rmXSjlqSG56lzPUS56QqCAIVN/XF3quen4A62Lf2VszNnU5EwwTaOn13+jg3tEo8kdDH99EYW7n2F0RnnUYtASwYIIrTmMCbwI1SGDtxb1Jb5J0LRJsYFfUR+4wAMliQ+my6upKT/0nVoXN3xinqFQ/XfordoQMfHWROL5PZdHpaJNGo0ZYlTHNaBYr8vuRa9kCKnnZhFPWZR55C0BiKnbgABAABJREFUsjnyO45EmEm50kjC2U/5qVV2HN6qGc/hwlW4KwR8WjwwiybaXdtBErnnHsjSr6Yx/iV+r19PQUw13k0ReDQPobxrBEQsZazf21S1x1Aw+DoB771OXpInfnV+WPeMELccjs2PllHfloTGv61fkqNDVxDgQN4anjl6guYuEc/aXHKUL/Nnf1duNp8C4PmR87kStYM2tzZykp0QTXqQHB1T5V6j+PpYH5Z8vxKfBh+b/fJ0HUytcaYieietN8E1/SK3vR+hRZuG3xcC+7w/IL0xFbXD/iIHzEBOwLsReS+lW1/k3jdGMvbE2L8gqywBux716cncYEWcGJyNqLapSLmRYrsmiQZEk8jghwN4TXiN+r2yDW49W6wM+ZDU/C049RvIkjdeZPr1bmJNsx3aPf+B31E+/zQlpfL76+ssSBBL+6+NvMaNqc/REhDvcJ/gUG97Qou1BfN/n0/4/iJ0GhVGpQlBlH9T6fU7R8NHkhX9E80jpmFwG0KUNNE2zilFP7KALaiNvkTUPUhyyRcku4+gwf0Ef3TA583wcXMalYHpZKRUYHaKooEcZm6Zybq0HWR5jGJY0TM4GcOp7DUOgGCnWE50wZJq+NN9ExdaZWSZV/tI7o55Cp+2cZY2OSaoXVS+zW5hOU0uFx3aLklQ47mP1ZnR1HruZ1IFVBpNDB54koawfqRNfoa6yEEEauXoZC8VtmCwPdAmN9jDqZZ/j5uAsisf/5IrtnfUBMh20AhLvPCgfjyaM1fQdLTS6COzJnVJ8FUL/Jh7H9Ummd47sGU238RVMajA7hMafHkwg3bH0OJyhWbXyw7zb2TVDkbn5OMk+XA74G6OSS9RoLVT2AG4t7gQWHSRDmkLJR3ZAGy+9W8udMn95l3vwuRDkwmukPWDb+igfym8a06hrU2Ld4MzHi0egMCECfDlplgihg5BL5qp82tAUijZsvQEhyf8QHN3I4Ko4KOLP/NLXTLnRsh6s6nXEzAZ7ZTBzqoWTrqO4oHSBLbep+bVV1/FFrwwg3+NP0ZXiT9m/kFVcBW9ExQkVLxJRN1qeU6UBHPixAnuvnk3/zpmT959ftTztDzXwlsfvcUr0ivsfXAvg8TVtuva1kqCck/RUtRIzKQYmDoVg7aH5l9lOQkZe2lKLydiQixtvtEOfWlGZMKbE5j6yVTEkEBqAtscrhslA90t3XRVNCJYaEQ1+lAi6h4kpGkxJmc3/tXxL7Rz5eQNQZCDGT+1wvBb7lzRHyI0XyC8JBwBBS6qJhSikSPtYRQZAG+QwrXoVZ0IChOoZBvjcNFDJF/OoPzhTCZeX4Jr+whim9fC6TlcqD1MVsxm6qLV5OzOIa/hEU726YVR0Y7S5IFzdwxRlVH47Nr4t4BIk8sljmkf5MfsT3r81dEYN5vlb8qqyacwa0jwTWC0xz34tI9GkuRElsG5B9gyezcKkwuNrjL9r1O3E9cmZHJm1BmH51rnyfnhl2kZ6BhwBTuCUxLMjgh3jDirWlCJBiKKI3jwqwfxL2u03CU/M9LzFqfm9uUVbxnZ1+RygS23f6BFKyNYJ0b9wHuJ29BYq2NZ3DJrR3GgcgRl/dLZ+OQFdjx6i8OTD//NHqkOrKYh0oWw8jACGpJwtYDslCY37qiEjX4N7J9XQ1lYmT0513sABEwgWuuDi3o0oiQSmxdro/G8u+8LAGRZyIO0Sq2Nfcoe7FMSVBmE0NZBVbCVHUGum1t3IjE1T6GqsrBzSHWUtOfZ6t2uyaHM73uu6DbT4HqKVm265T3y/L7LHYwqE6XLX6E57g0SvORkhsauYEqlMDImFrB/oadMdW61Zz3kBBD31lvcjoCEnHCCsi3arz1oPJdpf+bdfe8xfUcnEUVyEnTPPrUi+3QmHdFe0cTlxjHm5BhMnf/PglOilwfNgb0xaNwc/v5XZN//qTSeAG1tbf/3P/r/qfwn2Pef8r99ifGO4dayOlIvyxtXaip4edgDPkbklbknPNbqNDZJRppcL1Lm8xP5bTcZnPsnCeVv235XH+TG9dTrKCy/VwhGqtujqZTMHJl0BNFswKWpAoUCOiwc2p2mNpQGJWs+HmQxniyLZtFmODqGab7BzDn3LgClC0xIgiQ7s2tP46aq4VJLGP3T+uNbA/V+Mtd031gRKv5gZ/azzKgQqAqqIjviIy6JHyII8oG8zejE9tRX+WNxHR0TZtHlHuDQT6IILs0VONVVcXWgrBdkDUL6t05jazt8G3+LnIRcupy7UAhK6jz+ZIIWVoa/wDfNUaz9bC1R2bIxKkh2fZyipiLKmvfgXVmMW5sbgqSkplOmBLvMBpwUOl6dfY09T35OaGETd2+6G6uoeqfBg8v1/Sjzqackshb/SgmfevmgWNkex6WaJYzwyWaW+V+WsZPL/txHef7WGo4pr1AYep2v1u7gyqArDuPc33kOofUryEs08NvcDznvsbaHY1KgWB1N8uaFTDos0+GIkhMf++iRln7NvX7evN33ML3LXiLmvDtTij4nqEnOYNvRDknuJXzlD02+TeidA3DujkGUnKhuj6HN6Ex3ShaZU2tIOGUk4XYCTqI9GzS5dR19cxbjs/EPln0fSIbqe26qvuFUcjK5wa+RVP8SgutqTt9ZQYNvA4IgoDWE8uuFA/xe8ztnDI/RErGcA9MPk5OQg0qhILt+OJ9d3sjC/f+i95Ef6XXjGuLOOor8PiBRd6/c18GzeFy5nq+O9aXPSV/cbouWeS1S5/s06y/8CpLIJ2s/4cKEMDauyaLOv84BPdAzcCIJRr6XAYJkdvjQ75VnCDutZ9M9J0jvk86dC+XfHpkxmT+DBLwb3TF2NNOivUGjkNvjmQqQwKveyLLNy4jLjaPEcJVKIwQVwryWMXxzsy9xeaHonHSYLVQ3WlMBlO9hrZhJ88L9uHQ6s/zn5QiSkgznLzjZBdVSKwOD/8Bs1KOtKyUw7wyCqQf1rwXZp7dwkdvFphVImNArGukSa23zxuowMps1pHd4UlrnDl0m9E46GZkgKClolIPpXwc0MSUrA22nFt/GCOIrXwWgsSuEq22RVP4+iRm7vG11Udf+G43oSm8XNzSGRj7pU0/ZzCNcue97Ur/7nZPJvbFlwaFEYdbQebMMr8oMG7Jv2/AXme6xnpi0HdRfL8VkXf960FMFVAdwx+9Tqd9ynj63+oBkDyyFNiynb/FGIronoS6qRtDdosLnVxSCAVd1IwaPLo5OOIpZLRJ2/UNMHT/ZKEDHeuezr9daUoo2UTfCk03LN2FWCrJ+iXtv3DsLSO5vos07grIDGRhuZOFXctVGVdfsfJWjY76h8u0utGFaS73th2qNPoRhNaOIu6og8uZeTq7cZLumMChwaXdBn+rCZ498RnlouT2ZAQUh5SFMPpCEstNIs2czIiIGZaPt/sDqQGIzK9FLZ8gNeIeMRtlpUNKcTMvUDi7yPGcmK3njpdep986QjdLQWYi6Ok5HmGj3aKUkssqiSWNZpcLmkdUgI8pd58SybfEJnBR2kv6ehu4UxTuMychkkt8Kx+8C2XFyuJ8/B1I1lLUX4N88nX8qogVtXdzcF7XYTcoQZ7555B4Kk+QkBevSuHxXJRWqpTQYk3HpEKkLSyGm7h20imbokAM+FaFL+DLlEh88u4V2rzD7/WlP099nG2OuLaPlYj/6XJPID/6Qdy69TLte1lG7UL6AM6WL8ZoxjvKk+YQpZPqp+/o/gVo0kW+AZq9unKpr8Ww0ohLtAdCR+SeYfLOWoG6L86gHrYyTPhgpDj7cvIo7fxJJykhCQMCzcwD7Q43cFfMdnxnHEJaWwqAr/RlyaYjdyX5HLd/EfcUPpi7qAjpQ1Mn0MHqlvMeqjT6kN/bhpmhHZyRkJ9hQbj/flO2C0drrOOsjHITuAWK8r1PYGkW24nfb/T5tMgpuT6CCjxKaKI6uZvzBAfS70Q+lJdnhuWOneblK1gYr6lVu0w9LSgKzpCDK8yaCZGJ69K/svfsI1b3vJ6x+FdMi5pNvgJTBb9Hg9QDa3CLcWp0wKUy4GeIIcg3CXeUDOB7GBBSsSHmGRwfLzpdGE1ycmkHD2Pm2zF/rWlHp8icl/l9RJ2TQ4VRAteIKrWa76Pr1wEc41ieKP9zmkhX6FNdbDwFyYKLJ4MpXW8Zxx/YEZh77jJHXv8ZT645K7CbItYAgBYTohpOd3MnuebtBsBxOb6+HtKeYXf8Ynz/1BQCR2XKg1sdLpKJVDtIOcW0n6eIQIvIaGFS3gnemvoqvsz1Jxas7lQjFEJvz1Erl5aWpJtIvC1pbGXu8F+FlgTaUP8hz3LtKTiYpjC50QMGU+fzA1sBIMsIe5Xr0Yk5GzSe08W0mWNA4szS/UhIFXdEmzs65C282OxxeX/ZVkbJci0bnypov1gACS5fC1pff5pNRW5km9SZEEeugp2xFFlqTkOzIPnvgAqDJ9QLVrgfpEKqRkDBJJtZ4wuTI37in33N0egRh1GqJyY/BrdUNJ7XA5YrZ/JH7CHOdN9Lv6jkS0xypcwXM3JP4Kfqar2iXvqH/jf7yu7ya5LX14r2Q9S5xms10TB5Eh3sgqu42eZ0Im88Bt25mHPuZr9vNHFl8BZP7eKb7P8jkmMmQ8Trr/A+gNUWQmJ/Mmq/WyIjBnjrTIhT3m017iIpxh32pMKbz3xWFQqbCTPY7xVyPxxj0xDoyEi8w+tRo3FrcbOPfbXRmRgVUmcCjRS0nhkiWBA2TnoCuApyD6ygPtTNKuLa7ojS242KU54ZafZSCk3IgpDEoCe/uuRC+gFankfg6l3M+ohadRsfWe1vZM3ePbR1z60riwNxa/nykig+f+pDqkHoEAc6XzWfdkXPMKgnhrnefIr5iBO7q7dzhL1PjWjVu+od8SXjGPlw7vOh09+HMgT8YJSr/y0CftQwBRivV3DJeRBg0AMPIcZb5JUH661R2hbOmwUxkcSR37LwDz2ZPxo0Dj+ihlJpnMdvNQP+0/kSm+THt4FziNHaNNWvbzJgIaJlJaOPd+GkDbdedBBcCyurp/9MvTNt6kQNddh3Mo4UrebjaFZ22G4PGyb4O/KX4F18moOgSgiRhwA2SXoDRe2kfsomZWfdyfshZdi2U9Q+Vogi1pxnq+y11Pr/ywKURpLn/24bsS+ilpH9/uC9hIXclv8U3latJKvuYiXV7cMe6P8uVuDwkA/PYcZj/gspqcblKlffvNCqzbH9TKuxrRVrHcOrK/ejSunJ69Gm6pBbbtXGRm8kNqWXxwWWYRYFrA68hILJgAdz51EKq3Hpzsu4F1J3lhFaEkJS+BZ3kAXEPktF2J4VN/ekiiM4Dp0lKa2bUmVG2MQh0LQQgUQ0g0ex8jRMV++m0SBCIIrh39cVFCujRSqjpiGJA9Od4zerHxtV7GXQuhhFnB9iQWzUdkRTp3TGqDUimMgKKr7DH007TtTDxLcr8JvGJv/weee+V7Zdberigk5NDa6IGy/VoP45OJe/BP7RClHsO3/U4snq6i3w0OZV9SwRSgv6g3PdH9o/7jYLEai4PvmxLAAMIq7+X0RnpTFC8QqPLeZpdLv8F6WHV3jYRPi6c5bOX07NIgoRJLbdT5yFgULTYbKXBXjMIaVyI+M0ecr49Y+tDazmrWM1bhXfTmHuWAddS6VP8DcFNixBFGfHyhPM1/lhwCiSJtKi7OB8/llbtLUu97PadR+dAPg6uJTNcxb4lAp4KM86dzqhbdNxyn8LxJE+uNMnyCzpVFYrmAibvMQEC3e7+Du0JblpEP8VilGZX/FunEVm3hijXRMLr7uf5Btjc8BCJmYks3TQBL9fHiWnZg4CIuwjTnSGu9330ShtJ72s3OJ7oz83IlYRrk1ACH/tGMdcFJEnCbIaAlums67eewOY5Dm2yJqMUiYdJFzfRpS5Bp6xhX2wYR/uGOqBprUWp6CJ4UDPFKXPJiHqNzPDnMCLbWTmRUBAJI86OYNkGBY9/9DhKg/yuWO9r9As8zkv7euNV+icAn6z9hJyEHNQGP3a3wwetK3mw6haXBp2nPjiMoTnH6SXN6TFH7GvY34vAqTGnOD//NrF5sbi3uBPmEoN7Zwogn58Vhm4ifznFok3RFEnH//YEhVmgKLqRK30/Jqf1GgC7sp+mv6aML2vHs+D36Qy/MBy/Oj96ucqImBs6OCOEMWPbnYSVevPhUx9itpyVlKIJt5pDTHOVODjtGILJSGC1zKwgiDISyUXVzGthZxklyGtXeUi5rLOFrG27bYEnV+LOUm8owyRa6bMtPgETrPlqDZ6/q2xobbVK5NVXsdlUQ88NJeexHE7knOB61XVbWzVKDe5O7jgp7WumVRcPwLmlkpDckzQXyuevdrv8svzbzlYC6nLpqmpxoLu0FqMRupq6OPjYQcTwUEoj623XgiqD6L03k53LdpL14GcEVsmoP2tQBkCVn8WpN07RYWqmyeUSFfrbKEUlUSqY89tClv9wL8OPODHi3AgESWRO/Me8NHoOvzcuYtDX65BelFhxeACHUzypV96A8IWwVOLn7C8o6y2DCoY0P8iQ0g1ojHlgaEEhKGjybiJzojPTfphGYXShpV5qAlpmMD4jnyfjn0RdW0HclS2cix9Jkb+M8OpwyiND8w0nyv/4W1+0am9R47GfRuSgkVm0nkvsQSiQg4ECCvxbpzGr1xxElLb1ue+tvozYl4okSA7nLa0+grPTOkiq/A23m2fwqsx0eOYzI57hhZCThDQucdBcExrOsW2BJy8PX87LQcEE1gQy+FQpr776KoLFjhkaKgfSJzsDSJT7/MLa4yup9toNyIlAxZ2+eFiqE4icUOqlrebtfl9xp6cZo9JEp6eeLucuR60zSUTvpKdiSC8KUhcytPZMjx4T2dUB3xkVJGS4kno9FQERjdU9l/AYFf3LadBModPZgG/LIBuj15dXv+Kt9Ee4roMBQQPofKGTYYUH5KeKds2+3rd747r7KtpOx2Rja4ne+C/OzFtFYUyhg5SH9axRYrrMhYSx3A6Vbb9Nt97krsvP8tqFKAadC0DsaHMYB2dVC7+0dbIn4DUMYg7xOfEojApqagCPRFggf2ctx4fg3eBB5cQoImofIs5lkG0vkySBJfuXUR4/noK4MrlNPZF9Fs0+paikYG0Bg8sGM+7kOExd9vVSKCshLPMgTu0NhGX8Sa8LP9muKeOiKBh4J50eQQ598Z9g3/8e5T/Bvv+U/+2LKIh4qL3QdkcC8oLbk+LDqv2j7CHYbXUgG81GGwJPISrxb51KbPVzzFN8x4xjn+NiHoQh5CfWuMtOrDt6v88PM/sSrgK3NjciM47jXpdPT2a1AE0YA9V3O9RRQARdg5y1fnQ0a1c8Q+6Qu6gZZADBkiHjlUKbIYBNcReZWTWSiBy1zQDzjYqEJQZ+yXwHAZE7dt7B0l9lihtRhAiPDNyUOrLVXdQH6JA0Wtu1e+6B5GQo7v0YN2NeojE2gtLwUnu9AJfuWNb7wplQ8GxywafeB4Ug99F7vnCPh55ZUY3UxVYzZfcoxpwch9AD2bc9czulWR/h2m7k5KobuHem8HPmJzCvkhLFvWxNf41vut9jzYQtlMZFc35UBV7tsnsi2C2Xt2+t5cNmcG/RMO6QE33S+8gaA5KCc1Wr+D7tPTpMFmSSZTfIaRjKoaqR9L3en7u/GYh7swsGtQFVjwP3Qp83GHbrHSIKFLRqTlGr7pldKLG0cwcj64MYfmE4zxxeT0TdQ3KbbjwDGa/h1X0Mhb8a7rmbbv9wm2PtqXrIbo7DtXQYS75fxLwjvzM+Ix+l2QUnZSeRmiZCnFNwieuLf4mIb72vA42nKIKuB1WDPA6ytVTrsZ+Y1vvokzOfiVsTiSiOcNh0szvmc7hQ1v+zOgSt891Z1cId2lIKJ57g2KTbfLHmC0xKgRnC5xyf8S5bR6wiwrUXV4fLO2vSN+70vdkXhSji1/I13poqMCtp8m6iNkSmbZLr19MgsaJmzUiCiXvd4af8RTydP4qSiBJaQ0RqA5rocO3Ay1NEFEyUtYfR4trO549tpbOrkn7Hv2O7eqpD+106XJiy20hcfhzurXKWX1TVs3S0JfNo331EzP6WM7MX4N3ojaZTnt/ujVvg9FymUcbIwTkcn1RM50wnvNtlBNOjdZBr0PPKmJlUtmTif/UAoTnHaVP8Rr3bCXksbMg+2Uh2dbfTo3U4FXK4vw97wuN6BPsMiMAgnxu8Vz4At1efJGJ3G5Mv/M7Eq9cIUvWmpKUPn17ayLXaB3CtKUTfdwKZ418ioG0ZIGeivp37LJfj3MhO1uD1xhOkDy2gNLyUdRG/k9xnLSZtGKPLoKDLcRvu7XuB18dO5vs5kbBnC6X//pWY6zvoVMtBmhXnPuS3+n+RO3A6TuHuRGXL37M1KOFkCCCx5Cnic2Jo2XmZ+Tvn95iD4NM+ivjCGYSeqaFwWSwHpx1Eo1KS6HeWLfN92J6UhW7iWYx+GrxrG6l2+wSDsonJ0Rt5ffSdhJ89wbjdt9CUdNjmiygCZh0sbOOIy5fkDLuHuHuGoQ9zI32AmTZzLfHxoFNVUxi6nXPa46R/ns4z7z6Dd4MPvhYffu+Kd1jb+S96nWuhMTiR3mtH2gTHB10dwLr163BtcCW4/S1Gmz5DLVhpnWVaj963g6kY6c/uebtlHZzKVxmWfRpnUxBZiVlcGh+FSg+aLg2u5k72LRHYu0TBvozPOc2ztKvkDFlbJiCAkw/1Po38tOYrflt6lIrQCmwoka4aHh64gsFBP+Lb5E9i9e9MbrNnCwsClHtvosD9B7R44dadiLvSp8d3YdmnJCMmsQuzqEMQHelYlm1exquvvsqyzctw6rQHzfoHHWKdjwkPvZqUk/sJzD9rW6+buoO44fwLFeY5eFVnAwKSQolS6IZej8KEE7R5DWZrO8TmhOCcu5katz9oMlRD39fZU/oRvSpfZenylyjsM5rc0PW8f/kNdC15kP8t5yvm81XOchq+/h2/0mu2tla09eJ0fRKvN0K7p46m1c9QFTfa1peemhqC3XIQMKM1BRNRu4aYziW2Nrno5QzfpCHyoXPKoSm2eXu9aioHCx5mTP9jpI97lPPDrluCNVaol4BosQEi8vV4XD7K+fjReHUMZUXVd3zt2pvrDak8FJPGlgflQ3Xv271ta35a9SQ+u7yBQp1M5W0WHYXufZ3LyDd3UKawHywlwcgjHjDbzURHeAWnJt1ErVfi2expQ/b19j3P12EnSJXUdLh22tA8giD3x7fXPwZg4+fzWLl+BgFNM3Dv6sPnY7dStOYWs6LHcKpmD5q2EkoiKigLL8Ojuw9V7VWoDGksivqNnkWQRFxUsuN5b+lU9nZAl7Yb14xLDLt2L7PCVqAyeVvmnzWxw0xO8MtscR3MxQ473Xi3opYup2KKnPZQGPgh2e0XAPj88gZeznmIc8m3yE2oZ+DZBhJyE3FWa4j1vsbDgx9Enfhv3Pe8y5BL1qCB5ZuJuhuSX5bH272DY6vycB/0NjOuGnnnnnlI/R6yvX/k+pnURgzArO1JASX/O6L4AE+6X8RZHwlAXDE8ceQkL46eyzTvD5BEBb/feZHshGxbgMkkdvK1qhefrv2eE6uS2bFgh33+AEZFO+3KEvSqOkCiW4IaVSIdei8WJL6NWVLw7sUoks/0wbUrCK3JovF1+0PYG0tCxc8scG+mJshMcWSx3MMiuFT/yDDVHxzodZu73Z+26RiBhfUBmbYpvO5+xnuttMwtRxrPnjSfVpu2xLJMtLie4sJDC0nrd5W7N99NdGE0ThYYjd6kpckkEpUdT79rnkQXRGMNAcxPfJeyjgCeqYclv8i0Wb8t+I0O1w7ZJhm+GSYc5yPTZUobdIRnHCTdfxA/maeAqACFE5ItyeQvKCO3XoTGldPngS48543jxNgTNPg02Ow7CTM/h2k5MEBLt9ZAl9bRodPsfJWCgPVUux3gst8jXImdTV7bTULds3l74liudwfyZiP416gYf2I8Hq3uiKJMDbnwtw6OtTgx4OoArgxv57cl51AZveW6XVzB1KI3eDSkkyOTr7N7zm7WP7Weo5OOondPoEPVh3a9J23qPXSV5tPqoaFwwAIkUQFhd1Dp/gQJvhfxEiVLO2SqfmtyW1DzfOIaNxPrcq911iMI8pqcXT+cNpOKet96TC7uuOijcbPsBxuuf8IzN1/ncf0OTo+5gcH3AcZmZpF+/CgrTXad2v+u3KvvotjlKE2z4ns45yRIfwV3VRs7WkR65fYiIScBlUGNKMLaZ/wJDpKY5qqjKKoIc58gyvreywCXubbn2pB9kj3D/6+sH561cnJXblwuCPI8mBrzDfelPsG5LjXuDd5oqt4jz2sOGc3yOvLAgEdY7SrgYz6LaJTbaFaoCFadh8w3oauKLoUL59t2M3OnF8m3ZENBISpgwCfsrfiYUxNWE244T6syF2ddHG76XqxeqYHDw+kw+vBhg2xL+rZNIKJ7NhpBDgp7dgxi2rVOxmVk8mfdQk7FeXOk1p5chA11b/dG95yfB+vP8fb6j4jL8uf0mNM2pNnKlKdZO+Q+WnxX0h0xnOuDim3zAIDuapKuLuVJTyiKLqeor4wiS/XYCEdHc6PtAcpaEjEXX6H70GkAi066/PxrVdOY+HsZ/2qQv6HCgPWsOjaLas+9ljqCV8cwxuo+sNVZEOTA/vzI3cxwlTAqTDT6ttLh0mFbGz2c6oh3akYjwJWhxXy38jtMCllnVatsY3m/FxidvY6KsZV0ejaj1qsdkgS9RTMFVxLRTVrAgWkHqA6s7qGPCPtLZ7HfzlIHgoCTQo4CvDnkCaY6Q7N3M8fm36TTpRNVjzO+k9EP9+5kJEnkfO8RnI8fQ5g2HufuaJy7o1Eb5f1MUa1A7BJJ7J1IzyKJEhc/zWNs0Vie9B/OuYShdtYQESRBRAgJwtilt/Whtbzc0MrH+q04779ITEYxEfWr8W4fITt/JQ0hYirHeu1m4B+vo+jKptHtFEaFxVlrRcFZdEfdFH5EesioLy2wafkmqof4sfz7ccTmx/bQoBYxKo20uRloV7eSHv4QZ5Sv8t8VQYA+Jd8wOa2er4MusSCkjeLoevRqGfKiEEVcBFjlAYrGqwhDhtDoKzL10HjMgh6lqGa8MzzmVcTz3o7PVjrQrzqyUZhtKDwFINCpKqdbXYHJLDmwH41Ta/mqYCkLmwrI9n+MbuMWaly/otMkZ5PubIftbQp65fYCZHS51Wao7wxlf8kcMrqN7J17goNTDhJVFEVsxUo8O4ZQZIQ0QwqtZiVlYWUElzQTVt6bUGkI8SqQ4uDMnMEsdsWucfgXp3NVcBXdzgbu+uUuYvNjLXTTdqpvz6rbqDuMNg2wvxbvBg+iCr1RGVQIPc7zvdTpRNR54N4isHPeTm73vk2YNoHZBQd4r+V9woQOshKzKI5zZ/qNNgYUbpPrdnk1gTce5SM/qAqspi1hIOOOpzDg6gAEBERzJ+tGLGaJfxbxAbK9pzQqLRIqAkFucnBILwkgwdSdInfsuMM2J0MbHiIvOYKokGQWbl9IcEUwoiDYAhoA54fLKNj7v71fXnP/m6IQFMRVvkJ8xb9pDuxN5ugH8UkJI+37NIRtW1HqesjxJPbm+Ki1eI9O4tvEj4i+ut3hWSYT7LpLPkcJ4WG2ABeAWq/GrbYDt2A3/OaOoNmz2fJQiTq3o9S5H0GTcY2TL5+kpOYE53oP5fPKu1CJKuJUMGHsDVpHXubs5E7OjjwLiNyuH86u208RZ3ya5KhkWo61sOr1u2Vkfo+NrsjjR84PP8/E9JUYho4mwiMdV8NNCF9k659uZ4nYRbHU+9VbqmVfA3vP601nn8FcGK2jye0cXepiy28cKdzBvl8U+X/ClbhZ5FiSDP1aphHSsAytLoLy1nLSOvb/DVWrUEBkpN0nWhVUxYURl3A2TbEhdAHCw0Rc1M6IkhNdYi03Iu8mO/hF27eR4JtAkssYkAR2queQFiknUYjOARwuWIm3po6VyRdwGxTZ4+0WtGDZfD5Le513Oifh3zoVq//LGhhTK7uIdK4nSAkhDctYKPyKIECQawFlnf5cbNAQWOaKX4kzLu0uDna6gIhgFuh2VdESEIdZ5dSD8UNkmjO8GVxCTL4z4aXhgCBLjZxdDL95IQjQ6hdD7Z3/xl/8FY0hCEGAwwX3s6d0BoBNWqBnkEqhgIDm2XhNkwERY4762eoDoFc00eRygZLuW9jpj3uCUP6rkItAcfUcfNMXkXolEHXRaSTdVbqMHTz5JCxO+Yj5LrVoujQMvOjO4m2LcepW02LNL1J7kZOygQOKdiojKwlxmk2f0i8Z6j3DgVbZI8KT6rhRNkrpf0L2GSxI2Ve+fYU55+YwPNGuTy/WVBNQdAmVrg2loRtVj2/6v9Lk+w+N5/8e5f9VsM9oNHL06FG++eYbGyyxsrKS9r+mb/yn/Kf8TyqCIKCxBLgUChyyJYySBdnSE+bt4BixBAMFu8WqFjTEZlTinfsnapMPrgrZuq1ql9FqL3rDmFNjyI2vYPv0OVxq3Wm7d1rI3Szx/J7zc9bakH1KhQgJj8H8Wkh8jtvGabT6xWBUWLO3FdD7KTYXbqG3RyFjVgQjSgoS02UaTicLBWRq4B88I44guCoYr2Yv20HthxvvMfrc8zQbBfwqVGg2/ovm7jto0texYAG8/TZcbN1OdvSPlKYGWzQe7GhHhdmVp7wg7cVHeeSL1Sz9dSmioKB32fs81wBv5r/Bzf51VIW/jbl3IuGtn+PZOUheqA3teBmbKIzP5fT4Oh7utY/B+fsxowLJhFLUUd+eAJ2r6c0gJLc5uDr/SO8KGd349oSxPN/3M2LyY1j+4ziqg43c7n0bQVKgUbbTagjgQNGd1PfIjJX7zMTZCfeyOrCbdjcdHs0uaDu1DjR3ogjeFemkni2S+bclu/6OKJjob0hn7UefUhukI8g1Ed+2sZQZr5OR+D6G4FnEl84j/MhvSNm5GJUaei6Jjx++wq28TWhyzMRctztUuwxu/J7xGo2ZW+kb9SDbn6rn7MizDsG+FqcMan3zKXp2rmWOCOgFea3UqeVsV017A8EFWvzq/BAFOWj2/Mg76Ov+Ct75X5F64HXu3DaPxMxElKKCvgHHeXn0TB5xyaXP3FOk9yuRUXmCAg9nZ8a1HsX57FyeaJ7NrfG7kSypTIlZiagVAp7NW+jtdw4kBbF5scz5OZcHP+mLc4ezI7LP5gA2oVAZ+T4A7ondxu5eaUxbrMJ5yF04dStQGBWIgoBZUrD02C6mVsoHKbMoYVQaEbEbuYKkwKAyUBwDe2bv4drAa6gEJ/pUvc7Sst/xNRbir83Co6GWh75+iPicWAD0frIW0hPCaJ6ug+aQEcxb+SZ+rXbK3h/bzKyohi9vf0TZhHu4OiaRkwOXkhn+qGXuy/2gM3Vb2mVHTFgDZGZB34PG04hagHeHPMsK/xwuDL2A3lkg9cRBwoqMaBTyOnSkcBVnMh8l5tJvKCrLHObui6Pn8G3Sv7l75ibaI59Fd/kW2g41ufG5sgEeOofOUXupKRzM4o2yczszMRMkkeTYOpL9T+HvUorJ6RySAMV9ptDtJL+jqeI+vvM4w68zB/NH7mbGHQinf/oCnLvlPlObvHFXr+PazFfweXU+2+7cZjmb2r8bdVcLmqzrqBrlZAMvDyVNXUFcq5pCiqaLRW6gi3HjzRfflPWfJAXe2koAqtwfwKjWEpzlxvPX3ye66lHZoNwdhv70XFqV+ehUtVSeyKFNX86JQUso1d1k2TJ5HigNShTtIt6B3hiD3Iioe1qmAbWU+OVDqJq+irzeAneoRnE8WW6Xh/kRaiMG0ctzBLFli+nbuQqlJRtNlJzIi6vgi7Wn6ewVjU/rOOJc+6PVh+PTPgp3Uwzd2m5cm7uY9GcIvvW+eGNHdbSUb2PsBpElP8YQmxeLaLIEJq49jqrqD0aVQ7vUg1rPGgzcFciEyJ9Q+K8j78Vd+JTfcpgHggA3o+7lctBKuqRm2996fhdgdZxYtZAEW6DJWwSVk7x/RZREoDDLyL6nhy1letzXJBQ6k3JsHO6N9ZR5vsflelnLLcnvNAUNb7Ki2Itfl6dRljAcs6AnWHkCcj8DUxcKQYF7iztTDg4l+lYBV+Nmcbv9PARPo6RjGDPD9xOhNhOdfppBV2TknrotGy6vJs73HCf7JHBkwmGyEkvINsl6bH/mP8TDNx6mxQwLN/TCfOAdMsLX0aiT0WJ3932BL6cloFG246aPo0/pF6S0vuTQZzPKn2R631NsXLWRXfN22RysO24/y8abnyOKAjoXb06Nv0RRdJHdAXthBRMqv6GXzpnA6gCu97+OSexAZfIgWaVmScxWIp06GOV6LwF+E8lL8uHy4MsIKEgNOsgdvd/nz/o4NrUdpNLrN1uQ2bpG3LWzjgcLBtjqGRYqoDb6Ml2UA5Rzq6DJp5XPntnGqbGnbPuUi7oZlWBG0+zB7C3jLDoz8vwZHLKX9ZOH06L0pti/GtEs4Fkjo1dUCiWR1++D3zyYXbeQWw99wZ7ZMk2jSpIdePe6w1O9N/G4p8Oso9soX//y9mqazaDuVKAtyyG2dD5vDPgBjSHIYf6ZJJMNyajugVAXcXTwWPv6SuUsmmrvJ0q5nur4FeQPuJOmwN4olTLlzM7bT2P2GoDb7asMPqey6BBavpmAsVB9FCMK/G+60IICs7M7AgpUSgW+kYMpaEyhvLUXMXekUtpnJg3uRWTWZmIwGXpkqlppjeRxyjfAu+Nl2qZTP/thOvkDZRH1tHq0OjhsGoU8Wt1ybbaZjZIXuyNCRrWbmeICTlIb5W3x3NPvX7gqYukuep3+F3rhUZNj1/YU1TLV05j97D79LI0+3RyYccDe5omnOCv8zoZrn9BtkoNO1gO3ZKH81Eg+9C3ZwL2Bn9rqINfPGuyzIyqse9iAUli0q4jZftm85WekyV/P4SnXaA10xslJwM+5lMq2OBZVu/P2c+/Q6KNnzp65KAV5Tkd6pOPj1EyFEQpiq2j2aGbYhWGMzXuOILcgCJ0NAeN4oWQcl5Oe4vJIgargcvn93XV4my7w/oDX+JeXvPd3qou40XqMnPociLqbGx0PkNc4CCHEh1NjT1HvV9+DClHAJHbj0exM2SDYP3O/gzOkwe0kt8PWUe65hVrtSWo899FmbKCuM4Kvrn7OZ81DcTs4Fm2nwFcPfkVdQIPNgR/slsuR4c8za/8sBl2fwbiMXCLqH5DHQi2jI35rhzr/Cm70v0G7W7ulRgIFvt+y4don3OtVjesT33Nmigf12m3UuR0GYxeB7V8BMKssAO8Gb1Z87cPqDasdsruVug40pQ30vdkXrSVxSRSMjArfyrbwCs7d/xNODR2Ep++3zT2j2YmrTYPINkCjTyuBhRdwqy+kvb0VO4buvy8BgM7QRtHBV7iZcSc1HvsAAabd5KeiJ9B2aemb3peqqCRGdFy13acb8AtRuYGUh5WjXzWYNt8oR2d4D1RMndsxqj320mqwo+YrnU5wdPxRtq68wZYlW5CUlnkvSICEgMicPXMYdi6KmGwjLQbZETo6QqbCO1t1PxeGXeHoxIsc6xtNp2Rp8eX7UdefJ8Cjk9y4XERLXEEhiqB0pqq7P4dq+1JtlOffqNuXSVuZQ2pMOHSWkd0ymRMdFtkFTHLQy/qto0AhaYm9up+ZXycz6uwwbHzE9Nyb7Q7Ynsg+UYS6iAGkD3S13mF5j0BpSyJ1URsodB+Pd0scqTfvQmuytOnKGgD+7ISxxwcTUHQRjUaLwtK4RLftzEn4iPiShbZ3TQ1aS3yFXQvJbstJf3Mq2tZHHMt9qU+gFE08WyvgW+/O8UnXuTbwmu17vFIxkx+bk9C2urHsx4EkZSbZzp8mSUlGrV3L9fx9JWxftB0QGRkyjpMhUBTaznTxNn3cWimJKKHJu8m2d97nDmrBzPach0nxHENC+dv0M9/Lq9dy2XTrDZwUOma7WM4PWJNz/x5cMJnsDmONUsuYzAzGZmbbEnKkbqjPrqe7qZth2afw6BhIaL3soDaYDZgk+5pq7af09hNUeu/AJBkoTs/gcuxMWhQFtneu1t5i3dkJKHVQmGrXLuvpxHTylPdMn3p3W/3kf+30t9Z7Zm0xM2uLRI1RrnOnt4qLw7Op86vrsU+J1PvVc31wLT430lC0HSZTsdn27hqPA2RJOzGIrbRo06h3O0GjXtZOviv+a1KcrxOfUMb1gUV451zCubkCNa5Um+CeatC1ZGMKuYFebSSqKAoQ6DA208tyZPukWabekyTQKxqp7CyxaRT3DGDKbbP4OCSlQ2DDaDbaWFQCmmbzrXs8r8UcICK/g9nbnbn/u5UkZiXadDDnV8F9dc78uvRXttzbyrerv8X6uMq2OE5VTKTVDEVRZYRUhDB732xiCqNsiK7RTlcoDK8hxeiMa0sLCpOeQCGFDgksrHq0SWDGTJXnTl4rG0lO8MsO86vJv409s/dQElGCQiHaxu2Kz2Oc77eOzGEt7Lxjp4PNADCpeSuC28NsX3ydWv9ayzclsX7SEAr1c/mmdQtXJi0lKzELvdqAIECiSsu6gev4iuN0jbxCdagW3+JCFCat/GyfwZhUXjxSB2NODcctW16vp5XGkFLyDmpdLgOCDnGoMZqC7bLd88eMPxBNFkmahkE8dfgi/fPjQQCfepHE4lQSymW/jIuhFy2RKwjsHU9SVhLure4oRNGiHyw/oyy8DOflzuTE5zj41w7lH2LlnpVsvL6R1ftW4/KWC+elj4mvfJW4qhdA6UuXewBKFyfqs+sRC/Joc0qjXZPNgAGgUApotC5IkoBPUiA6F3lP9rcAWI1GqM2sBcCwbRfhha62d5dElnBqZTyzNszCf9lE/LofZ47qU1Yu9uFS/CSuxM6iK07WD1RnyKweSkGFUlQSqoSBE65xedRZSmN1lIWXIUgK0qqncCD/IcZ6v8zyzUEs/H0hGYOy0Kv18jdZdRh+Fdg6eDW3w9ZR2laIJMm2brnnC+A7FIWgILgimGG/tJH9e7atvsJfbOj2UdPIS7Qm7ygcfmM0/x3m2PN7kyToVfUS/Ys2E+van2OFx1hfOYvc4Ncwm8EgtlLq8z2/3d7Gp59CUIB8b3lYOSk3+zPx2Dgi6mXGj9deg/fek9/hU5GOhER63zQkwfA3WmijopUC1V47Fb97Ap9d/o5XTh6kbWwebotnc3V0BB8/9rFtHyptSeZw9ks8GHOY/sWbbN+S1cd2pGAlk8+/wHUdNmQpQKh7NmHOtUjFYczdksLUnxNZtPt9BvtM7tErcoL94O8uE5h/1mFfN4ldHAiBKXmJCJJAep90BElEq0VGwak9yc+6D5/ylwnc/ZEcWUbeN4Nc8/hk8IsscrXXyYG+VIR+Jd/yzsJNkJxMs7dlHC3rfZPrWc71Hs6Jw6+QcPE4rm2uDt9NT7Q6YENBJvqeZUnQTbR+T3B79P0EXr2K2PA656qPMm4cuA57lSpJy+QLwwkr0VIRN4LUEhmgQuWf8JsXTt2VvB6ZyZ8TL2IyO/pIAc7rNrBoxwIqvX6z2ww9BvqDyR/wx9I/GBAsn3Hj4uNIGZ6Cs8bZ9ht9cn9uTniCDs9QivrPI3PsGts1XW4x4ekHcOqw24XWfuvZj/+nFkEQCAsLc0So/i8s/8PBvpKSEvr06cOcOXN4+OGHqauTRZLfffddnn766f+bu/9T/lP+3xYRtVqNIMiZ0soeUDsrsu+/CvaZ/0Hwu1M4S9rin2j07iTyxm7bgniudAHny2dzpstK8SXR4tmCKAoMyTlCUumn9PcZafm9PVWh57vTvKeyqi0Dr5xnGfiy9aAv17ekczhLdjTTHrCQbnd/dE4WvTdJD1VHWNH3SV6JuIYuXsuFoRewar64O9WzJfVr1lXFMv33AJw7nWh1K3A4iFqRerHHbtlRgT0yBTOM/TE5CxiVEpeGXEJESUzN0yjPdFHb8CKiCHpnTwxzF9IULGdGiiKQ/SGrK9/FI6KKDhcD7VdlQ2ZGzIewO4x5Blf6+J9Ad+EqR2d8jEtLpcPIbc14iWBtDaMD28jtVUHGgC5qAmsQUNDL5zKvDupDbP+5XFN/JNfVshiuGfggzkodpUm3OTs+j5l7RzH80mI8LM4aACPd1IbJAdrgymCHhVQhGMlWxfNYLRyY34DPXdMAeKdpAH1+f4Afm428p/OmYEAGRwM/ZktQIA1ux1EAe8J8mBW1BUNoNIELR1LWexIjR0JgIAwMPsADA9cSWHmZ1hffw7dMAkGmtrCWw/5TOZs4iDpkCqC/ZvS0aG6S36uN9AlGSmLc8VdHIGBmaOhuZvu+ztTY35AwkZATj1+dH0qFgmDXPJL8z/KDKYbPmkGll+SMJ7NCzrxMeNL2fMFoonvyFPKntXJg+gEEUUVjyGuMjtjGLNc27vrFruWi1mkdNfusB1TBxAe9rlPZJjuzYz2y8TZ74VEu8cinc0i9nmoLJrt1JXF/exZzjv1ES28fvn7oaxTYUUgg0y6cm2AmLVXmbE90GYufcylfz0ygb0UKi9Sj0XS3cWr0KaoD5H1F8kyB2PspEryYu2EVo/6ss9F+JOiX8ZI3jFd781Mb6CUlkpOWuhBL5MjSDiuyT2/JHLdSPQkobHR2ZsHgQONpkODNa68TrOnEc8EhykapKOs9kQ5vWXtuaOgu3ps8kHN+93Jo2k305kaUDZtp1soZdmdKF+GiaiRC+gXRqKfz8Bli03tQJR4ajFv2W9zZsZXgKn+0vXzlwIOkYNr983g5q5thh39meeQxfn/3FJXRdi2QGM90Fvtn4yGCMRTKRizCXXwbvzY7574t4B3ly+3E27LT2eIgatdkkx+dT/69szE4IaN8RSXlbQm8dWYX995MofjdVagvV9ufh4KtmS8zdXsTR5K1HJmjRdOmxGlvBwEtMyyInRUUtFTSnbOS3qc/I+mJyfy5SM7gVyhkGo3QEAVDLw7ljhenET00Gp9VLxLU+SRKJQRYKJ6cQ7zo9gvFrLS/G6DNN4rSPtNpTy+i37GPUNZU2Nrp1zqZKbeaGVJ4jOgrWmYdeZ9HYj/uMfvkIGOrl4qzYypp8mqiRRHI+vObefPMTh7mOq5Db6LRqbnrl7tQGeSgGsYOREMbmi4NE49MZN1bK1i+50Ncui3jETCB7RXjeMfchiRIBOefQTQ4Clxbv6eb5i3kBL9CTselv12TJHuwT6kQaXKRM2rf9oUf/rWdj19/lbdeeAu9m7znFDX341L5LAoNSvrdSKEosoRDU/fZ9sJXx05jrfJF7nDrJjpHR9KpD7kcNZg68zAIuwMarxGV9y6/18fj3KXh5PjbSIJkc7D5a/NYO+Q+vNVXqA3rY9NGwXcIjDtEgvdlNvjDuZHn8Jz+HJWC3RH5dOxOHvGAsug2GrXF1DtvpM0oB1avVM7gy7JBnI2dQYX6BP9UjDjRKckH1KLoIoe9dXTYJuKzQ+il+g2PJhfL/LBcL/qJ6LZrfOL3IC4dXpbMcPnahbJ5rNxTQqBCwdstzkw4MpzKCDdu9buFIIk8OOBh7kx6mwPTx3Gg+0Uqvbf8Ldg3L2E9R5NO22hnQNYindNUSNylZQRVBjF1z2DCSkMRTVqbtuyS5Nco0XtwTdtKp6fEkKuz6Vf6DYIAOfVDOVc6n0zXQewfcgGD0ohr9T5atekWGtm54JGIXhIwI9skHs0eKCSLNmgTnKoZws52UKBkYP4uwutWsy3zBd48s5NX+33Cg+YJjD07AbNaQ230UAdEji2xQzJjttCbqgTN365bS8+xeLn/Cn4d9SBaU39yEgopDbmEWTBQ0xHFvty1iG3ZSAtGUudvJuVmCjZkH0DQFK6phzLq0GTu+C4aOuQMUYVCJKDpRy6Wz2Nj/mKOXfkWoymHA+H9Sf4qmdqOWtv7reu1NZlrljOoFTreLR7C0eFnKUmVAw3QQ1PZ8r0FVgXS96SOJTs/ZqLyVVuf2PVq5e/RRQCtuZnGrmAeP3iNXOM6oqbPoyQhhaibe2k3r5fvjX8EpqVRoE2g+eINhp8OslZObrM2kErFfPblrkUneVj6Qd6H6motDp+/ZMCOlF7gPi4SWr/CUje7nqkVMdFLDcGuJfy7aCxb9w4lMscLb9VexnSfJzkZRsaf5vlRC0h1MoEA1XETqYtbxV0+ckDxvfPbmH3qSzxE2HPHJbITsgmtCCW0bRDOKsshv6uGIUq45+d78Ks2W1B/Cqg6yNC24YwOvMhYZzkTutxnMy/kTOTDCx+CSzjfFs4jLf0Ktwv22sYtIlwkOhr6pwggCQy4NoDUX2UmDcfsbftYWB0voiDS3B3AgbyHyTf60+9mP4LLtNQE1mBQm1CpwM+5lOdGLGRY3Rs0TRiLQeuDe12BvY8HfcHWvlvIKPHj4U8X8tzbz/HMu88QVzIGZ3MXWlMhKlHPrOIUzhwazIztNUz9LZt8v6ch/2s8u45yqngJxQYVTV5NFMV0WZIGRFanrmVi9Ebiz39P2JYr3LHrDrwa3VEoYHb8JzwzYglT3TqI7BqE2FqHwXyZG22HARgaspvFYXL2vsIooO5qQTQZcXN1x74T//elBpl+vP/5eIJzteiV9YgKAbz6sih8A2lxei6NvUq7b1+czZYNt/R3nA/68WDTa0y4WYabIRidWI1O6oHE6KHZdyv8Qa7GzaGk0+7QbFTeIjvqc7LD9iCJks0OP5j/IIcL7mdbcAtlAzNo9GriVt9btusr95TwdtMvvF6bTWbCUTITL9GlLqHNHGl7tkvma9xO0OFkVJKcEcOMq2amBS2HjhJ0kgdPpd/DVZ3sxLeNMcDcMrYWbrDVvdT3O/KdN2HETuMMoO5sBiAvLs+RGhJrIoQdZd9Ts+/BoGA2PdEb9+46lm1eZrv3hxvv8/CBTHJbsuhQXKTvNTUD0+/Fq9uSKKJvos17KEc74fzws1zru5ULKQMp6p4Ogkg/1+9p03uz2buKn5/4hLSxM0kYMxf3LtmBPSp8K0cXhPKerxystCYN2AJMAhjFDjxNsdxR+yaf+oUR6ppBQVMqRysmUKtXsHD7eAZekm0YK7LPU1ODIAh0Kg0URzXIdIAWZ6TepOX5Y6fALY7jR2Yw4PsU4vNnotVFsHPOMcYMfhmiV7C4ezo5v0xn1Xer5Gdb7OtZLnBH9Db6lH7OR/1OElv9HL2Q7cZOvWyvb22XHdqBpsH4tk7E28lOXdnsfIXcoNe4KW1xGBuFpEWUVLZ9qjO4E/eZ7hyqPYSAglG3r9C35FumXevkrtP3cWXcFSKKI0Cyaxl9X7aO6zELKV8QxI5lO6j1/AOjaJ/7i52yWDA8m+JRKrzGdlDnfoRx8/Mt/dzGMf3bdH5vJnP0gxRFVVjukusXU/0Mk2/WMM/9Ldqdctjc+DB5gTJdeGDzPEZcfhzvOk/Oj8qiNqDWNhb2xAGzzQ0uorTN7ZuR97DFPJ8udSm3w57hYvx4rjUdBWBkxDYq9cEsqwZNRwceFVncDrwTP0UsTw9fx8r4yWhrjvCHYgo/L9/M+nXrESSRtNYjfNsK3gWwvc2SSGOGvOA3mLg/kvzAdy11c6TxtCdPKhwo+c09EOgCSn659QZ57ZGMqgnFrU3D2RHnKA8txyDJ9vJzXjDbRcKgltBrHJ3Rbk6NvD/8UVa6g3+NF30yZC0rJ528P3zk4ctq103UmBQ4F4VTGO9GRvx+JMyUvSCRGSwxfkcBtzsmEaccQ7e6gtzuc7Rrciz1E5m9ZzZ3fDiC9D7pNPg2oDO3I6DAuTsWJ8mLvJhjXBqTZ5PdcDLLCeIafSixukXEFvbmzq2pRBVFyX4qwYy3tgq1WE+ncwPtri24tbmh1sv2ZElzH06VLKYRLZLZmdjbXUTfsLOCEPcAlZNvcrJbzfALQ9B5BfLNg3tY8uAeghsPE5jWH4CnCyaTFS+vxau+X4VaL9tvXUZ3chuGUGuUz79H5w/C7+V1+LdOoWfxnZDIay+/RnZCts2/1tPuc37YmeMTjjvYfhm1Gfxw4wdOlZyi29hNp6HTNh8ABLMJhb4TQ5eRSe9Nou7xRzg2cBQnk3vz4ksmFAY93m1V6Bs6mPbzEios1JgjLKzBRiM8Wf4kUROiAGxardZiNMtrstkMwU0LmeD2KHNGy34KSTDSHRLD7O9n0x4t+9+UogqVQsUNHWxsgTpTz2QquV0eTnUM9NwAzTdJnJ/Isbkn6XTplJNzK/YD4Kkw8dLrL5E7/CSbO6PZH7acTqdUaMtFKSoRzSIedWaOLD9CYmYiglmFgECz8xVOJ/Zn4bcLUZcXoO2wB8jlb0dpqaeSsLpVhNWttDfWmmgjmWw2b3IyfPKJPSFCwoTZLDPn3IpaxaOHHrAEp+xjVh/SQbernVUmNRVMyhZeSVtBredRzAoLM4NFKgLgdMlpDjd/QZPrBVtfWQNrggCtOj+am/zpvFFIm4eKZq9m256hEAyMifgFT8N566SwPEO+/sTQe/hlwGfoJQCz7Z2HC1ax8NAfVBqnkj3YGWNMPHrPcQRYtD0BYmvWEdYo6+sF553miveEHrPDzK520Hm2kJmcjqfzcFy7k+T1vs/L0OtRprZ/h5OUhaa5FVVHs1wvAV4eM4uhftcIUcKN6hvM+HUGOYGv265b3c4mE7BgPhXJ9zIk9xCBzXMdxnHUbyn4VZYQUBPgAEwR/3Ke8rYgqCdE/8DDgx7ihUkL6HL15/jUdNlOsrEB1DBRUUt3TAl5Sd5oRg5FEC1sJ4Y2MDQTmf0qa8Sh3LltLs4Ht6NT1qGXOm37XIUpjZ3ZO2jX3LbZ0z2/8zGRY5geNx2VqGLAhgEM+XQI7fXtSGZ7lE5SqjFo3ZEc9HQtfVJeg3/JFVTdbX+7BjjQwP6fWERRxMfHx+Gb/F/6/v/RGx577DEGDhxIU1MTWq3W9vd58+Zx7Nix/6mV+0/5T7EWHx8T7e3tSJKEQgFOSjV9i79lSNX3zJF+YnJaI7OD7PRP/0R5pOzBKf6M3xFOjaxAkprxrL7BiU4Ld7bBi7fO7KK05Ff+XJTAn9NvgCQ7YP3aJhJV+yhGsxGd1EbstT+YsV+GfAuCAM3pULKd1vZyki9KxOQ5I7k6EdywmGAxBfK+YVboM3SpyinQZ1OY2o+chGIAXPQNcGIyle0JvHbqAPWPBnJoqpy58eKLkBBVj5emi07PFnIT2/h+xQ/c6nfL4fApCrKmlWtDB21uVhoR+RNvdrlMn6I03ph3nZN3zKArYCWDXeTMUYWkwVNVipC5l4RLP0CeXWtNFIHKA3SKrjSbIemWB02/yBzWqQH7qW6G13Zq2L13HhuOjeEn188ou/4YoWc/sT1jT86TBGprmR1Xy765F6mKsGfD1LRH0tAdzjtRN4hRQWj9CuZ6yUiP5u4AbjQm8HYTtLl1UZc4Gl/j60S5Jdie/WbFeHaNjWbjI0W8/srrtvb2DTjO9oVuhPV6mvqt+aRkLOVi0z7aNLflNgOVZhVf6DrJDC5k0p5hBJe44tM2jksLSpmtaeDhQQ+R4P0d3fN92Db7fr7SjeDNN0EbN4cMwwOYXdvBy5Xw8l7EF62ml3uKvd9Q4NXoxeB3c1nxwwqLkWQfq/OByziXNBpD0niGFB5jjM9iTJKKOVvN/Nk8gT53nODrtZd47eXXOD36NCqFksOF9zFvWzdbjbFM2HAvD386jXXr19nQPgRPhTvbyYs6x4svf41m7x4Gz3iXEfk1zPV9Dkkro0i/jrpI9Az5oH97+EpG5tcS3mA3Ln0NA4hSDUU0a1GJKh4/eAOA48XL2b9lLmJeDjf7FciZqIKIt7aCA4u1fBI3gN6XT+N+o9LSx47IPgCnbjMDsmbi3uJOuCaRdr0X2zJeIE0xhS92jabXtTQuD75MTaAlI08JSBKLI4YgunrjV6Pn1qvfYxBbmdr9Ey+GhnKXm3y4MUuSTMljQ1/J70yokCniHoqTD6s2zT5JYXPkS4IJk8Ua0YguYHLlSOk8hrrVsFjji8otlDMjj5HW+2u6aESrbKe3zzXWx1/h1tBdCOVppFwoICv0MQD25jyOyayi4/Ahkk98Djp5zienJ6MUFJDyLuibeWN6NLp7RxPx6UxKIksQJCVKJSiVAnqTEyZkJ6oVdYMk8NnUIawxlLLm7WdQXTXTFpKA3sUeADcJOlpUp6B9H4aadjS6MDwIt10v8d3Apd4zKVRtI3ZHBQOuDbAZpHqTFnXn27jUR+PV6ElcbhzuLe42J7lOVc0H+avIDH8MafIUBv38CJJVBHrgZ/Tuvs1onRvejSouP7GVuz6dyryd82woJ4UoUh1YTe7QQrQ+WlvgVqmU15r8wHeYvb8v2dpPSbx0i7s23eUQiAZQhPtS3CuIMo/0HtmCoNB34dJYhvP54/iVXnMwKkWUxObHMuxYGTqNkQ7XDkyiE9n1wyhs6s9PQl8OTzzK5rsPsHfWXowqi1E65Fv0Ke/xlMqZkedk6lhXhqIxBsrXJxxlff5dtDnpODrjODnxZRzzusf+XhFb32WYt5MX/Do57Zdt18Pr76NP8QaCdRPs1BqigNIsZ7JOtPjbR1RuZHTmLea7fIwgyCi3Dy9uIlwBGx/4hp3z9tnuBfg1/TUAhmpAr9ZT51eHSZToIhDKdsKtlzCpPJFii/lz2hly4ytAsARzjo5jXtijPHb5dTbvr8S1MdsW7BM1ARA0mRC3XO5wBbUA24PgzzD5MDgi7DfuiTzKIjfISK0lLi+a/mn9bYeWMPfbmNTVVLmepUNRhk5Zi15schir2a7XWX9pHDP/kHULrffemfRvlib9i7S2Stx3ZbHmy+X0v97fvgfOzIZ+b2P0uYtjd6Q4iNHrTC7UdYYT630VKfcm0sVrjDlQzKDLgwCR40X2MXvRW3ZqS4KVxlOFKJiI9LyFl1LvYDQLkgoDMMm7hOUmdxIyI5h2Zg3zzpSR7CHTnxwpWMW5jnB0Gh0d/iIqvR6tJNPRlbT0wce5guHNh7gRAbvu+Ymdsz+gxnOP3CdJz8OMTGY0RNOcFcPkI5PxbvRGp5aditUmeO3cZmK73+eRiG8JbJ6LR1cKSX5neGHUHQwNPMUE0wD8Q8fQFRSFQdFMt7m9x2GvB7LvH4N9f8lEtfTnNzPj6OV1nu0/30vi6Q2kRc7lStxMdGbZSRrhkYHTzdVU9hvEj/f8xJv/etMe+Ep7Fop+plX0ZnhfWf9Jf+VdrkUv4nbTNdT577Ks7yu8mvo65YuqGPrnVlzbLLosgj2j/nTMGF5tCaPRVXYm7JW3N36pTuLqoKt0upl54v1ZDLg6wIbIsdoHS39dSkBuIdHZehKVM+0N7IHEkDDzcxsUaibg71xCQVMqweIRJnjcSXsvE4cmH+J2jCP91EenX+b3KW9SGl7CpKN3ozEEy/VtLyavfSNH+4bxQ+sSB2RfSLAl2CdK6BVNtBjqkJDwFqIIE4agNYRY5lqPDG8kQp1jOR8GH00ai96sxPniACLT7WulSgX3rhuJFPsgo7UG4svC0GtUNAc5UtwdGn8/PwSAU5eC0HLZodLlJ1MdsjMAdgVyPLyD0rFnKY/owVThPYAclw9YdmQvj1fMJ1aY1IMyXwE3/sVi4wsMuGiGPVk8+M0LDE5/HKVC5OOP4fXX5fHITsim1d/sQGP31/ZaE+sUPfT+dgdtYuQLGzkzWWB82hXmt5xAoYDUoINEed1CGbuQOq8U/IsvE5LteDYVBZkmz6VTi0anwbnLmWnnnyPMXE9c7VIeHXI/AzRteNT52e55MbIWrssJVSqFju+dH2N1jZ6yfg9yz5qXiKl6llnxn5EQ9SkXBv9OTkoVZyYpCdYuRxShrKU3J4uX8ty5PCJ//4icAW18u+oNsjpkWuDpcV+yOuY7NvhDvxtRFETnsGf8PAZMncJPyv9nR/UNSoEEfSqNXo1sv3M7IMhee10DerOGaoWBrJRsJFFho81EE4jk1hs/pYngkmYM674i8fQ77Gt6p8dY2B2OtrFwOHuIqHVqxpwazew9sxF7ZJabJRE3Uzzhqu38dN9ZysPKbbawzuRCgzmMDgmmHRjHmi/XYKO8V3tBwlPoej3GO41weuwpClOmISDI9sTNF1kbm4LKmjn+D1pcTyUP5bfQZgCKAj/ipPdyTBZbqktdSlrkcn5bms5PL58jr1eeY7D5H5B9PfUmPZRVAJiM+fjW+zo4zuJ9LmC8OZywho/odNFSGzXEvsdNPkdp6kYAYgpiGHqxP3qxiUZDNAz5HoVgoKI1ntfbFLgXRtD/5H6qvjtge3YvH9l+WOIxialprX9DTAgCVHn9xq8uQ/EJ3M4kl+8Jcs3meNE9fJe9GleViZNjr+HariWyyK6hOTxsB3Nc8+jWdmNQGZhxYAaixXHr51zC7wu14DeSvbnr8GqUGJH2IiFNi+X7+74GQ3+gVXCmXdON2qAmsijShrhbUQMnKqZwX+rjDjSwwwO+Y1LMdzxwfCfnu+T5NUH3OUNzj5DkOdjW5maXq+SGvEq6Jdj3V9eVb+tE+hV9z12xq6j6vorze85T7ybrqomSGoWkpXzrRbpLugmqCpJRnVYaT4vt678tn6RLvQFHX0GjWUthYjbFw5TcUn/LpV6TuaT/HoUCjIo2Dhn/xdsZL9PlHoBO022ZO3L9lGY31AZ/nARXutRlHG//kn6RG9i3RCBz/FOMuxxB4Kk87OubHdnn0u6CT52GgpGxFMYUOgTSHKQWbBrf8r3PHz3Fq2Xr8av1I6Dam2/v+5ashBsIArw3/g3GDH2TB5XT8dk/kSffX4ZHswdYkgF1lrHRij3RNo6OYWddDL0qXqe/fi3QUwNP4Yjskwz2YJ+k4ErlTBLcClm24k++eOwkRyceoSq4Cl9NAKGN83nbF7YGtjP3bDWDLkQz4uwIBLNsMwS6yskaRUZYtFW2C8+MPMOR4e9Q47WbXOopMA5hZk0gXrkxuDfUkBHxMJcFOalFEGSK+Lulw6xw29KjXnKfunb1xqiNoyWuZyaUmVbnG3Rq8lFKsmN94Ilg5u2ch4Bo+1tw4yIEAbpcfbk6qIQGnwZEi0TJyr2lHO5IwpAShan1Xzz26WPE58j+iza9D+vPb8FtYScTdx7Eu87M4ek1FPt9YVsrwjzCyFikozBlHk5N1cQUBDO/Scfl3t/QHvIEP954h3CnVvx0si82LSUNvYUJRCEYmBP/ET+FFuEsgN5JgUJrD5o1up4j7PpbXP7oF14KP8WwnFM4K9wQRVn/Lq7jHp688iQNkxpQ69QOa6M1yGQym2x0h4KkoF2TTYv2Bm61mfQ//D5lh2V/S5tkT1cxSgakygoSLvxI3ZksK7AKjcYxmNJS1oJkkjDPmk134BpeTthOkk8Kmi4NvkUtnHj5BCX/3oy6swlRtGuOSYIJo9qJ+FnxtHvJ46kSZWTf5W44t3USY35ZxJ3f+jB7z2xAZErMBpb2eYU5+YsZ/tx23nrlLVu7FAoREp+DqdeJzU6k1b0VgNjboRgVbQS1fAw3nkcUFZSHlXN2gQK/QX7o1XpbsoNJ7KTV+QZNV5rw3LuZcX+62vqs578ahTP9SjbSr+Q7+zS0IftkCm2DooU+KXpcXSWbvSQJJgeWCyslo/VcH14SjpPRhYaAQJt0EIDOpGNv6U8cHfsD+Uk6fOt8bcFFkOV7fqh5xMIQgP1aYxpPDr2L/sG7WfrBYJo37Wf0H/n0udXHtk5Mj/uSp4ffRe8uec23rlODBorMmQPtYhy1Og9cBKj02coP5nEIAjgpO/lh3DKej9AixKyja+5imgMTHM6IoU1LCG5/jLLhgymKLKJZm4Wz5aysMvpwRxUMoJKM5HpSTW8S0DLdfn/YXB5tj+d6gpyIn+81jS5VhSV46cvl6tH8mf4eI8NGciDvAM1aWabCqtkHkPH1aaR/v4l/TQB+rZNtUgLWsTpx5xVqQ8OpCahxoPTtad8CqFQCSiXsy32Mxq4AzpaMxGwq5XZyBSWRJfbAUOhcHjEO4FRkAVkDfPEIkefPjBlAxJ2Q+iEAz5nLce3SkhuXy5EUf/ZWf2ZP7pTsc8WvdRIBTXP48kOvv9FrGswGrlddx3ezLx/4fUBmfqb9YmcnTu0NCGYTrg0l+JTdsGVdqoamcmPiU3R4hTo8rycTC/yfG+wzmUxkZ2djMv3dTv1fUf6Hg31nzpzhxRdfRK12zLSIjIykoqLiv7jrP+U/5f9bcXKCN94o5Ndf5QwQURAJr7+P6NZ70eKF2uSFRmkPPvcr/p5x6fnEmeY6aPbZiiBglGDL4u289fwbZOrlzJ2psV/z2tjp9DNOJqp8DGs/nsnIsyMdqCMPV/3K0w3u+JcXMejqIIbkHpadECVb4dwiepV8zczoevLiygl4bC2pRVsYoF4C5bsZG/A+sb3Xcv29jfQ+sc9WN5PGHwZvYEfOS9xo7Ivbt2U26quoKHh69hc4C91sdK3lzKQaSiNL5Gb0NMAEJUqj3EaVahZ315XZqEsFlIzVwiKfKjw7BxFR/wDxWtl5/e7EkbyWGkFE1WO41pWi2L2ZKtUyulTl8sLclEaVJgK/y/0JL3Hh1LhzXI6dydPbk4hYK3B2H7xQ38KGdhNvtDeg0u1nY/MTnGrpa6vb0lPfsaYW/Grdmf+DDxOvz0Bt9KOmI5oj5eso0zmjFsC9MwU/VSQAm9P/zX2X3iKgOoD4zECawvrS4RXqoJ9oPeDYMwgFJAnqOsK4WD4PVcNhkrN/IuXkSY6ff4ZS328BaI+BhfWbERBp8GngZr9btLm1oTC74Ocays2ELuq6fOnr/Qgf/zkPUV9OrXQbf3+YM1NHsuobxJkLOTLtBLHnmxh5ZQUjA6b3qJec5QUQWRL5N4SE3clqdfDbr33XcA99SkESjbh2J+GiS0CtVGGWFBjNatQi+Nb7YlSYuDzoMvTIxELpws+16Rzv/wUC0PLeD6h1Ek6igNk1nj/zHqCmI5oZX4/k6sxX6PAOc3w5MK3xAM94X8BVF2czvvbmrOVQfRy+JWfR1+VxeNoliqOKUShEVBbKQbVGT0HqQnQuIoMuD3JA9lnbP3WnglnbB9IrNxlPtR9teh82p/+bPYqVfBebxvEJWXzQ/xIjbssIOeeSD2FvNIuL/sU9K0opiWyjoPwi5b4/ohQVqKde5UZ7FEWR4CN14OraQ+sIK7JPhYDCdqiwIfv+QjtjzeD+JjmXaWltuHUn88L5LM6/+R5JZ0PJD3yHvODX0dHKieK72Z+/gggVaAQo7+vF1kVbkXoYc6uyHmJjbgNqXRvKPvIBz7vRG1FUQuIz0PtpMloX0qrzxWCyaycq9eUkevzBKL80RlWFsejZcQz943u5LWYNVypm0KmCmsAazG4CRknvEPTSq2rI9VjGwJPXaX7mF576YC3rVMUOc0+tU+PUKlE6xovMpEyUopIYr+s8PXwJIyJcyJzwL5wiY1n26zKiC6MRJAWRnjc5MqcPNVo/hlzoT0t3CRnOabRqM+UpJAgc8pzJHzMOsO3uSkJm9AOgOrAapZVOWFSQH5fP+flXqUivoOnoJjqUF2WdCAG6VVWE7fZn2tdNuLW04aRzsh2EKtUrcM27h1uGE/y49AG2hsyyZQsCuDaV0fv893T1G0Jh6gIHo9LNHEG3iwdF8X40e3YBEGCuYePsGNYOXsURoRdnu6E8tIrrA65jVtiNUmX1YV6NaWXX6m/JmHk/TQmyBpkgACY9vd1KCVPCuYFn2LLkO5o0+Q7fk/2wJo9xz+w+37YJRNTfj7cxmZ40ntY2Z7Q8yuFDc4g75kdYsRofwmz1WpT0b/JjW/HwbCe6KJyQ8hBb4Gt39pO8YvyR52pFRp8ZTZNXE0YnE5KglJ2oQVNojl/HMmMQZVFqjAor+l0B2iBqdb3Zr2vheMcpOlw6kAQL6tCyh/773EH8C8EowaxKmFc+FEGA1CA5QeXecm/0zhqOTzhDUZSdanNIyB7mecnorErNMY6kBPCn/3iH/nrY8xCeedEMvDKYRz57xHZYGhD0JwrByKgKJTf63QBA26W1J1G4x0PSczQJ/Uk9W8Az7z1jc7o5q1oIdC3g51uvcyNlI6ZQGXmlNCoRJAW/ZT3PXTtryG4Lkx1eFr1SkLNvnRQdDAvbzcGmYJpsGYoCbc636KWCf0ed54VR+ey/qwzX4ky8KzNsOjdHClciCiIRoppb8524MeVZ+/wBtmfKGaoXzyYTlZkk65r9JcAtIjLh2ATMgpnK4EpbnzztCV+Ovot7naMY773C9vsk/zO2/96V/TSK4UNo69WPC9ERDN7thkHRZJmbdie+Fc3w3yH7rMHTspbe7Cm5lzNNHri01vD8288z+NJgVEqBSM+bLEl+lR3GcE7VuOCkU2JQG+yUloIIhhYmde/j8YnpFKfWcDvuJlXe22nU13AjdBdPHz7Lvfn9MCmsTjn57QrRvtd1K6tolsqRBCMJziP4qT6Ixq5ADqRs51t/0DtL5MZX0OTVZAsSWefDqTGnKEsdRdboB/9C7Wc/ECOYUQD9On9lcOhenFUtOHEej64TNATs4fLgyzR5t8lrX+1pyP6Izzt/5avUavzqY0i9OZSAllny9WPjeUFzP5diy/FTFlucH/L8io60jIGig8P9vVmd749ZcEQfyXW3J7CpzO78PjrPdq2XcwOb79pMbqqe5OOf4lxmQSy4RSKoXHnYs4s52+4kLu2m/HdLe+O8r1DWEcSfHTD/txGEVoSSlpJGlucvdBo6wbMvhMzixSYNpR5NDDmtILwkXKZ39Uik1PlJWutnsUj4naniB7Y2KQQFtOUyqFcmJQ9/T1ushE+DJ72qHrUFbAVB7u/ysHLUndD3Vl/b/JLb25O21I7QjPW+yhfTk/BRdJGt6SAqT83oPX/gX+WNUgkZtWP4+OIP7NAMJ9tjB6fHlnJgynbKfH6S2128lTtvLuKx2CZ2zTtIo1cjF4ZeoCk6Dtx7295fY9RQODCNiyNbefXVVwn31IHSheyQ40R53mBe2JMoBDPO+kgmhs7Du2M4mbUjmeKfzukxp0mb3oHW6QXipTUIAlyrms4HF36hrjuWgIJzRGbKiAurA2jj9Y9o0HmToQe1XoFg6qRdm8XsWcM5ZTZjx4P/c7kEnDdLdC7NZ+virXRruwEBhaCHHb64Kw2suLqOKVnz6HVpMzlOFvpk/5FIrr1YM2iNLQBYHlrukEjo0TmAPiVfMUR60vYx9sxQFgSBcSfGMe7kOFLTUhEsdPq9fC5hMqv4Ivc0tR2R9NTAAojyvEGq+ijeIiiNIiqjCiQRN0UlRCyGsPlI4fN5o9SZCQen4NTRYG9w+ALSW+/kX712MthJnicnkxKZvq8vTZ0NULQJP00eTSbHhAWVIPe7QdFEhe8mqrx+swdM/gHZJyDi0zoev/ZxDqbyHaUvMuuriXjU+vHp2k9t+9CAoAOsSHmO2dp2PLqcMStEjE4udnvEpCfq6nJe9oazI8+yf+Z+kASitMeg6hBbGi6QVj0ZlVHF8PNywkiZZyF17jIC9Nf0V7lvbwFHdRZpCZs9K1rGxb6OVRuc6DL7Eu6WDsCGUfexMUgiP66U8JJAooqibAGmGzUT6TKHM6FwGx1ekaQnp4PlWzVJKpyU3ZiKNhO/dDImpckRLQtgNhAutFPWN4M6X5mZw5pM12SGKPd8hkftJavlCvVuJ2kQclgUs4Zor5vMidpCiNIxYOtAJWejmzbY/t/BvuvuTVjDvQwOHE7zPc0WVITj3tXvo3uI2xBHWn+Zlt46llaHuNhtRGGwIAZ7SH4s717MzEoY+EM3w75ws8wTS9KJpc/HHxpPaNYhBPPfERPWYrUntkWV2P7mOdYJfa8+LNgymrDSMJudbNU4H3IhGI9Kec6L9LAJbHTTZqyBQuv32GHwIlqTz/BOTwZfScGvzg8Eq8Z0JRwaxHRzAQ0+DXRrdARXBgMCfdxH46uAxhjYZQGmm809KEittpQ+kl5VL5Gsk+kArcmTMo2nvY4mydgj8dJiN95+jEeaTJiURttSr1SI9C/cTl2H7CQOz/iTxBuNTDo6iWFqWbve00m2GWuNkNuriLSUNI5NPEaXs2yzfNUCP3d/SptZyY8rfuR2nw60nVp5n+qsIKllGW9PGE245qRl77XXGcC7Yzii93o8Rs7h4S8eJTl9ECoH9IqZiOII+l4Kpt+tfggINopzk9hBgdPvFESe5fzofIu0hn1y3un2OS+UJDHx2AgafHQ4uQ78G02iU0Ml6m4d5wd/RYP7Kfl6wXfwq0BEZgodXqF0BEYTUu5PVZ0H7W6JtMW+TV7jQP7o+ytL51ZwZqKK4xNz+T1Qw/uThvLcyAXcl/ok091acRKgW13Odd02WxC82nMn6s5a6rILCSsKw79xIEpRZdGi1DK68Ufmj52P1Cox5tQYh0CF9ZsxSfZgH5LIuYQRnEnqT7NnB7XhA3AO8aLmVg2K4mJEk0igNgyNUoMywIfcmDE4x4dx6e1jeFVmWpJc5UcZjZD3Rx7FJ4sxt7Xj2zaWCUELmRQxi5jSGQw4UselTy/RlVdGq+YCBcbTDhSYLtdP8b7f+yjr5cCWUlSx886drGh+jintPoRXhdLtEUisz2xCG+4i2C2PJL/TtJtFhp0YheF1A1O3TULTpUEhiOAcDN79MaFgw+oNVK1pIyc+h3i1Hq0hCyLvkpN4gfoQMxMOTCA/Lt/mX7CyMjT7Nsv9prR+U0qH6/9I42lNOrFQEB/rG8aybCcKmgp6oMbkQKAkOgb7nJTyXufc6YxXmYKs4DtJD7dTL1rH1L/Wn0EnXQipCHFYu6zPt/vXLOteZxljI3/h4YGPc2hIKZ1urSjMMH/nfIZ0vmFpi1yHtZXXOJTiZTurJyWK3HcflLYPJsm9jIUuMoK7A3nPcFJ00q73ork7ALMZihR/Uur7HRVddmpla6mLj+WnFT/R4dZFQgJMnQr3LwrDS4Q4hciirePQXjkt1936TRX+yHmjNzkh5Wy7cxtFkbKtLIrw7NGzvHHiFDE161iafLd1BGz3KxRwMqk3r5Y9jGA04l2VaaOe7TlW+X1LuD14DJIqiHDXmL/1d88yYwYUN/dlSc77/Jx1koGHNuFX42r5vWWhODKKVYpCdBJE30rH8PFLFHm/Zl9HEp6A6BVUFQUjGgxURXTa3vdX1hIw07dkA4MKdtOYF0eVnLPEscJj/HjjR5mCHygNL+XqgKtU6+2BenXaJfqc/Bynjkb8Sq4SdXOPLYonKVUYNa6ytnWP8h8aT3vp7v77ue5/VfkfDvaZzeZ/jEyWl5fj5ub2P6VS/yn/Kf9UXF3Nssgq9viE2fzP8GCtIRQXXQwqydWO7OsR7FvfMIPHnlnNg18tkwWNLYtwsGs+A4IOMTr4a5w6m6j1a6Het17e8C1FFEQQ4NKwUsxjxuLXOkl+d8QSGPkbzl0l3DvxJrvvOIPk4mqv2/DN/FS8l6+TduGfUIJJIdHv9v081O9J/L3iIfZ+ilsHcqxPBO43jaTcTLEbjSGzOes8CYME2jaRUadHyc7dHpmmoqCg1b2Vc0tGUd1rDm5SqO16uyabE6GwPMuP1AOvEXljt+2+6vZoyjuSec+vmN8f2IDSqEDTVoJJ7JL7eVYeJ/0WYVKYqAqsITPqOm03bnL6zGZOmySO6LtZBkwAlgEnJCNnkRDPZ1JW/BbrJw1hslcZ92svMOuPZTh3wNLot4iulVFQp6sfYWjaXG7rAcHscKB+KHYri1qCGXouBqc2WefDGuzbvUjJteCLDLk4hPs/i2PwpcE9nJcKThUvRV39G3PvlfX2Um6k2A6fv7WDICo4493KHdeGUBZaSm1ALUaxlQm7Ell0eigvXHuDlzICWPHTCh766iF78Motlp+MD7O/A1q9umgYOo1WnyiHIKSAkgbfBq7c7cX3934vG0kOFFU9M6Wt42eib8AxQlVVRBdEM+h8BLNO7Wby1Qu4qTzw0lQxMPgA+1SHiFn/Ph+u28KBGQdAsBx8zy2BXwXu73qPmLFyNpKptJKI9P1olG343RjF5JiNBLgUIram818VqxYSwKb6tWyaF8js+E+RPI+S9uSHXJpTT1Lppwws/I0YjwRqOqKY/FsNYwqe5dKAHWirO5h8eLID7UxY3X1MrN2Ha7v8h4YALX09Rtr7QxIpiSyhwbeTtjcvE1xuOfSqNHJwInwh12pmcnB2Kd+u/tZOw6MNoMbgTZ0JTBIWxITjAdNarOtEhH4G/Rvfwq91ii3zDuzan4IAbuoGfpgTytD/i723jq7q6v69P3sfzYm7K/GQECQ4BHeHYm2hpUKhUH/qQl0pLdRLSylOS1vcXYKTAEkIIe7uycnR9499JGmf37Vx33vfMd5njcEAzj5n77WXzjXn9/udfptxqCxFvHGdR356kNjsWJsj4JsbnxJ1KIIHVz+DoqWDO3F3UCika4/3eYrZfmfZPvYAP67MRTl+BLsWH+VM6hlb0uxO71QmFeRzo+4V8qZuZujZoQhmGcr64ywImMIX/T5icXWo9F1rvlKzinfO7GNC9Xh+XrwRkxpi9rxDqbxPN/Rdo1sj+ycdRN0rmnZXv384TgLLAkndrkGvgrKgMuSCHE9NKamh21kUMJzH+67E7OfBX9P/ojikGAEZ4yJ+Qi4aOH0ngDHHh1NbepD3z88m1/8dqS/O348cM0faodVVT+OtUmp9GkkbnGZj1lgPDyazkYwDGbidzedW0EJbsE80y6nwr6Aw1sDV1AR+evQnm5Gq6uggNicCQ509N7DVeG52uMmFpOe5OLSeO/Gl7BkSxduZ823fG9P2Ez2rLyIq+jH2YBje1d4oLY6IXn4n8BK0TDg4gdRTfSVHktWZVH4QQe3HcgZS69iLcq8blDnuRyezsNGurmBjn7d5RnTimTXPkHoqlW4Oa8HuJDFa5Y7+DbRNGpsWh7ZMRKeQ1rptDSM5umcKbrk3iLqyHbNWYiWMjfiJlIB9fFGvRl7txcy/ptEro5ftcGBGRIcznZi5nHKZvB55YBYJlh0GtS8kvE6cdxwPtmZw367FPPH1OHv/DNnK9pJtgEhejzwa3XX4VEsnGnnZbtimoH/AbpubaV8bZOlcEAT4Jf1jluwuJO5qDWNP349CL5fyYljqdbFsBjV6Fa5iV8aE0G18bm8ZhsuzG8hObAfNGBb5fgbAaydO8MSBfBAE/pr5FwcXDkLu9g0uBsthqvEWWYWH+bPmTQxCuYWNKN14eMh2fpwaiY/fH6zsjOWLR94lPXUWuL9NXNlHyEU9OqOa+Wmr+LRBcnC5tvUloG4BLu29MJiUbEz/kAP1dtRinMMI2lUFfOoFHnITrx67Tk/9OioHTqfRN8a2H8R5XeB57ws8eOcPhn1lRl7xMiUevyIIEhPy/kQpb8z1jB6kXBxoCUDKpTmV/jKcnsZDmnpyBl8iNymM0Zkt+LVLAdI3PCDW6xIp7j+S15ZOqccWGjXXGD1COlQccKqi1iCno/we/ke2EZNjkW6zOSHsqOF2lXSY1shc7OP3v5DxfO/sHlaXjGPbnJf49eFzFIYV0urUipubQK/YesI9blMiOjN+wyKeWvsgczOOEF79tNTPyR9CzDMArGoycmZUJdmJkgNUJsrQqUK4UzuY/Y3+vPvGe/y5OIRm1+Zuz+9aN5Xej/cizvHX8QLOFC0gSN1K9fo5RFwUODvGTJN3CJ4aD+sbAHCt3zWK4rwo8T1JofG8/Z7mLs4TzPRSwU2nRZwuXMiOOW40GjsYv6M3w3+MwK/Szza+KN0D15/juiyUfW1wYoqMnIFSrihBAKKWUWkKRCmAYJZyIbm0JzPc+WGGhg621My+X5kFA3eFPZwzf0qTw41udTN3ATh8dfl7fkn/iEGuxXwb20S9ew1NTnlc8Fph77ToFYwt8+LE2GO41dUTeO0FTrZ8D8D9iW/io67jx2bwq5TaqNGtkWLhd5o7m2HUUUjdw1dNGkocWqn1NdCp6rQ7yEWI8byIj+KGpW7SmioX5ZD0LmpHLWExxZQO1ZIx7gVafSK6jSdrH16a18neqQe7MWq6y3jaWc8Ckjzr+w1jWH3PDb3STE1IH3B0RC6HspYYjhc8xCP732DyhkICC1vJjD+OXl5ncfhIrKr+XgNp8kth7dNrOTzhMFovL9AEUu7xBjcqx/BtUCYL+t8js5c09s4YnGHYn3i2bMHfOZ8f7+7DaOpqYwjcqBzHyZoEqd5WxkOXYLKXpoSenocoTZ1JaaSW8Pxwm11Y3JzA3HO7WNsIBydfJif2Hi5NLnj7B7N8YhyTZPL/MuB3CZgqh0/mw+Q+1ZgFM66NrghmUQIgRS4lr3ESPYveJ/CsK3UedWT5bLT93hQ8l7X5YzgzeBPb373Erjm76LqPOXZGElrzBJHGqf82MCYKIkWhRVzvfZ21K9diFi05SQP2sizlSZwV1QTfPsizn05j9LHRtnd+ffh0Fjq/wwo32LpgBx+/+DECAs6yYsj9Fq49jahvQjALOHQ40KA5xLWIuWQ3X4ag6WQ0Pch9QeeJVYIRHa0O2eQ03gKzAdIWkd00kaVldukyALkl2GddC6LuhvLQO8Pof6l/t0Okdb7JjS4Munuc1MIT3e7TiRq9QYbcoMCttS/eMmlsj+uxnp4+Z4iuiubLGb/z3WNvcyTRgVpni2x12W40jdfxlsG0PdMYlDYIAZEg9QUo2kaY6jCDgv5gi94Nnxpp373RLDk8QZLnq2qLwGC2gFtt64E92GeVUtsTexkHsZZw12ukhm7FWdnCX63Q6NbErvsLuBsv4qyQ1nuNohkPWRUBLYNxa3AmJybH1h71HVL0R2bWs8xdx92Z56nsIWnuCQKQ9SlceozvlIdYMDiHr1d8TWF4oW2fCZLDO1c+wb+wlBUZ/bkYM5Izwjt8m32AgoYkJoX/xlJXC6P63zAArOuEEZ3t/38PmABsP7YZt41uktPa0n8Njpe5EbaIve5bcBruRKe6s1uwz8rsEwxm3GrdpPfsshaHiY3MujIApxoz9RZVGhtgwVKvwecH45d/kZAiq3yz9HmNy1FuhTzJhdZNWINyH1f60apz5aOLP3JZNoxOb0+8a1xRdaps53XRpKLFTc+pMWWYTCacm50RBXuAs9vaKJjwksE4w2ZUsnb8nXJ5OfBNvhnuz7VRE2hzlAJfgoB0lop7AZ0gpzzlBk1uLUw8OBHBLDLMczaLLC48Y+1UHvP61fIM6/jqbrfaZLS7KKl0VQIwmvW4dCQx1fdJvJvH8sGoETwc9ht3q11IuhHIsm+WEZcVJwF0EXntxAlOa65TF5RETUgfsoY+SrJ6LgBppTMZuzudna1wcNJJml0kAIZvpa99DBj38YpbI0Pzohl12A3POk+pPtpq/LRb6elzlnCHExYGlDVobN9vAFSCO0pTDL2KN6GQ28dAq7wYtVZNp8rAjrk7EBCI1M4ntHoZcqMrh13v43rEvG5roygYGBW+kTx9Mludq7jSP4eKxOfoKUr7T7LfUfYuEJAdiILJ/flzkXWdstjDlZIsa6djbynrqU5LXHYo6y+OZdTlYTiVfsn7o6TUDZNmv4eD+lVS72QRph+Lr1MBA4P2ADDt3lgc5Qm0qDP5uWU+OYH2HNlfPvMlzQMNlL2/EaeGUonV14WJM/iFwYj+IuGF4d1yaP47Zl9hoWib763uThQnTcGtZxBpq9MI+/MyMqMMJ8taI3N3oTSwN4oATzLWncOtKgeZzO7bMRggbbWkFiJmZaHoaEYmg5f7v0Nk3a8Up8zikQuPEP7DK5zqO581jandJN5NculGelE6Y5hajXz/+Xfs/mU9bxZs5xfxZw571KIZOBnHzkg2pH/Kgl0NeIidHLl/GyRD3K0Y5Aa5lDJI3wKlu3nYow6DpoOGgW1U+1bjKAKYwCmCF/p8wITLOQy4MgZjnpEe1c8RVrO82zir9amlbuVb7J9VY+np7sw+g1mPXmxGLzbbx7XVV4QRk8lsY+ZpFBqbT9MsGCRfqIXZp7AE+649fg1/MZE7cXfISim3kAG679sggR6LI7QUhhV2s/ntzEHLWc0C4iFwCnP/0PLttS/4ztDBuYkSyNNpWDK/rEplzhwoUS9j1K48NreYpDQptrkh3bNKG8ef2c8TZnrX/q6ClMfXz6mA9PxweuxdTdInl1FVf8/txou277VobtKkPEHw6RMk30jGOmeefBKmToEjgXBaLqky6FycMWGQ6l38G9xZgyiIeNZ54tTqRJtjm80eUco6GBC4m1DXW5jM3YEbdhlPkduJtzE4K3Gov47rqB9timHYzlNGHHURjMu5y+5J1231DmydzpiMcibJP7PcW+Chh2DVKpg+VaTCv4I78VXM2TyYESdH2H2GoQvIwY2Yez1ITFeh6IBa1+PSOzXnQuE2zEkfsqb5BBkjllHr12Fra+ucsuYG73p+AEkBBGB12moe3v0wZ4qk4OjtxNvsm7oPuZN9HdQHhFIRORSDUkNVxCByUxbYNl9TazvqlhoEowG1mn+U/7/LeP7fLv/Twb5x48bxxRdf2P4vCAKtra289dZbTJo06b/+4X/Kf8r/5lLjcpRyxwOk8Tk3Qx/nZtOZf3zHbIbkgo1Mya5ictCDts+nOacT1zOfZpcWgkqCbAv6z+mfYTSJeLnuITRjF63O7dyJu9ON2WfdrK4MKsY4NFX6TATcekLIHDIT1+FbAD1vhdF24AQmQY8gmEHlSXHHEJ7Pns4Fj0pUnSKx+RN5b/hqgl2lfF7Ley9CH9dMm2Mb1X4CI2SvSg/1H4eDgz9zalKZ+nsEo0+MJrgk+G9oKzlm0UzcqVsE3jnxNwe/BdH+RyqiCWTaAmoNkoTWmou/8l76Nb5rgiyfSj751ydc7n8Z0epw1DczumY74YNv8uPjP1PlWIXudCkHTZ0M+C/6ZgBw2GgiM/NN9LpSQpwLCBf8ca+ppj6yH54jJdZfH/9DPJuYSj9HCb2Y6/c+Ge0HARgeuo1HonayIM6fkr7TUeV/zT3HRPLaJHS6TDTiKBpp10jGz6SDk2xou/6Be3lt+EwAqqPr+fHRHzk69iiiWcHzERvZ5DiWwAFrMRhluN6KtzjuoF19j4KmO7h3ZtCg9eSSk5Rs1iyYkVkdUbffY7H8a0zNjkRd8aHdVUWzuwbELoE7a04JzziiGs/ziOEa3iTY+8Ny/bTbEg4ne3Ko5nuUsg7eHzWGV/0/ZU5uAgMuhhB9aTMepTcRRRgS/DtvpU6hSebKvaa+hNUsZ65qPUF1D0n9JJdOaiGmfHb4Q/ZIyYGaFXOK9PYzaIMeobY9iA/vTeCB8+9T4fYn2YGvcKXHdOotOcKgewD9cusu2+cfRZ7i8fAmlK0mBpxqJeZeX7wcJINKL6/nvPgBBb5fUDzch18X/drtcOqi7Ulo5xRujHuR9199n4KQ45Tr7uKtKeK9kaNZpn+OU4HgZITO8jpklrxnph7LYeJ1Gvv9wI1zAcRnWPN0SbkFaMnjTHMC/UugzqzpxpiwGtA1Lke5HjGfbUWrpfYxjCap6RW8WkYimOzsw67BPjMCte3BKGWd3Ju5jM6J03Fs00jGvygn2CWLkcF/oFEaaXNsw2Dpewe1ZBjFel1guHsmLqLkwOg8cJKZm8cSXfYOASovOJSCLOtDqK1m/KF+YAK39mGEVa9A8BnCwYaf2Zg3k8OJN9j31HmujBkujStTdyvKpDZT6ddAu6adlL52B36bUxtXUq7g/dQCcgc88DfHiYwG9wZuD2nH3WsCI27dYWWPr7lcNpX5v9dzvX4gGekGGlvvkd47nXrPegSzjEs1izli/oKl3rn8tGQTXje1PLfmOQQsjpOibQTpiggoCyA814GQ+/qTPkDKJWEL9okyetzrQeqWQfiN8+OrJ7+izUlrC/aBjKyELC6N76BTYzGyLUHbu7HVrHprFWazifnb5uNR52EzhvWyJgqC93F26CHaHQ3I9O20G1vs72x5f5OoQBADGFj6J8kh96M3KqlrD+Bl42lS7kUzKC2Zlz79nD73fpB+c2cNsjuf8u7EM6Tc3onz3Z8p0z5Nm/qOxXEsMYWPaE24Nbkx8tRIZEax23P/zuzrGuxrVd+hxuUIjUIeHi2peLaMQC1T2a57uz7I0GVfkDVMRloq3BIPIggwMOgv/Jzy+ar4PmZsfASAa32vIVicRWvG9+UZ2TOEKM0UhhXiVeuFzGBpZMHyd+le+tS8h3vVHeo8Wyz9I9XdV51F1pDPWS5zJCrHCa9aSXZSdAiAwMk4K5sYahmK4zQwzKEWQZCkiRSyTnwci3AuzSY6RwrEWR3Dzsp6wlXtKAV7rt2uIAiAHe0TWFDpwOWhOgp7Tbd9bjCp8HXK5yU3E5EKcNLG4tLRS2Kimc1wIIn4CxOI1a4msNRJYsBZ7p3fmMzvWS9iMKpIFmVotHIMjgl4to3Ftb0Pzw+6n533ufJK7BZAOhAF1T9In4KtBDTMQ29Sc7tmOBHqVlxFSChaxyiXJwDY0AzP1kBxbQyKxhrOJa7jVL8lVBkkOeyhITu52z6N/MY+KFo6cWwqpcpF2uMUsk40iiZuxH3M90lSsGXmnzPtQIXyA1C2l9ddKjFPPUZhdACiSYZScKT6hWpuB5zjows/s75iCZsqXiE94gGEpM2oQ0ZB2P2Y2jdS0N+DCwUvY5BDYVihZU5Ja0VyzSeMullAQscT6GWNAPgpom1trjEG4NRhl83u6uBfGLiXjb5Q79nMpkWbyErIQhREHnl1JIr72zgihJDRS9qr43ZfxEEXYl+Has5hFJ0o/ukQM7dMRdEpXZAhoDJW4ahosCOaBXvOLJlgd9J2laURBFCIncyIXcOOynhy65xQtkOvoh8ZdPckff372ceaxRHeZjzDPa/F7Ne/aKuXwuiOhzEOtS6YsJrlLBP6kdT6K02dPhzJe4QyQxQV/hXUu9fzwOYHeLNVQy/xbYheAWPP85PDGCp+uI+ZW4w4uzRzf+KbeOiOQ/y/+FT/GXFFcE/viMkEfk3TeNz3Zxb1koKCXYN9JkHPTTZxlBdpsMiU9sv7k2nXzSTqJLaDIMDhvMfZlf0SHSYl7U2O1Hu28O3ytdyMtdjCBVvg8ADme7yBXP0596K1NLiVUK6XJIx25zzLuusf83Xqb5yemsLl/rcZeWoknnWe3XKNCIj0u9qPSn8dVX5VkoO85E9S6vrw2bhBTPR+CqPJzoaViTJwiWVC0au8XSNiNBrRKiroVFTRtQiIJGUkMfgPX0bcusKjnr/ar1nngGDPZyoTRXLrU3jyQBYVLOehjSuJva3i6PgbXPP5heHSNsn8ft9xI7iFyti71Pg22p4mNeRaGHOW4KiTDD89lAEXB+Bf7s/esAgqKi9S4fU2H537nXU14Rxrhwl7PHnh0xd4zaECLizEu/kn2nQu/Kg7whG3+VS6/cX+kq00OJ3lQslsVt2Zy7jD41jwYTTJh96nreECoigxvjZMD+HZ5ImUj+pNzGUFI06NsO0HSlkHDqJk9wSWejJ99wT8K/wxqzyoUp0lLulZhoswWhTZAhwDNgNjVGqGiwKvzIXqIT3Y0yjjie+eYOreqVILyxTQ/zvaDZ7Miv0YRIG8Hnno1BYHXt1VxLoLbDLcpcDvCxpNpdJ4/Dco9G6glC6ovAh1BT1T7nB5Yg5+bW8wWCWtjUfzH2H99dUovO5Hq5bASHqF3raO7Mh8nU6zA+c7QK/spEPTAYiU6VNB5Qn1V1Bmvsuu6DYavGsJKtRR4fEb9ToJjl6gHcvIU2vY2+GE2EU2XiZTwvDdnKl+lqSi71no8YX0uVll2xOsZ6KRJwYC4NTq1I11H1P2PhOut9Kr9n3bZ12BiCMdbzJr0WHSUzuYfGYTc9wk2dMtN9/llWOnQJTjVevFC6tfYFDaAPs+d+0ZjJ4D0fuco87PR1qTzSJ3O2YBEKC8iJ9TPsm+jRSmXKM8UEd+RL6tvm7qSmbEv83p5sFcjpzcTSoRrLaO9KxsHZxq92Zd+u8keEtrwqmSmTz2wwOMOD+b3qXHiHGX8p8dzF3GtsYbNLV7E5FTR8KdVPwNgyxvK7DiwE0AMou9KOho4ki/4RR5Weykkl1Q+ic7TQlk7xrNsh9eZ9TNAjSdPUh76BoXevTnu+EP2hgs0h0FcptHc61CyqXuWPARTxlL2KYZxKFkVzKaTtu/a2X2oe/2f2vRqUupcj3AuopPOTrmKKVBpVjdWx3KIsq8NnGj/BhteW2otCq6y3hK32vs6cLdGGnP7qoC9JX6T54QnNE5QM5Iq50k/d7qN9g2fxv1/vG0ORmQGTU2m69Jc4Min2/I7jxuc7J+WBnEgl2NrJbvZGdcIrcTbrPz4VZavRLxUEvnqaD6B3iosZ6I8iWE3SjHvcG92/7QNacfmJjlCIOM+xgaspM3U6cCkKN6FcemBu7fej+B5ZYgpNINvIcyx5jJGA1cSykmbWgpjp0xGDFwXgvfNyex9fo3eMlDLQE9e9ZAsEgSOtykRpDk3RYbz/Evcw0ercMQEHDX9cKlPRmzWcCrZRRPhn9FcN3DNHV606hzZcClAYw4EYdvtS9yg9wS4DTTqPXly4ofODL8VW4NCKPdrTuYedOYKfwRHkBI+UhSz0h+l8h7kQA86AwrnL7gaddG4nRTyEmQU+9Rb2GgJ3DZL5enT21nUfMa3mwM7CbjGeSSzbODFuBWewZdZjo5gx+iyTemGzi8U6wjJzaHk4/7E6q9zCLFXuRmR4p8viXP/2Nrp5BQsoYBd4/Q23MoKlkHzw58CEexhV/cyzk9MgOtk5fN1mjplOw9ofUeJQFzSUzL4qWPXsK2T/X/kebJ+cwuqSM67XMc6svpdNARkyIxb5zzJIWIt86cwOA3zVbX0vbedBocWXf5B5bvzySk+ggfhdzGo3XI38aOVFrDdVTPceBu2H6M6C2pMMx0mtto6WzhxdwX+Sr3K3677zfbb7oy+4xmO7PYumdb8ygbjZC8JJm8UUEY5AbuNWey4cYGe0BRkDH92ApKY8cgk9mZfa2tMPZTKY+fWF2JqW0TZ2v/QGtsx6jU0BCUiHe8NyYTGC0qCE5KJ1v96nr25sW6F2l1A+EC3HrzBOc++IC362v5tgE+qm/A9+4HfPx+EPl5n9OnD/g6FfBb5C7ui6+A1ZD2ZQkunVPQyJzh4kNwZgaf+Zcy8nQqrkelM1pGuys5CXUQtQxPB09cmzT4XzjKvkn76J39CrFlH0jvYLYz9yRQqwq50RnR5MC0aeDZMpzZxUV8PGATh/u4cri3m+1dPD2sCgcmizSu1N4ahcaes+9vMp6qLmdIT1k4Gm0PEtMCic+KRzCL/OtfliFrGWsR+RGE5Kste/M/g32mLsw+6R8iRrOKq2VTeb4WLsRncmH6Cnwem46PDyxeDJ6eXcGCZsKqV7Bx8m9Mi5HGaqTLaVp17hRpI23PEwTIqRvIprxZnApcCvpGANwa3Wx7c5LvcVKSZ3Ircire+ZXM2D0Dtdb+vpqz0+mnhtYmaTycDniZIp9vpHk3cAPEPMNZlzTGZScy+cBkFHqFDTw+IfJ7Xh8+g2ExH3CjwhKkM4tEeVxBlfsOXupC2xgvfSCOg5Mvs+be49Q7ne3Wz1YQvyBgI6gAKHBArfcnUhhLbOnHDNI8hFwOffVLWF7yAIv6FnIuNZ867xYMcoPdV+AcyXyhgJlNj6GXw7YF29ArzdI7VR2DCwsxHujJH8lOaEpfpM8JCTggimIXm8VyXvo3cudgD+D/O3apteiCIiiLHY1B5Ui7WwBNvtE254ou7Ro9T39DuFMN8+bZf/MfZt//N8r/dLBv9erVnD9/nvj4eLRaLQsXLrRJeH788cf/b9TxP+U/BVEUiYiIsElVCAJciZzG2ZDJ3BI3U+z9IyUd9oTx5e47yA58mWLzeeQmZxxMPjjINbbrgx3usmLxMYwyAw9sfqCbw3HJnhIm113h6NgDpCffBrrnCrEecg1CM1fN31Lstb67M12mJiQniokHBqE/cpYDfZUc130IbSVo5I3srU7ivH8Ju+c20OBncaQ1ZsKeKMJdr7OrWcnWxRvJ7zWBcJmF/aRvpm/dr0xt74tzM9xMvMmtxFvd6iUiAzO4VTaiN5zinNNz3ZCA/Yvh8CMbWbdiHV+ueI7zbXbHCmYRh3YHou9Go9aqLbJbMkTBBM3ZhLVnMUExm4d/eZZhXw5liCj8l4E+axkADBGUzPlxGQMC9zDWYzHpY56ntvc4W72clA1EuZ5hsV8ufjLQK+q42irJ04wKl5DHX9/8mU6NO65Njqi1ctvm9cS+O0ypHsCtpFtcGhFIQa/p3NcuoWcLG5PQGjSccJ7E1NpqyoLKpMTaZgVjfRZx5MEjuARNYFyLGxsXbURukONV44VOXoezCGnB8GDkFtqcW1m1ahWfvPgJrWYLcsyjDxmmgdCmIelUKJWtazjcx5XT1fbk2iJy5Ho57tk1eNaIuApB+Jv7AKDUe9kMbp3YiF5ej96sw2BS8vWVbwGBl+ae49iMDErix9Hs3QNRhKzaoay//jkO42/TfvhXxu7zJ3GrM54tljEy4AeYcJ0ONHyzpz+aSif+WnWHYyO+oUhXgMlnHL5ORbwSeYj95efRKkupdzpLlfseOhV2qv5+71Reqw+kUXMFk9nIPAvN/1xjMJ/lu+Jcacaz/Daqdond5KKs5fV+L/Cak5xnVz9GyFUTSsX9JAkLu80LUQSj0gG9UjoI1OsqUMk6iHC/gQ8l6E4MYsCpHtx8rpErfSRZXeuBfNn+ZShLdzDsdBgxd2LAGuw7lso7oVulIWxhJP5dxrNdWUC5xw5uNZ211aPrvAisu5/QhodsRtzrOeM4HrqQp0/sZ1jQWhaPC+Ri6O988ewX3E68jVwmo2/AQf41cCnjEovYuPRHvPOaefnDl5FbkP7PHbmKDBM7BC96XXfDmJWDaISeeSvxVfuBthqx6jgvRddzZc6fhKx5ADw/ILb8PUTXSHL0D/N1zkP8JrZR79eEXmmVnVGz9cG+3IrdxqNnRyJ0wq4Fp8mNzsXdRQpc2g5zgok6fRnnYgewXj/a1g+CWUajeyNFcR247j9Ej+x6/DUhgECb3p2Hc10w7vZDm5FOUuF6Egu/RzQrePfrvsRMnkGDpp2ywHKqesq5OOAitoTe83V46Gt5KqsPow77cO+n06ScTZDyFFqCfUEMZFj6RnpkhGEymaj1rsUkSoc8iQVnl27zLa4hPjMeNxcZYWEgVwgggKzGQFRuFGqtmtmzJUTfoAH236VsKuf+Lff/Ixm2qrUOt6q7VEWOwUE9BCfnnsza2ckzh68jYobXvyIzWYFRrsIme9TrAxi8BW15CclHP2PQhSSicqOwoW8n3WbW5TUcFtu5mXjT1r62tu7yTjYZzy6HqQKftVyKHs9dh80MzD3C4Lsn8Xa054la5tbG9J6ZpI1/g8MjV5Fu/BNBgC8vbuDxffcYUf8rDcHjODkqXXLCW1GImPEVSnjTQ3KIDD0/FLlRRhlToCkTjg6F4t/o67WVDhdvjo6/CkhyN9z9hkS33znXEEG5XuBGSiuOis85ff9lBJ+hMPwv+vju43iQJGF7OBDW+0sHI1dVNd9PieG5QQ9QNnA2okmkV3ov2/56NO8Rpt8ZSo2xK7Ovez9NdX2Vmafz6Z/xnK0NAUJcbzMy9Bfe8zQy8/tHGLBvK+qWavsaEy2xmczqTnbN/Iu8WBPO7RKo5G7dADZmfExqwAm+uZvCso8fIvHkOtszG7USQnxeyAnGa6Te+nvp53+At0MzmN7+Pl4to6Sga85xDjUEcvXab7h35BN4eAOe+RXUuB5Ea5bQufHeZxEw0yz60xAiI7gkGKuk5anCB7hTN4je2S+xIz6Hwth6ikKLJOlhAZhwHWaUcX9DJHWnUxj/20Wc6ksQBPB29EbmNIR9tbF8Js7jRssh+/jyGQ4KN6aYXuRRF/Cu8ERugFoviTFqRWAr9d5odGEoTa7El35Gqu4TnGV2JkzvhvcYkZnNfcXl3Jd3goc9QnFR1TA34T2eCN/KkPJgEm4FItfLLc+W3kuoOcMe4RC9Jl3g0ggv6nun8uHo4fgbd0P5QfAZTmVKPu5NrThXFJN0TQoKq+SQXDiezbP8yel9hBd+epSpWyu73L8rs8/OwhME2HmfKwDfl/bjl4d/4dYoPSG39uPYUNJtH/IkmoVblzB+ex1Lfl6CiMgYCSDPUK/pLOnIIqn4OyKqniUt41dKA76jocOPdZfXU2kIJ6ZnAWfGH0Gr1jLXqYMkYRU4hYH3YIyinLLAMsr9y3GJSWF+z3dJapKc+CpRsj87zc3d8lZZS1fkvFkwYPrbPiYg2va4DkUpD5xL5q0ZrmyZ5cmmtie58dkypmyNtXzXGr10AcdwglW98G4ZwcWRAjvn7bStEzcqx9Hf+woL5XdwNE/C1eIg0aq10pqY8QakLeZMYANhpcG4Nzjh2TyKQHkSGDtRmJvYXzyJ55rOsdYcbZNHk4tyqDrOqLo83nz3TcL/lFEhG8vZkKhuc8pfl4qjNg7BBKLFyTBE8kni6NBFxrNLPlNrecTlFkG9XKgIi+Fu4FtcU31KRARseXcD90ctI1xl4twDO4i/FcQDmx6w54x0jQefoRg7dERkZTPx0ESW/rCUQ1mJ+J8YRN+7IjvmuJFZ/RwhHmtxq3fEqc2JHTov6JTmz4XS2RTKD1Og+Y1Sz028dOl+qn2/5JvJCRwe/B6Ty8MAKArJ4YzHMwgC9HCX2I/r67y422rk9KirnBx50uYQe3XYTPaNnMDrysn416ZyOyGbssAyVHTyzeQ4zr3wKZ9tfJQTqSaedHdmniqAF5ziiHv0NYrKynl2i5k9Wh+GrX0CmUnGzaSb2PYpoI/3Nhb3ewPdV/EcmHzA7ohrykSWu4ZQhRHPWk8ee2dGd1Q5oJPXUuN8jDIu2fuiy6T6Mfo9fvOHce3+JNxbQLJyNoIAVW3hyEQDz/le4+bib1m1ahVnh5+3zeMjeY+xqv4MxzuwM9TMAqJohM46cAxDZjaR6uCIM2Np8GizzBU5ZLzG4qDROFU8w84BLQys2Girj1ymgqBpBDhk8IB3KT1VUkBJZlZ1WQukfxyacJqrszu4mSLgobIzhWRmFXKTI5i6sE27rCMveO7kcVfoddWVsIw9tmvFzQkUNiWSouggqsaXVsdWygLL7GvXgJ+QJX9ET9chuDdI0mmiIKPJGAURSwhVHiKzZhjxLSI7Rxzn7OgmikKLsO6T8xLe45Feq3jMNY1qtwOE1S7j42Hf4N42yDJH7E7WrS0wQlODv+NdfrqxmoW7aumR+ztyghEMBkSj3rYGvTx0DlNdhlHou5nC2Eh8GkYytOMTALw1xYyO+IVat/5svB7KpH1TcWuUYxK10nuPPAhT8zjrOB3hTk98y+W4NbkjmhX08e9Dm8sT3KgczyC13eMnCCIqsYWcugFsufk210oexMHshU5owSBv7i7paVkDHcxexJS9R3Tti3QtNc5HuRI1mfrWWsYeG0tCZsI/8mFNf3UEdyffZWrWo3g3TfyHjGfpWG/ODzkntWFXyUxg7IJjlCyfg9bJmrO0u4xnTmwO9/rOpFflNXb1bcOlI7Hbs02YuoEQUwL2scK3iAc230/KD0Ukla2lX94fRLvZAaGiYKbZuwfp48Op8q2yAUi7todZMGEWTNy2pN+cFPUNe++uZH/V11QYRqFXKjk+6ji1nvX2vcatFxvl/Si71YPY3MH46n8isvJF2gyNXNLCt02h1HUESRKeZqn2Ur2lG3S6ZHEmoRe7nSYAoMYVR8HLppozoyKd4Vk3UBm66NsBH5//jXDHUl4flsvZ1DzOTpuOl2IbLko3HBWN7LzPlSWKrVS57aVdkUmT6jw1RktgS1WHn2MpE8MnM+GYHbRdEC4Blmc4gZeshbNuOYgX1HhXQbtju9RmMiWdikhu1aVQampFa27uIksoZ07cx4wK245G/AjxQjoyi5SxUm5vb41JAgO7VRhwr66V9nLhn05z1/Y+eDePxVPti86o5p3TezmrnQ1AYLEHkZd+RVElKSjkNfTl/l01vKacRFp5MY6tZsoCy2y2IQondApnjpXuJT9cCkLvnL2DZdXrMcyy5127VT2yW94thdiJn1MB/fwPEOuVRpTzMQRTx79haApE50RDnZFver7ElYTnMGFdD8xsCXWi96O9OfbuMYy67u9qCwKZTXYZT0QbiCxE3UDE9d+pOn+PsNQwiuKVmEVp7ufU5SBq2xlwYys1f17AMcQTvYMLMhnoLOP4/HkIGRbC2E/Hok0ewIU+b/NK+mwbgNNkgsqMSpqu3EG0vLyj0h5VMWDAwcOB2vPZuByTcc5k5pD2bwpUJi1nzUbKc17m7vXniPS4xjdVKZxskWE2mFketp4+hdtwlfmB31gpSF4UzeCTIwnc5E7fq5IKTFjhRPjdDUEArZMXzWGJdNR3oLYoUYEdXKep0uCx9m0m75Pyrfaoeh4fH0k2VaMP6bb/WMuTI2fRs+hrwjtnoMfe7xqFxq6U8zdmn1XGE8BZ8AFjBcfGnuL8kPO4uws2UJS1H9MGpXFimhyNPgWNLvQf/fwPGU9tLbFeaTgpGwgtDOXNt9/Eo/gEaa1bOVkg+eACVWkcnx3OHn9pnXJr78eM6DnEekk2aorXZsZHfk8nLZZ7S+Myq2YIepOCmLAavn3uID+9fpqj447aAl/L+i3n4+B81E7t5PWU6qnstAT7mrJQ1EggkUdcCvj4xY8tgEPLnHLwB+8hXDO60egupTIIL7CrK8xPeAeATxTb+SnjB9s7R3teQnnnLbwciqQ+LwjD5UYVBrnR9h2pn62gFHvAzN8fFi2SWIfW4ickEVn5Iv000tpglY8fr4EOBx1bl5zi3LBzdpvBPRmiV6KInMPBBQnkRebZ38l/PAByXR2twQqCyn1xalLY+i+0Yw0PJr3WTRnkWFIQ+/soaFbftqW3sT5Lb5L8c72v92bOb3MwNP/Xwb+uRQgNoTxqmE3Nzva5FWj9/3Nm399jGP/Hn/8/+4OgoCAyMjJ49dVXefbZZ+nduzcfffQRN27cwMfH579/g/+U/5T/hSIIAi4uLrYFv5sT1aLj39URVOm2hzz/jykXrti+37XUGN0AuJJyjWNjjtnuG+6WTg/3a2hNoqTJfbcPvW/OxVNtd8BaF/ZnVk9i0AfV3A5ZKd0/43XY5YVb0w36V0hyX5fur7LXLW0RT4QPAESGnxnO4FNOtDhVU9ZajMFsBLkjBwqeY2GpK2U+1XQ4d8nzkPstJrkLx1pWcGPKSxyaloZB5dRt4fg04STj0msp6pXCsVFbuKVeZ/+9WcaVTjjpWUmdl8Sis8oRTo1ey3OJozks92L+jvkMOT9Ekv0wyxGMbXBuLkQ8TF3e1/iXO5DPLR43/o/BNB4zddBYvpV9BfPZ1+5MudsvdJa+y6pjwyjx/IUzRQvYnvcds7zzSbDaJ5Y6f3vlG1YezKCp3YsOZx/WPbWR9N7pyC0sy7KWGEpMkoFXGeJIXXCy7X0btH6o5e14Gut4cMNiHl3/CDKDDNGslL5z+Qm49wOCIOJV60VyRjJOrU4IZgVaMzxXA9PCdpMfZq+TVUIGryFECbeZH17PmVmZxKcH0PNWz25oKsEsR9WpIvxwLvHnJPaPk+BHz6KviC37EKGLkQZSANlgUnLo3hN8UvUKc/QNZEbns33qwxwYMhazYCS/oTe7c55FcAzC6+yfuFflYLwlyQfYhoFHb9a4fc3J2yGEZnsSeFNi+8kFGfiNo6AxkWKt1GaSE627QxGgXVZJk6kck9iJCQPnOqCmw4ebrX4M+uoxgjNEDizoza2kXJp09fg732N08H7e9jZR7VODQhNKXMn79BeX2vqjySGdu5qNOJfuYNWqVUTnRHOpYS+lLbEs/KOet2V/8VYdGDFyqGEdBT5fSO1SdxqyV/OG7gS+j/6OTqFj/OHx2IJ9MSup08bzsWwUY7xSMZnAta0viYU/MNXzebZuBYUo9YvOKKHmm8VC6lTX6ZRXISDQu2AzKWUbUJpdMWPmRvNRalwPYxaMbKuNZ396EIrKBlv7yJBzsXQGZS09+NQbfGTQ6ipIB7UuuT72Vw/gzMlkBpy3G/DuFZmY5S4wowgilvBkcBsMyAA/d/v9bQhHGTKDjF5HIwm5Ix32RJMKZ8N1WpucCDqeim+Rj40hpZBZnVIi7vXurFi3gvb3d5F0TU2p6WqXsdnFAaFUYRLlyGTgoqoh3uscStd2ts3fhslRyeQdLfS6PRxvLymg51S+m3WeAhqjQGVPgUMTD9klkUQ5ZkFEPfQaWYlNmA0mXBud6XvzQYKdIqRxKMqoDR3E0CNv4BTvhFOLE6JJer4oSnWLzomm/1E1iZcKmXp2OvNSRrNuHTh0KggrCKOxj4x333yXioAq/P3hjz9g7hwZgaWBLN4wCaNcIDsuu9t+cE7zAukh03GtvoOqrV5iAmnvsip1IgneZ2gbl0NtXRZNoU+QPXwpJpkl2OLZD2rO43s1hvaQUC4NuMX5Ieft488tgfb2fgTUzyN7TiWfvvApJllXhxYkFf1I/6KdOBNgmWv2zejvgUDrfOl/9yDBNY9woQOMBpGWTnm33zbrvIjzOs9TSVMwJoeQ21OFa1s/m0zOC0cvctq8lOvGJDzq/bnW5xo6hQmZYDfcS0If4wnvVr5b+iOOsmUkFK8lxDEKsj8h1nkvy27PIy8znN5XnPBojqBfQIrttydLHuODvFE4IAWGLnR4IgjwSO/nAfi5o4YsrzUodAqUOqXNSf+vIfPZECntyV3zrHbdn5eoBabW/Yu4Cz+jb1jK2abNADwz4GEm9PiOIWVqHIuDUeo6MbR/h1aolxqu3zq2J23joxYjfrVJjDm7jKTi7+haLlYO44CLBNpQdTTiOvpbql0Os/nWu/yaITE47neWULQmDDb5GU+HUsb1WM/Jggdwbl6GWheEztSBV8soxtws5d3E7/lqXgq3kwoYeXwA/a70s+1Tc37r4LWq8VyJnML5wec4OPGgffwAHXqpz4IV6WQPLuHiwIvS3itIcwpNACPi15J8QWI5ZvR4E71cYpwrzA24KFpQd7NvBIlxmvs1AAfa4ejk03z+7Oe23IvW9V5vIc2ZTBBe/RQDDP9CJdqdJtY6Ohj8GeJewNC2hUR5XOG++A+50pDEMwd6MfZwLyYcmkBQiUU2XFsNtWkU4kKRHm4mplEycjc9fc4Sp38V8jdAxqsEXvLhw1ckVptzk0WmWFRgin6WU0X3c6vNFaeSIJQ6M0qd0nK9S64mi/OxSXONR7ICbHVe6FXGw7IYPBt88Sm6irq11gYaGRm2ie+c3sCvfgQAl/tfBgTGj4dPP4UPP+wegCttjiOwfAVP9JMCyX1VJzgYCPqkO3z5zJeMlZn4izIwGcCo437tWcZOvYBaq0a+YSo5LUHkuqyF8/fzhOJ9HncBuVCA2Swh0rXmFjoN0r4kCgKCyS7LZHN2d5GcswYJjWI7uS0Z9HNoxkVVz7ORWrReM9F5RTLowiA8aiXWAEFTwX8c48QR9Lz2I6mHrWw5+4AZELQbVfM5PIuuEJMThlalpcW5RQr2Zb4HxTvxVzhTNaMWudCL4Rl/MtV5FYTN56JPHh+mv8kfrRYJPiuzT5BB9moeCzpCQeIt1M0K7vv9Pnyr/LoOVKY2HcRTth1RZ7CpUaxcCY8+CmtWjGdgznFiyz5kfM5dJl3Vk+w5EG9NMZMiv6G37C3Gz9iD1lHN7N9n49YgzSMXNwfw6EdqlQvZOol5Wu9Rj805YtTReWcteuNflIeF2Ori6NrarW4+2mlM9l7J2Sn9eP/V9wkQpclS57YEuaijOjYHL5kdKW0wyyluikct0/PUBz+w5Z0v2bRoE00ebYgi/HHnBeb93sjrlcHM+GYpfa/ESVKHlr64USFJKc/QOBKT5Yp/hS+tzq2ozbW4qS2yX94hkAqRI2YxK2w7AwdfY+kTr+PnJ7XrvyI30u45mNtDSribWIbMpEYwaeHcPKr0LvQuhtObV7Ny7Uqcmi1OmZA5mHo8wau+dUQWhiEzyhhwaUC3farB8SKXYsZyVLGSrnLT1rK9ZiKZl+KJ+GIECWe+6zLEzJwpWsDk4kBydDD4zjkmXzMwwHus7bcCIiIigkmOU0c8zto4aZ9KeB36f48w4Ac+y2ukVZPEtb6So1wuykFbjbcyi2CXLOTGOkxdgkNWJ+hw3y+YGbvahlYXUXQBekn1L4woJjb8IwbePUZfj5H8vbTJSjmc7MG+mMBun29qmsCu9BBaHMwUJU7u8j4m4rzOs8WtgCG3k1DqlFL+Jmt7BkwAbSUe+hN0qpW0OrUiIMNB3gQefbjR+RLVbWGoRSem7p/OvF+9LfeVfn+9YgKVrSG8J20B+DZP4OHEZThr47q0u/TdERalT2dlNZ1GR5anLOPxvk9R2Gs6HqU38c27YGuPrJqhlBggr8ejyNtzca+0SpOBh0M5M2M/R6vy53iElCt0wKUBtjUYpTuovXgp5X06vaRgjLrFkrdPhGqnh3FR1fBHYGeXdhJ4PTmO14bNoqK1B/UdARZlju55HQHbczRmb6IqXiOm7rluNoP1u52qTs4PPi/lCbYG+yx7XUW05HwfEP4S0RVv2IN9NklCAyZLYEvexY7PC9jFK8dPEnCynMjTGtvzuqo2eNR5oGqrQamE6dPh88/hiy/sY8xkNtqC2Q29L/Fm6lTeCr6LR3AVnY4yXMuykHe2dtt7xI5Wep7+hqB8d4bcvcMc0+//aA8pn6mRq51QIsZzsXQGB3KfZE/DMDZffIzEtLN0qjppdpOC5LQVwfkFGAUZ5sIgojP1KHTStcuNktrAky57GTsoldMtktykNUiUmiowZw4k97JL1cE/fStdnbsGsYUmQ7UtH3Bu7TSa1ZNxcP4VB1Myrp0JKOVyW36vKZpmxpwezpStOYzedYw/2p9HEKC3n5Sv0uA7CYcWyX5b/fzPlAeW06PiJd69cx879Zl0yKMpjRlDdpLa0k8yMGpxMOQS4FhiU1oRzAocRGdkJg2/Z73My+lPs3v4Kf5afIrWzifJ8ZiOXCYSVf46YVUr8DMMwL0hmIG/NxBzUQIV1Miv8/di7hIANJoVXCmfQqr6dw51hLBg2xjcago45zTb9p1mnRelMi/6nB1KcKkf2xZss0slthbilP4vXnaHq/2uYhJMuDa5cjtkBfeay6gbb2b6dj0vjw1hzUYF18MXcDo+kX3Cdg7mLmVQ8F88NeBRnk4ci8pUgT1oaz/rTzowiYC/7CB4mWi1S6VOjMiP4OrHVylJK+n2nl1lPBO8ExgSOBy1PsAGmFUatHiUZ9JSIM25TqHZ9lutQYtMIUNmMmLSm2itaEHUdyKTQXm5/Rm1OXUc/ddROp2U1PhIa8nHl9/kYmggsUff5vvk72n4egcqS6BHo9DwWr/VJBR/iWjUUFlZSfqprzhsMv43FagOmfT8ceAL7o97kJ+q+5K4ejlMh+bbxWAySbZj1BMw9ixi9WauDU/A5CuiVWtxlxmRGevALZETpfu4Gf4q6cPMPHD+Iar9ZGjllZY2l9qlVSXZGEaFXaHHKqMoBdf/6Vfr4z2EsJrl+OoGUW64Zf+dqCDRN5HHAr4hsuJlidlnUQawBvtePvYyabr1+FT7MO7wKPwq/f4tc8+3ypdRewxMPPMVIQ2L/3Hdta0vj5TpGXHbomhTeYT3UwfzRN+n+Di0HdEsEnsjC9/luXx0XmK4R6ok8Ht9FzWArmvF3tKPaTSZ6HCYIX1guSYTDCyJ2sFUR4tNadsPpC9UtvawvD/c7mfiqye/ot3RkgtN5kCjx2AWVsKVZiWPrX+KIeeH2NQ86KgEt578S5vEFd8y8sPzaXNsw3oe+/TCdpaf+5rr9m1Ksk8s89pB0YKAnMCyQDwuVuLaZHFaWtZjtS6E6LJ3GO3wHC3qbI5H9WL85vHcd5+UT7BJfYvbwU9xRr+6eyfH/4stkb/w6KlIJv+VQGL+PMKqVhLoZLFP730PDv7UdfbAt6SZ4aeHIzNY1gmnCFgotdHttJ44tGvI6lNn6z+/zm0ScaILG9wk6DCLBgQEW7DPajPpjZKd61/hT8/Mnph0diSB6sZFoi5uQqbX4nfvHP32vU1AjhTcFcNDKY8Zhdmpezq3/wT7pPL3GMb/6fK/FGKUy+U88MADfPLJJ3zzzTc8+uijODg4/O+u23/Kf4qtGI1Gbt26ZcsXKRnaVoq59bMum5hlczWaDeT5rua635Pcbb5hu95ifpmvN7xCr+yFXE25arvX5KiveTN1GilqAadWJxJue9In+2kSPe2Ozq4IO5AQb1L+MH9wTSS48Hu+XXCBb57cRWUPaXOXCTIIu5+ctmmcG/g5Aw0aXBplnI3tRdKGUIpxgknpnCldYqtLmedWCk0SG4l2SZakRhuOyazkgdJ8xl0vxU1lDxK4Kj1RGj2pjE6kKExCgtoOtmYZG32hMVBJQmY8Pe71sOUqmBbzBdGuZ3D0qaNw4GX6Xu9Lr4xekga/TAFxL4D/BCgpQW4w06JopbvL5r8uvoCLIpuDlQN4omYfysZDJKbJEPI70MmlDSmrcQIrrj9FhiWWZt3Uq9oiKBfK0ckfIPHAJ8TcsQQMZCJgprffYQYrGwgoCyAmvQalhWkGUN4SxQN/VOGvL2GUXytBpcHM+mMWglmBgFnaPPM38oKbM35uGrY+uo9Gb3cctT3QmmFNI2y+u5i9uT6sWrWKCQcn2F9K7kClEEaFYKTRs4VOtQ7BLKBW2NFU8dpHCWx8Dp2TkooAPefE93AUPAmreZKQ2kcRbQymf7J9MEyjLusbQstX0KbOodUhC4XFsSIKRmRXH8F/liPXU0eT/25/WhwsBljBFrj1Dt7mSvZO2wNAyh+BDLg4AHexBfX5ETR0+OOn7GCRszXfQ3fJS+nf9oOkyWxknjNk1/dibdEgLve/TGm8gZthK7gSMZ/S1gJKm2PYcfdRlteZ2LRoJ/KyIpKPfNJtU69038Vp94eISZfYt2aZEcFQhIPccggwi6QNTmPz/dsIKQ7BtUkyGGSVe+HGC8Sbq+kZWM/WhVvZP3m/fc7Fv0SnbhAvRpxghFs/zGZw1EUQWvsY/V2n4ewsIbQBdCbJgrvs8DZ7/ftS4mVHgtvyT1gCEK4izI9Zx9mmMFx+XEjiAZHkG8l41HkgE+VUtvbgy8vf8VKOL7EnR1MeIrJp0SZEi/RLvPdZTtUl80tkJifGlqB8YSWNk/tQHmRCZ5L01IXQ+XjlwdkaB+r2XELWchmtsgyxaBOL3cI4MeZ+tt95iF4no4nJyGdgzkl6Ff0EM0rY0HSO26nL6TN4EkPO9MSvws9m4E+N+IVktRmjzAglDQw5P+RvLDcR72pvBh5woSLFk7ODd3Oufhe9/Y7y8dhh7OpxlT4pOWj9lBiUjphEOe+/D1x/Hs/0pxmRE88LH7yIZ45dCkcUTVB9lvz4D9isvI+7vXvj3jcCfWAojeEr8dFIK4YoAoIIosidj+7wwuoX0LQ72Jl9yPGt8iUiS015zDhmrV3Kp+M+BcCvzIWHNj6EQ7ZV/lF6J7nc4nwxCyg7FeQNcObCkAvd9oNWsZS8iPOcmuhMk8M1cp3eoaI+jb4Bh3h56FwCFWa8Kl0w6A5wpcdMCr2/to9fz/5sMRr49ZGlHB6/jxaXFkmyBjO05PFTv8f5QJ7IzFuL8a/w72bMCQL4Nc4goOE+VLhYPuty3YrQ7opiF8CneQLR5W/xbj088/aDPPPu6zz7+bPIdVKww8OhjCTfE8R5HMNRV80j3/Ri/h8fkuQlyRUaTErqNd8xPjSD4JJAPBsi8WodibOQD9FPwfjLaNV+nNGX0OuqJ0NOOBBevRIfh0BI3ccf5b8CIpkJmVwc1oJRbmFE1F2Ba89Q2xHGtSvHWeF4nX4H/2B7w26JbV8+mY0Fk/miM5dm9R0aJs7HUfMrvg4S+OVW9QhKO51wNKkxWw5gf5fxBEgZexmA6GwVpZ2S03HP3afZkPEZGToFH730ETvm7uDsgLXoZPX2kW3p8+BiN9zLMm2fDw7axaoRE6jpCOA9n3w+eetrdj3kx5am5RR5f0dzpze/Z73MMyf2kVm3nYSSNdwKXcqBvkry/D5BLkrM66KmnlxXbeRwH1d+qpHkFB/vu5IRgcfYRhsXB1+i3L8avUJvk/4eFrKdneErGOR7CE2Nlg6HDts7h7jeJq9eYnz/vvkBJv7cC4cOBztbtv4a1N9guP9YlDpHKn0ryY78FZMoHW6T8305MHE8x7r4oQVBsNgLcFH+POc6wL3OhcUbF5OQaWcOAJQ67SYr6F8UKg7Zxt6/KyYT5NQOpMjhaTr0Ljx76Bqr7y3n0oBLXBx0h37X+lkYKoI0RtJfZh/hdO4ZzcCTenY7fU7v809zRpMOiW9Db2lOa5zbOb/CgWMTLNnIBBm3FA/z5cWfmJsznO+Wfsf5URqitCv51+B/oZQp/+GsN4rtNBgq8L4dxsqD6Twadpy5hUmEtI3h+2XH2Dl+AlcqpHwfzw1axJyAB7g78EH2LoqzoWcFAWJjQa22B/vaVHm0qnKo8lxJQUMvnhv0ACCw/EwPem+ah2+lL5UGBVoxAK4/CztUDNNlMVoDV1KukJZ8g3lp71LluBgarhPCZb73hQBFHVpjBzmBb7Ik14XXT7xua3urzJ0UZJbWhNkzZXz1FRR6f8PViDnck/9l26cuWkAAKlM5MaOKaA4KZPyR8fh1Daq59+KSbiitiiqCC2USIh0BHx94YdBCKtpD+dnxYcJvHQBg68KtdGg6kMlUMDkbpt7joks9pC8mODcdRRentATQsNjhmPBrnMHjYZ8xIXICRC7F3aeBhKW7yJhcxs77dlJlcdpZi/U+1bFqMmMvcLTlSxwdJYd5fHAAXi2jcOlIkgKeyJHJBEJcM1mW8iRXjS/y9pXVqFurSbydiMoq5RQ2HyZcIdugYtzhcZSEVFuYbJaBc2oCqutPcy9nMXsmfc+Pj/7IB698QErINUomF5MZW8VH53Zyf4+HCdd9iaMukRlO3+Ge+DIM24VJdGRk2BZ2NbmiM9ttBp1ZQVaNpLLwdSM0W7wbVjS8waSiXe+KEZGMXhncicu3XJfqtTvnOZ48Wsz2/G8pTpzM3qnSnJRbgBKni+YTprvHByYPBp7zIyjnBHJduzQfai/CvR+Y1z6akTPySRy3nkfqavFvnI1M0EHxTlzlDaRrIfpuNJ71nvbJLndEEBUEKgwURd3j8KxzfL/0++4sWrPdwR9b+hGJRd8S4Bhsu365cgefHvsQgKogb6qM2QgCPJz8Ihumh1CqV+KVH8bAQxdxqb5ne/TLQ+bwgWdfsvuuZfJ1PSMyMxmWfQ1/xQUp0NwhyUqIRj3hGXuIzwyytIkcBvzIzqrdfDM5Ad+OrTZbGkBu7ICdLrQbPXkhfSVpbZutI84++CzjVqETEe9k4tDcXWK20nUP6WEPkee6Hr28Ab2svtv6+Gf7aH45kUT4XWf2DZ/FjsYXAHhj+FTeTJ3GR22BXEg9w67Zuyxj3f5s88VHUNc/yf5plzg+oQSHllhiHbbB1RVUmkciF3VsVn1MXFYMWjWSI9RS3yvlU3jowE2OWkgegmjuriwjSHNSLcBoix9/VuRbeGlKGBryGwHOOeiUMjKSCzjb+3nuNEnnYq3BiWR1OYEyuNnPSNawx6V7CZDX0BuAoKrdXEtqpiGigCpfCSxnldOn7ipeN0czYPBpDk9t43r89xjEVlsesL13n+KNGrufSEDgXoskx/jC4AcYNGgwB2XL7Qy4f2Mnmf4LCTLrdc86T3rf6I1rk+s/2BYnH7vEW+a3EEOkcWsFgMwLeY6kgp/wNw7i7Ggzk68a0Qhetnu3uM0i714cDtcvEHjbAkoRxG5r31PrnqLXyR9Qtkq2SFQUBAR0BSSbbDmruuJkb004RkWiIxFXf0PTXGVbD+ucTrPfcy73YvRoPcLQ6MJwEQJs/Rxb9j73Kdfj2NmD8OqnWCMbjTstHM2XJN2fDl7EiuDdnBtykRZ3N1zb+0r3NumhvYQ+nj2QD+lPk4czIbeldf+tuJ18Gz+Yx1xhbtAZzrf9gtkMni2pLIx+ghn9+7J4Mbac49Z+OiG+yj6W0a4stI0/kGyGPL9PmHfVlzuBrzAp8hsKm/rw/bWvUHQ041lyA2WrlEe10+jIFxd/ZmebB+p2adDeTrhtWxvzGvqw/vpqTC49yet7H1l94xh8Pg6nFicC6+8nKHcneiGcy7XraFPvo09aM76VvhJDsyGDfhXRbJ4wgjVeUgAhuuINfo1rJr70E0pbYjlaOZQqdQeyDjNDzvbAoaVYykdb/i49S9Yxtn0DE9OknJ1lwR2cM66hSLXvH+Owwn0XRV7fU9FRZPvMgJLWZkd0Ch075u4gNzYPQYAQl0we7fMMCeYqbvW5wYFJpyxgLEsDXn8OdeEvLHeF/B75XO5/mWl7p7HqjgO+1x9BXfoDn49Pob9rCT4yAwZZMy2a2zzvnUf/wD2252+ti+LVsgnUuBzpNldAYN+UfbSG6Vn+9XKCi4MRRcEiTysgmEUuDJYkxNevWM+atDW2e87vOZ/y58rZPGsz74x8h31zTuPXOB2TKJ1tL4Vt5+rkNwiZO4C/Fv/F8K2dtoBLh74DmaOKC30X4TyiN3tSPycw5wQyGajseGmOvijlLGy3xMXUMgdkgohO2UqrMyQ/nIxuxjg6VdK5XiVT8Xjic4RXP4XMpGH9jz8yXFT8DylQDZMpeGTTDPRGOXdi7yBoBfJe24Bo7EAQ7BPW35SC2WUO3hv6U9XDmUTBB3XbVQiew9XqcxT4raHM/TwlvuUc6RfNhVhJpkA0qVDp/HD1daXpmTcpjR9nu6dSCR3KYq55vsB3We/a+sZausogput3dvlcIMwtjIley/BvnCUBndv70CdvJ++kvgfAuWKJqVzrVcvJEae6pTYA+3mp2aWZer9I9GqXbnuJTW1CMINJbgfLu/dmU+ZndBocmRFZQ8GAK/Z6We5/u3MpL5zezhPV0pyrd7zArpwd5NZJYBGdyYkQTTUhXdJm+vjA4OBdlLX78X2Vkl5Xw0g5Ho5PlY+tIX649iUz7vahWi/g2CSg1HujxEI0cgqntv9GTGb4zVOFR70Dzi3OUsBOBM7Ph4O9eXvQF/jpfuHXRb9SHFJss9FuVI7jZn3v7gPELFDRGonJK5V2kw+CWcbVfhJ4uv8FK4BXakcHfSDRFW8wTP0kRrGNJoebZNVk2W7Vriik0Hcdl8zf0Ki5SrXxnu2aSeaIutmZiHwvUs4GMzD9OaLdJQAPJX9SU7SHXNN39EorYdTJUYgmWbe+Khx6hNXqam4OuEaLizW3rMBt70M8e/gqQ/VvU/NMG7FlH9r2IhAxGLqPBTNmfB19OTjxIG+/+TYKT3sHiS2NODaWgdlMp0byPQfknpHq/08xHKlt/n8a3Pt7+XsM4/90kf/3v9K9/Prrr//N64sWLfpfrsx/yn/Kf6t0nSRdJTRMZj0IkgFuu24N9mGgym0/9c7nKGsfDUgLeZs5kerbtYQ1SsE4m8xY/hLGR67nbIiBh78bQ21AKNVhKbaDgfRd6Tkfvvo9ncoKZEYnqT7RT0L0kziV/MXWK4UMvvsY5UnLwNmyaUY+yukd4xnhcZSmhBw8L/UhITOa9N5XbfrwgwO2Mcyvhefr4Z7/BygMN4FhMHADPxp+4EyiG0POLCRgvy/6hIkIQv9/tJPZ3N3JAODUGcMiF/hs+VPc1+REpW8lTQOkZ7516hA+PiLvuUfhOvA8vVXtlAaV0rNeDjI19PqQhoLtCEnXOWjWYjpnorLjf6zPqgBXB+jjeYMtwKjjfelUG8mOyyasXiTEJZNwl0zuZqwiOMSbWnd78mgHeTObxk7j2D1Ptme54FvliVuDGzKZgCiYeGekFIDLzV1K7M0GYC1HBghE8CZTo9cxO+5j3PVVzHpyF9vbTRQHlCGah0ptMrsOai/ydPl02rbeR1J8EhkRMzFjZuK8cnzFOH74KZo7Xgd4Ehh4aSB+s6WDFBVHqSeQrU2ZhLi0sWPRaapcu0uq9uxYjkslZC66zG6fAbiZw0jlddv16NbHcK+bRKXXVppUWQgIOCoa+GzcQApMC2g/PQDfwsuMc3iRI2OPIhsqMCp8IzNiPkcsuInHaDhQ5ozB2EK49zMIwhq4uw7qLvEY8Gi8wLOBTbiXuRJcEoxcAMHUTpvOjdONAZQZSjELRkSZPWAjCJJB2TX5uxEDn3sDHOWaz1nO9Nby/u2BBJZ6U+lbjyiKtOnd+eHWa5xIWo+IkfbAADrkIu5CBaJoyRVhuWfasBaqPI7jnJDHtWD4veV97tYNoKc5g6XOoaSnRzHxr6H8NUNKPizErISQOSw5s46tt3Yh86mm3bGdxEI7M+Za4yMczJjI7CU9uumCW40POdJpQWeSHOSmv+U4MQk69IIOg9HB5jDylMGSpHcxVfbh6ylFJJZOYMbuGfw540/kKdK8ya4ewySP82iObyZvQDTxLl+QHBZGM/D0gCW0CFo2VFRgdqhnzLmL6POPcnD5OhYa0iD/DoJLPLIKH1Z+u5w2LtOUeopLsfkg+wwt3jiL1XgGtqLqGclNp8F4tYRLL6SBFhG0zqBrymHQhSQqfO+hlCkQBSMr+77ISkBY/i2vChf5vnZSN+amT9MEvDudcG3UUqvL5V7A+1yoXURo/Sv8kv4RDyW/zCQN7AkXuDk5CMFciKNjJKh9MLnEczz4MUoiwPd2I3PvzOXiKBly0QDHhjMwdCETzFtIrwdd00bMogyj0sG2fjZRzB3fN/npoDszYwdzqf8ljHKzjdmnMLiRN7KYpqhOQkr74jfAvqbXe7ezf9J+4p1mEJEXQUWg3XEsE2WUBpfy/fKd9Gl9DaBbgNMK0PAr7yQ+vYVTg77ipvZ9rBAOWdFGVHuVDK9qo8NYRVmPG9I6cXwMNN/h8WoTZqMRhw5H9AoTICAKetgbSaATqMVmlL8bGBUwjuOLK/h3ZRyr8c56gyH9Q7Ga+HZnkJHDyR6AQF1HDuAloY8F6IjOh/xwWp1awSKT+MyAh4n2vIR3TgjjclbjaYhHp3GzrfeTor7GzxBIvXIGt8Y8j2jUE1/igrfwI9xdC4FTkMk1hNd6M+rkMCCDwuQZttyz7aY6Xok4ytECZ2LOOnNl8Dq+vubCCwHukPMlPo6jyaInbkII/o0h9FBACXC2eD5lLWDssZ8x+2Nw70ijLvVplJb+N5llDHetYu6pIu46pVPuvIe/Y86ebNzBb8IGtM+tRjALjGElIEleykQD8/pd4uRFPdnxkgNXrBXBqIMTo5jakMkkuQLvKmfqA+yBLU9NGfFe53CQLQGsLCQ70nl6zOdMivqG108cx6UzFGdtV5S2SFVbBEv2FNGuuU5JxMvd6quRt5Df3IPHq/NQezXw65I/0SoqUMil9/J2LCa304tyfSOTjo8kIymD676CBbF+lEf7PEe+Ux/ORp0h+Px8Jp78jPoQSSaU8wuh5S4xgNfiX/jUU2KJWwPZdW6P4F7/I4fau0qeCWBhJtyVz8PIatTtSlyaXWzMPmup0hynwGcdTq3g6KShQnDgiWEp7NwpOSuvOXzIXd8tOJg9iW7vTWjHlwwNMfHD9bV4BBwnLrSGK8o8Gv0H0aleKLGL3ZJg4AZk+dfpUSYnoMAVl+aHOTPRAUQFuMZBxmuYEPA/501ExQ1kUQKopHlsMknvElq9jDFzJnKubgh92+L4xEIG6rCMcUdtJBpHA6IFzOEiE1g3MZnsmkGc2zUYZ/NJmvtW0aGutvXlsn3Z9O0n3cCOorbv28fyj7Fe+TSq8N7UOZ/ml5BSPOsduFR2ji8m9GV/6+NcLnNjanYczi3jGLfwKBNMvuC6CkIXcsxjOhveyMaj/jqHJuynZ4lln5qSzdHdhay9OYoGwRWDWgfYJeGkPrXbrWbBYHNsh4XIcHWFRscrVHrsor49BZkg7QUz672oeL4Cr+0RzIi5Qlrtr/y05CcaPSwGWt0VaMzkyapiri15ixXrXmBQ2iDyRgu88w7UHnDG0VPOilPzGTJ2LMNP92PBtgX88tAvEnDFNdZWN53ahfzkmXQ4+0hjsykL//Y0fkr9hAwZPFphwqtlJHODRjIsFNC3cLBlFflZ3qREJLDOeQRilxy51qJqraPBvZnLCbuIt1JN6e4ksO7logh36/rz6vETlPT6jc4776LoDOSdN37AkSD7D+pvsMKpA921fpQG1nFuiGVmCIDvaDrqb/JBbT1m0UStVy06ldU5ImJS+HCzahQrBj7K7U5vBOaSIl/CyOIAaBuIV73kSF1aHkKz/BYqK1vaLLAp432ON3nT2HKRAdnBtFHC3X5NCAK4qyuYFPUtP0Tf5AFXPX0/eJf4nOkM/mYG2yzVrun0RmkuRegsZMG2Ofw5+zg4hrFkdyEdBmfejhqFOiMZXY0DRycUURfwNYLwJtx6GyoOoZP3p0PvjHnjJjSu/hA4BpPoDPMNbP5XNq6uvxNSEkJmfCbtThY0vK4JYp4maddJql1v49PLB6ea6YSqk7C7qayOIBNB9Q8A4GUnhOAgOlMXMY0bKbXcDH+EeO1kprGPwsYkzhfPRmdOZ/CFAbg0tqIru4UgSHl61HKJ8aM2FnQbE41GS97Qiw+B2o825/XUhMVgtsiUWVHozYZQ1t99kDOem7nhbQdzioIMvAaRVTiBfW7raGvNIaxqJbGqUQgethEGwLzt09EU/45nxCAEwe6EbdZkUOq1EV3rGFsbdB2T4Yr+nOpRw66ICzQ75dFmqkUJZNUMo1Xngbn/YNpOVNErIxvRJCI4Wfa53eEIxjYeK7tDmcc9Jl3T0dLSTHPccQBiFD/T138YfQzvcVW9gjtJlvWhW6DSXpFGzVXOlmrplCegMvgiCKDp7IGroAIkR/ix4hUsTHwLgOGtx+gpm4s2zMiduGu2oM254vvI9btKlf4YC/+Qc6vnCnY7tdKHAxhMKnbnPM30mC8B+GvJRooM0LNojlSJc3OhNQ+lWcSpsw+tisPcCb1FSONLCIITEY3PQmgp45o1ILRb+khkU94W7pTH8kDSm7i6pXGjJg0XY4Slj+3v6908liHZFwkPUVDkkI5B6YAgxNiuW1Nr6BV6qnyraNe049cwS2o3q7Sa2UinoROt0YgZFaLFkT3EZyKn68DNx3r+EelCWsWtcDqT5M4UEUX9jBTiSz6nT+8hiCLITBpS88+gDLiArlxLtudTTNpSz49Tf8RTGdjFR2HEr3k8a0ILmLBLiVofwJXIaVS57WV8sIo9009zN3I10xuOAgPoVFSR73SIWDc/+l/6g4YR/oA931tAwzwGquCkAQLrF6JpKkGQ19JpcOCFwQuId0wnwwQNnkam7ZlBYa+pCLMA50hI3UNy2mKe9ZvALrciDNwmKygbf4dPWBAxj9Yrtxlb1owZM2YzBDYs5L2BCwkP7zK3sNtHt9hMMyUMlT+CRhfGPp+htHhXMrJiX5dcXzLmxH9Ioy6IS0cSCc/YDcClESZ0plcANccLHiYn5CtyJx7i6gBHKj120VOQ8g+WNseRELiTZ3Ymc6SPB4/+OIv4xkFSwFkh4qqqJlrYzP08z6PhSbQVzCeu7CsSgsLBwZty1+fRl+zhVEeujb1pbcsQ19usHvwCnzQpOOfcwq8P/kqVbxfZU6TAcIeLLxmjPDmd8BY+xoGAK38vuf7v0eqQxb3mvbioNKyfGk66YR7Py33QL8uj3CMbhUkCsgY450rzSQvXEyH3dACpp1O528uyT4UtoFPhxtATG5j5x0x63ewFQFSHE+71B6D+DD3c4dk7U/lCthdvS3+sbXBgducQQlwzOV20kJdlhyhXnsdPLknL2hk+cC/qHv3VWtwzBds86prjq82pDe0KLYfKDtGzoqftdxqFBo3CvgFYXXNaZRkAWcrNRAibMJtBrpEjiu7ElX5MdvCLaI1am1SoSaagx8L+3MsJxeVvwT73aC9Kzxfjf+ws0d7RNCa3oJApaHZt5vB0PV9+M53Tq6oxWqQUuwLBTCbYvXUrzxi1/+ijf1ce1ut4MTOTAw/d5EyiHPmDM9n01yYup7xLilgCt3+Gm6+zpr/I4/mZ9DINYnRhGjJ9LfXzZ+MR0BtZziZ8qnxY8K0rue05iP6i7dztqItg7M0K9r4Pzz0Hd/3focHpPGHVK1EoptApr+KOx2ryCv5pH5W3F1LrXICTGMA01efk1hQxJKrnP75nNoNa709Aw32Mt6TBswbr9Ao9I0+NwLXJhbKp9v7XKDRcnF3Fj3Ou4FF5lfKYcd32uEW9FiErH8rZ2+GYJawqES5pcPklylpfYefN53nHL4+aKW3ciHYmM/RFxln20CZjD9Jr1GgDQMBMge+XPLx/J+smriPKM4pWgx9Tzn/IcX8JJONFDIsWQfrmIrw1jXS0q5lwXBrzrUIvZBOlele29uBSqwc6Rxjzp0CLx4tUTrCzEZ2qjrLdH6pLFVwFG3MPgNB5UH6QQY2/MuxGMuO3r2L90jMSCUAAjaKJn8dOZ6MW1unC+DEhn7XrzNyUdaIfOICq066IZjmd6k5qBniSpyoEYPYsEXmmpJRy5YqFpWk5X3bNfW21oxrI51x8Cq2N03iV3ZC/kdGlX/DRwHR+8nXnkR/BXe0BhEk/G7GPvBMPIravQd25kLSBaRhlUvoC7n4DV5/EpceTbPEsIynQH7+6kdQ7yQjpFUtE88tsmvkjq2434CB3QzRjY5kLZtFmW1vHSohrCJUvVJLwTQJZNVndQDctgyeQHSL5XRsCErjmG43MwgQ0XLxCdFo2eu9ZCIJdytP68/9LMa7/T5X/W4E++F9g9j399NPd/ixfvpyHHnqIxx9/nGeeeeb/hSr+p/yn/LN0ZfYZBauMZ1dNe2uiVINdKrGLNEd/9Zs89+bbtLuF8+xX37LYQ9JnzqkbxM7MV1lSKcmPmUQBs0zRLdjnrQ6in8N9eLaMkJ5lFrsZhgTP4GTeUuLPfMfYdWHd6tZqCubJM5tpCZxPcaQndR6Sw1qpq4U7XzAh/Aue8Gi3y3JZ/9Y1sLTkCT6QK2yygFpnb7qWXeWfczPkCerV1y0/tR9OVXofvmwAbe9bFIYWcmTcEdsmVN4STXVHJDEufTApgzk18hSlwaW2wxBmI+6XHiTEfylm3QkCneL5sWuD/DfKrwqYMRj6eV/lCy84k3qdyyOqaHdsx0kj44ERf/Bw1Dy8HUtszjdr4PWt1MmoZHr2u1eSkZzPwIt9CC8Itx3satqCaZKl4KT8nEYPJaWBpZTLJCSazqjGUdlIkTqW2RWwe+opXHsvxrNlBIIogMoDvIeQE3yEhvjeGN29EA06BATe6Pcky9s/YWH4Llw1kyiY105uynx6qywH26rj9OMwkSUPcP/q0Qw8IznEusoaWYvRBnfpDm+Jbn+IqIrXcNZLOWysqF+DSckwt7dZOuFlOlxdMMqMiGYpGbxc1KGWt/KmbBSLq8C7yp2wgjB7LpoBP0HAJADqq92p6Cdy+LE77J6+mxbRG2HqXT65sINH7oyTcqR0kfFctaKcDc+/Q6jrLbCxDk2YzEb+aAWtQU2tXkN6J7iXCjy4aRqR9yJRKrqjZ1NPD6HcPZfvH3qQXYaHbe9rRcBmJ7aS3jsdjaU5PDVlvDpsNtPM7/Bmu57e9+5j75S9lAVWS3POMRS8BmISFSzYtoAnv36KkY4rcNYmSHPu7H0M9P6AG+3OVOkabfmMalyOUCtPl/qF7sw+O5tRWg+OJ4Wwp6cz1WRisqwlpQZYefgCsQ51zBuTTu4AE+lD++NjXotCVDEz9lN2zHHBL6KR7CGPounwYuiZROJFSWpvU8b7uMu1rPcPwbsuAdPlq3hXS5KH3o4tcPFhuPcdEWoTtT7VdAzx4EbvG1I7htzHQdMV5uxs46TLJ3isXECLV7h98FSf4b7ABcRpjuAY7ce3y7aQE5ODv48Ss1ng1ZP7uS/9SWJLP8Dkqqbesx4BkU8+kdgrrz2cgrvxWSqnPoVgMBJYGohcJqe0JZZd2S8yLmMkWc+8jNvhMi5HT+JK1BTpufEvIU7JZPDIp6mKfxqTYKZHXg+pzoIM+n4JJh3eF76j3763cU0KoaS3E02KA3SYJAanTmzEqTaDmNVuKP2VHJx0EJ3SbGP2RVQ/w56JhSQ0f4BZQnTYXlt0CuROz048ylxZtGkR42pX2a/ZUGlG4o82kXoqtRuzT0CGXC+nxQWOTMii0a0RncyFpw7eYMWBm8huv0bgBAmhNv7IeESrlI5nf/AdicqgYtGvi3jx0xcYcWoEkuyHCFHL+TnrJ37Nfhe5tysB5T5Mbt1lf64A1S4HKXXdidrsjlt7P9yVXSShbaAUvcQekHdhqZkVrHCD9Q+e5ef3VvHj4z9iiWtwpng+u3OexYQBj3p3mt1cqAntZ1vvH05+kUH6mcRoXyU48zBJx79AMBmoEKZI71S0He877/GnQUImXh+aQJ3TGTpNHaBvxllRycPBaTwRN4rcnsFc7/ktH55/B0LmwIxSErxOsnZSLE2GO+xdIPC0t12K88Nea/nOB+7GVWFycETdXG17p9OF97Pm0haaOz1R64IIql1MaOckupYaYz41rodocWmh2bW5G8Okr/8BfmIDS/MGE51jCeaZBcAEzTk4GBp50NVMjXcFWeHfcS1iLiCxCub+3kqk2x12N0fw5JeLiLjTZhkbIqMjfiHAOY+vJvVE5O/rhLS2zYj9nOPT+xHodsw6qAD44tIvLDm+G+dmZx7+4T76X5TyBErgDzOLe71KmcGdKwYjTS5NhBeEYw08pJXO5NC9x5Gb9XTES1K9AeU+OOoipPZMeBVinqFQM4rC0iKm7JuCe727LZBdFvAto/dcs0m6SWNOhJRvYWY5MzrnSHkbc8JRGBTk9cijl1d/+vWTHFhBgdJ92sRy0mJT2aoaTlgYbNgA69ZBh7ySFodMqjVnuOz9JXuNE7lUNg03dRVLQzdyOkjKpemle4qg5qck0JJjMPiOYqW7jF4PTuFeTCduHRpuDfmM2M7XwGSE0PlccplDypUUJvwVy4wbe5hwvZlkj1giSp5mab+VPBt2ijGVEfiXdVcNsY6zvvm/8ZzDLdzaBgLwkncjAHtyniE/dgZN/cfgUeOMSqtCQOCZZ2Di/Fg+0T1CqWIC4367w4w/Z3RDOrfqWqkVs2hXFQBmbnZCjTqF/IZkZu7QcqpjIdf6XuPQxIv4lpbQU7iCmmoInQ9DtjAmbi6xhcmkXO3DNJWc9HEP0adKGg96RRhczudB+Q1UZle7M8BiZwkCBNYvYJDDYmRGR5u0r1yUdwsEmjDa9qkFGh0UbmVN51Te2emH+va3lISUoNVYnFy1F+HWmwTLpWftn3aDHfN2IAgCgYHQ67HvKevxML1VcLX/TXJicnDQOiCaLCOsowIaMnDr+JiwW7tR1e3mUF8ftjevhMrjRDc8SqhzIaJAF7SwpSicuVf3HLJ9wbhnWudwdztos3s0ue6TiDthwK/Sr1tfFDblU+j9LRVuf5IRtJzrEfMpay+gRefJreqRiIKAf70nbg0aTDKT3SlRewnOzuRN11Yu/2sdtd4d9L8yErkVlNfzNU4lb+Juq5JlX03mlY9eYdWqVXzTEE7gviBis/1Ry9vomT2CQfd2UO2634Jm7wtlexAwcqFkpl1KTzAiAN9Hn2HTLB8+7fs+A0+F0edcGMPODcOhXY0owsTI75jf813cZQaMQLubEyZVCG4KCZV+X/wH/GvgKE76RHEz4i20LrEklfxBmEcYKYH7eHfkWK44jOGlgDvsmbqHa722UeeaJtUt7l8weCsfGFZy+HQT5Ocjtkp7rigTQJTxYMjbNEbC3jm/c3HgRXtfFG1H2BfJQMcOBJPAgIrZ9M/+kr7OU+1zrotN+Pd5CLA0fDB7Fwgku921XJMunix8kIyq0WRH5tE27jSXBmZzZPR27jVLudBXnT7It01mCtw+6zYuWs32vEEUbuH9wN9RddYSXCydfaSckCcJdkjj64LxbKq/TL36Gg6doQQ6hSAonWHUYS7WPYGzJVeUf8NsInQzbPV27IxgXHoNSpm0/+wZvZxzNXv/8c5WmXTB3N2+X+2VyrXpnzDycn+Wfr/Udh79PftlPr+4icUJy4gqW0pcdhwe9R72HIcefWjtsYw0rd0hCFCmGw6CSIRsJxWtkWT6rsH17ce4k2SVo5TuPyVqHYfmubLLgqO75f88s3aPptb5pK1f3Nr70bPha4aWwMTyYVyqWsDF0hn8nPUknWYYfqY/fa5LrAErSHF6zBf0UpXTbDaTFZdPvUctRkGHIIBS1sHenKfRqf3589gAFr/xJlF3o7BJcEc+DomreFq1lH2Xqpn9x2wEs2A7mzjocwhzu9lNQcQ6/jqNGmrb/fmiUfrUbJM0tw8wpcEL97YB6M1azib05lzoFKKj7Xey2ggdDh0Y5AZUehUO+iDLU6RnjvypPx8pPuLnm0lcj5jfjZ0MdMlR172Ete9hwphjXJ1qINBtEhFVzxLv2p/iYunezvXDWHzlRa5OfoPMHrs5eO8gWoO2m8ynyWxEbnLEWxGGWh+AStaOg2Ai+UYy/rdbyY7Po9W5xXa2tbrm6rxaqQkO4E7wJ1wXfvhn5SxlV/ZL1DnOpX/gXgKc7pHZ1oeFlVbr+e8vJYBZj0lrID3FwLYFmzBaGJjEPMXrrRulHIBme3t0Y/tYU1BYgsTmv8lDtsgLaFfnoaPdLtVulvP26QPcaw8gtsfL6BQmjo0+xvneX6A36dAomnhu4IPM1EiGzL+TeF8Uv4aFmjZJtaFRYpW46Qei6Yzg2YGLSZE9z4ZWT5RXemGUO+JknkOg2BccQyjx+oxHj5xiU4udkWi97wOJbxDtUoT/zQQW/DQNuUFOh0bb7Z1FEfRqZ8qjNdR7SpLQf9/LutZbFESMJjlZNcMw0IfeJetJzH2F6JxonFolR/iNynG8f/YPLsl7UKCHPhmjGXZuFD2Lv5KeHXIfuj5fUmyQ5DQB1j6/mldGHaAo6hXbM/dVx3d7to/MzNCQ39mZ+Rr7767AZFPqseddAwiuXUJK7j56j53P1yu+piSkBEGwt7dglqHQKegY2MHtnre7qw/9rdikAI2SBLxCp8Cproi2skamfDuFkrEPI5ol0K00N8wElN9Ee6+MhBcm0ugf3yV9hVRSP51K8sPJlncz46pyR2lJVWEW9JhM2CTQFYISQRC4WXeFOqcz6MxtNDU3/08pUOn0zYQq21mZvJBlTyzj4qCLGOVGSUmqSgJiXG5OIXD/Dk5MWYOitoJmnReiqQVyvkQmk6FTSmCU8vVFhBaFdgusdi1NmhvUuB5BqyjDYLCvE0bTP/Oj7S76hYsxo7ip+QKZoCDl3h4e8P9Auo+2icy2k9Q7Xui2dln70DZXRRMXBl2g3sePQCGly/cEvDU+dDqFkxel5WTSAPI9v7Fdj/WKJcV1EmDmpMdCsgNfIsjpJtScJdlHUh74secfPHr8RbyrHDDKjbZxIopmHoj5mje84vBqGYNVRtZ6fWnUOL7v8zkAPo2TeUD5O05OMPSJNzgevYUydRu7Z57n2tB8bve0s3xfHnofVX2O2XJAOtcXIeu00NxbC/C7sRyAvWIrJ0aekCSdretI1DIIXYBTyec4yRsBGHL7VxRGVwQBvp8ShbdDDXVGiY3t4SEgIDK+x4+o9nkT4ngJt7ZBDHWYRke/GG73kmRaw8NEXnkFHF21NDvcpMxw6x/v+/d/dytlewhoS2eHpycu+vFkj55JcVSYzWdFWzEpVKIMquTIhCyupFzBLFreyZJiyiPva150h8n7xpJ02Yvkwl/o7zUGo8yNO7UDMJrse29XZt/fc/YZLWDLLf23cLr3afp49bH/7m9biVmmwKCUgv7mphY0zRUIwj+VBrr+9j9Mv/875X862NfQ0NDtT2trKzk5OQwdOpRt27b992/wn/Kf8r+hdGX2/TspxK7MPqsRZGXPARjMBbi6NtGuqUBmkOMis+eb3HTzfTY3K9nw8AZ2T/iAUzHeXK8/bbse75rCkz47WfJtT1auXWmniJfugZtvgaEd5+wrKDtbqQ6zJJQXZHB1JTN978ezdTjjHZdgdBxCWZCkha5sL4Prz5JeM4V+lx6hwapvbDUoG66jFTVcU7ZxflgBfzxwmUNDHqTD2Gar1/n6Pyn2+Z5WVa7lt/b2kJkceaYW3hhxmF8e/oX8Hvm2AJOzso4w54tsd3qQp9ct4sl9aUy6qkdldpVQv2dmUqMO57sGgb7XhzOlaibnTXDpv9NHl4CTBgGN9xsEOZYy3AGu9M+kOLoR0SjywP0ig+bO4WrtQtZN7MVIj2vWlwbgcvlUtubdx7dNUO1bzx8zD1AYVohCLmAyy1iyp5jLHpcxKjWcnBrA+sfW28bApbLpKGWd1Cv9KTiXzPBz95GsX45LRxICJmjIAH0zbU4jacUb9fkTNhkfUTQiGpuYF72erzw8mDTrY5p8Y+wB3T5rKFEvZ555BsTG4FXjgne1t01qEyTZwBZ1Np1Co60v/t0GaE1uLQoSQ27lwVv8Wfs5dZGupM3yYPULq0lPzkQU4UjeYzy+L497cl8eXbuCpT8s4cFND9rHn1sCDN9Dfp9G3vniR+J3O6PVdKJX6m1GnyTRYUdmWh3aQYoSPMveIsztll3GUzBiwsBbdaCWazlbuJCSNV/R80YPjo09T1lgGT5eIo9MOsyxOSGYo2DY2cF451rYsl0PDtbcFrUyBp8fjk+LBBPLrUvh0wtbua7exIHLr+F3u4LSoFJaXbRSsM+kB6OWJGM5HT1zcGxXM/qrJDxaJWQtDek4qY5xOTGOH+69iMkE1a4HuRQ9nitOEpLZyuzTW5h9tkOPNaeHJXm2wazHbHGi6sxQUN+fCHU9QxxAqxEwuE3EURiOXCZS1xGERtHCFJcROAXp0NQU4593zjZGzpXMxWR0ocf5FGZvloNJsnSSMpIQHTwhdR9ogknrV0v9c9tpmhdAk1uTjYGmUECr6i77eJWDJW+j5TqF3l9T7XIIjqUS2nkUx51pHNzwBXLZQFy1Y0iKcWX5kyLzVkxCWfA5g9NGIaZX4NjqiCiIxMVJeamsThKxtZnYU0bCC8K7rI0CFThQGlgGeiODzw/Gq8bbPnY7qnDQZSET9NRPHU/FOi/8GmYjiDKIeQpz5XEifKS1svjXM/TYcxPn0q8p10pcNpkooyywjJOTzhCVHEWfxnck2RWZ3enSUduGsq6cnic+4J0hjzF2k0TpSW3+gdSsDMJ7zaIwcQpxjk/Y5pNclOPY6kKv9HiCsrVE5iUSpIm0XReRo2nXMOCMDq8aRwwKAyo68dKUYDQpWKueyGcxe1n35Desf2Q9JqshnfwBBM/iZy8T4YVSwLU0qFQ6PMjkkPI1W5pcOJCi4tTYs5RHDZfABF3m+e3QJ7kaPo96cvl7sctT2Q95VqegYFaQaolz6KyGsuXQcix/CccLHmKsYwc7ln/Pmqde4nhiKDdrJTmV109Ih9MzNZ9T5rWfZs8wEEQ6BW+ouwz5P6Pz6M/F8Htsf/hXjg16hLTYVOp1lbAnkqm+j/Pw7mLu3VhA1O0SBLMgBSXkGtAEYjTL6VTnsq4zzlZvLy/JYT3eP40eCigPakRZXYau8ytaDU0AzIz7lD7+B+g0OuLW0Yfkwl/o125ncwM86fg7uy5M4MVP/oVzs7PN6ffCoIU82vtZWryXoD1QzcJt95FyOUW6LlPD7BrORL7LqmYDe6fu5U7MDZtUtLW4yztxrvXFqdGVPmn2YN8zh67za8b7aBStjA/sQ7n7DpQqO0PeVVVNtOcVbrV60GC094W1TA7bxVe+ZjwaXBl8rg8BZQHI5dL1766u41BrImbRTEG4Rf5MkOSSqtvCcFdXENJ2i80BJvatSKMoaYq9whGLoe8aZpV1UF6YQe/03jh0OHQ5UEuju/uYEqDgF/gzABdzMZ1myI3KI7NfO25tI3h7wDrefBN27gS1Shp/BqTDssyyVnp5SfJCXZ2zz7nBIPESFa2R/DI9kL5ut/ngm+dZ9ukknOqLJbCMtUm0VZDzBSFOVzk3qpWjz36Ng8xAZOcHcG42nJ+Ht76IlQOkHFxuBbnITc4oBRPObWkMDvqTeT3f5/STZ4lN20grORQ0FGA2dz9ISlgaqUMe95Lyqp5XnmP/6IHcc9jKIz9NIyYnBpkoMnqkkWkBKxmnvk7/i6GotVgCTP88jEu5kEx80AAG0ZFFvV7FYFIxzuEa9VF+xLou4G7/haR13M9V2Q9SziqA2ssEzPCnJKSR4IOjATCKHlB+EHfDeeneZotj+W/MPlGEpKIfWOTyC2qDn+26NU+hnaFisO1Tn7vWw4X78aUUeYk3AcXOOLdG4WyQnIOELYTkj1ji0sSI7AR8K52p8anpNnYTc95ijRd41LgQfVfaHHqo7kdl0sGfAXBsOMMML1M5fQ934vIxCk3oze0QNI3b3oeYfegocyokB0KrKpfslotUtFTAqanMVPTGu/g6wq40Xvz4RUKK7NKPAJ1CI/cib9Po2SlJIHfpi1s16dwOXU6+72oqXHZT7rGDVn2j7fonblt49I1f2Df9GiFFISgseR2pPgVtRdwMfgLqviI5PYFhaQ8SXPew3R4RBAzy7o61mKIwABSGSnyd8m12kFZRwR3jQUoU/mCW+qS8JZrV/qV4yyR7IlIBkz2LEC3M2cGTz1M/wp21K9fS6CFJbWZUjebXjPfpfbc38r2j6dQoKEmYYKtTH//DpHil8Z0P9L/cE3VrLe1uAcjMrSzrt4JIj+uMjpzH6vvvYXR9XPqRleHlNwrCFnCkegOeFaU0u7Swac466pzOgFkP1WdoMviyvglanFoZceVbppWfkn7rngx+Y+ivaadvRjKB390i7uwP3e1V7POizuk0tc7H0XY5e7grs2muc8Hvt3aWfre0m71b3hLFn80u5AWWcXDCDu5G7KRZb0cn1OlLeTNzNhdihnEqIY5zsYMkwIL3UEh8B2JWsrhSxq55O9i6UPIxyEU5ZH3KJM8ltnmhMnox+lYhl+YX2e79eOQYbsTkW5rK9Ld3kqE0eFGWMIvvXjtMeWBet33YbicbbJ90/b3RLI23VmeBOs+6bsGpKI8reF9JomfVdjpVJnISNajMbtLFYbtoS3jLckt7TqJOPCD5E9rN/lS2RvBK9X7OHHid4UedeCtqH2FVTwESIwigyJpvVejuxLfVzyTnvBZKDJID/nLZNO409GKEBg5NPIVDu4rY7FhbvYNdswhV1GMWzWjaVfS/3B/BEqyL9rjM+mkRKBNe5Wj5NkSzaHFmW52oSyH2Geq1NWhV0l4fVhhmA1zdDTjA1fLJZIQaCFHHE1q9jJ7iLKaFvMQjvV/gvfNbuGd7n38j42llB5itABwRf39YuxY2brQrNyj0CkKLQnGzBIOkDpTqEJQjuf6dWp0QzSobgPdeazrVLgepIYs3bs3hWsQ8EO0I/JvmULa4VNHibsTcYj3bdM8vJggCCCJm0X6e67pmm7oEyeclvMvvcx0p6LefQTlxBF9vtQV95aIVPCkimATC8z2pCtCT2eNTMrFL+DU4pnHHeAidvJZmh5s0OF4msPF9RoRt4bkjV/i4aC0edR4k3YygPCqVNvdgyVbQNUBTJk391nPltA/Tdwj4VfrRNXAVLMult8qeO1EvNlOvraVDL7HFZX+T8bQFzyzjT7Scp0xme95ZwSyjqCmREKccFk8/zBfPb+HcsHM0uzYjE0VUsnZGhm/mY7dCYiv9mbU9itm/z7aBmgYG/YmLsolfm2HUieEA/LL4F5pCIpGZHUgrnclt08u83xxEo2jAbLHBrePGmg9cqq+JPN/VvFswnnL3HWTVDuV6XQJb3Sq4PPiGlK/WKOsG4L6k/IALMSOIO5rBQxse+ofTflzHT/Qq2GBjrguCQJvenVWnD9JqDmFS4Bp63LrOwm0LCSjzR5Iu1XCxdCY/O4zG8eBIfMtF7gxegsLoZruvqKsjUA57LGk5Qu714JYOGgOm0z7wKE/su8MEr7sMVdv7YVuzhiN5S3hh8AN8PyWKi9FXSFCCe9sgHvPawBzDKMLcMnDWxjHsUC2GV9Jtz7PK00r9KWPgxYG4P+COT7VPN5shvTKdpw4+xRcXv2Dub3OJ2+BNpfs23mA8ox3ArdGN2LRfKNsnsa1bxTLalRJ7u0PfgVwuEHPvBC2nrtsDhX/Djxcczyd3fy4ZqcHkRufiqnJDKVeg7FTS87rIuvDP8fz6N+KLP2NppBT8Wnx4MmmxqbTIC3F1caGS/7FSBQS6NvLsmQpO7Url2MvHbONfLhNh5FGY18l9FUoKwiWb1aFeYjGqG4/Dve9QCiKN7o0cnlWHx0xfWpxbEJDxgVQ11BY5UrMZsK5xyC12thXIZT1zSG09d65dNteE6R8Bk9vVt3kjbxQZ4Q9hMkGb6h7l7ju5VCbJ1lvnqmuTK4mZvYkpeYi+4pJu7y4IoFe70KnuRK5vt6UI6Hq9U1FJgdM2ql0PUN0h2YgTwtcyNHgnFWnH6Dh1mbiMRuR6uW1ujHJaykOJH/Ksu5IBuQdta5z1ekbjfdTpXIhS/HNvnnh3Nt8GGMmNKUHRfwkB7etIdJdAff0C9gPQQwFXU72pDk3BLLdITbZL/tSdLfBoWwdnUs9IKY3MXfxvgVPZFfIjGZYczaLRouogwu3qVLblPshHDZLyh9Kaks9iZzkpG4gv/YTHCl8h5OsLeFfbc7gCNJgLOJPQi3UtI+z5q0X7wP77Hm0rKd+hi3qFI5c+plfWLE7FPcT+wXFcr06TBoxjGM/LxrJX2UyL90BmtmWSkvuXDRTAKOm8/2RdMAqDnFafCFvfebXtRCYYOVR2nMcPLKbE8xc7sw/BJuP5VP+n2DFnBxOjJgJQ/ks5J6efRNZur7+8qgy3yjtgNuNYX0Kf/e8SmrEHzGaE0aNIH/8SODn/W1/nf/X//5T/M+V/Otj370pUVBQfffQRTz/99P+O2/2n/Kf8o4iiSExMjC1A1ZXZZ/pvBPtMZnuwr+uie0+vobDKlWuDcrgz9FHbAvRYn6fZMsuTQDkElAfw2PqFJGT2QPib/JUggNygl3JfWA+2Zfvg9ju0XFzK1FHLSe9Xx+VJUu4cmSCDpmyC1OeZG/8+zYf2Ep6xB5nRUie3RBh7jjNlj+Cfv55ZSqtcl6Vid9YgMxvYYOzgekoRNyMP0OB8vhvSVmaT17DntLKjWCO50nPb/8PeW0dHdf7/vq+9x5KZuLs7hAQJ7u7uUGiBttSNektLjQrUaAstdcGtUNzdJYGEQEKIuyeTjM/9Y09mkra/e3/nrnPOvWud77MWq81sefbjH3u/P2w3LSOkdrr9OsCaMcm82q0PYU3vIjPo8L1yEK+yHMnobNFD5Qmq1LE0XU7Gv8IFzdBJxCd8xFiZ4r90+F0ERosqYhNWcazoHRYe30y3Ioi9G8q8L5NIyE6QhBj3RM5WPcGJ2nCuOf9p+y7po3fcfomvsp7Ar8KPIUdTkBOLKE/BRalpNw5WMBtQ6vTIjXL7sy1GNzbcfAsvYwXzrvZmwIkgZDeu2vrJAPtTIf01nr0yiuNpT3NoxCGOdxtDq6IEmSaQotR75NV1Jc39N4r2HsbaepImawUUboObKwjVfUNaxEHo0YPQQj+JKrOds++o2wJOdk6i0Pkv23d2dPY1yfNocL6B0uTLQHMnFop/EOsl5ana0RDLsObTXAhaZh+n9opHlLWeFv8qqr1r2D9mP3YjhFkHVhMo3an3kRCDE9Yk49bghpvYCrnf0SdkB8vDLzFMrbApwZJA0ujelbqoL6nQRrSjfLTwU1IlQ3IOsfvOk1zJfxTvSitKvYJLPW9Q7VuNKIhMnuEBgM6i5tLIEehcZIw4NAKxQ85IqQGTt/gx8vBQsks9Oeyxh5P58zhVMIci2XxaeszlypA06jzrAMkYztXnYLMzz+sPs2jSeXLjBWSBkmNeFIGJOVxtjUMbA3GyaltlUptUCqkd8jZkn/VviB1bO9sUNDMGR2QmoFY0MS5zMX+88Shd98lt4yj9O1Uwh++ufk6RwcrN0Hnc7hHB2bG9qbETNMIzh+9Q7PUqOo03dO9Gi3OrhNRUqCF4HITN5EzD23ikn0Rj9EawCNI3VV8iSfiE8XHfkFJxntT3RHof+Itb4U9S4PcNdF7OPmMkV3teRB8KPW9/Ra87Owl1C2X0aEhRrmZwwHrCMg+g/Pkij3/zeAchs7K1lAblUUz1V8kYaSU9JR25KKNv6Da+HJ3CiIBiDj30G/UhZkYeHol/pUQNRdFO2BmAz8lhBGWvJrggiPHez+PbNNw+P3cHLOb1Lpv4ZUkmviNTaVG3khuTa49YFgUZVX5VXOtzg+YzzQzbEU906aN2ZB9AyYEMIg+tx6gQ7I41AGV1CZ1OfI2ptIrq8O7g7KCR6ezXmanXsxi7bwA1XYdR0eVLHotdab8uIKPVuZWrfUTuR1eCFbxN5SwfNJFPRvYhVxFMpgFqfKopDi12rCmAhixmeevZ89YHbFoEouZxnAyh0nWLmZG+pxiggtL4aspi+v9T0LUZGW6xidyAj7ivveW43mbAx0Ffp1A4jCaf1sMXu3ox95Pn6HemH6JZoJctCcXw+J1sDyuni8mJYUcHEVKgwmCRFJc7Nb1Zrj/K4go9sXd8qHY7xplEG9oxbAZ0eQ9d6EyWtmqpU2nxqpWc725Kd4iYz13tJKpbQlGEhVEWEUarc6tkYDU0QGMOu/PepkueG/UWiLoPAwqH88gj0DUqC4NVxYgSqPNqZsPcTZzu9RVak4Qy8XYuxcn1Ehdjx1LktrVjP9nKIKerOFlF1C0aHvjtAfu5rlY0UNkSTmnU1+R3GoxFsKDVaNtFw0Ny8hM8FHqZ6TsfZ/7v8+3zPtj1DmlBeziU/TJnos/QOmEG5wfZ5o9VxGKVszXrNXYXD2WbLot8v686IPuC3e4wIGwz2yrjuGcEbxFUNrmje+B+nkpeyYKQZnY9sB0nvYqwwjBbzj6BzKoB9Fffo5cTZA/2pKzTa6RWfWhfMz/ekFAthw93p/PeIKpcdtGqLOwYWS6IBJUG0ejaSFlgmdQnVithRXPYP64370qgYVLzfqebchY0SAR8Rlz5LXcu7gFp+FmnMCDjN1J9e0poDaVDZjDZqNXa9kr7uLTbN7QWaEGN0axiX+5jbLrzGXe1D2CWKUg49xOq6m0SOr/0AJyaBGnfcCFzGkGFCm4Z4IN7w7mu2SwZ8f0GE9N6icFd87g95ib14RJdj+Dkg35yA08evM38w7uRJ0hjtD2oE1FfRvH35OYSXY40Tg8VhPHX3SfY1uMb9kZWUOfVzJn+16nwr7D1lxHufsVXfq3sH7Of88PjODs2hYEqh97ikKEstCEifHTX6ex3ggSfc7jKdLQ0ReJc60xG/G981ORLvuxhuP8bXFyC5fhohgcvwqlFoKHUl64Hf+NWwAk4v4A+zf3ZNUtGuOooVqvju9vqbE9BBTBG/JSjC44yJGKITd79J7Kvrcix8vtDP3OjbxMvrJrH/CubpQsqb0BggqaJrqf7M+RYGhMu6Jjj85F0PX8DOicJVTPvj8k46Z3YOXkn0a6TkcmdIO4pSH6buZWBlNW4MuxQhA2lJANNOA3qUTQZJcO6FQs5ge/yVHof/rj5B7hEEhSYh/srVyge50yFfwVGZTsH2501zHLV06JpQdOokByv7cbXblQWzHbDm0wm0jtkJ3vmCOisSn5vgqRbQSz6aRHudRI9GpELYNQluvRZRVDpBO53ncb91MmOerM+YfSlMWwONfPHnA3c7HyTXfP+4LPEImoHHyM//gIF9cmMciujuwrqXM7yg24sv+ceA/9hlAR/S++QXSz0rMFDlJxBeUaYUzWDM6WSIS0xLZvWHipqvWuxyiWjUWbVQLZmvYag70ZsThc8iiu5G/gOt5slNoo3jx9i2/0H2NYMJUEVDjS/TqKFbrX6ERAeibu1gs7Xmnnlo1dwbVQ59gljMw+qCyh47BfWPPklpaoTGOTVKKyNcGQQrvJ6Hq6EmKxORN+8grteQsnh0wtcYng9sIz4hncAqAxP7nAmYde1LFyJnsaF+OGUtjicaovzvmDMgQTUpd4Elgfag1L6h26hT8hOXqwIJatVjlOrEzKTzB7Q0idkB5NcHqG5cQe1rmdods5G63QXP9klEEQJhe6dxpZaJxZ99QTT9n7C6GtNJHn0gM5vcrrhfb7r+a6Ug7o9PZ+xGW68goiZHfUS3V6p1yYKlPv+1djUNr/aG7T/gez7m7lkXWsVS1ZsJSRT5MDoA/Y2T4j7gkVdX8BsasLiqUXnFknvnEN4mBLsz3qfGGLPs3osJYzb3WaT6LwJCjZxgn2UNsVxv/E0ySdDCCqopqfHONxbUwH47tqXjN2s5fnqtg910E23td8iGPAwdiK1YC1/BLTytG3+P9XlfT7xgXqPOnyrvfCp9rGPxbH7CwiQNdJJCRmptzk18JRdb6lutVHkXn2Kp8ZJc9yvKhqlyc/RnxYjPa356LreIisxSzqb2wUyCIIFF6sfXyfdIrnwGxLl4+jt+z0Az6Q9xWBbYJOPKQXP5j64KNzs/dWiusc9/1VkKX6xjYU0NpGR4OXVDpUgM3Ou7znK46tpcpJy/Xo3D2TkjSqifn0M+QE56anpiBYn+/n7Q+47XIoby135dk5Vb6fMa0sHW8KnwjSWVUO/zUqsB7ZQqzlHrbFM2sOxcN/vS76b9D4phz5BZmyTc6V+C6qdzdCMPKYpv6VBfY2N1S8yv8ty+7tdJ5RS0TWUpz6fR0hRiMO+gYhVsJJwOxCvihbbuDr0qZvhj7G+ZQwN6mtkRM5j2dBe3FSM5JvL6wDo63aIhc4KAso8yfd6idOJ3aUHtQVw/gFOnH2eX3wfp8Gtgai8KMm5KAD1N3lB8xJbbJAoq1UKVuu+2ZdvLn9jm2V/c/a1oVNt60XEwVbRZosQrQqc5U2szF5Ol8L2DnQJWVqnC2DPHYmuvUt6Cv4VLiTfSrYfinLBSElzBPeMcKHXRa52u0qjW6NdUz147xGuW1ciCCLn+5ynyreBCs2PFFkvQv1N4kpGsn1SPB+6hxEh9qPJ+SY3mg/RospnV/YLLL34MXl+FZQGlvPod4+SkB3XwflUK2bj1JhDxD0vIgoiEBBwt0TZryebFhFa86A9cFQUBPqHbiHF/wjx8h8Y4PQtfsWF3I+4T0WAxObkJG/GX3MfFSbMMjPNbtjzYAkCcG0Zmj2RZIRBpV8ltxNuM+zYMLqVBSOz6LD4DKKkKZ7VCbt50dPRpyfCa+gRJOVhdHeqRm3bH9T6SAa4LWRNz1dZMyaVcbFfodN4Y7VaSLuUhmujK6LoSI8hWEVKgySP9tBjQzusiXu191hzaQ3bb2+nQd9Ara4aRB1vJexgkTv4awZR2Gk0nj1jufHLDVoqP+J+wGe2eWNFJhPITpuHun9XLr+wCc/STPJUO8g3XLXXUXgsF22lFq1CcjJ7OnmgkitRGBV0vexEU0kTVlcPoipf4IGYFwBQiArb3DMwae5c1sv/ewxUvyhk9O8ahMUC2buyOfvRWZauXYrMJEMhk4EoA5kSAZHNszZz/pUstg2aBZ2moKraDakfSrTnQFlIK4FvRkv2EWR4eEgO8xNR/en9fW/M7djGBKuMuDjHWSMTZATVzCawdhYA3bt3dK5fMKzndGIPdlR+KN1vl5FMWK1Q5XaYa9GzWHXuE9s8lMbMJDeh1egxy1UdbElWq5XlFx+nSrWapJvueNR7dDjn0svTOVrzM7Uup22/iNxv6g1d3qNB50O/sK0M9p2Na4ovPhUGVHqVXYarMncjry6FU9Zttu/v6Mi82TCTnl7ZzHOVrtpLw210qjAyqvoSVDeDENMgAuun4eccjI8PfHnxRy619CKo5FXKw/3YOOM9ToTOlp717klWn7/4qA4macBThOCG6agNNlnq2jK48iRrc3bw18AVHJ4chNbDkfD8oHYrazOfA6CkuYhlF2eS77sWpY1FTSmX9jPnKIkRp+eVsbzbeQcDwgbYxqPN7my2t7cDso+O89Eu7zr5Yu78ATdu9Cbu0h+E5/tJ9iBBkALMjg1jhkU6z3qc1RH647coTGrH2RswlIbu35BR4Q8ItKrVmEQtZquJe/JnOHRvCY3OGWzI+pU6zQWH3djqQPb1Ce3DzE4zkYtyBvw0gA1eGxj95WiULg5qWZeM88TYdAv3yhxEqwXfousIFvO/ouKlNv777/+nlb/7MP631/8/60VyuZzS9qFO/yn/Kf+Ti1Kp7PB3QskHJBetYY7uFMPSixjgN9F+TWxD9rWLLmvPnfx98zTe+GY8Y7YGkBWyzJ7AXWvwwE1Vy/0oLWk3n6EssJLikGKb4U4qVqvkZLoydBDrl6x30Hh2WQHjMjFazYR1yuTAqD9wV3fFv24ivopIGHaE3yvO80DKG/hGZWKSKwgsk7hYZCpP8O0HgpTHTrS0KZi2kvgSJ/0eBsCzxomp26cSmRfZQTltowq1/IuzD6CHfg7qKzdZ8mVnHvvuXQZrPAlxzeZY/kIOFr5AVdoOCvtPwaRSIzPaaCyc/GBWC1cD51PnWcedhCosgUFERr9A574fMlAmY4TSiT+AI8DvwCDRiQGCjKD4D4iKft7+LQB9zklGvTrPOrugUqDty6LMqZxsBc/mvkzwed7+za+mfMxLcg0J2eGk5DxDn7vHCXELxUnezJ45AsMrRQqd0hi3uYzp26bb64nzvoTVKhLemsX8x3cCoCu+iF5eBYJMihjWRHDO4xDz9/fGrdGNSp9MqR9F6YD6+OwG3jx2mOZfz5F29ARHW1dB0Q7I+Zqbbr/z661XMAR5cmxKK40+Sfiq/R1jYTfM2aKG/rbVnvV8jNOduuKrHcyc5qV0sZxgauInTIj7kninLJIyk5i5aTyDTgzCv8IPUYRoz2uMjPqe9wz7mfjMZr56ag1X0q5gd0wc7A2bnYi6IPDABMmYJ1pEphzeyQiv7nDpUYZH/cDi4HS+6/QRseWvMbk0E8MbBvqE9ccz72lGRP2IiyGWQDEZmUWDq8KDL4ePZGL8V4yM+oHwh0Wqxz/YUZDx6QVzrXxdoUWu6I9rpZ4uGV3+Nv+ksc6NbcYsEwlo+YwU3Xr6h0uCw52Wc7wnplHgtYLnPnuO6NwQSdnyToOIeZwPWMDG0lHc6hWJcWYvzEKrXWgtMATycyPUW5ywWBzO/7b55a3ryajr9ayOlMa3Ddk3oJ+Mjz9uh+yzGLEIJhSCkmSFgi3TPVnqdwezXIF/gYLU/StoaXqfNsF0z91nGHral7GrRiHW7uJI2iiOm94F4KPh/XmmxxwCx3fn1pAnOR6/ieNDjpHZOdNBN+zRiXPly1Acz8T1hZP0Od9HMmJUniDF8hLLkr9kWqsXAJWBctt4OkGXFXypS2PP2P1YLVZSjn6GR8Vdqa/1NXB9GYvSnuHq4H7o+0dxI/U2zoIj0nl/4Tbu+M8l4uxVVE0WmtyakIsynun1EJGeGXzqn8FGTRrqyJ6sX7Ke/AgpYIFCKbJ4fYEb/jl6FDmZNKZLUfMqsRn2dSGkJZsMA7RoDDRnF6PVtHI3/i6iLdK2zfhjwUz59XLUhbcBKWefIECp52bWCG+R0cfM2RF+HBx90I46lARjAaPVQK3mHGWiIyk4gNHVm7s959EY1gmEjlH4glWGUWnEtcHCtC1paLQadHLJQ+KirMdJsDBt2zQW/DpfChpoi1jP/hwsejwK1FwVDATWP0Bk1ZOoTL6IVh2cnsrLMWt41F1g5ltDccl7kqPOjzrqbRfdfY31ZIe8Qo7WodD6NowhqfAzQpqm2H/TqAUWL4ZFDyr5qREupkfh1ujOiCMjGKV+mQcegK8WLWNG9x94sjQAc1Ys/c/2JyI/wm64k4sGRNs8PTTyEBd7XQTBQjTroXAr+PSWEEMWgXF7xzHvj3nIzC54qLyg+6dcbHweZ99tXKn5BWVzAa5NrtI4FG6Gv+KI9TwLthrum6DWosLTE5If/Z0NMZsA6H06mp4X06jzqrOfU1dLxxChKcbFez+NymxMYjMmOkaSftj0ML+P3sehEYckalvbs++c2ssbx44xYHskP88YzDtvvUNWpyxAkBDABZvx1OYQo+qCU6uKCv8KO/JlYPhGlg+aiK+mEEEUMcUlURTjoEcLdcuic+BF3r35FJ/WSwasNrkBq0hRQxIrz+3maF04sQqojoajAZuJ9brMlMRPMFtlKHMgz7eSe8OWEGA+TIKnlFfDV13IeNd04hQwc00YyRcyUFsCEUWYnLCar8ZIuVRul3kScT+cIvdl1Lqckebf8TGwvzuvq+9RMfQ0x4Yewypabc4rCx4Nm1ArWpjj7Mt8v1WE1M4jRJ4KhnoAzvgU0+3+HwyvfAHfqwfR1Jf8zRktrUej3dnn1GEs2iuqaxrAhJJ35nzCH9lr6DnvUWpCUkgfPJMLvS5wKflVLGY9iEpwDubj9O3kXXiLkXvd6HyzMz8X96ZCNRMSl4FPHwCGVxs41qmSs51Wkx6xiPKWYpRKJXqzKyddtvHG7Jf4dWmjgx6opRTl3RWEuWVyLXIenxoSqHI7CECOyo1gt1xMyDiy8kH67NFwdsB1KgIkJCWiEqIXs6FJJVFdeqQSXrWUzooJ/9JeMwhWnvWAfLcZfH/tMz4Z0Q+T1YnDOd/ifygLs34bnuGfEmX5Dkr2wL0f2KPowqsF+WyZ+Tu7p2ahNHlL/Z32DfedlrGv0YO1+gcoM92mjean7ZwSRclIr7NosWImSExlaORQ/F38O6JErCYEZPioAnnK0AX6b2Wk/Cw3wqDJ3UhNcDJmjc1QbmqBiPl0Kkxgy/jdqPRGEs/+6DAy3FmDylDNLi00uEsO+ZDiEBQGo4Ti7fElJDzHSZ0L2cHF3OhWQqtzq/0MEwQYHniSZ9SpRFiHdWTR6LKCErMXdUF/cTnpNMfGyXByHu2YXJnvscRDB1bY8sRttszc0kFOkrdz9mGnPBWobQ3ibOE0vtWOZVd6GM56P7K7JjBQ+bX0oHMgeHYDBEKP/ETQnRMY2htRi3cBcE0HOfF32T59O7fDmwipy8EraAh6l14YLSrWRVzkCQ/JkCYAIYIOYh7GWXeN4sYEnjpfSO979XQu/IqP+vzO01Ofo//cGRzSj8U/V8Vuyzq63H2Tzi1LGDYMoqLgoem5vOuWCEMe5k5yPc5VO8hqkhCfJouK9Xde4NcmyI3NJy94O+mBg6i1AIkv4jz4J7j6DOYfZxCb1UxRSBFGhQ3pemY27I7kGZd7TPAyEHc3jtDCUECQxrH7F1ypk/SjnpfS0Du7Y1G1S7oX8wgfnt2O1m0E25ffY/28Meyt+dx+2Y6KsVrAFvjYlpNUGiMlueH57Bm/h9XPr7bv2Z38TjEu7hsWW47wxA+beOWjV5iwZ4J9/s3s9D7JTvt5zoN2dYkohGaoPAUXFoBJi1VEcharNMgtLtL88u1DvmE00S5FNoRlO3o+kxayPqJE14NXSyVHVaHvd5x0edy+/xlkddwMewxrzTIee380A04N6Bgw2oa6/y+QfedbfuVK0nvcic/GJDfZnTG9gncT7n6LwdUhrBzyHre7yFC2NjiezXgLeeNtsiTVAJ2yBL2iErWsEmqvECusp1/oVk7acirmJcX9w2AmWpz5SG1m3BULMova3m9tpdJ9H2cD07g+/DFSlCdRyZqZ3ekdQl0LeL0GKvyq+eTF9Zzpf8Yum7UY3ag0u2G0QqebsTjpnADJ+F/eHE29TjJ4JnidQD+pB7qQDwmsnyJ924VFcGYmz1iOMjPtHltmbaHSv9I+zu6tx7haOoaPbmVhbaezfnwrk98y3iXSI5Mlti1rZNMG+mWfI8492X6fVn2b26EvkuX0g20kOupT3sZUOhWsYZzLDAafHIx7vpp6jYRuEa1KlCYfXF39MKqM9t/anDlte5nJFgwoWGUd8q+nCPk8fq0PSr1Abkgm5xL7caJyK/G2lIGZYc/AZRMKQwtuTVIgahsaW2FxQ22IRIMfTU6Z7K1fxZycbmzNeoWn9qdzP2cPKo/e1HjXYVQYHYFLVhEEODTqJnqliMKg6KDrO1iNLCz1rWKyC6QadlHVEkZa0B6WhrzDK50q+Omhn9CpqpHr6yR9SRMJvX/CxarHL64QnZOOLhldsM/tQskwv6EJe1qNvxutXWSeRJe9TOfGZbbrf0P2YXO4tAu0FqwKvp8YwbL4D6Hch8nbh/D222/T+WZn2/wT+PHGKk4GN2FWfM+Fwa58/OLHdgf6maKZPHr4CNf0cK37NcwyM0+veZp67a8YZBKi35tLfON5n8FFUSRfM5Pn+xLXzL+AsREX3UVclQ1MVsXxuPNpu41EtCpQybTMDjrHFP0oXN2SSe+SToN7UweniCiKmOQmSoIr2TZtGwIiaovkEQ2sne5Yn+0QMy/3n8V7Q0eQbYziMWUZuybtYs+EPTS5SbJWj6C9fD8xii+0vzO7/2tcG9yDuwEvU+z1h/S+VsmeKot8hTEXrhB3Nw7XJleWZ/eg8+nBKLPfAOD521MkJJLNqXeg2Zn9uQ7Wk953+pJpkGRaURRYe2UtAL0ifuLskGrMg3wYt28c/dLXIwoOZ19gw1S6T+qOKcqEQWn410Acs8Vsp/2zWBU8dvsBPqwFrbqOysheuCSEcO7jc3S6IgXivNnrE/bM2YNMBs1+0QjublSdywFDFrudp/FuRQ97HY350rj2PVJN/xu/8nTa86jkSlqdW9k1s5JH7z5H0wwp8KBtLStktsBdq5ElDz/MGbPlv8VAdcIMeZZjBEZNY/us1zCON6IwKmxtFaH2Otx6nxV+uXipDZQHlFHnkYcWLSa37qCJQC5ToNaq6XHalfqT1bZ5ILPPjWr1WS6WXMRsNdEWZB3oL5fsTbazRi13pdv9jXS7/4dj7rXTmesshTRorlJjlFCF9uA0zJLtw5bTVmnrh7br9Z71HJrVSGV4MoLYkW799ztrKQjaQ150NfUe9e2CvWHP3T18UfAQRT4/2toj4qKsA9cYXjh0iY233uKJCjc29PmSs0PM6FV6+/dmGZbyzIEbNFraGHasHdrTYAzmrZtL2KWFKvcD/GywyeF5P+KkL6Qk/VfmfivD68sVBORK+tCaNTB66VxkXS/gXvgB/feV0v+YDzVOV6RnZU4o3OLYGQi7giBZ8KVXwVZ8G0dI87r2MpQfIkjQkVAQgUfxn5xKiMMkSut92QsWHozewuP6WXg7+XK4dCsN6itkN46GsbcoMkr5e53iw7DK5bg2OTPQdwqh7hJjhdiBqeefOfvapxbpUFrLMTfnUu+lpji2B2P/6sOwo8NsfSVA57eoFjQklQcRnieNsbUt+KPiBKS/iXvYZHYaL3Nt7AoO9H2TA91cOFj+Kw3mKKK9ruGtsFGdChZ8mobi0zjcLju0L036Js4UnuEP2R/s7byXIn2R/Vpzp57kpU4BQaAmNBWtexCV4T2wCiKWgiI8ym4jYMXV1fG+/yD7HOXvPoz/neV/2Nm3e/fuDv/+/PNP1q1bx/z58+nXr9//im/8T/lPwWKxcPPmTSy2sGdRhLDqJURWPYmnJQ5nYwjOMgfiK7LieQZmppPS+qxdAGuvyD3iupPZg29S5VOJS8Vh0lslBNaGWytYc3E9ZyuXEpcJVkRKQkrs9BoAF6oOsfC+yLGeS0hUrSX1/q/SBuYcCO5JVCYuJ6YABh3tR//dSaTd+5MuasnIobP6MO3S83zafAq5yYhrk7QrygURLEbGR67k63GdcRarbLXZdkbfvoS1ZLBUVDP/l950udlFirBup5S0HTTO+jCG3MxhbvPfjOG4knVZynPkoQnhQfkDrB2fyK/pK9mSs4rxl59kQ//hrH16F4cGf4ggc0T2jrv3Jp/2LGLf6N/5Qz2J28EvM37AcAq+NDN4oo4XXBKZ7iznNdGTguieyJ/3oHVwIS/2nc30pJUkuGcx0BlaNK3U+OsR40VclC5w9XneSfHFUy4pWp7N/Qh2kmgCFqS8yoSwffRKzmfb9OME3T2BZ2kmMhkoRL3926p9JMEq4Y6DkqZH0F7mdZGocq7oprH2qeN8NfUZal1OgaiA5DchehH7dG7oBQuWNvoVqwLB1Ihb1Q94eB1nfXxPAO7E3UEuKEAv1ZXY+BCbE2Io2vMyne5PIqHqN+I8HZR2DlqGf3e82oUCq5nj+Q8A4Kqq4ZHuz/Co32oW1AcRcy+CISeGEJubgkwGaUF/8VSvhzkri+K3Juk9KgGSNRW4igUQ7DBaDo//gyMvH8I8aQo14UMwK5NgyCF+v7WKB3cVUeIsOY5FQUQhUyCKChrD3+Z04Sz6lGzkOacMvJsHdvjmYdHrSYv+Am35Hpasn01MTkxH44gAcr2WjBGefP3E1wjtKUFsyunxURVcH/Mm/saB+Ol2M6v7d/w+LZCn5ZPIDMvFw1lAFhhCZPU70rNRC6Hv7/QZ+gvuu98g8dIlljcGUuq12Uadu4dWq4KHKuC3ij//FTEhF+UozO6ItoRnbTRBPbrJ8feXFH8AEwacjcEcHKCna/o9Dt1bRIw8Ffdhc6kYMpm8qLtkh6xHJhPoGnCQhSmvoHLTciPlBjpnk41yUGToUGg1uRARdJF3r3lwJXoaAdfMjNkv0SO4GgtgsxpuvoO1WYtneTYAVb5VUpBC5AMcV17k2fOr+TDxKt+8tYHTI2wRoxYnsBgJUTTgKwODH1RE9kKn8ZLGSpBjde/Eh41m9gwegXryeFRee3jFNdM+TnJRRoN7A8cmFFIWa0KwCLgLJtSKZu5U9+Rs1ctc+GMungVKSkJK0DmZpL5O/YjGYad52z+X9197H2oKKPxkA62KImQyC5j1yLAQdS+KtAv+BD8wlMO2PHhtTj65KCM6N5qFX80ncnIkFxdOoUlThFwuKWx6RQVn3Q9zufcdWm05ntsUy8Mxb/LVk1+QU3uI7oc3cbBpBu2LVeVEo18MsuZGOyVvW3G2+uBsCKPOx4mspGKMCiNWFx/Sy4dyIn8uKxt/pqdCRtT9KN54/w0eCMgkrHguFGyEvJ+wCjLi7sTR7dhePEtu2da0EUp202B0ZXW9pMjE3wmiVpbdYU38naqz/Zrx0vYlqvJZfLSDHM8gMHkyTJ8q40L4R3z8xCXKRk4gq//D+DnFolRCuEcWclrZ0OBJ8mlJ7jrb76zdWLRthpq3VcMZ6gzqFjXdrnUDq0CzEC05WixGNMU7+a64G2FFYZQElyA3u9rXq4u8ii3DZxCq34loqkewCjQbmsEjFZJexcuphEc99DgL8JwHvOl1y07KH9qSwRgnFc4tKlyaNbY2SxtJs8GTIr0bJqDUYxsHurmy13NUh7G6ZUpgYxOc63eO833PdzAyhLll8rVXA0PapXETBUEy7p6dDYd60d84Avd6Ay3qFruB7HzxFD49/wvO8kZizT/jXJaOXO/YJxZ1XcaKgUM7GNHaI4CbDN40G715PDidUIWj7rcGjeWXGx+y4vx63Op70/PaaP8RoHkAAQAASURBVESjEZOTC3K5gEwwsqTb8+xtTmWTbc9W6hoRBEmGqWyO4GblYHZHvcOLPU4C0ONKDxxo7VZovscUZTneSXlU+lWiMCikc18Qye5qZNa2elZfKmC81wv2OUfEHOj6CaG6b+gWeAAqK7HIFGg9gjrs6TI6Ivtk/wWyz1WEqiiIEvKICKpj40YIan2NzyaEUB/1FwfGHGBL/0JkW5wlSsERZyi1qsjqdR6A8fvGMt7vFhqLjcrW1IxW5oXsSD8WfNuPButfFPn8RFNrGfnnvyXcPYMAhR614Ahekgky0BaguvM23YIOoFMWUs0drIKFSOcURoT60TXgIEfEXlzyrqDW39SROkgQodf3LK2UrMqeFWW4Vt/v0N6OaDILC1whvGknldoIvr/2KU1eyVz2OU61r55ZW2bxm9FKN9OjkPgSTK/jrHMP6vcP5IlvnmDOvW/Z2v8FAhs/h7AZ5Lp+wsxiDWcMZRisWvv8atsPBAFOdErkhRoX6jWX/4EQFtvl7PNq7sehUaWsWXAVQiZzy5LAX7eDEM1W7nedislPijzm7tewK5gYpZGSoDLuJYVQH5DgeHev77mdJAUG/bj4Ny72vEjalTTyW/60G3vbvnHE4RGotXK0LlppLG5/Su98FW/2eorXvFuZYvmj49mr9OTx8if5NssfrwZ/Ugp+oEfFWkejhhzkqXIPIu9HMu/zTqSkp3Q0KNqNJA5knyiI3K3pxYdnt3HHFMHgE4NJvOVOc/AsfJ162243QO63sEWDpXcNd+IzuRwxgHKPXVK7h+znVI+dfFAH07dOZ+HPC+l2JRI5uXDladzr/sBoVvFMfm/W1UuGNBnwgLwE0t/ArX4rNS3B6PWhyCzuuLWmMCl8On2yX4bLjxMnz2LV8RE89ulkpm6QkVI5Eycn+OLJtUxVxDIt8gWiva+RfMVDCoqyjX+Y+y2iXKScd/7l3vS8GINoLMCscJfQimYt9+RBXBf8qfSrZffE3RicbfRXKh/wSOaBul6sy/FixrYZtuAhAWTOEP80/spqdnrEUhPbQG5cFeltefKKdsGNVyhrjsNkUTlQbu3GQq2PIrFoFV1bXqbNiNWWrxmgr8sF1qRUUtbrKk1uTXbH16/pK3n7xF6G+y7H2VdaxyXBJfbo5rVXvuFyy0zWNrRfhQLlwkjpf3WVcOs9cqJbcGtV41UuBR61nVN15gRGHP2F1fXQqMridGIPFh2aIiFaJ+RwrHJ5h7xJ7eVws6ilwG8dfsWSUVnnpOvQZjvNu0WBR3MvvHQOQzTAHM2fHJuaTnl8ji24TdqzPzizg6V/3UUURHyrfOlz9D6eZVmONXf7E4z+I3i6yvEui9lKtm4uABH8jtkqp8qllZuztpMXU8+R6p+pszmvoj2v8tGwgXRR/4SAYN/j2vZqhUL6S2+FbbXenNT/yKprR/B0LqdW58NBLTy2biHzf5sEguO5Rr0vF3Tv4JK9laCyULxqvextAliwsxy9UzAn7oaQVbQVmVFnmydAUy60lLBOPowzv4zjlZWvIDfK7fJIcPVLPJG2FL2lhWJdNnWaSzRYi5C5BOCmknSqn6RYA0eQTfvyt9xXHXP/gbslmsiqJ3lt3BcUzC6gKLSIv5u3mjMLMP9hluR0q8I+h+yOC9qcffIOe+9z1h08EtRCVYiJvC7SXBEFkfHjHX33/eLvyUvuR7OmyX69/TskBLo0Tkcb/Pk1fSX59V0wakX0XkH8seBPKgIqkNuCitveG1TqQeTdJkSL2MHB2T4Vw6lmNVua4LTz07gqa3g87TGaTW4MLZGoVMftG4dnrYf0oNIdvHsxzJDBeA3snLKTHVN3YJc3wmfzs/4VVtVLpnnpCOhoP1GLniSWfEhq/Qqpbe2oOqHdOdUBwSTnZP48CrTRpBWHE383ArNoRqvRIgoiclFPtOc1NEIterdgmv0GEFL/Et2dZtr78cdRgzgWAv4VvvS8LOnlrg0yTLIGFqa8wnBxJINVDahdWriSlkO9R73kyPXtx82EBiZtMvH17UM2HbEtD6eCYVE/83zq+8xviqHfsR7kxuRTEdjQ0dmHSKV/JX8s3Mut5FsIguBAOApm8uUHqHDfi1mUUGiiKLL8uBSE5CLWcULVyI2uN1CIAwmz5aguaYzndlUfBIuBAHU2SZeuMnmTCzWux6U299sAc63o41YSfusQMouM2z0vU9lHsusocz9metJKTpeMAY/BuLQm4WwIJb3Vl5lJ7/PN5W94cFcROosksLYq87nVsp2EoO1YrALemntkRD7EjdiruD+9AJVsIjKZw9nXNf8XNk/fjOYXDdtmbOsQiGNH0lrN9lQkSsHC2sTfWOYJycJsrFixWGDaxmmcGC05p5Q2ukVBsNLU1Aj+vgw++gbZyQ6nQ5ue1OP1EfbfIorTGBU3jAiPCDxa+qFzC0cT4EarpZE6zQXytDel97c5+zAQEBCA+8BkRsr+65QzEgOVkoUPfYyniwsfxu5gqnsdNY/WsObpNY6cfZcehow3eMirhJF7xpP2QydkJhl3Wr1oSt0ObgkoBSsqvYrO190pee6uLbe25OxrC/6W2me0rwtRkNkoZh3ghL8Xu0xqtWCw0e2rROcO1yw2ZJ/V5uxTyVUdxgmgQZnNvu5K9pltwfRmPYI2Hy8RPOs8ibrng0ar6fCM7O8AAqtItPsFODubnsF7KGzozI91RvLl1bR69WZp+E880+sZ2xiDWn2b5UVuHEh1d6B8be+fFPw08eomsnVSIFaDtVCqNGwWzck/02j0pcVNCqCv9rhJla4EFxfoLP+MkKoXAfCorqP7te7Ijbb+NbUQc7QrYTYdLc8gszNlCAIw/CSkfcuvyuMMKY4k7UoacqPWft298G0WJX/IbFcrizu1MX4IRLmeg1vv4K4s41rUbKZfd+bSiwHkps3uqE/ZKVfNCFYFTsZAfDW+9ut/D/q3yxsXF+NyOJYHpkexd8T7ZCXdpzygXNJtRRm4JTDJcodUjYlK/1bOjqshL2CNVHflach8D3YGEajOpuuBlQw+FGF/v7v5MqNjvsNdZnMSYiYtdze97x7GyRRg75+rpVfZmrmVzCqHrejdU++SX59v/9sQEEZtiJSDXK/xon7mw4Q8Ok4SyM6dJ/qqFBA+eDAMGgRPP81/iq383Yfxv7vI/59v6VgmT57c4W9BEPD19WXo0KGsXr36f9Z3/af8p/y3yr8lpQVwNgbjbAzGyeSITGtP9xUuLyNuSAF/Hkhj+JHhnO3rQKUeyltCvnwJZan3uRIzT3q2HbKvTYkIKQ4hqiGNUmsAQiASxZlZh4hAp1ud6HOhO3AdIiZKAlT5UcKcrPzUFMrthCzuPZWB0jSYJan9UVWfhZNj8HaazAcV3uwyvWVrk2BvaFzTaXo7qclwbSU95QrXu153JH2nHR0UMjT6GDyt4NzOKPqK7k12PbgVdfUcBhglYflk/hx7P9ZUl9LzYieKQovIjzwHlb+BqRUqjqE0t1BtgRlbZ+DW6MbmB0sYE/wmAR7w+hRocFlPf48HOb11KNomH27cNFIc20zvkF0oZa30DN6Jv4cvk9IeIdE9jeLnPwCrBTbOw0UBz4WfY40JqnFwd3cLPECxNpgBpSXEqvToXHwwK5yQyaDJ4M2EjVYaRk2nNHg7vy08TqsyBxek5MP7c5YyOvpbnLouY/1vr2GNHYpJYUKwKhzzxCWSZxvC0fQ5S6+LvfCt9EWwKpAZyvG5t4TpMZM4azXy9ttvAzBGeAN8+kL5YW7JJ+Iu20HU7RAaw8ogpOP8a1MEzDZk3z9y9rXRIlkttBpduRRwj693qQh0uceM3q/w/NwT9I2vJ/byZryEZxFFOFEwjzs1vage9zvFR3x5+8AYagLL+HLVt1w3BEHKSoh7gluHl/DTtw10u9wL87wYyBWwyt0gcARlLaBtBUN7AQigYCNuBW8Dh+yJ6i2Cga8Ln+BQ1TC+ij/Kb6Vd+S6nmLjK02hag4irep4AlwCovYrl5CT6uLlSe2I2dYFKji3SIQoiERGQmQn+DeMJ9wijqSoCgD6hOwDw6rYAindRp9dyYGssw7aOQlgyCWt56D84/J2qCokv8qHXhV7o/G1Kyfn5zPFy53NJl8BssdJGa+RAhEnX2s7YJPNs5gxOJTUg1WZEbYtENdrvX/J0KJcv/8AjgfFEOf3FK1cPsaXHBrBK9Xb2O8n0pI8orBNZ6bOLSetfpc+Jpzn4VilPPw0FBQdQXw/h/SYN3/8ZQkS+hNDzrfQFmRr8BoOunEExSXz9SBxWn4XkKHPw1HUF50DqZYFcrjRxzseKt8yExSaoiVYVHB3GRv/LrP56JiUpZtYuGIdoVbJaX4qrqyfWsRm8sUKGYBUwmzrSvYEkkOqd9Gg1eib8qkI3qhfmeGe+u/oFhQ1JuIf0wXB1FQpLKYpohV15RxMGcg8sMgsWmYUrvbOpNZ6i2a8FUfUBTLhD7BZPllf1IP9iELkXtzCXyfw1To5sgjTfXwgYSWiRyB2TjMyKTA5EzsGtJRWZ7DoqVTv0CmYSr9WjqkxDFi/91iwWo1XeRS+2IlcYOyy48uZyzoQsxWJUsfibThSEFfLlHTecnT+ltRWeTHqPHdvfIeHqN9yON2BQGXB3V1GdchS1k5HsknNULNzCmYCehBSHsMC9HM+G0zC5CFrLeaTpaVxfGwXUUqm/jEmMRFS6wlwrc57dzXXPSYSn3CD6XnRHKjDB4ehuU2IFBObPh99/h2nTYPt2OhjW2xscY+Qb8Pa5h8XLlRZlEHYm6iESTY9wuTPHhxxHbpLTommhLTjkVMEchkT+zkue8F2Fv/Rd1iKqZCOgeiScGINr/DOMCqtgVaQnx4ccR7QqbVH6i+nuFsz3BUM5lXCMPc9KlJDoAJ+e4NOTyENPsChQy9778KkvQIFEIVxznWFlbxHj2onX/V7EO+97ul3thjBB+q56vR9fl3SnSLYPV3t+pI6ICU+xmcFFkSjyRiDXfM60xXJ+RqJ86+J/jPHqVm4+t4xBzS5cGtYT506+IBOg/1b0GSs4U6lk4/wrlPucRomEIMuvTyG/PoWJnV8mruQw2RsmMRVYsHs3PRODiWguID1zDM8m/MQtN1jXbMHX0BM/fzOiPhKAYLcc5gZks6EU1LlwL9KP0/eWkFPbEz+NH4uagtAcPkFDyBVqu4Qil8sQBAtmixx/+TBGXrmC3nU1BlkJxS676CabzKniafQP28LEvOWUx8h4cNgx7vpWoGmj1Rp+AixGRq0fivGUyCPHh5I+eB6qBDcQBESZnBajOy4myG9Np9zjPpWWBHAKgMJtxDRfZED4PE5mPYBZ3sLRlK6kV+0iNDTNNtdsRhvBlo/H+u/OvngF+LTNvV4/IpyZjnPZYUrueeGXcQp3jTstwTaLvdUKTbl8btzHWz1gb1M9/cyBfJa4C+p3wZ01gMC+0E8IM2zCtcmJrlc7cXBUHk7WBqILn+DZnp35zP0Wj1bAwYp2RgKPZIwxy6g+GwLO0rd5NvdjRdJnjG+UJtFlOrN/7FoGZsbxwC+zONc/U6L+tpihtRRPGUz8dR6xuadp9LhGqWUEIDEOOCKlLXiIZr5thOcDJ7Dy9Xtcuv8coQm/41z0DGdGxxJ9sQfOLq3oCMDJLQEULgiijNyYXKLv9yBeOEqY+21MdcuBZ21njS2wxGrAXduDAXGNdPLt5NgrbCqZVTCRYdnE2st1jI8bj68qlDZ0fJtBSBCALRqwGDhvWcb9rZEMKvdh7dNdcZF5so5j4JkCIVMIrrqGC3BsyBEsMjOqpgrmMBs8OhFx7iE2BcCCIo0NzQNX5b8gGJfDhcUA5ITcZblnT0qdbUEnyMAlggb1KH46OgO5Z6zN2NQO2Xf3a2Zylh7rHqM+xEiL83G08V0BD9DXwolxzHLTsdrFSpO7HsEq2PcvAJmsHbKvzeAsCoyM+p6qljD6afaSPaGG7Xc9cKfdcZD5AdxawS2Tkn0jPmDUilfxrIzi+qBK6brCjUDvTjyU9BThmRK6OzI/ksHTlsDdkwSwhlD3ZPbURXDf5QJeggkTsNIUz6vNd5ADG269TXxoOiWNsYAale4WVElUVxEyyFIkkYcUiKNqE8INdSBz4ufbP5J5J5WiAWe5krybmUIqAE/3XEy8zyWi6mCPWcbthNuUBJdI58FeCeWUGb2SR8L2wuPSK/3rbHtn2lcA3Lk1hh67pKCv833O46+fZzfeRmky6eubw62nh/JNxWQ01gjgZYkmtPo8Dd4bqKmTMf2dGGK7TEF83DEWzsZgoiteIFQJp9XP2sbHcf25wC8A2Hcjgss1aoSu0ne1mlxRKxpJ81rPkT6jeG/Mr7Q45dnPyLs1vdhZ+xHndI58ZB1o6/0GgjqEMpNIbkoGraZqssMX02xaDSceYIlfFpv5VOpeeQ0GeQ23a+tAlINrDKmen/GVopAFtY53t68HYOeUPwnQpnAn+Aoz/4XG08kQQo+8bbg5WCUBGOd8BIDkq91QNniSN1YNArQY3XFXVTJFUc75iiBanVo50PcBesp2AZ1hbAZ6sxnOJLSrS8QgeEPMUhS567hT3ZvVdZ5oXOrwtpzg07wthHs/jqe2NzM7vU8nvzO46OTAYtpQIhMniPTzgdYbn7Es6UN6VsCMmhpeEiVkxdor37AqYxmmLtHkReYTUBlBcHlfnOTS/Hwi7VEsqkB21V2kPKQAhVFh11uiPa8xv8sb1Huk8dO9HLpei+R+Shmt7gHS/BshBXVd3vEEhpoGnPROeNZ52se53O99Mq4fRh0eybO3KzEkgtm8lL0Ln+Pi9mAONf/KOd0CQAqAhI76VHt0xN/HERzO3wM3txG+KZyYQTHUh0vPtCjvcy/gY3q8EYAcORFTI9D6tXP2tVHd2Z19ig51AyT2T+eEeghGteTgFAURhcL2jVaR4tBiSms6o3NqtV8XBGhwvk6J90YUphiwId4Eq8DClJcJcMll+1eT8SjJw/paRxuFYJUhWGVkJ5ZSH9hdQla1n7t256eFdJ2SWeXQ6L6JtwZd4JvL6/APUfJGTE92bTvE3rGrqPSrdbRJE8ZXLpM5cvAWw8/25uiwow6nrnsiFtXLpNyazuC+Gu4Bf8+11dZvbfqUg56vjcazjUrRiFdzX5I6G6nN6cJ3117j7VG9WT84l+HOeygMvU2LpgVRFPB0KmfVyD40VfdFJTuEf2UkQaUPk9Y9iKOCFOSlljdxtQUGnxgMQLV3Nfcj7+NVJBLjdQUjbvSsTGTQha5EGdWYx5ntergggMXqMPy3R/Yl+x0D4NT9WXiUn6Vn9teEG/p0dPbZ2t4pIwqT6IWpp0CYYRQ9cnfhZAhie2IfrLFmFEZv2/0CGRWDWbL7HjH9i1AKP5N4ow9j9w+kefhkGAn361N5+cgZdu8ycX5OKermX0hPSQeCOsw9QYBGn0icm6u4Hp1DVd4RFr8qx3rtZUpPxxFWM401oxbx1F9SXrj4LvtQBIzjodQX0ZlciPKqZ5sphHLXU6wrf42V0dJ7laKZbukpeBkUKMZEYs6jw/harZB7MBfD+wb8Ovv9ayCO2eJw9lmsci5UzOSCsIXDLU/z5CkdBZ5dGPHrQKr8pPxoCpnCXke/C+upL/bC8sMi5NZ2UCBbMTQbSV7cgz2XA2n2DkMuhwdS5rP1jXnIjDquf3cZc0YOZ4fMo+RyPDMGZqMUHc4+AE1aDEWqDAYcERmiVLJAp8MfKAd+QMEZLMTFr+Sl6R4cPriVxzOnkdF6guR2Pje5KIOEZaCvpNfm7Yy9JAV9dI7uTFWoDOf8VZC/ivk9r3G8eCLlUZkE5GURUTYXs1+gTQd0RAiarCZ7UJT8b/mYLVazIygUOTJZO4cbZoxWaX9pc/a12TgQzFgtYLEFwbc5PXfO2knIyk5UmnPg70E8delwqBdL3AU+Tb1BcaSaSs98ulb+06nroN0VQZDa8lDqMk4VzMavypeHv1/I/U4mBsycz+AI6c7Oqq95fNKTXNFBWhH/CG6L0JxDYQxlQsVmtjqPdHS4dw/kGZ8yMSmbP3iZ651HUqm4zJDGUEYRDNeexx+Qew2hKMqbkLxqxLb9sPwogkly3vnnQYPZQVNpL64xbLVEk5lwG99yL+o96qGojcnnUxr0fvyc/hFenX+3PSAQpLkJhVtwVz6CRTAQfieUuAuVZMb/yZGKViIaBhDqHtqB4thL25fJ90rZ+J6jah99T4bczEGf+DPn5O87LoROg9J9+MpAq9Gxd+JZtE65jrHy7sGP8jTOe16mZuFBGtU38G0YhSA8A3GPQ/VZKDvIK4lprDYtwyR39LWP5QyH7i2msNXXNpYdnU1tpod1V9bx/fXvmZIwBYABpwbQ+VZnjBOMENnx3rbyxBNw+DCEZfyF1UPF/a7T8BYF5HJYJgG/OXKk4zP/JyP7/r8s/8PIPovF0uGf2WymvLycDRs2EBgY+L/iG/9T/lP+UQQB6jSXqHY5yVn5O2SGPkde861/3Ge1Qu+7R5h0/z7JXr3sv5eapYiRE2P3s33a9g6JoKclfkSR/DhVHlcZeLILCbcTOjgK2wTtub8PIWjPt4BNQLryJOwMQFN7nmEmF1qddNTPetj+vVx9luEeT+Am1zH28AgWr+1M19wP+XrMOuSaEAifw7HiJ1lZ4UejBUKrH2KQky004twcWpQBvFyZytYF1RwdfhSLzNLB2Se2g/S31Tl2LCQlwaJFUGcswex5mUqvg2SYG/heLGfV+Q083WsR7/TuTK4PDDs2jIm7J+JZ6ylRmegr4eR4it178s6NQEJKQigJLgGryH3tQPDtx5WIWlTyFtK65lLw3He4NMlIykoiPk7kxSs6jpt3sT3rdY5Xf05Yvgeex3dSc7dGirR3i6dGH8n8oBukqKAl8HN6t8QxOWE1zx+8whN77jH+YhPd8jbw69yP2Dq8F6XawnbzQBqLSv8aSoNL7cplTWsIBrMzjYU76HzrFR5fMwKPOg8EZMiszXC4P2R9JCltRgXqFjWiRZQcG+pgqjofINYjixth7eYcAiS+AJMKSTVtZ42/lT1zjuNeeBOv4qu05x1vM9p56now+OYd5gl7/ubsk8bqethi7nQLIbN2O416HzKrBvJdxUuEFJvIUetp8o3C6Oxup9K5Xj6KWP9u9Ls4BAB3wQcApWAztDoHcMn7UbKr3PCqcqXo5udkhC0lp/madJ9cjzVkJR9ljaLI+2fOez3O3O1zKRc9aAl6lGptqGS0szn7DtV8zwmno1wrH8qtpmAG/DmBwDwl2YNeQ+P+GO5O7lCXgdhaQh91NpfTztLsIeBT5YOAwLJlMHw4/PBRAnHGmXhqe1Lg8x3XxWzK1PMhcDSMOE1G5894T17DnfhCzEpb1JoMyPsZLizG8lcSU57dQq2nlm7XujmU017fc1Pnx8/+kKSUxshO22vzjJjk9aSHL2Fd+YMAdLbM572h79EtsJstqq6N+sORqLlfP3j2WSjyeoc/Nj+Jy71c27hJSvzO7GUczF3C854WwuVQGKnjUs9LiIKITCZRdmX5TmTTjXC7ow8gOXs2Tq6JkrMmZApRmlqMcedRuPkTXvk44drpYDagEFoQEdA0a3hw9VS6n23n7Gu4hcUkw5SRiKIIECxYRJ0jL56xgVCzkrfeeQteWUt4+u6/GU6k+1qdDTTGxeDf8hYzY57nZuVgAlzyaHQ9xtFJ0TR463j9g9eJzYmUnjc2orq7hsNeKlLKg6jxb+FG1xsIVhkqm4+g2CWVwgHnOTHsPk7hUm5Ft0Y3qU6rBZ3VHW1CLuue/hZ5gFyiIrbIkMuxJypPuJ3ApI1RJGZoScpKsis0zloFPS/2pCHExLrH1tHoKSmQtJYjy/8Df+Eki7+OwyJaudTzDI3GWr7+Gp58EmbNcig5Hq1jGVV8jn4aJ4Y1BtHH73u+d53Ep/VQHT4QY9RPdJarqfRbDs7BUHmKVbLz6LvHsn3afn6fvpgW5X17nwq2fXfXlF2sXuagMJOutY/CbsspJDBrFmzbBsEJpdS6nKVBfheP5l54tvR0KHBAqTme+ioPipr+osDzE7JaDzsGsmAL6dEFxEe5UhRahEqnstf96YXf2KD/hLdrwbXJlTvxdyTnkAikrYNe30PXT/jRfI/LvVIpCSlBtNryPxVsIlB1hc9yJ7JHy7+W46VPMSbfkyqzlAx9bX0cCDK49S4AP1xfjVnhRH5EvpSo3tZZS7o+z3MhEuWKPer7b86+Ne5v8XR2Kv2OBxGVfkhCVAMPdHmdFP+jRJeF0qCQ1kNAuZeUZ06mhLDpHE/4hOlFR0i4JWP8X+Np7zwAOFs6mo/dJHSI1j2IgQETGBDTDZ/UKeQITzEi6DQTNNJYpTS+wutRf+LXOJpewX/ySMrDvJQzhINaCClZzmlVBb9lSArbxLgv+HD0UBo7peBenEmr9nGqDaWYLCqeP3SZbP0DuCvrOD9Gz+YZ31Gi2Ws3qtyqlFCd5mpPqvwryIvKo4MDVFSgFVR0ypAiKq2KQFs+XTPOzadJC96Oq9t2fq58iisxU7iq3wTlh6FGimdeKvzB+8PG8OVzv1DnWdqhS0a4P8mgW7dIqn1FqupvRlWV1RMnfShX9HDMxgKD1QJF26l168qrNwLpfDGFh9c/zOJTUTDXCo3ZkPcTl8RQzrSC1jOcSsVDADTJOkPpXrj7JTPyF/P55CsYPZxpdGu0tc2DwsDl7M55gZ/KOpFpcFATyUQZKFyQ3/uCQeEbOhg8BQFwkYzaszjMW14g1wv4VfnT9d5nJPomgKEW/gyjIKKKssAyjHIrf41Zz97W1+ztdVY440EYKmMAfbRTWOcHcXVrCS59nClTwKPpOqt8QeyUw4a5G4mt8+egpgwMNVB7nYcatrNyUA5qrUD5GTW/pr/P7eCT8FcCfWtiWRVYSyclGC0Gwqsf4ZWYTUxJlJRryegjt6+Nk5YPeHzf42RXZyMIEF/yLmOuaUlr+NAxQLYcnXHifc71PcfdhBIGHw5DVmsLXgscCf5D+ca3gKe+W8G8n2Io89pKscEWvWtqRS2T0c2/F/M3vkFKRgqFoYW0OrdKgV4lf4KunDM6DcXJt3BtUqHWqhFFGYROJSdoN8fzH6BA29uWw6rd2VuwmVm+pygffQyDs5mgnFMILeVtkwiUXlgRqPKrwrVBRZ9rL/Cg71f2ptlpPNsh++Qykad6Pcw7Q0YxXHWNYZ2a8GjwJ+bsu9xtlvJc45kKkQtZ2hTEDT3cSL1FdkI22CgJacwhNvMlPo1I5F7nGHt9+QFVGIMnovWcQJPeC3ueOts+5Wr7b73PI4yI+pHlaalUBM+i2OsPTlZdo95/NCils37sg/s4v2IbWd3DaXSxJVfr9BrMauV9w06suZ/S46w7DR4NiLYF+VeOlLMqUQlhhUHE5sRiUphQ6B0o9ValZExOuZHC1O1TUeraxetarWhECxk9rnJg1AGKwooQEJCbq2FvMo2mQKZs1qPbVMab772JU6vt0I59DOKf4bPUT3CWSUEkEiX0P80DFgu00Xgq29F4ri5Zxg8nE4lc9yAzt86ku0rKoePlXEppUxzjcp/kGyfJ0Qd00Fv+jtrCKuIqL4VOr0OX9yDuCQYXu6LTKan0zqTI90fMVjO4xVFnjmO4/3m6qRzvKNeWgMUELcV0ct3JLI8G+ioeAxzyW4hrNo8kv0uaCuo966j2b7TRIzu+K7TmIUbcqCA1/yfp2b8ZqVY1P8NTJ2LQKlwp7fwEU73eRhTBX5NH18BDvKi6S6eyEJx1zpQH5Nv7DdcYnMsPcKLzw0RUPGV7m4iXLAvkaq4rv6XZ4AkIjN03lqEHo6T6bf30192n2Jz5CovLNVyNmoFRJsn+XVMlVgmjRUOzSYOL7XubjC/QJ/A3aZz6L+RqkAeFMW4Elbgz+tQqotyl/N5H7y/EhDM+6iI0Wjme9cH4WrogCKBR1tEjaD8qQxX6hHsAGHRfUum1CcEmN4PkrMqNkWTlPllbkSPJ8lq3UTjJm1kVUImmTW5CxOtKf8YEvERCZD190nNY1JDPbveRHO4SRE5jhv297anl/u1vg6yWatcTfFv8IYeHH+Ze9D27c0yvqKLAbx0Xh13DFGuS8kBb5fZgQrueYHP2iX9D9r0rf4T4fBiySUPKMUmmbdub2oK5Iu5H4FXqyNfd5uxrds4mL+AT0s2b7cbuirR9TE/6mP5hO1AMWUVtoIxRR2YSUTIeV6Xk9PBrHMOrRhOTdg0hPr0eq2j9W8BomxHeQXGcqxrNueJpXC4dT0HzAJR1dSReu41rkyt6J6O099VcgR2BBJjraGrUEF4Qjk9tCs4Gh6I7QljCpz3ewk9M+FdknxUzWlUuDfK7WLHylJDFC2I+zoZwAFwt4Wh0sQhmJ0JrHuLl+J/xbxgPwMmqkayTNXAv5r4tOE3aC7RGD5r0nrjqzuFa/xTJJ74m6cx6qX8F6OJ/DBdlI+mqRKyiGyaZibWPraXesx6sIm8eP8IOSyFPxX5GfvIQLgxsQbDY0He6Styb/qRv+C9cC/TlnZYAzDIJgSlYFXx/7VOe2HeDHFlXGqdOw7mpEueGsg7OPgERTbOGcX8NYtLuScx2/gE3SyQB9ZPwaEmzG/k7FX1Gj9ydxHt0oXvQfoJccxmt+ZR7BDNl5yhUukaQO/ZsKyI/39xAQdEq3OvhXN9zjnEuP4p1f3fyDodwsesmyv3LsYgWrKIB0S0Wc98dnCuexrfjY/C9LckQogiJ7nvJquqHs0LL830WsDzkCu94/YR//UTqLXCywY+SxlgmX1tAYkYyAUcUnGn5niLvn+wpTAAsViu1ebVYT1v51eVX1o5zoPLtCCarxe7sM1mVdPfdxTA1WAUrgsWMUW/B2GrEapHOzldOPcGS3UuQyaDGKxLR25PqS/eQG6S50NmvMz5eUv9U3Sjh5g9XaHAtoyxoH3dqsm0BUya6HvqY48v24X+loMP3tDm52tayxWqGvvDyxo8Y+PrrrIoOZpHSiTfwQaQ/D8f/RlT084RqVzMh7kuOVCeSsHUyAc8EMDF9Ef71E1DI5BAxG+KfpsbizNqla8npdZ8a7xqCFDrkjZchaDweLuFoDEmUJE1jfvbrBLesIbn+9Q5ynbSajNJebhURBRkyGTgZgxl0K4s1vfeyr4eC/d3UpKZCXBzI24KesGBEcvYp25B97Wx+ZvM/aTwVMgV+ihjUumi7ncMeNODkCzGPckMvYFKY0DnpscgstKea/LuzT7CKlOm72t5tZEanDziQJMk3kZnXsRhtTsGWUgY6PwnARhujSWTFs6wf+zP9w/oD8G3eKfw195nm53Cq2edX5V58Ij/kZFIXHE5K23clSqi+vIRxHBh3kRVvr8CksDmwXKJoiXmcHoVQaQa9spQ93QWKvH+xoeDOgK6Sn63JHA4uYMO8DRIIoK1Pxmbw6tkL5JqbuVObae+vKr0UmOOuKEWwynFpdsEzz4DBuIl3Mudxtexqh/Gw/Bs6HVCgRqOPIdY8mdjS5fRUz5QuRC9ib7eDLDgfyqL1w0jIDLfVLUj5h0+OZ3RQZ7o07WTg6QlM2zYN2lKNqLxhyAGMromU5UvUwm3OdVEQcZPlo1Y0YLXPwX939rWtIWNbCimrgMws65BO0fP4Tjof+9L+d+Xm42hfXIFf4VWschl1wZ3/ISf9PUXdf5x9/9+U/2Fn33/Kf8r/H4oowrWomZyPH8w55dvc9/+cMp2DkqnW5Sw5ge9xX/EXTsYgXM0R+Hs5YG7H9H158qch9D80goKIArsxckjE7zyY+gozI8dQFPQCnrWuOOmc7Nfloh6ljdt/64x9XJoYSJnHTmkDCxgBsU/gVrqbL2Zc5NuXv2B71EPs66biUstmSP2Qi9q3OD/oVYZ5GjCLVmQmg/SsR2fot4Hs+mH2gye4Zj7xSsmpg3MwSs/upGWfJfneB4QWhqJp1nQ4mNs2a52ymNvBr3JR+SFOTvDRRzBlCryk3s4F0Z9Bx9S0tjzJ5cY/8fODPiE78dfc55hTC3fGHMarzktK2i2IoPSG3j+hilrAsNYnAMhKykJA4HTVCzDiDFaFJzfKRxCWp2ZrM+ycXcLGORuJiZLxxRfg5iZwsWQSGQ1z8S24ijo3A129DhqywH8424q+Z8ilx9jYBBbRjGApRylrxWKVkeBziWTZcRLP/0pctj9GRQ2iaEUla+GBLq+z1PkmIaWBzNo8m1kHs5mOlAPOahV49K8cFNpChnZPB2DQyUGAgCiYJdqZunRWuJaQrBTYN3YfFQEViCgQFBr0XqP4/fZTvFPbbs4JIlx9Bv4M4458LIdaoNlFS3ZCPqdSpnOj+pL9XpXSFuVoUeKij8NLjPxXZB/Aw35lPGR4iSjPGwhY0FtcaKl3wafcHcFsozBrp9s+17KXEUtbyRj6DB4LJ1DcGEelpR+kvw6nphLbcpL8/ucxyc006O9Q6Pctypa9sFHBgk7P8teA1+jNIZqcs8h33sHGWxup1CQg195kZqf3uRT0IJ/o46jw2A3AEjeI8bxBud6DqyOPkNHXpii1tSdsBtpBB0kugEMjD+NTquDBXx8jSTEWHx945hnJ+dV2/82IR/nJ/VMK1ZPB6kBt3I2/y8kBRynMWEuN4hupzZWnIO9H7mvrSHf+k18XrGfjnI12hB1hMyg3ObHQDYLlbXQ5f8vRKRop8v2BM82/YMXyD+HDp2kogbUz0FgDaVHe59WMiby6czqcm8/JmhM0XL1H4LnDjDg0Aj9b/sRmgxdbs15jyk0/ojZPRuti5PDIww4D2d1vsIoqfo67yea5V/n0uU/5YdEPOCmfw9PJU7oncAQR99w5fSeI4E319Ln+El0aXoPMDxjfouHPUePZWReLa4MLcVk2R55VAdNrGVK5kBVvraAh1cTsjbMJLg6WnH26KtjmRV6MmVudpOAHZcM5/mhZ2GGfcK93Z9KGeASXIMzuowlUB7NmTApP9nyUJ/1mMXr4A5QGVZAfo8RL/4ANRbkA1c038C4OZsq6RwjKaeMfl+GsaIGcb7nv2Z/lRi2ZXcoRFHJKAys4NuyYRNsliNSY+vCYxoPv4wspXVfKwl8XMtWtHqeqrRKyD5nkfG9WcGGwhm3Tt9n3NU2zkrH7x+J5V+y4hhqz8c1YRl93A/ei75PRT86dhDsSjZYvjBplcySKAhWRvXBtdsK3pRduzt7QWgaXH8dJMBOfHU90tgKVIOKuqkZmaYSWInCJwpTyBSVRg7mTcM9meBEQrEaoOM7etPFsDIDxe8bT92xf/u5g+jvlVNscUangQMlGziX0547PJ/TPvsCwvIuoFQ5Km4/re7N0/XB6721k2m8tXG+V8o9SeQoqT2I1xJFw+32e/+x5+p/p34HWOccygAs6SfGWaIoFfCynoO4G+PYHUYFFUNL5wi0W/LIA0aKUxnl6HTuqd/3ju2d3miNRvh0biUwwcVzrTLMVZpXDNw3xUvBGwnM0df6N6+WjkOlbuZl8k5y4HLsR9dj9BTSZVVKOpXa5ZduvyQYS6TdBymOlbqy0//715W/5+vK3GJGx9rG1nB0zgJrgLh0R1ba+dW9wx7PeH41OCmOemfQ+P04Mw2BSs8m1inff+oDbAx7uOC4izD75LSNKwIEkkq41GbzIqhlKc9UkRtS9y91B7zDBFIBMMPLekGFMSvicH9JXUxPalfSUTO6FH0Znljylcd6XmO+VSv+wLYRnWoi7G4dgczykBhwm0FUyEh45NIXRG+bYqL1tFd//DfI3ESvTUhdQQY2vEyaVRmqzRUdo9kCWD5zOc6nTyW6VkEWCIEg5RoDz7ju4Z4Tkm8m8+OES4u/Edwhc8pD746rrhG9rf+JL3qOH5YkOfZKqfZHhNwsZkZXLt2d3keP8JJQdhLlWsuLe5Fq3a+wdewwXrQuhTe7Seqq7DtmrOSeLRNw9nLFbG2lu8WDCRisXvW9C/83QRzI+W12dKBy+kMs9pWAUQe5KrecUzpQsYEXeAM7qwCLanH2CDCwmjN1/YOftF2lv8BQEYPQVGPgnncmly+UeuNfISB/5ooO2UubIR3huwB02PFzP3fi7HWSovqF9eUlZQO+cQ6jur+a39Peo830CgifC/q4IgsiYExH0WPUkCbcTHI79jOVwoBsqqwE5sGf8VnYN+42vSpLROXcF5xAUlhoe86kjUuHID9UxL5DDKGQVjJhtFOAKmYQ0kVlVUo4Ni5wq1yM8eLofp1SpAETL6lnaXUudn5xu17vh1tAuUt63DxniSlo9UlC3yBlybIjjuw/3RamvxHvgBQJKpPou9rqIQWkAdRDMMcPIczxcG0FNqS8R933xbOyBrzzCNtck5MyXrho+U/h2RPYN2IbSycDC6ac4NvUqXz3xFTt6DpTqVXmDuZWJGvBo6kVxz2RqQ1I6OJvthqx2Bm2F3LHYzwrbeXVPFWpdGGahinTZd9KF0MnQ52eqrEom7J6ARuvMpV6XHG0+NweKd+FUvZP7iVG8/9r7rFi+ggd9s8gL/5jlh9M4uH8E4vcbSfzWCfGkCE2QbvWGkReoCvuczKoBfFgRymXlX9wLXMkDZx9huSEWklcA8HkdXBFucbJzEvs0sxxjYbVgllVwqfdpzvU9h1Ork33POpE/n4l/lbE8/ylu9vDnp4d+QjSLyKx6EFXQeTnR9WfZYgpgyq4pdLnZha5530lz6P7vcOs9jnsf4vGht7jQ54KtQgFBsILViCBY0Yp1cK8GAEt7Xh91EDdbXchOvskXT6/h6ye+BgR7XjKzTEud5gKV8it2JI+83VhcaOrLqgbJKlQe1YeuqukAzOn8Np+P7k4+5/G5X8jbb79N12td7frU6pE9+SA0nF8c6a4BAT/ZFch8X9prrBaedj1F3/N9ibsrUfw7y52h22qONn3Hym4fsSniaSJqHNT0tJbArlDK9V2Yva2JVPkc25slQ7qvpoAZ8evoogLPOndGHBhIUvYYXBWejvlnUaMy+SG3ONZS+7Mm25zEzZxgwu+7Y1Y4IQhSkNjzfRbwZM9HecHQjW3DD7HmyTVYxHaBmS3FyO58Ri/zMXyahkrvtYpEKvdD9qcgqvDX5POCxUpQWRBGpSTPpnQR0WjA7DOE3zLe46hxL2Ve24iue4xPRnxCkm8SAE6yRu43h1Jqhm/94Bvfy8yNf4YknzMkel+jtW4MEXVfUdBpDI3eDr2kUe9DsPwkov9PFMfG0xDxGIOMKxFFyKgYRkVzBB61Z9kQZqCi92Wy49M5OXwB8nNToXQ/lPzF6tafeWh4Ln/OccWq7u9Aqgmw+86zTCiIoLmNVQRByt8KhFU9zfIeb+JqCadVrEKvLOtAaWfPEWqVEV32ClFNDnkWoFp1iQvxQ2gtaWLEkRGEFoU6aFht/7045CrRW6Lxcx+Ka2tyO2Tf/z2NZ3LEw0Sc+wqfUgUeFU629jhQbgIi8/6YR9Klo8y9bcCy3IKv2rcDYsdqtdjXzflGyWF/tsGfi8kZVETJ6X7Jl563vyXULbxju0K7UhWosveXPcCs7b1YGKRp5ngwlCp7seP2SwAsSZrLGGUnbvXqatfXAZC7gE8fmmUupKem0+IiY+zBCURVPC/1x80VhFq3E+iaYw/6tNrzpUp1tlLP8eRY/gyLByy4i8F4COF2hpRxTTsZcusu3k2D7WMP8FTPJbgrGnm/RkZgWTgPbvyFiVcOo7CxE7x9cj+ft8SS4yMZzotioqkzFyMIcLFkEsuPH+Dr2WdoifqOwi4zmbNxLp61nnY9QEkN/VVNJN63MPZPf5xbnaV5U3eDiMLJvNr3QVYFVNNMBRahzbGroKY1hOvmBvZ1c+VCy1P4FN1A1T7HJtKeom6R5PI6/wTUose/6vNezQMJqJ+Mt5Mfbw6cxDtDRlFliuGQ3kxZUDn3uk3HFBiKIECK/xE+GDoYz/yfudX3D35Y9APNLs0OI/6tdxHqrpFoLuFayq80uzQz/4/5PFj2GewMQiz4ReqnhiQMyijbd1ro4/MNNS3B9m/74NIpKk3daZNp9RYZwW45vBd7iD0T9lDco56El0qoE96VnGm2dXGkSwi9Knohd5VTeaPSjsqDdmumHY2nYJVxs3YiV/UiOmcdV0dMw23iYD4L+YxxO6Ptz264uQFRtJIdNxwxIpTMl3/HpU6ix3WSO9nrLz0tBRWYjMe5GDaDLZkS+ltUyKgK7Uq3p/uRPU5KndIWINnm5LIIBqxWCGqYQlT5i/SJH4gwSCT+sRIyPlbx4IMLaJn9GlnDLbQo8ymJ3MLH5zYDIpfTJJrUbrvCSMvdbafExFBPlLKFxsAKzky9THFoMaGqFhS1JyFgOIKTF1gsKLX1NFbrQRDszFqSvtjmCDLR5+4xxl81Y8qYZmd8cNUlEqKOta+Zd9+V/tvDrz+JRZ8Q2TzXjnqWCVJ7HbTzJnuAdvt+AAhTpNLidI9qzUn7t0gT+Trkfou/7RxX6yNx13bHydyOdvJfkH1N5hCYmMdh434K6pMxuWu5O+w4dT4uXGv5i6ulV+HCgwBsy36ST+ulJ72bhjA/eSGx3rFts4hGvQ8tZqe2TpJKxluYZWoWVsDo/f1Y8OUwwgrCHN/deTkzWyaSaYDIu2qSMpMctgCPThRFPc14DVhjoaeqXf8LwNWn4OLijsFG7fvEJYpasZqrycnsyZdyzwtWkVpDFMQ+RrMYh2CVcb2bpF+FFHp16Ke2eWjFghXrPxxbbX/7m3sQX7qCPq5zOvS1RbQgN8oYcahXO5CJBQQFQW6hxDWPpMd5Gcm3kkEQ7EHYtJZT3287s6nk4IQ9lAZLnS4TRMSxV9ic+SZq0WQbSwsHUz05kOqBTl5ud/a1tcFoszmeGnSKr576CoWXY92bndQYnN3tf3tHuSOGBVMTlIwl0rHG/1P+/1f+WzSezz///H/7hZ9++un/64/5T/lP+a+KKIokJyfbcz1AO2OvLWKyPR9yjctJ7gS/CdolxDIeQYDQUFi6FDw84NKtZgyNbqgr/WyRC9Iu/OP1T/B2Lma6zyVO7h1Ls6uBwhhXXJUSh8vacYkENN1nKZDZ6RKZXMJJH8psRTKUHYC0r2kqv8SMyzvxLAzA3eREUbRBenvwOO4Z9Gwr+Zp7Lg34GmTIWnOpbY3GXyEpAAODv2dhcD5zqqQDw35Y9PgSU8klgjTZKE9fo+eBxewZf6DDofViwneovvkSnbKIcwkDqLAEAa/YryfIivn6qyfoVOqLX60XX81+FPOEjzmsqmXwYCuLPvPG0ukG4S51pMSU8ZT3PDB9BlEPEpmzlgV9AnleqCIrPJOgum5oFHVwZCrBpkHA23jVeBF9N4VGXystmhZpcyk/gocYikHmywXrb0TJpQiu6enTWSuOITXna6aGHGJVyUyaraCwwpvN46nLfp4Yryt8NHwgd4tS+CLjPcIK73K30he5XMTDqYKZnT4A4CNtLCEFJijYhHXkW1itsKjrMsbHfQ1m6D35JO9p9ZT4VeJpFbHI3GFqOTRkMbtgI3f2LEUXdY+78XeRtVGtCHCieBJHvRyk04IggE6KstZQw69NkGQQudg3g/ywfGSCyIoVUFMDr1+XQyuY20X4RETA+fO2+dwuMvUvLSzzhFf6zcBXU8SpxrFoT/Wg87HBwAc0TLiKT/1E5ifnMzXxE6gCv0B/DGoP8hp68djeO0yeDEOKEqDxDgOAzB7Q//Vt5Fmkg9kgcwO/ceRn9GRt0XXOChcBiyMSX5CBIMNildGqKKbGmkOATPJ0PusJUMvGrt9zvZOZx88MwjPzZ+oiXGkxjkGtcMHq25dbtgDfxqj+WLTQR+VA0ubV5ZGjvEq9i4OiJK16DpxPhNDphDSV85YXHEv3JOKMhWvTtyGTPQ5p30D3z/ni8OtsPL0Jj2YX8iPzoc6RW+CKdjKa3LPorDBBLMencThv93aiT4xk/JDjMPJaBD31VJBdrSPYNRhBcCWhRELnBHSCPPk1ztXswc/qA63VuFlHs2FuBmkZ4+l3rh8lweWIImgUdZitcjJqXVhwIxWtc51jjgBkvEGMMoRm12YqaGT0vtEkZCdwfpIamb4Yrn8BQePwrPfkkfULgQKckqtBDAfPFIoUC8ksqqIo+DInRp+n1kuimWszBgcIyfjXTcBHG4AmR8G1btekvKIGaczkCjPbZmxDPns6m8yz8TPF2ftAJsowyU1U+WvRqN1sfdRov+6CkXitG6c8valOeAI3qy0YNWAkZpU/Qy9todPww/S4NpCQWw9zcpyIRlkPl5cS4TUShUGBk1aGU5gfxrJwkopW46eWIqJbrMHcut8bQ3ku6lQ1v1l/Y0tQMeq891EqZyBYZWR0uUlVRCghlY/QK3EgXQOkSMImLz0/L/yZHtqZ9D7fm7upNvSKRzLVPX5gz/3FFM7eQlLl29Iaa2fEX3t5LcdjfmLK9Un4FJloECeCZxf79dfctHhcmoMuD6rCjvOS8hxvvlCE9/GRYDFjHZNDfZOBkMJAaryNgIBoboSjkrGusDGeHlclepdj/Q9C/iYIGIYg+BJbthyjrI7C8A+oE3M6IP8cysE/czUAxMiLaO11i8biEPIi8hEE2/q5/iK0lrEqsxD5qV8BqAoAtVyNgIUnez6CRR5Comsvjg47ikqvQrCG4WbNhNx1ED6TVivcrltAp/yxmGVRxJaNtzlFlChkzXzb9Ru2WuCLeqnKucnzQVcINZdwkjcRUjOfvIDVqEUzvjK9RJMYMJTLdde4GDuWiTtTmXhxIhkpGQiCwMMPg5glp597Ef71kCu0UcwKHZweB+VZ/HYGCh78llrXk6gaA4HxZFYNxFneRHelgQvORm7FrsVTe4MQYSnotPBXAiMNtbztBZb8CBr8EulcJCFMWoxu1LYGYrZFt1qx0Oiczv6SDKxB8fQsXs8Qyz0+N71CixXcBEsHw0dW1QC+Tt+MynifHq5SsIMzFUxJWE1uXXeq9J48rfwUodvnGGW1mGRNKOQCctFA14CDZOmS2e28i5Sc7ijrYinvKhIfD16lF5ic8BmkrCTHowtwmZT0FBrDbH1yfRnoqvjJ2cqH04L5RLsfjS6HUGEtCApqIj7n2slT7HDeYe8/URBBHQyucTQpeqAAFEYFRrlRykv1N+QpSDm5YmteJ7WD0d1x3dfizR8jJkMrkHkNgsfi3nyHB/31bOuRwds9T6GNBnYGwaR8GHWJOydXYVRU4NJkJenESs5MGIMgpIHCDaqlg/CVP/ahKr+Lc5iKJrcmFHIFycmxNorANmRVO2Rf+RGUVxYQ4rbebjypcN/NwxnLWaxL4B3dX5xUjOHQsRji6l3JHNKuMQpXmG1k82aB4RkyCnzWt82+f23z0IhfeSDlDagCul6Agk2YFR5kmSz0rvZh1KGRDH1uBxO1Aji9CF3e46eKZv76/SA9M7qwc8pOujVUIjPXwbAjnDvWwqOH+5HvfIPpggErZiyA1Soi2BT3NspkKyY7nZNC7EgrZ7GAXlFOeu053vMYwaEHrCRs7UG4i4K97k+x8pVeyEWb06JoBxRsoVj2EblR4cQXXSU2J5aatjkQNB4sOkQRcntMIfrKr8zYNoPPnv2sQ58M851PqayWu9360rVkBaMHAGWHCa3az4bRm9EpW4i9Tzt6Kjk4+fFqzYPsbf4ZvbORaudqlGYHyp2w6ZTkqBhy4wXc6k7S6tGR6jzZL5m0nN3ILK54aHtgxUL04xpOFsymQeeHNm404Rl7cKsoZOUra9gf5AIH+8Co81B7lZWaCk7W9kOnt1l8rLaggpilmJvukuO8AM+qary9vCkPLEc4B13e7cRguYJVeh0BQHm9jvWVlzlzBZLmXIKEH8HzW7Kr+3K5MpwKpyJcbDJUiLkGfPuy0fAw924dZfquETS7NHN+fJFUf80VuPsVeYmneMQL/F6aTmhRCLwhrXWLBRJCAkgwfkndteOE3FGz7tF14BoNsyX6QLeDowjIDycLuNPrASxKW67e3HVQf5O78slU5Cbx8lortxNvU54gA5UvjM/m4vLrNPcOwHD4RUrCqzApbV6XxhxwDmbR/U6UOp8FrxrbOhB48004fRoqxLssudwHtSmYtlBvRTtkn8xq4X5SNp8v20//7LdItiFyLpeOR2dyIdL5CF53JLra/ldfpbNnGpuBzKoBxHlfpsomJjvrI1AZ/akXe0jVZL4PAcOY6HaRnX5+tKht+apk0pjq8eGjM5vpOjgOi/C5NG98uoHcFeKf496RQRgtTpgtjrx2ggCZVQOZv/cqO+O7M2/bdIJL3XCvf5aU5/qym46lySmLi3EjUVsCWCFcsf/uLjZTP/ACO0J9iCqTAjAeewxyDy3GElrLjfTDNNaWM2fHVE4NPIUQYOuvXWHg1Y3b3tuxYkMEyBQUmsbQl5cJNO0gxN2Np3zq+CC8iFvREpojIkLgs43w009QdK8aP5mEXghvnMuyvg4Dbbj6DDGaczjlwyPucEvvzr5b25gY/zlO8ha+vrEBv4LjCBazg4YTifr7VuhtNvu8xcPpb6A038c68DH79T9uruD5Pgsxm+Qc63UBi2szTjIj1pYCKQhSkOPskQLbRtAzI4XSWIuEAAai8gbzRE8Lg6rdMNpsh4IgQNJLUoDInc8pa4qxGR6t9rFqK226vZsplsSSlXg6fLLSdZusp9VouZR2ifKAcjzo6OyzWC08mPogtzc+SFat42x35OyzpVywyjvII6PLn8HbXcYxhpHRr8z2jMPZByK/LPyFvrd+Ry7v6Ci0p2vAEbAwNXsQPe5t53rEAkp8fqNPohyzbBEGjec/Uxc0V5OY18ygmEx6dNH8471WwYKb3EwnJaTTwhdfgNf5JDyE2+wzx1DrvJGl3y7l1wW7EUMA9wTo8zMLdgbT4APnwsx4NcfYxgNwDqaIPsy7OZjQiF+AhXYnZVsft6dUlfb7v6Eu27XfLLags1ixoGJQ+B/cqR/GvF9vEnN1C3AfmUc3KScaEqXvOUMkF5IPcStKToPmGmlNs4kRN1LVFMaY2LWId08g179B1PUdQDSR9yNBJpLoc5YkYS+dq1YSGPMF6QoDRoVRcvZ5JFMc9htnD6SzxVPKU2qlLWBPga+6kD0TBvFdhZKvFBVsn5lFrc87iOIGezvkgooGdx0HpsoIaBxHRMdhcrB3/AuiRyEYWKwsh0fWgfU7XCzBTKcQD6cKkv1PktwC6RFwYltX0i6ncXWArQO7vAPGRty/n8CcDXOIuSeNU2zIXWgtQ375QRJ9Ylh0dwil5veRxV0DRQsPFw1ktsWJq2WjuF42ijt1A3Cx5YxUCrC2LJEE0ROV2ILRs4Gq+Ca0teW0qnUdZF4BEQTotq4bydHJHdpkz9lnMZPgk0Bdkx61VUOr2YsamRwwYBEM6HSQND2JY5m37c+2mlopN+Xg6hqPGBlO8NLxNFZIAW9XSq8QK68E/AgcFEfh/iy6X3IlMz4K5XAlO27vYH/nx/CM6Mfy13bw1YoDHebkopRHaboyEZdWCYkVWrsQfwP0CIRxu3rxmid4+DRw3UNPrvgh1W5H6dr0BybNXMqaWrg8/GM2aHXcHjqI8KxnoMgWaH1iApT+xaEIeHzHBzQEV1DglUVmsy9NA9Jx9Q3kRuEVCrx2Me13BceXBGCKmYyzuwZBkJLPC1YFVsH8j7x87fcas+VvPIlAF5/uRFd0x8sLatnYof/9NH4sCv2YS2fUWL3+6ez78uKX7Gta+a9jhzoMoheTV/IbYCax5COcDWG4uv7zXt+G0XQpWA9WgcSeB+DAPFTiNxzOW8yr3stoGXCS8PhOFJQuo/j8HDZMO8R338GO9Ebo1sbW0NH5FaS+zrt3p/Kbz+sdGyx3wajwwmwtJ/WGNI6R9yMdts7WMtxEE2Zg+MEgYCarX/vd/nhs9X7elsypODengeKygz0g+W2ovcrbN78DjWR7k8bA9m5tEZsG92SvFsbbMzsJ6K2eED6b5sxEibXLCjkjVGS5FnXoJ5WoIbrsJYKDZNS5nOVi2Es8sqcz302QAtG08iJuB39Do9wdN15x9MeNVxmR/QV9ElrZOvsUD/0wivh7Y/Bx9pP0pR5fwqWlxMkSKAHO9DuD0JZ2I/NDyFqJa+hMbipaGVjngYfoBBHSd8kLf2HNmGXMvfwMEv+YBaO83tYywU5t3LaGTJZ/t0UA1PcaRaUt/jYwEHyTfLEUluBNCWy/SaJ7IOa4Rzo88x8kn1T+zYfxv7X+/85N169f/2/9u3Hjxv/iz/1P+T+5GAwOuhBB4B/0Lx0dgW0CvomcwPe56v0ihQ2FjBsnRV9O1RTx9hMHaXFx5ukvn7YrExXaKF46cg63u4HkhxRj1HSj751TJHhJimp+fRe0gneHegVEejm9DgUb4d5PmNwTuVTsycKfFjJqW2cA5KKk4VhFFStuz2erdwknhmRysPdIItaESPzZFx8m1WcXiU46VAKUe+6iwOhQLlUn+zC/8+v4lOYAENn4Jyq5I4LHW+WP2hAuRX/zTyPWL/qR5Aw8y6kBpyidKxkHZS33GD1GwMlZRLSKmGVmcmJz2NC5jj6Bm6HZRuSe9TEpmm9IuWxk0p+TEBDp5vkb5XdP8OP3K7h2MhbrD+WUHcrBfCITl1IXnDHDsRF0dV7Fs70e5qPIZzk2cAt7Z6VzoeQCuaoYEFX4OuUQhz9uIsxwgfXuG0nwOceyPlLC+o9aXMlMOEL83Tj8Kv2QiQKNeh/+zH6WX1qTyYzJITtZSXlkb/vBkl3dB51JTa5Hf4aUmznR/yxVCTmMCDyJs+mudJNrHFO14zg2/Aw13jV41yZLil9dBkGnVMyOW9uh/0REsHHCq62SQWTSphEMPN7dPv+6dYMRIyBFPY6o8ueRmdVkB73BZes6pk+H2bPh00870tCcbIWzsklUey7lYvksevreYlW/PCoStOSH5/N44m1CSh+mpCmOq2WjYVI+G7K/x7V0P623N/LJqFTSFC/B6GuScwwozgnB61IMcr3UIXXyKBi4i7PlD7H6/mCOtdqij2zGOY+qo1idgvjj5gr7urLaEBV/NILRrORYdSK/N4JXmYIulwooEt6guqUajI3IGm4Ro4BJuyahaK6jMrKX45DP/Q63owPJ9ZjJPf+PpX7TgIgR3DvBzeUEl2xkUXUoMXlRbJq1iaLQGkkIljmBwg1RlDH4xGAe/OVBCdGKRJfJn1G86neUFqsUt1inuoFHSw+WdH6WkdESD3x7Z59Z1LFbMZ/ErxM5nHf4H0bUNsquG2Y1zNLRScjj556lXO1VxOfPfE5edD4yGczp/A4/TQpDFdbAyldWozDAtG3THGtu8AFUpkYO+Iv4VCpIup1Eq3MrDc7XEfTFcHsVlPzFSC8teanpNEzqQmlEECZZI4RO4abmZ9ac3UG27DyDh/+Ip+FB0nL2MMzrISg7xIbwN3i3rDtdkibx7vJ3uRt/VxI4XaJhSimeuW25odryELYzzAkiWhctZ4bmYay5QIP1fW7WXAS3eOi8nImVU7n1yvOEH21yPCMD4h7H0uMr6j3rOdv/LFVBLRICBAGT6A1Dj+Ajs/LBjeks/q4nuqIq1PI+JOU+gIeTJ5h1WJGx/9KDlKw/jEUpcCXtCk8UxWLo+qMtStq2bwsmwksGsL7HZl7pLwUsmJRW8iPzcSkXGX1wNM6ttrDJmiu4311NrFIyPgzfamTMvjEd2lzSVEKd+jInRlSwZ24Itz0/o7ixGGbpYbYJ1/yfGTg3B4NSxLvoKgO6PE5kbn+JZjb2UVK/60Tc6TdY8Ns0Um+kAgKiUg09vmJN1nFOn8pGH6JCp9Ixz7NEQo/UZyAIEFA/idCaB3GxSo66f8uz+l/Rfgx3PsOHw7I5+PLn7JqyC0Fsp7R0/xxBAL3ag0afSMJ0R4j36oQomBkZ/QOjFSs4E+bKlD0vMWvLXNxb0ihSPAhTq8CnH/Lc71hhozPKSxlLUN0Mac1VncNbkU2qex4xjqA+1ApniF0KM+rp5ruTO/1XsVzYS3MMHAs5JEWKAp3vLOavzvvJTLqJ0c2DMWeO4K70YuJEGP/Mw7ykf4BcYzu66b/ReLaVOpfzlHj/QbHuFk5yiZYk3OMmO71LWXClD8mXG8nxfwWTqAVBBh6S83amK2Sk5NPo64g0/CvnKZYdvkivgKNca4pl+Yo3CL25k7fSF/DzjZ/B1IKcZjqLrjyh+47kgnUc9B3NlGtKSrwkBbuT91G+HNMVo+86XqyS3tst8AA/3/iYjy6vQacsYtmHC5i6TaKsEkUBhahjetLHFBkVHFAd5nrXa/ZI+xEjwG/AM9SHvo01+1O6Jh6hRaNDbpJjF837bYEB22iMeJfaw5OYtjGOJvl2Ca0jU9Ic/AwfX/6KdQ2OfhMQIHUljLrIyOowNgZKVLoKk4IWdUsHJ3iO7ix3gt6mSLPrX+df27jEe511/Jj6IZQfI7x0I1+4Kols0KAwtJsoFiOoQ3jCcJqYiSc51+cc2sQbnJ7Uk04N8ySZwncAeQlraa3aRkj2MaLvRQDgbChB+CuWxcmPsi7+ABM0IDe70bnlSZZ0XQLOQZjDF1LSFGffa43yOkr1uSxoPQFAaPALVCa8R3WfBykTx3IxIpGcGklmQpSDKMM3/xLTfi5l+OHhHZyf7dtcoJMCZWp8nwKfXjAxl8aAcVRGFHJw1AE867xINdlujnwAOr8OogKXZhf8K/x5Qq3g+LAldMm3UUXK1FQZXdBbpZx91yPnM+6inC8ufmGvty23i0Uw2nPItiH7qtwOcyNiIbddvrYbdRbL8iDzA254HWXlB6vofvAoeic9FpnNaFR/Cwo3k6vdR7H3r/yx8Do/LP7BsTemvEuLJo7bN5ZxO/Yo17raEGEKmUTjWX4USvay3KOCtKuehN4rdnRU9QUC6j8jvymOAy2SPBFS8wCLIleQGpAKLcX0Fu/wyI+L6HdMYqIQTTa5x9AASk/uaYcgM+lwy7lGo7CG002/2l/vo/HGv2ECPk2DkVtcUFjckMtkrDq3kfXXvsDFfA53n2Ka3ZUIVoHRLs1QcwEKtsDRoUxVNVD11E/UeNcy4NQAxz4Ts4SLPpPpsjmZ7idOsvTbpYx6eySeRwROmc0c1OuYBwwD5gEnzEbOWuGDrQKfffEdYbdS2maKbaxMpKrgpcYNcKA7c5Tr6XU7icDyQGJzY5EbbTG1+b/D/V84p/Ug1whXelzhbtxdRAS++AJWT53Nh93diHI5iV7tTk5MDlqNVlqvx0bBBoG7fhMZ55fFltlX0XoEOyZrrx9h+CkKvXZSvHMgzq0qwozP4tc4xm5MHODzPedCYf+in/lz0p+OeV+8A05NIkqpx7nFmQGnBhBxPwIREXd3GD8eXDQOFG1CyUoSi1bhrXY4br+NeQpdDCw1TqPB+Qa1lgIEAS6VTORG+XD+Cr+JYs4eTo6NpiB5ot1Z8uP11XxVa6FX91bGXtUz7OZ9+mdfwKhwIFOoOElf9as4N9bhW+ULVlGiUcxZR6rzGs4UzaSeVDv6YGzUFFB5QfdPqTJ2JjD0W26atttGTKq3Z9Aefh3bgzEaaRx0fqFUhffocA7VaS5xM+wJ8gJWoVOW0Cov7XB9tesL3IjXM+SID5FXVhBq6Ipz9oskT3oI524vIAoiTjonNFoNVsHq2Hfjn8Ea8xjbKg5T4rUBtS4GtS6SRiQYZYBpN3eqe7Op5RQei1dRFCtRLLZRQw5VTmfD1ACO2Luoo4H4cNVK1t55EFcRPO/B4qo+ZNcNZWf2Mlae3oYggEfFHZxqDnI0OZL8RglB88GwIcxxO4XWCpf7NlKcOFzqMwE8nMppNnihde3ED5cjmfPFU0Tfief/Yu+vo6S6urVv+Ld3ebu700Yb0Lg17hAsECQhQpI7rsSNeIgRI0KEGIQEC+7u1li7u7tUdfn3x66u6oac85zzfO/znneM515jMOjuXXvXsr3WXHNe1zWb9K5YIu+BgZ9D6mq6Rp8i63Q0gQWnKPZ9x36+MClDaNCG9lILsINzXSJpc57EqhYDJzTPohPrbO3tAUq5KZhys73QHbBTGBXE5sXi0+Bjlz3tvveeTxfxpvAmQoG0F3QH1maGLyax7HMS2p5m60At4zLyez1f3XGGgWMuUzbrYRoDOu1j0V0PwSqjIrSChuCwW5kc9no7mH2CVSTK8wqxmjaGnR1G/4M69M5eWGSOPIJt6gw2y+dyOfUCNX1G4tqVgKfoYP1F1D3O7c6f46btx438Vxh1aiWR1mKiGh/FIzSehpAlzCgvoCIwndy4ckRZjzxwMg3V7kPQNU+iwzOa2qjhjmvR9/Ot4RVO+r3PKf0arFbwaR/H/Oi7iPGSGDm9gn2Y2WdZwT7LCoxi2y1jkx65hGknXSj3WcfiLc1UdKZw7/CXafOOoLj/HDqCJBCir1MZq6cMZIFK2l96Srx3l6HBOxAqt+Nded3+t3rfegRk/Gvg4yQIH3NA+RhVuc1EFMgxKo0SqFYTSLv3nfxy/W222pT/h+TvYcdAC0FNi3h4kJQA9UajM/M3z8G/sok6zwO9Aq/Thc+ZfKMNmXwE16Ne5ILh597MP9s413r+TYXX7zTp67hrWzXbsp+l3JQKQEB1AKNOjcC1RTq3HC9dyqLNTex0HsfeDoHQ8lD6FPaBbhlPvzQInokoiKhsgJUvHv+CETFnqUqRwDih7lmYBR0thka6lJU0ai5glhUwIfJXjGYVh4ruJd3raT5pG0i15xYCZLAt4Qhh7lmM9irhd3cV8pBG/lr4F40+zYhij2CfVYZzhzMnKk7wcubLkp1sK8NDh5P/eD777tzHD7f9wM8jLuLfOYKR/msZowHBIhBYnIX2RiEzvp1JVcwIAprn2e/f0/QZ/vmnMKRnsL1iIHKZA3ipVZYAEDQhnrh7htHpbKLDpQOVTIXJYkKvqMMgb0TX3IXVIDE0u0GTdybdQ3TNS7jo47BYsOdr6x7LjR0wrVjBRfmvhJS64dnkyXCvDFxbtiEKZjI7gigxgSHCgCy2j+NeSxeofDnV8gp+W43EfuFJTEEaVn0Ucl0RnJxLbeV2ckM/AqA9p4aLEYnsdJ5t70+l2RNPtaedCdldBAFMYie5Qa+xJqc78HUrIM9iARVuKI1+qGVS4N9L48WCoOeIqH8UqxWCmxbTv2Qd8/vOB+BM+RluLvZ3yjUGtFUMVUv7h/UmNRNwrKsCIgqzOwqLGxpFBxiaCBV3AuDW7MayX5bRJ8cmEyoIYNYTq/iZLyeN4RsJ50uzy2n2FOyisq0SgMUhC3g4pmeuXlu9Ep6jLPx5ohTwzSPr2bbkFJcHXnYAqY9M4Af1HjQCaJ0kG1lptSXUrTuFmP40AJvyPyBT525/uigCIbMhYBL9Za0Ma17eo49tz94RCcABLT2KQLjwJxwag7ciC8Eqw63NDa2HldoACeTtCPY50bdyFZPE9yRpac1Zrtc61qsueQ2FgR9wTvEe7eosmkw2e7rlOnKLjs98ocmrjStTXsDiu4oI9yhbFWRoLWaqfS6xc34754eel3pMAHyk9Vtd/he/eyoZfWo0oaUetnoJCF2VXKycgbz6ES7fUU9yaU+/pngLs8+KFVelK2GlYQy6OAizzuGb6P6sry+88Qao3HvneNc7ef2HbMb/6Pf/m0rPGMb/2+W/xOw7evTo/+l6/Lv8u/ynxWKxkJubS3JyMrLWa7gcncF9nha+0jk+09OJZc99gpkKn7XoVGXUdd5BmLt0cJILMtTyLtoDBZrkRQi2AJ5K1okoWAAZx8YeY+rpB1F2NgMSlPDdk38z46GzwAjHdyHy+fl1jJ4SCnXHCI1/muVXdwKHyE+Q4CFyUQ4HRjDbzYXvrfNxa3PDpUOBUWGUjOfOUij8gcv1v5BWXk2rczr4reGgrgbYDIU/oVP48IXsMCVzlMR1fEaX2u8f+8qRMLt3sO+AeSgb4t+GeHAR4J2ut3j1zseg7iRoK/hIIVC16kXODznPS/57uVc2kliFG2yPwOA9kt/yZxNekEmXOpJLo1s4duZXbt8OY+VyPjQUSEhoGvixsoWCtWYumjPhhW/JPxoH5rV0WqC4Twk+FlfJlpEpYdQmru1Yz+4xT/Be4WS26w/QYHHG36WYv7JeoVNeyU/ur+CWVEZ2HwNtbi3IZSI6kys/XFlNUJ+P8Wo9xpnRpZhlmfSnEA8e4XjpUp4adg8qsxbVoVHMavakdsFO3kn9gLJOJyiJBrdY3hq+iy8+Pk5w3jHcFQ9h8gYULui9pzA+dDvHlDCxEsKrnyfadzgMvAf6vUfrwad5ybOAjH45ROdGElgVaM+FBjDS5S5qKu6iyWc3BREPoLMOQql8iKVLpetRhjkIzX0o9P8Iq2jimPJOXpkrSR9Rc4iL105QkTqRnbFjSBc8ODb8Bkf/SOJoyTJi+z2LdquOuBp/DOou4u+/RpXFB+ROEPMwm9qi+fXTLxh0eSCVXmcpj3Ec0LplZ6QJ4mD2qTsLUdX8iUbxqj3Y1+1Q3NgustTNwOnWeFp/HUBSixu/L91AdWCl9M4V/oQm/WnyI+D5gmjMmlK64qrRWZ0Ad7AYUViNWHHIQfTvthHinoSEFylsyuetjd8Sfy2FE8nXGOChJ7A9FzqUYGhilP4Gh0aUQWYSd/12F+en2Iw3lwhMdQKDVJBvBJOgtbezu8hEudQmwYJF1PViM3YbHlasmC1Wu7SgXKYEmQojSnRGGSaZkRb3FmQWDePGwbpTYzBbvyYrspEXDx2ns+o6Ls2e+CiHSA/0GUKHOpiGU74sWS9p1Dtrnek0rqBRuR//24qh+QrrYj7hkbu3cbk1gRPMI7BrLHBUciJYlBjqNajKO/Gv9cCj70w+fhHYNxyVRUfVn+BacwFs7BWZTJCc2ZpA2s0yZuyZQZCphtDUUEyRjpNril8q8RXvE1Xth192Odo+OznbJOf2uTnSB44t4uywswTpRhKY8R0V8TEIgsRgk3Xks23AI3y6PZkLk89R4PILcZVvYxVVEDABnwsPMTZeYMd+Jbr8SnypxBRtRBQngKmTkc4vUjCiHzsiRxM+NBrOwvshxSiq/0QmppDiWk5dazgxWSEkH/2SEytiiTpgk+ASACuUDdZxOHY9WlfbIFtNCGYdMqOcKfsn4NkIQpTQi+ndfRg0y03khH5EpyaXipowQkyF4D8BbivicrUzm+/6lHq/1azRxNDqtQz35Deg5jCbXPL4TTYVgILoAmJaBAS5BmIfpciGDmyeGcjJhq38qLZtSk69nT/jDZ+TW9bMgJEJPeamNC4GWQuHUkIREDCYC+zITHdRCsbV2MB29gNR0DRoy8UnPIY/HqjGJWczfjZyptkqZ9HmZj6atwwUfRDkoXS5eZNQsVR6tdWSs8EaNIO1IZ+z4aUPSL3xBPLuPj44iqGu8xh+9EtuRDhQeooeuQSrtYmcr5yNkznG/je6auHMnfhpr3LICDq1Hk1zB34NPii7JYAOjeFF93Cytlqo9Pqdq1HLEITeMp5jDBE4dyVy6ashHFwSz7qYl2iY8RWdBg/U8g7+lI3EafskRgCN3o0IsaLk3J14lJP5O5m58TZiwmpwrnJzVK2rhvKy77lx6htWG+uQ8xWROUnQbrMbRq7ncCW8PiIBhQIezMiiSDDYg+Vx3mcZFfwre0tu55y5gmMtUC/ehnjuCwDm9FnHR+Hwq2cLCdlxRBRHIBNF9GYnXj58hIb4HUA6B6YcACC+vT+iCJNnuMG5EtDXMyt+Nc++DJr8H/BpGyCNhb+Uz69TPg9jwU8E1wcjWAW7rSNJP96ErEeAgrVw4V8AnNRJ0tvNXm298icC5HWdJj/oTbRtcxHbwmkSXADHmOZqfiY9/hvK1W04F06lT//7meM1CP5yQuGaxL37Epm5eTbRSTLurm9m02sjJYlGXTWx1kaS5WH8PPEMOmfJyxbQtQHO10DdcQLkPrw62YM/Mxfj0SKNlSiIdJlVqGRaxnqX0GaaTdOl+0ixDOejybb2tV5jSfIbrC8Kx8MaiWiRwBzRgvQdSkM+gQH+mIsszDo0lI13FEvBdF0NHEpjkGUs52tGA6DT6BAEB5Q5qz6Lr/T3YIj2Z6/rMd7Ph1qXXDgwEiafxqtyC7pomGy9znnlO8ib5Yx1Hs9IDwn5Pt/ZinqOJ5U3OnDaOR4SNtHkvhS/rFUEaH1JFD0pIxa5VeOQR+uWqBOwS6FZBSOWbhlPG7OvU5VHhc+vyDq0uHVIjvg7KIRrr+DvEYBBCERlsRBYFYjOw/auJr0GHimMPXwnp9rjCL8WzfY55l5OVDHrPVyaS7gRAVkhMi4NuoTOWS9JIR6ZCJ4DCNNewXXU55RXp4DVIgHr4p/khvYe3thiYH9KNDIsBDct4d4oSPEH9g1hutNVVteNJrrAiedOPMcv92yWvtTQBFdfJM71efY6jaTDzYJOkUOpXg4s4z8rOxcLlLX2RUYjSU/UMeFof1LTU4nVm8hbeQiyPwRjG/Pbo9neXsALudGouxRkhDpABZquGkLk0OGsw9pp4ggHOWmxMvQ/+M6hwC6jlbSNsHjkdYYGb+fVqGs82gjZejM+NzFyYsLqOT0oi++mbyLQanOm91kOfmO5Z+0XRB1SkpAzjAtTpjHGN5yICCCiBarbmRr0NDcansKvPAajUoNo1kKNtG5YRSXtbu1Um06hMj2G0TkFQXgc3GLtc0hm0mP19KQxaYy0vpnbIX89nWZfnq4H1/IQRpWFcG5yiVSvoBnQms2S+sN0nIpmwpEJWAQr1gmOfIaO/IkWIusktQu3Hn6e2q5kTA0G1J9rmcB2dqTmMY31AFS1x/JpQziXFaXIvFU4N0u5YV99FQ4ehLvvFsjNVSP2iFnJZEDMo+CRCMGzef5oBwfuKKLQZzOi1cZ0LVpHiqaaNs00Pip/nXLPvwFJCq67jPd5g6ei1qMqAL+WGQxVLEdwhRa9L8UtiegsGVxJvUKS5zL0jb3dIVp1HqV+X6MwSUHNm89TpZb++OlyqPNrRHQvJE11FbKvwoCPoCWT/cr9fJC1HGOXhrzYPBLbbfcPXC2xtTf6UeMFE6/UYGhWY0mSQ/9VNGYepE3vQ01XIJZ1vzAWOYX3OM63BjwAWG/DYrWor3K+wp2+vn1xU7kRrLnE/KgfOFQFOztBj8Q2y20cxpjo1QTGydkkj2T29llE57ramSb12lB0ilrkwJlB33DBvJ5YYQbDhV+I9b7Aa2mzKQ15mzf5iocBRWE4E449z6VlS+3kLqETaiITKA3cRGbEDpILXwegJup3ms8+S0PiBsKKodwk9SgnF0D5ZhrDdpDpvczOOAAQekiSexiSGJZ7GFdXKx2qfJuz2xHA6u4bi2ih07kTg9KAT4d0Ru/eH5V6W25u0XEWAhgVNJGtdRPx9gGlAHJLb4dkpc89bLjmhKnlIgMq5iFzHkLftFT7MwYV/M3iJVb2XvHkROB8FmwS+XXOr2gUGke9rGaCmhZx77gx9Fe/zPih0v1PnpyJV6mFLTMSaPaspbTtChCFUd5Invg3VSnxeMhuVcsKal7AeGfYoweXOmm/DjAMhqoGmF1MbV0mLjt2MGvnbbQHTie+fpzUpo5iKFhL8JBPUW91Y8jxLWybOxO993RE8QUA+opnudsVjnRasVohsu4JvhgHQUHdfd2b2Xfa+gkAk8TnwOLGcafHKUo4IZ0LbOc90SrHaFHjoaokKfUEH5f9TrHfVygV+xGE5xAFE9Fe6TwD7KkJZfS382jyHMv5FxsRBJgd9ylh7lnMLfHF2bgWiOCH5T9QEVpB30aRPzLeIG2UnpeqvkUdq6ePcjyphRvpnyLJPArCTe+vTblh4gSBc8VzaFaqWGvZxYCpezEoDai7HHKSib4niNLouWidyKidp+hSyTn53AF8ucfxONuzs0KfBaCk/QQtXaP56erH3B/6Lb+JiVzZOB23VmeaAnbb51en0ZPTzkNQ77cSVRzFmkfW4MRCx/xrviYBEgZfJKI0goCaAJq8m9D6T8I6p4rjf7lxb+pCCl1hfZcZESg3qrhcNZVhITu4Le4zNiZ/xrwqOCiPp8UC59t88TaFEu2VjuXPmfQ/m8yO13bRHTzoyeyLKIlA/bGaywsu4xbgxj39pTY7KZyI9nLsDxYLdJmcudBwLwWKTYCR2CuX0YpdmM19CG94AP/WmdR4SiDzIFUMHlUZmOoNMHIcrl19cdbF0anJxSKTWOy5686S+/M5di0tos6/DpVchcrG6B57KJqfP1lFio+Gg4/1tqF61qlDrMQi06O3+vPb3N94eeddjNk7lWGnJYD2jTFtrHnwfSiET2ePZuaxV0nIP0kfnS/PzlZAqshysQvGS7nST74PjUHb8Km8ztDsb2gPikHQrYGWDNQ+ozGoDGxdmMUs5Wg6nTtx6wHqnpZVxdat8MgjcDr0SbqU5cRWrUQmS8EidJEf9A75ldxSmvT1NDsXI8jdmS3/Bt8L3zDd4X7sFQz00A7GWz+YwTYgyD/l3e31HtSf4IWEpVz7+k1UhqBbPjstehoN8YEc2RZq/1utcQAAkcJ6RoTOYbizL8biKIIrLRxJs31nye+MVd0PKjDZ4hs5wS8zb/MF/pj/B4uSFrGn9jOClF8wxQn2a8FDcHxHTMHzfO4L4zqaMXpq6FB0ONoil8gMKSq4ONqFiMoJzK59Q7qmt9HOBnzEvsMrwOPgrW32G412ho6Tf7ThFielD7D7CuKe4uvtPnwhvoRSVDH+UgOCVU6duAkATzEPwSonqiiKftsNFM9zo9qvN/ijeyy4Sf4YsANQDEI711MSecjYB7rOwdjdXL76Ja9ueYnUS9GoDY10egZLzzN1QvM1vrTE8brlU0K9+uPR4oHezWbPhsyCOeXwdyjz/XSs6JtHm3cUgU23E+QUjJC7Gp3pDpo7Y/FQgtxi7lWfm3P2DQkawuFlh7lr7F1EH4/G/Lrj80751/Br0vHip8MIDobyUr39mik+kaLo222QJUf5d7BPKr1iGDdT+P9fKP8zfMJ/l3+X/7+KADLNLYjsnoiznvrY3U6VnhtfLeM4cXEGonoM7REfsMBbkvJ7cdQC/lrgxkiNgdDaAJLOpePWWGxfoCI8rhFurSTVZQauOonth1VAb3aCzhKo2o2Q/gzPzlhAzeAJnB8pOdDlogKcQtBag/ik7xYeqE9j0MVYNDqNpIUeNA1ub+Zy/R30lMVwsKO+A6xsNrRSEF1Ml7t/7+vAodo/yAp5lga3I7Ze6v16T5ZfZHcQ+GqdMGnV1BICSk84OhnOLOEurw5MHq0MvTCUXSeSWJV+SpLDUfmSjRvfXn0FpVHkzJjr6C4X8fOOG5ywwH6DqRcS+ggmTmHl+q/nWb2zkxrrWD67+iajKroRjtLmoTG1g2s0h2rfZGv2CjLaA7igBy+hk4GBezlSfDcHy2/Ds8mTO3+fTURJOF2arl7OyqeHrmB49mYii6Npl++kjFP2ax+d3ojMamJpSTwD0wfSd+dKPr/yFjpVEpxZAgVrUXdlYIr0p2joIvQaD6k/XaJo67+DrIZhJCpUDNHcTt/KVSSoJ0pOl+uvEWnYT5osEGNoMO6trng3evcONts3/VslaQDiDIuJr3wPz85hfOIDd3f1kDIImMim3LdoNUoB5hyDAs/IJN5/H778EhJ0WegHXCerbxal90kOlQLLfdB8HVqz8VR7UR8gOfQDy6Rn+FrL4NKT9PPdzbqkjTzgJo1FN4KyLeYpOvpvJN77XA92lWSlGQUZ2ebHMUZEo3XS0uJtoLBPEZ0uNtS55wCsmhC2dMBXz31Co2cV7vmPs1n7iL09he7DUQuOYN/bTXA8vAq8B4NnCn37zGfK1D0cnzWIBp8GtoeWsyDqCbj2MuwbxCTdeZYPK+DI2KPUJ6lw0kdJfTzhCJld0VwMg1EaMAqdaJXFXKw9TUlLCSChm2QWqT/Mos4umSgXJSdIetQidg8SuSL/xn7tYadWuPQkm8Q5HH3pcRb9PJjxl/YzsXQPGg08/O4cZGO3UmKYg64aKhOWkTHxDRKV06Q2m/UU9n2L7919KIht4+DEg2T1zaLVvRVRrgGXCPAdxbJqD442qtFUmVDr1FLfl2wkpfMeFia+yyjdx+h/2ED86R+ZNg1JYmPQGrY0bac0aTqqxDD6FPTBqdNJGguTDi48xLkgPwZeHoTX1U7mbpvba/4leCcRW/EMEcVhnJpey9X+V6Vg2F9usEFgvkseHXP2UxlQSXBJDeVuNkM68z3E3YkMy9zCmO1dRGTbginWHsGaKedIj9zIsTljuDTwEnW+daSM3YqifhMo3DiufRP8rnEi+CsafqjjkTWP4CJaELsqGef0AL8Mf5XV+pWM2x+G1tWPqFl97fX2bwpl5ZsribkYiJPsXwyRSQg+PAcg6usYrxYYfm44RbEW9szY0+t9/Kf8T076Orj6IuwfzImGYvYafqTW6wCeLnoWB2RgUgRK66PFhFqwcuLlT1m5ciWVIZWOnJHA1OA3mBz1A9EuU+hbt4us2h8h5W1wkfLhNDufp9Z9F2qzLz7tY/FSOUAa9kT3gp4uZQU6ZXmvfW2/fiIvrx/N46+v5PZNt9MT8UnzNV71L6CfQUbq8Wy8Km8AEB8v0Gn0gDE7KPZcjVNLJcouifqlpFFi+5g6EfxG81mHGZ+CKOQtm2jTXJPeqdRPyNbdjX/LLF72P8Xg/F2MycgkNTAV2vKg/G8yWm7jrbPrqNC789KRA7RbQqVDidKTBuf+LK2BsohaKm5/kubunGkA7gkYZUHIBDMaQ5h0IDGO7hXs8xdLCQiX5JJHHZgkvU4WObmNwzhfOZtx406RO2QRubG51PnV9eovq9KdNguMOtEX14o13Ah7lOLCTzh2OJiQord4t6WanzvNvEMDFvEUsk/h+rZr0johwvbcpzlc8ZTtaY5cJNFel0n1286vbTIuKE8wWg06q5x6bTizYj9nWeKHOAtwfuYeALyavCSJFKuc5q4AnvTexpIe8jj2cdQ3QrzkINq6cxiz33+R2JLZOBkieu3tGY1X0Csa0Sv1NHs1S8wYQwsB6SkcWODfg+Vhs3PaJVmkZtU4Pm8B505nQsuCbXlshN6fBeqcj3AyMZXt4l09K4lWrKbF5QLX5DmssOzjgq4DRAUMWkNZ+HKOedXQ4NNCfIYZtyvLIHQelG+D3Yko07bRkn6cyYfv53yrkruuLOeq9xGIugdiHsXJWEtkQgmX7tmPs+p7Jl6txsdnCHlRf/DdjV+Zs9GMyuNvPDuH9z4guiVS0RZP/5KfecRYREDLbADuYyyM2Yl/7Rs8kbIMg7MnW+btpjLYBkoxdUJ7Pn2s37Nx4ff8uayO0yNP95o/OqOOCutF2jTX6GbMmGWuUl6Mgu8RBJGfylxxPj8Yl5ZEzhW/QqtiOFx7DQ6MoF/9Nh72yEHrIpKp7mDQ3s1UBnwB118nrnk5J/rv5D3Vu0RbpztAJ6LDUeXdPpZE4XZUxsBbc/bZFBuMghaLDZTSgLSv5bcd5dy993JmxBn+tfZfBFUEYH+oxYCPaMS3IIoBVxPwr/F32FDXXsGoDuJe6XXDLDfT6NOIgU5J/nHwt5C6mn0eVyjLcSM8Yw9HkiLY27waFG6YVaGYrFJwxQ4y6+7OqLu5Jr7FybRVNN01iguDL6B1sQEhNEEQNINOzc8cSYrBLOgRLWKvsWjUNVDh/StVnpu5EfYw18KX06CTKLVh7tlkGKN5vgFi88KZtWsWdGokR1DKO7Cwg0KrhuDSMP6eu4+/Fu1GtEpBJs7cxYCL89kQAFvnb2dT1F8Ml//Hgb7uMhQYq5DxafajOCnaCFd24SRI7JZDWnjOZwUM+0X67NTzGEYNkPK7diuQeCRD6BwMtoCuYFXi2TEEL6VtrMbtoy71Wx5qvsKJwZ9RHT2KSdercDO32O7vh8rcyQyNwNiTaczeHEKH8qzUJmMHGJrxqx+P7z2/UfXoGLs8k9zSCBcfxkXWxGctEJ4dx8D0gT3qlQROodzjW4ExpBqdh5zyxOkMdJtmb7sD9e1gJQiChPAGOCnfz9MbjvXoLckRtSDhPR4fcj/ftYRR2OiCXvcRuT53U9lZwtCh8Orc93A9FIx326/0LEHshrI/QeUHTkGsbfuDmesDmLVzFmK3fNWozezr3Mbf00azSP03ADFVb5DqNwS0lXD8NgQsvFYRhdkKIY3LiDHPBaDD4EkfzwzmtzzHxKtFnHSdydG+YVxpOuloQQ9JXVsv9KrjJc0VHl99mLhrSdyIz2G7tj+M+gvOPwjn78fsPxu8BnFhcDpmmbnXuUU4tYBrEu7Uzn5LVn4BN1ZS5CqdRZVdNQglJfjWyHqNwTnLD8z6w8qHTdLvZ4MXMezHYVKuJEBn9uWH9E8xln1KihJ+8D3D2vEKBCxMi/yDZ7zM6JU6jApbIMZWr7Pl8xjvdJWxTmAVjRgVjRiFdkQRipv7AxCe/RrpAyRbQmaW8VTsRoSsd+3tunx0OuKc8Wyfsx2raLU/WxAkVZydrZ7oejIJOosBiCibx0MeRnqWnmAtpcUDn/bxdImNHEuO5ZTf0n/8rFWw0ureil5lRW2S3qtu23fNi2vJ3ZzLx0l9KPb7wm5zdP9vtWJ/Z3ruN5/ro3nT8gtRJy8RfdmNmOpXiXPvZ7/Xt30igzwnIVgVlLtsZXPWZqx0pxZxKDcozV6EaRIZH7rD/mzN4l1kD5HxyNdL8WxS97CDuttjodn5PLlBb5Bh3dRbItT2s69TGSNDN1EUvBVmS/3pVbmVNQFm2j3aMSk0jnZ2lkHWB9CUjiIqhA4XC2Gl7hhlLdLDqvZyp/xdvvVz9MnN/dHTee3I5eWoc5usmHan6xjljfbrglVOnPdZNlfcQ9y1MVwLvYMa5w/I9nwVQRCo7Yzi07O/UmlxwqtCCjp4NXvZ2dh1nRGcrZxBRls9J0Yf5tTIU3i0eBDffCc+rm6cr5xN4NCFiILImZFnyBjel6DmOwiWp0DNIfrcCGH3IhVbAqWaapXFmKx6nngCZj97PyXRL2BQGTAoDczeMZuQikD7WDww8Emm+z2EUteKRmfFs8Wz17sMvZV6QJrbOxcL7FwsEKnIZLzQjFurM5kJmdQEN9rGrZTBQbtwtXZR619LQWw7g4qOEFe1UnrIkcmwtz+7Ai2URxaTkZjBwk0LWVQSi1PjKQSLDr3ZmWUhp7nPTRoLNxEOhhejkmt598Q2shtGsqM5mEqTxJZss4BSsBLtlU55RxiKPp1YA+VM3zMdjxYPRNHBgsMqozpQkq1NO5F2i7+tZzHZgvf9PDeRpjZhFa2kjxmKZsoodi7fjk9Zuv0MDtBlbeNa8lzElEQGnfkSpbYZNyebHSF22eaW9H1dNvCkUqa0gyCTr0mBoRabn0NmAyJWdpbR6pSOXl6LyQRXohZzNKUPh0r3cGfKnURZPKgwXGCd6hO+UK7iyNXjvLMValqgwjyF0MZ7GWSdgfyKnOffX4FFMNkk5aVSbD3GyQmt+Py+kJaAeOJ8zqHOfBpS3qLeWQJxVge3oJ/iSpemy87GBse7ZLVCg9shajy3MWlOQy9Fm38qRyp2crrvUC77PHvLNYPZQG7HeZqdz/7j2tXzXQXwb56Nn2gDmzZdhODZKPo8h8YQxrWI+zicHEm5yzb752O8YxjruwAQuRZ+P0X+n9Jq7gPDf6eSmYS6ZfNywmVcE92RmW0AREGEdEddZ1QOx7NjBNbudEu2fSyrfT5Dfa5zuwt4doxguZNtXazah0nuyect4N7sineNKzKTzHFuGfoTG839qDRBdahIfcRgRwND58ESK3UuSSQk9MPZ67Ct022BsSOTJZUEQGFxIy07nbTsdGSCDYya+jEn6hZKfSfIkVtckFnVNOqkA5ZS1olLVzyyMGmNisntVuvpXritdKoKqbfm2W303n5Bx8/jnWCRSyFcl3x/6sBJVBXcwbijA/AtPg7GKswWMxha4PITDDKVIlgEJh4YwvKflvdWxHEKoWPsYb7usHDorg2onF5jYNFfjFFUQr/32J0vAbTMZkcOVqncyuwz2/JCPvr+owz7cxhD4ofYP+2al05g4Sn791ZdlsgsZQlT6Jo2l38q/7cG9/6/Vv5LzL6by6VLl/jrr78oKyu7hZa4devW/+Cuf5d/l/+HisoLUVtMtMpLyiFjK/8Rs+9mzXmAXV3DKNzTRFTeCRq9XsZbIS32V2omMyhoL3tD61iwO4XKoEp2j17InU1bgRRWTRyFU1sHXeFaPr6Uzpn4UYDIQ3OPAQJMuQCNl2hwGURzSCKdzhLTSC7KYdRfHM+08q8QkaBZd/CMfArNnm/hI/ORHGhKD/ycsrlTFcjvIrRawO4UHLGes/kHIesR/KsUBJx4jvMjyrGygW4j+3zjXooCfiOgSaLw3xxgulu5j20/zOXR0/0oDytlyFtnodACqZ+Cyo+KtjSKkloIyD7B4LqviY04BYoEmHqRyqxN6Hz+5PToXGKMn3DpfCqnrOb/FAm9x2gm7fkVvPP+kh5sMqtdsi6y+Rhk3IdadpF16R+RG/QaOMPKzlm0ZEiI0HeHPoneZGWXXoFLh7N909/8YzaqQwmQDgGGCww42smgkw9w8LkaBAEmRf3IwMC9BHVc565nC3nnuSfwah3NkwOm09h5B4zeAghEn0wm8tgbtLj2oWlYXC+n83cXf2LdtZ9YPgO2YTss1R6D8i0U+PzImj/uIznGxOdPxdDuXNFLRlZraUGnbMNgy3t381h0F6tgwVWEEGsenF4MCnd0/hORVYmMPp9BxKVlyKaehzN3kuQzHBoLWKY9SOx8M2kV4CuDe67dxfTE8XBiOHSWMBEYOsOZNxqXUxB/CAAvaw3kfUGISxD93So53iZ9d0+Ho7r0C5amFPFOhpR7rDvYty/ICHzJs+qBHF/qzfe5C7EKEppTFETwH4Nldgm3vy29c5oOEOVuCN05udpySG3YTj8VVNkOfxagf+18OBkAQ38AQcRJbcSntoGndz5NxfO/Ed/vAPh+BR4p/FSazffFv6GPH8zKWJFx/sMoMc0G/uaEvoG8esg2gLugpdT3O2ZvX8VTQ59i9dTV0gHGosEs68Qs6Ozzz24I25xeJovRbqA9o6mHvC9IEefzS98OkjMHk7brLFVjerDMgmdwomEwXvu+QRdroTp2rGP+bHJnTMAkPrx7L29UtKOzjqXTuZOK0ArkVj10lIDKmxON4Sz5ZAYqg5LK+dG0xMqg6SLBhl8YGxlNq0VGDospjw6muelnompTSYm8k52muzEu20B6+jPctfEuts+7IeW+aJGSbg9yhQ8mP4Wfejcn5R/0OkwIAogWE355p2nWqDAPMaPCBCYJKv6I+goDfGBJYiNHB/6E2hwh3VgrMfzrg1Zh1TaSeC0Sf/mbFGvScLEWwh+xmONX0GZ9GUF0JqwsDMEqMGvUGpxKz0D/BahELXe7wapmcHfyRG7xoaYjkj5l6+lrYwq4xXtSNHQ41/ps5su2ETy470E+m/oZo/mUpsA9BPnFEluZRGqgrbNlaszewyiqPMwfT39KVP1KXLVJ+KodSEVHbjyzfc5rXR15DC/lbSHgp1aeLriNnx+q4sHOobyceBfe5+8HQca4pmDKusp6vLkCgr4e9o1hSlA2orvAgRXPEyHGkDNsOSTNsvd1dsgLNLkex0/7JwILe73/clui9Z5Ok56OhBxjAlUteQwBkjKTOCNIjgKOTAZtBbeVJKM/60FcZRunI9+kqPVdPngvGl3NdaqtTdx+5h485vVhQLGUfyTEtBH2PAYTjiHzG42qS8WCTQvoUhfx80O7EMUdEPM0JfvMpPnvJkUVzPXWGQC4qIDCbXD1RXw1FznVdwn71flEtT3HHP0ghjZfgUFfcl5zHK6OZdzBfgTkfM7puxR0mZ4GNOAzDP+KF+nj+SiWpjF4d4yhT5/eTNwXOj/kj8jncV2gQ60w4a1/mIxDr9Kkk8ZzhAit/lHsmW7LX9EuSLKAOZ8SrG0nXiE5/dpcG2m+foPW3B85ZbUw1NRbPmepxcR5YNovZ1kdcC/Dk6bwXdlM9MIuOj3/dEiGWWUcKV5GuWka101riHCHE6GwWZvNFmUTYyPWozW6ElvVihBexMWXP8AsM9idI3LRQKC8Ec+ewPLufT3nM8h8h3ZVEBlmMy4dajRttXS5+EjXt/iA3I2g9lb0C9VsbFI55ojVgmCRkJV+Pc70PXPLZvnsBCRZyYHpqRTEmXBSOHL/OGRku2Wzeu9TPVH8AI+Y34T9n8G0KwRqGwgxBHF9pJKoolBmj1oLl2QQdjtE3g1OIZCby9CzHoiGSVzsH0WrehxEAm25AIytNZHn0cmIugjUJhGFvLdMmyStbEAra6RRq8RbWwCCjINFy3tcl+ruhQkuPky11ZVfvxtIQNWPZK/IxSRvl/Yp1z4wZievbbuPar+TCLIQEHo7le1BDcFMmpORuW5Q6vs4gcHtcHwWnoO+ocq4lv57c6mNuIIy1IcE3XqovSCBvUZvZc9XHWQmfs2NxKMkVt4vtWXKebIvlXLt6CXqlPG9kL89WZpx5W8zJAQudDrGpBuU0h3sMwmdWEVpn3pTNYUvJ7xN0p7Z7ImoZXhxOcfGdKHztB3tOkpAE8jMuoFkTd/Jiqx4Zu6aSWeKrYPLt6Iwm7jsAOgCMFL1ECjdIUZihz6TMwxjWiNBVdFo1eV0WTtA34jK2MKC2F+Y5A0vNVjoUOVR1KEnRR+Ba+yjZNhUpHQRPuzpswe1yZYUUqaCrhpSNM1o1Q3smneBWo/LjBYcuYFKW0u4Gnk3an0oBnkDFpkOnek1+/W9ujFcvVJH/3Yn/lj0B38nVsBfzrDIBJYuPh9yNzvf7UJdJ+Pa5BftfYxWWsd/a4OiqByc98B7/3Gakl5lmd7MR3uPENr3Kz5y+pkapyOoutkrggi+I7mm+IIfvhuAc9Y5hvYbSnW/Hu9QeyGrgjL5cV4lbV9PIKDwNIIwyn65K3Aa+7UwSK9E3V6PaDKAJhCG/QweKYwv/Amf/AHsvDKA/Oh8zApbHpzD40FbTrKlhmv+Isf/aqTLIwi55lmsqngYf4gLa72ALwmpCEHr4sRkrUNCi7gnSNt0jOuep/C8PwRVzpCbnIbdwT4zzc5Svk2jZSBr1yoxmeCbjYVcSdzMub77qQy6TrIgnUG8NZWEe2QgticzY/cM+ub05UbSDdqNLdKDi38DXRW6jhUcT/gYARGNPhKz+AzoGuD0QgjW4S4InB9+lgavJoRuZp9zKK0WD4rbI6i2XsOlK564qpWkBgCmXKjeT5PpRb6prcHsVmTPgS6K0NLlz5bs5yhpmELslSxG/T2c42kGTON7+DOs3QCh7oBFb7lpUYROzxCupxoodWpnrXYhs8MWQMEPYDGgG7yVy5ZWupxPE1ExHqWbRpI4Pnk7aMv5sV0ELPbna62BYNYR0fkuyf6P8XDg7XzGM1REevJa/J/MT5Yw8446CPYxsc8/YIz3mzh5lbHl+rNcCwdowug/m8/vWEKkeJG4EtCru/h77m4qg4vs91W0xXNIO4gqkyN1BAgoFFCvDeNcxWyGhWwnyKWLC69u56LiKtsirVhzT0tSsCov0toOUecBG2u6+0iqo0fdGuSCilfzHqXL+2t8WwcTFzgJJn0Axb8iXHiAOa4CX3b06N+ewVHbj92OyJv3qW5bXqPTkJCdQFVQPXTHFuw5+8wYLTZZe6vMHtSo6Cyi0aUCnaqL1SXrKQsOIE1cZX/2EFM6HqUJGAGTl5997HvVywz0CITbmR72dCKOa3dtq8ZF2cwBr48o8lnHZF+4ODAdnUZnf8/sOfmw0OJ0kfygt1BbFzKMBYCUR7LA2EiXPIbIoD95cfDz5Or+BEIh8338c95mWZCRz9POMuz0LOJVR/EVncF7CMzIxHTjbaL8rRSJFoKqgsjudhxXSwzie2qxByzNQhc6kxWzRYlMlPUCVVsER4C2G2At2tyKFsHkeHeQ8fHkEZxtmMQvBZe57+f7cOkczB93bLT35dGSu9idspUzifs4E2Sg1vcGg4XFiCKcrZhHWXsELkm7KQgrZ8iFIYw6PYorc935cYOCxkYIVl1gi8dZXjZCZo+xQFRjUoahMFbaZ83JxFROXoZrg6+REpLIqIsfc68b7Amq4tCEQzR5t9rrZTSryNe7cTF2GTR4kht7GQ0D6FnGWz6gtMJAbvCrGOXNvWz4WtMQvsl9nbKJVzjR72MElReitYtBQXt4ZPAjUA9do2CfaSoxVS60+/hIY+EWBy3X+Vd5I2FFUSRlSilh7qyIJij9IShMxFl5lvuuPEh2wEdYBTN6K3zV5Iu6YjY36sbQafTk5+IhVLhuI6xtAlOinubrq0a+H/kUGdXT+L7lW4YN38GgrSau9ctBEKT87O7u0hxs8m6ibkAdOW45pAgOqc2Ktgq+uvAV7ip3TpSd4GpFNn1c1rGhfAtlCWsZ0mSlzScS0ceb6+9dRRPagT7GxX5/h7kZo0KD1dkFUa2gS9GIVqwGM1jELkQg+q4RZK85xt0/9uOzJ4+ikqlQySX7d/+0S0zdOxi9XzyxlW+ycGQIAK8ee4GTCRtJLPsco/EJO5BCKZez+pNPOLOyBbUM3jUiKVEZ4OftAm//LTJrtpo7k18lODGRlvED+eXvXxCstnQhRb9A0yXu8/+Ta2I917TfgNCXZl0AxogHUbgnIe+oQalXMnNrPLLaFoiVgv03BzusVseaPThVbpcC7ll6BlZldla9+Zbge7OumSczhkFfSG200uJ0CZOqjrLWJMLcw3r5PF108Qwu/JtBIbY/tOVB6QZEvweABPSKGnSqEsxiLw1LBAG0qgLKfX/Ep20Cvqrp0J5Lhuwd9hUEogybiHxOIIeyNjrqPvQHOLWAs+VzeMZ9G/vPwZm4NMCxNppw4e4zqzgR+AK98vmVb8ZJm8s+LTzz9d0AZKTeQZ9ZNnKF7wj+sqRSYbrGozu6cGv/jNrFTzkqrG/C7+Q0ng2EPRVQasIRGFO4g7GNvKz3sYZaaW+YhZsuudf5c3n0G8Qo4ZNWh9TA9dpxsNhC+U6BPrUwevhTdHh+S6dzbq82WQQjR5OjOQoky9b2unbzzwe08ELTMFbFPwP1p+lr1hHR8g2lydcJv7Gb63HfkdW0mkDfRBixHvHieww2OBNeKp2D+5X8hDASKPwRin+FlPd5wpZeIhQL3ppKIioXQ2sf7u63hy9yl/DGhRuU+sb3mGeOYN8diXeQ5JdEkp+0zgwbPgx6KDwDNIyaTUuDw1j2T/ZHcFLj1FaL4swJvLs8EeL73zJ//l3+58t/m9m3ceNGRowYQXZ2Ntu2bcNoNJKZmcmRI0dwd3f/Xz/g3+Xf5X+z2KmvCneMUY9zrtO51/VewT67Vr4J/iHYN0f+MK8t/pU6v04CC07bF6QduU/x3MHTPF/njV9+H9Rdaup9C+0BAq1RmuMaob6XXOYg5StQvhmKfqUtbDG3V7ngeul9bvtrGOBw6gqCwNw/9RyxrEfnosQis6AQFWBohuarjPD/ka/67uAO6wu2Z9sq5hpNv4pv+MIXFm+YiG+9Cx3qazeh9LsPp0Z7vXqWNvkg6lqknV7VP5u7FN/D+fsh5mEIm49F6UubRwx5w5fz6G3v8ULqaGiQvDWjc1awv18z1YFN5BSsY7Sg+K8hoUUrBWeWM9z/KE962JDBNidXi0sCJL5KmzmEgoD3yQ96h0fdYb7qCk26IN4eN5FRQUfQebdwbOxZ0k6m0e/GNJQyJSq1wzjqcpIkWJ20TggIqFQQ73OWkWFbaFJH8I6xk+//dZbqpMnsynuUFqcZEgLIfxx/mPtTnXSdS8k/cCYuTdr0O0pwzX+GJN/jWCzQYqyjU1mE1tIioWyA4FYp95xqywam7knDpDD1kvHc3PwSh1PCKfCVtPXFm8ZCJ9bRocpHYfLg2ZIkWsRg6aBV8B2kP8P08PdRGCVpj8eMzlCyHso2Qe5nbHYez4sNIFgE6k3gIe/CV7gICS+AXDKoXd07kfX1oN5f0hTXy/1gdhnHqx8j9fSLvNoI9GBZKjtLMQXdwapTm+yGp9oYxIaBjhw9rtYmkp1aCM+S8/pbrxKTF+NwVAoiT7ufZPrJPdwYomTdfet6BHgV5HhN5LDW4TiJlIOzPkMKIG3xhv1DuU0fip+6gav9rrLa4MS1lsUQMhcSX6JGEUjAvnHc9qeGhjYJ1ae1BkPme6So1HzcAkU2Gc9uA787wCMFt6STv0XU2Z3DMkEm5Siw2uR9MNyEVAVvWgleuJ+D06qpiRqOyV1iSpL3NRy/jU36Zzgzqpx252qEjoO0Wmw6HBGLwdSGT/vPCCY9w88OZ/AlCYHm3JEuacMX/8pzbidQGaTvr/Wvldatfu9zxLeDJ0+uY4p7LqtWfs7hSWf5svxeduRK6LfZbtf4K9iEMtpASfJMIlp/lNqrCYTEV/jwykn0Cg9aUnzISM7otfa1GVpocMviysQJlMW0o9FqcOpud+A03ukczcZPFpNw0Quj0gjdyMZx+2GRgfNR3uyfF41a70LMERe8OkZhEZ0geDb5zaXsujiEyItfcXHwRTYu2sicaqgLeQIMLeSZ7mT8sXAWvPo6YXHRFA1/k5LqRQB0WoJ48dBxGr1mo/MIRK8yojPpMJqNsG8wjw8YR9HABVirqhm45x1ot+lYddUiD1uIKreI0XmNDLg6klnH/2Z5zKv2NtsPTIKD6S3IXWHAxzD0Jx5r+I5hfcsBWPrDcpSihcCiJZIh3XwNURAZfWI0K1euZPqhd1GafBBFKwgChxpncletFZc2DS4tFcS4HoTN3lCyAbTl9oyRuYq/KPb9iuquInu95HZmn8Np0nOspmsOsPvRA2x/623effld9gd8CRsEUPuBRxLpOnd8S8MwKAwcnLgFo9WADB0uJ1OJOTWRxZoKBl4IxLnsdS72mUurfAgkvwnO4QgFa9mmDUdAICshC9Gisu+DMtHMa2mzSbxZQipoJgz/nQCnbD4KrCdQBrFRHzFUsw3qjgMQ1rSHpz2gw7WDZvd6ivzeQm+R0Lp49CNdNoHT/m9Q6vO9NA5C72BfG56UmyArMYv02DzyrbH2QF+IWzYeZwcyPuAHXNpdHP1lbIWMt+hTtJqzEQG4dHigN+jpPJ3BPov+PwWl7DWaefG9n/FrfI5OVQEnPR4kN+iNHghIEZ3JDWdlM78m/8UwNTxWB7c7ZfPGmOm8fWIHzx7ZB8CEwxPofyUFi8yCKAq4qer5clp/zmiTWdMqsQBGZ6bTr0Pa3/EbDeGLyQpcxIaobKnvO06il9dLfRIwGTz7Eyh04mZU4NruimgWkYkCqLxoHJHL1L9aSLHFoRNLvyRRNRX6Pg/jDxLV+hxvqBJpHhjEtdkPEt26g0j3Pj3m2k0O7VvQ29LAjFTDuVBwsbaBlwQG8Sn8mnMTF5HgGkZ+gpk5g7+GvC+lvLZDvoXOYnzTJPZvUl4cT0UexKk7X65LJA2ug/A8PownPpuHc6uEIBcNdfg3rSfR5wjxIeupaPuLas8t/OwdxMC1A6GjELH0V3ycpHdVCppJ632K0ATaCg6KozjQ9yo5KS2Od727XcEzWdfujU+9D1N3JTHm/AcMUTvYjA4ktIUUlYXHPUBtqgfvoTB2H+qQWQztP4fGgGD8Sy7wXPxcIvSfQ9xTMLeKZuc48i93MmNXP2brvLg0bT5hVcvBsz/NzrNZf+NtqnVJvRw+N0tQdR++l6o2smPRDkLcQmwOIUc+v+71wlWwgqjivDCGT07GElPYHxfXPcwQbFG2wu/hUBoBMiN6tZ6MQYF0+f2L27yfk65PvUzx0D9uGfVBqsWOX0w6wuUGJhwfy6BLiZKwhiCDnNX0zY9mXvQP3O8ugYeuRt7FsnMpHC+V1gKd6U+GKh7EXLnTNg62F91ihrF7GFMcg5PWiaXrxjDy9EjEHjWxS0f2YGOLgsjiLY0s3NRKrdmf2LxYkjLiqAipIMHNpqfcUQS5XzImZwUxY91pCuqHpq3W0ccTj3N57CW+b4Olvy/FtcGVAP5rxR/oaKu2/WYLtggm/GSwqv5D2BlNovElJnt9iXd1J9P2TUOjs+XVvvwM7IzmDo9KBgsBBJV1gvYYhR0Ss5iyTTjVSuAsnwZv3OvyEbGA0gNUPlCxHXnoHIyJD1DRR8ahiYcwy2xtCrkNIu9iets83j4fwZgTYwgs1mGQN0py0wETiHc9QnooFAxKpzFkIJ7YwC45q2FfKgmNrzI08zJh+hlYezrf6MGMx8Lp+JGc7juCRm0jcjmo1ZDMW9yY+ibKxHNonbV259q3l9fw8uGj/BJ8FdcB0vqW3Tfb4QyfeJyOUduZXV5Pu9MN2pyu0a65QYtCktnFaoHc1ZRGVdIy/Qgq51WMzciW6qarRSHT8fjZr/ik5SZmqVscLNJzvOlN+5yr9dhJpUw66/k5l6KU6WjVByDXS/OmxaOl13nKSd17bxYQe/XJMP1U/ng0HA+hjeFnh+MqGCRG4fj9MO0yjbp66Mog7VgUo9LfxdUaAlYzVO8F31GstAE3jiXHkRf/BCUWScbfV7+DBm0I26wKzj/+LeWxsYzzW0i/AIlNFiN+z0+3hfGzv8U2/3oH+862PMuGjNfQdA7jy8JZ/ND5HooJfxOZHI3Bbyx5Rpi9fTaLN9hAobZG9fU5Q4Yxhqwe8U4BkfHjITIScv3/BucI0jMnEPzzZCbk22QyE16S8j85R/Kpx3KebXDc392fnrWfMC3mG/o3vc26+EaG5u+jr2KalMqh7gQAd1d4ARKLz03bD7XcwQTSyxoo8f2GIrd19pr1LN1BssqQSlauXMm1QXVYnCVb3MkQwfjrJbyRt5k+z/bBs8kTwerIj/dLzhecjR/Ddc93ONr0KzWe23qN8+26rdzZt4aclDYaotS0aa7Zg9WiCBVe69le8T0GeWOPdksP99GOJC3jBvNMm2lyOc1fdSvJUZ+joq0vzh1D6VuwFCdjCAcnH6HdrV0C7NHT+e84L/c8z2eFruCTpjTq3ffxi/sqFlRDraERao9D1od0efRHkQ9OtW74lV5iTt/7Gae+C+QacIlGXraRJtMTfP3wx/x8z89gtc3txJf40vwdWzokXrvVChejb6PPT05szJCc+b2DfY7J0u1z6WYzWgWT492xyvkh/RMqddHMMLrh0umCQWGg2Us6n7oom5gY9RMxYhtdmi6avJsxKUy92CtPD/0X2wMhJi+G5AwJFCK3SOtPcO4kODELvVWG2QqNrsep8txMs7UY/EZRlXSGmX9YWF7Sv9e8adI1QfYqEvSbmGDRcOfvd6LuUtPi0W6fH6Fu2YhiM+UBO9k69zcykzJvsZMGWO8nov4RFCZP2/gLPL3/InqTGo1QR4foQaunGoPSQENMOWNqNFS0xbP28ufoBQVKYNBZJ6LSJZlrQQAGfQnzavitQ8GA9FQAdt/7C3+POoFVUEJrJgMCD1Cq86PUJPW3zgrnLW7cn/osY8L/6DVv3HQpLPCeyMsJP1HSksT4yF9I8j2OkBbFRys+ota/wd7Xra2OOZj+aDrHxh3rFcCs6ahh1elVfHPpG6raq6jpKkWlbOXeiCm8EejBs6FbcNHHYRblPF3/Ehvnf8yxZIfAX5upHqW+E2t4BK7PPcSN6FW0miUJRrMoIfgFpSMPtJPWCZVcZWf2lYdVMezdmYgug4itfp1FcfcBoLJdtwgGjEYHK3vf7zt5/5VXOGmBfUZ6KVEdNFk5YTZzdMcrVJW+S4h1KynPpbB5wWYEbKo1V56FvK9IULYw/OelBD9Yy5noJE66bMXUdyV01eJukiIt4aVeaDa02NR2HMG+S2HLGP/LeJrl2fY128VJZptnjnfbu20cEcYZ9t97suoPmF7hTNwYzrdIfoCezD2L1UJB4Puc6zOD3Xm7e90r3W91zC2A6H9B2t9syP6FrJAV6BU2G7xHXYqbizlW/xeNLidsDxHxV12HjLcJt26kVe9HZftAdJVdvZl7YbejtfoT7pFBkOyovf7261YrT0b1YWbwMUpMvRUDSHyVyiF7catfRFayTZpfPg0/Jxst99RC3nKtIKbqNXzrrKh0rZz2sqWb0FZK/iRbudAFvq1TURkDpXaP3gT93mVg4cskxb9Cg9tBx/dazHBsFpMCf2eyE+jMWq5G3Eul50YmRP4Me5LxFK4DYBbkaO97jPqYO1mZ8juJvlIgslc+U9va2PNvPfu23QLHukIkW+XK84jHJvHj/CiaA+I5NuYCJREl0jsnU4EmiHHGTAa7GymMtVI6cC5Kkw0U0JIBdSdwOTSSJcp+eLeNB6DT6EpVwKfgkUKgaz6CUy5/FHxDnftuex0Eq2hnhA4OHsw9/e9BFESm/j6Vx/Y8xs3F6OpFl5uffQ511HZg1XbhU3EV1fkT+JRf+V/Kdv7fHPz7n5Dv7C7/7WDfe++9x+rVq9m5cydKpZLPP/+cnJwcFi5cSFhY2P+JOv67/Lsgk8kcWrfaSlD701n5APEVHzC76Rhjb+SQ5OmgG/di9v2DjKdKMBIdW4mAFpe6Y1zu3GG/ltMwgpymTzg/tq9kCIM9kPPOiR1kem7EKHrhkPkSuWJaCa6x4BqD3qRHnZlFSLkzATV++LRNwE3hDblfEKf4hS4M1BnL0Sklh5VCpoDqg7B3AHqLK6tO/UmNwcvWEFulDC0462twFyEnrpgj446Q0zenlwFmR+mL/xzsu+h9kV2RX7B9WQy10SnUkiix+gBMnciarhJ77Q98Si/j61xq6ygfyPoIvdKbbZ0w9kgidcV/sLzbefu/KMuMcPr8ZaaEb+Ez394ynq1uKdDvbZRCJ9+m/MIdLhAuOBOvaJP6RdRzsnY6s6qg2auJuvCB9Gn6Hm8nLylHyRIr98gf5beBiXy84ls+eu4jREQGDICvL35DfWco7l6JlBzMZ8j1FcSEfszo8D8wy2wBG6UHn7a7keXaxOiTo1F21tqcCFVoSlczJXotI0I2sqX1aY6m9OF4x1oIngnuiTS5LKCP10UwmfBtCCGkZg6uSkeOKJmdsfPPY3HW6WWOJcfi1TGKMVk3yI2sgHm1ML+BRv/pTJ1yia1vvc1HKz7iQt8CKbfd6C0wu4zLqkRUe8bxxltv8NxHz/FCn/0kie9CzEMwp4yOAVtZ8f4mnNft5/biTdyee5Hxhk1QtQcTzsRWvc53QSaSytbwUFsjTc834V+5EXXmk1R1xKAyBuBpjUJh9sBXHQwDvwCg2H8d7//0Hc5V+eTF5tHhYtNRL16P8IfIy96P0v9sDgNP2pwP3W2u2kV80yGMSAeIQQV/86FsCnJrO6RthegHafCdxE9nU/E90UFOfA4blTKOmzaAk+TkT/WLx9/UF49mGRc3BzPjDyPnzGvg+mukOV2296tZ0P6jPNqo7Au85dKAuza11/WeTlQzRrvT+3V9FIzdSwLFvOUlp8kXTo7NISvUJnHRmgWVO1nqs5Xy6T/iVnuZuIsHOKSTEnQzbB0gIJz+hNSDP+LbIOldBVQHYNGEQt8VYLVyb1AUfo95ce75YOr96qWDskyJVXSmRe9Ptbn3OyMTZLAzliRVDS8/+wiqY040hA/EqHGXDg3OYdDvHdI7grDoLyA2tiFaVMgEpb2PzlQd51TSYMpdP2TS5lBi8mPQC84w/hCkfopn4Nt45MQRURROcEUwKr3t0CWIICp46OgcLsXMpmzYcMSlUqDOIAuEtK0EtpzjGZMXkSUhzNgzg8e/epxh54YRlXsf5H7OMpdkJvlqyY3NRR3kgkLsos0QAAETcRarSItdxVdlaSTuX82idQKjTo6yScX2p7qrPwDGAE9qQoNptCVyp+44wsV/EepSiLqzlfCMvfiUXflnZ6Vgxg7+wALRD0LEUs65DOfsuONsun0TR8cdRS5YESydMOkUDP+NpZoOJhyZAIBf62SUZi8p4DYjkx8rV1Brhiv9r5DTt4ZU/+VSTipE1PvCeC1YklYqUGwhM/xxCjqu2+uV4JVKXOU7hDQ48lT1dCQMU0nIwYD6pxlYvM/RoCFrYeRG1AIcnLLfvk8JCBJLvJ8krXW/G+hVevQqHRZRR5tiMCS/Di4RCBYD/oENXJt4hAtDLiBYlYjoYU9/hjq9yXsXV/FLmzen4oeTF/QG1e3VUg6lyKX4a/J42KeFEDnsDYYdHREw+QyUbCC5/EOe8YB6vyZ8G3zpU9jH4URtvMg1oxMXnPfS5pR+6wEUSJDnsKIsntv2TEOj1TBLvo9lA1ZyR+LbLOz3OdUtGbSvq2LFJyuILIqU9kBNIMzIhJR3qHJ6mR1LfNgauYVRovBfA6UoVbx/YjKv9PtAknrqwXoWrDKGBf9NsEsm4ZpmjFZY0wq/a5O5VDOPlq4Akvo0cb8bjDo9iqkHZzPzbB3BriGYLEpOl82n2CCBbJRGf9x1A3C12PJTBE4GpSdDSz7lRnwXp5f8yb5xb9LifFHqk5EbIG0ryywzKD8+iHnb5kEPB5gogmhRE1P3PDPcXyS84V+Eq/qDqQOOTCKw4xtmKwO5zbwZUR6IVeyNNO6JSJXG4SZmn20NVwsQKIN98qWSzbCrL5Ss58DZpbgcPk21cjG3VyrBfxzINBLT8sQcZGHj+eSZT2h58gceijjG4No4uPgoVOygNHAp/cwSW83c8QUZoY9j0hYTWPMxY0N+5KPRd1LfdgfFfp8BtvU8bCGWAZ8R5p5JVshz/KIaSJWnxFAaZy0Ht75ky5O4NPgSzV4GnvnkEaIKo2wB4Tao2k+43ERESQRhxUb6lI6hn/q2W/rDKpj5sVUgoAjU5loo/QOCpkBHEcnmh+kY0sr10aeoUhsp1LwCwRKT9+uLX7Mm/H4qgyoZUinlUnJv2wKGFmRWLTlBr/KjOpEr/HSLXdrtXDRbrFixECefzKy4WbgobUFtbME+TIhWBS5yN97r2g67YqkTQyk73Z+RJ0PoVYJmQtzTpKlbiNO50OrtQltAKgrRRqOWOxF58S7ywnvfJghIyO+dcXDubvb6XMGlXw6Zid1IZhn4pVHn+yIvHtvB2PzhhJrH2ueRTJDBhX/xsNMizH8HEbrOxN2/PUN85VzpC9pyYKs/j3g1olfpOTMqk/yY/F7zz2GvWxyOIlFgcdKbjAzbzBOuv/DSHSfZsCydlPJtbK37F4z4Qwo4X38V4p6k3m8UvqWXCMo92quPu58dUxCDGhU1/NdKLeCpaMHXqZQXWcgDlZdILdrIN/3/Zf9MpTWC0VO2cXLRLs6P0hGmsM0vlwjwH8c7GWWUbTvB6bRK1i1bSXb7Oen61RfxubScw8Gwb9oevnrmECeSZ2AwG+D4TMh4EzySUY97iDNjXKgNqOU273LUrccg6VUY8CEGQUHyFYl1cmT8EQQcctNywYSLCIUD02n2j3CsAwo3UHoSIHPFS5fMEZf72T1IZF/DWsdY2Jl9FrhJjgsgTfEXViuMSx/IqJOj6LlGKWVdJChUJPs/yHuvrCErMcuxH6j9sPqPo6CHgqM9qCZ3hsAp4JHC+jYXGswgN7ujMtmcTUcnMdtplF2ZoVOdR7s6k05jp/1Zo7ze53D8BdxFqPLewFHVUwCEumcxK/YrOkNeZ/dt5/nkxd+5MuAKAiIvvQSJiTBzem8gxM0BpkDLfgDcK8wkF/Vho/dX8Lft/etqIOvwCLzdX0CvEujwCpfqLFPDIgMM/8UORrOIXZida6WzbdAMzIKGira+PN8YRENhKB4NbY4vbTjHMOFBfJ3LmePkx9T0djtbtns89BYPHh/yML5JI3m5swCl6iXp3n7v0DlkHd4i5MblotPo8K3ztY/FwsR3SFNf6dVGAQG1Gr54YRd3u4dD9IPsL3qZ4AoNvpdfZNWpPyFkDozaCEPXUqfwp6y7u3owIWvjd3O8dAl3xj+Gyio59QUBqNwF+gZKog7SaJLWt+kNB0nLukqYa5S9HjpFJRnhj1Dj/A85mAA3azjxFatI9pCoCO2aTJTBWdx/P3i4KnAyhNO1/gyacg2+9b6IPYJ9dsCVTTbZHvjqUVoiyjk7voH94Ss4kdifc/UH7G3ICn2aT/MftJ/ne46F0uqGW1cSbpYImlxOs6n+TUJi1rFzscDxUW8w4+JM+h8oxUkrQcJk9jyF3fuQpReouH9/27hYHfZsuUlgcwfEtq+H4zNgyjnqkz/CapKhNCjZftt25MHVeIh5ktxvVy2f+T7Ge+cjefibh+lT2McRVFP70SiESiESq5Sz71YGukBE7eNE1T9pzzEr1ambldhDRcPOilWwPfcZ/NSV/DS0gq+fXs3HKz6m3k+KDPs4VfDk0OV87pbPsswrjD15L/2u9rOP84CA/QQ4l/BTGwy5IPl5biTd4HTMjxIYUOEGfmO5t2M0Gzug3PcH0vssINd00D5OAgIzK6/wlJhnr7NCVEC9xFR+r2gqHe4mtE5aeq5fT+67wgd1kqTxlP2TGX1i9C0ynnawmtBtzwoUNA3i9k06LuvGkxGnQt70Ci+//zLVeZKftLytLzvznkBY2EHX+Qw8mjVk9Gum1n1Xr/lX+Eg1CoUEvmnTKfmBDqpGbpOCIu198VBo8ZE5wEMtGmkuPTz4URJ9T/B+aDoj1NLaLQpmVHItWqM7KnkXC+RPI67bwXz1OlLKfukFuvNuH0NK1h2MeXUMiRmJ/5gWwWw121mrJrOaa23LpLQKIvTb/yEdX/+OTKPEqOotHeCu8iKx9CTWb77DbLZiER3+pO6fLRZIO/AKB+enMsDnTQYHDbbn7GvybKT6dDHOVQX28QVQym3BPlEvBfsEM7TDhtW/sFP/n4P+9pj0PP+HwOI9ubzxsi11RTcDePRWmHKRMWVzcSmMAMCjUYdB3oCs7TqcWUKILgeDysC5EUWYfB2B7+66NTmd4WjJUfS02AFqMlF2S97tQYXbmKXdYv/dsT6ZqbVm0OR6gmZjba9xkMbCZA++dwdFe15306X06itEGZy9m74N6ykK+AS9ohsQ5RjnI8VHeCv7Dsr8vpOuIVBny9kXb/kAV2Ujv9ROpOvnP4nIlYKd3fNkk6mGJl0QU11taja97AYrnWZ/zBY5SgFaXM7zm84GMHOJIKBkNVtCO9CGf0TJPW/Q5ufID0nZJpK6DjJdN4eyaH8qgyqpU0nvMLVSCqMutySEfOgyujE0fy/eHWmOdrtE8p1sBLs7pRybR5KibfPeAjUHuNg4l+WXJBnSCp+faXW+iIuyBVozcbJKa7w5KxfntauJyY9hatBSAl0De7Ud+EcZz54hl5nO8KrHJegshcRXsLr0Iat+KAYFnB92jbLwMuk8b7WCRwqrneew1aTn3FgrjcE9xnLgaskfAKwPv8Y9je9wNm4sm/t7oKn7GCq2cbxsGWcapHusggWvjpE87hzD3/P9cO/cQ89S11nH/sL9FL1ZxJvim2TnZjsu6rsQjXp7X8bPjsftyXswKp2wOLtSmjyTm8u/g31S6RXD+B8o/+1gX2FhITNmSKgDpVJJZ2cngiDw9NNPs3bt2v/F3f8u/y7/e8VqtdLW1obVaoWWayiyXmWIJYnomhcI0I/BRR+Hxpa4FcC/ZRYjs8/Qr+5D/onZV2WRDmW771vHdw98zNUOacFbkPAeDw96hP7cTVzRDB75+lHJ0LFZQYXNqZw3J3H7DT/Ox05lQOEf9K34iBomgUsUXH0BVc0BFvvqqQoto3LovxiWd4go5xTI+oB4xVr8wj9kqyGSK5FLAJvB6ZEEKe9wpWkhp8oX0m6SUI12x+/B0Zhlau6uhW237+PEGAlp808ObTdtCmmZV5lvcmhvA7iYMxkUtJvwlvno877jgmsGxD8Nu5PhwAhCMofiVp5HxI1d3HdmKG+nZ4CogqvP0+TWj69vBOLT4EaH2P7fQkK36GSsy3qOEeWAYEFpdcND7WGXZFDJtQzxKCJcAV8U3I5J9GftnXPYUH8Sw9jPURgU+Nf40hScgs49oFebTRYjFrELnabDns9v+XKYd7sCp5BUZNpyBmb9ztBDl7iu+5LC9jApyLTZC66/jiiINHs2UxxZjElukTZ9r4G0jClDEHU8N3wxmfoNjvkTMBEGfYl/y2rmjh7CzmFr8GpSMvbie4S4ORIMdzMmuo2vW+XRHE7FEaGb8ercBqIcVN7URNyPbxFct5gxqpx4oS4cBn4GKm9wDkUQ5YSXSt45S3IOgeo26kRJhxylJxfEYIrEzQhYcSltw9PqgUtXOtQexlnRzLDgXXhxXTrIC254ajwRwhaiT/ocs0VOStk3PGouJLhpidTXHikQ9yQlRjOe5ftQtRSxcfFGqoOqpXZ0SEwlH9l1WpIG0OolEJ0f7TAaYx7hTNjj6G2MhYCW2VwtWEGl34fgMxyGfEdV5IO8qSrkeNopIjtWcTFcxzDnN+DqS7B/OHeUr+LTe4/T6q6lsXUXtR7bJSN8ajqX9WFcCIU0TTezz8HcA+kgpjYFoLZ4IyCzH4q6nS12Zp/ViF/rDM5MNfDKkivgN4bOiM189cnPTDg8hcywJ0n3kuTAGPwVDP2JRa46IuRwZVAlByYf6LXGXA5Ywlv57iiMjoOMW5sbVpcYGPARBEykhWTaLH6Yug/UyKCjBHfTRXw1NfTr8ODFlU8zeack6ycTZaD05Lo+kCbMGHsmf+5+L9pyaQ8fQNKpLcR8ncUT3/7Ei97n7J+TizIEi4BeZeDG4HpqAmokhLW2ErLeZ6hXMukz3qAxOIIHfniAgCqbpGDzNbjwEEU+SuZl9kMrq6Q6oAG9vN4uiVTqN42Dg8+x9oG1lISXANCo6KLZbyHcWAlAhW89Wxf9SUtzC551e3h44IMg09BsSWB62B6SihyuV68mLzzogqo9pGszUFTcQ5a4jW+XP8AfGgkJT+AU8OxPgHMuiSe+wSxTUBWT1ut9c1Y6ozR7ITM721GVbm0ZsMkNLj3GcfdJrGuDzKRMjOPP8E70AVoDngLfkdB4jndcGsh/8BcujB9EQ/ggyXlg629vZSP+Mtg+Zzsb7/iWUpMZXPpIwQHASewtH9mTvRLvmUxM9SsENTvkPXs6EpqFGVzPGETE/klElPRhvbEdFrRLAZWqfRTEn2JmcDtOWic8mzwlx51MBYkvUzD6ENOrYNu8bWxYKsk99zK0455gvN6ZbaNOUBtQi2iVI2IEQxNy0cDmphDeafySFpdz5AW9RW1nrf3WozXPE50Vyw0D3FcLOzttjrn8bwGYUAmNXi2cHnGaRu9Gh3M351OGCZkAlPl+z+5BIjucb+vlZHhc9QmTa0NJvTCUJzfcxSz5fhbEv8nshO8Y2fcaM9oSKAortY2hmyPA6Z4ASa9Qpn4cqwycMuABc+++/4/Ksi49J0+fxUvVTKCsd4BdQMYraXO5O/YeFl65l99shNKftAMZeu/zzJ9WwcPjfuF7f8iccQ6ZyYh3jZQnbuZcdw53/oSrt5pkpWTHSGPc48tD5gBQURKAvM6HDpcORz4tW5HZ9qlGr0asgs0RZepEU/kdd/T9lMcUCSzwfN/hfGvNst/77ontdrbYzd9tdwb8A3sAHCj9wzoIL4Ec2VDACoKILug2vsobh0t1E8t+W0ZGeixMOCLlIbz8JHuUA/ihRY9Oo+MPWTsH6xNpUs+Q8gbVn2Rg3tM8NekaDUkFXEneQon/V+idwugceZA9pS/yWt40dneCReyBkBXlCOWbmRn7JVpVEbWydCxiF0HqaPY5jYJ+77DI+CdbA8GgMlHv24BBabDlMSyEY1M5HVzAjcGXODexH3WRDoAYnaUE5n9KmkbalzutUGsG/+bNUu5YoKvpOv6dP6NP3sa2cYeZXBdFkcs70HAWSjZyb923XBzcgG99IDUnl/HQrhyyExtgsycDKmL5ud9PDHDJooNa+zrUE519JfIu3hJEiv2+uGWOOgBsJqJqn+b83Fb7tTBrMUfGH+HKwGriT/2ArM7GPvMdDt5DeNG9gEmnRzF2dzFyfadjDjRfx6QOZG9v5SZyjDbEs0wFzpH82BnC6dGnODTpiGMsAidTF/g+da2DGFl3hjt0R+3zSC7KofkazRYXbtz5J/XxWiIL3UitteUmVrqDz3BkgFFp5OyoGxTEFPQCOshlPQEaDkfRbXFf8OTQ5fgrmtC6tZOQEcbkv85ypfADiFgEIbOh/yrOes3gtKmCM2NqOTliCw2uUt2p2EHSqYms8YWjY48SSRI/Cv+1o/CvCpg2dQTxPmd5st+/SHAtxqd9HENjX5Skq4Z8T7iQiYfGiFdCOzKPD+gv2hDScU/AhCO0mUJxr8hh9LEQFEaFw04a+iNmdRDVtmBJk+sJGl2P2yVvATJaqnj32sME5h/koW8e4vPQC/hkTrdfT5A1U5F2hg2LN9Dq3goIyHV5sEHEWd7C+Mv3MGjvEpJO/EqGSmJW02c5hN3ByhEjCXPPtK9RPfcpD5UXsZVvkdjkyC0t67Fof2P4jvsPxeGzeRYTD0+020Fx3ufwcy7hs7JqDhY8SnfAzH5vRxGy9lxce3W/gIslS2LL9v8QgqbxTKM/JSagJ3sv4k6KzEtYHvcTc20iL8eTkrhQfRqM7VC1Hz9lBu5yk13EvBu8kF49lcLmFNx8t1IY9CkmWRcIEiNnxAj44APw9LDZj1Ylrtok3I1xhIZCaCgkJcHbxjWM2xdLo1MnRYtt6Uvc4qFqH5T9yQwKCOt0Q2GwYlJoeq/5N97mNz+HXWi1CPgLR0HpQW7ALqmNZoHxR8YTXJjHiYbNZNZlQt4aAJ7Zf573smuQWxzSeN193m4K4kDFBAqNAFbChS1w+SkwduBy/h7yI6A6sBrfBh/CysLse/MPV1bjK1OxqGElgU0LbSNhq7RMDdoy9MXrsSZI0momnYyS1n5gcbC7XKx6+1jaWTGA2aUvLopmJoWvodOYQ7s6m05rA1x6HKr34tGygUjb1mWx50jsMSNu2g9vZla5C8FE1zzP2qFniHKxsSxEGbNnw+TJ0me8n1hC+W3llIWV9WL22fMm95Cy7jlWb3p/yJjr7ty5JoKBxyOkvhYdAA1H4M0Rsf5ntrY0d7dFS8BiT00tycOvofUKZtqeySgMCmR2Zp8D6GAPtiGSlgavvAKp/R3B927QSJHrEgmI4xaHIMAIhYy0k2mElYVx3ajAYHWD6v2wPYwoQwntogmVXoVruyv24JahlX+Jj9PpIP/fkgZFECCp/AsSyz6zS0tL/Saztd0hTd/NrOq2Sw7XLOHORhON7q0YVAYH67YjkmMlS1AZy+lTf4Whp6zM/XsuyZqpkry1zznc1Q1s7oCC6AKqAqs4OOkgNS4ZkrTr6C0w6s9bwEoKQQOtOXhWvceixLcYFrAeGQ7go0KmgLE7ORdpIapsK13Jk0jISiCwys8+P25P+IBvgtej1CsZfm44E45MuGWOVArnaXA9ilHWau+rhwY+yn39VzBYs479ch/GHZ4IwN0NYzjlX09Ll+RFUcqUKOpaUXeZuJzyK42uEjCF3K/gT2f8jw/A5B9OWWgFXk1eqLpUmNwSod/bVHYksCH1W06FanDuiiNRdOcdTQV4D6XLcxLuqnqW+hbTVwlaVRFntTVUaf1I8D0NQF25P/prRmKrBhLYNF1qk8XEY4MfYHHXHfSt/AhNjYakjKR/lJY0W8xSXjHAYFET7bwHyjYhilZa/WMQQ4OoSa/CqUOaEx9N/IQvp33JON+FNPjHQ1AQxhPnsOAwQHoG+yyiHE/9LAYp7yfSMxI3lRve+lQ821Io3ZOJorGINs0NqjrLAFB1B/sEAyaTzXa4DGPk8v8S6G+EKEN7ACK+jmDCIWmcZTLALw28ByEIAt8/8D37pxyg0buRaHUz8rxVEP0g9W5SEOz8iHxq3vW0Mdwdwb7uILgZ4y0S7T3TbPRiudHbV3Tzu9jtw+geC+tNwb6e/ohOdT67UxUc0L8l/UFbCVH38nOnk+35tne1R13EW+wikXZziBRcByZE/cyFJMm30DfTnYdDvuOulLvgzDJmyVPYkv00T9UbOZgSiNGW0kYURBBEfq44zqLInXzvJ7DcDdIjNsLB0WDuQrCaMFmkNlQqD1PjvoNWg405PWwdJrkPankHmUP68P2D3zvOLT7DYehP1Pb7DHcR5D3y0wkCUPA9VO3hlDyGDNu2pVUVSddEBSzS83vBVuIqPuauqOe678S9jwQicbFkkxf4Jn+ZlyK2t+HcUnmTtLfQK3CrMLvbwXrQ22YarIJZTiVwYh4ET+fiwA3Mrj3CkD2fMOb4YEdftRfAFm8STWVUmyHlXA0hF16j3HtdD5bmAxB1HwDeTpV028s5mom251hJ9SjATZSAF2kFp+inX8P12vEYRAkslN+Yz578PVyvlcDItf61ZMVn8eeRUnudgw78QsLJ7+y/y9VylDHhXJv8HC3Ln6HL1ff/2mDe/6r0imH8D5T/drDP09OTdpt8VnBwMBkZGQC0tLSg1Wr/s1v/Xf5d/reLxWKhqKgIi8UC/uOxOkUS6XHNdk36TM9FRm0KwLNzOK6G2H9k9h00zmTJt5NY8MWjWESL3cE6IOAA02O+4bHIJPxVhXSp9XSpuxAQcHWFF0fezhzhUQxWHXKzK8HNi/BvnYlM0Eu5avq/j7LpPC+MzeHAv36i3S3AUbdxBzltWse3gz7kIx9QmLwJa7yHufFz7Y7Kuq6+nIudxFHNE9J93QZW2AJqvcYC4N7sTOrlVDyaPHu1udvQlVk0uOn64UtCrz6MbRjHEvenSdn1PkG5xxz3Ws3gHMGXBg2XF28CoL7zOrVdiRKDbsoFanynIDPLaHVrBbX5v4WE1mjcqW0agW/dPgbn72Ju21Gan29ifMt+ODCSRkMMA4+/z4fNkqHTKB+DU9hw3n0X+sp287BSzfS9E/GsynT0pbYCjkzmJetu4pt8eO3tFUw6MAkBEbUals0twHngM9BynXHjpMOz8UI/anUBuOkOg+cAEFV863yFaa1+lESU0OGms6FvVeAcypacp7i9GrT2+SVIMpuHx1Pk+gCftECdXy1VsWl0ufjc5ETtzlvwz4wJh8ysmTuTXyOmZh5U7oGWGyiNrfjU+zD87ETmH77A3IKDElW/vRDaC1jRvI67X/mZlStXcmXeTg7UJ1CvmgNHp8Oe/sSWfoDXqHPUBzRgcvGkuqMP1cFfQtkmUrx38WraHBLkXzv6EsB/DGJHNhvm++CsaMFikdC0awqfZN3VdRCxlOKm7eTf/Sm7ZmeQVPo1g+vW4KRwguTXYImVRTVfciw5He9aExMPTXQw+/T1jCj7kvedltKn5nkArtVOxKwMgtI/bf0hUOdfR3VgC/HpHXh0NDFE8xbUnwJdFYTM40bzHH6+/zTlj//E+akLiRW/A89+XO66jq8MVAIENt9+ixO1u43d68QA4R6eGvoUIW4hNhlP6SBqQsqzpZApUG1xh7+cUJhrsRa3EJGTxdLflxJY7ecYxPCFDMqIJvTzB3BtU3C933VpjlitcG45mrZMfo08yycrfuOLx7/gx/t+pCC6wI7Uwz2eBZVhHE/fz8A1FYSWhSIKcshaxaDGIbwyaAV/IjECfepUtnklgynnWdOcxncPfUd9cAtRp58iw0fSWafxIuyKZ3dYG1fSTklz7CY2j1wmQ26Ss2DjcNRaGXX+dZIReu5uKP6VpMohvDVuEi3erhyceJBWLxtS/tKjUPAdGTUepGyai7F+A2tNqVR4/4rKUgWXnqBLFchXbVAVXIVGp6HRp4GNc6+iMDVCyFwaLP1Z4wcD1ZD+9Qncs0+yrSkI6k/jKUoBiiT/Fkoiajk6Rc+O2Tukd8c5nAaDjn5XI3Et7Xaq2BqlCYLmq7j6fcyVgbVcGGVk7/A0fi380N7mhwY9xIKiRlLKvrUfmCwqqW8p/B5BEEi9nMrUvVPptMKVtiDMqiCoOQSikrfEUVzw9qQitJYWp4tYRL3EFMpbw1fx89kc6Ojf56r7w20FoHDHNLOcleUOjXxpPgo9fu7+6Z8NwCt8xLd/vIl/3lniz/6MCRdQuEDBWqjez9ZWf2qrvLlz/Z0k30juLdviFEZFD7+ZYBUIMGyBg2lS4BZ4UVXGU5+/ycNfP4yAjODrqaD05GzXR7fkkRAFEbI/gW3BeKgqqTOq6LLCujbosiqg7iQMWUtn2mViz1oZdf1XAJq8mhxt7v8ecsFMnMIhPwYCUQ4QP6dNY/CcfoIG/1ZkmmTqZeNgwhFcbz+HcuJWRFHkt2W/8etD7ag1H0sOHYsJdLU0tJWws/Yryny+x6rnvwVK0Wpbee78akZUAIIFP+NQBnpOQGnyIb9xEB1GbxoNrkQr4EYY3OvaQUwM3BMxE7fan3jt6AEUhu8o7zuRTvcgBAHuugtef66ae4TfeKljBSGNd5If8B6VKkmuj8KfIPtjAA7fCGfooQm2AKbNmZS+ArI/Jk2ooG7aEfYucGX8jRKbY64Zl6yHuLvfyzw2+F7K9DeodztAk7kUqiTw0sWwBvQ29tz+2FgO9gugpLWw95j2KDc7UW8O/g02H5Zk/2ZkkhW4iN0jl/PrXb9S419DstoMDRck1lbZn+QpQnE+mMar777KyOO/8/7J7WT774KxO2HoT1L9vOvYMvcghdG2OsmdyW/2oUqbzJ/Vg0nXO5yoMlEGulqIe5I3ju63O0SDG5fyc2o+D8zZA6HzcLZ20r5rLDFZbmxbeI1WH3fJ+eEUCkovW7uhJriNZpcL1JkkdDi6GrxL1zFEBWDBUybN0w6nVDB1wgaBNouFPqcCiH13EqNPjsZJsErSpNdegfPLKVNFsV8Lu+bXUJQ4h8r2OEAGfZbTqR5EonMT3iKYrQY8OocwwmcGoTaQkOQ47nbsGLlk+plfrv6C1iidq2SCtE9ZezrDh/4EIbMJp4D1/RqoCijAairkvOoFx2f8RvNEUz/OR+UD4Fr6NuntNjTviduQ6xvsuT66yz7tO5LtN/06DFjFl9ooznX1cAj1cGb7OJXjr8mRpEkFhxOLKee4v+k5tvTJ4cz8Ni7NfAOjjy2PqlMIyNTc49lga293rkbH/OspX3WLHCtwyud2ZpYo0GmMtPrFEON5BM7dKwVcEp7n6cOvoTz5Pf0uqUnvtxmDwqbccPkpFMYmxnkGcG2QjqPPHuMEFs7zn5fzwDEzLJ0/jaz6Uay+tIX8ZskJZNFESEEz90QarYG8Uq4ipySgFwoaAJOOYKd0uhL6cHFoFR4tHg4ghP9YGqdc4c7a3t8rikrwGgxD1iLP+4znCnYy+uQQAmql1U206KT5l76CL5wO8GxSDZXBlSgNSkBEUDhDyBwaDcn0L1lHUE0IolUgQ9UDnOs9mC2V07nh/gf1isu3jIWHypPY6teIa1zRo149QCmEs1fZRlloGX/P/tvOFJodt5oXRy1EEKwE5h3n5XcfYcyxMdj37sMTcDowmDM9CKmCVcTVnA2Z70LW+7Zxl+bC9YgHyA98R+rThOe5bnqR++J+ZK7Dn4ZaoZYAaMem0mjsy5Dr42iy+/ykddVkVhLkUkiIHGJzY3n0s8XE5sYiiD3nnzTHXXWJjMm6waS6vchksGYNvPcedAjeNLdrCKwOJKMhlCUt78PMbLjyHOSs5kFxDqvG7eXt195k7+AYtLIqaZ+q2g/FvzJB43iPLWYLfuIFKFmPm/40Qa55fB1YjiG0iguDT7IyawF/Zf4FQdMoYwG1nZF2e/bm8+0U36eQiRau6GFvTA5jZLdD7ueQ/zWy1gzWtUnBvj8XHaUyzILC5hjWm5wIlV8jydgPr46R3aMh/RcwEbyHoWrLZEVwLdWzD9IcmMBzI+6AE3Mku6DwR15s+Iz1trScPZl9MnMzu/MfJTkjjTvzx3A8KYEjXR/D4K/BazAeTetYGyKtBd0pM3oz0G9Gw9+0T9l+PV6znaIO6ZzYbWd3WdvICnmW7aHfUDSvSAKFWuV2gJo9H6XgsCt7go9c0DGgJhiFUUTdKZ3rupV+ekrw9cpfZ6tQl6KSvMA3OS9+al/j9iskCcrW4OdJt8xD7+pNXH4sFtFya86+Hsy+7oDcsGGgVjuCgfNcu6iLBEFQSEoVXXWElP7EsSgjOVN8yEzMZEVZIL/rW8E9EeKewiIosEaVo1fpSb6RbH82114ikEIyGgcyR/a9vQ7SGDiCfdCde6xHYKKb2dcj16VP+wTG+t2Om9mbz6ak0t/zJJtbZAy4OIj7v7+fwErJcNaZXFl39UMqY/dSJg6iwyOYsoQpDHW5A4DtOU9z999l5Brg/LDzXO1/lft/uB/fOl8HYElXwwJVCakqR5WUghpaM/Ase4WlKW8wL/p1O7gTbEBrQGFtJMbrIq66UPwaExiW1w2UtqIQDXRaVIgWqX03km7cYidt5g7OxY3HKO8OaAjMiP2auX0/ockSw2f6LrL6X6Vg4EL6KX8hOMqHiVE/sW52CJx/gMixnfxxbwdVQVWOsSj8HsxaUHrS5eGLRqdmyoEprL84k/A9EXDpcUQRThU+jNL7VUbmnuKOpsP00yggYilN/Q5woWoWA9PvYmu7D9Vef/FawXgKker4Rekopg3dR9EULaaffkPTYZOON3UwLORvHkh9Cq27tF+b5KZ/VpGyWuzMPsEqcrH9WT6qruX2GzLOjrUgDhnI72nfM+iiJOE5OnwUjw15jH4eoyh3j8Di5o5+xwGM8jYAliYvJcosgVcsFij89iB9Lm5EZrN54nzimFN3mdSynSzMfI2z41o5kZjCa2cfl8b7H2Q8nTPgHn0PbeL/pDxoNlFhlphMAy8PdAT7ALRVjHeqRBlcx9nhZ2j1aMVbqUesPwruyQyOWcCozIuMP/0uAeVBBDcuJVJ0MMrEHmAtu4+iG5DcI8CGYL7lvA7dAERLr/t6AsXMVvMtzL7VU1bjJvO13W/GKpoc72zpH5D7GbVmhy8Ketv8PZ9vq5z0b8oFjjjnkdswjG0GBdnTDlAaG8UUnwcZEzEGFC4Y8OJi1Sy+b2tFr6y5helmRc7h4mW0NDzGxzY8MQp3OH8/Yv0RPjy7jphzvzBy7SkaZU9R1G4DMEbdw4W4DM61+6KX24wlq63DXKOhz714FX5FSx8Y79xDMgAg833I+eymc0/vXLyhLleI9LjaIygjInePhEFf0aycgFnUURAq1cVsyeF47d80aqVApKQUJfVZYMs8Fpe28Peiv+3PdjVFkpZ5jSTtI7zVBBkGL7sqiCiIGJQGagIaGHauP8nXkyXbUO4MYQupkfkgN4skXw/Cr0ZOrftuqd6t2VBzGIZ8y4p0K2fKb8cqWHjKA1J0u2DcAW6L+Zh1Q97nXW/Alru41RiK0aJCaZLkrtffWM+MDTP46sJXAKQPTOevO/5i30k5RbbMI52h8TQHJd4a0LNasbZ3IJoMt1wTex8h/68NBvaKYfwPlP9ysK87qJeWlsbBgxLqc8GCBTz55JM88MADLF68mAkTJvyfqeW/y79Lz6L0Am0p3u5XaXG6xAXXV8gJepW6ropbPmqxwMjsM8ytziDSI9L+9wHyy/T37cCoMOLW5oaTYEYhdvH6sQN8c+krzHSiyktHZhYojC5EJoqsWwcjw7bg1XqccZrehu4I2QNS/jtdFfq4ZwgtBu/zqbiX3QBsC55HIu1CLJ/nLWZtKzjrY0gtX8eqSY5k4KP8vyEr5TipKkgzpfKprxbqT0Py61jd4lkg68OijfO5bedtBFb38DBzs1TdrcWTes7sGoFoEfDXnmFmmyDlf5p6EcZs53OtJwfCC1jzyBoCk/N5e2AIlP8N3oMJqdvDswMqWP3MahoH6fn+v0hF/lUlZ+iw8ShQEGUci2/7BGmxr94H2R9CwxksKO0GRlTIb1wxx0LqJ1C4jqCMp3g9UE157CDQ7uWG7wDa9K3SYb7mIInWSrzk0qY88sxIh0F64004NEbq+9RK/lqygUuDLjHE9wIqYxFMOAzRDxAhatFc6kd8Tjz2Td+kRaYtpLI9li0ddIu7SAZCq2QI+nUdZ2cneNd7YdC434Jo6R4Ln7bxjM5MZ77SgYYBh3GXF7yS++ptwZTjM2BPCsFFXzGvog9T9o8i7twvPOU7BfYkw+HxsDOGLnUQxYYg/FpmUJX/Pm+d/YEupwGgr4OWa4Q0bOOHkaXkPrqZ033/ZMDgibjUfwkJL1Hd1Y/Xrz7FqqZiiv2+YJ/6Ph7Zfi+6tiLwHMjR4juRiSYsFjAoatlW/QXT636D9KeZY97C6pR6lAaR0UcjSCxfiFqutrdpa/vzZIU+y5kxjeyfst8xFppgiP4Xqq778WkfT4XX71R4rce34UM4dw+cXkxA0Rr+CoC4dmfcqvP57fxK6d7EV2BOKdZ+77L/0nKSrwYzVlQgEyzEid+CvokMY18iS+CgFty0A25l7olQ4vsNWw0P0ex8jjGyF1k9dTVRnlKEQbAd/izW7sBsz3EykDf6bvL6DyC8NBxVly2PSPN1qDtJs1mOU4czCqN0kyS7YIGin/Bqv45FZgHRiQmHJ7D8p+X4tc1B3XZZCszWHKai4zJDz4Wj6ZAR2DiNUNNYCJ1HkfsqDpXOY7NPBevv+oMti0ukeWUzvhe4XqUiEjzlbRiUegzyNlsAoAWAk1oFp0acofiFORQOuqPX3JOLMswyM+kDC1D4TGN4zgmm+jtwj/ViBPm5LmjlJZwedZoOdxsCPe4pGPoDSyxa1i9ZT8qlUB776jEERFTWBsj7Eg9tPi7tLkQWRZIXm8f1/lf4pgXUHRlQsY1Sy0K+KHYjfPtUDKME/lqwCRDA0ITB6sbByrF86VTPkYlXAStKvRKdzA2i/4W3awcfPvchTQFdzNoxC6XedmgXlZhm5nJ/cwk7Z3xFVlIZnZpcWgwNvdrdPa4Di/5iSu1BQkIc9sq8xr+YUxrDsPPDmLz6R344vQ+5qQGOTILsj3l90UkGXDuOqv4HCjxvw6CqlhhMlyTWxsZ2x/d4yw1w/XUo/Qsx9xOGu7T0qkfPg0aHsZVWzRU61QUSe0Cf3OuznpY/mTB7HfVxMrIHqRkojIcrL0DmO1B3lHcrFxL4t8RwzIvNk5yVhlbYPxzf3A8J7LVMCyhphvZcMGmh+Tp9Gr7Go9mK0qDEXWZE0ZUPFslBvXPM46ztEdsWBVHKH+UcgVrWQazKiMrWpys8L8PpO8A9Hlz64KZswL2+iJFnRtrYK90RdyMxlOLX82yLQEgIrFoFa9fCJvN9DK7X8+eSTK7FPcZl9yOSPKRTCMg0zFU2kqoC7/Zx+LZPlIAVHUWwLQCfXZFM1T9Ou9MNBBX/LVCKr7rREWzBQlrHV3w+8BCenUN55sBF3r+RwyTvPBKUoDZ7sEi1T2ImRN2Loe8HXK2ZRJfZldOD13Eu5TnaDK1SLt7M96h3XsrJkntpdD1ObsgrlKttwZbOUmg4S07KGjZ6Sp2dmJmIfS/K+xLyv+FZ4RzzXUBl9EdjDJbsCZUPHUMP8tGZ33nlzPesa1zO+dgppHdtBd9REDIXlbmMkiHufOQs0KHKR6+otTtnAIa4zWVk9nniKt+2jcWtyHiFyYsQOdzlCqNMf0OedBh0az7PF75gjCvi24e/ZdOkbDgwVGLDLuykUB5Mrb90GB98ophQUSMxRwHapH1079a13PbrO/jUS6d9hSggWAyIgtk+FpaeuS8qtiKcXoCrSpLJBckZ2XPN/lXzL/blBhJYrmF09iUm5GZIMjtqH7i9kaiK/uisUOm1gdN9h7NHu1K60dCCzns4F7vcUJi8eUo2lJwI8NSesT/brAmiSS3llBl/dDzfu9YwsjlOCoyN3MgJrxl8u78/07b7426pZHDQLpx1Z2DoDxQE/U3q+Yf4s0MClsRVreSjAbuYECWtQz0R3lbBzJ/6e7ln+z2066XFxbczjclX60krPGn/PH3uhbS/EbESJhO4kXSVT59ZRVaEBPQg+2M4OpU+Xq9gtr5LrX8bnepiyvXSeY6EF7HEPMyXU74mpeSH/+DtcDCiuh02clEGBd8TXjSFl0fN5ZXUVOm6zSl3s5PIgkVycvV0Dkc/xMcN/z/23jtKqmpr+/3tXblzdc45d5NpaHLOOUlSQTAgBswRFURRUVREzAFFJAlIztDknKFpOtE551hd8f6xq6u6wfO+5733fve7Y3xnjeE4nN671l5hrjDnfOYzAwCIK1rOiCu1TPV61/bYHnVqj+yTiaLkIEj6FkffUYw69DS9TvtxrW8xj8ZOgey1oCuD6hv8rL5NsHsDVe5VUjBqq5AkfQuDD+M3qpiBZ1+hz/U+KPuqGCHjXzr8zgPj5PDxDPDxcKKyOZD1TeUccvmIPI9fOFl4mOzqbPDqRQExtNyKZsovg+h64GOamnKkSvK2wukZLIruRuiw/iSd96ffyfaUcA/mzATRJQqGHIbAiVhEDUWlWgA2PbSJJ3P6UB31nUSHWLCDnwxdWV+p5NXPXmXEgREIFkG6f/XfRqUhjuERP4ISKjwq7Gu99BhUXWZplUiK1woaZZLxp10OdJuB3w5KaYtSdxHK8Q0rZt38X2jwHc1b7nX0D/qNHXdfYM3Fb5E7zEWmlPJRtXNwRj+DRenOtoZ2PaZKNUz6Z+5GKD7I116lRCskesYi7WY7gEtQ8sjRTbzR5qhXyVTSWdHjR7KaxxBa9iwR5lHWmiVnnxmRFw+dZnEl9D7TG6dGR8Kzw9uNf0+fAQy+kU1S5u52Y9Aa1R8iZDJg4HVSpl6nU942JnpaWR+6roTuX9Mgc8IvPYr33n+PuFRfWzQ2KSPBZwDrvHfhU91KISxw1yzlHXJv/AtnZRVTtNWIz6wjNbGVqk4AfQ3BbGF4+M9Uu43nWuhcdLLSdvMlYEHAgkyAcAVsbgiFiQVQcgjBrOezgiGMPPQIow7NoXveFfwcpTV4vXQofzRXcL5wPCqDD24NPdDShj4tSdr361oUXNRUQvIowrXXJUabNMnZd8WhK/ubwLWxO33SztoMfX5XEnij71QqDBpsEB9BAP9REvU8cDrtHaYUp3JSO4+jiVEUN+W1kYh/AB61KWZZEzUOl/g5e6n9HevHDZZmsn0/51jtV5iaTQhmoR2N5wORfffReD5VvZJVHtLL+eGSoMnbAAntjrkHadt0ihLSA5ZwWfGlbe/brukNsyy8UNrMqvAeHBhzma9fSMG1uZfNSO/S1Jm3zDoG38ymde9rZ4Rvo+vXmOG2Hswyq8d7Rxhi/mayXFeiKVLx8PqH2encQB/FE9L9rNsXjK3ZxRovONP7DLc6FqMyWKlxvQeSo3qWT1NSCBC7WiMS2+eEFARokZfSrChAtKh4WZnGC4pURIvkZXMQPFDp/ZGZHYgtXM6Hnbbg1twJB0UtKpkObbWWsXvGElgYiFpn1x+rmgNYV1nEwbhVnBgdRFl4MgDPPQfNRhfemfQRp0IdCc/tw+h9o3Gpd8Gv2E9qV+YPcPsjnlff4tGmOSgMHgCoREfwHUpJlyu8e+wAs0rMfGrxby9H1dfpluPFiz0fRqyrJi9hJI0eYRIQVDAxJPw3svXe6NQ6fn46m+qoFUxw/PgfndERxW/QJXs9wS7hZFR2k9ogNPKRuZ7NE/+m2i+WJM/fCcsewkOj7+Hi5QlZP+Ht8SSdzxUxd+1cbAwc/XfA+CwWWfqhvr0crwrpjqQNlxxjpH9NiOsNNt1+hxxHiXkgv7EbjLgIhbtwKP0do1lJeO7vvK4qJ6jicfxkkNPsxuWikXR0KmFwVgzhBxxoClRT4HVS+q7Sja8u/Myai99jEUSeSn+KVbtXsXRgm7VllYW2NJ6OMh2Rmj24YMCCBbOox6JU0+e9IWRG5gCgkNl1cgB930GoFj6GUSbdcR5KeAg3QQI/mUxQ/NcZtKV3ya/fTa2utt1vBVHEKG/V9aXxV1vZoiytzj7B9D8G/Tli5Piq4+xacAuPukHS9zZp4O8APvM+xcisGPqc6gMWuFQTgnGqHkKm46EQcGvqRoezJ5Avu0HXrN/oq3jefm60pguxGOE+QLJcJtI39SKLI7dzsJM337i42dpky12MCbMtst0K7G+bPxOTjQWjlTXLUemInzIatT7ABmq27U/eA6HLSorMVhCZLW+3fZ95AAxoEYlw2EfJH7Fs3fQmv//1OI+/pmP3xTMcVZympsaqcSV9wyFzCm1dDBElr/H1qG9I9E4EUwu9tJ9zPGcWe0tH4dbajZhF4N2fev9p7OniilOVBAJ0r3K335PytnLs9kKOJ8Zzz2fVA23m2hs4F/0NQK65hX1dHCl13WVdU9ugw7usbNnMy272PkkRrSYo2s8rCV15rtds8hut565FQCMUgSCikwUgWOTolXpOzazj4MAfee3KJNIq0qzzAljs83W/Y0shqHFp7kikbhYhJS/zTcsX0HEJpIyle0oSbwc3sXfMCeqdGlDr1BKgysEf4l7h0aaD/KSdBtgBB4IApH0OR4diPjKENU7uHOjsjkFWRY0Z6mW+UG+nLc6xrglBAI28lp4BO1AZpDFulSWD+T4HKVBUJP1vVYcBFMYOuS+aEVwqsvH4dSURlzY98Nv/lP9/lH/b2dexY0d69uxJhw4dmDZNEri3336bl156idLSUqZMmcLPP//8v6yh/yn/KfZiwTC+gocrCzkVn8R1l+Vk+n9Ihc5u2mtQpZPt8zm5TptwaonB3ZhgOwABusgv8tqUsxgUBqZsncLjjmdZ3H8CRrOSvRnP0DnPkSPdtnMx6SIgIbVUKmDAHprcuqGzgEFRSbGbRA9kwFWqWBOAoHLHUODDuF3j8buziQOdtRS2pIG+BqXYwJa8kexohFaEBSBRvxzsg48mjXSdIzoLvO2moJ96F+RINJIROV/xnpMf2iqJsyY/uMD++z8FtqgkipdGVSYZfh9wTWxvvPnaNJftQw7z26O/UfuqlY5H5dnKMYJoEfEu88YiWPgzsh6tqhDMOjA2ElB3mVkuLsz7+QmmZk/llPnfREIbjMwcFMDzPefxRbIalcwaPecYCmFzYMhRFGIz/dzTiFHAOx4w1LhYquDyIiyaQJad2o9e44VPmQfKFqtjTNsFBh3gA/k0LrpUcrGXjowuQxknlw5/QmaCJoB7niMZW1VCanQ6QcFl7MidSJ6fdVzU3oxvGsi2IYco9ypH06iRLnYVZ3FOiWRg8F/t+iMZAqSxqlN3ASD5XE+099azp6uS3Lps27utkX1yszOuzV3wEqPvq8u+9R625LPH8T3otgo6LEVurOPjgelkjDhFWUh3fIOtch39DMS/QciYc+TdusLs9TMZkJLE98mv0b82GkZegqEnALikg3t6kTTHH0n2O4qb7hIET6G0pSNbqv34veEIVU4nua34leD8tWh2R6C4+gT9QzYwtMc4flF0Ic9Tir74qFELca9yWDGO14vleBdr8Mm5gKLJqnTUpsLdr5jgZGb2hpl0uuqP0XkwEUorOvj625D9C/eaBmAUG7kW/ggP93gYte42JLwFuRtxrLpAz6IgQu+EcWKEI1udXfi5tkpCFANLjy8lNXchg4/Es/1yOG9fe4nTwkbY6sFaHxV+VdN42++ENbqgPWpOEKDMdTcXzN/ToL7TDmkkCODcHIdX7ShcDDFUOh3nrSszyVeFQ9RCZIVv8uMMNw4O/orlby8nL0wyhHHnM0gZRbBnHT+99CVeZY489stjNooKptWiMtRQ7qei+4VIElIT0Cv0dMz6DIWpCspSoOYmz3mWkzXqEJkTOuFt+p4uhmfBbxhFrq+x994sFuuayY64R6OzlPNAhgB3PqOjuogzh7thFC189/R3lPhbEW5+w2CWhVF53jRbIF88yYXIsfxd/6atz6IgYpaZyYooxudWAaE5gXg5hULgJOjxI4vqwrn+azzR1wuJLVhOfL0UZUzwVIiYj0FjJCM6gxtdc7kbcxcsAs3KOJhUhJOuiJWl0cz5fQ79TvXDu8ifNVcDkeul9nmKlym9voK4s8moTGoKAwt5t7AzBE5AKdSxKvUZsgwQdzuQQQfUxNyNkcY0fxuPe2bR5NiEtlhJtyvd8DQLEl1X2kpkexMJkYPcIGf+mmAmbZv0oKJi3SvdG/oSqB+Ks8oZZllgloUAfT6dH9tJUYAOVXMzYc7ZeGY/Bf5joIOk5MbvXcmYvSMIzw6XKM7UvjDkGK9lbmZNrf0745yL4NYyyN+KmPElkzyK71v79pvy+bIUTiZ0JTXwFQak3mRM7o127wZZPmFKj7+48tg7HJvwBonyYxJIovOn0GcTw4xfYXadxY0OqZT4llidzUZoysMt5ydOB7X9rkiRw+NSblCvXlC8nx6aFRgcTRwbdIxgldXi2lKJn/wU5Tp3KtvgRkRBlKJXhp8mxvUAl2LTSFJBVii4inro9BEYm1CfGc63Y2Oo9wik3LNcUtRa+xw4kbeExZzUtW8XQHw8+PlBf+0sxp2roO/VXxBMRjSmbKi5KTnGqq/yniaX8XvGMGTrNeS6Bgl5q3CFqGewINrowAxxHv8DUIqMQf06keR1gSlOdiBPW6XGS5PNZ/Hb6Ns0ioePbaJRjJYi6GIXYYyWIpZjzv5Gv8MiBZ6/YzC3SDly7v2OUe5Nfl08dsOdteKOS6HTB8TeeIavoiooDS2RIiFbFdAJOTD4MJ8K/dhUf1+7ZGos3kM5njuTj70f557+or3uiHkQNIWOhV350c+ewwrsRlAArdIbbWMPfGrGE1n8Fh2FWe3e7WB4nBHXKnm4dBW/+0KR80IYdhrursan4E/myzR0uBuDtkrLjnorpVDdXbCYWVb3O0/0S+NysgltH4HfJgYQXzpMimzR+GMYlYE5KwS/3ELcq6SIO2XtRTql9WRa5Gvs77Gal93sDiaZKAOPZEh4ixajpp1jVhCQwDDHRiEgsGHWBvZNzMP/bgrKxqp2fXIQXVAY3WiNDrGdxY3ZaCrP8qzTdgbfyqQu8zM23nqH+lBrhHDMi5icIon0qeXk8IMYlS10U+gwo4CopyFwHAICOrWOOucKeneZzrsDxhF8b6x13O3AEiMttr/Z5wW7ocCWJ8xuEJKhRGn0RGZ0IcfrGx4/Ppziv+Pg8ECOObzCG2tm88Ynz+HQ5GDrGxYTWAzEOfbBv3o6u6ZVsHXqVjvTQNQCHMyNLFTmE1wxv9040VwCd7+Ce+v52vUGEx1BrQ/EuamDhBZvzEHTeJZThWP5qEpgjaMHJpkka3JRDiWHedH5L75Y/QKjftZwxz2a3T4Swp+GbCjcTbUhDOemDiiNnijMLqhlGvt4tKGvsuV7AfAZAoETGKaoYkhAEDlRntwNW0KtSQYRj0PRHtjflWChGd2Yozg0OTBy/0i7sch/BPhKBovOp1MZdngYr598k4lJPvQXYYQC1gOHgT+A/jKRPgL0nxHAi19fpynoBQCKnXaToV3DrZCn2XxqGNqD3aH4IJ2EFPpFlGKUGTHKjIwN6ydFPxXthsKdfFcRzQF9JXtG7+FK1yv2c2pHOF47vFnu2guf6gm2cRAQYKsXbPOmyHsEU7zv8NWivwhuTqH0+k6aA56SDLxjbrHDlIh5u8RbWOxXTFvUepTjLp7r8STNz5tZ88wa+x5UfACuv4Wb2B4k2PacMloM1GluUKm8+o/PFyjmcyUYnjQHk5j9JiM0O+gbvJXwHr0YMaM7r/usw2X+Gpa+u5QdE3fYnX1xr2CaXMp7bZaogIBJdLH/oe4uU53q6FYzV5KBVqriC0/TU/Ey1aVTecdXh9IgGcPVcrVEeR/5ODKZkdmeRfQwz7bWLY31gJA/+W50Z5LUcHD4QU4Ousv1rlU4yu0hgo5KBxz0YagNEqjyfsPdQtliVnnBwL2JRN3awjBjV9ifJAEtfIcSQD2eRiXN6mYanBok56hMLeXXCXuUAf5j8K6V9gbBImIUXCmpgc833OCPrY/h8pSC7z4Aw6lKInQwu/JPiR4cmNP5TVYF76LA8zci655g6cClBLhITrud5b/TYHCmjxoC7sEntR3BIQB6/oIw8hKjyg4TWDYSRUtTu36tneBPD/lsSlz34tASQd+08wwyfyA9rLkNJUdo8B7KV2lejN84AzHLmStV8RD/Ogw5Br3/4LTPLH4pD8SzbihuTd1tdTf7PExqeV+iVM02gJAoiBIwzn8U1d4vcDL9LVwNcTTKCmlSZ0pGcVuxb5Shpc8T3DyetqVWlsmp+CQy6u3y2Xr/b42MeeeDdxj89GB63p6OyuD3YM4+W2Ree2efk7mOxF632P9wd7JipbtcexpP6dxTGj15JK+Jujfq2rTaTt3XSuMpxwKZPxJpKaHvyb7M+EbBjK0v0PvuSTwdJCeVgIgcFaJF8Y/MRG2jng83iQwqBJ+mFCmXbvhc6LKSQucXqHTVcqbXGaI9agkSd0v3EeCMS3+25rvgWdOL4PofCC1/RupzyEOkOa5GZ3TCbLZG71m/35Yp5ViHKI50CkKnzMdbFoO3EGfbD4Yav2DYjUJCyxfa2mtB4Pn917FYZFyMbeJSn9NcHNiBMPUPCAJEaC+z9SE1vuXfkO/1M1XOJ2hS3qPRXEnXrrD1zwoi3a/Q0zOU0Sl2XaYwQKJL59YyKD1MesgprqR9iNIo7QUqwQmUbphcunC1ZDgpxvZ3cQsWKRIX+FiXgfvVFNyLb9sc+iCw4+4iruqjQACvCi3aGjdcZN7t6ml1RvvUjiGgahZalQfvn9jNqvM/k2eQUgpEZkQSdnU7oxxfhNKj+CUkoRx/lY1e81hcUIq2ygXBIkgADZByvDqF8+ftddQ75gLwzdPf8HnLakrCVvDBDjV/bR3PmUMBPDMnjoz0ZTQ3l4C5BUoO4XZnDipZI8EuN9CI5YCZaCW8HnoOJ2UVAz0y+T7ASIN/DVsGf8+luGdta+JC4XjWer7O8RgP9v62F6FcwFHpaOtvWxrPVmefiAWtPBMP6zkSfb0M49mLdH2+LzlhUvsrmypJyUnhXtMN/Itvoty/G0GrxSiT1szfaX/TZAWbmM0Q/MpULndLZZ1iJumV6Tb5a31utrQH9raN7DMaIaBqFiq52/8I9GdWgpe/F+OVe0jKsqYL8R8LUQt5p7Ivgb/OZNjhYfiU+gByEGSwzRvh4pPtDgnBYm4jR/Z9woyBITfyuT6tWXJ8Wfvk1tSdUHVnECyYsEciRmvjiCp6j5Cq+f81jec/5OwD6O0yA728glrHK9K3Ws9tj+6Qt5nP3CTlVaMPxqk5DqXF2fbbB2k8BbYeOUbIc5D511ZWNqTxd72Z7ysgLPVnnpgXwJvvS6AXUYRnuj/NbQnLgW/NRBYmPU2oWyiYmhji8Qa9g7baItRLjQqpTZFPovcawGxn+GDxh3z++p9c7XLV3pZTU3mbbdZ0C/cxQuRshFQpeGJikQSCMMma7IBAbWfQdqFOcKDRhluy3pFMzZAigYKW6FI5XvZ36yjgLxyEiwtxsOQhIMOtxg23YhGhDb28bcxac5YKDzr7bCQOLX0YUrOACS4JkpNRJe37r2ih1KeC1YvWcrHHRfs9XeUBwdNRGmey9VEt2ydtx3ZOhT4sfbf8JDWR1birqrEIRtbWQZkyHi49y+HcpxlzfBUra8Agq2JvnD+HlKPZcXcRZQ6z2vXBYJLOwe8K49l2ahRyg4FmCd/Iv2KgdKqS1nezs/e/7PO/+v//Kf/flH/b2Xf8+HESEhL46KOPiIuLY86cOZw+fZo33niDnTt3snLlSrRa7f/Ktv6n/B9e1GorCuzeOpQ73env3A4O2g5pWq+5SWrQy2S6Sw6w+zeYUrOkGGX1uMzlbpdRCGYSvE4Q7HqLcK2UqD0nNAe5UU7EvYGo5VYjRMBoCntu4KzVWHk5cip3Al/lsrAawh6F3A2Ihlo6lUrosTO9z2GQ10jK7Y4wBooTsdF+iC0YZfXoTXow1kPjPVJrxzLpbg9S9VDaIhnAiH8DLi0Ch2B+vfYpF0bO5NNXPqPJUfdAvwbcuo137RjuBrzDZdmads9uC3Hs8CzkXvg9vqqBt8wbYFIx3PsN0laxyb2Y+b/MZ8yeMbxfouFo8avgEgebnVCFP8z++hyC8wNwN7jjmtyHsXLxv0RCj7UioZtc53CnohuHKnuQ47GWg46PMnfHPIpUIVIUiKKI7zr/wBwX+KAKck2ecGMJDNiJue8Osqq7UhGUyNJ3l5IRlSHNs8IZ/IaTK/PHANztKFIb0Ae16GSbJ2RqvOpv0O3gEBZ8u4AkpcC8mF9waLkM19+BksOIguTg7HalG6oWldURGYI+8nX6BW+lPBx8WgHmgggDd8MsC06GLLb7QVpsGppmNQGFPvcpY60KpPEf5e9+WppcVV8pp0uHdymNXMQCUzX7+xxm49RP6FPZF6brIP416PwRyB3g6FFcyzPxLbpAgvcpWmRWHiTvfpzpdZmkfCg0iVgwM61Vv7m1rB1VWKtye9oqyxYELheP4k6LSJnsGjqFdOnepXeFwAmkKzrh9t0jdDvvzbEJPcgNvifx9Wd8B5cXscWvBZVoRmHyplPuT/R2kHJ74JGEzmc42bJ9VDlJUQmOIrQoI6DjBzDDSF6PdQzOVqIq8abWZS1Jse+iladD2XHI+ZPB9SkMHC4Zs7sc78/B4v40CZEQ8QQtQgIviL1QmjOpcjyDb80kFvf+gIGhA6WxFttHTNRRQEFdAQaTAUGAoMrH6Jmxl6j6x2lUZ3KwaCML9fGQtIYMMYJNWVoc6iRrlM0wHDEffIdxLKCIEY5gUBjRK/XSpVMQQOFCjWtX/r4ZxICUTgAoDUqcq8sx+oyB6U3gO5RXfMoIG3Wa2mCtra2t8iIg4tDoQO+TyXiVSJd3hSjC1VeRC25c2TISt1NhrRLVTp4ERCZun0j/jyrxzs8lS2+PTmk1bsiNAvLmekSTEYtCC92/Bu/+yAXYO3overWGcVs86ZjfJuly9TXe1hqIUsDZfmnsH7UfEBBkCtD4oTTWERebR5GfBAVLSE0g8dZczHIJDBEibiUiqAbvT17CPEpNrByuJO6FKkkR2T1sGgKQFyKpZ8Pyk/mgYhVUX+Wtos54l3qT1rOSj1//mPdC8qSceyovBNcOKAWYvG0y0MYIYC37M/dz0H8QdwLesPel9g6ceRRKDlM38gbHOEB68jzqxw+kV+A2qkM+gR7fSzlpNshRxLhQpa3mdsJtaZ7lGvAZSL0hEY+6wbZqp7paI8y7rwYgTuZAYu7XyCxq69zaNwOFaFcM/qnorb+52QIVJlheO0k6D3wGgiBjauA8/HvVcWhUGhp9CCqZUlIMJhVSG/8e23VtjBKW9nQlBE9jqasnX7/2IQ0+Q1nkWYPOfQy0VBCt/IP55z7izUr7623HM7epL2ty+1JigmoTHGwKg/A5cGQQspoLrK/0pMD9Z9xq3FDqlXbj7u5YHrb8E4jCXiY3KXjJYQqdUn4itPYZBtRGwN6OsL8bXH2N55tj4VQSap2RFv1PCDIjaHwg6WvuDr9NhzxQGLX0V93itPlfR+u0lvNAihE6jdnBnOifWeMFCHZFXRSMLB88kM7um/nwxHYy7qzCq244x9UXYbwVFZn1BdtmCDQE3iSgKICIzAipXxp/mFTCHmEY10Pmc8/nK2uf26xXo3SPifE7Q+r0Y1ztchWbAqrxA6cwwuPfwpT7DiXardwJeL1dlMmDlGaiFOlzVlICf6xRt3ve9p7UWo9Lc0diCz+ku/j4P45RQfFDvH98F83ap6TozsvPY1S40/uSFz3WzeTR3x/lpdv+MOaOBMSpukyToKbSBKf7pJDarRoAV91xKf/dpWdR7Ivii9fGY+gdQEaURDEpqr2pdR9HcVM8erMcI/dRQ7p3gfoMPhve2yY3eV4/8NTVztypLQWlG52FfB5W+uFb7IN/xnFUTdK3qbgAfwocDzQz7mYRntY1a1uPYY/CqOtUmaTo2rSK3qy/+T4tzn1gWi10WYGm/ASXgsFp2Bk+f/sjZjb6cdY7FzylKIRu9ad5pG8a7lXupB3oQnGzF+X+H8Hp2fhXLGGkezahconGs+34t/67VS7M/03+p3pNKmdKD+HXlAZlxxntaMHV/Aq17i4k3kpE02Sd8/jXoesXDG6Mpk/Ld4zd6odHhUf7NZe/DSH3N8yCHv/KmQBoBFdozIHLi6BwJ12VjSTq+tAt+y8GpN5ggMdM6PQhmV3q2HznVZbUNKETq2zGJrkohwsL6Ku6iTmwBE2xO3N+n4NbtdWJo6uAnHVMVI5lQOoN3OsGPiBzvk6+dLu3gU73fmPotWKGXi/CQ+MJu2Nguz8OOZ/y+MSlVPg50vlEX7rc7g09f5SipxwCmapP4qc6sAgWLIKlXZ9NRQdpKd5JvZub7W9PdPUkZ99TJC98mJed4piu8ud9/zjudhyF6SWQTegD2o4IAnT0OcLtjgeY7Szd8QwW0IlqaKmknCAmRJdx8+MP+OCdD2jSmKSI+04fw9h0vqxIQLNrGGP2jqHSo9LeLi8JHLVAW0zHXDsThNBSbsuLZhEUmOQmfEu8CcjNYfOEIFzTn5Jy7spUzIt5FQ1TyYzM4kbHG4CATF8El56j2ezO8ELI2HmK2etn2/eg6OegwxKWBF4jRtFWHu3jVaev4kRCJ45G2HNctt1HrlrGU17ghdf78+h8aCW3jc8jM9ezcCFEdongxZJk/m6yEFO0nOFXqwh1toPf2hkULQIKoztyQQexL8PAfRD1NOtNzWQWSw4ewaKU7kllKfgIJ3FVVeJAs80p0dbQGeuwhed7Po6TWGPtk2Qgy6zqTlZ1Z7xlEuW4yuMruuWfJ9atywNyWO14nmOJMRzyGtvu7yct40ktcaHMswFtRyNarkLVJclo2FLOp4YdjK/0R6PTUOFpjaSUqaT8OgpXnOoPYMtBiIwjB5YT8rzAyV1yVjaksb3BwPcVkHy2jtwV8Pd2KecowPXSQbxopXaPrX2Odwe8i7+zpHOqxHrGh+wiyboFuIh6qEsHtTcU7ODJhFlU+8aiaajAvfCmbV85nT+NUlMN92LHken7sXW8WgfhClx7nQa/cWxtkfbj6hIPnrn0EfgOBpcYcArnxV4vMPRGPnGFH7X7fX3kCpqNzpyIP2GTMQEBDiTD+cfRaySqc8mY+GAkWys1mszkSGL+KhKaF7Sbi/sj0sEun62OiYxI6YyJb/4c98beD9B4/qucfU3xa1h2YidetWP+MX9dqxHfghkFmvZOkVbDbxvqvvcaNsKFJ3nbtIUhjtaIZX0b9FNrn1rxUZWP0jf1AkMUi9v0rVUpNWPPYylIjocun1EbPJsv05+h66lbuNa68odehYNQAgV/w/HxFKjDSSl0I+mcG07V+e2+69L4KbP69eO66U9bvx7sc/voFaV9ybUbO2lMLSwdOIIfxkWgN0TRYOyCxucPBKfJ+DRL7DvNRmfkop654mXmVQfw8PdK+m/7nb+qpYg1ZdZKqDwP/XdgkUsCtHbOWio9K6V+998JfTbS5NiHquYAG+hEJTiCSY9oqkUmGB6Im1bJVBD9LFfdPuCbOgt/TfmLzOCjZHhIEVlvL5YR1mc8/uo6nBEYv0VL9Ll1DxqwW6M720SuewX5cjh7HoMSrnHM1YGH1z+MZ+FNvswJgw5LpKhWQSDPIQbxagK+pb4cHXwU291PVwZ5f/Gss46isHvS3NS5cD7nYUL6vcmpnfBhTS6bW4p4Oz8N7+z32LDej1kvTASPHlgEOaFuN1g5pBPxmj+wYOZGC2wujcdibW9kxyzyXjzIvfB7drlvuMcPU3vxsGc+LnUO5H+Yz+3Nt9v1t/VOYrKYiHSPJEAZh07vS4FxIA6tcnuvHtON21gsdnaG/Zn7GfTbIH69txSlUYdYXoa5WWdz9v167Vf+Ukr7rNkMDt0TODjyODqNDpVcRX5tPhu8w0hJiJOcfW1y3wEMCBlAdMnbeNWNwGCAqOLFBAa8xDp1+zvwvyo/yBS4iLHET4vHXCEpPzIZ0G8LJK1Bz6dkJLjb3ncWDYhVZyBwIlWuXcj0/ZhDEzzxemUGBpkBi2C0yUorSMUsGBEtClQytW1NtR6lZvOD3pQYjzhiipYQXPmYzbnVyrQgCiJzQ5cQU7gMwaR+wNm36dYmfi19Tko7YS3tUspofKm0ruXO99Yx8HYqAXo7401r+9wakhl6vRD3w0ns3/M9J8xwwACzgSFI/3vMYuSk2cz3H3zKF4tnE8V36E0aGmxdatM3uTO/Fp7HR3ueyKjROGbCn/WekPalNK73vmOxO1hEM2aZEbPMbL8nBUk6fjd1m2jE1hynMhW4d8M87DS+BdU4NsfYnkuR9c2g9uEjh0l8V2sdj9b1JlNBz59YlbWDP9sw9QgI1IhSlK67/giCRY5XuReJKU54lbm1GydRhODyBXTVL6LK6RQHfIaxNGWpra4WoZZ0v2VccVzOp8N6McLSHdI+g16/kdr3CHHpCkbt6Uffs7Pxr5yBq8oVGvMhdQWEzyFTNwHfQj0JtxPs7fYZALMsWAQ5KhEGO9jtjj4GiW5UZ3KmpFni1zYLBnSKYro5VDE57gtEkzQQrWeK0Sz91jE7hBuHeyIzmTFacTeelw8QcOfwA3pLWWhPyqcvpDD2QXbH/zj77EX9b+5D/yvKv+3s69evH7/88gvFxcWsXr2anJwcBgwYQHR0NJ988gklJf8uduI/5T/lf15kMhmxsbHIZDJwicHiFIm3on0ib1GwX9J9fewI6XS/ZVxxWUKNrsb2rkE1h3UH5uNS2Y1rna/xZ3Myz+xN5bkej7NqZFfmuFYRU+3JgBMD6HPjK0JcQ6Uf5mzEufTgfa2zflcQQZChzPiG7SPSOPDct1xMkpLXKmQKCJ9DsTCGb7u/z68+UO10ll2JLgxfNxyCp8GkIjLqh9FqvHur2Jtl9TckSprCXQgWE3crk5HJoxmUVs/oK/YDnElF3AzLxFkXh9JoRwe2LaOEY9wLha75IfiU+CAIcgn5felZyPyeCIUefUg+YTlhHD0Xz+7CFaDykiLwtF0xWOkmb3a4iWvn7vQa8Qn9ZTKGq9TtkdCihIQeN7cTL665hU7wZ2fODIZXXeBy8DMUy08jVp7DP+19uLOSRrMPS68v4q8GONFsTaqb9ZNkSNcEkh3wGEl7VjBu1zgQrMqZ2QjNxfhSh7NeSdRtAafKnPYHycB9NKr9eUTvjm+pL4Vbh7H86mKUxmKJBq/4IOM0jdAxjd8f+R2jWitdupwjMSR8zLn8CdxswUY7IyJKSbN3x6E2FmGwQI1bDQaFwWrQto93pCaJ4LKnUJhcyfBdznVj+/D21rkRAUMkDG781PbMzTuZ9Pz3UVY8RJXzSUpVl6WLSGu5s5K4wflkJ3bF80kpYirH7WMJ1VS4G5WpDgWS4mHGRJMFyjS9oHAXvg5pHO39Ad9622lnwuRgDpiAaXIjy07sYmOZVfFuS5+GlKciIyqDO4klHO8ymoMhQyVHdegsiFzAS+Ua/pq5CXVjA9Hnfre3N3gqlbo6ml3GkBr8AgBPl0FmbKpkGBFliKKCrMgsvn/qe+ROTXziWUQP1WJrLqTHidKlMzW8km2TtnFxwAlCHIpQinXQ8wdqLB14LeklrjXMI8N/GT61o3kt+W36BEvGs3YKuWDka303gr4I4k7FnfaKqdmeD+llrsEWN26p+nLjhynM/XkcUelRqFuNqD4DoOtKvqzwRHeuEzc65bD+4fWSjJhNUHWFcu+RvCqr4MiQU2yfuJ3DQw5T69Zsj+ZwjmRQYQR/ZnricicfQV8mGXlvvEun4iQejfmZnS1DGHJkMP1OJrM4ZhNDI0bA8HNk6f7kXtencE3uQ/LZZJzrrU7uxnw4PIDi6AJ86iXDau8zvdtd8GVWK0fc7WBy4xy41mEfWbV34O8A2B3DSNkdeg68RpmfBbm+EaGV9u/kNNjXhSebXJix5C26nw23ybKMZqi7S1nnL3lNNYztM/K40uUK5QElPDEoi5bOv9ryZcX7nMHd8S6ymVXMWLyEm03uEDSZaouEctzlD4UBpWR3mczTL/VCYWmBpnz8xRYWfruQ+JMe6DQ6ZK1dMrdA3R3e1oJnhSeFgQ1c6Hmh3XosbSilVJNCncN18jx+Ic3pexrqMiFnHdSlE+QaRLiuMybDacwRixgR+RMmdbCEiNf4ctrswdFHP+KrRauodauV6DUsFrBYeMLvM3526sBok5QD6YOyRBi4HzS+WDos41Dam4SWP4ObIU5a822ETmajaPlnUMBhw0Se+HUgr7y3hOe/eIFz+li7w7/hHt3df8VPf4Nnvp7P9N1bCXYJs/3WtfMS+nUtJSljJ+71fXHSxeFgSpfyZOrKwSmMjY1ZdDnanz6n+xLpUI26ag+4dSTV8NwD54coiFByFG59SImuI/vPnKRjwTUeO/k9d82fSy8FjKPI/xEWNqZzudNuLk3oj2/Lz3a636CpNApOuLar+sHbf2ichBTUZlqtSMHTJKN0xONcNLvz5aIv2TV2F+c7LUOU2R2lNnorTKjVvsTEfsJIUfVfglJGK2R8vGIFHh6+/H7jQ1bcfpeOOT/zlzaJEUe11DqdIsTtFmpZPecKJ3LTdT3BgxUcLpboU7nwFKqMT9jTCDd7n6DarRq5US4ZoUQZWEzMrpnK07G/YBZ11nm29rnmFrha5X73owz6ZCbBecHYruZ1d6G5mOHhIwmsfJQq51Pken8r7SH1mTjukrN7pkhDBPaICQSokvJuFWnf5I8ah3Z9buv0yNelkum7giLtP9Ov2Jqp8+Vi0VgsokpyXAxJoTzsSTLdKsmIzEBbo8Wh2EuiCSvaDUcGstlxMBsuh9HviDvbDL8y4NR7XA2+B11W2KJl1Q4t3Eq+gEWUDAFKjwRcR+7kfMUTTLj4PKtqQGXwI0E/j/HR1mgO/9Hcq+6Il1w6v3SKQrIar/Oloi/02cBYyxnebvbAt/5hfp23l909J1LRVCHdzwAP8ykcFLVt8sBZx7ohB/Z1opPDGixY0Msq0cuqpHQgtalw9VUEuYbXcxxw2jyWkNtxXG92QS/zg2OjYGcU0Y03edwFrnW+xsmQLCYc+5U6z7mQ+yeeNd+wJmErQxzAhJ6z0YPpf1DN9jvbbWPdakRtNdqA3ZDVrMznZvDT3PJ7zXZ25ykl2LTGmEZC7zuUBHszav8otK1ONQClO/kkUmYsxrdYQ3xqvF3+jo3EpPbjRPzX5Hp9R5HHBgBcZX5SHumhJ6DrSq6G6bh2/hTaRruzx95m+2IOKX+SR8Jfw8/JD7p8xkf1s3l50kY2zz/C7jG7aXC2QoQ9ukPUQjq6rJfkUPsH10Pmc7F+h60uZ5UzgTUz8KuZhNroi9rgZ4vSAWgKfoY3ju5HW1ZLzws9cTcClZcg7lWYkINOUBJ9N5rbCbc5PPSwvc+745GljKDi7AQ2PLSa9bPW8/4779PX+zZ+w7/j5ffXkTQwlT/eHMfmjx8iOmopk/w/YmbiTFufmwyu3GpypcoECGb2NsGvvs+CcyReSMbzHKu/dl+9L4hKaKkAlygQREp9SrkdfxuDwmBvV+91MOYOm4p3SxEnIOXBaraitQLG49aUySsqFcOOdCL4xj40sgYci36A4oNQfZ0R5U8ye+xlFP1/Y+itJtwb+iIaKiD9axxkVRxqgpDMSCKzIu1y7xAACjcGuJTg0QaDZjs/q6+TeG+iDfQWXfg+0YXvt6Nwz1Z8w+N7fwNAr1KhIlNiLzg9G/YkcrQxmHQDyExOKE1a+zyemo64QUZxj/kMv1rFmMtGet89gWfLLoka0lAHohyLoMZkdZaIFoW0L429wz7LJdZP9qav8nFskVOiTMpJuzMSETOvnvmG6+ZL0txZjX53K5MJdb3BTGsQwz9Fchc25pAa+Bp3/d+hUZ1OvfxeW9HnkPAYM06HEJLuywGvI3zV/DxMqZD2guPj2CXvxNW4O+wau4tmTXO7c5/bHxJ6byQl2m3ITI6YrlSw/68lnDBZOKA3tjOinjDCKTN8shm+ONOJUmEIS1L2s7a2FTzU3kAc5nCEpZeXsqIaZjrDMu1VyUFeeRHyt9HdewsGtTPpMYVciH+WiuYyAE7mTaef+jxDNVidSPBqZICUDsF/NCjc8L2+iFOxTbR4l1PlXsWrcd9JlPS6CulcODePqYmLyfT9mGyfL+xUmTJILevH50UxlFmPawFB2mMAn9wFzO/flyuu72EUpYhD8R/yJ9pze7brcjvH8/2/ab0TrH94Pa8bXkfn5metX3pvQNBQYgs+IjF/Fd9HlJF893C7iGtD2EIuFo2z5ahTCTC05jO4+poVPGm3NTwA6sRq4G9DYexhtkf+NQ05zR9PN5GVNL1dv3SKIrbxMDdCnkRt8MOtKQkPMdz2u+kJ05nqthxtQy9GNcziS2VvatwfgzG3JBBa+mrek39LWrfLFAbVsa8iklPmdYAAxQeIc3QlNHIQzQ4iTlV59vFI/4beLa8xM/AUpy2fYLGAZ/0gxkZMsTmT21LVGeQ17Ncv5pjFbtBuOwYpHaLpc0jkXFUsalkjOQ29efPIKUw1SlxL05Hppbkuqo/mlYPnuGQJorZWuq+YRJN9b/QeIIHclG5URidzp0s44dnhOLXqPe5dwDWRG5UnKHb/A51SAtypRSfI24LfKTf+nqGkIMJuM5kbvIwYzxgIHE+5ZiCDHEDb5ECPcz6YrSwJPXpAR6/9zHI8xrTbUmSZQlfNVd22dvPcCiYtddtNsdt26g21fN5dYNdMAX+PenxN0j32QtIFvnVphg7vSQDH1BU4mxooDCjkyOAjVGur7U63Q/3g1DSWujZR1PUGNzrcIGJ9OKZTlzlhMrFfp7tvn7Bw2gKH16XxxfEQqnqlUt4UzOqcoaxp/J1Cjz+oNkNv13xiPc9xrCKW+aVQcTaKOWvn4FblYo1wasFPcY5Vwdk0OElAtKPHjvJ32t/k5EBlJQS6BHJ9wXXOzT/H/of381lYKi7NnfCQ3SZQkOb02Bg3hPHj2Dz8JwafWkBoxVN4O0rgQxNGqhKGYvTxR7/mJ4Iq5uOukNZlqXhNoqw0Q31+Da7VEtBeJVNhtpipl+fQpMqRnIi0p8McEj6ExLIP8Kkdi8F6/kZFP8Exg+HfAv2dNptwjpTuTpoNvwLt2Rci1cnUhj3HmU9aqHdzpKNDA+KRfuA7lJsuvUgLfJPrsWs4FriNfd0d2WZ4yi4jFiVyQYnF8uBZIwiQ7fM5m0uW2ySq7TOQ1E0ZamQmR+TWXJOCIPBY2HtEFS9GbnYkqvhtOhetJsZDcnKdL3yw17b7f8Z30JjLt01uUv3Cg+2yObEsKqgXybz9PntNenryz6UnsM9g5o2P/sSzdgk/Xf2C5Hypjmqncxy7d4zq5moQ5RQbutPF4wYDNbDCE17UFsOt9+Hu1wgWMzNKpGjY/ke64tDoYN8LuqzkO9lYLuuQQJlmOYIVVEHQJBh5CdFi4omIZ+iqar3QWB1juyLhyCCe6Pok8fmf256JIiAqIGI+95r70L4INImh4BKHSXRGsMgkangg7J5/u3ECiC9YwUDdl5hFHcWaw1wvvW57ZhBrSQ94l4sub/PJZStjSPZaAEwqLwr1Ip2vx9E3xZPeN1fj7xQg3Rszv6es4AB3+JteRxuYsm3Kg+DdsXdZUAYXdYBgYpIjlKi6wqhrKGQtXB45g/NdV9IlWwJxrKqBL8/9jGBuadcHo9mIgMC7vY6y/Y3PaFEpMFnPbMeCu7iWZz4gu6JRj8vp/bao6Haj93+wc69taefD+N9Q/m1nX2txdHTkscce4/jx46SnpzNt2jTWrFlDcHAw48eP/+8r+E/5T/m/UcxmM5WVlVJyS+9+oCthmrZ9TiZREPnxR3j+eeid3IqoM5Hp9yFXXJbacp8A6OR9uXVhEPHnvVDr1AxVpRHhfpnd6c+TUdmdT7wLScgNo95VQ613lH3DOjcX32vP0rONg16wiATzl7RpRz2DwmcQ51qeJLgoBaNc+qFCJoduX5IhexGVzJ7rSHpmh9MGOV7iVf8sQuVQ4PkbebqXoeIcjMvgWqfvOdohHH3NI3TfvRSXsgx7uzR+6JURgPCgEctaxgv7Of/1dMb//BgTDg1nuPAnXFwIfTZCv7/oVRnJryP2ciHpAk3uNfTy+hF0pdBrLTqVFwrv0+wfa6fuGzj4FXILCkh8+m1edopjkpOcZ91kXO3uxecfwS+DroNjiDWyyt4Ws2DkfCuQMf0rDBYXLqV+SW2OFHXzkW6cREkJaM7N5EyPPygMKEDVokLdbOWxLt4P2/1Z1vIrQ3Rd6HlKQezZ37hmkoxVXH0djo/Bp/YSo+fuIS8qgyL3SqaFb8Kz4jMpAiFiPq+IpxiycS5Tdr5OcuYJOwpLhN0ZzzO4UIqo8a+cQbC6A9TcgLo0Kl0m8VAJ3E64zecvfU52RHY7hbOb8zg65n2Hoy6au4Fvc8n4W7u5CDT1I7T0eZQC7G6E2JaDUv7EQ/3xrjxB30vjeOqreJYsWcKzVS6w3V9Suo+OoObmchJ7vMXvU8fzunYV/U+9Q6OyM1x8Go6Po9uFQeijIFmjw4KZWAU0KYNB7oxCbKGkxY0Kkz16oLMKhKJddllqNTha+d+3uObDmYeZqfuWz2ad4lzvbDwqPJAb5NIFwTMZenzLD7Ua6szQ4OpEo7MLRqyTrK8hoPokCUo7Eg8gJGcibHaEspM4NGYz1smdbnfiGf3rHH64FUeg/LBEu9B3C797P0lCHtzoeIPJve7y14CFDDH1BqCM/iy6/AZndWASJYf0AxETbZCo5jY5/e539rUi4AZaCsBQi7e5iAs9LlDhWc3sP2fjXeZp/4FbB/y190j6axK9Lkwgsuht4jSDJAfm/m6E5K+lRlvDmd5n0VZricyM5EjXoQi6IijcC/oacpqcefirZ4k6nMX1gCQOqxaAoR6FuRw/x3v07nYcYfpE8js9wSDvh4j0iAbPnjjIKhjddx2KxkpGHhiJa42b1KayE1B2AifRwsF5v5Myp5ad43e2W38hLiFEFS0mMisYWW0qt4MXkV5+3PZ8iJjKa1q4G1/NyXH9KPe3RlBbjODend0Bz1Lp40DijTAGHxkMFgEn4w3YHUt83Vme9fuTyOK38C3xRWUUGJK4AfWVWeAzmHP8QpLvHm5cWm373u1md7i7Cq0g5ZG6qJNhlsuoCuiA16jH4KEmGHyE83pPUgYeo8KvEd9iXz4tD4Dg6aAJhJgX2NEg45tnvmHnFIlrv+0FvNVgbsFEatCLnNYuoFgTJdUd+SSUHMG880+G7Mzl8q0Khp5/Bp37KNjXFVJX8Ey9H0cM5bb6REGUuPA3iHR1+4Vo9wsIFhmiScO15gDQdoLmEoTQGZwvnARAr+rVJGXsIc6tu62eViVVpyzgWGIMe4M707Y0oKXSRULeKQ1KgmSV0FIFezrA9TfpnT6LfWlnUDdVg2DPOcq9P6DqMoIAPrXj6H33JHGFH+Op3wenZ0hOJMCxwYmBxwcScieVRfuvoXfpBS0V1JDA7NCdPGRnNZP6XHwAbizGUSFFS2n1nQipeJLHnTbD3s6QuJjC8FcA6HuqO722nsW3orMdie7dlyTLJZ7O30V8/krgwXNqpewcfWv78d1T3/HTQxt5x/wZ9PpDMpBEPUWSXzIoYrnc/TKbOlbxfpQa0tfAyal431km0XBajYJhES/jH7Oc/qLwL+n5ej4xjBeHNRLW8gWxrpffAAEAAElEQVTppaMoyFmKd90IWsQa6o01GCwKZm+r4O+i79HJS1BrT7HGx8g09VUoOwlVl7FoAplUJJASlsmqF1ZxN/auZNw16aEhi1pBS0EbbJJN4c76GU5MoNhjEEdVkoOuQ8ZCHHWRyEQL7I6F4+Nx2q/l0bhV1nVoNQbJHDD7jiO7uhPHm9oaBwSbg6DIY4lt77PLnH28c1qukRb4Olm+K2lQpVNnKWr3bo58P2ejB3M0MYp73qvxK54NexLAZwBmpTuB2kb2TdrOwfEWtvUphuMTQNsVOq/AwS2OqMpudLnWhYc2P0SF3hWDMhR8h0jOD2BgAexwluj/XUVQ7Ayi+fTTgAWPhgFMdP6Q6KL3GN78M+8NfA+yfoULCwjT3uARrzw0Zh9EKzVmq4xVBK5gwx9PEXZlO7lBF6h2uSXlGvPoDmPucM7hEA16LQ9QqlpMGDz78lv9X5yN6Y9LP09eH+tBS+1ZyP4V7n6JRebID2VKoi52Z+y+uUwoe4oulYMk56p7V+51WMlLNxbQ6NhIZmSmVLtMBdNbSA+/zKs353ChwQ+F2RWz2ILe3BZxDfY8hHZnX2u/DPIqcr2/o8B9nc3Zt977SZhpxlF3iZd6PUJpgCcbZmygxt161y07CdVXmVvuzhcD3ken0hGaEyo5hAF05ehNLQzcOJnbwYts3xSRgcJFums7BD5oxBaBmps41exkXqd3OCIRZRBe+grPx31CkGsQBE1EqV5M98zt+HhM51LSJXQOVgucIILFhFKUzpZqpzPke/1Cju5au+88YDgQBXCUAA0bq+s4YJiHxXCJDY/9yrLIVDiQBCWHQV/NKLGMXhkxDEoZhMKgsDtb3BLRa4J4rRLKvCvIisiScuve982eQdtwZxdqfQBPa2OZmP6KxAYhSpFh09P7sa+pzZgIIrjGs1G2lj77BzHprSU8vO5hvC1GqDgrUZdWXmRT8DH697lFQHE0U3d/SQ+P4bY6LC4xFBiCMMprEE0a5GZHcI6SaDqTvsWz5jLzy0LRVriSGd3E6qypGBwT4eRUODMbmakaubERj8Nb8Mq7Jsm2SxxMKuZS/YvSlKt11LrU2uXe2AwRjxF7fSZndCA3O+FVOwoPpXVS01fj0nSOUdaApajit4gufqddWgRHhTMWx+FsedTCZ68uZnm1BiaXS1R0LnH2Pf5+g2KNZAjzrN+D0qS1vadTREDUQnCOAF0ZRc1LqAhYJo2zxR5GJAiwI3MOG2oLMMhrrH8TJPlSuFBu7Mh62VEuyNfi3JxAd/PztBIQrTq/llXSTzBZIyrayltZcxHZvp9S4XpIenbfOeWtCsbg1JO/pvzF+eAznG5xl6LqXWJA24WqyFfJKJ2AX7EfHpUekny0VMLOSNCV8katD+WuB+l8fR315y6y22j6L42oewxm3vjuFo3Vd5nd4V3b/NUob3Gr7BY6o3TfHqJ9ieH+x9AI8Kcv+MsaIf5NuPAklvp0HDIcuRQ2kttxF0iLOmGjwsut6cCGxmlcbYES7XYOdwjGRVEkgSBUHhD7EgBa72oOvrgGw4DzTA7aB+UnYE+sRFWbv5VEnxTSAt/krv+7NkO5c+rjxHmdZmVhIiWtzj5BgF5rIfE9SQZcbnDd7X1bn9s6RzVmb5IydtEl+w90imJahOp243N/jlAAD0GiPJXL7PNmtphttHGtbesd2JfIkjfwqR2Ls8wLpcmjnRzo097l7SE9adJcI7J+Hj0qFxKpS4E7n6KRN5CY/xWvBW9HafTkuPujPLXLbuBvTedgFgyElC9gRdR58kbfg1kWFqhf4dNqMKPnYCcfDnXyteUlM4r13BTWU6zdYqurbZsmxk5krNubuDUlMcjQn0UhZ/C0SE5bTk3HOf0LnESoCCqg96WHmFb+EE2WQAibDTNa6KJP59sQL3RqE5VOB8n1lIBqVEs0qJE5YMKCxQKxhctZO+ovuvt3t7Wj1bllkFVz1PghB5o+BMDBAS7JVnM6tjc5Xt/a9D0neRMaRSNJnuuQ6ZvpeHQVURc3UG74BQcrFimjqgdfyEewNTSDpe9+yLJ3l9lBiv4jwXco7/3Rhf3xPZE17Kf/yf54lXtJz43NUHWJR0sHMLLbI2jr+9Ah5wdcRX9wjqDZfz4XCseyr8FOe6iWOdh0hKDmXRwNBHlkHlsnb6XCs9Zeb+lRMi3zqdIFcjkZNsz8kzTDITw87PPRSuOZ7fsplyMnU9SYa583h0BGlvXn49dXs3/kflQys0RLn7sJrr3O0wXL+LpbMTVuNYTdC8PmmAh7BCIXMLrCHf+LXQi7GcZRDrHPbPkv94ldRnhjxVbKS3Opag7gs5JA/tZdo1kltWlZ1iCK68M5UNaR/XludD8YR1hOGDKjNY+laywt/rP5viyAJocmrg24xo7wHZzIvMRzz8HcuVJOuI4+HUnwTpCGyQhysYVrhjfZougCgE5jxCJX0lRaT2D5JLoVf2ejGjZaWtDr9RgCQxE7JBBTtJS1nfJse5peXonZDIWvrmLhN3NR6VQoZco2epxRcvZZ2kf2tcongMEALfIyZC4iH378EeNU/zXob6xCxdzRoUx/KIeMuRmk9NpPSkIComCQcnNfWMBsBz8Gh/3GaNVyBt/KwpL/Kpauq8Crrw1U8NhPI/H/3Z6/07bP3E1hfWwLwU2TuBY6l1fPPiyB0JDAEHf93+Vw1YN5kxsNDdSpb1GvzGCWZTejrjYw0GvaA++ZzeBf/RBR1c9K9y/+iYazzX3X1AKGen4Y9QsDbqXi0tSh3fgB9AjoweLEX4ksfpP8vB/pJyj+pey1lp7AQKWK91Mmt/ve9ZD5DFk3mCvFV8BiRik2MWz/Dh4pha6tV4nhZ6H0KPL6O1xugVl/zqLH+UQisiLseotTKFdl8RSbwKt2JGOuGJhcddbegKqrcLg/3Tz+xF/Rer+26kxBU8GrL91rDzNG7odjcwyOuph2fZ4T9AQ7Jb8z7zjmE1n8Bs2ySBibSonzQgRkNDg1kB/TSF5wUbtxbuuYbQV3tD2bWgEoagGqoubxbk0vGHQQMn/CK3ctvo4tbJq+C9FswqXM6lRz6wCjb1KQf5gGzSREi0CLsgXbPnH1VfhTQKhL5cdayDSAtr4vj6r86FT/B5h0jAiRdMcw4wVURmnPlFvgheT5+DX+KLXT2rbJcZMxv2cmICCccscGEAWbsy9n7LPc6ftEu/kWBBDNRlQF2TjfH3DBg3f4/1NLOx/G/4byP3b2tS2RkZG89dZbLF68GGdnZ/bs2fP/Vrv+U/5T2hWLxUJ+fr4tQbyl61d8XxbY7h1BEPDxgWHDQCm30yjeT0MB0MW4kJcXvES9mycP7VjJow5nmBr3CSfzHuKDk38zvTAKXYUbcoMJk6KNZ6+zRE8yzc2OsgOBcKT8ZpQcQPAfTkrpSmK3bWbuWolKq/XSLwjw/PFNfFa4EOcmCdGvEBVQlwH31pPgtotX/TMJU0CCEn70OgSXngFRRvSFhzgccY/ep6Sk6Ualo30jvbuahoIXSfd/h1LXndZWtV/el+lMvo8Uges/+DwD2A7Zv4BrArjG4+3gR7F/MXvH7GVp1zymhTwJuZLzrOXsfEY6zsBiTCU0Jx652QmtPANfMZNnn32DnoNuMG7Cal6seYdXLjzNtYY4qpV+sMWZYDYzKGgrP3qDsyg5fCpagzGcwtscBhauBsNzqkOg8YWNKlRlR9nUYuRa52skpCbgX+QvzaNDIDhFYnDthWfBOgAKAgrIMZ+WqjI1QVMBxdq+LGhq4pfZ6zEFDmdf1lx0MR9JEQiOYVTEbqEmKI46rwgwW3m9K86hPD2CTj5H8KodRZxsDB3yviXeYQBYk8Y7t1zDq3Yk8/9YxOufvC7NcRsjqv2f/+x4jTZPJDF/FfKmWCYVw23NZCmfR306pH3B/CT7gTpHrZeMt5UXoeIsf7jPZniRCW2VFk25B73cMwmrfVNy2lopDgCc9ZFYMLHNHzyaL8O0GsoMnZh96UXeqbRH9v1UJ+HIxGuLmJX4Hl9FpUgttxoc9YggqtAJTpQZISbVi+e+fk6ai8ZcCdWLRA+RcCuB1JizfD9nEtvqXof8bZD1C9l+01lebaddGOcAcmMpCHI43J/A7NVsdGihW/UwTvc+zQazhkv69yFqAQSMwSyq6HN4MEuWLiE3YxgAxbJJcHomwbIdnClLpsTqwGxUZnOj7KrtIn0/jWcrFYZMkCEIkOP1LXu7OHDS65F2ERUAQaZs5o24yv6xd7ieHE6szJrD5OIzsDMCpUZNRtJMFETT5/wUOjgMlxBinZajsuj4PcAd3/IO9Dzfk9DcUBIUAuqbz8DxMVB+Ek9rVFKNtolyr3LpMtjtC66H5rDq/C88eSiDlogOmOVKK6LWAsYmgpWHeaTDYpw7C/w872cqvKxJb7z6QO/1jEt9GQpfQOfjTolfiY32AyDELYSYomXc6fccWVHlONc5o2rtd8wivjaP4Mt35tPraDOn4ruTon1EetZ/O4y8yNhh75HX+W1MckGKEkHEqPCHhMVgasHx5jqGbT5GiW8Jv07fyJBt5xHMOsj/iwYxjsSLfijXSvvnH7P/4GRDAIhKWizuPL37Dj5yIxMP/0XHw5+TeyRTQirn/MHGkBPc7HsH3xI/Fny/AA/dk9B3o7TGTE3sapQjN8jpe8yPDjc6tFtztgg6wWjPQSJTSnWLcvSnHyaxm4R4G7J3FErRTMA5L8n4ofZGFEQmb53MkiVL6HG+h2SIkjtB0FS2lmzgtcNn6Giax+irTTwsroHtfnB8HOyKYmDMMiqcU3A0huBTOxqt0svWrlZKVZPYTKM6nTqFPZk2QDf5WbZNucqhj5dw+J0VrHH/AbZ6SJRcQZOpsDQgb5H6UxnQQZIRfRWcfQT2dyeseHa7+qrUo6HvFnCJhfQ1nLUiH9MTlTQp71Hd+QCMTUMU4cnITSx260aHnO8ZlHGRQJdAKYH6qKsEO5xh2eDB+DmdY0Lsp/jLjoJV4fYv+Jl1PpAbXEh5ZDKmVtkF8OxFvveXlNZ2xFEXjXfNaHwsXdq1sVyfS6HnOkr8SijyrKARV4mmDqAhh1UNa/iipjfhWeGEtOr4hlrI34pr3jp+9QGTrJHj8Z246/8e4REvMWhILoUR7/KmSyiPOQu8JXclP7QTppeg/8wBkP0z/npp/lvv4rZoD2vkkq/mNlsnhjHI/wiDCyBSXgnHx8PISxiGXEJAZOyusUzcPhGw5vVpyoPD/UkXO/FNbZtOtg5I6GyIfQmd0pMqT8mZ7FPVC2eL1cGS8BaEzqLBMY5ci1SBDfnt4I+pz3YW7b/GuEIlLdYADwERkr6GSUWElc7ht4C2H24f2de6L9Q6XiSlQww7LO1pPBuFEipdjjHQPZNzg55HZ6yE2BfBYsa9eCdpoZDgKsfdvIQo7yIo3CkZsuJe5kl3FQuHzaDORY+rysjaLmtw1p2R8jRpO1MZ9AiFRlCYXBl84x7JN2+BuQVN7nckaPcxx/8yjzn3xqd2jHSeNpfAnU+kSF6g7t4SHqsoIaRcMq7206dB9u/onPuQ4TOG3MSRODU4ITPK7Hc/11g+Lt7DqcgxVDqfsE6FdTxKDqKoOMXllttUOZ/CWwadVCC3NELmD+DVB6NnL2pda7mQdAHH+mY+SXwBN30KxL0MfTfRL3oiybULGHh8IN9YPLkwegxBqV1BpsSkCiLt1lr6lxbRp+kT+zlk3QNEEZybEwg1jELTEmabAxuNZ5vo9FagTqw+A0oOcVgcxFOHu+KW/xulPqW0aKye5dyNcPFpnKyRk5sf2syRIUewIcdHXaY+/l1i29A3AjSaK6WzxmwCfTWO9ZvxdL3EifgupCTEk6e7DRnf4585gVDXW3SwLk9b/kRr6ayqZEHpKZJu3Cd7hjoIf4xpBV04lhhDmeueB2RTZ9RR5LaVIu0mbgY/y62g52g2NMOEbJhlwSzX4FvlQWRWJG4KM6NcrJzD+mq49QFvyNMIHnuCS90uEZkfQk/3mxLIoe9m8gccY08jTNw+jHeXvcuSJUt4pMkTmkvtxrkL8wjNvkKW3ydYkEnU8WIbvro2be2ohJ71p6ClAjfXOYSmSwaUyKxIDHoFJP8q3aUP9CBQ0YhWBoJFjdoQgqfKalm6vhj97RWsxpWUxDiGXy9n/O06KYq79ChceYli33EsECo4PuQIB4es5YsqZ6p73YTuX0Hcq2z1+YFvjyfglHcH58ocqV5RARpfujj/SEsklPY/y7FBx+yG9OtvwhZXfBSSs6hT9RJ6Zuyli6uVkqnLSlLjLrLJ6j/+p1w03Q2PsnO2nK6+2fioTfzmvRFuL4dOH0L31aSEbuNVLdwOXsT1kCeo1FkdE2PTYLqOrI557eprUHSFpDXg3g0KdvCa41Ji3CUh8pGZ8UsdCOnf4Clc4Mubb7C4SjLjRhS/hlatlQxko66Q1vIorWaO4LIFxFomMXUqzB+6kXkD1xKbu5uBt+5wKWwqxxKjSa+9YWvD/VT799NNzzNM5c6InxhxaTgj94/ER6yR8pEmfQs9f2R2x9kkZD1J98vd0VZrJf3AYpHuDX6j+EsneQvKsnfSl3/TiKqQs+FYAVPjP+FmmBQhluI7mQ7fdiC7WsoZvqV8D+dy59Kt+H0mFcHi6iTovByiFmCJmI9erKX7pUQm7JDyQgqCgCgYeaTj25iQc9fqj9ep8ul0dw5MzJVAK7EvYlL7cvRuMAM/fJn+l5KkF/tth/B5EDiBP6NX06fyjm28WuVEVfYXHXyOgcU+hrb7WdUlTHIPxmcmtutvO7CWRYNP7Vj08goOd/InxW1eu3dl9xm1VQYfHEV36Zl1f528dTKfKj/lsGcAlU7HbTSerevdYrGf/W3l2ytrGcneF8gMWExM7TP45n1MjY+Ux3pVX2e86obT3WkiokVJpuM6NtzaYO9Dm8g+jSGQGKceUs4q7PJkFozoFWW0KErbRDNaQR+YqXI6TabvCtJNh9r1sbWNFwrH8cyeW1gEBdxdDXmbMbj3IDoXWuqcUNeXMz32Q0JEu+OQmls4WLLZNaWATQ+txSCvlOrr+SMrVIfIMtiIKx4Yj3+KQJeJIitWwA8/QJ2QS7XTWZqVOTZ9r5PHNQCKdMH06SnlsGvSNHE2/ANUKli1/B6/v/AyCYLEVGOj52u75tJXs1idT8xNV6IzoinyK8IpxLpq/g6E0zNIMYVwWw9+1VMJqXgCB7kzeCZTH/8Ty07s4skSe05Ym/ydnUNU7RoWlkHv7+bR8UZHdA5WfchikKKEAaNFTVpHMxnRGQgIPP00dO8OS5Y8qN8LggBDT4K2KyqlMyNrd9AzbR3+Rf6cCMiTaOkBRlzkniaWSzroe3o4ww/PJLT0OelZ4mLo8S3XjWoi02K5wmV6y/+1o6+19AQGimY2fWvNtd6mbeMcYU38LtIqe/Fx/GY+qB2MQ4OaneN2UuVVa5tnfbc/eDs/GoPCwKmRp0iLTaO+zl7P/bm7TCZwkNczVDOLQeYcANSNOkyIzLr4ImVhPREEq60LKQcsNSUYndwQhwwCJIedu0Zas3p5ubQWNXbbm0quaqPHmaw0nq3AXenvtbpaGlR3aVbkYzDAiYSObAn1YdjDw3lz+XL6ijBE3h70N1Qu0l8mY+ZTy1kxqYlxykRe/PBFTvQ7glFWh0JsgfSvofw05SYfCpW3KNRL9IhVukCE2Och6yfCcyWHuUelK+5XWnMM/3MUT5H7BnbmrJfuM9AuQhjac51cLj3LicQOXAyb8o915TXfoU5zHRMtDzyT3XeGedYNxUNmvVvGLoL414kw5uKsi+Nq+GxSEhIoVdpTfoS6hTIuaC4yswOlZV8w3/wg3fA/lUd1LZw9k0I3v7286+aHa1NHO5uVIEJTPq8EO/NY1B/oLdBLA5n6KRJwvM8GasbmIAd2jt9JoX8pxX7F0l5gMcNf7jxq3A/8Q0R+1RXY3xWAE/WBbDZJaRZseTC7r4KYFxCvv0E/nzMMup3GgNTrVr25GnaE08V1Ow5WUXfAA7nZmRjjx/CniLPhMmq9P67qBFJHOZIaLwGL297hW+Tl1FkKMYnN9v5aS6u+5SmD77whXlEJjkGQthLfzM/JCQeTUyLp3adQ7RsOWCS9WpDRteU23VWwb+Q+/p74NzZnn9W2zPFxPOcq9TMxfzV/n7xASdivkLuRkqYo9mYsIMv5Y5v9Qy6KbLvzCuXqSe3a2ZqD8y8HB66rPXGt72rXiQURiyh74DzQOXtRPmwmBXHDHpCF/zj/pHK/D+P/6yL/71/553LixAl++eUXtm7diiiKPPTQQ8yfP///zbb9p/yn/HMp2o9QephcnQO0YTds62xpG8nBPyS4bjS3EO5VBrJKXGqDKBMHE+VxFK26hMrmQK41u1EybTNq3U4Qf6aw4Rju7kEQ+RT4DKHbhQC6nzrJpchJCIhcl32Bv59acpwB4p3bCBYLolna2ZQyBZydQ6ylAy7Nr5DcsgZD2Q/cDH1KiuwrPQIXn6ZUt5nRF5/igvZ7fve1NjbhLag4j0Wm4WILHBx2FEddDLc7v4IgSAZKLj/PAKAwAEw6CdVz/wX0D/Exvur2Guak4ySpoK8wiyGj3pSQqPoa9vR8ms+GTKLCJYA+r74N3AXfYXB2LjrHUJZmXqPfyYFYVDHc7f0YPXrMg8O/Ivaow9+pkBW9XuXz9VJUR+LexWh7rQB9MSbBiXiPQ4x3hcWVUI+RchNc63+azv49cLpdyk/jerKpMpAMPYRoZFCfCQFjaXRO5Kkd7+MdlM/eUXup9KiUDO3azjA+g9oKKPkOjj9pJMP/J/pivSh3Xw3lp3FtyuTErWD6FHbl8URn5nZ9i0btOclwqHBG5z2Vuvr9+N07RVlIEqLoAi1ViFVnGBCykcSmJCrUizhocpMOqehnwa0TumpP3jR6U6zQAudxaHRoJ196sw69rNGONn5AGcAqn9LGf97lBRKGbAXAkLsFRdbHpD5n4C+PD2l29uDjAXukPIRAzYkPiN87jD5n+lDvUsu8FRsJqC8G3xXg3R9Th0Jmz/MAgxZLZy1f1cC3ykw4PAhRPGZrS6vB0FcugsWMkPUjUR6jiHDLhxq7IvdkczRXk3+mVJbJ2k/30e1uAyf6HabOpQ7Fnhhbn9JDRJb/MBFVrRcn+oKbqIOT0gXVXyEZNlqVuLfdQdN0UZpH52gIfZhzx4fieKGOmtE1ZCnVXDe9Q3crCtvLWI7GXzIQuW14mHH3DvLkvCZich3xEhQIlum2+tMCX6fPur9YPWo1z/Z4th3tjEUwYrKiTWW2S4sFs6wZk9Bsi+z7QDOBxYM/xvfgDOa5wPtuIgbPR/BvtfXJHaEhmz7iSK50eo2Wzdloi9MQxS4S5WrCm7hVXWFElYzs70baxugjowfykr+lNa304GL4TT75KpX1hckS1ZNJZpMPk9DMNe0nZItqzM5DOFFRSWRlPEEHExGECN547gVMjtdpmBmIo8kKj3UKBadQwutn4Xw1B7T7OJ8ktKPxtAHkqnLpcD2GUu0ldDJPKX+n3IGStHcRFFW41GrpfLUzVaFtNllTCzJjHQrRmTMj4lH5peJa0BWDIgg6LcNyeBDR1HLdPIGuV7vS9WpX9o18HovMAcHUxFBzLwZ5uXOy70mS/UcSIevPSMcCEERUQhUDQtdTaV6GSaFBqatn+bon2KhLkdZI7QsMuLkct9B8CvUlvOmzEy6/KI333VW4ihrcMsPocMMHs9ABV6XW1mxZOxooq7PP2AAV58ExlEOuI1kTvJZb8+7haVASIQ6iRTsMdexs0HbhtdQB3MySLrLV2mr8BQEc/KHfFo4du8mBzlrOiL705g5PhPSSPhoyHaou0S/8ZxaoPsdSuRovnqVtsZ1Tthxd7W/CSXKp76UmaLZAusGfaEURJLwNfsPpoBrDhQmn2Dt6LxrDLkoad+Lt7gH9tmO+uABTzSaOJV5i0C0pkq9FGQXBUVLljiHccq/l9OyNXPIoZYrzSVQ18+HUNJLkL/DGoVPEd3QipCISpRLUckDuDw7+OMhPEeJxkrOGXiwLhCv6qXQduQWuv01AzldMdoLHYtOQu93FZErHbFkIyODGO4iWZMqbgvEhGJ/ascTFtesyA4UddCuO59ie3pwYt4dHI7+H/d9JAA+1D2bv4eT+7MKjlkcZZ/6Dpzqd58UZrpDwFmU523j8L2nPqXe4gVujhEZXqIIIi1xKt349mO70Mpu/mAlZsMTZmvh9+DmupAi80W8sUZ5nGJXync0ZIxfM9Anagrdaxq36IDIMGRxrhu+berK0l7TviJVn+MLTQlF6NA71LqQMTJGUU6U7JL7LvfRGYK+tjzYjlkd3KNxFWPEW1naSM/6ZHSRUDOTH8QJsUsIMaT96LvU2vymkCDhbZF+btYyl7foWJNrJQ33xAHxaI3AsIi7NHVHK7Q6L+yMi/pG6Fagyw8lmqHecRHj0HNggw0PbnR8uTaXf9x0oCz7NC0cy+HLBl+BmNdpef5s4l+ksnCOwMHojSVqgoA/sTQRRjslzMJ95wrulJhz0ocR7nUbQSxEbQU5XGBv3DjdbnsEs9KYFHYbs31DU3YVR15g0rxNGo4WgIHsU57CWK3C3hoNKR3b3X0qfs8N4ZeUr/Dr3V8n41pAN+X9j1p+gwuUKLo2Sk9kGhPDqR2P8YvKzJUDHN7XSfzeTk6Dzx+ASj7YhjYK4CD7K7EadvBuXascR0aEP0W5+UhRASzk+XWSketcwsMQXuItoMUD5aVR6HyASs1maN8t991JBgNDypwl3fpq0qipuhzzXbo7kgmQokwAL0n4xtHIH3EgjR3iNO8UyRtyI4XK3yzQ7W0NyY54Ht448euJTmvO1lDc5UOZd1i5qxvX6S5wKAs9s+7x30UyUaBiPDIbweXQo/wWfHlBnDaiRi3IIe5RSUw+Wb57C9i7S9xrVGRQ0etLRFITiUG+GmC/xxZUXcais4JlLz7BrmhWBXXUFjgyin1MUx0wZyI2u7cYCoEZXw4XQqe3k0WBaBkfGgc8gRpRvZ/7sK7gn3sRNr+Kp/ES+f2oDpH4MOeu5EPYaObc0dL8s0LFyHB9++BSkV0H31TYHr2OjPb/Wy3UBlGSe4tstqVxM+ZNmcwEODlAReYh9wSOJnrSdULdQxArwcsjjNf9UNpvgjE4yVg2q3gP1L+JqLOapCV/ya0wER+TrOBdQDmmfQ9TToPZm4M5imrMv08tXRmNAG9Nt+mpUhjruhoDfPQCLtM7NJrgmAcq6JK9l8cwePK5bQbPvSd5RXUJZsQfi5wKw8+xTGEy5QC/2D99KYF1HREsA1ObSYnFlXyOkJd7G9c7fDGuxngMeyeCXRpd7laSasSGPbSKidMWgiabZaqOodbiK3KLBbEm0zZer6TR6nYKg7TL6K6dx7MVrDJIppTUh05Ch11JqlAz5+V4/0Wh8E7Dmk5Wp+ODKcxyPTwFA29gbQbDnLMSjJx9WB5Kul6j5ZnqUoKm7A5eO011IBstvkkwaXYkr/ATPNqzFPRyXk530F93zsemBGg1MHFEKt67iYQ7ESRdLszKPZlVeu0hbxf10S5b2e6ODWaL1NCEQq7Kw1OVLSPkSxtwGh2DE4xPp6jSIUrWeRlctoiAHtRZGX5Pae1hyCjXnHeFxy79vRP30rAu9u4zD3Xdju2etc1FpiOPF5CF45w3i4Ubo7iitLWKex1J9iwGaT8kOz8av2A/vUm+rs89Ev5DNyFuS29VpA0Lc+x0uPIGs/9+c3NcR54bfOV2j5pEz37JuWjL4WQF4Z1bSIrcCzyyC7XyqGVLJkfc2sS3uHebWQn59FGGuyZC2CkqPURqfgnDh5X/+Nm1kUbgvGttapNQO7cfKRiFqvfu1UhI2OlRjESy255W6Mmoc8mlW5vFL6SGKfSIQBHtbKpSBbCkrQIcU4dBsdKYydBVupV9Z37BgMgn/CCoWrSY2M/YcrJQeg5YKhpsukuMAd2yJIuz9suWAE8yUuxwiw38pWuPTgDTOebV55Oor0Cn8aBSbKNc54Fi/H+6tggm5NAiOcNIT9yp3SiN6cqFhCiN9v4H8oSBzwDj0BAe/NPDoj79zZHAvysPtc+VsqSJOCXf1N629s3A/raAdPGmwjr/Mdo9r7bN0Tkm61pniQZTVdsRVlc+rIZcI/uAS+UZQmCXKxHCfQrj5OW8gcMisJnD9VCo9KjHMsH4341soP8VTFSosVhnYM2YP3WXWSNDgh8AxmJUXUtndmEv8fRHEdtOQfW5s8mWxUK9M4Nva84zqctVK0Wd9T+EC05v47ectXI+YSonzMdsce3jAe1JQKn32vUBqTjl3gl6TxkMQwbsvjLoMtXdI8vgD5b5yBuU9TmPQTxDVBMFTwKM7O/0WsGXr1zxU5UhJVD/URl+7vJuNaIDs8GyKstL5oH3mmn9ZHjXAl4fu0KnfPbbE7+H7ZtiZP4zkkBiqtNfoFNUfMtczs9devnMZwF3FXbC4S+PUUomyaB+x6mZSq7U8+9WznOpzCnmCXQZqGhv5+vLnmC1mjuYcJbOwggTZRu6ankSuLSJJG0qvIwJiy++Y3nwBnaIEudIemWc069HWFOBy4RBG74doUmpoMLmgUWigGUyiDrMZnJa9xisN0lwoRIV9zgQLBqMZbWMyEcWvMXSMBDzYeGsjB6IX4FM9EYNhu/0OL8p58aWXWGf4gbrf63g71UALLRgdDTiGzeHKxiU0NPgy54OXSBBSSbh+GZmHTKJ9ljtIurFMxdzVb5Di/BnDDS4oibcPeNE+3Kxyf3zAJTpUdLPKSXvHSKsDvfXO2nq3a7umrB20/ctG+yeY/9H5/uTFHjQnNDAxL4Ny5xwcVEpajD1RyVXt9iKVwYfk9EN0C23zmbQvkFkAnqBRlUWDJhVjYxu6Auu36jU30Juq8eXfKz5AU1MNg8LWMcCrgCtH7nE0viN6RbnUJpkDNxvnYrE0E6WA5wt8Ge02ncjGXHAIxvnMNHLCIJBriJ7P4dfUkQCnEGnw9NX0oZoJjnBDuG/TL5cclU1B0xlwtG1qgjaUlypP7nZbx7cpNTRqbuDS3LHNXceNgznvM6LmXQAMZgOgoUX0BSw4GO8QWPUIY8TBKNb+hHJQAscH5Lcb55Nx3dCp8gkwSUDbtg5X2xltguGFEKoJZkZTgQTAzVnPjr+NdMlYxMGeyVR4FfJubSadnXzB3MLnHgv4KPM7PKKiiCp6h4ACZ6ndnT8CvxFwZBBfelvIM0IdZiqbA9GWroKma1wtf50Vtx7FO+Brsn2kNryutTDZ7TOo/gzurmJQ2CC+HvU10R5SPuUrf3px4cA4HIcqbZF96spCjDq5dYbt8gHQHBSNsey/d+79n+rs+99d/keRfUVFRSxfvpzo6GgGDhxIZmYmX331FUVFRfz4448kJyf/95X8p/yn/D8tdWkIuX/iLm9/82mrALSL5PiHS3ieyR29TsGZh/dxN3kO2fLnudr8EouSH+PT8ZIDLzYtlldWvkJwjsp2aUDuANqOmOQe9mTMFpEGMQaaCuDuV5hzNtIzZi4FHSvZNW6X9DNRDrkb8bKcoG/wJhKct9ouykqZUtqs+20nu3EAsszviK/6jO9r4dOaHhA8FU5MxCIqWFQOZ/qc4NCQHyl122HvfMQTZBAspcO21ivet7wf9+nET83vE9zcnYstcEqULshsVMLR4WhvT0dZWY3/vVu8/ucR3rxpBoUT3PuNJsdotqZ749SoQefowfbpChI0v0LXL0FUUa9350ztPOI+6kS9ezBOncIh4U2IWkil0Jufb7yPaxaUmSSU40tu0OnkAKi+itkio8IkkudwmodKQIFRos/rtxV9zAtggW6XuyFaPDCpAv6RmsBOXdrmJAmZjkmmYdHVXgw72Jl7V0PYk75QymGy3Q/SvuDjS29ytMdLnO59mudGJ/DjIAHcOmCcVE9tizczO7yPmXu0yEsxWJpB4w8OQXhUrmJGwvvotT4oDUq8yr3aOZsPVX/HwS6epAa99IDsAeiFenSKIvxEFV9rXQjVX7U9y3fpRmLZFTZ7fIgZ+KZSa3P0ATibG3G0IvVdB58nxrmYWz5HJeoiuQbRNZJGg9Y2Lt/VQpMyDMqOS0nQg44y3ckuJ8UmGXT5DIae4P0Te+l6XooqdG/ozfrku2x7SMpPEOoSSVCeN071Bo4OOUqNtgZLG4cKQgi13ZOodofOVzvTbFHDgD0wYA+ZoQsJldudfYvKZFT5vALV1yTZDxxHQ9STXOoXgCwui+LOp+go/wRSxsCOUEZVbeX70bco9m+mxcHNuuw0MMNAkTCW48OnMNFRcu63XqRbFYt2eTUwtVMAwE4LZUKPWZCMPWa5E5ia2OvyHI+sGsvoHdZEz63i1WUFDNyPg+kUFUHvcaWPL7eSk2gwV9rHo+cPFESntE1LTVeltYL418GtI+k8g+H2VrwaBkttQQYlR/Gs/40Q7RUigj/nkmI56f7vsST1Ic4UXoCwRznZFMbN6DSqI0T6pp5kQE4b1G/VVQb4fknQvSPE7Ctj3i/z2slfi0lHgzqNZmU5l3rmUuRfhCiTw7nHYGckKsHMtjnrSI/NY+KOibjWWA2iOX/Cubmod4bQoWg5kVkdSeYjtI3JNmX6ts8U3vS8ypqF35MVnoVO04RPYBm6hOW276uddNwaegzF5kZ6/RHMZK+D0PFDyhjAjMQP6Kz8hHrPMLbMLsG1zwnb78oNHQg9vQVZYR7FUf2Jd90Cd78EhyDwHcoa9SOM2/ccJrkSk+8vzI6009LZnWpGm3yo627BwWQo+Jts567sa4L84Hxm9sxiS7efqI36WspH2ZjHDGU55a9+y49PHQanmSgsdn7LcM0lejrV0EINAAW6XhD1DMS9Aq6JVBokdGqOw1byPX6lsqXY9tu25xQ8aMS6ahrI+othDPtqAcEnevFG7eMwywJBE6E+na2hKcyRa+h2uRsOdUUSYlaugaCJVI64iU+2iUZ1Ogc7+XAj5Mn2F+2AscxtcOSKthyzaObH+F04py8CjT91xHKvpjOVeimK3PY7YxPoa7hYs5Co631JaYaeefBJtVUACnfRognGMQuMYgM3AhZxO/h5TFaqMIr2oas7wLXQuWR7r2pft7WMFP8ktMURn4JApv06h87CJYkmsewE6Mow9tlJUbiEjDaIZiwyrZQbD3AJHE2f8D241w+w1ia1K9DlDlHuF7hcPIbVlZdpSOjBwdHSPIiCCBpfDDIfchoDOG2s5mbI07bz3Umu442+DxHldJDHrz3FH9YIl7OGEIh8HEqOIMv+hkXuZvLGSOvQr9hPcrqp3CHhTcLM6UxyBMfmGHqnnaSj3k71RegsANLv+aI9GUajXkKS4xxte6Xt+hWwUgM1FSG7tohnkp5ihZcRjQChpc8RpeotOfsAk+DIwzcfIlg/goG3b9M/9SreDnZF7X7k8f3y1/r8nA5GF0GBojPINBA8DXnMs5zJ/gFlk4XAtMOU5+RLEYWNOXCgB/Rez8nypXiWq/igQM1LN+dSon1VAkQpXPHO+JwJThCmlGRfJW/G4hBMrv9STpQ+z4tHd7K3YTjZPp/ztbMra678Ip2/FjMLuz3B5LhPJXS3de2sdR4PvX5nRNX7pMbfpNgnn4vdL1LvXC+NX80tuPoyO72uEKEA94a+RBcupZPamn7AKQxl5TlGO9J+DESZRPXrGofS3IS7RY+LyYvUyMsc1vniYLwDKaPh6iuYzs5lvGsfvMpd2HPXl1cvvUdhwmk41JeQ3HH8PtGXZO9frUYbe4Q52NeB2QwysyNzHTezccpG23Ob4Vgw2IA6KT6zoPPHiILIxaSLnO+VyaO/z+GRHCvLiksMOEcxXZNF57wwpm6dyqgzxxjmYY3gzPkTi9qPT9qz4pHkOE3K1xw8HXyH8WFTF3Y32p+7K3zBsweN3o+iN9m9K6fjejLxWAQ5NTmg9iXVPIbDQ8LI6SFKRtpWOXYKh4DxRCibkfHP+9/9qHRpLpCAcTcW0yx3Y12Dhf4n+vP46uc4UeEnOZrjXoEBu+mR/BGe9W+R2W0aoYOsFMyOIXDrQ8L3RXA2ENY9so7Vz65m//xfeTwri5Au0zn36XJWNqSxramB7yug86XbrH5rJNNetDse3TVFLPJPI9kaeLC9ATb4PQ/u3dGY8+kfsglPv0ryg/N5vzJRuuP4DIZuX+KIP8H5iUSm6Sl3WEtBc4ZUyehbGKOf58/WCLrWKEkr1SWAWpQRpPFk8AFvXlj6Ds8EXMP1+gRJ9wD6CZnIR5xk1fOryArciEFWjVh3C/bE4SCrZFKhggwDLIvdwPPBVkN56AwInMTqsOM8UvYFwXX23IQAFO4h7lYHxji2znEyx+M7SfmareV3/qD3OV+UGaFE3U6g0OgGqZ/A7Q8hdwNPF4/kd3sWBbuzuXA3ZP5EfuEP1Dvcot7hFrUOV3AznJFyBJefBW1Hfm3wJ9cIibnfkHLuKLl9LNDrD+4Ir/NW91d5Q3tfPremArjzOSqxjpO1vjSaocL5KOWC5MAgdhE4RTIl+l3rWFvat4v/Hgix39PIjNU3UZZrKO5wmyqzMyS+AznrIfN7hOrLODuVY1b60zV7J2rB2f7jq69z0D0NjQBNlvz/kRG1samBz8/+QXKOtt0zUZAiB404se76B1zNn0Fi7hqe8+gAR4dBzU3E2x+SEghGpR63WjfcatwQBQGjWcnrh04SJCtgYpv971r0Wim9gLPVMXzmESZ3eZ28Ube5klCOh0IHBTtt7/u35NFRaR8vm7NFJsNRWU2cQy2fh1xk0K10EtUjIesHMDXhnvMCI1zsm5CDLlICyFqLSdCR7/EbeZ6//ONcCPfpV466WFqQIh+VMgUDbqXSZfJX/Pn5nzQ7NCNY7PR6W9LWcyq+O1fDZnO0/ltK3La3y9P1V9D7LCwHMFMr5tCoyqTF2Agz9Cw6a6Dc5TAna9bbaBLbntUOFk/63DnHpIoLVDgfZXvZp9RcfAnOPsokQwpzVAE46uyAyVYd0h7lY4Z/sF8sO76Mdwu7ke/5C2c6RnKjSzh5zpNhSAqovVHUSrljE1IT0Os28ZfpDhUkS3TCKSM5eO4dXq3sTIuyBS+9mmUxW3Gs3Qs1t5llWERqCDiJYLZYOBHfCY/VAkeyj1jHuk1kn9g+r7vUzjasStZ9PcH9Bj0Dd3CwaCEdcqGk1XTTGnHjkQS9/kDEQpdqbyKzIul5oaf9PGjIAZOeEy0yLvS8wIdvfUhhYKF9j+rxLSS8aRujEre/KXXdI0VRlB7H+eZsnkl6ii/dpQ1bYXRDq7SCDXr/ztWgvwGo9Khk8NHB+BW2SdeQ+QPLHGcT7r8VQ6sj+757UjfZXCJKX0VhkECtgihIFOy746BoL9MDF+CRJ8njuFo/6Q7v3h0MDYiAwqDAoDShc/JkQMh65C25cP5J2Kgg07cIS0QuDbLG/9E+UWNwRhSMhKrr8JCBR0N/kgJW4z36JO49nqRZCEDtqKNS9CEkN4REi4B30waoOIfq8iP8HX0Zo9XGptQrUcjt8ldR28S7Ke+y5PgS7pTfociYSgvgJx6jvyWPzzrvp85vKKb4RG5vusg1/478He2J0SzVZ7QYqHYLxODhQ8utCxztGMa8axGoZCqr7LRgNmPXG5D247ZyZjSZ8KwfTFzhJ0yKkc7mtnqewWC/V7Tu5T5uAmuSgniMZ5no8zCukwIIin8cPz9fOzggO4Psb7OZ+tdUBGTI5JJ+gFJrk6+Dhnc5Ed8FbegHUpRmx/dJ6y7lHz7b5yppj1qd4ILMVm+G34esyB9PofrQP9ooWsEkKr0vQeaB9n4LrVG+JvbzIuejRnGn/lyb51IdJouR89EjOBY6gKrmKlu/5zjDzWCQ3x8FV3Ee/EawyWk0d/3fpUklRcG13WfKGss4XbaHSucTCCooeUDS/rmUAt6aCv68uZSf809jsYjtwW1qL/ZW/cr08L9Y4Qm/VLkSotgDO0Kh+gqCyoNDlWH4Vk8isHo2IeUL8FB7S3rekGOUqBJxqpiKaFZzMWIC55zelD4cPA2Gn6cp5iV6q8FdBPf6/ihM7lIk98Vn4e6XfHvvEvsCnqPIfZN9TJRaGHWF0zVv2vqxtWUBZS77qJRL+qST4YY01hrpkPSpHs3iTt8T4Bxgq6fVadt6R2+nu1n/3WyBQ03wuONNKS2CWyJ0/ojfb35K7ImfGXp4gPV9QbKR7etCjD4bnQWmrA+lx8EjqA1+9rn0GQjJv1FtkqPRRSFaFHhoCmjwfBgUbpgsCoZHfsueopUUuktMaEeaRC4WjqFcMRacY+js25lnejyDXJQzedNk6nq64dHHjEmuxGjdrwNObCT45p5/dNj9q4C1/zj3/v9R/m1n36hRowgJCWH16tVMmjSJO3fucOrUKR577DEcHR3/+wr+U/5T/h8WZ2ershTzPHj14zV3M1FF75Jc+Cf9bl8h0CnM9q6NKx+jDQ3YdtM9ZHmYUWv7MWLVSModPuZ0MyTM+pCwcAWxjn+TE3GBEfUDqHarpsq9yv7bv4NgTwdk5tp2ufGU1EiRHF1XQnMhfeKvcnnmahrctLg1JEto+ocaOa/cyiMd32KQ73u0KKSjUyEqwCkMgiaiN7vgoirHUWxCI8B1vZXyrdMHVARJSJGgvCBG7B+Be6XWvpH2/IFV8idosdgPmfsVorjiYXStX8+8T8Yya9NiwtT+0HAP/EZC0GRqErdQ06sHAEpdvWSs9B4IDzVQ6TcBg8LAvbAC6jzDkYvW3T92EYJMSW2LNyuzp7LPIuNG/zEMGbkJ0ldD0hqMohstRkfqzFKWHLNgJNsAgsUIKaPQWTyZff4dVtZIVdY5zrPlh1Bn/8xaX+hxoQc9rj3KoNTbUiRkxTn4OxjVIWeqk13J9JOcCTalr/gQKFxwbrrHmMHXACgosCDKdKivPALRz4FrIh82fs7c7FicGpwY5CsZTDA2Igiw8+4LPL0nlZ+b3uNQZ19O1a2HkkNw+TmqvBax4swmmgIi2D86DZM6TnLaApx6iPccvmg39vfPxUXFSg53CiDcEMsznnX01K2XopTurUfRmIdvsS9T/prCgJQB+DaoIHezlK+oYAczyn/js/mHWbJkCSd6n+H7e8NoVnWEbT6w1RO2efNGn6l4O+bQL/UyA26lYnTtC1hQyPQsDDvCUufBJKcf5Xl9Mce69oMb74J3PwQB5CZXNCZflEZvgh2jCas4APnbcKneQNz0Es6OtVO9CVOrJMVlloX9xmyqjU/iW6Skw80OmASl5KR0iSYxYxlPuoLc5EyX7PXo09ZRGfQxzDBAxDzKGst4s3g4RW7f0uu3WVzJDiJJ8QZUXQKPHhT6Tub7ig6cGOVFzKRCds0U6KB/FkQ5OsGPK5WdKDe1OvcfNKLGF6zk8focgsufsif1bjWiWqxUO4LBFs34bv162N+NYEsp6jpPwrI9iD/2AaW1v9sn0X8EHfLj0H4wGG3ednb378Xhuq8lOogDvSDrZ9w6xnNy8lx+mv8TW6Zu4R11CfqkDRJiVO3JZVZT/2cRD62R4VPiIynK6asJLZnHmIi1/O0PLqJ9TSNzgF6/cbg5ntNj92FSmum29wM0tdZreNavsL8rk0NexJikQucskhWR1U7+0qvSSEmMIzDtEAF5bjQ6NSLDLOXBbClnuTKFe0EuVPnFsH7WemZFZ0oGnzOzIXcj6ytdES/J8Mw9CxUSVaqj4RocGYLC3MzORqjwrMCx0RHBsZkdg+4hqDwh+TcKxIf43LeJ0VUSdVlReD/2lyyHu6u4KEr5CrTy5VwJn0GdtoXdOjNH3EZD4CSqDHGoGyqxNFTRoMrgi6J9UoeqrkBjLiGOaRTFDCO782TMClW7S6bNqYbJ5sSxOEZITm7PZERBpMf5HsxZO4czlRp+K0jGovSAtC9AV8y45q78Idbj1/AqMUVLUeIMTYVwcSEvBczjO29oFEu4EDmGl0riJacHwJibzLorOXJLNSlcD5tHTuMdW7sCnIKILH6TYCsV4f37RLbiNbaeegSfUl9GHBxBb5dZsD9Jmo+sX/iivAOpd0IYcXAEYffCUOnybL9tF2msKMMgq8Wv4Ttpj6i4YPveY78+xuRtk3n57igJCKErJYt5CJp0bpi/4mRcEnc9V0g5b6+9CX9pcVGUACIm4EILBMkb4O5XMPw8mT2sxj/Bfvu30VP13YKjPpOwwN+otKKk71cGtlsWsrfrRa50v8i9wSc4xHiYWiX91387U7dO4IdHJErV2A7ZTHEfBaUpUHoMdUsZvTxH41U31No/gYkT4bk+r7JiuKRAmTWO1PQaRW5Eg32cGrJRmwv56fbLTC4GMNsU8maDE5+c3sjVuscREOmgBFMkzNZYARrnn0CovkRQppqDATlc7dedsLq9uKis9xWznv7GXYx1BAd9CO4NfXG1hErPri+GIxJd3tl8d6JvdOSM5RfGnDgDIy/DkaFw/W2mGC8x2uZTsaJUWyoQ0r9iZOQPvOxhoL/8CaKK3yFE3Umi/QOuR9fQJed3JtTvx7kl9oHxbpvr5J/k734jalf9Ljj7KCSvhfA5mJy1XBnYn1uJNwnp/DgU7ZMcwi0V/HLvEj9feJrJG8Poe6ovB8u6UuSzQqLetZ7vK6vhYEsTqYGvskl2HEvk06jVDhgsThwkgzeqJpAWKCnfewiC/n9DzXUGh/1CbPBP/O3ej3xPKeKxTOkLbolUyALZubU/Qw5GkDIwhSqPKkSTDpryIWohV/UuNJnBvaEP0cXv0kkzzjZP8oozRFhtzFEKmOwEclO1lI95VxSofbjuewflpio6ZF5iWvQaAus/k36Qt4XjQiDvlzby+Uuf8/2IS5wuHiLRc3f9nFr3x8hr9uCYejkpjgsfiOxrNQqZzSCzqOiinMb0xOk2NHvbiAksAqIgUuLcGXwGMrx5DeUxRqrd6/m/2Pvr6CjPdu0D/t33WCbu7i4ECe5OcSnQ0lKltIUq1P2hTt2NtrS0lAJtseLu7hAhHuIuk2SS0e+PazIzgT5773etb337e9f7XGt1kc5tl8t5HsdxtvpGMiz8WzgmnMj4D2Ze8yTWhwrqXlT2JbRK25nt9CJkQx0fODn70tTwvtdYOH6naKvoeew1CXk0EGu4u8rLnudE3zPcr47rZv5UykoY9Tdv6gazPfUJto/bxp/zr+OqtcUrdYsEtQ/3eJfhIjk7apxYMf8A6lLIDgfAVZ9pPFngjoQHxQn+zHK/H3YNgvqzEDYFJBmXEweJP/cHuS1D2JD7qmCAZIv2KjeDRWGh3r+eiwVGCvcbOWw2s6ujg/nAWGA+cNgEx6xQtPo8n7z/JrIMRU29WLj7CKXnL5BRsI5HBnxMYtI9oPbiiDES/9Px7Mw7Ts+cRwm19IYrywRYwdzBs54ZZIQ+zelBJVR5fsG1lrO2OonA0ucDnraFIj+dOJETEbMFOzj2Phj5N2S9j9tfSUQVgMKs4PuKnkhWs+ibx+7kPo7yRJABr2YvvJu8ARsDPe0Vyg2D7cbziTHr6O3jhHwPHM5vud/QXD2H00EPClnzepvsX87HqA3XGeIUzeDG9rHIbuS5N7N/9H6+X/g9ZzqjIeNjMR9kLWeWyxdMuFBvN+7bnWrH74bTD7ItxOE4lKwyGnMplP4J+vJu33IxhDmYLzHzqVTMZFjoHoZrwaxopV1dJAzKugK48DStlgjuvzaGYhNU+2xmv+J5RwG0wZRIZVwLfRWjstHW/xxOyLSq9/F38vfdBMRp3sOJlCnU+VVwwGLgibYXoecbggV3/Q+WeNzLk2nfc2BCJSaNLeSCsUXsKRrO0GZVIsH/sRFV7RoNwNzaLMZfrEZh0TrqyNTK0nBP/LzPsDp4EYdHP8btmq+g/jScWYxUf4JHa+BqWiYfPPMBHYmFeFet7qp41JIB9T8Z54JGQvhMMOlI9/6DtLhpjCo5yyNx6+DkPXD8Hrj6FjOqV/K5XaXcwRZTNR3jSvVo5m+oo87iiP3GyG0Q/xAuuuMsjxCSaMOqf2PM1Tz8tYFMtIliTJzexqWY+2h2O9O9nWzJTfYhoeI1PBS+ADR4HKJVEvGUFAoJj44UAqQkrMouZqBsl/F0AK4c52bn1wcay5jmBrJk4lD4RFr6JZB6zh3WqrFKSq6FvsbX1XfR5Hba0Q62JKPCp20g/sa+VHtv4efy5/hTkQZD1lDf9wT7Thwiss6hiGVneTsx++xSddw8N1oxcbs71MaCq6lMtFPtcVxyP+P7IDj6wCrO9T3CZl0be6XjIizCgO9xM+voE9yCZJUIqglkTuhZXDquQNa7eFmreK8BDFabuss/MNBDG+YRUbsQlcn3H8rctU6Z7c6WzYV346JsZ3zICrL0MnN/u5Nly5YRVOVg9xI5m7KhVygq2srFfnpWz1/tWA/6vAe3tVBrEe82qkVbXbQ6ncf0VTyqPMkMN2jwOMKZhKli39maj6ZyDRPjV7DAp5VXTFZuudjI2MA77I8G6E9yLdyNZN9OSiNKMdgASJgNoK/gmimaattPbhJMdc0UzhJbupF5KkuSkIxuycHsHs89DSaOTNjNL3f/gt7dNtdlLYc/PHhEv4uPRt3DvhlpXM/4lmeG3IV37sPC1gDow+eQWLkWd7Pb/9E84eETTmVrAgPO3sMHjQ7Jw660zuU4A7b9TduZZm774zbuahlNcsOdYJNY9lcZUdOPTU9vZs/4PTg7ODs6HBNjF+hDsio4L30Cvd9DlqEuqi/m2AROPrOD2MJYUXdqse8wWY0Y3fywSjKKYsH41io87DYUiyycfR2f/8SyZctYGPoZbiq3bvtVg8nhCLQ76pzAkxUV3ORUi2zPYcCE01z4YBnfLVrB+L4FTIn82/6O4Jh3OD/9Wdq92sX6aZVRyCZozoaGC9zlft4O8GlxvYhZ1SRi3zl9u8/ZZPzO2PLl5JxscjvF+ba/aVU6JBTsNgjZ4SQamLebuYat9nuU9jJZKOMEtV47aXKKH9/1DhMd9rHqonSxfV/mdT/ooYFUyw3AzesCLPtjzmbyQt/ErGizXXeM5TPlZ1h6diqVvutp6wEr/oeem5VKmRFDB1ChS6RUPwQrMjeCFiQJvsl+iFUt0Ka9xu+6Ekh+Gs4tQVF7lA0HCrnvhzFkbFiOV3VeN8eWof8VmjP/wLutP9U+W6hSHxXXtEHgPwDPS89xLAJGucLgawfxaRuAiwtQsgYqdqCxGtFIkB/yDseSh3Sb71Pcd7DEKs7slyy/0+R2FpMyBMYeoNpTnNOtnt403vYgHcEPMzv6Ifxc/exl6pLbd0gcO8uzdt/THuqMFLGJTW1g0iNJ0ByYQPK1RDLOZYj5yzUC0l7BLKsJUYBvgxJXXY2jLcu3Cvnm2Hv4NN9Iy6lcTiaNYfnUCKw1v8L0AqZGv8VL6V+yMghAxrujN6fb+hDkXkiAcatQ8bClkuYSNuZsZJ68lm+T/GlzrbY78mp7jLZL8trL5CBId/v//6Sbk92H8b+Q/sfOPpVKxZ9//klZWRnvvfceSUlJ//1D/0n/Sf9fSgqFgri4OEFptwVfl/VJJFW8TkTTHXjp+6BVOvTY4z3TGXhtD72KV9p/c96QTpZX8WDPMhp8GlBLv/O0YibqzOfxnrkdxh0iTz8Ll6xU3FvdqQqpchwufPtB81WKC4I4HzeXnsU/EFf1HH3Mj8GFZ6BsC1LyU3gVQsiGycze9jnDck7gpnIT2suyimVXHmFufSZ5oUIuyu4kAnr7rOW3WwMZ73uBTaGwOnCbcBzEPYCmo5pnfeDeVfcy+ORgPHSe3Q3atkXft3U4Q7OPM0X6ulsdSpjIuyjYAv3USu5pGwA7egvnadoLPJz5N9+OfohPnv2bzlHv8HaaBIdngNKNyMLP2dKzkXW3rWbX8MeZeXwZvzbbkEC2aj0b8AQbau+iUjWfU2VrofYYnH4YD/Lwc6mmtwbUkjgMbNXbNm0BI2zPy/jK8JFbLB7aodDjZTj7BNpLz3KvJ+yesBv3pnJcm8QBDl0BtJfiYW3FW9PiVEZbheR+AWceAaA2eAhfP7Ka32c9iF7SY3WPFzFOQidTghf6BoFwTctK5PWzWaDyQK7cgoSFH2OGc8VqiycjSVC5CwC31oN8ETWac4Yp9M6cRe/r2/B28Rbfvv4H0QqH0V3k6wYjqg3Fld/hyb2byjArAwVL6cRdeBf/wEPtgaRfTWf0wdH8JZnh2O1wagEcnkmWewYrbUXe2w6rro9BaW0EvwHQWY/UWcvQyL9o7TkCt854vhiyGLWxHAb/glly5fm9R9hSvgKlxQ13gnFJegwiZkFLLtHeV7nDMpy7KsuJr3pe9K+LL0Lu12irv+D2Pq+ibtrFPavuIaAmsFuZBDrNyrbZbWyZvkWUufoQVO6mPOwOtrSBbNUQ1nAnYQ13IMkKKP8btvdEe/V1/vQ9TpLaimyM4NtLb6G3BkHK8zBsPQOGfgJnN5B0/hKWgBcACDFtgpojdErBPHPmPY51dB0wb5bI0JgCcDdHobJ43hwrCQezL6niDY5Pd1g+BwT0wnfAEq4MSqfR6zqXPT6ztfFfcO0LZFmmJKqERp9mZLMs2LQWE+jLaKw+zuJtsZxMGs6gk4OY++dcfmoBCYtAsNUcQcJq37gZ1AZxOOjxCkURW9hbfDfzq6DV0l0uB2CW+2Wa48A7rI760HQsGtvcp/IEtyheqArl+wF3c/m+Qbh57GNpkAN93fWOA2NOcG5AMQAeliZxMe5B8L+DvzcuIrokgbzEPK5INhnAgT/AhJO8rlDx6ZOf4tFYj3rrbxjlFjRSCzScR2VsxqvJi/5n+pCZlsmp8ft4ogbwzYDYe8hUvcuSvECiPhfs0WYfDSbJDO3X8eYSM7ceZ2hZC7Weu7BiocIM57xGQvh0kr3u468nfycv8CCxp5fzS/uDMLsO0v8FI7fxVfY+mgPi6HTzQdnR2m1uVMkqFFYXweKUupx9kSLWlk8vZpZ/xkKrBzHFMSS/8yQfF07Ao/hfcP4pKP6NBskF/5II+h0swL3hul3uhrxvKO1M4RUbobPGezvV0jkhX3vsTth/C2O8HEw+0R8dGQv3iCS5/B1iqpd0Xe1270B1Jm/M3kPbsN50jOvJ3erF0Gk76DVe5MfGVKSrYk7v2SuftIsjRKykP7zxOHUP450kzSQkTAp/8O4pGOq1J3izzR33NnfUBjWbanrQ1uNHSF+GJMGqSQNYHPIkzW5nuRryPDqDDvwHQ9xC3FVVTPWuw18hgo1Pdc2Bc0+CpMCls5yR2m7FcMzLVjNKjCglYXzd3seV7aru8u8XpQksa4AtU7exasBx9nKrQF0CGFu4z3KJW23Eyt+CIUq9S7BO942BzVEMrgiwMzVAws8PUqc9QF34ctzVDSR57cVbXWqfB2RJht2DSdfd6TCyYbFfN1m0HL1+O83maJ6I2c4gF9jQCpM1ObCtB2R8iKX3hzS3xxFQ2QOV2QdZGW1zIhbDoWkcU03jpXonBnpXM6u8wDWcowO28I5JOB/VBjVKhe1w3HAWGi8zxXiBCba2lLpkPL1SYFYFD/5dwANbihgtf4PGFCCuhc+EtJfxbt/FkPC//m38nRudfTf2v649U6oalvtBkvEw1BwCSyedJX/w1OBeKHpvYtvcv3inb6FguKm8YEYx141WyoOvAtD/fF8G+VxDxmboMuqwSioO6QXKtTD4QwoCP0a+/CJBZe/goWlGYTMRdKWX5UzBGDx5P8/su8T87DFUa45iVDbgpw7BT6UFcwdH3OewutOIXmuwx+pwPbsYzj4GIRNZ0NiPSvM/tEXFTixeqfxgW1unu8FfIeDSWQwXxZpj0ARyqGkLRqUFtxP9SO5wqq9bTpPnNZSNpxJ55qNnmLl/I6PdDHjWfA+Jj9MQ/Bz3n36VnzvzaVBk37QOAVwLXcYKf3eyw56/6dDszOzrV/gn1+40szBDAG90imBONLjQ5NNKYZ/ZJEWeEnuVkw/AH+7oLFrKvZooi1LRHBjveOmw9ehTXuj2HY+uSm+64ugVzsZci1bk7cJzhJ+N4I4eb7AyqoAZLRvt93SVyRU9fQyuxLekMiD/b4Y2feP4UPJTLCjPoMMqjNbwz+oczkkhyzDuMIzYQrsmmMTcRPqfScEvMpZ3Eu6E+lPCodJahHmdK/Hpf5Gd1sQffl/wXelU4WS85TQVY04wxzYtB+cGYzxUwXZz57+NhzQQ2Gay8sJLr1NdXYXBrKWmYRiWtt6ENt7GovTH6BuUBlYLZsy4NxuZs2E2t64NwK3lGRi0ShhU9k9gdsg9jI79jQEno+hxtYejblsLkTsdRrxG9xNUue8S0twJj4LajxpVMBeM8bR5tPDXrX/xQ0VvmvsfAo8kMLXypTSR5Tl+3LfqPgacHiBq1DUMer2JLMG3njH0VySSeuQp5mbanDwFKyHzXU7XzqdeH45J0mNR6LF0GYdTX6Ap+DG25y4mqmaRo084tVW49RxLY1rIHnMYj5hy7nA/I9ibIzbAyG2M9XyfAQHnkK1iL2F39g38HhIf41+NTlLhSNS73y5AZJFzoHwbGwOvMEILSBaGRa4T8XTLNiNJMHVzFlNsx4L9PWOp09eAbx8Yf5wCw632vbZ4s+3vpkyIuZcHmrPIC30Ls8IJ+AFQd4LIhrX42P7XtSMWN0v3GPG9Lbv5deh1rvc/j8qocrx7zH4Y+jsyMqlZqUz9U0VEx2n6+fwADRfEnsK3L/N1vWi3QlsP+P5GydB/k37RupAx6n76BO9mcvBvuJp9usdCkmRy2udyrSkVE1Yu6FVckd6Auc1CDtZq5esmiUnbJ/PkZ0/yip+VkPynifTMQsLKRsMvZGV+hU+rkCJfWDoFJgs2A4N+wipruNih5XXtx0xJ/IpE35OQ9iqUbYKaI5zwm8rnzbbMOslNu54cy729xDzTYmxAryqlw9oi+naDAM08f13I4nU5iAAefRQ2boTw0P+aZemm8Cap4nXW9K7HVSGkIZWKmxnTXfGIcGL2dbHprF0AVmfJN+DW66+xJRS0slAJMjkxGJ5Nz8Cr6/037NGhezzArvm+RBMDEbMweQ+iqjWumxPGDoqy91mL3YHkPAc7OzWuGeGrJjCoIkVMq/1jUbQVMqEcvM/1ZM6fcymKzyOF5WJMxi+kf+Nu3vCDv2b/xfZhhxl77CVaAh6A2AXkeH3BS/XQYQWLrL9JGUmWIaV8Ob1KvkdriLy5zF2S+Thiyz7a820AGjsj8LcqSMwTe1aLwlaZJj3oK/Bwi8REOuVJQzF6TSBJO8xR2ScXUB3R0s0ZXcwh8cfpRXD1TSbIufTvAiV09b/Y+2me1Mmdf9Wx8O8i+7POTo/w9s24GKKIvDqb8rByGv1tNGRjM2S+Q5k5jBwb3jJAAU/7HBD7XVuqs16j0e20nfknS7KQbQYkczsbTZ3sG3Kc6lAVCzw8BVivbAvE3oeicgc9tLsZtiuTjD8M7K5NoSPgVphRBHda0Y74A4/LTcTQg5X/JgbcjekXNcwc7mqvB4B2TRFNLT/RenwBVO6hT9s0evsvxutoJZaQUNo6J4n7mzOxTi9mweYSBufup9fgfhjVxm7zaGe7E8POxtbzVjcxmLuh+oDdkW7yC6TfV5PITRSxyNMC0lg+djn3JS/Fzc2Nmsn3UbaoNwCuCk/6hvYlShqK0uyO2QzWUjGxTw18HI1SI+L6mZPx0KdhNFkwKOtoUxfS1CnO7M7hezo6uAn4+96I52k2wzUDJJe9x8vu/syO3iScbRK80esjXo/P46vnvuK7Rd8hoUBlroFtqXBmMfe7n2aoE+glSxcP066BVw88my8BcMvu4ST9qbb1AycZTxsb1iQ5pKKdY705qw85p655zIrZDlR2Vu2yM/u69tZO9aCQFEQXg5QHOeY2dvb25rDeZheMf1CoZ0ndzwDyP4AKAEiGY1Yrp/iv0yngkMlC+MCvkSUTP0rp7O8Rh17txHxuK2Wy771UtKSwuQ2uRUG0KhsyPhRx4Xu9JUKZ2EIQNGsv0GrUiXni0itoG4Q94yZFL4sRjs1HXSvmhcud0OVk1GiAmaUweBXvNX7NpzZQis4lywbE0UH2R8wLmc5rkVe5JeBB+7v9zIfg+p8gayjz/Y2vNZHsiVmGxSkkAog+JNucfUhmJKvyH2U8Y5TQGQ9BijYhw7ktDda7cv8EiS0zX6EgtohW91Zxv1skxD/I9OatPO0D5WF1XI8xU+n9l8h3wQ9w7glYp8VfLcaZWW7nu2Yo9xgn5jBbOqgHjSmQicUXWKw8Q6fJBvozNFGpq+Rg8UEuVl0E4CUfWDbgCVpdcuwx+5riMmgMTbupzOCId3tjkrsv1f/POgO7+TD+F9L/2Nm3ZcsWZsyY8b+W0f+k/7eTxWKhqqpKxJNoK4W0F/nqrGCC/JMRy0vtQ4BuHN5tA+y/OS9cEdI15g3JRWlRkHZ8EDUWL8j9XARjDhzBlsa/KAyfxN/Tttmetb186Do6ou5mb7tYsCPq7ieoeSpNChtiOXQykiQRVhbGgDMDiMixIRGxQs1RPKzXuNzYk1O2EA0RLXMZGDZQaNL/4Y1W0cQvVUlsVm5kfhXkG30EMhzwqdrCAk+oDKnEqDRSHu5kRF4j8ZnxJbxlUJm98WkbTLCU3q0OV7l/xyf9dGyZp8L/Flt54h8GtxgAchqvkpTlR1CZjg/ThCQBnilQfwZ1ZzVeMjzz4TOM3RlCpslAiuYn2N4LyaLHS1PDL5FF3HetJ3P+GMbKjWlc8hoB+SvwIJ9ZCSu4EAmTyr/mkbZqDK8axWF+xAYUdHJv1A7u8YSnQgvxMR4WDpPcLzCFz8I1X8RbMCvVoh5BsChva+VF71fZ5iQvZW/j9GXCMB1zL2uKVtLk08ytQXp8NfUYh9gcHwo1r1pGsyPpKga1gRqdO9UdKVBzBPnIDHoH78Ff1W4PbCojCxkoQOczB0luI/J6KMEl2chWsxO6T+aiuXe3ur8Rpd61ATBarTTow6gI/kSgXIeuw6L25aWROWQ/uJqqjNF4RxhEO/T/Bgb/wjmvUaw5H82yZcu4bd1tPBS9mz6lCdD/K5iWT2vwJN6oh0NGcfAI8cjH4hIJMXcjyzLFTb2o64gT+ZIBn15CemhrErOS3+ON0RORMNCivcKKgpf5O+wR6Pspm11uZXKmEve6k0SUxTKx9T5Y6yIOLtvScTPeRfz5t5i4yYVm72Z6q6/DvlFw9lECdVfwN3xBQoUIcvDtlEQirvYRhuOWXCSsNJaEELNzHA0Ro6g33M16UxWkPOWow+pKel+IY9Puvgzbvo2jXldh7whSeR/3jkRmqr4iseI1h3SHE2oOHJuRQaqHeDDjQdzVwmOgsMl4WjAiW9V4abyF06vPh3jU/sRnE/tQGRXM9w99T12QbeNU8D2ce4IJ2maO3/cbydnRLPp2kZBvcQmAmaVYLQa2aCuY/ccIemT2wKQycj3YBdWZ+dBWAqZ2IjpTaL37OL/ffYpG30Yhm+fbl1avKeTW9WeNznboth2aNZYOOHk/Mao6Pv59BLoSAyvue5xt6QKJRuRsmFHMyqZAdBawWKwCxex0GO9CCno3ejB+Ryp+dX6YFO6Q9gpE3EpjyIvk7Y8gLksYHlNc2kFSgrkD/AciKSWafJo4MHo/u4b8QqXvHyhDR8DcRryazrBdDmDyjsmM3T+W5L2j2HohBtkrAcq20M94D/l6F/QueorGVXN2+nB2G0+AayQ9LS/RYgigydLdaZvWdhmy3qOnSwFGzzPIej0+jT4M0OpE7M2s92BrIpc9Huda0MP0OPQN1D/G5pJV9jJPSpjEA5V6hlw7bH+v85jsUHjgPnMfJweeJDcxFxlwK/0UfDKgz0cM8fLn3lX3EVzegqS/KPqYVyrcWsNX1b+wyWkOetQ3G0o3CGRu1W5m+RV3G/vORlLHmmWb16zdd8JRph/pEXkCRaIXbnEy3lyEjE/g9k4Ys0fEi+11iT3j9lDq3UiHNgm04eA3AHXldv4OcXqZVabBfQ6M3S9kQ5qvcrt/LfooHUeGH0Gyypj8RoG+knB5O7vLx7PdKXyDQlIIybeB3xOpPc7PcZcZ5gL6ePCV9cII35xJ7NUl7AuVGXup1P6sLElioVZo+dPlUQ6I2OXCqCwZcE4qjMS0eNP34igmnrrGVO94KFotDls1R5gtl9D3x7tYtmwZ7x9+hmzDAkh9Dvp9hSV0CsXWQK77fy/qt8vgEzGL5uAnifW5yNJe4+mISyaweSpv9f+BcbHjIPEJqlzmMz5iM2/5iUNkkKk/6d6DUFjEgchV2cTDMXuJUcHcKrho6SsyHD4LKXAE80o2MmPnfSSec0jsYO6A5kzCvQaQfq6SqJrFFAZ+SqXSFrss9VmIvI1hp6dzqmczhYNPoQ6v4sd+0+HSyzC3CUZsZJnbnbzdCKH1dzIk55iY02QVaEOo1cdS0xZNufEaDe7HaDZVC0Z+/WliS6eyMGMJFquV/emx7O8RR2OHQ2pYvkEq8cZ1quvgn6CC533hsma6MEJdfAlT/g8EK6/hln2esPxY7iu3xYZqERKir9R9wec9a9k27QKTHzjIDxlfkVAyQoBHTC2UTbjE1U6BlAcY5mLF6hoFhkZ6eq/n8phnWOGEJ9HbYtYx4Dtq2mOx2FbmhIrX2DC0gucbf4B9Y5AkmRNDTnB6UDFTt80huSYVZfkm8axvH/terkNVQYv2Ck0Wm6fApEPSV9rNaVvbYF4lmLSxQg4zej4dJgOZRXdwftIOysLKeK40iTz/VXDLGXCPRpIkGn0aqfdvZ0Hvp3l7zDj8ih4Hi8FmCLDJVGHAu20g/QNG46cVqGARx8+ISW7DoKznvGEdm3I22cuv6Cq/M9tjew/YP4FzrvN45lgs81YPp1r5AIsqg2DcIfBOh5BJ+Cs68ZBh7bzf2D1uPVmtR8S7QieiqdzOrlBHPZ/sAFUeMPE8nH4YCn9mrdsOnvS2fb5rTKm9MWvC2HxtCV+cX0uT5BjvSlkJOZ/wpOs6Fq66jwlfRhFYdArZYNv41h6Dk/ejloSYaZeh3VlG8Z+YfRJW6KwDbTBJzftY1LuUQ2Ou0eHuT1H7WBi2XjjIt8RywezO8aSfSMn0ps8pK98PvlWUxzMBF69EbkuZh8roR8qaZEZalf/W0deVBgKj1Cp+W/0DsmTCx6USF4XwDLuWfQHrtFB3CldaiA2vpSK6gjavUJ5M6AEV2wTjv+4Yv5dvYte1BRwck8vpAacdZd6ajHJTOD849Xm7U2NXf9gzhCK3HkxXHuKDpz8mu+cVZMDslSHiwI3cQrYcheZUHwCqg6q7yRUGqi/xYMg1RlvnkB38MWcVtvWx/hQUr6ZWc4CiwM+p157s3hYh42mJ/wKfoq9JKn/rH9vnFusLLPODe+vCiSkPo4+mVJzT3KJAG8wg97foGbQfsNJbA8OvRkP5NuHM6/cFK3QOC+qNjhwsRjxkMyogM2IJo5PeQmGqg8MzGWC+x46k75YvlScEDMZXeY1Xo84R0WV/68pzyVo4Oocw5Y3ghi7LlYEmt/4UGoXk35irBUxs3Nzt3knmTxihhf57xzDt5HC+cHsH/gqEzhpwjaS3sYAoJbRr2zGkLmZmyIOCqTj+GCQ84ph7+8JRi+V/ZEQ92NnJ5FtuZ2TUGuYlLEUpG7o7Y5Ru7Gz+mfHh21nkBWMrDRw02iStbzkBM0uQkGl3bafdtZ1zDf60+4wmNl7B8nEjmR/9B9G1jxDYLMIGXOwIFXNI1V449yTt0ffwYZOeFtdLFDX2Zk/xIohbALe1wOidZHsNZUNrVzs6nGaGtI+4XDOGsIwxLKvwY1+vSA62fwXtFRA+nYakVexvEiKFXQbtrmeVypvXqZvkpm3Neq5pN+1mMSadlT1yQ95gR+crNHU02Z53jAvlDTYt5zHjnDoQIKBLBtiYKGJDRbhdwN3moLPa9jHdGK+YKQj6kAtuyzHL7Y7rViuSRY9S7uyugCDJvP8+LF7kDDq6OURF1zcskpFTHfBYLfi27obfFRD3IFLv5Tw2sBircgKXRx0mR5IIs24RUtbAocA7eKfYnbTMNMJbPdk39B38Kt+A4DHUeDzK2Is1PN2pQ2n2+scwKF3pRmcKgMKJgR7YPIkhAZM4WDKHA0V34amqpTLJxIk5G/lxwY/UB9iAlXXHYUscLvlv4q5uILxqBBm5yxnkMVdcb7wIunyOd94ITrKlwp+hrZj7zffwVoOjHcUfMpJCjc7gR0tngP0caF9qsj7ArXU/j27PxHLNjYwr8xhbYGNbK91hwHecMvW1f6q6yxejy7X/tsH0EMdSHCuILEnQ8y2YcBIpWMRZzDiXweJvH2epJBis+A+CQT/BzHK2Xl+JyiBRFVjNE5l3og97qFvxlAY9GfTlkGT+n80TJrh/gi8aRTvT/YoZoHahzP9ntl9fgHvxT6DLo5d8meXRbTT3rsQweCTZzaMo834FwqYhuUfR0CkcuWaL6J9Wp7OJ0eDoCyKuGZgtKpqlPuypymbcPk8izr6C61cf4jU0hCafJmSrihifGJ4f9jyzYu5GU5lLyLpPcckWdiy17MKvs37lUe1RvNv7Y7GA5tEHyO97m90JrlVpeaAjm5GZV1HhSn7wcg70jOPDU0LZyVnG02RyAAe6fvfuu5zXdU/Rc+0LjNsezAfr/+Lj42vF3kyGz/Pm8V6ZC9oGLZJFEnsdle1snPI0ixvuZJXOUdf20C3nlxBz5SlGFx/i8uBelN4TSHDjLMKUPez9rMuZZ7be7JTrHgfT3M2W6XDsm/9xLHbNjc7Ovq5nlg5eisoG1jXLbZiUzY77PJMg813WuBd06z/OsQO7Aa5yIEaSmAb/tg+eAqYB8Uol585u4YkBCylNuAraQrvkryzJ0FlHD9dfSHVtYWjJZvRWMHb1r8g5YLWwsM8Ssoc/yLsvf8PWMbdT3lYkGHCZbxOQNYOwgO0YlHVdtSD+ufaZYO8BKcWQbxQS3ctGTsL93BRQuoHalzOaNE7ou8prY6AbmuDCM5R1DuX9Y+uwWhyAUS/zWcj7CpWlFrPchk4uRS/XUuexj5O1e2g3tjva0bYfialewsIKIytnOAgnGjwYmn0cP90MdrRBqdnTVmYxz412haaQKn67ay25SbmOOV/tzUGPMexsh+8f/JKV975JtfdW0U96vSPU2cwdPB6TRKTXVaxYWNYAJb63gzaMN85nM3H3H3zb7JCHH+K9nHDPbErVCyDpCbbmbmX0qtF8dkoA2mecH0Db5/ei6jDa58x/srXf6Oz775x5/686+7r5MP4X0v/Y2fef9J/0v5msVitVVVVCg734V9gzDD+fo+hcsrkS8Ap5we/QbHAwcuwLLAqGZZ1hdv1pPLpktYBCMgC4PG0He0Yf4K/OoSBrhNOtdCN11lzaNNdIzkkkuDLYsagr1OgzPuM324LftfgWqpaAVw8h19NaxNgOYfgqSh+MLJmQZSvsHU6y8TX7JtSjvQfDKtezuP9icAkCv/4UtI/nxZLenO6EjXVRfNvxG4w7ALuHYNaGM60gjl/v3cLbr7yNWWG5aeLUSjcjg7pSrqGTiwG7KIzYyE8Fd7IjsE3EHsv9AjLf5aDnZWZtmsWdv9/J60UBrK/4Q2hB7xpAS8AY+uarUJqVqA1qBnheJ0a1GUyCQaNWdDDRs5ngEGFIdG135azfNBiwgmp5IucrJ7M2+220dbcLJFLZZjj7JHTUoZQ7WZq0jgwNJBVDqyEfDk2DmWXIA79nxNUyQtu+Y9XdK9k2VGyckZWgdENS3ChDZqsQ3wwRW6hoFa7nn+Cltx8lrcGfUWG7kZsvw+FboWwzsizj2+BLVEkU6WoDQ4N/BPdorIN+oUfgISoS9CR3xaCQJLE5v8OCV/3PlMdaafRpxCLJqNsaHW0xt4VfjGKz7tWWwZCcI0zSvn5DPsUmqiHwR8p7pZHf0SgkL6Nuoy32IfrWGNgeWkRZ6AheuVwAU7OEBEvM3QS7BzMwaxQAqTUx3Bd1gBrPh0HpAR5x1GV8zb8aoM5s5Vroqww5cz8V8Z/bx0WH124Oae4mO+x5dsqP8cbJb4Xh2H8wB0oW8MWp7zGbFehcMvml6B0+zT8J3j3QqSMI3DuC5OxQrox9gTT/KWCxodSarzLH9Tdq0/ZRHdyIpkMjYvbZkipxMUOUjxHcNJNK742cbY7GrPQTm7O0l2lNe42ZdUYq/Gvx8GhmYZ+lBMnHhTRg1ntwejF3zHoDgPCycEwWNVaFK2R8SouUyuK0T5iqHkJo4+03SXdIElR5b+awyzPUeO5gruvXrJi2Ak+N2GjJdMl4Guz3M3AFpDxNu/88fvj7dYJLKmztZpsHMj6F3sv52q+AiW6QH19icxI5BmRLwBiWl7oSly/inSiNKlTNts3duCPg04uhmlwiB+2m2deFkIY5hEg9RZ5kkbOuZI/viQkKf6bd6kLemVS8LwfRqa6gQ+Ek8GK1okJi8vZJDPrqBCG5h244PIj3VoVU0+nqS2rZSnqFjodeb0LoRCraizk/LIoGfwMvvf0St7UbwGqC2qPQeImvPOuZ1OFFdkoe5/ueR7LKQiIDMLqE0BhRzrmMcwAE1gbi2+Arvlm1BzX1ZKuNvPfCe2RNLeF2Tyufh2+H8BnkKZcSoq0gQeWQ23zOB6bU/QYtOdxRdivqy8kUJJXz0TMf8UvkVTg8HdyiIXAEXp5r6XdMeN0yU47TYJOjsY85Wx30z9vCxKbN+LRdgx19oXQD+8MWcWcVnB7Wzp2zfXgvooC6YaXCMWZs5oPOLQRPcOdC7wv8OWkOJmWzmINcAjDh3u079/tcg9DJMLsWvNJId2vodt35oGa0GGjV5KHXFOPaGYO7ObrbvXq86NSrOa67xsa2Jj4zHhIsXIUaOmpYG3GIGWmdnBp4iu0GE/kZ+yH2HhizG/2wTSyqdX5bd8Q6UXew1fUCm6Z0cr1HFkcHfo3b1YVw7TP8pZN8nvlkN3k/53wXdUzgoYJeXOqEX1pgZ0cKBA6H43cgteax4twXdscG2BDrNtnmYcbt3dvlhm3oK8zlj+Y4pm0axeictQxvGgYn7oajcyH/O+5jDE2lwjBYevEBDup/BI94SHyE1sFrSMzNokNT6ni70+vLWpJYXKVkf2c7fq3DmRv/AD0Ce0CPl6lyf4gRYbt4zgfAwoz2v/l5+AmiFGp+nRVIb6+fmXzsVT5qhJCG2zD6H4YpV0GSkMo3sWJaIlqvNtSdOmqUT2K0GMArGW6tRefzAP6ymjrPfWRFLqVYucORKbcIAPxD67k8fQeJyaX4a2og52Nb5SmpVfhRawa3zljcDLGiHU16aLlGiEcB3p7n2Gi5i+PJw7jYvk3EOLTJQD1c6ckKf3faNUW0uxQ6sR4h2aM/g67tJ6L2gX9sCwklskXDnnaILYISzWDobID8bzF6JJJ0TU38rnHc9es97C8IhTssEDEHCn8myyWJPe1QGx7JH83f0GnSou24LOSrLv+LiF2pFMbA4rbHAPgyqBOpvYQa3/mU6/uyvSqD0x2OvOQgnGK4hBDoWkxvt67xbTs8Rc6F0MmkdxzhLT/wq/ciOSealw2zHC/ZFMFWv2OEKKA48AsOp/Vke4stnmjCo5j7fomHpAGrTFPNdI6WzEfWBAl2y5DVKJuvsioY/EedoddLPzAsohxkNewdAaWbGFC3iR/6VONX50rRpShOlM6kLmUD7BpIUOmjLIz7i4EuYJYM9Cr5npUj9tMnRDhnhKFA1H+HuoKfW+dx98a77VnXWHwZc7nI5kS3jWWvdPBMQiHJVAdVUxhTSEw+zHbZD9vTIXwGpL3MKs9tvJS5gEc/m4G2fj/XOzLt71V1VDHI05uelR/ZfzOBiHuWvwJai6i3atHZz6g2Y3jaS9T0Psml6nGcqLidLJdf7c8rJAVcfZMYRTk1GYIZFJm5E0WnDRVhMYChAY1tPkot/ZhxFyuZHPBo93fckGSscORW2DWA+M5rPJhci9bojfX0ft4qflb0Aa80CJ3MG9IgVrVauZp2lZj4ciJdywXrT1+Jb8mv/JY2kN7Fq7jqeooFmG761j+le/Qd7PhzDWEeuayaGcqgPpMo8V9BtsGKPnw2uAQQZr3M/pR2DIvXcXX0NK52zAT/IcIReYeZFe0XuNi8nFH7E1Eb1A6md/LTANzupPQjIUOrTfYrcIR9Du5zvg+37Z3Ayf6/oC3+SLC2EEbm/Ph8jg8+bmNTSEjNV2HfGIxWD+7dVEZWWxxnIuBE3GHx3n5fwdC1vJN+D4FJT2KRO23vss0FHbW41a4mwjPLjqTvdh04Lj/LqRI/3D9byMCvH+SXtmcF4rx8Gyjdea7qA95pLccid5LSBYZvdvTB7ixOCY21GqoPQEctRMxkYvUA9ulBrynhhes9aIr/AjwSsUgu9PA7R28nYqBClkV9mDsJVp3iodBMEmyyyXbWQtRtEDyBKW43npVsbaHLx7Mjm6FXrzA0Wzg/bzxrbVW+yAf53lT5t5HS0w1vuUWcOcyd0FnHvbotjLRqcdW70qquI6d1Evj0hoAh0FbMV9FpBOnTwQNcB/ZmilLFduAtoD+QaPv3LWA7ME0Jy+dZmaaazvqsl7g35x7Ohz2DWdZ3aw8JiSDXSvxsVXqo4Vko3SQca5dfIztSg6T1wbvZm5adJ+kYuJ+lr8RgSHgJN00bkV5XURsD8NCnMtmtGa68IZiQRb9gVftSZoIgpZXPfV7iw7wHBPPAVkFdeVAbA+hXsMkBtIt/FJNFxbdJB0izx/STxP7t8qsoOwpJLfuIcUUnuej7CodSe6EzNtnrWvnfxE+0SiZ0Lln8Vf3eTffIspBrO2B+m8aOrg2NQ8bzxnffaBbbE/0yPUtsspZdzi0kyPiEXwrXUW0UMgb/FKNJliE74lnOer+IyQYMmN+4FrYmE3jAldsnuZEZsQSvtgw82/qgkGVSUmDW+GC2DK1nwqVa7PJ3/yTj6cSC7HRJFjLFaS8gh04i1DWKtPNteBVG8WF9IP6chCv/gi3xmGUtp1tU9LzSk5CyBLJaIjG6CLWu0LZv+HDEbXhJeiSkm8BxsiziMRsUjbgaonja/Ty77tplz4dK0qI0eSFbXehXsIHPB24nxqOI0TGrKdb15bsWK5d6XaI0UqxjALhGQsgtaAo/JCN6Lr32fUqvvR87xlzu11B7hDdNaaiMUdyUZtfBsD+YGLuY6NLl9vaXJEBfiaLxOJGemTT0HMg7aomt/ST216wVzw74hvJo8XdJz2m0+cSg0dv2SEot+A1gjPoYI7XgqxtJ/ytX2KxfAUPWOPUY0YcCmyaTXryCANcgIYep8UMq20RtmIrpf0/HvbmOLWUfC2Bc/68AOFZXxJ68z/DQKWnukc1naWvR1O8UqjhX3uTSpgyq43aBO2iGujBRVv3XzhaVguWvPoTPmNW4qZv4PHkHn3o8RFTNItbp4KOgpyBuAfnWRHyDGim94wg5CdnkqzIp93sTJAWskZif/hoArw36gLKlZdwR+7j9O2aTY8wYzaLftxl9kCQrPTuyaDfr0Gt1SFYrRqttXFgdEtyybEVvkUQrtXXJHdrYaLZXWyxgCQ8iP7GKzNbDwvaHYx42mx39X2WzAzk7+ywWblL5QV/N7JOZpF5xwa8yn+YaD8paUpBl0a93VQ6lY+sYHvvoMQZfmIVne28ktac4G0fO5bo1kDqnJSPYpQXyvoPgcUh93ieoYwQGv5kkpC6mX8EGhrje180WCXQDNzoDkvsW/MkCn184ljyYrzWOPm63PWLBamMmy06HGoXd2dd502+eGk++CfLAEA/Byn9w2odNZ4/JB+ekkG6eZwDcrsIyq5UXgRHALcBvwF5gNTDB9vuLwKsmE1ePfEZO3WDWtCgxAwnVz/PB+A+I9IoEn158VtlG/7CtPJH4Pb2vQ6bBDQ5OhbbrWMu3MjzuM7b2VdgderIkgdpbyIMDnelTyA5/TtRtl6PQKw1i7qHllgt2Ju7RPv70Dd2Jomq7CPmhr+QPz8lOMYRte2mXQJhwgm2Nqznf0IM6g5AQxypRrRWS+EG6b+3rigUzZ+Jnsvj4BCp1wmEtnLYOBY4bk0JS4tM2GI+6J3jy0mISXGJFmdPfoGRyMW4FMOXXuUzaPgEVoDQ1Qc1RODiF8OhpeJm+I7BpcleubWeAVBi9A/wE2EDqPdweP1Yhy6BQU9eZQLPB15YL0Q/8VLloVW1EGFZCR/VNYI7CWg8qSoKIzKumft1erFYr0Xt/vClm338n4/n/qnPvxtTNh/G/kP7j7PtP+r8vBY0Dv0GEJy7hUI9UrgW9zbXwl2lxcvY1G+sp8f+OMr9f8G7vR5C5fzfJqgKpP0/vTSHoQjr1fvUcMGbAvA6o2gdHbqWHx2gUfi+Tkp2CW5ubA/F54XlcCh1IjRrvbTS5nhMTWsgtwshb/jc/35LN/rffYM20MdwzyYtB110h42PKVPeyctgjZEfR7eBMxK0wZg81nWl22YWY6qX0drNJK1gMuHlE0ys7nxGX9+Nb74vCpHRMpHdaeU77MZVmaNVmUhj0Ca5uL0LOZ/ZPTDevpybQlYnHfSlXT+Fy82GBlMn7GupPk2/R0tb3EgAnrkWT0z5HMMp6v4dL4FCm+glDTItvI7/1Xo2f4gpMOIGs0lLbHknQlQG8FpzHN4tWIE85QKT+GsQ/iKyQuFIzmg3XXqK1a8GpOSyYlOY2TLhx98k3eK0eco1Qp78s0NkugcjGFnp6lNN/3wHG70pHp8kWedZXQ/l2ehmuEqmEkIa5jL10nfEuLznqdPQu6n0GM8HvGgDHdg1k6ZGfUZT8CGUbQZfPndJVQtPz+XPOn+zvc5X58QuhswEp9m6Ol83h6wY1DbbNnSzJwvn0uwyyms2tCpq8m7geWcjB9P60Gmy7h8o9jFQeAkBtCsS3dRjBipRuXbhrYR3lCudTswhq3SxkBKwWJI0f19qUSM0eyCbDTYvlXebLLJ0lkzP4PtIfMtFq8KbM/x3Ifh9yPkPbeJZAhdjo5oW+RV7oW3RYHMij38bczidRqyn3W8NZ+Sv6Xf8c/IfChONk1o3m+xYLW4JGUxgsjH9qSQKLCSVGCnpkcmRkPlZJgdmqgt7vwYQTMHwjkyvD2T7gBNHFAdy6eTGd8i2CvTlbyB0O9XqLTlUV5+JvZXbbCWoS1gmnWcgtyJJMXUAde8bvQaHZy/iwFUxUDBVSKXnfklt9jg63ffzwwA8UTNvJvJiNeFvOQ/KTdMgR3JryIT4K0c43yXTKUOexj4uuH9Hgfowbk4s5EF/dCLz0fSjx/47XzjyIblM8rJE4ZVBw4lQ1PU9mMvToUNxabfqEXskQcx/PNMZTs2U0DX4N7B2/VyDHja1Q9CsmbThvm1pZd9tW1t22gfXzfueFnI1wWysEDgOXIFLLh/PFsQTG7h3LgJxf6KO4Gw5NJykrmHmp71EWA+FKEecSwKTyg7k6VunG8/N9P3NhRAHTtkzDQ2dzONUcgb8TKE++QHCAYCGqmnewq/lTe3lVttNUXkIBucOewss0nUC3IPv1ptyFzJ4xnZykTGqDlXxctRAmnhUszB29GWu2MHD5UkYe7GJNS6hMVVC6gfrw25narOfv6X+TF59Ha0wJK2eeR649Cr3eJU/zL/aFurHS8gFJJT3Y2aDmh4YB4JVGtXIqrw16jF1htkOcZCbPifTlprAwfct0+h8VEkBv1cUJQ2jFNjA0836gjgbfBkqiqslLzOt2IOrq+wBBzdOI6Zwu+rWhEcwGZEkmoCaApJwo4l3ySPU7i1Vrk+3S+EPayzQkDWbzzM0Y1UaxHpjaoSmTx4PmcTDM8Z1Z1yfCgBXif3q/z9vX+3fLhzN7pbglj4PpiVyImc+YK4XMqLjY7d4/jLOY9WdfxvxRxfTVxZxr3SIOl4dnQvNVwpUKEi4u5pW3X6H/mf4OGSjAGjKen1ucyo+EZ+cxEZ+zrRRU7rQTzoi/PRm6bwxVnR4oa3eCawT50uNOUlIiKYzNwjm6OZZWcyibG8IpMsG91bBbnyLY2CnPYh34E9vzHwEnp5IAYciQ+gIyVmKU3HDNkTqkYGLSbAZuve3ANGQN9P8WkpbQILnx9aJvODTJn3a/EDwUZQKVSffDqcrkjcrsLdr99MNE5I6hQR/G980qsgwOhKNz+vzC2ySXdJd4tFgVlLWkoDOF0VI/hZSaJXwS3EyAaad4aGc/uLKM9ZkvUh+ZQG5CLsUha53kuaz0yAvlkX6L7XXiiA21HXQC6Z3ZNJ0cXTg72uDzvNdh0kUo3QgN5/C2tuMlO/IlywiZxa3JfDMpnl+n9MOiuOB4d/nfABRHbmBfmxqT5KBoOteRh9IXf91oIurvI6b6SZLkyd3qI9k6i8nnO+idfZSArO2kuwYJRPuM6+jiHqHdtZ0z/YR6gb/OHaxmIel28n4OuY9gy5UIJm/wpD7HjTl/tHM1rR2m5cOAb+3fqK8bj6sEPVxE2Wr876eiI4Pnry5ghVP/zZIDRcw+v/4s6rOQ71KE09gq2dqx76fQ4xUSO89xR0MoVsnCxXFPowpWiVinA4USRIiig0BZ6WiLrmOQSYfq6CxWh97L1HNm+hdspk/RaiGzXvQb7J8ALv5MK5Vx+/x+fHaNpJ9bE35t68CshyOz8DTU0VMNOybu4NOYCp47/zIG7+Fg1qPuzGNR4m8McXGgsJ37n5iuulDcNkk4Z8aEpMDVEI3WGE52+HMsPjSLMzFPQ78vSDKf47k4M5XhFSTlJiG3ayBsmoj35hpKmftT6Fqmou3QkJib6Oh/f3ijQMLz9kaSGp4CwF2Cn4MQLKTZdZD2Iks7J9ilw73bHSyHrtdsi06hRnXG/rtSVsLYAyxrX8QqXxEL+dCII2yKmipuCBoNEbP5IPgisaZIeyw2V6UDOPFPTBLZaQ/vNnAtc9a34VuXgEdjI6fMn0LRrxAwFEZtQ5Zk4gqEgoFXjzxajB4iRuvmGDi/BMW15YCVNoWe4Ju+9M8pCGjRNaMz+LGyMpntiuNciX6YgXuf4WftWPCIp0FO5IkaOGFqZkfvEF5vbICcDwWQEGiihILoUxwacYhmr2ZHOXu/C7d3Mrp4Lp5tfbpqWMhFA4ROwbvhJD9o3ZixZQaJR4dQ3JyMNn8Z7B8L5Vt53/QT343KZ/ctu2lzbxMGIUsbNF5CKemp14fSaZXo5wLBKptxUFaCwhWjRUF3M4Stca/8C79rdzMy4VNaXXIcV506b6FyGtNLRV/V+UQQ79IPTK1waCps78nxziIu+ghn8O86ONmrTLCxN8fAGokjwc12PoFklfHS7xeyzLWCgXpv8Md4tw4CoLg9iPbwx2DaNS6qV/D6kAf4K2Coo49IsmCirXPBKqlYsLkES/3TthLZ6to7HTprecjbBqIy+uPaEYdG4ZAsr+97Gj9DPO6dSbbydu8L15Tj+LrQm6jCEE6VzmC3thimFwrJ/aNzWes5nQ2pl/hxwY8khTWQ7L5DWMTMBihZz8Cm74hvEfFCXXum0bNnDNOBfcAS4Bvbv3uB6UD4wGSWLn2MSvXtVOgS+dO6j6LAbwnTT+S5Ic8J8FpHLaO8nuKTi6/yrwZ40w8e8ciEI7NEmIDa43hZoggxvkVFwgiiw3PwODkUue4Q6qiJSNfXEh/1LV7t/RibfZ43QtcLB1H8gxA+C/ec9zgYDpPdoNntHE+kLoP1HqK+m64wo+Ir1gSqCWyezEhXA/Ojb4fmHCQJLlRO4JHckWQZnNopah4AniWv89u4aXjr+6JXVqBzveyQ3MTZ4C3OeuGmUd3aooNGDvVI40rrfvtvyn+QyOsb3A9f3QhUZs9/z+yjO7MvJO5ODIVvE1m3ELCQoIJRJXdD6Z/ktExFsjkx3PUp3F9Tx6VFlxx9EYVdraHLid6s8AGvFM6re3G4w4xs0TA8+xwjss/bnRayJOOl9kVl9sYq3bBO4VgXLJKRUVpYHwxGdSQMWyekOrM/IiNXjW6gitMDTnOizYM9ynNC3lvlgVVW0uzVjElhoUf9WFI9r2NReMDZJ4lvfJRo78tIdqbQzU6CY8lD2d3Hl0b3k4Qr+tgBKwBD5aVMvNhE+vUv7b+dqxR2DLNVxSO1oCqK4LYtXzPu+s/iBs8EGPQz+zQDOaAU+5h2d286rTaaaNwCGPwLBx44ze0VxYzIFAASd2w0aJU7SEoGumnp2T7KXvuSBJRuxO3YUN4dN4JjaaftedKbbe/27UuTIpTo0Wok98UEFx5HZdQ7OkBHLYNU50hUgcYYgGdHD+p4EEIndm9nILThTqLqHsRT7QXbUkQsVY0/ZzqgzbuJsvie5GvmCWDc9T/gxL3Ul24jK/RH1t6+Fim+hFsCMvHOng9H58CV1+jVfoHcST/S6NPIc0dfYEqPDEYoFNzi4tLN2TJSdmG4LPHiXDNLn1iEVROIrtOXd4/8yZmGB7BKFvRWaFd6g8KFFdKLSHlQvyOMhC+OMTf+fvoW+YBFIKzmprzJrt5+bM77kzDPMLSyp7285885xowdPGaV6ZAjMbgImYDDI7MxhkRwuM+v9LjSA9mqRm/Uc7biLJmNZ2n1CMSs0uBxVYAhlTY5SWclHn1hFg0s4sXckY66tl03m29mlnb928XsC6+7j77Ke0UYHYDWAtL6niJ4jsShqf0pTy1GoaqxM/uwyhTGirPH6P3DySj63e58xGLCU+5E67QOxLjWwJlFYp8V98BNLKcueVBwzEGyxYVbLjRy8Y56R2w9GXzaBhKjHIxVNtJJk/0bIe4hxFY9S1Tton9k9nXJeJr+gdkH8IBHHSoJvN26JGxtGTr7OGS+xa8Gx9lf2xmJAieQptN3rJ0QDCwFShCOvU+BR4DPgJG235ci9knt7c3sLHiYBVWudFohqv4BnhnyDCEeISDJmCVXXFXNuCkFACxS2QnV++DIHLB0El0Mg04OYMknT+BX5yfO87IS4hbyiccSNrU62t/O2A+dBINXoWi6TFE0THAFWdVMsS4SS+orsGc4nHuy+96yS6lCoQH/QbRaoznQI4nzuh222pIxyX4QMYd2l/7d5FZvtDUJwJ5YD6yS6d86voLaxpB+/Wv6aT3F2mnpsOcp8noE/c/2ZWFTKInHegn7hi6XePcAhmgewrd1WPd8W0wC3DR6B+NKXfnLxl7fEALxdT+C1cLrGdGcmDqWBZ7Q5pLH3zHxTK66zBenvqdRMQhMrTfJ5t8xYB8vPfs+vrUttOw/iyRJSBYTkqU7MO6/Y/b9R8bz/z/Sf5x9/0n/9yV/YWS+L7Ck28/ORtT6zgquRC+yIz9uTGoMtFf5kZCZRpJRyxLtBhFMftCPkPwUd3tVEFUZQotnC9VhKlQKGyop/1u0l57hFZvP6mz8TK6FvUykaSXkfAQqDwiewH01MnVlQXg1exGt0SNbO8GnD3XqyVyq78WBdrBInUhyd0pvlNsJvks4xq3usH/Ya9wiDYbsj4WxvedbBLheJ/HCGZ744glCqoK6PTsl8g6GZZ0hqGkmWRFP0dfzB7j4nB1yMch8hL1rx5N8bAiTzwxioukzOD4fBq6EwatY3J7KT4OOsG3yNur9mxjq856Qd0x9jkCFlWXRqfx1Zz37xu7lii6EOouNPdeVrDImlYnqoEqej2plcO2fgJjcrVgo9lhDme9v/OUyldtzsqmfcAm0YSApyGpKpNwEQQr4yThOOIhaC+HMYt4fOxBdYDCaTg3vadxgvbtAnB+awu26P7jFFZRmT7TGCNyVNpTSwamw3h2/xhMMmXqMi0OPcy3pGsND9iGXrodZlZD8FHO5StKusUzaMYmf6vyp7YgXspbAhapJLKl2o8Lm7JMkyR6Uu9MllcVVLhTGFrJz4k4afMts8iwWOPsYM5TrRJXYnDQ3x98RC2uBAd5pgOj2HfCHJ/yuwLXsT3oXxvHUp0vI2PkuSaqpQipzjQRb4qD+DMnh+2n1i+KnKx9yx1+NWJS+kPU+nF9C0Im5VMfCHA8jagnu9AC3diHfIsuwvXIIW9ocCDODVbZLxUoS6NXXqXU5RptGyJT8JR+EQ1OZ3PARJ27JpzCuHEXLNmI9foCLz4t4WhEzudjpRitWipPCUbksYJD7vK6Kw2Ix0EwhDe4CTS4hgcYPgsdB4wXU9ad4xAsGFMXjddCVOWcdxhtG72RP0L0MznGhU9PJ47FNLEn7ml7NMwGoUU7i3r+vskvnSb37EcLq7+L5IS+SGpBqL1OXtIJVMtJOLQ36BjvCxk8/iCHXDpFe8g11nvv5s+gHPNqFtIWxs4K9YzdTElnC+L3jSWrXQO0Jgd5W+7C9I5D4EwNIzrbJokqSiKl24h68q3diVBvJTygiIS+OqZtn0K4KhOYsyPoA9JXIksz8NfNJyKxDYeoUGyPvnujdh9Fm8KHECBark/FXoQKVO4O17fwS7kZCky99z/dF02nboFfvh9YCsju0XBh2nJ+eOMWP939KXscJe3U6y4LYfzNUwL6xkP89SdZibtPKVIcHUNnrdTTy6+DbV4AZUl8kQxfB8cHHybiQzoxNMwAJle4sHJmNZ9M521wAnZpOzK563K0+ov+r3NErYrEaXahbZSb+E1eqKvwpM/rCuSfxs5xge+F8Pm8SeXPtjKXEox/r0n+HKdmEKNvZM2szxTEVpF9OZ01zhED5xS2Enq/zr1pvdk3cxfp5+21t4djeZNVmscNnKpeiFjrGY+BwmFEI0fOYpWlm2fX7uWVbHD989RSLt+Wg0BfCn95w4Vno9RZF9Rn41/rjoncR7dx0Fbb3IEpziTaDAx1ZbvKA1gIRk8NqpKn2HlJK37dfdzaS2uMxOLWFc9JKbUTGCNRgdXQxA9VZAiRRthlyPuHz69e5ekLINc8LMJN6JBSO3QEXX0JZc4hwm0Sz7ct4dJ6Eq28KQ75Rx9HqeYSVqkk/N5jFh3/DGP0oYMUsufFk2me87ut4Wqkvh8bz0FaESmHEp3WI/dqTnodhrQrCptMQMp2L0fdyKfr+7mWWFeDTm1TTKTKc4l5cTl4l5jaTMLAc8a/k3v0NbLzbnxXTWvne4zxE3wEJD0PQKMLowNWrjYupv/PKtAHc5R4Bpx6ArSloj93Om35iYzvmcgnJ5e+Q1rEE8lfgqjtgqwXRL5rcznC4YidlLWVwaAaxupep14dQaARsjkBJggZ9KC/uO8SltoeIs4QzrG0mt0ftIrbjfbj2Jah9kILH8mjHGVYOeYI189dQE1Qj+oihEUo3UKUZxJbOekoCv+mWB8o2wKWXIONjjtc9hhULVWY4UjdNSOAduRUy3+Ft3Y987ZZKvccBCoI+sHWOEEh7mS15z/BxnRttNpuLLMmCca32odVrhv1w2pWcx0VXV/RtHUZa6af0Vy7odq9dyU/fhzRTGvFtq0UcX6sJpbGJGd5mzk/ZycqXlnPmlnMiBrBvBozchk7pjdKkJLBaRcKZ32nTFIj3ecTRdfT49PRvFLbEIQEbWwTlI774gZuYzQAjqRBO7i2xHC67h8+uC+N/fsg7PHJuCL9c+gWAo36P89SRdIYeiUHd0czGvH/B4FVw+iEY+CNH/CuIPtlOWMN8W1vYCtlyDYLGUGiaA8DkhK/4ZWYwqtYLUPQzVO1BUnuzvQ2CS6LI3zKaCRdG4du2STzf5wMOhS+m/+lAIq9HonPNwaTQYVX5wbQ8qhI2cvfBlfyqc6CwnQ/Czsy+rrXZ+QDuvIeo9zjIntJN1LQJdmMGRTwX2AQhC/n88c+ZGnMd9JVQ+idkf0iR94dcQTjaFGaFo8wBwwUyGEA24qsbTooa7vUEziwWa7TChT5+wwirn8+A3B30z98i8l22BY/Sd3mo7+PkJefg7pQ/hawAn15UW/3xahYqFxWhZRhVTlRNzySud0xmUNY5Qhtvu6mMaoWa/hU/0rP4B0ZdzWb0lTxUSrWQ2HYJApdANJUVBFboOT3vTx4Ly4UT90DuV2Bs5V5rFiNb/OmR2YMVFa7MPrUOXMMg4RGsyU9jGL4Tt44EJA048eL/y1QNeHp40dQRxL+KB7LDSea4a1y1KsJZlxvIHS8vY9myZRgNSrHPbcmF8u0cif2N1xIaGXl4JEOPDe0OdlCoucN1PX2KhIy+ZJXF2ntbGyQvxbW9iHGtguFaFTuED47+iSFkrgDiZC3H4D2SouzbePTLRxl4ciAgIfkPhDn15BruoKC/B8fcFnBfFbxQ2UN8U5cPGn/m5t7BQb1zebpDtn90+YUTySNsv3Xf0MqSTE1QDW+98jG14yZillxh/FGxPsfei5+sx0OGiNoFjL1USoDW5l51iwaEIknXUJAtWtpc+gnGoXcvaC1inOYivY09bfdaUBhroLMBWYafL77P6aal9rwoFQoxN0bPp86cQW17JJ0WmzHOeU4Z8huzy4VzeUjOUcZczSfOK1lcy19BwOlUIr2yqPM4wOGUDA54PtCtzGrJiDGigjX376AyqD96OUqscXEPQMRsrrqmc9ClhdLIUlZ2GvipbI+Yl9dpwNhMWepxmk1i36a/lMXly0UcAw4A84Gxtn8PAseAktN5fHIomgLXZUh2EWPo2/wm741/T8QMNzTQy/VbUnyu4iHBK77gKplg8K/izNReyrtXsrDqTMhmE0OStqBqOi7mCu8eHIl9mU9dvyIrYikgUd4xQOyvFC4QPd9e9pMdMN8DpoRtA/c42D8eLr5EmFsA0a0z6F38M4FuxfTyXg+t+aj3pjMxfgVb6+KodT5PpTwjzgBAaXMKFovD3dzNoO00D6aXfEe6obvMoeJGi6LTb6Ibi79/mvYbQ64dwr0j+aaYfV3JLlNsS2lVazgy4BtSWsdjlSyka8CzMx9qjxHrcRhX280SElr88Hf1d5RBdpw9uub0Pb7TYcQm/nSfyzfNgNTdBmDPhy0PUTWLGHTtAEM0D99UN1bJRJQS5nqA2lwjHMlbU7Dmf8dFayStuotM2zGJZYGVyJgg5m6YdIFBtX/yd4hEZVgrLZowVl16B6N7H5BVtKl60Of0/azjNfSqMgfAyInZ52zw/p8wOcbF/gRAvPt5Jlyo54GVD5B6vobw5n6Om7TB7HAdzQ7XRiHR/swS/qp7XVzzHyQkNS++gItSB3ZpU1nMT/Vnoe4Y0Vm9mZ3wg70dAQgYiqHHx/yR9RJv1TjAJJLT/OZiruKLYCMxqYXsmVjKxRibo1JfBacf5m/DFL5vEcCiRL9T3O+igpxPnd5l65//tI+XZKbWwQdLPuWHu27lT80UKN8KV9+Col+YXvIuq5J0hFSG0HmsL5m6UKwKdwFMGfYHfRsTudAJbW7CKfL0QAP5p35gxMsv84J3CnO1vjwWoOb4MCWpS0fg8cAS8BT922hx4XjZbCr1PbFiwUcGf1MDmPQoZAXuOndGHhyKW7sbOklCaWkSQOz+3/JlZTJGZQNtBrHIOavP7drhGI9BbkF4E4W7opMORQQVfqMA8GnUoqoUChttbm3IqChoLKD/9/1ZeECAHBqSB1PXT5ydZEnB/Zvv518tIZT7rsFsBuX3m7jz9zuRke3ttVLdiwNpydR31Dip+Ij8JPglkNayhNCGeZjN0KvkB+7U/oyXi01ivnI33gFNnHYdweE+t7Nm4q28PnIqkiQYhV/2Xc6HY3O4suQKVRkC+KY2Xhdn4wvP8rvPN7xvnYnGIvJcqI+BMXsgbCq6Th25Pp9TFPg5Zvsc59QNuoBcWESoHRdfe5kcDsybWT8RXuGklr1PfNWL9ji6zizfWZEPE1f5AhqjQ/+7y+m5v2g/o8t9mF8F2c4gCxAsuJCJmG3Owv552xh7pYRAc4b9PV33agzBuBFr3ycFAy8DZ4Bc278v234HsU9SucV2vUWU23meM7YQrDrN48c/Z0rbAZ7whiyjG9yuB7cIJIuRZgv4Nvji1eKFZJXEPsliAlMrOoWvLdzHzbGeKd+O26l7iVaBi0VLkwXuPLgaer4p/ou+g9tatrHQE/uzkoSw3Zk7mOU7hZpEJ0kbJBSSAXq+QYPnXd3asWu8O0tGBzfNILF1IQ3uR9jtO5MV51bY32SVTBQGfUxlwOssyHiCasUUQTK59Cp+F5cw2RU23P4HAPq8oejC7xHr7tRcyP+Wfq4fONWjLd8Hp8BaJRT+zKVOJVW2vjdWC96duWDU4aMRgLsJrgJ00qouoNFaxMzkj/Axn4Scj28C1/VSwzhXOD8sgeB/ibWnYNzDlPSawX/S/33pP86+/6T/K5IkSfj6OhZHBnzLgvzuUTacmRxdLD6z3E5+8HtccP3AHkgYYIJiJ+/fdoHagFb6HRzJEFUW5H4p5CR6vc2QkmFkKTswq4MYdTXLsYGfKthDo518XCDjZ7ExhvRV4JXMX9Vu3PbNw9y64VbeqNXQqhkAsgpJgs+zHueRWmh3KWBNgoJvznwjJBsuvUy4yymm+l0nVAGlBiUB0km48LRAbhyZxZsjhuFbVQ3A2PpfUdXvE0yig1PpVfMMQZ0puBqEBMCHdUkwwRE/6KByCidic7kecZ1ec/fSy7xTOK+sZlB5opAFu+rMgDNs71PIWL8XoGiVeDjvG3p2PESLt5EWt3Zmn3+Is6YPoHgNMiaUsoGRbi2MKUjkheXP896BVA6HLLC1HYyN+YnPhsxHl3QXhYptrM/bgV7tC7ISSbLiafVgtCtUxcIY5VUwNMDWZKzN2SwvGUpB9CnCy8MJb7Ot0MHjIGkJB93GckiP/cBk31z5ZoBrBHV+I5mma2PT+N0oUvMpaEnCOvagcFBJEi+rbuNg6lVyE3N5obAPb1/OE/FkNkXQK2h/N3aLhAxNAlEoS2Ykq4Khx4ay6LtFuLe6i0OjWQ/6cuqIIaz+LlQmXwoDPyPHuPuG/izem2OEl+uh0GOO0FD3Tse14STbejdg8G/GpDJyd7gDKYprOIw7wC9NhWjrjqNq2s+gqJ9JKLsFRu+EPg4prjKzBW8ZfgsGn5o19vr5LPcOljc6HEhP6gKgcieskbg16V32DfiaIAV2jfVjUigEjqJc25vVOogucKX3sYu4WXeApBRGLF0BWkki40IfJKuVhrB0QuU9woj/py8N/qOYXvsTl2JEn3g3oBP36h+h5iCcWYRLxRbed3ElqsWbA6MOcFWroI4BkPgYeCYhSzI9L/fk0a8f5Y/MMM7UZlDq9jRs70m8+TOKzWZWWCdzPvY2IusX8M7Yd+gZ1NNe5q4NmknRwsvNgfi970eHycnwiDgzWpyCZ+OdTmz7UdYOLmPT3D9Y8eAKlqblC8m2/WNhcyQyMt8u+pYWrxZGHLLJbLkEwojNKE2t7AuDyHo1vS+mY1YZSfffCbsGCCd8ayEDNM3URZZyPdYTg4u7TYP9LSpi/2Rb3mO8fvQQIxvPklr6Ca+mr6SHXwLUn2Wiu4axRgvRqT1Z/vxyGn1trNL4RTDpIlNL0rnQCR3aDjpdOrsdDpROzr5K7w1c9/+BGl0xNF4AfQUr1fN44eWF3LpGeHpcFC3QcAFiF0Dvd7BoYPctuykLq8C91R3JKmP27AND1uBubuXL9iEse30ZqVmpHEis4bm/rkOgMBLq1ANYuu0M7SWi7pu8myg2BoBnMg2K4WwvvJdPm0Teksvf4di9Z7g9fR5UbOWDoB2oe14joNqb2Rtmi/GYvBTaSyH3S3a1uuHW6sbta0bT/3T/bpvXpo4mSjTbqPPcR5nfL1zTrLbLzwD4Xv+VW4bsA8C95jx1HgfwPmlDK4dMYOGWhUiHn+axrx4jNStVMOhcQ6HHv1hbf5RvdjQwuPkTlvnC8/7nYd9o4RQ5PJMRLi7EVT+LW2ecrT86OfucZGe6+qpzSpOO8u2wYk69vwztkp95ym2NiEea8gzEP4wkgcHFk06tF8FewmhGZz1kvYv60GQKoiX6FP6MZ1sfXDujqfO6D6ZkgXdPKNvMJ9qDAFTGJOKvG4UxbTmM2Y9C6mREyGEmuTnyomyzxXbo8Spx2u2cGPQ5zzW9z19uA0hTSxB5O8gq1JdfZlfGLzR4b2dA7g4G5G53lCtwJOvdX+aIk1HZnkxt3eqgLOAvCsI+pVSf63RPK99Y9/BaVTwpl600KGx58hsAkgJF5XZe8RX8qC5DQJBpMwD1kR8Q53OOzJh27vGA3NBlLDw4iZ35O6H+NK7mHHwVcEvFZ/Qq+pmV7nFM2BWGXiUMFh7KSlbNDKdH8kuE53qRoy+Hc4/D8L9g4Pe0aC8zeUtfHvtCyFLKkgytRXB0LgVyPB9K++zF2JL4oYgf1uNV6PsZlG8lxWsrYUrRD3zVtSI22ZDfIPFxLPGPk1N2B43uJygJ/EYYB9wioNdbrMn+gJerfGnuQlYiwcAfYXY9kSUzuZp6pVs1O7M/m03VFAd8Tanfz//Y/7r+v0/ILlbOiMIqu4jYsfpKXCu28mcwZCiUtMtmtuk1QpqvbDOETeaWlt3M61tEflw+0SPPsmtmEh66bdCSB26RdPb9in1kURj8EW1WmF/uhdW3Py6GEqLcTvJa8u884AkKsxsp5tuYqLBp0gaO4mjZ3ayr6m3P59XmE/Qo+RqyP2RU3AJk63py+04kN3Ah6b2ihASs1QKeSRiVIZitKjsi3T5PlG6A6v20WBOwYkVndKNcF49V0ohnVV7gmYpFYeH4YBGzSdOhJCdsP0y6ACnPIEsybm1upGWlMbvDlx+GPkLYYRn0lUgqNwpbkqizyU8dTu3N4E1+nK88b6/rG6WeuklZShaywp8lM2IpRkUTAAnVGyB/BTUhb/PI72dJ23edqJIoljeEQNpLAsCW+yVF7ZlUe+/hu8Vb2XXLLjEHFf0GkXOxBI0h78Lb5Pl+SYPHEUqdgbvVh6C1iCc8ddzd+iCBLRMd/aJkHV5FL9HcEUiOQe7GCnNRuoCxhQxlNo/pQtCF1lAcXYwdVd5aBJKCvY2f02Lwp9RvFVcjnuCK7qD9HQpZQWzzAiLrHsC9Ixm3zniR74Er4NYqZH0pYR7XkLASpZRY5FsIUXeCNhTOP8Usax49x57hXMY5/Ov8GRewHwp+gr4fU5f0PC5f9+FgejI92gew0u46+a/T9y4avEJCOHMwhZaf/sDtS+AgjDPDyOpfoa0UWZIxqhxry70+BUIuX5cLR2ZypTOQQquFq2lXqQitEGWymOHARDi5gFD1cezStEggyaB0BVlFfcSdZBhq+WvOTmqi+nG9JQ19xnoYewD6fIS119+c2PEEAXUB9L72Eh76NPsYTtT8QXZ8G8O0sEoHf7bY5BevvgV7huAidzc02sfFgG8oH2nlQJtwTAU3zCax9vlu984yLcCaAC90PkhiyEbGKSaLfXTyUkh8gs0RP7HMFzSmYOZGbyTupFIAnsYdgDutNAwsYcSlMiafMzAobzcGTQIkPiKAAQ3n6GtaSLqHkFLf1GcNQceC4C8/Qsyb2Vu4gKzWaY5+I8ng0xOGrKbOkoGXzyGaNAdt/dZWpsv/gn2jaTb/gyESQNag9xzEOa/VZEY8SYvbBZpsyhFd6RH9FMpSDMxdP4LwrBcpaP4Usj8UKhnJ3dkDWgnuDp0Ked+KPhoxi9OdMhWe+0AHbacvsM1k/LdxIwcC241mXnj+eRobq1jU7zGaU8rxk8E55hvu8XxbW0utPphkNSQWw/KWDIi5CwZ+D+nCeeJRl0tw4XEq3beI5/wHwZ7h+OqLuG4ShkyTRc1PFafEcx3V4BqOyTONDa3CcNyzS451xCaxJiQsgnEHee/iVxQFfsFuQz1fF5yC4PFIVgMK2QhORlm7gVbpRkfgXB6/vIicgOWOJpCd90nO8+DNDqYbWQnOvzk7p4wmJ7ag7ZE+wb1JqHyF9JJveEFTTK+iVd1BGJlvE+BWhgUZKxb2t8OVCCH9vDBuErFqsZmxSKZ/ZDF0yRd2nS3shmFb+R0KAI7Ubmznw5yHuRT1IK6GKPx1owhUxtuvj4sdx61+rxDQMoFD+Y8RfewRUIXAmYehJQdL+Ez65hdQ7FmBObCe+wJq8OGiHVBlcYumU44mp5cfxX6jiPTKxLVlD2R8yJWIK5T7ruGqeiVGZT1+upGMj54onMld5bKtU22afPZ0vMPPF3+2X+uqA5PcyvYMDcN2uKEzeNKgD6ZWn8ArgxZjcnOjxS8Gk6cNTVZ/Bg5MJMlUfENNODV09QG49glaVQvtmiJHBWMVMU2z3uOa32PsapPteeyKSW9JXMqmnKd5u9ZRBvv43NGL+OJZjCyDlvWT6HVOQ5n3Zsd3VZ6Euwxj4vlWIuruI8v7V2TJBOcdAIOu81WN5y5qPHfRYdaLUA3Rd4NrBBIyIRUhZJzvw7HQgyJMicUIt7VxNnAuP7dAalYqKdkpLLj4IE1DrkLsvRA5hw5JiQVYN28dby1bxsiICwSkz+blV15h2h1ZRM/9nOZHDZjGtHLJ4xA760qF/Kgtlfh/zzrVJEoDfuAeT1hc/Sk0nieQWu5Ui0F8ZNgRvjFqMMm+YGiGhIf5qFKcfzbk/8rSnUs5VXOgW/uPLznOyQdOUrq0lKfkYnzwIqHzLYJahL2iNKKG+lsXMnDvnZg8RxGnn4dKFmPBaDHiajGgaG9F7RNFZO1DDPGbSnNHMy3WKkyKFiwW6Hh4GutuW9ctZme9lEOb9hodRoPD2WcbzD0CezC45RNiap74Z6ZRxK0srNPwd/hE/OpcWV3twsmqUXZgn8GiptMKtb1qaQ4SCjKytV2cjS1GipUPUV+xkFD9eDTGIDrNgcIWlfcdLjv7kB30JFnhT7PJtJitfRVsb30bZVe82C4H+Y22KtvfZb6/sa/9U3sNO1+7MTnbOu+Ne56U8ndx06eQUvohfWrft58xL1Re4GB7I2t00KVMaz9/JiwC/4E8oS6zvenmfHWNEaXFE9+Q+/hRdkJp/hfpF62WviPuYnT0r3wb1I6bBM2uZzlVdkrEt2u8yJ1+AxkVthsJ+CwAZrnb9vfD/4Khv9NTJbFz4k4+fPpD6gLqRL7bimFjKFPa/xYhi+zO9a5D499waAoARxueZ1+TUOcZEXwEKe9LSHkKwmeR0XGVSVKa/VlJAnQFsE5LrGY7e1udyykRrv8GtqXiYsqz2wStmG+SiZUkSKh8maH13yNb1RS7bOZKteP8ZcFMVsTTlAYvY1bSF3iabEz06+twLd/AtjBojy3hyqjHCPZ+GI3fIAGmwYqp5Rod8ll02itduRb5Dhol3nH+KT4J7LD3HK9CuBL1MShcOFrzKN9deI8a19P0yxNz2wAXC1drRlKquAvSX7/J2bfOEsRn5Sn0OF1I5QufU5vtiAnyTzKe/y4U3X9kPUW6yYfx/+P0H2fff9L/FUmWZSIjI8Uid+0LOHEvjabuC48zuqNr8jUrWskJf4ET7s9htjhQVyZFBB7qNmQPKHbV8bb+HqjYLuSvFC7UW7Sc6nuWrD4+KEydjglKG4xhbjtjy52+a5W45rJcSGS4CLbdkipBAW93bWdHq5ZrEQfhzCMkdrx6U7wBjVIjAlRnvsP1jhFEnLiHL5uhf1UD99TNEgHdC35Ar/Dkm5ZONt16kMI+tzIh/S88zowTC2DFNrwbf8PPtdx+eHgh4BrkfWWfXfdrZvJeVDYrH1jJe+4V/KH5AOY0ihgSLbk8pC7j+S+e4IEfHsBilbjWPgt6vQs7+2P27c9+7a/o1dftZU5Q/gRnFiFhwk3VxIb4LKa6W3HpdOG6TkOVu1hMJQncVC1EqsDFVo8hClC3FQvZSswcmDSF9z2D+KIJwAqVuyDlOej7OS8ajnFkwDpW3bOKB0w6IQ0ZMBj6fkJu8D1UN/Wn2fUCmeFPobN8JpxPPd8A73T86o9QfimBiTsm8qTSlWczXkFyDRfxx9rLWDp6HQlXnmHg6YEMzN0mDgcKLWjD6BW8h7XhOuJV4NXWn2BNLAxZDeOPYtSm8HlQB7mJuewbs49OTadw9klC2rLI9Qv6FP2Kd1t/siKXcNbwW7c2DyKdiNqF9v+vcJ8CU3Ng8mVUyUu4ppxN6ZwEPn7lbZ5UqQSy504rjDvE6sur2bvuNtJO7iHt9F4mpryNu/6YQOGnPEX19CqkPDjeAc0WmFkB7aH32saRGCngcDAoJIXYsKY8g8mqptWkQcJhcPxa0RfSXiDLawarNg9i6NEMdk3YhYdfi2irw9Ph73gKonIYUBRPeEElVswESkft5VPrnQYM8KSvAfeaH+HyqxA0FkPcw9xV6If3mT5orTIhbu14kQkVO0FfRUR7DumJYlPq//cEnjz5CdXuC8DcgYpmQrTVuEn/7DRxPux308t3MgZ0pS4H5+rkb2HCKVysbcSooN2zlYqwCtbrbYjL4PHgEc+Z0HzekpaRkN+HXldGEKvtK4JAh0/HqvIg2eTC3d8+jGyV8Wjx4pHAV8WHouaBeywr/S7S+4UfWTX/VbYOUHHI8o49T7IkEy6PINDcl5CmWUyPvJ9wJbCrPxIW3n3rA/SrdHRoO6DLMKINAp9edFpVxBbEMvHPVELLQx1GLsDLxYvY6qXEVD9JXugbXI5+kMyOVpjTAOn/wqLQkBufh1FlIjB3La8mhMLODDgyG4wtfNRrJmNqXmHtXUf47S4bA0HlDuHT8Wkv4PYAYRSTrTIzN47Aqzbf/m1JgoKQFWTONnFiSR5PRbTzTtAWGLUVozIY5+DgViwoKjfD2SfAtx/v1o2j0AinB11my7y17Iw8IeLJtBZC/WlAJjUrlcjSYDx0Ht02rw7wRxsXY+5ln9fdGHVFULBSzBeDV5MT+RNr7ivh/JLPiQ/5g46YZwXrOWQij+o24m9zjtcE1oh3u4ZDz2VUmQUL1WK18rAXzPfOFc6Bvl8A8GBPgYrrVf4lGQXriXJLuqkPmhQ6DqdksDvwlm7jJE46C8AVA2xtgy/b5kHQGOjzAYRORHabTvb8u/h7noY1RctpnGSFkX+L+SLuQSwKLyIa72JE9nmSKt7EovQDrxRhPPbuwTqtle1LvqQ22Yc5qe+iKvoctiYSz7fcuW8DQ0udMuOVLhwIEbMxWL2o0CXgrk/j1tDTGOXB0FYEf/rgUfgtVsQaEdgykXRzMuqWUyLW256hjHDR0+tcNXGVgnXfO2eBmNtcBKAmtmES78en8PjHI7nj7EAeMC+DtS4COHD+aY64DaH+q7uYsGcCT+zrwQ/tVkh9HqZcxTzjOknFtphjtoPYMe9MqoZc4Z2/WtmweS63LLWy9z0wHK8Gne1Ae2slOf5/8WjGs+wc+SQRTXNplcup7ajA26WGO3osI0h9gU25D3OgTaZcauaXjt4w9qCQiKw5wvbIJqLrAvGv98ejxUMYK91jYOhaitQOM64MlBr8xF7BLQqQoHo/wwI+J8KthneZwnd9bxESUNF3QtAoLBmfs79irG1gyDfPb87xfGRJrG+/y3i2/M2F9u4xJZ3HRa2hlKtRj5IVsRS9qhw9jd3urZLOcTp+GhtDFrGqeAJF2ong2w/2DEXZfp35xWr6v/Yid3z7Au9ceh76fw0x9wCQ1pHDJDdXVt+9mvq5W1HJZqKLp8LWRDh+F9aqPfzW923iIgTqvz2lFqnhDLiGo1YamBt+hDtcU+lZsoJZ5nX49/9EZCrqdgwWN2SLC0qLo2w92i5A5R4Uxlp8vapJPLeTmRsGMcBm4ETlCTkfE9C5CbVCj0NS1VYfcQtgxBY+a3iEoykDeN7/ftKbj2FyTxb7oOF/oqg5iC4OIsafpGqelYmKiZjUEVC1H5qz6aPqZH7vOJqCqplicSPQpY4O38lQ9AualqN4KTvwsviitnhjUDTQbGiwy9U5M/us/8jskygM/pCioE8x2Zx9UVWboXAVZnUwNa1RuDV34F/nz5U2dyEDpsuFMXtJrplMYtTdJGd64NPkzXTFRjhxF5y8F3I+JiH7FS4GPSXa3Aw/6AQwgX2joHgNIdUv0idkt1NegF5vUdvvLOszX2bsdQezFGzOvg3BPKVdTWduLB4Vgby4/EXcW1zFDfVn4NQDhLoI6c9az10UB31BUbsTsIl/MBRYOmFjKJx9nKqzD/P+gqlsvuMrLupUTC4aKfpfaz4U/cJjykm8XSfT93xf+pVF8XzSh4LVDMhmPV62Zu9r6ccRzP82DlJXWgIc7ehEdfI4H7XmsKGtne/qYPhROPg27FhzDIxNRFqzONKznYvvvMWZ199ljl+BMCQlPgH9v+OJisn8kRmKT1Mk/h3PkO49CLAI51jhT9zmM9wRPxhZgPnqTgkgoUJNg1ZPcWQmppineXrMQFS1O4TBx3+gfX9nkZU0hvUTRtKOGrj+FwY8ea/OhSKjQHjP8BTOM6LmQcJi5vpdJcEmYOKrG4mX6kaGkriYVPEG6dXvdqsbi+RDR7sG5Qce1K/J4de24cJA2pwFKg826npxVC8MZH1DbfFK9RX250Pcw9Aaw7rFdbKngGGsMr3OX8q9ABysj7NfijJ9j1luZ4v1QUdenea3ZM0vrJ44igD/9chmLb2wsdM0/qANQ2vvYN3jQVGxHXVbNkf9P0Hn2mVc635ua1IkYTHLVEeXkJSSzVPKpUIFoOR3AF6s+4o1PhEAtFqhqGOMUBMY+huE38r3V9+gweMwwfumM0LS/FtHX1caCIySzZz9+w5y6/uzrtkVI6BT5lPYWCjOt7KCDqs/S3q/wSPekGeExq7Y2ZFzwTUMdVQQeTHf0OTVxNv1OqqGXBBABqvJfoJu9DjGwbQUTlj/JcBDRath9yDa+3zCbCE0wPP1MPHAIbGGJSyCMGFkbVeVkhn5BNPSlvFI3EAwNGKdco28+gGc7/+bHSgrIdZzyjbTETyfSp91ZAW9Zi+v83lelmT6FPxO78JfsWLBLHVHC/0Ts08rCzaPWB9t+yyzwwbQ9UifkD6kVL5JVO0iPK1RuJiCu807xrCZfJf1BFVyDZF1C/GufALJT6zFLcZgXCqf4G7X1WgNkRxwW8Syg8scZZAczL7Y6qd4L+UAC70kyP6YW9s28rgXdKqq2Z8ey4EeifbvGs1GtlSsoDTgh39UfZiUMIl5QW8S3DSDHqVfkH79K7SmUij8GYauE+sVEFoRSodhNP86tJMM08MCYJT9MUFDf8bscpHxfxuxND3FsMi1uLQcBMDVmMlsDwMesnBg9i7+mQ2zd5Dkn+SoT9t5qlWbxbaOl/ni9Bf2vOVYtnEycTw5YS9jkQ10mNspaMygUR+Cr6aY4ZHraZzbm31TzRR4bhQPdTZA5S4e1K3lIY2aZcuWseDHBQ7D6NknoeAHFlpHsNJnFGfjZwGgs9o6Y+/lEL+Id5tkVrqvQGXyIaX0I3t9drW15ATetc8TweNp8Z7FYT2czDjHlfQrSBbbNW0wjD9Ks3I4wS4N1HkcYKfPV+xqTxcy2V3vsvWvCr81nE6cKOKFBw6DIb8AFj7zDueudfOYvmUGl2oCBaO+/9egdCUrYCo7L0bjV+/HtQFnSfcoR1v2lTj7dtTQQ9lpZ82bAJ0FJFltL9dQ/8s8aiOuuUngIxnBIhzPGkUbh0c9zjPBQnL+dAcc9xwNrhFESXl8k9rI+Tff5eCog7RZlFxOrBcgi4sv0tNV7AUv15/l01Ofcur6eUe/RkJTOxjvtoGoFCrMZmjqCKTY9TmsSrHGd7jqMQSEERmSQK/ilQzTfS4k0RGxyj1kMwFZh/Er0dKz5DvuiHpW2MIQ53SLBYzhfmSnZtulKsW3RV0bTKabZBTBMYcbjVaMimYMtDriZHnEE3U8g0ffeZFHvnmElafiWZ+72B6z7/mzb/LB3+OJfS+W/WFxnI6fCp7J4mzc/0suu3zH2YopDK1cx/hLVSSYZjs+KqtQ2hxQZqsJJEeYgIEDoWfJ92wdYCSu8UEuRS3k5eOPOPqPLBQqjnR8TbdkMWLuqMCizaRdXcT85itMPWslw2fUTeVVWFyJq36a1KZnu/VxfwUkqLCfrGVnB3rVXqa7Ghidd8YuD+k898X7xvNS7y9JrPgXEZEPcsRq/G/3SaeAgwYDs8aNJMX/GPd5GdBIcDbqLgb9OIj8hnxwi+Gw7gOCtDVMcIVhpfBXaxTs6COUzM49yelIKxaFhVYPIberkGSxh/fqQV/DGe7zhNCG25l8rpMZrdvExxsvin/7fcn+huVYseInw/L+LyOds8WcVLrBbW2sPPcbLoZwXIyhNuUhD4i+ix26ddxTKliSs8xriap9SJA1AK+2nd2YfdZ/YPaBAI7fGOvU+b4SI4QVwqf1OQKQNyWL+iF/cV8VeOfG4NJWz8jE9agvPQK6PKjYycqAB7mlcj0VvkI1bF/v9SQ2PwJpLwr5cOAuTwOPeYG7Phlf3Ujc1e6g0LCp7Eu25jxHgKo/roZoAF72bWdSwndEmFeDe6w9byOjRqJ/WU/R0T788dnthF4XMrsVZyrwKTiLR23hf+ns+++ce/+vOvu6+TD+N77/v/LV/6T/pP/DZLFYuH79OhaLRSCijC1obkL33Wzc7X7dsSFoUg4mp7gPhoR+tEQ+RYzXYyKmy7knYGc/4lXtZOSkMGxPJVpdjWOCarmG0lBPiqeT/AQyRkWAYH1VbIPi33kofSFtQ7zZO24vbwd0ENz0CZj1yBiYF7mTrzzS7U+7qlwh/mGYeo06Y2o3g/d1s7dwxp19DKsk8ZquhqykwzSEpXO6fDr6Ht8IJPOITVz1WcIFr9+o8F0PgI9sFPIsNidniukiPwdBuCyRY4ByZR9x4N6WBpdf5QF1Kdo2VyLKIrjjcE821W8QC2NnLTUKX8Zevpsa7+34W7VcG7WMOHk19PkQWami3ejJ8qyHKQubxveLj1AxuB4PtQcgJvctuUsIL4JMm6/lHX8IPDBcxL2SZPYV3suf1T14ohbC5Xo4cTekPIsUISjj4/eMJyU7BYPGZNOoFk6d+T3uYXj2aXx1wygK/oQX/JbAHpsEZPxD6DzTebEwnUGnBpG15X6+Or9csA/2j4OyzciGGnSxiRT1nM7HEwfycT8JDPVwy0laDX7c6mkgjQwyCtaS4j5YxCEpWo1X5Scs8DZRH+rJkRFH6HTptAfDJXkpDaqRgNWB4rphqo2TxtGr5Hsmq73YFQohHccE6rfuJPj1Z33RN+S2xNFuhYNtniKmgS25NF/BNeIaHZoOwu/cRl+ffC4nNYHaDzrqUClUeHam4NoZQ6cVNreBxTvD3hZPJq7jPT9HAOHbtW0Q+wD0+YAdRU9z67lHhRSALe92PXxZRmVUofNoJTMtk1salGKToRFGoQ2tPpwdt5/KkEqKtD0xm2wBzOfq6PQdyFgnNmxCvjcNCT+CJgACR+IZ0I8nJxSyb1ow0aPPcLLHYfJZLAx3e0fQs2k/n/Uv53L6ZQoTzKhNflhU/jAtlybFINbeMoHp7sLZ164uIq8hlzaDgylkZ0zINwexbtJcZnevAA6mpdnRt1Ht1yDzHS76PcTiT6fz7BuvMjBrHV8GNQtAQPq/YMAKOhSxtDf4UNTvYUr6vUJvz7H297ekvsbQpjBKw0sxy2Yaoq7zS5tWIKH7fQkaf95pTmZtlRZ3nQbJIomNYe5X+FW+SYr/EcaF/AtXucZeDjS+0Ps9yplKbURfdD1CCa4MRmmyzXemNqjay3dBnYzLn0BMQQC3rb+t24bT28Wb9IqPSSlfbgcGODNIgqwNtE06wJUeZ4jMvcaG8mgIGgsWA5x9nCnXP2D8Dg1Dj4kN8EDfbHz2eUPhKsj4mLweh9k9O5Hs5GxafRvo0XsLtAvDnoYaJqS+S0G/t7gWfJV2C7RaXEDWIEnwUNoHXAudyJSzFhE7pf4o5H4BbtHs1/fj1q8WM3bfSCwBI1HhJeLJtJdD6BQORFUQUSqMa4dGHurGZnTEOHE4e5W6HCH/WHuUAosLm1pOUxz2N58nN/Fq0kY641+BuPvBrCeeFgxLVrJs2TLKIsq6obOSXH5nYvy3pLUs5cVtRVzS3SsYWUmPQfwirjb1o9HtNO7tPQhtnIuvi8OI6oxYb3G7QIP6Ms7pAjN450g8C19ZRq/f5nLS1AeQRMw9QyMvBf3NjE4DfY7V4l5fIvqIQiPmi4ErKEk9j8VJxlFhtcU7tRjBpzfv6fxpLQ/ExXqGe3u9hLLoW3ANp1nqjVqfwkT9nyRUvMLY6wfR+GUIpLKxmSLDZF7Yd4QzFZO47Y8Wzlk+ssee6wi+hR7XwWhjHDzSfxH+5wcJB5TKE4XCA40pEA99Or66kfiR3K3MsaadaLSinVL3jyLKmiOANDbW7PAZx8jPGEiTVxNNPk3dnPWyazi5RmG6PZoygKqQD9j01wdExffi6jfv8G5LET80wfs1MPBEA4qPYe/qPfbxda5iMptyBGK7a2z4auq5M/11AtWX+FdJP1YrjpKsghZcIGikkA3M+4Z+WiOtvTIB0Oq1toDyPhB1OwP06+3S3xZgRc0ECJkk1r8IYbjKM0BOSwSGCptcV5xDNq6ytYImN+H4tccUarwM+8axtN+tbIisIaRLSQpZGNmBdtdB3FeY2q1+7fJ8ed8y2CRi5xmVTezrFc4mw2Pd7m2X6qjx3so5aw33GXdz1agRBrD0ZXRGzGV9h9hb+NUp6F1wDyQshsrdsM4Vl8kXyWurYPiJe1meFcDsE69RHvGTYB8HjcKlfBOeCghUwMx8wXq0eqZwPeMYJfoR3PFXPSerThPWcKfIjGeiAKVo/Jmd+Dq7e/7NXYXVhNaL699Gvg4jNuFT8AjvDRks+qKmg80KG+vZuxeUbiC9cRbtabdQ47XdVl+2+nCPh7OPMd99F822urbXl9INTG3ICheO6EFfGM79Ift4rvcXuHRmCvWFA7fQs34HL3geA4OWlQ0KFh37kpbkb+HSi/iVvcz2yWNZwbNMrzprB6Z07VVlGbSGKALbR+DaGQt0n5dlSUKydIEWhLH7Ss+vYOga8hoOcXXIKM70P8bQ40PR6lwF4KDpCshqeikqiGzxYuThkcTnJTBO3uFoZG0Icyq7NTt/tHoIFlDyUxA6mWtxJ1ldkcGeXsGcSBwt5hn3GMyefbEid9vfPJX+nvgj5l6uaj7lcNTL9mtmlc0iEDgcEp/ATb2M7PR0KvyEg+RG1Gul207KfdaSGbGE7LAXbd+NBU0Ald5DebYW0k8NYOa62ynVe4LaSwA0pl2jTvaktVPFutvWMWTmIfHCoWvg8Cz8tkZREi1+WvH0ClxHK7lFwb81ZC0BfkbIKe7q7Ogms3jYBEct8PYfSj75YRch2gi8jFFENd/NTmsnY4umQft1McHE3Y8kyWg6NfjVexLQ1JcAbQjIKrhNByO28GydzJG0DMZcLmZWQTU0X4Xdg6B4DSpDA/01cOuW6fT6NYRRQafRnp4C14VkflvWYyTP+Yr9c26h3dtmxGq6CkfnoJXrea3GnTwj7AqDj0IuisKFToSgsTwdepg+GoitepYh1w7S09sWG+b6X4QckHjVX7Dxu/qtczrvsYW7fxNgLK86fzKNUZD5tjhbXHyRL5vGsbENCkLe4+HrEeT3ugQZH8HVt+HcUtwqV6B02pv5tG+Frakino02mM/qTlCB+P77RWOpGWeF8ce5pn2fd0ZN5IWQXwCIqF2IRqkWRr9TC5Ex8k1pBmUmiKt+mmTJJkGV9DioPDgZCcOyznIx5j4Op/amvM3GGEp+iqrYt7qV8cbYslmhmdz53XXCM1PJ0bSx0zhOxK9uL4Or/x/23jI8qqtt+//tPT4Td1eSkOAJ7k5xaYFCSw2q3C1tqTvUjVIKVdrSUihaoLi7a9CEJCTE3WV8/h/WZJJA7/d5ny//53iO915fILNn9l576bWu67zO80N03h2RK4Qz+CNfiNPvhHCnczj1dXbqdhClBHP2GWbb27JM/LvykAUOHjnN/uxHmZ0fQK0dDvrOJHZJLNXGarA24Ke6ypLTP/PX9ZcIVsAAdRFsChWZvLnr+DyoFI1kRm1WY3eApu6CM4t8NQnF63jZ2/kwfRrbYxfCRj9B9asLxuPwSHKjZO5zYix6+52AS6+5svJTT7+Gbwcxbk40aLlZPxpsjUgSNFg8yWz0psG5BMiS7HLOut98jhfD2to+rc/zsgyhVfdjVBewt2sg+3XPtP2uou35SmeKQuuUmGgNoujxWxJ7ugRQp71xF2AGwOoc2q2v7Q6ezVOqJVyMfoiEwgV0LficYL0P3HOOLzJyUFRMIVn1AEq7gWu6H/jz6p+u34pniwC23hRLZ4/B+BZthrRFdDVdYJheZPw1arJpVGe7ftfaXq9wP0SO/zJyrG1Xp+Y6emuL6BhwCKN7P5hSCmGTkMuOs8gP3G1KsNnJr21PjmK2OL9enA91N1HoNJzoX4Smx2Ey6kOoDX0F8reSVDCAVREVhClb2F/udO42n6ea96HmjC2Aeooo99jXZg/10NQQ63ORYsNjRJ94nD+9X+d66JOc8RQAE0JGwcRcTJKacJsICEXkRbS0g8YHvLpwsPQWVUphZ0rA30Gxgi4/6VUIn+Sy+SPKniSy3AkAyF6Fal83Xu57P7uiS1x+I9d8Tv6Ssogloi2rvRh2YBiG1pk9dRkMNvdnSNRK15n4m7pxMOjvVv11R2apJLVIb1Se5xm/WxhqPDFqjDzZFA4jjgggWPkZlA4bJo2J3MgS+qvGsrzrcvQ3X4dDo+GvQNa6ZZGshZ6ne/LqZy8zuCII2Vrveszk0EMs8RePfdUbfrL8DXWZSBLYHEpuNARw2zmuTxrhuPdoMEQguc/kmdQn0aTPof+x/szQNhJS/rYAplz/hC3td7XxWuTl3u2lr6kR/9psYHco8bEcJrhc2HJ2LCgLc7n48S60dWVIEi45HKvdQpVswKzzQCovc71LczDQLpmw28HuEBWXW/nEmv9vs9tce5HKaUOZrCYalfkYVYU02evY3c2L12rdW1h8bEbujyhHEVjOtaRrlAaU0qztKMsQUDuK3rmvEpUWxYxVozEry2kt6Vlkvkmpx07KJWFfJ3rvhz+VoA+jeugRau2A5Lir3gqFyPCVHErMUh15/j+zLuOXlndqFUBvUyovotsWSlK/jhxNSm4VUGnpizJjAQ2aTCxSw10/lyWZshi4GQU9tOBd3xcPhZNsM3sVeLRHOSkfH2N3LsTM4GhiT6oVGa7fh7iHMLPdXDSWYHJilxDS7UHGKDX/1k46DYxRKfjkQT2DNK/x88VFRKYmY25syUqWJRkM4VwyvcTYyM284g3naoN5xPNZsR9cnA9KAwsq2raHJEmCNWnARs5p+7l8ibJDjVJyppl3eA2m1WMJHIHKYxgh7plENi9NoRNgT18hPaPUYzB3YfjlPAZevyTWf10Q9F3JLes0F0DDzR6GyuZFrWYAAHrzVVRWL3ysSRhs4a66tfYt26QmjFS7ACmtrzWvZzag0AZ91QVwaAwYS1AEDWVPyUhGbptC1OW/uZA7HIfaX/g/TswgzJTbpj3a6WsIbRSyELhFw9jrbKhTk2GBHllrebb2FdopxJiQZQj3uI7WluM61y6pduNA9izSla+B2stVN0mS0Cq16O7tjrlHS7pe5KBIQi9uxz/3/D8G+5rj6f8J7v1zaRPD+B8o/wn2/af8rygOh4PKSqfOVuKL4NmBPR32tvnOv+P4/6frF+WxvLp1CsG704iv8CVKZRQOxZhHwGZke9hxOqslajxqONrpPurMteKH2zsi7+7B2pTv6ZQjuJglh4zWUQQh40QQQGlA6edBbUdvyv3LedmvidDyN6H/BjIMnzIk+DAPeJfg2dADQIgHa3zAIx613MBgrYJ4FbzuDSnqQqEPNmQXtfHzAeh6KZGgy6+QEDEPTeaHwrkbNpEtjo6cDVpImadw5BxuDBTZirVpAMwwLSPpTHfmvPMuD//2MGqMQusj/F6InkXDkGsUdhlFg7snHfO+xV1RIBwrE3MwBgqKOIVNz+hrWS2N6tcXSVZgsWs5fe17VHVf0i1vH1mxXkx16lSJw0HbFX9rAzTGvwAKDbIss/j0ClYVCdrFI7ZO0OcPIbp9/VPOhENQmT++Fb78HGQR2lD7B8NaA4YtEuPivnEF1U41xQsE681lULQbj9pUho0U6HFT2VQMCoWg14t+GAIG4X04nO6lPxN06wTt/C6KyinFSXZb1nzG/+mgXeV5DOYYsWlVnofM76kPnM3Df5XTL7OFms11OD37DKNq3NGqS7DJDXeNvdbFT2GjlxZ6Fz8jNHL29MFedZmmqvMM3nyax398HI8qg3CKHBgFNxbRp3wLy8Zc4ZPXP+G39pd57cocZGzwdzRsT8Jnky81HW/weNHPd9VNkmCA30XGu7UEPp7VVUL6Vy19dYfR+ZT9ImR8xxDreV4abWb3ff7UetYKB58hAu4tg5kO5ldGkWOWCCryw7vKG51kEdkrshr/6+/wqEfLPYssSmyGRBifDp1EtptKqyD0dgNB773E4svdMEjOtKLAYVzyn8TkIjg2zMRrI6wcmtyR5KKO4LDTpIpj7c05pJkFmupcuykkLE3gWK7ILGyrm9HiUHJpUEgSZlU5ZmW5K6NiQO5XcO0DNJKJCt8KlDYFo9fd4ELaFIFkttSCZxJnPI9Tva6K6IubsDtRidSkwToPIgtWcWZ2Jpm9X2DjvRu5HHWLBU0G4eRX6EBWssEYSf+Fr/LC4udQm9XicJCzCs/S7+kceIAZSQvJdfuAYq8tnCjdSbVdgqRX2FF3htv3jaBYcZCnfniK4ConeCD1LTgwgrHe54iOH0BOewUHhxxsCXwVbIPTT+CvdxqOznmjdhgFjV31Ncaa93A8HE72PsXieYtZYI+AYfuE3kz1Vej+NcZcBz1ORdP37AtUVvfCrvaDc3Oh6iKNqiT09sF4VXuhk+GFoe8IdBqgd+TxZWgxY/aMZOricWxMe4yPy5eB1h9ZhkDDbfzVN5CQuBT1KJ2Or+XvpB9BH8YE/ceo5eHo1YMJy/6eDZWFED5F6M+pPKmyKTkw9AArHtuBwu6LzokwFeO/mUKlhV4NnxQYvAOCRnAu7yg3tn/PGx/+i4+OxvHNrXvEfD/xIJQe5n5pJIurxc/ClbC0T7jIKtw3mNEeM5ncdS7rg2P5eNhQAjWpIiMLoOd3zLodx/HEXuT4LHfNseaivHOfukML6aZiOBsLxMTpcL2DmHOn58CWCLj6Pi8VjeLUuXhibzrw9Xwew+EkyNsMjQU4zLVM2Z3MkaRumBUCnedf/S1sChK0rIDSqmLa+ml4pt5iytlncHgnQ2M+dYpOxLvnMFARS0Lh+wQ0DRLIyu2JsG8QCsnGheiZbE2ROR7+BD72HwX1csg46nqtdNX/tt9PFDiEYwJrI3RagF/JBwyKXEVY5YP0TT/Eqz52IRZvqgBghXo5U9xvsXvkbs5P2cIG5Xzo9yfMsEGXDwEoaRfP4hcWMzS+mEc1Mlx8FdKXIpWfIFQhdptGTTbjat5i/Yr3OGKzs9tkucNJ7+C4A/Z8s4Gv3nkMd/MZ9t2exofZAynw/NuFGi6oa8fcHVe43PAEINFeBTeiYLTqhgioXHkXTGUEp4Xxa7KYMxW+FW3W/CjLOXq18h+9FboBzv0LzjwFm8NpNMTydwPU2iXOFo7n2ZMOgc7/KxguvkzNvsEM6/ycc4w4M/usdVB5nnZepxjoZsTgfJwsScJJDuQlnoRWjhKDMb5lzJ19mmRLyx4BbbVA/unvTo1rYHM4OGxoQ4bTq+5r9o33oqJDd6akfCTGnnssBAwGtSeW0hqG7Y5mwt8TyKwPp9bvEej5HYQKp/vLZbCuHsJwbg5NhbhfFsHOerMPRqvBRZfjUHmLrNbSQ6gURjTKRiRsrgxOh0IHSh1XFe1YZdeS1a6K3x/6nX0OI3R4SwReE19mtTGeq/qjNGpuOdurGfYvgzaQOruYg3218LYPKC1Fgs78yCRkQzhXIn8l/o9p7Fwznj8uL6DRTTgCUHtD+/mclX5i5+gzbO56jqKmEDwynof286kJe5NdmU+QU9O1DfK3NSo4tHImw/IOE1Mi0Nl3ItYl2gb7bIYoMETSvvJr0jpcozo6h5O9T7EoKgN8e0HHd0HlznTjbL5TVgMQUOXDWvsjMErYRZgq2Vjf0sc9NbA7+JIY192+AJ9uvJ72I1uj7sOkKsGkKnGOv0YU9jpSgnfwtq8RfXMWRSv6x2z1PPzyxFqz5NklmHROr6MuGNReDHO7gdr9ast4u8NOOhI2louxM8gO/JqsoM+ErTviGHR6h3LPnvyU44nDLnFq0GF8NCY4PAG2J4E+nDnxI5iW/iLT100n7eAf/HBttVjzC7YCgsoSwKqyUjPIgnGYnoEKBaO0WlYB+4A/gF4KiaXAbvg/0ixut1h57bXXyG5I4F+bLlGzsR29j/UTYIsr7wmnNPC0zxl6dbxNXhRI9jsCZ2HjWVrdzI/oEHNZHyEyuQMG0q7+IltswURnxmGR7fQ9/QQSDkFTe3AM3qV/0Md3M+a6ueTrnsSsLhLaPH3/JN863OWkr7XosLfeawIGMSdrBgea4C46r6yfAKixibFXr71Bg/pWm2ob7fVkhv3J2mnrWfbMMm5bA1suFu5o46y94PsjDeoQQILLb0H6YrwznuRCh64c6NiOtFBncFiSaGahaLMOubS4+tCgTAJaKGQ75n2NVqWB+hzI+hkjAXx+ewC3LII+rY3jKWQsZ0sewqsxhUZNBrX6VMx2pwfRbsa7dG0LVSV3A/YkCSwaA9c7VrLbq4TNtkmCDrPqksjSHH6YP9LewmCMJ7Xen7TG+4Se8+nHoamIA7ZwGh1gMze5tI7+qxIIVBibo3F3rNWSDJUXedC7E3pVLcUmbwpj4AFDurBZTz4IJQcZmWdgz5BDfPbqZyQE1uJ9ZbagXlZ5UuaR7NSsxbWfiLbuL+hHgXCVHZ0lgAQVPBLzI9z4DLZEw7l5ROf+yiu+Ym2qt8vEu+2E4v1Imd9jtumZff5lzphAtukIU3eCwdsh8SUUxlyeay1XwN1nRlEc/3hNcUewRUJyBfBaZ/YBmFVlSK3GQp2pjgZdGoXea9lpe4k83xVtxklA3RVe9QZ3J+vH8Ohf8T0RB7u6E6AVLBZ2ews46E5NXJkWqnZJQgDBhh3gc78FTC9uHTxveWjrexR6r+Vq5L+4btnu+qyisYJCczpGVSGdw9bw8bAhqBtTRXZK5g9IN7/mBW9Ie2IlxYNSMKiruKL+WtCtjjgBFWeJVSznckoZ9vhsfNV1Ims66yeU9ioezg2iwPrPgf3WZ0S7JIIozUEaaAlwNF+TkRkwQozwhweuJMf/J57+ZhRvfvQm3lWeLTc2hPNhwJu8Z2zi58d+ZvG8xS393OldGLKDUKWdZt99oALGu2eJcwuAqZI37Ot4wQvarGF2M1jr8NPnkaixuEZC62Cy1pSBIw76dsrhZO+TWNTObEpzNeRt4IZjMOvl42QHfk2iGh532wflZ1y/76aaTlxhS1aqovU+5h7HpzfSuTgshBWPrMCmdjp7076GPb0YnLuY97sWcWJgEaXmsW0b27cnXxuDyLbAgKMD0DUa+KQmArlSBFJlGb7LmMYgJxPjGRPsVcQJHWkJrHYNsy7P4LMq0BtjeDT6XfqG9wXA32MAhdeXEn0kliGHhnCPykxQ+QfCDvIXgGlVq3kQFNh2vckOWMLvmZ+TsDSBn+WeWJWl5Hh/Tm7nRfQMGEzcrf74b/2VzD9Oo2wqwC4b29B4muw2bEoNdrsVs6ISk70RjcKZ2SeLzD6bo9lGagHvN88ns82KX+1wokrnkhwswMxnCs6wKjCck/FDsbTK4nWB/7OWE9c1g30vLmX9tPV82rmUqe2Wu4J9AyP/pMOYXDxHeJIbkYuEjNJWAbkbofQIHRsmE5U8hkzPHwBotPuJIJI+so2tZnW0ZWZoHmp2e+uMr1ZBILnt/tKyvjkwuSWQ46RW/qfsqRfOjudgpzhKDHupMpykUnvedU0hK1jntHPKLEr6pR0nWSfAhRTthozvQFIgSVCrv0iN4axr3rauW5XbcTKDP4Y+GkZPfIGBCpmRWk0bO2mgQkE/CXo9PoIX3viYnZlPY7Lp6Xz9PIOvZqC2+rW8m7EUg1zIwrOf8EYFKLBTau+Daw2susRnlWJNiSl+mcS8L/DVOiPaHvHs8ZzOkVbyPS2VVYFCj+LYFF4POEBvrchuf+7EN5CyWLAjmCvJzdqAFPADNbpL3FkStb/xebA4L9ua7TOFDvr+iVUZQDtjD2ZWX2NgdYsWX/P+I8twPvY+1kR5U+C7ytnPreeNeD8V0E4FV63+QnKhqRAvcwnDi3ZT798flamBy7UfIZnLRICz3xo8bNX01oJv7VAGXLvIxozHqNTcA9c/ExnybrE8WuzJrkZQKYy8N3gshtwvXPVaNCqFMPNIYUcDR5p0eGuLSbB+AvuH0TmwM58M+4QnU8T5q+Pk97kcuICiWLFenFiRwa3+syiKG9Cmvf67NJ7/r5Y2MYz/gfKfYN9/yv/OEjiYbZURbT76Jy2kNtdbGdO9Gx/g3TFLqfKqI8Z8jL4NA0Xgq/evMHAz31W3o7rUG89aT6rdr7uolujwOhiLiSl7ihZdDZn4pjfh5hKxeYZNYNbtwbD1IgMPD+TRPCekv/IssgwfH9vI2ycvY5VFANFD4yF40hsLiNIf5vduy/mXYxgf+cGX3rthrRY8EjEU/s2aIJiwdQJhuQYKHfnYtaEgKaA2A4OjlccGuGHyFpRaSiG8ZFEEkJMnMsT6xhfyTMM4OPUIuLeD0HHgkUSh/2BuDJrHmOSTPBUcCbtSAPC/8hrGWNDIVqqMwYz/08FqBJ1m68W+ecGv1/YWOkqIxT7MPY0H3CT8nfbNX/Vg6vC22JyBMo896CK+YkMwuElGCBwihGfTFuOngJz4DGJvxTKg3nnQ1QaBrRGHrMFk07v64oOqmYIeI3cdZHxHkz6aX/3y+OT1nymKH0ReXQcheNvhdfDqyBldb24HpHEr4jDRx57l1Us2gZK59jF+ehFsMtkbsMp12LEKpw2gsFXQYPHG1mqHkyUZjOWQJZz6O7sEkxGy0DlC2i61dsmMRVHNumpvAtIDuBaw0HXNmrmcfsFT0DXaCS0MZbdnsXCKFO+Bq+9z2Wckzznps08a4Wh5ZzwbtosgUnOQAQjzSKOjGipjwD1XCI7LMjxw4lM63cZlKD3fGAE9hfEa63WWh8OP46+AgOpxrOx3geH2W1C8j0DjVcYFn6H76fbM/2I+s80GIebtLA8GfEl0xRL+mp7Gqd6neL92jjhc7u1HbcLLfFjVPFsgXm1HYa0QlA1/KuDyO/TJURAflMPNuJt8UxlNuuJlMR+7fkKVvh3Zl+KYuCGBqlwxnhX2Wrj5LSrq+PHKW1w0gR1r24yJirMoJeNdNJ4KSeFaLxSSGIMOyYKtFbochR4PawmRo4+z9d6jlId14YZpItz6RTg3Lr3OX8WLuNwlhwp/Exiv4bBli4CItQ7MVWhLN6B11JFyPoXBhwdjkSoEcm2dAWqu817ctpZ+V1rF4WDAXxR0PMXuzMdJzlGwUfkN52Pu47kzYwQFBtBFdZqDMeV4J9ZRkDCEPnXOLEq/PhD9MJ/fzOZ01X2cH6IitWtqC1XdtY8h6yeSI3+hUZ3jCmjo7ZWCpvP2n1xUp/D2ur4MOTSAau9qHArnBJ+QA0P3cs17MPvuG45dYaDviY6EO4Kwe3QGQyRFdYU8tccHQ/bTZMVm8ffU9YzMUwsNHaBJGc/Y9HYoD/bFUKyiU95yumkE2l2S4LMTa/kyIwcQ9EY3qvOoQg35W3jYKxr30V5IFjNd9nwu6jRgoxhjIWNYfyuLXlmX6Jf6IxOOX+He2Eddbdu8H7TO7JN1gRAyGgzhDMv+iI8ixCEn5pcHOFDeGf2VJyBnFZQeRZZkUs6l8N577zHu8sPolTUii7vmOqcaZzOuyE6DMh83dRVB6kuwVidQvZvC6OnUGir2/JtC77XUmCtb6qVou0/d6cRqbz/AkWkXuPj2FygWLuIb94Xg68wqd9g51hhDdaM4IA/ocRJVww04+xRsDkM6MpF7HBdp1F9if+cI0kLfwKjrBnFzxQGi9Ci/h92iPqiEjNgsbtSHY096FwZuwSL58Eq/+5kZPaulMgUtY9VPfYPnQ6+SpIagkDXo7WL9IOYR1OXHmOspDjSahCcYH3KC+qi3IHQMaIMo1g/inPt6Mp2HjpEBrwm93Crh7KtUxJBqhpN9T7K5XRppir6tThQWOHgP02K/Q2FV8IGfGUlyCIfP+Wdhb3/yYxwEmGOgDj7cYGar9f/CSf/hr6huL8Am13Ou3WTOt7vXBSCRlW7I3h2xoWVxl2VMdoOFFdBPlS3Wrr6rofsyJGTaZbaj05VO2JQ2sb5UnIONAZwyPMb4QpDtWgZm/o1k9wHvLiIzMGIatzp8ysJKMGpyqdafxyYZhdPFLQbU3iSZM0nWNI8RZ7DPvx9MreKFI4UEXI0n0wL+NSOJ0naBhHnQdxUelStZ6N9AQNMAeqfvY8jVdHQqZ4q10o1qObptg9xxMmteN/pqYWsIxBkFHRT1OeiLdrKj6/sMCY6ivruKkR2Xw9HJIjt9yA5oyCUgVGQlh5v0PBS5B0PdPig5BCo3bJ6dyHA6lm/XJvLcwUKQFHjX7kKnqCbM+wR2xX6uRjzHxyollds6Q+obcONL1qV/zJPbMmi0uLvWsGBzHtRlkqrpwbzGAk71Oo1VaSVEKYPDAgV/gy6En4xdyLeCb90gYotepb12sKjE+eeh9gZfVzvnkw4W+oJsKhJ1do6F+9o9glGng6Im9pTosSt9BA3tmFQsvr24npfMjNXD6XStA+8nv4e27C9I+xKz70iWnf2B1LJ7RLDvDgqq1naU3hTNw54r+HzE5226xkVz6BybOlstGMsp0yazuFiNtjYcT+XPeOtiofIcXF0AO7shSzKN+kZSkxtICB3CdHmFyBgbl4aUsrhNnytbDwGHHRx2DJK1VUaE7AJC+B3yokfIdt72N2GQIbpkHn2DRrh+3mA+hlenYxR2MVLrUds2YNL+BUbnTiCjZUm+a/1r7aS/85osyfiX+dM1tSvdg2s4HLtH2JPu7aAug+FX5zCrcz5lEcnkNXXjUsU04fidYaX23hrmOW2o5PPJ3LtlIjG9vbmdm8PAN9/kJfdEZhgMvOAPZ8IdDFb++zWkufQCBqtV7Ni+HIXVSFSmhRH7RhCqbIRhB6D2BuxM4UGvVMZL7sSlgcO4h1JjvmCqyN8KVamuNnLgDEjoQ4RGq293VIGDaIyZR0WEka3jt1JmdqPhHpHNg293lhoeZv7pCMbsHENAfg5WZa2g9o66Hw9FHtsiquijhXHbT7DgptN2vfQaHByJreYtYm7sJKJMZMS4loIOb1Lb/nuWVwrH/IXY+9kb17HNuwcbv2PV0OeQu15jUrsyFnsvF4HiEcdgzGXe8d3M4854vp8CAoqWCr3m0Zdg0HYer42lSJNGozaLav1Z6jzGwdhrEDQMCnZwPHAXD7hDTPFLfB5STMA+Cc7ORZZsvHFwJzOLcbUZICglp9aQY5/qCg5WGU5RhTNIWbANbEY253zs/N0dQSRTOZqGa7i3Gq53Bvuiql5i42xvvHUN+FT6CPvfZoRJuTBkBxabBaVDy8Brqdw8XEq1NQaufSTOB56JfGHrRakNrIYaiv+LsdVcSgCDwZO+4Rv4JrgMn1ZVaqYWP934LukVvdHU9mdlXj+uSI/DyBMQMgY8EjnapMbs9DfN83L+OGgYFO/lVsh0V+C/1AZjsx+HqTUiYzb5C9CFUmjuzd+VkbziqSXKkA33XBASAsDxsCd4Uiz53DCp2V36pdAHPvs0Q6N/I6bkBT7VORhzsZGO+uGg0AoaYl0MIy/e73oXjTm4Dd2VLEOR1yYXhdmdfaG8I7PPoyFZnNcQ47jnzV3Mtp5H59Qyax3o2ZO1h70JiVyIvZ+T0pcUe21uwxbQM/0NPvEDd6UZo6qQ7CYfrPo4ULrzUPhoQoJXkmraTL02raUfnEWSICVrA0NyDtOgyWBr8bdcaagHj3gCvOIw1A7CYIy/651a36OZgaX1Z0tOL2FeRnuuhc/jZ/8XmVsKNm2soLM//xzIWiKyJU4ZIbL7YL4Z3RUvx0WxFvj3oTZ9GdQ+Q7H3X6yvh+dSn8Su8oGun5AWfpS/qn2otQsaz/2dogj82o1rpddc79R8nnJl9ilaMvsUzba28yylkJV4lgrHuNorkgFaCZVVfN+sdhoATUWQvxVPez02pY28iDyqvavbZnqnfcUR30JinI8yOmB9zVQInwS7e8HVhfg7avBTQLnHPirdjos1LPZRHOMyeXXfcRKu9KbROfYNSmeg8coCvMuWsbYOKi8kEVgSiEXjXEeMZXDtI/Js7mzTCiaAGBVMNJwVFNjOkqKZQULhAlfmvSxJkLxIXHTYqDDHkR/sQ21IMS/55sJ6bwGw7foZekslHTUwdksibsfO8dYBJ4j9vioYdZrfLMHkWeGPB/8gdcJ2NrW7IVgInH2R3RDBMWd8ZlsD/K7uBbrAVqac+E9g9UReC4ylX8a7UJ+NJIHCYkSfW8vtiNu8Vuf0raR/DSOOkZQ6B1Mrv/Sdcy4t9HWWZbzCzYqbFMln8dEV0rl0IPF1F/hj+EGiKz+hITIRz7nt2Di8J+t9ktsEhR1YSOvzMJdGV7Cnmy8Lrz1wV2afrTlDrg3Dgfi/xWojrPJBuhYuZUTsCOdYawZ1WttQ9roCcWpvzpqV5DiH3Ux36OJ/2kXjOSnhSybEvk/YVxFsnbAVHDKKxnRBiZ7zJ/1U1+msgUy/JZxIGMB5cz0M/As0vuhu/Sw0VAErLX4GEPfO9/2dz7JmcNtjdZtrzddbg6yDcZ7xqi+jqU+n2hkkPOT2NOdi7iOvsSX7rvls26DK4XhiX/aFDG3VVjLTi0HKwAXicJXuX8PQvew59xnp/p9iUpWIfm61ztSZ6jhTeohKN6cki0PioXu6cnuJnf6P9GG+WyLTNSG86ZvI5ZQu2F6EAfcPhPi5nC1/EF9dPrHeF1BIFmgNhvgrkNk+oShlG2eN0NChhBT1e8IPCdDjewIbhuFfM5KY4heJLZmPl9ZbZJ0emcJQVTURpU8iOZRciJ7JeY0IatGYD1vbIdeItWp/I9gaY7lcMhB0oTAhC3p8R8TpqUzo+BQ5AS1+MRrz4dRsBmqf5hm/KmRgv+oFKt2OEdj0O5yYQUDFF3QP3S6Ctg6ZDrlf80qXRYKdzVnuBJa3XrdlSQKHRIQKMqIgWVUs5AOO3gvb2vPloCjsCQGc6nWKY16l5I8sEgDXsIn0r9zGM56gsQTg2dSVFed+5lrATnGuSVsEazXM9vDFvbEjNpuBr0//jDF0tuv9fr85m6+qMlpYNJBJLXEyUfmkkOifyKv9X8WgMvDQpofIPzaED7v2QnYyw139/SJ1/jE0eQT9YwDv32X23Vn+E/z7nyn/Cfb9p/zvK2UnQOXBshtvElM8n/b5H9M37SjezgMH/HPGRGvDUeUwkhxbgs6koCK3lHPyw2LBvfohuLfjVNM7XIjtyA9P/ECTrqnlcNB5IfReQZH3K20oGsvU4wHJhX4syNpOcJEXKRe6c6S8C1kxR8FYgpflGDUmX8pMbphUgjvJXeMOaV/C5jBsaPnu3FL2lifS9Tb8WN9dBKdkNaraG/TVwqFBh8gLyyNda0SyVAvarG3xPF/zPB1bIVG9FBaRweYWBUB6hxJ+167mzzkq6gbXkaceBX1+h4jp4LAjm0sIv7WPkMwjPJEyD1myiSy5WytwqD3Z3Qj2VmLifXgI/lQi2U1oFA18MzaKPsEzMSqLyQn4AXoso7i4mHVr32ff7kGc+dABS4BDQF1bJ9frnb/kA1+YZID+HjFgaxKaH3FP0+6Wmlz/UspDIvnm/D4YfREGbICZDpbHfs9rhoXcDvgeDxme9twhUFj910PkdDSykpKsk/S5spQHu87mkc7zofdv4CH0B763JfL5sJ2sn7aeWv0l4WAtOwmpb5Dkd4xOAQfZ7zacXckenK7eItoTMHv0IdrrEjhkvOv74lXfW2glOaxgiOSo9GAbDZs7DeVUVrK7mzeeTV0ZmVqCMvxtcageewO7V0ce7ZZL5vufsv25b7kZWCt0oRR6GLydAvfOnDkfx3vvvcfcpXOJ1JURlXev4JZPWQz3VTJ3fzEnbs+i081NZFb0RNaFuJ7tUzWZaQVFDE8t4AU5k8+nHwDvrgB09NvPa+22EyXp0ZpDae/VDe6rht6/UdNxNbOWZ+NfkI9Ja+TDoHrhfHXSlTyqXUn/6yamre6Ae507Nkkp6KUqz6GuTuWG06k3oOALrrSrxDPndZGBonTD4hbPzuJo5D2+lPuV02iwUiOniExbtSfB7sFEGVMIzffgyPahjP/TwfWwE3D+WYIt61qJX7cE+zybsmF3T0b4zCOy7CnGZF8jrugt4A5qBUfzAcPsyuxLj3gc+vxOu7pdrAqCwvbpHB/axBmtRWhJmish43u6mL+jdtLXeJVkE5j2Fem1X4ubdv0E7DbcU6fSc8teYm/FYtOYuBzaIAIbACp3htZ3ov2z9aycnYFNaRMHGV0QDl04VcZgUk0KjA5wyOKd9MZC2NWTUEUZb/w8HPX+KIriBuLQONOGIqdBnxWUWf0x2qpQ1TehtChbgs0dXseu8mCP7wIOdI52HcitKn/o+ydETMMrci41Z++h15lehNd48UXgZdgcAQeGgsqDhzY/xPHEvqT2iSC//TA6+B9FWX4A+q7GrWQ3x7xVdLvUjf7H+zP+x9nkXGknsl4Ah9Kd442e5IbnYh0fQo/Q7STrvgC7BUkSwdq/5Ps5HzMVm1xHsAK8zMUgqzA6fLDZlTQZ3GjycSdJ95nQfrr2IRyfhrsUhGeNmrjzm/EpaMkWgZaDVXM7tv4MoMgjmfMdrrJnxB72DdsHDgm5/ppwVPX8nr6UMPSYQLQZr88VmVejTsO9pexrfE8EmiUzbupqcUObESQlNBUwK0DUpdZwkQux91PQ2JIVYVAZiCp5Dp+6Qf+4TiRLhwTC3p6M1jyJRilOUCXOdECvn5AlmUtdL7Fq5ioWSFVUxb4lkIIJ86A+ky/9QSOBTdGASVlGg/tw6LFU7AnmKpJ1Ddx66TvO9DmFQWFGwgI3lxLk2M3atGdZWdaLEwkDuem9FLuxSGj7DD+Mv+oyL4dfobcWzkeA1aGEzu/D2adwP/cUSwNAK+NyzFjdOoGkgrJjnNMMZa12C0XegnpuW/E3ImPL1giAt6OISQ2+jDw8iMDiQHRSE5x6TFDCFe7AWHSA4h+svP3B2zx3Lord1v1wzxmhn9ftC0q9nsW3ejych76S8v/aSf/9kSAmxv3I38Et2rIAP35n4+tPy9Ao6unvdxl/BbxbCVvNHcX+6NsTVJ6Mc2ti0OneDDswjKGpWU5KVR14JmJWCE+3W1MiYU29CdZmgawRVJ1BI+h45j5qYmGYHty7d+fb3jqRgT7yOHR4g7cD3+ARcRbHU7bjo7wJNrFWyTIEV99L+6bH6JC7hHBtB6EXEzUT96q/eCr0AiPyj+BXN6zti0+rY6vH7jYf3ZXZ5zyoBihgmA5u6idBwvNQtBNKDlDaGIf50A0KSt9lS61BZCE5ATwN+0cQphnID0/8wLT5q5kfv56IzBGwfwgU7aax33ociEzZayELuKjZhyNGBOjdVKUsGTmQHiHDuR0gdEwuKJ2ByV4CiZ3v8weHw0ZT5CPG0dT8RXDpVZet9sCqe3jmu2d4TWEQIKzRF8G3Bzqn3eZfew+JBZ/Q1TBG3Ne9Hfj1xeoMDvxSC51zZKyGRIieJRDcxlJ8c1+lemAVF7teJDtoucisspnAbuGvG3/xZlVf6gJL+CrUwrDQA+LebrFIEjSqc9gXNoT9/hNdGYn/pPehsfrTV/8w0zpMc/VFa/2n5pJ4diYcnUS+YSjv5ekYuq0/PoVX+TbrLMQ7NUrCJpOiyCPS3cTV7nrKPUeQq3kGTj0K29ojXXmP8hhQO59/wgirLE5t2TVKuL2Wlcrf+aDZvG7OLA0ajin0MdZdf5N+N7rgMCYRVfov2nt1cd5oFt2bHuL96Z+i9bnCzDVPEdzkdGJl/wGbgmmnrqV17shd4vWtNJZwyEgOC1z9APL/JrZ0Iyt6FrD5oTVUu4/kQO0UQY+tNMDuntD+RQpUg/DPvYBvfipPJE2F9R7gcLRxwEzYOoFOF7uxzaIhSF/Fm2+9Rf9R1wmc9gnlc8FQD4/eneDyj+WhJiPH9iwnbFgHrgw/gLV3Dqsi94igqXs7ULnxS3E6W0+eYP/wK+wd+DGZdali7TsyAXZ25XREI2oJzrebwumAp9o+wCcZRZ9X2T7GQlmXa/T1ykWy1kKfldB5IQ1Kb5RloqNKAkvaBEi1cjk9dBa8ZeE4dDW1tREsNfg54gisHcnVyLns6eLP+Uona0XAAIxhT2Kxtzpc3BF4TbJ8T5IGJqcmo0hNpN6hE0Aa/37gsNJfl8HY+gcBWOYPwfkL4MYXAvAQOoZUKQiL6853uCW0/pw2B1BkFZnJwyPFfCfjW7rW309rN4ZRVQQ4xBqk8iBEcZAVHTaRrIEKjwMcdCwQX8z/G668Q6bhWzKDPsGqrAEQNjxA8R4s+niOt0lyaPvOYXVfAhB4KYhxOXE8ofwB9vRxASRP7J6IrrcbVyJFH3Zz+15QXd9XDQkvuMZgQ0f4SXHHefXflN/VMHDwMBJ8T/OoTx0jMlt0NGVJBrcozhrf49X+k3l84FDm1N3E4O7UVxy8HQasb8MesKYOGqP+Jaj4zz5NUNVJ1zUHUGN3EzpJWb/Aru7Q8S0O1y1iwI0zZF3czh9p34BXRxh3A7p/TYUhgbNOvNwMrwaS3DeKzNKh+7hUcg/3d1yAl5zW8kJlJyFwCA0df+JWo8jcSCx/gxGXC9E3g1IQa9/5dlPEGY271wmVrCSmeL7r72KfjQIwg9gfPZu6EmhPRqPQOj9rGTN3sgC5wAytSq4FSqwyhzok8mXINC502g49v8eOkgq/1fxunkyx98aWfmhVb5/6/vjWD6TQZw3LcuZy6tYOMFUwNSie+wp/pF3x6809+I91atEhvzvzzy5ZSLPAt9UysqMR9g2C4FHQ6V2KbArMDjhireDbkhASLW+LfrSZOBo5nymtqJu/6vwTvllPgmcSDfr+rn3GgRWb3ECDpcH1fEkCv7phhFTMRGMRGXttM/uc2efO9ldISkGbG3k/NBVwJNzBqee+Z8mzS7C6mYTMSPlJODKBZ5SZzL69hb6n5xBSENJij+T9BWXH+KHRiyqnC6LaDj/VWoWN3lQEkpJXNXN5swJqDBc4H3tvm34A6Jd2nCcKHYw752BAgFNzL/sPtHVHmHtzCPlGhVMSwfkDfRiMOMYp2xDXvfY1wk1LiIsxqfX9cVLXy7IE8f+CqbXg24ttHhH45H7Li++/xUP2erBUQ/A9kDgf1fg0DpZcRdekpySonCN1PtSPt4t5B2y+fzMTcjIpCSphU/JZPrXXUdqg54P33+ev3xO5teFJDEtx+VXuBMXMizzGTHdRN429ULAT2Yzoa5ezcro/ipEWqrp3xb3wXXIj1gv7rjGfAMLxqu/puk92trivl7aErkF7CVK0fY7R4k65+8PgmYQsg1nvTcmgaWgnBGFRW5CQXZp8opFK6LbnC8IPCNC7UlK2ZPY5Nfv0BJCY9xkPhrdkTTaPL4utJZjfXJoDXw7J1laf06XLPIfnLyQw/uu5vPzZyyTuj2DhiZbMvg+vzuWRK968/cmbqE1qATZyjxdn44TneLjpLd4WBCRUuh/D7NROpmg3utRXCG8+7zjaBvsUCqjRX+Bo9RqqtBfaXGt5B/F3j4ytPGh3Bnz9+lCZ+A7pTtaiXPVOin020mCtbvVuTiCz3KzzezdtJIC7sUPb9lJ7Q9piBmUt5Ergay11abUOZVRm8MihIZR57nZdK7QNIcgL3hlwiB6Dr2OYMxrTXB3mYZXg3vJMhwOmJH7O4ntS8NSWtYAnW9VJJVlxAKvrYH9jPnT7EqY3QckBDkRV0jtjJ1prq7x3cyXkb6J3/jvMsY3Bt24Ihb5/kq887Kzw91B/C4dXJ9QZUGSD/jdOsyD5E1irEWBRScFBr9GsroM8/585GztJtImpEm79Qpr1cQacnIcdKJEvUqO/gNpW4qqCTlmHwyEymKNLn+ORhBdc47o1+0ZrYHnrfpaQqbLBR5WQ6lw/6fwBBA6lzuyPydObXaN38XlKIWF7ggUQTFazI2g2X1cLhgKFZGFiwiK8jXtg8E7oIeQYFgemMa9yESfixvGK+6tkKsJcz/3j5hxWO7M8PczxbAhupHfYJjaoHdBjmauOV0uvsvLySipWSGz6YAyB2adp8AxB4aH/x4DenTSed5Y7Jer+E+z7nyn/Cfb9p/yvKJIkERQUJAzO22vhzBO0bxpNUv4XtCt+DZ/6/kKrwVk81d50z9xMpxyRcXDnATJPISg0Tz+xghVTlrKyIVwEPK59BBnf0Us3i+TLE7h3471O1GarFSrmYUZdWsDVyGeIKX6R8PLHKNdNFpzPmT9A9TWm+TRR61tBfsI9jCpcikNhgNQ3CTatRu29l/MJbgJ1C0Lbzq8PJMyj2JzMjoy5pDcEkGqGdU3J0HsF/BWAQ+1DRA4cGnKIn+f8jFVhF4daWS1okRDYGbemRHql72Wh/6WWwAKgshYR53OGjmXzOHTiJHkhuyBgEFx4Ea5/im5nEIHXjxOSdohHDs3lt/Jrwgl76lEsfv2ZWARGycrVhJH07fw44Wyk2NaOLxd9zPH9KQx+NZcXP/2TA1URPLS7G0+/8SSRYWEUrfuI9ypL+a4KviqDAcdA9RX8+HxL5t89IQdRAL6pyYSovWDfQLjnPHR6BwmZerd6iqLjKXJ0cQWmABos9TRpcgHwkGGs4Qwcny4yV/QRyPWZdEndx6BtF8kpOikOTuUnYGsC5G5oY3iM9qzkgchp4uB7zzn89Ll8NGwogW6nXGOQTu/CmCsYSn5lyehuHO3Yjg65S+ifdlJsYrogmJjDSWVLVpH47T87UUPV9fQK3YLSWgJKd3Bvhz1yJnIG/GFr4qxPKY+WRQpnkNMZLksywUWiv2PdTbycsIYmbSdXULcBNauChrK3fQ+qK+5h4f7TOCKEo0WWIcYjk/b6PLSWEHykWCLdg1y6jieKHuSlPSeIOl9C59zvxTupPUHlRrWlASoP4llZyaZ7N/KQEUh8uWVa8Cv6IBtFIU0orUpiFEWCZrH9fIzxz7m+5182m3XX3sDoPUI4jUPHY4u4j/vKCznb/Sz9gmPZElWMlmI4MQsuvc7I4hVsGScyiwp811BlOINN6QdD91Kv6saH/R5hiE4YxM0ZEw6VN/j2pMTSXWiEGZNQW4VDoQ2qjpbMvt7pBzh+bz6RvZdAxL24xy3g/d9XMHLTG1wPfYSBoXOF82JaAwzZyTOGTIbq4Vj/Y5xPOU+xIwSG7oWIaVQGjeXVxmgUVmHkKkwa/O0q8E4Wun26MBySApUnNBpaGYaN+ags+ShlM55yaxI+UOEAYwllDn+uVxpQV+jA4WhrQFlqua4fgZHxTFgpMe/bZcwO+UZcCx1H/YQ8Njvp/ZtRw3aVO0TdD96d6R7Ui4Iur5LV3sDsr55nsKUBGvOg5gYU72W9Po1ni2Jp1NVSHqTmcpkznHJ7DUa//qwzlLN9/N+YVWYkh4Rkv5uu5JfZv1A7zpMBUb/R3/CycMDKEO9zho6eayny2YBNbmKBL0zMfA0CBvJy+TxO317OjeAf2Pvc09zj+SpcWSDmeuh4Bvh9TkC2UBOo941sM9/+idZZKtotMvCyV5IWeC8vlsOJfif4fMYxFnf6lab+x2FMKhjLeN1xFrdHN3C2vycVYV3atLdKtrqCQ/PKYHtdDNxbATMsEHwPAzxvt3lu6zXHQ+tGx7yvScr7orlWbb7rULWjtDQQ322fc/bQx5zUnncFTjFVsC/qR96MaKDerZ5sk0x9zEsiqyFlMQw7SP88BU3NBjpS23ESNoEuucn8VS8oRc4NfBdF6ktQcRoNpWwrGMZr5p+pdD/K+YBnIXYOTK2GgIHkmMcyInUc2xvg1XI4ZOogMrpM5eCwMTgfGu1CL3RPg4R7xovi2sX5xJqEQ7HGcJY9Xfz5uDZfZAgfFs6XR40P8Indl74Hh/D8nw8yw7QAbv0qdOjyNvGM4UHKnJn2V2t1lDBUtEngIEicT17gEox2BYar8Ljt/85L/5DRwoFDJ4nySGeEvi19kaH+EvLmAOK16+i591fmOjOCDlgTof86QS1Ydoy1kaV4aERjuzc69bK8OsDwwxiVfnTXAJKderM3S29dhg5viBt5dXY9q9gKvZvpPsuOuz5vDV7K75jOQz4JUHEa8jbRwXcfD9j7MaruK9xMCW36uDz6W57cltGGYsV13ViKlz2vTTvcnT0lnru5AfRZkOE2Tej8msqxeXdjftr76G6a6LvzHnpVuYn9MXAYXHyZU+4D+KKqiaKQIvqbK3j36qPYnEFPbq3AfUd7toTAcD0UBn8OvssgbApVnX6m3h7G91mTXYdTgJPqRPGfxnzaeZ1mXNRa6jxEpqFB6cH5gKkQNYtOtX+zLQRqPcSPF1XJAuBTtBv29medx990VrfKAmou7Z+H4JGMMojPy2xw1aRAUuqFs3LQFjBX4Vn4GV6DtvL83C0s73gCg/GkYF849Sjd83/iTKQd95JALmf05dPja6hN3gn1WXinP8KbvZ5hQMAhSrXHW2g8W4Ge8nx/ZUt8AKmRs7mztA72Dbx2iZsPNaGMexLC70MhyzQYGsiMzcOrOB2amiB9sfhht894U7WdRzwA7OTXtSfE/HvLjZV6LpvaHtbzLa0aXpI56GjPGWPzGBG0T8Q/Q0PHn6hsCsH/yiUmlVzDYGrXMr5y/kCBg5k5GnwO9iAmy49+Nc51ThcEPt1xV1jbaG/fSePZNrPPGey7/DbkbURlNxLq2UT7a8l4/O7Hucq1wg6OmA4JL3Arei4HTDGc7VVOj1Ef081vE0Q+AEV70Jz7F785mSY33LuBqsR0CjxHCN1j2tJqOUz8t2gWGxtq+DqqiMB7zlEX9ywHKj4UTBTdvoThh6m2tsNSZmfYvk6EFoQK21DWijHq1weDJA7mtfpUCvU7RMbzru6Q8ycVjRV8dekdYq6k8sDa6fzWaQP6fe0EdSng5aijJimdLRO2kBWbJWzW4gPwVxCyZKPTmUc4UhPN/vE9eS/e2fDdl0DKEj5PUTM0+nfMikrMqnKsDmf4zVyNW+brPODoRUC1QN5Ljrb9dFK/nAWZHijXTiDot+m8XP2E0HmrOAt1WXxfa+fHs4IyeFsDmDTR0PFt4WCzNrgyrpp7WmdJF1T89dng24NXa/q4KEYf25JL5bAaiJxJtbo/QyLXcL9TQ+5Qp3jhcDdVQuUFvOR0kj0K8HEOoxhVrQjMd3oXYh7j4Zh3SQt73fVk1zlPoXPR+QMobAa0Dp8277zZcz1Dr3qQHZ5LSEoakfJtCJsixlD1FfrUHOAhd2GTTmr/JdnGewRIUe0JxfuYpzhHiAJIgWN2+7/VQWoup4FDVnhuwDVWX1nArL+rqK3u3aruok8kCTKqO5JmsWFV1Ak7erUkKMkvvUZTQiXRTszAqjpoTBQZe/RbQ5C9kmeMj6IxB6OW4DHvk7B/KNRnQeV5bBVnyWYGlV27cLGsNyeKHhLUaPa7s8/8FQ68VNmCeSZoGLJs54FO7+EmX8SsqMSKCS6+BLnr0BSuJEApooQO7t6z79IBuuM8r5AVJOV/wbfhta7PlIq2FHoOBy6Gntbn+bskFhxtg33HUjbQJRfMrTTYFbIMUTNZcvs2Z2pDgX/v3IVm+j7x2wmF38HOZNxODeKJlOdwUU42B2FNFah2duN1p5nnCvb9QzDQdc0hY1OHCtrmDm+ALog4tYReEtSOX5eE42c7JDSq12rR22rJck7v9irIb/KnMWAW2G0EVi8hK+k6XTXN4Mm2TnpJgriit0jOXoVnowhst9bsc+ljy2LDUEpKqL8Ft9eAdzeeK1Fw2b+YSt9KvgmuEIASj/aQvAhd0SYmB+9g5M4wnvjpCaJ13cRNc1ZDyUG+bPKntCWGw5qQTXDmCZFNm/xFW5B3cz9WpSJl/US3oN30D1+H7DC16RtGnaakw0H63DzA7a4P8cvsX1rsIKUOdKHMVi1mrjMR0OSARbVTBHjIuWnW2Aup015rRdEoQ+aPQsc8fws3YvPod2QgSquSx4tHwuRCGLITJBmD2oB3gwdqswOP9tv5dfgY3LbKgjXi7DNEnJpKilsxapMafYMe6ThEhYdz7KOP+LgqjQ1NVfxQLvwqyq/A96+tYCxzjfunwk8yzU0AjY4px1E5Lgs8E8kwVnDI3sSZPpsJcptDUmN/QUlurYfN4XwWeovBN48wPHgqALXV4oYd/Q/z/pCRDNC1tZ/qLT5orDmQ+nrLnDOZMJc0orAqkFGgV+mZ3H4ygfogFFovSiOSqQoXfaaUlYR7hhOh7orGGiiCfQ4/YkteZkpoi//Akwj0pmgcdgVmRRVmdQlNlibXPZrnhcW5JiloGZtYG/jGqMavzB9Do4Gogrlg83UF+27UB9F42Z37/7ifgUcGIjlkFHo/cTb2TARJSeuYRrimEM49B24xmIfsdbEUWB3NWV0ta5CLXag5668V+1jr622AOGovtFXn6KMVTBAOFz1tq7XApUfvHNetbCaFpGBHCDjiIE7bwN7OoZxtFNnRNBVB4GB+tMe26ce2OtF3hghkTARAl4+xtn+bcI/rXOu1gkd9L+BXN4RHI99nQOQA2D+M+6Ke5WzhOD7Ib8fe2JFYnIFRWZJhpoNlVQ6e7riIvaHwQDFkmYthWzwcGgfGUgL02ciSTWTkG05ishnBtweMFDtlmEdaS3s0z9fgUdD1UxxDD7jAQ31DtzMo8k9AEnT8Zcc45zmE7U4fSK3+omhvzw4wtYbztk/xyllMD10z0E6i2Fv4uerdhrG6uD1/+iayz3sqd5bWkjEOZ/D1TvCH5FBQaYc3K8AgW2BLDHh3pbb/JtoXpyOf/IinvnuKyyaEPMr1T2GjPzZZy3kTVLgfJC90IXOS59OxbJTIjo97UjD2ANFeqZiV5ZhV5a3ABy37h9oSwJTCdIIUE6g2BrrOhjXGGs4WnOVy6WUAyvUNyD7VXOoTS1G7/nj1TaLL5oVEpv591zsD/0gx+09//79a2sQw/gfKf4J9/yn/K4osywQFBQlEXvwzEHk/DyS96kwPF6X1HNIoNQRVTySgRqC27zwcZClHMG1HEh1XT8XQYKBechfGm0ILqW8x0W0oMd6XsSlsOCQnEthug/3D4NpHlBsFNVtE2RME1owV6O72L0LMo1CdysKeWdx67RtKI8L4ZnRnQgqfg0FbyTc8yyvdFnLByUAa2jQKH52PQMGmLKbRHkRa6Jtkhr6DpwzaZk7q8MnYWgW5APQyWCMehZB7YHIhX/n/yXUzaCzB+NcN51jNQqjPFE5YwHDrPl7rMI72+z4lIPu0aC+bUdA5Wev52x5O2bAjAJxsvES1I0nQuwzahs0r2fVcrc9eXu+wnG+2NxL5SAZnv/qUL+rT+anawQcl0O+4hax3L7H20584YrOx12y8QysJjtrhs+XpfLVoEZIEo/auY2qxOBDZFF5giBKHxIqzPOxhZ8yu0cRcPoOfLg9OPAQXX4Ybi+hXtY0BTudooRVKbV4iaNaYL9D4nh3oFOLUMLwSxaJLS+HM41B3EySZOZYDvOM8w/8Re43OXuuh6gL4pHCpdCzPF/i6qMZkSYaC7bCjEw6VLwuKvTEpq1zZYK7xl7uBsbbPCGh17rjTaGrWNejvUcZbAyehbbwgHIa7kpEdNtRGDe1vtOfefWncW3pa0EBF3g9qL/oXrWDp9JN8+sqn+M3/hfymAJTWUjg/T+gAWBsINlxHaUhz0X01b8iSBP/q+QT/6jy5pc4bfFyO9hpLGOkVfWiyuFHhdphfb37AycvfQkMuWYV/UzRyPmunb8PT+BRlpW9C2ESY1gj3VfK+cRPnbXUEF+oIKA3gRfeVcP1jSP4Cz+P3cSoskcS8z2myerDy8oeYA+4VgbGk15ElGaPOyK2YW3hkexOrzCHG/hPk/CEC6F6dKHQMYvELq+j80BY2DJ9I9ywdlJ/GIvvSxf8kA6wzSMr/3KXLZtOHwqjTXGkStFR2OyjsenppH2Jmpxa6UwXOzD7ZjMrmTZA+FK1SDCq7VzeK0gOIvl7K/eumM1UyQsl+UOrBuysPVg3g6qrRBBUHcanbJYzooSEPjs/Ebq7ms4JsFs9bzE9zfuLnx37GrxgYfV60m6xgftNjrP/LyuhN4iSpkBVwaBwBV4YwPmEplXFmurfS+rJ5JMCk2xy0DGXLpC2c7nOO4EvPcsxN9CcZ38HmcLZFnSQ+qhmNpkQjO2kmGm6jqr7qOg40t1XrQ7J79tt8ObYPpWFKjvc9zrCCgSK4iQNS30AhgfdvUwlLP8KFbiE8mPK0+OHNb7D49OS5SjtnUy5Q7leO5F3D9bFXhNMPMNjSqep2lgVZnTm1dwlzHBt4qexJkFVIEkxMWMq6YEEBaZcsbGuAawGTQaGj3J5F+2sGulwM55YFHi3qL2jZbq0AUyWjg17hauJRrnUs42DKExwp2ul6pzifOJ6vsTE8VazbkkMW1FOBw0EXLALohcEMOjSIwmJv6qw6JOwii9zWyC+6Yez0tZPZvpEe8d+zpLcExx+A2+t4ynMABdHiOV4yXDYGCA1WgO5Luf9GCz2VmHP/ZOz9sxbNdbflvPzbbiJSd5J07CdnH38vnHeFO7hiDCH/WgxP/vgkSbdiUWBzBe5xb8dZs6IlpOGQ8azdDEfvExkGwNtBZwmteJBaO2ws7Ckyqh0OKhX922bVNNfNmcVrxpu0Rj9KbPBZFTgkWdCaenaAQVswnC2l7/XzKIGRBgcKU4Fw9vT+FTdbOX2cY9qsKsdCPSTOh5jHAMhSDCAgzgngiDZh1w2AKaUw8iR0ehdZkvn2mW9Z9uzfeNvfI1g6AOWnRfYMcKhkEwU+q//bTvr6hhqWnP8aXRbU2cHXnkTnwM7I+mCIe5oKa2dApp0K9ofCBJ2Tl3ijP1z7kK/P/kFD4AMUx/TB3orWCmBUzSf8rO9OTPHLZPv+Sm+/l+DETOHQvrkEENTWV83waRU8e+kE3FsumAaK9xFrzqaTGrzr+5JT6zyc12fD0Sk8kjib9waPRa07RI3uEo32aqHruCkUuyaYkoYYjNRwOKkTh5M6u6iR2J7E+NphrV0hd4NS7rCbvCyZgiJb6U592FS2+g5hx+gdGA31bKt2FywDx+6DG19w3a07B7L9ePe9dxm54SuO3HydW8k1Iit1gMiCKLDCkSZI1sDRhNPIlWfw7vQYDtmNX25NZH0rZvIKhQ8MPwxhk+gVvI73E7bho4CI0ifZP7yG7sPXQvgkvC35DNXBz8+u5qM3v+Wo5C4o7LSBoHTDTbLgIYNZVUKD+hZN9nLY1UMAB1Lf4AF3sSZqJfCVQXK0cj779iQuR6bxyyFc/Hom9TYVNqUzapSzCpPKh3xdA1vuvcBO5dscy52Ox4XRANjVgQwN20lHtXDceDb0oItfT1egQ5bBJhtFBq42k8vGbS7dWdE3rRwJONAqtUhdPoD2zxNVt5P1EVYaAgpQGcsJ9JkCnRYKwJQ2gB+sw9hYD7f9f+B62CtcUtwv7CsAcxUji5Qu50iUEooafoLxGaKvIqeznKGuvpBaOcMlCTSKBtxU5VQqL1Dmvo/SpgJxcbqRxbYX2WCycCW5jJzO40HjRPYHDYewiXzkf4wurXXR7lgb5TZQFwlJqRXUj53fJyf8UaJzoFxnxqz3FjiJ3PUi46LDa7x/5H0OFTxMRLaNQqWTKDFnJRy6B1XOb8z0EM+62ukqX0//k+MebwrKTGdfrEj5Sujqafhv0SzqDF4My5XYV+JGdmkHTtS8ATs7w0lBieyuKEDlI3GqTzqlAaUi8KBQQ8xDMPIEXXO9MLYCaGC3CD1TmxHz7fVMzf6K/sf743k1kTdv3iO+2JgHWT8zp34lqzpWktollXr3enFvpR7c47DgQZecX+hQ8xJq+Y5Aty6EU1UjuaK8Rq1BaCy6AiKnZ6PP+YRxGl+S8hY569V2XahVduSzeitmlZmz3c+KdeTwRJFh6XzvZqrMlXVwK/miAO1t9IV1buzX7heBL8T4MhjPCE3T6ivOuojnpYe+Rb72ksh66beKfMM8ZnT4iNdbxeEkJCjeC7tSaCSUrideZF+jcHZsjNws6qMPBXMVUz0a27yHa//tvJCmsDkEOes0/HI+YxrWtfluvSqBK7VqYm/F8me2HwusXwr96aoLcPkd1oQ8y9OlUOD7B+3j3iNCc1Bkt1dehKLdjJBv4a0A3MGrd3/GKRX/NuB3GhirUvLJK1NwhM/DZNNTb/Jq43iWJRlKDjHJfRCbMh7jyyrYEGok0vi8+ELOKqhN4686jYvKMMjhhc/eIMHQ4ZOCvvQAw9UBhFY+SIQSHvM5BSUHBWNNwgsobv3CG7636eZ7GTtWxkV/JPSDL74CRXsYk/EaV5xn2+WVnjTYAuC2oK27WDKGMefm8FTDTPZ08+VI3XLo+BZo/FGXbOR07x/QSS0AtH/KHrirn5rf3TkcbzQdatsezt/e9vuR4/JH1JprgLY0nnfr/bUN9jnUnlgdAFbAzgQDRGe+Ag25bdZk+79x7hZ4ryHT9xua1AL0le2eDNGzMMUvYF/WY6554XLH2a2AwzXDmgN6rfdiV/aMZGGKG1yJsqEy3hRBbP/+cHst1yMs3N8gQJ7lVjU73Jtg8C6IegCHrMHbebtRBujpk47FvRscGkNYyTyKqrszovQQvnVDXODJ1u3pahvn2bd1Zl+zvprkUOJbO5RuvgMEaBjAVMGyahVBZ5KZtnYaXW3Om8lqiJuLrefvHCh4lKzke7kyeC69PKaI690+h3vOt2nbjmpEEL8ZpWIz08N2nZ530p0X7oQzTzCry1u82n86Kqm+7XtofAAHT3d/mokeDYy4VMzggh3OF7SD3YJWMroy391lkCUElabzJlubXuNwxxZqY5+SVWL9Ov8sqL1ZVqWjMOkGt5ISSVStEaCUa58IEEfeJiKjrrBmViZ1PcTaa/cSQVQyvoOmItQKIy989SIDPu+Pz36JIzYbu4x3+1WO2WHVmmq++mqJqy7jzj3JM6VQ4r2ZB090ZM3VNQAUqboyvhBUO1II+v1zXuxyP7EZPUQGOtDT7zc8NOWtpErEuxrU1ZQ1hJNlaWvjSsjUekygImgcKWv9OB3dmdi1n1D10AXC88Jdga+lY5aybdp+NFotuZ3HU5DkDIhJCp7v/TwfhF0kpuQFJ1Uid5W56jMMvXKLYEUHUqMeY3tiEL+nCvCSK9iHFYtTf1UptTIwKk7RsfdVpFf3c2ziLFJC+hGuL3QF+xSSg1vxNwGIy4jjcf8ClOkfOMeXkQRFngsoAeCvroCb34C1HmXQUAaW76TXzd0EWnvjXzMSf7UAnYrlRrynzRmU+3c0ng5aBfssdeiLdwjGLsn+j7qgzfPNRRvZiv1hWodpjBaqJ3hrKjCpC7EgAqOcexYuvswWU6SK0gABAABJREFUR8Rd/eiq153gK4ckZBOSXsHeYQEmm56zDe7kWSGgeiyzIt8SmpDGEvTKSi4Vj+DDmiaKtNfa3rP4AL10b5NX3ZWi4okAbG3wg5Cxwr9SdZG5+wqxOVScTBjC8cS+lBudVphvD4qGN7Hm5myszgCiq84BAyDpFaSs5Tji4BEPqDT6cSJvCgw7KIABGd/dAf51gmBlBag80CpqCXG/2RJIdMjICgkS5lHndS8myUy1Mo0a5U0q3I5yqeJEGx225v2gff6nPFVmZsGQBS3XJOiZsRP/GmG3mR0KAVSVZCQkLHIdGpOaoJIgDl4PQzaXQ94G8O6CRSF8OGZVGalBH5DR6ESDVKVCTRqkfMV3peVsz5iLh2zjVhSE5wt7TZbh/Z4v8o0/NINLDjb+wodHtxBrWwZXP+R43nF6Lu/pWh+mx5/hnecWUxjtR3VwIt4DO1ETnECTR+A/BvD+h6To/teUNjGM/4nn/4889T/lP+W/WWw2G1lZWdhsNkHBKKvpH7YKi/Ym6SHvcSvga4H8cJbmxUht9adP2mGm1O9tc79w+3kSVKBu0jJKoWaB+itBbzDmMvT+BR/5ItpeeXz3zHdU+FWIw4HDJqglU99kfYAwGJsN8XDjMhGAKtwGwaMYeMuH4pOdca8s50zBePQNx6D6Mk2qRA7kj+GzKtBYAhlVtgtfva+rXqHaU2xP+Z6XvaE6Fn7y2Qdnn4ZuX+Lw68NkjZ/ruw97NaK++YHr7+YDTDO6rNzaSRyWa4XxEmY8waUjXVE1KIit2EFY5ctwYBhETIMuH7LUnsTKzudY8fAKKv2qSdZ/KdCggYPRVJ1hkR/oJSi0wfjfEvlonYIjNju77zA611mFWbPT7vg/aiWtsMLLL73E4g/iSN0wB+034HfxEkcqA2DwNqfmzPv8HGxG9ozBoijgjd6JwmFz4wu4OJ8eNbtdxowdWFD1GIw6JRzau1Kg5ir+vXPYM2kzF5Iv4Ksrhtp0gaIOn0JnWw79naDil2/Hs690AcTPBUsdhQ1J/FTpTYG1eUxJkC2MyoaQp/i8zA9bq/YGhPbi+Xl0sO/GUwYcEr1u7maQfm6b9282orZJV3kyO4FCORBwQPUVlHnrGdbkxf1r76fTsTWMVL4qKDzrMuHQGBQqD0wqBU36Jt69Oo/Pzn5FtY9wlFN7A2XlGW5EwVQ3sMa8wLg+w5Crz4vnyrD21hS+Ko7nUtQj7LG/RrqhMwQOdtbLjlI2IUs2yjz28mvG2/S5Ohcuv03H8pUc6FaJGnf6nRpCcvW/xDOVOlB782fFIo51fZut43ZQGFLIFuMwiH8OrA1IoWOx2WcRUf44ZR57qXA7As0B9MwfUJQeZpk/9C4Oo+aimic3pdEgtwOf7iLQ2eF1/s5dRXR2CM9rdATpnYZf8T6qdGOZur6B8oLVxJS86OoPnaUKMpfjp7pBleEUV/3epcL9MI96/cbyCctb5k0rd7dDsoq1Y09fuPgykgRFcQMpiPQmOiuWkTe6CtqrnclQn80tuxdeBSGEFoQ620+C049BxSk01ZcAqPaqpdfpXsz+ZTZRllGQswZOPgLGcm7UHaf9dS8i8sLwrxmNryISYh6mMXgO2VWdWF6tpKzV8Go+zCSrrnAhHKKsdjxqtDRK+eILZSfBUsu3Fe04FZrLpkfCSe/3WItxdn4euv396Occ8x1yl5KSuZH2Ghn+CoLrn2E0lWGpbaTWK5+9I/fSqJcF5WCHtyDpde43dmbz6B3E3orlodUz6e7bYsirGm+jsCoIKAmgKLiI8g5pbK1XubLRbLIHW6p8caycwpgNI7hlgUxrJEgSsgzbbv6LqUU455WFvxvgWrCg/7nX7RQXHlzLlU5X6H5wEBeMAdDtMxi0FVIWM+p2Iof6LWL9fcsoCjhApanUVS9JkpAlGZXVi+SsNYxqWCVQgoO3QtBwEkq38FJ9MEMODeH0J2/xw7GdKCqPwN5+kLaIxybvo1vaMeTaP+gY5aQ8KT0Ix6djR8kaZyLMy94wzTMNcv6E049D7jrqbC18/nCno8pBk6oAo6oItcUfrd2/zXd1lnMMH/ADRh8ZazcToaYXwalHQNZy3it9guxLIjPsA2+JkL2+Ail9ZBLkbUYrtzxLQkJjvikokczV0FRMpGILweo6im3wzo0ZgnrIrw/IWt7uMZ8twc7fOmSkVrRFCsmMl8LicoAM1NwQ/wmdgOzfD63dD5XVm7jWzG/eXUAXSoTpdBu66TPxywR60U9kKezSfYxPsZHPXvuDnWHfcNv/Z9D6i+tu0XSz5dPNzYqPcTI7B73ICMUw2NNbIKH3DiC+YDo6bfF/20nvrW1qk7HykPEcqU+l4ubbFXp8S4F1EL19r9JDC13UKqboy+DsMxA8EqIf5lj+A9z0S2HPyD2MHtAfxXqlyIa9soA67xlsuf425R57uBb1BFY5F5Cg9CjkrCK/1xoWlwn6w0YHNNgDwFIntFoLd/FQxUre8gH3po48uz2TZVUOCB4BfVexPmsRi05/z0b96xzt0I0rDftFYMstBqWtktL4qWyO96VOf5U6/ZUWJ75J8BGlZG7Gp26gc4z8MyglUAH36KFz/QpxIeFZlHU3edsH6gecYdErX/CGGqHDBdD3TyyyDrPajIRE+8s1tFPJeJT/JvZfZ+bYhlv/oqqyP/lWWFqchM07haysLMRu3vZEOadpj6Ap2zeIQ/lP8FjqI2JtlNoGLk4FzkOfBZhDGZlawpTbedCQI2grE57jnrpZHDNCZvDHHOwcy/6azwX458p74D+AtypERs8rbn6UxVlQ1F9publSR45VxqfCj9qr8Tx/ow8WbazQlh28kxvB07n3hgf9DsXjUZ4NgDH6Jej2BfWxnzD8rzQWVgqboc/N/Wwee5pQD7F3CMexaP8a/QWWVY1n7o4Wu0GSoE/6QQZfScPN2L7NO3tacxhrgItjd/HNewt5vt0uyNsobKC/26EIfgdjxhJscgMdYj/Hx3pMBE4TX4LgUbw76F2SqxeK8aCFxf4NQm/S9ezW48LpRL3+OYarc3iw89v8MdmfIt83OZ0wgoOFW8XXFBockgab0sbJwQWURyQjtVqP8O3F7zWdKLYJrb8hVzIZ4tcWGNEacCA5ZGEzeHcBQwR+el86Xb+P/sc64p1ymVH+wyFvs8hotDYyz7if+91AYVOwolpmVf6clvsO2YPyfhM9bq9BaVGiMqv4vaYlm1KS4GJ1O3Y3CprFFW39mv+2/K7T0XXoJE5V6pj5+TMk7/4Em8MCMbPFfMv8kYeD4pjaYxS9TyYQlxF3V4Cz9RyUkIVG68RsiH0UpcOMoUls3gcHH2RdcTdMo7LEl/P/5oK2O3+cacc7779Dp8udhJ3p1xtGHKXS0Y1eoVvwV1UyIA9m5Q0RvyvaC1UXmV/Yh188v3A92+Xo00fikHWsvfo2uAITbeuscdTg49nIJ29+hPtIGw+73Rba1wAd3sDu+Axz/EOt7i0BMsQ+Dn59OeMIaKUPJVHnNlLQMvv1hdqbPGu4TB+tAIcEBa1HXboFig8gy7Do1G881sKyJSj0vDpBl4+ooQP+tSNR2HUtNc5dD5Za6LGMoJttHZ0uQ6nkEN6XZzCr7Ev6Xz+LwubGnUXvqKRLeDl/Pf4nXpp1jApw2uDdl0K7J6jWhpPjPDuMKTRz3fgQrHeHXcmgD+Gv6L/JbBBRSn1SV+576HUGKhSM0mpZBewD/gAGKKGfBPc9PZIXPt5IhW4iXtpi5Kj5ZEXMa9tf+4cQqjyCXtmAVobxBkhvPAeezkCEsYwXCvrSVNeFwVfSWWp7HNlaDYZIAbCclMemm2+gtnrT1BTOqqohMGSPyFCPaem/zQ0waWgMY6I+FXTB6Yvh1gpK3Tqy3xk/jS9ciFquFxkkfyp4oMMbpBndqHMt15IAuVpqkKx1bM56kG6ZG7ntuYrj7ftid7QYvv9VZp8sQ6P6NvvrWphsmgFssgy3gr7kiOpN17U2wb7/gsYzKX0hde0gSmPEIdnoqgGfsr+g+ipdPVYQoRG+gH/S1pMkyAz+iMuhz1GnvQ7AVb8x0OUDVhi9ecdnLpej5uDW1B43k9DuQxcISa8xKeEHhqXmo7R5trRXc51d2TytwM44oKlA2EHZv1ER9DqVNcn8yxO2xaeipgpCRkHfP0jO+5FT4eJ3u6q9eeHC89gMiRA4mEqvR3hz30kqawehcGhc56nms4ckiax4u2QmsHoSL3gdZcHgBa3qpkSyK9GZI+lzcz/L+m2Dk86xY65AQmbC1gkk3Uji1zIfEfSXZFCoOaqI5m/tFq51uIbJza+lH9yioT6L37xKSFIKe9ogQ4kVwR5z63coPczD5i08r3Hujc17R+R0GLKblZc/5amb/fgh1I9t3SVOlzt9QrXpaJquMCbue5L8zqKxBqJ1OH0yNddgWzzn7YP4qhp0piherF7Itz5LBejJlSkq+kNjDmZW7b8IyXqpZQDpQni1zJMfp60lr2M73BTF4gyY+jrUZ2M6N49kzf2knHRQvmUYE0++hXnoOaF3OvYa34e9wQ+qnzjX/iwH2Mv2/8KvstUKr739MaWlwvpVZS1jfL6DoKopdFBDTP1lsDaIce+AzmdjADhalYzKkie0i0eeZH3ezzRaPJkX/w3p/0onpHIGAKEe6fgb8qi0tKUU91JX4NZ0DJNbPFWmCrC3sAPUeNa4Ms5e2vMSPVZ04JbbLzgcDqxO9o3m8XW+cRP1mgxsNjA66qgynOF201XIXgW7uhOoTgXAZgPuYEdwyTVIVqz/lNlXegS11kJp4Y+4OcL4bER/ZnZ6zxXs+7b3q5zpV8SyZ5axfWIqM3yqUF57W5yN0xbzhupn5nq13C69savI0ox6ABmINI/Ev3Yk3eveo1fGbrq7CxB1m8y+f8gAlmVIyv+CMSyjY9epFLg5J2fZMSzuiayua2YtcmYWS3evX/9E4+mp9SQiLwrdTRXHGkU7uNansMmQ9NrdQPR/yBpsdZVY5Vr4U4FcuInShiimZnZmVZ0A6rvm69ir/Jq+SvzfOQ9Da+7lvUHvieSGk7Porv2A1VcW8E36LH4IgMlu5aB3vrd7HFujY9nVzcOVISxLsrChSg6w8/oy9nTz40b4K853brVgZ69EShXZ+vlWWBp6H29dXCACgYO2QuLLzCz8ms9crlThe8DaAJUX6Kd8jB/GJSDZy13XdeZrkP41wfnP4KupBsAkV3Gy/UBm7u/XovkrtQ622lFKKte4br7uVzeUHjUzOOwfTKIuXIC/L8xHf2gUL3lB6qCj2GQbploD9UkfQ/dlMOwAE4p/4lBod9wbO2MDHrk2idTQyyJbe3sinH2G1U0T2J3YFZuiiUobOJwBQlmGMEMukcqWfmruq0jHKsj47q5xMM4Az3u02nO0KrJ7Tqc0uu3qcyeN55179X9oPEVpE8P4Hyj/Cfb9p/yvKXV1raiFenxL14uz2NGhIxkhC7ge8TwmW5Prsh0b+T5/UOizBu+G3oTbB7W5V7xtNy8OvY7arMbncB+U2CBtsRC4DpvIgIIRvGr42PV9WXaib++rhLCJLodZuftB6jXp6GzCsYMhBjS+3C7x456/phCU/jvPl+ip8X0EPDshSbA7dyLvV95BI3XrNzgyCTdFCZIkMgvWFnVFrQgUgu7GEtxvfsWa0HoGXBeBm4/KPLAGTxG6Z38FMqv8BcKVUKu7yG2/H9jXqIWpdRAt0H073J7it7gr7Bq1i0EvfYVn3WpB3+IMIDYfgCTvGsp6XmOA4SVh7NrN6CpO8oI36CSorYGde9PZZrX9o9G5HBgC/0etpK+AycAAh4MPyjPZ0FTBt+XQ7hA8Oes+vvp0odBQVHvz2ZntmO3eeFW78Vu1Eto9JYKyAzbyd+iLLKsBr/pejLiWSi/dvYJGJ3oW+PWlxn840xsKOdH1Er1iSwjU5Qvh6yhhuL7s9hQjnUD0FeURHK54B26vg/UexHidvxvlZGsApTtWrz4u4+1E+wEc6Oikr2oqgKZCLigfINMCenMU/rUjCVUltXn/5sPBLQv8aE2nQeEDw49B/L+QTBVsSKqlIfkyjkAtg+OcFHZhEyHyfmJHbONyUwX/Wr6JIUem4dXYg7KQjwQKv+f34BbDwgq4aAJFyI88GXUARa0wkCUJtpcn8YHlIPl+v3HE/imzSxWQJIymgaE/s2m6FnVyLzJDPsTigB26vhB+L/lew5hfBhG3gghNP8jCTmECeXhiFqR9TUdVIw6/KrLaS2ikAaTxAFSehXVuYDdzwfQ6DZoMGjrewwdDRqIt3yyu3/wGqfoqszVqgsr9uZKio9jvGkXqGXDPWfDuyvab21l19R4mbh7CQzs7MP/E15hVkVB6iJh6ocPXvIc2B+ANTblw5nFGej5GleE0NwIXUuy98S5jQ4kGz4YUvOp7cz3sJd4/+zzWxiIwV2G6/AQrHvRi56w3+PCND7ntbQZjiQj6H5+Bh2xl9eM/U+tRy9QdY3he7URUBgxEtlRTHA2jL6TQ+UpnagJKecnQA07MEHQStkYma7PJG7OX1J4GemXsoLNuDLR/gYbIV7lUPIKnirVthLXVlhq49RsRcgHWgkBM/uV88+w3NLk5v9RnBdxvZklFIpfNUOS1kdSoxzhWI9DVeHcT/ewM9vnVDiO4egpe+gBwjwONL8sdHVn4e1eG/2WgQ+lcxmqSoGCroF4LuQdZkrnc5TLXE6+T71fGiqwpWHuuhtGXUNff5HetO8989wwpF1K4qjDy+PVgF1rUqgpm8UFBT2hys6CVQOs8sEkSpFf0ZWOdjL1VP8qSDDVpPOaxhw6hVYQWhNL3RF+xBnt3FevjiQe5ZvIguDCYlz97mR5netyNSpRA4dARUjWd9ta2TuWAusvM7H8DBw5kuwq1zRt1hnP9j3oAAI9jR5i+bjqfnYni02vn4J4L0H8dm5rWuagd3WSIVVeLrK2s5ZD6Bgn6ijvmfssAtNjN7O8Sxrm4iQy5msGU8nNtvmto/IZZvb/j1hvv0POpj0lxfAMpS2DIbkhZwjD1G+iUL5HRLoOLgSJrEa/OULgDjk6mJtqEn+u8JlER+ArMtAsHcuV5hqkm0d+zyNk+ElLgYIh7GrusRatows3VhLKgzwYYeZJYzRau9FzFI+5wKhw8pCaxLgcOhrNz2XivjLemjLHOOGddwhIIGgG+PdkQ/Cd/tNrKc81e4j83PgdLHQkeKQy7dIuBl3cj2YSWI9VXoPwM2Ew83bSHORd7MWKzjcs5/Wl0hArNg4ipYKljtF6MK6GF1NYh8e/K72qJkUO6Eu6ezii9yCyVpLZQRQUmvuv+CfdJSQw5+A3Vqv5i/evyEbSfhyyDVa7jVtCXaD1PIzlsIsvoynuYdUmcKZiAAwcxKujtliYOn31XwuBdhF14kg0hVroodQQqoI/33yIjf/RFaP8CO70n8mNNi80gywh0eNRMLlXcy0LNcm4prjt7WRKi7iOO4pMzj9MpG9DILQcM17zo8ztX/HfjVz2R2OJXCC+bQ4yif5t3Dpd6M/aclUer3mBnKJg8J4l7e3ZAWbyXhb7QvTiEV43+lCZmirXRuytE3c/Est/4MaGWtKQahs3cx0/j2xGY/Qhsay+yykYcJ6vgTSrdj5FrhQUVeoqtMSz68kt++qoDVVseIeY7XFo0kVanRz96FsWNCZysjBd6ptiRHUZXdl6zI+SFz8fRfdsC3u3dC47fL8AKgUPuWhcsaF36OJQc4PmQcsadc0DaSnZmPOmidgTAZiJKJXE5RdhgCrvTMRzziGttBFDJTXRLeIN2Yb9iSvocEucjKyQcDjUOWvqx9V4kqHaanVX/TAnnZkrAzZRAatSjzDv8IKZjD0Dqm2QGzsf/QgQzF7xB322jmXS7C8Q8LH5oKqO7/0CiS5/Fq6EX3jIEUCjWJ3M1yGre8jLTvV5QFR1tgrllnpCxFE4/AbXpPM1eHnQXtzMYnbZOyX7URWu5Ud6XX8tCSNfsahlfDpEVPVg+QGk0DNAd5EJIO7Z7COcXpccgfwu7G7pT0hiE3hSDwRSLQelJ6yLdmdmHA8xVYG2gs9rMw+qRFEUpcGu3kw66I2Jshk+B88/T3lZEfFgFDYYGDA0GNhfPFPomE7JFoFxWAQ7e+vAt3vzoTd5qSqX41iU+eP99Dm9P5JUPz/Hq215gEvSJ/1c0i2Yzw8bcj0XVsmlPDg2FTu8Adrj8Ln/VdmSHqoSDgw+SE5Uj5quxXOji7h/G6z6xeFgDXO/cuhhDJ9OjqZIlzy5hQNxk1iYcR646LdbeXss54zaYHdXCqefu1kQ3v2Oij4FAxUneGjiJTu6ZHDPCRaMTWHJzKZx96q4AiisImbKIqpGNlNtkjOo8Z63afndA/X3kRcN8b5ipimaC5ldIek3YpOFTmOv1KjNjRWbcQh9IOOEJe/tCrx9h5HE+sPemopkGChm7OlDQMmv9oDGP+/UZJCsFqm9tyre4XZgEB4YR0vQLGRW9iSm9o3c8k6DD69gkN0a7N9C38llswJGGCLE+nX0GNoXQmiRdYw5C1eyYa8zD4jeSkqLpeDV2R0Z5l904pWIwe0Ilpvw0gw5Zq/C17BVnyLLjEDIard2Iu7OZ6u0Kuhu+FH+0nw+Bw+kZPp7QChFgdjgcjJvyHrfz8xnwxpvMd0vkPoOWZ/zg2ABY9zl832sH5P2FLMOYdt+xsccilCG/4mPpyNwec9sErSI8bjHOAKpMWFSfAmOvwMhT0GMZM8wHGHj9Em6meELd03GovMTevEaN8eZ35Ktu4F8zmm6XcsmvPyDmSuUFqLyII3Q8X1SJjGyzqoKchnYuynA6vsXV8Dm8WKbGr3Y4UdZojlYvhITnIHQihQ2JeCuMLtpe1/iKnIEx6Ut+PrMSt7qBmJSlVLmdbBMElyQYfCWd2KLX8K8ZTTApbfpCluFgx3ZcN7VoGLYO9rUG8Xg2dEdJC13GXQ5th9zGQelbeRQA79LZOLDzYSWk9U8HWxPj/R8l2V0Yfz51/ZlTd5u/Z/zdpt6yQ8zHZn3s5j3CaDViVpWjsnkx+NoNhtw83/K71Ddo3/ALOkuoy5Hdet9qyeyz8lc9dMnWYTF0hIOjxBdiH6ci7CNuNwbjq4AYTRPetpMCfAQUevfjzzrwru9D3LlKxnlX4nN9GnR4nfywX0V9nfOx2Q5vrS17LnYKO1I0lHpuJ07dnw4BHVx166SewNgLFvqmH3Z9n3ZPQtSDoAvhUqSR2/1Ocb2LguvV3wobp/oKbI6gMut3MoM/ptB7PVa53qV9hrUBam7QS21ldtMRet3cw2kjpOR2EXbfqYch7y8qkjay74agR3cFbd2iIXgkV8uHstpYjdlp1jXZnEbogRF457xExI0IHnWsoFGdjSxZ4cAIAR6JnEEpIkNLZ46gstS5h52eLajpaQFFRZXNpTHr05bB0+UjcItCQiYhLYFO107xXGi0OAMCjLnCxdDZPFHYQGhBKD6VPkg4x59/P/BMYuXlleS4rWW/1376KP99oK+59AL6qFQ89OADnD2UyOl9IexYG0/96etMAe4p+gGMpfjbsvnCHxq8qjmffJ53lMep9XlAtJdfb5bWF7AtMYntBSuI941HbROAzU03XuLZnZcosbZFwGgVJjxqt6BtuAVAlVclN6a8jmZDF6p8qlxt1JwFakPMB5u9JWC37to6vi6ewvHE3tjtUOK4zPHEXrxzbTJUp0LleTq5/SZ+Z8OVddo8NlvTeEpWAyEVM+iqndxSybinMfU9TeMtK5rijSw+9RyHbou+kGU4W5bCpnoISAqic9FxzmY4gZ0KPRgiWWMfy+ZWbBN2h07Y4Xl/wZ8KuvgJoJPVCfJoXkdaB/vcTYkMTy1g171nXPeRJPBu6E2UfSgrgu38FOIE7zqsKOwmQqtmEFXybAtN7D9kJrsYpmi7nr3cfiWYgzCqmsGvznU1Wpxrv5VPuXZzlcXXlSkI/0SrLtHgCBcJAvrQNvdr1GST1ZBKeWO56/tj4pZxKKaEYAW0r3iFdwe/i7fOW0gBALk1HUBy8IQn9NHWQNRM6PkDxM7mXq9yULQcFGVJEv7ZA8PpW/MXka2Odi4wWtrXLmDBvFKhr2lXNDEkbKvI6vMfAD7JaG2NuDXvQ817Q8112JWClnK+KoriivmQ6/1861uy+seH7QNagrat20kA9pplYf69hERIYz9S9A30cBOBdswVKCpO8rk/NCZf5f133qdvz3RUdVcEpbi5BqVvd9yUYwmsGQtAhdlAkzpJAHS1wk98KOIEesNNau3QPQ8qIp9z1Wv2gb+ZUAQWZSWbg7tQae/G8JhfOMZ6GHXqrn3wq/JYvF58lzGrTxN+bRdXH/0S2dToul9L27d9t/8qmPf/arAP7ohh/P9c/hPs+0/5X1Gka+/jXeOkVjjzJKz3QnEXlUfL33aHjUsxs0iNfgSb3JaqBaBCTsBbASUplzgck8G71tcEUuvy2wKJ2OreWlN4201v4GYerBCH8WuRz5IR8gFZHl/ADBskfwl2K542sXDWu9WTa2+iMnQBHJ1MbO3LLVogtELC1N6Agu2UWbow6vzDfFsDb2bcwxHVQZhcJIIDuiB+ufqTC0UiA6rcnwV1lrEUb1shE4sWEVQ9iczop/gqbLTQ43PW/YZ2IKu9ijnV5xRjG+v42/MXGLJLCKxXXuBN6SLPbpjO5F8fZkO1BwcblwtjdVMoqtAxnArKQ182A85Df1nxb43OLcCs/0NffgV8DBwBDkKbrMDdFjhqd/Dx+9/w1aWxkPQql8vHcLNzCl/M/4IF9Uro+Z1A7oZPodiQRIEVNNYA4tQyDxn6wdUFIjvPswOeZfvQX2nPrN9n8WiDPw/EfyLQQRnfQU1aWxSmQ0KnqBYo15jHSPA5xvm4HBdNqCxJMGgbTMzGkPcFe2Nv4+H8uUlVBE0lgoJl4GbSlJNwAPZ/0J8Qfdd2Y1XIMvj1hO7fQMRUXm5sYvt9f/Hdg9uZWTFD6JIV7XbRoTScvIxffioR13czLPrXtk5D9zjerYRzJkGRlnhLjSNsirh2x8FXPLulLqXGOLYX9eCmRhw8LcAht34QNoFyj77sONmelLMJ1Hbx5RZdBRIr5w+48DzrAs4zTA/RpfPokfk3PT0mufSoHO5x5FrPUeV2Em8ZknVmFJYyMd+8k5HaPU5slgLN9Xi8FMeIa/+aMJQLd0PlBQLL9zMtXgTUPW7GcqasD2rLbQBsCh9SQrZi0W2lynCSyMrZzOs1D0NgPwDcFIUutJVdsmB2NGK2tRhqSoeBATfO0T/tJPm+K/j1xtdkD9gDvZZTp/TmRI0CRaMSWWEjQWMUuis9voOwCezy3sl7PuBR60FQkxsJssiipf187G4xpFug11ZhnHmWBjAr6D1xPfw+0IfxrD6V8ROPc7OD1tU/rf/F0XaN0zXlw6lHcMh+vL1zMIlrJ+JR69GCbJNkJyWmgvDccJ5Z3JOItJtkNDqdYQnPYxv4N3/UOm/oPDw49FEw4ijEzkaWZCcNmIVJW5J43/crOHavoKWpvckUZSm9tLBu+jo2DN/LspsP4fDuBuZKVA05TAlsAV0MPTiUrjkPtow1hXBYNz78FGeeK2JTMPwa8L6oSvMrONdHrTmMpWG+DLn1Ifj34bWKp9lTrmXf8H1cef4HLkZtFLoYuhDQBeMmOYjLbIeh0YDKompDe1RjrGGbdjrnY6a2PKv6GlyYDxXnsA74i290v7B7xmCin7LTL3w91vjXod9aMYY3hdDzoevsGL2Dkwk3KDF1EcHPiKk0SHG4N3YCIL15WPVdLVCEbrF8H7eR+MJ3XHVpfVBr1pWBO+hbnMXHIcb4nkaYWwYfNb0sKh88EjzaM9DjVTr0vsaaGZt5ByOloxuEzuD9Zui9gr+a3NpkTLRB2Xl35YuasfytOEOysR+L4ncIOq+jk/G1HePtkz8zrKBVn0TcC33+AI9Eah2xrMvvS44VghWQawsQmqIHR0HGt2yviOVS6GscNUKBRUJ2mAXV3AYvOtRvcukIAky//Yj4T10GWGrpU5bC9wkj6L1jJX2L/oWt8ntBd7SnF5x/jpX6YVy91A6/Mlh4bRpb5Hzo+KbQzxtzicC8AIpt0Nv9AMftjv87J71NZujM1YyPW86uUAS9WnPGWOVFODqVUM0Jlp75gbNXlxFZ/iR5hpcEvahHeyg7wdejDcT0Ftl5n1YocQw7JNbrMVe4qenJjdDXKPBbSZ0dttf0Ad+eIuBgqQFLDQGaa4QpbbzqDY9HvygcQl5dQB+KPvEFbhRMp1Z/gfv6jOJpT0lk5HD3mt56TzP53MMPRe3aqNO57KToWdToRgIQWDOWLrd/oqfm4TZtI8sSEgrKCubw9emfsXuOEWv+iQeQlFo63YaYJU+g+Hguq/MDYOAW6Pop1OegcljQSfD3xM1cjYvk+9S3W2584QU48wRLhgTzgDOIZDpVRkxkJBm//ML7ZWn8XGNlYYnQolEsgmkHRNviaLabxHvm+f/MpEPhGBvywVKLn7WA8RoD1zoX0OTuT5kxFnySBW3vyYf5OqIHD1zNI7L0KZpHN9EPCRr2MamoJIEkvlB0D9+e+x6HvlX2T9VFMiItRM3cTtGnHzAmJFtQaJ+eA7nrCa09y9y4Si6+tIxnp+wj0usqkrEYzr+ApmwznXzPEaVscVK1no+ShAsF3Zyx0XpvdiFosVHou4ZNWatQlh6AsuMk+nRm0PUMbCo9XlXeXGzwgvYviGDL5GISM2K4v8NCwMHvdfCx9IOwVbKWC0fV1fdRKhrQG2MotUE5iSJokfWTQDuTzmApmq63/iA5+09R70FbqR1Rxom8+3imROiQuOpsN8PefgyW9nHNDIkfvsATP87C6qiHQ+PhzBxI+5IphkcYcbmI6NLn2rxjc0mp+oSkvEX0STtMvxunhNN3g4/Yi3LXM3/0E9QmmtmYHkSPW9OEbrGxDCrP8pz7o7xUqMK3wpfBdj1jA/8SlPduUSK7tPQYXqaOlPkJZ33aKSOR8d059tFHfFabxvqmcr6uqGbgGQV2B9wjS/9HmsXxSvjkw3eIc8+hKclC2VfvYf76PYKUDWKM9fgOBm5hefUANud50vNMT/zL/Fvmqy4YSg7whs9p+t9e4RyZstBEy/oVam6gkBXYlDbc6jzoaz9FcuARlOcegczvQe3D6NhxqHWvs+iFRXRPzuSDng+LTLYrC7GjZc7tSFarf+N+N3jZ75IYA53ehW5f8krIVga1ks5zBVscdmRjPhntR3EqYbjzs7Y2ZJlafF5mg41pL7JXOinmG4BCy481k1lZJ0B5w82DxeettEpbj/POuhq8q34TUHGHA/z7sd89l725gu5rya1RmEMEWMfLLAIKequgX5XsqjZjKEg+zDuDJtDJPReAV4rHiOBC4FAIGk53rQkZ6HvjOCMuFxHq5pzreRuRG7OobAqlxHM7xxP6c0z3apt3vqWbis2ipDgmh+ROR+jUOAfKRKY2wKzc99nqzIxPUDtYUZEO91sh+Qtwi0JdfwGdMwCEQ0YhmQnyc+ett96gx+DrGB6bRN2/gEHgcjUevZeoqle5XDKUhUUBlNmgX/1ilo5ZKhzdnd/nuvlJJsb+ysPuzR2JoJbX+EFTERND5+KjE87WD49uoXFMFajcIGwCRfWlXI3vzcUY4QB3kwuFnm7274KxossnvOz05c4rg/mX1gubM3AweCYxJXEKD2ab6H1zLxMTvmK83ywBpBj4F9WmEM73+Z7JTiYWWZLFWT77NyRn4KWF0rIVKMpcjcJajpspnsSCj+mVsYMutpYsXGgLlGj5TAJzFSHVn6B07hUrh+9nwI2zuCPGC2eeosvNBW1+d2dmHx3fYdnF1XjmvwOSHRsgq9zAvx/rS7dxukaMGdmhwZMIQtxDWu4ltdC6NdsTvUvWwvkXmFz6Dd8HtOwFrmIsh4CBVHiKd4wueZ4eGdvoprnP9ZU7NftAFkAIYxkkvgIxD/PD9Y9JD32bBZXQ4fIAkhunwu4ecHoOhb6DeK+ypb2VshnJGVjTG88xptc48gIeplp/tiWg0kpb1uXDkO6oO//GmavQiXWoPhtPazSeSXOoD3+NxwOvCApYhQ40fkwp/Ylv/KHYZyO7kt3ZXO4MnB2bLiQhptVR0JTiAss851UqnPuDtkL8M5gDp5BeG+/qx3+3v4q/nRVNeJ6G4CcpopoaZSWXI5+kXdyDAkSd9RP0/BGDVO/UUraTXd2VY/ZVQsNabpst5cCOqTWTR4fXofYma0IqmbFmBj7XylhXnAS9fhY02doAKry6cyrHH32THnvsbe4PO4Li8suQ+jbUZzNGUUakEgxX4Yl/Hz9wla+AE0Yj1kMH+bI+jTXGYj6uziD57E0+egueOdoLNP74UMh8b7j91mK2TtiK5JApif1DAKUPT6SdPg2jJo9as6Cr93Uu5966Yl7vfx8PeJpcz9TY/ChtiKYi7EPUlkrRFrIVm6zG7qSMvjPY1xycstrFS6lkJWUNYi+2KCux28FGM+BTIQBVGn8Omw9yNLE7t4znWwLRzrHprfOms+VxwsofRWcJIzl7NY/6/NLSOBp/rv/8I5HHN5J45iZLq7ZzrmyweIYMv2Q8zPJdyUx8eTyGqjwUkhWH2kfoNUfNoN7nPaqctPsAeoVZ2BJqH4iYxhXpHLf9fsRob0sV22Z9cshoLSEEu4W67tM8Tm02B6bWxnrMo8i9f6Z96QI65i0Bl++vZRwPDZlEVMmz6E2x4pmOlnUwtTiVC3mP82JQnmCaoi3YFGsjemxIQJfsXxmVWo5HM61nQy6GqnOuvMiB1y4RU/wSpY5+0H8tkl8v3NUVvB2czSg9pIW9ypxzXVl9ZTVkryTe8yAGVQ1hKhsq6Q7wZP917Gr8m3Ed36R3ylS8s+DNilgBgGz3BGT/zorQWnSt5q8sy0ImIu4Z2jec4GGPlmsu/4e1XrAIjU7FzSqYaIbo4O3kNyDrZ2E/Aqui3uOZspZfC/BkKHT5mAvSYj4uSGihUnfI1Hi26PNdqugq+qpZe/EOQJ5XQy/CqmZQ6XacPe4PsCVtS5vreb6/ckq3h3s336ZA/TCkvgW9fsE4PoseuaA7nsLIvTO5T+ODJm+1YALY2x96/cJ583s4sBOuhLR+S4kte0xIR03IdtYF5rRql+YEDlmmzdpXpb7MAx7XGR+/hAZHKOjD2mZ0ApmpAqBd463HrHVH5aEn/MpOvIqu/3/svXeUFVW39vur2rFzzoHO3XQCmpxzRhFBEFBRFHPC9CoqYs6ImDMGFAVRcpCcc+pu6Jxzzr1TVd0/avfubsP7nXPvGd+9535njcEAqmpXrTjXXHM+85n8u/Kfdf79T/nfU/7H2fc/5b9FEa6+gWfzHvU/nqng1Z84pyZ64vZ7ok11PRD9hf7vk6H/kp7F6jKbnZen0eQpkxOXowq6SYfVSLYjNzLRuZw4HSSWrGFFw4cYzy5ReZFLN0PDxV7oFwensyCqkq18G+cG5VF69/ccmLaH3xKO41WzBkKuo0OXyG3xn7IvBKy6er4LsUvm/m/AQistcoSjHS3Ol2mQmtU8grmfAAInq27BvaMfUy7W45+5H/PwHRD/KMyppCL5JDUVS3HrTEEBCqyeqrJ68nYAEs0nOB0iMLzBH6XRHZvWS6W3OjYfSjYySqjC2OmEe7Mnz57sR5b1TtUA4T9OTUCsDUVCwCcTlkk9wo3+VJr551xJVcDTwDb+OfJvKLDNKvH06l1UdXqT4fUqfS59w+L1i1UET+MVldJSUTAoVgyCqmS3mO1x+Tl2Kpe4B2gKms0TpQlEF0Tz667BrL78tXpwPf8INJwnTq4mzc7xfzr5OCvivKBsCwz7irrOSOQeTl9PatX8KQYfNNZaIvTWHgQRIuR+DLv6gdYVb62G6OZZ6G2+lPh+SaG1t5moC4l0syuURYJn+2lIfwmurEJ0jeRE3QMUtIZR43aUS5wHQWs3hjdD2Tb6xh6hJiyOOU/+xuToL/AvfwJ+cYOmzF6Gkw4Fci06BIOn/bvwZPLHHAtV7+uAf2lz1bxnQG7zOJ7JWMaVbl+YI/m6RhRpc20jOy6bzf3fZVJDAYTMUqMsJx3l/ZYYMszdUQuTbAPVcVqkgKDFLAwmM/xhjpkg5FoEptBl0GchhM1B0LlSqTezevlqJk87zceRp3FXrsKhaXBiEUEtF1geV8+RCfsRxp4m2j2PnL5FMLceky6OVWOvxxZ2Pcf7jiC+7gnWTFtDkE8yzMjgt+YDDkW7xfkiD1W5EPl+ZPfI9dgFu6gwDFp1UlQFLuCtX4dzx+rljMpM4Y/QTJVCNvZeiH+EbeYI0nNC2TtlL9/d1COni0cS1rB5jC1T6bYKIwrJGXGKbypfgbn1MFo1kjzZNo4V9dArt+Lpu3DPeZDUgANscx7IA+3fkFjyHiv6f4CzbxqM3Ybe5Tnc9GspHxhERFEEWptdHjVlQs5HrA89wiSzGiUxcf/EbkVO74EYMstBL1XptYly759oMjc6qh5mLWT4sGsUxzSiqWrgj4pVMPgzQICcD3lSV8CMz5dy80+qwU1QRDSXH4X9EzD3f4ex5tG8uPJFLgy4QGdgA8uT7VROgFZu4r5B95MUc4QOLwv7OuCYOc0xN29NXUFVlA1fDfQv/JYHEmfiYy4DvRd5UgRTfljILT/cQqlnA3+0h6n5OAq+BWszWbFnHJFo1TEFPF53i0pnCVgkC9m6X6j03kSF10bytVuhrQCyVkPzVYI9oxjochOKVMr8xGcYH/cuiv846DMfRB3FNg0nPb/gzNAz+GthsM8PUKaiOIfov+EXl+EkVLzC5HKYXrxIjRwOmQWJT3G4/G4SKleit6qRFD0pR3rmSvw7w8kV3W3ccyiaR1eu4qY3H6OCUDVXW8VusDQyxPUtYptPsvyD95m3PwsXfQ9DQ9QSBk9pJbxoLS6d8RisAegtRVC1X6VRdA7h0+Z6CqyQ3HoDU0L3QlMGOIdh1fj3ciAJiqgCLCIXg96DBiWN748ep+nSLqYcf4V04RWVFgaQ4pczqyGffNeDNF37Ejecccl5Akp/g7AbkTTOjnwxAPoup5qHCjpxsV7FyUV1GLtUGvDq/A1SXlRloO8Izhv68slNP/PBgx9wKvnevxwmug5fOqMX905P4jrtP0flnAZm6rW88dZb+PkFcrhoES9k3kpIyat8aUgk7oM4pI5SKN2Eq1jKnvy7OaQ7htsoH7aXPqHSi55aCvlfUWKFBo1q6OhQREcEEp7JxJXezbtpquGqWoKz7X1VA2xLNrioh+vCjvH80SGwtwNy2oaoEZyWBrB1MjZiLGENt9LscoElEfbohaIfYHMQDyRN53LiZfp35aoRBNWwlvkaHf6LebEkzYFm7wUeyP2EpOpJtAR+SInvV381OvYo1e2R7CtYis0YBZmvAiAHzyTDAk0eTQC8dC1Odb4fnApbI9kadC/zCw2MOBLNmuxzfFYVR+FwBYZ8oRqU7HS0nQqEngHDkRKOSBJ7zea/5KI5rsC5b8/y3tlREDCOWZGvcWXsCpLsdLAV5jr2Jn8Gae8y2nKareHtmBMe5ou7N3BTTTMM/VIdj84KYhseJMbriqNtbmInuMWpOSP3jWWCOOsf+wGnINY0a6k+OIQb0wdwZ1AuGqVFNSTkfkZw4yle8oFzJvDOh21Fc3BOXwrZazDWbGTNuPks8wBZsPFHagjDfg6nobPBPm49DEJ2w8ifDQlFfh9zLbTb6dA2IwsmHUJnzic19DD1IUbicuOI6plnUBAxCS40aivpMOR3z5H+r6s0iS59yBu+lQtOX9NhVFH5HZpkNb/TkC8gcCL5gzrYdjCb0IbFjrog6hB1Kkilp0wTEdW1OuBdNrOE8eXdVdEiq1T3rtEw+FOGOT9PtNd5Sn2+4VrI0+S09Y5sjm9bRlT1cnzaxuBuSgZRq1I/+o9FiLqdVYd24JqtZ8LBCQiKatxn2Ncw/TKiIGI2mDk07hCzBucyK+AXNTr4Z2c1unTfaA7E9uejBz/ixSmreH9/y9/mQjosSZwArAiMFsV/pFmc9dgSlv9rJTaND1vaodAKrzXCjKKbYWsk7ExV+wwRq85KrV8tHc4d6lgYfWHqaZiZyY7OvbTZurxugiojTi+Fqv3oO0pZ4iwyb8tsTn2TxHun1mGbehVGbQJzLf0qPuKtuH0MLczDfHE/n12279vpL6AXmvil2ZViG/wUBEu8clR91jsNPFO43uucYz1BD6Ngwbd4HgpjlT3awGAJJrp5Wa9xKnZ/nSE7d5B+YROVJi+qlAFQtlXN8Xr4OvabxnKoU2W6eO3QQaomKCqw7fhCODKHdaFezM4+ydSLTfwQWUhY+e3wkwjl20FjxKwJo11W18JXJRMxDfgR5jWS7/0x70/rzxv9Rtt7y76xlO+APcMRkHnrzFpO2RoI0MC7QTtVfSF6KTiFsKtPBUYBh/7l2EuGfUtH2s8UBKzmathyGt2O06DpbeS67PU6vulOBBZEcKDGwC7uVMEwrlFw4XFy3IexVbUt8lOwhdmes9S8QPY+9b8wEH8/9UwrynpiTK+oLBgtOQgCTG18mFuLT+HWkcrPbXA05kX7uFjJqB3LOzWB1ElADwcZ4QuokCfw0skvubUaJjpBuKYNjs5RKZDrzzDc52OcnAo5GTeek3ET0GW9AFUHYPBnRJZ8yQveKtVXiFs2D4SEqL/z7Af+YxB2JqHEwqOe9vXpehkOzVIdggBX32Jm7OOU+HzNtxVpFJqmq6ABoLYzivWV/cizds1sAZzUg4ghexVPjJ1IfVAPGtkuUMpvwURd7E1v/rcGxT85oEURuPw8YfXPMNtDNb7bpD+BLDrK8LDVk9SynJjKZ5hXnUlC2eu935/6IqerVeengkyEFpxbL4Pem0LLTGrtevbf6W8qEKd3BFRwewZU7cVZbsNbVPu6V5s6iqH4J05V/kBm2KMYLWEENM8kQBfjeEdaUBqz/ZcT1HgTE9rn8IyuP4LcBnNrYMCbUHWAmfUr6e+ZB4BGdiLL+V3wHgj5XxGo1TDIYzpebcOZ2/dNBgfvoDnuE7j2LpEF47gvageK93e0GbPwbhvNyNBRjjzmqgNTnccNrkc42Pk+R4qP/GVsGl3OsKe/FwsPDFEN8LIZ/Eby+L48SvIn4tZYzdTgV1VGB/cEmHqGSkME9T260bGvBE2ByNvtF2Us2hoCNfCUd6WanzN4JnimUNJSSK27GmHumA8Zr8AGPf8aPoeCpKsE2Zeg40zU9zE6gu4g2aAQpIE6jz+4LniHem/cLrA2MUf8lCc7lxFR8yD5Ae+w3xwNQ79wpCfoYuppcDuM6PsbJn9VF+bkEjDVMNnFhCIoNHg18LLVR5U/nVVQ+D1aZDqcOzg19BSJseU8m/AzYt5ayHwFtsfzrC6XoUYwNhv/lzmou4DUR4FDivKnfUzmuAy/fHWe9z74nHLDSMILofzAEOZtnMfaoCJ8S5+CjnIo38pXfX4kUQ+7q9bx7P5nqdaoe7NONKMTzYTV3MdzYQdoebqFBcW1aBQjzm1HcK6y50pXJAIu70V5tJD4gsX0UVSgcpezr+us7WZKJqR+MSneQ5nTd46jLbKsOKh8taJWjdaeW8M7TVaaXc7Tbmt2rLkuPcnfxZ9p1s9JKlvtYPrpBXCUOgly3o774CscHHeQ6oBqx57RBZqzaW3oO/VEnd/EqfI5CLOy1LMPkOg5iMiah3Dv6I9GciXB5aqqS3RWwKif+UzzDukR97DDfzS7B7hzsvXnHu+2O+jpZrLpKqIIte57uSh8SVwxJBR6qjdMtbB/PNOiPrb/tstO0N2om6PvJ7l0LYGNc4grf4nElm5q56u1V/nGO4tXfWGQ43xg/+3lZ6G9iLuF8WrajD+zTWStIfzcbQRpQWtzx72zH07W0G5QrqDmb3w0qIRxPUFCCHDqDsYFvc/GqyuIy+pDiQ2ajFfIqMnAKlkh/CZK5OuYGfkTt7pBkwydXdbcKytB0LKg1IvOHo5PQZHAXAsRizjmcxs723uMa5ecSH5WZV/TGJnn+QljtV4MtLebMb/Dthg4cy9jI8YS1XZL7986B0PS0zRqhv1JZguYnZLBOZxW7wWkN6igw67IPvFPOnqfumUMLvkRgzWAHMOPXK292uv+1bDHyOhzPx36Qlxs59Xz1P5xiM6hnDPDkGOjGHE8jq92dp9RaM6goSGDAusJWp0yHWBZv7bv1BQEWieYU8kbDV4c7QQXAR73BNemE4AKsh7tc41PvN6nb6m6x95ZHc67J9ajV2qgraiXs08G3k47wa6HPubE1ESqo0cS/8oteJdn4FZf/Je5++/K/9B4/n+j/Ldy9h05coTrrruO4OBgBEHg999/73VfURRWrlxJUFAQTk5OTJo0idzc3F7PNDQ0sHjxYtzd3fH09OTOO++kra2t1zNXrlxh9OjRGI1GwsLCeOutt/5Sl40bN5KQkIDRaCQlJYWdO3f+l7f3f0p3UaaeR077QD0Yxz0AvsP5KWFrD4qy3pF9PTn5c0JWccT4RM/XYXEaz7aDz5C6azgjGgIYLZ5QDT7Jz4N3Gm/6HGeOKyDIhLtfRSzZAMcXqNSSByezwLnnTiNglIrUA4+tA1wiSFdmU+MdT5t3I2mudXhWvQehc6h1u4Mgl3LSDOri68m5D+CmrWRx0GUmO8N7cbuwttyqGqVnZWFKeZmdwcM43ncEGsXIq6MXost4Ut2knAKxuA+jw+qBIkiYFFhebUfhukQAML31Q6z7RzB17f3c/vPNuEkVqsE6aDpE3soN4my+n7ybzMRMOo02fLRXQeqEURtQnEKQTeeRddXYTP/szAPw4J9zJf1HKD6x3x+nFfnq1cWcGPocOl0toizyXkCb6lDbFgs/iSzJvY/Vnt40uZzmlPeHbOl8AmLvU2l6Lj6OZ+UWBk8+Q2NIBRfCi2i1+kPGSyptTdBkHrTt5zvjKK47JyGa7A6ggPEAXKybw9DsOI6aVHqD+5QHYd8YaMmmPfZN4q8m96Ie6lL8yXydaW03ckPDCvxaJnMlYhkXTBt7tc9LCCe4/mbaFCi2AqIR0l+AjBcRWrKZ0vAaT79+J6tWrSK5VasiZ8PmQfNVqtLfYlLsv/j4zkXc5HmZZ64uwLntiIpqyv0YoTGd/SEwxwWMAgRrBERZNZ4LAhhEKy725WEQYIZY6Tis91RKAbxEuKfhO7i2mtia7/h9RBk7b9pEVWBVt+HYMwX8R7GuPZpsKyBICMjoaVbrBAgFX3GDS48OUOxJkdPeU2lYLM30c9Ix8cRIsl66n6cyRuIhX7SPxwSywu8lptzG+XFH+W5iLvMif0Ey9gGDNx2Gfrx36iuO21N2KmIPQ6dnEi1KNNjb9HfJsbuUD1mwOPjh3St+h+qDaAQNpWEqddXozXNYUxCnIqSLNoDWFUIv4Pfj88zfeCse1cv4XGOnOtwWg7FE/ffhcYfJSsjC1ujOq5aNqsO2/izIVtKlAK5aIDP8EcqGOXGXswD5X6FrPU+Qax7T+xwiimCiah5lceyDuLiGQcgsREFiTP8tuFpambVjFjqLXTm99DSce5AwXQdK/6v8ctPvfHnXl90K/qUVCHtHENN0D31q7iMz9DEuRi2ipuESpL8MNUdI6jzNVwFweUImx69fwlH5Tgifp85vjZHdfZ6iniASshO452p/Nox+AKHmgDpfzKV8MvwPBhT+TGBVIE5WgRlJax1UYho6mBH7CU7u71FiucS7TfB12wLH3GsyBZJpEVVQvyDD8G9hbi0oCkZkchOvkR2fTV6DC0/XjlIjcvxHQ8B4NrYEcGbkcV56/iU6g+wSqExF1fV0gF+Ins9u51shcCLMLlZp31pykE5tZ/ovBTy8bjSv5k1DkNpgZ3/I/YTntON4p0V1iF4Kh4V97oAj18OvfgzVPcb4iO9QFIXzYfCK/0GVxs7aAn1u5sfcT5AVDcklH9Cv8Bv8nXoiOgWHUeJ4wgj2ey6mZ6kTE7lsP7W6t7pjwAL5X8Oh6XBwGvdXvMDma624NpUxzfd9nA6kwNGboHwndKjG2ciahxifmUX2uBeIzoyEA5NU4ADdCNFrzVHM/7VTpf7rKEUQYFjQfhbaowP+SvGm/u3fMo3YymdJdBkLYTfAIgVlQLe+FNQ4G3edfa9sL4aEx0huWsfLbc8RUr8YDXCfz3HVsTXyJzhyPTbBianl8/hx0XrWz/+FU4bbIelpWGiFqCX08eiDrBFp9K3ndf82rpPDVaPyqaVQ/AtjjSqNpyJIfLD4Cs/MhzEiTNXxt0b6uctvYflUE+7tu8muG8OFK9/hW7uMFqGU3IZcxJDrYKFMlmUJktiBp+dpfgxrYKR4WqWfKvoebK1MLYjjV7sqOcFJRjg6R5WrploQhB4RljDD4xRcehIuPQN7R9AafAOftl/AIprJz3uOD3P3qnvUr77qYXBHCk8lqSCWMVnJbOk8DtF3gls0bVZ/6mw6bI6cCQIU/QhXXviLPHdELcgSnL0fL/N+8vo8xJWIuzDrqrHSHZUL0EIZF6JuZk9/H8q9NtBqbYLYB6DvEwjGYLxF2LDkO/bOqMDLeTJU2I1kXgNwNXgQSCSjjo9iwc8LALvMj7lLNToCz9TB5kqo2iWy3fbvc9HssNp4eu0xqtI3UtsZw5H6vrT3MAZ0UTiZgm9nzamv6VQUGtyOUms4CZJJNTpH30me/3oKm/o7jKzBmiY4dTsUfA3+Y9nebuJwYj+0YwVunaFH7Cjq/ohLH1Y06IndPo2TP8zkvTPvYzPGwIT9MOpnmhOe5Lb6qbQr0CiDRTYidBSBxonOuNf56OLL7GzVIyo6zPoKKtpLe62pfxd1LwhQ4vc5BYHvOq7pNKrcd635jpfHT8GUaKNk3lZ2x51S96iL/4Ky33jMOpOHdJ/SaSghQQcpyimwtql01LZ20jYs5pSnSuV9mxv87vWVmqM65i4w+iNonZB6GEDUyOgMNM3nGBX+M9+F1RHYZUQVRdUp1/cxtO73kFL0GV89XMCqVasQNCIkvwCRS0BjxE97AVd9IxXeG8kPepOijox/mAH2b2qMKvVj1O2ca29jjcsdtGmvkjl1H0Ocq+Hau6pu2F5Ikq2M4S0+zNoxi7fP9eH1/NVqdLDUieI7nEdrQRYkaAXxD9gr/XsA2n47r92Ahx/mcde+zHV25V47zaL0GMy+QzVUmgyJ3FYQRcrKf7H0q6Xq3it3eThEnvHdyfK+tfy64Bx6p+eIdU/u/pBHIpXKZGz2aCABQdWvxu6AkFkYGs+zWuuBe6MX7a42fqkLQ3CNhGPz4PyjaFouE+19HoPNj4bmQUyN+FR1qs+4Qrkyw6EHbW6DQ+0hoLFbwnxHMDvneb5qsVejfRDOGjsQsUCl9TvS4g1AYum79K/rTnEAoBV1BDTPoMZzB0fSfPmj8S3VCA9g60Ar2KkMezJdSGYo3gBlv+NT8RUBeKOTPNie8xDtziNVvdA5GCQT5S2/0Ob/vb1PRDX3o94TRePCxaop7GnvTZWIrR06SjEJwXzV0cQFl70YBRjpXKLm5gTos5Bnq/2w0TuiDICOMlzOziQh/mnajbmO7/YsTlpXXM1j2DVtF1v7HuakLVKN9NE4gaChJu4pvqkaAMBnTTpKrZPh2M2wOQAs9fwsDCZbm01c2cskVL5Km26Aui507ggCfD51DA8mvo5WVhXnBvd+sEih2Gd199wA2jUV1NecUY232+OYZFyAJLlgVmBfKMw25ql7RWc5VOwiLW8k68KnU+9+iBj/g+izX4KSn0HnSlnoLeztgHZjDldj7YZ3t1hHXtqukmuFG1xgTug6VebvG6uCSos3MDroJ65E3sl+tx+IdtoBuZ/CuYfwdy7ihdypnLMPkSAIkPI8LFKwRj7I6KCDVAbbdYeu8wFAwARMrsOpcd9NdvDz1Ljv5s95Wv8+sg+IvoNy39c43aYqMja5B6gOYNg3GKYcY0TrahLKX8PNlIiTNayXQdJavJE7+t2A0fMI4XV3sdKYSOi5GWBpUEkWGucyzfoZLqZYDhke55Ozn/Sqh2A3YkdVPc4ridtom3waZqTza+gq5lepfX04KZmj0eO6+3viAVaWnqIw4H0kO5VdzzqNixjHsvDVRFUv5wbzDF6OPoauMwtMNXDmPij4moEGG14aeKjiBx633EKJ8WEY/wfc1EyyJY9fwyIQ0HLKcx1XmuLU6M7ybYhyJ2MzxnDKpEYODs/Zz6ElR/F19nXUo6uvqzy3sLH1UX679pujblVSJmej53AlYilWbRPtthaoUx2+NF1GECD1wPv0Pf4l3+XbDe4t2SBq2Rz2BKtU7As+Ivzo/jykv6hSwSY+xdpfJ7PZLZJLUbc5wE0M/VKtlGQm9UIijw1YBUBMlZ0dxjUGgqZT2R7LNZOTQwL1imSx1XMxqpUVqohjQWEyXJ+vysdLKyg0Ps7enMepdd9LVMKT3KC9RaW27XqXvT/q3PdjTL4RY429P4z+EHodqTlzWPPIGr668yt18llbVTvE6aUMzH2JU8mtlIeUs80ksfTcYygzcyBoGiQ8wQpbMkc7wdXq8m9zUP9HgdQ7rDaefuopmps7KbXB2D8mkZyZzAjXVryq3lYdV8O+IdsUgAI022p57dhr1GtV58OkqG/wcyklo+AJkl3G42ZwQ5LAoOmgLeQR6seqNId6iwa/3JPYStpILXqXcbLK4OIAD+tVmRvSMocBhT9wY+RSnLTdXiObYnU4+0Q06pm5YB0e9oVgk21/iTqF7rVtk2S7fa2HbL/6BoF9qvn8hs1cmnCYo0ltzE94x/G7BRG/c8/4PM4MPsPRUfv4eEa8am9Kfxkar9CvIpX5ia8y5upFpl9sRWOdAUnPqtTQPeaBRWzGpmntrp8Goque5KOwJqLqHiQj7BE+uNhN9yqKUBCwhjPa1ZTaoMxmb0TLNRTPVM7UDcOkrWJ2QSnTL7QT4Rr/l/Y6WUOJq3yepLZuZ58oiDxRC7kWyLGrIGKXc6slCxrO8frE15lWeRj/5hm93oedjvXO1FXEV7zieKefeBbO3IfQdIna9nCGX5rGh/W+jnOsKIgwZgu7Sp61/0L93rmgu0n5JIXaDjWkThBg+eENzK6AVD2EaqxwbTVkvAzVB9ne5tKLF0ujWFWGl9xPyfZS9xGv1pFMvdjIdPM33Q92VMD2eAY4v0+y0caHzTDvjwOqXc9/LLjFkdx2gYWawehsXugkz14OqRjlEzbGncZHhOSOBwlqnKfK3huKqY7Z4JB9f5d70WFLkrtBHH/OCYuiwUUAryFp/Fj9jerEdotDU30IZwEyk1UdWK7rjpwlainlVz+i0jySGs/t1ElwV+YNNLpcB3uHw8Hp0FnJp03eHDOBuwjv+IFHwz5Hve4Z8CRzPb7AvUN1VuZZnSlpSWKEcDvsSHLUM9wjnNon6yjxCeKCTw2KKOJRncuVuz+kJHEqFfHj/8dh93+jCIJAWFgYf87R/b+r/Ldy9rW3t9OvXz8++uijv73/1ltvsXbtWj799FNOnz6Ni4sLU6dOxWQyOZ5ZvHgxmZmZ/PHHH2zfvp0jR45w9913O+63tLQwZcoU+vTpw/nz53n77bdZtWoVn3/+ueOZEydOsHDhQu68804uXrzIDTfcwA033EBGxj8fVP+n/D8rolcSXqGp3YiWkFmsLhtGe499vGfSWkEAQe4Rffeng0Dfhvm8sGgRJqORCcWTWaTZqBojE5+B1JdZ2TCKre0q2mXTtX+huESoSviIH8HawrMezT3eLRLY8Y164GkvBu8BnOzcyLXG/nQoMPzMUvXB+lMIAqw++yGR6YOQ6WHcab4GlX/grc3mpZh9XO8Ci93hRqczsHsg7ExBm76K8wmnaXZRqTVOlM5FDpmroplLN9NUu4ti//cp9/4BgEbZVUWrN6iIrDJdEpc1qjFv0shMxrWsgLzP1N97JKLX6MmJz+HQwo28kJLNROMiNeqvowwp4xXG1Q1C570fm+GfnXkAs4Hv/+HeFv49xWfPcpvZxu9bT3G4Q49NVPBu8OYGd/sGGDABfEeghFxPZ+77+LZMIS/4VX5uD4HBH0PjJWjKoM1/Ci+5VvD+ss/p9BtLmzlczeuSvBKM/lgT3+D37MdQEHlwVwYv5Skq/dCJW/E2luHWmYKvaQhD8naqSlJLFuyfiCiCW2cyWpuK6kQRVDpUgOr9mDRh1HWEdaM2/yRqwzVDSCv8ie1tIiPLoMN1kBq9AlC1j7tjxzue/davUEXvlP4KVX9wKeAmhpaAc7szJS1GyjsDKUs4DEO/hohbEORO+utFfDUw3RlKYtsQyn8HVEXutctPMqBEfXebAvOUSaqxHQhzvcgLiesYaqcu1QngJTWDpR6dbMZLAxN3T+PpN5/mVdc2NZLMXroOT5cil7AjzYnH256E4BmqIT5qKfMq1TUYp4OZbh2ItkYV1bk9ASp2cj7EQpo2goykDLa1hdBonApTTkO/V5EM3mizo7j7zSdZ/vtU9hQtxLnjBBy5ERfbVfYX3kG+XZnt1BdR3lKOVPwz/CiQbPy8R3Jsu7PvTwrYoaREdjogYOBx6THI/gCflpPcMTSfTXd8Q0aaN+fa3lTpCE8shOML8DB4UR09gQ6vBEZdeIo+HnMhbK76DamDNwJciG32YvjJ4SReS0SrGODQTNgzBKwtiEJ3TESQvnufah56kn2FdzJvk5nsJtVp31NH8FFO8tCQu3EZ0sSGBRuwGezGj5i7YdCH3FH0Fn+ULKQsUqEipKKb9qM5E+pPsdbXiZSSj9HKrgAMPDcB0ldC9SHS3Sdy17qxTPgxnkNJkezWPAB6T5WycNCHTBv5JnXDXsGis+F+OVml50h6To24sLZhrP2DyZuz6XTq5IO7Puf+45cdOfskXQB9L03gjqaTlFszes0bQYBtOQ/zw8UmBp9rw9ka1t3g1hy+81vBtBmn8asN4sGPHiNONx7C5qi5LF368K/qRKRGD8YcGYNSFsRez4chTTWM9UySDfb9QOusRlXpXDGdfYChXg8BEH1sBEXtQWjTn1TzRJiqexkGvm6Bay2z1DxxHkkcsP3GLb/VEl31FHuynqHO/DQcng0bPWCjBx6up2k1XsW7bQxh9bfjafDqXjM9nDEdxjxqdb0jW8Ll05yaWETOu6tY9u0qHjWuhRB7zpCQmeTZJAr9VPT2sKRdiK0ZULoJDs+Ek0vwzrkTN33vnIGkvqJSMdYe52jwOW51g1ZjDibRrDpPr8ulRT+MRXGf8b63B6F1tzG+ZrPqSP4tBMz1+Gkv8OSImxkQuJfxEd/hrXTHzollW/jID7xFGB6iOn8sfjNhwNvgHEZj8AqKa6bh2T4U346B3Ox1XqU/M6k5JWo0yWQGrSUnLpfTPtV0aHx7Vf/xhi/4w30yegEe8+3ABbvDpOAbuPQ0G/3qCNLAqfjxDPvjK+67IZriD2Dkoniecovldg+BJ/27jfQ33z4DLj+LZ9uWHl/pktuCqpwL6t+fzopmUcR2FlSCC2Y1WmfiARj+fS8Z/4i3HVZsqYfN/pS6zWCBfdOM0YG/tkHN3RR1O/R9kjafEWRYmvGW3YmveBmj4KGuuehl4D0Im6CjU1Qt8RmdbtQqI8B/DEw+xg/53zI5ewAZ9mhsjSCqIIpp53Gp+JR18UccdNPdkX0yeA+i2P0pKuxV3dcvhG2mp3r1tVVspcL7Zya5N7Bp4kLkuh8g/iEY8Daa6j+oj4bJfRq4OSGWt6OKIfVliHsIpl/gdh8PNvR7AQDvuGLuiNyGR/lbcHQeGINQ+izmohmVFlzQ/8cAQBr4asNBztXOZ82+DEeEslGA8Pr9UHcKyXsk+wvvwGJ3mDztY4KSTdBwHgQd65pN7At8nEovFYBTpfiqzses1VB9iGdrq2l1vkKiHuZ7WBGlpl71EAWR8mA1XK25cqEqlwMngMGHpIjpzAndjY8tmFQ9uOuaMU1Ih/ntKM7h7M56Dp9TZubkNDje12UsEEXQ23zxaB+IwRrQ617XfQclnL3o605B/TmKnNJYXhhLU/0+6jRWfm/3V6NCr70FZ+7pJb9ucIUFyhpoy4PJxyFlJf4axTF7y2xw2hoEihXMDSBbMbSeQhP4FQeT47gUsURt87kHcDo1gwiPdOZ5dDjmWM86x+sVng6sYISUAIANjQrO0BjBfwy3Vb7AJ54vUuuhRgP8+SBcZzhNhddGroY+SX7AO73uKYqCwQzxOfGM12n5KOiQuiZ8hkHTFe7u2MP9yVV8e9u3nIvNptocrhrmRm5AmHKCtU0CChD/YxxjFe1/aP6NNxhwc3NjyPir+N16P+12msXhfhDecBgkC37GYMZn5OJkciK8NJzprnkqoATgyGz6GUsZYgS/likkdY5ndPEA1UB/9U11/lXHciJ+LCOyjnJ9xSUweEPIDHCNQPIfyx3NJi4PP0m/W37iy7GLVUd2zD0Qch1Zqd9x08lVNLqcQlJ0OOuaVIO1Zwp+mvNkJV7jJleYWwmPVo5T2UgOzYLfQ2iT3DArEFq3hNHXzpLoOUit85BPaR95gq0NahSWIlj/YuyJbH2ebQsFRnmUsz4Q1nm+oMqC2UUwYS9ve7/E274qPZ9L39twy1qmMiTcbIG5dbRMLKO6TQXb/Zb1JMVxx1RaZu+BUH+OZfICbncHLfDjwPdx2ybAjwLO1kzWXXqL2xtVh5xvy2S1Qn3mw5wyasSJ3ZSDlcvI7pirsplcfQsKv6WgfAcDrx0hI/xBjieMpMFs5/bS6FGcImiXegMlepZh9csoHLiXsVcHE1IeosrdjhKVrrXfa0yOnsyYGhXw9UWjgWpphOpUM9WARxI7NGmq7BVk9Ho99c7z1JzLzsGIImzMXMGJ0ptQkFgXAMPyX7GvERjbZz17YvKI18EB9yX8/uswyHwNgA4lAJs5hOTK57i7Gg5ZwlUAZNRSCJlFoVyBWas6kN7t2lpTVsHFf9HiNcQRjXhUvMbEwmfhuhw1P1P0UnAK4mSnlh3tsNwLYt2uwIJOFYjoM5iLfd8krbwJgEabjl8bdqsAopwPSfA5Sk+TU0+ZJPVfS8qB1VRKf9PX47ZTmXKCWo895Aa/wpm46ezTPN5rLP4ManHtTGSiOA1KNlLj9wyVVvVAs+zwNPanRGDpyl+1bzTsG9uDRq/7fV1Fd3w+44O2kBd3A6nFn5N3bS3WpDehOZNHg1y4Jfg8KZa70UmenNev5ufMn3vVqytnn3vHAEb4ziLUPRQE0WHLkMR2Wp0yaTVk2T/oDmfvZ1OgeqCpcztIqc86Km29I0u71uDZipk8s/8gVtc01bGa9ym4RBBZ5sOOdpgb8y1PjVyoylaNXn1/9UH8rNtodbrMZ9Ys1hVNwtiwC8ZuJSelluwOL9oVNQL9z0WlhLOfp+zgyC7QCYCJJqq9fqfVST0bagRtt2PMKZTElBmOZ9cqB9U9SOMEmW/gae3O9xXQtdV12PW78m08bN5HqFE9vF40w7N149VI2l9cIX0V5ZoQCq0QVruU8Lq71D6KuBnGbuGna29xXV4ctV1j3NWBR+bgXvgsL9U4O6KFWiQD5HysUoQWfY9ZCKC8NR5FkAnQQD9dHhy+HtqK7O/qnnunTdDpNUGl+RzwNgDzlc3EVa4mFT0fBVxS2Q+q/oABb2MyBlNkg9lbZpO25zFKih5DcO0D43dB/9c4jT+VEpSOaeCL3tt/r/KfAlLrdFzee4DJdU9hdZKpCKpgSHYane7jVJ0h6nbuq7iLaz0YfmRZ7a+MmjHszrsbq2REsdvfbDbwcqrEP3MaLqUbALDozZT1nUjKm4vpdA9wrKmuyD5RJyEIAtYuJ5QITrpuZ59F6XTQeGpELRT+AKfuYKJzd5RuV2Rf1/lOVmTMYiNmbS0VusPsHKjn+aoeYBrvQWw0udMoq5LIRQC9PcJQFGGU7zVudxcZ/+b1nBy1k+9qI1VWmvSVYKpCkdswG/NpNaq5LxussdDvFRUIe/Z+JjipHfJnikdRVPPFG/HApKmhKGAtm7J/cFSrJ9CrLgoKIxtVwGj5DoSmK3wVcDP7+gchKxIa2bmX7GyzNmPSVWITe1OHqv2m4d0miCuGUhu4daTiLNrPn6N/hZEbSGk9Rah1GBejFnEifhwd2Negnb1sSeoLuHb2JTfoFercDuAu5ELepwjthUiKDt9ru7i+upbARhUcIgoihMykw2kwEZ6XucVNxM/WHR4gCiLsSGasYQm1nSHUSHC5D3wZFK2CxPTeMGL9X/ZbZ5N6zqXoB0I7VVkoKjp0kicGQbVjUHcatqs0vq2ykXXtrZgUOzCmNR/G/AZ9H4Oz9zI/eA9TLzUw+tpZtc8aL8GeYcTL79LPpQmjCJ62ePSSN/51b8CPAhEXA5gf8Tsu1j44SUG9xrir7xUUbLLVYWvqlWvVDkoJ0sKWYBiqK4Vp56E1B82hqbTHgFuKifx+E3H170AafwiSVkDU7SQ0doNvAQ42xGC05Kig4srdsDuNJ73Vc0XM1f08sfckmvhnHN/9/MJaDjSudqwZd0HBoOmgWJ4L8Y846mnQGPBx8mZXQADbTk8h6aKCTe+ES1wIJjc/JJ3xP5Wz739oPdUiiiI+Pj69onL/t37//5Wv/t8s06dP55VXXmHOnDl/uacoCmvWrOG5555j9uzZpKam8t1331FRUeGIALx27Rq7d+/myy+/ZOjQoYwaNYoPPviADRs2UFGhctivX78ei8XC119/TVJSEjfffDMPP/wwq1evdnzr/fffZ9q0aTz55JP07duXl19+mbS0ND788MP/Lf3wf2KROqrJyTiFJElQ8B1U7OTrqoG09gD49Yzm60WJxF8Pah2COwF+5RhNJqoLB3DB8w84OFl12Bm8OWCK5poFroU9wdG+Q+H6AjVXV8RCmHKS5riDJJSphysUkUanGarDxBjg+H5XrsBmizflgwoBAa/OnRitQaQVbLDXy17H7LVwcAptShh3XHiQtxpBnwuvtKtUechWBNlCpaQu2isRy/i8vR5t1ssqkvnoXPrnziCi76O0OmfgJsKjXmdUA6jdiPq7zyoeiL3AqlWr2DfyOLlOc1SFcsT3oMj8MuY+nvjlW5Z8+R53+jbQqkRA7P2Q/T6CWwyvN0CrDEG+z/Cl2J3g/M/lLtRcfH9HndbMv48K7FkCgGazkbmlPpxNSufo6KMk5Pqrh8uJ+2HKcYSxWzhUdAsgk6KH5zw+UqlWp5yAiMW41uzlQqUHiZkD+C04nVeGjYbEp8F3OCgKtvDbOVXWLU9EEWi8DEU/EOl+jue1w7m1/ke82odw1KdcdR4kPIa29RKPytOZlLvfPo4i+I2ByScgcDL5nu9T3xnag7Kh9/zr3vDsHOyiCCkrYaEMvkNx9mil4+kCjrz+IpvcBRB0MOAtGPgBHU6hNKbH8tTbT/HkO0/ioulUD0vRd4DfcPAdxooTEjknMsm3wgf1RgSPOKqqqti29WXydizF+V1w+RA4BM0tqKhOwNtYyo2hR4m1n91qJPgodAX0exVL4pvM3fsWsVf7Y3Nv4R7/RtW5DVBzlHXeJ5luZxGURYuKlravBzxTaJXVl97oCr9EVKHpyHdQueGZSpXPSqTjAciijE0PktZXzWOo98JoayXMZsCl3QWOLKGocj59skdC2W+4Ws6pRnn7AXRf9CBC3wulzu7sH+9yD6IdXavYFfuezh9RBJOuotf4mEf8Akkr8Gw9x7PeYOvTiin4ISINs1SOdABzPQkNixgy7AjudQX0q1zP1AatGh28SMGg0fEv93YWv/cIns2eWJ79gHPxJ9X8nABaF35z38gee6BXugX2mG6EtDWIIkiyhgzj51xyfo8Kr184V3sEqXIf/OJKk2kfS7aORvpBQisF4CTYXxJ6HcQ9wHDNQwzK+ZKUzCl4Nnp2K5zDv4PgmVR3xNlnn6zSWNnHgJh7aDKEUd7iREh5KLML49kQvA+OL4INOlWpBARBYdeNOrYOc+L2fTsRUlaqERdlvxFVcj1O7RLRBdE89Oa/ONhyZ3duC42GBqsT7QqEalN51ymJu9yO9FoTWskNrezCuaibWPJZBFcvvgl6H5pJouzkVXBfihgTywqfi2pkVdYayPkQAQ0B1QGMOzyO6PwE8p3HqXmr4C/JpwVFVNH/bQVgbSPLfQTvuZWycd5GNt+8AZ0gqVRzfqMh6VlmW9Mdc/u5evi+ZJtKGTLpEAXKZHYlDORQUhI/Zqwk0/QwBIxVIz+Bm+IXciYlibzA13q101GXXvtU75tJ1h8BFZn5fQtcsA1S18QiBVJfwV/TSknaJdY8soYZHWG0Ti6E6Zdh8KegMeJS/TUZ8VNp1xfwyO4LFKVkq/U2+oHGSIVkoE2GivBn8A5/Q422cItBEZ346uIaPr/wO/2LviXENBX0Pmo+U1GPq1jBmD4/Y0qZymPDlxAsf6ZSTf8oIB6bx/2eKrrQ6Pc7FlmLzWMobNDClj5Y3IZwrW4kkTUPMeTqGdYWF6mH5QMTYex2Cg1T+ABvlvw6j6E58eiVTjgyRwUNFK4HYxAtVg/MCsTn+LNNVwULTHBjLQz7hqebfKmXwaZppVFXTlHQdwQOfYRHXtnGrFnbKftY4fFn/GAs4Aayzh2mX6TG9xnm9n2dT64PBn+130VBVPfP6kM4iXWYJCNVEvzSBvulRNVI6hoN5joe86tmiB0vsKFFq1IU+42CmHvpNEQ4xtRPAxGGKhVUEno9+A4nKP0pdoTAWGcrN/Z9i7f6C7AjSaXkCruBj7yXcEunSsniKUpEaH9X0fD81dAJgpqXzCsVXec1RntUOta4sYuGR9TBtLOUer/6p7nYe59yIJUVaJAA0aA6zn71RWzN4fe6oQS+vwLD4RJGBX2tOlMGqcARMl4iSV7Jt48c4qYn1uNvaMTQdkYFrVx7CyFiEVOdwScT7pJN/EfKbVb4/Xg3wiu4fhEAHiIkF6yGwh/IrLvMybiJXI64HYAVPp1qPuYxv4PfKHJazlHhswGrVo3U7VRcQOepvjDuQYfh7rVGMFzzRHbv112B1nwux0fiNUJHWfJQNBprb5oaaxt6pY7BTmYu94EJIfsQFCu05qngFtQ0ZFIPms2uvUgQwL9lOqOvnSOp5H3gr6CUPzv7NCcWwsXHaTDG8b4lj0HHRqHPiOfp6kQ1GjvsRkheyRTlDJPt8uu3NvhZfAyKf1HzgZxeRl5oG/72Tx3ohM87p6oUyL/6QPVBQjKHMz3tLtqNuXTqy9Q5FPsA1r4vsj79JVzSoxzIcU1X5MKuASS23MailBdJ+KWdF1a9AAgqtdmR2bA1ClfNeRrcjjnaE02OSrNoLwf95nMhej4Fge+QF/gG2DrVvSjvc4JL1pE3pI7tj7/P9tQLvF4/RI1AsjbDlRfIinyY49I8/KT3qEtr5Yk+a1XDXJ8F9v5WF4W5soGl/5Bb+c/lts5Ofv/xR5VyOvgCs+xsBcs8YEDReyCbMdiKWDbwEcoXVtLnoR94KeCQSrOZ/DyE3sDiigeYVK4iv002Fyy6CNXpdulp2D+esog8QnUyomxAh3Ov77t6xnPvnMMcHNlIW1IOFSZPxCv/UkF7gRPZV7Sb8zFzOd53OCOHjVUhC7H3g7keGS2lFj3tMvRvuIPZhgVqhK/3QPAdTpSxGl/N39B5eSQiew93zL1OQwkmTU2venmbVb0gJXAvhzrgiHWguse49AFBi4wW2b5sn0r4HpeKL9UcXIIWDD5sKz/Dwb5D2NvPj76ptxCeP6Vbt3QOZYMphXMmNSdNgKEbaBne/BIAOlm1gvUr6R1VFcQuDg57m+td1HadbXtSlQO1x6BkI0HyYHzaRtPqlEGj6wkke/4ozj0IUjv727t5Tf+8N3u070KWRPT1bizFjTuFN1SH7b7R6hw89xC3RL2H0RKK0RpCqW0KTDml0oxF3tojz5dEW1sbgq1RNdzZ+/6d8kQ+tF3ArTOZQV6JBDSfgx8FPDr34aJrJlxnccj1ze0iOIdD2hp+67zMyrGzeD/1K75ogUyb3aM37CuIvY8EvZpLFeD1BgHL4PXqf3I/wrU9r1cbHc73y8/Cz04w8hdypVJ0Vh8eroHV2WtUx/3ojRD3ABUWC1WKCij9pk8pkz3uBv/xMKeCS7XX81H8XgflW6A+RmVeuLISZFNv2daTbrrxEi4Nm9H16P8/j4VqRO0WxE/61xEs7IHSzYRWPchYl272pk5DMYJon4yu0ciKTIe2hHLvn7jo8QJVHlt76WiS1o0NrXDV7vi4Uj0RKf4pcAqmzDqW+o5Q1IBBe9SgYNct24vRCR0OSjhHfr3aE9Bwgej288xw7pl3z/5RRYHAyRw2qf1R7PcJlyPv4JrlD0ed2i3t1FlKMWmrqLW4cqluMIrWQ2WHCZwEiU875tdPuUt57eivpLXMgP2TVMfbxP0c9SpxgFHnhB7CLet+VWfVuOCjkdALqmP/74pD37DTTXc5cIBeKU5Ajfol5UUVBNmSxQtRu7j02iu8/NzLeDtZ1TNiWx5cfobxNeuZa7fbX7XArW2vqZF7ma9BySZeMEc49plqCbTCDDVVhM8wcItlnftdvNnYXa+ee7P67x42oq5/t2QjWmt4rc6NXarJhgCtRU2BYqqBeY0cVyaRHv4AZb7r+LoFfuqYrkbK2sEBCYYJRFQ/qI6XDRRRr9KsA5jriXHZy7BTFUx95xGGye2qXIxfDhGLyYtezpxCPYIioCEYZ1swQkcxNF+111OdF/JghWPSP1PS/6eA1CYTF/7YxN7hb+F1o4Xd03aDIFAf85Wap7xiD54ac6/fyJLaX5kN0ylt6Ytb+Dv8Vv8qsR/EciBsPC1mX9rDHkdj8GVQwCDcTUOoiB6G+8AoFGTHmuuaKxZbG4qiYJEsSIIZBQmDxuBor1nq7I6MEzQQqUaBTnNVZaRNlvBvnkF06+1EeEYA0NDZwHt6b/7o749VUevfFVUOQNgcdl+L5b7nV/HIylVMPBrGxmwVZC2K8Oaxjdz37XlMP7Xh2uTBT7VR6u/C54PfKF4QHmGZ+A25QWqUokNOmOsg9xMGGNQ2OqK+xG5nH6gRX7LDgflnIJf6f28N+GkVdS+IewDLmO3U2e2ckvJXquBV5+9hX79gCgPep8XpCi3a/O73/snuNPbqZfq5TOu+UPgtnLkbg7aDerdDNLgd7l7zozfBQjWPfK3HLrJDnqfWfS/FylyYWwfBM9CINnydSzGKjY5o665v3nYbjAz7lc8icplbcKB3nQw+WBV3fI3VBGnh+VoNxfJNagTvvHoo3URZXBl+GvBvmklc+Uu4aLv7a2r5qyz3/BsK5MLvVdkbez/hBdCuqDnl7098U9V1W9W+aRj8FV+Vz6LZ6WL37yUzdJaTo1tB4rl5lNu69XRZ5wNucWitNQQIzswtLWJy1V57ezSqLl2vOg2zQlawY6CeCp+fgN62pi4K5GoJFlfBEWuEatsbtxP6v8nJ6iX41LzNjjELWXpdfzQHx6l06FpXdkWt4q4acDHFcUPRr3wYWoyTVT0DMukwAA94NnKHO9gULdn1wxysboIAp6tHsK7tIvmBavT8kbBrvDB2BrnyndD/NcI9wlk5ZiUPDnkQrC3kr3fm1K4RhOSl0+4VStTzizE7eaCxdPSaU/8rms7/cfapRZIksrKyVB/G/wvlv5Wz79+VwsJCqqqqmDRpkuOah4cHQ4cO5eTJkwCcPHkST09PBg0a5Hhm0qRJiKLI6dOnHc+MGTMGvb5beZk6dSrZ2dk0NjY6nun5na5nur7zd8VsNtPS0tLrD6gToOuPbKeYkGX5b6/3vPbvrit2uM3fXVcU5T98HfjH63+u4z9d/69qk7gzieC8B9VnKnZC9vuIfxIaPb+hKEqvw4OgiL3aVCokAnD8pfe4mnIrjYZJKBG3QsNZ5Csv9jo4+HieRa4+gmRpV7/vOQCT6wjkrhw3ikCrfhBy7ANg8Eaqv0iSYSDhEWswCjDLpwBD6ynI/RhX03GiPM8zMmgnGlSFU5IkpPCbkQd9QqstiLrstWjKH8IKVMo+SNfbkW1aZ4aVqkeKCu+fyXWzU2aZ61Fi7qVRH0WZ/dzgLsJCjwyUuIeRJp9EkiSmeUfzTsOHhCoerG2CPKc5SNYOlL0jkLM/gP0TcMsqwLW4mVu+K+SQ5Xtk5z6Q9zmKZxor6lVu7f4JYziumP9R6QwE3gBm8VfF9N9RfP65VAMeXv4IiKSnpnNk7BE0kj+yoP/TvFNQBBk3ERJ1eXB0LpKpEdkjCatTGHdeGsr8jbM5eXgsl2qvQy7aABu0SMUb+THjO3ak6dg7WGDIwJncG9oXWbYgzWtDr23j3kEPEep6HgWZwI4vkRsuIsU9gr5yPU+MWIyXUTV6CIgoB6cil/2OMn4PV5RUtg/UUBj4nn3+4Zh76jyXkLCQptfxrKceg6kYqbUEqfYUklsSoSXFvGX8jgNmhbdqI5EszXDyNuSK3YiyDTeLalWOSc7njf5r0LVfgrKtSFaLff7LyNi4YoEnqt1596sD9AkNpXXna6xpKWFrK3xWB6OPwaEH9/Hu688jSRLXmqYyev9nbGi1z31DADcn3oyiKAS5BhFVsRSnTg3nUq8woCQcAsarbbJ2EK7TkVTzMG4dyTgLMFp7DEmyIs23InkPJUqnLtjt7XBHSRA2pygUyYyi9wLPJOpDn+f8MD23TLrEpr77EBQTbA5APvMAMSVf8NuQKtrcOmjzVB1b9UErkJNfoEOfxJfXRTDftbdAsHgNQU59jQumpxwUE3IPGs8uuacosgN921XEkFlIngOoCrqN8bviGfzDdURygmWR45Ba8pAW2JBGbMS/4ye8w9/j3KhgTAMCKNfNRdJ6qHIs+k6sUy5h1dkpXpzbybHoIPl5pNFbkNDR5LSAirzncOtI5iEPmGrcDBceRdN4Ajd9DQGJz5Pn8wQXohdw28GxWAQnFN9RFMuhHDWUURPYRlL5t0xt2tgty021JLn8QET7ESbvcOHBDx/ESbCiSDYkjSvS6C3sKp+PSVODIlqwKJCd+gPS8B+R9L4IwO7puzkz+Ayp6+djNIvIOm9AUalUfhSYaL6fpKxxxNR9hnunipyUJIlW/6msMAlsuOMbmjya6HBtw9+jHcnSqc57yUKCsZ1gDYx1up/ZzjoG6S8iSRKCoJDsf5CZYStx09fQYcjjdk0xMbmvoRh8aFD6M7phNW6NxQSMs5Bi/ASl8g8U9wRkt75sCPBggLyW+qAY+hrXME/MQaq/pL5b6T03BESk8j2wNRrKtlDjNZI1TZCZnMnFWVm8mfIlSsITSBMOIqHjRutFrrcbIfrU3IeidO81LkouCR45dBjzUFD3MCluOdLcJpTouxjsVoxBgBqPXVS5b6PN0uyYe7Is9d6nEHvtuTWa/hws8yT260X8umswB6SJ3bLPZuGHkPdZHW6myauJQq89SIYgZI9kiL0HecQvRJb7kWE4z8HUaPbq/8DmFIdkNau/9+jPwpoB/NYOAwywbuAbyMW/2OeRRE79EDLrxjnWiBT/GNLEoyhaV0qk6cQcfYS1TXBdBaxrcUHprFZlnEsEfvlQYoOnhd/xyHZFX7PN0UZz40GuhjxJTtDLyIpAkzUMKXAactQyJI9+jGx/lUnNfkSmJzP9x4UMM61DsRt+lfozSOP38XnechSgxKrDLAQgoUXSeSH5jiIw+hParE6IQKpbKW7te5A7q5BFA1VtUdy8qYGLJwtx7UhSZYIionj2w6oPwyIZqdFUkhn2mConRA1K1QHYP55gzQEWH1vDU3bga44UiNR3BVJbKXLlAZ4LLGek3ZC5usGANOx7JPdkpIEf4i5VssSOyrSk/8oT+ZnIUXeDoiB79nf0TabUibvrFbWtzmFINutfDgcF/c4wST8HOf0l5CsvkebzIw8GlBKmBSdTJIG6SCRTE4qpnta4NaQeeR5TSxqphV8x4UoBYNf3Gq/i274Vf2to98uV3rpXl9F1fycMLoUa41CUwvVgrkf0GcyBvJO4lOipPO/DA1suIreXweYAlK2xSAM/5IL2c+QOI7E5et66dgc1cRtQPPuhtJfA4Zks9wKN+T8JAGppJdbzMHcNeITBnWMBVS9J7/sGUtQy/EtWcy71AFqPMwAMLfJGGvAeSvYHcPIWPtevY4YzOJui6VP9ABH6AUhjdiLNt4KtjSc8u6PuFEQHxZ8sy0jmZqKsuUSEOPHCnXfwwuTh6l7epb+ee4iZJj9qJJnn6uBqQzJi6U+wPQ6n3JWsnZ7K7Li3u50KqMaPrjXWJRfc2wew2PNDHh7ycA89R/5L/iel35tIcctBUVAEhaz4XBKy01hg+h1J54M04hdIWM5Nyn5uclUNJwEXr2D0eg7C56L4jkR27sOXbQY6erBkxLpORbHTDUs6HzZpxrPJbi8XFI3a5tC52CLvRUF0ABYCG24kwaMfiiyhWFupEZK5+dBnCLU6BARi9O3QWY6CgNxnMX31NY4cSgB3K6+hlG11tLkXI4ciINlMUPwT1J/BbAjmiFUmefdkhrz2GFvaYtVxSn0FKeVlkoe8hbtpHQHNMzlYeAuFbaORTE3I195D2Teei2EyTgLUOzf85+ZfczOCoLAi6gB32pku32+EM9HPIaFHay3n+vgPie5bwAfBeTxZORWp4Yo6fyKXohOc0NrcqXc9hE/422gsxcjeQ2HaOeTkF/mjwxmzAjISGsGsghx+FJBzPkVQBFJ80rhx43guvrqMBzMWovgOQ466CwktIeZ8Jtn9g1XOZ+m0OSOXbYNffXESapmak8jODrg49BteDLsexdKAkrIKqe/TfBH5LivMtxFR/RAqCM2+D1naMZ6azoMBhQBkhzzHrj4DesmJCz57GFgCa5rgg2b4smM+UmcdXFmFXPwzq5pe51/2APNVDdAU8y7yjKtI1YeRas9wOX89ZpdLWHR1vJW0HtfWP1COLUSqv4zsHM460xCOm8Cr/kZu3XqNluktKKmvUOe+hDkJb/Gyj8U+RWzqGmrOQs7/Fp3SSI3ZnU4FPN3PgZCN5DUEaeSvyCkv8nhifzwMNXSB7rr2dTloJlLAzD8xwgi9zrkXomu5fVcZn9xzGte5e2jDEzlkNopLFFL1UZSybaR5n2XilWJ2e/ZliPE5JK9BSBp35OJNPGfbTJoBckNeosJvPRPqvVU2gM4qQKHGfTf5/qtxMcWh6ZeBEq/Ss/m0/sCO3PtIvJrKZbsD6oBZh3RdAVLsg3Tix/7CJewvmUNs+UqGuc9Bvvw8yoEpUPANJyPKSLGbN3a2i1h9xoNTMPJ1hXjVHeQpe+CHnwae8NmG8rMzcms+iqhHyXiZUYb7mOIkktHpTHlHHFLZdqTmHGRZxslSS7RdROWbjNRY+6MIIjgFode0MzXgAsvMXzDzrExfpzEo5dtU2raWDIZ55TqckDrJyyFXlX3j8c+ei6vYvQ8K9rHo3hvlXoCXlYF2Z3RrLn6NH3GDNIbZDYcc90X772WtB2LLNTb49eFi1CLSfV6i2mOb49uyLJM/8RwLq9T1aBEbsIhNKIoMHn35rXE7+2tTyGcPLU7p6rsFEaniD9gSQYrXJuLKX2BIzk5kwczOiq+xHZmLcvZ+htdt4CVPZ5xNMfY22W0BZVsh5wOuWLrYSOzGd/teLMsy31/+niWXwzmRMIrdae4cTIlGQUQa8TNS8ktI5kamGy2EaGGd3+N8I19AVNqhej/sHsiZrPVcf8aVOvd96l6oMdGcugla8/CuXEXmwO1MdQar2Myefr6ErAmgpbPFoRv2BheBVtA6xkMj9N6jNIIWua0ISn9F9khmWrEXxyQbklbit/BiOLEYOXAq0uTTBHfmcluP6BUUdZyUyj9QmjPZZfOjpoc6NM91PZy5GzntfYhe6nABN7icoMWYrrKOlO9APvcIw4N/ZIV/vSNftFFjVOs8PZ3GlN965Xx8PSQXrC1IM7KQJBu3mwbzdPLHjvu7zGOQ5nWA9yBkWSZVfx0JZW847jf3XY0SNh+p6GekujPcFjYVn2K14sPLxyIN/Q6p/9tIen9QQBEUBEWgb+BJtt4sohycqYK9TtzKNs0hbnWD0KZQQuJDmaL5rwFSt3RY2HT1SSq0PgCMd2sCa5NK8X9oGk/7/e5YywCKLKAoCj6u1SxLW07fyLX8UvcceQ151Locod3ijmirx5DxAvtu2sfIrBM4tTayd9cn7E018pMwA1mWGRk+kkeGPIJX8ygURSE94Cl2DTSyNvNZZFlGL6hKtFlqx2bXk0Q0SO6pSCM28EWTqpdYJSvRlf9iVN1XDAhQ96OejhWLrDolNNgpnu32wado6X5Gb6FLrsuyRIfVFU11A9IXbUzdM5VSiyvysO+Rkp5HFo2OgIIKn584FTsVN8NvKPsngakaaXYlnzap6B+pB4NQl+5W6/YH39QvpchXBaSIgqa3bmffa+ZXwueNIUgySHo/dJee4Fm7TD4deAcXI5bQYKp2tKkLnFbveoQjSf046HuTo62CIpDXB5RYGGmk+ywnSUhVh5DdEjgX/yqXndf1MFKp76VkI/LOfhQUf0Wjy6nueSAYkLSeSGjxMFTxzexwZoSv6s4nKNtQfnYhuOQODhffzCtHfqO2Pdjxe1mSYdJhTlrX8vboxewIhtcaNBTLs9X7LfnIgp70dl/cWoeSUPY6cZXPg9cE1e5yXSHtxgg829MQFJHLfe7isviF2qaoZcijt0C/VwnWiXiIsMJLZHzIbpSoOyHrHaTiTWxtNrPW+X6yQp8BFDU4w2sQ0nVFFGuXOPTsc27P02y8RIPXUqQZV8kfamN9+ovIsoLe6k182avcl/iMmuJpzxD1XX+WjaLWMR6Koo5zqww/tkKB5IN88Unksh1I8Y/zfe43OB/L4f6Pb+fLVruga86A3WkIopZyGziZwwmzRTA90p4aKOExJJ+RyEO/AyBI9kajaNFrOpFtJrttRcYqNrNb+hf17qrjdUu7D+cqpmOzqeeOMPcwVo5ZiY/Rh4d2L0dMcEUfayRr5FIURaGjvI6UQx8RUKAy1HW1SZa7fQhAL9tIV5vV6739DP8n+jU6Ozv/U236ryz/Jij8v1epqlJdCAEBAb2uBwQEOO5VVVXh7+/f675Wq8Xb27vXM5GRkX95R9c9Ly8vqqqq/u13/q68/vrrvPjii3+5npmZiaurParG25vw8HDKyspoaOg2OAQGBhIYGEhRURGtra2O62FhYfj4+JCbm9uLqjQqKgp3d3euXr3ay1AUHx+PXq8nPT29Vx1SUlKwWCxkZ2c7rmk0GlJSUmhtbaWgoMBx3Wg0kpCQQGNjI6WlpY7rbm5uREdHU1NT06sf/qvalBB6C03teuqvXUN0uh9fPy/WR+/i7rz5VJp8cKucRMHQAjrtAtLZOaWX0FVkuHr1qqNNx+oGs/xAX6ZtWMCJ4Wsp7ZxBQ//n8Mz9ElPeRrb5l/BBaSorbVd4wAM0B8ZR5Xs3Pk2b6QhaiODzPjabetKyWSUKCvKJjXUnMDCQ6qJLjHO+QEyrGpGxJmkTlooiznmt4a0vs0g/MgFJasFZA53JZo4cOYKXlxcwjPKqRiTJgF40M9wIhtZyMrOL8Pe9C9+AyUA3he1LAfV0GmJwGvI2rR4T+KJoCBfNSwGossGNpYv5cvwDlNjHOzJrEfPcO2l7cTmZyS00RbWS71JOGJ60t2vwHfQRllQtukuVNNbaKKlopNbjBgKun09RRXeOwlCPct5cqHDdL7DN9ve0ESMAkygwSoExWj1LrWYCgD6odBOL/+Y3fy7fGo2MHhXHIr8DvNsKgaeOcF2kHnnnADQtmdR5zaWxPY/QkeVsN1dTaYKtTUO53vM0Zac/wckjCD9rI4menVQDh4qHcfRCXyaEqYerymYNI/JW8HOwjXtq4Pk4Ne9mZ/FOspnPycKBZORvZ3vMYxQNWsA3bQGIndVccnoEW+dwdh3eRGNHEC6tKeglN2z1l7E1l2NwjWFK01pGO8kctU/htja1/7rW0+n6THYNuoUHhRBe8Ssnv72U1kNP49l6kKyoTWhsOibvm4BFbyFroIWrJRYCvBeDLZCBVR9zfk4GxpQM3vCBA8W38GDh43D1AC1Bd1Pmvog4Yz4t1hgGpx+gIX8zb2Q+yxFJYuifDMeLberB4boXXqOyrgOr9CbtVncHzn2AMRXhynrk5OlIjS0Mjr7CLkMsp9Jew8kcCVonWltaKKgN4GDNKc6c8UGfmourewa3GtdTc0GkItCJxNwZrA80M6gUmorv4URjPzKHl+PtuRKvSC9CgceOLaDDtYqcrWMx33mEPo2bwFqDXLyRQtdFXNZe5Piyi2xJWM+O3Ie4KC3FV26luuoadHhjlit7tS03O5d6w3SOtyfi19zMgLMHaHW7TF7CcqwWK7IsY7FYaGrqgODeB9TO9k4KCwvJqWuio96b6LxIyPsDp8k5lBdm0tAUB4rCYzU34vqjldjCC7zwyFoKy9/i1VPvElz7IYz5nQppANunDeTywBlghfWlQbTd9RJF+fm0pqdzsXkl6076Iw37hVM9gluUS4+S6LWMn6KauKECttiX37VKJ0TvN8gqzqEw6nWaLrWTcOZH8oJvJz29mtSO1Qhlv3GDfxulfZ6n4ZiGjLSL/Oa+A+vREkq0ExHlNjaHv4wtslvuVpSX4lx7jQaPWUyo24EpWsOEvi38Hvw7G0rGsmn4TJL4CDpKOGTzJe8ERFQdxKNPCwkR+WC9g6KSWqobPXivoRP6FDPapR0vszOHR1zj6qWDWPTBlORbOJxyiC+bYXNlI/ce282UyfVMTk/Hau1Lst9hrot8mXfrz9GhK+GlBngv9RHiTSZa280oVQruVTn8dm0BhvhibtMcx7l8KxZdKIPdy3CVH6Vw4GL6+W3Br+xpKHuazNg9aD379BpfX0Gh8epPuHiOwskjkfqcLAIrAxl1bBSfTTpPg0ss46VO6i+splkO5BPrBH5pOEBM8Yt8HZvJ6D4a+BmqfO9hEn+wIAw0eXD3xHgsSl/S01X6nvjk1YzbGEST28tgzOdc3PWcubaTgSlTsFgsZGbmEFCxiEaf/XQ65wNCrz13f/0Sfj6kY2auH3G5cZgnRlN67B0iyp+mxW8e+1uHs7NdBRkNNkDxxc34+3sTFDOcsqomWqyqGVMAhvoeJ+aMAGcgK2ojJmMMFrO6h+Vb4e2suVwf5U9nejpFRa5YpGDy3H4m32cDEW1zSE8fprYpPh5BMGA2y5gV1XlvMzsjJz2PNWQxeYWl1F2c7EB6uoqgbTmLjJbCsPdwKvuIYXFZ/FofTFTJE9iaLnO11htZfxfJO1QQzpmgMsrjcokdmEVmy3C8B79HeIg/pWXlrPntSXaFvYcGCNdK+HOY1sPf06BLps15MOHtSQTVLaQt6Gs2DvwC6oF60FZlYrFeQFa8MLc1I/dRpVxxYSEmvyoE2Z3155Zyj/tjjnmiETSYXZKoC3yKzJxgOjvKSdXD3hDYYr1IxpXz9MsaRZPbBK7PSiJLn0lM8cuMsIQi/RpEld+9WMOXEN3wHc97w4/1friUTmT+4KWImzZDzD1Yq89gADa1qnk15rGeh9KrePzWbNwPLKHJfQrhVXuY4wK/tcP3tf7c6G2jtd2ZwOIXGOo7niWe+VzJGE1B5gqkYCNte2/AveMkytgOgrKXE4Qa4dpKC7Lsiqm9Bf2ewSTIHUQIUXTF6Siy0ks/bG60QQ/jW2lxCfUeIj6ICN5pdHZ2cGZgKL4NRzjkdS8rC5YTYKpBoIa1F87xYcb33PHFbErCgrk0TqC8ooym4HU4dV4jnr3s7xAw65X/HADILxJfzUlmJ6zla+twAMwK1LkNIb1Uob3FhKHHNpdt1pNe0EqA2R9PQxTlbW20K50E1N5EWO7TLE2bj2bjDKqj3yWgcjMTjKqwDddCtNZGZUkOXl4Jdh1WpDHgDNe+20ydZj76x48wpKKMzk4VCNhSr+NqM1xVGrlY50T/8giUTBVv3+Y0AJ24hzKfLzikdFO9Wc1W9KKeiooKWlvt3qPWACa43ce0CLNjPOrrQ5D9u0EL/oYgrrT1B8C56HOyI+CFwDo0tTDDdwlZF5Zh9I4lOrIPL9sW8k3DT2jagqAmnM5GE3j2Q6g7Tq0SzZP1TrTIqoFqjBOMsn7NtZBPsOjDoAx2SwPY13EQAMkmk5+fh07XSXBwFH7OjYzVGjltCSUo/0GkagFTkgH9zGze/egTfg65h4QFCYi2PqShOpYFFEqsfXncaz2nTbDR7khc27yUqZGT6ExPx2g09sphqCgC6deKcBuSRXRUDAVXD3J9qcgU1xa0wTqSLPOhbCvmjPcp9n4IU2M6NTX+mMzerD71PcHBjUzeNQ7XzssAOAn2vNnOMlW9wcr/dv4ZDAZaWpq5f/c66hUrUW5ZjJupUGEZiD7zGmdLFG7ITESoz8LQtpAL4f0pydxLZNmrlHf6cJfbWOLTx/F5yFIyg1+izhqNpslGUPRAaiwh3FrxNQ0U0xZxCwZrN6i0tKIWueo9fK+9Snjxo0AIVquN9MYYoqo/pPX4a4xo3MwwfwgrgrcbwavhDpZGXsAr/A7aCqOQ/wQqNllBr5PJKm4jr/BJcrMnkxH8KBbnMmbV/ogx3YXYgltxMWWQ6BQC9mAPRYKaAw8hi0ZaQ+5G0cRwoUcgSFNTEy2nnsOr8jPQeuHbcIR+5/dwedBUtrVDZ5+Haau7SOiZ8QC8BezRwRULNFp1eOmsCPUnKbp6BM/E0G6Kqaa+WBqduJpdQGLiQ7R0ujMk6EW8PCw83QgtpkpMpnAMlX8gXniYktr1rDi3jMKYldzpd5lb/JaQf/EjWl1HEFBbjiJUUOb3mcPQmZ+fj9FWi5t2GmHCSfobFM52MTKbrRQVFTnOuedzC9mYPBFZ08nNVbDC/UF8LZGEtRfQlP4Nn4t38mzNSnzipuLnXIdiaSL7/A5c28/h62zCR2lzROa1G3K7O8/aQmurE1/H7+W8AJ/ld2KxmJFTXqewsy/XisNobW0h7soXKD5lnIu7HhTIuHwerdRAbWUUx68+RWZaMHuH11FueYCm0jy8W/4Aoz9v1nlTIdnP4oqA9uAoMDpRM2AnPq15xNhV4RQ9THO7DBIUKKNw8vInuOoDonQwt/lNTOfvxT/6NJojKq14a8qnDCh8n0Mh6vx7sTyAB/12YTt0I7qBb3ClLJaxZxsJCrLS2tqCzeaEnLSKHGbieeljvhy4jiNF0Fw7i4HZv3Pt2lWqqiT6W5vUuWrrpvjr7Ogk3S4jEhIS6Ohox79iMWV93u81v61uKdCaz4/HXsbTVwBv+7w3dZBxpRijZiY1XoFIvOf4jc0qUVxcRHp6K2FhYejMtQw1QkZnO3vTfHjHF8R9A2DWeWpqqihw/ZJitzfQOKkbpRaF3AoL3t63kn0yCE1VAp5u7hxPGMk7eSeZ2O9m4p0GYnXy4JddCSQ0GziZOhRFhvT0dDwlCxFh8yi1s2hIdrnc0a4KqZqaGgcTlYSZZe7weUA11wp+IcMzEkNVKS6dO/jer5mFNthgq6DecJqNjV8zIeUYwbp8dJZG0vQdnDTBRGcY7XWZEm0iHieX4Fd/inWV0RRb8+m0NWPV1VPTrtpPjBojBQUuGBr7EqC9nnq3w9i0zdTX1pOeno5Go1Ej+XqOgdlKXVMH/kBbczWH2ozM2jqXlIwUrnzwHhO1p2nJ3kiRMAZb2Od8/ZE7nkEf4ep7lO/dnsG8+UOuxfyG4K0gnb7P8d5ZLpBiuKSun4qzuHklMbJhKze7wgayOd9nISbTGag9hpizlgE+1zMquJDLG87TaOvHwJk49tbmOjMHwjr53aLKzZ0N3gyMmkX1lTO4WPIwCPGkm9VIu1gdxJrPk59zhbjkoTQ2NFBR0U5rq2pOvdkV/M5NQTCV0VZXTFnQM3xTEUXhpP2U6cyYRS/SMzIJK38Br5a9JMa+wh/xD7BskQsLw9LZXzyBhCH9CGnJhKZ02l0GEFR8M3d9pW6SzaM+ZszJesZotdxuVu0qVUAT/zkgtYurF6uz5tD/7GGWlC1l1Xcv0pT9CIx6EavXMIZwisFyCPmolOUoAhaLmWTPn6lpC+H7OiewpwgQFJHW1hZyLCPwCQzA1tQIijsphz+Gw+DyiAsWJxuNjY1Mi5lGuCWcy9U+tEqtmK1qJLBW1HL16lXG6ZdxLctIo3cnHmI0sfUrGeytJz3zGhDPpXYjiGC2WmhtbUGrtZGeXopGoyEyvtt229yuyjjZnsy6sbGRlsyviR+QS/HqVWxshZvdwNN8jPR0d8xmAdFYSOm4N+lzdRpuiHwaeYKy4lSaG8PxCyplmHScsU5wuBPqPPZi6pyKUn8ewdpCbmkTJkkELcg9aDyvXr1Kebk7dcJ58k3foHFXHYIWk8Ux/2pr/cEe6Xfx/Nv4ey8mPasIHSYSe0QxV3j8BkBh2TIggMbGRtpbVX3VanduCmgcdtiSqhIG2n/uZPPGZDJRWlpDenojMYXLcZLKeUsZxzHPjY5vNDTU09iox6fhImJzBisy7qLRfgZQZDAIjeRfOIBJDKC+KZwvSwZwXNlLtWcOAJUVlbQ6DaC6zYvS5r4UN8bT1t4djZ+bk0vgoEDMZhNb8m6gKOJdZNHCt2WPcHv9EsQ9g2l3SuHd3VcJbdSi0WjBBerr63A7dwM2rS/ZQVs48nEsARGfcDXscfRtzaSnDwYgMHAIgemryOzTwUM1kH9hNet9Spkx5ja8jvSjpaWTCnOq2u8ee7gUfif9xa8d9vLW8hqu1wl8L0Cb2Ea1eJK8PAM2m5mKCh2KkojNJtHRpCGg5QFGzy6nPLiFkJihdF7qwGrqrWRpRa3DXl5TE4Di0T2eFosFU/FuRLmdLP1ddHRE42fuQCtp2ZrnhxTYhsYeKa9FxkcEm2QhvTSCtcff4OGRT4O1lfLz62jQD+SXjBKOnnCibmwqR+MrKTm2mPyQJ6iq8uP1EQvQ+UBaKThZg3ijKoGgM/eyInQlHDqPacgGsnML+f3y72wq2sTewDRch92Ee1MHNNST+8inap85ByAIONpUUuJOa6sPGo0WFxcX2tvbSU/vji5tagoDfDCZTFitFnJzKxEE0/9xfg1PT0+am5vJzMx0MCb8uzZ1OQj/q8r/b5x9/18vzzzzDI891m1EamlpISwsjKSkJNzd1QN+1wQIDQ0lJCTE8WzX9YiIiF7v7LoeGxvb63oXJ2xiYuLfXk9JSfnLdaPR+JfroE72v7vu5eWFp6fnX677+/vj5+f3lzr+P22TorxDXUYGiX37otFoELJOYyg1k1r1Er6tcSgKJCXKdH2iqEgktfhLmp3PkR/0FhpR6+gPNzc3pvruxT2slRzXViZF/85tzU+AsQzlpiaM7cW0br0dp7rZ4PkEv7eJPOYlE9D4PXil4VH6AYaSjymIkPBtnkRo202M9t9JwPmvYOw2AlIXE7rrZSqlQsZm7+f18jqMLQt4esN5xuoMPGc2EYiqmH12zMS0yZN57Y03eHT5cvqFX+KdtFA2VsdzUwBsMeeT7JoOrmkganjKC1Y3gg2Y59mO3jMawm7ATVGIjIgEO0Ng/9zfiB4QgadQgadrPkqfW9BczeJy1hgAhjeKzPPdTkDzMRA7MAy4n7dPvstHE9fhMcqVgTaZ6RFu+J9+C0ZvJkrMY08w3FkDm/2/YFzrB9xyRy5jvv6IcTodt5lMDqXzMy2ckeG5u5NZ+twuXlj5FU/88iMdQi6STsbUoDqY/h23/GngsNXMuuu0BLe1sb4DjM4GVgwZjbZFVaT8Gn/BDxjkAV3Wys3SImbOP0r4rlSob0awteE23JtvPL+gxutNlrTkoDgFI0TeSlDyDbTnPIWESk865+oYZnp9xB3LnEjprMLZPY5fjM9SrcvBRQA/jYqm6tf6Oo0DfubTLbfQEHOc5OIPCeu8DvG5yzjtGQBn78YHMPSIm77ZPw+2ROI2fg8pKSn4+lSDGba1uiPu3sZjKyNxC5VQ0iuI7ZvC3QcEfE+rxvXlY9eTmJSMuPlHcO4g3TWJg0WHkRVYXgdplTewLFSG3AN4+EXg7m8gZeidPB8agvulF1h35SOOyMq/Tdq9zSoz5v33WXrH/YwilnKNSuEZ6W6kT8VzKJ7tGNrqeXrC51z+5Hnmbr+OvDnpIJkdMqKoCNzcRDRakRoJPjA9wv1D78LHvS+C/jG+3fsC0EFU3cO4mfqSlCTjXfUmQtEB8F7NSsMWNlkm0lgRzvZzX7J6sDe4LEGIuJVh/uO4+PpKgs/dyXmjSCMdjI0IINzHjNQ5mSd+upk9aZ6g6U4ykObfiHv9FjINj2GQwglmHLV2lKyLk4tD7nl7GxzoP4A4z0Tc94SQGnw9sYM+ZofLQLL7bSSgqJiJRf3JmBpI+PkByKmvYXByxWSspMNZVbT9/QIIsqj0LVLtcR649CaZA7sjvjWCBi48QVTNUeRJx2luFnBzE5E1neT1YMzRpq6ibFMET1a4k2GxUzsh0C9VpZQbmLEecww8EVhPR8VYfHx9SUnxRzhpBedwnsgJYGviyyh3ryCl8U1gB1qPGKJq1yPUHWexuzPf2h2IAjCq5V101jpCfA2Yo6Zy6DMdQaVxbBnxOknaVBJSRyL7rkf0SOCNvEouzt7M/Z/cT1TWCR5c8DpY5hAREYF/iD9sh8SriVQHVGOOqODNOg+emDMU9J6Iipl3/+jLYfEaLu5OdDj7EBTmS0oKnD0rsjv/Xt5xXcs1z13YFDjUCQ2+4zAajfzEXXw79hMG6s/g0nyS79uCuCdyDZLXKbTGAN78wJdKkxMIkNs4hjavG3Bp2U1CXJSac7VHSTQq+DZuUv/jPYDhprf52tmFM5nJ1GQms3lJKM9U7cS/8Dn8QufS7h1DedEBRji1MrrPL473BNZ9Rr52Pm/Xq4rjIM9yyqwaUl0vIpy9G1rn4KKN7vXtyIjIHntuMgOqPqWufT+nEyYjIPbac23V5xg+rIQ9Zf1IiLlAatgZwkNDoBzcTOl81XwrOzpPIgJnwoHyRVAOXITQgR8wyBkOt6j589albnXUIbb/ZLA28bBfNT81wxkzbKqaxWOD5wHQ2AivTJxKuN9+Aguh1fsUqe4fQ0cJgrEfRrGREb51HNFAlQRenl6IoojBO4a+RjeS9grk2SBYA/01qhIrYiMicTRy+QqCNGDWV3BtcBy7+pYhS/eh9FmEJktVkJd0tMMilV7so8A3CQ0LA1EkNDyKCenpVLvBsU7IjqzCbJmLoaoeT0AJnEK8UsR7zsmcU+CtvGk8FaMazdwsGRj1MqHul+mUfDhhl89JYZ447Qgm2OdR3NxWqwj6LhooQcTgm0zQ2NdwuiawpO991LpAthUSdbX0yxqO4hKF24Dl5B5bQyMl9DGnclviZvS2WkJdG6HiafJ8FrAo50UQJYJ8bLg46VECJiLYOtALZgomXuK2z/o7xkdAJNTfBc2l3Xj0GUFwxRFcPVVn36Mlidjc93Pb6Eak9vv47V0Xak83Y2iPJ1R2JyJCxi12HrT0xWDOY1xcPhk1YzFLasiPKIJea0OQO7AJblTZ3BwnAa1W20vf8/EuAyvE61TjWnIIeI3YgCyIaFoyuaPf43zmFcwJz++IkePxi5sE/vuRXWNxvraTQuESkmYWzpHlJLgXEumTQoBbFej6QCHsbxdpTpb44pgKPPlfle8MWmYvWsSl8vuZcETmePCzXUE5OBudSYlMYa/2AWb8qq5RvQDBWoWUvjGIhVvBNYxX2mZxuHMt0Rpwc3PH11kN1QzIfxzCF3DTtQwgk9vc4WWfFlq81ANnlw67O/MEPo0g482H5akcDAslMlJ1Yp3yWsJ957/AyRzB9CsFIIKY9g6yVIMu8A5ueXE+u9PcwL27TS7OLoiCSFhYCKaQfaRH3INHexqiuKmXXu7nJ6DrQZ/+6eQfHffyWvzpaITCwZeYnjqU4SGbkYLvQnM4Bk5DjfMLFNogXK/h7uHvcaPyEdgOIg/5Eh/3RPTpu7HbpxhggLHGvUjhz4Kf+n6nwu7vajV6YmNjSG1fgpLewPjIEdzaN53H910im1QSEmSMRnVxhQb44dcCTeHuDLy6C58GUIZdh9JwgdCYG3kz4xhnTOr6TNbDdR4nCA+aCmEpjnXQVURBJCU11f4fDb4+vri1ujHs9DBCRtYwK/IV6FyKse0CsWl+SOdu5bakMK5XvNHIBkJcn8VFa0LROCFPu0zKBymYFTNO0Ua+aDSx+D/ArvOdkxPzb7+di5c8qGxSacUSW+DVCWA0qofzmrY68guvsuqtVeocfO0Jwgc3IQtnCdIWMtltDSlDA/iiQuCYCco8ZjHEpR2srfj7eKLT6sEGnc755NsMKkVd/SlCIxNR9N5kXOuDLEp0jjrDLeFGUpIeRyy4imtgP/ZqZ/Pl+bWO+t6ZugY/OQBGnsVaXMxzviK/yp4MPjuN8QELeNPVF3I+oq+uip/Nb1DSIWB1updOQwmiTj0rCszG3DGOVX/0xd3/C1qcLyGKWoJq1QgFadzbtJVncJMr7GqHR71gpOtR3GOfAttVSHya6zsOoqny5bK9XhqplQAfJ+R+b4GtlV3Z26mVVEqtZbmj2ZR4AMV3JBFD70ToLONll924u0KGUeD6lC0McK5FX6FFq9zBy0d3sS9NjU45N3C8Og4hM5GcQ3E6NQKjXnWMHO6EGpsb0SUPII3eAkE3E3T0OLWBKx39FRsTS0qCP1TtQ3P4BYYY3Tlr15Nc9B6O86q/vz+Dgk/wYUsna5sgy6rOz5CwaCTjATyNAYSXXES5qNKHPrb3Akuuv8RNeWkAyEO/YXn5Mk6Y1qj9IerY79fGhHEgaJ3w8BBIdG2myQIWz2zaaj7A5BJI5PC7aNBDfHAecT6ZnGyM4RzwuJdCvyxVw28Ne5fjZ5Zj0tXQAbh6eOIxfRvKRmcEawtvN/hRL6gGqh0hEgZzAYgh+FtPUznpLB+8tBF8XyTdAi/X3sSzt79KhGsUmGphywdU2YysUnZxYsHHGLUg93sT8fK/cG0/x1W/KawrtB9CRR0l1qlEOvvB9gRuSn2GT9pexcPbRHO7gFanQXQJIW5QEFJNAK99KFBr/AaN2IZGI5KYmIivL0h+u6mpN2I93E0F5+Li2mufcnNzoV/Ne8yPXcZqSzILS11Z2edB4tPGkFE/BYOziKdXt4KdFpxBv2vXofgMw92tLz3NajqdgaioCFJSVHuE6677OBUGA0skLpjBTQSt0gbmeubFvEe9UsWXdKcIWCCWklCwAMUYSHzQFM4XuqvUZXadwhS/gPC467BY4OrbGjTOZ9XuEjT2NqVAdiFjnXawuw0Erfo7N1c3x9wLDw2HS4AoU2iDzc1GhvVPIMBfRrN3PnLUMh5o8OSkqYmtQZAadpycsEiihkYiCAIRO9P4PQgCCuG1BjicvpqvJkWD7m6qnebz/IkrlPnlE23sji7on9IfnUZHaysk/PokQsuTXIy8jXKfHwgPCXeMh/ZwD8c14Orsin+5mm/V1TMEjaAhJUN99pWaQCYOmIl7QDwp3hGg6Qvv6glpriXNXQaOoxctpKSmIRT/xA63DOa3wFETtMhQbIXw8Bm4Nu2FqjCmcZJWV9jQpq4pZ2cnSHwaKfpefnlM4MeMV2knBo1GRKOx28LKttCi0yKU6ChomAbsZnNrGO8Pehi/iu2IWSvZp3mAj+20mje6wvO+m5A1I8ESj5eXFz7BIhrfMgA6FVA0rsjJL+La52YSXKP5boeR2lFHidDCdN1MUt2uIDZvRXGNxq9lL97aozT5jeUl427Scr/hzJO3INueRdQY8BKdaNghcGzyaCacGMnoyTJPfXOML37cy7Mf/ERzex02l0baXCU+L+U/to8ZjfSbNJn1ISOY+usqAD44/Tn3TvwEDkxEM/kkv2xSUC7EkTxsARmd+wERvd5AalgW/q7lVFgHgD1CWEAkOrCBZNM7CGF3Yw0Nd/D1KVrocO4gwOBkB7VDTEwMRmMbouiG1m4o0YgaEhMTuTf/Xb48JhKRoFBXJxBfmcydt8mkdi5DKNlAoNN4atr80GqMOHkIePjpSExKRCNqsCrda1znon7fqFeda15eXni5qXLvWCfEdw7ni5iTbG3ZTkrK61gs8MSwRxgReAq972n66wVWu2lwr3wZKl9G7lvJjbafaXRT9xKAau00mPcoIBHbdI1wvUiOrKYx6dmm9HQBY55ajy6KT1dnF8eaCQoSEKtVZ98TSXuJdbeSkvIUwqklCK25vNrYe/xCgsMcbfL29IZGEHT2lDGKxmGHLXUqpY89DNS1MwB5cCoDfD8kJWUS+L2GYGtFPLu517v9fP3Vcbp8DlnrxklTt9NCFDSECjuIy70VadRmtM4jebY6mBrPHXi3jWV68lCmDJyFS/CLuFSD8CVcibmdNmOW4x194xPg6lvEufTlrcInOej5LkdD4YJ5K9SooFynoGF4e3thtUGr0xVaBBFfo4C7HdiaOvQjjEajI5qwp54s5H0K2Sro45wZDM1T6LQ4s1A6C1PP4G4MY/mOBPoHw8wK0HicZIDHOhIjJiDUHMTPfye3ef9CQfEI9lhO4GR0IS2kjFDhCGFefuwKvsKPPu9ikL1Iy9xLamgOQZZmcAnH2dkZg32+dRWtqHXYy4OCBDSdOiK1cDQMdluyMEw7hLitD/3Kb2Xy0AJ+bu5LyMnBSBoJecR6NMdvRIlfzvj8V/g1CMbJR6iJ/pZf6ocxO/AL+uQvI5wvCPMcQF74h1xMTyNbV8l3LXBdv3GEJKQQEiJQ3RZDm/UkKCK31pYxdChsdhPo41kFNUdAtKIJ1FB9UbVzjtNdoF/4w/zU/gme7tfjNiSBjPYILGGJjrnn6elJYaFq9+sqbm4uvfbmc+cExxgZjUbi4lwdeyv8n+PXUBQFDw8PkpKS0Gi66H3/uU0Wi4X/yvL/GxrPwEA1gL26urrX9erqase9wMBAamp65xqw2Ww0NDT0eubv3tHzG//0TNf9vysGgwF3d/def0D1NHf96Rp4URT/9nrPa//ueteE+7vrgiD8h68D/3j9z3X8p+v/VW3SaDRERUWh0+nQmKsQw+fw5PEyKlrjgb/WXxQFghtvIqhxvtoOercpxHqM2xOqcWtzozY7ggyhH2LOe2guP4XGPYo/lCPsLlbz5R3rFGHCHwhjtyNMOQb9XuOgRVVa+tTeT0DrdDQiam4WFDQ6PRY0yIBZMbD+cCmvb9RwRFbYazaxGJiIGt12xAZHJIk3VzzN2mUByKIre6r7s5tsHq2FS1ICYu4H6p+yX3nTFwx2YEjo1WiUod9DzscIP4nclDeeYUaVniqsdQZ9de6IWW8jnrkTjShzos863gnwZfcckXlPbcEglSOEzUGIWIgoilysuohrUwspzc38OvolwjVbEJwCwVSF2FHEUCM4C1DicpqLJh0PX+dD8U/DGfPsszzu2pfb3bWsCIBjo8GyHMY9/REhISHcfsdK7lv0Ds2fyjx6722MnrqC6/T6f6QBPQ1cp4U3FigE951MYm4o6RZAUNiY8yb0fVJ9MH45v6V8x332JR2kgcGGDDTmSoQBbyCE3wRhN3Lc/BTFIeU8F9bJxJBvEeaUQ/830Gg0fO91KwurVOfp6VZ/6m3JaPI/R7N/FF7Gaiw6NVrMBhzT3qXOn/YCNBoNZl0NJkMpkrZdnX8uIWqOJuCi+7vs64EW99G2Q3sRQtlvvdCPDbJIQeMARIMnmoj5CNdlobE28n54O6Ybd+IzvJHhAS3qOk17DyF6KaX+M3gox51/vfwc12+5HgERS+BNsEhBTH4GjUcciyoMbLSWU1CwmfGaf3b0dZWhwDi9npLcN1g7bZAjEfCPZdsoSX0fIfIWsr0mM70M3AoriU9P4VRsLjRnqG231GDofBl9yMdqXh8FspV+aLyS1fWY+AQVrCSufBW3xfzEd3OCMViKEJvTEar3g2whQSfhlhdJfXAUnpZxWJxSYfg6NEET0Wg0yJW1pBzpzztH4thSuIDQjKFodiUR0P49giAQV/0MMRXPOtpkbLmMmLOWmc7THDLCYPNnkPNcJkVNcsg9URQd3O8A4e6RCH5jED2TMBSs4bs5oVhGdrJm+RpMbjZEJzU6XMz9kGRdHUUz/8Cr0Yvlv89jrvAtwuhfYEY66DzZbjjJnZvnsWrVKq6zufK0dytkvYvQcAaNVotQOZsRI4ZgMpSSboHd8utwQylC8Awq2vryYYMH+fbzi1bUomnNQnPlGdzFZraWeNBkbWXXlG855HUfmrJfEWKWIczK5HBHFLlW0MquWDuT2OmpqGuin5qja4Zrt2U9QQ86q2rwFr3TIPputm67jZSDGkZi5AaPAjQiiJGLwDsNjaihJqCGE8NPsCdxD4+cfRuMAWg0GpwKPiOzD8zfOJ+0i2k0FgfxQW4YGicfdd4bnHm3PIVt7bCp/VHKR+rI7XjNIbObTIFkmfWYezAYaDVaBEGgUS7DtSGdIWcHMvjcYKK0TQiiFs2RWWhO38F5+XeuhT7HsYRhHIxcxD7fZxAWdKLxiEGr1fKkrYURWUcBSDcb1LxlsfcDoFPMxKUUkB+dx+XUywiICM0qnaIQNodAvQFffAh0VtFde2vfgpuaYWYmZ8UPeNt+GAsvgqdqr0OsO4ag2BBKNzLUrQh9DxZRnU7nmHtarX2/tt8X/rSXBZu/Z0TMNrzG6Zg352t85FOIkbfADeUIU473ykX4ZC1Y+74EARNB44SY/wV7/GuJ0KqRTztr42gMfx0WmNAYPdCYKnjUo5ApguqMFEVtrz00t+H/Yu+9o6uq1r3/z1q7Jzu9VxKSkNAh9N57FRtFsSE2VLAjiCiKoCgiNsAuVZEmvfdeQgkkkN57z947u/7+mDs7iZ5z7z1j3PG+7/3dM8c44xjWXnOtNeszn+/3+T6JHDE2vbecvR759odIkkSg8ipbumxilhfUxMAMt71INalItz9Cce1lbkXZmJz/LVM9YFtk02FRUX6Gc2128WmV+LtCncfmyt7I9gYUSa9B2ESO+f2AyqyiVVYrRlw6RTufnsiF++Dyy8jGfIbUn+VpTzBYNfySOZ5k9Qro/RMkvIrk1R6D3UaZ5xFMDvgxdzDFge8C0NBxNT7aIlaP6UKnXp3wre/Lgl4fEemXADGzMOl70zbgHK94g48MaoeeWN/YZv0h81LCjwzQwqA8OCaJ3BdSx0UoQscwuWwrE28UE1wzkTCPu6ALRQ7oi5y7FU+PtrhfTSIhbymB0ctor9+MFDUD+v6C1PNbWh/pgiEWBumgjxbGBH2Momg/jE9Bbj+ffeEvMq8UNJYgOuSsRqmUxZzyT6TcEs+9ip6USZXUa9Ow0ICcMAep5zfoCn7gvSHjcPNM4kzb3pyPH44sS0hqbxh1kdtR16m1Nh1OZamlzaVSiH2qswZWBIC3NQNFdRKK5Peg7Cx9fQ/jXXSFRXkJ7I66i6z1g+ChyPoIJuV9SnprB2veXcLqZw7wYNAtvAo/R3FiDIr6dLi/nO+rlSi78R/momksF4DjVjuznn4ai8OTsgY/zM514vNOy+l1ZgiKW4tRKps4jD1VOtLjilFkrEXq8RVS5w9duVIsykoaVMVkS+OaHlJ6Cqsz59CBephXqEd2C3fODxmF3cj64xPJ6HKdwuBCQMzhxvZSK4X3q4fOwMcjBtAzbA9SxBTkNs+iUEgt1CbcTK1J8G2PQlY4x5iMXTZh1GRR7X6ZFNNxkkuTm/WHjNSMn6lRaVEc7o/iwmNUBk2law4M/3wWJ3YZGH33IxS+XVy/1Up21BLkBKwjRXuZCkVrKNiDnPE9iuKDfO9TTYDz1dZVw8CitiiOj0SxVY/CmMMbDT/wrjMqRkKBUqlAttUj2eq4VjiKby+vJtNeRZX+PHXWGiSbESn9ewaznZLWMK1EjVdpmhh7rR9F9m6H4torXDW3Jtu5FellCKAAhbWqyY7/S95theRAUX4OatPwqr7G22Emrg+8gEdrK1Z0EPcMkndHFKmfkW2s4ZJtH1X+G3i3/Q9081uPNCEF6WEDCq84hkYPBeD5Cy9w1ib918af2czTs2eL/pJtKGQbIKG5+jiKP7xRSKCSrMSrwKq0oNRamBXaGYXOF7nmNnLxQc4Zl7MleSGNKHU7wzbka3Phd0/ko4NY4lfqyp9YZ1dCxGQIHoHCuz1y8CC6Fl3n/UVLePqJA0wPv4Ci7DhS6DjkDvMp9EzkD2eU5Hh3yK7riNTjayRJwkPK4I2oC4yxDOKy92aOWf8U+1DOb8gZ66iSMyjw3YJBm9E03hUK5C4fYOuymqiy5+mYvcbZFxJMtcCD1SgUCtpUzua3EHjXTzy3p+omivDxMOIkcthYeqvmMqa1IEGdDYfA477I+zoit7ofufP77NH1p9DppL5Z509+m/1IfX9BoXZDtptoqygiXAklYct5te9jaG69hnR1LmHl72Cw+lLRDK2RJAnJozWKyPvQynU8HXWcjmpIs8D66r4AKE5NQnFy/N9ylKpUzj2wYBe28IfZViPWxgG3rzDe8muLc25i0f085w29tPCSN3RxnELO+A7FtbkoVDoCGnJI1EC551EuxY2nQQ6B4JEQ9Qhy8FDXWRdArVITZDuCVJ8JkowsS/S9Mp2HiqDAbzOeuW/gfmEmitsfoJZNJIYc4rW+jxLhIYCVAEWTwVQpdeepLq+zxA+65MBJaZw4P9xXCH3Xt8gfXm9X4FB5gTEf+cxDhFWdonPVYiJLnqHUBqcN7ZG94lAU7kaRugLavs7SCgc5+uNcKh7A1fKHkdu9BlNKkLp9SmHQeFZUibof9yujyDYAqfNSaL+QvPqu+MW/yO5QN/Z313G2dhOSpQaFuRRFQG/+zHyEKjs0qEposa6GjcLuN6iF3LksNfkTAJRKGUmSyHGIWbzZVEe4fRty0mu4W1NwhHzPTfcmEHxOGwHUS+Xn8axOajEGGte3Rn+EplpEDOQ716lnSsA06gZYqunutoJe3jlAU+49o/NsJ5mKCHW7RYXHaXL8fqRGJ6BupULZ4t0dThBQotk3ZW9hql5E9DXmLZPlprHXuC87JAuHDTAjNwC7Z1cUh0Wku9z6UR4YdgwKnifFAjcNXgRZ96DI2YwsyxSE3M98p6xuuR0eijyM/2ENhIyiLmweCQVLGXHnHpFlT7cYoy19TJILuNAqtU02Q7PIPu+63sR5dRA5+1o/jtxQTG3bPPIf/Y09Y/dwy6GFfhuQ67NQbPNBUbQHN1U90aVzsCdv44JuL9KYqyhqbyPX3aUCdSMnhZNGiMpCnGuyN0HpWb71n8/LpY39KKNQSKD2QuHRimpzJLk1HbA5tLTwGV18Ave8r3h0ewkbLvzOkBvpDCz/BcX1N5BvLoJOH1Akt3d905/1kGqNRr42DyouI8syhw0rOdlBEFF21oPNswvyrXdR6COd7SXT80JPZv/4Jit9DyOXHAVZjdTnF6Se33BEcaFZzjMFCqVS2FIqvev+S73u8MHrn7DG+DHBsb15Z9EiZr18h/7DSpn5kJWhcVmccyj+a/uYxcKDozuyLxQK+1+mOrIjhzNnURf1rshb7NGacvpQb/HDTuP4FG0WOHkjU7YYkVvkDxbzT1JokGQlbb6J5lCnMM5OfoHc1UGYNWYUkhJZlqk311NoKETlU4csS67vVjnnReOZSKxTYpwplQpk7/ZINiObQ3SMvF5CnH0SZ9v255dwL07mnnT6VZreyYo4vKhktWveyJ2XMF/5Cp3/eIlem55g0e5dHM+b5XruobyxLCqTUNgUXG+ALkkPN32h1p+ftC+5zi4g8lHKsgymUhT7O7PSqwfd07YTUjkFn9p+eKmc515lk+3mkC3O71W1sO1w2obPRB9iqN/bKHJ/Q/brBmHjsdpbpqFQq1Sub2r8ZodzLsooXHv3oKhBrnuM6lwM2jQsktG5ro5BLr/AevPvLaTUFbLoJ9rMwdjh/RbPlVBQJXWCju+h8G4n+sm5h4aXP8LLMa/QW2tHgQ2lUsHkhM9I7rGRCJ/LrjpUCgmS3iRGsQmtLYhuGZtorQI/hUXk9Rx5Djl8HPfHvoNeW8KpDl052b4zctURVx2a22/SNugINlk42VznlqsvIV8RuTMfL/YQakmSnUltViCfeRDufILCPZhS93ZccyoRzAkoZ0roEyhy1iNfeByrwp8VZzdQZFU7v1lGX3cMOfkD/DJfZlLMr1Ror1CqPUeN+zXqy3cip30FxcdQKCRkJ3E8tOJhnq+p4JFOjzTzf8t0zF6De303si1QL2lRqHRIQcORJAUzvAw09L3Ct899y2ftKlBeeR76/4bU7TMy/YaxzUk8ymn1Cn7hP9OqyLk++/ZAqrrGA579aO2bRKkNHisGR6uprueuvrqC6UWiPWRZZqD6KSYnrOS05Tukh+tJrc6j05pOnMo5hacMz9zpSumHz9P5XDYolIS98jCl0b1c+ELjNzXNV/HvktTS169Uys5/b/J5/G/ENWRZJiYmRmAY/8Vv+u8s/78B+6KjowkODubIkaYFoaamhgsXLtCnjzCA+vTpQ1VVFVeuXHH95ujRo9jtdnr16uX6zcmTJ7FYmhgihw4dIj4+3sVK6dOnT4vnNP6m8Tn/Lv/9Rbr8Ap7pS8QEuPgs7GmPTTZgVOWREbSSbP91uCjeNCUBdW+IpefdvYyz/NSivgy3qSiUdjIm7mNb/E1+lV6A4hOQ8hlUXMVgr8KoFnmCJIcskl4HC+cA7eezyCiQ+EYDv8R3LoxPAX00WGrormkgWAFmYxmpKW/yp8X2n0RW2Xjr53Lyq9x46c4UfqiBVVWgdJsG3VeLhO+6UPpmemN0fqYNGeWuQLj8gquuWBU4JBsh+jSe9I8BJBj0J0gqUvHjz4Bt3GqzkaePXiIrYgO0miqScpdfYrl5L88cGkv3X6fxYW4ESSyHiAfg9IMQ+QABmWruWkAF6JUmvBuOEmw5zYKFC+k55Da3V0t8twQYBHg0JeqVZSiqa83O1Jcx5j/P8AEfMv/l4QxUyIzSatkAHAbWA4OUMECWmT+rDfMejoFWD5NQcID26Ru5Ffk8C0znIXyS6I/Q0TiU7jTO1B5aeEG/TuSACBoK+taQuw2TdgkeMsz0gK7+u0SC9IvPQNXNFgmMJSBIfR20AdD1U+K8T1HR9TwzPISzPFU5CkachhFn0GR+Snrnk7RXi8OYl6YUCg9A9y9hYgYlbpNd9Sam/85DbqfFHw1lzmeJ5za4JXM7YRgGc5PMAZ4JjM3XsSHxIttC5/BhRpFIOnzrfSg5hZvKjbZFnVHalCReS+Sx8LMtk+RqfNlaqybVApdTLvDoP86x/rcy02gk+eZxvr/1BmedTv56Bxy26MGrHWZ9HPuNEFIQii3cl8OFb4E2RPyw+jZjHO8R2HYO9Voh5/DX5NDjvN4kquQlyhwN5BmCccgqiH0GOi8Fv+60LwyhzAYxXqkMifoFtaMcbi+HsvOQ+iXjO71CSsc7JHVJEi3oBKhsqkDuS/iEYcaxxBbNF88GlLXCEeItp1CvSScteDk1btd5MWgrn436rKnfJVwMMYAGWwMM2QsdFuDw783Wy3ORM1W0b3BnrL4EAgfAoD3Q51c+0e5lvi/kh+UTHFJOT46C2ge8OyD59eSbKqjRCYN0mI+Z+f7V4NsDwoT0UXvbMQYHCUP4Q/dIRsvzKfpjMB9/OJ9Lx9uhWF2A+5fAcZDrFCKfwp1PMEnevJLmScyxgXiXG6hRZMC5mWL9AmTn+HJIzWiedisE9IVJWcwrEmST2IIFRGV9RkP4QzDgDwgbR625hqw4Lyp9LAz54lnme1wCZz42rAaeI5npOgXHhhzjYJerXKtIBIXIISkhYQdq/IQnodvVbujr3Vq0dXOH985QeEL3o+uaWmEkUIErzvJwGHS5IaR7ghWl9DZ54pAcWJ5Zz68hf0DVTedcj0ZWr6MkbBlV+gtUeu2n1phPc90yDR7oTQl0yvyeRNOnMOwY1GVA8XEyY+YRm+NgyyMbmT3NzGM+1UhxzwvQNnQM75Wv5k+/wRzzXM/H5TJh2ksiv47SA7vCp8U4l5Cg2yqRxDzuOT6K3kxQs8OU3MyIkySwyrVYFdXIdi1Kx19YgfYyHHaJ30t9GbB3OweUXwm5F7dQ0PixwHcTC3xEm6+oAnu7d2DYYXjYAN1X81GNN2U2qLDDS3cmUxv6IjjBDHw6cVBxla0VYQzQws+JCyHvT/GOMqy/tZgZzXWBenwDI84AUEM8S9JGcsIgWLKldn/I3wnXF0DOb2y79y7GugT21sOzBXps7m1EHZdfIKTyT1eVtXZYWzECPBOg/AK0X8CQ8idJU4bwxE9PMP72TiJMe+HqPLj7Bdz9kvX+jzG7BNxLnuD383+Sq3ocWj8GiSug22f0LdCT4mhcTyXKQhbDdAfW1nOot3izpDCIXQ2VhJZP56XEtwgL6AS91lHr+zCJwQf4NNBBkAKes6eQ9GwS5PwBe9oTrDzD8+eX80YZeBq6MCT6C+j4PkgyVN3gm2E+jGu7hNTQRcy8MxrG3oCoaTDNhiPofnxM7Sjx2sefviv4rvwh8O/tHBBN+aH1MrzuA6MDV8CNhWK+y0rK3OJINoN3fU/e6vAljyokuPoamCtRySbUijouxYzheMc2pNY3RRKbQx5gYcr97GrTnyr9RardLyE1lELZBdC3xqKOJiHvI9yN8U1jt1lpzAdy2AC9cqDSvTfk/iH2WF0IyrsKfH99kIxvprIsq7W4qfQcFB6ixC2GowbQVI5m+ektHL83D1OQIFCR8hlsD6E4toF5kWAfAWMU/wkBSKNh2cefEBwcjFZZSyu3UpzkbZYnvwlBQ8AjBt+6JF71hmBzFG1v3OBI7ovgmwh3v4ajw5gsXaSDGnIC13C4Swiy5bem8T1wB+3UwgtwqQG25IzFofFvehFDDpv8q2j95HY6vLOGZ0KSW+y9gUV/sjkYAlQWgvXpuKlqkKuuwNGRKAv+YGzM9/RzBskNuHOFU9Nuue4VmL+ozKjJYXnxUBYeXdiiHdrnNMnUqRQqse7KatdeW+FbgdZgIN1xEnQhMN0BD5v4pGEJv+u7gGTnA2kPPygWgKyF+mwov8R4XQN9GmaLT3RAsU0D9gawmcBUipvDhFsjIcEhC/t64HbsQ09yr6IHe+7N4XjgW5xp25dLJafAXAUXnybckcLXVeD183DaXNzIA5HPwp6OUJcJJSdROx3dQZWTUF9Jwq4Mh/Qfm4/Apv90yGCtg8MD4PbHeNXf5Y3QBty1MkdOJfB1hVPiXuMPah++857KcyXgJsNTXhCruwUOh9gHS06zt9dkBpV+TUGbAoYygvH/JBdS4/gbq1Kw7OOPCQ4ORpZh0wOevDaiPdn+6yhT+GHz6wWSjIf1HilRYP30Qzq9WUmRuTvYLTBwJww9yqZaG6vdvsKgTae7Bko0naHPemj9BNiMPO1Zha/zs+/zrIakt6DoEHjEufq569WuzN3endfuTIfq2yKPYUM5SkXTRvOYB3Tz3w92E5yeCsi8vP8q+0vbs9gXNoVvB1OJ2Pf7/84zEaPp1GVaU8s3s9+UZUeI8bkCTjtJI0lgLATn+lDoOReAs0a4//ZgVlu+d/aZ6N+NDb+wuFTYAZV2sKsDIOFVcfYAFwAP0EVfjl12B6sRLLXg04npdc/waRVUY+GlpDlYu37lHBJaAt2zSGipHOgqHtxlXsweummhkxqumlo3XWw1jRleFXg3G2KuvTn/T6T6TKLvbqVPygncTS1Z5gAZgR+ztx72G+A9jwCGKq+D3SzGqN1C79yv+MmZ7WNYyGm85VQYegD6/goOK7O8dYQ5t+L2nmV0qJgEeztCQ4VzTWlai58rAZushZuLiSh+kcsFY5mX9AJH3P8A4L1qdzHXpzsoYSDx/ucY5FS9XJczD8ovQ/ExKDrC7rAi+hBBvzvn+HmPBVvi16ANFucqYwGx3udR2j1QWwIJUdjh0vNwcjKkfAoRU7hgUuGvgBmGs2zMe1nsf9oAUHm2GDOz9G500q4GrT90XkK91Z+fu33FJH1jW8tw9RXYHoJsr8PT0JVOWd9h0KRxKWay6wzPnRUEnwuhtbbKVfdfQVpZBrOikmvWja5/u6n+HPx6EpPSgekd3+WClyCJaiRwUzrtg7jn0Nan4d/MPpMccos1vSLxa/rlQnEzU1ohy+AeyfqaDNbmCCnaRlv7EE2S8TdqppHjv44b0U9hV4hDVf/br8CF2Rj3xLJokoZr0dPQNkSgtTrPUiWnwSOOosgNDL55FzezGLPyX6KcxTOtrvaQJCB4BLR7EwIH0DmoC16GRN4og6fSu9OqbhmcewQO9KFeH88PQjAEP4eWGrMn5qD7QFajrbvAuz1fpqt7CUqbx9+e2bxt4goX8rLffibET2j6nRPUkO1q+qec470e34JCBxk/QUMpf9a6caf9bW72usS+qHuQvBQ848X6d+p+XhrSmTzf9VQ3BFKiHANu4eK8nv4D8+jLhWbpDgDwiIeJGRA/h7ZRD2CpHO56E1lGRKTWpuGmqqEw+hn2dZfY3V3iTrXTD9j7Z0wxgiSqtOtxN7dmdtyHUHkdeq6Fdm/Szn6ER51N4XnzKNcUayDxM/CIc7ZN0wCKz/sQW+AIiHoEbEYov8S5VncZu28s5nQdB0pWinunNkBAXwodOtaVbaXU6yDPe8GrbdbD9XfE+aLkNKmHJ1MTPgqLUjAKgxxdm9pahjGx3zC3wwDCvVVMm/AkE9TK/3AfG6dSsGz5MkJ9FfTXQf19u7k+JJha7R3MgRPBpysc7EOI8iyyZOUJ/zUcn3aFgOrRSBKE+RWz7WEdH4blueqVHDIl9VHYWj+DlL4WtSGPBnUBdsmC1SrmRSMxccPNDUStiiIp/GWEoLcYw42gVYU1j1ptMgZbDSYqqdUmU2LKhbavQsgYahyizUVK6ZZ1Nyc/WhEDRdk8h6Ss5On0ZDpc9SUwu4DU3G4UG8TZRJLgRMEI/jzdnoUfLmTwiQno7MEQ+RDEPguykix1e+40C7jxVlVA7jYwV0LHxRTWPE9w1WS6ZfxOv9TTtPXo7eqnv+Zyaz5mZBmiS16mR/Uyhuc5F76GUoh/GYKHM9LtL+td873Z2W52Z57A5nlmPTWevO7rwbVIaOWM/Gvhl/Fsy3U51EWaa3HduyMqWeGSdXZepUbuBB0XgWc8bupafo69yPNe4MCBZ8MZONQPdkWhKfmDCmMolw1uGB3ga+rO631fR6tyh1EXSbK/xzPd5vBWu+8Iy4QlVbHCf+rfG0rPMi5qKUmd4l1PNvqOhF7fQfevUOZtIqLbCO6Gvev8ZglsZuHzCxkFD9WxvU7YG6MGdSA+4LCopOYuNFRwMfoVFjrJDskmHen1IyBwMAzcSbbb6xzPnkaNC2CVqA15CcbeoKLtJn5PexwAq1zPqXbdOJS2WvzszidO/4boD8mhQosPWmWTKoYsQ0DtcDS5b3P/jfu5p5ohbOG2r8Koi3jdU1N3uA/TN0yntSQjmYrg9ENw6gHkdvMpUi1BZfWmmxZ2Jjrt5PD7YPRF8f5AdNcxfxsbCsVf9kzJQbS0gQ6BJ2hUi2w+LtwlKMwU+1FpiPAVGXPLaHPuF3wKkvlXcKj/Zszqf2yRJAlPT8//dhDvv1r+R4F9dXV1JCUlkZSUBEBmZiZJSUnk5OQgSRJz587lgw8+YNeuXdy8eZOZM2cSGhrK5MmTAWjbti2jR4/m6aef5uLFi5w5c4Y5c+YwdepUQkNFAtHp06ejVqt56qmnSE5OZsuWLaxataqFBOfLL7/M/v37+fTTT0lJSWHx4sVcvnyZOXPm/J9ukv81xVFwAO6swH59EQQNhvDJhEVP43piBLcjXuFm1Oy/LSolngco9TyEd30vohxDW1yrVXfggxRfLEX+mNVmMQGHCDkhbixC1zAat279APgqyAoHekFDBdxcAsXHXItolfsFjKq8ls+uTGJvWC4P6CGzbjJDlf8c6GssvYDBGh1nz/3sWpQjSmfRxWuY2MBsJpDVdMiupFfKCWSgn5sRe+dPYMA2eLiBQ22PsbkWjJosUt2OcbTuMYh/CcLGg6ygTd0BqsJUjL/nRrJ/BzJrU6A+E24uhvJLOGiSL1ubHkOF3Bf8+0Db10Djz8jQ6YCQsTg7ZC7e5hMQKJhDsgz9rjzE/YXCUT/eHfTGTEAs9rk17fju6ufcLe+FJMG8bsfI3jKYgQsWsDi4LY+4+fFKAJwcABM/n828Zduhz6/gsBOn1OFlCaba/Srl6ktwdgYUHQa3cHwN6cSoRMSD4uY+yhztIf074SSNeYr6qMd5LXA/I92gYw58dWcn3HwP0tZCfQ5dTDeY7gFRSsjvtZXHg7pA6mpo+wrFxnbsqPQnxwmWdbLugkP9wZCLQxPMdYMXJgcktX6U8oQRcO5RKDkFkpLQhj0uIym4ahLpjscEyNP2TaDJKH3cA9K7HkVbc7xpMKi9OGFQU2hUCicRgEILPp3BLYIhjhx+GNCG3J7jmPvFZ3T2zMUzezFslCBdOFkax5DZVvkvJe02NhjYcudVrhmaDnmNTM1GR8i5vtfI9h/IvoKlAnQA8O7E2zUD2V8vDnlTHF1YqZnV5Lg7/xQDVI9R7X6Jt9w+YUCeA9wixVxuP9/VJrvH72ZQzwMMjVmF3npDOLrufEpZyQUC3fdxc9oWno4rp53PdUr6F8N0BzZlAE92fYM4n4uuw28HNShzNkObl/jBZKNOm0JK+FtkBX75t3VCksDNHOX6OzGwabaWeXVn1e0q2u+rZ8KfE1gZkAzVyRA2FgIH8p2lD/suxpETmcN7PU7wkWKtcDYCUvAwni+FTWP2snjxYj4ymFmaux1GnoFBuwB4xTSVMQUQV/AuvRsGsXIPtHo8nQuffsyndSnsrLOxpgwGnAbrJyZWbr4DE9NJk/pQGFLI8UHH8an0ob2mXjhnC/bArQ841Opn3vSBPN/11MTNpF9VIGxWiQO3eyuq7KJPQyofQFEwD0XfLRAxBYA7t5YzbHJ/UhNPUeOm4OPiOeDmlCqwWxhPNh3WzWTu53ORoaXxkjCXjjnw2YurSW6XjDUin2+63xaHIUCWbOyNvcd69ZOEKDqyrBJOWkeIazKMjl1Ddptiejrt4ywLmLVijA1WH+fXydf4fsl7nIxK54fq7mBrgJzfIOc3NoefpIsTN5nqAY/mTRFEhWtvwIXZQr7Q6k9k+ZN0ckwGYxGUnoaGUiHXaNKQkNaTvu6n6OSfBLog8O4AKg+IeZq79b2xA2/42Wnv4cxxUHKCLvaFXIgQ695IN2inKQaVHjR+0GoaK/PHU9Us8kBlq4a6LNc3H+/QjiuxD9A35TQPVjcBNQCXtM8x8nQEIzdfY8zmS1yvOwq16XDxOSg6ShdtJiNNzsgkh9RybAcO5FuDD+XOZ6slG5Hn9cKhCKB0p1ruSplVhUoCd4URnDI4koRwrruKA/RRYg0CjFIEv+b34pQJxhbAUUs/CJsE+hhI/Jxt9xZTZfHluhl+rNKhqBcEAEJGobHX0K4J3xJrVcIrIlKkQgDfYTH5yPEyk2ZuQWe+A92+EBGLoWMpVYdyz9J00JcluwAXrIL22Li2hilha7fV+JV/RdH2Max4dRAnDvXm48XlnP8E8m/9SklJE5opSXAwYxZ9s5VkW0FqTFiPHRx2JCC9ZBQepY+yxNMPj4YzcHORANnT1nAibxaptRHcC11CTuAqSFsDRUdBkvFLGsKPkyIAB1lWqLF7Cedp1maoETkF7hgeYE89rKiEPwo/huEnhDPdkI8CB0oEKaF/pBOcKjwAW32ZFvUIfzzkQV+/FGd7yHD7Ezj3GA6fHmwpaUNN4/hzSEilxwRgcuoBOqUF0k+rJK7wHULLpxGm6NJi/OmlAEZdrSY8/XvMKT/jrU8Qh2oATQCgID9U5HRZXxwqDtOH+sKxkVwNephZJdDgkDmd+xDpld2w+AyG4Seh/dvCMQ7cMslMHAGvPahioEJmhEbTggA0QAn9JJi/dCnznHZ4F78d/NnvHYY4eQSFNmDwHmj9OAHV50QUotJCYV0sm+59IfYZ7w6g0PGAfIbOmqZvXGWYBQ/WQtyzcO4xNgZLjLlaz7jLVrplNOXWA0AXymvVvuysh0c8YJB3QYs5516XwsMecMJaSUJyV/YX9kdx5VkoOoRcn8KcHi/ycGP+E2wt7v1HDqHmIIgsg74hju5p2+mS+TORHtEw/Dj0/gG9MYVXPDXUB5QRWBpIN22VWBsBJAW1/o+Sm/sMXvU9msZI2DgBNAcPg+kOIqtE1JavDFFKK0zOh3F3wKczS/VzedPpHNFYQlwO38b3vxo9g0r38853lkHjC4P3c1T9HC+Uwg9P/MCvj+4iS3ECqm8JkkaPb/jMfwPT3DToTQl4GjtTr+4Dft2bvrlFZJ8knMaJn0Orh/Dt+CbPH8jELbUHwTklHG/4HAwFEDMLOi91OS/KbeCZDltL5sG9b8Q+eHgAXHwGN6WRTdM3cWDxQaqHaxioUPyNgNY4/gY+N941/iQJ9pUmsMdxl5tRswk5tJJTMQtBkrCqgllSDmeMMFH6mudLvATY8mcsZG+mzJ5OqddBAPrrINawG9yjoPcPMPwU95c85JL0DlM1Y2rZzVCVzCc+CibtmkTPXeORi18XDsH+v4OkZGDel1yLFD+fXw5Lk3aJts7ZgkqqJaOyK1VWHe/6Qay6UjijtQGgCSBUVYpvc9CjEWy6vhD16eE80ultLAqxj9+MzYWdkfC7t2hj/TSkuzLb66FtxkY6eE4RJIlNMmwL5qrZmyN6IZs9rgCqhpZAwjw4OgIODWBO/TbaOveELe2OEJEyAPZ2gFJBLhkR+BgAZgecLOmGPeZ5mO6gOGQ5L/V8igthgiSjtgo5T1I+h988ccha5u6/zPXMWVxvBevDhPQoMbNAG8xnIdmENkoYW5uBVWOvY+n1GyGmLvjVDURpb7KHG0uJ7/OMKxAg0Pzje0jxWicc/HUZkPIZqcFTWOaM+F8XWkNn1fJmN5+iW8GHjDQJ2+epsONN14xCTWS4dw4DnHbQaRNc77AKPOKodx9AcX1rfnZc45bnTrR2X2Z0nNHUbxK8efgM3XJFxGFXVQWkfg6nH4Dyi3TUyAzI3oBPvdMRnfUTmIqg1w84bi9jdOx8Isqe4O3SH/gp/H0xZwDiXoCDfbgQWcdsTzBoM9Cr82FHJJycAsZC4rK+4Y8Q0BvbsuLkQYqsfV33lzXEseTeOE45iYSyJAuiVps5kPkL34/uQ6IUjF02U+5xrKk9rr2OwlyEf+UUIktn42HoRIDUtkVfyDKcbteNdMcRJKAoGkLtOwQQDhyoCnL9dqQc0XSjPhaTZwdszZzdfwUSldEzMFa/TETpkwBM8wBl0T6QVdQRTZ1FoJdBlROZbb7NqnHfQo9vQaFFp6hq4XwX/SOBJFGui+ay2YzGEsLwmzkMzzonbIErL0POFkLcgtA3xCE7bfXmEZmN49QuWeinhZUhFSgNKWIP7LIMio7ifSyMUUEXnd8kke69EsImgiEXpXNqu5li2KiYwXNtNmPo9CMkvUlocj/6R/5OgC77b2BK4/i6Hf46exJVFPhupqNuFFHeUa7rfqoIxlwxMeaq0fV7VJ7ClgkayiN5wZw2QUzZ43TXGQRBzFoHvX8g1W8UWy0Z5Pp/jwMHrSxrBPDV5iXouIgDjxzgbbODTlnf0UUDZyJ8YYtGyNCpfejqFUKXiifEcxtJKckfwZ9xTI1/jcu917rWGYPVqTgRMRmrd1+CerWnpmcQVW6XaOVxBXShEPs04CDR9gej3cHdGI9/7RBk1QixfrkLYLdxn4osnc1Y83CUDSWQtR7OPwmSAqNDxhRaRJWfnhv258T+mPEzXHud0posjla8Dwi/ycSQE1B9Eyw1cOkZ4kt2EuR1kOe+fo7FixczxvCti3wpSeCpKSNIdxeFZGH+hBrmP2D9J/uYRD8JJs2bybxXX6NKPxiPdFhfC9dj+vHz2G7os94Tiid1GUx270dIvwiOVH9Le99E1DZfARrUZ1Jrb8X1eq9mo9oJBEsOcDjQOEkn3fdvoPWLZWhMGiTnGGr0J5it9TgcDtcYUzptnaVpD3CiQwcy7Me5p9jBiQ4dWHb7GfGYIXu5bPkIAKu1Keq10U5qHmmmsQQTVDWBWF0z71t9DoHep4mdUMr1fj0pCSxx2TCyLOa+xbnn9rw5imdU/cR86vmNcx60XBsiNHfg1P1QfhE6vkta/egW1xu5NwJsEX/41g5i8M27rBn9S1PryeBT35Nw4ziOGB143AsUvjuHAy7PYY5bJBFlTzTV28wubJybdrkR7GuZnetjv1q6aKCjzjkfG/f1o6Mg8ydWaYa4zoiyTde0B1ZeR331JXo1YVVIDqkF2K+QrAz2KqWdGkzqXDJsehriXwFjIeqKg5zMnsaUjBjSLdCl/EM+HvExGpUO/HpQJ8UR63uZDt7NZH8ttXB4EEgyPdPiyXTUuC7ZteGgCQSlG+dil7O6qvlXSrDVG669JvqrPpsdIZXcr4coFXioqsQZsuo63Hq/BbC1rcqX9QUHBRE6fCJmVRS3I+aRyTFnzTIObTB4d8QUNJW7lZ1btG+SxcmqcYt0tpEz0hLrPwW6Qqqm0D1jKxNDnoPr8+H4WFCoMTgUxN1tQ5t7bRiaK0NrseeQ+wfxfnE8ELCQsPJHyLTAuUonqSRklPj/QX/ybHE0PxmrCFHA94GgK94r3kmC4ZFbeMvJS97m24sJJU+x/MxveEu3IX83SnuT2lORDR7tv4+P332PzASxdxav24VneSZKc/3fzi0teuLf4N4/LDabjZs3b7bIzfd/svyPAvsuX75M165d6dpVMFteeeUVunbtyqJFQnP/jTfe4MUXX2T27Nn06NGDuro69u/fj1bbtFpt2LCBhIQEhg0bxtixY+nfvz9r1651Xffy8uLgwYNkZmbSrVs3Xn31VRYtWsTs2bNdv+nbty8bN25k7dq1dO7cma1bt7Jjxw46dOjwf6gl/vcV+7hUCgJfQk5eAoEDwb83H0btIq4Z66S501mW4WarZ7ka8yD12nv/oEaZpGJP4i/2YGJtAJMcm0DpDn1+gVbTmKm/wEinI0nr0IrN/A8/4dw7O4M3tHeRgIzgT8kI/hS96bw41JirQd+a98oDOW0C2w2Y+S9EVqVe+YVFsX/yghfs67yNKPNq0AbCxDRo9xZaZS0SNvQyHIjJRapNgYj7QKEm0rcrXe/twbuuF0fD57Ckwg9CRrjq71f+JTeP9CRh20Rmbrsff/MN8V1e7SHyIRbpJvHjwCMcH3ScBo0VT1Kg7CxEPQqWWj5rPZnY2lFkWmB7fn8qdOMF6wmxwFfbdNQ5wFsBf4ZCcOEO1zWACv1psv3XsdF6P596TiV41DoWLFzIWx/cpvvQYkpfAHkQqLyUInLgUF9I/YLl/VrT1ucOWglOR+cJxy7AwX4MTn2DJX6ga4jCUjGaS9LP0G21MMCOjsA96ycAbpuhyg4mux8UHxZ5UIJHMLD+JPN9hM7+zrJI0kwTodN7AOQaujMrowOnTMJx3MP6q3hu2XnskTN4MLW3S2LxmrVcsACvvwVHhtKx8iX6O20AB1bO2n8U7ButEwhyGhspFiGb5dA06T6DOKgs/HAhg7b/hkZ6WkSLDTsK8XOg9AztvL6nPrw1jxysYsXZjcjOJL4UHoDq2+TE1DDPG1Tafy1pt0bjhcYaSI97u9E1RNJRDQ/feQZSVhKR/QWGGCjvcJNKHzP1ckHTzdoAzlsjybZCfP5SIjPXCVamu/NgXXsXN65To7vpbBS5pVFgrmaSzsSIyz05+O0EXsqOpc5tEHT6ALp9Tl7cq3ikO9BK8JE/dPa/7DIyTPpevHLgIkdKO1CnvU10zaMMazsde/sFED4JWZZdTtRGMLBFW0vQJ/UY/e6cJbzsMYZFjBYAUeZ6zLYGbrfZhVVhRXsrgdeyuonIqGtvQtV19to6Yb7WjuFHhmO1KGlQeAmmM40Hj6aPLLZBkSVOHKacxSC5OR3xdl7a0JkPf9Nw0g77zfa/yf2eccBH77zHyrU7CFFUszLQjfiSYLpf7o61+VaevJRUcwAFVqjXpWIP+hUvnNo2F2fD7z4u2Q4XYHLjDQEWX3+HAMNdXvGByl513O2xDIW0ApTOxVDlwePyGA63uoe7wZ0vr/fj6IjhLgZ/C8ahVYlaYSdRbxWZvQFZlojSZxGuVOAhBfNtNVyxD3Beg8zKzqyp0FNkA9mu5f2GSHI7fAzAdVtvnikBfXorMtPC+KW2F3i3h/i50PZNPivvRKazezMb19y87XDnE0hfxwXl41yLmkmDspjuvj8K51fgQIh8kHb2MtZqnmTKhtF8uOhjvrn8DVTdEm1yYxH0Wsu5yoeos8POWpntxRug989w7lEiHb8TLCuQJNgcDLO8zguHXfZv4LBztvglAgpec7VLYu6jsCu6afy5xqbtb4ayQ+GJp6cAsUw+VQRSCLVpkPatcJDV1LDk/Bp8ZbgY6UBxcSZsCxGkhJpUPNSClddGBSd6fSsqbXRy2G0cKn+fWq8jXC7ryuxz+1yAryTBqNY/83GzoCbMleIw5uwrXUOU65KMDF4JYp9KeJnrfu9wO/xVUVfj2AzoD91WE1n6HTOcvtMQBXwUskE4Rqpvu6LUJ/9u5be+fozKf5Ykt6cgdLSIWAwahMZhwV0Cr4AdzB/ahwTzAtjqA7/pIW0tj+qEDF73rLX4ySrWbtpLqwf2c+6Hk3xal8KOOitryqD17Y0kdgln+dK34cwMPKp+p8wQwQWTUuRdaYz2jXwQxt+hxNEP74Y4BlY+x0vRR/BsEFHeSErQ+PFSUQCrvJYA0EtnE06sjJ+g+ARGfWf2V0RS7CNA/jaaTEHMuPU+XHkROn9EskHspWdNcL7qaZA14hCYvZkZ6W9xKsQbgyaDiSdX8Qfl0HMNxM4mvW4Yv5VFUOAc+5IkCedXzm9OpmnTeiMhN0VjlxxHZSvFalcTXjGDxMyN9NRNbzH+FLKMyu5JVMVMWlc9iIfKHaJniCg4bRADdHB45gZWz1mNQW8WTj2FG4RNcK0FVe6XKfM4hllZhqyQRWS0p3DW7slZzD6jnTo7PDykPRlXthD/1FMsDk7gQZ0fz/oLWfCo1+OZN9IgolOAQmNHvs8Y59p/AQprhZO8KGQqvbOVlFi0rrEMQOKncF8hN6POYTqXRXDlfWKOSUoBzhcfh4B+3FO8jsLuxtjYtawe0xG5vpn9qPZmq8mLaw0QlglPpQxvcdgtSliA4h6U2aDUax9WuV44tL3aYYt5kQVHD/JlFc7nNpvvGyWe1Er89UjW3MnaWIKrJhNePhNfna/r3yIsd/k0qAFV99boX/meza3OQUM57IqDk5Moi/uF/WnPAg4musMAy0ZBINP4i/5yFtmu5nlvOBB0S6gQeCWAQkOUZwz+1SNon7OKTjnOdSTnD6S8rQyL/olDnbfRQd3snRVaCB1FpSJK/LRVDukxV1lU3AY6fygi1WxGHFJb2qb/Rtv8Zfjq8qnSjWySaQc61s8lrmARXdM30Tt7h4hkTBBsd9TelFeHoGyAytHHGOt+D0pPwclJUHaG3sYkpnkIvY9aO1glN7j1nrPixTDkICpDb9ezVP20ZOflMXDBAt7yactTHm6sDJG50b09hz6DrwbddP1WkmBB6hSWN8ul07jXW1WhLKqAU80jUNzCROSpQsMH/j+z28mRWl8LxwK/EsQSALUXUwK2kHhXOGm+Lg+BgbtEBLFCC3VpvKAT87nO3xd/hbsgOZx+ENK+xdszhrpaoTqSZoFXO06Dy3Og2xcUSaO5G3cfWeHv8GYZfFreVzir63NB6cYTWStYWdXsexoHZ7YAvL+oVHGxzVgATjZG6zua7CnJSQ6J9rqFh/02GJyRHz5dUUrOXcAhMyD5qvNeuwAbDblE2orxcD5uWXZi00t4xIKljn7qQvo09HO2s4yEA+w2JBwczZrJ5nThFFPYnUa3LhT8e2OSw0iv7EaFrRm6HzQMYmdD9CNMy40nxwI97u1idFI13hrnnKpJRbM/ijFx31Dg8zsX4kZxSbWC5kXRbOJnVSdgVLYBhVqciXy7U+A3jI3OrfaxPB/u2OYIcHSjBJYaarrs4HqdACd9VM7BMvwUeLdHluHTuGMs9mt6nsGzHUy4S5Wv0+nrEA02zLCO1T0fEfVulAiQRFyPjwyrAqCvOk9EpgDU3uOz7DJulwqba1TMWvDqIMBijS+F7ZfyZsNxziYMwGjxoNTSzLEZ1hS9ddok6h7gvxUcNmFnJS8lXG6gnxzLoORblBkiiFAJki5HhhHvfZwNeX255YqMkcRe0n01klKH1a4Cl2Rgs/NB+CRqo96lqqofnbLXMOj2dbpIj7boi+bkJAmRU9eiCIHEz8lrc4j9VYLy+FbbH2iVurvpRrcwyuLforJ50r6/RPZ5Fu5mV+tTjKsTz/zEHxR3BHCrVxQQ48z5qrDrCZDaEll2GMrOwYDt1NuCWsiPAlzs8CX0XENSxByeL8UlYwgIsmjlVei0BJtSGF8xxW+QmL6FdtomwnKjs98hWWmvhmf8alGY84XNemgg5O+myKEl3XMXT3vCi8Fp1Gl6waCdcF8eYXmbOB8h9qBrRSPYmTIXSaEBQy5mfTd67dnLt9YbFHvvavE8V1tjxyFb/6FDW5ZkFA4NjZGpkgRUXhMRel7tGV5wiME37xKZ9TXLbtwQN1WINSEp7Ak+rYJyz+O4DZbpZHoWkj+AqKng3w+Sl+GnvofkUNBFA321IgcbDaVgqcHjWDSLOn3e+CaiH4OHQbu3SKkcyg8loVTYmt6z6Z2tfNf6NrMCSrgV+RJPpMyGsUlQdhEOD+KY6jVeKBF99UDbZUywhglQ39Ufoi4HNjoEnkSTMh+8O4lIS99ERua0Y9nsb/n8xdfYb38VDPnC1r2zgla3F1ERI+p5qBDuv7BKRHp3WgKdPmSOcgw/1YB7vSA09NQfArNg3sgyrMuYRHxqAj+2i2JxhZkec5eQnZvLwAULeMOzLQ9rQnlN35bLvcLQvAoPzhwBDkdLIpEEVVYVmopDIlKt/2+cMnXnNkVUWPNd0T8KBXB7GW5SIb+XRzZ9v90Nd1WlIN71/oEsmzAGtEZxdrEpbCicYJdaIa41SsA2nslVCjFPNLJYw812I7bGqD9JCfe+hevvsKN2IWfjB3JX2vUPwegu8iOEl80ksHosPdJ28XDQu02D89636L3qORY6gbRxYzk4pSsTYz9yjdPH4tewpm8WJyacQOodwktd7oP8PeJem5mF1c+w06Orq7oiSwfouxGCBOk9Q7GPPN8NmJRFrjob+6nxnKew69A3xBHp1dR+jeuN3eYgXgWtVc59Ne1b8O3BmTu76ZT1XbP+ahq7iYF9iCh9Eg+DkJFtTpDKqc5hvnEIH1aIcwU0A/t0waANcdWVkLecsdcMaCVxZuTuFwAcM4rfJ6b/TkT5E4RwgKItvfngrVkc3tuXgGfVvPku3Lu7hMmHR6FJ/Qz0MRhiPxD1NK7LjeRJhwOs9ciShZeP7iOhKIdpHtBFXQUlJ8TZ5fYyMsw6l2qXvwJiTnrByYmQ9BaVXl1dhChRtwxuTl9TZRLsac8QNzPBCpheBE8cPwPRM4WaQNAQ2hX9zvNOrDpEYeXhkPsE+A5Emr7iXI/v6NhISHVIyPZ6MJUhYeOvNvoxi6eIqh91HlkWanJBlRMp8zzMQc1srhVea9HPxV57qQn+iskd38THfEIQhCQFpP9Id42D0+P2sXv8UTwaekDv7wXBLXQsJL1FZ/vrOCQ7OVZ4M3USZf5z4dKzsCMC8nZyzexBshl8FPCkF8hOiWpZhuFRv/OGE+wrUV0i25KK1a4hQbkOTkxAZa12vacD8FQ6aKN1uAQGGrIKqQqMozSqZ4vv/0fk+ebl32BgU/m/BfQBf6EA/D9eBg8ejMPh+KfXJUni/fff5/333/+nv/H19WXjxo3/9DpAp06dOHXq1H/4mwcffJAHH3zwP37hf5f/viJJyHYDjvD7kLw6gNqHV34/TypOlvtfohokCWSnkysnYA13FfnAFNf1rnWf8GWnelbq64g3eNLPcQTqs4QTyrsTL+w9yB+VvxBT+Aa7ri/n8XcXCmCs6wq48hL3GU+gQORy6+WZT0JOH8hB5GTwiGFtTQAF1hLcTPxrkVWmaqaH38NRA1qFgTpbtXD4Vd+GyIf4ov/XTNueg9f1u3xZs4UX+reHnN+hYD/B6r60MU2gvG4gVfoLYhO/Mk9EsU3K5p7HaI74pWLSmJj68HH8atNB4ykilVQeyJJMRkwG3u0y2O7rQRAr4Ob34B4NxgLaFM0nStOfww3wXuaTrO/RDl+dcLDJMrTRVaFWidwGM4tgftdxBDn7ItLrFs/0H8k3tUb2GuCL1EheHfODq68kFAx3g0NhsM6WDFJTJNGGwhFc8tiEAtArHELuI/VzAO6EPc7GSz+5jL5quRvEd3M6k0yYQsbx+NU93DGLaC9/bZ6QU9IEgELNLp8H+Trja4pt8Hxaf16L3sA808ewLxEP9UEaN3atBHWSP3pHGdxcjBQ8swUDtMKmgvDJAkypS6NIP5Or17NQqW+R77uJAksvMAUKIEjj5zJOTxrF/9L0zRK0Vt6gNq4Sp6nEnNDtwLqm691W8Wf+p9iKs1CqSzHrozHGLsY94QHQBoHDyj2zgnKblbhu8OsRmPFfAJx/0emI7zCWj4f35UDuRM5pcjA6oNItliBtCBa3Og4boPefI/EzbmTo1B/AlO4CMBuNTIdkI62iO5d0u5jYOPhHnKLLh60pjXiNIToYqi9GslQATueKIY9vfEtZrDdzvdN1ss3+wjHcQcirKGrFoe6OGRJzICr7PqbnfQlHXkQf/Sf3KsZzpGM0Rk0WY/Iu8Nlkp0FSc5dg+YTL/K1xS2JmhsSPPw/h6GNHXeNPdqjxqe+DT30feoU64MgAiHwQ9wYz33eo4IMX1lJT5oVsixdO+DsfgyEPhezF8YEn8aj1oP/dBIK65ApjVpKg6Aibg+GDCrhpBg8ZPBQVwsnlNLDD5GpiVWAxVpOZspTTjv9E7rehgYFvvMGBH1YyzbuOhT0C+b7/9ygbOgiDs/QcSBLz1m7gjOVLQBj5z5ve5euZb8K1V+HeN3wV0sDoQijy2Uak3ynklE/EQ+rSSQuazFvLShl2agAp/SuRJJ+ml5BkaiQtRwfso9+xIeRnhXCyeBBDGvXaKy6zLCwK09OPA/Dz41cov1jL5zOd0hayzCPbS/H3B0eMkNuRXeAo3CwZwheFAdSr6xiccYxjvzY5Y3McCWwodOONH5+gyquKpAVWAYJ3WynG787b5NmE8+CMCXZF/srEaG/I2wnp31Gk/IN8/zq8DN1ItzlPPgXCqepdsI0pgX/wKa+hqi+iXJ0OeTvEb+qz+fLil2zUfMJwN5jkYWdPgxE8O0Dc85zPfpZVmzoSGfY2L+g/IkzuRvc7n4rILaCV3YQpbwRZgatRyA14NCQ3NafUPJrn72Cfn+Uaf3Qp56WVi/kiEK7a+0DoW8Ip6Ncd6TpoLaGMu1FAZFAfcb+pSPzv6itci9Yy4d4cCNpMgLoMu0KP3M+5Z1bdYK3nu/ibYW/Gi/g7mpyssgy9Q3fTxQveKHP+46GBIrpkQio+XGVnz5VszH8Kf89kvNwaml46ayNfdfyI2SU2XrKM4omoJOoHJuEe2h5sRu62WsmW7HkAuMvQUZsjnGP12QCU+b8MQEbQpxi0GZQ0DAGa2PjPFK1iQBjMKyulS0AZRfQTALrdAnl/8qZ7LluVkG8KZPjnL1CctoCTdujV3KEHzLBZuQCMf+8TTOOsNHjlc/n4IjyPqkGtZU2XTlxb3ZkTL55w9dX6KQHcM/jQMzWKAXE1fBrQF6aJxbVcPYoGtdgPPw92Js7yiIMjg6npsJnpBiE9d78euuuuw+C9oPaDmjtQcYVIbSGtlJBtde46ah8hcRw0GHVtGjfuFlGn20GyzyYs8qOCiRrQl4vb4WfDUQp9cwGn42ngDsCO+tZXHOm5mpHFglgCEpJXW0EgMpWQFbaFC2YDRr9f8K7rhSQ1yeY0fjNAv4itvNFvGoWVW6HN/eDbDYpPcCzCzByNmTYRBrqpzMKB8LBw8MQfe4wP/WB1VQm7Rg9lV9aD6HL6wrkvoN8m6Pwhx1JtIMONojGsLt/Mp0+5M+vpWLp0XU11xkvkBH3FF1UQolbCjXeg3XwIGkJefRd+K32QO9HCEROiALdbiyB6Ku1bjSPgrAWt9jZ07cWDwUVQtkXIA6m9wKM3DSax5gO0kgvg8otw90tQupOjFHutLNlQypZmEZ6iNKo6GBxgsqtazFmFrMYOhCpgkBs43PKh63JIXI5shhvFI7gbLoHk4FjHWIZviefqc5ea2vuvYN9fIvuKvXZTo7uBf+1QJKk33FsDGn8agh5gyVfdUJw5SXJYAksjPXlb7QV1aeCwUGYqpsotD4MmnQnu0M+2CWyvCKKERwwl6Vu4456OXTZz2gi7rYMY3zgnQ0bxqL6Bwur53CwZwsu9nqDDjZ9AkpE82gBvIEsun0ALR1SQPZvfgmFtDc4cxpLow9BxYMhlr+UUF/NV5Pr9RKvQA7Qt3wylG8FdOMISjE/i4eQUqZtFAwNgqSXArRDPSgftzHoGep4Dv17g2x3qMhhSf4YAL/ijDtqrwcdSCr1/EQBV2DiiV0WTH1zJa5+8hr5eT91nHxMc4MOChQspKV1IRgbo9WDRrsPqtQK7V1M00V8jnx/3hOic7yFqUAsQ6E0fCJaui++pz4a0dZjsGuqca1GZDdrV/AS7XhCkupBReEo9USlMzvEgQfgE8T+A4OF0K5HxnrCL79o3QOg2sP0k5NDDJuIVOprlnzgYqspGLRu5472AxIDt4N0JP+kiqd13MK8UPq6E3sqOvAqCGGcqRmJRi+Z1SVqOuYrZbOfU6YXikAIsvTeHqSvepVESOrbsCezxVt67NZvZfZ5Ga/WDhCsCmAU+KNAyKBSmp3VkZuQx3FI3Q0OyIMgFDaa4Ih/1cwUMsflw3uYHrZx2oXsrqMugS/VEHvCNJrcBTg99BtVWEennE/4jRzMfp9ajPcSvbpo/rR6CVg9h2GVDqc7HrE1Bugff+o/gGQ7BgZ4QMICTBh/qZCFB5upXAFMJ9sChXLaVczt8FSZNLmWNMovO0iG9O444mF0MR9tMIrN2CRh8BLgQNg45pylP0e5aHfcH9xDpGAB8OpNRH0Wq9g0AzpVHE9tqMMGB/V33rLr8JVVWBUr/eSS1ribq2hMQeA4cPgxqtYX1nc7zQDFQbxdrl7NopVJifS9jU0P/XMi31jMnoK8AD5wkC5OqkButZrGm8zGkTDXcFTaUvdsPnDCCCjvJpQPZWHmZlwOdZBHPNuDeiqOlBZw0WjgUBsmOqzA+VQB6fj2RwsYyb1MeBT6/EaJQsNt4jPtH1MLJyXhpxrQgv7WQk4t5iucO3c/d4KWNV5v6YuAODMUtmv5vdlLzyBk7Iqduqed5uLSDhvgkJMcqAGx2G33CtzXdeGYqwYHD6GKbTXF1BTFFb6CxBjXV73DA2RmEu4EsiX1+ehGcGCOA32HaaYxLvIJvtniyJCFII1m/QubP9Pd/ipXFLd1sTXKYTQCR65tCx4jz6dV5BNQtJyXsRVqVPI9vXT8CmlUT4xPDuOCnuXlD5nB1Gd0yffhz4AC49aggPETNoH+eRKaliFNB0EaRQWqzNtMolOgVerzre3I652FqG/wYc+c16PUdxeWRlO99i/SQ5YSWT8W7vgftmunkirYRbV3ks51TdW0YWjXMFd3XuPxdj3qcMs8j7MpczqtSsogir77N83HfciB1BqnlcRQb28IDlUJh4+xMPDXdXM8psUG2PYJWA8XYpPIaXJ9PgCYBq1zHtjqocgxk+/R3RIQwUB32BPtSBQgnNYJ9YeMgbBxXvoZtDeso9yxo0Q9sD8PNZzCPZoeRKuVTpT9PpskbktLg9keg9sXPbTaJSRmUeR4lO/wwOgqE2lO/TRA83CXJWO1+mZ0FU3mk1TOoGzIharqzzWSCC4J5yu7JEz2/g0w/QdLo/hX1dcVsThMyg3UOqLZ4CfJRByHhXX5xCzV2WPH6Cnxk2Cr7cVMf6+oLq6KGIu1JALYYttO2vjP9Q0JYsHAhqXcXcs96lDy/nzH5/8LvwTD6+nRoPw2dvYrx7nCjAXKsMOJOPzK6jkB97zVo+zpr6h7jlOkysJmVlztSp52CTkqAmKe5mDWKNgWJTIt5kVkTuvD2C+H4ed1BuvoydHjHZSNd7z+YoC55+NaPJEzfBWgC+xyyAPu86rtjleuJ9op1jk0B9lkwYnc0A/NyfoPiY/TXdGavx3Vq6qe7gMLmdtLD6p85ndU01lusFaFjWXltPbv8J+Jf487eajOlhhjX73zUVcR6NXCr9w3CrvXh+zu/8tT4Hs6KFNT7TKTk6kBk9zvYJTP19jCI6i4kpw8NpFNAKr+pSsR3mIMZXbeT4fRsAfY1zvfmRZKgyu0yVW5/cDIcApUVIiI/dzvU3qPQ0M4ldwotJX0nRk/nSPZ0qnXXcG+IJUwX67qWU53DsrxjLZ7lGvd9fob8vUwvfJ29CmiU6Ha1lyYQgodTmXkGMOJd3x2dOZJtexeyZuMFhqiv8YnJTDBQVA9rTsPZE7ByGswblw6aANoFnGJISAGfG6BWfY/MyjZEad2RtgfRQ36OX1mCAtgYDDvrCwXxJOoRCB1LcOpHVCN8rH2aRRdiKsbTlNvim5Liv4daRET1eUGGKbQP5tuKi6AwAJKw/RPFuh17fhaPuXvxdXU1E7yrSdDvgJvX4fKL+CkGkGXywuwQ5xgJCe+8JXB+OSHAB72H07++6dl/JeQFVY8jqHocZxL6clO/jpzqcXQN6epq27uhiwjyvsJTUZBhUUPgsyKy+fpbnGtl4j6Vieqis7xalyzIjX1+Bm0QtnNP4GFzo14rjLBaqw73ugPioYY8OPcIn/p7MiAPUszgeVdJ1YNviz6XYdX57/Fqc45WFWfIDvyWLupa3PX3uGeZRqdBXUDt7foGtQTfl8ZxetUMrvcpw+IHoR/P5co+Zcsxwt/34v8MzPvfDPb93yz/oyL7/l3+F5fySwSXfSeYazfegaQ3OVnd1pWYvQUTkCYHPkCu/48cV77doroGTSe8NTUo1bDb4GCZYpXIgXOoPyjU5NqDKLY1Y193/gCGHgSfTjBwB/cZhru23kSPZrFTOnEgfNZ3KyqrL5LmX4us0mp96X9qEbNLIDbXxFe17kIqU6EDtTe/VYRzPmo6kZ6ZzOn6jjCATz8EGT/glTKLvhF/AHYSVLDM/w+ouiHul5VcCpzL4oBMPp6/jPdjr5Pl9Sj0/BbuEw7Kwda7PH9gHKOWvkq0ykS13FXIq6WtAb/u3A34gSxLo+NLIrR6OZy+39Xem9rv4YcgMDng11owegtWqCyDXl3JKA8jUc4zg7+MS3ZNkmD58H4cCoPNtWBQRQjpUYDQMbxYWcBl3WnqHZCY1low9AFCRlHc6jGOVrai2v0ycZ2mMcHmC4WHBIMx8gG0hXtcY+RKJDwYNVdIkyp0YKljxqAvCDpjYdxlG10z14u+thnBUk2QLoVPIlPpp4V0C2z3PgVDD0HiZygqzvFWWCqtXAcwScghjb4KCa9S6vMakbdPEFw1iRvRT9HG8SJsCxS5BwB32Q+/msHNxmuzwav2ItU4hbrXyzi2fDEf2VpG/V0puc3HBx6m/ZltdDm2GV3IT8hOaQTcI0HfmpG5PvxSC1Vd4Ijtn+eiaSwXgONmMz26PkSgezYOz7OAYIefa/M+RE2lKvRBJhZCncaE0quWRF19E7O77AI7fTbyghf8zWh0lsYD0Ug3WBhSgGxtRot3C+eFqhDMV9vjU+uBu8LSghHkZshkvDvYgGsNUNUQhEfqiwAobcVoFPWonM63FtF7t5cxWjkYnfz3iL5/ViQJkQOi22rUpmzGuYMUUsK9NvdwKCTw7wmD90GPr/hBf4JvfaYQmx7LxLSuPGd9zRXFRl0GD3k4CHaOkZX+8GGrgcKwc5a3VLv4LRiycr9ggML+X5P7lW1sO3iSZzdcQ/uLL3H34pqM/4A+4N+7hSPFDpglN1DqoMfX0H4hp8sHEF42k7SQpZhjBPBC343Qdz12lQe5HtWoLUpGGN/nlVCdkJQBcNh5PKonoy3Ps2TREj6ctJWlyc3INQX7edMty/Xnoz91Q1Fd4vpblqHM4xjJ+lVk2c5xKAweVP3kuiae4TSeJTukfgFpjc53BWH5Yi64RxSx0n+7iF5prPsvEitGdZhYRxJXwv1lpJnF4nM7ci7LTD/ByPPC2WgogIRXuBf1O9sfucLZ57/GzfewYOXGPgPt3qRvzpc86ZtDkRXWVyuwONyFIR49k1qFkMV04GBTLVxuiBJOm4B+oPHntfaJhOjTaJ/zBV1yPxMv1/px13s2yq6cbzOcE24vtfiGOIOQ1jtuhOdL4JJDMPIJ7A8KLSb7OxgSO3He92tm/ZmFrecvIlfgiDPg2wPZlE9s+WxU50q4b4uJon7lYowAaPzZYYjgegOE6YoZFP69S15UkmDVhe9pndXsZSIfgFYPO9vagVK2EGqJY3bEebppCiDzVxFdcHYGCWo7OhncavvSYAoFW72Qz/vdE29dMD5Xb+FbO5A0C0zKfkfIg958F4YexoSG76Z40t0ng6E68HAUw+Ehou67X5Hl1Zvd9XDeBIHJsaS4rYCpZgF29/iap2o7kmJuype729LwHwLoM8xWlmyHy5susKIuha2GGr6sqqLH6WrOzD3Jyg9ehbtfo6OI07n3c6g6gEtyFrmWehHtaCoDQz6rI25yv1D04vNyNzHXIqZAt9XY3ZvIHErAjgwqb7GWeMTC3dX01H9BHy0c9otmeVsf2B0v1hLfbtDzG77LmQhAG009PRyzBcmHxnnTNPYlSQJZAbIKSaGlxqprymbskAU5Y4qQQK72fojMwC9Iin6MEq/92BvXLmexOsxcb/UUf4Y+y+78gRiUgU0XPWKZW6LG7b251LzxJsFoW9wbVXuN57wEFzbZDOUNPsiOBiEdfu11iHqUrsEfMMkd2uesRCd7NvsGmBJ5gOc8xVpQaNHBmCRo84Lrm6VmYEuQErwyv4Oy80gOCxqFAauyHL3XRYZ55bik8bBbUNoqUMkml4N/tu6XJmd5h3fYXvoRJ9p34DmvF2mXpcShb9P0UfU5HPfN4g0fGGucQB/jiBb7nI+1gtn+feijEw6Mjr5XkQy5kLsDySTeodHpY1XUklGdJm6clM02S9rfZTz/4kgo9Pmd1PAF3Al/A6PVCDcWiHfXhXKnrB8+BVbIDOeXqhihVPFgDYy7g5Q8kc59umNRVvJOOfyo+1bkyDz3CFx7jcALUynwEsSa40bY6RgjGMMnJ0JdBt6psxgVI9bhHqHOyJg2LyL1XMuRzCfondKem86IHVmWRdTDZg1jrd/yoAd0sgmH2sPe2ZC7VazrJyagkoQiQU7AOna4byZD7gKF+1uMgxb/ba6CvZ3h5hLKkz/ly/HxXHvuG36IvcV75cOF1HBdBhQeYL2vkJGNVMLVSBjssxlCRwnnL1BtqsaiqOZGpxvInnV8HGgQtp/zWQ7smB1GwssfweSfSviwHS3e5c2YfbzqLf5+QA9hBYJEobPkcDRUxaMe8KwXjNbfET9KeBXiX+SjqkeYWgRKmwc9s7/G5DkSAgYIgODKi0ygDa31gvjQSWsQkQWNEktKd3IlBVfbJ3PSXcWZys7C3k1bQyPcqpAl3MxRfNr/aeK9jwty0tW5ONCwvjSCW2aIU0F/XY6I1G79GEQ/ygCPS3RyAqp6Y1uX0xWVB5Lay7VPxRS+Rbeqpc58kWJ8ygiyR4j/Ud6tz2aHqaXKzS3rAE4ZhaPz8S5vosn8REjxG8XpKNwzDG9DD9wbYrFZfcjqYxfruawCXQjHNUv5xZKJ2QE3quJw6OMA8KgTY+VegHDiGVQtnYD+9hNsvz+cyRFHAMhumCmi/kBEF/wlt7FrvF14CslhY716JyaNqPOv+Uwlp52bYYHkzscYaJgnbIKLz0BNKn1zvmB/iNr5Wxmz5C/kgh9ugMCBrL/zHfXau7QqfoHdx/8gza9JYUiW4XLWLNLynkBt9eOcCdzr02BbAP6Va6hp8OeawYMaG5ilaozRT4pcbRMzKZDG83iXN/g1WJCeiu16QS7rsBB8u1GlG8CdzjGUeO9lUj7Y+2x2kc8UNmGbWpRVnI0fyBXLb+DjJAHtag2jr3JfoQBiQzLg+6zPBDDRabGQtgfKNBe5FjONT/s/zjDtdKGe8EAFd2uGs6vHKmY6o/plSYbULwXYbDViVhWTEewknTUn79am4Z75PoHBm0kP+oRC7+3/MHrgr2unSRkNHnGEpoxgYZiQuLbZ7aRVNAFKuEWisZTzoLyUbhm/423ogc4c2aJ+R8AA9pXM5VZ9EKHl00jPnyaINsA922P8du8JupetxL0hnuPyIrZ6jHTde7NmhmveeNf15r0OW+hRdx4KDhBXupvXfaDa/QqnE3pxNnQqmIpFZHJdBt/WKEkL+QiTuqDl2AR6hPXg1TZr6ZTzLe3StxKauQ5JoRakKb9eEPeMi3Tw6emtLDiQTEL5VGFDZfyCd98fKImsRWsO40LcWEKCdqLO/gqMhSis5fTyzCdIAWprAANTL3LmyTNN474ZQa1ee4+fKme1iF6xOsxci36EPP9fMKnzqbfWwohTMOEuGHIY0epLQnzOYfDbTOvA94SjujYdsn6ld84XLHbi/CeM8JHldYi8X+QcTfuWL30fZ6H5JZJbvUiNHQ7XJwtCVcKr4Nudn+UufISQSI4ofepv+0fzOewiZXi1w+HWip3V3lxqEDkdx3kWQW0qeLWDyTmYtP0IV6gp8drLu8rNrDI+Dyovl6R5I9G1xu06u2JHiHWr6BBYDWAs4mVvO4/+/hCqddNpMKsgYCD0XAdh46mJnMazziPSQ3ro6JkGFVeExLuljgCMeDpftdIOmRaViCB2flNrjwwe1gsyqQTIzQF1GVb0fpGvOv4CwGEj5AQIqUlP823+DIVRziBtySFjajVXnB3KL+AtG1z1LLu0gFrtbWFrhozCV07ioaDbdNaNJ1gX7hyTe4WvydZApEq8Q0VwAK0jZ9AzbQ/DtCJi3xXZJwvwvHXJXHqkb2d0tLBvXZF9DiM2J9inVCih32aQlEx2F3ujzWFrkvH8S/675qXF34H9qc0O4oUP3+bhZa/z/PUoLpc+5Lq8+sa7RN3zpNRSzq7Wo1hjyBH5JAFkBSXx29mZOo+xV42Mv2LDWxHSNLhqUwlSNtnPDeqiJuUcGUKq7ud1OZ+Y4tdICXub31I2tHjHHP/vuOWzjJ+cR23SvoMBv8P429jlOmyygRFJJYxMKsVD3SSh2jjGvYxdaZu/jLbGWa5rCkmBBC12ruY54yk7y0jrbR4wriWk8sEW9dF/Mww9xMuJC4jPX4LS5klm+qds3PybUB8ymf+x+tBvsPKQF7K1kjZ+F3klqJhQJVz0f5HWX7QWoH/0Y5TTm3Z+Fxmog/sK4OfaEGG/9f0ValK5FX+TNk57pKC5+8anC/2Tn2JdIGgbIhh2Pa/pWu4fznf/neP2nThwMNINEv1PwcF+sE+s/cqx1/jy3tfIdg2xamfliSsgoC8FnvN4+PojpFrA39Af/5rhWLx6QeyzmL36kVsbQ/Nyv7ZWqMqkfN5ivWmMXFUpWhIlJIeCbCt0zYF3izMEQX9SNgzYxtulGpLNwvZ8rvszcH2hyElqt7I/YRWh925R5nmEgTo42usLdKYk8GgDI4WHr7+uhiV+wucjGTsgOfMFyjKUG2KQ6x7Bp64fCmBD4GVmdFpEsbUntH4MSdUkVR6hhOx9wtfgXiNsO5vNgWdZBmpD1X8I9v21/Bvc+3+j/Bvs+3f5H1Hkq3NwSCoYuE2EaVdeb8mC/geRfZK9iQr8V8Z0nVt/bBot5hnBlIXPIdZ3kjAq7Q1w7XW8ZDPuzgN/77CtkLlBHLgbKkChRa1r5aqruKHJSYXSDapv85zHDJ7WBaDoBD8q/2vT7BethnY9ZlBh8cTg9M7JkixkxAy5IGt4tLSGLP1pugYJJpikdIP2C6Dje1SFzeGktZhc/x/xlKG9Jl9Emw3ZC2pv/MypfOwPsSrYVge1uq7icLo9BHJ+46GGCwRf7Yym1oPEK/3IUL4gQLHKa6CPpeel10lzP8NQHXzQYS1ehv1i43a29/q0GezOug8ArULXwgC7XToA71sxfF0touSu+OfAxWdd96ZXdGNLcTzTiuCuFCu+edRFCBzwl5wHkjiYTjVD/80MjhrMuHtZBFSP5o73ZtylSjg2UnyXb3fMAYP4IRCG6WBVbheuVUwTTtJ9naH8PEpZRilJBLrl8UL353jUt4MwGCamo1EaeSIgn0QphB73dhGr9RM55CouI1dfZm7oXSKdQM6SoArYohNa/IkrMGkbAQBh/E3UCeYdIcLQDlTE0T19B8N0sD4INIaUpk90b8UfxVs5VNKdE0bYWxvaYpxYajNQq64D0H34RWZEb0NVlyTkBmuE5JjWIaLt0rSgGhzEBI3mP0zaPUGtZtnHH2PX9uTxnfn8mDXWdd2V5NdpNW+cvpElM36m7b1eIpIDQOXJHWsQBVZICZ9PRW9PfKsnQrXT0VV8jJFOzfhVVTAgpQMObXjTS6i9eHlUFieGq3nrqQOsTTjU0oma/wd/hkKQAiINiagcCmq77Yf+v2PTtGLrQ3oe83HKHSpKqMvbD0eGgjaYi/bVWO3NZJyafRMIQ+hK64fZ3V3iQtwYJFkSiaJ1QTTEvUibQ1H0W/4yy5If42Lrk+KbQkeD2hsVFjRWN5IHzKa+42iyvRa7HGDEPIV/rjeHnGemY0b4rTrWNWcADtgT+bYa3G/BU9Z/HrXevMy0wNGTVyiztyY32o1qr2q66arh7KOQvAyqbjLXJ52p9j7gkPCVoa2cIQ7MmRug8xLKcg7Q2Slxu70OSsfcE8CEJKOxG6jpmcTNztcpPu/Nubow4TAEsFuZmD6fJ2/DlD8eAf6ytkY/Smnn45yYoqUwuJC6gDLiQve7nJWyDF1iP0ITPhcLBgIUoJMMrmudgo6y3yeB57IO42vsASkrnY5M6K88yJbEEi5N3E3AmAxiVDVNz83fzfrQ/fR34g0j3aBn8cdiPtRnOyXF/iJnaMgVkbilp6nWtWKPwUxh/J8c7VnCY1F7wL+XIEP4dKaLOZX+OrhuhscLtVTbY4TWfskJ/BxnGRb9E22LX2dEUjHj3d4X7P4Rp6HjYiTJTIM6m+CqyQQVz6PG7xGoSnaBwo2OE5vCQJHyfIu+LtDdx9oML+5/ezEFP08inZb5AmZ7ruCN8JvUae+KuiRErsCAvtBpMVWDc8mu7ghIKGUL3qmzxBgAcI/gw5ou7DVAgt8lnmw/S6z1zr6oNQfimf0RAdVjGFz1E3R8FzoJYLdK7sZze1LYnvIqs3alc0exCJTOw4JXOyLTfEhqgN3ZDzH3wFXsHu3FoQZQY8LD1B6/2iHoje3QEyAkmhI/B/dWhJd9TJCmlvHucCQcok1HwMN5wLLW02n0fq7fPSH+/qscsHsEtxw+1DkgrXIWgxT8hwD6SmAjcAY42GD6++HVDh+9v4qV772ApyKT92/PZIH5En4yqLDC3i6wLQAKD3KfTyGdnVN7baUeWk0XkSnxc/Co2M/qAHFtSx28XzJfyPJZjeAt9opym5D+M5SNItfYW0SwewtpHoPFgEElnBzbE84T41gH5x6HM9Po5v0Dn7e+RDfnFJWRxfgqPYst6ikevDSPVEvjuJeEfOjeLpCzlcDSpcRpxbp5O3Iuew2LIX+3yGuLkN7JDfiB09ZqJhhOUqCKamo8tzC+qdLgXuWNrV7HqovZgsDjlJMLGnmEm4H5WKp7MSQffs8ehzHyJYh8GKKmgbmSGV5mOmkgxjuZoYFLkOsziI+PR6GQ+PD4Qd46L0A4q6Ry5qwVYH+0xzl+6PwzE+RoAJIboGFcCrR5HlXqErY+5E6kPpvNdeCXEiekzgGKjtL2th+x3UZQ6CvAmQOW4dD7RwEuVN9mlnoFtToRfWuVW+anQFJQ5VBgsMMf7Q/yWsIvLZxJwQW/s8bnHHkmd6YUwI3yXnB7GZy6Dznvd3Y8rGCP4gkG3boFNHNSuUdSL8egsOvRGxNc9f01sq9RkrXC4xQ15ioYtBu6f0G1qZIrbUaQE36HsPywJqemygOUOhLNlxjqdOoV2aBCjhJjy78vhIxmUZ0/Rc0cKiZbHUQ8APpY0PhR2WUfXxd04GTbRMZfm0Ze6+3Q9g1hn0li1DUWpawQdmPIKCq8XmLCJgcH776KBLwbfEPISrqFQ9wLeNiewti5D5X6s+RbxQh1OWyAKmUKRd47SAlbQJbPD4BDAHIOCw2e7fmkEiL2jKT3D49yz+IEoocehu5fU6IO444ZKmzwVhmkGPqJfeDCbNjTkeLwSiKVcHDUQX55cwVjCj1BqXe1dWDYOqQebpxpl/gP2csTgq8wQSir8VQxXE9cD4BOluiu1NPeOJCBefBOySRxbig+KuTTnMBKSOUDLIhIoVXJhyJC5GEj9P+dNTWRHA+eT9eMjbyvbw8nxjeRqqxGolXw9PdPcWtVP76riBQRq4krwTOemrvfMbTTbCrdz3GtcCSXSqeLM01lEhqpgteyunDYAHej4JPgfVCXKQD01o/zStAapnpAQPVoBiffJtazvbMTbqLcFcjq+H2AIFN19NsnpMxLToLDTnrgeqZsMfJ+TnvW1cAFWx8hD/pnPNx4lz/MC1heCbVut5h69VlqB96BKUVgrRFysPk78NSUoldX8M24eEJu9RW5e8ougsqTb6uquG4W0T4vXH4bx6irMGA7ZcHvMLPzfH7sIKRGvU3O/bHwECTNB1nJjvx+pJhhpXscT/r8AcYCkTM0+lEy21zkidLl3A19l/NtRmC0Oo219guwt57VAkD66xkyo+0dJmxycLmsOz/WQpYiUYC2+tZQexc/jZ4Qa2eUQEFCPn0UzwpgTKGGvJ18ULWc+9wByY67ux6Fojlho+k5DsnO0yWQHykiFtTme1wrGsn9aZ252ACH3GaRsOVxkatIHwXAjpRX+SR1Ggqgr88E0RbbgiH3D1aEnKaLu7DFk0wqpNDRMM0G0x34pn/N706TPsLvFE96rxCykgCeCXCoP9lRJcz0gAo7NNi94c5nIgIFyE3bhFu4cBr/VhFIkX1gMwKRhCQ5mkUAS6Lu3O1QmcSU1hsJU/6lrR12ODUFfca7RATt5k7EG1yJncJx+4d/m4+NfaWWYKk+lCrdWDDmo6s+TJhaOCw/u/cMn/i8RZ6xB4SOh+hHoTIJvbL8b/U1/kfdwD2Mrf6ck+070znre3rkNClDpTueZPeN1cRXzkVt9eO0vIStqbuh7yZIXEmWcYjrvfxrhzIs6AE8by2EjB+IqDjM054CaFssV9QAAQAASURBVK7SX6Rae1NE3DvL/gaxFpV7HKPA53fKbdn8o6JXVxDpmYxkqxepHvpvgfpsnnOrJFEDxSZ/iupihIwnwPnHoKEMSYJK/Vlej9lHW78r1I0UihBe2YvY0GU9o93AIVn+4dr3V2C1EcABIdWf79cEZDR3dhM0hDFXPmC55nsqE6bxYvQHkPyhkKl72IjWWsUY96afe0gGId9YfQtkLX+WFZDjEOC7mwQTPPoLqfTQMRA02HVGDaqcRGT5kyKHXMoqODqSXkHr+S76FjGu13F+2NBDmNsudZEMw5TwbaskkX96XDIo3ele3I5nu82hUWY22dZRSNY7AW65ma9DI4E54QNoNVVEoxnyeN3/Ku4VAsWcWjpMkPXCxgMOlw0gAVtCYGboBbj1AZydBuce5YuGnbzgBb3O9+LJnx8jvMHQ4jw1LOQYm0OEmsD2EHi7oIl8KUlgcSgxO4+Wa6rhdrSItpY8uvNGyiQu1oYSoIBngjJR1V4Sa+fVV3jN8yuG6Vr0vDMHnZ14xXd0DDyO3S5y50nYmdnpDSHDeOdjV07ugLwsGk5ddL1r8/EgK/+esxiaRfY5jNic+55CUoi5Mf4O8ysFCGGzW5tkPJvZSXbJjE02cC94KXsTNazJf775R/CYew4ARq0Rk9bc4pqnJZ7RSdU81moRNW7XMSjzW1yvsVRQ5XaFOk0qAFHawyJtQt5OmFLMotJWLX7f2Ley7JQ6tYdiUGeSFvIRO+5ubfHtjX6udyvg2SJPiH5EqHvsas3DvXw5kOiFWVmG2urfQj3AYjdjkWuwOcHZ5m2qlJVYY8EeB1P0IkenRnZOsHvfiPon56IwPMStyBe5GDsOq8PZJkVHYJOCt4P98arvQZbbalJT3mSP9T8mT/5phbd+qab64pscSn+SHrd6k+QUfpGQkDTe0OcnMpnJs10W8n0Q7KiHCUHnmiry783a8mAqnUGQVxogb6hR2OgjzlLiO4jLDaC0u6OzhPF+qlEQc0uOw4izEDycUtPrxGiNfOYPs9ouEZL1lVch/UdwC8enYTpjr5o4eeoG3+alinymQ/ZT4zbc1Rc+xm6obX40BNwHPb+hptdpvrv5DmpLgOtVO6lM4txzdV6LtrfKgjijV+v/1s8NDkhqgAqHu0j3cGwk+PXk1/KelFQlIjtUfHNlrThv31wMR4e3CAhwk0DtHPt0WiJIoxPTSWrQU2CFfhlbeKLwOyRTkeu5bqpq3OQikOw4gBXV8RzPeqQxA0sLf22FDXynylQNNlMSLaKmDVdTiL3yG55lGf9SZN+/Gvn3/9ciyzLx8fGuiP7/48//v/LUf5d/l3+1tH0TR8QDSHs7QtQM6PgexzovZUQjK+kvQ7l5ZB+0DHsHKHKfyICTyzlc60Hnhh5EKRsg4n6Ry81u5VefVSz3h8zglYzqOEswoE+MF3n7rr7Kis5ziSkSRlNdYz4IX2e4v8OGmio0so02feCU3f5fi6yyWBnQ/zESNEbiVTDLE6KkXKFF3XeTM5+S+I4lDTfpc3KxSBLd+QPouIi7YXPZ4v8eFmUFFxvgvoLXIP5F1zMG57/D+LS2TF+wmEd+faRp8/BqD+5RqIYewhDfGrNGRdu8lSgkq5B5ur8M9FGuyJmxFa8xPvQcCoexKUG1DEevr+NO0jZmJudR29afzmW7Xdea95Ed2GTyEg5xZ1+tvfoFb6QPavyhAPT8ekDhQTZHZJKoAb0Ej3lXCwdAY96zg32Z2eEVwM5+A3xrmiekau58BuXnUZeeIFwFXgo4fuYSN6qnwqXnxL36GOSDXflmXDze2mLGxK3BT5ksDGmHnYy6Qcz4o5S8MxnEN/TE3/SHcIYr3XG0msEzu1ORbxwCIK1BI1htuTvg9sckprkxNvZrF9j3avUbMDkXYme52sSBnVg1zPAElaW0xXiwUM+QXbm88OULqEwtZWAiCrZwpG8BP32wmH3j9nKoYBCa7NUiz+GVF6GhnA36qQxxGurGgcXMX7qUgQoFw9Utk3YPUsJAGea/NoV5r7zi2ogbgfIoJbTLXw9l5wmtvcb32r5E2TpjVVlbRFbg1ZZFhrFsd8obtHWrpZ/8p0uaj6S3+dhPaPEU2eCW0R1J2RKAUyvUxKbIbFo2k2OVrVo4UetCJvBYEXRR6sjufJVJkfuxB4+CyAewa0I4kvEY6Q3ig48H38cTv08WpAC/HtyT5mC1u7V4VvPDAUC5h5D0LPXaL6R76rPBVIpClrEpbPhU+WD6PZri6m5N+euAlNi7HNgwhvan1nK5cCz5Pu82VSrJlL1ZyYAcIRW5oRaWlfcSTn5nOezoxtoacDT8a3K/VfW1ZPXqiZ1bjDowhh66BpH37Pp82NuJKZ57eMDen7CK6Yx1h1dVayBni1jH7n7lyrsBIlpSbS6G42Mg+zdal+0jPQoKRx5n3ax1LKgYI1j8AJICOiwi83YMnW7G8kFVP56MbjrYo4/C6jsIlW02NoUNL6uKZzs/JRyOiLn+eZujvOlk7468MZ6DiuWuawFu2fQP202Q2sSZqHE8UOpBdntxPVLXDS9tJGHuP3L4zgU2G2sEsAVga8BbNqN2jt8Oaoiu3Q+728DeDrCvC/pmVuZEfZ2Qnop7Dvx6kFGZwWenHmfaypd4+/fenCzrIdjGV+ZCyWkW+L/Kg4WujqXC2g46fwT5u+lonsvc3k+QlDCINSMHMk7/QlN7tHmBaffm8Evr4SRHiP3Cs3w9VFwSuW7g72SGZiXP43HezRPjumtS15asTOCXuodYWQUm/9+I7dlHrE8Zv0DRUXDYee34s5yPG4nsfZjuIXtxK/4VMr533d+4pm/UbmP+7XeF89/ZF97aIgbU3U+ve3tp3TClxXNlGbICvmZXNxW7Ql+gyqGCiMniMDYumcZ9qsh7B6Ueh5HUXsK5e18BHhlvM6PjO8QXLGbc3TNM14eJKKSo6fCniNh4qBBWV8FzJVCk6Q29vhN1txOyZw7sBClglEc9XrargrF56n4wlaKSRCu63YQnrP9cI78IeAv4k38OCPYC/rTYeGuLTFaVHyDTRQNlMTBWcV04qQAqrxFzYyyLnSlkPGREwvgrIlLTrfIITzbjBA3Tn4D9iSJaYncCdl0YW2qFBPY3Vz/j+9xzwgl/YyFkbSL7xAwmtREA3NeFcdyTXxLXszcT7naRhwIyXVHzsiRD0htwdITTMde0f6hsPgJgrLoOpx8kpGQBbXW1zb5YgquvigMmoJBa7j1/jW7p55jPmaFQ0LozqJRCSrWxaAOwKENoysMkiQj0/pvFvPPugM9dX5ZWQJzPDUYELYLaNNRqNZIExfXRVFUMILZwPj0U00UUpZNJr1PWEOd7mXb1I4krWMSm4dfReMWD2ocSdSi/lYVzMeRN53ObbSTuEZyWu3FOc9r1T+es/ZoibS3VeDmjwLtpYJZPrVNu2lncwjjV5muO3FrBj8mvcyTz8ZaH14gpJOu+4I4ZttdDqSkMTIK2L3nFczF/IlmVXVo6qWwmODuTdtIKfOp7MDj5Dgl5S53t3wxskJz94ywahVoAPd4d0Rf+SEG3w6jjslB1u8X2yGOiXmdZ5rOU6U6JCW8Z3BzVAkAqOwtqHzY3+LjyVj3tCa+rfoXuq2HiPdAG8HXBXX4LWECN+zUypUrqfSaDmyAhRXnfZJZfiSsPrIQsFC4G7aLcy9muOOiuAbVsB1OpyFUpyUzUbSXBv4ngsFfzEoxPERHbp6eS7Psol2PvIy1kKakBy4RzasJd6PQ+Ft+evFkiUaarJzUhBaWEkLE+ORHKzzMgrAejtXMJy/yGK8fzqW54BA4PhvR1YDNx0KTG6HR0pFkgzR4HcpNM0f3RO9gaAgplBRHGhVhSVrfoiyHXHmC8U2K02AYWZ84Ys64t07dVsDavJ7lWKLN5irbOFUobg7VXedxTRILfLe9Ftd/jzlyXWggexsfVAeQ6TChtXpwpnCuA6EayTclJrgYY8S8NJKAsQMx1tY/IS5v5C6a7X/N43DpS2velW8xqBocK6WBiZlErJzRFzbvGs5NwpQ1mRfGLbKiFvykzHB2O1FCKyZmHyawsY3DEl0LK/PAgsBqQZJkcz33k++xsbCER2VJ7F26938LBsd1jHQ6PeAG6XXwGLr+A4sx91HcawImOMRgkA6r66wJIdEbDNo/al5CR1HqImIzVvT3+brm0U4l+61r8hbOdTsDtZZilIN5NfpJjRng6KIMg5RVo+5oTyG7NvbqxqKqHUet2E5v3YTRVJ8FYDBH3IxXsZrJX09qoV1ih5m6LMSCKnedK4JpykpDCqsuA6hQYuJ0lKWtQW71JMrhRT6SQ3T39EJhKyVVGUGEX0Y5PdH2b8KoPWtT92bjWzB3RBg9jO17qvAg6LIL7CikLEc785va3LElQdh6yt6CRqrhcMI4rVa2xxsHTbkkCxDYVQ9k5niuK4apzadBKIDma1my7Nphi55bZVQuD3ZsitoieCTV38FY4CFHIdFCDj7pISMOfmgJ3v8bj3hf8GiLGz6riSMIUBwXpLncbVtx44MxyfnY2qY8yFHr/ANPtUJXEC53eJ17VrI8lhI1UdROre3sOF3d3vUrj+un6fhkabQ6dBPNDCvBv2Am+3akLepI56U1y8HW623yXdx4G/wnxL+MYl0KxzYt8302kBX9Muf5Ui7OHsi6NJzyFVLRDsrHpPnfRhzSdbe32JmJnCPUi12nC3BbAmAObOHMPOw4dFnGtzfsMaYYlSA5ZkPUAYmeL1BDAvZAlXI15iLumUyJSrDIJi81Cna0Cs6KS/pGb+GpcB9Q1FwThquw8FB3mFbcyemnBW1OGt7aIQq95wjYbuBMqrxFqWINDstNbC7Ee2Ui6ILj+Nm7FP7Lk3ngumMAu/T0nX2PajealOdin/IvNoJJVUHpWyJsqtJzXbqZUl0yGBZ7Pj4HszXB7KSi0HOmwln7O4Nw+WvhE/ZY4r1cnw5D9uGFz5R563BM2Bu4UtopTjn9kyXciMlCy0ynoCMNKZHEmKjtPmHsyk3yL8XP2maLFmuIgs30y3wUK0tVLOV2FPWrIg8wN5Gom8lu9hSKf7Yxyg7Hq/WBpUmhppelMcKWwkd/yAf0+b8jeJCRVPRNYX3KGW6Nhy0NbkBoHzYWnYHdbvHM3sjEYdHZ3lp76gyNZ85vsEbU3+5XtuWiCMfvHEJkZzRE3vetcLctwpGAE9xdCgU2kbEhxa+96L1mGl5Lm8KBz758e+7wrV5ybRxvuXNuBrvg9IpTwfuRtNIVOCe22rxGlzKF9syO6Sxb10vNUOjrwfvKT7Kx5l24bIkiLnSry9blHwdAjGD074GvuRIfzl7D9KUgiCkXLseL4i9qOCwx07nVWR4NLxlMpK6E+BxQ6Su2ezus2gqsmEmd+sEX+4iWGMPYlulPjdh27bBb5XRtL6mpahZVw5ZPF/LZwOYtjS0jwPem63MbvAoNabcBhtzBUB0fa/SxyrzuL48502vTuzq1Wc8Q74CaIiE6io/yXeaFUNIF9ADYb/zDPYPPIZJMDNta6Cf9XXQb4JHLO4GyzfyAB+tXNDziQ6MWNVrOp16RjlJuUdJSy0uWTKbLC0FtptHMfKP7hzmdCjcUtHLskckyXeO9tkq0vPAgOO3ZdKwp8N3O3cDH9ZOm/pj6kUvLj4XrqLT4E3zzHgCSx0LXYxyXYkPwmr5Xxt/YgdDSLiqMptIHemEBM4ZtojffEuVapIyXhA9ZUCyIMgA2tIAXbLWCpgovP8JJ+LVP08GoZ/JiyUKT7UGihLp2ailuYtLuo1iVRZ/HBS5Ut1hjAw3KJ5yLOEq5sWtOb7wc6Sxhjk0sYlCvG9XJjhIiGHX4SSYJ8303sSVRTrxOAcHOwr3E/kAFP2QnYOayCGNVQwTTTSXqk/cmZhH4s9Z4rfKR52wEHvvV36aMF2aajMukgE3bdJrWL2A/I2wX61jxY1JlvqqGVPofPR3eHjB9dz53beybvJYaRGbgKO7ChvjWXC8bRQb0adrTCx9ykQefhEYmm9zrS9B+ipjUAtVuFr7PWL6pFf/9n2NX/VnDvHxX133IR/J8r/wb7/l3+RxR7+BSyHD1whE4Qsg3aAJLrI6l27X3/PGefuNpyE44unMma/k9zM+oZXuj5DF2LugkW9Khz0GkJBxsSOS3Ij8wutQkgr/CAyDWSt5OIGx3xcCZVv1UbRpXXQ8KBW5cF3h15uWIBKxvSSFZD256TmKD851KKF4AJGjXLln1EqC9s6/cOX3oFsy4IOsu3xIactwOqbvFDsAF/BeR77ee21zbhhLeZwVzlcnQ3FlmSIX+vkMxqqECFhfwicbgfpFYQX74CSs+ITU4XiiJ4MFntnqJg0qOMDr9DuON3EbVw71tI/4FbYeX01MDujGlM2WIkqa1VOGb4CxPVIWNXh4j2dF7TKWsY5m4iWglmB7xvDHEZtI33uoet5ZtA0NQ10xg3FtLXvR5fhUjSuyq0UMhANZaGMtxV1a5NP9nWTVRYsBvurcGu9md5hYhkRHKIZ7V+HOKeB300Gdo4/rBm8UXrngw48QlrKutENE/Gj0gS1Jj9sdi1RHolE1f5nGCa5m4HtS8FtW0w28UYW1/p48y1MBeS3sQme3Mo+A3y/X8FoMzu3+RYcX2znXVVEuq7YPEd3Oybiwn3C8K70kZAWQA/+11v0a8Vfv15s0zke/mqGq6UJ2KLfEyAtrGzoaGCMeHvMswNHvWA1FYw77GBZOflEXT/Al71bMUUD3jWH64OhBU/j2Xe2wJQUSkaGBy1ni6eWQBEqaB14RaouIJX3S2ejDhLx6pH6ayGUfpyl/MVYHjAo3TKWofC5s6mWlhuXysAUIDOS1hQIaIydBL4KGzIzQ/sdZlEXAygbXg26QHFfF/QpYURYfVN5JdaIad6MP0pcmvaomjIhdxt2JXefH7hJ07VNXnTDzVo4YEKiLjvH0r8/NWJ2jh+xN922BkFl19EYcwnPiGX42P3URUYw6bifS6AG+B0yR7utT5FabAeh8PQUgbPVAIlJ/FSN8mV1tNkiAM8GfsOcQXv/styvx46mWvt7uDRKZeq8GHUFB2D/s550fZ1fqlIZdvtN3BgJ6kBdtnGCtlKjT9cnsP9rV/CqhBelyAFqEpPiWjYsrNUe7Rj6YUoAs91o8azpulwCkLKJOYF0kaKCNWIM/EMDjrY4v36bo7hcOcQDo48yNkpO3m6oCkyUJbh5dSRLKuAIfJCeqb9SUevfq5rRzIfZ8ImO5cLxlGpvcof+Tep0wgYNE/zDHP3XsPjwGXC7rSM/CTyfr4tv4X1chIjkkpIP2KmtONBCB7u+klnTVPf9NQZhIzdvW9AE0D4vU+43Qp8KnzR7BjNxfJOggWYukqsvbIKB9BbCxt8wvBV3RVO1tJTFKgeZlKuliLtLarNXljwhMz1IsLpdx8XqaLM8yjVXgdo0MSJeeFi9P7z6AFv8zXO9Cqm5oWfGLXka4ZJ21pcP2vuwwEDtFLBypjzyPnbBXP76DC4/TFB5XuQfA8RmDiCN/s/jDFsNvQRESgYCnjP6woP6AUAf7W6+//H3l9G2VWlex/ob62t5e7ulVTc3Z04RPEAwd3dQwiQQAKBQCAECXGBEHd3t3J3t+1yP8xde9dOd58+Z9zx3vee8fYcg9FdWXvZnHNNef7ygIdQ30kSPNrzGb6flEobWMOpeXD9EwC0VHFX9H56akAbtIvzDe1yijZlc5ePiWAFBCS/ztBed7kAE1mNWRtLnvYsmZHvEeGTw10hk8SYXitmSIMqmg0tUGyB7xqhXpXm9s5cfIX7Yn9goAdsSighxLLDoRDYDOef5ZjPEQZpwWD4rwH0lcBw/mvlH47jw9QaDh5Zz1MdvmWeL3zbAIX2ELGBHLYTUh7DipK2HvZ7tON9W/Lh704Yk14lMl/80xvmu0lWA10/FpaCocPQj73Icw6+h05TAJIdbFaHislCbMNxpjn2jN9WpHBF9ZVQcsw2s7NyKamn57DVEXsK0kQIlmv3z1A0XuCh6NOkWBJJrHiBEdezIOVR57s1+s3kaKNLOR6maBEB+oDugKs/9tHAj6Hga7jmqpj6K/wRv4ZxGTGEjIJu4WsEaDGtQthH6srwtGbjqzTzXiD0Cb6Mouki3PpS2GpKMq2YsAKf+r7Pl3knsAX14+rVq4Adb3UFYbKSpIpXmObVAzYGQrYALzKbxjJnUwPXr39HWtn7pPukiro2N1EZMIhZ9SXkKsqIUsIQT71QVgH4dWS9cgrHHMHumJoHidami/woayQwN/FgbR8AJnrB0qhaJzu2rdzX+UHCGiayqqQPO2vSb7OJGkKR9mla25wZkKDbJzDyIFLoEBYc28KOnCewOfI/qWSVAOUKfqWn/DJKRx6boOYRzAlexIyOM/5lv1QqVIKYZLNi8UhgTTPkxxYRkxJPiBTsHF8ALLInRjv46DPY7DGcx3WzhD3smNMQc6dbECZGCamKGjfg1mxzzfPD/SrwrfsdtneA3X3pFHqYz2MKSVGL/DtxvgnO35pMxcTFfo024BAlFlhbnwC9vhYWZkkP82r9E6x39Nv7fGCe/hlBsqjYD0XrGOfTzur7NqC57Zm7XunKZI2Sr0M3C6tqlR9ISkbeeJrVEYeJq34MrTkSpawS4Hxwf5hwmXn1/lRb29oJVJJMG81ZkmBbwVRml0OzqooQ/cfU3VzqehIJGq1qWhztHKoAtanaeUwUO1FKCFToRY7oEGG9PNXrKK8GCILRhvoAGrWdBVhSeRj0Fe3mABv5LQPEerXtot4JGBKeQxfaQuD4I8wOOyyIcZ6xoPbnZtzjDC0VBB5JVYPe4lgTdf0YhWRleeJ5JntB533LWVJpF04UZx6D889SZXgO1a1fSKp4xf09Or6Bre8vvJ45FYCS4J95pjELoqc5Grke/9btvDpwJikqOBINM9UbhMqt++cw4RrjVct5zl/83AsJyVQj9h29v4N+P/Nn4ExOy5mUSw2MvnIHTbEO0pRnNOjKWKb5hncCIdGUwcvxR5GuvgNnHkVjzGTxyd/ocCvZ0TCOETj9BZiUjUGZ4KzPXlmJ/Fy/X4BmlQfB3MQfpdsoahLzyxAPiLk+XtiLWpqRi9eRoTU62/zbmF3CWtmh/guoXcGv00IZ6N3kqC9JPO+dtdBFEDNCLN0ZcamexdtbuCm9IqxxizaARyQ/+z3AYT0UhX7PoIQfCWj9061/FZhlMpXZ+Og780GwgqjK7eARjl3pQ5zfVd6MzKOTI3b0kXc97OkPx2fjJ4k9maq5D39WdQDtcEh7FrTh4BnN3y3BlDv6fWlKE+wfBkdnwLHZtHb/iqcc89CWFniq4iWRW3bCVejwMmhCqLQP5pt6BVfjYGrkZzDYocQ1VFARez8PVjrewS6TaXtS2JkfvZNO/tuIqX2Au67ZmXjOTrq3C4AjagqvHNvIhTYFiE0QPpCUMDmPpt77aTG7rOtuJ++2X+M322Dy2ScxqZPAI5z61B9JLn+dfuW/AHCPDzwa21vYum0OxZi5jAWEczFxLreiX6Xab5er79vtaA9P4KcwSFeDVdZxuWoUBHQDIEn+lQWj+iP5baLZ4yoA91ouwu5+8HdnJoQ+RXTtffTI3YDKGsDBqk3U+nQC/wxUPgk0NndA2faNShL4dxKKQ1nDkgBRkW1pCWRJFjGIax9y6so3TD4exJ7ugfwc9hSfVIRi9UwQypbjs8HUwIzmZNY3w3uDZ7JofLp4J7+OED2Z1sxvia9/jEavswwpgfuPL0FqvgldPqK+81bWlAzmlhmaPa6yu2McfX7o49Y3b99PtVfvybLkdlwpK+H8s3DqAbDoSFWb8Jeh0QbbmkKg8wdiTK45hQKbMz1Kt/Zc0KHboWQLW+T9THesg66Z4JhpuHA2qjwIpX8To79BXy00eVxB63UTsxQAA9fBzCb+LFpA+NmJnDHe9sxX3kVZvoEbeg+KLI7naowW8/KpeXDyHgoUPfhS8TcAk71gqnobZC0VoD6Q4TmCTkXCheCiQcIaPAz6/iSIGipvys0DuBHpR1Svm8z1yxLWzGU7wCcZj8bLzPEBsHKyZDq5jQMgeoogt/VbxUZtP/brYf2M9VT2O8sfHrXO/FqSBPnNycKpySZysW4Pnuv2XcgOsldowwS+jI6hY+5nbm1nt9vJNMGE64OF+8apByC4P881L2d5g+t3Gb5Z9Aj7C2wWLHiRE7Sa7c0fUN5aQpX3EW7UDoPWAqg+yrb7j3Jn9WWuDnuS8/MU7OjhwabW5wCI8Y1hfo/5RNbfgd0OZ5Lv4O+eStZniu9T7dgrmjFgdfQGhayAQxNgVw+CFWJdYrVZ6Fz0LZMM6535IgFkBxxsk/WOc9spS60GSuxaLhshRgVPB+mJ8XHFV8Yk/sBLA+5BYTNisEOFyaX2B+iq28MdXlDju49zSdMpNHUTDi5RE6BwPf08XNan4K7sa/K4yk6eJTd8EeBwQGjXTm37wPXh8G2oYx47Mh0MlXxRJ2JpV+If4Wrs41jaETTaXIpKg37nYOdkjvq6yKYKWcFd5SBlCyAY2s3rQ7dB72+5lL2BbO13rmdpWwuWbAH/LuSoU2nyvITXNXjE6g7Q/qtyn9HC9l0X0SpbCPIoQSm3Op8HfQWcvJ9YaRNnyyfwZyvUJUI/3BWYAdZUfFu706noG+5nAMEnu4h9rdVEiFcw4fXTsWPlesxzdAueLpSfCfeJeGmRAGj/aoXdOrhYPVoo7mfpoetHqA+M5KG0KVxKvAd/bSVzwsbAyXvh0muEN6/ghYTDxCshJ2gZLZpsPCp/h2Oz8Mp+nUmpS7HbwcuUSHLZm8xMeVg46ThdLsAuu9rHR+2yxxR7BQUpKmhMglnqi5A4D6w6ODGHp1N70yt8D0Z1GQaVg0kWOwsS59Gp4Et+DANvYxohTaPxMibh07BZ5BU9MgXWSDzuJ9bL533X8dOV9yF0mPO+Z8rGsrzRRqPXBbd2Ntu8QBuGv0cgm2aKubyosQjVjcd4OmgaalMjAB6TRlLYaQJGLxew/s/Kf5R9/7zYbDauXr2KzWb79z/+P1D+A/b9p/yvKXElb4Dd7PjPzodZSygqv5PQhjvokbfuHweVdsq+24OoWlsN/TyE9cHWmgRueT4K1z6GXb1AkjjIe+wtGwbABb0Khv4Fk3JgyFbo/S0N0e9isIkbvpW0F/9GB/vHodo52LocAL0dNImP8/LLDzBEITFGo3FTVg1WSgxRKHj9gRiej1+Iye7HH1ffYW1NLGNK4Zx9gGBfF62D2jPc7WfCWxLBhQytQajc1mlgYwDdTidzj2NeCVFAP202FPwuFCymemoHFbNSv5bNT2WS9uRO7JoYiJ4MPb8Sm367HVmykRp4hpcHzMXLni827LoSkNWUWRWYACuSAMDKH4atgiUmy/DyoEk8NmA65RaJut6nIU2oGiQJIn2y2ZZYzCzH890OtkxIXs4PoSLHSX/vYFdDJd5Pwq0e7NNBdOYmvrqyS0yOjnK1zybuarxCje8++mlhsno9NN4QB1MeRfJNQ2lewpC8jSwaOYShoZ+LBXhvETT8y3MQz9RYsQKFqltYJS9hC3L6YTyV9cT6XeNSt1hWxk7ggPdiCBsBvqnINh2BHqVoJQkvQwqeljihNoycCEB+5O9s04kJXwGEKBrg9MNwVjDCKiw32ds1ApUllLHn7aiU7Rgf5ibm+7eifOU7/B//nXovdzZXc0Afvs/z56V33+OhlQ8Ja4SwIXDHNaH+9Irlw+tHWdoggj6Yw0FSEB4ezoyZb9Fj6C26j7tCxphdzJ90gkkTvxFsPkAt63ix/71MiRLKB7vvEJiYJdS0HV5k3rZCSnWhPOkPP8bcFLm4AHRlTFNsZ6IiEV99V3R2aJaCRfANIHwU+w0BACzyiSavyznQt/NalzXk2TxpvRiA2qRGsstuQVStUku0qhOljZ1ZdmYl16qG4XUoA47eiYdOgARyOwWL06bz5udMsHci1u6+QJksFQkQnLYFmIv9J0mSYH9HTURdvou9UWAdcprjowOpVme5XcejZhmqUYsIqWihT9OTRJS0gw7K98C+oXT2E/3xWX/4PPScCOI7Sue651iS+jetneAHxX9vOv5FLdN/SCcWFHlizozF6BmAye4rbIjn2qH7IhrtqTSZgkGycc0EexgnbCnHnQeFJyq/n9jdXbTHA77geeV16P8rdP8Mv9ipnDk9h27HBnG/KYDHAo673f/FPS/yo39nzg7ScFDzPB+ebme/cfNz8sPy6O1pJLAukOCDA9lXEulUTMgy7KlN5bgBbFgZHv8LwZb97dpBIid8EVdjn8SkrBV2iTa98/IqQzPalhqCtEXEy2udwVmAIEUcvvquaCwhWO0qTIGjRY7NMaeg83uUW1zRg09qoqD3tyKAJUmYvBI5613P9e4X2T1mt1gIR08XY2Pnd4gzl5GhhgH60cwKy8RPWSyYvzNbKNC8wI4WDRZgdKEfXzeOArNYIGNuYFLACcIUYFJVkd9hHBpjtgBTHIvt8KYJKKxiQ3m7cipet5ZEjR1jQgXBoTKBqgi34212LnlmmFyscXynLwnLrcbrLPQuIVUFpw2wMGcMrQlvOhU5WJoZpy1xBlXaz5GyDKdKpvJT3mQuxz9IgXar2EBVCRasF4W8m76Ze3ygKAEmqXdA/RXY2R1O3sNvUU10UMF4T/g0vBFJXyKUd8fnUpS8lDftu8mOfJ+qlnh21P8MSHDiHoibQ3bcj3RU48xTopBlYb2yKRiqj9OU+xvpIWs4b4DHiyOp1M6BCddEUCdqIq3Bs6FuDPZ/A6BvA+79L463L/fp9Vw7vYYx0XvoqoEnquGKPU7M95HjwDuRXl71xCshpvph0ogUY3HCfWC3Imv8aXSs8edFXiJBfVVYyfVdCR1ewmtrKKYUmOUNXQd1pLvvcpGLLeVxSLiXA2lfMKLU1U5qqVnk25KV2GU1qpY+BDdMomfORsK18WJeT30CRd0xXkhby/TSVXQs+cIVQB15CMZdoCTmF5otLq8mX0eAhJg7Rd0rJLDLJKpgnh9oze62RgrJTPP1GyRXLmFa/MOiT3uEgcqH2sMz6FGdhsL7Iu8GQd+gS6irtsKF5+HGIrDo6a21Eq0UuWgqjF0FWOjof++MHM6nE9I52qE3ensIJD3itDyVZWjwPMvF+PvIjvgIT/05oZLMXeWmfp3oBdviipz2tACS47uLqX6YrgU/8ZTyVdjlcGXQVziPr26CCXmRbuQOzM0ocz4nOmkBZ1MmUxjyrdu6s6KlgilntUzwbaUuEQZF7hE5GcOGOefCy/HzOJc8FXCAfSpf6LaIPO6jDdAKaO3LxICXGZ8y3nnt2+cplayCzWFwcDSmoGHcXQHJ28eSszGKb0tyXe4HQKS1lBQVeOs7cq14LuU+T4oA6cExcGoef/vlO6233qmDLzS/w7ZYBwDawps1r/OHAzn/MvkoEfn3QNMtCB7AmbKpTL/Vl1stwaSVfkCMb5wI6Jx/Ad+at/l64NP0jdlAV1sC3pKHAL3qzsLhSRTYop2AW70NWuUgMFaBbyokP8YNg4vAIyELVWrO91B1DI/qw/wVZefc7I2Ye+kpsYwRQfjmHChcBzHTKTBNJMb3Bp+N7kcn7zXQaymMOQFKT7ex9o+ASG5GnXWqIWUZbtR3ZV0LWID+xXA4+vF2fQh6WzuQ7Kjio6lRdDl/n/jD0khQ345oY7/gbLSKBaG7xAm6EjBWc86yk28PXMAmGzmTOgF70zEBMO0fBifvZXVIJp3UcCXuUS57Lsat+KZi7vIZv88u4ul7DtDZu0Cozaw68EpA7xHDJaOwefKV4XLdJAGulu9GLdUzPbiYDDWADVmyiPvWX4b6iwQrEomuu5fMyHc5lNGRwhZhCU/6s0iJ97ophG8YPaEtb1DJFsIbFjPD18ILAWLv4YkefFOgw4vgn0FP5R7myp3wlaEpzYDXqRFCHWVuhMhxZPn2psLqamdD0ATo97MAMWUl9XZfGmyQbOzC7PSvhYtFzveElLzc9nUgvg7HRTSB4JOMp72Qjzr9wDhPuKXMZp9hi7AovvginHmEPGkvRcErAUEmtEtqYU1/7insqc+wsNxF0NvVlAJhw50geFjRk/hrq0n1y+H3cOhm2SGIgJvDoHw31y8sQJ+k5kjH7o4XUwpgosdiCBvqBrCPPfI0NyN2Of+WZXjsxlSerBZENO+K74Rrx46uaPUXifG9yfMRhYxqEn2uvq1tgvrRQiJP9HqMFf1e5Yf9NwgM+UbYh04vh55fus3z65vUIn9Y8UYoWodH4xXnsUYblFrDhUVi+R44PAm6f8412xuMutjM71feJ7tprpjn5tqhywe0BvRmkwO8/zIuEw+pUljMDviDYn1fJiQvJ973vPMbovGGUGtow7heO4gGxxw58mqh60PzjEGSlXi0IwdKt232ZRnCGichejbs8v4DP+MhuL4Qz/rtDPWrIax5DADdNRChuSD21YDSVNvO4F2AlO0Be8lQQZMVLhthb7cwBjUdEnM3oJVqifK7xtXYuygOEY4Jx5VJwn5NkgEJP303IuvvojDkO966MoOcWrGHGBCcyF35e+iZt9F5X5IfFf0vaxlpSkGwaAP7JGShbq8+jsba5Hzec0ZYWBmJ3TMBzjmcfGKmkW33ptYGa1ts/FEThZ9hP5x9EloLqUt5njHtpvIhoWfwPNARJBlT4BgnUdqiaEavLqK02fVjSQJPYxLBTS4iXXtlnwADXTWqlFUCYO+2CGpPczolywFugR0ZOr8NpkbY058h9lIerVuFpyGZX5rgHfNHMFMnbPLqLrKXaPIdcfQjevi0qbcg9DXeAF0JvyUtY3wZ6DWFxCQuRGWvd+ZXlCSJ3jl/MfGcAJszAnuI/cO1D1CWbWTWjQl84OAESUjCmrFiL3T/jEpVb+f7LKiHLabpwn6v+qjzndvm5r9aZexBg+D0PJEP3G5no3EunU+eZMSyR3k1WATbCegGXRei6v8rX3CCmLK3KA5ajU5VRPvy2qDXuLPhODcybvDtuL95udVIRb2Zjz78kGULO3Bj20S8vgYOAc3u6ilZhvFh57jPB6r8d+DRfBCKRH9TNx/jxynh3BFyhqiS96ksfgpLxJ3Qcyn4ZaBQhKIyxjuv9fPAV3i+x2QY8AunbSuIV7v2Za02JT9e/wkixgvSsd2GQgFG72BqI03YZAM2Rz9OC05j+YTldGl4XvQBSdhxyrL46HoGjiSx4kWibAMINfYlseJlBkeME3EfQxU/BAnQ1eYY62+P+ykcYJ/VsZZVSu2+7o4v82DdICZ9+ST93n+FjPMJnKx8wHn4z8JZ3J/Tjd9KF3PCAA/mzXZatQL8GvkNDznIDBUBW5DkdoH7E3OZH1BO+9LmIKRQgE6dzznFUvQaMbbdnkqkDSDvIIVyt18V7B0kyNupTznH7AbvUxSGfudS3/GPeZ1l/kkcxFHs7VWOfh2h6hDdzs6kxvc15z87x9b+v0HflczdPpEmz0v/Y/ehpuZGRiWs4uepMaQHnXc9q6kB8n8hwH4JkLDbhU3n6dY81wWKt7AuqoFZxesIbh5BcoA4n+ipIMmkFyzjNfsdRNXNJT/sK6YFbRHk6vxfhKLWvzNTKxK4YYJ+mfvwskUKks3FV8Buoyx8Mr82QbPHdWYMGCmuHdQPbnyKTtuDWScWcNFBCmjxuIGq5TIUrUdb8AWjk1Zis4GPKZX0so94OGGkUC7brP+UCBF74RFnWgQQxxtt8EMj5NpCxFql46sQ2BtvZRVqhViD5iUa4fgc4YTS70fyou/jvVqhNozwzmF4/K/YFd5ifxJ/DwAv+N/gDi/I1J7jIY93IaQ/4Ii75D3Csw4ij5cEfwQfZ3bGB1w3zINxZ8A3Da0jx1+iCkp+NHJydTdkg1gHyv36UB3f29FHXO/3P1X2/b8K9v3fLv8B+/5T/lcU6eanKOwG7KHDRJD+7KMkmO+gV95G+uRsJ7xhitvvZRkSK18kok7YbdwO9lX43g1APw/4SPsdm6y9Bdu87jyU7aS33yQ6FX3jOFcSASyfJBG0jp7EW7VmboV8h6chkVMljnt7Jwu1h6me2d61dNfAgNwtpBi78+wjd1K41E6Pe8bwkncq0xzKqrOD/CgsKeH5px6A2Luwyr6sufY+2+sj2KuDWilcWGlqgsArjsCsEAotMM8XTqRlCS/7dsXb8Zoz61/k3aB1ENgdJucJMEehwVvbSErxShbtqsQc9aZYRFcfA10JxvU+PDbMl9WB7/LyxRcoVd8jgmuZSyB8NJOqw7lkhMrU2URGr0RlKRVWPIgB3NfvHIrgLezrFsG9OydS0lTiPFbdGsczRbHsbIVAGb7wLBEMHEcZHLeWjiqZoFzQBT3k3vaOtjPa1NxsGCssUhxFb9FT53EOgEFaGK1ysGNlDWgjkGqOM9z7GC0d7iIw4BQ+6kpx372DoSnLaeXhLUG6ZwPxqu3i3EEbCfO4yTcTOjMtqJgqu5ESzUgxqUaMRSpex+qp0QRm3E1SxSu8Y58rFhJDtsJcOzrv4c7gSIQSvvT7UORKqtznqBMJu2wmVGUgPfgEkqWxXSMmkJoXzSdeFTwXls3T1T3c+7Yk490iwIGYkhjmJm50n3AVGn6Q36HKCmua4ekLv4l+4GgLf1UrqSqJZt+92LC4seLMdm8+PrKFDYUCtGyxWUXQRu2PWelJrr2BOu11fmyC58o6gMLh/26sIkH/KelBJ7FjI0CGICrdrMTaFqUlpfeyK2c+UjvGHJ6RzKgN42JyFl061PByoLt9TGTRam7FNRLqUcG+LjHoVaVYwyeLg0pvnuv7AEO9Wp2/T1cjlD7GOiSsqO3uNp4D7eUCBHco8dJLhW1abNWjYpPe/TNIuAdt5DiWHlpG3Lb3SQh6hI4BM8U34yj3sJ8nA+yc7HeSkE45+NnbKfeCevOj3I21Ph8DwpZmkne922pHY84nyrMGesJJ+3/T7tdiY8SYMSysVhCcFYe2pRpvZbUIsNqFIuiU8XmO9VYg2ZUMv5LLmLB7RR4viw5mtXJnUbLzmkf1YMx4B8JHg6wiLTgNZdDb1ITYCP56HpO93ceY14z7eE/hS32gjSq/VFqtQa6DPmlsblVSZwXfJl+iSqNQWN2B/bbFsM1u4+k+DxNnXOo85qupJjHmB/QRgiyRGQ/RFwVT0WBYT6eRo9j4+EK6PPYYo7RzXPdtLaSjdhVhXmLDEOxZjKr5nOh/wX2h87uUtgP7THYVeEQJC6HKQzTEzGFyORydvo2vZp1kZvROkXsx7RlQ+fJgwx98FgzbqlKYuaGRaks32OALl15DkiXnGFXjt5c88wlhsTjXDj2W8FzUOmeekAYrFEW8I4L/jr7XqXIB3fJXizq4nZQiGbHZJOo23eD1NVeo8Vvqdvxt30/4IxyabLCjVSWSyXf/DCbehC7v80xrIjdNcNUEq0oGiuCps61SGFY9lXdqxXzyV98pIvjraIvDhXN5yPonxcGrOeB3n1BrDRXja6sijXkXnuCnJviiHrJsaVB3TliK1J7h7hJ/bprh+yYYlhsmFA/Fm6FiDz7NZ52PsDW1Pz832cTGUxsGiQ/SOWcs1+PgnUC4GQcJur+F9Z6xFgrXsy72dXoVQaEFfq8LxKBMAv8MiJsJ8XOp6/AH2UZvB4DePoznXhr5n21eDfoa7j6w1hkkU8oquPKeYH/ry9mYdJTH/cC/tRcv7s0TioekB2HiTRSaQLo6pPcLTn/DzuZ26nS1i4hQZ4Pn/aGX3ypB8Kk4AIBV5UOzI67wR+pxJhl8hT1t1TH8VEWMtQxkZNFPRDTc6TZu2qJn8NKek+TVd3O2K1WHhXrPNxUFerTtAgeFlgiRa/DK204Fi2RXsLEFvHOgyX+c6+IBXfiwaBsc8KFw2wieKnXMy6cehFtfUeDbm0/r7JRYIKMQ1uRPwxoiAq5YWkFXxPFYPY/6iTw3PqoGZyBdlmF/6Sh+bgKdNodiizf0/d6p1PVW1dAneiOaiF/JDfuMFkWQAO4De+Jbd5SfwiBWKca2F8sjBfgN0HiL+y3rGesJIixsR0WjGPtGHYZO73CHh4CICyxwpNVbWMy2FXMjmusvMzD8MMej4bW4k2717ZX7PedjjHhKUFg9EqvBpUqVbCbm93yGEbHrMKjF2kilUIn5puPLnJFXU6/O5nBGJ06mjvznm+J2CnSNQi0C7WEjnPm+jg4+SlGn0f9w2n2NX/JViAgU7Ml7mPygr0W/C+oNmkBst5EMdO2CyWAjT5PGZUfw4918hyJnwBrouYQ6QzTmi6foc7Ga0CYHOGmsgcwleNibeLIKTjeGM7H+FSb63wBjtVgna8PwlCxOR4q/WqFc0UGAQJpg6PYJV/QuNY+EBJYWYf1YuAalsZphnhCTlczN9XeRbxXKABLuhfCRNHZ8m526BymIeQ1fn+sgl4l+V38Z8n7hI79mQh1T02FbGYWKKS61tQTtt8dFFtCrXPOcLMM3vT7l1zDw0XUmqcubKNIEkUttLePnxJvM9YG9Wc9TZhL2zUzJh36rqKcbefXdsWPnJX+QsMKUQqGC80mhi7oVXxmM6jJeTfoEdnRxa5t1t36hxzE9Q34czAf5DwB28a3azajsFnxk6KoR685+ob+LHIkn70PGSsejr/JZjZbXunzBs6EqODBazFXD/ubpyEDu6/o6rdpMWjxuunL3AOT9zOyAGlTmIEdb4MpzmfYMJSGfs6lJzdpm6FAIv1kchLyG69BaxE5FNu8f+w2rHVY3KrFE3wN9fxRAEq61YZAMD0RcRbYZIPF+0IaANpSPLA+ztAHyDD7M354DI/ZDYE9ag2eRFnSSCSGCUHUozhG4M9VDazFaKpkYeYKOaohTQoCiFUIHi8Be+kuMDJ2PLUkEnSOUINlNQoWpK0Ey1SG1C5yuq+8qcqI5QM7StD08XK7ilyaY6wORtpsCuPeMAVlFdOkffBxiRqto5d4ubxIt/S1SLqQ/D7oSxuj2OPOs6qwe2JT+uBfR/1q0NzmV+Bs2v07QcIXA6mWcLx/Ho9szuVos1kBfG0JEW4w9iVkRisnqgcHSbn1ddRSufgC5P3Eh4SyTHUPaExXeSN0+hlFHoPe3aJuv845ajNGhCgdJ0dQgQI/yXSCrudb6MfUdx7EyZxbXG6eKNZZjTdweaBjo00CIdAK0wRA/Gws+PN77SbqGukBNbnwq1Bo2AyqpPfxucz131jcEHAhmZqzL8eWfKfvSyt5nhmE3MhBljaAsYgnE3knQtUnc0/ktbA5i2IvVEouKLTDHBt0WoixaS1eN29Xa7ackbOMv0alIzM3gTirMlJ5j5KFvuNruUzmvTBB24xMus7t2Wbvr2vCToe/RAYL4eWQan43u57SKk5DFwJMrwOcJdUmAy/ZPliSx/5ycT71fb9oXp6Vr+vNCKecZi58k5tUlDfB5aSe8TJcge7lwLdEEc9iB1Qz3AA+FEUv6O+CTirr5DFv7fMYsb7BJYtC/nZwbXXc3/bL24mF0kERvA/vauyqpZBVETxLrM694Pq8K57wBpnpBZuppsS4MGwbdP8frxge81fEzYmrvx6t+Ap7qDlBzXKhuyv5mmdSdsy6xLdvrFwni3F21kPKoW//bVBtJduD3ggxRso1ATRGdQg/hpWpwqzsmXMPS5XN65q2jc8EKJnnB6fTdQgk78hCkPEE/3UIWOob/Ugucsg6B4bud45fJZsCodOz97BLWiImCeGeqh8brnE78g/S8JEzFETxRNh0ixsH4ixA1AY3aixTv/hiDtrFv5APMTHtaxBLWSJD9HclXnmNqQLarfk9AXHQ0xxYs4IPKW2wwVLOiBoYcA+USKP/pOyoqKvjoww/Zsb4D61Z8x6XPgUPwo/kBTNPE+saInXy5ksu+K0mpfJ64xsnYvZNFHGx7OjN8zMzIOU6MQtjnzj6ynA/OCNBlvNybd+LPuOrQLnN/x6egfCccmwF5q522nVbrP9pWQjsLXAcIqFaI72pY6J10LPmcRNtYQvVD6ViyiEnxc6DDC9D9C/YaOqCweoFNiQ2LO+AGyA4bWaukd/W/duVbw3VCakLw1HniWfIIJptLeZXbmsov9kuYnddyf2aTKthJSAAIUuaJ/Ir1V2DQelY0xLr9vk29116511b+MR+z+HvKxXu4rrtP7NVSnwB9OU8HuCsG2+8TFbcBeu3vo5SVYt2XAoOI52jH7uToHe1magT/LuzR9qSqHZ/c+Q1pQyH/V4ZoxOD2P3Uf8vWwk1ffnZ9Ku7Mv4mXXtX1TYWYrN+S3eKP/fZyIgdGl8H5du3dsySXD/2+81TW0aLL5MWcOdSMbnO5FUvZyuoXvdY6dfzV2EOepfKHzezDhChdNol3XDbufZ3veKfb6Nz+DhiuUxtzH146Q2986K+eanxUknEk51Pvfh67wdbzNDuK2XUKfthDmWGkZepO3j6zjeOpQ9scNxo4V36IPYO9AWKt0WMa6z02a6sPCargl39nOFVaYXwWnrQnCwlMbBhmv0yk/kg+CBaks0yi77ddrgoezvgWaPa8QkfghL/S/Tzi7TSmAAb8Kki8wzj3UJepdbjdHAP0MbxOpCkKt1Dv50ja7jbOlYm+ulaD/vG3oZvxFYbSI8eov3qLL3i/wq8z6/ypn33/K/53yH7DvP+V/R7GZMKqikMp2iMBN0sNMin2dOL+rzp/cPgCFNU4gulYwH28PojZr+zKrHM44sAhZkqDTm+KP8t30Mc9mcsIfAFxLbIC/UkXg7ch0KN1BiSPpeHz1U5y/tZCb3e0iv4lCA7oyFgYVMMULegRe4bfpYciNFwmf8DsPP/kFi5++i+bvIPp5iBo0mPCwMGGz0kvYQJQFrKfSkftClmRhh+krgqFGuwI7Ioi1oCJc2IrOtcNcOwUjGvnOMYE1ljzOAf1qiJoifNTtNiw3n+PTPr1IvfweAaXXRH015wjGi7GWS8p49hlbyVRf52fbZUzKWBE46bcK1H7Ohc+00CxWDH6EZr9pwqrCUd93Xr6H8WUCzOvR/Dd2hwpEloUd5tGbH2PM+5QnPbYwVt0oAsOOcxcdX8fArHTqbKBvp9ZBV8YYr0bCFS7/7PYlpHqvI3AHq5pgsXGhYKYN/RPChkL8vfTw2kRnNUTnqtld+ZlYnFUfA5uZng0HWBAEfbWwp9smRnvOhJzvIKgP9eZEPitJd9p+yJIM1xfAzS+Q/DP4qawjt+QSTMpqeoT/Bbe+EIE7XRlRVS8z0lO8h84GuwzDYfAWp+1p28J3io+Rz0YPRNV8wfVSspIaqwqfsnCGnd/IJPNvbu8cXraJdb0rOPz810xe+imy/TZ2jc2KrMrES4Ju+auJNo10XVqGSalL+eGOzhD7BVW2IbDBTyTlBpBVnCqdSm5TCgDXa45TW3MZLDqq6rMo7tqVoqiPqSx8iqt1j0HbJs+3A5u997C6Io0W7U0e9oPnpceFPRrAsVmciCokseJlThXeyzdnVyBp24FEjjqpCq2iW2g1vf2K3N9J6YOeCDr4lPJDTAkdg85g6fOb6PuSipGJq+ll7UJIowhUTPIwCyVR7Az2qG9Sq492u5e1zVJVX4EkQWz1owy7mknnom/cx5GAzpzJmkrySTsdl87nUb8ssLp2m8uku/j6RAqeOk9eDCjgV+9vXSf7pvFyYSH5DueL2RWQkNfXqegC+EnzMt2rCsAHHpri8+/tflUyC9+ahyVsHM2+zVzqdpGygLX0ix0usrPbTGCs4ffIL/kw2IbKEsjEyOMMbRwvwIFD48BY67YhOGEAa8dXnRaOFG/lzZHTaO7cwM1ul5hXdp/bc0TbG/DcM4Qxf5rx6OdLk7bdE0dP4oEaT7LNkJCfgJd/Cyd6n3MCW7IMe3uuYmsEHLV/ytRyOztsIoAvSdAp9DB/J+c6802uagJdmAheB1oP8H78eQIialnSauGT+qmuQb/uAqN95uGf/CoHM9Lp3ms0oRd6w/G5sDEIjs3mcdUpuuaL8SpebRL5ELThoPBAlmTURjWdz/TClhuLp7JdRAE46DeOlQ1qzJKVFqsKkxQogAergc76h6lPqydUAc/4wxBtO/VnzHReK3yW645gUIsdtKZCODLVaS3YnhXMbUH3gohfmLW2nrSTv9Dp8PJ/WDgb7J5OEEi6zcoa70ROWIOdQSpuPy7JPBn/KxpjPKUWuNjYUwTZ+WdsPbvot46Ns1X25Ux9OldN8FINXLX1gKiJ0PE1GL4bY+4ponJ+psQCZ/RaUAXAuAvQZwXeLRe4wxFs1Glz0FMnzp2UKYAoR7lqBJUhgSBthFCd9l4OqU9gV2icm/GkileRMQk76uYcAG7VX6XGZx8jBsNJu/lffk9+/M82r54efrRafAlXwtpwGKyqgvyfRV6EW0vYkLeE2gtHiKt5VNTzqXkimAmor7/MpTg7GcWfc8leiVI6DXsGCGAsT/TJtc2wVyc2vttqfxX5daLugMZb+BrLnfmM0tRtURQD7BtMb/9vWTy2D11i/hAkCLsZzjwOu/she0WQWduPakU5J1NHcD5+rmC/npkPV94h/VoAd4bku7qEpBT2jpETaOsvEhIWoNUOinb5RgFW2gex8qGVrL5vNYd1jvE872e4/BplwSN5tw7MgD13EcbG/lgD+gt75R5fgCaEV6vV7GoV+YnfTI5yqi0kCdbm3sP7Dqb9OsM8t/uGe9zgi56LuMMTLMomcs0W6L4IQgfh2ZrFg75CqXPDKLG6PgQ8hYU0xmp6WK+QpgarohWL3MIej2sif1PoEMj9npd9brpuZL/tm9GE8qihG5/oCglXgp/S5M5yxY5KgmMGeOvgbq5Xj2h3LQuTUpcxsp1SP9Y3TlhhN95CQw022Uizx3VqfQ+QazhDcWOx69qyOzNbqVBA3x+g01toG8+xIhQ8O2UjjX8Bf+/73errmPdkfmqCisBN5IZ9RoW+SIAeI/bCgN+Y3pLCDccY1U0D1vqvYJZRBMSVPmwOeISFDkfNv2pSye1rh/g5zrYCaNbeoMnjCnqLTpAOphZzyfdZljfCNZOS3TnzeftqqwAZizaAX0dWBb7D0y48j0JlTzGGZC6DjQF00LjUA9hlYdE5Yi+kPo0xbi7euVBlUiDbHOQgQw14J0FAd1acX8F79Yns9PiLqKIWjrZMhivvwM5ucPZxHvbSEegY55Y3wn7N51RU1/HRhx+y9scOFO4YRrcV4HEYwvXgYW1xq9N1OfezogmaPa9ywbuvUPMANnUkc8phXTP8duMlrhiedjtPIZlQyQbAzihPiGjd78iT9CL0Xk6f8n5Oy616qwo8XDa71F8h5eaj9LzQk5EHRmKTlIKQlzQPNMF0yVlATaJgyh/QweaihY4gkASSRFLJQuKqnuXuUMc37xUviBaymjJTR/LtdZhUNaK/tTXsvqFIpx/kwbiDdM8Xa9G/4vOEgs1R9B49uLskwAkgyMhQtlusObbFYVEEobN602qHh8s9MCW/IoL0O7vDrt48UvQeoz0hWgnvJR5D2+BuDd4WgMwJ+Z48swzhI2DcOVpD7mZq+hLWuQveBbC1LRaDHEu/vav4qgFuxMHTARuERd7Yk+CTxIMBBSQ6xtUv66Gkb6bYX064AsmPkKASH0XPnI3M19wt5hlHMfqP4OdGNXU2UGXDPs2LIp9zaz605JGf+CyzykH2zGZmxgLC2SPyfOlKoPEmg3WH6eIAmTJ8StFYCpxB+r1bOqD7fSlRy6Hq0nYGrRlJbsankPEmjaGPobf4UtacSqtNjMe356BeeXEJ7x3ayZO95xNq3ARlO+Hqu1B9lByTJ01tawK7JOb10MGQ8hiKog08FFhFZO0cvgiG78M/EoHK8l3Q5UM4MYdnfY8xOuoQZmWdWCdsjoD13tBwlbib71AULy7d59pA6ukuFPtAvSWJR86+zlveb7G9l8TV5m0QNUk4LBRtYOvkMCcIaZesyMYyAUZfeE60fVO88/1uV/a1AQsV0gXCFZDf6TohjUIBDPB11lxy/b8X59oVSLJCDFyBvSDtWWqtrgWPWy5ySUIO6EqxYw0vA5+F6oVNO4510r9JEdCsvUFZwDr0miIsdqgPGSXSgsTPZU/efGgP9t38XJyo8sN+GzggS7LIdXr+WeR2Af4pXrA9OQtl00XIeF0o5WrPctjnEq+pOjuuLVHt96hQ2IUOQ2pzXAFWhcGkmD1YOr4PJ+4h8MJQNDZvMkoXkF66ULyT7P5ObcUmie/jH5V9DpDDpsFPE+A6wTuBT6ujOWMU4Ol5vY+w+Lz5hRj/eizh77xXSCl/i+GFvzFbe0L0AZsJen/rBuZN8YJdkYjcyA5FY6Qhi16Obyrf6IW36RIcvgOOTKVnyFo+GTmcBP/LrveQJEEU84xldsYHzPWBlPwVNOo7CYJs2FBQehJkvUmq4xW9JLBLSkEQcuwjT7b8ztEMQaid62NHmfeNcFW6tRjU/vzZHM71175iz/R44m2bBJHw0uuC2NKcQ5D1KBI2rhqh1hTs3C9R+AdU7ifEIweVxZ++FyDogMQRq5VdBgN3AyOBu4HDFjhmgx9W5REbFcWxBQtY2HCLHxptvFIOg4/Bi3fO5cX3XwXA6tuTfsWwqQV6ROxm8ywP1KW/QEBX8Mugn3UOoxJWYXc8S35zPCW6HlC0iSbS2VrVwdkWiWobXYJ3OP9+7+yP/B6YSp33MSwOJ5u2fHZWm5U6fZ0THG0Ds2/Pb2ezif/a/xsdXkDpe5bxF1tIq3mZ3d39+NxLQX5tlkuV77iP7Z8p+2rPEh9STehLf3Jyend6KO7EQ+kiNrV3heighnkhp51WrQAB1kqi2mFrwcocQYyrvwQx0/G3fEq3vN/wNCbgq+uGp8rT+fz/kErktm+qbU9ul6zsbloNA9eIg3mrmOSjv+1c13egvG3sb583MMY3hicc66tYnwKaPC9jtDvIyn93gOsL2OE5iNJ27pzOOjBUQtYyOjvmwH9HnmxfflHB5DHp3KgZxMuV3lyTSlzvLMmg9MQua2g1BWFuEWQmjdyOVNfhJV44a+Z6UwqHOqeyv1MH7Cof8d1ICpheyfKzK5zkt4+rHCRCc5NYV5TtpDD2Cs/6Q2FrFLX6WJG6B6Bok9s4sr4ujINNXwr1u08SslKNl6oepSNfpIQkUplIMnbvJGoMEdT5HKHG8xjN2puU+E0Q40D0NCTJ5mxnb306jxsKsPdbDbEzQeWHJEFyxRt46zs67i6JufXCC9BaRJNU6HyuO0o8oPun7drF9cyrPH/hiSv3YvJqJwTo+gmbTTdYXB5HJzWcigEKRAxbkmBiynI2RoBagv2aD9muucovlz8hSL4INxbRUn+D9w6/B8ANE6S31rMo47xzwLccPIba2IK1vQsZ/wH7/reU/4B9/yn/K4rU+V2U444hVR8BTSj4pjMmaiERAacoCl5JWcDGf8o28NP1oEfuBkbaP3G7nq/pIn4ylFhgeQhMMy0Vgc5x5yH9RaKt60gLEiyHfa0aaM4WDMSSLXDxBT6W9xGmcNnG+DRsFgtOUz14xTGvKp1fm6HSEMiO7MdR5H0DZx7F6plCTl0vVjQKm5Rx+lWQuVSoRGpOoZUb+K7PW3wbCsZkmKk6CQoPkd8nuB99NRJaCY4bYFFlpCtAD6jbTcQ1hlCyLfcJdk7hWlinJaLiR26e6YimwIu0vI14NO0XzKE5VggdwgafcU4bC5DQUgm3loCkgqZMnvZuJEEpggiLb91LS8B0SBRAQHvmSKgCPg0Gr9p29n5AdN09JFe8gp80Rdyz2yJnW9UbIqiUq+ikhtqWK66Tqg7za1wms1oewivxDZYNUoiNgaPEZC3iFcdeotYGZSSKYM/BsXBwPBT8Sr3Vg8P6dsnG1YEigOKfQWLrZe7zhXwLLC7qxRHjj4K9qw2hyRLLktJOnDbAXd4ws3YgJNwD/X9BCu7D+wUDuW4Si7Ofrq2Ayfnwhwxbowit/Zz+HiIwF1j6BtcVyyFmqvO52ybuI0YD7xZ2xeblUlphM5OmtvDYiscY9tc1Bil/dqtHtc1IjIeN3KAaJmXfycbMZ52bXQD0JRQkl/F+EESGr6dvwrOgr3TW9eXaHiwuj6PWKvL+NXp3cMsBAOBtEIuRUZ4QtKcbFG3Ap/AX8hKgg1JJp+Jl9LI+47qnQsPaql3sT5yBRdnIaQMc5y7wcERhPGOwKXqTXP46NllPqybHfRFgbuIxj1KGXejFn9vv4KOj29wBh7Sn2Gk7jQI79/hCmGep87g5YAh3rW/h/KXfSSv9CIC9Ji+hPPBOQJLApKijfenS/XVIuB8szShlIxIS3sZUJBRIdgscukNYzQFGv1CaHJ6CswrHiICjo2QrEtGVhNH1Slc89B5ui/D27dxWwhQd3f6+3uiCIg4Ev8rcR95liCwxVqN2s/sdKqsZopB5fYaN57vtIsx2k92R0LEojtAKJftbvCD1aUE2UHqyraUPh/XCAspD2YzWWiRsUFoL4c9kBnuKBX9YwyS6Fn/rtlk3txaR4HWGlvRcNt2xnTq7n9szv+A5l58jhILu0VNDeCT5c7fjbcGOyrBKrDFl5Js1ToBTliFXF+y04tmht1Ahp4rzFJBX352XSmI578DaXq6B5gSh9L0ljWBgMVw3CVvK46YM102D+/Na9UTWaDfS6pHJRlsm5X6jxXhtqoOidXTwuElQ8zAyipayJFgjgkhqf9AE4lN9iI88PBj410S2L3qE3beWur3TuAl/Etp8CUPECmr7hgtILmoS5K5EaddxtFWN2Q4fBcE4rysi79Sx2VC0gcutHZ0WjgCt3v2EPUtbonrJglUWG1UJGWpOCxUboLBUM67j1zSO88bzkQbUhr/cnmu54U3mV8EID2hIbRbA0xpJ/NdwzbmhfD0ANvf8Dtlc6zrZaiRKukKyQstuHXyY/aEzF40kwbT0z9kdKTYH470s4ppb45zt2D4oJkuSmGu6fQIRYwiwpeFtSEcJaNpypQZ2h9hZ+JT/xFTHvm6AFn6K+kgEz8p2wnWhgpWyYXUzPLMjE33ww0JVn/I4+KYRbCyjmwaeUnRj18C3CTLtFpY3f6VA9neYL08i1bOJFk94ZFI4k1Sqfwr4TQFW/5N//2flFw8POve/nwTvfAZoYZYPjA/MgNEnhHVzYA8OlT/H+YZEcsI/JTn1EdEWNz6FrOXYw8bw6+WPqPTbTlbyLFo5K77FvJ8hdyX2IdtY1iCUGDlmaLanij509nGoPkaf3AVsjBA50eZvK+UvH7tglvf4khzdBFZd+pS1gR+yv2s0BbprwurcZgSblUb/rRzpnE6t70FqvU5Ah1fES90SFoEtFXORrSL45ykZReC33ypQejjGaIlAGXpqQNVe8WWo4RE/A6HJJeQn5rtsEe+4CVOKnBZCga3d6dFwP4HWMGHVpPQRajlNIJcrv+G4QXzT5xofRvaMonPnzsI+tB3w3UXTLIB7R7C92pjO25eeZ7eDDKyWXeNXQ/xDeOfAtVZfJp63MbPukuuZQwbymucnLGuAssB17Orhy6E6h8PAjc8gqB9vNwom+3uB0NQ5U+QyaisKNZlSAEUWSCqAN3JGus1T5rRn6VAo2P/CmqpdJ1J4MO+vMl6q8AVgyPVL/D17r7Cz/rsD02wheCtdqvG3i/qy+ORit34YU/Og8/+3nz9VhkLm+4kcu7NjD/BQ8O9u55kTXuFy1tsAvNTzFeIaXnM7PrfTXLqbRMD0cT9Y6HNeqOgkARJJ7eexNpu7K+9B5jJGJazkrzkSvt2HcSSjK1kN1wUpwDMam1LMHXas2JGx4Cna3qGAOmTswzUT+Oi68GTVN2SozI78YiEAdPNsdr0vsgj0hA4Dvw6oFWoCDT2JKY4lIuo6SYo3hKLw8utQsZfB9bv5uB2fSJYlZ15QBm+EqaWE1LrWkp//erdTMbGg9hY/N+t5qRx6HYW6z+DmNx+0uxbsu/YZf5UMB+BI0SEsNhExsyv9WNssOdQ+/0hQG6nsyJdTgzCpqplQBidj1ogchf9EHfVmeRoMb6eEMjfSXyMmx+oOt4jRVIGhGi6/AeV7afDvxeomqLLCIA9I8TkmlN5zbbQqOpIefIIITSN3lcP+1jQBfJnqoeEqH1Q/z6vq79vVt6ODtRaBf1cWn/qNNlLKCZ23cDK5vgAabyBhRuMIJ8z3CKSXps6Z0oD4u5EsxzHHOfpcG4Cu8hVzaEA3WlRB6G1i7Jt5ZTqG8Htc72zRcYd8jLGeYi7yDDgkALP6KyhsjWy99Tx3ubuniZy4ac9hVfih0nXGilA4ndZ3df0mdgbD88ex3xFHlSQIvXYnXHtfWILtG8rLHin0ydpBQMtAHoofJuaZGqGMkG06OqmFGrH/9Yt0DRwlVH2dP4DgAeh9MzhmgGordMuM55b8Bmzwga0xICk43nE9Gx348ZbBy/j7x6nO/vdp4y1+azbyaZUI0isWw6qfhDLQ6N0HlWzAEvY95ZFivTtW1eyc970QAUMPVTPjkn/Az3JaKIABZA0PlY/gZGMEvbP/ZId2osjJ3pInLAeHbGPlzbXIdg07mzX81TJCkJrSX4AwF2nwgA4WjhxKr6AvxHwBULAGiyaUHMfaLqniZfyl6wL02BxOZ7+NnGyMpNARWJ4jfQCnH4KUx8A7mROld6EqewyAq7GPI+sLxRyGRGuP9Zyu6dmugd0jiLIMVslItmILOrvIqWv27AKD1lMzqIRtXOVG2LsApKokkjx2CZvGkm0QNZkSi4vE8k2ntXQrdazT7Xa4uRh7CozxBKUEzwTpRYAW8JHyGBN2xqkQBrjPeAIOT4bcn0j32kR5wCYuJM0GBGGmsPNnkPIoF3wH8phtHWeTJ6OyBKKy+Ym68k6E4bv4pfsjDL5+EV9dV8cby+J5c3/Ao50TTKAC0jUGJLsOynYJclvVYWwJD1NaN4DPg2Fh/Dlh+dblfRh1kKBrb2FPEbX4cg2syp7jzOlniLiPJ//O4kz262jNYv92O4DZVroW/MzToRuJ84tzP+4IeA++cZ5+4UNFfewd4noPhA3nvcVdwCdVWDcDxUEj+KG1juKg1YxL/p54vWN8DuwFSk+es19kkKOp0tQw1guxl7e0gK6UURUr+CpI42gbGxEty8G/K/T/laym0XyYNYlfEoexvZdESWueaF9DFbK1iZkZHzM8ejtZRfP58so5Qf4p2ggb/Djn9TbTHWPMBo+hLNI+5eiborQHbkbaeqEo+hXSXxTqZM9oXqnuwkZJh8nHJnLVtxbDjYVQtAHLtQUM1A+h3us8I0phc8VU6PwuDN4MvZZzuO8unq8vwaxv4MJ22G6x/8s80ycArR2O2mz/AAYescBxO6xdtIIlixe7PXMRDewtnIZkLBeEmJQnuK76hCtVw5mh/onVQ4/SW62mS9AOuPIWKqmZXTWuvWyI3I6gnfIE25sMNCiyMSuanGBf2/1y63MJ/SKUbbEix5ndkZdP5VD26a0t6NSFtNgr0Us16NQF6ExiPCZzqXNNZbW6gMLQa6/BAQH4KNqUfQ6wT9Vub0vxJpRqC3nN6+kfrGDFxDS6B292Hn4gfSENiSKG1Uv254mwo9B0w3l8SulLrAh1Xa7COhzGXxZkRSBNOZroursZcTWPITcuEukZD7Ttl9wVeKrbbDwj6maQXPYmGYlfccUu1qAcvRMixjCn2N2DpD0oebtVZ3vlsYfKg44l/nQrFDlYAUKlYkE6iZsL0ZP/IUbhJFIcGg/eiWy1xoi/e8Ixm/W/5z5kU3DPi20kdXE9D2s483vMF/euOYOWSr45+z3PnX+FD4NgrLba7Tp7vR7gQOdEABKUoGo6L1KgSBJXGytZ3yWAnIhPHHeQIMrh9FR/SeR3RMRYhzSd4afMtyCwhxAnxEwjOWcR34Q435h7QvqKMaq1mOCG71h7VyADvNr2OjJKQx5UHUE2lhPqUeZ8xiOdOvPAxR9hajEM2QySwjm+Ka1++EtxSP6dRGy5tQBZhuDmYQyoe5yNgRFM9TKJPbfSGyr3c7evnlSV67nal165n3I9USj9ii1wsLYTNrWjMzbehCvvcKB5AfnKQtQSRChwrSllSAk8zzQvnJbVFrsYvyJUp+DSq3gaSt3uN0ArHAtkxzdmCwzGpPGhJSj+v1T2/bvy/yr4J8synTt3Rv53vqf/p+7/f+Wu/yn/Kf/TYmmFkw9g7/Yp9FwMEWN58MZnfBIynyvxj3Al7mG3n0uSSIjb6HkRP113kqRRbsejGr7nI0cgIFUNQfZSoUKqvwyekTzQ8gkTDbsBeKHSF0YeFHaak7IhdiYxUiMaSfg5PzdgBtH5d4ocO6Z6UHlzzBBIrhk2RDzNgpv3Y09+CsJHI0lwunQqj1UhclRINghw2PTcWATYSPIux0+G/Q3heHuNEIDbviFwfQEH4suZXCYs7iQkYbFy+mHYGkNA1mvO/EtlgeuosmaJZzoxF/wyqPXqzuq4W+T1Ps/dT63G1uBgYkmCtdI26c/yhsyuhwkzbxV5mhouQ905XvOrJ1UNZ43we8Ek7BoX0ChJMCwwjwmewv6qZxG0xM4FHPkUvPJYMTGZqelfiMWEJAuWjuNcf20ldwXUcDUO6kpdbBaC+/FbziryagZSpr3O/lYf8E5wHi7p/BmvCyIyI7J30dOrMzgs8QjshTVqCg9V6qmzwWwfG+Ee12DQehgjgMgdUU/RqVDkvFpa3Id82xzI/Qk2+KOULbQNkfVWqFNmCDZfQBc39mJm1NvsCXoKvOOdz1UYs4FvakVEO7J2DlGadNG3HLZsbSrJKyb40HAdmzbG9c76Ck7GFSNHCd1J/7B2ajEgst9SLnvpGP/zPoadehtfY4b7BKr0YVWDLycN0Dnyb+YnLgV9qbMtztZ25cWWQqqssEcH3/veKey8HMd/nx7Euz3eRm0OId8MNTH3gF9HzIG9+LQOqi125vd4hjc6RrlsOm1mQuQW/BwzypXqfpxWPe9kPtLjc06od1Dtu5u4vj15athAJFM7AM7cxCOaYsKeWMOuScU0qfPdAqUNhgZeLU5ni86GJgeOlExGef4hWCOhbT6O0eqF1a5y5rfIt3sI5UFLHkm2b9F6ZbrVIYE9BKt9ezrpfvvcDkmSXeQzaryBtWAdn04J5uKzC/ny2S/Jtvk7+61oR5mzvc+Sm5jLtMpYoiwupTEV+zga1sB4h/K0sxoe8B3kdq8M8pnkBXFVj5FS/iYz73mPwtIyhrz1Ni/5pDPNW8VjwXB6QCB/HTzI83cA2nC87JX000LZ61+z8uGVbG9OEPmIAFQ+/NA0gY0tUBz8I1/oi/jZa6vID5b+IpgbWBguAkIxNQ/xVKAJ5b4hwgYUOOXRnbHbUui8vDejs2fTVeNuTSJJErlJuVRElHMQA+fq2qlXqo7wRaCOVBVc6HGBtUk3mJHXud25sODIX2w4swul3cFKd4w7kgQVLUn8WB3hDBi11TGAXg7nhAGnks3NyskjnNOGJModopmjBrga/QFMzhXAhTYUtdyEpymBhKqnyW98GXxSoOYUGGvwajjHU2GCwWjwiURjDeb2Ikl25vjA6YQmQtWXRNC51zdke3/O1KJg6m0wrARWNI4S/adoHVx8yc0KJlUF4ZVfQkAPZ36yo/GjuZR4Dz1y1/K28m2xqa0TSl9Nyw4e6vYW0RNf4uVhX6Jt+c7tmQb7z2XY1UwabHBEp0RqrwK5/jEHvc4w3EPYcvgp9W55vDBUMaBlIPODq93qua0twr1z6asFJSLIBYCuCABf21UujXyaJcHwZwQMUAilNIYqyF7B4oEh9Ag+y4dBUJ9RhKzLg6ZMMDdS3/8S79eIOjHZod7qK6z9cn8EoDJ9I4NunGXQjXOChS8hLGdaCsBqYlj5j6wLB6NdoskYjEURLNRoAK1FTFAUEqOEM0Y4GvAhT76xkCEKBWO1WjcAfbus5AD/WknbVk4Dh0wmhgx7mE/7vMLL3oEkHX2OyoCXRP7DYX9B4gNIEujVhdyKfo0FHVeKQHbKE3DuSezBA1l/403sQG489Pc4DtNKYXolDPkT6cZC/owMp4dGBBo8lM1ijuy2ECLHURg2hWUN4AYeBPaA9GcpMQ7mw5YyLtkd7SjL4lsffxG5aDVHxk9zqt8luyxssMNHgTqAgg4XKCx6nA6lnxJWP4XeHnbY3UdYazn6wYireTxgvItzsRCgO+e6v76EL0NbGOcJe6LgeNIp8e9+6aANIbl8HXujIEHbym/Tw5jR8ROUJb/AWhVkCYv0FOtUAA7pYUfNCvBNxWQSarlH0lawyrHEiFIaBcu9VSjddLZQdpeNIMsxTgSaMuHgBCjfg6zQ0moHm2RnbNL3vNMjHRocVsSSjCSrnfo4TwkGKo4IotalV+DW5wyM3sHoS1VoMldzqmS6ACfbit2Olyy1y/HkrvxrC4p2VsPoIRmE+ruUQEgSDcYIGq3ibDs2B+jhIlNo5HZ+cPyj1ZOfrjtdCn6gc4EDlDn/PNxagjliOgE58HcrvFEDz5QPdrtOz/DepJV9gLc+nZGeEG5xBbCov8zroT5Ms78OwMpG+LC1p7A8OjwFgOHNf/KQGK7Y120jiackAYoUraNaF8/OmmSK5WrHM8tiXdCcQ7jpPNdjYWZQOXndEqnwvk/Mu+kvgEc4P+nnsLtVxlffmf6+1SQ2fSrGgqB+jvZxBRF7eehhrRIyvwQgwN7KG6b3MftpSYq5RrD8q8hf4tcRJJkU/XWmeYugxz0+EKIsggG/i/yeEePAMxKLg7DQ7wJU/XnmnyomjljgiA2+/q2GJYvdwdc2uzrp4ou0HJnV1szOgM/9Y8MJ1wxxO+ecKYlNOoFS24DIll1wYJRQR+X/xiwfr3bgwW2RkdDBdKlJZs1DP/LevL08FnJLgKfJj0FwfyoiZzDf4SY3rhR2lH0g2sLchKe9gM9GD2RG+GU2tcBIr0zI/ApO3g/nn/sn1oiOv6fkw4RLFDR2wSaLANGnVdFCiXP5Tai/TFLpdBo7lvNhEKyIrmOk6oxQxcy1w4DfGGubxsKOGwlVwE8RelT53wjFTkA36PsD22Jf5ZhBgCHnmmKwebjAA6wG7lTsZZo3dFPDn8PmwfZ02NkV/7JFZNb25+YN0Se8TUninJip0HMJSEr6epjp2TyZt2rhoL6fAEzWSLAlmmqLD0ZHFwuVZVT6HFHnZTsg5k4uFzxBaNN4tJZwztYLey2qhVos9lIClxJbecgPugbcwMtWLBSand+G4L7O+dQK5Bg1GOV2AVvfdDonzsCrVoBw01d34Mvfbv3L/nfcDisXfsuSxYuRZegWvo+dIx5laOR+AB72d41IdklFz4idDIjZyN2bqsn3fk98E4M3Q8rjPOixhdFXyohqHsOoxNXiff5Mgq1RYKwhz+RJcvnr1B9tpNiyXwSy4+8GUy2Ej+aXZj+yzJCsMeOtqhDf1KD1EDeHqpTnGVEKanMIIz2UXJI+F4p+nxSMdn83h51QCoQSAyB0EEvPb6C8TCiSa30OYQvoA3c1wMxWrJEzCGwZTEjjeFTmIAKleLe+qlDA6dSxVCjP0GATOXV9mnfArh7gEel231f9PZgeOF64K2Qtg6L1KHH5ePbyK8TTfFMAPXabyO8IGFvSMdkhJSsSeoj+Fs4+Pu22lC4a8NKn8Zh8jl5BsSK9xeU36Ou35B8UPW22ggaLgWaPG3iY4hl7qZZRpfuFPW3fn+DYTLqUPI+fvqsrn7MkixzSQGjdMef1VjVByrVeWAMGCrtzUx0E9sTe5wdO1KcxyAMG+FahtDeKoLC5GUtgTzY2i9FFd8iGqXYyqiMjIPYuWjNWuPoS7kCNeA4oDF7B9l4SZ1LHUWHOwk/r53Z8xLUcxl5oxNuQ7prj1P7QcJ3Ncbe4yxtCGsfSw/wcjNwvgIedPSktPcj5kBfIDV9EZm1f8j1egaklMO4s6EoZZi/iJc0qUsreZlE9DKqcC31WCKV4xT7kbotZly1iQY+GF2KVfGDoX5BwD+WG7nzToKDMsUfQWVrF2Lg5DMXFJxlzdTz3N9+gytdBrshcKiwpbSbMCn/RdhZ/skvupVHuCuu94KaYD9rUVMGNo9l67BiW7suh9C/n/reNLLOo2zJeiNQIcB2g5zJaIyfxbDXoHeOQjCzcnGKmgX8Gf2b+yU3Nz3Ae+sv8S6CvAngN2M6//k1fYLvZwmuvvEJj2RVeDYBeGlgd8Qj3V6iwhwwBWQ3aMLaZ+vNDyDMcMX9Jt8BBfD16FE91vgMy3uCU9KtbXsazek8WXr4k/vCIEPbkgF0yY7W723i29X8bJsdv3JV9f5au4ECXeParXuSy38cc6JLAHzc/gsbrcP5Z+ltmAA6wz9E/Vc3ZggCCC+zzNCYR1DSCCG2iqwIS52EfspOWMj3lOY38fv0JynSu/WmNPprTBrDbFZhO57Es+7zI0+oo1wOmsc7FP8Js9xfxO00g/JnMtKCpbvXdNoXKsmutoLIEMPj6RV4b+Jbb7/z03Ymov5PlIfBzxHmhTm+4BqYG/A2TCWweiqPC3JTNt9ujSrf9/ev4PHKaUtDbBTHlSdX9sH8E9PgcfNJ4tGkzndrwULvkirvE3Anxd2NXOjYPPhDaaxITlf+cPAnt3IdmWQkP9qF7+B5+T7xONw30rf+Sz8Z8JlIP7elLnP03x/NaeSsQhnvUuC7UWkx3n2t4q0Rlvx0Evif6gE3Emzx0BcSoXHuhVTFrBAHDO0ko2IGdugB+duJ1VpCUED4S/Drh3XyTHo4h/63IAiLUZ8T522JRm4vZWNqP67Q59EhoipbDvqF4nBrF9+PdLZT/lR2rXbKilE2wq6ewT27Odu4VEltHcmdQOSN9oxzrfzvUnOK78AaGe4px+Z1gIxRtcl7bWxtAuGc60TVCYCG1dx1pyYUbn/B10G/4ynDBCHEFQPLDzuf68vQqFNkSrXbwkKC2fjppQSfJ0k+HsWdQOvL7gegnR6Lh93DQKEU9G8ZP5croFxxt5nr/fwfe3Y5t/b8K9gGYTKZ//6P/Q+U/YN9/yv+KYmu4iaLmsGCu7R0MR6ZSaYqh1bFAkm7fFAN5YV9wNuUOKvw3/8OxBu8xhCqFb/yoUljtuVgsGE/Pg5YCN5sMhc1DeMr3XAI+ydDlA+ZLcymyQHHwKmKDj2BV+IpAulcbEOV6HovcjBTcB2pO4lO/UTyr3XFcssG1jx0PdQmLHMjww18ytwIezOlNpcfdIpDpyNm3tehz6oy+vOQPO5IyxeI990fQleBRuJQnzdNRWQJIyHiEx3z6gLlBnN+SR1HIDFZ61rB24l+M8yygMug+tzoZ33qMlwOg3gY3dN60qrqIoEzReggfRXbyeW5ViwlBsstug7gsw2sJB1kUDAa7mGyc1lmAza5EZ1VR4XGKdYZpbDj4jFA2IAb/5/rdz4oweLsWQkNmui7sncDp6geoMISQbYb7i1NdLFXAGDKUM0axCH8s/hBzLB2EYmvMKYgYjaJ0G96ysAf6IwI6+W4UwaTqE2BpxaL0dnqwOxOz+3eG8FEEaXLY1OEgc31gvx6OBv3sTPZNzSlWpO92sgx9PEVuEiZcgb4/0eI1jGarWEHF+mYy2DZJ2EcdniT6VLvKk5DdJ0CVL5/XBXLi2e/Y9vZvvF8/h9uLpbGViJxjpOesJzXorPtBTSBPVqnY1AK/NcOYgnDBnmx3v/bl9o1cVm1fSprTsMp6rpugMeM9COqNJWQIr9VCmd1KnUVDnSkK5xTSksc32u95KQASKp5n0K2TZPgOcLtPo6USnTaXnhqYHlgF1nbWFNow7tX14TOpEb/O36KK/dg9iFp9gnu9MwmQBUBgs6sE89K/KzbPRFICz+LhdQ2zooE4/RSmpIkgJeW76GZ+glBtlXsdHZ4sgpWA3hbgdkhSqGCWDvr/DJKCJpoxq03Ifs0EKU3QzmZ2oflLtoTJxBXGMbY5mD6GP1wXsttRAo6843wSDA/5P+F2r0n2gywPEQq8uzp+QrBuA+Hh4bz51lsMGH0D6ZEutD4FxlEVWP20wkqp/gJGVSKhmWH82oxjqLmN6ey0MjGQG/EpebqronF7fA5DtvFxVbx4RKx4q+uhtYA21r4syZRHlGOVbUw678+7Ie4qt+6WInoF6vnu0RV80vsolxsGtmvk68zzNhKlhO4XuzN73WzSdWNc7agAb2MaIY1jCSGZ1iSYYP2CiooKvl72IWcPdcDy7QVnwvmV3ir8C350PJcAniSEY+l3Qd+4v/Ptmx5ZK1jSiQ/A9Ep+br3O1dgnaNZe50bLDJiUBTObIHwUik5vs8TvLQ7dG0jkLEj0v+B2LfYN597wNzmuh7cqPWmxx4ugs1WPSRmJ1iwCeReMUGoNFmBKx9cgYix/pT/CPeaxjnbB4evv6kPt7VtMNk9IfNCpiA1rFTkKljYIIPG0NMHtsZL4nlGxW7lghCnF3hA5Vtjv3dUAcXO4ZvOj0Qbv1sHoUy8ja3xdJ6v92WKdyQZjLWOUYTwW8yu0CCux71d8yLzPDxH5JEjLYNsOKxWDc2C2QFessh+HK/tQaBHq3zCpUsyfm8Pg7GNkG/zIDPiFi0ZYW++NXektArTb4lAa8ik3i4HznBFeq3xObNKKN0G/n/GtWcvRKb0ZkfwV/XvMwNR0VCi1/0yAK29zPWg8C+vhJ3MWDx39gRZtfxh3TgSWO7/DXOsU9uugY9ESYmoe5LHHX6CwpIQhb77Jyz5pPOwn8WYY5PQYyOsz/P69da5aycKn+hEeoGTbzVc5eH0RHUuWiO+p4aoAdg3VPNstmqF9xyABcwsjIOMNSHpIBPC8YikM+Y5a30Osa4Frxu7iBtpQEdipOUmQIoduGiiMh4kB98HZx4Qq3zMaUh5nc1VfjMpquqa+Qz/9GDj7pOhTsivoIb4D11gg+SSzo3QIZZa2viaL8WvEXrirDqOneI6EqqfpnbuVMNUDEDxAtKUjEKs1R6LPf5+fLy0En3bKGO8kJpUE8XszXDbChXb51QC8jBX01EC5Oost+Xdys74T9sA+rvfWlfFyykie8Rf/pFRI2Gw2MjMzARudA68wxhFn2KeLhtkmiJ3R7g6uecyLVmEBq69AZa6lpwY8JSs2uwKrXe1UFmNuZl5EJ+ZknSGgpT/BCpitWg21ZwQI1XcVPip/NJYQjhfexyfHNiF5tbOANtbwt+oAX4bAfT7Qx7fMXWHXcJHH/GCgBzwXmUO0b3tLUDvRvreIdAxTdskqztUEwVw7G9V26g3uDO7bFRVKmw+xNQ8TV/OI+Mf81VD6N55afwadtZNatITLJjhjcPc19D0/jid7P4odO/EF8JdvuzG98hCcf5ogTS4gyFzZ6jGCKFb6JwC9Wg8zQxtAStlbVNUOc507aBPXa0dxb2EMl4ztnrn+CvyVQpLOtfaeH57PqoRfhfou+1u4/jF3BD3HxPNWuuf/xo6sJykJfE8Ar9oQmGPlZOUbxFc+TcfiLxjcKhTeXHwZzj0DFQd4sedE5F7lfOxVzrt10wWZofEGNGfzR+Rz9CwS+et+DYck7TkBBvpnCFJeUxZRto7QDOe3w9+WfxcktfHaK69QUVGBLMNLA+awLF6Apn204NlwUfQBcxFliVbeDoQhHpCscmdMb9ZP4gVHXKuHBozqdgSN0j95x2cXM+qF+nZuQBnk/OB2vkJWkB9TzK91c7lc/TAYKoT1fONVEgMS6VT7Fnd4wdYICPfIgvU+wqpdUvJx7nD+kE8SIMNJvSMIGjEWOr3JdL8/mNLOSav9OILNTHno15xLngY49lthw4UTSuQ4dB6i5mTg5VNfkKt+1e2ZL0gPsLIJ/GW419+M3HxN5G1UOGzO2gg/gAK7Y450FJUvF8PPsbS0IxVWWFswTSj3QOTYA7wNYn2rtvkL9r++Aox1eNmy+XxMf2Y4rIolSXYF2r0TiVPr8JagY9GXLLc9iGQziNyfZ58A7NysHUxJ4K+cS5rO59WdRQ4vxxhk8BVB4GwTLBt6N1HNS9zeOf3qc9QkijVLhNKKkhZnygW8YpENpXjIJmiG7ftusd1s+S/7319mO6+98go1NRVUtiTwQ2U8t0wwzvgjHcftds6BRjmSYfG/Ma/7K+gsvtgkT/FdhA4BSWaAz3sk+F/CYlMzc2OLAEv82wLedvaEdeVQ5zR07d03biwSe5dhO3mtNowSC0Tkw56a18T8FTsDArrQIbgD82sbGXE1j9cH3UlX+6tC0T/6KHXmdK6OepIPAsUlDZKDRLE9HawGxzzWFsB1qIn3DoRTD+JxYTajo/bRN3sHYy/X0E01y61+lEp3pwEAQ+A4iJyI/8nO/N1lCwAzQt4j59IWTrZ+AN2/EKB/9VF6aFznPntNkFQ5NkMMusmPsaN2JZbMVQBUWLQiDyFQrRjJaxdf4KoRNJZQoqSeeI10tMXgzeyp+9rNMj9AhqjMj6F4M3EFK1gTDk4bTwkBGhwYBbpiqlPXApBS/iZd81eRoOkhCGrdFtHs28ntXZ17Olkrgtphw9iWuZX80KX0K4YhlycRpNsCf3eEvYMwJT7CjIq2O9vQKvRIjZfBWIPCXMHwDu+gSHmS4mDxzu3VQ5IEtnaklHM69/iKJIHK6o/K5iucUiRETqmhf4JVR1eNxICq17i78UWeDT0jHAysBjDVk5GzgB2RgkS9KGoC3+oyRBxhW6JwfZil51rLvShsYtxQy1qh3Ov7IwQPwB5/LwerxRql1arEoEoU9ny31xOOPbikEISs8FFctTaSqcwlIP0x+sTPAZW/6NfjzqOVDAzxEHP23ryHOKvZIBwdHMTjtr2HXbJisyvFuNbmNmCo4sPg60z3hu3SRQ61dhIkh9HHIfYubKHDWNogFDfvB0KfgHPCfvbKO9CcS0dzIekq8LoG89vZLd5eVgLD+ddzWFvpCwxTq1n/+woWBou1Cji+n+CBkPEmlO/Ebi+l0es8VbabqHXXXBeIvIOO9gW8n+QiyUrIVBkzhKtDawHDlY0A2CQz1tuUfW0uMlbM2O04Cbptyj61QuM4bsTqAAQDZaMAwSPGE2bbyumU8eT6rXQChc2dPxLqzu0dyNCMJLzuTtJLP6Z/1n5Ghd7tenbfVJpvniD8r01EHMjmueZjlOpdiuE9JfMYWwY1Njut5gAqTD3EPOUoxtR3OJTnWIfYZVQKs0jHYDWCfydKLF5U+G9Fp26bb8RPFQqc+V9VlkD89N2I93cRWtrH0h6tgmqLLECrO25Aj8XcaXuGHnlrnLXdfmpO8k8lsm4WKou/uFY7ZZ/OrONo7qeMCsrGVxZKt5u2IRB7p/iBoZJYSzm+MiRWvMQd5137CHp9DfF3E6IQc3Fk7Rw6xHzFnfP+OXlysBIGywj3oTnpSPpCAj1KGe5bR7ACnHtezxjotogaaQgDY9bSt/v9pBbAR/XtJJMl2/g97rwzl+qGZjCkfeL8ppJOTWdpiOvnYcpmYfXbkivW8XfcIFMrxAzvB0K/0O3CyWRbHDRncav3H/QX2DDdPVtosUYIl5a052kKeoB3M2c4be0lu4w1fCr0XIbdO53M2l60Ly+pMwV56OxTSDY9aksIvrquNHqd45TyJdcPo6cKFzOvM5xRXGPMlusU+r4jSFEzW6DXN8wqDWFnq1gzvB1scpIuARi4hmshu7FLNl4OgFOD30GjuyyOhQ5z/uzzf+QpO/qX5JyL/GV41usgQ+LW0moNE2R/lSs2kKIGhQSHdc6TsZeVEZF9BLWu3u3a/7Ht/O+Vtr2trc2b+P/H5T9g33/K/45iErtju2eMYK5pgghTNeHlHFjcARPBsBBMmpKgXyiUDrtdzug9mAt1HdzyHRHiYEPnfEeaXEAHNUTV3s3vmtlw60sRIKg6CvoKt8Dy5DPPk9ejUQTSJQlaCrgac4IFQdBHAwt6iAA93onYFd70jdrGmgg7cUrYqp0GlYKdyYSrDsWY+CwNqjIsGMTkaKwFdQCHK19EWTeFnvXzSFdpRPBiyJ8w5iStPbdw/ubHhDVMxmCHBpuvWJD2WAw+qQTqrvFXqIIBJm/yWlVYVe1mS6C74RrTvYTaa3pWX5o1/QXrRBMCmlCMnj1ossk87Qe/DXgZlanIea4kwUe5Y3i2Wgwq3hLIdoszcLxjx1h6vJLPJ99uJP/AVgbfXOZUckgSnCi6iz+uv8y+k/sYn+EOLuRqNpIX1ma35D6TtG1CbJKFrJq+5CsfhaA+ENwXgvpgTXmSX8JhmAdML0ME+fN+hhNzwFCFr7mWDmpxfF3nzYxWT4JLr0Knt1HKJlI8mghq20e1BxP0FYwMLCDasa76ISYPdvcVm+aEe1DYm0k0dke2abD77yfcvkfkQeq5BMzNKBzbrJnecC7OjLLpvOvaaj/+bLyfXTq4qMjhsOWy2ztTf5mk4O3oxnTi+QWf0DtqO7cXPUI1V2qB/aZa54ZHlmFY1Hb2REGKSgSjBtfvdLLiJAneP7yD36+/hlUhfB+cvvPtlD8PqD9ndHGoK2efOpCd1r6c0IvF+5ikH+jZNFYk+wYoXI+hoT+ZUW/xYg2Enx+B5NkuyCWrKMSXSiscioaX4o+7LYDV1Yf5NBgy1NBPC97KRqTE+2HCJZSGPBaP7UNi1xmcSZ1AgmE6i4PMIpdI1GSOa/dT1pJA+3JF5wAaPSIp1rkztdzGkdg7GXUxiFFvv8Xbu6awJ26z2w9uyUmcMcKqB1fx0+ADnPVoB6BHjGZoTSh/O5zZvm2Ev1tfdLvXdnk0j1YB2Lm78zuEtLrs1wI8yvnGJ8qpyvGw1ovFbtcFBPgMZPTlCjoWC2bph+Fn4bxgXWHR80bgGp5yxN57aKCH7ZRg8B8cB9GT2d8iFtbVfrtYXjwI+9RSJ8CkNVbRrUMxJcnFVOUFc9XgDlDOMxzmkas9mb9iPv71/u4Wbwn3M9kwkGN60Bg1mEOaeSnKZaPSljNyTMK32O02Duphxd8NxEVHc+3bBXzRcoutrWZW1MDQo2qeeMXMTz+IQE13+1+YU8S3mmeGYmt7P5UD/B25lHmO9eobAdA3Z65gRmYug7KdXLFuoDD0W0qCfqFKaqfABHx94xkZ/wb+nhaeH/wQ3aJWuh2vaSmjSNrNOSN8VuuFjmgBxFQeINj0Nz9qRtKlZj6eEqixCpVvt0+gw8vcap5KQLFgG94yQ1XkqyLA7lBStgGzdqx4KuuFOsAnRbyW33xerIZCCxzWQ4Mc5fZcHawfMyNhI+MvtDLqiiOorFCD2g+iJ5MxvZa6oqdQWQKQbR5u56LyYVmjiiN6GGnpw/jw1SxZvJi46Ghu/SDa4s9mWFEDA46ZiUtIY8mXYmw2KmNZcvAU+y99R8cTz3BR9b5LxRtzF8NLYY/yLCcK7mPRmRVIHmGu+VWS8bGLXdwXwbA59gXADsP3gFcsHjUC4IyNXM+baZtRNO2FaIdFi18GhYFDWNUkcq5diX/YfXOh0KKXPLAAnXyKeaj7Kyh1N50Aev/Rt2h9aCRFj0PtxMN0H9XI6zNhiEJmzD9sXiUGyRKvPzmI5/scRyW1sCPnCVZaL2DtlcrZ0hVw6iHBkq3Yh1rRhFLZih3Y3uwjAqymBggfgSLnK1YMeBqtBM9UwznjQMj/XdhSOuaVMkt3NrbA901QZJkAnd4WgWcgIzSDuJYZGDTFfNTzQ0IsewVbdEdXunivZlvX9U5r1DEtT8CmUMhdhRQ2hPcuv8Tlto0rsrAj3xwOLQVE5N9N54SllPtvosnjKq0kiPmzcj+Ym5xjcHFTRzbdfBWrp4s0gsqHowYthRZhBfZsmYshDZCV8hqBeVBmhemWTexuDMHu00EEu+NmATbUygo8JZgtpzM1ZC605FJTU8OWzR8x7u0aUp4Cr6+h+mAWFVW1TlV1uMcNto+4l+f8xb1a/UbDrFZIvI+4xuOci4U5VV/yibSDyTl3CGULQFMmKdljGRN1AKXVl2orfGd+CfqtFgqmki2MbnAHLd2KQst+exRnjbA6HO6NuOzW/5RlO/g2FM4ZIK0ALtcMdR20W1k6pgMLw0Ug7FjH3jyxc77zsKjr/5qI0+hxifzQpVT57hL3nZwHQ7YgWRpJ8L+Er6rF0c7u6yS7sRytRyF6TeE/Ho+9k8ZB29hpca2Rk3w6wZRCYU0O6Ebk8sv+TNLKPuSzE39Qn/ClsI1W+7sp2cCxTvAIh46vUeg7m4wi+LUZtrXC4qoeIq+Zf1eInkZf2730i9pKcdBqLgX8QoMVqDsPOd/DiXvpYx9Fp+KlJFa+QF9fh31SUD/w7wQhA/jh6ipURyPpv38EBrQC1J1WAb2XY1N4oHcQz6aUQa5poFi//5UmVGXb0ygNueffKibaSluQ9MeVK5EkCPEsJlIlEM7BJVA8ZK+oa0lFtkmixgqxBfBWzT23XclV92vCIbbuV/FNzLFC5/c5r/mNrGahpn8ksMSpZATAVM9wtY5oJawvmEpm02iRH3H4boidRVT5Fr6NySM++xdu1QxAZw0BxxpZIRn4vjKZUwbIiYf+Hg7QyyMSwkczyXcDE9qBfRKSAM7yf4W1arb1e9957G1/LzGvBopgaGXIW/TctZ4N51ZzufheGhU9BBFxjQTHZnNV8SA/N0GB3p+71rdi7rxEWKnqSiD7O2ZLudxduYJnaz8je9BSPMrbzb+yklZ1T4rNSoos8HXm40JJNjGL5ri3eLTnU+wePRttWz9sLRBsoE1BWGU/frj4CftbJbZEwEzvnZD+vKjvjNf4O+5vxnsBkpXz5eNojXtZpE0YsgU6vEq1zz5uxLxARcAWyqRbcOwuOCnas7bDWjQ3Qtmjg9dLY7gpuQcBdX5d2NEKySrI7phDum2R62DNKbz3RTMl8hich0Gy4r/d/zasX0lRUwZvFXXijNGRx9Mn2alulyT449q7vL7/ILF+19HYykTQcFMw3FpCf+/3iQ7Zx/nEWZyJfFEoQydcEXVy/lmOxwmSx8MZnzPaYyYUrhdKx5CBsMGXsvgs3nYAdrIkCVDikrBoVZRsYWz0+9T6HOTLa89SJ/WDEkEYMOPL3sruziDq7ywWRCxJBQ1Xmdf5MULjRF+3yUYkS6NQ8xStQ1W2jgT/S66+eVtAUZZdpKkIhcipK6t9oO4Mypbr5OlFwN5qs3KlcgQXTG9DhxeEeqUpkwFegwhoGUDX/FWU5L1No+dY8IgS40mfb7mqewi7ZEEGklU2YZ0L6OQk9lUMotLajrzRVkIGUGPt5mbf5yuDf/EfUHMSr5YsB4DUDuxrvCFybMbfy69NBrIjFuBpTCKm9gFC1HHCocUvgxC1F9MiZhLYPIg++uHM1QYhGatgwG8wxwz6MnwvPMbYwAJxbSQMWgdRp+EKallFmk9P/Fv6sHxCJ0Yn/Yh1ai2Y6vHMep0Xun1IUtxyygLX4mvoSFJAkntdt3sn+bbcgm11cCp1FAc6pXClpp0bQFBvXjrRwIEbn5Dml8mI8IWCTKz0hMk5GDVheDmmEptCjw0rFK4TeTBDBoJCix0FdmwkKKGHqgIuPA/mRvBNocnUQJOnIF0sKM4gL3wNHJoEW6LpFriRQ532MVDb1ndlsVbu/Q32+PsJVlrxl2GAfyEPRK8VAfBB68GvI32a3uIbj66klX5McdBq8iwKGLpNqO9wzdMt2hv4Bu4H31TADgVrwNLCXT7F9NHA4gZ4vbqnWJ9jh0aX1b6PDO8EQU//i5CzAq59CKfu56GmTTziB3YjuFOB3Ms24N7/4nj7cp9ez66dx+hZBH80izjA54lnkJuuQc1JyP2Rp+TneNIPiu2nOVQo3IXW5y8HTSCelBAt+XG/11p+GXCBqTmHSPY7KgC33B/pqhCbXrtkxsuQTFDTMMLUglzSBvbZJDN27AS09CO4cTShXiIepXHYP1swYnNk5x5kvQRHpkD/n5nZ+ALVfrto0twAh/JfaaoXJO6mW9zv/wa98jbiq+/i1h/bSnPBMrzCqymIK8Csdv9m9ToHsIENpWzEQ9kINhfC2iG4A6ll74nr2hXEqXfBpiDRh4ds5dkaPeeSp3GgSyIHOiXTYCgAm9ktHtkGbt6ujmrV5FARsI2dOhhVEgITHATdg+OYlzLLSYS4XSk8PHYsPfLW0i3/d+IrnyLK4lpz6sw6nmj9lK2RMM0LLMAW+wLo+KpwBDM38H7I844cwTb3vO8tubA9nWkKERsKah6KpymOu2a4yJOv+HXg8QD4OByODYb734MnpgZA0y2UFX+yP/9Boi+OYJ8OdIoyanQ1Arzv+DKNyt6MS17Je4GQbYZKa7sKCRvOG2WdyHLMFTtbZUwprzmdaeqTnnJTWE4teFwA7JoQoWDXhDBStY9RnuKb6hu2w7nPovQvIn0iCbSlA3BXdjdW1pZBUC/ouRizV8ZteVgl7CGDIO0pzD1WsvTsCtoXTRuImf0NCvQEtPalR+56AE7wM/RcKlIWKTRIkhChXEiaSZnvfkHMaCvaYPa0elFkgb43T/D8vuvC2cVRDBYDpYZsdJo8Si1wrTkGm9JfHFR5w0wda1viOaCDMAXc64Nw03H0r0ifLJ5p+QAVIl/qw7Xp7Mx5DLvNKpzzbGaWjV8GQJYJppXBw1U4E+KqL50lKvMg2ta6f4i1ty//Af/+/7P8B+z7T/nfUQL7kBX/C3LBL0I9l/ggqzLmM7PNYcku/eMGwC4WFc2e1zjMB27HjN6DeX/PFc5X92W8J8Rbr4rBPnEehA7lVe03vBMoArDDY1aKBeWWKGGneWMh/e0FTstCcFe5ofSkyDaELDNEq2BE5BGkurMQ1Buj/1Bi/G4wx0d47eulatA6vKpKtiJLFkaHnWOSF7wXd57K6vcguJ/YjMXexT7P+VxInMV311/i4e01wkYsehIE98MaMZWS5nTs2NitgzfrnhWskYBuENSLuKo1hJ7uwbCPX+Lh1Q+gxOhWJ8uDH2GcI54i2WUUkkUwaPv/ArIS2dqISraglcFbpaN9jF+S4ER9Mvv1gkndnAy/vv+wW+B4S6uRFTWQfhhin4YlWxuc5+7Je5g1VxYR3DzSLfcg5Xv5vtM9DIk4RLoKPggrEjmtHCX26BiORCmxKlr4w1zEXl4VTNMj0yFnJYpsof4ptMCWVqi1ZEBQXxF49ohiZsseToYHMqTgV1I0FgxSpNgAqvyoNneh27mZLGuEgVro2vKVU41IzFTSTzzHWhFbY0NDMKQ9DdcXwuU3SM9M5EOPDIKbRvO777c81fotDPtb5B7Y4MsA7Y94GpJQS+AtA3Z3yt4U7WLee+893nvvPXxvY4LUZ/9E55b7OTj8LiY26djSzjsfAFMjayPszPMVA7wCu1ONJkkQqK2mt0bcd4AWejfsE8mYcU3cOgfLf7AWgs89LPJJlu9gRySkOZrHTSGoDWEDY9mpE5veUM9C/C0nXQvlonU84SMChzaE0lNWtPtg7TaCJAs+MrxQDesru7h9U9akh+lVBB01cDIGOgWfc37vdq8Efrq4iHMtgtJklywCuPJJBp8U6lQjMFjcA7hXFA5gov9q7O1M2dqehdIdUH8JSZIwaIV1ROOZLmxoSnX7qa3jbm5u/J1HVj5C6eUZmH1GuB1vH6z9uxW+bXJv51xFCjt0UBTyA5MvzqY4xGUT5qOu5d60PxnkAGqDdZcFEN1wFYVkpW/UNiK0ggTR27MS6tsAYxu9NZlOS985PjDL/oOwrCzfDdnfEWMbRmDzEIpCfuBo4ig3W5CAhrP8GQk1d2/n7LgXuGR8xu2Zz8c9xZWq3kSWR/JjYwIjQre6Dio9WTPzGClFXxJSHYKn2c6I8A9d9aGA8WlfMCJ9ARXyDSZthDV/XOCI1cru2+yrDllNHLHBJz9ms2TxYprlOP5oFvmIXqyBRU3tGJuaQC6bYp0KJl8ZvA25sKMznH8GDk0gXik2jrkRixgZPkbk1Gsrpnq0ljwu+v/OHaXwZ407c/qnkIeYXdmmRHXMNd0WQXMOMfpl3NnhM3yURqoS4a1Al/UG4SNZW7aZ3LredCj+jA7Fi/A2ZAqLT4dyqo3xeSnxPsIC3xL5l2xil9PkM47vGl0++9Jtk9wa61s8VKFAmTKfh7u8Cw3X4cKLcNqRGwHoVLyMNyp/42GvgH9YmLcB+OvL+jLu+zf5ZMEKjlit7DH+Eys7q5VPXn+dJYsXO7+9uJpH6VT8FWnefYRyYK4dBm9wjg2xNfOJqpsrfj/6CMw24XNhCut9euNpSOKFNtKsuUlsAveL72fYidd5sc7IkGKo9BolVHJz7ZB4H95q8T301sDTIZUiH2VbnsLmHBLlJnxlSPEpYVr6YpSGAtf7yrhU9cCd5fD8HVC41EbPx+bzoncHZmkiec43mmOD7XT5rDvPf/oXTKtAL8Vgx0ZEwBl2pGSTYNrnysPams/47Ol8WNfWQ2Q4NlM4BdjtSKYa4lQ22rZ2iepsETTeO0jkGokYy89NpTTZYEHWZG5ZHwN1kADo83+HUw/ySeovADyaOZTTPoeg1zdgNaCUTQSp9Hg62jbYelUEPi4859a/ALDLEDJIjPd/JuBbt4ag8PWcT76LkqBfRTt1eksAPVoRWroZ9ToHO6VREvQLNns75i+uOcDDGE+01d2uUCWrBPO+3a/duq9nNB0LfFlYD4OMw+notZYvv/yaO8aNo2bLQpY0FzjB5rTDt4iLjmLJIuGCYEVDhT7EmQtT1S5nnxQymFWXPqXYrKAiYDOlqoPt7hlDTdyXXK4ciR0bejtk2bsKcN3cDP6duWVP5lBGBvahElNGxiAZK9u9lA9fyb34uQlm5iWw+earbt+UlDiPR+u7csMEWWYwWNtZgEoym269yV9NLkQlryFXbMi3xdPD/DAesvuaTKNw2ctJElQEbOV67LNcjXtc1KXaH1Q+KKv3sXR8dwYE3uJoNOyI3eZ2nVUhLzJctxubbGC0JwTbct3aYfTuj9iuFLZSy0PgBcNTYh3VZk3uGUOTUQTjrHYVKv1NYYHXnEWS/1kWxmTR2dEESlkhCAvdPgG/CaSVfojGFM5BPfxc11kQVnK+g+pjxNt/I97/CrnhiyiOe5FzUhehSraZoXANEZ7XnY95vPQeocTp/B4kz6dFE84Lig8oCs2iKbGQSEWDIORtCYdLrxNkrqSLWswXf7ZCsz1aKLaas8AziuW6AArt1f9WMdG+3KfXs3XNGiQJXt13jNm3XPN921xvU0cwsiCQbxvb2s190J3suYcvHOzrX3Lupi78OccPZfDrQJn6bqqNYr3yaEkXGLjedXLjLb7zKWKqN+SHfUmt4rpgZEeMEW1VfYRBUWsoKrubDw8cJ791hFBbjNiHXpEovn8EGA1Ax9dFUNtq5J2qb3mnVvyzhzEOpawSealOCgLToeoeAIQ2TGCaXy3k/yLGbasRWYbI+hmUBq5hX5dIjtavFXlqAAJ7ij4B2LBjtHqKfH76MshaCgW/4XPrU3qqfbHWj+JI4Sxsnsmud7bbqdFdw+J1RVQTsgCofFOQ1H7U6qK5bhDfSZMqSwBTjmJVBvF9RTp7FVcY4gEp6lbXdX3T+am+E7dMgix4sng6No94odQOHQanH2Ja1ycwqcQaa3n0OqFKrToCzUL15qfrQasdFuqKyZIS3dq5Nf1l7quEBht8Wx1EvaIfnLgH/lCBpZULPsM5Z20g4IoXj1jN/HfKfXo9+7YJhUcbKdQmmbGbWwTZ6/hcFBgpa06lujWOr8b1IEb3lVCeABhrmFc2j4V+b2EO2cADSWvE+7aVmOlsbRLza2DkbwQpDgjw1GaC6OlgFfazZrtI9xCuzRKqvxufiv8tWs/06MWcTZnMMut5AjkngvRXP8RbVc2LVx537pmSpHMiyDj6CBgqGRW3At+wDQBYFI0oag44H8uc9ALf3XiagpDllAau+Ye9vkLhqg8/WeTUVeuvQcoT6DO+4dnM8aI/OFRGYh1gF2DjbDM9QrYy8NZxYmof4M7YHTR63yH23NkrIHsFQ/yfIsEnh6TmcVxMKRBrNMd1fPQdSSv9EC9DCkdtn3LtwsdQ8peY97G6BeeLLdA6qQA6vUNO12VE54NeU8DJ1BFY42bA3xlifBrwC7/d+pbMqLdo0d5yveiJu+HwHcSencuHqU8zIPMoD5qm83WHv1DqbkLDFbjyLtScZpRcSScN3FP/KrNbHsLo1V0QISZm4VO5k0MpfQnV9aFSrqbRsd4k8yuUDad49PR7bGgBv9aejM29zp9z/nSr7/bv9A+OGhLcinqDGt/96LQ5mG0mQRhwKJQlCYzKSjbWxPHKrT+h6ZbI3agrJqvjJwx1KG4e8IWHpSVw7QMxxmW8xR8Hn2U1UWRGv8mDvvC5/w7hCFF5SPSbfYNY3FUAU0ktY+hclAFl2wVhGglJsjupFu0JL7IMZzudYF04fNMIE7NHC9Ve7Vm4/CaVAc+z7spnVPpvY1T3B+hufMC9/znqwKiuYETfcSgPDBCW9lYjeMYxvuRB3m3LWiFJou+dfxaOz0Gb+RWXHZkSOhbCpoqZkPqkIGHEzmKz7wTWNYOkEVad/6o08l+Dge1LGNDU0sxFg4IqK3RQw5zQPOSmK4K0128VMjYcqep5/NZyBmzfx+na+bC9IyDx+eF99FDPItm7O93CDvBE2ggRH5lWzhrEPtkumUmqfZr+WQcZ4S8Ucap28R27ZKZT8TL6Ze+hX4xwfdAoHco+u0vZd0nTR8ShVP5OgrdVcq2VNDVHxf+Jm41ddq2Z2qrbWS6/SZRXA2uf/Yb8J35mc0qBACkdpXvQQRYESgTIENe/I2+l+Yt8uo7icW4aT3ZazITzRsZdbKTVHifyrzvIme2dGHTaXHpVzoK1ajTWInx1XbizLJPUsvfIjviYA/mu68oylAdsJjtSEGoKLUrwyxB9O2Qgx6vmo7GEMuJKAcOvZXM7UAgQ1jiBTsXLSDO7VM8KScECR7+7ahI5b6fI78MaWeRJrzzI3Z3vZrplA9G197vX17WPQB2IT+QUkspfw9eR79bXeoHwquW8+cxMxky7Qcb8ydifAobCShuc6rIS+q3CHjmx7QkBOBPwIt1XdHd77u/OLWNUqSBo+Ejt5kD/DNY1xjvTctxOXmtJe945jwy7msUA6TVB0DFWi/l53yA6W99mqAd0KIA1/x/2/jtKqqJr+8c/55zOk3Ng8jAw5JxzzoKoKGBCQUXMEUyIARQVDBhQUQwgIghKzhnJOQ8MM8PkHHqmc/f3j+rpgN4+3uu33t+7nvVaa7mkp06oU6dO1a59XfvaWa8J2zG8EwSmE2fK4hHNeE/7MnXL4KogGBnq9zO/1Q900go56tD6zl45Vns1meHHRD48d1ng6AC3V8Gtxbjc4FtjxKkDlVB3aTIKLJUemU+9BMHNnmDl1Uf8nqtxP+VCosDYEgISPXV5Zxey63wzqgIPsrwOpp181Ct37rCAtYJPaluywii+5+9jgRKxhjaSrD/s+BJhClhcsM8cTH5tC5rrVwkJ+TOzuaOlUC2odMIRawuu2jxYH7gJ8qYgH5L1X5T/Cez7F/z7v1P+Bfv+Lf87iiSjdZbhzHgcMqZDQCrry8Z5Q63/IrKvEeyDP7PPUrMHsnRcEzKKXuPLsCiG2lcKaaluX0PSBFbab2NprWDdvbjvAnT6RMgOxgwCUyGPu7aTrBLL2JDI82gbfGTfdNEckXfzS2kzfjVCnw17BOPk8kdIjnp+vfgCqiw4aQEVOhjtzid27RsUycp77T7jlXB4KgySXFfFgn9gElx4lx/SvycyeiM2lQ+7wlQEZQeR7DUUh/5GQUSjFrYsNh07h3ii6M6rxYQ9LL2U6Irl+BaLEkiNU0QjvhB/hQDHJdjRX2zk8lbR8nwovaIP814V3Lr7S1z6JM+5igLhxt6kF71ApQMmroaFn+37j47jfU6YN/dL5rz+Or+uFvJ9R3dEkXvEwJKFd1Nc7DZrFS0VDi1mF6Sp4eGIYqj1SmOpo/vQUPY4ocaunE96khsWoetO/hrI/xVXdH+eKBVAS3T1aNFnqZPFBlPRIKdNYe2lmZy4cTf3rinhmLxYROBlf4MkgdYehd6SwjjT7TSt+8QbqQZoHOGef39f6ZZtOz0LLr5PTchtnC/tS6NQiicqzi1dFKkU0u3KVn6sgxbZBpxhPrxeazUTwgd4fi5L8jLuASqihzK6UEQJ7TZBqdk/2genhTuDoJMWXgsHazO7p89kGdZmPUpYthh/X9TAV/HPCcMSsRDf0XIu45NEtGATFaiKt4K5BMVcTG+9YCCOD4TpEVl+t200wnOjP+N+6TsWaTZ5jZWOC5hQLuQe09TQLbAaHD761ZYK1uh382lAJp/UwO6KDH8nqiGB4xbYZ4IXi8IoMLbyVhoSWXPpec7Vi/dhl43YW7wAQ/Z52FQ3s+EOa1uJzfzVr2kZssGvTnLZYM8ouPg+1FxiWpMG9t/+K+fbWdhW4y/VFB0Qg0vXgdrINJoXvkHz0Lbeyvo87jM0kOmDJd5stPo6Ai+YA7H75BcoaWjG7T9eYGGlcJjXB7oBqMKNGKyneaXvOPqEi/c6+vp4kVcUQBXAoxU/MTNrCADf1MIy5SkYsEmw1o9O55fUdXTMFjJBPbWSABScwtg2hnbioQPJRC4fzOn4lmxomO3X5t69PiCs3VwAXCWRaBUfB7W1GrnuHDp1FSsnrGTZg0v4osA7fmUZpp67jRHl+VAH8jbYaLf9vXyVxcLMF17gojGDScVw3goxVbcQq/YBXsPaM6fyLjYLXxQzK+BguwNChjhQOA576Y2ew2NVVmi44fltPz+fdrnp6IKPs7EBCixeIxu83++0YNiTUkWIch1UepBV5AXOYurv2QTnvMOu3LswSfdDwXoPAOWSrdhUVSSWT6FL7STQJoik7u4Nou/atNIoifaWij4Lspylvil8Eg2uDBjiWOrXriuOEPZrDpEQfozxLd4XEmWXFojcLde+RnNhJonBFxjb/EMe6viEv6FtreGLoPW8Eg4VphJ27H2HdRbr378Lq5WZL7xAQ+U5Hur0OL0TV9IpbhOBDh+5wuqzzIqsJFMNA+K283jXB4WTFEBSsCRO41LhbUTVDmNFtaGxh6FCMMDN4cPYEzWPGqfIvWjTxPi1Y1zVGo5Fd6S/Ad5sko/G6Z2TyVnGB2xkgB4WqlcyfvsvOCK9TFdZcjAr/hoTvMpOWFq/TmwozLqnDV36X6DnkALC75oC/UAfqhesSX0MSApP9LqFF5KP8VgpFKjaQeZzMDYPMmZ4NmphMuS2uCLY5qYi+ElGynyappcTMLrgy2hIVV+DvmsFwzb9QWj1CpvrazBI0OrGh2KDpYsW+ZIMTXDW5xOlFX24sTKVak0/aPYojLnMmYZp9D36EMvdTNcruknCGd5nDVQc49W2H9HF7f+QkKGtl/hU3PRnvsgTcsvZse9RaXkIdo0Q8qS/p9FD9zw50YvoHX6FL/reh6ra64Cl5iIV6Td4PRzect7Oy2H+c2yvQD2nuy+nlb0pK2NhYNyeP7NA3X22/NqdjF0+n3fnCbB5m/Um4N/uZK/DybxX57BwwQKq7ek8v30/3wq8nABnpfjmTEVIkZ359eILlFrC6amDqeElIsISQB/DXnU31gb8QHmIiMSSZVl8b78EQ/7vPFnVAqP+At11MDU633uuu6jd69zF7OcoK3jQ75mUkGZMyDyFC+HA0Co+a5wks/LyW5T/kU2bnC/c11ILNnx9LimOJcT65i3By4IHsTY3aK8CwkEMiMjlums4glvwzo2WbA5axj4THDb5y3j6kgQ2xkOnhm/96n0lG0sdUClFeJnQgNRwnYYm73CgeW8GdxlDYLGb3XzsceKDLvNg9A3S1I3X8nZIlMrOk4FhjGwQTgQB1BhEFO+AzcyxrudF80mM+gu8FQH3V48XQHRTIVOaELSe0uBNZMW9zQnposirsnu45/pG9XWSc5PpURPJYyFbxfqWNgUiezK09EfW+IoHSJKw0TKmQ88f+dAUSaHjf46Y8C0xQE1NjWcebWTr99aBvuqYu69h2OlykkofZogBmmsK/a7RXn2OiUEQW3UrZ07/iF4bKdaKLNGnn+Y9xLmkGTQrmENa5a9CdrSxBKXzWkMSh8ywre06hiS85HdtY+cvGLN3O5UBhz1tITAFYgehpoaN7VbzZCgMK4QyexBcmCfAmN+SKHUkU+KAkPqODDqbQ0JgigBm+66F4cdYmCVAP6dkY+GZXWIe+SUEspcSUfk56yZKDAopwphh53Z5sZBineSCls8zxP40X0eDTl1DUNosAaCX7hWEhV4rYMRJTpcO43p1e947uAJ7hNf+xWFmXHkbvoiGthp4t+MrULhJvFMZVl2cSd+iWswu0DjdSiThnWDESZxKsAfgTD/TjUPWPWJO3tQB8n7hePU3SJeXkB+xlKPp4wm6NAOyl7gd8jYUH3vN3JijKigDXA5Cc2ZS2nkz97tzl99MxGka3pTbG7ZR7IAXC+MpU4+EnGWC2KeNYF/4BPabwWmx/1fjr85YQ3LIWZZmHGW4AbZppnNiRRO4vBByf0KW7MiSHYtDz49n3qBKO1QQRmMGQfJEjlrKqcJChhpej6kT+7vfUsQYTH+QuWVi7hhVZOfRsjeh5Qtwl1lEwmkjOWjWs7kBVsRB68CNQso1ohvooilv/iJt3HxICYkrqhcEoHb2NWK1J/3s32au/UIu2loDsUN4ZFuZx4HrwokruCW0fQtGnsPW5gPqdVc5lzyDk2mTOef4QBCD3N+McEiK93zJBtHHBgvCX/VZbEn3e6Ttfquay7ABydwXFCNyx65Qw8nnUfnwYR7t8ihJpU8IYsDRR+DoI/QI+hRjsymk5r/NxpyZwnEMxDk3sHZkb+5VWqKxR7DFOZPorAUCDNzYmgdi2/jJeDqBgKBkUAf5AQMVwbtY1kIoGhCQCseeYHvwcZFbKWg3JSEbqHOUQfpUzzlWTQoAJ4qG887+lTgDW4gIo3NvgCQzxjWUuZXwWosfearrdLFO6WMgOAOKdxBb9xnVwVvpWlzBF1mTkAtWQ+dPqe+6iRMVHbhhF8TJv4uiFL3+Z7CvMMxLUFAkRSgjnZ8H1mpi456iom1PdqTfwko+hjazRQTLjTWoHV4wPlkFqVyBDu/D4D1QeYyJRR8zKFDMp+vqYWFdNyHvWHUSCjbikLVYXBBW14tbdGHYlEihfDTsD87V3saAMyPZ7yY6KLIswOZ9tyNdW8wPZYn8Xi/WvyuWcBFlubUHnJ+LSdeeU8VDcOGivwG6yPtEvtwCsU/2BTxX1WpwpkwRtl2LZ0BWGBP4DSn5b/JsKDwZdgSqT4sIdn08Eg60ksjtqb7yFSrzQPG93loAzR/nVEAnjligvjV87TNOby4h/D0Y6FtKgOCgYNqUPY7GJbOkBjKPToCgdJG72FTEp+qfmOej2Nc39gAvtm4JsoYyWZBcnE7xX1ZlF34rWCQivfWxON3ReU7Jhs2N4TTaSb42jVPy2keN40zjBuvskhfsK9WmQeo9YMymgyoHFWBTvI3TXf9G/CP5LuxyKC5cHE+7ky3tw9lZ5lXLIagZh4kl3w4RCnQymISairt0iDzBrAgXLasncELKZmNNJ69iCSDXnCY65BQmzQ0Ul55KVzshgxzZHS4uYIKb0NxYtA7xRnTOIhSXjiBrM+r057nc5BW2ZG/2Xlf2ym+eSYL1TcpEFO/+CYCLJyxfsLGTFqPuEgZr8k3fpAsndpy+YJm7qGQVb1eBlCUUDgbpoZW0DXDBHbXQ+VPaWK/RSmnBlbg3OJnqo0JwYzVE9qR/l3eJqOtPrf4MdbqLBNlPiahT4zVkGezn15JxyEWQSdgpMVUH4No3uNThhOsLGRhUTZT785AlWaTQ2dyFWMfvFNS14IIVCtPgl0R/IrWvz+a9SBcBB3p6fjfmfZRcCsHWVHoGrwOXQxBwi7d7Ito+rRbrQaUlQQRzDD8GyRPg2GM8Fb+KUcfsPFj1KsOD74Yj02DPLQTVrGVU7HESVCKaUeOIQJU1Hza0RslfwfM976GT2gsoaxSNIN3pY/BGQLj9fihQcViQOHJXuFUwFFLUcDAR+qtOeR+4/Aj5TXN4zhCNyqkiVFcM9gZPdWzhKt6O8Osh7zgo2wdrE5kdehqAsxZ47eQHECdILrIM+/Mm8OHZWRjdBMkYbQoAVbZU8YfLH6FVvBPMT7Hx1KbqiHOJ99LQuS9XO03ArjH8bWTfzeVfsM9bFEX5nw/6P1T+Bfv+Lf8rimLOJznvWWR1INRdhYpD/Fj0BIcbGap/cqKD7PRxlNxkkNo0CYTpSkkNPcVXR7+kJOp9yFkBKw1QuJE9zoFsaYCi8JVsC50JzR8TG9dB26HN61S1XomtdAIaCT5u8z2hRQv/dH9P2D0qSJoAGdNROatwuhTaZgsDJJwowZAFsFXjkrTMPPkMT5VBk2z4Q7lFLGJOK9iNhMpO1EBIsxk0z5wKDYWwJh629SJocyh3dZkEkouOWhhn2CkkGSqOQOxgTrRZxQMJ5/jx7dfJu/M3TPpWfm1+qf0dPFq5hTvqHuHZJlfRuMpE7ilzMQSmstoUTq4d2md/T1TtMH8wRoKvOs9lS5utZOwu4tffFNZZ/z4HxSSLhTfnzKFirYj8W2Yq58VSE6eXriQ5IYGFCxZAdF9uud6dTQ0ib17nK50h8XbvhXosZcWVBbgkJw+HwAR5voieS7pTSKBVn8boEhFqWe320yH4G/+GpN3Lb1nPe36qVED+Wrj8MXpVDQ9J3ZiSt45jZ77ibMplIT8FYKlkAt0Zkf+l+x27O6PFc6CPo6TJQo4WjsYlOYlRoKVy1i2jOAw0IUTIJ6FRvuXmnH0uJ5HqMyTM3ofzrffZdpPjzhaQxoZ6PBK0ys2rrTaKB84dYUYZnLPC0fI73UmA3d+FDwhe7QRHYLpgWiMW7lEZixiStJYO2cvo22Ul0kQbJIxFbv4knf94kdP14UwKgqcifRin5jIel5bxoFtG0azJx4GP8RmQTI4zFICnQmFd5hEkW6W3XtHTkDCdY9Up7j/4R4EoLgcBElyywul92VQb/Rli4gxhqBwIfpKeS9yGYdYXjKqVaR/pn5VrumkzbGwLeT8TrbvoVycpKrGBTJ8GlcdZGO0ioNsZKtMepYVqhN+xYQVz6JexguDybO6PmU5iwRRvZc0F3gqqZIpdOC6XxsD7Ef4A+wP2H8lyk7NCVDYUvMadEy0FoUVUKsIBbNeniLFtq6GCAB7OyWCZbg0qewhqDPiG2nYxTKB71lYiawdx0QpXVF1E/ro7G6Dpw9TZYz3z08OhiCgjdxSmLSCFg6WBJF9uzrOFGTwe6t2UePooOIjfbr/BzNhg9pY+7K3IX4thVxtG0pJuh7sxYvFUPqp8wVOtKJBniibHDqoT0F/1z+XTLu85zNCTFQw7UcNseSj3BflvbW+OoEAJgI4fwJD90PsX/jB7wfkphZ1h6AHP76rg1rxTKRhtcFO+IqBl/SluDRARgwlqB7LkEA7FHj9Sp+/H5ujXONR0PPPPPM8Vxyug8d6rV9Q4yrtFczTjFhKCLxF+/QkoP+gZuL5rk9GhBuNVqL0CQGylWFeOmEXegjI51a9dakk4Cr63X6bvsXshdqCQHsp4FBwNqLPepT7jAT68Ooa5+3/1n2ckCYNkRydBSf4n9FXkf/wutmxZwphmi0hsM5nX+4+kifVz70HV53gxqoJMDaTFbGJo+jfCqbKmCfykYE17lq3XptEm71PWbs9neeVRAQAcug9SJtOQ/AxPhApSgMxfGKl2I3rFxA+1MOhya0yaVjC+VOS3aDKGlVIHLlmhwQXXHRakxgTziPntmfhsxgeIPp8cBJI2EgZswRo5nE5xm5g3vAUxaSIaVZEVscbWXEDGTqS+DBWC/V2laSlkoAISQdFxR1gWvXWQroZQxSlkDtvNEzlmVQEe6au7gyBGKYSEscLJlXoPbO/D3kQLj4bA7O7TaaH5Gq59LeS7Y/pzoOlrdCgSTi7JJRPovCSc5TQOI28vtTF9IfpbHw/1OYxO2OIFYhrn/tur4I46GiIngE/kn1ayQt1lwdDWRmAlDAkJgyTAK8XlM6erg/jD1IrKstGMSNhOj6gv/N/TlU/ILBtPjNrGHUGQEph303tsYHJwLd10YDFVsmnTrH8ANtuY+cILGI3FKE4DGYWvkl70IkENh0VOqdJ9VJoqOZE2kRNpExkTAG/F5gjpd3fJqjlPTszHgLAL3lHuhapT0GQMdPnU44R9txKanWuDFOSVMMNu4r3gcmY5biGkobNnTPkWl8vFHYHCgdE+aq9fnci7F+DN36KoxTwy7AintYsx13UkpnqM5/ibwT67XO/3m2294egjSMEtmGW+wAUrvFQBb5V5HSMA6fXHGOUOKHy4FC7px3srsxZzyHCEQW6V39crYUPoYr/1RLOrFS92mkVV0AEmJO72npt6D8dKbqPlqQFsci9diqyInH07hxFR/gnTOz9Gj7AsFkfDV4nutSRuCOjjKLIVUhC2FhBEnuuaPlDqvn6nT/jOfJYjzUZyuckrHAv+FCJ7iByHeatRFW3hegpkP/IdWRNXs8PUTswj3b+BtHupSrqbtfX9mGa5A2NKAF30ByBlInT5DNTB3jxx/0PEhG8pAUJCQpAk6BS3kaEh4szF0RB+1h3t46zlzlZv0jcsj1WxcE/wDr9rLDQ+SbMcn/2BO+8cF96B/N/4MfgHhgbXorFHEKno/Rw+6KLp02cZF80KGglUbhmzxnK1Oo/1cYOpa9eTMb36EKc/B1c+gwOTkCUnBsWGRgKdNZ7tDS+KXKkAUX0IVWoIlW+SFZRVYp4K74TsBrucsok6eyiEdRBKKMHNCTaKaLpbY8+wywRl+BPQgsgnSQ1ttLC82zuo8pfBmVeEY86QwGWHng2Jt7C1XTTFoWv91ylZxSG6sdckHLSdI477yc4DhBl7ANCx4j1IuUs49cLao3Pk8H2vZ5geIp5LkhD3rDoF594kUelCdM1IjPqLvJDxm/sl74Y1CRDdjw8KvOSt5wvuFHvAMVcguDkBRSJSqY3ewskkaGv5xX+wXP2S++I/R3ZqvPPuwG0iV1dkNx8CoPq/Gn9BQSEEaSoZFFJMsnteP9yYd7rF8zglPTO6PMIPt8ax7soTVGkHCVnBQdsh8TYiFAd6CUwu2FevgXZvexVLGvJR4XVkeuygww+JvGkDt1Gq30+ODW4vgov1owUYPewQpN1PjRLKOfdw3td6FwnOVdDpIxhxmgJbH15qtsojN52McExSsA4kFQ2OSKzu4ezCCcGZQg649iJK5V6SDF7QXC3ViXfYINYUAdb5DhoFjfEk3FiN5sYSZiQe8dTY1CVIktNro1nK0UoFFIX+Snb0h8w69JH4e97PnnMOmlRsbQCLQ8vanHmenFd2OZi8mlaY7EE4ZOGM2Bg6Wjxz3AhyLMP8nNZaCUHQMJcRUHfRk5MWYEON25MbNwz0seQ4DbiA67ELOZoxmhzrccgQkSCu9AdpkKNwSGaKjE05cOMOXNpoaCgQ1whtg0tS4wS+yB7DsjNvYDAdgg2t4do30OkjjicXU++WoIvSlSMfuB0qj0FAiudbd0m2v3TO+uZH/KvIPj8wUJZF/vO+v0J9Dq9nfMTwiGyCZYhTWUCfIKLqTzxNTPEG7nYHw8+phNelryB+hACFi7ayTNWRP9wCG0ctcF01XYAx2ihQNBxKf4VJxYDkYnDaN0guu1A+ojGaU/Zvl8MiQI3K47ye34JPa8AgQbBsEzKGLge0eYNKbXuuxM2hIngnYwthgf09Ia9cJvYQUepkQo3dAdherxbqCSefE4Ch00aU6hTR2mpuCYThgVeFakPKZGgzB1erl8jMhXIHZDb0I1wOBLsJTCXgsHhB4U6wz/Gf80uPBX74D3U3l+/1ekaN7svpXh/ytjwcO2C0a5FDW0GPHyCkFWqf9x6lQNuwC+iUOuj0ERWqAbRN/I71Da8yYnc420O+4kj1DBENWXOJtgFBhJOO4gj4W7DPJXslLT1gn0/OPqdkQwY6WU+KqP3LC3nJsJwgWYzN2MrbyGAkhHUUJ+8dx3slg9nQWaYofCU2VRUOfML20+7jW1s7pr/6Om2fe52U0+24UHOLp3rNped4aN0VGgwG9plhYdntfmSbTZlf0Nu4i+Ppt3va7SlXFjEuKNuvny8mrIReK7DpBOHU4fDJUeiTB1OMTfGeL1iht94KaxOhzRuCSOheKxojxnzL6qzlbOys5kBmL8yqYqx4Sa2+uTYB4i/9Ro3UFtrMEXuX0j2w71bi9EcoDl9Ncegarz07rgB6CrWP3KjPOZvyMJWB+6kMGC/2Wu60DGrZikq24sLJlGDIzF4AdZeRnfW0i9nBz82OMaXsXcAN/DstYClDJVlQy2YMLg0fVUGh1N/b0LzVbEvfTl+98OM2sbb0U9gIP3Iv8yNFf6gVE2OiHhREh4vzoWgjZMxgVEknSh0QrYBOaRD7pd1joOwgtlaz+aV0OnW6izze1R1d1+wxKNqKOXQoHXcs4vd6b39LOMHlwBU9kLn7VhN6LIe04ucAGG2wwOWPoeYCEjYqAw6yp7UgZier8KjVEN7JQzwvscPTZXDG6bO3UAdTYBqG/sarlGTezpfD44REtrvcyHiO0e4lcFb9M7yScEEQ0wEMIvBiqKGAx0KgwglHygd4lDlkGc6X9eWTunoa3IETOyJWM7nNqxRaukLyRIgdjNa9V+6rh766HQSpzIRJ4hqWyHiq41rgkpW/Bez+Bff+uiiKQps2bf6vAX7/gn3/lv8VxaWJwiVrcIW2gYvvwtWvUCQfR4hL/vMk08jE5M/ss5pQEequS3uZQyV9qAscJiI1HGZwNPixk0NCN0O9NwqEkJZYY+6gxqHC7oIXzk+mLtYnHNtWS7xtDMMirhKjQMfIUwJwy/octeU64fpC2oVeIEiGUMUhZOZkjXAOSgrnL33ApbJeFDrAoQQJZuSYy9B1MZ2ym3LaCg/En+btlktEm/XxoI3CGdSKHKvok546mBi4HuKHC+OteDvNXMXMLFtMvVPkDTKF9PbrE+2+W3mx5QscuziX6RsuUK/rJuQzirZARFdm16dywgKdg4sZnvojMmaKi4t568032biyBaNmH+WWOdmcOjmJ/ir13zqOFwLLgQPAdps/i3+r1SYk4156iYULFiC7pymLC4rser+E29C4mLjoooU20j6RD6j3CghpiaQO4fEQIf94rao1JleU37mH8g/xe+tALvSQaN3iSRLkTUL7u/dKglUFvNj7Tnon/kK9LRSrtpnobxCbg75D6BUt2NyLErLg2BPQ4T24tRCHNpE/mg2kLGQzvfUwQ7dAMJoUPXT7loO2L3DhpJUGHgyxIVt8olO04WRcT2KqtIM37EY+q/TPhdQ4NttrRbTPxNSbTHxJwuHexG2oSOLHayuERJ67r+INBYwLgAhZSN6kaFQeoKdRnuqdA7/QpHISo1Pv8FxWr9bTtvQdMgve5rkyuC13oPeeTistuU5Ps/hbBy00dR70RIthMxLhNuxXG+GVvJZCJqyxqAOpb/EZ68qbcToJZqbu8nOiqi4vwNgU2miEtK5vndp4igVDu3BreIHnb8PU1UKGIiCZEvV4jwRZY0m1e/t7X/lzN/WfAulThARu3BD6ZwWg+ekWxtoXMTLsXr9DtcWL6dr2C850b4I6VY9N62O8RXSBgTs4VCUAYgk8TEXPIaH9uGZsgk6C092WkFTkBc7UihVbykx3cmu3EzV+FIS2pV6dwJe2LK5aFW47f51nA4Z7HTYuFwmq7aSEnvE4FBVJEfK32Uuh8yd8l7uRRvbZx5WKiAJyb7JkSeZ86/NkNc3CuKMnyeoyvzY71rdisHwrra4+QJsb3xLs8omCC2vPN7YU1sTNQHbKuBQHQWqvDK2iQIhiIUqB0DNBTPln6lXcazJxfOc3PN1xJhnBOQxvupjm8mfeA+pzmR1ewmOli+h/7iIz8zfQ2uTeFutjIOl2Cu3eeUO6yfSxRQ9gVgVEKlCfDpOT/B13vas2MiscPqiGXkcfxCg1FazLQ/ehclUTEnKYusA/sClVYj6K6ikcg71/YXjYJpqpoSrwICcsTsqbLQV1qHBw4M9gzLIGwe2VIq8Q4FRFkm0T0ZkTiuGcMsivXU+rnuNIItS74Ihhi4joSroDunwKzR5nujScVfJhvo99gs3WKv81Uh3MA8bbeaUCoi/AA/Z/LiV2YM9mBu9axDMVdh4oga1mH2mP+BH0vJbGDhM8ajlM/LkMpIBEIdsGyKbr5Icv42rsPErkckrtnSGoGSSMg+RJhJ8cxkdR8G4EODIgsXa1fwP6rObFq3MpdsDJhiBcSqCIIglpAeEdcTWby2WbmNua6MuRXT6Rp5LMI+svsfCSAOXfjgB11iKI6IZLn4BWaSAk8BJ69/CwOWxw/i3Y0AoNVTy77wf6u6cZnaKHk88LWbbqM7zeZD/3BEOPRuXKmIFiHumzGqn+GneGNqCVoM2xu9hsXgaFmyHrc7/E6Nl26NtkCxHyaeGIcggHYuOcb5DgdJdl9KpqAdv7QdYXhKsuc1v0eVq6zaGDYb9Axw8hJBMSxjJq6w42VMYTUdufIddOwelXRaS/rEZbd5Akba3n/oesnaD503BsBpjLaHDFg0tmTT0k54A1fJi3Lw0JHG84x7WTvzNv73o+z9rv/57SHyIrehnHG7RIWfDtlfv9gTFbDR/HFjMpCK7WjaefovrHYPP5M4u4K/ND7nL2pkXBO1gDu0H3b0W+wRu/sqbdCjrqzXxRA7fmtBE2EkDdVaZUzeERt6qzqmooOVIPkfes3+9Qd41H9MeRAKMLSu0aQf5oLC47aVV7GGhwsW1kPx7pfqf/M517i0GlCooEH1dDaUOyX/tf7jGESa1n43KzsD2S5RFdyNM8hNEajuRDUuvSpIvn35IETtnsdz1avQyp9yPL3g+794VjjFIv8DusW8VyZoeD4gjgwtGdaMPv91YGpbPDEUGp248UaGpBl8iBfuefCR7Cz24f0vhTd1PRZgOMvgSp9+NAT4VNj9UFWlsM4fpwEdVa/gdWJZgn93/Fyjq3dJFvFLjdRJpynUR3964ywjXdMCFVl/8bhHfgsjnMc/ibcXnCGW2pgNI9yCo9ZQ5o+v0EjF/exR6Lv/Ry8y5zCdXtJrToSS6X98ImxQqQKGc5XFzAN0H5tNSIiIklfxMx4Vu+0ukIiYtj48oW/LByLCe+3gO74bnrUJ/5IgCKs467277GgNA8ZpTBWqO/nW2T9BhdUBm0j/dGNEdVsw+6L4VuS0AVQLEzAJNLSEu+3DZD5Nn1Kb0Se6OpHkZmLiwtutWvTm8qoLsOWmvhoaT9ROmuiG859ydUNDDo+P28VwVvGTKYGPyKcGgD9FnNO9ET+d7QlaZFLwM3OWkuvs+zzb8DoDJoP+fjeghbtvsSiOlPYcqP3J+bxJxKGF0I+7nFr12bDesYWgC5NphVEogzsi8M3C4ilczlnCvYT23AfjoGlfF0uzmojGe8J8tqViqTWVIL++pCmLCh2BP5KcvQJX4dM+Ny0EtAozN0Y3vY1BEZB8FqIzoJ+ofmECbtFzkf274FXT7nzuihdIoVigitGz87S5kAlQPTUVxeGWL5JjJRWZdTPLTuCl9WBBEggeLrVAYoWM/wiHU8lHWazZEJxDnXCfJkVE+ou8ro0s+4NQBq2hn52keG+O/K93o9Q8dN4lxZP1IP383iGvH3Fc4kYW90mA+SzOXy7uy6fjcOp8r/PZYfYHfSBu4Phs5a6BNgBXM5jC8TShc31jA/RtjGvXXQUn1F2JUVh0TEbeFGOvAWXXXCji+3Z0LNJagUyjaKw+TJcd7gVHBKOrG+hbUFSc2dyVu4t+ZZRh9zkaefKQ48/gTYa0kMOus5l8b8d0cfhf13oN3bj0lp3mixcmeEAE2bPuIZB40lUIIxrvbYM1+F+JHozj3BownHaF7wFgAvlgTyk6UMbiuFXj/D9e/YWTSI401v40LS0/xe76I2cJS4WEhL6L+ZEQXhmFzC+et7r2p1H17YvY49NVGeXHHFAS0E6arj++ypW0hk3UASyu9HZQ/hjebPCmn5rM+Jz/mCX+O8L2dqbhvxDuOGQKuXmGFqj9kHy5dRhDLMnSZOJU6n9xYdmzrp2dhRy7H020R/NZ0mIo004bSSakhSwQLDFyw31aJyGUUOxMMP4lQHMeZAd8y6LJ4Pg4zg67h6/gTRfVFV7OTwqJG8FAY1ASfYlNaSe9d49z0355b9S7DPx6ZVyYrIZR/WHgKSmXGjNWvr4bEQ+KPpPtGm1Pug/yYiyraxMMr3WpKwgy68CxVH2KJuxRnfLZSEiKrpsRRiB3u+0XrtVRadep3rKZtFzskL84nSXuG+mCxS3XO9IsmgDYdJLqSuiz0A5twIONbyV0HCGV8KLV+gRcFoPuv6OiB2TeVSPIy7Aa1mAZBu6Ej7nG/dTZKE3yVlsiBdmcu4I7ADUxP3M7IAJhXeKQhiPX+ExHEeMC9JBT+PbcaIhDlw5lUhR312Nk+WfMgb4UAQOIfAUOWvAb+pwC7+MxjYWA4Du61W7r77AbZlT+FKfQzxCrQJqBJ739K9sHcsnVzr6Oae/gbpYVL6z/xe+AnE9KON/VnubTubHba3qLNXUR60w/tdbGrP28F1PK1cpUnVXZxJnMGW9hFsrvrE3e8Kk9tMpj33Irlk9rRsy6b2IRwpEIC8R8YTC06sDDfAmLofYOcwSJnMN5axmFygdoTROXsV9yobYNAOAbDqYkhSmfye1xdcBHjJdsTnlz+puFmbKIqMGVjdaU+Um+b8Rjur1nCKM0mPEKUcgwMToXQ/DNzKS+UD/I63G1pD8p24NBFYlUqOh77E9ZgPxbV9Unz4RvbdVQxTitPE3Jb+AFz7is/icgC4kPgUl+Nf9WtzI2hYE3CU7e3jOKaf66lTZIV18cJPNMIAV2szOKJbK5Q71jUHXSwFrd9ht/VKY0t8ru2EfbdhvfI+9bqsxt7CqYSKvZY6EFmGZeOjeK3vaFw4Gd+4xVb0SMZLXCrvzieHvyKrLs37zDEDYGwOhao7mNX7Nm6kW3mqHC5JPjaDrMLm0qG1JNAu51uW7TpLw2Cv9LzaeJVMRET9udg5LCx6C5o9IciVPZdD54+pdOpIUUNJGtzZ9G0RJFK4HmovcT2wDTPMT3Mwszc/nH6bTcZV0PkTuMuMJXwodqcGF1ActgaTugBazoTRF5GiuvFH/niqzbFE1A0gteQp7pAuCTWxDa3Q2Is8JH4Aq0sRZM/u3wmCjyzmxkonfFgN2fj4TkIy+blkI3uK+1KhyeXzSr0Y0+5iCWrOEQvorAkMDS1mYvMvkRrJXsHNoLfwWXRyf7P+hAvx/+woQXI0OeG0I43CumY4nUCv5dDvN8+3Z3TCVVtn3tjzu8cvdFM2oT9du7H8K+P518XlclFbW4vL5fqfD/4/UP4F+/4t/yuKUxvF6czDkLcKJBW0fIHnk19jktQatT2M1nmLvJPIpYVo8r/5WxlPS3B/2uSKXB7Fob9RZS2FJrfAoN2QMJ6X1Qv4PFx4hA6kFQnH7qlZsCocCta77+XEAWwo7oIttJf34nYT/dXrGWiAoQb4sNsMETnVeyW2oI70TlrJt93eprUGap06cc+RZ6DpNB8pAztN1RDgMvo22+Okfq8K7rueKYz/WwvgtlJMA8+xpDIUgJ+N8GrVC4J9GzsEOn2MIWcR83o+zBhnJ2Kqx/zJCKL5E+wumE69LYz82ha4FAN0+RwG7QRJ9hj1I5rs4IluU1ny5XySExLYP3cu71Rf4tu6Bl4qq0Wq2MW9lpucUj6lGJgJrOM/R/X4yvdNUmfRTgM98z7lEas7SqTxPZ59k+RWadQEHGdqKbzBBpHEt+ygAFANiXTQQVrZdF7etY1r5jF+9wm+8BYbE+sZrId57T8m3vW7YHTaG6hzJDBv/y+8a/iYPzqEY6w/7c1BF9KSxSe/4ExFW7TWeNrrTcKAMpVA7WUSr41kSowwkI6b4SfrVBi4RRgiuctxUsb+Fl0YpFP4Is6M3HDV/z1LMt0OdaPngZ5/Wh3Dc7+jLl2wc1bVQaHJ36GI007zwGM0U+OXSB3Ewtwtbhtr4oWu90fRzRl+bpoHKJIkKKlPo9IkwEGV5QYUbRUONmMOnePWEayuJccOlyze6CUMTTjV3MKagzsIMGfwRCiMtb3syT3GsRlsityCXoKDtRFsuPSiACZ8yg8XviQ35mO0EqgkJ74EGCmsPT/VBTJGE86iIQNICzvpfSZsBGvL0Pmw3IcpFULOJn4Ep4JWkV/lP9J+CnE7yVo8z83Fr7t10Ryr19DqREcq1+lJ163xO/Yj3TSG1l/n1+HTGBSyji2Kj7GvjYDYgVS48wXeVwL3lXb1O786cREzzt2DE/ipuDX1Qd7zw3T5HGt9lBfc/s7Ews8g5wcYsAXZ7RxySQ6SQ84zKWEc3Ghsm4vhmiHc2eotXJKTz6PhRfMUAX4emgJVp1mlHsuOdoINdsqiErnH3BubxLoTFCYH4Op+kg0DdjCr8gG/Np83W9lzroimp7bRXf8jsXqfyMiw9rxRBecpJ/FGIpGSzO7mf3iqZRm+ar+Eo4lg/S/lq+rraxje9CuiDHm8tfd3djnXew8wldBV+xUdDA0EmjMZm/ElIRenwd5xQp5q3x1+TonBAWUCbHGXRnmpBqdbGtfiLx25NfpeZpRCXOUE2uZ+KcZI7BBQDKTUvMz5Flmkq+GRHhOIZLr3xPhRPJb9I2vdU9bmpkPBnA8HJ3qkRiLN3rE5P/acWGPcpTjhU3qdECzy9KKZJBky/dp1iW6srxfAVrLaIRwjphIhVaYOolwOocH9WZjV+f8xZ1+d6b+TsqtvqKXSEkGJA76thXypufcATShXbAbqnIKgUedUIcky3FEDY7LQ5H3FmNYvcynhJe5r9yK3hg4V7NnevwiWNSKScVkd/FwHZm2S3/335OxhfcCtqAG95ESWnHDpI7j2LTgdDEoeTkTtAKYGw/q+T6GqO+U5V5IkCuqaczhM5GeYVAxS7SXYMUDknc2/jaQc2OYOpqkyVwlZ59av4ZACkF0a2mjgbBK0se8ReVlcdrj6FY/kjOLDajhe1pUPL74kcpptdn/v2Uv5JrGYYBmiSh+mlbYAdo8QzsyTgmywrBZ+NQrJnWOuD6H5E8LpXHWGsOqjjAsAF3CxIZwqdV8RrX10Oom6/cxvvpFxcgY9C96je9Vkka9vuQSyGr1xIP3PFNDjilvmN3uJuOeh+2lyvheT44/7j4eoXgJ4NRUQKOX5Sa/dHEUuy5AfsZQfWiWwRJ7uV5ejjmPKleXU6L2OBL+iCWdMgYYvaiD0PNxvs/BPyr0mE9nnfubedi/TJV7MAXZtiiBFBaYgO8zEKCKKItcORxrCPJHrACqXDRUQVTOUgPNbuBK6T+TUACjcwG268ygI4D9Da0Zy+ngXVYG4bq/l2fwkdlvMnHWV+K8Xwc1ZX69hRwM8WQaFDS392p4SchJD9BrOJou+UitqMBXDjV8xuMQaLLu8kTVtY7yRRZIELfLno7aH0+LG++K+LZ+H1MnIlX+wPwFGGuDu1N8YGrLI774HYx/mmTIIre9KZN0AgjQh3srYwcyxt+Ss+zEnBeiIq3rX7/zjkbfyjRsTLrMEo63ZD+szwXgNg7qWTK2VMJeBFvnzidBHQnQfmFBLVsgkPo6ZxrHADTySm8aDuQ95L9qQzwu6eR7gFaBCnSkcI+WHYVtvwlReJnutQyXIQ4pOSEjGDqbrDTifnENVVCStdUP4q3KxvBezd2+hwD5Q5DM9OBmyPqOVYiJIBjrBXof0PzpJnwL2m80oBw/yTvUlvqyx83yxjb77FLa9LfH5WsE8d6qjGXhmMK+Yz/NreSq5dn/g9JbQ3jx67TB2uQYMVzA56iHtPgEQxA7myboR7DLBhaSn2F7dzjs2AarPoduawv3ROcCfZcFjri/mj0TY2gBNsxWu1g8Vzv/40diVCI8T/oEEn6dNugNUBs6YHyMrZwbXYxayr0VnykzFwqZYLsHJ5xke641SXZF8BvZ7iWAuVQhrqiI8aRVuVloJ18bQOncRhQ5YWB4oFApcLiGXfvgBbjs/lUlBwiadFH8K2VLgd37jOhVVOxSdy7s+ShJ0jt/Ic/E5GCR3VGLxDiGTV3USizqF8buWsrAafk4qI131uJAxbP0yBKQSq/kDZ7iYQ4YVKNS1WyX2DSmTQB1Mv6AKz71uDTkBZ+fA1S/FGmtIZHWTqVxWbtAsF84Z7vFrc07m62RcD+JyywF0iV+PwXldyGwXbYWGApIazpGuhsAusN9l+0dO+l0WC/XGWo7ubkHDkk0ELAJ2g7naLPInOgUYtTV7KktPv8MbA4YRX+8Tda9vwqq6DM5bBbje9WqcAB91kQKQKT/AyCAha/dhFEwL+QHyfxd7mxbPwemXSXat4RHjY4w4UY9OCoK9t8DmTlC6l8jzr1KeLube4ecHoUhWWBUBphIsRHP3ryUsP/c6APnaKWJs9l0LVad4u1dbJhtEZEHvi0eQsxd79g8uVTDbC4Z6HmNk4D7Y0FLkxMWdRqJOOEaba2BJl3fRFC2H0r04Inrz1OZTxFaNc49Mybu3iOgMXRdzye4lFYSpGyiLfFaA0aPOQ3RvUlQQIMG5Nu2Y1CEaskQUuyxDdcAR9rXqQFWgsHN1Ku9aI0kQaM6kfc63DD9VzbCU50TO0eh+BLZ5le/PfEeHbKH20ULXIAg8bjv+ZnBZlhTY1BE2dyHz2CTGurdPD4RaOddhParqPyC0LTQZC+ZS3nXtZ5p7bq0KPIg5dDB0/QpaPI9kKiHUkYNBhvmR0CP6EFLKXXBgErpTU9hZ1JfLbu5XrfYiOdU5fm3R2rzkLuUvZDx9wT5FUtwkajNowlhXE88FKxyxwJLKZCj4XbxLWYOr8yLevzQf2amjtQZacEqAKS4H9Fzm1ycvhsGL6udgS1dBtAWSKndzTxBY1aVsDv2CuNKX4MCdcHY2CYajzE09xgM3VjPqmJOkoDS/Rn+cnMWccDhohp8r00X0naIHSwV2VTT57q1/aw00IVdEFLqJWrIMLjfB9u0oM9KeEUKy98ZqUAfzXV1ffpQOU+8Co9NNGN49Grb1Rq44ypbmT5NSOoOVeUO40tBK+DAAbLXoAxJIs94vrj0GHr8D+ioKw3Q6lgHbgR+BCSohBTpS/s+A32FgjFbLO/PnE5PWjXdPzOFISV9eULqzqe16pPpsSLgFmj1Of9snPBggxvJxCyw8+xQllvZw+hUalAw+uHSf57pTIip4Iq2liGRs/Qok3+X5xuxKLTZVJXYs7q6W+G7sd/SrmofKEYxdqcWueAlnTQwpJJU+TLrlDqJrR5KV+xBZ0feLnH3Rfdnv6o7Z5Y2QkySETGzZfjCXEK74oxKNko8A5K8jUV+FbcEbfPPGHEaG1RKs8UaMyVIpNzKHc1n9HcMN8GL0Co/KCkBs7TH6uV9fUdgqApUCyF0h/CdBTSl1+hgzQGLZ23DqJSF3qdRxIWKep8436s43Gja26lY6qq9Bi2cFQFWXRYpGzIP1uixyoj/zVz2S/b8/332uSlZ5SIDRipj7kuxL4MYqkRs06zOWVFlY6XxHtKNx3Xa5ROR0yU6OXFuJUX/B/XcZxeWNOpUk2FjSgQ22Iur1l5lSAlcyZop8vbKeImMGW7OnUmwS84WvdLEkwemSwfwoOK/8Vjbf+xAJY1lyuRztkRs0qZzkOd7zjGOv8/3uYlJKHudy7AesazALVYcj08Rade1r/og7wAthsCJvENlVg8Sa3m6uGJvudthVNXxvzyLHPtajoKB2VtA1sMZDsk6KWYect/xPbYipGUmrGwuR2y0XpLvWr+JUBfnNfWUONfRZDWn3/uk9i/7w9881Bi6UOeCZkhBvTnhAxoUKEfX+9YmFPLQuC8l3bCfdzjKXi4fzoxhsgNUDekD2957rjm2+gIvJkKCCIgd8YruXXTn3EKk+D2uT4femyHVZhOnCOGGB18sfZVNdABaVIHv7gn1/lTPyP5X/V8G9m4vT6SQ7Oxvnf0JN/w+Xf8G+f8v/vhLSErRRtA7cR7+aaQw7VUlc9W3e+ovvo8n7mqja4R4d6ZtlPEPLv+TeICFjuKr/g6QU3C5mpZh+ICsEKHpi6oTj/YPyUMGcuPCOkBLIWkzgjkASm6xAdmoIaejgP6FpIxlW3JdXK6D26jssOb1IaMrvn4BiK+VcaT+eKIUcG5SSJ+4Z3NzdThc/3BrNuoxzZKVAF+dG73VrLjI5pJoYRRjKG2sj/Z5JSDSKyV9bci9a1b0ehjXHn0A2CtnFlOKZ/BqrJsp03O/8TXJTnnN8w4lWHbBH/YQi2UTeoJgBUHmceYHZ9NTBG9YTDFx0G5++9zZ7HQ42m/0j80L5e8fx18AA/qF8n1rBcO4a3XTQP+YEr7bP8EuYrMr5kdvCvTrpsqSC7O9gWy84PA3K9nHN2o13dJ+zo23SnxYeg62KDI1w7j52YRw5hpkw/ASEd8AuBfO97RqnndU8HV1Fv/z2YrMOYGjCt5WhbAn+loSKe3jnQC0M3iXYeOszCazdRIZWGGg5djjiHChkFE0FcP17wuVs7Kpa1jY4eHn3bziDfFjpThujAqoZsXkEQ7cN5ZmI035tduhi2WcSutx3FMOhCn9nEo4GXkqZwQthMCIym7vSHhLsXcQYOVk0nIezW3LZBkbXaGj+pGBHIjYtIdpSwvUiCkdXtVFIj1adxFmwnhe730JC2BECJAiW/ZnMjX5gF04+r4ZtmlegMVl2zCDWmNNwAM0L55BU5R8hh91Et/w3eTgEOlxJ5quTC/wNioRbkMKqKLWrUQyXQFXjqXOFd2HquhxWljXz/G2Rs7l4j+52KS4t/c96jfZyVaSQLwpIIULrZYwBSE4z/JYKx58Bl5NbSi5THG+nQd/AgNyJfseaVCFUuNdul+RAkf0H2DNbnmFvpNcJdLNh3kgasLrglayh1EV5gTWTPZSFhYnsbIC04mcJ0scKB7+sJsx6kvxUmBIMBcY01hR9J+R/ACSZ74x386ZrA+DEeeMZ6rQ9BGMY4Mg0pob7gsv+ZkBkUCKVpR3QWdtzoeWFPzkUfwgex9duqZA21t/JDN3uV9+4IT/b5izV3Y+zqs4rQyvLsKWkPd/XgvJfyldpAiJoc/g+3owfw7fNm/FLjQ/YF9aOlZbrbL4qIiNXnHuV2va/CUkugBuruC8433P4K1EXhIyYu+gLf2dTPKgkGFUIeyr8GZqlAc04aoHWhnIGpHwvZKR00VBxGKs6lU9Ko6hywoSwSuLlnWDMEU7SQ/djdEVh8SFzWcIGCZkndy6I1rXP086dc+KYKUJEudWJsaqxF/BjVCAfV33Jx3E1tJPO+bVrh3w/L1fAg8FwOaNcrDNrYoW0c8VRYmQLBgkOJMCmHm/7d6rTwTDNZQbowaL576TsAgzBpAYUEe0ezn6OKYeFKMWJVhIbi3Z6s8jPqQ6GoKaoStfTM0BsrDTRawiU3cD9tSWQLfqhe57C2nrBcq0O8Jck1JbuYHIQvBgOBe0PE2A7K8Ctww9A7gpCjvSmX3A5Ry3wXfYYT1QziO8tTFdEmLu5B80Ie6LqJNriFX96VqvDKvLmtZ2DUzbQLWY/twaCIoFaUkPr10RulrQpHKpP5qIVyspHcOz624KIU3kUjs6A1HuYlJNErRNahJ5DL5WLKA+A6vMw5irP+KZrkxSRZ/XKx1CfQ2LeNyyKFtJrE8/dwsmIPSJX0sBt5FqHMWfPekpObiGx/EFk3+hhWy3pYccJ1pZ5np+RZ0Vd7gpMIQPZUe4FpJqpsgVQ3PNH6PghRlIAiVQV3BMEKov3G8JcTjf9FHrEbyJOgRi1P1hntBo5ULaBQAn66yFWX+IPNitaNtcrXLRCzX8JNpssDTy5+QSrLr7ofa7Gd5Y0gZQcOGAWM5uCJJwXAEFNWRbzAYtq/loSiU4fcX/dJOwIuemjLc5DQ663XpIoNNdRqLnCyEL4Or+7/zMl3cGEYoU8O/Q9dxatDzABMH17OfcWe9cAtawWUTH7bmOQMYXEyJ0U+kg911v9ZTuDTW0YeqqM9JJn/a4rOU2kqoXM8IDEtbQOWOpXXxbUjv1mCFSsfDk6nSblr/jV+36/d0bkE1fxhl+9yuchm2hrcTTma770AW0it7Kn7XZeLPuGhIp7b3JEifMkl8LAc9cY6PQ6u9BFscZ2Fxvdj/hUKHSr/0w4ZdzOkclhWZ7D55ckQ/OnIHYoaEI9bd44aiOzHtzN3YE+6wFg/2MKA9SDOJP0CBebzMJCrVdNoMtn9K5rKdIABIFziItRas1/dJI+BSxFKFFsucne3eNwsN/pYuFrbwjpeVnNfvUxihzQ9comuuv9bZ2uPMa8ASOQUNMiF64nvHlTX3sdOM+V6cX80lhkNTUuDXX6CzwcAq0Ds/zOrYsbzbNlUO2AbJuMzRUITR+C/utwynomR+XRQwe3ZrXl05pNMOyokMjbNZwDlk/YlXMvNYZj1AQcFzLsJ54RF24/n8m7vf37TWWC2BccvBvy16Gx5dJeb8YgwXuR0FY65NeuePsORgQ0Ojnc0RT6WBFFnDCO6+GDyLLCj7XQ5cDTOCL8bdqJju95IhTCVQ20i9ospJUR38TP51+lZa7Yy+2PvluAUwBt3kCWZQ9Bcmop7DC5AaOaixDaiqG5U3lP4/7eXDJBp28XAJPDDEenMyHKaydNi9gFZ1+HIw+DMRuV6RoPxF70OFNvtpMc6hBynJXss9Yz7mc7efon4Pd0t5R/GN9mLOH9akgOg/kTXYxRK3/rpB/s/pauLv2ED4yX+MVUzuJy6LMfTj13lIX3h8AKr/NQkezEBGSjdvnI5Qc34/2qnuw1if46b9GKbyJ/nSDSDtzOpAvC8Tu7En423g6xg0TUafJdnstskU6wfFw87UO+EzKvAGX7sUb25osacLoAl4RVihISymtiydCvpcYSjdmtsBBvXS760loFgWmsvfY6x02NQJmEZKsSwH6bOdj77eJ0eXfP/ZuoS8U/3HmPVSrIKH6JtteXcMMOL+W2Qg5Kg36/Y+u9jYK65h552hGBNjKVz+DkC7BjMOgTKCPUc+2VXd4jruRFEdn1kwIrAzmdUsroAAFitjdUgVHsOQ3OHKY2+56uXn4GU268LfLeXlpI3yB/MqFdFSty1LlVQw4U3uNR39jZ7IiIqmq4AQUbeUCTi8HXwS0pCBDdiewwe2QWKxxw2RQiyLlXPhX2n7mI7+R2bKqHTfHwbfPt7si/qdBhPtK5OVxIFs/TNQ+WXnlA5E5MuRtb+jO8emwuq314xqqbJAeja0cw4kQ9g07f4NE4f2WZmx3akiTD1p4ibxVewvL2Bphb2lJE2jYZBdpwAuMGUF3xPCmlM3jaEM89zg/Ee0h/EEyFvG5ezyT3FO4EahyVgqzgMIExm6bFq3nRjdtqcKEznRHRMUP/4Fr9YGbt2MXF8j5IjXOQwwIlu6A2i2HB1YyQ01hphNmF3QXgduJpWNuE0sinud+NCS2KginyXAHEVJ6Ao48SY/8Nh1ti2yAj/C79N4uIIHUgn9cmsq0BmqshXWUW+/HCDVB2AKlgLUOdC6kI2cydlm3srEsRgNnoy9BuLtrRZzldIqIGG1wwfgjkXr9M35dfZk58C+7UxfJIJOzrA45nYfjTE/8SDOyjgj4yzHzuDp5+5hkkCfa2asfp1Clcy53B92fmirlvjyBEX4j8ndOHjhDqTCfLBnuL+pMZvAHOv42aGk5UtPN962JoSopoe+pkH3+AsLNUf4oAFTZZI2jXOMbSQzJpm/cFHetnkVz2MM1zFxPR8VuhVIEXzHJIFhySBSQbnHevoQO2cNHu7xfzXU8p24ckwbIGJ0MDZJalXiXB4CUO9wr/kFMdttBMI4gnA4JOCqKzu7TM+YC5EY1PrHDDOQbusgl5cEsFESp/5Z64snlwYR7Btb/9iXituimyr7H+kdjLtNJ9BesyYEd/iOzBLbk+pC+X7HcdteJ/XdnnPrIkk54jCITLKyNIbz+WWNv34ptIngRNH/bfuzUqpDmtYp1KmczWQO9eWEIi2rRMzDElu5EkeD6nHa9axL60b+KDyOkPwvBjSPFirc2Ke5ujGaM97cGYDdlLMbjy+O3y0zxeBr/GQWe8ssngVrLAjlldSJsmP6EU+NfLsldyHCQBMKY/KEgvR4QfYGsDzLs0ieNlt4u1pNUsUBmIO/4Q+8X2m8rAfQwPuA1WBkDhZgyVa/i6x/OMUoTf4ps+05EPTYYrnyGVH2D5+HA69mzL4YxhmFXFOEK7Qru3oO0buFRh+PpTJBThC9g+ALKXeogQaWo4kQiDZO/Yo6GAuNBmqDq3d78Lf3si7cxTVKWDRVPI+YBtZNlcSI3+pppLsHccCdJvuHBS7YCL1R1BF+PtK2RsLi8Na3jMFAB0UqWQwzZeg4Z89ty/hz8e/INg+R3G9BxEtX4PICRo/6r8t2Dfv+Df/53yL9j3b/lfVZy9VkKvnyC0LbMvWll35QlPnWcSGbgDS8fvCavvSnLpo8CfpSaCqlfxVJgwdq/aoF65Sd4x/DTvnxUOlXfLIgUjpOtXMOoipN1PgRSOyQndqu9j/eA7Cc7z2bDLCqVOA9VOyGmIYVuucPbRchZow8mu6sAnNYJdEeZqIjaWLneeDFnien0cx+31zK2EAsVHwrF4Bx/FFZDpu7l0WGB1FCyX0B66hfHBQsIgrWoiKdpkwSLddyuEd6E+YTKTTk/ieshquiesQWXJ83vmc6XnKNMc4b74U2waPIkA+1lvZUMBo7TlpKnhdG0Zew+tYZ31r/PrhPD3juPfgHv+pt633Gu2smiPnl/r4aoxlu1FM4Uz3F2sfdcz1E0Cbq2BJM4KuRkQTtzE2zhsuoMEFcyOqidKc8Hv+oeTnyAtB67bYUt5S2yaZJGgekt3JAlMGuHoO2iCG8HT/HJx2dQV1AQcp17rH5UHcLV9BTPz0z2/ZUmG0n0iX0NoO7qqZgOQZ4cTJQOQtF5mKQ4LX8RkE//2Asa8+jPDA/ynaVP8GEYWQpabefmnnH2yjpdK41heByNtfegR9RW4E5/LMhTUNWfXjfFU2DRcsPeETh+KKDTEN/TWwEG8PVAYeNaQvoJ9G9wCa2Rv7i2G3YGrWRsRy570DT5ttkLVckLivqdBd40jFriiHiVyvgCk3ct+7SziSh7l8YQTfDSsrdjkNxaXg15SPl10kFr6JOHG3n/OhYSTL80lpOfZyK7r4Pl743cfVTsEtV28nwo5AEJbQ30emcbH6Ri3Gb1VSCbIQBulXrA2j82gR+TnfveRJIThqA6CG2tYPiwa/ZRLzH9xPtJN80i4q5oUFegkIf+SYPKy3yndy/vlC3nMTfabEAij9Pl+51fnvc7kVo2MNsXvmU2OUN64kc7mBghu6ICj6SwYcQK0EUiygTybyNu4OWkyXxlrhLSGu+y09GSrtQGNLZpNZz/gapMdIo9l92/BWsPTPvkWV8Rb4FcfV3vcUD7b8S1JK5sx9kY6rTX+84QsyZRFlXGx3WmWxeVwqNa7EaF4JytCC+mnB2OgkT+sTuaVe79XSYLvbgzhtUqoaVvLV/9QPu17vZ62/UdzLvw77ECk2oZB8jrCUbScs5RzPuJrtrdNYFXIRxQFdoE7qsWmObo/LVSP0bRIyO68VtpeSO66i8pcTC+9kIBqfMabn1lCYkxEPs/0uI8AuVREEg3aSUnwI7xamESpAyKz4YOauzw5hchbSby6kFD35W4LhCane4AuVjg73X3SuGlZV5sggL5LCwEIrl3DoLTvaBO9m5EZnxNs9XcHyu4Bc9wCC8sDQN8EAlJEZf7vLLKtZUyAALWOVGXc9E05eVK3n6nBjVJ2Wv5J+V6vp/fA0azu8wwfR8G1FBgg/eo9oHAjFzMu8LhlIs+Hwf6M62AtF3JK13/EPKqChy4J4HlyMTxW8o6Qyb7yCeiiqet71pNbpW3O10Tq/GGgjJK1fBgFZyywrqA3DiUcMmZAZE8RnVp/BQwX2GuCj6/c6wFVG/v623FNWOIbuNltCTS5BZcunihDLvcbDCLXAtA7MECwHuuuIklwf8v3uT8YWuZCQdAdEJQu8u5F9WCgdSEjjpvJKHxFzCEZ7ki3rM8gpBVFF1aSemMeX/V6gkT5e5G/aZILBmyEjW0pSRNzRH89BMp5Yg5q9QrEDKQgdQZT3ctactkj4vpB6RA7mHpXAscKR3HDHEq1U+JwcqUgMtxpgrKDvDugM21TP+ZI0zGcjn1GzPVt5kDccCqaL+d4tXed6q09InInOsxwYR6Zqq8AiV56+D4W1PU+m1N7LV30S0mN/4UN8bAiyRvBCxB79SMKUqG7HnYlwKjYo38eTO7vzfZfgs1abQjZVR08EeiBlSvFeyrZ5ed0XxQFeS33gMWLpDYSLpySlaSQM7QwPQWVbuKTLopKxIS9qwHmFzXxW/NxOVm6cwZBEds8D3Dz5lWRVAzSw1vdniYl+JT/40r+UQ/h+nAhbxY/CjtBOF3+rM9KU6XfueKOsvf3zmFw/GmIGUjiNQ0rjDA+O5mX3PNcY2nMvVUddIAql40Gp8/OvfwQTytZtHHblW9duY0ryQf9zh9Y9Bnz3T60NR2+Q1/6sxhH1mqKTS346exrXDYFUK+9hsNlF8zv/N+IMm7DlQEvhjnoEr+etiFeIBNNKHucIzngFoHopoMUs1CRIHoABGdi8ZHil5CFTGGH98Bcimy8xvNhwikXZchHRwm+pbD8LOW2nQQnLGZq63cIVY5B1y9hohPihtI8whuNPP8WeHxi/F86SbspsAjYwj9ToigvKyRccREgwdIhI+mnn+Z3bI48lZ/PvwruSBA/+602i0m6s15JyZsAJIKbM666CWuM8EU09A894ldtiu7Pgmqxr0lUSahlr6SZ2lXLG5mrmKa0YY/6CJsbDnkjXkt20uCspjxoB05ZOCwTqpZCfY4gLbR8ngBTf08U0lflKSIPec4yqDlPdP4z7Gl+kZ9i4bkwyJD8CWrNjM8xvdWHdNPBgbRypMLfhDOx/A9If4BTyTM4YAYbUGMPRLppLergOkpfPaTHbWBWjxFQslP0nQKVpnguWsEzols8B0MPQauZKBgZnbCVrlr4sQ4u2lsJybUNLWFtIrJPvtJxgT6MnMqj0OwJvsgX0WLJpTP4qXAn9FsPA7ZCSAvCjnblvfgyHgiGqcEQ6fDaVABqSxnN1e6IjoRnxfca0VUo0xiaeGyMPDtkRcxg2mPP/EcnfW8EeLHP6fwT2LzXDvucMG8lLDx/J5IEfZN/4q7WbzBj4wVuBL/s167GZ9ZLECE7hVrJ3ltEpGZ9Hu1szxBdPZoN9XDM2lnsoxQDFG+DZo/zfk0iR3QHuWpMxOwMF3mlRp6D9GmYku5ieqkY2Q/HXqVa1VOQ7po+RJ0rhYD4xRzrJrG+s0Rmg5uwULoHDEmsy5nNMTfY58KBM3OmWMNav4IU0dHPWb7BOEjkbm/7lmccABj1Fyh1wPsNBUiVh2HHQJSGq2hC9lMQ/iMAj4SZ6CbPgIvvQf11qDqJRvaOt/XFXakNuV2AOT7z8UWr2LN3vngbdBCRz3pu8GCz7+mu9/ZvhT4NQlpB/u+00n+LSV3AqZQpHEu7nWzjef93IYunBThgdO8BU++Fwg08qLmO79ZPkRQYcRxGneda702scoNxa+phwoVhuELaeReJgFTa9PuGy+W9MbvA4lJQOSshb7VQlIgfxVuVMvVOkfuuzhYMq0Lh8kKcGU8S1NCGnhcPkFYswMoA2SXUEwq3eG6hOA3obQmEaXwk3PnzGqdT6QRhKuFWqM3iWPOdHkBOSF4OE6Dtlu7QINQnWua/z/mTv3IxdKmIcGo5E8wlBLis6Nz3f68KuueHQdwIqDgMNZc4k/Y897iXgTWtt+BUAmDIPghrj8kVw7nS/p6UDpKEAHJ2DIQrnzBlXQkv7dhPr4uH6GZ6XQB514QChFMT73meRTVw0DVagDGnZkLW57isP7O/ZRcAnikOQurwAeweLkgFeCOa1kbE8VHMPnC4pSMyn4G0BzkRuo1imzhGkWWhehPczEPClSRQOYJ4qxLmVr1PbGI6L7/yCrPnX6DH4EJCJk6EfjAg7lG+eHsxR0+cwNGzJ4/JGsYj85gssz8BVr0OM7qKNUM25fBtXANjAuBMRUfWXJ4FAcliXo/oQnXgGHJr2njA6HhDIaNin4TWr3Ih0h+8+q48jq8KL3hI63P3zeVjawdyIxd7SFW+RFeL3YJNMuLC4QEDG/uocXy5XCKaKC7wKuryLQLEufwxH+m/4oGLVwmwpLOpk44tjie936nDgkryAfe4KbKv1UssjHmWTusfRPvNXD744weKzd79a4G5Gx9WQaUDPqqG23Leh0jvqn817Slmu00yyaUIIFtWCZtlx0CWRO0hM18QmgLMGeSmb4XQdqAK8BDxG8vNMp6N38y02GwGBvgoIBgS/JTJbg5c+LvIPt9iU1fQNOQKIdwQUZC2GgCeLJrvyefsAcElWUQgJ45Hq/iuxTJmbUuR3y6gkTwvXljTopeY224OTbXe4zvFbWJT97kMDqrztrXsDzg0hVCXIGN3uP4NYwKgq95nEq29Qu/IDwkP38P2dk0Y3vlutGce99bnr6N11E7ARYIKNqXOE/N5t68FOQBY3ZDMr0ZB6rvZRndqwylzGwzbW+4nVe0mCO0egSQrfHViIafN4jkm71oqot6PzUDCTnZVB66pr1AWshWj/jz1Nn9Cnu/c95RqORx9ROShNgsVkCaVd6M4VQTKoPbNuWwp44GwLLrroK0GlidU+QU2mMO6eub8woy7USJXe5/LVg35v9GPcUzN2U70QQsfHtsj5lbE+Pr98lO0zYMbdkiSdfS0vU/72G3kmnqJ/eDgvRDdjzYxbegeFMpnMVd4KRw07jQsjQFp/7/KdP4L9v3fKf+Cff+W/zVFp3NvTNcmw77xSBJYlSqKQn+lLHirdxLJ+gz97rboVXVE1g6i3fWldJcf9btWdezzqCXBVh9ZCIcC/SWoXJIDhywMMglJRAc0nSrYRUm38WnA3ZyxglNyUmxMx6X2dwglKiZ31INLGPSxg6H9XJwa4bhU2wS4MjNwBPys90h3SBJMOzKb6aXwcgWYDKO9100Yw6T8NM5a4McYON/ykJC3cDuylOJ1DA0SbZ6cvpy7NUFiox7eCSqPUho5gp8My/lWtZGxK5yY4rwyDAAjKn7hhxjhOP62OB27yseIjx9Fz+peLKsDjkMf+T/n1xnL3yeKruG/Y/HbTYGE3ZjNVu12XivXexheAEpgMkXuRfutCJjsehFavgB31ELaA3BjNYr8E9118FJUA+Fqfya0P6tJQiWZhBRK6t2EqPI40fVbZofDxgbIjn5HyN0AVJ1mTec3eDAY1EoD3WOXQW0WjM2BoX/gUgV7jOHpSite0z4vNgYnn4Ww9txwjvTc9mzyo/4LoKLjsdKmfCXXMj37bb4t8HecyJKMRtaRooJvglPpEnrRrx5Fww/Vqew0wZKLjzL7bCXoEyguLub7797k6O4WVHz3DZELksjZfpjiYq+bVZZh5/V72XH9fgAcgS0EM9zQBEIy+aEOShyQdf0pTlf7OLHsdXQonkzvDt4xdXNetNHxU8kseBuNy4BKqferQxXAndLdTCuBCfH7aRuz079PCjbS3TGFNLe97BtBJ1lLGZS6lJ62TrS6IUCSUNkFlkqw1ZFkXkRmxB8eeTStBCNKlnpY6xdrx/o1RVLpYPhRaDsHApI4WDiRitwIetWF01br75KebPyKDfFC4mdWODSx+DjftFHstIeQ7QZlZ4XDw8H+4y+85ldeCoMgGT7J2ElI5VK/d+E17F1+QI05oB0984UUU0XwbsrxdzR5xrUbRBJM+hgBUA3cysQ8AeYazE2pqR4M4V085zqcDowRCnbFSZsdA7g3aKvftXuZjnNHWiU/37qGZSmXqLT76M7b60hXbATL0OZsGwbsGvAngLRxUxfVFf5w/vMcE8OGj6KtRkhWHk+EF4K+8h7ksFJhW0VR8rOYNQUURP6A2e52dAY3g8G7yLfPIK5yApn586g3PyMiE9zFkvEYwdegzAE/yLfRL7DBrw2PFH9MXlI6n1mvcEduFBY5Fs6/A2dmgyrQY+A3uMAmaSC8IwzeA50+Zm7SFG535zIotEND5HgwePtMkvBsqI0OjYj6c7PoDXXCqfn26WcYs2krF/WP+bVrgOsn5kbAHhO8XBImcoGMvS5ApKQ72KJqxWUbPF8Or1+a6P9NSSpOh2/mg9JIJvWHA07LP34XI0ZNY8nV29nSAFUOsOGVriIwjYNlj1JZ1Yt19fB6UQySOljIKf1xD3LNCSRHkM/zS2INqzkPzZ/EcHIizuY2FkTCW+m7CDOf8WvD5YQHmFAEe0o78uXefdg0idBlEQw9ACkTeVwzkV8bxKIguWS/70aWYUlJKhvc0099hgrOvQn9fsMR0ZeMiKN826SBbjp4WLub79oPEazHdRkomPn61FwedgcVJAc2ExI/F96DyhO81z6Ax7vfw7Cu43i3f4xgygemC+BNFUCEuRtFYat4sdzFHksHIQV2+hVwOjzOn6EGAYylScuEPND5t0CSaAjvwuYGiLDH82jTX+lQ0Q82tAFzmef5treLZ0uHMIpstUKS5sQzENyM7y4/z0/hb1Eaup4yw15BOGk6DQZswqWNIbrGuxbtsPSH9GkiT1jMAE47XgOXxC6TyMVlD/aRITYk0TY3gzcr4fs6WFntL7fq0oRx3Qb5pgA+P7aIs/kT/MefpRJjuo1PogTY/K1axz8p3+v1pLeZiEFdg07l3v3KOhFpK2tR1edwbxA0UYH5yqccr7hdEDsAbLW0rt9JN53ImVbYvh2plo88krqYy4lT6pCAHSb4JGcQki7Ce3OnnZcb1jI/UuSUHBF52b9xRdv4IMJCLz30jN1FsKbMr7pjzDp6GsS77nZtHR+P+BgM8dB/PdvDa8muaed3/IUyLznpLzfV9dfBVOSuEwPhfPB6jtuO+R07Nvc1TsXGUOOEViU3OKy701tZe4WxSiEDlMH0uniIgGufY9b7tyPSnO0BoH4o7II5ZiLcVg69f6bE3Irl5+awNPFWdrVpSpmpRIAle8cRaD7HL3VwyQa3tXyH4dH+EYmNIKTiCODDPds4kXwQfkuB3OUw+iLfVHTyHDsmpFzYUOubw/l5yLWXmB8JndVabt3zBRus/tKjB1KepesNAQbODIdI5ZpwYkkSGHNY1WMyo2wzAYhXwcNDq8nNz6fvyy/zapQ3YuJIIvRS/qEShUbDLys+obhZDfMjwWgLwu4K9DtuuzWROdZj6BULD4dAYP0pb2XdZe7THaOD21+2IP4EXP7Y73xFFjlfBuXD72VD/evca+vtgZCbbqVpoNdZZFfCeW3XZn4sbkmABFpsIn8YwOhLDDGMo0/3wZ7jU8veF/9ImgDGbFJCTnjWKUmSxBp3pxlaPI8xQqxX+0yQch2OqJ7ya9fFoPm8kp+GVoJIxSnUExLvgPgRnmcCITXWPKAUyemfd2mOZhF3FcEpCyy48DJEdne3A3QqI9GKjzMjIFFIcZbsQu2s5JV2H3CHe7mRkb0EtOh+9DFkk+L+uTLeR6ni2hJw2bGXPE2nq6tILnuIYmtPEYEUNwQ0oZiTnybbouGgGb6KgQ54lSMAIi69zaUU8Uyt49ajcxXBsMMw0Qaylp5qI+lqQdjaXdKZyVPnecbfrLAW3G4I5pFI2N8VkGCj0/X3YLMdZr67iurqYppHHGJo+jdoVf52DLVZzI3cw1i9jreUAeS1yBOyZ2kPCKKQuZiUQG9Upoc4ce1LIWHacSE/G0PIskKPslyumIeLPOahrUAXJcYFoJfh9eTTxNtXQHgH6LoYI2ms6PcIL4ffBGFnLxX7WCCx4n4A8qK+EnPa7xmwqSPKljbMaP2B5xQZoNVLYn+LV1WkNGQTADZVNVLCGIgbjuqP8awe2YfCeEGqe7UkhF3Sdhi0CzouhKzPGKHz2vWLro1DZ7kAW9w2ccI4JhTGeXLFqTF4jjVqOnL3ru/5odb7OIeTn4KO70PfX/muMovKoL3kRy6lOHw15ZW/wra+kLcK4/57eGOgllNp9yA7dfxQngpjcwXA0+oljmauJvPCblR2IRXpK4H3JzJaY57XzOfgtgoISqdjXEei64ZxaxFMvzwEjeUq7L9dyC8njOGtKi1ml8gDbHEYILo/dP4UyVLCYx1eYlhIBcEmoTijlmShnpD38z905oq2dslaR3pohpBd7vg+KFquWwMpd8D9wbC4yXFBDAlqBrGDse+bwPweevIivuFKRTdKA+4ThISN7SDrM14KuNMjJ+3phyajBbgf1YOoxGGcNIu+2V2dgDH0VpEb2yZOyo1czPrOAmyuNJcLtYnOn0LSHWIuscUxOe44T6S9Jsh4rWfD0D8IbdjLdDdp89L5TykLnCMAyNjB0Ho2Z/HaUEllDwn7PmmCmItstXwRuYkXw2B5QQ9Omp8RBLTbq4RtGJzB95VXUQVc4ZdY6BXqnbNxWCg6+hx1YSOwKwIwaar1poyRJOgYu5Ufk/NorYFg1/d8/uYTdOnYEdXBgyxyWlmDk0+dTvrkw/jXYe4hMYvItkomh1porQGj7jJWVZnIYRiUAVmfC3l8YIT5O35v/hyvd3yLLWULoekjtKq8neW9n/S2Ax87+8BkBhT/QKHzFGZNAY254X1BqbRP0vg0JJM6/fk/RfY5cWBVKqhzFWNVldM95VuCjg0X4LukwikbkCVwuvMeOyWNkNvtuxZcDiIVf8WhTMchMY+U7AZNKE8HVNLmUCJR503szrmbOocXyL1uGc7T5VDsAKdLQpIDwOdbq4zox/aGxmdW0MsVgjxiLoXkieQzjabFMxl9zMWAc1cwhw6Bkacwhw7+i8g+fxnPMGNPUkuepF+hmQ+r3EpPnT6BkNYMDfQlRit+36DqZinIm8DACcF6lsVCqgpmVcDnymExdoMywGWnVgnF5OE0uCP7ZLUITgCSneU+15ao1/cU+e3c5OLXUvfyfJjYwybm3AFbvWuzw6XC6FB7iDi3Zt4KMf2h33pq1N0ZnfEJs1p+i/oqzK71UbeqOsno+KextBbf1culOiydf/R2wZGHGNh+KLnRn2F1wUlTKgQ2FZVp98EkF3PrMklSwavdH6J91E9+fVLdfiG3Fol/G50qLlgfFoSVbl9jjxrB75efIssqnE05xnTovQoG7kCK7M4ru3bwc43YqxxqPpiwfelCxWf7ADTW6x5fgNYaT7yigaItghDa8kVkGcKNvVDlvEP3c30472uHB7egS04mn9YIe/TWYLOI8m4srV5ka6DY/38fA7/1fN07DiK7Q9xwAAZ3G0Fx6O9+a8TNxPlolUyy6UOahR/B5ZKhzWtCer9xTLqVheZUACrRD/9JffLfHH3/vHgwjP8L5V+w79/yv6IoikJmZiZK6Q6w10FUH2IDvyOrSzjHm97GmeSHvQdrwnCGdsKmLsKiLiakoT3pvrm0AKuhJV9Uy1x0G/A3G891pdOJ7jAMBdiQcgPOu2Ubsr4QEi5uwyWHCp7duRlT4gzvyXYjv8fsYW4EZLZ5gHf7Z0C52EDJMvRMXMXxtGq6aqHCHgMpdwtGFY0SGKItYcYeZAT7RM0EJJNcc4XkrJ+4ZoOzpkBhCN+SDeNLsQy7yuwC4XA76qjkvP1OwSRtNxeSJ6FvyCInBW4JrWVjRy11Vh+rGQi1V5KmFqHvL+d2xKHxRkQgK7SM7IELyLgEUx3+cgW+ZSp/nyj6f4r88y0lgEETTogllerAwxQo+/zqFXMpCe71afvl5zitnSv6RB0kksK3m8udQceJU0GLHMi3+Y+DqPorTAiEJ0LhSM8FxJuXwqUFEJCCEw2XG8IpdlsqfmNEUrA6hREjRWxiWqu74eoX4j1GdCWgZjPtdMIqDK4dhEuKhMynoe9vkDaFo64FgEhSnt3tR5QqHya9rGJjfRRHLeCUJLdSt7ck2/LJavcEjxYtZ0rMdTINpX/qt8YxZHGqsbjCWLjwQ5ITEshbNpcPjJdYaSlkgfEq1o0LSW4Sz8L5QuJPkmDNpedZdWGW57enWT7P/9vVR9hQ+qm3UhXIb9p7PZveTTFB3GF+wFt/7RtaV0+gJmoJU1SL6HapvWej3ngjRVGhlmB25q/0T//UL2cftRdIdS1noF5IFoZovNIaKtM1nuo+hQ5xW3AiDP1XpVOwvhkEN2N/VCG/XJiFhITs1GF1QXk7AQqS/iC5pj7+fedrnER0YWphNGk/xjFi8VTmRW/xO/akrheLa6HKCU1z4HyoDwAa0oIXLelsdG8OppbAZvsCv/M36u+iXR6k1I5kZOxx9A1eaV2Duo5fM0/xeAhcjXsbe8FKuPgBuJx+EiCPh8BvKd8LOUB3eTPkDRZHiwTTrTrcSnCJD9khqCmXLAK0Ti19nF1nN0P/dZ7qvPwdxHVPo6D1OarVsNXsL0PWy3Sch4O9v/2+i4Sx9Klpxrp6cCgO5Lgy3oryd4I9G5/FekM/zIEw9BYYrVb9fY4JtcI78+fTKvgyp5NhiAGW1sIxawfvgcarfBrxLk+Fip/bm0CLk31EZPPmrpD7s5CoNbWnafFMWku3+92n8RliVHB3+mpaBN8EoMcO4oq5E5dssKEuAIcUAHYj2OuIMK5kRdoV0tTQSQtxSoXIHxDdF+JHsrH6Xk65FQ7/MIMp6k44+rBnTTgUMZ2zKQ+RUfgq74d2F2BMTH8AKuJeZfQNPdtbdmZ906Gcr/dnhreXz3O3phE4vMmqDmvLKl0vz71x3RSFJEnUGIZx3qrQIgLevFPNGM1/lrLzzfcRHt2ML7Lu5tta6HwDDkvjfO7bjt+LPuV8dXO2N8CC0mgkTaBgMYe0QjEXkGkQrNKBeuisPS2icMcVQEhrlDohCZNRN5IxacvQWPzHjzG4JbtM3mhIzzO5cy40OsWmh8CHnd5GsnoBF0mC2XntWeKep/5whEPxdjjyCLIMl8p7cmdOIvtNCKC84weQ8agn8vl86WCKS0cwU9UJg+W0kGI69QLcWMOl2pFcbAglJ3gDZRghOBNuuSqckmdf57vRoSSoLXxQDfmOZnB0Opx/G8rF3H/edguzKuCxYi0l8lBInypy5rqcnvEZKDu4q/WbhFr3Qs05+DWaNM1v/DxBwzPhZiJkyKz7SFw363MITOOX69M47V6uNZIkHPwnngV7AzEnOvBo09VkFM4msnYwDVJ3EWlWeQxyVxCunKXn5X2oCx7n6uW3cWp9aDqyigqnBqNLJJtfXOmN4AWoS3+E3vmQZXexMWsG16o6+W86ZQ1Zpl5cssL0fnr2Ocz/GGzu0ulefr49lEe7PAJAQ/gtgqAR1RNN5RG+i4V2GtiePY1l138BtRtwsZTTt+pLbnUzmTc3wAKWeKXoTs9iseFHRl88zbCTVbTP+c6/AZLCb7oerKmHF8Khb3i2f33lcR4OsbChHiLOdORMtf/a8nCbe3g+ttB9LX9dHMHu9nbQsPBHuL2ld576y030mCvQewWSpYSJQQ4y1JCsgnh1nd+xuujeFBaPQ2sV70/2vVjyBBhfRkT9BsLqu6FX1aI4a/zOX9v0I0a6m/15zhBc+iYil1LdNSQJsmLn4nLLesuSLMZQj++pjryLCcWwth6eqC5mTpEPec1UwjPqV3k2FMLquxNVNxiDJhr0cWKd29ieQJ/8O5PCSoRtBsK5Ed2XeSeOcDr/To40G84e49d+bW50pm2sF2tzrmMwnHlNOGZOPA37xtNULyzRycXQoyiW2NhYXn7lFaY/f4GnHpjPd3OCCK2Hh/6DhNHN5V6TiW3r1vB1lY79ZmhdcZp5VRl+x5TarlMQsYwoRUTnRdRs9FZG9WGO7THW14vZfGxovmDhNxZzORPlPLpo4VL+JFzGSX7XDs/+jOOJIlrsw0oV1fY0b6Ws5WTxMK6aQslOgdcjlomo5jZzILg5rTT76eJD4j/U4hKMLxV22sa2vN2nK41RSN8knoZTL4KiBVmhIfIOIk704f1qiLj8Cxlho32bRbGmN8vV29lrguZZUUJlIPlOIVt46UM6XV/AEAM8HALrOi9GafAnRbUI74cd8VzbCsdDkHDsSRLc3eZVStIgTgXhlg4ip/fmjrBrGHZVNLN3b2J1QQ+uJsN9gUvE2Jnkgtav8VbMrwwPEH19T5EaY/uVYp3q/i20mU2opQ1x1bcRbGqLSrJ4c3YD5qav0P5SJlvq4YlD86gKud+vzea4EbxZAd118FvTK0Tat3krG/LIPP8oj2jEvk+v1aNWK57xd/v9Fxg07QFaPgtooOd/ATZv3fI1351+h/vW5tM2ZieBtlPeg6xV9DdcZWz5HGpyX2Br9iMigrz7Erg1H869zUMZQ2mb+yVnQnrxRvivIhqsZJewSVaoONrkHPMicYMPLjj2JKxrBvYGAnJ/ZEWs+PPz2w5Qp+oAZ98EmxGHFMiXV8ezuUGMos8MPoSEuixe7NSXSW5lmuqAo0iF68B4FapPI9WcQ+MMIKhBOITvC9kk3vFxoeyjUsHR9HEY9RforBU5dak+A0WbwVzMsrx+NM4kl+tjKVMGieeJ6g3aSCRJ422LS8FsaC/2zv03Q/dvKLML+X6DBIODij0RWy45gJy6DKrcF39Et5uBqQPFD00YVsL8Ij0MkhPqLoO1Cpuip8xpReUIYtKF63yRckJEiwEYmpCSOJ6Qun6oHWKfpJJccPkTyP2ZsPxVnkjsbjp4OuE0sikXZEUANqYi2NSB+xMa5w4Jp6Gp33hRySqaqKC+KTyRsVqkoSg/iPrcs4xu9imZkYc8UVc2WQuDdkOb2UgSlISs94Bme2r810hJgq5Z6xl8upDI2iH+gzQgibtyerOkFlpLYfQNqBBRedXnoO2b1IR1ZVODmQuJT3uuxfVlggwVO4RPR33KXL2Z+IqJDNTDu7EthJTwpQWgCaNZWBrtSuYA8EFeZ0zBA+GXYFibTKvg1ZwZ8gh3uU0Bm9Mi7IJmj0J0H7QZ91HXsQXJTZbTKeInAci1fR0iuxNR8TlPh4rz4qpuI0zXDNrPE6DeuTnESHnitZnTmeTqAzVn4cavkLUYgFilnnAZfinqzFnzdAEg5a2EHAFCLL7+LMEy3B4ECbocb385rcRlfUDnkM0ADDdAH/1ykbsWYTPEBF6nX+xBohXIPGbk/Xe/+8v0KnvtcMAJn33ws5CbDmtP6IVY5lfBkC7jeXNwa0FQs5RD3RU65wRx15BYjqrfJ0XfCos9iDxLf7DX4VRCyKr1KkL0MNTROfhTAawarxHpEiQsp2T7y8g+jSIGr0u2ecC+xvrChmy2dojk5+hmHGo2mI+C53KiyVPCt9LsUY5GnhIE+8bzJLWIPrY3wL5b6aDxAmOhxm4Eq1RiHqnPEfKpBeto1ieHGxltsSoV/mRAn73s5MtnuCOwrVhPGut9ScYuhTRlGWzvA5UnodVMzvGa33DXNxwG43UUBb/Ivh5Z25nUxrt2SxIEWprTpGIyV23wVmUN3GWH5o/BmVd5N/a6x3aTGgE5d1H5OUrwiwIE+LGJnklBMMJt8zqlADG2L38I5+exLO4xdjZya3z3iI4G2HcbXc0+87SbPFlcXMxbb77Jhp9b8PbcU2yYBxUnd3DePhhn0+meZzpVPIQh5wewrQH61n7BGwPeEMTxJqOwq2JpE7OLMfE+9k1jiRnIfbnD2GkWG5dDJjWuGC+xSeq6mNfc+VNKHfBq6d0iGKOxmMt5K/g0M0LgjiCID/QnQ/sSJ2692Jv91i8EYSX9QdDHUhS6hnqdsEFah58XdkpEV1C0f5IpLja4CVM1Z5FdDT51Tkqs7eDWIrGHxGvDp5c8S8/LexnVxCfgQtFSYNdR4xT7koALCSLK212iA6J5ucMnxFSPYbURfrox0H9P0H8DC2q78bm5mCZhfzCh2YtiXLrvmxF+lKcCA4mQoa1xGweibrA+6zEB4llroOq0V20rZiADctqzoBoUlfgmfCP7NmVt4usTX/s9k+fd3PT7ZqDx/9XiwTBu+l7//1X+fQ3/lv8Vxel0UlFRgeua0C4nfjiPJd3PRDdj0xN+DnD5I1B0ZBv280dmfy41efnPF5Q1LKpSccAMs8Mh1eIPIo1gOU+Gitw8LbRWKNwsDPGj0yF7KcPMe8lQQ0nYWgrCf7rp2lqWGpuyqUFsIgUbVjROkkQuBa0krp1tbgc9fxCJwt31U9JWMSsMVqWfJ7reP0au0ck/uxImXm8njMbAVMGqDEon4oaQNflavYGFdcPEwho3FEyFxF6ZQ7IagiToF2BFY/dnnf/a5Al6eVQGb3IMW6uZnTGeVFsGtQ1/H5kXC7wDjOGvAb+xwHd/8fe/Kt/rNDw08DITkrYxPhB+SDopkkC7i7x3LHvcmOTl7OcoC3wB8n8TDp0ri+D0SwDsNwl2uV0K9rt+Rtk6lsdCqR0u18dg1mRC61chsjsmYpl4bhyLa4TjuGXR3cKYBQhtzb3H32FprciZMCOvHaRMFpJae28lPmsMj8WKztxS0pXd+hOQdLvILbCjP33kqUQrIp/fJqOESx3m165GsE7XbAa16nf8O6XyOEnG+VSbY7hnTTH7Sx7xr3dY2Zx8iHciwBa/iMsnn2Xeyy+x1+Fgq8V/A7DNZmOv08W8V0W+mZsXan3h14IVV3UaJe8XclOgrx7sSo3Qy28sipazun4cNEN09Sgcxfcgq5K99VUniTL9gqzL93s+T3E5aU0p6WoYnA8flcb6t6X5kwReVYhU4OsYiA7wYTwFZzJnz3q2VCVwJvVBAEpCu0HaFJDV2FRx2JyCUaO2h+IA8iP6CwZs58/+9Mw4bSJap0A44a4HCukqu9HA4ir//GEnAgbycbXgLl6zgU3lLwfsu3k4boFC2vjVG5VILtug1KZm7AoblakLvV0q2ekYUEOqGoz6S0ScuhNOPgdI6BzFvByqoYsW6l1Q5TAIhrW72FwaT564volraWr8xntTh5kQRWzA/kriQttwnXciwXH3Rs50/YBAzVt+9d+H3cXtxQKoMaZDr4hf/vKZA+oD0Fm0DA6o9KtvG3aeflGnMNoCWN8exj78PH0VhaF/IV/VS4JZ78zn6WeewahpwSvlcN4Kr1XCFqs3CgFtJOtN/T2ScCctYArqDAcniyiTA3dhcs7kQsJzVAbu57HEVNjjjeg0WIpZ0nIKQblzmPJ7FgdK/SPB6fwJq6qeAnyixRQtVJ3CYD1Nn8BagiSRG+/+gM1iDJXuBaeNnTWvkls63HMpSXIJ+WU3mx23lIfkUtEscJdwAjS+R30LKuwy6WpoqQED/k74kxH7mbLnB+4MhDVJZVB3TeQHWC6Bw+LZ1LwcBlOSd/zpXe8pXY1FXcKckhDy064xa968v3wX/VXQV5GYNXeuJ9+HyuGdS28my1xQ/ci1uMZ5y72WdPwARp1DOTWD2alig/d6BDwYulQwSWvOw17xTsb85GL2lVvos3sBpcEj/a7deK9BUaeY0XM8akeReN73JhEbAAEAAElEQVSVAWAqootDAK8d7G3oEn7BI3Uq+h6/XAiO7itA1lJ8bDHvv/kYWzYNYsO8fGo+gl3b5pJfmA9dPoXbynHJOmSXhqFlrzIv9TgR5gNCPrT9u9BqJrPyJ/KitJKF1TDoaisRSVt/Q6xXhgQu1KVQrhdApuL7Is7OhsxnuOboS5kDFldrqJY7QdUpYW9UnaTZuee5nqymxGli8o5fONykELovhYRxmFwxHK1OJc8uwOrMuoUQPwr6b3T3lw+DGFnI8+T+BCsD0NSfQq1YaF74Ot2vbKNDwCjh/BpfCl0XU+7qRpC5Ba1vfExG8Uv+g8dpI0Nt8eRtvDlnVePYcygN1OrOeZQSPEUdyOLiRXxaAyEhMrPvjPyfwWaNhnfmzycwJJEtV6dyrqS/9726iz26H6ML4ZjFGzHrKbo48lv9xpmjW9Fa4zG7oE6KFrJZAHHDuBH0AhpzU0Ylr+LtgQOhocB7vqyw29CNn+og7Tq8n+1PHqLZY6TlBnLaAjUBR/8E6H174WsWFguDxSW7665+DcslOtaMIDbAY4AxIfY1PwfZ3zFkpdqL/BhvY5JtKL/Hw4cxm/zqnZ2/5LNjXxCqOHgmFKKsp7yVig50kYIhD7zefyQtrjX3Oz/cEE5wQzuSyqaRUfQKqvpLIvK0aDPpIX+wpNOHDHUHvKhkRTiTUu/Bofc6BHdyjb2enFwALsJkiXbFr9E1axPpYSfEtzzssIhsrj7NLYYI4ismkVLyOMsvfSaiTwAie4AmjBsNXai2a5kYBK3UuX5tjjOe5a5AsT5es4FdCoKAVFGZ/iD0XEZtg/f9ST4dLMuQFnSd20LrcFn+OyWKOmMdjxWF8JPbVLTiP+5vVS1nVxOhkDA4HyqifAA7TQi9Ez6hyfXPcAEtL4yDHl5GO5ZSpilZDDFAVO1QAp1N/K6tB6KkQI5b4LlSHeU2bz5OSYLrUYvIi17MT3VwwtIMkieCyw43fuWV6v1M8GHhyZIE+8aLXOUBqXx77WFOpQkB/uY6oyAXlf0B9bkeGTIZiNcXoXZW+LVLpXjnYA+Abq0W8rm1l2hSuYdmatjdAJ/k9MelT/Q7v626nrYIgpKv7SjLcKGsNyvyBtLgBIMzDg64+7PXCiSVnhNFw8my6Cl1gMkV4L1oSEveLR/HjgZIKJtKzd46HBH9BXBQcQR+T+P5rmPIi/yak6n38GKqHlao4ScVFO9wKy/I1LvgcEUHLBp/MMXa5FZeq4TTFng6L5VadXdYHS3WKk049k5fsbEmgt462DNoCkEli/3OfyNzFSviIOAcPOQfrPIfy70mE0f3LMfq0KNRzDzb4x6iGlZ4D4jowhe1Tn69+AIniobz1enPRfRRY8l4mPfLmrO9XTzF1GJ1hQk7J6yjkEt0l2I7zAmHOO0pIalalwVZnxFkvMydQTDo3GUulfckwJkFZ1+D3SOJUF1g8dUJnqh6WZIF6DrJBS4HsYbLhGgEC0dyycjnXxcHhnfG1W0pS45/Rr8LZxl9zEWJXchLNubOUxQ8KhYyoHeGizm9+3dIA7cy/8pkLC5ozi30u3BG2G+2WrF/HrST/Q7vPPV2q29RnPViHireBieeYWnsee4x30G8Cj5K3gTXxb5ckeyEqWsJkEBvSaapqh8xVz8S9fU3iFSd8QP7StUtYXwJNJ1GdeYsut4Qa1S1OYbPz66EEW6ZbHsDsq0cWXLQvOAtWucuIkYdJcDN828Tc+FVz3zbTQdPJx1HMfvsiQCTqYz6gMPcHwxjI64KELCxvy8vYlNcAwYJtuSO5Wpld7j2DeQsQzIXc+dvl3ivPIjsGLEf6SpVwI4BUCpynjl99n4njRv87itJoLVHo7PFobiEc5wzr8PJFwHo2TCPnpf2s3nPBV6/aBO28pWPofYyNRlPMKEY7Kpanr1FTUZBhldiO6Y/mpwfaaK9hMYexV1B8LB+L1hKhRynywlburKusyCESMjE5LrlEFPvoc6RwIbyVG64vyX5Ji/0+5mreDf5ErdXXeeOU+tEvvnclbB3HBWJ8xhbKI5/uftUOtbdIk6KHQSdP8UkCTKYS3IwtvlCOPaYiAIOaw/qYCaUTebFCriU8BL7LZ8J0sCRh4X6wtk3qUtrQJFAyoLVZT6EUcXA6oTneMu9hZoUBKP0n3hIB5IEm68+QvKhe9hZCh/8Auvsfx8BvN5qZeYLL1BSWorFpcIBhMoQp6kXawFAu3mUqdtzhhKKlEPcUN/P9A2XqHC0gU0dkDHxxqmZnuveElbB4NDHxDc17BBfht7l7g/bnyL3wAv2OSSLBwxsrFd76qzYZSNnrFAVd4eHcHeyeoeQxI0VkecaSRHzQHAmdPmCBqUHUTXD6XT1V3pfOkRImDt6qi4LLr4LlnK29esI08ezdFw8iQavvHyLgN9ZEwfpahiSupw7w3uJPZW7dD77ODtjRIin5FIodg0WUVvuSPNSexZlwdswai8jSw6SLnaH39MIrPgJyQfsC2voSkKwl0zvyW8ouYhUIER2wsGJYq2IG87Ggp/ofG2t576+tmCUIZroai+5Rr6JHF7bfAnra4LY3gCZamjCCSFLCRCQ7NlPxVXezqCzOeLv1mqRdzS8C+eDhM9DcQQQUdePNd8+THKTePbPfZt3qi+xuBpeKoY2J47SY8KbPLdKrP03y87fbIvLMrx/cDmh2S4G6UFvP+Gt1EVxyhRHRSPAdLM/MuEWdjfcJPHeWEwl8GsUI/TFqNzf1LqcmfgWQ8EaZoR4z85QL4dDU8BWi6ZmP8t7PcVYt6mwuNcjcGSaSA9kqeTutq8wzOBdkA/HzBBz6m3l2AytUByBaGxRWDTFHOQzEaldtAUqjwtRCe0VzBEr6Zs5hwCrzzM7bSSpbJ6UHwH2VL+oUrIWE5//EC5cfFULn1y9zUfGsxaKt3PMmsj2BlASFzAqdb7I++7u6w5xW1gYZ6SJCiAAmzqBBluIAPGKt8Km9oIcDaAOpMyuJlD6s4wnwLifxzFt3TTOlZ77ryP3/l+N9GvEMJz/KUTy/3D5F+z7t/yvKC6Xixs3buBsOQsG74OAFFaWzPdEzXgSywKkPYAzdgyySxgNpaEbKHKe8bteTPb97IozoCAcjqmWvX71P8nP8Vw5BNb24/k9VsGA1ccJRqoqkCGWvTTTQJwCD6VuRFPn46JStHxmbMVqI6wwwhNHNkBkN4qLi/lw4Zss+H4OHZ50cOZ9WLl3rJ+MoiTBA01/YWoIdAwworL7GPDZ3/Nacz3D9Sr6XDhOl2L35jBvNZx7G6nmLM3MXYiqEY5lWZKhcAv8lio0o4EFVVDngp0JYKj1BzgbF/3JQfBV04NonUXeyoqjJF7syejgOoya/zky72lgFtAH6C9Jfo7jdQrs5B/K99kctOjYj8OqE8Qp0FFf7ce0klLv5atqN/NEsvnJWaEJh8TxTCmBQ2aIUUCt+EckXo0Zx4Ri8Z7uO3U/dYYBAtC74g8CtdFAmHGLXw6HRqPq/2PvPKOkqNa2fVV1np6cZ5jEAEPOOecoIChIMCOKCQVUFHNAMQGKmEAFRImSc5Kcc07DJCbn6QndM52+H7unu2vwvEd/fOtd71rutc46MruqelfVrr2fcN/3U+mELWUxQtbo9reQtZn8xCWsyhJGUpnPWUz2AiFC798UjPHEyltopoWtlTA6RxZ/r20OG2fizvNjOCyPy+bxsC+VD6b+Y5yKvMONwq6UWiIEy8i7SRKyZEMlwbjAA2zfO48t1X9dXxFcEkAuB6CiIpdHW73FjK6PiEvJauE0S2oktZEcO9Q4oUvveNpGtFJcp9ZYd0p2fjj7LVfDvJzA9l9zf9XTHAv9ip56eDgsVSQ8apvTzpu27bwZLOTTkiuj6rBANFQjs6ocBmdBvtkTWJH1QZzJvo+tkR6JsIHDtonaPk4nvvbrhPmIQGC1VszcP1P/FOzPXR0YEjFT+ficNYKtk7EGSq/wXWQF55/4nTVj17Cjsrni2FqHUYWQjdPiJU9amcFM7V0G+HgdXzeZioUIFYADh1ON7FVsu8oeRPcNa5hRqDwHScLPUcDssBpGWfvziwkmZ04AXw+Kf07lu7zkyuVPyoMVft95zj/5NMca/oFOEsWpuya+Adc8Emg1wZ3peyKSkE+mEJpx7h7jrEQTRopVsBkPmqHcFuHprMrmkQAN8WpY9sQyvhtvYnmRMtn32aldTFhXitMlwzL24WdJz8yk66tv8YpvUx4wGng2FNI7DCMzO4fpM2YAYNY25eMS/pqNrQ9nlXkUu137wWuFkN1koZDoaSreb5RmIymRc0kPXcTdmiR3fQkAXe4uJlmX0EAjc9awn2JJGaA9lH6I36wP8HEI5DZLxSjdFfLIsQ+SG/ou49bYKcp+gllFcLC6vXAs9/aGbU0JkxvSJnUpAN11oCs7LBi+YUKOpxYJ6JTsLEvfKVgLx58AQGO9y/H6lXwZClfjoY11s/JlaHzRlXdjYME79DZIYm00RIm+4rN865vOWHsPnvCH+yLP3IOye7V6AuuiBMMzQIpl+owZpGdm0nbqW3wYEsLzQfBSGMQNg/Rvde53oaGUHzrM5it1fz7UdKK1Js9rglzgmdiXaRWxj3cNiRyJr1TU53S2msPy7HYAvFMEv5Q9JWp+3l0PgK2xAOecj53KkXozMNktijG3vvkeRYnQXg9D4jegcZqg4w/Q9DUoucQT5m30NcCcOwMYs9aMbPDIMEoSfN/gFG8Hi3+rJJn5F4YQPxVOLVzM3IobbKpw8mMhRBzcS/3YWOY/LEHqcmQZ3u89lLd6jeKtP/dR5j8GVFohG602ki+fw6Yuc71TWdS92xQH25pDg6cZfjeYfDucjoWu2m2i/m+XpaLmkU8soao7olZgbTI5uINgBfrE4RPShjuV9bGoTRzXHcWmjhJyNb02kO/owrOXnmBNhUhqHI7YK2pXRA+Fkkt80flBN5rd5lBDy/eEzB2Q1+EK3197kXz/HZQZztNbHg1/hIA+DO4sobfqIQB6x69gzRhfdEVeLCRLPgdikpkVJGrjfRhxUfGefIqO8FYQdNPD98Na0iNpzj1rSQGCqbooqx2hfQp4/eOP6aVSMVCrTDb310j0kmHWezNcyWaZhacXsztFoHm1FWfh2udQlYl/UHPUp8tIuL6d5h1706d+Vw8jR21AHTESU/FA9NZo/GUIIdtjU8SNISP4M6rtPgQZckgIvAiOasWYVbIKJ6LOb4lVKdGIxpe8MhmfQxD+Hexa2ZCOTZsy+6OPyM3N5WzBGI5WiPl4KuFBFp38WgQSgFDrTuL9PCzW8+XKhN1fOskZawXrJrAFHx9ez+HiJiwqgw3lyn3KYjNj1mYQqC9gbhhE13gpCdSUYSk8wyX5U3a2CWRjZTWlgY8qzh9ilHm24BtapS/i3V7DkW2u7zn/EAZNOU0NZQTLtc/Hs8jobHl8GwZDfWrvwesmDJEc8E9hzZUPyA5axcROE4hLcwVRW30IwGC/bNql/k6Luwuwl42H7itg+A2xxjnsGDSVqCQ7yyNguOG4YsyN8zfxbbhg48SoQStboMGTIjBTbzjjz2xmg/ojAPoaYLDepHjWq6++TcClxtj/hr1b2/IAP78Ad83RGYHQSXdNcYyfXEmUGsxOYevUGLyYfw4bGmcpGklscnZJLdaY2maszzS6s7MKzg58kvvjX1NcW99hLrPPFNDy6km63tyvmDMaZwnn+k3juzCYVggbqnqD0ypkjA8/eA9go2HmdA+rsP9+/kh/wt035vpMUet5TzdIXkTQ3XcpbH+AudzPlt4v0bDqHcW1OhUN53SsYJ0+FJYigrPFp8Ua1vE7CkdmceHG71ReXc22UxuUNayBxpn9+CzSxAgj/NxtBOQIlpwkwbHMB5lz42FKHK51N6KvYEXHiyCvAxtWVQndMmGD+TFRT3eFBKef52BVO25bxXkqpw60AQIkEfsAhPXiiqkhV2OnkRXyG/srWkP0MIgcCEWn8TvShrLWF3gjCK7EP0e+JVMx5trnmW6DZUWRmDVJ0GSaSADoIyiPGc8xZxblDrhQ3AC71pNSlmXYcO0Nll+bBv8w2VxVVUawIZsQQxafH11JgfERzwGShCRJVOhucSHhCa5EuAAcKyR3YHl1qUgQv3zlUQ7ZlguZxIGHBJhRG8wxi5Gz1fBuCERqL0HvzYK5ENQGudMinr2QQp46A4vPRdJ9XhX7csFhfFXZ1PpmGiDRtl+wCQACWzDzRB4Li8V6WqMuxtluAfTdLRjbicqSE6m20eI77rPV/bxqQTynquG9g/ngqIFrc0Ab7Ga72B125vTvzfCaKFgbADvbQ9rvNPHriE91fWILJtM/7BKGyiPC17u1AFKWEq/dhT1sLXl2mJITAXFjAfCx32LL0MFCxq4WOHf9M1GW4+zLjA9oq2CBqGXPt1w7P2rBwGfyxwqGEsDl9wg5EEZV7Af4m1uTUPACfuoEIQ3f6SdKuqwix08wXbZn9eCBox9iD2jreUCGKF5SD+VDUynvBMMz9S4pbT9LDi20Eo6qJD47O5ejmeNFzauA5jj67CXfLnMmehblPpcxWuNRGaLE3NeFKX1s7q0RVrvmHG7agYPNWlNgzoWszXB3HdSUMjHqD/r5VqG3uWZ19FBoNw8KDqN2MdYAbttsFEqRIpkyzizAcSefor5+O+Dg8xL4wPyoAHXm7YP8g1gj+rO7TPgjM+NOY9PGQv8/ocMCcmo68/zNQW5AoEqWoTIDtjaDG1/xRVYbPi8Bszaba5p1YM4W0vOZm7Br6xF86ThNMj8FXRZORxH8OUjI/V6dTW+n8K/M2nR+vjYN2nwO+ig3iLv2Xf8eCZMDF3rsDf/G4J/E9io1FY6/eJ6yikKfhqS6zJcPimFMTnORgPd61pmhv8JZ6Cb/fQbw8iXf00lvpZ4a3iiCnhceA7UvJP8I+Qc5GraYz0ugSs5ne95iAv3P80a8GuxmSgIexs/SjH6Vi5kWs5KTp3bzR8l+wRIGNCqX9J9kxWhpREBlR4K1Hj/RzeyTrARWdCHM3A0fjTAS9K56rQ5q3NKlfjqXnVVyiUbVi1GFr3Nfq4dPCmxOhCsfQqMp9Ar/nM63dxBVOhoAmzZaxO9iH4DwvuzStWWTPJ1yO2TW6HFIHjp7kCaFYUYIkuFcXi8OV84Dg0fmUx+QRFWZx28zSc2g2eti37j4Dq2dD3AyaRAHWjbheFJPiuKEdHDInSnoZa/Yk+yZ57XvsVpdQKHfXi7EwcqoQoh9UCScG07mZsUIauVQ3YktpxNOPEV763U6JW+hVdpPxBQ+QYijmeLaltBRPJzWgFtWeDMYRtkeEyy1iU5oM4cuZX/SUguyU4/a4SvmlKMGKlMgsh+FLsBWeNkwcm+sYfFPv3DI4WSnpfpe5qgTln/6rWCOApG+d5gScZsmGqiRyjBbzUIdbbWRYNthrA49NifsjYEp/l4AIacTjeRZZQ4nlKPd10b5zFzxt2gVTAneCbmi5AV5+93HvOaKndS1bXzTf+OtYPEkn4+8TTvN+0JOen0E6vLz6FU1qF3f1nXXfsiWhmC6yUPNPqav0ZPsqwvIM9Ykumsb35QWQ9EZUQ83+SdkGTJDl5LTeByvtX2foCqPohIVKZxIuMqMQBiasYxp5vdFKZraR5K7h6CixZi1aeK3vEtUVKTA/sF8Fijsot1VMPX4Eag3Qty/DPtTH2XK8Tn431xOqBSO0ZGMUVMqkngBLaHVRxDgklK1mfkm8hZZiWDUCv/Hm9nXNFTEK++W3f2vzL1/ZT1Fq81hOJ3O/5Xf/zfZ92/7v9UCmsPt7yDtN86YHvdIlHkjP6KHIlsyCdF46j4cd3ytuIzFtxth+lJGqGJong7n/JXsqExVC9e1nWJz9W8Eo7Ohxypo+DQLAl/kYBXEa+DFhhvRlh1VnD8u4n33f8uomD93LvExMZxfIGQUN5fDj4XQ+kiqkFH8RCBPJAkeP/o1fTIhKMXBDmcPz0V9ojldFcv5kB8YFLuZJ+uLDYUjY+DS22j2tHLJWjmY6AeTfBaDvVLIFxjqkdn7FK8UiroXz+VDjV87xZjjq24x3CgYJEODM1F5I5L9kyiMm8tJs47KFvCTdwDiP7TpQDe9Hke3vrzmm8TzQfBBJJzqBW0ntGOETve3JOPeropinfM635ZBq1uDIKSj58AmLzO/OBCAqT2H0aakO8SMEDX7QrvC3fXEqWGSP+QmoqhHAdCp9ctUX7xFnyvXScx7RUgUpJ5l9ve7+OGLxlh/W0zEd/D8WtgdflsYdQDVxTxUbxddXEB1N6thdC6MuktlxBOYr6wnsKIzUQnzCa16Q7CydrSGdvM44NzEVRfKXXJq6sjryZid7akp7s8HxfBbmTJwV+5UMf3ay+xs58OdbhLVemXSFllD03Rh6CzdD71U/7m+Ym2rdQCOH/uJRsGnaRUuDCdL9BMw7AIENkeKuZ8ud0XidIAP9PfzQtLbzDxb9hpfhOJG8tW9J52rbtKj/vBZwlnBcnT3q/hd05Ol7nhbHSSXOU/I3jqEIVPjDPSc6jquRu0JydkdHkZFx/xmPNVWJAJbpv1A74AeTMv6RNTLsluQJSVsWlLpYdBJwfCsusuTgWZCWt3mWvNrSHWMxsnOU+w2DKRVTSsy60NL0wpPZ3UBYzV5tHf5Erfj4RH1A4rz7zP/Rm4ilAdvIbDRy2hq0t0SGXs2NCdr+/MYFwIH4Hybm8JAB2p8WjFt51k25gtDtK4x6y1TkWeHcrWXJG9EH3aUdUJbE0Vu0AY6x80Vkn+1TRvIVbudgJIgejh+pI/GS1ce6ByWxED9RM5Xw33ZcL2in6ez+CxvSBd4yCIcrWLfEzjrWHdV2nSyglcSqalhWzTEVm4kMjKS12e9Tcc+1/CbNIrKFyGpWw8iTeugUKwS3kHklZEwXKeUVFXVCTrIkiwk6lrPhvuustwk0IxZoct5NCcM2n7uOThqMNlxyzgesAxTw2eo1p5TXMsv+TteDs4hzQonqgw4JT00eg56/uF2vMHB/FI4bW0JWleCKaAF48P60zt6N0lZ7zPAch/GrG9FIKF2nK5g0O3oDzkl/wiSxo3oC78r6mIsL4f3iyBL3UYxruLyzVQ1H8RH5UU8srlKSBd3XQ7jqsFajpy3j26lT/HBtlw+3nuGuu2iI46zFmjsm0NTv81gryYyMpJp09+mS79LGGdC1YtwqKOeyAkXvMZsJzHoPO1VQbyTcIrWspfsqSWf4X5CEjqhui3RWosInLkCilLkAG7emIfs0HHQDBeq20FlupBBbvQc+DXk7UFNGehrpYUWNA6lJKHZryn7qkQ9gfhz/ajRNYJGU8T7DGrDMuMDoq5GXZlPhNPTPyiHznph/B7+cipz5m7gkAN2Vdfc47weccCcNTD/3ScAyDA140peHy7l9cOqjfNc2OlgesSfPO4nGK8vhOYIaZTaJknIqJGBEBXoJbOoPZv4uECHn5tON833LAyHq/VriHZugZwdcHWOcLzbfsGM1FGogTC1Gf/qY8JpdtfsE/Ol2gkVuuai3gyA00qIPg9f16fjDhSMr4aJTuy+zckN2MyppGEkR82hlNailkfWVgjtShoTSQn/iqNhn3C2rL4A0NQ2jR8LSiPYXSXYBe0NyqSsT/4BZoeKGpu1s0bR7NX0C57OOF9IzH0VWYZp06ezbedOYifO4uMwP54PgudCobR/e9LPrWL6zHfc79G76cuPCFnBijQkSULj8Kdak0vb8JOMDDmhAOl45oOT0UZ40TlF1HNxtRNF2znQvCmPGt6l2YXBChAFwEulK/gsBOLUkGjuquib/9lH1HxhosNhmJcPK6rymHbjBoc/+YT4mBiuXZ2nYHqkmnLEnG/zOWeCDnO9qItnnHWe1186zSeegmufIelDOZE5mruWQL4tg7XlbRTHpp+ZTrve8WTYoNtdSDWO9XTm7ka/uyNq4yxs6jK+yYsnN+oLxfny6Sd5so0ATTQNO4JclSzWmaQXuWUaRPNzI1hV4TpWUon5syGG0OI1PB8IHfSC9bwmXqlUUXtP12PeYLblFpcDpoikbc5u6LePQ8UeVQ69ukIAhE6/IIKsubuZ29mXvom/8EAObDL3Vlz7dszjjM+F0b5wtz4kqA8o+jNNmZTKyQC8EwwfB3jsB1kGJzJ2oKIFLFKC5f9j+9VgYNDIsSyPKeVpf/giFHoblOvuH86XaOIynZpkfoKvxs/TmX+QXvlBjIo+jhpoZyhys5cAUBtIlUJItsL2skBSzeH3jEEr6+nvX8yX7T8kSn/B0yFrOVzciCveYBm1rwCtNJ5Govo8bVxbmdYaSnipixEW1h30oW6GsL6mHp2r3hc1MtvOhaihqKwCWdQy/DBzi7ScsnoAFgBmTRLXa6CjHmZ0eg5V6SnI3iEAVZJMqG808eUTiS55CI098J65fjvoTRaV12BzQpXdVzDB8arVFiiSPsWaiyKh1nmxuA9HPusfMvJBvGBMhWkSBNsaIHODO7De0FDIr6Mi0ab/IALwfw6EwJZ8XRiD3VVf+ov88dBnG3T5BS7OQmUSiaoIFZQ2T6VeifKb8U/5gX31RI1otz3b/E3B3pIk9qXuoVqTR3JZY2Zt2keVl/SpJMHJ1Bc4dvNtZN0/SzYbfQIYkbSATwf05lZRJ6q0LTwH2Coptr1BeuuWdElYxuzmC8Xz6LRYSBmr9KKuIXA9diY3LS4Q7J5eArQyKpMnC+pz2CzKItypHi2UZTp+K+qXySpuqteT23wge+5vQ5xloWC+j7eR6+zHjx0/4NVACFDBoOpXBJtghSSkqmVIjRB+epU+GSm8u2C3X3wbKeUXOsSs5W7IMu6GLLlnfghlLM8fNWpZ/FtSQdrvzG31HZEqsDvtXMnvRZ56mBtsRdEpnohoR7/LKbROX8y4P8opD5sEK1VuucSz5hmsKBf1Fc+VPCzqEAJ2dQib7zxGds4YVA4fjtR8S2H/49BhIcSP57TlfcV6H4gJ0ldDRSo++ft5NRBQmbmSNBRH1BQhvwkQ2pWtjgTOR35EidHls8oaIQ0f2omgxHF80GsXw884aXL9MNaMdzzs9Npn4gpCv3xlPN8f3Yi6Ol086z8HQeuPSW1mxV48lNtRH2HWpkP3VYIN67QS75tKhEqweUanp/HmqI1Cbvz6F4pSI/AXdrcMaWHfUWY8S7nPJRxOOww8AsMugrWMYTHv0Sp6A4GRK4n0XSLuq/SyYIZmrmG+yBfxTD5s0kwXYIeDIyF9FfsTZjK16BvSIhaSbIXN5XkiQZrwKOgjORp8H5PLhWTf/cHZSNjdtbnFuD1zRHLpLtVOpo1FDVlfIeIfbQ35wmbQhUDL97EbEoipaURO0B/0KTvPIsfrQoa/PBmavs4V2eXXSU5+iZwsZOmTfxDypA4b9/lk000PsWoIU1WIGsKtP4YW70L8OB7J96PIDn0MEK71+tqdTrTY0bqGfccKF63+bv9AliHYkE0/A/j/QwbwltXLOdggj0n+tc9DFiCw7qsgor+iVMTZ/GfoHPcb6dV9oeuvyNh4Iuknmpkn09VvPAXlzcm29xG+UO6ftKgR8uZOyUqzzLn0vH6K7qEj3NerTfY5pBq63N7N0JyjRPtFK/qQnNjkct4MgjZnhwuFhYIjjHSuppGH1EUPowsg5VJKqbs2+FafFcwnv0ZQbxgXpTBe+vAdmr32PsNudKbQ5gEtn6l4FV2yUIVYGvoGqyqbgj7U68dWs+jEJgafL6HH9VNKO7TgMO20nr26VH8LU73XwC8J2V5Og6BL9Lx6jsaZH5Me/AuX8y67j5VlKPE9ys2Yt1higi0VPgKsMt4CBgF89q9qQ58r1+l0a5e4x5LzkPIL+mtCyjiu8CnapC2hvn0I3k2w7sXe/Ws5HFe/I3xMANMtepTs4FmfSaLWJK61VB8uGMhtPqV1RGsS8l4kKLcbN2+8zlar7b8wR62COZqXS1zAVd6Pv0wHPZz0m8WA5QNEbdjw3tjkICJ979BGB1PyYJnJ60JZWzjTeCkTXabRObMaZ5BX3G9nR3bHiOBvn9xPeTDgqGDjg0iwd/yO7b5nMcrimwrVK4E4qi4/Myd7FloJ3o27RrGzlQAG+zfFEdaPBw59xzqXPfvGwSMCPBn7IPg15MWdN5hb6LHZ2hUshXOvQsYfqGwiu1jLVm2vtwiZ14i+0PBpsQY5VaRbYVAWrKn0iqNqg/mxNIqjFugedpo3O/cX5RRcrbL9QoLuQLnPFb4KhXeaLfPMPx/Bzo9Vl7EgDKqcUGhJFIB21xwoqIrnUlFHUvy3Y/H5lI65jRiQuAS7HeGLtnhbSJkCZG6ir28ZGVaQ1WLueCf7jFqx15htXjE8PP3/07//bf877d9k37/t/15LXwkFR1BLXju+N0W84CiqW18QqvEkq+qizyzBQpKks2UA8u1vCfbvruivDSxrJCsdwte6tY8B0AZh0sdT4YRL1fDQsQ+oiX7Yayg2HlG9wawALW10oMl4kTlvChnFXXV01PfbEDKKH8xj/rx5wlAuT3BLTXgHuIkcwMScKHbq9tEndjPDG7iC1bEPQPwEHA2e40BBSwoCdtNJB120RyGki6jJYc5CX3GF14LAKMEPZWCvo+HfuWgzC8PgrSKIPTkRq84jbYIxnseTT3CKdB7pA0cdVk4inNDZQEcgyfX/s11/Pwkct1r5+pvfad/nBhGjfyDzORg5DE4PP8esVx6gl0rFgDoo/oE6Pb1UKi/JOO9l6t6dY1JBET6WhlQYLlPjcL1zjZ9AmUUN4bUgSNDAovwonPo6skcaA/62RvQMzODx5p+z/rfXiR93mCN/mPgg/xa/mqzMzYdeh9XcP6ShG7GEJZ+XGi5hmI9wDs402QM3FwipFJ8Y9wbnlBwMNcIw1S+gjxSySUFtyGIkja8dppcBvgrRIFd5BXQkma2WQ3yb1p+lJthToXxPDquJu2Vb6WGAa/HQI1TJAPBu2efhCevf8wAeM5u5fnYF7x7YzRObRDLCe6P2RuUPyILBdyYoxlwt+2B2QIn/Xrr1C8FR6sUmq0iltTofvQTzS2Hizb5K51SS6dlnE8l5A7A1hI+SNipr9mWs5misldYu20hRfLgqjdVj/Jkd5rlP27kZopaIJJHh/x5HMgRDJb5wCu8kbUIObC7YASNusadonvJByCoI7QR+DSCiPzG3olhugvXB9fgh4oDiUD9rBg19c2l2cw+bbkxD7+XQENiKUdWd+aZU/HNnFVy3N1Gcn6ZpwTelgnmwvMMCln01ifiYGI588gmfld1gbU0uPxZCzyPQpVUzz/xTG7lT0o5iq4FRRng08LQioD1Qf4bH1UkARKrA1+lhVtHgKc4UnqTNLcHSGZ8ZAv09aDjZ6cASWUBGXDq56f7crPYkpgDuS/+MVVFn3UEMlTddMbg99FjDSZNw3BoFXiNUo0xG+4etIKD5RHxl6G0AX7uQyZNlMGpK+D5MzbvFc2lr7yvkcLJEIC+k5gxHYkTdipFGmSZqL+axpYBXfH/gaZfjOjMIQjJ/FMGaygwwRFHmJeFVNznq9G/CIZII8E0mrT4MDFFKEyWUHecxP1hsgtFpMdTIHrlWvS2V9lHbaZ8/k95XrtDT70kh4TjRCV2XEq49T5C+gMS8V7h8YS0FHc6Lemulwtnzll3xMWwVgbMuQnbV7NuTbeVa1lcIVG+2WgnQ6FLxBr82OUChn5dEp6wSAZLowTDOwr7UJyi1RFBqibjH8J5nHc4nJTAh+jQTY+4XdQhd76LYHE1Z+nQkSywNK+cqmJB2dQgT1xcz9/hvvLL7BOm6Fz0XjRxI/K3u/GSCb29N5MlNmUh6L3adzYRvTQPCS4ejtYaikfUiqNL8TYgZhfrMk3QOucHbwXA5HkIqDyjGHNfjJ37N+BUrYHaokbz3SEME1/TNSLdB4/pfkxj7i8IhlyRocnoMI7JBXQFzFl9mS83/7LxuscEbKxyUlubyy/m5fH5stfta3m1c8FmGGeEBX3g/KgNwiiDXyDuQu5c3IlMIUkFiGmy1ToUbX8GWxmDJhc4/s8QUwL4qUNdEimCOf1OIHQ0qA06nE7tkppEWTnb5kWZ5A4TE58GRBMjJvJq4g0E+oAb0jlIPazq4PS8cvshPLkd6kH8R7O4OhcIx9sldQo8QERzLCV7Lj6YmgqVxcATcnI8s2amJncNGrtKz5Aq2QE8yCo0/88rqsaMKXti3g6V3vL5HwKfVe1yvf5YjZfVongG77o5UPjCnjWeCchhihNbBV2iq+RFslYSGhvLoY2/Ra8BpWj39IJUvQuSAIUS2HidqYbra1E5PMabZHAAqQ8cLgEZQa8hYx/IRvvQOO8czedDu5iB3cgBzDuH71Izp1ZEy41ku18Ax6SF3DTCSFzGo/CV0RvFMqjXKewLwc1SSqIH0+vBSAw8yd/7cucx55wOOOGCfFYWdt8ti4ZDdzp1zr/LY7QyiiwWAIt5RKBDakQMo1/egstqzrnjLXsF/cKK7/S5AKcDl+Oe4HS0YcXUThbFFe3nGX7DJjlvAovGygwKa8ztJXHUlgerWhAFwtvuK+ZktOdloML3+/AFzs6/BmACBLcT3JXnQqipZFsFMn1gc/p14ZH0eG6+/wgkLXDR7/a7dgrb8HVQtB1OtzWZLJWT6jRBJ2zMvQEUy2dYICv3+JCViHi+17wSb6wsWR3ky+MSwvEwrgr6VcMcepxhzuV9z9lTBjRqYXwLl1Pd0Zqxlo+60m3X/bhHMKvOMTZYh1Ocu7YJu4tMBjtr/phJFTQ0Txj/M2AAzfaREut6FlRX3KY6rDcIPpzlXer9NRKGXTKehHsecXTlk2EFj0whWJuwXyU/3i3DgIzkxOWBkQQnrShIU17blH6ZF5HsEBe+jXdQuDLIHGe6Qjbx4+TG+K4Nvw+BBnwNinRlyBtrP5xHjdD4JEcm8QRcLKIybKwJcrefA+deZ3+ExdJJga7QN+QNMN6DpDAjvQVnSt0zalM5TGbG8WlzDDbmLYlypYd/zWJ6Q03/x/PMQ1lOwmtvOhbIbUHCMIF0WD7d8h88G9BBBeq+2riaBDZWwowpmnF7nrmkry9ArfiULm2whSgU1cqniPKes40xBR25aYZZfEH0NWQIQM0Cw63+P/pr37A9Q4nuS21YEmEHtD42nQVgPxd7s9kM0AdD5Z6p6HGPESic/FAazvRIq1cr5Z6guoKPOlwd94VTTc4RY93k6q7IYcuN5ZgWBxh6EXCt36P4t8f/JUXMo/4fJ5k59J3IuZzDLLn5CRU2Q8luuSGVW0Gc8GlhDCy08GFAuAuENJ0OvjXB+JqsTNgOiFnRD9Row3YbyW6J246lnOFvvGhP8RFmEGoIh/7CQgHTYsZRcwR7wKlZgXUEClepmLla/E0mGhn4Z1FNDhQMOGD7z2BS2SjqErXbb94Cw3W7OF/VnTz7FyMZfc7H+E1ysP4n2up4icZX8EzidqE1niNcJ3y9eDQPivhf2vdUEN7+id9g5/GVIVe3klYoULhl+FnvzfVch7XeCq719KAlDhYvRGtYdRt5hVM6f7t5GTk9S1q6JYPGpZWRfXotdrmRD9YvctWvAJxriH+Js9TvuQDtApCMNjo6H/IMYczbzRZhQBQkO28mSZosgZZk4MPYBfnC2pcgBRX4HKPQ7QLW9Uqks45qOOlUV/roCJKcXW8hmpqf1Jt31cNOUREZxd2QXY4pcDyvWFrqSj5ovo0vMWrF3WvKRcnexdFAdtiIIaeXIAa6A9X9m9oGoFe7pl8W+rTaCPoppN7/lDcdOerebyNT4SWApgIbPwNCL6ItPMi3Icx1ZlgUTN3cPVKSxrcLORavYk2UgRBMB1gqIHgIBTd3JPF9zE6Zvu0Nu8/1w+QM4/SJh2qt8mHjYDdBVybKoA3ffVWgyzb1nzgmBXxJ2CnnbBwuh5XtE3RjMt8NaUFuzVJY18JBJqPmcm0aV7B1XkCCks2DCqnTgtDHDbwuvapvRKxNeLXxI2OdJUyGyn/sZJmpgfwz0D/TaaRxWnrz1AktcpDgNYHcWgdNJbm4u6/74iCN723P3U3AW/zMGsKnCzKd3BrKnSgDURobeEpPizk9wbjrNTV/zuCunMdoIL7acxx7Tz1D/UYJMK7m/yXzsdpEEkCU7KpXLBrj8HiPLhP/kkDxz0tsO9zD/au7p09XOU8ChMmN2gkPjD5IaYkaxWPU2573EFuYXjxO+VvM3YUsSsXYBuD3QvDl7W8ahKlkiavRahW8zvuYMHR/dysb7NyKhlMSUZYmhZy2EVwzA5HORcqfS/qux11CtzcYmVxCkcnC/lADbXJKYAw7wdNk097ESKnFfXZZQEfc6pZYoAsxtKTWe4mq91zie6VEiqE0CgVA7+a7Mq9zMkYd4tqGRXW2DKDYew7e6kRhzYGsYfgNL30uKMda13UJPtaa0zRkm+8PeKrihmQSFx8X6mbkehpwj3zyVu6E/cTVmBtraNdhhg/VR9C5YQUTZCEpuHqGH9PeB4z//9BPXC3sw4mo/drrCcrIkCzB+3+2YdS14pNXbHIuFn02g8XnKcxGfGHaaGpDmmj7Tcv2wtV/s6fdrRKpVrKt5+UP4MTcNGrkIG34NIfEpsit2MsxHfFOdI9YqBxnQlDD7+/Q7V8L0PQc4Z58jyhcMPQcBzd1rQWBFF/QEieRo/cdAH0ZOZWOK7Z7Yc8OSbXBjLhwZi65a+A21yb5smx6avy0AL8HtXO9GRYUT9lRBrjPAMyZ9GN+UdOJgSSyXS1qy9Oq3CsUvjTaIUtfy31kPbQJve2JkuhAYkUyxw4DdKdQswgzZ7jr24vtyUqW/QXbISq7Yb5IV8Aa3izryl6qS/o3YmD+Il8/PIManDYBICiKY6LW+idVuvQd4+d+Sff8m//532r/Jvn/b/5nm5+eHdEUEM6j/OK/EtxfIOOoYpYXHcDR8noyKePe5Up2itRb/PnTZtpUN6cNpXjGUSH2Aov+l6mc4GgOOgOM81/IhUcdhWwvXBrmFl1o8RoxpAFVOuF2RgFPnJWWHRLDjKElaB11tsHHfbrbU/DcZRYGGKSnJJUhtIU4N9xshyHFXcawKsRNPLqzkiXMfiz/2XAfdV+Ds8B0LbAJt+kohTCtbKhD+zV6HTosJPTuJz0MFuhaUxYYB/Dot5LeUL2mT8hsNXCw371btklN7JiSRj8ZpGCCriAeOANOA713/fxiIBwbIMp9+/jkREZGAhKUmAosTcmxwQdOA6S8L+b6osUK+b5wumld9m9LmxbdIz8wUknGV6XwcdYLhRmiqhYG++W6jDYDzM5nUeALg4JE82KD3SjDJGjBdx1cW7KaVR/Zj1XslMAFOPsOiQcG0i9pJTvpHLPtpPoccsLPaqgjWHbTbOGS3M+fNN0XCxRjH7JOHOXztE2xOSKsxgibQfdno802Z1uVxwMEPZbDY8SUkvSgSs0ceIlQ6hVNy0E4HU0PLwXJvUNFZW4OiTmJCn7mejPqCWfBBEaSZW95z7o8BkxjkA7JZ+48cALO5TPE3TelREZSuLobSq3xp6EZDDdgBu3eyXaVjTegXvFsMBglmRRRTr3qPpz/5R95SraKeWsgwHjVFewKwrhZsCEJjjWRzJdyoiFYaBWE9mJFqJPAwtF8EO1bEueXR8kqt3CzsQmFZa/fhhcm/C2YMcDfofY7cfcjdZ5WDhRxOk2nAvSwRCacwkuw1oNJSZtdgBXzNLTBKSsS6qcNeXtpxibLqcH46P5/KQE8gAFnDqkcP0MYk6l5MLYANNWMV51819OGlAlFk+r5lTZm/+PB/LK5+yG5nzqw3RG1F7FxrMpC70Z+L2hUhR/BOhN+nXcEMn2j8zM3ZEAVjTFOV9yjhZj5l2zQCne1qhuITmBqAYdQefhm1nuVVSjYi8eO4XTOBbjWdWeAXR4LhhqfPJxrixqJPF+v0onDoYxyuOP2Z+HXsqge3rOB7B/Ij33aPSaeuYmSD5bQOucImn3eYreknpNcAf7WaJoTT7e73PLTaxiWNF0vXYaWx+jYNXFNqsj/4568ViPQtDeHMywTInmTwC8EXIfV3z+lOBxP2dSXHBl+UQKrFS1YXOJT4tqeeqVPJOg0o38D7fe6jOulJvukzlr4ajyQqwe15L+coL/i/wr5WcdTYDciOcjj7sltuxLug+gcRp9116wDKYmbxYIYHWSrVsZSv6R5gTglE+98kpPEET01RV/vj2lrO138ER8xcQn2Uewl41pavrZf4LfcLwfRwvQu1XE3HvPcYdCWVHowDS77nPBlKjCfY1TyJZSGzqFJ5BTskCckVACrxPYpJf0X8vbZezIU3eKXrw3RI+YOzoc34OORzAZJImgrnBAP3vXwjswpFEN5iUMrSADhxEquGNj4mZGcdmUXXb/8QDs+0+EzRJ8ugcogIf80Z6K3R/m3ndd/enwBoEnqMTeNlQgvnex0lMSp5Ks/mw5clMCy5lXDAwroLZlj+IV6OvEOoXPvcXayD8luixlSDSbxVHMVXpfDIwaUUaO6DhAlibzfGkn/lS8bEfUOpHX7NbktGyJciUFp0Al9VFk/FH6SHHrrqoW9mU0he5PU8PMHGAJVVSL5cnAUb6hF4YxITYg65+x2STYy5wdNgiKYJ81kfp5Th9W6dfMcSn/8ChprYe506YwzVhnZYXTGgunMXlYHuae15pQAGRh6hu+ZZsJbj5+eHLENWeWNSch8kpmAy8Zr2ilNlGbrFrqNVhAjCOnURAqCh8cOmCeJERRBnQxdS6QSTXe/xMFUGcny7cFgtELPnquGg+ilP0LkihUbSXXQStNDC8MBCpb0BnOqwhg8vTuO3lFFczB0AQG5uLm+8/jpbrPb/GfVsc7JwawqSSSxUkY4SwQA49TQGeyoaNARUCukvX3WQ4vy/dJJjRoj3VXSG851/5MUA+Dkc3gj5U3HYhaQP6JsldgitBLI3OC6gGWvkJlxzJfveir1EZPYMxfn7bIEsMi6mIGA3Zl0ausItArFceAp/bR73BWVR3zXN1LJKsFUHH6cmZDBl1eFUOdS8Ugif5g/yXNRWyQD7bIbE7nb/ySH7wODToubfqSmclL7kROP+XIt9hS1VViFrBaIeUmBLpuQGcMgFLq5rJ8X4x9DT/1Gyckez8eAFLN4JKLUvuRgwux7DEQske80xSRLStYdioVUoTH6kHSO0mr+lRBEe1Yj7V9mYmdKRU9WQ54hQHNtAusbDflBskziVNRK7weOnENCEn+1D2Wc1YasJY0vWfFHrubZVpLDSsYUPXATbukld251lTKn/IXN8v6TtgS/IqOnn7hN2jlgbH/KDrvoixbm7ql/jm1JPbZ2Qu7OE9N7eXnD9c0J0hagQbI1JSWPhukdeXlbJ3HGYSNcL/6Puu6iVO8+1wyL9SiRjNJRegmMT4c5i2NOd8KZ9qIz+inC/6/coKCjAXV7JBUmCOP+rPOhvw0+G9qYP4cIbcEHIUzrVAbxxcj4/lMGHESUkystF0iG8B/gmUuVIRG3qznmy6J5lwRY9BjZEihpghceZECmASjoJpoWuhTNT4cIsSF+NyiQAoKlWmVE5kOWjZFTQcSGT7vxCmQMsDhVObx9UUmFTGbA4obm+gme7fIjO7GF6SBJM7TGCH9vPhfb/LNk8dNhkLuf35XD6OH4amUhs2QeegwzR/GgaxvZKeL8YAq809UhHAtgtGFy1RN8LgZ66/QKYZKuAqIGQ9hu+soPuBghXgUaqFoH0k09B1iakm19zKFYwzF+60xUfRyqsC4E7P6GWbQzes5rphWBxQrZurJDknegESz6TGo9ngiu50EEH0mqth+kWN47Flz703EZtUvfU02CvQnWgD5/Gi7nXSgcPJ70ARadFwq/dfHoc/J1brsBxmfG0x+Y3JsCAwxT4TybffwcZIT/TqN7vVISMF0o+XZaCpZCBPibi1MIHXZg0TgA7Ua7JdsmCBBitRW7ZcsEm87z3Um1z6LEWwvtQ2WQmnTLABmTZ4OPslm4WGnjmfHbICk407ovdfkiwDY+Mg1VagjPfxomTwQ0X8fsD4YItW9tsFTxsPsBEP1BJNUK9xBAJPf4Qv196mcDyP4jS2njCH5L8bwiA0J/9kW7OZfm1l9lTBU5ccZX8I8JeiRl1r4xnndhKbf1Oz32oRH3P0sug0rLLfpgUdSpLTfBObnvx991doeAIFa2/oJ4ghfFKILSx7YVkV5C/1QcKAMzySNgdtBwuvy++SSAuayWfhwr70GLzpd6F5qL/7loCNRk8FnWVpn8BGAVY0mgP66MEoO/LPNcubroFOXuwBA5ic35TyoxnGe8LjZ2uRI1vA4gdg6RKwFAtkv07EwqEz5H2u1AMkLUcVW1gqUu6vpY5y+b6oi5a7j4+D65ELcH8i2+SXvmgZ1CSijsB3Ths9txzSvwt5s/9kviYGAo3fMKHJbl8XwKRjn/GAA4ICOLoqd0Upr7JjCD4KMEF+IwbB60+ItH0Mw+61DN/NMG0I4swOyNhXz9sugRe3HOAE8a3eD65HpP7dOKFIFlI17f6kP3hEwmQI1HbPQwob/CuxuX7O/8y2adUjZpfClndtgkApU802epmlLn27YiSUUSqG7ufFSo9h0pWsaOtLxWGa1h0d0n3c4GSr34CyYuJcxTzR9szGLteYFpkCn5eJQgC1Bl0iDhIgOzg5UD4KPh9kWx2tdzz71O/azxHm3ZhTLNP8SVdrI+19+jFhkzSQNTF9lB+i6qkT8mtaOC6Z7G+es9lb/Yd1LGVjQkk1/jhkKvdpS/ESSrwb8x5Uw7b2+nZ1SYEm1zhvn5tqwX1lzugffIfROoThH0FENgGgttS5HCQGforOcFr0WgQfmTGWrBVIukjuB31MaasdUyuI2n/n9pjZjMbV6ygoiYYx+WdxNwWig7eLGBZhoNpE/n8ziCc1LFlgtvxTu4gjrjEQuoVP6y0f7uv4MUcEbO1IlPuiBe18WrbtTlM5i1G+8Lz+XCjrL9ifA5rJRb5BmZ1IbcLe2IlQMht2ipRVWczOmYPzbQQpTXz630+cOJx4Zc7HQQbchh5/SJ+VYIRuidpGQw8Ct1XYdU3wqy5y+lGAuCVYTVC64/camTeSd26zwNgsnEjHZM3syPwO96x7lLU09VYcmitFTb8U8e+4uldh5WxK78GjC9+jOmFMNIIC7p0EOoJrt+9v/FXXBzwIl31kGqzkxU2h2uFPUSyr+SCAIFm/OF6/u3ZeWMXtjtziXcBIWuTgiXGExxKFz6j1WH9r8m7f5N7nubn5/ffD/r/1P5N9v3b/k80lUpFgwYNkINaCYSZLgSrw5dqVyBJkezL3YtkycHh8DA59JITbn0nCsoDIanP82av0aQ0HsOqUYlE5SprX6Sq23PADCYHPHU3QVDqy0R9GYpO4r83gqFBuWgl8FdXIaPchOfYN/FkgY0V+6GHrPnbAcU/9/3E2j6TOBMrszEaGtR4GDeUXOCt0GRaaeGm+haHNLsV1/B2LEQyRg9Vd4Vj4qoLU2n1we6E6/HgU7pfcb4hdji5xa/Qxdqenn4lqPDa2Esu8L3uEBP94M1LL7Du2ky0EhwCdlIHxY74e630hBiXg6bGXOLVcLoalvmNgPBeREZGMn6CkO976IF9fDX1QWZMuo/ISFeKqqaUUQEptNLCo37wY+wZjwMIUHKR+n7H3chyhQFvt0BlOgtLdXxRAkjOexI7+Rg4bS1nSvV8Zq6U2Wq1/s9J2epq3pg5k9xCE8llPcipDuSWFcal9IHEx9zHOjQRZOguUWY8S5YNsmgOaoMYU/FZBtOZcYm/s7gMGt6IhSBlMLPG1oQ2jd9mTz14L+yg8j0HtWZuCRwyC4c9zdyaum1y2BKeCYAQo/MfOQA+hgAaBJ2jTaRI1OkKtwlnvroISi7wSswxBpZPoJUWmuuUwaImQS1onPkxVU5ong5XjF4JpnojWSuNoNBlh9YNUgH47G/J63FneSAHVmR1U7yr+Uv2s+CdSip3wfRs+K2ywC2PlpDYlEd/GcKV0x7k9Me+Y0VNLP4LsihlGU2MG5X99gpYY4TTz4K1glY6K6Eq+ODch2yq2Ks49lbJNe5EzCU7SBhJivllr0ZfmUaI2vMdlTvyFef3ixmOf1UbSstg156bbPlv888FCjBl7OFO271MC4Sth4+zNPeM4sYOydv48ew3OHGwsgJu6L0Yh6nLGRX+MEZXfbEotUORyMFQj8XZBsruxGJT2+55gDVNZ3KoajoDS6YxNTKDOJ2S+ffohkc52lTI231ZAotNzRX92/IGuusJAvjpPFITpZZIHlmfx6Jrz3Jbt5cFaZfdMnpVfkN4YmMep1KfFQW7vcEKPtEstt9hw/HddLn5J+/uuoy54zaPsZy2nKmBp92HPxl4SaAaXU1OX0V6gkigzyyEW2Ylg86mDabADiOM8EWIHq1kcvdV+A3m5dRWnNWdJkBXhJYyEexZIcHGeHeyxaouQfY/IhJhvbeIxD/KYNBvJU3E/aavAUBtzeFo/QIWhMGBetDEqlzzU/T38X2ZqLP6Q7NVYr2vbdXFVGWsRxXxO991eZVXew+kbhuvPsZzAaLOzbmKB92SpLIMo5vMZfWYQBICL/FowhjY5gkKyljpGb2T5v4ZVAbs57rpgOeiNaV0MZRRTw1hCfNo2m6kSJzXNkcNTk0eqeELSLH4UOZIct3MEii7glMXzucFvhy1wEfFYPFRsmFJWcpTkSuYGgi7m51AY/faDwqO8UHJPJ72hxcLYNHNx+/5/h+ueowucijGK/B4tbKG639qj1lqOHNoBf3rL2F4o4Ucv/sANTovxrUkUWQPpMQBaTY4Zw4QYIZTU+DAcEh6id7XBpJug2cNEbTTFkPjqUL6LKSTuITrUna56p59yi93O68FQY4dPkweRGHQ84LJNiqLAkdnBh57m3mlIpie4z8JAl0AkJoSekb+QTuVHxprCIVXz8EDOZD3J5izKW80l+/ujHb/Tj/tLkhfBZ0XQROReD1UqaOdDj4NAVWll1xrTSkLgnfzlqEhXfyKiPc5ohy0zYzaUUK02sbzAdDA/7ayX5I5bb1OsQPeZQe7bXtQGcJo0KABarUKCQdxJWNonb6I5wP2w0q1e++XJHh0Qx7v7RdSvpKzRtTBcTqwh3Wnb0EmRy2C5dFE75UA1wZyOv5jfnZ9vuGlwwlUe9h0tPmUWZo5ZNvhET9YkngFqpVr9uCGgwnNm8oPV95gX55IIP20eDF9NX/Pzusm6clKF2yum8Y2oqP4LF3yExnUYLE7oKNUrvgv+5jal3NmNXl26ORsSGudUtLNbogi0wattFDdENqbVyv6ve2mfoG5+JVvV/SbaxHCQN+QG575eWcR0T6X+anRSQa6WHJalWfcktNCaOh29L4iEO8to4bGn994z8183xQFXfNeEjJMnX4E4OPoP9yHf1PQEPrvE6AAfZRrPCokoDQRphiV6O3mN99lT8R2OtxZj7+5tSIQR/RQXnF2dtdvAtB7sUZlGS7kDmBqvgDi/DDkHLOmtKaXSsVgvVKJoqcaukswa/ZHQolClnA4VTglJwYJ1N6JVaArW1kaAcfUV5h57i0sIcp1uXa/qHJKHC2aJhKnXs+M+o9TWJHIgjDoHKCUCK1pMJkhWVDqgJvR7yrZYs4aZjXcysN+0PnMBA47lKUH0pwvkn5zLknZ7wNQETrR09ltJU8fPUyVU+xjL2V0gfqPw462cHYauqLtjOg0hlZa2BYNbe1KtYmo8p88wMza56ELgaSXIG4cd+q/yGZ7MpOLKuh5/HXl3g5MqPqYReGidvaYeuuF7DPiG/jt8kfEnHiEW1ZXIPXuBsje6u6vDXP0TPPnjP0TYYOnrQS7mXXlx9mXKQJzEpJgj9QbIRg7KT/TL0TMW50Eg/1Pi8TDrQWQuxvtpRf5uF9f+hmFfSf/RVSrGWPgoI3XN5ViMvTxdBgiOdxsIfNLITbwChNaz0NnTfG8Kwlsmhw0gN4fRj0U+N+TzRoBrgwJEb6T1aEjubg9Nd5AHF0wu8zdOeMySZ1OWbmQdFzImDTx3gdkweqqJyG8Nww6LtiOQKHcjo0VkJcITXz+EIwHgMoM7DGjmZqPu/5YhbaDYI+efo4w+RThpqEMP+Nk+Bkn9YwegBmGSJbcXstvSqySqLU16CS0/5p0Uxv3n+eWeoERzryIZKskvVqsecfMsPD6PsEeTXgUooYQZulKbMFkAGaElpBU8wHs7CTsfFs5X96Zx6mkYVypP5l5vR4hMO9rkSjc1x92d+aXyBQG+4DVCcXWCPEdAlpnAa93f5DujT7DqinCKEHSkYECtHT9S0YZexJmjSWm8HHiCp4hLrivSN77JoBvAy5XheJEgFF/LmwMoa4dJGMtr8snifFidNrxhbiHILQrlvD+vHz2E7Z1kPkj+H02pIzHqfeCdWoC+MF3FAvLYHfXT/l4SFPxmuMeFL+f9jv1M8dSJBfjmwy/33ke9glggNTsdX69NpNdVeCUrWyL7si203Pg1kKw5NzD7KsbsBbNM6fUKhUcfwQOPQBOB4FqK0YJjlpgXVki+CdB4iTQBKDSh5Pt8hFfCYJW9n0QNVgwgKuL+Sj/S95xAR1OmOG8s4P4XmUtVKQQUnKMR/ygWptNXPBR1DWuRM2Qs2RU96P9qcdZ4ZpjkiSBpRBuLoSiMzhdEJitlbCquLlgNu0fAvsHURXxKM9WCQbVnFDo7lwtFEc0AaALoatjAe1ShM1+xqyDsB7QZ6eQxpNkclSjuFIRzkgjdNSlCjZ/dREUHILiszzjb0YNbEgdS26Nl1yhrOJE7PP84MLgHjbDw+thztvvcMhuZ0+NBxT6GLD8L97EX7VfDQZGPVy7vjv5oAheS3Mx0M9Ng+xtnIjfx2SX6ZNihfMFXWlr/Aby9iNh447NxnnfTyixZ3O8xkyKbYxgQEX0ZUS/H5kfk0PzzHmcajiCfS3jOVfq8c/71+9PO+1D6OwR7G4dzqp60ZRaSgHQyGp3/c3aplW75pjT6QIrQXB5bzre2cAIX3+4PleAFoZd4rAtyS29DFCjayAUZ3TBUJmOGQ17q6CvD7wZcws/tSfZ11C/iQ/7DqaJ3oROAoOslKYPy1rD1ACwaLPYa60FIrgAvCUXaaXJcB+rR0a2FoPNY8/fiZjrlpyum/iq9QMXh8O7wZ7x0/ZznssS5Xxu1HuLtLDvxLdcUwqHH6Tl1adxyNVY1cXsbOdHikppu5laryP0bF9WV8CsFt/T1dRZrD8TndDoWTIKLpHsFL6w5JQFs8+cI4A4SS9S3exNqjU5OKv/GXO0rKwMWXKglRyoXDUKdWqdqLd7fiZ6Rzans0ew+m4/rsfD4walPaInCH1NDE3vfsn7fuHIyd8o+g32KGSHlryIhVRI6z1A19x9IsGPUDD7vgyyzcr4mvPAMF6JacWBlo1Qa4popPoFdnWEE5NQlRzlzZbf0tcAfn4X0aldmfZ288FqYvGwaCa1eZ16xeOJz3+O0ODW4JcER8eTeCFSoXIhIYPDDuvC4eTTyDLITjXNXXZ4f4eXnWTO4/7A+xhWfyUmn4uUaa4rxixfepcL8cIWiS4Zi7EmUVmzb5WeJ30E4OOmFXZmPg9+jcjNzWXZ0o/4+Od5xD4vSIi5h85RXCwig3Y7AuhRkSLmlKv1jPiGt3rej49W2P61yT6r2qMSZbXfm+z7l9n31602h6FS/dV++f+//Zvs+7f9n2gOh4Pc3Fwc0SOFzJQunJ9yzrE0R6ArkrK80Itjy5DazRUJPlcLUdUIaaDDowDQVF1npJ+VMBWsutuTEn0Hxe9dCPyEd/N9qHbCZlOwMDgHHoWRqRA3lpqE50mtNjDUBw72ew5dnjJ4srdYoNFsl2CSzcrfaY+ZzZw5tII9KU+xqNiPh3Phrr6P5wDTLSYFpZGkhWAZwtSuQGVtLaR9vXmj3jUAGmggUZ0ikGkXZooAcsfveX7nGWrKW+OwG/9SfkOSYHjSN3zcbwBqZ6mnw+nAKNnQSlBosnL+8hy22/9nFPt2u4M3Zs6koCAXlWzj117P8bGLGOWNapIkaBm+n6mdJzO+xWy0VR6EKwEt6HzrUT4vgRMpU1h592dPPSKAfrvomdyHKl0qE/2gkd0rGSNrIPYBAtTPMsM0k9kdX8MPZcDxQuh99Mu2kX8Kusvy35coWLwYtWxFg4zGFojGqQyuFbQ+wDsZolaGVgKNbBNyN/mHoPcWchnIWf81mO1aom9uRPKSrgCYEnCL8X5iY9dKymCRKqw7rxbiRqqqVfcu4x9mv8b7RdCyrQ+/au7p/sv2q8FA804Tebjlu7zdU0iuWWKfFXI3PjEQPYT3T5zhZFkcP4bDF1FeNdOcTtpXbWOi3BKVLZBrNWBWewUZwrpxUtOPMocwZpM7/H4PC6m8ppxyvStB5/RIic2fO5c5b73FUSf8aftrebSsG2+SeWuZ+1q+Oj+BfAMSSl5hdl9PwEzjLIZL7wuj8MJMugQpZTwlWS0CGKHdoPgMRxNzeMgXTIYLlDuVqdO8uyuo3/hVTE3G8vRwDeWZXgZpxR3Y1pQHXMnaT0Ogdx1DPL50JStjilCfgx6y6m/Pv1Wb9/JzqcylasipqkeeVWnMlsidyShrgRMHC0rhkt8Tns6iM7TwXcGlpsK53xWfD/sHu7u1wW1YfGcCDbcN5tG7jeiqvai49oKTC5hZEsBy7QYmb07hZuUorwdykG+qVjEmTNRRWF0B2ysTFeefMXXgxzLwl+ERZxd0NXcAsQ44nCpu6U9xMfJTQCljXDsfJBy0jtiLP0pjOFQbQ5hpIKHlfamoaIHkEwc91sD4Gmj0AhdqPCi5sXfHQvsFnmurjeTYoca1ZdRlPRutRSRqYLgmkufqXUQteSSirT7NWV0YT4EdGqdq+N3ylEdGsSqDloa9NHOBVVt06kvw2e6iboELXRloT3Jfa1d5AyE7el1It/kVLKG9j9inWmh0hNUxFNWuf2+shOcyIxTF5Cm9xOMlqxhuhG/LYGOOco8D6C9fZIwLvesd8JckSClpw6aMYVz1X8vR8jaC7eVqKsnCnA4f8FkomBtCL8fPXr97mVUxF3nACE/5ww/xqWCvgvOvi8RXn628mDyQq3Ev80JWJAdqfhPJ0SuzQR8BQ84SohKBcqgjZQ2Up6zg/uBdbKuED+82xuHNgFL7Igd3oqIqnq2VcLJAJNJq62DuWNuUPzb0xfRdIZj+mfNaVVVGr/hVdIjexqdH/6AycITimDh1OVEq8LNG0930oQjoJC+C7G2gDyW9OhCA72PyaKfeIk7qtwc6LIDSKxyIvsGXodCx41CMzhTFtYtafEIbVxxBQkatRgQcDVE4ZT3FphZg6gzpb5MX87Nb5o6qTJ5r8TQzKl5l8MVCNE4XsrD/n9D2S8wxz3PT5Am6dtYcEfXQAAqO4ETm1ZxgWmjh9WBQm73G5XQQJF1AG7KVJ3v2oXeU8nnYLr5NixvB9AjI49twaBNyVvlQnU7i1VbCVWIvy5MG4JDU5Obm4nQ6eLbzY2wcr+VwRxUljhZQb7iQh8QVDHfocLpcGP+sL2CNL5ReUszjL0Nha6IS1FQbZDFaGvGy5Ukesr8hmAeuZlAL6YOV5TAltSXo6tRFS/2NyCZTOdq0C3dDhdzuphUreNSiDAz9p/a0w4zRZeKo1Ab3OlBgeIBSSwR2WQQXNJLSJvhLJ3lDDBweCwFNuC8jmLUV8MKB9azzZnkABlsZMWqoqWrA/rSHsft4gRkKjjHbfpjBrlzXoEuDSWl6WXF+jxtvsD1asPa/a7UMdcEeURM5qC151S157lYv9pqh6d3PMWh1UHIJrn6CvmQPSwbex0PxO3ktUOLxoEuei8oaUlUd3YxCB+CUZMGkNNaH+AlcsyR47h8ZgtoK9ovTBmXX+THSRH8fuFARgk1qq3w2AS0olHrQKmIfb/ccSaisTIx526D7wsNY4y/AILm5uezc8RGrNjzC8jeDKPkimBf3TGbCSwtIz8yk11tv8WZwUyboI5npX5/LHVtxZ6HM9A4HXeO0Y2gxmsDINVQ1hJcC/lD8bpH/l3x49FcAjjTriMm7tqfpNpPlNQzzgbthP3NLtUFxLvpw6LqUAxX+TA2Epr636txzM3ZVCenuQUYretkTmJElB4/FivdscUo468yvAHUYCflTyQr5jWONe+NXsNTTGfuAgiG8vTxaMEqtJrj5NcGX7uPVejdZEiGkuYPqQMzCypfyir8fD/tBaWMTZO+EW9+KxFlIR/JiJnDWtWVKdZjzADrMGGUYKDXk4aQ3hYwrteAqyZ18cOKAYZdFcgpQYebF5l/xkC+csJk4ak4WTI1jE+HAUEoc6eQFbqaZFt4MtSBXJougXt6f0PxtXr8uAGsmBzyU+gmMugsDj4EkI+GkVcQBHgsuZ34oRNQoU3GWghOk+oVzuFUCTrnm3jpzrnVqSyWM3LsQc4DHRpUkeP3KU/TJAl8Z/hhZyqznOtBLpWLQXySbe8kw65kWrrIHMCDxF2b1GMPCU4vI95tc53fFe0xQQw9jlVvqq7bV+oUXqiHfGSuC5Bl/iLrO7b8hXfcakVcOsCl5EqWOxtD4JZjggMYv4QzvzcIyMf++TDwhkjBDL0L7r6mQk+gW+wdJISeJ9rtFl5woEaQHUBu5XDaG9FLhz4+vciUQZR1ofMEQoUgwOSUHDDkn6ouVCTtwdLAIvhc5IKWyH5TfhLNTIf8QXYPv0KRc+DRjA8uJrv4Nil3gr5oS1F4+1qdXJ2EOvh+ufQZVnuD9ATMkW+GzjF8h8QnX/LLQI2492gZvAIKlV5E4RcQLLHkEyLfxtUfQJm0prdJ/JEDnsVXC9AE8U5RG84yvxTv3uj/KrtGdTAK9/mSW60GP1dBkGuauK/jVJIzV3dYyFp6fg+TnpVqj0pKsb8D1GmEb7jcFCQWEyx/C3Y0QP4EXiiPItUGlE2xOHUQPFUCK6CEKtl6R/gybrMEiuZ8j6oWpHR6ZwbpxhNpvsrapZZVI6jd7DSz5HE9ax6cusQoJSaiK2Mrh+CP4Oc28IH2LBPTKhO26V4Qkpm8i1BSRomtAisvv/aYMFlpHi5IdFclgKeR683k0SQebqpzZ/QTrnsbTQBOIQ9IRkfI9fc6VMvRcJYG6QAGOOzsVcnbx/M2HGOPK30iS5CoBkCr+oAtzM3IezYW9qhcEw/LKR2DJRe/IpFbi893cEGj2BhwYIoAHwO6Sb0mJnMdXYfBEwBHxgfvEQuJT0Og5FhlWY8qYQYX+BtWS134A9E7ozWj7SgC+zYTV6wXouK6vOBnYz99nAD81YTDzhrTh8bAM8jKfIbXU9c21eAfqP4bdJ5GiGg8ARkIiXrsbNIEUxM4jXFfo7vs5L5H91rVunxs8TL5qTTZmXQZ2PMC6D/p8wFhpEf4V7anRFFClynEDgVQqiZiix4ktmERU8RgesQ0gJG2JAAze/YNZlomM8QUnQqmlgXY9nH/VvY7VlT/XySqxH7T6EFrPZob/U9y1waIyGHClN6WORu5j71r78fqdbpzWnOHzEnin9CsPCw7IaDmP0a45stJxgzVqp/AxAS6+xUzDb+5jr1l8yOuSCuE98T83hC71NpIc9Ynn+Xg9K5FAF+Me5qPmIT+PTyj6xUJg1RSREeZS7cjbD3fX4286pyhuo3bq65wLaocvaps/KhxIipJHR4jb05pE48e1RwtmnyFKMIATJrD51mYq9beRdP+UORpA26jdbBinZ3g9oTRhUBvctT+1CMWQcv1NzE64U3XNc4Gy63wdkc+0HKEyNrThD8gZHiUebn3H16pJRBWPIzZuEe9HPSjKO7nuAVnLT5bOHDBD46yP7mHxOqIGsaJc1NDcOCacGHkbqHwgYw2SIYJpx79ic6UgJjx08ACMLRdKUCo9O1Kmcim/Hw1zZ9Ey41ua6hCJstrmxdzrYqiA/QOhugAM9Twyng7YUQklkpcH6qgmRrOfaJ8sJvnD6Ybpwo6ubXEP8l6xGrsTWoUfoHHICQ9bVlKBo5oJPucZboTz1bA69RPm/7KH+JgYUn/9hNmlmSwtg28KoNORYu6/LxprwWSRxAvtLECgDV22Qu4+Hm78El1iNlMji4x/rYynw+WfwF8z+/5N9v11c+cw/lI39f9/+zfZ92/7P9GcLo1yp90sZGWufoxaVtP72kWGn3ESVzTJg3JIWQqbE2nv62FflDlcEU0XWrosQWxug3xgguUwZ1RK9kmzoPa0SxFIYVG4WANh3QQiJqg1G3wGs1d9mgwbbE67H1VAI8X53bQ5dNSBusr/HwcUl1z4gtkFfqwoB5OmseeAevcxKG0o2yphVRQcanpIcb5UfIpWxlIAPgzwZbrPmwLhH9ZDbEZRg6iyxFF19jSvb6xAihisOL9kZ0+mtjWyUPqTLy69ilPtpRse3I6nrUNZaoLc/LfoKf997e6VK37C4VSx+NqLrKkQEi2jK/90b2SSBB2it9Ms7Cjv7t+JOcQjY4eswooWG3CrPJ7LZZPcRWdrW5ZaPId3g6G9damnw26Gu+vpq99FYuKXdKm3FS2lymfmMpP8rsDT9r9f227j77/y0wAtz7T6iHaW3nwZ0AiKlcHMWuPtmzB4ixFQdFJIFklqDql3s6LSgr+6hjh9oRirVxuf14OXC4TD826BEvlda7i018Gf9aC134F7xvhD1Q6u1MCWFib221R/2wHo2WsyW269xPdnvgfA6RMngjpqA+hCuFkVQ6FczFel8FNxJ6+blQhOf5V+9X9Fwk6IDBqnMvipdwVRU0pacT53pEBRe7WPjBN4qwi+DoMH/WpQqbzk0f7CualtnYGdjmpu33iV+snT0DiNvFC/PRSfA0BrzyPMmO4+XucshCsfiGRfjz/YW/y18oJqA3RdJgwf3/qsS3+DlLzBNGsxBVl6WnFo59L1LIuEIBmG+NowWL2YVbowTgQOZLkL5T7JH7pp0xTnayvPMTg4k8Ar8LT974MC9mzfzjO5OnZWgU5dgU42KY65VLmZA62jcMjVdL9+nCR/L/Zn2y/ofed5N4L291I/SHjE3R2gD6CH5kcckhPjkU50155WXLtXyU6+CwOzUyKvsj5WvL5HjR93HD4Ue9kzdYMBtcmbhhpYnnQCvxLhLKlUovaDf9RS5NANdNBBTlSeCLoAku0uDdoOI6Obitn9BhJsedZzUYeVaLaSECgSk+HGNKRqF2JT1kDHhewze6Q5s+zB4OOVjI4dRc8sDRVOwUxo76dEGXZO/ZL99eCznHo8vSUZm+wtrYnbwLdos0ipPiUACROdcN9VJgY9zzjX9nO8SkVl/XeEM+UQL6CpZRJNMmvlJiXotsLNbNFVnsXhlPh9q5Mn/rBQGfqhYlxdKz5nZ7QIzP1eGqiUMvFvzJbA+9lvFtKka+8qpUwA3rI/yUM5sDoSXon17AeSBGdzhjGqejv7wuYwuWIRtJnjOVHW897Zd5lXCr+aIF/22qd8G/B+QRIHzPBtKUy8kySk065/LhJfVZmoXLWQMkOXcqv6oHgWshYSHkHa1oTMJvmsj4JDMeBfcUwx5pSGM4lLg4Nm+Da3kUBX17agVlh77mWryc/1NCXmz53rroP5aekNfih18mYu+NT8M+fV6BPAgpM/M23XOfcz8m5LE7/h+3D4IcTIuu59wVomgEmtZoO1gmhtDU7g2QtPc0ueIfa/mwtE8FmSqXJCWx0sDAd/pzLZ4vSpR7IVGmtgYdP1JGQ/AitluP4lKqmaNjWdGZKyncbZHynHZYxnwcVNHMoYD0Cs3yVIWyGSJk1fQVLrFbIyP5hfF8GeFRJkbqBQ6kF7fQ3rKyAxFawhfT3X1gXzuzOV2emdmFXk5JvCFsoxh3RkUZlA8PfJhKN5dZilTju361v5ykWskySPnSdJTm4Wt2dFOVhxctX+mKgp5ZrfkgQNgs6SGCSk9KzGViIYqQ1BNl3nsxBoo4NVFfB1gZdMna2KlrkLGO8rJMqCDDkEWI8LViCA6SYtnTcwSnCxBjaVRItgs1fL2fUU/pf30eQHMK36hI5Nm5KZlfWP7Lxa5dkQqqHPdhhn4WbYOk5kjqZSLxI4epUSQPSXTnNQG3e9QQ9b3nHPsc1vvMuJWEi3Scw7/hsm/1Fez6SSelQQINeerUKqI7FdrQ0i3y7k+WZdm4Cj0Usw6Bi0eJsqRwR3zx6g2XEnDfJeE79dfBYuvoXaXsL7RSJQ/ry2EaP9sxTX9Q7CjM6Bc9Hfw/owODoOuq9gTalnzXoz8gacfBoOPwCpy8CSy8MBVSRp4K2D28nXLlJc29HyPXaxhOp6c2kbvR2N99dedp0JUopb0u2Ks4BKbWP3WmHe/QlzK26wzlzCN6XF3FzxG/FNerJyxQreevttxj11jd2zW5K8oJBBMReJb/8kRPRz/XANa1pu5M0g2HF3CDbpEcW4zPrunM0cT1MtfBUKxgqvJGRNMZ2kCzTVimTMnAZPwe0fFOefzDzJBc0FolJgTfYYRV9tInuUL2yLqyJM60liI+sYdHAuLxdAo+DTYFdKvQ5Vd+OVLo9R7HeYYr9D5EdOFx1tPoeSC/SL2IrODbiRRBJo5B33+bfNfswvFdLch7VKOykteiVTj/xAnh2OVPoIVl/9R6Dtl4DTPe5+BhgYduWe+bvC73MezoVtpcF8cvqYYGogvoFwYxoNY3/FKMGpgJmiHparHrSMlXENf2Owj6hjo5Vs4nvp9jv02UEj7csENZpJKy28H2ZBrrgm1u2cXWCroMoe6Hlv+IA+TNgNgW1whPVhQW48K8phWhAE1SjBR+qUJfwaVUi7gExGNvoGg8Njg2Izk5i/kf4GkWxJLQ3HqfKsM0LWTXwbpXbolWNg+scrSc/MpPvrb/Gqb2Me8QnltYBoLvSMZOOZ7Uxf6AFl+WhMhBgy0aiqlSzxyrvMC57HjEB4JgC2JaZCldc3mXeQSQFmVICvBBrJJurEpS2Hu39AeE/Om3ZxI/QbPrv+BAUOlw8gSSDJ7u85Sg0TwlMx2q5CYAto/BI1cj1m9RjLiKQFOJwqLOoGog6g1z2rHMJH2J/TWwBhbsyD7a1gUyJf9+/jPraz/gRY8iB2jEisAWezxrlZg2o1Yo+LGgKZG5nVaQCNXMzyESkJnAk+Bg8Wifl77GEeVwkf0gGsvTsUjfm6kIMF6LSIIVltRE1HlMkEmyaGfuuSecWV97A4oab1J0IJqO0X/FaVS0VNsPt4/7L1sC5UsMJOP8fX3X0xqirpqodNjfZA5iZxYIt3GCuN54qX8IA3E8hH40nCgKj7eG8yWRz/UgHMz+iOLDvg8ntCQj6oNdurA7Ej/PFQXaFgoXVYADWlfN7rQV4O9FzLKetFkD57J5IEwRXdGXS+kD6XbzAh+j3F74rEhWcwsiQLKf6Gz4DahzUlSRwxw8Iw2Fl/mwBF1bsfms5Ec+ltFjZ8gQbFEyko7YBd2wQcNXB4NGSsY23Yo/zuhRHdlbcUmsyAsSbBxtX4YXL5HgszulLS7HeRmC2/hVaupL5fCsEqGyqHjwD0+TWE/geg/iO0L/qSpKwPWBIBi+O3CZWL7qth0El8c77lh2jhYx2xQK6quairHdwODPW44Lee2mSfhCTs+4RHwE/Eheb7vMKyCHg2H36ulRQclSEUFDR+NAt7CJ/I5fzWdwLN/JXg05hr7/FUkEsK5Sx0V/GXvnAk8Ckwgv+c8DsJjNBq+fTzz4mMCEHjc53i0N+JKnmQpjwgDmr2OiRMJFbjYML1O+jswSyNgD9HtWSf+XcYnUVU+lOsGTzM886RPYmHE0/CujD3ulNbt0/jxfh3Op2YTCbwkpys/a4kCdqkLaF1+s+0T1nLk/oofK6+C047+MRzTd2TuzaPHOYl2+simed0wM0FNFZ5mHoArfKfF0AwV3uhajtnYwVjPy93LA7J4O4rdTbnS9s5cl3DUtUBUtcY67tB1pJTWe+PpBf4vWak1zNRi2dgq0JTcoRgnyyFVGddwHutHZ50vQFbLF4qGelrmBrslQirBUIHNIPWn3Cj4yY8tAZQS8pknyHjK7Lab2eAlMTTeWr2+HqpwxiiuBHQnes1tWOuBRL6CQZw1jbql4lyGZUtYLFKKbH6n9qvBgOjJk6kxBLN1pxOHPQXSVCDxiCS7yNTsWiTGN/iAzY1OUyXu7DL6WXLmLNpF/ATMQFXsarKmLFvN45u6zz9t79nQNx3IDnIsMJ6Ux9RwxBEHczx1WyytqWtDrZ2/JpO4XU4r01nMrVA1LBeXRjNXedosf51+x0ppAN30l4mvyoKJ2C2+QvfFEClZ9mVBXznOMGZBg9gNlzE90BT2NMVdGFkN92rUK0KVDmERGavjdDqfbfEcboNRuXAdU13z5iMcQzLfoHH+R0nYHciWPG1LeZ+5pt8sAIzOz/HpPYveMlRx0PTVwGY4nKHr1/8ljlvvcUhu53d1cqyMAdscMjh5PTpZZw/p1xvAKhMx+6EwVlwyXxe0WX3SvZpZM2/Mp5/s7lzGE7nfz/4/0P7N9n3b/u/1UpdSAefWDr6zXfXbAFQ7esuZGVs5RB9H+bKNiTmigXQ6tTCsEsw8DAAdkMi03N8OeBdONar1Sv5gMcSVxOvhmuNLwjWQc4eOPUslN8hs1wg/nJzxrL4+EbUkUrza7b/Id4KhlCj7R8HFE2GS1i0Qn5C5T0utZECexBmJ2yogN+KXLU+amshjTPz6E1hSB5MH88F6QvQhUHL96HVbGzZO1k71pfW7UbjkO5FZORKRi5bq9jJDb6pLEJSGxT9tUiketftPFXLXPkv7TGzmV3rV+Bwqlh79R3OpT/BcN/udDOfg8o0QGwGW29NZfqu01zO76MM3tottNAXEqMWaM57No7CUwz2FZbKY7lw2PC2p88YD0PPE6O+QY7dQcCVJCp0HRWnh5cc58Ng8Lf+Q4mC8koOFvTnmCOL0IAzjIh/V2h+u5pf7veMCRWowJMWuEJ/aPCUYFRUZaCXigCZSf7w2+DByGVK9lSqPYgMGyTmzmCA5l1Fn5R/gK3REv19oKUOfDXKRA+ADyIZEWd+hsFDP2OETve36s0EBkZyPncQ+1KfEL/ltAmEo9MJDhs7YhLIDlvMrtzOnClXBrFSm+/lu5uPEK0vp7ABtKxa6OlMXswrNZ/Rp/J+Nic/zfyzG0RSzaupZBkfGV4KhK6BIgDxT+TR+qidaE8FM81sIv7qG0JCD0iN+o0pW5Pdx1rU8TDsikAkh/ekwK5kBNQNlu/PmsPNsqa8EQwP+yuTQJeCRzMlXyDAdMmQGTbJ06kPY3aplj9dtlGTdJhdeZ/i/BOhr6BOdmK1/LP5V15R5jYqX+zbmeejQxXHvKqdxJ76ueit0bzT7GcaF3vNIZUWhxeqf25hCDR9xdNvus3ERpMo65DJrebXWFmtDCjGWZIZYYTS0BVc7xCAXc70dAa342VrS7a4YucbouCL0IOK85+J/ZVr8XDXBjNu98Qc5HkmOlUle9uu48MQKHHANosB/EQiyWC5wldNdtDLAFPy4JStm+eitip6OUbQp9Wz7GlVj7lDm6A/N07Um1whQdoq7vOfRVTxQ0hAhKpKIVkBIikZKEMfAwRrChR9d8MGsqAUiu0qUXvBy1ELLP2NU61200UPj/lBU7UXe9gnhnVl37K2QvzzkFlNVezLsDFGMM2pnW8iQjEver/Q2XAFM0sa/cDkzWnuy9Vd//ROExHuodTpNERx2a+T25G7px8YnvgBRQ4hI2Wye2Zg3d9xUsdAlTXszR7Etkp4PA9uq7xqcflEU1K1hoqM6VysgZ2mEJFw7bZCSEPmH2ZImEhmvRgAHXTLQR8KY0sEwtCVeEmxQlOVD0ZZ+dtOfShZLlxGdPG4e8ZaZC6k3OcKWfWhWdmLboenbh3Ml4Bl9zyRv26/amU69Z1IkTkGp1NiSvsXMVYoGWOHi1/h5tXvyE+fzvGcR4SD2PFbaPEW3PqGY803MaR0Cqfy+pDjrA8Fh0Xg7fqXENCMthkyE3KhdyaUeTuBgMpqIloFYSoYHHbTIxt+YSaB8i1+HlmfMc0/olHIEeLvjoMCV4JU48/l4pFcduZyLnE88Qkz4NjD7gBv8OH6/NJptvt3Kgl2B6gAwpyHeD3cRIUTUm0I9KtXm5/9AMlRc9hQCfsrYhV9ctxYpuTDHSukZEyivFKZDESS+bJYZksl/BgOEyU1uFhOkgQ7Ux7n4VyocsLeiq+U15bhjR5jmdpJIFGrQ0dA16VgjEWqSGZmMDTTwroK+L7QC8jlsBJfuJb+PiJYtP32CxwMN4l6fwCpy5lg+cZLPk05uebPnUv8C1Zu7ajm7Vz4rdzCtBs3cFRU/CM7r3b57VR5SgSzK9Pd8zgufwqRJaNpaGynOO8vneY+W0US3pzL+2Hl9DVAZNOnccjKWpUlEYP5sQyq9MncDVlKlc0rYho1kGe1o1jjWqM6a/XozUp75ELDt3kiTzBXNud0QkUVbG8N2TtdAQcHZk0WFk0O4ISYkTDkLDXh9/NhkYr9Zph5aDW/53ikvLBV8brtfn7yIk5KsgSJT0L2dlhtVEiZDvbPg/QV4h/aEAjrRb3bCSwqA7Df83zOZp/lqVvBzJV3oLtjJ1sa4Oksu8Ijzhu0cMVwXi6AWXsau9eKvVblWrGnxqKo2SxJcDxzNJtTx3E9bDaL1J2g6QwAZJWGV7NDWVYOsy+8zQ3H84pxieflJEENLweBrsprvwjpyKvSPOaXCkUInWwTwe7aVl1MzdGJPOQnJHvNdmUiWnv5PcwN4FYNPJ2jw+RoqHi2uuwZGAomsr7BbdqrpijOtaPC7KqFFiyDn2m/kA3M2QHHH+H55lPpkv0JvhKcbLwFzor7ZehFyjqfpM/F+9wyjHUlLW36BLKq6rG3CsakxYoaNk672Otuf0/Ls48yJQDeDIYPm6y9513W+mc3Dae4Ud5KkfTvm/Ab+2MEeEgNogaqCyDkVPkydtcuXi2E8gbwsOFLccGEiSDrGO+/ib4G2FkFHe4E4gjvBynLoMM3EPcQ9XSCfRBibsUon45CKcVugc4/YWv/Mx9lJrGrEuqlQLr/eMWYnYlPMD5H1P58tt0b+Nhuej3sKppk/MgYP/CTIUJfjFSn9myfsEs87ie+ucMVZjDGExkZyaw336ZDnxt07FdAh55ZvNYnh6Fth3qelQybb07j5Z0X6BP/O4Fmr+Cu5AlFbq6EmdnRIoFZ21KWMCX8Z6Iqu1PSACb5fCVAcZZ8CO8DO9owWbOUxxqso1J/U7ynY48JG6v0CpqbX3E5DtKs0OTMaKo1iXBkPJRdA1nNu2c+4g3VCn5q0pDdAb9BA5e9bCnkg1YhfBgtwHL51SFC4tqvkZgn1QVczvcATR71XQ3X5ghf/4oAQO1JFfO5V/xKPm0tQ+pyyNkJhmh+vvEZ51RXACi2aXFowkSyWhcO8eNJI8rziJwyNYG9ofF0wRJNfJI7NgEeilbB0OBfoFAwp2WVpJBhBw+osfZdWGUT5+pP5ELC41Q4VRDQUvi4Yd04mf8YNiS0kquebu23LsmoVEowZKAzQ8QfMjehvTCT0S4syH1GeLvLZOQKr/llLWdO4TyWhroAu04JWWOAPjtgoEhsypKKBho4HQdDY9cKP29LEpx7hfoBN+hgGkuIqY/4bWqg+0oYcNAdYNbaQ/CtbkywxvPsXFMMJbPP6z40/nyS353VFVDpgHKHVrDnUpeKpHC94RwsnEWLtF8YkbqFdgEtIcPFnArrprjWswHwUVABHHkI1keCrEZvyaKla01fltWRmqiJcOIJOPIQUbrTfHtfC3rGrXa/GzR+ENEbjPF0jV7JEz5+RJf3I4gIEWiPfwhCO6Er2c9of0+QW0ISCV1NANz+jqqaSxxtKvyRaaFlggWb9htkbQYg2R5FshVup0ynvs7D/gKgpgQf2018ZIeoXVYnFkXqUhJ9/kS2G4i5Bs/8D5jk6cAsoBcwAP5SbvqNTz5h+owZ4JtI25stmF8K33Z5lZnNmyquFXe1DTO7jceJkzMWOJLTH7scANnbsevqse7Oo+5jnwpPpZPatR/4xFGorcdHWd25FvMqTkkMWFtnPiM5cEieNa82vlRXwn7N9fdFMk+lg9BOmBL+JPBYJWq7H1vbq9hn2Q+hXaC6EM6+TDtNquL8Cp/u4rtKWwGlV6jvKKHMHkjHjG9oXvqwSIJ7D8sFsuiuh57a/WDzvHe1vcoNinoipIBetoFQ5SpjET2UMuMbRBePAyDRGYxPzhLQhVAxrILtt19QJPvulfEUfYqSRAAZa3k2xAPakXAlGf0bQ/NZOPyVZQ40KOMqhoyvUcl2WoaeoTBgDw68JpB/Y45EPcH+2lt0Ssrnf+VD4otd/nt7OOKw/X3m6OTJZJha8VxaEw64rm9QG0AbAL4JSCotPupygtVW+l++xesNVnkuEt6LL1KLWZ/bjl1tA/mjwQAlOLfner65vBUnTm5ZYVnpSI8EMkBlOj8Zf+etYJGCryPU455rBXaYmtaScqkxXHwT7NWodC4fR3IQqYKRcatgtU5IOiPeVaH/HnKDNlBuuI654SwBNPRvAiofBXjSz7IJxhRDjIfAEGrq5xlHHSB0gV3YgktM0O1OA7EmejWNK+n4UrGJRWn9le+q7Rcc1B5iy+HjvHD7Z86ffPu/guS3O2ycODaT3IxbQtq8xGX3N5hExMXu7K5CMD29Wi2zb2TjkTzZ9sn/msyr+z3/2/532r+v4d/2f6vV1mvTh9E/aAad436jwH83JcYTSEXHBIri4lugDSKvqiFRJQ/QIv1bWmnuh4AWQjIC0FZdIdOmIt8uWA1xVXsUPxNiWs6g6MPYgNvVBtAGw/5BkPwjZG9nRNEv9DN4FdutY8d8VtGN+aUQ0aQhyzR/7zP71WCgY5+JTG/zLuujILM+tK7e6DnAVkWcxoKvJHSoP81pds81auU3DhcncUv1qrC8Q7vC3XWoXcHlgsBtDE/6GpUlTXHu+ejHeND1eHHKykXaUsggOZXWWiio+meJCVO5CODpbOG0SVtCnH6PgHFHeQLEBVVxpJe2wFdbguTwYrlVZvBr3CbeUHfg+abfMi0pSBmkv/weS6KFsXWqGoo0rZQDMMSw0NSPnVVgU5Xds/GElZzgnRDgH0sUBDL35myWl8Mps4Y5Fy9C7IPuY3wz5vB4pED6/lrsz1mfXwSKteAYnJpCJ/sjjA8q5pQFFmQ1VsrvAcGyHX8ZJgQX0taglJ/CnEM/g5OrNVD/alMuVjx8zxivNDjMlmjooVcxZWgUsz6eTS+VioE6pQTQAI1KSAC5HIC6G7fPjWmwSitQtJkbKU2y8IAvtE1ZQXt5kuLYU9WwMmEU5Q74vhRKtN7BSicGlYFGOW8SWTqKGldQybt1sVykix46nXySXy9+giT9M3m0x61Qmr1SBIZafgBNRR3Oe4wRWQeBzd1ysLJXMkGSEHIgJ56EO0vcf3PiYHIefFE8UnGpPN82rK/w/LtuLUJvqY5iB0hqpWN821UH1PYP55+/rw+/RtUwygg7qsvZXaGc99cdXdhdJcbdLOwIPlUnPJ1VmbTU56F3PZcWFdMU5zqrC2gfvBy/+4+xfOhWCh1KGbv1sa8SmyaCDMmNTDQxKtdOb+CEWhL/UzRJxu6EEjusz2+F3V+MXZLA5tTzdUYHNleKJMHU8kiIHQVAjU9rRmcL2atFJkh2tvRcU+3D79VTmOs4QbU2m69N1ViixggmLcCxCfTWLSO+4Fla5b7OprjlohZlbSu9ymsBdvLt0PHAXK6YZimG3Kzr15RWb0X2PUtay3bI3kXQZT0lNh12J/wSAQN1XoxrjT/Xqx/0QmhLoNZDvZGQfwDW+BJrPOeuY+Qj2121x1woYbWRVhF/0jzuZ9o3mIeqRhmEPxX6JW0z4LUguJx0ByqUjm6tM7EyEl5rvJK6LUKuJE4l81IB/Fog+nNzc5n75UdcPhxHzHwI+hb6n7aQu+Mxzy3LKJwauc7CGqNuTXBFT9cduyZAwgRo9yXcnM+z8TsBeD4QeuldwcjkxSJwA4Rfi+eFAhi1cx+O8FGKa2trSmiggdf9fVndYQFqm1ditiqT9KMT6KGHEwWwfMfN/+jwTAYO8jdlj+wSQ4ZOxk9bRKPgMwxP+vaehMihos/YmTWIJaU6Pkh5FLxqgBHSkX2ZL3PVdyfnEx/mbs1lIUsJIoAM6GUj+XZRi9WuVibvQy7N5G59OGpWMWKlk5zEVdBzPTR/k2opnD+uT2Op/1dktexJoGkNmD1sDUmC4016kx28mqXmXOi5zl3L0hrSjxv5vTy/IxcIyawHi2CIYKp386nGKAnJG8Xe7LAy0FBIe13t7yg/dJWscr/7xlmz8XUq1z4kmU2mF1hZDhctkC2NEElh7pUC66W7AGdeFoF2V1t15R023njFfY/uy0YOon4qbKqAPleuM6TmJ0+nxo+jLXfwaiGYdelsb6fncpkXijr2QXb5TiXXDu8Ew+UWB4WsLh456cMO5z1y0lOdzr+dOF6khkrX0tWsxVRo8R7oQmhY9DTDGn5Hq4wf6HBn/T3ytfckQLy7qwt4PayCHgaYk3ia4X510LpJU1l++xkS1PBi5ydRl61TdHeL7UYreTydbm3n+6Q/iUxXSv95v1sJGclWIYB32duJM57kt9GRJPaMYW/raDFOXYhgPWgD3OtEcXUIlQ6vOSCpSZPbudfGDy0v0LjmLHT8TrBh7VWMDjjsPnzI7T4iWI0kAkayCrUjDhwqnmwxh3DnEsWYw/N38mmI1/PyfoCRA2Doecpq975y+PW7uf81OFJbs7mqKpcdyc8x//YDXAl/h4WnPMAmWa3mh6Jg9lR51afzavHFD7J6rJEDZohOAXPkA17PREZS6XEgpG1HpjwgQEm1zV5FT3sKPfSC5euvVgK9JP9G7KmCs9WwpExDldMLwIGTKN/bhGuqeKcI/qzupjh3jXUFw6vFuhyhRqyrGWuEXJihHgsubsZkFx97cnWAAGgk/wQ1xTiCOyGhwiDBMB+IqWPfaRzFmKOEWoRddhlMal+x1qh8kB0WtMC7RTDr6sP3zPXGNUcZaUR4N9ocweJwtXO5Q3kuH7LsolY56Svd8pCSLFNS3pQSByw1Qao3sMuvEe8UvcdPJlHj8HK1BkkbCLe/g8wtsH8wnzZeTOvUpXQteJ2nIroLufOtjWFnOzQnxxKutqGTwHj3PaL8vewRQA7pxOoKWFsBQ243VgINNf7c7bSKz4vhCT/YO/xZ9BWenUiW4dG4vXwRKlZBXwm3NHhtslglWZFdgXRSlkG2l6w+4KstZlyLj/G3HPD80SeGt00fMa8UbhX0JuXSRaUaQNNXWZm3m2qHxDIT3LK3Er5JwiNCPtfVUqzwXpdniFQfA4vLcs0/CDhQSyLAaiwcj8ZRABmrYVtzQqRz7Mprz8naJdyrzAYqLbmWVpRWiMB1kZ8LSNPsdSGp32cHy69+4T78+/InxLg2xoqEdO+tJJd2BaCwKoYbFaNFLa1BJ6H5G6wtacGJILE+NNBa8XGkCinZG/Og7Rccc0kAB8qwtudL6ExHof08wbzf3pKD9U7xpD80UPkwMvQrKBSgPxkrbUJPkeQKiIaqwHB0LNz5BUovk6haicP3HNkhK8kM/ZV0TQsYsB/Ce1IZO54HCtM5V28WB83Q/8ZoT/2vyru8m9SPvikeGV9/OU/EH4pOI91aQG9XTL+RBnrE/YFkLfY8T1mDHD2MzNKOfBEKU2LOiHkTPUSAYK99wcmwWwTI8HohXCzqBje/hvLboA/n8c2FrD61huAKYRv0tV6DnR3AnE3dVtenrpUNBGie8TVGrQ8cewQOC9BgrY/0ehGMzxgm5nXRaTBn44h9gK+LmpMWtJpfRsaRVPiYkGz2TQS/hgwo3cFwV5JzmA9MNJaJZLU+AhxWGt+Zx0G36qKMIftnEROKuR+TPYEfkkezJOIFtraXsTpqhJ3tsrVHNviER5v8yDeH9vFTslBv4NL7sLkR5S2WEXtNxI4KEvQ8bBcJZpJegF4bCXJ45JQf95ch7Xdo/qaoJwh8XDWRj4pFvOie4Hfqb7TLboKsKaJFBpysGK3ozhp8ladzI3CozEh/o27adCAdiAVekNWM00Uz3T+OIz0h/nWY8ZwHlFD7Lm5UhpFR1UZxHVPkCxzJeIju+cuRtOuZc3g9AapUODIWlcPE95fedB87ODCPBpJr/231AdvjZnDLcoxyw5W/ZPY9uuFR5mgTSI1Y4P6bN7PPLlVjVZVRoyom2xKOM6Szm+WkUalROXzE2CUHBpVNJOR868OAQxyxKwHdhREzQBcqvp2UJRgd5VwsuMjMqLusGhNEsJfaTiPdWq4nmumpF3XQJ/j86FF9ABLOTuJcnPjvaI2VYOcpMT9drYl/R9qlrGL4GSdjS2cRcGMSFB53v3PZ6XkGdZl9PtX1iS2YTNvAZI5VeEnit5vHA+lejEEvv4uyGwSWHHYz7gE0kh6OPSoAGNc+o7zDLr44toxlJhjrCwnW9YrnoyQ6uFiDJRfgjxBIeJgb9YWUNX5Qr83jjNCo/xZwPDIyUlEzF6BLTBdRq7IiDVmy8cuFL3l22y1Gxm0nwualNCBrOGRfwf4ksf7U19jFebXNvxFb9N+QHbKCABlCVSZRYxOEvb4pgQi5nGvVkJRdyNliT2IaQLo+j0XusIbTM8bc3UhFJ9kwTsPLwZU87g9PN/pcADdbfgDAtA4PMj1UrLWnEidyPPB+Ya+qdARmfap4P8FSoviPq3Mg9TdkGQLMbelY+hjfBoTTUkrzDMpeTW+fApq7wApSHaAh177gZLwGfxnWVMDhgg4eNq21HG58xcXyHWwO/pwrGd/RV+X8j7ZsLjAbeBHwc9rp2q4Ls1+dSO65pe5jknRVDPRxlSDyarXMPoMLrP9X6793q1ui7l+m3/9O+zfZ92/7P9EkSSI4OBip3n3wYCE0mMyKgqO8rFrCyaTBnGw0BGeT1yBmNLT/Buo/hlNVjeTUEljZkYHGPCF7lSyCv37p7zE/spwAGR7yg0Cv4ugA6wLn0rUglSwbjE5vDo2eE9du/iYY40mqOk99DXQNus2r3cZD6RXF+ftqEjlohmOdLrHfJv1tNMygwZPpGnGcXga4W63HV1/fc1Den2yKW8+L1RPFM6n9fE9NEYZ49i46+go9kSK/Q1Q6ioURvcYIJYKKPTVfUNefa/caqopr3kNwb/qfhsCfbf5AdnoF9qru8qJ8imFGqNT+w8SEn2DqfT6gG4+2elPIJhgiQSUQkJIEarmGbrHrWT46CmORV40TfRj7ij/nVF4PslU5ZNtkpfxjkxm8licCl3H5zxBt8HpeAPpQpub9SbEduvvnoaFC0Z0eP5kmaZDdDBar6kBY/kOrlSiorX1RoE1ji3OHCH64WnHLLbx6S8iW1c9/iQh9nGAt+DeBrsuJdOzkAV8hCfJy1U1kvzjFb2yK2MWaSHil0To6+NYJ3CVM5EIjGxx00OfqNQLletRthyrjOGaGiTFHGB78MNOnTyM9M5Mmk9/iFd+mjNNF86pvU0JHv096Vo5A+iE27mfavcTiEYmAE3tAB4ifIALXxgSWlqlJtcK7nV/koQhl4ksr2dFKombGq3cTKTB6GawNnyG12TUOGfYQ1SOWEe27Cw1+r3ZfxS6mBMAdzW1KXAZ0mcn0jxLLVmuZMCYSHxOoTMBYfZa+Cb+icRXblpxWgVS2VcGO9jwV0Vh5IYdVSAEXHoXSqzzdMoGuDRdwthpu25TsFY3LYA+WYaAPGO1eEiLmXGY7jrllFZppoa9/G8X5IfY8+hlAahzLz7JSfuM/tV8NBgbfN5iJAVZa6kTtigVFwxXHbOdZ3iuGUt8TDD47mVMxGz2dyYv5LnoFCWqonzudrxIOwyUP8686oDXBlw2o3x7H2K2PEqUJU1xb5QrI36wR8pDFNg97gMoMJqkz6OK6lRHZ8E6Jcp7syFxAh+RIbIDKYVAYgw60fJXRlc0u/8rhFdRzaiPZWAlpLttT4aTIGs44+nLcFUh6vxhq4p6GXpthyBmQtYTLxwkt70uDnPc4V/kyRHrJCpac56MQK021EFE6Ej91sGLMalmNSlLzSbidyy3Oo5VK3X0VwWMYdmUIp6thZDZsqx6iPNdrvXo2sIaA8yOg4/dCutJWyS3DSm5Fv09U8Vi2Zl8XgJatYk5qLLeY1uVJRrefzPudXkEqXay4dqOAFnS6tYNKBxTZVMq1seQiU7I/Z7K/kDSsb7w3UNM0syeroxzu51krY3d63id8XHbXrfFv3+Mkfvhy5s8Ta5GElU2D7mNNJPwcDo3sXiy30qs8Gd6c+2P28F0YpLU6KRxmazlUF0Onxcy+9TgAw7NhUcUs4ailuIIFfXfTNOMXOt3aga+lyT3OQXTyXJITIFhlJ1Cfh+QdNDTn0s20l24GeHArdJf/Myv4b8seqeHTGYMIDY3k5c5P8nqPcTy6IZeyMKVUnSxDifE4l+o/xRX/uXV+bADr0r6iwi5R1QCGquaJmi1Dz0PHhWApYHPCM3TWg76m3j33XB0hmKW1yQOVWoLY0dD6Y6qlCD7KbM7WSuEEnmpaBjGuBEJFGgt6+PGhazrftEoQ+4Co9wdUtfyZFecWE5//PEEVXXnJ8CEcGiVYD8Ht2G+4zPTDv/K01JGM+qAt8XrPdgvzgi/xfAAcqAcLow8qB53xBysiJR7xgz1joxlUXylHCNBSIwJby/Oac1S1AUljJDg4GJVKonfsRhaFi9qerXW3RX0vp939rPelPsnBdGELGfLXwPHHwZyHpPElzSpR6YRnmy5kaoNRnh+UZBz6SMpcy0qSrppExxFP/djgtgQnfESbS1cISf+YwvIOIGv+q5z0P0kcH7UDLhyMFDUAWr0P2mDCK36hdeTee09KWwmrdNTXblb8WZKAyx9Ayq/gl0Tr2/X4thSmFcASUy/FsfGB8bTP/pEIScfkAPCv9qr1Zs7lpcgInjN8RLhpKCuvvEdJ5AzF+bEFu5nsDxEq2Nv9PaQ8F0PPaceOD7csRrd0syxLIoBbU4bKks6fsVae9oe7LTuTKn/uuahKyybdbL4qBb+qlrzYaC1RhUK+mBZvgT6CbsZL7sOtTo2Q7JtgF/a4tYI3dV/Q3jyEpxK3oK/5XjHmwMKDvBYEMWoY7AMGyStYpA2CoDZUOEVkpdNV6P0f5NG8W2eENP2liyKBXKkXSOzp0mXB5sAjwagBnu7dC7X9GcU18gnjiMVGtRNy7OBUe7HzbJU0kwvczNK6iiMYonnO51EWlcGNBBgXek7RLTWYzMgcgVgHpzIJLsGiEUl8lHCW78rgVE07xbm1NSL9ZbgWD5oaL3nHzj9xsXAE1Q4VFU54LL2/YD6dehr29SVoh0Rml99ZEgHb6kFXSVkzMv7ug+xpu5ruevgwKkeUFSi9IoK09YZzo+tmvimDYxY4VNTynvWvj2UlbwfDy4Hwa49GUOS5fmpZezaljqHQDkOlbqIebtvPXc/PSX1dDfWtjXgqH07aPIA81AYy7Ynk2QWLMkh2Cttw6DnB/Ks3gsOZLxBb9Dg+Jfdx2PQJNJnurl0rlZzjapv9fB4KTVR69FR6D9n97vLscKLSqKwtK2uIbTiOQlMrLlTDb7dH4tB7fABJgrm3HuLBHDF/yxsCF2aRm5vLnE8+4tzBxlw7ouXW8XAOHfyI3L3Pwc357nNj/a/RIvwQr+05yv9j7z/DpKq2dn/4t1blrs45d9NNzjlnBEQEJAhmBBEMGEBBgpkkiGDAiAETQUFEySA559jQNDR0zjlUdcX3w6yuqtW4z3Z/+J/zPtezx3XtLV1z5TXXnHOMcd/3KAic4X1Z9PV9iuFnHPRMPYDBqQSVENiaW+a+tL/1G7/sKsas/1IA45q9KMCyQe35ozaEVCuM8XPip8oWa5nOn0LkEGj5Gi0yBBu7uxxOue9I6PYt+DV1gerEM0lQQ8uaTzylDzT+fHNnP5tuiXXBK8lbYJ1afKc9vofw3kioULmYrLnOe0QiJbijUK7J2crCoUFs7SyxLKEvmwo3CcDL7m5QclrBoPgt6TatSsfAgWFC1rLkjLtdBpz1Mnm1OSJpU3mdaHUd6qquaK/+ztc55yBBzDtqqvig3wRecb3aaf7fIBXsg6obcGMVA3WP0CLQw7jTesngOXGSqxFz5r3nqulf4QVWyFhPy5TJtPdiHBZJPWBMkUiAPpDNm67c3oflMGKdEymsh2d/lR76beHH/I485geDgm8pg7K6UG7bdeTaYPfZH8ksGg9FR0FSQfsl7m2dIn1OuiZWgFlNea51zim2dpbY2lniXKUyySxJ0DJrJQMupxFbIt4ndSWibpWthvei9jPVHySHRkhHBrSAptOh+AROp5PN8mNcaDSRbekTKTfeJxRxRt4C38Z0rjzEG0HTCajpzCP58Kp1lgjE19yGulL0TZ/l69zeAPzS7kd8b80D/xbQaCLVzkTeK4jkdB0gOcVaqviYiAtdX8nbt0fzZK4P6eEfesafK+9A9U0ktQ6HK1FztqAPZl1f2NFJ1O3L+IXu5lloAJwS7x+6JOQ/Cw+Le8ZTumBPxx+5Tz9U8bwI7szHlcFk/Z1fA5wrSuMoYl4s1/yz2EskgtkXGtaEnoNzaD/oOPSDa62B0y6mubWKx4JK6KKDx+r28UJ+guIYBTGv8Zb0J9f9P6aNfjhqhx/ttaLmXG3ERAVL7akbPdkmexjqGpeP6JAsXsw+T3ylHgTrXf+r/huUJDjYqiW7OgSyu0MIu9v6UVHr8qsr00gomErbiL/cScSnfL+CX3xEPefwPlTJHn/Vr7a1uJYBO6HrV2Jc7/Ej1Q4frpW3YFf6RKz4u7d3SmpqHaKq3YnT29nPTvdaGaA6YgjrXEvFpbnJbDVUgNE1bp97lT7lTVG5rutGSTcqOmyHsD6oC7aSHHTW/cxaFb1FzzgP2EaWQW+NJr54Mr9GwTR/r3WgMY5au+fbluoTcmlfwLYWRJ+bQpiXD63G4CkLI6lw+jbla8sdCu2COd/S7KX6UJ1O/9yvGW6sP7aL2SfrIagdRAzE5uOJeUS0HMvzzz5MXxmG6rUK4HhflYq+KpUbOA4Q7ZvK4ibbGeACJgxNHgpXFsEfjdA5RE9WyxamdnqZaOtaz3XZagjVniFUI57l5Sb5qM54+VuWcvzVgkQwOwi+iXnbUzev8joAxQ69e4y8KyFVsI9H/aCRGj5KvIyP5GI5ZKyHumKOF/bkhqOaQyZYd+d5eCAbwsW40iF8K+19PKBDlSxD2TnI34tP2R94p1UC1GmQuUmwBm//6B5XuhW/wnPhhXTWeYGGLeV8FHqMZwKgux6eDi4Vtezrra6YRhoIre7u+kH2jFOWcjg3gxd1SygM2syly7d53Mrf2kogATgCvAz8CiwsKePwNi0JQz9h5YoVOKuzWJuUwu4Y0KuVYHu75Er2aZQMUvez/W+y72/NncP4f/QA/pvs+6/9jzBZlomPj0d2mOCvAZD6EXm2nu7Ar1OyCwev728Q1BZSPyI5YgtHWnbmeLOBmJ0uGMcp4XibIx4lVuOghwGkNEgNflZxPrs2yB28kJDECNX3N2i3CGJG8HnSV3xTCVEBl+mXsMG9qPOYKwDpB116vf3vZRRdOuohIZGM3fM7oekwKD2a8uAnPRv6JnO6ejbpVQm8Gwwr4lyO/s2vhNN0/DEWx4kF19zGWxlk6wl6DyvHMqaEVRWiMGzfLHAEd8fbksqOMC9IoGPtyEjeM6RfMkWtd7K5LISa1vC1Ssc/sdV6Pf6RUZw+0IIRb5/kja9XsmhDc/Ye+dyNmJJlGN9yEa/2fJQL+YOw+XglX7RBnK6exdmKJJaWwSOZ9yhr6EQNZnOluMfTnTbSu3QYf2eP+cNfseDjvKP43aYLI9UKzk5wxGH/5xIFTz7Kc8lLmOgnAgUO7XlF8sru1470WnFdj0SfpHXNFCjYB0fGARL7fNOY4pUXajj+bzUls7UGxuY5+KFiDA1NLasI1ufTK+5XghrUgQOYWGBgSRl8WKrhz/LfQFIRGRnJtGdep0v/FHoOzqFL/xTGjH2dyEilfF+d3YdqSxCy5MAa9yT0WisWvCGdmZZv5HQdaFRm1JJJcc6RN0ewKQqM5iYMvHKLcH2s8qIkBza5msE+MDEixR24rbefgh/njRIICz5CbYQIDAf4+/9HiWWN18K83kIqf2Bmj4n4uSSZfO2X4bcIgWSNGEBmnSfpI0mIex1fDZ1XgcNCuC6HAFkgrOUGaKueeR+TniiSKbtjILjqgKfRYSURIb8HcCYOBjuXK/ZvU7Odv2KhdVdfjjgt/7j/jX9iHn7XQllUv5ht4CB6Mwqvxb1KodkrcBc1lM9KB1FkF+Nm88C/FPUm1Wod1WqxUuxXqucZ4y+KY4db8uiph0sWmF0MRTYvVmFtNk+qMtzJPoB4nZJ1GCDFElU2ji46ONTlK3wLPUEOSfKAGJpoYJY2QzjNeBzmcBVcT4B71FuU99xAEkOSJPBLFpKYE8x8VjOD1Oi3yPLdx6GaD4XjV2/Rw/jE+AKmzBn0CjtDoCZDcSzOzmBa8AL21MKqwjDhDHlfs8uJ214LWSR6PY9cXghpyXt+YiyIUjtRV18SfT/6PjAmcMMcjlO2Ijv06GU/wTpx1aEJuDMfgFeL4ME8yNL2UT7L2v0MjTrEZxUwML2xm7ElLkxNnWzA4oQWGfD82ddoaGc09/J9FcwKlMg68aJbxm5XA8nLPVY45MBLxk4ipyaeWidMDoAIx3XvB4+Fakr893LTCqdr/EQQ6Vd/2BQCvo1IqRZjfLoVSp0xIsFefBwSH4fCgxy8dxAPJv9E+2Zv4DDdUVxzddhAlpTCvDITj+44jkPrxTwNbM33MfNZXQH+lwxMaQAoaGgzELJHfYD+kqRwXvup1fSVJeaOhxntdyDLcCpnJJuvz6TcHOGuC1VvIyP78VDnSTzlDx/EXHUzMeqtWHeKSt0d/jJBKYkCqRzQBnwSwJTLPaoPWBcJplY5hFp3Ki80eTJvFiQQqnLSJPZ7jDWH4cyLUHhYMGLr2TKAWq2B+u9fZSC9qicZbnCmUhK1ft5pk/kpva4fI0v9sqh5cXkB5O+jRt2CsvxHkW6uYvO1V3AavcA0KgPzyjvyZYVgxlQ5GtTzKL/EQ34OzE7Yl9uXQrMS0ALwXMgcXg/GrSRQv85TqWRahpzi6QAwSvBD7QPC4VYZFNddb5rqs3D7B7BVg81EglqwYYINBQRrvYBcTifNdRIP3BIBykf8YKRlrqib6LIAfQB+5lbsTZ3Hu8cOgSHq38pJ/9PE8TCVFuMgP7zLnNbf0NnEEt4/5pEyci+/DJEQNRSTM6LhLpCyTAQoVDoSr13CWDiJP2vgjDlZefwL85jR9WHO1jkJuAXZQU962iquwNGHSNSLBN5ftydRHfKwYvfEgi3MCtAQWj6E8upmgh1y7zlo+RqFljaMvNGOTdVe13XrG9gYiKb0L1pqnYSrYFujPGZHKoFL7YP6MuR8MT1TD7Po0O8UJ34k5CEvvg73Xebroo8JrRhMSOVAxqmGiVpqv8cIWf3yy0xL7MawoGya3IFfLaMUx85v8SbJdwQDZGcMBONJHOKwgqWMWElIvNZegEnWf1ZL4wmTiYyUtYxruYQVrb9DAib5AwUicC9Ldk40uc3KMPF7kuqyYv9L+kkMywWDJOY4jcMLgFZ1i0mWt3jVOYgkDYwJuOEJYgFIMjZZR64dPrvTnzvlSllwWZJJtN/D0/5wI8lEmOai164Sa26NZa2rrnDDNUMr1UbG+4LVKWSkSmNfE4CEe8+CXzJOdS0p8S8DroCPSifYxV4WJMPca+MoMY5X/F4VMJqvK6CTHl4NtotEiq1GBLMc1gZzt3TX950X/zMvFkpcs8DRwjGCqVG/tQRGs5hPoqUIIdnmkn+VsLF6RGOWRNcHFr3AMA47RrkOvQSTNYnkNy9GVbhTAFKOPQLGBA7kPMedsM85EfUaGyp7i6RXk2cU11bnhD96zyW4/AfF79KVd8lrJBjRMg5kb1CK0wl2C2ocHDbDp6kvgq/nm5UkuF6ZxGGzkDssihjJyt+KSIiN5eiSxSyrSuPzMni9uAznocUkPFfHyiOCFSfL0Dl6Gy92m4LDqcLuDday1dBS/QWtww4zuvlylvRu7ZGhc9k++7vsaR/Ojeh3PM9r3xA4Mx2GnWd2RTIbq4XffNs+Xqyxmj4H/k1QSSqesaQxveBLPh0ygCDTLlGvbUQqFeoubB/0MN9GCMnVVjUrFaUP6hmLADdMfiKZd+192DsAUt7n4ZZvMux8FfefcdLROFKs3+45KJJFsgaLV504tRowNhJMuYqrfNnuaUYYBajgr6tLKPR7WtQ4ih4Ot75moCR86VIHPHT4c2y+rYTUur0Wwvty3PIxeadPYqgYTJGtAxhcY7HKyCfnFrO2CnBKxGonwwQTtH8PfAUILtnP8/0G1V2FS29B1S00mRv4LkIk1yONt+kTuQaq74gNIwdhafsJhbXxtMhaTvPspfhogwSgVOMHPjHu2nTefcbbrhVdI9N/Pc0y4KmrY5XtyZOYVNmCOzbQ2IMFcLXnz6L2ncNG95hf8E2ay53wTwC4pU0UjJkTkwBwSJ4A8MVKL0aO6zr01iiMdY3R2APEex2wQ7wrh42h/ukMtwzl9dKPmBTkmncrrgoGVNqn7IoWz2Ss4yf+crQWgezNMXDjExiVwcW6BfjUJVHtBJU2TkjktVsi/LZGj/JtQQfxCHVVWMIfgPtTIKjt3UFoWRZy0PEPgl9Tzllk/lKdp2XLGcQGvSU2ir4fmr+CylJADx+xlnz7wlzu+L8LPrFCzj9qCCd8V4u0qOSkwhIh4i5VqWAWcaEHtYd4wg/O2ssE4MrbwnqwsjoCsxOeD4BYbaqiOch0m8auPFlVa/i2oa7ev7AfDAZadnmEaZ2ms3lkSwanHqAk4TkPe7SuhI/jbvOgX/17a3BdkoOigJ3kGXfTTFrF5Pavcsf5MLRfhq7mLIt6TnUzymocOmwq15ics5X2hZswSOCUrDjdzD7P2Fuf9K5nCUlO2X1+UbvTs7beGAUBf7rWnXWFhJavRps0hxJ/0e/uOFsLGUUAh43eIfcRVzSF5tlL6Jdymbj818S6wb+Z8AWddiL8n+LNsEk8VWjDInnmkgz7aLrc9ueoGTKrkihUDRU1YF3maDmHdwpd6F3vZAuArMaKmnK/I1TpUyg3R2ALHwb6MHQnRjCi6SduZl947QAifD3rufrjOHEypxg+rwjwHNdayXDdJDrd3Oh6VipkexWcFjFLc9xDyBW93ZtrnEbos1GU9mnxKrK1lNDgI+gkmFIA54xetY3rSkmqOE5LLfjVtqXXteOueoDNRcmZ6GHoHR4/xr+2PU+8/AMZOXn0nf8G80Ja8IivgVfD4XBfO5vft/Nch7/c2wcZ8nk8tJCWrke4JXULRPSHFq/iVPvRPPQYfePXM3/fXtINXmo6ZRf5Ln4Nj/qJJMXKYn+csQ942vcPZWNToU5xzAzbqvoJNQkQEuH3HGSD4Rti1PBcAET5KMkY9P2NkHSIVMPDoXkCvNtrg2CwB3di/tkl/F4Dx82wMeN5hdT1lD11TLojGHvBMrS7/hwUn4SuX1LQ/qSCvanVvydifp0+gm5fIUlgVudzwV7E2J17KAx6yXNN2gAWlXXnpyoYYYRlUflKNmOHpeT3yKXcruZcHCxo+4mnzSvOuzMG6uylfwuSXwksAQ4BO1Gqk+wyW9xS9SvefYFEXR0/VoKkVvp1vlaxTllzYQ3fnf/urkTqBdMfirp0DZN9/1vNncP4f6Rr+t9k33/tf4Q5HA4yMzNxWKqh/DLUZqOW6vAMqw6klPcE2sVcCAV/EawvB8CuquaKJV4k6loJ6QFLYD/Sq6PdH4AkK0ckf0c+SRpIcETwtX9/F0qrBCpTwVGHyuUEfVYBffZugvB+iv2/DfyTnyNhtBGWjMlg7rvz6atSMUSvlFEcqJGEjOLCt7xkFMW1OCWbsphnQAuO1CylNPsF7nV0YqCvq+Bg19VC+qjzKrZcW05ATUd8ZdBLtULaoPkrYIhCVX2LNRGiztthM8h6JXslofwIrwXB/BIYcuFBZLVXQk/jjyVkKOlWmY/uhaOOf5+YeBk4YjajPXWMD6qv82W5gzcLzSTsT2V4v+dYufAV8vPz+fmnBcz74hsSnjEw/O0Mln+7i/x8T4onnwtkh3wv3lNDpDO467ocKexMhX6gsrHqFlfjoZFG1PmyapRymRq7iUgVzIqHKSMiuV/7z2rbRUaEMTjyN/r5wEgjnGi5AbJ/d28rOUwEO8Tz7RpylZi67yCgFbRdAEFtqdM0psQB43xhVzRIldcV59pim8iqCthvMbHXpJTmw5RPQM0OusRsZU7v8TTyacCoAAqdQh5yr/ost22j3atJSYLEwIu82uMR2oTvJ9axEW58Rn5+PgsXLGDR3BZMW/4jvefWkpq6mMJCZaqtPhHTu3o/T+Z1ULTlBgxhfy1EquuY3XMCIdVeSaLyy1Tfnklt7Hu8WAS9Lj7uZnbWW5YuiUK7QKxPTRTItlGPPMKP+n/IeNNAaOyD4lZ39YCTQoqsNOgpFhzaQo0lEACbHCKYAUEdoeNyDtR86rk/yfV/aqO4vuAOtL3Zmi8qBNp9VdiXinNaNCGkW+G6BZ4ugGo/r3CwMY4p2gd4y5WQW1QK1+XBiv1vG/vwTCEkBDlY+rDjH0tkRERG0/9CEbFFU5kTBC8G/6nYdqD0E2+5Pu+BBgi1XPM0hvVkc1UPShxQ6XOOqaePQf9t7maV3UwnPxtVAeVU2wMplAYpjt29cAN/eH1GCrm5oPbM1Azlh0rx58v2YUwKDFHs3zTgL14M0NI0by4VplgkL7lDWXayrs2vvBMsgmTPByICAYCP6RSlSWLxbnEC3rVSnA7m657lU9eafEc0GNOXi0DWnbVQk8E1837Sot8lPWIF1c5C5cPVhTCqw8c0lgNY0Odh4n02KpqrSs7hpzrOpmp4PTceh+xxRnWmK7yrb0vXWiFLLHsvq2QtFY4m+BSIvvhOkS/lA4tEvZnANjDqDrJXrYbWvmvFXNJZLOTrIh9iZUE4R8ywsRqqNF7JFiCseDlz23zMwEt3GJjTIEEU2Ar/UWlsye/k+uFuRNkaczRfVcCLaifr/vj1H8vYlZQWM3/3ed65NJ2mp8ZxzWeOZ8OAlgzJbsdHljQ+L0hg9pkPxbdkTKx/KO57DpHBV64W6OyeP0PCQ3B1EQAjkn7mvXarkKuULJHa6PuZVyJAKVfiXlQ63Co9Vfpoyh1gtFn/ESt4BmIuvmI08ppfMk/6a5gZBpX33E9GTi4zHmsPvsnIMuxOn8LPlxYQ638Nlb1UcZxY3RWa6mz0NcCE4BwlmCFnK882fYg4tWC7nmO8QHyuV8OdH8GvMT/YVvN2CWyq8MGmVl55gD6AOOtgehlgRZ8n8Sv5SQS/9vZFLxXxWbvVPBsgagr51Rx1177DEMGqq7v4xvU9LorMg43BbidSn76Msa3nUeJ7mAqfc6Sq5ooA1+U3Yd8gwh370KlqSCvtyrcXluP0a+G5KFnNPksCp+pEsfnXC5SJaFq/SfAdfzZWw6CaQ5yoSrrr2SdpL9NCC0/4O2hvfxGHzUpmZibgYM2ltwm8JWqTmZ1GUbPDNf9LErzWazyv9xGs4eqkhQKg4dsICg9yJ8nJq+aneConkCduve05ocNK2JH2vNT8e3zqGrGxGvbq3vTIaN/4jEaX4kkIuOw+D/wzOWnvxHE/lPVyhuj19FWpeOaBONIf90runH/NXQvEoQ7E1jBhChDSHYI64CdnuX9y9/lh56HbV+CwEaUzE4TO1a781p1l52geegCbbKHSgWLMJbAt9P6VzTUF7G0bS0rsq3c58Mb+f/DhXxdolLqLWXtOiEB76WmwVQtn3iuRIcsIBYOkyTiCexJ9049FZUJ6eVdVO6+LctK8ag73xm+lMGA7fzlvUqZrLOocXV0E11cSJQ+ne9puetz4izeazBcBLlOeSOoaE9hTvIzjVaHctEK1M1BxzZIhkjs2OGgSQa4q2Uv2Pm8XbAwmwk/UeM6s+c+k6c11FTQJPkOf0Ks4Ad+bQPdvAVCpJLSSE4tT/P5DrbK2cX0C5R4fuJEIPiVezBhDNBmhKzhW3pguOngz8qgCiIPDSpKjGB8JProzkNRy5XpXyt3Bd6HNaVPyDNbaxgqmtyTBh7eGs64K9sfAE4ZfFft2kt/jlSCh/BEgg7F0K/T8SQSsTz3LZ70iaKQWgfjZIUWQv1ewizssxxrzKN227eDlk+9zOO1VanSdFccuj3iRNwuC+a4SJu7fLQJybd6AUemg8aVx2X4m1s7keEhj/uz0+V2B+Vrffpyoc7KjFr6++Rn4edQEBiauYUeXz2mnhTM1jWBsITQS0vaSrGJjyiy2V2lZEwHd1B5ZRKpvsSp0Kq8Hw21TEDtvPo1kTBAPqvUbEHkPZlURV+Knkxn+JYVO1xrKvxkMOY5zVBaRpx7go3J4r1hHkaap8qJ1oaRZYVoAlLa/gH/dIU+b3QwbdHyUIAL7dXUWnF4qBpIEoZVirVjqgKWHmrDkg1//tvbsXzYzh+wOliz6gpUrViDLcCJ7NIsPb8Js9UXlqPSc11pFN/l5ujX6goKg37Cqi5RZojMvsbvRe0SpYHnT3TR2fimSzdZysSbe1Z3rUadY7FrSyTKCEXbuFahORyo5Rf/gX8izavnmyiws2iThO9fmIstwpzqebKuoY34gbJen9IHDxqCwN+kZJ3yG7yu10HezkDovPAAX59Av7nv3ZT6sDhSsKlMe7O4J/s0YkSrWXz31MDRkupB27P0LhHTD7vSssbakzqAw8Dno8L6oeZq3g46qQmSHjuCqvgTUdkBlda0Po4ZC/51cc7yARVWKQ12MTi7zADtVOv68NYWDJkByKlUGEh7itn0C2RWt3D+F2q6LGoM1GahKz/CkP/hI0CzkFI83nSIYIgDBHTmib88l362EVg2gcf5sfGWLYMNayqAyjXsixVjazNaE1uEHkO1e79lhp+r4FMaFZFHpgGq7/q4xPcIQQ1tnDGt6zqRv7Boxt95cDbk7mNV9AkNbvIdNXYnkVOOn84NmL0BdMb4p0xTlCtQNYieSBBZVGUead+do854oAEYafxal21h9ZAcj4rcyKGSOYPc2mggDdiHVZNBV7wLQymZs1ImYiylXADSM8VgJBBw0UkMEhQIc12oOqA1Y7VZq1SKB2P/ky5jafAmHRsPhBwlVX2V96y2Md2GF1SpZJDV6/wIxw1GhQgZeD4aRER+Jjfr/CR2X45M2lx1JmTTNeYc6TR5VtlLotwWCOsCJJ6lVN3MLNgf5XRSAg8C2bin34ZrjjPeDh/Lh40ovBqbL6usnrgqHJj7Ksh3dr8/mQ5dfk9gdDtmt/xgU2qv3FAprErlZ2hn/2s4EdPwUGrnmI30ED6W34usKwZh+LFi5zo66/QIfuvJgwdJqBid/TZ50H7SchcacRufww3QofJf7dAu5t+Q1AiQXsy93By0KNuErg0OyYjQ3JdDUnGjHLSgTgJt6SU8nNgJqOhHmJa8sSaCzembj/bVgq09QhnTl5/CtLLZ5ntEpRop6zXVFsF7DIPvntMtYTeN84ZP4l/4E2Vvc6g2Z595iZKBLJaFBuRqF7KTkvPub8Y2gc+EqADoYqomy/+6Rj2z/Hu/YXuZg04EcbN2KSwlTxfFUPti6fMeOm9PcoND6BCgAV98jquoT7HItZb7H+aEKdtR6AdrPvsQLMZHo1a54H7Kivqu+zzrapu0hrngSEWWj8JGDFNfse7w7R9vu4dVAIe9dofXqf8Ed+a3tWj4oA409AK09RDn3/tmU1tdEzNSnrhEGq1AdiYyMZP7rr/PYtBROrDCy9z2gH9iNIHsBcVJLe5F4oaerrjK0sucJBY7A9jjVAQxq9D0vdHua9LIOmNRe86cxnq9L+nHMJKrZv1kQjJT0hKc9bhzbysS4uq0G1lQ86En2aYMgpDvdNVk87Q+fhkMj3yPKF6nSY3aKeSj5zH3clJ4TSlA9fwRDpCKmITco+CfqK4oxTy+BsTZN+HqNp2IL6Ira4UvbO0L5YVNNhqhVHz8ejAnIMhQGbuVIs3s4EvIRTrW/4pp21TXjpBn2X5/D0rMHhEqBl5lsNVjUxZQ7wGT3YtapdPCIk111TThQC1r93QzgfGAO8Cf/WsGi3seft2IrU68mMLsYUCvBBfFVDzK9y3QA0svS7zrGl0WPcSL7BFV1Ve7n1fD5/W80dw7D4fj3G/9/YP9LH/t/7X+aOZ1OSktLcdYjHTT+vBSp5ysXOCZYbYOLc0Uw4PLb0HIeZwo9SN8z1h9Foq+dCCTa/Vrz0p85bC4No68BfB3KobF/0Vy2RUOMrGF47BIoOgKbQmFrcyg6Sow5lRiXH+1A4w5C1ZukakKRDXoZoF/E18x4fiIZ2dn0nD2f+YGxPBcEr4XDmT7hQkZxlphQJQlaBlxjuBFejsykpnS94riXbJu4HjOPaedm8toxIdlJ4ykieZEwgXMFI3FKDp4uhFV2l+Rbx+XQ+g1Uu7sy0V/ICUqIGh7edj7+Wbq44kmSU77L4a6vXTYgHF4cFfN/ZCu+DKwBjgK765TO6X6bYIm8uWA18TEx5KxfzMLyHL6tMrGs6ibnPl5MQmyskIwzFfBMxDAmx5xllBGeDk4R8nf1dvo5jjcSDtLzeX6kBsz+2+s5ZxbBJqdGmeDsXLSRvCSYQAc+G5/HtFcX01elYmiDpGx/NfSVZY9EgdrIU6eO83wh3LbCL6VNBcLVZaFnu/F7hx/xMScxpjSXL/W7wb+pCFLt6kaQ/ThRskSUCrrpAbtSAmhcxFyS82a7nnuDF1F0hKa595HeaCoT8iClrjEN7YewdkwPgITC5xTvUZYhSJ9Pv8R1RPneItn5OSsXvkJCbCxHFi/mrbzr/FKXywfV14m6uZg2LWNY+VJ34dyWXeCbyBp6u3JvDevTXU94kxXl4K+20ifhFwzWq57GggN0KfuIxhqocECx1fcuKKpKlqlywMJSOF4iUMpTnn6a/dZ/5tzst0nEx092HdbhrsVg8WnLqZyR1NmF42XRJIjkePRQ9zOpN0lCLN5LTrtRtvUJ5p01cN6ifNaGTl/w+r69FNUFsim3J04/Zd0Wb8T6ojI4Xa/f5rIifRu+rIDdpJIW9haz3przt/2vrxr6qiTmvv4cM2bORHaaaBZynGhdBYN9oLfPVcVxW3KYsS5/YV0ktCv/UNEeLIukUanfEfYEPahok6pvciIOdDO/ZX+bydxWK5HspfGPsaqiLQMNsDUaYnVezqnahxUP7iS08FUAXmp0gFbSYsX+7YK/YUabFeSWdeO1nVexRk9wt8myRLJPMbFqOGyC1QkLRc0WwKny5axZ4rIF2mbCIac341WixhlIuQPUiHFXVXMdDtwHxx6FExNpphFUWmfgPgb49RLsk3pz2JDtVewN/Iwn8+FynTIxcbbpOzTNcLjP5d1nDKbTTO4wi8b6SqqS4Sn9t55GfShbao7w542XSM6bTWLhi0iyBDdWwZUFcGEO8Toxp+WGrKVfyHRIec+9uzX2cd7J8zCiGjoet0Jf5OlbnQluPo3xiT97kjz116Yx0PvaKd7K3cJgWSlBC6KOlRPot7Y1fVTafyxj9/tm4cy0yvqEZrd+JdLQQrFdvcRxcv5rNKpwoW5H3RZo08Nj2NxJBNlPxsFzvm8IJpp/czgmGEXTL8xnSA6MzQObf8cGxxbfVHc9TAjNRIVXEsZhw89RTaAMMb62f8wKtgKNYuPo0O8m7fvW0bZ3Ns88sFmwnvv8Bv22ut95ctA5Ph/eEv+SHxXHmJ75BWPyYGohtLg6xM1CA6D6Nn38MwhxDQeyJAkHDeD6h6A2sqp4Kz9UwZMZidRo2ysv8PpKPo7fzVULvJM6kprYFwTzQdahkm10DUqjqQbG+EKT9MFQ7mH0ePfVLItGBKhkcW5t3loGJq3hePO+XEqYJrYNagftFkP0cNrVPsbno2I5kzyGrJDv7kbHeyXcw6TmykZZLZK49ds26LsAXTMH8mg+DNcbaeL4BKfTQWlpKeDEYgugwiFChWGqGlHbsL6WpQQa2YxWbXKdSiOC0ZIMvo3YlfEK18wGssK+JkP2Yh7IauqazOdk9iicOLlkgZva+0QNOAC1L3XqUE4kj8bcV6JH+x5gKf/HctL1ieOTWpgWCo/5RvNRixb0mz+fjOxsunZ5nvWlAoWrcuhFsLrqJgAGSwqJgX/z3iQJrrxLI83vd5/Qv6lgGNTc4YuhMUyLPc2ZOHg3RAlWuN3+M0Kz8jFK0EEHOrsXYlgfzuzUU3xlfguzNod17X4k7LZSepLAVmRVeiXLyi8J+fiC/fipc3klOpWe9ZgccxEcHg0aP5x+zUkoFLJh75fB75XKESamaiVdordyKWEKl5KeoNRUCPddhrA+kLKEYM1N97Y7st4RjJn2yyB5CvhE81p5Crt1fxGjBl8vSTAAjbWCBDWkWuGbSrCqvMY/YwKHNc0542K52XX/mTS9XhfAkiObGPWb2L9DXG93YFdWyXRKbcbMYqhxgl1SqmAk2fbxXohYN36e0Qu8a73pQykMmsGFyhgOmOD57GEQ5iXJailnbu3vzA6C3OC11MoNrrr4KP1DV3ElfQav7kilzOFJNIj+JDpVkAoMkpJ5fJRfmepSmxjjC7raq3D0YThwLxT8xa2qntQ5RV2zp8OPQf4esXGLV7B0/onwynspCNzCmeTRXKtUrtgE+12s7wrropVAr7oSDOdeYpyviqLie8koV4LIAGpsnkRGw1qWSBJWF9OoSN1A1lSW+ezaM/xgLmGiPzRWeZUu0ARw1DyQs2Y4XWvk09Nf4QxsK2SMCw/BjVU83XoCSA4S1bAwfKmQwDs4Cg6PRbo4B71NJJTnltVRqGsw/jV9nr7Zogbrr2XB2NReDHRZTXH4UI5arDzlJ/FRp4+QLR4AkiTB0p5PU5UQwKAjWXyy6MN/DMQpK8snr7oxGRWt+XR4a6LKvGTbtMGstXzNHOlX5lmP0+VWPBg8UpEEtuKUqQkOYGx4Knr7H2B2+Zl+jaFEvNcoNQwwgEEqgoxfRO27ktOQ/TsTouZzMGESC8tNyFhhb184cC96Cpl+YgVvlkK1E6q0rTylD5wOBoQtoFfMDgDKfVwJ7lZinUCXL5h76Dz5gb+TE7SefGcPkfy5OF8oAnjJnbXWQkffT8HkYo1F9OOVa2v4s8YjS6noQmOKsDQ9wH3nzNyT/gdjEjdj9+8g1iot58Cdn2ireo/y7iGYe4QxMy4Y0r91H0fl8CGm5FFUdiNX6r6F3B0iQWqIopFqA691mkuH9LV0S91DXehEGH4VQrriaL8U400BZvnZ5wdmp70IoR5pv3VXf+Ry4lQKAgQQL9y2E7a3gcyNsLUp21p15f4zTmaZp7Fk0ACoUJbl6Fp9jPuM0MvShT6VDynvufAI65I70LauK/76HDTaApFIPPcyZG9m5em1fFYuNn28II93I/yEEoq9Fn32V+D0CoZzd7IvN3g95b4nKfM9Ltas2X9A1maQJCRZRZ26mE9SnuGL3F9EIvnwaLEe7/gBoelaahywKQpambcKcDdA1FDO3viVD8viyAveyOoImO5cqjh32fGprGkqEvoBtR1Q11wVINyqG6hlM018Sgn1XoN52Y8tvuRWIkwtgHfyXKDM3F1w/UOssU/wzbmVZIf8yONdH8VY+b5oD+8Lrd/CrvawtOb16gIHR7p8SBEMf8c6g6cL/v685Gznh+A76CTolw1XTEqZz/TYiaxxDX+fJsGih6R/rxTlAoUGBUWy+fqrvLF/L3V2o3L9pjZwsDqMG1bBQB8RdEFxHEPVSbq6huleWdd5cvceDHIR/BaBUxfGmF/qiM57g0Ha+czv8RS9bC5VpVbzOdT6S0rtIqnV48Y+xtzZQd+88bBDJKi1rnWhxh5En2tneNykZPhq7J6E1YpysHVxgWxlDQ51oAvsKczNGFT5QPwEarRKf+FO2wwBbtvRAS6/Q7wlg0fy4Ul/+KTxEQwqj88UpLrOC0FmkjUwuV9Xxlg0otxHvV16m3cSDzDoYjavGePpVjdaUbPP299/IGY/Qbt9oGA/UtKTpJb0oP3tH0nKn0m5/gK5VS5GddrnhNf8SK02g2txr979QiMGcb7meQIr7qF3yhnaZnyJUxsqGMQPiaShSXebSsMV4ounCDnurN9hZxcoOIAtWijYXHDhExTJeUmmXXRXxhg+JangFfGTBFTdEqU9AlphjRpGfOmTRJUKtr7OdBlSP4HabCQJ3r0xiukuYbOReSB19/jVTtSoqnpRn9o8nr5DfO+3ViPLsPXGdN4+sA2rXY+EV1zPJ5a1FX045VqiSA3Z/i1n8VXhAAACarrQTjfa0+Z0wpnpdKqeQ1sdDMzUcL1qpPKZVqYx0/85HIDFGi5AUXfWQuERqM1leouPGOoDX4TD/JYzFLs2CzpIB73oE7l2uNDtCLRfDKY8tKYUJBzueSbb5i9q1ctaqCt1qwBpJfAP28qVoj8Ux64HgmXX+ZNW0Q/UXj5k6VlW72qJ3ec6A3NgeapXXMalFLCypjeLy6BJJ/ixAQH4a2AA/0yqvp9Gy08HneTbXSx5L5MkT7Le6rAq5hUnTszOKnp+25MHNjwA/FfGs97cOQyn899v/P+B/TfZ91/7n2WGaKHRHzmI6+ZHOO7y76PUXhOF044oFO4Z7Xror8OZFwTrDwi6MIRVw1rTv+JpDsZCM5OSHXUnYDSflcM1i8zSK1eEbny9mYsYnr2U0UZRe6Bd4HXlggA453+Al4tFwfn5l26DIYbIyEjmzH2djn3TqJoJZc9D6IAmd8kovtHpTbZGw+JQCLN4BdLLLvBc8GzaJayhzHjSM2hWprmlWGp0aVT6XBDHQha/n5jk1mk/YgKHExxNQF2ilN+w6KO4YYXhRngi8orSOahMJeyAmvkRJbTNhMs+e5i7+O8TYz1UsArYxb+eWI4BeoeDww4HeyzKZOBus9lNJ1/58ecEqkz4yjDBD14KvYQSKRgA9nZobEHkB2+ixJKnPJFfMq0yYVWF0G9vGCdQR97D3szpLN1/kFHr7Ux5eiYZ2dn0nT+ft6NaMM4ngGdCJToO15G2urFbixxJwmwPw+QUaKkFeX2ERITL6sIe4ET2aJySgzon2GWXVoZfE4gYSJfKQewObsonFZBwpYNiX4CO1qm8mHCY0iR4L2yzoo3gTrxcGsiWalGjqcJxtzzaOL9MJkod+S6mlKeCPEEGSYIL+YN5YL2FPemTeXNHJ5ZssP8tYni/w4UY/uwkK1d+CLU5PBpgIVkD432hv+8NxTnVrln9PHkEXGhPiu90T2PcaE7EruaSBaJUkKSrcAdu621e4fv8Ee7LGyVwuEQEwCIjI3lv6dJ/69wMk3U0ab4clTZRfBdDT4Jr0fl3SWu3pX1OX9+XlRvYqmBXV7i6GCzljPEvo7UW3sxszeG6dxSbhhlDCa0axNALZfS6fpRIo1fgxG6mmzOH9l6xPl0DSYT6wFWZA46XJTPjtQXu/jc3qDkPGwJ5IQxK+2vJ+NjJjEfaA6C2ZLN8SE8ejbjG0ByYmDtPcdyNum8ZdqM1AC8Xw82gyZ7G2z/zSeQnPJonGI33+Fa5EZcAGGKYVaQif3sfSp1PcKhcmdRo1XE+obqjRLoCPka1l668wwrmQgwqkbx+pNDEEZQO+abSYfTNgnPRQlbSezEoSTDwwCc8VSgSMMnRPdxBVKtPS4Zm69hUDTpLNH7qEMWOnztWML8EbID/LbB1/h6SJgkJycJDTPEVgSQfGboa0qHCi+2Y/Tuxx/3p6FfI91VQ6lAm++qduBcCYH2jNFR4xvvaoPuZueskFTdWcamoJyqtkgkJoLOF0zxnCfebh6Et2AijMkWQPmUpMTrP83u9oL+QTjvveja2UnLanONEHBQlQdPa3xXHLdY152fVQXpE/sVDzV4TjlS9mYvh2gqah5xkZvdJPNb2rbuu63HnH3wZDqUp2Uyy1t3V/nf2hMnEge1rGd9qIf0TfyI56Cx6p5dMbF0pj/hn0lUHQyJO8GirOR70K0BYb66VDCW4qjc/V6q5ZHUFs++sBWslxI1jk+5nMm3wWzU49FGK8ycX/sHZ6GY8HwAfJp9GdnrNvVU3mHj7NV4OhKwW8M0/lJv+wWDggUeFUywhuRGsFByE9O/BUoIkwejmy5nWeTobrsynzrer4hj1Qa86J9Q6tMqBptkL9Eh5hQt1sDwUmkoHIaQbNJ4mAopOJxZ7KSqgae6bd6MfS8/T1z+LTBusz+2J07819PsDHjJTJ0XR5cByZhSLwHJezAq3hBjWakYlzOEBFxH187JAGPSXWwq7ttsuph77AIAK4xla1vaAax9Aq7nQfysZ+pf4pQpax2xmYa/JqCq81iMOK3+G/smnYfCO/UGe928wD9Vmc2zAWrpqNXwaBi0Cz9/13OuBFIuuP8YOQ674VnExW3RlDKjrjVaCJ4y/izqWXlJPCw//wZv7RcJBZb4DxafE+OPfjI3py7lYFUd/AwzxzfQ6oUxRo1f5znoTk+6O2Ffl9bCTnuBYwo+clW8x1AdeiD0B1or/SE7aCqj9wfKciskvZHM6JYX5rwu57MOFM9hy8BIDL91mQlYB9PwBHsgAoHH+eGb3fMjr2bj+UXwSooaRYnve3ebuH6Y8ITuoCeTn3B7sUJ0l0wYldqVOaH0Qob0OzsVDRJUyyFBfY2eAAbr6F4raZd5mqyUv5i1uNxvOvR2eEhKMPvEQ2JYAbS6zYm7Q2xWXkNQ6wVo2NkJ21PC03JF7q0WgSAEQkiS2hR1iep4PdlUtlsbQOvt5EeDouxnUvowIGkOZ8TgZoavZVNoZEiZAy1luZlcRKfjJkN0IxumUkpLh1xdxpxHUxyoUQInANmwwDnLXeHW2hW80/8wd/sFgoFFrMVbobBE8dLOCgyM+cIODZBl6XztFROVAOusgRFYygOOth3ktGDJqw9h+7AjqYCUL7nTJX9wJX0VNdUs0pk89koEAaiOb9T24bIHfG1+nXdRHyotrMYv5Nw5w02bGIVnvWvt83P5D3g2G9pnwjflJRVuFuhuXzGr8ZLhW2gFz6EjosxmSJsPgo3x+bRe5Nok8Gyy+cQCaz4R1KtgUivbGQj4e1paxYTdIbZpHS9MPimOHFixmTVwxATIQ+oOyXrM+Egbs4kD2FL46+wkfHP/5rmfeOK0j5+OFvNWTyTMUsruHsycy8vTLXLRAH7m1SIa4kj8ChS/eq1+qgQsqr7W0IYLD1tVcvjmXav0NrsW8Jr6r5jOFnFfVDeIMIpmrkyBEXSbkR0tOCabRnZ+50/MLpviJHnZXfUVgrPprdtfC9Js9sei9WdEabrdYwJcV0FrtS/e4fahkm1thY82qFjzwzlG6zXaScvEh+quc/xiIs/VPVz1JSyB/pL5IrY8Xk0Ol5ao1mRt2MR83BOzReCoLih+lwA4+N2Gd9SlRF+/BCpdUYiDH6gLYWwv7YiFCdVQk4Hv+LNgLyU8xKBtybWBTVVGnbSLYCeWXMZJBfchJK4HOWSnGa9fz+CwjhVmuvOKT/giZ6rjR8GAVJD2JyRHCmcajOZ/8MG9XjhJgl/TvoN82CO7MpPAbSMAPVfBNaZ4ADNTftivAXWNIZfrgJiSUzIC1kvhfxRWRKAAC9QU823k6umKXUsb5WXBqKp01c+lrEDU2z9c8IoBJgOy08OW9XXg/RMKuquGyeZ4AmGWsB1nNNccL7L/zODGlDxNWdQ9OTRAEtASNL7Lah1qXC3TF5yA7TFYh2QxwZz3vmH6liw7KfI9R7nOaKlVLaPMOBHeGVvNwRIjkyqWCgXx+ZhWSb6LnPUoyX8TO57lC+LPpRWZ2fkW5pig6RHj+26T4bicuq5rteQ+JZB5A0+mczHtY1LbDVSKlNgvKzoN/C0xNFyr6jXyXfD4UBmxX/M2lN+DCa+B0oNN9RlG7DvyYOJI15s2gCRRgZR/BYpFRoZfgASOE2m9B1BDovxOMCXQ6M54XXNKfayrhus9zinNrrKUkasDHGsrDgZXob7vKJnT8gGJHJzqdmsRnFfV9QhY+wOnnoOgopyubsbUGTtXBJZOL6X/gXjg3A0fEvWxNewFwMiMIEu3HRfmUylSQZAaUdKKRa6L5psxfxIu6fin8D2BQ3FI0+VOZGwSDDUpAAnXFtNaIQNb1vOE4aadozo6ZwEaXGMCaStD0beqOvQzWKmMvgzR6d920hx95hO3bFnD6QAuO7YnhzIEWvP9MEvkHPCocseZ7AbgvB2ZkKRnoJd0v09M1zFY6oKwunMbyd2AuRHZJTzscYLfDzrRnuK1xSRL6RGPxTcIO7rp6JqufADsEtASn050scEhiHmgIuFU5vJQH8Bpb7XWE2m4T5dXl+ksbhfS3xhd6ryffdxLlPmfZ0cHI4RZdhP+Y+KirVnUgGeporligiw5Gh91Go/LMRWGqc3wQWUtnKYxz9ipuODuD7IkjOvP30CXiD2Snlp2pMzlv+MkDaCvYT1/pL/SuObfAYsAWMgB0Ie77CzC1pzBgJxfCZ5Fa7JJrHXKUm5G/IjnFTR2KhU/DCjw32OgxXi32ZXer1hT57/LUMS+/IoAUG0OobfIYFcbTnG4yQrRdXQilZyB7C5bmbxNz4iG21UBWInQvG+g5tq2WZHshXXTNKfLfRVrUInGt1ekCjJr4MCGdFhNf/hiBNd2oUxeirzoMZ1+EqjQkCQpufERK/hD3Ib2lsvXqGtoaKgh3va8nhriAsIUHUWEio6INKUV92DjeSKuap5Xv3OVPddDBnwn5kLNd0a5yvZdXAzQ8EvCcZ01RdBhuCaLD6yWwv85KjUOp6MWF2bwf/iUPXL3GA9VPoJOKBSh4bx+oSmNs4nq66SFK0hOkLVLs+nzbsbwb41l/uMFHF+YSdqoVWpUJJAEMdo+N29vCgeGuZJ+KZhpIS4RmJq81q7WKdSG/sDxIj+xUu2rcetntn1jqm0mQ63Qq7/VGXQls0LHSfysApe1hn01WxMy2AI+7/p0PLAS6AE1d/12IB/A20Wwi5kYpt2OCaRWoLMfjxOn2F6x25Rqz/nsGuJgvgIv/Tfb9/4f9N9n3X/ufZfowaPMWRN7D7uqf+cqFeEqzumogtXkH7jkMIV2JMHgmzFb6TMGmOHA/OJ04NaEkBF4lVvLl03NLkULHKE6TFfoQn1RAmTaTbWwVmsjjymG80PC3d1jFudIWTAmAzzrPFnr2XuZ0Dfa1Tqi1R7pr6EgSWB169LkvYnLCGJ94uO1xbiUJ1lyfxpP50D4Dcvw9rBeslbTWZRGlgtD4jwkIc6HLtjYVdUwOjOCNHgLtcr8R2soHRfA0fY1YkIzKoE82XLfChiqEzImXDYrpxJO275mhac2bjY4rnQO1L1ekWNKsEFU6Hn9HI2bM9CTG3ghrwSQ/A+9Hwak4GKD71/Vt6unkW/n3dPI58xfySOrTLCiFucUwIXOYOyAIQPslfJdzEnCyLBSa1v5417HC1UnMDIQryZUY7HeUjQnjWXvjE0w2P0BCpfJIFMxblMKzD/3OpHsvYPM7gdxlnWc/p5NW+moGlojkX0Onubbxu3x/8T3AQRMNhDluiIRK+nfQ6AkyfOezK7f/3+4LEOXYQo/ATHbUwnVLg4WKbyN+qQ3gtit+rlbdvf83BddYfnAvBdVNKLZ7kG6CnSljd2owmQv57ucV/Gmx/p/fgw3mzJlHvtSezhef58cqWBACTwQeVWzbLHs5s1yAvErjBaVqoE8sNQF9qHDA8jDY0WatAhEHQMwDnKyoR9977mnGK68wd/FiesmC4fZ38mjJHRaTlDzTfY/eFly6mg3j/GgRKq7XaEuBww8K1GbOdlrrvlA8H1QGaL9UOAe1WXwefZsxvtA8ZzEt9fcojq3K/Y0JrRbQOuwgX97fBEORF6PCWslLlkPM922PBByIgT72dYr9W1X8xrUEiFELaQhZ9vS/MU9cJXT4Csqfhzt9LEQ8el3UpwCc2jDevd2d9c5L2AC7F7ABoEngPXS4chmNNYR1VVDk09/T6LCglauQZStqYH18uoJNhj6Mj8p0xJ/swgM3W9JTvYeGJksy2/O6E3OxG5m2+zwN5Vfgtwhe9Rcd4bgZvi3bq9i32N6Iw2YI9ktlZKs3UFdf9hxXhuiy8bTIWsZo01cMDIpws9UkCfpevch9187xoflV+vkoIWca71WlUxLB3eYvwcM26P49f5qF1HK2De7PmSMYz/VmTGR9rQ936mNPDVhIAZWXGWGEljro71eJSvICl+jC2Ctf4EzEAqaff4EsvReT0V5HJ927SH0ltnVWMazZCoznXJIjtVkga7hS7WEEXDDHgTnPjRw33F6KUeXgYh1cqQOrKvBv73llpY0nr7wsAqf1Zs6H868Q3Xw6b6U+wC9X59PQop35dNCB3V7+H8nYVVdX8GibNxncfiof3tuZ2Fqv+gHmfF4LPstwI/SKOsgDTZcqJS07LOWH1NX0TD3M6T1WSjTfid+vfyAkWDquoLtPHQmu13sXEro6neb6O3xRARNT++BUeUneaIO5GjSIE2YoaA+HHbZ/Xod1yhQifW/zep+R9I7fIMaCk5Phyjtw9iVkGRoFXSDOP4W1V97GGqCUY2rvc4HeelH0va3hbjCDSlJhlOGVIEjgtFjLdP0CEh+CymtciDnM3hh4reXX+NiVQIrqzp+hvSmkAe2yCZVkcclbFSjm6UsWqIh8EXxcc4bdzOCYpQx1JfucDdj86CMot3iS5nFSigh+2mrAXkeG7zxeywsnWgWDDCBbvRhhSJQSi6mqA88k7KeL/yfKY6d/T/NbQxhkkHkuEOKMd0u+9NUX0lUHlTY9FjlKkcwb2ewj9rU+Qqeq+7nlHAatPP23YZcwZrwPu7uJxBdwKnAW12NfY24QvBV5TLFtlaWC1FhxLGcTeLq8qyL5YNAIOPsHZTDk2j2gj/qP5KS/0+rp3BUszexEGy4o2iRJJP59LIno8Fe0FQbP5Pfrryjv8cpC+Ks/5O1QKDG47/+PZDj+OOhDebHYytYaGJMHn1YOVxxbX3GFIT6QYYN5xVBj7OJpLD7J/KKVTAuA2UFQbVdT0uQ7xf41O7uxuce7SCHbeb75t4JF8kAGhPehxNaKflf68m09+erCHJe038uoy0/wWu8JjA5P5acIeC5kl+K4BU4tF4NFYuiXaqg0doHyqyLY1G4xO2u6crRFTy4nTuWC/kPlg664ztbYqzziBx+WQapDyS52RA3j55rGTNaHkR0dTYhTORJ4B6W2jYCjdsc/His6dppCQsBlOkTuxk8tIe/uJuT5EMGN4UnfMzKwlNPxMMKwVXGMS36zSL4DJlwJF+++XJPFuPKneSasEI09gGCpkfIC1D5Ed13JEZMIgoXoypXt2kAWS/2p6dSOPp0fwFfjCVTJMrQKSKepKzbZxNBdsavktKNGhUGCFsHncehiIbSrAE3pw0TizKnGCuRakgUjLKQb1JWgufYGjQIvMyeykAI7OGRlsFZffZwRRlGTakfnZW5pbnFPBqpDerIuYCb7WzejvIGMHUCBpgVHTNBRB73DlAx2SYJmOQsZcOkWj/mGw5EHFf7Y4l6TeCMYanHilJXfcLgmiciysfT0L2Rxu0+Qyi8KdlXKEmjyLDMuiXk41Qov5q8Q0n1ekucAE/3t7I6BsJq/FL9TcIBxgX8QIgsASUOrD2DPLQjgnq+L+HLNOrfCxsLi66yprmJeUSW2oqM8Yfln0k9PmEwc3bOWIcmr+WpEY45mjaPS737FNvWJr156mBRceBdQtT7IanKCA63wt/YOEAmb3r+Q6/8Rp8zwbAGUOduJ5HziI+JF+DVmn0lI1G5u/Sc6RxYMOQoD91KracaTyet42A/GGGFIbkvIcSXVJIlSewvyXMuEe30QMtVlFyB3G9RmE+vvAWeZnZViLXzfJQjvA7e/563Y88iA2QkWVaSHOV+bzX1hm2jt6vdJvtnorbc8N1yThc5yg0L/nVzRnuSV/ZuxhtwD29uLgDmQZvXjxyq4ZoFtVZ+IulOApJIBCYdLvjPPpodu34hagMAZ58cuP9D1bB3VYCoAhxXZlEtnnUh8QkM/0Ik4MhQF7ORIy65UqJtBmzchuAO0W4QzVrBZ0ss6sD3teU+i0PU8TZoQyhzwcU4r9qQ/1aBm3xTSm1/muqtOqSxJ4h22XwYBLV3fuitmIduh88fQZDrow7A0mY+3f6b6GxnPu/7u9LFIfjlsvBL1PK9E5oiEr+QQ/afsokg2ll9llNGJ2Qmqm3DC/xWRbCk9DaY8robez15Xd92W1wNrxErFufJaLyHpDvjIdmb2mIhUVwCD9kNQx7uuS5YlMU6kfQ4V1/gqbyQvuIZL95rTVZdZUuvIC9xMrf4WUelwyDAHUj+G1A/BGE+Frgc2h5hPVhYHiyTf0YeELCqgc+QRrKlhZhD01V1SXkjSE3Qvas8+EyQUPYOxgQyjN1vs12pYXFrjjr3EPzqfV3yb85Aumld9mxM5TigI4HSSEBtL9Y7FfFAt1HqWV1/n6LrbJAxa5lJNyudY57f42tiWXDsU2QIV5/WOKyRrINonE6ekB2MCtpABdIjaRlrkHN43NWd1uZ5MwwtiY1stfo4a/GRfd9KuyhLCrXYFgtUqSe5kwT9J9r0eDOqLc8UfNZkMK3yc6a5LDasYQgv5HGT+4t6+zJrPkZadsatqqdWm42M6Ld5Hh2XQ/CXWBD2EzQkvFEHy8UlY62sNArmOwczec4QTVaF8WA6/skCR7Mvq8iMhmUX81TaO1JIe5Oke9ah6Zf7KSNbh6/rzr8oIzN23QVA7pN3debufSM7Xs77c303qR7TJbEGgRsxpdic4nMrOaqUWq7oUuwtEK1nLYFcXwWp2WClzJZA1tkDX9/YJtJgNzV8WTCy7WOMfLepIhW6A58DVt2FPT1rp15AR/jm5QS6/J6wnDE+BuHEAXIp6mbONx1BluIw59AFRfzOoA7IMaoeRooDdgADDcscT44gPuMzmzl+xoHIBL1dZiA/vCMlPg8rgfud2p5qDdx6mSuMVhaq8wUfR3/GEn6iL18VgFzKt9Xb5Hd6IFuuqFsYiEjTbPO9BFw5xY3mzdgAprmX9XeDJRhPZV/0GbcOP8cnggUQ69wrwGoAxgUcP7WJFGYzPCmLpReV6ZOOtD/jr/LfIdgP+MgSX7IUKEZupTXoDm1PmcsKzADzodwV+i4Sg9hA3RiT7UFFkh8WlUKj1UnWQZDTqpvgXP87MNh/wVX/ZTU4BIGkSs03NKXXAfF0zhgd5ERtcSgkt1EXMDIRbeujZe5kCJF+BkKpfCSQARxAKbJ+7/nvY9ftKhI8v2aqJ1VfSJLiZ4v5PRj/GsmPLALDYLcpkn+xR2bHY7/624b/Jvv9X9t9k33/tf4RJkkRkZKRC0tB7EDGBcMLavCkkrA7cR8dQj/O1vmowRAwQi0anjdpGgjnhlzyPtTm9sPp7BT/wLHQ0QKX2T1FjQRsgFqY+0TibPM8Vs55tNbDqxjTlQhvwrZrLY36ihk604bqb2SBJoFPV0sinkEAZxvmfgZOTFPd0Km0eP1RouGgBp9Yr0RPelwl5b/N1JawMg1ebvCl+77tFFJfV+FNtF4P+3CAYLH0hZJ4ihwgNeW0wjxcv5kqd0I4nsLXimo1nn2V1sylsOrWB+fv2Kwdpnxg+Vg/n5yoYYZRo7kLp1ycmnnk1hXdnjuP7RWCshifrGiDDvew/oZP312rJO3MBEIGqVEvoXbOFLIsg5hR/iK7brzyIpZzJ/pV01oPFoUFqMPPkVOawLjaCnF4S8XFfKu5ZlmHBgME82uZNbpe3x+bXUbHvvA7tmZW8k0EGWBq7zyM3grjErJDvMOky+SgMHjBPE++g6hZYK8gOnM+GjBE01sAw/3KFDA3AE5WT6VqYw6P5sLa6Nw1NJakZboSCRtDSsPOudpMzghprED9ffpeddZ52WQYfTQUtQo9SnPcB/VT/Oilbb/Xv4Ztvv6N7zSriC6YzqQA+LFEGFCNKdzFGHY9Rgr4GMDhyFe0+GuF8/14NX+Z2o+H0I/X4jp+KEtkbA+NilBrrM8bGcvNjieJB8JJ/DI/7KeXRpg0JYeGAQYT5uII8Nz6DPLEAtWsjuV7cE7NNOG4aewFkbYTqW9Dje36syvJcg4RYOLWcDdH3gjGBp7KbsqEKpjX9kXYaJUvNeud7Hmv7JkX+ezDbjDi9k26aAOi5jl1lPVEBLbSI4IeXxfg1wuqQGGmE00MeQ8rwJP5lWaK60Uf4SkIOTApo5pGb0wbyXk0+R83wWM53TDD2VxzX6LhBnH+KG2GmkL5KnsQPZTe5XpmEA3g5t5GQRfMyWVJzsudRBg88TyNZGaR3nnqGB/W96Zl6iN7XT6BXewXQ9GEc07fnd60n6Wl3WhT7y5IIfTXXwtNtF6KpOuNGtB/e0YKLe5Kp2bgK56n95K9t7Q4IaWx5PNP0B4ZEnOeZTjOJcijZKSP1p5ld/Qpdbn/H/OLVyJVeAcWkJzhvb+u5hoZLn5DOzKoIwyDDnURob1AGpeOzvufHCHi2EJofn4ZD5WHOSNix+J6hOGgL1YZryuHJYaGr/i1Guzb/JfNeTB3XiyBD+2UwaD9Wp0eqQ5Zk6Ps73H9d/F2XRaVdxbRCGJADOT7KZHOL7BcpTxIJzK0c8gS4AHyT+DNxHivs5/jQ8C0Hqn1paIulSTycD838/jMZOz/fAJ7et5nHC028VgzX8ZIwMybwctEovquEZysz6J7SQyHnCFCjziIz9BvyAjd7nld4P8F02xLP5oQ8/oqB4iQwVBxUXkDnTxh3eylHzbC3LBZJ5XVsQySmdu+xoxaCAyC5/TP/XvJIqxZ1WCMjMWiq6RC1m0jfdHFd7ZaIDYM7I8uw4vhPjPnFgsOpvstpeSb8YxaGCEb+n40PKxOctdl0983ECSTchuvaBixLTQDbTRE4gfHxe9E1GDvrA8NjfGFH50/xyf9eSIr/NQDJaaNvcBptXY9BIfusDeLdCzdZmCfWJ2+pHxLyPPXHNWcT5+NxGt9zrBPKCb/4wgY9ekc2OCV+qAKfW+CI8JKYktXcCL7JgSPnWHRwN9tLv1beU+RgiuJWsrZSTXg6HCtpIKUDvB92kjdDoLTxVPy4gSRJyLLM16sX8eqqdcRO8+POl6n8uk1LfoSHKS5J0D3md/oliPHSEjFafE8aPyi/zOImK7nPB94phTfy+yvOGX9pIu8Gi39/mh9Huaop1Peh0rMklnxNnFrIq92oCwSV9j+Skz5sMzOkN6yvlKlzKgN3fSM+5NWe47GqKrBLZsjdCdlifCsNmszudM847GZEAPTaQI0Ur2wDaPIcxIjnKrk4bE1y36K/8VnFeQNurGBzlBgndp1ZjyrI65mojRRpIim2w95a2Fbc6K6+XRzUkw3VcNMKDx1/Q6F04ZAM3KyOpbi+u6d97tlRVrHw+JesK/Onsx4aaQsVxw12FhLpink9kg8FEVPh1NOCWR8znIM1njXGxuRdgmlTb04bISobEjCjGM45lCsZv2bT8I9MIynzE8yWcJxe9esov8ITNbsY5Bp6d8nwxKPJ/1gezc8vkvGtFvHugKE4UAt2kyswJsvwbJdnGR+aw2vFcN6q9C3smgjSrdDDr4y3B/VALlGOb1rMaIEXwrOZn6wVcnBe1i22G3cqWhKSDrvKxysv0lJGkkaA/eY03k6Ij3JUH7L/Kx7Kh2c0zemhU76LofZWXIkJJChjIc/sOoI5xkvNpGA/o+JfI1JjRQWc0z0vam0N3OOuK7OzuDFTC6BbFlw3KMGTBc3/xO8WnDALOdeGQMOimiJyjduYFHGDUS2UayyAk0EzeKEIlpTBnAtFbmYVQHzAFe5v8inxKi2Xy8dBr/Xu+rCyDK2Cz9FcC531VvzwyMJSV0JfzWMMjd1KYw2MDzAh1eUL+WFNAMhaZKdnPSfXr6F0YZD4OLRfxpx9x3mjWKKrDrTe0rgA+X/xUNAfLKh4j++CQ9DbbiuaE1MX8mwAmLXZZBct5uNFb/ytwkYg/1k9yZqaCspMkVwv7kGt1V/px1kredfQl6/DhVLKsug7AhBab9l/MC1oO2Eq6KqDEKlA+L6y2sUiLqG94w8SNfB1mR+1coNktMt8ZYjR1aKiDowJEDkI1P5MbPITMx3DiL58gqKAqeCb5NlHXehmLUwtBB7IgcvviKTJoVG81V183xrgYcNWUTPSECMkIuPGMT5tMA5EcDhMe030T4CqNCbGruQ+1xqlW0on7iT8IWQ6H3HC+Zlo0+7jVNNhHGv0BMdqAlCZb3uksB/I4cnC7hS4xje11zgiq9Q8t/0KH9wUCdUKpw6SJwspbO4ObAYVfQqbI6HsAlLqJ5yOF0onU/zhVOsvoOCA2DDxYRYHPOaWsYO7a+MFrzCwtbPE1s5/Hz0NtpcSpYK3ref5pbRBLS59OG9c+wIftYWJfpDgewOi7oXmL4Ok5vPB4XwZLhKBvwf1ZVPKJqEaUXgIddEuJC8ZT9XfMPvu+juin4i/yGo+KunPj1WiBMCuxLVio9ZviIRg+nf8Gm12M4FklQpqssQ8WHaeK9GP8Yer4kV9MtLb6tdJlVIV7xz6DWtLIa9H4SH0qnLuC7lFC9dUryndL2IvnVdB4qNuJahL8fBzossPG3YBRt5Cd/11vu//GCrEusCmDoLev0LUMEhfw+3Q74i+LkAAaqcR1L6ivYmQ2utW3IpVTffRLQs+rRl313WrJBUDDHDqvhG0N36uaGubMotVYcptQcReHn/idaY/+h7F3+Yy68n5LB11hXUrprBk/nwO2e3stSrHkl1WOORwCtWkT77mWtk9pJsCBYBcXaM4r67yGP1cNLVNUbC495PcUr8Io+6gLTvIu/3vpzx2KUXOVIr8d3kYPClL6XbqAbZ3OEHv6yc40KoVB1q1pLTOA75vHtqclurBSKj4q00jftB5VDLqpXHrbYQR5KxN4g99KClBz7GnFnSWKLql7eKi/ynhLzmdcGoayWaPD61BRfTlrkLu12WPVvzGrURRC87m1KBSeTpsnRTGteJelDpk17Uo/URZpcUBOOQ6skLWKF9isxl8o3qTCtc4IXn7CD4xlNdFkhu0kRpDquc9uuq0mnQdqNcgGJADL5V4qZnc+pYFYTsIkOFm1GJygzaiqfOaS8aVcN0Vbguo7STG+7Ae0GEpGBPQXX2FO70+ZawvvH7zXm77L/Lsa4jE1GYRu11DpYSrhqHaCAEt4PI7OM+/htUtdSoLdbXwvqAVicVl9/TknAuj/VYIcMvjB5SY4vj+4mJSCweL76vwiCBO3H8dSW1gcvtXWTsmmK/PryDHZ5rnuhxWotTlhKLnZsZzPLG5DpImetoLD9HHXwAiHy2q4u3S82KOAlGHs89GzjjiGeYjAH0dA5XgNeIeYFTBR6yLfoovr8ykUt0RRqTBuDIwxlNuiaTaCXXaPGxytWLXE4UTOZszlriSyXStHkFiykuQ+SvEjcbc9F1sDk88pMKpEioUHT+AlrNQqcS4lW+H+SVQqPNS9FIbOag/w+r0URx3FLClIkhZCiKoLcedUZid8HpMOvfG/Olp0/hCN/HcR7qAnZ26vOJmAA/RiWtaAywBDgE7QTk2uH5f4trOR6tj1r5LSPUxJ5fZveTyrY6GzD7PhFXvh/63Zp+wv8th/N+0/6WP/b/2P81kWSYyMlIx+Q4wPs0r+kQAWhTMAVO+YJYU7IfkKaRVeOTUrOhhwC542C50v32SGZ4ey09VUGY8jsVhUpyvfeEHrI2E3ga40PSoYGRdmAcbQzyoUcnBCTNszh0L2kDF/p1s3zAtAN4IgVnNO4C1gvz8fN5bsoAzB1qQ/e16fFfBzE3+5Df1TI71tydLVgJk0EpK9pPaFTR9vwyW5bvkHmJHiuKyvX7mpZtjAZhRBH9Krwu0S7uFYiK48Bo/9JjHU7p4gqv63D3oRg/nZOmzZFW2JKWoz931eVzOxsoWvzG22bsN3g/8cv4zOt/2QW35PzunW/DQyf+dPWEyYbl4kh56SKgazFhZWf+LnO1E+w8HdQVN7sDpkAbsgrpiloQUYyq6n+d2HcOmU0qNWW9+zU+xhXwXAUu6zbwLYfbxxddZaj3DuRbdkdwluEXj1swFbC9oS1ONzLCAW6JWjcsMt5cwu6V4r99VwjndMxDSGTquhMpUVJKZ2xErGWNUsbbRLXfdnnqzeBVgvkseKH8fRyMyGWwQSFNrA4YAQKAqjXDjHbG/a/f8/Hw++3QBJ/a15fctvcm4tpInrea79v07e8Jk4veffyTJ/wQh2mqOmeG6RVkDrHbwHZbtvkwTLRyMhegaL5Zb+hp6XGjEMB+BTtxw5xU327XejmYepTJwBx11EK6rVLRhiCTLJ4R5IzVsfTeMmbPPK+TRAnV5JAefQ6euQZKccOZ596KzJnAEbx3Yxe3y9gBUGgbAww7hiOmCsUgeT+queVjjz67qMFKtMCx2D/HyBkXzlegnaJsB3wQtomVuAbl+Xmh5lQ4SHyLVHIMNiLgNX9kHKPZ3Js2nf0ofzprhUGFnJB+PZIIsS1xufYXN0WCQEJK8drOrzePsvtZ2OaMCxiqO26r8Id7oOwInDi7FQ7tCJavrSN3nnG4yEgewoTwSIgd6GmsyORenJunBPXSLOc8G+2TFvhnlGWRZzpPZuiudorbjqyn2NPrE8l5dHJtczKSsRFjq/6ti/0fDP8TRRASOJx57n49/y3Qj2pdVXuen2lIWV2RSefg3El6UWPnzBQDUtgImtF5EpDGd1/YeIkMzTXHcdnXLeDAihWY197Kw5xTk218o2lUuZ8AoQSf9bZF4926XVFicUGgHG0oGQF78RJ4qBENdAm0yP1f0E9+SX7jeYzUjjTC1zSIMpiWeRrWRTbUXecMVB/w6ajL22Amwu4dArvvEEabyJAS2J3wuCpi7io1bOq+l3XEh3xdQ04kQg1fdH6DO0JQjZoEON0pOJZtM7UOpT1NyXFOISZtNQ5MkqHJAVEdYo9Hc1f539oPBwMD7HyG9ogWXLLCsDIq0Xv1ebeSqNZk7NvEsMyy+yg/rxqeMbvwKlxKn8FyXqXSQXKjdew6AX1P3Zj9VwcU6kNRKZ6PUVMoO58v1d3DXN9s5ujPB1ihKkuHD8Zn/Um5a1MGUmfvWK2555qyqNoz9xczGFNc1JYwXAcGuX7iDGQkBl1k4cCA+JUrZwB+Ln2dhGayrgh+zpqGo45u7nQ/j19JOB1LuM/jqWin2xSeGsbnVDMuF0FtQY+ipaFZVpjLYB1SAzSl7wEWV19DIZj7rsIpZfoG8qepO4tX2bklBZBVl1mQ6nM/j/jNOBgWfh5Rl7uPqzz/O6l4ves4jyaANdv+dXPs2H8d4gjR3o+MhPWIlX8QP4XvTt8rG0K68XVBChrOGIjs4ULJ9AKYWSqwog0/DoH9Naz5auZIuHTuS+vUSFlVk8n11Fe9XpVGyxauOr+s6xrZ8j4nthCSVLfQeIe+o9gFLOQN87MSq4ZgZDtcmKs6pr75Ksivgt2P/Ha7HX/c0Fh0nLuc9GmnAR4JAlRkc9n8sJz1Cq2HyCB1f7IMpbzhYvrwXXVq0YOGCBeTn55NoPEq/hF/Z1SGQH+INIgh19sW/fbaShIslch8cnUA/7ZN3v4eOy6HJNKgr4YeENCb7w7TwO3TRKdnUdY2f5RlXbieqbJybvQhAYGs2Rj3PpmroZ4AJEWnoK5T758RPZFaxYM1cr0pyS64BqGQbkU5/fCRoUvCaZ6ekyThD+7LQ/x0O6A7TPANeL1TWAhxRMo6vvEuZSRI0dfXHvf3ctT8B/GWLYJx6XXf/zJ58XgF6SwyJBi+UtNdzOpI5gZd3nqdS7RVYMRfSse46TVz94Nvbw5k2/7B7rLhHoxwrBut09FWpeH7OHGpqavj+0xaMX7iXxtMjOXfhffKDJ0KMYLjLMky+OIkZ5UUsK4M7TiVYK9EQyLj0HYSpICnkBFi9ZD6NcawN/pbl5XBFzuJSbWfF94itBnZ2YXa4eJlaTYNOk7qKW4lw1AQ9M1TY9MmeZysDLtbWsrg0mjmVrJh8eTiXsh+k1O8IXwztTfCpTu5ENMXHGRS9jEi1qPl9rMkfcO19ERQckYZtdAUvXBvBX+6g4d2AvITC59lnggW5jT3MYwBTAYk7m/JhKDwVAONjG4A78MjMamqb4SBUsXZsHnyYZzq/QIz/DYrqWgipV1eASpJg3OYsHs2Hkwl2kqweKVzsZhqrfiY+fDffVILflXikqMGQ9iWEdAFtEN1DhBpEqAp6G45D0TGovC6YXf5N2aLdyYE6B4HpUBzyiPKimz7PkKy2FEe/z7DkNWgcSjmw4LwtDPKBthbIvL6cPy1/X5MvgP8MiONrDOB07gg+Ovkdj7R+m6ByL1lUSUOKfSBn6+CTchh7u6WyfxUdYaTfUaJUcDIeBqg3iHVs5D3QdDoce5jG1t+Y6xhBv6tX7l4vX34He2Mxb3c6fz92dbiQMLspajS9susUn53+nBsl3ciM/FIwHlw2PS6OtSFiLZ6Qv0j0ER+Xj+Hfgh0ZYmzRSTBCf1iAwErPwvHHoCqVW5U9cQIT/WGcoaVgBQIEdeDLWwe5cEuwpt0B7XprOZdTag9LX3JK2EP6CWnSB3JAH4EDscZqr4ORAVNF0BrRvxyShfPJDwNg8/LbAFpJy3ii3Vz33xZjRwHQ0EcgxQxne+5SgnNnkGODk9XxinfR0O+Ls/wMe/sJVvW+wbzsJ3yBJ/3h0/taKaX4gccyl/CZa2wt9VWqsGAzUV5zleYaWBMJ7UMOQs0dWK+FC3PIq2lOrs2VOFPfQi4+AhEinmI8dS+yV0D772Q8vU3xrCWZzVWd2F0rgCV7a5LAXgcnnxJxm8RH+LzkRaocQlI60JENvo3EujCkq4KN/UHcTZoVKhNnPuXneMQPLLKN9fY0pOD2Qj706mL85Aw+a76bV6pfYtDFLNGv9ZEiYaw28HD4Ad4Lgf0mOFvr8sP8GoNvEhJWdJITlQRNNeDjLBXJEG0QVKSgkq1u1YTv4+7A5TchfpyQwAVOSgNYW5dHuhVKGwCAqMliedJ9xJaPZndeN0rsStk8P0cNbVRC6vmTMPgl3APSkGUoMcVwKGMChTUJyCUbmfPB9n9W33Pe27x/9lP+vP0wh6L8+TB2o2I7Y8rzfB0h7unLCvjj1pPo5VKRgNeF8dHpTyl3JbZWNz1Eq1oXwCi0OzSdjkMdiBMn1YYUmgZeo8eVSFinhtpcnuzwJK/F7SauZCIm3R0qpUxPF5EgoLYjYRWCCfd46mhRxxdAG8StkGc4YHLJy4IA52pcPsbN1YTXHVLcR3XjZWAugBOTIX8vEbYiNlZDEw108i1CrbJ7ndvBzdhZ1BiuMskfhvCl4ljaqlR6uLr+u53eoF+lF+vJvwnZqlbu+nQD/UrRpswSTN4+m/js3Hdkhnl8UrWsBmsFXFuOr/kYsUYBBJYcGpbFpXqOW3aBPoab+Li+q+yQ73H4JHlAoV7MQ523CpTDBnYzqiIBem6rlaChsocuhDP+ffi4wqWI5ZTF92s3i+vO/g1LzlZMWnFtslODLDlFktLpRJbhWk0I513D3qBsoMtn7sNXWGLYmDKXGyXdxHGLj8HVRXDra1Hbs6oJlwoGYndolONEYCvWFFdwY7+JNpmfcpcN3MuyG9VElI+gTptPhq3BqrzoKNt9v2dJKGwtbkSFLfGuQ9Q6y0izwgclRmrULQXgTxsIDhuJPvmEquBRP0gK2qzYr/4622SuIiTte2zd1gq5aYSqg/fa54/qSBh8BPybufeVnJ4xrCErWviYTr6phKdzkgV40cvUrnf91Kn5rEtdrmgj+SnWaI8zIg8m+6qZ1zqJGU/2IyM7mzbPzSdf68d64E/EGJDP3VKeexCJvg1AWcBEUh02qrxUFAABUnTZ3TKenvnP5iK4NJTx/N9qf5fD+L96/v8nZ/2v/df+Q7Pb7dy6dQu73TMxt9R+wxg6cv8ZJw8zADZHiQLWVxeBNogis0fWJ1btUAR39fk/0d5Qh58Mv/SahT1L6fgG2wvp6IwnzwY/lcVAQGsh7WIphdKzWLYmMS5SIP98HFE0tHdMo3iqAM5lPcjewjdZ+fFqEmJjOfbeYhZVZPJ5GbxXCJq950joOtkdxFKpYG7v0WQnQXkyxJd7oeUtZXTV3yBJA3/UwB/lSXedt37hfaoO6vRjPPJUqR9DmpiEGxc9x09+LZGqlQmmmyGDmFh8k+PNe1BlOK+cfC1lTLAdYaQRJhda2ZKhDLQLOr8vssOAUf9/dk7r6eT/xCIAlaWKJ/3h88QU5jdKUG6Q+QuTQrajlaDEAQ51g4W0IYavTMtY6jjE+aRH7pp4fGpuMdoIX1XAB6kT7kr2fV1m5HdbDlfbnCQiZahi3635L7DeXMGvWQOZfcHqlnYB0Jbs4b6IC4BIbqUYnhANeTvh+gc0KZ3KkoTz/FlrZ8GluQqEK0BzdSEddKK+0zBXDUa3yVoqnCr2mqB/DmQ5+t713B4O68crPR5jUKPv6KBZwMoPPiAhNpaULxaz2NX/jA77f/QeKspLmNm6B4NCBWOqoTMqa/2oc+jJscH0QqgweiWQjAlUBY90o2MbWe5meeiuvsVYXwhOh3VX31A2hvclP/IyReZwWgVfRiMrmaM7Mufw8KYysitbiHc4aD+0FgyWv5URkCQRjDflE6K6iuxyHCQJIRG1sytcFfI7Iy3rCa7qS+uccl4rG6E4ltM3jsuuT6xOm09DtaZ1l9dxWOtJtIV6199xm4NTdTDn4hyBgPX6/dfCJHbUwER1cyHVmycYZ7Iln2ONs5kZCL/cHsdp05uKI15Qj2BZZSV2VTX26tbotF4R1cob9DfuJ9a15pRouBpzEqUKIKyuiRAak5SsrKuJ0+mYBQMDyni7/3AitOcV7d79YmctpNiUzmu2pQUbqkQi6OCxGyxfsORvEe1/2eo4ZHeyZOEqVq5Ygd3YnGEnZvKsYRHvxw3kVG2G4rjnQg7x1dmPMFn9WXH8e5wJXjUoDj3Aqz5fAZCogYWhP8PtHz3tZRdZGViEGuiaBWm2QYpj1wZ3Y1M1tNab6BS1w91fAOyGRmwqaEGWDV4NthNu83IOJJkKua074QauPtZytkjobUlghL+VpHzBWDljShTIx3Lxjamo44MQLd/VvMfnquEkOpWMr8KYV7k/F94PhYxW5wWKud6cTjTYUQPXE2BR249oaE2cGbTVwZ/N4OB/IGP34IQp+Guq8KsHpzT4yAJVYmQJkSFaU6dMQmZtZnjEDgDuCSgmyOmSuTv3qpDNBNreaMTbpTAoBxxByppWjrKLjDTCdxGQ1mUDKqcX8tJcCIfG8HBgDV9UwIWSzgq56XkhLZigj+SZUDjcB25nZTFjnkdmy9tx+TtEYnzAFXrE/UZy0HlUtmJF+zlTP/bWwoGidpzK/kKZ7Avrwwf5o0izQKy+FI1TWcfLdXbqnGIeU7AVAfWNT9gdI76nEWefxRk9Au45BD3X4pD0fHjiWy6f/ZO48vFITovnRpxOgnSZ+GvFtf6U9iXcd9F9XFvsY2xIF+wgFdBcOiHkch6ywQQTensmDwZWEqsW0mqyVRmwTjK9zcDIA8jaYpCUTiHAvpw/8JVEgsBPU3VX+89VTvaZhITj67v6uxHpu+sajAVWrzq+K1Ygy/DV2Y9ZflwwAxQBxvA+BKUFuCXeGyYeCgal8qhrqG4RegSd3WsciX+Q7PaHOGcWUtUX2mwV9bnwyEn3Vanoq5L/Vk6659B7+fpPC40OwJfF8FNVLi9fv87hxSJZ+cqmbhhSvMbijsuhi0DyRxfOZcUQT1+XJARLpPv3ED+ebMdwZZu32esY4l9Kay08mbSZ5holy9IZeQ8/VgkZxPcH98JYqkxUa139bXm5+Nsvf7WivX5MVwFGVZ2n1haCRXO2xxd8aZ5Py/wFHtZW8XFkleyWp+udcoZButcUx73p/xQbq0Vi9YtwCC3fImRtu3wBtdlMDXrfve2wm6MVgSRxXSr8ZNgRFsG9uouKNq5/RKeaIWRFLyI9/CPl9xzRnzcj5vClq8sWBm5DLavcY0XIA/OZHxDPFH89bwbG0H9EHfOe6sbihQvdMos/15bwTlk+AZeXKBLRsgxb7elcsUDHW+vp6j9acVmJJa/z69Bh7K0Vax1ntHItpFGJifm3angurzP4JSvaLbW5RBqKGeMLsT45ynsO68l7pXChDk7UgVrvSbBLEvQNuUJPPTyUFcBJZip2vaL9gC8vz6cicCdp9WR8vauvNn6GJVductUCpQ5YV9ZI1A67MAduf4/ktBKpNRGmgllBEGm/oji2znSZ4WoxJ2gdDdboKj3m8IFctcCoXHjm7Cs0tJbVv/J6MMRq64j1Oa+Q3T1fNIq5ew9wq7TD3yYa1A4RMJtVBJccXsx4QzTzKncw2XwMJ2BHFsoflSmiltelt5ievAAQAf5pgV8IqcHDo0Vy4tijdAw9TjNXrPUuuWlDJEfN2SyqLmHE1hOYfToqmq/1O8Lj+dDmMgxSO/5lcH4U8OO/aGtoPxgM9B0mko56dTVdY/5EZ/EKHKsN/G5fyucVkGaFozWBHlYzQKt5zClaQ5pVPK8UZx8XYHSRkI902XlHEV8P7U2EpAys49+czTUCQOREBIMpvwynpuJnv0JmZSuKaoUP1/Bxna9+jn1lIrmXoLWJWvRNpwtJ/fbvsTNLrKNrnDC9coZgiu0fArGjIXY0va3zuf+MA9WFo1xyvC5qIgNoA8mq60uxRSSA7w8swWg6Kmq1bW8LyVO47AIrddbBsRED0OZvErKW52bCejXHY/YwKxDi1NBSvwFqBLNGlmF449Xc5/rMOuissCVJ1PkF4qTNDEz8wX2P5qDB0OVTMMZDRH8Ol8ym0q5hRy3MzBgNQS71icpUetdddkuZ45TQOktEv3TaoeQ0Sf8Gm3UheAibqmFXNMxPbiDFn/YZ23wPYJTh/lw4VzrMw8gxRLHk9CEWFHsSel3K9sIJwaxxakPxN7di0KUM+qScZWjERMWh/xa0sqsb7BD9p34uWVYGS4v7gaQWTFx9OAR3xGz7iOjq/uyLhVa1W4Tc+YhUCO5M7/RFvBwojhurrbuLLRuc8SM/u5xaySmjyl4v5PIj+lPjTGDpkQ2cvTMVgzUWVflpUYcw5T2wVDDYP5fJvj68VAQfFg1UHNfedjk9rvZEDaQmQlfTKrBWQ7MXoN9WIio/J9ZHgOk6GGpFQtbLfnH04bMKiFSBv6yUzaXwEPdWv0114DaGVp/kcp0SCKa+/ypHSwW7O0QF4YrkFNws7cz7x9aTUtSHWUfeYoBW94/VerbdHsrlhGf5M/seUqumK7apa/YOv576BZwyn1fA7zenCCnsy2+jqrnO7ltTqHEt6xN01RjqVWuih0HnT7BqYnC6avYF17uYTjtYy7Hb7RQWFuBwSVnL7sq64puKKX2Ybmnbuf+Mk97FvyFpPYDmeoar0wWE97df9DzvsUWkRaxxb2tDwtJ4lkia3P4Bcrbh56jh7VIhD/pb282oZU+CIlG1hWt9ljPBTzCkOrFJ8UwCrr7DNlc+rcDig0mV6Gl0OtF4df7ufuVoby4XCT3qEz2ej1Ylq4REcoB43wl+QkFnUoCVeMnrvJ0+YmjuLLfEseSURVIqdhTYTZDyPsNLlxFU3YOW2e+LMXdHR1ivgSPjsQy6yP0bc/ji1v3MafoLMaavFPfknUCX6sGTOdsEA7ndYor67PC0OzX45HwmavUWHkKS4I30gTzlyj9ftOBObDU0SQIau8BeVxYgy7Dz1jSWHNnEE+3mEmn6XrG9UOxyEOqbRteYzUpJS0mU3HHioLMOWmi95rjqO7BHAKzWVMJzNwaSWacEWXNhHmfiRAKk2P8vcW1rJdgcCzUZfNpzKK8EyHwdDvdG/qDY9cXWQ5hxT0vON3qcLF0KJDwsVNJSV2E8MQA/bQOGP8C5V+Dsy24ZzySNGJcbm/d5tnHYcJQ/SHj7Ea530cABvfwuf+j20UgNB+RrZFo88rMCMPE0jRz7qHIIFTOTPQgkNZGRkbzw4uu0aTaUnohv//8k5TkaiAd0Yb/RrENbDmcqZei9mX1qWa1YV9tlZSIQ7k72/W9l9v1dDuP/pv0vfez/tf+JVlWlDBZ9W1nKiuPCDaq0hAuET2A76PMbNJ6KLENQtUDsPea/A7a1gLw9YDOhz/mOZ0LKkACVBA1KNFHVZQ8PHV/EdSu8UdBS1AXo+TN0+gj8mmBzCPmgFX7RrOr8mJtxU2+ZjnBuWOFASTOWbfVnyRtv/21A+6DNcVcQyy7XUmKH1RVQZ/BIz1GZypyg1Yx20bTvmgwyNvBwmNDuTiyYToQuXgTudnUV2ty6EJ6/MJV841GGNfkS2VKg2D2zIpNbqu1saHaCP+7v3kD+pZrB9vP0M8CGarhU2R7ALb/36dIWnN4XSvRXNRQYYbWynJbC/lOkqk0byCflcLgyjIvVTyk3aPMmj2UPw+wUMok+9jxlu9rADUdzInSVTI5IQ+1U9qH8pq+iugmfVcCW3H53yXjWI5o2VoM5cLBiXycOigJ2UBywV4ykXsHd6s7bGHjIw0KUJVkkkYI7Q9/fMVov08XHxDUL7C7o72by1NtM/Z98EQ7TAyTu820giRrem3GlTdnqArmrGnZe4GzNDP5Kf5J+CevYt3uRO4i6yyuIGsl/9h4CAoKYVxjATtVpbifIfBy5VrGNo/wcIaHbKbLDpxVg8vFC0kcMoLzNRnLzxrJU35ZX2g1zS1jUW4uq04zxFTUhg01daWgWp4mZVTkEpmmpccYo2rwXFLIsicSrS6ZWW3eDJ9rNpUnwKQDU9mIhl2MqgEtvMM6nNQa1d79wCCSgtRKqb/NBqzbMD6ugzAEmlFKIGqeVIBkaa+DlQDCYvdhithqGX3mKj8KE9NB4X3g0op1i/4qCP3kk9gjB8t8wOIG3U/uzohyKynpA0xfAVwT+JJz4yQ70Eryv/4aN1cokeIpuGB/WFqO3xPHG7suUNfKShik6wnOBv9JaK5IxxxufEWjNejMm8Fl2Ol8UxtFTD6GSUu6r3jnYac9gak4g5XhAFdTmMNtxmgmux/R0Iayz9FPsf8b0AA/lQ3UF5Fz45l8i2sELhTp7NkWl5ezz/4kyB0iy7S4UdY4cxzGfHexo0ZbPrdeRw7yOKqkJ1MTiX9ueXBusKh8FsV6JW1Muo/VVNHbFvRrW7KtH4M0IreLt/veh8ko2WwO6M/vGfZypg8Tb8Jv9Cc+OTic+qhJ3UuzXSDDsbQyt5rrlSUutTcHFGn63eKTolzvFt6MqO8qYFsu5r8lnPNzmXYx1yvoB9TX7TplhQ3mIYDbVW3U6j6Q8xYIQyLRBaV3gXc/3ae1Fwazxg/tHLf7HMnbh4ZH8cl8vjsTCiThoZvbUaaD6Dr9GzOZzYys+C4ezrQ6hQJT2Ws9zF4Vjl3AHvrK4+ma9VM+4MnItIiAcVfqgO/hdb8aMn9kSDYU2OF/ayiPfAmA348z5gwRDJc8Wwq6sBwCP3PTjz6Rw5mMHx5ZBQPfWREcp66Hq1TV0jNpJtJ+rZt62VnBSOKiSBKObf8Cjbd5m0pYsaqOVLKXGmj4MuphJtxu7707GBLQgvWoRCQUvcKTTLyTXva1sNxXwWVgdU/yhlRbUNAgINXqMqQVQ54TQygFingrvA4kPg6zmr9uTSCnuzZbUGeR0uC6k0wAcdbzZNoHHuk3gYuJkTqm3KZxxR6On+OHGJPFcZXhIekcAg2QVqPRcCf2D5JRmDPGBHTEgl59T3nPtO/SP/4M7ibAqarvymtO/5/egNF4IhKsJ0CXoGP/K1ubC8p/2/TNE+uzZVFbmk1balZSiPgAYbr4NO7u4pdvqE3x/RsNvCesVx6nvTzoJ3h/cj7gyL2CJIQI5uAfVTlFv9KeSeHfdHoAZM2fy+fZ3Od/PwbRQGOcT4JaTnvf66xzbvZsjDicHbQ0kcswiWXnu4OvYTnipSEQMEHLRgMpRhZ+u1A0kkGWg8BDsGwiZv5DtGObezd2/jkyAK4IFE3/pHmYWw713QvmpxovBhGfM1ksQ7HsDh63c01ibS8+yXfTW4665W5WgZILHZP3EF+HQXQ9HBz0DNzxoa7MzhG1p07hR1gaHygxj8gXYpt1ipJp0MpvlsiAEuoUfIUar7APpIa/xQ5Xoe9MCwK/aJVnZZBpEDiZC7VnP3TU/WsoY65dPNz30Cr2In+NKg+bLBFj3EJj8OkPbvowBL6CEJOOnD3aPSqvCIChdJJkiIyMZM/Z12vTJoE1fE10HXMYeeh+frjn9t2v4/XZXInrObFauWOEK6GgIlOH7HvNowTLFdVX5DWbTjWewuE6ukPWx1dLSvJOe+r9pA1AbecPvMb6uENJqHYwNGDuRg5hbAsV20EoOVLJnjSXLsKDdRyz2D2W7tZSd1VmKXSUJqg3XCFdBEy2YEmZAqOtr1AVTZksmNu9VcmzwSdFYwWRMWQqnn0X1eyiHu33B8VhYFgpRFuU8FZT9Dp92e4OJfrAu4YZQKag3bQDlXdewulLMU7kmJXsdIKFmN0/5wzMRd3ixaUdF/Z5KazRXivpRYw1idNx02BSqYIB2jtrB2OrH+PH2KDB4JRIlCSQDdoSUYi+fOrCUCz8vdhQkTOCrdLEuum6BLdXroPlMaDnH9a5q+Ln5bpaGwigjGOwNmPM2E3q5FJsTrgT+hCQr5zFJG4TJCXtOwOP/uuoBU4D98I+BOGPGTqFx8BnuSfqO57dfpSjqXcV23myCu4Be2kAStBPR1LRneTncwmsdLmsgfjzr6tpzWHMCC3VIDX2PhAmMywN/GcaGZGLXRAhZ1djROFT+4H+Yw10l7H0l4rLHibqzLjtQuZKfC5sA8HbTH2FPT5H8ajkbfBu5fTMnUOQIgJCu0PgZaDELri3no+5aDOpqrhf35KpqARhdrECnE4eqklJ/AZT7KvEa4YVLhNxw+WXAidq1jip3wF+5A3D4NRdjb+F+9/VdtMDWGvi0rEoEeBHf1NQOM5kaUH9tKsH2koWc+gG28/x2j5z83yXCnC6pfcmbIVd8gnE1+2jtUmWXnCqyfWfC2EIIbAMPlvOuWSQz11QizhHUVnFs384rWVfuS4IGwrRKKTqC2vGHNYEbFthWAyWWJFD7iaRb/DhkGTqme+bNGxFjhcR6k2cx9TsDSBgs8QTUdiREp4SMSpJSYlOSgKCOECzWtKuj1rAitL5NEmsNtY9g9rmeabPUrXx2+jPUkc95HdlBSPU1GrvyJRNuteVK3FnFucsbPc0DuRCvhtN9X0d1+yuIGwuxo7FKgRzJGk9WZUsAVMWud1tyEnK38c2trUzevZcuaX/SSZqsOG49I8cGLCqFbG0/2D8UtrWE8otEly+juqmQ4G52Kwz6/aHYv15680aCmncDG3zJYb24Hb2Ws2bROVT/h2j4I/nwgeSRgG/Yn84dXsvj5n+u1hOSms44X/juzhBOVLyqaHdEj+JY1oPuUhCSU8YHMcY5Q3srGEr3XLmHyyGepEBlXSXzrt7HkRZi/thbCze61QqVjADx/OvqzO6EnXeyr+E9xfld8iTzrNX0y32ab8zzwCmxs4M/rSrvh2MuYKcuBLVGqQSiVgM918L4GmjxCr/GzUXlbMWh2xP5+twKJK8SA7VSHOvLfciwwuQC+FajVKUxNZrMHBfG7+2MzpwP3OVpvDCHeXXjaWkX6lFfZXekbuhtwUy9s54Hmi5TsrokFRy4H+wm0hodJa1UgEEWh0JzyQuYKUkK6WAJlecZFR2DC7Pp4exMr+vH0NlcQKtgF2gsvB9IMmYc5AX9yZMRtwg27/QcuzaHduef5KVA8adGgh6FcaLcUZPnIaAlGq/SEJJTjcPYmHzjCBZ+vJFvP25Bzrq3Ma4Cvx1tuHj/TUUsNMr3Bu8P7sHARt+L8Vsb6GYv18ds1LKVexuvJtDiBRyxVtLcsJagmK/Qdm3K3J5joPySp738Mo18DgJOloTCiwFe7D/X2iCHCD4sByTH3f6YrYYgFYz2hT1NLxHsFLEhTDmgC+HP23PYb3YwJg925s9S7urUU6UqJSfkJ6oMYv4Q++aiqrqMRuWJS0z3v1fU3c7YALk7kGXwNbXCR4JuejA6vRKDTgcPqDfysC9M9odlkXfc5Z8A8InhJkHUOaEsZANVhhNe+9rh1tf0s83nibRj5B2/wcc3zirkpG/dvsJURKLv30l5VgDmS8WsDAPJoZw77JLwX7Y8tIUvRyiZr97MPqvDitPpvCvZ97+5Zl/DHMb/Tftvsu+/9j/WbHIglVINpb5HuVSngQkm6LNRBGf2DWFgzCck5b9Cs+xFZNldtS0O3AvmPOqavkOc1sZYX2iZAUWhytoXKhXuRY47yJD4CDR7EQJbc6Dlp3xTCWrJgV5dI9BpXuYj2TBKYDVVsn/fa/84iFVWls/rpz6gdaaoW2AKut+zoW8yX1VOYkct/BIJPyY1QOvd/JJnYsSEub3DOnpXdxS1JwBsVTC2mK9063lb+pNRW08KZ8nL4vM2sTEKDppgZ3ET5aBsiGK+YSrvuggJKkl2s8WOLF7Mu4UC6byoyEzHTDhi+9fO6X+KVE3u+iRluY+zqsLO0uL+yg18k7hhjcQBnI2DNmVv3XWMa6Z9jPGFpbGZqKlUtHkHkCSnUt5FkuD3dutZHS6SFjUJsxX7vtRsHCtDBdoz0XhAIP3qj6sxYHPpzn8eDvfVvAzmfDj7ApScIiXhPB2yhId/K+Tju675N3s/lpdB79OTOGHbcFe7LKlorYU3gyFCc/uu9rO1s9idPoX3Dr7H/PW2v+1//+l7eOCxiXxaFsglC1zOu58q2yjFNvoT9/NcLw+KvaHTotNoaH/nO9rZWpPgq2SDAWyIn8dD+TAsKEvUm/O2ihQ6mBbRWScCuQ39oRB9Bj1ifyNQX4AsK+UMNZZMHmz5HsnBIlAdYD0Gfw2A/N0QM4Jz1rexOcSCX5IQzvoDGdB+Maj0ZJs6k2cx0FwDkSolIycxbRmlydBDL+po+tR6BRwlFYWqIPJt0LT8QTZEQVTVN4r9nYWb+SRc1CV4pekPQobYZd5SnZdqIqDzx+4EpuQTRfvUpiwuEzVfCu2piuN66nsoE6oARA1hcclznKkTIIc6W7hSG951bl+VhaNxMEC1TdEWVnmO5wLghhVW15ZjVXklXm3V9CTXHaSAu2t61F9b8yv/Z0R7vdWjUH/Z8BXRKlFLxt4EBjTovQWmi6QnvESt/ha3wz9Wjl99NnLBdw+N8+YRlDuTm/bn3cEHACKH0K64OUdNsEjThcY6JUupbfoHFCaF8GWlhRnp7cCrNqP3e8qwQa2kdDYf0oTyrSt+mWEDp68r2ZL8NDSfSYG1I0685prEh6GVhW+mVgABAABJREFUCLarbgqn7+WD65m8ezclDVgiQaWbWRoCa6tgek6iUk5a48edoD6cr4MhObDyegOgBJAX/gaflsO6SJh8b4t/KXnZRw19VSrmLl7MjJlC7nj77YfYUysCpRqnV3JKbeS2/UGySnqyrQa+KkxEQXnVh1Jt9wR03VJ91koI7w+nplHc+hw/RcC7IU405juKa7bET+CxfHi7FJbsPYek8pIkNMazqd16Zrsc8oYJAkmCv8ri2FEDY1Ut4LcIyPjF3R7ik8M7/YcxqN45Vfu7E6iyDH+lP8knJ1djc2jvclomhffj3V4TeaTTVKa3vltSMErViuuB61hWCjecTZSN1kom+VtZFQZXEsDHomQpyREDWF0pmAePxBxHbc2Fv+6Bq+8JUIomh22d1Gzv4KO8LknDocIZ/KjeR1bodxT5f6RAyEoSGCwiMWh2wh/SaxA3Bvb0gdPTkWUZs0PFYZMIftSjkOvtVdMjzCwSzLwTJqWsM04besnJTSt8lTaBUrNSmhQgLdGfnyNBPqOjn/qf1489ekTJXJPtVcLJlySwlHGv0UYjNVQVjKHSomS++xTtobcebE54tVDFNXUfT6PDig4TKgTA563sHqALVuz/UMUHnHscnn0dRj47idMpKTw1ZQqLFy78t+u87fY62FeF/P9j7z3DpKi29u9fVefpyTkxkSHnnKMgWQQJIioiBjAiCGJABQUToCJGVERFUVFyUHLOOQ8zMEzOeaZzvx92z3RXD+cc/fL83+d6zrqucx2Z3bWralfV3muvdd/3usOeLzf2Ex7blI7DNZdIErBniAhIx43DjBvJW7/+5WyBYhGoqAuYnZTTOWJSqjboz7zA7Xgjh00QmVlKUYjHPGLKo2PJFsYYm5Ff3IvDmffi8FcGjv1LTzDRV7Cx12f1VbwHtUTz7qUnWRg5kT1NG8PPerG+HrgX2ZzLVbOWfBu81nE+PXxfVfRbxygsskP/3V9QmfQenHwWdnSDAX+y1KVsoQbGBqbWy+cBUJPD4rCL9NRDm32LOKN/V9G3vdNyjGmCzbg0DIzOW+5GawUvNunBSJc06l0+oPeoDer5DRVXmnn7qx1sslj/vQ9vtfPSnDnk5eWR0W43W6IhRFuGxqn0OXN9hvBcSRWhKhhtBJUn6M5aQc/yJUz0g3G+8FzIfrAqXxZZkrlhhftz4WxFzztezxshYEpx4i+luo+TYcXxL1ieWwcYUk5gTW1vM7L7QModMLsQrOEeYBhrFcH622hcwRy5Lkk06KCij4Mm6HwbsvwmKP5ubvQ489J6kaSBnr4V4FACderm6XAVBHknJoBNAS/RPVMkW/7Mf1uxzoln5QScVNhiIaB1/X5MlmFuz/HMiyiic9p65XrgsJKgvkKSxhX0S8oVTJ/Ss3BmNuhCuFb2EK0yVtAobRW5joki+Rnmmi9cDNZrFlgfDVEmL2bptWUUJ4t3KzJ8M7JTWSoizJJDsgaqTP9e6SQSeAcYyb/eUwkJYW09ECc+4ALjWrxDpG+aVw1hO4Pld3nEX+xLclsdALOHT2suJlCViqZe6cJrkeu1lhXVcayvhqY5RRRJvbmTjTLCJ41PoLVlCFnVPr9j1cbx46Dh/B4lgJlhVesUtcpVKtBaxZj+UZwCzWbBmbkieXtqJi+07YeMYBeHysVCSrjLZ6I+VWhXLlbdXz93KgKMxcdZ1DSAsSnC955+syWlUXOg12/Q/gM4MYP7akTbDSu8fvJtkZjd2Vesla1eY3ZJZzZXi7dMkn3qAUaSBPP3/MUrrpjtRasBhp4S7GTAJgdRbQ2svxRj0Ro4cJ9Idqd+wcTkYByxHzDAADMj94r6dABRg0lv/xXVt94S52mgvgHrJ6zHx5REh7RfGrQBNA1tSmzxgzTLgJlXxikbI+9iua09N12xZJUsQcuX4J6bYIxnVNJb3BdQiX+NCBZbQ7uLeSj1MzAolYzuxKb1M4mETmL+c6K9y2f1NaXUkgOVBHOCYGawC6ygCxfMvisf8HBwKPaIb9lwcwImPw8JZI0/uzqt52lXnt/X1LTBPlAX0ZsN1WBxwpnyRJxx94uYUFh3ZBmygn9kcyeJHe2CURW4kh5JUyCwNRZnIFR257nY09wbrGSzSEX7eShUsKjfv90BS9AUUT7FX9znpYQL7Hd93tEmJSgY4H7W8G4IrM5PIkdW+vD4JrDTGoWvxsyKMIjTKAFV1Te+o0TrPqZNkNuPkmWI87/EjE5P0iL0ILU1hf9IrUeyWBnkA1Z1KXYsiva6sW2X/gPHojrwVJsFFOuGQt/NyKbbfDUipT7x6pTs7m8uZwc+Rx+ipnobFR6qRD46t5/+y6VfeNPWnGNNBNDJUwpWkgQr2CFZsMnVzGg9Bg65JJIlGa05nWBNLRIqbKpKzmmmirgcQPlljDZ3jfkIlR3jwc6CtarSgU8s40IC2BHTgfzLq9hwbSay2h27K5U68WhWBIdNCHCvSvmu22JGeChGyMp3P6gthcaJdLhyjBEnnSSn/YTklyBAEjdXM7bpm4oasCpZJVjkVekkZI7luQ5ziSwZy+hcWFDqAd6tTKOjPgd93bmcMtqirfCLv1BYGHyUWxXuvawsA12/FInV5rNQlZ1kUvMPiVNDVJqO1FBlDEJlr63vu6OhFr09S4B4Gj8GyOgrLrifDRKf/HqR+LFbObh0JW8VXWVdTQVfFEH7k5fp3KUxy55pX/97jWwh1CcTo8a1nzYViRqZ3VYhSdApegv3t3qTaRvTSAv0iIPV5jHE7wGat5rO7lp48nacUFers3Ov8EDMQAoCt7K0FH6q8pDQDmoHYwqZZxlHWy281PxbYvQnFPdMp49IviXKo0RqLMiSDXqvE7WXdcGsy3iVP2tgWw1kVyt9rK+ub+SxdJGYNTadjrQhAtK+gXaLMI8ooqQ2GpXd6BrPG6Le6F37YOQ1EVcxp5BXNJDYS83IDPYAjMoaXjc9yvwSGOQDDwcXoNg3Jz/KjuhXyLFDVTK83MytSIPKUA8Ef7b3KPKCfm+gUmO1VqAFXsIt5Xkn6wpsAXIqYWIq1HrFbxwuZp9Bbajvu878TC1Y0fiWeAz6IGwO239r9v3/xP477P+1/7UWojrHzc7hHG7Wiz8buxaCLS3h1POgDcLm1BNVNpaUvJcp5SmxMe24HLQhOANasaHMnysu/8Y7MVGY+SOxrR6mux4+jjoHRUcV7XW/fyU/hHlHrinZBcASn+/ZFwvxRR8zQO38+7IKm1eicgqnSG3zx0fjwdTQh2FTfYLp9iwsTrA4vDYAnT7h6VQhqXGoVqZI7iK0zJs+D3EToPwKhxMrmOALG5t2RVIpdUD8q65yr1Eg1+alDlFOyrIao39LTE7IToSAvPvq2WLeSOejdngauJs7b07/KVJ1WP+5GM0pVPpc4JZzv/JHdgt+KrEkvl8QQkXgA8r26gy2hH5IvAb6ZIJTE6po1tVm098AW6Phs47vNWD2SQ5DfbrEe1MTosskUg2T/WBG8iCodAdW1LWptPHPACDGHokPpaI2QMflENZLSBDINmYEwIk2WxrUWzhKR36tgkKbDqukDDZSncH2VmOZXzOTN0MgRuMl5eRxreeubaGv6s5B1H/6HB6dNq2eTfruuXkcqlEi1iuSnubTcmirhYuNVISVe7B9Ss7gd+0p5OYPMMS2hrsuvKSUuQMKy5yY9sD5r46w/o+uilpHVN2khflr7veD4f41aDzkAgBaBO/k5d5jSQ46jV6ugJ9kOPYYABb/7jy+KZV9t8S7YdK1Fg5nSBeIHcUFXsds91GMW70ZopiZeT8fOI6yJxam+yoTrzXBXfi8HHbVQI9MqA3q525U6fk88H4Wl4LdHMs7B3+hIkwpf5sdNIzemRBoD+O++D9E3QyXSRIsaPYX84MhO1B5XkkCXIGNnyNhTrAy+N2z5j02RkGt7jZ9O96HutjjeJ9YLlhaUWQHVdEYPr2YIdDTdWarJkcfRmzoQWYWyKTLSvna2MIdrKjL1Tgl1GqPQfNrwnjNuPoade+GwCj9BcXx3XzO8WkYFJ3594h2T3uotpZd67/jZtMCngyEnyuhUEpS/qZ6PNuiheTWzeQapGsfKtpVKoguHUeLrCW01HttyGUVNllDMy28nHCCOE2qstknhiJ7JPvMdlaXBSJ7RJM0Vad5t8k2uukhXg1+ngW9JYlUnuAvVy5sdhHYem1xI1E7LCFdtZWM8M/wr2nLO6FlAs3XWrCNpGbP89yN7vyU0oNvEwdz06SUCAmu2M2cYBEAU6GUfkQfzqn45/il/nIaQupK/QZwxAQT/SBQna6QvJzl25x7/ODJULjSJ5mMrKz62naSBCvOLmR2kWDnpflP9zhvGAecv7AltzurK2FBTgvlh1WTTSODmLN66iFWdrFh79onWBO3RfCqlyqKaY1/Q2XKUFyzM6QzP1ZCrVOg4hvUlpVUGCXxXQyLU8rwyDIsvjmAl4rBhErUgtG4JYLKzZF8fGwlR7LuFf3efQQ6flh/7MXCvhzNvocuMRvReCVUqh0x5DhMmEM2YNSmK9rI2sCMRil0MFiZWwy3Vf2V7b5JLGE9Q3NgUV4YVo1S+rYu6DrKCE93mIfKXgb5u+DcPCTsrB7ejo/C7fTxrcU//3Mh6wIgq9ics5SfXPmC1XHXYbtbKlJ9dSEf9HqYyJKx6KrbkKYaI+rMFB6E1BX4W44zruoxSooHsO/6i0hGL0leKYrbNni2EL4r76hoI/lRxlV15tcq+CLtPvItSlYzgIVYSuyQeMXOFMvfR6RfPLaGhf3vYvVoEdoytVgC99wS6g7lV1kfW8VoX/ji3EI2FG5SHO9zehpvhIANWFJuJ9/gEbC++T0hu/15vuhD+lw6S7/S1Q3Onx5yF7tq4f0w6BEkQBYrv/qK/pq/l6wcoAbtGehesRQ2NxcJPf6F/FmMC+xVcZ0O6jsg+sdXQd8N4LDSx1hKcy0EymCQlIE7lTYQh6WZ+98qj7XXvzkMv4yP4wBxthSahR5pgMZNa/Mh4TeF9N9bV56AKPf8KcuQEyzWF6u2wJ3IiRuPFNiCIemxfFIOTxSaWVupTPi2zXuCxSEiiG7MfxS9PlxIUxUfg6NTUbtAFT4yvBl9GG5+5z7YGMfGii1sy+nN1di5HCxV1hyqYzD9VAndM6Fa45GAL7sEf/VioK+Y55tlwHP2gYp7Sgk+weMdn6Gy4K1/lIj+euVK/iqPYHM1RGeV8L2pu+J3Vkxkh/xAfwP8Ee3FltUGcSDiE1aUiSTRhMCz4PRAdzvsdDZdpJUO1pUF0ref1zp2+1e2RUONA76vAIfaPbfJMhzMHM/+qmCKkmC6frni0Gj770z0c1LpgGFG8D8xwF36IP0bXmoWTxv/bJppYEHURsjZBmE9YchpaLuIgceeZUo+FGZOp1Gwcn5zRAzmHcdBXi8BY5pTsB3qzG7C/+J87veDPTGwstM7eJuvbwvy7LCrFg6UvKxg2/aKXsWm+2U6RW9lf9FLcNceEdjFtU6d+JL1115g6eDOtLK/7PEgKnjcMJ0Xg+BALczJDQG/FOGXtpoP4f0xykEkFM6gUfHD7r1B1GAYVwH33Kbx8Sf4qAxmXx6LLdBrTg/qwOflMNYXjjS5hc6i9ClCD93DihAdku4/K2zMBOYBvYF+kqQA4gxQCwnheYsXM/OFF5AkOJw5lme2nUMtW9FaPNQmJIn28lqmmO/Ddu1TzhSNAQ9mC9c+4n7fJkwq+IBrIW24T+exO3BYYV0YB4M38mskIDkaBu5y/+LTMC1HTfDqkRU4NKGQ+jkUHRXytpnD2FgNS8vgaJNiBeD07oCpzI4V+6C1JbEi+VSTCeZiqMkkyucyagmi1LDCfzlcWAgFB+DXQEDmsYIA1rU3EtRfop9jAFS51mBdKKcrh3HN5WtuKovEHNAH4sZC81lQm4u/0xPcpULSuKQpUqZDmwXkSOMIK7+bARnfEKM5Kq4Jt09wyTXdjg16UzEcRjKID3D7v7raS0LBwFYNtioC1WWoJZhQMYMHQk+A2YVSMkSR1HwaPuUChCw7dLQuHysSkA4bZG2go9bKgItp9CaZYSkrhCKEp+0fzdz4I+KxOxuG+lSSih56qE7SMCjiNyi/KvZM+fsYkbiQvvFrsKrKAAjQB0CzmeDXBMOfSTh9zrG5k8TmThLXK0836Duh4Bn6XDpHct7cBu/IE/lP8lwhDPOBIb4u367nj9DrF9AEct1axbm4Z6jW32iwJvaJ78N7kbcZdLaARyz3EFC9TdEeZgwjvnIieXZ48ux0pJTH4I8Y2HUX/nIafw2YxrwgkdyyjCoXtdu7fQuBrbEa1lHauje9UpbS0vcnRb+q2z/wQcJFfCSIK3ySEH04tJgrWGrHH8ehDsZiF3GbxzS9oFiZXGjsTKW7Ad7KasVN9XS8beX1t4hXw4xACFMrfUft2RcZHbgegLt9oLF1b32bJEGoMZOhKV8QF3iRMH3xP1LrKVMZeakIZneYR9sQ5fylOzmRt0eGkhv0G/GaciKNmVjUsRAzHIxx1FgD6pnxI4OyCLIeEP+ovoUqeyORHj5EkkqLX97nkL4aavNRy2qs1GJTlwENZTyzg9ewtaOO7R18eaM8B5o+JxrVPmR1zuXrM0vrJUJvaO8XCVuAnf1IyJ9W35efPRzZUgjll6DwiEhYZ28mRbsWh2TBiV3h64h9tXhhR936hQ5GZbzIEyDzQFgqsbUeijkJk0iL+Ikyk/BJA3QFyLW3hIJRp495/dCxemZflGkA8QHx0P9PiBqCRdeUfRmTiSuaxlETbKvyADunfcVbwd8R4xoiGTVoAiG0u2D5hnbF7HCv896mOv0YU1su4YkAKHSacXjUiMcnhrRe23nXhbk4Vu1Pju9Tolaq3QRHH8H3hDtmkXP1ez56685xv312O/sdsHhlar2keVZVKx7ZkMWm68+KeeDMbDg8CaLuRpahdfhe7m2+BCeyYk3HEM2nZXP4sVIAMNaWhbrrlAM0foJvikSCb1sNHLV6+PCyBnTBjJP38kaI+KZCtMq1t85+qIRWl9pToe0hQI4thMx8lca9ZmpkveIYwVwW74jJCXafxHpSRd1cVwf+PVQ7BYaeq689W9fe7fpO+l26QtOgVoqOb0vxpFthWj60uta5QYxseofn8KtpzfeVcKGireJYRt3gN/t9rKgpolHkH/QJXwK1efXn1Wp0/AL0518n+uqsK9AD+H0fmL0UuGwuZp9BY6g/df1lIBOui8f5upOSuSVoVJr/1uz7/4n9N9n3X/tfYZIk0ahRIwXScJShF5+Eif/urUc4qr5JUHEFYkdxtOgxj+MRG9OmT4M2AEmt54XsWPbVwvQA8DUpmTFRtz/i63BR32m4f56yFpK5mMTCrXTWQaXhEgW6hvJUuy0xbKiGrNPwkPUO7Jo72EO1tRz8cw29o/7k9WD4LroGY5lSDkKv8kFrC2NyPjyeMVjZQUALaipEsu+x0kI2Sy4mR8dlYlO1pQWd9RCvgVZawKaUCrve7E1UN+q4QF7sKYed6SnDiJRlThbBd1tP/FsU+4fAFKAnop6N5+b0AY0OhwTDVKr/LBn3ziKeCU/i7aa/MzsQXgndp/zh6efZFfctISr44dp0TKGPKNtVRn6thD+q4IAJZI1O0RyU8zu7Y4WWvv0OzL652y/xxYW5bNGOxFCgTLgsvnSJ+/Pgrxp4L7+ru0YEoLsyl++6CBnPqemtOR99WiRec7fBgbH4mQ/T3wAVDsiwotxw4yG7FfMZBdYNynuqvEF8wavUmMN5ausFcp0D8bahAZOY0elJKnO/Z4r1zkHUv40Y1si88957ROry2BufzURfIWHqvRGrTXiU7yvFuPmbU9B5JpOr0jBkfkq8QXiVkpcTs2zJEuaPeJmW++D5HFhVVaWodbRszQU6XIvl4HbIWQbLFiUqkoGpFf344PCP3CxrK2RhGt3nZm6pjeRWNabWJmq3WLSJ0OSpejm7BnW67CZRc8OV4L8ii+f+QSnstyjf+KrYsUwvgBy7kH2T6mrcuEzjKqhsdsKhzHFY/Tsp2s26aA6a4PtSX2b8mQ0RngFHiWFhtxnsA32DbsL+MfW13CRHLQ8HldJDL+od6SVl1szHUUwj1/C/kLIO/6LvFO11gVAn9oYBG0s5yyOKGG2E48f2ExykZLRmxU2lXxY87AfpiU7CVR4bW0lSoOcfD4DumluK45P0F5keCLW1/6x2Z0V1DSuKfPmtEu7Pg1T1AMVv0nV3s60G7ECuTQa1h+Rq5h/4Vr3Ojcj38I9/h/GGFEhf5W63VvBi44GYsh9n+p5fyXYOVfRNu3dYUuZKwDmViE61+TZjIy/QRAO3EmG0pEwQnFF/zg8FjT2HCCquQtbvsHck7XwP45BNaBx6uhi/gBy3HKIU0Y/DVUaiVRCmApWXdGl102Xcs/dTJvnBR9E3GrBA6p7zi0FwV4SX5Btwpew0h0wgp8IVm6jZUSd5OXLEVl5/8hXaPzyO78Y8RKTDzcYVUnUeG3Sv77nSUcDNCIHUbCA3fWIGS1qKZOe6KLhLvUT8veKaYN4Drx87xCOH32Pc/uXYAzsrDq871/1+8FD755Dw0MC3W4iovEA7HUzwg5Z+mcpjZeqDBDut2TDoQL2MIoDJ7s9f6Y9yo6TTHRHrAI38rzCv1zh0JcpA05rKDxlffZwnC2BgqldNUkmNyanGpCq/43ghqzBJvuyphXfyI3DovVhyFxaQGq/il0p49sBKVAFJQr4soj+SJFNqNVDpgIf8IThtuqKmmuf3va7CB5LdDE+pOp3koNN0Tv+FvpfP0dy/i2Cs9N0EbRfTqOItvuw+k+7Xd9Ei670GYxIgWeslar2ZQuAx/2lykDxqzdTZ0srZPFMIVrPjH80FtaZyrhZ150ze4Ab3iF8yT+b68WeNQJx7z2/2Tl+x/uxH7jHwvCm/FCyxUzCX9Wd05FmebPFQfWC3zq5GjmNmIXS9DX/kiiDDhjX/QD7LCgHHm9C2ZqYI1vg1AcCn9hh3JX1TD2QRjIivYNRNsNcSKe/1uGavTm1VrGl8lJd8YklPgDn+Pyrb27/Le5cOEqWCB/xAZ/IIKKoNENAcM6FY7TqCDXn4XVPKgGq1fmAJx6+mFY3KlSoYvqpcPmj1Dff7iXWu/v3K2YakC6r//tfWmDlYq2Q9+5vP0tc8kCHnbxIfcAm1NQ+6fgUh3SD9W7roz+Ff0xarOZh3bn7gRvADaPzItA8j26JnkA9EyMowp7ryGsN8oNgOR03gVPt5XHQCdFhGptktMa2QN5Qg1v8qI5t8QnHWrzxs/vuJ6PVr1vBUejcWuwJoJoeS2RdT8QMbo+CCRciySUHt3I0qHZFxT2G7PYfZRTAm4zG3OgeA086Yis087g8Ptp3MoB5ekpc12fQyyOyogWlZ4dg1bmaCJEGp8TiFAdvYUwNZTqX092HDLjpmCP94QB3G0ORKIAR14FDxC5zy/51wNQzwuyISgVU3QRcKLeeRYw5FBgwqEziV/ohGAxpbIAAty5SyWDhsGNK/ZJAPrK6ELbnK5ChAjFoiQQ2yQ98geFRsTuTg7XGU1jacQSQJ9mVM4nz+APx1RWjwWB/VRv60Ps9vlXDRrOLC0YvCh7+0WNTzPTOLN9rGkBbxPs07DOFhoy9c+wQ2xMGv/rAhHhkVOXbYWNgSu0+y8uTRQ5heAKsrYEFeCA5thLK99Rv8WObDmB7wnabh3OltM4Eeej2a/v2ZE9CcCbpoZvs2xzBoHhlXDzBzhpAPV6mg1uaP1a7j1T6jCSxZ5TkibNbm88mxr9mRNp2vrq4T+5I6C+/DxuqhbG98N3rjDXQKUJ1cn4yqcsLTAeCDcn2l5CTTAy0kp/7M6YxHkbHCienwZ3e0zkI+vDaZj8rETzVqP0UwM0G3lY6BQqWkHrjU9SsYcQ16/8ZrZwoYfMJJz1MlpGpeFEywY4+Kmlj6CKok8f1HqiHQcc4NePFLZk3xx2yuBpXdSNfrfyrXhIE7WRMgksDNNDCr9XsgqQQzJmIAHH+St4POM9M0jcEGH+7x6SnKAOBmlcoO4Qt19omHKx/UK3R0dD7N0rvd/ktF4tsiweTXGJrPovnNEG5YYX3awyy6lNGAOa8niKbZC2mc9xLhpt+FtKjTDgfGilpzQNvInUzv9DTU3FYcays6TrT/Fcb7QtdAJWCKnO28ojlGjBpOljam0tpIyMqnrYTio7x96hQLUodRqxPHtbo0G3Z0hsrrSKZs8JCqu1Z1XNG1LIPeGoV/bRv0Vtf8c3WZGBegs+F+Ot74nbd3HWV1sZf0beNpDM+OweyEr4Z0xZnhIS3psGPI30kTbSE6WxhT28+mUdEs5fHHn+R823WCqeSUkSouiZq7TjsOSUdqTQAFdbXP1DrRViOuYWrsAjY1P0jn3HKW5S5R9tv0ecZfFQyfLzssIaJ0mfh78lTovho1Vfi53uVZLZ+ES4sUhy/RL6JPFuQF/c7Zai+58+ytbAk4gK8MYelw1UsxJ7vl2/VKFa8EQx+z8Kvz8vJY9e1CvvzpecIejeStlR+iDm7Cap0SxP2vbLXBAC2TKHVATwM01Xs9C40/hVSSH7SeV278wVsHNrqTGSlP8PyO06S5pvnlyaeJrXAx6xs/jjTJwSGLe14ZYlATcH2GqPtYdLg+UVB/Ktz/liSQPeQuvy33hWR3XKfSWsaFuBlY1WKBrfMxAWg5j5KgaTQqnEajwqnEXNqHbfgt8T3/1QOOPw6FB5lSOIUBd+t4Z0SMYj0JlC7xSUwhvfTwart36Vqi3K/7X3iZ4y7XfGrUNRrVfKxor7VXUWE4R6X+Mg+1nYd6SyLYqsCvMbm1reqTffGmESKBbquBwkOUhTzC1tSncEpWV2khj4uKuYfD1k8JzhRgL9mhxxHSAwbsEDKdthpUshsQJMuI+uebmsDZl3A2n8fNWn+2V4va2Vq7h4S21/h1Svud9GDBvOPKexA1CHudbHUlZFxY/p/LbljtzH3xRebOmcOeTc05/FcMJ/Y259DBheSpXOO5U5TM+fXyPB7bdAN/XRFam8daovHliq0Pl10gCtmb2RwznD3VbhCyYj9lN8HGZEaqL+JAfFOppjHK4wv2M9VfgGSd3qAVUxHzWz7O9ACoSIbJCQ8rDu0Y+jMPhIprPWuGir5bBHCk/DKq7N8wqCuxqYXPd956Rcgrl1+CwkPIMtjkSqz++2id+DEa03VF3z6yA50k6tJW2oOUjn7ebvTXXiFMbWVKPmzJH+tuczqh5Ax7zaF8WwFtI/YyPGo21GaTl5fHxx8tpKw2m33Agw0f2x3tCUB3AWwOJXAw3CwAe1PWT+HV3a/eGaToYf+t2SfsTjmM/0n7Pzrs/7X/bSbLMiEhIcgeM8UZ2xt869pHt9AhHNXmLwqN9oAWikmlu/Y52O+WQjAcG8bBpHwaa+DTcPCvUqY7ihOf5fECgQp+4UKRqOFQZ7V5tMj4lGFGUathYMSfDa73y9rWLCyB3Jp/FtCuri5nQpOVvBECkwJs6ExX3D8ov8p4XTxPB1fT5fp2OlpeUHZgt5DgbI3eIpJOalkWCMLtXeCMcIp/qBABmQvxNCgiXbdgzg2CWfEHlJOypZSggynMCXZwzyboJf/nItAfAr30ehzdezDLtzlTfP14IyiS8DGvkpmTy6vvvUcflYpBOmUycJBOq5CMy3R054wzlwE+MNA3TXmSkG5sr26BxQkOydowCKYPZXwe/FgpJjtZciqaq8MG8mQB9M+GZ07Na5DsUzuMtCt5kmHxm9BWKeuQ1DlkJ83wY1lzUUjcZc6Eh1l+dQoAZm0O1rogfaOx0GwWMXnP822EQBb1zAICmin6nqX+ld+j4GoC9NG8rGgjuBNpjY+y8+Yj3C5vhUOlDKABBKmuEe2Xik7K+LfvXx1iuA8wCBTPoZ+sp48sMe+hCMHocdoJlB3oZRjWqycGtdKB0rrkMM6aYfrWS1SETnE3xoygfGAe30gH6aaHDsHuQPmyJUtY/MorHHA42fsvah3Nn/cq5xdmUbpdJANXV+QpkoF7jm9gX8YkSmqjseELvX+FlCcBkCUbIYYsfLVCrkjxXqev4m5VX8KNtwCXo2KthKNTREFvawWzQm8wwghLyuCQrZvinuveARlRl0/x+jkd3FVzmLG+cDNyGbmBf+ChGOI6ToRCTTgpt0aLwGtdmywx4OhM+mZBkgbI3gAWcQ8qZzXL4zKZpotjVC68U/acot/DIR/S3rX3754J2dEe79DtdawKn8F9vqA1XqJ75OtC+7/OdMFMzInl83JwYmvwTZl9U9hXK1BtRTYJPJO6dhMd5Yp6eZehR1/kdvBexfGnzPPpvXcpajn8H6FQ/f2CmJMXxlYXRkHllTA5FTCLd0shzQq9bvlDYzfCk+uf0Mn6AVdj53Iz9HMsTh+l9HLJKaaUfMgAg8xJk4YqlJvRguoC1lQ8wUdhcLLVEUWCyRo2nLG/lbA9tzvvl8I1qYPiWFmG9je/B2CWMRhV+ucw5KQI7ORsJkwjNq02ycbK4iwwRMNffUTCxmHhRLudnIuHgiRoVK2UVJUMEVjyJzOmZgITggrdAS4ASykjSv7gIUMgC4JheOwevK1L/hwuxEFU8f1IstIrTwo6y5x2b3OveSTDg15XJEdlGV5s/xorjE15Wm5PuMOD5WYpp5v8FHeHn+WlIFjTyIt5HDeejXlTAXipGM44p4i/Z6wR/9/xI3IsIexJeJDfop5B1voqDtdfXUJmAjwVABOafobkWQ/QWkGPG68z2R+Cz3Zi5WllTQFJgukJO3kzGAoMBxqMh+e7rpLMIkCVt7P+nie1ep35fUfw6YlPsYYqGa+V5OGslz/zcq1jhvNs7gtcswgWeTPHH8p2u4UIZyrRKgCp4TomazDbjWTYYK/JhqTWQ/v3YOBuJFli7OkneKUYmmqgNuRehSzvg/H38ooLzPtJqR+0djPEHF2+ZcIf2ZT6nEEfvIV+pbFw+T3BKGv5EgUBT/LduXfpHfczX41MQircr7isFboVrImE5WEwOeCs8prLLnKvNo/xvrCi+3PE+SuZVwAmVw3dctn4j+YCgyGAHy8sZNlRkVhXF+0SAA0AfTiOwjOQupyerZ4l0W+K4ngp7j5uZTxLoD2AU42gRZUHOju8N+b233KrrA1JQWfoHL62QT1mWZIxOeG4GQrNIeL6Kyr+kZ9ntVaJZ9zrFyHPDASWfs9zXR/FqC0T1ykB2gARhDUmsN3hTtbXvx95u4TsoMqHVZe/YGN+a1ZVwBmr0p8AcKpMtNXBD5HgW+Ux3ztsUJvPcfMSDvv+Lv7kJePZSq/lmawNTMn5nuc7TxGMGpfpVNWMDi6gTR1eqctX0PFjaD4b2VHDK+GlDDfWXbfyxU5vfZu39/6FNmQrK4a1RX9bBIS5+whEDGSUcR59Lp9l4LliyipnieSoh8kyBKvN/BkDPaTdyrbUz9kSAyEqUZ9RljxAd4YofrXH8FvtXkAwjBtTrOj3UOZ9PPhHLhW1zn/0bMvLy5HR4CPBQ34QJyuZGkbrDQb5QI5N7C/wUdYfFoAEB5UOKHYEKNHdspofg+9nfy0ss34NN5SMfpo9zzuZecgXN9Mx7bcGShVf953EH1EwLg+2WZX+m10OpMIuE6uGKxYwJ85yA6bCe/FX8RLSrbC/FtqnPghNZsDGZJH8OjyZK33m851PZ650+5bw4qWKvg3X5nIpsZZ2Ohil86uvrQmA2gfLyHQ+v/wiv1yZyeaznzYY1+bZT3G0ESwJs/FIo76KthsVfXn30C+klXakXeBPcL4hA9bhVDFt003Oaz3YjCo9J+wPsatWyEf6OFxPuc1CMbcaoikwN+FKozlc8dvBLVsz8azqkq+mPK53+ZS3QyA1egFmh1Kms84OmuDdwsAGyT5Hsxf4saaWnj1hr935txQ2jlitfP/jjwwZe5ndi5vy2fPDuHfoY0Qe7wEXFwDiOetUNahlC1+dXkZ1oAfwRJKwyOHkalO5HDuHqz5KSTci7+KP2rGkWqHz+QFcN7ztbpNVoPHnsCWQoyZYHg7+KIOVNHmGBTfPkFbeBJuqHIcuqv4d0lBJcFUvtNZQOtUMIMh8TPEefFuWx5hrgum3KGkTHJsm2B7+AgxRFzSstgZxRfeeYLZUpoq5xlbN/NBT+Mvwfin8FVAMAc3r+6730yUbP9wbQehNJbsq3EckpRpLAdybuA5V9VXRkL4KbnxBIj+QGHSStNJWHLMugcA29WP909hgdsSKuSVIug5nXqxndt2WH+Dni/M9h9+FlBIvZh2jqdqup9wWBy5ZY27/BmskWiQsILRiII3z5nE5+Ddo954AhnZfDU1nAnAgYyLzdu1R1OIF+K3JUkbnm1kVAVNjvYBelTforcqi1BzA02ee40rFKJFEBAjpRl5tK3aFL3BftzFR1AFLfBhrj01UWwLdY+vlNwJUGM5zpEl/ziY8IuagtG9EfAa4y3iEkT5adFVdseHX4FhZEjXJ7E6webJKnHbYP4pWLnDYV6c/IjP8Q8WxDmMCZ2r8iFTBzMYbkW4Jv5uEB6gllvGX7+ZrV8xIdW4mrI+F9Y2g4CD7y+9icSkU2uGYfaeiXymwBXnXPyQpdzYt/TNRWVKFb3h1GejCaH6jKeP8xPh9mvqWm4nmMr3OBXiJhA7OFcobVhvJcRgpdQg5a7vko2g2hfXmsMsFeakIPrENrC+hcnP1Ij6ousbPpjw+qLqG+loGu8zWv63W07JLFGEqaJ4Bs3OeVvzG2flLhmcI/+Yyv+GQLPXfYN28mpLzOi0qn+HtY5+QG/ySotFX7Y5LHClNpDrpTWjzFgS1VyT7fGubEyK5gRKiLIKH3KVTCYQ2Fq0lKd7ts91dPQlOiW+BZjOpCHmIthlf0Tbja3S2MHE5jcZA+yUQMZBrAb3YXPklBgmMKotijTRKWTwSXEFTLfyVPZBsP2UyWSNBoEPM449fvZuLQb+7G3O2EZ47kfS27djXqiXbamw4m80WvrjdhL+uCJUr2eeUbEKyd2dv0AZSGTQOJ07KfU6Tmwg/hl919xvWnVSmczpMvPeyQ+/2/279CL8YaRfuTiCrJCtcXSLmxoyfcTYaT7/T93PGLGpnx5e/5u7bVktg9jp66EFl96FT0HUaVSyCkpOQ+TtEDESuS7Segl7Sf1Y4OAw4HQ5OLVvGexVX+cWcw5Kqq8iHFxHf43mW7QqDqjRkGaoswZSbwlh2d2fiK5SsaI0LAD83CC40O+tmatfdpwsQvzkanvfzkHCvvFGvkDQtH8pNITgkJTuPtG/4OgL6GeABfwsqzxrpThuNfa/SytKWM6UpFJibKw7tG7mMWTHuda8+MXvrJ1RHxhHik0VKjhjjYvsxIZN98ik4IGqhlhlPkdeqH4u6PYd/mXJPtFz9Jj9GQrfKMcxweAEOCg+iu7GIYE2dVI/XJnF7Bz4wfAUIgNFn6UdZtnIH8bGxnF2+iDC7hRr+WTw6yg4aW6ni772LV/H9vd+TVprGsexjiu+nwnCOz7Kn8e5B9/PwTu79X63Zd6ccxv/o+f+fnPW/9l/7h2a327l69Sp2uzvIesk+i7Wuee/rcuC+EkiYLGQZqm/TJMC9+feTbkLWetjSCsqv4AwfQJShmGS1mv5ZUBGsZHJYwnqxoVosgBpHiHLD7ZvAqeYfsKpC6M4/HN9wczogTGj2/x2JljrLB3x9A1hy8Tl6ZoI2FcqjPZBtsoZyRwAZPgfplfAtw4K85DMOjuP9LoE4cfBGMLSxrxWIppIToqD4JCcP5sPBWphfjAgqe5ix9jYDDDDeF0aFX1FOymofahNf4s8aMF6ER70CYP/KHjOZKM/NpXO/y1z4yMD6+SncN+5VIiMjmfnCC5w4fZrKtj14WlYzVoKZarB3bMaJ06dFgklW80H1NBZV5zMyB0ZkPKY8QdJDLC0bQYUDvu3wESG5r97xOlaEizpfaocSpR+ZcA9pNzaSnPoL4RVDGgRH7kr6hsZBpxj3ayXVyUpnpHXgJga7/HKnV200qdE97Dr7DWq7H71DLqEt/1os+qdnga2Kosj5zC761+MWoY/BaIlnYTEcsyjRnmgD+Oj2X/zaOpKjXSWsNOzo57JTvLpnFyp14N+SCMoAgoBn1Foe8fXlo2iobjmdW1k5zFzpQoIFd6TjrSRWVcCTAdDNcF7RT+Dpafzgimc4JKtykVfpkQwRmJ3wUhAsbSQYmnl5ebw099/XtDwM6O12Djlhz79IBl468jLpaSLA5O1MaCy3WTW6Efe1EEjYwOrNQu43bxeYCgmQLqNTCWdPlhHBnH5bRUF6Ww0zQtLrn7M3IyeueBd/BfRgmj9YUsCv2DMZIzGwaj/jfSFaBX/eNYng7NcVxycXr8GeAr2DbmEI+U0ZBMOBXNQGO7A2tyfcb4dwwSSRdf7M37Odb/PEJslbhljtIdN23AwWQxN3oyGSy9YO5NnA3/cG42IXQK4HYEGlZ19tICUO+K7vVJLM7yv6DvEJIV7XhrVV0O22lmpNO3ejqYAXTduYZxBB2Sy5AItXMtou+xKYO5OWiSNZpf57Gg+rDQYG3zsJnCpi1aLeTJJDiShWezIzvF2bDsv4Xhab0SNSBs+WPAKJk93tvo2pafYBv/uu4nLKaKqcygSVPWMts4PKqXFAhV2DLLtfMkmlwWINwubUMqcIzknKGmGd7VN5JOk34gue4pkgJ3L656Kh/BKE9+NgjQCSlBtPcdpyCRwmwfxz2AQqHvisDFaUQbWusaJvp7mA8saPMztfx7PH8kEX4nHRJtS3VjOmZiIz/zrIt8d+aDCu1bIfVQ7o6p+Hj6ScKa6V9GD+nu2cyx/AmqKD0PoN9z1L0D32d8YHVrE8+Qzx5l0e562ho/o3ehugo0ZPl8Crin5JfIBdOUICeFUFZMguZl3ONiFPE9SO4SnPMtAAESqQvWqJSNpAMm3wVAF0OztcIamKxpeL8U/zYwU4cOD0qjcgyzA8KJOJfqJuEBcWKEAvQfpcvhyRzPiWb6GRqkTg7va6+mOtDh25VckcuD0Bh7GJou+ehj941B/6G2Ck/60GY62RtRhk6G0AP6cSXUttDtOYzsk42NL4GnqrVxC15TyGXu9HhQN3cjNrIxTsFzVyaoUcTA8DGIr/EIFZlzX23UlbF6HdKXmtUxLU6NI52KITZxIfxKRqJBixRceh8gZlvqP5IqcVN0K+psTuRFIpAy/7nR3YVg2T/KCnT6HymvN3M1t3mY46gSj2UTesxdVR/QOT/ECb2I3vdPoG7Xey1QYDbbpNUvxNc3O5YHm4LERKxqTNZmTMXgb4K6Vc69YHSXIQqAK10yuZJ8P+Fu24V/6IdqdnN/CTOtz+nOVhot5HiFmszQH+/v/Iz9NoAhqsU2VhT/LG3q31gVRJAnZ0h1PPQv5OJA8fo/7YvcPg/HxQ6TiY9zjHKsN5oQi2mZRzENlbMMYHcdYM9+SAOciDFV15Hf6IJEU/m7sDCyiyGLAlKOWmOTuXd+/qRZjxNn3jfxCMrrrDSSbkdC/mFQsJUU7PEnKbWevBXMi8iEJGGOFqPDwX9LOiW1Gfx87+Rk8xvxjK/HsI1syNr6DVK+y3rGp4z3VWlc5Ufy0Px+3nsXy4iBKIQ8JkHskX7HJTYwhwKP3l/Op8Mhwi6bkmEh5yuBOgkgQWu4EyUySSKuQfPduAgABejT3Pq8HwXSR0UCkD7bkx72FIgzw7xBfMUPpJ1gqSjmtZ1Hg7USqI05Qq6g8jyaQZmnHUBFfMdqrxQg8BfnIYY/2reaPFKnQevqEsQ6EpjDzXFsrblzE602ipdaIC9BI4deH1cph1x/vVive9vTxF7InqQJC3BJP07vAz/FDqRwaBir5lWYXFKfG4PyxqM98tV+i6J61fItGZ79Eyayk6mxdbEcjyG8LyMghU2/BVKZ+G53vROuAXuPSW4po/HtKWuT3HN/gtuBU0JgZUsfSuNlByRtTWPjgOwvuw/JZQddhcDd9UL4VG90KbBYo+BvtAViLoCr3ADNlb+SkSUjQgO3UNzp1dkY1DNvFkLcQlvclIne4/1ORTC4WNyEhkGcJ8bhOoz8eGUchfRw6qv8eusRv4ZFgbiqobYfZVMuOxHSSjTSe6p7zPw/EL7gBmEGtHQeBmsr2YBwy7yKsVzdhYJcoilKuUwCY0vmy1r6Rf7w6suzcOtS0PBu6Be3OwauN4tv08vnNOY66mI0lpvQWzqu68MuQG/wpArM9tRVkESs7QPuQXJes5vI/wiZs8DUWHGeeXiY9rjBVsgpocxoa86gIc2MmqaI5T7zGn5+1imL6cESedqI8W8MDvBTiCu8MaSdREBS46xjNZ+y7fNO7KZccL4J9Sf81Hs0ZzsjwBld3ICYsWBh8RUodAlmoiv152A+3Utdcgf48AZZVd4AF/M0Ey3G48iUr9l25GviGGq4aW7PZdzu1QkSQr8hkLLV4UN58wCSJE0ruwJo6LBf0UcuTgDkJPzvLnj3PK5DtNZnC9k5X0zGlU629gl2sgbhz03QJBbfHTFhPomh8m15zAp+dqMd63f8EZ3FnhX8t38LHyAzZT7L+XrFBRm4v+26D/XwD00D3HkJSPCAnZiVGlrPlJ4RGeDyzF7oTWtyEjxINVI6spbruU5/P3sLmTxCr9D1T7KdWN8uIeol9eCUEqeDThL5D1MCYfEieLtcbD/5GqPN5tWyWHKwfxfqkAVkZqKvC2YFMnLgZ/h/+tGg77zYD8fZDxM/gmUhQ6kws2AbR70fk+RPRTHJsip9FdD8ONkCgrGZhE9OXpmh4cM0FLLfh4FfT1ZHkdNsG6bcX1JVT+NCulFA9Yzf+2hAp4qCYtfps97f7kA5dSpTcQR3xDYu1pFLyAlND9bmZf6WlGN1tK99LH6VT4MWdvTafW11Vf0VQIebtp7iPAz51u/EHMuYuYUuZDq1fANwG1C2hpNKXQ79JlHtS5fTTB7HMn+7Y0KoaTz9T/OyFjJrOD3Nepd5aAtUxxfJ0F6AqQb60Ch8Uln1vJIZ+utNLCU4Uw+OSTirmiQBpA7MUUVlfAQs0aTumVrC66fMHbpy7T78I1tGkfYtJ5JIJKz9LSsoUwV39/VPhA+/eFrOSZF/mkfxidSh+lUeFUnJKNGtf7Qk0mkdmzeKbLNK7HvMGGajhkUoJNq51F2FXCf44vfAJNwWZRz1QXDIkPUWZxq0o5JTXccxuGXYSRN0TyVHCseaMYyowe4A9bJWEX5/FaSDeaZS+mfeRfxJe9IgCOHT+CqLtRSSoaVYwj4Jw/0xz/Pu63DFgMHAR22myKd3O3TcRqFv9QxrKsBcgyGDWlhBhy+OnCfEqMo90d1eTwVuAYFoVApQOyrDrxLdfZmTm8GzUbFUKyXCd7lBfQh0PrN9kd/A12oIXOgU72+p6bvcD8kvGM8YUVSefQUu5uM0Qy90QZ2w6c5f0/r3O4/BXFoetuf8EjqYKhmKwBw9UPoPQcxI+Hnj9TUhtdD/p8OvAwrAuBpKnQdpErka3itg0eyIObOiVz9ITUmX018GbCMWa18FpbU2bwus9kLslZfBMBQyM2udskCeInopfsfBsBuXb4fc9BFs9fwH67nR0mEzWAln8Wj9booNbZcM9Yxwa12q2K76dGd5NdpV/z0q6XGLh6IIXVhf9l9rnsTjmM/0n7Pzrs/7X/jWbykknyXNRtANogODkD9o+GM7OZ0dQtibfbvhE6fy7QYU4HziTB+uhh7U7OjTcweqAAATRq8WlEaewk+uwDS5m7UW2kMqgjGTZ4phCWpr6Ft92n/ot3QiCsrZpVmr8vq9Bv2CRuFfXhsAmsoAxk+iWzrOYd3mcPDyVuYlD0QmUHEQM4WjwWszaHJwKgsX2nqGHTZyO0fRvMJUzxh2oHLCwBfJRBrJjMH9gVC8NzYNTpqcpJWe3DR6Yo/qoBX+s/Q4eUV4iF9MU/j/DB4TW0sL0Ov0ewbPF8OnfogP+5w3zisPG7Ez60gfr0dTp36FCv+61RCcfQDtglXYNzjPZfiAQ005uQLF61kizlrAiDRDX8WubTQMZTp9bRqGYkkwNr6B+zQ7EwyTI81HYeI5t+hMnmi6RWIu5Gxz7Nm8HwQiD8mrBRFF13WV0/ThwsCIE2lYsFIjO4IxjjqA0eQdr5H+muh0W+yfXa2nV2I3YL064OYn4JHLG2U96T08HJ/C1M9oPCJIiVGzJ26r4NVejTrNL+5yBqJFBuMDBi4mt0HlBIYpcKouI/IDJS+aTrNnmtMmBZqdIRVlmKCVOJxFazXsloaj2SgdYqVNWXCZRhSSksuC1Q5f+p1lEeoqDwZv59QeFtDjM3rs2iY9hyfFQlcOIpyBTsGac2mA3XnuNyYS/X2DiEjj4StHiR9XIhmRUt3OOm0kL0UAhqC7pQ7r7Zn4UlsCVK5nGjkg2kK7/IXeGHibv+O3tvTULnl+IxWBJfRTzD7CLYGA3BGhNOnXI8a/QJ/FwpGMIru42D0rPk5eXx1sKF/PhFS/K2PknwCqg9lCNqF7pMVms5k3c3101+jPOFTrqzin5jrGeZrBOZV70EasnDwQjryXHbccqvfkm6Fcbe7KWQ9gNQoUYGnPp0cmovKNqaXF/ErbjzGFxSOQpnThsMnVbwV5mQCDNGfYfdelFxvFFdQHzABYZ2TmG/w/63UaiTx43m57hcXgwSyeYY6Zbidx0qP+TNYGikhlnBJhG4q7OgNhSo3ayQBjKKxkZkREykkc5EdWNoqt6haPbL28aCEJhXDAMvd0PymBxV9nJSgk/QI28+3a7tpn1gP8Wx0fbf6RB6jta3P+Htncdx9vwV8nZDu3eh19r6gKMMWKQtIqk2tkCwPHUhnKwKYX4JPF0IFUblBsDvxlwOd/yZvLDV1KJSLor6CBhXzndnlpFV0pOCKiXyG2CLYTyvl8DvLfYQp96saKu0hnMm725KamPIs/VUIOVlGSb/UcCMLZdZsG8jpQEeiVNdOJ/a9jCvCB7I9mfGXqVcNIBBDkBtF6hudZ2MTLOZonbkzn6MjvyTrTGQlwRyqbL+ibrZ87ye/ybnLJBbE4PkkXhFpSc7YgTHTNAm5DRBfko2tizDmMuD6JzpkrK+8LqS4S6pqLEGYLXrsUn+MPgoNJ9df+yvl1/m+e1nqLIENwje9tT/xNOB8EIQLIw+qmysSGWE3y7UgF8anNMpkd9og1hZ04YTJmih0SI7G26qnU4Hv0fD0QGuY/ffAzv7IknwRMQNZlU9z7hte8ntoxzvhdcqGe+aOlZElMIZD1ne0rP0id6FBkiTSvlR84mYC/7sCptSUEkWLsY9zVeqnbQsuIUUpkyo/MIoPi2HHice4UDFOeUFx47mZtN1LC6F8JtwpXZYg3t62PciTwZAv67B7LNa/vZc0LffNIY0/pxH2glJQGuz+cLPASi7xItNE5kcdZIJefBs7gRFH9KOjrw4qBWlciXJt+B6sMd4FB1Dd/ohmgadwwnk6PY3yBAEV1/jUX+oaQxT48Tae8+kSXyv/3vJyu80EtHxw4V/de5VuCUSYDZjK07lDsXqEP3IMlDmGtOWrxIin3XfQ90ldfoEUmYAcMu4lqxQwW5sML9l/MT7YVDigI3V4DQmuNu0wRREj+O4CdZVwb7SeFSyUgGBpCm8m34XX2l/ZsTOgyLYXH8tEk5XUDBIBq4tg5JTYC1H0gXz1NaLfHljDDk2qHQoWbq+lX/i2+IBylw+qSVkgEj0HX8cHDYy7PdQYTiHI+oTFrTUwlkPhrrKwElTGBfMsLICMmWv+S2sO6sq4JRZKFrY1R5AiPKrTMx8l0dd8fGXi2GHpqNifA3qCmL9rxAWO5bV/yARPXrSJCaHZtFH68OIHDjpvEvxmzpQymQ/ONXhd6RCD59C0lDi15+jZLAsVOL72K8B5bOQJZnzFuiWYeGKTytFG1U3iTWspm3sbwxO/gaNVO0+ToY5x99hegEsDoHu6r2KQ9vVPsyORjYybHD/n39iTfH4LvJ2cn/UCDrorYTI0MRwStQI674aWopnsvbKGwzI8OPBokpy9MrajFL7RXRMD+LbCpiXOkwEKevM6YSKVPw1OUzvNJ1p7Wc2GNe9UmveLoVH8uGLHGXJhQT/U0zvNJ3koFNsyV8Ow9z+iiRBiSmKSnMwAxO/JdSxz32g3cSjmkl8pO+ApKrBockVY+2bDAkPgCGmfu9RN+4ABLSGtoth8DGePpDLrCKXFL9a+W5TeZ2JfkJ6b1dCIRqLMsgfdmIKn4aB5FATE/0cc996iz4qFXd7lT3oJ9fV5Hu3vmauLMMTm2+w9Oj3Yp1qtxhiR9a3ZVc0ZcPV58mpatxgnbrHNoDlYaIm5MMhtwWoqM5u/cQs/w9opYVH/EFvVTKcSF/Fb8GnGOUryiI41EHKdnMJ5eoVFNrhTGUYksoAGj8wRCHJarrEbKJdxF6OZ99DXsyHCjWUSPURuri2Z91va0UN3zpL+5rJCRMwasuJMN6ka+0wkWgpvwLnXoH4+3m//CilpjBGl8wmyvabULUBsJbRxfdHOuvAKtsYd2kYVQke7JZLiwlNfxwAm0NLhTkMlcVjPzbJyZT8WrH/VFU3UH/56Ni3/HB5JnZVNTsr1kNoNxFwpmFy2Td7OewaIOIJ2Zv5PLKUBA08EX2JT5s+IWoUAoR1Z1P4wxw2CRBYtTYdvSNLAEb/pkVXnqOXHvYUtSa9wGvtlWQckgO50RKeaPkB8b6HwC8ZYoaBNpAFnZL4IU4AY52SXSSC/BqDrEF14yOMHsAd9R2YfcX+7j2pJCHkcY1C//DH6k94puY8T/QexOgAL7n8wgPMCy6qr0+m8O0kmYzgvmyuFiVVzJr8hlLrrm/0klnisU03hOzzlSWQ/h06uZyZsWfrgZv03SKkWic5IXpofdJydwy8Ee6lvHDqBdaNUxGttrnuSYLe66Dx4/BXL4oi53PM5bJ5A38B7jP9xlfBgfimwWfWxxq0q2QVAwxwMR6aaZTgkPhDwzlQl8uohDNr9/7tEioDNcq5ZLBe71ZNmvUCvxbfxZZqAVBrrs9U9CNl/cGzIbXIwNooGN3kM/e7X7CfR9vPIsJ4E4sLi1e/Dyw8ALsHMsgo/PparejX87vRqcWH7pDMDdpETTR3zCxM5VC896UtfmJxoTsWszf0qAD4A1xaTOOrzcgJWsvu1kmYGj+JfPwRuPYRFB2G0y/Qu3wHe13jKTlVyv2rrKH51X1YAZM2i1ovCW6n04kGAwZrLCGyBrXTIzHb9Dm+D9nKeVcOT0Ltfj/DerE3+wl0ta3JCvme4/4vUyLpRUwyaSr66qMkB4n96hMF8FG5R5wg9TMe0PSmqetCtbZwoWRx5T0IagfdvyOzyr0vlCRJfGuBLUFWIx2eyK2enzEjAN4sgXI/D5UyTSD038EN+1xqtbf5tNiPC7GnoGAvZG+Go48g7RpA96xfUNX4/tu4X12sZhP/PlazyWLlpXnzKSzM495mS/hsRHP2ZUyi3DjC4zloyXU0I9MGn5bDiBvtvOKVTiRXNOqxtM4cxqOkgj4cWs8nw2rhUX84m1xKkuEv5YUEtSFE9xknT27ltYM/NFjHPN9HgzLvSp65HedqAgFoogH9lUVQekaoTsRPwOwIqJ8DLlhixRyRMAmSH3G92ypKHbCmEqoMyYq+f1M/zIpSHX8UJfFnrhIYjj6UMk0ENqdYm1v5K8H29PyJLKklcWrwK0xi356GIPqBwLf8PfsCkNp2o72XkoIkgcal5mR1KEH9dd8zwO6bu6m2Vv+fTe7dybxzGP+T9t/H8F/7X2t3aUbxhQsEGq1CyBnd/hXyBaOv1u5GukkSkPIEjLgCgS1R+Scwetsefrn4GgkFz+CvU6Liws8+zY14GOln5smUfmIy97DejYQcyWkzXK1UojMAYiy7GeULbwx2st/+92UVxt43DR9LIj0yvuLRwiWoLUqGSZ0z+3BhDcsy5ik7afYcr+aKTXaH27Bb79KVjx0pkITbO/BtBIzy2pPWmW+zGbyf9gCB2bMxFj6AziuvVmkpx1eGRsZ/hg4J8BfMnvzqJIprY7HIYSzb4cPiN969IzrtTxdba/HLL7Ns6VLu0f7IQ34C9dZMW6A8QcZaBumnEaISAcX0mI+U7fYaZgTCNSu8f/hXJI2XbEjqZ7zdvRkzu01hTLP3GzD7Fh34g+/OvUNy0ClkS77i0N+zvuLD1DGoJTBIsmJnJ11cyId3d8Ag2Xm+EM4GviNqY/g1hvJLQqoJB30NMC8qTVkTkjpnw1U90XtHU3SEw0FHGeQDn5aBSRXfYNzjNDtoH/knjeIeY5/t779/g++ehtWhp9bmhxMZKWczZLkQRKYinvdrRGutQKVXOpXfjGPALu7OgaZaWBqbjW+1B2qz8CDGPS151jyZq/mDKcwVLsd/qnW0kr9fULiPSuLQhSPoVeWQ+ikUieL0aAJZefpDjmcLRFul/ygxD0QKdoP3M1eYrCbD5k+hHVrJITTSeG3kOn7Eg1ttHM0azZIjP+II6qJoHtdzAYHWB+joihVa4pXyQaVB/bnfJTP78+0HWbZyK/GxsRxctIi3i6+ytqaKjwuh47GbxMfGsOx9UQtCpYLUqLfID9rI2yEw2keZqGlV/T1fRFWjlaC2McRnPKtol2VAslPrhAum0PqNuLjIMm4knuP1EGiaAfvle5X3HNKZG84R9CGJl/1CMMoe767GV8h75XwJwJk4iC9TqsS30q7gk2FtyKkdSvv2i/4Gol3LO++9R3hoAMP8TEQWPcD4P25RE6xkHMaa9jDSKJD0iyKqodgj4eJ0oncF7pI0MMywVwSKPOzRrRPJsAq2WSVxirayZi/TJkP8t+RVz1RdfoSld3ehebN5vNvtWdpZv1McuzOgiH55RWztoOciuUj+KXB2rki6lJwmSiNmUz8ZVkZ+LIJXddb+fcZccaOXvaVL7WF380EpxKqBoDeUgTtJJrUin7Mx8yiIfR2jrhpvy6pO56oFni2AIpS1ktzTjhOVZFEwTCQJLFItu5Pu5WPfpVRrPZLcsgqzHE61EyyaAkwq5bzJpcU8kTKQ4WeLuRLUlWGSiynR9DmXDKmTc7V6ZhTA8jLAEOV1XaI6XJgKYrRVDZCDKllFiAr2N4KRjZcr2mQZquw6KhywowYYdh4i3cH4Cks4z+84zR9XZ4OkgdCuIgiGe25oGnKE1fdGoM9WSqD9avqICbmwoBhmZvdTXlTZeR4I/J72Ovc9KEwbwM/mttyTC70Pvo7VqJRRpOAAY2M2c94MWwuVEo2yDA+3fZlxYWmYyvoqJDzFudz/3c9ohmL316ZK/5wFXZ/D33VvNqdZUT82pmQBBxpnopUa9gWQ5NeCqJJxaCu7NpCfwhiHJmIMZY6662yo3/JK+T1My4dX2+7nrQf8//NcoNPxznvvERQUSdeYjQxLcakqBHUQgUoAWU2R3c5t/51k2iDX5hWQNiZwXeUhw+PBgqY6A3Xm98RrBGq3q0+pUhoXSO+xgX5n7mN9XltuVwgAwbTHHmOP9e+ts/vsTpokTRBMvUtvC8T5HUySgEGHRLI57Sv6cK+yDaDxY6LmpN3MynaP834owsfTewGAWr9B19sSVtcnLHs+SEMk+c1fY2uNAJ2MjbiKpmCT4vBbvm15xbmDzJCfKFcVgexOgmhUZnr6VpCsgds24J4MGFsEo9JA40d2VUvyrToGZMPqytGKfiNuTuHZJr+47wtJJPwBzr1MseMm+1u243TcMxyujgZPMI0hijklg/iusm5MGr5fnYOGsL3Ch6VH1mDVJnq0ONE4LGgkwcwqtENVqDsgLsvQMWo7nw1vwaRe/uz9B4noR6dN46UD13korQ1bqqHAay0xWtO42wesTsitikfykO5GbeBy4qd8WFXJhkotx6pexluuaV7O2/wY6TFenpa1kQdCH+YD9a+0P/QCZ1Pd67oYHvGuPxcIXTRK1lamdhofF4S7RserJnNtLok+OwlVOelugJei50DuDpHIafs2THLyU9rTXHQKySdvpQFJArM2lxNmeIetQp6xzpwO2NyEMb27kdxoFU0jvAJzQPco4asZTU0a+GhhhnSGpXxOlG8alfZYCGihOO8be7fz+alPeL7bVBJsnrLOEjqpllBLMquqbLRLC4fgDnD9Eyg5DVXpDIv4ABVC7muI/ksBDMlcJ/z1nK3YZR/21wopflPwQOWFNX2WMdnvcsUCcfoqJGyKZpWpgHA1TA+00T/5d56fOZOMrCz6vPIK88NF2YN3w30obTmPjKys+kQfNAyOe5osQ1ppBzanPs1LPccRVKDcE52Rn+XnSni2EDpfHQSeIMbaXBLUN4lRwzcR0NLpley7MJ9g2cpwH1A5NBh9vEAB6au4kiBKG0xI7SwY5msk2DUQCStTN9zmxb8Oc6WoJ6XhzynYaHf5TOa7gER0lijuLvAqjZH8CD9l/kKN1R+DpoJg2z6hjHDjS7i8GKozCPfpyl3n8nnI30nbinGCYQTg14QlBZd5x6VGVup7SPkOtXmToiZrKPDfQXXIWqIDj+P0bwZDz4q5rCabZupc4tQw0ggj1e2h4KDiOThlwcjTSSqhzuEQ4Lqm9nd5965e9cohpogJ0GmFuO+4cUzMjiLNCvtqYU1RbxF8d1kdCKzC5xynGo+hY25TOOLypTc1hUP3AzC2+btsul+CYiWwqePtT3g7lHqWicJqczFU7KOdDuYEQ6zxkqL5SME0tlfUaS/bIfVzUYfQWoHq6mKCdG5pN+96WrIsFJEUVpNTD4bdVJPHCWc+n5bDT5VerNPEh7k7J4Ucm6gJGewJGPUYE4DdLQ6TnDFI0a7P287bIaJcR15VsgA7X3kP0lailSqYFXuxPtnn/d3cH/Y1f8YIn3N7VVtlY2BLDuf3xKwuZbQRgqzXBSjUGA8BLZFcwW+dBOlJFW5ZSZdVJ7/F0uzmrvHxmsAqb/CgNhWdBJ+mDadKUo6JJaAtZ1wuyNRUGKBy/u0SKsXNRQmVCbpoXvRrTt9XXnHPJbKGyyV/cTBjEj9EwqNBG5WdpH3NgnAB1L4vFzZc92Chx41n3p7D7NLvYG9cT34Y60ujPNfeNqgddP6UYk0zVHYjdrmKUU0/JGibJOaC1M/RuhQiHLK4MW+QteRRs6/vrabQ88f6f1vD7+FCmXuO13v6u9UZSE4zp5MnUqO7yWHNMZx9Ngup37aLIKwnjUxpvFECQ33g3ojzinNrqKK5MZ8oRxAfhUGvyrnKZ5G1GVVTH7Z1MPLZ8Oa0Kh7ublT74FAH1Rd4mN3ouKitCRA/gTWpn1NlCa7/HlWSCsrOQ/o35DTdzswdbuUBpS8j4ZRUaBwidmVTlWFpOh9G3gB9ZP2Y1ZlOroCKVPG/oqP1bGGdBGHlQ5XvvUoLUYO5ZLGSHrmEC77bqdV3gLv2i5JI1nJwWHA6QaP59+oV/yRW00+r5ac1K7lc1It1V16kyhKkvC59KGssX/KZi3AXZuqp7KT9+7xd8iVWRA32BnGbjLU8UvkkTwYINmOJvTneppX05JT04HTWeDFP1JndQtuQ9TQOPsmUdnNoE6BUo1DJdjpkijX1QC2YBh2DGDdbUpYhLUqoSP1YKUGXL+rXWMGydL/b3kCJcQkz6JK6nd8qdbxeVq5ow1pFqCRiZb434Psbn+FtV8Mu8uaOCww+H8MAtV3xLJzAeOAA/5r1W2fHEIpajw39g77x/er/7pAsfBfrw9hfxHttc9gU30/d91x/yV7MP/i/y+z7f23/Hfb/2v9a85GyiXBNJNMDgW3thSY4QPQwfsxwB1Fi5O0KqTrp/DxGd56As81gdowPQZ/puQkEhyGaVKuYzN/MbQN+HpJdVemo1mpYFC4mXu+NLcBn6oW0zYDPnXYeHf/g3w5ihYdH8taAARy66zFW9piFscjt5GAppb/2N/oa4JwFNttONOirDp2Wawer2qNGRNpKqBbRaq0E+2IkqExTHKuPGcbZqz/Qr/xRWqj8lYuvrZZHKlYz1R/KE5vzjefi+G/sK72egOgoTuxtztGdUZze14Q319zmpa8z2WSx/HsEkNnMS3Pm0LRqA6N84eMweC/id+UPS06Rovke3zomm8rrWejC6ZrTjpeLABwNFppam5kSVTqTcyXeP6es2adSwdWiHjicKj4c0glDnlIGL7V6CMcqYnivFB7MmKEMStur0esKkNUmDpqg1Ke363pPQ8kZotInsbbHy3xbAYOvd29Qb+FWxjP0S/qGnTFwr36fog1dOFstEXxbIaQoKmQvmU+gj/F5Hm3/As/0WMycB7r+7fcvNDSSIH0uyUGn0Klr4ew8OOOSkq3J4NWov5ggtaGxBkJVSrkRtawmpnQCZ8zQPwtqQzwQZH6NsTV7jfLbz9M1dQcap8g4/6daRxv4+wWFp1idXLh6lipnHIwpgFYCvftvCwhX3aQRvxKoz3e3lZ6DtT6iJoPTQYTKTKAMIw4v4IROKdVkclhJ9/+RzJBVdzxXoOQgSg5DSoWkAzMbOD5JgSJ4ecgECzcEsPjN99lvt7PdpEyA77PBfruDxa++wbKlS5HtlZzvM5+vwuGt8y9wxqFMMBVFzuf9I9/hdAr9dpO/RyKn/CpdNTNpE3ADGTDI1AcnAJC1HKwN4JLLb2uwOW3yFKc1m5hQ9QQLY7LwkZSAhKVHlrI/9CEAFpfCCaeSfZDv7M+P59+ktDaSpi3nMW/Rojsi2vur6xDti8Xm1KcRo9fa+P7ED9Sa4nHKyoDCpcbHeezA55Snz+fpA79B7Bh346EJvGgSyPdWWnjA+Idgn9RZ2QW+NZylmVawB4qk9oq+ncZ4LlhglBGeCa5RPGenX1M+SBvKLvVxOgVdxWhX1kKVVGrsqmocshmj8RpYK0SANHY07B1KF6MIzFuc8GtlX7GBy1gr5LWcTr5rso/fouDXSAiqVUqXOhpN5MUimB0EXzX5xI1mB3DYKMvZjT3mQ470WsAzPe7B2zo6rjDRD5aXQzliHqljlp7aFce1gxIZx32pPKQjb+No9z1J0CTkGLGhuyn230uZ1QOE4bATJqcRrYIOOpAbjVWetPoWMYYzgmGLRH0w+/iTcPFNCOrAUzlxfF0hApKSX5Ly+IIDTAjYz29RcKDjT8pvym6iz/nHeDVYJDAP5SmR45IEPWtG0FwLNkuMQGNqA+vved8Wd0H59b+/SV5Wev23IcvQMmw/Q1M+p6Q2WigJeFiZ1ITrVjhhhoNVygA/4X15r/gdjpmElFO03BChXwfkaVAwHiBrAx+GwRfl8NJ1FwJ2xHUYfhlJgrk7D7Lh2kw+GNQVY45SVjzZuJP+TuG/PHb5Sxi416NxKm8dX0aVUwCmOtu+h9LT9Yh32VlDsMpOOy3MCgSqbin6flTezLuGNgywdyZIpUwe1I13IzWM84UgbW6D9h/LdnLdCrMrchn72Hf17JbBXnPBII1MH5n6Or6yDMuOrOaxTWn156k3/6b0yQzm2woIVUGwSllLS+qzjqdyA9BJMM1XS7jFg3ncaAzWMTX8XgXPBMCmxNT6Oql11i6yHf43P2b5oT84ViAC/JGRkbzz7rv/cZ0dptLRuOkSqp2dBGthxDXo+CEAwXnv8Ns4A3H+l9z3FNwe4sZDzChOSx/W99Xg/ZBkzA41ZieMM8p0N3jVD/NrzDmTH0N8oCJRi1+h0o+qQ+rW1zMqV/qWtVbRX6gKWgVcB4s72OujKmRTkws8FSDUFzDGueWEnU4CjJlotXmu61YukMWJH/N2kYZGatgfCwH53wtGfZ/1UJvD00bhIebZYVb2CKirH+MylaQiRg2p8dDXqUxQcvFt9kVcZdKVG8SU3K8cs4Dm/JHyLp+Xw4dhsCQUBeBPkuB2RQt+vvgqOn0k70x0MFKj+ls+VGRkJJW2SMrsLsa2NzO0+Fu2x8BRE8w4/CmEKMGCWhcoZW2VxKGqtxs4FarIAThVPrwaDMYqL4nkqME8mAcXLHA2fClms/umZRnGxq/nEX9oez2eqwalL5Ojn8Kmqy/RofB5xkaeQlXuAXBMfJAPMk2sc6Ry0QzvFXaFkC5w5GE49jiUniMl+UXa6WBVBETUetUIy9/DMwEi4NjAJBl78zn8ZMmkWaaJ8Sefa/CT5IKlHDIMZ17ZW6T4KJ/zhZKRPLCukKPZ92BUl7iTPLiHzumUeWPvFm7qX/QYSB3b1DdYfqVOstb1Y0MM+CZB5u8MDFlMctFUBqii6av7XtQDOv8qXP8YLr7JI02fZJDLDVF5gxkkmd4+T3Fqbw4Pr89GHahcx0r77+K+XHgjBO5pIWrtREZG8sqrr/Lsy5dZ+eJEDr+jo0+nh4ms3grl7mcty9Cz0a+0idiFr5wNu++ur6lbn4ByyljsBhS1iYEL6hf4uUrIo5XYDcoSFc1fYG7lN+ythWHZcEE1usGzMEmBnDCDramVSG8ZxtBuLCiGbBuC7Vu3T8zfjc5ZiBO5fpy9feVT1oWsv/Aug87nMCKgpD6hBkBwRy5XjcNiN3CrrC07g6uF/3T9Y8FED25PqOosGtnCnluTuRL8k5s1KKtBjkdV2wS9BLOjr2Mo8QDHhfVke63E8SZDCGs9kS+GdkVdeljMQ+degfWx/BB2kqcDxd5ZQ0V9fTtZhsmtX+XhBNFfX302/OJTD+DQO/OJ9E1D5UoG2oN7CzCc2gB+jdlV40+FA7bXwMoiD8Zr2SUGFv9ON9fwyQ4deX5PuWVzjfH1gf7siqYcvH1fA3/kUtx03iqB662P8nRvZX1hsjcTf20wFick3oRzFeMUzRsyl/BJmXhvftENxnn9UwGgBJyJj1BU464z6l2zT5LA19RC8Tf2DIbdYq2sC3D/UAkbqzsqf2eI4JrVFz9Z1ISMqFYqqTQ7NKQe4H291hezJ8gM0BXs4eVgwTAP8bkt2FwAkYMwy5EMODeE911Ll5S7A34LgTUy1OYTrC6miUbUZt/sDeBOfpSHMppgB/6IhsTq36DiOoR0hc6fEnNrIjMrX0QC8qxGoXDiYdboCWwrjWGgARpJXhLulalM1V7BKMNnN4dyvUAJ9Cprv4xnXVPa3mPwkJW/ZY+ZTJhLRAmVHoOyGTziMK/c4yQSt79TByp+oRDWVSgTp3RYwqCbUTgQjP+rxd3cvrZPNGnl3TkduZAS42HOmTRYtS6f1zcJUqYzpe1qOt1YT07wWq75/4FTEwxqP5B1BOoCSZS7YNbks7dlS/ZbPqw/rbeMpwqldLwsg9Ym/IuOt74hwbTWPVd0+ZxbbdwS4+U2A1LscFH3s+U8COqI3lHL1+UwMwheaaxUcwqWTvPx0PaMDzDRQgvBNiWLXH3zWz53vX+bc7sqJTHNxYTa0zC65rUmxhIodEuDyzKUGo+AS0pWW5MhyjU0eRqnR5L/vVCY5ushiZ/yJDtUF7noLAOgxPcQki5IgBDLzsPRR0gOOFL/80T9X7C5CfzVE/7sjtTpY9r++TWflcP6JseIKHWPNYDdYSfdIoCxRtmGylEhSgTsGQzt3oHBh3A6ITB6El/L/zru909iNQ/V1rLj9zWczh3Czxfn83LvMUSXvaP4jU6lQ3KoGa4J440gP7Aq401GlRizSVFnCbR7gDprsuGQKKP0Zon4X5nTaz468TSPGHw51jGQYt99yrXIXs3UxvcyoslyxjZ/n2SDEkj9cOIo1o9qSVTJeAJK70MV3F7M29c/hXXhJAe7k7ZWXH74yWdhS2tUKsHsa6kV0t+xxd8r+k4sfIL7Gq+kyH8nBapTijZSV/Ba2Qra6CA0/yn0DuXcB+Lb8Te14tSFdB70mickwAK8A4wEtgJvAZ2BJq7/f8v19+GAFonR0a8h293sVrtci1127y1sDpvCr7Z7MPsALHZLg5jX/9Waff+v7b/Jvv/a/wqTZZmkpCRF4Hmr4xSPXRX1fs4X9xGJvrjxQqO91y/crHUj9jvLM2HP3ZD6mUCY5e1mSmgBsgTbC1tgNSiDc+Y2bzM0By5a4IfSFFGYvc7UvtDoPi6Z1NxKgAXNpza43m35a7EiNvRdh6z+lwHtgRqdW1bhhRdQqeBiQV9O5gzlq9PLMAcOcHdqKWWo9muG+oAaaCAOmvY1c2N/QIOQrfTDI0gVNx6SpjL3+C/UmCJpqlELHXMvczphXu8xvNL7Xu8WGtkyiddAdZM27Hf8ZxT788BBkwnV4cMsqbrKz6Y83qtM5dzGpXSz2/82AmjqrtE8WQBfFvtzpNpLurTNW8wpX0emTQQUfWu86hjKKvqGvsrzpoeY32wNKknJIiuNHU+rTCs/Vjk5Y1YrFqY6zfqS2mi+O7sYW3AvxbGeNQAaIKzbvcP9B76noo7VIMlicKMGQ7MXcKgDKXBYKbTqqCh+QImuBXpYNrAwRATLw1Wlyr79U1hQ20IwUwC1quHqeah2Cd+efZ9W4fuYM6yi/v3zDqLepXXLetw/aRI/rVnI0d1t2Lq5E8f3tOSt/Z3Ii3eNuW9jfszZxqaSGM7GweMBPynHI2cLTzgHU+aAvTUyDp3HN+PXGEerBaSVdmR2j/uZ314En/9TraNy/plkrNVaDpJKbPRdLE5Zhnm9xjChpbgPg/micMxqsiBvJ91t40kMPCfuQUIwYyL6g08cWCvZl7Sd90PBpMnB6lnIGagpvYQc/zDBLaYy5m5/HGVe798f0bwWuYlQFcyMP4yufK+iObjqFF+F+JFoguvHPv238iz1Ehhz5pBfWMzuilAuWOB8cTvKpP6K35r9e3IsZxhW4OF8qIzykI6pyaSl6kN84pbQXQ/nmq4XaPY6U/vwTulUVlfCY/4QixIUcL34OpOv6nnD5z1e2HEMs9aDaWQuYeKthUyLFojwhSVwEiVStVDux8+X5iNJDpIDjzPzmcfqEe0vBQkU6mzf5ki9XlEg2uucRJ2qhoTA8+icyhphAT5RRBU9QVT2m+QXjQWDB9ghtCepmruQEQCOtytmi2+xzpx2/CQb+joGk9f3rHaYCZbhET+JlxMOKx1WYyLfZAzktBmM6Tb26pXyG0GO46QEiPoz6/s9hnRoojh3+jcApNtEIKbWCT9WDhHJhUMTwVIOmb/TPyCHuwxwnx8EOZVzQd1y+GsVvF4QqkQo2qrofOFJXg0WMncnKxqiG3s6r/FmsLuvZUuW1DNL3yrL5LMymFtQw4GNED92U720siTB/J4P8lcMVCRDUqlHTRhrOZMZxfwQeCMYViYeUZ60yxe8fa0Em0PHi38d4axejAN1cnZDT3HV7J4PG2wO8nYyNWgXR2rhi9ymXpsJGa3Wjwq7iuXlcKlEMN3rknlfLG3Oid8eRLscpAPV5N26xLL336u/Z8+C8iUbFxMfn8yyOcPqx6dr7AYGJq7m7f0bsEaOUVyWQa7Fz/U82jqUsrjoQ7ltb4uPLArKp9i2KttNhbylX8OX4dCm2VxUDi9UZ5On6JkJ+Z4gff8UCGiOJEFqSWfyqxMJMuQpNmcAE2LGscw/huEnHbSxT1Uy0EO7sC97NGYnJGqgh+1zRYAiN2opPU+Pp60OPghDWUcJCKr+ncaxq3hrUCda+gxXtHFjJZH71LwWDL9EQYLhLN6mdtbgI8FfNVAVMIoXZs3i4pUrtH/mZeYHRTI9UOKVgHg6jGlNxh/31s8FKhVUWEIpM4nVQXtoKPzhEYCUhHe0IxpWRDWsVSmhxl+Gr6IsJNRudzfIamSNgbCykayrgk/SH29QC4mCAxhajWJ3myRy/DfU/3nmrFnMW7SIXrJEfzV3lM/qMWQRScl185kkglAu39KmS+B8/kDMdoOrHbj2MezoAje+4JY0xX39dY9wSys4Ph1kDeNv3M+rxdDp4MscU3kxYxxW/CUDZQ64UtwR2XNurM0n9txMJvvB84Fgc0o4m72oODws7RNONxJgh886zIGC/fVtVgJZdHUiv1dBcs0IqEoXgAYAew3fDY/j9cZ7mBMEXXVKSWhT2H38Wa1FKwlJJLXVVVM59h5IfpRSj4BGA+aepYwZAScZaYQqJ/hqvDwFtRGrHErn6C080+VR9HalTFkdSyTHBq108ECNO4kky3C7vBU/XljI8exRTJq1kXlvvPgvQCk6kYh+dkD9+2kNWU6r8INUJsMQlZK5WRM6nrfOP0+xHQ41U7KpcTpIzF7Iw37gkE1kOu7gZffbwoZaPQtDwFipBJYQ0JwfKkVSor0O7hnpjvjIMtyf/BMzAiDbJmOVlbXxJAkiS+/Fz/ciczq+gipfyfSoW2tu2eBw1UMQ0Axuroa0r2BbO95v/i1n4uBhf/C3esnpZ6zl43D4IBQy4tWKhByShNRucX39dW/ZUgBt7RWahZxiSttX6BOgrKHjlPVUWEKxOXTcFz0BNjVW3NOwxp/SL+EHTuUOo1KtBPHYnGZKfY/SVguPBJeCqUAooRgiIWkKX2ZdoOmtr7h06BK/W28Iv7DNQuoSVl1Cf+LdEHg5CHS1XmCH2lyuWEZxvekQanQ3GiilWO3i2QzJgeWn31XsbSUJVpz4kkm/lxBpvCHqkRbsVTyLZ7pM477m76KRa4SKRbVbLq9Z6GEe6/Acn5/6hMqopxTnrUvORKsgRVflkrR3myypMDthWw2UqryANq1e41bQYkzXPmdP1lAkQ5iyPawHr5eIfcvLsZcE+LLvZmj5Ck6VHynBx2kcfJLpnWYQf1WpgnHTOYlDmSLpNLXFg4L17HndHuu8JCFABR0/hpTpkPYNw+X2hBiyuVnWjlL/iW72qN1CjDaNrqWPYJRgdsw1dMXKAK7NKZ7FJQusuvIUcmBTOP+GO1mEqNu4rgo2y2n19fIkCQYkrqZvtFATumIuhYTJIhkHXNIv5eH1udRYA9zX7WHPqi4TWyTKISjqTNfcpk3FQVFXGFHz8XbIe0ISDmDAn9BRKPcczR7Nu4d+rVcgqLOCkD78VSNkjG94J4hCe/C1oytnzOJ7thGgaJZlcLiYqFapWkhWdvhQjHWbBVjtbnkglVfNPkmClNxXiCoZT+ebv4o/Jk2BJAHU+Cp4ATtdS3WDfbPdzP0hL5Brh75ZkBtyn7JZF0GJyweant6eAyYlsMnU/CVSbon5fNU98aL+a6v5EDMCSdagvr6a5IuH6HvpPJx6xgXkcUL5Rb4pfJ2EW3X30HBPLaOm3AGT8iDL7164uAB29gFzET4VfxJtKMDkhHtudYDWrymOPZZ7kNzg39gZC8PU25Qdh/VkfehXnL45DbU9gP0HlQzgYEMwQ7UCxJ5X88/2w9U1Qva2bcROInxvw4X5kOtiTjsdTIoewpzQSvbfHk+GxSvm49+UM7Vu1pzklN2BfacDndokSg84YUxqO0qjlEpTKhVY1cVU+JxjVblMzbBiGF8ByY+QGJzIgsQ9JOQ/RZXhMjVetWW1tjB0FnGnT/qFKPyN4KPtWN/klOscFTQpnSrWII/j66/Z+/3qsIQVSUvItcPcInjm4iTFvFItJfBR2hAOWWoZlA07I5XPytHyFQZmif9+JTeSgpDZ7sZbaxheNJkOrnn+mSvjYYwL4Ja9hWnNJ6KO/rz+57raTKFCFtQOfc0p2kfuQGX34WE/eDjII3bidU8FAVuQrQVivq++DemrCPe5Ud9e7kgUNVwTXKUV0r5BLWnQSNDUUIPW4QG6s1UjrzUwwUfs396Nu0rH9ACIuhs6Lq+vV+10QqO4xzjg/Ndxv38aq6moFHscCQcRxnQ0dg+fwFbDcN2PLCz4hSfMExnX5FVlsi9/L/f4nSSyqicLgqGp5MFyczEZr6vasKYSgit7NdxD+jdjZ42TpWGwe+hQtE6PWKnKyE8ZP7AtdTrTNqZzuHap4tD06oEczBxPp/Sf6J3zAxpbBdgtAuAW0BwH7oX+Qb8sODhRgEMkiYSs+/i89xQsTiH9LWkDFX0HV/xAt5ALvBcKPzQ6q7zm4E4c8+lGsR1a+2UTor/VYFyDzdvpHL2ZilrnHZ/FKmAm0AMYBexCxGk/c/3/TtffZcBH7yTeupLiKrffbJeUIELvZJ9DVsZXrY6GzL7/q8m+O+Uw/kfP///krP+1/9o/NEmS8Pf3VzhhsgxdU7cx4qQTn5v7RPHf6ltQcADURsUkdNK5DKJHwIkZUHkd+ghN9qE+MLTsMkX+SoRZiE8IrTJcAXCn1+ykD+dmq/f4traMP2vgRpXXhh3o7RtGBx3EFE9GlmHmCy/UB7Tn+LsD2n3vcZLxcy9FLYafL77Om/u2svHa89gDPBBmhhg+N3/O8nI4HQe/Ja9VnrTwIIODjqKVRHvvWo9NcWgX6PY1uSXjuLAvl2m/WRQ1mACqT73Igrv8+a7KzLp0ZZ0b1D4sDn+dGQVQnbiWHr3+fUH55xELyyFghxdLKdhh5/F/cZy3PVRby9Wjpyiyw7biZK5bZil/oNKSaUtHK4mAYkTJN8p2h41IKZv+CWsZkfQTKi9Jk7qAjxrAqVyIZBlWjkzk5V5j+O3KSziClGmYJ5O68XubX7jH3o2nArR3KHQv+j7RCHrmjRCdX14Mmesob/IpY1Lbo1GZCVBJSmYVsEX/EBPyIDgdvjUpUZcgNlgDDbA2EoK51KD9tm0Yp3KH8tz2sxwOvFD//nV6/pV6WY/Zvs1JmCwSKjidxMfGkrN2Ea+XFPFZKSwuv8mBj38mvsMkEeTXBpBeeze3LBreL4VjZi/047lXebTlAkBsDO4kLwRQVhtJsSkB+M+1jgL4Z5KxvnoNaskEZZfAJDYQkgTtIneSFHQWAN+afXDyKRG4jhzIMe0v3Cxr475G/ybQb4souKzSsb9sCgfLw2nTZCFZZW8ozulzaxUHGkFfAzwSXInarGS55UWO5KeaNAqT4Jm4Y2hKlSxNVdUlpgVX0vkS9Ff/Z3mWugT4N6t+YMqNrnxcBg7Z0mCsq2xlHGwugimdUjdg0HhInYT3ZmHVKn6shCI7bCyPB3+lNOD4wCUkm1vzZQQ0dSjrj/nlbuHDUDPlDgupJV2EdGedueRsPcN13sznOuevd9wvzO/aFUrP1iPaxzx4mcHDLzBg8AFevMefyENthDwzIGMhOuVFjH2MLB/aFk3Zu4p+9eaLJAaeRSVZMWgqlZKWzZ7jUMBiHECpA9IdTerrqgAQ1I7J1l5csLiYCfJFRd+B5+dSnAzzi53M3amUgarT4a+zmzXnFce2rRjLZ66Y2JflQIyLmdXqNej2HWr1fcQWTRH3KMkCqd7rV5FkuLkam0PD5F+cjPzJiRyjDNxpbn3G5mg4YoKPi4NB5RFRVBnISHiCP6pEHYjNeaPwtrWqfswrhvNxcGTboyx+5ZU7Mkt3WGG/3VkvrSzLsPriyywqhYO1YNLEKs57SnqWzdWCMfhadocG572jDJkhSsj0be9EUYuzpCfAuiiQq5UJJho/QYfb8EYJLMxqjtqTuKDSwrDzvFpU9zwkRQJzYcFVviq38WIudDlUTqPEVrw1b94d73mn1cx+Byxevrv+nrdcf5oX/zpEmSm8wTc3WXc/xxvBhkg1K5qMVzY6HehVMnk2eCIzmiLDaK92O43Udh4LEPVRtDblPIJvIodN8GwgvJi4W9EkSaBV1VJmiuDRjRnUJr6kaN9WuJw/rr6ISrLTPGBrA/naOjWAs2b41bAa4icKuaVfA5EkaHV7BRduzObpfb8KNo+HXWxazIOXRvNaiZPN1co5BGMcp5xRbKyG8bmQa/NC8QPXEtTsjoWw8iEC9SpJpKSkMGv2a0wZ/wW/vDaQe0aup/+Us0SOcrPRVCoINuQQ4yeQ187A9hDmYs+bi5kRWEgPPfxUCX9Ve0mipn7GY4Emyh0w+8iHlIXNcLdZSpFLjtIv61NUJ69wOv0LpeQg4Dz+JD83Oc03EdAz+Jyi7f6hLZkwxJ9LOnhGhrGyitdCQ+nw7LNkZGUx8e5+DG38Gf7aIlSyQwDPLCLoURs+kQX7N5NfnVT/XDnlYjklPIjsAeqpf/dkXb3sqsrlb9gkh0JmE4DjT1KQks8lC8w++gn2UA/JQXsNvvnbaa2FDdXwU24bZK1S7lzlMOEnwxkzfJH2BAS42doO2ZcD51ZiOFrAJNNs2JgMqS6lDEnDj/mt2F0Di0Ik7vVtCFaQnRrSrBB5E6oSX4STz8BPGmj6HBvtIuEUo4YXwg5B3i73wbYqJhrP00QD44++ji1cGZSh2fPsC9qBFLpR1K/zBL9Zymhauo9OOoh3DZXe454956dqayDm0JHMfHmxQmZxqp+Rj6LB0OMhMn4ZwMzpD9Qfs7zNHL4Ih1NliQT6KGv2Wf27sCf1WZpp4ZVgpwjYeVh4/tcMNQqgxANBk/C2zPJMtpaW0CYDKiLubtAOMDdI7AGCjO57lmWYe+xDHsiDCOMtnHblvNqsZiaLek/klM9ORt72wxnnce7aPJIM2xmbeog2N7+mm9rFhht2vj4gCEIyy+cG3A58AGXns5h67FVybFBs9VGyyXAHZXvpobX/TbytoNnvPLw+l09OfMmeyo8VbXpVJXEBFzFqSrlePQqS3cAmWYb7Wiyul/tVuCJOJzH2j4lt/CqDjfBhTI5QPym7CGlfg0pHiaMVTmSqrYFUy8kCQObnStAnPMjeiiQ2VMPboaCt8WJZZm3ky4jdjI88T/9Gf6BTKWW0/Qp2M8wHTpshr6qLYm/rubbcrmwL/bZCtFJm9oMja1hz8Q0qnCkiiO5KMMgyBOoK6BC1g2B9boN1aqBlMu+HwvthsL3x7nqWGgCVafTQ3yBYrrsOr4PbLOC4Wc0R1Tnmnn8Ks85rznfZUCM8G52K5DRDzHBo+xZOtR+ze0zi6S6PIUs2JKcyA+X5bFZdXS0C1nV2+V1eTA4jyjcVP20x4dYtYK+F5KmC4RbelwvyW1RbAwFlYpDaHCb6t2Fy3E7KHND//CBMjT2SMUcf4cHbgglyyQLfX3sOyWEWKgNV6dDzZ+4v7Mj+2obXKcvw3PYz3O1ari9aA6HH90L+m4aBTWPqXNjcXOwVr33C3AQ/2vvmMz0A3mn0tZsxHTGAHxsv4SdXfFt2av+x/FldUndoDnya7lWzL7Alf0otKbKL+U/v9W6OT3ic18PcwEbJP0VI+qV+hlSbg9EeQZ+LF+h+dR8dQ5RAQ0kCtcOXjulriSl3Jeuaz66XZz5nbcHBWiE3/XrIeuV13fiSxb4P0tPZmBNl8TgMymRzbo91zCuuP5OnsrwwfSQ3rHDLCj/cult8MwmTwTcRlewQ0u/m1gSYW0OHpSKh0esXCGxNoqon8QXT+TkSZgTvUPabvZn58Ufwk4VPUWVoK/oN7wO5O7jZuZo1GUPqr8vbWqc9x45oeDwfDjl6Kxs1/iRFTyEodiUf9n6QFpFKIJjPjc+YG+wCAOn+2X7YaAygb8KPvDVgEBrZIubrZs/XX2eUbjdB4VswmlJoqVWqYGC3MCxzJxEqKEqCh1q/6X4H07/l+xEGd/1DJPc3V3IKNjcjqOw7HJL4xiWnxitZL2E0+mB3JQe0skdSUQJ/cwsGnc9l1Ckrb8T8KUD6dZfl15rrNYLFWitZuRy+RdT7Big6jn/Rt4TVCibXI1InWGuEzN8FC31nH7qVbONCHJwzw8W8oYrvyqyKY35lPiddBCXvvasc1I7dHvkORXNoNy4FzCDDte2UPeQaqbhGp7C1BGvc35QcOUBInTe6j5DMl5jZ/SEkVDTLgB0qd1KQ0rOkOJcT6Rq/4KpeaM8/CxviIHoojKvgVOH99T8vsncQNVxTXGv09eX82uVdJmasY8pvteRGeOybJTXOqLs578L9H6oyUhwwFcJ6QdOnRSz1xkqcTtDrI2na7F+rV/zTWI2/fwB94n9iTs+JzN15iOyIJe4fOMy0ZBEdo7ax7soc5h85rajvyq019NQ9g0G2MzQH/jBPc7f5JsAkJ78aZzPYBzalXCJG6wZtAND0aUbmwGkTHKrWg8qDOqHScq78Aa4VdyO/OhGzpARFHSl9gdnX7+Jc3AwaJyyAdaGQ9QfET4C79pFT7Vbaaq+rgNtrodNyGHYeh8qfKqsvqVYh/V0bOUXR9w+hvzGw/DwxakjweFcAiBzI7uB7KbTDpjbrGRz7aoNxbVz1EtM6zLyj5GossA8Rmz2MiM3uAcWed6/r72ZAp4akm/D2Cfe35/Bg9QE4nA6ljOcdmH3e69b/1WTfnXIY/5P232Tff+1/hdntdi5cuIDd7t6YxEob6BwtEHr138/ZuXDwPtjehWnxbiZWnjQEunwGA3YJeQ5NAB/kNeJP13wqe20+yfiFxxO38qg/bE3+UyQPPKykVmyin82KZWdWQ+3kaerNrIuCNU33MlxuB7glWiZN2kjqCgfvTn+Ie++ZRWQ796b63zr0Kh35UguybbC1Gg5WJSjbO33KvVc+pdoJX+WlUObrteEuv8pXg7QM6TICk8ZLSgKosZqolSpZakrnszpKmofZHXYaqaEsGV4bnf8v2Yo9VVo+AXZwZ/3uf8zWMlURoQKnZGv4g5psmmiyUAHjcqEkbJqy3VLGC5rnyHGYaXKmL5IXg05XcYmnA8CaAp92UaLxVCo4kTOCs/kiWOM9RxeYW3HOVsWIqKOMC39NmewrPsGI2K0YJREgq9K7agAMvwydV7h06VXMCoQdXZ6CciXivVDXmGMmUfugd5BX4rU6g5cNl3k2EMb7gY+kZDh5X2vdYhwZGcms2a8yYvhm8lcW8Mlzo3h/2H5+eue++iD/XxavIL9X/cQ1znvID9rImyVw0eolz9fxQ5ZcnsMQH3A0teFX6FEKuOwiqr396dHoN1aeWcYnlwXz4D/VOroH+P5ftHnbag10ap1CiOYabG0FN0TAUZZhwm8VLD4oEPaVgaNhoEDU4ZtEnmZcPUOkwTqs0nOg9Fu253XjhwgYpFZuAh3RI3m2AD4ph8A0sIcppVC2+A/hUw+SjjN5uqK9JuZBQtLgxHF42Oq9c72zPVRby/o1a5CdIsi7vt80+kjKhEpS5hPcbHUBgwSLmq3Dv8CD3aLSUyNHUeMUtSxfzusJ0R5BQ4edHvpZ9ParYFA2XNIo78lYdooZgWDUVnKqeWfUKo+5QhvIQt8JLHTFGHfFwDBJOWZJ8hoWD+xDhTmUDRnvgm9ifZskwfuDuvP2wP7UOgRSrq4GmWyv4ItOHzDWF14rhly1Uh40Iv1+Xug7GP9ujVk9wr9BLawEv6ao7EZUgFZy4h0pUEkqkjSCmeDnJbVjC+vDF+Vw1QpXi7sqa/ZVnOLnTssY7wvDfCDCqQxW3vJfwNIy8d8vFCFknA6Oh1trIOmheimdUBW8HrwSio9D3H1C6qnDEhYeczPjJC+pMLnqGgMMQh6tjmXsviEdeXGPsK1+z9DQyWwScx8XLVBeASvXrvrPzFKXtHJBQR5/pT3Gh2UwLAcqQh7xGBADN33e4VjOEP6qgXWlXuyAkjO0CliHWrYwJPkLIu0uRs2Q09Dz53p51VoHjDaCZC1THu8TzRkzmJwQVNW9wZpptVuJ0Fg4Hwd+BVP/ZQJzjU2Iim21O/4Dm9bGS3PmUFSUR351IiW10Yxt/h6aCqXUYa48gePpT1KR/RiZtV4AoJztfBByN4/ah3Ag9QUqNF4JKEMkrbMjaJkh6qPYdbHKdqcDNTDGCKPDlNJCsgyrR0fyah+RzPWewy5WTWajqZyMhOk8mjwcrn3o0fgWPw3pRKwaqp1Qom7hZsRay/GtPcKw+D+IylpIdsF9oFUyAH7L+JK0yCV8WAYHa72kcqIGs0QeyOZqwTy1yl7vAVCseZSzGQ+yJ76MhLT+9X6ew2HnePYo5u/5i5tl7Rrck0oFT3R8mhXDxDmdbRdBL1dtDVM+r4RlMNgHPiiDHyv6KA++9jHTg6uxOOFoYVesPh4y2IWHkP7qTpfI3fiamjUMJgIXIyewrMzBI/7Q2Der/u/LliwhvvUwiv4qZ1k1/OqATx12mlRVsWzJEn5as4ZE/XZmdJ5BqDETvboa1sfAaRGouqPcdKcV4h/5uxhma4zWJUla/9uhp6DTRwBMCjnPcCNEJrxHldlL7jysB2sqDFidkBe4EUnyuDFjAnmjcplbDO108GD0OWSLMlRQ0uxlUjKEL7Mx+yEFi0SWQe0worOFUW1OgOZz6oPdqLS8fXMY31fC6DP3c9L+nqLf8HO92NSo2qMvCQJagtMGhyfXJzBbaeH+4DNKZqk+koeLH3KtNfY7BhFGHQphlnMTAWlg1nusF6Z8Omev5B5fGJAN/Q/PxdLBDRSTJGgeeoi3BgygbcTO+r7rfPinX7rM8w9/xgdP3U/7zouJHLsLEt3+9uqCLrxXClMv3Uu2bqrimkS9Zifd9LAgzCoAiu4zc7PreZ4sgMYaaKxtGD7LPfgI4/yEVKdDE6RsvPkDtxOgxA7zikDWuoE4sgxl+eMpKunNlXgnTWruVxyqd+YQYbxFlRO21lpx+nrIRBUe5L6QoXQOyOQ+fSjTIttA3m4hgzzsPPTZwJiTb/NmiWCoK0AnAP4pnClryuJSGHylq1vmte6u9w5nQTD8GAkvNVPWyKl7HgAXC/qRY+unaEsJ2MuKYa3pErOJU5XPQIcPFMct2L+FT45/xW/jfGhd/Yji2L7M4YkAWFsJo9OThJy+LlS8wwGt8NUUIuHAqCnFh0zBGogaBENOQYcPeCl7JB+VQfvbYPFKehDWg2cLoIseFnRYjI5iRbP/pdd53y+ebjdXYamqxG631zPQP3q7Ocd3hXPpYCzHz6wgT24vmGwez/JkznCuFvVosP7JsmB7PbQ+lyBDLrpKJbM+wHmDeDX8XgWfFDZFEQbK3sgEzTyGacKpSYZe1oZ72wlVM3i51WcU+e9s+M1lbWBXDPxaCd3PDRRAiZPPQPp39UDW3y6/xIoTX5LZ6ozi0P7qMSzoL3zNs8Vj6tlzAOjCKbS0wu7UEB94gfYVIyB7s5A7Xy/WymvqV6iyBDO1/Sw63IoES5k4VhvECfMb/FnrxA5cNfkhGz1VR1Io0jevHwW1ShZAsIiBIhkUP4ErtmAAWmghybmyvsa6JEGlJYRyl+vlzXIL5Bx94n9Cr65y/V52Jbpl8Ikl09yXSoeaZA20NGS4AWoqHVZtMCbXNG1ETZPckXDdtR5c/QjSvwOgfeQOnuz4VAPQQN9rL7HFlYeX7hDqkyWZfga4lQhtA9Yr2loGbmKo2p3Ex1wiEmchXeDPrjQJOYW/qRUhVX0I1inZnd6AWW/7uvYh3igRNZeDVF4B7YCW3JQfJf7qZu66kE6iv9Jn8KzZ92xUGoMbf/v/sffXYVLV//8/fjtnemO2u5eFZWmQ7u5SQkIRxU5CRcQgpMsWBQsUpKQRpRuku9nu7un5/XFmd+fMoG/9/fH5fN/Xx8d1va6X7HPOmec5c84zHo97yNpFSznhCrhvhuW3JkJQR9jdEM6+gk5RyM+PBTOxxXSpXxGDpYJG9CjQBhOtO8lT6ijaCBE0UrvoJuWf4KnQK/g6F8HD+4MuEq7PRxRsXKw3Fg8Bng4+IbOMAbCJGqrssCGnOXEhS+TntllQWAsJVkCSGjyU8sIr97+liU7yh27WBn5Q/bO07Rqdjrbdx3E+ayCf//k1+dVx0nhdYzciCPRPfZWncmFbq9UM0LsAS049ycYB0v2/YoRiQ2hdYt+7Picyx5NtAU8BZoQ/wKtst+O8ChCUXCk6w6V4Sdixp3cJyltz4No8SN+K1WolLS31ocU+oBbAZ7eL/JCyWyoWO6Ky+VpW/yllBe4HfkWJx0BpLw+Q8hPBD56hR+o6Wjz4Ca+CZyG4G2iCpX5VphFtTEGvUBKf9zLhxaPcGMM170pPHYSa5EBXZ8naj+sfIqykbq4hoA35IR9IvsVAO59MyHV4Jye+zpQzZi5U1BWO1GofSfJU7UNp5ExWnvsCwa6k2AYmZ4uKnIO0sLzOiPR1NE77hLi8N7CFDYaGU6V5VuWNoHABd4E0l42zg74B8T7X6Rt+GHB5JxUahG47+KRE+ufaYj8yo76VvFZBUvw5/3rtx+PqTeOdjx6e98sX4Bv3Xjw01uh0DBo1jmCPVJoEH6kdH2tDqWev6jZL0puxJimeHzw+BNHpnUyaxh7TYSpsEnM5lzhc4/GqZXwaBPW1FajFCrd2kPI2I5JjJXlZp5C89WwEe6agEwvd2gq9j5AW/DV37CUS29rLsR4uPEfXqLXoHM/Q5uoFMMYCCGA1IWDjakHd3lCrlAPd8ywlGO0wPge6PnABBQBqpRqTXWKlHitxB9He1y9l5bnPaZrUjDUq+Vw0EogBPgd2Ov77YTKeMcAfQG4FJBeD0VYHyqmR8PTX+WP9wMrVl666MPvkxT6jxfifZ58jHlbD+D8Z/4/e9v/if2O4viQteZsnmr9Nqe4iAxLehD86Qc3CsyqNUktM7WdFEfCIhNCekrZ92S0OlfuSYYHPgkBX7qKPnLqel+L2Ii3LXShfplKi7y5mtJdkZP+wwWu/vQMfFUGqQU+5IPdjs6Emp6Ie5SZ/Ltnn10lzICWxBiR8xdwevfmkfws8c3+sO9BuQysa0AjwTiF8nO8yGSh1WJAWCUuy65Hj6VT4urcKdiehUZip8t9Nh7iVdVJLjihKeJVmaVBiA8GVzWi309B0k0ZqWFUKqRVtZWzF2eF1bDFDXCi9VH9t1PtvEUBhuhLW+gfyZXQyfbxd2AH3VjFb/wnRKthcAWZ9e5d74sG66nZsqIBcsVSGjgfQFJzgs2C4aoRzRfLigSjC1+c/50TaSJb0aY86R1482J6zmpczA1lWDJ9mfynJu9ZE2kbebLyQQAU8nweZ4Q6kuz4RdGF4FG1lYtRhLhnh58JwN41/jUqBAnhSWZ8EVZ6sDWMRg1RZnDaAcBeK1HLkOMBQr14s7dOOxkFH8bXW+XwJAhitnpzLGkR6WRLFRam889nJf5zk3+u/i5l+0CT1S1o569UDhHRnud8r5FphczlYtbF1bZYqKLmKXlMgO+R/8jp6Fgl99D9Jxp4BDloU4LMCA8HQ5EMI7lJ7zc5hVUdIckwOjwtXpC6lt+DiW1DwZ+3f7IKNcTlw0O7iARbchc9KJYZcqQ2UKnmSq8YLqc3dUF784wAKTxfpLKUHRTbI/ZfyLKWlpcyJusaT3nC4Gu5Y5LIfVl0C5wygFaBP/BrUZU4JH6uBEEVe7ca1Hq7sADtNlMvp7FXG/iooU8gLD6UNZxD0AJ7zgawW5/C2yYEQNcCJ9/yhpwcEIGdzaIQiIvS3SS9L4mTJ2+ARWZvk+nVNEh3eyqbf+zl8sesBOU021PlZKr2YkhbLkmL4qAgKVS1k503zf4q55fncEtNYU+wlbaRqImU9TYqm0Sp7DlPpxlc+L0C2UxHSWMRARRZ3TBBxuQ0FCrl8kEfDVzlgWoSAwO3YZ2TPlGA3E6Aux0eE3RHQ3rpNdmyB/mk2ue41PKIklsCOerRSz8WOHa0A9VXp8qSNdwI+mjyaRf5CUtg2MGbLTmNr+TEe9+FlHzgYk+7GLlaICgSkJOrwcHm/AILUnpwzQM8tvnRTqP8xs3Tdz6tR2LW0ububNnd3oHcpAumUOoJKJaSzG5Dm3koeDx+JTlnOpFZTiTU5pJhEpWQoDzyR3JXGadDswGcQIJeBxWokQNQyxx++TjwgZ/YBWednMNgTSsrg14MH/nJs+7eG8hs3rEYUrER432F8sw9RlMr9E2+r5vL9jSl8+mAgn2bNkp/EI5x71sc547GXm1FvUu0itSmFwA2TJBXmJh155QPM9SWP1qcP7JA1iSIcSR3H3cI29Iz7EVXFZbf2i/XGcjtwNW9mJ9bJ+wCo/cmtjMdql2TBtUKllHDsfwG678WvdA1vtJuEl7oYrbLajYEeY0+leY3UmOvvDKgcrLPEzLl4OM+PjrihXs6uC2vIr4zHqpGS2bm5uXy84iPOHq7zT9yzZhA5p1fIrulk+kg23XjX/TZ6xjEivS2ryiAhewZdNa/L27tsZXyalMg8kdSBIqPT/OrTCFou4X7RI4xstIC57ZvWMsRroiiwGx8Ugtc92JAu+YCuWLbMUVS2sdfMXwJmPt9bxcyDB8guT8AuKKHB6xAqzd+a8tM80+JNwrykgpYgIAEDxtmh3rNYRD2iQ5ngYYWtqaGHeFoPB6MstLTNkjfWm8T83MHUU8HTTWejrLhS1yYI6JQ64oUeVFZL84iQIofYOKNRFcgTFFqxmC8GNmZM4zkUGWOg5SKJ8eAIFRIz8orBi3K7XGrOpg6hwKzBR4SX7J3RVt2B+i9KMlJlt2mjWIQKaY4bev8FiJtQd7CopFgIwg48GXUYT+Nh+TXnHeNNX/AQoMwGonPWwSMSevzBhcrHWBsCyxuvr72nOTk5rF83l43bxjByzmE2b3ucH6apyTk+3/mWcSjlSZaeWieBUlxiTW4Xvi8Dm2B22x94Zy7n+0EtOGOAFqmAn1PSRhDQh0RTYoMncmFossuzC7QqO8IYL/AXQWF3AcApPcm0SJKDC4tBqa1jpQoC6JTleItmlpbAdbsckHBVv4GEdGmisokGeb/9W7OvbDV3C9sgYEfAJp0w9zAUX4TIoeSYQvEWoY0GvFxliO02yrwkXyWDOgO3KD5Pglra13yXPNCtWV1+lrYR0rjnej8LjIn8fHUWKSXN3I4TBEgpaUZGWUOu5PagUtlI1vi7+AsfFUGaBY5W6qVxN30z3FoOl9/l1bBgfLT5jGq8gBGKaMkjabM//BoCv4YQoZLWfQ+KW6DRuazgfJvyWSnMKYLH0/wRdYHy9rZfs/bPHznQ41UW9H+cj5cvr2Wgz869xbrqfOaWZOJ7fSExkZG1Etqu90BNCWRshzKncQPw1eby/COT8SiRS9H97n2J0TnSPHMhY0ddYhcgrD9nVT9x3ajkQDWUitG4xgZzY/Kt8FaTj9FZXViYxgKaayQgTklRL0my6s7ncHoiClsZh1Ke5Hja47J+1oQdJTab0u36AKj3NOvzDpFXGUtmWSI39N+DZ6wkbyqIEnPLcRklhhDJy60mOaz24YJtOr8oDyICvuoSRJtTkanxuxyNWIgNGOIJSzqOB2M+9NovrdEO9GRP8Cne9IUeOmhje07ya3P0M8zrLgmOr3pEXQ6nn5F8v4EI83re6jiu1hO8OnEBDLouKRBEDef5nFYc9tjFmwUw4v6iOvaKpYqe/mEkWaXViQ4lPtX7JNYpwM1FtdKF9fwvMKjBl2CUz1Ma7zhKK2OZFwADgy/I72fGDhazhWgVLC2GPFNjWfO865m8fVBiUSvtWvijPfzeRgKiiWoqLXX71YcBcdzazr4syU1TVxAdnAVvFkyQHxzaky22FynTH2DHWAX6tDmyZp/k1Yx0LCNeDklGlyOfpzxvLiIzHkIUoLDrJKlDu02S61To2HP3Ja7lda3rV/FlyZ/MbqOtfjkftp7JGzsz+K3suLxfjd6h68XnMdghKw6aljlkI1suhr6n8Sg7QIIK9CLMDTRD5k7Z4Sci3+LRbFDYPFGLLkCI/GM0vh5KgAJCkuFMtnz8s3Tbzau5EpBgbl/Ju/2f7IcPGY1UV5fxy9bxfLhyNru3deajD2eQk1H3ztbInZdYVZjs8oIHwV05mjGe0Ptf8cbJley+81rdexnclW+u/cRlE3iLMC0sFc8yR4HTrwUMusZ+a1Ttqfr5FqC88aHkd3r/O6rMVcwsbUVWwM8AqAX5mqK22IfI3cqB4F+X9xFFUNik4qBFUSF//hKeJzdpN5bqBkQWjed2QS/osQeCO0P77yDqMYKq7+E/II1t9a+wekisDNCl5yZ7E24z3hs+DYLGRXJLBk49hTVBkskeGvgAr2r5cxKkC2HwOTuDz9mZFfMnnHWAe0WlW0FOUXwJrs6F6mxMAf04mT4StTmYJDVkVKyv+2DMaI5rj5CfP5C4vNcR7WrsMU9Aq2VSDq/wHN5OOaLGHmskD9cKx+/8yGdsvfMuB5In0jlqIzqDfH/gvLazY5WPvc3nQfc9sns0ZVQUqT+3puvUSXwYmsRInR8vBkLeI1GcVvy1r3FNnAEOm0w8NfFZNt98h7Fbion1vYqX0elIUUGxEMXByNfQKsx4KsrkctP6RK5YQsn3Oo1GAKXotC+xVME6gQTLZS4aITjFTIbVybIDIH0rSwOlZ9fNI91q4J3EcCa3n8i3Q+Nor50uOzTJaxuL6v9OsAI2en4Jbb+WfJeLLsDvbXipxQSizJKUeJAqTppfcw/BBg1+xT/S3P8ycZYY3tfr8SyX7yHD7Okkqtx/FwAydzOueAMJalhcDJcr3IuBxZreXMzpx+iOgRy2WmW/xbNAMtANidkXAxxHLuN5zPH3k0BnhYKwy2CzGmrfsZpin4fKo3a/5zzex+VOZm3LDCK8I9AoNG4yn/+vx/+tQh/8V+z7L/4Xx3nhSyaV3eZY41bk+C2DymSJzt/kfeh7km15P9d+tr+9ORxyWkSde4Uvo+8Qq0Iy3q6SS9qUNZlDi3Qbq8qg372+UhKoJqwGgjI30kUHC0NL6Bjkjn7cVu3Nt2UwoeIGp0S53GapOZrp+4/z+/3nH4rKbBO+mxahB9AqKxHsTr56hlymK/vyhafkd+aG1qtMJ0mXhgoo117H7ixZqZCKgOvLJI31OY/MhWp5ya1m8P4lFOYlyhOKAONL1/G0XipeXSiUkuE1SOc5y24wcMgFRg9fR2mJ8Lcm0v+KraXT0bVrF/YVNEKtrMCOC4strA/brCPJc1yqRumykFZ6MD7jDFeNEON7CdEZ0Q7Yo0fTNV2i1K9+8KisrWYSUysMBOgyEWxyxJ0ggEGTzi0zbK/KkUtnxU/ig0tfkGcFjSnUDcHjkfMDc0Mq2FMFTxRlyVC7AGNL5pIfD98nXKae5TP5Nfk241LTdC4dzmDA+SrUSrXbfSu3xZJTGc+0DuNJLJcSRjk5OSxfOpd9e7sz6IOzLPp+IaNXRtBdrflnCW+Virn7PLlvhpWtPqaFYs5DP3vRCKNywOzvhHQObIswsoC9916kR+wauoXUJW5rvI66itBXJfc6mqDVYhMEBv7NIvIMMFChoUHDxZiEJlQTBs1m1SYca1D6TYKOAEjPQM3qNXsffSuC6RItoclFEUm65+ZSKL0ONgt9A9vxROwutlZCCvLCfQ3SL0CUWBGiVV7V6ZT1He/5Q0Z5QyorG6N08fxTWct5LmUf3sT/qwK4j17Ps6F3GOgJz+TCTpOcxVsWM5v+WZJkZZvDCylPdJI4K77EZM2TPKOH9vYYPgw9AvlOxUBBweCcAbxQJBXpvF0KOQqNHwVWSfZvbYkWuzO7wGamu+kavXTwpi8cqoLd2pdlxz9QvsqErbk8KG6JRiP3iVtQfIsfKsr5sCgf+7H5siSXoNTyY2E4xx31LNfiQlHgGL4rg7NGeCUrRF4kyj9BYOkqEkvGo0z7kBTGyX1YqzKYIl5loCcE5j2PVulyzaICD9GXLeF2rrZb4ybf0v/4Ar4tg4k5cEUlLwjXjCM6ATaHIUnctVwiPYMVD6jASI7fFjLMAt9UZkho+C2B0nNoqWZqq4HM6zKWxd0fxVQgBxyIIjTMWIC3CMGu3p1WE0kXnmZOAIzzhsY+Lv5OQMucORTXA+2NSiaaDW7tD4sJ1dXs3bKOWd0GsPqRJbwdexyvCicks6WaDpXdeDZmH98Gw86EvfITxE9ia+46qi1efHjod257OMnKRD0GfU9Dydu0SP6RoPLe7oWNW8spqGfgST00Czzj1h59bwX9PKHrNugs/vXYtp1/Zyi/b9s6xjf9gDk9+jH/2BYsEWNkn1EoIMt/PWfrD+G8KPeSwa8Fh6zfUWCF1FiIr3aRm7YaWBn7GK014FvRzn2D5NuMDSVasi1QbpYXzQQBvjr3FUdSxzGl/URUedvkfQ/vwtYwKQH7XZm1FgQBQIOXmXPyFMaSzoxQh/JUWTdI2wT+LSG8HyXBrzDr8B46R29i/aMekHdIdu6xhq/4NBiORcILPnJ5UQr+ZIL9NLP84WTHBcRqXKSxqEsGLjv1MwX1f+Tj5csZ1L8/Fz9dyLKKOv/Em5v2ENNpau1YoFBIBc6fr84FQLz7MVxfIJ1MqeOGKYhMC8yK+ZMh+rnyL/VpyAOjJxFKqK4HMTlO85hXPCS9SVpZYxSCBYVoxtVDTBREbEhMSKtdTU5ODu9Mn/6PADObty3kz9RGVFu8sQk6iZUXKyHq1VVXeDRpGaFeUpJGFJEKjQf7QcZWjnlewGDxkt03bn8mMVyAV1InMLMApubDcUtHtz4MFj+noRpm+IOy0okdajXiU3qBt7Qria9hpAbJ/Yl1JReZpIcJ3rCi5ZNQeK62TRAFR/HH7v7c2m0sq7+FKb5QEvINt43ydWVpsy28lBZLtBK+bHAcj7ztUkOPvRDSnVbiQjQCGO1QZNPLvY1tVl4Je4V21ia8H3+c6sLP5d+dvZclQdBMI/nXKXAa35SeENaHdIs3Bju09ElDmbO9di7K/GU+80oyWFli5/3CIo5ttxLT/X2ZZ2lN+KnT4eLbUuHLEQpRSZACXmr0GRWlTvJUgKDQUkg5+Va4bEIugw14mNMId8wZFou7wsZPUe+xrhwK60F0sUtSOupROmRIcyC4A5mW9W3Lb4lXeK8QLgk9ZYdKaHaBFhqoqgfibScGilcsN02TyK2M50zmMH4svCkBpg70gIN9YF9XTnacxP4I+DMaGpjlSSwuTOV+p6+ZqIeXwm+CRS4HxWO5jMuRJPIO57m/RT5ZC5nZZTi/jPChv7ecgVJgasAv1z4kuaQFvQKmS4UFRwgCaBSVqBRG5hzdTbqX3IsyV9GnVsKslhmfNB3afQsBbbhS9SImq5breV24aZ8iSUY6vRtbEj5hRzhEGZqiFB6eyLlmgu3lWkS1XFWEsD5cy+/GkZSxLNmpZNH77z+UgX7I6gALzJjO7Fmz+GjuXDb/kMTNY0qS//Tijz3vkLNjeK2SgUIBPpo8ovQ3WXTiFypDJ8m+1lutp9/FYvpfKCNAqC/vk08S2ZrxBFw+x4q9KVj8XZLdwBcVgVw1wku+oLS67MfqTSIxOYFbJkg0JyGISoh+HAQR0Ul5oE/8t3iWuBREhI3MOvIbGkUli9oFwfnJsvaadVSxIYxcz4mS6oO+IbRcDgWn6WbsRIjnA369+TbJCcdk8stKx8FRSrjZ4iDaOx/Izl3jo+MvQrTXPen5rM6WxtbcQ4QrKggwR3Pj3nROqPdKDCmkd+aDbkP41UGW6qlvDA++hypJ1zNXN575x7ZQYpDY8q5rlQf2wwA0Tf2KR3AC5xaeJfp4f573koB2ZRYvLtQ3SOpEAD3+gPY/ALDz9huM/zWvtk+10WULU1Na8q4/9Aq+Im/DjlIQuGOCxff7kW+WzxeCKGJH+r0ERIgZB7FPQotFMOAyGcZAdrUW2NVaIN/FuuChEu0Fp6FQAnI9qt3FFF/pz/EaF3AusCb7bY5HvcIf2e2xeMmLkPrby1gQ1JXuV+8wfe9VzK3lzD5lWC++LPCnlRZWtFoIRedA5Suxu5RefHXuS46ljZH6tbMB/NYCjg6DkiscrBjP8/facjn2Gfd5TO1DodkXkx2yKsNRaSKk9cbhAWAzEnJzIM/ogiiwwtu3v4Ik+TijECVRx7NJF2hQMU1+bl04v1nrcdMxDr3ysvwhqVYH8kuVxOh8qhLGPDf2by1UzgC9Hc978Y7PatdRi8tucWzBQmJi69XOY508Umijgc75Gcwrbik/UYNXmJLZhALv4+irmiOgcAfGIgFd219rR3n0DPktc8pLfJ7ZCHu/C9D7CLT+FJWootLJJ03jwuyr1t2rfb62K+XjvSbtCx6vvxqAMFFJuxQdXHAw/3ybYAociMnZ99g5oh6FVsvReHlzs6AjV/J6YnHCyygwEa024iPCyuuTyQ9fKj/erxXbKqWtW9yZsWTGb65ry9xD3L3OJAZIe+nNd+ZB84VSmyGPeO/jBDnA+Aq7BuH2J5KHYv7J2nsZXNafIxEwVvFD3Xk9IilWda31/QSnd6zwLPzehhYBdftCLzFD8nA1l8H978Faxea78ygzBjK98+P4laypO7fdDuen8JSjzvtWSA4RWU7qQ/4tIaS7rNgnmIsJVSUzc8oE3vzwBvVGfk3lq6DrFsfcj/4auA3SszlEo2Hh4sWEhUngGI2iijc7jiewZLWsX55CNn4iLA+Cm02OSOxipxBE6YcrjIeJqoV1DTVSyMDYHEkdy5VdRu4hpvnBM3qYG5qDaHdipAkKSixxFFZHsPXmNDJt8kJhuO4CY4MyaqWua6OojjBSbJGe/WDFPcg7Dpl1696beb0YnraJOSFlaMuPyk4xovQjvgyG7jro7+Uyt1YkE1p2jnCrBBh386Ok7nnfk7ycxx+fJ/stQpEKeeHAAuAosBcXYKLj7wuASKuViBugsdYV1C0KCcSl19SBUWUqRzZPgjQRpE9Jx/CegR5xPf7z7Pv/SPxX7Psv/tdGvqIXh43SgvSNfODRLEm2odkc8IqXDUIVNICCE5IXTcYOSHiRGI2RJBWEPABDmIskoWcM1x2LL5urlpM2iFtdj/BOATznV0UDX3fZmVKz3HAYqGWv7N0iIdbPHU7i8tYG5BydJfvsslM/MX5LPi/suoshss5/AoUHqYqJ3ChNZJY/PBfoMqXeXMri2A8IVMCmuHTCipwWX3HjYZydcbmwrhxGZFMn51BzySUXeNUHGqshTOOCzBUEyluu5PMS6Z92m3zEFkXoW28Vnw5oSZWh6m9ZSv+GrXXYZKLN0A38klefx3PgnYIX5B8K6sQ59aNY7ZAfqyIobcZDz7UoEK7EgELhIt3nFcsxA5TbAOSJFVGEMY3n0DzkAM/sSMcaLUchtvH7mumOOkeeq2+eT0OulHSj2g7TAssIdElEVSUuoVP6X197jrYRWytgbDactrkw90QF3h6RBCvtNA/8E41Q4nb8EcO3LD25nh8vLyRD/05tEuvsivmyJGr6hcNMMPzDJL/BwPZTCn6pgCbeaejFe/IPHOzHvWip4Nnp5qm/nNT71ltN38iFsr9NmTqVzbumkdYTXg6EMV7+fJKURLeZM0nPyuK9BXPpLEI3pbwY2FerpatCQeseH9G04dNoFJUPLaC/0uYFXm7zIgABeYthvSglLFV6KhSNqDJLixdBQEpiDU2WpBSB5h5naemoIYsuF6W8v4rL0TDdHy5Gg1giT3JFlp2npw5eCklh7aOhCPfkwABV7la+6dOHri2i+d5pc5TDwyUWcpAK4MPHjaPzpTG8klfTL/lFq5xWWOci3gGFU7FZF8FJ25P8aYBojYFWPt9JhaWaEARy7CoilGCqp2CAIE/wq4xFNFLD7kp4LssHm9aJ+WczMar6KE/oJTbGeSOE6qJkx9d0rXf8d4i3o1jw7rsPTXIdtNQkuWbUeqYJdgXN1HA0EuLM+2Xn1aicpXdcHr4WC7mQmMll701s5zKnxZ/kyRGvOHaHvcbVjFG01QgSWt457n3DS16rOF0NWwsj5cw+AUAqAPxYDpkKedIlMe9RdgV74SHAo55IHoR2qyTfFv8MF23PY1VUIdrVeCsDQBsCPk1BUMIJqaD0XC68kgfVnnL2gqLyFj198nmvEDrd7S7/nQUBlTEfvQjqu7D0tjsLKtejFYerQGe2/Dtz9YpSFKIZX20uIxstxrPKOelsx9N6A53PKarsUGFzkUQKbMstw1gsNg03CjpTrnC6Jl0olN3m00Yv8Vz0YZqE7pYbswP4NufbUoFuGTD0t8Nu48ydlt8wswA8r8Ek61+PbaX8OzZteUUpD4pbcChlPLcKOiC4SNE1ME3n5eazeE4Pz/mfdzuHkVIsdsixAgoXFLW5jJH2zzkbDfeankdlcmFMxIzmmYwwfAU10V5psqaa68+vjGbOkZ1YwuXyfEabn2N+AwHXXa80Pna8fYzQWzvJ8XxGkqRJ3QCZu7B6NeF89gBSS5twJO1JKdntFMc9RvBJCQQrwMtFwoWyG/S1XcVfAXeq/bDgzuxrYnmbJ5pK3hPff7usNuHt6vP7hxnJP9EhJ+22gUxZK6kXANhteIoCKqC1/y0iVIflHzaX0a1gMUY77KsCo1bONquJDdffZ8aJW3KvEKDxjXc4GwWtNRCoLmX1qlX0UKn+EWCmi0JNepqU1HCdpwwhY3hmewrX8upAKmwNg5w/wGqQXXPtsRemSv5igK56KVmlTfi8FK7aXFhOyT8xMug1TlRDQjJYQ5wACaZiONCTR7w/I9AjnazyBGkd7RTemdtZHQLBooheWS1DWVsEX17ec5Nfrn1IjPd5ODQAsuoKu70DrtBVB6X1YLj2I9l5RRFa399G9OXzzDvxM5bwkVKC/fxkSJzMLq5iNvviL0JnVRM5c9lURD9jPE8G5NIpHa6oXdDbCS/SNg0GeUj+dVqLyzsFhIiNaOQAYn2+ePpfypn/YbJx1Gpj3owZDOzfn08XJHHuYAD3T/tx4c8Z5JxaIiXeHLEgZjM/hcDb/hBik8v9VkW+zCPpkG1FYtc7e8sCut/b8INvNB200M3TnQVXpQnivBH+EOLwCmzl1u5d1ZR3/OB0hBKFrUx2rw9n9WVjlQQ4cl3L+JqOMsrbTIUNDlSD4BUra38oYyfWATLKPwZAQxXMSq+P0cul2OzXik0lXgzygMWBSH5rfxGC3V2SrCL8dT4+/SM38jtTbJPLBTu/R7Hag5C1V9b26YAWLOrdWd5vR6gdFJIZflDQ+Jo0N+fsg9NPgV9Ljhq+osrsw9msIVwSl0sAoSgH69/Bvo5SwrV2a1EVyBl0ZOzkSrTk5wz2v1wPLzs2m60n/pqBDtLYMc5kYe7s2RybP5/5Rbf4utTKjLxK8vb8SMzrIiu2FdZec2Lgad7tMgKVaMSmi5GdK8Syi0cCL/Bko2W8WK+nm5z51cr9HGway/l6I9C6gieRGAefl0LgfTB4tHRrH6/eyC+aPmzr+6jERO78C4y1gtqXmV2G837XIUxqORWf3E9lx9U8X4JgJ7uqiXx/WnCazj4fEuSR5vgMEjBj8E1o+AaYS/CwJ6NSGGXnAsBUwmBVT6b6SgCNHfefxBbg9Hymb6V5wbdoBWn99vSRi5K0/NZwieUZ1InrqkWcOJKKZ+pC8pX9QONf24+dd15jdYHE2sxTPwujSiWpV6Ba05RTGY/VgjTU+bvq/McKTvOuf6pDwrOcJl4HJZlYAM8YaPI+BZV9qJc9neCSIfLnx7cJeEvsEaPVgzJjkFzmDigzllHgvZ8GKbD4touHcOQw5mtf5FA1eBoSJelSp4j2PENQuFRIMwtVEniy4xrJd8+nIaKyrrCdVf1AduxDi30DLsAASba1u/o8E1UJvFk8n+FeLmCG3MMs9L2Jnwj9Kk6T5iVnHwt9jnHJuhovY32yK+ojeMqfbaIe4+1cXzwEaBPoYDOOKoZms9375RlX9weVnvum+nzLGZrFfI9O8ZPstBgLideWUGKDUVdHURr6uvRbGXJBUFBUfzV7Sn2wAA8M9dzAu3Flx3nWByoEA3mmXPm59Yl8aW1HshlGe0Gotxz2qTDm1YI/Mi3w1Asf1Vqo9NXIpRR7q7V0FgTUwDGbzd2Wwyz33v4waDMzHSRNV386gGyP/ZiDf2ZCoyU0CT4s8+V7pvEk2mjAAtw3eiPUMJtNxXD7MxLsdXNXrkmPENBSAuB6xaEUlTKwulrhIuPp+KJABZytvxEu1BVI1enfMyRegouXq7Io1fWsfRdA/vw1CjomAXEqUiRp1ZKr4Nucn3b2ZJxiGVOSW+DltCwtVzanybW2fFkKp3K7YvAfLr8hDd9gRDaYAdHqjcJZVsRSjsp4H7vuNhWa21wtGgJRjuOz9vBqYhce8ZIKVomW0RJoJKw/RD2G3+1n+GZwPayCgUXFcMIqV7CSgBJ147R46U04OUG67hYLSa+qK5qfN7wrebjqIuDMM5C8BqXCRpkxgIXHN1Lq7wRxFAS4/TGDHLiIoeoQvEu28beR8Dw8lguBbRFFSPJOpZcOzD5HeaPxj8x4rcdDZT57KrV0VYjMmPkiU6ZORRQhxPMBTUMO8fHp7yj0dyoy2owMNdbjkyDJE/7nohD53vb8FKZrmxChhF/KIRmn9a7aHzpv4m79CwzLXMmHJcvwEF2g1E1nEZ0sreEnBxchOjMDRRXfZ5xgzeUFfHdpKSk2+dh5svQd6p3vy20zdNECx8dIrL6IwdB8HlvuzSNfK+UDe6o/g4O9IOEFaDQdiyqMCc1nkl1Rjzf2XqA69GnZuS/ox/JNKcz0h4WhV+V9TnwVxpi5alRwJgqGhvzo9tM0K+zP5wOaYLFpGDDkXTerJSOwAUnG82+BicBGILUcyqx1ID6FTUeEqTvBnsE8vvlxntvxnCR96zQ3iaKclfhfce//G/Ffse+/+F8RoiiSmJhYi8AD3OS7AIkWvyUIrs2jg28dovaUchN02ykxB7ShEDmUErMngQrIs4KokqMuFZYqAhXQXgvvezav0/8HSTZEG0ylHeJT4Nt0OXIXYLrHFfZHwDRfSePfmb2yqEQqtiytuMXlHXeJ6TFHhlivNPtSZpI2D7KBUu3DFY/vOX9zAU9pwhnkLZftIHIIvxfMo8IOHbTgbXMpxhgLWRxIrV8DKnnC0TP3Dz4Lhv5ZMOmKfBICMMeM5p4ZfgqBlv6/ytpEEa7ldmft5Y/QaHz+lqUUCiwEhvDXBT9nBJCHnwd5PtJGWnTxRADQqdXYgFsGD6xqeQETSyWHIyBOJfl8iS6eV0pBkrn7MghGRsqLGgoFdIn5hS4xEjPTddJqof+R13xhbQhcit8CVnmyU+nwm3ne34CvM5oKEPQNOGeUPKl+CsHNb+FB1EtMyoNfKuBgVYnLNVWjqTzH4AafsaB3d/Q2V9RmXV+PpI5n5a77tUks1ySqD/Z/lfC2GKTElG+KgWVlrqbvTbho0NFcDZObLkZV4ZTwNpVA2iai9Df45Mz3zHRIxMj6XK87AT2g9BNY8uEIzt64wcz33iM0NJQpL4wk/TPw6gcvBcJEv/DaYmBqRgYDuw3nl5H+jGkyl0DFZdjfHdK31t6Ln6/OYe2VeQCYtQ0gaoS0YQ9sx3nfw5x3SKcIApJXmles9H6ISjokT+TJXMiPhzGinJEj2m2oBThngHcLqNNud8RvzX6gZyZ8EJEi/aHGW8ARdt9W/HxlNm8OyOGYzcQZYAV/L7FwwGBg0nPPEZb5DeaKpswLgM6qfbLzehZtZl6AJOP5iAZUzuhbzyhOKV7nuAF+NRQw536a5FvhFFFKI14iHCtogkktR3973f+S6zEQ5hh/ZeOwqOVL/SgWFsG0ArDkD6e7tzzJrxfu0jl6A2rbdRZszmCnyfQ/eKaZJJ+47AecaXSOT4Kgvgo0yJm2canT+C0cOmnh64gcKHLyhFF5YVTouBrzGjeip2DGJdmo8qZPp09oRTSruj6Lr1U+FljzT9NOc47PS2FySjN54sCUz1PeAo8gMQXdTN2x4GdsSKENAq62hDZfSICTxMnQbnVtkdJDNBGj2CZtAnsfkhIGob3YXNCE1WXwZSkYdfHyft9bxRetljPwwU46l38ibxNV2Ial8EaegBmw4VJ0AwITF7I9vR/RXvZ/xyz19uH9Q/t5Y+8lnt1xn6KgyXUfUHqw2/cOL5UU8Vo+TL+9wO0cNbdIwCZnW9uscGYS/pp0Xm74PZ91eFNeiAaIGMizeXbSLHArwt2k3OjfhmwreJqEvx3b/q2ctLe3DyfSR/Hpme+w2lQINvkzFGr7g+FeEotytK/LRq3sNh3FFwlXQrt0yPeWsy1Q+bBB8RZflMCNak8EV88rID7nTTYHe7CwrQtqXAHPtprMkAafcTZrMHbvBrL2LQU7mJArsZsuxqVJaN+ayDvGYw3m4qPJ415Ra+4HfguBbaUi85EhtcmdK7k9+fz8GinJ6BQ3PHrzawUkpsKnpcPkHY4exRvKJ3irADrkZZKBC/MACLNspW3ETrpFzmHxorf/sZx0SUkOoxrN56MevRAFC0L3XdDHgZKtSOZc3C6+0HSh35XhfFTkwvja34NVbV+jwApDs6EoyAnEU3gOfnuETlGbALc8OABWpReRSjgbDb1DjrF93Tqe/IeAmWfM1XhXvE+jwON4KIvh6HC450A0q/TkV8VgtklJDUGgrujm04R403I8VSXyE/Y+LEktAb4KP/RGaf5xk1QtPENr/UZswH2ziKhxWvup9ND6CzaUwzfVOVyt8naTA/ZqMp0PDu3l3qEq3j/7QHpGHOE83HmrC6XCj6EmoSnQ9fQPjMyGb0vhjkX+bOqyVjM88g/yFbnsKPHH6pUE+cfh9ieQtYdKRRP6XCpmUekyFsY+JR/TlR480LzJ3lJfThqgQuHytntGcdYIOysl/zq7xklCuzIN+3oVw32+5JfcRmy/PoQZq+/97fN3Eig3maj8/Xfm5N7ip6oiZhaUUHZsMzGvK1ixq06FI16bjyhAfDL8KcoVI2ru18s+UFwPWZEQgEZvs6cwjvf94YOIzbiG3lJMgRVWii0gqJO8seQqy8WB9La0JU7wdUvCfHfrJWYVwa5w6CFslB0aXbWM7yMquWeGIVlgj3JaE+Qd5UmfeDpHbyDG5yrNPb6Eqixo9x0MuAzBXfny7ru0PTWVrbfexujpUgSKn8Brd3vwZgGMP/Ouu0xx5m5GVD/K2dBAlibJ+wVg9uvGoZQnmXN0NxfN8jE/yvM8C3p1o23EDtbknYWh92XXfDxtFOeyBvF0i7cIMayXHTuiOoptYXDXDBfyB4LKR5KQbfI+eMY8vMAZOQS6boehD+hxdxYv5knWBjLQEwA2VAIsDYTshjmIJhek/u8dWN63NRnpq+gi/D1YYAWwDjgBbmv4/WYDR602Fsz9jBXLlyMIkFzcnM/+/Ia7RW0QkDMOG5Y9z+gWk9EEbidId16+sXmwhtftYxjkbWJOeAr+hj1ufVnqeZG5AVBoA1HhsqaoSKGD5wXuFT3Cr3deQhAVkuRlZbrjayQJ2DlHdlMcvUh2aBS/0rfeKgwWLz67cQgaOcmnFZyho/ccgjzSaBn6O+1zYyXp0ozt0hgaNoDDfllklCXRNWY9QXnznM5sR89dghUSC+mzC4ulPEBNZO8lsWg9Osdt8BD8JMBV7BPQ5zj0Oc4D7du1H3dlN+25+wrfFkmoT6sgSM+2o/DmWj9Rp39Tx1gsvsRzvhnEKOHxmD2MCRtaVwT3iiUn/mWuGkMIKutHQtkwfCr3Qrnj2TaXgamGZZFPtP66G1u28PJ8nvYv564Z8o3+uIZGqaWjSst3jXYSq5Ovd8dEjeGtFu7rNvKOw+lJxHvXKSHZXUCyDwWlOMVhzRUWHDjDKw3W0Ub1jryx4gEDdTmEO/YUFruLRJB/K8rs0n7ET5vt5udst9upVmaxqQL6b78iMbkcoRKrmd5pNAPrfyE9iz1/lySyx9nBK75WpeWHEGinkzM/ufMFO5t9Q6+SZ/CuborZZoKmH0BgJ9jflerwSVyp8kYEPJVlYDXJDm9QtIMZftAiDTaYe7ndE1EQ6KqDDWHgbbosa9MeH8mRSPgqGC6HBVBP58GUqVM5e+EC1a078qqoZIQAbyhEjLEGRAH2WK3/aB01L3UQH5fA677QVuci9Xr/O1ZEXidGCc8n/ErjoGN1v2dlKl3Cv6NrxUiii0fRofAVFDaHqo2xEM6/TpKlTknE1+4lPa82KxgKUCgUMjafl8JX9tUqx/tjtcPZ6lBZMc/cYQuP3pHmlyIb3IvYLUl/A9xcRtgxT+L9pHVCgt9ZuLkEDDnw4Ac4/wbpF2fztOUsKoWV8HBo5lQnktjt0sNnf4gEN0Bj0zMIQDcPKzqzk0JCzOMcTzjD8tCnOdy0IVmeTuCPgHbszPyYBxXSHtGGBe59DeV3QBCwqQIoqg6nUnubZSVw3Nqi7ti7XzOoTEHzkLqciXhnGaSslUABjaaTZXAHXaD2gW67IWMbP/ZX8HSLtzmRPgqTZwv554alMa1Auv+Tj2wgpUVKXdvxMbBOxGZzWgiby2ql7QUBXmmwjm3hUjFBMJcwZXzLWnufd/0bMk4XzHTv+nj2fJ3UT21MGWCrvdedojczs8tjpJU2weDhBFwSlCSrX+OPKvi5HKZlJcgVEPxacNc+EIMNns2Dk3antYpSB9Ej0ViyeDX6NO+3f4kAhct+TONPugWmFEC7S/0QnAuJ/AVgwRFWPKiwqbAj5RVJ2wDVWVJeOX0rOlUF4cVSgfCQsT20Wi6x0JvOoShyNotPrKfS5MuD4pYSuNUp7voMZ0MFLMtswK5s97WfIEjKVKEK8FBUurVXK+uTXtaIII80AsXzTHnjVc5euIClQ0deEdQUIMl4/iNgIlBqgQp7XbHPr7IDIyoO8Wn/T9l4fSM770js/JoxP9N/PV8lT2b/g/0PPe//y/GwGsb/0e//v/Kt/8V/8f9HqNVyucKupq5cjpIe4X4eQNpmMGRL2vX3vqaDTx17SBSRUEVdtkiJCm0wz+0tYF6xZIqsssrZA5oLr5MfDyO84On4D6UFTE3YrGgr7xPuKBSabCFufQ3XBEha5n6wd+dHtcUWV/bK72Y4aqtDWokiBHqk0yzkIH3rrUJVKS/klFqzSQ/8kREXJ/Bjmgt7ILQ3x4onU26DwAdwPdAJ+VF6A7YE8pYfdJWDqGqjMvYpuqZLGyLB7j402LDiI8J4PUTo5Ew2QYAbBZ3ZeGMm3VtHsNZdWVIWU4AZQGegp1L5ULbWjPnzmTJ1Ko3LRjErLItHPaGLzmXSTl7L08bpRCihS04pVdFT5e12O43UcLpa8vly9ewT736Jrb4kSdPVV061Uyhg+v7jLDqxgR6xa1BU3Ze1/5q7ni7pcM4IhyrDJQPomrixhPVdmtPN3IZxl54iPVZeHFUIJjwEaKqR7qesmAz0je9HWJGEIHaj61emEHmtDcroxbyaBwZVAq7RUPUtwxKXYzDk8PWXfy0x9m8T3sGeVgY66uJW5BsaWi3jufRYEtQwMm4rqmonadzKVDg+mk5Rm8mpqEdaqTxpDFKCMtcKX5dCusGlqKELZQVRLBsJZZ/A4s9SZMVAo82Xvfee405hW8mMufRGrZyDKMKpjMc4lSFt8KsCHoUum2s93dw2p+YKSe/eIiHgaxCIV41QKrj4riS+xqQrn7CxAj7JiXNDwysdSfurRkiv9oVQ+UbP7tuCX65/wJY7qxjRezy9FQrm8/cSCz6iyPp164jyzCNC0PKyDzRXyhmFusIdvOsPYQo4Fw0eacvl7QoJymcRreyyz5c8RJxiU+AfTPeD1688S4mPCyo4rC/zizX08YBvwktR25yQqqKClwdvxFo+movR8E6DbQSXyBHcoba9TO80hvUnSugiav+xZ9ratWsw2BQcq4awZEjxkBcolbYK/BRQTwXjfCugyokVUZ2Lv+UuGgEGeECCabD0TDqFHRsbfJfxdgHkC3I2oumRT1Hclfx57IJJnkStus+09s8xzMtKcix0M8ptylNidjLj5HdEFTxNePkwCSyStgmuzYabS4gTDwMQpbbTW/0opK6rOzjxdT7K6Ff7T4ULWEGIeZx3bg3BJ+JrugXslvsaIBmA97ucwxsZ22gqujy7gFJUUnDjBzxDn+IHldat/WGxRqdj4ChJWsdiU5NbGe/mOap08gFqaJZLzHJ1Nq+GhxKgy+DXxzW0qhhe15ayFuwW9hX1pHsGvJSH5K/1kGirgW76XLe/K+x2IpT8jwXMfysnPWCEdM1JgSf4eUQQqjS5hNRBz73US4FxOTAx3UUpoDqbVsodNHK8Zs73R+q0hp/LSnk1H3plKLBrw+XtmXtYEnuJK8mvsvP2G7ImQYDOURt5JFxKzLruJQQB2t8+RGBpTwRrPXAuJOYeZkT9D/DTZdd+FoA+J2HAZQIy3mXzKB1BHqkPTXyonSr9eoXcjxSlJ+UKPUZHnkCpdId4HvG8ytQ//qQoawk9FPZ/PBbs+2M1IZ7J1PO/gEK0SSyQmudEpedG9QQuVatIDf6CB2YXqcPoEZzOqRvTlM4XZreAqRiVwkijoGP0iFxZOw/URHKjubRKg+kFcL+oN6VlZf8KMJNfoaXc5I9GWQ2Zu6FU8mESbFWEeCajU0prUUEAOm+G9t8DNhpWTcNHK1G5a7sc1KlWYn5Ggp4fGpzmcjSMVMnlfmm5hIh74VTYIFJlkxeqlR7Q4GWWVX/OCyGp9Au5CFkuLCXfxlzM6YfFpnF7DpSCiYEJX9Is5CA3S/rC6AqIdxRQBQEfawJWpKTMUVN32bG6tI8Z3nAFfzYYyJ8NBoBgh0aOBHDOfrTkIgpW7ha25Xjxu+DpNC4rPVlZ3ZyfrHdRAQqXpDM2MzoBThgk/zq72omJq/CgwLcdJ4ypZIhZfHfY8rfszBXUSR8dQT4vH7AYJQb6B3NrQXsT775Jn0xItoBJlBe21OWnmeYjkm2BNQVhoHPZPzSbw5b8eiwqhkVZj+Eao1LnsTYU9j3YR5VZ/mxSdJ5nmy7i+O0PWHQjX1ZUEwTALo07nbUQhFx+L83rHcZl1BWBZb+zoMJk98ViU9M0+DA9vV+BivvSusGvGfQ+wqXMeeR4HSMl+HPyDVnuN9EukmqBq2Uxcsl7gJNP8F5wGarKZtgsfm6HPkw+ribUiioi9TfxUhdL1+jsL6mAtVfm88u193ksaSmBRnnxKl/Vm/NGye/74ys/S4yuwHaSOkzWXvp6jMZPm82g+p/TxTZUWqdffg9urYCzLxGsLOS0QbI2sPm2kHcschhJqfBjGVys1iC4Fsb8mpFc0gKxaDmTbH8NFsgB3uEfoPEdCfz8/Bzyq2K4md+JrwY1Qp86R/bZu74rmVJ1lZEVl2h7z4UNq/KmhFB0ArweXIimymUcAGIVlbTVQhM1KFxBUzn7GRPwLNsCFvJ+RgKCKMCuRNgejYCVece2M+fobm4UdMbsLWelNrB/ylPNJWUWt7kmdjw/FV/jfnErrHYVZjFIYsRefEsq+JlLa9fx3WN+JiB7bt2xaj+2K3KYkV+TxLfKmX9N51DY6S5dzlp46c51OofuAUs5dHQk07N+J870MUmBJxjecBmDKjS1MsY1j5raHIzC6onCliHJuTnUCKKqPmHdY/7E+koFHFPSAujpAObFjqNvZhdOGmC55Szf570HTgync1nn2KF7lFsR7xDkkUZCxgBIdoBGf2spSegC/eqt4otBTaDsluyWhWZt4A1fCFdAQmV3+f0sv8fbgV4MsHSlvU86ngr5SulQwWy+yJb2laPZJEnjXp4JFffgwXeE6uqAxpWWUtmxD81l5h2T/geYFcEUW5R8fmkOR80uQOm4p+hcOIAbJgmA61t2TNZsNhQzNyeBXa0FVg2LQnlWLsRuf7CGLREGQhWgschnZYVopXP0Jhr4n31oHx/zmU1BPLyaBxsresgbg7uwK3kmqQFrGdLyWaxFjv18+ACo/zKiCGWeF0lSw4+NR8INeaH0SvRkHpWWWO5eXOX3eFNxHC8RRmWD0VP+XtjjxvNtGYQqoJlXIWG3+rFi2TLatGqFx/mTfG6z8KsdPrXayLkn0N72z9dRm44aOVwN8wKgh+6E/EP5J3jcL4cUCzROhb33Xqi7b+GDmXyihHtX1jEwfzY7+j6Kd4ajeK+LgN5HuOYl7XWDSwaxMqIcNvvCqSfg1yCwVKERpURC12tXaarrLwGeHe+NWimNlcU2mJLVD+rXsb6U+hgqi+r2RLLwjMbs3x2DRdrfHkp7FoY+kEC2Td6D8EFEFJ9gdDa84QuzO+2XgVVV9hJG+eXRVA1fdRtL2F25JQMp6/k02MbAOwdZ03I1flnzZc0qJ1++L1s+LxXLAHySOFX8BtUFQ4ksmECwvTl0/Bm6SIDkqsQlvHPgGIX6wwDcr3IqvHrXI0c9ilKjtL7WKBxzfoNXaz/i/EiFqs5JYFIEiBgI9V8BIK1UksR1e/Y9o+jk8y6JmXNRWYJQqJz2BwFtIOpRSaYdeKPd07DJR/oNDQUIAvyS/Civ5El74/SuJ6CFdE/sDqSc1abEaldgtWug/Y9Q79naPp/LGsjHp78ntzJW3i9RyS3PT/mpJi0ruKzt4p/ioPg1hY4/J3jI3xmuziU6eTANA0+y8PhGSpyZfwDGIh716U+JFaoKhiC6MJu7+M2jT/y3fNB1EElKuRqTp7KQph6leIuwphwYY4bwgWAzQdE5BsYsoL6HND5etbSEBq84vrOA4OTnSQo8iSDY8VCVIrjM+yrHw3iiSsudKhfwZFUmx898iEmVQ0wKbM59Hde47/cZi05sZHjDZfSxtmbFknm0adUK1amTfGE3EQhMdDvq4fEMoBLAZDW5FTyVjvWbxaFKUfPb5ev/YGv2J4zeNJqh64fy21339cP/y+Faw/g/Gf8V+/6L/xVhs9m4evUqNlvdoF+hbEpm3gD8yzuz2j8arrwPLZfCgEvQ6xDrC47WfjbRugCSnVJ7Rx/jg/YDecIwnBsx4FEo95uxh/TiqxKp8LDw8g7QOSXfrJXEnhjMnABIVIG/ohDXOOU/j3bp0PdCA+asz/jHiPX8/BwmtZzKvJ69eK3t86gLnRg7liraGJ+nR4MlXAv4DpPdx+1cZrGs9r9lCUUnRLSnCAXxSObVztfsVY9jBqlw2tHPhRUIKHcn8Y6/JAm3K2WWrM15ou7VthuHLP+zTGdHJG+A5i9PZZpXEuN1QcwP9KXblImkZmQwZapUuAu1XqGZBt7zh/HeLr4/1mp8BFMtZ8WNMq7yIjgZJheA0uLjvtDxbsCpzMeZvu8Yn5ySy7GKIlSY/AnxTGFqh6cQi+RJw0piSLbAJyUwNa+jPIHhFcvN4j40uP8z/ve+c0P96i5NpDIBlhRD80vPuTEm1Mmfsij2PNlx8Jhym7zP2mC+szRmWTF8UQpWtUtiGGioWs3whstpqmlJd+VfJwn+VcJbq2VCZwNddRIjMVaZ6vaZctVddlVC9K1gTMFOixXPGOj0CyfTR+ChKsVDVep2bO+0T9gWBi/mwY2q1vJGlTfbjYEsK5YYAoILc8FAIF+c/YZTGY+RZ+8EI/Kg3jPAw5PftVGdTb3KWTQOOlrXlrkTdsRDtiQH1d4jmxYa6JkJNz3krBqAplWvM/icnZ5XH7g9f4FV92ipgWZp0OvP19yOrYkbBZ3xilmK0W5nF3+f1NljtfLO228ztVkiS6Lv0SgV1pkmyz5X1WAxje55U2iTktLmkMF1jZVpjGIyb1mHESBCkqagFiVcExtNSfxaAddjXuNayWlZmy56ONqA4ySoYIKvAbXgIrNotyMINrSOe5HmJy8WFqgHsODYZu7fO8bTln/uE7d7y0Z6Xe/KBw7pfrVLgbKoyS+0T4efyiHgVoS0Ea+JO5+TeKcdcUpopYEOykMybX/K76H+1YvHvaR3skqUF5sVCok9PMUXFkbfkMt46hOZfWQXN24tpqoyAb1Wfqwogr66Ke0yPuVxukhFxubzoPFMuDSd+sI2QJLo+aL8CQkheOVDiTkBbEn8gdw4uB4NepfkB4Ft+dh6jr7Ru3i+/lS3Yh/JP9E+6Dwf9xjO0Fg5kh7AK3slz7acii5wDkdt5n9lrt45egPdY38iyCMNhbWk7kN2G5HmbbTXwkgv6BLi4l+nCaTYWh+bXcGJ9JGUqJzYKV4JEPskn1WVcKQaVpYCLnKZZO1lV3B9NoXBqnh3VnPDIx1ZGQxFjeG7hzDkauLfykk/OeFZWofvZnzTDzibOQi7j9xPRqnUYkWS58u2ukhWBnVigyKD3ZUww0eHt8nF0wrwVEjzeVTB0+7zVPFF+ies4lTGY+y5J/fAFEV4Ydddvru4lI0jvVDdXyFrT/L4hSdDbqK5doBvUq7WesQBkPAcH565RFZ5A1qE7qN+/pNQehOCOoBfM2we9biS24uGgad4seUzEojCKYYVzefnUHhZVZ/xerm8HuZy3koaTRctzPIHP9GFoYnkw2mxaVh7OJgnTQ+h0T0kJlRXc+bQOj4/u4qxW4qx2NTSWsbxvqANYn/lj+wsjKOjFhLV8qIGjd/li4w++IsS60Zf6uQ3Gdgehj3gcMoTdI3+hYmNXwJLhfx+CyLZVvg0J4r8gvH46PX/CjBjsMWRXtaISls4jDXDIx8DoCv+ndVD42kbIaFlBUG6Fmxm0DfifMAJCqrkIARsFgktD5yt7ME+Sy4aAZSuyRGFlkoEhnlBehwo8tz9E0EqegBS0fMhUd//TzqGfi8DJylEEy+1eYUesWsfKtnTzctKN6MkTeqqrFDZYiOv3hhOc7W0JtUmfwpqX4ktpvRkmDUUX20ONwo6c6RknlvhP8d8h8ZqMNWHDuZN8i++voCqBEmWXvpupzZtILcaLeCHMpgUUEzag31/yc78t8WWnJwcWlhfwNMcQLQStC4FEU3RPpYG27hugrUXttWCjpyjQL+fI9WQXOae0Lnu14v7ZljlX0FR6nZ5Y/hgKnse4dXFScxbJL8eUYRJjVawPBB8H8A2hXwtU6HtwNHcjsQo4Xt1L4T8uv0TQR3YZrzA6YxHOZXxKL+WHpRksPd1gfUqSN3I0w2fYnzYBQ4kXsa3dLe8X+nbWBh/kkglmL3+dKfMtvuGTTdm8N6hA3xyRg6iAPC5P5l1j/nzfKvXiVfJgXNp1V14cmseB5OfIlJzXOadU1PUMds0PLM9hQe+ctDTVd+NfHvvcQDuectVGyi9Sj3VJjxUZUT7XCeU3wE7pK6HvMOQtokNcZ/xqmMb5grEAeij+oDPS+GZy5Ok59o52n7NZ3+uxmIt/VuwwGqgB/8Mjd9drebnnySmcJVZz5GUsVi85IzqEq+hHHI8klZcpJejHuXtK1/yawXEJMMl7RNu3zO6vCu/V8LVGNAYb8sbQ3owJhsuG6FalSEv0tZ6GtoBd3/PK6pPmHNkF17qIgZEzoIcpz25NpASe2OMVg+u5PbkcthZCGgL5XchcQqo9ISb1qFXF/DVuS/JbnZOdm6FAuyihSQ1vNm1L2Kuk4+wLgS7ZwI2u4L2kdt4qenjEtgP4P53cLg/TYxT6Bm3hryKWAqU/SQGKHUWAd/HpGJVVBJt2Qp7W0uyboBVFcyD4paYrRKQyq5vXOsljkpPjlWPwQ7HzJXsqoitk+Isu02H2+8xxgsqtLfJM3uQHr4aIodK7TFjIFJirV3P78K6qx+6sUSSmyxlXA5kxsPrLWbJb3bRBYKTp3LZ8w88Hli4Yxwja75aMYEdxdL57IJV2hPlHICYsTCqlBvF/cEuPe+JPnJW0UML82dflP4HWMwnSG2ZwLKIx/iuVP47ISpAVBGokAC4nlXXZM3KXfVYHSyBbr/Pj5Ck85wPr7jLIA8lY7K+oX3wSbnPl8KTIettfHzmBxSiBa7OdhRFpCi3R3DZCFsq4KrJBUAb0oNtyR9htNv5KBD8K45KBV+7Deq/TNDpesz0kwoeq0sBP3nxoV3j58nM784bvtDA1fLDWEgXMRUPwTH/auUSoGLDaSwsltjpuwqjWXIg+i+Venxtdp7nn8WE6mrKLkl96ZEB6yvlzwBtviD+ZmvyrXDDBKXG4LoiuUKNye6D1a6i0uTLztuvYfF1SEkqdRDcFYMjL2EXzFzL7wn1Jkms6YQXsFmMKGzSxGxVVKFSmGB7DPz5AlDH7AN3NSeFpYDWpWNIyH6XgQ+2ElXwVp1sePQoKlvvJqtcUhAwWPUSgEOhlUBRAW2oVAdyqFpS4GhYLl+HeykyWRl3k8e8IMNqw6xyAbDlHqRn0A/4GJNYee4TqkOcgIyVaYSUbcfhZEKJxUe2dxFF6V3KCFzDJeVXEtvLTypCORccd4TBZE8nUHtob676bCSlpJnjflgkWUiVr7Qe/70drQPW1H68ufYTyYeyZg3X9AOeP2znQk4/fno0iIAceYGS8vt08GyMTbAQGrkKrcFpT5U0DbpsweYACTUJPuJ03F1EEe4kT5aKXoBaoZSpmM0vqrOMMR5aQkzbZ1jx/f7a+5FW2oTTGcP5enADQrMmy7pVM3Y8p4cvIu64ge7UDvDM18HQEic1AFOx5IUIfHl2JSfSR6H0cgFU3VjAr8F7eSbzZ5ob28vVZYBufu/TK+4HmoUcxEe4I2tr6LmZ/Y2P064GGysqJQCvQgMdf+ZGcV+qHctvpeAE8NnfDYDz2QOI97vIhpG+eGZ8LDt3j5z3+TYY7ILFfS2dtYfO9+fQyvG94kPKNzXH/Jk5hOl7B7Lgw4UykomWOuuKv7OKAQmYGKgSGR3fw00h4q+KfTZBWk8UG4rZeWcnD4rd93z/r8bDahj/J+O/Yt9/8b82bnt/wZcndtHx9jE23f0O2n4joZT9moN3PUpsSbWfbWidD5emw9lXoUCa4JsGHSKsujU/XX8Tlb8LDb7eM7ycD/fMsMuQLS1gakLUYGk0kx2VcC0GRj9ESsyGNAjeO1dJN+U/83Pprlazft1qTqSNYvutyXxwaC/mUCeGgN1KonUXHbQQ75EH6h3yk9z7huciOhGrhKGeEOAko0BYP+i+h5hkaeF22Ygc4Q8EeQTSLG8OXwfqmBLtIi0EVCr9yLeCzeqF1cVXQxSha8x6FvXuRL44moWLlzJEo2EPD59Q9iDJdC5asoTZcxcxZPBOrn2i5cYnJcx8sSehoXUbly2++xmUBa/kw1dlcj8iEp7nSNRRHlhgjp8SbdFeXOMR74EsCICTIWHuSdTwfqw4+ws3CjpTbZHLmioUEOp1nyqznrlHt2MPlqP9PFVFNK6WZLaSlC4IsOhRfH1rJ9kV9Qn0SEdhzpM124O68mtOMyx2sNg00mLBKZS52xgTlMI9M5TbXfydNAFstiVx3JHLeRhK8ZDxJ947eIADZ0uZYLK6f8ARzgnvv5v8zwCHzWZSG89heYlkZ9Fb84f8ZPe/44NAA0Y7pCvyEFVO74zaF2IeJ62sMXN79OGzAS5IK0Ctr88Dg3Strgv8nJwcdMc9WD8fpr7pyay3mvDR3Lnk5OS43QPXRZIgwEutX+KbwZLEmVfJDjj7ClTngiGP2IrZNA4+WncefUNIehMccnjLQ/Yyyx8aZM6miY/cQ4LiS3QLX8nA+l+wuHdHhBK5BEvb5CWsCtTTTA2fNtoMOXJ5A0XBQRb26kKT4MNcv7aKnmr1Px4rpm5PYoOhmGwrVAtyZhW6MMIv3aTMBksKVdiDe9e1WQ34WU4RqilhsCf8GLVZkk5zip+sj3CkGlYEQqDxpFsfVKKaRVn1iLrUEZuHU7LSboP1ImvDqjDZ4bdKuGiX/yBGdQInM0ZgsZT8O5+4slIaZi5k8IMdzCv8mnoq+QJd65DDtAHVdpu8+B7ai6r6cymwwZISmFz2Mfg4FdiVnlyw+3PPoRrkKoMnViTTRQtDPGFUQDbOAC1R68f+khBOao4x8sILFIcslR3rW7GNeq07c/+RCD7o1Ase/ChJyF6fBz5NuKGcCUCZDU6YWkrFhWtzoDoTbq2gnq6QMpuUInOT5wOM6mwWFcNz10bL2cWA/dRTdGnyBvPu9eNk1ni3Y1Vlp+gWux5PXSDT3vx35upjm8xmWocn+W5YDH55TgUmu5VWZU/xui+86ANDIuXMAhq8wuaSYxQbwlh6cj2pnnUSWQR3hg4/YEZBxMNkugHKbjPI5y4/lsHs+33cmkuC+7CnEh40g6M2y19eT42c9GD+BzlptYqFixcTERFKvO8lGgcfZ+2Vj7AHdZN91s9+izYaCFFAgMKlAC6qUKsjGH7vAPODq9EbjsrbzeWs1nzK3Rh4I/o4CpsLGKLhVMZsLq7d8DuHIIDR6km5KYDr+V3dCiKtvL5gbJPZUjdcHx9dKFlVzTFZdUTpbxJU+ZNMxcAQMYk5R3fhqS6hR/T3UC0va+mUCjzNoXwWc5dGKhe5zMxdJF3qzjv+8GEA+ClT3Pqut14h1vcyVUbDvxoLKivr7o8oAgd6wuE6Wek71j9ID/qWH0PgZd9tbud4oP8BX4WkvOBdedqtHWDnndeZeXC/lFRxiqji47ytbEVI6WDsdhg2bhxrtf+MFfu9Wodv+Li6fjuF1bMBW26+RUZZQ8Axj50YC38+D5k7qNR1xGTVyY/doIHjEsN5QX4/FhdDw1TYYnVZJ1Vl8IjWQrIZVhQD3k5y0+Yy2JXEHH9pfHtQ7e0m68yld/l1tJrHkpbyRMIzYKpLotpFHTMPHmDLzbfx02RCxk6ozq5tn9y0N7OibrA2BLqqXMAKPo04qj1Itd2xJtWGSMkahQ6iR3NffBmjxeOh9wtLFW9r1jNRDz+XQYEYK2/3a84f5hjGesPFkEhUZjm4rSZhUYqRSuNfe5b+22LLt6tX08LnBK/6aEiNg9Y2eUHOFPk03e9FcN8sJXXc4tgI5oRJhWsve5hbs6rFfPZWwhhvUBuy5Y3aQJptfJqhK+uxYedgmR+gKEK7kKMMc+AQwl0koQUBBLuKcCVMjDkgK5rVHA9QWB1JpqWHtKbLPy4lFU88TofQNXwRYiVQAaLdRfWh8E8mBuazLhRudvjWrYBO9Ciu5LrL29WG2p9yUwBDEj8jUimXLHde7w3xHyOt7Zz6/GSzd3m6xdvkV8VgVcn9N9VKBeFFY+nvAc8mLJSPb03n8L3RSmZ5Il+d+4qtaqM0FjSfLz2forQIWBooSfEryl1UR8rvM9DrLPEqEOyumTunj5kb/C1YYDvw5N+0O8eE6mr2bFpHk6AjLOjVjdMZwzGFjpR9RuFQN+mohT7e2W7nsJgVWIA0C9iVAW7tw6Kf4Wg1zMpXYdO4vDXe9dhQAc/7wCeNNku/TZ+T0GEtokJFx6jNDKz/JTvHivjffUZ2aLmyObcL2+Ojyad/xGzIdSr2WY14KvNROnxhRRHQBkPPAxLrJ2s3jUvHE+Z9j/yqaOzOazubhWj7L7TXSuyo/sE3ECpT6tpNpVCdQpHPb/xWFMU3N3+RCgRQK5NrQ8nOO69xMmMEF/U7QF+/th8hnsnEaqX5Po1EaL6g9vhi/VjeO3SAzPJEAElStabQba4gSmWqlQ+VARitBryrkwlWgkVZyhW/7yn2nwT+j0jtzedB0/fRauFGfhfWX5sl9zgELN71uW6Czwp1XMjuL/+dQrpzr+FmjjiKvgoXxRtBqPP3tWOFYWnQ96SUM1DpQVDS91I+Pa7eJczj4Z7cNecBoNlH0ExiW/Ywj+DnaEnS1tU3FEMBzZUlZFgkT8iiiOfk7bHj2etQr3svIwGh/gvy9uYfMWarmUStkVndB0oebTX9UgrUeHmrFUa4OgvS64ADp0yv0StT8qBz6xdQpr5Nus1Ci1TIDX4WMrZJBUxjPnalD9V2yLDAK7nektSvUygFO3pR4OMgaIHLeiOgDc+rX2V5CSRkv4tGKWcA1zABVwZDnC2HOV/99pfg8VL+nQ+1xpTLJv9wzhkh3x4h/4BCS7VdhRIIUkiMsto52FxOjP4Cek0+xYYwvrnwKbbQQXXH2m1oHWwpu2DlYNpz0G41xE+EtitB7YNakNYyVrEShWiFuAlS7o46Zl+wAl4PPCP57dXcj+Mj+bx/axpmziPG0oDQ4qVQULdHdS6ceakKJXClzSK9d41n8lvDTyiwSuDdnVFTZZds1UQx+/jP/FIOI7IhL36l/J60XEqzuy1Y3zSM73MbYAlyYkYX/kn99Jfp4FgKvnv/LcmyASD3CK/Wa0bHUGlMU6KR1l0O6V1NwS5GNZpPs/QV1FcpaesCAHJe+xiseoh5HBpOkbzfq7PRKuvm0+umlyQGnaiGDR5wYiyiCFabirTSxlhdC5h7mtJdnM7d8NnMazWfgHsuFgNIt07ARqiXAwTh0wT0DRAEUNgkFqVOgLXzX2LBuzMeqmJ2wGKQFBBcfLdtdgX3ih7BopYXuVuUP8bbfpKV0gjfQjkALWMnnWzvEKyAZ/QQZHXKqwhK8GlCYeg7XM2TcnbeLik0grtzS3ibFxN/4KdBrRFcwHE/5F5g+em1jNxUzXnk4Mkcc1tW3nye+yaIEJUSY7kG/BY7ji9v/s6fHpLk6nD1YtjXWWpzjN2zug/Ebhf4/d6zWL2by84dqIC46nZsjirimRiX3ymoM9diXiPFDBO8obHOXT0hpPJ7RjWaz5nUJqz4+Xc3ixQ/pHze/2QVswIJmBjkJfBU8ydqn7/b4e/ztT6Yhccl1byaYl/tb+ni226xPWR9+1/8X4n/in3/xf/acJ7U75b1guAudX8wV+ChqJOaOKL5ExrNgLtfSHIX7ST0Y3D991iSmSShVJ1C4cSKS9e6oFQVGgyN3mFHJcwrgiulLpJdgG/lPiZ4g+Z2JhNN/5y98tuWdRxPH83qiyu4mNMPu2es0wV78ZP+Ii/mSZ4b40JdFsKWSnyV5WgF2B4O4aVOEhnaQAjtQ1vDO2wshz5pOmmz5BTqax9yudMHbL+yiHUXP3Hr36F6H/BhIfS1J+GjqpPIy8nJYdPGuXy9bipDZ53miy8GU1lVRauuXRkKHEA+oewHhgId+/ZlytSpKBQQqMsgyDOddddmQcRQ2ffWMBRPG+C2We4fBlLCxEeE9wMtaIpdClA2K321uQzzgiiPXLl8iyOsVjuxvpdrzd9rQqGA6Z1GM7n9RP7MHIroJV+MDA0Yx5nGZ5ikh4l+7j4jNd+1ot8jRKbIk2f2hBeZfmcQoUpo6ZHnhlo6ET2N8BQ7XTJgv8Vlo4aU9H/KG65Fg7fVnd1SQT0yyxPJKPX728V/TcK7N38/+fdWKFi4eDHNAt/HXtKZcTlw0uIipZG2iWm+CvQiNFSB4CKNWxOnM4ZzJHWce0Obz5mTE8i6UGjmVZdoqkGKBe88wdcF8GtVJR9k3eLY/PnEREayYvlytIpy3uwwjr7xq/ESMyF1Y61MoyBIKOdyUwACNrSVp+Hul9JiW5/IpfCr7L37Yu1n8W8JLZfUsi2XF3fl61J4LW4/wUYXebTsPxib8BJBwdsI8kxBcElyic0+4qfcdlyOgf5BN+sQw44QLOWE6++woFcPqpLn/WPvpwnV1Rw8n8NP5dC2ujvtvV3GL2sJAbpMpO2tKC+A6htwMLCML9M7cMEIKws7SQVOp1CJSsKUMNkPgixy+Vruf8c4++s8dv8gLW6eQKlyXkYIpPu0Y585jZZp8EIeqNTyolzNe+HnYf13PnF6PeOjDvJszEHebf8C+mq576Ou5ABvm8fySNEzvFS9QM5WDOmOsvl72Eq6YbIJJHqMkxcDdWEsVXWl1AZ/RkGQ3QUZfudTjkbB07nQ5txo9E7KbIIARo8HpAWtItd3l1uxOSh/MdNjzpNmL2P+/RGS5IfdDq1WQNtvsGgSne6eAPHPwKDr0oYq9xBGm4L6qdAkDar85AUmbi7jQSyUWGGtcMyt0p3/yDe8U3WX9+y/c7okCdcoT/qWL89+ycZRXrw+pqmboXeNtHJPlVxaWRDg0zPfMffodrbemorRy6kILii46reOj4vhtXx4Mq2l2/f+XXGeHfH8FnmWjDi4GQOUu7DMG7xCh5uv8mER7Mxz94EL6LGN3Ze+oE3OLnr2+vsCZkfAIIp0EUW3a+6t1kiG8rPeqp2ntt6axuObS0gtbepWfGheNZUd4fBrGPwYtU3eaKnE136JiupYpvx+lkJvl8KroCTFHkaCGl6POYNodUmGK3VUmn15q+MYXmv7rKxJFCHW9wpK0cTsI3uwRcrnmv1lK5l15DdifK7S0X+ZfAyyGvHWFKEQzOy68yp/xlVDYN1vWbPG2nd/Ek//Vl7HSnCEfsB5Vv+ezdyjuzhbMUPeZ5/GlMa8yfwiSQaqSOyIazSvGsfbHcegUXv+q7HAy8uHen4X6BD5K0rRDAnPSYkkAFMxg70mM9Zb8undXNFTfoKrc5gfdY40M0QnQ0nktLo2Qz7c/44Yn6tklidKBQgXBnFI5kYWxV1gb9Ip2oT+xLPPPcch8z9jxR61GJk55D5R+huoFVWQva/2+bZ5N+aHS4u5XyyxAkQRSHWoDfi1Qimaatkxte9M9OOSxyd1xSuF1YtWPi5z8+1POBCZS5YF5j3ojiBb7wogiNiAdeVwtDRUYhM6h74BF7L7s/POa3x9e5dM6UJUKLiS25OMsiTq+x6Do0Mhv04SbFPyx/xcGMITeqinks8loq0Sb0Hkjhl6ZYI1aizc+RJ2NQCvOK7pvqDS7Eef+G8ZE9xbLs1st9KAVDwFeCIXUnRdZOcmchg+DVOISJ9PpNKOUnQCPZnLiX/wCSO9IEIJPrq/ljP/t8WWbevWMTziJV4PzmNpMeSI8nHXrovkbLWWXh7wTrN33D1Jiy6QqLazMRTeq9cb12ga0pR9Jh2Ku1AZ7iL1ZLOQV/KAV33hqeoDst9RFOG1g9tJTIFhOiVN1XLGYYPiZ7nb4iDF6S8xYt+vCAlOSb+qDJool9d6IdWOff3OQmKdrPAzuRCXAiX+w+X9ajIT//uwpgx+KKOOweQS/RNW0jFyi9vfjfU/4KXdt5i0I5kbCjl4xFNZQOeojUR43+Z4+QJIqmMsCgK0idhN+8ithHgmo7bLffPqVcxkVPRuOmjhuYBcWREbUYHCRd4LQQDPOGkcbPMlG/I+ZUahJMWvMLkUzoovMDnwN1YpBjHPX+Xmg8mdLxmZtBCf8PGsFv8aLPBvE/hl5aVY7CqqLHosdpXb/NosqzX7ImC6H3wcdlHeWHyZyW3X0lAFcUrQ2l3mIaClcBeFAMuy62F3LfY54hENdNHnSc9JUAeIewJBFBiZtIixTWZzPG0kZm+5eofk32shtzKWRdeuy2TqSP2Fp/XBtAzdR4zPVSJKF0tgKM9oqEyB0N7cCNxKZnkDPFXFKG0ldcfazLSzjOUVvcihaog6MwYhwWnvfP0jAo/FkZk0kF8iJ3CpZDio/aS24O4w8AqH9cm11gOua5cPDv9B71vStWSL8dD4HalfuAMUtCd6SRJ4AOlb2B22jx46+DQIVka9Wef15teck22386njMra2Wk1ElpzRD6B6+GsktVkNeIrwelE1O3LkTDO0wWwqykWBJGvvpZQ/u2PDh/JD/VMAbLNPclT/RGlNXXQBb3UhamsAnsYEt2t86Nou6tFan8Qz9oGsLoNzUbDQd5X84Jx9fOF9lE5ayRNSofaUNQutP2Op457YBetDmeQgeQx/emY16OvW1grRTpPgw0Tpb0i+uAOvSHK9NffLsTG5HQPTfNfLT5i6kbeaDSZOCZdNYNWESeuN4O5w62NKO1xg/yXpWmLs3d36ozjSi9NNj9M5HQ4L8vwGgohCqWKcN5zpsAKvSrnKFBff4ocQeGCB5ftUdP8buel/a8vhrVNTZFGhE0DlPD8CVKbRUldJGy3kxcPgBp/V/baFfzK92SN0jKobq2vbTKWwXsET1gM0Tv0Mq2AgOUAuhQgQpZSUIE4n9iTXmg1F5yRfPepYW0EKeNL3ChScqjswehT7kqUib1Z5fa4lpEDDyVJb0Xm0t94iSi8B3YckLILtsVCVDhemwS8quqQswZoAFjsYVX6yPokaPSez+nPbMW2qXQqvqH24yiXsQLHXSXk+KbADKfE/csIxzMvnDjtKwYhFIwGOisQ7sCtJAqkB6pxNTGg+k4SCF3jrVzP3A50ALQV/0rD8pVopYEEAQnpIgFHfpjA8jbMldWNDrq2jJKMuiBDSC/KOsLKrQMPAk7x78DDlwfL9A0lvsd8gFXqXZsVR6VwATVkHZ19FJRrwVJfU/T2gDTnFZnbtnMv90/7EfQzKT2DWsu3sNJn/sQJCr7jv+XxgE9ZcXkBxiLzwGmTaTVu1iufz4InTl0DpVLErOksD2w/4i+D/AA6qnAC2Km8YdJXCyAX0jV/Nz48FEmh3ASxHDOKKYhEHUyaw9eY0N7BDgbUFeZWxdfdb1taSTTfepbJgME+rW8L+rrVMbpCPf3aMdWuwzr9Q1OAHDiRPIKu8Pp+fXYUlWE4SKGl3mlevj+UGOdw2uajS+CSRGzaMShv8GAod/NwlMoMrf+bRhktJT1v1UJLJMCQAf40k/V9ZxSwA5oow/MVZoAutvSazooxqMR+z45pcmX1WUb6mNLvuI/6L/2vxX7Hvv/hfG5GGVYxu/BHwkIThqSd4PqgO6VapTJISQo9mS6hlpRcvpbRkVSlUae8j8SbqQrj7NRtCYXUwpDbfIbGAnMLkWJTPKoIrJe6pgE7mA3weDMHWf7dRKy+XI/pl1yUIiEpPLMDyEthW5rKAbziF1dnZGDNe4b0rL5Pn67Q5sNsgYzubkhYyy8cH36q27h3wbcapzNGcSB9Zi4hxDq1SS30V7Ek6S7tgSTagphCTu3k+s4tz+LrExuyCFH6e/SGn9u3jBBJzzHlCOYxkNH/yjz+YPWsWSxbP5YdNLxA8KYQ3P1vHR/MX1zK2ACKsp+no2Au393BJkpbexKvoSwSgUSpUx7rILNrNzPc9z92iR3h87xF3dHjecV5o8RyfDWjBU83fkTWJoqRTv8PFI6kmHpgf5bv8CAZ5Ql8vOdqY/FM8GjuNMK+7/Hb3Jcr95VKGogixea8xy/ACPzXc6JbwqRSVFDjW3m4a/1VZLFce4A1fCaf4UC8loRRvdSEq1T+TGFPz95O/GsBmxUfxAL3SyPpySLO7+O61+5bXr69isCfcjAV1kVMxpjIVtgQzpslsNt14lzWX5Z4GAOml6Xh7JjPWG0JV0uZzxbJldZIlZpe+GeqQYsePf0K32PXUD/iTIMVZOPG4ZCbvuD8/Xl7EtD/+xI5IaeRMeCy3Vt7DqGtCmSmw9rOu8Vt1Ew5Ww6sxxwiolktIET2aXhkwybyf8FQjQmAbeXu9p/mt2AmlV0++0LZHDGPC1lz23nuOkmrhX40VVpM0Bh1pcoz21smydo87b7NhUDuaqOFUtBlVinyzVWLN5n7YQq6aYF1ZOznTA5jrcZaRXhD+AO55yTenNkMuNtNRMlskERu0D4XgVOAUBNZ4D+H98mt8HARpcdCuSC4UG2b4iZ8fC6BBgw78oJIS6f+TpMQanY6hY8fx/COTqed/gQXHNlPuKWd16VKW8WH9nUwU2rGs1UQolEu0iCJ0uH2AJ27dIErIrEus1HZdQCdIG0yFC+NCiB7F63mQb4XotBV4eDg1Fl/mSM8JvOsHz0QdQSj5SXZsfsynjMw1UGiDmbYtENAaLr8LF6aARzgeDg+G5mr43G82pG+SJGeUOui8iWHXF9eey1PtYriq0FFoxeGo4C6DaArrxzkH2K5aKHBrFxRKsisSOJ0xHEEbwpSpU2vN1Wf4J/G4Jpw3vZLwHTRTJq0sinC7sD1/Zg7lu4vLMPg7ARIEkWKvx/nTCDdNcMXgIjedd5R2nnPRa/J5qfVLRFQ7JXzK7tYW6Q9UgdmOG+sZUclp5efYgSq1vHheE11NXTk9YAgzhlbWFjC7azSMAhoBUUAIAp0EeHPGFNIyMx3X3JAx2jDe8kokbNR7pGZkMmXGPEAqfJltWgI90nmzwziEHDmwJNPrFT4ohPXlcLx8uLxDJdfoYWhJx6hfuVfUGovKhbGj1NExrYzQB9I85grEwZBHnO8lovQ3CfDIkDWJIrzb+VGmtH9Kuv0u00WRtTEPilvSMPAU/YLelKTPauLulyzvGECDAGlsFJRaSUrLEZriP3im5TS8NYWYbF5uXltqNdyKeJelfm/za5kL6MSvGTs0HTlhkNQE7M6bdUekaN9ky823ea53KmvV/2w7skano3PfcQys/wXvdhkhed81ersu4WOppr/nTdpp4JcKOGNsIT9B7kH6+KRhARSZr6FzZkJWJMOZSbQKc1cIqI22q3gjPQIP70sYdRcJDQ1l4aJ/xop9bfxInmz1A8GeKfiqM+FQX7j/LfBwRjrDUiQgRtktOqZpaBR4XP7ZTusgSXonJ/n/ylt+MFVTj0aKu/KThQ9kWUkCJTaILnhOnqRSecOg68wqgp46mBh6173AHj+Rj47t4EZ+F26XD5JksZz76YjUinbQ/gfJ68UR60tjWG29jvYerDdOlJ3W41Q3jjd0fW5aSP9/dVbtfBygyyRMcx6sTshhlTfves3iZUf9JtjV5xLp3dh8YwZPb8/A5hFb12A1EpK5gZ46eCQN8sJ6/SU7898WW0pLS1mT+woT8yy8VQBZarkagChYaZrzPo3V0DvkmFxqDmBYMkNzrJIcsFnu9wfAyQnM9zNjA0RX77u0TZQnwD0TDMuiln0G0u9ksWuxAdsiLcRULZEdalTGcre4OVVCGZe1p2V+f1Sk0FY5jcZBx3i04RJe8ROg8Kw0l7VcBkPuMu7Sz3xfVvNdLg+z0hNN8VBWl8HrKY1lzw8Avz3C+12HMKHZuwys7yL7jAQQstkV5FXGYtfI2XlB2jtM7/w4rcP3cMv4JETLwZczDp1h6u/nWD00ntjiN2VtYZUr6Rq+nyXF0OR2M/B2AhNWphOmOIxOWU6830XC7HslkI5nDDScBl712FZm5+MSydrAFuzCMg/qytTs0Zj1pxjXaIV7AT15DYMafE7XZm04bv9rsMC/TeDr9T7cKujInCO7eSTsNzQFO2WfMajrk2yW5MqnZcuZBRRfpJP1awZ6woM4SKx2kTYFelSs4GNtS1rd3+S+707bTE4cfFkKXY98LLWfeQ5OSuyAL899xUdHt7PoxCaqIuTFq3bGYWwe5YnFpiHf1EguS6lP4ob5ZfKrokkMOENM8XQouyOpJBzqC6KWEq/hVJj8md29P0FnnRjwoppz2s1suSfNj2prkGyOI6gr9wJGUOEgd6iVzrINCvBtilEhzRGtwvaSWDm1lrlcMz4V6KW9zuVSeaHG23SW8U3fJ8RT2t/ZgnvUym/i05hj5tEkW6DYCiVWX9laR+n0bjf2ziKg6CtIcRShLk6HC9MQRegRu5b5Pbu7jdlx5ydy3kG6K/W4hGuklSfTRQd7IiBSfdyl1Q6ONbAFA+SfhOLLknzt3kdoEbzH7Xy1t+zvgFzAGXEYX5XCRSOkWF3WQf6tOKJ9lvtmyRNSaS5yPwHSvvfH+GuSd6ZzVDygadAfFFRFse/BJNkzpFDYWNCrB2OazEFw/K41RVmAJPUfzPGX5PSLbS7jriGXRI8s9CL4iqDEBN4J0vias1+SaMROYzUsDbsIuYdkhxf4tGRDhZkTBii0u8xTlkqeDIsnpqI798tjsCtcJOALz9HXw4eWoj9XLlYy4W8Aof/KlkOnI77dDJbdfJrSeIHXvFze9ZtL2BN3BasdviiBB8Ut6wD2XvXYnTGPu4VtifC+zYzOj6HOd4wzCi1Ej8bm04YqzQOKvU/wRv1v4dI7kHsEToyD4ktM1C9HaZHusx0rxD0lsfuoK7LdMcEz6e9CAycLjAavsPG2lPez2lVYNTF1xfnSW6jvLSXCW5JevJzVVRqr1X4SiNcjGh9DGnurIFIJ3pYS2SWrVJDpL3mmT/UFn8Jf5PfEkEeiSmIIb2+7DM/UhXVtHhFUBDxKhkV6PseG7JRUDgBCuvNV+m02VjmBJ6Ifr2WAWpJm8cbeC5IkPS7vTfldIqpWEuop7XV6x38HB3pJHoeOcCY+yI7tvhN8JGBXyzBpv+KW42g2m2k5kgzqyupczOFOqhC5B+HuF6gVBiw2NT9eng8d1rDieBIxkZGU7prPewWlrCqBfkXQyf7PFRC++3Y1ZpuOEkMIFps7KOVouJHv9lTS/5wVH2tz+YUlvc0+fT4ZFdGU2yDPUiw/uOIBCRc8ebnNi6SWNAGVHDRQcx8OpzzBd5eWIrjcFJ2yFL0mn0fC9uAj3HQ7zsMUQ9t7OxEyV8Mjn9StGwrOMDRmBs0tEnj6kP1j6OeY2Qv+xDtjEeezBlDteO5dr1mhAAQbUwtgWo4LsBdQKVQU2WBkcgTnSp5za78b+B3T/jiDreCzh5JMhgJ3+QeS9MA9G3Tq05mMsozaftoFaQ2jVUrrN1dmX76PvABptv5X7Pv/SvxX7Psv/leEKIo0bdoU0WlQDjX8zNAGEgNtaTu9JHlUE2H9uGGsS6yr7XmSgaouVEqipm9GpahCL8I3redjy5Mv0oWic4z2lpJUe4vj5DKeNgveR4fwvr+jb7jrjQkJ3/PO+ZmICp9/vVEb2WgBv47WsHOsgCbl47oP2G0E266SoJI2U7vL3BkTSoWCxumfc/n6F1R5OiGdcw7AcUnOJaF4DF+ou9SaIdeEIfIxxmZ483vkc1Sr5Sw3gKisLXT3gMn5cKOot6wQs89UR9nvBTyw2vmDv59QxhmNzJ09m1OL5rO04jbrDbksqbgjY2wBtCubwZJAuBQl8KSnC0o/9xDdSj8mQSUllu1al82DoGKt5TWWleeTHvit+0Kn7BZ9633LjfxOnEiXy92IIvx+/3nyq6LZMNIbMVNe6LlleYHfjtzk533JfFPgwn4qvUqP0OWEeKWw7tpsykJekTUrUn/gnUfe4UH2KNbdWi75dDmFl7mQRzSST2EjQe6TBDb8qGZzBTROA4PWXdqtr2YYKwcn0r5ZLGs0f20Km4PkR7OHf+AT984M+tjrMTVUejYULpKBeISTY/XiqhEWZ0dj965DVSJqwCeJEoOLdrpTVNz5mk5aUN6FU4XPk5OTwzvTp/8jv8s9e2fR/4c0vjy7kiLaQKcNkiQgD9lwKj2lZLqoBLsNDQXolGV1n83cDYcHSZtboL/PVCxWFY1S4RerC3vAK5aD1ZIEo1npvjHNKs/itvfXeN6Dlkffd+tMzT+/OPsNNkXsvxorwr0MzAuAlVlJZCnlz641qBff5CRgAyIFT0RrWV2jpZpw8280qfEywp3umqQooaHCk2wrWBXyhbLYeAbBD+z001fxWe++qG1yidoaVnR/R0FM79NA3jelPyklzQiLfIUjViuT+XtW6WQkz7Tnnn+Ol098zsiqo7wbOpI8Qd4vY+JcFhz/lduF7dhwZ4k8cZe2CcXBTsT5XmNkg68ZrGgJRqd+m0p5wnKRarvETChWdZbfkJDufFGqIFIJ8b6X3ZLll0sak2OFJUHgVSov9ln0rTnpuu4O7gb1X4b93elU3h7/8s6YgRJ7iJRAqM6VEtsKDS/4qvjJMIuvSj/D1+byjDV4mTbpsCAQCpJycQ2FoMBTgEvRMC52lXt71X1MFh2LTmysldAJDQ1l5nvv8dLLv7NvXhNWTe3NpGHdZdLKDy1MOIXSgWpVC6ASXRb8uYdop/sAX20uA+uvJMDkVDQzSRKSi4vq0ztT8rp087SqzqW9Fn4Ph11NH5J0uvAmAxJWcjpjOMW2hkyZOpV333uPkyYT+QjMBH4AlmOnp0bL/IUf8/XKldjtdgTAbhdw/Jf8XinAR5NHq7Df6RKzEbFKPuYXeY1hVRl8XgqXK1yY8R5RPPD6iFsFHdAqKxDs7pugKks5uVZpHhNdEcXJa/l0QEs++3M1sw7LC1GCAFtuTudMxjAmtngbsVCeuFOKZrTKCk5lPMp3GSclr6Oa8G3G8dznKTGEEOSRimf16VppIQB16UkebbicKP1N4n3PSWxo50hZT8+QE1ToblDh+mwC53JOo3UU0FWiu6RLpmYiB5KfJi5uAoes/9w/ceiwZ/nj/nMsObkOi82lcKALJTo5krcdaqRuklw99zPo6kxUQKRHFqLFqd/6ROi+h5PpI3ilzfNsHyPK/OkA8G3Md8V6YlLg9ywpWTNl2rTaonJPtVLGEO2rrWPF9njse57cms3l3F5U24Oh7dcQORwAZdUNPuw2kPaRWwHHO+UZA4NvQov5FHqPp8wYWNfmEl09zzDcExbHXSbS9K28MaQHa6sSaK2BFS2XSH5DD4mCGjUj54KwS7iuoRSCmZ1jBd7sOJYiUxzEPyX1u7ZdWh+rqhrS0ecp2bH28GHsKA4mUglz/EFZfBJCe0HC8yAoaWMYSYAug1+uf8AnmcVuoBSVQkGUEtbp69NGKS+Ck/Ub4dljsUUsJ8d3u7zfGn9udtrHjELYFwFbx53/S3bmvy22+Pj4cKDaj31VEFI8lJYubGxt8jJO95nIwSrwvhkC/q0eeq438uHpzBZuf7fmH6eH1kIXLShMLu+cVzzflsK2SthRiayoIYoQ532HRzQwIQdStHK5rgzf93jtzCLygn5GG7RR/q77NWOv7RTH0kaTVd6A++aRUgI1czdcmwu6CIxWLyKVMFEPOlOKvF+GPDoJDdAK4F3lUmAC8IyiqDqcWUd+45sLn7g1qwr2MqbJbCL1N9EqSmRtReZEFhzfxNmsQQ99L+yiFoPFky0336LUQ46kvxp1hcFFyVTYIcVeKGccpvxMH6EHEfrbjG40j05mx7EHe0oFpgM9SLD9QawSzIDoygLUhbCnqpynCosYdny+tO50js6bmHHgCN88OpCXBzX+S7DAMODHh/z9YbFGp2PoGEk1w0tVTP+EVajK5HKsaZHreT4Pjhtgb6VLGTtiCBcTt7G9EhYXQbHGfeV9Qv82qRYbK7o9jtr1d1bpKbM0JyD3OaLzn0OnA+6vhpSfEQS4V9Sa24VS8dv1typWdeV4+iiUogkvVZ6cCRnYltPWL0gpacbpzGFcj/xTGquvfyQVyHShtWuyUxmPYYpwAuCKCnI1I7haGUKQAtrrsyQGd01EDuFSyCQKbfCBP3zUIrgWcETmHtioZ0CxwGttn6VhwCliqlfU+j2LIjQNPsQwx0/bT5MCv7eXfG8BL/MFxjT5iBCH/J016UNo51iHBbRmi2kEN03wYRHMyVlSB6YxlxFSeIxGrlu3muJ99m+QuQsAvaaAaJ/rbszRypC+bK+ErWEwLtrFozVjO19VL6WBSlKryLfJQYpbCnbyWLpTwevoMDj/BugbQfMFZFS2qG16aMK6prs1r8TBPnBQYpvX7Bufy4NvKx+VH6xPxCtmFb42P67GgG+eC8v33Ot84A8/hsBg30LIOyJvT/6JWV37EeyZgmuIosjKc5+x/8HTEkPeUCBb6ySpT/B+AAzNgjUVLuy7xNcYfGUh5TYorgdh2Y4xqufv8FguupRldAy4SrgShuoz3Aqv98Ie5+V8UAKCKyiv/D4d77+G4HuYtoW3UAS7ALF7H+J3WwkbTv9KafXfA0+cbTn+LmrWUfq2QdwLXcTmonAMKhfp7shhbMyezHUTvJoncDGnr1OxL5ZDee9yv7gV3ppC2kVsR1HtWA8rNNB5A6aYF7GJEqCyh999SYqzMhVS1yNWZxAfH1vLBIpTXIK8oxJTEkleefA5O/3O2WmkmCex2Jyiph8q0YDGklw3V0UOxTIomYs5krzm2azB0GqpJDkd9yREDMHDlM/EXNgRDgOy5XsiL1I40X0ys/xhpj945X0vvyfXPuJWrCQvqhDsTj6kNf1xrHUEmBCyCdI21La55Z4eWQ6N35Xa9HE8KG7p8A3dip/JqVgcPYJT4fmcy5Zk6v212VB8UZK1NJXCgx+J8KhjlvXRjYbdTkDszhtZev0Cv954m7FNZqErc3lnnMIuWOV725ZL4dFsqsx6DBYvNt+YwYpf81jw3vsctVrZb67L+6UgPX//JGoUEE5kjGHGgSN0itqMd+k22WdEEUS7igjve9TzOSH3o1d5YVMFYhGstNKAv+i0QrOaYEc9RFsVe+69xLsHD2PQuqw5Hqyhs3kAAbqM2u9yjheCw5nfswezug+ivrBa1hauPsn8/o0IaTac47bbkPg6eDpQFcWX6B22kAG5S+hy/SLt9E7j290vUFXdxI5AiOcDpnUYX1cgd0R56lfUa+gA1LqWZ9K30ebCE3TSwhZLJtlGeV4FwKyKJruiPtWm6oeOEzuA7vyzgmxnUcHkWT0YvKZn7f2pkZ2vKfZZ7VbsdvtDFdPgP2afczyshvF/9Pv/r3zrf/Ff/P8RJpOciXEv+Cem/fEnYOd+eVfwaVzXWP8ljlbXsFnsDKgKgeNjpEW4sQhuLmVqcCp+CmiqAYXNRa6k3SoSD89meQm8dr+PHOWKgKriDs2ru/CnfxN6hcx166vFuy1p9z4C/2n8oP5nfi5rdDqGPD6OAF0mKoWJi9l9sHvE1X3AbqN3yQjmOGwU3NCzpTdo4vkDPhp58h0AD4mmv79wAKWae4xp8pGbdKTBYiDV/1uON9/NzG793E7RrGA7Qz3hkxI4ej36Lwsx/8TjZAWwDonh97vR8JeMrRXLl3Pd9wM+KoLj1QK5NpdiS8QgzkR8zHWTlFAUrJXydlHBMUsCoi6N3iFn3ZMBcU8wbksBMw8e5FTGY7Kmmgms3BjAzYJOCNpAt3aFzQOjIRYTsfLzxoxl/vVUrudJ/XVLjheeolfcGpKLW/B76hRJZtUpklI+4Vw0zA2A7toS+cEekbwsjGB+sbyfzpElDuNg8gRWjt/L4b+RGPt3fjQqpv/egBO2XLLi4DHNOvmHDAX4aO9w1QQLs6OwezvJV+lCofcR9t57kYkt3uaZltNwjZCC/czwB62hAd7mRFavWkWPv5Esce5bF1FNSuqP2BExiBEQM7o24SiK0DxkPyOSFqFRVKG0ZEvG1jYzGPJpcT+ICc3frf0s1ZmQexjMEsv2Wc+RrA3046YJKhVBD+1DI7Vk+C2YCmV/9zw5hi1h4ClCM32GG0NYMGTSJfoXgj1TiGn0z72f1uh0DOso0FANU6qvccYqv0u2yMd5K9eL6yZ47MBmrInT6xpNhXSumMQkPYzxgi/Df4JiObsiuXk6E9OCqa8CreDyTjliVyXMyoxC1MiZW32Ld/GSD1iBP4qi8HCwr2ui1HMgMw8eIqt6AElJA/iBv2eV/gAMGDiQsLBQ/rCl1kqsuDHZAtqyw5LGupDX+LjIA7xi69rMZQgVD9AoqjiXPZAL1nmgcuq3pZKB1lv0cGA6BNyzhgpBycpgWNz1cfk75xXPezdn8UMZtEmDS17yhHbNZztrHbKUaVsgpDtkbIXKFPK1o7CKBm6Y4CvDcgmUsjVU8kMqu82jUa8zvuksXmz9GsqKa279ArhuhKOV7ppOwX+0YGs4eAigFtwX3h5pi1nUpws6ZbkMHQqgVhpJDDjNkMTP6KUaLmsTBEmeeMtoLe93HYJXgXwsaJMewk8BfuyLgOMJcrlVEl5gc9VlsssTGL8ln1u+ToUJ/9YwqoydlZLko8LqgVtkbONUFNRXQ7nJz7095SfifC+z4Piv3DONZsWyZXyxcCHH7HYOY5c9Y38YDLxktTJ39mwOz50rM5TP2TSXmIgwVsyX5CkVCugRt5ZJLd/krX0nIOFF2deqlCKdb5yjy/WL6ARflxsdTlX8TETByqZR3oSUu8ga2SwcazyESXromvMRAi4m3kGdWXv5Iwqq5H58UAdKuZLbkxFJS1CUXpC199ePZ9Mob8qMgWSZOkiJj5oI7cWmlK/Jrkigf8LXNMroIJNKtCRM5pntqcT5XWZupzZuPl7VF97mldijVNSDyf4bZG3kHefVqs2sCZFkoEKFU7hGjX3RljurmPvR4n/sGRkcHMrtwvYcTR2LFbUE9Lro8H4URIx2HSY7HIqAKT4uWHdRiRI9CWo41moLXplOxQW1D4QPILcyngfFLTmdMdxNcvB23hUsHlIyV3DyLq5hxWqe6M+bwfCUXsGS+ES6zaxjxQoqD0oMoVhsGiyCj1TUCpTGboWtgibBRwh0JCAEbBJDoPQWhPYmNeIn0sqkNW7teuLPF+Gu5CvzZu5i+mdB7ww4o3D357TaTUQooX3gFQSTE8vXZoHbn9PbA1rXKAcFuCQcs/YyqeVUFvfuyNzm8vFeVIicSBvBncJ2Dy22LGnwBt+FQDudCR9BDiITm73P3IwkIpXwfgCIxaclZkDbryGgLaHmLXiqpHnYbY9ss9LIepveHjA25C46w2V5e8k1Iip+IbrhNBq2HI7C7jSPCSJ4RFBqg1IbtAotYeHbjz3UZzobcEn5/WWs0ekYPm4cCkFBggrWNjhFkEGesLbpG/N7+iAKbGBFdGcuZ/1ON8c8ZMdFWg3YVG8+c4vgaBRoC13eqcB2PJsHF4xuhyGKMLXVTLaFKVlbDseMLmomApgVxbTXwrWGKZDsBFpR/f/Y++toLeru/x9/zDVXnu7uoru7pEs6VDBABUUBA8UAFQUDURFbbEGku0O6O88BTnf3OVf9/pgT18wcUH9/vL/rXh/3Wq775uyZuSZfr/3a+/l8bjfyNZ0prAzgZNpI9lT8KTFbzs6GK2/B9pasazeSlEj4wR/cy+VjEPFfs7bdR3zkA59EnVCx6um5kRWnv+ZWXqc6uURH02Vt5KEWC/lyaFMaWeWMxEq7N8dSxpJeEsfDXs2lvuwOFu5xhQiPy/x44QMK3CbKfDZDMHk2qVdOjL5afl6B/TmnWU5ueSjbE2ZwXv+ddJP866VV34nbxq5gqf+dYJGrsgDctO0gywrnxJvq5+wcSlZpJD9eWEx47AJeeffdBiW0d4ha9vPPE/hPTJuGr1My7YK28+reg1REzZNtJ/+OFDGUwZugkBHcNsO8PCh0VsvIbqvQclR7kVCXO2gExXMMHMCPty8QmPs4LfwOSzKTD6bCiLt1v2vUljKj/UxM2XLWTJLTS3x8/FfivE+ysIU/xH8t89fGUcVVvpSbOkiAvdAx0P4LSFlHu7seNPP9i3XX51Hd5F3VviWmK/Q2wW/N10kMNQfTa6WBL9MCWVXNJHAiSGsAiwSKTS5qxoYbL3DML0UGJJvU/C2+95PmgfbuHaT1Q01/qWKP8Ty99QY3czsD6jHMXeddfw4aB0BxeRqx1+czoYbkNef8s6SF/1rPWh10FoZeRaOBTTfn8PCGnLq2A3XX3HIhC/JgqDPEuiqAEKYgzotx/FUhyepWaSNkbkGA8BwH5mXrJVJS2y0Wmr1CZmUL2baO1iCzT+dRF3eMZDk/+Nf6G4izRYi6dJHfLi/EKUDBlk1ZxyAnCKqddrv+JvcHD2P5qW/oFrpW6tFefKv+uFqBbfHPcj5zIL7OyZKc6uU36/xXNO8y6cBPxNz5luYmdesMURQptME3RVDp0gaufwyrtJBzBNONF+kQvZy95WBIAKIek++rEfHUgDkWntTLWX+Ygihq+ja7y0GwawhUt2nl2DHoEb4ag86pDnjSkBrKd8B8YDh/04e6Jo4a6necCR4VzEqKJUGcLd8w4AEO5j1PmR1Em/RuOq4Rap/zjdyuPPiHFeLkCkjxhddJ8pN61g2/+DwMPAURk2FCFQQPp9pShb0GCOgh5kv97GvAK7W/oxcr8NAlyoHpVxYxu70Ebo3yvEDjm1GQUFO007kiukVQ1dC6ASBuJhebvI/J5M8eMRb3pvJr1uiMnCt3JcUCPVOhuJGiZ1/gQN7Jg7sWGHB2KpZGr9X7MvcRcyaIh10lRZJ5yR/Uy8RWZNHW7WsGl01EZ/Giu+112WFFKnHWFSJgY1bHaUSUOuQURaNU2Kph/a2/9RqMzZdAkJWZcOJRmrnWxxgVdn+o7aN5aQEk/kpmVRu0YhWTW7yFseSw/JoOj+OjmvTTicgiXByVrPQeYArAXlMqqKy8NwD7/x8FBI1GKthOaP4uLsVyVphn9X6MMXPo2aM5c5t3l9bFtVaZi5v9CibBzNkw6GZZUe+z18dM35xdDjQQO5Yl4mM/zJRW83m9xwjVGHat8jEOJU3mkxMrSRYmyHxaTTW+pjRKvTaRZlIogIRPYMnVeG5l98O9ojWhbIC7NeuPGvB4+8AdBLvdonfE74ilciB/wLwu0XIAAQAASURBVN3FvOUF091gqoecUYhGi11jrFsZlmsaAPdaC/A0ZmAmpEGA2ibgsQb+3pBNt1nRXQLRXFTfk68mhxDoEkjJqyWUz5dyyMr7OzxOAkH+x+yTm7KG8X9p/xX7/rP/CbPZbNy8eRObrT4JZjOGkFUWCQh8c3MrNJdPoI4DUKJupoQO2tYUklZB++VEGCrpaQTfO0DQMPkPOi7MlM3VNSKMyeHnI/tp5JKOhy5Jdb61k0do2HT+skjFlvtJ1dUu1B57fBpfn/2c4avsvHlwN9ZAh54cgsg199f4rQTWBsKLvgpWQ/pOBrk/RqTnRX4YGUJIjoNkjXtTmGzniZxKnrfsY8zhj8DgLdtdn7mXL3wh1wpp1XIfQFqHX3kqGwxmf1KS712I2cT9e5zUssn+lkpeo+2dahvAsZz2zM3wYk3F4/INncMJjn6eHilfkh0FzonvqY6VVH6FVzzho6gGwl/RSEm1d10gJXOJ8FS7ZxnZaBkLD+5E8Jf3K2pjXMrrPUbg65SEp3hLvrPOlRJbGGabkXndxuGdtlDub/cpY9eUUVLthU2R1wXI8X2At/Kgy60IkvXqYrJJZ6StAWa4g96Wo/Lf0c/h+/NLWXr6KC/PfumeSdS/e1aONqWikn0ny9hcCsmVbjjrFGip00/zexPpGzRr8xqUxARoH7id9oFqRs6d6Jd4IA26OJXhJGSy6fff/3EPu8ctFVRkfUeI2/UGWUedQzbyaOtXMOmKcU95D7Y1g6o80LmQ6zWDqzkORdmYJ2FCWV1/Kismqu0CfiKYBHXfFUsMfOADvwWg6smnwYYIHAyGn1r/oJJ3EQrO8XK3SSzoNYRJPb3r2AX/ZKw46P8OY2pabeRaFElUDQh2KUltE6rlz0LvxUX3L/m1RCpC+otVoGAaabXQ3rmCWxEQWaaQqC26zjgXSYZnUUk+olEuz9ey9CRDnCXplxOZvVXF6Np/j4t7ghvXt7KL+48Du4Ad27eTmZmJHh2DnSAvCgLLN6mu+VLENPJdD3PG82X5gaKfgNEZ3MzrwoXM/ly2zZfk62rN6McrzpNYXQpPuBhxFeRFW64v5WaEK1tLNXya2kb1bmtFERtwpgrKtMEyn/+1wRRFSdIuljpZSg24REPbT0jyWISt5r3Sa4yS7ErsM2AKhnPSGP7y/p3M278Ti0dn+Q/nnuAFD/imGEY5IrFrzOrVnpOVEJcEK2/OVfmr/cdzPOVBHmv9MvoqOVOt2BbDxHVFLPprI+dsH8h8ggCXs/pwI7cLrfz3oquSyxBX6JtRWNyS3WWwt1jOxsEUQCEtMduMFFf7YBMdzlujhVvLORz6E7uD4Q1dBzWzyqcLr2f70jsVXjnSgGjRsOu8dUgaXyoq7s8OdgSd7DWbZYXAPdVm/rLB4reWsuzjjxEEuJbTnZ8vvktOWRiCRj7QROU9zYpuTzGz5SIe8FT3JBVFKKz0Z++dR6k2NlV47XSuWs93/nCox+uIFsX759OJNddeI9b7NK3898pcte9ifH57pm2+gy1CXmzOsPVi352p6MVKRKFSjpCl/ns8mz6EFN+PwFgPaBCNHuSUh3E1uye/X1uiYlnGx73Ok9lwqAJSqxWs7eoCQq3pZFjhp8xYLNowlNay9GFe7yGh6GfPfYF5ixbRQ6MhVhSJQ2L2xgHRokgPjaauZ6QqoZh7vG4xjc3MM17jCdeCxWrCQ6tghhZeJdbtJHlWeD8fLB4KFi/S+LcjYQaLj6xXMXJCz06nIgamuEKES5rMF+Cp4+VxruxYAmlfWvlt7Qlee/31OlasXlNMqNs1TNpi9fzo3ZFxf5axNV6SrNJQJfV0uSrFNI7jaN0cl/ANZErjc7XgTokN9lXA8TLF+xP/NWv8LnKoAoy3bRDkkMy0W+DsLGa5hnGr3ItD6f3kEnpI9/fBxsvILg8noUSegBU0IkuOrmXzzdl08P0dNobXSWjXnqy7BvbG3qGRYhzRaEBv8eViFbRK8MQaNlWSvtrfH0JGss/PRnJxM8LdLxNrXC8HqNmtPFL6AwOcoPGZEeT4KOTWGz2P1x0DcTr4JQBEq8M9sdtoYhBpXzWMdjV5/TlDdXQdMEDVZ/pV4BD1ydN7zc3bqS+2LAl5j9Oh0N0tD4NFvj6wBw5lRlIs1XZo65IBFgWY5tQ03veBqa7wsOcNlKYRNFyogueyweKu7lkK8KM/FEUh+9YFATbfep7Fmf51x3E074oNjOkwmaya7wJPB+UQuw2tUIlYk+ype//67oKuv0OpxGS5Xg1D0qDCo4/8hPx7sSytNXF6GOd3B5RghhqT+lKqGcC2JvNZcHAHm27MpkgvB/zJe+RoQaE48WSrabzW40HVtgAmayKhWkmy7UJUDpQ7xFFe7UjUP0thpT+XsvqRrKthQtYWVQIk9kihDY6GgqZQUeBM24Y1Bp73AIPH4XpkQ61V5uBpymDttXkciO/N87Nnk5SaSpvnnmO2qw9jBC0zBS0Jehd6tQtluF53/wS+Xs+SDz4gODiAQNd4Hm/zMiHu1xEcVWkA34IVzHSH1QFwNPKC/EDWajDnUwtv0GoUwRuQXH6TDwvAP9GMzVnNLrBa4Yk2c3mxS80c6BQMLhHodLC4X09Wj3FnSOyXGIr+ku1X+2wKKgI4njujTmkAgLzT9NaOo6nvYbSaarSUgWiCHmsh9ikw+FFmaE+FRYrplMClvgW+fNhoK+er4NWUJvJ3O3E1g7OW0b5kJH9dW8x3dw/Wj38hIyDmaS65/Mqmm3OosLhh1oXUAUAEAVZdXciKY3/Q72IKNq+P4cEUSSISQO9JWkkjqqzSHCLeeBcuviH5sv9imDCdbkYY5QzD3DbWn5NTMKUdvmNNDQbZ31CCTecACNXoQKO9Z886AA+jB/qSrugTYNHlWXKndwd+chrD4ZrljF6x/G3i/CcPuUrfqI8QK8XPofVg2Abnovv5evwJ3SUwkA+pNNbBy54wyKgAK2QdJDK+Cd4Bv7Py1qPYveUSyIy4Ta9UyKrN5wsKgJtXW3bfnk5+RRDlZncZUMfx26+yOktgLYf+xGZtEKWZU5jtl0tP41X5cUtu09vjEhakPuTWgDESy9wUBOXJlHU7xnt5InbAYAtSyZ0Hpv7BMl/4o9BEoaC4JqMPlzx7A7DA24ZQppCmzz2FPmgaFZ5b6ddRzy9GI8u4txrKe0i9qLsDfUWFyoBBJ+u9PdCwgSfdwKapbnC97ix6MiT3VbZ5xtE+aFv9sy26xrNxXegdUQ8I0Tj2PTszC7fUetacBoN0TzRaEPXY7HaWXquXIUzTPgyt34dbK8Bmrvt+Y71OM803sq6XHwD5Z2nhIzFVCyoDyPN7qV423FqJUJaIUw1AaHjcZ5JKj7lE6o9881NaGTUk9RzL9AlnIE4uJSw4BTA2oSXfF8PVauTqMADBQ4lPO4Nv+lwapy1RKLy4Y3ZtS/SNbQw5Y8XV+kJ9vFx2l8HeT9PZ+zoDLuTQl3fg+FS4IQHNxEsvsXqsJ+7GbD458SPJ7gvrj2suxsVyGWddofoBmYKh52auVdRncU5YlkOfmsJZwldw7QNaeG6k3OzOM9uuUBb4pPwYBedoUvP9GwQ7gt2xqJYNRdcRBTNNfY7QTNeePjqxwbXU/z8KCFGeFxgc+xVvHNhNXvBbsm2a5ozgiaaf8EupmYWZUSA4fFO3v6NLfguiXbJ5NRcuCV3rfaIBhl4lqW0ajX2OMbn5ApzsKfITaPEmW0yl6DRVtPA/oMpRHK74nD+vzWff3ccoFOXAtyx7b1pe6cYfpdA5cCVsb1m/9tB7UGiNodoqzbmNbe9LLTsAeqyjym8M/aJ+Ir0kllF/VGGOkcuKJ8W+zawceMwNJnsoFL2Ch5HZcyd3zVASDcN8P1bd17jcR/h2eBQeQZP5yaAGjv/bguzdUrhWkk2pXorvapl9elGPi94Fk86EIAiIItgdYjp3owQI/I/ZV28N1TD+L+2/Yt9/9j9rentuHQ1bFfTGf80ApwnoxQpA4KrTCmi5CJq/IbEHvNpxotSdzJqgUaNI3FGSQEDMAqa5wWvhu9RIVMBm1/LQ+jzWp32n8gXeHsuWSQJvPTCT+TPG8IAo3leq7gFRZMkHHxAUJB+KZQGYIFDiv4Djmb0I14KfVi7DSeiDbC7aSkpRE3LLQ7GJCtZDdQHfhB5hlAusD3hRFZCKhReY4QFPZsPVporiJxATPZpeCVu5GuiHOefLexZi/m5C+XdsMj17t63CYPajWpdLqvmiajtRBFP+KLbdmonVQ3FUm5Vf3b4lTAcP160QHKy6kDjvk3w2uCVjm8r7yGk04OuUTKBrQoOLKh/xAm0Dd/J858eY6NlO7rSU4WO4jVFbSkv/A+jLLyhO2kiV1YlBMV/zTtc2UCJPtOf5D2ZhPlyu0lGOgs1hqWBpbGs+0z7AF35gsMkTjgAGA4DAjdyuPPbM+7z63nsNJlFv8e8mf3NlBaV2GBzfjFtaeY9DgkewLWcio3SB7A/RIBY69EyzlMHlt2gftI3ndl5k1g45kwzAbArCTQN7otNo6rmZouLif3VuBk0Sk5svJEL4A9b5SsE90tiw8cYLzN11itJqLyq9BkPzN6Vij9aZrNAvOJIsobcaWvB8aV7Mc0VZZEXBA9U/yp0ukWwqg2+LanrkKGTG0tt9z4MZ0LSOMdFe5tf4tGfx4bWEuV+nb9hWlrz//j8eK5xdpKT8riDoKchZNbrk7/kt8gZROugW+wFCkcP91jpR7DGds5Uavi+GFblXwUv+/u46/wh+pkzezYdyJ4VMbOpG1gRClE4K8JSB8g/hbzE5E/6qgEEBp9Fmy9FvLpZLPNTiTfacu0JvUfOPx4Hvv/uO890+4Cd/qdBoE+WABMOZyZRHwwQXOB5eLmuarTRVvKXRUq73oa0BvgusxAN58Qq7BYMAq0ttfJMdKh8PKrOZHvEDA50kBp0WebLS7NGVLWVwuBJaJAOho6ReCMYACJdYgq6VzYjU6Blq2AFaJ+jwObg3hujH+TpzPB+GDuKD0EEUKdkBmXv5yBeCRPAoUxQCgfIuv/FGTY67ukr9cle49+NKTi8GxnyHzipfqtU+15NpI7mNHGQhCLDywke8tv8AY/+soCRcDrS5FbqfzxL78G4BLMlSsLGtlZg0eWg11YS7X8Zoc0Cd28xwUULL9neCBdGHoErRa9CzJV/me5NskWRnVKb3xFlfyJPtZnHzzLx7glL+Meik2lzXUP5WXie2x88gwOUOQkW6bFuRSjAkExa4Dj+9XBackgQibnWjsc8xPj35A2WuCsaEoOVWo5t8e24Z227NRNApejPW2Mz2M5jUXL4o1mjgpa4TmdvlEbLKIlVM2xvWZ/jk5I9MaLaIVyNNcomprAM8EjWRSI8LXMvtTq7PCzLmn2grwt/5DslFTdkUP08mzwhQ4dGak5UwNB3WFg6Qn2zIcJaHvM/zOTA/sR3l2kYoTbQX46wv4qWuExHjl9UlxMMEgQXASmABEIYgYwGIIjzS8jV+ejAQN0Ou1Nuub41UmbmYNz3e5R2nGGadWcAJq1x+m/MvsSJyLdkWDR8k9sTs41C8KrwKf5gY31TORpZds08PrlXBTwHQ1lMxxlRk0Cd/FccroFEi+AfLgRD+lp18MbQZbQN34a27AVviIP7LumuSmaCF9isgagrknSY0fRoxXpL8Zt08NTYPOksif2H6TGJrcpuiciKzlBKiM6AVQGtR9NDU6KHfQfReuyhImkuUS4YEhHG0xnN5dGMKy47/zO8pa+W7O/yU2WaSEuWioe5vC+98y0OZMDcHTlmayfblyiIWh12iwg7l6TMQjN5QdldC+d9YhqiVnnm/yJ940HsMVDkAmzRaylqs5MfscG66b+Z0/gn5sUU9lYh8VyxJs2FwSJZbymFrI14KvMwzOfDqlaksO9qMY7t3q/pMPwN8AAyDe0pOH0LqhzJ4yBACAgLIsfqyoQycbts4LPZT3a8k3y+Y6gpHQlH3R2z3GW/mwXR3eMxLzeT2L71OkBaWF4GLTxu5M/8sP/uDtwjHKwEHhrogwNGUCawqdCU5Avqa5awFr4otvOJbRooZXskDfB2S0gXnGW83MarxUtoF7qC74RkoT5OSmRGToPELrEiaR9Mk+CuvES5uiufs15O55RcYlg7OCdQzp2rt1hcMiP6WVWM8ebPncNU1a1zCOJcxiO/OL6PQSc668Tdc5rfRPoxstIw/ii9Ae7kM6L6U51h//SUW9BqCf4E8QRab1J0/AuBABSzJc6rv/VRjDRYumrwEwxOg91aeLfuJaVnwRkojienoaKYANpRJRY2EJnfUPfuOTmD5YDlgbtVvv7Fs6VJaV5Xyhd3CBruFjysK0V7NId9soYdGo2L+9dPVyAQvXiyxhwVILGrP/H37ySqNQlMtn9e98r/gZWMsmoynKKpWxHYp6/D/y4dPK1/ihGt3IioVzAXgHduvbA2S5l7VuqjoGn193+Zk2kh+uPCRzGUwwJXs3uy7+ygPr8+iNGaJzB9c9QPPdHiSrLJINmd8Iakf1FpFOuHCWvydExkWt5w28S5ST+bLb8HmWPDpxM3gvdwpaMOM9jMxXJ8vO3aJrjV3zZBghh/zwuTPqjQBbc5uOmYuIibzFfk4rHWGjl+SXdMr3k2fS+qNBNmzvJbbm9NpozGZQ9Ry5kIlnsaMmjwEaFL/lMDGAJU5NNcm4SfCw24wws0hj6BzwyX2CQrN7RCAV5r+RGjCoPp1YsElyJOUakLdrtEt9E81KOrSQt4PkbbX2OVrfYAQtxBGOkNiuEiYTt5/uJPrEqY1khQXZDL/BRdhV2fa+dSzMu/H7GvI/jAsp0sqPO0ODxgVCW27lfLqNBJCXyHN+1f1sUQjC4IvsOX4Fr47/x6UKtpn1NjO208xe9c5qS977a6ipEYxo/1Myix+0PFLWQETskiNHceklq8RLSpAZOk7WBD+NU314Fc4FGedE4SMlIotlxagC+xCRc4oPDXwqHaiXCYWcC84wRgXmJrmQbxNLXa48sJKOhhhgTf1ErK1dmEeC6O/x985lVcGF7KvqpL3uL8aylEkOEXMY3OZ7x7GNFcTb3pH0uv1N2W9t7+vfpunsmFp9DGEMkUPsFsreCqyHRFepxjof4Eg13iJqQtgs+CuS8FJW4yrPo+W/vsQqxzi4dvf41dSD01o7ZIGhZcl2cmco1CRQam9XoLaKJqg8KK0LrKZEUXY2l7gz5herCnuKJe67rmB5w5Lua/ssgjywz+oA+aScxQ2R9IrQno/g1xvSQBbuw0KL0ngqKQ/IH6FGmSDBBJwrpTGhhZ5j6Gxqbfxs7SjQ9brjIj8HV3x6XqHd3vyWu3jbMYQNICLvqQ+b+jWhDW5uzmaPA4BjfTN3P0FciTAg+Dbhd23n8BsNXI6fTjFRgdgeeY+WqW1pH2QBGCM9ToNKeulMUjnAiHDKbQ1Vp0nAAPPgCmAqZGjeLjFGyQXN8Nu8JNvM+I2Q2seXcu7RsytHVRHrr4H25ribszGqCvlr/NZPFLZMDvp3/aLfHDyZBr5nGBqq/nY7BrsBnm2J9FnOZ8Wwq5y+KxQL89X+vUg0XUh6RZYUgA3NA4FOUED7k1xzlvLzA5PM6nF2xjt6hyZIMAHx/5gwtoSNXhXK99O6RNqCCCigLyPtKUCT0MKBlF6b85rV0DXmvHeWo7FqxtPbb1JVmkkFpsejSLoN/t042glTMqEJ1OHqs45wiOCKjscroBCS7jKX+A8nJ0JT9OvdUsOmatV4KB/W5Ct1EOlHc56vwTU9+zTiXKQhUZTz/oD+GTgJ5jfMLO4nzyn+p/9f2f/Ffv+s/9Zi8ycytfD4nDR5zMqfC6kOyxM8s8QrVuDTiMNxIKAJOvW8m1JNknQMvZOa1aVwFgXECoUQ+CNTzgUAuNd4VG/RFRyJ8lraeoj743jaGbn1sTntaNzyEZ01kz03D84qwXWiSJEe55lTJP3mdF+Bprcg7LjGnU6XCua0iEF5mcqpJpcoki2DiWvIoSX9hwn289BYqA8FdZ6MdDZTHsD+IuoEP62RrPxvi2hc91cFdcLYC7BGzdM2grM1oY1oeHvJ5R/xyar4Pjm9/gt7hSvecJAo6L3wN1fCThkItrzHF+dXYE1UKG1LwhcrYbVJbClITXCtK0sHdCZSI/LeBjltHhRhEWHN/PJiR95tPW8uj4MtXZc+IXRa6rZd+dRzlcqmERp23ghLoa2gTt5aH0uOU02yv1lKcR6ncKkLcWoLVU9C40gKXbrneLJqVIkfMzF+F57Bp3dyPx9+6kQFcwZoJFlMXM6T0GrqUIrWsFuRxAEwhVJVCP/bvL3dyrnWXewCpXqhX7UFI5krqJbxgc0cs5CsDpI41oq4PJCOgRtxWYXsdrVkoM6WwUFVqlXTUmQJ+5ubv/q3OyCH/vuTpXk0dyb1TG3RBGyyiKJz++AxabH7DsEWr5Vx9hQJXSK46VeNNUSOvBC2Taq7fB5ISTrFEj6oMGMyYBNtT1yFMkibU2AuqJAx6Ybs1UJIY1zIMdSR/PU1pvszPkM4B+PFZ3dr9PTBJ2M4C1kyI9bcp3+biU01sGy2MNoMjbL/HqtiN4iFcvKUDNDJ1Vt5Gl3WHl1LiGhU+TOkAcZlwHjXOB4eCVaW4HMbdF5UmKDNYHQ0esGYrZcnsPZcpWJzd/hyIU0ppj/GcqpVuN/V24LPiiAB9KgzE0uNWz3aMHecjAK4KWxy+Q8KE2ExFV4m1IZ23QxQ3Rd5T2JbFYe8IsiKXMEz5+Zj81N8ZybzmNO4YukW0EVNlXlMyZwLYOcoCwGuhYvlbkrot/gyeRI+T5FVyBlLZx/kZiCmbS6+xNj036ii2aplMSqtdBRbCpqjrHmW1PJi0Y9yqTTc+lugteDFUljpP4oBkGSBekRpS5+Wq2wK2E6j2zIQOMr79ti1FXQKXgT4e6X/zah05A/w1MqspiVMrC3Puchow9x3if5fEhLooocJWYLAbhtHsrYvWt49sg3UkJFYTZsPOwKI8M3qHwU3aCZ72GGx33OxTM77glK+begk++/k5Jx0V7nWPJAL4R0+TeVEvwjrTJz6JYKj6QrZjibGV31XVz00vWpkliCQJU+js03Z/PV2RVojS5yf8p6PhnSja23ZrE/b5FyV1z1+XiZ0vF1SkKwyCXJaxeutwvacLn0Ebkkeeld2nj9gZcpve5Ysn0Tv+G7EdFEeFxqkIEuOjA/7DY1C0RfM7jaBLOK2AJw2rSZNw7soXvYGj798hfef+MNDtts7LNYZGPfAatFJu2t0UBxlTeZpdFqeTzRiQPmOfxqSeCO/1L1vY59mm1pS3ngUgqd4nfI/TpX8O9HbmUUfSJ+5sl2s1RAr6KYmfRMhWHpcL5A3o8N5whetrblgwK4ZQZfP/k9qdA3ZfWV10kpbiL9rmiqQyyL9jLaB20jxE2KMzRanYQ8D3gAyhLxKvieAGcJhFD3nPSeUsIHeNHzY9YHiFhiYIyg6M/Z5AV+1e2jyg4DnC3yYp6gAf9e9BvVmAmtPyHU7ZqUmHM0vTt5FSFY7TqVzJhGA0+2fY6B0d9wpWgUDDxZj7RHkj+utMOyQjhnVuycsZthXhn0uXaSNoUTEKzlEDZBkvK0luFtOYCTrohDSZPZXvibvGAnaLBHPsb5ciOtDWC0FMqPXZFFeyMkm6XYT6NzkPbS6KHZfI7ltaUtwUQJibwy/817MoDnAN2Q5Dwbmpv3ICVYj+3ezbKPP+bL3Nk8XhNOljv2ywW0uXv51t/KXYsU62AKkv9Y6CjOZfRnahYsTH5DdS7tUr9muS90C+2Gi14xTpSn8ogbrCqBSfmeqg/aJpZTarpBuhWqFD1vM/3epUW8H+UNfKfovUnSPExSUTNivU7TTP9FneQaAG0/4miRBBBssAgEeFe1o9oOxuLu6oHm5icMi/2cI8njuZzdW7WvSCVOuiIEbKpdrZhILGxBUZVvg797Omcye+48TnO/Q5jMcqZkgc+z/FqsZU85HLz7s1ztJHE1fSuaEed9krmdH6FXeU1h9chY2N0ZdrSmkXidS9XwVVYYGpdQ+Q97tWNR7lheyYVVSf1VjENCx7LvzqN8PLATE5t/yCcff9xgD/Ta1gZH7XbctFrE3r15w68JEwxBvOjSBL8HXyHp7nXmzH6u7tDVuFNU5cvbfQZiSJHLYaZHb+TdQ9v5+exXrM5UrKdcoqkKmUZlWTNaep/HZFXIPwJJNh3uGglUJdoUgNOiawwIXEBWaSSHkuTsdqMRfrv8NstPfU9RlZ+iNQZ4Ww4wKOZbRMGifo7BI/go3caBxEdIKWpKnvtU0HvB5YUSq1QQsdRgrDoEbUHMlYNtLnnv4b0sCRzXrECxVmv+OuXDq0guak6n4E3081tY38utIgturaBdYT/6RPzC6KYf8JBrrEzq2vFc3azn4c7PdWsHt+IN/DwqqE7JxNbnkCRnCBA2hsVsYEMZvHu3E6sLHOK+GmstTiYs+6n6PxScl/731JNweAyCAF1D1/FK9/FQJlf3sGXsZqRnFv2dIMpZDkyi6BoDi9fR1gAFZmcZOANgX/FXvHhtPACF9hTY2RFOPSUVGMpTpTVrA9cP8timLma4vbKOnRXu3JgWiV8z68iXnNcpQFEB/XjWMo67Zvim0+uYkxUynQUXaG0S6O6ZyrQ28+GqXK6VhO/4elgsER5qMKko1vbLszc4TrSxvcb59msZmG5ja9VsuTNwAB+mPU2ZDb6MOosptwZA1ORFaPUuerESAxq6meCzqI8hQ14kT2nxMUF3oUqfQaHxgvzYZUm8XbEJHxGaJiGbOwFo+jLP5cCVmjBEALZyf4DaVmp7FdupsjpRUu2JxWbArgjCcoVGJJglJlG0RsFmtJTiLGazTb+P2L/ms/nm8/VrZc+WfJSQyvaEmcR4neHdvg8gZGyr3/fBFG43qi/2L437Ec48B/lnYE93hLTNsrWMp3ADwsZL8pRap7qi4h0zfFHUCXy7yU6twb6QAM4R0ORFkoskEMPXZz+HCeWSPHvsDIh4GPLPMJPubPktjPi9I2XH1VHCvNALDHOGMx1/wu3yJPk9SfiOR8P6EeN1lpkdZqLN2iZzazRQqcvAxf0ks4M96lldendSLf3JKQ+rP+cJlRIrHiBiMl+e+44ysweg+KbcmpDmtZDEQmlNOih6BRweI5O1dCxONdZ9WyftjnMoNJoDwNmMQbgZctDYK1BasF5SGonNfE0OdggcCM3foNLiwrmMQSQV+N0z7zeNf9cv8olp0ziXPZoXdp8gvzwY0Voo2y7f4zF21gk5KNsadCPVc0HNehyinR1AM3Y7HBiET9LzaAUzM7ddpUoJWC6OJ8C6BZNWis2U40Fvp6eZ1fEJvhseSSObHLTiLOYwxjuNJnr4oRgYfrOehZ6+lXmNwugQJL0Xxdr24FejHJK2Fecbc/EyZmDQltPY5xhChbwIqat5kEkWSDV7qc7Zdn0ZrhoYkg7nih9DaXmeT/Hd+WWMbfUnSybaGG7Qy2Tp7wLfqvZq2L4TjZTVpEFKtYkAOFVFEWBrh4+TD49teowpG6ZQVl0mMfscin0mnQmtRtugTPN/9v+N/Vfs+8/+Z0xUoCBKPUex5dYsPIxZ9A5YBrnH6p0dvuKHKitlZg90mkralo2tQ1ADsKsDJ+Ku0tsEfwaCUBtE11rYOJ7NhoczYcqZTfWNsWvt6CTGNl1C19C1hDqdUZ1redTrzN19hoE/pvPWt0fZbrXeNzjbbrXyyssvk5WVycDob3m09SsMif0KsfiCbNvYu11YElBI6zu/0EI3WnWsBvXyQZbYtgOZUdT1JKvbV+dKvg1e8YTmZfLePABsbcJjrV/mqa3xiIaAexZi/g7h48j8u59cIUiMreJyO1lCPo+7Q0e9Ar1tCsTs3Y/iKm/1NQMIGt5nMm/nQ0SWQsoEwLMVv116i6e33uC7c5/IXLWvW4THZUY3/gBK5Wyf2gDrQOIUzlYpkjJujTmSP4/0kriGz+vmJ3w8sBNHksczZ3+81AvBwaJSfuJmOORHQ2utYnGq98DcdTM/X3yPy9l90Jrk7AEAb+tf9I38hQ0TjKx+v2tdEmGvIok6C/ipZp+/exY/G42M6WampQFmhZzHXvm56nc1GjiY+DAPrc8F/76yc2bIZVZfeZMYrzNEeZ5X7Rsd/z6nwqSiWocBTRg5+Z/3sFupMyH6zOJsxhAyNYPggYPgIzGdBAG0ogWDWIamgURUcOJExjR5v+78SV4Dh4ZJDANgoPESw51hVg7EGxTyVH9jbtm7meQKz+aa+fDWJJVfOheB9JI40ktceGXevH88Vkx1/Y63vAQ87sAeUf7+2dt8hNu1II5USklpIWx8vdNcSpukEJYEFNJcD02Nv0l9TB1svbE/b+eDraHEi3sT1paCBfDX2hFF+QLS25xNhBa0AuRWuUFTec+YUrchzNh2jZJK27/W+F+YMJmPCqW/aRXMZJq+yogM+KkEYhI18kVz7jE4NplY79O46vNxQYGetRQz4uYcprnDmfShBEYoAm1gR+FSvvaDlU22yh0uUbx95Tof5un5sRhyjfKFhShCZNYcIrWwwNQGim9CzFPQ4m1IWYd3+TpEu4HErKHs0Z6TGH+Hx0KutGz6NfoDKmLAHgtO+YrkiFMIOUlzmWTrwlP+apSz882PedEDvvGHmX13qP3JH/Bh/67Y7CI6g3yOc9Hn8XrPB/l8SEv6CQrpPgFGNvqYaW1n0y5wB9pKeaExMG8Rw/0v8aonzPdVFCE9WnHL9hSFlf78dnkhec5j6n06N2j9PkeqBrEufDzfe74NChky0rayP+IWvwTAQ3HyRCYAB4cwpukSHtmQQWGF9p5zzaf8O9DJxt9/p0vIemZ3epS9dx69bwKi0i6fW3FvQmqHdM5nDGBO5ym4Vij6tgDVRem46qUijGrYM5cQ4XsXz8a9mTxbXmDSaODNg7v5/fJbrBwZgZAs758Yo1vN7M5TuZjZj+0FP8slGiOn8tplM2czBjO5+QLi7rSTIZ4Fn46svfYyTX2P8G6vTqqefc1OT+ZmuCRj3NNFyXLLxL/oCE+5w2dN1+FkVvSgQEoGWmx6Bv2YyqtfXbpnsQXk0t4FBZlsujmXeXuPUG7xgrTtkFeDstaaOGebwe5ymOKdg6tVIXcTMoITeTOJMFTwTs/RGDIdekc5h0HvrZzMmkybgD0Mj/scZaLBqDWSZ4NtZZBXrZAm1bnQsd2rFNpgRGh7FYin0tic3y6/Q3JRc4rsjWHIRYiREPWiOZMFvYbRL/InVBY8kvjWhRxPHQU4xBMFFyUgA3DVOo+fsqLZXAaZyh7CgKCxM9QZtoWVQcGFeofdDnYb7/z1OlesNRlzJ4WaQHUBrz+fwAsDF/F0D7UCwpDYL2gftL3BJGpvz+08WFNXEpWygH33MHVLPI19j7NmRCvEpJVg8odxJeDWlBbZfQl1u8btgrbcrJqsklTVaqGZqZLzYRBdoZCbTl7DX0HltK7JY5schxFRT0Hci1wJtvOAWxlZNw/dtz9wJrAD2M0/k56vLi3GQwODncDdJgfiaMuu8riHlWQzfFaITDa31myaKm6bIb1KDeS6FjSF45Xwk/4cZCrmg6Bh3Oh7ngF9vmfd+HWqfWe1f4ZdQdA5BU4Z5XK/mAJJqNbRxgDbg4B0h0KQSwTnDL9wOn04G268wK/lWeDmwCaI/4qvm45ks0cMuU0TMeX8KT924ip2RaQz3Bn60EgtadlzI+8fXcOnJ39g3XV5vACguTiPP8Z6sGxQO4KsG2W+IlsMr+0/wJHkcbQyfir1W3YwUZRk+8b9WUZq0DcyX2HwK3yfKzEdPGwKuV/sUnEROxUWFwlEBhLzpioXiq4xy7iEb/wg113x7tXYY26rSDl2i9+P7ZbaPzha3Ex+u/w2HsYsqqszmf/qq387/v1cXc3uXbuw2cy46AvxMGbiZr8sMYTT6sEnep2NcrM7v19egNVbHrNanWK5LKRw2/9DkpBLaeLTkcpW37L/7lTGry0l20eh3gHs93uZIxWwOhBc9Yr+8AH9+eDKJc5nDlDtVysVKQpmwtyuoqmWgyvveq9g4tp8ojzPMyl0HOQ4rOcFAYtFGlzOZgwmI/xHKbEPED4ZypIJyH6NENcbPL45GWHgMdmxRRG0VlfGu8D3HadBrnwdWTtudQzeQh+ft+oT6UVX4MyzeFbvp33QNi5n9aEk7BUZ+31up8msHCEVen0r1sKJqVIvLcDi1Igtt54ls0x6tzRGd6mdSI118xlOrytX8b96kFybg7pGWQqs92e+dxEtkr9ie/wM6e+1vVQbzYYWCxAEOJYyhiVH1khzl4Pldl1D82TYHQwTIzfKH0ZZMi1K9pNkgQHXu5GvlccUObYOrHeW4ohqygCbNJd5t4dRqZzOr2en3U/Gs86ufwjXJRBcnHCD0bpQhMynqdSEqjYVBAEnDTTSIQcUABweQxfhUdoE1DARw8bJ/RodosGZlv77eWnYUjDXFyVFEWbvOseXZ77E3ykejkyQQJ01lqfryeeFcKYS9pcrCq9ucSSXzcU/dyqjvTPRlFyGnOOQ/Zf0TP4w8Vn0Sa5Vw7KkoSqlFEHvigB84wctvX5WXbPObqXEJgGtlXMcQYNZVQLrS2HhPif6GvR/C1A7BmC3c/uHZSwtlfpQLy64weFFCwgPCWbZx1JM5CxaMArgkgBr7c/ID9J0HqOSHiHJAgXOxwBBVlSqfc4pxU356uyX4Osgh27wRmvwqPvnVymjoNEscI2Dth9j9+mCziGnFm5dCQcHS7KRyN8hm0IphcLLNTKedqI9zxF4c0j9XOUaDW0+5EaeQioVJMUU326U+Pblt9tHaKaz4FIpBzPoNGU8E3qOAU6wLrMVFl85oJSKNMJdL1BQGcBr+/chRDsAYcuScL3zOmWdgtjTuCvHywfX9WOGBgB+ol5WZK+95q+HxdAkZ1T9du6NyfRdQFKRVHnZfXemVCTUGKEiA9a40Mu1Xg6ymfgp3Pyk5pxSwLcrC27aySiN4bfRfrikO/S3A8jYQ38nExrgEd8EtHkOa5OgwdDybUkSF9Dp3O7ZM3I4Ul7pn/aLDAgIoMLuR3pJLF8Oa4JXinyuqb1fS7zhUEQi2Kwq/+CLyVz1bkVHi6KElSG9D+/8tZmU4qZyoBdA8h90rhpBj7A1tAnYpXo2EeI6Wvnvp7jKG5tGvv700ifwRdRFBjtB0/Kn5Tu6xnEsfw7ppVI+TzY2Jq8BQCdWEeN5lg/7d0OXJgczRJwexfkw8BJgjOZN+bELL6K98GJdPCsKivwH9XPsxhsvMPLpzXTtr5alP8Y/K8getVdxcxQEa8HZJq0ZG6e9xxPWM0xsPpEfL/zIL5d+odJSiSjKmX2q3Mx/BqhrGP+X9l+x7z/7nzBRFGnRooXsYynxm8ZPF98nvSSWty8lQVw9shGNiFYrvd46sYoA8zpIXCX1QEn8HQIeIMSYi5cIE9IFuYY/gH8vfi53I9sKT05rqo5ou61m863nebX7OIbFqYseLjWA25Tkb+l9nyRCrdWyB1Z+/x27bk/no2O/Mn1LArZIueSDxpqPYLpJ1+DNdDbKkwjc+Znp7s50Ct7EuKbv4VrqsFh3bQTjS3G+bWJ/uVRQQSPPKGqq82mul6QkmpVdUJ9kxCROpknMuZiW9y7E/B3Cp5b5dz/d+fAafxZg8gjn4SwbXVLg0zJFX5aAflR03EphpT/zuo1DTFdIdgHrpnzH3SaduNVfLUmDRwtaTH6TUk0j5ilyDKIIzf0OYkdgxo4EuawM4KG5TpuAXWg11eokl2dLDhQsIbGwJe0Cd6AvUxS3goex8vwHlJndG2Q8BAhVROrg6yIwByl040UDq4uKWBHbkq3tBbzUdQkueGxj/J/FrLs0hje+OXPPJMI0JAmq2dz/WcwGDprNFHXcxBt5sMgHIjkkP9iNT5gSNgydRmLSyO6JRgsezSmoDOSlrhN5rpNcFhAkebTPC6X/72wwMG369Loedvezk8BhazWhYdK30pCMzIRm77J2vAuhbtdwSVoMe3vXoXedCzbS2OdY/TkHD4fOP4CTtHgebzjM8x7U+BUPOvcEG/078aM/FEYJkgycg3kkfM7bXjDJFf7oP0jeywgg+y/+GOvK2KaLuXR+xX0TjrVWO1a8uqU/H2ZJTAdlcCWKINhFimuS0oK7Q28VQaRaG0Kh2cQIZ5jq8b68Vw1w3dCCPCv83Hw9zkUNfDfAG3kQk+CG1kn+Ao5L/5RffD2x2eFsQWM0zgrmgs6d1OImuJmEf63xrxV0NNXD217gXCW/145xlF1QIgF7QI913MrrxA8XPmRVZZocWa4xUhE7i1/0mznWpBtBIQp5yNxTTHbNIVoHPnoFU03UU2FvTGFlMI9lwV0XOVrUlL6SBdF76GPuxsKQ81LBpCwJLr8JLd/lUrSE7quwuFKkaQPWSkhZJ6HHzzyPl7aU/eXwZwnYjP7y37ZZKPRay0uJbZmyV90HU5PwBa9Htyat/Uq8O7yk8ru7VBPglsKovtcQbHIGnEXwYunxX0gvUbOnahHlIxt9ysLeQ3DKli9a/HPfZbKzjuHOMNhNIYka2J/Twlekl8Sx+soC8p0dQCuiASImY+ASPYxQrU9VJ4ZtZlw1MCsbptxW9A0FaPICW28+R2FlAE5O7veca4z8+4byJl0JGsHG7tvT6vvy1Jhb2U4ed4NuRmijYImD9H56OaXTN/IXjNW3VP62yaH8PsaH+T3GIlgV6NuoqYhj0xn2WDe85eq1deNdTnkYG66/gOApLzb7iSfpF/kzJp2c8SftLNbI4wjotRWI1gIZA0Xw78VPF9+npNobT2OGiuVW5tWJQxXwkid0cropP3b+WSYWrOYpNxjnCnq7QhoS8LNspUfYapKTv/9XY9+G9fWSZ4IAHB4Fl9+u+1utzMwib2ih/QaliSIYtWU08TmKWJWi8ut08NXZz3lkQ4ZKctDj7kq+rFFC8nRSA23GNh3L3e4j2GQ8A4rneE9EOiA6+fPJiR84kiwlMLXWXNjaRJJSEvVoDO51jPi6aWhHazgvJXpuMYsfcoMYnQEnNXKpQwovk5r1IJerYFoW4NZE7l8l0iLhXdwshdzFTS1JGP8VnbJj6e31Bvo0eTFSo4FHN6Wx7MRPhLucgGvvQ3k9k2Ss3woWecO5UOit2aS4aAPlhgx2B81mRb4Be20POo0WwsZy12dFTV/uBgBTgH1rALMDkpifC5miAr3t25U3C0zM9JD61zkJ8tnGYrOwL30jrrpCNh3nvv2B/y0L2H7pG5b7wvZgiLAclW0jxD5NwNUIjleC3uynPsienqyPvYCrBpxs6v51mV5d2VoGZksFFmV7AY1I44DWTHXX0qd4r2pfvaYa55r3TqO4oXqhiECtDW8RepiASvkYVvu+Vlmdqdb4yfphcXoGrtp8hvsmcLjUFateUcA0FxJmKGKpD6xt+72qCI57U7IqJJnfRmq1X/DpQrmuCdGeF4gLlSdoay/D05hFd9NsOPOszP9w3FMs6tNPdg21JoqSxOGTbvBa4yfBUeElYhJHPa9zM68zX575kpOuB6W/+9UURtykE53uDufDQCi8ID948S16mebQ0e3eUU6V1YknNifx6d6ovx3/lgGjgB52O+/mJrCypJw3cgu4s2kL4bMElv1UH1s28TnCDyPDKK32wubTU3Ycnb2A+CZ96dD0ZZq6K9ZTinvU0NrkzX4vkZT+CSuTvsfVRxGP6N0pEVvwQpeH6+55rQkCDG/0OS90eZgVQ5vjnPS+fF+dK2VmT3ydUmjhvhYcZbKrCwg3HcTLJMVKzZohvX+jMiU5xsLLRJW/R6BrAp6egipeCS9ZxEMeZQiAgA1HeVtKExGzd+GkK+LXS+/wZfIV0NYs4Gt6bpXpWvD1mc85mzGYqsaLZQoeuRXhpBRLzJhs50eg5+Y6NQKrW1u+ObucOwU1+YXiG/WyvRVZ+Ft3EC6446SxohccWJKiAVwbUWH3A+x8eWYFtzvZ6wt6ERMh+gkEQSq2HE0ZJytAAoiiiSIbPJkFh2/PkN9r/z6k9b3M2RtvE1A4QvVd6HTgWywVbIcYFsGgM9CpPqF+v559DTL7uq2GrlKMGFk8j6fbz8Dd+RZau4I5Wp5Gf80tCq0QcBcskY/K/U1fZUleCeXeNYoOwQrJ36ipuI6/wNAet+jp+iJY63u8Op6zuyFLSr6XxNf9Lds4gVk5oBfA1EDfUC8xhO2uP6GLh9yw56D4Gtz6DMoSIeJhdtuSuGOGudXbwF3ek9lQkU43ozRW+FYo4iTncOY7jeHrIvATUcv91tgLHnD6RPk9ZRRrbRmwGCnW3WeVg3t3Vdv5y2qrU0h4RvM4+4KhzA42jTqfUyKk4aGBFv7St1dX7KsuopX7L0R5nie3PJQ9iU+Du0NMUXwTt2rpW9VaXYm/+4skmeocCo3nIHq3oYW/NNqF5j5Gum6CxJLMPwvWyrrfmekOu4O/qQM+AnB1Mc80HYQoWHA15GEsPlRXXK+12nEr2vOcVNC12yTVEJ+upEQ/R5AoyayfipEXVESTD90uDuCdfJhcdh5LlKIA2mIBU7fkkVzUnEtZfeU9rMtTcUl6l05GyLXCD6VP1Beji+N5zlfLIy0ltS1RY5HADKU1/RmT1/F8h4fxMqWTXNSMKp0ceOL4Td0p6ihJZ2tESQHBtwcl1IOCDljWQY+aPNiBAXBwCBoNFFX6se3WTCyureXXdGo6b3rm0STrBV5r9iNi0g80ZNGeZ2nTrOl9e0ZagXygB9AbZHLTfWvlpt9ZWCcjq9dZEAUL227NpNJdDkppnNyF3wIkpR5njU0+0KRsoFlGL5q4peNnzEXnuLYQBIh9hsLG35NfEYS7IVu6344WPILzxp/oH/0dr3Yfoxr/tohpPLk1gbm7z3BXK38HysRGLPprIwUnL/Kw7QmI/7peKcOzFXvyPq4b7/tUtIStNWNBjFQYfLvPQHRiFd+d+xibr/yaLR6tOVUJu0Lg6dCuMh9+vUnttJor1VIBNNZJDRj1L1jKq91Hc6egNV9tusWxPXtksvTjgPf5+4LsMFHD+xPtlHv7Um0HZ+rneUGQlMdqmbkWm0XF7NsRv4NJ6ybx9Zmv6/b5f90aqmH8X9p/xb7/7H/C7HY7xcXFMhmC2oDAZtdSbAkDo4PET+ldAsVD6DSVlJvd2e1tg/afS0joyixoKck/9LTHcerOs+CkSEgDoqghTAshzvEqH2FjePzVPpwxrCKkz9Mqt3P2z8zq+ASVWd8y5T5JBEerZQ/cLmjHoaSHyCyNRtDJZXquNjrFxLJzvNb4T7q5vyU/gCmQDGsvbHaRKa1ew61kS71PI4KgY5CLjSvVEktJyZgQ7/7M5XB4NAu2ej6oPsE2H3KXqQyI/pbXnut5z0JMALCEe08oI4F3kALS+8kVLgbe0Wp54MEJAGRboQQ31fFEEVz0BXQPW4tQomYPON38iAhtIjqfFiofQMtmVfzyziq6N5EzNDUaGN/sXZ5s+xzZ5dEqxF2c/TPe7jOI5zs9xkCnMSitdkxf2HsI7ikKuRH/Pmy48RJBrvH0Cf0WquVSiHT5kQHWvjydDWkeapTa8dRjLPCS2D5BztdU/mqzhgqLK69taEUf3b2RgAHAYOBH7v8sfkTqR+PjHUyeFVonwVlBUbArukaU8z5C3a/TNXQtGrODPKTdDpU5OOmK+OPqG2y43kDhoekc/iyVEifuuX8REBDAkvffZ7jB8LdIsVFjPuDTYWN4psNTeNtPwKWFMlmbpJLW7LszlXKzO2J1moTYrbGULuW8e1hKQgoC4NkSoh6tQ+B+Uj2BebmwNhC62BSIz6LrjHQ7SXj+ZO5kP6B6R0pbf8wjWfB7AASbCtSSc1pXQGBqq/kkX/76vglHR5tSUcGpC5c4WC4y0hkCkbO6hNIEBjgJONUEWbJgS2siIeoERw9f5eq1z9hSshGcI2X767RaPEVo4ZpKbrFCEifhG0pj3ehkBOeqGBnaE0Bo+gprC4PQJcDia1MlaTYHE6nCw5jJA+3c+EWt5tqg1Wr8vxCxma/94A1vcFZIcglJq/nIB9oaJFkaWd8M51AIHU1+hTTOqyQJtSYKGr9CoAhl0SDkKt64tE185QczsqFXgoIBYLPgY0qiS8Zi2t7+gwhFU3dd7k7GRWyh6sp6lpw5CP79wOALbT+BkJGINTdQq6nGIBRKi+JJNggbA5ZSkqv96JcG4zPB5q5IaN9Yxpm+s7EFreC24j4DMOAExj7bCI57DK2iCAQgtHgdU9uXGOvRG4rk44hddOJg4sM8tTWBQ5pdqn3fPbyJp7be5NOT31PlI5dPvh17lucz3HggDd68vUm17/2SRezuykSXb/krFKyxQLmiGBM6igez+vN5EVRVKpDdAHHP4N1+Mm2ibzBhyoMsEsUG55oA/n1D+f13p/LopjSu53ZTbeOdt4LPfeELP3jVW1HUN5fgnPM7JVXejF1TRomvGuwgNJLASl1C1oFdnWwCYFME/DVK9iedDjoEbSXS4yIrL3yE4NtZ5r+sWczYNaU08z3MEK/H5SzeylyiXA7jZsjhxwsfcKvxHSmBoLD9d6fyxOZkeR8vIKfxfJ7KhrgEN45lKIA2nq1JavoeYzJAiIc8jfqexZnf49mO06nI+u5fjX0Htv1OK/+9jG+2CCddEXT6HhrPljawlDNBHMUbXlKvtj8rFff63IvMjmtBYmELJqwtoTraYS6qyoNzL9IuYAvlZncKKwNUL6gh5xBPu0NpNCwaL5dxoiwZftfgUXAS4maBIB/g3M0nWdS3L+0Cd+AipsONTyBfAgIJehf2Jz7K7QKJFSBgq58vzCWYKs7jZpDGtLpTavkOhE8EIM+aSJ7bQUBajMssbQuLPG5Rbofvi5HHu4IAYeM4XgmfFsJ1u6uMEQGATxdo+goMvgD95EkGQYDCSn/Kze7EuB6CC6/IEvV/ZmzmjdQY3DSgVyZRi64TFPQrt83wbF4Vgr8Dw8SzFfneMymsDODhFq8z3TtQktVzsBLBxPVqWFwA2VpFfOfVji/LnDlcIfWvE0QH5rLNisexccz3hGwLFFXcv/C/iX/HAr5z7iKDneHxLMjQK2TZRAO2isZMd4dLoQZVv2acgsmucmN/MCyLfUF1fAGBrWXQJAkKPdvLndWFWLIPY7v1OfbrH6r2XX5+Jd1SYZ4nRFhPy3z+GfO50yiD81XQ7tQcqVdkrVVk0Kb6CTqHbMDDmIWHcE3eq6amuNEnFQZmFWL16i3/4dgZdL45gBdy4Z2k1uo4yFzCx+8X8e7IGSycrGAeAERMxGncNZhsR9NYXsxz0+cyufkCAl3j2VfxK/SUzzcmXSmuhjzaB23Dqeqi/JrvPsbXvnpCtBDhdFWVaG+wd0/AAOnb7r2T79jEo5nggiiXDQeoSKOV6XNe7v4I3zyiXh9w+R1e6TYWsJN15/e68a8hhY0h1K+XHPtJPgTsMVfzl83O4kVf1DF2yqz+HEh8iNTiRqr5NfB6f06HwaNu8JinQrov7wymc5NpH7SNDkFb0VeppcFNd1ew4jEnHnvl8ToJ4TqzVvPo5Dz0Ri0BoWogRKfgzfQIX8Pvlxdg9paz/0ykE+FxkeOpo3jjuk3eTy3/PJN8+9ApeDPdQv9EOPGotI4vOC9JRAYPhWE3mLGwN98uPiqt8x0suPhDxnpl8UcpjLw+Rca4IflPDMcGEeJ2g4LKQPIszepZmM4RMMnK1bALFFdL+QVlrm7V9cUsOCjFR5X6xhAyvO6+KLcVDg2FIzUqG3knaV04lFb++3ml+zie8HIYgYx+0P8vLlU+g0EsZ8skDf4Jkxquvt7DdKW3aK2Hb4vhepaCoSQaSLdWYY14k2HN5+Gi6FE92q0HZ9tK87nGkUFSXQR3fyXY6UL9Nd2n2Fd3up6tpHUVEO8+nxn5Bbw/pBEtqrrLdy6+zlOaw/Q01R5LcfCYaXyYn8C6Uvgx3009jgBOThDU72Xof0xWlNVo4IGolXQO2cCdku4wySp9xzWm10kPKzsKZrmqmckvhkfS2yQpmmg0Omne9WwLN5ZB11/4M1GKgQPz1TFpVOIKtgaB5224FjxF5fc2efOUO2RFUa9QUGvHH+VKmFRAyq/Q3HeeyuQf9qGuYaDvTe3DtnJobwBvQdHOoeAio1xT6ecEB0Kgc8jm+mdbmcko/yl0C5X696oAtvsHEHJdmrtsgro4abfbEW3Se2XVlFOq7yAxt46MhcocibGftJwVfmDUWOUMz+hprEn6BjsaLmT2J717GUTVsNRL78LBofQIk1ipo5p8KKn02K2QthV2tCL64iyuR0CQVt17TKvXUpkxnRwraKymBoHDDcnZA+DZhvyu1/mxRkRLKzrsrHMl0Tyc1GKJDW8Si2BPN7i2RPIXXqJH6G846wp59/AmkrwdWkHknSE2qScdgrao77XeE/rs4Ia1vphfIjSqLzZHT4fSuyyIFegZvoqvzq7A4vOA/LzbLuNQ9QywePPciQ+hiYOS1O0fYG8ffJ2S6R72Jz9MWc++qqr79ow8CrgiAcXf8I1hqrMHr7mHETpkNEmfWZkztl41onvwL/w62p/T6UOp9JsgOy27xkSVHWbnQpu7IfJvvTofQ+V1RMHCY5tSue3xmfyaOnyONWgsE5ov4tfR/riYFbL0ni1J00/ht8vv8OWZL+oYcbWmNeix2aXxQJnfsGm9OJk2kpySlrTw3g6nn5bFu47PJ1fbDwJqQC+xT2Hpc5QfLrzP3cJWbLo5B8FHHhuWt/6Qp7JhTSn8laoYJ4w+eIQOxW6HeV4QYVRnxZwrT9MpeBNVlRl8+sm8BkH+c4BXgZ5Af+QF2QFGIz01IsP7dqd/9zEciXmTHCtYhPrcQu14Xwswt9gsaLVgr4ntRUHkeu51Vl9Zzcm0k6p78v+qNVTD+L+0/x7Bf/Y/YTabjTt37mBzmGk9Mz/kjZ7DcdYV4G24IzEiai3+Cx6gN54mKZ0naAQp2BybB43ngEbHW1cP8vvZb4hLX6D+wetLOexXyLpACD2qbhgPEBmlpf2YiWj8Oqt82qxtDIheiV6T8q/ZAwB6sYJAl3gEs7wIpNNJn+wTWbC5aLr8AIH92V29nXMZA5m98yx5fg6Tts0MZ2ayzr+KZz0a/n3BtxvbjZ2w+/WkY9zYBrd5bdYNZnV8kt5Nrt23EDMHmIzU56SfViubULZqIYF/EJACCRYLD7az84QbRGrBU1CwAwouYLz+InY0DF9lwxw7X32wq4skqY0ealkjsg7BjlYIxyfDnZUylyjCH1fe4KuzK/BzTpa/X0CG/iGWn/oGT2MmXuIVmY/8c0z0H0S7wO18cmIlZcEzG77O4M081UaBKK4xWw36WdWnq7qIZUXf8qwHrC1B1fsCwFB5lea+hyhM//2+SdRaeapd3P9Z7AJ2bN9OaPl+munhYjVUi+GKDb9h8Z0Kmvgc5dXu49CUOBQPrJWw3o+n2z3L/rtTVf08AJx0TmgBgwDamohizgsv8Op779FTFBmoF9WBiSjy6nvvMWjwXHSaKkSNGW9OwJW3ZH01LuSM5JOTP5JTHkZZk89hTG5dsVvU1k+DDQUlyUIYCWYY4wLRoqIoGz6BHwrT+fTkSt48uFvqC+pgrkH9yNA5SHH5KtBaXm14dvtlttx6lpKKfydrWVmdh8aQycYgaGlbLd/gzg9sikpmlAuURAlwc7nMrdGAyRyCNW0WqbaRUk8DB3uqeCUveoDnHUhyVUgy6b3JsUfSUXTjEfcqNRK16ctkpP5OdqgnB/s+A5floAT3su38MiqQ2OiHOGAV/5XG/1Cfc3hWxfHw3vVUeAyTb5ixkxc8YYQzrPRHQtw2YE19jhAt/qaSBXnn0DukWuBgBWgUvReJeox+qZBmATuKpF55Ki/ERDCl2WJea72YiDJ5srG69dc8siGToipfbhb0kmTq0rbAudmQcwTXSolV2j5wO8OrPCXEcW1E3fl7nkqulzcRlS+oezN+LhLRCqDzXau+WJfIBoEsMvPtBs1el+RDHex+TcoByq0+pJfEsffO49jc5cx4s1NTkq02yu1QZlfsnLmXrvaJxHidYXG/ngQXLan3WavrinvfFMEfJYCokGABYsJcMAowcbhe5QOYPOg8b3dswrhOFuKt1gbnmr+Tm3a02mKzRgONvE/w+eDmkLRGtk1e8NsMS4cFefBNoSIJX5mF142H6Bb2J1VWJ6kfm9LaLYPJduk/rSKJWnJbUiQw+IGimOziApOav8XjbV5s8DlpdEaqrM7Eep+ihfMPcrZZ1n6eCO9JSz+pgKPaP20rC3sNJsLjYoN5Rhe9C17FvSku6EVxtYKl5BSMLXwidy0gWp0wmtQnd0W7lHXX52HQJP+rsa+0tIh2gTt5pOUbOOsKIfLh+kU1As5CDu4aOFUFmXY5kAG7BVEwo6lh/8rlzkvgxlLG9/4LL1MaMx++rkqwCj038IKtIzc1frgHKpgtWmcIeRBaL4H2n0lSTQ6m05QS6XERF30+Hto7cG6OJAVWYzJ2ssEPhl6FZvMh7xRBl9rSNXSd/Jybvw5hUqzWT/s4fwbAp77Q3H5cfl6h45hREMudhkkD0H0NywolBuYQIQ0UvUTw7w2tF0tJW5O8755GAyGuN/B1SuJswVQYeFqG8k83d2NfqTMxSXBEUJTMzs7m89b1Ral7ScKVVHtTZIuqYaHW25bI13mlJiS02dTv1+gmo/myCH7zeUQmn4cgoC28QJQOZuSA6OZ9T3mqDkAq/44FnF3uQd/bwfxQDMU6hTJDZQ6tTKV4acCoqUbVD7zbKladPsfZu9NJtT+qOn6fpKUcqsmXqYq6eafQ7u3JgvjTvGJQr1tEQcQowBIfaGyRSxlWefRmZXpzKu2gt3nIdzQXE2ZeSYznWYbHfcZoQzOJnV5ro1J5NTmRgxWA0HBPrDuGjWwpgzerL6gf9J7uRN7sQEvn73Ap2a3e+T7mashjUou3mdv5EToa5quk/367+xvz9h5hQa9heOd+KvMZyi/SXC/wZj60uNVbAgXVWvFNQiq+wM85kR5hqwmr+kr6e8REaPICpG9HoIKfSqBlfKhKug+frjAqA++o5gTaFNLfAAXn6BC8jb4RPyLYswmgYQb6o0g9If9pAj8zM5Ocqkb8dultJjZ/B32KnKlR4TuOX4qlQvSk1LbyA1WkoU1bRWv/vbzZazgeJQ3EFDeWobn9TcNo/YydRJ/3odXQ0fiP26hyf3r2Dx5an82qKwux+MnZx0GFS1g+uDVuhlypyOP4brs14kDJ51zN6UGM11m4+5MkN33xNWkM1bqAWyN8A10wHB8u/c3BroWe5okaxdE8vUJuOnAg5rbfkVUaibOuEFdddv2YL0hcQKPlDh7GTIbGfo7rmYEycOY9W2cAhrLzvNp9NK3890q+Rs9DbA042LM1t7xWciO3Cxcy+3OzWl0E0mjAXpOmc8lbDek1Uuwnp8OenpSXw6jGH/HrKF9VT3mXS/PZUdPu2KZRrAEtZehLbzHACd4LKMYNOdss296L/QXSt2DHKvUAS98hyQYef4QWHvVr6fvJeNala6yVdevna0IEG6uK+aYITlkU759nG5YwlDNVMN4FdBWJqnsCcKQCHvUqhsvvyB0FFyUAjUYrAZMc2Mc6HUxp9SojGy2reawambRuWPUvrAuEdaVwxRwnP67OjTxrCCKScoO+OkOa652C68BRrRJXMr14LofjbkKGfAwTIh7ma0sohbYGGHTmUvrb7+AkwF4xTjW/onMj3wZtDOBjst0XoObIQG9oHqtty1HLQP/5WAu+T+nI6TDoyx/yg6VuZpHfWcptMDMbbhb0qPeZglmbvZEDiQ/TOmAPn/ZvJJNEpcmLlIdJBTibUM3Cjq3gzPNQHA/bW2O/9SXmcgksYtWUS99Q+HgJtKX3RKuFyJxnWXttHsmWIRLAotYC+nK2cHpdMUa2JLJWQNYB/J0TAdgZ/7Sk0iOI4NkaPFqir0jl4wLobQKfMjmYQdTYCBY1eGtgVYAWfaICeFJ0nU5B62gbuJNNEzVww2E+0Tphc2lMoQ1CtfC466eQsrHmfgWwq2IDBxKl+McqmKDtxxBak2dr/jqP7aokpbiJ+posZRirbtT1+36u7XjYLI8pHL85vVBYL8XfZC6ES0W0IFeJtKAat0NHsba0gKvB89lqTkfwbF7vq8yCwkvoxQr23nmMj0/8iCAIf9szcjvSWnXGS4dp26eAlj2SGDbsFQI6z5XFhnnVsey7M5W8imDVeSVG76/re+xqV/Rtj36CPyzZdaBLq2JJTvpOvPe7M67pErbdmono7IPS7Ha4kNmfA4lTVMW+QP0ZWvnvZVTjj/BBHks73uszuY9A3731YOnCy0wMHEH7mp5914zLoH197kUM6Mr66y9TWCmtG5TX7O3kTYvEb/j5zoN8lKGQOLDbcNEaCLtwhYd3/sWxXPkcB5DfZDUP/mElxD6YPpp7t4OZAyQhsS/nIDBGdOWD0FB6TWnKn7+fIMdwiD8z12KqyZFZNdKa8VzURJZrItl6a6us2KfTgcEcyJCz1RwcWIyuZtw11+SGVM/n/0FrqIbxf2n/Ffv+s/9ZM1RcpE3AbrqGrufVZtGQ6SBbE/IgF3XLKa32RKupwttyoF4+A+DKIgL8nsfYpje/PxguFX0czW5Dg4Sw+M2mSBYBbI6W+irdyzp8AQNP4+557952SqtlDzzU4g3WjXfim+FxiMlyfXfvgjUMcoKd5bChXN2jSaMBq13H7YK2WI0OGv7mYrj9PQBuGtgYJMqRuYDg150ho0/w+5g/6OzXgJbOuRfRpa+BPrsgaKisENNfb5QXYnTwpSgy+4UXEHr25FlRz2g0PCvoSfOLpJco/iNJpF5aLYe2f80LnnA4FKYZFD2aim+hjV/KIyNvMmWKgLtHAyvQQeckVmdDZimV0OKRUyDmSZlLo4GrOT0xaUv4clC4vIcEUGTozu7b03n9wD7WlCukOczFhBiO4mnKZN/dx6j27Cv3J3zDxwPaczWnB28e2AVO8t4FqXfW06zgIKsDILxQkbjT6EjQBrIoH8Zlou6vA7TXv87iB3pjEJLv2x/xYaQJ/5/JU+lIPvQCcz20uGvA1MDsodHAuYyBvH90tVwqTKODmKe5mNVPvVON2eK/ZKYHtE0GQh+s+/ucuXNJSk2l5xsLWRbuzkwvLZ82aUKv114jKTWVOXPnotXC3N1n+OzkSlK1j8CQK1JwX2P3k08zlp4kzlsqOQkCkmzapghJ7x7Iy7aTb4XwY9PJCFIgyLROXLckkua6lyqtWrpPq9HSPqQ1Qjx8EdsAYh3IKQ/nm7PL0Rt9/9VYEehSyNpAeCgT8pxnyzcIeZD3L7zCbTMcKzeCSV7w8ctdQpeQ9fXXrLAy0ZWs2gDNrrhhYWNIj7zAiMrRLI+8qhpHTqae5ILzYryNNQkRHzkYwmKKYuutZ8ioGMjMZ/8Zc7NW4//N03dYuOcsRTmjsGvlBUrafMQDRxawM2UIL118U14UydwHf3rSJ+IXhsStoJ/pYWS9uKxVPFu+jTg9DE0HwUMuw4NrDPsroJcJRrkoIle9O3uLHmaX5iKjAy5gKpWPE1onz7rg3m6zSauMgAekMenWciLvSt9Ednk4ieI00HtB1gEpsQJM99nMnmBY4Qu6cgXSPngIU7OtvOEFu9o2ULoqT5eYQ6t0cFa9OCD3lNRDsOnLqqKgSSzkh5Eh/DgymDD7KtWufi6phLhK7ErlO6S15uKsL6CxDlJMCimcktuE2f/A25RGiNsN9FYHuS5BA87hbLBE8VQ2TMxEztgHKL1DH3sCWZHwaLaCMQ1wbApcmAfNXmfziUJ6a7X3lDCulZu+X2LEsdgc5HqbPhG/4OucjDJJb3Ftw/4K2FgGBysV7E9TIIUtN3AqbThx3ifRWhUoaqUpb2j2ITj2ELR8C9p8IHO5uMB355axI34Gb/QYJr3rjn5NOrFep1l1eSHLs4rlySTPNuzK/YQ7ha1pF7gdjwJ5vEFFBk19D9PY+wR9In5WgVJC01bxuaEHQzM+w6gE4iAtBEO10NvJjo97scrv06SLVCzG/77FltpnAdLY5+rizsabc5i14wJF1YrkmNbEeu1VXswFSaBU8b22+4Tld29g0hbTIWgLGsc4yikERibh1+c1Vj47h8FWxTgAaHQuLH34JG0nZ+EUrmDsGLyh53qJGd6Albn046H1eRxKeohCWkL/I/VST+ZSvhkazuNtJES87BVwiaYo4l3i8zqqfTXmocklSgfPeUA7neL9covlutCKB53hdrhG+u4bsPm5cMrURlVQrrOSBJlEJ0jz6ccD2/NMh6cptQRIfZ0cGO6iCPaavjuispdHzJMsTWtDC73EnBcyHZKkyetofLsJTX2OsOnmHNYVH5Xur+NvCxpa6eFoCERVHZQfO2UDX+vOY3/mBD+PUrzXgobKBzOYlg3LfWFcx5L7ylPZ4F+9n0ajN7eqpXlTq+zVlrKePc2OcLIShp6Yr5ZMBb5a6otf229oMuk9lc/JORhfEZ5yB025oijr1ogXcmBTKXxwSS1p39Z/D+NdoF0ypHu/JvNV+o1j+eV5xOZPZqiLWQaWwiWGPZ7FrL76Jhez+nLB+qb8WZTewWTYQGsDfOgD2jIFkr44nkWGx/EToXnSF6rzInQMhI2HMTl1cn8yS/oD9veXmAaKa67URjBz21XWXX+FSrufSnZXFMFsNbDsxI8U+Twh8+W0P8ejpyUwjQbFc8o9SaPiZ4jwuMTQ2BXEVCys9xVehTPP8ADvM8gJNPYGwBuiQeqP2nsrTKxS+3tuoKhvGnO6PI6/p5kfaVjtpKDmf/+phOz3332HVgs6TTX+zndAARgtC5/HgnxItECCRdEfLHg4lQ9Ws+bafOl+OQ9V/1Df3VLvrbXe8ncEJCZczFP3HD8qbZ4UV0kSr8oxrNRtCL9cegdnfSGRTgcl2b1acwqmKvwZkoua8+uld6Seni6RUHBOeg/tVmlcslRA6w9kjC0AsymOVIuk+vCkZ5lc9cGzJfaoJyiq8uPZjtN5PtQBwFGWDKs0tEyKZWb7Gfg7JyIWHAcHed3OwWt5qMWbgJ3IwnmwxrXuHdVac+gUvAk/56SaYt9z9etM5zDyPR4jozSGTTfncLjyy/rftZTB5beIMmyl2mpiwcEdmI0x9bG0tQqsFbRtC0VVvmRVNVMx8jVxz7K0AG6Fw/iWCiWVvFO0OjsZZw10TIZcQc7YvyIuYkaGBDjaWbkQzsyChK+l+bHnJi4VP8S9zPG51uFkdrSGnRKLpZZB93Y+HLDIZewweHNNDMdLhD8CQcw+KPcfGc+fAfB27dBjKZH7c45Ihd7CmnWJA1BHp4PFh9fz7blPcdHlSiCbyvqek262KzzoDC/kwn6zgnEYPJT52c9RaIMjoeCa/acEDGrzIXT9FU4/w7iYnwgq6YdJWyZ7PwAIG8Nv5gBa6cGk+B4xFzIgbz3eIvyo6yC9147W/jN6psKQdOjfReQXw71l4GoZ6P+kRcqUigqSr/+O2Q7zcuGyXXHN4ROZm9eDo5XwZRFklDsUgXQu3DWPJLW4CXa7gNWmlxfnG83CpclswnKm413SC5umuJ75bCkFWzWCKP271HiT5mXT4NgjEDIStM51QMNY71P4iSdU/U4NNUO8lykdU8HWurUS7k1hQjlrrkqg7ys5vaRYTNCAR3Pw7YaAnffypf6JURlykKyWEjb3H8MyXxjvUSJ9646W/Cevdh+LSVvCidRRUo/AWrOZEaszaJQ/iVBzFO31h2QKQjqHKcKKk0Q6COwv/UGjwy4YAIFxTd/Dt+TH+o39e3GzeXZdobCgKqR+fLVWwfmXidHXAw9Hik1hv0NxtPNKPkyzs/76S8zrNg5D/k6UVmSV7p9FVMTozV6BsXmklTQiraQRm08m01f/9z0jOwF9DAaOHa2X2y+0t4C2S2WgmJTK7nx9djmTmy/EJeMr2TFEEXyK+/GAE8xzVcfhvr5S/9c+Eb/gUql4TgcHA/Drpbf56uwKVX6NhG/oVxpKlKekqmGQhwy0LRvNGz1H8Hibl/CzyeXQvcQr/DHWjTFN3ie/OlICGdYy3KsLiTTtwtdJUpVSznG1/w5zv8Ing9qgSfpJ5tfc+pwpsb+Q5bmRPJ2i9U/aNlitZ1DAGYoKelBm8UJpte/YkfPpPHIvYF+NBQCvAcuwYzKF8MfHM3it1zl8XaWTjHPdTZe8zXhrwFrD7KvUpVMoJFJpqVQx+wQENHYdJq1THWPWbP2bk/jP/s/sv2Lff/Y/a8XNfmX0miruFLTmYPbL4OLIoOlGkuFZys3uuBlyaVvQF64uhuS1UHgZUtYyyes6NiChwl3dFLnpSzRLhg8LYJ1WPdHg1kSi0K/3h/NqSUIM3uDdnpGPzrxnbzul1bIHCioDyCqNYPftx7F7yRkCAUkv8kIN6UTVl6rwMh20LxPleR6TthjB5rDA1LqAd0eSQh9FFI0Md7Kq+2bU2qYIKfBSWtpmyD0OgQPqegfMmTuX0+fOUdSiK89qdIwWBJ4Tddi69+a5559n+SefoD12jM+t1WzAxgp7NdXpiTz+D6Eej1ssbD7tzLgMWF4IV8oV5xU8FEYm0nFod8b1PFhXoJGZRzNI3w7XP1L7gofCuALo8pOsOAT1RaHEwpZsTZiraoDuGLypGGH+vfk8u4S9dx5v2G8pw82QS3ZZuNTMXiGH45r4A5/7wQRX8KhMku+rdWK50xA+LVRfTq25tZ5OruvD+LmW37c/4mVAjSdt2KZUVLL6eCC7ymwURkNnu6LwVXSNWOft5JSHcSR5AhonB6aHRgsdv6TE91He7tOfBf3V75cl9xRjXCSNdmUPuoCAAF57/XXOJBZyK8/M6WvXeO311wkIkEqZjiwks8ZbeubaekZQK79dzO48lUCXBLSFR6VxoMY8r09mZgcJbavRAKKzJLNYcw7fBn3GiVCwCVVq1RhzMbvKHiCszTD69A2Eyly5f093vrZso40BWhUdVsmQUZbCQy3epInPUZp3qu+D+XcJxZ9NJrp1CCLFAhtzI6kw9pIf17sDm1KGcKISRieGSJKQDuafOZ/eEb8xsdnbPO7qKiEuHex0xDzm5cJIZzCY1UXMV2/24i39Nva4LVUlGUIuPs+8MGkRdSDxIQgdJfNbXVvx9dnPySkP5fFJY+uZm0Zjwxr/771Xp/FfZffEZhfxMGaisSvQykYfTCkL8bm+jZSkt+Q9TPQe4NWO4ipvNlx/kW0lG0HR4DrGkkHcfWRFW/q1YqE3fBZZKHfoPdleOpffS8A5AW4HLZK5xeoMQt2u0TN8NT8OFKV+fF5tIO4ZaP4GOSESs+VOQRsuGr8F7FJv2YxdkLGb0R4HecAJZnqArlr9LADWlsLywgaSALu7wJFx0jjn3kztT1kHp56EqlyVS9SK5JaH4u2UThPbYpV/ZrtpfDmsCd8Oj8KUImeORl+LZk0g/BIAK4MVrOeY6WwwWjiZNoJHNmST7Oswjmi0MDKRH3QNnGut5Z/leS6TYYWz2nC1v6ImAdnqHTZtOsJjloYlMWvlph/g/omRBzSaumJzM7+/GBr3BR8cXV2HmK01R0BBpE6xHNY6Ywt6EBd9AUsHdMalSCH/CHBkovRf1gH13BzQD3pskGSjFKbTwbXc7iQWtaCl/35VQS7K9g0fD+yItykNs91VwZiI5ULF86SXxDGy0ScEpsmTpMRMZ/zaUkSNmbldpkqFYUdLXsOARh/y/YhIegRPlPuyDxOyrxNbgmBvWAVtotQS202bwoy5QYx/6pn7Flsck1Q/m0wMGDWZ/IpgEgtbSWP2xlAJOV5j+hpGXU4UzDSpC8IaDQS73eLNXiPQZW1wcGileV7vgRg+VmK8qrLSd6Vi2f3kWG4sk+RWFUk/x3nKqnGTWLW1RXaNlqIqf8qrJRCDaC2AWyukfjUuEVREzed2Qdu68wdgT3e4+AYAP1u/pH0KRF+PINNbDW6K8Qum2g6V1fVzW52deY6M4W+wLcKD9kYFqwYgZT3sHwhbYuH4wzKXRgNrr7/CoaTJ6MVKSQLU4f19OjiY442v8oQbhCgYJISN4decZviJMMoZOVtMqDkPwV73T6U1z9/DZFcI14GLcgNrJVRlS0zpC6/Uo91rrDbGcNXAzN7V7KuqvKc81SzgJ/5ZEvVnk4mY9n3o5ZnCnQiIqVQk17w78kn8KBLMcDPoDfVFJXyHX+EXjBoFJpPabe60kg8L4Cs/0Cpl653D+bgQMq0Q1MB0MDjqcz71hXNVUK6Vo+W1WnCraEk7lzyWtFwkB0BqRNC6YrHpuZTVj0v2t+TFvj3dedN/DufD4EVP0JYrWh9k7uG1sJX85d6W7V0b6AnU4k1otUiap5XSkCDJNWbuhZOPS+Ojg4k6AynFTdly6zk2V5+ukwustSYeu+gV8Tv7706l0lUuJazVQoUhmZZ6eMAtqa6PMwCBA7jiv58bOV354vRXnHdzYOfVyECGc54dwTDJL15eQALpWEU36vv5NGC+AS5Ye2ykXa/erKZh9l5tAv+fWG0rCH/n24xq8iGfnvwRW9xc2Ta1Y4e7Btw1imScoEGj1VFc5cv+u1Mp1zfQ+sCtkTRuebQEjYIS4dkSOn4FJTfhzs+qXf1cUmjlv5f3+vbGkPqtzFflNYg1V1+nlf8+Hg3to5LiHDAAnnwSln2qk94RQQPNF0C75ZC+DTYGS4oJMdNUMafelk2TksEMcoLXQw+qmMu19+R8xkDOl86sH2wc3vHjqaNYeeEjzA8Wg9G37u8dA9Yxsfk7aAQrVbpI6d7UxMRWvwE8+IeVPXeeaFA15J7te6yVcHkhkQbpnTuXMYiM9vH17OSuP8Og0zz9NET1m4rPhIPqIlHYGL4skvqxRbRQFJicI9hl7MTOMjhdBTFNPGRunQ4q9FIcZcUMffdAi7el+x4yggJrY9W9q7UGi30hD9b11+uR/zyXw2q3bYCN3WoAl6pgbAYObP0aM5fgpgHP2t9soVBlCh0tAWhufgJ/GGXzkF4P13O7caegDY08/4K9vWQA8STvpYgJkGoBTQOpUY3GTooFZueA3be/FJ9vbSwxHu/+QrvAnZzNGMJTWxMgeIhs34KKAlKyTnMhHKKz5cofGHy51vxTSsIeondEb9XvAjzf6Xl+CfdlxgATByz3BqilIs1N/6RFyhGgtDCFhT42vk/qS6Y4WP6jbrGcsARRXHMLle9qLRvqYtYDzD96GYLk++u0GkpMV8lzO8gTlz6GDp+DWyyMSMAeN4sBvg/SJPlD2iesx6pxl4qBG4Ig4eu6OOmPK29wzCIfJ7j4OrOjY3DSFdHU9zCeF4dDztEG75vKWr1LSp9jFNhgShZkBE+Sn7PBwE+pndhWBrp4sHZQzFWho/ng6CouZPZn8ZF1EOygLlN0Dc+DQbxmaorn2XjWipXQdJ7ks5TR1fA8PWrWxarXviKDOM/jmLQljG7yId4lv8rcjt/Y7zc+hj41/dntFrj+IaGa+n7tiTwEITVtFc48B8emoNOBuzGb7mFrESsUJIG9fXjb/Sf8RLjb5XtZHK20v1OKcrQpFRVcOfU7Xw2NY0b7mQ3Gbzqd1L6iY/AWdOWXZD6P/JW8UP0Qb9mHMTNIrrpFaSIDG/9BoMtt5naZQmDF96pj2wUtG+Kl+MrXV+HUulApBDO706N8MqiNLJcHkO2/gC/PrGDurpNkuTyu2NeZG7ldGmQj4teD5dlV7EiQZFUbVS2Eaw7gzMTVfDygA9Ge5zBpSxFsConbu78wMeAiH/nAa2H75T6nYAifRF55ML5OSVL7AoWZqq/Ryn8v+WX6f6VEYTYXcc02B8YWUmBvzZCYL5gQOIKmOZtZ3ONF2tqk67HV9OXTaXSyYp8STF/L7LM00HP6P/v/xv4r9v1n/zNmVBTNagOC2wXt2Jn5vrxBsIO/3OzOTbcV4NtdSnwm/g4P/IWbWM1IZ+icnS6hkRX2YpcXAYjzilP56L0V2nwkIRwNfmp/RRYUxzNt2hP37G3naI7sgR0JzzBty12Wn/oejZ98cZrbbBWv5kpovXd8FbryJfG00H5I28CdrBnnjl/m6/U+0QADTxLe4wfGT6tA85Bd1bOP5LXc+TOUq3iR66nuE8eQy1KjbgdbtnQpHdq2xePKMT63mdlgt7Pcaibtr8N88/HH/GW1srOyUhZwumO/L9vMsajhDxSVVnGtUsv7BXCmRCFdqnUG53AoTYR9fSBZcU9AKsBcXwq37sHuu4eJIszt/AivdB/Lr9eWqhDYEZVL+XpYDP0ifyBEVKOlRBG0miq+Gx6Ja8ILcmfjOUzbkkh6SQPvFlAU/gjjM0AfD1cDJ6n8NruNEc7wuS/qJANA8BB8hv7IyOnz79mzSnoW/06eqqhCYGeFjW+KoFDbUb5B/JdM8B+Ki15aWDYU3M2ZA2HBVbRsodbx13T8hiYpeh7yMOBcmaHe+dTTcO39Bs9NFKFPxC90Dt6ITiiV7olDoBHiep1+kT/jYczCdPdDOFqfqC+PeYu1V1+VzkEDNH4eBp2uY8AIQQ9ysAJiQn/mWqaCPZW2nYvh5bzrDa9621W9F+2+PdhQVMm5MOiWvVrdf6wijYnN3+GD/t15ZEAUB8xmZnP/hOJspLGisuk3TM+WZFIaute5bhJLwqlKzRxIbXaGlec/Iqc8jCxbN9VY8Ejrh+iuc2VjEHiVX5DvnH+W8barZFhyMLRor0ocu1RnEaSFp7Nha4pCApT6ReOsjtNpdjeujrnZ5rnneN7ZhzGClpmClngDzB7pwaTJ9ZKvwa7XGN34Q34ZFYhz0Xb5gavyCHC5jUawqvsqeLWDfns5mzGE2wXtuFk2Uv6CigZe9HiaX0vgAx/UoIErizjndpEfiuFmI/ViSF8z2ZTbIbVC/u6KV9/gi6HNKK325ET6eDkTN2QkJYH1PYg0GiQEavsV4N0ZbkoyMRPX5vPQ+mz0AQokdOY+VvpLskZvFykS+AAx0yHuWei5UUqCKS36CQifBEcnyxPtgE105aU9x5mz6zRntStVu57InMy+u1MorfZUFXxL/aazuRQ+L4SvChQyooIGrU6EGnli1bt7fCqb7Vuwx8KXfkh9YhzNtwcD06BXKpz0Ha2+pn77pYQPUFRcfN+5Zg+g5/6JEb3DCV7N7cuivzbWFV0czT9hItUxsCcY3vBS3y8nJ0gpasLK8x9SZlDHG2TshOQ/pEKvsv+Tc7jEdo7/Eu7Kx6DaOOdM+lDG/lkOkXLUfb6hPz9ceB+9toIA/VlVAaoW1frzpfdIi1KzgWqP/fahLdK76Wh9djEwoSMfFUCyk2IuE03kalzZWArPZIOgkDgGEA4MpHNGCM89P/1ve4H8BbwH7KusZNLkaeg0lTjrChAFi8Rcri3u223E2lfTwyhJciXZFfJ6WQdp7/4VOWVhfHz8J2z+DkkqmxVK70jzR/h4aKWQCQO4+Drs7iQxCPLk8RCWcokFce4FKfGsYH/q7Tl0DVlHoEsCGo1NkkCuzYiKRt48eorVV9+U/lmdCWeelRLZNCCra7dLhceauVInitiBTKuATfSQn9fNz1lR/QV3zPC57xTwUry/Cd8QUHwO9+hH0CiSdoDEmsk9DkFDIVwuwa3RwJqrr3Mg8RH6+b8Df7rKVDTiq/pxoQq+84fu+njlkdEIGvZVgJgAQqxDfBc6mqRmV7mW04N2gdtpb1ok3V8Ha5O9lqYGaH2hP9FdFBLuEZPI6HucOzmXpbhBIcMupm+lkxFa1bz/AtxTnqqWBfxP38/FA7azMxgKzAYCXBQqGV5tWJ8wBz8RZrgZ1eNb/AqpWHx+nvQeKay0upSdZdAvFexeHRo4W/jZH25EqP++8e47jEiXCpyCvVTmc8n5iYWdn+Sc8y5mJkfL+3Naq/G0n8XHSZoXG0pWAmwshYi7YPYZKPcHDsTefR0hsRGE+KqBJXWWf16K5ZXW8h0YdkuSefORr02MBivuhmxM2pIG46Duvp8yo/3MBs/bWHSAsOi3mOYO34RcBnNhvdMUQKlzH4qrfUgubkaJ3mHMNnhDmw+5aQlhXzl8649q/qTgAmxrAmdnQ6YicQdSMStrPwQOBYMvXWn43SviX8boRUW4G7IZEL2ScI/Lqmt2S3yTb/1gWxDsCFWsTKsL0BYcxcMogYoaxMlU5UPEJHjggNRbriG7tgQa6Bk5tdkzLOrbnzD3q2iq5MCl2rjwak5Ptud8LmcHFl5Bs6MZw5t8R6TvHamvmc0iMU/Cxkvbxj3XIEsWoPHdVvwQe4JfS+CFmy+Bi8N2d39F3BZNI+8T7L4zjQOlDgocWmeYZKVoqJ2T2VMIDESVGF5980NmbLuGzS6S5fY09NlZVyRU9SY+NEKaHwCyD9PybgQ9w1cxrul79DQ5tHrQucOQy1ywvIlRW8rT7Z7BKVchswi4usLIkeClJnkA0L/JeIYWBNF+hCIZ7hLJSe8hnK8CkwBxMfJ4I47lLPF3YM3595YV0f9pD6a6Yl/rJdBaYilX6YO4bZbGqFG6LfIdco4z8vw4pnro2VHlhODInALos4OB6VBkA7MdNUjbFCgVW4OHQcRDMtBK/XOzk1bWEtp9Bg5gaq0WnCqjWRUAo0zywgOFV3jQbROeGqmvrd2rgwT4NgVJva5HpbLosKKI52Daq++wMxgW5kGOuyL2Ew2Y/Xpx/uZvBJyfJcUfjpaykU/CQ+nRaCyhka1Z8v77PCCKDa4RzcD3/LMWKd8DJm0FPXzPU2aIp5FTV9W2/b2m090IV8M0tHB3aINSkckMHzceay0B3VVF65NPwuExdcUBETWK0svdmaisF3CtbEZ2yFJJMcpWDSXxdce7nN2HFBQxvmikoNQVATvxeR0pbf4DeNfMg5YKSNtOmNtVAGZ1nAZbauLS/LNwaAQ+195kTzAcrQRNgFxxSWswMjfbyB+lNb0ZlXL7Hs05mjqRMrOH+sYafKkKn0VCfnsENIh6Q72MrM1MU/EzWvlLqhvu2kTY0UaK5wESVzG/Q1fC3K/y4u4TJAU4fK+VubgX/kGwqwSUkn17oglG3OaCph7Ifp4P64uMhRch8ReGuD9ElcWJ4atsVIcpWsqYgsiwSt/TplIBPBwAHsW3IHktzroC5nUbh6v25r/K3VVUFFFudqPKaiJQPAgHh0NuvXx4pOtxnmg7lxd2n6Q4Ts7498l4hzFxK1h9+R1+StwhP+fsg4gnJhLifp13/trEXe2zcv9kO8IkM6veX8OGF8aiNyvW8xGTueh/guu5XSmt8lK9vyV+T7Dv7mPE53fEZpCDoqp1kSw4uIuDiQ8zPOxVWCXK+i47xunB5p8ksFmtWctxN2aTWtyYp7bGy+NdwNpjPZ0ziulrgi7ud+Un5dUWuv1OcZUPK0dG0NVHnc90Tv6ARX37o9O5/islCp3OHbPNCHp3zBaRB6JWYrPrYHg803u8SwtxLFDfl0+r0aqYfeX6JM5FTuKDy8/V+czKFi//j5uyhvF/af8V+/6z/wkTRZHGjRsjOozK+vKrtA3ciUawqBd5t7+nZ0lzQt2uUWlxIcNlpiSX1HWVlNzUu7O2NILd5TRsuafoVH6Wb/3gieINDW+jd4f+h6FpA8y+Cy/D1jgCTnZmyWuP/SupOkdT0cD9B3CmCi5VQ6rZU+4MHMhm7nAocTK7bz9BpbOiGGMupWpdIDf3jOBW3i31iVQX4lyRyrSkDL4qagC1LhokibBVOrjzI8uWLmXxa6/xl9XK7qr6gl4/4I7Vym4aDjjd4b5sM0eUdBbg7uaKsyA9d6uizxaWcinhZfCREuSK4A2Q0OiCAH33qX3l6RLj73dBSuI5Xq4IOeVhJBU1bxDEL4h6KiyuPNb6ZbrpZsid1YXEmLYQ7HqL4ipviS2m3F+ASc0Xsn68XsWYqPZsy5+lUvDeKVQRhNssjKo8wWe+8IwHkkRrQ6YRGTF6wj17VkH9s4B/FhQ4O7uSb4OnsiFLP15+sPDJHLF8T++I3/h5lD9CwVm5/+gkXJI/xHvCX+j7qBeuOq2eC+NX8pl3FWLhBfXJJq2WEPqKZDdIAdbUVvMY3eQDYi1LYb0fFN+o8x/KmMnYNaXczOtMVdRL0K1evsMW9jBHUsarjllrpg7f831mLAdCoHnlVrnToxnv5WuYngWxiUhJeQcTWi9merbDH9wUiT+PVqw4LQX8sf6FDB48mB+5f0LxR2DwkCG4efjhrYGERmmEl7wiP278V+zuuJxhzrDIv0AWYAPYPNqSVRbJvruPsrt6p0oKVnN3JW2MFiZnQoVbG/mx88/zjCmPaB08vfVp1f060uxzBqdDtA6W9pgKqZtlfkP5RV7tPoZysxv5fhLqfNVvv7Fs6VJaV5fyhd3CBruFJaWVXNiaT3hICMs+/hiAZxr3Z3yzd9ly61ksJkUi4NKbfDs8hmG9GvPzcL3E4L6HVTWgqiUIAp2M8JInUKkQVHWN47DNkx1lkOmiKHhU5fO457M84w4dDOBilRf7hJDhrL7yBleye/HZuT9Usqa1U1oTn6O0qXhIYtnFzQT3xtDiLd48sJMysyR/JSh6gFF8g8fcIEBb3+NTZs1fh8az73kfcIuTCjglN6V+eQ5Wu2hJyG9PoaguTp3MnsInJ35izq6zmMPki5bSuI9YmRXNTyWwvainfMfKbPz4Cy9TOt3D/sC5SlEwuVvPCHjaHXmPOQBTAO8/cp7ne73H3C5y1kKdFVyAfX1xd9Lcc655FKnYt52/6T9htdb1QiqoDichvx1dQjZIUlEOZnVvy7GsLugL+yIKih4T5ekYtngwMOZbNtx4kWKN4h0CGFcIfXZL85hwD8j/tSWqXoGiCG/1Hsh3wyMa3KXE2I31119mYPS3TPZqL58vco7yiHdLOodsICG/PRXuCiR9eSodgrZQZXXidPowlYxiptnMGe1fvJQLYlvFPOXdno2hL7IwH74oQupVqTSvdhLLYm+v+xZbqPn7VkAjCIgiTGi2iNVjvaQCxMCTksQpgN1Gu+rHedbViWnZcBZFIuDuTwzzn0GlxYUDiVPkjNfqPEmi/fLCe5wFED4R/HpJhXjlGGMzS6CiqMdgkkXWMwjA2XKdV3uMpX3QdsTsPbBaXyevDoqefU5h0O+gdKzieNxPdaVfpIQ0l3oOCTAqDTpJUkmxmpOMcoZQl7uUVFyQn5dTMPHaQErscLtMzpgB4MEU6Pa71Gew1SK1v9GzML5YArkpQAOOyaf0yraSbJauXmJ5d/kSpmXD4FQNGaan5Mc9O4d1jeXzg6PV3o/OIZvobHpDNRbsi3qTOTnQuIkVvwbqDkN/H0qn03uISQR0HvLzPjKeVzyhQzI8e7YHfQ3G+8pT/Zv384alG4vyocftEIrcx6m29fa286ALfBacLZOSA6Q1Sq/NkhpFupoBnHzpfUa7wP4KpLjX0fLPk9LYD6MAVz36qPbNrGpLRvYQiqOhdbG8GKMrv0YPv9PkWWF1kSu4OEgRV2bRPrc9Ixp9wqCYrxig6S1nEPdYz+Ijf7L40HbCLp5D0CniXddohLDROA9ahzBSkcwGuLQATj8DO9s2rJSi0UmskOjH5dJpgJs2jV9H+/N2nwG00H6gup8H8t5m6fFf+X5EOL5pL8t8zjefZ4Uf/FgMj2ToVO+I9P7Z0Wkq0QoKRkPjF0gOv8y8hO48fXaOWrrSKURiBqdtgUMN9H2/+h4cHIyoEbh47BhPqrcA5DH631ltK4j0yg48vimRkiovNMVyZr2+8CDDnWFDKawvUYzJuafQ7OvOpOZvsXJEGK7Z36GyXR1hr/rdAqDgEhyZANHToas6Tj+dNZHfLr3FwxtyMDeSr7c881fy/gPdKTe7ca7kGYWsuFBfObq6WDoHS4l0HpvCJFWW9p9KCdEdbeG0PKFd4PUEG0p0JFvgXHmUnD2q0YPOreEeYACCBnd3WLkSPn/nPELaZlmf6iJzCKnFTQBBta9oK6Kl/z58nWpkPEtv1wP+NHqsOl/MVgNtAnYTrXOY1zVa8GiOWRuEQSxnaNwXGIsP1/vTd9bPG1kH4dJC9Thy5jlWuyaRNDsJN4O6r/vcLnP5JLYF5TEg5MjZskG2LTzhLq219Tj0TK7MhjVuDParl4RvsHdjjTW0bo4PXMCDGVLPtAhNstyp90QIGswHD66hbH5ZHUNfaU9kwQelHpCjkO+z26Q4NnaGJK/pcHJ6PXw9LIZPBrUlpzIGGs2S4t8ac7Le5tGKh5joCs31CgWNwsuMdd1AU73DNXs0A+wSO1DnRrXVRKBLAgOivlMV/sXqQlwFeC8fWrScrboem91GIz0MM1XK5WtB6k95YZ7UBzDnCBRevSdArTXQjX8m+dsVcIvtQJ80K4eiU4mtVkhG31zOa7phdDICdhE7cnBkjq0zWWURhLjeYED4Z/IiZXkqlCVS5HwaLTAsYIvEyLaUQ8K3iHnHadYspo7ZGRCANL5PtkPbpfdXTWr+Op/dOk+Z2YOsskgsYY/Wr7ur8+DQUAZES+NWmdldUukB6bdzjmDMPkBbIwQaPWjmJ1cQ0WggInsGIjDI3hSxgVY5Wi2EuV3lyXazJNWFWnMKorrlZ5zLGIRJW4y3/Vi9vKjOjU2aNL4/v1S6fYJZkoGtXXP59+LP2x+TUx5GWkkjzHqHPELJLYLuTqR9kARufbTpU3BdWg8jaMAlCqu2Pg6QfY8PHIKw8cQZfmds0yWAgEZUfLDdfuP5nAgKbPD0zQHy+C51IxwZR5BrAkmFLTDonP9V7s7JyZ25u8+w8vxSnIV0yNwjA6f7mu7SL/LnepljB8uK+5PPTn3P3cLW3C7vL3f69Ybua7iV14lTaSPIt7dGZefnoT81AW36OrXcL9CtG1zQfUlGU3VeUBRBI1gxiOXotHLEi+O7mVXRFIJH1vdYN5cQadxFoIsEdjvudgp6OxABoh/nic1JxOcrcrM1JjgFk2aBtinQ/3LDOamiKl82XH8Bs5v6K7eFT2XFqa9o1zyOn/Xaf/ScVupMeARNRrTkQP457NUlLD6yjt8KLkjgGVFfF4fba5l9oo6rM6+S+1IujXwaodNBtTaXdO/VHMjcgEErIeiqLFKipVWrBi/l/ylrqIbxf2n/Ffv+s/8Js9ls5OXlyZpbmlKW8VbvwfQIW8NjEYOkBF+t2W0IWBFqpC4FASnAj5goIdSKb/JnZSyXquBNL9SozPStjC0/QCcj+FkbkGG5/pEsSaOy4GHg0QpEI3NmTLinVN0DerVUXdugfbzafTSfDGyLkLpRdliDXpoRx2bAV4WKBZfWmSpdJDnl4Sw/9R1l3ookw5+uGKoyuXBnCxcvfqyOxGOmEXAXTlTeI3FccElaqAQPJbPUmVfmzWNLVRXhyItE7YEe3DvgHFmz/T+RmnhHq+XBDsXcCHVip3tzhnkopDgz98HmKMg7ISXIFVKctddF3CxVkkC6pvP1yQUneYJWFOGXS++yPX4mL3UcLknyOFi2xyxm7zzP0uO/ctKqkLQsvc1QlxF0CNrG3N1nKI95W+4vvMKAmB8orfbkbMbg+mCh9rcFEZPA/4+9tw6P8vrWvz+TycSNJIQgwd2huGtbXEppS41S6u7eIhVq1N0FK1bc3d0dQgJxd52Z/f6xMsk8Mil83/Oe33t+p/d15WqZPc8zj+695F73YnCgD/WsugSncjKs/ARHSyEqFkOCCZBzWlKb5fO+8tizCuRe/MG1y1Pd3iOZhbax9Di3mQDdMVOzJykBU8gsqktiXkthnbnjyiJxVjwh/xK+Fiv0XyHBVD3GJQtr85CxsspqhQ92L+D7w5+Ta+0izp5bYFpZfCh1BOJUVlRk76o+SaBpzmy3I4706Q8rJZ18fCDb4cWjaZAU7NakHCCsHdOybOwogYvlgAfndFwSnOvwNdiCtQPe/gR2eJgn9ihCbhjPmjVrWEf1AcV1wJrVq2nl/I4poRBfDk4vbRAeLxtFDl8aeMODMXu1cyPgY8nHz1uY/aaqHOc+59awUublQ1h9XU/ImPG0iYepIXAg9Iwm8AFgreiv8IKLi5CvJRaE+KTRo95STqQNwN7mPQ1pYGO5tgp4XZliu8PBe6++yiezZ7M17UW+OvAd3x/6AnuQzoKMHsr7WXDA6yIbS8rB2+1aFyXCsddpU3M7L/S6nadjdEFSoDtpHCyRygTCtFJgNJjIX+G3k+CwcjxVx/pVdhrZ9tHdD/bXh+b6vmf1xjDnxAzKHP7GisOjr1L3WHuslnKig2KpZ5+r7UkU0YXsktq0rbmN+qEnsTh07JQmDxB0EYYFwJxaHqSRS7NEmiXBA/O4zaswIUucbTd4W8u5tfW79Ki3FC+LMWrjzmDUw9cXonPGAhCILgqfvJ7u+f3pXHstL/W+nag8nUyPxZv4oI4EXoQ7GGpaPdAxuiOvBOfhc3KGYYzk9XD5T8g+xpiRN3isbM7m+nsh+flBvZCzPNr1UcjQSgeVN32Rj7bs5sNtm1hXoKswt/piiehKQGQMtWpBC13OvxK1h8o6ppdRvLIIFoZBh3eh2zeaIYsFYrM7klrYiBtqr64KMFTAdZ/2JIxjV+F7GnljlMLbUiLVcZgE7lI28Wb/0TQLN+/x5lV0lWYVzq+Pt9GJaRIua+4NtW8wjAHC9u/xCz+uzWKQ77X3Apk390cuZHVl7cUHKbKHar9ksXIsbAXrj/xAj3ObqOuru9gtn+e1zRspdcj6pAkmeQdCqxcheqjIY7r3PnGh3igYsBpGx0L9CdoxW7CsUzd8Yn4Coa34YNc8DiXfzKWkaEkcutklI5t8RK+YRVXHUqu/yLMpO9bieAJ9jNI9LvTja36Lloqu6BxtIJ2Ycbxj7YW/Bdpkrjc8I/jVBJsxIHwt8PKC1/qOZWqnZzhbeAv0+EWT2PXx9qZEwdpiJ4VW3b1wFBHiBRFeMMgf7XHlX6RG+pfUDrrA/JNv8Ff+UU0SESA3qDn5TrjZN8kgRU3+RQaqePwscKkcQ98fev7OWu+WPB4Gsev3VStP9SMwiGubKwb6+vL9lkDeyARlcRiDlYmrmNl2EmfLYGgiBruT0JaiUjLsqKhp6FA/bRVvVFTyuIIqlVAO6voF0rr7+3QfYQxi2WzQ5tJ8NsROpixY26OprPX7hJ5qSJwdsv2Oajf0CeNqxHscTBpODb8UQjmlXfdjxrH76gQaFPWhuVcYFqcnFqUHpGyAhKXQ/m15J/QoSpDAvt24X4tPMKvOP4qvtYiu3i8Zkh5p9i4cTRkqfXO9tfZXWat3eSFDZE3n5XtpFQ6uLqHrBQt96y/g/SF96ZHh9uzmnIJ5Xgyp8SlvD9jBy0/MFvKnOwJjpDK49zxJouvR9CFo/zZqYRA5GVc9Vu+5bPRrgasVhMXqTYAtj6d7TMErSUtWLem9jToX/fk4B9bn6qphQ1tC50/IsvYhuySahk0CMKDhnTI3HX9TJHs1O0+VSgZbsKkvdiB9UmXlsh7ezmxqBcbi41ViXIfC2sDI0xUSnROg4/tgDZAgrp4MpJwG37a81du8X5iGBYhX27XfbzARhh3hQlZXbm87nRtDJhsPbsctBF54FZ/Yz2D7GE2f6gBbPmF+KVhwEl64uEIyWI7Ju+gM7wwaQu/6i+SQRpwSlQWAyO5cbn6APQnjeXPreuYVu1WfKCWKNOV55JVGcueSNPIauSVjzn8pyXGQZN/J6UaFF0cJlpJUvM9/Kd9xR85JgraP4rFmfaVfva6n9+GABYzd/ikATXx6Chl25+2i4BDZk3yqiADVVflV3oajr1TKTfv5yDxcPw6+LH9au0FoS1Jv+J7zOQk4FtaAuHna8YRlTAyCaRHwWnAOxGnlDon9BRb4ig2og48PHEoextGUoaYJyqj8H/mq2wweWXmGYz46Akqdm7lj58dkOOBofbAlVvxu3yXQ5QvIOUG9kDO0iNzLE90fMPhbRR0/ICZOyLv+Np1fXJZNhy1daGQDywWgho5c2fF95td+kmVlYaTkwMvv/spqh8N0LcpEqtCvBQ8AOYmp2IFob4W30pGGA+qR5tWTM7FPMXHnbPYkuFXY+dRgM+tZfeExmkXsZ1KLp7Tkt4GrsQwTsq+fBZ5o9L2QdO0FsP9BVNwcCguzUBUPSGjmL6akp1f7jGO0l7GNzjA38QHN8+cTDt1/Zlu8qCH9fORjuLHCTo/qC00fxqIchN2SwoFGfgZSAM5yvmo7jx+5kzXNT2M5q+2PzYVv+GlENL1iFjOq+ZcaQjFUJYIahR2nX0lv6TULYPHCbqtDsV1srDwlcqa0rIhlhN/A1pRnyCquQ6hvGjblphAU0oKMlgvZnziSCP8EBsZ8D0fcKv4LrxDgVbXm9eLuqmQgQMtniHeMZkvc3bSN2opXqVE16fveW2meOI2OeiJ13VHQax62Go2Zf+pNvGu/WK1SlD5216l3lQrEZTUJbi8R+7kCZ4sncvuibErsgXgXndH8tKrRhbgc8fEN72tQQ6h/K1nFMm8ZTLe5FjjzAYS0lARyiFb1jawj+FyYxatPxXPzzYbLQd2LY/h+ZBMWTQykbt7HmjE/Syb3dHiFG2qv4XD23dKj22XvFsYzMvBm+jecA4DdWtPQc95qpaLy/gcsOdpYgldJGjEV/ppTH5/NPQP7HuTNZy6SFP0R/SYOMRy3d92BrL30EI8PLWRzmf2alCi2lZcQU38q9dV8WHsDwfbDpBc1oNwriuLcCxy8upskLyGLu1f21QysSURAhFT5eVclAq0Wb/y8pYKttGKd/D+U3/r/FcxyGP+d+DfZ9y/+R0ApxdWrVyuNAwBn/Xv5fN+PBPpk0yBgl1YOp+kDHKp9hiu5bakVGEvX1FZw7ouq8Z0TmR2xh+5+MD0CacTtjsaTGZgAPa7CQ17GSZUzH8v+TkwXzXY96t8Kw4/CqPMQPaRSqq7fa68xvXYrbvOtw/NBrWg6+TXiExIqE30A7aO30CtmKU3CjxgqTIK3NmVHbTEW/ZROu8NRQlF6PP7eYrQZFshmj/BMnrCcbk35Dr3ElAuLakNPdxahC3vuFmmWfn/z47KzDLTZ2I0xSRQA3Ge6Z8Fo4ALXJjVx0W5n1Jgx7I+bQseAfGrZdKzg4CbQ6gVpzOwJnT6UAIKZ3GV4Z+izCEZflgSRG1xGZFRgPO2jNhgq6FwL2JGUm0hEx9wNbMgPJ39jf5JouhvuRdIaHu8yhXOZPXhnxzJDwCfi7DsUNYWNdcrgkk4SzstG3k1HaDF8GwefiDf234EKZzGFZb9/77FnFYhjsIlrl6e6c2hjQhwRzGr7A9FlRslUqxV2XLmdVzZtM96T20uh/zKI/RUSVhi25cI3sPsOkZHwNwl9ePtD50/lfumHvOF0el8uZnUhw3cEdP26UoYTINgnnWbh+wm05Rjuhe+R+/hwqFRbFRUBVxdJZW6FnnrekSd4pPY5vsmF0ghdEgjwtnoR6lXRI0evUX7uS2ZHebOsELyjhxq2xVnO5NtS+GJ2Pn/++SMDbbZrTj7EH5zPncHQOwGSQ3Vs+Cb3M/zUSH7Kg46n2ks1sxtq7a3Hq33H0bXOShrnv2xgk66tdQ9jk8QosemqU/AN53QZHCyF7cWgNyNq5R5ibAWxf3NKN2jxjGY8sOlQfi11oho/gFIplaSBaueC0lJefvFFNl69jU2XJwMm71TMWF7OhB/yYFQS4hC4UJIKp97hyTt24V2jBbZoI7PuzuwFPBoGfVrdKVXMOvh5+7GtroOpqV/oBqL4rPgMD6XBixmQF2niPQDNI/Yxpd0jQppwwVmKt1cJFouTrXF3MtdZIkzUla1Ebhp4sd+NvDdkAF8Nb4cj64B2p1YfxrW7i3a+MDjIeMwceBT23APnP4c0kzn9xHSp1NYRGQBsVif3dHiN1/qOp3+pUdZ5VKOZfDmsLeNafoQtV1s5GnLmfp5qsJPfa8HMaF3gJKIrZwM/42xGT97buZCMMF1NQ79llLd+mRlDPmJip0eMia/E1TDfV6rczBKYx9+UKuAJmYy+b4bHyuZlXH8vpF4xi3l70FCWnXtKJA3d4M743KG/1L4RMGgDw29vxQ+3dMEnwyj7TOo2oyyk+/YRXSGim6EKF+C3Y++z+sKjTBswwnCfa5fN5+Mbu5FVXJvDJS9rJVej+rDScp5dV2/lvcH9aHSisXbHNfvwyd5faRm5l78mBBl6o4QdfoyDMbCyDjTL0AXYCq/SLPkvvouCBf4nRe7KDI0ns+xQJHeXmJTbmuCe4mLWLZnLvsQxfHXgO8qcIXBqVlWAz2IhP2QkqSmTeK35EtpZftXuIKwNx1MH0zT8ED+PjsE38ZeqMe9A6PS+SKYWxnmuDvYOkCScnrxh8ZK1qyAOkjcYgs7WgJrsuHI7SfnNuZrfQRIBtapIW+ObvM7gRr/JrlBV24e2ouimRJafexqo+NheJEntrMMArCt5nntS4O0suOJtrMSd0nEKA/1hdk2M9m7uGVFIOPKCuVR2QZxIpK/uYOh9bLNBvZAz1Aq6bOrU9wuYzasVxI/UsjjtYLfvePjkd3T3g031EKKNC9lHibj8BE3CD5NZXI8s1cFg64xrejMDgwN5zfss8ed/0e47fTcfh2QxyB/qeWPsg9nwDk4QTqQVCgrLqpVKvN654vLB9TTwhkdrxVGUs1H7BUcRNWwpJNphYxHa5DtIAqUsR5KT+rUXONf4WRYXgKMp2NK2aQcjumAZE0tUuxdN+2Hd32Iic8ZH8Pm+XyiM0p6R3DsLIwMhviGQ6FZVaAsms9bLHE8dzLyT01jqlS7JLBeOvsKKOyx8M6IlP45ujC1TWynE5TmwKFwSISa93BiyDcZcgbavGfoLA3DxR9jQC/4KlPfKDRbfGnx76CumbVvDasd+be/2ivMqsQfx3Pr95NTT9khUtUeyorDi/9E9H/51yQi4hcyiehxMGk6qTRvsBrCcnM7NqhMNGxoPuRJ1holcth7Rg6DZw1CzP4EBfh4VNhYBm6FSGcaTAsdqqlpB+NuKCPbN5OsDX6NitKRPH18LA0+dZeCJi7Rz6jp2BzaAlk9zxyuTqDtlP0FttZK9gFRQ1+gEJ2ca+mASPQTucECjew1KASA+VYAtl94xC7EWaNeDogbPMXlZEq1r7uTBum0g56RhewDq3AStXxQbbfBWGHlWJPq2jxXy5/CjBkJMnTrQPHEGH4f7E9t5nkEq1vWqtKm5g8Y+Jr5Jxl7IPSkJ2p5/aNbQSS2fE1l5nxxqFK+R+dPViymwEV8d+IajKUNMq9xcRBy70wcHbkkgRwksqUmD9MdReImqg48bGaP9TBhQ8X42fxSGnzSSWbt/L3Lmh5+Bqzp1Ikcx5J4QX6vnbwZfzeldgwyHHJyvxV8SrJE9hBAyaB2nynW9fT3A4eKfXV0CiWKrNVfLebFiPQhxNDZs8/ORn7l31eNccPhVvmeVODmT3xvV4a6GFetbI92sHNREZKbj5lYlQytgs8H3h77g16Mf0DlyAaxqp6nMyg8bw9cHvia7JNqorOBTgyvZ3SlQEIANiyqXdff4G1CcBFuG8VT3KRxPHcSbW9aKBL8bvCxe1POGnfVAnf9Wu28vH45ZYzjmMn8MurvNuWPbJ/x99RDvbPZnoI+QoszmgRSuT/K3uCCPRt7QIA7mF47WfiFmHAut73KmzAur8qevjufqIskeSbmJj45tg5pGGVCAYgVvX/gaWj0vldMD1+Fs9jhpaQmV3wmI/1h6myWuhMwDle9FRlEMBRbdc522nUals/H3zmNAwz8J2tSgyi71DoAm93msnCKkBdQbJ+9vUOOqqj83dKmzima+Jfxw+BMNKRgAWxhF3s3YnzSK+1cma8eLU/DdN56BDX8npbARJ2wfQ1SVqkmY1znPMthUzQXfjmxOo0S3qi7fCMprTyC5oBkFZeEkFLTV+tTLGtDZURW7qs/cKlJ1+m4oy2E7yyhz+PPe4IH4ZOjaX1z6hbF1VvLHXY+w9OmZ2nU/tBU0vJ3nXo3gxhvh/Q/GVKsUBdrYXZeuoxnZ7AshjJpkG7xt3lgtdmYMvJnABC0hxtdmx9urjNf6juH1FuZEtPsmZbLw1mDGxnhQeBle4WvrL3jmPjj2CpyYUSWl6gYvSih1BLLp8j2UB2pVWHy8crm19SzaRm013seAuuyy/8CBRIkHBjkvaG3tvAsMbvQrbWru4IluD0KKzmfaOZH9MdDdDzoG6ur5i5Pg0g80iTzDY4+BmSKky/7+8cgnWCyWa1KicNmJKeU9ocM75NobEuqbRs/AN/Bf1Zypf/bmD4fEMV2yvN46O9zbW9vPb0yLMWS/lM2aO9dojut/M8xyGP+d+DfZ9y/+x8Irui8bYu9n9YXHeO1YvrCg3eCaYCwWhbJ4QWk6LKkFR1+F5o9R11JAHW9oG4+xh0lQY7YWSw+mMhOnhSFbJaFwYpp5ss8E0dHRvPb667z5wWl6DU3k/UfvZuaYJIN058YrT/DoqlOMW1AizqAbLDX7cDKvAa/VgCF+Ojmm9J3cG9aQcS0/5tmedxGUvVg73vVrfskp48c8eCQNQLdSFcRyWxDcEgQtvUykIVs9D62lt9myuXOpWVJiyvABrcGpN0qHU33lnwvdgf7e3qw4U5efj37E1BVx/HlVJ6MY2ho6fSBJlpWtIW6+6b7YNAg29DF+7l9bggsm/YSsVugds5Bgnywm/V0EkVoDMsR+kDEtPqnsb6GBbwTLT9xDSkETxrX8CJ8sbYCC+rfw7u5VJOY1N24LKN9anCiF1zIgq4buSlkshER0pGVEc+pTYKisAqDzRzDmCrnlwdVqrH/C9clTna+7hqVnnqd/w3mEOHUVTqc/5MaSRkQFxpnvyBW0P/wcnDOpfIgZLxJ2egfPhawj4uCbSLUa+hnp0DFyGbNv6k6LyL34HbgFlldVMVm8rJUSPmfOINJLN+6rrLb0TvqbiRUx3ZBg3ZKZvpvV0aXsqAeJjRGDzB2Jy5gS7GBeNDRZ18xYfZB7GpbWhu1jWDbn9+tqfr1odw0+2vE3g47HmRrS+f4nKFFwzOe4gXVeWudeDiUNo0OtTQyr/74wLt2Qao2gmQ0ymvrSpFgrMYujhJpW6V0xPAlDxUSL+G/4IBJOlMKlgvrGigpgyhR4su9L/PTG0OtKcJ488SPtorbwSp/x+BYe+oet3BDaGkacoU6/h+j18HS8h6w2fOVow2e4HNaHm6NbGANVKZsYnb+RLAck2+oatvW1eVOs4MNscNbSSSGe/5p3Bg2kdc2d3NjoW0kiuND5Y7xGn8fu9EXhRbnDVwLarndl+1hiAlP5Nlf6fXgFxWj3XZJBzwAbd6TAtBomnN703VB0BcYmQNs3jOPBzSEgBpLXGaomLFYbT609zMGkYWTajISXSP94GoSdYkqnF7BlagPatrQVdAk/RZQVQq052g1DWrDhypMk5LVi99UJFPvrqr6svjRNWcxzQfmMi25iTKD71YTaN0H/lSLfqEfHWZXShsuXLfNY2fyf9EKye0VwJHkou6/eYiBo+Cb9yQOdn+Lmpt/Sq86fpvuxOMuwlGWC0ySxtXMCrOsq65jhAAbCoA2SYNL1BXXhYlYXPt37c1X/kgp4W4qFLexVZjpPuHr2nc3oSXHwAO1gcBM2X76Xc5ndOZ3e11BZVVb/Dr7KhYH+EKxnDBfG0SzhN3r5uWhFJhPzxe/h6Kv/2FvRXU66FpBXUEXs8vfOF8fdTS7YdU5DG/9EtFpn/F0kwJpVXAeLvu+PC73+gPFGFjRnP4FVbaRvlL6yRSlJnO1/ELbcaFiM3P/Z1kTJ9YOjO/nxsKyNtvz9MM9LqszROsxOJyL3u+duSfgBMS0n8XchvJEJlyy9tTtO30OT+O/ZVwKTC5sbiTjre8oxX/7TILsMiKTkzomQc1ySgm6w2eCRVed4Z8ffdAr9HXbfoyHetbAt5rVwKGkCHUo/1u+ZwIy++J5ZyOyjr2v7sUX1I7PjFk6kDsTPu4CUK1mG5KltRSPeDCtlbBLkh+sICdGDuDc7gluD4WojRDZLh6bhTRkRCKH+1UslXu9cQUkaG+rCRzUhsERXFVv/VtpeaiwynGbYPBRWtZXEf9YRw7AltDXLCmFDWQAWv1omO0B8nF13Gj5OKrmBg0lCUnC9Iy7YSmMZGFiCFUh2YOjD6q6AYLCxTs8CICIgicVnXsC7hi6I7xshCaJLP8P+BzDAy2ZOWnOh9k1SbRvYyEDEcR1XVnEdCny7GvoPj4u6nUW3mlTx4urT1ZRZEZDayFeq4F2I7M6x4EWczujD3JPTOenrZrMG1AFXv8Tso0KY0RMD8i/BpsGV76cpfCNw9l9F7xET+cPPz1Rh4wWgCXBjxWdmChzbEBLlsOHDiY6OJjrgDO8NHkigLRdLaEvt9So8zg2h8Tzc4idGhJtIpiJzTaCHaREQefARpw2yzlgsYrusbCVzig4jG87gh1FNeLnPRLyTtGTBSrKMRWGxKDTrRVk2nP/KIEePX6RU/pekS6JA3xPbtcuDj/F+dDbqwtfsTrhFq6SScwrOzKZWYCxvbNnI91kmPSXHJYq0bs2e0OguzbN6qXAgK88/ht3pQ0r0LBh1sZIEYg2sxdqLDxOX00Hm7CuLIK2isrAogfDUD2lS4xD1Q09S03q46ve8vKHpwxxPHYS3Vxn1Q07hVebmZ4Z3gugKO9MvSiofrSbRX99ISfi5KohciOgKt2QY4gsuBHKVqAjpM5nnzIS+CzVy8O5zR3UynpU802GH4Ua5dxGpK3gt1I+p+Y/T2zdMu0FRIn0zlhDsBa3OpUAdHXHuhs/x7T2XMP9IqezUSeJTawD0niMkaV3Vn6ZFt8UBqhz3Z6w8pAdrLz5Eg7AThKDzeZzl3DW+kFNl0PFCPVSj+2Q9ydwvlcRtXmX1hUfJKq7DycybDJLlPlmHmBICzW2QnqNroeIdyAw6saZQpE0NNoWjDH8LjAuEPXuKubuk1KMSj5Prk/ytYctnXgUn1i/QGOe6WnoS6n3C8GafEXvU7Zo4yuhoeYXeMQvJKalFbH4/7VyQtgPiF9A+7gcaJb1GYckjYnNYfaD2jRDSEj8/Jy1bKpo2BWv/BdBngcgdn/mw0p///vDn7Nb3dUxYTovC5wj2zaLM7g9+tcyffaBv/flakv/5LyB5jUivDlhdJf3ugpeNcQvKmbVrkRCronV+T8M7YMgOGnXuxBvvRmuJOo5iLEnLqB96mqziulz0flYTVxxU3IoHOj8NQJA1WQgsuRXVbAnLeLFdJ1pG7mbdpQfJDR6j+VnX3FjqCOCtfSfgBrfKvVbPk2qrStQutpbJtQQ49irsuYuICOkX/s3BL3Ho7aQL32A5+yHdOmQSmPiFcY4FoqLgiZu+4sCKp6tVinLBFbs7fGg5kzu+xI2NfyTQclXIXCVV82uwTxYxoaf57di7lNXWEksi9rfns5s7cTGrCxcLdfNAwnJYHMn47ivwq9cLv0idvdH9J+i3XBJtqVsqK60rETMebjog7+/JmYbjz2y/jsdWn+LTvb9RXENL6lR+MTy44gJLzrxIhxqL4OCTVfauTw1iLVMre7p3ye0Lu++q2jh9B090vQ9vaxlvblknJAp3NLidb3Lh5yh4v7GOMRrVTxR4avaGrSOFQKGD5eR05k1qgbfvPAb5+l7Tfern7cvVKz+Sau8CbV4l196AdwcPoJ3PZ+Q0uI8MJziQuSGgtAk1LS0J9glmxrYZPLzyYS5kXqio7Kuo+rPY8PX2JcwvrLLC71p7vP6L/+/w7y34F/9j4R78qGSQuZB7mjqF3xDhn0BKQROO1D4FLZ+B4BZiHDd9EAdetPKBU2UYGz0DvlZf+vvD8zHNDGOEtBCDefjxqma47jj7GSyOlL4MxVrzy9sbBjf6hXs7vEqNfGN1VLGqxdW81tidxkoNS6/f+O3wV7wY6s8gnzjtYEADNiY8R3pRfQY2nIN/qS4Z47TzRHAxPsC3uRit9JTNzK8Ni2pNpe7wPcZzanS39Ng59DSZmeksQBg8ehnPbKoMTjOjNIBrl5qYYrfz99y5lf8267UlUJ69jv0PQe4pIwuwcqeZIjGSog1YW63QK2Yxt7ediXIa2RihJZuZ2vlZfhzViJu9+xnG77wT/KyFTOn0Ar5puoU5qDGR7YfTquZuXr35GYP0UGmLp2h/Bd7NhpNOnVwmiBFz5HkJOpaYJBsBAmMIjahbrcb6EqAn1y5PtXLFjyTkt2Ds/DIuB76l/ZJ3AKVE0LrmTm5vO93w3JO+SwIjfZdItaUeNXuJDMeiGnD1b+P49tGwzyRQhLxT34xoyac3daaB41cJTJZWye9eKezGL0ffl+RqQANN7zxLjx95aaNbxUpQQ0nsViSp1jd6gwnJcLoB1E3SVRWW59HBx8mJMvg0G2OlR+8FNI1T3Ob6WD/uGykBrNQt5GZnXFdAsaC4hLKcAdzbdA6hJZu1X8g6whNhpTSxQX1vDAxsS5fPWXbuWRacep33z1wEP+0v+2DHAqTgZ5wbrywmrTHcbKLwBBDX5BkeTYcRSXAmp5XxOSjLgZTNkLSSZevOXleCM/PcB7w7eBDd6y7Du1y330u/sDGqBveFwNdBzbSVvFY/kajy1VVDu6FF13e4o15b7op7E/K0kiJkHWJA2QlmZsEsu249cJbTynqEthVBx+AA3bxdnEzDsBPsTRjLvcvTJXBpgjC/FCLVbknAjTgFDSdBWEeOOm/gkTSYngXW4IbajRKX8WjmL2wYNp2HbnjIuNPhR2WNCqhrlBkDcV7rjYXDT0vywA3eNi9iszsxfdtq4kLfNmy68MqPTPirgOfW76U8RlvLrcam0ju5gJuT4N4r5ufrgsER2HOPVBGdnA5rOhh7kkZ0lcBb3RHGqhiQgE+NjnBlMcv+/M1jZXMo198LKdE+gDe3rpfElw62jLWMbvE5t7eZwZgWOjKDvQiOvQaOIhhzGeqNMWxPp48hqKnnfn0Af9cznQNvbPwjQxv/xKbL90lC0A3ZNe5j6oo4utRZxS3BPTQSZBSn0NzyHQ3DjvPbsfdJb6KrIgeCg2F/4mimbVsj/R3dT6vxfbyaCYGX4HDde7Ub1ujEz9FPMDARmsWWCktYj6tL4PR7hPo5rq+Pb3AoPer+zfO97sDXmg83H9T0Tu58KZQXet3OvX8nc8CmkwI7+BTLbvciq7gOz2/YJzJulSdUDFuGiXKDJ5QXCEljXTfpaaKBglWtpQdv128Nm0b5HOf3cbUY1fwzpow9LPcyoyoZlFDUheSCpvIPWw1hjoe0AHshtsQ5NA2Xyk+nE1k7+v0tPfKAfj0DCSquSOIp3TOUc5TGKYuwWeAUIcZ1qM0rIuc26hwMNKk6jR4sFS3jkqHrV5oh94rWev57Ie6PqsoWYL3azYDzzdhUEITVT1dxmLqFAeF/U55+C1vOzJQ52gW/KHxiBpBbGsVDNzzBtLYRBlIKDSexpNifZYVgD9ARIQLqsanUjz/z4JNsjNXaK1rwWNlu9njFMGJQHf7wk3GzZHM519fbuEaAg1pW6H4FMoNMpGCxMCsCyptiqDKi3hiZ+3dPgr2TDVt6Y2dXMTycFyUtCdxRmikJtUs/QtpWw7a7c17is32/8GyPuwnJ0kp22RJ+Y2XjRM6Xi6JJZTIBoCyHRhd6MKHVLBrXOEIMC7UBtEb3Uhp4A5P/vsqvRz/Av5buXa9zMwzeJAFWl4yhO3JOQtpOWN9Lqs31qNkTBq2HMbGaagkAf+88Xu49gRHNviTAp8BQwZlh78DJ9P6Mb/UB/jnaZ9u2exQ7G6eS4YDU0rqGhKN7btngY968XyTCes+HOsMNEqE4SsTe3XM3rDWpNDn6svQHLS9g3PjxbCot9aiwcQbpx/WLh/ENwC5g9/r1fDJ7NoXOuvx27F1OpPU39n0/+gJv9R9Bp+gNNPZdpR3MPAirOwrB4PT7UjGnx6FnReEltJUxQVuWK8msiG5yTXQI972Kv3ceH+6ei7OetoLTVnKe/g3msjdhLD+lnK7oiVaBknQ4+LgEeQ8/D+t0SZ46N0uP1Eb3SlIwSUfwSN1E26htbLk8mfd3LdJKq2UdhCPPERMqNl9paTXZKxOcKriD7w59SYk9SObk4CaVZC1ND1aFvM+n3pMPCuKomfAirWvu4oluDzDEx80m8LJBt29oeuNkwv2T+GpEW0ISdS0sXCjPF/9R388+ZaP0P67ZW9t/U/+dPZMNPeNbFz3Gqo7SQ9DPvd+80wFHX6GlX5VPfk09+7wDK2017x4/8vL6Q3za9nc6Wl7TblCaQZ/ig3QzEakAxEes1R+aPgAd3jFvCgjQb5kQ3HSY2Podbm87nYMZk6Qi1I3A6+UF3l5lvD+kHy0dOns3cSWPlN/E08WTaZ44U+5rrf6gHCJb2/xRetxxN+HhMMNEWd43/g+mR0CTeNgfZiSrPt71ce4JgS310JIBAfZOpqgpeFsgs9iLnXhugfIE8Jv5FTHgd39/Rt5yE1/kwD3B0L/ZKe0XMvYx3DKXW4LgvYYnaBbhpiqi7LSwz6Jr3ZVYcOJjc2jvxZkPYc891M+YSsukmfj4GO+TxQKzZilmzwZLWBux2Xv8Cs0f15B3De0Cmj/GkchdZBdHszvhFsoH7Ze4FIidvawhUzqJzOXNTb8TP8KFFk9JjGHnBCFB/geoUweefjSXhjXOaAlEQY3gDge/HZtlul2s/+vsuirJnXDvM0J6ca3RznL8rHl4e5Xx69EPyIp06zOdc5LwLZGMbfkx/t551PI/r/XnO31Iir+b7e2uCND6FXCUMjzHwp3t3mL1hccgREf06vk79F8lPWfHJQmpv/Kgf5UCibSdEDeXZX/vqFYpyh1T7HaO7JzLa5s38/PRj6jrtQ62DtOsKY0CtzBrSH9SCxphD9f6VKr2CA4mDWfBqTf4M+4v7c5toRDaFgLqw6B10lPaHU2mSJX6gYeF6K8nYftFQUQXSYr21xUQoJ239c+f1WYjuaAp+WURNA7aJglkNylpd3v4asAzYle7ED2U9/eu5FjKYI6k3GgsMmj+KDMzbMzIgnmnXtGOedmECG8vhtRNxtZTABYrQaG+XDrx1zXHVaaUF5OTNLeSmFFeDtvi7uSI8x3y204n0Q52xGfsfmE1z/meoVPtTsw7OY/vDn1HYn6iRsbT20SNQk8s+xf//fg32fcv/scgOFjrTHmdfpsfRjWmZeRuWoWs0DLe03fSKOPRSgO+vByZKIdur2SonRl8hEkp0DIowlgddeo9ShqXsqw2dE80aVJemikLfVg7Y88NkGbFpZkS4NOxDb29YWrnZziZ1o/L7Y0TdoBPETEhp+kds1Db9Ni1a7+zTDz0KD+e0QWTQppxzPoRG2Pvk6rANrq+LcrOzBpl3KHzSSsRPRj6LGRCr5l46aXTXMi/AOc+o8xeSn8wlfEcgPQ4+QRzoxSus6IiM4FXhj5B1zormTBAxzrKOiLBudKsigD57cadJK6CiO7CRtUjdVtFUvZtQ18DiwV+PvIRM7atoH3UekPQOa/GJF7YsIvDyTdRSAPtfgvjucVan+HNv+K59XtxNH/R8NP33AOTR+ymZ41PDTKKUYFRNA2XoJ++ZB6A5Y1EmqXtm+a9dlK3wvE3GTNheLUa62HAg8atTXFPcTF7Vn9Oz3pLcSgTJnbzx7DcfJCGkXHc2W6asWH81pHiqNfqX2Wc6xFQB+qO1khwVqLtGyK/8Hd9w5DVCsdSBnMibQChzqOSLHALaqeUtGfJmRdJLWyMvcNsGGA08CphLxZjusJ5UbZgCpzgUBglBevczLu1nufOFJhRUMNYlegbTkaFD15qsQkLUXO+dWHIFuj6NaFh4deVfAgMCCYqMI57OrxGeIkuQJu2nWn1j/O4f13iGyGMYje4JCDyyyLILG1iuJdD497niyh4wt7ZUDFNSHO+yYF+/vB+pNHTb9b2Sc7QiCuNYHaXmZCkCyjlnIDNg6HpQ+Q6o65rLigsKeFIyhDGLnBQGqll3FEYR8+Acm6lPY/UvqB9p5wO6QdVliu9IU4YPfLY7Fju3fktX+X7G+Vdmj1K7Vg4WgrHUnUV1fZChniNYUYE7I+BqOTvtOMdZnLnkgxSCpqQVRipDTqn74IL32HzKqFnvaWM8OktwScX2k9jZ5OqN9QwL4d35WSdu0hMP463SXAXkCrFvPMaRqUGje6GXvMkUOUGb2+R3PLzLtBUdbhgtUKpI5Dzmd0Na6CX1UJooQT2m5bqpHBSt/Bw/fYMbvQLc8ZHUDdd59S4HLMbPod20429P13IPuZZGjL7KOycQG5O5n9pLyRfX+gYvYFlt3vBxR8037F0+Zx7/05k5vaV/HpWJynoLBXiT3UqAI3vgdEXYISJdGTuabElogaYBlF7xSxmVIvPTHfrclYDfXII9NI5vQWxtC99mA61hOhiJjf95ZD6dKtrUu2F9nk0yP3aghjU+VkyHNA7Rldp5kKP36DhXYzplHpdvUBG3jaJ+qGn6N9gviT7wm/Q9K4tDu7Lldw2FJaH4fDSGTxh7SiJHIfdaSPUkP9WkL5D+oym7TQnnbR7Q2TTWj5nTGBavGSd6vYtNDMm3719/Amt25wpj4QTar0sSRk3G8/fpwhfqwRynEHNoc9fUG80lOXgvf8uBjSUagWnEwme1htTmfCxbLmRuNbn2FgXunrp1reGd/K0z2gOlMDx5INGScvWL8k8YDNJBIIkHBvdZSqvbbNB1zoraV9rE5tzP4OJhZq5RNlqUvPIab5elUN64GTtxhe+44nuD+BlMe9jERigAMXJtP6sv/QAeGknomP1pvJ6qgTcLCaVowrFggJ4NgMj+z+wAUdyEnkv4Spj7rmDLeV2j5VT3sD3XHtvY2ujh6gRC/tLweGjW/MLLjM5xIsChUhI6gkLbV4WtYo2r0n/SB2anHielMYQlxNnvGCFV2Df/eLjjEs0DNts4OtdxMBGf+JfpJWE9qo3iqeu1iPFQyzPVp5IoE8OQxr/TG/nRCivqt6k56/4jjnImDvq8dpr5tsDEmCrM8z4+aGnYOvNEug2k9uvBr62MnrXX8zDXZ5gZFGwwd86VPIKH+6az30dXyIwS6t2YgmIJrncj49y4Olzb2ht6dwztCx4jFaRu7iz3Zu0Lnfrb+cohQOPQ+xv0OA2kafXB+7C2sCETJHwNJEFx+oPyoHl2MvUdByvlN3Skye7Ai8Be4H1/IN8WoXceUIm7E8YzbQBw/E6pyPWNX+Mn458zLPr9/PEBp1/qZxi++afk2Rk+m7jDyWvlV5UxSnGeSTnGGzsLwmmDsaKifmXf2D8X2Vsj79DgrVu8Mtaw/O97qRe8DmK9VWvAfVg8Gbp2VeeayAmuZ2AJAVjf9J+PPIsz6yTILMhP1RnGAzdxZn0XjQLP0AkJkTXlI1SrbT/YVhaT3Pe7nJqPipd3sGKH7EWx/LNiJaMbvGpfNRrjrzXADU6kN55HzuuTGTF+Sc55jAqL7RsCYVlocw98Rbe9dzaAOyaBPO85XdOvi0B+SJdLOHCd1Kt7SgzJgLLssVHuroELv9m7HMZcDt/nXqFjrF/8kztOSIHf/kPMRBOz6Kh99+V370mibacE1U93YIacyWvBd8e+oKTdp38Xkgr3gq7n/VFMD0creR9BYb8PoTJC28RedITOsJpxn4h0OSdMV3H+jRYQP8Gc4mMNAwRnvEVXw5vx4rzj5PgrZcHbYSz8YN0rTWGH24uwFZYcS71xleSD3r1gt/e+pW2J30NPQPtTR9mTBIUO03WKaedwVnLeL7zvThu+NzQP5GovvyWB938INTHyU94boEyFanyNdG70GAfIvk7+fkvWFIAv0VDHb36UM4xhli2kGCHAQnQaZjb82f1Y2f4BX458iE3Nf2eT7t7SwLChdYvi+0CRPgnMr2FFxx6Rp7X+T5Ydt9BcHCwFAJbkFiaowwa3wtR/Spt1sGNfqWZw62KDSCoEYUBvSh3younef4sVvCtSVG5zONfH/gWBpooO9zwhczdiUaFl47RG+hcey2f39xB2/sORM3hxAzpmbmqtcgGm6BV5C6G0UljP8aHzpB5D8hydoABa6taAdSfwCeXLnEybQCg7yMdgAptR25JFJ1rr+PdXi2kitkNrmvgZXFQU+2QqnIQyeNWQoLLK4007huEYBXWRmIA/rW1pFBbmBRIePtD3yXkOiOuy18vKs7lXGYPEvJakU5fqbhzI3HkW9vx85EPuJzTweB7WLt8yC9HZe0q0xed1uov6momKk8AnPtSiKIpG0UtSufb4rTLMxfczKjqBgRmLeb2NjN4qvt9BBdo3wubt4MI/wQCbdmsT31HEqSueEFJGiOdTYXsDlwJeknWrModx3A8cwS5pVHmxw34EMyCAjhTrpPVtReKzxvUCG4rluIVPdq+DsOPk5uZcF33qbw8l7qWNbCuJ5FqF3+dfo2z1lcre0I7KEO59Fkq7pOPVWzxcke5QcYzPieeB1c8yHPrJOne3tj95n8l9DmM/078m+z7F/8jYLVaadKkCVa3ld3i7U9hWRg96v3Ngy1Ga9lpdUaw228TF7NuIMwvlabOL7WSOHvupdGZN/i1dVfO1M6s7HtSiZAWbLbX4JkMONpAq/0OwNquwhgpzTRtGk+Xz2DsVelFEaKVUbHb4Z0df/PD4U+pU99YmTA05hO+HtGGl/tMNBojp2YxvO1LrIv4mFSThNwjj4DCi6AQX0LCdME3Lx8GJgcQboXjxnyJLCIxt0hvArN+CXvuld5PYxOwWf2IxjyZ9wXCMn2Zf678+yekAqG+dnrW/YvXBt5OFz9dkL48R4JzJSZyWy6MvQo3mThwIA5BzT7Q8QNhfemQWVyPxjWO8Fb/4cY+D4H1OJvRi3d3LmWPPmzs5UO5d20KSsM5n9kd/5raagsu/ojfsgAadO4uDERdNQZJa/g4MJmLDaB2qkliquWzwrBvP9082Rc3B07OZHTwl9VqrF+vPFVhfjrDmn5Lq8hdWIqvGr5TowaMf/EhGH4CQnQSpZ0+gBbV9Hq48I1UYXZ8HyJNjrbpgxKUdOtx5IK3N3x76Ct+OjKbxOhP4Q6nJmHobuAajN2UTYxq/hleFTIE7LkH/qqqpgwrTyPQC9pdgfKmRtmbD2/8EPWWIuulLMMYRQn81P952ue2I39MkjkNNrABNHuEMXdN5g8zIXYT/O7vz629s3m971ieXHOElFDds1v/Vl7YsIuEY4tYevZZo2zbwae4s92bBPlkEWy9aggGJId2ZX4+bInbQnK+7t2K6Epg799oZIPnwozHZrPaqBPsVo3qLs0GENxUJJCjBhAaEnJdFRP4NuXNLTqn1IX207ltUT6/7djGlGXx2uBb0VVYWkeS+nFz4dynhs2b7BrB3SEwLS9Qks6akwoixQGPh8GL4brohjWAZWnfs/PCE9RREfgqE9lnIMgniwahJ7Rs0Lg5cOBhAmx5nMnozSFmC2vx/FeVa9LN6d+hmsHWulTJvrhQoz0z0kuJTFpMjJ4JCBKsu/wbrGwBZ2cbx1M2ynhUH0PQ2dsb5o4PZ+GtwTQo+dywaf2gw/SJ+Qt/7zy80JFl0vfQ0a+EkYHQIUA39yonNi9JxMfltCc4SpfYr3+rMFJbPAHt3jTIspF3XgIHazrKu6rH9rHy12seoWERHteaqcAWrj0wcv/UqUT5nmZ0809JLWgEgQ21X/QNJ6u4DpeyO+MVrvNuvENg5DlhnF762VjN80/IOy/JwiZTZR86fLrvF/48/jY/jmpk6Gnlb7/I4Ea/8vfZ55hTcEUbeA5tzbk669l99RbGt/qAGknvands9SPfHk2z8IPc2+Elg4Sj//HXWV0HhgfAsJo6o8Jpp2FAKPn3r2J779s0ldZVO6gF7Wcw+rEfr6sXyMhRo1l05mXGzC8nPrd9VbCoAumtV7Lg1Bs0DDtGkCVOu6OmUwm4aTE/fZvHby/NksSwC94BMLEAun0nc8UeD2oAYW1EKtuMtNJ+hiQAzBDSDK+bduDd9G5J4k3IgphxlcNvdWrKjIFSCaZZp3wjoO9S1l+SwIEh+AGUBjbmdLmDzr7QoIYuGG4LwR4Qw5NhUNoUCRqYIecE5F80H7MXSfBKZxvabPBo14e5u/3rWKw2uYZu61yo9RJ1gy5jwWpc/tq8wk+xa2lfazPfjGip7eWbvB6vBV7c2ORHNl2ezPLU7w1Jk8T8RIYEQE5jCE/V2cpXFnMhKplxnuQIB61nRkENngqDdtk/MGzYMH7FPNm8AyG2eaq8ciWj30V6G7/yvASx/S3gbdFlFzL28kmds5wqhXmW54VlbobG90IjoxRnSlA7EuzweU2M9yq4CQxYA/XNn7+OoX8wodUsJvxVQFkbbaWQJbIL6y49TB1vmO7fRLtvnzDSel3lt2Oz2BR7Hwd854NPWNV46lY48xHj2nxFj+LhRn8q5wQce70q4K9Hs0dF7WFcEnT90jh+6ReY6wV77zPa4b4RjJqn+GDXPKlw0FXYWa1QbA/i2XX7Kaj/qnbb7j/w8BnpR5Nv1SVLiq5Sr+hrYkJP06PeUmrbdW0RLnwllVpmPS7d0f1HGLrD+Hm7t2DwZrwufcuKv5cyyMfHlDz5NKLA0YNrU+AY4OPDurU/Uurw52LWDeCnW1/rjZbEORbsDp0xHNlNCCedPxGZs/oTMWDESUmQLa2tTfiC9Ey74XND9aUL7ok2fcWEqj2Cd3csxt+WR63cr7WSqt4BYvcHNYbuP8Bo3XNfli3rXlECDN4iJEgT9G8wh4c6P6atSvWLgpq9KCyvwaNdH2ZMTeM7x5574ciLEjQOqKch/XWJ+INX+txCqG8atROfhGUNpOILaRFQ7vTB6aywGeuNEVsLwBaMJbIbOSXRbI+/g/NOHe1y5+3UK/iYwvIazDs5DVt9twrh8M5Qd5QkHSN7Sr95ncQ2bV+Dbt/DAl849rJ2rCBW1E/8asOt+RCp7beWGTSJv06+Tb2sOwn3CxDfLHWznPeoCxz3qXpHzYhgBuy4RSqVgfKSLA50Ceb5yHtZ49DZBFYfciyB1PWGNyOQWIQ7Ng3iR7WJsS73Qk+eLLgkBJqkNaaJwlc3beWFDXtoELgfLnyreca8rF6UOfxYce5JMmy6auwaHfHq8R0T7giji3pYKrLKsmXtjrkFdt4uf/51RblDl1zwq9mTvWU+3BoMMSrHeH3Of0EwpVhbPGFUH2n2CO3GHGJ+vReIiA6hN57ngWhgFjAKz3btPmCUry+zPviAmLox5DhhZBKkROvUJhrczp35w/m7ELYVQ2CE27W2eOHwb0puaRQpBU04mn2bVh2mZq9K9Ypyhx+nC26BsPZiF9QZjldEV208b2ld2DaycnPX3DCk8c80Lf9Ae1xOB1ZLKRactIjYg9e5D6C4Qt3I6gs3H2D+SUkCJ+a30NpoBbGiDlBnGJz9uLKPpDum9R/Gne3exMdarFEoACqSfW9JpWqb1zUKQTjKIGUjn888wz13SaWeSMVqzwmglAhJxAXWN4w/2+NuolLd4lxBjXEO3MKWuLtJzG/O8ouvaIkSO2+jWbEQ2f28Cxhg7wen36sab/cmuxooDiYP46vhbbCm6OJJ9kIhwdqLpXe7+7wbM1YKJMJvAP9ahIbWuE5CcijeXmV4e5WRb2khvotbz/FSn+ZsiL2f1/uOIeDiNM32LltxWNNvGBRu4tuCHPfRlw3JTw5VxJm6/yQ9TfWqNvHzYGGIkB0cxgq4oLi3ubP9Wwxp/CveRVqf25d0fh0bw53t36RMhUicydWqxOKN3RJMucNPcw7usFqhWfh+lkz0kZYA7jj2On/FCNnvoUd0rKvso+Lzxs3ln3C998lmC8WqCqHoKhanxLPrW5dTY9+9tKwIIzstZZpz8q2wxUsdpYbKvpySHH44/APzToqqyo03wn33wYcmgl7/W2CWw/jvxL/Jvn/xPwJOp5OUlBScTjcmYavneHrdYTbFTuaPC79pG7MH1CWwySAKysKpFXiZmNQnRKv7xHQJWBRdJTBpOTc2HUZxw8lGozFmPIMvZ/NLHiwvNJmxG90tRvbiSCO7rPIY6onTY8IwO5k2AKey4ldsDLrEF9zAriu38NvR96D2UO1g7C9MqlFAtBWGj9FVNeSeJWD/CFZ8/gd/zN6DpVjH7LV4cdTuw8Vy0CvSVELZpWfIqXeNY/YicWIC6qIslkoZT73hGQ0MQZzT6ir/rgW/+/szduoLWMbEYe35ExZ3iQEQB3BiAdQaLAHy3LPGnVgs0qz70LPGsfDO4oi3fkF6Qeng753HhayufLH/e42RAmDzduLtVYaXxWFc1P1rk9ZpH6suPIavtcgYDPevI8ceWF+qYvRVEUmrGe1XSKQVTJeGTh9WyneZouVzUG8sy+N7VKuxfr1Sdr7BDVlz8WE+GNqHOiU6wZCMfXDmI+m3EdbWyKRv+oA4AH8Fw9ZRJr9gqZCw8yDNAtKPq6dRqMR9/fTygirKoCDKuZHZN3ahQ62NeF3+Ac5+WrVB3BwevOFpfK0VSftaA0QOqGL77le+YnUd6FmvJy0i3Yx7EEbslcVw5CXYNsZQocmBR5mS8jnHh71M5Kk3jf0YipLEEb48h6kPPMCW8vJrTj4MHnA3W+Pu5HJOR+w+uurigDqczejFucwe/HzkY0MfLxL+pnPttdzT/lXeaFdfkuZuuFT7Vn7Pg1drgDNP12MCqTZ9PieIx71NJMo29mdblDgMxQ3vMSYa/WtLcKI8hzGje1T2qqkP/I4koEsr/vtbxeeuiom23SYR7p9I+1qbsDpMkqtAYXkY6UX1te+UTyg0fwIie4sc3U0HDNsllDvIckBGkQlrvCybAxP/4KWagdwbpHuXrT6cKHiA82c/5+G/M7DqGe25p+kVs4jBjX7ly+HttWz55k/AoA0UlocSl9OegrrPUMlOT9kIF3+gaYEETlv5YOqY7E3Yy7tZ8Lu3SeLh4BPSI7PNq/Jc65F5AM59ZqzCRZzPDbH3AxBRusow3iPyG17qcxt/3RqK7YLOId81kQ9j4pgVAffU/l07Fj0YRp3j7rfuI2LiFvw6PKkd7/OXSMmYJYdAErfnPhXZn+ZPGseDm4msV8PbGXPXvR4T6NcbGImOjqam3wW61l3N3JPTjWtzcSrTnz9H5/aFPPaITm/ayyrkh4I4qbwxk0fb0A/m+0kQynCwgyVZWG+0cQzIKYkmuaApeaURhuqnkNJdPN3jPhqFHTVKIvmE4VV3KOlFDRjQ8E+C0nVza62BzD61n7zSCCa0/gCKtDaFV2Ec/QO8WFUXbPpqiqzDsCicoN0T8Tr0pFTW6lGcCiiW70y6rl4ga9csx6msOJU3NWyXxXF3s8NcAcjPbu5Eu/LnTPcVYL+A9cQrsm6ZofUr0OMX4+c5JyT5UGo+/wCw/xHpu1YdXLI8bgmsI1m3cyhJKje9Cs5IRUX6LllLY8ZyJVf2abUiCbuFYZUBg+QWL9MvAcJjIanxI9rfKi/gne4PEhzRgbiw3saA4vaxUqm0aaC5VHbcfPgrEHaME2knN9hs8PWBb/jj+NuE+1yU6+lW+dLLMZ5Pb76B+zs9S5RDx7Kv0YGgZjfhVF7YnTZt5bxfNMRMIK2wocdL2OrqTzwdBvtKQPno2Nt+UcT5NuDuENjZONzYhxVIyk+iiy9kZOSxZs1q1uE5kGrh2nsbd4i5yPQIKGoKdQp0AcWofvweNoE9JVA3xqSM7uQ7sM+z3kLLvt9zJmwAT4RhlKeyhcjcV5puUKoAaOy/jlEtvsChvPH1N0bpbfYadPaFN+tdMhBLXPJUl7I7s/nibVr7btNAOPKCrFtp24x2UO5pOPUOrG4Li02Sm/VvEZvAE5xlgJJnr+CSZsjHV+y0HVdu53L0r4Znu5X/HKZ0fIFL2Z1wBuhIdUBgSUtGBsKE8EVaieOo/jAuha1xd/H8+r3sDnGbs718JLAPEmw89HRVwNmF8nyxDc2Iky4ExOAceZHFG5M89kD/TxQ4ClJ+5flek9gadyeWxneZfq9p+EGtNJ87bMFShanrPQbIO1prIDR7DLx0FYsBdYWkc/n3KrlKN8QE7Ofu9q8xd3w41ktfaMZ8IpqyJ2E8LSP38kjXx4ztCZTyLNuYfUx6pCYsFztHJzdNxn5aRu6mfa3NjGj+tSYIj3KCoxQvi4NFp19mXZKJDuMNnwnxqMPbcJNWXaa2/wl6xSzBz7uA4vARQsSsmMcsQQ14cs1xVl4wITkqJzavIry9ZF46rMuPc3Uhfnk7+eYb+EW/DLV6HvotFbsiZqyQ5/T9E2t0FIn2BndIv0x3BDWG3gug/gSwBRn6amv8KVsgjE+Fzp/KB8FNUb5V77BN57q6o5tLJbPV85VkWrXrDuKbSLK1Uyed4+woI9KRzdFSaBVHVeWTC+Fd2V4MdVx2jN4HjrkFJmSLvPbWmw3HU1AWTkFZDYZHPgAHHtE8Y0V1H+GJNSdILmhGmtEcFtToAAPXi/pM/iXYMV4UdkrT5a/2UJGY11ULWSwW2gSFMjcaamXrqmUtVhibCF2/8fCj0Ll2Zx63XiQrJe8fW6A8A7wC9AUGWCzMATYCfwI32qCf1YtX3n2XZ559Fq8997K8ng+rCsE3Uuc/2EKIrnETJRWvnD6pG+SVQIhvOkdThrIwab7xnatAXlkkyzIXQZP75IN+f+Ns+Zw2ntdwkiRJNw2CDX0rZf8+3fsrB0K2aHd4ciY94/2ICT1Nh+hNeB17SXo0msDXWqRN7HvZJG4X2ACGHa2qsnXD1we+5Y/j7/DwqvNSZe+O+hNh2DEh3XWYqVV2KM+BzUNpVP4FbQcPgNGXhLhYgTZZE5jaSaqxDHZ4QRxdavxIdNAlutZdSUDRLs2w6/txOR2Ye/pdrcRx2nZC7EflEBy+HPP+DBpUKFztmQzLm2kkqL1tundu63BY0Uz6Ma9qBbFGKX8ACq8y5taR10VI7jZwEn+Mq8W0/sNME1+uXZU5/I0xsJPvcE+HV+gZs4SB9b7WjuVdgONviWT06fcNLXjwryNqXibESEDaJTS+TwgIK43tBSzdvubljdu4dWE+OTW1BG+rbyB/n32GE6kDCbElS5GJiyztG87O4CMsPvMSAO2zRop94MLVJfwwNIAO0Zs4lDxMWri4I+c4g4JsHG4WResruhU/sAG0f1sSzElrzGU803bCuc8Zc/fUa75Pv/j4E1ZnEmcKJ8C4BGKLhvJY14foYb8N77TNuLjN+f4n2dGqGx+XtCEuJ66y6q/UXorNBlG5I7jpSDbf919dOVZil3iFxQLjx0ul+v9WmOYw/hvxb7LvX/yPgFKKlJQUlImxfzWvNYey7zEka9q3hxdfUDz1Zmth+8XcAiemQcJSGLyZcosPuw/M4I4r2UYJFjck5ScZP2w/HdrPhKYPQUQP43jOSTEC3eQA9Xi6+2S8dt9i+HzjqZuZtWsRi868rGUOAQzewk2JcLEhdM16STtmzxf2XfJ62NBLZDd0uK3FKF7MgNE5MYYxriwSR7zuKPNFsu9C6bmRdwG7ctIfz4GPdCRI76nybzfXUVHx4CNS3dHgNs9l+7knJdCQabLXjH1w4WvTHiaASLgWJxsTMcCDNzzF9AHD2JVwhyGIEJb5PUtv8+WVPuNpYpK+bNgQnnk4kUUTAyUZ5I66w2HAKvCvJ4aLPhDV+hXaxENkLJytMcD8uM9+BhsHaOVrXQhtCf2WsmxzWrUa69crZXfrlPs5ld6XHw5/Qoa3LtidukkCPi7JQL10jAt1RphX7jV7WJhY578yZ4DvuFX+TODlBeNbfcDtbWbQJPKMBLqcVeddVlJGqF86PtYSrLHfayu7Wr3ISxt2UOqoqGxq/hj0/LVy+ErUcH7Jg7FeiZIccUfOcdg5AefZ2aik1UYmYP3bJGi8+05hx7ppuwPSw+vqEthzF9HOI8x6/31G+fpeU/IhLeIj5p6cTqAtBy+lSwIpSURb8GBYjDrPCxt2cyx1MHuynzFIJVotVjr7wTuRYCuK026bsZ9JOUu5NHk139ymkw8FygIacMLux7u+N+HXthpNr0NPMbX1ZjaVlvImEkxtDLwF/Fzx38YVn7+BVExMGtyQx7s+xDuDhuBToIuO5F1g9nMriA66zAuPxGruPz41oMvnEhgJqCMVEDp8HXorh0phVR1Eiskdl/+gy5G7qdXrB7wHG6VbJk+G8HB40CwiF7+AV/rcSm5pFLvzp0llowuhrSB6CNNm+HLnndCvH1IJPnirBGmuLqXM4oPlAtS6DITrgjZJa1kYepUiBa9cPGr87bZvSGVBh3egtklitvlj0GASbB5iSK55e8NXB75n0uIMSm5YYNj0YPYD/HL0fbbH34ZXRDvtYPu3mZVXwOPp8HSKuQxnRATUNVHAZl0PWBwhRJoNfY2V8zV7V1TN7xDpSz06fVjZF+qfEuiuwEi/ij/3wMhQHxv9rJbKwAhAls9gHlxxgX0JY4w7O/4anRNbMr1DLcIPDdCOKSWJ/bB20h+jpomspdVf5D4Pm5BSbMFS/b3nbiMbFAj1TeNqbmueXX/QILtbEjaIGdtW4OtdRCu/3w3buoJ1b2zeSG6XrYbxoiLYdHkyU5dfkgCw+yEPWE7AbfnQ/WdsTSZrN/SLgiYPSC/CwVuM1esgMmMrmrNs7tzr6gWyfP5cwv0TaR25k/JyJY67G4M7Kvl1hjT+mTknZnCu5A7tDhKWSxI8uIn0+osZXzWmlFSXZVZcx/oTjAeQuBL2TZHEReo24/jWkXDxWyPZBeQdOzFDepaWZglb100efG3abP46LXOmrTRe1gx9NS8VPTCs/kLiqKjcsbr1enRJgFfiykJC13Xg5QHv0nD4To3kKSBVXIXxEvxqcr/xuG1BwiZv9pgE5tzg4wMHkkZxPHUwvYKmwfoeGln8Kz4Psy1uEmNbfkJouVFhYcQIKAkdzLG6J6QPpws12kPfhaRbhzK82Ve80OsOg9x+dNYuoqxwUxKUROnkpqP60vrWy4xrdy+9bQUYSERnP+HuYAi1wo+bYaC3le6YV5jfBQzk2nsbz/98Cq+Hw+954BXaUPulgLpc8GtJTz8YU7DZmBhL3QxXF4k8/S5jlZGXxYtxI1dJ/8QID0d06BnYbUzyrMv8nKnLL9GkxmFsZbo17uL3rOrxLoWZg7jtwDMQ5dZDx2nHP+UPWkfuNP8974pSmzavwm2FRju9zjAYcQZaPK3drzuc5XB5DmTsNY41e0gSDjfug3DtHORjc9AuagsxIadNe8M08lvDmJafEmjLxQsdCePyHO6tt51xgfB8/SNa+83qC/61KHP4U+oIxObvplZgsUgCqMM7EtA995mBMEVxsvSG2j4OLprQG1O3wLnPUb41ycjO9UiehOtX4LA78okMSCDAlmf8wsEn+PzmDjzZ7X6e7qp7vkrSRJ4667AkK/XtLUDGgppIBaZZz1yQyonEFYaP+9f+iolt3iW5oCkWXXLKFdDeEDuFVzdv0c5RxckwzwsOPSltGeJ19khIC6nGrjXA3N/e/yDP9LiXHw9/yl1LUqueVxC/d4EfA5osYdfVW7HHmCRH608wl58F9pa+z6h5itTCxhTXugs6f2yU+3dhcZRUCQJkHSJ0bSBjWnzC873uYEpTnU1xezn0W0q9GvFEHh0gBJPrhV9N6D3XMGfjU0N61frUEJ9G5/vWz5vBu4MGSD82X4us5a7qmKJEAryqkmRmlX0//ADPPQevuzpnNH2wMqZgjx7KD3lwsj70SdNKoVMYzxt5fzA1FM6WY6zI6fQ+96bCmTI4Ue5t7Cdu9ZGK4xZPQWttFS9ArcBYutVdQS3fiqo/t8qq4mLwtRbyxbB2dLbpKkOzj8GuO2S9rj0UAmPEnvEOliTf4E3y5wlHXmBzZDoPpkJx7ZHaMYtFfJLYXyXxoFd9iP1VkjZWPwrtPtc0DzyD2LAnAwP5tFl9Hq0BnzWOov8rLxIfH1tpz1KeR1CFhLahXYi9kLvqDee1sntQzSCicKFmuPPVBjzeVRweQ7HKwadgYSg3DhIVk9t13VUM8bxu30Kr5yQpbfXHFQ9PLWxMoVWXjKnRnszgOykuD2HDpfvhpv1C8HPhzMeM7CBV2LOGDhRVExcKr0LyOvi7rhA73O6/C+tjp3I0Zajhc0B6fdZoL/aQHrYQSdg2uMM4BgSUn6BO8AUAWvgtgPm+VTKf2YcZWfMBmkfs547F2Vxt6kaKKs3EcuItOkaLYoehd+z4ZM7Uksq2cqcfsbYnIXqIjPlGQMFFeicEEuqbzmOrT+FVT5dArzMSGt4lCbI2r2vjmdnHpUAi7xxsuYmpzZdfFyF5+Iip7Ii/jaOpQ2hs+R2WNdas760idzJjwE38cfwdyprrYgUJf9Ov/jw+3DWfR1frbOCCi3Byhih6jbpgTNqOSxRSxqlZsLK1kWhYsyf0+BnavCFxYR0sNXvSZmA/uvQIomlz7QRn9Q3mpyOz2ZMwnqG1XoeVLcFRWDnunsgNtJ/QtmLyrcnJ1P6cSe/NOzuWSTzCHf2X87DPGJIL08grLdSOBdSTam3vAEnQJpi0Vkj4W+Iq94y9pvu0GthaVkJ2wm988nZdurZqxbq1M6ntt5Y8awfUbcXsrggtWZ0BFPidIdV5GofToans8/cHL2XD5ggjIjAMP2+/yjEXHln5CM2+aMb8k/P/4aj+70R1OYz/Dvyb7PsX/3ORvpuxLT8myCfLaGxcWYRlnhd9GywkpnGIOAHBzWD4ceggVWt7oybwTS7kl+Ubdk3qFq72GUNC8xC+t5o4nyALf7dvhZWqx8UfhPX1V5D0udPh59ExNAk/gqX1y8Ztq0NAHVId8HUunPbVBawjuoqWc6ePoPNsqGVMjH1b9Afz63jTv2F/wxg+NSSY3OrFKmNBj4QVsLI5Nmc5k6s5zEyotvLvuioqAnIl+GaG0iwJvvnVkoCiWUB7z90iITFMT51Egvo7bhGJP5Mg6rHUwaw4/ziBQXoqFhDcnK1xk+hRbzkN7d9pxxwlcPp9BrbeKg16dUHSSpz9GFY2NzaLDqjD6TKwY96Lhh0T4PDTkuT0lFRTity8vGqdg+uVsntwyh2U2gNZfu5p8nx0d7bRvXDjXri6GJbUFIPMHZuHitRJn/nm/RNBGODnvzA2KQcJil5dZNpvDaBfg3kMbPQ7oUkfCNPcWWVoHEoezv3L4zmQNBLV+y9hZ7oQ2pIWffvgVN5MMclxp0YNY0kBvOh7RRwFd4S25cm8SJpftuN13m6U5Gp0pyRcXNBLrgY3Fe3+0DaAF8889xyvvPsu/axWbvLz07Iy/fzoZ7VWJh+8vaFu8DnmT6hBnRwdi/rSjyy9zZdJ7abxZr+Rwvhyh9WXqQ94c9lxK01unW0I2rTJWMXwAOgYD+Xh3bTbFifB1UW8u2wcq48aq5CKO39B9zOXGJG3DsvKFsa+BjmnxKkNiIE2b6CUwo/q5dH8AZSivfUdutZdxVcHvsESqqOJXVlAs6TRfHf/VPrlNDGvJAJh4BcbmaC1AmvRyAb9/TEmZcO7iiMU2dPoIDpKqXeoEb8++iCjWv9iTAjXvxVnz/l0Hz+Ktne8ZZpo7NABbu/2K5Y1HSXoXqu/BBR6/sqKZtXIhDlKiLaCn8Wknx+IM9HARIrLBVuIzPmhbcCineNca2p+WQQ+QWGGTS/ldGPJmRf5cPd8rPV1FWeN72VJAWwthjQv3TpUlCDVa+m74fibkKRLGLsTNvLPYwjSW/3E8TFzuF1IWgdLooku2/mPCfReQLCPD0E338y79Vsxyb8WLwU3pe3DbxKfkFQVGAEmTgqiWZNyPrr7FWM/ozojJNjd8E5jYNBZKgGG07OE6GHS+4xB68Q+GWhMoOMolYrGhGVaOfIKPNr1Yb4b2cy4HUBADAeSRtK3/gJ6WSZrx3JO0mC3NxPbvE1uaRRegbW14wWXeWXM+0QFxtG5X2PTBNbU1U8yYMdvHCjVGWFBDaH79xKorzXAGJgDSbQpB7mpZ6+vj29uLn3rL+D9oX0J9M4Ux90tMWe79Am9Yxbx16nXuVSuS9hl7JGeK8opCUJ9Rf/20dVL89W/VRJixclQalICkHtKAgg3m9gsZdlSgZiyQeyWNZ00CUNNRUWdQTA+TZ4nRwksDOPxrlJ116QJUik6aENln+KghIW8GwG3BHlRs1x3XGFtpdrfLOEKIs03YIX0AWlkEuyuO1J6SXb90lC14EoWNww7Tku/OSKL7jaXJAU+wneHvuC+ZVfIrPmMdr/7HiBopQ8fzypgjEn+HOD99+GWfrtpwHxDIH9vh5/omyD/H+TjYT7o+SvcXmrsm3bqPe4PgeCL8Mfh2txdWu6xJ98JwIRWYIp7iov5e5edqalwbyqENOtp+I6P1Yf+/jCg7LgQAt0xcK0kthwlGhumEue+lN5h/tHGc8o7L9XBzlIhPehQ7AjHqax8dGMvrBd10swWbxzlEUTHfU7p5dlamU5lJ+jEPdzc7FumdHqOZxpEaUlmN+2TavkO75hfFFuIENBu+AT6LjaOH3wSVreHPXfBxe/N9+EXJTKTugSAr3cJ7w4exNcj2tA49V5D4uIwn/LEmmPMvSWS8Eu6CqtLP/Jgw828lQVDj98EVjc7yF4EuWe45/ZMBt9wkj7tdUoqoa2FKNP9xwop/sbacf/a0gu3JNlc/eXqUjjyPOScoNxeXi158noVOELDInh23QFCfc0rPKOjFXvzX8a7o65PXMFlIWAce1WqpS+ZVHnse0ASH2bIOQGr2gnBzd3GrsDCY4/y/s4FvLBhtzEBFb+AuePDqRdylj7jB2htUqs/xEyQarVT70i/Nnf415aEUlAjSQrumawdb/s6c07MoNgeLP2S3G2lwAbQ8E7ufTSGRx6B28wVcCuO8S84rX2vfH2r/DNDHMJeyKjmn9M2aqv8O6x9lfy3bySq4T3E57YlwDufEH9dHMJ1jPZiWVPc15q4uZJQcZRKInnHrUbSwJ7J0tfPRA2iaj9/wrpusn83BKir1A89hcWiqFPbITa7y55e1Yau9ip5a7NkX3Q0DBhgLmNnb/ogT6ZDnhMc+spQ33COhA9jXwlEWTE99rEtx/J2Fjhr3CAJc3eU5UpCrs5waPG4YduXet/Ga33H4lDeQtR2s0kahR5iVIvPifBPpG60jnhUnALx86t6oYHIptrztWoHaTuFaJGni/kENmRdISwqgHw/rUIQIPN2caJU+ekvmkuKP34eob5l1zwPlAONYmI4cPwg548t4cCxPbw2/X2i67ol0fsv42tbLwqbelE39ivtDq4upVNcMzpaotl55VaUv/a4MyKfYV/iaDpGb2B07Ye1SY3A+hDRjccehXk/J9Mo4/GqftVnPsZy0tyHp/PHMGg9hRVLYqAtB2eJTkEhZjxX6v1JelF9sktqS9zLfa449ir39v2NwYOhZocRmuo62rwsia3AhkK0tev8PCAmBgJt2dzc5DujAoejVAhbVxbCpsFaCXirnxCWo/rKtbj4o+aanGt2jhnbJSlXYomG2jdX+QGRvVmUuZYTqQMAnYR7WQ6cnEGHWpvoFbOI13oNNZCh3ecdzePT+WOofytWihjS2ANZoPULcMNs8Tc7zKySGQYhM5+YJoSzJlOIvuHB6yIkR0VF8/XBb1l0+hUc+EqM0a2Cz9urjBDfdGzWEvz1nNDBm3hizXHyyyLIKta11ajZB0ackkrL4KZGG74sG7aPh2OvVFwQD6mO9tNEjt8Ed9+ex0tTD2IpTdd87l7JfLF4pMQFXPOY00Hjsk/oUkeUcA7Wja/sXQlAVF+mbVvDqXRzmWsAh3IwIglWR3uoSgxuDt1/FrUXPZo/CkN2EJ31G7Me6VjtfXoaGA30s3ozu+gif+Ql8fTZszi2vkOXlxN5edNEvCxepD6fytdNkgguaYWqaHVjs9oqE3ol9hIC3F4/X98qiU9XZR+I5P7FrIvkl5rE2//F/+cwiWD/i3/xPwRJa7i/09vUDrpEt3orIXdtlWRcQD0JJPnXqQoQWLyEWQ+QuJoDOSnYLPBQ8WZhU7lX92Ufo17qMgjrYBqc5dCzImnZ5TPzY2t0l8jNWLxMe6odSbmRnJJaTJxkFGXoWmcFj3V9mIiAJIj9RStVUZRIjDe8mAFz+3kQdPCvZd68FaDRvYwtTmJsjEllQfRg8wXEhcTVkH0YWr+C1fv7aoNzZVCt8+o6un5AT+ABJIiXAvzu58e28nJmuSoqNt8ImfvlOta+UZxrF/LOwLZR0mOi5dPmP9b2LWk+bIaihCoGql7qEEj2vYsahcl8dmMbKNikeUZU1CA+3jOIBSffYORoKxq+lKNUyveb3G8qO0nWIYibJ89n61fECHKHvZDv+j6JV9oOBoVFGLdP2ypSIv2XGcdAns9znxBqC67WOXBPvHpiFVcab+++S/SeDrzSRwwWgyMXUFf+SioCVbYw7bhSVCvRmX1MWJI3H6xIfukwaJ30yoz9WSR1dHhziwQX/hx9HCJuMEoMVcArxBjwvPduOwMHeNGwkZcEegvjRRoH6FS7E5fKoX8C/NqvD5qt/aP5OauYQhMCdCUsXjA6VgJJ+uAciHZ/nZsq//nMs89yx6RJ/PD990z79HMcznz8Qhpx54N38fvUqURHy5vX2vEWtdudYt3FqTTppw3AEtSUWKcEU9rX2gSlOsMx+yijejsZNUq3XQUal8YSHgATUzDOX/XGMJqRPMJKhpx5GjpqZS78bWK1dzC//GJ8FyVC3lk+2Wm7Znm0PsADP7ZnYNfPOZoylLt1pF/qjoaA+mLWR9yglU+2F0l1Za2B8t4VXYVxCZrNnwn35tuo7ixr/SST9BXVNXvKX3GyOP7uyRqLF/hGYHGWSsVPy2e1c0lYW7zC2tLHLM5+7HVJAI08J1UF9ooKFNe74hfFLdFNUONnCHu9LFs7V8SMZWxJR94KPUr7UA96SoXxsG8qNLzbWAlXlgN1x0jFjq43ntMJT3Sbis2rFF/7O0itdhXOnXO7BCZBnSVj1/Pl9jn8MUkn8Zl3TiSUWr8s597yWZGec2HIdnnmanQwPx97sQRG9lRIxLqvBwCX/5Q5IrgZ2EJ45rnnwGKh34svMsBm456SEuNa8957PPPss6xYAd9XxJqfekaCVu4IDSrhhfv2wL6vILej9CZxIWacpv+aBhZvkRw1WWM0cNkneqRuFVmqbj9oG75XYF/iGHytxdzZ+SMomKBZp1xM04WnX2XT5ft4w72YwxZCaeQokvKaExUYh4/dCrhV/eedp3Hey7z3lI3AljXAUUs7h6XtJDhtE9sS48gpyTE/9kr7y+QhaXwvOEsJDX6elOxrcwJTgdDQUE6mDeD7Q5+SlNvA+KURJ5nzgjzP3fUTS5tXhPnvEybVK1a/qgCExQI9fpMA8PbxYl+M1LGKg5vKM6d/7lwYc9nzwQfESELXt6ZU0bSfqVFvuLHWa3TuWM6vRz/A5usD3hVBDKcDIrrSrldT+vgamfIAgYl/80o4gBPn5d/w6uT23kV0lb+EZXDmI+krGGBWVnv9sNngo6HdaRG5Xz5I36mRhSstBYeykVEUQ7G+eDOsnQS9CmJF/rHOsKrK54LLcOZDQmPGw+jfgV8NhASLLZDGNng6KoI69jTALSiUc0qCpA1uNyQoARi8GWviQY6cfZvhObHsBBYj5BL9IzOd66usyiuGWu1f5enyImoG6uRFE5bzcvK7PFwGtzv6Mt9Pt2fXszhEF8h2IW2bkJ7avCJBS/d52zsQoocKwaOhMSET4JVKqF86Px/5gCmDdSSeJlN4cu0UvL3KCPDNB6d/ld3s5QP9lnNDr3rkXdiEjVbae+HyuwrihIAW2VMrK+gohbIsCZCbVYO5ZDr7LjWttiDnpEh21Rkm76Zbssbm58s3B75iXKuPiOZ3UFqfrEZ0JMn5/myMnczArtreZHT7jv5ftSDBDgn+67RyhtlHYEMfbr3hEwj5Bvb5yrvrwrqukrzp97f5u2QLlkR8aGvzhEvLZ6BGR6yb+uCLX7XkSZcCh7HO04jf/f3pNmASoZY0bmv7DqT7aSW8u3yBfxcwTdeFtJQknbNcEmh6Owig9Ytw5mMhSfaaYySAKLvcYxMyzvnM7pzP9GDp+UZwKbszxeXBtG6lwJ3k6BMmyjIAIa0lUG8Gi1WSgrrqT+pPYHs8RPgnEGjLBdWq6hmK7A6R3QkHhifcDHsDoN8S7fYbB8q+rb6SYGr9QuVQDZ842teK5XxmN2okfgKXj1Rtby/kwRueYuX5xziZNgAGu8nNBTXC0us3Tn4CB7ePYMoUUbKoRNrOCjuoPdyiDTiTslFiAx3eFp/86qJKf6XqwDoK0Xj/QyLn6W6b5J2HDX2kzUjH92VdckNWsx949hupugvyzZQq9qYPQ7dvoPljBNhisNlEnaE6Gc9K7Jggz0mvPysr0HslwLzuk9E8Yb4RlLebRtDptaQ2VkKIca+uv/Adi5q3oDCkGyHZ+ySB5N7DPXWLSE33/NO03+m6Sw/gFz+J1BpP89qdWnvElrGBezu8yrlGB2nRUydpWXuoVFqmbpYEatdvxbcfny7zddIaIW0Up4hqTL0xEOJGvmr+GDfPk+Rj3RCTuWJ1W6g9TEg3enT5XEgSF79nTM9n+WOllTtLPTZiqcTv/v6MnTRJEiEx44QYnXcOAupr1ow/b/lLeo9GdNTuILQ1GZHPsSXubq7ktuUzHWcwq8FHbLoM41p+RJfQ76DkvirCQ6vnoNVzeAFB3pelv6lflMzfV/7CUpwMDSueR3uhPKPRQypjXXUqlvE3+4+kUVEsoFXXCq3w/WxeJWB3CBnA9T4P3YWfTxhP3wSgixNc+EYS3MOOwZoO0OIZSXS54fUbOlJQ5EPziAOQOE3b8y9urvh49cZJok9P1HEh8wDsf0ASPRXXxL3iK9PaH/q7Ee79a5HovInsEmhbcxt+xaFARxkLiIERp/h7SSR96v9F0xoHtPNf6hbCyn2A3kQGXGVQ8XA4OxVaPiXjnWej6o4jZXMNPn74K8i/2TymaYa6I2TNC2wAtrEAPNOKf/anKmJ3O904xonW22CYlkmR6zeI11ef5sYmP+CTvUX7LttCaNAUMq5cpXXLMsDtmG3Bsq467VKtaQvRkoAWVahwRfSAm4xqEmQdEmWsZo+IX6/Hhn6QvkP+v9t3QiSpgLeliNf73s6x1MFcLHkKOmh9vuYFzzKw4W0cTBphqjTQrRvEnYpjdPPPIHWMdm3OPEhurFS+xWbHajfMPSstIJo9XCWJq0dQY/k7/hrPDM2BRu+a3qe3rVaSHQ52Ad0d2uDVnfZS9gEjf32FlrXP8swr7xIdKCT2qr583pVxnuzibBraIDNoOwkRvzP3wg080EOsC6dyMu/EPO5odwf2CqUlQwXxv/jvgfoX/0eQm5urAJWbm/t/+lD+R8DhcKj4+HjlcDiqPiyIV8/du0d9/cyX6vwX3ZTKu2C+cfIGpeag1PlvlSrJUKo0R6mVbVXi3BD1+ufIWNZR7TblBUoVpyrlKDPf58q2Sv0VotSO25SKW3Bd57JwoVIjR8qfGYqPzJZjmoNS8Yu0g6s6qJIlMerbA98qp9OpHSvLUyphlVK556s/gM03KbUgyHysvFCp5c2VOvKycWzjIKUW1lBKKdWlZUv1Z1UXBcNfQ6h23PWXDOpWUGFWq2oWFaq61EG9/fydKjk5uep34/9S6uxnSq3vo9SRl3QXK1WpS78qlXNKKf31cEfqdqUy9pufb84ZpUoyTTfLyVHq4uovlHN1Z6UKr2rG8vKq7mNRkW5Dh12pjANKFcSZH0/sH3J/r/5tPn74+apn4PSHns/LEy79otQc1MxJEeomP79/vA+zQXmDGuTtrf4EtQHUH6CG+lqVj9VLzf74Y9nv/kfUiR+nqLhvOqiS07/qzrlcrqfDXv2xHXhSqYs/GT8/86mcb+p2z9sWpypVmGj4eM+e6t+pe25NUJ888Yt68I5zShWna+/3sTfld3NOyb83Dqx8zuV4H1frfrQqpqEuZV0y7DvgnQDV+l3U0A8wzhfHZyi1YYBSTodhO6WUUvYSpZI3KZV7zjDkcDjU4MFFasQIhzp61Lhp6h/dVNr39dTIkUodPGgcT0xUasoUpZYuNfndlW2UWtFSqTOzldoyUo7DDc7yYhU8AxU6A5Wce9WwOdNQ989Gffl9qGHMef479dGX8uzGbxhuft4Z+5Wag6obFqCGXMM8oUANAhXmH1l5n0tLzXdtivJCpeZ6KbVnijxnx6cZv7MoUu5VdVjeTKlV7T38RpFSCSurniM9rixVan1fpTIPVX0W+4dS28Zqn+nsk/I8HntLqaIkeRZdc4F+nVJKpRemq4ur+ir75puNv7ljoqxTf4UqdfI94/jpj2W/absNQ1u3KlX4a7BSc1DOrWMN4wtefVWpOaj9H4xQKnGtdnDzTUptG6fU6k5KreupHSvJlDU5P1auVVGyui6k7ay6HtvHG8e3jlZqns3wcXJyspo5Y6ZqFh6pGodaVauYZurtmTM1a82SJTJX3DUhWTkPv6TUzknanSSuld898bbc7+vF5TlKzfeTdVqPS78qFfu7+TqWc0bmTZP7pJS8D58/9b0cW9I6zVjm8RWq+LcA9cFjc9X06cZtU1Jk+8RvmyjHyo7awdIcWceOvlbxnOzUjm++Wak5qMu/og6sG6MdK4hTaucdSq1oJduazHEuzJwx45rWKQXqRn9/9fbMmZXzwJTbLiu1626lElZofz6vXBUv7yXvkRkSVspxXfzZfPzAk0ptGeHxmKvFlaVKnf/6ujdL+rG9SvimmRo5UilnSY5S2SfkHpgh/5JSR15RKn2PUkqp0rxLqv27qNs/QiXFrzbfxrXWZZ/Ufp6wQp6bTUOVOviUcbvsk0rtnSrbnpqlGXI6ldr6zh1q3YwpKm7BfUqd+UQzHrvgYbXn/dHqgdsvqL3bs8yPK25+hb37V9Vn6fsqfu8D822UUgdO/aLecNnwsX9qB68uk88PPatU4hrP9uGarqpDLZQN1F4Pz1yXa7RnVcX3urRqpVTmQaX2TFYqfa/299J2qoS/26oB76Nu/tNkzs455fE9V0opZS8Wu2IOxn3/Aw59fb9Sc1C3jjX3O0eOVOqLp74znUeqxY5bq+bkOSiVslU7fvVv+Xx9H6WOvXFdx6yUkmfKtW/dtUlLk+MeM6pMJcQVGO5zfnqaeuD2C2r0KHO7lGmowOmoGjPQblsQL8eatlupC98pdfFH7Ybu55t73rOvWB2KkpXz4LOqYUSo2vgPfpJPNc+n628vKB+rVcVfOqeWffyDOrB4gfi8/9XYM1nOu7zQfDzrqKkfOnKkUpMnXlFzXn6rct5yR0KCUqlrn5N951++9uPJOia22YXvPX5l5EiltrwzSfbt6V5tHqbU9luMn++YKGtZ3kWlMg9rhg79Ju/i43cdVWWbxout5YKjTL103zY19faLHn2T1FSlNmxQqrxcN7AgSOwoM5RmiW3o8iucTvP5Lfe8nO+RV7SfF8SJLap/piuQmFjlT8VdLJB14+pyzXcKC038Xk/Y0K/yXMpOfaAWfItiGmreiXnm3888qJz7HzVca7VpiFzfTUPlvPIu6s73rNhmq9rLnKTDkiVK3XOPUgWbHxJ/wP3dKIhXKnWbxFE8IeuIUtvGGOfGtT2UWlJXqbJ8iRPYiw2bJsSvVvYFweJz6XHsLfEFqsPJd1Ty/G7Kx2q95nlAE0dxzaHuNlzCCrFJPeDMmarn4KrODYyNlc/HjS5WH72T4fmdcpQrVZRSdV2LUpSjKK0qnleUIse1/1GlElcrdexN5SwrVCNHKvXT8x+qnK0vaveXtE6pPVPUirkXVdzSirnCU5zFcEKfik1XlKTU3gdMY3dxP/RSm96+W70+db3x+UrbrdS+h5TKPm7cd3mRxM4OvyA+TeIajW8Tv2+9en3qejVypFLzTB77r76S61nwS4gqWnWjYdx1HwzzyJK6qmR590pbOG9Oc6VOfyRjV/9W6uS7MjdcnivX6spi7fanPpD5POOAxPgSVppdOQOSk5PV2zNnqlpBrVS4bx1VK6iVwZ/au1eppW88o3594T21cKFxH8XFSj35YFrV/XdH3gWVE39cpc3tpxwLI7Vj9lJ5brJPyLaHX9COb5+g1L4HPR/8laWy3d8NzWOdR16Wa3HqfXnn3ZCenK/KfvdRa6dPVe/pXWqnU+1evks9cPsFNXKkUuc2LNKuc7ln1ZVlL6jvnv2sIrb3sXb7raNVyR+oez5GvTenk3Ys65hSS2ordfZzz+fldMj75rYWuO5Tk1qtVKRftIoKaKxsFsu1zSFeqP3bPlDvzNuoBo9LUEyTeTu1IFVdzLyorNOtqunnTVV6hkO1u+8bxTTUyD/GKYfTUfldpqGUUurGP25UTEP9fvR3z8f/fzFMcxjV4L86R/RvivVf/I+Al5cX9evrGJ+B9TmXWZ9zmT0I7fIYzYLNt8U3Skr5g5vA4prCcrvhU+psHkJDb7BcAHVHe+023oHmslMuDD8uLKlVraSCoDqpNP3hVLA93ug3CnaHQS9t1zS/Dk9DRw+VeY0n4+ss46HMjXA6S9i9LhTGw7YRwjoqjBPZJhN2G50+gvJc4+e5p0W2Jf+8gUENQMdZwoQHxkyaxB/vvsudJeYSIVaujQkdjTSeP1yzJuc/rqhBG/4ShLlt7ZJhaPGkcQd+UVIdcGURrGoDvedLbz89dk8C/7qG5up4B4i8kAeEpv9IaNAW6L7ZIB8UXHaAb+/7jqzg2/D37g64VUB5WUW6szgF9t4nfRDdewPVGwujLxtlH12I7CkSQfVGQ2QP8+9kHoTCy1KZY9VpqTSeDI0nM3VQCjMX1GMf1feb6QVYvLzo+uyzfLpsCbkZ8YSGhjP2vsc11WR0/Zq2TY+idk7EYtOVs53/SqRFh2z33JcF4PznInOm7wtZ/xapSPHyFcac/v1LXCmfm9zf5s2hZsCVimb3TQ3jLz14gtZp9/HFvh9g40fyfLtYlGHthf3v+r0Ba0G5nVtxCvWsHvoaZB4krn4ZNV1Ec0exttF00RWRwiiIlYosXZ8SSjNh82BhJY66pKnI8fLyYtYsfyIv3keDU79B61xNtZrXjTuYOlXYqaUmal916sBPPxk/B0SqFyek7RBJOV3FpcPLmzuD4ZsoyMs+ACFuEi6lWQzwh9WF8FNeLo/pdm2J/ZmHQ2F+PrRo0AwTnr6wjoGivKJqGe3umAIcLsthYMM/mNjmHWx5f0KkB3lcPbwD4I5/YMMO2ihzaOIqo+xg4io4O1sqvT29j97+2p5TLpz7Ak69Le9z/jltD7pGdxll83xCZc4LbQ17p0g1X5evhKGr77VVGE9k6lYi+/1mLtHnHQQR3UTuzwwR3eR3Yn8SBqvbPBIcDHcuSefW1u8xqXcrw6Y5JVGUO2x0rbsKcgdqqlMBeaZrdJSqEHf4hnuWiXbhyEvCom75nFwL94qLgHpSFVhnhFZyxoWuXxv7/AHR0dG8/sbrdPS10SZwLtHteuFv3wbRVZLC5eUw+8aupBU2xJIdBHlntTsJaih9p2oNMlRCcu5L6ZngXwfC2kDrl4zH5lcLag0G30jj2Kl3hIl/8Am4NUc7FtpSJHlSt8g67aqkccO+hDGklXVg5gRtNYZfWE1OpvUnpySKySblHBERwjreGPcwd/fX3SufUFnHHEVSneKvk9Np/TKPHVvLfSFQ5NRV7pXlSO+mkFZSbWEzMdBOvQtp25k69RdmTp/+j+uUS076t6lT2VuhTxPqmw5xf8g1r1vRD6cogUAvBeWxRqnN0kwoSZfqusb3afu9uMOTakPsr7Kmt3wWWr1glGRNXA27Jso61uwR7ZhyCsPe6mt6PR5fsRenkmfdkroedk6E3gvM7cuCODj9nsgvRfbAJ7gxx8vgeBncV25FI8iatl0q1ls9L8ftrau6Ofi4XI/yPPAJN/5WzjG4VFHJqKvYt1jgoz1zAfi0xdNw5S+NykIN22Ua1VtHj3rLcYS+Axh7KVFrEAzZJs+KC+GdRMbUO0hkpEozpdelW4Vo6/PTiQqBIdn12VhvlHaf0YPFxjrwqMzdd+j615ZV2MBZB4iMgN6pnp+9662sGtvXH9ZWrE3RN2l7FNfszcd+QzlddpJaiZuFne5uVxx5Qd7zbt9JJVw9nUSy1U964jnLPFdnbhstVeKDN2s+Pl84gosJtXAok3Kc7KOMar6N3NIotifcRz//66j8vOLWz6nHL8aKsKDG0PwJeXdyT0ll6fWg3lhZmwsuGSS7XDKCDmXDJ8CGXvU+6NKrfD/qR3JvzAbCtIP2QhaN/ZHiPVO5KwSRP3VVqgXWrzpOM/Z//YnyrINI8Y+O1a7BRYmw9gZoPEX2o7cdnXbwCcPS+UMia64mJdPEJ6vA9Spw1I9wUD/6AWjyOvjq5o7k9VLZlbxWjqmiv+11occv8ucJmwdDaDtDdeqDXV9gVNMK2bSMMIMtVbcuUNoeAiZo5yhHqUh3RnQXf08Pi7WiusdDSGvXJN4fcoVlZ5+hgKaMdJd1yz4uPRebTIGBq82377PA46nGlQ1n75FIsorqYum3WKuX5WWrlGzzsjjg2DRRLKg/QZ6P468TVXcMQ5rbIdmqrb7r9JFUV5ZlSyVfWPuq90qvBGNWNQ/y3k3INqqcBDaAods9nlNI+QH6xFxmd8J4vP0Cocl0w3fcpdv+EUPcpKoz9zExGPaWQN3cQ4BbmXpJhvjq9cZi6fqVcT+95kjV6a47ZL3SVyiFtJC+VmnbpHJeh3HjYOzA01hWf1dxMG42XGB98cdjfxX7KspNaq8sVyRqgxqbvy+dPpC5wxZkLi+ftJa68T+JraGv5AaRE0zbKTZk43u0aiqFV6EkBXJOEG05xaxZsxj1+uusKC39ZyWe6GjI2A9bbhLfpM3r2irO07NEmlQvqVsBb29oFr6fIY1/xb/kYaAqTlbr0iTubt+IP46/w9Y9fjznrpSYslHWsJbPis/rX6tqzL8WXkBlOM8vCm4rAZSsfee/xNLsEX75JYDs7OcJ1ZtnOSch9mdGDr0fCrtA8r3SO9EFl+3vHQBHK2yNjtK6h4w9kLRKKvxavaCtvqxAfo9dfPIK9OkD6E00l8KLGSxeYtd52cQm9L9ZMxyd8CRTO1l5fM1J/PO2wqFlEtMKagSpW3go9GacTb7gpyOzuWtqLSqfTKcDSjPw9/an2G5UCaPzx/hYxVdNK2zIUvs57nGZUXHz4MoCibdEdJNWN3rVpLStcq8a3i39WEszqsYcZSJV6x0oNm9AfegkEvfR0dG89vrrpJ8fTv3QU5Q6/Hml7bNgb4dYTPL89Gswj5SCxsSVDIEz26SnYYD4EX7OJD55biXK60cs7hVuAPvuJzTvLHR92/guZ+yBTQOg3XSpzqypize5qsBTt8l722SKVlGg7kiYWARrO4vP1lHXBkX/bzfY/IMY/5cEXF4c8TZs2gID18laarFQFtKL5Aol8eZpE+DM+Crp8oI4Ygo+pP/Ar0hunUDtGN39bPEkDxxazswIcFjitWM12sO4JJknljWSdlR69Ybjb4ofOfJs5Vrhuk+dOr/ON9/AhfMz6RT3Ft3LqBbdgQE2Gw9+8Q1H216mXeh3VdfAy0ZEQAQO5eBi1kWsAXmER0rMzN/XZtpS5H97ZZ9pDuO/E/8lKcN/cd34t7Lv+mCaFXfY1eK/itWLLzhUsZ5IlXtOqf2PGauE9j+i1PlvlFJKla/uojbMiTFnl5VmCcP2zGxhA5nB6RQmhVnVzsl3hLmxeZhSpdmaoUuXlPrz5WkyvnV09SfuCfP9pSJEc8zZwlK/8INSyxqbM9b3PmBkk7hwZYkcUzVML5W4RqlNQ1TyqTXVMsz+Iyb0nvuUOvvFf3Y90nYLmydtl3Fs+/gKRtMS45ijTJ6RTUPNq+wOPK7UHIt59Uncgipm7/q+xvGyPGF3z0Gpo6+bH/eFH+S3TarVqkXyRqkunYNn5n8FZn/0karp6+vxXu0FVdPXt6p67z9F0nqldt+j1NFXhQlbnKYdj5sv7PuiJI+VlOr8txWs8vXGsbXdK1h88aablq7srRyLouWerWqnGXMWpanja1aoK2fjlTo+Xdif14iMwgzV7T25zzmHX9MOZp9QRatuUAt+iVQHtj0ojDM9ygvluHffYzJWoNTabjKuq/ZxOBwq9cgvyrk4WuYRffWdU6n7b7uktrwzSaUe0Va1qPS9UomRsELYksWp13y+Sil17Pxf6sXPUN99jSpI05UNJq1Tag5q6uwq1pYG+bGq6TuoOz9G5WybaHy2SzLlPV/dUYVZvapltLv/bQAVZrUoNQeV9WdbbYWcUsJSXlpPWIqbbzbMu9eEIy+ZM4Yv/SoVdimbzbc7/aHnKt3YP2V+yDlT/W9nHBBmeqlb9cvFnzzPHUoJS3MOwtr8T7Hr7gqWvpambbcrNWOGUl97KFBas0ap0aPsauHcfMOzeU2oZPzmG8dWd6qaW/+TfeeckvdcX8GkhEGfm6vkvBdHacauXJG1ecGMzzzvuyxfqZzTxufr8PPCOp+DUhv6G7fbNtZQFaVB5kGpKtgz2Xy8NEf2vfd+w9CimR+oAx8OUx9+oC8PkHli5Eilds8aqxyLamkHSzKUOvCEKr201LxS1mGX8zWb1yrgYm++t0NHc3U6ZX31VNWsVNWzt+9hNfvDD65rnZr16HwV+1U7NeuFvfLsuh/jwnCp0DaDqyoq65j5+IoW5nO1C67qTjPlBaWE4b8gyLw6qyhJtjvwhNgj63rJfa+Ahr2dfVIY/65n+Pg0bUV8eaFUbrjmi5IM9fiicarVl61USbnunYn/S2zGK0vNzylhldgUnlCSKfOthzl15Eil7rk1UeXPaSAV0m5wOpVa+HuiSlj5knH+jF+k1MGnzecAd2yfINdNV71Sfv57dV/FOpRWkGa+bfJGsbP0WBqj1JquSh17S3WsH1GtvXq9lVXJKyYqteYGOS+Tioc2X7VRX31V8Qzp1+YrS2V9XFxLKuH0yL8k67vZe1WaLZWM8wNMfYu33pLKqotfdDIyxM8K23zmk9vVBTORlL9CpbrpylJ5Lt3ft9Mfm85LBhSlmNtvyRuk8vyvEKUOPvPP+3FDfp5dXfm6pdr3wUiVc3GfscIpfpHYhNsniOKFOzbfrNT8AFVy7mvxjf6T6rzE1VI55b5uKyX27/o+FRUEjYzbHXlZqTkoR8459dyzz16XAsdAq06Bw89P+VitVTZ8WZ5S8Qvl2ddXKu28XY5p042i6uCOrGNKzfUWxZpdd0ulkx677vasgFCcLhXNu+4yVtoqpdJ3fKTiv++qYvfvku+6I/esVJDr7TqlqmzoXXeLbbC6k/E7Ssn7tv9R8TXcsfself/3QPXqq7LGa5CwSvatfzY87V93Pb/8shpVEadTjR5lV6NHlauHHyipOIc7ZcxdwWF5c4PPUokKFQx16v2qz4pT5Xo5ysUWSdliVFSJmy/V6Z6qL5WSc17T1WCb2nfeq9Qc1C1jClS8/nXd+4B59fc1wuEoV5ZpKPufqJTVvbSDRUkyB+gVfPRI3iRxGf0790848IRS83yq1m93OJ3yzs5BqrfckbpNPv+n+ERplihM6Sv7Dr9YUa1qVIapxKFnzf3bg0/J54siRY2l8Kqa/fHHysdqVTf6+WnnARtaJR6lxBbeOFjmAz3S94htcPhFie24I223Klw1XK2bMUWpOai8M8s0w/aF0ergRzepuyakqMfuPK49Z5cSRO7ZCvWkU1X3qjhVOTKPqfi4WGOVS2GCfLe6edheLLaww2jrKqXEpnfFY1a0Umpla+35bhlR4W97rmBLTlaqrLqlIOe0rCvX4WOmHFqp3npgtRo5UqnVH3+grczPPq7ifhul3npglRo5UqkM92LsCptx+ZtPqIfuOKvefmipaczHNQf97B72y78kscA5iK1pBkeZ5+vtqoCL/UOpvxvImqHD4tdFheqTJ36WWIZb1euRI0rdOjZX3TKmQG34/EOjGkHSevnswnfG34793aASUYm8CzLP6xUE3JGwSuygORhjUS6UF0kM5jpQUFB1rY99f7/EBdzm3vKSUvXR+yVq1Solc3DqDrffKxRFm2qqh9ddXKeen9tXpV5ZY/6F9D1iW+orNJWSmOTO2+W/urXb6VRq4Y+nVOuYJtcVm42uG6yYhmp1/0eVvl5eSZ6mei+9MF19tOtjxTTUnYtlfdNX9vX7pZ9iGmrByetTwvu/Bf+nK/tMInX/4r8D/yb7rg92u10dOXJE2e1uBu2pWVWl0KdmaRMIKZtl7NxX1e84P1YWDL1B7Eo6uEq99cg8VL3Uzrkvq7Y3SRSVLmysypZ7cFgKE8UIP/ScJhD0X4L5ASKnYRYoKMsVI6asmmfy0m/i9CdvrDaJNBPU0GtcUFyyXB5x8BlJ9Jx42xisyj6h1NL61Rvh28d7lkPJPOQml/mRcdzplGfp9EfGgJS9VIy9g0+bS23OQaTsyvKNxr9r0T/0rDg31yNZo5RSiyKUWhAoxoSZoZawUqllTSoN+OtyDlxI2y2yh+4yBvsfU+pENfdKKblfZk7LqvYitVMd0vdJstBMkjdtt+x3eXPzbS98LwGjA094llq8FmQf18glHk46rJq+g5r/LSrrwn8gQWAvlXf5sgfJmqJkmYN0ckt2u12lr7qlIqFr4tRmn1BXN31pDAQoJQn7OYjkzD84NmbIWttXlf6JavBJA+NgQbzav+Ve9fd3qMsre5hufz7jfNV7pSdc5JyuIELcrBpGVh9k1RueDcP8JOBohrj5Mk9sHCjvhj6YlLxRZIEOPKHUvoeN25fli4F86ZfqpXzMMAelVneWgPqxN699u6T1cjyFVyWpN4d/Tgq6oyhJEron3zV3mJSSOerKEnG8zVBwRYJ8ZmtCxn5zyZoKZGZWr55sivS9IqG07+EKiSkTGRWnU347fpHnZNHlucYAhVJyvi7JIndZwGs9vPR/kIh1JXs8SGCpsnxzB3KezVRa6pqQc0oS/ltHmTq4xVvvU445NpWXkWUqo3zlilIZm18zEoQKrsi5eAqwp26vCHJ5Tn5O/nuyCn8/3HOy5Z+wY2JFIsehPvrwQ/N1ys9XG8xWSv06bY66+k1zdXCtUQ5OHXuzktRlQMoWedeKkszHN/SXNf3iT+aSW0pJEOXiz+br9tVlkrwwQ1muBBLj5guBYHG0RlbruQdOq2fu2W8eOP4rRGTMPKEiiOVcHG1cD1woTBCbx0Rm7P8N/v7wa3X+8y7VB5XMsP/RChs5RZ5b98nEXiIB8aIUCdyYkO7KHeXKfzqq7tuojDydzlh5kdwfT3P5kZeElHf+a9WsZuA/kk5mg6pZTcLvmklTOWfU3/Naqmmfo97+As/BptRt5tLzu++tIGeYbFeYUPE+P226y2nTROor+btGRnnUwgQJTHkij20ZLony3fdUL+H4n8A1B2wc5Jlc6QFOh1Mlft+6yt4wk5EvyTAP4p/51Cgf5kLeBUlqxf4udpSehLFtrKzd/4Qjr5gnR+IWKLWup3Ku7qz2L379muX5bF5e6qUXX1TN67ZSDSLrqC6tjPJpSimx2+dgJLZln5Q1xGxdzb8s5+p6xszk9nfdKf5H8ibjPrKOVfzmP/gIZqgu6eZ0SoKrLE/sBjP7TSn5jpkkXHVwlFUkD8rkfM0IHJd+lVjChgFy7m6YPdst2Ze+V0uasBcrNQe1+e271PPPOcS2K0yo+F27vMP2Ulln3YPC7ijJFDvG3S50zZvFqW5kLd37uO9B+Tx5k3GtK8sV8sjWUeI/60lR6fvUt68uUHffWabK8tNkTTz3pYytbC129vXgymKNDOj0rdPVn5ufUPnpHuIbKVvFb9fPf0XJQsRztbnQ++zpe0Q2Xic5WoktI8UHLU41Xq8zFe1Tjk8XH8UdhQkyV1yeIwlUT/LJnmTxi9NkPvGUVNk2VmIFKVuNa3PyRlnPF0XKvg89Kx9XyPN1adlSNYuOVF2aRKu3H+qmks9V0wbDDLnnKgjJr2o/v7pcOef7q9lP/Kom3ZKmigu0BKKcHHnm/3xpumzv7icUp8l+7aXyXM+hal6veC6PH9gi8byyfLlvZvbY0dc9v+uecOBJ8YeUkmvpfp8PvyA+WtwCsZfMEqAXvhN/fa7VaP+l7RYixJaRFYSxo8btlZI5bJ6Ppr1BQkLVPHHqqEvuteqaPvdc1Xieu8lSlqvU/sfUe4/8pf54yTNRbeRIpe6akKxi130rcTEXLv4o27jkPK8HWUdlzTSRXHbh0Ukn1YyHlqmXnzfaI/HxVef07SeJsh93e6w4XWIT+Zf+uf3L9WLTEDnvlC1GsmJpTsUzl2K+7bkvhaizpquhPUBpiVO9MXWteuKuI+qpp0y2/Sc7XSmxTd2JeteC0hyZfzyRFN3hKdb2V4hqFuF7XQTryHB/xTRU8wdmqICXG6vwmfVVcbnMUdbp0t4mMS9RzdoxSzENNflvsZPWXlirmIbq8E0HpZRSvX7qpZiGWnzaJEn5vwCmOYxq8G+y7/8S/Jvsuz6YvigJK4RBuOO2ChaRW48Ae4lM5OVF4vTse1hrOK7rKSwUTwHWjAMSMEpab95rZk1XMfpTt3msNFLlheJkmjlVxameeymc/6bKcdX3fTn4jLGP0PWgvEDOfWG4+bi9WIyy5E3GsW1jlVoQrPnIUxKpn4+XslYTGDEwoZOTZVHePl6uvTv2ThVm7ByEteKO3LNyLzz13fknFF4V1m91/UkOPFkRkPIQHPSEvfdXOUh6uKooTdivSikJBi6pXRFoNdHovjxHGMWecOwNQ2Ks0jlo1Uo1q1NHdWnRVL398gMqOV73fNuLxcjdWdHfwt1xWlxLmGIXvjOtmqmEWQ+J9H3iJOWcMb+WF38yrXDT4OxnnitT/1Ok7RIHxGUgr7lBk+CPP/ebuu0jYSjlFHsIhHlCxgFx1D2xEKuB3W5XJ/evUfb0g+bbb+gnc5C91DhuL5GkR85pua4FOjrznslSUZp52LSf0cVjH6unPkXVm13P8/FtGipVLHqUZlUl9OYHGNmP5YWV79SLD45U/a7R8OwL6qUXXzT+3rVinq84aas7S89CPZbW++cgxp4p5oHU9L2yFmwe5pkhnntW3mv3JOTJd+U6ZR6UYEDSOllP9j9SxdxzOS2LIjwn7FZ3Mq8eiF8ox2wW9FNKEhObhhj6ElTC1S/w+AzjWPIGCbokbzBWp1yeI3+n3jcSMXLPS2DjwvdCHvBUEekJueeFYToH8+qtnXdUXFMPlUipO2SNq47U4vpeyhbtZ9knhS28utN/RsRJ3yuBBLP76Cj3XEGXtksqLqpJuqmzn1ck13XVGPmXJMih77fn+s3CBAm47LzdmPTLPS9zxf5HJTiiD4LtvlepBYHKfnKWMQlZXlhxrT+R51mffHehLK/y+bHb7WrDhg1qxvTpqlOzlqpuSE3VISZKvT0BlXxQm1wtKFDqxImK3nZmjrujXJ5BT4GKjANiP3oiba3rqdSSOuZj/wRH+X8052cuGKDtUS8nAAEAAElEQVRyf44wr6bNu6hdN8uL5N65KnIvfC/VS+t7m1eyKVVZTeSxv3XcfOMzr5QEKfMuiE1s0lPXuaKNcs7xqiDHNdAOpmz2/I6XZIoNdum3ClvD7Xs5ZyqCkJ4rm0vKS9StH4mtXHBBZwNWVKBX18dLKaXUxkGqS51rU6KYjfT262+x/GMyWpVkiM+hD+hU9Ir8dfEAtS9hX/XHZobE1TL/rO9t7IvmKJf1w0MQKX7DF+rIx4PVay9UE2RKWi8JBE+kk8IEsZX0fk3mIVlfl9YzVmdlH5c5/9Iv5v5S1jFZRzwROy5U9BFcWt9oyyipQrfHL5f1xiyg6XTKfHQ9VeLZJ2XNPf2RzAX6KjiXj7Z9wrXvU4/Mw8o5318lbHhaffTB9VU2q1OzZC3Z97AElvX3vPCq2FnX03vRBYe9ov+2hznMVR2oHy/LF9+xOvKivVSSAfqet2X5si5e+E7Wm+shXJVkyJqetkv27WltP/q654pvpSqqaXsbP1/TVZ69Ux8YErdbfv5dXf2mhXruvoOSDHT3rR12lbHsNrX4vS8NPc/+ERv6yXtohqvLpXqqLE+SQee+NN6L0ix5p+ZgJKBVkgK05+KO8vIK0lNhgiSajr7m8bv/iGVNlFrdUf4/96znZLMLsb9XrAfaajK1ZaTY8S5ygJ7MlbJF1uztE6onOJ7+2EgyS1gl+62ObJewwjwhvW2M+Oup22W/ZnNcwRVJqJkldf9u9M8qTylbJAHhiUj0T4j9U8gU+rXfXio2oge1nWPHlDpwwPh5fr4kcV66b5ua88p0z+ox+Zfl+XPZYVeXK8fRN9WxgzsknldZuTpLYg/FaVVJ0XU9jT5mSYasL2W5Yufsvvfa+2effE/6K6bvraqq1WNxlPxtG2u0W1xqTsenCXlX71+c/byi//F6eVbdfIQrV6oSXzkmYYSXXpKx85/foJx7jFXyJ04oNf/b46r03J/a3900VKlNQ1VSklInNm3TxozsxXKdnE6JkZkRf3PPim9QmCg+p8n6Wh2eekqOe6WH1+3RSSfVw3ecUfs8mTqVBI3HzMcPPGmM+7lQeFUIC2ZxtIwD5nFMpapswwOPm8exNg6usjd0CWGnw6HUHNSO925Vt91msu99D8pzZgZ7ibwPiWsqEuC6ZPL+x+Q3HeXG+THnlOdnVo8rS80T2admqS5N6lwXwbpx3UDFNFTLqe8aKkf935ZE4OXsy2rmtpmKaagHlj+glFJqZ/xOxTRUk8+aKKWUmrhwomryWRO1/qKJatf/Avyb7Ptfin+TfdeHal+UwgRZVD2xleMXySR5eZ4Erk7MlETMHIuwe89+cf1yEPELq2QX/hMWo1Ji0JpVJmSfFMPn6nJjgGzjYGFsJG8wMjwKr0qS4sDjYgTpA3MuHJ9hLrljL6laUMya256Yadq8vDKJ1LyRahZuFYbZzJlq2ltvXZ/zOs+3whhZYfgN5XRKYiI/1vyc0vdKQNFTFcraHhLYu17kXZRkScpmYyC2LFeOyVMQUylxmDMOGKs7s09KoNxMskYpMY7nIPf6Oquy5Njy5RmqLiHocdu8qqowF+PVhfKiKukFfRA/dbu8X9VUAim7TsrGHSlbpPLF0zX5f4P0PZJ4OvuFyNC4O66VgcYKxzL2d02VlHPLcFU211t9d9CkcqooWd6ntd3EodUHk/bcJ/v2FHwoy5V7rGdVqqo5z3HuW3MGedwCz9Ur/4QdE6UCY3tF5aAOx1KOqWEfoBb/6GdOdqgOGwZUBcL0jbNdyD2r1Kn3VfK57crm5XXNjPbk5GR5Ps5+Zi6tWx0u/GA+37pw6DnPc3nBFVlHFgSJA3o9SN8rbOGDT1ewft2khsvyxMlyf8cK4ioYvM9VyeHMQdjVnhrRZ580Z5m6kmKxf2rZni5c+EGq7K4sMQ+SnvnUM1PeJU00ByOpwSWf83dDSa5eLzIPm0syKyXnuSBYqrXNEjWXfpMglqdApYsVHvu7eRJi7/2S9DJLoKbukAo9MxJH9glxstJ2GYPwLlz4Xn7bLADrkmU+/B8mtF0yyvpEYmV1XjWN3ZUSYoiJTI9SSt63+X5GRvuRV8Q5NTtuV7LG9We2dmcelnm3gtWrsfNcQYD1fSRIm3PK/NiSN5kTo8ry5XNPsqj/JIGbd9Hchso8LM9QdeucPuh7jXDGLVA5+7+UVzFhpQR80z1ESdwllv4Jruq4y3OEYKC3d9d2lyD3fD9J6urhsqE9Jf5zz0sQ8dJvxuu5oZ9st+lG8wCEUvKM7pioJXqVZEhgw1PARilVsG2iSvgN9cmXqIIUXTI795zcp1UdRJrVDPZSpeagZt6CusnXdk0BkP5+fmrIoEGqS4tmqll0hOrSIESS0efd5quElVUBI32lUHmBPFuekhmHnpNjrg5nZgsRwxPL+9Iv5qSTw88r+/xQVZBhwmh3OmXOdM35ZhKO1cFFDlvVwfhupGyRdWYOBunka4IrOKavILkWnP9anj1P0vFKyTnvuK36BIQeqztWHdOmIUY/oLxQ3pV/sN/d57zqFTh0yeT1vSU5dfYLCXi6V3rmX5I10pPKgNMpNsF/UPmulJK15vzXnq/XxkHmiaTENVXSpvr52gVXxb/+euac8kys/adA6JWl8nu775Vr5o6yPLEpi9PEnsg8bLL/0x6fu/wTf6rUn9uoncsPiL/licDpdIjN6vIF7MWS6PAUXF/fx9xPuh4UpwnJS/8uO8rkfK7Xhv5Pkbyp6hhclcElmcZKN3up2GRm90ApqbA8+qpSy5rKPjwl7/9uKKRNM5RkVr231wt7sdjr+qrAo68KmcETyvLkXZuDZ7KqGUHWHS4Sm/6aOZ0VVfEe3sXCRPEjto6WNcP9Od48TNQFqkNplvgXOmUjZ8pW9fzkXVJRpg+tlBdIXM7D/dHYeUVJQqhI3yfJ9DlU2TxOh7Hiy0X8SFpXfZW7UvIc6e1GF9mgLN/cR0jfZ+4ruR+Pp/s012pUz6iAY9NwFft1JzV5slLO4nRtpWfBFZW65hn12v0bVOLP3c3jb56wY2KVYogrwe+aJyvjD3kSw9o+wRhfcUmkV9qTHuavhBWmBMeCApHrdGSdlCSXLl7gWNJQFS3topz2Mnl/9M9paY4823rC/qFnxYbaNMRov+VdkOM+NUuILp7mfU/vVEGcqHG57BU9itOrnRs/f+p79er9m9Syn3Z5lsa3F4utc+Dxqs/SdlaQLJ4R30mf+D/yihBLFtcyxhnKcsVPT1gpx26WpExcLc+Op3VSKTVzxgx1k4/l2lTXfCxq+KQWimmoZg++qUaOVOoPN3cj9L1QxTTU+Yzz6s3NbyqmoR5dKbGCpLwkNWvHLPXz4f+wEOP/Mvyb7Ptfin+TfdcHh8OhkpOTr1nvVtmLxUAvThODozhNPvsrVBwRpbTa3p5QkunZSSxOkwCaWTAmdbs4LcmbzMvT82MrKhMGX9v56DEHqYJzR1GSGLhbR3t2pjIPe04CuhzqPZM9ByvzL8niZpbgyjktC6eb9ItH59XXZnReFwR5ltv8J1z61XOi0BXU23Of+balWcKOLDShXh5+oSJQaaK17967xyUboUfm4erHs4/L9dQz05yOf5Zc2P+oUotqXjurzR1ZR8XQsJcYf8fpEGdpDubPbmmWOPt6g8LFHDvzaYUciUmVir1UAreeggxXlsr7aZa42DzM6Ki748hLwqa7ssQosZdzWra/skQYmO5SY2X58o57ut5pu8VB2/ewseoh62jVM7Cqg/Gc0/fJ+7BpqLnUrCsg7ZKbcIPD4VDJCbHK6UqemaEsV5IL+krJ0hx59qoLcCkl85OJHOHm2M3qmU8xJqeUEmdhZWtJlpjhzKcSZC3JMHfCdL1OZ3/0kQqvRsJqL6hw11xx4Yeqvmj6eTfrqDgbV5YIUeI/6b9zYqYkL1xSSy7E/lnhEC0x3668yDPZpKIXkrr4swS7PVV0u/D/sPedYXIU19pHWkVQRiARRM7JGDBgTDLJ2ESDwWAMGIzBGGywDRgwBoExOeecVznnnFbSBkmbc855Z3Z2dnJ3n+/H2zWhu6pnZyXud6/R+zx6bKZ3erq6qk6d+B5dMzPf+2E4Oq35llWgLFXNRW+pnAouHop+WEnhb4XhUfis3YDsyjYdWfWOhocSq8/FM82dOAieUMZcBLvlMshbC9m39gJQrlix5Fg4UpsWp+YM3fX3OOfvZfbrwuHVXy+X2aJfpGq+g91wchVOt19zFeDsk635iB9jLnlZHqjpqxrcHAloAcy31WkZcpksAWvNbG3JWbLttxizWVmaoOdFfNiPqoQVTzno1jsy4CyKr07ddhuczY0L7A6Khnmo7nCbfWasDp/yt9QOD2YEs8Q8y3rLrjkf17L+YL8W8cM5U/k+nFB16Wp5UPs1KirEueBvS5TzrgJkPVuZEGRI5sjZfjsqgJuWyAPoPbsQzKn6yNlBvOxEBPrj0b4BTAOzx9orU4PdzvRmAjv+Ik0a8K69lLd/iqp7X1hBK7nzIThQrNj1t2gCRtt7oDNPiYlCJD1t/Q2cTvEB1A1XxKoVVNXYZa9D/7cG/XY8gASgeZPlcsQJuoZA4rz9EnsVxSPkxplvldkiEaF5Odapdb8WPIXvhT3Yz1a53LEFa9aJjqvyfSS+yBDug7NLpsMzY41E+uX3r3xf/T1B3da0yF5xVvYGAioZN5pnYArBvp5dOOPy/oFzxKqDCepQVRCqeSlz5l2s99Un2LY2Bo7jj+fnnnmK21qtVIx9aspVfysCnDKqaeEAnre/vfddoANys/YbJHbI7LymxerKYYGlx8sDVaL/2KZr7BXCWgjP7W/Dc1jnYsZQBFSK/2Pv5SSqyPuqY9S/8Vh/uY2VJoqBtvzYXYTc5rlwN/67rxr/nfcP7HdZRSEz9O95k9W04SoIKkWnCnNXHpICrMkfuY/gN2X2Q+d2de/qgaBtbUwnzrgp8VrIlfiOVMh9VG3PMycGF+PRvgk+hnSyU/oyY+3NGIrzJh7uIlDjDaSvowyF0037Yb7aJsu5H39jPV8rP8D5teocJOBa5Z9hxOTMrFH2yr2oD+I/9t/M/ycSPvvr7bIk0An9XrAEWRMlFx3GlW+fyVddxXzffZb7Chp9BTWt0p/Xtg5+DWvP9Hi48jAWbw1kRqQ/8SzKewznCDNsOevZH03+VyR57Q46MtT+tex7WNt4PXoBWns4mnPk2f4fjli3a8SHOVKxRDlB+GTSSd4ugRnnUNkb0N/rZiSeke4i7NHWNepAZuGz8N0VPG0mTS9KvF7xHp6j6N8Juj4zI/g1a7TcV7n1N7B3ZOjeEdOvnCBYbVTnZNXHg2IFi1ZoLr4awUYZtABsgXhGnf562InJEqmy71UzWjQtwZhqvrJfizIFqRPm29raeMTQIQPWde+dcS/TdOIj//gIX3UV84y4bbP/y/szTScu7ijmx9c9zjSd+MGVDzqP7XuKVGMYe4N9/yXYG+zbA2hbD4XPlQ/DJ944EP0KrApOfFZOR4ZaKLaujlWoqHqEOUFUZqWTvdcbMwxxmcM6+pxhGAeqzLXKD9X0LMFuOIFlDry5k9Q0Jr2lUDhVFRXMCFikk3Nj3IKnEhSTBOP1wCl85sFp/Nz9P7X3mIj45M7Z9o0wdEJue0ZdsBvUPcKhKHtfq89FgEcGXxPeiaw6hRmGTflbMB6tzg1vDdZXOsmDlDn3wWgoeUlNFSaMX2twIRmEQ3HtBfJ3FuyGM1EVCFx5Rmx9yiq3RDZ+z65EZ5QrT51NroWw1rPvNTNzLT2c1l4IB4MTqj6Gg1+2BgXlrirrbcsNzAunDex3UnHqMGMNppM9YBfx432oKEyYYcDOGq2mqWhZYdLHSIyEHX8250GR6apqHC9oYjf8HIFbp/6iElRvuInzPyce+YzEASaCfSKZQYVoJr4lA81bZ1YCPRz9SCQFXDxsWEJSwEVpQxOTAoRM7thid5KK3gSC/lZVOdGxRU3tV/Y688oz7Qa3tw6GkCoLe+ZIVHfmPpzQp4GZ4TBzF8kDnyF3rIl93j9A1aiqXpahOweO5aZFgwv6M8OBL3N8fFfwt2JdV32EPVs/y/43DfOwbp0ot5wQpYFapP6b9o3y/jzJ0JWN57LSDffsQvCm6Dn5mGbtg/3ohFCvPPgT9mD9zR4rTxzJ+r1p1LrV915zvryqRtDbytBfj0CQqgfO7kL0c5k1KvXvRmmJJLTPi49Sr53K93Hmq/ruzNsf51/ILWd8CHuhKyw9Qa4Hbb9dzSIgkju23wG9wEkHtGLREXbnfDxKXoZDsPBZuyO9vx7vy8mJtjsIuWIOXFW/QBkEO0ayYHP2H6XBq87+TqbpCPYFIylQNDJDr192MmR6b4ljD2opE4WvCTJMdm678uR0qMxwAvU3wBE1Z4J6z277rbwyq+Ql6G8yp7GoWll1tpqaOVoFZbl323rQZan6EM0ei6CJaJuwJ3vsCPmVTgPrg2eFYAVZc578eqAL13MsXmnBbGDoJnVWnI4f9iI5x10IFgWrs7t5mTNFph6Bbjh7rDzwWvRczEaVnfldmQj8qiqak8FbA+estZq28n0EN9vW2WWP6N8l7FNZ1fC6S7BuZfCUM6cPUTsq/a3QB2XBfVH1YA3kCRQ+g3N10eHQ0WTob8A9dv098XN3odqu7a/HmdqzU51kGfFDnyz4lz0QlAw59yMpOOLDfIogQ8iNvdy+CbLA7MVmQ08uzqP4RLOqjzEP/jY8j4yWXrSf8LXYrxk67DPRX92qo5e9Djsq5Db3wZxYRdiqsxBE2R0YOuSb1W7RQkhIaVwAeaRKhGlagjVqtT/8bfi+io1EUOIXPG3fcx2bIduWnWgPZrsLcV6UvY41rmJqqZsJGWkNuLWswruWzQUzZMu232JfWgMTpa/E2BNE0qAVM4bFrlt9WloA70MVZHQX4nvWJAxBdZh9L96XdUz1s7hh+yKe/+L77F98XuLY2jdh/fVVwfZce1HMLmqYi7Wlqp6LR0+uWk9TQbCKMMNPZg2KuPIQSCp9Te530cN4th1/scuMYDfWSc59SMpT2YMdmzHfKluuZSWCb2JOtBD0CZnNKpiWMu/CWbHgIPV9ZbJLJHMuO1n+HScI1oyK9/AeZUH+mi/h/3LlIwinYklrWoQkvHhdz12EXsBWut5kGEgC6OZfmkURv0q9v7CvGQHOmi+l77q5mTkjg9lo32y39fIeU1fU7wmEXGYiTqf9WsSHNbXqLHlgdvO1zDn3D0jXnTwCuq6o2Jvy4HU87pEf8kH/vCB6u4NfO5iHPTuM89ryuC/Yx82eZnb5FfO/Fylhb7DvvwR7g32pQdM0rq6uTiyBrXgPwlzQqcRXZvnboJi3b4qjIeiDMKz+DEpV2Zs48JedaD+062chO3bN+fJM+u23J1YHWeFrgYKU97jcsCl/W92IvmlRTHGz0vR1ZQ6OmjH+d1ecDseyDILKRxYMqfkSzsrab9SNbZmREbj2wtSfzdcCA9SaTSr4+YWTLB4D6OvCPbvUwQ7x/flT1Mpn1h9MR6riEJNRTTA7v4eeXLzLwmfxPq3KiK8ZjqRtt8l7LW28ypmWVFSJqJzLIjCSTvLsXUPHs1kDu/G9F1Ro32Qat5bgQ8FTUFhV6Mk1e2s6VCqsPAMOTSe48uXvTAUtgO+IzOzN19qzkvUw3tNgAyqDgKZp3J71BuuZv5c7BHP+hP5IZW/YHXuecjiTCp7Cfre+05aVjhlt4ez7uebLoXz+Zw7vetU58p59zDG6VqtcZo453tIpYc+JpIAzjjmCj54ykc88/lh+7t//TkwK6G9UO6wDXXDYtKyArLLK3TXnQe6rnOa5j6qzHwXaN8gDhVm/RzZyOuHcGShEn7X2Tcga3nwt5KC7KGaIVn0MSo+Cp9TOj2Unyanq9DAM1gUHyrMn++tR9aOS5+svRaazbE9G/HgeWea4qN70tdgD4cLZLBytqRpG4T4Yh+kkN2gq3gM10bbfyh3Xho73MpiKwVBvLCvXqfpLhurPoFd4a1KXIyI7W9Z3kRnyLuPXoMOxVqFrIcxzb6m8l1LhdARHm5fZnRsiaSrrbuwr6xnYMA/zF+lXV2c5OXiZcf6ZAUypnleXDsemtULK12Im+TjQaDsh0IW9bHWCufLhHFtxekLv1j2CaFWzAfaExgWJ7zT3UTXtbdFzqGBUQfTRSid5BSczAnKrzhkcfZseUSegrTgdv7v81OSVxPFoXgaZ27wcTsV4HSzQBSe3U59KZl6a8TA3Lvmh3SmYzJGkh6G3bboG1LzswEQxXEKjOFiIPlqVHw7u+yKRsPJDu2M44ofuJ/q9WuEuwn5rnK9m7/C14F1akxS9NdjL1Z9CT5PZCF2ZCGBY15foLaZyUpa/DZ3WlSdPnOrKxnlR86W8gr5jMyo4VNWMER+C4dZKXD2s7pPaVxWrvJo/xR5IXHQYsvrb1jsnj4U98soCLRBlB+nKeMJO7ySSGcrfxnxYbYy+Kjjqa75Char1ek8uqjFUrCIy6BHMf88uBBlkgUZXHmwAawUvM7675XrI7FThrUWgt/YbnBdKWsJme9BE2K2C+teacCXQsVm9N5hRoSKjsl73UySIrPyRvSdpbxnmyFsL/c1aoTdrdPIkHxlqv0GQWYaCp5EU6a0zg30SR37zUtjOy06yV6SFes3g+Pn4O6f+iNF9YOrGTYvVz6XC8lOc/SVWVH0iryYreg7zs/wUXC99JfF6sirN9k1q2r3KD/DdtnXq54pSf1vWd+03qIYsfQ1VvrKEWC1oUoBKkv4WToOD3gnCZpcxxPiasEarP1Xrte5iPL91rv2tCI5bExd8LZBBqsp0gcJnzMCXImGnZRX2jgiKVH3ExrzJ3JQ3BzKvbS30BxlLwqqz8G5U8FTY9Y2IT82yItCVbSYmvWy/tuiwmB4V186DmWNVVavOxnlg6xOtmVTYr+PvnBLnVaj8IFFniZ/Pyvexj+KDlHXpMT9h/SzYa9YkRi2IcyrnT8nnMx7JaEv3BMIe+NhUlZbtG+VrgxlruWWl3CdQ9Jy07RAzQ8dfcbrahtt4VWwNyBI3l52k9mUtPhp6sBPcxWBtsM5T83JUK8psJW8tkmwGYi9v/IXdV8qMc8LsDfr6a6/x8KFD+ei0ND6GiA8l4mOI+IghQziNhvL0xx9nLn2Nd5V9y4+umM6H3/dnpunEo5+NVcgbDuvCMAze1bqLN9Vt4rAW5mtmXsOnfXga72pVVx3+N0Nq2zpgb7DvvwR7g32pQcp3qwVxUNR+iwo/VTCm5gszA3xpYu+HZSdCWVx6gpzC0Qnbbwed3LKTB1UG7oimxciOXHyUPXN8/aX4XRmiBq1Dtimz2fT4IPk1f6uZXfuA/ZoIuqmMsGA3Ag8N8+TOpJIXYUzJuKaZY5mIsoqJxgVwqFvfdcQPR7S7SMorH0Xl+85VHk6o/hSO41SzgyJ+tYOzfQOMQBUdjggCyAK+A4EIGKuqyfobYZSogr79DXA85tyfqMRXf4oA5IxhdlqZkAuKtxN1ZKQfNJ2yvgUqQ2pPwN+KqqvWNTD4440xd3Fi0GHtBYl9i7bfgXclU2x0DeOtnw0F3/o3eY+hUkRV+adrmGuJcyIq8zw18kzVzDuT9/ZRYcMVqDTKvFOeFczMob5q1ju2qPdU8zK5Ilz8n5jzVxWEr/wQ2aqqKlEnDNYAWX8Z9nH9LDkd59Ljle8iigUHqXv2aSE4X6znkBaA47r0FRi98YHZrkz0+4gP0AQ6Y1UIXZkxOZBO9uxagfx/ygOVgtJ59bnyfVX9melUUSSQCKeKrJpW0PvMHmOXrct/gPNrwcEwjOOhh7EfVesqGeJ7wckCq9l/hHxSyaH8J8yKZoXR27HZpPm8yO6k685JvXopHhXvmsHdjfZrxf+BcSrb68FuOB5UFb7MsUoxq/NDUA0n6y08e6ydNlDXTL1CVKFZAt3xiTjWve5vxTqIBv2TV68n6HmRfqzbefvDyFXpDe0bIG+sazjYA11p51/l36ubiedSVXcWP5/Y21Ug4sM+d1q/JS+p+5U6Ied+7BtDx/qsn60O8LdvgHM5fr4jfjNxRRHMK3wGe9I6FxXvQXebO0lOv+fKQ2VDOsmpk4pfUMv6jJtwfjYtltOhM8vlUKADCVOCkkuGllU4q2cOtwf1BEXU+kvlDlJmyKJZ+5hrG8HCRBrFA/nMQ/fh5353JLc1KmRv6WtwSlrP+JYVCA5Yqyw9FajiUTkC2zfgfHSqnBOUWao9oQXkQSxBNazq/coc62WTakVFPI2xlTpXJCukE/THVCFo+tMp9erU3hKsDVVfJ/E3Vsq5sBe6gqsANpX1evy5XD/LHgQ3DOxdp0CKv531yk+5LHOu3QkU8WEfi15d1ucXfUFllZYhN3T7wfT7Hgh2PgjdQIXab+VVIJF+vMvNv1RTQUb7cKVgb4m2GCqZKyo3N1wB2nMVtt8upzssfQWBSD1sn+f4FhLbbrP3PA12D07XSVbJPRC4i5HoZa2W1ALQk1T07/EI9+EcGoyuLpB5Z8wWLZwO34uKWo8ZiUmV79sDSNtug54imGesQd9AZ4yyOtWWINHkNQdKaXcx9EdrEtlAqPDbN6mDdU2Lkvttdj4EmylZIMsKEcDf8YCp98bpl1WfqH0A3yES9LzmpVjrLStw5mTcGEvWbZhjD1S3rkHCUvsG2DczhqX+AP42JNtLetDxzgfh+5AFZr21SD5SUXXOGQd9I+yFjyVeVtR+q6ad1MOw7QKdqbeziffndWxBAqTQ02u/hj4XcseKI6xyt+Bf0NPrZiJJUsaUMRBoIcgKq96S8WtUUqrgb5f7qjo2453Jku4i/RiH2Ouq5DYVenbFWtXIenrXfo0AZPNSud2y+sdqBgJRVBLxQdbFv09vLQJxOx/EWrGyiYmzL+dPdh+ZvxVrq/ID3FumHwa6IKeddB1m5vK3+PUnbgGbUhrYlNYR8bdEfOGQYTx8yBB+8b5zmNOJ9ZqveMS/R0QZNCa+ONH53iYMw+Ah04cwTSdu87bx0W8fzTSdeGtDCon4/0XY27Pve4q9wb7UkOpGSYArD8axp4J58ZGxgFI6Jc+mcrxvAaopZFz6WhDKrBNf/qxRMORSQesaHIBLjoHyFw/RI2LREcg4VfHLh3rljtDeEgRi5oyTfzfYjYNPpdAma5Je+y3ed/ELcmVJGM5OjggVKj80s/LWyq/P3ndw/QAr3sUcyxyw7mIc+tl/VFPtuArg8Kmbmdrvugpw2HfvUB/cDXPxrlOlpBRoW692BtSl433KgiI9uXAyWrOt6mebjg+HrM9QLzJSZVQ4WgAOsLoZ8vVZ84WdniceXZkIGMqoeNxFplI43b4WAl1whKuoaQufhXHhKrAHcgTtkfhnRdkbCHbUfCV3zOnhuHWfaNgKmWesu1hNmzRY9OxEBuuOP6t7haj43wMd2BcyQ4kZyqxQwHent8eWG+wGQLAHQbB0slcz6hEkfwymVx8z1l/zcjjqrc6ZlpXIXt1yQ+qUhkI2Zd6J7NtkmZVRI2EZspjTCXKgt8TuwO2rhmGqqlwue0NuzMR/f9nJ6t6xWhDBENlcNy1BosyMNHtWZu4jcC4UTldXsDtB9FqQnSVhD4wxFU1dMtTPxrm7/FS5M01kZm6+DtVS8Qh0wkEiqxgseRGOtQUHqwNMndvwbmQO61Vnmef3YfLvhvvwLq3BAwFBo2w1AkXmdu3XahopZjisVVnt7iIEZKwBjf4GOKm23WYPmvnb8C7XXoB5lFFf5T7CvPCQqEPcFuxbcKD6XQq0rILOEx/sad+IvbzuEruj01uLZ+3YjEo569ke6Xd2zopEmuWnyoNXInAlow43DPzuQKj5okEXBeVwzVdwdA3EOdOxGc4NWXIGsxmcPxryRtpfbBmeZcYwe7VFMuQ+AgdtOqmr9LSguscmM8YY1ws6inWXqJPfBFQO/p7cGD13Osnluq8J16y6NjOCrEuOwdpbOC0xwLr5Wnxv4TS1TtqzE/qV1UYQfSx3PKBO9HIXQadUOcwXHymXbYJqWLa+u7Lg4Kr6GGvP2oPO1+xcEVr0HJzR7RvtOmugC2OZMwHvTAY9gt+QyYlIP+yHwulylgNRMSuDtT9SPDyVcCxvugbnWCrI+weSg8T6sVYYR3z4fOWP5PZDoIO5ZydrgV5n27ZlJebDqtfUfgMbIdgDx3K8zhuls5RQQ9bNhM0yZ4KdJUMP41mdaOkHgpkjmDddbf9cMI6kk9o+bVwIGWQd78YrcRb019vXZqATZ7oqUXDzL1Hx0rHFXtnnLsZ7SLVSX8DfBtshWS/mkAvPIfwFgU6sjdJX4XC3BoE6t0Pv81TifTgl+uxptK7GbzoF5AaLgqeQ0LL5l/ZkVy2IwIUqSVUgWdV9xq/Vgd/62SaNpMQeC7kgd612b7AHeryKdcbfBr1C5RtpXABdecMVav2t9lvsS2uQ212IvZJ9L8YtC/bl3I+93r7BzhrRW4r9tvla7P345IONv0hOod64EDqcdf1t/IXabujYgt9T2CVKf57QrZz2YtNirJHmZXg2a1CktyRms6690J647i5EApLqXN4dZP1ezb4R3ze0/G0k1gl/grcmVkXeuTVGmcuMNdW+Qc085S5W20IigT7vH0hWCXTadazcR3AmNy2CLyDeDxLug+7orYMuIysCaFuH+4uefNbE5bUXYp00zocNHC+7+xuwbhsX2tf9lutR9NCywp5MKnyNOx6ADFHJxpov4C9Q+QRWnROtdBsUttyAJGEZ4pN2BdzF2G+qantvDfTKuZPUrGANc82keIlPUfTKTMJs9frtY3j/4eq+fVlEvP/wIfz6Mw8wBzr50DcOjQb7DnhF0gqCmWcUzuC/rPgLr62J7auxz49lmk5c2V3JR7x5BNN04qymFJPI/kuwN9j3PcXeYF9qkG6UYDcMZpnx52+HY9VK5yCowxrnQ3lSZTX5miBUt9yg5tJ3gnCUz95XrgBm/xHXi19I/d56BI6m3Eftn3dlw5G16Wq5MrPxSrXzt2MLDtfyt9W/XfkBFC3Z4Rrx4ZBp36imhXOiI6t4L/WgmKHDcdG51eThlxj0gqdfpixF/GaW9BBkkVlR+T6c4TJj3ZWHd5FOMFxk16s/hQNE5qhiRjXOgoOcq+FkED2pnCotnZD5O3WAihlKWP0sNSe9DL1lcHhv+RWcIFaHUuWHcmUxHqWvqh2cgoNdFZQTyuGK0+yNnbUQjLGQG86sVAM2Jt2SzTgNdKDqqeRFtYIVrZaVZIIZRqxhvGVPaZrGdZueVzsbmRE8WP4DOzVNV7ZJ/fgvvFMnJ78MHVvMcb1kd/z07DJll0JhFfJu22+htFodBoaOzDUVjVn527E+mht/kXhtwxX4POPX9qQAUamz4CAYJirHsRNy7sc9rE7Nji0whFT0UDn3Q6aJNWb9rlNCgEDjfGT8xb+vtrXOjeQFbZGMonOg2PBzVLf8T0EPQ1b0VSPhQEYtKYJ9g8naZcZ7yfi1c0Aw+x7IKisa5siTHAT8bdANrM9d9REq0WaOlJ+xq86SV00JhD3Yq6qeRap+RMw4wxYe6uykXXmmPJC45QZ1HyItgCBAKudAKhB9qzZcgZ9LxSASlEUyWrZFR6jpMKs+wm+qMukXHabuCcUMg33efrGkMStW/xhOHRnThGHge1t+Bd1vzgTsBRk8FViLwomc+7Bz4N5dhOcJdNkdgp3bQYGtYjfw1jhXTOkaHLFO1WY9OxGMlgWEAh3QWaxjrfkSTrBkzBqqgF1XlpriSaC3VF6ltvwHZpB4CeSEVY/SAgiYpZM8Kapnl5kRLwkwFD+PCgUniCxuq57kLsSYZo6Q0/9174A+KasY04IxfUJWlcqMpKWZI+22R+23pm20UP49QWVZ+oo9CWJ30TA3VoXulNClwpwJagdYzy6sy6XH2Ssrtt0aq+51snuc0JWNwJu14ksPx/pXy2SRSfWmtW+Ry7ywB2tX1R/MCXoY+mTdTDt1qtBxNl9rZwMQZ8zyH5jUehIZufEqrB8Zgj3Q8TPvku+5xoVwfgY67HZxfyN0fFXAbvHR0PFllKrxz7/2Qru9VfqaOrHWlYcEnYEwqPTkpp5k5CnHP18Lzitx1vrbcWYXPB3rEyuDSEyOZ7XpLTUpGT0IfljpBplhe66/1B4YjUfB09D9rIwR0QpgCTVj5u+SM2AkQ8RvVvn+NvFzPWLaLU87f78jA/sn1aD01t/gd+dNtgfuQq5YUovV5xOfLOqE3jKcsdY9W/Y65LlTtbRoA2C1H4QvSfyT0S/HX7fud3871pxMVwl0wF+z8Up7b8/uHQiorvkJ1qe1sjnjJpNZYAfkTLzO0TAHunvDPNhpDXNjNqivmbW6WVycvdwu8wx98MnLAvGVbstOQrVuPNrWJbeZwn3OvRmbloAlQUXHH+zBnoo/E7w1sQDzzr9CRxQJIiEXZLEskUrXTJv3JtifRc85V4vHQwthbtNpcJT0gu60+HkkJsh0isJnYhWrmXeqg5IyRpWuLHxmpeMV11T6asgFf6FTv+t4NgAZ9fyewI6/2HU0TzmSQfVIYjuOVBDsUfdy7s7BmpbJkq4sBF13PiTX45qWcFv+TB6RlqYM9MUH/EakpXFbWxuf+sGp0WDfQa/FWOH+sfYffO3Ma3lHyw6+c9GdTNOJX8iI6ZUHvnog03Ti3NZcnvb6NKbpxDtbFEni/+XYG+z7nmJvsC816LrO3d3drOtxioAQ5luuR2Z5vNHsa4HD2Eot1L4JzoWGeci86MjAoWE1uBvmxA4KGR9+0xLnnhtCQVcFU9ZdrOZ17quCkbjyR8788YPBqnPM4FUKvN0C/nYou6vOcXZczxyuzpYJuZBZl0rT+azfm8bpdXYjzluX3EFQ9G+7wmf9vggAy1D1MZQYVXZQ+TtyZ9Dio9U0i8KwyP4j3qcsgzLshfEgu7bidCjT7iJ5MFn0PFCt0XU/TU5/UPMFni0+0Ln1luSZUPlPINvJmt29+Trn7OmwF0ZBxXvyPkyBTnxf5YjtyIDRkfuI2nk9ENR+a1e6+6qw9q1VbgNBxAdDXVWhEfFjvJb1pes6+7c9oM5Kb1sPZ9Xio+yB6o7NMHhEP9NUKbnEnpMpnOE+GIjLTlY7KJjjMt4tBrlweKeTnKqpbiYoqmQyovIDOL1l6MqGI2LDFZCd1oqJ2m/NNX02AkFWuIugIHdlp14dOGcCDOJke8qK1jXIeuzKRDA4nSRZ6x1wAAc67M72iB/Gfc6f1M7d/kYYiKpeWoLiSoaGOXYnwO5COBSFDFJlwvqa1HQ5zDCOa760f+6piNFLplqFNBC0rLA73waCtRdiXw0G3loEsDdfJ5cFpa/CYdmyQk1/VvaG/F0vPV7dU0hULuc/Ib8e9kJ27U5Pjw0/i/aBlep59bPl52tHBt6HjA6xYY66V1tfNWSMygma+wgcGLv+rq4qDHRCpsrkY1+1s35V/nasr+i6SxIdqp3bcHbLnBNrzkdwSoXtt8fkqoouqjsHQbc93XtWUPqmU2o9+wqfBZNEVyZkXLwerkcw911ZZhWeokrc14yghFVnCLmhh6uoQ2u/xRyUvw2HpjU4G/YkOlH3VJVLbwkCJvlPILChkr3+Vnm1kKhUkwVntSBowJzkjLcGv2+t8tSCcDKpkmQKnoauu/latR4nqotTddiKANTWm+VOqrAXOpiq4jbzTnU1LTPWxqLD7fZgywrIRdkcBDpwpuc9hupja6XamvOT949mBguCjP2lO4d50zVszBjGfYXvJ8o85uQ0wwK+FvkaVyVlhfsgv2RzFOoFbW7O/XhfzUvtf1P8PNaYqgfwYAO2nnIEw5L1PC58Vt2TzVOBQM5gkmiZYY/JAmdlbyA4v+wkeSWHCFJUf27XCxYekpxByNcsT5A1DMyXuzhRPkVpfItQITRnnP274m+K/m3XpQ0DyQjLf4Cz35oMKPrsaiHYoPP2j/U3U7GBiB7NMpS8mFjVbRiQrypdpW4G3rPVZurZhQCAoPGs/SbxelcWKr1V6yPvMTtVuYDokVn9qT3IE/Yi8a76c5zZHZsTrwd74EvIfxL3SMW/wQxZsuJ0+b7sq4L+M3O4+XySxGHRAiOdBh4IEujZiWojaz9gVwEShJK11RBMK6rAq0hMFvqYSb3YV/gBZJ67GGOSJQll/Fo9XyrE97CTwdCR/LT8FDkLUf6TsfYF1grDrizYvbPHmO9a4b8tfBbXB9umIn4NGAbWc+uaGBVr/LvSgjFfi2DriU9EbFmF5IiSl/CuU6FH9rdj3zr5PYI90EWSBdS0oJ3mUwtifXdsHlxQzAnCBpQllgR7sOZUFYGiSnHWPvIzsPJ9NYPVqrMh850Q7IYcs+pwUf1JYk+FPdAZZc9jxYw06GlWLDqM/33bQfyzUaMcA33i3+WjR/Fzzz7D539+fjTYd9gbh0Vvd86n5zBNJ15Utoh/u+C3TNOJX90WkyPHvnMs03TiLfVbooG/vLa85M//XwipbeuAvcG+/xLsDfbtAbiLoNSvuxiCWVUd1TAPyqGg3lp4iBnEOylW5m5VUvsbzcqmBrngFRUmle8nltzvCcQHGq3Bmqzfqx0+hjGw5sROFBjBblQ8yAzMaC8FBa2kYcBJnvsoKKas6MpGJaWq71TDPGTfWhXp0tdgVKcTjDHr82b+Tk1PJaCH5Vnp4T4ohioaPOZYNmCqlULVn6l7IoheXFYFW0A4lFXKvbtQTiUnIAIHZsWEDVG6DIeeDeVvwYEc73DfeBWM6R1/Sc2pxwznm68J8yx7dtFPU+YYEWhcMDhOeT0CBVjmPAt2oyJH0IHMnQiZItAwFw4Qp/nXw/IAUdtak3pKUuUhIHrQqZ5bD8tlULIemsww2ruy7E6yrb9BsLhuptwwdhfBSHAKem39DZztsu9WfWxW5l0pf74t15uKuCSo+11BUNWtuwTOQStWnqGmUIz+zZnyqoawFw633IfV9DDtG+HIijfU6mYiUNiyMrY/Iv04r0TvGEEJl05qiuMlx8qTUio/BEVLOsnpNBsXICis6tkn6Pdk/biaFiGLtnC63dlY8jIMk2232avnQ70ITNZ+jbWnooNVwdcCuSSqE6zY+VAsK122b5qXga4uWTVRxXv2qkpPBQzA1efaM6gHgsYFkMkyZ1BflVqX6G9AcpBT9YlwMFjvEehAhWzVx87PtvBQe5Ag5ELW7bZboTNYnRfbbotV2lqzcLUQfnPTNVjPTjJQBl2L0Uk7OdU9FTgjrfMZNY4VVNXNS6ELWfUNgWUnqil6ksFT4RyoVkEwPgxELvbXwwEk3mv7RuyrzLvUFcgyZxEzdOdAB845WXVUqBeJEusulgfQ1/wE9+3IsDsb62YgeOqEyg/w/fjktkg/PlMlagmo+o6KrPSi55wdUTl/MhNq6hI/N3SsqU3XwBmqCvZ1bkfQ0Pobnkp8bnVidW5DZZSqj2Ay9JbBeTd7jDqQqULJSwiIqXRwZrzPFac7B6xlcrVlFd6TTAb5mqDbb7wKSUpWhNzQVVR6UP2suOQhxd7oytyzyZEhF+RW2RuofrZWwYnq3qXHDa76jhk69Moz5Ak17kKcu5uuxt9Y383Oh+AI33ilnYpP1+DILHnxu6F/lP2m+F1/u3PAIdDpzP5S8DQSY1JJIAl7cKYnC0xsvhbzpUI6Ya5l35sxDHatlX60YV6sqnTp8Tg34lH+VuqJQcwIOKie1V0IuRpy40xWUUNGewxbkj+iFeaSsVrhrcE+SDYGQVcvc0YvPjJW3SoSrJ36StXNhH/GmqQo9I3VPzbPC4uu3VuCasalx6f+zl15OKdklXMCjfOhr1t1HZFs0JGBAKTsrHDlme0vJNWy2251Zn1gxlmy5jxn2e2EgqdQFRvv6/K3pd4D1YqeXOwB1RnrrYP8Fuegr9n0rZm2TNmb5rvbAjuqfVPsLN7yq0RbXHy/9hs8d9bdSN5PFZ3bYBfJWFOWnhBLcLP6KboysSdLX8W+s8rk3Efhk2xbD7snXs8SiXHJMHtftd/G14x1FG+Xb/gZdAFme9/u/gboJ1oA/o10svu6OrciwaErEzbE7q4HK5qXqSv0mPGO0knN+rTuEnUlt7sQskWW1O7UdshVYPqAT5Trs2I/b/iZ3Dc4b7KaZrPmKyT46BrmOz64GvHBfyWS8a0sLtn3xOSIVR+NMkQNIJG9O0euvzUv5zOPPpS/HUCgj80efmceNZWvnnF1NNh31FtHRW93wRcXME0nnlsyl38999dM04nfyooldZz+0elM04mXVy7n/V/en2k6cVGHovJzLxKwp2NEw2gv9uL/AHRdp6qqKjrmmGMoLS0NH044Gf9O+DuRoRGljZJ/Oewi8pQS6QGinX8m8jfj8/4aov3OIrp4PdHI/RK/s+80on1/rX6g014iOuB8oh1/IjrjbaIJp6Q4oCBR/uNEk88hOszyOwdciGcadyzRqAMTr42YSDRqf6LyN4nGnUB00M9i14wQ0Zx9iSaeRnTARUSH30q035n23z7mj3gXVkT6iJoXE1V/TDR8HNGhNyReP/gqotEHEQ1RiA1fHdGSo4hOeJToyNvt13sLiUpfIppyMdHYo+3Xs+4k0rxENCTx8xP+RnT8X4mMCNEQy7WR+xGd8wVR4zyidRcRnfku1oQVc/YlOvAKoguXJH4+fCzehwquPLzPU58hGjo88VrYQ9SyjGjCSfgbK466i8jfSlT7FdHkH2M+BcYcheeeJJkfIhy1Q9KIJp5ONPEH9usTTsHvB7uJRk22Xz/tBcz/PgfL7z/xdKIff020/0/k13t2EBU/h3GPPyH2+UVLsUa2XEe034/wL/qdndgPJz5KdOiv7PccMQHrfuuviI59gOjMdxKvjz2a6JTpRBN/KH+mYCfGPeZI+XUn+JuJlhxBdPzfiFpXEB34c6IzXsc1ZqzdcA/++7w5RMPGxL7bsoyo7iuik/9JRCMS7xvuJdpxH1HDLKKhI4huDiVeb1tLVPYy0RG3Ye/KMDMN6+HKEqK0kdGPIfOq6VjPGzRU6yP6yczE7530ONFRdzuPe/SB+GfFxNOIdB9R1btEfRVEx/4p8fqEk4m6thKt+hHR+fOJxh6VeJ2Z6Nxv7fuRiKhoOlHTfKJRU4imXkI0ZKj9b055lujIO4lGjJc/tyuPKOMGonO/sa/R5iVErcuxVmRjU+GcL4iIiSadLr9+3EOYT81HlLZP4tj6a4nq0nFmyPb68DH498NX7Nc6txBVvoe137KE6NAbiUZPxbXDb8Y/AbFHPGVE408h2v5b7LexxxBNvQxyOR56iCjYTnTpZvs1IqLOzUTeKqKf7SDa9zD79WH7EkU8kM0H/dx+/cdfEm2/lWifg+zXPGVE3Zn4N+aoxH3ZNJ8o0EJEQ+yyfsR4oh88Z79fPFpWEPXmEx35e6LRUxKvDR1GtM80vMcTH7N/99AbIbNGTJKvT1cuUc0nRMQ4/4/+Q+L1DT/Dtd4CorHHEh1+S+zakDTI8SNuJxpzROL3OrfifaeNIpp0BtGUi+y/7W8h6soginjt17J/j7Vy2G+IfpKeeG3fQ4l+nk/EOhEb8j019RLIj1FTEz83wkQdG4jGS85F62+M3D/xsxETiU54GGdYXznefcJvXkzkqyVqnE00/kSiQ66NXeMIUc49+P8jJ+O5rWhbg/k47i9Ew/ZJ1POGGFh7o6bI95VA11a8u4tWEO1zSOzziJeo5Hmi4x4kOvhK+/eMMFHYDd1Rhl8UQc5Z4c4nynuU6Oh7iQ65jmhoWuL1SB/RxsuJaAjRtXXq55bh2Ptxz+FjoSuVvUp0aQbR+OPtf9u0iCj3IaJLt0AXnXKRfM0REfmacEZNOpPogkWQM/HY9HOckYFWokNvSpRJRESundDPiMw5viLx+qVbiGiIfL81zo7plae9gLPQiv1+RHTKM4kyZOgonMX7Hi4fExHR1puIGucSnfWxfX2PORIyP+9RovLXiH7lsn8/5CKqeh//3yofhwzFempbRTT1cvtZFeggapgJ3cCdT3TINbHzu+Zzop0PQM++aGWiHrb/uUQ3B9VjyrkP8umybfh9qw43/niiw3+L/z90hP37RES5DxONnER00hOJn4+YSLTvEXIdPuwh8tVjDYTdkDUqyOa5eRHOroOukug6Q/GsrcuIPMX2fTFiAhFNUP/euDgdlCR7kgi2lAzlbxCVvx6zqWTY8QBRTw7RFTlxzzQR5yoR0fEP2b9zyWaiwicx7oUHEf2yDfs2+phMtO0Woslnw36RYb8fkX5ZNmSersdsWyLouhNOIdr5F5zPVpkf7CDyNUAPtOrMoU6iVadDz55kuRboIAp1E9V+jnNUNrZkOO1FrBErhqbhvJ4zDnLs3K8TrzfOI9p2M9bWuTMSz9bo8zUTeUrsa6xlOdbQgZfZv6OHsGeGDic6/Df26zVfQE8ZfbBdpod7TTvuVKIbuu12HhHRT2bjf+P08yj2OQQ6x77TzL+z3P+4v+B/tQBR93bIs7FH4ZkbZkG3Gz2VKG10oj475VIi1rCnIl7od+L3xdogssvyeIw9muj8hTib4zFkCNHNmlyPICLS/NCDR0yEHBX7wAnDx0KGDpfo9Jdsiv1W41yiyrex9iaeape7JS/iua8stt/nmHsx/yX/wXNNuSTx+vgTiS5eSzRrFHTQo+6038NTivU19bLEdzfxNKJxJ0LXD3bBz2LFtOvxz4oDr4C+OekMogPOs1/3NRHVp+McPEiii5z7Lc7f3mKMQTYvnZuh64yaYr/WW0I0bB+7TkqE/b7jj0T+Jqy1+PW9837Ir1scZH1fJc63KRcn+hrqZxFFeuE/scqYkIvItYto3HFEYw7HP4F9Dib9kF+Zep5OadOug1wefxJslfUXEZ36HOzt8+fan8edT5R5G9HZn8N2kckhIuzz2i9gm0y5MPZ5xIt9dfE66LxWXJ6J9x8vywUmn0N0Vbn894iIGmZAxpz0GHTjeGT9DmvyFoPInYv5PuhK6NSRPqKMG4kO+gXRIdcn+lzisc/Bdn1g6mWxPXTsA6avwfRnVn1AVPoi0dVVePaTn07UkYmwLqs+IPrB80RFT8Emstr7RgRn6IRT7XZixAsZ2ltI1DAb8zbuuNj1giegB008HefS+JNjckwPE4W6iI7/O9HUSxPvm3Mv5OOBV5DNJ+hvIdpwGZ51+Hi5XB65H/adpwRrK/48GXME7IVNvyAqfYHoIIs+e87n+KfCRStgO9d8BjkpZDxRzPfpayBafDjGdvqr+CzYQbT5SqIj7iD68bd2X8Tkc3EurL+IaMpPIcui49kfa6+vArr4sX+2f795CfxNp7+ZOAcCB/+CPH6NhIXYTkSfEtFiIvIQ0XgiupaI7iaiqUQ0hYg8vgidMGpC7BH3icmA4aYsCethihiRhM+IiMaMgB+tP9xPuqlPDrPakN8TSGMY/5PYIyHDvUgZeyv7UkOqfLesh5FtZK0+WnF6LEN04TRkzzmhK1NdeRByI4NH1s+mZSWqD1R9rURlVdbdSYdigxPf/fbbY1WHsibHjfPVPTlEFmDuo+qsu44MZErLsjKjTdIVPaZCvcjImr2vvFpt83VyepCBoPIDZNvIOLxF1riMF5wZmXJlb8ozYUpfUdMgespja8naP1GgdTWuq6oqmpehgidVTnEthAxGJ1pMJ7SultPVCPSWgFpEVjGpBZAdaKWD6MrEntr2W2RuWrPefC0mjdi36j1V/jayD2X0FyKrU0VDsePP+O6uv9szWUNuVF01LUI2l4qeTQZ/Wyyr0pqVGeiIrYHtd0i+22rKgoPUskB839IXUtM0Ls5ZwcasUeoMX28t1pY1Gy/QgffsLoS8UFX/9Zaom86Xv43sR2smqWj4rqJbalmFimenzOy+KjXVjSsv9k5W/DDxmshcTCd7/ydXHtbIrr/h7waDrD/Iq8Tb1jrT4rry1JU8gsam+lP8jary2t+OrOL4fbPjAdBdqtC53ZRtr8mvR/xqWhCBtRehSiFVCNqmpiX2qqqQS15FO1BsujpGozIY+FpQsSWjWYz4kW0v1pEVW25A5WlPbmp0TKKyLp3kPVyrP3XuvyjWyeofy68LSs28x+3X2jdhzav6PDCDbim+WbxA6+rkfc+cIPrm2npWabiv6FMifaY/JVSYJ+h5hoFKSFUFubsIMr9xPqq6Ap2xa+svxb7xlNt1lcYFzPMPUFeI5z2mPs+ZUZUi5ln23kTPNFXl1sofgaa9ZQUq8VQVj9WfI3tYZMD35CZW7LmLcIbL6K9sz7zJ+ZlKX0EVkadSTm/a34Bs9u4dqVeh9zdgfledbadK91TgTE5Gkbn+Muh3VhQ+a6dXtKL4ebxvK7b9FhWjG36WSDEnYOh4b4sOk5/d3TtMPfxWVLPEZ9uL3uDNS9X0Zjn3M8+fat83Fe9BF04neeV8MjQvdZYj/Q3Qf6wVdoIKTiWjsu5GVrmqkqi/AXRfKlpSZpwVsiq2QCdoWjf+wk7NNxDk/SNGM2hF9aeo6s//p50WsPBZXNv6G3VbhWQo/g/etVUvjacrl62/pkXMq85mrWN7arbtQBD2mH17JK0Jch+OPZeVhj3khuzc+SBsNBmTQPVnkNsy6BHoOMtPBRWaFYXTY+/D2gddC6IyU9XDdOGhYFbYcoOdstzQIUd9LWDGsd4jajtI6IsH2o9td9FXbZ7fJtNByGX6AP4ABgVVhZKoHFSx1xi6XL/rLYOt71R12rwUtNXWMyhKRymZi+bl2GtO/Vud4KnAGZxOqEK3Ysaw5HKv/C21zssMu0h2jtXPxv3TSc4u01dlzpGlirevGra29fOBovJ9e+W6FYKG2ypHKt+HHjx7X7wv2XsX+zmd7Gdpfz2eXabT1H6Dvdi83M5UEejAPGfcaFaEWejS15wHe1sG0Uu84l3pZaU/L9gNHUVlkzJD32lcKO8nyQy5t/pcyKJ0sjOxRKtdB2kjMmNPBbvt+y7kUlc1V30E5gZDh+4RL49CbsiAnQ/Kv7v8VMxTpB9/O9Cq544tcbL+Qvnf9JZCj/W34WyPf35vDRhjil8w96uEEUdcEzLWqhu0b8C6j/YBjTtThK0vay9Q8iL0PplvzNcCP47qzGeG3r/8Byb7keJ9ta5xZvVKBlUfdGZU9WX/MZElLeyFXp+MFavkJbUeVPsNxiTz49Z+HbNtFKxMZx5/PH9LxK8T8XAiPsb8d6j5v0ebn78uKvtOOIEfWP4A03Tif65PPHd//u3PmaYTf5H3RbT675Ndn0SvX5l+JdN04s9yP+OT3z+Zp7wyhWtcKbJR/Jdgb8++7yn2BvtSg3SjtG+AA6HyfQRj4pWgKAXQHYk3Kn01prgEutTUn62rYz2YZM3VDcOZPq/sDbUzkTlGe6RyiAa7EViLd2AJ6BquqWh2Iv4YHZwVS45RG7WBDrxLJ9qXZFzkWhAHtCqw1pMLpUMVcJShYS6c3m1r7QppqBdKsow2VEBQsMgQ6IrNk4y+qr8Rh3P7Rvu1sDdGuSrrUbHtNigDzUvViqmgC0iVZ19Q1+bcL7/eV413oqKqE/3znJTGnp1QLOIpIJuXyYPI8dh6C57Nqmwko4FlhrKy5id2RwAzDLV0gkNIhqJ/mwbIAXCo7km0rMR9rY4qw4BR4fQefS2gYZLRKDLD6VY43SaLNE3jttVm8EkVSIqnEIpHRwY+nzvJeb+qsOvvaCgvM5h9zaBXWX/Z4OiJmE2KPEUAUwQ90gl0F/EQ1KXBbrvR27ws9j1ZEFzXECQvfwfOBhlqvsA6ta7RcB/eoeq8mD8V/zb83N4/0anpvK8Fz9LfGKPwUP2GDP31mKvyt9VBFT0MI1SVUNA4PznF455EuA/ya/sdJt2SgqZ4683q3j+GgbNAdj4yw2GeTuo+X81LkYwjk/nJ0LYWssD6XX8bjNeGOXJjbuG05L17VAh74axZcKDc6Zf3D5OK0MGQWvHDaG+8BCw5Rp30FOiA7FKd57sL0b9xybHMnKJBJIIasr6NGTdCVsjQthbnRPsm+XXRl6krW92rpH0D9A63hJKm/B2104YZ85B1N5IS0kmtG1ix6WrIZBVE37wNP7MHMsMejEdFnbe7cOWDbimdnIO7VhQ9Z55Pec5/V/CUPbltd7HlV0hoUkELxc4TWXAj7AUFlkzHCrkgD2R6QagX+sT2O0ynpIIqu+Bfcl2r+HlQKMrWHjP6nq2/TN3fTtCbWp2dvWXQoVT3XXqCGRy9Ao7nPYn4AJSq96wTFhyE76psGxVVWP4/Qe0tCzZrQehljQsR/BiMrqNrcDjKAtImRbGx8Souy5xjl3md25CwIKP9GwjcxZDt1gSl1tUYl6fSrmsEu9H/bf4U06Eo0ZOE81+m0ySjZDMMta4sEudUNk3zMjiN509V9w3v2Yl7WHv2hb3Yk6peSE2LoGO7C+X7ua8KTuENP0dgWIWI3073v+PPkMmhXiSeioCzHsYZ5iqAfFO1L+jJRZJOfIJkxXt4Dz25kCGyXpUisKraz51bEWC0OuCZEZTP+RP0f18L7tW5FddEoCJZu5B4+JoTbQktiDmSJdz05GLPlb+t7gHo9Dt5j6vPdpGEUT/bnhjVthZ2yZrz7XvO3wabueg57AlVv9/i56FXWt9nbylsGVVfsurPoBMXPG0/D5oWxxK400lOOzhnXOy6Su6r0LkV37NSCrryY0Gchjl2Ckp3EdZkzn1ofRFvM/lbEdxyF2Ndzz8gFlRxF7Ixb39uXvdwcj2vYW5yf4MVmXfBtjR0JFRZdbhAB3Sv7HvklK2ecgST119qDwT527CHRWsElSyr+gR7S5WI1bkNvpWBBsxXnI79mPV7eUA3/vk85YlrSLQqWnnGwH4rHt4avMuy1xGAlbUuaFuP5CFvLeSfakxt67Hn45/d14Jkgz1Jvy0ggue5j6T+3Z6dCOzv+ps84UELxfoNWn0zu/7urFcOFiK5NtCFPa7SG4M9sSQSK1adzf++fRofn5bGY8yg3hFEvB8RTyDicUQ8kYjTiHgfIj5u2DB+7t//5jnFc/i+Zffx4vLEPujXzryWaTrxRzs/4iu+vYJpOvGXeTF7bGbRTH5+y/Pf2z598dgb7PueYm+wLzVIN0rLCihWoq9QfNaersEp1pWN7Pz62VD2I34chpl34RBU9Ujp3I6sx62/kTdi3fxL5vQh6gfWIzjcVEGttrVqh7NwYqWTPYjVtBgHplOg0QntG8FxrTJYtCCMD5mSUvEunILtG+UZksxQ+GaPlWeyRvxwaquU0d4SKPhWRTn73tj7sDrRRLDO2ssvHqFeNVd6sNt00l+hzsAWgbFUseV6daWipxwZQbv+BuPGqiD11yNjfMHB8h5gJS86j3nrzc6Z4dtuVWduCuz8q2kIxlWurjmfedZoBONUAUp3Mda+dY02LoTRqAo6uPKx52SZ3wIbr4SS7oRgT2oBE9EbqiMD/z1rn8RswFAvHACDyWSN+BCQcMp2V0DTNK7Y9iXrJa/JkwJy7o8F+qzGZ7Ab77v0VRhr1urjhjmQf8Ee+XxUfQKHilN11tLj4RiXIdiN51Ktr5VnQg7JEHJjDaQTnCADha6Z8r1CXjm69gJUh6h6ImTf41xFxwxHvmx9VrwLJX/WaHVgJNwHZ3v8WohWU6RjvvIexxjqZ8ecK6KacfN16iqgRUfIk1L662Pfl/Vm7C2DA0PloF90uJmRLJFBvmazek5ieHprce+29fbAV9hj9sX8KdaPLNjnhIg/FiSQBXREos3Oh+SOu0AHkoM6Mpz79kX67Xsj5I45O1V9DlXozsF8Ny9NXslkha/FdKIq+jX0N6J37fJT7MEjPQynmqrHa/NSOFKLnrP31vC3Yw1suALr2hpcrfoIMrl9k9oB1p2D9azqedqyKlqxJtXz5k2WV0pG/GaFuUIX0SO4rqps8JTDEWV1wEX8mPfFR8NR/10h0Gl3zBROR0BPluzSutreLyoeYk+kE5wzMuz4C/QzmRwxDOwHlYwJuSGTZLqhCPRtvUXeO9HfLneMuQvh9Kr6GAFYa9Bv/hScX06om8G8+KiYvBTwVGLtqt5FMhgGHH5OSXuDwe5WFtTNZJ45HIliMjlS+43a2d25FUHE7hz1vujZCZmsqhJuWiKvcNeC+N6239plQcQHB6oq+aJlJQJvKr2tczv0sm23yeVIsBu/q9INtSBkYCqB6PiE0VmjIOes6NgCm8Upgc0JzcuZ04kbNk63O4FaViJZK/N38iz/9o1YC7v+Lj8HRYVdqucrM+Rm83L5Xg+5kTQye6z9WtiLwKisb1syaAE40oufl1ckOkHXcL658s3KPkVyXNUnankQ8ZnzLekRKvq2rzrH3ie4twS2WOdWs/rQkmiYcePger+WvqIOfNelw7bsLcHfyNa9qyDm6JfpfrP3RfKRt9bZPhEVzEJe9VXBr2K1h6o+MpMxJftk0RGpBRuEDmeV6bmPwiZW6U8i2Jv9R7nN39+IdyY7i6IV/4oev8zwx6QT/DDxqPoESaiFz2KNyIJ6gU5UEskCmCt/hLXjhNpvsO9kiWShXjyDsGFlaF6G57NWDvrbkchglY0RH+Y5WcC15EUEKlXB34Z58BsI/bOvmo0153H9xn9D5hU9ZyavSGTzitOxdlRomOvcD06FxoWYR5lcFexfc8bbdXzBpLL0OLn/xVMOuyf/ScjHgSZyxWPHX9QMDDVfIYgTr/NWfgCZZeixfuUyu6b2GyTay3RLZsxfsEcdRPquEOyGLLMGmwUqP1Qno/qaUeErS8prmIu+izJ9wVuDNZf/hPy+IlCoCq7L5LxA5p3QR53QlY0EJWsyc106flumgzsF8eIR9kBnkdkK2+/ggm+u5aFEPIaIhxHxECIebQb6xpj/Owpc1DyEiAvWKdiMmPnGOTcyTSd+J/sdvvCLC5mmE88ulvjL92JvsO/7ir3BvtRgGAZ7PB42pI3hV0AxVimsRf9OzB6OL23PfQTZXqlWVpW+CodlOjnTIaqgh1F9JTNue3ZB4dh8rd1oEVVTEZ88+JB5F2hl+qrUQbW1FyIbUAZBFSY7BHMfhgKkCoqF3GaTdAUNkJOCxRyjAbBm+wW7oQiWvmavsDMMGNruQhh5KidW1h8GN0+GgftaaSF3F94aBDyc6D3FGrVWCg0EjQuZZ45UU79UfQRDT0ZBywyFIePXMCLjlfiurFjw1Rr0CLmgjDpl6fc3Irha+Kz9WlcWqmkHM0/JEO5DVm7pq1Ck451OejjmqDQM5o1XYQ8J5NxnJgSoqmy34D3t+LP9esHTyasddj4orfqLyrzmFXK6uMr3ISNSDRwwx5IcFh+JiiMZurIwF6r7uwrkFYOiyiidnGk2nBD2wDkoM2x9LfhdVfBAhaJ/Y566suT0jNtulQfMmHG2BHvgiF9/aWq/G+iE81VU7sVXZPc3InPV2ow7nZAsUD8r9i7TSZ0pWvmhvMIp8y4zKPZreXVyMuo2QUmY/6T9Wnz1ujVxZc35cJzMnZQ8gKqCqgpA0F+pzpKKdxGkUVW5xVPByvZs7ddwkM7axx4UtlJxxUML4NmcHGeC8kYWjBY0N/FUjdF7h+D4kmXWxo9r4aH2vwl7cF8ZBU/0/kEzi/tW+fXCZ0w5Vpj4edbdsXcpozqcvW/suqp6JuyJJobY9Lx5+yUP+pe9CcPdStHnrTF1PAUlp3BOqipnar6S74uQy6S9cqii3H67nT5qIKj9FkkW/Y1wMhQ8pZa/Fe8iwzw+oSJaaW5JtDF06IwlL5tVXxZ9t+BfsQQh2RqIp1au/MB+vWmRWpcR1dhFz6kDzlWfJGb/C6y7xLnSq24G1vyyk+1U6946OJgXHa6m7Q73YX04OQzXX4rArxWGAd1h+x1Y5/F6kq8FiVyzRtvlbqATQWZVtX3jfDj8nM51WSJWPFz5cqfeQCopnSj8nCCeyXrGMSfSna+9KLX7MkeDYoOqXOnKhGxSJSMwIzhlDYoZOqpXfM1qWsCdD5rVaPfJ5X5Hhrqyipk55GajK4v7Omvkti0z1t+c8fbPN/4Ccm/Dz+2JU2EPqgkr31f/tqd8cM5oZqyjzN/Jr+kazkeZ7dHfADty4TR5AhEzxjNrdGrPIwKzqmfyNWNPrDhdre9qIZxzsgTd5uWwP2VzHN+qYedD0Pn3BFadg8DW7sBdjPUjC55WfmgPWskQ8eM+KhYF5lgVmNgLVhT/J+bMz3sMidKq8zPUi7XZvskuA4ufB5vTll/Jnd5aAHJL6Lx7Gt5aOOStwZTcR7EXVfKYOY6J4Av7NU9Fcnm7489mICfFakctiL229gJTD4vzkRS/gAq0QSSjsh5WJ1olQYKeVzcDNnd/A+TWspNi9kZ3jp0+tHMrKhkrPxx8Qlagw6SulMj06s/kwXFmrMe2dWq5ufQ4dQJs5YcI/MvakBg6dJDuHNPu+PnAxsEc09tEUl/+EzH9rvpTMEl4KrD/5k226/C5j4L5StgfHZslz2fgnk7tZnwt0MGs5/O6n4IVQAV/OyqTrXqeSH4V+l08Ij7ov0KfUQUKlb/ZhqSEdMLas6JzO+wDT4V8vnY+ZOrmLrWe5muG7hvvvwv2IMl/45Wg/rYWkohg7dIT7BWJER/0wop3kYAhTbKtw35ySGJ99JFHOM0M7A03q/guMik715n/e6H5eRoRX3wWApeb6jbx+Z+fzw8sj1Xt/2b+b5imE7+2/TVu87ZxeVc59wYG4Y/6HsAxhiHB3mDffwn2Bvv+B9FbagaC3Phv4eRJJzhLBgtXPhyyMmeRvx0Hhuog6K83HVEPp/ab3ppYzxiZ825GWlyQyKFCSoa+KgRb5k9RZ+BG/HiPskCjqLKTOVCZYTBs/AUOOpmjQ2RROxnlKhT/x3SiSihI+xtN5/mv5N81dDUVoxMdXKADCv6s0epM6ualcASpgm4qDvZgN37TqR9P8fPJs94HC0HHI6PUcRdCobQ6MUTvCydj19eCDH6VMt2xGYFAWUC54j0YJqqM85YVMNa7su1GV6gX8yQoaOJ7ZUZ7Qyn6BDTMxT6v+sR+Xy0Q228yh0zzMhidOx9SZ12K76uqT51o9gaLYA/mouBfeDYZRNDO2g8w2ANKkXgu+oTvPY6stdpv1VmETtBCUIQzbrRfcxchuzWd5EGNrkzIyFQdggJVn8BRb81S7dwWCwin0seNOValuO1W7KeBUOk1zMX5sf6yxHVllRPeWshTFR1l83IkKqiMtJAbhsXKM+XXtSCcELL96MrDHC0/1d6nofpzJODUfu2cKa1Cxq/VgQc9goSYwToqO7cjK1NVsbP8VDgRsu5GckA8tADWgqyXUcnLsXvK+neK5675Ul4xK2STaq/7muDgklVN6GGcJbKKfz2M+Sh6zu44ETB0zKc16Bz97WYY8ra+VDrO65KX7ME2ZqyPzdfBqJVlDAvjVuUkW3+5OtAX8ZnVeZ9jD8QHGGq+hCNrx1/sDk1POc6J5uUIqNnoqYqdqcy7MmPzLKsgKXsT11QJVTVf4rwRPaZULA0ikKCi/qr8EGtVRkVle2YFdaPA2ouY50zAb8oC6D07UU22/NTUKaGrP0MwIp3UVHXJUPamPOg3EEq51eciwciK5qWQjemkph1kNqsLDrN/7i6MrQOrw1Qk5a27RB1Ya5yPBBTr/ItqnKXHy/ueMWPPOSW8zNtPHlTzt2H9uovstklXpplh/zp0KesZ2rlV7ggUqJ8VYxGwjkkL4YxYdRZ0KRnCXthSVn1DwFWAs0Rme3gq1PtAJHfIEoc6t8EeXH0uHMipoq8KwaN0kp+xs0bhmoymzFuLhIJk51h3jjzZq3M7qixlCHYjMUmmv1d/FnP8WxOX9DCcrCr6YmbII3exWm45JY6I/ZZO6mBfR4Y8OLX2AgR6mpfZ51IPY6+o3odIUOzeYQ9Ee+sw96oe1MmghXCGJKO19FQiGCD6mEX8cOxn3Y3zy9ozMNhj2oGbYLs69TEbLILdGL812NO4AL8pC8jpEbMPdNz8i967rvzkrEO130AuLjxUnkTr1L9QYOWZWAsy6BoCF2Yf4ASEPTjj5k229yJnhl637mJ7QC7UC7mssnmSoXEBAioZv1a3XymcjjVulW/dOfATbf4l7AGZT0AEfAqftdsIwpbf/EvI0Pj7Z97FPHuM/BwRCUKlr+B9Wc8xp6B8XxXs1lSDMFoQe8QpwcddiPfQuBBnitX26NgMPcPfDvlrZYPy1iF4pGqrsTuofF/d1z1KwenC/4+v5DR0M0B9E/akteq+6hN1tXPEj3ly8l8tOcZeRSZQ8wWShutm2qlNw16MqWkR7qPqPbvxKiRaCBkbjx1/xllQ+gr2bLzt09+IdSTbqyI5tnOrXc8OdMbOmM7tap9h/hPYzypsuUFd3TcQLD0B1L8yuIvt/rBQLxLQZImRzFj/ER/0za23yP+m+jMzKVTCLCNaoDgkr02dNInTiHgoEY8n4iwiZsm/LPP6ECJubmnm93PeZ5pOfMEXF0TvVdFdwT/57Ce8pFx+7nb5ujivLY93te7iMz8+k8/7/DzWB8tK9z3D3mDf/yK8++67fNhhh/HIkSP5rLPO4uzs7AF/d2+wLzVomsaFhYWJJbC+Fgg3lZG34GB5n49AF5T7GWnqA99bA+Vr9li10HVC5QdqZyJzjCtfxlmfDBEflDuZUhHug3KY95g9g4cZh63KmdhXDSVIFYhhxsGYTnInvmHgUG5apM5mjVb3SXjYXfkw9q3KbrAb8yHLPjMMOLKaFiUGdOMhFGFZFaUWjDl+ZVmOrauhgMnGE3LD6LAGjwRaVkE52nCFOngq6GFSrVISDb/3dO8UAT2C9eMUbLQi3Adje9nJctrT8ndQ8aMKajHHVcJJ3rfocaSi4REVeKpsLWY4Bdo3qQ0uFUS/QGsAXdcgQxoXqAP7nnJ1dRQz1sis0baAiqZp3LRpujlmhZyqS4cT1urkcOUjSLT6x0gKSLVXU/MyM4g0w+7gF8Hz7HtT69sxULSsUstOQT+b/4R9r4skiHSCE0UFLajOZBX0HVZaLm8NjA+Z040ZxtKma/DerA6Sviqs/WRO+dJX4HyLf9+eCufAg7sIBm18JWqqyLhxcNUWg4Whw2nSuADvRVbtU/a6GUCfMLjfqP0ackj17oSckSWmuAqcK188lQgWWf+mZVWsX4jMmFtyrD1D1YqWlfLkIcOI0WnJqqIHUkG86DA5ldaW69X9M7UggmlO9FC7g7qZeG6TEkeq56nQtt6sNJNUr6w4TU2l07TY1EEkDgbm5JRagc7YuS871+MzrWWYPxVBFeGAVlEl+9swn+I+ef9AlZqKMrB9E9ZBywp79r+3FrJtMIH3gaJhLoLRsvM97EEAwXqt6iPo6bJK13is+QmS0KwIdMJB6+RAiPTLKStFgLOvWl21Is4TWTBanIOyBLWGOTjvnSomVL3PQm7ce/ZYyBMrOjbDmS1LOGDGXk4n6AYyqHo0tW/A2lR9b/kpkGGZd0Kv2JOo+SrWJ24wlVGLj1LrSf62GFPGrr8lXhOB1eZlctaQQBf0uv56td7aX49zRrYGhbNV5nA0e/bp1V9xYUGuXeYFOnBOOlUQO8Hfjvdhdb6WvIikzu132NkAwn143pnDoVfIKMwEA4usqifch32Vc59c5+zZhWCX7JoexvqyzpFA5l2w52cMs1NpCnRlITBhZQdpXo5goEx2+ttgj6uc4fGo/SZ1itLOrWbiWyXOf3FeaaEYPfbcSepK8JovzTUa97vdOzCe/nok8cj2TOWHqJhxCtyKajGr/BIMBN07zGBmZ2y+RQVQvPxpnI/5GWiibrCbeeEhSAyUPVPZ62av64FVQUShhxGMVlEUx9OxWmW6rynGYGDt+ehvdU5kFmhbC13c+vvNy6CTqJKCw30Irs7b3+7/qP48ds6oqhXFdZnNFPFjLVgpUQU6t0OvsPpkPJW436LD5PSPpa9gzutm4qyLnytfE9Z6+VtmAu8zMV9CxMd6wdNct/WtAVPapQQhn3wt8ANY/Vn99bE+dCp4a2CfqPSCwmedmTJcefBJxSeA6FrsjMj7h73FS+3X9sClwNxJkKvlb8MfmUol5ubr8D5mDBv4dwR8TWZhwV0I5snYY8rfAotU2ZtIjlT5jPKfxJ6PP0c95bDVZTaNvx1rUJbcE/XjONjUIulu9hjnMaoQ9mCtOPk4Cp62J2l0bsN7EgUEezq45SrAGS7TC/obYM/J9Dtm5or3ePQQ4gPNqj1VoC8+4JdGxJfdcBnTdGKaTnzxVzFd5rKvY5/L8Mq2V5imE1/05UVM04nTnhlEO6T/EqRk2/LeYN//GsyaNYtHjBjBn3/+OZeUlPAf/vAHnjBhAnd0KCgOLdgb7EsNUr5bQZ8xdyKyb62K4bpLEPSSIesPKF+v/QZZaVaBLhxJ6SRXwmu/BhWeKkjjKjCVSkX22a6/qbPXPOVwNqWT3QDVAibtVYpKsEDO/TAAZEZcMriLTXqqW9TBA6EcWLOpBMIeM6vYbb9m6KZCZBmbyHCfN9luzBuGWbV3g/q5G+bhQJYpSILiLJ3UDqec+7GWZNDDWCuyDP8NV6iDcaFeZBxv/Q2yvGTP5mvCs8tK8gU11mD6yA0UW64HtVY8Nl+L/eaEkpflVVn5T8KYUVUqhFxQZutmqhWsuRPVmVq+JgQoC55W0xImgxbE3rTSK/Q3wpnp1ONLhYhfXm2Y7FE0jRvXP8HG4qPl1WTNS0FhsuYndgqzvioEjkTWua06rxvGm0p+5T1uBvUlyrSuwRE1b3817eXuoLcMe05mnLZvAF2Sar+UvgJjT5aBWPGe6WAfIq8uaF0DozjSr5avEb/8na35CeRTOtmrwZzQsQXyuPZrOMjmjLM7pFwFoJBqXaMOIGy8Um18ugogs2WBCT0MWayq/Ct5Ce9ssD2JZNAjeE8LDnYOZKsgnLBrL5S/6+4cOFbmTlQH+4LdkNmpBqtFFXA6yQ1fJ+z6u5z+cyAIuWPnnMzJ2rgQ9EFFz6mrWutnyzM+dz6I7NqQ237vUK9ZqXg7rlkNV8E0sDtB/4wbo9RfNj1Pj+C9yeS5txYZ4FbZx4xzQEUZGerFnlD1pqv9Fs7i9ZepE730MN6zNAEpiZOyIwMO2PpZCDbE762Ozah2lNG1Fj4DeauinI+nUVT1fGmYC53YiY5pMIhPtFEFoWSomwHHa8McnPvWgErxC3B2BrvVTiRXHhx3Vr0i3AddRFZVxQxHtsr5KuAU7GNmnjEU150SmKzoLUUfotyHzUqYFHV5kQjRvFTuSFp3sbq3pxbE+9r199QD+I0LoI9m/g5zJkN/A3Qd6/oyDOgNKn1VVHztfFCuhwc6ILtUa6v87YSkARuCPWAVsVLQ9pbhXJXJL2FbZNwInUHWN1Q485zmsHmZ3Inra4pWQ9Rtet7uBGpbayYzSGhz49G+UV4lJ/qXpdKCQA9D/hU9B51CVg3ZOB+BaNkaE/14VevP8bc1BJSTnZG1X8PmksFVAL1AVYEV7Fb3hQy5kYwoCwIU/wc6xYxhuH88Ij7YtIEuVAhbq6eFrZaq07e/PkbtaKU0jzLObIwF4a2om4ng/aar5VX3ay+Co33X3+3Bqfjf68hI1NPqZyFQHO8D0COwJ1b8UN6GIfMutU9ABlGN22VJoG9dg0QqlY4jqq2tDniBmi/g75Hp0U6U9rqG4FHlh3inVl3aU4mAsqjeUtm4Kqy9EPaDDOE+2G/5/7TT5cc/++ZrnftIh/uw9q2yqjsHSWDWRNZAB/S+ZG1EttygTmpntlPAm7ps9/LrIPMaF8LmlvmEVpwup9AWkO0pVz78W04Bsbp0JCTL9GGRhJVOdjnjyo+d+XMnqe8v7pGKXiDgLrIzbHRuhWwTjCfx52h/PXRgPYI1sv7yxDl2F2Pvlr2BuVQFwmXQgtCfBsIeMVhE/JgPlf8t7FHvd13D2S3TK3Y+hIRnWTJduA9rTlUtGd+KRNb+Iv+f6upTUeHoxC7U34gzxXq++ppM/Umij2hB7PFk1bLi3Jc994rTeJxJ33lRkkCf+HchEY9IS4sG9X72zc+it3t126vRz/+66q/8z/X/5Na+WMLHl3lfMk0nPvSNQ5mmE098MYnv8L8Ye3v2/R/FWWedxfffH6PI0HWdDzroIH7hhRekfx8MBtnj8UT/NTU1MRGxy+ViTdNY0zTWdT16L/FZ/Ofxnzl9LjhhZZ8bhjHgz5lZ+bn1GVWf76kxRSIRzsvL41AoFPu8v42NxkVsbLyKjUVHKMekFzzNxrzJrHng8DbCvqgg17fdHhXMCc8Y6GWtO5852CN/dlOg69vuYK1j256dp7hAo9ayOnFMZg8mzd8jn6euXaz1xLIHpPO05Bg2lp8qn6feSjZWn8t61ee2Z9d3PYzf7q1Uj8lVyJz9R9Y6Mu3z0VvGxsYrWaufJ5+nCtCras0rEsfUsJD1LTexMW8yc8nL9vkofom5abF67WkRvBdPrX0+wkHWWjew5i5VjynjZuZlJ6W+nzqzWGteJZ8nTw3Wj9lLxjZPrXFroG6WfUyhftaCHtYike9MRui5/2Bj8y8TPtfzn2Jjzng21l3CWvvW1GVEoJP1gmdYa11vH1MXKlf0wn+rn707l/Xe8tTHpGusNS5mrTOHtUjYNlZjwcFsbLqWNX83czqxkXlX7PmqvmBj49XM/na13HOXsdadbx9T40LWt/yKNXe5ekyuItZ7K2yfh0Ihzs3Ntf1e9NnzHsP6cJep12TIz1qgl9nQEz7Xcx+NJjIYuY/ZxxRys+5tYC0csI/JfAf6zr+ynv8v+5ha1rKecQsbs/ZhrvpoUGtvIJ+nvPY2X8vGjDTWt97KeunrtjEZG69iY+Zw530zazTzhp/LxxToYL3qM9Z6CuT7qfpL7Kf+ljgZkc3Gqh+zVj8/YUzGwkNZ33Ija8E+1rfcGJMFvk7pWI1VZ2P9WsakF78UO+sqP7KPqdGscqr+XD6mmcPNPfmcfUx16CdorDmPtZ7CxLHueIiNDVewsfwHzNvvSJynSIT1mm+YW1aw3raBtb76pPOXsPZCsfPbMCtfEv62SFA675SPqX423nl3rn1MYs8Evaxvu52NivcTn72vIVqpYQR7bPdXrj0xHyWvsbFwGmuuYrtu1LaJ9eqv5Wsv2IdzqOpj5X7SdzyINdJXZ5dvi45kPfMP6v3kBwW3kXV34ufhIHPLCjY2XmXKmtLE/bcT1dbGvP0Tnj0qI/L+aV6fzBzx257dOlYh8yIROGr0XY9g/W39rfrZm5aysetvbPjaEj/vq2fefB0bteny+Sh/l42ZI3H+S+bPWHsR61t+NTgdtnkpa41LUpZ7eikCOVrbJjaCbjb87dHzyib3WjewXvJaVBdkby3rdTNZL34R+o5kTELf1UN9iWPq2sVa0zI2Nl3NXP62fUy+9ihlptaw0L72su5lY8lxrPm7WAuHEsdU9g4bG69mrb+VDS0kl9mlr+De7VsTP5+3HxtrL1LvM18H62Z1lt62KXFMXlTeGesvY3YXyedJj0Av9HenNE8Jn9d8yUbuo/Zn7NzKxq6HWeutSvzcU8nGitNZr/hQMaZ27LOQX772/D1sbPwF3ld/64DWZHSeil9OeM+yMem5j7ExaxQbfTXS9648c+vnxs4pc8zRZw+4Y+eQWUGV0n6Kq/aP198S/r6vgTVvo0IHcrGhhVPWI4zse1mv+BC6TtVn9v1k6lF6VzazYaRu53Zmsbb1t1y+9QsOhUKJY+pvYb02nY15k9lYerx9njJuYWPuJDZW/JCNRUcmPrseYaN+Duvl7yWcr8nOJ9kzSj9fewFsatmYImHsKVNXsc2Tq5j1srfZ8NbJ99Oma9lYcmxqa0/T2Fj1Y9bzHncck7H8VOYlx8jHFOhiY94BrO96xP4O8p7AO66bw1rH9sQxtW3GGih5jY25E9lYd3HiM3ZsZ6NxIWyQVNbe2ovZmDtRLSO6drEW6GU90M0c7JaPyfQl6OXv2sZkrLmA9YybbTLCNh+eSjay/8hay1qbTInK+K23xpzjq862jclYchzz6nPxjI1LWM/+E2veJvXaa1zInHET672VifcxKxH1qi9Ya11nX3veJtYz74aeX/pmarK8r4GNhrlseBvU89SwELqj6fMRz66bfU615hWsdeYwayH7mJpXYO01LrbPU+lrrO94aECyY1D2U6CbjeU/xDOG+mNj6snFeR4ODl5G+JpZdxWrn72/jbXuPNZC/eaZq3G4cxcXZq/mUCjEes6fo5V4mqaxXv0V66bOZuz6Gxs599nnKfcfbDSvYGPGUNaz7hmYjIh/9tb1bGy6hrkryy4jNl3LPHtf1mu+Zq2/JXFM/fVsbLoOPplwyDZWPeteNjZezVrHNjZqvmLWYu9V663E70YCjnauMWs0G3E2ZsKYwiHY5EI/1nXWs+6J2oZ6xs3M86fGxtRTwHrFR6z7WkFTnE6sxc0VztYVrOf8Gedcxq9Z663Ys2uv8gPWC59Vyz0vCgT0nD8nzpO/h7W+ejYWHsa84of2tadpCESmE+tl7yTdN4lrsjVmQ5q9ehP+1rSbjcy7WOutsI9p6YlsLD1BPqauHaw1LoIu3Lqeub8x9uzhIOvFL0Z9qXrZm4n3zkWQUS/6D2ttmxKfXQtArm273Xmsnmr4YSXnjdFXzeOGDOGxZm++gQT7viXisUTRoN4vvv1FdJ6WVSyLfi7+FbTF/B/zS+YnXDv09UNTm6fdXXv/i+IamqZxbm5uQgzDaUy9vb17NNg3jPYiZYTDYdq1axc9/vjj0c+GDh1Kl156KWVmZkq/88ILL9Azzzxj+7ykpITGjBlDRESTJk2iQw89lJqbm8nlckX/ZurUqTR16lSqr68nr9cb/XzatGm03377UVVVFQWDwejnRx55JI0bN45KS0tJ1/Xo58cddxyNGDGCioqKEp7hlFNOoXA4TBUVFdHP0tLS6JRTTiGv10u1tbXRz0eNGkXHH388ud1uampqin4+duxYOuqoo6izs5Pa29ujn++pMR1++OFERFRWVkbMHDemy2nEQVdhTHHjih/Tfm6NJqYdRk3VDXTC6UeRv2kD7Wv+XX9XNXWckkHHDNuH3C7JmPabRJ3t7fYxnfEW9aSdTPtVPUwN4SPJPWGflMY0ItxMx3reoGHH3E2l3pMT5+mYM2jEtW1UXFFP3DmKqLMoOiZ9wtnkG9dMvOE28u57LvXtd03CPJ1UcSkN193UP/YnNObHr5J7yDH2MZ3xDrl7PdQY974mTZpEh04dT75dz9DY7u3UmvZj6uovShiTFjqDRh38H/JUd9LBh0+SztNJrqdpePtSKh31R9JbY/c/7rjjaETYQ2mty6k7NIXaXYcSDxmeME/H1L1L+xJRU301HX4wxa29w4nGPk6j9n8Ga6+nxzKmG+gozxyizDup4tDPKTzysIS119JYS9MyzyDX+Cup8eDnJPM0kYhCNG2a2zamffyFdDiNphHnzrDvp2OOpJGVz1NT30TqHf8z+9prGUmjA26aUPx78ky6ho790fXRMQ0xwjTqiG8pbfhBdDSRbT9NSgvToZN+RIERh1Ft1ziK9BYljKm5vYf8rZmUZgTIt89p342MGHIz0TgiKiqKjYl/SRMOGE2HtD1PTTVFdMSUn0THNEzroROqribP/jfRpEu/lMuI/YI0tPhpap98L7UfMClxnvpGknHQsxTyHkL+wnyaeuDBCWMaGaqlkeFGGn/s9bTfeLKP6bCpNG6f4VRa0US6QYljGjaE0jKupf7RP6QxgTzqnHQr7XfZV9G1d0Ta0RQMTqTOqmY65doG6g/oVGPuj6mdW2lKzyoirZ/c/mH2MVXeREPduRRJm0Qlx61PGJOnKYcmNs2j6qGXU/8+fTT1wGm2eTqt9Ie42c1hqqqqiY6JmSkSidDQlT8gL0+m2mlvJo7puIepLPJT0ur6iYbEZIRSlvf1RWX5WN/hNH7KvTS5YxHpaeOoeMivE8d01FEUyH6M9mn6jEqPWUaR4VNia6+pgfxtuaSn/YwiPIWmdnYmjOmQ1g9pcu980saeQsNGTk557ZXkZ9GEvtUUGH0SBUYdZxvT5J4ZNNG7jva9YjV5wyMHfj79+Fvq7Giltm4/UQjrOv58Coy8hfY54BQK5MykcYdfQlMPPDA6puHhVjqo820aP2QkDZl6sWJMB1BJ4EekN+pEVBQbU28mhbffQ0NYp5HhBqoYdTcdd8YUU0aMIJryPpGbKK2vNCrLaeg0Gtc0l6hprvkDvyPf8KOouryWeGhLbEyHHUi9FQupY8I/KDTyCNuYRrRU06RhU6jqiK9pvzEn0VSihLU3KhiiY4eNpaHuXKqK/Ng+psu2kyfzCer2TCJvUdyYRoygpoZ6OpyIhnRtpYaSDXTYuSdG5+nwtmIaHawgbfhk2nfYPhI94kw6/khTltc0EZE7Ye31FbxHw+s+prYDHqD+MWcn6hE9XTRtwnXkG30KjTj8T7YxDY+cQYectZzGjz1OPk+RPmJXHjWUbqa+MQbRkLQEuXds7a00XOuk4Vo3GUaEQtN+F117aXofTRv/C5pwxK/JGxpGtWWxM25ipIgOCy4mzd9NraN+Hj0T4nWjcEcv7WeMp4aqWhozdWyCbnR403M0wbuOQpWf0Mifb7HL8mNeov1GuKi2NIt8+r6JYxo3jqr5Iho27UjyVrUTD+1NGNMxxr7U7wlTW7wsj5MRwylEJx11NwXHnkkVcXoB9tPPqd/dThEfUWt1C2nDQtExufa9kGhCLY3vW0+tjVVEw5sTxnRi5Sc0gogio4+hEUOG2sZ01PBcGuvZRLVj7yGfPoaYmVwuF/X399OECROIKt6h4IgjqHrknaQVFUl12Kmdi2hq92cUPOR2Kq+EnjbECNE+Wj0d07KUwqOPpzLPibExjUyj4488iHzaPmTscza1NXRTsKPIrsNOfoOIiCY1J45pZKiejqu9mUIHXEmjL/iS6hs7E8Z02KQwTdx6NWkjj6Syo6bZ5qm0tJR0TSMaMiRhPxUVFdFQ/UeUdsxK0lpH0yl1f6OhdV9QyXEZZKTBRojXy4MFn9MBrm+pxHcCpY09AnIvNI6aepuImomoOTam7qZiSiv+F/lHn0TeoxfTmLYeOvSwsdExHdnwZxrn2048ZBjRqCm2eTp8XA9NqP+WiIga62vJ4ypKGFPJPveTPu2PRBXNtjFNa11H+/UupbTFS8k4+0sKHXST/Xw67BryD51CNU0R0tuKYmvv0q3U6w1S667VNIQMCo84JGGehuU9QJPd86l22ps03D+ZphFFxzSEIzT68K/pqJa/UdqmK6n+lA123WiUj9KWHUmdk26l1qkP2+fJwX4aprloaud7NLl3AfGQNCoackv0b6e4vqID29+kIURUGziBfPv2x+2n48l11hpqqS+nIfnbSE8blzCmIQWP05SeL6nlgIdo6EGX0oEnXmmznw6fcCVN2OcQamjppD5/R+KY9tuPgst/Qv4RR1HL1EcSxtTgGU9jJ1xPnQ0uCrcl7qdhmovG+LJpdLCDDjjgYgpqQxNkQTKbMNieQ/sQUc/4a6iluo32nTAkOqaO1gY6dNzlNLFvDWnFL9OIo+9OzSaccAqFRh1FZISprLgkyTxNTpinyT0z6JCOV0i/eCOFx58t1Y30NRcS95ZSyXHrEufpqP9Aj0g7m6ifaGx9fYKMmNA3gQ4noqFrzia6xbCN6cSWe2jE5FOpfvLjcr28ZzwF9vkbuVwu8peU0FFHHWUZ04l08L6X0cSxI2iIYSTKPf84mjLpTAof8Udqbaonj3ktLS2NTjn+MBqy7SZyTbiWmgLnEpE7Oqa+pmzqrc+g/V3fUv8+P6L+o59I3Xa/YAnV11ZFfzN+TDUVBXRswY+oZ8K11HTQ9IR5GtezhA5rfZKIiEIjD6Fho6fZ/BGn7nMYsa4lfJ6Wlkan7LODQp5mKh96jW3tuXt7aZynijz6gdRMdlk+2TWbJnhWEk84g8ZOPixhTEN1Hx0ZnEVjDruUak/bjnkyf1uMqXLUbRQ87EaiXiLqJTpylDc6pqFBP03a/4/k7Z1M035WSSNGjLSMaTSdcsrVFPY0UU/eB+QbfRoFRp9AaUOH0in6txQceTS19fgoMmwKBUafGB2T74DrqY9OJHfuehoZbqS0yafTEcedHjdPQ4moLjZPjY32eZp8DjX+YB15+keQbhlTxbSPoBsVJcpyjKmNRoYbyT/qODrmxDNpxBnvYkydch/LET0NNJ6IPON+SuPP+8IuI45ZFLXdI2XLaWr3R9Th9lNk6jV0yA9+mbD2Dml7noaNPYQmnDcbY6qL/eZBk++gA478HekbrqLI0PFUeeTMxP3U3E+R4bfTSfQtueq30YiD77TJiLH92+nQfVpo6Al/o6KKmI1ARHTKcT8lrWUN1faMgj4t1t5JJ5KvPZ/qO8eQNuV9olaiUa6qqK3R7T+RRh80nTzt+9I+EybRUfuPSPAbjQg10BGuN2m0r5G6Xb3U2hO3j6dOpakn/I3a8udQZOtr1Dv+8oQxVZcXkO5rJ23YRDKGjk7dv3f8YZS2YDIREbnHXU4NJWWUNmwknXLKKRQpeZ1GNH1LhcdtJSNtX6n9tK9vJ+2nFdCks56kTg9Hx7Sfex5N1CtpzCWzqLlbJ1eDZUzexdTlDlH7iAvMT6ti89Q1ijq9aaSVlNCwoTfS4ZfcS2NHHUClJaV0bPmTpKftQ5WeE+m4E56zjWl0oJyOq3uJIlqYfGMuon7fBOouKrKdT2mai6Z0f0GhiRfQtLP/EB3TECNAk3rX0dCJt9DBk8+2+/eOeJsOvdDcT1WdRNQZG9PUw6j2kFfJ2+sl6i1NXHtVVXRgRzmNiLRRRes+dOSR19G4tJFUWlREuq7TQe2v0gGudAr+rJJGeLKpvS6Puva7Lfq7P2i6mfTxp1HnpN9TeOhB1Bs/pr4+qq2tJhqShv0UN0+9kdNo1AEPUXd5NY058Dk66ryjomPavyedDu54lTpCk2jKob+mLu1Aaqt1k5FWFBtTdzYNrXqHXOOvpEme5VSRdjVNPna/BL9Rmu6hg9tfoVFHXEf7nHh3wtobHmmjo/cP03BvMYWq0qn+kJcoPAJ67ymnnEJU+y0ZvWVU551GI8NN5J14JZ38g9PJ6/VSXXUJjevfSiP3f4BCaZfQERTzhR3c/grt75pB3snX0dgDT0+QEUONAJ1afi7RpB+Rb/9fUlv3MOoPxo3J1COM9k00THeTZ9ylib6wgI+GH7OCTqr6BWl1c2n4UXdZ9tPldNwv/xy39ixy74KNVFVdQweu+Q3paeOo68C/xNZe8zAiOoxGN6+m4+p+S3TaS+Se8nusPY7QD8oeI9+Ei2nfK3Kpu384tcat7QOHnUFTTnuZhuY/Sp4xF1LdoW8mjKn2rAoa2rKQDlj8A2o+8HGafOwVCfM0zruZjmx6CDe7sY9Ky+ttMmLCxAnkcrlpKg0MU4iI4/47FAgRmfPU3dxt+3ufx0dF7RhTS0dLwrVgOBjdy9+3uMaECRPI4/FQSUkJDTFtP6cxGUacE3NPYI+EDL9naGlpYSLi7du3J3z+yCOP8FlnnSX9zt7Kvt0bk67r7Pf7OWJWMg1oTKWvsl75iX1MYS8ba85nY+ZwNpadIn/2cIA1XztzxbusN8yXjynsZ83bzFrQa3/2hoXIDNl8vXRMep7Z+67k5dTnyYfmtCJTP36e9LK32Nh4FRuzxyJjx/rey95iLvq3fD68tbjvrr8r50Ov/or1jFtY93fK11jTYubytxIqguKfXa/6PCE7PCE7sOozNjZdw5q3OfW1V/0pG2sujFbvJcxT4XMYV8039jFFIqy1rmM98/esu0vs81T1BeapaYl9rJFgLFt5x1+k86dXo2GuVj9fOia99HXmbbemvp/6atmYPYaNOeP+98gIfw8b6y9nY+lJzLv+Zh9TTyFzzZest23CvpHICN3MbJJVWkarPHry5M9u9pIwFhyMiof4Z9d11is/Za1uDqo1qj5PSe7ptTOY5+3HevMK+98vP4WN2fsm/GZ0TFqEtS7QKugF06XzEc2IDfclfB6JRNjfmoXvbrlRPh/tW1jP+QtrrpLEz/vbWM99DPKvM4tZVVER8bEe9Ng/D7mQPbn+Z6z5OhLHFHTjmbbfJV9j7lJUnUSCg1t7bZswj+suse8nU24as0Yz+9sSx+QqYmPF6cxLj0Mmcir7SWTqmzQpeiSU+IzdueYcPiN/9ob5zI0L5WNq38jGkuOVFc3Rf51ZzHEZ4MbsffGb2X9Syz1vjVJmD/TMNZadxLz0uMHJiIAbGatBT+p6hGFgf1jOdGZmYzsq543Z+w5qTHr3TubKD2yVL9EKl+z7IWfM7PKENZb/JOR5sI+NsM/+7CE/qgAslWp69ZdxjAG32dde0fOs7/ybep46tqOiJuse+Zg6M3Hv4hftY2pejt+MqyKyzoex4kxUyFvnqXEJMtMjkrEq9s1A95PmbUb1lWrtiUzWvH+ypkHmeb3e2N+7y1GxIxuTq5iNdRezXvERG956NrS4jMmVZ7Gx4GD52msB/bte8b58rWb9IaGazDam3oqYzO7IsI8p6w+2ivz4tWcsPoaN5aehEmTTtWxEgoqKsa8hx0I+fNa2ERnHITf+3lON7F/zOusa6+GAtBJQ70XfWN2sQLONqXERMsBV+yngYm5ZZatSS5ARfXWoNgkHE+cpEmGtawfr2+9ko3Nb4lg7c6Cf9dc7rj1j5ZmoiLWuvfq5rO96GBUuKhlR9RlzxXv292tWx+k7HpJWK2qaxnrB06jMl+wnzV0eyw5vXp74m9n3Mc9IY8NVELUPbGPadC0bM4bax9S0PFqtaGy7zT6mgJt1TxVzxGefJ12PUlrqW38jn6fuXDbWX4YqlfgxtW7A98yKGJsOtOUm5pVnstEwL8r8kfDskSDWYpw8tz17dx7rPfn2z93lrBc+yzx3EqrRJDLFNn/x87TtDjZWn8uat8n+7C2r2dh8AxubrmbDckbruY8xzxjGxtZbEjLmU5F7evl70QpzW5XIyrNM/e1X9jE1L2Nj9hjWar5lr9fLkUhkz+nlepiNyo9QjRUnD3AO/Sd2TmX8OnFM4QAb8w9kY8XpbOz8K7On3L6fSl5lXnshawGXfUw6bB5j9U9i1Tnx81T9FRtzJrBe/BIbgS77Git7h42G+fIxrTmfjbn7sbFwGuvZ98vno7eC9fx/sdaZnfC5XvA0G/MPZL2vzj5P/S14Hzn3235zj9pPXVmJ53ckYvoIfmmukZukY9LL342y/EQ/D/lRORnys+4uY+6rts9Tdy5z0XNRVhHZmPSyt1FFFHAlzlPpG2aF+UbJfloL1gN3SUxnEn/fukF+5saPKeBG9ZusmkfT2Jg/lY3VP5HLcvHsDQsSKucHOk96+XsxmS2p4DN6Ynp+wrOHXKbP5W75mMRnwT7W/D3Mhp74uah6rpupHBPOgzTb53rtjNh+LZie+toL9cWqoq2MUB1bccYV/hvVd/Fj8ndBri45Ft93FybKvQyzN7G/nXWfpMJ83v5g/Qj0woYMh6JjivS3s7enMerPSxhTewaqo1RjCvnY8FQomTWMXQ+DsUHoeXn/TJwnc7+r7KoB2Rqd2ax1bE9NRrRthH8u1MvGuovZmDMh8e/XXcJGnB8pYUwLp7Gx8kdg7WrfIl97sjXpqWZj1ijM04xh0A+tYwp7WPM2QqaEg6xFwrEx+bvZSB/CxvrLsWdMKt6Ee5RB3zA2XQuGtTj2D8Mw2OhvZM1Tzfr2GCtZVEb0Vpr268P2Z6+fBzvM32Ufa8jHxpZfwT+rmqfuPDyXqWNJ58lTzYavNXVZbsQxQq06OzZP8X/f3wJ/a8/O2DNGIlg3vZXOa69+Lmttm6VrT6/+ko2Fh8L3Yx1T20aMee1PmfWw/dkD3fzoQ39MubJvXFxl33Uzr4s+e0Fbga2yr8Edk6urq1YnXJv44kTpmJT7zGk/pTBPtv30/yGuYRgG9/f3J8Qw/icr+/YG+waBwQT7rNjbsy81RIWrEcdF3ZWF3gGbrpb3z1lwIPPqc51v7GtCPywr93fn9phDZ+0F9u/5W8FBruqh1DAv9n0ZNl+nbpgb8aG/TvHz4Im3wtChfA6mj9LqH6NXkgxaAD1jZP1iBHY+ZFIuKDjpu7Lx3qs/V1zPRK9Dd9HAn7n2a+ZVZzEXPCXn9V5/Objylc/8V/SeC/Xar2mh2DzJ+kBoAeaar9D/wArDwDpZcx6zSQWQgMw7cd/eUvtcGQY+23ileo04QXDCp9KPY0+g5iu8z/h9aIWqV2HhdNNocJj7xoXM2fegX4AVPbvwfWtja4GGeeA0n5Emb/w+EOgR9Eew9vVoWYUeF53b5d9zQn8Deh/I+gMwY003zGVrjwHDMKK0W9KeGyE3eOfTyd6zxRPnkDYpUlJC0b/xPVnfKy2EvbjocPR22dMwdPR5kPXnKXwWfQple9ldhF5I6cQ8bz/79f4GnBmZv2Ou/lT+201L0C/Eeh4YZs8h1bpfejx+d+Gh6AsyUPia8a47t6M/WDrFzpSWVcl7SoX7IHt2/V3dryPYg54Psj3FjHeiWte+Jnu/xz2BRYej90k6MVe+b78e9jBXfaJ+l3oYvZBEH5B4iPWTTuo+EA1z0LtF1f/MCc1Lce+aryzPFGHWggjAyt716nNxNg4GegQ9pxYdJp8r0eNG1mNJYP2lzFl32z/P+VPUaSP93Vmj0Afzu4AeZp45ItoT16bn6RHoI7I+Sz27mOdMYC5/y36t8Bl1j6DeMpwxHZvl17fegt5DNV/gPJKhewfOXlkvt/YNzBXvynsrMqMf6a6/o+/RjDRO6AEa9qDfqky3M2mblf1TRJ/VmSPtvUgMHT1E/G14ZqfzezCIo1nkQOfAv1f5vlr3Yka/lUAX+paVvCz/Gz2Cd2gdk2FAl1b1kyx8Bj1wnCAcZdLfjUvUkfVDVSHYg16j226V95ZlxpjqZsj3euWH8jNfYP4U6KQqdGxhnj3G3lsr0IWeUCp5n3U39unuyDEVSl+NvUvV2eyEtRfiu6re2r5mXLfKhJqvoDP6W2XfgtwrfBZ6eukrqT8XM76b/4T9887tzHPGs7HwENB5W9dvdw7mMlnPPhW6d8hthN5SnH+ynqW6ZtqIB+K7ravtfyN0TtnajPhxbbDnxewx6l723lo8+6qz1P2ShBwyqWIHBD0Cmeutg50n03m7c2L7bttv1ffq3oF7xWPX39CLub8eey/+zPDW4tyt/Vre15EZem3Fu4n9wquRNMvNy5kXH4nzyoqaL8x38RHkoBV1M9B3e9Yo+/nqKjB/s8XsFXZjTN5UfWSujTWxv9fDkLO9ZfK+5u0bmTu3xf474mduWizX37QQ9PDse+330oI4I1X9D30tzJuuwfuUQfTU69xm9920rmFOH8K88Sq7jaoFsOYK/gV93dpTTSD7j/IzMNiN31T1cCt5iXnxUfa+ecxYjzlITuMdf5Z/PxmWnoCekzK0rDDlrsWH4TepDrfeDB+Z9RyN+KA/b7gCvR+t6MrCfhXyor8hesmYtQ9o/g0D699bI5/T4udxTqeCqk/gE/Q14zetck4LQaZvuILZU27/ftt6nCcLDkR/23gEOjAH6QRbU4Wc+5gXHqL2D/ZVYV8NFDn34wzZdA30ZRVaVtlldonpQ9h0DUv7zTsh4kN/+4p3TV1K0uu5twRndjIfgysfez7+nYT74LtV2Wm7A+GrLH/bfs3Q1XYaM2TBih+ih6vomxqP3jLILkO3+W04/wnsB1WfZ2Z8J9Qrf58qGAaeK5V1Y8Wan3Dbp/vzkBR79h126LRowO6G2TdEb1fjqrEF+7p8MRmX0ZCRcG3yy4q+pN8DSGMYDtjbs+9/AUKhEKelpfHChQsTPr/99tv5mmuuGdA99gb7UoOmSZpb9pbCMEgn5lmj7V/qLYMy3bwUzmmZA0U4WK2HoLcWilvhdOaeXPv3djwQM/JVDW+9dWrD2VsnV3SZ4diPOhEszt7ar2FsyJoZDwS+FjjCVUZxsBtGQG+p/VrZGzA+XQVqZ1H3DhiLqgbpqmbyzGgEnn0vlId4iMBDOjEXPG3/3sofwRgZDHQNxn7hdLVituos5oXTUr932eumgiVxNgXQI4mz7pYf+P2NMWd4y0r79ZZVUOJkRtx3CdFsvmGuWlEKueXX3MUwpr018nfSV4216RRAyrkveYPiwaDkZSi0/WZPrl1/j13z1uB9DyYw4GvBHh7EftU0jUuyl7PWvFL+feHgb1xgf59aCE7h+tlw9libRdfPhvO/YZ58r7eth3PZSYGfP0UeeDUM/Hb+P52DD98VtKB8X2y/PSZHtt8uuX4HDCmVHGBmzvoDc/k79s/bN6IJ+soz8W5l8FTgvcbv965s0yH3MgJytd/CgCh5iTn3YRhIBU+ZAczJ6gSP+QfAMLGiZyfzll/Jg1PM2JNbb0l03Fjvm04IrNnGU47n7NhsD2x0bIGMKH8L69OKdZfA2M64Ue5QTIZN18TkpxVmZQsXPCU/mz2VaAjvb1PfXw9jbuIcFMyM97/qLBhzsgC8E8IevMeSF+XBKyfokZjjRQYthADWrNHyYEv52/KEFWY4RToyMBe13yReM3Q4q1ecjiC51VlV+zVk9o6/yOVIfz3GW/CU2kHnKY86z2x6nplpr3RytaMaCee3xegO90HXUCUvtW+CI0rluFPtqe8SVZ+YZ/4q+7X+ejjQxF7Tw/gndLH62Ug2mD0GznwZrAkF8dDD2IuyeWTGOVj1Ef7Xii034L5Fz9llb1+1KfdkelAngraFzzDPnWR3ZCw6HGvPCWa/P+7KSvw8bFY0ZP7O+ftOCHSp149h4HnbN8kTU8IefG7V7zxmRaAqYJEMXVkIHqw4XX7OufJx1sjQugZrwKpjx6NxIfaFLKGGGfPVudX+ua8F62DpcXb9T9cgn8relAfBe0shk1WBYpF8ueAg9XMXPM1c9bH8mh7BXKQqd2fvy7zuYgTQZTpD83LYibIAx0BQl86cTly7+bVE25YZZ/Oa85iXnSzXeetmIMC/5jzm9ZfZr3trMNdNi1N/rrAX70uVsLDqHPlc6BrOGaez0TDU9mPnNgRky15P+ZG5fhaCRO5i9Tzv+HOibh8PkyWBcx+xX8sFqwgvPxWB6Xj0VeGcqPkK+uOmqxOvl7+F+ZEFV52Q/4R8XpnxnrL+APlT/hYCkVYEu5mLX1AH0Jcex7zspOTP0bo6FjRkNgNu3sTzvPytmG4tswkWHQbdWDxXb6l6bTGrkw1Fr3HZGcSMuUgnOOpl55iTo7Uri3ndT9VJc8zq5JSqTzDGnX9lzrzLbm8xQ+6XvWEPBjMjwDhzpPp39wRKXpbrjyE3xm6V2YYOfTdZgKjyA/WeYoYdkH1vwvrXdzzITevRK56334F3Kmwbw4idmWaPzQToEcjbji3QOwcTKIrO4zr7tfWX4drqc+2BHg8q0XjxkfICg47NkH/5TzKvOlut8zph/aVY5zI0LbYHr8rfwd7yNZt7+kT5dwuehg1p1VVCLugLnnLYAYMpInBCoNP5fPS3Imkv7zH59Zz75X4/ZvhQM++SJ0r07MT5KhtP2GsWclwk1zlEgDqdYAdasfJM7HcZqj5h3vBztX0Z8cWSUqxzWfUx7i3zGxsGvrPlV/L7DgRlbzDnP8F/vPdeTiPirCSBviwiTiPiNWvXRAN23b7uuEcy+OoZVycE9PpDsfed3ZydcO3wNw8f/LP/H4c0huGAvcG+/yU466yz+IEHHoj+t67rfPDBB/MLL7wwoO/vDfalBseN0rPT2WEoqqCsjqr2TTDgCp6CkZIKWlYwLz8F962fldp3mfEsTYvljiZfEw6wwul2o8Wk9+Hm5fLquu134LmaFqmDMVtvhoFqhahUSyd5Fmv15wjkyZTVgWLxUVDgZNhmNvmWGexhL+ZY9dstq6AQyIIiYS8Ut8FUwYV6oRjKMu6Y4eBwymQP98mN5kg/5kqmMDLHnJzpBAXsfwsCnVBkrZmdzFBs8/5hdxjHQ5VtzIx1v/ai1LJyU8HS4xBgyP6jPXix6Ag4z4I9GEP8dacqN2bM4bzJzJt/ab9W/o4ZtN+ofq6MX2O/WyBknl7wjNxR37EFDgGnDDIVyt9injM+ZhjLUD8b8kZmuDJDaZU5G3c+GFu7MifE7qIrG+8jlaw4ZjiCip6D7JU9d+mrcNTIHOERP953OiE5IBX4miBD1l1ir7KM+JA9apXV86ckVmamE4IPquB+21p5pr1wUqmCMW3rcb3iPfl9t92mzo6s+Sr2bFZZsPFKZIzPGoWKtFThb0V2tTWAIyB+V+Z0blllOrQVQYvKD9SGPjNz8X9iZ1HOnxKvmXROnPUH+/dCLhj7XVlqZ2O+Sd8tc8bn3Ge+a0mlmmHgbHNK7qh4F/JN5qSfOwmOLBV8TWbly1/l13f+1ZadzcyJa0D23MtOjl2X6Vi+JjiGzXHZ9LwZw/BdVVW0CJank924Fmdo8X/k3xXVRCrHdEeG3JEUcsNx5iTTl//A+X2r0J2D88dTAR1syw1qB2XW3Xj+gTiR9TDO3qJ/I+PZOhc7HohV88iqzdzFsfcs04XcRXgfsmfNfdh0Ap+nfte13+J9WasWC55y1ttEcHT9pXYHiRaMJXioHCT+VsiqVKqxo/cPQXYvOxkMI/Ho2QWdU7a+In58pjq3a7/GWeHEsCFYI1SJhrXfyNdnxXvJKxGFjZEqG4AIusr2oqHHri3/QWr3ZU7Uh1NF+0bIc1VyJTOCSwX/sn+ezFkrdENV1Uzt1/Yq23joEdYCvZyXm6N2Ai08FAkmVogknowb7ZV0WgAO6WRO+MEwVTCjCqj0Nfm1sAfBC5lO4S6OnQmq83fljzBm2329zmty1mg4WWVwF8LxO/8A+/ozdJxD/Q1IfpNVTnjKoVfIzt/eElTIlb6GPaBKdBVVIPEysr9eXlHPjDW7uxW0vaXQx2Vytytr4Ml4TgFaZqx/Jz2gYU4s6FzyIv5OFaTp2Yl1Geiy638Nc2ErLToMcluGqC7zoPx62AM7djCBmGA37EBr4mftt8wrz1DbiIYeq8aUJb/pkcE9z0CgR2LJcVb5mf8EmJ6ckiDCXtgn1rnob4BOodLRDV2ZIJug59XPRnBL2F1brgfjgWGYlf0W/4oIuAl6UVUSmxZAUFh2Rkb64T+TJSB1bpMnNzJjrP42tU286RrmmcPl1yrfR3DdU4n3Yn035W/Dl7DoMOz9gaL4BXzHWwe/ZHywuvozJCC0rsHvr73QfjaLpGHBMuUutv+GFoCu7RS0687B71nf9/pL4WtQIdAFfcd6XtR+E1u3q3+ceM0wYC9tud5MZpAwazlBD8cSB2S6oa8Ze1kV6K75As/buU2tV3TvwHzXzUj8fOZw+KKy7rafgWVvIsFNlaRW+SH+puZLuX9XMDrNHiN/JmbYL8tP5YMm7cvjHQJ+WUQ8nohPPmp/Zma+c9GdfM+Se7g3YLcvL/nqkmhALxInD1r6Wnjs82Oj19bWKAo9vgfYG+z7P4pZs2bxyJEj+csvv+TS0lK+5557eMKECdzerlDcLNgb7EsNqW4UZoZQC3abjsNiu1Ky9kI46Z3gpOC6i2EgyoR9XzUcyyqFVlCBVbzr/PtWaKFY9VHO/fbrm66Oc8BKAqD+NnkWsrgmsmdVmcF9VXIKjIFi2604iKxOQ+aYEaCidHKChCoiivaNcsetQKATBp4sA6jB5NpXVeuId928VH7dKVOfGcaYzBDTwzjMnZy7234rp5j9rtFXZVbGWKiPRObRyh/J6TGYoZBm36t+HzVfIRgd7LZfK38bholMGWXGXK27BI5Y2fpdcx4UT1EJkvBcNWpFticXz5xzvzzYLKqfZA4ZVwHW5vpL1RmjYg1ZjD0h84yZw6H0yhBymbRvlmCyrkFhTVZRWPmBmnI3+x48l3UuDAOyU+YcY4YiuvFKOMtTzWZOBrGX08k+tpAbSn/NV/Jq7IEg9xH5PIqM4Zz7U6+mFQG1rb+BgTCQzMl45+acCeq/C3bDOFBVifSWQDapftPQYSgmq56Rwd+KQMr2O+z7rSMDTk5B5ZMqBLXkspPl17WA2smQDN4aOOLXnC93sKz+MSpnSl6WJDSIoJskKz2eylAWDPTWwInQvUPuZBB0O6q56MnFc8ucif52GOMq53/PLhjHskBNxI/vR3zqdyrL5hcIdmONyYIT5W/D4ewulutSomrVdDrb9Lz8J+SBZmaMVVCVzZmQ6IApeREB36qP7LLAU4kEi7oZ0BesSQOta9RJOMzR3sZwjEgoqUUQUSWzi19AYpIrT+3wZo5VywnomjlH5rup+hhjHIg8CnSodUZmBHiWHo97yrKk+xsgFzPvVDunVejchkCeLDFODyM4nUyfzL5HTg2Y9ziqr1Q6LTMy9WUVmtWfQl+eOXJwVXb99bF1YA2QirMz8071s5W8LB9TyYvM6UPw/dyH5d+VBQ4EIv3qpBRdQ8CkaZH9uQQtbuF0XLPKgsaF6gAPM+Ra6WtySkzhuM25X603+FuRea6ymQId6nfZthbJPDIIHbzyA7s+0rIKQYFFhztXDaoQ6YdMnpEmv77gINMhnW6/1lfFnPcP1jq2Odu2/lb5O/W3QpbJ1oAWQgAm/0n7tcoPYfemkzxBrfh57A1vjfycc+XBHlJVZgW64GDe8YD9mqCAnDNebjfrWqzKxIqMG2PJGzJdumkx9MNAp/18FRVuXdn29yWqPLbfIR/PnoIssWrWKMjGBQfDTpCh5gtU61gpBZNBD2O9OyXFuQqQOCwLqi07UZ5MFeqFYz/e2SwSqJwC2wIdW2LVXDK5KxIvnbD5WnXrjkg/bBprpTcz1tbSE/DbsiTdnlwk2ln9MloQtmXx887PpYKgysy8S27bMkMnkQW5u3NwhmX+LnU2CWbsBXFOWeen5GWzKqzFvgb0CH5bsAFY92TUPlQECjf+Qs64xUn8eWVvwE+k0kUDXUhYapgHG9eq/7RvxF6qm4nns1ZFR3w4X78LavrO7erEtIp3oWN5ylH1t/CQxOsLDkbFtAyF01OnM2WOVeWK4LfMDuzYDD27YR4C9fHr09BxZguKeFn1Xf0s7NlV5+BvrFW3lR/g+1m/h+8vXjb727Dmrf4YZugJM4fDdpAF3dIJ7ZDCHnVQbvMvUWCgQu7DasrfgWDOeDVLWlc2gnq13yZ+nvV7Z3Yqw8CZoKpmLHvdtJm22a8JOeOUFBXsZl58NHPBU3z8scdymknV+S0RryXib4j4ArOi7+QTFVWiFuS25jJNJx4yfYj0+pkfn8k0nXhphcJP+j3A3mDf/2G88847fOihh/KIESP4rLPO4qwsiXKhwN5gX2qQbhR/K7KgVQ6TpSeoS9qZoXyrerL018coC9dfmvoDC0VDJXTXXqTOokmGsBcZpTJ6R2YYgLXfyLMg1/0UWf4yhHoxZpXznzm5gpcMwrkn48EOexBwsRonviYY8ionU+NCKFK9JXKnUbJM5WiViCTLtbcUSpZqnkRwVWZUNy2CApR1tzr4JSpgUoXI0F50eOrf3R0EurC+VJWwfdXIZpKtsaqPQPnmZLCIqjBZ1apw7MqqUpljPQHTCdlPMugRjEGVmaeCrE+FQMdmOMpU6MrCd1U9hwqekiqMmqZx43pTyZZRMDJDaUwnu1He3xh7F6vOUgcBVGicj6Bt21q5g19kSKvW9XcFsT7K37bvdUGPlk7qvqR6BHOlqqrIvgcZ8VaEemEIyZwfzHEBopfsTplgD84a2ZrWNbP/qg9yQjgcI/2gqC7+jzPdUagXjpHBOiGYEQRSOZW/K5S+isBq9h/l77R5mZnRee7g7l/wNPOMoWq6uoybYPDLHKVhrzNlb89OfN/a862/AfO14WfyoNpAZL2nUk0rKaiHZOezoC5V9aFjRlBkwxX2z6O9VBUZqnn/GFwfrYGgcyt+2wzGpGQQxVebZd+TeG3thfK+ncwxeazqwbXxSmdKLS1o9v4ZIk8qKH0Fuqcs8YgZTvjFR5p9mxVBAoH49Rmlq1Scny0roavmPWYPvkf8CCKp6Hr3BAqfhRyUyStdMwP0lnmN7z3lhLUXoHJiT2LzLyEjBouwJ7b+rNUF3Tnq6nPDwLtYf7l6nYU9oMMrecl+rX0D9rGqKkv0rZXRTzHH0fJZGDQ8FdBDVEwlm69Fpa0ThdVgYdJZYl++mvzvrVh1lpnE9az9mh7BOZBOdno00du1faPcvujchiBmywq1fBQ9I2XoyDCrB663X2tbx5xOrGf+ngt2brHLPH8bxqPq5ZYM3joknVgT6xrmwf7Mf1LOGjIjLTYXsupQJx3d0KEbyui1mU1a+41q+3HhNHmyFTNkRebvYkETGRoX4rrVweophz0m7VMYwZhkAdl4hFywJVXOfBXa1sLZ3b4RAZ14GZz3GM7uVefAppeh5gucJ/F6Zed2JEV2ZWFfymiuVe8iHuJ9WvWdhnn4zY4M2Evx+yPa4y1OJ+gtw75VJQlb4W+DniRjcan+HHaNu1Ddb1WFiB/yTWWXiCSyLTfYbUBvDap8Zo60rwUtiESI7XdgHagCUVUfI1BlPQNdBfhcxQbgKcfZKdPPWlbGJZEpApxOMAwE3lW+LtU6CfXi83n7471ZK4laVkKn2PU3JLVYUfYGPq94F/py3DvRc/7CHStvTa7neWtS7xle+loswSL3UbuM08OQfzn3q22r9k0xFhjrd3vLoMM5tY6pm+l8jhX/x04X7C5U7x/hy8z6g3M/XhniqzpT6afMHKvIX38p9E+ZfdG4MNbjumWFOvms5CXoPPF2e6AT+sRgzvxAh7Nfo/h554RRJ2gB6HBNS9R05szwC1krdds3wp4eLLV3MnhrkFwpO0PDXqxP1TMXPYfgfRzWr1/PRx1xBI8j4jFEPI6IjzriCF6/PrHo4sMdH/IV317BX+R9Yf9ZLcwlnSWc2ypPsj7v8/OYphPPK1FUvX8PsDfY9z3F3mBfapA2t3QXxjl3JXQXhc/CmAv14mCQZpY/DceQ9Vr8vWUGZNUncFCpnMb+Vihvqmy9qk+QDSwLmogqknSyZ7n2N8LZk6oSLFD5IbKtBkOVKDjtcx8e/O8zQ4EYYJNSZo4FedJJ7vxYfioyVVRw5cH5qlLsxL1Vh/PKM9WOJj0CI0jmHBYNgmX31UKgmNrwczntiRaE8q2qJit5yaRuUPQu+K6Q9QezqsqhYqvmC2TSWtE4H0q2iuok7IEDommJen3MGAonqQyGjn3eMM9OB5YMgU7sWW8NjFAr9ULIhcBvqkFCZuzxsDfl5tiGYbDWuISNTdfKHU0tK7CX118u4eF3w4Gx5Fg4UKxKcX89lHRVAL3iPQSvZNmxAnMmqLPatAAMeqdA1WDgrUGAQOZEjfTjnRROl9OHlr0OOaEKGDfMxZ5VGSuGAcefbM/l3I/9KCoIBgrhhCx4Cg4hEdwKdscyLNs3wdgqek4dZF9wkJous20dZKQsGz7swfuUOe6YcUbN3jd1SjcBLSBPwBDVsFan0UAhHE4FT9mvdWSAVm/Jsc4Gl9MZ1LFFTg1j6DEHcaoZofWz8V1VhVwyOFX49uxCcH7rzfKzOeKHoS5LsmleDqdf/Sx5gGrmSFCDd++wz2VfNWSFE7V3qNfZybDjgWjgwabn9dfDASZzWIZ6oV9tvMou7yN+532sazCW+6rtskTQws+dCNm6pxHowr/W1faAS/sG/KYqa3fbbWqHkKBjSic1lXbJi+o1tDsQtLdzJ6Um8zu3mpnOb+Lst55HuQ+rqxEFmpfjHrK1m3mnujJUsH7sDjwVSOJJpXrZW2dm2v91cAwZ9bPMwNUjcjmS9w91FYC/HZR6ZW+qdaRAJ84461njKUcAMfdRJIjI0LkdznrZGnAXqqlJm5bgjCp7XX4W+ZoR0FDR6ZuBM6sjK4pgN57bGlQOe3Bv1TzMPwCOTFGlaUWgC+Nyqq71VMjHFPGhsimdoOdZzyNXfmw/yxzewR7ct+wN+VnUnaO2mZzgyjMd5Y/If7cnF7qhtTcdM2TrmvPU1ZvJsOMBrC1VMFDXTB1MsY5cefi+ip6yc7s6Ma+3FPJLlpRX/J/YXFid9GEv9mTPTpzB1rEX/MtMcE2R4aArC3tVpqfUz8Y9G+YgCCCbC1ce9tT8KfIg08ofxZKHnORQ05LEhENvrZnUFkd32leNsyyd5IHHNT9Jjf5e9c6aFuGsUSXSRHuuK/wuHZtNdg3JfhWVl076e9mbZlKVJQDvKsC6EUncMn+Arpk9diUyO+PX+J5Kv9fDsNcGy1jCDDklS6zr2QlfhFXn0EKYZ1VSucDaC53pZgWVdtw7N5aeyMbSkyDzCp5WU7IuOxnVUTKE3JCt1rkM9ZqMDQ72tqrPLzMYk8Ret+rLvubYNVkCh8D6y+3U3gNFw1x7UC3Yg/Fuv8Pui3IXYg8Hu7Hfre8y7MFZX/UJ9pVTIqMM/lbngNfuIuzBOaiiTu3KljNoMOOd9OySnxeVH8CelvloDQN2RdFz8vsK+ZNO8orqnD+pbQOhhztRJPeWgunHyr4m5tIaZBZYfoqcNWag2PgL9boUgd2Mm/DfER/sCHMcdy26i2k68WNrFb0VHXDzvJt5wosTOKd5gLTR/4WQxjAcsDfY91+CvcG+1GAYBvv9/sSNogXhZMq5T23kMUOoppPcQZx5p5zKMxlHvcgaWThtz/dUEw5gmZGX95ja+GPG4eGUycfMPH+qvIk2M5TBuRPlwUCRMTXYPg+DRV8VDt+VZ8gpzDq2QFEIdMgVaS0A47V7h/z+gS5nBSj7nsEdsCEXnM0yBV4L4l2quNkFL306OTcM/59G4wI4QGaPlVcY+JqcHWi9pTBsZDSevhbTWPyL+vtaMOWgWRQV72Jdu4vsVCorz0CVpKsAVBPxfYJqvgQVhFMj8OZl8ozfxgUwxp0qblpWSBVdwzDY7/Ox4W+XBxlrvgKNhBMVnAqC2mpGmnpt+9tgTKkCE+VvyasAmhZBLqbT4HohfVco+BcClDsflBuworerimbHqgxbEeqFYaLqlVrwNAzX+EC5vw3v37p2RHXG3EmxXn/ppF77GTfKKVbinVQyquHObWYl7Ovy+87eVx2Qa5gTu7fVGMv8HeTEjDS5k6e3BIZ0oGtwAWGn5I/qz5zlZu23WAeqKifR9yCdJHROgZjcTiVhRWDX3/F9WfVe02IELVQByuIX8M5ViPbxksiKhYfCuaeCGHP+E/Zr8SwH1rXdMC/2rmQ6g6DXHmAFlU3PE9RAqr57ouJ6xjC7nHLlMS85Rh1MFjqcyvm77bdyeiHDgCHsRI9W8iKeLVU0zMHZ2rQEz+9U1VP5ASpoxLgjfuhKLavUZ3DGr0GxZ0XLKgTbFh8pDxQFOiGLVFV45W+hKkImn6o+go5d/ZnaiR+tLrXIkRU/dKZh8lTE1ojMETojDWtPRfEY9kJOpJocFI/cR+V6XMMcOFes9w50YW1ZE4oEPJXQG5wSqqI2TYq6UDLZyJz8HFQhXhbIgvszR5jJk4OgIxPVuOmDcFkEumCzOLGReOvkNlX1Z9Bxqj+T70VB2etEy+UEVwEbxS9woLPY7gSK+GGrqcYt6PXmTpRXe5S+At11dxIzVVh/GZIgZdAC0K9Uwa229bDZVYmKy04C3V2qWHx08v7ACw8BhZ4M3hoks8pYNOpn4d6d2+z08X3VZvD+IZyz1mBXoAtyOdW9Gg0ASarIojS+SZKGmhbjbJRVLeb9Q82SEg9vrcmcI0kUEhCVtekkTwDceCWCPsyxylKnHprtGyEnrE7voufMwPvv5PMU9mJc6eRMOSxDpB9r1knutq2Dvm5Nbqr+NKZja0G5buhUxd6y0mQrSaFNTSoIdMnlSNs66EGpBoBShSXB29C1mJ6XcVMiE1DTYug1/nb0vLcmQ4dcSFhacTrG07Y29efpyICuK9O1C59B0MyVZ5edIiFZxUSQcz8SIVwF9iCRqwCBZpVuFvbi/rNGIYFtoIjXYzf/MjERpycXcslVgGdLJ3tCVdMiJJNm/g7BpD1dkVb4rHNgVARQrT02vbXwZ84cAXlhRW9JLPiezOcpw8yRpg0pSe5oWoxzoug5uR6UcRPelQwhF57dUwk2Fqvc3PEAAo3phPUdDxEo3HCF3Ne56hx1OyJmnDE1XzonooZ6kZhhDUYaBvy/XZk4L9rWJvjjRN+9a2dea7vl+tr1fN+y+/jTXXJbyxP08LbGbQMOdP03QhrDcMDeYN9/CfYG+1LDoHr2CdR8AQFpVaS23rx7NIgNc6CkqDJSnNCyClRzsmbNTujYAqfushPlFWHCMbzgYHVQrq9K7myM+GOOE1ngItABBWYgPaf+p5Fzn9rJIHoRynpIJEPndhjSKuVu683IqlQ5RpoWw9CxOrEMA+9RZQAGe2A0lr6mpipomLt7fOODRcMcZEzJKvTmToLBrEL7JrmSw4zsxaZFUHJkBm7PTii2qmpaXwuCaqostPkHwPBNJzttROlraidU3QxUCbWskM+XaBS+5Bj7tcYFcDCUvqKuolM4czRN4/xdmWZwYQ/3F+irgoK84EC1Ii4ofmT7vXEhqlBkyH/CVFh/5lwZOBgYOpwGcyYMvmebCsFuGOSyfl2BTmSZzhmvpsNRoXM7c/oQPPOyk5L3UWSOVQWLfzVf4cywOnf0CNaW6gyq/QaGg4oa0tcCo0fVr6a/wTRYJI47EShMJ3vCS8G/IAe23z44R6igrFRR31a8KzeGmOFsCbnU2eotqzCPKuqYtReog4l6BIGnpsX273nKsdfTSV5ZygynYebv5JmbwhhfdpL8u74WGGEqJ6moyJE5AHf9DUEJp/Om7A01TW3rGgQbrVnUgQ4EileeIV+DG6/CmFTVWdWfw2ljGsQ2PW/REc5UroJqyio/PRW49/JT7fs50g89quZLOAZVVa0qiP53s8fIHYM9O83z4Fj59/vrB0Z3tv4y5+xw0dNvIMajFpRXnQhsvdlceycnJroIOLFNJINILEkntb4b6IKMsQZQxdha18idegVPmwlo2+TnQdiLNSTbU/7WGIXZYCuRmKFzW2ktRfWvKgjJDJ1QpqMLh8+mq9W9jVtWQf+XOYcNA0kawrme8JvF2Oeyyih/OzLsc+5H9ZJ1r+thfKZab6J1Qd7jcjkjEk9UNI2Gju/J5FekH0l3qopCJ9R+q7ZpQi445ZadNDhbsHO7cxByyTHQO2VyQgsy9zeyFnCpbVt/GwLw1gpgZuj+Ox+ETSajLZwxVN6Tr3tHtKLQRn/MDJm6O85/lfNWPLN4X7LqZWY4WGW6eONCOPld+fJEiy2/Uld0ta2FE37dxfLq0G23yhNdBoJIP+7pFLxixnNn35OoE2fcBF2v+Hl7lUfndthaXdkIEqiqPb4LeCqgYw2UOSbrbnM93ZtcLxcJOmsvkMu3LTeoqwD1MOZ+/gFqOkNvLfSRyvfl1yvew1qRJZn1NyI4IbNbGheoKbQDHfiOSrf3VCAYPf8Ade/Fmi/kZ2+wG/4Hp760ySDohK2JM07sQ9Hn+gpzYk30Kn8HslwVvGrfgKQWSZDY0Z8nznRV65RAB1guVv8Ye9aapBH24rOWFbB9ZEkaOx9U6+i7g9JXobfJbOZoVWgGdHFrhfCSY6F/NS6w6+F91epge18V1ofM7yHO5B1/gW/ElWffn1Ufwx7adI2cDcVTjuD7yjPkCWy9JRj35usQqLSuh82/hB5b+w307Xj9IeKDXlD+jn19icB+9h/lMnvWaOyp4ufVa6X6M7UNqQVwD1nF/kCx/Afyc4451orC2nZh3mTIPi1k19/6qjH/ToUApa9Bv5Pple0bkydFOSV2MscYAQqnQ2cyzzUR7Ltprj3Z+aGVD0Wv74Uce2k8v6fYG+xLDdKNokegnKkysQqeUvcOYMZhqyrFjvgRmNh0tVppdELTYnUAgDnmXEmV/58Zh+n8qXIKs64sGBSrzpEfBoXT1TROogeJqh8CMwz55T9wzmr//4G6mXAEyKoa8p90djTlPgIaPBkVWds6KOiqLGxxsMpoK4M9UKZnDFU7UbPvwcGfKkS/oln7pP7d7xKiJ+PWm+3X3IVwqrmL1BmMwmiUGU2iL5WKglb0r0snuRHZswtOw8LpqVenlr9lKuqSCr2cP6ECQZVh2LHFDHBKgvPMcDpLxqRpGlduNTNGZVVbzFiX6y+zVw+E+2Awb70ZxlqqGaPeOgQYs+6WO+7mTIDjRIZIPwzU7yJLVVQ2LzzU7hQUgat0GjxVnaqpfKALhpCqr0r5O3BQRfrtxpSnAsayqrpEoH52zNjvr4+NUxYYF0hWITwQzJ0kz5z8LhHxYc26i+R7XRj8Tr3TnNC4EMkdqiSMNedDrstQ+616rzLjnh2b7cF7T0WsEnLX3+3fK3wW57Lq7PS3w2BWVVkK41FWLdu0CMa4k1xTOX+bl8L5pQpA9dcPnsY1GYQDJPteZk7RIOpvjFW+WCs6NvwMwTgZBEW7Krlj263IJlYFNcJ9sXNG5uAVQQ2VbrfmPFCrbb8D54YKTUsSz4XObXBMCGdhxJeo+wa7ofNJKcTCqLZWVc33lmJNqypF9AicQE50Th1bEIyWvbfeUjgprDpWVybet6q/nICq/7YrD+enYzXGevyzQlRGVX6QegAz+vsFqHCyZpZXvAcZ47Rv1l8uX6O9JTEWDVmCmr8Ne1bloM19lB2rYdvWISnBWiUsdMqCp+Xf235HLJg3mOo8ZpNeV0bxWQw9Jp3UtKtOEEkFMkd6bynW/eKj7fNU8rJp89w5uL5BzLArrBXgAoL2XsY8YwYK9ZJX5TJP17CnB5tc2bYOOoXViSqCLcKhZ8WsfWLVBbKet0X/Rh8vFZVm/hOYS1lgpGMzkh9ViXOF09WBd7En0kld3dCRgbPMKk96dqJ6RZW0ueAgPLMTIn6ckQNJ1kr4Xj/2qnC8x6/BBQfHKpRUCTGFz5prOz/2WbAHY/W3I+lJFtBwF2HdOVU+7XwIiQpWnUT00JTZzL0lSByJr6zr3I6g9EArbER/bZnDeeMvktNnu4sG3h8wHvFU19Z9pYVi1cmyBLV0gh7vBE8FzmnrOZrMBmSGDi7rMxzvwN/5V+ffVyHYg8CHlf60vxH+ra2/sTMf6VqM6jWd7MHG4v9Ap2iYI/dviGrHkpds61PrzuXKrZ/K9byw10zWG6T9KJI7tt0KfVymh80cIae+FYj04xlU+lD1p/KEP4Fwn90X1d8IPSTklusz+U/KfXrMMeahxoWpVbFF+tFfUMyhSmY7YdZoBIkXHQ593Yqar0wd/Er8U+liIjktHqLiVFZh3JML/VeV5FD8H+c+q2LP7U5f5mRYdqJdR4v0Y6+I+bbqgJF+52p7Q4fOokpCrJ+Ns1nmr+yrwnzn/UP+XXch9kPrGjmbhRbCmdK40CZfRTDvj0vtc3X97Ov3BvuSYG+w73uKvcG+1CDdKN7amENQZnyuvUDt0BPo3IpeYlbEZzLL+MR7S6G8q6hhujKdD5rMu3BdVrUV7oNhve5idTXQYLHlV/hd2XMbuklpqehLpoVAM7bosMFTKQ4GrWtMx/c0ee+BjVc6z3PVx1CkVRVhYp5lhi0zModUDndfM4xTWRBIBBlVGVndO+D8XXG6/LoWwDzJKlQE57zKMfxdoS4dRrGqUogZRvWK0yTfnWEakJL9xoz1VzcDe1nmEPe3qvcjMxw96y9DgF3VFzDY41wJEeiAk9W6Vto3qvsnJkN/IwyeFB2KmqZxwwYzeCqjnfS1gI43nRL7ZzBD4RZVjOmU+n6t/BDfU1EdioqInQ+ldt/dRd1MGKaybNJgD9ZGOqGPhBWufOybpSfIHYrBbtxfRneZDKIiOp1S60EX7IZDpvYbPPusUfhcDyPLL9CJ/x/2qqs4urIQ1FX9bl8VHHwqQy/QqQ6KdWUNLiElGUQfJKvzKx49u9R9D8IeOGpl50HIDRmUTmo51bRIXsE5EIhEHlVfNBV2Pgjn3mCdtytOwxkoy/CunwXHRs1Xavm2/Xa5s6nsdbPSQpGQMneiOrDPnJzy3AmRfvy2WYVk0/OCPdiPsnn0t4FKZ95ku1OyYY76bAx24z2oAkwlL8ERtPNBdf8eTyWeS6a/eeuwLlXVMdWf4jey77XTqftbsb5kgZzab5yDiJUfxPaUiu5VJLzsaQgHm1NGsQyi/1z155hPa6JET65J075cXb3sb8XfyYJIPblqOsH62ahw2p0KcVce9GFZ5ZUKoV4ElzJuVCcS6mGcV7J1IM5elfxx6nPEjCDBqrPt71MLwgmvOguqPsE+XXyU8/0HA7EOZo6QJ2rpGt6bykmWfY8pR+6QX3cXQz5aA6AdWxBwUNkGG35mVm4ejrWSKvQI5IiMcrC/PloJXr493e4EEhn4sgSNeITc8h5kXdmms91S2RDsga2jsvNy7kdgdPa+8gonESxU0RTPHDH4ZKtlJ6srog09tndUjCZCDqXKvsCMNZD/pPwcbN+ESqJ0gg4pgxZEIo41yCnO1/aN2FvWNayHIZdVfeii50ncfDUvNRN/voIzfsPP7d8TfSw3/1LuRxAJjCvPsNuY/nZUZgU6sR+XnRizQdyFSEKTya6WFfLEisoPE+WVFoJ+q5I1lR+ayRIWh7e/DYFFFdtNoAtrVxbEZsa5u+xE3Neqs3RujQX+ZQxBdTPgRN96s90O1DXswYwbcQ/r+9QjpvxS2GEFT6H6SUYp7m9D8HHBwc7BwsFCBKGt1VOGDn1TBKFVOt6KH6J63gothHey8SrYovG3XncJazPHQOZ1ZNirWgUy7wQTRipw5SHhxanStuQlJOnI9Mp4OmrruRD2QBdOJ3W/emZUPC6cpr4uKlQHiqpPMPerzsE6UKH4eXsArG4m9Myc++GLSrVfeOGzzgxSgU7IPKeEK2bIFGsVph7Bdzu37/m+gCJJQsUA07JCXanbvDy2BmQ+n/aNav1LMFmoWBmYMdbOban3jK75avfYvIT/V7WXZ45UUsg+vfFpPvSNQ7m1z/7M1826bm+wLwn2Bvu+p9gb7EsNmqZxYWFh4kYJ9iDTKZ1gNFkheMKLX5BXGjHDqSPrYRLqRYCnabE8k6/4hVjFjb8t9QHpGv7JhG58vzYrPUvVx1CudqfHyNyJaqpFTwWMWllgQjiTBtMjbHcgKChUGXf5T6r7aA0EJS86ZwgtPERNPeSE1tVYH6oeXqp+VsxY22LMMrrEvmo4a5IpWHsaIjMy49epV3uE3JjL5uVy4zPkxr1V1biGDsN5MLS5zDHKJZmxXjgda0goedWfxa65i5FVPJisOG8tsvwHkYWqaRoX5Wex5m2UO7mEUVI3U32TnlwEL60KfvMyUHhuvVnu/PVUQKl0muMlx8opMAIdkFMbfu7cc+p/GiL4nk4wBKwQ/cVUdDlaCHJAVvnga8Ha2X6Huj9j2zoEGeMNfl9TLGjqrYlVJuz8K7Jqqz+P0aI6Kemz9pE7fNo3xqglpVnapajUVgUKZ43Gd2UBUHchsrALnrafkQ1zkQSR+Tu5U6jkRdDHFk5PTn8lw9LjzQo6CW2XSCqofF/uyHfl452qkjuY8U5Ue2v+VFCCyajTkqH2a1TMynSKUK/Z41URfFh2IpxJKgi2ANn38/+prg42DIxFtQ4KpyOgseosewCrYzN+c9YoefCsKwvreOtv1M8tnoElep7oR6JKthGV5DIaH18zMpFViR+138CZrwoIzxypTixJBn9r6j3PmGMUPjJ9RAvh/Yv57atK3Ds9uXjeFaepKfI2XiXXdw0Da7Lsdbn80iNwZG+9RX6WZf0+FrSzwl2E+ZMFVHQNTi9RsW2999oLUEXkBCfar2SB6j2Bjgx50oGg17I6pneHVp4ZQYF5+yEYI4MWUDuVm5djn8sc8gKlryFzXNVLNdgjz7bvq44lGMkcdxt+Dge+LPgacuHsU1VNufKcg8l6GI6wPd0/fcPPsH5WnC6vlGxahDGp+n4mQ82XbCw8lKuyvrE7gQRdsGrMFe8iqCaq8Kzw1prBqxRZLAYCp4Cyu8iZcjDQib0qW1+hXlD/5T6a+jOVvIx34pR8svpcdUA4ah9Ier0J1pB1P7WvsUAH9qOQgdYqlfZNCPgOlBJTYOdDSGSRJY14a3G29ZYieVKmZxt6LEFElvyx7mIwdCRDy0okyMkSDgW23Rpbq7LewAunxWxobx2ex8nRXfRvJPBaz2bB7tK0RJ40FeyBfjZ3UvJqcSu8tQiIOMlGUQlsvbdIVsi8E+tHFlBp34RKX5n8y/8n9pMqiLknsP0OexKkFsQ7C3vUtPfJ0DDP2X/SvAw2cPzPNizkpk3TIfPWnJeYrB1yQWcLe7BfrTTuegSBlNxHkdSX6r5ihl45Z7xchxftK7beYvevhNzQKVaeKWfOalmJyvfchwcnw5hNu1yhL3fn2PWzqo9gp3VsNu3AuLMg0g89JOyJySfr2dxfD72zYR6C7LLEkd2BK8858BVvA8uw+sdq+u62dUhcku1Zf5vZdkihxwgWIJn9GV9FLrMPNl1jC2BH0ZUNZhdVkpmrAOdUOtkZASrew3M1zpfLgqXHq9s8DAStq+HHVQV8M38XYzgI98HOjXv3qn5zv1/8+73BviSQxjAcsDfY91+CvcG+PQh/u3PflU1Xy+nA6meBLkKVBe2E3jIYr+nk3OxeBXchFA5Z1kjYC6dby0q7Alb+DhSj/CfllDWCmqj4efU7yX1UXtli6LFgTnywQ6B9ExQGJyfpdwUtiPGosuWT8fDPmYB3lioCHWYFiaI/WedWZweD4I6XOT9KX3XuYaMK+Ar8/2p2K9aIrOom+17npu8tK9XrK+KHsaUyHPQw3uNgjRLxPmXBi41XQYn3NcFpFL8vnRqrM8cMwNU/tl9rWjS4KiCBkAvrWkbd19+ATGanKksV2jfBiEonNX1R8X9gTKVa+SAy6pIFIgeL2q/lwbpk6C3BfKhkSPNyOJ1lVc9aKEY5qKLCU6G/Eet6wUFmNXfc7xsG1rNV+V56XOJ6TSdkdcog6KVkSSdirxY/Lw+wO9EpMccCpLK137Qk9mxW54cIPKUT1m+qcBcjaKfK9lx5JgJQsmx4dzH6Pagqqp0oqpjRK1U8u4wubMHBcNpYEfbCKZj/pFpm73gA95U5unIfcU52SIbab2Fkygy5ZNU+vSXIiFdVGuU+Aoedde/0lsaoS2X0yluujwUDZXAXQddRJcTMHovvqwzn5mXYW7LvO/WHZcaemD1Gfe9Ah6LXVgCOMxWdNPPgqXH7GyHfvLVIPnHqf7v+cueeflbUfgPdTxaQExSNqgC6cMioko/CHrwv2doTVeLppE7+aF6KvWGdx8aF8ucVEFVGGTfKqzGEc3jN+fLv95ahKsGJKjkZVpxur2zo2IxkrnSyV90bOt6Xqkqt+jPo+C0r1I63HX9W72c9jHmun22/1rQEzAeySgo9Arm09Rbs6VQrkEWSRTrJn1skaKjmwgn9jc6BLye0rUeCpJPDMesP8h6HyVDzBZ5pRpr8evHzg6c8ZcaeLX5efi37XvSu2nqzPAly9lg5HaJhwJba+eDgKuCYkSCmep/9DaB0tPa7ZUZQVMyjiuli0eGoNrPC3wrnrorWTfU98d28x/G7smBf53bYJr1l8iSNsAdrUFaNE3JD1pe8DDlm3e/Rv3PB7o934rZvgKwIe+z3LnkJwZlw3+BtPW8NEu9kARFV0F4glYp90bc8neTJgz25sb71wp6q/ECu3zUtUtuAPbkxR7zqbBQ9p2Trjxm6TvtG+Vmlh+XVlwKGLtfXu7KRxKXqUR7xxSh9VcnSwe7vri+jocvlp0iycWKs8VRinVrXp6vAmUI71Iv3otLtBFpXJ/rgxPpQ+dUEG0Q6IWCn0t/6qrEfZfacp0JtN/vb8D5SrYBjjlFoy/Zb/Li6d9iDTPlPwrZcf3lq/UOblyHp0pWPtR2flBxlIfka9kHOffa1LZ5r0RGmbSKxEwNd+DtZYEzXcM/ar2F/WBONBH1noMOe8GUY0C2z77HbYzVfgEVABNGt2HQ1nnnhIeoEcKc9JYoXZG2LdA36kEr+dWXj3Va+r04OaJiD+1v1sEWHg8a/7HW7D7dxIfQj1XP17ESyeu4jcns/vlBEhfWXq5NamJEoOGc8dOf1l6kr2ePQ7m3ni7+6mGcWfQf+nu8p9gb7/kuwN9iXGgzDYI/Ho8wskMJdBIcPbmC/vvk6tZEm0Fuipnjx1sCZLXPc9ZaCYkpFdSIapA/G2BL0pdYGv8yxfjHpJC9Rd+XLK32YocSmE5wXqgzb1tXOTq7/XxDOKpmDQhy6KmoPVx4cFbIgQNRhqOgjIt61ynEsKhNUmb9Ni1PjYI/HspPgQPmfRqAD60jm0BH0fDJnkhaCklT8grqhcu6j2DeygF406KZQ7FpWmBl5iqy43EdjWYSpBN/87fjuuovlRtHKM00nvSQAEOhEFdPCaeo1ooBhGOxt2aFW/JjVASxdwxyoHA8CLSvVmd+Zd0HpkwX7Vpyurpjo2AwH12AqnpMhvn+FFRE/HNUZN6b8rqPY+AtkVloh+nSpmnn729Q0TKIqYdttkCeqXhRhb8z5EV/JkCyTb8YwdYVwyIVzyokyZkYazsJUoQXQD6biPbuj3deEtRnqHVymaMG/TCfqIPst+NvguJOt3bAHRqLKWSQSZnLuTy1r2NccmzNZVm/PTjjyA11yfUQY5qvOlt+/cSH2o2xte2sgH1UVFaFeOIVlZ7e/1aQYU1CSJ0PED51Edn52bIGOpNLbBEOCSXNn0/PqZ6lpeCL9qACQBaLzHkdQoyPDXp3sa0LwU2Xs1s1EZa1qz0R8cVm/kvUhsnZVQe68fyD7un6Wsy619eZEqqhQL5xEYq/VzbBT9DnBqVKx8n3IkLZ18gr2UC8CTFUfpd6v2deC786daA8UBntw/sgCdfFYdQ4ST6yo/RYVI07VwdvvkAftS1/Bubz0hMHrtOE+BLSturhIeNn5kHpfZd0t79fcMDdWjS0LvCaDcHargleecuw7a9Ket0Ytu5ix3rbfgfvL1r2/FQEgp9YDFe/K+1/rYTiZtt2mrox3glMCUMsqBN+z77HTiDUtwrqaM14dKEqGbbfKq5mYIYNU9oO3lnnT1WzUzUzdth0Isv4Q6wEcj3gdSrZGsu/F501L5HZg+yZU06qc+K58OGlllIPCFlt6vJw1oq8KtoHsXYjA6txJ8iQybw2CLbKzzJVv2oCKNbLihwhQfpcQva3ie30tOhxVgemEaiAZRJAyVXaQcB+SUZ0SoZsW4wyxvu8oVbDENg33IcgTn+wp3q+qp3U8vLUx/4fMJk9GgciMQLWqx2bYCztTtr5Ee5V0kuvhbetxzVKJxszQGQfbG7tnZ0y2qpJVV52DhEArXAVY83MnDZ7VhhkBaWtPuIZ5OPebl8n3eu03YJSQJagV/dtMZsmX/17LKmUSmKM/z5WPe6vYq/QIAtjCt2f7fgH0q7UX4fdl9u28/XavT7lKVgc68NyyOW5bD12hZyfWt/XMEAmMMuz4s3OPQRV6S2PrXVUF2VeNfd62HvacLVh3fSwxWHaeCBpYUSln1Q/b1iGhb+Mv7EUXTnZ1XTpYKlS+s6XHO9utWhDnr8yeZ8Y4qz8fPBuAsAO23Sa/3rkN787KslH1kZzWO3pfPxLfVAkPTkwWQvdzCvbNnejss3Tlw+74Ltp3fI+Ragxjb7DvvwR7g32pQcp3G+pFRU7eY/KA3KZrkPmoQn99LNvMikAHnKzplLxnggyigkkldEXlg0qhCXRAuZcpDZF+CGrVs/vbcUDKAl/bbjODYpIKOEOHk17VU415YAHSPY3+RiipqkqPunQ4VVSZOEXPOSulYp5kgZxAJzJwVAERQSMrcxbVz0YGT94/1NlFq89VKyNOEIbj0hNS/+7uoL8BSoCTQ2fpcfIARdtaM6PzQ/V3d/0NWaIyJ70wxFT9QEQV3ZzxjkMYFJyqGQNdzhmZHRnoceFEdSKBpmlct8n83fpZ8j8SFI/WSiItkJj9mCoa56OCSdVLYevNoINTVbx+VxDZeDJjPOyJC5CdLP9+qBeOQZUsyH9CTpWoazBaVFmkwrDcfru954sexn6RyWM9jPXhqTSdjuPwua8JMk31/uOR+Ts4P5Jl0KpQ8FTKa3O3UfY6KKrWnC+nwHMXweEtC6Az46yK9KsdJzn3q7NUmWE4b7l+UI/Obesh36zVwVoQ+ybvH3I5Ic4KVfJQMux8CN+XOTmSVSsKw1Tm3BXVUapebttuS72adaAQzjczQSOlvgai+l1WSZ7zJzhSZeeIrxlnlCpQlnmnmSyVLe+ZZhioBFv3U3miV8tKBPpU+lnGr01asyT9V6wQ55+MRpgZ8zcjDckysuBX83J1D8zdhb8Nusj8Kal9b6D9LzNuSr13TzLseACMD7tDndZfbwbl/p74ub8dcyCrENEjcPitvUAdJAp2IzglYx3pyMD6Ub0zkXykkn0q+u9gN/a5yskjaK5njmRefqr8bwaL5mWxs1uVIOkEMWarM1ugK8t0Vlr2fNXH0I9UDr/6WdBXS18ZnK7TV20mgkmqtd2FzOnExpJjuXDHervMC3ZDVgy2t2xvCXo1WYOr/fU4H8vfkp9Fiw6LzUX3Dvv1wmdMZ6NCvs0eI68oZMZ+CHSoEwacZKKnHGerU2BM1S9JC0LHUgXeGxdiP/nb5LZx2IM9vuYn8jPBCW1rYbNUf453Fx+0a5iLz7N+r2YYqkvHeRKfPNeVDTlfPwtBIFkyoFjzqsqM/sZYIMdqN3fn4Dc7NuO9V7wXSwLx1tllXrgP5541mC6gBRJloa8F8kSm33ZlYm3WzUw9UUwLQLdsWyu/LuylnQ/ax9zfAL1uzni5jrXuEuzHjgx1v8tdf0fg2Ap/GxKxVX6Ezq1gVpBRGfaWgppUPPeeRs2XZhKthKp/5ki8j4p37faLpwIJAat/LA9Eeesgb1ecZqvm1nMf48Dcw1gLKlhWBFpWpd5juHMbZNCOP2NsskSibbfimWXrK+RGsmHGTfYx61rs/HRiLNn1d+fqvOpP7XJZC6ntmc3XQb9a8UNnn0+g0550t+23MT/AoFomnIAk6ZYV8v3dlYnfaFmJs0UVzKj6xJ5IJFgyUulzP1AIn43qLHKCvw0B57zH1YUPugad3Bp068pEYPO78I0YOp7NW6umaneCsItV7DE7/pJaAuFeDBh7e/Z9T7E32JcapBsl2A2Hczohg8OK5mXIWO7JlTfdZsaBLmsuHOhE5sjM4fIswurPoGSrjMSIH04ZVeCsfSOyPWUOYF9LzNiyGhbJaK+SoWOLs8Mo4ocDS2bwdGRAGS5+YXC/PVjE9+yr/MB+3SmAyYy57C1VU3OIe6sU+MH2fSl7E857FV3Nup8im1jm7NYjcMwuPlIeEG5dAxrZ78p5p0L1Z6ajKF1dodSyQt7fwVMBJ6yKykuPgPKh7HX59YgfGcMqQ98wzMCbYm8ISq8tN9ivuQvxzLXfwpEWb+wZOpwFqh6byZAKHU4cNE3jssy5rOc+Jq/QEw3l119ud1jqGhxZ634qr+bw1iEIrlqbLavgeHHKsle9S2YopG3rBi+nVNACUL5lTnxDjwWjZWus/J3YXpc5BWu/wbt06tlX86XcYK/8IFbVqsraM/RYH1mBkCumfJe9DgOVGfMizixRserUP0xVnRfui1WZyCqgtIBzluy6n6oTISL9kKsyGp9wH86R7h3ySqGMG3HfeZPhMEgVooI973H7NdFHbtVZcpmuBXHGOtHUuYvgjJLtj6y7zQz9FJ/bXYgzoX1j6tS4zGbvWEXVn68ZThdVb9neEuwJ2RnpLgaFUP6Tcgfu0uNRYVnzpf25fc0IjlV+oD4/XXlqmR/xQUcz5YxNz2tZhUQJGQOCOCNXnS03fFWUWwK9pfLeP8HuGG3gYAPCA4GrwE5/274Rzm5Z9VOoF05XlcNQJLtYq0fikfk7daDGMEyHk+JcD/bgDJetIeFEUK0/w5DPQ181KizzHkNSnTVAlfMnOLWdUPkB1r4q4U+Veb0nEPHDzlBRBssQlfmKRIZkENnbsj5tzNDNqj6WX/PWIju/frY68NGzU87+oQWwV8reVNP91s2QU3kxYw+rklfcRZAjzcvkgSB/K6peVEkHPbvgzFRV+wQ6Ib9UASoVlp+CPkgqR7u3FgGXVPtXC5gOyYrtkp598ZXiMngqIB+3XC+vHOzZhURC1VypEPHjrKr5Up7EFuiAM10VqN75oLyqbyCo+QI2uawfFrOZ5OOXr69ABxysFe+qWRYqP5QHNZidkxFFEFFmk2shBAfK3sBayf+nfUwzhqXe7qNlBex0WcV8VyZ8HrXfINFMViHnb4slocmCX5uvjSW1OKH260R9Jyrz4hLfmhZh3tIJgQErnHQXGVSJUc1LETxS2SWi7/vW38h1LG8NzlhZkKe3xNk3whxLHLauT3chztZlJyNoJ0Og0+whJtE7d/zFXFuKKkw9Aj/A7thTNV/Kk8HcRTgTZDq+oJx0QrLEuW2/RZA8DnreE+yfdzRrATeCqNZeggJLj1cnbrrycaao/DpOEIlzssC9oNpPJ3tyeXwSrSyIHvbingunYZ0OBoXT1WxPOx+096/r2YUz10we4Yyb5N8tfwsBysG2QfmuEOiC7Spr7cIMf6dKb/DWQD7JdL+WVdCxZPYnM+x11bqNl/cyutesu0F1LjuHmpehbYFTAm33DqwPaxK3pxxzqaIDXncx/JGDxc4HMSZVhbu14jX/ycEnG+1FAvYG+76n2BvsSw2OG6XsDWde4bUXwqloRbgPWSUqo9kJ8Y4VlSPLCVrAbIosGY/IWJ891q5slL6qpiZghhGfcz8US5XTZvYYBIpkaFwQC+ZYIZp9/0+Xd4f7cChm/V5+CAY6EahpWy8PqgS7YYClmiEmkP+kPMi4u1h+qppyMp6XPtVG498l+qpi1ESyAFRXpvN7blsL5V1m+OoR01C7Zc89bzzi+6BZnYqiwXzdTDgs4995lHtdUV3X34i1Kas2E8H1QTg/NE3jgl3bWWtdJ1c4e3ahMtSpEleF9g2xd7H1ZvnfdGU60ww3L5MHQ0Sz+nQa3LMlg1NA1wkNc+FwKH5B7pwtfgEyV9WTNNRrOml/L7+uR+TVZloQjpH1l+H78c5MLYjzy+qMWXVW7B0KehSV048ZGZMypbz8LXxv7iT5XAnniGpMIstf5iQQzrF0slc8Zt4Vt74k+9ka9EwVc8bh3g1z7ddaVsHJqaJ/TNZHU1BYpZO8z+usUc6BVydk34P7yhztdf+PvesOj6O6vmdXvctFcu+9926MC8WmmG5TAgFCCZCEmkBC7zVAIIQSQgkY44Z77733KktWl9V7l3Z3fn+cHXY1c99KWkkG56fzffoEep7d2Z2Z9+6795xz5/L+U/UGTVvjufi+fZa6h1ttKjKdUCP1ftI0FzvYmFjJO+r6rqQit560q6MdqynOmxcqKzV07Lhd0+ZY5ELS+bVOpZCi8KGrRFRqBN1NwAiHgwUXlSuDpvGe96ZXV/ZuzlEpS3h/Su+vY899ZvtenaylSoDte9iUeNM0jXHjhmnOAvrT5nF97psD2R4+daU6Ma2vnxuv8NxHeo7FvM7t/4PnpFneYarz5kBO6qyb6EzgKwoeZencOzQkxtowVdM2XVnzb7ZyMvt/6mC+v2zltHdXJf/zDpNVrrovNY3zj6oPpk56keaYn+N7D/eV7jpSX0KCuzpPKgjrPWslS9XaUBhT+xqoQkkyk3eqPqqaxrVM6qWVvZcxXPYeOQbTVTGqGKok0fM8URSn2ZKXaMcObDPvbR0Oxndrx8vH6nOrKsF78AmSYiQ47E4ViIKgURt23aW2lq/I4T5V2qNWFdHyduUQdRJ2UbRcMKoNK/qrVS/2alffUdXet+A0984SgTdnPwkJkpWmrprfeAVfu7HaXPxcAPKykKxpLI6tHCITShPmqltbuKMsncd7ci9ZFO26FyWrywOPud4rdYWmLYzybLWfd4SxnbEol/wTCbBLusiWgvZqV0Eu5p+1fbKaqEs8mrOPZAfj/uNny0pFrKFprp7fEvG88AznH28KV3WFNH+mLOX1UhEcmwg14rwdt9Wcv3IPMUZJWsi4Qlon3fdHnlyGVMg9yIKatE4lL+KzJ9ntO+x8bqQYSNN4T+oxkqnH4WHOLyqCbf4Jrjdz/etn2am7eiQtYCHQff7JO8x5KXW5qz2BsUCVsoz7pUVtSRpV9UmvDaVp8hp54HEKKlRKtJJkvq/RHaHgJAmwcyC7OaUud+1LvCGMrhmj3uPln+D5JM2XyUcn32bM58nSPns3r7eRYLTjNsaFS3uai7rue2ppLjnxBgnWJYly0baqmA5BUgyvI3Ge2h61soD3zJG/8T5tyF63GTXQXOz7f4rmYl/9YLPZtNOnT9f5QamBlCUya8TbTa2O5J9oTyBNmrYKbtalZt+axqSaKvDzhOJ4V08YyR/fPcmq8rr2hAUtnccKAYndxgXbm8bFTQ2dVSidm97/SupxWBvyDpPtoirW6M10Vc2eT/2dgY6nBtgS7DYWTbJ2qr/voy/IG56mRupyMqYkVty6S7iRU+H8Wtp0Sr0HHA4WbVUbppx9DO5UgXZtWDXc9WwYmypnbCKDTJoLsvcy4XL8NTk5smEqX3P1SPNYwWkGldtvqfc9YLPZtHOHltXONvUGVUW8t9eMUW/4PVn2nv2Um1fVmF5Aqk/Ps7qgqpAFDem8HA6+X0N6BcZ8Iifp9Qb380LMTbVrQ8FpHjs/gvNUXdabYy+57tU54GdSbWr3PcLNh4TM7dy0qBLH9mqqZVXWHWUZLDBJ61hRnCuBa0xo6s3gjzznHTlEL1KqWKqZ28y9EOqKgpMsoEqqQE1zzenrJsobqtTlMunE3UZWVTQ7v5bftZTU0XugSC4Fmsb1/MjfPCetVTj3NWMDleKhMo8kDVXSJ/cQN9RGApG9inPnvodkBrjek0+l1j77Ka+FkzBlivO236K2bXY4eN6i8mEPE4xbrzfP9dVlVG+c/oCsYOPx5dm8t1XPqcPhus6SBWT2Xo4t7igfn73bmUTwoKLTNK6TnlRtm6+R1xwVlnSV+95pmovxu/5S2cmiItf1metrQ6vbyc2B94SrM/+QY/iznznXc4Vtm6YxsbJmrPnveUdc/bAaEkdtvd5M2tKTVJ5iw5QlvP+M0Fnl8yOZmJGQd0RtE+VJOVieyQSq1CKgMo+fZeedLGIZ486y84zBVNZ6Ofv4Pcd9KcesukJd1fdKTzRJRUqHnUlNVYGgskC97uvvK90/JYl8zfmRtffGlaBbGs6xyOO62l8iF1WXaFraGs2Wf8q7vW32bsbKFblyIvWn9vJ9X5rqUnxJdl6FZ81zZo3zLnWSSRUKkfwTfG0prkxZ5kYQUqhhz30tk8TSVlMxc/RFee9x4g11cVO3U107TiYp7v8D50evnDvsJDvVVpRL/okxhfu+fNMMJn133mH+zGXnGaskzGWBXHJQaChs5QqV2ymu6XXtZ3X2X56fbXdkbmVx4cdAmay18fLa7aDnhZoJFjoKTrEYLRXvNc2pFleo2/OO0qpTuv9Pv69WudiruTdUxQzFCdyzLO6kXj8PPcVY3ojybMZRifMaZjl99lNzLH7iDXW+R8fRF1lEN9oZnv0X90OqOLwolvGJYMvsMZ+XvYeFJ0/Wjhum8RrGf2+OpauKncWt2+kKIc0VO273XlnvCekbGEtJcXTyIu1nIv26S0j8c8f6ycw1FZw23yOZW9U2v1XFvL+k51h3OVnWi/NPRY55jkvfyHlx6w1c/40F0PPrmGv6MZDuOEbkHaYqf34E7wcjjjzLdfDQn82kGN2dQyLT60SaA4/KSu/FnXmfpCxVr/2qZ0rTuG6uHul97Fee5Zm8FPcVx43kzU3T1UrW6jLX/kF13oefcZIUhXxS/vHaSVE/+LjuQwmpK53x0ld8j4bkU5rxM+pbw2gu9v2PoLnY1wiw27gBUPZfepZBimpzGvcfBrRSgGarcBXVvGFo64usatLdNJ1j3qgCS5IZ7MR9ZR4rz2IQtvseObF39EUGnCplzE/tqXZUYdMMFgQvJBx2BjQq1ltFLq/V6Q/lzZqecFRZW60c5EqoG1EYQys7FRN6YZRrYTSiOJ4bksWd1QmfNaOpOKsv9EW9joqJCwY9UJDUsiVJtJFQJbA0jV7nPwbJYwk/eM8M1zTOA/knGDTXVxnmqU9J3FcM3FWJ9NTlTuWLYqPoCboCT1WYOPoin1lJMbGsN22Ijz5f//ctPMN712hJpGPVMDJsLzT071JSKdltrjnX0COizlg3UU40OBzcCKlswo4+z3kie6+5MFaezXFV8kHHsVc0be+D/O+MzWR47r7HcyJb05w2w1M8/5tfGwpj+NzEfy/fu/qzLm0e64KT77AIr7LqnB/OpJKE7L1qqy9N47x+9l9mlr+tnAmIOZA3r6feZaHak5Xc2U9lC2RNYxJ0DmSiRMpSzpsq9bGn/napy6mKUN2fGZtlZUBjQE8S7r6n/sfaq5yqGkE5qlsxSWu6XrgyJlt01NbvUdNqFuGNiP+eYypV9LabOb6iP22qVMjZV9OmuzKfsYJqPi9NYxyUulK+73fcrrbAripyJoIaQOSK/VydbC9J5n1tVIdk7SLLuDYF+OJOskVoeSaLzJ6SoCWJcsJv38OuuFClOKwNtnI+d8aYOX0j53JPBJ+145msMqIix6WOl9bf0lSSO1QW7rpLgSrO+rm/p7G4kMFEkMoqU4+BlvdzrVONhYJTnOvnQG2D7QnrL1Xvp8qzGCf9YDW/tp6Q2/ewWlFtr2ISTlXcyt6rtsHWCRxSzFoU64ztPLDws3aq97e1ofgci4FG1X3hWdf8JdnF6QqQRdGyy4GuXvF0/226Ui5qFMbwWqgIUwceU6/Lek/SOVDPgfHfcV41Ek8qcpg0VilADj9NW1JPFnfZe6g2qi85uCyDJJ8Tb/A7dS+O7Ljd5ZSiIvTpRAz3YmJZBufbnH38LiRCVVEc4xHJcUTH1utoMWrE+XVOdcnn5jYE5dm0K3WfJypySD6pq6ox9wBJPpIj05HnSOr0BFV/s5+tqBXXUSfRSftmWzkLEnMgf45F0bXvK9LWME4zoiiOexYVKU/TuA4u7WH+e8Ep1/PqDWHZE+xVXGtWj5QJbIs7umIVo+IrZQn3+Qcek9eaolgqMJf3Mz932XtI4GmKArY+r/4YzGda2uuvHlnTrtCIgtMsdEvxkK2ceUOV1aam8fMZe7nZyhkv2spJPDbaiyfNVzsQbL6KxbZjr9S/p9rma1z3T33FBZrm6uu5+Wp5ntEJHOsnec41bJph3lvbKtUxfHkm50rVPZK60jMpRZ/DPBW+PMFWyZyRigBSmU8bUGMxuzKPBKOCU67r3ZhIXUkCr6S0rsilS4YqntYLztLzrGm8Z3MPcJ32VuHZjEZBY9eIfNGMZlwEcDgcyM/PR4sWLWC1Wp1/rAK2zOB/93oIGPUv84EWC6DZ5RftcS9g8QHOvA/0/0vNMXsFcOAP/O+yFPOx+UeBlEVAl9uBiL7m8YBo/o6+VH7vNlOA0gQgtJt5zFYG/NQWsBUDtxQBfmE1x0M6AdM2yK8bGMXvotdD8rhfGD9PwXGg7TTz+A1p8nEAUJkLaDag5XD1v2kKFJ4GVg3kf09ZC7S7oub4ideAmA+A65J5vY1oNRro9QgQOVB+/YLj/G31N4+F9wZ8w4CSc/KxM44AWduA1qPNY+dXA+lrgakbgdCu5vGk+UBVPu8FCaXJ/Gk5DPANqTlWEs/fw96Xj20qZG4Bdv0GGPYO0PV283iHq4BONwEBUeYxWwmQslB9HQCg/VXq76PtZfztqKz3aQMAMjcBiXOAHvcBgdHyv8naDuTsArr/DghsXfO8AloDQe3Nx/S4hz8qtBwFXLociBxSr9N1OBwoLqlEuF8ELCFdzP+gIguI+5S/rX7mcd8QwFENxHwEDH7F8OJ2zi/WAMA3yHxs4Sneu93ukk9u3Hd8JvfeD4z5d70+V4MQ3hcY/BrQ4RrzmMUK9PszEPMPoOVI83jufuDoc0DGOmD4B0Dfx2qOF58Duv4GaD1OeG0LMGmx+rwKjgGZm4F1Y4DxPwBdb3ONBbY2f/86HDZgzQig3XQgZzdQdAYY/RnQZjJwq/M+r8jiXBHSBWg9xvwa1yUB264DTr0N9H/aPJ69k8f3foTzWX2Quoxr3YBn5HvMW5x+Bzj3H/73pGVAcIea411vA9pPByDM5wBQlspr2ekGoON1NcfK04FTbwFVeYDmkI+fvAbwC5XHpLncHflHgf0PA+O+r/l9+gQCVx1VHxfUHmg9lv9OhV6/V4+dfBPwbynHFAFRgG8oz63LbPO4bzAw9G2gxVD5tTUNgCaP7X+Iz9bVJ81j9iqgJI7nFdRWfe4qtL8KiBgARA4CIMR5+ceA9DVA55uB0O41j7X6AeH9AB9h/ur5INBuBuAfaR7zCwemrgeCO8nn1O5yzolbrwY63gAMfNb8b27KAaqLuCYY0elGoF0m4Bchv37fx4HOtwB5+2G6v0uTgcQfuNa1GlVzzGIFLL6MWSUUHHXFrJPXAO2vrDk+YQ6QvACI/dQcHxpjzPoi9lM+EwAw4G/m8ZBO/DFCswPVxUDBCX5fURNqPh/p64Gi08DklVyrjAiM5lyQugzoeL35mU5dxjWw803mY7vfA0RNBNpeLq+BdYHVH4iebI5p2k7ljwRbObBxCp/ZiW+axwNaAZ1nMY6S5r+yFODI08Cw98z3CMBnucVwoMss+f1DewDd7wVCutb8e1Ab4DabfAzAe9InkOurHn8aUZEF2Er52lIsrkJlDo8LjGa8a4S9Esg7BAS1k2PpliOArK1A+jog+pKaY4FRjBfOr+I86I7oSZzH20w2r0EAsHkG5x8A6PcXYNjb5n/jab3o+wQ/U3h/81hgNOe0U2+joMVMhHca69rb6lg/gfP6rGL59R12IHcPENgWCOtRc6wiG9j3AO+TiH6uv4f3Am5XzPUA98VFp7l2WoQ1v4Xz+qT8JN9/6ev4ftLzHt4bGPyS+r0rMoDSJHms3eXAjZlA/NdyjKZD08z3XkArYMSHznVOwNC3gJx9wM5bgT6P8r5wR+ZWYONk/vfsCgCG1Fl5JuON+K+Adle69ioAkL0N2DELmDCP5++OCXP4PTuqOb+7w1bKn0Evc071b+UaK00A9j0IDHweOPEq0ON35jgIYFx6+l2us8bn5uSbQOpSoOsd5uOiJ/FcfcOAc1/yPtL33j4BjEHcY5+AVsCwd7m/O/MB1zh3HHwCCGgJDHyO/99yBHBrhfl9AWDIq8DhvwALWwHXnOHz+/PnTuKa0GYyENbTfKy9HJgfAnS9Exj/X/N4xACuvcPeNY9V5vK76v1HPgNGTN3Ee2/VEOCSRTXfv7qEsfuxZ4HCk0D3u2seG9YDuCFV/rwAY9n8o/y8RoR0BS5dyfeOmqh+DW9g9eP9V5HFfJoRrScwRzVUWKc6XsefxR2BnD3m9SasJ3BdIjA/HDj7MdDx2p+HtLSVsJx8FY7oKbAmzGFM1l3Ya64bz3X/ss11/0yBbYCx3wCRg+W1BGAcfH6Vc44zPHcJ3wO77+R/3yo8lwtbMD/osAHdhGcHYOwYMajmefsEumLkGQfNx3S+hb/LM/mMuceuA/4G9LgfOPwk46T+f3aN2cqAikzGwQf+yHmy5wNuxz4DRI1nnFWeLp+vJwx7j7/bz5DH208Hbink+3vaI05ZZf6b1Y+xdM4eoOhszTklMFqdpwF4z3hCcSx/T9skj8d+zvhNyt2krwM2O+Pn69OAYEPeJ2UJ9+vjvjPHFIWngC1XyTkGHWXngfMrGe9GuMUG9iqgPI3X3r+F+bj4b5ijGvUp4CPkKwNayjGKjtjP+bvVWMBf2J9svZp7mukH+P+Vuc54rrP6NZtRJ4g1jAuJRikZNqPeaFb21Q+i363dRqm/bs+hwrYb1WxBXTpvhL2aqhqV1YjetHrVcJk13BC4WyYZ2RWxX1DxobLwqcglo9MT+/AHX3OvFx2Z2yjnl6wSf7Yh8NDvoylQdt5lFydJz1NXUNou2f9oGtlrZefV/SnOr/VsLzUvlHZi9UVhDJmmKhn84o4yW12Hfg/kHzOPVZc4rRsUypWmgm4xMC+k/vJ+h4PfydlP1UrHBS2oyJFgryJz0siYqyv0nkQS0+v4q1Sq7bnfbDGUf5yWlJ5UuKWpMtOqNI1KEUmNUwtq9fjW7ZKOvqB+kbJ0uTdK7iHXdyE9z5X5tBtSPVO2crINJaVbUSytcBa0UDPefwnoa8UcyJYop97lmMomtjyL9/2Bx+Xxwhgyd41Meh2xX1AR4c4UdNjJ7D7wOOds3Y5r1295LgefdPVuUTEU7dVkse66yzym9y2bA88sSBX0YyVGZ/4xKkfXjDWrGc99S0bp0h6yMj59I1nK8d9515PmxyC1VVhtvdpKkqjSVNmYlGXwvlYpF34M5o839ibHXtK01SNkt4HiBJ67ilG5+x7Zdk3Hpiu5VknY/we1dYymOVUTvrJ1UvJiKiHnhZhZriWJrntEetZ1du2Kger3doNpztt0JY9XqR09qePyjnA+Vz3PR1/kGlx2XjiRSjoYqNR/tSF9Q/3tfjXN5Qhx5iPGt55s5VKW1lSglmVQCbbzTnWfMFW8W57Je2/nb+Rn3V5NS92FrWRL4KMvOO+BzeaxojgqSj0plH/uT2VQ2O28k3+Xeuvo56X3wZSss1YOISNepeR32J2qLS+t/D0hcytZ+EZmuq2c992BR+Xj7FU8RtUPpqqYa6s3jPW01XzOJfWJjj33U5FRX+iKijmQ7WlXj3CuQ8J94LDzc6uUpSVJntfA6jIy0r1VaKpw/DX2PNp5p6yaTlnCecJTb2NPiP9Oc2y8Ujuxd6Uc5+17WLZO0zTOEbqziNQ/L2sHz99Tz0BvsaQrr6cEexXXN9UzlXeU8ZdqP7Z6hHcWe4f+7Hl91DQqurbeII8lL6YdqxQX6Hvf7bPke1S3ypsDqm/cUXyO6g9PPTgleLJrqyzgc1R8zmkhqrjvdUtgqeftvoc1bWFrtfORjvT1VKN7Upj+1MH1fP7UXh7XLWWLzzEO8mQdefoDrlVG60i93/K+h+U421bpUiLVV8VUXUIVpKfjznzE62x8prJ2Om1Mf6POJcR/TzW3dN4Jc5inauz5qzaUZVBd3xAVT/Zez7FO1k7TuK0wTju74z+araKQ94t7/q6qmHvS3ANUvu6+2/yauvL9p/aerdBVOPKs3LtY01yxzL6H5Phr3SWcn6TvLH09c2t77jfvMatLeO1rUzPO9VNb15dnmmMZvVfuqb+bHSsq8xgnFsVy3pMUrYVnuU4cfELT9v/Ru+/TE86v5X7KU8++OeD7u8NhZww0B3JP3NWjnLmoMNmVxG5zxhSKGFrvMyrti9NWc2xha/McpGm8N+aAOQEjyjPpGpe5jcp4Y44ubRXzAHNgVnLmHuTfN0yT1229TZMnJ4vSFPU9VnCKCvOMzfL1iPkn58DSNMapy/up++A2o174pXv2NSv7mnHxwupDRluP39X2D83sHIBsZFsJMGW9cIivWmEEAB2u5fue+w+ZxcEd63XqSPkJSFlMxpTxWL8Isuj9W5jZF8EdqRA67WTbGM/xxKtAzIdk3o3/Xj6vUZ8AwQqmRsxHQOpiWdETORgY/r6L2XmhENTOs+Kww9VkaW2cIisrCk8Dq4eQITnoBfO4USnojoosshRVCs2UJWRa9/mjeSy8N1B+Hkj8nkwrIxNp7LeeVR4+wbwPJTWZbwjgSCV7NzTEPN5UaD2aDOfT75BFacSBP5Itel2SmV1rsQC5e6kUmbRUZmgPe09WSwBksPlHqMdrQ/l513+XJlEppcMnkM9d74d5rdzPrSyNz3mrMWbWOAAcexE48QoZ/kYGYnUhEPsJGVcdrvbuvFUI6wFMd7LdVVCpbQKjgG6/BRK+BdJW1FSiAWSWnXmfjLuZcebj7RXA2K9l5l36OiB5Hv+7Puz+uuLQk3z/UZ/U77hONwNRk8jgN7LxALKfNbusIAGAI8+Q5WZk0OsI721WiANAeQaw63Yq/wAq93RYrMC1Z83HVObw95m/83fkYKD1ePl9C08BV+4DIgcI7+2858d+TRZhfTHmS7KdQxQKdEc1lQWVOWR36yg4Bpxfwf+uyjcf60n5AvC+O/M+FUjSOtzjfqAqF+j/V/NYi6HAJT+p1ymrP+eocEGRDwBbrwHyDgADX5CVC+1nUB1jfLY0jdc5/yi/b0mFaSshW1JC8jzeY1ETgcu3m8dHfEQ1mb2SzF8j+j4hs/QBoKrAdU9JCGjN+19Su3W6Hih6moxS4/PsF0lVXsFxoDLLfGz6Wv6uLpDfN/cAlWw97pUV37qrgkrJMehltXNDaSJw7t+cs1uPNY/7R3BttQjbIB9/4GbFdQKAlQOAsF68z4yxZXUxsOkyIOoS4PJt6teQ0GIYMP0g47YlHan8uWKn/G+P/IXnrqucg9oAQ15Tv/bp95wsfYH5e/YTrmEA/41RJWKvAJLnO/9bWPcHv8wfCZXZQNxn/Llsm7yGdplFhwyjEnPA34Ce9wH2MrpKGBWIST8CcV9QSSG5CYz8CNhwKbDjFuCyrebxnD1UTw1/36xGaQgyNwMbnfNbpxtrsrR9AoHrBbcQHWnLge03Ab0eBvo/Y1ZI+YVSfVQsrMsA55fdd/Ma9ryv5lhgNOMQaV/gqKY6rjwdCPQQU6hQ5ozRfYLkdT9iAJB3EDjzYU3lE+BUrXpgO+tri1MBbIJvENVCEjK3ApuvoLJL5Xiydhzvranrav594LMABGWvDr8IKrq23yCr5Q4+AZTGA5OWyMd3+w0cnW9D9fHj8rinGMfqz3m78y2yqmvPvVwnjMpkeyVVBcdf5n0kqZhqw9T1aoVx/lFg7Siq5YxOA3kHgTVO14Vr4xTqeot8/5QkAmnLeO+4qyF0ZO9grCWhqhDYfReVPFFCHJW5hUqQWwqppDSi042e1ZC9/0CVeXtBTR7anXuKwlNA4o/cbwa05Fjij1ynoi/lfsRdQdJ6LOcAWwmPDevDnAfAtcv4DEnocC1w+Q75+xr1ifr+spXxvALb8H1qe6+IflSktJtORYsR18bgZ5VySQJw/CUg/zAw4h8192EA5/OyVFnN1WosMHk1HZ3KUoBLl9Uc9/EHhrxJp4vic3LMm7qMSqie99f8u28IXREKT3Hfb9zbVBUC3e6U9/ohXXm/t5kqx4y2UuDsPxkrSwq6rrczpixLMyuMNQfXX4uPHPc1BCkL6QYwbTPVkhIyNjKH0vOBmmqi3AN0Cuh+j+xuZa8EkuYCoT3NjhIhXVEaUszvfMramjFYVT7j6D6PU/kq4ewnP7/Oz8+EEalLgaIYoO+T5n/jKU7yFMsAnmO6slQgbSXvS6MSLmc3sOlyYNRnvJYhXajC03HgUf6bXg/Lzysg77dDewD9nmJcdZsDNRw6is7yWRj6Du936TkuPgsce871/9LzWxQLxH1Ot4EoQyxeXcy589TbXPvHfVNzPGkeFc+Rg3idjfu+kjief8cba/49dQnjIIBOBUYUHOPvsJ6yg4fVB8g5AMAiq+/HfkMFXtoK8zm1n+55vh/0Mu/7Yy/w37qvv4HRXFPPfiI/WydfB3L38b4P71fzdQPbAl1udca1nwHD/15zfMQ/GEftvI3nH2iIeUuTgaXO+VQ6/4h+vB+O/o3rgnF+7HYXsDASOPAIMHEhP4fDg9tDMy4eNErJsBn1RrOyr36ob1Vc0zQytM58zH4kEnbf45klYbeRQaRi+JdnU91QGzNOwp77+N7esL/zjjoZhIJiokYDdKHfQuZ29lwwKjF0eOplpGnsi3fkufqfc1Nj3yP0vJegN/pVeZ3H/YfKLokBpPvlq7zIf1a+KHpBHHzCyTpXNOU+8QZ7CtYXtnK+7spB9T+2oXDYyQqSGK6H/8rzknoY2irJ2Iv5RGZEaRqZ+CqlRm2KndqQMFfTVgygwqU+PfvsVWQCLu0hq3h33cXzivmncGw1Wb0qNZkH1GnOS1vFPknSHPQzS0vBFnU4qPqTelPZq6lmXDdRPnb9pWS9SSiK07SkBfXvi1gXuKsHJGyZKTPm6orVo9gfSsKPQVQzSsg9wGshoTSF7P8Dj8nqTx15R1y9ZpIWunqLbLvR8zkv7sTz/iVQksz+FEbljSdFUl1w5h8N67eQvZffoTRHORxq1YymkZm/oIW5J19doJ+z1Lssew/Z+iqGv662VfXMjfnEqZ4SVP2FZ6gClfquaBrVa6tH8Xs1ojier114Rj62IShNZbyiYvWeft/Zm4wxhWnOyzuqjr884cSb7O+UuU2+1nP9qe6UEPcl1TyS4k+HynlB01wxmNSHRNOoYlvYmvOrqgeoppEd7h532G1UFugqx/QNtfcBdceK/vyRkLGZSsfCs7KKzl5NNwv3flNGlGfKa4mtkvHAxsvN/SrL0qk+8dTHUtPY60Vai7J2kfmfe1B97NHn5bX5xBu8TusmqpWj3iLuPy7Vvapny5qxct/bvMOatmYMjz/+qnn85x7WijlMf29Vf8bMrVSgGPcelQVqlZimsS/T6lHcd0hzkK2c64FKsaVpVBhJz3NVMefdnzrU757Wcfp9Tdt4mRwH5R/jfb/uErNLRupKTfupHfuHedM3VNMY46uUaLr6QIrxyzM1bWl3zX7khfrvbeuCs5/JSu2ydNf8te8R87jucHDgUTleSVtFpxOVg4EeZ0t9b3Xl8uar5Pskc5tacZ+2ynXeKmVq7kHn/sIQe5ZnUzGz/w/ycQced+7TFH23GwMnXud7uM9Veq/JOVD3c9OVy5Jaw27jnCvlOKqKuPfI2qU+p1N/l9XrWTv4nqc/FN6zmnOL8ToVnlXnWmqcVzHdXeaACj4jNl9NNbYnnPlIVqDr51dVLPftq62Pl752S31Dl/bg/tQb6OqlA4+p/83yvnxuTMe6qZpVz4a3yDvMXNSxV2SnCv2enQOzQknPqyQvluNsfT0R4lmbzaYdPnRAs1UL18hWTiWtp5gg/5h6za8u5Z7Ik8q8IbBV0lWnvvvb4nOMRXL2MTY1xp97H1CrqHbfyz1JUaw6lyRB79U9B1znpHW7soD3QXE8Y17j3m37La7XOPuZ+fiMLa7x5X3M40VxnIPWjJF7FM+Bpm27yfz3nP1UdqqUtsdeceZQFYrj8kyOS++paYyfsvfKjhB1QVEcX1/Vczd7D6+30Vkpa6fnGKeqiPOjKtY+9GfOj1JepzTF6aQizCOaxmcrbRXnb2mP7u5+lHdUfY7NqDd+aWVfc7HvF0Jzsa9+sNlsWlxcnPlBWdaLG+bM7eaDDj6ptvfRNAYolXly4cte5Zr0pOSbrYJBtspeqDYUnGZhRJWMTF3BYEpsEFzBRKQqcWyv5r+RjtUtCqRCoKYxgJa+Sx1rxzNouJCozKOsfduNcrElaT4TSarkrF5sUdlIeQr+bRXc1KqKUzGfqG1V9YT9sZfViRlVgF8b9E2zqsDZVCjPZJLMk6WlqmBcnuWyx1DhwONyAkLTXJv97bPqd84/v382NzSeChHVpdwoGv/NqXeZMPLGcjBzO5tT19PSUjnn6XA4aJc5B7Ld5rwwV5BfX6StcRZthWSRptH2Yw40Lfmn+r92Q1AYw89z6u/yuG7NsWGKeay6jM/N/j+qEyBnPuJnk1CWrrbH2Ho933f1yPrbZcZ9yXt7zRjau2kaNwix/66b9VPMP/lMqDY9lXn83rxZq6rLeLzKXs1b6AlFlY1KRQ6TECoLs+oSbtRUBZkdtzo3+k1gz5f8k9ruq/AsCwfSRuzsZ87P60UyW9Ncn0myd9Gf19h/y8eWJGva4s7yva1bY8V/Jx+7+RrPNukNQeY2U7Gvxpxnq3DOx/W8/xLmMqFcfE4e336LmmRz9HkW9k++4yq+G5E4j0l+aV0vjGGcpSJynXhT09ZPop3ihmk1x+w2ks+k17XbnOvfbPl1s/e4bdSFuaA0hVaxngrd3iL/uMv2qD7Qr79UhNY0rnF2G5MTKrKWtzj7Ga286pM8qyscdvU967AzmbxmjLpIVJ7FRHv2XvNY/jE12U/TeN97SpbrxcDUlTX/bqvkd6KKU859wzlEL5A2JtxtxVXEOU9Y3tfzd6JbnRmLRBlbOLedXysfl/ADCUTbZ3tn1Vl2ngUCqahfkfPzZ44/vb/xi305+zRtfqS50F1dxmRi5nb5Hj35ththVCjonfnIleSXsLgj721vsPUGrq0SKnK53/oxSE3mOvGms6DmwSJSQnEC18G0VfJeryyD+7y5AerXsJXzdYwFr8xtmra0Jwv3SQtr2sIVnOR7nn5fTWpJWcb1xD1+yz/GgoFuJ7f/T+bj9CLR/j/I11l/5n7wEY5NJvkjaxeJIPsedtmJ2iqdZJbbzcdVFcn5lLwjNe+lsgzOQ1JBpzyb1/fo82qihAoOO+NGqSihaa57W9o/VJfw+s4Lke/7I89p2pZr+doq28tN02mta3rtMt5fqgLW+bW0yD71rnmsIpftPuaA16SxoT/PmVvNY4s7cz+fsdlcNKsqZHw1N8Acx2iak+j6hVjktMcwJ2NLWcnvUkWYP/OR+lqqYKvk97X7bu+IYsXxzCuqSMf7HvZMptE0tvcx2lK64/y6+hH9j73M9iaLotX2o7ZyXidjzKsTsNeO81z4V+HgE9xPlyTKxfySZFfbCen+1ZG6XG4ZcupddeHeE0rTWPBV7WvPfc1z2nmHPK7neiSUJPP7WjNW/jeV+fyeS1PNYpGiONpce/ouvEVZOte2+s6LOtZdoib5aBqLrk1Nevl/ilrzeQY0F/v+R9Bc7Gsk6MxJ3Q/eHUWx3MzFfiH3W9A0Fq8k5ofdRu/ttePlDXfyYleSorETitVlbqwkA/Op+BwLIlJgVhfozI+9DyrGU1kUkwL2rF1k3sZ84t17ewt3Jqp0HU9/yMKGKjFXm8qkMKb+m8O6IG0NAwbVtVo11NkzSNH7aVFb9cJbHM9g01NhtilQcJLntPEy9X1fcEruE2crZ+LB23u3+ByLKonzvDt+6/WyZ72mMVF66j2XMsw9gKsu4cbYG6WavZrfhyeViLfQWXtrRsvXImsXgzopAVaexeLBidflQmH+CbIJVdeqLMNz0Jh/jHPjLxE0lqbKCpPEea55RJrD4r/n/KYqZFcWaNqhv8iM9fSN7GuypIta4VecIG/2fwxkgjZhrqtYk7qc92NxAs9rDqgOlFBdoiallKZRpTQH9e9homlM3s+BXDSpzHP21jhknmOLEzj/nf2XvME8+Q5fd1kv9bztCfrmUirYZO3i2PpLvStS2io5V6nWDb2w64nwIKEyn4XdmH96xzZePYKbSAnVZYyBJGatppGMolJDlGfzWd1xq7wO6t/14WfMY1VFvIbrJytUXeVM1qiKkFVFjNFUhKyTb/G9VYpFT6jMZ5JR6rmhabw/T7wpJ5qqipwqJy/VPnVBVaE5UZCzT63ycDj4Xan6SOtrs6dn3RNZq7qMz7IqHik4ybhXSn4ceVadWLFVct6T1sDKPF6H3ffw3jWuRXpiTbq3dBz6CxOS0nlvvIIJy18CtkreW8Y5qLaibW0oTfXM3s49oO4bVXiWyf/09eo5KGm+Z8VpylKZoGGr4FyhIgac+ZhFDQnlWVT5SHutuqD4HO8h1dpbmc9nS0XaU2Hv713PlNQbufAMixK5h+p9ypqmkaU/B3K8XBsKYzgn/xgsJxYLTjIJn7Sg/q9dmsp1WUqkOhzmPlvuiPvK+xg9ew/3Fqrr6Am5B1hEz9gsJ6XtNl5PyXVE00h2URUx9WKxihS68w4WeiS1WtZOTVs5mGql+iBpIWNKaW3U+w7Hfs4YXLrGtnLmMOZAJqPqc2tthLLTH9aMd3RHEJ1sbLexEJa0kCTP+eHm1/ipg+t+sVfzWE+OSKreVBmbWXRVFYAcdh63pJtMzKsu4Rwgqf6qyzinHnlWfV6qPopFccxFLWqrVvTnHyNBTDovXWXuSaHeVChO4H5DyvmUJNW+d93zOzVBUt9nGpWSGZtJUMw9yOLqpuny8Ut7yIRmu417OBXpsyHQCQOqZ10fkwgieu/jeaF8vrzB9lsY96rOLfbzmn/LP8H9o+6GcvBJ+di9D7IIaYS9mvdkUZx3biaaxrhCtUewV3veT5WmcQ9h3MuVpZOI4ImImLWLyjppf1pwknOxivC3YiDzb6IC2K3XvfS8/rxuC69dksTvWXKT0JGxiUpz4zxWlkFiuMq96shzahJOXaArdRPmyuPbbqypXo7/jnO6NyT3ZjQIzcW+/xE0F/vqB7vdrqWnp2t2u5C8S1pgZqm6Y2l3OWAoPEvWuicGjgq6leYceGfF6Qn2Kk64G68wL5CZ252soj/LSZf472kFkPijzP5w2LnIqQICXcYt2RplbmfwpSqcNhUcDm6qz33DwFRCyhL1AlkUy41RU6iQ7NXcJKgCGVuFenzvg9x8qqDfX78mlk11GZ+lOZDZjZlbGbSpEqyxX7DgplJZNiU2TFUH8LpSbf+fyAh0Vz/8bGmjSFTlHnCyyl8yj+mF6v1/rPfpepzzNI2b5T2/o71afeFuDaOy7Yr/Xr2Zstv4LErq4oQf3FRbXqqYPCFrh3dJwbzDtEpJmi/fnylLWDBRvXb6RidL9fH6va+tnGvU4o4M8E2vu8G8hmy+xvUduv9IsFexkCMVnxLm8Ljl/byzK0lbTcs3KTGs3/dzYJ6XdYsVT/eXJ9jKuXFWqZCW9nQWgXabx/KPM2FY3+Sajp13ODeY38vjnhKdhTH8UREhdtwmk3g0jQmROZCt+zSN10+1/mkaVUIbL5PHtt2ovn80zalWhMy81TQy2uf6m/+uF8XmgEUAI9zvAwnVpUwyOJN6pjlvboB6U10bkuY7CUKKxPPeBzkukR3sVYxzVHNB0ny56K9jWS+v5nytOIHPS9oqrkWeEgbLejNGrCvK0rleSFZ2ifNcSnDp/rNXu66jpCzNP8bXkCzxi+PV84SO468yLjUmVuK+YlFMpajW56B5IfJ1PPy0s1CjsGEvPMMiUWOvU+VZLM5L8ZvDwWdFRTArOM15wpO16LoJtDaVUF3C+0dK3GVscpJdPNxXm6+ieqq+cF8PJKwc4lQE3Vr/124IShK5ZksFDx1n/kGlkRFVhVyHKgvk/ZReBFrWW37drB2e9x0VuZq9ME5LT0tWx3kq6M4hcyAn23ffTaWcBF0FUhfnAAlH/qYuCBfGcA6TYvzybM6Nc6AmFcyBbF2uq3xVBJ71k6mOkmC3uSwaJUKCvZrEkJh/yoXX0lTG2arva14YyQoHn6wfCchWqVaX6HuShuQXzq+jAk9y8sk9yHWsNseH6lJnLKaw4tZbSuz8DZ8jyaoy5hPXPkVXoKuU2mXnnaSWk+a4M/+YKzaWlFUOBwlmc6CO/xx2723mi2L5nRoLlYVnaR2p2iNqGkmXc2C2EtY0xlFl6XJc2JRIW6Vpq4ZxLlDB4ZC/r+pSrm1Ge886oEact/f3NQvkDgef4x23cy4xFlY1zdWCx5OCvarYq3PT8o6QnKYihuQf95yHUhWnCk5TIaeKsTM2sZD3g6/aYl5C1i4SPk684SRTuxHAC89w33nq74x3V4+Q31dfR6T9qabxWhfHq+cqu43rkTT36epj1fdZdp5kCON6UllAssocaNrOO4X3rHIVr+rpmqRpGu+vOZBtOCtyuLYXxsj3fupykiwyt6kd1uL/y+ttvH91AsaWmeYYTxdjzIGce8ncyvXr3Dfy3rg0zRljCarruqC61PX++cdIOpwX0vg57v+HqDWfZ0Bzse9/BM3FvvrBq559msaJOnuPrB7QmW3e9pZKXckFVLKWbCrYyl2JOYn9o28QVIw9e5XTz16RjJwX4kxQKKyNaks4/lLYME3T5kfIYxmbNG2ORe4P0FAs7+dk2ihsUXVmsCogKI5Xf9cFpzwn6Ddf5dlKoqmQvp7fpbT50y0jVAzF82uZgPgl/MCry9QKgZJk9ijZPtsc3JWmMIm+8055Y6wrXyTr0uoSJspXDKi9L5EBdZrzqoqZkJICUt3OV4LDweMOPK4u3m+Yok6enHxHbaeatpoWKCffVidpvYWtwnNCMW0NN1qqRFJt2PcIE+IS5kdQJeKp754EvXfAgpbqQo4RBx6tvcinY8etZINKKIpjErMp5uyKXG5KV48yfyc5+50J1K+82yjo9j+qJGp5ZtOoZTWN57tuojouKI5Xr/k/b9SERGh1GTeJSQvkhKVuO6Pqq5a8mN+3lMS3lfO8VPZS6RucVq+H5fHqMiY0VWz7sgy1+qQojtdZIqWkrqTyWFVwS1nKz3zuG34M45x3+n1ubuubnMvezfVx113y2nzkWRbTcvabY6HCGJ6Xqpezpnl+LvMOc0zqx6ZpnB+PvsCNuyciz4+B7F/kDofdlQDdPttzDyAjPKn6dKvzLTNltwC7jeeztEfd+jK5ozKfao858M5GStPU6oHaLGg1jepkqddq1g5XMkllF+wt9ELz3AB1f+yTb8vrgU4umgO1CqkiVx3LFJzmsZJCxVZBFVr6BvNcYa/ifLvlWjl+Kz7H51R1z5amMXGmKqoVxzMOUqlAiuNJsPBmP1Vwmp9J2tdk73UWfF80jxfHuxKGa0bX/32riqg+UPUFWjvOed8L/XdsFZp26u+aLWWld3tbvSeWah7afJVMSrGVs5g7BzLRNfeAUxGRIK9TZedZHJCK65rmajFw5iNhbLvrnFXPRcYWOVbWe0OtGk41ixGpy52tL6TChNMtZ9N0eY08+rznfVxtqMv6dPxVZ98tt325XrhaNdSsttCLLCdep0tBU9gvF56RVamFZ/meqnm1JMk1/zjs3KvW9bktSWRyX0Wo0nuEqb5Th4NkL1X/xYKTzI1IcVJFLq/BvDD52PT1JJ1IytIjf/OOwKNp3AfNAQuCqhht42Vy77KqYv59x+3ekZ48IWUp4xSVjXFpGi3H58C839fJBqrilaZx7d0y0/TnWve2m6bLSmodcV/yvFTxxOkPeG6LotWv0RQoSeTeRSJE5R3hHvDE61wzjPtMvfgk5VWSf6KqS1VwV8GdaHX8NVkRVniG1rd77pO/8+TFnHPd4vSax8e4FHgSIfDcNyQCrJ9MG0kj1o5Xx7EOB/cl0lqjF/VjP1cTFjZdqSa8lGUwd5M0Xx6vDXlHnCTvV+TxU+/xehvXlKPPy2QWHSWJnIOkfqma5rl4WpnPMZU9fG3QiRtz4JkA0Ix645fu2WdFM5pxMePsJ0DyAnns5BvAXCvgEwC0Gmke7zwLGPlPwDdUPn73b4EfLMChp+TxDlcBoz8Dgjt6d+7ewCcQaDkC6P0n/jZi4LPAZduAYX8HAlqZx2M+BhaEAbn75dfveD3Q434gpJM8vu0GYNNlXp++V3DYgZy9QFGsPF6aAkRfCoz5Uh6PGAD0/gPQ/beNf24+QfxdHGceKzoLZG4CWgzjdZOwcSqw/QZ5LKIfENJFHitJBM6vAk6/V+9TbhA0DWg1Guh5P+Djbx7v9RB/H31WPvbQ43xeWgxu2vOU4BsEBLWRx0I6y0Sl+wABAABJREFUARWZQOpiwFFVcyy4IxDaHUj8DrCXmY/t9RAw4Udg8krhPUOALrcChSeBYsX92xDsvRdYECGPLe8NzAsE9t5vHrNYAL9wYMT7QJfZ5vGis4CtHOj/jPza2duB5PlAVb55rP104JrTQP+/AP4t6v5Z6gL92ijn7Dv5s/d38ritnPOFrVQezzsA5O6Tx/o/DQx6EQhsbR478CeuFef+A5Qm1xzziwDGzwWmrgMGPie/NgDsuBXYci3/u8VwIHIIMG0TcH2K+hgAqMzhj4SwHkCPe4HQrp5fwxsEtAQmLQam7zN/J61GAn3+BPS4B2gx1Hxs7gFg113A/kc4lxkR0d/zewdGA0Ht1OO77wYWe7kutxgKXL6d7yGhNBk4+RafESO63cXf0vmnLQO2Oq+vRQi9u94JTNsCTFwkv2/uPiB1CVBVII8t6w4k/Fc+NrANn9eUJeaxjA3A/GAgbQXgG2weT5wLJM8DQrvJrx3Wg9c5pLN5rMNVwJV7gS6z5GMrMpznsFEe7/s4MP57zlf1QcFxro9dbpevRcK3QO4e3qdW35pjST8C264DSpPUr+9pXrNXMPbq/7Q8fn41cOIVYMOlwNG/qV/n+jRg/Jyaf5sf6rqHJv4IjPjANVaeCWy/hc9UZa759breDvT8vfxePR8AbteAS5cC0RPN41YfYHY5MDMO8AuTX2P/I8CBR81/948Erk8FrksGWg6vOZZ3EPgxoPY4JnkRYykj2l8NzCoFutymPvambHltztoGZG7kM9fzAc/vX1+0GAaM+Adw1TE5DgeAxDny89p6HDD9ABDWC9Ds5vGKLMZRWVvk1z3s3K90vN485hMA5O5lDJ93wDDo3CuF9zNfJwDI3g3suRtY2gXYfrN5PLg9MPQtoM0U+bxCuwFXn+AexYjSFM5fu38D5B2Sj/eEvb/jZ0pdah5rMQQY/j5w4mUgxTC3lsQzVu39J2Dcd+ZjK7KAnH1A9i6gPN087hcG3HAemHFEPq/u9/D3+VXmMUcVcPhJWM+8J1/n2uAfwWfy1ip5fNSnQJupQPbOmn+3+HBt8w0FOgp7j/NreH8t6waUJJjH01YAG6cI948TJ98A2l4hP1PhfYGJ84FrzsrPxb6HOPeG9TSPBUYDQR2A/ENcs4zocA2v51wrUHiq5pjVHxj9BedkfyFebjGMa7dfuPyZACD2U+YbJHhan8rOA2c+5P3TYmjNvWDby4CIgUB1EaBVm1/TYgHKUrl/cEdFDufa2M+A7TdxjTaiMg+Y68fvVIX9j3APaoSjku9Zmc33KjzF2FnH8p7APuf1tVi5V61rHiSkCzD634xxfQLM4x1vBHo9wv2YppnHLRZgZiww8mP59R1VQOdbgPDe5rHE77lnCe4gHxvzEVB4AtAc5rH0dYzhPCHmY+DYC8I5+wFtLwf6PiHff1X5QFEMYK+UP0/KIiDpB/k6NwQdZ3Lu234j5zkjNk3jOtnpZs4b7gjvzc8T8xFw9l/y6we24XeWtaPm30sT0TH9NSB9tXzclNXA4Jd5D0vxbo/fAZdtBaLGycefeIW/u9wuj9eG1GVAvCKWzj0ArBoCJHxvHgvpAnS/G8jazO/NHS2GADfnAn0eAzpcC0QaciD9nwamrDXHowCQthw4+RqwZTpwRLEnlxDaDWh/Ff/72HNAwTHzvwnvA4z5N38Gv2weT13MObfXw8ylGaHZXOtX97vN491/C4z/js+cNL9W5nD+k2Cx8LiAlsJ59we63wu0nsDcjhEZG4D0tUDGOvm1g9oA477hXCGhJJ55UodibQ5sCwx8EYgaX/PvtjLGs5GDeL2N+4/BrwCXKPZ4AO+hq48DfYVYGuA+ccx/5DjcP5IxwZgv5GNjPmaeIuYjedwnkPuA2zWgzWT1OTbjokNzsa8ZFzeOPgfsmAXsFDb74X0ZpPiEyEFjm0sZSB/+s/za+kZcSioWHOeG9/xa78/dE36w8McIeyWDv0EvcKMgIfoSoN8TnPiNiOjH3/mKDfX479ULRXkGg+DoSbWefqNCqwbWjQVW9OZG1IjMjcDxF9QFtcBoYORHjV94AIBpG5jQkRbGotNA8VkmLP2E4sSZfzDI73a3/NopS4CTb9bcZOmwl7PQOPi1Bpy8F6guYHHp4OPyeEgnbtSkRJPFwg2DlOj+paE5uBlzVMmFyr5PMSkoBautx7BgZvWTX7vDTGBmPNDuisY8Y0InOkjJBv0a5OwxjzlsTAKpCuiVOUyGq0KE4e/z96m35XHNwSBZmncbAt9QFlav2C2Pj/wn0HIU0Okm81jBcWBBOLC0s0wQKTjJze9EBXmk31Occ6XNpx54770PyD9Sc8wnAOh6q0zOAIDF7ZlwsZfzBwC63wVcdYSJU0c1C2MxiuTGxIVA8TngyF/l8aR5wKbL5eLUL4X0NSyex/5LLqpETeCG49oY+fjza4GFLeXkR1kaizkVQnK2MZC+Boj9hIlgI8Z9y/NuP8M8Ft6HG/3QHvLrWn0Yk0T0lcezd3Ataz3aPBbciXGKisQT0hm4dAULPkYEtnHOfdVAdYl5PO4L4OCjTMgboWl8lrIVz6P+b1TzQNQlJF119VCs8QZdZgMzE/h9SrjyAHDJYqC62HxuHa9joWbNCODwX+Tjb87jdZbQeixwU466gNT/aWD6IWDQy+ZEQ1kqsOtOIOUnJjiMicHOt6jjr8ocIGUhnymJWNL7EQAW4LCiCNkQnH6f7xv3mTzuF8bYwJjc9YsA2l3JxErMx+ZkZ+oyYN/vgatPMhFmhNWXSdhjz8kEjnNfq5Nj3e9lkabVKJm41BCE9STZQUo4A8CiKMajl203j1msXCuuPQv0vM88bivl3iR5vvza0ZMYc0gFO70QPeBvLCa6w+rDWHbQi/I80GYycOlyoOVINQktdz+L2RIqc1mQlmCxMH6PmsDrUV/oBcbyNPOYTwAQNZGJyBADYaHVaD6LA5+Xr9WZD4F1Y4D1E4D4r83j9irGOlJ8DwBdfwNcvoPkONN5BQMtR8KStRm+9gIPH84D7BU8r1yh8FaWBpx8nURJd1j9gOsSgVnFcmG/6+0ksHW6WU4oRk3g7yQPhQffILmQE9jaWYjpZR4DSEpqKRBzAe5drzoGTF7FOE38N/2d522I060+fN4DWrP4ZkSnG5gU33QZC7tG5OwF9j8MHPiD/L7ZO7lObrma85E7SpNYPA3tTvKWe6Gp76NM7M48B3S7s+ZxZalA5hZgyJskLLh/n7Yi4OxHfNZSfjIXAwH+e6s/5+Tic+bxk29w7yyRYCMH8j37Pg7EfwWsHMAi2M/n/QS/M4D3f3kG19LT7wM7BPLglmsYG+toPRqYcRDoLJAGut7KffHidmbiZWkKcPAJNTkIANaMAg49IY+1GgMMegmYIuQRAH6fHa8HOlxtHrtiNwuRc33NhL7qEiD+W+aS4oT8iV8oyX59H5PfN+E7oCyFBEkj/FvweR35T1fhpjER3JFxmLS37X4vMOxd4JIFfIbcETkIGP53Fv1VBZWh7/C3MR6pKkTr/EWw5O4Hdt6hLhauGQlsnCaPVebK6xTA5+za2JpkqLoi4XsSDvYoyOFrR7FoVpktj9vKgH0Pqol3vsEkSfb5U82/txjMHEHWViDfUJQb/Bpw9WkWhI3fZXUxkLaS88CK/sC5r2qOj/qM9+7AFzin1BeDX2P8NeIjWTQR0R+YXQHckMG4WYXx3wOTV5j/Pn0/C59ZhljIYWee6ejzctzQ5lJeX79wRZHcDoT2ZL5AwvFXSdKVkLMHWNYDWDvaPAcBQNJ8Eoy6/xZoa7g/7RWcB5IXyq8NcC9+8HEzEcdhI5km77B8XPJCzm2eBASqvwOutbXVGPW/cUdZKs9H2us24+JCo+gDm1FvNNt41g92u11LSkoy+91m76UntadmuIuiZfm4ptHDekkXeawsw4Mt1kaX3Lkpmpeq7Fn0XoEq6XhxPO1MVPZ5up2J5IGtabTtWNZb7lXzc++yD+r0ERoNDjstEueAFjFGFMWx35HKDvOXQmU+7ZBUdjerR9EWUAX9HqivbWBTwl3mL8Fuo7WRqvdAxmZaoTS2HUlDcfZT1+da0KLmWFEs7ZCkni6aRquJtDWyNam9ivYVqufVA5RznjvmBpit3uqCqkLX5439t/DmNtpEqOaR8mz2DZV6peafcNm21ad/SVMjc5vLWk2a3/Reg6peH0Wxao9/TaONUeoKtb3aoadoK2LErt/KlmuFMexztHqk52eusoD/5sTr5jH3RuON7btfeMb12sYeiPHfadrCKI5JPQ6LE/g95+zzrq/GwtZOm7svzWO5BzlmbPjeWPipA+0QvbHV2nGruh9WwUn2aVDNjWc+ku8THWtGq9eTPffThlFl0xn/vfPeX2weKzzD/mCS9ZXD4boHJPvHtDV1s6J1ok5zXl2RtoYWQao5aNddPC/p/qvMZ8yo6kvlCdWltB/1xgYn/4TTcvAFrieerNGOvkDrVx22CvalivmnOmZQxbvF8bQ5XNJVbd2m90+U4t34/9L2TeqvU1WoaSsHsV+q1PdM09jjaw7M1qlHnuXfC8/Ix1UWuHoIS7aAW2Z6thWvKqSVq6qfTEMQ/z37pkn9Ybdcq+4jWF3KWFtl4e6w0w7zp/b1Pyfd0vLk2+p/s2Gqps2PrP9ru/ebkbBmtNq6r6Fw2HlvSXGnrYJ7BFUcrmm8RpI9bfpG2mYdekq2jNP3gYef8e68k3/SHAee0JLjT3k356WtVr9/ylJaR3r63N5iYRTjWglVhbQCVc1BKUs510jWa7XtTz1h7wO0P1NBtz1V2bQnzuNzJfUBzjvCmEMVxy9q6+whFsS9qDuqivg81zdPcPp9tQWsvZqvp8+XKsvLU39X2yfH/pvXsLae8Jnbeb1KU+Vx3aZ+30O0m5QsOFePctnplabSUtxoW+qOc9/QUt9ou6vHdku6qu2C14zhv5GuY23I3OrZyjXuK/bcMt7bJYmu+1b1PR17ha0AJIvH7N1cDy50i5SSJN4HkuVpXVGZp7b2tldz/jE86/bqCi0l7rBmL89jXGl83k+/zzYAR56T9zVpq/h9L4zy/rxVSJznXCMVPSUP/1W9t8jZz/hqx+1mK+KqYlqf1tbC5Aef+vXsyz/O8912I9vKuOcpKvN5rukb1T1Ni2J5viv6My5pbCT+yB6LqnVIf3YOPlnz7w67K57Yc5987Kn3vLeHX9LFw54nhv0L146TrcFPvsV4OO+IecxezdZRKUt4vY17rvj/umLWs5/WHKsucX1mKUdx7mvuP6U1obFw4k1NO/QXPtNxX/FcpDxPM+qF+u5tG7tGJOiFm9GMXx+sVis6dxZsolqPrt3mrMN1QFB789/P/INKimlb5ONUln8AmaQD/kZ2nE3BLmoIbnPIap3AaDLPTr9LS5CO19YcT1kMHH6S/31DOhDUtua41Z82nyqLsvivqEaT1FcRA8iuDu9T30/TMFisZMWpENaDLKwjTwM3NpGaQ4W4f5P5Mu5bs5LSP5KWRGc/JrvKyHSdtASAB+VTp5vI2pJsP+yVvNbhfYCWwxr2GeoDn0CyfjO3yOPn/g3sfwiYusHMeAKA/MP8PrrcRqbrrwXhfaiUtJeTyegOh412RpKiCwCO/pUKtzZTgWlGxqkVyNnFa28rl+0mFFDOee64pVCtKPQEnyCgz+NAzAeyCsTq47Kyu0Fgy1fmUJERPdk8VnTGZcsREFX/c2sqRF+iVuMA/DzD3wcsirBo/8P8HXWJPB7cUbYyqi6m7VFFlnzPj/tGfj17GVBwlP/d7ymzEkNH9k4ygiU7Eqs/7ZKGvCbbaTYEVrf5zF5hGLTwnvdvKX/m0K6erUWzttPmcOhbMtt56DtAWTLtfIyI6A9cE0MWf1Og10P8bJJyYfUwKjunbgTaCvZYkYPVbOT0dVy7z7xHazgjOlxNBVR1iawkGfOVzEIF6HLQZioAhd1Y5ACg/19ldUt4H1rPSApMiwXoPJs2n1ZBHVXitLeW7H8E1GnOqwts5UDhcT4/NsF6uSSRSsju99Biywj/SODybea/6/gxkFZn0nxSmkT2bdvLZMW/rZTn5N/CbNkU0Y8saWjAvCCukxN+kM/h9LuMQ3XrJJ8AczzojuMv8z2nCXaYmVtocwgA1YXy8Q4ne1pab7rdaVamuKPgOH96PiArnPs/TZa0j8FGtv8zVJcUnuI6HGm4j/L283nr+yTgF2l+3dFfAIuX0VrNyCq3VwJpq4BdtwGjP29cK8/sXbSkBKjMMeLSZYwpShLN82BpErB+ImP0qRupsnGHxUpbepUVdcEJYOtMoP+fXbbqOoI78DlvPVY+NuZj2nJ1vaO2T2iGPveolMudbqLd8PkVQDcvXt8TLFY1o70oBlg9BBj0CjDo+ZpjDjtgK6Y1aeRg2je7o+1UeR7Xoa/3p94ChgrXec+9tF6+Kce8p9KoALe0vwqdpLWkLtAdCiQl+Zn3qfiSrMOPPMOYdfgHarWRpqntKW/ywPTP2QNsvpLPnlHRmH+UcSXA5xqGeNhipdK7laBeL02i00nHG4D2V5rHrQGyDbWO1UP5u4MwR55fTavVS5fJ8UqLIVS6qTD0LSpMdLWbO/zCqIZMW0Er0P5/cSkm476gbXTnWVyf3dffNlOBkZ/Q/jN5Qc34zupL21wdquvU+xHauEu29z3vk5XDAOP39LWMG6InygpQHT6BnI+iLuEzrtuPumO6m0WkrYSq5NSlnNuM89upd/nek1cI6+NAKraW96JlvmQZ3OdPvL9VsdDJtxhbD37FPBbUnvZ/PoHm+6DgOGOhHkvMxwVEU/Uc2l22CHXY6QZRXSjvAVuP5RqXd9C8HmgO/t2/JXMdjYmCY8C++4GxX6tV6HFfUGk26p81/16awj1R51voRCLhzN8BWLkWucHqG4COPYbyf27OpV1njff8nHHZ1cfl19VtelsrbDxtpcDBx2i1OORV+d+o0GWW2nYeAIa+oR6z+jEOaT/DfF9XZgM7bmabHJ8grisdr3ON73+E3/XgV6ncckd1MT9TQJRZYRnckdevxVDzHq+6kPmJLrdT0arZzOdcmUeLWECOKWxlnB/jPudaYnSWqC5mHiwwmkpd4zqcs5fq85AuLptTd2RsBAY8x3nKHRYrcM0ZquslV6TcA0DMh0CrseqWDvHfspWKpCC+ZBGVowVHgU7X1xwL7622xwYYr0YM4j029C26O+mw+vL/Dz7B/Mp1yTXXpbTljFmvSza7jVkDuNc//R4tbo0uLt3v5r2/6w6q3FWW+g3BUadDUNfbmDMZ+80v03bnfwyNtrf1Fo1SMmxGvdGs7KsflFVxW4WaLZy5jWwzFSv49AdkwhTHy+M5+6m6kJo1a5qzgba9bs25GxNZO8j+OfMP81j+CU1b1kvT5oVqWlm6eTxzG1lUEiNF0zRtzVjPjOHdd3vf/LUpceS5X+a8dBaO1ODaXu1ilqsaCG+fJStfaoPOhlp/af2PbUqkr+d57X1QHo/5hI2H69to+kJg6/U8d0mdsPMOZ/NsYa7Rm4En/ii/btx/OK6aRxSoExPo+Kvq7/rEm1R8Ze2o1/tqmkY22tbrNG37bHl812/VqpiqQjYCN6o0LgT0ZvLHX/Pu+OV9ye6WsHIwVQ8S0tZQLSGpvapLqRKqTWmWNN/cLPzsp5p27GXPx9U2ZzclbBVkADq8UCVUl3CNktQYCXPqpQgzIf57si+bAqWpvL8lxqd+zjGfmMfyjpA5m7Zaft2iOE1b3Emtujn2El/byBjWNDJRN83QtKQF8rE5+zXtx2C56XtxPJ0R0jfIx1aXef8s2238qWOM1GjKvtSVTuWnQo29arimLe0ujyXOI3s2faP69TddqWlLusljOfv43rt+K48f/qtTQd5Svk80jffWsZfJDHbHob+47o+iuJoqAruN/69SY2+7Wa3YKk7ge5Wdl8c1jaoKT4r8zK0yG1nTqHI8/LRZ9VCWwfintrXxx2BN2zTd/PeSRDLwVfGsppENLcVnZz7mddg0o/FVz1k7eX/E/Uc9N6rWmso8svTngGpeI+xVZI6r9i1Hn3eyxj+TxxN+0LRVwzQt95B57AdfrvsS0tbwOizv4506z+HgfVaeaR6rLtO0dRN43unr6//aB5/ksQWnzGNl6Vx/f+pg/sy60tGTkqM2nHpP3otpGu8tlcrcbtO0OdAc22d7P+dlbGb8Kc3PGZt4/1UVm8f0z3zq7+axs//i2OJOsjovZRlVIKr58cSbjJWkZ6owhvvTw3/l5ze99lKOSXFl7gHXeUuKfk2jW8C+h2WV7+oRmnbgMfm4U+/WrgYrz/JuHXQ4uN/Z+wDfw935Yeedbp9JsVZtvUHT5lhq/s1eze+yKJbXQVLn2as1bf+fqJJTYc/9mrbxCvPfddeGI3+Tj9s+W+1wURscDroPzQGVI0bse4jrlHTfahrvm/NrNS3/WP3fO2Wp57hSjzsld48NU82uL0aUJMuKbHs1X1e1n9I0qkPXTTD/3d1NR3JhaQjKMxlT7LxDViQeeJTv+4PVPFYUR3XT1uvU6tAFLZ37xJrqJrutWjt/ap1mL4yTjys+59mpyVbO+0O1tq7oz/ddPVL9Gp7gKV4tz+Q+rUhx7ipUl1DNmrKE53boqZrjZ//F71KaY/Y94lKCZe+p+3vaKunKod8/UkzgsDOmUH2Xh/7sOl56LorjXeNSTKHvH5b1kpXX88Pp3FVfJC92OgUonChKkjk+P0Ied9h5TbxxaNE0KvQWtjI7eDgc/MxZu3i9jcq+8kzOEap7rLqEsbRK5XvsZe5fvHHEqQuOv8o8r7SGNsNrNCv7mtGMOkDTNOTl5aFDBwNralkP9mmYON+sbCg8RSZdpxtlNVrfx8i4q8hmI1sj1jp7R/hHmpmT1UVkloT1UPevaAj2PkBmmqT6iJoA3CYwdAAyn6/10Jup5Bx7JXW+hWxFIyb8IDen1lFwvKai40JhfihZR9fGmdltmZvJHBr23oU/r4kLqLCTlAtZW9mceti7Muu4PIOsX4mdCHhm1gZ1cDLqBGVLU2PP7/g8SYqb1s5+HpKaAiCbODBa3Xz4l0S5UxUqKbtajeHzKI31fUzNjgbY26DfU+Z+MbVAOee5I3UZ1Q2jhV5J51ewn8L+R9j/rT4oSSDzdtBL8njURPZFS15kZnX6hQMRQv+HC4FApxq76Ix5zFbK+y95PjD6SzNb2WGjksRHob686qj6fVMWAue+pLp4ytqaTETfYM8qoeOvcH6I/8aplHJbx3r9Xn2cjsEvA6f/Tkay1Me18Az7h7aZKquEGwKVwq02xHwMHHT2S7hit1lp0uU2MjL9hcbsAPsRpa/jWmhkJDuqgVNvUtXQvwn6k514jX1wbsxmDyJ33GqjOlhSuVXlkbHe4Rr5dcN6ANcny2MAVdGAzBy3lbCXoIqB7xsKRF8qK08rMtlvLbS7rMbecTPX2NmKOd0TjCzkWlCnOa8uCOvJOTdioDze835Xf0wjfEP5nJx6i3N+++nmf6Pq+QNQgTthnpnZraP1WKDdDCBzA3vgusNewX4dwe3Zl9mIxO+4jvT/izkOKj/PfqQAWb+Rg2qOX7KAMXHhKapf3VGb0haQe8DpyN5FJW5wR9lpo81kWeVYXUD2s08g0PYKsrndY56KHPbeHPyKrBgL6UJ1r72McatRPVWRQya6tPZGDqSar+8Tje9WETUeuC5ePX7idQCarCb0bwGM+ZKqvDBBbVGVzx7WvR4GRn1iHi9zfv8qdZ6jkox/zW4eu2yLes4NaMU5JH01+7XVV51nsajvMVuJq3+NpNCsDVnO9TV9rasvuY6gtrxH9tzt7OnnppoIjAZ63EdFkvScJy9g/yVHFZVqUs/Tfk+qz2vkx8Cx583PG/DzvWpJnof84Ee9m/NUzxXAfdre3/Fa9zX02L58B+8z6bz0Z6Ushc+VhMJTXI8k1WPiHCpIJCeB8N6e96fpa7kW9X2M6787IocAU9dTmRWk+K6yd/H4/n/h/eqO6UJfQx29/8gYP2cv5xSjUqQ8E1jclvH/bdXya5SlUgHVamRN55zCk8CqQXxerz5V87zGfEHlUeFp9XrR7ymgy601/1aZDazoA7QeT+eQIW8CA4TepGc/ojqv8y1m1WP2Tjqx6DGzO4Lacw2L6Ed1TPzXwJj/uNacnJ2yArzoLPdRURNrrv1J86lCaT+D80DUBMZ9kopylDMW2fNb9qN2Xw8c1eyX12K4OfbSsekK3tcjPjSPVRXwGk6cJx+7yzmnRQlxVO9HgNjPgAOP8ppJPe623wRUZrHHnjssPlQk+iqUOOdXcy3sKqjjLX7s/Zn4vXcuLp4QGM3Pcfod3mfG2DLvAO+FGUIPsbAe7Pv5oz8V+ZIDQfd7qO7LP8I10QnN4UC7w1dAOzcAGP4e5zD3HIl+X+z/A8/PqKbzCQQ8hZadbgLaX6Pev3pCSQKwrDsAC3C7kAvbcjW/l5GfAGEPm8dLk5iX7PskMMytp71vCPtROmzA9Wnm57HXQ2Ylvo42l3K92P8Qlc2TfhLOOxE4vxJoM83V99vHn7360lZwTpPiHItVdvLS0Xoc565BL8qOYIFt+FlLk2QHNd8gAEHMlUmY4JwbHHbzfqEk0aliFvYtHa4BZiv2WoCrj6m0ZgOcq6qL5B7o5enAYudnuc1u/n5Kk+gcc81Z9tY2Yn4we8deInzmwGjmx4rj+N/GPblviLovd0kic9WTfjKvj40FyQmgGQ1Go+1tvYSHJ7wZzbgIoMuz44VmuN3uBG7MYkL6rLApBrgZ0u12jBj0CpOFUsKw8AywaRoT6SYLs0bAuX8zmW5EeSaw6Uq5CXRdoFuYpC6Wx6vyGTCUZ5jH8o/S9kGyQWlq6PYCxXHmMZ8gBgOeLFyaCp1vBibMkROwQR3Y4LrlSLlot/M2BjmqxOFcK/CDhUU/I6y+LHBLdjZNjfivWNSQ4BtEC1qVNeGID4F+f+am7deEoljXJvy0oWhcXcLESP+nzbYydUFZGp8ZPQBvTPT/Mxu3S5i8ihYMqg3Psl7A8j7yMxXUFhj2dyZgJbQaxQ1ehTBP2KvYcH7XnQxOLyQmzqe13vjvzGOlKcCJV2nrVSE0/D6/iglB1XxemgxsuQZIFDa1A/7KRuZtrzAnmXRkbiYxwIiEb2nlNOZLYPS/XX+3lbPx+pkPOA/8oLJgHARkrGPh14iSRJJWtt/oSgI3FqryaVN26m1uYN1ReIYJ7R23cqNpRFgvJpXbTJGTTRYLLW6iFZapO2cDe+8locKI4lhu9DrdWP/PVBekr+VvyULF6kOLTR9hA9pmChNZJYoigObgs+MQkvAA/+4XYbZ+AViIiRzIgrGEojNMgEl2OC1H0AYodTHtooxoezkLYzsVyX393iwT7H4rsoDN04EDf5KPbSqE9wZ6Pkgb0TLBErXX75lE23WnmdzU4SpgyjogYz0LqPWFfyRtoKREOgB0nAlMWUWboAF/qzlWfh5YPx44+y/52KtOAFNWMyYoTeEzqMMv3FWoURGydswCtguWRgBJbwnfM/EsIfEH4Oiz8vyYs4u/uwqxtKYBSfNcxRx3hHZncbDwJPCjr8sqVEf8f4BVg1lkMlot6Tj8FLCkk2xBdehxJt6MrwswwTj686azpS9JBHL3m+dGgJZu/q1oOSfBvwWJGyGC7Y9uyZe6RD522Hv8TqV4uDSZxZiBz7MoYUTUBCBlkfy8thrJ+/bqk3KR0V4BzA9Xr1PHXgDWX8ICrBF+EXzmrj4tn1dtGD+Hn7udULADeF/OKjWPh3YFxvxbLvQBtDirynda7wr3V1Esn6eUJfLxYT2YBO840zxmsQA9H4TmEwiLJzt/T8g9AMz1I9nHiKB2QK9H5KJb1AT1/NR+OuemWaVykrXjtSxa6OugEWO+5HqiQlUh5xppX9PvSSar04W2DVZfPhPTNsjXK20V76MbMoAg4bwr80hIOPMP85hPAJC7Fzj0mGxXXeEkAqrIUpuncw7aejWfeXf4tyB5qc1UFs/c9xA+gSwutp9u/q6T5gMrnNfIaCvoGwYMeJaF6lH/ktd1iw/tNYvOyOS3bOecPUmIG/3C+J6Rg1hYLDxZcw69PoXrEMDE+PabuD6cfAPYOBlwGNaIQ48BJ1/nf9srWRT1b6VuJ5Kzl7Gycd9cmkRL/DPC/a4jYz0Q8w/GHkZ0v4vFWlVs2OM+5n2k8+p0I+PWsx8xNndHZR4JyfmHWBgxwmJhXHr2o5prtvvxPkGyTafVh/uZ2zVaXTc2ej8CzIwHwvuZxy7fwTYOqusEC20npRYCAPMjg142E+KtPkiPeogEuC0zzC0yqvJ5rc+v4PU0oiofWNoNOPai/L6DX2GhrR5tM35GUQx/q0if5c44V3rmKnMpMPBvaSad6NAteI1tX3SsHAAc+GPNv3W+BRj5ETD+BzNxo+w8sLQrY7sDf+A8pkNzAN1+A1yy0Gn/KMRntlLGZvH/5X7TiE43cL1X3QO+wSzYXrJAbn1Udp7tbML7ygKD8D7A+gnAUYGssKwb59WYj81jVl9aYm67Hig+Zx5vOZLE2wHPyue993fAujEukrc7atyPQjyTsgTYcpWroOgOiwXodhdjVgkVOSzcr+jNfb8Ry3oBu38rr4+5+4Ddd/H3hUDGBmBJZyB54YV5v2Y0GZqLfc24uDHiQ2D6IfZaMsI3GAiMIivtvJC0yT/CzUFPhXpi0PP0Y5b8oEO7MlA/v5Ibl8ZG/6eBIYI3uFbNxG7cl0z2GJE0j5vtXXeaA1KA59x5NlmMElIWAQcfZVLACL9wMtJ/iV5rN+WQXRYleLS3HsvEcJ4H5uYvgYi+LAL4hsoLd5dbgV5/qP11VOq+XwotR6p7sRWeYqCSKfQFAtgbaOdseSP2S6Iqj8UewLwxqcwBdt8JJM6Vj01dxmdun2Ie2X0XN8FNgc63cLMmwS+cm0NVkrQkjv05peJVQCsmnlUJxRZDgNmlsrozays/c+L3TBT8WhDUjuz8K/cDnW8yj4d0YXFe1RcjdQnne6lYE9odGPgsMHWtuSeV5mC/rI1Tgb1Cj5TLdzHRGTW+BvsVVfnA1mvY/wvgWiXBvxVw5T5ZYVAcS9WEStXVEFQVcG098oy5b23uPuDYc+zllrHBfGz76ezVMW2TrKqvyGZxtFwoygIuhnQ7gezg3wro82jTqZ79I6kYkxSNyYvYV0hKaANMYJ35UB6L/xqYFwDsVPQLmbwcuKVAfV4dZpL1KyFzIxNkUn9hnS1ecNzVa9MdfR9jETt7u3nMHVLMkL6OSeGzwma9qZG5hUz9AkXfl+xtnKMkBEYDVx0H+gnzW1PCvxVJFlETWRg580HN8YCWnNc1O1V8+9yY4P4RfKZu1+T+O/lHqY6TCkxpq4CfornOqQqcu+7gOiYRdbrdzULkwOfNYwCw81b2oStJqPl3qx/npbaXs2+VxcDsjprIZKGqJ0vBcT43Aa1khnf7q/nbmDzTkb0LWDtGJiQ0BBXZTFStHS0rYC7fpWa6V+YCq4Y41X8CfIP5vajUTfZy7gPyBSW6vZLFCKkwD5BQkLbc1QtOQkR/2c2kKp/971TI2Ahk72Bx1ggff6Dd5d4TosJ7cf2Tji9LY2IuaZ6arLX3Piq2jej1IJ+n2zVz/zmAapyURcB2xbybsphrv1TwBYDRn8FxcwkcVi+JiulrqKKrzDWPnf0n+yxJPTIrclj0MhamdFj9eJ+pFB8jPqAaToKjkr2Mzq8W3jcLWBjJuUaaR3zDuFbp5AF32Cv4vEtEVICkqeMvcM8vKcpjPqICVCJ6VWRzbrx0uVwEDetFZeDVCiKExYc9H8f8x5zQDu7Agm/bqTx3971gaTILW/UlP/qFsg9zj3uoBpJU1xYLi4yDX5MVN93vZuHeqAA3ot9T7FeoKg5XFwMpPzHx3fUOYMRH5j64Y78FhjpVTpW5LAieflcmNuUfpRpntpC/8G/JYtqpt+h+JGGYsxAoFQAAFuglgiPAwr+xr6c7hrxBVZaxH7TVjz0Lh38g74kAIOG/XD+lGKvbHcxxtBGcFZoS2TuBn9py76ty6ChNkglA9koW4jpeJ/eqBICQriw2CWTozKgH4JiwgARHY+Fr993Asp4sQl6513Qszq8FShOBE0LfRR05e1yFu/qg9Rjg8p3AdQny+HXJJDSH9TSPVWST/NjzQXNvuuoi4Kc27HNYFGPuU5i6lKrR6mJzHKRD76PmDqsv8zHtLud+yr0IWZkLLIjgmtt2quwqUpLI2GzPb+X9KcBcY9oK9X7ME85+DKwZwVyDXeiD5xvCeSpyqHmstzM/VibsLaqLmbtIWy7HHYGtSQ6R5j7AVYyTCJJhPfmc31Ik59/aXsb1L/+oTAIa9y1zYMuFOPzI03QL6PWIuTcjwGJywn95HxkRNYFuENm71TFFQ+FO3rQGsu+lqh9yMy4aNNt4NuOigMViQdu2bWGRJt6WwoQJcHEtTQGuOcUkihEZG7j5VNk/OmwsAvgEM7h2R2A0MGU9rUxUDJ2GYOhb8t+DOwJTN1JVeH6FWfbv49w0Jn7PAqiR2eSoIjNIskQCXAUPiZUU2o3MN5VypSkR0Mrz+ybP5+IvNd3+JXHkr7T4u02wg+j1IJlU2Tu5iBtxq63eNmgXBKM+lQMcgCypxO+YYJAsBdtd4bTni2zSU6w3Wg6nKiyog7npd2A0iwr5h+RjM7fwd1WePN5lFlm6+UdlZpsCHue8uqAkgQk4lf3aTbncYEjMb4BBtE9gTSsSHSffZHJl2lYzmy+kM4tmHWfKwWxT4tzXLDANedO8LvhHyAk7HS2GsDh28k1gspD8jf2M7FfJ5sJRzc2vKkF2/CUq2EYLimyJDQnUfEZu98D833Ezk3Y3CgzFVqOAK/Zw7m7sZy6oAzD0HRaFjZuBdlcAUzcweScV82pD9g6qEQH5s08REok/n1cbFgjgwY66IZihmAcAXguA19rYcL66iAQPle2bHqNIJB4AOPwXPpNX7jOrCquLOW+F9ZKP7XonE63F58yFE3sF0Go0baZU7N2pChUHANyQzqSlFDO0GsVr0f4q9fFuaPCcp8NWyiRDqzHy/Hb0WZI4ZpXBxNwtSSDjtv0MMrAbGymLqTxoM41rg7uyzD8C6PeE81rFmhPTRWcBaM6i3WP1m1/PfEBbccluyz+CNnlRE9XODR2uZbJVsg8KbK22VbNYSBDKO2C293bYmLTrMgvoIxCfoibwZ/UInttIgypHczChOOBvcrKy663AmfdkokTWNmDHLfyuVZau3qIslb/9wl0xuTtCu5KQ5xsKjP605pjFhw4aBcc4j/SUCCIeCu+5+5jcG/6BeQ0M7wVcto3W3xXZLIy4Y1FL3lMzBUJLaRIdUlqNYWLRaGPnFwmM+y8Z/BKmbaKlpVR8ApjASlvOz1tfQmH2LqqSu91lVl85KkmQtPiQ5ON+3mXngcN/BpJ+oPKqvhZWLUeS+GH8HnWcfJPfdavR5lYQALDvQVjC+6Nt29nezXkhXYHQnrJyMLwP41Z7hfla/eQ834Evco52R95BKmY6zmQR36hSLz7HWEd1nW3lTFhKz5Q7MVZKaFv9gBlHZEVrSQJVvgALb8b7aNDLJL8VnuZnNxZ2z35Edb3REhBgwXb/Q7QJlewZfUPU9y0ATBYUMUYceJR7o1mlLtXtqXeo8gVom+mu4Osyiz87bgV23W62hqwLAqNZdAhqK4xFUQVUeNpMfitLA1YPY9FiyKvmYzM3s6AXPZHEkts1V9uJdpeb/7373/wj+ayd+zfV1UabvbjPqY66MRPwMcQjAS3ZFqM8TV2k7HQ9z0kq/JckUlHj3wK4Wdivxf8X2HsPMGmpuTBy8HF+bqklgl+Y55gUYEwR1ltdfFjYinu1aRvNY0s6c2995T51vssb+Lfg869p3MMYbUJz9gAbJvFazzbs+auLSaLo+XvzGqZjzz1A1hbe8277op/jvKhooI2Q++h4nSuWlfZTrUay8CrNqQALJUeeYcxyqaBc9QT/FjVJl0Z4cvcJ6Uw1ZHAn85jFFwjrQ5Lkir7AkNdrOjtkbeccNTPBbHed8B3jxlGfmveLgdHAdAVpQ1cnxn4KdLnd5YTmjqB2jBVspXJBP20FsNUZE1663PxcVOYBi5x7l1sKzfNnuyv5tzUjea9dvqPm+NpRfC4kW/CRH/NHQsExEnCHf6DOYSxux9eW4qU+f+K1ltxjrH6eY//IAVy/Dv6R96oUE4d0kUkc7aczlznoZVl5evUJ4Mjf5P1UcAc6TcV9oc4RNxYsPpzfp18gFeH/OBptb+slmot9zbgoYLVa0batELAefY5Bx9C3zBPn+TVUEU1cIC9ynWc5N2sKuf3aUVT/9fuLnPCOnqjukdNQVOUzwDIWGQFuZEZ/7uqP5o6O15KtV5osJzgyt5KFOn6O7GXd8/e0/JAsygBanrYaA0wVbFaaErpNa9fbzcFWeQaVB5LP/i+J4nPsbxI1Ua3O23UHA6CpglXFr7HQBzAIkDbEAD9rzwfVfak6XM2fXxssvky+tZlKqyp3+AZTNaBSK/b/CxP7EtMP4POS+INs3+IByjmvrjj0BINhlXd8QEsygyWUZ7CA02W2PF5VQHZiSbx58xHeBxj7H+/PuyE48z5QeAIIiAbGC9bOZWlkh7ccISfoAlqq+2p46gl6+Gn2nuo8i5s493vBYgVmHOXGIkTYBJYmUamzeQafK33j5hsMTPixdm/+iIHOYmOVOTHnH0mmalPAx582shKC2soJJh05e4Ft1znt29aYC4J6UbyTwnKwNmy9FiiOYcKosbH/DyxwSn0zRnxEVUxLwYquJJ7KqcGvyX1aO1zLPoAqFmVZGtdmqXhQmkwV6MAXgcEvmcdtxSTEtJlijlmKY5lEHfSy3Ccu5p9MCo77r7yOebrW4X3k11SgwXOeDs3B/sTdfisX00sS+F1Km+3iOODo3wBoTDg1dm+MojNUP2dt5fMp2UgGRgM3CgqWrdeyqHXVMapr3GErY58Rqz9jQOP81uM+Xn+j6gJgQa22vq6ekma2ciawoibIxURVMqoyG1jeiyzpkR+pX99RSaKaES2GqBn4P7+3wvGhJJGFvklL5WJJQxDWk2SH0B7yPVaewQKUVDDxj2TB4/jLctHDVg5suISqxcEvm8d1q2ip/yZANdiRp5k4Nt4jnW9hoU0853SqcQC5b5BvEFsnqOATAAx7Rx6zldO+FmBysL7Fvl13sGgc2MZseRjanUnS/Q/R9tK9z5ythIW+/k9zXjaiJJEFclsZ1SvGZ9ViAa5WKIcBui7suVutMor7AlaLFW1v/aPnvkkqdPsNfyREXUJ1Xc5e7gtrHHcXvy/JFrAolg4G51dSLW6c2/OPAAceAcbPZfHYiGPPcY6RrBLDelABrFLfnXyD99jMc2a76oAoxm15B1mgMhbfIvqS4LHjFrbvMN7bk9fI+2mA7jBD3uC9IvVK1zQWNlqP96z8klCZy8JDdQGVKu4FlU7Xs3BVXahWL4d0NquDbaXA1uuYNC44Cgx4Xk6WrxzAvfHk5eYxewVVRprNXOyz+jPxH9iG8XLRGVrg6nmB3Xebiwx1TWL6BlPlF/OR3Eu1292MwfMOyXatvsHApCXq18877Oy7K9id67bvKmejc18wdggUYhBHNRVGRTEssEv3774H+e/GfmUe6/dnFh6kvntlqQAcarKWbsGfvb1xi30R/RkTHPwj1wtjLuzI0/w8Q14yH+sXBoz4B1W8oV3ldjdhvbjO2StrrClWqxVt4/8CxFXK/RN1olzOXt6Lxs8c1pOuSSpkbubv7veq/40Kmkb775CudLUw4uRbwNG/ApcsMs9xvsE87vS7XJ/di9x67/ay89zvtDYUFAc+RyK+1Nag8BRt9lMXq2MlXTHs/hz6hdKNK30117nwPmYVZkBL+XPq0FWwPR+Qi2o17mdhDmgzmT/l5+XP1nqCd64zId24J1e1esjeyfhOpZoOaqvevzqqgXNfct1XkTO7/5brhlHlC3AeCOtjjtEBxlgq21uAa9BEwd5Tx+BXWKj0pqVMXeCJWNwMr9Foe1tv3/8Xe+dmNKMesNvtOHfuHOx2g+1DyiKyYXbfZT4oYgCb0zpsLqatO0I6s9eBdCzg8nI2Kn0AJu5W9GPA2hRY2BJYICSd7VXc7LQaTXaJhIj+DJKlvkH6oqqyseh+tzqpXV3CZHRI19rOvvGx57f8yd5hHitPZ4PpgmMX/rw8oSqPmzIVU/74y9zE9Re8yn/N2HkrExgSfAKA0Z+prRB/rdDstF7o86g8Pn4OcP15eSyoLftlqZL0Pe8HZpcx4K0HlHNeXaH3T1IVZuP/C6QKSQAAgIWJMGM/Kx16MVflHV+eyb4ckpVwU2LCDyRn9BQsfqqLgCUd2SNCsq8qPMUkgaovRrSzaC8l7lqN5u/k+bKdYYvBcqEPALbOZM+XNlPMhIVDTwL7HgDmh7EvioSeDzCBe/Rv5jHNQZb+otbqXnHeQnPQDqxasIYEmEwqjmOPHiOKz1K1VHxWtrkL7c5Nh1SkBoBjL9FqJFHYFJVnMikiWSc3BrK2MIEhoc8fuaFvMdg8FtyJBbUIxbpt9aE6SpWQ7Ps4MO47OckU3IEJqvzD8rER/ekIIG0yA9vQfqsqX7aDy95Op4BMgXEOAKfepSWcaq7wZJtlQIPnPB1+YcCsErm3GMB5YtpmIEeYv1qPASYupPrvuFBMaSj6P82i7qSlZsVjRQ6wejgVHxL6PEr7Hwmag89SZbZsnxw9kYWL/Q96d97VRfKzDDAxUpasVr+r4BvG76MkgTbYxmc2dSntTMd9q1YPnH6fiW/JIir2U5IojD2BAN7zt1Y3TQ9qvzAmT40MfR0bp7B4cdlWebzFEBbTxF5cFu4BzivURJ1vZkwpxem2chKbhr7DxLMRY77k95EnzCMthlLx1/U36h53iXPl/joAi0QZijnE6sv1r9cjZqVZXdDNuYdTqWZaj3V+ZkNcGtYTmF3JQo+UPMvaxrYGR/8qk71sZUDGJvX81vUO4OZ8WZ0JMBGrOXDuXFzD5zwjqvKolJTW/XHf8t6TVPcdr2O8O/g1mfSpJ1dTFGtzmymyvTbAGDlygFpB3moMFc9GBTDAtfGy7VS4dbzePO6w85kb+ILcrzKiPwtXkq1zi6FM8C7rIY87qhljJQgEMoBxyJpR7CNWeKrmmK2U82NoT6pU3JPjbS8DJi2m6tWYtC46SyVtr9+bVWOaxjmgNInPs6pQHNiGDkBSzJrwXxaRugnxbmAUcNkWqq0T57IfmLt96vC/A4Ne5H9XF/EZKE2mknXlQLON+bIewCa3wkerkSTjiW05RtMSVLLGrSoANl2pnmMA9i07KagRAVr3TZxPu1UJ5Rm8F6S+oaP+CXS6iaQWo22gw0bnhbgvZPtkgCqu9jNk69LsnfweVQSNWwppLekt+c0TWo8H+j7p6vfrjv7PsKhv7C0McK/f7S7uf1WtbIa9B9yQaSKP2O12lOWd435pvaJYA7Bwv1/RHsMTJi1l31FVCwtPKDpN54VDj8vjR51FRolMr2nMNcb8Q91WJrg9CaHGfIB/JOP42M9oq+6OoW/S1jZqgrmFir2S8eLO2cBcq9k+ecoq4DY750XNC+vHPn+iM9Xoz+WinF+Yy+paJRIAOPdJyvkR/+DeI3mReWznHbQ+LThhHgtuz7WqIlveg/q35PjYb+XzOfQU52wJFZkkQkiW4wAJndtu4PwqEaMS58i9JnXkHQQ2XSH3SDz7L5e4wYisbVRcFxzzjhxUX5Sl8d5yz5eUnafy8Kxib9UMEY22t/USzcW+Zlw0KC4WfJmv3EdrLHe2po7IAVxcdt3GQEyEQ52kujGDC5gYMFjJdjv4qNl7uylhL6Mi4uw/5fGiGFoYSB7rgMsKU9Vb49ATwI++ck81zc5ARvsFJquBL1DBIrEfw/swCaoKlH8ptBwBzK5gIC2h8CQDkl/bedcGi0Vm2V/MsPpSxdZeSFJUFbBIdEKR+K3MZfCTrgjuVg+vudGtB8Q5r67ofBPnL5XKbs9vgW0zZc/6oDZM+EhJFYCJwrFfy5Y9xXHA4rbAaqct5oVE5CCqsCXFtcWXtibBnWQLoIosBumqPg/F58juThaYqF1vpb//zASZHFKZx74IK4VCT4/7acc8+jMzE7DnA7RVs5WwL4oE3xAmbCQ1We4B9rWozJV7WjUE5Rm0A1sQxsKeO7J2APOCqNo5+br52E43U3U3u0JtweIJeq81yZrUUcFr3V2hWm0ohr4NjBGY27UhoBXJKnsUxeS8w8DasewtJaHVSLWKwz+SherSJHk85iNaf0uxSmA016qYDxnTGDHuv7R43H23/NpH/gIcekwm22RupW3WcgVjXUCD5jx3xH0JLGhBeyQJh/9ChZQRfuFMCvb8PYsEjQ2LlYnrjjNlUkx1oXMu+tGceO79MO2/q4tJEoh1K4D5hQI3ZFBFrFJGlafLfVdKU9hT9AcLE5YSFkSw35aEyEGcwyYp1H8/WIC5vjxvd/iF0pXDJ5D2bUZ1SFWhs++owja8Kh84/CSLmBCK+0WxTNJLc6fVh2zzhO8anwgBsAfcmtFyIqr7Pa4ClRGag9dAFU/41NJHpf01jJclBXBVLr+v8vNmu0sde+6VE+0+gSwOjf9OJtPYSmk3eFDoCQnQBm/TZXKi3urHgseof8qFmtow+GXGOtK6bytnIaL9dHOvQYuVpMicPUDBSfOxbS6lKnD05yzMGVGWxnl1j0JB4qhy2uMp1t7+f4Fj1JcoLlbc3w2BZqfVr6qflgq+QXRQGfiswrLXuV4EKpRovR6mDZ60FjnsdEHYonD+aDWKpBIpJtXPLaSLTIjZdz8JUf2elM+7LJlERalHNcBiYOdZsiLMx59qQ5VK+PTbTO4HtAKsBjvh4I4s+Er2oZ6Qu4+Fq3xhXfULBW7JB67cQ9u3rrfJr9HnMRbAJXVLxABeK9U8pKPrbbQYdd97d77ZRfgrOstnIGku5y7NDpOFeqsx7GkIsJC7ciDJNCr0ephFACM0O5PoB/+k7isa3ofFM6loG9qVhCeVRd/0/WoCBkByzoC/medezUE1V5fbgRkH5WOPPc99pETEaT0OGP+DWnHoF85nqrFtxcsz2Fe57TTZAaH9DO5tVPCPBG44DwxXEMR9/JX23mc7fgJHp5tltVfij8CWq+lGoMqfeIJPgKygrAv8W1INO1KRY5u4kH0GpbxNRSawbiwVf70FUtbxV2WCIsDYvOgsVZbxwp7dJ5AWmEZlsaOaCszcfSxGuz+n9io6FsV9TpefJIkcmQGsGsoYba0Q71qsdVftSjj3H76uKsayFdPSN1fozZixjrFw/NfysQnfkbwr9fSL6EcFsFHVriNnF+ds6Xr4hfMeUBHrrAG8TywKdd0NGXSJiv3MPBb3BbBuAucxKc964BHuESVilF8kVa4qAndjYN+DvBfsFcz3HnnapYgGgIRvgFNvNg0R8n8cjba39QLNNp7NuLjhFwZMmOvhH1jIpo8caB6K+5I96iavqf/7hnYFBr3EALcpPHhVUmrfECaHz/2bthpGe9KCE7QMOfIMcHOBeUMf0oVe9apFSk+qSFZ2fmFsTtyUC40Kg1+WLYsAJgeO/pU9pK7cc2HPyxMsVl6HvINUexgxcf6FP6fGwNWnav83/0uw+pNNZ6+Ux2M+YvCTtpT2aUZYfJlEr8qXEwm/FIb9nYk/qyJpuPVaJgQu22IeK01i4layu9GVWr4h8kauKWErB7RqRd+VYCosVYi6xNnnVMFO3OdMcKr6j/mFqZmNSzqyh00nQZ0n9arSkbGec/rUjeqee+dXO9VZwqY8MIr92rrM8s4qxRNqJNQMa2BIZzL0S87xezUdGyQzInXkHqCNdvur5f6J4/5LNZC04Q7pAtymsG5pDHiyIZ4fxsLs5DUycaDzLHUByerHDW/mZrV9ridcpWCUA0yidp6lToC0mwGMi5bVPj4BwJDX1AWXYe8CqctkZrhegGnrHdmhQcjbz+SgNBfkHWJxVLLuc9iZpBrxQdM0pi9LYwGrxVDzMx3YmknlwjPAyn7AgOdkYoK9EkhfCwQbbB6D2qh7gJ54jZv3KwUGdUWGy/rKmzlbt2ryBM3Ozy71Uhr7FYsqRnS/i9aQcZ+zcGLsA1zmVNsPfFHu2TfsHdorZ6w399CszOXafebvwLjv628d6QmlSUzuqtD/L5y3M7cI35uFCRcAuGIvlS5GjP5MLuYB/KxbZlCl1+N3Ncf8W/KzStcAIGnJViorORzVLBL6hctxjMWHySjJohhggTBrC9XcEqqLqW4JiJJdSbxFVS4Jkn0eM5Np7FWcJ9ZP5H7KqKAK6UJllQq6Qi1Xse/Y/xAVVFeflPsh+beAFjUeKFXElg1B3iHe251uMNvBbrmae98hb5jt8GylLI4GtVXHq1cqbHkBFk5338WkuLG4WppIFxYV/MK5r5YIU+UZvK/bTKWVmbGg13oc50WpFyDAawzI7SvOr6WCYdg7agcGT/PDmC+5bkjPqsXKef7U24xr3N0KYj+lgqTTTfxM7vdI28uoPs/dz3PrryIse0C/J/gjQe+HKsFWxvdsOYLJclV/PID31qh/Aa3GMhE9UCjiuVs7W6wkuhaeZFHEaMd6/FUg7lMSuY0IaEWS2E/RVBNKTgU9f09Sn6/CIWHHbN7Xo4VEfOFp5nR6PWy2tMzczHVy0MtmFbDVj6pn1XsCwDknQUwiM4R0Bk6/RzWuVCRK+J73UPurGlfRU13I9dE3WN3/7uDjJHAarWAdNmB5b57TKEVhrBZo438ErMLnKU2kknrI696RAfOPUAE1+NX692ENauu5R7XR8tYdvsFAl9uoyJfizhOvsPge/x+6NLi3O4n5iITiUf8i2cwdJQkslrUaYyY6+AQxTgiMFpwENBIjM9ZTzSoqqi2u2N44XwMsSB35K5+LqRvM/Qwdds6tIZ1lS9bqIloV77kH6Hgj0NfgnpTwHZ+pfoKK7rpEIGWJHK+UJvH7bDFUbYF86CmSe43vCbDX387Zcm9Zv3DP90DHmexluP1G4IY0877KL5Qxq2+oOX4oPsc16voUeU8+8hMW/AqOm/e3LQbTNnTnrd7tEeuCsjTXf0cM4Dzsvs+InswYb1Bzse9iQrOyrxkXNwrPyMw3wCmVvoyLgdQ/ILQbg2yVpUjcv4EFkUCSoigz6EXgmjMXNoFv9aPNXORgeVPTZgqTu1ET5MRe3iGqjIxSfx0j/gFcc1ZOxFqs3AipGMG/FDQN6PFA0y1+3sJWxg21ys4BoO2LSjHxa4XFemEsBH4t8A1mEJyyUB5vM5VzzJC35PEus5lkLRRUM02J0+85lRpfyuP9niCpQMXcHPI6e4xISF9Ndm25YG3aYjhfd1aJOvHcVNgwiQqUIx76Oahg9eF3FickAQAq7DrdJG8+U5bQ1iN7p1nlBjDROeJj4BLFPQTQmvLMhzX/NuBZqgraTgVaDpePS1vq6qVkRGg32iWpemg2BP4RLvsWY6I9pDMwbQP7aUnMyupibpZSfpKLSPr8orKqixrPHjWq4tXhPwMHFUmupoSunJc2kGXngVNvqQtIkQOZMBrxYf3ftzTZaSsusOEBxgrJ82kDI53X/odYpJT6WJbEc4OpspDq9xTJLFJ/sTaTeX9c6B6/ANnAbaa4+j+6I+5z9jAxFkMAKhQXRNDivSmQsog2jgtbsNgjIbg9k+XGmGbvA8D2m1kUvLWyZpJN09ivLXOzbBXmG6pOikQMJInnplxaH0mYXU5rLBViPmLRV8KMwyzQG9/fXkFbtthPlQoAQOP9Gfdv81BIF+DSlbRLlGCxApfvkhMTWdtYDOlyGxVcjQnfUPa1G/u12pZ37/3yOmWxuNZdSWkLUAWgKmroc6aK8FKWTGa5pHxJmst/Iyn3iuOApV3ZYuCMMM/4BFJtJPUyBagQul2jnZeEBeEkxdTXCrY2+LekrWTMh+bPXJXLRKXF13NRT/naEXxOLxUIKYDLNUal0tw4BdYDCvVAQ9HxeipQAoT97XmnTZzULzRrO7CyP6+zZNubtop2cQkKe7S8A5xPpIKdXwQLKZco7pH0ddyjSqQTzcEk/pn3gfMCQbfn/UyGbrlatgHteicw8Hm5cOWooCpHit3qgpYj5EIfwKJI3mHGiGlLa47psU/KIrMSMqgt18/zKxiXGpG61GWbq9pbbL+ZVvAqrBkJrB5h/ru9goWH84r7etOVwEZnIjowmqruuvaS8wmkig2Q1Z8BLZ2FVcUe078lcH2qWinZ9VYW7SWL2pQljIPiPpePTVtG1yS9R547Ev7Lwqx0j1gsfL+CYy7SjOm8W5AApyJLx30h39cAez1vvVatuvcWod1ZXM/a5mpb444993DeLBbcTiw+vJbnV/CZrSeCyk/DEv81404j+j/NeMObQh8AHHP2iW4K1628w9yrFQmuNX7hLPy3niDPnTMOswCZu99sfRo9iSSbjjeYiSEJ3zGXufdec+7O6sO5R7IMt/qzcFiRyZ7eCd+Z/01QG2BmLNdmqViXvh6I/4p7GkmhbrGQ7JKsyJP2fZyFrZzdshX2uX9zPZKK4L4h3OcZCQE6Qrsz9pNysGWpjO8OPSYf23oMi4k9vHSACe7InsuScCL/CHOp0n5/2NvA7FI1+bbn/cBNOUBPhd1+9OTa1dgNwaTFdCnyCWTs2mpUzTxO1HjZdroZv2o0K/uacVHAYrGgU6dOsBhVdDtnM8Aa9RntjdxhK6N/flW+/KJtpzHAKYmTA1VdyWH04AdozRP/DTcX0ZPq/XkahKgJahZ/QEsmd1Wwl7EIKsnHAW5ef23FPIBFC4A2ecagproQOP6CnLj7JVGaRBbzQAXDO+8wGWyBTeDD34zGRbffqvuQtLmUQbwK7a7kBrWefQyVc15doc97sf9S94zxBCnhp0PvB5S736yabgqlc13RfgaTTSoV3JrRZPJfsYfBvjtspXxWVQzuMR422gXHaJOSuoRWK0bGtKcCzsHHWDTN3sUNgHuz9KPPUqkY2p2fKayn+fiBL1ClkrxIZp2mr6fFS+8//Hrm9szNVFsALDIYWdQthwNXn1YnynMP0E61+z3mjbGj2pUcG/F+4553bbguUT1m8aHyU6WkBeQEVV1g8QUq0uUkFQCE9mBRTnI40KqBwuNAxWT52BOvMta5pRiwCtejKp+JWL3Q2QA0eM5zx5iv2QNFQo/7qJyQEgwBrTlnn3wDCOtFVWxjInoyLYLKU83fmb2STOyw3vKzXJYi24ABADQyjQHg2jjzetP3Mc4vBcfNRTffIPMzaIQnlWPGRlrah3Qh69mIFkMViTsLk0AAFQLG+7M0ifPi2K9JIjHCL5SF0ZJ4zo3G+6Y4lgUdqYdm5GBg+Ie0d2xs1XNAK3XRC6CFrK1Y7VYx7F0m/qS+QACTfp1ulNsX+EUw6dhS0Y8mpBuVtpKCfcZhdRExoDXQ+4/A2Y9lyy3AVWTxxopTh6og7S18g0lUKIph4cUdfhFU87cYKisS09cDm519E8d9J9soe1J6DHyeCUHpOgFAVT4sWZvRaVgjzXnuKEulteXwD4Fwg7Lh5nwWwqXCQ40YQ3CYCXK6ORQKLhYAE7+VObLlYGBrdR9VgM/r+VVUuRkR3J73Z+JcIFwoBgJUJRSelM97pIIIA5DgoNlZQGs50gPxQAFNYxK+6BSLme73v60YWOOcu8bPqXlcv6f476vyAH9h/dQ09pySCkzbb3b14JowT1a/pDhV3INeNF/r7N3MBUQKc6NfBHDVCSbRj71AO7oZh13raUBL87MEADl7+Zpd76gZax5/ld9pr4f4/7oNqLSm9H6ExaNt1wPTDeo+h51khuAO6gLA6mGc2yQ3Hf8Irr96v0EjUpdQKWVUVgFA36d4f229Bpiy1kxwK02ijWxIZ+AqgYB+Q6r8ngDnGc2uthwc9i6vQVjd7dDrBKsfFfN5++V7zC+CfcQvF6zQLRaSuJZ0YhFJRUYUYLFY0Nm+Fdb9nwNVWWYluT4XbpxG95gJc8wv4gm9HmFh1RsCR21YP5F5tNZjgXDheuTsoa3tqH+57ncdkQP5010o1rSdyh/N2VrInVDd/ioAFua5AlqbVZi2Mqr30texUKoT7ywWzjGdbuS8WI9r9DNajaKlafsZ8t7YYmWBU9UOScdNeTJJfNoWzvtl581rRtY2xsRSu5CQLnTBUKHS2TdUUgwCbDFQHMs9pLEPuq0UmO/c60gua0VnWYgc/YWc79h0BeNplSVwRRbj3sgh5nymbvstCS7KM5gr6Han/LqNAaufi0SrOXhvWX1rztWVefx+Q7o2rgvD/zAadW/rBf4fyTOacTHDarWiVatWsBol/zr7IWeX+aDoS4CZcWyOLTHjAAayJxWKnEtXMqiTAsPqInprb7qMVjAXCpoDWNwR2KcIYhx2FiJV59RqrHPxr2cfh18LdItAd/iGkenSWmFJ8kshsA2tEttNl8d1O5sh9ezl0IwLC00jwz5IkTjWHNwoSWoKgHNTdZFaQayAcs6rK4a8TrslY4JBx7xgFtGlXjW1IawXk4kBQlLEVgbsvB34qZ26h1hTYfArDM77Py2P5zntpyRFWHEcFSS5gn0QQLbe0u6yUrL/08Dk1UDfJ9T3SexnwJkPzH8vOsti4fQDwKUGZUwr53e8dpS675k1gJ76+ULBuTyTycpjz/EebEzYSoGf2lIlohk2RNVFwJZreX8lfG8+NrwPrSN7PuhKHhoR0VedhN82k2t6rqCa1hPOTcl+9AZBbZg4KRSazTcUwe253qgY1oWnaAcrJXdDugADnidrvEA4t85OddlmBdlhYUtgUWtuRI0oT3f2AhFUHgIaPOe5I7gDkLHBZfXojlajWHDedIV5LKQTCwD2sqa5Vi0G0yLo6pNmlYm9Ath7n1o1M2U15wlbGVUA7uxyi5WxRmgPtXrgwCO08zTCYWefzf0PAzmK+W/tOKqXJZQk8HcfwS4JAE6/L/cv8QkAZhXx9xoheZu9i33gfMPkgh0AnHgd2KKIsRL+S0WEtA6F9aC9k6Ruagxk7QDiv5WT4rn7nYoz4f4DmFgJaqcumgV1AIIFyy2Alru3FMpM/+oi4PATLJZI4/6RVBxuE5xQAqOAkR9xfVXFFPND+CNh/8POeEPxTF2fxhYDkpVYQ9HzPqrMjaRO32AqD1TWo37hLrtUyd62Mo+K6uNCcQrg9zzwWbUNYt8ngFZjG2/Oc0dwe6D/M0ArQXHmH6meI8J6uhT7Emmq5XAAFnUPpklL5WKHjvyj6j6q3X5DRaBqHWsxlMoI6ftMms85+9qzcs++ghNcp04rCEA5u+nqICmNasOGScDWq+ko4DBYsvoEu4iCh/9sPtY3mHGOMbmbspgKypxdciFv3HfsSTVti9pCub9TOSz1aS132rVJlr1WHyBygCuuiOhXU9UzYa7LjjR7N7CiL3sNpy3jGlNheL/YT2r2Sjz3H/YvVjkzWKwy6UCzkSS27yGzMkpHcSyQvV12jGgzBbhss/r76vskezJLltCRA1jsyD8CsZi8rCfnV9V+/uy/WMCUerj6hrJ4FKSId/s9xVyWZFvfEDjsJEnPTJSJZiM+lAt9OvxbUqVbz/7YVqsVQb2danxJlVqZx/imJN7ZA7KeaH8l13bpOjYUeh97yQLXUQ2cfI17fYmcVJWvFh/omOsL7DAQzFqNpN3yjKPy8zo/hPdW7L9I/HRHQBTJZedX8x40wl7F2H/HrUCsoHgN780CvCcb4x6/oyJNQsEJFoMdlXI8Y7HSnllyJdl+M/evJ9+UX/vUO8CyHnJ8F96fCuCBisL+iZdJhlGRVjyh4Dhw+CnnXCBg4AtUaEp77qKzjMG3Xc8CrRErB9LlQFKlVuXRHUa1fjY2is8BC8JqxjcJ3wGLWgEr+si9EpsholH3tt68/y/yrs1oRj1ht9tx5swZ2O2Ghb/XQ8DsCtqVSHBUU11lDD4BTtT+LdSLVIeruBmXKvEBrcmQdVSTGX+hYLHy/RK+k4OkzE2cnBe1lJty+/iTJaZivP9acZsdmF0pb/KsPsC4b72X4zcVAlpSdVKqWBB7/h4Y+w2ap+FfOSwWBlipi+Xx5AXAvCBXTxAj4r5Q2+t5gHLOqw9ajVQrNvTioze2WVHjgSt2y/3rytNpB1aRISc6f0lcsZuKMYnhGNSBGwOVUjtxDlCaILORfQKoEBn+dzmJuvcBFhJjBAuzKatoBx3U1lwQrsx2JdVU/VUCWpGFLSmI7c6ER7srG79/oqOa66pEtHFUuyygiuPM4+F9+LlHfyZbsFRksX+UyhKpxwNMhkvJD98QKv1/CaX3ideBFf3VRe78w0z0NwUG/E3u9wJQEZa9nfapEvwjmGS3CsmR9tOBLrfLFp/ukF5bv3dVPa0MaJQ5T0fWduDk6y6GrxG2Upk8BADh/RhzDHqp4edRH/iGkDQQNZFFVCk5A1BhsGUG7ZXcMWU1k4LStcrezR4c7sphHRUZwIZLaKdZcER+z9w9avv3Hr8DbnOwcCLh8JOc/ySrMIBKkIEvmP8eNRGYuMCswtZRmkR77YBWcpzezsmAPyokyADaCM8PZWK9MWGv5Pe55245Dr9sM4tPKswLpmJC9bxen6JWLecdBPbcKxdtLT5kY0skHYCJqNw96ufi59fxgp2sf5ZYoTcjwMJCU7REqMrnd3nkGfW/WT0COPAn899bj6Hl1u2arC6wWEkEUxGEUpYwcVuSKI8P/zvsl+1ovDnPHbkHGLdKau/0dVTNZGz07rUv20I7OgnnVwPLurMPnhGVecDqoSyOSbBX0DJQUi/bK4HEH9XrZ+5eFpFUz8zJN5kIl0h3pSlMZl+xhz2e6os209ivsvMtZjWuTwB7DwNmxWLBCRYfc/aYlVVBHahI9Q0xJ/AB2lV2vI7OIioiYbffsBgjqWU73cQ5W2p/oWksplUVMqa4bKvn9d/ix2eh+73sL20kaV25j+pDHfsfYl9Syeowa5vTnlmwhLb6AUPeJGnv5OvyuejkJNXeI3Wp2j67881UPUs2sADj1VsK5D1A3yfYe1ZlmR/7Cd9byhdFjaMSzFNvxKaArZjFkmMe+tplbARSl8tj5/7DAqyKiKOA3W7HmdxWsM9MlRW3BccY3/R7Cpj4Y71eGwCJZ7GfsQdjY2PCHK4Hqv7smZs5H0gxy/I+wOIOjDeqCmqOJS9ki53IQUDr8eZjrb78niUCZPd7WKS+IcPsjrDhEuDI04zNpFyno5LPYvI8WthKyNnHwppEnAP4fZcplKtpK9jDNXmBvA7aSpkLk4hPQ5zENCkusFdwT14SLxMDfPwZx1QJcyfgImBIexPfEBfhRUKbyXQ+UsXKff7AmHOH0CIl4Tu6Iwx4lipjI3R1Y7JgqRrak0pwyW2uKeAfSVcr93yJns/sdJPaeaIZJjTq3tYLNGeZm3HRoKJC4ZXuEyBLiavyGahM28wClxEl8Ux0qxiO1cUMCqVijY8/MHkFG9Z6suVqCoz+wmnHKaga9CKerbR2Wf3FBIuV3/nF1ivu1FtUgEpocykDpDRFn5tm/HrQZrLa+lbfVEq2CwCbapcmyB7/tUCc8+qKghNk6kmsXoC2PFfsUStPPSHhO+BHX1fvF3cERnHjesXueluXNhgn36B6IEPoDQCQPSuxpAHaDJ1+R630zj3ApLJk5VWeyQ2RKtGUtozJmykemn4XnzNvmPxbkaF4u0Z7UAmHnwJWDZQZsCFd6b8/aVnj2234RTCJM2mpOfnr34J9B2aVAYNfqv9rl6UBp94EjilYmYNfAm44L5NWrL7c8PpF1v99G4pjzwFFp2XFoaOaG6Rh7zT++1aX8N5TWeD1e5I9NCW7cnsFN7cj/ynbEgFMcExaIo/dWsXXlp71iAGMka6rOwO0QXOeOxK+ZdFOUm6depdz10Shz0hlHpMTCd83TbyRsphz1PpLzAleqy+Lqy1HUsViTLBmbmZSKDCaBLeO9XBoyNrM43ULZnf4hTNhNOQNV7LUiJvz+Uyr4KhWq9t7PUI7Loug5EhbwXMa+Kx5LKQTE+VLu7B/kBE+wRwf+rb8vq1GAV1/4yr6uSNnD23XbKXq4pe30GNv/xZynxsAWDVYVtABACxcC1TFVU/FtvT17FcpKb19Q0gKqciW45kds3iNpglFoKp8YPMM/htVj7DpB9l3VMLoz2nHqrKgOvkWsHq4WjHmLSw+LPinLDaz5W3lwKqh3AdKRdna4B9Jy1yp1xFAy8HkBepE6NaZsOy7r/HmPHdE9GOyXLK83HwliaGSMqHwNPCjP/uySQ4xRTFUcalIFI4qKn5UCVpAJpUAtFAc951sIW8rAXbdBqwdLReyB79KokTObnnvW3CUa4EUv+UfofK06IzZ0q0uGPwSVa9dbpPXjJ73MxHf1fDe6WtJhFg3znwtWo+meu7E63wuJHiacwHed/6Rsh26TmKM+Uh43Urgp2jggKJnd8IcFxElahxw9XEWOsN60IrQqOAJ6VzToq/bXU4CkhDTZm4Gjj0vE7QtVhK8+/+VNtwShr3DZ1IqxhSeoaJmu2LeTfwBWNlPtik++Djwg1VNWBn2NtD9bnkM4FzTYria0LBqMHDoCXnsBwt/sneqX98b+AQDHa4FoPH8jEhdRveq/Yr+Ycee97qPYEVlFQmO0vcR3o/WkG0U9se1Yf9D/JF61DUUtlKun9J6YfFlnDTmP/KxfR5loXj7jVSguqMik0SG0f8G+hnug8QfgWW92C9Vuk5jvwL6/4XuIUa1rKOa+8+E/8rES59g7ksv+UnuMZe9C1g3hgVD1Tq2uB0JNRI63cSf/Q8BhwQy2LbrWXCTiJs9H+DeV7JjtZezKNzjPrkAaq8CVg6gOlBC93vYa1eKDWtDQCsW3OK/Vv+bng/Ihf9ON5Lk3+8peb817jv2mJZs2H382fpH1cu+sREYBYz7hiQIHQOf5TW5ZGHtBNBm1ECTxHl1RHPPvmZc3Nj3IAO0q46ZbQhKEtnTb8jr5r4+AFUpN2aq2QmLO5D5NORNYIDACg3vC4T18W5z0BBETQQu2y5v4iL6AzflMiBRJRkuRhz+M61dJq/03Dvm14TyTNrrtRUYwQDZk8eeAzrPunhtVf+/wOpvZuLp6Hob0OVW9TwQOYj9JVSWNU2FtOVkl516U+4l5t9CrZioDVK/Hx1+4eZeBRcKOgM1bbncJyd7N5Pe3e+V7SO73sEChYQxXzKRJSHpB27U9QKYcW6+Ps1pTSQkaXP2MfGx/yGzz3/kIFf/BRWiL+UGoCLTvAZarPyxl7KQ0JjFC4uFyXRxzOq5h1v+EfZWAYDrksyfseUwknRUPSYqcviZgtrLz9W264CIQVTQXEhcto1KxyiBmWvxYa+u9leZe240FLYSbsZ7/0GdBFOhuogJ/F4PAS0FJdmpd6nUnbpevqbuPR6M8I9sfOupuiK8H59xycopMIrPuUXYAjmq2Dev4ChJHrU9f94ieweUfMvIgcA0gbBw8g0mhG4pkPuw6r2Nb8oxX6tud9POSSp++oUxYeQJqj6oAPuILOnIWGfqOvP4qH/yR8L2G0k4kZQcAOeZDtcBLYRCdWAUSXcq+AQA4xUJP02jHfWgl2n535jwCQRuSGfSW4oL8g6ysBClUDhduRdI+rH2PooinIUO1TqWvROI+YA9zo2KoN6PqNc4hw1IX8P/9g1h4c4ITz2BfIM9J8OPOi0HVYQZb+EXDvT/G/sdlaXScUOHxcq9Ur8/yySMslT2oqzIBEZ+ArQYYv43nghNw99nMj1a4fqQtpwzQH+F/W1DYK+kkihikLlf5oiPqXgQk6gVTBBnrOO9YCQJlaXRLi6iv+yCkLyAdmOtBfvQoHZUk6mK1SmLGEdJfY99w6hkS10qW+H5BlPpnLyAhScjrtgNQPG+rUYDoz7l8+CwMVaqL478lapA417OYQc2TqXaZ4hBDdnpBhZNq/LV9u+db5KtH9eM5DwCyDEUAOx7gESG6QL5qDKP+4Og9kAfg6rV4st4oNVoft/5R2hVr++9Yz+hqrr3w/I5G1GawhhBj7n7Pukk4gjEu54PkLiWtVXuMewfAQz10PoicQ6vobFgAlA1CjDhLkG3nJeshCMHkVx2fhULBVIsvXYs193JQjJ+6JvqIndJIuNDVc9m/5Z8popj1Q4f3sDHn4rWmH9wHfQ13IOJzu9jlGDBrf99120sSEtkHQ9olb8QPvOGkXBlfF6D2tAtIPZzWoR3EBxkPCFyEPsvqmyyG4Jdd9LlZ5pgB2ux0OLxxCt02jDaQw98ls98/lFz//Xej6idOSxWoDKLVsH9npLFC2XngeoCxjTu8XiP+/n8VuWaLYYBxieevieLlfdfj3vlPCoAXLJIjqUBFrQCnc+9rqZzR8/71c+FJ/iGkdyhis91dxUVSTswGuhwtfr1N11OIcdkQdWqOZz5AAVBaNv1gG84MEKYH1sOk0mXOkK7AYMElwuAa8nIj9Gs02pGfdF8xzTj4kZJPBNdpwRmb2g3YMxX3PTkHzWP+wSyT8hGRb8GvUmztBBWFwFzfej5fKER/w2TwyrrjoCWZET/Qo1AmwSn3yMTVbVw/1oR2kNu9g24khuSB3szfl2YsgaYobC7tFg9F/y73kEGmWTv2JTodBP9+rv9Vh4/9x+qJTwxg1XocC2ZgJKNJ8DNReJcOUnRlJi6Drh8Jy11JKwfDxx91tWvxB0VOUyeSI3qASaOz68E8gTFRNQlTGRVF5IgYoTVh1ZMkh3T6beBHTfTXs/I/E6cQ/upHyzAPEUvpJYjaX0k9cZz2GjXtbClmonvLRx2vq/KKid7F610ioUm6u7sVFUBss1ktep+20xgaVd1zwRrQNNYwtWG6EuYFJMUdhYr7c9GCH0bG4qAVs6EgBfX2C+Cib2KTLlPhL2MKqFzCrby7rupjlH1CU5ewPvkQmPct7RNk1Qe3e9mgl9i8Ae2YWK44HjTMMM73cBi3DUx5kS6vQr4MQDYrZizB75A5m9tsAvs7+D2fG52KZRVtnIWN1QWZp7wc+8Rhe1RVb46dhv7DdUSm4REUPp6ku4632xOSOvY93v2TZMQ+xmwpLMc/0eNY3+vrrfJxzYEFgsT26r569gLTO6qCqCRA2lh5Y2tW4freI9IRV3NwQLO8A/lHjw97+f9n7LEPBYYRSv9kf80r1M6dt0JHFVYwqWvB858qE6QDX6VigipoNZQdL2NPdiNyXSfAGBmrFptXXaeVq/ZO1mcrS/8woCOM9XjkYOghSrU1A2FxUqlnFS46vMH9qKXetu1HMbeUJcqiJWtx/K31G8IALreru6ZbLF43pe2HMGimKoFxaQlVBW0F+aK8gzaoo39Ro4pqvKB0++S8GVEUBundeBVtffUkhDzT675IwRLQouVasPsHWZrydDuLAKN/ox7dncUnqbaMGqiuUgIUA1tDSBZVEVYtvgxpq0Q4gK9UNhDyGFYfenO0f1uKmKPv1TTOWLMl8Bkp6tH8Tna0xedZR7mB4u55+LakTX7kLUYzHtEKpQHtaO674TCJnZZT8YcKpx4lQRHCW2msFfusL/L48EdWICU5sYe99Jeca9AsgFo2Ze711XYMCK8LxWWUjxSmsTvN1hRuLgxg4qxLk2wVnW5jfOuROYZ/DrJix2ulY+NmsDnNai9PO4BvrYC/keS4K6g4+CjLCzXF3rvdlVvxoagyLnfURWoMrewV7RqHonoz/VItT/Z95BZLdllFnDtORbCWwkE3f0PA0s6UMlmVMSO+QKYvg+YeY49GFWoypd7zLUeC9ycywKjpBAGWDz3tM4NeIZqW0mx3fU3tGKX7L03TuV8IvWos/pyfTq/Wj7vgJbMrY1RtHg68CfG2tKeHOA11FtRGJG5mX1KJWUywNesEvZSPx+/hXGtex9THSff5D0gvm4ObXdPC8XepkB1Cfci7nu//KOMuddPknMhzfhVolnZ14yLAlarFd27dzc3t5y0lBt6aVPsH0mrxGX3MmFl3ETaSsngihDYY4DM6Pj5hJxJmvivKbu/kOq+4y8xwdtltnlTU3yOCpEO1wC9//i/U/AbP4e2hP4ta/+3vxYEtWH/HBX8IsiclGy1mvG/gx23MCicVb/Cl3LOqyvCewM3eSiO65vWdjO4majXyfmo1ahV+S7V1swE9QahKeAXJquqdHSezYA5TEiEWqxUb6qKfflHqDAe8bGZmddqJG0lVcjZRyuU8L7ANYbiWJ/HmLDpfIt5Pr/kJ27e1gznGiYhqC0w9B2Z8Vtd6Cpo+ChsZr2FVg1scBIZpN4G653nM+gVYJChn2DUOHU/hLpg8CtMMEUqEsPXp0DJ4m9KVJeQPatSNTYmK7sGrOwh7E0PBZ8AWp7Gfko7xADDGjvoRaqMEv5LqyAjEr7l78oss51OUawryVeH693gOc8dR54Bzv2bTgfGzwQwMVkcw+S0OywWknRGfOy98rk2BLSS7xGLD9nPPoEsCHW8oeZcoyvQ8g7TgnLgszV7U95mZ9JQVCzaqPYKE4oLmgYsasUi4divPSuwJLQexyKQSrG60Pn9z64wKy273k4lttT3zC+CcbsqMVZdAsR9rj4vi5XEuMQf5CJS3mHe293uAiIVSjhvsXYs595bbeb9QZ9H1XapAHD8VRY3VEUTTwhqCzj6qZ09Tr/LtaavQk129G9cKzpdbx7z8VerDwCXEkTvtWN834z1XM+GCsn4gR76RjUEjmoSDiIGqvssxX7OQqCR2Bk5iMluWGQVUkMxfi7gqEJ330aa89zRcjhwlVDkBljQL01kYUKaG1sMVvfh0gtMqrUsrBdQeJJ7UWMxx1HNYo2jgq46RvgEAZkb2R+xvon62M+AEy8DVx2XxzPWU3Hj35LxhxHtr+H3IRVAa8P5VbR1k9SMFgttjGM/YewpOUpIKEvlmtxiqPxdS5Z7RvT5I+dWycI9vC/nbKlvlDuGvEE1kXv86K7wyT8MHPwTMOFHIKy33Mup50M1Ld8WtuLnkuyC7ZWM76Q+XACT/Anfsggqqdv7PyOTXQCSL1TqJKD277T3H6kulWLL7O3cS41VkKJOvs556Dah2NdyBAvski09QKWWJ2V9Q3DyDZJLpPteZeuuIzCKinyjbWstsFqtCB38ELSMFrC0F9pIVOZRudpyBJ25fk24ZDHXMJWK/dRbvA8k9ejR59h+Z8oa81hpMovkcZ/xGTIWxgJb0wpbQpKzr+HA583P3sZpTuHDl+rP9KM/5+aWo1gYNMJeyXHf4Pq7w5z9F3DAGTNIewDNRtGGRDTUCQaZm2T3hdSlQMyHzIUaiVUBrTzfO0UxJD6lLJbnkeHvq/vcB7WnG0WAwsZyyhq2Aoj7wnwdYz8H9v+e9qm+Qm7kxGvc63e4xqw89AtlcV11Xo0Nzc59n8XXtc+I/dQVc3tDjPl/ikbd23qB5mJfMy4KWCwWhIcLLFnfYNmuQUdANDBxoZzgyNlLxqaKpeoJPoFkh+XsvPAFtT5/YoCWvNCcpK/K46YmYz0D0/8VGBNyFwvWT2QB4eoT5rH+z9BG50LbOzbjwsI3hMSC6mJ1skmAcs5rLExaysBNmhsbBGcwEznkwnu6F8UCJeeYKJJY6Z6avQe0BK7z0C9IV4+7+9fXFbuc81fXO8xjnizkgtvz53aNCXkJKYvINJSar/uG0Ia6xbB63Xt1gsWPG0vNJo/3eRwoOiX3LWgo2l7m2a7yl+rtusD5HV+2XW3d1hSw+ngmltSG3n9gASBY0XdjgqInFcCefHkH5UKjXqSRGL0CGnXOK01iwUNKROUd5LMxWChKaA7OIe0ul9VRDUXxOeDcl3wujElcqw+TZ2krgK3XAqE9Zcsfi4XJPyO7XLftlXDuS7KV+wuW9BaLK0Ea7iEZqkJ4n7p9V5oNgMCInzBX/vetR1MVf+RpoKoQ6HprzXHdlqqnoqdQp5tp9S8lJSqyqBzI3k6CSGMX+3SShXQ92l1Bt4qEOUA3YU047rRxippYf4JA2gpgz93ApcvNc6/FSjteWJm8M8aeB/5IlaWu2nGHw84eY0Ht1DZUM+PlBBYADHiW+xJVgqw4jkm/1uMblyDksLGA2fVOoKOgUIn7ksm3TjeZi32+QWqr6sZA5ABYADRhlCdj9TCg+Cww4iMWhNxRXQykr+PzLNkoOipJAAlQOMvkHWKs1O5Kc7Gvuhgo89C/NbQHMHGBTMC1lQPzg2l3OeMIk9/uaDuVRS2V402MU3Un9ezL2gHsnAUMe6/ehQsAwIQf+DxpmpwPaD+D8aVxjdTneQC4PrVmsSd6Eguix18CDj7mWZWjQp8/qRXRIZ2ADjPVpIAds3nv93uqZr89wJX89wulc820LbQcDoyWSYDGvs2anQn8nH3mAmnsZ8ChxxhDSZi8Blg9lCQOCf2elP8OUDm/MIJ21pcuNY8nL6JF7Yh/mO/9tFVA+mpgwHPyNb5BcAupcbyTPC4da7GwINJhJtD/z+bxo8/SCnXkPxrfsSJ3n9q21l4FbJjEoojkbLOiL+fz6fvr9ZYWiwVhbQYAbRTqTas/P2f7q8wWxL80Ivqq+74DnBtVCs2SeK6hP7UFph+q+VxlbAD2/o6qfKNKriQeSFvJ9UmKs65P4X5MKupnbgIy4bnY1/5qFs6kZ7cihz3hKzLZLqm+bgPuLifHX+b3446Tb1E5KtnATl5DQp6quBXzIdcDVUzhCQOfJ6FAlf/oK/QX1BHRj9fkxMu0WTbCVsYioqTy9Q1mUXXsV/L6OnEBsOsOWa3oG8LrkDRXbfXZmPALB2YZWkJ1v4fkvi63yfdbM0Q0eT6vFjTbeDbjooDdbsfx48dhtxss5zI3y/ZlAJkyy7rTnkNiKYZ2oy1SCwVD58AfKSGP/6883u8Jp1f1BX6MOlzLiVbyqm4xHBj9BQP0/xVV38UKRzU3vipbLHs5Ge/7hL4Zzfjfgd6Lw1Y/ZZ9yzqsrznzI+WuvItHecSYwZbVn/3hv4B/B4tRVR7xjSTcEu+4AtsxQW4k5qpmoUBXOPKHLrSyKSMzs1GXAggi+rxSkD3yBLH5P6oVt11ONpIJqPi+KBbK2yIpEn0BaqEi2Vw2F1Ye9w1SK1RHvk+Eo3V9VBbRR2f/whbd6bUrotrYq+75fI2xlTJ6d+4/c3y73AJC3X21D3HYaFX9SMTmwNecCFSPZgAbPee6oyGTSRSr65x1i/zEpKaM5gHXjmGRtCuQfJvt7vYdicPRk4OpT5qTP7t+6VBEzDpkT1wcfBw48araLA6jk6H63OnFym4PXSlKn1AZbOZXiqlj5ukT2jZLUxauGUqWogr0cOPtP9lo1wi+Srz1E0cPJvwVwS6Fsr1eaxEJfzwfr3+eyLhj0ChN3qnn7xGtAwjfy2IiP+buqsP7vqxfbVCrf9LXApmnsvWZEeQbdTiQCimZnz6A1w5m0kxDaTU3waXMp7y8VOXN5L2DzlVQ0NSZ8ApjMTPxOtpTe/xDjlIFC8sxhZ1Kv4GRN6+lGRKPOeXWFrpqSCpnlGbQVXzWIsZIR+UdpF5egeNZLE6nwbim8tm8I1WTTFL10Mzaw911ltnlML0iUpwGFgnovehLVdasV+/m+T/K5khTV/pFOK/b62xECYDzzUzSdHyTYSjin+xrmP/fecMZemT4BLJ5lbWXx1YhzXwFLuzOOUmHLNcDqEerxtSPpPCIhZSGt2KvyzTagW6+lbSDA573Npeoiq4SRn5BYJPUHjRxEFYnq9UK7Mu4crJiDPCFzI+PknJ3yeOoSFkdspeax3L1ch6oUtn8AcOYfauvvDjOdqkABtnK+fmmCPH7yDc5f51er39tbdLuTCnepV+q+B3heRscGHa0nUKlWT5v2Wuc8v1DGCxIx6deOzrMYc0q25RN+YCwS1tMca0dfCoz7nlazxv1DwXGqZ1f0JcnfCN8QdeGl7eW170cmLQZudwAD/moeq0h3WYN6U2judINLYZcm9LLM288+1BJ8gxjrSm0RADpf9H5E3rfUhuiJwJRVnLsklKaQ+KRC19vVtro5u5kbHvmxeazbnVRPSoU+gP0pb8lX28v3fZxr2YWAxeLsO+1GBmg9Buj+2+ZCXz3xi8R5bmhW9jXjooH4kBz5K4ORiizzJtIaQMsGlW1GaDeyFHN2yQmOOIXXM8Bk8f6HKeWXFBVNiYgBTJ5JyTWrz4U/n2bIcFQzUaVS7ugBcksPm7FmXPzo9yQVXf4Kaz8PaJTAQMWA/V9EzweAfftdfWWMWBDBZ/LqU2Tn1eu1f8/khMNmZsLaK1jkO/k6N3tGckn3u9Sve/RZqm6qCiHaAznswKEn+JmkDUDP+5i0zljPjYQRcV+wR8CYL71jrTcFSpOBs86N0IBnL6zVa1NisrCZ/bXD4sOko28wC11G8lLKQio1Wo2V1U8Fx2nBo9o01xONthmaIPTD0NHpJiqI3JOtOiw+7HOa8C0VAD3uaZzz0RE9iUXhCIWS7NBTjFt73GseC+1Ola5KQRLzIX8PeR2wGp6pNpOZNM4/IicaGkIOS1vOYnH6WnmuC+kif9cA1SyFJ8mENjLLS5MYh1+yCGgz1Xys1YeJosocuiQYUXCU59R5trlYHd6HKrewXk1DSjFaF7vj4OO0WB6p6NnX5w/88QadbyIbWzXXtxgO9HqYpBwjLlmgfl2rL91MDj/pHVlGc7gsZj3da3W1OawrLFaqlgH5O5m8igULiRBaEg+s6M3/nrREnaxvIC54AuiaU+qxoLbck1Rky0TWICdZoDhGPr66kPGQpBjzCfBsA2v147Mova/VD7jmDB15whWxW3AntU1jl1tdNu1GAkjkQKpoT79H9YxEEPGEyhy+bwsFcW7fA2zZ0NYwh0UOUttb61arl22T21fEfcHiUMJ/gZEfya+RvZ1xqcNuthIuPMO8iarH3K02Pqdbr6Oi7Va3wly7K1mwAFw96CxWOiUlziUpzl39tPMOKgmHvsX/73aHrGgG+B0deQbYdDlwfZJ5POYjFh2kWLc2hHTh3Cc5bAAsylqssn13v6e41mycBtwoFAEyN1ORGDVRPreJHtwRShOZL1CtkVM3cH1U2dY3BL6h/D4l29SoibwHRv5LPrbfE0DSDywKR0+q19vWOudVZAGL2wK9/0RF48WCjPV83qesNSvFAbXaNqwHfwqOc510J/O3nsAC+YFHGCvB4C6Tf5TuB7CygOWOYe/JfeTritDuwLRN/K0q+tb6Gj24f+77mHls2mbGj/lHzCrOxLn8Pga9JBeXJvwAwKqOhz0hdTmf2YHPyVbWS53fvzQ/l6UyNlXFA0efZa5A1eqk+ByQPE9WrhbFcj2RrKYBeV/QlMjZx1jRXVFansmCZouhahJoM0z4pQp9QLOyrxkXO3QmiuTRHtSGC+TR54DTiobM5/4DpC6Wx64+yc2FlLywWOitva9u9lSNihX9GAhLsJVzM1T6/yjB/2uFTyBtCPop2J56g3tVQNCM/w2EdOHm8UIzofo+Bswqq1tvj8aEpgHzQ6kqlNiNTYme9zE4V1lt6uuEN73NMjfSRir+K/NYl1m0lhr7NZMaEvY/zES+EQFR7HVyfSoZlkZoduDsR+q1xlFNJrykINYctLFLmqu222wIfrDwR8LijhyLFVRdIV1YkBj1KRDQ2jzejAsHnwBaBZ5+V2aydncWu3Yo1FerBgMbJ3unQmpKJM2jekpCQEtg/4NUKRlhsZA9G94XQAP6SqoQGM2i8LB35PGYD4Hk+VTYGJU1g14Epm1gEeL0+0Chof/nkNeBHverk9Wn31bHu6f/DizuAGRurdfHAcD5NKgDMPwDefzYC8BmRT+/ySuBwHbAPoGkVpZGAkVpktyzSHMAh5+iOktC9k4mjjM3CecczgI2LE2j2or7Ajj2ojzmG8KCSkg3ebw0SVah1RWeSB17fweUJKhZ+st6AeuFxK3FyuTu7ZrZlq8u2HUHewOl/CSP367xp7F7U2kaY+yJ82SFQLvL1VZx/i1YpAntyYTl/wf4hVFVc32S3GJAVwarehsPfp2xkKrnWvICtQK40w10hFBZ14b34X5cKggn/kAVnMruMnMLiwcqJ6CKTHUBszacepuqQlW7iYAoOgxJOPgEFdtG5B92Kii/lxPZ4/7LOaSvhzYm7ZyODqJ1pA+LphJJwv2Y9jOYqHdHvyeBUc4CUPw3wFwf2lwWnmJvQqNVa/Z2WmfrOPaCyxZfPO/L6WAk4chf2Q/LuPbVBRH9gVGfqPt69/sLbfQk+IWx72t4b3l8913swzVG0bNv5+3AWkUCPzCaCnNVMa/tNGDUPxvfahrgszZcYV/b8z7uaYyKVB0RA4BrYpqGYH72I96jvzYbz9oQ8xEdDCQr4tz9VObpBXIJqwYDhwzKrcDWQK+HgFuraSNrxPYbSWqSFM9FZ1zKPBWO/JV7teOvmMd8Q6g2VBWia0POHmDnrXTSkCxIK7IYG+h9B91x4A/AqTc5p0jY/xAwL0BW4taG028DMR/UnJfqiqpCIP5rKmIlDH2bfQRLBbJC/hG6YBx9Vn7v3XcC68cDJYn1P6+mwPrxNR2H9t7HdXT7DVTiN+OiQLOyrxkXNzpep2bGAUxktRgiW0IUxZLV1kmRGDb2GzCi1Vggd0/dz7WxENqdwbPEhClLBtY5VS2evpdmND0sViD/IHtjSEz6Po/SBuBCNdttxv8/qDZpTQmLxRV8q/qB/FK4pYjPpTdKjuT5/K3y+G8xhD8Stt/M3nqSZXTfx2TGow4ff2D6QTWr0hpAUkubycKghdZU7WcAvo3cs682lDtt4iSLFf8IYMDfLuz5XAjsvZ8qzRmHL64kRauxQI9S+bkI70PbGFUBIXKwM5H5K4s3cnaxH40EexUQ0l3dW6XFEOAaL5KJdYWujJKSsDMTWBBZ3I59VY1WngBQeIIKq8A2NRXKnp6p3P3stdFVoYg47CQilJ+v22dwR/srgRs82C/GfETFjwqDX5aLxS2Gsk9MdYlThWKI4x1VTKio5uQop1Vq1jaZEZ08n8WvS1fSPqkxsc/ZR1CymxvyGhPmjiqZBLRmBFCZC9yQ4VJSNRa636tOWGdtoxOAdM81FHqyMHOz3OemKbEgguq+ifPNYw4bE4ZdbgPGGwpBga2BK/demHO8kDj1DpUgk5bJfQw9wScYuClHHdvF/AM4+RrJskaVnb0C2OHsM+/J7UCFYy+QhCEV1aqLOBeoipBnnIRfSflXns4E/fg59Vf1AUzsRgrKUB2Rg5h0lxD/FefG0f+uORcEdwYGvghAY1LY6AAT1ot21J6S8CM+Aga/Kislw3t5VnhmbSchQlUM1BHSlW4Wwe2p7u8w00wkuN5Q/Dvh7NXW7W7z/Ja1g2QEVe+9sV+xeJA0FxgsFCdqw7GXuGZ2mW0e8wsFKixcb4xuE5V5QLe7ZLtfgKpn32D13FpwVH0P+LegjaN7z0Z3lKdznlIRCRuCfb9n3HfVEXn87CeMVySl+cm3AHspMOzdxj+vMx80jbNCU8PqT+tMY59LgK01kn6gm8mgl2o6xKStAvbdx5inyyzzsZoDgMWs0AWA/k9ToSnZSh79K8c63ag+59jP+Dtzs9wLLmk+1XfD3qt/PJLvLIhVKuxvi2OB3n8Eugq9VMfP4Zyv6hOo29pK9vC1oe3lJIOp+lV6yp+G9+U8p0Jwe2Drq7wXjK07cvaw7cbQd4CO15uPjehHt7rsnb8O1dzQd5hz1hHQmiS1wa/KCuhm/CrRXOxrxkUBq9WKPn36wGqthxjVXkFWXO8/AZ2uN487KulHrVI8FJ4iC7HTTXKxZspq9caiKdHzfhb7SpPNCcX6+OY3o+kR+ymT7X3+aB7zb0nGTNtp7FHQjGa4was579eCy7aSzdzYLP3acPRZ9rcY85W8SZSsj+sKvadFtGBXWJLIjVKbybSHNkHjxnXqWvXrpy7j/C1ZkLZU9KEB2Pfp1NvAtbEuayUdFgtwg9CbqbFww3m57wpQO9nkZ3ar5X+nv6xuL3YxETg0jUmotpepn9fh76mPv+poo51Ko855Xe+Un1WALOjE74Cx38rjGyZTbTvmi4afhxFpK4Gt1zDpfPVJ83hIJxblev+h5iYboJVX3gHGtJfvND/vjmpeT6mAVJbCBEIvhY3e2G9ZhPfGqtBRDRSc4PwlJSyvS6AtoITYz5mMltQBvsFMLqwaLNt5WfzonKDqux3aHRjypmyVXniahT5AMWc3EIFtPBMsFrenNalk/Rvchcmx/CNAUCP3W40cAKQuZXLNaF915gPuh4YrnFCWdOZ9dPkOtfpKhUEvcS+l6r+z6Qq6XVyX3LhJbYuFfdrOr2JB2Whfurgt16JfyE7/F4nzjjzN39UF5rHKXGCRU20vreH696lC63F8VsX+d851XjUv1wa9SBTe1xwT9fo9C5Hp6+R4KaA1iSuSqqsyFzj9Ducbb/p3th7Lezd9DYlVRkiODTo630KCkL2s5rwd0okK2qXd2QduhkEFYi/jvOqpPUBQW7Utrr2KvffaT6eS3YgtM0iWmLLGPBbzMQuQ476h7aa7PWldiHQDnmNBuPisuXhQmsRCXpfZ8jPZbjp7m6l6XnlCwUngxMs8R6nYl/A974OZ58zFvviv2JPxij1ykrvLLM4vNgVpavSX8vMGkGCz63aSdaTCxmLnszRtE1VWjYnoicwlSUj4juqqoPZysS9lEa/hoJfqRaCs05x32VbvHFh+aRifU3f0/gOLfSdfBwYZSEB+oSycD3rJbP+Zs4d9pDte5+x3aYixenpwGCtLVe/RdFxzivbakl1weSaw0/mseNMns8vt3Puq+qEe/ZvT0lywIm4/nT8qWKxcb6QCaG3o9xSL9yobY0+w+tAtQpU7DupA1xqpJ26nG4GWI0kKkHopDn4NaH8NHQV+DTC2xxr6lsuOuRl1xi+dz2su9jXjooG/v5DA0BUTl+8yexw7bAzsut8tF/siBwKX7VCrX1Y6LRMC28iBZfl57xglDUXn2UCHawAfIbjyb9Gs6Ps1oSwVCFAUYG3F3NT4hjYX+5ohQpzzLgbUs39Do6HDdUz2NEWgPP4HYMJciH318g4Ae53KkWvjzKrwSxapXzdzM1nUx19kkD95ef3OK2oSEL2HPv/G5D8AFJ2lzWfrcTLDuyFQJW5rQ1kqsMSZ0L0xmwqK/wX0e8K8Ofq1w2JhnNTpJjkBdvJN4PjLwPT9aoZtI6LR5jxPypHQbrRFUrHwi2JoCdfn0ca3ztI3+CqiWEkCkzkjPzaPpS2nfVD/Z+Sk9Y/O706KATtcS1t6VXLYG6WNjtz9wPoJtI2TksP+LdS2kTEfsGgnFfscNqos+jxKJrYRVh+1cgVgfN77D7K62C+czOrOt9S/f2tdcGOGeix9Pa9/9GR5fOJ8qlKlPnINRUk8i76OavNY/6e5X1LCufal/FT/Yp9PoJz80qHb2qvsHxuCLnfQFs5eBsCQQG43g0lWqeBRXUKFra0EmH4IaKnoydZAXPA477KtJC91uNo8Jll31gcdrlKrZH0CGrY/HfAc5z+VqjDhv0D+IWDAM+YxTz1cw3qT7CC1A6kLKrJp2xvSRS72eUKP3zFeVhVLBr9M9wYjdt1B4shtwnNcF9hLgYx1vCbSvT/8fdorH32Oqmp34kvWVuZdxn1T85jqYsZ2Qe1rFtUzNnKfqxfJetzLGF1SWXa5lQWkvIMy8cTiQzWqlCivDZVOwklvRT/UnN387Sv0kG49jna+id+rFS1Lu7JAOU2wt1P14AL4fU1cKNscAlwnsrbIVv0NRVUBr6eEHKeqWVJEA1T0bZnBdjhSHzoPqHXO8wmkpXm3315cCqITrwOx/yIhxkgiihpHMktVnrlAFT0JuGKX/JoBUZxbUpeyAN9PaAeR+APVwNETa/594PNASS2W4J72cb7BVLG1ncZ1sr7wj5D7A+sY9LK6GGkr55hfuEwInbbZe3tR35CGkbxmFavtWHfM4jw4M9Y8FhjNWNhWRsKFkZgX3OHCOx/UFRVZwJpRnD/7K9oTNUOJXzKfdxFKBprx/xEOhwPHjx+Hw2GYXPWAT+pD4hvM5usWHyB7t/zC68bQekvCwBcZuKm841cOADZOlceaEsdfoHrlQvcAa0b90fkWNVP+rLPngaT6a8b/eyjnvIsBSfPZB8DRBH3iPKH1aGdRogl6WxSeoF99jrAhi5rAQl1QB3WyMmuH7NGfsYGFvkEvqa2LPMEvjJt1lWXhyv7A+olNcy3OfACcVqi+Yj8DNl0pf2b35JWUiG/GhUWfR7mRk3pfhHanC0LMP8xjALBuAvt9NELfsws250UOpBVN5hZ5fJyzp1T2jsZ/7+hLmPCeqUjArBrsUpwZMeR1Fuys/iwWeer9YoTmAFb0BQ49Jo+XZ5AoUV5LfxcJem8STyoTFSYuYjFlpTBnF50GVvQm41wqTAC0rV+hKNYlLwAWhFE1bURwBypuVH22Goqy8/yRoPet0/thGhHWA+h6m/dkCk+oLmGyULLjaj2W87UemxpxbQwTbL0UPRI9IXkB+1aVK4qgl64ELl0h2581FINfAq5LZNLUiPHfAUNelY/T7Lw3ASZomwC/SJwXPUn9PPmFAzdm0U64KaA5XDbG9cWQV2kXLJFvi85SYXaJQkWXs5fk4CxhTvfxB1KXqHt/1obzK1m46vlg/Y+1+nHeNBZZi2KpMIdFtvWLnkzyjTfzNcCE8+jPadMmoecDLKplbqYq1h3jvwdmlfI6ZmwE9j3EHqPpaxlvphsIHztvdalJAQAa1ZmSQtTqw/kxaZ58XmtGMJ71BtGXcu1VKVM6XAu0mSYX+6ImkEwX/7V87KYrqdyrb7EXYAzfWeEgBQCXbeZ5q57ZhmDoO3QJkDD877zOrRU9DsN6UsWr6jWoQJ3mvILjdEaqrVD1a0P+YacQQGEHHNJJ3e5B09gv19jrN6wHBQ0j/kE3ACPOfcXiv9T3eOBz7LvoCSUJLBYWnDCP+YUBly5lMdcbsmhJArDrTpflphGRA7ivPi6oBlcPBRZGcg6SEN7HO9vlxoLq+2gxjOQACQ4b15qFkSQOGHHyDWD1sF9PD/TN04E9zv2IrYTrRlkq+3ufUqwdzTDhl87nNRf7mnFxY+THwOQ1QI/7zGMWK4O2c/8B8vabx22lZGqpvPYHv8RG5SofdQAoTfTmrBuGhO+YYK0uMo9V5DDxtnHahT+vZpgxcb7sgQ7Qaqb7PbJ1QjOacTFj52zg1P+Y1UPGBm70iwW2XlA7KvJuSJV99lOWABsuAQ4Ihf3ef2Rfqv7PKPru1YKIAc6eUwpSimbn76ZQTBx6gtZGEvY/ROZ41jbzWGAUkxe3aw2zVm1G48BWQmtwqSDcZTbv70JFDztdHafqffFrRcK3/JHQcgQtszpcc2HPCQD6PsG4dftN5u88qB0THInfA/OCWJxzh/5MSbCVMmHZQqFOWtwO2HwlkC08r7Wh7eUsAqnsHz0hcgC/5wih2BfQmixiT4qw/CPqhKB+T6Yr7JPPrwG2XMPXaGws6cAfCf2ddnAqxvvhp4GtXtip1gWpi7mWqRD7KQtzEnwCuUZJCvLacO4rOlkc/as83uGqpklmA8ChJ4H/a+++49uqzj+OfyVvxyvOcpw4O2Q6m0xWICRAyh5llAKlUCi0BCi0tMwO9mihlAC/MtoyAqXQEgIhhBkyyd5kb2c5sZ3hJd3fHye24/jKiR1JV9L9vF8vv2Tdcy09J3IeS/ec85yN/7bPUb5y83d5nc2Ft4QM6Qq/+T+VE0OfqfZvMq/x/k327cktQrNfkGVJb8VJbzXy0tPB7YH3fiqYYsq4B9o7dv2bZjXaAZu9Rf2V5u/c4PGNi6tsl4mrMXtUL/mD9LXN/3V/uVS0tGY12pFyRpmBhVUBBuaPRZcbpczu9Z8z+tu6e+7FJZvJ1B6PyZ2rx5vJQpk9zZ52R67YG/gXqcfdNfe/vsCUsrZTVmgG4075r317Zi8zELQiwOSjozm43f7aiWTy8hmfBd6f+MTnzaCLncSmUt4ljZuw56SF99gPEklmIl7lAal8j3275TcVAVo1sixvfebdLqV1tt/XLJKVbjelIe0m6iz4jfR2klSyum7bvvWmvOfOb+z3TU7NNQNudiWKS743t3aTWSYPNZO86vPZaWawcP7d9u371ku7Zpu/lQ1VtNS8Xw002OdNMu+3U2zeK1WtdC1Z2fDndVK7S8w15w02K2I3vG1W/iU1NyvKj7Tm7yanBnrPGm7le00VMslM/DxngZks1qR9+LdpQaMx2IfoFt9Eyh0TuBxTk/Zm1lKHq+q2le40JT8C7RdkWfXPQBz6uikhFG69fmdudwZ40ymZetKIbLljTQmXo9VTB6JN+8vN4FMoBpicsneRuQ00qFafFU+bW7vyKyk5ZoZ2Y1e4bXjLDKwFKvFz2sfmwnJj9jU4mh98b1Ya2TnpHbNfVjub0pCILH3+JF20PXBprHNXSadPsW8b+qp0eeXxl38Lp5I1ZlVBoHJe2z83F2/rm+jVWLvnmL3P1gSYbd3nIanFyWZ1w5GriSqKzWqxJh3Me9qGxLd3oSnfGGgmdNX+z43Zuyy5uRkECrQ/VH0q95v30XalwlJam/e7y580ZfLspHUye8LYyeorpebZl0cr3yt9c5FZlVMWglVbHa4KXNUhrZP0/XNm7ys7yx+Xtvyv/vf4jXXqRDOgbHfhbs4tZuBi6Cv2P7vqBTPZsDEG/sVcuA3093P7l9KKP5uVh8G29lWz55bdStjF90vf/9V+ANQTQ/vJHm5SH3PBcec3ddssvzT7Z+a1iDTv55j9BEt31G3LPdvs1RxoNezBQ4N8dqWqSwvM6rNAE1qOps8fpZGfNu6zXFJz+7+dWb2ki3eafVqn25RYTmpuVmW1DvKenlWmnFLzvEeuYNm3zqyUtPzSCbeYUuzNBplBvj4P1S333eHK2qVdfaU1pTqPtGuGGQjcPdM+rn6PmVUz9ZUGDKRsp/kd+qoR7+G3TDIrkAK9xie9LZ0cYJLE8frmEjOJ226g+nglt5JS2tq37d8k/aeFKdFtZ87PpamN2OPyWLS/ypTwDDTwGqn6/kkaFmASWVJL8/vzYde6bfvXS4vukwb9VRrycu02y5LmjrNf/SZJnX5i3rvbTZbZPduUpa/PCbea/7NdbVYmW5b0v46mClp5gIkW9WnzAzNAHmgi2PxfSWU7pC42CzZO+a90ebnUuZ49CSOSZa592K38S+9iSqWP/KRuyVXJvP6tRjauZGoojJlZ9715kzxTcra+vSIRUaJsKi5whK2fSNunSj3vqbvZvCR92MXsyXCqzSyx5BbSiLdNOQk7U08zqxKGvCJ1tim3czx7nByP5sPMxZF0mzcMyc2lkZMDz1xD5CjbacqRdLtNGvhnp6MBgmfEW05HEHwdrzb7jNiV+CmYKn0+yvwtGT2j7oy3gX82+8HY7XNUedDsJfRRL3Mh9MgPekcTl2r2AQg0sFrfBufHK8Pmb1CVdpcGbvOVmtmm/grpvLWU8nTah53NANNIm9m3278ye4V0+2Xg1R6hGEgOpcoS894u0IXSxYf2EWl3SfCf218pHdgk7a+nTF7328zXkZPNFt5n9h67YLMppdYQaV3MxeFAJbnOnt+wxwuWL84yF34vClCOzldqVrC0ONm+PVA5VMlMAjxyVUqVygMmJ5/wS1P+L9jqe338ldL6N8xAp90enye/ZyaIhGJP8NUvSWteli4prLsVQJMOZsVQsk2JT8lc3JXM6vY+v2/Y82acYMqmBjJ1pLnt8CMpweZv7PFof4WZFFN5oO5jb5hgBogCvQdf86q5GNn5BvvPmNGow5VmRVh2gBWzVQPr3ccF93k9HrNiOtA+5sfqYIF5z3O4tE6mXNyi30kX2qyM6fFrM3hgJyHDDIA3dq/pwrnmesHA56RuASaQBDL4ZenE+lYUWvYXjbdOMqtHut7UsOdrqF0zzXvLw1dXL37Q7I942T4z4bqhpfT6/smsYIm3qeqQ2dOUCQ60x2tG18B7mx1N1aBHoJVq9SkvNGW9D2w0A5t2Zv7ExG83qe94bDq05/eeBVJqgIG5xjrx+cBt3x+ajBLob2+zQeb6W8Fn5u9GMPW820y48pVF1+eD+nJIjzvM/2W7QdvsgeY9mN3Aq8dTU0a/1ci6z5FxgnRpgLKPrU63H1Q/XM+7Au/Bdvhkl8bskynVv19lYnbgvwcej9nPOdoULTV7oNt9fmg+1HwFknt240oBh8P+TSYXtTo9NPtJI2RY2Yeo4PV6lZ+fL6/3iF/ZFU+bWb8Lf2v/g51+Ypa92+0nE9/EzPILVD+75NBFhECbuH5zsalVH27eBHORKr2zfXvr0aG5SIXgqtoPqIH17uEOAXNeNCjb3fi9RCJVVl/zJt1fVretqhRv8Qr7CwnZA+wH+iSzCf17zc0EDbsyLEeTkmM+XBR+Z98+5xZpcj0ftkLF8psL9X5f3TZ/pdnny65cDcKv5anm4prdypc986WVzwTeNyOIwpbzsvpK564OvGda69FmAGr9m8F/7hbDzGBd/oP27dMul/7TypQVPHJVUatTzeBUY2a7JzY1A5w7A+zP45Q255rct/zpum2lO6X5d0oDnjZl9RtqzyKzOsW2lHAr6YItgfdrC6WqUpb9n7Bvz7tIOnOalB2g5OrxaNpX6jZOimtSt63nXdLglwJPFjzlf6aMnl3J1aM5WCDtWWj+JthpPcasVgpFaai2F5gL8HZlFkfPkM5eGPh5Z/3ElF8rWhb8uOTQ+7wTnzelSe0m63i8UpebGrcv47FoNbLxeypfuFUaMydwGdm8i6SO19q3NWlnSobblQFNyJD2b5S++2Xj4iovNIM8ga4l1MfjsS8v66+QNr5nVrINe61ue+E8U4rVbp/dYDjza2n4P6QZ15hBrMO1v9zkLk+CeZ+/d4n5f104X/pspClbf7iPB0rfXlFzv92lZmWLXUWmtI5m8tyi++zjWvyQfTn8Y5F+gjTo+Zo9eRui44+kFieZ1X121r9pVhAH2gf4eJy/wexFmR1gkDFUcsea17l/gL25Wp5mBoL3B5hQE8Ax5bzvn5cm9bYveRnNuo+TBtjsc56Qbt6Xbp9qv6J/yCtmwpYauNL85Pek847z3/DCAunSYnPNNNj6/N58rj6yJH002/COtPLZwO17FkjfXmUmUh5p93fS6v+LnD37Nr1vSqBL5jPRvNvN7+iSP5nVzjgmTl/Pi8KriHCr8nKb8gn5D5rl53ab1kpSm/PMLLQNbzf8CS/YaPZrCLSP0qb/BN6sOZRm/8y+xr9kLqKuedX+jwgiS9Ub5MZcOIEr2Oa8aPBec1MuJ5Zs/kD6ZJC0c0bdtuz+0vkbpTGzpZRc+5+fPFSacW3d400HSF1+Jv1gudTv4YbHldjUXEAL9Lyr/ha4JFIovRVn9hVbbTNrPT710EXWBY3b5wbBlZBlVr5YNgOznQ9d6FvTwBWnjRSWnOfxSFOGm9JtdtpdambmprYLzfOntrFfrSFJGyeYMnUzrq5b0jDvImnQX0wOmvVTs5fKsfKXm32tigMMWkz/sSkVVhDmEvA975YyeppSikfylZr32XsXN+6x9y6U1v9T2vy/um3eOMmqMIOfoSjjWZ/sQVLHHweeKb9rVs0FlmBb8Yy048u6q/qqTMqXZgYYBG97rjR2iX0pxKOZc7P0cT9p7Wv27SM/MWULQ7H3Z1pHMwhk1+ekFocmJxXY/+wJt5p8YFdJJUgi7n3e4BekE49jL7hQSWltVhPZTXZY9y9pye/NeyE72z6Vpp5uykTaSeskZTVytcL2L8xgcGP2sgzEsqRpl9jnRcmsRL3C17jSyQ2R/0DN1iFVcs82g+dxieb95aR8ad9aU6Fi7yJTCvhwiU1r57rvfmkmSwfS7ZdS15/bt619zfyb2L0PP5qUVtIJP68pWd1QuT8IvDp01Qumn/WtXm6sJu2kvAtC/1ofqdWp5nUONBEi9yzpisqa94gNcNScl9nDXBsJ9kpGJxV8Li3+vf2EUL/PHP/28ppVfIfrfJ103iqpZYBVloGsf8P8za/P7J+Z935zb7dvT2kVur3VywrNXoWhGCR3Su/7pC431CwYOVzhfOnz0dKGN03OPNL8u8wemoVzQh/nsVh0/6G9cGVWn545TWp9lrToXnNNBMfMyfd5DPYhKvj9fq1cuVJ+/xEzz1sMl85ZJLW/zP4Hm7Q3H9YyezT8ST3e+vdrOGex+eAbbql50sFt9kvz/RVmJurU08IeFhqo+zjpsgOBS4LA1QLmvGgw7B9Sz187HUVwVa2cC3QxskmeKXVkV3Lmy7HS7ln2pTZzx0iDx5u/VY3hTTQrswJdjOx0rXRCI2dCH4+qcoF2+2V5vGY2Z9O+sbknUrTpeLU04Bn738+EDGngs2ZFWYiFLef5K6SsfubLTvOh0knv2u+pEWrnHjYLO1Cpzz3zTRm0hpQjK9ttJsY1DbBabP2hvdhCMdhyNCe/J51uM7M7ta2ZUd5iRONWVlXte2JXltLvM1VBvjxHKgrz+/h2l0mdfxr49fv6AmnapYEHoI5H73ulHgFKdm2ZaFYKBSoj5SsNvDfs0VT9faooCfDYZfarwIPhizHS3AD5q2iJ9FGPwCuJBj1n9qxJCVDa9Dg58j5v1k/Nxd0NE8L3nMEw/UfSd7fZr0BPai41GyIlBNjLbd2hfbTsKuL4ykx1hEATe4+m07Vm5VVCVuN+3k5cojTsn+a9kt2qem984AkjwbDun2aFSocrpQ4BJsVIZqJ1/kOmFF+LEdIlu+vuv3XGZ9LgF2vur/m7mcRxcFvdx9uz0JSRDbSKqGpVXlVlnHA5sNkMuvW407598IvS6VNDs3fx7jnS1snBf9xjMftn0oIAlbMW3GMmQB5ZbvwojinntTnP7BPdmL0ZI1XBFGnxA/aTPvcukt5rZiZ/drcZdNu/wUzi99lUlqnP98+awbT6VL23spvsJ5nyyO82Dfy3+3gULTb5M9glo510YLO0/Anzmtq1le005Zs72mwFZbfa2Ukn/q1ma5HELJPj00+QzppnBjVxTJy+nsdgH6Jb5X4zuznQB9CVfzF14eurkRzI+jeliT3Nsmo7Wb2DO5PvWOWeZWY42pXDiUuSkpoF3ocQkeNggfTpsMAzN4Fo1fFqqd+jTkcRXFl9pIweZrLFkfZvMBehtn5sfyGqxQip/ZXS0L/bP7bll5b8sW75o2Ox81tpwa8DDwwMfVUaVE9JkVAZ/a0pFRZoL4/d39mXq0H4bXrPlCKzu3hYUSzlniO1PS/8cYWKr0wq+FSqDFCucPrVZmVqqAYf6pPeWTrxBfP9kat1178lfX2h1OEK6ZK9ZvDumFnmIkOg98pnL5RGz2z8vlWNtfI5U9YyzeYivMdjLo58c3HjBiYye5nSk3kX1W3z7a9575Ue5gssll/67BRp0QP27VUrjEpDUAp71yxTltLud3vboQvK7QJMnnyvpfROE2nblIY/b79Hpcv2mz2q7UxIlt4O0UCzN0naHmDF6rRDe8vmjg3Nc0eivYcGt0Nx8TaU1r9hLl7blfbLPcus2FoXoERj7tlSzmipSYBtObZPbXyp1oMF0oonzSSMYGo9xvTZiUHZRfdLywOUb1z2hBnkKdttVn/l3y8lNz/2x66abGC7otoye6n6K+x/NnuA2dvM7kJ5KO2cbnJFoPesmT1N3FX7AgbT5MHSl2eFf9X96pcP7fH6f/btJavNpHO7QdvjlZBmrnPFkm6HJpyU2PyOJLcwE4CqyloeafqPzCT+klUNe87uvzr6HpJnfmM+qwX6nLj+X1LFXjW4hOix2PQfs1o30CSNaNS0rzT0Nfu9LnPPNnuddrrWfq/zvo9IY5cF3ls73FqeXPOZwF9p8rLHa6oZNbG5FoKIxGAfotvC30n/62j2PbKz7WNTsqYxZlwjFS8PzZu349HpWrN3gd0eUB6vdPEuUxIOke3gNlNmqnSn05EAOJoTbpF+sMx+5t3BAnMR6stz7D/49vqtNOIN+8fdOln65hKzsmDjOw2Pq91lpnxooBU7G98zgwSRZvKJ0pQA+xgivLZ8GHhPvg1vSx92Cf9M+lCKT5UG/kXqcLV9u79CkmX+PofbtilmUsGVlpm4dbjilea18pWaGe8NWYXXpJ104nhzAdlO0z5S8yGNj7uxdn5rfsdKd9Rt81eYPve+z5RVbaikbFN60m41jzfZlKc7+T/hL4+29SNzmxNg+4Hh/zQljkMxCJmUbS6i2l1o6nWv9IPvzSopO1UXIfcsaPjzerzm/12g8qG97w/8u3m8Wp0W+GLiCb+Q+j9lyuTZedNjvhpbSjYSnTpROneVmZQVTU76t9lnNdBrWfCZtO0T+7Yev5JOnyzFp9Rti0syq9ftqhAci4oiM/ARaD/KxkrINGXTAg2Qh9KIt8xqtU9H1F2JVLnfDPT5K2sfLy8y7zcPHzStKJaWPlK7TF/328wKdrvJ0k37mb3Lds+yj2v/RqnyYMhW2gbUfKj5/Qg0mOwrM+9nFwZYBXc8ql7/UOxnWp/KA2Z1/CibPW+lmnLOdnvioq6U1mZroHNs/paktjUrqNoEmHTS909S2wsb/p6gy08D7w18rPo/YfYMTEg7vsex0/Me6bSPg587nbTwt6YUp90ECG+8+X+1e5b9/rFJ2aYSXWP25Q61DROktxNN+c7yvXW3GUDEYrAPUSMuLkCJKSnwDMXhb0rJOY2r737ye1L+7wN/CHTKd7+QJg9zOgocr6qygIEu+MD1bHMeIk/TvlKfP5oPY4HepK96wX7vqH2rpc3vSwP+LPVtxJ59Ka1M+dBAeypMu0SafmXDH/d4fXebuUga6EJAUrPGly5FcLX7YeAVX1WlLsO0Aj0sOc/jNbOsA/3trVqNajcAFWpfnmVmcO+xGWjs86DZIyc+XSqcJ/kasAdE2S7py7Pt99CUzADOssek/ZsaEfRxGPYPcxF16ql12yoPSN9cKB3cagYjgykuUer7RynvwuA+7rGoGjhqNti+Pbml+ZtiNzBxvEpWm33q7KS0kuaNk5YG+Dt02kfSRTvMvlfB1uchs29fKIx4S7p0r31bt19IPe44+mN4QlfeNuzv85Kbm4EWu5LjkazdxdLQV+wHevYsNHsJnfKB/c+uedVU6gk0UL32tcCTho+mwxXSRQXB/yw35SRz4bgx25Acr+ZDzR5NUt1S630elM5fZ16Hlc9KU04xfysPbDTvNze+V3Nu2S7Th60f1xzbu9S8DoF+//atqdnT/khzbzP77Ta0nOHxatJOSmwm7Zln3/7NodXjHa8N/nMP/LOZ/JM9IPiPXZ/ut5nXOTNApajsAWYCUbMTG/zQrvxsW3nw0ABPgBVyfp/072amzPKRWp4infKfwJNljsf2L6Uvz5UKptq39/iV2TMwFLyJ5n1poFXE0ajt+VK3cfZtVZMfppwUHfsUfnGW9P6hCiNN2kntrzDVjf7dVJp1vbOxRRknc54DmzMADRcXF6f8fJuSRb1+K3W+wWyMbOfAFnMhteOPGv6kbc+LzNJVe+ZLu2eaFWHJAT60I/K1GG5mTNmVr4LrBcx5iDxxyVLv3wVuX/OKNOfn5mLUkX9Tutxkvo62R2xjDX1d8jVyn6Xj8f2hkjDF39uXBrx4V3jjQT38UmWACVPNB5u/UcktQx5FxOS8zj8xM6ydKC3U9edmYHXXdDPgY2fFM9LKZ6TzN5gP4MciIUsa8n+mtKWdjw+tDG42OLzleeISTUk2u/LH8almz5CMEF3sXv1/ZmDr5H+H90Jqj1+Z1RqBJobMvkla+6p00fbgr+bwlQaeRV9RLG2dFHi/LG9CdH7m+PJc06eT3q7bVr7XlIlte35NmbXDXdmw/agaKmJyXjTYOd3sK2u3ArlombTsEbMHelbvuu1bPjSVegL97ne6NvDvvVMyTjAlaJ1gWZIsU469PqU7pOIVJn+ntpNGvC1lHvbvn9JGOuu72hMM5t9lrmFcXll3hXH5HqnrTYErVWTlm73PVo03g1HhNOprSQH2XMoZZVZutT03rCGF3I5p5v9Fts3rcXC7GaxJbdugh3Rtzlv5jKlGNuwfdVdVl+40E/nLC6XUME+CnH+XmXyemCXlnFG3fetkqWxHaFaCxzcxuTe74QPGEavygNmzr80P6n72LZxrfg/SuwZ+fx9JsgdKidnm+5Ynmy/JlJxtxCC/Wzmd8xjsQ1SwLEslJSVKT0+X5/ALonHJ9V+YSO8inTm9cXv2HSwws3DSO5vniRR5F5vSRyWrovODN4zUPLMytbzw2C/YwTUC5jxEHn+FmViSkGHKcBxp26fmtt8jddu8cebCSsVec2En2OU7OoV5b5Mq5642A0gZPZ15fhy7Dj+SOl4TuP08mz2SQiBicp7H69x+MX3+aPb3TLPZW2r/hkOD5yeZ1TmBJrnZSUiTOtczE7fnb6QNb5kSauG0b53Z26SZzQV8b4Jp+/xMqdc99oMxjWVZptRS1fOE25ThUptzTXmuIx3YIvnLFZI9ctpdZlbb2Kla0dfX5u+UZC5U7pphqp405HfPaQc3By6X/90vzX5+4V41c0jE5LxoUFX2++JddUsctxkrDftn4N/LqgGJQNUE5v5SanWG1MGBKgiBdLrW5L5mJ5oBsHCa1NvkyB/Y7GNYONespGx3iVkd3fePNW1VpR2rxCXVrBCs4j804Fq2o+7f2dId0uyfmRLLdheTu9xkJiXYDeiG0vYvpG8vl/o/LXW8qm5799tD99wf9zcrIc9bJ6V1CN3zHGnvEumzk81+hGOX1m1f97q0+kUzgbEBq4Rdm/OqBrCXP1V34MxXKm2cIHW/Q8q/L7xx9fmjmQzR9Wb79i/PMrehGOzbOMGsqu58Q/Af2ynpXcz7rGSbFehpXaR+j5vJAXZljCON3ftTyZScxTFzOudRxhNRwe/3a+3atfL7A8yqCmTWT6UZP27ciomvLzRveAMtbXdKm3PNPiON3V8AkWH/BvOhZuskpyNBBGp0zkP4lawye8e+18y+pPTg8WaVht2F9IPbzT4z/842q/9iRXpns/9KoLIzVXshwXnTr5Rm32jftv0r6fMx0o5vQh4GOU9mL74Ww+z3kdv4rvTFaCkl11S1CFS6tzH6PSKdv95+skIorXhG+uwUqdxm/xLJrBxI7xL8wSWPxzxmx2vMapFwKt1uSnmWFdq3n/KBGdBIDMHK0jUvSUv/aN/WfIRZcVi1PcKRlj5sLnpvfDf4cYXSwL9IA5+xbysvNBfgAu1rNOcWc7G9NDQr0cl5DVC1utxudV5Chvm9/u4X9j/b+z4zASlQCdsTX5S6jwtKmEGT1NKU2E7vGv7nzhkjtR4trfyrtPWI8rob3zMl3A4WHP1x/BXmPbHfV3Os9/1St9tNOeojpbaVuvxMSutk/3gZXc2FZrsVSKEUn2ZW7JR8b9/ur5Qm9ZMWhGDPvr2LzG3p9uA/dn2q9r9sE2C1YtXerlXxHSPX5ryc0dLgl+wHUFLbSj8sO/799Rojd4x04l+lrABVHwK9HwiGrHyp008C7xMcjUpWmX1c7Va8NsmTet5lv1I20m2dLM260UzQQ4M4nfNY2YfY1ukaqWip+VDd0IsYw/9lZi7lnh2a2BrrwCapbGd0zaxFXcWHPjSk5DobB4Djc/gHFV9Z3YvwiVm1L3Ycbtsn0sxrzQznljb7VkWrfevM397mw+rOwpfMajK7AQ2EX/5DZlDFTtlOqeBTU/6xqoQLQmfKyZLlk86cZlYYHi7nTLNHThObVX/Hy19pSrF5E0JTTjiQvAul75+TFj8oDX6xdptlSV+fZy48h2JW+SUBBttCbfMH5jbQKiJvnH3ODIbB/yf59tu3tRkryTKfMeyqhpyzSFr3r7ordSKdXRnpKqdNrP9nV/3N3JbtMKtp4ZwLNps8ZbeKqPKg1On6wJ/zk1vUXwnHqQoI9SlaImV0D//AliQNfNrk37e8Ut4lUu5ZNW0dfyy1OMl8dt01S9q/Xsq7yKwY/rifWaVTtdpvx9fS56OkE1+oWZ2Yd2HgvVLjm0jr3zCP2fknddsX3ittmSidNSe8K7KbnWhei3Wvm/1Fj7T6RWnvwtC8R7p4l1mZHO4KQHkXmcocLYbZtzftawZr4iKs/G2k8sZJXQKsYPN4JG+8eR+UlS+1uzSsodXr0qLQPXZGTzPhZvccU7Y4FhR/L+2cZq4F2JWGXv+2tOxRU1a/2aDwx9cQa/9hto4a+Iy5XfOyyeMz/2gm4fW6x+kIcQxY2YfYFpcirfl74E2V65PeWerz+7oXXJy2+iWzIswKcPEY0SEuSWrSocH17gFEmOSWZlbmZfvsL9Ie3Ca9HS99fX7dtmaDpYHPSmd+G7pN0J3wv07SV+dKWz6ybx/+T2dmsaKujj+S2l9m39bmB+a2bEf44nGzndNMqcRN79dta9pX6vozad7t0ldB3k968onShCSzciucWo00VSp2z6nb5vFIngRzESwUds2Wlj95bCtUgqnFCKnfo+b9X7hldDUrru1YPvM3avmT9u1Z+VL/x6JvsK8+lQfNSqU9AVannPy+NOivzrxWqM2bIMWn2H8m3/aJtOBu85k/Vmx6X1rykFkd55RRX0n5D9Q+ltldanOOKQ296gVT3tJXdmgVdrfa5euSW0mdrqu97+rsn0nvNTcDt3aG/8uU8bRT+J0ZVKsqjR9O3e8wqxLt7Jppbvs9HvznTWxqBkLCvZ1MWifzOgeaWN7uUjMBpPng8MYVrXbOkL651AyQH8nvM+/9lvxB2vSf8Mb11XmmysrsMJcKlqTKfaaEaMnK8D93qLQeI3W4WvKX1W0rWi5Nv8LkMIV2P+Cg2PqRtPLPZiJg9zukS/ZIWX2kTf825c8RFSJsFAMILDm5EW90Wp4iDXq+9hvNaFd5aFbugc3OxoHjk3ehdP46U7sbsNGonAdnxCWaWXx2q2IW3mtu7S6yZvaQuv3Cfo+uaNbt0B4m0ViuBDU8CdKpE80HvTBwfc4781uz6jXQ3lKSuUBiVy74eJTvMbdOXCz/wQrp7AAT8kZ+LCVk1VxMDaZ5t0vz75IObgn+Y9cno4eUkGlW0EWSqouMrc+yby9eZT8oG822fiRNu0Ra+6p9e94F0gm3BH8v3cO4Pucdq0l9pHeb2g9+pXcxJWjTY2R1iGQGU3JGm5K/4bbscWnR/eYaSn37451wq3TSu2YgKqmZdObX5v1slaze0tBXpFaHVa0oWSWV7TYXkI9UukP65hJp4zv2z9fjV+Y23LnzwGapZHXga0lDXpYu2x+aAblN/5G+u00qD+EKq0A+OdGU7rOz6AFpQqq0b22DH9aVOW/fGjNIsuOrum3+UumzU83kz/4BJtuEStWE80CTgEKpvFBqf7mU+4PwP3eo7J4lrR5v9hY9kuUz7+1P/Jv9nqSRZuBfpAu3SvKYaxyJWWby3UU7pdM+OdpP4zBO5jwG+xAV4uLi1L17d8XFxTXsB5c/aTZ/TW0TkrgckTvWzIa2q3eP6LFlovRWvCmLBByh0TkP4Ve6S5rYQ5p5vX25zrbnmRnBPQOUvKjYJ317pVmFHisGPi1daYV/PywEV/lu8wE1VGUFD0POk9RiuFn1alfeZ8M70ocnSD3ulkYFeQXeOUvMrN3mQ4L7uEez+v+kSX2l4gAzu/dvkhbfb1Y7Btuu6WbQLT3M+1+X75Xm3CxtmBDe5z2afWvN6qmmfezbP+4jTR4s7fw2vHGFUlWZzkBlYn2lZoKlFZpZ+OS8Bti7WKrYaz/RISvflA1fHkPVAvzlpoS2E5MCNv7blKy0G5Bb9oQ0IUXas8D8nWp3ScNWX4+YII2Zbb+fc1yyKSecGWCAsdUZ0uXlUuefHvvzBcOBLYf+DgXIfZ44U350Zwj2Nl78e+n7Z01J9XBa86pZSbl9qn275ZN8ByU1rOy3a3Neh6vMxLlu4+q2eZOkAU9Lff4Q/uuVJ/7NfFbr6sDKvgOba8r2xopev5Mu2iGlda7bltXb7I3d9eawh9UoKTlmixGPx7wX37NQ8pWb3O112f/f4+B0zmOwD1HB7/dr9+7dDd/c0uM1b8JiSdefSWd8zv4RscDyhXffAUSNRuc8hJ9VIRWvkNa+Yr8fUtvzzT4f8TarZgo+kyZ2kza8FXurJhD9Cuea0n4Fn4X8qch5MhfYVr1g3+aNNxdDQ1FaPiHNzNoNd9l6q1LauyjwYN7umWZPyQ4/Cv5zn/SOdMoHpu/hVNXXQINqTul1j7mQHmiGf5tDZagr9oUtpJDrfb804m0pe4B9+4QU6Z006eDWkDw9Oa8BRn5qSn8HWmUZogFZx7S7zOzXFmilbSiN/MRcZ3grzpTdPFxaBzPoFn9E3vRXmIGpw8sQFnwuTf+xtHdpzbHkFoFXtcSnmX3+Cufat++aYVbh+ssb3KXjkpVvVo3aldeWzD5ds2+UVr8c/Oc+43PpjC/M6tVwSsmVcs8xfyftdL/d/P1sYFUS1+Y8j8cMZNsNcnvjzSBgVl9nVnA6JedM6dQPpaYxVAEmPsXkuEATIArnmjxR5tCe0Q1RUWxym69UWvGU2ZO1bJe0+zsz2QPHxOmcx2AfooJlWdq0aZOshr6ZH/J/0pgQzAgGjlerM8zvZ5tznY4EEajROQ/hl9JaOu1jqd9jZobmkfw+c8HEbuWeZZkLHCf/x+wNFCu+OMfsA7FtitOR4HhUlfgJdJEriMh5kmb9RJrzc1My7Eh5F5k9cvatCbwXZmOtfln6ZFCjSnIdl643Sal5gQc4Fz9kLv4mtwj+c7c5X2o2xH71SkhZUmq7yNsHrmKf9EGeKW1q56S3zQqA3DHhjSuUWp0mtf9h4Pa0QxfYjxzYCBJyXgO0PtOUcbQrlVg435RRD/eKr1CKSzQr6uNs3lOGWlK22Yu63WVS9hGrzNtdKp020Qw+zbhGeq+lOW75pcUP1B7sK1kprf+nVHqM+6JaPnPhv0k7+/bvnzfvpe3+PoZSfKoZ7AuUB75/ztx2ujb4z52UbfJUuOWOkU77KPAerUnNzETGBiLnBWD5pPdzzKr/cFr3L/NZzYlKA6UFZm/3da+H/7mdUF4kfX2RmRhwrDnRSSufM5ORi1eYgf8+f5ASM6VvLqo7CQQBOZ3zQrTrOQCgXvEpUufrnY4CQDDknmW+7Hz/nLT6JWnfurr/51ufKZ0bQ5uTV9k5zdxW2qx0RPRIbSe1vdCUl0TodfqJWSFctst+Jv+2T6UZPzYDRW3GBu95Cz4zM46d+P/a9+HAewWe8oFZbegrC/5F76V/kpb8Xhq7VMrsGdzHrk+7S8xXpDmw0X17ge/faEqTdr1Fyr+vbvt5q8IfE+xtfNesNLD73FS2S9r2caMGH2Bj33pTovGkowwANGkvZR1aoexNlM5ZLCU2rWnvfKPU8VrTdiy8CdIZAcpGSlJWL2mDTHnJcK6M9vukYa8F3pOv3SWmbF+zMJfBDiXLMp9d0rvF1gSPSPXpUHPbenR4n3fdP83ttk/rn/gSCnGpZoC82eDwPq9TSreb91nZA81nq0jXYoTZfiSphan4UPW7mf+gmVyDqMBgHwAAQGP5K6Vtk81eC3Yl0Mp2mQvWA5+1/3lfubR7tpSaK6V1CmmoYXOZzebkiD6JmdIp/zn6eQiOweOlgc+YiyBHKv5e+uLQRbc+fwzu8w77p/kAn9kjuI97NMUrzQBjy1MDnGBJn50qDXpeOuHnwX3uJb83twlZwX3co9mzyJRD6vtHqddvw/vc9dn2qblte4F9+7dXmnLTF2yOnX3Q544zF+ASMpyOBEcz7TJzazfY1/pM6cTxtQea0Hgzr5WKlkkX76jbtvNbs3qv68+lPr+vOe7xmD2pDueNk7wBJnI0RrvLzN/GFiOC95jHomSl9FEvU/a3z0N123NGma9Ysme+NPc2KaOHlLvM6Whin79c6nxDaFaH1mfwi1LJqsArOENp/3pp7WuxVcazPqltpFM/MhP5wl0+vjFanWa/qrjzT8IdCY4DZTwRNdLT050OAQDChpwXJXwHpa9+IH3c3wzcHanvH6UrfFJm97ptBzabFSafnSyt+EvoYwUimOtznr/SlEOz2+9j70Jz2/mG4M/AjksM/0CfZC4cz7lJKlpi357cWup0XWhW3rW9UOpyo5lkEU5Vr61dyWcn5ZwhDX7JrFCxs+Etc1vyffhiCrWm/czAQdeb7Ns/6m1KnIWw/JLrc96xGj3T7NsXyILfSCv/HLZwYlqna6VO10izb5I2/692254F0oqnpQOb6v5c2W6pfE/N/QNbpF2zgrfPZ3oXqfs4KaNbcB7vWMWnm0khKTn27aU7pbcSpO9uC2tYIbVvjbkNwSp0cp6NcxZJQ14K//OmdTCTJZKyHXjuTlL/p6QWJ4X/uZ0Q30Rqc46UcYLTkTTcwnulT0c4UPY+NjiZ8xjsQ1SIi4tT586dFRcX53QoABBy5LwoEpciJWQGbvf7zGoOuxJpRcvNCpOMHlLehaGLMdy2TJKmX21KlwLHgJwn6eO+0r+bmrKVR2p5mjTqG1P20m4wMBq1PrRScWWAiQ7TLpZ2zwrNnkWn/MfMag+3jB7SiLcir4x7WmezCn3vIvv2H6yQRkyQsvqGN65Qyr9fOnNa4BKxVSv1PZ6QPD05rwGaDzEXpe2U7zErzbrfEd6YYlWna6VO10urX5R2z6nd1vFq6fz1UvOh0ppXpaWP1LR9kCdN/1HN/fX/MuUJi1cEJ67FvzeDakXLg/N4x6pJnnTSO1KHq+zb1/xdsipjq7Rd7jnSeeuknr8O6sOS8wLwV0rfXCyt+HN4n7e8SNoxTTq4PbzPK0m+Umn1eLMvs1t890vp7URn/r0bautk6fPR0q7ZUtlOsxLT45W+uUT66jyno4saTuc8BvsQFfx+vwoKCuT3M6MAQOwj50URb7x06V7pSsuskDlS4RxzEX/uuLpt2QOl0z+TTp8SmgvaTpl2sbnQ46YPcTgu5DxJ+9aa24Nb6rYltzCrkJKbhzemUEptY0p4JgdYMZE7VmodYC/U47X+bWnmT8xqlHDyeMzKzMSs8D7v0VTukxb+tu5KnioZ3aT2lzmzAiBUSndKi/8Q+O/U8H+Zv+shQs4Lkr2LpSUPmT0YERzpXaWLd9Ud7EnIMHv1xSVL6/8pLX+ypq3LjVLrs2vutxwp9X9SSs0LTkzFy82gWuHc4DxeQ7Q+M3C5X2+C2Uc3lsrbxaWa1y3IA5jkvADW/8uUxy38LrzPO/NaU1lm3rjwPq9kBo7iUsz/Hzc4uM3sg+mvkOKDWN44VMp3m8l25XvMxLgLD30u8ZWZgVocE6dzHoN9iAqWZamgoEBWCEupAECkIOfFkE2H9jxrdmLdtqRsUz4tVvZAqjLk71LuD2JrABMhRc6TGfTv+7CU2My+/cuzpQk2+/lFs1FfSsNes2/rcKV0cKu0/cvgP++sn0hrXzVlmCEVfGZumw+zb981W1rzitljMVasfU1afL+05SNHnp6cFyTpXc3FyFanOx1JbJj/a+mrsVJSs7p7S1XsM5NSKvdLQ1+TRk+vaRv4Z6nbrTX3mw+WetwppbQKTlx5l0hpXaTUtsF5vGDpcad0/jpTZjRWHNwivR0vfR3ciiPkvAAW3Sc1G2ImmIRT7tjat+GU1lE6Z6F0wi3hf24nxCWbFbMnjo+OfYI7XCldWiTljql9/LQPpdPrKamNWpzOeTFSBwYAAMAhb3okb6L0w9K6Jb86Xi1lD5JyA6xQObDVXMTveI3UI0bKUHW40nwBOHatRpovOzumSdsmhzeeUNsySfr6fDM5oNOP67Yf2CpteFvKGR38iQOpbaXKA1JKmPfsi1RxKaZEZ1oH+/bpV5iL/M2GSFm9whpayBQfKgfY8Uf1n4fIltJamvcrM1B9eozlSCcc2GhWz+1dbPJj0mGTTzb9R5p5jXTK/6S254Y3rnYXm69IU7pLWvpHKWeU1OYHTkcTHJ5DJeeaD3c2DrcY/ob5GxxuXX5qvhB6iU2l05yZWNRolmUmxO34RvLtl/IucjoiNBCDfQAAAMfLX25/PCvffNnZNVP69NBKCqsiNHEBiA6rDu1fMuwfdfflS24ltb9c6nC1M7GFQlJzU5Yt0J5O+9ZIXW+WOl0T/Oc+9UNJHlNKClLb883Faq9NKWpJOuNzqWBq7Az0SVL+Q1K326SM7k5HguPV9gJTahbHb8Rb0s4Z0qQ+Uv+nak9Cy+ot9bhLSu9sBrmkmtLSc34uJWRJ/R4291c+Z1bPnvK+1KTd8ce1/Stp8/tSt19KaZ2O//GCZec0s+9s5b7YGexLaS1d4ePvY7i0OMlUMNi7NLb+xqK2ta9Jexaa8sbeCN+38uB2afpV0vapUkKmKembd5G0+UNT2tNugh4iDhkcUcHj8Sg7O1ueEG2SDgCRhJwXZc6aK506se6qPkkq3ytNOVla/nTdtsRsqe2F0in/rbs3SjSb2NOsdlz7D6cjQZQg50la+ay04S37/TAyupqLsG3OCX9coVJ1cb60wL5922RTajMUFxy9SVJFseSvDP5jR6MDm6R3M8y+fXaatI+tPakkKSXHrPC0fI48PTkvSPYsknbNMPuaIjhS20j5D9Yt65s9QOr/uJTZU/ryLGnyoJq2LR/VlAOWzIqQ8j2SglS+bNvHZlCtaHlwHi9Yqiar5F3qbBzBFoK/u+S8APyV0udnSEsfDu/zLrjHfFZb9UJ4n9eNfKXSzOuklX+OjkH0PfPNQJ9kcv6Qv5vvVzwpzf+Vc3FFGadzHiv7EBW8Xq/atQvCrDAAiALkvCiTPSBw28q/mJm/cal1y3RmnCCd8p/QxuaE5FamRFpyS6cjQZQg58nsg1Gyqu4+SVWW/EmSJfW+N6xhhUx8mnTWdzUlw4404Gmp5HupbHftUnLBsPB30oY3pUv2SIlZwX3saHRgi7lNzHY2jnDaNUP67FSzT2ave8L+9OS8IPHEmVUHgValomF2zjCD//kP1H9eux+a1WxVzl9X+yJ2z7vNV7B0+ZmZoBFpe0G3v0zK6mP2KES9yHkBfDrU3Ha+LrzPm3eRtJr9TsOiamJZzij7icGRJqtPzfddbqz5vv9Tku9A+OOJUk7nPAb7EBX8fr82b96stm3byuuNgtkQAHAcyHlRZuWzh8rs/bBuW3o3qeWp0sA/2/9seZG0cYLUtL/U7MSQhhk2o75wOgJEGXKezCqj1Lb2bWW7pUWHBvliZbDPGydlDwzcvnu2NONq6eT3pbwLgvvcG940t07skxOJDmwyt24qhfj98+bWocFecl6QZPUyZfB8ZU5HEhtWPCVtek9qd2ndi9I7v5WW/NFUouh5V+22UK9WSesonfi30D5HY6Tkmgv5VpBWMMYwcl4ALU8z++HmjArv8zY7UbqkMLzP6VbxqdJ566SEdKcjOTapudKVVt281myQ/fmw5XTOI8siKliWpcLCQlm8kQLgAuS8KDP3Nunby+3bOlwujfpSyuxRt+3AVmnapdLsn0mb/xvSEIFIRs47Ct9Bc9v+SmfjCKcm7aSOPzYl44Jt4HPSCb+Q4pKC/9jRqNXp0pnTzEVHt2j/QzNjve1Fjjw9OS+IVr8gbfvE6ShiQ/c7pW63m1WvWyfXbivbLe38RirdUffnCueZfair7PxWWvOKVHkwtPE6rWi5NLEbpRCPATkvgIFPS4P5/YlpHq9ZhV5R5HQkx277l9L7rU2p18/PdDqaqOR0zmOwDwAA4Hi0PV/K6G7fVlEsrX9L2ru4blv5HqlgitRihNTxmtDGCCB6peRKF++Whv7d6UjCZ9P70rp/hGb1XbdbpUHPBv9xo5VVKa0aL+340ulIwifvImnwi1JKK6cjwfEo3SX1+LUp84jj12KY1HqMVLJSqiyu3db2POmyfaZ05ZxbpXl31rTNul6aflXN/XX/NMcq94cnbqdUDTJT2g6NdXCb9L+u0rLHnY4EofTZqdL/OjsdxbHZNVOaOlIq3W7KZGf2Mse/vlCaQEWMaEEZTwAAgONxygeB2zb/V5rxY6nDj6Th/6zdlnGCdOFWKT498D5dAODxSge3SolNpdQ2TkcTHm3GmvLI8U2cjiT2lRdJG942qynbXeJ0NOFRtFxa/JDU6Topd4zT0aCxSgukZY+YVbq5ZzkdTWzIHSNdtL3+cwqmmNJ0VXr+RvJX1Nw/4RdSm/OkhIzQxBgpmvaTcs+R8i52OhJEq5XPSftWB96/GLFh/zqnIzh2qYf2mRvwtNT99prjzU6UPAwhRQteKUQFj8ejnJwceaJhQ1MAOE7kvChTeVCS3/6i9MGt5tZuPz5vgpTSOqShAdGAnHcUll+alG8uLJ493+lowiNnVPj3sHGrtE5mpVvuWKcjCZ/VL5n9clPbOjLYR84LkrRO0pnfSql5TkcSG+bcYv5fnL9Jij9iBUfpDrOXalYf6dyVtduO3LM6q5f5inW5Z5svHBU5L4DCuWbSZ487j34uolePu6Ws3k5HcWyq9uw7Uq/fhj+WKOZ0zqOMJ6KC1+tVTk4Om/kCcAVyXpR5J1V6J8DKvM4/lc5ZInW+vm6bv0Ja/7aph7/00dDGCEQwct4xaNLerCAAgi0uUTppgtRiuNORhE9yC3N75CBFmJDzgiQ+1fzeNmGwLyhS20reZGnjO9LBI1b3Fc6VvjpXKvjMmdgQ1ch5AQx/Qzp3ldNRINT6PyZ1vNrpKI5d6U5T/eCTE6UNE5yOJio5nfPItIgKPp9Pa9askc/nczoUAAg5cl4MSWpmZjcHWvU3/QrzfXqX8MYFRBBy3lF4vNL566W+f3I6EiA29Pi1dHmFlD3Ikacn5yEi9brHlOScea1UvLx2W1YfaehrUouTpR1fm8G/KrNukD4ZXHN/+tVmb6fDS3vC1ch5AexdZMpol+9xOhKE0szrpFk3Oh3FsSvfIy1+UCr8Ttoy0Rxb9y/pu1+Q14+R0zmPwT5EjZKSEqdDAICwIedFkQu3SedvtG/bOd2s3Fv+VN22hEyp173SaR+7Z58kIAByHoCwqSiSds2QSo+yN1kIkfMQkXLPlkZMkDKPKMOZ2kbqdI2U0VWadqk09/YjfvCwsm9ZfaRWZ7APGWoh59lY+6o0b5zkK3M6EoTS2tekNS87HcWxa9Je6veoNOT/pP6Pm2MFU6Tv/yr5y2ufu/Vjaeooafd34Y8zwjmZ89izDwAA4Hik5ARuq7rQ4U2q25aYJfX9Q0hCAgAAAWz71Kysz/+9lH+f09EAkWHLRGn3HKnnr02J1ED6PSElZtbcH3LEReyed5kvAPVrfZb5rJjcyulIEEoprR2rJNAo3gRpwW+k3B/UbEUy4Bmp32NS3BH7uW7+r7R9qlS2M/xxIiAG+wAAAEKl+RDp0hL7Mp6StG+ttOIvUt5FUqtTwxsbAABu1OxEqUlHqe35TkcCRI6tk6RVL0gn3Fp3sG/XTGnaZVKfP5gVfgCOX8erzBdi24VbnY6gYaxDK7W3TpQqSqSEdCkp2/7cg9vMbTQNZroAZTwRFTwej/Ly8uTxeJwOBQBCjpwXYxLSJLvXsqJY+rCb9P2z0t6F4Y8LiBDkPABhld5ZOn+t1LSPI09PzkNE6nWv1OIkaWI3qfJg7TZvopScY7/ir+AzaeVfJctv7q9+SVpwT+jjRdQg5wFRxBsn5Z5jvl/2mLkt3SkVrai7Z1+XG8zt4gfCF18UcDrnMdiHqOD1etWsWTN5vfzKAoh95DyXsPxSXJLU7jKp00+cjgZwDDkPgJuQ8xCRUnOlnNFS0/5SXHLttuwB0lmzpXaXSh/lS3N+XtO29jVp7i9qLgJvfM/s7QQcQs4DosyQv0vd7zQTQCRp6cPSRz3q7nWcc6ZZDd7ytLCHGMmcznkey7Kso5+GYCsuLlZmZqaKioqUkZHhdDgRz+fzadWqVeratavi4tjoGUBsI+cBcBNyHgA3IechIlXulyr2SUnNzcqOQD4eYC4AD3rW3C9aZlZ9tDjJ/FzpTsl3UGrSLjxxI+KR84Aot2WStGuG1ONOKTHL6WgiXkNzXrDHiNizD1GjtLTU6RAAIGzIeS5RuV86sMVszJ6Y6XQ0gGPIeQDchJyHiLP4QWn5k9J566S0DrXbDmyV1v/LrN44e17ttsye0uFvYZNbhDZORCVyHhBFZv/MlGS+bL8p39zmHPN1pMlDJXmkMTPCHmKkczLnsYYaAADACX6fNPd2szfKxnedjgYAAABu1Xy4FJcirXy2btuBzdKCX0s7vrL/WcsvVRUNO7BZ2r8xdHECAEJr9Uvmdves+s/L7CntninNuSX0MeGYMdgHAADgBMsnrXnZfN+0n6OhAAAAwMXyLpSaDZHW/6NuW1Yvacwcqf1l0oo/SwWf17TNvV16K046uM3c/+ZS6dPhYQkZABACJ70j5V0iNR9h7q97Q5p6ulSypvZ5Q1+Rmg6Qdn4T/hgREGU8ERW8Xq86derEhr4AXIGc5xLeeGn4W1KT9lKzQU5HAziGnAfATch5iFjD/mH22ztSfBPzXrVstzTvdqnLz6Sc001b035Su0slb6K53/FqqaIobCEj8pHzgCjT7lLzVaXke6lwnlnFfaSz54YvrijhdM7zWFbVWnuEU7A3XwQAAAAAAAAabOO70oYJ0oCnpSbtardZfqnygPl+70IpqYWUcUL4YwQAhN+BrdJ3t0r9HpEyutUcX/JHKaWN1Pk652KLAcEeI2JaBaKCz+fT4sWL5fP5nA4FAEKOnOciO6ZJU0fVLocEuAw5D4CbkPMQkYqWS5vek0pW123bu0R6N11a8YzUYgQDfWgQch4Q5VJzpf0bzIDf4ZY/Kc27Q9o2xZm4IpTTOY/BPkQN3hgAcBNynkt8drK0fapUXuh0JICjyHkA3ISch4jT+z5TknP2z+q2JWVLna6VsvIlf4V0eIGwrZOl2TdL+zeZ+7N+av8YcDVyHhDl4lMk64j/x2fNlZJbStMutf8ZF3My57FnHwAAgFNanS41O1Fqd4nTkQAAAMCtPB6p/RWSr6xuW2pbaeir0p6F0tuJUp8/Sr1/Z9r2zJNWj5e6/FRqkicVzpc8ceGNHQAQWmdOq3ssvbM08M9S2a6wh4PAGOwDAABwyhlTnY4AAAAAbrd/k5Q9SMoeGPichHQzIJjVu+ZYt19KnW+QErPM/bPnhjRMAEAEsCzp4Fap+XApMdPpaHAYyngiKni9XnXr1k1eL7+yAGIfOc9F9m+UNr0vHdzmdCSAY8h5ANyEnIeItHGC9PkZUvHyum0Ht0szrpF2fyeNeFNqe35NW3wTKbm55GUtAeyR84AYUDBV2vjvmvv+CumDttKcnzsXU4RyOueRaRE1EhMTnQ4BAMKGnOcSX58vfXORtHuO05EAjiLnAXATch4iTs4oc7tqfN22yhJp3T9Myc4jle+R9i6RKkrM/a2TpZ3fhi5ORCVyHhDllv6p7n6s3W6Tdn0r/bejVLrDmbgilJM5j8E+RAW/36/FixfL7/c7HQoAhBw5z0X2LDC32QMcDQNwEjkPgJuQ8xCRmvaTmrSXSlbVbUvrJF12QGp7gTTrRrPCr8rGd6VJ+dKuGeb+zGukBb8OR8SIEuQ8IAbkPygN+0fN/bhEs19f7/uk5FZSRbFTkUUcp3Ne1Az2/elPf9Lw4cOVmpqqrKws23M2btyosWPHKjU1VS1bttRdd92lysrKWud8+eWXGjBggJKSktSlSxe99tprdR7n+eefV4cOHZScnKwhQ4Zo9uzZtdpLS0t1yy23qFmzZkpLS9PFF1+s7du3B6urAADALcYulc5bI6W2dToSAAAAuNn566XRNqvyPF4pPkXat05a87J0YGNNW/YgqfcDZkBQkgb9Vep1b1jCBQCESctTpDZj6x7vfL00ZqaU3iX8McFW1Az2lZeX69JLL9XNN99s2+7z+TR27FiVl5dr+vTpev311/Xaa6/p/vvvrz5n3bp1Gjt2rEaOHKkFCxZo3Lhx+ulPf6rJkydXnzNhwgTdcccdeuCBBzRv3jz17dtXY8aM0Y4dNctRb7/9dn344Yd699139dVXX2nr1q266KKLQtd5AAAQmzJ71lwcAQAAAJywYYI0sYe0a2bdNl+5tGuW1OxE6ZJCKfewC77ZA6Q+D9Zc6G13iZR7VlhCBgA4pGSN9NV50qb3nY4ER4iawb6HHnpIt99+u/Lz823bP/30Uy1btkz/+te/1K9fP5199tn6wx/+oOeff17l5eWSpPHjx6tjx4566qmn1KNHD91666265JJL9Mwzz1Q/ztNPP60bbrhB1113nXr27Knx48crNTVVr7zyiiSpqKhIf//73/X000/r9NNP18CBA/Xqq69q+vTpmjnT5k0RAABAICufk95OlnZMczoSAAAAuJZHKl5RU2L+cGU7pU+HSt//VUpsKsUlhT06AICDZt0gvZsp+Q9VUCzbKW2dZAb7lj4q7d9Y/88jbOKdDiBYZsyYofz8fLVq1ar62JgxY3TzzTdr6dKl6t+/v2bMmKFRo0bV+rkxY8Zo3Lhxkszqwblz5+qee+6pbvd6vRo1apRmzDD1x+fOnauKiopaj9O9e3e1a9dOM2bM0NChQ23jKysrU1lZWfX94mJTy9bn88nn80mSPB6PvF6v/H6/LMuqPrfqeNV5Rzvu9Xrl8Xhsj0uqUzM20PG4uDhZlmV7/MgYAx0PVp88Ho/y8/NlWVattmjuUyy+TvSJPtGn4PTJsiz17t07YF+jsU9VMcbS6xSMPsXN/aVpiE+JmT4dHiN9ok/H0ifLstSzZ095PB5Jiok+HR5jrLxO9Ik+0afg9Kkq51mWJcuyYqJPR8ZIn6KwT20vljcuRdoyUerys9oxetPk7feYlJwj/+75Unq36gG/uN3TZS28V/6e90g5Z8r7UTdZzUfIO/x15/t02HEpRl6nKOzT4TnP5/PFRJ9i8XWiT/Spvj4ptYM82YPlr6yQ4jyK2zBBVlyKrKb95Z1/h3zZJ8qT0jaq+hTK1+nwnHe0Ph0Zx/GKmcG+goKCWgN9kqrvFxQU1HtOcXGxDh48qD179sjn89mes2LFiurHSExMrLNvYKtWraqfx84jjzyihx56qM7xpUuXKi0tTZKUnZ2tdu3aafPmzSosLKw+JycnRzk5OVq/fr1KSkqqj+fl5alZs2ZatWqVSktLq4936tRJGRkZWrZsWa1fom7duikxMVGLFy+uFUN+fr7Ky8u1cuXK6mNxcXHKz89XSUmJ1q5dW308OTlZ3bt31549e7Rp06bq4+np6ercubN27NhR698hWH3q2LGjEhMTtWrVqlr/SaO5T7H4OtEn+kSfgtenNm3aqHnz5jHVp1h8nY63T61a3Cyvx1Kr7IEqKS6OiT5Jsfc60afQ98nn86lLly7KzMyMmT5Jsfc60Sf6RJ+C0yefz6e4uLiY6lMsvk5u61N2y7vUqn1/xfv9Nn26U/4Fv1XCzGu0vPP7KkvqYPrUbL+somXauGaxinbmqIs/Q+UHvMqWIqJPsfg6RWOfDhw4oLi4uJjqUyy+TvSJPgXsk2+s1GystGyl6dPKP8sjaXlJN8V1fENlW5OVVbE5uvoUotcpOztbK1eurBV7fX06cjDyeHmsYA8fNsBvfvMbPfbYY/Wes3z5cnXv3r36/muvvaZx48Zp7969tc678cYbtWHDhlr77x04cEBNmjTRpEmTdPbZZ+uEE07QddddV2vl3qRJkzR27FgdOHBAe/bsUZs2bTR9+nQNGzas+py7775bX331lWbNmqU333xT1113Xa1VepI0ePBgjRw5MmB/7Fb25eXlqbCwUBkZGZKib1Q/nDMVLMvSkiVL1LNnz+o3CNHep1h8negTfaJPwemTz+fT0qVL1adPHx0pWvtUFWMsvU70iT7Rp+D0qSrn5efnKz4+Pib6dHiMsfI60Sf6RJ+C06eqnNerVy8lJCTERJ+OjJE+RWGfilfKs/0zedr8QErraN+n7V/K2vGVrK6/lBIzI79Ph8euGHmdorBP5eXl1TkvLi4uJvoUi68TfaJPDerTN+fLSussf/+nY6dPQXqdLMvSokWLqnPe0fpUXFysrKwsFRUVVY8RHQ9HV/bdeeeduvbaa+s9p1OnTsf0WDk5OZo9e3atY9u3b69uq7qtOnb4ORkZGUpJSVFcXJzi4uJszzn8McrLy7V3795aq/sOP8dOUlKSkpLq1jWves7DVf2S2p0b7uMej8f2eKAYG3r8WGOp+s9g9+/VkMeRIqdPwTxOn+hTsGJs6HH6FLo+eTyeBp1f3/FI6VMwj8dMn3bNlAqmSp2ukSe1bWz06TAx8zodhj6Fpk8ej4e8d5Tj9Ik+BSvGhh6nT8HvU9Xzk/fqP06fwtinvfOkebdJae2l9E61zy8vkqacIeVdJE+fB+s8RsT26TiO06fg9qnquQ8/J9r7FKrj9Ik+BSvGhh6vt09bJkk7v5F6/sZM9jhtojyS4vyVku+gFJcseRNqzg9z7IGOO/E6+Xw+25xXX4zBZB9pmLRo0ULdu3ev9ysxMfGYHmvYsGFavHixduzYUX1sypQpysjIUM+ePavPmTp1aq2fmzJlSvUqvsTERA0cOLDWOX6/X1OnTq0+Z+DAgUpISKh1zsqVK7Vx48ZaqwEBAACO6vPR0qJ7pQNbnI4EAAAAbtViuLndOsm+vWy3VLGv7vHSXdKGd6TilZLfJy19WNr8YejiBACE3/bPpWWPShV7ax/fMEF6N0Pa8pEjYaEuRwf7GmLjxo1asGCBNm7cKJ/PpwULFmjBggXat8+82Rg9erR69uypq6++WgsXLtTkyZN177336pZbbqleUXfTTTdp7dq1uvvuu7VixQr97W9/0zvvvKPbb7+9+nnuuOMOvfzyy3r99de1fPly3Xzzzdq/f7+uu+46SVJmZqauv/563XHHHfriiy80d+5cXXfddRo2bJiGDh0a/n8YFwk0cg8AsYic5xJN2pvbrN7OxgE4jJwHwE3IeYg4Sc2lrL5Scuu6bYmZ0vnrJG+i9OkIyVde01byvfTtD80goVUhLfydtPGd8MWNqEDOA6Jcz7ulc1dJKbmS5ZcWPSBt+o+U3lnqdJ3UJM/pCCOKkznP0T37GuLaa6/V66+/Xuf4F198odNOO02StGHDBt1888368ssv1aRJE11zzTV69NFHFR9fU630yy+/1O23365ly5apbdu2uu++++qUEv3rX/+qJ554QgUFBerXr5+effZZDRkypLq9tLRUd955p9566y2VlZVpzJgx+tvf/lZvGc8jFRcXKzMzM2j1WAEAQBTy+ySPVwpy6QYAAACgQaouDwZ6XzrnVmnzf6TzN0reQ9fZynabkvRN+0npXaQ9C6XELCmtYzgiBgCEm69UmpAidbhKGv4vp6OJesEeI4qawb5Yw2Bfw1iWpZKSEqWnpwe9li0ARBpynotU7JOsSikhwwz6AS5EzgPgJuQ8RKRN70vfXCQN+5fU8arabf5Kad0/zQqOlqc4Ex+iFjkPiAEV+0wJz6SWZrJH8UopvonUpJ3TkUWchua8YI8RcVUJUcHv92vt2rXy+/1OhwIAIUfOc5Fvr5D+3VTav97pSADHkPMAuAk5DxEpuaWU2lZKSK/bZlVKs34izbxe2jUz8GNYfql8r1n1ARxCzgNiwKq/SR/kSXsXmknKmT3MQF/pDnNNY90bTkcYMZzOeQz2AQAAOGXrRHObkOVoGAAAAHCx5sOlwS9JLU6q2+ZNlE79SNq3Wvp0WO22kjXSey2kJX+SSneaSWzz7gxPzACA8MgeJHW7TUpqYVZ7H9wuVR6UfAelDW9Le+Y5HSEOiT/6KQAAAAiJ0yZJ+9aavU0AAAAAJ3g8Uu7ZAdq8gdviUqTMXlJyKykuSep8vRk4BADEjpzTzZckFS2TPuol9b5fyn9QurxC8sQ5Gh5qMNiHqJGcnOx0CAAQNuQ8lwh04QRwGXIeADch5yHq+CvMbXxa7eOpudKoL2vuD/m/sIWE6EHOA2JIQobU9edSs8FmooiH4aUjOZnzPJZlWY49u4sFe/NFAAAAAAAAIOjeSjB79506UWoz1uloAADhtP1LacUzUq/fSc0H1xz3V0q7Zph9XzO6ORZeNAv2GBF79iEq+P1+7d69mw19AbgCOQ+Am5DzALgJOQ9RqfUYqe/DdQf6Kg9Kix6QNv9XOrhNmv4jaeO/nYkREYmcB8SAg9ukbR9LpdtqH/cdlD47RVr2mDNxRSCncx6DfYgKlmVp06ZNYiEqADcg5wFwE3IeADch5yEqnTZR6nVP3eP+cmnJ76XN/5PK90rr35D2LAx7eIhc5DwgBnS4Qrq8XGp7vrRrtvTNJdKOaVJcsjTwL1KHK52OMGI4nfMoqgoAAAAAAADAXulO6YvRUoerpB6/qjkenyads0RKypaSc6QflkryOBYmACDEDmyUNr0ntbtM8iZI3X7pdEQ4DIN9AAAAAAAAAOwtuFvas0DKu7j2cW+clNWr5n5cUljDAgCEQVmhVPidlNFdaneJdIVfEqt1IxFlPBE10tPTnQ4BAMKGnAfATch5ANyEnIeos2GClHuO1Pve2sctSzq43ZTwrCiRdk43ezsBhyHnAVFu70LpizHS1o/MfY9H8hwaVvp4gPTdL5yLLQI5mfM8FkWTHVFcXKzMzEwVFRUpIyPD6XAAAAAAAACAuvZvkOJSpOSWtY9bfumtOFPOrfvt0qfDpP5PSj3udCZOAEDwHdwmbZkotRghxTeR9q2TsgdICRnSpD5S8xHS4BecjjIqBXuMiJV9iAp+v18FBQXy+/1OhwIAIUfOA+Am5DwAbkLOQ1RKyJQ2/08qnHdEg0fqerPUerSU2k7q/5TU6jQnIkSEIucBMSCltdTlBimzp7TxXWnqSKlomWk7ZxEDfYdxOucx2IeoYFmWCgoKxEJUAG5AzgPgJuQ8AG5CzkNU2jVDmn2DtPm/tY97PNKJf5M6Xy+l5ko97pCyBzoTIyISOQ+IEQe2mJXcC38rDfqrlNbJ6YgiktM5L96RZwUAAAAAAAAQ+b6+wNx2+JGjYQAAHLBvrfS/zuZ7yy+dcEtN29p/SAnpUt6FzsSGWljZBwAAAAAAAMBer99KJ46XMrrWbfv2Smnpw9KOadLH/aXNH4Y/PgBA6HgSat/fv6nm+4W/kZY/Gd54EBAr+xAVPB6PsrOz5fF4nA4FAEKOnAfATch5ANyEnIeolP9A4LbN/5UqiqTmw6SKEsmqCF9ciHjkPCAGNMmTmg2Rds8y9z87VRozS0puIY2YIMWnOhtfBHE653ksiiY7ori4WJmZmSoqKlJGRobT4QAAAAAAAAB17ZotfXaS1O9xqfu42m2WZfbuAwDErrWvSzOvrbl/SaGU2NSxcGJFsMeIKOOJqOD3+7Vx40b5/X6nQwGAkCPnAXATch4ANyHnISp9d4vkr5CadKjbxkAf6kHOA2KAr1wqWlJz/+T3aw/0sZasmtM5j8E+RAXLslRYWCgWogJwA3IeADch5wFwE3IeolJFsdRqpJR3Qd22HdOkwnlS8Spp9f9J+zeGPTxELnIeEAssaflTkjzS+Rtq/y34eIA0sZtTgUUcp3Mee/YBAAAAAAAAsHfuysBt31woZfaUOl0vzb5BOvVDqUm78MUGAAituCTp9Ckm1+/4Rtr+udTpWtPWaqTkK3M0PNRgsA8AAAAAAACAvaIV0uoXpfY/lJoPrd3W73FTzq3ZYOm0T6Ts/s7ECAAInZwzpNId0rc/NPerBvsGPOVYSKiLwT5EBY/Ho5ycHHmoBQ/ABch5ANyEnAfATch5iEprX5VW/lnK6l13sK/zdTXfp+aGNSxEPnIeEEP2rTW3bS90No4I5nTO81gUTXZEcXGxMjMzVVRUpIyMDKfDAQAAAAAAAOr6IE+qPGD2akpIczoaAIATyvdKu2ZKGd2ltA7m2Pd/k0pWSwOfdjKyqBXsMSJvEGICQs7n82nNmjXy+XxOhwIAIUfOA+Am5DwAbkLOQ1Qa+Kw09DX7gb7PR0vfXCytfFZ6O0naOT3s4SFykfOAGJKQKbU8WWrSvubYsseklc9IK/7sWFiRxOmcx2AfokZJSYnTIQBA2JDzALgJOQ+Am5DzEHVyRpmVHBX76rZ5vJInzlz8bT1GSswOf3yIaOQ8IEaU7ZbeSZO+u7XmWIcrzG3BFGdiikBO5jz27AMAAAAAAABgb+vH0rc/lEZMkNpfVrtt5Cc137c9P7xxAQDCx6o0t564mmO5Y6W9i6We9zgTE2phsA8AAAAAAACAvXnjzG1mD0fDAAA4KCVHurREik+tOdbyZPOFiMBgH6KCx+NRXl6ePB6P06EAQMiR8wC4CTkPgJuQ8xCVWo6U/GVSVn7dtvVvSZbPlO8s+Ezq+WsppVX4Y0REIucBMebIvVsrD5o9+7L6Sm3GOhNTBHE653ksy7IceWaXKy4uVmZmpoqKipSRkeF0OAAAAAAAAEDDTOoj+cqkvIulZY9IP/heyujqdFQAgHAoL5L+nSV1vl4a8n9ORxN1gj1G5A1CTEDI+Xw+rVixQj6fz+lQACDkyHkA3IScB8BNyHmISls/kT4fI+3+rm7bkFek4f+Sev1GOn+jlNYh7OEhcpHzgBgXnyadNVfq/YDTkUQEp3MeZTwRNUpLS50OAQDChpwHwE3IeQDchJyHqLP8SWn7VKnn3XXbmg2q+T6BylWoi5wHxDBvnJQ9wOkoIoqTOY+VfQAAAAAAAADsVe6TUttKOWfUbbMsye+TDm6XSlab7wEA7lFRbL7gOFb2AQAAAAAAALA39BXJX2nf9ulw6cAmKfccac3L0iWFUmLT8MYHAHDOf1pLLU+RRn7sdCSux2AfooLX61WnTp3k9bIYFUDsI+cBcBNyHgA3IechKnkTzaq91LZSUnbttpxRUtkuqfVo0xaX4kyMiEjkPMAFOl8vpXV2OoqI4HTO81iWZTnyzC5XXFyszMxMFRUVKSODmuYAAAAAAACIQEv+JC26Vxo9Q2o+1OloAACICcEeI2JaBaKCz+fT4sWL5fNR+x1A7CPnAXATch4ANyHnISp9/6y5TevibByIOuQ8AG7idM6jjCeiBm8MALgJOQ+Am5DzALgJOQ9Rp/ud0oHNUnLzum0rn5X2rZMSMqTdc6SRk8IfHyIaOQ+IcfPvlmRJ/Z9wOpKI4GTOY7APAAAAAAAAgL2edwdu2zJR2j1Lyj1HKpwTvpgAAJFh68eSv5zBvgjAYB8AAAAAAAAAe8ufklY8I53xuZRxQu22k96RLL+UlO1MbAAAZ501V/IyzBQJeBUQFbxer7p16yavl20mAcQ+ch4ANyHnAXATch6i0pqXpYNbJG9i3bbErLCHg+hBzgNcIM7mb4NLOZ3zGOxD1EhMJHEAcA9yHgA3IecBcBNyHqJOWmdp/0YprUPdtv0bpYoiqaxQ8pdJrUeHPTxENnIeEOOKlknle6UWw52OJCI4mfOYVoGo4Pf7tXjxYvn9fqdDAYCQI+cBcBNyHgA3IechKp38nnTxbvu2ubdJH/eXFvxGmvmT8MaFiEfOA1xg3h3SF2OcjiIiOJ3zWNkHAAAAAAAAwF5ccuC29ldITQdIzQZLvv3hiwkAEBm6/Exqc57TUUAM9gEAAAAAAABojPaXOR0BAMBJeRc6HQEOoYwnAAAAAAAAAAAAEKU8lmVZTgfhRsXFxcrMzFRRUZEyMjKcDifiWZYlv98vr9crj8fjdDgAEFLkPABuQs4D4CbkPMScJX+Stn0iVRRLqXnSaROdjggRhJwHuMDi30sb35FGz5AS0p2OxlENzXnBHiNiZR+iRnl5udMhAEDYkPMAuAk5D4CbkPMQU0oLpJJVUkprKSXH6WgQgch5QIzzV0hWpfmCozmPwT5EBb/fr5UrV8rv9zsdCgCEHDkPgJuQ8wC4CTkPMWfQc9JFBdLIT6Qh/+d0NIgw5DzABfr+QfrBCimxqdOROM7pnMdgHwAAAAAAAAAAABpv/0apYKpUvtfpSFwp3ukAAAAAAAAAAESh3d9JxculwvlSm3OknFFORwQACKfCedLu2VK7y6SkFlJcshSX6nRUrsRgH6JGXFyc0yEAQNiQ8wC4CTkPgJuQ8xBT1vxdWj3efO/xMNiHOsh5QIzbOkladJ8U30Rq2l/K6u10RI5yMud5LMuyHHt2FysuLlZmZqaKioqUkZHhdDgAAAAAAABAwxTOk4pXShknSMk5UmobpyMCAITTvrXSvvXS52eY+2OXSmmdzAo/1CvYY0Ts2YeoYFmWiouLxdg0ADcg5wFwE3IeADch5yHmZA+QOlwhZQ9koA91kPMAF0jrJOWcLg19VUo/Qfqol1S03OmoHOF0zmOwD1HB7/dr7dq18vv9TocCACFHzgPgJuQ8AG5CzkNMqjwole6U/BVOR4IIQ84DXMKypI7XSENelnrfL6W0djoiRzid8xjsAwAAAAAAANBwix6U3kmV/tNS2vS+09EAAMJt7T+kt+Kkdf+UUttJfR6SUnKcjsqV4p0OAAAAAAAAAEAUSmktxadJOWdIGd2cjgYAEG5N2klN2kszrzH3r6Rsr1NY2YeokZzMpp4A3IOcB8BNyHkA3ISch5jS9WfSZSXSKR9ITfs6HQ0iEDkPiHHNh0knvlBz/7OR0sEC5+JxmJM5z2OxQ6ojiouLlZmZqaKiImVkZDgdDgAAAAAAAAAAwLHb/Z00+USp78OSr1Ra9bx09gIpta3TkUW8YI8RsbIPUcHv92v37t1s6AvAFch5ANyEnAfATch5iDnbv5Qm9pQ+O1Xat97paBBhyHmAC1g+c7tnodmv7+Jdrh3oczrnMdiHqGBZljZt2iQWogJwA3IeADch5wFwE3IeYs6Or6Xi5ea2cp/T0SDCkPMAF8jqY243TpD+01qqKHE2Hgc5nfMY7AMAAAAAAADQcF1vlsYuly47IGX2cjoaAEC4xR22R11pgbRhgnOxuByDfQAAAAAAAAAaLrmFlNldik+RPB6nowEAhJvHI7X7oZR7jpTWSZr/K6cjci0G+xA10tPTnQ4BAMKGnAfATch5ANyEnIeYUnlA2vGNtG2K5Ct1OhpEIHIe4AKdr5dS86R9a6VWI52OxlFO5jyPRdFkRxQXFyszM1NFRUXKyMhwOhwAAAAAAACgYZY/VbOK44JNUmpbZ+MBAITf1xdJm9833+f/Xsq/z9l4okSwx4hY2Yeo4Pf7VVBQIL/f73QoABBy5DwAbkLOA+Am5DzEnKb9pab9pO53SIlNnY4GEYacB7hEcqua75ud6FwcDnM65zHYh6hgWZYKCgrEQlQAbkDOA+Am5DwAbkLOQ8zJOV06e7404CkpvonT0SDCkPMAl+hwhdR8mPn+6/OcjcVBTuc8BvsAAAAAAAAAAADQcC1PkU6fIjXpKPV7zOloXIvBPgAAAAAAAAANt+1T6U2P+ao84HQ0AACnbP6ftH+d1KS905G4FoN9iAoej0fZ2dnyeDxOhwIAIUfOA+Am5DwAbkLOQ8zZv77me0+cY2EgMpHzABfxJpjbhExn43CQ0znPY1E02RHFxcXKzMxUUVGRMjIynA4HAAAAAAAAAACgcUp3SsktnI4iagR7jIiVfYgKfr9fGzdulN/vdzoUAAg5ch4ANyHnAXATch4ANyHnAS7j8oE+p3Meg32ICpZlqbCwUCxEBeAG5DwAbkLOA+Am5DzEnIMF0tw7pCV/dDoSRCByHgA3cTrnxTvyrAAAAAAAAACiW8FUaeUz5vve9zobCwAALsZgHwAAAAAAAICGazZY6nqL1GyQ05EAAOBqDPYhKng8HuXk5Mjj8TgdCgCEHDkPgJuQ8wC4CTkPMSejq3TiX52OAhGKnAfATZzOeR6LosmOKC4uVmZmpoqKipSRkeF0OAAAAAAAAEDDWX5JHokBHQAAjlmwx4i8QYgJCDmfz6c1a9bI5/M5HQoAhBw5D4CbkPMAuAk5DzGnYKr0Vpz0FpcYURc5D4CbOJ3z+EuMqFFSUuJ0CAAQNuQ8AG5CzgPgJuQ8xBTPoUuLKa2djQMRi5wHwE2czHns2QcAAAAAAACg4VqNlK5khyAAAJzGyj4AAAAAAAAAAAAgSjHYh6jg8XiUl5cnD5s9A3ABch4ANyHnAXATch5izv5N0nstpGk/dDoSRCByHgA3cTrnMdiHqOD1etWsWTN5vfzKAoh95DwAbkLOA+Am5DzEnJLvpbJd0sZ3nI4EEYicB8BNnM55ZFpEBZ/PpxUrVsjn8zkdCgCEHDkPgJuQ8wC4CTkPMSd7oHTKB9I5i52OBBGInAfATZzOefGOPCvQCKWlpU6HAABhQ84D4CbkPABuQs5DTEnMktqe73QUiGDkPABu4mTOY2UfAAAAAAAAgMYpWiaVrHE6CgAAXI2VfQAAAAAAAAAabs8C6eP+5vsrLUdDAQDAzVjZh6jg9XrVqVMnNvQF4ArkPABuQs4D4CbkPMScxGbmNvcHzsaBiETOA+AmTuc8j2VZTLtxQHFxsTIzM1VUVKSMjAynwwEAAAAAAAAAAEAYBHuMiGkViAo+n0+LFy+Wz+dzOhQACDlyHgA3IecBcBNyHgA3IecBcBOncx6DfYgavDEA4CbkPABuQs4D4CbkPMSUA5ulNz3S5KFOR4IIRc4D4CZO5jwG+wAAAAAAAAA0XGIzKam51P6HTkcCAICrxTsdAAAAAAAAAIAoFJ8iXbzT6SgAAHA9j2VZltNBuFGwN1+MdZZlqbS0VMnJyfJ4PE6HAwAhRc4D4CbkPABuQs4D4CbkPABu0tCcF+wxIsp4ImokJiY6HQIAhA05D4CbkPMAuAk5D4CbkPMAuImTOY/BPkQFv9+vxYsXy+/3Ox0KAIQcOQ+Am5DzALgJOQ+Am5DzALiJ0zmPwT4AAAAAAAAAAAAgSjHYBwAAAAAAAAAAAEQpBvsAAAAAAAAAAACAKOWxLMtyOgg3Ki4uVmZmpoqKipSRkeF0OBHPsiz5/X55vV55PB6nwwGAkCLnAXATch4ANyHnAXATch4AN2lozgv2GBEr+xA1ysvLnQ4BAMKGnAfATch5ANyEnAfATch5ANzEyZzHYB+igt/v18qVK+X3+50OBQBCjpwHwE3IeQDchJwHwE3IeQDcxOmcx2AfAAAAAAAAAAAAEKUY7AMAAAAAAAAAAACiVFQM9q1fv17XX3+9OnbsqJSUFHXu3FkPPPBAnfqnixYt0sknn6zk5GTl5eXp8ccfr/NY7777rrp3767k5GTl5+dr0qRJtdoty9L999+v1q1bKyUlRaNGjdKqVatqnVNYWKirrrpKGRkZysrK0vXXX699+/YFv+OoJS4uzukQACBsyHkA3IScB8BNyHkA3IScB8BNnMx5UTHYt2LFCvn9fr344otaunSpnnnmGY0fP16//e1vq88pLi7W6NGj1b59e82dO1dPPPGEHnzwQb300kvV50yfPl1XXHGFrr/+es2fP18XXHCBLrjgAi1ZsqT6nMcff1zPPvusxo8fr1mzZqlJkyYaM2aMSktLq8+56qqrtHTpUk2ZMkUTJ07U119/rRtvvDE8/xguFRcXp/z8fN4gAHAFch4ANyHnAXATch4ANyHnAXATp3Oex7Isy5FnPk5PPPGEXnjhBa1du1aS9MILL+h3v/udCgoKlJiYKEn6zW9+ow8++EArVqyQJP3whz/U/v37NXHixOrHGTp0qPr166fx48fLsizl5ubqzjvv1K9+9StJUlFRkVq1aqXXXntNl19+uZYvX66ePXtqzpw5GjRokCTpk08+0TnnnKPNmzcrNzfXNt6ysjKVlZVV3y8uLlZeXp4KCwuVkZEhSfJ4PPJ6vfL7/Tr8Zak67vP5aj1moONer1cej8f2uKQ6G0QGOh4XFyfLsmyPHxljoOPB6pPH49G+ffuUmpoqj8cTE32KxdeJPtEn+hScPlmWpf379ysjI+OYY4/0PlXFGEuvE32iT/QpOH2yLEslJSXKzMwM2Ndo69PhMcbK60Sf6BN9Ck6fqnJeenq64uLiYqJPR8ZIn+gTfaJPVccrKyurc17VsWjvUyy+TvSJPtGn4PTJ4/GoqKhIaWlp1WMY9fWpuLhYWVlZKioqqh4jOh7xx/0IDikqKlJ2dnb1/RkzZuiUU06pHuiTpDFjxuixxx7Tnj171LRpU82YMUN33HFHrccZM2aMPvjgA0nSunXrVFBQoFGjRlW3Z2ZmasiQIZoxY4Yuv/xyzZgxQ1lZWdUDfZI0atQoeb1ezZo1SxdeeKFtvI888ogeeuihOseXLl2qtLQ0SVJ2drbatWunzZs3q7CwsPqcnJwc5eTkaP369SopKak+npeXp2bNmmnVqlW1Vh526tRJGRkZWrZsWa1fom7duikxMVGLFy+uFUN+fr7Ky8u1cuXK6mNVo9AlJSXVA6qSlJycrO7du2vPnj3atGlT9fH09HR17txZO3bsUEFBQfXxYPWpQ4cOWr9+vTweT63/XNHcp1h8negTfaJPwemTZVmqqKjQwIEDY6ZPUuy9TvSJPtGn4PTJsiwVFhZq4MCBysrKiok+xeLrRJ/oE30KTp8OHjyowsJCZWdnq3PnzjHRp1h8negTfaJPwenT999/r23btik7O1sejycm+hSLrxN9ok/0KTh9ysrK0oIFC5SZmVk92Fdfn44cjDxeUbmyb/Xq1Ro4cKCefPJJ3XDDDZKk0aNHq2PHjnrxxRerz1u2bJl69eqlZcuWqUePHkpMTNTrr7+uK664ovqcv/3tb3rooYe0fft2TZ8+XSNGjNDWrVvVunXr6nMuu+wyeTweTZgwQQ8//LBef/31Wr9AktSyZUs99NBDuvnmm21jZmXf8fXJsiwtWbJEPXv2VFxczTLYaO5TLL5O9Ik+0afg9Mnn82np0qXq06ePjhStfaqKMZZeJ/pEn+hTcPpUlfPy8/MVHx8fE306PMZYeZ3oE32iT8HpU1XO69WrlxISEmKiT0fGSJ/oE32iT1XHy8vLq3NeXFxcTPQpFl8n+kSf6FNw+mRZlhYtWlSd847Wp5ha2feb3/xGjz32WL3nLF++XN27d6++v2XLFp111lm69NJLqwf6okFSUpKSkpLqHI+Li6s1eCXV/JLanRvu4x6Px/Z4oBgbevxYY6n6z2D379WQx5Eip0/BPE6f6FOwYmzocfoUuj55PJ4GnV/f8UjpUzCP0yf6FKwYG3qcPoWmTx6Ph7x3lOP0iT4FK8aGHqdPwe9T1fOT9+o/Tp/oU7BibOhx+hTcPlU99+HnRHufQnWcPtGnYMXY0OP0KTh98vl8tjmvvhiDydHBvjvvvFPXXnttved06tSp+vutW7dq5MiRGj58uF566aVa5+Xk5Gj79u21jlXdz8nJqfecw9urjh2+sm/79u3q169f9Tk7duyo9RiVlZUqLCys/nmERnJystMhAEDYkPMAuAk5D4CbkPMAuAk5D4CbOJnz7Iclw6RFixbq3r17vV9Ve/Bt2bJFp512mgYOHKhXX321zojqsGHD9PXXX6uioqL62JQpU9StWzc1bdq0+pypU6fW+rkpU6Zo2LBhkqSOHTsqJyen1jnFxcWaNWtW9TnDhg3T3r17NXfu3OpzPv/8c/n9fg0ZMiSI/zo4XFxcnLp37x5w9B4AYgk5D4CbkPMAuAk5D4CbkPMAuInTOc/Rwb5jVTXQ165dOz355JPauXOnCgoKam2YeOWVVyoxMVHXX3+9li5dqgkTJugvf/mL7rjjjupzbrvtNn3yySd66qmntGLFCj344IP67rvvdOutt0oyyybHjRunP/7xj/rf//6nxYsX68c//rFyc3N1wQUXSJJ69Oihs846SzfccINmz56tb7/9Vrfeeqsuv/xy5ebmhvXfxU38fr92795dp84uAMQich4ANyHnAXATch4ANyHnAXATp3Oeo2U8j9WUKVO0evVqrV69Wm3btq3VVrU5YmZmpj799FPdcsstGjhwoJo3b677779fN954Y/W5w4cP15tvvql7771Xv/3tb9W1a1d98MEH6t27d/U5d999t/bv368bb7xRe/fu1UknnaRPPvmk1vLLN954Q7feeqvOOOMMeb1eXXzxxXr22WdD/K/gbpZladOmTcrKynI6FAAIOXIeADch5wFwE3IeADch5wFwE6dznseqGi1DWBUXFyszM1NFRUXKyMhwOpyI5/P5tHjxYuXn57P0H0DMI+cBcBNyHgA3IecBcBNyHgA3aWjOC/YYUVSU8QQAAAAAAAAAAABQF4N9iBrp6elOhwAAYUPOA+Am5DwAbkLOA+Am5DwAbuJkzqOMp0Mo4wkAAAAAAAAAAOA+lPGEK/n9fhUUFMjv9zsdCgCEHDkPgJuQ8wC4CTkPgJuQ8wC4idM5j8E+RAXLslRQUCAWogJwA3IeADch5wFwE3IeADch5wFwE6dzHoN9AAAAAAAAAAAAQJRisA8AAAAAAAAAAACIUgz2ISp4PB5lZ2fL4/E4HQoAhBw5D4CbkPMAuAk5D4CbkPMAuInTOc9jUTTZEcXFxcrMzFRRUZEyMjKcDgcAAAAAAAAAAABhEOwxIlb2ISr4/X5t3LhRfr/f6VAAIOTIeQDchJwHwE3IeQDchJwHwE2cznkM9iEqWJalwsJCsRAVgBuQ8wC4CTkPgJuQ8wC4CTkPgJs4nfMY7AMAAAAAAAAAAACiVLzTAbhV1ehucXGxw5FEB5/Pp3379qm4uFhxcXFOhwMAIUXOA+Am5DwAbkLOA+Am5DwAbtLQnFc1NhSslYAM9jmkpKREkpSXl+dwJAAAAAAAAAAAAAi3kpISZWZmHvfjeCyKJjvC7/dr69atSk9Pl8fjcTqciFdcXKy8vDxt2rRJGRkZTocDACFFzgPgJuQ8AG5CzgPgJuQ8AG7S0JxnWZZKSkqUm5srr/f4d9xjZZ9DvF6v2rZt63QYUScjI4M3BwBcg5wHwE3IeQDchJwHwE3IeQDcpCE5Lxgr+qoc/3AhAAAAAAAAAAAAAEcw2AcAAAAAAAAAAABEKQb7EBWSkpL0wAMPKCkpyelQACDkyHkA3IScB8BNyHkA3IScB8BNnM55HsuyLEeeGQAAAAAAAAAAAMBxYWUfAAAAAAAAAAAAEKUY7AMAAAAAAAAAAACiFIN9AAAAAAAAAAAAQJRisA8AAAAAAAAAAACIUgz2IeI9//zz6tChg5KTkzVkyBDNnj3b6ZAAoF6PPPKITjzxRKWnp6tly5a64IILtHLlylrnlJaW6pZbblGzZs2Ulpamiy++WNu3b691zsaNGzV27FilpqaqZcuWuuuuu1RZWVnrnC+//FIDBgxQUlKSunTpotdeey3U3QOAej366KPyeDwaN25c9TFyHoBYsmXLFv3oRz9Ss2bNlJKSovz8fH333XfV7ZZl6f7771fr1q2VkpKiUaNGadWqVbUeo7CwUFdddZUyMjKUlZWl66+/Xvv27at1zqJFi3TyyScrOTlZeXl5evzxx8PSPwCo4vP5dN9996ljx45KSUlR586d9Yc//EGWZVWfQ84DEM2+/vprnXvuucrNzZXH49EHH3xQqz2cOe7dd99V9+7dlZycrPz8fE2aNKlBfWGwDxFtwoQJuuOOO/TAAw9o3rx56tu3r8aMGaMdO3Y4HRoABPTVV1/plltu0cyZMzVlyhRVVFRo9OjR2r9/f/U5t99+uz788EO9++67+uqrr7R161ZddNFF1e0+n09jx45VeXm5pk+frtdff12vvfaa7r///upz1q1bp7Fjx2rkyJFasGCBxo0bp5/+9KeaPHlyWPsLAFXmzJmjF198UX369Kl1nJwHIFbs2bNHI0aMUEJCgj7++GMtW7ZMTz31lJo2bVp9zuOPP65nn31W48eP16xZs9SkSRONGTNGpaWl1edcddVVWrp0qaZMmaKJEyfq66+/1o033ljdXlxcrNGjR6t9+/aaO3eunnjiCT344IN66aWXwtpfAO722GOP6YUXXtBf//pXLV++XI899pgef/xxPffcc9XnkPMARLP9+/erb9++ev75523bw5Xjpk+friuuuELXX3+95s+frwsuuEAXXHCBlixZcuydsYAINnjwYOuWW26pvu/z+azc3FzrkUcecTAqAGiYHTt2WJKsr776yrIsy9q7d6+VkJBgvfvuu9XnLF++3JJkzZgxw7Isy5o0aZLl9XqtgoKC6nNeeOEFKyMjwyorK7Msy7Luvvtuq1evXrWe64c//KE1ZsyYUHcJAOooKSmxunbtak2ZMsU69dRTrdtuu82yLHIegNjy61//2jrppJMCtvv9fisnJ8d64oknqo/t3bvXSkpKst566y3Lsixr2bJlliRrzpw51ed8/PHHlsfjsbZs2WJZlmX97W9/s5o2bVqdA6ueu1u3bsHuEgAENHbsWOsnP/lJrWMXXXSRddVVV1mWRc4DEFskWe+//371/XDmuMsuu8waO3ZsrXiGDBli/exnPzvm+FnZh4hVXl6uuXPnatSoUdXHvF6vRo0apRkzZjgYGQA0TFFRkSQpOztbkjR37lxVVFTUym/du3dXu3btqvPbjBkzlJ+fr1atWlWfM2bMGBUXF2vp0qXV5xz+GFXnkCMBOOGWW27R2LFj6+Qlch6AWPK///1PgwYN0qWXXqqWLVuqf//+evnll6vb161bp4KCglr5KjMzU0OGDKmV87KysjRo0KDqc0aNGiWv16tZs2ZVn3PKKacoMTGx+pwxY8Zo5cqV2rNnT6i7CQCSpOHDh2vq1Kn6/vvvJUkLFy7UtGnTdPbZZ0si5wGIbeHMccH4vMtgHyLWrl275PP5al30kaRWrVqpoKDAoagAoGH8fr/GjRunESNGqHfv3pKkgoICJSYmKisrq9a5h+e3goIC2/xX1VbfOcXFxTp48GAougMAtt5++23NmzdPjzzySJ02ch6AWLJ27Vq98MIL6tq1qyZPnqybb75Zv/zlL/X6669LqslZ9X2OLSgoUMuWLWu1x8fHKzs7u0F5EQBC7Te/+Y0uv/xyde/eXQkJCerfv7/GjRunq666ShI5D0BsC2eOC3ROQ3Jg/DGfCQAAGuyWW27RkiVLNG3aNKdDAYCQ2LRpk2677TZNmTJFycnJTocDACHl9/s1aNAgPfzww5Kk/v37a8mSJRo/fryuueYah6MDgOB655139MYbb+jNN99Ur169qvdNzs3NJecBQIRhZR8iVvPmzRUXF6ft27fXOr59+3bl5OQ4FBUAHLtbb71VEydO1BdffKG2bdtWH8/JyVF5ebn27t1b6/zD81tOTo5t/qtqq++cjIwMpaSkBLs7AGBr7ty52rFjhwYMGKD4+HjFx8frq6++0rPPPqv4+Hi1atWKnAcgZrRu3Vo9e/asdaxHjx7auHGjpJqcVd/n2JycHO3YsaNWe2VlpQoLCxuUFwEg1O66667q1X35+fm6+uqrdfvtt1dXcyDnAYhl4cxxgc5pSA5ksA8RKzExUQMHDtTUqVOrj/n9fk2dOlXDhg1zMDIAqJ9lWbr11lv1/vvv6/PPP1fHjh1rtQ8cOFAJCQm18tvKlSu1cePG6vw2bNgwLV68uNYbhilTpigjI6P6AtOwYcNqPUbVOeRIAOF0xhlnaPHixVqwYEH116BBg3TVVVdVf0/OAxArRowYoZUrV9Y69v3336t9+/aSpI4dOyonJ6dWviouLtasWbNq5by9e/dq7ty51ed8/vnn8vv9GjJkSPU5X3/9tSoqKqrPmTJlirp166amTZuGrH8AcLgDBw7I6619+TguLk5+v18SOQ9AbAtnjgvK510LiGBvv/22lZSUZL322mvWsmXLrBtvvNHKysqyCgoKnA4NAAK6+eabrczMTOvLL7+0tm3bVv114MCB6nNuuukmq127dtbnn39ufffdd9awYcOsYcOGVbdXVlZavXv3tkaPHm0tWLDA+uSTT6wWLVpY99xzT/U5a9eutVJTU6277rrLWr58ufX8889bcXFx1ieffBLW/gLAkU499VTrtttuq75PzgMQK2bPnm3Fx8dbf/rTn6xVq1ZZb7zxhpWammr961//qj7n0UcftbKysqz//ve/1qJFi6zzzz/f6tixo3Xw4MHqc8466yyrf//+1qxZs6xp06ZZXbt2ta644orq9r1791qtWrWyrr76amvJkiXW22+/baWmplovvvhiWPsLwN2uueYaq02bNtbEiROtdevWWf/5z3+s5s2bW3fffXf1OeQ8ANGspKTEmj9/vjV//nxLkvX0009b8+fPtzZs2GBZVvhy3LfffmvFx8dbTz75pLV8+XLrgQcesBISEqzFixcfc18Y7EPEe+6556x27dpZiYmJ1uDBg62ZM2c6HRIA1EuS7derr75afc7Bgwetn//851bTpk2t1NRU68ILL7S2bdtW63HWr19vnX322VZKSorVvHlz684777QqKipqnfPFF19Y/fr1sxITE61OnTrVeg4AcMqRg33kPACx5MMPP7R69+5tJSUlWd27d7deeumlWu1+v9+67777rFatWllJSUnWGWecYa1cubLWObt377auuOIKKy0tzcrIyLCuu+46q6SkpNY5CxcutE466SQrKSnJatOmjfXoo4+GvG8AcLji4mLrtttus9q1a2clJydbnTp1sn73u99ZZWVl1eeQ8wBEsy+++ML2Gt4111xjWVZ4c9w777xjnXDCCVZiYqLVq1cv66OPPmpQXzyWZVnHvg4QAAAAAAAAAAAAQKRgzz4AAAAAAAAAAAAgSjHYBwAAAAAAAAAAAEQpBvsAAAAAAAAAAACAKMVgHwAAAAAAAAAAABClGOwDAAAAAAAAAAAAohSDfQAAAAAAAAAAAECUYrAPAAAAAAAAAAAAiFIM9gEAAAAAAAAAAABRisE+AAAAAHC5a6+9VhdccIHTYTRatMcPAAAAAMcj3ukAAAAAAACh4/F46m1/4IEH9Je//EWWZYUpohpffvmlRo4cqT179igrKyvszw8AAAAAsYDBPgAAAACIYdu2bav+fsKECbr//vu1cuXK6mNpaWlKS0tzIjQAAAAAQBBQxhMAAAAAYlhOTk71V2ZmpjweT61jaWlpdcpgnnbaafrFL36hcePGqWnTpmrVqpVefvll7d+/X9ddd53S09PVpUsXffzxx7Wea8mSJTr77LOVlpamVq1a6eqrr9auXbuOOdbXXntNWVlZmjx5snr06KG0tDSdddZZtQYsfT6f7rjjDmVlZalZs2a6++6766xK9Pv9euSRR9SxY0elpKSob9+++ve//y1JsixLo0aN0pgxY6p/rrCwUG3bttX999/f0H9eAAAAAHAcg30AAAAAgDpef/11NW/eXLNnz9YvfvEL3Xzzzbr00ks1fPhwzZs3T6NHj9bVV1+tAwcOSJL27t2r008/Xf3799d3332nTz75RNu3b9dll13WoOc9cOCAnnzySf3zn//U119/rY0bN+pXv/pVdftTTz2l1157Ta+88oqmTZumwsJCvf/++7Ue45FHHtE//vEPjR8/XkuXLtXtt9+uH/3oR/rqq6/k8Xj0+uuva86cOXr22WclSTfddJPatGnDYB8AAACAqEQZTwAAAKhuGDgAAAPGSURBVABAHX379tW9994rSbrnnnv06KOPqnnz5rrhhhskSffff79eeOEFLVq0SEOHDtVf//pX9e/fXw8//HD1Y7zyyivKy8vT999/rxNOOOGYnreiokLjx49X586dJUm33nqrfv/731e3//nPf9Y999yjiy66SJI0fvx4TZ48ubq9rKxMDz/8sD777DMNGzZMktSpUydNmzZNL774ok499VS1adNGL774on784x+roKBAkyZN0vz58xUfz0dkAAAAANGHTzIAAAAAgDr69OlT/X1cXJyaNWum/Pz86mOtWrWSJO3YsUOStHDhQn3xxRe2+/+tWbPmmAf7UlNTqwf6JKl169bVz1FUVKRt27ZpyJAh1e3x8fEaNGhQdUnO1atX68CBAzrzzDNrPW55ebn69+9fff/SSy/V+++/r0cffVQvvPCCunbtekzxAQAAAECkYbAPAAAAAFBHQkJCrfsej6fWMY/HI8nsjydJ+/bt07nnnqvHHnuszmO1bt36uJ73yD356rNv3z5J0kcffaQ2bdrUaktKSqr+/sCBA5o7d67i4uK0atWqY358AAAAAIg0DPYBAAAAAI7bgAED9N5776lDhw4hK4eZmZmp1q1ba9asWTrllFMkSZWVlZo7d64GDBggSerZs6eSkpK0ceNGnXrqqQEf684775TX69XHH3+sc845R2PHjtXpp58ekrgBAAAAIJS8TgcAAAAAAIh+t9xyiwoLC3XFFVdozpw5WrNmjSZPnqzrrrtOPp8vaM9z22236dFHH9UHH3ygFStW6Oc//7n27t1b3Z6enq5f/epXuv322/X6669rzZo1mjdvnp577jm9/vrrksyqv1deeUVvvPGGzjzzTN1111265pprtGfPnqDFCQAAAADhwmAfAAAAAOC45ebm6ttvv5XP59Po0aOVn5+vcePGKSsrS15v8D563nnnnbr66qt1zTXXaNiwYUpPT9eFF15Y65w//OEPuu+++/TII4+oR48eOuuss/TRRx+pY8eO2rlzp66//no9+OCD1asBH3roIbVq1Uo33XRT0OIEAAAAgHDxWA3Z/AAAAAAAAAAAAABAxGBlHwAAAAAAAAAAABClGOwDAAAAAAAAAAAAohSDfQAAAAAAAAAAAECUYrAPAAAAAAAAAAAAiFIM9gEAAAAAAAAAAABRisE+AAAAAAAAAAAAIEox2AcAAAAAAAAAAABEKQb7AAAAAAAAAAAAgCjFYB8AAAAAAAAAAAAQpRjsAwAAAAAAAAAAAKIUg30AAAAAAAAAAABAlPp/OHMcXolfq+AAAAAASUVORK5CYII=\n" + }, + "metadata": {} + }, + { + "output_type": "stream", + "name": "stdout", + "text": [ + "948\n" + ] + }, + { + "output_type": "display_data", + "data": { + "text/plain": [ + "
" + ], + "image/png": "iVBORw0KGgoAAAANSUhEUgAABv0AAAN5CAYAAAArSffsAAAAOnRFWHRTb2Z0d2FyZQBNYXRwbG90bGliIHZlcnNpb24zLjEwLjAsIGh0dHBzOi8vbWF0cGxvdGxpYi5vcmcvlHJYcgAAAAlwSFlzAAAPYQAAD2EBqD+naQABAABJREFUeJzs3Xd4VMX79/HPpjeSEFpASuhNqiCiIkWaFLEAokgvKiCIgOAXpYgUpaoI2CgKKgp2RJqEDgpIL9JCUSBISQgkhOzO8wdP9seShOxi2E3w/bquXLJz5szcs3tyS7hz5liMMUYAAAAAAAAAAAAAciwvTwcAAAAAAAAAAAAA4N+h6AcAAAAAAAAAAADkcBT9AAAAAAAAAAAAgByOoh8AAAAAAAAAAACQw1H0AwAAAAAAAAAAAHI4in4AAAAAAAAAAABADkfRDwAAAAAAAAAAAMjhKPoBAAAAAAAAAAAAORxFPwAAAAAAAAAAACCHo+gHAACAO0Lnzp0VFRXl6TA85r++/ttp9uzZslgsiomJ8XQo2V5MTIwsFotmz57tsRiaNWumHj16eGz+7CY6OloWi0XR0dGeDgV3gHbt2qlt27aeDgMAAAAZoOgHAACAbMtisTj1lV3/MTsmJkZdunRRyZIlFRAQoMjISD300EMaPny4p0PLcvfee68sFoumT5/u6VCyvc6dOztcvyEhISpRooRat26thQsXymaz3fLYP//8s0aMGJF1wWbg888/15QpU277PK5at26dli5dqsGDB9vboqKiMs0hnTt39lzQ2URqcXvz5s23dZ6zZ89q/Pjxeuihh5QvXz6Fh4frvvvu0/z589Ptf+XKFQ0ePFiFChVSYGCgatWqpWXLlqXbd/369XrwwQcVFBSkyMhI9e3bVwkJCf9qzBvdjmvfZrNp9uzZevTRR1WkSBEFBwfr7rvv1ptvvqmkpKQ0/TO6jseNG5em719//aW2bdsqPDxcoaGhatWqlQ4fPpxuHJ988onKly+vgIAAlS5dWu+9916aPoMHD9bChQu1ffv2f79wAAAAZDmLMcZ4OggAAAAgPXPnznV4/emnn2rZsmX67LPPHNobNWqkiIgI2Ww2+fv7uzPEDB08eFA1a9ZUYGCgunbtqqioKJ08eVJbt27V4sWL0/2H3H/j6tWrHlv/gQMHVKZMGUVFRemuu+7S2rVr3R7D7TR79mx16dJFR44cyZK7KTt37qwvv/xSH3/8sSQpMTFRR48e1Y8//qgdO3aoXr16+v777xUaGury2H369NH777+v2/1jXosWLbRr1640dz8aY3TlyhX5+vrK29v7tsaQnscee0yJiYlasmSJve27775Lt/AjSVOnTtWmTZs0ffp0Pf/88+4K062io6NVv359rVy5UvXq1cuwX+p1/vvvv6tGjRq3LZ6ffvpJTzzxhJo1a6b69evLx8dHCxcu1MqVKzVs2DCNHDnSof/TTz+tBQsW6KWXXlLp0qU1e/Zs/f7771q5cqUefPBBe79t27apdu3aKl++vHr27KkTJ05owoQJql+/vhYvXnxLY6Yno2v/30hISFCuXLl03333qUWLFsqfP782bNigOXPm6KGHHtKvv/4qi8Vi72+xWNSoUSN17NjRYZxq1aqpYsWKDuNWr15dcXFxGjBggHx9fTV58mQZY7Rt2zblyZPH3veDDz7Q888/ryeffFJNmjTRmjVr9Nlnn2ncuHEORXRJqlWrlsqWLatPP/00y94DAAAAZBEDAAAA5BC9e/c2OeWvsL169TI+Pj4mJiYmzbHTp09n2TwJCQlZNtatGjZsmMmfP79ZuHChsVgs5siRI54OKUvNmjXLSMqydXXq1MkEBwene2zs2LFGkmnbtu0tje2u75HmzZubYsWK3fZ5XHH69Gnj4+NjPv74Y6f6L1myxFgsFvPoo4/e5sg8a+XKlUaSWbly5U37pV7nv//++22N5/Dhw2nyos1mMw0aNDD+/v4OOW3Tpk1Gkhk/fry9LTEx0ZQsWdLUrl3bYYxHHnnEFCxY0MTFxdnbPvroIyPJLFmy5JbGTM/tuPavXLli1q1bl6Z95MiRRpJZtmyZQ7sk07t370zHfeutt4wk89tvv9nb9u7da7y9vc2rr75qb7t8+bLJkyePad68ucP57du3N8HBwebcuXMO7RMmTDDBwcHm4sWLTq0PAAAA7sP2ngAAALgj3PhMu9Rni02YMEHvv/++SpQooaCgIDVu3FjHjx+XMUajRo1S4cKFFRgYqFatWuncuXNpxl28eLHq1Kmj4OBg5cqVS82bN9fu3bszjefQoUMqXLiwihUrluZY/vz5b2mezp07KyQkRIcOHVKzZs2UK1cutW/fPt31S9e2jJsyZYoqVqyogIAAFShQQM8995zOnz/v0G/z5s1q0qSJ8ubNq8DAQBUvXlxdu3bNdI2pPv/8c7Vu3VotWrRQWFiYPv/88zR9RowYIYvFooMHD6pz584KDw9XWFiYunTposuXLzv0TUlJ0ahRo1SyZEn5+/srKipK//vf/3TlyhWHflFRUWrRooWio6NVo0YNBQYGqlKlSvbtXr/55htVqlRJAQEBuueee/THH384nL9jxw517txZJUqUsG+/2rVrV509e/am6+3UqZPy5s2rq1evpjnWuHFjlS1b1pm3LV1DhgxR48aN9fXXX+vPP/90OJbZNdK5c2e9//77khy3/0vl7PWQOlfdunWVK1cuhYaGqmbNmvbPtV69elq0aJGOHj1qnyP12svomX6//vqrPfbw8HC1atVKe/fudejjyjWSnkWLFiklJUUNGzbMtO+pU6fUoUMH3XXXXZo1a5bDMWevP0maNm2aKlasKH9/fxUqVEi9e/fWhQsXHPrUq1dPd999t3bs2KG6desqKChIpUqV0oIFCyRJq1atUq1atRQYGKiyZctq+fLlaeb566+/1LVrVxUoUED+/v6qWLGiZs6cmabfiRMn9Nhjjyk4OFj58+dX//79043bWak556+//tJjjz2mkJAQ5cuXTwMHDpTVanXoe/LkSe3bty/d74vrFS9ePE1etFgseuyxx3TlyhWHrScXLFggb29v9ezZ094WEBCgbt26acOGDTp+/LgkKT4+XsuWLdOzzz7rcIdsx44dFRISoq+++srlMdNzs2tfkmJjY9WtWzcVKFBAAQEBqlKliubMmXPT90OS/Pz8dP/996dpf/zxxyUpzfdKqsTExJveNb5gwQLVrFlTNWvWtLeVK1dODz/8sMN7snLlSp09e1a9evVyOL937966dOmSFi1a5NDeqFEjXbp0yektUQEAAOA+FP0AAABwR5s3b56mTZumF198UQMGDNCqVavUtm1bvfbaa/rll180ePBg9ezZUz/++KMGDhzocO5nn32m5s2bKyQkRG+99ZZef/117dmzRw8++GCmW7sVK1ZMx48f16+//pppjK7Mk5KSoiZNmih//vyaMGGCnnzyyQzHfe655zRo0CA98MADeuedd9SlSxfNmzdPTZo0sf/DfGxsrBo3bqyYmBgNGTJE7733ntq3b6+NGzdmGrckbdq0SQcPHtTTTz8tPz8/PfHEE5o3b16G/du2bauLFy9q7Nixatu2rWbPnp1mO7/u3btr2LBhql69uiZPnqy6detq7NixateuXZrxDh48qGeeeUYtW7bU2LFjdf78ebVs2VLz5s1T//799eyzz2rkyJE6dOiQ2rZt6/C8vGXLlunw4cPq0qWL3nvvPbVr105ffvmlmjVrdtPtMTt06KCzZ886bCEpXSsk/frrr3r22Wedeu9uNr4xxuEf1J25Rp577jk1atTI3j/1K5Uz14N0bZvH5s2b69y5c3r11Vc1btw4Va1aVb/88oskaejQoapatary5s1rn+Nmzzhbvny5mjRpotjYWI0YMUIvv/yy1q9frwceeCDd7yNnrpH0rF+/Xnny5Em30H49m82mZ599VmfPntXnn3+uiIgIh+POXn8jRoxQ7969VahQIU2cOFFPPvmkPvjgAzVu3DhN4ev8+fNq0aKFatWqpbffflv+/v5q166d5s+fr3bt2qlZs2YaN26cLl26pNatW+vixYv2c0+fPq377rtPy5cvV58+ffTOO++oVKlS6tatm8P7npiYqIcfflhLlixRnz59NHToUK1Zs0avvPJKpu/dzVitVjVp0kR58uTRhAkTVLduXU2cOFEffvihQ79XX31V5cuX119//XVL85w6dUqSlDdvXnvbH3/8oTJlyqTZ6vbee++VdG1LT0nauXOnUlJS0mxL6ufnp6pVqzoU/J0dMz03u/YTExNVr149ffbZZ2rfvr3Gjx+vsLAwde7cWe+8847zb8R10ntPUs2ePVvBwcEKDAxUhQoV0vyyhc1m044dO9LdqvXee+/VoUOH7NdZ6vtzY9977rlHXl5eaX5hokKFCgoMDNS6detuaV0AAAC4jTx8pyEAAADgtJttXdipUyeHLdeOHDliJJl8+fKZCxcu2NtfffVVI8lUqVLFXL161d7+9NNPGz8/P5OUlGSMMebixYsmPDzc9OjRw2GeU6dOmbCwsDTtN9q1a5cJDAw0kkzVqlVNv379zHfffWcuXbrk0M+VeTp16mQkmSFDhmS6/jVr1hhJZt68eQ79fvnlF4f2b7/99l9t6denTx9TpEgRY7PZjDHGLF261Egyf/zxh0O/4cOHG0mma9euDu2PP/64yZMnj/31tm3bjCTTvXt3h34DBw40ksyvv/5qbytWrJiRZNavX29vW7JkiZFkAgMDzdGjR+3tH3zwQZotDi9fvpxmPV988YWRZFavXm1vu3F7T6vVagoXLmyeeuoph3MnTZpkLBaLOXz4cHpvld3Ntvc0xpg//vjDSDL9+/c3xrh2jWT0PeLs9XDhwgWTK1cuU6tWLZOYmOjQN/UzNibjLQ5Tv+9mzZplb6tatarJnz+/OXv2rL1t+/btxsvLy3Ts2NHe5uw1kpEHH3zQ3HPPPZn2e+ONN4wkM3LkyDTHnL3+YmNjjZ+fn2ncuLGxWq32flOnTjWSzMyZM+1tdevWNZLM559/bm/bt2+fkWS8vLzMxo0b7e2p1+/171+3bt1MwYIFzT///OMQU7t27UxYWJj9Op4yZYqRZL766it7n0uXLplSpUrd8vaeqTnnjTfecOhbrVq1NO91at9b2Qb37NmzJn/+/KZOnToO7RUrVjQNGjRI03/37t1GkpkxY4Yxxpivv/46zfdtqjZt2pjIyEiXx8xIRtd+6vs/d+5ce1tycrKpXbu2CQkJMfHx8TcdNz0NGzY0oaGh5vz58w7t999/v5kyZYr5/vvvzfTp083dd99tJJlp06bZ+5w5cybdz84YY95//30jyezbt88Ycy1veHt7pxtDvnz5TLt27dK0lylTxjzyyCMurwkAAAC3F3f6AQAA4I7Wpk0bhYWF2V/XqlVLkvTss8/Kx8fHoT05Odl+l8qyZct04cIFPf300/rnn3/sX97e3qpVq5ZWrlx503krVqyobdu26dlnn1VMTIzeeecdPfbYYypQoIA++ugje79bmeeFF17IdN1ff/21wsLC1KhRI4dx77nnHoWEhNjHDQ8PlyT99NNPmW7Ld6OUlBTNnz9fTz31lH0byQYNGih//vwZ3u33/PPPO7yuU6eOzp49q/j4eEnSzz//LEl6+eWXHfoNGDBAktJsM1ehQgXVrl3b/jr1823QoIGKFi2apv36rQMDAwPtf05KStI///yj++67T5K0devWDNft5eWl9u3b64cffnC4I2vevHm6//77Vbx48QzPdUZISIgk2cf+t9ei5Pz1sGzZMl28eFFDhgxRQECAwxjXbxXqrJMnT2rbtm3q3Lmzwx11lStXVqNGjeyf9/Uyu0YycvbsWeXOnfumfdasWaORI0eqXr16eu2119Icd/b6W758uZKTk/XSSy/Jy+v/fqzu0aOHQkND01ynISEhDncKli1bVuHh4Spfvrz92pTSXqfGGC1cuFAtW7aUMcbhs2vSpIni4uLs1+rPP/+sggULqnXr1vbxgoKCHLaxvFXpfSbXfy9J1+48M8ak2WY4MzabTe3bt9eFCxf03nvvORxLTEyUv79/mnNSr83ExESH/2bUN/W4K2O66ueff1ZkZKSefvppe5uvr6/69u2rhIQErVq1yqXxxowZo+XLl2vcuHH2PJ1q3bp16tevnx599FE9//zz2rJli+6++27973//c/o9ub5PYmKi/Pz80o3jxvcvVe7cufXPP/+4tCYAAADcfhT9AAAAcEe7vvAjyV4ALFKkSLrtqc83O3DggKRrxaN8+fI5fC1dulSxsbGZzl2mTBl99tln+ueff7Rjxw6NGTNGPj4+6tmzp/25Xa7O4+Pjo8KFC2c694EDBxQXF6f8+fOnGTchIcE+bt26dfXkk09q5MiRyps3r1q1aqVZs2Y59RywpUuX6syZM7r33nt18OBBHTx4UEeOHFH9+vX1xRdfOGylmerGzyO1SJP6vh89elReXl4qVaqUQ7/IyEiFh4fr6NGjNx3P2c9Xks6dO6d+/fqpQIECCgwMVL58+ewFu7i4uJuuvWPHjkpMTNS3334rSdq/f7+2bNmiDh063PQ8ZyQkJEiScuXKJSlrrkVnr4dDhw5Jku6+++5/vQ5J9s8rveccli9fXv/8848uXbrk0J7ZNXIz5ibbsp49e1ZPP/20cufOrXnz5jkU666P15nrL6N1+fn5qUSJEmmu08KFC6cpmoaFhWV6nZ45c0YXLlzQhx9+mOZz69KliyTZP7ujR4+qVKlSaeb5N8+YlK4VffLly+fQljt3bqc+D2e8+OKL+uWXX/Txxx+rSpUqDscCAwPTzUWpz7FLLdyn/jejvtcX+J0d01VHjx5V6dKl01xX5cuXtx931vz58/Xaa6+pW7duTv2Sh5+fn/r06aMLFy5oy5YtkjJ/T67vExgYqOTk5HTHvvH9S2WMuaVfBAAAAMDt5ZN5FwAAACDn8vb2dqk9tWiQWrD67LPPFBkZmabf9XcJOhNDpUqVVKlSJdWuXVv169fXvHnz1LBhQ5fn8ff3T7dYcSObzXbTO+5S/xHfYrFowYIF2rhxo3788UctWbJEXbt21cSJE7Vx40b7XWfpSR27bdu26R5ftWqV6tev79CW2fueytl/TL7Vz1e6Fvf69es1aNAgVa1aVSEhIbLZbGratGm6BcvrVahQQffcc4/mzp2rjh07au7cufLz88vwvXDFrl27JMleeMqKa9HZ6yE7cPYauVGePHkyLEQZY9SpUyf9/fff+vHHH1WoUKGbjpXVxYx/m4eeffZZderUKd2+lStXzoIIM5ZRjFlh5MiRmjZtmsaNG5duwbxgwYLpPiPw5MmTkmT/HAsWLOjQfmPf6z9vZ8f0lGXLlqljx45q3ry5ZsyY4fR5qQXkc+fOSZIiIiLk7++f4XsiOb5/VqtVsbGxyp8/v71fcnKyzp49m+57cv78eZUuXdr5hQEAAMAtKPoBAAAA6ShZsqQkKX/+/GrYsGGWjVujRg1J//ePrrdrnpIlS2r58uV64IEHnLpz5b777tN9992n0aNH6/PPP1f79u315Zdfqnv37un2v3Tpkr7//ns99dRTDtsJpurbt6/mzZuXpuiXmWLFislms+nAgQP2O2Qk6fTp07pw4YKKFSvm0ngZOX/+vFasWKGRI0dq2LBh9vbUu+qc0bFjR7388ss6efKkPv/8czVv3jzT7SWd8dlnn8lisahRo0aSXLtGMipWOXs9pM61a9euNHe7OTPPjVI/r/3796c5tm/fPuXNm1fBwcFOjZWZcuXKaeHChekemzRpkhYtWqT+/furefPmN43Xmevv+nWVKFHC3i85OVlHjhzJsu/lfPnyKVeuXLJarZmOWaxYMe3atSvNHVjpvffZwfvvv68RI0bopZde0uDBg9PtU7VqVa1cuVLx8fEKDQ21t2/atMl+XLp2Z6qPj482b97sUHhPTk7Wtm3bHNqcHTMjGV37xYoV044dO2Sz2Rx+MWPfvn3245nZtGmTHn/8cdWoUUNfffWVS79ckrrdamoB38vLS5UqVdLmzZvTnadEiRL2u4lT17x582Y1a9bM3m/z5s2y2Wxp3pOUlBQdP35cjz76qNPxAQAAwD3Y3hMAAABIR5MmTRQaGqoxY8ak+6y7M2fO3PT8NWvWpHte6jPDUrfc+7fzZKRt27ayWq0aNWpUmmMpKSm6cOGCpGvFrxvvoEr9B96bbfH57bff6tKlS+rdu7dat26d5qtFixZauHChU9uEXi/1H5ynTJni0D5p0iRJumnBxhWpdy/duPYb572Zp59+WhaLRf369dPhw4f17LPP/uu4xo0bp6VLl+qpp56y30XjyjWSWkBL/XxTOXs9NG7cWLly5dLYsWPtWwCmuv69Cg4OznQLVOnaHURVq1bVnDlzHGLatWuXli5d6lBg+Ldq166t8+fPp3nW3O+//65XX31V99xzj8aNG3fTMZy9/ho2bCg/Pz+9++67Du/LJ598ori4uCy9Tp988kktXLjQfgfo9a7/7Js1a6a///5bCxYssLddvnxZH374YZbEkpmTJ09q3759Tj0bdP78+erbt6/at29vf2/T07p1a1mtVoc1XLlyRbNmzVKtWrXsd7eFhYWpYcOGmjt3rsNzNj/77DMlJCSoTZs2Lo+ZkYyu/WbNmunUqVOaP3++vS0lJUXvvfeeQkJCVLdu3ZuOu3fvXjVv3lxRUVH66aefMizOp/f/hIsXL2rKlCnKmzev7rnnHoe1/v777w6Fv/379+vXX391eE8aNGigiIgITZ8+3WHc6dOnKygoKM31vGfPHiUlJen++++/6ZoAAADgftzpBwAAAKQjNDRU06dPV4cOHVS9enW1a9dO+fLl07Fjx7Ro0SI98MADmjp1aobnv/XWW9qyZYueeOIJ+/Z7W7du1aeffqqIiAi99NJLWTJPRurWravnnntOY8eO1bZt29S4cWP5+vrqwIED+vrrr/XOO++odevWmjNnjqZNm6bHH39cJUuW1MWLF/XRRx8pNDT0pgWZefPmKU+ePBn+o++jjz6qjz76SIsWLdITTzzhdNxVqlRRp06d9OGHH+rChQuqW7eufvvtN82ZM0ePPfaYy3cOZiQ0NFQPPfSQ3n77bV29elV33XWXli5dqiNHjjg9Rr58+dS0aVN9/fXXCg8Pd6nQk5KSorlz50q69syso0eP6ocfftCOHTtUv359h4KEK9dI6j/49+3bV02aNJG3t7fatWvn9PUQGhqqyZMnq3v37qpZs6aeeeYZ5c6dW9u3b9fly5c1Z84c+zzz58/Xyy+/rJo1ayokJEQtW7ZMd63jx4/XI488otq1a6tbt25KTEzUe++9p7CwMI0YMcLp9ywzzZs3l4+Pj5YvX66ePXtKulb0euqpp3T16lW1aNFCX331VbrnFihQQI0aNXL6+suXL59effVVjRw5Uk2bNtWjjz6q/fv3a9q0aapZs2aWFIBTjRs3TitXrlStWrXUo0cPVahQQefOndPWrVu1fPly+3aOPXr00NSpU9WxY0dt2bJFBQsW1GeffaagoKAsi+VmXn31Vc2ZM0dHjhxRVFRUhv1+++03dezYUXny5NHDDz+cZsvZ+++/3373ZK1atdSmTRu9+uqrio2NValSpTRnzhzFxMTok08+cThv9OjRuv/++1W3bl317NlTJ06c0MSJE9W4cWM1bdrU3s+VMdOT0bXfs2dPffDBB+rcubO2bNmiqKgoLViwQOvWrdOUKVPsd9Wl5+LFi2rSpInOnz+vQYMGadGiRQ7HS5Ysqdq1a0u6dofkd999p5YtW6po0aI6efKkZs6cqWPHjumzzz6Tn5+f/bxevXrpo48+UvPmzTVw4ED5+vpq0qRJKlCggAYMGGDvFxgYqFGjRql3795q06aNmjRpojVr1mju3LkaPXq0IiIiHOJZtmyZgoKC7HcjAwAAIBsxAAAAQA7Ru3dvk9FfYTt16mSKFStmf33kyBEjyYwfP96h38qVK40k8/XXXzu0z5o1y0gyv//+e5r+TZo0MWFhYSYgIMCULFnSdO7c2WzevPmmsa5bt8707t3b3H333SYsLMz4+vqaokWLms6dO5tDhw6l6e/MPJ06dTLBwcFOrT/Vhx9+aO655x4TGBhocuXKZSpVqmReeeUV8/fffxtjjNm6dat5+umnTdGiRY2/v7/Jnz+/adGixU3Xd/r0aePj42M6dOiQYZ/Lly+boKAg8/jjjxtjjBk+fLiRZM6cOePQL/V9P3LkiL3t6tWrZuTIkaZ48eLG19fXFClSxLz66qsmKSnJ4dxixYqZ5s2bp5lbkundu7dDW3rXw4kTJ8zjjz9uwsPDTVhYmGnTpo35+++/jSQzfPjwm8aY6quvvjKSTM+ePTN8L27UqVMnI8n+FRQUZKKiosyTTz5pFixYYKxWa7rnOXONpKSkmBdffNHky5fPWCyWNN8vmV0PqX744Qdz//33m8DAQBMaGmruvfde88UXX9iPJyQkmGeeecaEh4cbSfZrL/V9njVrlsN4y5cvNw888IB9vJYtW5o9e/Y49HHlGsnIo48+ah5++GH769R4MvuqW7eu/Rxnrz9jjJk6daopV66c8fX1NQUKFDAvvPCCOX/+vEOfunXrmooVK6Y515Xr9/Tp06Z3796mSJEixtfX10RGRpqHH37YfPjhhw79jh49ah599FETFBRk8ubNa/r162d++eUXI8msXLnypu9dejkwo5yT+lldL/W6zuxzSp0no68br53ExEQzcOBAExkZafz9/U3NmjXNL7/8ku7Ya9asMffff78JCAgw+fLlM7179zbx8fFp+rky5o0yuvaNufY5denSxeTNm9f4+fmZSpUqpVlPejK7Tjt16mTvu3TpUtOoUSMTGRlpfH19TXh4uGncuLFZsWJFumMfP37ctG7d2oSGhpqQkBDTokULc+DAgXT7fvjhh6Zs2bLGz8/PlCxZ0kyePNnYbLY0/WrVqmWeffbZTNcFAAAA97MYk8nT0AEAAAAA6fr+++/12GOPafXq1apTp46nw/nPW7NmjerVq6d9+/bZt0cFkHW2bdum6tWra+vWrZk+/xAAAADuR9EPAAAAAG5RixYttHfvXh08eFAWi8XT4UDSI488osKFC+ujjz7ydCjAHaddu3ay2WwZbpULAAAAz6LoBwAAAAAu+vLLL7Vjxw6NHTtW77zzjvr27evpkAAAAAAA/3EU/QAAAADARRaLRSEhIXrqqac0Y8YM+fj4eDokAAAAAMB/HD+ZAgAAAICL+N1JAAAAAEB24+XpAAAAAAAAAAAAAAD8O9zp5yE2m01///23cuXKJYvF4ulwAAAAAAAAAAAAkMVSd4oJDQ297fUgin4e8vfff6tIkSKeDgMAAAAAAAAAAAC3WVxcnEJDQ2/rHBT9PCRXrlySpJiYGOXOndvD0QBA9ma1WrV7925VrFhR3t7eng4HALI98iYAOI+cCQCuIW8CgPOsVqt+++03NW3a1C3zUfTzkNRbOENDQ297ZRcAcjqr1aqQkBCFhobyAwUAOIG8CQDOI2cCgGvImwDgPKvVquDgYLfN5+W2mQAAAAAAAAAAAADcFhT9POx2P7QRAO4EFotFERER5EwAcBJ5EwCcR84EANeQNwHAeRaLxa2PeLMYY4zbZoNdfHy8wsLC3PLgRgAAAAAAAAAAALifO+tBPNPPw2w2m6dDAIBsz2az6cSJEypcuLC8vLhJHQAyQ94EAOeRMwHANZ7Mm1arVVevXnXrnACQGV9f3wyfcWqz2XT8+HG3xULRz8O40RIAMmeM0blz53TXXXd5OhQAyBHImwDgPHImALjGE3nTGKNTp07pwoULbpsTAFwRHh6uyMjINFsfG2N0/vx5t8VB0Q8AAAAAAAAAkG2lFvzy58+voKAgnicIINswxujy5cuKjY2VJBUsWNCj8VD0AwAAAAAAAABkS1ar1V7wy5Mnj6fDAYA0AgMDJUmxsbHKnz9/hlt9ugOb1XsYv5UCAJmzWCzp3h4PAEgfeRMAnEfOBADXuDtvpj7DLygoyC3zAcCtSM1RNz531GKxqECBAm6Lgzv9PIyHhANA5ry8vBQZGenpMAAgxyBvAoDzyJkA4BpP5U1+OQNAdpZRjvLy8nJr0Y+Kk4dZrVZPhwAA2Z7VatWhQ4fImQDgJPImADiPnAkAriFvAoDzrFarDh8+7Lb5KPoBAHKEixcvejoEAMhRyJsA4DxyJgC4hrwJAM5LSEhw21wU/QAAAAAAAAAAyGZiYmJksVi0bds2p8+ZPXu2wsPDPR7HjerVq6eXXnopy2LKDkaMGKGqVat6OgzAAUU/AAAAAAAAAABug+PHj6tr164qVKiQ/Pz8VKxYMfXr109nz57N9NwiRYro5MmTuvvuu52e76mnntKff/75b0J2SWpB8GZfs2fP1jfffKNRo0a5La5UVqtV48aNU7ly5RQYGKiIiAjVqlVLH3/88b8ee+DAgVqxYkUWRAlkHR9PB/BfxwNoASBzFotFRYoUIWcCgJPImwDgPHImALiGvOm8w4cPq3bt2ipTpoy++OILFS9eXLt379agQYO0ePFibdy4UREREemem5ycLD8/P0VGRro0Z2BgoAIDA7MifKekFiZTTZgwQb/88ouWL19ubwsLC3NrTNcbOXKkPvjgA02dOlU1atRQfHy8Nm/erPPnz9/ymMYYWa1WhYSEKCQkJAujxZ3IYrGocOHCbpuPO/08zMuLjwAAMuPl5aU8efKQMwHASeRNAHAeORMAXJMd8qYxUlKS+7+McS3O3r17y8/PT0uXLlXdunVVtGhRPfLII1q+fLn++usvDR061N43KipKo0aNUseOHRUaGqqePXumu63mDz/8oNKlSysgIED169fXnDlzZLFYdOHCBUlpt/dM3YLys88+U1RUlMLCwtSuXTuH5zL+8ssvevDBBxUeHq48efKoRYsWOnTokFNr9Pb2VmRkpP0rJCREPj4+Dm2BgYFptveMiorSm2++qY4dOyokJETFihXTDz/8oDNnzqhVq1YKCQlR5cqVtXnzZof51q5dqzp16igwMFBFihRR3759denSpQzj++GHH9SrVy+1adNGxYsXV5UqVdStWzcNHDjQ3sdms2ns2LEqXry4AgMDVaVKFS1YsMB+PDo6WhaLRYsXL9Y999wjf39/rV27Nt3tPT/++GOVL19eAQEBKleunKZNm2Y/lpycrD59+qhgwYIKCAhQsWLFNHbsWKfeZ+RcXl5eGRb3bwfu9PMwq9Xq6RAAINuzWq06cOCASpcuLW9vb0+HAwDZHnkTAJxHzgQA12SHvHnlitSmjfvn/fprKSDAub7nzp3TkiVLNHr06DR3uUVGRqp9+/aaP3++pk2bZr9rcsKECRo2bJiGDx+e7phHjhxR69at1a9fP3Xv3l1//PGHQ/EqI4cOHdJ3332nn376SefPn1fbtm01btw4jR49WpJ06dIlvfzyy6pcubISEhI0bNgwPf7449q2bdttLe5OnjxZY8aM0euvv67JkyerQ4cOuv/++9W1a1eNHz9egwcPVseOHbV7925ZLBYdOnRITZs21ZtvvqmZM2fqzJkz6tOnj/r06aNZs2alO0dkZKR+/fVX9erVS/ny5Uu3z9ixYzV37lzNmDFDpUuX1urVq/Xss88qX758qlu3rr3fkCFDNGHCBJUoUUK5c+dWdHS0wzjz5s3TsGHDNHXqVFWrVk1//PGHevTooeDgYHXq1EnvvvuufvjhB3311VcqWrSojh8/ruPHj2fZ+4nsyWq1unXLXYp+AIAcISkpydMhAECOQt4EAOeRMwHANeTNzB04cEDGGJUvXz7d4+XLl9f58+d15swZ5c+fX5LUoEEDDRgwwN4nJibG4ZwPPvhAZcuW1fjx4yVJZcuW1a5du+zFu4zYbDbNnj1buXLlkiR16NBBK1assJ/35JNPOvSfOXOm8uXLpz179rj0PEFXNWvWTM8995wkadiwYZo+fbpq1qypNv+/ojt48GDVrl1bp0+fVmRkpMaOHav27dvb7xgsXbq03n33XdWtW1fTp09XQDoV2UmTJql169aKjIxUxYoVdf/996tVq1Z65JFHJElXrlzRmDFjtHz5ctWuXVuSVKJECa1du1YffPCBQ9HvjTfeUKNGjTJcz/DhwzVx4kQ98cQTkqTixYtrz549+uCDD9SpUycdO3ZMpUuX1oMPPiiLxaJixYr9+zcROYI7cyZFPwAAAAAAAABAjuHvf+2uO0/M6yrjwp6gNWrUuOnx/fv3q2bNmg5t9957b6bjRkVF2Qt+klSwYEHFxsbaXx84cEDDhg3Tpk2b9M8//8hms0mSjh07dluLfpUrV7b/uUCBApKkSpUqpWmLjY1VZGSktm/frh07dmjevHn2PsYY2Ww2HTlyJN0Ca4UKFbRr1y5t2bJF69at0+rVq9WyZUt17txZH3/8sQ4ePKjLly+nKeYlJyerWrVqDm03+3wuXbqkQ4cOqVu3burRo4e9PSUlRWFhYZKkzp07q1GjRipbtqyaNm2qFi1aqHHjxpm+T4ArKPoBAAAAAAAAAHIMi8X5bTY9pVSpUrJYLNq7d68ef/zxNMf37t2r3LlzO2w5GRwcfFti8fX1dXhtsVjshT1JatmypYoVK6aPPvpIhQoVks1m0913363k5OTbEk96caVucZpeW2qsCQkJeu6559S3b980YxUtWjTDeby8vFSzZk3VrFlTL730kubOnasOHTpo6NChSkhIkCQtWrRId911l8N5/jdUeW/2+aSO89FHH6lWrVoOx1K3wa1evbqOHDmixYsXa/ny5Wrbtq0aNmzo8PxA4N+i6OdhPCgcADLn5eWlEiVKkDMBwEnkTQBwHjkTAFxD3nROnjx51KhRI02bNk39+/d3eK7fqVOnNG/ePHXs2NFe2HJG2bJl9fPPPzu0/f777/8qzrNnz2r//v366KOPVKdOHUnS2rVr/9WYt0v16tW1Z88elSpV6l+NU6FCBUnX7s6rUKGC/P39dezYMYetPF1VoEABFSpUSIcPH1b79u0z7BcaGqqnnnpKTz31lFq3bq2mTZvq3LlzioiIuOW5kb15eXkpKirKbfNR9PMwV5I6APxXWSwWhYaGejoMAMgxyJsA4DxyJgC4hrzpvKlTp+r+++9XkyZN9Oabb6p48eLavXu3Bg0apLvuuivTZ/Hd6LnnntOkSZM0ePBgdevWTdu2bdPs2bMl3fq/M+fOnVt58uTRhx9+qIIFC+rYsWMaMmTILY11uw0ePFj33Xef+vTpo+7duys4OFh79uzRsmXLNHXq1HTPad26tR544AHdf//9ioyM1JEjR/Tqq6+qTJkyKleunHx8fDRw4ED1799fNptNDz74oOLi4rRu3TqFhoaqU6dOTsc3cuRI9e3bV2FhYWratKmuXLmizZs36/z583r55Zc1adIkFSxYUNWqVZOXl5e+/vprRUZGKjw8PIveIWRH7s6Z/DqGh1mtVk+HAADZntVq1c6dO8mZAOAk8iYAOI+cCQCuIW86r3Tp0tq8ebNKlCihtm3bqmTJkurZs6fq16+vDRs2uHx3V/HixbVgwQJ98803qly5sqZPn66hQ4dKSrsVpbO8vLz05ZdfasuWLbr77rvVv39/jR8//pbGut0qV66sVatW6c8//1SdOnVUrVo1DRs2TIUKFcrwnCZNmujHH39Uy5YtVaZMGXXq1EnlypXT0qVL5eNz7Z6oUaNG6fXXX9fYsWNVvnx5NW3aVIsWLVLx4sVdiq979+76+OOPNWvWLFWqVEl169bV7Nmz7ePkypVLb7/9tmrUqKGaNWsqJiZGP//8M3fN3uGsVqt2797ttvksxpUniSLLxMfHKywsTOfOnVPu3Lk9HQ4AZGupP1BUqlTJvg86ACBj5E0AcB45EwBc4+68mZSUpCNHjqh48eIKyO4P8vOA0aNHa8aMGTp+/LinQwH+0zLKVVarVRs2bFCdOnUUFxd32+/6Y3tPAAAAAAAAAABygGnTpqlmzZrKkyeP1q1bp/Hjx6tPnz6eDgtANkHRDwAAAAAAAACAHODAgQN68803de7cORUtWlQDBgzQq6++6umwAGQTbO/pIanbe164cEFhYWGeDgcAsjVjjJKSkhQQEHDLD6YGgP8S8iYAOI+cCQCucXfeZHtPADlBRrnKGKPY2FhFRka6ZXtPnhAJAMgR/Pz8PB0CAOQo5E0AcB45EwBcQ94EAOe5M2dS9PMwm83m6RAAINuz2WzauXMnORMAnETeBADnkTMBwDXkTQBwns1m0+7du902H0U/AAAAAAAAAAAAIIej6AcAAAAAAAAAAADkcBT9AAAAAAAAAAAAgByOop+HeXnxEQBAZry8vFSpUiVyJgA4ibwJAM4jZwKAa8ibAOA8Ly8vVaxY0X3zuW0mAAD+heTkZE+HAAA5CnkTAJxHzgQA15A3s5/OnTvrscces7+uV6+eXnrpJbfHER0dLYvFogsXLrh9biC7cmfOpOjnYTabzdMhAEC2Z7PZtH//fnImADiJvAkAziNnAoBryJvO69y5sywWiywWi/z8/FSqVCm98cYbSklJue1zf/PNNxo1apRTfd1dqIuKipLFYtGXX36Z5ljFihVlsVg0e/Zst8QC3G42m00HDhxw23wU/QAAAAAAAAAAuA2aNm2qkydP6sCBAxowYIBGjBih8ePHp9s3K+8GioiIUK5cubJsvKxWpEgRzZo1y6Ft48aNOnXqlIKDgz0UFZDzUfQDAAAAAAAAAOQ4l5IvZfiVlJLkdN/Eq4mZ9r1V/v7+ioyMVLFixfTCCy+oYcOG+uGHHyT935aco0ePVqFChVS2bFlJ0vHjx9W2bVuFh4crIiJCrVq1UkxMjH1Mq9Wql19+WeHh4cqTJ49eeeUVGWMc5r1xe88rV65o8ODBKlKkiPz9/VWqVCl98skniomJUf369SVJuXPnlsViUefOnSVdu0Np7NixKl68uAIDA1WlShUtWLDAYZ6ff/5ZZcqUUWBgoOrXr+8Q5820b99eq1at0vHjx+1tM2fOVPv27eXj4+PQ98KFC+revbvy5cun0NBQNWjQQNu3b7cfP3TokFq1aqUCBQooJCRENWvW1PLlyx3GiIqK0pgxY9S1a1flypVLRYsW1YcffuhUrEBO4pN5FwAAPM/b29vTIQBAjkLeBADnkTMBwDXZJW+GjA3J8Fiz0s206JlF9tf5J+TX5auX0+1bt1hdRXeOtr+OeidK/1z+x6GPGW6UFQIDA3X27Fn76xUrVig0NFTLli2TJF29elVNmjRR7dq1tWbNGvn4+OjNN99U06ZNtWPHDvn5+WnixImaPXu2Zs6cqfLly2vixIn69ttv1aBBgwzn7dixozZs2KB3331XVapU0ZEjR/TPP/+oSJEiWrhwoZ588knt379foaGhCgwMlCSNHTtWc+fO1YwZM1S6dGmtXr1azz77rPLly6e6devq+PHjeuKJJ9S7d2/17NlTmzdv1oABA5x6HwoUKKAmTZpozpw5eu2113T58mXNnz9fq1at0qeffurQt02bNgoMDNTixYsVFhamDz74QA8//LD+/PNPRUREKCEhQc2aNdPo0aPl7++vTz/9VC1bttT+/ftVtGhR+zgTJ07UqFGj9L///U8LFizQCy+8oLp169qLrcDt4s6cSdHPw7LL/yABIDvz9vZWpUqVPB0GAOQY5E0AcB45EwBcQ968NcYYrVixQkuWLNGLL75obw8ODtbHH38sPz8/SdLcuXNls9n08ccfy2KxSJJmzZql8PBwRUdHq3HjxpoyZYpeffVVPfHEE5KkGTNmaMmSJRnO/eeff+qrr77SsmXL1LBhQ0lSiRIl7McjIiIkSfnz51d4eLika3cGjhkzRsuXL1ft2rXt56xdu1YffPCB6tatq+nTp6tkyZKaOHGiJKls2bLauXOn3nrrLafek65du2rAgAEaOnSoFixYoJIlS6pq1aoOfdauXavffvtNsbGx8vf3lyRNmDBB3333nRYsWKCePXuqSpUqqlKliv2cUaNG6dtvv9UPP/ygPn362NubNWumXr16SZIGDx6syZMna+XKlRT9cFt5e3urYsWKbpuPop+H3XjbNQAgLWOMLl68qFy5ctn/wgsAyBh5EwCcR84EANdkp7yZ8GpChse8vRxvtogdGJthXy+L41OwYvrF/Ku4rvfTTz8pJCREV69elc1m0zPPPKMRI0bYj1eqVMle8JOk7du36+DBg2mex5eUlKRDhw4pLi5OJ0+eVK1atezHfHx8VKNGjQz/rXnbtm3y9vZW3bp1nY774MGDunz5sho1auTQnpycrGrVqkmS9u7d6xCHJHuB0BnNmzfXc889p9WrV2vmzJnq2rVrmj7bt29XQkKC8uTJ49CemJioQ4cOSZISEhI0YsQILVq0SCdPnlRKSooSExN17Ngxh3MqV65s/7PFYlFkZKRiYzO+LoCsYIxRfHy82+aj6OdhNpvN0yEAQLZns9l0+PBhVapUiTukAcAJ5E0AcB45EwBck53yZrBfsMf7ZqZ+/fqaPn26/Pz8VKhQoTTPqwsOdpwrISFB99xzj+bNm5dmrHz58t1SDKnbdboiIeFaQXXRokW66667HI6l3nH3b/n4+KhDhw4aPny4Nm3apG+//TbdOAoWLKjo6Og0x1LvShw4cKCWLVumCRMmqFSpUgoMDFTr1q2VnJzs0N/X19fhtcVi4d/ncdvZbDann3WZFSj6AQAAAAAAAABwGwQHB6tUqVJO969evbrmz5+v/PnzKzQ0NN0+BQsW1KZNm/TQQw9JklJSUrRlyxZVr1493f6VKlWSzWbTqlWr7Nt7Xi/1TkOr1Wpvq1Chgvz9/XXs2LEM7xAsX768fvjhB4e2jRs3Zr7I63Tt2lUTJkzQU089pdy5c6c5Xr16dZ06dUo+Pj6KiopKd4x169apc+fOevzxxyVdKxS6s8gCZCdemXcBAAAAAAAAAAC3W/v27ZU3b161atVKa9as0ZEjRxQdHa2+ffvqxIkTkqR+/fpp3Lhx+u6777Rv3z716tVLFy5cyHDMqKgoderUSV27dtV3331nH/Orr76SJBUrVkwWi0U//fSTzpw5o4SEBOXKlUsDBw5U//79NWfOHB06dEhbt27Ve++9pzlz5kiSnn/+eR04cECDBg3S/v379fnnn2v27Nkurbd8+fL6559/NGvWrHSPN2zYULVr19Zjjz2mpUuXKiYmRuvXr9fQoUO1efNmSVLp0qX1zTffaNu2bdq+fbueeeYZ7uDDfxZFPwBAjhAQEODpEAAgRyFvAoDzyJkA4Bry5u0TFBSk1atXq2jRonriiSdUvnx5devWTUlJSfY7/wYMGKAOHTqoU6dOql27tnLlymW/yy0j06dPV+vWrdWrVy+VK1dOPXr00KVLlyRJd911l0aOHKkhQ4aoQIEC6tOnjyRp1KhRev311zV27FiVL19eTZs21aJFi1S8eHFJUtGiRbVw4UJ99913qlKlimbMmKExY8a4vOY8efJkuAWpxWLRzz//rIceekhdunRRmTJl1K5dOx09elQFChSQJE2aNEm5c+fW/fffr5YtW6pJkyYZ3vUIeII7c6bFZPR0T9xW8fHxCgsLU1xcXIa3aQMAAAAAAADAf1lSUpKOHDmi4sWLU2wEkG3dLFe5sx7EnX4exm3GAJA5m82ms2fPkjMBwEnkTQBwHjkTAFxD3gQA59lsNp07d85t81H08zButASAzBljdPz4cXImADiJvAkAziNnAoBryJsA4DxjjP15nO5A0Q8AAAAAAAAAAADI4Sj6AQAAAAAAAAAAADkcRT8AQI6QK1cuT4cAADkKeRMAnEfOBADXkDcBwHkhISFum8vHbTMhXd7e3p4OAQCyPW9vb5UsWdLTYQBAjkHeBADnkTMBwDXkTQBwnre3t0qUKOG2+bjTz8NsNpunQwCAbM9ms+nUqVPkTABwEnkTAJxHzgQA15A3AcB5NptNp0+fdtt8FP08zBjj6RAAINszxujUqVPkTABwEnkTAJxHzgQA15A3AcB5xhiKfgAAAAAAAAAAAACcR9EPAAAAAAAAAAA4JTo6WhaLRRcuXHDrvLNnz1Z4ePi/GiMmJkYWi0Xbtm3LsI+n1gdkBYp+HmaxWDwdAgBkexaLRREREeRMAHASeRMAnEfOBADXkDed17lzZz322GOeDsMlFovlpl8jRozwdIhAjmKxWJQ7d263zefjtpmQLi8v6q4AkBkvLy8VLVrU02EAQI5B3gQA55EzAcA15M07R3Jysvz8/BzaTp48af/z/PnzNWzYMO3fv9/eFhISos2bN2fJXMB/gZeXl4oUKeK++dw2E9Jls9k8HQIAZHs2m03Hjh0jZwKAk8ibAOA8ciYAuIa8mXVWrVqle++9V/7+/ipYsKCGDBmilJQUSdJPP/2k8PBwWa1WSdK2bdtksVg0ZMgQ+/ndu3fXs88+a3+9du1a1alTR4GBgSpSpIj69u2rS5cu2Y9HRUVp1KhR6tixo0JDQ9WzZ880MUVGRtq/wsLCZLFYHNpCQkLsfbds2aIaNWooKChI999/v0NxcMSIEapatao+/vhjFS9eXAEBAZKkCxcuqHv37sqXL59CQ0PVoEEDbd++3X7e9u3bVb9+feXKlUuhoaG655570hQZlyxZovLlyyskJERNmzZ1KFTabDa98cYbKly4sPz9/VW1alX98ssvN/0cfv75Z5UpU0aBgYGqX7++YmJibtofcIXNZtPx48fdNh9FPw8zxng6BADI9owxOnfuHDkTAJxE3gQA55EzAcA12SpvLqooben/f693vXmt7cq5a68vHrz2+s/3/6/Phs7Ssgf/7/XfS671+XvJ/7Ute/Bav1TXn59F/vrrLzVr1kw1a9bU9u3bNX36dH3yySd68803JUl16tTRxYsX9ccff0i6ViDMmzevoqOj7WOsWrVK9erVkyQdOnRITZs21ZNPPqkdO3Zo/vz5Wrt2rfr06eMw74QJE1SlShX98ccfev311//VGoYOHaqJEydq8+bN8vHxUdeuXR2OHzx4UAsXLtQ333xjf4ZemzZtFBsbq8WLF2vLli2qXr26Hn74YZ07d+0za9++vQoXLqzff/9dW7Zs0ZAhQ+Tr62sf8/Lly5owYYI+++wzrV69WseOHdPAgQPtx9955x1NnDhREyZM0I4dO9SkSRM9+uijOnDgQLprOH78uJ544gm1bNlS27ZtU/fu3R0Kq8C/ZYzR+fPn3TYf23sCAAAAAAAAAOBG06ZNU5EiRTR16lRZLBaVK1dOf//9twYPHqxhw4YpLCxMVatWVXR0tGrUqKHo6Gj1799fI0eOVEJCguLi4nTw4EHVrVtXkjR27Fi1b99eL730kiSpdOnSevfdd1W3bl1Nnz7dfqddgwYNNGDAgCxZw+jRo+3zDxkyRM2bN1dSUpJ9ruTkZH366afKly+fpGt3Iv7222+KjY2Vv7+/pGtFyO+++04LFixQz549dezYMQ0aNEjlypWzr+N6V69e1YwZM1SyZElJUp8+ffTGG2/Yj0+YMEGDBw9Wu3btJElvvfWWVq5cqSlTpuj999MWb6dPn66SJUtq4sSJkqSyZctq586deuutt7LkPQLcjaIfAAAAAAAAACDnab7b8fXdr137SpWrVNo+tWc7vi7URCp0Q59Gax1fl+n9r8JMz969e1W7dm1ZLBZ72wMPPKCEhASdOHFCRYsWVd26dRUdHa0BAwZozZo1Gjt2rL766iutXbtW586dU6FChexFse3bt2vHjh2aN2+efTxjjGw2m44cOaLy5ctLkmrUqJFla6hcubL9zwULFpQkxcbG2p/5WKxYMXvBLzXGhIQE5cmTx2GcxMREHTp0SJL08ssvq3v37vrss8/UsGFDtWnTxl7gk6SgoCCH1wULFlRsbKwkKT4+Xn///bceeOABh/EfeOABhy1Er7d3717VqlXLoa127drOvQFANkTRz8OuT+oAgPSl7h9PzgQA55A3AcB55EwAcA15033q1aunmTNnavv27fL19VW5cuVUr149RUdH6/z58/a77CQpISFBzz33nPr27ZtmnNQinCQFBwdnWXzXb7uZej1c/6zHG+dKSEhQwYIFHbYoTRUeHi7p2rMAn3nmGS1atEiLFy/W8OHD9eWXX+rxxx9PM2fqvNliq1kgAxaLRQUKFHDbfBT9PMzLi8cqAkBmvLy8FBkZ6ekwACDHIG8CgPPImQDgGvJm1ihfvrwWLlwoY4y9YLZu3TrlypVLhQsXlvR/z/WbPHmyvcBXr149jRs3TufPn3fYprN69eras2ePSpUq5f7FOKl69eo6deqUfHx8FBUVlWG/MmXKqEyZMurfv7+efvppzZo1y170u5nQ0FAVKlRI69atcyiIrlu3Tvfee2+655QvX14//PCDQ9vGjRudWxDgBC8vL7cW/ag4eZjVavV0CACQ7VmtVh06dIicCQBOIm8CgPPImQDgGvKma+Li4rRt2zaHr+PHj6tXr146fvy4XnzxRe3bt0/ff/+9hg8frpdfftl+o0ju3LlVuXJlzZs3T/Xq1ZMkPfTQQ9q6dav+/PNPh8LW4MGDtX79evXp00fbtm3TgQMH9P3336tPnz6eWHa6GjZsqNq1a+uxxx7T0qVLFRMTo/Xr12vo0KHavHmzEhMT1adPH0VHR+vo0aNat26dfv/9d/vWpM4YNGiQ3nrrLc2fP1/79+/XkCFDtG3bNvXr1y/d/s8//7wOHDigQYMGaf/+/fr88881e/bsLFoxcC1nHj582G3zcacfACBHuHjxoqdDAIAchbwJAM4jZwKAa8ibzouOjla1atUc2rp166aPP/5YP//8swYNGqQqVaooIiJC3bp102uvvebQt27dutq2bZu96BcREaEKFSro9OnTKlu2rL1f5cqVtWrVKg0dOlR16tSRMUYlS5bUU089ddvX6CyLxaKff/5ZQ4cOVZcuXXTmzBlFRkbqoYceUoECBeTt7a2zZ8+qY8eOOn36tPLmzasnnnhCI0eOdHqOvn37Ki4uTgMGDFBsbKwqVKigH374wf7swxsVLVpUCxcuVP/+/fXee+/p3nvv1ZgxY9S1a9esWjaghIQEt81lMWx46xHx8fEKCwvTuXPnlDt3bk+HAwDZmtVq1c6dO1WpUiV5e3t7OhwAyPbImwDgPHImALjG3XkzKSlJR44cUfHixRUQEHDb5wOAW5FRrrJardqwYYPq1KmjuLg4hYaG3tY42N4TAAAAAAAAAAAAyOEo+nlY6kNaAQAZs1gsKlKkCDkTAJxE3gQA55EzAcA15E0AcJ7FYlHhwoXdNh/P9POw1IeyAgAy5uXlpTx58ng6DADIMcibAOA8ciYAuIa8CQDO8/LyUkREhPvmc9tMSJfVavV0CACQ7VmtVu3bt4+cCQBOIm8CgPPImQDgGvImADjParXqzz//dNt8FP0AADlCUlKSp0MAgByFvAkAziNnAoBryJsA4Dx35kyKfgAAAAAAAAAAAEAOR9EPAAAAAAAAAAAAyOEo+nmYlxcfAQBkxsvLSyVKlCBnAoCTyJsA4DxyJgC4hrwJAM7z8vJSVFSU2+bzcdtMSJfFYvF0CACQ7VksFoWGhno6DADIMcibAOA8ciYAuIa8CQDOc3fO5NcxPMxqtXo6BADI9qxWq3bu3EnOBAAnkTcBwHnkTABwDXkTklSvXj299NJLbp83KipKU6ZM+VdjdO7cWY899thN+3hqfbjzWK1W7d69223zUfQDAOQI/DABAK4hbwKA88iZAOAa8qZzMioczZ49W+Hh4W6PxxkjRoyQxWK56RcA17gzZ1L0AwAAAAAAAADgP8Zqtcpmszm0DRw4UCdPnrR/FS5cWG+88YZD2626evXqvw0ZQCYo+gEAAAAAAAAA4CGp202OHDlS+fLlU2hoqJ5//nklJyfb+9SrV099+vRRnz59FBYWprx58+r111+XMcbe58qVKxo4cKDuuusuBQcHq1atWoqOjrYfT73D8IcfflCFChXk7++vY8eOOcQSEhKiyMhI+5e3t7dy5crl0JbKZrPplVdeUUREhCIjIzVixAiHsSwWi6ZPn65HH31UwcHBGj16tCTp+++/V/Xq1RUQEKASJUpo5MiRSklJkSQZYzRixAgVLVpU/v7+KlSokPr27esw7uXLl9W1a1flypVLRYsW1YcffuhwfOfOnWrQoIECAwOVJ08e9ezZUwkJCRm+/5cuXVLHjh0VEhKiggULauLEiTf5tIDsjaKfh3l58REAQGa8vLxUtmxZciYAOIm8CQDOI2cCgGuyU96cVnGafun/i/316jdXa1rFaUo8lyhJOnfwnKZVnKbf3v/N3ue7zt9p5oMz7a8PLjmoaRWn6eCSg/a2mQ/O1Hedv7O/vv7822XFihXau3evoqOj9cUXX+ibb77RyJEjHfrMmTNHPj4++u233/TOO+9o0qRJ+vjjj+3H+/Tpow0bNujLL7/Ujh071KZNGzVt2lQHDhyw97l8+bLeeustffzxx9q9e7fy589/yzHPmTNHwcHB2rRpk95++2298cYbWrZsmUOfESNG6PHHH9fOnTvVtWtXrVmzRh07dlS/fv20Z88effDBB5o9e7a9ILhw4UJNnjxZH3zwgQ4cOKDvvvtOlSpVchhz4sSJqlGjhv744w/16tVLL7zwgvbv3y/pWgGvSZMmyp07t37//Xd9/fXXWr58ufr06ZPhOgYNGqRVq1bp+++/19KlSxUdHa2tW7fe8vsCXM/Ly0ulS5d223w+bpsJAIB/wc/Pz9MhAECOQt4EAOeRMwHANeTNrOfn56eZM2cqKChIFStW1BtvvKFBgwZp1KhR9gJrkSJFNHnyZFksFpUtW1Y7d+7U5MmT1aNHDx07dkyzZs3SsWPHVKhQIUnXtur85ZdfNGvWLI0ZM0bStS02p02bpipVqvzrmCtXrqzhw4dLkkqXLq2pU6dqxYoVatSokb3PM888oy5duthfd+3aVUOGDFGnTp0kSSVKlNCoUaP0yiuvaPjw4Tp27JgiIyPVsGFD+fr6qmjRorr33nsd5m3WrJl69eolSRo8eLAmT56slStXqmzZsvr888+VlJSkTz/9VMHBwZKkqVOnqmXLlnrrrbdUoEABh7ESEhL0ySefaO7cuXr44YclXStmFi5c+F+/P0Aqd+ZMin4eduOeyQCAtGw2m3bu3KlKlSrJ29vb0+EAQLZH3gQA55EzAcA12Slv9trdy+H1Q689pIdee8j+OqJURJo+j81+zOF1qSalVGp3KYe2rmu7Ory+t7dj0el2qFKlioKCguyva9eurYSEBB0/flzFihWTJN13332yWCwOfSZOnCir1aqdO3fKarWqTJkyDuNeuXJFefLksb/28/NT5cqVsyTmG8cpWLCgYmNjHdpq1Kjh8Hr79u1at26d/c4+6dqzBZOSknT58mW1adNGU6ZMUYkSJdS0aVM1a9ZMLVu2lI/P/5Uyrp/XYrEoMjLSPu/evXtVpUoVe8FPkh544AHZbDbt378/TdHv0KFDSk5OVq1atextERERKlu2rKtvB5Aum82m3bt3u20+in4AAAAAAAAAAGSx0NBQxcXFpWm/cOGCwsLCsnSuhIQEeXt7a8uWLWmKsSEhIfY/BwYGOhQO/w1fX1+H1xaLJc1NLtcX31LjHDlypJ544ok04wUEBKhIkSLav3+/li9frmXLlqlXr14aP368Vq1aZZ/PmXmB/yqKfgAAAAAAAAAAZLGyZctq6dKladq3bt2a5o687du3KzExUYGBgZKkjRs3KiQkREWKFLH32bRpk8M5GzduVOnSpeXt7a1q1arJarUqNjZWderUuQ2ryRrVq1fX/v37VapUqQz7BAYGqmXLlmrZsqV69+6tcuXKaefOnapevXqm45cvX16zZ8/WpUuX7AXHdevW2Z9FeaOSJUvK19dXmzZtUtGiRSVJ58+f159//qm6deve4ioBz6HoBwAAAAAAAABAFnvhhRc0depU9e3bV927d5e/v78WLVqkL774Qj/++KND3+TkZHXr1k2vvfaaYmJiNHz4cPXp08f+PD9JOnbsmF5++WU999xz2rp1q9577z1NnDhRklSmTBm1b99eHTt21MSJE1WtWjWdOXNGK1asUOXKldW8eXO3rj0jw4YNU4sWLVS0aFG1bt1aXl5e2r59u3bt2qU333xTs2fPltVqVa1atRQUFKS5c+cqMDDQvsVpZtq3b6/hw4erU6dOGjFihM6cOaMXX3xRHTp0SLO1p3TtLshu3bpp0KBBypMnj/Lnz6+hQ4c6vO9ATkLRz8NIHgCQOS8vL1WqVImcCQBOIm8CgPPImQDgGvKm80qUKKHVq1dr6NChatiwoZKTk1WuXDl9/fXXatq0qUPfhx9+WKVLl9ZDDz2kK1eu6Omnn9aIESMc+nTs2FGJiYm699575e3trX79+qlnz57247NmzdKbb76pAQMG6K+//lLevHl13333qUWLFu5YrlOaNGmin376SW+88Ybeeust+fr6qly5curevbskKTw8XOPGjdPLL78sq9WqSpUq6ccff3R4LuHNBAUFacmSJerXr59q1qypoKAgPfnkk5o0aVKG54wfP14JCQlq2bKlcuXKpQEDBqS7LStwK7y8vFSxYkW3zWcxxhi3zQa7+Ph4hYWF3Zb9mwHgTmOMUVJSkgICArJs33kAuJORNwHAeeRMAHCNu/NmUlKSjhw5ouLFiysgIOC2z+cJnTt31oULF/Tdd99l2KdevXqqWrWqpkyZ4ra4ADgvo1xljFFsbKwiIyMVFxen0NDQ2xoHv47hYTxgFAAyZ7PZtH//fnImADiJvAkAziNnAoBryJsA4DybzaYDBw64bT6KfgAAAAAAAAAAAEAOxzP9AAAAAAAAAADwkNmzZ2faJzo6+rbHASDn404/AECO4O3t7ekQACBHIW8CgPPImQDgGvImADjPnTmTO/08jP9BAkDmvL29ValSJU+HAQA5BnkTAJxHzgQA15A3AcB53t7eqlixotvm404/DzPGeDoEAMj2jDGKj48nZwKAk8ibAOA8ciYAuIa8CQDOS82Z7kLRz8NsNpunQwCAbM9ms+nw4cPkTABwEnkTAJxHzgQA15A3AcB5NptNMTExbpuPoh8AAAAAAAAAAACQw1H0AwAAAAAAAAAAAHI4in4AgBwhICDA0yEAQI5C3gQA55EzAcA1OTFvnjp1Sm+OGqWa5curzF13qWb58npz1CidOnXK06HdMaKiojRlyhRPh3Fb1atXTy+99JL9tTvWvGLFCpUvX15Wq/W2zoOMDRkyRC+++OItn+/OnEnRz8O8vb09HQIAZHve3t4qV64cORMAnETeBADnkTMBwDU5MW9OnjhRxQoX1toxY/TSvn2a/vffemnfPq0ZM0bFChfW5EmTbsu8nTt3lsVikcVika+vrwoUKKBGjRpp5syZLj8TccSIEapatWqWx+iJQt2JEyfk5+enu+++263z3g6///67evbseVvneOWVV/Taa6/Zv+esVqvGjRuncuXKKTAwUBEREapVq5Y+/vjj2xpHTrB79249+eSTioqKksVicfra3rFjh+rUqaOAgAAVKVJEb7/9tsPxgQMHas6cOTp8+LDLMXl7e6tMmTIun3ercmzRb9y4cbJYLA5V9aSkJPXu3Vt58uRRSEiInnzySZ0+fdrhvGPHjql58+YKCgpS/vz5NWjQIKWkpDj0iY6OVvXq1eXv769SpUpp9uzZaeZ///33FRUVpYCAANWqVUu//fbbLa2DB94CQOZsNpvOnj1LzgQAJ5E3AcB55EwAcE1Oy5uTJ07U2KFDtdpq1S9JSWov6WFJ7SUtSUrSaqtVY//3v9tW+GvatKlOnjypmJgYLV68WPXr11e/fv3UokWLNP8u/V8xe/ZstW3bVvHx8dq0aZOnw/lX8uXLp6CgoNs2/tq1a3Xo0CE9+eST9raRI0dq8uTJGjVqlPbs2aOVK1eqZ8+eunDhwm2LIzk5+ZbPPXbsWBZGcnOXL19WiRIlNG7cOEVGRjp1Tnx8vBo3bqxixYppy5YtGj9+vEaMGKEPP/zQ3idv3rxq0qSJpk+f7nJMNptN586dc/m8W5Uji36///67PvjgA1WuXNmhvX///vrxxx/19ddfa9WqVfr777/1xBNP2I9brVY1b95cycnJWr9+vebMmaPZs2dr2LBh9j5HjhxR8+bNVb9+fW3btk0vvfSSunfvriVLltj7zJ8/Xy+//LKGDx+urVu3qkqVKmrSpIliY2NdXosx5hbeAQD4bzHG6Pjx4+RMAHASeRMAnEfOBADX5KS8eerUKQ0ZPFg/XrmiWhn0qSXpxytXNOSVV27LVp/+/v6KjIzUXXfdperVq+t///ufvv/+ey1evNjhZpMLFy6oe/fuypcvn0JDQ9WgQQNt375d0rUi2ciRI7V9+3b7nYOp597svFQ//vijatasqYCAAOXNm1ePP/64pGtbVR49elT9+/e3j5tq7dq1qlOnjgIDA1WkSBH17dtXly5dsh+PjY1Vy5YtFRgYqOLFi2vevHlOvR/GGM2aNUsdOnTQM888o08++cTheExMjCwWi7755hvVr19fQUFBqlKlijZs2ODQb+HChapYsaL8/f0VFRWliRMnOhyPiorSm2++qY4dOyokJETFihXTDz/8oDNnzqhVq1YKCQlR5cqVtXnzZvs5Z8+e1dNPP6277rpLQUFBqlSpkr744oubrufGOyUz+zy2b9+u+vXrK1euXAoNDdU999zjEMONvvzySzVq1Mhhe8gffvhBvXr1Ups2bVS8eHFVqVJF3bp108CBA+19bDab3n77bZUqVUr+/v4qWrSoRo8ebT++c+dONWjQQIGBgcqTJ4969uyphIQE+/HOnTvrscce0+jRo1WoUCGVLVtWknT8+HG1bdtW4eHhioiIUKtWrRQTE3PT96h48eJq2LChPvvsM12+fPmmff+tmjVravz48WrXrp38/f2dOmfevHlKTk7WzJkzVbFiRbVr1059+/bVpBt+EaBly5b68ssvXY7JGKMTJ064fN6t8nHbTFkkISFB7du310cffaQ333zT3h4XF6dPPvlEn3/+uRo0aCBJmjVrlsqXL6+NGzfqvvvu09KlS7Vnzx4tX75cBQoUUNWqVTVq1CgNHjxYI0aMkJ+fn2bMmKHixYvbk0T58uW1du1aTZ48WU2aNJEkTZo0ST169FCXLl0kSTNmzNCiRYs0c+ZMDRkyJN24r1y5oitXrthfx8fHS7pWiEzdi9discjLy0s2m83hf5qp7Tfu2ZtRu5eXlywWS7rtUtq7CzNq9/b2ljEm3fYbY8yonTWxJtbEmrJiTVar1T6Xt7f3HbGmzNpZE2tiTazp36wpNW+mHr8T1nQnfk6siTWxpuyxptQx07tjJaeuKaPYWRNrYk2sKSvWlNnP6Fm9ptT5Ur8sFku6Bcf02j/68EPV9/VVrUyehVZLUj0/P3380Ud67fXXnR7/Zu3Xu/F4/fr1VaVKFX3zzTfq1q2bLBaL2rRpo8DAQP38888KCwvTBx98oIcfflj79+9X27ZttXPnTi1ZskTLli2TxWJRaGiojDEO54WHh2vGjBn28yIiIrRo0SI9/vjj+t///qc5c+YoOTlZP//8s4wx+uabb1SlShX16NFDPXr0sMd6+PBhNW3aVKNGjdInn3yiM2fO6MUXX1SfPn00c+ZMSdeKQn///bd+/fVX+fr6ql+/foqNjc30c1q5cqUuX76shx9+WIUKFdIDDzygSZMmKSQkxH6uJA0dOlTjx49XmTJlNHToUD399NM6cOCAfHx8tGXLFrVt21bDhw/XU089pfXr16t3796KiIhQly5d7GNMnjxZo0eP1muvvaYpU6aoQ4cOuv/++9WlSxe9/fbbGjJkiDp27Khdu3bJYrEoMTFR1atX1yuvvKLQ0FAtWrRIHTp0UIkSJXTvvffai6LXx3njZ5zZ59G+fXtVq1ZN06ZNk7e3t7Zt2yYfH58M37M1a9bomWeecWiPjIzUr7/+ql69eilv3rzpXnNDhgzRxx9/rEmTJunBBx/UyZMntW/fPhljdOnSJTVp0kS1a9fW77//rtOnT6tHjx7q06ePZs2aZR9jxYoVCg0N1dKlSyVdu9uvSZMmuu+++7R69Wr5+Pho9OjRatq0qXbs2CFfX980cVgsFu3evVtz5szRa6+9pl69eql169bq3Lmz6tSp41BolqTPP/9czz33XLprSvXzzz+rTp06GR6//n28/rPK6JrcsGGDHnroIfn6+tqPN27cWG+99ZbOnTun3LlzS7pWUDxx4oRiYmJUrFixdOdOzZE3/iyeWY7ISjmu6Ne7d281b95cDRs2dCj6bdmyRVevXlXDhg3tbeXKlVPRokW1YcMG3XfffdqwYYMqVaqkAgUK2Ps0adJEL7zwgnbv3q1q1appw4YNDmOk9kndRjQ5OVlbtmzRq6++aj/u5eWlhg0bpvltg+uNHTtWI0eOTNO+d+9e5cqVS5IUERGhokWL6sSJEw63e0ZGRioyMlIxMTG6ePGivb1IkSLKkyePDhw4oKSkJHt7iRIlFBoaqj179jj8T7Rs2bLy8/PTzp07HWKoVKmSkpOTtX//fnubt7e3KlWqpIsXLzrsUxsQEKBy5crp/PnzOn78uL09V65cKlmypGJjYx1+I4Y1sSbWxJqyYk3GGJ07d04XLlxQvnz57og13YmfE2tiTawp+6xp9+7dOnfunHbv3i2LxXJHrOlO/JxYE2tiTdljTX5+fpKu/Wb+X3/9dUes6U78nFgTa2JN2WdNqT+jnzlzRoUKFbrta/rzzz/l5eWlpKQkGWMUFBQkm83mcIOFxWJRYGCgrFarwzaE33/+ufpfF8PNdExM1OTPP9drr7+uK1euOBQh/fz85OPjY48hlb+/v7y9vZWYmOgwVkBAgCwWi1JSUmS1Wu3HAwMDZYxRUlKSSpcurV27dikxMVFbt27Vb7/9ppiYGPvdSW+++aa+//57ffXVV+rUqZMCAgLk5eWl3LlzKyAgQFevXtWKFSvs5wUFBcnf319jxozRd999py+++EJdu3bV6NGj1a5dO/3vf/+zr6lMmTKyWq2KiIiQl5eXAgICFBYWJula8XXs2LF66qmn7MWXwoULa8qUKapfv74mTpyo48ePa/HixVq9erXuu+8+GWM0depUVa9eXVevXlViYmKGn9Mnn3yitm3bKjk5WSVLllRUVJS++OIL9ejRQykpKfZrpm/fvmrUqJH8/f01dOhQVatWTbt27VLZsmU1ceJEPfzww3rllVdks9n01FNPaceOHZowYYK6dOli/5waN26sjh07yt/fX8OGDdP06dNVtWpVtWjRQpI0aNAgPfDAA4qJiVFkZKQiIiLUu3dv++fUvXt3LV68WJ9//rkqVaqkoKAgGWOUkpJi/0xTrwer1aro6Gj75xEYGKiAgACNGzfO4fM4duyYBg0apBIlSshqtapw4cKSpJSUFPn6+qa59o4ePapChQo5XHujR49Whw4dFBkZqfLly+u+++5T8+bN1aRJEwUEBCghIUHvvvuuJk2apLZt20qSSpYsqQceeECJiYmaM2eOkpKSNGPGDOXLl0/ly5fXhAkT1KZNGw0fPty+LWZwcLDeffdd+9+V5s2bJ5vNphkzZti3pn3//fdVqFAhRUdHq27dug7f276+vvL19VVUVJRef/11vfbaa1q9erW+/PJLtWzZUnny5NEzzzyj9u3bKyoqSv7+/nr00UfT7PDo7+8vi8VivzYKFSqkxMREh++n66Vee8YY+/WYUY7w8vLSqVOnVKxYMYfv44iICEnX7mxMvcsytfh39OhRRUZGpskRqZ/jn3/+aS9mlihRQsHBwbd169U0TA7yxRdfmLvvvtskJiYaY4ypW7eu6devnzHGmHnz5hk/P78059SsWdO88sorxhhjevToYRo3buxw/NKlS0aS+fnnn40xxpQuXdqMGTPGoc+iRYuMJHP58mXz119/GUlm/fr1Dn0GDRpk7r333gxjT0pKMnFxcfav48ePG0nmzJkzJiUlxaSkpBir1WqMMcZqtdrbrm+/vu1m7TabLcN2m83mdLsxJsP2G2PMqJ01sSbWxJqyYk1Xrlwxf/75p0lOTr5j1nQnfk6siTWxpuyzptS8eeXKlTtmTXfi58SaWBNryh5runLlijl48KBJTk6+Y9Z0J35OrIk1sabss6bMfkbP6jUlJCSY3bt3m8uXL9vHSD12/Vd67aULFTLLJWOc+FommdKFCrk0/s3abTab6dSpk2nVqlW6x9q2bWsqVKhgbDabmTp1qvHy8jLBwcEOX15eXmbQoEHGZrOZYcOGmSpVqjjM+d5772V6XmBgoJk5c2aGsRcrVsxMmjTJob1GjRrGz8/PYcygoCAjyezevdt8++23xsfHx+FzstlsJjw83D5Weu/LuXPnTEBAgPn999/tbW+//bZ58MEH7f0PHz5sJJlNmzbZxzl79qyRZKKjo43NZjPVqlUzI0aMcBj722+/Nb6+vvaYihUrZt566y2HPpLM/Pnz7a8PHTpkJJlt27YZm81mrl69akaOHGnuvvtukzt3bhMcHGx8fHxMmzZt7LHUrVvX9O3b1z5GsWLFzOTJk53+PIYNG2Z8fHzMww8/bMaMGWMOHDhw02vJz8/PfPXVV2naU1JSzG+//WYmTZpkHn/8cePt7W26du1qbDab2bhxo5FkDh06lO7n/tJLL5l69eo5zHn+/HmH97hTp06mYcOGDucNGDDAeHt7p1mfxWIx06ZNc+n7Iy4uzjzzzDNGksP3yM2+n5z9Sh3jxms7o7EbNWpkevTo4dC2a9cu+/We2nblyhV7LSm9cRITE83u3btNQkKCQx5LSUkxW7duNZJMXFycud1yzJ1+x48fV79+/bRs2TKH/WtzCn9//3T3kPXz85O3t7dDW+otnze6sZ872i0WS7rtGcXoajtrYk0ZtbMm1nT9nN7e3ipduvQtxehqO58Ta8qqGF1tZ02sKatilK79HfP6vHmz/jllTXfi58SaWBNryh5r8vb2VsmSJdM9X8qZa8osRtbEmrKqnTX9N9fkzp/RU9stFov9S1KaLQFT3dgeFhqqU3//nW7fG52W7He7OTt+Zu03O753714VL15cFotFCQkJKliwoKKjo9P0Cw8PT3ftFotFly5dyvS8wMDATGO8fnzp2iO2nnvuOfXt2zdN36JFi+rAgQPpnndj243HvvjiCyUlJem+++6zt5n/v1Xsn3/+qTJlytjP8fPzs//5+i0Srx8zsz9fP0aqm407YcIEvfvuu5oyZYoqVaqk4OBgvfTSS0pOTk4zfnrrdubzGDlypNq3b69FixZp8eLFGjFihL788kv7cxZvHDdv3rw6f/58mnZvb2/VrFlTNWvWVP/+/TV37lx16NBBr732moKCgjKM8/o5rv9ven8ODg52OP/SpUu655570n1+Y758+Zz6/ti6das+/fRTffHFF7JYLHr55ZfVvXt3e5958+Zlur3n4sWLb7q9Z0ZruzGWVJGRkYqNjXU4FhsbK0kqWLCgvf38+fNOrdXb29ulv2tmtRxT9NuyZYtiY2NVvXp1e5vVatXq1as1depULVmyRMnJybpw4YLCw8PtfU6fPm2/HTUyMlK//fabw7inT5+2H0v9b2rb9X1CQ0MVGBho/8DS65M6hits6TwzAADgyGazKTY2Vvnz58/whw8AwP8hbwKA88iZAOCanJQ3Wz3zjD4bM0btndji89PAQD32zDNuiEr69ddftXPnTvXv31+SVL16dZ06dUo+Pj6KiopK9xw/P780z0N05rzKlStrxYoV6tKli0vj7tmzR6VKlUr3nHLlyiklJUVbtmxRzZo1JUn79+/PdAvDTz75RAMGDFDnzp0d2nv16qWZM2dq3LhxNz0/Vfny5bVu3TqHtnXr1qlMmTIZFo+dsW7dOrVq1UrPPvusJNmLkRUqVHDqfGc+D+na9qplypRR//799fTTT2vWrFn2ot+NqlWrpj179mQ6d2qMly5dUunSpRUYGKgVK1aoe/fuafqWL19es2fP1qVLlxQcHCzp2tq9vLxUtmzZm65v/vz5yp8/v0JDQzONKdWJEyc0d+5cffbZZzp06JBatmypTz75RE2bNpWPj2OJ6tFHH1WtWrVuOt5dd93l9NzOqF27toYOHaqrV6/an0u4bNkylS1b1r6lpyTt2rVLvr6+qlixokvj22y2NPWk2yl7Z+XrPPzww9q5c6e2bdtm/6pRo4bat29v/7Ovr69WrFhhP2f//v06duyYateuLenah7dz5057lVa69uGFhobavylq167tMEZqn9Qx/Pz8dM899zj0sdlsWrFihb2PK4wbH+AIADmVMUanTp0iZwKAk8ibAOA8ciYAuCYn5c3uPXpo5dWr2pRJv02SopOT1S2dAsm/deXKFZ06dUp//fWXtm7dqjFjxqhVq1Zq0aKFOnbsKElq2LChateurccee0xLly5VTEyM1q9fr6FDh2rz5s2SpKioKB05ckTbtm3TP//8oytXrjh13vDhw/XFF19o+PDh2rt3r3bu3Km33nrLHl9UVJRWr16tv/76S//8848kafDgwVq/fr369Omjbdu26cCBA/r+++/Vp08fSdeew9i0aVM999xz2rRpk7Zs2aLu3bvb7ypMz7Zt27R161Z1795dd999t8PX008/rTlz5tifE5eZAQMGaMWKFRo1apT+/PNPzZkzR1OnTtXAgQNd/4CuU7p0aS1btkzr16/X3r179dxzz7lUrMns80hMTFSfPn0UHR2to0ePat26dfr9999Vvnz5DMds0qSJ1q5d69DWunVrTZ48WZs2bdLRo0cVHR2t3r17q0yZMipXrpwCAgI0ePBgvfLKK/r000916NAhbdy4UZ988okkqX379goICFCnTp20a9curVy5Ui+++KI6dOigAgUKZBhL+/btlTdvXrVq1Upr1qzRkSNHFB0drb59++rEiRMZnlesWDF9++236t27t06ePKmvv/5aLVq0SFPwk649W7RUqVI3/brZdZacnGyvHyUnJ+uvv/7Stm3bdPDgQXufqVOn6uGHH7a/fuaZZ+Tn56du3bpp9+7dmj9/vt555x29/PLLDmOvWbNGderUuen86THGuLXol6Oe6Xejutc9088YY55//nlTtGhR8+uvv5rNmzeb2rVrm9q1a9uPp6SkmLvvvts0btzYbNu2zfzyyy8mX7585tVXX7X3OXz4sAkKCjKDBg0ye/fuNe+//77x9vY2v/zyi73Pl19+afz9/c3s2bPNnj17TM+ePU14eLg5deqU07HHxcUZSebcuXP/7k0AgP+AlJQU88cff9ifLwAAuDnyJgA4j5wJAK5xd95MTEw0e/bsMYmJibd0/qQJE0w+f3+zMYNn+W2UTD5/fzNp4sQsjtyYTp06GUlGkvHx8TH58uUzDRs2NDNnzrQ/+zBVfHy8efHFF02hQoWMr6+vKVKkiGnfvr05duyYMcaYpKQk8+STT5rw8HAjycyaNcup84wxZuHChaZq1arGz8/P5M2b1zzxxBP2Yxs2bDCVK1c2/v7+5vpywW+//WYaNWpkQkJCTHBwsKlcubIZPXq0/fjJkydN8+bNjb+/vylatKj59NNP7c+3S0+fPn1MhQoV0j128uRJ4+XlZb7//ntz5MgRI8n88ccf9uOpz5tbuXKlvW3BggWmQoUKxtfX1xQtWtSMHz/eYcz0YpFkvv32W/vrG+c6e/asadWqlQkJCTH58+c3r732munYsaNp1aqV/ZwbaxI3znOzz+PKlSumXbt2pkiRIsbPz88UKlTI9OnT56bX9tmzZ01AQIDZt2+fve3DDz809evXN/ny5TN+fn6maNGipnPnziYmJsbex2q1mjfffNMUK1bM/h6NGTPGfnzHjh2mfv36JiAgwERERJgePXqYixcv2o+nPo/yRidPnjQdO3Y0efPmNf7+/qZEiRKmR48eN31W3d69ezM8ltVSP9Mbv+rWrWvvM3z4cFOsWDGH87Zv324efPBB4+/vb+666y4zbty4NGOXLVvWfPHFFxnOnVGuSklJMWvWrHHbM/0sxuSAX8nIQL169VS1alVNmTJFkpSUlKQBAwboiy++0JUrV9SkSRNNmzbNYdvNo0eP6oUXXlB0dLSCg4PVqVMnjRs3zqGqHB0drf79+2vPnj0qXLiwXn/99TS3HE+dOlXjx4/XqVOnVLVqVb377ruZ3nZ6vfj4eIWFhencuXMOt4gCANKyWq3auXOnKlWq9K+2aQCA/wryJgA4j5wJAK5xd95MSkrSkSNHVLx4cQUEBNzSGJMnTdKQV15RPV9fdUxKUgFJpyR9GhCgVVevatzbb6v/DXf1ANnFoEGDFB8frw8++MDTofxnLV68WAMGDNCOHTvSvUNRyjhXWa1WbdiwQXXq1FFcXJxLW6Peihxd9MvJUot+58+fd3gGIQAgLZvNphMnTqhw4cLZ/nkBAJAdkDcBwHnkTABwjbvzZlYU/STp1KlT+uTjj/Xd558rLi5OYWFheuyZZ9Ste3eHm0aA7ObChQuaNm2ahgwZwt9VPGTBggUqUqTITW/8yihX2Ww27d27V3fffTdFvztZatHPHR8yAAAAAAAAAOREWVX0A4Db6Wa5yp31IMrCHmaz2TwdAgBkezabTceOHSNnAoCTyJsA4DxyJgC4hrwJAM6z2Ww6fvy42+aj6Odh3GgJAJkzxujcuXPkTABwEnkTAJxHzgQA13gqb5KnAWRnGeUoY4zOnz/vtjgo+gEAAAAAAAAAsiVfX19J0uXLlz0cCQBkLDVHpeYsT/Hx6OwAAAAAAAAAAGTA29tb4eHhio2NlSQFBQXJYrF4OCoAuMYYo8uXLys2Nlbh4eHy9vb2aDwU/TyM/0EBQOYsFosiIyPJmQDgJPImADiPnAkArvFE3oyMjJQke+EPALKb8PBwe666nsViUYECBdwWh8WwGbJHxMfHKywsTHFxcQoNDfV0OAAAAAAAAACQrVmtVl29etXTYQCAA19f35ve4efOehB3+nmY1Wr1dAgAkO1ZrVbFxMQoKirK47fIA0BOQN4EAOeRMwHANZ7Mm97e3uRqADmK1WrV4cOH3Tafl9tmAgDgX7h48aKnQwCAHIW8CQDOI2cCgGvImwDgvISEBLfNRdEPAAAAAAAAAAAAyOEo+gEAAAAAAAAAAAA5HEU/D7NYLJ4OAQCyPYvFoiJFipAzAcBJ5E0AcB45EwBcQ94EAOdZLBYVLlzYbfP5uG0mpMvLi7orAGTGy8tLefLk8XQYAJBjkDcBwHnkTABwDXkTAJzn5eWliIgI983ntpmQLqvV6ukQACDbs1qt2rdvHzkTAJxE3gQA55EzAcA15E0AcJ7VatWff/7ptvko+gEAcoSkpCRPhwAAOQp5EwCcR84EANeQNwHAee7MmRT9AAAAAAAAAAAAgByOoh8AAAAAAAAAAACQw1H08zAvLz4CAMiMl5eXSpQoQc4EACeRNwHAeeRMAHANeRMAnOfl5aWoqCi3zefjtpmQLovF4ukQACDbs1gsCg0N9XQYAJBjkDcBwHnkTABwDXkTAJzn7pzJr2N4mNVq9XQIAJDtWa1W7dy5k5wJAE4ibwKA88iZAOAa8iYAOM9qtWr37t1um4+iHwAgR+CHCQBwDXkTAJxHzgQA15A3AcB57syZFP0AAAAAAAAAAACAHI6iHwAAAAAAAAAAAJDDWYwxxtNB/BfFx8crLCxMFy5cUFhYmKfDAYBszRijpKQkBQQEyGKxeDocAMj2yJsA4DxyJgC4hrwJAM4zxig2NlaRkZGKi4tTaGjobZ2PO/0AADmCn5+fp0MAgByFvAkAziNnAoBryJsA4Dx35kyKfh5ms9k8HQIAZHs2m007d+4kZwKAk8ibAOA8ciYAuIa8CQDOs9ls2r17t9vmo+gHAAAAAAAAAAAA5HAU/QAAAAAAAAAAAIAcjqIfAAAAAAAAAAAAkMNZjDHG00H8F8XHxyssLEwXLlxQWFiYp8MBgGzNGCObzSYvLy9ZLBZPhwMA2R55EwCcR84EANeQNwHAecYYXbhwQREREYqLi1NoaOhtnY87/QAAOUJycrKnQwCAHIW8CQDOI2cCgGvImwDgPHfmTIp+Hmaz2TwdAgBkezabTfv37ydnAoCTyJsA4DxyJgC4hrwJAM6z2Ww6cOCA2+aj6AcAAAAAAAAAAADkcBT9AAAAAAAAAAAAgByOoh8AIEfw9vb2dAgAkKOQNwHAeeRMAHANeRMAnOfOnGkxxhi3zQa7+Ph4hYWFKS4uTqGhoZ4OBwAAAAAAAAAAAFnMnfUg7vTzMGquAJA5Y4zi4+PJmQDgJPImADiPnAkAriFvAoDzUnOmu1D08zCbzebpEAAg27PZbDp8+DA5EwCcRN4EAOeRMwHANeRNAHCezWZTTEyM2+aj6AcAAAAAAAAAAADkcBT9AAAAAAAAAAAAgByOoh8AIEcICAjwdAgAkKOQNwHAeeRMAHANeRMAnOfOnGkxPHHVI+Lj4xUWFqa4uDiFhoZ6OhwAAAAAAAAAAABkMXfWg7jTz8N44C0AZM5ms+ns2bPkTABwEnkTAJxHzgQA15A3AcB5NptN586dc9t8FP08jBstASBzxhgdP36cnAkATiJvAoDzyJkA4BryJgA4zxijEydOuG0+in4AAAAAAAAAAABADkfRDwAAAAAAAAAAAMjhKPoBAHKEXLlyeToEAMhRyJsA4DxyJgC4hrwJAM4LCQlx21wWw+bLHhEfH6+wsDDFxcUpNDTU0+EAAAAAAAAAAAAgi7mzHsSdfh5ms9k8HQIAZHs2m02nTp0iZwKAk8ibAOA8ciYAuIa8CQDOs9lsOn36tNvmo+jnYdxoCQCZM8bo1KlT5EwAcBJ5EwCcR84EANeQNwHAecYYin4AAAAAAAAAAAAAnEfRDwAAAAAAAAAAAMjhKPp5mMVi8XQIAJDtWSwWRUREkDMBwEnkTQBwHjkTAFxD3gQA51ksFuXOndt98xk2X/aI+Ph4hYWFKS4uTqGhoZ4OBwAAAAAAAAAAAFnMnfUg7vTzMJvN5ukQACDbs9lsOnbsGDkTAJxE3gQA55EzAcA15E0AcJ7NZtPx48fdNh9FPw/jRksAyJwxRufOnSNnAoCTyJsA4DxyJgC4hrwJAM4zxuj8+fNum4+iHwAAAAAAAAAAAJDDUfQDAAAAAAAAAAAAcjiKfh5msVg8HQIAZHsWi0WRkZHkTABwEnkTAJxHzgQA15A3AcB5FotFBQoUcN98hs2XPSI+Pl5hYWGKi4tTaGiop8MBAAAAAAAAAABAFnNnPYg7/TzMarV6OgQAyPasVqsOHTpEzgQAJ5E3AcB55EwAcA15EwCcZ7VadfjwYbfNR9EPAJAjXLx40dMhAECOQt4EAOeRMwHANeRNAHBeQkKC2+ai6AcAAAAAAAAAAADkcBT9AAAAAAAAAAAAgByOop+HWSwWT4cAANmexWJRkSJFyJkA4CTyJgA4j5wJAK4hbwKA8ywWiwoXLuy2+XzcNhPS5eVF3RUAMuPl5aU8efJ4OgwAyDHImwDgPHImALiGvAkAzvPy8lJERIT75nPbTEiX1Wr1dAgAkO1ZrVbt27ePnAkATiJvAnDVri93acfcHZ4OwyPImQDgGvImADjParXqzz//dNt8FP0AADlCUlKSp0MAgByFvAnAWQd/OaiFTy+UsRlPh+Ix5EwAcA15EwCc586cSdEPAAAAAID/sJKNS6r+qPoq3ay0p0MBAAAA8C9Q9AMAAAAA4D/GZrXpj5l/yBgji5dFD732kILyBun8kfNaNnjZf/quPwAAACCnoujnYV5efAQAkBkvLy+VKFGCnAkATiJvAsjMoaWH9EO3H7Tn6z0O7esnrNf6t9fr2LpjHorM/ciZAOAa8iYAOM/Ly0tRUVFum89ijOHX9zwgPj5eYWFhiouLU2hoqKfDAQAAAAD8h1yIuaCtH29Vta7VlLtEbnt7ckKy/vr9LxWvX9yD0QEAAAB3DnfWg/h1DA+zWq2eDgEAsj2r1aqdO3eSMwHASeRNAJkJjwpXgzcbOBT8JMkvxO8/V/AjZwKAa8ibAOA8q9Wq3bt3u20+in4AgByBHyYAwDXkTQAZuRR7SbYUW4bHjTHaNnubts3e5r6gPIycCQCuIW8CgPPcmTMp+gEAAAAA8B/y03M/aXLRybp6+Wq6xy0Wi1aPWq21Y9e6OTIAAAAA/4aPpwMAAAAAAADuU+jeQgrMGyjfIN8M+zzx+RMKvYvnzwMAAAA5icUYYzwdxH9R6oMbL1y4oLCwME+HAwDZmjFGSUlJCggIkMVi8XQ4AJDtkTcBwHnkTABwDXkTAJxnjFFsbKwiIyMVFxen0NDb+4t1bO8JAMgR/Pz8PB0CAOQo5E0A6bFZM36W340u/3NZZ/acuY3RZB/kTABwDXkTAJznzpxJ0c/DbDbnf+ACgP8qm82mnTt3kjMBwEnkTQDpuXjyoiYVmqQtH27JtK8xRlPLTtX3Xb93Q2SeRc4EANeQNwHAeTabTbt373bbfDzTDwAAAACA/4Azu88orGiYgvMHZ9rXYrHowf89qICwADdEBgAAACArUPQDAAAAAOA/oETDEirRsITT/e8fcP9tjAYAAABAVmN7TwAAAAAAkCFjjKdDAAAAAOAEi+Fv7x4RHx+vsLAwXbhwQWFhYZ4OBwCyNWOMbDabvLy8ZLFYPB0OAGR75E0A6VkzZo1Ci4SqSocqTvU3xujTBp8qIDxAT3371G2OznPImQDgGvImADjPGKMLFy4oIiJCcXFxCg0Nva3zcacfACBHSE5O9nQIAJCjkDcBXM8Yo9VvrtbOeTudPsdiscg3yFc+gXf+k0HImQDgGvImADjPnTmTop+H2Ww2T4cAANmezWbT/v37yZkA4CTyJoD09NnXR4+894hL5zyz6Bk9+fmTtymi7IGcCQCuIW8CgPNsNpsOHDjgtvnu/F/XAwAAAADgP85isSisKI+WAAAAAO5k3OkHAAAAAMAd7tKZS4o/ES9jMy6dZ0uxafWbq7Xloy23KTIAAAAAWYWiH+BBWz7aophVMZ4OA8gRvL29PR0CAOQo5E0A19vy4RZNLjJZJ7eedOk8i7dFG6ds1LZZ225PYNkEORMAXEPeBADnuTNnWowxrv2aH7JEfHy8wsLCFBcXp9DQUE+HAw9IPJ+otyPeVvPpzVXj+RqeDgcAAADAHezQskP688c/Vf+N+goID3Dp3H/2/aPQwqHyC/G7TdEBAAAAdy531oO408/DqLn+hxmp/pv1FVrk2jf5vu/26fuu33NNAOkwxig+Pp7vDwBwEnkzZ7Gl2DwdAv4DSjYqqUfefcTlgp8k5S2X944u+JEzAcA15E0AcF5qznQXin4eZrPxA/5/VWBEoB4a+pDKNC8jSdrz9R7tXbhX5w6e83BkQPZjs9l0+PBhciYAOIm8mXMcXXNU75d/X//s/8fToQAZsiZbdXrnaSWcSvB0KLcFORMAXEPeBADn2Ww2xcTEuG0+in5ANvHIe4/opWMvKU/pPJ4OBQAAALeRMUbrJ65XwukEXb18VZf/uaz44+77zU/899hSbPqk9idaP2H9LZ1/bO0xzag8Q7vm78riyAAAAABkJR9PBwD8FxljNLXsVJVuXlpNJzeVdO3OPwAAANz5jq87rmUDl+nMrjNqNauV+sX0U0CY61suAs66/M9lxR2L08WTF2/p/AKVC+jB/z2owvcVzuLIAAAAAGQlin6AByRdSFJQ3iB5+3k7tBtj9Nt7v+nMnjNqMaOFh6IDsqeAAP4xFABcQd7Mvoo+WFStv2qtEg+XkCR7wS/hVIJ2zNuh+wfc78nwcAcKiQzRy3+9fMvPXgrKG6SHRz+cxVFlL+RMAHANeRMAnOfOnGkxPHHVI+Lj4xUWFqa4uDiFhoZ6OhxkI189+ZWOrz+u3nt7KyCcv0ABAAD8V3zX+Tttn7Nd3Td111333uXpcAAAAAAAWcCd9SCe6edhPPAWN2r2fjP1O9KPgh9wHZvNprNnz5IzAcBJ5M3s67epv+mPWX+ke6zhuIZq83UbCn7IckfXHNXur3YrJSnllsf47f3f9E6Jd255i9DsjJwJAK4hbwKA82w2m86dO+e2+Sj6eRg3Wv43rRy+Uts/3Z7usZDIEPkEsPMucD1jjI4fP07OBAAnkTezr7Xj1mrztM3pHguJDFGF1hUkScZmZLPyD2nIGltmbNGCpxb8q2vKy9tLvkG+uvzP5SyMLHsgZwKAa8ibAOA8Y4xOnDjhtvmoLABulnIlRWvHrlXJxiVVpWOVdPskX0rW2nFrlatgLtXsVdPNEQIAAOB26bq2q65cvHLTPonnEvVN+290V627VG9EPfcEhjvag68+qHKPl5NfsN8tj1Hj+Rqq8XyNLIwKAAAAQFaj6Ae4mY+/jwb8PUBJcUkZ9wnw0R8f/6E8ZfJQ9AMAALiDhEeFZ9rHN9hXKVdSdOqPU0q5kiIff35sw7+T/+78yn93fk+HAQAAAOA246dHwAOC8gYpKG9Qhse9vL3UZW0Xp/5RCPivyJUrl6dDAIAchbyZ/SSeS1TKlRSFRIbIYrFk2M/H30cdV3S8aR/AWcZmZEuxydvP+1+PtfOLnbr8z2XVerFWFkSWvZAzAcA15E0AcF5ISIjb5uKZfh7m7f3vf/BCznJi4wnFHY/LtF9EyQh5efMtCkjXcmXJkiXJmQDgJPJm9vTHrD80qdAkHV93PNO+FPyQVf7Z/49GB47W2nFr//VYv7//u1aNWHXHPcOJnAkAriFvAoDzvL29VaJECbfNR0XBw2y2W3+QOnKmhU8v1Nwmc53qG7s7VqveWCVju7N+qAZcZbPZdOrUKXImADiJvJk9RVaJVI0XaihvubxO9V/71lpFj4i+vUHhjmexWFTu8XJOX3c30+z9Zuq+qXsWRJW9kDMBwDXkTQBwns1m0+nTp902X44p+k2fPl2VK1dWaGioQkNDVbt2bS1evNh+PCkpSb1791aePHkUEhKiJ598Ms0beezYMTVv3lxBQUHKnz+/Bg0apJSUFIc+0dHRql69uvz9/VWqVCnNnj07TSzvv/++oqKiFBAQoFq1aum333675XXdab8hiZszxqju8Lp6YPADTvXf+flORQ+P1rF1x25zZED2ZozRqVOnyJkA4CTyZvZUomEJNZ/W/KbbvF9v3zf7tOXDLXyO+Ffylsurtgvaqtxj5f71WJFVIhVRKuKOuxOVnAkAriFvAoDzjDEU/dJTuHBhjRs3Tlu2bNHmzZvVoEEDtWrVSrt375Yk9e/fXz/++KO+/vprrVq1Sn///beeeOIJ+/lWq1XNmzdXcnKy1q9frzlz5mj27NkaNmyYvc+RI0fUvHlz1a9fX9u2bdNLL72k7t27a8mSJfY+8+fP18svv6zhw4dr69atqlKlipo0aaLY2Fj3vRnIsSwWi6p2rqqqnao61f+enveoy5ouKvpA0XSPXzh6wWGbnk3vbtLacWv5SxcAAMAdoM3XbdTvcL87rsCCnO3yP5d18eRFT4cBAAAAIB05pujXsmVLNWvWTKVLl1aZMmU0evRohYSEaOPGjYqLi9Mnn3yiSZMmqUGDBrrnnns0a9YsrV+/Xhs3bpQkLV26VHv27NHcuXNVtWpVPfLIIxo1apTef/99JScnS5JmzJih4sWLa+LEiSpfvrz69Omj1q1ba/LkyfY4Jk2apB49eqhLly6qUKGCZsyYoaCgIM2cOdMj7wvubOHFwlX0waKyeKX9hx5jjL599lvtnr9btpRr2ynEHY/T6jdX60LMBYe+J7ee1Oy6s3Vmzxl3hA0AAIAbWJOt+uT+T7Tp3U1OnxNWNEw+AT63MSr8F6x9a62WDlyaJb8YePHvixqfb7xWj1qdBZEBAAAAyGo58idIq9Wqr7/+WpcuXVLt2rW1ZcsWXb16VQ0bNrT3KVeunIoWLaoNGzbovvvu04YNG1SpUiUVKFDA3qdJkyZ64YUXtHv3blWrVk0bNmxwGCO1z0svvSRJSk5O1pYtW/Tqq6/aj3t5ealhw4basGHDTWO+cuWKrly5Yn8dHx8v6dp+rlarVdK1u8C8vLxks9kcfiBLbU/tl1m7l5eXLBZLuu2pczrT7u3tLWNMuu03xphRO2tybF/y0hLFrIxR5zWdFRge6NSajDE6ueWk/IL9lL9ifofYm7zTRMfXHZfF+9o4Dwx5QOVbl1do0dD/x95dh0dxdXEA/s1u3I24O4SEAEGCE9yKu0MFCrSFtlS/ektpKdIChSLF3YsUDy4hQIQQIe7utjbfH9sEi+xuZndmN/d9nj4JuzN3znQ3d3fm3ntOfRs0TaM4tRip11OReisVbdq1Ia8TOSe1OyeJRAIzM7P6bTThnJp7nJwTOSdyTuScWnpOZmZm9bFqyjnJ8jhXz6k0qxTFScUoyyirb7O5c6IlNHIjcwExYN/FnnPn1FTszT3O1ddJE88p5nAMKvMrMWDlgBafk761Pjq90wmu/Vw16nWSSCSwsLAATdOvta+u59RY7OScyDmRcyLnxMQ5NXeNro7n1Fzs5JzIOZFzIufUknMyNTWFqqjVoF9UVBSCg4NRU1MDIyMjHD9+HO3atcPjx4+ho6MDMzOzl7a3sbFBTk4OACAnJ+elAb+65+uea2qbsrIyVFdXo7i4GGKxuMFtYmNjm4x9xYoV+Pbbb197/OnTpzAyMgIAWFhYwNnZGRkZGSgqKqrfxtbWFra2tkhJSUF5+fM0Kk5OTrC0tERCQgJqamrqH3d3d4eJiQliYmJeepP5+PhAR0cHUVFRL8Xg7+8PgUCAuLi4+sf4fD78/f1RXl6OpKSk+sf19PTg6+uL4uJipKen1z9ubGwMDw8P5OXl1f//JOf0+jmVVJZAIBIgLiUOHh4eMp1TVVYVLo64iIBZARi6aSji4uJA0zQoigJfm49uS7qhrKzs+TnpApUJlbDRtUHoz6FwW+gG2oXGG+FvwMRUOhhIXidyTup6TsbGxhp3Tpr4OpFzIudEzon9c6pLgV9SUqIx56QJr9PAfweCltCoqamR6ZxoMY0zvc/AOdgZY46P4eQ5aeLrpGnnFLQpCIJSAaKiohg5J68PvODr64vCwkKNe5008ZzI3xM5J3JO5JyUeU56enoad06a+DqRcyLnVHdO1RXV+OejfyCqFKH9x+2hpaWl9uekLq9TRUUFVIWiXx3W5DCBQIC0tDSUlpbiyJEj2Lp1K65du4bHjx9j7ty5L62kA4CuXbuif//+WLlyJd5++22kpqa+VJ+vqqoKhoaGOHv2LIYNGwZvb2/MnTv3pZV8Z8+exYgRI1BVVYXi4mI4ODjg9u3bCA4Ort9m+fLluHbtGu7dazxVT0Mr/ZycnFBQUFA/WElGxMk5NRb7jR9vwGeUD2wDbVGcUoxdIbsw8fBE2Ha0bfSc9g3dh5RrKVgQvQAWHhacO6emHlfX14mck/LOSSKRIDMzE46OjtDS0tKIc2rucXJO5JzIObFzTg+3PERlTiW6vtcVuqa6antOIpEImZmZcHBwAI/H07jXqanHNe2cHm55CHNXc3gM8dCYc6qLUZNeJ3JO6n1OEokE2dnZsLe3B0VRr22vjufUWOzknMg5kXMi58TEOTV3ja6O59Rc7OScyDlpwjmVpJTgd/ff0febvuj1RS/pohI1P6eGHufaOdE0jSdPniAgIAClpaUwMTGBMqnVSj8dHR14enoCADp37oywsDCsW7cOkydPhkAgQElJyUur/XJzc2FrawtAOgJ7//79l9rLzc2tf67uZ91jL25jYmICfX198Pl88Pn8Brepa6Mxurq60NXVfe1xHo8HPp//2mMNeXU7VTxe94f/qsZilPdxck6yPd7vq371v5ellaEsvQyialH9Ng2d0+gdo1HwtABWXlYQVAiQ/TAbZq5mMHMx48Q5Nfe4Or5OzT1Ozqll51RSUgInJye5Y5T3cfI6kXNiKkZ5HyfnxM450TSN5IvJ8Bwq/Y5pZGOEswvPwmOIB5yCnV7bXh3Oqa79un7zxW3U9XWS93EunlPS5STUltXCZ5RP/aCCLO10WdClyfbJ60TOqanHhRVC5EXnoU3bNtC30FeonVfPKe1mGi59egl9v+4Lj0EeMsfO9depqKgIDg4OjLx+XDmnpmIk50TOianHyTm13nNS1TV6Y4+T14mcE1Mxyvu4Op5TdVE19C30Ye5mjncevwNrP2vwtJ4fXx3PqbnHuXROEokEpaWlDW6vDA1HrCYkEglqa2vRuXNnaGtr4/Lly/XPxcXFIS0trX5FXnBwMKKiopCXl1e/zcWLF2FiYoJ27drVb/NiG3Xb1LWho6ODzp07v7SNRCLB5cuXX1r5RxANKU4qxpX/XUFORE7zGzehTbs2WBi1EC69XZrczsTBBO4D3UFRFArjCrGz305E749u0bEJgiCYVJJSgmPTjyH1RirboRCt2J3f7mDvsL3IjZJO6vIZ5YPp/05/acBPjRJjEBx257c7ODr1KCge1fzGDZCIJM1vRBCvyLyXib97/Y3og8xdB/C0eMiPyUdFjupSFBEEQRAEQRCKKcssw6bATSjPLgcA2HawBU9Luuov+2E2y9ERyqA2g36fffYZrl+/jpSUFERFReGzzz5DaGgopk+fDlNTU8yfPx/Lli3D1atXER4ejrlz5yI4OBjdu3cHAAwePBjt2rXDzJkzERERgfPnz+PLL7/EokWL6lfgLViwAElJSVi+fDliY2OxceNGHDp0CEuXLq2PY9myZdiyZQt27tyJp0+fYuHChaisrMTcuXNZ+f9CqI/MsEzc+OEGCp4WtKgdwzaGaNO2jVz7mLqYYtgfw+Ax+PWZuARBEGwRC8WIPhCNZ+eegaZp3PvjHuJOxTW/I0EwyGuEFwJmBsDY3hiA9Ga25xDpqr+YozFY67oWWQ+y2AyR0BAhP4ZgwoEJcg/6VeRWYI3zGlz4+IKSIiM0mYWXBQatGtTshEF5OHRzwPLC5egwswNjbRIEQRAEQRDKkReVh+qiaqRefz7hWlAhwKYOm3By3kkyyVUDqU16z7y8PMyaNQvZ2dkwNTVFQEAAzp8/j0GDBgEA1qxZAx6Ph/Hjx6O2thZDhgzBxo0b6/fn8/k4ffo0Fi5ciODgYBgaGmL27Nn47rvv6rdxc3PDmTNnsHTpUqxbtw6Ojo7YunUrhgwZUr/N5MmTkZ+fj6+++go5OTkIDAzEv//+CxsbG4XO69V6AYTm8h3ti4VRC2HsYKzyYxu2MUTXxV1VflyCYApFUbC1tSV9poax9LLE4vjFsPCwQGV+Ja59ew3m7ubwHuVNXmtCZdq0bYOxu8Y2+JyuiS609LRQU1zT4PNcRvpN7rHraAe7jnZy72dobQhTZ1MY2RopISpC05m7maPHhz0YbVMT+xXSZxIEQciH9JsEoT48h3piecFy4IU/Vx0jHQxcORCW3pbsBdaKUBSl8PiRQsejyVAuK8rKymBqaqqSwo0EQRAEwRU0TeOft/9BwIwAuPZ1fem5rAdZsPS2hK6JLoTVQoAGtA202QmU0HjP/n0Gvg4fbiFubIdCtAJiobSYO1+74boPBKFuch7nIPFCIrq82wU6Rjpsh0MQBEEQBEEQnKbK8SC1Se+pqcRiMdshECpSEFuA2vJaVo5dnl2Oje034ubKm6wcnyBaSiwWIzExkfSZGiA3MheRuyIRcyTmtefsg+yhayJNub1n8B6sc1+n6vCIVkIiluDcknM4NOEQa5/Nykb6TW5JuZqCH/V/RMTuCLZDIVqZo1OPYveg3Yy3m3A2AZc+uYS86DzG22YD6TMJgiDkQ/pNglAPKaEpuL3qNqoKqxrdJjcyF1H7olQYVesjFouRlJSksuOpTXpPglBnNE1ja/etsA20xZzQOSo/PsWjQItpgKzrJdRYeXk52yEQDLDtYItFTxdBz0yvye08h3nCtpNt/b8lYgl4fDJXiWAGj8/DjPMzUBhfCF1j3Sa3zXmcg4fbHqL7+91h4WmhogiZQfpN7tAx0oHPKB+F30OlaaW4ufImvIZ5wXukN8PREZpMIpaAljB/EeA/zR/OvZ1hE6C6NEXKRvpMgiAI+ZB+kyC47/GOx4jYGYH2U9o3+DwtoXFs+jGUpJTAc6gn9C30VRxh61FRUaGyY5FBP4JQAYlIguAPg2HiwE4qVyMbIyx6uoiVYxMEQdShaRoURcHc3bzZbXt/3lu6j4TGsRnHUFtai2lnpik7RKIVMXc3l+m9WJJSgrD1YbDraKd2g34Edzj1cMLk45MV3l8ikuDBxgfQMdQhg36EXCYemqiUds1czWDmaqaUtgmCIAiCIAhmjNw0EoFzA2Hi2PA9aYpHYcyuMRALxGTAT4OQQT+CUAG+Nh99/9eX7TAIokG0hEZWeBZsAmygpUs+FgjlqCqows6QnRj2+zC49nOVeT+KR4GW0JCIJRDVish7lGgxmqYRfzoedh3tGr3weZH7QHcsjl9MBvwIVpm5mmFp+lIYOxizHQpB1JOIJSjLKIOZixnboRAEQRAEQRAN0NLTgmtf1ya3setoV/87ue+iGUieLJZRFMV2CEQrIBFJELYxDIkXEtkOheCIsowy5D2R1mApjC/E1q5bsW/EPtA0N3PAUhQFJycn0meqsbwneShOKkbRsyK59x27eyxm/DuDfPEkGFGWUYYDbxyQuc6tjpEOLL0s1a7/If0mt5x77xxur7qt8P4Uj4KJowl5PQm5lKSW4Paq28iPyVdK+/tH7cdGv41KSR+qaqTPJAiCkA/pNwmC+5KvJqMoUfZ7MIUJhfir01+kvp8SUBQFR0dHlR2PDPqxjMcjL0FrcOvXW9g9aDcq8ypZOT4toXF20Vk83vGYleMT3HNz5U382f5PZD/Kho6xDvp91w8TD0/k7Bd2Ho8HS0tL0meqMde+rvik6BP4T/eXe1++Nr/+d64OTBPqQ9dYF6O2jmq0pkFDBBUCJF5MRFVB48XPuYb0m9xB0zQebX+EpEstK9xeVVCFxAuJEFQKGIqM0HTZD7Nx8eOLyH6YrZT2fcf6ImhhEITVQqW0r0qkzyQIgpAP6TflE3cqDhU5qqvnRRA0TePk3JPYPXC3zPdR9M31IagUoCS1RLnBtUI8Hg8WFqrLHkTR5O4ZK8rKymBqaor7e++jy7QubIdDKNn5D8/j0bZH+DjvY/B1+M3vwDCappFyNQVGdkZo07aNyo9PcE/m/UzEHInBwJUDXxvoS76aDHN3c06lahKLxUhISICXlxf4fNX/DRHsK88ux8m5J+Ha3xW9PunFdjhEKxO5JxLHZx7H2D1jETA9gO1wZEL6TW4RC8SoLa+FgaWBwm3cXHkTlz+9jHm35sGphxOD0RGaqqa0BvlP8mHhaQFDa0O2w+E00mcSBEHIh/SbsitMKMSGthvg1t8NMy/OZDscopWgJTQSziZAUCGQa8KrRCwBj08G85kmFovx6NEjdOnSBaWlpTAxab7USEuQPFksu/btNTLo1woM+W0IBq8azNoqKoqi4BbixsqxCW5y6OoAh64Orz1enlWOfcP3wSbABvPvzgdFUajMq4SBlQEoHrurAGtqalg9PqG44uRiPD36FO0mtlN4MNnAygAFTwtg6W3JbHBEq0PTtNyfx24D3DBo1SA4dlddOg4mkH6TO/g6/BYN+AGA1zAv6BjqwNTFlKGoCE2nZ6pHBojlQPpMgiAI+ZB+UzaWXpYYuWkkXPq6ID8mHyWpJfAa5sV2WISGo3gUvEd6y70fGfBTHlX2mWTQj2Vj94xlOwRCRdhOm0jTNECD9YEbgl2iGhFyHuc0euPa2N4YwzcMh32Qff17dmP7jXDt64qJhyeqMlRCg8T/E4+LH1+EpY+lwoN+fG0+FsUugra+NrPBEa3Ozn47QfEpzL4yW+Z9jO2M0ePDHkqMitBkpWmlKM8qh7W/NXQMdRRuxybABjYBNgxGRmi6qoIq6JrqvpQmm0k0TePErBPQMdbBiI0jlHIMgiAIglBX1UXV0DPXA0VR6PRmJ0jEEqxzXQeJSIIPUj9gJRMY0XqIhWKFvwM+OfwEN368gdlXZkPfQp/hyAhVIEO3LLPwVF0uV4IdlfmVCNsYhoLYAlbj+MXiF+wZuofVGAj2Re6JxLbgbYjcG9noNh3nday/qUjTNGwDbeE1wovUUiMU1unNTphxfgbc+rdsxTEZ8OOWGytuIHJPpNr1Dcb2xjC2N1ZoX4lIArFQzHBEhKaL2heFbcHbkBedx0h76vY3R7Bn34h9+N39d6W1T1EU8mPykR+Tr7RjEARBEIQ6omkae4buwc7+O0FLpN/deHwehm8YjsnHJ5MBP0Lp9g7diw1tNyi0r6hGhLKMMuQ9Yeb6hVA9stKPZbSIRvKVZJJ6UYPlPMrB2UVnMXzDcFj5WrEWh89oH5h7mLN2fIIbnHs7o/OCzvB5w0em7SmKwswL7Oec5/F4cHd3J0XC1ZS2gTY8Bnsw0lb6nXRc/PgiRmwcQVa8sEhQIUDYhjAY2Rqh/ZT2kEgkoPiUWqQCGb9/vEL7xZ+Ox9GpRzF2z1j4jvZlOCrmkX6TO9wHuoPiUbDyafn3wBOzTyD9TjqWxC9hIDJC03mN8IKwWqjUY7x5703wtNS/nyF9JkEQhHxIv9k0ca0Y9kH20NLTeinjlqz3YgiipawDrGHiqFjduPaT28Nvoh+09MjQEVN4PB5cXV1VdjyKJlNFWVFWVgZTU1MceecIojdHY2H0Qlj7WbMdFqEENSU1yLyfCStfK5g6kxoshPqqyK0AABjZGLEcCaFOyjLKIKwWwsLTgpE0xxn3MrCj7w6M3DwSgbMDWx4gobCqgirUltXCxNEEhyYcgq6xLsbsHKMRN38bkhuZi/NLz6PH8h7wHOLJdjhEK3X+w/PIi8zDtDPTyAxxgiAIgiAINZVxLwNigRguvV3YDoUgCBWpGw8qLS2FiYliA7Ky0sy7MmrEd4IvBqwYAAMrA7ZDIZREz0wPHoM9yIAfwbqa0hqFU4IVxBZgrfNa3PjpBsNRyUYsFiMqKgpiMUmrp24ebH6A9d7rkR2ezUh7Dl0d8GH2h2TAj0U1pdLi0wZWBjB3l64g52nxpDNYOV42Nu6fOFz58goq8yvl3tcmwAazLs9SmwE/0m9qpiG/DcHMizPJgB/BGdXF1YjcE4ms8Cy2Q2kR0mcSBEHIh/SbjasqrKpP6dmQmtIa7ArZhYsfXVRhVAQhH1pC49p313BxOXmfMkEsFuPJkycqOx4Z9GOZfWd79Pq0F1k5o8Fqy2s5UXsl9NtQXP/hOtthECw6PPEwfnf/vckvn42x9LGE/wx/eAxiJkWjIsjFhHryGOSBbh90g21HW0baoygK+uakkDRbki4nYa3LWiScTah/jK/Dx4SDEzB6x2jw+DzQNA1BhYDFKBsXdzION35kZ/ICG0i/yT5aQmOd+zpc+OgC26EQrUz2w2wcGn8IqddTlXqcqoIqHJ95HE8Oqu4mhrKQPpMgCEI+pN9s2PGZx/FnwJ8Q1YgafF7PVA9jdo3B0N+HqjgyorVIvpKMs4vPouhZkcJtUDwKz/59hsg9kQrdRyRep8o+kwz6cQRN0+QPSENt77EdmwI2sR0GovdHI/pANNthECyy72IP98HuL+WTlxVFURi9bTS8R3orITJCk7n0ccHQNUMZr/UWtS8KB8cdJJ+dKkaLaRjbG79Wo5avza9/jW/9cgt/Bf2F0rRSNkJs0vANw7EwaiEM2xgqtH/8GWldv+riaoYjIzRVbXktDNsYMrYyT1QjwvUfriNiVwQj7RGaqyixCLEnYlFdpNz+yszVDBMPT0TQgiClHocgCIIg1AEtoWHb0RaOwY5N1kNrN74dHLs5qjAyojVJvZGKsA1hEFa1rLbz1H+mYlnGMoXuIxLsItUYOeDR34/w7/v/YuaFmXDsTjp8TeM+yB18XfZTMM27OU9j6ywRshnw4wBG2ilOKoaxgzG0dMlHCNE0mqYZqePXkJyIHCRdSkJhQiGsfKya34FghMdgD7z75N0mX1dtfW1o6WlB11RXhZHJRktXC9btFa+hXBBbgOgD0ej6Xlc4BTsxGBmhqfRM9fDmvTcZa4+vw8fNn2/CsbsjOszqwFi7hObxm+gH3zG+Sj8OX5uPdhPaKf04BEEQBKEOKB4l870XiViC8sxyUg6IYFzfr/qi0/xOMLRRbLJrHQNLUo5MXVE0F/IOtkJ1hRtLSkpQcK8AN3++iZAfQ8gNJIIgOO3JoSc4OvUoZlyYAfcB7io7Lk3TqKmpgZ6entIGkQjmXf78MmIOx2B26GyYODBbpLi6qBo8bR50jbk3sKSJipOLwePzZL4gFQvEnKs5Vltei8K4Qli1tYKOoY5CbdSU1oDiUWrxviP9pubKe5IHMxcz6Bgp9j4mCGUQVAigbaCttjPBSZ9JEAQhH9JvttxfQX+hpqQG7z17j+1QCKJRqddTUVteC+8RJPNXS9A0jZysHNg72qO0tBQmJszeI3sVWfbDAR6DPTD7ymwy4EcoVUlKCfKf5rMdBsGSM4vO4My7Z1pcX9LK1wp+k/yga6L6G946OuTmprrRNdGFlr6WUurW6lvoq8XAiyagaRqn3z6NDe02oCK3QqZ96gb8Es4mIPZkrDLDk1nGnQxs6bIFj3c8VrgNPVM9tXrfkX6TfYkXE3Fn9R3UlNQw1qa1nzUZ8COalRKagoy7GSo51tWvrmKF8QqUpJao5HjKQvpMgiAI+ZB+82UFcQVY67oWj3c+lml7v0l+8JvkB7GQ1EYkmENLaCRdSpL52r05J+edxIVlpD45EwqiClR2LDLoxzKJRMJ2CIQSpd9Ox/FZx5H9MJvtUHB85nHs7L+T7TAIlmTezUTO45wWz8CzCbDB+P3j4dDFgaHIZCORSBAVFUX6TDXT69NeWBi5UGmphYuTinF37V3GvswSDaMoCj2W90CPj3vINYArqBDgxJwTuLDsAiQi9v92zdzM0O+7fnDt69qidgoTCpFwNoGZoJSI9JvcEHMkBhc+vMDozRyxQIy86DxU5lUy1iahec4uPosTs0+o5Fi2gbbwn+av1nV2SZ9JEAQhH9Jvvq66sBo6hjrQ1teWafuey3tiwE8DwNfmVoYUQr2VppVi96DduPHTDUbaG7JmCEb+NZKRtloziUSCUqNSlR2PFGTiiIhdEci4m4ERG0ewHQrBoNyoXETujkTgnEC2Q0HgvEBUF1azHQbBkrfD34agUsB2GARDastqoW2oDR6/dc/dSb2eivNLz8PI1gjtp7RnOxyNQ0toFCcVw8LTAh6DPOAxyEOu/XWMdDDpyCSYuZlxoqaspZcl+v6vb4vbOb/0PBIvJOLzis85l8KU4J7+3/ZHwIwAGFgxVw8j6VIS9o3YhxF/jkDQgiDG2iU0S8iPIRALVLNyoO24tmg7rq1KjkUQBEEQXOXUwwnvPnmX7TCIVk7HWAfD/hgGmw42jLTnM8qHkXYI1WL/DgwBAEi+kowHfz5gNPUPwb6gd4LwadmncO7lzHYo6Di3I3p81IPtMAgWKVrD6lVZ4VnY3ms7nh5/ykh7hPwufXYJv9n+xukVbvkx+bjw0QXkRuUq7Rhew70w/d/p8BrhpbRjtGaHJx7G1u5bUV2s+IQRlz4uMHWS1gG8v/4+bqy4AVGNiKkQWdF1SVeM2TGmxemSidbByNYILr1dGK11Y9fJDr0+7wX7IHvG2iQ0j+9oX/hN9GM7DIIgCIIgmhD6TSg2ddik1qvlCW4xsDRA18Vd4dLbhbE2aQmN6iKykERRFbkV2NhuIzL/zVTZMcmgH0cMWDEAn5R8Aj0zPbZDIRima6xLVgIQrHq88zHiz8Qz1p6+uT5yI3NRkc3dASdNQ9M0InZF1M/Yt/SyhGOwIwytDQEAohoR5wYg0m6l4c5vd1Caqrz0BYbWhvAc4qlWNdbUie84XwTMCADFY2aw4tH2RwjfFA6etvTrZ2157UtpP1NvpOLq11chrBIycrwX0TSNjX4bceGjltci8BziCf9p/tDSJQkziKbREhqlaaWQiJlNe2Vka4QBPw4gg34Ep1z4+AIuf36Z7TAIgiAIghUSkQT/fvAvki4lybWfsEoIiVhC0rYTnEVLaKx2WI2j046yHYraKk4slt7nYG4eaLMommt3CVuJsrIymJqaoqSkBKampmyHQyhJ6vVUGFgZoE27NmyHgqtfX0VqaCrmXJvDdiiECtE0jVXWq2DqbIq3w99mrF2JSKLSdH00TUMikYDH4zG6WkJdROyKwInZJ9D/+/7o82Wfl54TVguxb8Q+WPlaYfj64YwN0LSURCxBcVIxjGyNlDooR9PSFJTGdsbQNpCtdgLRuPLscmjpaUHfXJ/xtsUCMYqTimHlawUAOL/sPJ4ceoL5d+bD1MkUp948hUfbHmHKqSmMpxARVAjwd++/4drfFUNWD2GkTYlYwukUu6293+SC8qxyrHZYjS6LumD4+uFsh0O0InGn4nD6ndMYtXUUvEd4q+SYfwb8CYlIgkUxi1RyPKaRPpMgCEI+pN98WW5ULjYFbEKvz3phwE8DZN6P69cUhPo5OPYgakpqMPvqbMbaPPPuGRjaGKLf1/0Ya7O1kUgkKC4shpW1FUpLS2FiYqLU45EpyhxB0zQK4wohqhHBNtCW7XAIhhyZfARmrmaYf2c+26GgLKMMBXEFoCU0ZwYFCNWYHTqb8dTBbNTnEggE0NNrnauh/af5ozS9FN3f7/7ac3V/0xSfUumsoebw+DxYelkq/TgPtz7E6bdPY+rpqSq7sampaAmN4zOPoyC2AAujFjI+8MfX4dcP+AGAqbMp2rRtAxNH6ZfdwasGo9+3/WDiwPyXXx0jHbzz6B3G2vur81/Qt9DHzIszGWtTGVpzv8kJFNDt/W5w7efKeNNhG8Nw57c7mHdrHoxsjRhvn1BvPG0eTF1MYWDJXC3J5swJnaP2WWtIn0kQBCEf0m8+Z93eGu8lvid3pi8y4EcwTUtfC9oCZidEj9g4gtH2WiOKoiCSqK7UCVnpx5K6lX5FRUUwNzcHTdP42fRn2HWyw5zQOWyHRzDk8c7H0DbQJvU0CI1TXVSN8C3hsAmwgdcw5ddTE4vFiIqKgr+/P/j81pEul6ZppF5LlelmsahWBL42n1MD+um309GmXRul3wAsiC3AvT/uofNbncmkmRaiaRqPdzxG/pN8DF41mO1wOO3YjGPQMdLByE0j2Q6lUa2x32xNHmx6gPvr72P8/vGw8bdhOxyCUHukzyQIgpAP6TeZk3ItBU8OPsHQtUNJeSCC0DDJV5ORcDYBQYuCEJMRg969e6tkpR+ZTsARFEVhwE8D0HVxV7ZDIRgUODuQDPgRrCrLLIOgUsB4u7SExuXPLiNydyTjbRNSd1bfwc7+OxG1P6rZbbV0tUDxKIiFYlz9+iqSryarIMLGVeZVYnvP7YzUT2uOla8VRmwYQQb8GEBRFDrO7cjagF9teS2SLiWhKLGI8bZTrqXg/vr7qC5mpvj4uD3jOD3gR2i+oAVBeDf6XTLgR3BGbXktki4noTi5mO1QCIIgCELlMu5mKHytkXE3Aw82PUDGvQyGoyIIZgirhTg24xiufXeN7VDUTtzJONxZdQdigVilxyWDfhzSdXFXtJvQju0wCA2V/TAb0QeiIapV3VJign3Hph/DGsc1YHpRt4GVAebdmoc3tr3BaLvEcx1mdkDHNzvKVdusPLMcd367g7tr7ioxsuZRfAoDfxmIdhPJZ5q6iDkaw/qN2uLEYuwetBvRB6IZbzvmSAzOLTknLZ5NECpyY8UNHJ50GKIa8t2LUK0bP93Ao+2PVHrM3Mhc7B64GzFHYlR6XIIgCIJgm6hWhL/7/I3jM44rtH/ntzpjYdRCuPR2UWj/0rRSJF5MVGhfQrOk30nH9R+uozS9lNF2tfS0kHI1BdkPsxlttzUYsmYIFkQsgIWHhUqPSwb9CEJJIvdE4nfP35F+J53tUAAAD7c9xNGpR1FbWst2KIQK+Yz2QdC7QUoprO0U7ARtfWbzhDeltaQMqRugNbQ2xBtb3oCOkY7M+5q5mmHW5VmYdGSSssKTiYGlAXp+3BOeQzxVcrxn559hQ7sNSAlNUcnxNE1tWS2OTTuG0++cZjUOU2dTjNg0Al7DmU8Z3Puz3ph1eRaM7YwZaa+6qBrnPzyPyL3cXu3cWvpNrsqNyEXi+UTwdZXzOsQcicGtX28ppW1CfdE0jVsrbyFqb/OZAphk7WeNob8PVUnad2UhfSZBEIR8SL8pJRFJMOiXQeg4v6NC++tb6MPaz1qhfWkJjX0j9+HQuEOoKqgCANz74x7yn+Yr1B6h3p79+wxX/3cV1UXMZLipQ1EUliQswZQTUxhttzWgKAo2AdLsLKrsM0lNP5bU1fR7MYdrblQuDk84jK7vdUXXRSTNp7qL3BuJWytvYfz+8Qp/eDMp+2E2ipOL4TXcS6UDNYTmoiU08p/mQyKSwLYDSa3IhNryWuweuBs9P+2JtmPbtry9slromugyEBm3pd9Ox9GpRzFkzRC0Hdfy/2+tjVggxrPzz6BjpAO3/m5sh6MWhNVCrDBagbbj22LioYlsh0NwmFgoBl9bORd3e4ftRfLVZHxe8Tl4WmQuJyFF0zTKs8ohqhbBwlO1M4oJgiAIglBc0uUkRB+Ixqi/Rsk1cTvtVhoqcirQbnw7lKSU4HeP32FoY4iFUQthYGmgxIgJrqktr0VBbAFsAmygpavFdjitXsK5BOia6MIp2AkUj2pwPEhZyKAfS+pe5JKSEpiamgIAKnIrsKPvDnT/oDuCFgSxHCFBEETTBJUC/Gz6M7xHeGPKSeXO9qFpGuXl5TA2NlbKqkWuyH6YjT1D9qD3l73R/f3uLWrr1q+3cPvX21jweAGM7ZlZ3SSrfSP3ga/Nx+Tjk1VyPJqmNfp9QbQMTdOoyKmAkY0RKB5z75PStFIY2xtzdrCltfSbrVnekzxo6WrB3MOcvMYEZ6jrZzLpMwmCIORD+s3nmPrsO7/sPO7/cR9vPXiryYnVNE3j9qrbaDe+HczdzV97viS1BGYuZi2OhyBeJKgQIP5MPEydTeEU7MR2OJwnEUuwznUdAOCD1A8ACsjMzISTk5NKBv24eZeiFZFIJPW/G9kYYXHsYjLgRxAEI56df4Y9Q/cg60GWUtrXMdTBoF8HIWih8vssiUSCpKSkl/pMTWTXyQ7vxryLbu91a3FbFp4WMHUyZaWGGS2hGa8j2ZTWfpHZEqJaEYqT2K3lV0csEGO973r8+8G/jLZbXVSN1farcXohs+lLTZ1NOTvgB7SefpOrakpqEH0wWqm1Mq39rGHhaUH6QOIlVQVVKEkpgUSs+r/9K/+7gpVmK1Fbrn7lBEifSRAEIR/Sbz63ved2HJlypMXt9PqsF5Y8W9JsJqXsh9m49MklXP7scoPPkwG/1kkikiAvOg9igVgp7QurhDg65SgebVNt3Wh1xePzMOXUFIzcPBIUj4JEIkFKSorqjq+yIxFEK3Prl1t4cvgJ22HUu/XrLfxi+QvJ692KlKaWIvVaKmiJ8gZfgpcGw3Ooauq2aTKxQIzaMukNMsM2hozcwG07ti3evP8mK6m9pp+drvJc75lhmbj8xWVU5lWq9LjqLuliEn73+B0Ptz1kOxTwtHng6/AZr39Gi2kELQyCWwizqUuri6qRdjONlYF1gvvyovNwdMpRPD32VKnHqS6qRklqiVKPQaiXxzsfY53bOmTczVD5sQ3bGMKusx1qSmpUfmyCIAiCYAMtoaFjpAMtvZanUjRsY1g/YPfo70fIDMusf04sENd/57PvbI8pJ6Zg1NZRDbZTmlaKuFNx5Nq4lSmIK8Cf/n/i6tdXldK+obUhJhycgJ6f9FRK+5rIrqMdvIazU++aDPpxTMa9DFxcfhEVuRVsh0K0AC2hcfV/VxGxM4LtUOoZ2RjBpoMN+Dqk0HJr0fntzvi88nPYB9kr/VhioXJmErUWMUdjsNphNRIvJDLaLo8v/ZhPu5mGwvhCRtvmmsz7mbj5003kPM5hOxS1YuxgjMA5gXDt58p2KKAoCgsjF2LQykGMtmtobYgRG0eg/eT2jLYb/lc4/u79N3Kjchltl9AMlj6WGH9gPLxHeivtGGKBGKtsVuHf95ldHUuoN/vO9uj2QTdYeluq/Njd3uuG2Vdmw9TJVOXHJgiCIAg2UDwKMy/MxJgdYxhrU1AhwOm3T+P6d9frHzs47iDWe6+vn9Tt84YPdI11G9w/8WIiDow+gOyH2YzFRHCfrrEuen3eS6kT8/0m+cHSS/XfMdVNZV4lkq8mqzQD1qtIRUeOyQ7Pxu1fb8MtxI2snlFnFLAgcgHAoYqZHWZ1QIdZHdgOg1AxJutXNURQIcCmDpvg0scFo/8erdRj6enpKbV9NumZ6sHa3xp2newYb7s0rRQ7+++EU08nzAmdAwBIvZEKpx5O9YOCTEo4l4CCpwXoOL8j9ExV95q1m9AOLn1cYOVjpbJjagK7jnZK/9vVVB6DPcDT5qm8ZqY8NLnf5DrDNoaMDzS/iq/DR4+Pe8DCS/Urugnucu3nyomJHOqI9JkEQRDyIf2mcmjpaWHW5VngaT+/Xvcd6wurtlYQ1Yqgra/d5P6ufV0xZtcYWPtbKztUgkNMnU0x4McBSj+OoFIAikc1+z5szR5ufYgrX1zB1NNT4T3i+SRQVfaZFM3mkGMrVlZWBlNT09cKN1bmV6K6qBrm7ubga5MVWQRBKO7R349gE2AD+87KXem3e/BuOHRzQMj3IUo9DqG4sD/D4NbfDVa+VpCIJfhe+3v4T/PHuD3jGD/W8ZnHEbknEp8UfwI9M3IRyGUSkYRzNeke73wMAAicHchYm3fX3kVWWBZGbh4JHSMdxtoliKZIxBKlTKwgCC6jaRq3frkFHSMddF3Ule1wCIIgCEKpastrcXjiYXRd0vWlG/sEoaliT8Ti4LiDGL9/vNInOKqzwoRCROyKQN+v+r40vtPYeJAykCtRlr1a8NawjSGsfKzIgJ+aqympQWV+JavLeF+VGZaJq19dRXFSMduhECpQU1qDU/NO4f7v95V+rJkXZip9wE8ikaCwsFAji4SLakRKP0aXhV1g5StdAUdLaPhN9EPXxcq5GTdk7RDMvTmXlQE/QYWA1C2Vw4WPLmBTh02ozOdOrYfbv9zGrZW3GG0z60EWYo7EQNugdc1E1OR+Ux3s6LMDf3X+i+0wiFaGpmls77Ud13+43vzGSkBRFMI3hSN8czgrx28J0mcSBEHIh/SbQPbDbKSEpqA0rZTtUAgCuwftxrn3zin1GFZtrdBuQjsYWhsq9TjqztLLEiHfh7w0viORSFBUVKSyGMigH8saGhSqKalB3pM8FqIhmBK+JRyrrFch424G26HUy36YjevfXyeDfq2Elq4Wpp2dhi6LurAdCiNomkZ6ejqnBtKZIKgQYI3zGoR+E6qyY/K1+ZhwcAIcuzsqpX0DSwM493RWStvN2TdiH7b32K5x7xNl0TbQBl+HDwMrA7ZDqTd291hMODCB0TbH7RmHT0s/VUq6473D9+Lg2IOMt8sETe031YW1vzVsAm2Ufpy86Dz83ftvRO6JVPqxCO6rLatFaVopyjLLWIth+rnpmHtjLmvHVxTpMwmCIORD+k1pGs2Pcj9Ch5ncKqWTci0Fa5zWIGp/FNuhECoiEUtQkVuB6sJqpR7HyscKEw9NhFt/N6UeR51lP8yGRPz6ZAiappGRobpxAlLTj4P2DtuL4qRifJT7EduhEAqyCbBBp7c6wcKDOzVW/Cb6waWPC0ydTdkOhVABLT0teA3zUsmxBJUCXPniCiw8LZS2ekxTVeRUoE3bNqzVBMuNzEX0gWiE/BgCimr5gEhFbgUE5QKYe5gz0p68OszugPKsckhEErJiXgYDfhrA2GvPFGXUtQSkfaIycOn/HcEtIzeNVMlxdIx1kBedx6kVuwR79Ez1sDRtKas3YOsyCxAEQRBEa6DKOvay0tbXhrGDcavLdNKa8fg8LIxc2KoH4blAWCXEtuBt8Bntg4mHJrIaCxn046COb3ZEVUEVqQWixjyHeMJziCfbYbxE30If+hb6bIdBqIioRgS+Ll8lN6S19bURsSsCDl0dyKCfnCw8LTDn2hzWvpjdWnkL0Qej0X5qe9j4t3xFSsTOCFz65BLmXJsDlz4uDEQon47zOqr8mOqOa4NWErEEomoRY7X3BJUCJJ5PhF0nO5i5mjHS5oumnZnGeJsEIQ9TZ1MsL1rOub9lgl1svh8kIgkK4wuha6ILE0fl1iohCIIgCLbEHI1BZV4lAmcHcm5wzaGrA968+ybbYRAsUMV3wOiD0Xi07REmHJhA7jO/QlQrQs9PesK6vTXboZD0nlzUaX4n9PqkFxnwIxglqhWhIrcCwmoh26EQKnDqzVP4xeIXlbzeFI/Cu9HvYvq56Uo9jrExO6vhlKUwobD+d7Zuzg1dNxRv3n2TkQE/AHAMdkTXJV2VtlqLYE7U/ihcXH4R1UXKTf8hryOTjmCFyQrGBsIL4wpxaPyhVpv6UNP6TXWRficd/37wLwpiC5R+LIqiyIAfUS/jbgai9kdBUCFQaP/CqkIUV7esFEBuZC42+m3Ew60PW9QOG0ifSRAEIR9N7zfjT8fj4icXG3zu3rp7uPTJJRVHRBANi/snDo+2P4KoVqT0Y5VnlSPtZhopH9UAfXN99P+uP/wm+TX4vJGRkcpiIaNKLOPzSfoxTUPTNPYO34ubP99kO5SXxJ+Ox2+2vyH2RCzboRAqYN3eGq79XKGtr5oZZ8b2xkq96cjn8+Hh4aExfWZmWCb+bP8nQr8NZTUOAysD2AfZM9aeS28XDPt9GGOrtOQlqBRg96DduPz5ZVaOr07iTsbhzm93lJb2UlEu/VzQYVYH0GJmBv1MnU0xZtcY+Iz2YaS9V5WklODWr7eQE5GjlPZbQtP6TXWSfisd99bdQ1VhlUqOl/ckD7dX3SYpPgk83vkYx6Ydg6BSvkE/CS3B6AOjYb3KGr4bfFFYVdj8To2wamuFnp/0hFuIetV6IX0mQRAtJaoVNVjHSVOpc7/54uSY3MhcJF9NBi2RXn9IxBIUPSsCADzc+hBhG8JQnlX+WhvTz07HjH9ncG6VHwBU5lfi5s83kXo9le1QCBV5sPEBzrx7Bjwt5Q/1dFnYBZ+Vf8bovaTWgM/nw93dXWXHI4N+LJNIXv9CUFVQhR39duD6j9dZiIhoqdrSWmSFZaEosYjtUF5i4WmBLou6wMKTO3UGCeXp9WkvTD4+WWXHEwvFSL6SjLSbaUppXyKRICcnp8E+Ux2Zu5nDY4gHfEf7sh0KACD+TDxOLzit9vnftQ20URBbgMo8cuO7OeP2jMOi2EWcu0jttqQbxuwYw9jFioGVATrM7MDYatZXlaSW4NLyS0i5mqKU9ltC0/pNddJ9aXe8n/I+7Dur5kI49XoqLn58EVkPslRyPIK7un/QHRMOTYChtaFc+/EoaZ8roSXIq8zDX+F/KRyDtr42Bv48kJU03y1B+kyCIFpCLBDjz/Z/4g+vPxReba1u1LXfzIvOw999/q7/d9rNNOwK2YX4M/EAgNPvnMaWrltQEFeAYb8Pw5L4JTC2f76ikaZp0DQNHSMdOPVwUnn8sqgqqMLlzy4j6VIS26GonbrBX3Uz4s8RmH52ukqyBmrpaZHshA2gaRqbO23GxeUNrw6WSCTIzc1VWTzkFWJZQzdY9cz0UPC0gHMptwjZ6Jnp4aO8jzB8/XC2Q3mJbQdbDF8/HA5dHNgOhdBAEpEEe4buwc0VylnhStM0cnJy1H5Qqo6BlQGmnpoK20BbtkMBACScTcCj7Y9QlKD4ZIW4U3HY0G4DUq6lMBeYnCiKwgdpH+CNrW+wFoO64GnxYOllyXYYSqfsPsOukx3m3pyLDrM7KPU4itC0flOd8Pg8mLmYqWwlre9oX8y5PgcuvdVrkIVgnpWPFfwm+imUfeHHkB/xY8iPAIA/7v8Bgbh13LSuQ/pMzVJbVlv/WpLXlKgjqhFha7etODb9GCpyKxhtm6/DR99v+sLC06I+64mmv/fUtd+MPhCN/Jh8lKSUAAC8R3qj79d94TnEEwDgN9kPnkM8Ye5mDlNn05cG/ADgwkcXcH7ZeU4PDpm7meOtsLcQtDCI7VDUQvTBaGQ/zAYAhH4bikPjD6G6WHpPXlgtVIv782auZirNspAZlonoA9EqO546qC2rBWg0WmaJpmky6Nfa8bR4+DDnQwz5bQjboRAKoigKWrrcSplGtB550Xk4Ofck0u+kq+yY2vraGLNzDPr8r4/KjqmO7v1+D08OPWE7jNf0+bIPPkj9AJbeig8CCSoFENWIoG/ObiFnUtuqeRU5FUi/nS53+jdViDsVhxNzTjSYQkcRhyccxnrf9Uq7GaFrrAvnns6sv+8Jbsm4l4GKHGZvJjbF2N4YLr1dWEutTHADTdMQ1chex6VKWIXZJ2bjYuJFSGgJ2lu3x0c9PoK9sT2yK7JxMPqgwrFE7o3EH95/IO9JnsJtEISiakpqsL3Xdpx77xxomsb5Zedx7btrajcwQTBPS08Lvb/sjcQLiYx+Zta9twKmB2DmhZnSxyQ0doXswpX/XWHsOAQz+n/fH3NvzIWZqxkAaTmAft/0A19HmqbUY5AHxu8fX/9vAIg5GoMdfXegprQG2Q+ykR2eDbFAzEb4MtHS04J9kD2M7TS75iITastqcfqd0zg69SgkYglKkkpQmFBY30dc+/YafrH8BY93PGY30CbUltWqrKxAndCvQ3FizglIROq10leZ9Ez18M6jdzDs92FshwKADPpxFrlpqb4SziYg/kw85y4qCmILcGj8IcSfjmc7FELJcqNy8XjHY1Rkq+6GIwD4T/WHY3dHlR5TndASGvd+v4eLyy9yblagsZ1xiy8I/Kf64/2k92EToJw0irIqTS/F/Q33URBbwGocXBZ3Kg7be27nZErK3KhcROyMYCxFq4mTCSy9LJX6vUosFKM0rVRp7RPqRSKSYHvP7Ti94LRqjyuWoCS1RKXHJLilIqcCP+r/iEufXpJp+6MxR7ErYhfeOf1O/WM6fB0s6boEAPDbnd8Uvp7ha/NB8SjUlNQotD9BtATFo2BgZQATBxMIq4RIvZaK5MvJkAjJjcnWipbQKE4qBgD4jPLBu0/ehY6h9IZ+0bOiFt27yYvOw46+O+prwNWpyKlAdVG1dOUHwSkURcmdAaskpQS5UbkoSijCjPMzMPXUVM7VRn8RTdOoLa/l5CRPrtE10cWc0DkYs3MMeHwexu4ei3k354GvLR30dQx2RMhPIZzM7FInan8UfrX6FYkXElV2zJ6f9MSUk1M4d++bC7gypkPR5NVhRVlZGUxNTVFcXAwzM7PXni9MKETCmQS0Hd8Wpk6mqg+QUNiWrltQmlaKj3I+YjuUl6TfScf2HtsxePVgBC8NZjscQsmqi6uhpaul8npdtISGWCBm/AuwRCJBRkYGHB0dweOp73yV6uJqFD0r4mSaXYlYgqi9URBUCNDl3S5sh6OwpEtJ2D1oN4b9MQxdF3dlOxxOyovOQ/zpeATODYSRjRHb4bxEWC0ELaahbaANiseNL8vN2f/GfiSeT8TnVZ9zqraBpvSb6kZYLcSDTQ9g5mKGtuPaquy4B8cdRPzpeHxe+Xn9TQqidSlNK8XFjy/Ce5Q3AmYENLt93x19cT31Or7v/z2+7PNl/eNF1UVwWuOEKmEVHr3zCIG2gUqMmjtIn6lZxEIxeFo8UBQlTfUpoaFnpsd2WARLbq+6jatfXcX0s9Ph2s+1/vE7a+7gwrILmHd7HpyCFavNFrYxDOeXnseca3NemwArEUsgrhVD20AbEpEE2Q+z4dCVe9eBilK3frO6qBoXPr6AHh/1QJu2beTaV1QrgrBSCH0L9cjuUVNag5VmKxE4LxCjt41mOxzOomla7gEaiVjCqWs+AEi+kozI3ZEI+THktXS0hOrc+OkGLDwt4DfJr8HnJRIJnj59ivbt26O0tBQmJiZKjYe70xJaicY+GLPCsnB+6XkY2xuTQT81M3z9cMbzwzPBsbsjvpJ8xZkZB4RysZFqLvthNv7u/Tf6fdcPPT7swWjbPB4Pzs7OjLbJBn1zfU4O+AHS2Ug3froBYaUQnd/pLNcX2eKkYoT/FQ7/6f6w8Wd3pZ9DVwfMOD8Ddp3sWI2Dy6zbW8O6vTXbYTRIW1+1ExWY4DPaBxZeFhDXisEz4M4FoKb0m+pGW1+blclVPm/4wMzNDMIqIfimZNCvNTJ1NsWEgxNk2ja+MB7XU6+DR/EwJ3DOS89Z6Ftg2xvbEGgbCF8rXyVEyk2kz1R/hQmFyI3MhfcI75cmIOqa6AKQ3twVVAiga6zLVogES5x6OMGltwvsg+xfetxjkAf8p/nXv0cU0eXdLvB5wwcmjq/fvOXxefXfDY9OO4q4U3FY9HQRzN3MFT4el6hbvxn3Txweb38MtxA3uQf9tHS11KqEj7a+NgLnBsK5l/q8PqqWF52HYzOOYfT20TLdOxBWCfHPW/9Ax0QHI/8cqYIIZecW4qbSen51aJqGsFJISgxAWjM29JtQeAz2aHTQj8fjwclJsQkmilCfHktDSSQNp5hwH+iOWVdmwa4juWmpbrg6c4sM9rUeKddSYOJgAgtPC5Ue18zVDE49nJQys0jdZhG+Kv5MPEqSS9BxfkfODmpQPAoTDkyAmauZ3DPXUq6l4NbKW7DvYs/6oJ+uiS48BnuwGgOX1SV44OpnQk1JDYqeFcHMzQwGlgYtaqs4qRhhf4ah/eT2r93kYVKn+Z2U1nZLqHu/ScgncE4g2yEQamT7o+0AgKGeQ+Fo8npq9intp7T4GLEnY1HwtAC9Pu3V4rZUgfSZ6i/8r3DcWXUHc2/MbfBG95/t/4SeuR7m3ZzHQnQEm5x6OGHG+RmvPW7d3hrj9o5rcfsNDfi9KmhBEKx8rWBobdji43GFuvWbgbMDYe1nDZsO7F6vqgJfh4/R28kKv6ZkP8xGUUIRJGLZUj9r6WmhPKscuhW6kIgk4Glx/z2vTLSExhqnNbDpYIPpZ6ezHQ7rtPS08H7S+xBUNJ5SVyKRID09XWUxte53KAc0ll3V0NoQbv3dSAoKNSOoFHC2iKmwSojkK8mv5ZonNAtN09g/cr/KawkBgL6FPmZenAn/qf6Mt03TNIqKWlZvgU331t7DpU8uQVglZDuUJtkG2ir0udNhVgcsjFoI94HuSohKfjRNoyyzjO0wOKkkpQS/tvkV9/64x3YoDUq6nIQtXbYg6WJSi9vKicjBnVV3Wm19R3XvN9XV9R+uY2u3rZzM+kBothsrbuD8svPN1g0WSUTYGbETADC/43ylxROxMwJXvrgCUY1IacdgEukz1V/w0mAM3zgcTj0bnkXv2t8V9l2UNwmI4BaxUIxTb51CQZxs3wOrCqvkal9YJcTOkJ2IPhgt0/ZuIW7o/13/+lqCmkAd+037IHuSBp0AIL2H8UHaBzJnYqJ4FKacmoLJJyZzasCvIrcCf/f+G5F7I1V6XIpHwWOoB+w6k8VKdUwcTWDla9Xo8zRNo7i4WGXxcOddSryGltAozypnOwxCDndW38FPRj8hLzqP7VBeU55Vjl0DduHhtodsh0IokUQkwdDfh5JaZhwz5eQUTP93eotXLqmCWCDG/fX38ejvRzLvw+PzYN3eGnqm3JiocnbxWaxxXIPKvEq2Q+Gc2rJaWHpbcrYehU2ADQb8PAA2AS2fges9whtLEpbAe6Q3A5E1rqqgCofGH8L99feVehxCPYhqRCjLLGMlzfbxWcdxZtEZlR+X4IbY47GIPR7bbD3UswlnkVORA2tDa4z0bjw91f3M+xh9YDTePPWmQvGE/BiCRU8Xga9Dbq4SqmFsb4wuC7s0ms1g+PrhGLpmqIqjItiSdjMNj7c/xqNtzV/T7B2+F9uCtzU7aeJFeU/ykBWWhYps+Sb5iGpECNsYxtnJ4poqYlcEnh57ynYYKvXP2//g5s832Q6Dc8qzyusnY8t7f0bXWLf+MybmSAzEAjHj8cmrLL0MeU/yUF1UrfJjj942GiHfh6j8uFyUFZ6Fynxu3X8i6T057MDoA0i6lITPKz9v9uKN4AYLDwu4hbjBzM2M7VBeY2htiOEbh5M6VxqOr81Hx7kdWTt+SmgKHmx6gP7f94ellyVrcXCNtoE2XHq7sB2GTGgJjZs/34SBlQEC5wQ2mwaytrwWuRG5sOtkB20DbqQude3rCoqi5Lpwby1sO9hi/m3lrexoKUsvS/T6hJlUcHwdvkrSHOsY6SDuVBwMbTUnXROhuJAfQhDyAzsXv3nReaRWVSv25t03ZbrZkFGWgc52nTHcazh0+I2vOBFJRDgVdwo2hjagaVrutNDy1ksiiJbIuJsBmwAbznwXJdjn1t8N827Pk+n+h0M3B5g4mkBQKXvNR4cuDvg4/2O5B+/u/XEPl5ZfAl+Xz9kU8ZroyhdXoGemh7bj2rIdisrEHImBXSc7tUmzrQq0hMaRKUdQkVOBdx69o/DK2yeHn+DIpCPo/WVv1ge97IPssbxwOWgxuffBpgOjD0DfQh8LIxeyHUo9ilanddgapKysDKampiguLoaZmVmD2zzY/AD5MfkY8NMAjUoBQBCE5oraH4Vj045h8vHJ8B3jy1i7EokEeXl5sLa2Vot6AXVomsbDLQ/hGOzIeq07eeQ8zoGlt6VMN06ufXcNoV+HYtTWUeTCleCUwvhC6JrqwsjGSOnHEgvFnEsVpK79JqE4Ul+EYFK1sBomP5tAJBEh5f0UuJjJP3mppqQGteW1MHUyVUKEzCJ9pvoSVgmxynYV7DraYc61OY1uJxFLcHbRWRhaG6L/d/1VFyChUrSEBiju1q8WVgnxcNtDBL0TpPYrodWl36RpGoVxhagqrIJzz9frfWoqUY0IPG0eeHzuvjaqJhFLcG/dPZRnl2Pwr4MVb0ckQeg3oQheFszZDDqqUJZZhitfXIHHYA/4T2O+zI+6kIglCN8cDi19rSYXYUgkEiQmJsLb2xulpaUwMWm+HmxLkEE/ltQN+qniRSYIovU4v+w8np17hrk357KSSlJYJQQtoaFjRCYqANL6aevc1qHrkq4Y9vswtsNRisr8SoRtDEOfL/qQm81q4OInF2ETYIOA6QFsh9KgkpQSHJlyBB1md0CXhV1a1NamDpsgqBDgvcT3GIqOIJpWU1qDB5sewH2gO+w7k7pRhOoUJhSiPKscDl0cGF3pFPRXEMKzw3FwwkFM8psk175ioRg/GfwEr+FemHJyCmMxEcSrBJUCPP77MYxsjdBuQrsmt/3D+w8YWhti3s15KoqOULXoA9G4/sN1jNk5Ru7P4sr8Slz79hr6fNkHRraNTxrLvJ+JlGspCJwdCENrkumBIAj2ROyOgKmTKVz7uar82FWFVfjV6ld0fa8rhq3TzPtdTFPleBC5O8cysZj9/L8EMwpiC3Bo/CEkX01mO5QG1ZbXYkPbDbj8+WW2QyGUSMdIB3wdPiu1hABpGktlDPiJxWIkJiaqXZ9p0MYA085MQ8f57KVcVZSwSohr31/D333+hqhG1Oh2hm0M0e/rfpwb8Lu58ia299yuVoXllU1QKcDtX24j/p94tkNpFMWjUJ5VDkG5oMVtdZzfEV0Wt2zgUFalaaV4vPMxK7UUGqOu/aY6K4gtwOVPLyPpYhIrx68tr0XUviik30ln5fgEeyJ3R2Jnv50ojC9scjuRRASxRPY+oZtDNwDAvYx7csfE1+Yj+MNgeL+h3LqqTCF9pvrSMdRB18Vdmx3wA4C37r+FuTfmqiAqgi01pTWoKqhSaIVx5O5IhG0IQ+qN1Ca3i9gdgUvLL6GqoErRMFGRU4HcyFyF9+cCdek3i5OLIahs+bWFusmJyEH2w2y2w+CM2JOxqC2vZbRNUY0IN1feRNifYYy2KytaQuPsu2dx/fvrrBzfwNIAywuXt/oBP2G1UKbtxGIxkpJUd53IrTt0xEtEtSIcnXoUod+Gsh0KIYOC2ALEnoht0Rc/ZeJp8UDxKVB8bqa5IJjR/7v+WBCxgNU6oEXPihBzNIbxdsvLyxlvU9l0DHXgNdwLth1s2Q5FbhSPQv6TfIhrxdDSk5YALk4uRv7TfABAbmQuHm59CImYG0XoxWJA8MK1XEV2BUrTS1GZx61iymzSMdTBR3kfYeDKgWyH0ihTZ1MsTVuKnst7tritbu91Q/DSYAaial78mXicnHMSGXczVHI8Waljv6nObPxtMP/OfLSf2p6V49eW1eLY9GOI2BnByvEJ9rQd1xaDVw+Gla9Vk9uFpoTC4CcDjNo/SqZ2uzn+N+iXKf+gHwAM/HmgWqX+bk19ZnFSMdshMEJQIUBtmew3cfXM9Dib9pFgRtA7QViWuUyhFXjdl3bHrMuzmh1AHvzrYMy4MANWbZvucxsjForxh/cfOLvorEL7c4k69JvHZx7HWpe1rW4y6LFpx3Bs+jG2w+CE/Kf5ODTuEE7MPsFou3wdPsLWh+HhXw8ZbVce085OQ79v+8m1T00NcPMmcOMG0NI/i9ac3hSQDvyutl+Nf5f+K9P2FRUVSo7oOS2VHYmQm5auFrLCs1BTUsN2KIQMfMf44rOKzzh7EaGtr413o99lOwyiFbj+w3VE7IzAxwUfs5JilEtqy2qhY6zD2X6hKVp6WphwYIK0LsZ/ziw8g5SrKXgv8T3c+PEGnhx+AoduDpyoV7h8OVBQAGzZAujoAIN+HYSha4eyHRbnGLYhKYiUwWu4FyYengiHrg5sh0KwSNtAG47dHVk7vrG9MSYcnAC7znasxUCwwzbQFraBzU8wiiuIg0AsAI+Sbe5v3Uq/8OxwCMVCaPOZSx1KsCf1eip2D9qNkJ9C0OPDHmyH0yJhG8MQ+nUoZl6aKVOtLrFQjKSLSeDr8uE+wF0FERJsULSGGUVRcAtxAyBNOX/liysI+TEEZq5mL22npacFj0EeCsfH1+aj3zf9YGxvrHAbhOzaTWyH2tJatbwmb4men/RsdQOdjbH0tsSITSNgH8Rs+n2KR2HG+Rkw9zBntF15ju/SW7aayzQN3LkDhIYC4eHPJ0yHhQGLF0vvoSiiMr8SaTfT4NjdEcZ2ra9PqyqognNvZ5i7s/MeaAoZ9OO4dx6+Q2pjqRFtfXIhTLCn6FkRHm1/BL/JfqyuLOs4vyM8hnhAS5d8xGztvhW6xrp4896bbIeisBdXjfb7th+S+yTDxNEEo7aMQtsJbTkx4FdZCcT/l7EyJwdwdpZeTBPP0RIaj7Y/QtvxbVlL/yuriN0RMLYzhvtAxW/GPTv/DNe/v44BKwbIfCHUEmYuZjBzMVP6cQhuK8ssg6G1IWv9D0VR8Jvkx8qxCfbU3dCT5WZmfKH0w9LH0kemtr0svWBnZAc3czfkV+XD3li+m2Xxp+Nx57c7GLpuKGwC2P++QEhZtbWCSx8XeAzyQG15LcQCsdpO1LNqawXX/q5w6CL7pJtD4w/BubczGfTTQEcmH4F9V3tGBrOLnhUhal8U7DrbIXiZNHMELaER+k0oOr/dGSaOLavFVNcmoXzd3+/Odgis6DCrA9shcAaPz0Pntzorpe027doopV1Z1JbVQttQW6aJDlFRwIoVz/9tYwPk5wNXrwJZWcDnnwMWFvLHkHwlGUenHMXYPWMRMD1A/gbUnImjCaaemsp2GA0i6T1Z1tzFGRnwU72qgipkPciSe78nh56gILZACRExJ+zPMMSdimM7DEJJMu9n4uaKmyiMa7qei7K59HaB/1R/RvsviqLg5OSkVrPzaJqGa39XuA1wYzsUxjh2c0Tvz3sDAHRNdOE3UfGby+npwLNnzMSV/UKpgpKS578nXU7CnTV3mDmImku+mox/3voHt1beYjuUZv3z1j+4t06xVHJ1qgqqpLWt5JzcWiOqwfXU68guV6z+RU0pd7IzqGO/qe52D9qNTQGb5NpHJBHhavJVVAqYS0VcW17bZC1WQrPkRuRipflKhG8Jb3bb+CLpoJ+3pWx19ngUDxnLMnBr3i25B/wAaZ+YE5GDilzVpTJSVGvoMyUiaUp2wzaGmHlxJnRNdbHWZS2ufnWV5cgU5zPKB9PPTgdfR7bJFnxtPsbuHotBvw5ScmSEqlUXVSPlWgryn+Qz0p77QHe88+idlwbnUq+n4vr313H9B+bqZwkq1LfWXGvoNwn1JqwS4tKnl1BVqNwyTMVJxbi96rbKS5+cefcMfjb5GcKq5mvK3bwp/RkYCKxbJ82Q9P33gJEREBcHLFv28n0VWbn0dsEb299QyURbdUdRFBwdVZcVhgz6sYzHa/4liD0Zi7CN7BQFbY3u/XEPW7psQfSBaJn3qSmpwZHJR3DjxxtKjKzlLiy7gPDNzd8QINRTuwnt8O6Td+ExWPFUI0yqu7HABB6PB0tLS5n6TK6gKAojNozAgJ8GsB0K5wiFwCefAB9/LJ1d1lIvfjktK3v+e/imcFz86CLjBbvVkVOwE8bsHIPObytnhiOTJh+bjL7f9G1RGwHTA/Bx3sdw6dP8xUetqBYb7m/A8L3DYbHSAn139EWXLV1QWCXfBIpz75/DSrOVqC6uVjRsRqljv6nOaJpG2/Ft4T/DX+Z9akW1mHBoAkJ2haDvjr4or215XZyofVH42eRnJF5MbHFbhHoQC8Sw7WALI1ujZreNK5BO/pN1pR8AmVOBNsR/mj8+KfqkRWnwVEXT+0xhtRBbu23Fs3+fz7gydTaF5xBPlaamLk0vxa1fbslVh68hErEEZRllzW/YgHYT2qllvW2iafoW+liWsQyDVw1mrM26tMk0TUsndPZzxdTTUzFkzRBG2j+98DRWO66GqFY9J+qoQ795euFp7BmyB2KhmO1QVO7MojPY1mMb22GwKnJPJG6tvCXX/V1FPPr7ES5+fBGp11OVepxX2QfZw3esL7QNms46R9PA3bvS38eMAdzdAYoCAgKA1asBR0egsBA4pkAJSGN7Y3Sc2xGmzqby76zmUkJTsGvgLmSGZcq0PY/Hg4UiyykVxN2euZUQi5v/4Ln9621c+uRSq/yQYkPPj3vCY7AHfN6QXgxLRJKXCp1X5lVCWP3yLAq+Dh8TD09E53e4fTN15sWZGPjLQLbDIJSEr8NHm3ZtoGemx3Yo2N5rO7b33M5Ye2KxGLGxsTL1mQT3xcYC5eWASATcvt3y9l4c9Cstff57n//1wdsP3272S3BroG2gjQ6zOnAy1/yrvIZ7wb4zs/UWmjLv1DwsPrcY556dQ7WoGjyKh8zyTLx9+m256mA4dHFAwIwACCubn2mpCqTfVC2KohDyfQj6fNFHpu1rRDUYd2gcTsadBCCtmfb55c9bHIe1vzXaT23P+TS+BHMcujpgzrU58BnV9EBeragWKSUpAGRf6fciRVajqtPqD03vMwvjClGaXoqcxzn1j1EUhfH7xyNwdqDK4qAlNC59cgmXP7/conZij8dinds6xByNUWh/sUCM6iJuTNJpjYTVQlz/8bo0KwODeFo86Fsw+/lXmlaKtS5rEb1fOmjgPcKbsbIu1n7W8BjsgZpi7mSKkIc69Ju1pbWozKtslaUfRDUizlyXsKXTW50w+fhkBL0TpNzjzO+EaWenwblX87VlmdT9g+4Yt2dcs9vFxgLFxYCBAdDhlayvdnbA229Lf797F5AoOHdfXScvtERBbAHSbqbJ/JkgFosRX1eXRgXIoJ8aGLpuKN55/E6r/JBSpfLsctA0DR0jHcw4P6P+JvHF5Rexe9Bu0DQNQaUAe4buwe6BuyEWSL/YVORWQNtAG+0mtFN5By8v517OsPazZjsMQklyo3I5k1qujV8bxnOb19Rw49xkde+Pezj15qkWz2TWRI8fP//9DgPZNxtL72kTYAPbDrYy5bjXZJn3M1GRw/3UakyK2B0h80qnxV0Ww8fSB//r8z9ELojEvTfvQZunjWNPj+FE7AmZjxkwIwBjd49tcY0XJqlbv9laVAmrMGr/KJxNOAt9LX2sGbIGI71H4oeQH1rcto2/DcbvG8/576SE6j0regYaNEx0TWBtKPv1QLWwGv5/+sP0Z1OU1JTIfdyEswkvrS7jMk3uM20DbbEoZhF6fNRwrbPK/EqlpRmsLqpG+u10ANLVhb2/7I3+3/dvUZtGdkZwC3FTKJ1YZV4lVpisaPHAI6G45CvJuPrlVdz+jYHZf5B+1738+WWUZSq2+rMpOsY6EAvESlnB03VxV0w8NFGmldpcxfV+c/y+8Xj74dtsh8GK0dtGY0HEArbDYBVFUfAd4wuelnLvB5i5msFrmBdn79vX3XMJCgK0tF5/3t8fMDSU3kuJjZW//Sv/u4KfDH9CZR5z5QrUQdCCICwvWI42frLf+1Rln9m674KpCfvO9rDwUN3yz9aotrwW23tsx5FJR16a1S+sFqIsowwBswIgFoihra8N94Hu8BjqAb4OH9e+u4YNvhtQmMBuDTVZ0TTNaMpFgjskYgm2dt2Ko1OOsh0KAGDU5lEY/fdotsNgVdqNNETti4K2IVll9qoXB/1iYqSzzlqisfSegHSGI9frrSoTTdM4PvM4NnfcrPIaA4ra3nM71vuuV3h/mqZxbvE53FxxU6btg52C8eTdJ/iu/3fwt/FHkH0QVgxYgW/7fYtRPqMUjoPgPnlWcjbn6ldXcXDswdeyQTRk8dnFuJR0CYbahjg3/Rw+6P4B/pn6D0z1Wl9aHKLl/l36LyJ2RTS7HY/iYZLfJIzyHiXXCjx9bX1UCasgpsV4kPVA7vj+eesfhH4dKvd+BDNEtSKUZ0lTBxtYGTR44zPjXgbWuqzF/Q33GT8+TdPYNWAX9o3ch6qCqvpV0XWrkWmJYv2wc09nzDg/A4bWhnLva9DGAG3HtYVdJzuFjk20nPcIb8y5PgdD1wwFIH2ftmSVyNNjT3FzxU1U5TNft0vfXB/vPnkXQ9cOZbxtQjXUadU5wYzEi4k4NuOYyie+FsQVqOzeQ+LFRByZfAS5UblNbvdias/g4Ia30dICunaV/q5IJiZLb0t4j/RW6xqlitIx0uFsH0MG/dSEoFKAtJtpbIehuWjApY8LXPq5vPTHqq2vjYmHJqLf1/2gpasFikdh0C+D0OdLaeomh24OsPa3hq6xLluRy2Wty9pWn9NbU4kFYvT+srdctYQI5Zp4aCKWZS5r9avMXlVRASQkSH+3tX35S6iiGlvpBwC7B+3G9p7bFb6ppPZoIOSnEPT/vr/avBftu9jDqadTi9qYenoqQn4IafR5mqaRWvJ8xjaf9/KszA97fIiv+n4FLV4DUyGbcGPFDRyfeVy+YAlWFMQV4K9OfyE3UnqhLBaKEXMkRuG+ouhZETLvZ0JLT7b3jKOJI3aM2YG+ri/Xr6RpGtsebsP5Z+cVigMA7m+4j/2j9jM6qNma3VhxA+c/fP561JbVcqZWrKBCgHtr78m0kq5tm7Y4OOEg9ozbI/dxujl0AwDcy7gn974jNo3A4N+Yq7FFyOf6D9exod2Gl9J6vsquox3c+rvBxt+GsePW3fSkKAq9v+yNYX8Mg77ly2kXH+94jC1dtkBQKd9Nwsr8lq0koCgK4/eNV4s6x5rMpbdLfXal88vO4/CEwwp/boX8GIJ5t+fBpgNz7+EXGVgayPz5Lq/Ei4nY3ms746lOCeDJoSe498c9mSZkaaLUG6l4sPlBqywVFX86Hk+PPa3P0KYKZRll2NB2A07NP6WS7+B5UXl4cvgJaHHTx0pNld4v0daWrvRrTI//EgHcuSO9RyOPDjM7YMqJKWpRSoQpD7c9xP0N91X6HpOXetz90WCyFrw9OeckdvTdgepikndeGXRNdDFm5xh0ebeLTNvXDQx6DvHEnGtz1CYdg89oH7gPdGc7DE6L+ycOhycefik1SG5Ubv2HNk3TCN8Sjsz7shVqVRVtfW30+aIPAqYHsB0KAKAipwLnl53H0+NPGWmPx+PB3d2d00XCG0JqKr0uMlL6JdLJCRgyRPpYS+r61dQARUXP//3qSr8Oszug+9LunP4ypkwUj0K78e3Q6c1ObIcis6Frh2L0NsVXClMUBZfeLnDq0fjA4c6InfBZ74M/7v0hU5sSWrZVktnh2Yg/E8+Jmgbq2m82paakBqVppc1vKIPK3EoUxhfWD/qlhKbg8MTDuPXLLYXaG79vPD5I/UCmmZ7bR29H+tJ0TGg34bXnNj3YhDf/eRNv/fMWymoVS1OWH5OP5CvJSlnxoOkElQKcnHsS4X+F1z+WdCEJETuer6S7/dttrHVei7zoPIWPQ9M0Ei8ktvimkI6RDpYXLcegXwa1qJ3m1A/6Zco/6Oczykct0s1qYp8JSGs+Ovd0bjL1FF+Hj2lnpsFruBcjxyzLKMOWrltQnCxN5dBufDsETA94rX+sKqhCZX4lSlNl79fLMsuw2n41rn59lZFYCdXLepCF++vvo6pA+hlF0zRqimsgFooVXiXC4/PgFOzE2dUWTakprkHO4xwUJRY1vzHHcL3fDP8rHJc/u8zZlIvKFrk7EmcWnGmVq6+GrRuGhVELYeqsuiwaJo4mGPb7MAzfMFwlfVHwsmB8Vv4ZrP2bTtleN8G6Y0dAT6/x7Tp2BHR1gbw8IFG2KhmtWtTeKFz+7LJcqWN5PB5cXV2VF9Srx1PZkYgGydoRBM4LxODfBqvllxguo2kacafi6md1K/L/V51ek+F/DMeAnwawHQbnvDi7tCS5BDFHY1CSUgJAOkt1U8AmXPrkEgDp633jB+lKDpIqtXEUj8LdNXeReJ6ZbwsURcHExERt/t6Kk4sR908cmajRgLrUnoGBQM+e0t8jI4HycsXay3ll4nrpK/eNOr3ZCX2+7KO02blc1xrfg6JaUbOpTL+79h1qxbWoEDSd8mXzg83wWe+DH6//KNOxR28fjeUFy6Gly/77Td36TVmcfuc01rqsRX5MfovbcunjgveT30fADOlkGev21uj9RW90nNdR4TaZqBcyO3A23M3dkV6Wjk8vfapQG4N/HYzPyj9TKO1da1ecVIykS0kvTe6aeGQiPsz5sP7fVr5WcAx2hFVbK4WP83DLQ+wZsgcPNknTZbZkooC+ub5MtURzK3JlnsDwqs720hVREbnNpxFtCE3TnF9xr4l9JiAddJ12ZprMN72Lk4tbfIO4NK0UBlYGyLiT0eR2wcuCsTByoVx1wIVVQviM9oFDF4cWxViWWYYDYw4g7M+wFrVDyC9idwTOLTmHilzpdzCKojB6+2hMPzddoQxKz/59huyH2Wq7ut13jC8+Lf0UXsOYGXRXJa73m5OPT8bUf6YqvZ4bV3V7vxtmXpwJHUMdtkNhBRtlsrou7grbQFsAUMkKUx1DnWaz+dTV82sstWcdXd3nKwHr9pHHndV3cHbxWfl3VFMTDkzA3BtzQfFk7//q+kxVaZ09H4eIxbKtPPAa5oXuH3SHnlkTw/KE3J4ceoIDow/g1q+Kzeom1F/knkisdV5bn4Km05ud8EX1F3DuKZ2RrKWvhe5Lu8N7lHf9Pv2+64eZF2dy6svj5S8uY9eAXZyZxWXQxgDvJb6H4RuGM9KeWCxGVFSUzH0m2+JOxuHAGweQ86jxVEqt1YuDfnZ2gKsrIJEA9/8rI0PTQHIyUC3jWFVdas+6WWuvpvdszcqzyvGLxS+49NkltkORS/SBaPz7wb8KT6x48OcD/Kj3IzLuNnyzsbSmFMklyQCAhV0WNtmWUCJEfGE8bqXL9j1B10RXri/+yqRu/WZDCuMLcXH5xfr3QucFndHzk571N4hLUkvkSrNYEFuA3YN3oyBO+pn/4qCYsZ0xQn4IUWigLDMsE493PEZ1UfMdV7WwuskbkwbaBtg6aisA4M8Hf+J66nW549E20ObM+1Dd2PjbYFHsIozcPLL+MQNLg5cGTPyn+mP62en1N1mKk+QvTOs3yQ9BC4MQMD0A8Wfisd57vUKp3XKjclEYXyjTze72f7aH4U+GiMmPkfs47a3bAwDSStNQUlMi176P/n6EH/V/ROqN1OY3ZpEm9JkvEgvFqMyTLw1myrUUrPdej7CNLRsIc+rhhPcS30P7Ke2b3I7iUfX3N3KjcnH+w/PNDg5belli0pFJ8B7p3eR2zdEz00Pi+UQUJajf6ip1F/JDCKb+M/WlwV4tPa36gaNn55/JnI6QpmmcXXQWB0YfANRzzA98Hb7apOB/Fdf7TV1jXbj1d2M7DNZY+1nDfaA7+DqtZ6VjWUYZDk86jKzwLFbjKIgtwEa/jfXp16uLq7HGeQ3OLnk+KFacVIyakhqF2hfViBC1L6p+RX1jcnOBpCSAop7X7GtKXYpPRTIxpV5LRcSuiFaTTtbQ2hC2HWzl2kcsFuPJkydKiuh16vnJ0ooJq4RqO4OJizwGe6DnJz3VKu1ZS9z8+aba3fxVNnN3c+hb6qOmVPphq22g/dIKDTMXMwxZPQQuvV3qHwucHVifJiD/aX6L60owoaqgCoXxhdA21GY7FADSGSzm7uaMXsBw9WKiId4jvTFqyyjYdbJjOxROycuTDtLxeID/f+Un61b73bolTdP5/ffAe+8BP8q2sKp+pZ/3f/d+KiqAV98qJ2afwN7he1t+AmpGVCOC/3R/2AfZsx2KXBIvJOLeunsQVik2O9LU2RRew70aTedSd8Pb3tgeZnpmTbbV00n6Br2TcUemFTI0TSPpchKenW++vpYqqFO/2ZBbv9zCndV3kPVAeuHu1t8NA38eWP/8mQVnsHfoXpln0iZeTETyleQmL7DFQjEe/f1IrsHE6APRODn3pEzfB+aenAubVTY4EH2g0W36u/XHW53eAgC8eepNVAvlW7FLS2gkX0lG6nVuD7JwCU3TKM+SLjmXZdZ0nSv/u4IN7TYgNypXpu3r3qt6ZnoYsXGEdKIARUEsECt04+f80vPY3HFzsze7i6qLUFBVgBpRDZxN5U+1aaZnBicTacrk6Lxo+fZ1MYNbiJtarLhX9z7zRXGn4rDacbVcqfadgp3QdlxbOHZ3bPHxKYqSa/LB7V9uI2x9GHIiGp8wx+SNRB1DHXxS8gmGrB7CWJuEbHSNdeE90rvB1WFPDj/B3qF7cWul7JOyx+wcg6G/D1XryS4FsQW4s/oOZ+rFyoOr/Wb86fj6z3Wi9Ui+koyYwzEoz2T3tdc10YVYIK6/f69npgcLD4uXFvKcWXgG69zWobZM/r/7/Kf5ODb9GCJ2Np2BoS61Z/v2gCwLzIKCAC0tICMDSE+XPiYUAgIZ1haM/GsklhcsbxXpdAWVAhQlFin0vUSVfSb3v3kT9R5ufYhz753D/Dvz5R5NJhqmb67/0s0jVcsqz0JBVQGsDKxgZWAFHb5yl93HnohFeWY5Bq5g75zZJqgUIPTrUATMDIBtB1s49XDCophFCq3aq8ipwPYe22HX2Q6zLs1SQrSyG7V5FGia5lRqjeqiauRF58G+iz209bkxGKkqFp4WsPBUfToJrnv0SPrTxwfQ/6/cYXAwsHev9Ll33wUq/7tnHhEhnZXm3kwZ0qz/JvF5ewNRUdKVguXlgJnZ821qy2tRW1rLub8RZTN3N8e4PePYDkNuA1cORP/v+0PHSLHPxLbj2qLtuLaNPv8kXzq7zq+NX7Nt+dv4w1DbEGW1ZXiS9wT+Nv5Nbk9RFI7PPA5jO2N4DvGUL3DiNSM2joD/NP8Gb0DTNI027duAFtMyDyZ0W9INPm/4wMzFrNFtovZF4dS8U6gurEaPj3rI1G7w0mA49XCSqd9/nPMY+VX5MNdrutD9r4N+xZmEM0goSsCpuFOY3H6yTLEAACjgwOgDsO9ij9lXZsu+XysWtTcK/7z9DyYfnyzX3673SG/kPs6FYZvmV4iKakTY1GET+n/fH+0nP18B5TXcC0ueLalPv5VyLQXW7a1hYGnQbJt9/tcHSZeSmr3ZHV8YDwBwMHaAkY5itcjHtR2HouoiGGrLtxrWLcQNbiGtd6UFWwysDODSxwVOwY3Xt30VX4ePCQdfrzUqj4x7GYjeH42ui7vK9V141JZRCFoYBLuOjU+YOzX/FIqTijHzwkxoG7T82oILqbhbm9zIXOiZ6TU6Mct3tC+6L+0u86RsiqLUomZoc+LPxOPiRxdh08EG7gOaufghmlVbXotDEw7BuaczZl1m9z4Nm67/cB2h34RiQcQCWPs1XfdNU3SY1QHOvZxVWsuvIcb2xnjz3pugxc9LSc2++vw7OU3T8J/hD7sgO+iayJ/W2MzFDOMPjG/2da1bsddcas86BgbSjEwPHgAbN0onUyckSB//7TfAtomhCCMbxb5fqqO0G2nYO2wvRmwagaB3gtgOp1HkW44asfSxhEMXB4iqFa/5QEhV5lXiyeEn6Px2Z9ZmIWSVZ8Hjdw/UiJ7P6vW08ETo7FA4mLSsRkFjZvw7A2g997sblBuRi3vr7qG2vBajNo8CoHgNHiNbI3R9rysjs2HrlGWW4cZPNzDktyFyz4jm2mDG/fX3Efp1KN4Of7tVrXijJTQkIkmrSqMhq7rUnh1fKJnl7Aw4OACZmYBIBHh5SQcEIyOB06elq/6aUpfe09ERMDKSDviVlr486Dfp6CTO/X0QjZPl5nlLPMmTfdBPi6eF7o7dcTn5Mm6n32520A8Ahm8YrvRz0GQSsQRpN9Pg2tcVfB1+o4MFFEVh0C+D6n+naRqiGlGDk0yEVULwdaXps5oa8AOkqRuri6rRab7sWSBMHE3QzrFds9tVCirrB18CbQOb3NZUzxTjfMdhfdh63Mu8J9egH0VRGLl5pEx13ggpYwdj2HW0k7tOmGM3R0z9ZyqA/+rWielGv1cWJxVDLBCjLKPstefqBvxqy2pxaNwhGNsbY0HkgmY/u1z7usK1r2uzccYVxAEAvC0VT4m4duhahfclVE/W90ZjSlJKYOxgLPe18rNzz3Bv3T10mN1Brv209LTg1EM6QFlTWoPsh9mvpeXT0teCtr42IwN+gHSC4tPjT2ETYNPiGoGEbM4vPY/0O+n4pOiTBq91+Tr8l1Zf3lhxA0UJRRi6duhrN8ZpmkZuZC5sAmzU/nt+uwntYNvBltH7Cq0ZX4ePsbvHQs+0dZdHsvS2hM8oH8b6THVh7t70xDpVMXFo/Hs4RVHoMPP552RVYRVyHufIPOivb6H/0gSyhpSUAE//W+zfvbtMzQKQpvh88ACIfiGxQ1kZsHo18PPP0qxNjckKz0JlXqVa1iiVh4mjCYI/DIZjN2732SS9J8t4Tf21vMKltwvmXJtDvggwIGpfFM4tPof40/GsxXA1+SpqRDXQ4mmBR0nfB3MD58LSwFJpx9Qz02u1X3wkYmlaNqceTph/Zz5GbBzBSLv9v+1f/4FWkVOB9NvpLWpPIpIgNTQVt3+TPYl29qNs3Fl9R+66HcrmOcwTg1YNgqFNy29+83g8+Pj4yNVnsqXoWRF+MvoJt1cpkAhdg9G0dPUeIJ09VoeigClTACsrYNYs4NdfgenTpc9duyYdxGtK3aCfnd3zgb7S0pe3UfcbAYoQC8XYGbIT4VvC2Q5FbrVltShMKISoRrFJTidmn2jyvOtX+lk3P+gHAD2cpKu9bmfI9jfddmxbTsw6V6d+80WhX4diZ7+diD0Z2+y2FEXVD/hd+PAC/u71d4OfheeXnce24G2oKqxqtk2+Dh/BS4NlnnUrrBLKVMsPAKLyokCDhq2RLWyMbJrdfpjXMLzd6W0McBsgU/sv8p/mD5c+Ls1vSACQpo6dd2se9C30FW4jYmcEtgVvQ0lqSYPPt2nXBovjFqPr4saLqugY6WD4xuHo912/+s+uhko7VORWIOex7HWD6wabfSx9ZN6HSaHfhHL+e5G69pkNaWmN7/C/wrHObR1SrqbIvW/fr/tiQeSCFmUmOjL5CPYO24uiZy/X2xu1eRRmXJihcLuvqsyvxD9v/oPo/fKlrCUU12VxF4T8ECLT5FaappFxJwN5UXmoyK147fmCpwXYHLgZlz+7rIxQVcrMxQzuA93rJ4CoC672m1q6WvCb6AePwR5sh8Iqv0l+mHx8MszduDEIpmw3VtyQ6fqBi/YO3YtD4w7JdE0hqBSgNL202e3u3ZPeg/HyAtq0aXbzen37AoMHS//74ANgxQrppOynT4Fjx5re9+Sckzj9zmnZD6amrNtbY/CqwbANlO+7Do/Hg5eX6gZEudUzE4SK+E/zx+i/R7P6JeBuhjS58uIuiyH8nxAFHxfgs16fQU9LeYNyJaklyI2UreaIJonYLb0JU3cBbB9kr5Ri2cemH8POkJ0oTmq6mO6rciJycHbJWdASGsZ2xnDo5gDf0b4y7x+5OxIXPryAsszXZ46zyaGLA3p82KPJGU7y0NFRj4sgiUgCzyGeMHMzYzsUTklKkg7g6etLv3i+qF8/4O+/gYkTAT4faNsWcHOT5o6/1EQZUqEQyM+X/m5nB5j+l8Xj1UE/iViCB5seIHJPJGPnw3UlKSXIfpiN0rTmLwi45sHmB1jvvV7mGlkvqi2rRcTuCGTcyWh0m5HeIzGx3UQE2cuWiqOurt+tNNnry9ASWuY6c8qkLv3mi4IWBiFoYRA8h8qXHlXXRBda+lrQNX19JYCuqS70TPXkGtCpLq5GxK4I0JKmi6UlXkjEL5a/IHJv8/3L45zHAJpf5VdnuNdwbB61GSO8FZuoRNPSledEwyRiCc4sOoO86DxG2itNL0VFbgV0jV8fMBYLpPU7tHS1mkwpSPEotJ/cHm3HSlMUl6SUYHuP7a/FePvX29jccTPS78g22Sy+SDro15KVfgAgkogQkx8jd435yD2Rzdad4QJ17DNfJawSYq3rWvy79F+F23Dp64KAGQEwaNN8itlXURQFG3+bFtVXG/DTAIT8EAJzD+mN6vMfnsfNlTfr22eKpZclJh2bhJ7LezLWJtG0tmPbIniZbHnmKIrClJNTMP/OfFh6vT4xWttQG92XdYf3qJb1a1xB0zSKnhXJ3b+yjWv9prBa2OCKerZIJEBV83POiBaqLavF9e+u48HGB2yHopC+3/TFiE0jXqr515jYE7FY67wWT481XbP3zh3pT3lW+QGAjg6wZIn0vwEDpPUA33lH+tzevdL7Oo3p/0N/jNw8Uu36MVVSZZ9JBv1YJpHIdyFemVeJg2MP4tavst94Il5naG2IwDmBrM6kupspHfTr7tgdPIoHSwPL+osYkUQEkYT5NK5nF53Flq5bGG+X61KupkBUI0JFzuszBJk0+LfBGLBigNzpBG79fAvhm8OR/SgbfB0+Rm8fDev2sudcD/kxBFNPT5V7lok6kUgkiIqKkrvPZEObdm0w9Z+paDe++VRvrUliovSnr6+0OHRTKAoY8d897rNnpTPUGpKXJ31OT0+6yq+uOPVrK/14FEK/CcXdtXcVjl/dWHpZ4pOiT9D7s95shyI3x+6O6PVZLxhay79KWNdEF19Uf/FSaqhXvdftPRyaeEjmgZfujt3R3ro9QtxCZPpsLkktwQqTFQj9OlTGqJVDnfpNWkKjKFG6osPEwQQjNo6Qq9YSRVHo900/zL4yG1q6WqAlNM5/eB4VORXSNKArB2H6v9Plull88+ebODH7BNJupTW5naGNITrM6tBkHao69YN+NoEyx6Go5KvJWGm2slVNdpBXdng2Hm55iHu/32Okvb7/64tFMYvqB5eTryajurga1cXVWOe+TqHjZNzLQPbDbJSklLz0uP90f3Rf1l3mtEJ16T19rBRf6SeWiGH5iyX8NvohtTRVrn1nX52N+XfnK3xsVVCnPrMhdZMbqwqr4BTs1OAgiaysfKwwdvdYmfq1F6XfSUfy1eQWTzaw62SHHh/1qO+zI3dF4tm5Zy1qsyEUj0LbsW1hZNt66hCxqTChUO59KIqqT5mcFZ4FsVBc/5yZixmG/DYEzj3Zz67AhMufXcYfXn+gKKGo+Y05gov9ZsKZBKxxWoPoA9xYwbtnDzB5MvDtt8Az5ruxJmXczcC5984h/2m+ag/MAl0TXbyX+B6GrGn8GpDLvEd4w3+qv0wTZszdzOE32Q/OvRvv+6qqnmdZkrWeX1NCQqSDhyKRNM2noJGEAr6jfeE1zEvjMy0dnnQYZxadkXs/iUSCJ0+eKCGihpGafmpGz1wP6bfTYWAt/6w7QqosswzaBtrQN1c8fQ8TpvhNgYOxA4KdXu6Bb6bdxLtn3sX8jvPxfvf3GT1mwIwAOPWUvZi7phi9fbRKjmMbaFs/8CYRSRC5NxIdZnVo9gNvzK4x6PV5L9j4v5zqK+VaCnh8XrNp4rT1teE9gpszHA+OOwhBhQAzL8xkOxSCZRn/LbxylvG6vG9f6eq/nBwgPBwIamBRVl1qT1tb6UBhU+k9p56aChOn1lXfiuJRalnDwaW3C1x6K56WsLmVNIC0KLlYLJ3J2BxTPVNELYyS+fgmDiZw7uUMC08Lmfdp7a58eQVhG8Iw98Zc2AQ0n/ayMXW1VOPPxOPumrvQM9VD36/6AoDcK/w7ze+ENm3bNJuizinYCU7Bsn23knelHwDUimoRkRsBfS19mWpK1jFxNIFdJ7vXVj4S0pVQ2gbacOjqgLnX5zJad1jHSNqplGeVY/+o/bAJsMGov0ZB31xfodSh7Se3h3NP5/r6jMXJxaAlNOw62sk1IDO+7Xh4WnjKVMu0MXweH65mrojMjURUbhRczVxl3tfUyVTh4xJNoyU0Tr11Cpn3MrEwaiFMnUzr60wyQSwUy1zX7+aKm0g4m4DlhcsZLSnxQeoHLw32MK0yrxI8bR7r9wc0Wc7jHGzutBn9v++PPl/0kXv/+NPx2P/Gfgz8eSAcgx3x7NwzhPwQ0qIVpVzjPtAdwmoheNpkbUZLGDsYw29S0wMiqlS32urBA+l/3boBb78NWMs+x1thBbEFuP/HfXgM8UCbtnLkd1RTxvbGMLY3ZjuMFinLKEPq9VT4T2v8O79TD6f6GriNefBAOkDn4AA4MXALmKKAxYulKT5TU6UTs8eMaXhbmqYhqBA0mPlCUxTEFsC4jPvvNfJpomb42ny8n/w+Rm0exXYoauvad9fwq9WvrKdC/LDHhzgx5QScTV/+MhJbEIuovCh8FfoVcipkr9Uhi/ZT2qvlqg91dGf1HZyccxKxxxvOKV5VWIUnh6QzPPja/NcG/KoKq7Bv+D6cXXy2yaXxqddTUZwsXzpRVeLxeUpJpcplod+E4sZPN9gOg3PqBv0cZSxLq6cHDBok/f1MI5Oo6gb97O2lPxtb6QcADl0dYGzH/S9mTAnbGIbMsEy2w1C5woRCZN7PhKi24RV5aaVpSCpKxjffSjBpkrRGwaNHja8mVQRPi4cZ/85A57c7M9eohnPp6wLXfq6w8GJmoNSuox2mnJiC3p8r/p3H0tsSgXMCZa7tJ4sBbgPQ37U/Otl1knmfn278hG5bu2H13dVyHcvSyxKzr86uTxNJSF3+4jJ+9/wdgkrpFGXH7o71g8VMMrIzwtB1QxHyYwis21vjncfvNHkDpyl1A340TePknJM4t/hcfa1qWX3d72scm3wMLmYtq/MYYBMAAIjMlW8FaW15LbIfZqO2rLZFxycaQAFmrmYwsDKAqJq5TDE1JTXY3Gkzzi05J/M+/b/vjze2vcF4DXltA22l1aVPvpqMVTaryKpoJdM10YXfRD+FJ6q6D3RHh5kd4D3SG7dW3sL9P+5z+hpYEe4D3TFs3bBWU39NWZyCnTDh4ATGSoy0hEAAZP53Odajh3Tg5N494Oefmb32aEy7ie2wLHMZ3Ae6K/9gLCpJLUHU/qgW17PlghOzT+DE7BMozy5v8Pm6dPHNqRtsZmKVXx1TU2DKFOnvtxsp00zTNDa03YD9I/czd2AOWhi5ENPPTWc7jGa1rjuxGkIdZ+1zifsAd3R6uxMnvgQ0ZF7HeQiyD0JZbRl+uP4D2+GovatfX0XoN6EqzynddXFXdPugG7xHSi9sastrUZ71/IP73OJzODL5SKM35Q0sDTBm5xhMOjqp0ZWCNE3j5LyT+Lv335zNmT3x8ES1+DBk0qNtjxB7Qj0LSCuTvIN+ADB8uPTngwfAhQuvP1836Gf332KHxlb61SnPKlfLGnfyqsyvxNlFZxG2PoztUBSSGZaJPUP24Nm/8ufACdsYhq3dtjaaznnV7VXw+MMd+/M+g1gsvWD56ivpjNvLl5u+ABeKhYjJj5E7JqJ5nkM8MeXkFGjrM/Md18TRBD5v+NSnBGuJyvzK+tSjr0q7mYbNHTcj5VqKTG39OOBHXJl9BV6Wshdw7+rQFQBwL4OZFJStSWVeJUK/DcXF5RfrHzNxNIGVjxWq8pVbYIeiKHSa3wlu/d0ASCdBtXRFCkVR6PtNXwirhMh6kMVEmHILsJYO+kXlyb76GQCi90fjr85/yVyDkJAdRVHo+7++mH1lNqP3CfTM9KBtoF2/elUWth1sETg7kLEYVMG2gy06zO6ANu00fxUMm8zdzTHh4ASFy1Fo6WlhzM4xaNOuDSYenojZV2fDwoNkVCBeVjehhyvS06XXFsbGwKefAuvXSye2JiRIr2+VTcdQB8b2xnKlzFdHkbsjcWzasWZT8quDkJ9CMPn45EYnKx8afwh/9/67yclfAsHz91ePHszGV1cfMDYWKCl5/XmKouAW4ga7zsxl0eAqdUhhSgb9WMbjKfYSxByJwfll5xmOpnXwm+SHkX+OZDWG2+m3kVaa1uBADY/i4ecBPwMAdjzegdIa5m5Q31x5E9t7boewSshYm1wXuTsS8afjVd4haxtoY+iaofWzx2/8eAOrHVZDWC39fz949WAM/X0oHLo4NNpGuwntmr6YoYEhq4dg4MqBavGB0xI8Hg/+/v4K95mqtCRhCaacmMJ2GJwiFErTdALSFBOysrMDxo2T/v7HH8C//778fNZ/9zxt/7t/YPpf9rCGBv3Ks8qx2mE1rv94XfYA1JSeqR5mh85Gt/e7sR2KQgTlAqTfSVeoDmvbcW0R8lNIoxN7nuRLV1gb1fiiWzdg1CjA0FD6/ly7Fli+vOF6G5llmTD92RQdN3dEraj5lSqpN1JxdNpR5MewV0OD6/0mTdO4/MXl12qVcUl1UTXWOK7BpU8uNfh8SUoJipOLoWemnBUowPNBv9iCWLm/Ez499hQnZp9odOWrpqsurkZtaS0ebn0IWiL9zh30ThBmX50NM1czdoNTkFt/N8y5NkfmWn4AUFBVgKzyLEjoltdcqksxK+9KP6ceTuj/fX9Or2Dhep/5qrSbaTjz7pn6tJfKSHM498ZcDF41uNntaJpG3Kk4uVegcoG+hT7G7BgD9wGavRKGTRl3MxidoKqtrw37IHvG2uOSsI1h2Nh+o9qsWOJav7l70G7sHrybMxOiU1KkP93cpKv8nJ2fT2rdv1/5q/2EVUIUxheipqRGuQdiWae3OmHEnyPgFuLGdigt5tjNsX7hwKtomoZBGwMY2Rk1mU0rIgKoqQGsrABPT2bjq2uTpoGwRuYXj9g4AkNWq2dtRVnkRuUick8kqgrkn0DI4/Hg56d4qn25j6eyIxGMijsVh3u/31PohlhrVnfBz7YpR6bAZa0LbqbdbPD5ELcQ+LXxQ6WwEtsfbWfsuJV5lShOLlZqTQSuWRK/BFNOsj8AYxtoi8A5gRCUS7/AG9sZo9uS5m/I0zSN2JOxCN8S/tpzFI+Czxs+CJgeIFMMAgFw/DiQlydf7C1RkVuBW7/eQur11Ba3JWisWjDHaOlpqX0ueaZlZ0u/GBoaPl+NJ6s5c4DR/5Xl3LDh5YG/uoFEWdJ7GtkZodsH3eA1TPYVNuqKr8OHa19XRutUqZJbiBs+K/sMgXMC5d7XpbcLen/Wu9EVXlE5dYN+fpgxQ7rCb+dOYPZs6czb2Fhg2TJg9+6X97M3toeRjhEEYgEeZj8EAFRXS2slNKQytxLR+6OR90SFHW4DuNxvpt9Kx82fbuLGCu6mQ9a30Ef3Zd3hN6nhi7OAGQH4KPcjmeoQppSkKDSRq41hG7iZuYEGjQdZ8k0LzwzLRMSuCJQkl8h9XE1g5WOFvl/3xYfZH9YPiGhS/SdZbXqwCQ6rHfDWqbda3FZdes/4wnjUiGS/iWjd3hp9vuwDS2/LFsegTFzuM19ES2hc/PgiIndHojhJeSkOZZ1UmHAmAQdGH8D17zV/YhUhn6TLSdgWvA03f274vgfxMmGVEMIqoVplJuFKvymqFcHSyxIWnhacmRCdnCz96fJCZu1x46Q1xRMSgIcPlXv81BupWO+zHk8OP1HugVhmZGOEoAVBMtefVQclKSU4PPHwS9nCKIrC6O2jMfHQxCb3rUvt2b27dLCZad3+u415r5UmIYk9EYvjM48r3E+rss8kg34sk0gUmw3X//v++DD7QxjZGjEckWbbPXg39o/az+rMn8yyTKSXpYNH8Rqt6UJRFN7r9h4AYH3YeoglzAzSDfltCD7M+lBpNRG4iKfF40Qq1/ZT2mP036NhaG0o1360mMal5ZcQ+lXoa/m75f2QuXgR2L4d+Oabxm9UM622rBaXll9C3D9xLWpHIpEgLi5O4T5TVUpSS5D1IAuimta5qqIxL6b2lPeLJ0UB8+e/PPD3xx9AYSGQmyt9rG6lX1PpPSmKwtA1Q+E7xlfu+NVNaVopZya5cElhVSHyq6Vvmu7u7eDqKn1cVxeYMAHYtAno21c6QH3oEHD16vN9KYpCT+eeAIBb6beQlQXMnSt9b/77LyB+5WPaa4QXPi37FH4TVTeT71Vc7zedezlj5sWZGPxr86tI2DRwxcBGB/0AQEtXS6abSwtOL4DZSjPsjtjd7Lav6uYovbq+lynf1XWPj3rgk+JPYOVrJfcx1R0toUHTNPRM9TQ+rVVzUkpSAOC1OuKKsDOyg4W+BcS0GE/zn7a4PS7hep/5IopHYf6d+ViavhRWPsr9+3647SF2DdjV5Co+517O6P1Fb7WtZfvs32fY1mNbq6yFrGxWPlbo/E5n+E9VrKZpaxO8LBjvJ72vNulmudRvaulKU8AO3zCc7VDqpf4379nthQVopqaqW+1n4WmBXp/3gm0HxdLqqoOciByNrBecG5mLmCMxiD0Zi6rCKlz9+iqSrybLtG/4f+sFmKzn96K6Qb9Hj4DaBv7XS8QSnF5wGle/vvr6kxqgw8wOmHhkIix95J/IJpFIkJCQoISoGkYG/dSUmYsZDNvIN3jQ2tESGjpGOtAx0mF15k/dDRt/a38Y6jT+Gk73nw5zPXMkFSfhQmIDxayIZj07/wyZ99X74o2nxcO4vePwzqN36lOFAkBBbAHWuqzFte+uydxW3WdLejpw4gTDgTbC3M0cc2/ORe/PeqvmgCx7vOMxtnTZgvyn7KX04yJF6vm9qG7gb8IE6b8vXJCu0BKJAC0taZoJ4Hl6z/Ly1wdhWguJWIKNfhuxb+Q+tkNRmLBKiLhTcciNzJVrv+yH2fjN/jdE7Ipo8PlHmdJZrvq1rpjwxuuTpiwtgY8+AqZOlf57w4bnF+sA0N1BWsQgLCsMBw4AlZVAUZF0u8WLX67Noa2vDV1jXbnibw1omkb4lvD6mivuA92ha6Ie/5+qi6pf+veVL6/gxk83ZB5gf5zzGADgY+Uj9/y+OCUAAQAASURBVLG72v9X10/OQT8DSwOlph7lsvgz8VhtvxrPzstfG1TTJJdIbxK5mbc87RVFUVjSdQm+7fctLPTlq6d1YPQBHJ91vMUxEM+p4u+7JKUEPC1efbrB6z9cx9Wvrr6UNljPTA8hP4SobaYLiUiCwvhCVGSTLEpMM3E0wchNI9U2pbKqtcbV6EyQiCX1E/u5ssoPeL7Sr26iYZ261X5xccDjx8o7voWHBQb8OAAOXeWor6FGaJrG4QmHsanDJs6kdGWKzxs+WBCxAF0WdoGgQoAbP9xAxM6Gr3FfJBBIr08BwMNDObG5ugLW1tJjNfT+5fF5SL6cjJSrKcoJgGVmrmZoN74ddAxlr3nMFjLop8ZK00uReCGR7TDUBsWjMOXEFIzbN47VOO5m3AUAdHfs3uR2hjqGWDV4FU5NOYXBHszMgs+JyEHk3kjUlmveTJiGnFtyDkenHWU7jBazD7KvX9UrEUln0VE8Ch1md4BLH5emdn1J8gsTg/bvV02aT54WD849naFvoa/8g3GA5xBP9P+hP+fTV6laSwf9AOnA3+zZwC+/AN7e0i+ZAGBjA9SVkTA2fr6SsLz89TZyI3PxV9BfeLzjseKBcJywSojOCzrD5w35Bxa4orq4GgdGH8Cj7Y/k2k9UI4KJgwl0TRseRDp5WzroZyXxQ9eujbczZQoQGCidubhihTSNJwB0tpeuYLiX9gChodLHxo2Tvu8yMoDvvgOevrDwpTChsMWrnDVN3Mk4nH77NEK/CWU7FLlc//E6fm3za/3rKRFLELnnv5rBMtygy63IRW5lLngUD+2t28t9/PqVfhn35L6pURBbgKwHWXIfU93RYhqG1oYwcWQ/2wPb6lb6uZkxU+vmm37f4Ku+X8HFTPbvoIC0b9f0ukKqcnvVbSRdSlLJsbou7oqOb3aszxRTklKC8L/CIagQQCwQI/yvcLWvG+o13Asf53+s1t+duEhUI9K4G/GqUJpWijOLziDpsmr+xjVB2IYw/NXpLxTGF7IdSr2SEmn2mbpafi8yNweGDZP+rorafpqKFtPo8XEP9Pi4B6cGe5lSVz7AzMUM827Pwxvb3mh2n8L//gR0dQEDA+XERVHNp/h8K+wtzLk2RzkBsExYLWQ7BJmRQT81dm7xOewdvlet3nDKUFtWi9yo3EZTjuTH5ONP/z9xf8N9AOzP/JF10A8A5nWch1E+o8DnMZOb+snBJzg+43irqAVJ0zRGbByBQb8OYjsURtA0jbA/w/C99veQiCWw9LbEmB1j4NrPVab9RSIgLU36u7OzdMBk0ybVfMGUiCQoelbUZFogWfD53M/R7tjdEX2+6KMWs35UqW7Qz4GBSYZt2wKrVgEffywtIj1y5PPneDzpAAwgvdB6lb6lPsoyyjR64oOusS4G/zoYQQuC2A5FYQaWBhizcwwCZspWr7SOUw8nvBX2FnxHv57ClaaB0JhoAEBnZ7/6geKG8HjSFX+WlkBmpjSdLE2jPiV3ankSannF6NZNmuJzyxZp+hSaBva9sMDyyhdXcGD0AVa/p3Gh36RpGsIq6f8Dn9E+GLx6MPp+1ZflqOTjP9Uf7gPdYR8kLSDK4/OwJGEJJhyYINP+CUXSpfbOps4w0Jb/CryjbUesHLgSBycclHvfXQN24fSC03Lvp+58x/hiQcQCWPtZsx0Kq8QSMdJKpV8AXc1cWY1l7vW5mHpqKqsxNIcLfWZzqouqcXH5RYT/9Xq9b2UwsjF6KVX1iD9HYPbV2TCwNEDS5STcXnUbDzbJV2+Uaygexfo9Ak107ftr+LXNr0qtO6mJJGIJHmx8gPjT8WyHIhMu9JvCKiGqi6th4sSdiT4pKdKfdnbSAZhX1a32e/oUiGh+AZdCSlJLsHvQbo2d8MrT4qHz253R5d0ubIeidI7dHMHjNz+EUzfoZ2mpnHp+deoG/e7fBxrK7qtnpqeRn6uiGhF+MvwJ/7z9j8JtqLLPJIN+LGvJix20MAhvbH0DaOWzQo5OPYrNgZshqpbOMIw9EYvNnTbXFzzVt9CHsErIifpGQrEQD7KkF0WyDPq9iIlZcv7T/THp2KRWUQuSoii4D3RH27Ft2Q6FMZn3MuE2wA1V+VVy75uRIR34MzQEPv1UmhIxLEw1xXcvfXYJf3j9gZLkEoXb4PP58Pf358RFBSEfmmZmpd+LKAro0wdYs+blQT/geYrPsrLX9zO2N8aH2R+i25JuzARCKIWWnhY6zOoA+872jLWZkAAYpo+FT/5neKv/0Ga3NzUFPvkE4POBGzek7zUTbQs4G7sDAMoMHtanATU0BN58U7rt48fPV/t1fqczxu4ey9g5yIsL/aagUoBdIbtwct5JANLP5uClwWqX+tTc3Rwzzs+AsZ10VoFYKAZfmw9TZ1OZ9k8skmbm8LTwBCBdRSrPant9bX0s77kcfV37yn0B3febvujxUQ+59iE0R2Z5JkQSEbR52rA3ZqZPpWkaKSUpOJtwlpH2uIILfaYs9Mz1sDBqIfr8rw8rx+dr89GmrbTemG0HW+iZ6qHzW+pZy+9FabfScHfdXbbD0Cgmjiaw9rOW+bOSkDJ3M8eip4swZPUQtkNpFlf6zV6f9sJ7ie9BW1+b1TheVDfo92pqzzoWFsCQ/15iZa32kwglyAzLrL83qmnEwlZaz6MJdYN+Vkou5+3nJ70GLi2Vpql9lVgoRvyZeI1bsSyoFKD9lPaw62Sn0P58Ph9+fo3XimcaGfRjWUsGcjyHeiJwTiC0DbjzwcaGft/2g+cwT+gYSVfWiIVilKSUID9GWlPLyNYI7yW+x4mbvFF5UagWVcNMzwzelt4y7SMQC/BN6Dfw+N0DBVUFLTq+tZ812o5tq3Y32xRRXVytUelEKIrCmB1jMOvSLIUGbZP++6x1cwOcnKQzywBg82bpYKAyeQzyQPCHweDrKn4xQNM0ysrKOP2aVuRUYLXjatxZc4ftUDiluFiaHpHHk850VLa6Qb+GVvpRlGbP5KZpGlu6bkHot6Fsh8KKK19eQfTB6Aafi4gA2pQPxGyHnzC8bX+Z2mvbFliyRPrevXoV+OYboF3FIvhm/ITgtm4v1UmwtgYGDJD+fuCA9Kf7AHcETA9g7QYEF/pNHUMd6JrqQtdUt8Wrvbki9mQsftD5AQVxsn8ne1YkrSvnYS5906xYIR0ojoxUSogv6fxWZ7SfIn9KUXWWeiMV/7z9DwpiW/a9WRPUpfZ0NnVmLHNIlbAK7uvcMWLfCORXyl7DuDC+EPc33EdpeikjcTCNC32mLCiKgrWfNWz8bdgOBcb2xph/Z75G3I94tO0Rzn9wHlUF8k+uJBrWZWEXzLk2BzwtcttRXla+VmpxzcKlflOWVVCqVDfo59ZEZu0JEwBtbSAmBoiKYj4GC08LfFryKXp/3pv5xllWW1aLn01/xuUvLrMdCqcU/PfV11LJ1Wa0tIDO/833aXAhAQ0cnnAYt3+5rdxAVMzA0gDj941XOKtSXZ+pKtzqFVshSUPrYOVtQ6QZN1EUZR9kj2mnp9X/23eMLz7K+QjuA91ZjKph7ubuODjhIFYMWAEeJdufnzZPG//E/4PkkmRsDNuo5Ag1x54he7ApYBPbYXBGXT2/ui+dkyZJZ5cVFADXryv32B6DPTB41WCYOik+y1MikSApKYmRPlNZakpqYGRrxKkZhlxQt8rPzk765VDZ6gb9Shu5p5gblYtr319DWYbqvmypSlVBFWrLalFTrP41kza024DDkw7LvL2gUoAbP91A7LHYBp+vG1zx95cvjgEDgK++AvT0pAOHWmHL4JnzGRZPf/07xqRJ0tV+Dx++POORrUwDXOk3Jx2dhFGbR3HuZoyi+Np8GNsboySlROZ9EoulK/08zD0QFweEh0tndO/eLfvM7uLqYuyP2o/199crEHXrknYjDQ+3PISoRr3rjDHB2tAaHwZ/iJkBMxlr01DHEG7m0i+UT/KfyLxf9sNsnFt8Dum30xmLhUlc6TObk347nVPvbU0Z0On+QXfMuzUPuiaaPzmWUA8ZdzNw7btrbIfRJLb7TUGlAFu6bkHkHhXMopJT3f0XlybK37662o+QXXVRNVz7ucLMxYztUDjlxfSeytb9v+R14Q1kG+fr8DF2z1iNKbfEFIlEgpS6GQEqoBnf0Fqxg2MPYqNf6xwIKssoQ9rNtNce52vzwdfhZloWMz0zTPKbhAVBC2Teh6IofBT8EQBg/f31qBZWK3z8B5se4Gezn5FxN0PhNtQBTdNw7uUMjyEezW/cSry40g+Q5pUfNUr6+/HjpHg0E6x8rfD2g7fVupaaMjBZz08WTaX3BKQ3HUO/CkXm/UzVBKRChm0MsTh2MQb/NpjtUFrM1NlUrlXN2gbaWJq+FANWDHjtOaEQeJCQgQLjy7DxzJY7ls6dgZUrAXNz6b+DgwH3BuYV2dgAISHS3/fvlw72bWi7AYcnyj54qUmqi6ohqhFpzGBfHa/hXliasRSeQzxl3meU9ygsDFqIHk49cOTI88djY2Wv45JZnolpx6bhs8ufQSyRPZ1Rxr0MrPddj4jdSioYw0G9P++ND1I/gLV/667nBwC+Vr5YNXgVvu73NaPt+rWRpiZ6kif7oJ9biBtmXpwJj0Hk+7miStNLsb3ndpxdolmpVbnAJsAGTj2cOHsfQd3EHInBybknUZJawnYoauvh1ocI/ToUhQmFbIfCWQVPC1CSXIKaUm5NeBSLgfT/5rc0lt6zzvjx0omx0dHS/xiNQyjG0+NPkfUgi9mGOcDM1QzTz05H57fVP700k1Q56OfjI/2Znt5w9rB249vBJoD9rARMCv8rHGcXn4WgQsB2KDLRrKvwVsjM3QzW7a1bZS7j26tu4+/ef3N2tiiTJvpNhIupC/Kr8rEzYmf947fSbmHEvhE4HX9apnYMrQ1hH2SvESlYmkJRFIasHoLBq9T/xjcTaPr1lX4AMHSodPVKSoryikfXOf/heewdtle5ByE4iel6fs1pKr0nIL1h/+b9N+E5VPYb9upGEwZZZvw7A8N+Hybz9hRFwcTBBObu5q89Fx8PZBqcxl2fgfgm/E2F4nF3B1avlqZkHD07BUdijqCs9vWR5YkTpelAw8OBZ4kULLwsYOJkotAx1d31H67jF8tfUJxczHYojJM35dZU/6nYOGIjnKmeuPtfyaguXaQ/Za3j4mvlCwNtA1QIKhBfGC/zsfXM9EBRFCdqW6uSqbOpRvSFXFU36BedJ/sdSkNrQ7gPdIe+hb6ywtJ4WrpaGLBiAPynyrlsnZCJRCRBZX4l22FohPQ76Xi84zG09FSQ5kND9fq0FxZGLYSllwru3qsp+yB7LMtahk7zO7EdykuysqSTDvX0AFvbpre1sgIG/3fbiunVfuJaMQ6NO4SwDWHMNkxwlioH/dq0kb7HxWLpe74hYoEY1UWKL1zhmmfnniH8r3Bo6avHZxu5ElJzQ34bgklHJ4Gv3fpmpAUtDELfr/vCMVhFd5IZ8Ff4XziXcA41IvlmImnxtLC0+1IAwG93foNYIsaDrAcYuncoziacxegDo7Hj8Y5m22k7ri1mXZqlcbMtiKYVFgLl5dIb0c7Ozx83MgIG/bfa/tgx5cZQnlmOktQSiAWKT1DQ09NjMCLmPdj8AA82PWA7DM5ha9CvsZV+hm0M4dDFQSMnP1z96ioSLyayHQYrChMKUZbRcE2RqCigQk+ab9PXylfhY1hZAaNHA28cHYCJhyfifub917axswP6/1cy8MQJYOqpqRi6dqjCx2wpNvtNmwAbeA7zhJmrGWsxcE3dZ2337sDixfLVcdHiaaGjbUcAwIMs2T9rrHyssOjpIgTODlQgYvVTEFeAuH/i1GYGrrI9yXuC7PJsSGhmU6/5Wf+30k+O9J6ANBsHl2umcf27pqG1IXp92gtuIU0UiSIUtsZpDQ5PaJ2r85k25Lch+CjvIxjZyF+LnpCy8LSAdXvur1hnu9/ka/M5N7hcl73PxQWQZZ7YhAnS1X6RkcDTp8zFoaWvhbF7xqLzAs1aDUfTNI7NOIbHOx+zHQrnNDXoJ5KI8NXVr/DllS9x/OlxpJemt6geJ0U9T1+bmvr685V5lVhhsgKXP9ecuouTjk7C0vSlLZpYqMo+kwz6sYzPb32DdUyx8rFCv2/6qUWBYwCoEFTgndPvYPi+4Qql6JzfaT7M9czxrOgZTsWdgp2RHRxNHGFjaAMJLcHck3Ox+s7qBvdNKUnBxrCNKKgqaOlpcB5N09jeczuu/6jkQnVqpG6Vn6MjoKPz8nOjRwNiXhX+Lp4Lp9/cEJ7VQEJuBozbOw6LYhYpnDKHz+fD19eX033m3dV3cXftXbbD4ByurfQDpDPOCuI0qz8sTS/F9e+vI+ZwDNuhMCJidwTurL4j8/bnl57HOvd1oMWvX7hERAAVetJafy0Z9KsTZC9N4dtYfzlypPTnnTtARUWLD6cwtvvNwDmBmHRkktp8T1OWouoiPMp+hOSsMoSGSh+bMEGxOi517z15Bv1am8jdkTjwxgEUxGpWH6+oEftGwH61Pe5mMPv9pD69Z/4TuW4Y7R60G+t9uVmXku0+szk0TUMi4na9QXXnP8Mf7oMayN9NKMSwjSHbIWiE5KvJuL/+9YlmXMBmv/nk0BNc/foqqou5t4qobtCvudSeddq0AXr3lv5+7x5zcfD4PARMD4BjN/VZKCGLiuwKPDn4BFlhmpe2tCUkEqCoSPp7Y4N+7dq0w8Wkixh3aByc1zqjw6YOqBQovsK97j3e0KCfQRsDtB3XFnad7BRun2soHtWiySx8Ph/e3t4MRtQ0MujHssVnF7doZB0Arn1/DVe/vspQRNwnFoiRfCUZErF6XfTUpWJqY9AG5vqvpx9rjpGOERYGLQQArLqzCg4mDrg+5zoSliTU1/z78MKHmHZ0GtJKpbUOS2tKMev4LHj+7olFZxfh192/4vIXl5H3JI+hs+KeipwKVORUoLqQe1/+2FI36NdQDSqhfgYiO/dBhtUOZFSkYOrR6bgdVo0TJ4D8fOZiaGmKLYlEgsLCQtaKhMti5qWZmHBwAtthcEpNzfP3kaoH/UpLG9/myOQj+LP9ny1aeco1Jo4mWBy/GD2X92Q7FEY82voIN1fclHn79lPbo8+XfcDTermvEQikddMq/1vp52Pl0+LYOttJZ8s+yG544MXDQ5pKWSgEzh8oxoWPLiAlNKXFx5WXOvSbrcHFxIvo9FcnDN45HCIR4O//vAaGvHVcuthLc4I29t4DpKlCY2Kk7786iRcScfmLyxCLaFRr+NejDrM7YNj6YbANbCafVisgFAuRXiYtg+BmxuzKMF8rX/AoHoqqi5BbmSvzfl7DveA32Y+Tg1dc7zMLYgvwi9UveLj1IduhaKzBvw5Gny/7sB2G2qkpqUFW+POb7wWxBXh67Cnn6qypI5qmcWn5JVz+/DJqy2rZDuc1bPabUXujcGfVHU5mPZN30A8AAgOlP2XJ/tDaGdsb49OyT9Hvm35sh8IpJSXSgT8e73kt+hfpaelhoPtAdLLthA42HcCn+IjKi8Kxp4qn/WpqpR9FURi/b7zG1F0UVgmRfDUZFTmKz+qVSCQoqhuZVQEy6Mey3RG7EVcY16I24v+JR8QOJRfk4pDEC4nYNWAX7q1jcAqMCtQN+nlbKj6qv6TbErzZ8U1sGbUFANDGsA2MdY3xy6BfsGLACgDA/uj90OXrAgCMdY0RlhUGMS29sV31rAo3f7qJgqeaO/vZ2M4YS54twYAVA9gOhTMaqucHAHcz7qLLli7IpMOhLbSErtAGCUVxmLXtG2zbBvz+O7NxRB+IRsRuxfoqmqaRnt6y9APKZupkCtsO5Cbji+pyu5uYAMbGqjmmmZn0Z2PpPQGg3cR26PlJT4hqGqg4raYo6v/snXd4FNX3h9/ZTe+VNEpCTei99yLSq6A0QbGgogJKsX39WQAVxC4CShUEqVKkd+mBQCCUBBJCIL33sru/PyaTQsruJrvZTeB9Hp5dZu/MnJudnbn3nnM+R8C5kTNODZ0MbYpOGLZyGFNPTtW4fcuJLen1Sa8S22/dgmxFJhlmYYBuMv0kp19ZmX6CUCid/N+xHM4uPcu9I/cqfV5tMeR9c+vzW9n6/FajvmdXFXcTRcnd3OgGgJjlJ1G0jsseDcozS5l+VyKvkKcs/f51+jTMmwfffVe47fY/tzm98DTff57ChAlwtwarADs3cqbjmx1LBAA8iTxIeYBSpcRcbo6bjW6l/S1NLfmq/1dsGLUBa1PNs3m6zO7CkJ+HGOX3o+k988bfN9g2YVuV1yvNScvBxdcFG4+ncolPMR5UKhUbh25kVadVpD5KBSBwUyBbxmwhIbjqFjZrKoIgMGzVMF6/+jrmduaGNqcEhhxrjts+junnp2NmY6a+cRVTEadfy5bia0gIZOhQBfunJj/x97iaJ1tsammKlYuVoc0wKiRpT0dH0fFXGi5WLvw69FcCXg/gk16fALAhcEOFz1me06+mEXcrjnV911Uq+EqlUhEhSWFVAcY32n4COXzvcKX2H79jPDNDZurIGuOnVvNa9Py4J35j/AxtilbcjsvPMnCueJaBu407K4evpKlr02LbBUFgfvf5HH/xOF/2/bJgYi8TZPwy+BdebvMyAEmtknjr9ls0fLZhhW2oDgiCgIm5cem6G5J7+WvNRTP9UrJTGLhhIFFpUbSo1YJXVBdpFboa1+Rn6ah6G0GAgACIitKdHcc+OcbJz2um7GpGfEaZ9cSeZKpa2hNEByOIdSzzyvDptZjQgr5f9DXKyXNFCf8vnJz0mlPDyrmxM65+rpU+ztWrkG4RDIIKRwtHXK0qf8y2Hm0BCE0KJSGz9AWt3r3FDK6QJBdGHJn5REWiqlQqMuMzyUrMeuKlPQHuJogeNvOMBvj4QJs2xT/v21d8DQgAhZrk40bOjbA1syUzL5NbcbdKbROUr/B76hQ8EJO86DKnC8MPvsGxS7bk5cHBgxXsjJGTm5FLXnbNCeaoLGFJYQB4O3gjE3Q/7X+v63tMbDkRW/MqiuoxEqKvRnN9kwapuTrGq4MX089Np/GQqpOFetJICktiy5gtXNtwzdCmVBsEQaD3//Wm5aSW2HqK94LWL7Zm2MphuLXUbbDBk4p7K3ccfbRXiqrpyOQyo6x5mJUFMfnCWpJDRBNcXMTa4Eol3NCuXG65ONZ3xNarZj2nb+28RexNHcpS1RDKk/Y8GnqUL09+SXB8cMG2iS0mAqJPIjI1skLnlK7xqCjILiUZOeVhCn+N/IuLv16s0PGNCRsPG5794dlqtZ7+1OlnBBwJrVxRSzsvO6NMadcXDt4O9PmsDw71HAxtilbcSah8pp86enn34oMeHxTb1senT0FWQoZZBs6NnY0yGkoXJD9I5t+3/yU6UHOZoZpOVhZE5j+/i2b62ZnbcX76eRb1W8R/L/3H4vk+/DpnEDc/3MfG37wK5CUOHdKdLSNWj2DCngm6O6ARcXXtVZbVWcb9k09AiJMWGMLpZ2tbWDA9NbXqzmtI0mPTWd19NXte0yBVqJqQl5VHemw6KqV6R3rgpkBWtF9RTFqq4LNASCsi7akLJ5SjpSP1HcUoisuRpUf62dpCly6gksk5d8ep0hLH1QlBEJh8aDIT9tXM+722hOQ7/ayzGzJqVOH9SaJhQ7C2hvR0Mbq7PGSCjD0T9vBg1oOCmmqPIzn6VCrYtk187+jjyAF/14Kw3zNnxEWlmkbA2gAW2y3m/qmnz2KA0ERR6sHHUbfSnpUhPSad7ZO2c/n36itR2feLvnyQ8UG1m4s+RT2mVqbc2nmrRpfC0Af1+9Vn5JqRgBgImZ2aTdvpbStcy/0ppXNr5y32z9pvaDOMgnPfnSM2yDidPpLDz9pae6WbFi3EV11KfE78dyLPLntWdwc0MHnZeWx9fisHZ9fQCLZKEJcv6Faa02/V5VV8dOwjVvivKNjWwKkBXet0RalSciysYiXD7O3FfypV4RykKJaOltw9cLdGZH7betjSaWYnPNt7GtoUjXlyViCMmGOhx8qU6NEElVLFw4sPeXCmlF9YDSMtKs0oa0BoQkGmnw7qCWmLhYkFANlZ2aQ+SiUnreZkgxTl3qF7XPjxAvF34g1titEQFiY+gJ2cCmudSfi6+DK/+3xszW2xtYW2bcHZWVyNfOYZiLU9zKHDKrWZB5pSt1tdnBuXMgLRENuq0oesAO5t3Gk/oz21mhlftKEhMYTTTyYrzPYrq66fSqVi86jN7H1zb9UZpkdkchkDlgyg5eSWhjZFZ+x/dz9Lai0hM1F9AbKc1BzSItNKBLRkZcGdO+CY3olve//Ou53e1Zl9ksxiWRKfUCjbePJgJnePhRtk/GLI++aT5Ogsj6Ao0ZPnYdGAHj1Kfi6XF9ZxuayBH6RnvZ7UtqtdpgO76IT72DFx8Sk8HM4fz8Q0MwVzc7Hmh5QRWJOw9bTFp58PLk1cDG2KUVCQ6WfvrZfjp+WkcejuIf66/pfG+5hamRL4ZyCPLpUM0jAGNLlnBm0NYqn7Uu7suVMFFonc2nmLbS9sIyGk+i+aGTPWtaxZkLaA/ov6G9oUoycvO4/NozYXC3hUKVWs7b2Wg7MPositOXWzjYXAPwO59MslUh6WU8PAAFT1WDMtKo0Dsw5oVfu7KpGcfrUqsCwgSXxee5psXCaCIDBm0xi6zOliaFOMDkne0+WxYXB2XjZ77ojBwWOajin22Q/P/kDoO6FMaFHxYM3yJD5NrUyZlzSPgd8OrPDxaxo2NlUn0/50Nm5g7C3sSc5OLnfRSBNW91jNyS9qpmxeUf55+R++q/ddtavDpFKpdFLTr6J4O3gztPFQOsR24Fuvb2usZEnraa2ZETijWqVb65uy6vmpY3vae5xvMoAL5gs1WoTUlOyUbFIitJ+oyOVyGjRogFxunBGjPn18GPLLkKe68o9hCKcfqHf6CYJAwt0EUsKNa9JcUSydLOk6pysNB9ace1+9XvVoP6O9Ro6jdq+2Y/bD2SUW+m/cEOUS6znUZVavlxjffLzO7Huj/RtsHruZSS0nldmmVStwdQX7q6fY0Hd1ldd/MtR9c99b+7i1q3TpySeNjJxM4nIeAjC2bwNMylAelyQ/r1yp3PnS0wulfZo0EbP5tm+HvzYqaXlwCa0f7qV7d/Hz//6r3LmMEd8RvkzcNxHrWprXmKvJhCbpN9PvXuI9ntnwDK/veV1jeXMzGzMWpC1g6K9D9WJTZdD0nvnI/xGm1qZYOFhUkWXw6NIjbmy5gam1aZWd80nF1PLp31gTIi9HEnIghNBjoQXbBJlAv0X9iL8TT9zNOANaVzMZ+N1A5kTNwc7LztCmFGCIsaalsyUvHn+RzrM6V9k5tUFy+rlWoKKAlOl37544ptMF/iv8ufhL9ZdWlJCbyfEb5Uf9/vXVN37CkJx+Tk7Ftx+6d4jUnFQ8bT3p6NWx2GftPNvh7eBdqfOqq+tXU0ov7XtrH7+1+a1SQS1yuZz69avu2q0Zf/lqTPc63dn7YC9HQo/QqXanCh1DkAkM/mkwDt4OujXOCKnTvQ4OPg6YWFSvS1eFisNTDnM77jYNHBtU+fn7+PShj08f4oPjuRB+Addmla9pZIwIgmCUuu6GRKrnV9Tp9/vl3/k35F/e6PAGfX36lrqfr6vonL7t9RHL9jdlY4dRAFy8CJcuwcCBxWsEakJuRi5fOX1F0zFNGbt5rFb7KpVKYmJiqFWrFrKyqhI/xahQqeChuNZd5U4/Bwcx26Uspx/A6wGvI8ie1hszVlq80IIWL7So1DGkTKYWlTtMqfTy7qW2jSBA//6w+1YTXH1sqryGpCHum2lRaVz8+SKKHAW+I3yr5JzGzL/nxMVQE4Udzw0pO9Ndcvrdvi0u8liX47PKzsvm6/++5nLUZTaP3YyZvDDDVcryc3aGKVPgww/F+n15eTI863dh1GRHnLrBkSOixOerr5aUG31KzWG032jcbdzpXre7Xo7fxLkJckFOcnYyj1If4WXnpdF+ZtbGWWZAk3umSqXi3LJz1O9Xn3o9tSjWVEn6ftGXzu92fhpcVgUk3U/i/sn7NBrcCCvnp3/vsqjTpQ4zAmdgX7e4lEzjoY1pPPRp3Ul9YEzOPglDjDXlpnK8e3lXybkqQmy+6mhpmX4qlarcUgNOTuDlJc6hr1+HThVbIi7GhZ8uoMhW0OGNDpU/mBGQk55jtOMIQyM5/R6X99x+czsAo31Hl1vjOUeRU2xeoSl164qvZTn9MhMyubnjJm4t3PDqqNlY0RiR1o4qU15NqVQSHV115aierpwamF7evZAJMh6lVk7ipO30tk9EpEOPBT0Y/NNgQ5uhNTJBRkevjkxuNRlzk6pd9CuKcyNnBn0/iHo9qm6SWlU88n/EvSP3nkqJPEZ4uPha1Om39upatt3cRkBUQJn7vdruVaY2nQnAltxJHL8VwJ9/wmefwb598M47sGiRKB+qKaZWpnR6uxMNBmrv+FapVERFRWkcSV6VJD9IZkX7FVxdf9XQphgVsbGQkwMmJhWTN6kMkpRteU6/muTwW9lhJbum7TK0GQYhKSyJA3MOlJpFFxkpBt3csV3JibATlZJSryg9ekCacz1u2HfFulbVSXmAYe6bNu42vBf9Hj0/7lll5zRmTh9ywu/BNwxxeB8bm7LvObVqiYs8SiVcVfMoMZObsezcMnbe2sn1mOvFPpOcfnXqiM7uJk0gN1cMwqg9tT8DP2hH69aiUzEhAW7erGQHjYjw0+FsGr6JiPMRhjbFaBjtN5olzyyha52uejm+uYk5jZwbAZS4FssjOTyZoG1BZKdm68WuiqLJPVOlUDFs5TA6vFX1i6dPHX5VQ8i/IeycspPwU+GGNsXoUClVnPj8BKmRYtFspwZOlVr8fIr2ZCVl4b/Cn5jrxlF3sqrHmiqVipgbMUa5JiBRlrxnriKXZzY8wwr/FeXar+u6fs/9/Rwv7HlBNwczApa3Ws66/usMbYZRUlpNv1xFLrtui+sEj0t7SkSkRDBs0zCa/twUpUr7chTqMv0y4jLYPX03gZt0WKzSAAz6YRCvXXmtUsdQqVRPnX5PEuOajiN+bjw/Df5JJ8cz5odfZamutfzK4+HD8heldU1Nvj7OLj3L+v7ryYjNMLQpRsXjkWYRKRGcCj8FwHNNnyt335VjvqWeYgAKeQbDNoxm42axFqSfn5gZcOYMzJwJhw5pbs/AbwfS5qU2WvfDmEmLTCMtMo3cjFxDm2JUSNKenp5izaqqxMFBfE0sR00xIz6Dy79fNtq6QpqiVCgxtTJFZlqzhnR3D95l6/itxAbFltsuaGsQ5749V2ot19hYyDJ9xK8Rr9JvXb8KTWLK4+T9k3x1+ituxpbtOfH0FB3fOTmFE7GajnUta+zr2KtvWMMJC4OQAHcaxrzHikkfqW3ftq34qk7iUxCEgpqSlx5dKvaZFOhTt674nB43rvCz8fnqtqam0DFf2ee0cZbDqRCxN2MJ+TcElaLmjnWNkWauzQC4EXtD432u/3Wdv8f+TUygcSxaa4PMREadrnUIPxVOxLmqcTBf23CN65uvo1TUvLmwMdJocCNGrh1Jw0E1RzJdV4QdD+P4J8c58dkJQ5vyxJJwN4E9r+3h8iod1t+oRiSEJPBr8185NFeLBYgqRlp/eVzec97heRy+d5j3D71PdHrZi/66ruvn0sQF50Zlq01UJ5R5Sur1rIdXp+qbLaZPSqvpd/L+SRIyE3CxcilT+cHZ0pmT909yN/Eup+6f0vq8UqZffDykpZX83KmREy/sfoEus5/WYaxqatYKUTXEztwOBwuHSh/n1s5bLHFbwt0DdytvlBGSFJbEEvclXP69eg5udt7ayc8XfuZWXGGNm5s34c034eOP9X/+8xHnsfzSkraftWXzqM1c/0vzaNzqQo8PejD0t6HYelZtIWljRqksrO0jPfj/vvE3AN3rdqeOfZ1y9zeRmbCs62bMczxIMw0l2mULs2bB11/Dzz8XLhr+84++elA98OroxeyHs2n3ajtDm2JUGKqeHxTq2EvXf2mkR6eze/pugrYGVY1RekImlzH1xFSGrRhmaFN0SlJYEje23CD1UWq57brM6cLUk1NLVTuIiYF0C/G528CpQYXkSsrjmzPfMP/IfI6EHimzjVwO7u7Q8OImto3ZpNPzGyP3jtwryAB40pFq5nXpolm2syTxefmymJlXHmU5/Ypm+gF06CA6/l56Ccwi77O2z1rCToQVq+tXU+LB2r3Sjvkp86u1bJAuSc5K5uyDs0SlRen1PAVOvxjNnX6NhzZm5LqRODV0Ut/YCEmJSOH0otNVFjR0/H/HOTzvcI1SKDBm7Ova02pKqxpTg0iX+PT1Ydy2cTy77FlDm/LE4tHWg7FbxtLnsz6GNsUgyM3kdJ7dmUaDGxnalDIpLdNvy40tLDu3DIC1I9fibuPOibAT7A/ZX2J/KdMvNBRSdTCkzknLISOuZgTGy0xkjPhjBP2+7GdoU4yOjAzIyhLfF830ux5zHStTK0Y2GYmJrPTnmqWpZUFCwIrLK7Q+t7V1oZM7vJQkeUEQaDy0cbUOCs1KyuLUwlNVFvClK546/QxMeXrO2mDlaoVjfUcEec2cDKREpOBQzwHrWuUUOTFifr/yO2/9+xbHQo8BkJcnOk0UCvFhXt7CtC4wlZuSlZdFXmYet3ffLjUjorpTq3mtp06Xx0hKEq8xQQBHR3HbXzf+AuD5Zs9rdIxBvR1pni7KfGa2/pY+fcTVwTp14O23xTZhYZCSoplNyeHJrO27lvM/nNe0G4B4r3RyctLZPVMfGLNthsAYnH7lZfo5NXLi+V3P03Fmx7IbPcVgtHmpDR/nfaxWulwQBOr1qIdMXnxIm5cnfv9pFrcBsfaUrmnl1gqAq1Hl6zFKv4HMzKr1rlT1fTM3M5c/B/3Jnlf3VMn5jJ07dyDB5jS2ja6SlZeltn2LFmJWaEyMKE1bHpo6/QQBJk+GUaNAkasg8nIkqY9SadMGLC3F8eetW9QYTC1NkZk8nd4CnH94nq5/dKX/uv56PU+zWtpn+rk2daXV5FZGN6/T5J55ZfUVto7bysh1I2k5uWWV2DX15FRGbxj9dJxZxaRGpnJ7921Dm2EUFFUL8hvth4nFU4eooRAEgWbPNavyOtFlUdVjTYd6DgxcOhCfPj7qGxuAvLzCtT3J6RcUG8RLu14CYH63+Yz0HcmuW7vovbY3U3dOJTGz+ITVwaFwHHddB7H6W8Zu4Tvv7yp/oKcYNVKWn7U1mBe5PbzT+R1S5qew7Nll5e4/o/0MQHRQV6T8mDqJTxBlPqtrOaaEuwkc/fAooUdDS/08MxPOnYNsNcr1giDgKC3OVgFPZ0UGRiaTcSDkAF1+78L0f6ZX+Dh1u9Xl5bMv02CA9rWyqgN1u9flVf9XaTJM94t2VcGd+DsANHYWi1rv3Fn8ZqjvBRdzuXjXj3KO4pO8T+j1SS/9nrCKSQhJqJHyr5WlqKa3XA73Eu9x4eEFZIKMsU3HanQMCwvY+9lr9PXux+cDPkJF4aTP3r4wlV/TAamlsyWRlyNJj0nXpivIZDLq1q1bZQXCtcF/hT8hB0IMbYbRYUinnzSOKi+gQm4qp8nwJth52VWNUXoicGMgZ5aeIS+76uvV6ROZiayEI+9xLv12icjLpXtH4uLEDKYMK/EB6+viq3MbC5x+0eqdfiEdXsDipQk6t6E8qvy+qYKhy4fSfkb7qjmfEaNSiU6/AO9pzLzRmnMR59TuY2EBTZuK7y+rEbaQnH6BMYEFDsWsrMLo8jqlJPL79PFhXtI8WrzQAlNT6NRJ3C5lJFZn0qLTuLzqMsnhVaiZb+TcTRDVX+o76rfmu5TpFxQbpHUZAZXSuNJMNblnyk3l2Hra4tneEwt7iyqxy87Ljrrd61bJuZ5SyN9j/2br+K1kJmQa2hSDE7wvmN+7/k7UVf1mDj9Fc9Jj0rl3+J6hzajSsaZKpTL6cjXx8eIY0NRUXCtRqVRM2zWN9Nx0+vn04/O+nwMwsOFAfF18iU6PZvbB2SWOI2X76ULis9GQRjWmvMql5ZfY+8ZectJyDG2K0SE5/ZxLUXKVy+TYmJVfW76dZzu61+1OnjKPXy/+qvX51Tn9/vvmP75x/YaoK9XzOeLa1JVXLr1SasBXRATMmgVffgl//13+cWQyGXVKm6jpCeNbPX3CUCqVCILAuYhznLx/0tDmPEUP5CpyuZcoDsiauDQhOho25St8SdE/N8suB6QTLEzESakmkebVDZVKxbp+61jVeZWhTTE6JKefJO2589ZOAHp798bNxk3j47jZOXHkxcOM9huNTCj+2JAGpJo6/cyszZiXMI++X/TV+Pwg3ivDw8NRKo3LuatUKNn/zn7OLVO/oPukoY3TLyY9hp23dpKn1I3jShN5T4nqHHEGcHnVZU58egK5aRUXTtQzWclZ3D95n5SI0tOI02PT2ffmPo58ULq0plRPI9tWj04/d9Hpdz3mOgpl2deQ9Bt4+FDnJpRLVd83Ta1MafNSG6OWXKoqoqIgJS2PTPMwABo4ahaUJ0l8Hj4Mf/0FK1bAhg1i5HhR6tjVwdnSmTxlHtdjxAewdM+1twe7UmIZBJlQLBJfcvoFBmrcLaPl/on77H5lN6HHSo++fRK5HS9mKEkBh/qisXNjfh/+O4cmHyoWGKaONb3WGN3YXZN7ZstJLZl+YTrmduZkJel/XhV5JZK0qFIK5DxF7/T+v96M2TgGc3vjyKgyJAkhCcQExmDlbGVoU56Sz6bhm9g8ejO5mYatKV+VY81I/0h+qP8DN7frefGsEkjBV66uotrC30F/c+HhBWzMbNgwekOBvKKFiQV/DP8DAYE1AWt4kPyg2HGkOs8nT4p1wStDp5mdGPTDoModxEi4e+AuV/64gonl02zjxymtnp+2vNPpHQB+8/9N67VjdU4/r45etH6pNabWphU30ICYWpri2c6zhETpuXMwe3bhPP/s2fKPo1QqefDgQfmNdMhTp5+BUalUtPMQJQmDE4JJykqq8LGurr/K0Y+O6sgy4yHpfhK7Xt7Fg7NV98PQJaFJoeQp87AytcLDxpPly8UHd4sWMGmS2KaqnH45WTncO3yP2KBY/Z6wClHkKGjzcpsqk9ipTjzu9HO1cqVz7c4Ma6y72l8ViUKrSE0SlUpFQkKC0UX3CYLA1JNT6fP5k1lXoSzS0wulNb3UlFe6+PAirZe3ZtTmUcw/PF8n55cy/VJSSi6WF+XM0jPVOuIMYNzWcUw9MbXG1fqJvhbNml5rypzYW7lYMf3c9DIDCKRJd6q5/uQ9Gzg2wMrUisy8TIITgsts5+UFJtnppB44w/1T5Wie6Jiqvm9WZ+e5rgkOhkyzcFRCHuZyc7zsNKszJy3y3L0Lf/4Ju3fD5s1w6lTxdoIg0MajDZYmlgULRY9Le5ZG+H/hXPnjCgC++X7w0NDCGiDVFZ9+PozfMZ4Gz9RMxZOKIKmM6OPeVxRTuSkvtXmJTrU7lQgMKw/nJs64+FZiZUoPaHrPTAhOYFntZZxdpmZlRwdsHLyRv0b+pffzPKUk9fvXx3ekr1rVgSeBzu90Zk7kHOxqV291jJpEjw96MPS3oQYf/1flWDMjPgOZiQwLh6rJsq4Ij9fzu5d4DxOZCXO7zsXdxr1Y2y51utC1TlcA9twpLo3fvr14jJQUOFrzlngrzLjt45gVPuvpfbkUSsv0W31lNX4/+7H49GKNjjHSdyR17esSmxHLX9e1G3sUdfqVdjvw7uXNiN9HUKuZBoXOjZDMhMwSQRZ794rZfZmZolqLTCbWNJTuA6WhUqlILK8GjY55+ksxApytnPFxEDWp/R/5V/g4N7fd5PTi0zVu0eXh+YcE/BFA8v3qKdlzO64w0vb8ORmXLok1W954o1DG6e7dykfwlIfk9DPJNGH9gPWc/1G7emrGjIm5Cb0+6UXndzob2hSj43Gn3+RWkzn78lne7fxuhY6XkJnAwlML+eTYJwXbmjcXX+/fh2QNf6LxwfGc++4cSfeTKmSHMSHIBLw6eOHVQbMF3ScFKdLJyQmsygkK3nx9Mz3X9CQyTZRonN2lpLxJRbCzEyVtQaxtWRae7T2rdcQZgKWTJR5tPQxths5xaujEsz88S71e9Ur9XBAEPNt74tnes9TPY2MhT5ZOiiBWE9dHpp9cJqdFLTHyISAqoMx2Xl4gz8vG6dIhbmyrmfWBVCoV3/t8z47JOwxtilFw5w5kWBTKK2rqDPHxgQkToGtXePbZwmfslSsl224cvZGUBSmM8hsFaOb0O/ftOXa/shtFjgIXFzESXZIirc5YOVvhO9IXWw9bQ5tiNDxeWsDYGLZiGKPWjTK0GVpz9tuzhOwPodM7najdSb/65SqVim7zuz2VTDYwCXcTeOSvfX2jmoaZjZmhTXhKEZoMb0KLF1pgYv7kZDw1HNiQmcEz8e7jbWhTykRSGnF1FV/nd59P0BtBZc5xpWDsf+78U2y7XA4jR4rvd+yAyiRS+q/05+9xf5ObYdisUF0gCIJG9YBzc2HtWrihebnhak9pTr8LDy9wK+4WCZkayB8BJjITPu/zOb8M/oXnmj6n1flr1xazW1NTC4O/axL/zvyXhVYLizn+JCnPwYNF55+fn/j/S5dKOYCBqDZOv0WLFtGhQwdsbW2pVasWI0eO5Pbt4gsnWVlZvPnmmzg7O2NjY8OYMWOIjo4u1iY8PJwhQ4ZgZWVFrVq1eP/998l7LA3g+PHjtG3bFnNzcxo2bMiaNWtK2PPzzz/j7e2NhYUFnTp14sKFC5Xqn1Sb49Kjil8dA5cNZNaDWTWugH2zcc2YFTGLRkOqp1yUNOlu5NSY9evFbaNHizfFWrXEjJS8PAjRY0mwgkw/sxwG/DyAlhNrTlacUmFcco/GhDTorEyKf1Gux1znw6MfsuTMEuIyRI+ivX1hVI+mg6pI/0gOzDrAgzPVM3u3KBlxGTWulpou0ETac/HpxTy/7Xmy8rIY3GgwyfOT8bQt3YGjLYKgWV2/6h5xlh6TTkJIgtHVRdIFth62dJrZCfdW7iU+u3/qPuGnw8uNKo6JAZnKlI/rHGPVsFU4W5VS4EAHFNT1iyq7rp+tLVi4O3Cry1S8J/fQix2GJjc9F8/2njj4OBjaFKPgzh3IMBedfg2cNM8+EwR44QVYsADefBPGjxe3X71aMmrW1dq1QCYKxMhSKN/p121+NyYdnAT5iQHS5FTfihP6RKlQkhb9VP6wKNl52YQmiVKnVeH0C00MZfml5WwM3Kj3cxmak1+c5PY/t3n2u2f1LmUsCAKd3+lM6xdb6/U8TymbnLQcfm3xK0fmly4lXtNJi0pjRbsV3NlbzSNDajA1LeBfE4pKlRsbj2f6ATRyboS1WemOquFNhgNwNPQoqdmpxT4bMACsreHRI6jMcnPk5UiC/g6q9k6/9Jh07h66q1Gd1ePHYetW+OEH/dtlLEgB/0Wdfv6RYlKR5G/QhCmtpjCjw4wyr9myMDMD9/xpe1klLY5/epxtE7ZpdVxjoW7PurSZ3gZTy8Jg8dT8n+yYMWJiT/v8P/PFiwYwsAyqjXfoxIkTvPnmm5w7d45Dhw6Rm5vLM888Q3p6ekGbWbNmsXv3bv7++29OnDjBo0ePGD16dMHnCoWCIUOGkJOTw5kzZ1i7di1r1qzhk08Ks1ZCQ0MZMmQIffr0ISAggHfffZfp06dz4MCBgjabN29m9uzZ/O9//+Py5cu0atWKgQMHElNeDmcZSA+sDp4dALj4qOJXh6OPI7Yetkb9EKwodl52mNtWTz19qaaGRVoTIiLExT/pshSEqllwMTcxp1e9XvTz7UfL6S1rTDH45AfJLHFbwqXfjCiUwogoqusdmhhKSnbptbE0pUfdHrRxb0NmXiZ/3yisUKutxKdPPx+mHJlCo0GaL5YIgoC7u7vR3d/+nfkvi2wXVftBtK5R5/S7GXuTD458AMCcLnP45/l/sDMvlAySalRVBsnpVxMjzSSurr/Kj41+rFLJSEOjyFGwY/IONg3fVGa9PxCDHmQqM3rV683LbV/Wmz2zuszi8quX+bT3p+W286ojI825HnHplnqz5XGq8r5pZmPG8zufp89nT6WO8/JEBYc0C3H818ip4o6Bpk3B1FQMXlBXE1K679YtZ4jn1cGL+v3qF9QAlSQ+q7PTLzYolqXuSzm16JT6xk8I9xLvoVQpsTGzKSEnpg8uPLzAjL0z+PHCjxrvkxyezMH3D3LvyD09WqYdmtwzX7nwCkN+HVKFVj3FkJjZmNF/cX86z34yFWWiAqKID45HkfPkOZaqA1vGbuF77+8NakNVjTUjL0ey/939xN407jI1UtD1ZeWacgMCJXxdfGno1JAGjg24n1x8PmdhIWYQAWzfXnGbBv84mE8Un2DlUr1rcoYeDWXDMxsIOaA+W+LyZfH10aOqr6luKB7P9MtR5HA1WrwGtXH6FUVb2V7P/PjtR2Ukx0dfjSb0aGi1TNxo/1p7hq8cXvD/3NxCtT7rfP+o5PS7dg2ys0s/jiAIuLm56dHS4lSbXPD9+/cX+/+aNWuoVasW/v7+9OzZk+TkZH7//Xc2btxI375ifZfVq1fj5+fHuXPn6Ny5MwcPHiQoKIjDhw/j5uZG69at+fzzz5k3bx6ffvopZmZmLF++HB8fH5YuXQqAn58fp0+fZtmyZQwcOBCAb7/9lldeeYVp06YBsHz5cvbu3csff/zB/Pml1yPKzs4mu8i3npIiLlSpVCoUCgVt3NsAYqafUqks9uMSBAGZTIZCUXyw9fh2lVJF0v0kTMxMcKjrUKK9TCb6eB8vslvWdrlcjkqlKnX74zaWtV2ysaJ9ykzMJPxkON69vLFytqqWfVrYdyGTWkzl58WuqFQqRo5UYmEBKpUMQRBo3FjBf//JuHFDxciRKr30SUDgyOQjOutTWdtlMrFPVfU9pTxMwb6uPVYuVjWmT+Vt17ZPMTEyQMDRUcEbe9/g4L2DrB6+msmtJle4T6N8R3El6goH7h5gRocZqFQqmjZVsnu3jGvXVCgUKrV9snCyoG6vugXH1LRPrpJOBhjN91S3R11MbUyRmRfa+vTag4gIAZDh4aGg6EdSnxaeWogKFSOajOCrfl8hE2SoVCpy83J5effLbAjcwJHJR+jj06fCfXJwEFCpBGJjlUDZfTr26THi78Qzav2ocvtkjN+TZ3tPOr3TCZdmLigUihp17WUlZbG291p8R/nS97O+he3lMGHfBLISs7CvY19mn2JilKhUAk5OShQK/fWpkWMjjfrk4QE3rqsIuZBIl072mJqb6v17UqlUuLq6Fow1jekeYczXXmX7FBqqIDtbRqbVLaCwplpF+iSXQ5MmAoGBAlevCnh5Fe/TzH9ncurBKdYN28jDh81RqcDTU7zmy+qTIAjkZuUiN5PTuLE4Hr15E1QqAaWyGn5PJgKtX2qNR1uPYvY8ideetN3B3IEfn/2R9Nz0KulTYycxm/Bm7E3y8vIwMTFR26eslCzOLjmLIBeo36++0XxP7u7uKJXKEseX+mrvY092Sjabhm+i/jP16fRWJ73d9/5b/B+3/7nNc1ufw9bTtlh7bfr09F5euT61f1NcxVOpVDWmT0VtLK9PPgN8eOf+O5hamz4dRxhhn1ybuiLIBLIzsovJfFZ1n8qbo+vqe7p35B7nvz9Pk1FNcGrspPc+lWd7eX2KiRHIkSXxw90ZLAnO5uL0i7T1bFtqn6S+np12FkdLMVpVoVAU69OgQbB9u4ygILh1S6Bx4wr0SVCJnymq9+/JvZ07g34aRO0utcsd76lUEBAgrgMIgsC5c4oCqVRj65Mur734eCF/DUScB1yNvEqOIgdHC0fq2tYtto8mfdoYuJGFpxeyacwmWnu01qhP7u7i3z0yUii1T2M2jQETUKEqsXZmzNdead9Tamrhmr6ZmbjmVbs2ODsLxMUJXLsm0LZtSdtlMhkuupJi04Bq4/R7nOT84lFOTuIN39/fn9zcXPr371/QxtfXl7p163L27Fk6d+7M2bNnadGiRTGv6sCBA5kxYwY3btygTZs2nD17ttgxpDbvvvsuADk5Ofj7+7NgwYKCz2UyGf379+fs2bKLeS9atIj/+7//K7H9+vXr2NraYp5rjqe1J2082nDv/j3Skgtlatzd3XF3dycsLIzU1MKU7zp16uDs7ExwcDBZWVlkRmVycNBBWr/WmhHLRxAUFFTsB9KkSRPMzMwIDAwsZkOLFi3IyckpJpcql8tp0aIFqamp3LtXGIFpYWGBr68viYmJPHhQKM1na2tLgwYNiImJISoqqmC7k5MTdevWJSIigoQiGmua9unR4UdcfP8i/b/vT7e3u1XbPmXd6UrM7TSsrBKpV+8BgYEq6tevj52dHaamd0lNdeP8eQXXroXj66u/PtnY2LB/yH7cu7nT8O2GOvueJKQ+Vdn3ZAH9/+5P/fr1iY6O1um1Z7A+oZtrT6mE6OimmJiYEZ94i5NhJ1GqlJgnm5OamlrhPtVX1gfgyN0j5CpyyUzPRBDCSE2tx40bcPlyNB06NFbbJ5VSRVZcFl6+Xhr1SaVSkZqaip+fH66urkbzPXV4owNRUVHFjv+kX3sAISG+gAVKZTiBgYXFHqU+dbDugL+9P+PcxxEYGFjQp6AbQWSkZADw/N/Pc+nVS7hauFaoT1lZzqSm2nHnThbgVmafws6GEXUximsB1xDkQrX6niyaWOA+1Z2QiBCIqFnXXl5mHhmpGaQki0FS/jv8yTbJxqaejdinBmL2Sml9cnJyJjQ0nXCHTfx5I5rYrE50btbZoH1SKjNxDbzFg30XCGo9klb9W+n9e7p27RqpqanY2ooqEPq8R9xeehvvDt54jPSo9teeREV/T8ePPyI11Y42WdNo2LERbZ3FxZ6K9snR0YHUVCeuXrWjR4/ifbpw/wLX466z5/J/pKTUxcJCSUTEfVJSSu+TSaoJu57ZRdOXm+Iz3Ye8PMjK8iYnx4KICAtycqrn9zTi9xGEh4cXO++TeO0V7VN38+7U9xPHbPruU05kDjJkJGcnc/HmRbq06FJmnx48iCAxMQFlrpJ+u/rh01qsa28M35OZmRnm5ubY2dnxsEhqgPQ9RdyL4FHoI2TmMoL/DUawF+j0Vie93cuz0rJIfZjK3ci7yOPllf6equra0/f3ZKg+edh74ObjVqP6VBO/pyepT50XdDZ4n6Q5esOGDfH09NTb99RpVie8B3oTlRVFcv7c0ti+Jx+f+sTG2hFutp1sZRY+Nj7IYmRkOWWV+z1FhEQQQUSZfWrc2IWLFx1Yt86GPn1SUCge4uych6mpSqM+RdyIIONhBg7NHKjtU7ta/546vtmR8PBw7gcWZkU+/j2Fh5vx6JEXFhaWmJmZcfBgIg0aRBhtn3Rx7dWtW5/kZDtSU1OJigojPV3J7rDdALT1aMv168VVlDTp0/qL67kVf4sF/y7g35f+1ahP2dl2ZGV58OiRZZl9unv3brW79qywwv9jfzye8cChhwMAMTGm5OQ0wNHRgvDwwu/J09OZyEg3Ll0yw9a2ZJ+sra0rXR5OGwSVtvmaRoBSqWT48OEkJSVx+vRpADZu3Mi0adOKZdMBdOzYkT59+vDVV1/x6quvcv/+/WJSnRkZGVhbW7Nv3z4GDRpE48aNmTZtWjGn3r59+xgyZAgZGRkkJibi5eXFmTNn6NKlS0GbuXPncuLECc6fP1+qzaVl+tWpU4fY2Fgc8zXIKuvlVylVHJ57GJ++PjQZ1sRoPeLa9CklIoXgvcE0HtIYh3rVM3sxL09gxgwZMTEqXn5ZyfDhhTYKgkBWloLnn5eRlwfLlyvx8tJvn1Z1WIV3b2/6f1Po3H4aWVfz+hQfDy+9JEMuF3j/+//osbY7TpZORM2OwkRuUuE+KZQKPJd5Ep8Zz6lpp+hWpxtKpZJ33hEICxOYO1dJz57q+7T1ua3c2XOHBWkLMDErOyJcslGhUHDjxg2aN2+OqalpjfmeytteXfukUMC4cTIUCoEVKxTFahoU7ZMUMf247anZqXRf050bsTdo496GE1NPYGVSXI5Ekz5t3iywcaPAgAEq3nmnbNuz07ORm8sLbHlSvqfq1qfVPVeTEJzAqwGvYuViVW6fUlNlTJyo4mSzVqRaBbJr/C6GNRmmtz5tur6JE/dPML3tdNp7tC/RXqlUcv68ih9m3adedjDvb22PcwNnvX9Pubm53Lhxg2bNmiGXy/X2PeVl5bHUbSkNBzZk7N9ja9y1p+339N13Sg4fFhg7VsXkyapK9+n2bZg7V4aNjcCff6qAwvbvHniXny7+xHO13yVjx7c0bqzim29UZfYpLzOPreO24jfGj1ZTxXqUH34ocP26wNtvC/Tr9+R8T0/7pNs++f7sS0hiCAcnHmRAwwGl2r51q4zNm1V88omS5s2Nr08KhYKgoCCaNWtWsF/R9lfXX2XnlJ2M2TIGv9F+1fJ7Urf9aZ9K375zyk6C9wTzfuz7IC/WvNr2SbKxrO8pOyWbS8sv4dPfB/fW7jWiT6XZ/rRPleuTujl6deyTOtvL2p6cLOPFFwX8648n0mkL87rO48u+X2rcp6y8LDJyM3C1cS3WpwcP4K23xL+XZJ+FBXz5pZKGDdX36cRnJzj5fyd55fIruLdyr7bXXlnbH7d961ZYv16Gjw+EhQkIgooNG5QFEozG1CddXXtxcTKmTxeQy1Vs3apEEGDG3hmsvLKSed3m8WWfL7Xu07Xoa7Rd2RYBgetvXMfX2Vdtn/z94bPPZHh7C/zwQ8k+KXOU3D10F0tnS7w6eZXbJ2P6nmICY1jVcRW9/teLbvO7AWL99vffl1GrlsDKlYU2XrwIX3whbl+xQoFQRPlY+q7PnDlToFhpZ2eHPqmWmX5vvvkm169fL3D4VQfMzc0xNy9Zk04ulyOXFx85Pj7JKNq23O1yeHbZs5q312C7IAilbi/LRm23q7PFsZ4jHd/oqHF7TbZXZZ/uJ91n1l8/ck/ZnFbOUxkyRM7jp7awEOWVbt6E27flBTWwdN2nVstbEZoYysl9J2nt3rrCfaqq7er6FH8nntNfnabt9LbU6VJH59eePrZX5bWXmAiCAE5OcOrBSQB61euFqYlpqe3V2S5tl8vlDGgwgL+u/8XBuwfpXrc7crmcli3h/n24cUNOz57qbW80pBF2te1QZCkwMTPRqE/SgKCitmu6XdPvKfjfYC79cok+n/cpmAyX116XNmq7vSqvveho0fEnFnOWFxvolGejtN3ByoHdL+ym06pOXIm6wqQdk9g+bjtyWfF91PXJ2Vn8DSQnF3csPo65den1Yo39e4q/E8/mUZvp/kF3Wk5sqba9rmzUdruurr2209tibm+OrZut2nPGxIBKUJBucQeAFm4tCpy6+ujTjts72H5zO36ufnSq3anUPtWrB2mu9Qk2q49T/cLtlbVF3XbJ1qJtdP09ya3lzEuYR2ZiZo289rTt0927MgQBfH2FYmO+ivapSROxVkR6Oty7J9CoUWH7dp7tALgef4UGgoC3d/FzlnBc2MiZuG9isW3NmsGNG+I49Jlnqt/3tPfNvZhZmzHg6wF6s1Hb7YaeP+0P2Y+jhSMt3VpiaWpZon1gIOzbB4MHywvqMqs7vro++bn6EZIYwu2E2wxgQAnbb92CDRtEGdm//pKzaBFkxGeQeDcRr45eRvc9lbbduZEz7d9oj3sL92KfP73v1fw+1e1WFxMLE7JTssusi1Xd+qRue2xQLEfmH6Hf4n54tfNS27469Enb7cbeJ6VCyeH5h7Fxt6HrnK4aHUcffdL3HD09Jp3owGhqd6qNmY1ZibbG8j3FxoJCyCbWYR8Ao5uOVjv+lrZ/f+57Pjz6ITPaz+CbZ74p1idvb3j3XdGZEBUl8PChWC/s2jU5TZqo71PjIY2xdrXGvrZ9layllLW9Mt+TUqFkqftSmr/QnEE/DCr3nNeuiWsAAweKY50HDwSuXpXTo4dx9UmT7ZraIiXUubgImJiIn9Wxr0PzWs3p5NWpQn1q49mGUb6j2HFrB1+e+pI/R/+p1vY6dcS/fWQkCIKMx7uVm5PLX8P/osXEFtTtWrdCfa3I9sp+Tx6tPfgo6yMUuYqC42Rni321ti5+zjZtwNxcvB88fCinXr2Sx5fWJaqC0ntoxLz11lvs2bOHY8eOUVvyjiCmiubk5JCUlFSsfXR0NO7u7gVtoqOjS3wufVZeGzs7OywtLXFxcUEul5faRjpGZVCpVMRlxFX6ODUBRa6C3MxcQ5tRKa5GX2VH9FLCav3IuHHiInhp+PqKr7duiRrU//wD77wjPrB0RVpOGqk5qWTmZuruoAbkkf8jAv4IIPVhqqFNMUri8m8jrq5wLOwYAH28++jk2M/UfwZ7c3sURWr/SItGj2XZl0nbl9sy6IdBmNuV7nCpDiSEJBCyX30h6SeNiHz1jNq1KeHw+/T4pyw+vZiU7JRyj+Hj6MOu53dhLjfnn9v/MO/wPK3tyFf/poiqRKnkZeURvC+YiPMR5Tc0MjLiMsjLyoNqp9dQMVpNaYXvCF+N2sbGQoZ5KEpZNpYmltRzKGW0rUvb3MSMKalYemm4uYGJiVjwO64GDvPkZnJs3GwMbYbBycoSA2DSLG5xR76De4n31O+kBrm88Bl79bFLTKoJfi/zCiqU1Kmj/fFdXKIIvvM5Sz71o7GXFx38/Pji88+LydoYM/cO3iPibPW6f+ubqTun0vn3zgTFBpX4LDsbliyB06fhgw/g22/hselzhfBz8QPEun6lnXPZMnGOA3D9OgQHw6H3DrGq0yqykrJK7GOM1O5cmyE/D8HF14W7h+7yyP+R3s6VGpnK6cWniQ6MVt/4KaWSp8zjzb1vMvjPweQp8yp1rA5vdGDE7yPKdPjVRNxbuTPt9DSaP9/c0KY8pQxkchnXN10n6O+S93pjISoqii8+/5wOfhUfYwT/G8z6/usJ3hesR0srT2wsxNkdIU+WhpetF+0926vfKR9PW0/Sc9P5584/pX7evz8sWADffw9jx4rbHmn4CPLq4EXHNzti7WqtsT3GRnZyNu6t3bGuVX4fsrMhKP/n0KYNdMzPH7l4Uc8GGpjERPFVWv8A+LjXxwTOCGSU36gKH/fjnh8D8Nf1v7gTf0dt+1q1xHlLTo6oPPY45nbmjFw7ki5zupT80MgRZEKx2qlp+RXZbB6b/pqbF87bjh4Vr70dO2D//sJxcFVSbZx+KpWKt956ix07dnD06FF8fHyKfd6uXTtMTU05cuRIwbbbt28THh5eIMPZpUsXAgMDiYmJKWhz6NAh7OzsaNq0aUGboseQ2kjHMDMzo127dsXaKJVKjhw5UkzuU1OKeniDYoNwW+JGy19blrNH+Vxdf5Xf2vxGwl01q5zVgLBjYSy2X0zgRg29CEbIzai7AFhlN6Bnz7Lb5V9+XL0KH38MK1fCvXuwcaPubLEwsQAgdEMoN7bc0N2BDUSLF1owJ2oODQc1VN/4CURaVLZ3yuG/B/8B0MdHN06/CS0mEDc3ji/7FcoESDJNDx6IRW11jSAI1KlTp0qjYtTRaWYnPkj/gFotaqlv/AQhlcEpEpcDQFRaFItPL2bBkQVcfKh+5N2lThfWjFwDwNKzS7kRo919K185u2AQXBZ52XlsHLKRiz9Xr9lAna51ePvu27ScVPExQ00lNhbSLMSFZ18XX2SCfoe7UvZ8QFRAmW3kcnB3h7qBe9n14na92iNRVffNhxcfEnEuAqVCqb5xDefuXXFCl+y5jRf3jeb/TpSs510RWol+5RJOv6auTTEVzMgmhQzz0IKI7/K4/td1tk/aTl52HsuWLmXksNp4hixkcdItfn30iHdv3eLUwoXUq12bZd9+qxP79clbd95i0oFJhjbDaEjOSiY6XXQUNXJuVOLzf/4Rg2GsrMTAnGPH4PXXS15b2tLUVZzMBMXeZP16cXEjJ0f8bP16cXHSyQk65SdD79gBfmP96Le4X+VOrEO0uWduHLKRE/93Qm+2RPpHcmTBESLOPXVoVwSFUsGUHVP45dIvjPIdhYmsWgpcGRRTK1PqdquLQz0HQ5vylHKYfn46L/33ksHOX959c9nSpdSrXZvTCxfy7q2KjzHqdK3DgCUDqNdTv0F8lSU2FqIcdgAwoskIreYfAxsOxFRmyp34O9yOu11uW09P8bVI6dkaj6WTJZMPTabnR+UsqiIGFeXliYHvnp7QoYO4/dIlUNbgaYoUvCWtf+iKNh5tGNZ4GEqVkoWnFqptL5eLga5QtlO61ZRWeLTx0KGV+ueR/yNCj4aiyC1MekhPF1+tSokFkq677dvhs8/gjz/g55/h7Fnxnln78YUyPVJtnH5vvvkmGzZsYOPGjdja2hIVFUVUVBSZmWLWkr29PS+//DKzZ8/m2LFj+Pv7M23aNLp06ULnzp0BeOaZZ2jatCmTJ0/m6tWrHDhwgI8++og333yzQHrz9ddf5969e8ydO5dbt27xyy+/sGXLFmbNmlVgy+zZs1m5ciVr167l5s2bzJgxg/T0dKZNm6Z1v4qmjdazr0d8ZjyRaZE8TKnYHVyRrSAzMZOMuIwK7W9MmFia0GBAA1z8XAxtSoW5Ei46/VzlDbC1LbudlOkXGSlOuqWMwKCg0iMkKoLk9AtbFsb5H0qvPVndsHGzwcy6jPTJJxzJ6Zdmf0HUhrdypZlrM50c29zEvMTk2dYWPPKf3cEaBOEpFUq2T9zOsU+OaXROmUyGs7NzmSn4hkJuJkcmNy6bDI1UC/nxscz2m9vJVmTTwbMDfX36anSs55s/zzcDvuHgpIP4ufppZYcU6ZaYWP4g38LeghFrRtD53c5aHf8pxktsLKRZik4/ba+biiBl+t2MvUmOIqfMdrVrg0V6PAk3Y8pso0uq6r55/H/HWdN7jVEFZRiKO/lBsCpnccGmibMGXjgNkJx+QUGQW0QEI+i6KdbpouO/Sa/LBUFk5fHI/xGBfwby1f8tYtGHH3JSoeCoMouJQD9gInAgK4uTCgWLPvjA6B1/giBgamWqvuETghSJ7W7jjp158TohqamwbZv4fsYMWLoUGjQQFy42bKjceZ9t+Cwnpp7gHc8tbNkiLm68/LIYyPhPfuLC22/DxHx12f/+A4cOjek+rzsWDhaVO7mOUHfP3PP6Hva+uReAob8NpePMjqW20wXefbx5+ezLNBmmm3vIk4RSpeTlf15m0/VNALjZuBV89njtHm0I2R/Cry1/1WuGpzGR8jAFZV4NXiWvIdh52Rl0LlrWfXPZ0qUFY4z9WZUbYzg3cqbrnK7YuBu3okR0tIpEGzHYeqTvSK32tTO3K5gfr7+2vty2ktNP00y/Wztv8UuzX7h/6r5WNlVHAgLE19atyZfaFzOxUlPFGtk1FSnI2cFBfE3KSqp0hrvEJ70+AWDDtQ0a+Sk0uT6VCmW1er6cXXqWdf3WFbNZcvpZl5J82rUr2NuDqakoz9uggbh9yxZR9tSpaEqmnqk2K5W//vorycnJ9O7dGw8Pj4J/mzdvLmizbNkyhg4dypgxY+jZsyfu7u5s314YTS2Xy9mzZw9yuZwuXbowadIkpkyZwmeffVbQxsfHh71793Lo0CFatWrF0qVLWbVqFQMHDixoM378eJYsWcInn3xC69atCQgIYP/+/bi5FQ4oNaVoUUprM+uCRflLjy5pfSwQa968G/YutTtVnedYX9TrUY8JeydUuyiAotyJFZ1+DZwalNvOwaFwgbxxY/jhB/DzE6PFdVW6UnL6ef/kzdDlQ3VzUAOhUqq4tesWqZFPpT3LQnL6+dVqxK9DfuWDHh/ofEH2cTnixo3FV02cfjK5jLATYUT6R2p0LoVCwa1bt0oU8jUUeVl5XP79MnG3a6BOXyWR5D29ipf/4NC9QwCM8h2l1bX4Xtf3GNBggNbZWvb24mBfpYLk5PLbtn6xNR5tq9ez5vyP57n9Tw2evVSCmBhIsxC1XSTJOX1S174uDhYO5CpzS5W2k6hdG+50mozdvNf1bhNU3X2z86zODPpxEILsqdNPcvqlmt8CxExTXVCnjhi9m5MjSsEDhIXBwoVgn9aeWoIfvfpoNnnu+WFPpt2dxmdf/x+7s7MpWYVSpBOwOzub+XPnGq3UZ+zNWIL3BZOdmm1oU4wGyelXmsP577/FRQofH+jVCxo1EssJQOUzBtxs3OhZrycxYWKwpiCIkef//CM+hwcMgHbtxHO3bi0G4/xTuoqZwVB3z4w4F8Gji+IqVptpbWgwoPz5XWUwszajdufaRr/IbWyoVCpm7JnB2qtrkQtytj63leFNhgMQkhBC2xVt1WbRlIWplSlpkWkkh6sZVNYQfu/yO793/d3QZjxFDbmZuYSfDic+WEeR4lpS2n0zKiqK+fPm6WSMocxTVhslidhYgZ43rrHI9yi9vXtrvf8rbV8B4Df/38jKK1v2WnKqJCcXOh7UoVKpUCmqb02Iy6suc/KLk2odRVeuiK9tRPV75HJx7AFw4YIeDTQwUqaf5PR7/+D72C6y5deLv1b62O092/NV/6/Y9fwunK2c1bZX5/QL3BTIQuuF3DtS+RIIVUX7Ge0ZsnwIppaFQYbSb+9xeU8Q52zr1sHWrfDjj2K2n7m5qAhz8aKCO3fUS6Xqimrj9FOpVKX+mzp1akEbCwsLfv75ZxISEkhPT2f79u0l6uzVq1ePffv2kZGRQWxsLEuWLMHEpHjGSu/evbly5QrZ2dncvXu32Dkk3nrrLe7fv092djbnz5+nU6eyHmfaIek+V9Tp9xTj4kGa6PRr7qV+UjhvHsyeDV9/LS6WS4VmdeX0M5eL2ayypjJqNa/ecoTxwfFsHrmZM0vOGNoUo0Vy+jX0cOP19q/zbud3dXr8oNgg6v9Qn7a/tS2Imm2UryKlidMP4O27bzNh7wSNz5mVZTw1X+JuxbF7+u5qLT+sL4rW9JPIU+ZxNPQoAAMaDKgSO+TywoGvurp+UL0izlRKFYfnHubS8qdjhdKIjYXU/Ew/SXJOnwiCQEs3MduqvLp+tWsDglDwG6kKquK+2WBAA9q90k7v56kOhIeDChVRebrN9BMEaJmv5PvppzBqFMycCRkZ8LzdLzyYH8TzzcdrdCwLBwvW/bmWPqamZS7GSXQCepuZ8fuqVZUxX28E/hnIxiEbSb7/ZCzCa4Lk9Gvs3LjY9thY2LNHfP/ii4U1dyWVhtRU3ciz38tfw5k5E959V3TyNW4M06cXthk5Unw9tDeHVV3/4PCCw5U/sY4o7575esDrvHzm5SqxIzk8uZiM1FM041jYMVZcXoFMkLF+1HrGNB1T8NnsA7MJiArg0xOfVujYdbvXZU7UHPxG6T+YyNColCqaP9+cps/pfwxV1SiUCg6EHNBJzV1jIPFuIqt7rCZgdYDBbHj8vrlq5UqdjTFCj4bylcNXBG013rqFEjExIMOEAQ37YCrXXoFghO8I6trXJS4jjr+u/1VmOyurwjmuJtl+viN9eTPoTbx7e2ttk7EQsCaAc9+dQ2ZStgsjIUGsqy0IhQoZUFjXryY7/aRMP0ne81LkJbLysopluleGud3mMqTxkIJEkvJQ5/Rz9HHEp69PMQeasVOvRz3av1a8Rmd5mX4AMpn4D8DODgYNEt9v2SKQmVl165rVxun3pNDGXQxJCIgOqPAxgrYFce3PazqyyDDE3oxl49CNhB4NNbQpFUahVJCgDAOgUyP1Tj9vb+jTR1yoBujWTXxg3bolTtQri3SDzszOJC9bN6nehsLKxYphK4fRfPzTwuJlITn9XPSkjuvt4E1kaiQPUh5wO15c3JQy/e7c0axIbdFCuNUNBx8Hnt/1/NNr8DGysgoXDYvG3Fx4eIGU7BScLJ0KnnPaEJcRx/sH36ffun5aSTNJA191Tr/AjYEssllE2PEwrW0zFNPPT6ffIuOphWRMxMRA+5AdrO1/kJ71yq/9oCskic+wpLAy23h5gUl2OkmnAokN0sGD3QiojFRaTUOlEq+9bJNo0vNSkAkyGjrpru6wFAyWkyPWKwFRtuijD4UCWXhNUOYp2fbHOiZr6BCekpnJTl0WmdYhzV9oztAVQ3FqWHUyOcZOwZjsMaffxo2iNGzz5tC2beF2C4tCOexIzcQXymR/8AG2p79HrN1BGjeGfv1E9ZKlS4vXPGnbFurWhYxcU2JCUshJK1sW2diQFhxX91zN5tGb1bSuOCvar2Btn7V6O35NZfN18TuZ1noaL7R4odhn/+v1P0CUm4/P0D4rSpAJT4ykvyATGPD1ALq9383QpuicGXtn8Oyfz9LghwY0+rERcw7M0ct5AqMD2XBtA3vu7EGp0l9QoXMTZ/p/1R+/0cbjjN62TndjDEEu4NHOAwcfBx1Zpx9UKhUxseKY2NW1YscwkZnwZoc3Afj1UvkZWpJjpbLP7erCxH8nMu1U+eW0pNrEDRqIThaJtm3FNdbwcPjpp+Iy+TWFopl+2XnZXI+5DkA7j6oPypSCycpy+tXuXJuJ+yYafY1OdZRX0680Ro0CExO4dUsgNLTqZO2fjFFLNaKNR77TLyqgwsc48ekJjn9yXDcGGYi4m3HcPXiX7JTqK9dzJyoCpZCLTGlG1+Ze6nd4DCcnaJZfgk0X2X7NazWnW51uxEyIYXnL5ZU/oAGxcrai7fS2eHXU/u/6JKBQiE6OHHk8+6N/53LkZZ2fw8rUih71xBXIAyEHAKhfX4xmSUzUrBZlRlwGN7bcMJgcSWWwsLegyfAmuDat4Ki+hiJFmZmbg6Vl4fZDd0Vpz34+/ZDL5FofVybI+OniTxwNPcrZiLMa71e0rl95OHg74N3bG7m59rYZAkEm4NbSDfdW7uob65HY9Fi+/u9r5h6ay8x9M3lt92vsvbPXoDZlZ0NKCljmejGm9QBcrKqmLvBHPT8iaV5SQd2D0vDyAvOMRFxPbufG9pohzXr7n9sscV9C8D4NU7xrMBkZYuBDmqWov+nj4IO5ibnOjt+pE/z+OyxfDqtXw6ZNojqEJCujUCo0qt+Rl5VHVFg0mt493IBkdRrJBqJWs1q0e6UdJhbVN4hI15SW6RcaCkeOiO+nTi3M8pPQtj5QWWy9uo9gl6XEOx4oUde3KIKQn+0nCDx87l0G/zi4cieuAtKi0ri5/SYpD1MAMLMx01tdcWWekjYvtaHZeN3U4n6SuBYjBj6Pb1Yy87mdZztau7cmR5HDn4F/Vuj4adFpHPvkGMH/Pn3mVUf+uPIHKy+vREDARGZCSEIIV6Ku6OVcm65vYvKOyQzbNIx+6/pxP0k/9dTkpnK6ze2GZ3tPvRxfW5YtXUpwSIjOxhj1+9Vn6vGpeLbTX//SctJYG7CWbUHbKhzMdjH8Ov/Ur8dNr3nUqoSo1vS20/lfr/+xc/zOcttJZTQ0eW4nP0jm4i8XiblRNTXF9YG5rTmufuWvu0hrpq1bF99uYyOqDQgCHDgACxZopgKkbzIzRalzXVC0pt/1mOvkKfNwsnSirn1d3ZwAuPjwIh8c+YBzEefKbSeNKaOiNEsEMHbSotL42uVrTn55stj28uQ9S8PJCZ55Rnx/5Ii9Di0sn6dOPwPzeMFbSR4qIiWiWK0sbRiyfAjjto2rtG2GxG+0HwtSFtBwkO4ipKua3Li69L/6iCHR/+HoULGFZCmq+9Spytvz9YCvOf3SadqNbkejoY0qf0ADUl103Q1FYqI4gEixO887R6czecdkvZznmfriU+tIqLiSZG4O9fIDdjSRqY65HsPW8Vu5s1t9Y5lMRv369UvcMw1FTnr1iUqvSqQBtJNT8UXFXGUuduZ2DKhfMWlPJ0snnm/+PKA+8rEommb61elah4n/TqRej+oRcZaZmGkUmRFHQ48y//B8vjnzDT9d/IkVl1cwdNNQPjvxmcEywKTMeEtLzSPvdEEt61rYW5Q/gLe1BVMvV+62fQ7H7vpfzK2K+6bMRIaDtwNWrlX4xzZSpGtPYZ8v7emiG2nPotSqJS70uLiIk0zpPvvizhexW2zH/pD9ao9hZmOGk6sTmlbpiwbs7atucqopKpUKlbIGrCbomG8GfMN3A78rKBmhUsGqVeJr9+7QpJTLUl1UtqY45olSgHkONwuUS8qic2fx+g0PL1SnMDTl3TMjzkWwZcwWQo+IKjQT901k1PpR+rHDREb/xf3pNFM35UOeJM68dIZLr1wqs57Wy21Eedbfr/xesXGKCk5+frJaSA1WhrPLzrJl7BYyEzINbYrOuBx5mTf2vgHA530+J35uPDvH72R+9/k6OX5GbgYPkh8U/N/XxZf+9ftjbWrN8bDjtFzekvVX1+ttfGyoZ6JMJsPa2pqFX35JvVq1+L/33sMbSh1jRAFfAB2AxvmvvwFWZenj6ZnA6EDe2PsGnks9mbprKmP/HsvIzSOJSdfeOfbdmZ/IMn9Als1tLCqRxONk6cSnvT/Fw7b8WvOSY0WTerzxd+LZ9+Y+wk+FV9wwA5Iem07M9RgUOWVLXt++Lcp3CgL07Vvy86FDRXl8a2ux7bvvVk2WpFIpKnQ8zt27MGmSWOutsqXXVarCTD9HRwoCGdq4t0F4PMqrEvx66VcWnV7Elhtbym3n6ipmtOXmlq1Yd/n3y+x6aZfObNMnuZm51GpWCyuX4nNddfKepTFmjJh1+uhR1SmUGMfq6RPM4z9CO3M7JrWcxJwuc1AoK/brr9utLu6tDRv9rwtMLEyqtfzf3bsCFrkedK7bXn3jMujaVXxwBQeLkRK6oP+i/gxcOlA3BzMAKpWKb72+Zde06vGQMARSll2uk6hxIMnO6ZpudUXJlwsPLxRMYCSJT03q+rm3cWf0n6M1kiMRBAE7OzudDlwqw68tfmVN7zWGNsPoeFxPXuKLvl8QPzeeya0q7oCe0X4GAFtubNE4KEbTTL/qxulFp1lku4iEEMOGKY5vPp4/RvzB2x3f5sMeH/JiqxcB+N/x/zFl5xSy86o+Wz82FqLt93DP+0NOhZ9Uv0MV41bHnETPpqSbOapvXEmq4r7ZeEhjpp+bjleHp5n30sS2ufkQtozdwrud3q3S82fkZmic2f/CzJdZr+Gq1DpLS0ZO0Lz+blWR+iiVhdYLOfH5CUObYlT0q9+Pdzq/g6etuCJ44QJcuwampjCtDGUsXWX6mSaL47lU85tq29rairWgLVOi2fvxWdJj0it3ch1Q3j3Ts70no9aPqvZyVDUdQRBo59muzHpaE1tMxFxuzrXoa/hH+mt9fBt3G169/CrDfhtWWVONmribcQTvC8bcTnfZ6oZm7qG5ZCuyGdZ4GAt6LMDO3I4RviN4psEzOjn+O/++Q6vlrQoUL6a0msKhyYcIeD2ALrW7kJKdwpSdU1h0epFOzleU639d5xuXbwj/r+qdOt99+y3N/fw4unAhkbGxHACeB9Y/1m4ZUA84DbwL/Jr/GgsEhYSw7NtvSxw77lYcW5/fSvhp3ffrWOgx2vzWhl8v/UpqTir1HetjKjPlaOhR0nO0ex7dT7rPluA/AOis0q1cbFnSsNo8tz3aeDDlyBSaDNd9MFpVcGvnLX5t8St3D90t9XOVCtatE9/36wd16pR+nLZtYdky8fPERNi9W3MbFEoFIQkhZORmaLyPSgXvvw+vvFJSBWv9etEZ6O8PGzZobkdpZGYWOhYdHOBKZKHTT5cMbiSqMuwNLl/VRy4Ht/xSgmVdnw/OPCBgTQCZicYfWOLo48jUE1O1rulXGrVqQZ8+AvIK1PysKE+dfgZGUYpbf/2o9Sx5Zkmlim7mZuYaRRZARchOycZ/hT+JodV7lTYkRHxtWIlkRQcHaCkmf+pE4rMmkJOWg1dHL+zrGV/UubEgLTym24oSN/py+rVxb4NckBOdHk1ESgQgLuCAZk4/C3sLWkxogYO3g9q2CoWCwMDAUu+ZVY1KqcKnrw91e+hOLqGmUDTT73FMZCYaFX8uiw6eHWjr0ZYcRQ7rrq7TaB/JDk0kPC6vuszuV7UY/RsQz/aetJzUEvu66u+DDx7AjRu6O3dqdiqx6YVhe1NbT+X7Qd/zRd8vWDNyDSuGrkAuyPk3+F8i06q+0ERMDEQ77MbfeiEH7x6s0nN/89839Fnbh6OhR8tsI0n+RD3Uf21dY7pvPglIz976LrV5rtlzDGhQsczmiiBN7DWVKZv+yiscy83lvJp254HjOTm8PH165QzUA3mZeXj38cahnoOhTTFacnNFSVgQa4mUJTmmK6dfzkMx0y9eGabRomnbtmATf5+w5QeJDoyu3Ml1QHn3TLvadrSc1LJgzBq4MRD/ldo7jTTh3Hfn+GvkX2TEa764+KSjUCrIUahf+3C0dGS032gAfr/8e4XO5dHGo6C2Y01l2IphzE+aX6P6+fdzf/N6u9dZN2odMqF4v84+OMuk7ZO4+PBihY69+fpmVl1ZRVJWEpamlsU+a+jUkJPTTvJJT1H+/YuTXxCZqtvxsY27DS6+LqgUVZvpt2zpUhZ9+CEnFQr6ZWfTF+gETAeOQcEYYxmwCDgJ7AcmAv3yX48Dp1QqFn3wQQnHX9TVKG5svkFGXMXuhSqVKIX+118lP/vhwg8oVAp61uvJkSlHCJkZwqVXL7F+1Hp8HH2KHEP933ThqYUoVHm4pPSjtWOPCtn6OGcfnGXA+gFl1pwsmumnzkRLJ0t8+vpg62mrE9uqGvdW7vT8uGeZiS1Xr4rBTSYmoC5GzcMDXhRjVDl3rvy/3d2Eu7x/8H16ru6J/WJ7Gv3YCLclbiw9s1QjuyMiRPWrhAT444/C7bduic4+Kb5o61Y4r25AXg5Slp+Fhai8VZDp56Fbp9+A+gMwkZlwJ/4OIQkh5bZVV3NywFcD+CD9AywdLUtvUA3QtqafxMSJClSqqvN11Jyn+FMKCP8vnIVWC7n8u+7reFUFDy88ZM9re7i185ahTakU26OWEFT7fVS1rlXqON27i69bt8KPP8KlSxUrPvv5ic/xWOrB/731f/z79r+VssmQmNua88I/L9D7096GNsVokSKJEkzFTD9JNljXWJpa0sKtBSBm+0Fxp5+m6iW5mbkaDaiNZeFakAkMXzWcvp+Xoh3xhCNl1BV1+iVlJenk2IIgFMgybbimWUiclHGoSabf/ZP3ubzqMtmpxl9Lttm4ZoxaPwq5mXrp6M8/F2sXhOsgSFalUjFu6zh6renFw5TS9WReafcK+yftZ+fzO/F28K78SbUkNhbSLMQsk6auTav03FeirnA87DjnI8qeubm5gc+V7QRO/KpKZJj0fd88+P5BbmzRoVe5GiM5/VwNUOq1rUdbAI0z/R7seMBgu6EMMzMr0/F3Hhhmbs7ir7/G3d34FEScGjoxcd9EWk3RT2BTdeTSo0tsCtxUUNdvzx5xwcXREcaOLXs/dYszmqBSQeRdF8xyxTqqt+PV1y1t2xaS3H0J7zcVj3bGkS2s6T3z3HfnOPm5frLJ427HcffAXb1kWeUocvB/5M9/4f9xNPQoJ++fLDOTpDpxPOw4bkvcmH1gttq2L7d5mdbureng1aFC51IpVTy69IiHFzTQ1avGaDLGrAynw0/T6MdGzDkwR+vMqorgaOnIr0N/xcHCocRnv1z6hT8D/+SXS79ofdzQxFBe3fMqAB/0+IC+PiXnhyYyEz7t/Snd63bn+ebP61yBwbu3Ny/99xLevb11etzyiIqKYv68eezOzqYTsAuQ9FzcgcXAMGAfMB/YjegQLI1OwO7sbObPnUtUEXmr5uObMy9xHg0GNqiQjY8ewcaN8OefEPSYIu+zDZ6lg2cHfh78M319+iIIAi3dWjLSd2RBm+NhxxmwfgDRaWUHpdxPus/qgNUANHr0P5ydK2RqCZKykjh87zDfn/++1GBCSZY7PR1SU9UfrzpLont19KLPZ32w87Ir8ZlKBWvXiu+HDNFsDN6mjegci42Fe/dKb7Pj5g7armjLkrNLOBV+ivTcdExkJqTlpBWrF1/eGlZAQOH7kyfh+nXx/Z/5JWX794fhw8X3335b8TFYUWlPgNF+oxnlO4oOnhV7xpWFvYU93euKi9P/Bpe/nqwumMzKxQpTy6rLdqsMQVuDOLXwVInyPmlp4qumNf0knJxg3Liq07V/6vQzUhIzE8tdNCoPpwZONBvfDKcGVacTq0s823syfud4fEf6GtqUCpOeDkHyjdxzXwL2lSva3L27WLslPR0OHoT/+z946SXt5eoycjOISosi92wuAWsCKmXTU4yb2FhQCJnEqsQFl1bu+lsQm9B8Am93fLsgIq5uXTAzg4wMzSLGD88/zELrhUYh66Qvjh0Tf7u6wP+RPy/vepmJ2yeyP2R/hWWg9YWUUVdU3rPTqk7U/74+AVEBlT7+uGbjMJGZcCXqCjdj1cuHSc7HxyU1SuOZpc/wQdoHmNvWHCmjnBxxAqFSwZkzlT/egbsH2B+yn3uJ94hOL3sC3L9+/4JJAUBYUliV1fiLiYFUC3Fm7+eiXjoYAGUFImlKQcqqvhp9tcw2rq6QYe+B4OtLboZuzmsospKzOLvkLMF7NUjtfgIQn71ZnGEJe+7sqdKF9NburQEITw4nPkP9DU9uKmdgnWd5a+YH9JTLGWhhwZ/AYWAD8IyFBT3lchYsXMis2eoX0Z9iHGwK3MSE7RP49eKvJCQUZjdMmSLWOS0LafEwLU2zxcPSiI+HlBSwyRLvu5o8oxs1AlNnO2Is6xERa9zP3j8H/8mvLQtrCg9fNZzndz2vl3MN/XUoC1IXIDfVrdMlMTORVstb0X5le7qv7k6/df2Y+e9MotJ0VEPCgPwd9DdJWUmkZqu/gPv69OXKa1d4qc1LFTpXbmYuf3T7g+P/O16h/Y2dlIgUrm24RsrDFL2e55NjnxCSEMK3576lxa8tNJbu15aAqAC1Y1CphMBf1/8iIVNz6XyVSsWMvTNIyU6ha52ufNr70zLbCoLA0SlH+WPEH7jbGF8gjbasWrmSPqamBY68ZERnn8QsYAEwHOhK2Q4/iU5AbzMzfl+1qth2CweLCjsHAgML32/dWvyz19q/xoVXLtC8VvNS981T5jH9n+kcCT1CuxXtOPvgbKntFp1eRK4yl8Ym/XBO66Ezp9+gRoN4uc3LqFAxcfvEEo5Hc3NxjRDUr7nE3ozlM/lnHP/0uG6MMyL++09UWLOwgOee02wfMzNo1058f7aUrzUxM5GX/nmp4He9ZsQabrxxg6Q5WRybfJoxTccUtP39yu+M3TK2QPWqKFfyxTekstjLl4vXZECAKIH5/POi7Lqfn7h2tmhRxer7SevCDg7i63td32P7+O00cm6k/cHUMLihKPG5L2Rfue3UOf1UKhWRVyJ5eNH4g2dubL7B0Y+OFhuTKRSQlSW+r0hJ0iZNqk7W9KnTzwiJTI3E6Wsnuv3Rjcxc7S8GG3cbxv41lsZDG+vBOv1j4WCB7whfHH30X+9GX9y9qyLDXNScblG7YpFJEjY2sHKlmK0xeLB4U0lKEtPYtUGS1UuYl8DsiOq7gHPisxOcWnjK0GYYNXFxkGp5AxVKXKxc8LApvxB0ZXi/2/t8P+j7giwDExNokH/J37mjfv9azWvRdExT8jL1L3WnKy7+cpHdr+3WKCMsMxO++07M0q1MBP3+kP30WduH9ivb80fAH2wM3MigPwdR/4f6bArcVPED65jH5T3Dk8O5E3+H8ORwnWR9uVi5MKXlFN7p9A5Wpuq1FCTnY1KS+sxTa1drTK2MP+IsNTKV9c+s5/rm62rbFpU1PXeucudVqVR8dPQjAN7o8EbBb14dR+4doeWvLfnw6IeVM0BD7sfGkmsaj4BAExcNa1dcmglH+kNu/gJXWliFzi0FWJTn9HNzg+j6XYjtNQYzG7MKncdYMLczZ/bD2fT5vI+hTTEKYmMh3TyEdVHvM2n7JASqrgatnbkdDZ1EPXlNJD7bTm/L61df55Ml/+N+RAQ9P/yQZb5+PG/hyXs2fnSY9SH3IyKM2uH33zf/cfIL46vbaUhuxomOttqWTfjgA3ERqUEDscZNeWizeFgWUrS8h4mYYR2coD4YwMQEWrUCVEounjBuKUvH+o44Ny5czXVr6YZHG/2Nr3Utq6hUKZm0YxK34m5hbWpNQ6eGdK7dmYOTDhbUf6yu5Cnz2HZzGyAGh6mjsllWZtZmDPpxEN0XdFffuBoSdiKMHZN38ODMA72dIyQhhGNhxxAQqG1Xm3ae7Yplz+iKa9HXaPNbG7qv7l5unekutbvQ0q0lWXlZrA1Yq/HxtwZt5cDdA5jJzVg9YjUmMpNy25dVa1IXBO8LZseUHeRmVk1A2a6NG5ksrXoD9sDj4QOzgBaIcp+aMCUzk50bNxb8/+aOm5WqX36tiODWxYtwX4tYfBOZCXsm7KGJcxMepj6k++rufHz0Y3IVhX/fxMxE1l4Vr5euuf8DSi9xUVF+GPQDzVybEZUWxaQdk0oE+5bmWMlV5BKWFFas9pyFgwV+o/xw8dP9b6wq+Hvc3+yYsqPUz6TLZfToQueaJnTuLL6WNj92tHRk7ci1zOkyh+MvHufF1i9il92UyZPkfPtuN+bMtOHLL+HAkSw+PPoh225uw/cn32KO4by8Qqfz3LliHeP79+HLL8VtAwaIkusmJjB/vigRGRoKd0svW1guj2f66ROprt+x0GPl1jfURDZ+be+1HHr/kC7N0wtDlg/hVf9Xi2W/ZxTpurbynlXNU6efgZHJSn4F7jbuuFi5oFApuBH7ZEkmqVQq0qLSDG1GpQm4nUCeibh46OPgo6a1ekxMoHVrmDEDevcWt5WVil4WktMv0zyzWhflvrzqMje3q48efpKJi4MUK3HRuZVbK51LiKijcX68gSZ1/VpOaslzfz+ntq6fTCajSZMmpd4zq5rQI6FcW3dNIwdRZCQo85M9/CtY+iUuI45Xd7/K8bDjyAU5E1tM5O2Ob+Ng4UB4cjj+kYUHzlHk8POFnw2WAfi4vOehu+JArqNXx1LldCrC7yN+57tnv6OeQz21baXBb16e+uwFpUJJ5JVIogKMO+I95UEKD/57QFqk+mdlXJGg6bt3i/9fW3bc2oF/pD82ZjYs6L5A4/3Ck8NJzUll0elFvP3v23qXcApJEbP8PK3qaeQYRqUClQKU2WBiC3Hn4Z/6EPSN1ueWMv3uxN8pcyIk1dSKjS31Y52i7/umIAjYetpqVFvySSAuDtItxAz7Ji5NqvzZq63Ep4S7uzsffvQRl24GMfbFh3ToHcRz4z4ySknPogT8EcC1DZWT0K9pXIsW/x4n/27Jw4eiI2/+/MK6MeUhZftV1OknLVSNcf2Ih7Mf8r9e/9NovzZtoMWxH7n98Z8VO7EOKe+eOfinwYzbWuhQUuYp9ZKtnRGfwdX1Vyu10F0aX5z8gn3B+7AwseD0S6cJnhnM2ZfP4mbjVtCmusp8ngg7QVxGHM6WzvTx0TwIJSY9hi03tmiUHfg47V5tR72e6seh1RGfvj489/dz1Ouhv/6tuixmcz3b8FmC3ghi+ZDlejnPl6fEFXYvWy/MTcpe/xAEoSDbb7n/co3UKVKyU3hn/zsAzO82n8bOmgfc34q7xdgtY7n06JLG+6gj8kok19ZfI+Z6jM6OWR7JKSnFMvtGAOtLaZdO8QzA8nADkpOTAfFeuGX0lgoHe6tUhU4XaTizbZsox7rq8iqu3Uni00/FjPj0MqYmvi6+XHjlAhNaTECpUvLFqS/osLIDGwNFT5OjpSNXX7/K4n6LsYkXa/np0ulnZWrFlue2YGVqxeF7h/n0+KfF7tOPO1Z23dpF3e/q4vO9D6fuF/7dbD1sGbdtHC1eaKE746qQtKg00qNLfkm5uWLtehCTI7ShQwcx2+7+/cLA7LScwrn18CbDWfLMkgJH/YUL4vlUKvHvfe4c/PSdBT+0P0Qnr06k56Yz68CsgnvHrVtiJpi9PbRoUVhHMD1dXNsdVyQ+xclJzPYDzdbPHkdy+jk4QFBsEPeT7utNYaepa1Pq2tfF1ty2QEq+NKRrMyqqcC2sKIIgMGDJALq+11UvduoSK2erEkFe0j3D3Fz8PrVBJpPRqJHuszDLPF+VnekpGiMIAm3cxaKbVyLVR+uWxpU/rvDn4D9R5BiX9Js6Eu8lstRjKUc/KqlbXZ24mD/zdTTxLFHMubJIWVTaRoFITr+8yDwe+T+qMqk1XTMzeCbjd4w3tBlGTVwcuCeOZnWfg3zU8yO9ny89J51T908Rmy6uYkvPME0y/bTBzMw4smKe2/ocsyJmIZOrf4QWze67VMF5nYuVC+enn+fjnh8T+k4oG0Zv4PtB3/No9iPWj1pPH29xgUOlUjF803De+vctPj72ccVOVkkel/c8dE90+g2oP8Ag9piaipF1UDzrrVRU8EfXPzg8/7De7aoMXh29WJC2gA5vqNfpf7zPFc32UygVBdfUu53exdVa86Jl09pMY2HfhQD8eOFHWi1vVWwiqktUKniYLQaF+LloWM9PEKDTSuh3THxvagfek6BWL63P727jjquVK0qVkusxpWdiurqCSXY6Tuf34b9WfbZmZdHnfTPxXiJJ95OqbY0QXaJUis/eNEuxHrWvS9VL1Pf17suwxsOo71hfbVuVUsXFXy6WyBiuXVt8faC/BA+d8ar/q0w6MMnQZhgN8RnxPEwVZZJyHrSgVi1YvLhwsVMdmkRll4c0L2nbsDaetp4aO73btoUEj2bE2DUoFjltKDS9Z/418i++cvpK5+ePCYxh55SdBO/TnWzy/aT7fHHyCwCWD1leIAcskafM47MTnzFq86gqmR9+dfornL925rdLv+nkeJtvbAZglO8otZlWRen6e1fGbx3PqfCKOhVUZMQbwUWrY2w9bGk6tik27loWKtKQXEUuawLWAPBK21ewNbfF2UrMos3MzeRcRCWlKfK5GXuTv2/8DaDRfHhii4nYmNlwJ/4Or+5+tdwsFgABgTF+Y2js3JgFPTQPhgNREnLbzW18cOQDrfYrj/avtef92Pfx6lA19VHt7eyKZfZNB45BiTrBpWUAlkU0YJ+frmVibsKoDaNoPa11heyLiBCdIWZmIIkWnDgBP/23mld2v8KgFVPw9xfrq02fDn//XSjXVxQ7czv+HP0nm8duxtHCkavRV5l7aG7BvbKxc2PmdZ9XQu1GVzR1bcrPg38G4ItTX7DCf0XBZ9Jz+05EPBO3T2Tk5pFEpUVhLjcnT1l9VJTUMe3ktFLHe1JAr0wGdiXL/ZWLjY3ojANR4nNb0DYa/diIE2EnSm1/SxzeM24cfPGFWIIJ4MyOluwYvxNLE0vOPzzPvyFirTupnl+rVuL0csCAwjWyZ54pWXtQ+qwiTj8p6NrREd7d/y7e33vzx5U/tD+QBgiCwKlpp4h+L7rEWKIoLi6iMywvr+xA13avtDN6dUJlnpL4O/ElMqglp19FpD0B7iTpeKG0HJ46/QyMsjS3N4W1OSpaAynudhyhR0P1rsWuD9q83IY6XesY2oxKcTNKnPl621VO2rM0ijr9tJmXSU4/pz+dWNl+ZbVzCEuYmJtgX0e7rILr18V0+ScBhUJc6DdTODKixQB6e/fW+zkH/TmInmt6FgxypEHLvXvig14dxz45xpEPj5TbRqlUEhgYWOY9syoRBAErZ83y+Is6/QIDxRprmlJUhsbD1oPP+nxGHfvCe6OlqSWTWk5iSOMhBXZNaz0NECeTu27t0vxkOiA3t3Dw7eQkRowfCRW/1wENdOv0UygVHLl3hA3XNqhtK02+1NVBlZnIGLBkAJ3eUVd1wvAIglBMYqIspMw+KWmhok6/Tdc3ERQbhIOFA3O6ztF6/wU9FvDvxH+pbVebu4l36bWmF/3W9eO/8P8qZlAZJCVBipmYadXCU4N6forswgeptFBo7wdd14FLR63PLwhCwfjtalTpEp8WFmBrL8Mt7CK3/tUyZV9L9H3fPPLBEb73/p687JqzsFBREhJEx1+6ZX6mn7OG0rI65LX2r/HPC/8wtulYtW0FmcDRj45y6dfi0Sh18h8xESVLkxgdplamONRzMLQZRoMkK2yVVR9PF1sWLRLlhDVFV06/BlpOfdzcQNm3PxGN+xarv2QIyrpnZiVncWjeIe4eKoy49Onro5esiVrNazFu+zgaDdZdFHg9h3ocmHSABd0X8GLrF0t8fjfhLgtPLeSf2//ww/kfdHZeCYVSQY6icADs6+JLQmYCr+99nW/PflupY2flZbHlxhYAXmjxglb7SkFzR0O1DzZWqVQsb7mcPwcZPkNV1+SkazFZqQAPUx/iaeuJm7UbQxsPLdgemhhKnWV1GLB+AMlZyZU+z5envkSFilG+o2jp1lJte1tzWxb3W4yAwO9Xfsf/UfkSLbbmtvw4+EcCXgsoWGfRlE97fYpckHPo3qFys2W0wcrFCiuXqtOZGzFhAustCvvtDiwGhlHc8VdWBmBprLO0ZOSECQCY2ZjRcmLLCmecSs8TX18xi6pVK1AolfxxSbTGLfYFGjQQg53S0mDdOnj7bfF9aYxrNo7rb1xnSqsp+Ln6FcsKy80Va9oCOqvpV5SprafyVf+v8HbwZkKLCQXbl8dO4EiLeizO8mFj4EZkgox53eaRND+JIY2HMO/QPFy/ceWbY9+wb+Y+rq7Xsj6QkSP9zW1tNVM0eJwuXcTXf88HM23XNKLSotgbvLfEGqtKVej0a9dOvJZmzBBlHe/dgzuX3Xmzw5uAWKtUpVIV1PNrm18NQyaDDz6AqVPFOn6PUxmnn5TpZ2+vKpD41+SeV1Hq2tdFJpTvSpLJChUk1JW4MeZklOTwZH5q8hPHPjlWbHtlnH5KpZL39r6nA+s046nTz0gpcPpFB1Ro/96f9ubDjA+rXV08pwZODF81XKcTnaomIwMiMsRJoZ+77p1+deuKURPp6drJg0mD0dgOsfRd2BdBVrWyU7og8V4iDy8+JC9L8wXGhAT4+GP48EPNHFDVnYQEcWBiYlJYzFfftPdsD8DFhxcB8QFvbS0OgDXRzg/eG8z1TfrPeNEFKREpBO8LJjNRs3qrUUVCG3Ny0HhBKzw5nCY/NeHPa9otJoxvPp53OolSM1N2TiE4Xv3IMShI+xqhpSE51UxMxOi5gKgA4jLisDWzpZOXbh1px8KO0X99f97d/26x2gqlIWUdqs30Azq+2ZFGg4z7+RNyIITQo5pFMcTHi6/SpOb69bIns+UhOfTndp1bYZnWZxs+y/UZ1wuK0h8NPUqusvC7Oxp6lKk7p/Jv8L8VHvzHxUHTB0sZ/eAeszq/o36HwE/hQAfILGM2EnceMrWTe23l1opa1rXIyislXDgfl9oWXO03iwbvDNPq2MaG3xg/enzUA1NL46+FqW+k8ViWtZhpaginn7ZM3DeREatHFNsmOf3Cww1gkBakRqYSeSVSq/FgTUcKNLDLbMXbbxdKCWuK5PSrSP3h1NTCIJP69eGzE58xevNowpLCNNpfWhS7rJ0ybZWR+jCVM1+fIfxU4Q+jy+wuJX4/usDKxQq/UX44NdRtukgfnz4s7Lew1M+auDRhyTNLAHj/0Pucj3g8V6finLx/kvYr27P49OKCbcOaDGNEE/FvN+fgnIIsxIqw985ekrOTqW1XW+tAx74+fYGKOf0EQaDBsw2o26OuUS9YaktuZi6L7Rez+9XdejuHt4M3l1+7TMDrAcVq3Hk7eONm40ZaThqrA1ZX6hy3426z6bpY81wb1Zs3O77JocmHWPLMEnrU61Fqm5TslGJS9RVRdfJx9GFgw4EArL+qqUtMPSkPU7i185bOjlcew0eM4Gh2djEH3yxgAdATGAj8CTQGjlIyA/BxzgPHc3J4ebpYAVAbFYndu2HmTHj4sHCbNOeWsrnGjoUEm/9IEsIwUdjSz2sEixbBzz/DnDmisy4yUvx/WT9pT1tP1o5cy6HJh7A1ty3Y/vgcWB/M7TaXkJkh2JkXprSlEUWmeTi5Qiq+Lr6ceekMi/svLlj3y1ZkE5cRR1JmEhd/usi9Q/oNNtQHqY9S8V/pT3xwfInPJKeftll+Ep06gULIZAtjSc1JpaV9D2L/WsiMGcWzPmNjxe9YLoeGDQvPOWaM+H7dOpjVcS7Wptb4R/qzOeCfAudd69aFx3FxEfexKCVGQHL6PXhQesZpeUhOvzyrh8RlxCEX5LRw07+Uq0qlKvf5py6YLPJyJN/V+65EAKIxYWJhQrd53ajfv7iKiuT0q8jv/fC9w1yIvaAD6zTjqdPPSJHkPa9GXa1QbSZTS9Nq6dSpCdy/DxlmYQD4uene6WdiIjr+AEJCNN+vlnUtWrm1wrGfIz0W9EBuqj5LxNjwX+nPqo6rSLqfpPE+ERGF9by0+XtVV6KiIMskirAGC9h2c2uVnLOjl5gRc+GR+PAShMK6fkFB6vefuH8iM4Nn6ss8nRK8L5iNQzZqXNxeWjyTJCY1qeunVCl5YdsL3E++zzdnvlHr1HqcbwZ8Q7c63UjJTmHopqHlZlMFBsKCBfDJJ5Wr9wbFpSUEobCeXx+fPjovXN/buzdu1m7EZ8Zz4O6BcttqmulXFGOWKzw89zD/TP9Ho7bSd9q8OdSrJ2YCV0RmdsOoDeydsJe3O72t/c5FsLewZ9XwVVx9/SqrR6wuFoW44doG1l5dy+CNg/npwk8VOn5sLAjI8HHwKZYVWyaCTJzZW5SSDhN7Bg52hqDFJT8rh4X9FhL9XjQzO5V9T6vlJpBraUdMbPUepzV7rhl9P+9raDOMgthYUAq5JJqJNdXKk7zRJyqVioiUCOIzSi6OPE7tzrVLBAdKTj9jl/e8seUGK9quIOJcNUhJrCIuPih0+jXWRi1JmQshKwoWZ1xz96MK36bVuaUsPw8PMfJ9281t7Li1o0yZ48dpaBNFff+/CdpjnAuSTo2cmBkyUyNZ7cqiVOguM/unCz9xO+62Rm3f7PAmY/zGkKvM5bm/nyMuo3KDwuy8bCZun0ivNb0IiApghf+KAgULmSBjx/gdfNb7MwA+PvYxC0+V7pBUR3vP9nzS8xNmd56tNvPgcaT6fwFRARrdMx/nmW+eYeDSgVVev1Wf5KTm0Pz55nh28NT7udxtimsPC4LA2x3FceaPF36sVH3yT45/glKlZHiT4QX1bjWlX/1+zO4yu+D/j1If8c9tcdydnZfNyL9G0m9dv4KyFhVlSsspAKy7tk5n9TRPfHaCzaM2kxKhX7WvZUuX0qFtWxrIZAyBEo6/+4iOv0WIsp/21tYMlsvLdPydB4aZm7P4668L6glvGr6JX5r/otapnpUFGzZAWBiszvcVF63n1zJ/qtGqFaT6iA7W5vKxfPmpFZaWYkZS795igLhcDqdPw1Et4wCKSnvq83YglxVfv1s+/Ge63zxPjxuXOT7uGp1qFw+ytTETPRJppDEnag5DfhmiP+P0RFRAFHte3cP9kyUjySWFIVvbEh9phLMzZDZbSYrVNSzy3Kh1cjNxMSY8fFg8COl2/mO0fn1RLlZi+HDxO4+JAf9Trrzd6W3R8XcnEpVKHFNrmvnp6Cg6BVUq7dcspTWOCIWY5efn6qd19rG2vH/wfep+V7fcdRhJXj6qjPhZG3cbLBwtMLHUsiheFWLraUv/xf1pOLBhse2S08+qAsnVfwToR3q1LJ46/YyUxs6NsTSxJD03nbuJWhZvQ1ywDDsepvHCtDGQnZLNqs6r8F+hwaq4EfPgAbS8v4J3FBG81u41vZxDks65p8XceFCjQQS8HsDPQ37Wi01VQeOhjem7sK9Wck7R0YXvr13TvU3GRnQ0JFtfJMB2MZ+d+KxKztnBU1wECYgKKJDvkQbYmmSQWbtaa1Qfzxjw6evDkOVDNK6XIDn9nnlGfNXE4bL80nLOPDiDjZkNu57fpbXDzFRuyt/P/Y2nrSd34u/QfXV3fr5Q8ncfFwdffSVK0imVmjkky+PxWgbd63bnjfZv8Hyz5yt34FIwkZnwQnNRxunPwPKzISV7NMn0SwxN5Lt633H80+OVtFB/DFw2kIHLBmrUVsr0c3YWoxmhYhKfgiAwuNFgrM0qKFz/GC3dWjK19VScLAszGV5p+wrDGouZb5IsrLZI/XVx0XCHVl/CwAui8+9xXLpA0/nQ8FWtbNDk91qrFphkpfHgZCjKPMNLFj+l8sTGQqrFDRRCDvbm9hrV1dMHk3ZMos6yOmwM3Ki2rUqlIj0mvVi2nFTTLzm5cDHFGKndqTZ9Pu9DreZaprPVYCZ6fEmH4H9oazYeS20ST8K3wr21uNtFIAjQ1nUbnJkMOUkaH+JxaU8p01VTh5N37VwcI4NIvx+nlYpJVSE3lePUwKlYjbOAtQHsmrarRJ2XyrJx8Ea+8/6u0sfZe2cvM/+dSfuV7XmY8lBte0EQ+GPEHzRyasSDlAdM3D6xUk6XpWeXFsjNzWg/g4DXAzA3MS92vo97fcw3A74BxOzQ6LTosg5XJvUc6vF/ff6PWV1mab2vu407TV2bokLFiful13F60rCuZc3oDaNp90o7vRz/0qNLpGSX7ZCa1HISDhYO3Eu8x97gvRU6R0ZuBkGxQQgIfN7n84qaCoi1Uvus7cPozaPZFLiJqbumcizsGDdibxTUUK0ow5sMx97cnvDkcE7eP1mpY0m0frE1I1aPwMxWf/Wcly1dyqIPP+SkQsFNhYIPKZ7Zdzj/3xETE4Llcv5v6VJi09L46Ouv6SmXM9DCoqDdBmCAuQU95XIWLFzIrNmFzlYHHwdcmriodaofP05BPdjz50UHzYMH4jjGzKwwg0qhyuOR/XYAvpowkcfLtzZqBPnKoixfrl3WuzTH1Ie0Z3m0cPfD17Yj9pltiI0uOf8ocPrlpmHjZoOZjf6uC33h1dGLCfsm0OCZkskUlXX6KZQKbtqJktaNHn6Kg9yjYBxzoUgiliTt6ftYuW4LC3ghX1X6r7/gjVbzCH0nFO+41wFo00Y7eyoi8alSFWb6hWaKTj8pgUifxGbEEpESwan7ZdfElebj8WXE1Nh62vJ6wOu0maZ/e3VNReU9cxW5agPWdU31WGWtwchkpX8Fcpmcj3t+zK9Dfi22KKYxAmwcspET/1d9BrBxt+NICkuq9hIZERFipkGzOl64Wruq36EC1M9fS9LG6Sdx/sfzrOq8irToCmi8GZi63erSY0EPTCw0jwYpGlnypDj9UqxET1sr91ZVcs76jvVxsnQiR5HDtej8LIfW4mfXronZReWRm5nL/VP3ibtddlSxTCajRYsWZd4zqwqnhk60f6091rXUP+FzcwszrQYOFLN0IyPLr5cTkRLB/MPzAVjUbxH1HCpWx8DD1oPLr17mlbavYGNmw/Amw0vYtmiROCGS5wcNXrxYoVMVUDTTD6Bb3W78PORnrWusaMpzzZ4DxIzC8p4b2sh72nnZYelsiaWT9lI9VYVPXx98R/iqb0hxJ1jnzuJ7f3/Naktm5mby/sH3ScjU4A+nA7rU6cLcbnMBuPCwYpIXZyPO4l9/HEGWK9Q3lpCVkfUuCNB6Edg3rZAtUHaNglq1wOPuaeKWrtNr7WV93jfDT4fzW9vfCP63AsUnaiCxsWCb1YzPva6yccxGg2V+NHISVwwuR6nXSbzw4wWWuC0h4nxhtpylZeEk3Zjr+tXuXJueH/Ws0vpFxk5SuBduycPo1ECz50MBMjPIS8XMwgJnZ7jwaBgR3pvBRHPNIql2jTQ/8XURbbgdr5nTr353LxLf+JBYn44VqmejK8q6Z2bEZZDyMKVYFt6DMw8IWBNAXqZuJWbdWrtRt1vdSh0jIiWCF3eKtfumtpqKl51mgWp25nZsH78dK1MrDt49WOHgwai0KBadXgTA6hGr+WXIL7hYlR6NM6fLHDrX7ky2IpuVl1dW6HyVoa93xSU+Afa+uZc9M/bo0qQai1KlZPTm0Xgs9ShznGdtZs0rbV8B4Pvz31foPFamVgS8FsDxqccrXdfK0dKRbnW6oVApmLB9An9d/wsTmQnbx22vdEa/pakl45qNA2Dt1bWVOpZEna51aD21NRb2+snyiYqKYv68eezOzkbKKSua2fcd8AZibb+TCgUXL18ucOTNmj2b+xER9PzwQ77z82OitSfv2fhRf8qH3I+IKObwAxj842DGbRtXrj0qFezN9w3b24uv69cXrvk0bQqm+b6wk/dPkpAVj7OlM30b9Cr1eGPHiuooWVnwzTeal4aR5ltOFVi2rSzlSXMXdfrF3Y4j8Z4WsjdGgpWLFY0GNcK+jn2Jzyor77kveB/ROXexUDnwYpvJ/PYbvPSS+NmlS2JQNBRm+jUpRbl/wADw8hJtee8te/Zvdy3IEiwq7akJknSoNpl+mZmF8/rbyVXn9OtRV5Q/PhledsCCNJ+orJqUITnx+Qk2Dt1IbkbxAK+KynvmKfP4rM9nBYHOVcFTp58Rs6DHAl5v/3qZg+TyEASBIcuH0H1Bdz1Yph+8Ongx++FsWk9tbWhTKoUkiSRFS+sDKQLlrvZJoGQlZpH6MBVFTsWjN6sTRTP9bt4UnR01magoSLHMd/q5VY3TTxCEgmw/qa5f/fpi1FVmpvpopeTwZNb0XMOV36+U2y5HE0+FntFGdikmJl850EKUN2ia7zsoK9tPpVLx1r63SM1JpXPtzsxoP6NStrrZuLFi2ApC3wktJnW4NmAdy35N4s4dcaAyV/SzcPVq5X4fj2f66Zv2nu2xMrUiPjOem3E3y2ynTaaf3EzOa5dfo/O7nXVkpW5RKpQaB8YolcUjTxs2FF+zsjTLwF1wZAFLzi5h4IaBVRaM08a9DTJBRmRapEaZCY9zLeEMkU5/E6I6VH5DlRKO9IVby9QfVJkLcecgT7M6ngCz9s+izrI67Lq9q9TP3dwg0d2PjJ4D9R51q6/7ZmZiJpnxmU+l5POJiwOZypR2Xi0Z3GiwweyQJvpXIst/ngJ4dvCk3WvtigU5REVFERb6OReP+/FsDy86+PnxxeefE1WWNs9TjAZprCVFimtM3TEw+BpYuODpCRceDic4YxjINAuwCwwUnylyOfTIL4FVkOmnodNPZiKjoa94PkNL8Zd2zzz/43mW1V5G3K3Clatnlz3LgtQFWDjqdnF9wFcDGP3n6ArvH5YURp+1fYjPjKetR9uCWn2a0rxWc34b+hsWJhZ4O3hXyIaPjn5EWk4aHb06MqnlpHLbCoLAF32+YPWI1QWBP5ry4ZEP2XFzR4HKSEWQ6vpVVGEg6koUD89XLuPLmDgw+wCHFxzWy7GPhR7jQcoDzORm5Trj3uzwJjJBxtHQo/g/qpgMiVwmp2e9nhU1tQCZIGPV8FW81Pqlgm2rR6xmQIMBlT42wIutXqSBYwOauTbTyfEk9FXvdtXKlfQxNaXTY9vdgQ+Bi8Ad4DrQz8KC3f8UL0fg7u7Ohx99xMWgIOZ99pAOvYMY8MxHBZKe2hIUJMp6mpnBZ5+JAbZXr8Ku/OG3VM8PREUiAYGRviMxKeP5JpOJ9f2srcVn6nkNy5tW9Ry4KJLT72Ept6ECp19OGqs6rdJrrU59ochRlDkPrazT778HYgmUt7u9ypyZ1jg5ies11tZiYPTt26JDTVpzfTzTD8Sxz5w5ouMvPR22bBHXgBIcDrEm+h2t5tCSNLs2wU9S0LWlJVyNyXf6eVSB0y+/5umFhxfKrGOvidPv/sn7/Pv2v2TEZejaRJ2QdC+JB/89KCFBWtFMP0tTS2Z2nMnKwVUX5PTU6WdglEr9yTq1mtwK797eeju+PpDJZZiYG6+mrybciAzhUv2x7EnTn7Sij4+YhJCQoHmdqmvR12j0YyM+qP0Bsx7MKjVaxpjJzczlu3rfceIz7bJXizr9cnIKI3VqKmKmnxjeVtnoRm14vK6fTFYo8RkQUP6+Tg2cGPDNAJo+V3ZGjVKp5Pbt23q9Z6ojPTadRTaLNJZ+lDL63N3F32v79uL/y5LR3H5zO7tu78JUZsqqYatK6PZXFClwJCcHPv1rJ1N3vcj/RbUlxeoK778PXbqIk5SsLLhxo+LnkSY8jo5wIOQAp+6f0roeoTaYyc3oUrsLQLmyOG755drKy7CsLlxde5XF9osJOx6mtm1Skuj4k8nAwUG8BqVsv7Nny9/3WOixggjrz3p/VmVZS9Zm1jSv1RyAi4+0Tz0NzQoAoJmzmoCHrGhICoT0kvUhSnD7ezjYBeLOaGxHUnYSESkRXI0q3bvq6gppzvV44NUZK2f9ZSrp877ZZFgT3r3/bokaB/omPSe9QrWX9I0kSeiqH4EHjZFqF92IvVFQP6ss6nSpw9DlQ3FrId4kly1dSr3atZGfWcjStFv8Ef+Id2/d4tTChdSrXZtl336rd/s1ZdOwTWyftN3QZhgNJ8JOsi3+/4i3OaWd009R/BqRFg8fPQJSgiEtrNzdVSpYm5+gMnBgYf2WJi6i0+9W3C2NTfEyjcE+6rZBnX5l3TNrd65Nx5kdsfMqXFk0tTLFzMbMqOq5XY+5Trc/uhGSEEI9+3psfW5rMUlNTZnUchLBM4OZ1maa1vs+THlYkLW0bOAyjers9avfj6mtp2Im1zwI5l7iPRaeXsjYv8dWqv5gH58+rB25lkOT1QQLlcGUI1N47bJ+ynkYgls7b3H/hAZjowogXRfjm40vt95UPYd6PN/8eQQEjocd1/j4ubkqPvl7Iw9jSgZpRUTAvn1w6BCcOiVmJ2sa6CgTZKwcvpJfBv/CzvE71TqytaFrna4Ezwzmva7v6eyYe9/cy9fOX5OXrXvH366NG5mcVfoC/+NMycxk58aypcbLc1aFHgvlwJwDJIaWv9C1b5/42ru3GHA8ML/6gRSnVNTpN7vLbCLnRPJxz4/LPaaLC3TtKr7XtL6xsTr9rE1Fj0RaThrd53en5eSqWxvSFVvHb+Urx69QKUs6zyor77m4/2Kuz7heTB7axATa5asbX7ggKqvl5YmZpLXKUJRv1Ah++QUWLBDfZ5pGcL7hEH7x/4E1AWs0tkfK9IuM1FxiX5L2tHdQsbj/Yt7v+n6V1BVv5NQIN2s3chQ5BUH/jyM5/RISxPFiaURejuTCjxeIvqa9vHdVMGL1COYmzC0x1qtMTT+lUklwFcpaPHX6GTEqlYqbsTf59uy3lYpgqw5ymXf23OHIh0dIj0k3tCmVIicHgrPOEOW0jctJFZs8aIKFhRhNAppLfCpVSkISQghLCtObXfokKzELK1crZCba3bYkp5+Hh/gqFXWuqTyITifdXHyIVFWmH8BI35F8/+z3vNPpnYJtkqSBuqwimYmMru911bhOnqHIScuhXq962NfTzGEuTTikwbjk9AsMhOxS1mLvxN9BJsiY330+zWrpNuLz6lWYOhWO7PDCMtubDPNQbrQaho9fIoJQOLjVpOZgWUgBCE5OMPfwXHqu6cm2m9sqbXt5SNISlx6VbXjRGlVpGqgax92OY9/MfYT/F64LE3WKhaMFXh28sPVSP7uRouqcnETHHxROYs+dK1+y5pPjnwDwattXGdRoUGVM1prHs4a14ZEyAIB2Xq3Lb2jpAaNjoPVX6g/qORia/w+svTW2Q7r3BkQHlPq5NGlMTRWzoZ9SNiqVipX+Kxm+aTgNfmiAzSIbXL5xocfqHhwLPVbp46emiioAlR0qR8fmcbXeK+yL+0mts02f1LarjbOlM3nKPK7HXNd4v6I1eg7nZjER6AdMBA5kZXFSoWDRBx8YjeMvOyWbnFTDZ/8bC1sC9nDd5VOinP/Cx0fDnTIewXZ3uFNY81car2TF3IY9jbmzcxHvvFN2wNyFC+Jn5uYwfnzh9sbOYrh6THoMSVlJGpmT/M8JGl76i7u3civ9e9Q1jQY1YtAPg7BwKHRUpEam8ujSI51m1CSEJLBnxh4enNFwpbkIN2Ju0HN1Tx6lPqKZazP+e+k/fBw1vRhKUtuuYpI1XnZeXHrlEov6LaJrna5a75+nzNOoluCGaxsA6OfTD09bT63PI+Fg4cCUVlMq3F9TS+3qbhs7b999mwl7J+j8uGk5aQVzghdbvai2/Vf9v+L89PPM6TpH43N8vOYQnwdNpMn3zTl0NAeVSiwxsWULzJwJv/4KP/wAX38Nn3wCq1Zpbr9MkDGjwwxG+I7QfCcNEARB54EDLk1caPhsQ7KSNHPOaUNySgqa5uS5AcnJyWV+Lq1ller0OxLKuW/PkZNW9nM+MRHO5MfjDRkivo4bR0GtPnPzkpnvbjZuGpXOkAJYNK3rZ6iafgB18sV8wkuZtrrZuNHWoy2NnRrTfX53Wr/Yukpt0wUe7TxoNKhRqcoikmOsopl+AM1qNcPdpvhV3VGMZefCheL1/Mr7qcpk4jx76VJY+2Nt/tfz/wB469+3NK5vbGNTuGapaQCU5PRzdBB4vvnzfD3gaxwsHDTbuRIIglCQ7VdW8LUUdJyXV2jn47Sc1JJ3wt4x6mSl0u7R0pqSNvKe4cnhrLq8ikepVRuJ/tTpZ8SoUNFnbR/mHJzD6fDTWu9/a+ctvnL6iuC9xl9v5caWG/z31X/VXibq0SNItBYznbrU7ajXc0l1MzSV+JQi6qwirLi6/ioZ8caZQl0Wtp62vHrpVXp80EPjfXJyCh0R/fuLrzW5rl9ODoTnXAFBhYe1J242blV27rYebXm709vFIota5fscb90Ss8jUoVKpjDpIwdHHkUn7J2lcbFiaKEgTh9q1xcX+3Fz4/ffCCCGJBT0WcOmVS3zQ4wMdWi2ydq04MG5s04GV7a7QyLEx8bkPeevft4BCh2RlnH7ShEdlHV1Q27GfT7/KmK2Wl9u+zI03brByWNkSCZaWhZGXpU0sHyc7JZuLP13UW6RzZfAb5ceUI1NwbqR+VinVlyg6AW3eXIxUTE2F62X4Ay49usTp8NOYykz5X+//6cBq7Zjedjqbx27mjQ5vaLVfRk4WyabizKxr/dbqdxAEkGuQAWHfFFp+CrYlC8iXheT0KyvTz8oKbCwV+J1awT8z9ml8XGMiYG0AN3eULaurK9ZfW8+re15l953d3EssjHI6HX4ahapiMuWZuZliJmZIDK+9k8R7c/P45BPNF3ceJysLInNv8cB1Fd9cXoCp3HCLwIIgFGT7XY5UX9dv/6z9rJu8rkSNnsfpBOzOzmb+3LlGIfU59cRUnt/1vKHNMBouPhDvNfWtWxXUL1JLVhTYeINVobNDcvqdv9mEw4/msfroBO7dgy+/LCmRrVQWZvmNGFE8w8HO3A4PGw8sTSwJT9YsgKb7rPbc7zCW1DSBmBgN+2BALv58kZUdVpIUlqSzY0YHRuO/3J+Eu9rX0m3o1JB2nu3oWqcrJ6ed1LiOX3moVCr2Be9jyRntJEJbubdifvf5xbYlJakfg627uo7GPzZma9DWctslZCYU1P+b3HKyVrbpGkWugps7bhJywMC6tDpCEAQsHXVf13pb0DYycjNo5NSIzrXVS+jXtqtNB68OGh8/T6FgRdj7ANRKHM4Py8xYvBjef1+s8ZaXB35+4nzHz0/c5+BBzRWT9E12XjbbgrYRmVrBgUgROr3diXHbxmHjpmWxKQ2wt7ND0xFANGBvX3agbHlOv96f9mZm8ExcfMsuc3TggPi9+voWros5OcGw/FJZzZuLWVuA1sFY0txd0+GOIWv6eXuLr5GRJYOKe9brif+r/qwcXvX1UnVFz496MmbTmFI/k+Q9tc30S8pKKve31q6d6MQLD4eT+f6s0qQ9S0MQxGv7g15z6evTl4zcDEZtHsX+kP0oVeqVVyRHtaaJYNI9zNFRs/a6RAq+PhV+qtTPTUwK7SpL4tPKxQqHeg5G6QfIScshYG0AsTdjS3yWkb+Uro2857agbbyy+xVe3KU+8EWXPHX6GTEyQVYQYb/3zl6t97fxsMGthRtyc91IxOmTkWtG8trl17By0Z/MVVXw4AEk5Tv9Onnp1+kn1fXTNNPPPH9xs3ZQbXZO2VktC/lqi5TlZ2EB3bqJ72/dKix2W9OIjoZkK3GRr1Nt/V5/muDuLjq58vLUy0b6r/TnG5dvyk3tl8uN/15WFGkBWYrYEgQYOlR8/++/8Npr4oSzqIpUG4825UreVISkpMKB4zffwMQxDqwfvQ65IGdj4Ea23NhCq1aiJv3DhxVf+JYGnUGZYi2QNu5tcLXWr9ZdbbvaNHVtqjZKVsr2i4hQf0z3Vu68fe/talUTtzSkwXVRp59MJsq5ApwuI5ZIkvUc33x8paLnNWXHDjEqUrovd67dmXHNxmm9YHn+XhAqIQ/TPCea1VGzb/hWiDyonaEqlVgLUANauYtOv9CkUFKyU0ptU8tDjqBSkqnn+Bt93TePfniU/xb/p5djS2TkZjDnoBjl/1q71zj24jFi3oshYlYEPzz7A328+xS0XXhqIW/te4tT909x5sEZzkWc425CyaioTYGb8FjqQZ1ldWj9pxub6jlyuJUXR0JO8dZbYkZAeVmwpREbCynW4rO3jUcbjeTs9IlU108Tp1/kpUj+3r+l1Bo9j9MJ6G1mxu/apEc8pUq4nSQ6/dp4aqHw4NQWnr0MXsMLNklOv6go+P7EYu5n9cLTU3y+L1xYXA7v2DFx3mNjA6NLKUEX8HoAaR+kaSw132iAD47dm6GSm2hVz0bXlHbP3D5pOwdmHyi2rcEzDei3uJ9O566+I3yZEzUH35Eari4WwdzEnB3jd3Bw0kGcLEtfeVapxFpDmqgeAFyJusKQjUP44MgHPEguP/swT5lXLDDj8fN+8AG8+Wb52QuhiaGEJoXyzZlvylwcVaqUTN4xmYiUCHwcfBjtV/H6hxLJWcksPbOUqTunar2vIAjsmLSDM99oLgFurDy69Ih7R+7pRRZSkvac0mqK1pltIQkhrA1YW26bRfvXkGh2DVOFA3M7f4RcLmaBBQeLi7KzZ8NXX8H//ie++vqKz/rHSs4ZjJGbRzL277HsuLXD0KaUy4gJE1hvodk8dZ2lJSMnlJ01Kj1vUlJKShnKTGQ4NXRCblr6GDY9XZxLQ2GWn8SECfDqq/D66+L/VSoVTX9pSp+1fQhNDNXIdmnurqnTz5Dyng4OYkCnSlV6tp/Erpd21ThZ9IrKe/584WfqfVePz098XurnNjZibT8oXEPR1OknIZfJWT9qPa5WrtyMu8mgPwfR9OemHLlXfv1YbZ1+UgbdHYs/ORZ6jMzcqpOQ6VWvF23c29Des32ZbaSyB/HlVGZIi07jkb/x1WFJuJvArqm7uLGl5EJmRWr67QneA8DQRkOrdF3zqdPPwKj7soc0Ep9ie4O1d/rV7lSbqSem0mCA5tHphkKQCbi1rLqsJH0RGp5NimUAUFjjTF9ITj9tM/2uN7nO+J3jcWpogFFJJbiz5w7nfzhPTrrmHjspUtjdXRxYOjuLA3wpTb+mERUF3jEzmZJ4j4X9Flb5+W/H3WZNwBouPBQd34KgucSndS1rXPxcUOSUnrkhl8tp0aKFQR1/AWsCOPrxUY0nw487/QBGjYJPPxUjwJKT4ccf4YtVAVrVvdGWy/nrvvXrF05GOtXuVJBROGPvDJKVj2iWryhakWw/haJw0HkxQZQ2HlBfN0XudYE2Tj+5mRxHH0ejqtMD4qT1n1f+IXCjZhrF0uDa5bEg2e75vsyzZ8XvrSiRqZFsvr4ZoJhUr76Ij4fVq+H4cfivkv6js6HiTcYltzVyuZrv7sr7cFlzySjCt8F2N4jVzEgnSyfq2Il6O1LW6+O4uUFQz9dxfXGw5nZoiT7vm8/vfJ5nv39W58ctipWpFf9O/JdJLSfx0+Cf6O3dG1drV7zsvJjZaWZB3dPkrGQWn17Mzxd/pueannT7oxtdfu9Cwx8b0ndtX/YFi9mUCZkJvLHvDZKzkxFUhX+THNMYLjcZSqw8gPXrYWv5SSYliI2FJCuxWKuUZWdInm34LO92elejxfApR6YQ5XJXZzV6qoLUyFQu/nKx1Mjb6kCuIpcbMTc4EHKATYGbKn286LRoUpTRoBLo0aS5djsLQjG9Knd3CjIFW7USxyifzw3G2lqU8fztN/E5umqV+B7guedKX/SoZV1Lawd4gwaASmmwun5l3TMfXnhYIiitXs96dJ/XXadOP0EmYONmg7mt5nX4AqMDC1QybMxssDYr/mXk5Iiyhu+8A2PHwssvw+TJ4oK5OnGNth5t6ePdh1xlrtpsv7UBa2nyUxM+OvpRic/Cw0UHsUIhPvPLOu+bHd/E0sQS/0h/+q/rT0RKyUHblye/ZF/wPixMLNg+fnuJ/lYEuUzO/CPzWXt1rcZOAQmZiYwxm8bw7Hf6fR5WBee+O8f6/ut1KlkLosyvJP9WXmZmfHxJh3RIQgjNfmnG9N3TuRlburpAanYq3/h/CMBg6094fYozS5aIGX1du8JPP0GfPoW3OkEQfwsg/g4yjED8qGfdngAcuHtATUvNuPjrRbZP1L2DZ/orr3AsN5fzatqdB47n5PDy9OlltrGwKJyXFq27rshVcP/UfTITS3de5OTAF1+IjjYXl8LAbgkzMzHbT8rWuxx5mXuJ97jw8ILGKkjSvgkJpZfkKEp2dqEDwBBOP6BA1jssrOw2ifcSSbxbvYL+s5Kz2PXyrjKVRaRMP23kPdNz0vnu/HfkKnPLlcDuWGQpVyYrrLenDZ62nvi/6s+szrOwNbPldvxttdKbFXH6qVCwOe0N+q7rq9c1pcdp5d6Ky69d5ou+X5TZRgo+ji1nyL5lzBY2PLPB6BS/HOo5MH7neJo9V7LsjvSs0tTpl5yVXPAcHO47nGbNSh5TXzx1+hkYdRf2gPoDMJGZcDv+dqkRyzWBK6uvEHml8lIGxsDFB1dRyXKxk7vg7eCt13NJMgZRUSVlAktDcvrFOcXRYGgDvUh36JNrG66x/539yOSa37akTD83N3GA3zI/2NgQEp9hYbBrl34nFtHRICDQpJYPfq5++jtRGSy/tJxpu6YVW8SSJD4DAsrf13eELy+dfqnMun4qlYqUlBSDDgZubL7B2SVnkZupX0BXKkvWk5Ro106cgEr1b34Lm4Xfz378fvl3HVss4i+uRRdIeEp83PNj2nq0JSEzgb139lZK4jM5OX8RR1BxIiLf6degapx+5yPO88K2F5h3aF6ZbbRx+gFkJWVx99BdspJ1Xw+joqTHpHNl1RXun9RMdrQ0eU8Q5W5sbcWJ0uMSnzJBxhsd3mBQw0HlRu3piiNHChf/Dh8u3H7x4UUWnVpUZo2A0niQEINMaYaXSWv1jbtuhHbLNDfU0kOUwFNoHj0pZfsFRAWU+rlU1y+67OTmSqPP+6Zne09qd65YDSRtaO/ZnvWj1mMiMymzjb2FPf+88A8D6g+goVNDGjg2wMfBBxOZCcfCjvHThf9n77rDo6je7pndzWbTey8E0gi9996bdLCACIoVBOGHCopdUVEUxYKAiiCI9CJFOoTeQguBJKT33uvuzvfHzc1ski0z27L4cZ6HJyE7s3uTnb1z73vec84PAAgZu2bYBrTLfx+jr1dhQaESGfNLMaDFAIilcvQfSeSxQufA3FygpE5l39Wnq36/qBExuOVgfDvqW15zsFgqNmpGjzmQfTsbh+YdQkqk5eWu6kJJdQm6ruuKdj+3w6gto/Dc3ud42T1pQ1QmaXiwqw5Bh9Y8qw83lwFnJgCKhvc4Kytg2TKiivnkE8At4yN4Xg7D8oVxYBhip/bqq2Q9W1lJimDUwcBQsEoWZSvWIPja9mYj/TTNma/Hvo5nj5neRjL7TjYK4vlbe15IvYDOv3TGjN0zUKNQ3xR55gxw6BBxhqmpIcVLuRz46Sdg9WrdBW3aILb+xnrklKv3Xa2srcQHpz+AXClXqzJUnVNv3+aa0RrD3dYdv0/4HXZWdjiVdAodfu6Av+/+jfIastFNK0nDp5GkuPjz2J8bRAoYAnupff3cfTHtouDzw8eHw7Odp1HG0pzouaAnxq0bB5mTcR1HPO08kfRGEjZP2qwxTy0hgaiz/ve/ho48Ia4hGBk8EnKlHNN2TENhZVPSYsXZL1HKZsO2KgTLR8wj54WQ7L5ly5o2vwGkoO/vT2oo/xqHZzMII0NGAgBOJp7U+FkWgoxrGYjeEW30OBdvb2988eWXeMLaWiPxdxnAE9bW+GLlSnh7a19dUItPSvopFMDKpQXYOGAjzqlxk1AqiTvI3bvEKv/996HT0ppmSY4JHQNbK35NGvb2XDFf1zqdqvysrcmYmgMt6j5WjUm/vIo8BH8fDN9Vvnju1HN44eILZh+bIShJK8HN324iK6qp5FKp5GqgQki/tdfWkr+LSzCeaqfZJl6V9AsKIiS1PghwCsA3I79B2uI0/DXlL3T15fYJay6vwfmUhtd5cDCpW+bnN7VVV4fCQqDUJhqVyhLYS+3R3qu9fgM1Eej8q03p1+2Vbhjw/gCwCssi/WTOMrSe0BoebZo6VwlV+v378F/IlXK0dm+NYJdglFDG2gx4TPo1M5RK7Rs9J5lTvVeuPmq/mxtv4uibAi2szIjKwkr88/I/OLFMu8z5UcHtAqJwau/aw+QqEQcHrmiYyKMpUdUysFohzNfcEjBi1QjMiZwDiUxz4a8xqCUDXW+2r7sHNgfp99NPpDP69dc1Z2kZClWSszlAFxl3cjglEiX9EhMJMaQvlEolEhISdM6ZpsS0HdPwyu1XeH228/NJUUUiUb/ZlEhIoazY9gYyrE9DIpJgRPAIo49ZoeCKK41JPyuxFTZP2oyjM4/ixa4v1j9+5w6/DEZV1C9KvW4jozQDNhIb9As0jz1mYVUhtt3dVr+pUwdK+vHJ9AOA21tu488RfyLtIk+W0Ayw97LH24VvY/DHg3UfDM7es/H1JxZzFp+N1XVe9l5YPWo1Dj4jfL0hFCwLHDvG/f/2bW4O23x7M945+Q52x/DvUh5l/zZGRZVhnP37ug/26A14D+M/WI8+wOgbgA//z2gf/z7oE9AHLjL1IQuenoBNcRYytkeiLIun15pAmGrelFfLja4EUEVBZQHu5d4TdM6goEE4+uxRxL0eh/gF8UhYmICEBQlY0nsJ3u7LNQTYJE1BUOJHCPCTYMVnDHzc7LH/qf04O/ssXhtFronExIa2y7qQk6tAsW0UAMtQ+glBZWElbEQyo2X0mAN+Pfzw7PFnETom1OSvpVAq8OuNX7HoyCIsP7kcX5z7Aj9f/RlH4o/o1Yz57ol3cSfnDmwkNmjv2R7Dg4fXExosy+Kr818ht1yYgjEyjpB+ztUdERDA86TyJKDoFiBuWsXq3l1FFePRDwh5Be3aArNmkccZBujZk7gWrFpFVBXqEF8Qj+k7pmPq9qm8hsSIGHi290K1nSvi43Wr0EwBbXNm47Vf9I5orO+xHpk3jNe4uv+F/dg4aCOvY0urS/HMrmegYBUQMSJIxerfiBN1W+wnniDqzF27gDlzyPt48iSwZIn2wuIA/6FoKe2OSnklpv/5AmoVtU2O+fHqj0gvTUeAY4DaPF7afEZtvn7/XfMc+2S7JxH1chS6+XZDYVUhntr1FF47RJ7T39EfJ2adwDv93sHsTrM1D1oP9PQjBseX03RpmNRDXiXXqEx6VODXww9dXzRN44q/oz9mdpip9jGFAvj+e0L2ZWRwto0UP439Cb4OvojOjcbEvyeiSs5tUlKKU/DNpVUAgB5FX6FLRw0TUiMwDGdLvHdvQ+vi5kAn707wsPVAWU0ZLqYKJ54bY+Q3I/FuxbuwdTM+C7Xof//DshUrMEAsxlCJDFsAHAfwJ4ARMhkGiMVYtmIFFi1erPO5Guf6XbsGXLltg9SI4XDuEdbgWJYlc9iFC2Qf/e67nMJNE1iWrd8fTolQnwunDgzD1Y90xV6oWns2l0mMJqWftdgaCYUJyCzLbPC5eVTg0cYDbxe+jd6Lezd5rKyMWyfY84yvrKytxFcXvgJAGlq0NRT6+XHXp1BrT3VwtHZsQDImFyVjybElGLZ5GI495DbEMhkQGEi+59MAVVQEFNiTDX1v/95afydTobK2EtE56rN8aB1CU6YfAHSY2QG9FvaCSGJZ9JRSoX6hwrLCM/3+ieWsPZVKJZK0yXKNDMv6qz6GWhhi8Rn7TywurrpoEm92Y0DmJMPs07Mx8IOBzT0Ug6FUAnmlRRArbNEzgH/wtCGgaj8+Fp/WEmsEuwRjdOJorHFf88gFjjv6OSKwX6CgcxqTYFTpFxsrnNQwBCzLZS/m5JBciw0bjJ8teDHzLK4FT8YtsWkUY7rQ3rMp6efkxC1EdVl8Rv0ehZPLT5pqeAZDai+FW6ib7gPBbRA8PUlXtTo4OwOp/mSjOibwSQQ48a3WNYS2wlhsLFkU29sDYWFNH2/j0aZeDeLvT8ZbWyucGKd5fkWuZNE6tNVQo2cTakKfgD4QMSI8LHyI9BL1rB5dtGdmNrW0VIfg4cEYuXok3CM0B8g3B2TOMth58ltdaguVpxafFy6oL7yZw9r0zh3SmGFrSyyYAFKABIDuvuQeejXjKu/ny8sDRKwVAjx1EBI1RYDS9GuiZf2X4fzz5zGjwwy1j3t6AvaFqVD8exI5d9WrJywVsf/E4jObz9RmHBgD8w/NR+dfOmPDDcOy4wKcAvDViK8wMIisMRUK4GDdUnrqVG6j5iRzQlffrvD1JZvt6hqWd4MAANzLjoNCXA4pY4Nw93CDxmwslNWU4XTSaVxN1/4ZKogvgFtaIP6Q8CuU6sroMQdsXGzQamgrOPoLaO3WA5W1lRi6aSjmHpiL1ZdX47PIz7DsxDK8dug1jN4yGpP+niTo+a6kX8GPV38EABx4+gBuv3obB585CAdrEkaz9c5WvHX8LfT7vR8ySvlnm1xNJp1kLW06greTb9+/gCd4bB68hwI9fgYcQzFlClHO/PYbsHw5cS3QtL4BADEjxo57O/BP7D9QKHnceAE8+890ZHccgfJy/llKpkZ5bjnu7bqHouSiBj+XV8pRllVm1AaI7q91R7+l/BqmdsXsQnJxMgKdAvHjmB/VHpOVRTK1KcHh60uK5ZMnA599RtboSUnAli3qXyM5Gfjf/xj43voOIqUMZ7L+wbN7nm3wfhZVFWFFJIkU+Hjwx03WfuXlwL26Ho533iHr0eRk7n6vDqFuobjw/AW80+8d2FrZwkHKBTb1C+yHz4Z+pvsPJBA9/etIv3ThpF9hQiE+d/gckSsijT0ss0FRqwCrND7TXlGrW2m2Zw+pZdCl5/btDZ1x/B39cXjGYThaO+Js8lnM2jOrgUI6hB0Nt5JBmNltgtY5qTEGDSJr5IICoohtTogYUb3a70j8EYOfT+YkM2kBfdHixUhOSwP6v4v/2bXGc06e+K51awx8910kp6XxIvwALtePrrmOHAHkMntkB/dBkrJh3efCBaJaZhiiCKV1HW2IK4hDbH4spGJpfU2VLyjpp+teREm/xs4q5gRV+iUmNqwLqCob407H4d5OYQ11zQ2GYSBzlkHm3LSmQPP87OzAe+2z/sZ6ZJdno4VTC61WwxQTJhAF56BBAgbNE47Wjhjeajiq5FUYv218A+KPWnw+eKD7eYqKgMI60q9vQF/tB5sAt7JuwekLJwzfrN5dhI/Sz1KxZ+YerHRf2SR+qLKS+5zxIf0USkV91MS4MCPZYwjAY9LvEcCYUJL3ciX9imC5/6jVo/Bm7puQWJuf8ecDRsQgoE8AAnrrV+y2JOTkAK3SlmPc3WJ8OPxNs7wmLeJfuqT7WBEjQvyCeKyZsQYBvQOMbt1hSijlSpSklwjejDQm/by8SKFVoSAZUuZCVhaxz7GyAkaMIDeJffuApUv5yfb54m7paWS57EG8/LTxnlQA2nq2BQMGOeU5DSyA+Ob6xeyKwfmV56GUN5+aTxPkVXJk3cxCdSk/lay6PL/GSClOQbIDyU8b5SggX4yOSU46HmfNAs6dU38MtVTq0kX3gji5OAkV7X4CQPLehIBex0NtF+PK3Ct4fwAPtZWR4GjtiM7enQEAkSnqCy4eHkSNIJfzs1N0C3NDr4W94NzC2YgjNQy5MbnIic7R2HWmCpbVnOkHENWzvT1R30bX8TZvHn0TkcmRJrGCZNmmcx21Uxo4EBhTF2t3/Dg5trsfIf2iMqMg50nQaVI2NkHUEmC7PVAtcPItvAlceoF8NQI8PYEi79ZIGTUX/r1Nb5NpTNh72aPdU+3gGmr88JK/7vyFv+7+BYVSgQ5ePCo6AnD5MrHidHQEBgxQdwSLktB1ON02Aqfv8AzTABBXSPIzQh066d9hm3MOOD8DUBpHbvDdpe8w+I/BWHVxldbjPCI88M7qd3GGVRglo8cckFfJzWL1bWNlgyDnINhZ2WFBjwWY330+5nSag/Hh49HOs50gC6VaRS1eOvASWLB4tsOzGNpqaJNjWru3RqBTIGLzYzHkjyHILOWnIJuAXzHw7j2MD5zDezwAAJGwrE+GIQ0aOufYOgQ6BcJabI1qRTWSi/nZUkskXKMY3zwbUyPrZhZ2TN2B+MMNmyU7zuqIRSmLENDHeHvYTrM7ocd8fpnw+x/sBwA83+l5OMnUN7ucOkW+duzY9H1r357sQwBCeDSOijh2DFi0iJCCLcS90e3hbjBKK+y6twvXMsji8k72Hbx++HUUVhWijUcbtUXUW7fIvsvPj1guTp9Ofv7nn9qtRa3EVvhs6GcoW1aG70Z9p/VvYQxQpV9UVhSq5cIccZxaOKH1xNbw7sjXKNnycHfbXXzh/AUSTiQY9XnHbh2LkX+OxIM89dXr9HSAxsS+/jq5TkpKCBGoig5eHbDnyT2wEllhx70d+PD0hwAAN0kgwqJ2o0fcIQwZIqxhzcqKFPUBkuUrb+Ye+ZHBhPQzRq4fq2SReiEVqRdTDX4uTfD29ka79svRbdA9rPzxLC7dvYt3ly/XaempClV7z5wcThUMkDWbKqg7yKRJXPOiLlACtX9g//oGG76ge3i+pF9z5fkBRBnGMIQIo424AMkrpcTfla+uYM+sPRqewTJRlFSE7NvZTUgXgMvzc+D5tlbJq/Dl+S8BAMv6LYOVWIcvLIDRo8ncQJtTjQkXGxfsmr4LT4Q90YT4C6/rH4yN1f4cLEveb6r06xtoftIv1C0USlaJzLJMtc3XdO2hLdOvsrASG3ptwIl3Lcv9z72NOwL6BDSJ9qHrJYlEs9uFKu7l3kNhVSGcZc7oE9DHBCPVjsek3yOA1u6tcXTmUWQsztBo3aEJjv6OJpH1GwO1FbVIv5pucYGd+oJmRQX4SeAgMzxUnA8GDyZdtnfvag/uVUXw8GDM/HemWbJ4jIX8uHx86/8tTr4nTAWmzu6SZo/89htZXJoDyXW1joAAsqF5/32yQImLI51qfOxZdYFlgRQF6ejv6W8epWlj2FrZItg1GAApBFBQcjpVx75j9JrRWJy+WGNnokxfM3UjIOduDn7p/Asuf8+vA5huELSRfl+e+xIso4BbyRBY5XcWNJ7ycuDjj4F//iEdXitXchZOqqCkX1cdbj1ZZVlo91M7bCuZhwL7c7h0Sdjml24w3FxF6O7XvZ60MRcGtCAVfE0ZcAwjPNcP0Gzr0Bw48+EZ/NzuZ7Ubn8YoK+OUxOo2oRIJ0KsX+f7cOSA2PxZfX/waQzYNQVFVkfEGXYd9+4DnniP5QXI52ZReuEAeGzGC2I3a2pI5+e5dIMwtDI7WjqiUa7YLUcU/sf/gm/LOiPX5SHdB2qUL4D8RsBa4O68pBBJ+A3IvCDqtorYClbVNLb88PYFamQNyJH5geaqs9IEp5s3AfoGY8tcU+HTWMsHpgdTi1Hobt/cGvIcefvyK33xx4AD5OmqU+k0awzDIdDyAcpsH2BCtnSxThWvORIy8UYyv+mwWPqjauqpF8lYgfR9QbJwubKpauZJ+RetxUnsphi0cZtSMHlPjwIsH8Kn1pya1mKX4fvT3uPnKTXw3+jusGbMGv034Dfue2oc7r97BlsmcPOrP23/irWNvQaFU4GDsQYzbOg6B3wbi+X3Pkwaf4mQUVhXC1cYVq0aov7a6+nbF6edOI9ApEA/yH2DIpiHIKtMtd0uIl8ChKgI9I9RnIjdBzlng4a9ADU/P9dvvA2cn8ztWBWKRGKFupF39ft59XucUJhbC68ZhOOQmNFuuX+M507OdJyZvmYyWQ3X4yJkRVfKqemJgfPh4tcewLKemG9qUYwYAtG1L9ibV1RxBCJDGtR9/JM4P3bqR70eHjEaXhL8w32Nv/fzyx60/8OftPwEAK4asgIgR16svKOg6lFrIjx1L7n/5+U2JHXVgGAZigQS1Pmjl0grutu6oUdTgVraOLsVGEIlFmLZjGjrMNG6jijlh7WgN326+Rm12O5dyDqeTTuNU4inYSZvWRlgWWLOGXGedOwPDhnE2wnv3kv2NKoa0HIKNEzcCQD3xfP48WVe2CrRBUJDwMY4aRRqB0tO5NUJzgcY8RGVFaczP5A0G2DxiM069d0r3sQaAEl6envo1PKnae/77L7kmOtzajFbXtiM2VsVJpoiLqxg5kv/zU9JvVMgowWPja++pzVnFXJBKOdVkcqMeG3sp8b4MmR+CyVuE38ubE5e+u4S1HdeiNLO0yWOU9OOb5xeVGYXiqmL4O/ob3R5aX1hLrLFj2o564m/CtglIKkpqQPppK5VXVAAlSEeldRJEjKi+ecWcsLWyRVvPtgDUO/RQBWx+vubfReYkQ3lOOa8ahzkx8L2BeHr/001+LjTPr71Xe+S+mYsDTx+oJ5vNWdd8TPo1M8Q8tMgMw2B48HDYWNkIfn6lQom8B3koTGwaetzciDschw09NuD6uuu6D34EkJJCZjF/M3Jp7u5An7pmgeZeqJoSEpkE3ed1R4v+6sO/1aG8nBS+gYak34QJpFunspIUoM3BOdPFF92MdO9OclD8/Ig65a23yKbFkLGUlrLIl5Hi3pBw4xZKhUCdxSdfewyXli6w81B/9xSLxWjdujWvOdMUsPWwxcAPB6LlYH5FHxpIrqk2mlKcgvU31gMAwjLeF0T85uYCb78NREURy4nu3cm1s3p1wxyMggLOVlYX6edt713vMx/d8hWUlNcIsvhs7i5HXaQf0DQ3QhciV0TiC6cvUJ5brvtgM6D9zPYYsmIIrGx0dybSDaijo+YONHrvuH4d2HWP5F0MbTkULjbqM+j0BcsSux6AENMffEAsFuVyYlEdHEyu4/4kvhjHjxNlejdfUiHkY/F5LeMacsQ3UWGdpJv0C3sN6LdN+C/i3geYmEbO54mndj4Fh88dsO/BviaP2dkRopNRyJF01zSZfs09bwqBklVizr45KKoqQg+/Hnin/ztGff6EBEIoi0ScslQdng97CwBwvnwjsst0y4JZlszJVkpHdG0ZLGxQ91YChzoCZYlA1++AkdcAl47CnkMDuvt2BwMGiUWJvDLiFr6xqD6jZ6TMsIweU8O7izdaT2wtKOOZL6rl1fj24rf1OXuO1o4IcQ3Rek5cfhxm752Nry58Be9V3hj31zgcjDuI1JJUbLq1CTKJDCGuIYh+LRqHnjkEDzsPjc/V0qUlTj13CgGOAbifdx8Dfx+olTCLy0mpv89TOyidSNwEXJ4LKHjmj5XcB/KvAALdZgDSuApAo8qnMeRVclSevgLH/MRmIf3UzZkOPg5o/0z7JvbuRclFuLXpFoqSiozy2smRyVjbaS1iD+po7QdwIuEEKmor4O/oj07endQeExND1t0yGdfk0xgMQ5QMALHNo/uQzZuJOq9LF9Ko6OxMCu0+RVOQf3FsfVPYoKBBGBc2DhsnbMT48PHYvh145hmOzGNZTrlDST+plDQBAcCOHU0bMFlWuwLQVGAYBj38esBabI3EQiN0Yz5iaD2hNZ47+RxcQ4y3kP/07KcAgDmd5sDfsWmB5MAB4jYhkwHz55PrsXdvMpdVVQF//930OZ9p/wyuzL2CT4eQ56a2nIMG6ZenZmsLzJ5Nvt+6taErhVJJGnTNlffnaeeJzZM24/68+/Cw1Xyf4AOGYTDmhzG87YL1gUJBHEMYhkG3bsF6rTW9vMi6rLqaq2V5ezNwcGDAslzTwNmzZG4ID+fILT6Y1mYaJoRPqHdOEwKh9p7NSfoBmnP9KOkn6yJDxCQTSNZMiLCxYRj4wUDYezcN7aMNJnyVfr0DeiPpjSTsmLYD1hJrI47SMFDir19gP1TKK/HB6Q8QGEjmxYoK7c3KxNqTNKN29OooWM1qLNTHcqiJFKBZl3I5R9Q2BiNisDBhIUZ8NcKUwzQaaI2Zb5YkALjauKJfIJmPxWIxwtTl7pgIj0m/Zoa6oHBjojy7HD+2/hEXvhLWmW5MsCzJCjh+vGF2kEcbD/T+X2+EjuG7S7Vs/BH/FU63jUCMw09mfd0nniBfT59Gk+7KxhizZQx6vt8Tfy34CxnX+OeFNDdcWrpgzA9jEDJKe+FFFVTl5+REbpoUIhGxq7G2JplS5iBL6eKrhQpn6eMDfP01sdypqgK++IIQOUKz1ChuJKSgxioHDCtB94BOhg5Zb9STftlNSb/CQu0beVbJouBhAQrim9ruKZVK5Ofnm3zO1ATnFs4Y9MEg3jZOupR+D/IewEnmhN5eQ+BWNhBJSfxIX6WSBJcnJ5NF1JdfAu+9x80DP/0ErF9PNiC0IzI0lHwOdOHLYV/C3dYdxdbRSPb4RaNlqDoUFADRAYvwR9HzDd57c4EuoqJzo5FXoT4pmjZk6FKcUtj72MO/pz+qCi0j+Dz8iXD0X9af17HarD0pqFVJdjawI1p4yD1fJCcTolUiIXPx7dtcftCIEVyRZtgw8vX8edKUoW0D0RhRmTcBAI4VnXhbzwmG2Bqw5amkqYOzzBlKVolbWU1VAwxD1A7tT36HI3PUVLaMAFPNmyfeOYHjS48b9TnXXF6DE4knYCOxweZJm3nZ7gjBPyQ/HX37as9dmdC5H5zLekHBVOP7y2t0Pm9REdnIMoweBR9rD0DqDFg5ASIrwIkQJKjMBjIOaz1VF5xkTvWEi66Mqn8X/4vPbD7DvJfnIzktDQPefRffhEXgSWtfLLGPQL+lwjJ6TI3ei3pj2vZpJnnun6/9jMVHF2PQH4N4O5GEuoXi+9HfAwDyKvLgInPB4l6L8c/T/+Cbkd/A084TACm8UYWUNrRyaYW9E0/BTh6A2IJY9P2tL0qqm1ZKLqVdQsTaVrjh9zI8PJUam4yaoP1HwMADgA3PE3r/CUxKAwS6zQBAuBtpV3+Qz4/0cwt1w5TIhUgPH4L4ePM05qlCyJyZcS0De5/bi7RLAuwDtKCmtAbVJdVgRLqZi8PxZH4YHzZeYwYvdX/o16/hPqgxhgwhe6LUVJK9FxcHREaSOW32bO4e3b07WUsWFnJE3riwcTjw9AE81+k5ZGQw2FbXT7NxI3mupCSyPrS2JqpCiv79gXbtiCPBBpXo1uJiYOFCYM4cfnn1xsav439FybISPNnuScHnZt/JxtaxW3F/Hz9V638dV9Ov4t+H/0LMiPF2v7ebPB4bC/z+O/l+9myyHgK46w4gDWPqXHm6+3VHF58uqKri8iL7GuBqN2wYIZOqqoBffyU/o44qixdz9qPmwMwOMxHuHm6UbO1Oszuh1bBWRhiVehQX0zmahVyu31pTIuGasysrSYPBqxdmot3H5B5PLT6pEnnwYGHPP6fzHOx9ai/aeLQRPDa6h8/OVp9/TmEJmX4A11zeuJGYkn5lNaZpMDQlWg1rhUEfDlIbVUXrnnyVfgDgbuuOXv4aumCaEdYSa3wz4huIGBFEjAiMSImQurLnfS23lMJCwLtwEiblXsO3I781z2DVgO7Zr2Q0dReRSACXun5ibRafloayrDIcePkAHh5ruhihmbN8lX6NoVQqUWDMjCcdeEz6NTOEWFt+fOZjtPupHe7m3OV9jr2PPfot64ewJ8zHJDdGdDSwbRvw3XfAkiVcRoNHhAdGfD0CTgE8qtGPAGJKL6HM5j7sHHl2zhoJERFELVFTAxw9qv3YuII4FCQUIHZNLLJu6bYMepShztqTwscHeP558v0ffwiz+9MHVOmnSvoBpDvkww+BqVNJ52tMDCF03nuPbDyEIPIhucl6KjtAJmk+G8xn2j+DwzMO47Ohn9X/zN6euylqs1StLq3GmpA1OPNx0zR1lmWRmpr6SNgBs6zuTL/hwcORuDARGyf/CoYhnU+NbWzUIS6OPLedHSGNg4PJBvnFF4FpdTXQ/fuBuXM5YkWXyo/CzdYNnwz+BADw0Hslzl+u5m3xmV+oQJrbJhzL+90k9pC64G7rjvae7dHFp4tGOzRK+vFV+nWe0xmzTsyCW1gz7+L0AM2307YBtbcn12eFNAlR2dchYkSY2Hqi0ccSWRez2K0bsaGlY5JKSZ4fRXg4UWNWVxPij24gSmo0tAWq4GYmIdVcqjvC2VnLgemHgMgpQBH/dVQD1BQDqXuBsiReh3f0IqotTVZhnp5AXkBn2HcxzRrNVPNmzK6YJvlWhqC4qhgfnP4AAPD1iK8R5mb436OmBnjwgORJXbjA5fiOV++EV4+AAAbhuUTt9+OVH3UWSS7E3cOF8AFICl4GiVDhWfAcYOTVplaz56YAF2YCtTo6uXSA2qPqsvh0DXFF6OhQ1JTWwNvbG+8uX45r9+9h0Nh0dB90Dy++JCyj51FFcVVxvTLl5a4vCyq6vtb9NRx65hC2Tt6K9MXpWDVyFcaGjcWCngsEj6O2Ftj2czD63L0K19IBeC3iIzhaN6xqKVkl3jjyBhSsAizk6N1LxF/lYusH+I3jPyA9yD4KoaSfSCJCRG9nSK0ZVFZyrgnmgro58+BrB/GN3zdNMp0D+gTg6QNPo8UA/i4k2hA6JhQLExYidLTuZthvRn6DY88ew2vd1SvPa2q4rGdN1p4UdnbcvfjQIULYAUQ5RZUjACnc0eYcmslLwbLAzz+TBgiplBTIV67k7EU7diT5aRQMA7zyCmnEvHiRNKmVlxNVYWIiKeZ++mnDbCpzwNveW3CUCoVYKkbC8QQUPrQ8dyVdqMirwO4ZuxF32HhBmp9Gkrl0ZoeZaOXSkHgqLSVNi3I5cZ1orL7v0IGQwnI5d+9Wh5gYcoyHh2ZnFT5gGODVV8nXs2dJdtySJRy5ra3o/iiAVZpm70w/n87OLNLT9V9r+qn00w0fTuaannX9MVFRpAEgPp5k0/PN8jMG3N3Ja8rlXDOlOliK0o+Sfo3tPSPcI9DZuzNSPk/Bl65foqrIMppZDQXfTL+c8hz8G294Tqap0d2vOxIXJuL3Cb9DxIjQuq4P8IGW5VNRESCCBKF2XTEwaKDmA00MGutyLeOa2nlA1eJTE3JjcnHui3NGc08wFAXxBbix7gZy7jYtXlJ7T1seKWprr61Fv9/64a87f9X/jGVZpJm6AK2Cx6TfI4TTSacRnRuNS2mXeJ/DMAyGrhjKawNhKqhezzTDbN0PNVCaaAHSXEhjSVFlQLB5rRUZhlP5HDxIrBY0QSaRIblFMrqe6Iq209pqPtDCcPK9k9j19C5Bi1ZtpB9A7Gw6dyYb4507jTBIDait5UiGxqQfQBa2zz1H1FnjxpH/37zJhVXzxbVMooZpZd181p4AEO4ejlEho+Bt33D3Rd8HbRYZMicZBrw3AG2mCu/GMzUOzjuILaO38NrQlJSQbkWG0Xz9AaTzLswzqN6mhE8uJ92AdupENrkUDENyMD78EGjdmlx3lPihlkp8MKfTHPg6+KJKmoYY6Sbc5cmNxJZfRq2kAI5Wzugd0Jv/CxoRUS9H4fpL19HOs53ax/XJ9LMUpF1Oww/hP+DeTn55X3xIP4CQxpkuROU3oMUArbZz+oBlOdKvf39SQPz6a1LkeeGFhrYYDEOKjADp7B3Scgj+mvIXtk7W3mJdVFWElNIkAECQrCNE2la2JfeAtL0Ao+fyN+8CEDkJSP+H1+EdvbWTfl5eQEbrIZAM5qfgtBTMi5mH2WdmG+35JCIJ3u77Nga2GIhXur1ilOf8/HNStFu+nHxfW0tUzzQnQ+NYJEAv1/GwqwpDcU0RNtzYoPX4/Q/2o8AhEuUOAjKgFDWAvK5BTKSGKWy9GOj2AyA2LJObZnvoUvp1f607ntr3VAP7JIbh7jGW1JmrVCixZ9Ye3NokLHOLD748/yXyK/PR2r21Xnkvo0NH4+n2TwuKY4iPJ+u/2DpHR0qexMQA1nIv9HpwAp1r59UffzzhOPbE7MHWO1txOf0yJEp7hGd8qtG+sQlqioAqgW+ospY0O2SpCQ7WgXD3cEjFUjDgT6BWF1UgVJIIRiGvbxRtTjj4OsA5yBlS+4ZkkIOPA8LGhcHB1/x2WlKxFMNaDavP0WmMyEjSie7p2VBhpwnU4jMykqjxJRJg5symxw0fTr5eu8atMwBCMN66RYg9Gl+Qn09y2QD169AWLbj96y+/kPVrQgJR+tD4g88+4/KJLR1uYW5YVroMvRc3zxrYEOTczcGdrXeQH6ulIisAxx4ew/4H+8GAwbJ+yxo8xrLAt9+SJlAfH2DBAvW2nEOGkK90DakO1B2nQwf9rD1VERzMkY/ff0/2CrSgy9chxFjYemcrpu2YptYlQgiSzybjK8+vTHK/BDiyi6p49AUl/RgG6OKdgQurLsBVUgIPD/L5X1NnutClCz/nGorNtzYbZNUrEnEKVG31C0vI9AM40i8lBQ2adrdN3YYbL99AREQEvDs9Wg1c67uvx7G31BfF+JJ+H5z6AKO2jML//v2fkUdnfAQ6BdZ/T/cr2kg/jng33Zj4oL1ne1iLrVFUVYT4gqZNodSFR3Xd0BhZN7NwYtkJpF22jEJNQN8ALMlegs5zOjd5TIi9574H+3A+9TxSS8x8I1HBY9LvEQKVIgsh/SwBtEtzwAAiyWdZ4NZXR/G172pU5FU07+CMhJj0dFRapQOsCMPbdTH76w8YQKTtubnAFS0N3TKJDLXSWjD+DKwdLcfLWhcyr2ciOTKZl+UNhS7Sj2FIvh9AiiumQloa6Xi1s9NegHd1BV5+mcu5uKy9RtcEheXlECmt0c61eUk/TeDriz/448EIH6+jKtsMqMipQGlGKa/Of6ryc3NrmqeWWJiI3TG7oWQ5nxBNHvzqQC07Nan3unYl3dWffQb06EG6soVYhltLrPFmnzcBAA99PsfZc7qlfiwLxDMHAQBDWoyERF0R2wwQi7RnSVBytaREtxUyxe0/b2P/i/sNHJnhqC2vBSNmILbml5dBN6C6SL+QECCrjvSbGjHVkCGqRWIi+TxIpeR6BMjCf9ky9blq9Lq+fRtwsHLBU+2e0vmZ+/susca0rW4JPzcdlYeIJcC0UsCxtdBfhcC9F9DjF8CPXzYItTvOKM1QazvLZxNkiWBEDGTOxlOU20ntsKz/Mpx67hRE+hKyKsjOJkVphiGF5bAw0uTz0kv8CoOhwWK0yloCAFh9aTXkSs3z4OlMMj90tnuC/wAzjwA7XYDU3eofD5gMBD0N6JjTdIFaSV5Jv9LgnsMXlnh9VuZX4vbm20azVKRIL0nH6kurAQBfDP3CLPexmhpCSO/fTxoiP/iA2N0dO0au027dSAd3TAy5aGsVtZh3aB4mb5+M2XtnAwBCMt6Bh8wHbfj2SiX/Bez2BDJ1WIM0gAg4/xQQs0rQ7wcAXX26ouKdCpx87iTvc67+eBU22zfBpjSniWKhOTBg+QA8f/55o9jtacPtLbcRvT3a4Oc5cwb48Ufy/bBh/Oa8kBDSFEH72saN44rdqvDzIwosliUNk7TJjVp0TptGCs9LlzZc/2pasz7zDCEMMjKImsreHvjkE6L4s7cnxc4ffjCvzetHpz9Cx7UdcfShkM8IabQWSy0/P1cdggYFYWnxUnR5wTg1DNro9GS7JxHu3nBPt3cvcPUqIYiXLtVsj9arFyGfk5IIiaH2deq4rI7GicLFzJkcqRQeTrLSAWJjqSmLyhTYHr0dO+/txD+x/BrMNMHexx6uwa6wsjOuXToFdagxlPQLrotD7tYNKLz+EMeWHENZVmm92o9a/Qqx9kwqSsKsvbMQuiYUxVXFeo+NOvZoql9UVnLuTM1N+nl6EitnuVy9Sr734t547uRzRl2/mxLyajlqympQU6a+84OPvee93HtYd2MdAGB8uA6rDwtCXH4cdhd8ABYskpPJdaYO17Mu41aLF5Bos8u8A2wEK7EV3uzzJlaNWAUnWVNmns9+Inh4MF64+ALCxjafQ6EqGIaBnaed2s8LVfrpsvcsqynDyUSy/h0XJsBhw8h4TPo9QtCX9Lu1+RZ+avsT8h40z66d3nTatCG+6H37AjW2ThB5e8LW3bAuZlPgWsY17Ly3U5BFwbFoorJyqW0LVwc9zX0NgFQKjBpFvteWUSeTyCCWi1GaVoqq4kdH2j/j0AwsTFgo6BxdpB/AkSEZGfxJAKGgBYugIH4bb7rAvXOH6yLhg4FlP2JUVAkmtHpK8BiNjeMJx/HeyfdwMfVi/c/4kn7a4MA3qdkEmLZjGl65xU+BQkk/dVYzKyJXYMr2KVh4mLueqQJUF+lXWsqpATo3bTqqB8OQrtf33iPZKELrVC92eREBdiHwzX8G5y/VaFUP03FlORLSb0KbscJezASoklepLXDLZNyik6/aL/lsMqJ+jUJZdvPmILQc0hLz7s1D+BP8CHE+mX4A4N+yAtUSovqYFDGJ13OXl5MMlYQE8nmu0NK7o2rtqS1TiCI4mHRsVlRw1zoAlFSX4NjDpp2ehZWFePfkuwCAltmL+OX5SWz1V/pJXYCQlwB7fhkpDtYOCHYh1Qx1Hdvu7oBdQSqK1/6FlHMaqloGwtjzZmVBJZJOJ5mkactYRXVqKdehAykYr1pFsnla8+R6g4MB//xnEcqOw87pOzUSQDnlOYirJGvygd4CSD+JHeDZH3BSr0quR0UakHGE//M2QnvP9lgzeg2OztRevFbUKHB86XFc++Vag59bIuln52mHZWXLMPRzHZ6FAsCyLBYfXYxKeSX6BvQ1emEoORl47TVgz56GPz94kKhdbGyIouDGDe6YOXOA6dPJ9zExhPSQK+WY3mY6bCQ2ULAKuIlboGX2IvToQSzIeMEhHGg1G3AWUCUXiYG+W4FOK/ifUwexSKyzIacxQseGwn/uSNTKHMyusAH4z5mpF1Oxwn4Frv6kO3eWD858dAZnPzmr87gJ2yZg0ZFFTazMWZbYun/9NVE29+oFTJ7M//VpI46dHXftqcOIEeTrwYPAjBnEXr6ggBTHp9TFAgcFEftOgERQqCMQAaKkonELMhnw0UfkXF9fknMuEpEsr+PGjZDVivjCeNzOvt1gD8MXhQmFuPbLNZTnlptgZKaFtaN1EzWrvljYcyGW9F6CH8f82ODn1dXA33URxi++SK4NTXBw4PY66tR+5eVEKQ2Qe70xYG9PmiZffRVYsYJc0/TaNedcNDqESG9339fQGMQTbqFueOHiCyZzd1JV+hmy1hw4kHzeFy8Gur7UFXPOzYFnW8/6mghA7pO0eZAPqJ1jL/9eakkIvqB7ebq3bwz6N7C15bfPMSUYhlP78WkktnRIrCWYFzMPY39SX1vgQ/q9eexNKFklJrae2Kz2l0JQVlOGbuu74aurH6OsxXawLDS6HlwuOIJUj99wV2lC2zKe+GTIJ1jce3F9jrUq+OwnbN1t4d/L32j3IUORdz8P+bH5ajkBvpl+JxJOoEZRg5bOLRHhHtHgMXs+MkEj4THp18wQ896lcaTfvdx7wjpWWEBRq0BVoelJnuTIZOx7YR9yYzjrGEr6UZVF27ZAVkh/lIyfYfLxCIGSVeLTs5+i54aemLZjGn68+qPuk0CKBdvv/wkACLJqPpUVtWa5c0fzhCqTyOCV7YUHQx/g2s/X1B9koRDaPUnJJW0e/w4OnKWEaoFZCGpqmnZTKZQKJBQm4Ej8EUQ9JOyCOmtPdfDxAQIDiTrwmoC3KDsbELFSBPryt5UyFf68/Sc+jfy0QZcsX9Iv/t94/NTuJyScSGjwc7FYjODgYEFzZnOBEkrUTpKitLoUf90lft7T23IVFb5Kv1u3SFEnIEA3mWMI7KR2iF94H91LP0VFsS2idTSeR6emocT2FsAyGBs2ynQD44ERm0fA8XNH3Mm+o/ZxoRafgz8ZjKXFS2HvZb6FmTHAV+nXLtwWg+/GYtCdB3CAL6/nXraMbM4XLiRFmxkz1DebNLb25AORiCvyREWRrynFKQhdE4rx28YjpbghMWZjZYPFvRejpVVPtMh5VfvnojofSNgIlBuhcqOoBrSov1TRybsTAOBm1s0mj7m5AWJFDUQJ8ShMNH4OkCnmzdSLqfhj8B94sJ9fRtfmzaSARhtxVFElr8LYrWOx9/5evZRoANdtScGyHOmnK8tKE1q1AsSsDD3iD6Crj2Z/5ENxh8CChWN5Z7RpPOFrg/dQYMgxwFFLNyvLAidHAOef1jvbz0pshfk95qO7X3etCkqRlQhXf7yKe9sb2gdbor0nAEjtpJA5Ga+6tu76OmyP3g4xI8aqEauMquhSKIBvviEF499+4yy6S0u54vdLLxF7w5EjibJl1Chg4kRCPkskRGWSmUnmu0+GfIK41+Pw2ZAVGJB6CGJWht5C3AS9hwC9fgdstHTEqUPAZMClk7Bz9IRvV190ndcLtTIHs9txq5szT7xzAtE7mi6EZM4yBPYLhL2PcdYH07ZPw7hftHeCJxUlYf+D/VhzZU2T7Lm1a4Ft28j3U6YA77wDWAswdRk8GHj2WXKetvp9//6ETAyscyErritJvPxyQ3Xf8OFEyfree9pfd+BAQvZ9/31DZ4pOnch4AGD7drIvMgf42iKrQ8LxBBx85aDRlcimRsa1DGTfUXOT5onS6lK8e+JdpJeQPAsrsRW+GvEVXG0aSp/OnSP3bG9vrllZGwYMIF/Pnm2q9rx7l/zMz0/3WlcIWrQgBDi9lgMCyFdzkn5T2kyBRCTBjcwbuJ9nuYGClPBycxMZtNYUiUhWn709YOtmi8C+gbCytUK7dpzFap8+wuazIw9Jw9SoEMP2pbrqF3z3W+aCOtLv88jPEbomFCt/XImT751s9kZWY0GXvef5lPM4FHcIEpEEXw770nwDMxD2Unss6EHyoC96zEWp7J7aOmVBZQHOVBLv234+w805RMHg20SoqFWgKLnI5OPhg2NvHsNP7X4C1OiAqDBDF+lH1drjwsY12FuIxWK00tb1YmQ8Jv2aGUoBK1hPO0+0cmkFFiyupGvxcGyEjrM64vXY1+HfS0BBgify4/Jx5I0jUNQSKUjKuRTc/O0mFNXk/0olkJVaC4CQflXFVXDKJoWimBjt+XPmRF5FHsZuHYv3Tr1XX3j69tK3qFHoDhLY92AfzhfuAsOKMdbzVVMPVSPc3YGIugaCCxfUH2MttkapQynsn7GHbzd+Bd7mRnluOe7tvIfiVP5EN8uS7mlAu9IP4DaY2vyyG6NKXoXssmykFmZhwbuZmPtKFVasIIWxvff3wv5zewR/H4zRW0bjrbRQ5Doe1Uj63cm+g5m7Z2LA7wMwZssYTN8xHYmhS6AQVfC2+Mwpy0NOLrkjGRJkbixQS7s7ORzxQselrvCrColMAnmlHDWlDT97SqUSWVlZguZMY6E0oxTX1l7jnXWhifTbcW8HymvLEeYWhn6BXBI5vTZSUrTPibqsPY0JqZW4vpB45Kz2FdrSyNcAAN7y3kbPhBMKFixqlbW4mKa+Q5u+JzRnUxfsvexh7dD8VsiX11wWZPtFF9W6yGE7O8DXh4F9dVi9fY425OcTy06GIZ29UimxkVm3jhRlVBEfTz7v1tbCciUp6Uev9wDHAES4R6BKXoWlx5c26LiTSWR4p/87eAEXIIJE+++bewG4NAfIOMR/MOrw8DdghwOQx891YXTIaMzqOEtt9pKbG1Di1hJ3JryLDjON5E+lAlPMmx4RHhj13SgE9A3gdfzp02ROpPZvqvj1xq84FHcIC48s1GqhqQ7l5YRMeeop4I8/uJ/HxJDijEwGYWSICoKCSBGquJgr6NzJvtPEonX/A2Lt6VX8hM61hmAwDND5K6Kwkpi26YBhGLx651U8c+iZBj+3RKVfSXoJMq5loKbceEFfT7Z7EqNCRmH1qNX1lqjGwp49RBVNsWoVWSvu2EGu4RYtSHaVtzcwfz6waxcwbx55+6VSYrkINLSh93P0w5O+y1Cb0QbW1tqV/0aFahalAPxw5Qd0XdcVay6v4X0OLbRnZjbMJjI1Gs+ZtRW1OPf5OcTub1px84jwwMwjMxExKaLJY/rAu5M3Avpon1cPPCAdNv0C+zUgVCoqgMOHyfcLFgCzZwt3eRCLicJPl2pKIiFK1B9/BP78k5CEH3+sfm3arp3udQjDkKwuaqOninHjSEE3K0vz3tbYoKTflfQrglx/ACB0TCiePvA0AvsG6j7YgnBo3iH8OfJPvc/fcGMDVpxbgVFbRmn9mx2pE66PHMnf/UYqJc21CQ17QY1u7akJzUH6udu6Y2TwSADAlttbDHqu2H9iceClA5BXG38ipfaezs7GW2sWpxRDKSfPI5EQZbGVFZkL+KJWUYsTCSSD1lDST5e9JyU+m9vak0Id6Udz1oquFiHy00iUZpjI3srIyInOwfV111GSrt5blyr9NJF+v0X9BgB4tsOzCHOzDMtIvnh/4PsYFDQINSjDtZCJiIopanLMeyffQyWTD4fKtpjV6VnzD7IRWJbF/bz7+PP2n032dJQUz9dRSvtr3F/4qe1Pgu+9pkCHZztg8MeD1cZL8bH3VLJK/BNHSL8nwhq6wSiVSmTrKooaEY9Jv2aG0AvaknL9WCWLY0uO4cqaK0g4RlZi/Zf1xxspb8CrA6l+XNsai4hja+BYkgoPD2D3M7tx5tXtcGYLUVVFiofNjdzyXHRb1w1H4o/ARmKD38b/hs+GfIbLcy836aJUh1Eho9Cb/R9CMz5Av2AzVOO1gKopzp1T/7iPvQ9cA1zh/bY3Wg0zX3eBIci4moEd03bg4b88KtJ1KC4mFiIMw3WqawINydWl9IsviMd3l77DyD9HwvkLZ3iv8kbg9z74xd4XaW6bcfEiUTPEXPFBlbwK1mJr+Dr4Qo4qXA15AolWDX35Y/Nj8fSup9FxbUdsubMFkSmROBx/GDvu7YDYOQNyUSmuXSMWPdrAsiyG/TECJ8M6odz+lkV0mrX3akr60YJoVpb2bI6ggUFY8HABWk9s6MPGsiyysrKaZRGQcT0DB189iJTz/Oz3NJF+dPH5fKeG2TBeXto9+AHyN6MKgS5mig3t1QsotLuMd/IC8P6p91FZq77YN9JlPqxrfPCE5HvzDEwLevuTKr8u0k+IeiA/Lr+J8tScYFkWp947xVudXVXFLUa1bUKr5FWoUdQgJIT8n9okaQMtPLdsCWzaRDJ9nqhbx377LVGaU5w5Q7527y7M8oYWsOPiyIaOYRh8M/IbMGDw192/0HVdV/xx8w9U1HL2koUFZDmrdb537w302wH4juY/GHVwDAe8RwI8M79e6PIC/pj4h9rCg6srAJEINbUiQXbOfGGKedOllQt6LugJ93DdcmOW5Qoily5xRTqAWGN+eZ503i7tu5TXeosiJoYUtk+dIv/fuZO79k6QOg/69tXfakkq5VQsDx8Cv0f9jq7rumL+ofn1x1TJq+rV7F5F4/mTfg9/A86MB8qSdB/rN5ZcrwYozworC/HrjV/x6dlPtR7nHOQMiXXDa9oSSb+7f93F+u7rkX3LeJtlZ5kzDj5zEPO6zzPacwKkueQvIu7HvHmEwCstBT79lFNHz55NCGYKUaOdOW3mu9dQhIlLddvALl2aZgdrRMEN4N9eQMZhIb8GQfYpYLsdkCScGMiryMONzBu4nX2b9zln3tiDtpFroVRqtlUzBRrPmRKZBAsSFmDIZ0NM+rryKjlqK3Qs+AGcSSY3Vmr/R0EtYL29icLOXHByIs0VpiKeZTJgbJ2z2+7d5sn26+jdEVKxFPmV+XhYyH/vCQCO/o4IGxcGG9fmd10Rgt7/643BHwsITFNBraIW3176FgCwoMcCjUrppCSS2ygW81fh29iQNSTQtLHsdt10YixrT02gawFzWw0/05404Wy9u9WgNVzqxVTcWH8DBfEFxhpaPej6ztnZOGtNRY0C37X8Djuf4qwKn3+eqOLpXoUPLqReQGlNKdxt3dHFx7BNM197T0sm/eylpHGsaHgR5sXMg0dE8zbo8kXC8QT88/I/aq9dluWUfursPStrK7EzhlxHszvNNuEoTQMrsRW2T90OH9tAlMvisLF0BuQqneFRmVFYe30tAKBtyg9o4W+a3E4hYMGix/oeeHbPs7iXq945JC9P+308YmoEus/rXi8gak60nd4W/Zb2U/sYH9IvKjMKWWVZsJfaY0CLAQ0eY1n2Men3GJrRy68XAhwDYCUW9sG++/ddRP0eZdSxMCIGT+17CovTFyN0TGj9z50CnOoZ8dz0GrAiEVyCnCESAf2X98e4deMQ0pV4a+uyjTMHPov8DMnFyWjp3BKX517GnM5z8E7/d+Buy88/TyaRISL1a4RlvtekyG9u9OlDvsbEqO+kWD9+PZLeSMKMDpZlraoN3p29MXnLZLQc0pL3OXQOdXMjXWLaoKr003QTOpFwAqFrQvHGv2/g6MOjqFZUkwdYBmBF8O8WhbZtCdEYubMT3neOQ/k75Yh+MRHehZOhFNXgjUuTsOseF7I7YdsEbLu7DSxYTG87HdumbMPvE37Hhic2YPezm+Dj6IWqKm5TownnU8/jTl4UymWx8Hfyb1Iwag5QpV98QXw9UeTpSeqW1dWcFdCjgsC+gZh5dCaCRwTrPFah4Ii7AJWG7ft593E+9TzEjBizOs5qcA7D6M71S0khGwuplFgkmwPh4UCmy3YomCp8cvYTtP2pLbbe2Vpv30PhVzUCQ+4koKtP8zY9ACqkn4YsFn1Iv72z9mLndGE5r8bG3EtzMXL1SF7H0rlfJuNscdRhT8weeH7liesOHwLQnBegivt1LkM0G41hgLlzyb1HLidZKD//TOzq9u0jx/C19qRwcyNFFpblSKIuPl3w5bAvYSOxQVRWFGbvmw27FXY4nXQaAGc/qLXpQeYOBE4F7AzswPfoCww6ALj3Mux5QD7PTk6AXWEaov4WIDd/RFBW1lCls349mSOvpl9F13VdkVqSCn9Hf8zpPIf3cx47Ruxlc3LIfYVmvKxeTe4ttOlJX2tPiuC66T4hgTSyKFkl/o7+Gz9eIdbvxVXFGN1yIhwq2sG1ugv/hpvSWCDrOGDNs0KkVABF0UClfoG4RVVFmHtgLj4+8zFqFZpJherSaqRfSUdVERcFYIn2ni0GtMCQFUPgGmpYhS2rLAsbb26sn9dFjMiotp4sS/Ika2oIITJyJLlu7ezINSWXk2K1LuV+mzbkq6rSDwAu1t3iBKlZK9KB0jhAHytd+xBCQtuokWPpQIgrqdYKIVCsHaSwcrQFWGWz5PpRMCIGLi1d4BTYNBOqPLccx5ceR9xhHjdPHYg9GIsVdit0KvpvZZObYlffhhcO3U+ba31oTowbR+6VcXHmqRtIxVJ09iYs5uU04RafAB65TL+209uiy1z9yJHt0duRWpIKTztPPNtRs9rkXxKxhl69iFMEX1CLz8hIbp9eVESyUgGgfXvhYxYCum8w9zw0IXwC7KzskFCYoJfVLEWvhb2wJHsJPNoYn+hRzfQzBuTVcvRc2BNh4zhVFsMQpZ8QrL+xHgAwNnSsVmtzPqCkX1kZ1DbncRanBr2M0UBJv9xcjpigpF+JTQncW7tDIuPXtNjcaDutLWYcmQHvjk1trCorOXckdaRfYlEiPO08EegU2MBd6VGCh50H9jy5ByKlDBl2h/D09tlgWRZKVon5h+dDySrhm/8UWigHwYzxcBohYkT1a5Or6Q2zjikpXlvLkbXq0PXFrhj+5XCLv0b5ZPoxDIOJrSdiUutJsJY0r2uUBZSHH0MIXuv+GlIWpWBpv6WCzjv/xXmc+fCMScZk7615lmHat8Pdwa/DO5TorgN6B6DznM5o255cepZA+s3uNBujQ0bjl3G/1CuUVPHHzT8wbcc0bL2zFXkVeSisLMTppNNYe20tFEoFqqs5K0nVIn9zwM1Nt8VnRV4Ftk3YhhsbbphvYAbAwccB7Z9pD5dW/FeUlPTjY3XZsiVZTJaVEbKmpLoEB2MP4tWft+KFF0hHyoAWAxDkHIQhLYfg6+Ff48y0e5h2T4lx15X4PUiBQ/N+wuefE+WBmLVG1IkQJCaIkZkmRZeHf6Nl2dOQK+VILeF2DCNajcC4sHGIejkKf0/9G0+2exKzO83GC11egJVYUh9efUmHqPf7y0Rd5Zc/Ey08LGPF6W3vDTcbNyhZZX2nj0TCqQZ05fo9PPYQZz4xzXylD2xcbRA8PBiOflqSouuQnU2KeVJpQ9XR71G/AwBGh46Gj0PTopmuXD9qddi+vYCufgPh6AgMqv0aXeN3wUvmj8SiRMzYPQP+3/rD8XNHnEki71FWFsm/sgRrWWrNFlcQh/yKpp0PNMMzK4u/ZVift/pgzI9jwCqah/RjGAburd3VbnrUQZUA01bDjkyJRHF1MaQOZPUtROlHC9EAUab873/kZ+XlwKFDpCNWIiHFHdqlLQRUzXpD5Tb1Zt83kbooFSuGrICvA7GnPpN0BnI5UFgXh6dR6ceyQLXxu5z5okZRg/t591Fe07QQ6OYG+Mccw7kl+5phZMKx6+ld+L3/77yOpQS0jQ2x30lOBpZs2Yj+v/dHWkkawt3CcezZY5BJ+Evytm4lb+fAgcCaNcCSJYT8y8kBli4lGzFPT2IrZwgo6ffwIdDNtxs+G/IZAGDBkQXYd38fvOy98GGHPzHg3m14ejD8G246fQFMyQOsdN9PAAA5p4FD7YDkbYJ/BwBo4dwCMokMtcpaJBcnazwuens0NvTc0EDRTu/Z5eVEQWwJ8Ovhh/7L+sPOQ0eQhhZklGZg0MZBmLNvDt4/9b4RR8dh/36SOSWTEdtOhiGq/kWLuGP4WDDSBovUVM7KKjOTfJZEImHWyfB/glx7+qid7QKAAXsBPwEea3UIdiEfpvgCHjeZOoz9aSxcF80CGFGzkn7VJdUoTi2uj7Fo/Nj5L88j+YzmzxVf2HvZo/0z7eEWrnkdX1ZThocFhDjt4NVQ3nT3Lvlq6LxniXByIha4AFH7mQPU4lMfZ6X9c/fja6+vUVNmPAtiSwXLsvjqwlcAgNd7vK7xXl5dzSnz+WT5qaJbNzKP5uZyURy0IbZVK/UFf2OC1nby8rhCrzlgJ7XDxNYT0du/N6+4GY3P42kHO087oza1AGQdRu09jaVys3awxshvRqLT7E56P0dFbQWOJxwHQK5JQyGTAc7O5Ht19Qu6zrUUpZ+dHZkzAW68lPQrLytHWVYZ5FVm9Mw2AA6+DggZGQKZc9N5ha6HpFL1dZE2Hm0QOz8WF1+4aDDx25zoGdgFI6pIPkJ5iQQMw+Bg7EFcSL0AG7EdItK+gq+vQYYgRkV3X7Lpv5rRkPSTSLjPkS6LT0tAcWox1nZci+vrr6t9nDYAaCNbu/h0wZ4n92DTpE0mGKEwPLqfgP8IhN6AxSL9AnLH/TKuSVaHoTi84DBidsdoPSY9HWBFYvg2io+jnYjR0eax6tCGTt6dcGjGIQwPbuqHsvPeTszeNxs77+3EjN0z4PmVJ1xXumLwH4Px6sFX0f/3/ribQGYuBwfTLzz5oF9dM4smi09FjQJxh+OQd9+C/JqMDLrI4WO3JZEQy4hy63hM2DkCrl+6Ytxf47AxfQmyc1hcvkwk9jHzYnBi1gks6vU/7FkfgcoKBhERwMyZ5HkYhljqDB5Mrun16wmBI4IEM+0245+n/8Ebvd6of91vRn6DA08fQCfvTmrHFdAhEVEtZ+LojQcaPyOpxanYHUN2wC1zXrcI0gUg8xol0O/m3K3/ua4wbIro7dE4/f7pBp2yDMPA1dXV6JsWPqgpq+Gt8lK19lQd6vVMsmh4vtPzas/TpfSj5IfZsnvqEB7GwKdoMr4KisG7/d9FR6+OEDEilNaUYtLfk3An+079+2kJ15+rjSvC3Yhnr7pijZsbyZhTKLhmDV2ImBSBdk+1g0jSPEumyoJKVBbyz1HSZC/bGGeTiVfS2HakjTonh9tEKRSE4FON56iuRn3uH20uoZBKgeXLgUGDSP7Gu+8Scubdd4V36ALcdR4V1XCN4GbrhmX9lyFpYRKiX4vG+wPfR2EhOUYi4Ta6TVCeBOxyA24bqcCfuhs4/QRQw0+23HVdV0T8GIELqU27cdzcgMyQ/ghaMN7oalJTzJsiKxHE1vzWorQD2ssLeOYZoNz6IVYnPI9qRTXGh4/H5bmX0dq9tfYnafR8eXlkbp0/nyhZbWyAhQvJ4/TaHzLE8A0wzVePjyfX11t938KLXV6EklXi6V1P41LaJeTkAAwY4Xl+Ei0S3MZw7QpEvKW3slTEiOqVVtpIl8C+gRiyYgjcQjnSwdaWUwtbktrPEKSXpGPQxkF4kP8AgU6BglSmfLFrF5dhOWsWIaEpevYk8+K773J5fdrg5MQ1q1Cl9ebN5GuHDpqzbDSCYQAzF8CCXQnpl1aShio5f/a4ObK0Gs+ZMbtjsDpwtdqIAadAJyx4uAB93+5r8OsG9gvE5C2TtTb33M25CxYsvO294WnHXVQ1NZxS/7+o9AOASZPIpXv1qnmuhz4BfRDsEszb9UcVLQa0QJe5XXjZtVoC4g7HYU3oGsQf4U/KUxxPOI5b2bdga2WLV7u9qvG4yEjSPOLtLTyDTyrlFM1ff02uAXNZewKkqEuVbEJcQoyB3yf8jgsvXGhiDScURUlFyLqpn1uAJlRUkLkHAFxdm2+P3hi2VrZ4uOAhdk7b2UQRrS+01S9oRrzqfb65Qeuu1HmIkn62Z22xymcVEk9aQL4SD9RW1GrcF2mz9qRgGKa+SfRRxsRWM9D7/ln0Y98FAIwLG4fNkzZjltcq2NT6q83DbS5oIv0AfpEB1aXV+HvS34j8PNIUw+ONqqIq1JTVaLQZpSpabY5K2sAwDFyMJZHmgcekXzNDpKcXH8uyajvGNcGvhx882xrvblSSVoIrP1xB/L/aF4j0ZtOY9AsNJYu4khLzL6AolCr2NhkZ3IdXFVMipuDM7DN4q89baO/ZHizIjSfIOQgTwidgZoeZKM0hH9jmVvlRUIvPe/eadlL8cOUHDDk4BPZn7DHi6xHmH5we2DpuK35o/YOgc6jvOp+bIMuyyPJbh7NtOuJ60TEoWAW8rILhUTQWSlFl/Uaadi8mJJAioEwGvPVWU/vQWbPItX3vHrBnD/lZyxZijA0b2+A4XQT+b2lLkO62BVcclmnMG1x7bS0UrAKtmEFwrOzQ7PayqohwJ6zAg3zOso4umnVZWPd9sy9euf0KbFy4TAyRSITAwEC950xD8MfgP/BDOL9rUBPhcuzZY7g893KT64CC2nEkJjZthKiq4rq4dVmBGRu0KJn60B6fDvkUN1+5iZKlJYicE4l1T6xDa/fW9Z83SyD9AKB3gOZcP4bhNmZCi9jqOv3NgUvfXcJK15XIvsPP+50WxLTdk/Ir8hGdS6T2w8L61c+VDx+S62/lSjK/7d3LnRMXR8hAV1f1ijoHB6L4e/11ovCzMSDSpl07Qhbm5alfI1iJrdDGow0YhuGnbGQVQNCzgKseskN1KEsCsv4FSvhZclKlS2x+08nc3R0o8QwBG97a6AUTU8ybkzZNwqzjs3QfCKLAZKGAqyswejTQ2jMYgbkvYYLTh9jz5B44yTSxtOpB78eBgQ3z+jp0IBZwFFQVYgiCg8n9PT+frBEZhsFPY3/CmNAxqJRXovevvXE/nUx+vIs9iVuA+A2AkK59qTPQ+UuD7GQp6ReXr9mG0L21O/ov6w+3sIZKI0vL9ds2YRt2PrlT94FqcCf7DgZuHIi4gji0cGqBM7PPoJWL8fKtWRb47Tdg40by/8mTG16XFL16kX98QZXV9+4RF4/ISKLye+45gYO7/y2Qo6EjkA9S9wBnJgBVwi4GD1sPOEgdwIJFYiG/YmNtZS1qTp6Da/ods+4TG8+ZbuFu6Lmwp1oFnthKDJdWLg3WqqZEUVURgpyDmjQLxsYS5wJXV8tZhxkbvr7cZ2bHDtO/3vS20xG/IB4fDPpA8LkdZ3XEE+uegJ2n/mpkc0JZq4TYWgwrO+EdWlTl90LnF+Bmq1mleuQI+TpypH4NOdOnkzVedjbw8cfAcSLkMgvpBzRPAwIAwXE+mrBx4Ebsnb3XKM9FQZu67OwAmcw4a83L31/G9qnbUVnAv9FRHeykdpjSZopBz6EKukdqTPrJ5dw1QR17LAG07kr355T0y/PKQ/f53eEYYAEqBR7Y0HMD1nZYq/YxSvqpa3xKKEwQ1GBk6QgPB9zK+qMgnqzlGYbBzA4z0a76ZQBN6+zNie5+ZJ99O/t2k/eAz35CaidF4slEFMQ2nzsPAHi198KChwvQY36PJo+xrO5Mv9j82HpnBnUQiUQIMCN58Jj0a2YoVdvoeWL99fVw/8odbx17S9B51SXVqC6pFvx66uDo74gl2UswYLn2ziNNpJ9EQiYwoPksPqdsn4L5h+Zj15FcvPwysWZsTPwxDIMBLQbgy+Ff4vart5G9JBuFbxcicWEi9j61F691fw3paeRjZCmkn7u7ZovPzNJMXMu4hqSiJLOPS1+4BLsI9qHnS/pllGZg/Lbx2Fz0MhTiCvjXDkbMa7EYHRePjsnrIVbaNiHcaKd127bczUsV7u7A1KkNx0FVXELw2dBPwECELJc92Hj6dJPHK2sr8cv1XwAA4cULAFjWTf+tvm8h/vV4fDL4k/qfUTWELqWfW5gbvNp7NVBWKZVKpKSk6DVnGoqgwUEIHcujJR+aST+GYdDDrwekYvXenKoe/AsWACdOkHyqw4eJZZ1cTogW2vFvLtDMS9XPgZ3UDv0C+2Fqm6morrSqnzcFq11MhFHBozC1zdT6TJbGEJpTxbIs1nVdhz9H/mmkEQqDTxcfdJrTibfFcUqdO1+glui6cymk8BvhHgF3W3eEkH0E4uJIFh+9dxw6xJHQ1NozIsL0NiKq2ZVROuKIqWJTK/HiEAL02UTs7YyBkJeAaaWAe9PNgDqEuZEPkjrSj+aA5OfD6Eq/5pw3AfI7XW/1JBROcRCLiZ1h+5SfYX3xA1RXCd+C0HmIzkuqeO450vQ0ZQq/hh9dkMm4a/B6nbuLRCTB31P/RpBzEABgUypRjvIutMesBG4vB0TmzaoIdSX3LyH2ihR0vrQU0q+mvAY15cKszqrkVVh+cjm6rOuCh4UP0dK5Jc7MPlP/PhoDNMOPNnvNmUP+GWOupGv669eBn34i30+divp5mxeq84Abi4HEP/QfSHkSkHEIKOOfzQeQ9Q9V+/HN9RNLxUj4/QzcUm8iLc18rjCN58yA3gEYtXpUAwUsBatkUZxSbJT8tsMLD+PU+6e0HjMqZBQSFyZi/1P7G/ycNoW1bWs5Fl+mwJS6Gv6pU2RtbEpYgmLJXAgfH47X7r6GFv2FbVYVSgXaebaDg9QBi3ot0nhcUhKx5RSLgWHD9Bujvz+wdi25BsRi0oAmEplP2UprPM3VqF5QWYCTiSf1Pr/v0r7o9YbhOdSqoLb6Li7GW2vmROfgwf4HkNrrl2ORUpxikvx1usajdZ3610sh+3M7Oy3xAs0AugamdVhXG1e0cmkFuw52GLNmDLzaW8iGXQdaDm2iCxjDAAEAAElEQVSJ4FHBah+jzjTqlH5P73oa3l9740TCCROOznygDdiJiQ2jSej1aEn1vxZOLeBu6w65Uo5bWbcaPMaH9GNEDN7KfwsTfp9gwlEahpoaLk9Sk73np2c/RciaEHwe+bnax5VKJVLN2EXymPRrZuhzY3KWOaOgsgCX0vn7zKdeTMUXTl9o9KXVB3YednAK0NylrVBwih51kxFdqN27Z7Qh8caZpDPYe38v1l5bi59+KwJACofr12s/z9POE84y5wY/42ulZk70rXOaaWzxKZPIwCgZsFdYo9s8mAqjvxuNJ3c/KegcvqSfXCnH0YdHIRVJ0SZ1FbrcPY6cmFDk5XFy7bQ0EhZMQUk/mrOiDpMnNyQEKaEjBG082mCkJ7GdWpk5HEuOLkFJNZd8u/f+XuRX5qOFUwtYJ5FCtiXd9IOcgxDsGtxA0cjX3hMglpoladzvy7IsCgoKTLKY14XhK4dj1Lf8QijofEA3iEpWyWvM9vbEKlYmIxvk1avJ/3/6iaivJBJgxgzzF3SCg8lr5udzXZ2qoJ81F5eGypvmxJPtnsSOaTs0dnkKVfoxDAPPdp5wDW2ewIbWE1pjwm8TILXjtwnmQ/pRa09qGUQ3FKdOAb/XxbWJROQefucO+b8q6WcOqMv1UwdepJ+xYWUPiPmHcteTfgXqlX62RRkoWLoSl7+/bLQhAsafN0szSxH5eSQyozJ1HwwgJS8PWS67sbP2BZRWl6J7d8Dfj0FFBXD0qPDX10b6yWTAsmWEWDQWqLJa9Rq0l9rjytwr+HjQx+hUugyAgGtv0EFg8L/C7RXT9gP7g4Gcs8LOq0O90q9As9IPAP559R+s67quwc/oWsZS7D1nHZ+FZ/4RFleQXJSMry58BblSjomtJ+L88+fRwlmPbiwt+Pdfck2LRMRudvJk4z03VfolJpJmoIAA4KmnBD6JlSMw/BwQvkD/gYS8BEwvB9x7Cj61jUcbhLmFQa7klyUkEovw/PkXkNxzGqqrzXf9CZkza8pqsLrFahx/+7jBrxuzKwZJp5J4HdtY/UObZ/+LeX6qCA/nIhXWruWaMRqDZQn5vm4d+bdhA/m/OkcfXVAoFWrzoXXh2FvHsH/uft0HPsIQi8T4ZuQ3yPhfBlq6aJY5HTtGvvbsyWU66QOZjNzf16whav65c/W3VhMKuqej62tz4kHeA3h/7Y3xf41HaXWpXs/R/dXuBuXkqYMq6WesteYTvzyBd8rfgVgqPM6oVlGLfr/1Q/uf22t1NdAHtOE2uVF8a2KdcL1VK8tquKD1L7pH7xvYFw8XPMTuJ80UimokjFo9CiO+Uu9MRkm/xkq/B3kPcCX9CspqyuqjZh51+PoSYrmmpuEcpElc05xgGKbe4vNK+pUGj9EmV11NhM0Vp6KKhOMJuLnxptr8S7qWEIlIZExjKFkljsQTeXsvf/XNFizLopBOomZA8/9FH0Mw6MVzK+sWb4tP1xBXdHi2AzwiDG9DUdQocHvLbZ2djTk5hPiTSrkPuSoo6Uc7FM2FspoyzD0wFwAQkDsXtpWh6NyZ3KxPnAAuC6y58bFSMzco6RcT07BQL5PIIFKK4PahG86vPN88gzMxqqq437kx6SdXynEqkeukDXQKxJrRa3D1xWvoXLUYSoUIv/5KHhszhhS7WJbLsQK4wrc20s/amis8isX6E8IrBn0Fz6KxUDJyrLq4CkGrg3Ajk1Qfn2z3JA49cwgf9F4FeY0EIpHlKK00gS/px7IsVvmswr7n95l+UEYEy3LzAX3Pz6Wcg/tX7pi9d7bO8598khAus2dzc2ZgINnYbtwIDB1qilFrh0zGkUdxavZQtLHjUbKUot2YfDP9AGDiHxPxxC9GUomZEMXFxPKEYbTPO5EpxCu/f2B/AITcBcj1q1QCAweSbD6AFLJZlmt4MBfpR3NfHjzQrvKgxWCNxItSAZwcDsT9bNTxofgesWvkAaq00qT0q7W2R62TB2zdzFTB0hO593Jx8p2TSLvIr939csEhgGFRw5TAwdoBDANMnEge27+f65TkA5bl5iB1pJ8pQEm/O3e47BoA8LDzwHsD30NNNrGG5H3vtfUHXAQGGgGEsBFJgdoy4edCmNJPZCVqsMm1NHtPfRDuHo5VI1Zh1/Rd2PPkHvg4GDcAJTWVaxqcPVt/NYsm+Ppy3ewMAyxapEdWqtga8OgLOBtQBJPYARrcCnRhy+QteDD/ASa2nsj7HN8u3vAKJN1E5rbVozi65Ch2z1BfKJXIJOi1qBeCR6pXIgjBopRFeOaQZjKbZVm1BXW5vKEDyX8d06eTtbBSCXz5pfoc7LNnic3ugQPk37595P8vvAD8/TfJIuOD3TG74fKlC57bK8RHlyDzRiaSTiU1S6OiUFxafQl3t+lfjKHWgepQW0uayQBuTWkoAgLIHPiEGZfk6uw9a2vNc18McwtDS5eWKK8tx/bo7aZ/QZ6gtRZjx1KJrYQTfgCwPXo7UktSkVeRh0AnLV2PeoC6kz182HAtqEr6WRIak34U2bezsX3Kdjw8Kkytb4nQlOm35Q7Zl40MGdkg+/ZRBsNw+/T4umV8TQ03/1hSph8ALOmzBPue2oen2jXsTqP7icbxU41RnFKMm3/cbND8b27cWH8D++bsAyNqyuarWnuqI/tvZN5AbkUuHKQO6BtoeOazMfCY9HsEEeAUAD8HPyhYBa5n8lPu2XnYYdKmSQgdw8+iThtSzqdgz8w9uLFee/s9Dbb19VX/gWjdmjDkubnm7SB+/Z9FiC+Ih01NAMJSP0evXsAHH5CQcIB0kJXwnGOUSu73tCSln7s7uTmoFmoBQvopxApkvpyJri+ZORxMD1SXVuPokqOCwsUpoeTg0FByHZsfi/6/98fQTUPrre0A4KWuL6GDd/t6Eq+ggFyvo0dz6hdaaCwsJEQBw+guOg4YADz7LMm3kupXI0G7EBf0TvwH3eMOopVTKAqrCnE8gXQUixgRRoeORlcbomby9m6aL9jc+PrC15ixe0Z9hgslhvLzyWZJExiGQff53RH2hJkqu1qQfiUde2btQfrVdJ3HlpQAZWXk+qBdV1fSr6CgsqCBSlMb7O2Jfc2GDcAffxC7sAkTACdh0VdGhTqLTwpLy/OjYFkW8QXxeJDXNHNNqL1nc6KyoBIbB23EzY03eR1PCxKenuq7zwDytxkdMhr9A/ujf4uGpB9Aihvz53MFmgsXyHtfWkrmMnNtbgMCSNNEWZn2DQIlnjWSfpXpQP41oMzIwfXRXwAXZwLVunMHqNIvqSgJNY3y3NzcgFobRzwcOAcdZpopoEZP+Pf0x9wrc9F6kpauFxXcrjoAABjow1Xnhgwhm/ScHOBi09hNjcjMJBstqVS7itWYCAgg66maGk7xSqFQcBtuXqRfVS5Qka6fT6HXIGBcDOA3Rvi5ALr6dsWZ2WdwZvYZrceN+3kc5l6aC4mMW0xY0nxZllWGKz9cQU60gI6NOszvMR+TI4wov6tDTQ3JQK2pATp35khtY4JhuCaIKVO4takg1BQDPFV2WlEQBaT/Y/jz8IBSroS/dS4k1eXNZquXdTMLaZfVv7hYKsbIb0ai3ZOGS+wYEQNrB83q8eTiZHh85YFRf45qQCQlJJBmR3t7882LzQmGIeuT9u2JC8tHH3GKD4BMrzvrIj979iQk4dSpZC4vLwf+/JM00qlrYmuMAMcAlNaU4mLaRShZYbaFT+9/GgseLrB4m1CWZXH6g9O4vk6YE9SqC6twIfWCzuMuXybvj7s7mR8fVVDSLyuLzPUsC7z/PvD8803VX8YGwzB4ofMLAIANURv0eo68B3lY23EtLq3m7xKmC1Sk4mokE5Tq0mrc2XoHBfH6ZXnRv838HvNhLeHvxMEHXl5EpSqXN2wET0ggXy0pzw/gahCFhQ3dqioLKhGzOwYFD5s3L40P8u7nYd+cfUg6k6T2cXVKP5Zl8edtEsUxs/1ME4/QvGhck6T1Fzs79bmGzYkhLYdgfPh4eNg1FBvxdQ5JvZCKfbP3IeVcM0ir6zDo40F45uAzalXHZXX9l5ry/A7HEf/xYa2GaYz1MTcek37NDH0Xg1TtdynNeDdvvvBq74Xxv41Hm2lttB6nS3Isk3GFRnPk+qWlAXO+3IuNdzYALIOOiX+gW1sXvPkmKSzOmEE2TMXFxFKPT10mO5ssAKRSM1uL8QBdgKguRq0l1gADZA3KQtCgoGYZlxCUZZbh4qqLSD7Lf0WtjoTYcGMDOq3thEtpl+Bg7YDc8qZ3G1USr3t38n42vsFSArVFC92WIgzDdaTqCysrstHwKh6DTb3u4rfxvyHcLbzBMZR0NnfWGx/8dfcvbL2zFbezbwMghV6ZjHy2dN3wh30+DD1f5yykGIaBt7e32TfQOXdzcHvzbVTm6w4VVyVcKNFLrQ16+PHL/qKQSMhGyhLqBY0/B6qgJLulkX6fn/scoWtC8Wnkp00e06eIzbIsTr53EkeX6OFJaABKM0qRFZWF0kx+lj58rD0ZhsFHgz/C2Tln67th7exIQcbJiVgkymQkLyooiBD0v5D4UISFma+5wMqKa6ZR181PodPe0y4QmFoAtP/ImMMDQl8FBuwDxDY6D/W294a91B5KVomEwoQGj9FNUHl5w825MWDseVNqL4Vfdz84+OjeYdYoapAo/hcAMCaYI/2kUqKkB4C9e/m/Nm06CA423zXIMJzar7GVXF4eafyysuLZ6f5wPbDXH8i/ovtYI8PR2hEDWgzQS+FmSUq/nOgcHH79MJLP8FsTbr2zFTuid6BKXmWyMf3xB5mfnJyI+sRU9+yXXwbeeYc0k+mFa68D220BuYH5c9cXAheeNUvIXuKpRNSu/gmuGXfNRvo1njNnHZ+F1+NeN+lrlqSXIOFEAioLNd8AbmffRn5lPrLKshrM53T//F/P81OFREI+C76+ZF7atIl77Pp18nmUyYA33iCfl+eeIw10b75J9kqlpcB2HoKpTt6dYCOxQUFlgVqVvjZY2QqV4jYf5l6Zi9Hfj+Z9fGx+LN46/hb6/tZX59+F2ngPHUoavR9VODuTdTLLkvrW8ePEqYplzVPDmtVxFiQiCS6lXUJ0jvAXdPR3RFVRFRS1AuwVdEDV3tMYa828+3nYPWM3orcL//0qayvrSegn2wqLhOEDhuFcTmgsEctySj9LI/3s7DgFXGYmUFpdiq7rumLknZFYVrsM3V/t3rwD5IH82Hzc3HgTRUlFah+nAg1Vwut86nkkFiXCXmqPCa0tNxNOH9AM58aknyZxjSVCVemnbQnZYmALTN89HUGDg8wyLnVwD3fXKJYqLiZfNTXkH4o/BAAYHaL5vsowDLzMaNH2CN9+/xsQ6bkC6ulHiuGNvXK14cavN7Bp2Ca13rRCYOtui85zOqsNNVcFH59hmlNBLRNNhepqYNH7mdhSSmw9O1UswdvTB+P997nivFRKNuxiMXD+PNlE6NrT0k2on5/lTbgt6uJKVEk/mYTY5FTJq6CUGxa2bA64tHLBgoQF6LmQf35I4zy/uPw4vHTgJVTKKzGs1TDcffUuJkVManJeuAqXNnYs+aqJ9NNm7Wls0IVkWrIUczrPabKIUVXUWhpau5M/VEwe+YAzDKeI4JPrpwqRSARvb2+950x90fn5zlhaspTXwkNdvqe+pJ8lgRLicXFN50T6PlqatQT1k1dV9VKoZvrxrVsyDIOkU0mI2R1jVrsmz3aeeLvobfR9k589hCF20x99ROxl6bkMw9nU0TnQXNaeFDQPVRPpp9pAoLXxhmEAiW5yThA8egP+43k9L8MwWNhzIT4e9DEcpA0JMxsb0kTiknEX/8w7bNTry9jzZkV+Be815Jmks6gVlcK61gsDQro1eGzsWEKWPXjAf/1HST+9VE4GQF2uH9BQYcpr/efaHQidBzjp+SHKOQdcX0QUWyYCy7K48sMV3PiV+2VVSb/mdqrz7eqLWSdn8XIBULJKvHvyXUzfOR377hvfKry4GPj5Z2JTC5AcP2PbnKnCyQno3duAwrlbDyBwOrHoNARtlwG9NwIClU8FlQXosb4HvL/2hkLJr/Ds09kHAU/1Qbmzn9nsPdXNmdoK2dunbMepD05pfJwP4g/HY/OwzVptk29l3QIAdPBqqAanERn/H6w9VWFvT5xUAJKn+aDO1GHHDvJ19OiGbi8iEXFgeftt8v8bN4hCUhusxFbo7kfWknxUbapQ1CgQ+0+sRpWKpYBhGLiHu8OzHf/O5c/PfQ4lq8S4sHH1LgbqkJMD3LxJvje25bG5wTBcM929e1z2NWCenD9ve2+MCxsHAPg16lfB50vtpHgj+Q3eewk+ULX3NMZa06WVC6Zun4rWE4UXWS6kXkCNogZ+Dn71GcbGBt3/0DVrXh5R/EgklqmyVrX4tJZY40bmDcQWxKJCztPfuJkR9kQYlhYvRdtp6m9uVOmnau9JVX5TIqbA1sqy4xKEgtZikpOJ2pjW2S2t/kJxOuk0Pjz9Yf3aBeCia2pqOLWcOjj4OCBiUgTsvTRbR5sSLMuitkKzJRkl/RpbywJAfkU+LqeRrLDRoZpJP5FI9Jj0+/8EhZBAExXQArIQ0q8woRBpl9JQkq6/P25NeQ2qivh1zPIh/VSLyabE4cNARuVDiERAhHMnXPzsE0yc2NT+LCQEeOkl8v3OncCvv2ovclhinh+FOtLPXmoPNxs3hP0Zhm/8v4FSYdnEn0gigktLF0GTvmrnCwD8E/sPWLAY2GIg/p35LwKc1L9ZrVuTc9q25SxIaHExK4ssLpqT9KPdZI1hiSG+FBHuZIV8P4/zmOWb65d9JxubR2zGvV2kpU6hUODhw4d6z5mGwNrBGhJr3dISSvrR+SC7LBvJxclgwKCbbzfNJ1o4WrQgBfqysqbvm6Uq/Xr594KIESGpKAlpJQ0LaVRBWVvLLdz4YPqu6VgQb367JoZheIda81H6XUi9gMLKpuHRDNM0J2rw4IaqquYi/TTNf8XFZPPAMBw50QQJfwC5wgp2gsDD3hMAPh3yKd4b+B78HJvKst3cAMfch7j3+xVUFRpPlWTseXPfnH343PFzsErd7M/uu8Ta07NoLNxcG16/zs7AoEHke75qP0r6mSvPj6JjR9IMlp7ecP6jpB/vPZvPcKD7DySfTx/kXwYerAYKo/Q6/djDY3jr2Fs4FHdI4zEMw+D8yvO48j23t6Cfq6oqLseiuSBzlqHl4JZwCtDtd30h9QKSipLgIHXA+PDxRhtDTQ0hFl58EThU96ecOpU4RFg0wucDff40/Hl8RwP+EwCRsNwlZ5kzbmffRnZ5NlJL+DF4tu62GPjpcJS7+JuN9FOdM+VVckTviEbefc0y1+Szyci+lW3Qawb2C8ToH0bDu7PmhdTtHOKY0dGLywRlWU510s5wh9FHDu3aEQUZyxKHnrt3yd9DItFssxsURIqkNTXAtWu6X6OPfx8Awkk/APh78t+4+LUAD+tmQFVRFSryKng3GyUVJWHzrc0AgOX9l2s99uRJ8t506GB5ewR9QBs6N24kNQHKb5lrbprbmTSub7q1CdXyavO8qBao2nsaY61p62aLttPawqONh+6DG+Fk4kkAwOCWg022R6O1n/v3yXVNrT0DAiwvXgXg6kKZmYBULIVULIVVjRVij8XqbaFqTjAMA2tHa42q6caZfrWK2vrMy2c76GuJYLnw8CCqRrmc1HYb1zstDT9c+QEfnfkI/z78t/5nVlacOo6Pe0hFXvMQ1BV5FVhhtwKHFx5W+7g2pd/Rh0fBgkV7z/bwd9Sc/aVQKJCQkKDxcWPjMen3iKKLTxcMaTkEz7R/BnKeGQ0D3x+IZaXL4Bqsv/n2/T33sdJ9JeIO6Wbp+JARlFRJTCSTmClQVUUIPNeyftjY8w72zdwOmZVmr+8xY4BXXyXf79tHbM00rYXpQs+S8vwoKOmXkcGFDk+OmIy8t/IwbcA0+HTxQWWBkb3EjIyKvAqUpJUIUiU2tvc8GHcQADCx9USIGM1TnrU1ea8//5zr2rez467f+/e58NzmIP003Rfo58wS7T2p0k8f0k9sJUbKuRSUpHJNCqWl/CwOjYmUcynIvcfPB7Kx0u9qxlUAQIRHBByt9Sz0WgAkEi7HTTXXTy7nVFaWtqF3sHZAJ+9OAJqq/ah1KiDM4tPey15toLMp8fDoQ8QdjuNdkNFF+lXLqzHkjyFwW+nWxGZSHRwdSS4OhTnnPoCb/zQp/ai1p6urhk23oga48hJwt6nNq1FwdiJwIFSw6qUx3NyAjPAh6LHjf5C5yIwztjoYc95sNawVur7cldfn4FQyUb+0rH1C7XtDi7KXLjXMZFIHuZy7B5qb9LO15chuVbWfYNLPUATNBJ6IAzwH6HX6sYRj+OrCV/g3/l+txz217ynMPMploVhbc/ZJzW3xWVtZy4twBlBfmJ7aZipsrIyn8t24kTiBVFaSRsEVK4h94P87KLUEM6uBiBGhpQuZ0OML+Od007VtSQn/vHVDQefMouQi7Jy+Ezf/uKnx2Ddz38RTe58y6PXcW7ujx7weWm2T1Sn9YmNJM5ZMZr6sXUvDnDlE0ZeQQD6LACECNeWMMQzQh/B4uMCDx+sToB/pJ5aKMfnPyRj08SBB55kbNzbcwFceX2lVmapi3fV1ULAKDG81HD39NbvwsCxw7Bj5fvhwY4y0+UEbOqkN++zZ5Ks5lH4AMDJkJHwdfFFcXYxrGTwY60YozSzFyfdO4uHRh7oP5gFVe0/A8LWmIY3op5LIenNI0BCDxqANISFkn1FcTGoYlmrtSUHrV7ROZC+1h0OpAw6PP4zLay4338B4Iic6B1m3sjSu+Rpn+lmJrXDxhYv4bMhnGBQ0yDyDNCMYpqEDmSU3/QOc4xKthVHwjQw4vvQ4vvL4CqUZ5q/9sQoW7We0h2839X9cuhZVR/qNCxuHPU/uwYeDPtT5OmXa5I5GxmPS7xGFg7UDTsw6gS+GfQGJiF97icRaYnD3i72PPSImR+jswqmt5Ypx2iYjb2+ua0FbZo8hOHyY3KC9vYGpI30Q6qbbG2rMGGIbwjDAwYPA+vXqib/Gyh69USnQ65AHXFzIRkip5CwgKfq/0x8zDs2AnYeBNj8mxtWfruLbgG+RE53D+xzVzpfymnKcTT4LABgTOobX+Y0/IvQG+++/hDx1cDDvDZYuJrOymlrRyOUceWbppB8lLWiBNFtHY7RbuBveKXsHvd7oZcoh6sTOp3Zi7+y9vI5t3ATwX7D2pFCnys7JIfOitTVR7lga+gX0A6De4lOfXD8AyLiWgas/XdV9oJFw6v1T2DdnH697d2kpUFREvtfUiHIt4xqqFdVwt3VHS2d+O9VRo8jXVq3MHxZOlX5paWRd0Rg68/wYETDkGNBOe0e63vAYAARM5pWTJVfKEV8QrzaL2d0dqJU5oFRpb3YlqRD0XNATY9bwu5f+1usKesQeRoRUfcUvMJBcU0olIf60ISmJvP/29s3TYEAtPlWVIfTa40X6VWQAR7oDCRv1H4SNF+AQQq5pPUAtr+IKtDft+XT2aeKuoO98aWwceu0QPrX+FNWl2lUO1fJqbL9HOr5ndpip9VihuFxXK3vhBeCbb4D27Y369KZB8X0gciqQddzw56otA3b7kGYKgQh2IUHuDwv4F53vbbmBjmd/gLSiyGy5fhT2XvaY8tcUtJ3evN6Z5TXl9URpR2+i9JPLib0sQFSmYmHCy/8MnJw40r20lOzhJk/Wfg4l/a5e5ZpiNaF3QG8AJKagoFKYOqbt9Lbw6Wyh3mt18Gzvic5zO8M1lF9D+JH4IwBIxlxj3LxJ5sTPPweWLyf3SDs77u/9qEO1mW7QIGIhC5B1tzl6UiUiCf6e+jfSF6ejb6Bwm06lXInITyMRs8fwTB1Vez5j2Vr/OfJP/Bjxo17nvjfgPSzutRjDWpnOR9bKiqsJxcRwpJ+lNlyo2nsChPQrdClE2JIw9Jhn+XWJE0tPYH339YCGLVFj0g8Awt3D8U7/dyAW6ETwqEAd6Wep9p7UGvtqun6kn18PP3R8riPk1SZSBWmBvbc9Jv85GR2f7aj2cW1KPwdrB0xsPRGTI3QsRMwMCxQjP4apwCpZJJxIgNhKjKBBQXo9R6uhrdBqqO67W1YWKQbLZNqLwQxDOmeiosgERkNK9YVcKUdUZhTae7WHTCJDVRXw9tnX4Og8FAufnCJIfj9iBNlErV4NHDhAfo/p07nHWdZISr+iaOB4f6D1/0hBJ/kvoP9uvQs7FAxD1H7R0UQGbqmdSNrg19MPPRf2hKM/P5VUbS1XlPLxAeykdoh9PRank04j1FW/IKDQUODMGeBKndtVeLh58xudnEjHakEBKXyqKm1ycgCFguRRUp9sS0KIawhEjAjF1cXILs+Gt703b6UfwzAaF3rmxNAVQyGR6Z44qqu5a4/OBy2dW2Jw0GAMbDHQhCM0D+hCU1Xpp6qqtUSeol9gP3x/5XuNuX737wsvYl9cdRF3t91FxBTzeM2PWDWCt70FvR95eJCcOHWgTRD9W/TnTS516gR88knzkC2urmRDV1pKfr/Gm2udeX4iid7KKF6IWMz70BuZN9BzQ0/42Psg438ZDR5zdwfAKpEdU4iiZDGcWzgbd5zNgIoSGTxLRsFby623b1+i0Dh3TrsaQNXasznmmq5dgT/+AG7fJsUuqZS7h/Ei/aqygMoMoNZAqVJNEVAcDXgIL/jRNZAulRXLskRhz6DeRtPdnbxPza308+nqg+rSakjtpVqPOxR3CEVVRfBz8DPq/be4mCN7hw+3zPueWpTFA2l7AP+Jhj+XlT3g0hGwDxZ8KiWeHxbyJ/0YhoHECpDUVCA11bk+C94ckDnL0O4p7b6ZSWeSwIgYtOjfQu/XWdd1HVxDXDH176lqH7+bcxcsWHjZecHTjtzs/v4bePiQ3B/nztX7pf8TGDkSOH6c5Pr17au7MTM0lMxpeXmk/tBTs2AN7rbumN1pNlo4teCdRakKpUIJsOBt0W5uhIwMQchIfsWX7LJsRGURe+kRwSMaPMaywLffcjlvFEOHkvvlfwEtWxKll1RKFKYyGVl75uSQ9ak55qZ+gf30PtcpwAkv3XhJUH6jJtAGQysrrsHcUHi284S9t377qtGho7XmZxkLrVsTwi8mhnOfsNT6GiWDKDlkZ2UHVsTCZ54P3IIssGjUCJ1f6IygIUFq96o1NVwjvLpctf8qaC0mOppbj1uq0q+rD+mWTC5ORm55LjzsSPcgrVfq2k9ETI5AxGQzZ4rwhLZMP0uFXisQuVyO48eP45dffqmXcmdkZJhVovhfgaEd3QWVBbiecZ3niwE7n9yJU+8ZFjjOB6qWg7p+RdWuBUPAsiwmbJuAHht6IODbACw7vgzzN/2IOOefcSP4KQR10hAKpAVDh5LcDgDYvBk4epR7rLiY5JswjIETrswLcGoDuPUACq4DOWeBkgcGPCEH2pVGc/0e5D3AkD+GYMr2Kbi36x62TdjWLB0UfBEyMgSjVo+CrRu/MF6qPJLJuO6LIOcgzO40W+/PGr0+qdLT3JlWgOZcP6rg9PW1zOKTTCJDS+eWEDNiJBaSwauSfrocCwviC3D5+8sozy0HwzAICAgwuwqm46yOvLq8MzLI7+PgwF17czrPwcnnTmJ2p9mmHaQZQJV+Dx9yVsxUrWlp1p4UtBP2dvZtFFc1DO+jyhVawOWLPm/2wZxzc3jPSYYisG8gWk/g56nJJ89vf+x+AMItcDp1ap73mWE4tZ86NwB6DWok/SoyDLbeNBYo6ZJZlonS6oZt4W5ugFVVGQo+/AHnV5432msac94szy3HtgnbcOevO7yOp8U/TTZrANCvrn5186b2TvnmyvOjCAoiv0d1NXDiBPmZIKWfaxdgUjoQ9rphA7k2HzjWD6hpmsmpC5RwSSxK1BoLUJRYhNUtVuPC15ydHd/OXFOjx/wemL5zus7refNtYu05o/0Mo3Z8U4t3Pz+iYHlk4DcOmF4BBKonlQRj8BG91NNU6SfE3rPz853h/8V8VDj7miU7S+icuW/OPhxZeETv12NZFjZuNrB21hw7wYLFsFbDMKQluW/HxQHbiZAVr76qfY79/wCGAd5+G5gyBXiJhwBVqMXn7xN+x4eDPqwvWvJF9PZofGbzGeIOG1jgsBCkFKcgzC0Mnb0715PPFImJ5J5vbU2uyYULgWXL/lvWx66uwMqVRM1IP3PU6clcFp+q0CfXz6ezD8RWht8TVa09GcY4a81Rq0dh8p+WpY5pDFWrd9r4ZamkH61NFhQQgsxeSgjVspoyKOVKZN0yvsuYMdF6Ymv0XtRb7WN0vyASEQv+36N+x9TtU3E66bT5BtgMoOIY2nRtZ2d+Bx6+cJI5IdwtHEBDi09af8nPb45R8cPtLbdx4OUDKM9V7+KjSen35bkv8d7J93i5WTAMA38z5oMJJv2Sk5PRvn17TJgwAfPmzUNuXZv1l19+iSVLlhh9gP91iET6d37dzr4Nt5VuGL55OK+8H4ZhMO6XcRj6+VC9Xi/vfh42DdvEa/EqxGfYWKQfwzB4riNZXeZV5OGL81/g18z5AIBnApcixF2/u/L48cC0aeT7H37gNgh08+ntbWAXm8wdGBYJ+AwH2r4LjH8IOBmHWaK5fpT0q5JX4VTSKVxIvYC8+3mIOxzHO6/sUYCq1N1Y3FCrVg2fy9yZVoBm0s/S/bwB4PTs06h4t6LeIofe7CsquFwETUg8lYgjC48g9XwqRCIR3NzcDJozTYnGeX7/Nfj6ksVNTQ3pjAa4DY+lkn6+Dr74ZPAn2PPkHlhLGhbU9LWr8+nig8C+gWbp2lbKlbzzqwDunqTJbjqpKAmX0i5BxIgwpc0UI4zQPNBG+um09zw+APjXxDY2Md8AJ4bq7GJwsXGBhy258BpbLLq5EXvPys69ET4+3GhDM+a8WZZVhrhDcSh8qJ1wKqgsQLuf2mFD8ttgodBakPb15WfxSdeHzUX6MQxZCwLE7j0mhiM1NV57mp7IEATNADqvgj4yeD9HP8gkMsiVciQXJWs8zjnIGT3f6IngEZySi86Xumy5LQFypbxeSWZsa8/mvg4NgtgaEBs3L1Qogl3r7D0FKP0Abl2VrPmyNRpU58yTy09ile8qFKcUazx++MrhGPThIL1fj2EYPHv0WTzxyxMaj+nl3wvHnj2GrVO2oqaGKKqUStI00b+/3i/9n4KHB8lY42s1SEm/y5e5RjZjwznIGSGjQmDtqJnQbU6wLIvNwzfj4jcXeR3f3a87Hsx/gDOzzzR5jFpfd+pEIlKGDSN/4/+Kyo8iNLRhnAZtsjNHQwLFvdx76P97f/T5TbhvqqJWgfSr6ShMFN44pAq6/qGft+bco/9w5QccTziuFwkqFLQGRPceHh6WS7rY23Njy8oCApwCEOQcBBEjwo5pO/Br719RVVyl/UksFKrWngwD/Br1K3bF7NIr6/JRgqtrw3ucpTb9U6iz+KRKPz71l2trr2HjwI1Qys3bvJt0Ogk31t3Q2CChLtNPySqx5soafBr5KR7k6xbviEQiuJqxY0vwzLxw4UJ069YNhYWFsFHxj5o0aRJO0PbXx+ANhUK4VQRFa/fWkIqlKKwq5L2BajutLQL7aZEBaEHBwwKkXUxDTZkOA3xwBXA+OWOU9EtOJl3UhmB62+nIXJyFd1rthkcpITfdajth3QzD8nyefZZY+bAs8OWXwJEjRijyl8QBKTtJPgadsaVOgLRuNudB5OpCY9JPJiEb/ip5FXrM64G3C9+26LyBPc/uweEFh3kfTztffHyAO9l38MRfT2DjzY0GjUEm4/6ODNM8xR5K+lErCQpVRa2lwt/RH1Ixt+uTyTjrQWoPoglh48Iw48gMtBzSEgqFAvfv3zdozhSK21tuY03oGqRe1L2bazwfZJVlCc7/sGQwDDB4MPn+2DHyVdXe01KxfMByTGg9oX7uozAko4pVssiMyhREyOmD6B3RWGG3AnGH+HXE6FL6bY8m0oCBLQbC296C37RG0NT0AOiw91QqgIAp5J8pUZFCLBerdHfNhrmRG0hsfmyDn7u7A2AYpLUZwdtqiw+MOW96tffC8url6PuWdmvJ8ynnEZ0bjZsV+8FArFOF0rfu6c41cuHNzCTuCt98wxXU6HqxOTB5MrGBq60FPvqIcxXgZe+SuAVI/8fwQfiOJpayUmfBp4oYES+lFSNiMOrbUQgbyy12aHfxrVtGWZrqBaVCid0zduPWpltaj5OIJLj1yi0Uvl2I9l7GDdyjpF9zXod6Ies4cRIxForvAVdeBXKFqZJDXUMR4hoi2G7fNuU+vB5ewN27XI6UqaA6Z9p52sG5hTNsXDX4ZQNoM7WNURs1dGH3bjIfOjsDr71mtpf9zyEigvwNy8uJbbMuFFUVYeudrU1U+trg18MPT+9/GkEDg/QepylRWVCJjOsZKIgXtldxsG7KclDSj+bf/n9Bcyj9PO08cSntEm5k3miyltSF7FvZ2NBjA6J+izJoDFTpRyN8DF1rJp5MxKH5hwRfi0VVRVh4ZCGGbx6OvArTWxG4uDTc81qqyo9C1eJz1/RdSFyYiDGhY9B+Znv0fbuvyfex+qI8txw/tP5BY0MCJV0cHYGc8hycTyVrkafbPW2uITYLGKbh+tNS8/wouvsS0u9WNrdup84hfJR+JWklyLmbg6LkIhOMTjOeWPcEluQsgbVT04YdllWv9LuecR3ppemwl9rXuzJog0KhQGyssPnbEAgm/SIjI7F8+XJIG7XuBAUFIZ16zT2GWSAVS9HZuzOApiGZ2sAqWb0sHcPGhmFpyVJeVmO0I5+SJdrg5kY6F1i2KamhCynFKRj558j6zKacHODHlV64vX0Sej44joXyDNyYdx621oZ12TEMMG8esftUKoEff+SsVfQm/ZL+BM5NAwrVLLwS/wQOtQNqNHeX8gEt/ubkEGk/VbtUyasgc5ZBamfZLXiZNzKRfZt/azklIXx9gX9i/8E/sf9gV8wug8dBb7BBQaTIZ27QRWVycsOCG51yLZn0Uwe6SSjU0Wjo4OOAkJFcl2xVlfk70iQyCa8u3cak3xfnvoDbSjd8dPojE47OvKCZW1euEMLW0pV+2mAI6Xf6w9NY12Wdya1RbFxsEDQoCE4t1CRFq4Eu0m/HvR0AgKfaPWWM4ZkNmpR+LKvD3lMkBjp/CbRdZsLRAej4OTApE7DRvfvSRPrRzsfiYkIqGRPGnDcZEQOxVLs11MU0skn3qiFd6LpIP3UWn9u2EZu2NWuAU6fIex0aqj0j2tRgGGDRIrK+KK9zfPH05Nlle2spcPsD4w1Gj2wpAAh145fr1xjt2hEboaIikpvVHKjIrcCdrXeQfpXfXtNZ5mzU12dZjvQzNH/c7Lj0AnD5ReM9X20JEL8WyOXhjaiCULdQxL0eh53Tdwo6L/2fKPg/OAFFjYKXHaOhoHNmzwU98cLFF3RmSBqC9KvpOPvpWRQlFal9nGVZlNWU1X3PNV09/7zlKkweBYhEQO865zg+11Tf3/pixu4ZOBKvv5WrpcHWzRZvF7yN0d/rzkIrqylDjUJ903dZGcnIBv7/kX7NofRzt3XHsFbDAAB/3/1b0LleHb0w8MOBBjcq0P276vrOkLVm4slEXP3xKhQ1wtY2Z5PPQskqEeYWBj9H8xRDVGNeGueMWxooKUTrYxRtprTBoA8GwcZFc0NLc6K6pBoisUijkx0l/RwcgJOJJwEAHb06mu0aaE6okn6W7PQFAE+2fRJ3X72LHdN21P9MNS5AVxNh/3f74828N+EabF4Pc4ZhYOdhp9auuKqK26erNn3uvb8XADA6ZHSTRnNNMGddUzDpp1Qq1XZxpKWlweHx6tPs6OFHbKuupF/hdXzWzSx84fQFLn9/Wa/XE4lFOgs+SiWnLOPbAUM30EIJ70/PfoqjD49i+Yn3sH078ZG/eZPYSbzyCvDtxz4I9DFO9pJYTDzqZ8wg/6fZJpqs1HQieC7Q/WfAXY1ftbIWqC4AKtL0fHICR0dOBp6Swin9quXVYFkWxSnFuLb2GqpLTW+JoA9ei34Nz53iHwigqjw6FH8IADA2dKzB4+jVi3ztI9xJwyjw9SXXdFVVw4WbaqafpSK9JB2z987GhG0T6n9Gr0ldpB9ACh6FiYVml/YDQIcZHfDqnVfh2Va3fxu1+qAEGJ2TaY7SfwGBgUB4OKBQkEI8Jf0sudNMoVTg2MNj+PD0hw0KFpQkKi4mlqVCEDYuDH3e7AOZk2k7AEJGhWDG4Rm8rr/ycs5uR1MjyoGnD2DN6DWYHGHZmRmNERhIiJWioobq4PJyziKYkrjNAokNb38VSvo1tvd0cACsrADnrPv4uf1PSL9ieU10uTG5SDyZiNpK7awkJf0cS8jaRhfp5+tL1orU4nP7dmDLFvJY27bAk08Cn3wCfPGFwb+CwbCzA955h+QWATzz/ACg/26g62rjDOLWcmC3JyDX4Y+tBqtGrELqolS82v1Vrccln03G2k5rEfsPWZRLJFwxV5sNqylh722PdyrewZBPtXfQ8ok70Af5+WTNIhJZfqGvCbp8o1cGn0a4dAEmZQAR5on1GPr5UIR+8ypYRoRTpo+lF4TtU7ZjfY/1ep+fdDoJp947hdJM9QqynPIcOHzuAP9v/HEnWo6cHNJ82Fe74PoxeIDu6a7zEMHSveTeB3sFvcb5ledx7O1jAkdmXvCxq//+8vdwW+mGFZErmjwWFUWKtwEBAu2u/wOg6+38fK4ZyBx4su2TAIBt0dsEnSe2EmPQB4Pg190wcoSuxfna6erC4E8GY8HDBXBv7S7ovFOJ5IYwOGiwcQbCA6qkn6Ur/Wh9qDHpR6GoUaC6xPJqgK7Brngt+jX0+Z/6wptqZvixh2R+pUT4fx2qTWeWXH8BAC97L7T1bNsgW5s2uVZX654zrWysjJJJLxSJpxI1qgupyk8qbSgEoWuDia0nmnRs+kIw6TdixAisXr26/v8Mw6CsrAwffPABxowZY8yx/f+AgZvTetIvgx/p59TCCX49/ODgK4ygVcqVuPz9ZWTf0a26ysoiH2SplP9kRLsW4gU0H8cXxOO3qN8AAI7XP8HmzaR427498P33wNixxvc5ZhjgqaeAxYtJEQQwYPNvFwCEvgKIJE0fa/UcMOY24NxW77FSqFp8UtKPBYtaZS1idsfg4KsHkRLZDAnUPCFksqeLGlu3AlxIJa2bo0N0dzDqQo8ewMaNwPTpBj+VXhCLufeRWtxVV3PEsyWTfjKJDH/c+gP7H+xHeQ25u1O1hi57TwCI/CwS37f6Hjl3ckw2RmOAvhfu7kCtohY3Mm8A4Obo/wqG1a2p9+0jJDTDWPYmX8SI8PSup/HRmY/q3xOABH9Tm1mhaj+/Hn4YvnI4XFoZabdrBNAuYzc3Qkyog7e9N+b3mA93W2Gb6uaGtTU3x6mq/SjR7uTEkTANcOFZ4Morph4eQd4lIPpznWs6TUo/hiFzB8uIUFNei8pC4YSOqXF93XVsGroJ5Tmad2lypby+4UGWx4/0Azi13x9/AJs3k+9nzyZE38yZJCfIUrKBWrQga0AHBwGFd7fugKeRwresPQCXjkC18HtiK5dW8Hf0h4jRvv2T2ktRmV/ZIO+FqmKai/QDSAFAV7NFm5/aYNDGQUgsVOMHbACoyq9FCw3zjSUjcAoQYMRmD7GUKJsN2GQpBKhVPdt6YsQz7gDD4O5d/RT6+uDk8pO489cdrcdYO1vD1k3/5tKuL3XFS9dfglcH9R0E9F5hJbbC+UiyX/wvZqU1B2jtIS9Pd/GRFvIOxh7UqHhTh7hDcYjaYJiVoqmQfjUdDw480NnIAwD/PvwXZTVlahXUlDTt1s3IA3wEYGfHFbHTDOvTFoSJrSdCKpbiXu493M25K/h8lmUNaqalTbvGIv0YhoFLKxcwImH3lFNJzUv6WXoDkKq9509Xf0L39d3xzcVvAAD5cflY5bsK51cKs+m2BHCkH4tjCYT0G95qeDOOyHx4lJR+6iCVcgq5PB6OvGmX0nDlR348hzFQW1GLTUM24eS7J9U+rs7aMzY/Fvdy70EikmBMqGXyYYJJv1WrVuH8+fNo06YNqqqq8Mwzz9Rbe3755ZemGON/GqJLhoXMU6/cqMwo1Cp0L9psXGww68QsdJjRQdDr5Mbk4sjCI7j7l+6FBSUlWrQgHbF8QCewuDjtx1GwLPDGno+hYBXwKB4FNrkfXF2BN98EPvvM9HaHgwcDq1YBb70FBAfr8QTlySTLTxMYESCrky6UxAGZ+ncJqiP9AGLxGTE5AtN3TUdAX33liqZDRX4FondE8w6bVig4q7eYmn+hZJVo69EWLZx5eMzygJsb/+vZFKAWd/TzRQlOOzuemULNBDdbt3qSgRYvhCj9ggYFofu87pDaS9GqVSuzhoSf+/Ic7u26p/M4pZL7XdzdgdvZt1GtqIazzPk/pfQDgAEDyIKNerG7uRGFkqWCYRj0CySMArWBJj83zOITIJtmU+YhHHjpAG78ekP3geBIP72V5xaOxvMfwJF+aklnliW5U6XCbAz1RuJm4NY7QJl2j/KuPl3x/oD38Vaft5o85uYGFHuFoceWhUbL9ROJREabN9s91Q6jfxgNe297jcfcyb6DitoKOEqdYFNGKiN8LDkpeUY3UzNnAlNMHMVoCPr0IWrEoUN5HFxbCggoEutE64XA0JOAnXHWNurg08UHi1IXNdgrdOlCmt3S081b3KQoSipC+tV0rQXqjNIM3M+7j8iUSHjYGVf+S5sSHzlrT1OFMFZmAxlHAKWwuIgvz30J95Xu+PD0h4LOc5RVo703uVmfOSPoVEGgcyaUQOSKSNzffV/r8RN+nYAZh2fo/XoyJxl8uvhojFug6+ZQ1zBERpKfDRqk98s9hgrs7DirMV2ZbD39esLLzgvF1cU4k8T/Apy2fRoWZyw2YJSmw7W117Bt/DbUVmivH5VUl9Q30o4MHtngMZbl8vz+P5J+QPPk+jnLnDEqZBQA4RafGdczsMpnFa6tvab36zcm/QxZaxanFiP+33jBrlOFlYX1WWGDggYJfl19ERhIrvWePQW4PTQTVJV+WWVZuJZxDQmFZJ/iGuyKoIFBsHGzPIvP5LPJuLb2WoPGM1VQ0q/aLh6pJamQiqXo38JIjXUWDicnsh738eH2xpaMk4knMXP3THx78dv6n6lafOrClTVXcHj+YVQWmK8ZdsyPY9BhpnquhFrLqpJ+++7vA0CaD/hGC4hEIgSZ8Q0UPDP7+/vj1q1beOedd7Bo0SJ07twZX3zxBaKiouBpye3+FgpGZGXQhizULRRO1k6olFciOjfaiCNrCJdWLpj570x0nNVR57G0E1/IdUxJv/R03d12tbXA6x/H4GAq8X9ql/MJJk8Gfv6ZFKTNpQJu1Qror+/95doCYI83IK/QfpxSDpweA5ybrne+nyrpZy22hrXYGk7WTqiWV8Mp0AkRkyNMblOnD7JvZ2Pn9J2IO8SPCc7NJcSflRWwJ2ETAGBC+AQdZz06oN1kNPcyI4N89fMz3zWvL1q7kxzQ+3mkeCJE6RfYLxBjfhgDj9YecHR0NJvMX6lQ4tTyU7i96bbOY4uKyLUnEpHf7VoG2Uh19+3eLLYEpoStLafKAR6NPD9K+kWmRDb4uSGkX+LJRKwOXI0HB0wTcFVdWo0b62/wVmFry/OLzonGkD+G4Peo3404QvNCXa6fVtKPYYDR14HBh008sjqEvQ6MvKKTiGnh3AIfDf4IU9o0ZbRotzifcHO+YBjGaPOmf09/9JjXAxJrNe4EdaDWnh3deoKBCE5OnCuCNvj5AWFEBImnniKWnpYO3n/SeyuB7bZAsXbywByorK3EsuPLMH3HdMgFkjW2tkCHuj1wc6j9on6LwoYeG1CYoLlb6HIaiS5o69EW9lLN5LQ+oPED9Dp9ZBC7BtjjBxQYWW10/xvg9GjBjRUSkQT5lflNLI61gWVZ/BD+A1xO7QZgWtKPzpkiiQgLExdi+NemVQ8UJRVpJV0o6eesCENpKSk0dRDWt/sYWqC6R9YGsUiM8eHjAXDZPXxg52mn9Z7ZnOi9uDfGrRsHG1ftRf9TiacgV8oR4hqCYNeGnc4PH5JmHZkMaNPGlKO1XDRHrh/Q0OJTiK21S0sXOPg6GJRVSvfvdD9vyFrz/p772DJqC1IvCPsDqsZoeNmbj30TiYAPPgCWL7f8+gtV+uXlATIxWROV15JCKyNiMH3XdPRepCZmqJkRvSMaB189iNpy9fdGSvqJ7ArQ3bc7+gf2h62VceKcHgV89BGwbl1De0lLRUJhArbc2YIDsQfqf0b3u3xIv95LemPOuTmQOpjH3sDK1grdX+uOkFHqO/zUKf3kSjlcbVwFWXvSOdNc0GsVIpFIMHOmYQq1xyBQ9Nxo0B1DxIjw/sD34SB1gJ8DP3lb4qlERG2IwuBPB8OlJT9dvtROiuAR/CRttBNfiM+1oyMp3OXkkG7ajlq4xcuXgd0FHwKuSnSynohda7oZzV7AbPAbBziGARIdNyiRBOi5HlBUA1In7cdqAF2MpqSQCaZqedOumdKMUsicZbCytRzJjkcbD0zZNgU+Xfh5xNYr33yTsffhvwCA5zs/b6rhmR3083T/Pim40SK/0VStRXeA5L9J7ovYuKuICPcInEs5h5i8GADClH4UCoUC9+7dQ5s2bSAWa88VNQYYhsGrd18FeOyjaJHe1ZVsBm5m3QQAdPHpYroBNiOGDwdO1rkeWLqfPAD0DyTdGZHJkVCyynprO0oW5Qh3yYNjgCOs7KwEh87zhbWDNZaVLdPZgU1BiUt1JOy2u9twKukUHKwdMKfzHCOO0nyg8x9v0o9CZKZ7mlNrg5+Cdj6mnXmIyJgM9H/H8K5Vc8+bLMsiwDEAre17IxP8rD0pli8n9vCtDf9TWhac2gB+4wH7IOM9Z8a/QPwvQM8NgDX/P7K1xBqrL69GlbwKnw/9vEkBt8FLXMvAgwMP0HtRb8icyZqgZ0/gxg2yDp861eDfQhBCRoXAytYKToGa18K0CNjTr6dRX5tlOaWfqrXSIwErR8DGF5AZuTE3YDK5pq3dBJ0W6kb+gI0tjrWBYRj0XtwbCpEEF86ySEpikJRkmi531TnTuYWzzuPv77uPgrgC9FmiX+j3L11+gVcHL8w+PVvt47EF5O9Uk0nY5gEDiOX/YxgHgYHEnpKPSmti64lYf2M99j3YhzVj1ui0SQaAmvIa5EbnwsHPAY5+lmXL4tnWk1dm9L91e+rGKj+AU/l17syvwee/iOZQ+gHA+PDxmBA+AZMjJkPJKiFm+E0MNq42ePnGy3q/Lss2VfoZstYMGRWCkatHwr+XhkByDbiacRWA8e/3/yU4OBBFc3k5oKwkpF9ZjRanMQtBnyV9ED4+HHae6vMqKOnXr2VPvDbhiuAmtscwH6gr4fXM6/U1GNp0zYf08+lsWYUmSvqp8nXL+i/Dm33f5OW6SKFQKBAdbTrBVmMIvj1v2rRJ6+OzZs3SezCPoR8W9xZmG1GcUow7W++g3TPteJN+pZmlsPe259XBow/pB5Du2ZwcYvGpjfTbf+kuMl23gwGDTXM+fvQIPwAIeZH/sV6DDHopSvoVFAClpWQBoIrbf97Gnmf3YPru6YiYFNH0CZoJ9l72aPdkO97HU9LPybMEQ4KGgGEYrQWtRw2tWpFO+9JSYmFLYTQ/bytHIH4toKgEuqwy0pMSUKUfJf2EKP0A4MavN3Bp9SV0+8F83jGMiIF7OL/sM7pooZ1LUVmko76zd2dTDK3Z0bYtIfsyMx8NpV9X365wkDqgsKoQt7JuobMPeV8MUfq5hbph/v35RhxlU0jtpBotvxpDNdS8Mbbf2w6A6wp+FEGLuykpgFxOikv0fVNL+iVuBkRSIHC6+VpxlXKg6Dbgqp3sTytJQ3RONIKcgxDuHl7/czp/FEXewckbt9DtlW46O/D5QKEwDjH9Y5sf4d/THxN+16ygn9djHub1mIcjR+X4EcJIPxcX4+XDWBSCnib/jInyJCB9H1D4GuA9jPdpIkaEYJdgROdGI64gTusaKeV8Cs5+fBYBfQLq7WZ79iSuGg8ekKKfOd+vgD4BCOij3b/4cjpR+hk7SzcrCygrI04SLbSLeS0PrWaTf8aGe0/yTyBCXQnpF1cQB5ZleStDKKkWWQVcvAicPk1yP00BhUKBqqIq1JbVwsHXAWIrzUXs25tuI2ZPjF6kn1KhRNeXu2olsuPyiSIy70EYXAAMHCj4ZR5DC/gq/QBgSMshsJfaI700Hfdy76Gdp+49aub1TGwcuBEjvx2JXm/0MnC0xkNNWQ3EUjHEUt0EzYnEEwCAEcEjmjxG8/y6djXq8B4pNJfSz15qj71P7TXviwKoqCCuW0BD+3Z915puYW5wCxPWPAIAS/stxfjw8bzJzv+PYBhSJ4qLA2rKmpJ+FXkVODT/EAL6BKDnAsshT51bOGttuqF7XroGlYj+n3YcPAJo69kWNhIblFSXIDY/Fq3dWwtS+gFkrVSSWgLnIGeTjZPiyo9XcP2X65i2Y5raOqA6pR9ArkGh16Gx9ud8INjec+HChQ3+vfbaa5g9ezZeeuklvPHGGyYY4n8bTMJvQFTTbBdTos3UNlhavBRhY/l51ChqFfi+1ffYMXWHzmPLy7nue6EbYz65fkolkBIVhg6Jv+K50P+hvVd7YS9iCWD1CE5W1gLZp4HCW4JPtbHhiqLqNjV+Pf3QaU4ni+tAFApK+nXybY/js47j4DMHm3dARoaNDbBmDemuV73RqLPz0+8F/IGgWUD4AiM9IYc2HsTzhYaNC1X6ySvlqC6uRmWW+fy8KwsrUZFXwcsyhS5aqFLn5a4v4+WuLxu98GgpYBjgxRdJmPmjkC0jEUnqvf5p6DtgeKafKZF3Pw9ZN7OglPO7X2gi/RILExGbHwuJSIJxYeOMPErzwdOTzIFyOVE7A1yGq1rS785HwO33zeu9c20+cKQrybrSgk/OfIJRW0Zhy50tDX5ON0GlHfrhlVuvwNrR2lQjFQxWycLW3RbWTvzGVFxINj5CSL/HEICgZ4Ap+YIIPwqqtKJkgia0nd4WL11/Ca2Gtqr/mZsbWauzLHDliuCXNikUSkW9tXZPf+MWr+i+pGXL/79qFo0QGFHRyqUVRIwIZTVlyC7XPleqw4B+pEhy5ozp4goBIGZ3DL4L+g4Jx7TntA79YihevPKiIHs9CpFYhGGfD0P3V7urfVyhVCC+gEhMrUvD4OPzCNrLWjiEkH4yiQw7pu1A2qI0XoQfALhHuGPIiiEI7G+szZpxcPWnq/jc8XNk3sjUelxBZUG9Kpda5VPk55MGEOD/N+lHlX45OUCV+ggyi0N+XD72v7gfD48+FHwu3bvb2ZGMd0OgqFHw3uc0hkQkQQevDmjr2dawQfzH4V8noCzNb0r6WTtZ48H+B8i6ldUcQ9OI6pJqjffU6mpSb64RF8LKvtTMI3sMoZCIJPXN1lfTiTqX1sv4xln8OfJPrO20Fopa05NkSrkStRW1Gu2PG2f6KfWp6zcDBJN+hYWFDf6VlZXhwYMH6NevH/766y9TjPE/DSbzXyDuJ2LfqCfkSjkikyOx6dYmXpsOqZ1UUDGptqIWned2RvBI3aopam3g7t5UUaYL4XUN71evArc1xGg9eACUl0gRUfU81k//StgLWAJYFjjYBrjyqrDzKjOAE4OB+9/qPlYN6KYmJQWYu38uRmwegZhcorpyC3XDhN8mwK+HsXwijYNDrx/CKt9VqCrit4KmpB+1G5SKzeP9bE54egLPPQf8/jvw5pvA3LlAb0Ot2K+9DtxfDTAioOs3OjOp9EF7z/ZgQIrvCqWinvQrKuJXtOn2ajcsSFwAp3D9LG71wdWfruIrj6+QeV37hhjgFi10ETOn8xysHbcWLV0Eyp0fIXTvDqxcqcNa0YIwOGgwAE4JAhhO+pVlleHgawdxZ+sdQ4fXBOe+OIdfOv+CmrIanceqWu00JlmOJRwDAPTy7wVH60e3sYNhuAzdTZvI76zV3nPQYaDXb2YbHwDA7wmg7XKdh4W4EtXUw8KGxRY6f+Sw7vDq4AWRRPAS3WRgRAzmnJ2DUatHaTymsrayfg1KSWg34c3b/y2UJwNnxgPp/xj3ea0cAKmzXqeGu5HFti57RQcfB/h08WlyHfaqE6uYO9dvfff1OPia5mau+3n3UVpTCjsrO7T1MG4RkJJ+j5y1p1JBMsTTDug+Vh9ETiGNDgJgLbFGoBMhQHQRz41x9M2juDDlG8gkcuTlcdnWpoBHhAd6LeoF99baHR/cQt3g283XJPnNlfJKPNfxObTCUNjUBGLgQMvPkHrUEBBA/qbFxVwHvzaMChkFP0f++2U7Dzv0X9Yfvl2NZctiHDi1cELQoCCdCiuWZfHxoI/xQucX4GrjqvJz4McfydeICG798v8RDg6c4s3caj+5Uo6r6Vex+dZmQeexShZRG6KQck64Jyndb6iq/PRF7D+x+MLpC5Ploz8G+XwCQHZqU9JPbCXGm7lvYsKvmh08zA2WZfG119fYNn6b2sfp/iLN50cE/OCKd0+8a8bRPYY+oBaf1JKX3i/4Kv3aPtkWPeb3gLzK9DauvRb2woL4BRrFMI2Vfp+e/RT+3/hj1QXjuqQZG0apKISGhuKLL77AwoULjfF0/7/Q5RtgcjYg1r+jW6FUYODGgXhu73PIKecXTlQQX4C4Q/w2WzInGcasGYOuL+ne2Olr7QmQAOjevYllwCefALFq6hFXrpCCUteuj2i3bW0xUVRZ2Qs7z64F0O0HoLUwK1cK1Vy/yJRIHEs4hrwKnjNtM8HRzxEuLV14E9RZWUCGyw5InHWTNI86rKxIrseECQYWAGqKgMyjQMbBhgrUzKPAyRGAwjgti74OvihdVoqYeTEQi8T1N0q5nHRr6YJILIJIJEJ4eDhEIvMUwn06+6DLi13g0kq3f1lje8/HsDzMaD8DN1++ib+mcM1JlCzKzdVPMWBla4Ub62/oVALog/Yz2mPo50Prs7S0obwcqKnjBhvb7R19eBQAMLzVcGMP0ex45hnSVRwTA5w6RayOAY68bQDHUMCjr1nHB7+xQMdPABsvrYdRS0Wq4KCg12NBAVBRXIPiVB4VSB0w57z58ZmP4f6VO7679F0T653/tyhLADKPAJUmWJdUZgMJG4FKYR3aYW5EKvQgX3eRTVGjQPqVdLBKboLsWSeiu3WLdFybA0qFEqySbTCOxqhV1mJM6BiMCB4Bsci4dl+PLOlXkQLErgGyT5rm+a09SF6gwE5nVYtPIbD3todPZ28EuFUAME2Bnc6ZgX0DMfKbkTrXgIpaBapLNasStOHeznvYOm4rcmPUdx7ZS+3x9aBf0D7qOBiIH1t7mgDW1pxNPR+1338F7Z5sh5lHZmpUM1C42brhvYHvYcP4DQ1+fuoUadKWSID5pnW6fyTQXBaftYpa9NzQE7P2zuJdAwSIpeai1EUY/PFgwa9JozlU13f6rjWt7Kzg39sfriHCbCH+jf8Xz+19Drvu7RJ03v9HtK3rgcpMcoSbjVuTBlC+MRLmgrJWifYz2qPlUPXFZEo6F7och1wph7+jsCzIxzA/tJF+fJZOXV/siiGfDoG1Q/M74DTO9LubcxfppemCG79EIhFCzbipMFoVQCKRIMOULXf/VdgFABL1IaV8YS2xRoAT8RZo3DmuCcfeOoa/xv9ldMackn76hKszDFEvdepE7BE+/LDhAjy5KBlvxLdGsvsv6N7dhJ4upoTUGRh6HOi0Uvi5YfMAlw56vSwtJublEXsSAKiSc4ROyvkUrOu2DrH/aO/8Nif6Le2H588/D0bEbxJNLcpAVKunMe5EINJL0k08uv8IpM7AyCtA322AapEs7xKQf4X8MwIYhoGdlJvnpFJiCwLwt/hMOZeCC59d0Fr0MyZCx4TiiXVP8MrUoko/NzfgSvoVXMu41uDz9RjNDx8HH3T07ggRwy17XF3JfUcu538dqsLa0RoLExdi/G/jjThSguDhwei3tJ/uA8GN3d6+odWOQqnAyURS7FWXxWLxYFng+iIgdS8A8vmaNIk8tG4d+Wpnx80l9ShLAGot1/KlXulX0HC95uRE3j9WyeK7wG+w97m9Rnk9qaH+SwBy7+UickUk8h5obha6mHYRBZUFsJfaP1b6UXgNBqaXAy2fNf5zZ58ELs0hpKIAUNJPl9IPAE68ewIbem5A3n3ufQ8M/D/2zjo8rir94587E3fXNmmStkndXahQWqDQFqcsUlwXWRZnl/3hsPgusLi7FQqFttSFuqSeJk2Txt1t5PfHyY00I/fO3ImUfJ4nT5uZK+dm7px7znnf9/sVfWdTU6u0m6vR6XXcuPNG5r9pXaJ4ZNRIfl78M99d+p2m566uhvTmr2qPk1b07QcLs2HQva45/vg3YcYyoRShgil9pzAncQ6h3uo6iMl/m8yVK66kz2Cx2uKqBXY1febyO5bzTMAz1JWol58vO15GxsoMm4tFmzeD0SiSafv0rm26BDUSnwA/H/2ZuZ/M5eU/Xla0/a93/8p/B//XocBwd6S0tHUMtnixhjYTPRhZ4jND+xxAm3i7e7ckkskWGkqQJImAPo6pf1ir9HNkrNl/bn+uWnUV4YMsZe9ZZ0X6Cj7a+1HLHKcX6/TtK+aHfuUT2bKomHXXrGv3fkNlA6mfpZKzrXusnek99Jz/zvlWPVBLS8Ggq6bAYzMAc5J6flLr6c642HHoJB1GkxGz2dwyN6yvFx6hSjEZTA7LAStly0tbbK6Hn1rpJ/e7jiiMaDE/V4rqoN+PP/7Y7mfp0qW8+eab/OUvf2HKlE7Oqj4NMJlMUJUOJ75y6jhJweKBf+oikjXG3jKWhR8uVDQA/fqSr1l5/0pFx83MFP86EvQDUcH08MNC6rOqCv7xj9YF9f/7/XnK3Y6SF/I1Y8b0cH0TR8uzTAaoUS/FIHeupaWWg35eQV5UnKigtlhFz9uNqKuDIz7vY5aMjIuZoEp65U+PRyB4nrLwMuRhOD8dIqa77LTyZEHOGLTH4e8Ps/HJjRSkqveAcTVt5T0fXfMo494ex0d7P+raRvViF72+tW90VOIzoE+AS2S91CDff6dWVZXUlTA6ejRRflGMjRnb+Q1zFmM9HHkZNixqeenCC8VAW64Qtijt+ce18EOcY/65zrL9Nvjdts9aYrDwSCupK6GsrjXaLEnN1yNJJFw6nuQFyU43x2QykZqaKsaaTpCzLYfVD6+m5KhlAwaDydCSwTmp7ySrHpN/SnTuoLdfsaua6LNg8mcQo86rU5b3LKwppNFoWz44ZUEKZzx2RjvFBUmCYc122qnaKxt3K8xmePllsTARHQ2xPW1oKUng0wd8ulfD/znjn6y4cgULUhyTFJODX64I+sl95k83/cTS65ba3b7v5L6MvGYkOnf1edRT/j6Fh+setiqxmFeVx+oNYl4my1v3ogKzCTI/h8Ov2NxMbdDvRMUJVqSv4OuDXyvaXtJJuPu401TbpOwELiZteRpfLvrS7nzKbDbz05GfyK3KbfMa/Oc/Ygw2YABccIGrW9szGCys6/n559Ykkc5C9pdUE/QDEew58uMRStNLVe0nB/3azjm0GmsqZVuuSEoeHzu+U87Xk5GkVonPgwc7vl9XWsd3V3zHng/3dGq7HKWkBMr8NmOSmogPjG9ZA++l+zIgZAAVD1Sw7YZtSJKEp2erDZhSic/MtZm8GPsih3847LJ2mowmVv59Jbvf3W11m7aefg2GhpYESqU+vy3nMpk4cOCAw21Vi+oR6sKFC9v9XHDBBTz22GMMHz6c997rZP+U04U998Pmy53KTG8J+ims9Euak8TwK4bj7u1ucztjk5G8XXmUptkfEJjNrUE/R+Q9Zby84J//FIPw0lJ49lk4WZ7PxweEtMSZng+r9gvsNhx7Gw487ZiWnNkMvwyDdeep3lVeeCspsRz0Cx8czr2F9zLympHq2+Uifn/od1I/V7aiVFwMuSFCtu+msTe4slmnDzvuhD+uA5OFSahO3xoItPS+A2w4sYFp709j8beLgdbJgtIKq9E3jWbGFzMIH6ouG9ARzGYzH8/5mC0vblGwbVt5TzO788RAYVTUKFc2sRcHOFh0kKu+v4olS5e0vOasr5/JaCJ9RbqmfhT5e/J5JeEV9n60V9H28nfo1KqqCN8IVl21iuy7s3HT9UA9bDdvuMwAi1ufl97ecMUVrZtYDPrFXwopd6uuPtGExjJoLBEJOlbw8/Ajyk/oiZ06ZotsVgYNvXgWE++0nOXaFaQsSuGGHTcQP82y5+u+gn3UNtUS5BVEcmiKxUWhPyVZ30LZHtcc2zMU+l0OXurMlMJ8wsi8M5Pqh6rteh/HTY1jxj9ndKgIGN4sOmHNf1trCvcXsvXVrVRkWZa8bTQ2klelvYTq99/D1q0iIfH++6GT1MW1o+Ig1Lowe9/UBKmPw7G3XHeOU8hcl0nRK5/hWVPKyZOuO0/hvkIK9thPMhtx1QgWvL8Ar0DHAvuSTrKqaHLlt0t4zt2XkyGf9gb9HOXg03DwGTA2axFvvx32PwltEh7UBv3OTxbqDluyt5BfbV9eee4Lc7lxx43dRkav+FAxR346gk5vu0PLqsji/C/OJ/7leOqaRCXr5s2tsp533y2S53oRQfnx44VyyDPPKLOu0Iqh4Y4F/fL35PPFgi848JW6hWetxncF+wr46qKvyN6sLnvDYDKwM3cn0Bv0U4oclLYU9AuMD+TCLy5kyn3do3jn+Jrj/HD1DxTutyxXW1oKVd7inh0fO77LE297sY8kSfh5tLe2ktcslAb9QpND8QnzcWnFvCRJ3LjjRmb83wyL7zc2iiRAEPKeR0qOYDQbCfIKIsa/e/n2norq6YvJZGr3YzQayc/P57PPPiM6OtoVbTz9GXgHTPkSJMcX5eTS/owydboC9qTy9O56/nrsr1z42YV2j5WfL74I7u4Q4+R97+8vKv58fIR/z63vv0GTuYHg6klcOHqGcwfvStLfEf4ajjygJAniF0OfRWAyqtpV7ljLysCzOeO8wdhqxiJJUrd6aBobjWx8eiNHlipbSD+UnS8GAGaJ+QPVZb3/KTGboXyf+NHZCPzn/AxL+0GZ8yt7kiSxMWsjG7M2Auor/UL6hxCYHNgp92ldaR25O3MpSbNc1dKWqiohcwbQ4JFHUW0ROkmnOuOnF9djMBn4eN/HfH3ga5qM4kNzNugnSRLfX/k9qx/WTmKmqbYJzwBP3H1sJ+XIWKv0k+mRAT8ZC95cZ53VWulhMeg34BYY9g/Xtssakz+Fs3eDnb+5NYlP+XoKlVuzdApegV7EjImx6jH5x8k/AJgQO4HqKh1GoxiynCr/9KfC1ASbF8PeR1x7nvpiVUmDkiQRHxTfTurYHqdOsuVKv6NHO8fXL3NtJr/e+avVioRtOduIeTGG8W9rtwC4fz98+KH4/403QlJPTCbfej0sd2ECkuQm5jTp7zu0e2VDpeoFnIaKBkq2peNTkUd2tmM5lEpYsmkJN+xwbRJh6mep5O60bo2yP19kkA+IiGvxnetFBZJOVEOfux/0nmCohbzfoHAdmCwH/ZTcT30C+jAuZhxmRCVcT2PSPZN4oOIBwlJsJ4xsyxHVVMMjh+PtLqwO/hCPes4/v1XS0inMZijepnpdo7shSXDXXWIMl58Pr77qur7pVByt9IsZF8O5b5zLkEvUydJZ8vRzhILUAg5/f5iGSnWDiAOFB6gz1BHgGUBymPOKGH8GhgwBk9TIK+WzGP/2eGoaW6PSkiQx9NKhBMUHdV0D21Cwt4C9H+21el+UlUGNZ/OzMaSnGS13QzpZFUce88nrL0qDfv7R/tx64FaGXKxeRlMpkk4iamQUkcMiLb4vS3u6uYkYhdznDo0Y2q3W0S3R03IWT08iz4C4i0RWu4OorfQzm828MewNvlj4haLt3bzsLxrKVX5xcdpkfkVHw513iv+vy/8RgPjCW5gwoXt/qWwy4xeY9bvj+w97FIY/ZnEh1BZBQSJD2WwGvVnINJ3qOVaRXcHaf63l5B8uTJ1ViM5Nx52ZdzLnOWU63bKmexQjCfX5s5sIKUCS4Mw1MHuN7e28o8VgoNb5e0KelGRXZlNeX6660g/AbDSTuSaT8hPlTrfHFj6hPtxfej/nvHaO3W3lwUpQEKQWiSq/lLCUlglyL92HoRFDCfUOpaappkWKUK6CdsTTD8QA8bx3zuOCT7XTOOo7uS83772ZwRcNVrS9Jf+0ekO9ogz0bk1tDmR8BLvuhQNPtbys18M998DYsTB3bhe2zxIKB/13T7ybDxd+yKS+k9q9Lgf98vaX8PFZHyuu9nQ1NYU1NNZYl4LcVyASQ8ZEj2F3sypKRISYGP2pmfoNDPqb646fuxy+C4cs5ywCbLH535t5MfZF6itax4xRUaK/MRjgsOuUdloYcskQrl57NdGjLSeX7srbBQjvVi2orITnnweTCWbO7Ib9jFKSrofBD7ju+JIEczbAbHVJL0aTkT4v9iHwmcB20oFKSJqbxD35f6ey7xDq65UvGDmCkkWcw0sP88PVP1B5slLVsQ0NBr674js2P7fZ4vsNhgYK6jMBOHtcTzOT7GJO/iQqUBtKIGhoq3KJmw+cewDOWAburVUHMTHiWVVXp/x+WpiyEIAfjvxgd9u60jr+eOUPjq85rvJCXIeHr4fVClMZOeg3PqY1meLQIfHvyJEaNeT4x7DvEaDn+x36+4uKcDc3URG5bFnnnLdt0E9NEoW7tztjbx5LSJI6HXatKv2GXzGc+8vvp9/Mfqr225qzFYBxMeNUJS/9menfHzzc3CnwXsv23O1UN1Z32Ka+or7dOK+rmHjXRB6sepCYcZYrSEpKoNpLBP1kj+peVJL1NVQegeI/RGJYteufTfsK9jHp3UlMfm8yoL7SrzNoqGqgsdr6XFcO+gUEiOFvS9AvvPsn+iuajt9zzz2KD/jiiy863Jg/I7q2WjFNVeDumG6lXOmn1NNPkiRCB4biF+Nnc7udb+8koE8AA862n0nhrJ+fJSZPhunzc1iWvwfMEkO85vVsI3PP0I7+aY5garJdoXUKOp0YnJWUgGT0QifpWipdZOrL6ln32DoMdQb6TOzaP7Kkk1RlHG0tFAsOQ7xnu6hFpynutr//hIyGBSfAjgSYEoK8gugT0IeTlSc5UHiAoCAhI6G00k+n0xFYEcjHcz5m2iPTmPX4LKfbZPecbvYnE239/Pbk7wF6pT27KzpJx4x+M/j20LesPr6ayX0nt0xald6Hlkg+r2szTS1NwFcfX825n53L/IHz+enynpeJDkDZbvjjavF/zzAYdH9LwsuAAUIGvAPbboaqYzBzuapnpGaYjHD8Q9D7QL/LrG52wSDLQWJZ3rO40p2GjVn0m9HPqebodDqGDRvWfqzpAN9c9g15u/J4oNxyAOFQsVgJHBw+mJ+aq6PmKMvZOX3RuUMf9XLsqggeDfGXg1+iqt3WZa7jjR1vMChsEP+cYemL1Iqblxv+0f5U51W3SBhKkpD4XLNGSHyOGOHwFSjCN8IX3whfq+8fKhL3n1YT7w0bRDJFbCzceqvjNtxdTtK1rj9HgPrnn16nb7EaSCtNU+XD7ebphp+nGzExwtMvO7s1Y1wLdDod/WP7c/Crg/SZ0IfgRNsr2wX7RFXCxHsmdpDBtYUkSVz242V4h1hOENuZmQ6SGTdDAPNnWipp78UqmR9D9veQfHvH99rOZ45/Ak2VuA28ldhYUel34oSy+2lhykIeXv0wqzJWUdVQhb+n9fWbpromfrvrN8bdNo6EmU74n2hAeWY5x1cfp//Z/fGPtr3mJPumTegzARBjzYIC0R8O1GqtvWQrVBwAQ43wmD/4vJBo943T6ASdy8CBcO218NZb8OmnMH++658fA0IH4K5zp6qxiuzKbOIC1f3tDPUGzGazXcsfGXm+1FbJwdGxpqe/p/2NTqElGN0r7akYd3dIHijhZvLDoK+iurGaSFqrmbI2ZfH+tPc564WzmHT3JBtH6hw8/KyvO5WWQnTDRUwdGs/o6NGd2KrThKZK+GMJ+A+E5LtEwK8uD/xc+2wK8Q7hj5N/oJf01DbVEhbmA7SuoynBbDbz652/Ultcq0iFUC073tzBqvtWcd2W6yyuhctBv8BA8W9cYBzT4qY51BfpdDqGDHFd1WKH8ynZaPfu3Yp+9uzZ4+LmnsZsuAh+THBYC2Bg6EDeOPcNPl70seIsn0u+vcRmJYvJaGLF31aw8emNio6nhZ+fJQJGLwcgqGY8M8aF99zJd2OFyKowWs8gUMShF+ALT9VyOnJFy6ODvsD4DyO3jLul3fsRQyO4dvO1zPy/mc61TwMaKhuoyKrA2KhM7mN/9VoAxoa6PhDU4zHUwh/XQtEmZdvLE2QNvP2GRQhNsNTCVIcq/SLHRzLz8ZkMWzzM6bbYIm9XHkd+PGKzskWm1c8PdueLEpeRUSNd2LpenGFmP9G/rckUVa6O3IeWMDYaKdhn3/9HCRue3sCBr5V7bMgD5pA2ybor0lcAEO3Xg2XXQ8bB9KUwaxWcn6Gswt1QI3z1uiLgB0JObM8DcOQlh3aXK/0Kavx5qPohpj3kvJFTY6OTYw5g4PyBjLlxjNX3Z/WbxdykuQTUjeDoUbHAMG+e06ft2Rg7QffSOxKmfAaR6sZtBTUFfHngS1ZkrLC77fjbx3Pjzhs7SMHJEp+pyqyXnaK+vB6T0boE0cFiYVQzKHyQJudLb86fnDZN+Iz3YgNTk5CAV5kpPiBUJJOmlaSpPmXlyUpiCnbjXldJtjo7KEXk7snlu8XfKfLqnXLfFB6qeciqHJQ19B56ks9LJm6K5QX6ZZtFJUOYNJDw8J468e0iJn8Kc/8ADxsBW2MDHHgCDj0HhlrVvn6DwgYxIGQAjcZGfj32q81t/aP9uXrN1Ux7uOuNGdOWp/HjdT+Su8N2ha3BZGBH7g6gNbgiV3XHxYGv9RwMdYz7r5BE9wiE8v2w5z7Yep1GB+8a5s0Tgb6aGlE17mo89B68dd5b/PaX3wjzUefxm/F7Bs8EPsP+z5VJg5pM1uU91Yw1S46WsOeDPdQW1yreR6asvgwJqTfop5LBg0FvFMnep1b6RQyNYMjFQwhLVnf/uIKTW09SkGp9Pl1WBv2KbuE/c95jWKRr14NOS9z8YcavMPoFSLxKzK/DJ7v8tLH+sUT5RWE0G9mTv4ew5ltNTaWfJEmUHy+n+FAxxibtJaHDB4Uz/MrhBCUEWXxf7s/loN/NY29m/ZL1LBm1xKHzaTE/V4qiSr81a+xIwPXiMCZT8yQ2dBy4+YoFK3vVNxbw8/Dj5rE3a9o2SSdx3ebraKpVtth/vHm+p2WlH8DQyEEsHnw1/tWjufwSbY/dqeSvhI0Xw6RPIOEKx4/TZwGU7YHQ5sGOsV4EcYY8CEHWH36hoZCWBmWllmP9kk6i7yQtBPqd58hPR/j+L99zyXeXMGiR/UWcqxu3sTZzHVMnTe+E1vVwijZCxvsQNALCFZo2Z3wEu+6Gs7ZAgOPpncMihrH82HL2F+5nbPPittIKK5PJRNqxNKY8OAW9i53jd727ix2v7+CenHvw8LVd5SgHXEJDeyv9egKzEkRiwObszdQb6glq9ihzptIP4MtFX3J8zXHuK7lPcbasJQz1Btb+cy1JZyUp1q2XA5Zt5T1XZqwEYE5iDy638o6EPuer22fyx51npGIJSYLp34OP7Wr5ekM9G7M2kl2R3W6yIFf6lZRKGE2goNjYJiaTiSNHjjBs2DCn+s1J99jO/JWrxZ5/Xvw+fXrrpOhPy+YroHiLqJbvZr6asiTS0ZKjDh9DDvqlpQk/b1cGx96b8h5ms5nbDt5m8f2DRSLoNzhcmSSyPTKa7dG1TmLsVI69A8fehIkfQpALs4mr0mD5CBh8P4x8RvFuA0MG8iu/klaqPuiXtSkL43c/4j9yESdPDle9vy1MJhNl+jIu+OICYkZbN6hfkb6C/1v3f7x93tsOBZtNRhM6vfUOftPho+AJyWEaypcZG4S3HYjkGHQi2HK6oXOHEOtJKoD4O8z4VWzr5qM66CdJEhcPvphtudsI8gqyva1OcrpqXysGXzgY3whf4qbargY7WHSQ2qZa/D38SQ4V1bxy0G+QFrkVZnNrCZxX84QwaKiQCg4Zp8EJug53d5GEV1IiKiM7Yyx0zchrHNovfFA4CbMTbFbSt6WqSgT+JElI3MmoHWse+PoAax5Zw9Vrrlb93fj2km+paqjCXd9FyX09lMGDwe2QHw10DPp5BXpx0ZcXdU3DTmHpNUtBwuJ4r75eBNOhfaJrLyqQJIiY2vq7V3P0rSZbzFW8XZMsLEkS42LG8dPRn9ies50ZYSLQqFbe88IvLrS7NucoA+cPZOB862OutvKezmIymUhLUz/+dZTuNQv9MzP4/i457Zp/rqGhsoF5L3VMyZYkiYihyiRF6uuFcTFoH/SbEjeFKXEKgxPdGf8BwlsjdKyTx+kvFjZlCtZC1pcQPFIE/eoKIH+VCCS3CdDIC8K2yqhNRhMF+wrwCvIiOMFJsXYnCB0YysS7JxI+WJlmT01xCNHli+irLtH2z0n0WXB+OrgHKd/HOxq8Y6C+0Kmgn+w7kFqYSlDzYRypsDIZTFRkV7jsHh1z4xhixsTgF2U/AUMerISFwf/G/I/d+bt75Sa6MSlhKYT7hFNUW8Se/D1EBU8EnK/0G37VcPpM7oOx0ehU0M/Ny42/pv/VpqZ8W8zmVk8/Oes2tyqXg0UHkZBagpw9kraLQjXZImGh3+X29+tqOQAFyRS1TbXM+VgEZC8Zcgm+HmLBJTAQPDygsREydpZRtu0ogy8abFeGqztQWgqbmgvIz1cZqz0tCRwsKj9dHfAr2ws7boMBtyn7fgD9Q/oDUFxbTGldKSHetldPjv58lLRf0jjntXNafKAiI8Vzr7hYLAZr5vFkgQHnDkDvYXkhsaimiOJa8SCWF6edwWBoXfhPVKea2r0w1AjJJo8g157HfyAMeUSMLVUgV/o5EnhOmJnAkP+7hD0b4lxS6ecZ4sngMwbbXLye+4kwerzts9v4aOJHhKWEtcjfKuH3h35n9zu7uXHXjR3sDIqL4XjlUQiHiQM0CPo1lsHqOaDzhLOaO+nDL8Ghf8PcrTYTRnsc2d+Bf7KyQLdfv5b/qg36ATwx6wlFvo8AZpOZqrwq/GP8Fe/jCnwjfBl8of3kCFlCcWzMWPTNKgty0C8lRYOGHHsLsr+FCe+0l/JUWbWuCVXp4J+k6SEjIsR6i/fRh8BvGsScrenxtcI/xp8rflGeiC7PlQICnPNsHnPDGIITgukzyTE7GVtyuj2GtkkYnUBKCuhNYl0jv6wauqmC7rRHpmE2WU7eLCuDevdcJO8KdO6JQOf9/U4LijaJccCpa9Glu+HX0TDkYRjxhMtO3xL0y93ORUI1WnXQz1UBPyW0lfesaRTRZ3n+3t1xKId4x44d3HfffVx22WVccMEF7X566TqOFB/ho70fsSlLoWwfkLUhi8PfH7Yom1N8pJimOmVVftnZYn0uMLA3u9sqwSNg5NMO+V/YJGYezNsFKc3em5WHYMtfIPeX1m3yVxMSLB6gK/I+Y9GXi3h317sdDpW/J5+3Rr/F7vd2a9tGlcSOi2Xui3MVyQyYze0DL70owC8RPFWkSEWdCefsa58Z5ADDIocR6x9LjH9MOy81tYU5/xv9Pz49+1NVhuVqiBoRxahrR9k1uYf2nn6zE2dz7+R7CfbuuoB5L7aRJIlJfScxPHI4NY01LfdhZSUYnVCKGHrpUKY/PF3Vop81AvsGEj5IWcJDba0IEEFr1qMs7TkmZgyhPhp4yHYVe+6DrwOh9iSk/gM2L4aaLOvbZ34Oh18GQ12nNdEq9UUiAccKId4hBHuJmy+jLKPldUlqlfg8uiqLX//6KyfWq1iFdAHVBdV8ft7npH5uWcexsKaQ0rpSli8X36EhQ3p4sEQrhv8fTP3K9efxDIXyVJGUoxA/Dz9i/YWPmpKgy/HVx9nx+g5Kj5W2vCZJnSfxOee5Ocx6wnICg+wn2S+onyaT75MnReDPx6e18rZHknInLMoBH+V+eQ6hc4MRj0OEOunCASHN8p4OVPr5RvgyavEgDJ6+Lgn6qRlb+m/1592J75K73bZc4qkExQcRMTQCv8iOyWVpaRBWNYshDddyZrIGSa/ugdBnUXvPnqARED0PApoDQBUHYe/DUNO1zxunMDbAlmtgqwovy8YK2PswyW5iTpydLSqZlKAmePfzbT/zUp+XqC1SL2eoFeWZ5ZRnliva9pwB5/DpBZ9y98S7AdEnykUJmgT9qtOhfB94WJiL1uVBxgcanEQBx96BZSmQ9S2c+BJKdjh/zLT/0S+6iBDvXGIrn4eMD50/ph0q6iv4PPVzXtv6mkP7m81m6ivq7W5nyUPcEXwjfBm2eBhunuoihwaTwbkTdyXFW2HrDa2/H3tL2Ds1VXXK6f38wK9ZTe5wRnWH97O3ZPPhzA/JWJXR4b3OZPgVwxlxpWWj6NJSyA79gJUDB3PjshssbtOLDXbeBaumi+deW4KGQ/+bIHKGS08/LlZUce/M29lSjFJXJ9Yy1JC9JZtltyxTZMOjhm8u/YbNL2y2+n7boN+XB77E/2l/rvz+Sk3b4CpUB/2++OILJk+ezKFDh/j+++9pamriwIEDrF69msDeaI9z7H8Ctlzt8O4f7/uYq3+4mk/2faJ4n4u+uIi/pv/VosTIJ2d9wruTOgaGLCFnxmld5bcqYxW78na5bHH/tCF4RGs2efBIsdDUtzkIn7cSVs9mnJeQ38qsOcQPh39okSJsS+TwSKY9Mo2B52ooJ+NCzGYz535yPntDHqNJV9kb9LNHxWE4uVSY+KpBkjSpnhkZNZKT95zk8ws/b0kOMJmEXIgS5KzrYVcMY+jlQzEZFM7MVWA2m61mmFmiradfLz2D7y/9nr0372V24mz8/cWtbTa3DuacwWQwOfW8yt2ZS22J8tGvXOXn6ysqxIAWf5mz+3fPzGLF+CUKGWv3QOh/i3iuedr4oh17E/b9A3RdlwUIQOVR+C4CDr9gczO52upY6bF2r8uBBmlAElf8egX95/V3qjnOyiHXFNRw7LdjlGVYLod9ZuMzhD4XytPbHgHgvPOcOl0vavHpAxeWiiCPCpLDRAKakqDfpHsmcU/OPYQObP/9G96srLhvn6pTa0qodyh3jL+DxUMXa3I82aogMbHri4Z7HCqefXKlX3ppOiaz+rFcbCy4NdZSWWHW3Ddr7+N7eSn2JZv2FnIwJHlaMmc+dyYh/dVpjY27dRzXrLsGN6+Oi96lpRBbupgr/N/lzMQz1TXeEpIOhj4Mk9usD8RdCNO/a/XKzfoaDjxlO7HGWZoqoWR7s7SoC5DchLz28MeV76P3hqOvE1jxNR4e0NQEeXnqTiurK9gicXYi424f16XrGWseXcOr/V9VFPiL8Y9h8bDFnJcsHugZGeJv4+8PMdZVb5Uz6jk4L82ypU3qY/DHEihX7m3tMBHTIGI6eEXCpsvg8IvOHa9kB2y/mXnhd1NaF8Mnlcdh5FOt77vo8y+qLWLxd4u5b9V9GE3qMhhNRhNfLvySz879rEMRQG1xLaXprck+shVCUFDH4ygda1blVTnk5Qcw/f3pTH9/OqkFnWAmrDWH/g3p7wgfXAA3P8hbAXXqEkacIcw3GHdDMNknOz5zdXoduTtyqcjWYDLsIkpLocZLjFm1UHb40zHhHRj3ZkdZb50exr8pkvxdyIhIEcw9WnIUs74Ov+buX221X87WHHa+uZOsjdqNVwwNBo78dIT8XflWt2nr6Xeg8ABmzIR6O74A6Gq7oraoDvo99dRTvPTSS/z00094eHjwyiuvcPjwYS655BLi4rppnXA3pt2HXboL8n4Fo2NR66RgIU2QXpaueB/fCF+LAT+T0cSYm8Yw+nplMnVy0E+Wx9CKO5bfwZi3xvD94e+1PXBXsPoskWHhajyCIO7iVrmM0LGQfCe1MdcD0FgrKlHqDR0zuvTuemY9Pos+Ex2TW9CKlfet5OtLvra73YGiAyzP+In0yOcI9PXCs7fK3zaZn8L6hcKDRS3GevjjOtjzgCZNcXMTk0dQJq2o1+tbvAKm3j+VGf+cgd5d+4dlfXk9T/o8yaoHV9ndtm2V6ZbKr/li/xcUVFuv7umle6CTWp95Ol3r5NVZX7+9H+3l+fDnKUxVXnHTFrPZzOfnfc5Hsz5SvI8c9JOr/AwmQ0ul37z+HWW7exQDboFZK8HdH8LGi+eam41KnulLhR+MrvMG0RbxS4Kk6yHMtjF6UojlMZtc6Vfe5Ef/uf2dqh5t2286SuTwSB5peIQp91muOJErrXSV8YSFwcSJDp/q9CH3N9h2E1Qds7+tFjhwzw8MEcldR4qP2N02IDYA/5iOclqn+vq5goJ9BXx7+bdWK16HRAzh1bNf5cnZT2pyvvTmr2OP9vNrLBeJpFpUrSghdzks7Sf+VUh8YDxzEudwzchrqG1SvwD8xzNrGbnieTxrSjWt9tPr9SSMSiB8UDhu3tarULzdvAFoTGhkyt+nENQvSLM2nPpcd5iTS2HbzaICzh5DH4Uz17fKU9fmqKoeVkTJdlgxWVQUyhhqtDu+Tg9Rs9VJzeo9YO42pJm/0Kd56purYg3+032fEvtiLLf8fIvN7QZfNJhzXjvHYmVnZzHi6hFMvneyQ/fqIfGYJyVFw2QIdysSjQNuE0lefi6UDJADLwHJMPt3oWQz+VMY87Jzxw0ZA1O/IidUBJ4z8vuI6zCb4eDzsPEiUBmUU0JCUAJebl7UG+rbqUcoQafXEZQQRHBCMIb61kq60mOlvJLwChue2NDyWtbqY/iWnSQosH3wUs1Yc/O/N/N8+POUHLXhN2OBvKo8tpzcwoasDYT59MAs74nvizmNrtkCIvEaOHuX9gpgNnh54lLm7iklOPfiDu/FjI3h/vL7GbVkVKe151SyNmXx2oDXOPit5SSKtkE/2Zu6FxUEj4DEq2xv42AcQglRflGMiR7D/IHzqWioaCnWUBv0G/6X4dx+5Hb6z3UuKbYtbp5uPFTzEOe9Yz1zta2n3/6i/QAMCXfMM1uv1zNkiAv9tk9BddAvPT2dc889FwAPDw9qamqQJIm7776bt956S/MGnu60y/ia+D4syhcDUAewtoBkj/w9+Wz+9+Z2bdHpdUx7aBrjbx+v6BiZmeJfLYN+GWUZHC4+jF7S92xfIgCzCWqzob4LAgIewTDmZQKiRBCwoaY56Ge0vUJjSfK1syjcX0jOthy72/2e8TsAodXTiArv4uqOnkDi1SLDJ2i4+n11nlCyVeh+a0CTsUlVsMVsNlNZWdmun2qoaqChUsFihpp21TSRODtR0aS4trZ1ofOdg89z+beXsyFrg+2deuk2NBobMZgMLfehs75+AX0CCOkfokgixxJmo5lpD09jwl0TFO9z6uKgyWziP+f8h+tHXc/4WGXP7x5FTbZ1+U6PIOc9c7VAp4cJb0PfhTY36x9sudJPDvoVFIh+rzyzHGOTY4tElvpNR5AkyWqShVzh4F8/iKlToRMTF7svxZuEdJPJeqWQptQXwsFnoWCd4l0Ghg7EXedOVaOyUvu6sjoOfXeoXSV8RASEhwtZV3lRWGtKj5Wy/4v9VOd3lKNyBW0r/Xos5fth36NQsKZzzucRAu4BYFYuvabX6Vlx5QrenP8mfh7qAyGxE2IxNxtJahn0M5vNjLpzFFetvsqmfKOXm5hL1TWpl5NurG7kxxt+5MiPlgPu2cVlVHkdwC/IyUj68U+Eb1pDkf1tJZ2oepJ0YtFv3XxYMQmaNPjeyZWcUbNFpUGfReJ3Qx38Nl6bhFiTUVSGOfKsCxgAkq6lgk1N0G9YpMh82J2326GK1c4k8cxEznzGfiXHsdJjvLTlJTacaJ3PaObnV5cHv8+ynSAQPLw5ycvbyZNZoXA9/JQMZXvav95vMXhFOHdsSYK4i/GPFlkjhW3j5mV7oPIINJZa3NUZ9Do9g8OFVO/+wv2q9z/rhbNY9PGidn5ZwUnBjLp+FMkLW4NSOZ+sYcC2z1qCftmbs1n3+DoqTlYoHmvGT49n1PWjCBmgLqvh57SfAeELFu0frWrfboG7X/tKKkkC/+agRe5y2HiJy8eM0c1/tkIL+RySTrJYCNKZGBuM6D31FivgQcx5qz17g34OUbbXdsKB2SSScta7zpRdkiR23LiDpZctJcovqkUlq0Rd/B+fMJ8OyiNatc/d293q+23lPeV+dmjEUIfOJc/POwvV3+zg4GCqmrXYYmNj2b9fXHB5eTm1agVZe8HUVjjeI9Cp9Cm50u9E+QmajMofGjv+t4OVf19JaZrjg5Cs5upaLYN+y9PEgHBK3BSCvIK0O3BXIOlg/iGY/FnXnN/YQHjTryQE7cFYLyaqDQbLwZLcnbm82OdFdr61szNb2I4rfrmCO4/bl6panbkagNCq2b3Snkrw7w8DbmrNMlODJMGcTTDrN6ea8PHej4l+IZobfrqhxRNASbDFZDKRkZHR0mcWHSrilX6vsPHZjU6151QC+gSw+OfFjL3JfvBAHqT4+pk5UiJmxIPCBmnanl5cw6IvFxHwdACbsze385d0hoRZCdyw/Qbipzn2INS56Rh/23hVWZanBv089B4sHraYt89/Gzed9SqFHsHBZ+FQG4nMzC9gaRzkLuu4beVRqM5wmXSSK7BX6VdYCJue28QrCa+Qv9u63IgtTu03HaHwQCEZqzIsSt1VN1aTVSEGgH51g3q2B5qWDPsXLMoD/05alDDUiir8E18o3uWWcbdQ+3Atr579qqLtNz69ka8u/IqCfa3Ja5IEA5svMUs7lZ12DLpgEA/XPUzyAsvZ8Dtzd1LZoM3E2WwWMnbQw4N+IWNg7jaIv6xzzhc2Qfg+93HdYtGpDDh7AH1vX0CDX6imQT8lfebe/L38Y+0/mn+Bt8a+Rdpy5QoaZcfL2P3ObnK2W05u3Fm+gnVDh/JM7hxVbe/AlC/EuN1HpXqL3gMG3ApJ11mWX7RHW9+tk0vFQqIcPEy8GqKbr6upEjwjhLSis5T8Ab8MhUPPO7Z/2V5mxQhpRzVBv8Hhg/Fy86KqsYr0UttJ17/97Te+vti+io3WNFY3UnhAedXm+hPruWfFPTy+vlUmVQ76DXJ2ilO2F4q3iGpke9Rkix+tqS8Cg5Vkl6ZqyPpG/ViyZDtsvqqlvfJYSCRvIR6WE98T30cvZZ7dapEXnx0J+snBHrPZzLJblmFoMCBJEvNemkfKgtZIr/7ceZwYeg4hYWL7zHWZrP3HWqryqhSPNVMWpHD+2+er8sQE+OnoTwCcN7AHashn/wC1NjqWst2Q/b1QfXMh8vyiuNiyj33xkWK2vLTFYflVZ0mYlcCt+2+1ajOUXVJCk7tYfJEtEnpRQG0OLB8JO261vo2kA9948I7ptLm0o5V+AIZ6A2nL09p5jTtDSVoJGasyaKiyXkwgx+jMXmXkVonv85AIx6r1TCYTmXLVVCegOOgnB/emT5/OypUrAbj44ou58847ueGGG7j88suZPXu2a1r5Z6JwPZz4yqFdo/2j8XLzwmg2tizCKGHCHRNYsmEJwYmtrrwfzvqQH675QdH+VVWtC49aKrz+cuwXAM4dcK52B+1qusogxFiL1x9nc37Kf9GZhQamJXlPEOby3iHeVrNsOgt7g0GDycDazLUAhFXO6g362cNQKyQ6neFUDXAH8PPwI786n9TCVKdkFUMHhhI7IZbwQa6ZPClBHqR4R+RS1ViFXtL3DkJ7CBISDcYGtudsVxV8diWOVGNpJgPWHTn2Nhz/uPX3iOkQdyn4WBhopP4LfkxynVeQWgo3iIXOvJVWN7Hn6VdQIDKix946Fg//rqtk3/X2Lj6e87FFr0lZGtLbFI6HMbRlQeFPjySBd1TnSc36xgtpvtHK/Yi83LxUJQYMWzyMc988l4A+Ae1ej40V/548qfhQqnHzcsPNs2NbKxsqGfv2WAKfCaSi3nkfmuJiqK4W8uM92rHCzRtCx4Fv365uiV2qG6vJq1JpotZM3+bL0zToZzCR+lwqR5Zal72VEzX6BfXjr2P+Sm1xLYY65VWOEUMj+HvR35l4p2Ut5NxaUW4aH+Cgxmxlc9t1eghwMPGg/w0w5CHxf2MjHHlVWRVKzs+wanrr7zUnoC5HVHidinekkFYcdJ9jbWyLR6gIVKqR9mzLkVcYp/8b0X5pqoJ+bjo3hkcK9ZSdebaTZcvSyyhILXCJH7ktdr27izeGvsHRn+37t4LwKgJaKseKi8WPTgcDBjjZmJh5cGEx9F1ke7uKwyLJy9Egri3iLoTzj0PwyI7v7X0QNl4MpSoTnws3QNaXUCmio2FhYhjQ2Ni6UIzes3UeXfyHuEYNGRreHPQrUh/0k9n3yT52vrmTLS9ssfh+ZWBfymKHtsybxtwwhpv23ERYiliEsRcsclR1oq6pjpXpYjx9fnLnJZdoQkOJuKe2Xm99m/43w6JckUDjQpZlf8zW5DM5Fvayxeqq46uPs+KeFZzc6sIBnROkl4s+LNyzD74eNuweOoP6ovYJLt0ZSQeDH2ytsrfGlM9FcoKL16vNZjNldWUta7dFCsQITiV/Tz6fnfMZu9/XRoEs9bNUPp7zMeXHyy2+bzBATbMaeb5B9N19AvoQ4BlgcfvuhuLZ3vDhwxk3bhwLFy7k4ouFDvDDDz+Mu7s7mzdv5sILL+SRRx5xWUP/NOz6mxggx12s+gunk3QkBidysOgg6WXpLVnk9ggf3H7R3Gw2E5ocSl2xMskS2c8vIgK8NVJiMJgMrMsUMkVzk+Zqc9CupHgrVB6CPguE3GZn4x4I499mx66h6ExigdFa0M8nzIdb9tn2JnAlDVUNHP3pKLHjYwnpb30l+0DhASobKvEigMDaUYR3XeynZ5D5Key4DWb+BpEzHT9O8VY4+l8Y+5pDQUA5E/Fg0UECI4yA3qFgi06v44pfrlC/ox32fbKPooNFTHt4WjuZE0vIA2ZjsHj4JwYn4unWayzZExgbM5bvD3/PjrwdzA0Srzlb6QdQmVPJqvtXkXhmIiOvGalq3w9nfIhXkBeXLVVendE26FdSW8I7u95hXv95jIgaoerc3ZKzNrdPVPCJgalWKpn6XS4WNz27SfRT5y4WXm1Ieg+NGMp757/HgND2q2hy4KykBKLH9aXvpK5duB+2eBihyaEWvYhkPz//epH+3/scRiTYFKwV1VbenVT6KElCms+FRI2MImpkVIfX5aCfmoVyNeTuyMVsMhM7PrbDe4eKxP0X7RdNoJfzSUlylV/fviLw12OpOQFeUWKRubMoWAvZ38GIJ617dZ3CB3s+YMnSJcwfOJ+fLv9J9Snr1v7BwD/SyA79C6DNIlVNYQ0Zn2cQEhLC4AsGW9zmZKVYEB0bM5YhZw9hSKa6TG9JkvAJ87H6frEhE4Ck0H6qjguI5OHfZ8LI52DQ39Tvb4mj/4HdfxPPtQF25odZX4lnn6lJbN9nIcQvBi8rmZmSTnx0BWsg7X8w/n+OJRgGpsC4/6rfTyblb2Torie/OgmTyhj06KjRbMvZxq68XVw21Pr47eKvLkbv0fn613FT4xi5ZCQJs5QFkQ8WC8luOegnV/n16wdejlsMt2LLm1kmIBkG3gExGiZ+n/wJQkaJyldrFaz9b4LAweCvILpZXwRuPuJ6Uu6C2PktQXZ39+ZxeYlI4Apse0vX5sKqGRA8SoxzNVpgl+fXctDWEYZdPgy9u97qvSLPk+SkXZ8wH3zCfDAajRz53xGWf7Kc2w7fRkCs5YXwZTcto+hAEVeuutKmjN6prD6+mjpDHX0D+rYE2XsMbr4w5TPwtDFA7qS5S3ZFFkX+v+NVn0BhIR0S9ZLPTyYsOYyYcTGd0p5T2fvxXoyNRkZfN9ri+1k1RyEIEhxNZtGKgjWw7nzhAZp0nagQXjEBEq+DQfd0bdss4R0NI5/q6lYAYv1v2vvT8NR78tEIMXFQK+8JEDs+lrkvzWXgfG3uhZQFKfiG+7YrgmqLnLyh00FhYyYg1v16Coor/datW8eQIUN4+umnGTRoEFdffTWbNm3igQce4Mcff+SFF14gOLgLghmnGyOfgenfA45lwsgSn2pNfM0mM6sfWU1DZQOSJDHnuTnM+bcyWRE56Nevn6pT2mRv/l5qmmoI9Axs0cvv0Zz4Ev5Y0nVVCJIO+l9PpcdEdOZmTz8rQb+upuRICd9d8R0HvrI9aN2TvweACNMoJPS9lX728I6GqLMgwEltlrLdkPkxFDkmq5kYnNhiNt7oK7KZlQb9vKzMNmuKamis1sZ4+NB3h9j03CaLVQWnIg9S6nzFwuOg8F5pz57CuJhxAJpX+nkGeHL4h8OKPEnbYjab8Qz0xDNA3SJt26DfivQVPPD7A1z1gx2T7p6CVwT4Wim3MZ4ivxE7H4b90/VtUkroBJHNnvAXq5sEeQWxZNQSpsZNbf96EHh4CHUVRyZCp2Kt31RK7PhYxt0yzuJipRx08aruDfq1UL4P1p0LGe917nnNJqg4CNXHFe9y/8r7Gff2ONafWK94n8bqRoyNrbpQrq70W/n3lXwy7xOL78lBZ3lx2llOC2lPgJXTRKVxZ1L8Bxx9DcpTFe/SN0AkNKSVKJfGbItUVoZPRR7lOTUt/srO4hvpy3mrz2PKA1OsbpNTKZ7vsf4dA9FKKD5STMnRknb+mDJNTVAhie9wSpQDlX5+iRB9DsRqKIE38HYY8xok3WB/24kfCGlZ2UbAN856wK8tlUdEwLD4D/XtMzU5L0cWNISg/pMxo6OwUGT1K2VMzBjAfqVfVwT8AGLGxLDgvQWKgyyyT++QcBHM1kzas3CDqBhtUCDHJkkw9lWI0Sjxu6kKNl0Kmy63vV3QUBHYthd4NpvF8XY2W5FIug5VtW2l2tvhEwNjXoEJb2laUSMH/Y6UHHF4jUfnpmPoZUPxjbAcmJXnSZaWfMNHhhMzPsZm1XNwYjD1FfWqAn7QXtpTrSyoTYyN7X3OVk6F3yZo662n9xLFHJEzbG9nqIX09yH/d+3OfQqyf65RV23R1y8gNoCEWQl4+ndNAvOWF7aw4ckNVt/3KBlFcs7jXDboyk5slQVCxkDwCPBqNkmsSmuWq+4miadtMZvVPR/T34XVc237/zlB34C+lNaVkledh95fSGY5Iu8p6SQm3jXRZoGIGqJGRjHu1nF4+FlO+Jf9/Pz9IS6wL0tGLuGc/uc4dU5n5+dqUBz0mzZtGu+99x55eXm89tprZGZmcsYZZzBw4ECeffZZ8vMd8xv5s6PXnzIAjJoN4VPE4MEB7ptyH8uvWM6iFDvlu6ew862dbHhyA1teEuX8nv6eBMUHKdpXlqPV0s9vc/ZmACb1nYTOwb9FtyL5Dpj+g2VZsk4kJASiyhfwxdA61l6z1up2JUdLWHbLMjLXZXZa22SCE4O56KuLSFlo2y28sqGSIK8gAmqF/1Vv0M8OsfNhxjIhOeYM8ZcJWZRYx7Iv9To9KWHisy1zExNLJRVWer2elJSUDn1mzrYcXun3Ctv+s82h9pzKBZ9cwO2Hb0fnZr/fkQcplR5iRpwS6qzDfS+dhbxIk16Wjt5XzGK1qPTz9Pfkrsy7OPd1dd8PSZK4/MfLWfSxumd326Dfr+m/AjAvaZ6qY3RLTAaxCHiq74vJICblGy4Uv5tNULSp05tnF0lyeDFHklqDZwUFsOeDPbwx/A1qCmtUH8tav6kVU+OmcvWgWwmvPAsvL/DtYrWdboFvPIx7UzxzO5PKw/DzEFGVo5BDxYfYkbtDsQfQ/i/382zIs2Ssak0slIN+paVoFnhpy8R7JjLnOctJiPLitNZBvwQHVRW7BWYzJFwFCZ28KJZ0rRgbhlmWrLSEXOWcUZaBwQGZrLNfPJPjF/8dg5efZhKf7h7ujJ45mqA+QVa3OVklItyvbH2FZ5c9y+73d1OQar2q+1R+f+B3/jv4vxaDfmVlUOuZCUByRD81TRf49IEZPzku62kJvQck3w6yHLClBXE5EUeSwF+Z0lA74i8X8naOBHkOPAU/DRS+vk4QHGRicNQO/NxLKFD+cTI6WlSl7MrbZVe+8MiPR9j1rra+XcYmI99e/m0Hj/OawhqO/XbMyl6WqWqoarGIkRMZD4pulhRnpzgZ74kgmTU/PWvUabDGqPeCyZ/B0EeVbd9YBuk2EneMdaKa2sZSalup9g4MuAmCtE1q7xPQh+8v/Z60O9LwdEGVt8EgLH2gY9BPr9cz88aZXLXqKpuL8FMfmMrVq69Wfe7ZCbM5P/l8Lhh0gep9bZL5KXwfCYXN352g4RCQ0pq0YDaLeYajNFYo96U01MK2G+HwS46fzw5y0M+gtxz0g2bpxeNdU6Rw8dcXc9GXF1l8r74ePMqGMyDvEW6eeE3nNsxshkMvQtqb4nf3AJizEWKbgz4ho+D8TEhovrfri2DbTULatasp2wM/xIoCFCVUH4fS7VCT6ZLm+Hv6kxAkBtmFkkgScybB1dBgoCLbeXl/e8hBv8BAMfd9b8F73D/1foePp9frGTiw8ypWVUdTfH19WbJkCevWrePo0aNcfPHF/Pe//yUuLo7zz+9hGsvdAIuGtyajwwOcqXFTmdd/HpF+6iSFRl07igs/v5CJdymfrMnIlX5aBv2uG30da69eyyPTThPJWL8EIe2pwj9FczZcxLVRI9CZ3akq87IZTDXUG9j55k5OrDvRiQ0UeId4M+TiIR1kZ0/ljgl3UPL3UhIzRLl6b4VBJ+ERBH79nDqEnD1aYBbVnEoqrEwmEyUlJR36TFlyLG6qNgF1dx93xVlDctCv0Nxb6dfTCPEOaamMz2EHoJ2nny3ZLi0xm1uDfoFBJn491hz0638aBP3q82FZivDqa4vOTZiMe0WIP0DmpyIIeOztrmmnLcr2wIFnRIa5FVILUnl759tsymofuJQXiwoLxfO4vqyeiiz1kxpr/aYaXhv4mlWP57MHnM1dA/9LdPkFhId3nW1xt8I72iULenYJSIHBD0Cs8rlYcmgyAEdLlHk9RQ6PJGlOEu6+rVn6fn6t0mU56gqcFZF8XjKjr7cs9SRX+g0K0+bZe1pU+kkSjHhCyM11Jl4RYmyoIlGzT0AfvNy8aDI1qfKil3H3dqdvnOh0tKo0rcqv4viu4zQ1WK/0kOU9AVZtWsWP1/7IsV+VB1eG/WUYs56YZTG5rKjYRJ1HJgCJISqizye+hJM/Kt/eEUxG2HwlrFvQvnLAbIb1C2HjZY57HHkEOp6U6OYrfCydTKyVsr/h2ZnjmNjnB1VyxUMjhnLXhLt4dd6rGM22qyM2PLmBVfevctjbzBI6vY6GqgZ+f/D3dqonqx5YxafzPlXlzyX3qVF+UYR4h5CTA2lpolsZ7qyq4tj/wMwVIjFGKbv/Lrz96q1EKJSic4e+C5V7Pu64E7ZeZ73y1M1HSDaOe8PqIaxW+rXl5E+w5WrnK1URyYMLUxbSL6ifttVwzciJkXq9qHZpi8lkorS0tOW+ztqYRU2RSFRrqmti47MbMTaJ74a1KkJbXDzkYpZetpTZibMdbr9F3HzAt5+QkwUY9zpM+lD8v3Q3/DJMebDEEpmfiPs39zf723qFwRk/wuSP7W/rIBG+4qas88i0el/+cvsvvJr4KtX51S5rhzVCB4QSO85yBb083/Xy0s5OSjGGGkh7XfxYqwLV6VsnQZmfwLG3IPPzzmujNRrLhOetp8LqiEH3wqI8x5J3FCJL9J5sFEG/mhqoU+Yq1g5jk5FXEl7hk7M+ccor12w280LMCyy7ZZnVbWR5z3ZSzU4g95mdhVMRiP79+/PQQw8RHx/Pgw8+yM8//6xVu/40WBz0/TwI3PzgbG0zwWyh99Az9LKhqvczmyGreZ6mZdDPx92HM/qdod0BuxKzWWS1uXex0adXBE3uwmDZXkZF+JBw7sy8k8A4jXo2FZhNZiSdssFqVZWEqdEbSYLQUBc3rCdTsBYO/VvI34WOc/54hjrI+w38+wspFJXIWfk5Dcor/cxmM9nZ2QTJRgLN6D30LNm4RJMJjrHRSEFqASH9Q/AKtF9yL3+P/m/sOxC+vyXbt5eewdiYsaSXpZNRvwOYo0mln8zx1cfZ9+k+znvrPHR6+4ugG5/diE6vY/K9yiXZamuhsXl9J7tpD4U1hfh5+DElzrosWY9B7w1DHoYwC3+PqV+1/j98GiRdD3GWM0O7lLzfYO+Dwmst3PJn8tHej/j3ln/z1/F/bfe5yYtFBQUw64bRjL15rENNsNZvKt7fZMY/xh/vEOszbNmA/VRvkD8tJkPXJHhJOhj5tKpd5EorpUG/8EHhLP55cYfXY2JEFmxODiS5bp2gA1pW+tXUtC7O9uigX1dSXwh1eUL2SgE6SUdScBIHig6QVpLmkD9KVN1xcnOqOXFCmyD7tte2sfGpjdy4+0aiR0Zb3KZt0K8krITFPy8mbJByuZHBF1q/X9Py8jDpGpHMevoE9FF2QGMj7Llf9D1RZ4qFbFeg09PinWisbfVmM9YJ/0i9p3N9X1MV5CwTCRNq5haD7hU/zhI1m80VD3C0ZDzxKoJ+HnoPXpqnrDpn3ivz8AzUtgpL0kks/GAhOnddO2myGf+aQciAEIt+qNY4Vdpz9Wrx+pgxQlHCKdx8IVqZdUwLYZNEn2Kodfy8DaXQVCESsJUy4gmImiWk2k+laJMYm0pS83fCMjYr/WQK18GJz2HQfRCkzhu0s5ETI4OCOiZ4tR1rbvvvNpbfvpzpj05n5v/N5I+X/2D1Q6vxCfOx6tXWZcRfKn4s4dMXGoqh0YmF+aDhwtPUyhygAzFnO34uBcjrFFXeB8kprAU6PiuSzkpC56bDZHSiwtEBDPUGGqsb8Q71trimU1xioiBwOQkBAzCZ+3euEpy7H8z8VSQ36RRI0ybfJZ7FnZ38Z4moWXCuctl1PIJc1hSZYRHDWHpkKYdLU/H1FePv4mLhp60GvbueyX+fjN5d71QiTVNtE2EpYTaTtttW+mVVZBHhG4GXm+PynGazmZOu8kWwgMPflvXr13PNNdcQFRXF3//+dy644AI2beqGEks9kbhLRKfvwM1bb6jnk32f8MT6JzTNIrNGSYn4our10Efh3ORPR30+fB0IuzQyVHeUca+TFvsLtR6ZfFR1Fbcss27GrtPrCIoPckmmmD2+uugrXop7yaLszanIlVZBQeDWhUWU3Z7KI5C/QrvjVR+DDYtEFpMDjI0Zy4x+MxgZIwZDFRXgRCFKy31acrSETc85/hwqSSvh7bFvs/nfmxVtL99/I+ISOC/5PGIDHPN36aVrmJM4h0UpixgeIzI8q6qEn44WnFh/gj3v7yFvZ56i7Xe8voN9n+xTdQ45QczXF1ZniSq/2Qmz8dBb1qPvUXiGikUXWTrFGn79YMLb4NENPaXjLxPyL8GjrG7SP6Q/AMfK2leJtK30UxI0dhWSTuKatdcw98WOkmtldWVsz9lOVoGoZOyttkdI3H0TAjvv7tp2KJSjctQH/FTk8b/WlX4laSW83O9ltr+xvcN7dU11HC8T3mdaVNnLVX4RET1cpjb1cVF11eh6uaMOrJkL69TJ2sqB57RSx3z9mn79nb4HfiXzuDZz3r5T+5J4RSLBSZafKWazGbc2ga0qjyoGnDOA4ARtnkE1lZ4k5zzONLe7253HJnoP8ayZ9p3rAn4yE96Gmb+IAE7BOiHB7eYjLCzGO1lxX50OmxfbllV0JZ6hpPk8zYmKYeQpG7qpps/EPoQPCtdsfr3tv9uoLanFJ8ynJVkxa2MWxUeKCewbyLQHp6k618WDL2br9Vt5ctaTmEytQb/ZzhZYFW6AGgfUg/peAJM/cU5hJuMD+DER8lcp38c3DhKv6RjdKlgj1CX2/cPuIRRV+g2+Dxae1Czgl1eVx5Prn+S+lfdpcry2yImR9nLIRl8/mgXvL2DYFWKOP/neycx/az6jllgfC9tiRfoKDhUdwugijzGreIWJz2bgbY4fI2IaTPlUBI2U0lAKWd84fk4bxPjHEOYVBZKJQ+W7LW6TsiCFs185m4DYzi1WOLHhBM+HP8+ON3dYfP9oXi7bB8znm4ghnXcv1BW0+hT791dewCFJrQE/kxGqM13SPJdRXwT7nxBFAy5gWKT426QWprYUbTgq8Tnp7kmMv308enfHbSw8fD24evXVzHp8ltVtWj39zAx9fSjeT3o77EfdFahaScjNzeWpp55i4MCBzJgxg2PHjvHqq6+Sm5vL22+/zcSJ6qUhe7HAiCdgxJMO6SRJSFz1/VU8uuZRCmpUCNI7iCztGROjXdBlZfpK7lx+J6syVAzOujMmg/DXCNGgwspJQkPBoK/ioPvHfHPI9oCioaqBtOVpnaKT3JbQ5FCiR0fbrPb74fAPJL2axD83iUFtr5+fHQbcBBdVQLBGGXaBQ2H8/yDlHod2PyvpLNZcvYZHZt6HJIn8Brls3hlW3b+K3x/8naKDRQ7t7xXkxczHZ5J0lv1ShYYGqG5WvuitMu2ZXDf6Or679DsWj7qg5flVoVF3N+7WcdyVeZfiDOvbDt3GJd9couocbf38tuZsBWBmv5mqjtFjyfgIPpO08XpxFb7xIrvXxiKsHPRLL01v9/qpi0XZm7NZ8481nZ55a4s1mWsY/854/pUpJkldFvSrL4KDz0Ntc8akySgW96rSbe/nChpLxSKPbxf5N1emwY9JcEBZxV9SiHjWHS8/rngRpeRoCZ/M+6RdkkJMjPhX66Cfoc6AT6gP7t4dM6uNZiMvzX2JOyfcSbiP8zffaSHtCVCTAcVbukZdZODtkPI3VUmrA0Kag34OLp6Me/BM0sde2uIv7yxJZyUx7N5huPtYzuaXJIkjtx9h2/XCR7quSZ0mVf7efF4f+jr7v7Tso2moDGNA3iP8JfJ5ZQc0NJ/fpw+EjVfVFoeQvcJyfobfZ8CR19q852TCUdAImPgBDP678n22XA37HnPuvG0QfZmZgrwGVfvVNtWy/sR6vj34rd1tm2qbKElz3uspc10my29fzu8P/d7yWk1hDe9Pe58/XrIiS2kHXw9fxseOZ0KfCezbJ5Ib/fxgvNpbq/wA7Hmw9fedd8LK6c7JWBob7W9jibAJkLgEwqeq37e+CNacA9tvFb8HDYf+NwkPUzu0rfSzetleEeJHI2qaanhkzSO8uvVVmowaZTE2Iwf9TvXzOxU3TzdGXjOSsGSxOKN31zPmhjGKVZzaYjKbuPCrCxn8+mAOFx9Wvb9Njr4OW64Rn7E15MQLtfft0dchd7lj7dp1N2y8GKrU+XEqQZIkRkeOw6uhL4VVpVqoymqGb7gvo28YTfRoyxX2BwuEIkWILhF3vYJqOy048CT8MgJKLAci7WI2w7rzYNUZQmazsyncILwFK5WpebRgrIV9j8Jx10jNDosQQb/9hfsJDRPzWjmR3lHqyurI2qheJl4p8pqf3reCqkaR7NqTkv0VB/3OPvts4uPjee2111i0aBGHDh1i48aNLFmyBN8enRJ5euHp5knfQFEbK2fAuhI56Nevn3bH/OnoT7y67VV+OvKTdgftSnz7Cn3wfpd1bTuK/6Bvxb+I1YlJWXFtMUU11gc6WRuz+Oyczzj8g8aDLDuc+fSZXPaD7b/V7rzdZJRlUFApnhC9QT8FuHnblCFRhSRB/xud9vbT6yGgeV1KiZ+a/6kmAqdw7pvnctXvV9n1g7RGQGwA0x+ZTvw0+1rF8uCkKmQDL+54gvUn1jt0zl66HklqzVzVytfPN8JXkTxyfXk9oM5LUqZt0C+1QGQijowaqeoY3ZbcX2H1HCjeZvl9WX6ktPNk0B3CZIBq61VUctAloyyjXdDlVFmow0sPs/7x9RQfUj8rstdv2qJwfyHrn1hP8ZGO5z1UJLx/AhtFlVWnBf0KN8DWG1oXAE8uhT33tWaklmwT907a6+J3OQjYGXhHw4yfIaWLKv18+wppXDdl87K+AX1x17nTaGwkt0qZnp13iDcn1p2gLKO1s3RVpV/k8Ehu3HkjI68Z2eE9Pw8/7px4Jy/Pe9npqpnSUti4Ufy/xwf9Jr4vqhO6wmAz6TrhJaji3GfEn8H1o65nWvw0h0459qJ+VIfEUVIqUWXdPlUVSvpMXw/xHXMvcudJnydZ9aCyPqahsgFjg/UAu5ztrkhKsS6vuYLpd/vbak30XBj0d+jXUe7XYSQJEq8W/agSjPWi3y9Xp5Jgi/iQY3ywoA+jPJ5Rtd/+wv2c8cEZ3PzzzXaVlt4Y9gZfLnLCJ6yZ+OnxLPpkETP/1Zrs5Rvhy0VfXoSbl5vTSUK/N99W06eDh9p47q67hHSloaa5sZfB8P9zrF9qrIBfx8LOO9TvCyL5auJ7oHdAiq2pUiTzuDVXanmGwvg3FUmFhoWJy21stJPYamwU1a0ZH6lv3ykkBicS4BlAg7GhxZ9RK+T5kbWgnzNjTWukl6ZT3ViNl5sXyWHJ2h68ZBtkfwPuduZqmZ8Jbz+lQZvGCkh9DHbf55jH6cDbYeKHyvtBlXx/+decdSCL8NLzrM551z62lo9mO38/qiFqZBTnvXUefSZYlo47ViYCVzGeAzuvUYlXC6nOkDGO7S9J0HcRxJ4HOselIB2mYI1Q5TKpTJjwjRfqAePfdEmzBoQOYF7/edw05iYCQkXikjNBP7PZzIczPuSLhV/QUKkuYQeEJcu6/1tn08eytllhusZdBD/CfcLxcXdOWcHPT0UFsJMors1yd3fnm2++Yf78+ej1Gi0c94LO0iK82Swy19z9hKGsSuID48mqyOJExQkm9Z2kQSutI2dWaunntylbyPNN7qvc26gXBRRvwS/jMfp4n4lvXTI13kfYnrudcwZYlk/rO6kv816dx4CzB3RyQ+2zp2APABGmkVTSG/SzSU025P4CseeKTGAtMdRCbQ4EOHaPVDdW4xNipKIi0K6fml6vJ8mOWZBfpB9+kZ3zAJV9rCojf+Efa5/hlrG3MD1+eqecuxftMJvNZJZn4hbsCcUxmvr6GRuNHF56mMrsSib8dQI6t/Z5VmXHy3hnwjtMfWAqk+5R/6xuG/Tbf+t+9hfub/Fh6fHU5YkJudFKFUWf82FRPnhHdm671LLhAuHtd0m1RR8IOejSZGriZOVJ4oPEYEqu9CsuBoMBxt40lmGXDyMsRd3DTkm/aYvszdmseXQNMWNjWrK2ZeQFJe9qFwf9jI1i4hp9lphAF2+G9HfEYnPkTEj4C3hFQkRz/xs2EUY+C30Wid8PvyiCgpM/1XaBujui94JzLVcQWdxcp2dA6AAajY2U1Ze1JA3awifMh/vL7sfNq3UKGduc7JqTI6YwXRFvcgSjEZYtg08/hbo60OlgggULpx6HEs+ZbsK5A8/l3IHnOry/ry9EhJmoyijkxIkohqq3mW6htqSWj+d8zJgbx5B0s+1+09tN+JxW66rpN6Of4qSd+Gnx3JFmPXhxpCyVKi83/IISADuLhFXpwsuzKyredW4w6jnXHLuxQlhjBNhY5DcZRX+34Ljwa9OIsPh4ijb1Ia80FINBuYrR8Mjh6CU9xbXFnKw8abMvHXvrWMwmM2az2eGEBXnf4VcM7/DekEuGMOQS9WPBmsYa7l91P4PDB3PloJvZvFmMWR2S9hz9snhuywkog52Qm/QIFMfRO1Bk0FjunEeVfxLMdaxq0t1djM9LSkQCV6C12JIkwb5HwD0IEq506gGqk3SMihrFuhPr2JW3i+GRHe8PR7EV9HN2rGmN3flCgnJYxDDlcsdKmfQBjHnJfoWyyQANRVBxCMIVrE16BMJZm0Xf7EibQ8eJHxfh4+lJWJhQEikstJxgUplTSVlGGQ1VDXj6a+tB6ignqkXQL96vE4N+IWMcD/jJ9L9Bm7Y4wrB/iDmSrwo/UxmlPpQO4KZzY/kVohL28+a1DGeCfpIkMf0f0zHUGXD3VT/+TVuexpZ/b2mRJLaEHPSrlEQ1YVygc4ouer2exE7MMlRc6ffjjz+yYMGC3oCfxlRXW8jCkiSoPAQVBx06Zr+gfgBklmc63jCFaF3pV91Yzd78vQBMiXNdZ9OpHHgatt2s2GPFZcQvhnP2UWwcRVCtGExsz+nokyLjFeTFhDsmqK4+cYbyzHJ+ueMXsjbZLs/ek78HgIDakUCvl5BN8lfA9puheKu2xzWb4aeBsMmxCtbbf7kd/6f9SQ8SWUT2KqxMJhP5+fmYFJj/pa9I5/urvled5frrXb/y4awPFe0nD05qfUQl7KAw5z2Feul8bvn5FhJfTeRYoPCn1Drot+q+Vex6Zxc0z+HNJjN1pSKQJUkSAbEBhA50TB+2bdDPx92H8bHjW6oPejxJS+DiCog8w/o23T3gB8KPJvkuq8FLvU5PYrAY9B8rbZXyCQoSmfVms1gwCk4MJmpkVIfAsT3U9JuWGHrZUG7cdSN9J3dcwJSllnSlou+L0E6hqj3H/gdr58Hx5qzjhKthwQkR8AOx8NvnPLHQAmIMPfi+1mSUhCthwK2tQUBXsvUGh71uu4r9t+wn7Y40VYuDbQN+AFFR4s9eV6ddtTTA4R8O88fLf2Bo6Jgtv+HEBnbl7aLeUO/Qsc1m+Ne/4J13RLsHDoQXXujhlX55KyD9XWiynq3sUuqL4LdJkPqvTj1t3M7vGbzhLY7tq3XqODWFNdSV1FGcU2y1z/w89XOmvjeV17YJWctSz1IW/7yY0ddpI5+/zHQn64YOZmedAk+niKmiqjP+Uk3O3S0wGUT14p77W1/L/Byyv2v9fd9j8JUv1BeKhXUNPX2DQ915eP1Wlh29vaXSXglebl4MiRCBtl15thUIJv9tMlP+PsXhgN/ej/by1YVfUVemTlrWHoeKD/Hf7f/lX+v+xZbNOhoboW9fGOBIXmfQEEi+XbvGzV4LY15Ut0/tSXEvrb9Au3aoRJGvn84dpn0PZ23RJGNmdLToi3bnWfZscxT52W7J08/ZsaY15PWeUVGO+QHaRUnf0e9yWJBlP+BXX9TqW+nfH/ycHEzUFYgESBcQEQFmzOQXWP685r8xnzuP39mpAb+V969k2c3LrL5f0CQk+/sF9Hd9Y6qPQ+FGbY9pMsKBZ6DyiLbHtYdfouP9SuURl3vsyoUbzsp7Dr5wMMP/MhydXt08GWD6I9O5afdNNhWa6poft+Vm8R2Xk3QdxWQyUaBmkOEk6v8qvWhKSYkVCYg5m+DMtQ4ds7OCfiYTZGeL/8dpZF+yLWcbRrORvgF96ROgcVVSV5H7M+T9KiYmXYl3JAQNwy/Ih6Ca5qBfrvWgn0xTbROVJzUwXFNA4YFCtv9nu00Js9K6UrIqRFDQs2wE0FvpZ5M+C2H6DxA5Q9vjSpKQoEj4i0MeDVF+UQBUeB4A7C8Ums1m8vPz7crmAGRtyiL1s1Tyd6vLfq4rraM6r1rRgEEenJTpRbVLSliKqnP10j1IDhWZ5CXuItlEywVrDz8Pbt53M5d8c0nLPfX60Nf5YMYHAAT1C+KGHTcwcL5jWYty0M+ev0YvXUjiNTDqWZv+WrLEZ3pZqwedJLUuFuU1z/lNRhPH1xynKk+5hp2aftMSngGeRI+KxjOg48RfHmN61iYhSQrl6Byh74XCPzbuYvG7d5Q6zzzvKBj3XyFxDcKjwgVeKTRWwPEPtV8oUEtdHuy8G7K/V7S5IwvP9eX1bH99Oxm/C+lad/dWSdpcZSqhitj70V5W3LvCYrD7nhX3MOatMaxMX+nQsUtKYPduUd13xx3w739D/05YS3Ipx96CbTcCXZRk6BECdSeFCoQK6prqSC1IpbLBsblG5Oyh5PWfRpaTdi7hg8K5I+MOIi+ItNpnHiw6yKbsTVQ3VpN2Rxon7jqh6hz7PtlH6uepVt+vQFhzpET1U3ZASXKsmqS7onODmHNFoqrMzjtFxbaMbzxEzoImjfRc2yBJEN2sqpencr1dDrbszNupcavac2L9CXK25mA2amvIdbBIJJsPDh/Mqma12tmzVa4XH3tHVPhpjdyIpko4oVAa1TtG3Cf9rtC+PQpRFPQD4TvoYX3BWQ1ygGxXvrby93JSpKWgn7NjTWvIlX6jojUO+mV+IRIJlCTk69yFl2l9Eaw7H4693fqefL1mE2y8CH4bb9sjUCmVR+CHWDjommrq1f7XsXJEBOsyN1h8X22CoRacWHeCzDWZVt8vM4kHfN+ATvDMTv0XrJoGZdpJR1O+B/Y+BKn/p90xbVGVLrx3nXlO7nkQtl4PtRoO7NtQVldGladIIHU26CdTlVtl0ZLCFl6BXkSNjELvbr24Ta70KzGKMV+ck/eh2WzuDfr9mZC1+zvghBl2fKCIPLs66JeXB01N4Okpsny1YFOWkPY8bar8AM7cAPO6ge+Q2QQNpcSEV7QL+tkaoDVUNfBC9Av8euevndLE/vP6c3f23Qy+aLDVbeRK0MTgRKqKxQC5N+hnA89Q6LNA/Ks1Qx4Q3kUOLBrKMoTFkphkallhNeGOCdyddTcxY2NU7bfoo0Xcdug2RdsWFYFJaqLEJBbqB4X3Vvr1RGQPvDzzHkDboB+Ap79nO4/JQRcOov/ZrSvLjmSkychBv1+rn+OWZbewLceK/11PpGgLnPxJZEae5jw6/VHWXL2GiwZf1O51OZlKVlQ4tvwYH836iH0fazgJtUNlTiWN1R29IGoaayipEwNY78Y4QkKUy6CpxicGRr8Abs55JwAiC3vrdfDHEoeSVWziEQgXV8Lof2t7XLVI7nDkZSHr7SKMTUZ+ue0X9ry3p+U1WeLz5EntzjPvlXksWb/EYj+ZUykMBGMDYh06dn5zTlBkJJx1Vs+RJLXJmFdhxnKbSQYuRaeHhdki0UEFk9+bzPA3h7PhhOUFSHsMuSCZ3OSZZBVr0EdgOxB+skrc4P2C+tE/pD/RXtGsvG8lqZ9ZD+S1ZcOTG1j3r3UW36upM1DrLrJph/WxI8dVsFZ4RtVkKzpvj2Lie8KSQGbKZzDujdbfk5bAzF+E9KIL6BPTyC1jb8H/+D9V7TcmWsjA2av0A/jqoq8c9vU77+3zuGHHDfiEaXO/yxwoFEmYiX5DOHRI9IkzZ9rZqS2GGiGlveN2143dtt8uFGaKtlh+v/o4ZH0t/i/pYOrXEHeha9qigFP9mW1SVwBpb4p/naBtpZ9JQ5Wp6uYCchdY91lFrlbU3K889R+w6x5aZFiUYDYKuf6qo82/m2HtObDnIXGc5Dsh6Xrw0kB+yn8gJF4rgtYuwOxRTqN7MXuLrCf/Z6zKYNPzm1xyfktc/8f13LzvZqvvVyKedfFB9mXonWbYP2HYYxCsnTwuIWPgjB9hwjvaHdMWWV/BuvlQtsfxYwx5EGatFBYKGrP6+GpCngvh/l1ChUWLoF9dWR0vx7/MqvvV+biXHC2xONdtixz0K24UwWdnK/06m9MoNaxnIi/adcBkFF9WnYfqwYpc6XeiQl32oVrkiX1srHaT5c0nNwMwuc9p5OcnSeDZeRKZVqk8DD8PYVbMo6zf/iB63PH38Ke8vpxgb8ulIp7+ngy/ani7RWtXotPrCOhje7FCzvoaETmyJWjeK+9phbo8IZXj2wkDJJUMDheB3XzDIYZioqxMuxwUrSfCliguhhrPYxgx4OvuS6y/YwuPvXQtI6JEtXCxIZMmfTnl5UEuPd+sx7WbwMkByk0l35J6bBszE2YyPna8ZsfvUg6/ILJwL+8o69fjkANMkz6w+PbEPhMtvh4XB5s301K9kjgnkTMeO4OUhZ1XVfzp2Z/SVNvEX4/9td3r2ZVi8u2j98fdGOiaZ3DtSShYJ6SVtFJK8I0XMloBKa0D14pDEKhR0obeS/x0JV5hcN4xxdJSO3N3ctsvtxHkFcSvf1GW4OUb7suVK68kenR0y2uxsbBzp7aVfoF9Awns27H6wWAyUFAjFkYdffbKQT+tkha7BT4x4qeH0T+kP3vy93C05Cjnot7fT7aYOHHcRFO9CXcvx5Y3Di89TGNNI+ZB1hMCTlaKyW9bNZot/97CsMXDGLbYuh+MzMVfX0xDVYPF9w5kn8QsGdGZPEiMiLa4TWtDlorgfv+b7J6zx6Fza1+9GHVmp54+KsaDkfWroE5d39ISbMm3L6tobDRiqDeo8vXLWJVBwuwEJEnCP1r7yMvBYpGEGWIcTCGQkKCygt/NF+bthMZSkQDgCkY9CyGjhXcviCoUnQeMe138fvwjSH1MKNz0WdDl2RyKK/0AClbD9ltA0jvlA5Ycloy3mzcGk4GsiqyWdUFnqakR//p2kotAfnU+BTUF6CSdpt6EgEiOqc1Wd394R8FlbfruxjKoywWfPuI4fS8QP1ogSTDBdVLxw0LHsb3mO45W77C6zY43d3Dou0OMunYUPqGuX1cBcPO0/Ow2m82ML32JgoYskkId8KdTi1+CCPxpTez81v9XHIRA60UOTtP3IpEA5ownoQu9JWWFrOOVaQyQ6qmp8aK+HrycmEJ5B3sz7ZFpRI1UPrA3Nhn5T8p/GHTBIC755hKr28lBvxlxZxEV5sPYmLGON7QL6A36dTFlZVYeNpJOyFn4xqsO+o2JGcPyK5a3+MS4CnnCHKPR/NJsNrdMpk6bSr+KgyILM2Jqq5l1V+EVBUk3UJc+Br3Zm2dCirj3DvtSEue8dk4nNE5QfLgYN283guKDrG4T5BXE2JixDA+ZwE6jGBf1yttZ4eh/4cCTYhIWoo3fSAc2XirkVmYuV7VbUkgSHnoPGoy11HmcoLzc9iBOkiRCQkIUT45NBhP7v9yPscHIqGvty4JUF1Rz8OuDJMxKUBTkLiqCai8hSZASluKwN0cvXUuIdwhxgXFkVWRR6b2PsrLpXd0kRZhMIvBsxkhalagwGBE5ootbpSEp9wgPtq6WxdaCqmMiQ1gl8c1JhHKln5unGzP+OUPVMdT2m6cy9LKhFtUAgr2C+fecf7N7fwMVuCjxZv+TcOxNsaBiy9tRLW0rSIo2w8opQkpu8ifOLRDm/CzG7EFDnW+js6iogPF082RrzlaCvdQNpBLPbD/HkCv9cnJUHcYqZpOZ6oJqfCN8O1T65VfnYzKbcNe5E+7r2M3XttLvtKA2V8iQaVFl4AyVR+DkD2LBSeF9OCBEmIallaY5dMqYGAiszKbfH1+w8fUzmXmPY1Jwm57ZRNnxMi7ZdonVPrOlwtQ/ln+u+SeV9ZXceehOIiOU3UgRQ62bnx7IEdKefqZ49Do7z77RLwj5aBdVu/2ZiY6Gv/1vKwOHBqPGnVJWMMmtyqW8vpwgryCr21629DJVz+XUz1L57orvmPP8HCbf65rEaLnSz7taLETHO1LI4JcAuHBR3jsaUu5q/b10J+jarBL3vQgCh0D0PNe1QQWqKv1izoYpXzjddjedG/tu2Ud8YDzuenenjtUWudLPz6/je86ONS3h7+HP0suWcqL8BD7uGged/JOc7zs9Q4SSlxXPbs1oqgJ3bYP842LH8l4WZButB/1mPDaDWU/MwjvEW9NzW6K2uJbsLdnEjInBP6bjtUqSREzxlQRUQ4Q2KriW2XkPRM1uP09wBVnfwsaLYfz/nArw2yRgQKu3ubOU7RPr2BqON6L9ognxDqG0rpSmoEPoy0ZRUtI6l3AUtfNkY6ORqQ9Otbv2Jwf9loy4joiI6xxsXSuSJBHciQvYp8FqSs/GanWLJMHUL8XDXyVBXkHM6z+PgaGO+QQpRc7mjbaTjKgUSZJIvSWVE3ed0D6jp6vI+BDWzoMaJ40mtMAzBCa8RX3oAgBqy9Q9NQ0Nrq+4WHrtUt6xU/Z+7ahr2X7Ddi6IvA8QWdp6FyUU9ngizoD+N0KQC4MBZgOYmlRLpbnp3Fr81Kq8D9qVVdTpdMTFxaGztxAiI8GaR9aw/on1ijwGCvYVsPyO5WSuzVR0+OJiqPYSfn690p4a0FghFs01lKJRihwsq/TZo6nMrCspLhby2vW+x6g31OHt5k3/kJ5uSNWG8MmQ0HU+LJoyey2ctdnq2/WGet7Z9Q6PrH6kXV8ly3tmZbXvXk0GE2XHlenQqu43T2HaQ9OY/nDHQHikXyR/m/w3ZugeAlwU9BvxpJB0i3BhIN4rAoY8AqOecy7gZzbB5itgu3Vpok7FWA/5q6F8v91N5QTBsvoyyurU6RtXZFdQliH20TroV5VbxYsxL/Lb3b91eE8OvET7R6NzMDFAXoTt9Eo/UxOsPU94CmnJoefguwjh49KVlO+HPQ9AkXJvS2eDfno9hKWE0eTlT1mV4/nM5755Los+XmSzz2xb6ffK1ld4edvLNIY3KloYbaxutFrlB3C0MBOAUElB0ETSQfBplOjTjYiJgerGEPLy1D0TAr0C+WDBB2xYssFukEJtcGTAuQMYc/MYRi4ZqWo/pdQ01rTawhSJ4KWqoF/xNsj8TEh8diZn74a5baQ+g4ZA3EXCg60b0LbSz+5U1CMI4i/VxNuvf0h/TQN+RiPU14v/W6r0c3asaQlfD1/OTz6fOybcodkxMZuFiojRej+sCp0e3C1EQbVix1/hh76qfXLtccZAUf1V6ZZOSa1l2bmIoRGEpYR1SlJzzvYcvjj/C47+fNTi+2Zza9DFZZWmtSch411If9v+ts4SPhlizoHoua45vrEejLblKhVTcRCWjxBjTA2RJIlhEUIdwRQmkpeLNLDDlKkpUvYs8vD1YPaTsxl+hfXYg8EAjc1/Tm+NYuA6nY6+fTtPia036NfFlJTYGAFEzuzWGXxylqxWQT+ZuMA43E4XU/KEv8DY/4C/RpkWGiBLdcjSmEoCIpv/vZnnw5+nptC1g/nR149myn3Kqjzl6gd5YbQXC0TPEVlErpJZAZj2Lcxe5ZivX4SYWFZ7HbQbbDGZTGRlZWEyKQsK6fQ6LvryIm7ccaOiAWvsuFiuXHUlA8+znyxRUwN1dZBUcC+7rjvAI9MeUdSmXmyQ85PQnj/6n04/tewVUemzR3NPP1chL6zro4W/29CIoehd+T3vxXHsfC46ScdNy27iyQ1PtkgWglh4dHMTfY3sdWA2m/nf6P/x6bxPFT271fabapEnaC4J+nmGwICbXSvP5d8fRjwOPk6mlppNMOlDGPygNu1yltocWD0b0l63u6mPuw9RfiLylV6mPGBUlVfFy3Evs+Ep4cMmB/3y88UE2VkkvcS428bRb2a/Du/lVLVWWzlKl8l75v0GuctaZTgbSsRiubMJL5Ezof/NimVdXUbkTJizSZXM2YDQ5qBfiWNBP4D4FG8OTr+ZxmT7EpvWiBoRRcLsBKt9ZmVDJVWNVYDwkvRyExVGpcdLqcyptHv81M9TeSbgGdJXWP6eZZSJSr8Iz362D5T7m6hS7oIkqT8D0dGgkwwkuH2P8WTHpANbXD3yaqbGTcVD72F3232f7mPVA7a9h8wm8Zz3CvRi/hvzXSazl1aahhkzYT5hlGaLB7qq+fXR/4jEl3olOpZ/HuSxUUMDVNrvIgR1ecJ/uBtR02b5x1LQxdVjTc0o2ggbLoS9D3d1S5QROFj4+jVa84NyjP6xwfjUi0TRDcd2Wt3OUG8gfWU6JqNrP9fIYZGc/975JMy0nPCSmnuEfL9fqfXIxMdVSqM+feCc/TDufy46QRu8o2HGMvBt7mQ1DuqS9S18HQC5yiT7bRI4GIb+Q3hMaowc9Kv1F2sZ8tq0s/z+0O+8EP0C1fnVmhyvrrmY16ir5WRdGvWGeqePaTKZyM7uPE/m3qBfF1NaamfRpqnSoSqxFekreGL9E2zPsW7Q6ix5eeJfrYJ+jVplJHQngobBwNvaexN0JTvvYmDVXQBklRYw+8PZ9Huln12j54A+AUSNiHJ50G/UtaOYdM8kq+/XNdXRYBDZWbLPkUPyI38GTE1d3QK7zEuax5VDrsO/bjiVlbYXCs1mM6WlpYoWumVix8cqlqXwCvIicXaiRf+gU5EX4AN8PRjVZzDJYcmK29SLFeIugqizXDKotMfcpLncM/5+Ykovo7a2NZurOyMH/RqC9wKnmbQnwI9JsNG6tn6PwlAHxz+BvJUW3/bQexAXKCZ+x0qPtbzu5tYqny4/7yRJYvhfhjPsimGYmuxPwh3pN2XyduXx6TmfkrEqo8N7u/N2sztvNycLxYQqwrpinXoK1gpp6s58hpkMcPxTKNzg2P46N+Ef5GpJIKX4JcKYV0QQSAFJwSLBML1UedDPP9qfyfdNJnmBeP6FhoKnp6gIUCRlpuD45/znHAYt6lhJ3yKxGNADg36x82HmCght9qQ68qpYLM/+3rnj9lkA49/ocg8rPENEFrsKOTK50i+rIsvhxRTZ1y8z06HdMdQbWjzWrPWZpXWlxAXGEe4Tjp+HX0vQ7/dZv7N0yVK75whOCGb4X4YTlhJm8f1E81wG5vwfEwIX2D7Q7nthwyLVChu9KCMkBNzdddw14UoM+7StbmjL4e8Ps+WFLVarP5vqmvhw5ofsfs++R6CzjIwaSfHfi/lt8e8t40tV8+vR/xbqVH6d4LfVg3B3b022VuTrV5MN38cIeXMnKK0rZcnSJUx9b6pD478OzWpe/vHysqyu5MxY0xqvbn2Vn478RG2ThgGRoOEw6gWhgtQTGHAzTP9OBKQ0xN0dIgzCk2xDhnWJz/VPrueTsz4hd4eGZs0WCOgTwKglowjpb9lE9NN9n7Nt4Nkci34KT1cW8fr2Be9O1nzP/x2W9oPC9dod0yMYImdAgEbrU8P/BWETtDlWG4ZFiqBfuYeo9JPX15wldnwsgy8aTGON/QWdn2/7mS8WfIGx0boNhxz0qwr8g8FvDGTU/xyTkG+L2WymrBMzzXuDfl2MzYi2oQ6+jYDtt6o+7kd7P+LRNY+yJnON442zgdHYOnjRYsJ8svIkYc+Fcek3l2I0qfe+6ZaYjOKnO1GwFv/aVeh00FgeypaTW8iqyLKbXTv0sqEs2bDEphdFZ/D5/s/xf9qfG3+6saXSrzfoZ4WNl8BvE8Ripisx1MLhl4VchkquHnk1H174DlHVQt5AcRakChoqG9jxvx1UZFXY3q6qQfFkRR6UhFlet+lFKaW74Y/rxD2q94JZv7XKpHTiYtaUuCn8e94zxNafBdAjJD5lee1yz+agX9RpFvTzHwg+p0kZtyTBH9dA2htWN5GlWdsG/aCjrx/AlPumcMY/zkDv4drKzsqTlRxffZy6so5+JfeuvJfRb41mT/0PgMaVfgefhV1/E9nunUV9PmxdIjxwHUHrLF1nkSRI/isEK5PKTwppDvqpqPQDmPPsHJLPS245pRykznXt+hDT46fzwlkvcMUwxySA6+tb+3mXBf1qc0VFVtvf5fFY9ByQK4GS74RRz0PfReL3hhKo0yBq2pWYDKpkRiN8I/D38MeMmYyyjkkGSkhIAMxmyj78kV9u/0X1/oeXHuZJnyc5/MNhq9v0C+rHibtOUHCv+Hy83UVSWeySWAZfPNjuORLPTGTRx4sIjLOcXBZWO4WBeY8yI9aOl/qEd4X0cW91v0uQJIiO0fHvLZ+RHviiqn3zq/N5Z9c7/G+H/WqROc/P4d6Ce/H0t7yKXZ1XTXlmOaXHtK3ysUaoTyjhpuEYDCK4o+q57hUhpCl76YD8XDyhpHjPty8k3w2x5zl1Tj8PPz7d9ymbsje1yrY6gS0/P1dQ1VDFnb/eyflfnE9No4YJ5x6BMOgeCHCtBZJLMGjrHdjf7QxCK2fgY7AeUBxy8RDOfPZMAvoEaHputZwoFxVRAeY41+Q17bgTDr3gggMrwKevWANp0qYqDYDYc2Dmr9omYZjNUH5Au+PRWumXb9Y26JeyMIWLvriIkCTLQeS21OTXUJFdYXNOLUvLGv1EBq6cqNuT6A36dTHV1RIN1mSl3bwh5R5VEiky8YFipUiLB70liopE4M/DQ2T3OsuPR36kqrGK7Irs00eirGgjfO0Pxz/u6pa0Mm87uvn7iY8HHW709xOZCttzlVWEms1ml2X7ZG3M4q2xb5H2i/UA5OHiwzSZmvB082qpfOiV97SCZzh4Rbm+ylRyg91/h4wPHNtdgqAg8X9XJLxkbcri55t/Zvf7tjNlPzv3M17q85KiYxYVQb1bPhvDlvDiFnULAr20Yf//QebHULan9TWzWVQ+bLq8UwN/rr4PtUZeVG90EyPk08YHV2bmcpE5fjqg94IzlgnfOCtYq7Rq6+t3KlpmVFsi+fxkHq57mEEXdKy0yqoQDTKXiwZqGvSb9k172ZvOwKcPTP0GJn/m2P4rp8JvE7VtkxaYDNBYbncz+f5zNOAi34uyxOfJkw4dph07/reDby79htrijgHVEVEjuGfSPSxMWejQseVKRF9fF/nD1BfBiolQuLb1tQ2LYO05HSUZPUNg0L3Cow1g70Pw62h1Ki9HXhVJXlXH7G/bGWy5En4aoDgYLkkSf5v0N56a9RRBXkEOnTI+HpAkzPkF5O3KV72/X6QfQy4eQugA+5NaWTJervSLviOaMTeMUX3OUyltju3YnVeHjXdobaAX5URHw7ac80kvVZdQlV6azg0/3cBTG5+yu21wQrBNNZLgxGBu3HUjs56YpaoNztA2oVbx4nrheuHL3YtFBjQ7vBy2nk/QnjEvQh/ngn4eeo+WKprd+c5XisqVfi7zUzuFvQUioTHWP5ZwX40GmIUboKlKm2N1JsZ6WDldPFc15KyQm5l0dA2j9daPGzk8kin3TSEg1rVBv8/O/Yy3xr5l9f3s5jlHkM4FHmiGOsj6AgpcUyhjl4CBcH66CNR1Z3bfC78M1dQ3emjEUG4eczN/iXsUMybNgn4yZrPZZgUfwCXfXsIN226wuY0c9Gv0Fg9IOc7Sk+gN+nUDbFb7jXwKktTLnfUL6gfAiQrXaILL0p6RkdooySw9ImRRHJ3Ad0skPYRPA99+Xd2SVnTC1Lm/KCogxjwOgG052xTtvvGZjbw9/m2rRrvOUF9eT01BTYt/gSWOlwu/iyiPJGpqhMREH20VD04fJrwFZ9iXG3IavQfM2QAT3nNo93pDPU2hezHoqmwGWyRJIioqSrWhdNJZSSz4YIFN2ViA+DPiSbkgRdExi4uhyieVPXzAWzutD1J7scO07+D8DAgd2/714i1QuhMaNHR0tkNJbQmVYauo8jrYI4J+svzSV3O3UHJfCRP7dMOAQy+txMwT/nFWaKn0K2u/aC8H/U7NEP/jlT94IfoFaktsL6o72m+23V+nbz9VMJlNZFeIrFvvxji8vDReCHLzhagzNTygQvqcLwIwajGbIWwihI7Xvk3O0FAC34aKibodkkOTSQxOJNgrWNUp6srqeHPkm/x6p/ANkYN+cv/kDPl78jnw1QHcvLVPXJKDfqqq/OoKoDqztVqvbC+sPU9IM52KZ5gIygQObX3NbBQSnJKdqXfEGRAxXZ2cl6Eaqo+LRK/uQOz5IpBpspbV2pF/zvgnD057kBj/GIdOGRQEgYFwZNI1TP/QgXnzjH5c9OVFRAyNUNxnykE/JZKkJqOJT+Z9wo43LUupNRmb2F39K9VehwkOtpHQ0VTZ6+XXCch9We7JBjAorzQaFC6SZLIqsqhqsB9gqMqr4tD3h9q9tvu93eTvFYFrn1AfJJ3rJXtv+PEG7vntHvaki4wyxQm1hlqRzLDejiTtn5hBzXlTioN+Mkbl/acl5Cqa/YX7nToO2K/0c3aseSp78vcAMCraeRk9QPSba8+FNfO0OV5novcCz1DwCNK075dl+YsUTLWNjUYM9a5TjvKP9bdaAQ+QWyPmHGHuLkgGdPOGBdkwvgvXc5rXZ6nNFX69zlC8DdYvhKItTjerHbHnw8A7WtuqAf6e/rwx/w2uGXILEjpNg341hTW8O/Fdfn/Iwhj9FHRutsflctCvzlNMxrWo9JMkicjIzpOS7Q36dTGSJCm7wVV28nLQz1WVflr6+VXUV7DmuMiuWJB8Gg0aI6YKubqIaV3dklaqMyB3OQOTxATVp1wE/ZRW+g27fBhDLx1K/DTtMxwGzh/I3dl3M3C+dckFOQvdpyEREJIZbt3ELvFPTdhE8HJM63LCOxP4PHAkpX4bbcoq6nQ6oqKi0OnUPbZ0eh0jrx5pVT5HZtbjszjnNWVZVsXFUO0lZm8pYcoChb1YQJI6LmxKEoz/H5y9S8gFdRKPrX2Mb3zmkB32XreX92xqal20jo2FEO8QPGSpuNOB2hzY+4jzE5/uRlOlyNi1gFxpZU3eMyurfeGru487/jH+VOfZloNxtN8EyFyXadHPr6imiAZjAxISXk2xhIdrZCOW9iYceMrq36hTMNQKP8GKg8r3kSQY9zqMfdV17XIEz1CInis8bOxw6dBLSf9rOs+f9byqU3gFeaHT61oCc7KvWpptxXhFzH9jPg/XP4yHb8e+bWX6SrbnbHfY/80hP7+jr8GPCVDVfHE6D8hbDuX7xO/Geth4qagkkCQY8zIktJEfnbdDeHzbo99imPJ5a3Aw6xv7Ve9DHoILClrlsbuafpeLymYPdUFkZ5Ak0V+a9O7KZPSsYKvPfGT1I0x5bwpfH/gaAG83UaV1/JHjLLt5mc3jVudVk7Uxi5I0y5m+JypOsDzkbDYMGkOwrT/b5ivh66DuZx9xmtGvHySH/sH1AX5w7G3F+4V4hxDlJzqWQ8WH7GwNvz/wO19d8BWVOcLfoLa4luV3LOenG35yeTW/TKOxkff3vM9Lf7zEyRyx3qTOz+9FSL7LJW07HUhpniZmZbVWzNll7XxYYTtZ1R5DI0TSiRZBP3uVfs6MNS2xO09UJ46MHKnJ8dB5iPt0kP0kqG7JtG9hwjv2k4ZUIAf9TuRXUtlg3V/l+JrjPBf2HAe+0lbasS3nvXUel35nWR7YbDaTVysq/cLcXVDpByKB3cexhCPNMBlg1XTYeJFz86CKA5CzTFXSlSIizxDzHBeosMh2OTYLoVTiE+aDyWgCG/PTjc9uZO/He+0eSw761bqL+1CLSj+dTtcb9PszYTab7d/gmy4X0kEqaBv0c8WgUcug3/Jjy2kyNZESlkJymEaGo71YJu1/sPYckuNEJp/hhAj67cnfQ5Oxye7uQf2CuPDzC/EMEAEUY1PnTjrloJ+uQgT9ev38LGA2w4YLbfpHueSc1ZmiskAlctCsyvuAzQoro9FIeno6RqNj91xtcS1Hfjzi0L6nUlQE1V5iMj8orKP0XS92MJtg931QvNXy++4B4O7fvG3nLHqMjBoJQKXPnm5f6ZefL/4s3t6tkqSnFdXHhbdaibIK9B5B5mfwdSDkrbD4tlzpd6q8Z3Q0uLtDY2NroAJgzA1juGnXTXZ9dp3pN1c/vJpvF3/b4XVZ2jPUIwad2b1l4cBpMj+BI69gc4bmaioOwI7b4ZC64Fe3ZepXwtvPRUiSxI07b2TOs3MAGNxsa3b8uIrFTRu4eXbM6jKbzSz8ciHj3xnfUnGqFoeCfuFTheWCd/PCUOAguLAIUu5uPugqyPpK23vn5FLYeDGkv2N/W5cY3XQejcZGDhYdZFPWJoePkdBsX3No6RG2vmplfGGF7/7yHX+8/IfNPnNP/h42Z2+mokFIGb5z/jscuu0Qnume5O+2LSka0CeAB6seZPaTsy2+f7hAKJl4N/YjNNTGZxk+GeIv6fXzczEJCZBX1Z+deediVqnYMyR8CAAHi+wnj4y+cTQLP1rYMq/2CfPh0h8u5aIvL9Ksasoex0qPYTQb8ffwpyxLlDgqnl+7+UD/G6HvQpe1r6cTHCwCLGazioQYvwTwHwDGRofPKwf9DhQ5H6yxF/Rzdo5+KrIkqWaVfnov6H99q3duT0MO9pnNTt0TbYmIgL3x1/GUOZAP93xofbshEUQOi8TDv2uSS0vrSmkwCT/DSG+N5b2qMyH1cahxbCypKTo3GPEUjP0PSE5U0yUtgYsrIHyKdm1ri8moqbdfXVMdJ5q2U+q3iaoqrNueqUTSSVy/9XrOev4si+8bm4xseGIDe97bY/dYctCvUtcs7xnk/AK00WgkI8MxSwVH6A36dQPsVvrpvcE9EEz2gzIyctlpbVMtJXUahs2b0TLo98PhHwBYmLzQ+YN1F8wm2HSFWOjrTvS9ACa8Q2xCMHo9mIr7E+ARSL2hXnUm2J4P9/D64NepKdLGYDn1s9QOEidtKasro7y+HICGfDGz7w36WaAuB/JXQ4VaHREnyF8pMuBPfKF618FhYpWw2vug3QqrqirHtfi/v+p7vrrwK6oLOlbGbH99O99c9g1VecqO31vp5ySlO8XC6MnvbW+35yFYNa1TAn9y0K/Cew9lZZ3nJegIsp9fRtJ9nPXJHH499mvXNkhrQsbAuYeg3xX2t+0pBA6FflcKr1ULDAwdyO9X/c6em/e0S9TS6aBvc2KrJV8/JTjab8741wzO+U/H6uesU7w1NPPzm70GZq8Fve2qbJcSOk7IDo99Xdn2ZhOsmgGHX3ZlqzoVZxIFQ0LEvMBsdkDK7JQ2HPruEEWHOupOVTZUUtskZuCxAbEOHV9x0C9/Nay/AOryhUTv6BfAo40EVdtKttj5cM5+6PcXh9pkkYgZkHgt9Flo+X1DDfw6DjI/1+6cWmA2w4aLYNffFO+yM3cnQ14fwmXfXubwaeU5QdFPW1l1/yrFcmRNdU0c/OYgudvFw9Van3myUphVxvqL+y4xOJGUsBRu2X0L12+93u55JEnCzcuyPMmBXBH08zP0w9u6zRsMvl9UfPTiUmJjoc4cxuPrfiDfbaGqfQeHi3mNkqBf3JQ4Rlw5gmPLj1FXKha2k+YkEZzQeVWyh4rE3Ds5NIW8XBFoVCTvaWzo9fJTiFztp/i5OPY1mPqlqEByEDn4fLTkKA0G51bS5aCfNXlPcG6O3pYmY1NLoFKemzlFY5n46enUnIDlI+Hg05ocLiICPA1iEGRrDdA3wpdrN13LoEWuSXIuO17G7w/9Tu7OXIvvy3MOj6YIAn29tD151teQ+o9WxYauJv4SsVbrbFKPm6+mMpzt2LBIFCM1lGpyuKVHlnLGp+M5HCfGi1pKfMr2FGaTWVT9tUHvruevGX/lnNftq3zV1YEZE5WI4LAW8p4A1dW21Xq0pDfo1w2wW+k34V2YuVzVl9fTzZNoPxGRc4XEpzxhdjbo12Bo4Je0X4DTzM+vJgtOfA4llr0buoywCZB0HR5+wcKkGx3jgucyJ3EOBpOyybGMJEkYG4125cWUsvqR1ax7bJ3V91v8/PyiyM/2AVR4DvyZ8OkjZJ6GP9Z55wweJXS+g9QZ3gMMiRCTkiov25V+zjLl/iks/HAhXoEdB4z5e/NJX5GOd7CtlRaB2SwH/Zor/cJ7K/1UEzoOztkHA+xInRlrxaJmJ3j7DYkYgg49Te4lnCizPPHoLsh+WYU+a1mVsYrqxs4bNHYKbt4QmAJeWkWTugHBw2HyRxBuWa7J082TWQmziAuM65Ddb83XL3NdJkuvW0pdWZ0rWkzi7ESGXDKkw+vyBNzPKBqmWdBP5y6qp7qavovEPQj2Ew5qc6DqKNR10z7D1ASbFsMu+5JW1/xwDRHPR7SMydWw4ekNrLhXVLHK1X4HnEgEri+v56sLv2LzvztK/OZUiQ4wyCsIH3cfh44vz2HsKuvk/gJ5v4lnkRKChkCAdYl61XgEwsR3rfeFFQehJhOayrU7pxZIklhEq1SurjAwVPzdTlaepKbRsWTCoc0WimkJZ3HVH7eg91S2cObu7c5DNQ9x7hvn2txODvr1CVBfbXBiwwnSV6Z3WHiSSSvMBCBUl6D62L1oj17f+uw9flzdvmqCfgCVOZVsenaTXYlYVyHLkPb1HoTZDP7+2JaYlcn9Gb4Lh2w7CXy9qA/6aUCfgD4EewWTGJxIQU2BU8eS16Y19W+2wrHSYzQaG/Hz8CMhSIP+8Nhb8G2EdXWZnoJ3LBjrhGewBoSHg3+deGimFigbsJlN2ifFFuwrYOPTGylMLbT4ft/AvtwS9SEpOU/i49iQzzopd8OMXyHacjVYl1F9XPhGq6XiEBx9HRrLNW9SCwlXwoBbNAsqDo8UFgSVXvsxY9JU4hPE/fX60NfZ90nHwK5vuC/hg+xPYmtrwSQ1Mdf3Aa4ZeU1L4ldPotcNqxtgN6LtoLzDt5d8S5BXEEkhSQ7tbw2zWbtKvyZTEw9Ne4gNWRsYFzvO+cZ1F/z6wSVVXetNY4f+/SEjA672/ZIrr1S//4irRjDowkEW/VYc4eKvLqapzno1q4feg0uHXIqfhz9Zzd60vZV+VtB7OJUdqBqvcIf9jOTJcZX3QcrKzbhK2q3fGf2svnfe/85jzrNzrGZgt6WqCmqM5TR4iFXD5NBeSWKHCBpmf5sRTwtJlk6QOPJy86Kf3yAyqveTVrkH6L4DupwcMGOm0CwWlIZFKPhb9iQay0UGuWdYr4QZrQuPp1b6FaYWsue9PQy9dChJZ2k7zrPFjH4zeH7O82z9ZQD1aBD0MzaKzOV+V4B/fy2a6DxmE+x/AqrTYZJ12SN8+8LCHLEQ0x3RuUP5XkULAFWNVRTVFpFelm5321PJWJlByZESznzmTIYO1fH777DfCRshN083Fn2yiMC4wA7v5VSKoJ+jk26zudUT1W6l3+h/Cx8+v24QiMn6RkjgTv5UZHGDSKBZcAKkbthPzj+syn8o1CeUUO9QSupKOFJyhNHRo1WfMjpa9JdZWVEcK4W+KoYOOr0OzwBPq/J0DYaGFuWcGH8h8brs6DK2ntzKmLIxDPUdSv951vuvDU9s4MSGEzxU/ZDF92X7gihPG/dayXY49G+xUBk2Ucll9eIECQkQVPsLccffgOGvKu4H5HmNUlnFnK05ePh5MPeluQ631RnkoF+oKYV8xHdI0bDbIxgiZwt1hl5s0jboZzYr/Pvm/gr7/iEq/hx4BkmSRMG9BbjrnV+cl4N+tir9tKJ/SH/23byPvOo8bSRuA4dAn/MheKTzx+pKdG4wb5dm3r1eXhCtH8puRKWf2Wy2+fdedssyig8Vc83aazQ5v0z/uf257dBt+EZYjiiH+YQx1v0qsovRPuinc4OYrul3rdJUCT8PgZDRMGejun2PvQ1HXhL7umqMEHex+NGIASED8NB70EgNtR6ZFBcnanZsgMC4QBqrGmmsai+Lm7Y8jZgxMVbvu7bU1oLe7MnFEY9x1QJNm9dp9KhKv/Xr13PeeecRExODJEn88MMP7d43m8384x//IDo6Gm9vb84880zSThHPLi0t5YorriAgIICgoCCuu+66DqWV+/btY9q0aXh5edG3b1+ee+65Dm35+uuvSUlJwcvLi2HDhvHLL+ozY2UURbRPfCnkIs2WMwQtManvJAaFD8JD48X/sjLhL6PTOb/Y4+fhxwNTH+DnxT+j09Cctlvg5gueoV3divaU7IAfB0D6+/RvnpseO+b44eSAX9GhInK25TjVtJixMcRPsx7FGxoxlC8u+oInxr9NQ4PwOVLlx/JnoCpdLFTWOKgD1wUMCBmAm+SGUV9t059HkiT69u3r9ASgIruCzLWZHV73ClImGVFcDLWeGWCWiPGPIdCr46JkLzYo2wP5vwtNeHu4ebfOjDtB4nN4+EgA0mv2dJaVoEPk5kKDez4N5hr0kp6E4G6wIC1janL+szr6H/g+CipStWlTd+HwK7D2PKt/n41ZG3n494f5/lD7rHk5ueXUSr9hi4dxd/bdNgN+jvab+XvyeTb4Wba/sb3De6OiR/G3Sffiky1mPk77kOf9BqmPQXZH/8AuQ9JB2W4hRWxPvkyShK9Rd2XeTphpf46SFCzuIzn4oIYLPrmAOzPvROemY0hzcWhampgrOIK7jzvDrxhucUwoV/o5Ku1ZXi7aJUk25jClu1vnW90h4AdQsBqKNoPplD+qm0/XSuJaw4E5naz8cKDQ8TLRSc3F1FvWN5K1SdlYuCC1gMx1mRgaDFb7zLxqke3qqfckxDsEgF/SfuGJDU+w5//28P1Vtqudpj86nfPfOR9JZ7kvzqoWwfa+fjYSOCoPC99IV2bx99JCYiIEeRUQzW8iAUQho6NH8/tVv7Plui2Kth90wSCuWXcNAbEBjjbVKWR5T+8aUWmvOKE2cqZQovLtld2xR0ICeHgImcyTJxXuZDaI73y54xk0WgT8wL6nn1ZzdBBtHhY5jLOSNKq+ip0P077tns9JtcgBP0MdHHjKur9ffbGitaABIQORzHqqmipaxlbWMDYYMRlMNFZr4yko4+blRlhKGN4h1tWW5PtP06Bf9ndQeVTDA2qEewCMfAaG/Z/6+fSIx4WXd+gE17StLcYGyF3u9GHc9e4MChPPniqfVIo0FnfyCvLizuN3Mv728S2v1ZXW8dm5n/HzrT8rOobs6afl/SdJEn36aOxRaYMeFWWpqalhxIgR/Pe//7X4/nPPPcerr77Km2++ydatW/H19WXu3LnU17dWW11xxRUcOHCAlStXsmzZMtavX8+NN97Y8n5lZSVnnXUW8fHx7Ny5k+eff57HHnuMt956q2WbzZs3c/nll3Pdddexe/duFi5cyMKFC9nvQFqrJEnKtGtLtouBfpX6DFytkf2EIiLArbdW1DKFGzQ1OdUMvbdYINC5tQT90tLEM6WktsQhL5em2iY+OOMDvl38LSaD8qB0WxoqG2isUTaIkBc++/QR8iu9tCH7W9j3qCjv72yOfwq/DIcqdVFkd707CYFC1im73roUjk6nIzQ0FJ3O8ceWyWjig+kf8O3l39JU14TJYOKHq38gfaXyfrWoCAJrR3NbRTXrr1nvcFv+tBx8HlafCbUn7G8LYDLAlmtgy1UubRbAxISRABTp91BZ6fLTOUxODtR4iu9ZfFC85ok9DlObAz/0hZ13OneckDEw4FbwjtGmXd2FyoNQuM6qt8i6zHU8tfEplh5Z2u51udLv5EloW4TiHeJNQB/bi4TO9JtRI6OsZkAWF0NpqUj+SnK2yDDmHJi1ChKudvJAGjPhXZFV7WElsaOxQviOlnfz4LReWUJLYrDIrnWk0s8/xh+9uxiQRUUJaTiDAY66YD3F2Uo/WdozPNzKHKZ4G6yYCDvvcqyBrmLMa3DmelFdYzbDD/Fw6MWubpV1anPh+Meq5q1Dw4XUmFqP8bZMbE5uL377Bz6Y/gH15fYVV7a+upUPZ3xIY1Wj1T4zr0oE/aL9o1sWtr2bZYAbLmuw6H/alripcQxbbL0qP79BBNsTg2x0qAlXwiW1EDnL7jX14jwJCbD+xOXcsqYaos5UvJ+fhx+zEmYR5df9M1PNZnOr9GORiqBfd86M64a4ucGAAeL/iiU+o+cJu44+57msXUqxJ++pxRzdJZyu92na67D3Ych4T/xecQjS3xWBGID6fPgxAQ6/ZPMwsZGe+NaLNRh7z935/5vPtRuvxcNPuzlnY00jacvTMDRYtxhafXw1u6t+pcGtSLugS1O1WFvYeIlGB9SY5L9C1Cz1SkduvqIKrxMUkth+M6w9V5P17mGRYmxU5Z3aYmGiJTo30S8dX3OcLxZ8QdnxMs5/93zG3jJW0f61tVDnkUWZLo16gzYqfjqdjpCQEE2Opeh8nXYmDTj77LN54oknWLRoUYf3zGYzL7/8Mo888ggLFixg+PDhfPTRR+Tm5rZUBB46dIhff/2Vd955hwkTJjB16lRee+01vvjiC3KbI1mffvopjY2NvPfeewwZMoTLLruMv/71r7z4YuvE6pVXXmHevHn8/e9/Z9CgQTz++OOMHj2a//znP6qvyWw2U14uJsc2GXw/XFQCAQMUHzutJI0n1j/Ba1tfU90uW8gTZmerrH479hsf7f3IYe+Gbs3W62DjRV3dio4EDYFz9kLClfTrJwahlVVG+r2USNjzYS1+FWpw93Fn7ktzWfjBwpZOVS2739vNs0HPkr3ZeqVXUU0RRpOxReKsV9rTAil3w8zfRPZlZ2M2CkmCevW+ATeOuoWUk0+jr+hPkxWFV6PRyOHDh61KLylBp9cx5/k5LPhgAW5ebuTuzGXfJ/s4+Yfy+15O0ogO89FcOvlPwbB/wLg3wU+hfIPOTfhl1WYrqw50gvnJ85ha9ibJuY93kFLsLtTWimr72uagX/+QbiKHCIAkPCfaSj3tuBOaqtQdJuZsGPdf8IrQtnldzeiX4eIK8LQ8yJf7k1ODLpGR4OkpxomytLpMXVkdh5cetpp562i/GTUyiqvXXM3gCwd3eO/noz+zdPsuTFITSUmibU6h00PUbPDuZgukniGtMtmZX8COO9pnVResFrKkRSrld7qCE1/CsXdsbiJX+qWXOpZcWHK0hPVPrMfYaGip9nPU12/jMxt5Ke4lSo52lEJpqfRzMuhntUI1eDjEXw5J1zp0fJeh0wuvUwBDlRhvNVj2wOkWlO8VyTr5qxTv0lLpp1AW0RJJSRAWBoUxI0m+Y46ifUZdO4p5r8zDO9Tbap9Zb6gnPjCefkH9Wl7zchMB9dqRtRb9T2VMRpPNpEqz2czsxlcYmPMvhsbaGVe6eXeufP+fmH79oMnkRV6BB6eIQp02SJLEybtPkntPLlVZYjwZp6Rwb9fdsGae+vHdnxjVvn46t1Z/YQc5XHyYye9OZuxbyha3rSFXWlmT99Riji7z5PoneWHzCy2JFk6x8RJxn7p4/tjpJN8pEtP6Nxeu5C6HrddDcbMPsn9/CJ8OobYlHqOiWn397AX95MQuLTn26zE+O+czdr29y+o2j655lPebzqbEf612QT83XzjjJyHh3p0p2Q477xbjPXtkfQN1znl3qiL5ThjzqiZzddmmpNI7VXkltANkb84m7Zc0vIO9GbVkFImzla1F1dZCeuS/uTF1IP9a+y9N2mI0GjnqisxIK5w2dVrHjx8nPz+fM89szcQKDAxkwoQJbNmyhcsuu4wtW7YQFBTE2LGtD74zzzwTnU7H1q1bWbRoEVu2bGH69Ol4eLQOqOfOncuzzz5LWVkZwcHBbNmyhXvuuafd+efOndtBbrQtDQ0NNDQ0tPxe2VxGoNebMZvNFBWZiIyU0Ol0mEztJweSJKHzChcP0jYPU0kS25/6gNXpdEiSRFpJGo+ueZRhEcO4deytLdk3JpOpw/aWXtfr9ZjN5g6v5+WJ16OizBiN5nbbW2y7lWt6auNTrD+xnuyKbB6Y8oCia7L0uhbXZKntjlxTS9uHP43UVIXZaOy216TTmYmLk0hP1+FmEqlbu/N2E+PXWl1hre2nvj7kMjHZlfXA1V5TYL9A+p/dn9BBoRiNRovXNOndSWRVZHGj+1rM5kn06WNq+To4/DnZuKa2bewx955ZBxGzxYudfe/FLYb4K8Trzdsqvaa/TbudjS+aMRigpMRIeHjHz8NoNFJXV4fJZEKv1zt8TcmLWj34YsfHcsfxO3DzdGvZz97nVFgIZrNEaKjo/zrt3qsvAPdA0Ht1z3tP6TX5D0QKSFZ3703/EZPkDmZanoGuuKaU0BRmBw1ixzGJ7GwYPLj79RHZ2WA26zAECOnyxKDEduft0n7POxrTnK0iw9BohKqj6I++ihkwjWpfkdIl915X9+WSBzTvY+maEgKFjOCR4iMYDIZ2Mkl9++o4dkzi+HFjOy/lPR/sYcU9K7h82eUkzWtdLJbbLveb8jU7e031hnrmfz4fgDlSEQMHhrSMAx36nEq2YtR5QdDwdm3vVs9cScKc+TGU7sKU8gB4R4nXY8/HeOYW8O3X0i9113tP2vdPMJswJSyx+jklBIn7L6Msg8amRvTNfppKr+ngdwdZ8+gaokZFMXjwADZsgNRUMxddZFZ9TR7+HvhG+OIR6NHhvIuHLSYpOInxMeNb3lNz7+XlSZjNElFRlj8PSe+Fcfy74gU735sue+a6B2Cctxu8YrrvvRc6HtOUrzGHjO/QRmvXOihUVBrtL9yP0Ykx7IQJZpYVDSCvX3/c/c0t57J2TTHjY4gZH4PJZMJoNFJfX99h2+lx08m8K7NlG6Clyr7OUGfz3juy9AhLr17Kgo8WkLIgpcM1NTSAbv8VDDRIjBlubPdeu88j9xfwiYOgoT1jvNfd+nKV1+TnpyMszIRvUyoluw7iPeUSxde0NWcrS48sZXD4YK4acVW3uSZrn1OwezRFBWbATGysmF/b/JwaK6A2B5Pk7dJxuTPX1N3uveRkE2azxKFDYv6o6JqM9Uhp/wG9L1LybaqvKcgziC0ntyAhUVVfhY+7j0PXVF1txmwGLy9xb5z6Odmboyv9nMxmM89vfp6KhgrOSjqLCJ+IDttbulaL12Q2o5ck8Xqb+aO1a+1R9x46zP2upvnCIGY++oCBmAKGYDYaAXeYuUq00WTEVLgBs3tgy1hbbnt4uJHw8rOJDgpiaPjQ5vVB69dUkl7Cqr+vIuWCFIZebv05pPSaokZHMeupWaQsTLF678nWL14NffH0NGIyafQ5RUzvdn1Eu9drc9BtvhKpqRzj8GdsxwBqc9BtuhRizkM644fOuaaAYeIHy+t+ar5PQ8LFOnKVdypZx8wYDKaWYkUtv0/THprG8KuH4x/t327dz97nVFMjUechFKriAuM06SMA6uo6zw/+tAn65TenbkaekroZGRnZ8l5+fj4REe0fHm5uboSEhLTbJiEhocMx5PeCg4PJz8+3eR5LPP300/zrXx0jwzpdGVVVElu25DJ2rC9xcXGcPHmS0tLSlm2ioqKIiori5OENULSe8sCzMUvu9O3bl9DQUNLS0tpJmCYmJhIQEIChWJQPZpRmsG/fPlJSUvDw8CA1tb0U0bBhw2hsbOTIkSMtr+n1eoYNG0ZVVRUZGa3+Hl5eXuTlpdDU1ERDQw6pqcLrxN/fn6SkJAoLC9v9HUJCQixeU513HetPrEcn6RjrPralTfau6eDBg+2+OMnJyZpcU0pKCmVlZWRnt1aaqb0m+XPKzMykqqp58S01tftdk/E3KmslMpmEt3cYVVX+BDcMAvazIW0DcfWtKX7tr6k1m8/aNfnX+pP2dRoxS2Ladbb2rqmhXwOD/m8QR08ctXhNRrORE+UnMJgNlGSGU1VVSVNTAamptU5+Tvavqcfce24nyMkvp9TQ2jf1pGvy8KihrMzI1q05xMc3dviczGYzpaWllJeXEx4e7vQ1RbtHk7MrB92w5urUPGXXlJpaz7a+d1NcomfIgVuZNnyay++9E9veJ+H4LZyMfoCS4AvFNdUc5EBWAyZda9pb1/d7tq/JsyGD2OQzCAgKVXfvNZk5cqT1dVdek04XQmNjBNnZXt2yj9i505eqqgg8PBoJ9wkn0iOy3fG7ot/zaDhBcpSBxohz2l+TTsewMa9RFb6IjDbb2/ucajf/lYbKk2THPNZl1+Ts53TqNRUWFpKfl4NfzXbMkjuefed0uCaTwYRO0lFUW8T2Q9vxNrZmeIeFDeDYMV+2bcvH37/V8CB2ZiznvH4O5V7l7dovX9OBAwcoLS3lwIEDSJKk+Jqyl2XTWNDIwhcXUl5Z3nJNWdWiBNbN7IO5zhd393RSU2sc/5yOvIR0chn7B/6OSefdPT4nC31EdsJrVHun0nisCCgiXr+L4H7TyawKpyo3H8hvvaZueO9Vhj2CUedPQ2qq1c8pPCIcPw8/qhur+WnLTyQFJKm6JrfRbkz+32Rip8eiLzRRVVXN1q1m9uzJRK9Xd00BswO48bYbKSkpIT21tfLQ39+fqUlT6e/Rn/z8fFKbpVXV3Hu7d4fT1BROVJRHu2uKy/kHvhGD8Bz7TLe696xeU6GRqqpWSfRuee95TiU/PR8oVXRNQQ1B3Jx8M0kBSaQ6MX8KCUmjqiqaFSuMDAvbx8Szx6Hz0Vm8poLjBeSV5SHpxQqTnPRbXl5OThudKUufU0WxmAP7/dePF254gTOXn9mSrNH23jtZcJKAwQEU1hUSWRbZ4ZoOHfKmvj6RqCh3yssPUlpq4XPat5fhhy+m2mcMGfH/7fbjPSWfU7f6Plm5Jn//Ei6MeJS+2T+TujeBhP6DFV3Tjxk/8lzqc8wfMJ9Lki/pVtdk6XPKzw+gqqoKf38DJ05k2f+cRr7OkcMHodnapjteU3e79/z9c6iqCuDAAdi27QQJCRH2r6mulsFpL2FwC8IQc5XqawrQBxDkEUR5YznL/ljG0NChDl1TaWkD1dUNZGdnU1Nj6PA5yXP0oqIiYmJiHP6ciuuLqWioQCfpSAxKdP5zmvoVpcXFZLc5zul7782nMD+f/PxT5oQhRnSrZ1IaOJ+s2MfbXZPRmEtQ5gWE15xHVM1JysrKbF5TZn6mkOMMMGAaYqLws0JKj5aSdHcSHkGtBTNqril8UTgBfQIoKSnpcE3xCfEt6g7G0iByctI4edLHqc/Jqz4NyWwkesg53a6P6HBNY9cQFWgiM+skVZUVBFcspyJgFrHxA9tdk2RqJDDmacLjR+ELnXpNAT56EhMSKCypdvj75NPgw2OT/o/fPppMSUkNGzYcJzhYtF/r71NmcSYUq/ucTp7sQ3XccUDYqmjRR/j6+v4/e2cdHtW19eH3zEzc3YUkBHcobi1QqFDqt+7url/ltqVyq/e299a9VKhQQdtSpBQo7sFC3N1tZr4/ds4kISNnLAnteZ8nT8iZI3vIyZ6z12+t36KqqoqeQjI60sSrDyBJEt999x0LFiwARJ+9yZMnU1BQQEynNOgLLrgASZL48ssvWbhwIR999FGXmwAgMjKSJ598kptuuok5c+bQr18/3nrrLdPr+/fvZ8iQIezfv59Bgwbh6enJRx99xEUXXWTa57///S9PPvkkxcXmy2rNVfolJCRw220VZGYGc++9BqZNs66IG3Y+imb/M+jnbIGQUTYzTOqa6gh4PgCAkntKCPcLB5xX+e+7T8uhQ0Yeeshg6p0g729u7AYM/HTwJ9LD0hkQJqpsHlvzGAt/X8ipqaey9KKlXfY/UTK2LKr8zXVd7Bj63Hv6PhajfzqGU9awYgX8738amka/yK/a+zlv0Hl8ce4XNsduafvym5ez/d3tXLXhKmLHdVQMOvuesquySX09FU+tJ3O31qNv0/L22waTPdMJn7Hlinvv1+kYK7ZjWFBsuv96/D0VrUKq2otmyH12vSckuOreg+wrPsC/b57P+PGS2Uq/ffv2MXToUDw8PJx6T0ajkVfjXkXjoeHmjJvx8OlodG7r93Tfgy284uuPUdKTfXs2iSGWM35cdu/VZqFZfzaG8R+KDG99A3wTCrFnYJjytdi5cCWatmpIOBcDXS04+kT2raENzfdxEDQYafY6+++90s1oMv6FYcSz4J/qtvf0+bIcnv92OSnxQXz7xD+sv6dOY+ypOeLLLyUWLZKYNcvInXdqaNO3IdFREdYb855m3XwoXA6n78Pgn275PVXugqChoNFa/T0Zf54ODTkYzjjSa+/J2naH7z2DHs03gRAxDeOM5Wbf09D/DSWjPIOfLvqJualzTdu//17igw80nHSSgYcf7tjf1ntqbW1l3759DBkyBK1Wq/g9fX7652SvzebhhocxGjsqZVYfW82cz+bg3zSQ6Xv28+67BiIixDkU/56aSsA7Umyv2okh63OMI55T/J56/TO3pRrNj8lI/W9AP/QZYbto6b2eKO+pffu5i8+loaWBZ05+hlHRoxx+T0YjXHSRyI598UUD/fv3nSqehx+W2LdP4v77JSZPbt/eVo9m1Tjx+TTtuz7/ezqh5r3WetFLvAffU2urgcsvl2gpqWL0xjdJnp7IJcsvMTv2L87+gryNedx65FY8fMSzpbzul89n6b2+seUN7lh5B7fsuoWRhSO5Ys0VpjYH9rynJ97awaoNpZwxbgQP39Y1qdj0+2hrQcr9GqNnMMTMVe+9HnpPn3xiYO/q9UweV8np185B4+Gj6D39lvUbsz+dTWpIKodvO9yn3lPn7U+ufZKdxTuZqL2FDR/PYfhwI089Zb06uy/+nk6Ue+/666GoSOKJJwyMHq3wPVXvB/9Uxffe8dtnfDiDdTnr+GD+B1w2/DK735Ner+Wcc8S2RYsM+PmZr/SztkZX+ntak7WGWZ/Osvh3o957Dr6njP9gjJgKwcO6bC8s1HPddRp0Ovj6awMaje331FjRiE+oiDO9PfJtkqYlceq/T7X7PTVWNOLh54HOS2fxPRXUFZD4aiIaowdztzby0ouQnu7c70nz+zmQ/yOclY3kl9C3fk9W3pOU/TmaTZdhGPokDH20b9x7ZX+g+W020qh/Yeh/i9N/TzffrCEvz8gTTxgYNaqX3pOZ7VdfLfF5Yjitukr23rSXQeGDnJ4jDAYDf/zxB9OmTaO6uprAwEDcyV+m0i+6vcFccXFxF9GvuLiYkSNHmvYpKena/6CtrY2KigrT8dHR0d2EO/lnW/vIr5vDy8sLLzONT8LDNRw7JlFZqUVeWxy/yJDRJF8EoSPRBqaBtiPIoNVqze7v7+1PlF8UxfXF5NbmEuEfYXV/c9slSeq2vaBAbI+P13L8IZ3H3qpv5dNdn/L0+qfJrMzkh3/8wODIwegNej7a9REA14y6xux17RmjK97T8WN3Zrt2zWzAAKdudniM9m636z1N/wnJIxitVsuAASBJoM8fCYmwu2S3U7+PU547hdHXjyZ2TKyi/QF2fbyLPZ/tYd5/5hGWHmb2PWXXiJLqOL8kDHod3t4QE6M1lX5bfK9Wtrvz94HRiLb2AAQPhZYqWH8ODH8GKWIi2tYKyPwQ0m8xBUJccu8NexypLhOtV3fD/R679459CDlfwYCb0ejMd/s2N/YWfQufBg7DEKjnWHkuk7Tx3a6p0WhITU1Fp9O5ZOwXfHsBf/7nTzRour1m7e/pWHUmRj89froAEoITLL4na9vtHntAMpy2o0PKa2uF4U9BYHrHMYdfFz2mki5EK2mgLhPyf4Kki0EbLt6TRgKp65hcNu/Zek/GJhh4J3hHK9v/+O1t1ZD3Hdq4MyBICEvumMuLvTayN/kWCpvGodVeYt8Ye2AuLywUc3Z8vJj8dFrzj3E9Ou+NfwcKV0DQIMztLUkS2pLVsOY0SLsBxr1udYzS7LVgNIj7uBO9Mpdb2O7QvafRwPj3wS8JqX2/4/cfGTOSjPIM9pTs4fT0003b09u11MxMTbfnLgBDiwGNTtOt54aHhwepqal4eHiYKlCUvKdzPjuH+uJ6JEnqclx+nci49WlOJCxMIiqq6+ewtf8DrVYLRb+Kxu8n/wKRUyB0NJrQ0Zb3d3K7W573fEJhxk+Q+RHa5nzw79dt/z537wG0VIrvniEW9//uwu/MnkPpGI1GIyV7S/D082TIkBC2bIGDB7WmXkaWznP8e/rtsd+IHBbJkPO79klrM7SxJGMJcQFxjIkdg8aBOaKkRMyhUVGdtmsD4fT90Fql+L3a2u72tcaJMO9tukI8E17Q0EUcd/d78vTUMn48rP41BN/Zk5j+UKrFscePj8c3zBdvf2/T2FJSUtDpdF3mviuXXMmh8kM8c/IzzOwn+mb7eopn+Jzzc3j9otcdHvu3uW+xP/1dBoc8jlb7hPn9PbwgpfsziXrvufc9paZq+Oqr6bQchvmetveXtw+LEsH1zMpMmtqa8PHo3putL/ye1mSvYX3OekL8L0SSJFJSpG7PGF3OYzSK3mHRs9Emd02Mg77xnly93ZXvadAgKC6Gw4e1yN2GbI4ldJjDYwdxL67LWcf+sv2mfex5TzU1tD8LQkCA+diLq9boRypFst/A8IEWx2jpPF32Nxpgwz8g4Vw0SReaPcff6t4beJvZ7ZGRIpbbYmhk/ZEMRienEOQdZHWM/hEdcaZLV12KX6QfGq2GtqY2fn34V6Y+MhXfMF+b72ndk+vY/clubt5/M4FxgWbHnlOdA4BPaxwaSUtAADbj5Tb/f4c9DlEzwS9B2f4KtvfIvJf8D2jMQ9P/RtN/glarhfKtoq+eX6JDY7e0XdF7Ch0FUTPAN8Elf08JCZCfL1FY2DE/OjJ2S9sd/T3VNNfSqhNrqMSgRJfMERqNppu7pDsxP+ITkH79+hEdHc2vv/5q2lZTU8PmzZuZOHEiABMnTqSqqopt27aZ9lm9ejUGg4Hx48eb9lm3bh2tra2mfX7++WcGDBhASEiIaZ/O15H3ka9jD1v+/Bdb1gzitqvjGDdoEE8/9ZRlm9DgIZB4HngGKz6/3HA8qyrL7rGZo7a2o5mvFY2THw/+yMA3BnL1D1eTWZnJ+LjxzEmdA8DPmT+TX5tPqE8o8wfMd8m4+gxGo8iiCTEfxOoThI2DwP4AJCWBTgceFSMAOFx+mPqWeodP7RPiY1Hws0RVdhXZ67JNWUPmyKwU5eURHimAaDJ+fKCxT7FvISwfBUW/iCa8ZRuhard4LfsL2Hk/5Cx27TWjZ3U0dO4thv4fzNsJGm+7DvPUehKhEffkvtL9ZveRJInAwMAuQRhnSJycyHlfnIenv6ftndsxGiG/WXRgTw8d6LKxWLzY5uuh8Ofur3mFwpAHIeGcjm2jXoCp33SIerlLYNsdHfed0QjfxULxWveN2Ro6Pxj6KKRd69jx0afAWdmQepVrx3Ucpw2bBECF5w5Kqxrcei1HkN3G4uJ6dxxU7YWCleLfvrGQerX1/SMmwZCHoN9lHdtyvwVDq/n9pb/M42lXki8S/xcWGBk1EoD9x82DqaniM6+sDI53A9mzaA/PBz9P1pqsbudzdN70DfMlYnBEt+3yAty7JZFBgxz4HPYMgcCBJ/7vN3IaTHjPrODXJynfAl+HwuE33XqZ0v2lvDn8TTa8sIEh7Xrdvn32naOtuY11T63jwDcHur1WWFvI2V+ezeT3Jzs0vpYWKC8X/+62htFowSvMofOqWCB0LCSeD3rl64qS+hJ+OvQTq4+tdurSEycCksS+iBnETxDBvcbKRoyGrtncUx6cwvx3O9ailubM7YXb2Zi3kTZDm2nb/AHz2XLdFl6b+5rVsSy7dRk7P9xp9rWqKihuERa2EwakWj7JiWnMdMIjx+Sys0Hf0mR9505E+kUS6hOKESMHyw/aPqCXOFAm5tnyDNFPc7St8EVDHmR9AqXr3DyyvyZyAtfhw3YeWHMIDv7boXlgaORQAPaV2vlh3I4c9/P1tfzM56o1ekaZWGPLzmAOU3tEJL5W7nTuPH8lDG1QndFlk1YLERHwx8ApnPzlaNZkrbHrlAExAWi04nl+3+J9bHplE7888IuiY6NGRJE6J5WA2ACL++TW5ALg1SwELV9fi7sqJ3S0SEI+0dDoROynsw7QWgu/zYFlw6HRcpsxt+HhDzNXQMICp0+VV5NHXsjnlASuoJPTZq9jNEKlXqx9g71CCPCyfL/agzxn9hQn1Kq7rq6OnTt3snPnTgCOHTvGzp07ycnJQZIk7rzzTp5++ml++OEH9uzZw+WXX05sbKzJAnTQoEHMnTuX6667jj///JMNGzZw66238o9//IPYWCFUXHzxxXh6enLNNdewb98+vvzyS1577TXuvvtu0zjuuOMOVqxYwUsvvURGRgZPPPEEW7du5dZbb7Xr/WiBplUv8VJdBh9VFXBnRgbrFy4kKT6eV15+2fKBrTWKr+Fq0a+wUHwPDQUzhYvUNNdw9fdXM/+L+WRWZhLhG8G/Zv+LXy//FS+dOODzvZ8DcMmwS0zb/jJIEox/G076X2+PxDIGPTQLH2IPDyGgebVFEuYVjREje0r2OHl6A7s/221xgXs80/9vOg9WP4hvuOVP8mNVwkc52CBEv4QEp4bofuIXQNyZED4RYmbD6Qc6BLmUK2DyF5B0kchE+/MGOPK249eqOQgl610ybKcJHgohI7pkdCsl0UdECA9XmV+U6PV69uzZ061EvieprIRqT7FAHho1yM0X2yEE4sz3le0fPBTiOiqDSLsOpn0PQe2R17Z68dlR7FwgzSGMRrHYcAaNzpSV504GRifh2xaLUdPG8t1b3H49ezAaRaV9le8WLtqYxhVLruidgRhaYe2ZsPESaK1TdozOD4b/E8JFghX5S2H9ubDr4a77GY2Q9yNYmAf+Elj5e7hy5JVk3p7JRws+6rLdx6dD6D1ypOsx4YPC6XdKP3Re3as+HZk3DW0GyjLKaKlv6faaKeu2JYFBjkyBoaNh3g6rwqeKGwhIh5QrIXi4ot1L6ku6iBtKiRgcwbTHpjHyypFdRD974pRaDy23H72dU549pdtrcm+XmICYblV+SigtFWPx9gbTWnvrHbD1NtA3Wz1WxQEG3gGTPgUP5YGN7zO+58zPz+SFDS84delRo8DTU1R2ZmVBbUEt7014j83/7nBhMdfhxNKcWVBbAIh7TybSL5KxsWMxbjOyfuF6Wuq6z5ktdS1seWMLx349Znacu3dDg5cQ/YbFWRH9dj0MS5KgId/yPiouJzoavL2N/HvOANpWnqz4OEmSGBguSpwPlvVN0a+soYyyBtHcSF8yAG9vGDrUxkF+CXBOqUjyVLGbziKyXRx5WyRyOiBiDYkQH8Z7S/bafSx0iH5+5g18ANet0TPKhSgl/+04TGA6nF0AA++2ve/fhTXzYNVE0Hf9nIqOBv+mwYDj9wjAiMtGcPO+mznznTMV7T/mujGcv/h8q0Jx5zUHWL8HbVKfCwUrnDhBH8HQKhIASjeCRwBMWgSz1oKPlWqcE4Clh5byZunFHIt6rU+Jfo2N0Ogp7sPEoEQbeytHtkTuKU4o0W/r1q2MGjWKUe0mr3fffTejRo3iscceA+D+++/ntttu4/rrr2fcuHHU1dWxYsUKvL07qk4+++wzBg4cyCmnnMJpp53GlClTePvtjoB7UFAQq1at4tixY4wZM4Z77rmHxx57jOuv76igmTRpEosWLeLtt99mxIgRfP311yxZsoShNp+UurIB+Lm1iUuAU4BLgJVNTazT63n24YfNC3+broFvIhQvTlNChEhyuNzelCLzyEWInRxUTTS3NTPm7TF8sPMDJCTumXgPx+44xr2T7sXPU8zSRqORrKosNJKGa0c7WPGh4hybroJvwqCtEYDISLF5WvAl3DT2JgK9nMs6kDQSv/3fb6x9cq3ZBbU5tJ7WRSK50i9QL+7n9qLbvoXRCHXti/rgITDtWxHoBvBP7kiP8wiEpAtB6ynE16JfRVWWsd1jWrbgUsqO++GXaVDjmr9xp2lrgIYCuw9LCRAPnNn15iv9gF4V/EBUCNR5iwXJoAgnFyS2CB0N84/C+HcdO94jAOLng097fxgPfzgjA4Y94bIhKqZ6H3wbCZkfO3ceowGyv4Ij77hmXGaQJIkkrRAk1hz9w23XcYSaGrH4rvc5RHbtUdNiqMfReMCED4Wo7NHdTlgRMXNE0GjQfV23t9XDuvlw4F9OD7NPUrASFgdAztdmX47yj6JfSD+zi+D+ohi6m+gXMyqGS5ZdQtK0JLPntHferMqu4o1Bb7DmiTXdXutYgCd2sWy0yb6FUNleddyny/T/ongGwYQPuiaGWGDkmyOJejHKoeCPJEnMfHIm8RPiSUsTWdk1NXDokB3n0EiEpIQQ0q/7g15+jRA8YgPsc5SQkdcw0dHtt6FBDxVboWI7aJRX/au4D2erUmS8vDqqljZuBK8gLzQeHWEPg97A/4b+j7VPdXc/OH7ObG5rprxRlIiau/cO/niQ1Y+spqm6eyWYp78nD1Y/yOx/zTY7zq07Wmj0FBEuec1u/g2Ft391r8BWcR+SBP36SWwrnEep4SS7MhjSw0RZ16FyOybAHkSuqgrXJaEz+JmEcpt4BoGPmUCQik2S2h/TSkqgwR4zkdRrYcZyCLI/22pI5BASAhMYGjnUoWSeuvbcPn8bj/uuWKPLArnToh+Iiihvdb40kXQRDLgdDF1jyFFRENAoPnf3ljou+oFI/FJS7ak0Prhg4ALemPUxCWXXoNWKYgWH2fWwED4rttnety9Tnw3b74aa9qrN2Lki4b63MLTC2rNgkw3HHxvIltg1Prv7lOjX0AC+Lf0YUPQYV41yrdtUT8Y1T6iefjNmzLA6SUiSxD//+U/++c9/WtwnNDSURYsWWb3O8OHDWb/eeuXM+eefz/nnn299wDYYb2X7j83NTLv/fi66+OKuvQLDxok/rtYa0Nr+ILthzA1cMuwS0kLTnBqrjFzpZ07089J5cdnwy/hw54d8uOBDpiVN67aPJEl8evanFNcXMzxKWcbxCcWWW4V91YinensklomaCVpvMLYCPibR70yvF7nKdjzIJpIkce6icwlKCrL5wZ+9LpviPcUMu2iYVXvPqYlT0Rv1hGWOIZdOGdp9iU1XQcFSYW/pq9B7zzscZq8Hj2Bhd2bQi0zegP4w+3fQWf4/MTH6JWF92G7Z2qsY9PBNOETOgJnL7Dp0UPgQKIL81r5b4VNaCnXeotJvUIQbK/2MBnE/uHqxIvu9Gw2w61HodykEDXbtNczRUgX+Kcr/LiwiwZ7HoKUaUq9xm0XgsKDJHKj5mm0lG9xyfkcpkLX0EKH6pIW45nPdIaKmO3e8xkNU/skU/So+l0LHwISPwM+8gHXC45cgKsA97LcHSUuD337rLvq5Gg9fD6Y8NIWk6d1/B1em30Pe2lOJaJ5IipX4dBdqDsOeJ6B0A8xY6tKxqriecN9wALYWbGVk9EiHzmE0GqnOrGDs2DDWrYNNm2CAQreuxopGWhtb8Y/2N9lGyciVfnEBjn2WyNUVpjWMRiuetVoqVDHaHbTVw477RNuD/jcpOmRwhHgmyavJo6qpimDvYIcvP2GCuPc2bYKLL/bkhh03mPqe1hbUImklDK0Gm+cpqhNqsZfWixDvDjE6ryaPRXsW4TvLl+uvvB6/CPNlCF6BXngFdne2MRrh971ZEG/ER+tHlF+U5UEMukd8qfQ4/frBu8tepbIfXGnHNCFbFMp9yvoaB0rFesavUQgsJ52k4KCcb0SAOaAXnz9PYAICICxMJJFmZ6PcMSFooPhygFCfUHLucjxJUK70syX6uYLdN+3mUPkh5+w9c74WVoepV4POFX6QfxEstGGIjoaAze3JNiXOx2DKMspY9/Q6pv3fNMIHhHd7vbGykfcnvU/KnBTmvTbP6rnSw9LxSUxneS34Bjj5mDZiIYSMFOvME5mANBj7ejfxttfQeEBzmdMxGTnhq9mzgNK6CmprQwlwjZOmUzQ2gn/TQEZXP8mdE3p7NI5zQlX6/Z0YD8zw9OS9d4+r8uh/I0z6WHEwOCk4iSGRQ1xmo1laKr7LQlFWVVaXDLaHpjzErht3mRX8ZBKCEhgbO9bi6ycsRgPk/whlfas6pBupVwkL0na7Hfl3WVLiukvET4gnIMb2TL3rk10sv3U5rQ0W+jq1c9O4m/jyvC+JbJgJ9FHRL/4siDnVLhsjQGRLyuJeW504z5hXlAl+ID78B9xu3zXdhUYL/W+GWOsPceYYFi0CPWXSfsUZYD1NVRXoNWL145IsRHM0FsFPA0W/M3dRtUdUUu3toeSEyCkwd6sQp51BkmDCx8Ie0I09wSYliH5Rhxs3YjDaDgj2FBXClZlmfxFASg21YgXmDvRN8PM01/cjba2B9efBxitB0kHK5c6Lin2VoMFw8s9WK66+3PslFyy+gK/2fdVle1p7jM1cL5i8zXksOn0ROb87X/0ZEBPAKQtPIe3U7kG9qLpTSS2+h1EJA5Rn3Ab2h1PWwniFVsUq7uHYZ/DLDGgstrqb/Hy+tWCrw5daevNS3hj8BiPTRInA5s02DujEzg938kr8K+Rtyuv2w0jZewABAABJREFUmlzp56jot6fdvX7IEDoqdiRJ7eXnLrQ+kPmBXZZaQd5BJAQKK6/je5vay0kngUYDx45BcTEmwa+lvoWyjDJu2n0T0x+3/VlTWCcyXmMCYrokM+ZU5/DALw/w6rFXiRkVY9a1pHB7IWUZZWafawsKIL9BWHumhqa4t0+0isPIlozHzDu0WuSaUdeQfWc2H5z1gesH5QLkfn6aikFIEoy1FZppqYTfz4Pdj7l/cH9hkpPFd7stPkH09mupduVwbCJX+jllragQf09/RseMNjmEOcTBf8PuR4C+GUvodQxtIkm7nc6VfhllGbTqrcfkbFFfUs+ez/ZwZLn5ZIfm6mY8AzyJGaWsWrhzT0mn8Ev46yTO9L9RfPUVZq2Fad85dYpAr0BTW7Janz3kdV8C9ApyRbZL+kn2Iqro14e5vLGRJTaqEnuaqirxPSREZD6OfXss5y8+n6Y2YWniofVwWYPLEw5JA2cdgykuDoi6mYh2/bi0FBpaG9ict9klge6mqiaO/nzU6j6zX5jNpSsvJTBemVBW097Oss+Ifo3FHT2tEs5u713ixP3vGQSTPoFIy6I5ICw0dz8mmlT3NYFs9Isw4Da7DxuZkI5k1NKiqTb1TumMRqNhwIABaDS997FVUwMz9h3gxdAa+oe6qbKyJkNUxkn290VUTMgImLlSVFSdaISf5Hbf+ukDRqLR+9BIJccq7YzyuJHq9nV+ra690s9FFfyKqdojvlzdU8gjEKZ8CdO/F70b/+ZsL9zO4v2L+e3Yb122p6QIfaKiokMANmGEoz8fpfRAaZfNrp43M9rdZOzu5xcxscNqWKV3aC4V/YAarf/9yqLflgLHe5qmn5HOuJvHMWyoAZ0OcnMhX+G0ET8hntHXjSYkxYy9p1zpF2i/6KfXi/6CAMOGIfr4bbxCPE+puAdJA2dlw7Qldh02JNK5HlQyAQEdPco2berY/u3F37Lo9EWUHSzrVk1qbs409fPz7xqk9NaJ9iEtTS00VTdhaOu+dlp17yreHvs2RkP3Z/WdO6G+vZ9fWpiVJJ6mEth2J5Sss7yPittIbf/VJDe/hn7pWOG4pIAIvwgSgxId6j/aE7QZ2vDW+OHfOIj0dAgOtnGAxkv0pO9LweYTENniMyvLzgNzFsNPAyBvicPXluN19qCkp19fWKObmLkcZqzoaLGi0kH+UvguBopXmzZFRwvLfp3Bn1ZDK4crnGsXkzg1kZv33cwEC2VRwcnBXPPHNYy4wrYl5Uc7P+KXrBXopSbHRZejH4jWIH0tXvZXwkVr92GRssXnnj5j8dnQAFW+W2nxP+K0IN4ZjUZD//4959LWB2ZmFUtEAdXVZrJ59j4trAQV8tbWt7j6+6tdUrItDycoCL7e/zXljeU0tTVR1VTl9Ln/Ekga8Art7VFYp3wrbLgESkVFolzpV1SiJ+JfEUx4b4JLAt3fX/09n839jMaKRov7+IT4kDrHerVKXUsdeTV5GIyGviX66Vtg3Vnw8yT3BI2qD8Bv86Cx0MyLksie3vWwsE/6CxAV5sXg3FcYfXQxXpj/BXsqajbhPuT5LzI4AK3GTaJc1AyYnwlx891zfpnok0VfSeiS8edyjr4Hf1wODS5M2Wqtg2OfQGut687ZidRkDyYeWsOpO6qI8+3hajoryPNfpdRLol/YOFiQDWk3uP7cMXNEFVzhKvjC272Vrr1N1hdCbLDwdydbKu4q3tVlu7c3JIgCGI4el08TOy6WB6seZMx13W1r7J03/3jpDz6b1/2zu6S+hGWZ31Pjs1t5P7/cJWKxrW+xawwqbiD9VjivUvSLtcK42HEA7Cne41CAECD99HTm/Xse0f0DhcBGV9HFGgmTEjjz7TPNukXI4osjlX6ZmcKmx88PkpMMUH8Mao+IajQV9+Edabcn15AIIfq5Yt06oT3uuHFjx7apj05l1nOzCE0zv147fs7UG/QkBSWZstBlfNodOQatHsTzwc9Tsre7Zcr4O8Yz67lZ3cRFEKJfZPVp3BL7ETeOsSKkVB+Ag6+J3pMqPU5amvjs9ZSqaK4pET2V/gL8e96/edBYQ0L5VcqsPXW+oie9rcRUFavIlX52i36RMyDtRggaYvc1lx9eTtSLUZz22Wl2H6u00s/ZNfonuz7h5qU380vmL06dB52fSDRT6Y5/CvinCUvzdqKiQEIioKG9r5+TyTaSJBExuLsjXX1JPdnrxdyp0WlsVrY3tTVx5fdXctvmeeg1DY5VmrY1ws77YOcDfccO869K9pew9Q6nTiGLfrW+fUv025p2Np+E9Gd7oWufwXoyrqmKfn2YYiAoKKj7CxVbRaaGwma8X+3/ig92fsC2Qucbl8pB7+BgWHpY9Ga5ZtQ1RPu7t/LihCDrcyj5vbdHYZvmUsheBDWiWbIs+lVXahkQJiJ5xwcbHWHsTWM57b+nodGZn2aqc6qpzrFtUfFr5q8kvJLA5Pcn9y3RT+Mhqvtiz3CPZ3zNAShaJXpd6Ztg87WiDxsI68+Tf4W528GjB0z27aFqH/w2124LQF9fGFB1G7GV56Fv7B7sMxgM7NmzB4Oh9+wW5fnPbfefHBj38O+Z/kJGI+x6BH47VdgTu4PSDVC4DDxdmAyR9RlsvBxynbOSsERAACTrTsLDENhn7CVAiH6t2mrqjGUApIb0oCApZ0h6BCq3HnYEjZdYmNlrk3wiUbYRjn1sseJKFv12F+/uVnVvyeJTo9Xg4dvdb9ORebM6p5rsddl4+nddjKw9uollgQvYlXyNctFv71OwRa0K6BNodIo+VxKDEgn3DafV0Mqe4j1OXdJoNDJupPhcs8fi0xL3T76fl+e8zPh4S13RLbN7t/g+dKj4e2HGUpi5Qu3l526ay6HoF2g+vjzZMnJvl72lzgUfoUP027+/4xkublwcE++eaFaIMzdnnj/kfLLuzGLRuV3dd+RKv4KoAkZePRLvYO9u5xt41kBOurW7omI0wt694NeSwu3TL+fUtFMtv4mIKTD/GCRfauvtqrgBSYKzz4av9z/ILauy0fsqT7h6bdNrXLD4Aqfskt1FSwvs3qVBY/Rg3DgFB6hV0S6hs+hnV/GRdwSc9D8Is79FTrR/NCX1Jewq3mV3Cw0lPf1csUZfdmQZ/9v6P3YU7nDsBK11cOBFmxbmf2uCBsGpG4V4305gYHtSYel1PHLSC4yIsl2Bp4Sc33P46OSPaKoWyWMr7ljBh9M/pGhXkaLjC2tF0run5I2HPgQfR5aeOh844xDMWC76xqu4j8KVcOg/wpnAQYZFtYt+fajSr6aulSYPkXCYFNy9172jGAwG9u1zPrFNKaro14f52MeHBRdf3P2FCR/BOcWKS2kHtgs5csNmZ5DtPXU+9SbrqdP7W+5N87fBoIctN8HO+3t7JLaJngUXNorefogAt1d7y8f0QPFBv6vIedEvdXYqY28Ya7Z5PcCqe1bxatKrVBy1HgjIrMwEID4gkcb2woNeFf2MBuGnL0kw+AEYudA910k4B04/AP0uFYHwkvVQ3sluKzC9b1rhab2hZK3dmbCS1GEtI88zfY2ldf9kY/op7GxxQxVS9QFYEieqYnoKSRI9OloqRGDOHUx4X9zHrhTGE8+Hcf+12hfNWeSKqr7y0AkiYNmqrSLddzyDwgf1nJW2vglWjhdVm+4mcqpIZog62f3X6i2G/h+cXw1+iWZf7h/WH2+dN/Wt9Ryt6FrSJ4t+R8y0yqgrrmPrW1sp3V/a/UU7mPfaPB6qfahbf6p9uUIBDzTGE9LdedE8M36CKV91VBWr9C5Fv9hMlpAkySUWn3VFdfw79d/oVwkbqYwM25/tWWuzeGvUW2T+kmn29blpc7lr4l0OVTnL/fyGD+1UdeqMHbuKMvKWwOrZdvU7P7nfyXx69qf8e+6/nb58RISwZzQa4c8/nT5dF2TRb3/qfua/O5/g5OAur5uz9JSpqhIVNJLU0TPOIhot+CeDd7hT41VxnOnTwS/Ai7IyiT+U38qsPLqSxfsXu7xCwFkMRgO7dgnhLzy8Q4iyiNEIP/SD9ef2xPD+0sTHi16j9fVQ7ujSS6HFrMzgiMHoNDoqGivIq7Evm1GJvacrOFgmktEHhivNKjuO3K9hx32Q9xd2CnEDkiQsPhPLr+bsyPsYED7AJefNWptF1m9ZtDaIe3XGkzOY8eQMokcoKxSR7dxDPeKQkBy39/QKhSAH7ykV5Qx7Es4tE+4ODjIzeSb/nvg9ozI/7zPxl9zqfJAMaPEk0s/x92YWdyXcm0EV/foom4E1LS1cc+213V/0DLIrM1X+8Mwoz3BqTC0tmESXnTWradY3kxSUxOCIwU6dt0/QUglrzxKBd0eQJJE1POIZ147LHWg8umS7SFJHtV+iZ7vo54JKP5nmmmYM+u6T2tRHp3LS7SeZ7dvSGVn0i/NNAcSDck80kzZLWwOsnACbruyZ6wWmi++SBKf+CSev7JnrOoN/CpxfB4Putf/QkHpKApfz8Z733TAw58nSb6Q8cDWtOjcIZE1FoAsA3wTXn9sao14U95Z3dysOl+Hqc3uFQv+bwCvMteftRHyCgQNxD3DD5imUNZS57Tr2UF0Nvi1JvDl2E/tv2d9zF649LKrSWqrcfy1JA6GjxPe/Kt7hVisZdRqdyeJkZ9HOLq/J9v9HjnTPEK84UsHSG5ey/xvn7w1J0/0ZM7tSBIpCdXbMUT4xItFIpW+w/R7YfrfN3c4ZeA43j72Z4VHDHb6UX5QfISkhhCb40r+/MtGlvqSe2sJadD6uTWjq3M/vFO+LYN0Cu4OmKg4SOR3G/McuS7rEoEQuGX6Jqbefs0xsd3rrbPHpCnw8ROmBwWig1cz9tPz25fyn/3/MtjnIzwcjRipS3uK3nFXWe8VUH7Bg9a/SU3h6wumnQ0Lgftq23IuxTlli44AwEUCXBY2+wr2r7uXMlYPIC/2Mk05SEFbSN0D0bAixbg+tYhsPD4hrd6jOttcptrEIfhoMOx+06zAvnReDwkUz5uOfK20h23taq/RzFoPRwMFyJ0W/+LNgxjJIMlMwodJB/jL484YulbtR7S23i11YJDnlgSlcu/lafMOEWheWHsb0/5uu+HjZzj1IEws4EPtrKoMj79jlMqDiBH4JTre4ivKP4vLx8/FtSaK0FJr7gCNrXvtnfajG9f15vVucb6ellL9wVKXvY8npZjNwppcXz73wAtHRZrIhjEYoWi0mbQWYRL8y50Q/2RZFp4NfcoS15+n9T7fpyXxCULEDCpZC3veOHS9pIGIyRM107bjcgUEPZZuhqsM2Rxb9Io2uFf12friT50Of59jqjklNLvOPHhHNvNfm2bx/jlWJYyM9RSpsQEAvujHpfCGgP/j1c28fNHN4mrH67YtIkshKduTQoHz+TD+NF/ffQptC++KepFISnnqDo9Jdf/KomXDm4Z7vQ6Dz7agYzf1W8eeKTco2w9r5wu7VXbRUdZnHXEligoaS4KUcbt7A+uz1brmGvXTuqdujBA+DM4+InmAqzmM0QNUeq/eubPF5fHCmXz8xxVZWQsVx69i4cXFc8M0FjLtZiU+XeRorG9n+7nbKD3VPbMitEWmXEV7xyk6W973b+m6qOMiof8GED23udt2Y63jj9DeYkjjF4UtJksTlv1zOtEemmSwWbfX1G3L+EO4pvIeESd2F5fKGcr478J391l/Fa6n75SqamiDA34Bv3CiQdCIBTsX9BKTBgFvB31Y5m/uQ77+dO6HJgTaVcz+dy8T3Jnabj+VKv6SsJL679DuKd3eNmPqG++Ib7ot3SHdbsfx8aPYoYkPIjcz7bB5GrFjubfgHrJpk/8BVXMppp0FSyEFmRr9E0Y4Vio5JDxPrhUMVh9w5NLv5I/cPivUiLjRWiVukzg8mfQpDH3HvwP4mONzXzztK/C4csMC31C/aFj1R6ZdXk0dDawMeGg/6hTj4WeEZArHzTpx4SW9R/icceRtqO+ak6GiRhLI99wCL9y12uJ9zZzQ6DXEnxXVzDVGKLPoFSkL0s7vSL/dr+PN6KHBRXEPFNo2FkLXITt/irgQGiliv0Siek3qbgvocAMI9zLvzOEOTV8+1aVFFv15kMjDH25vPgF+AT9t/nqbV8tDChdx1t4VsXEkSlUa7lGX5yKLfkYoj1jMJbSDb8gQGGVnW3s/v9PQT3NqzIR+23wuBA+G03TDEvswpQGRdFaxwaoLrcVZNgL3/NP0Y0V6IE9ggRL+sqiyqm2z327NFwqQEAmICSJwsJsqMJRn8d8h/qTiiPOsmqyoLgFApWYyxp609W+tg//MdPTQnfQpjXnZY2PpbUJ0Bh/9nd8A3NSQNrd6fFmNTtyQFjUbDsGHD0Gh652Orua2FOl0WAMNi+7v25Pr2h+vevKfaGmDrrbD1ZtC7ILWqdL1IpHAXBj38NBA2XuaW0ycmQmjtDADWZK1xyzXspaZGLMp6VPSTqxd0PqA1b9WsYidGAywfDbsetrjLyOiReGo9qW3pOod6eUFSe0uB4y0+tZ5aBp0zyJRVC/bPm8W7ivnxuh85vOxwt9eK6kWlX7SvAtGvap+optpxAlie/52ImQNRyjOtXYU9ooskSWaTwbYVbuOcr87hsu/snPP3/hPf8m8BI0OGapCSzofx79h3DpUeZ2fRTl7d9KqplYQzJCZCTAy0tsJ2Gy6L5ubMrQVb2ZS3Ca3U9RnNS+vFb1f8xktDXyLjiwyqc7uum2Y8MYNrNl5j9n7Oz4cGL2HfnBiUiKc1C+S0G2DgXTbepYq7CQqCwAFzuGPFdj78/XpFx8hWeX2p0q+prclkNxpaN8lkG67Sczgs+kkSzN0Cwx63+5pyrzZ3VPo5u0aX1/xpoWnoHGld0tYI9TknViyut0i/Bc4uguCO3n1ypd+zRVO54OsLXNISyllk0c/P4KDol3SRaIkVP9/FI1OxyP5/wR+XQLXjCdnbC7eR0+8pCoO/I88+J2K3UNQoKv2ivF3Xzw/EnDlkiGvcLBRdr8eupNINPTD+nnt4qf8gLvSK5T7/QUx7+BGy8/IsC34yY1+Hsf9VdJ34wHj8PPxoM7SZrBIdoXOVwbKLl/HsKc8yM/kEqGyzRtankPESVO6AoHabUqMBMl5RLlgceBHWzIPi1e4bpyvRaGHkc8Ierx250q++PISEQJFhvbt4t9OXCksP4/ajt+PhKzKqyw+X4xPqg0+o8m68sqe3n14EGnu8yuXAv4SNRvaX4ue/QmWru8n/HrbcLKpZ7CA0RENQwygAs/0vWlpaum3rKfYXHsMo6dHq/egfHeO6Ezfkw3dxQiTtTXS+MGOFsEZxhbgz6F44KweC3fRAo9GKnprpt7rFEz0hAcLaRb/f+oDoZzQK0W9z+mzm/JjukkCoTZpK4ft+IiNUxXVodDDqBUg1Y9/ezlUjr6LuoTr+Pa97Tytrff0AagtqqS+tN/1sz7wZNTyKi368iAHzu/f0KG0Wq6/4QAWin2+s6LuZZvk9qvQSRkNHookVGlsb2Zi7kfwa51JtN726iZXnvkNMlIHWVtht4dGyOqea3x7/jZJ9JWZfl8cRFxhn3wBmLOXDnFWAxLBhQOAAURGg0nP8cSn8NMiuQxbvW8xdK+/ii71fOH15SUL87oGcHNv7d54zm9uaKW8Ulc8xAV2f/SRJYkbyDBbcvoBHmh6h/zzlCWH5+VDfLvqlhtjI9k6/GQbcrvjcKu7j9LP8yKwcxcaNEiXmp6ouyJV+mZWZtOh7bw3TmW0F22g1tOLVGkW4R7Kpn7pVttwMe/5pez8VRTgs+jmBJQcJWyit9HNmjb6vRLjCOGztWbIGvk+Co2pCj028I8Anqks8KzoaJCRCWocCsLfEPS469iCLfj5t4pnPbtHPMwhSLneoKlbFQVKugIkfO9Wq5tdjv7LB8zEKQ7/sE339SlqE6Bft4+JKv5L16I984NpzWkEV/XqZu++5h60Z+5k6N5+xM/Zz402Pmrf0PJ74+RCpzHZHkiQGhg9EQjJZJTqCLPqFBEsMixrGg1MeNPUzOGEZdD+c/DPEntaxLWex6HmyT2F/vgG3w/CnTgxrT5nBD3QZryz6lZTAXRPu4qU5L5EU7JqMhs5l/aOuGsW1m65VLPrVt9RT1VQFgHeL+NDv8Uq/IY/A+PcgWfWIV0zCeUI8sqOHC0BICAS2i37HW3gZDAYOHjyIwdBzTW87sydfVL74t6Th7e1C4bepGHzjwdf1tgF2EzK8I/mhk9e/w/jaGZy1l4F3Qeo1bun/FhIC8W3TANhTspvyBjf0cbSDhgZoa4Na7wNkVh/Gz7MHGps25Aox2CPY/df6uzHwLqvZpz4ePnhozdsPyqLf4e7FeORtyuPluJfZ+uZWwP550yfUh/Qz0rv12jUajVToheiXHKJgMecZIhKLQscouq5KD1F9AL7yh/0v2Nz14m8vZtL7k/hy35dOXbKxspH6knqSg6sAyz1jstZmse6f6yjZYz6SLgeA4gLs+1xpM3qzatt4oEP4UelhvKNEEEiB2Cwj95PcXeJ8AiKIz3TocK2xxPFzZlFdEQAeGg/CfMz3EdZ6aNF56br0Qj3w3QFW3buKuqI6s8eISj+RiJsSkmLHO1HpTRITYdhQA4PCfidjg6VGLR3EBcTh5+GH3qh3KvHalWzME80tg+smEh8n2c5lNRqE/X/ZH+4f3N8EX98iDh96iq8/HER6XBzjBg3i6aeeoqioSNkJjn4Ay0ZAW73tfdsZGT2SGckzmD9gPgaFyZJGozLRz9k1uuzqNDZWidesGXxiIO1GCFdtkBVRlyVaG7UjV/r51PYd0e/uiXfzydmfkNg8D7BT9KvaA00KsjJUXEvICOh3GXgGO3wKuad9jc+ePiH6pbedR/+CxxkfcYpLz2s89B88dt3j0nNaw7Wd0lUcQqMR9opFRWIxHB6u8ECjQXzYewTY3PW7C78jzDcMXw970yQ6kBdKPV5p5U4kCaJndd2WeIGo8ut3qbJz+CXC0EddP7YepLPo9+xE91nI+Ibbd/+1Gdq4d+K9lDSU0FYv1L4A27e78xj0UPyrsMLSekLq1T1w0b8QAaniy07i4yGoQTSJ315kw4OphzlQIiLsIQYX9/MLHQ3zdrr2nM6SvxQ2XQHTvhe9Su2hIU9YO4x8AcLHu2d8x2No6+hL6CIkCVKiIgloHEKtzz7WZa/j7EFnu/Qa9lBTA22aepo9ReA7LbQH/JhCR8Pp+0BSrYz7ErLol2kmfhgzOoaRV48kfoLCvnvH0drQaqrM74zBaOCMto84WJhL8pRY6yfRN4nnU53jz5sqbsI3ASKmgl+yzV3Hx41nScYSU4DYUaY+NJUZT8zg7bclOGhZdBn6j6FEDIogJNV8FZ7s+qBY9Nv5MEhajnr+H01NngQEdFRXqPQwo1+y+5AR0cJ6bE/xHgxGAxonk3vktaucwKqUwrpCQFT5mbPp/GjnR5QVlzFLN4vUoan4RwkPvMNLD7PjvR1Meah7gm5bm1jzNyQqqPTL/Eg4QYx/F4KH2jd4FbcwdXwFc+pmcKTkVMC6jb0kSaSHpbOvdB/5NfmOVzK5EHlOD6mfSPxgBQdIGji7QPTRVnGaV156iQcfeIDJRg/+z9BEdB0UFRTw8cKFPPXkkzz3wgu2Hb/0jdBcCjUHxbO6AsJ8w/jtCvtcQpqbQa8X/7Zm7+ksr817jcdnPI7RUXvOkJFwUi875pxIrJ4lXH1OFxWWXUS/UNhb2vui3+iY0YyOGc2Oj8TPdol+f94A1fvhnBIRx1PpWZorhD7hQO/sYVFC9Kv3PkhWbjPQu61FYhtOZUDBqZykoCbLHoxj3ySrfhzwgGtPbAG10q+PIE+2ljJgu9FSDYuDYNudinZPCEpwSvADsVBq0Vbwlf4SPtv9meMfzH2BhgL48yaoMdNYW5KEJZW2vfF60S/me1y1VArLR4PevWN1B7nfwZIEKBKWpHJPv/Jy6KVCKrMEeQfxrzn/4qMFH1FbKxbbPVLpt/sR+O1UKFjeAxf7i2I0ir8RO0hJ6RD9dhTuUJyJ2BM0NhrwbI0kQuvCfn7y+5OkvmUbG9AfPILs7skIQMnvULbR7t+9w2y7C74JBzfYJsXE9J2+ftXVHVUBId4hhPqE9syFNR5uqaT825P3I/zQ3/QZbI73d7zPyDdHsnD9wi7bE9uLgisrOzKwZbSeWs567yxSZzvWHPzdCe/y9pjudq5ajZbYigtILb6HqDAbC7CC5eL5NOcbh8ag4kY8/OHklcLyyAaTEkTG/B+5fzj1vK/z1iFJkqnSqsJCS2eth5bYsbH4hJh3gjCJfkrsPQ2tULgCin5mz14RdBg6tG99zKpYJy00DW+dN/Wt9S6pkFJa6Xc8coVpbID5ZIcn1j7BWx+/xZJZS7r0Qj3z7TO57fBtXXqsyhQXi0B6o0+76BdqZb5urYb6LND1QHW/iiJGjg/njS1v8d8/Fpr6nVlj5aUraXi4gVNSXFsp4AhGo5GNue2iX91E4pXmB0ka8Oqh586/MK+89BLPPvII6/R6VhuauAQ4BbgEWNnUxDq9nmcffphXXn7Z+olSr4GzshULfo4i399aregp7U5CfUIJ8zVfTa3iYgbfDwM7hGUvLwgOhoDGvlPpJ9PQbjykWPQzGiH9Nhj6mCr49QZH3hFxmVLHKsPjAuII8gzGKOk5XJXR6206GxvFd7vtZW3hGUytf89VJquRnD5C50orRXgGQcypENxzXjVVVVAatJLNDYt4bsNzZjMe+zStNR1B9oYcOPKmqOayRvFaWD0Htt7W/bWcxbDhH5D3nevH6m48g8ErAoxtAISGigc6vR7Kyg3sK9nHZ7s/Q9+HBE05O7dHRL/022HAnRDV+wu0E5bVs+3u4eLvDykBg9AYvKltqeVoxdEur2u1vVdxdEbYPczZVcw8r6dcd9Kfp8CfN7rufK4iMB3OPAyxc+0/NvkfMP+Y+HzqCXwTRDVii+vtN2NiRF8/fykcL13vZppVV0O9lwgo9kiV3+pTRWKMinvQ+YnFqMFMQlE7dS117CrexYbcDV22+/iIz2zAapNzWaixZ95MnJpI8sxks6/JwXKb/X88gyF2nt32zip9i7GxY9FpdBTUFpBb45zHTlVWFVVfrMC/IodKM/kgrY2tFGwtQN9q+ZnT1NNPSaWfxgNO/ROmfsv+A2KtMlQtkupd9v9LfClEp9ExNFL80lzRY1yu9FMi+nWeMwtrRaWfJdHPW+dNaUQpqQ+nEjeu496UNBKhaeZFkgKhI9LoraDSb8DtcE4R+PezPXCVHiEmBg62XUNmxQi2brW9f4RfBFpN33BMaNY3My9tHmH6oQTXj1Um+uV+B6UbbO+nYpWioiIefOABfmxuxpIPynjgx+ZmHrz/futWn1ovh6poAGqaazhSYaEp9HF0tva0FfbrtTV6XSYsGwlZzvd//duQdr0QjjsRHQ3+jeK5Pac6h5rmmt4YGSB6Sr+7/V2WHV5GfYNYyygWXSQJki+CQTaqZVXcQ+hoSDgXtI61AJMkyVTtV6HbQ3kvdldpaG3giP5X6r2OuFb0y/0Omst6dM5URb9eRv5ly238lFp5AzD1axh4p6JdKxoruOHHGzjts9McztitroaywJ8BmJc2z6Fz9CobLoKyTeLfYeNhytfC/9saEVNgyEPi63iSL4PU6yB+gcuH6naiZsK87cK+EmExK9vKFpcYGf/ueC797lIOV5hpGtSDFNUVkVeTR5uhjZr2Zw+3iH6tdSLAfeBF8bNvLIx5Rc0QcoboWRB/lsi4t4P+qTpGH/2SV9J30S+kI8ih1WoZNmxYry0q5PsvKNBFH5utte0Ltj5aMS1XdzWVdiRL2ELezzeu50oqBt0NM5aKfg4uJiYGoisXcEdrCS/Mtt0Dy53U1AirC4AB4QPcezF9M7RWiQoDFfcQfbKw1Ym1/Cw1Pk6EhTbnbe723CYH6syJfg1lDbw99m1W3bvK7nnz9DdOZ86Lc7pt31m4h0OaJdR7HbEt+kXNhOk/QFDv25ipmKFguXgebiiwupuvhy8jooTFolwZ4iitDa0UfreZoOKDZkW//M35vDPuHTa9usniOeyq9ANh+ewbS06O+DFFbZvWu2R9Aof/a9chwyNFX79dRbucvrw8b9kS/Y6fM7UaLcnBySQHJZvd31vnTVVIFRFXRRA5VGTwVmVVkbU2i9YG88+/eXlgxMjV3kt58/Q33f+ZruJyJkwAjaTn2PYdtnfuQ3jrvHl3/nvMPLAHrdHHtuhnNMLWW4RdnopTvPvOO8z08LAo+MmMB2Z4evLeu+9a39HQCjsfElbWCvnp0E8EPRfERd9cpGh/udLPWj8/cG6N/uIfL3LKx6eweN9iu48FRFuJ5lIw9p1E9RORmBjw1IcQohMJLvtK9vXaWHKqc7jux+u4+JuLaWwQ8QRFoou+GeqOuXdwKtYJHQNTF0P4SQ6fYni76Ffrs8eUJNUbHCg9wIqoWfwxYKrrRL+GPFh/LtpttzJkSM8lx6qiXy8jB3LkSj/F9p524uvhy7s73mX5keWU1DvW2LSqCir9xMJ/auJUF46uB2gqgZL1cOxj8bMkQeK5tgPTGi2MeKYjwzLzQ9jV3r9P5wPj33Z5L6neQr4Hy0u1pgwLVyy0neHZ9c+S8EoCj65+1PWiX2udqP4E0Xso7zurVmsqdjLkQTjpLbszEVNSILp6PobC4eg6/W0ZjUZqamp6zVbY5ZWmHgEway2MfcNFJ3QDBcvhh35w7BPb+5asg6VDoexP94+rh4iJAQ06igp7v6q9uhrqZNEvzM0BQq0XnLoZJn7s3uuoWGVk9Eg8tZ6UN5Z3s7eTA3X5+d2P8wnzQd+sR6PTuGze/Hj752xNO5tjUa/1TLW9ivuoz4LsL6D2oM1dZYtPZ/v6RQyO4IylN5I/cJZZ0S8wPpBp/zeNtLmWq5hfn/c6r5z6ivWqKIDKXbDlZqjaQ0tLh4NKQoITb0DFeaZ+C6fbZxk2PKpd9Ct2nehXVyd66lni+DnzxrE3cuyOY7x0qvm+hD46kc3e2NZo2rbn8z18NOMjCncUmj2moAAkJCYkjOOGsTdYbr+hb4H9L0DZZutvTqXHGT8ebj/pGq6IGktrrfWM7aK6Ii7+5mJmfzK7h0ZnncpKaGoSCb8xSvLlpiyGUS+6fVx/db5ftIjLmpoU7Xt5YyNLFi2yvpPGQ7SgKViqODlTXj/sKd5Dm8HKRNiOXOlnq5+fM8+aq4+tZvWx1RTXOxgEjZwGZ+eL6i4VZdQeEdWRGa+YNsmtA07VPs+3F3zbq/1H5SSv2IBY5D8ZRaJL9hfwYxrkLnHb2FTczzCT6Le/V0W/rKpsAHxaklwn+nmGwsSPMQ64g5qanqumVUW/XsbQ3kBN7umn2N4ToPaoyNZV0DfFW+dNv2AhXGWUZdg7TABK6yqo8xHHToif4NA5eg3vSFiQA0P/z/Fz6JuE4Jf1KXRa3J2w5H4LOx80/djZYlbO7nbFQtsZTJndAXGuFf3qc2DpIJEhB6Kq6bQ9omJIpVdJbY/nHe3q7InBYCAzM9M0Z/Yk+0v3c09uMltTzjNZRLmMvpw0EDoW/JKU9YyoPQpNReDVC/0YMj+EPy7F1cbvcjCkpARaW41UNFpoRtUD1NSAb3M/kjzGmAKhbqcv35t/BXK+EQFdC3jpvBgZPRKAzfldg77WKv0kSeLG3Tcy+/nZds2bh5Ye4odrf6DiaPf7PKtCXChYE4/G2sqhYgf8Nk9Yo6v0TZIvgwvqREWmDSbGTwREXz9nSZkYBZJEbUkjDZVdA5+haaHM/OdMooZFWTz+3MHncueEOwnytvEhXLIWDv8PmkopKBAfCwEBPWQNr2KZgDS7+9KdN/g8Nl6zkU/P+dTpy/v7Y5q7qq0Usdv7rOmt8yaiJIKMszPY8YGo+ko/I51ZL8wiZpR5RUWet+NsFa3WZcLOB07MNhJ/cfr3hx2V/+CrfQ9z4ID1cJqvhy+f7/2cXzJ/obqpdx0UMsoyyMoRYk90NOhsPeZJkrDQd8TuX6UL1TU1RCvcNwqotjZRyUz/AebtUNx7OzU0FT8PP5r1zRwqP2Rz/872ntZwdI1uNBr5M18ki8ruFg6j9h9XjlcEtFRCJ+E3KUl8D8+/lLMHnU2IT0gvDa6jl26Ub4ettiLRJXAAxJ0FUTPcMzAVZRT9Cr/MhIptDh1+7qBzeTx8H2OPfturot+xCmEV4tOS6DrRT+cL/S7FEDaRrKwsF53UNurs2EeQRb/SUusZiF3QeIiMhsqdinaXMzYcEf2MRshqFR/K/YLSiPCLsPscvY5nsLCdcxStN8zZKMQhnWM+xX2K/B9h//PQLAJ8Ee2/0tLSDtFvZ9HOXhqcoLOdU22t2OZU4EYWBXwTIGR0156Y3hE9Z0n4d8DQCpuuhr3P2HVYSgq0aepZ1/wfbvjh5l6r7OvMofJDVBqzafTKck3gMO97WHc21NhecPUq3hEwb3fH30nzcWJAwQrI+Vr8O/UqWJAHATaqMNxB+RbI+QoazWfVO0pIiGhuXuG7mcRX45n5ke0gubuorob0wsd5MX0r8wfMd9+F2hpg4xVQvMZ911ARZH4Au/8PrPTO7Wzx2Rlroh/gUM/lvE157HhvB4a27kGb3GrR0y3C00a5VM1BKF4tkqRU+iYe/orFlxnJM3hx9ov8Z95/nL5sUBBoDG2k/fk5H874iNaGVkoPlPLZvM+oyqpy+vwm0m+D+ZkQMYnc9laE8fHq412vYzRA1T6o2qP4kLjAOCbET8Df00aZiQIkqaPaT0ksXSneOm8A9HV69M1iLo8aFsXk+ybj4Wve6aKgALIi/svvjW+begaaxS8JZv8BKddY3kelV5Ak8EmZy2d7nmLDtkir+wZ6BRLjLwTgg+W2K6zdRWNrI8P+N4xJPwbRrCtR1s+v7phyi38VqwQFBqK0i08xEKQky9Qnxi6xSyNpGBGtPMYj23vaqvRzlMzKTMoby/HUejqW0Khvgd2PqdXQ9uIZBAuyYfB9pk1ypV9eHuh72SlVFv0ivIXo5+EhvmwSPgGmfStiviq9h6EFKrZCXZZDh0f4RTAmcTAaoweFrg3t2MXRclHp59uShJeXC07YWAyN9vRycx2q6NdHCAkRk5nRCGVlCg/yTYDzqmDEU4p2l0W/A2UH7B5fQwOU+Qh7n0kJE+0+vlc58CJkvGZ3bzGz+CUIW76/AkMfh7OywVNk8nSp9IvuI5V+NUL0i/COo7lZbHNIdDG0wrY7YfPV4mdJgunfQ38bPR1VHEfjAYUroHS9XYeFhkJwoJZ9cXfz9o7/kVuT66YBKudwueht6dfU3zWVfhU7hB3LiTCXaNr7M9TnwLLhcPSDjte23gY77usQ03Wu7HJsB8OehPOqRS9OFyJJotrPt7kfRQ0F7C7eTXlD73SUNvWUdHWl6fGUbxY22JUnVp+aE5JR/xJJRFbUCNlVYVN+115ncoVIYaH5xbnRYGTtU2vZ8NwGxcOZ+c+Z3Fd2H6Fpod1eK6wX6mK0r40oYfI/4Pwa0bNQpe9SvR8KV9ncLSYghnsm3cP4eCcz8BFVVkGhWqojUomckIKHrwe1+bUcWXmEoz8ftXrskYojfHvgW2U9ZiRJWPJrvU2iuKLgtop70TfB8uGw+/FeG4L8+Wmrr19nRr01ignvTiCnOsfs6z4ePpRGlhL6bSijrhnFtne2oW+1HDFtaICKCjgSs5DHttzAkYojli+u84GIiRDYX/mAVXqMk9pbFm3d0oax1boDkNy3UUl1lbtYeXQlbYY2fAjBsy3C9rxoaIPlI2HN6T0xvL88Z118MZ94eyva92MfHxZcfLGyExesEI4nChkZNRJQJvoprfRzFLnKb2T0SLx0DkTVq/fA3qegYJmLR/b3IzJSJLo265v4ePOPvLzx5V4biyz6hXmKdb2iKisHBSYVNxA9G86vFq20HCS2PaTTq/aelUL0CyLJNYmDB1+DJXGiDUEPo4p+fQRJ6iq6KD7IU3kE0JlKv+pqaPIQq+fJiSeI6Gc0igz6w2/C0XdAUq3KuuCfDH6JpoBj5/tvWKSo7CmoLaCsQakK7VoMRgOFdSK9I0gSEU6dDnwcKbKUdKKqquagWoHQk5xxEGausOsQSYL+/bwJaBLNbbcXbje95q1wseRq5EW6X3O6ayr9hj8BZxeKDM0TBZ0fBPSHuk4BqpHPwaw1vV9C4R3uturr2Fjwaosk0Vvcj2uze8e2sLy6GSN691vURc0UFZv9rnDzhVQIGgSB6VaztMfHjSclJIXBEYO7VD1HRICnp3CGMNcLWtJI7PtyH7s+3qV43pQkCd8wXzTaruMxGo2UNovnv7gABeqJ1svuXq4qPcyWW+D3811uiWyLkFCJwvTpJF07C4CUWSncdug2xlw3xupxyw4v49yvzuXxNTYEo8ZC0Yu2VVhDqKJfH0LnCyOfh7Tr7DpsxZEV3LrsVr7P+N7pIciVfrZEP3nObNG3sLNoJ5vzN1vsu/fI1Ef4+bKfmT9gPtvf2c5P1//E1v9ttXjuggJo1hXT5JmPhGSycDZLU6lrElZV3MKIETA4aisvTYmldNsXVvdND00HHG+x4gre3vY2AIPbLkVCsj0v6huh/02QcLb7B/c34NrrruO31lZs1aRtBta0tHDNtdcqO/G+hbDtLsWf5+6q9HNkjS6LfifFnmT3sQAED4e52yDlKseO/ztTsQ12PwFNIvAsSaLazyjpufrn+dyz6h5K60t7ZWiy6BeqE/E/m6Jf9QH4IQX2/NPNI1NRhEbntN1uhedOdiZfwUrN7T29VDGRVyuSvYKlRNecMHw8pF5nctDqybimKvr1Mlqt1vTv6Hajb3MBHIs0FkHW59BgWwYfFD4IcKzSr7oaRmS/x2UF5Vw8TGHmUU9j0Hf8P9QchJXjoGKLyKSf+k3vB6b7GkajCJDUiiB+Z9HP3zOA1BBh07erqHeq/UrrS2kztCEh4d0m/jgCAx38NUoSTF4Es38XNq0qPYODlWypqRDUIPrI7SgUFUdarZaBAwd2mTN7isMVLq70g97pfecMXmEw6zcY/nTHtsRzhf1UX6A6A7K/dPlp5b5+KRph7bnqqO3qGHewo+0zlo/25cVD9gVMHcI3Dry6V3upuBijEZrKTAtuc6SGpnL09qN8cNYHXSw7Jcm2xefFP13MzXtvVjRvGg1GDi87THVud9+78sZyWowiWSYxxIpFelMpHPqvsANT6dsMugfGvQVG2x5O5Q3lfLr7U97a+pbTlw1pbxFTVdVxL5urLD0e2fUhLsCGRX/eD7DmNNFPBFX063MMuhdi59l1yNqstbyx5Q2WH1nu9OWV2Ht2ftYsqhM2TB4aD8J8zD+zjY4ZzbSoaRQvLiZ8UDizX5zNmOsti9j5+VDtK5LZ0sPSCfCy8pz8+/nwnWsdDFRch6cnJKWHUFibxsFM66rIsCgR5OutthnZVdmsOCKSMKMLhJhkc170CBDJfWnXu3l0fw+io6N57vnnOdPLy6Lwtxk408uL5154gWg5MGiLkc/DycrXJnKiwc6inTZbaCit9HN0jf5nQXs/P0fdBDQeou+8f7Jjx/+dKdsEe5/s0iYqMRF0Bj8iPfoBsK9UgbuCG5Db+wRpFFb6STpIuhCiZ7l5ZCqKqdoHGa9Am/UqeEt4+teRF/4xuUFfUFbWO6pfXnv1aLiHi2Jd8WfBSW+CpEGr1ZKenu6a8ypAFf16mc4Nb2XRxS7Rr2Qt/HGx6KFig0ERQvQzGo00tzXbM0xTVmRUYChB3u72F3OQQ6/DTwOg5Hco3wrV+4QPvc5HNHZV6YrRAN/3ExZ9QHi42NzSIqzknj3lWX74xw+MjhndK8OTP/Cj/KNoqBNVmnZXuWR+DBmviuCqZ7Da5Lmnaa0R/euq9tp1WEoKBNaL+257kQiOGAwGysvL7W4S7gpM9p7N/Z2vtNr5IOQsdn5QvUVfTZ7Y/aj4LGytcelp5TV3bIMIVC47vKxX+kyWGA5i0LQQ4OPGpIWaw6JHY2ud+66h0kF9FnwbAfuec+hw2eIzP9/868HJwUhaSdG8WVtQy6LTF7Hh+e52oHk1QjnxbI0kIsSK/VLJOth6i6LnUZVeJu4MYcWqse2AkVGWwWXfXcZjax5zeu4Lbdf3KivtO65zf2erRM+CMa9B5FSMRlX067PYcR/JVSm7i3c7fVlZ9LN2/3V+1pSrDWIDYq32SW2ububHa38kY0kGk+6ZhM7b8t9VXh5U+4nn2jGx1itciZmjVt33cdJGpnL/L3/w487zre43Jkb8rrcVbuuJYXXjvR3vYcTIzKRTaClKA9R5sTe46557eGjhQqZptZzq7c1nwC/Ap8Acb2+mabU8tHAhd919t/KTRkyEsHGK12fDo4ZzybBLePaUZ9HbSPxRKvo5skY3Go1E+kUS4h3CSXEOVvpV7ga9fTFNlXYSzoW52yFymmlTUru2EaYfCsDeEvviN67ixdkv8unZn5LiJcRgm6JfYH+Y/DlETHL/4FSUkbcEtt8NVY4Vj0xIGIfW4EOLRym/H7S/YMlZjEYjt6a/SP+CR4n2THX5+Q0GAxUVFS4/ryXUCHgv03kBHRUlvtsl+kVOh4mfQpTt/imhPqFUP1hNzl05dvtmy1mRbu8n5AyBAyF0LISOgn6XwFm56uRvDY0WhjwCSRcBoqekHJApKYHzh5zPmQPOJMQnpFeGF+Idwn2T7uPqkVeb+lnZJbgY9HDwFch4yeVCgIpC6nNg3QLIWmTXYSkpHZV+sr2n0WgkNze3xwWX+pZ6CupE4Cegpb9zzcybK+DAS0JYUXEtA+6Aqd+CxtOlp5U95X0KZ+Kt8ya3JrfHF0EtLVCtOwjAkCg3JrAc+1hUFjj4gK5iJ76JkHqtoucUg9FAYW3XbuZywC7XStvT8iPlrH9lvc1509PfkzPeOoMhFw7p9lpiUCLnGr5kcO6LpqC5WaJPgek/QuxpVq+l0odQ8Hk6JnYMHhoPSupLOFblXBWnXOln7zrXJPrZqvQLSIUBt4NXGKWlYu7U6TrWVyq9TM0h+D4F9itPdBgeNRyAPSV7MBidS/qS17DWKv06P2vKFaYxAZat2LcWbGVR9iJGfjSScTeNszmGggKo9hXCz+hoG0mVQx6G0S/aPKdK7zFypPh+8KDo12iJEdEj8NR6Eu4bTk1zz65J2wxtvLfjPQDOThRVe4GBEGDNjKWtAX7oLyo1VFzKXXffTXZeHtMeeYSFiYO40CuWR8IGMf2RR8jOy7NP8JPRt0Czsp7j3jpvPj3nU64ZfQ06G4k/Su09HVmjS5LEdxd+R/n95fQPdaBvaUsVLB8Bm1RrT4fwiRYx004OWIntLobetUL0U9RH2Q1MTJjIJcMvwd8gFjpWRT+DbccKlV4g+SI4+ReTlaW9eOm8SJTE+vjXo2tcODBlSJLEzJArGVDwFMG+zgT/2tn9GKycAI1C6DEajeRZsupxA6ro10coKipixfKn2LJmEP96Io5xgwbx9FNPUVRUZP1An2ghcPkqs/8I9HKsTOXVg7ezMX0mhb69Y22miNhThf2crj0dyTu8d8dzIjDs/yDlctOPERHie2nvWHh3oV9IP16Y/QLPnPKMaYFul+in0cKsdTBjhV29L1VcSOAAOOkdSLEvUzk2FiINI8CooaC2wBR46Q0qmyoZHTERv6Z0wnxDnSt08wqFswtg5LMuG59KO5FThW2Ci+17ZXvP8mIfZiaL5Jqlh5e69Bq2qKmBem8h+g2LcaPoN+B2USUTNsF911DpQKOF8e9A4nlWd9tasJWQ50OY+sHULttl0c9SpR/AmkfXsOuZXVQctq6yeAd7M+b6MSRN7W5hEuoTSlzVBcRXXGZd9PMMFhVkJ1Kv0r8rrXVCfPnTtm2bt87b5PiwMXejU5eVRT+7K/3anwHiA62UpuibReCzHXktHRMDveAKrmIO3zjR209no2ykE+lh6XhpvahrqeNYpXOis9KefjK5NSKjIiEwweI+SzKWcOOqG1kdsZrwgbbXnXl5HfaeNiv9VPo8kZEwa8gKbh5zHQf21Frcz9fDl+oHq9lz0x6HYzGO8nvO7xTUFhDhG8FgzQJAQZVfQ66oBFdgAa1iP9HR0Tzy6KN8vWI/k2bnM+us/Tzy6KPKLT0701oHi/1h250uH6fSSj9nkCTJaiW1RYx6GPo4JF7g+kH9XWhrEC4v7ciVfprSdjvi4p29MKgO5EQKq6Lfzgdg2QirrRJUegH/FJEMasfz3vEMC5gOwJ/Fa101KrtQdP8pxdAi7lGv3tEnVNGvD/DKSy+RFB9PzqcLeakug8/qCrgzI4P1CxeSFB/PKy+/bPskrbV22aXYy67aXygPXIOXb4vtnXuaotVQl9nbo/hL0Lmvn8FoYOmhpTy6+lEaWq2kL/YAdlf6yX8LHgEQ3L1yQaWH0HhA2rV22+tKEvRP9iOwYQQaNOwp2eOmAdomPjCe9yf/wcy9B5239gTwjhAPQiruoaXKpacLDxdV0G1tsCD5Cu6acBezUnq2Z0B5ZRv1XkcBGBjhRtHPO0IIfxo1Qt6XSAtNo6a5hqOVRylrKDNtt9XTD2DyQ5OZ9OYkQtKcq9iXg+QWRT9DKzT0XMaiipN4+INfgmKBdlKCyLb9I/cPpy7riL2n0WhUZu+Z+y18HQwFom+V/HeRYFmvUelpdH5w+l7xOaP0EI2OIZHiOd5Zi0+7Rb9q26Kft04kGjUq6FtjNEJWUSWNXtkAjIoeZXnnghWw7hxhX6fSpxnffxunpr1L7r4Mq/vJ90pPMyN5Bntu2sM7Z75DUb5ww7Ap+gUOgDMOwEAHqs5UFCPHXcrKQO+ovurhD8mX2uVuZTQa2Vuyl9f/fB29lUoppZV+juB0bMkrDIY/AQkLXDGcvye/ngy/dNh7hoUJgSOgViSk7CzaSau+tUeHVFRXxDvb3uGXzF+UiS5ab5C04BXRI+NTsQOjEeqt2NHYYFLsDAD2N67pcaevXUW72Fz6K026Inx8XHDCkc/B/KO9FmNRRb9e5r9vvMGzjzzCOr2elc1NXAKcAlwCrGxqYp1ez7MPP2xd+Nv7NCwOUiR8bcjZwKyPZ3HFEuWVN5WNlRTphZfumKg+VgGgb4KNl8IvM0TQScU+yrfCyvGmHmOdLWYlJK798VqeWf8M2wp6vgfBscpj5NXk0WZos0/0Mxph1STY/YQbR6diF/oWu/8+U1NhdOYX/Du2irlpcwEIsOpF4z5cYm/cUiV6TLbYWeKgopwNF8N3cSJz0UVIUkdfv5P8LuDlU19mbOxYl51fCQcKj2HUtKIz+livdHGG4rVqL7/eIPtL+Hka1Gdb3CXYO5iB4QMB+DP/T9N22Xq2pgZqLRQYRA2PImVWis0s6m8v+ZY3R7yJ0dB9UbXqyM8cMC6hyaPAsuhXsR2WJMAB1YruhGHWWhj+T0W7Tk6YDMD6nPVOXdKRSr+a5hpTcNCqvadHEIRPMiUZqf38/jrIFp89JfrJz5r+nv4kByeTEmI5UUsWcprammxev6ICDA0hzN1Vzm+XrSPI28pDZc1ByP9BrbQ6EUi7gUu/LeaX7cqeDV0dvKxvqee2ZbfxxJonqGjsWtUvX2to5FDOGniW/fOipIYK3UloqLCgNhjst73uwoT3of9Ninc3GA1MeX8Kty2/zdRGwxz2VPrZs0Y3Go0MfmMwya8ms6Nwh+LjVFxMv8sg7TpTsrwkCYtPv+Y0/HXBNLU19XhLi51FO7n+p+u5Z9U9JtHP6v034mmYt11xT0uVHmTjZfBDssNxmWmpJ6ExeFNPCRll1pNqXM1/t/yXp3NmkR35hmsq/aDbPervjmwKC6if5L3MP594gh+bmxlv4fXxwI/NzTx4//2WrT6Dh4vSdmObzesZjAZ+PfYra7OUl8luzt8MgG9TGglhfcwyU+sNkxbBmFdFVZGKfeh8RbCxvTpGzojOyhJ2CxPjJwKwMc85SydHuHHpjSS8ksCnuz+1T/RrKoK2WtDXu3V8KgrJ/Ai+8oPS3+06LCUF/JvTKcgSiwitVktqairaHvbpMhqNjvWUPJ78H2HTFZDzjUvGpWKG6FmQfLGofHchssVnQYFLT6uY/SXC2jOUdDTuCMC0VMGaubD+bNefW8U6zeVQvQ8arN9c4+PEU2Jne0Vvb1GJCpar/eR5s2x/Gbl/WM629Ar2wjfcF0nTfdH8zLpn2ZJ6NmUBv1kW/XR+kHaDEF1U/nJMTRLWsntK9nSpNrWXzj39lMa9PbWeLD5/MW+c9gY+HlbSbeNOg1N+Af9+gCr69Vkqd8K2u6FKeSBxRNQIALKrLSdHKKFzTz9L91/nZ80nZz7JsTuOcctJt1g8pz2in6n6NDyUGSlTre888A64oEGs8VX6NINHhVPdHEl2tmQ1oSGzMpMJ705gwOuudWxYn7Oe/279L0+ufZJ+r/XjiTVPsD57PTf8eAMzPprRRWRUNC+2NcK2u6D4N5eOU6U7ktTRWqWkB90JtRotM5JnAPDrsV/N7mM0okx0wf41+u7i3WRXZ1NSX8KAcAf+HoxGWDYStt9j/7EqHaTfIpK/OokRiYkgoeGmiM/Ze9NeU9JNT1FQK9ZDsQGxpvvPJZVWKj1PzFxIv81h0S85wYvQuskENQ6nrEFZz1JXkVkliql8m1OdF/2yFsG+hV2Sq7VaLSkpPef8pYp+vcwMDw+Lgp/MeGCGpyfvvfuu+R3i58OULxRZ6A2OGAyIhVNdi7KsfjnIFFI30blKF3cRNQMSzuntUZyYBA2Gc4pElg9CaAE4elQ8T/Wm6Cf3cIkLiLNPdPGJgdP2wDBlGewqbiagP8TOA42XXYelporv8r1oMBgoKirCYDC4YZCWmfnRTK7ZnUa5/1rn5r/Y02D8e6LvnIp7SL1a9EjziXLpaWXRr7AQmtua+fnoz3y6+1OXXsMa2pZQYiouYLDHXPdcQOMBo1+BAXe45/wqlkm7Ac4rh4iJVnebliTsd37O/LnLdlsWnwaDgdzMXN6b+B4r7lhh8fynv3E6l/96udnXctot7kI0Ceh0Fk4QPBROetMueymVXqZ6vwjslm+1uWukXySDwgcBoseko8iiX0tLRzDRFj4ePpw3+DxuHnezXddSRb8+Sn02HHwFKpQ7iFwx4gpK7yvl/bPed+rS8jOcXt9hW3c89j5r+uhENFKJvefh9tZJ/fopOjVoPVW77ROAwEAYOzCL/qF/sttKMWqkXyR/5v/J4YrDFNcVu+z6c9PmsvDkheg0Omqaa3hy7ZNM+3Aab29/m3XZ69hZtBMQayk5ec3qvFhzAA6+CiXrXDZGFcu4RPSr3AVr50P+MsWHnNLvFMCy6NfQ0JEcYUv0s3fe/OHgDwDMTp2Nr4cDEfW2WmHpqLp8uZzERPE9uGwuQyKHoO3hzyCT6Ocfa7vSdPN1sP+FnhmYiv30u1QU5ng7VjQUGQkTjy5n6r5dDPKb4tqx2eBohWir4hLRL/ND2PsMaDtioQaDgeJi1z0H2EIV/XoRP+DyJtuZgQCXNzayZNEip68Z5htGlJ8IiO4v3a/omE35mwAIqe9Dop/RCDsfhKp9vT2SvxTx8aJ/VWMjFBXBxAQRiNyUt6nHvZQ793Cxu9JKkkCnpgX1CSImwfQf7A4GJyYKy5NdAc8z9s0J/HbsN4qKinr8PjxQdoCS1qPoDAHOVfp5hQlRylv1nO8RXHifdBb9NuVtYs6nc7hzxZ1W+2C4klj9JMZkfslFEc+55wI6P+h/I8Sd4Z7zq1hG4WL61NRTAWHv2bnSSq7OtyT6GY1GymvKmfvvucz7zzy750+j0UhhnTh5tK+qnvylaCoRgV2FVfifnP0JhfcUmuy2HcHLq6M3i9K+aooo3wLrz4dS0XOwvr7DQlQV/foYUSfD/ExRla+QEJ8Qwn2dd5rx8OgIHsq27cdjNBrteta0p9Lv0CHY0e9SNgfeT0m9lQi/0QjHPhPCvMoJwa0jFvDglPPZudPyPv6e/qaqJmuWikpoM7RR3qny4YEpD9D8aDNfnfcVQyOH4uvhy6XDL2X15asZES0qZYuLRcKFTtfRS84sISPhzMOQep1TY1RRhvy7KC115iwSFCyH2sOKjzglRYh+v+f8bnb+kj9DfXzA09P6ueydN78/+D0AZw1wMBHWIxDmbYMxrzl2vIqgOgN+mwtZX5g2JSWJ79nOFdY7jCz6xQXGWe/p19YAed9D+Z9mXlT5K6DVQkyUcPLrSbelVn0rOdU5APg2pzgv+k3/AeZs7OJKaDQaVdHv74IRiFa4bxRQbWmFAqKv3xbL9iOdkRuiKxH99AY9m/LaRb+6iZatnXqayu0is+Pgq709khOfsj9h37Ogb0ang+RksfnoURgTMwadRkdRXZHTtjr20NDaQFVTFdC10s+mXfyWW2HztaBvduv4VNyPTieEvxqf3Wwv2cy6nJ7POK1prjEFZvya+jue9FCfA83ONGtQUUzpRvhpsOiF4yI6i36TEiYR5BVEeWN5l/5q7kT+6HdKdLZEY5HaZ7K3KfrF1FfXEnGBcQyPGo4RIyuPrOzY3t7iLD/f+iVGXDGC+AnxZnv7LbttGb8/Z174KW8sp9nQZBqDWYxG+HUW7HOTKK3iHsLGi6byA25XtPuY2DFE+ytdtVims8WnErbkb+HbA9+asm7NUnMQ8r6FVvGwKIvgYWGqLVSfwyNAWLD2UksGpX39cqtziXs5jmkfTLMayLZH9Nt9uJL8sM/4vuxf6DSWyqaB5lLRr/6gGtA+UaiNu5sv9z3Kzp1Gqzlno2NGA7CtUHmlqznuW3Ufk96fRF5NR8aPRtJw/pDz2XPTHuofrueTsz9hZr+ZJlv4ve2Ouv36iWCqRSQNBKSBb6xTY1RRhiz6ORX/DRoCF9QLW2CFDAofRIx/DE1tTfyR+0e31+XxRLnWPIW8mjy2FW5DQuL0/qc7dzK1j5tzaHSiorchx7RJrvQrKNTz8oZ/c9l3l1Hf0nMtc+Sk/9iAWBrbC+jNii46Xzi7AMb9r8fGpuIAO+6HdQscPlzuX380t47qJitaiAvJqc5Bb9SjM3rj3RrjvOin9YaQ3rVqV0W/XkQCLHTp60YxEGQt4lyyFrK/UFTdMCRCiH77SmxXyVU2VTIhZio+zYkENQ2zLbr0FKFj4NTNMOKZ3h7JiU/et7DrYajPArraKvp4+DAqehTQtZeQu5GtPf08/AjwDFRW6Wc0CEuS6v1dyqdV+gDHPoF154DBdt/RzqSkQGjdZACzCxJ3c7hcZEz6GaPQGQIcF/12/x98GyUqK1Tci28stJSLXmkuQhb9iopAp/Hg1DRRdbX08FKXXcMa2VW5GDG4R/Tb+zR8GwN1mW44uYoidj4obBZtcOOYG3ls2mOMjR1r2mbL3vN4stZmUVfU4WvX1tTGgW8OkLM+x+z+ckDRszWS8GALn6utVVC91/QMoXKCoPMB/xQR4O1BZNHPWu+rzry/433O/epcPtz5oeWd+l0qAp5RJwOqtWefp7HQbqeW1cdWM+eTOdyx3DkbaqWiX05NDgW1BeTW5JpNlpCZmDCR7y78judnPW/1fOXlkN0sqruSg/oR6hNqeWedH0z9BlKutj5IlT5D9MTLWZ19HWVlEoWFlvcbEzMGcK7S79sD3/Lq5lc5VH7IrvNsa9cZx4yxsWPVHjVRsQeRRT+n7D01WmEHbAeSJJmq/X7N7G7xKY8nwsUGNT8e/BGACfETiPJ3UFHc/4KohlZxDv9U8ew0+H7TppAQ8PcHjFpe2PA8n+7+1OnKZHvo3NNPtve0KLpodKqDUl+nLlP0cHbQijcmBg7GPs4FW8N4c+ubLh6ceTIrRUwkUJ+ChMa55MGK7eIz1diz7YmORxX9epF64GNvb0X7fuzjw4KLrVihTPkazi1VlPEi9/XbX2a70i/cN5y3pv/EyXuyCAzQWs8M6ylk4SBsHHhb86dQUUTqtaLk2E/U88uiX2Z7DHhC/ATAuT4u9tLZ2rO5WaKt/VduNegtaeCUX2Gm5b5FKr1E9T7I/9HuoLAQ/YSH9+b8zQQFB1kNvriaQ+WHAAhs6y++Oyq6RM8R/dLU+cr9+CXB2YXCStVFREaKrOiWFhG4Oy3tNACWH1nusmtYoqqpihf1iSwf5Y+nvxsyLaOmQ/JF4Ke0yZCKyxn1L5j8uc3dbhp3E0/OfNJkDwYdokZhIabPyc5IkkRoaCiSJJHzew4fzfiIdU93VE3rvHXcmX0n89+bb/aaue39/Hxa4i07PXiGiN7AY/9j8z2o9DGaK0SWt0JLrne3v8ucT+Z0qTa1l9B2rUOp6Nf5edAqWm9T0FMV/fo4v82DtWfadUhjayM/Z/7s9OeunLxlSfST50w54SEhMMHq+WIDYlkwcAGTEqxb2B88CBX+GwCYmDDB+iB1fqJXffh46/up9Bm8vWGQaHtq1eLT2Uq/oxVHuer7qwC4d+K9zB9g/rP7ePR62LFD/Num6PfbqbB6tkPjU7Ef19h7AjWH4eh70KZ8rSD39Vubvbbba7LoZ67Sb03WGi765iLO++o8rlxyJbetuI0/a/9UtEb/4ZBwYnHY2tOgh73/hEzneryqIOLGx/3OJKnD4jPN9yQAthRs6bEhdRb9LNp76pthz1PCnlSlbzP5c5h/xGF3h9hY8G6NoY0W09zhbo5WCmcR/1YRFHeq0m/Xo7BiTDcXOkmSCJGzIHsAK94SKj3BmtZWNgPWHus3A2taWvjo2mst7+SpvARlSMQQInwjCPJSdkx1NUhIfaOfX85i2LcQpiwW1hMqzhOQ1uX/MiVFfD96VMSB7pxwJzeNvalLoNHdyJV+na09PT1FPxibeLijHEbFKQY/BMOetLsCMzUVAhqH4GEIpLalhmrvajSanstVOVwhKv18G50U/fpdIr5UegYXV65otWJRXlgovmanimDIjsIdVDZWEuLjvoe2g2UHAfDQBxMdaqmTuRMkni++VHqPqJkOHxoaKoKNTU3i3kw4Lj6t0WhIbPfqSZicwPTHpzPi8hFd9tF6aPGP9jd7fjnw7W1N9DNdrHfs+lScYNcjcORNWJAHvjZENYTV5s+ZPzM8arip4tle7LX3NPV3CbAyvtxvIXAgBImkxlyhVauiX1+l/02gb7TrkMmJk5GQOFxxmKK6IoetZuV5zFLHDHnOzM8V65CEIOuin1IOHYIKf2GjPCVxivWdDW2igkHlxKGxmPuHzmaVdCa7dj3DaaeZ3012z8mpzqGsocyuXpVNbU2cv/h8apprmJQwiYWnLFR87KFDotdpQACkp1vZ0WiAgfeAR18I+vw96Cz6GY1OOFbmfAW7H4WgoYoTBk7vfzorL11pdk6SRb/O/R93Fe3iwV8fZMWR7gnWMf4xXDDuAoK9g61e8+4Jd5MclMyCgQsUjbEbGi2ccRBaqhw7XqUrFdtEJVLKlaZNiYmwbx9E68cBS3qsnQXAonMWkVeTR3pYumXRr3o/7HkMjG0w/MkeG5uKAzi5NoyNhciqMyDpJjbmbqSkvoRIP/cm0c9KmcW7Z77HB/8RGQ9OOS0NvBNi5wl3lU5oNBoSjl+0uxH1ibKXefyJJzjz6af5sbnZrPC3GTjTy4vnFi4kOtrKAqe1Diq2gm8CBKRaveaUxCmU3GfbQ8BoNJJfm091tVg194l+fnVZ0FQMur7iM/oXQd8CbbXgFUZyMmg0YkFcUQEpYSk9PpyB4QO5b9J99Avu18Xa0+KDsEEPW2+FuDMgzkl/eBXXY0dSQmf69QONpCW4diKlQSv5YecPDJ8zvMeEP1n086x1QvRzagWn4hBGA+x5Qvzfj3jKJaeMjRWiSkEBDBsWy4CwARwsP8ja7LWOL1wVcLBciH7+TQNcb++pb7HbDkjFTRj0YGgWPSqsUNNcw6+Zv+Kh9eCM9DNMGbkHD8KRI91FP4PBQF5eHvHx8Wg0GmY8McP0WlVWFdvf287oa0cTnBRs9nqn9T+N8/mS3JIwy4lfeT8CRog7U53rTjQSz4eA/qBRlpAzPXk6b29/22xVgFLstfe0WenX1gDrz4XEC2HKF4Ba6dfn6X+D3YcEewczLGoYu4t383vO75w3+DyHLm3L3lOeM3OqheWxrUq/qqYqlh1ehoTERcMusrjfgYNtVPqLNglTE6daH+TvF0DFFpifJQLcKn0fr3B8vZrRG3Ts2CGcITzNPF4FeQcxPWk6wd7B1DTX2CX63bvqXnYU7SDMJ4wvz/sSD63yYOrWdrOeUaPEGt8ikgYG3aP4vCrOExYmHp1aWkTsxeF4W+J5EDwUApUnaUf4RTAndY7Z14qLodEjj/DIGECL0Wjkyu+vZGfRTnQaHdeNvo6hkUOpa6ljb8lerki9gkBP2wuV2amzTcmTDuMbpyhRSUUBB/8Nxz4W1eXtifNyXz+/qp6v9JuePB2AhgbxNwFm4i+B6TBrLfiofUdPCHK/FZVuyZafkSwREwM+rfEEN4yhyncbSw8t5apRV7lhkB2khaYR5ZHGkjLxs1MWxzFzxNdxGAwGcuUMxR5AtffsZW665RYeWriQaVotp3p78xnwC/ApMMvLm2laLQ8tXMhdd99t/US1B+HXmZBt2yJKqT3ekYojJLySwEV/DMCIoW9U+g2+T2T3+Li4q/DfGaMBvg6GTWIC9fTsCBwePdo7QxoTO4YXZr/ATeNuUtbPr/agyFYvXd8j41NxgPItUGxfoNDHR3zYh8h9/fL+wKjQhswVpIWkMTb6JPwbhgEOin6broRfZjrsZa7iAJIGCpZB7mLFtnW2kOfEI0fE95P7id5R7u41mVEqLGb9mga49jPY0AbfJ8HW2114UhWHqD0Ci/2Fi4ENvtr3Fed8dQ5Pr3vatE22FDtwoPv+RqORioqKLvOm0WgkY0kGa/+5lvVPr6dkr+UksKTgJOKrLyC89hQsupDsfRK23KQKfici0SfDoLvBW1ngeXqSCMZsL9xOdZOFUikb2GPv2apvpbiuGLBS6SdphPtH/5sAqKsT/VdBFf3+ashi2fpsx5/1bYl+8pyZWyOCMbZEv8LaQi759hJuXX6rxX0MBtiWtwu9to4AjyCGRA6xPsjAgRB2kir4nUhotHicc5CVhU/S2AjbrbTAWnPlGpb8YwkpIcqTav/M/5M3trwBwKfnfEp8oH2Tm9zPb/Rouw5T6QF0OiH8gZN9/QIHQPxZ4Bns0OFZVVl8uvtT088bGt/jt2Fp1HmJ5ENJklh48kL+MfQfHLjlAP89/b/cPO5m7p98Px/M/4AwQ5jpWfOP3D/4I/cPDpcfpqqpijaDGf95R6ncDU3O/EepdCH9Vpi+FDQdWQrys5OuRPQQz6zMpLyhvEeHJf8tBATQvaeazg8ip6mubycKOx+E3Y85dKjcYiWyUlhZ95TFp3z/BQWZT+BRRH2OxTiU0WikUmn2owtQRb8+wF133012Xh7THnmEVwcN4rKAWO71H0TMeY+QnZdnW/ADCEiHMa+JSic7sBZA/z1H2JD4EYGEpncr/eqyOnr5eahVfi5F0kDKVV0sxo7v67f88HIu+uYi3tv+Xo8PT5HoFzQYzi4QfdNU+ia/XyiqMe1E9PWbTJA2iqAetpt5fMbjLDljM1HVZ+Dt7cSHvrFNtb7raaZ9D6ftc5kQMaQ9Rrdvn/h+14S72H3jbp6b9ZxLzm+J3UXiggHNAwlw5UdfcxmEjFSzJPsCvgkQOVPR4nVu2lxABADLGkQKojXRzxxlB8r48uwvaaxo5LJfLiN1jnV3CDk4bvEZcNybMP5dZRdXOaGJC4wjNSQVg9HAhtwNDp3Dnkq/oroijBjx0HgQ4Wch1VbrLSocooQg+eGHon9VQkJHIFWlj1G+BVZOFNnfdmAS/XIcF/3k5BlL9p4ypp5+Nuw9vXXegLBetERODtQYC/BsC2dK0mQ0tizIRy6Eqd9Y30elzyFJMLW9iHO9i3NQg7yCOLnfyVw2/DLTc4BSqqo6knht9vPbcT/8PAVaax0ap4pjyJUkTol+Mnb09JMprC1k4nsTufy7y/ls92dc9d01bIq8FoOmmULjbtN+8/rP4/NzPyct1PLz6q+Zv3LyRycz+f3JpL+eTsjzIXg85YHHUx7EvxzPhpwNGIwGh94aABsvFT2yVFxD2DiIO008S7Uj93GsKgqmf6hwO+qJar+Msgze3vY2v+f8TnFx17F0oalUOKSonBic9A5MW+LQoVqtiAVGVQnRb9XRVTS22mcPbw9Go5H3d7zPsgOrMUitXeyN7aK1Fn4aCBuvcOn4HEUV/foI0dHRPPLoo2zZv59n/53PuBn7mTrtUeuWnp3xCIABt4sgngLe3Pomya8m8+AvD1rcR15UJUniCbbXKv2MRtHwffkoUZWm4nrGvQED7zL92LmvH8C+0n18sfcLlh5e2iPD2Veyj9zqXPQGvTLRD8AnRnyp9E1GPgsjlPefkElNhbDaGTygzeehEQ+5YWDWke8/h+e/iR/BrHUuG4+KQnzjXJolL4t+ubkiWNg/rD/DoobZDt45yZ4SsdiOMo6wbslkLz7RMHM5DLH8DKDSQ2i9YOayLv00LBEfGM/wqOEYMbLq6CoABos2ZmRni549togYHMH89+dzxptnkHJKCloPy38n725/jwPGJbRp6i2LfmFjRb8ClROTX0+GDcp7zsrVfmuzHLP4tKenn2ztGRMQY3mu7RT42bsXVq4U/77lFrX4tM+i8YD6LGixL8tZ7ju1q3gXNc01Dl1avv8sVfrJJAQm0C+4H0lBSVb38/EQJQiNrY0WE2kPHoSo6jO5x1DConM/s3fIKicK1ftZkPIYcQEH2bxZ9Nq1hNFo5OejP9Oib1F06gHhA/jlsl9464y37B6WXHWYmqrAOrKtARqLQGe+z6+Ke5ADy06LfqvnwI/97T4s2j+aswachREjl353KR/ufh+MGoYUP8MVYy6w61zF9cWMjB5JcnAyfh4dvcjbDG3k1+Zz5fdXOif6pd8Gg9W1i0sxGrs4EkVEdFjOjog4CY2kIbMy0+3DWHZ4GTf8dAOvbX7NbE9JE6tnw9JBbh+PiouImg7BNhwOrDB9OgQ2jiDAkEBDawOrj6124eC6UtpQyjU/XMMd22ZhRG9edFaCvlEU1Zix9uwN1J5+vYw5q83Y9sT7ggL3Xju7Opu9pXstvi6LfjEtU6mkF3v6GVoh6UJAElVpKm5HrvSTRb8J8RMA2Ji3EaPRqNgi1lHmfTaP3JpcNl6zkZoacW2rol/Rr6JxtWr72ndJutChw1JTQUJDVpaR6Ohot997Mi36FjSShpoa8THpVNKDGnnsHUo3Ql0m9FMe0LZEQIDonZadLar9Jk1ywfhsUNNcQ06tWGQleg13/wVVTgjmpc1jd/Fulh9ZzsXDLiY4WNggFxZCRkbXTH5JkszOm6OuGmXzOm2GNm786Qb0yXpmVecTFOTXfafWOmEJpPaGPHExGgHlNsjTk6fz/s73He7rJ9t71tZCW5uwNrNE/9D+LD5/MXprGd0bL4XSDbTOO8Lrr4v7cO7cjkQNlT5IyEg4p9Duw+IC4xgRNYIo/yjKGsoI9LLfc11+lrMk+slz5jfDv1HUP1qu9DNipNXQiqeZufCQcOlmQLpEsHew9ROWb4XMDyDtBghRP/dPKGoOEVb4FONTk/l25wC2boUpU8zvevaXZ/P9we9598x3uWb0NRZP2XnNLUmSSWS2B9na02aVH8C41+0+v4pzFBUVsWnjO2xZs4jt62t49pFAzrr4Yq697jrlyf8yEVNE0qNBb1fioyRJvHHaGxTXF7MkYwnBnuEM2PM5o4NnoVUQeuv8rHnxsIu5eNjFptea2pqob6mnsa2RhtYGEgIT0GmcCEGnXef4sSrdaSyCnwZAvyth7GuAeC6LiBAi9K39/8UH57yJv6f7EwEOlx8GID00neJjYptZ0SX2NDUmfKLR1iASvRzoxTl1Krz3nkRi7v384yI9o2Jsr2EdRRa3g7VxaI3ejvfz844URTUWkCSJKIcVRftR/1p6GXMLCln0y8+382Q7H4RvIkSjTBsMjhCp4ftL95t9vaiuiCMVR5CQCKwW0c1eq/TTesLQR2HoI700gL8BxWth3QKoECuDfv3E5tJSEZgZEzMGnUZHUV0R2dXZbh2KwWigsE4EA+IC4mxX+rXWiowfB6wjVXoBO3usyVWnhYUSgUGRHChT6GHnJF/u/RLfZ3x5dNvlgIP9/A68CAdecllfORU72f0IbLnBZf0Uj7f43Ji7kUu+vYT/W/1/Ljn/8bQZ2rg8+RHiyy4nMsCFPnWlG2H1qVD2p+vOqeIcBSuFBXJdls1d56WJqrqfj/5sqiwZOFC8drzFp0ajITo6WlHwutuQagvQG/VIBg+CtNF4e5vZKeMl0Y+wap/d51fpI8z6DSYvUrz79KTp+Hn4Ee4b7lC2vr9/h9Bnq9oqzDeM8wafx4VDrSQN+adA8HAWf+NJfr6o5LrySruHpXKCsOOGHay8dKVd/dA6I69lGxtFFcPx2DtnyqIfWLb4zDikx4iRAQMUnLB8Mxz+LzSXKrq+Sh8iaibM24kuTQge1iw+pyVNA+DZ35+12u/swV8e5LZlt1HRqKA02gwGA+zYIf6tSPRT6VFeeeklkuLjqf5pIS/VZfBxdQF3ZmSwfuFCkuLjeeXll+074bDHYMIHDjmdaDVavjj3C7467yveGLqbiNpZiqtcrM2b3jpvwnzDiA+MJz0s3SHh2oSCGKeKnXhFQOi4bi0GZL1ZXx3TI4IfwOEKIfr1D+tvvdJv5EIY8bSZF1T6JPoW+CYMttzs0OGhoTB8OPQrvZX0yjuIDXBfa5KjFaLiJdggKmAcsvc0tNqM/Wk0GlX0+zuh13fPXo1rF8CLi0UWrGI8Q0Vvs5Yqm7sOiRDRy6yqLOpa6rq9LjdJHx41nObqYKCXKv0ai6DNfb69Ku20VkH+T6IqBvDzE5UDIKr9fDx8OCnuJAC+PWBfDw57Kakvoc3QhoREtH+0MnvPcf+D1GvdOi4VJ6nYDt/FwiHLWS/mCAoSfXmapUr6vZrIyLdGmnpZuZPDFYdpNbRCm49pHHZz6HU49pFa6ddbDHsCZizDVY86Q4eK73vbC+RL6ktYtGcRi/cvdsn5jyfUJ5QLI55mZNZHrk26qd4DJb+pWZJ9iYZcyFkMtYds7jo+fjweGg+K64vJqsoCOiw+jxf99Ho9R48eNfusaYvsKpHg49OSQEiwhXslIB1i5grhReVvQVJwEpUPVPLTxT85ZG8sSR3rCSUWnzYZ8Qz5/X9icfs0fP314hlWpY9T9Cscecfuw5x1e/D1BY/2FsvmRGd750wvrZfp3+b6zDQ1wR91n/Dr8ASWVD5p+4T9b4YF+RDeA3YCKq7FMwhCRjB5qhCCt24V4rI5bhhzA2E+YRytPMqXe780u8/ekr28vOllXt/yOlvyHeundfiwSN7188O26Fy5UyQr1uc4dC0V+3jlpZd49pFHWKfX83NLE5cApwCXACubmlin1/Psww/bL/w5gZfOi/OHnI++SgSBlAa8nXnWVIzRKHpk/XG5+67xd0SjhVN+gQG3ddksi35FRR3bLFlYuwqT6Bfa33pPP5UTC62neLaJOdXhU8yYIb6vWePeXHq50s+nyQnR7/D/4Md0qwmxer2ezEz3W+bKqBGfPkhwMHh7ixu680Rrk8H3w6y1iiwOw3zDTCq5ub4cv+f8DsDkhCmUtcfXZUueHmXb7fBDMjS7IjKgYpHY0+DCRkg837RJtviU56MrRohGpG9ve9utH/pyEDMuMA4PrYdt0c8jAPrfALGOf5Co9ADeUeATDx72l8ylpICHPhhPfTBthja+3v+1GwbYFfnBM8QgeiM4VOl3+j6Y4v6xqlggcpr4clFvP7nS79gx0TttWtI0JCQOlh+koNY9ftzV1eK7S0W/tOvhnFIIVdO++wzJF8MF9Yq8/7113iZrky0FIhA4qL21xcGDcHzMpba21qEh5VSLwJ9PS5LlpK/ki2D6D6BzInNbpXepy4T9/7KrWtND6+HUJeW+apU2Wrp9d+A7vtn/DSX11hsdLVsmkiRHj4bJk50amkpPcfA12HpLl56M9lBcV0yr3v4qfkmybfH55f4vSXwtkduW3WZ+hy7nk0zCn7lKv8OHodxvPU2e+UgeVpq8dR6gb6w6p56otNXTL3QfcXGiknTzZvO7+Xn6cffEuwF4Zv0z3aqm9QY91/5wLW2GNhYMXMCpaY6tcf9sN3QYORK0th6FC1fCjvugIc+ha6kop6ioiAcfeIAfm5sZb2Gf8cCPzc08eP/9FCkNCDaXw583QOZHTo3PEcHF0WdNxbTWCGtov0T3XkcF6BD9iovhi71fMObtMbyw4QW3Xa+prYnc6lxAVPpZvAezvmh3RjnmtrGouIHRL0G6Y5V+ABMnioStI4WlPL/qPb7Y+4ULB9fB0UpR6aerEcmsDol+cksy/35W96qr61545S5U0a8PIkkd1X52W3zawfmDhcDz/s73u712RvoZ3DLuFqZHzzf13AgPd99YLBI5A+IXgFdvKI5/IzQe4qsTsq2i3NfvoqEX4efhx8HygyZR2B3Ift79Q4XYImeCWwx6OxgwUOlhfONg7p+QYn+GnixApzYsAODzvZ+7cGDmOVQuKm4CWpwQ/XR+EJjuwlGp2I3R4LIARmiosN82GmH/fgjxCTGJL78d+80l1+jMprxN5FQUYsTo2P1nDc8gtQK1L6HztSvI+9/T/kvm7Zmm57jERJHJ39wsRGlXIFt5+7Qk9l5PZxX3U3MYdt4PpevsPtRRyzk5idCW6PfYmsc4b/F5bC/cbn6H8i0Y/rydo9tE+fX8+eq0dsIw9DE4xbHPzakfTCX6pWg251tQVGwgi85yUs3xFDQUUFBbQF2rsoDMRws+YvH5iwnz7W7DffAgVASINdOURAsN3jpTsh4a3Lj4V3EvGy9HWj6c6VOFd+w6K9PqLeNuIcgriANlB/juwHddXnv9z9fZnL+ZQK9AXp/nWJ89o7HDYnTiRAUHpN0oksdDRjp0PRXlvPvOO8z08LAo+MmMB2Z4evLeu+8qO7HWB468DcXOrUmsWiv2Fp5BMO071dbRHeQvhU1Xdymy6FzpV9VUxfbC7XyX8Z2FEzhPZmWmWO96BeJrjKC+Xmzvdg9W7xHOKBqvbudQ+evi5wfjxkFJ0FIe2nQtL290TwW0XOmnrRHBR4d6+g24Dc7IEGv7PoIq+vVRZNGvwJ7igbZG2PecmAgVcM0o0Tj6h4M/dMuinZ06m9dPe52BHiLrPCpKQYaYO0i/GU56qxcu/DekfCvkLzP9KAstsugX4BXARUMvYkL8BIf6uCilc2m/0djx4Gm2l7XRCD/0g41Xum08Kr2PLEAH5Z0HCPthORvMHRiNRpP47FknRD+7K62KfoWagy4emYrd/DoTVox1mRfE8X39Tk4+GYDfslwr+hmMBmZ/MpsHy2Kp8z7geCPp48n5WthOtDW46IQqLqNqLxSuUrTrmNgx9AvpZ7K6kyTLff0cpXOlnxwk70L1ftELuGi1ay6o0juET4DZGyDpIsWHVDZWMvD1gUS/GE19S73dl5TvJ2v2nkajkWOVQsHuF2whW7b0DzRH/oPUWkZwsKhmUTlBCBsLEZMdqsSP9BNRQLkVhb3YqvQrahJVNQmBCYrOd+HQCzlv8Hlm+x5tPVBCvfchJCQmJdiw7NQ3wy/TYdsdiq6r0gdJvACGPsbUSaL32I4dmBxrjifIO4jbThLVpE+sfcLkopNVlcUjqx8B4IVZLxAXGOfQUA4fhsJC8PKC8bbUJRCiSuS0PhWo/Kvy/aJFXNakoPIXuLyxkSWLFPbd1fnCOSWir58T9EnRT8V9VO2CzA+g7qhpU2fR76wBZwGwOX+z25xtOif9l5aKtU1gIN37iY94Rjij+MS4ZRwqbqKpDH6bC/sWOnyKGTMgokZUvW8t2Eppvet7H8uVfn7Nqfj5OdEuoI9lIKqiXy9jqTdBbHt/SrtEP40H7H4UspQ9GAyLGsY5g87hoSkPWezLIV8/pqfnVaNRNMFU6Tm23wmbrjD9KIt+BQWiJwXAG6e/wcZrNjI9ebrbhiGLfmmhaZSXC8smjUb0detGWz0EDxdVZCp9n4LloomvnYKDfC+2lQ9masI0jBj5cp/5HhiuoKS+hNqWWiQkWorExc2KzpYwGmHjFbD+XPcMUEU58WdDv8vA4Jrm78f39ZvZbybgetHvWOUx6lrq0Bg98WtKJz7eRSc+9Lqwb5J0LjqhisvYchNs+IfDArXc12///o5tkiSRkJDgUB+szpV+8hzchZpDohdwi2q/fkLjGQQRk8AzWPEhwd7B1LfW02poZWPeRrsvqcTes6yhjPpWISgmBSeZ32nAbfw3P5tD5eOZMaOXkhNVHMfQ5lDf9qmJUwFYn+OY6CdXLpsT/SRJotooSgCVin6WMBphQ+4GAPoHDSXEx1z2ROcD9DDm35BypVPXVelFki6EYY8T3y+A1FSxhrVWpHXHhDvw8/BjbOxYjBgxGo3c+NON1LfWMy1pGteNuc7hoaxt794yfryZwPnxGNpE1bfqntMjVNfUoHRZGQVUWypLNod3hFMB59bWjoQcpfaezjxrKqIhD36ZCQUr3HP+vzv9b4bzKiBsnGmTHPeoqIAwrxgmxE8A4PuM790yhJn9ZrLuynW8OOdFk+hs8f7T+fQ5UUXFBp4hULEVGgsdPsXYsRDmGUNgwwiMGFl1VFmirD18ds5nPDLsHfyaBtif9GA0wurZomWCDSRJIt5lwR3bqKJfL6PRmP8VyKKfXfaeGh2cugXGK7QAAL654Bv+OfOfhPsK786iuiLOWHQGu4t3AyJDDHpB9KvLhK9D4dAbPXzhvzGDHxZVle0Bx6AgIbQZjR12YZ5aT7cP48IhF3LvxHuZljTN9KEfHm4hmOPhDzN+Elk/Kn2fso2iyqj2sF2HRURAQICERuPJnFhRjeBOi09ZeE4KSqK8RNhH2DcHGmHsf2DYEy4fm4qdDLwTRv0LtLYiHsqQRb8jR0QyxNTEqWglLZmVmRytOGr9YDuQP4P9G4egQec60W/aEpi5SjTVVulbDLofxv0PUCb6vbX1Lc796lyT9aHc1+/AgQ7dUKPREBYWZvFZ0xrPzHiek7K/IrzmFJOg2IWEBSLbNu5Mu8+t0scwtEK98up5SZKYkTwDMN8X3BayvafcM9wcx6rEg2dsQCzeOvPzd129hp//SKRF78PJJ9s9DJXepHI3fOkNGfZbNMmi34bcDegdECmsiX4ajYbiRtFMKCFImej327Hf+GrfVxTVde27lZsL+VohTM5MVWDtqfOFAbdC3BmKrqvSt7npJhGX/u23jt56xxPuG84X533BvRPvRSNpyKrKYkvBFry0Xrx9xtsWk7JtYTB0WIvOmKHggNpD8FO6SB5XcTtBgYEo7NJHMRBkj91MU4kQx1rsEAo7UVYmniE9PZW3tnDmWVMRlbuhfLPo66fiejyDhSjTCX9/8G0v+i0uhrMHng3AkoNL3DKEQK9ApiZNZUbyDFM/v26iS1s9ZH2u9vM7EdFo4exiER9zEA8PmDQJIqrnAbDiqOuTAGYkz2CS97V4GALtF/2aiqAmQ1FbGY1GQ2hoz7UvU0W/XkavN79YcUj0AwgdBV7mSqKU8cAvD7D08FJu+OkGjEZj74l+rdUQMgp81AquHiPuNEg4p0vmzPF9/WQqGiv4cOeHJisSV7Jg4AL+NedfjI8fb/rQt6vKSqXvkn4rLMgX1Zl2IEmQnGygrq6Ofk1no9Po2F643dR3z9X4efjxj6H/YGb8Gej1YuFjttLU4oA1kHA2JJ7nlvGp9B4RESIJQa+HjAxhezw9eTqjY0ZT2uA6m4ldxbsACGwYgZ+fA/aylvAMFlU9Kn2P+DNFlYDCIN/Sw0v59sC3rMsWkb30dJEcU14Ope23ol6vJyMjw+KzpjX864cTWXo+UV7JlkVnrZf4UjmxWTsffhpgV5Xp9CTh+LA2237RT4mbiU1rT2Dfb2vx1xWTnAz9LO+m0hfxjYe4+RCQZvehI6JH4O/pT01zDXtK9th9vDXRT6/Xk10pqpyVVvrds+oeLvz6QnYW7eyyfd8+KA9YA8CMftPsHqfKCYi+CVafCjsfZsAAWLBAbH7jDaiz0CLyjPQzGBIpvOP7hfRj/837+fK8LxkQPsDhYezeLe7vgAAYNUrBATo/GHg3RJ/i8DVVlHPWxRfzic3yS8HHPj4suPhi5SfP/gLWzBNVNQ7Q2dpTaTGVM8+aiog7Dc4tg/iz3HP+vztGo7Dsr9pr2iRJXS0+FwxcAMDqY6upaqpy63AsVvpVH4A/LlbsaqfSx3DAzv14hgyByOq5AKw8stItLacctjf2iYGzshUVo+j1eg4dck8c0xyq6NdH0WqLOHzoKVZ+O4j+sXGMGzSIp596iqIiG3lB+mZhuaRX5hMO0Kpv5fuM77npp5v4eNfHSEi8Nvc1JEnqPdEvdDTMXicyyVV6lk5Bn+P7+gE0tzWT9u80rvr+Kjbnb3brUGTRz2J5/477IOM1t45BxYV4R4JvrEOWDCNGgMGg5+juCF499VXWXbmOtFD7g0VKGBUzis/P/ZxbU0Q2UnS0nUNurXXLuFQcZMcDIgjjAiQJhg0T/5b7+v100U9su36byfrEFciVfoGNw4mLc5GLSflWRdlnKr2MQuFFvt825W0CRN8eOVGnc1+/JoV9Y45HtgkdONDC/Zf3A1Rsd+jcKn2MhHOEvZMdNsiy6Lc5fzONrfZZNMp9y4uKhP2dOeRKv34hFtS8lirG187gyhEPqFV+JyJeoTDtW5HoYCc6jc7UH8+Rvn7WevrVtdRR015NorTST65EbWrrOtfu3WskvHYWSV6jTJWxVtl2JywfI6oZVE5MtN5Qsx+aRADl0kvFfFdRYd3mszNR/lGcNdA5cUO29pw8GXRK3Nz9kmD0SxA9y6nrqijj2uuu47fWVmxFUTYDa1pauObaa5WfPHq2cI0IdEw0tmmtaAFHnzUVo/NVk8zcyYqxsOPeLps6i37pYekMCh9Em6GNZYeXufzy/1z7T97Z9g41zTWW43/+/WDyFxC/wOXXV+kBmsuFi1/J7w6fon9/CKmfiE4fQGlDKTsKd7hseOuy1/H+jvfZUyQW0Q71NJU0wolOAW6fMzuhin59kFdeeokhA+OJPbqQl+oyeLOwgDszMli/cCFJ8fG88rIVK5SDr4ps3cpdiq+3MW8jC75cwJvb3gTgmlHXcFLcSRiNvWjvqdLzVO2Fb6PgwIumTbLol5nZsZuXzoszBwg7r//86XiJtjmK6opYn72eknrxxClr3GYfPI0GOPwmFCx16RhU3Ex9TpdMMqWMGSMC4bt2SVw38hamJk112PZGKQ7Nf0YjLB0Ca1R7pj5DYwE05DjUO8gcQ0RCtqmvn4+Hj0vO2xmT6NcwwnXWnpuvhhXjHO4Zp+JmGotgSVK3Rbcljhf9oKOv33Yntbi8mjze2/tvSgNXmbf2NBphw0Ww8yHnLqTSN0i7Dka/aJcNclpoGjH+MbToW7rcg0oIDRU9pgyGjue847FV6VdULPH2ttdYl3MR093XZlqlj+JMX7+ICPHd3L1X21zLqLBRDIscRqCXMm87+RngeNFv/36JwXkv8P287UT7K7EskcDQIqquVE5czsqBCR8AwinkjjtE4syvvzr/2ayElhb44w/xb3Vu7JtER0fz3PPPc6aXl0XhbzNwppcXz73wAtH2WB4FDYL+N4pqagewaK3YW5SsFwnezWr/aLchSaI6KbWruCzH3+R74rLhl/GPof8gKchCn2UHqW+p5/E1j3P9T9fTqm+1fA96hYlEoeAhLr2+Sg/RWgtbbxXVyA4SHw++Xp6E1ZyChMSOIteJfp/u/pRrfriGP2pECyG75sCS9bD9HrtaJfQkqujXx3jlpZd49pFHWKfXs1rfxCXAKcAlwMqmJtbp9Tz78MOWhb+IaaIvjB0Wn1MTp5IaItSdEO8Qnp31LACVldDcLD4H7M32cYrK3fD7hVBmwQBfxT14R4F/apd7R64ayM4WjZ1l7hh/BwBf7P2CIxVHXDaEFUdWMO3DaVz8jbCxsFrpJ2ng3FKY+LHLrq/SA6wcD5uvsfuwfv0gMFBPc3OH2OIuCmoL0Bv0JtFPtiJThL5J2ON0aoat0stM+ADOOCAaf7sAuXfa4cPC5lOmprmG/aX7nT5/bXMtRytFeXVA4zBTVYxTGI0w+CEY/qTa/Lyv4hUhPn99lE0442LHISGRXZ1NYa2YrCZPFq9t2AD1ThSLbC3Yxk/6O8iIe9SC6GeASZ/AIGUCpcpfjy59/ey0+JSkjmo/S20M7pl0D4vPX8z5g883+/qv64L48dDtSLGn0oNtMVRcydH34fcLxHxiJ3PT5nL96Ou5eJgdtnftJCaK7yUl0HhcLlBMQAzvT3mfHdcrDyTJlX6dK15LSkRvLK0WBigtuBnzCpxuv12pSh/juGesQYNgnmhBZKrAcydbt0JDg7CiH6IkNm7Qw7KRsG+hu4em0om77rmHhxYuZJpWy6ne3nwG/AJ8Cpzi4c00rZaHFi7krrvvduwCDib4OWxt5y4yP4Ttd0Kb6qLjVgbe1a0tSedKP4CHpj7E5+d+zuTEyS69tBxLDPUJJcw3zHK1qcGCNYTKiYFfEsxcCUP/z+FTaDSQlgaDc19i8bhSrh1tRxW0DQ5XHAZAW90f6EgQU0Tud6JHtR1uiz2JKvr1Mp0b3hYVFfHgAw/wY3Mz4y3sPx74sbmZB++/37zVZ8REGPW8XT0SJEnioSkPoZE0vDb3NcJ9w4GOKpfISIXWEK6iYhvkLFY/3Hsa7wiY8wekXm3aFBEh+gHo9ZCT07Hr6JjRnNb/NAxGA8/9/pzLhnC4XEy2/UPFZGvT3lPrDT5qw78TisEPQtqNdh+m1WqYNs0HSRIL2qyqLG5eejNXLLnCpcMzGo30/09/fBf6si8/C7Czp6TOR4hMwx536bhUnEDj2g+w+Hjw8RFJMfK8uOLICiL/Fcnl313u9Pk1koYPzvqA8c2P4NUW4ZpKP0mC5Isg7XoXnEzFLWi0MG87DLpH0e4BXgEMjRwKYLLaHjgQEhLEvbl+vXjGTElJ6fKsqYQ9uaKnlV9rImnmHic1WmEJGTPbrvOq9FGaSmHdOcJ2xw7OGXQO142+jimJU+y+pC3RLz0snfMGn8ewqGHdXjMaYfVq8W/V2vMEpmI75H4DTSV2Hzo2dixvnfmWqc+QPQQEYBKKO69twLE505y95759UOH/O4mpDShs3aXyV6HumHCiqcsybRrXngfo7hY+RiOsWiX+PW2awhyvpmLQN0C7ra1Kz3HX3XeTnZfHtEce4dVBg7g6NJZ7/QfBlEfIzstzXPD7/QJYNtShQx0R/Rx91lTEmNfg5J+FYKDSoxwv+rkLWWzpH9qf+vqOpMVuosuK0fCLWr58wiJJEDMHfJyrJurfH/xaUijOUl7kpIRD5e0f0OXpgJ2JD6Nfgnk7ILC/ot01Gg3Jycn2DdAJVNGvl5E6PY29+847zPTwsCj4yYwHZnh68p5Sc3gFXDP6GpofbeayEZeZtskTfI9be6ZeJSq4Iqb28IVVjkeSOqr9Olt8Ajw69VEAPtr1ETnVx62aHcT0oR/Wn7Y2kSULFkS/6v0iYKBm/ZxYDLxD/I3biSRJTJ3qA0hs3Qr1LQ38b+v/+Gz3ZxTUFrhseDnVOTS0NmAwGqgvElFJuyr9VPomJetg290usbbUaMQDJ8DBg+L7mJgxtBpa2Va4jczKTMsHK8DP048rRlxJ0tGnAVwj+vXRzDMV5zje4lOS4NT29pUrV4p5MzAwsMuzphL2ZIvP9Fi/JDw9zeygWsT+tdD5Q/6PUH3A9r6dOG/webx95tvMSrG/D5Qs+uU50GZ03z44P+l6Xp47ngnj9bYPUOmbjFwIFzT2SvKeXO13vOgH2D1nmhP9Nu4p4o+BU3kzKIzaZgVJrLVHYe/TULVP8XVV+igV/8/eXYdHdaUPHP/emXiIkpDggeDuWigUSr1Q31Lb/uq2tV3q7t3qduu2dRdKlUKhuBe34AkhkBD3ZOb+/jhMhMwk45b38zw80Mm955x5Mj1z733Ped+1sPoGyK2vW9RLPUMkK8u1XfjNMZngxRdh7Vp1LWD3goioDnDWThgsO/18ITU1lfvuv5/VW7cyf8VBRk7aSruU+0lJcWFejEiBqK5O7aJuccG1Fc5ea9oltI3UmvSGzO/gx76Qu7TupYZBv4aX/TvydvDwwofR3XQvULfov23Pus9fXBxNF8wkDIX4wW7pU/iIrkN5lku1iy3fpxkZliZd/xyWVZfVPU+MruxJeDjE2pfhXdE0SBjiwOFqzvQWCfr5mKlBbrDZn37KZXYWdLy8ooLvP/3U+g+XXQYr/s/6z5oRctxuiOxjz9F9Us8vvC0YrT1pEh6V9QOsvhHM9bk8LXX9dh2XxXNs57FM6TaFWnMtzy591i3dN1zpk5urvhfCwiAhwcrBW5+FX4fLjtBWwmQyYTRuxmjUOXQI4mv6cUKXEzDpJt5d967b+tmSqx649G7bm9ycUMDBOfDPs2FbM3VXhW9k/QA7XoTCjW5pzpKuy7JqOzk6mclpkwH4astXLrdfWKjSM2maG76DdR1+7KNW/gr/VroP1twKh3636/AxncYQZgyjuKp+hf7kySo7w65dkJFhYtOmTY2uNe2Rkat2+vVM7mL9gM2PwrepUJzhULvCT4VEwkXlMPK/XuvSspjB2k6/o+VHeWH5C8zZMcfquWqXn0ZMG52wcKPHxig8LDTWpXu9GlMNK7NW8uuuXx0+t+uxTSP79zd+/bo515HyTApvr3nb7rYij6UOr2hQN/iPvQsA6BbTl5jwmJYbOboSNj4ARRL0C3jtJsKkX6D9qXUvxcbWP0DP8MDXZk0NPPMMLFigFqb985/1n3G7ebhOumhZt27quUdpqXMLYuqMeAUm/+zw77S2Fo4eVf92ZJeLyeTctWaL8v9qtGNWeJAxSgWJa+qfq7Vrp+5Dq6qgqEi9VlJVwoi3R/DIn48wd/dct3Td8PmfzdSeAGM/gBH/cUufwkd2vwvfd4bDC51uwrLwelXePCa8N5EbfrrB5WFZUszGh7UlzJRY99m3S+Z3kDPPof5MJhNbtnjvek++3f1IUXEx9q7pSQGKLLPv8UoyoHS3y+OxpPf0atCvcAvs/QSqC73YqaiTuxgyXoey+rtgS9Dv+J1+APdPvJ8QQwhm3ezyKgtd162u9LE56Xa7HIY8A2HWIoLCb+Wvg1+Gwd6PHT41NLSW/v3V52zNGrhu+HUAvPPXO5jM7rnR2HJEfQH3iBtAba16eJ6UZOfJlXmQt0zNwcK/9L4VzsqABPesELQE/Sw7/QAu7K+Cal9tdS3o99WWr5i9fhEmrYKUFAgNdak5lbap7RiI7eNiQ8LjzDWw8z923zxcPOBiiu8u5rUzXqt7LTYWxo5V/547V3PqIUx2mdr+MqSbjaeGESkQ3c3lFC3Cjxicm2gsgZflmcsdOq+59J5bcrdw59w7ue2325r8rLoaliyBV1e/Sd5wqf0d0EzVkLtcZe5wwtzdcxnz7hhu+eUWh8+1FfTLyM/gSOURwkPCm2+gdJ9aYFtbxhWDr+C9s9/jrF5nAerhaEaNCvpN6znZvgF1OgdOWQUpkq824EW0gw6nQkTjmwfL7gR3p/g0m+Hxx2H5cnW9eN99KrWn3ba/DFnWF1gI7woJgQ4dcsjY+RinjO9Lr44dGdm3L48/9pj1sj5udvSoWicYGgrx8Y6d6/aAH8DaW1WaUlO1+9sWjXU4Bc7aoeauYxo+A7F8/GLCY7h2mCoV8eDCB92yy6ph0K/h8z8RhJJGQ88bIdL5AEO7dsfKT5l1lmQu5ptt31BVW+XSsCypPduH9azrw27r7oQ1/3C4T4/MmTZI0M+PxMXGYu/X+WEgLi7O+g+nLYeprleKtgT9vJrabt8nsPxStwQthRP6/BPOyYY26XUvWYJ+e/eqG4uGTux6Ivtv289rZ7zmckqHnNIcymrKMGgGuid0bzm9ROpJ0G+WS30KHwiJgapcp7f1jxihLi5Xr1apxRIiEjhQdIDfdv/mluFtzt0MQMfQ/oD6/Bnt3UgQkQTnHoGh/3bLWIQbRXd2qNZtSywPbzIz1Y48gHP6nINRM7qU4rPGVMOVs6/kykUnUh6+xz2pPUOi4YTPYdCjbmhMeFRMDzhrFwyxr1ZuZGik1YfTlhSfixZBdbVj380lJVCEehI+uq+NnX49r4dTlqudOiI4lOxSi3FqHMue8Pa6txnz7hgeXPigQ+dZgn5FRU3T3e0t2AtAt/huTc5bsQIqKtQNef/+DnUp/E1tCfw+DrY959Tp47uMR0NjV/4uckodeyBuK73n3kLbn71GDi+APe9DzjwmdJ3AlUOvrKs/uWUL5MWoopOn9rIziBcSCW1HNgkUiQCl61CV3+glTwX9Nm+GdesgPBwefhhGjXLgZFM1rL8LMl5r+VjhcS8+/zxvvtaJDrue5L7M7byenc1t27ez+Mkn6dqpEy++YGcmmeoC2PgwZH7vUP8tLrj2tj53wOCnJPuXD1mr6zdr/CyiQqNYdXAVP2f87HIf1hb9N3n+l/k9rL0NKg653J/wofiBMPJVSBzmdBOapr5P2xZPJjG0PXnleczeMdulYVkCz201FfRrUk+yOeM+gmEvutS/p0nQz49MnzmTj6xU+84BHgdGAr2O/f1wSAhTzjrLekNu+JbW9fqgX6o3Sy30uhnGfgjxQ7zYqagTmaJWXjT4DHXooHJqV1U1XZGtaRodYtwTFbZMtl3juhJmDHMqp7wIALE9YUYm9LzOqdOHD1d/b9kCek0EVwy+AoA3177pluFZdvol1qqniQ7vdNYMqv6A8D+mKjj4IxTvaPnYFiQkqAtCXa9PfZwcnczkbmpV/+ebP3eq3VUHV1FWU0YbQ1vaVPatezDuEqm/Fjg0DWLSnbqOqzHVp+UeNEh9d5aVaaxfH+1QOxu2VFIVqr6AB3R0ND+YCFgHvobllzm862piV7WlZOmBpVQ7sBo/IgISE9W/j09j1lzg5Y8/oF30Pm456Sk0SYUY2MLbwtDnoPvfnTo9PiK+LtC2aP8ih861BP3y89VCB4BqUzWZxZkAdE/o3nwDsb2g60yIbHoPtHjjAcojdmPAyISudtanL9wku1mCyaIZKoVZg+uvhkE/d16WbVZrFRkzRn33O8QQAqett3uhkfCcF59/nqfuu4/FZhN/mCu5BJgCXAL8VlnJIpOJp+69187AnwE2PwJZ3zk0BktqRb/ZZdV5BvR2fCe3cNLBn2BL49qe1oJ+KW1SuHnkzYB7dvutvXYtC65YQP/k/rY/g4cXwI6XwVzrUl8iOPTsCQZCGG64CoC31r7lUnvXDLuGeZfNY4TpVsDBOTB5vNop68ck6OdjBkP9r+Dqa65hQU0NKxv8/EWgK7AEuA14/djf7WtrefH5561/8ZcdgD0fqL+dVFpav/LWq0G/qA7Q7TIwSI0Onyk7AAX1da80TeWYB9jdzAbMzUc28+WWL53uNj0hnddOf41Z49XuPUvQz+rn79Bc+L6LpCNpRQwGA71796ZzZwPt26u6A+vXw3UjVPBwzo45rD642qU+zLqZrbnqgWdU6QDAwaDfthfgqGtjEB5UuBH+PAt2uXZhaGEtxefMATMB+GTTJ061OX/vfAC6mCejYXB9p19xBvw8SM2ZIjDUlKj0nuXZdh2+eP9ihrwxhLM/P7vuNU2DadPU3ytWpLF5s/2X+zu3hzB+2wr+Hv0liZGJTQ+oLYfVN6mHAyJ4dJoBJ3wNMT0dOq1fcj/aRraloraCNdlrHOvSRl2/fYX7AOiW0DjoV1CgdrT0SFzDEO1eKFjvUH/CD/W9U9VAc9KJXU8EYMGxGnr2ioysf6hjSfF5oOgAZt1MVGgU7WNauPhrkw5j3oO2I9lXuI+fM35m3aF16DrM263G0id2JLHhduyGrjikvqfXysPtoJE6BbpdCqb6Oo/p6areXkFBfd00d7CUBRowwImTNQPE9XFb6nvhnJycHO6+6y7mVFUx2sYxo4E5VVXcPWtWy6k+w+LgtA0w0rEdnM4uuLbcozd8rukyB7MOCDfY9wlsuA9qSuteshb0A/jX+H8RHRrNukPr+GHHDy512zG2I5PSJhEZGmn7MzjsBZh+AKLcsRpW+NSO/8Cvo1xa6GSp69cu6yo0NObvnV9Xl88ZydHJTOk+hYh8tbvA7jmwuhCcKDFkMBjo2dOx+y1XSNDPj6SmpvL0M89wVng4K1EBv6eARcCv0GjFzyKwveInbwWs+DvkLnV6LNnHnjW1bavSRXhFVb7U8vMHv42ClVc1eqm5un4AK7NWMvD1gVz9w9UUVBQ41W3H2I7cMPIGrh9xPdDChadugvAkCLfyQFL4v9zlsOlRtfPKAWFhKr3HyJHqv1evhj5Jffi/If/HPSfcw6AUR5e4NlZZW8mdY+/k/H7nY8pVH3q7g35l++GvO2HX2y6NQXhQ4ggY/oraUe4GllXbDYN+5/c7n2enPsu8yxwr6GxhCfolFEwBcD3oV7QFyg+AQVLjBIzDC+CPkyH7R7sOT4xMZMPhDSzct5DK2sq6108+WdX3O3LEyP33qzo/O+zY5LpzewgJZaO5ZMgF1tN2l+5VqcDyHKvhJvxcXB/ocp7D11UGzcCJaSrw8uc+x0oL2KrrZ2un359/qh0y5bGnwLSV0H6aQ/0JP+bkToGp3acCMG+v49+5lrp+lhSflrTc3eO7t1yy4I8pKlAHfLP1G8749AxeWvES//0vbK9UqT3trueHAfrfDx2nO/oWhL/q/Q8Y9SaERNW9FBYGaWnq3+5K8VlbC9u3q387le64OEOCK37gnbffZnJoqM2An8VoYFJYGO++807LjSYMUun9HeBKli/LPbpb6Dr82BcWneu+NkXLBj4EZ2wDY2TdS5bPguW5nEVSVBL/GK3qmN05905Kq0txla7X7zZt8vzPYFSlOjQJXwS8yiNQcVD9cZIlXlZ8II2p3dQuu3fW2TEvtsDy+bM7vefa2+CrGKgpdrgvt86ZLZD/a3zMfFyRtNvvvJN7nnySCQYDs4A54PiKn3YTYMJ3kDLJ6XFZmnM4tZ0rMl6DrxMgf50XOxVN9L+/yUPx7sey3Nja6Tey40gGthtISXUJr6x6xS3DsHwGrQb9OpwGp61T26lF4Mn+ETY9BCUZdp9iNpvZtGkTZrOZESPUa2vXqgvEd85+hyemPGG1tpUjokKjeOykx/jqgq84nKN2G9td0zSyA5y8TN3oC/+kadD7ZmjTQq0eO1l2+jVM1RQTHsO/xv+r5V0CVpRVl7E8UwVSwg6qoJ/L6T07z4AZB6HdiS42JLwmaayqDdBukl2H90vuR4eYDlTWVrL0QP1ir4QEeOklM0OGZGE0wsaNcPfdqg6lLWZzfbpay+e7ibi+6jMlc11wciJ1kmW31Z/73RT0s9T0O26n33y1JoLxk2IgaRREOFJ0Q/il/V/Ct6lwxLH0nBaT0iZh1Izsyt9Vt0PUXpagn2Wn3+58dZPT1ti2yf15E+1Pg5KdsPtdIkJUaY6NWyuYOxd6HL6bm9Jf4vLhF9g3kMgUGPwYdDzdofGLwGMtQ4QrMjKgulot8HFqkdiSC+Cnfu4ZjHDa7E8/5bLKypYPBC6vqOD7Tz9t+UBTtXqmVmN/MMbZ538N79HdorYUUk5yqe6XcEJsb7UArEHGNctzOGubS2eNn0XPxJ7cPOpmokKjmh5gh2+2fsPd8+5m0f5FlJVBebl6vUl6xSNLoMKx2r3CTw18BM456NLzmIQESEpSz1/OSLkWgPfXv+9QmQGL4qpi7v/jfv7314fk5qkHOnan90wcrjKlOFjj3mw2s2WL90oUSNDPD91+xx3cfuednBgS4tyKn8j26kFfpPMRO8tKH68G/eIHQdeLIc6ZpWrCbXrfDN2vaPSSZaff7t3WF+MaNAP3TbgPgJdWvERJleOrBmdvn82yzGVU1lZSWQlFRep1qekXhNKvhlPXOJxGzGLAAFUPKD9f7T5tuCK7xlRTt1rbWbpev9vZ7jnQEArJYyHemfw6wqtMlVDo+oVWw1RNeXmuD2tp5lJqzDV0iO5MVGUPoqIgPt71dglt45Zav8JLIpKhz22qZpQdNE2r2+0yd3fjNK6JiXD++Ud5/XUzvXurXQGLmnm2fvAgZIXPJbPDf8g1rrfRoUGlYo/wl6Ivwm1+Hgy/21mDrAFL0G9p5tJGtSVbYi3oV22qJqtYFflruNNv717Ytw9CQmDC8IMOZwoQfio8CaK7qgweTogNj2V0J3W3PG+PY7v9LHX9LDv92ka1ZWKXifSLtyMIMvgJ9XfOfMKPBf0O5VWiafDIzX3576W3MrT9UIfGI4JITQks/7tKY9ZAw7p+7tAwtadTl3ndr4BesoDH14qKi7F3c10KUGR5SNKcXW/Br8Mhb5nd47AEdrxa2sea0BgY9yEMuN/HA2mFqougvP6izPJZOHoUqo677IqPiGfzjZu5bcxtGJzcgffd9u94ZukzLN6/uG6XVXy82hldP6ZCmDcBNtzrVB/Cz7ipjJfl+7RD2Zlc0O8C/nvaf536HO7I28ETi5/g7nl3o5s1QkLqa463qPctMN6ORRg+JkE/P/XHnDlcWWvfalubK37M9t94H8/hB97u0Ols9T+N0Vv5RIW9unRRD1rKyuq3PR/v/H7n06ttLwoqC3httWM55M26mYu/uZjx740nsyizro/oaGjT5riDdR3W/ROyXMsfLnyoTTe1MsbJ/9dDQ2HwsfITaxqUENpfuJ+x745l6odTnUozsfHwRjKLMsnP16muVjfQdq/0qTgkxaUDxW9jYOFpTqcTswgPr0/VdPyq7R92/MDUD6fyxeYv7G5v4b6FAAyJm4KGRqdOLsTqzLUwbxLscj3VhfARs8nuOgEndz8ZgN/3/G715+3awenHNpGsXGn1EEDt8juY+BkbOtzKr7t/tn5Q8U6V4tPF/3+EH0oaC4lDHf7dDkwZSEJEAqXVpaw7ZH+2DkvQLzu7vkujZmT1Nav56oKvaBdd/wX8h8qYyOjREL18okpFLwJf6klwykr1t5OeOOkJlly5hCsGX9HywQ003Omn63Bh/wv54/I/uLb3tS2fbAyH8/Jg7EccOqBSoR2J/5Fb76xgoqMlCudPgTVSzy+oGKNg/+eQ2zjgYnlIuWuX2lnvKkvQz6nUngB9bod+/3J9IMIlcbGx2LuH6TAQFxfX8oGpJ8GAB9SiCjtUVNQvuPZ50E/4hqkavmkLa+sXAsTG1u+oWru26SlhxvroXGl1Kdkl9tUjB/X8z7JYZ1THUbazfGkGVdOvy4V2ty383MEfYe8nLjVhSfG5JyOULy/4kgv6X0CIIcThdjLyVeaxTlGqweTk4FsrLUE/P+Xyip+fh8DcsU7375OdfsI/lGfDL0Nh0yN1L4WE1O/2W2qjVKTRYOTeE9QKnOeXP095TbndXWaXZFNRW4FRM5IWn9Z8Pb/KHNj+PGTbeCApAoOpEsqayTPXAkuKz9Wr619LiEwgtzyXvYV7eXPNmw63efl3l9PlpS58tlbV0mrXTn327fL7BPh1hMN9Ch/oeR30usWlhTEWDVN8NrQmew3z987n/fXv293WI5MeYen/LWVS+O2Ai6k9S3ZB8TaosP/mS/iRvR/D13F2182z7PT7K+cvcstyrR4zYoS6idm71/binV27oCJMbXvpEtfF+kHr7oAf+wAS9As6o96Aka85fLdr0Ay8fsbrLL5ysUO7m1JS1HdsdTXkHvvYGg1GhncYzvn9zq/bxW8ywcKF6ucnnYTKCpJ2iUNjFMFrUtokxncZT6gx1KHzLAtrSkqgsNCBE48sgkUz1PeswUjm3oi6H72bd5Xt86wx10DVUalpH2wMRjgvF074vNHLnTqpTCWVlc2n2raH2Qxbt6p/D5AkIwFt+syZfBQR0eT1HOBxYCTQ69jfD4eEMOWss1puNK4fDHpUpWy0g+XZX2wsRDmXqdE9zLUwfypkvOHDQbRSxjDofRt0qE81rWlw4rEKEX82k8F90+FNDHtzGBd8dQFm3b4VDWuy13C47DAxYTGcmHZi3aaTJqVVQmPVAoUOp9r/XoR/2/gAbLjbpSYsi2i2bnVtHerOo+ohThKqQbtTZR/4BhZfoGrj+jkJ+vmYwWD9V+Dyip/kE6DtGKfHZfnit7uelav2fwHzToSCjV7qUNgUkWy1GOmpx75nf/hBpQizZubAmaTFp5Fbnsu32761u8uMo2qy7JbQjVBjaPP1/CJSYfp+6H+P3e0LP/TzIFhof/0Sg8HAwIED6+ZMS9Bv5876lYmx4bHcP0GlAnlr3VvoDlwBmMwmtudtByCmSqV2snvRg65Dl/Oh69/s7k/4UM8b1MpmY1jLx7bAVn0Wy46D3/f8zsFi+wpVhxpDGdd5HCFHBwFO1mexiOsD0w9AnztcaET4THRXaDsKewNrqW1SGZSiPjfz986ve73hvBkbW78bYNUq6+3s2gUV4aqems2gX9olMOBBtfJWiGMuGnARJ3Q5odGq75YYDPXfs8fX9Wto/XoVlImNhWHDgMGPQ79ZrgxX+JN9n8N671/Th4XV3+fu26dTVVvV5FrTqpLdcPAnKNyInruSQzvrV+jcP9HBdHSGUDh9PYz90PE3IPxbaEyTlwwG96X43LdP1b+KiqrPOuGQ1TfBHyer3T3Cp66+5hoW1NTQMBHDi0BXYAlwG/D6sb/b19by4vPP8+ILL7h1DK6k9rRr3rRXeRbkr4Gyfa63JRw37DlIb7x4xRL0W71aZf2yJiY8hkOlh1iWuYxfd/1qV1c/7lSLrE/pcQphxjCyVGZ37z1/Fr4z/GWY+L1LTfTtq77/8vNh+3bYU7CHp5c8zZdbvnSoHctOv6hKtdOvi43b3yYKN0HmN049TzIYDPR3eou+4+SO3U/ZWvFjzYeRkcyYObPxiyP/q/44obzcB9v7Kw6pYsNh8V7qUNhkCIWzd8PAhxq9fOKJKr/x0aOweLH1U0ONoVw68FLat2lPZa19BamhfrLtmagm22Z3+mkaRHexO12F8FPdr3Q4TUN1df2NaVISdOt2LNtrg2xifxvwN9qEtWHn0Z0s2t9M8arj7C7YTZWpisiQSPR8VUfI7otOTYMhT0N/11YsCS+zcyVicxqmamq4GCI9MZ0JXSZg1s18uMGxh3mWh98uBf1ApR8LPT4/sggI7SbAlD/U33b6W/+/cemgS5sE6xrOm6OOZUS0luLTbIaMvVWUh+0HoFdbGzUF0y6GAffZPS4RYNbfDWu8V+PJsqPZ8rDnx50/8uLyF9mQs6HuGEtqzxNPdGD3vQgcB2fD1qeh1sbTRDssz1zODT/ewHt/vefQeZYUn5v35BP5RCRdX+pKWUUL40i/Ei6qhP2fof8xDT1rFGN2/0TGDZn0S7ajHqA1wZZLSkB1AWR+p9JhN+CuoN/mzervfv1UMNGp8VUXuWUBnHBNamoqTz/zDGeFh7MSFfB7ClgE/ApcAkw59vciYJHJxFP33tty4G/bC/B9Z/W7boGr9fwaXmu6pE0anF8AAx5q8VDhHWlpKhBSUwPLbJSITItP49phKjX2iytetKvdOTvnAHBmzzOB+vJSTe5/Nz0Cv46CyjxHhy78VbuJqtSPC0JDVcp/gCVL1P3DPfPv4dXVrzrUjmWnn6FAPYfu3NnOEwc9DBcUQ5S9UcLG3DZn2kGCfj5mtpHQ3dqKH2i6zX8AML+ykrPOPtttY7Ls8ouL8+L2/j63wfmFEGXv/2XC20JDwZJN4ttvbW+jvmfCPWTdkcXVw662u21LPu/BKapQW7NBv+KdKkgs9YQCW/97YOADdh9uNpvZsWNHoznTstuvYV2/mPAYZg5QiyDeWveW3e1vPqLunvsl9+NwjvpqlPTGQWzTo/BtiksPGUHdmMTEqPR0GzY0/tnfh/wdgEcXPcoH6z9otp0nFj3BdXOuY232urqH306n99zzIay/V9KFtTL3TLiHj875iHGdx9W9dvy8abk52rSp6WrdgwchX98Dmpk2YW1Iibb2BSyCXt5ylT7diWusOTvmcNNPN7HlyBa7z7E83LEsdvhk0yfcMfeOuuvCsjJYsUL97KSTgMxv4c+zoWibw+MTfmrwU3BOtqqD5qQ12Wt4Y+0bfLb5M4fOs6zo3nBgNzo6Jt3Enl17bN6f1zEYoeeNbDE+jgZM7Xo6Pdo5sVJn32ew+z23LEISfuboGlh8bpMa9JY6RNu3u9a8y/X8xn8Kp6xwbRDCbW6/807uefJJJhgMzALmAKNtHDsamFNVxd2zZpGT00xuMEMohCdBxeEW+3eltI+1e3SXaBqERLqnLeGYvBUwbzJk/1b3kr0pPm8ZfQsGzcC8PfPqnqvYklmUyfqc9WhonN5TZX6yudPPVKXK+4TZUctSBA5TFVTlu9TE+PHq72XL4Oxe0wFYcmCJzVIXx9N1vS7jXPUhtSLH7p1+oBZXO7Foy2w2k5HhvbSgEvTzU8ev+AHr2/zvAU40Ghk5bFjj1T5H18Cq651Kl7lnj/rboQ+8OxiMstLRXxRnwMaHmnx+Tj1V1SLYt0+lW7ImKjQKgwNpv8qqy+q295/f73yghaDfqmvhx752ty+C18iR6u+1a1XNH4trh6uVZl9v/Zqj5UftasvykLJ/u/6O3/js+R8sOhfKDth5gvC5sHhV56LC3kTa1mkaTJ6s/v3TT41/dumgSzmz15lU1lby99l/559z/2mznY82fsRb695i04EDlJWp3SxOB/0OfAk7XwGjfdkChJ/K/gWW/E3Ve3KTDh3UCkaTqfEOaVC7Vcsi1GrHXm171dVTayR3OfwyTBVgF8Fp4vdwVoZT1+Nvrn2T19a8xm+7f2v54GMs85wl6Lc1VxWp6pHYA1B1pKur1ec2PR1VRy37F0kvG0zapEFke5fuAS11TRfvX0xFTYXd51l2+m0/rG5+u8V3a/mk7F8gbyV0OZ9vNt8CaCrtrDO2Pw8b7pPPczBqOwpGvQ2dGi/M7t9ffdT37bNdX9eahQvhlVfUebpev9PPpQxh8rnzK7ffcQe333knJ4aE2Az4WYwGJoWF8e4779g+qPctcNpfKuV/C1zd6ec2+z6FIzZSSgkv0CB/bZOa8Jag38aNKp2iNWnxaZzT5xwAXlrxUrO97C3cS8eYjoztPJbk6GRKSlR9XbAS9BvyJMw4oILYIjhU5sKX0bDBtdTuQ4dCZCTk5UHl4a4MTR2KWTfXPVtuSW55LkVVRWho1OamA3ZmWqrIgb0fQ3l2y8f6Afmm92OWFT8TjUb6Go08gfVt/vNra5tu8y/Pgl1vQsF6h/u1pJuw1CryuMJNsO05KHOxorVwn/IDsPlROPxHo5fbtIFp09S/v22hZJ/JbKp7eNOc33b/RkVtBekJ6QxrPwxdrw/6Wb3w7HY59LtbAsSBrjwLFp7pUqHu3r3VLquyssY11YZ3GM6w9sOoNlXz1dav7GprS64K+vVL6u94TdPi7So9lQur1IWX9f4HnLwEYtJdbur0Y6Up16xp/AAnzBjG7L/N5pFJj6ChMSR1SJNzc8tyeXXVq+w4ugODZqB91SRAPYh0Oo3dxNlw6hoJ+gW6kgw48AUU2b9ryqybWZ+znnWH1tk8xrLbb8VxC/x374aycLXq0GZqz5pCtwYhhR8KS3D6+mpS2iQAu+u5QOOgX2VtZd0CnGHtVRTFktpzypRjw+o3S6VWjOnp1BiFH9J1tWiqZLfTTfRJ6kPHmI5UmapYcmCJ3edZgn77ilXQr3t895ZPWvF/sPZWqqvVrmmA4c5mqZrwjfojgk9YHPS4Gto0DiTHxcGAAerfS5fa11RJiQr4zZ0L//gHPPEEFBerupQ9nZkKNz4Mu5oJFgmf+WPOHK5sWC+gGZdXVPD9p5+6pV+/CPrpZlh1HWyS1J4+03YkXFCk0lg3kJKiaqjpuu0yPwC3j7kdgI83ftzsbquJXSeSeXsms/82G6hP7ZmUpDYYiCAXngRdLjpWv955YWH1pSuWLqUu6Pzd9u/sOj8pKom9t+7l/Ul/YNQjSE5WQcQWHV4Iyy+DwwucG7iXSdDPz91+xx2sXreO3WYzP+HANv/20+Dcw9DtMof7tDw872XjmY/bHZwDf/1LBZqEf0gaA6eug57XN/nR9OmqdsD69bB3b9NTAQ4UHaDDCx0Y9faoFlfcntPnHJZftZyXT30ZTdMoLVV1JQHatbNyQvr/Se20YBAaBznzmtS6aI7RaGz03waDWuEDqrh0Q09PeZrfL/u9btdfSyxpKLpGDqC8XD1ctLrT1JohT8OF5RCRZOcJIph07AhDhqgboV9+afwzg2bgwRMfZPONm7l00KV1r989725Gvj2S9s+35+ZfbgZgXOdx5GXFA9DdjueONhmMahejCGzd/65uvNtNtPuUl1e8zNA3h/LQwvoHJsfPm5abo7VrG9eh3LULuuReyyuDV3D3eBvfsR1Ogxn7oeOZdo9JBKDDC2Df5w6fdm7fcwGYv3c+B4sP2nVOp07q+zY3Fz78bSMm3URSVBKdYjtx6JBKYadpMGlSg5MMRtmhEkxMlTA7TdWTdJKmaXW7/SypYe3Rvr1aYFNsOBb0S+jeZM5sYuTrMOABspZ9w3MnDWZk91X214A5XnRXSB7X8nEicNVWgKlx7R5LSjJ7g35//KF2PEdEqGtNS13ePn2cWCBmqlY7TA986eCJwhuKiouxN+6WAhQVFTV/0IFvVPamZtTW1i9adLa0RYvzpj10HSb9BAMfdr0t4RzNYHPhl+U6rLkUn+M6j2Nkh5FUmar4dlvzOwQ0TSMpSj07sVnaorYMtj6j0o6K4KFpMP4TSL/K5aZOOEH9vXQpTO89A4C5u+dSWl3a4rkGzUBafBqplZMABzIdpkyC8Z9DymSHx2vhljnTTnLH5GP2/LJ/mD2bk8LDHdvmHxIFEe0cXq1bWanSRoAXg369boGT5rtczFO4UUg0JA61ulOkXbv6yfVHGzunO8d2JiIkgrKaMubunttsV5qmMabTGM7odQZQn+IpMRHCw51+B8LfhcbAhaUw9Bm7DjcajQwcOLDJnGlJ8dmwrh/AyeknM7X7VLtTzd474V7+Ne5fxJapKGLnzmr1kN2M8mENOLlLYclFULrP5abOUNMXc+eqBzPH65fcr+7fJrOJDzZ8wJrsNZh0E8PbD+eZqc/w7YXfsvvYRgeng357PlRpWUTgC41Vfxxg2Wm1cN9Cakw1VufN3r3VToOyMth6bDO+rqudfqHmWM4aOprBqYPd9S5EIFr3T1j7D4fr+nVP6M4JXU7ArJv5ZNMndp0TEwPnqlghb8xWO1SHtx9OQYHGY4+p14cMgbZtj52w92OnShcIPxYSCQMehK4XutTMyd1PBuDHjB/R7fzshoTAsGFQHq6Cfj3a9rB6rdlI5xnQ8Qz27jUTHlJO/z4Vzm2OrciByjwnThQBY+8n8FUsHFnY6OVx49Qjmh071IKH5jRcUHbVVfDcc9Dv2CVlo8UQ9jKGwYwsGPmaEycLT4uLjcXewgOHgbi4FuqcHfgSNj8GNSU2D8nNBbMZQkPV8xdH2bpHd5jBqBa6ObDYTXjA0TWw45Um14Djx6sF1xkZ9c/rjqdpGs9Pe54lVy6xufC6qLIIk9nU6DXLTr8mQb/SvWpBUPbPzrwT0QoMG6YWxOTmQnjRALondKfKVNXiM+iGMo8lHLR7AVdkKnS9CKLsTQvWmNFopL9LubkdI0E/H7PnpmT2p59yWWWlXe012uZfsgsKNjg0nt271fzetm2DG2xPC42B1JMkFZm/MVVC/jowN00xccop6u9lyxrvFLDQNI3z+p4HwNfbvnaoW0vQuZu1shp7P4FfR0H+Xw61KfyUwf7lqbquU1xc3GTOHD5cXYDu2wf791s/1576LjMHzuTZk5+lOFstcUy3N+tjTal6CFmyy84ThN+oyFE3w0dXudzUyJEqJUlxccsrt40GI79f9jv/m/4/Mm7JYM21a5g1fhbJ0cl1u6edCvrVlKiap3/d5cTJwi+V7lGrtO00OHUwyVHJlFaXsiJrhdV502Co3+3387F76IMH1aKvsLAWahnsfh+yZjvxRkRAGfYcnDjHqVOvGHwFAB9s+MDuwMvll6vAXl6YWrDQPXIYd9+tbsLbtoXrLUknqgtUOp0dLzo1NuHHBj0MXS5wqYkzep1BmDGMrblb2XjY/sDw5Mn1Qb+0uG5WrzXrNHj961UXcN2PGbQffKJzA972b/g2Wa4fg1lcH+hwepP0/wkJ9bX4li1rvomNG9V3dGSkCvL17g1PPw2ffw5Tpzo5rrA4iOnh5MnCk6bPnMlHduY3/DAykhkzZzZ/0OAnYPp+CGlj85CGZVWcWcBg6x7dYVX5KsWn8K09/1MLv8qzGr0cF0dd/dp5zWyon9B1AuO7jK+rDb6/sPEDmlm/zyL1+VQ+2vBR3WuWnX5NSqu0SYdpy1X2ExFciraqUj+ZLdSMakFYWP0mgKVLNWb0nkF0aHSLGUd0XeecL87hqcVPsfOA2jFt104/XYfqFnZYt9iEmjO9RYJ+PmY2t/zF5vQ2/wWnqJtjB1jq+bl1l9++z2C/jRQS1YVQvNPh1cTCCzY9Ar8Oh+IdTX40YADEx0NpKWywEVe2BP3m7JhDVW2V1WP+b/b/ce2ca8k4mlH3muWhd1qalRNqi6EiS+1EFIGvdI+qKVHZciV7s9nMnj17msyZMTEwdqz69/E7T3Vd58EFD9LhhQ5sy91m15B2HXv2YnfQr2irmmclTU7g6XA6nHvE5d0FAEYjnHaa+vdPP7V8/IB2A7hiyBX0SKx/6FJcrApRa5qNRQ8tDiIKpsyHQY84cbLwS5sehSXnq4LndjBoBqZ0nwLA73t+tzlvnnmmCv4tXaoeOO7aBbWGMvb1uY031r6K2dZDl/V3qTQ7IrilTIak0U49/bug3wWEG8PZmru12dqSDRkMMGsWVMStB2DV7GEcOqQySzzzTIOHQIYImPg99LjO4XGJ4BcfEc+Zvc4kPSGdw2WH7T5v5Eid1PIpJJZMoDqnm9U5s87eD+CbdhTs+JOsLPXZHezsxujkEyD9GmjjSj5v4dcSh8OJs6HdCU1+ZEnxuaSFEpSWxTmTJ9fXutI0iI52svzqga+bPMwX/uPqa65hQU0NK1s4biWwsLqaq66+uvkDY3pAdOdmPyyWWvbO1vOzda3psCUXwHcd5bmgr/W8TmVhC29atsSy8N9WZpvjzd8zn+7/6c4NP97AA388wNQPp/LBhg/IK8+jXXR9HR/LTr8mCw9DIlXZIfmeDD7GCMiZq2rYu6hhis97J9xH3qw8bhl9S7PnrM9Zz/fbv+fRRY+SnaXCYnbt9KvKha/jXVpkbTab2WfZ6eIFEvQLAE5v8+87C3rf5lBfbg/6mapgw33w151QW97059uehx97Q6Zju8GEF3Q4XaXbsZJezGCov1mxVcx3bOextG/TnqKqIubvnd/k50WVRXyy6RPeXvc2Vab6oKBl/rMa9Ot5A5yTDbHeyj0rPCpnPqy6Bo6ubvnYZpx5rLzUggUqZZ2FpmlsOrKJwspCHl30qM3zn1j0BEsPLEXX9br0inYH/WJ6wAlfQadznBu88J2QSIhIdltz06apVGE7dlD3OXLEHrXRgPbt7SwifTyDEZLHqz8iOKRfDeM+BaP9HwhLirvm0pp07w4XHNtU89prsG4dlIVnsC7sZR7+82HbaZEn/QzDZJdVq2CuhTIb2+ebERcRx4w+M+iT1If8iny7z4uJgUVX/snkXStoWzyFjh1VwK9Rbd2QSOg0XT0AEsHl8AL4ZRhk/+pSM+9Pf5+MWzKYlj7N7nPCwzVu6/4u43YsYvPKFp56h8RAbC827Uwm3FjGZePeIbq0mQJHzel8Dox+S+pTtlKWFJ/bt6sFX9YcPQorjpWyOv10N3RacUgFVtbf64bGhCekpqby9DPPcFZ4uM3A30rgrPBwnn72WVLtidRVHoHCzTZ/7GrQz23aTVQ7vp2KZgu3iR+osrCFNL33cCSzDagau2bdzBtr3+DxxY8zf+98qkxV9E3qW1eSQNfrg35NdvpVFzSpiSqCRHQ3uKAY+rmeoWj4cFUW6sgROJqVSERIy7ulP9v8GQCndT+TotwYwM6gn6kKul4M8YNcGbJXyVVmAGhum38O8DgwEugF3KFpxLVvT05Ojlqlkf5/DvW149imLrcF/YzhcPIiOPEnOLoSfhvdeFdPj2ugx/XQcbqbOhRu026C2jESbX32mzBB/b1iBdTUNP25QTPU7fa7/4/7KalqnEt+9o7ZVJuq6ZfcjwHtBgDqS7/ZoJ8ILu1PVSnEXHyA178/dO0KVVVN0008fOLDAHyx+Qu2HNnS5Nxtudu4f8H9TPzfRHYeOlR34213esXwROhyPsT1df4NCN+pLoB9n0LRdpebio+vXwzxiX3lrBqxBP2cSu1prlE1GGR1bHBpdwKkXQyhttMyHc8S9FudvZqCigKbx110kZo3i4qOLZiIUKu+eib2tN142xFqB5gIfnPHwLxJTp36ztnvsPXGrZycfrJD5/XtEcV/7hrNBWcm8NRT6sFSI8fVgBFBxBAOVXlQ41rKpNjw2LqUYo6YPFn9vXQp1NQ0c36X81gVt4RXPuxHqLGK8ztfA/s+dnygksKu9Tj4Eyw8o8nuusRE6Hvs1sHWw/O5c1WtNct9jstCYtRCop7Xt3ys8Jnb77yTe558kolGI6dERPAJMA/4GJgSGsFEo5F7nnyS2++4w74Gfx0Jy2ynAbWk92zf3tWRu2jgQzDiPz4ehADUd1R103sIRzPbPDX1KRZcsYAZfWZw+eDLef2M19lw/QY23bCJ8JBwQNViq65WC2cbLfQCWHUdfBklgb9gpGluK+8VHg6jj92eLlqk/tZ1nbm751pNO2zWzXyx5QsAJidfDKjv5Gh7kslFd4bxn0K3S9wxdK+QoF8AsLXN/0WgK7AEuA14HXhB1zEsW0bXTp148YUXHOqnoEBNupoGPZt55mOXmtL6GgVRnSBhEFQdVauMGu7qie4Co15XRaVFQOnXT02OZWXwl40Se3eOu5PkqGS25W3jr5z6g/YU7OHfy/4NwIX96lPr5eZCebn60m+yvV83q7RiuXYsKxKBIbozdDwTwu0rIBphY/GDptXv9vvpp8Zxj8Gpgzm377no6Dy26LEm5762WhWyP7v32VQcUcvL2reHqKgmh1pnthLxFoGjaDssuwQOfOGW5i6+WN0QrV4N69c7dq5ld6BTQb8ji+C3kbD9eSdOFn6vwW74lnSO60yfpD6YdTML9y+0OW+GhsKtt6qd+1Af9OvV1saqr9oyVetXtA5dL4a0mVbrOrekTVgbpwIvoNLHX321qnnVxJob4Zt2qjSACC7J42DGAeh6kVuaq6ytZH3OeruOLakqoVffapKSoKxMY9eueJvHzpkDjz+uaqD27BdP9bifoe+/HB/giv+D30+QObU1qMyBnHlWy2U0TEl2vAMH4NdjG1/dsssP1AKitIvV/2/Cr91+xx3sz8pi4n338VLfvlwRn8INoUlsCmlD5+RkPn37bR5/7DG10L8lff+psiXZ4I6dfrauNUWA+qE7/HmW1R85mtlmUtokvrvoOz6Y8QHXj7ieQSmDMBqMdT8/eKz0Wvv29fckdZInQPcr5VlxsCrPgl1vQelel5uaNEn9vWgR1JrMTP1oKqd8fAo/7PihybErslZwoOgAMWExdDepKLZd9fzcyJtzpgT9fMxoNLZ4jLVt/i8CTwGLgF+BS4Apx/6eW1nJIpOJp+65mxev7gCHF9o1Fktqz86dnUwtZqHrsPg8taqodE/9653Pg7P3qN09Sy92OYWL8ILVN8HvE63+SNPqb1ZspfhMi0/j50t+Zu6lc5nYdSK6rvPeX+8x+I3BbD6ymbjwOP4+5O91x1t2+XXqpC4mGinPhPV3264PKQKXHcVwjUYjffr0sTlnTpqkVuccOqRS1TX00IkPAfDlli8b7fYrrS7lgw0fAHDTyJscr+cH8Nso+E1SjQWstiNgzAcqjaIbdOxY/3Dm3XfVCu2GTM1sVHFpp19EKvS/Fzqc4cTJwq/9OR3mOJZ+4ZmpzzDvsnmc2fvMZufNnj3hPLUhn4pIVVPBZtBv93vwRSQcsfGFL4JL3zth8BNgOP5izH7lNeUs3m/f5+W5Zc9x4083surgKtsHtUlXaadC45wekwh+2/O2k/pcKid9cBLVduwOePTPR4l8IoKCwQ+haRq7d3dtMmfqOrz3HpQue4iT0t5n2jR48CEDYWmnOVdywBAKGNy2yl34sa4zVQqz1ClNfjTuWOxt2za47z74+GP47Te45x646SbIz1cLICy1y12im1V6TxEwUlNTue/++5l51VXkleQx0lTKixV5vJmTw23bt7P4ySftW+jf+xabQT9ddz3o19I9ul12/Ec9H6yyPy248KBO50K7SVZ/FB9fP3dZao66whL069jRyg973wKj33a9E+Gf8tep3ZyHbJeksNfQoapUQEEBbN5kYHRHtfXv1l9vpbymcZmxzzap1J4z+szg8EEV+LArtSfAxgdh3Z0ujdVoNNLLbakVWyZBPx+zt+Btw23+k8LDuQuYA9hKsjQamFNdw93vHyInO9OuPixBv9697TrcNk1TDwt6Xqdy9TZ8PTIFSnZC9i+Q9Z2LHQmPqy2D2lKbK70bpvi0Vcx3RIcRTOiqDvx669dc9cNVlFaXMqHLBP667i+6xtfnK7EE/bp1s9JQRCpMWwG9bnTyzQi/tOgc+L5zi2kJzWYzR48etTlnRkTA1Knq3z/+2Phng1IGcV7f89DRuWf+PXXb/D/e+DEl1SX0atuLk7qd5Hg9P4DEEdB2pAMnCL9iCIXul0OUtTsN51x8MbRpo+az339Xr23fDtdfD9deq9IpHq+ysv6mx6mgX3x/9YBe0swGn/iBalW+A6ltzu59NlO6TyHMENbsvAnq83rmmRDVpYX0nm26Q5eL1N9CtCC7JJvU51KZ+tFUu2r7fbX1K15f8zp7C5pZ7dtvFkyZL/V+gtWRRbD5cZfTVPdM7ElUaBQFlQX8uqvlBaZ7CvegozO0TxK6rrN8eTVFRY3nzC++gO+/M3Nev2e4aOzX3Hxzg8WJzqQdG/02THWyFqAILCGRqtyJFW3b1j8837hRfc7++1/YvFnteBk3Dh57TO3Md1neCviuA+x81Q2NCW958fnneeq++1hkMvGHubLRQv/fLAv9773X4QxfFsXF6h5E06ykVrRTS/fodilYD1mzITTW+TaE+wx/AQY/bvPHZxxbY7pwIZSWutZVs0E/EdzaTYCJs9XmIBeFhNRvSFm4EO6bcB9d4rqwv2g/N3zyVN1z5lpzLV9uVZtILh5wMZnHQiV27/TLmg3ZrkW7zWYz+fneW+AgQT8fs5Zj1hbLNv+Q8eMZp2k2A34Wo4FJ4ZG8+9N+u9q3BP2cDjrXFNevzmk/DYY8bf3GPK4vnLYehr3kZEfCa8b+D05bZ3Old+/ekJysLhbXrm25uXP7nsvktMk8M/UZFlyxgG4JjaN7e48967Faz88YrmoJxboalRZ+JXkCdL0QTBXNHqbrOpmZmc3OmZYdVmvX1l9AWjx04kNoaKzPWU9JdQm6rtel9rxxxI0YNEPdTqsePRwY/+i3YcQrDpwg/FJ1oUpB7QYxMfC3v6l/f/QRfPghzJqlPpNHjsAPTbNMsH+/es4ZH28jrZ1ovQY/DuM/cyq1jT3zZmgoXHcdHDG1kN6z4xlwwuduDZALP6brsOQiWH2zU6d3iOlAt4RuVJuq+XJL8xkaakw1bMjZAMCw9sOc6k8Egf1fwsYHVGYPFxgNRv42QH0Jf7Kp5QK7ewrUxd+I7t3p3l2ntLSCZ59VJQdA1VX75BPQ0VgUv4/2Z71af3s7bzL8kObcQCV43XoUbVcpzKzUcrz7bhXou+kmmDIF+vRR15Dvvqt2/Lmllh+AMRK6XAhJ7tg2KLwhJyeHu++6izlVVc0v9K+q4u5Zs2yn+qwuUllp/mqaithyStu2EOZkBkV7rjVbNOY9OC/PpewCwnv69lXP66qrYf5819qyGfQr3aMWhx+0o3igCExhCdDpbIg4voi3c048Uf29bBmEEs1Lp7wEwMd7n+WWhzIoKYGiyiKmdJtC17iuTO0+lQMH1Dl27/Q77S84eYlL49R1naysrJYPdBMJ+gWY1NRUirKzucbOL9XLKyr4/tNPWzxO1yFDZXZyPui37DL4Y6raHdaSNmlq5ZsIaA1TfC6xY+4zGozMu3wes8bPapTL22L/sfi01aBfZZ4U8Q1Gfe+A0e9AiL1F9Gzr0AFGjlTz2WuvNV4sPjBlIG+f9TYfnvMhseGxLDmwhE1HNhEVGsUVQ66gvByys9WxTu20EoGrYAN80xZ2uC94e8YZ6vNYVARffaU+i32PbcL76SdVC7Uhp3aZWtSWww89YeuzLo1ZBJcVWSu4Y+4dzMue1+Kx+RX5dbuxeiQ6supBBC1Ng+LtULrL6SauGHwFAB9u+LDZ47blbaPKVEVseCzpiTYmwco8WHUDZP/m9HiEn+v9Dzh1LUS2d7mpSwZeAsAPO34grzzP5nE1php25avPePeE7sycqRMSorNhg8ZNN8E776iADMCFF2pMPStV3cNatDtRla1wxIqrIONNx84RgW33OyqFWfHOJj/SNBXYO/VUuO02+Pe/4ZJLIMk9z0DrJQ6FE76ARFlYESjeefttJoeG2rfQPyyMd995x/oBobFQXWA1c5M76vm5jRueBQg3Kc+CpZfA3o+s/ljT6nf7ffZZ/TMUZ9gM+pXth4Nz1FhEcKvIcctz3n791IaUigpYvRr6ajNoV3wqZkM1v3QdxuRXZ9I2qi2fnvcpu/+xG1NNKEeOqHPt3umnGSC8rctj9SYJ+gWgouJi7P1eTgGK8lsu8HvwoHoIGRbm5IoyswkSh0PCYDDKF3bQqClVN6bNPGSxpPhcvpy6SbM5Bs36tFNdDZYFD1aDfiuugG8Sra6SFMLi6qshPFylyTk+zefk+Kvopk0CoKiqiG7x3bhk4CXER8TX7fJLSoJYezOLHPxR7YIoO+Cu4QtfiOunVj8nDHFbkyEh6rMIKtXnXXfBM8+oC8qyMhX4a8iyy9mpgHPlYbUTWm+mYKAIbOvvgQ0POHTK77t/5z+r/sOvWS2nt0uISODIP4+w8uqVRIdFWz9oyYVuDYyLAHDqWpjsfP3tmQNnYtAMLM9aTsbRDJvHrc1WqSKGpg61eY1I6W7Y9QYUrnd6PMLPxfZSAQmD67kMh7UfxtDUoVTWVvLon4/aPO73Pb9TWl1KSnQKvdr2YuRIuOuug/TurVNRAbNnq0U7J58Ml56fCyW7wVxT38Cgh9UOFXtV5cOBLyBvmfNvTgSe7lfAiXMgqpNv+ncxZa7wjdmffspllZV2HdvsQn9Ng7N2wPAXm/zIstPPp0G/4h2w7zO1uEf4B2Mk7P8U8v+yecjUqWpBa1kZPP64CrQ4qrq6fld9k6BfymS4qBK6/93xhkXg2P4SfNce8le73JSm1e/2++UX+Pe/Nfrvf4UYc2dMxlKyc2rYsUP93Ggw1j17jotTmZpaVLoXcv6AmhKXx+pNEvQLQHGxsbQcxlMOA3GhLe+8s6T27NEDnKrBazDCwAdhzPuSriSYaBqsvh72fmDzkB49YNAgqKlRBcidlZmp7kliYmykt0uZAt2vVKsrRPDQzWr166ZHWjw0xo5v4w4d4Mor1b//9z+1oKG2Fj74AG69Va2izcyEM3udScYtGTw37TmAuqCfQzutDi+AjFclEB3oDKEqfWLnGW5tduRIeOUVeOsttSNa0+CCC9TPZs+Gqqr6Yy07/ZwK+rXpBmdshv73uDxm4aeyf4HMrxw6ZVr6NABW5q2kqraq2WM1TSM5OplRHUdZP6CmWNUwKLB98y+CkItptlLbpHJK+ilA87v91h1aB8Dw9sNtN9Z2JJyTA+nXuDQm4edMlWp1v4s0TePZk9Xu99fXvM6OvB1Wj/ts82cAXNj/wroMJOnpETz9tM5VV6l60SecoFIvavs+hDk94Ogq5wcWnqhS2A2RnfmtSvxA6HgmhLbxTf9bnoDfxkKZa6lzhXc5vNDfWtHwFliCfu1d3GBtzz26TVk/wLKZKruA8A/hbeGCYlXbz4aQEJWeODFRPVt56SXH1xdkZ6tzoqNtLLo2hNisiSqCRNIYSL/KbfU8LUG/DRvUhpT0hB7sv3MXs6K30OvgI7z+OljKjzpcz+/Al/DHFCjc7PI427Tx3vWAPD33MaMTEbbpM2fyUUSEXcd+GBHOjMuvbvaYjAxVcwhUjTaH5S6DWieWdgj/FxINJ82DobZvTjWtPsiycGF98MRRluKq3brZiBv3vUNqpwUjzQCHfoec5lPQGY1G0tPT7ZozTz8dhgxRq8eefx7uvRe+/lr9rLZWpWrSdbXCJzZcXWBYgi4O1fMb+hzMyIRoe68URGuTltZ45diECWo1bXEx/HZsA7XJVJ/aWFLLCqtOmgtnbHXolJEdR9IxpiOlNaXM29dyis9mhcbCRRXyHdza1JTAng/g8J9ON2FJ8fnRxo8w21ggsy5HBf2areenGSAyRQVNRPD6eRAsPN0tTU3tPpUze52JWTezcN/CJj8vrynn++3fA3DxgIuB+mvN0FAjM2bAF1+onfpGIyrw3PefENOzvpHDC2DlNar2kL2MEeqzLFofX+1kMlVA5SGI9IccjsJezS30zwEeB0YCvYAbgfLqatt1/Ur3wpanVEmDY8rK1INxcC3o58g9ulVdL4Rxn6qMYcJ/hLYcyE1MVLVHQ0JUHbWvHFufWJcWtFMnK8//jixRf0RwSxqjSv3ED3RLc2lp9ZkLQ0LUNVxCbBj3XN2PVGM/du+Gjz6CN9+Et99Wx9ldz6/D6TD8ZZUlygVGo5HuXnzoI0E/HzObHd8hcvU117CgpoaVLRy3ElhYU8tVt6iUULqudvRt3KjqDIEqvHrXXXD0qPqwz5jh4GAqDsGCU+DPsxw8UQSM1CktpiPp0UOtqtB1eO8957KIWNLbWU3tKYLbaX/B1EXNHmI2m8nJybFrztQ0tasvOlotati2DaKi4Prr1artrVth7tzG5zhVU03T1P8bsvs08FUcgnmT1Q2xBxmNcN556t/ffgu//qoC09XV6rPp1E33zldh3+duHafwMxHtHJ5nDJqB8/udD8Dnm5v/fPx76b+587c72ZCzwfZBmkEtBBKth7kGVvwd9jiQvvA4Z/c+m9jwWPYX7eevQ013iuq6TkFFAQDDOzSz0684Q/2RnfXBLf0q6PI3tzX30ikvseH6DVw34romPzNqRl4/43WuGHwFYzqNAZpeaxoaTrvtJsLQf6v52KJ4u6rXVrq35cHUlsP2F+07VgSfZZfC9x3B1PzOe48Y/AScvcctqXOF99ha6P8i0BVYAtwGPAEMBmry8ujRvj3pKSk8/thjjQOA5Zmw4d66Rba6rhbB5uVBSgqMGOH8OB25R7cquiukXWxXkEl4UXk27P8SKnObPaxPH/WMBVTWr2UOZK+2pFfs0MHKD9fPgiXn2d+YEMecc44K+F1/ff2C/vh4uPxy9e+vv1ZlgEpKVGmfqVPtbDh+oKo/HRbn0vjMZjOHDx92qQ1HyJNKH9OdiI6kpqby9DPPcFZ4uM3A30rgdGM4jz/5LKlJMZhrqnjzTbjzTrjvPrj0UvXnpZdUWsbRo+G559RqDYeEJ8GgR6HvnQ6/DxEgdB0qDre4m/Pyy9XkumEDrFvneDeWnX5Wg35HFsOiGZDXUqhbBKSwuBbTAuu6Tk5Ojt1zZlIS3HCDajY9Xc11Z5yh5j2A99+H/Hz17+rq+u39dgf9TNWQM1/9vyECX3gSFG+D6nyPdzVlivquPXoUXn0VFi9Wrw8Z4kR2bF2HjQ/CTtmBFdR0M+QuVbtKHHBhvwsB+GHHD5TXlNs87rPNn/HCihc4UGSjPmnhFrXby2RfbRkRJMITYeJsdZ3vpMjQSD465yN23bLLalBP0zS23rSV3H/l0qttL9sNrZ8FP/WV2lTBrt9dMNCx+qXNSU9MZ0C7AVZ/Fh4SzqWDLuV/M/6HduzL19FrTdIug/OOqtpDLTn4I6y7Aw7OsXf4Ipi0OxG6XQG1pb7pXxYoBhxrC/1fBJ4CFgG/AkeAS4ES4AVgNvDokSMsePRRunbqxIsvHEvPmDgcpi2HHmoBxO+/w5IlajHirFkQGen8OB2eNxufDLUtlyISPnDoF1h6kV01aE85RT1n0XW1mHWH9YzajZjN8NextWBN6vkBDHgAhtpOLyqCyJ7/wa+joLrQLc1NmQLffKM+lw2deioMHAihoSr70iOPwLvvQs+e1ttpwk33ILquezXo51qxBuEzt995J2gaE2fNYlJoKJdXVpKC2ur/v/AI/qyuoVevJ0kjGb5sw1eHZvPTwrMBaNdO5be17Pb7299g5kwnS/EZQqHP7e56W8If7XpL1fWb/Bu0n2bzsHbt4Kyz4Lvv1G6/9HS1osIeut7CTr+SDHWz3Fs+a0GpphSO/KnSZLppaz+o3aeDB6vivJb57ayz4M8/1Q7AN96Ak09WO//MZnWc1XqS1pTugj+mQr+7YYhnd4cJLzCEwjnZXnkoEhoKV12lLjI7dYJ+/VQh9IHOfvRPWQW1gVVQWjhI12HhmRDbB05ZbvdpozqMokNUB7LLs/k54+e6nX+Nm9bZeVQVdu7Z1sZdz643YOd/YUYWRFm7MxdBq9PZLjdxdu+W20iKSmr+gLSZkDhC1RAXwgnLMpexImsFd4y9w7kG5k2G5HFq15SFIzXaOk1XQfSkMc71LwJbj2vUH3co3gGrb4Ipx1J3Z34PRxapnajHz5FrboGYXtD7Fvf0LbymbqH/ffcxp6qKrsDdqIDfaBoHAEcfd+4ltbWsBM66914Abr/jjrq5JzNTpbYDuOwy6NXMehuPK9sHP6TDgAdh0MM+HIhoInUqjP0Y2tqo932ca65Rz5hXr4bHHlMbSoxGlV1pyRK1m/SKK9QmAVALsDdvhrAwGDfOSoMdTnPfexH+rbpQzQVl+yBsiFuaNFh5pGMwwOOPq+d+IY5GwirzYE469PmnWxeneYME/QLY7XfcwcUzZ/LuO+/w0qefUlRURFxcHDNmzuSBCVfz6qupLNn8FynmmazenkxoqNrpN348VFaqGkJt2thYWWGPwk0qtV2YvU/JRUBqOxJ6XAvhyS0eeuGFauXYgQPw97/DqFEq8HL0qEqxuHOnyrF8ww2Q3KC5wkJV40rTbBRSTf8/6HYZ4ExkWvi9ioPw55mqVsrQf7u16eMDzwYD3Hwz3H47LF+u/lj06ePA4ofwJBj+CrR1IR+K8C9eXAU9caL64zJNgxhHctKKgGQwwsjXIdreogOKpmlM6zCNeYfnUVFjfbf+odJDlNWUYdAMdE+wUV8g7VJo0wMiXSj6IgJXdQFoIS6n3tJ1na+3fs15/c7DoBn4YvMXTOw6kfYxdnyuulzgUt8iQFQXqDSIySdA/3vc1mxmUSbTPppGWU0ZRs1Ij8QebD6ymYsHXkyXODvqMpuq1MOoNt2Oe70aSnZAaFzL9Z2N4W4JogvB7nfg8HyoyFF1+qrzYcdL0PkcaDeh/rjaMtj7EaScJEG/ANVwoX8XYKLJxGjUQv+GAUBrRgNzqqqYOGsWF8+cSWq7tpQfzuDZZ3pTXW1kyBA491zvvA+bdBN0udCti36Fm0R3hW5d7T7csmv0rrtgzx644w4oLa3fHJWVpRZd3303rFwJ33+vXr/9dgdqqong1Osm6H2rk7uQHGMwWA8Itqi2WO2YDsCazJru1D5s4ari4mLi4uIoKCgg3t7tUA7auRMeekhNtjEx8MADajeB2/w6AsoOqN0RBokfC2XzZvjf/5rf1h8dDf/4R/2qnuXL4cknVQD6jTe8MkzhT8wmdQObfALE97d+iNlMVlYWnTp1wuDUN3VjX36paqq1a6c+d506wbRpjYPRopWpLYddb0JEqqotEQjKMlWdtXBHc3OL1sBsNpOxL4P0rumEGK1fp7204iVu/+12hqYOZd11TuTmFsEt+xdYeDqMfkfVW3PBbb/exssrX+bGETdy65hb6f9af8KN4Wy5cQtd4+1/sCSCmNkE3yarh8Cj3HtD8PDCh3nkz0cA6BDTgeySbB6d9CgPnFi/Ytvha82yAzC7K/S+DYa/aPu4gvUQGgttbCysEK3DxoeheCuc8KVr7ZgqVW22Dmeoh6SmajiysD4jT3k2hMVDSJQKWFcdhShrRbNEoMjJyWHswIE8npfHJcDjqJp+v9px7imRkUy8917+dXIhYbuf59o5GZQbe/DKKw5kuGmGu+/RhZ8xValFK3bKz1cbTfLy1H8PGgRDh8JXX0F5uSpxUVQEJhNcconKOtdE7jJYfC4MfQ66Xeqe9yGEnzCbzWzbto0BAwZQVFREbGysR/uTSI2PefKLsVcvePZZ+OMP9TC7vTsXaOs69LwJagol4CcaGTBAbeffv1/t+lu7FlJTVcC5Wzf4/HMVkH7qKXURkJOjUgGA+rlV+7+E2N6QMNhr70N4kcEIPa9r/hCDgS5Wt4E658IL1R8h6hjCYONDkDgscIJ+6++G/Z/CBcUu78IRAUA3Q1U+RLSQCvEYg8FA7+69bf7crJt5bfVrAFw7/Fq3DFEEmfhBqm5ZtK0LNPsNbz8cDY3X1rzG19u+ptZcy7T0aS0H/Ep2weLzoc8d0P1yl8ch/JjBCOfmeiSN60MnPkR5TTn/XvZvskuyAbh4YOPveoevNSPawYCHIHl888et+yfkLoHz8hxLCSqCS/E29Tlw8CF6E8YI6Hhmg/8Oqw/41ZbBL4Og//3Q5zbVjwT8Al5qaiqhYWGkHvvv2cBtdp57eUUFL3z8KYk1/yHJFEp4VDj3PuCegB+4/x5d+JGV18K+j9R9piHUrlMSE+GZZ2DZMhg5sj6r3NixKrViVpb67xNPhIsustWKrhbJSEa51iNvhSr303eWV3b8+ZLBYKCzF7e3SrTGx8xms0fb79wZrhj9EmQsgvbfuq9hTYP0K93XnvBvmx9XE/GkH+0+pWtXuPpq9aehoUPhk09UcdWNG9Vrmgbdu8P06VYaMlXC0r9Bl/NdXxkp/Ju5Vv1tZSGB360iXHgGmGvgpLm+HolwF0MITF0AMbaDJH6nw2nqoaME/IKfrsOcnirF5slL7Dql4bxpxsymw5sY2n5o3c/n75lPRn4GseGxXDrIxkrayiPwY18VcBlwnzveiQgkUR1h3IduaeqywZdRY67hqh+u4kjZEYyakX+fbEdK7+oCqCkGc7VbxiH8nIfqNmqaxjNTn6GipoL/rv4vYzuNpUdij0bH2LzWzP8LirZAh9Mb76w3RthXh6rvv6DjWRLwa+3GvA/GSNceaGZ+D4lDVeo9aw79DvFDIH6A830IvxQXG0tOtlqwUAR1AcCWpABZWUX8vG4qsbFTeeIJSEtz37hcukdffTNEpARcjaxWI66/+u6qKXEoq0y7djBjRuPXOnaE559XtfxqauDGG5uZCpPHw7RlTg9bBKC9H0PGqyrTw/Gp1P3B9pfVfUi/f7nclNlsJjMz0w2Dso8E/XzMK9lVCzfBod+gugjC4tzTZlW+pBNrTYp3wNFVKv1dSJRLTYWEqCK+Y8bA1q1qd1/v3hAZaesMDSZ8A+FtXepX+Lm9H8PKq2HSz5B6UpMf67pOfn4+HZ0uQupmIdH1QUoRPBKH+3oEjul2qaQ9aS00DTrNUA+Z7WSZN0PjQxn85mAKKws5eMdBkqNVHuPymnK6J3TnjJ5n0CbMxsPo2lJ10y/fwcIN/m/o/2Eym7jtt9u4c+yd9Evu1/JJbUfC9D2eH5zwD+XZkP0jJE+AOHfWpVCBv5dPe5nTe57OoJRBTX5u81oz82vY8iScsdW5+98Op6g/onVz8R6amhJYNlNdq5682PoxnWeoPyLoTJ85k4+efJJLKiuJQ9X1s8dhQNPiiIuDJ55QC7Pdyel7dF2HA19AwhD3Dki4T59bgVvd1lxUFNx0k9uaE8Gk103Q7TKI9JNnfcfb9br62w1BP13XKSgocLkde0lNPx+x1PTLz88nwV17622pLTu2qsxNu2MKN8MvQ2DYC9D7H+5pU/g3c43dW/qFcEruMrWjtP+90O6EJj82mUxs2rSJgQMHYjR6ZhW4EOg6lO6GmqLACwAKcZyG8+aId0awPmc9E7pMYO5lc4kIUcFDs26mvKbcdtBPiMzvYfvzapdKTI8WD7dHtamaMGOYW9oSQebwQpg/GYY+D33v8GrXNq81S3ZD4Qa10+/4hRcrr4aKwzBpjvVGS/dBmzRPDVkEmuzf1DVmVydqDJhr4dCv6jOYOtX9YxN+LScnh66dOrHIZOJ37K/pN8kQSX6/e/lhzv2k5d2i5rPJP7ttXC7do5tNUFssaRxFY7vegppSr18DCGFTdaHKfhPby+WmTCYTy5cvZ8KECV6p6ecHOdKEx4VEuy/gByoA1P5UWZXTmvgy4Gc2+a5v4T3J49QNiJWAnxBeo5vgl2Gw9jZfj6RlB76G3ydA/jpfj0QEgA9nfEhseCyLDyzmiu+vwKyr9PIGzSABP9G82lKVNaRsv9uadCjgl/UD7PlQ1bQUwS9xOJw4B9Iu8fVI6sWkQ+dzre+0rsqDqiPWzyvZBT90gw33e3Z8InCs/xf89U/nzjWEqFp+EvBrlVJTU3n6mWc4KzycYcACYGUL56wElurV/PfVq1VKz4ocqMjyn+9Tg1ECfv5M1+Gvu9TCbG/KeAN2vOzdPoXvmWuheKevR2FdWLxbAn6+IEE/H9O8UaTSXAs589VOGndIHKpqu7Wb6J72hP8zVauHLkcWeb/v1dfBt6mqnototTRNIzU11TtzZktyl8LmJ6A8y9cjEe5mCIEhT0OfO309kpZVHlE770OifT0S4U2rb4alM+06tOG8OTBlIN9d9B2hhlC+3PIlnV/sTFVtVcuN7P0EtjylrgNE69T1b3B+AaRO8U3/O16Ctf9w7wJG4b9CY1RgIzLF613bvNY0NTNXTvweTrHx6N0QCt3/DimT3DRCEfCGvaTKVjiabKvqKBRneGRIInDcfued3PPkk5xjNNLdaOQMbAf+VgKnGsK5+R/PMnHisQqAJ3wJp2906/ep0/fo+X/B4T/l+tKfaRocnA2Z33m330k/qZIvonVZfC78PMj/5oSKHCjapjY/uYGmaaSkeO8aV+6efMzhYrfOWniaemgjhFN0WHyOb1bctEmH+IEQEuP9voV37f0EllxkdXenwWAgNTXVe3Nmc7J/hY33Q7X3cnELL+p1Y2DUQ+l1I5yfDzGBuepMOKl0NxRtsWuV9vHz5kndTuL96e8DkF2STb/X7KintvdDtcJXUny3XoYQ9eDHV4a/AhO+9l3/wjcqDns924fVa02zCb5sA8suc7zB6K4qLa7szBIWqSepOqWOzqm734Ufe6mF3KJVu/2OO9iflcWlDz9MVLt2jAcmaRqfAPOAj4HJIRGcoBm5+NonefGlBikSPfBd7vQ9+vYXYf4kMJW7fUzCjSb9Cqes8m6fke0hvr93+xS+1+VC6PtPMFf6eiSN7f8CfuqnFv67gcFg8GrQT2r6+YhXa/oB7P0I2nSH5PGutbP9ZbXaY8z/ILqLW4YmAsT+L9TD5cShvh6JCFZ//Qu2vwBnZaj5qgGTycS+fftIS0vzfU2/mhIo3g7xg0FqEgUvUxUYw309CiEac+BzaWvefG7Zc9w17y7+c+p/uGnUTc03UpkHFdmQMMiVUYtAV7QVsmZDr5sg1LO1J4Rg48Ow+RE4YwvE2bE4wU2szpk1JbDqOmg7Gvrc2vSkws2Qt1yl/wxvW/961VEIS/RtwFz4J3ON+nxEptp/Tv5f6h5p9DtybSoaycnJ4d133uH7Tz+lqKiIuLg4pl88k79dfDU9ehz3GTNVqmc64cnQ8XS39O/0PXrucij4Sy1iFMKithwqcyCykzxnEf4hdzlk/wi9/uGWLBSWOqhDhw71Sk0/Cfr5iNeDfu6y4T7Y+Rqckw0hkb4ejRAimFQXgCHC6tziUpFwIRyh6zB3nKozcfISX4/GOlMV7HxVpdluO8LXoxF+qrl5s6KmgshQuY4Tdtr+Eqy7Xa347nCK9/o114K5GkKivNen8L1Dc+HAV9B3FsT29Fq3Tl1rbv03rJ8F05ZD0pj6138bDdqx6whJTSsszCb4vhOgwxlbITzR1yMSrYm5Br6MhtSTVQpFN5B79Fbg6Goo3QNdL/J8Xzl/wB9TVJaH3jd7vj8hvMxkMrF8+XImTJjglaCfXIG2JrruepqUwU/A+Ucl4Nda1VZAdZH3+iveCatugLwV3utT+E5Ygv/PLbpZ7Xhorr6LCGyaBvEDIKa34zVXvKV0N/x1J2R+6+uRCF/Img0Zr7vUhF0Bv9oK9T0s853ocgFMWQgpJ3q33/x16gHl9he926/wrfbTYPTbXg34Oa3zuTDpF4jtU/+aqVrtDEwaKwE/0ZjBCMNehH53qfuelphrIc9W1TYhHGQIhfFfqLTDvqSb/fceSzS19jZYeY3bapo1K7I99LkTkkZ7vi/hf9beDvO8fK8R5OQqtLU4+BN8FQdZ37velty8tE4FG+HrBNj5Xy/2uR52vQGl+7zXp/Ctom2Q+b2vR2Fb+UH4qT+su9PXIxGeNPptGPOu/6blik6DqYug+999PRLhC9ueh7/u8ny9q4J18GNv737vC/8U1VEF/IwR3u03JAq6zoS4Ad7tVwiLA1/D6pugIsf6z2PSocOpEBZf/5oxDEb8B4Y975UhigCT9jfoc7t915j7v4C5Y2D3e54fl2gdOp8DEe18O4aDP8HXiWoRm/B/gx+HST/ilfBBXF8Y9pyqfSpan+oCqC5UqYj9xe8nqDJEASrE1wNo7TRvPVCM6qQmzpBo59s4/Kd6AJR2GUQkuW9sIjDE9lGF6GN6eK/PLhdAuwkQ0sZ7fQrfWnsb5C6C8wsb1azQNI3OnTt7b860xRAK/e6B5BN8Ow7RuoVEqblRtE7DXwRjdIuLsFyeN8PbqfR6Mt8JOLbTfZu6DvRWTan4ATD+E+/0JfzL9pch8xuYutBrC06tzpmHF0DGazDggeZP1s1qnKZKMITJIlnRstoyWHMzdLlIBY6taTsK0q+GTjO8OjQR5KoLVT3Sdq5f3zl1rWkIg/iBENnR5f6FF6RM9vUIRGsx5n3/WnRtqobKXKjKd1uTmqbRqVMnt7XXYn9S0883LDX9vJHD1W1WXQe73oLpByC6s69HI4QIRod+VzfBHU6X4s3CtzY/DuWZMOpNX4+kqap8CI1TaaKEEMIbtj2nVrpOWej9NJ+i9Vn3T9j7IZy2Ti1e9RXdDJWHISLV+oOo4gz4ZRD0+ScMfgw2Pgz7PobJv3p3oaQIPKX74Jeh0PVC/7zWFMFr7jhVruLcw95bxCMCm65DyU6I7e3ZftbcClVHYPxnnu1HCB/yZjxIlqD5mMnk4dRM7jTkWZjyhwT8hPfkrYSyA74ehfCm9idD5xlNAn4mk4nt27cH1pwpAtvR1XBwjn/WM1t8Dnzvw4egwvdqStXD5mbIvCncKvVkVWclIsV7fW5+AlbfLLV/WqPBT6gH0l4M+FmdMzWDqjFka+V5RBKEtYUeV6v/juunAoXRXT0/YBHY2qTB1D9hxGtNf1Z2AA7+KHOf8Iw+d8DQ59Rc5axj90cmk4kdW//CVF1m33kVh53vU/jOqmvhx75QmefZfoq3QsEGz/Yh/NuBr9VCwyBlMpnYuXOn1/qToF9rsu9TWHm18xePYXGytbu1qy6AP8+GTY96p7+Fp8OyS7zTl/Av1UVNXqqs9IPc3quugyUX+noUwhtGvwvTM/1zBWzKFOh6sa9HIXzpp36w5IIWD3Np3lx2Kaz4P+fPF8ElYbCqsxLXx3t9HpwDB3/wr1Q/wjuM4T75vTeaM3UdDi9U9ZxtCUuACV/XB/k6nA6nb1Tp4IVoScKg+qwNu96uX+y64X748yzY+4HvxiaCV5fz1UKFkEjHz60uhLnj1S7VYxIPPIfht+GN50prNbmKd8DsrrD13473K3yr45kqWKzXeLafk36HM7Z4tg/h3zLegI0P+MfC66OrVU3dqqNubdabzzWlpl9rkrsEdr8Lgx5TKxYdUZ4NpnJoky433q1ZaBzkr4XwZM/3pZvVZzUswfN9Cf+y6BzIXQznHvG/miil+6DKwyvchH/w59q1Ax/09QiEr/W6BTQPp3ct2Q3GCM/2IQKTrnvnfmDaMqhpughItBI581UQJP1K3/RfUwjzJ0P3/4Mx79o+LmlM/b9DpQ65cELBRrWTptN0mPg9DP03pM20XetPCHeoyofQWDA48Fg4LB66/x0O/XrsoXwI1SHtILRj/TPGJRfBkUVwTnbTa4WksZA83k1vQHhNp+nqjzfI8+bWbfiL6rmzPyy8zvwOtj4FSeMgvK2vR+MUCfq1JgMfhsFPqR17jtr9Nmx6GKathKRR7h6ZCBSaAc7e651aa5oBet3o+X6E/0kap4K9tWUQGuPr0TR20m+Saqc1KdwEWbOh3z1SP0/4l37/8nwfpyyX+U40lvkdrLsTxn/unfsBzSCLv1qzLU/A0VXQ7XLffAdrRhj+H8/XMBIiYZCaV1OmqP+OTIFICfgJD9r7Caz4O0z6WZXXaIluVteEBiP0uEb9ATCZyE26gtQBAzBaFuvGDVDf36ZyCImGjQ+qtPRDnoSpCzz2loQXWO4LPBGYqzwCh36H5HHQppv72xeBIX6gr0dQr+d1Af959LMtFK2PweDFX0FEO+cCfgDtTlSryhOGuHVIIgB5I+AnWrd+/4Ix7zUK+BkMBrp37+7dOdMWWX3Weuz/XKWXOLrS1yOpl/2bSjFbuMnXIxF+zi3zpsx3oqHQWJW2sLrA831VF0LOH1CZ6/m+hH8a9DicNN9r81CTOTM0FnrfAu2neaV/0cp1vci/s0yI4JI4XNXqNdqZ4nPjg6qmeE1po5fr5k1jg4UZAx+A8Z+pgJ/ZpNIkH1ng+QwVwrMOfA2z06Bos2faz18Lyy+FnN89074IHNWFcGSxr0ehUrd3PNOtuw4NBgNpaWlua6/F/rzWk7BK8+bDFF2HIicLo6ZMghH/kYCPUKu8dr2j/njSlqdUrviyTM/2IwKCpmnExsZ6d848XsF6ldNbHkC2HulXwdTF0NaPdrgXb1U3XeZaX49E+JJuhj9OabbmnkvzZuEW2POBWnUrhEXKSXDWDuhwiuf7yl0Gf0yBrO8835fwT8njIGm011K9+8W1phBCeENcH5j8M7Q7oeVjdbNKtVyR0yRw1+K8aTDClAUw6RepdRroDOGqDqSnSp0kDIMTvlbBaNG6Lb0Y/pgKtRW+HUdNidubtMyZ3iJBPxe8+uqrpKWlERERwejRo1m1apXDbZhMJg+MzBYd5o6FNbc4eJqkdhINaAbY8iRsfdqz/egmtZI8QHMnCxdtfQYWzaj7T5PJxKZNm7w8Zx4n8ztYeRVUHGz5WBEc2nRXN8OO1LrwtD63w4VlED/I1yMRvqQZVK2z2lKbh7g0b2b/qNI+le51fowi+HgzGBLXD4a/Au0mea9P4X/MNVBxyCtdNZkzNz2mFiBW5HilfyGE8IlCGzu3SvepOVgzwNgPYMo8FfRpwK5rTYMRIlPdN17hG53OgjO3Q8pkz7QfmQJdzgvoVIrCTXpeD8NeUM+EfcVUDV/FwbLL3NusycSWLVvc2mZzJOjnpC+++II77riDhx56iHXr1jF48GBOOeUUjhzx4xXRmkHV9Ot9q2Pn7XkPfuwH+X95Zlwi8JzwJUxb7tk+BtwP0/dBSJRn+xH+qXgH5C6F6qK6l3wa8ANVu2DCdxDbx7fjEN6l65C/DqqO+nok9UIipcagUN/DJ3zZ7CFOz5tdL4YJ30Jcf+fOF8GraDusuVXtfvekNmnQ+2aI7eXZfoT/0nWY3VWltPaSRnOmblIppqSupBAiWG15Gn4eBIePq7WXtxJ+6q+yL4Fa9BNqfXeKz+/RRXDQzb4egfAXnaZDr5sgtI3vxmCqUDWlk8e7v2kvzpl+tHQ9sLzwwgtcc801XHnllQC88cYb/PTTT7z33nvcfffdTY6vqqqiqqqq7r+Li4sB9cu2/MI1TcNgMGA2m9Eb7K6zvH78B8PW6waDAU3TrL5Ozxswm83Q4GeWugVmc+NJ1mg0ous6em01Wm055rB2YDJhNBqbjNFyvK2xe/Q9WRl7S+/J2uvynhx4TwnDgu89BePvKZDf09CXMIx+F9Ox+cpkMtX1ZTQaffOeIjqgdzhLvXCsn1b/e2oN7ynrO4xLL8A84jUMvW7w7XvSdbTMr9DjB2NM6Ce/J3lPaDS9cbCM3TJvWn7u0HuK6Iixcxc1xuPal99TK39P5Ycw7vwPRHfGFDswON4TQfh7CpL3ZO56KYTG1M1DnnxPljbrXuv/gPoDGI+9Lr8neU/ynuQ9BdV76jgDY/4azDF9Gl/vxQ7EmDoFc9K4Rq8f/55aukeX31OQvae8ZWgHvkYb9DCExbv1PRkWnAzlWZjP2Ca/J3lP6j3ptZh0Q5PXvfKejG1g1LtqjGaz294T0KRPT5KgnxOqq6tZu3Yt99xzT91rBoOBqVOnsny59d1PTz31FI888kiT17dt20ZMTAwAiYmJdOnShaysLPLz8+uOSU1NJTU1lX379lFSUp9TtnPnzrRt25aMjAwqKyvrXu/evTuxsbFs3bq10Yesd+/ehIWFsWnTJrVy8liKnoEDB1JdXc2OHTvqjjUajQwcOJCSkhL2VIyFtO9hVy4RESX06dOHgoICMjPra63FxMSQnp7OkSNHyMmpT4Hi1ffUQIvvac+eutcjIiLkPTnznqJ1sjd8w9Hw0e5/T706ULXpBfbX9KMisr/33lMw/p4C+j3F1r0nXdfJz8+nsLCQ5ORk77+nXj0oPLyXA4fL5PfUyt6T0ZRCSuJlhIX2Ix58+p5CavMZsHMmxckXEXvy5/J7au3vqWsHYo98zsEjleTHTGvynrZs2UJ+fj5btmxB0zSH3lNkqJne/YfJ70neU5P3pOltSJ+0mjbth7N182aPvSfznH4Uh6RzoOMT8ntqxe9pZ9T/qfd0bKyefE9hYap2fWFhIQcP1qdyl9+TvCd5T/Kegvo9TfiagqNHydv4NVEVmziacL56Tyf+wJGcHHIatH/8e7Lco+fm5tKhQwf/eU/B+Hvyg/eUdHQOnQ7/h6rkqYR0Pt2t76lDbQdCjJEU7Nsnvyd5T/St/ojwfW+xvdc8aggPivfUvXt3oqOjKSwsxFs03ZshxiCRnZ1Nx44dWbZsGWPHjq17fdasWfz555+sXLmyyTnWdvp17tyZo0ePEhcXB3gpyl+RA/MnoXc8C33wM/WvEwBR/mBcuRDI72nRdPRDv2Keng3hie59T0eXwrwTMQ95Dr33bd57T8H4ewrk95S3DHPBRvQe16HrOpWVlURGRvpmp1/xZvhlCOaBj6P3u7vR8a3+9yTvyXvvqaYU7dDP6FGdMaacEBzvqYXX5T01857Q0b5JQE8YhvmkP5qM3WQyUVlZSURERF0bdr0nUxWGb2LQul2BedQ78nuS9+T996SBvuAU9Ni+6MNeCo73FIy/pyB7T7quU1NTo4J/pmq0Hc+jJ58AyRMC9j1ZXg+m35O8J3lP8p48857Y+CDatmcwn7UXItvb9Z5aukf39XsKxt+TT99T5RGoKcIQ2ws0LTjeUwtjl/fko/e0+w20rB8wjXgNotO8/p7Y+zHkLkIf+BhaZIrbfk8Ahw4domPHjhQVFREbG4snSdDPCc4E/Y5XXFxMXFwchYWFdUE/r9DN8Msw6HK+qpnWkuxf4PBCVQcwqoPHhycCSM48KD+oPksh0e5tu7YCirdBRKp87lqzRefAwR/g/EL0kDaYzea6CxqvK9oG256FtEsgdar3+xe+p+tQW2KznoUQPpG3CmJ61C2+achyg+PwvFmVD2tvVTUMel7vxsGKoGGqhNwlENkJ4qTOrfCgmhJYfgUkDrPv3tUFjebM0j0wpwf0/ScM/bdH+xVCCJ8zVcKmR6DzudB2pN2nOX2tKYQQ/mzVdbDrLTi/AMLi3dasrusUFhaSmJjolaCfoeVDxPGSkpIwGo0cPny40euHDx8mNTXVobaOjzZ7nGaA09fbf9OUNVs96BbieKlTofsV7g/4AYREqpt7Cfi1bv3vhZOXgjESs9nMpk2bvD9nWsT1hTHvS8CvtaopgR+6w+qbfDsOKXAujpc0ymrAD3B+3gxPhHEfScBP2Fa8E/44GXa/4+uRiGAX0gbylqrPnIc1mjOjOsEpq6GHzINCiFbAGAFDnnIo4AcuXGuKwFVTAocXqAWx7pK7HFZdrxZaC+EPRr4B5+VBqHs3aZnNZrZs2eLWNpsjQT8nhIWFMXz4cObPn1/3mtlsZv78+Y12/gWFka/B6Zsl+CJsq8yF2nL3tlmWqXb7idat7UhIGgMGKT8rfCw0BhKHqx1VvrTkQpjdDcymlo8VrYOuQ+k+KNru65GI1iR+IAx7Abpf6Zn2s+bA5sehusAz7YvAoWkwPRPGfejdfo3h0HYExKR7t18hhBDCn218AOafBCW73Ndm7iLY9aYKKAphkTUH/jwbaoq937emQXhb9XcAk6Cfk+644w7efvttPvjgA7Zt28YNN9xAWVkZV17poZtfdyrdB3/9S6XtbIlmgPj+nh6RCFRZs+G7VMj63r3tLjwNfh7k3jZFYDLXuveC0llLLlIPIEXrNeFrGPiQb8cQ00M9bDcYfTsO4T9qS2FOOmy8z31tZryuVtvWlrmvTRFcNA363O65e4TMr9VDJQL7Rlu4iTHM+31WF6h0d0IIIYSo1+UCGPqce0te9J0F0/dD4lD3tSkCX+keVXKscJN3+9XNkP0rlB3wbr8eINsnnHTRRReRm5vLgw8+SE5ODkOGDOHXX38lJSXF10NrWW0ZbHsOtFBImWT7uIL16ti2o8AQ6q3RiUCSNBY6ngWRbt4JmjbTvekCROD68yzIXQznHPXdGEzVcHh+y8eJ1kHXfbfia8jTvulX+K/QGBj4CMQNcF+bh+ZC9s8w4lX3tSmCk7lGrcq2kWLWaSNeUfXE3VhDQwSwmmLI/BbapEO7Cd7p869/we534fx8CEvwTp9CCCGEv0ser/64k6ZBdBf3tikCX/qVkH4VhLbxbr8VOWojSs+bYOR/vdu3m2m6Lk/WfaG4uJi4uDgKCwuJi3NvjtgWmU1QshNiejW/W2DZ5bDvIzgnByIDIJgphAg+u9+Hkgz0fndjNkb7rki4roOpAkKivN+38B8Zr8OOl+HUtZ6pZyqEG+m6jtlsdnze1HWoOgoRSZ4bnAh8VUdVuuGuF8Joqe0nPKjisMos0v1KGPOex7ppNGfueR+O/Alj/hfwqZ2EEMJTnL7WFIFPN6vMcK6qOAT5f6lAYpiXn40LYU11Iez/AuL6QruJbm1a13UKCwtJTEykqKiI2Fg37pi1QtJ7tkYGo/rwtpQerPctMPwVCfgJ++hSvFl4QPqVMORJCI2hurrad+PQNAn4CZVu1lStUk1429HVsOZWKPRe4WcRYKx8Dzs1b2qaBPxEy8LbQsczIH6Ie9s1Vap5TlIrCovIFJg4GwY96vGu6ubM9P+DsR9IwE8IIVrg03t04RsbH4RvkqC23PW2Dv4Ef54hmZWEdWWZsOstqC7yXp9h8dDzOrcH/Cy8OWdK0M/HzGYfBUpqSuHIYlWvwJa2I6H3zd4bkwhMpmpVyHfZJe5pb/uLsPAMtapXiGPMZjM7duzwzZx5+E+VS9xc4/2+hX/peT2cvVvV1fO23CWw8z9Qlef9voV/qy1XdXBXXt3oZafmzeoild6z8oibBymC0vjP3H+vULAefh6gdlULYdHpbIjq5NEufHqtKYQQAUjmzVYqrC3ED4KqXNfbSj1JbTZJ9lL6bhFYsmbDquvgyCJfj8QtzGYzGRkZXutPgn6tVebXMG8i5Pxh/efVRbJzS9jHGAaGMFUj0h3Zgsv2q3Q6Uj9DWKy/F8PcMb6r87j1GVh0DiCrvVs9Q6jvVv33vhWm74Ok0b7pX/ivkCj1nemO782CdbDgFNj/uettCeGM8GTofy+0m+TrkQh/U3UUqvK90E8eLDwT9sk8KIQQQjTR51aYuhCiu7reVpvuagFZRLLrbYng02k6nPAVtDvBe32u+yf8NECl+QxwEvRrrdqdCEOeUaszrFl9I3ybonYECtGSST/DuA/d8zB8+EtwfpEKJgoBUFsC5kqM5mLf9D/wYVXTxRDim/6FfynPhjX/UKvOvEkzqBsrY4R3+xWBYeqfMOx519tp0wNGvAqpU11vSwS/qqOw+ALY/pL72oxJh8FPyAIH0VjeCpVKbPe7nu+r/CDk/A5lez3flxBCCNFamWtU5jAhbInuDF3O9+6mkJAo0E0Q6tl6e94gQb/Wqk036DcLYnta/3nCIPXAJ7SNd8clApM7Cvg21FK9SdG6DH8Z86nrISzRN/0njYK0v/mmb+F/DCGw8xU4OMd7feq6SjNb6YYUKqJVMRod/D6N7gy9boS4fp4ZkAguobGQ/TMUbfX1SESwixsAaZdB/ACPdmM0GiFhMFxUAX3u8GhfQggRDBy+1hTBoXgH/DENMr9zvo2cP+DrONj/hfvGJYJTxSH31JC0x6BH4cxt7n/OfYw350xN132VL611Ky4uJi4ujqKiImJjfRg9rikGY5TsYBGuy1sJa2+DIU9ByiTn2ijdA4d+hw6nqwePQviaqRo0owSiRWNl+yGqi/dSfVYeUbvv06+G0W97p08RWExVsPlRiEiB3v9wvh1d910KWxGYTJXu3YH853T1OR79lvvaFEIIIYQQ7lNxGOakQ9+7YOADzrVxeCFseUo9Q0wc5tbhiSCy9xNYfilM+BY6n+Pr0bjMm/Eg2ennYz6NuWb/Bt91gqzvG79uNvlkOCLAhcZB4UYo2el8GznzYfX1ULDebcMSwUHP+oGq1Xd7f87c+yF81UZdkAphEd3Vu4ERQygMexG6XuS9PkVgMYSptHcNavHpuk5xcbFj8+Yvg2HJhR4YoAha7k45XHEQKnPc26YQdqibM/NWwuEFYK719ZCEEMKvOXWtKYJDZArMyHI+4Adqs8BJv0nATzQvaTR0uxwiO3i+r+IdsOEBKNzikeYtc6a3SNDPx8xms+86TxgCsb3VTj+L6gKY3Rl2vOKzYYkAFdcHzjsCPa51vo1OM+DEOZA83m3DEsFB3/c5YRn/xlxZ4N2OI5IheSJEp3m3X+HfasshZx4UbfdOf2EJ0Oc2qbMmbNM0mLYCpi6qe8lsNrNnzx77rzXNJojsCOHJHhqkCEqVebD3Iyjc5J72Tl0DE71cM1UEhtxlMHc8ZP/qkeYtcyZbnoY/TvZIH0IIEUwcvtYUwSUs3tcjEK1BTA8Y+4F36n3nLoEtj0PpLo80bzab2bdvn0fatkaCfq1ZZAqcuho6nl7/WtkBVTcrNM534xKBKyTatfMjkqHjmRDuo9ptwm/pff/F1p4/QpiX56ZO09XqszZp3u1X+LeSDPVAcO8Hvh6JEPXapLmWrt1ghMm/wMhX3TYk0QqU7YPll8OBb9zXpqSYFdYYI1T9yIpDHu3G3OsWGPWmlL8QQgghWnJkCcwdByUOBkmKd8CicySjkvAvaZfCaRug3SRfj8QtJOgnFEtBzITBcPomSLvEt+MRgWvP/2DxeaoukCN0XeUFF8Ka+EHUhLb39SiEUOIGwLCXvPddufgCmH+Sd/oSgUvXIX8dZMkuKeFF8QNh/BfQ42rX28pdrtLUVnt5V78IDAlD4PyjkH6lZ/tpdyKkX+XZPoQQQohgoNdAwV9wdI1j5xVvh4M/QnW+Z8YlgsvR1fDrSNj/hWf7MYZDwiDvbzbwEAn6Cdj6b/iuPVQeUf+taWq1txDOKFiv0t6V7XPsvMoc+C4V1t7uiVGJIBAZZoDMr9WqMG+oOKxqWx38yTv9icBhMEKfWyF+gJf6CwFNdhyIFmgaLLsEVl1XV585IsKBemt7PoRNj0FthYcGKIKSMRy6XghRnVxva/+nsPJqqC5yvS0RfDSD+uNBDs2ZQgghZN5s7dpNgumZkPY3x87rNF0t5Gl/qkeGJYJMVCco3QOVuZ7tp2BD/aYoD/HmnKnpUnHVJ4qLi4mLi6OoqIjY2FjfDubAV7D9JXXTHt0NRr4OxjDfjkkErqp8MEZCSKRj55VlwubHoP0p0OU8z4xNBLa8lTB3DPS6GUZ4oe7okUUwbxIMfxl63+L5/kTgMdeCXqtSjgnhDw7+DKGxkDzO8Yfj86fC0RVwQbHHH6yLIFRdqOZCV+bD8iy1UrzTdEnxKawrz4Y970P7adB2pPvbL9oGv58AAx+Waz8hhBDCEbou12/Cc8wmz25QqimGr+Kg60wY/4nHuvFmPEju6H3MLwredj4fTl6i/l20RQJ+wjXhiY4H/ACiO8PotyTgJ6wym80cJR3ziNeh//3e6bTdRLiwBLr/3Tv9icCSuxS+jlcpjYXwFx1Ph3YngGZQ8+bRo/Zfa544G05ZJQE/4bg9/4OvE+Dwn661E9UJOs+QB0bCtqpc2Hi/R9I7mc1mCgvy0OMGQHiS29sXQohg4/C1pgheB3+CX4fZV7KnZBfsers+25wQ9vB0RkLdBAMfgS4XeKwLs9lMfr73UtrKXb2P+cVGS01Tf6b8ASfN9fVoRDCoyoetz0DOfF+PRAQJXdfJzMpCT78GIlO813FINITGeK8/EThiekLbUZ5/MJi7FLY8rXY3CGEPXYf8v9DNZjIzM+2/1gyJhrh+nh2bCE7xg1WN07B459uoLYPKPLcNSQSp+EEwZQEMftLtTeu6zr7CGMwn/QFpF7u9fSGECDa6rjt2rSmCl6lSpV8s3NDysZnfwqproXCj58clgofZBJufgC3uvwYEICwBBj6oFiB6iK7rZGVleaz940nQTzQW6uNUoyI4mCpg/T0q/Y69ll+hzhGiJbVlkP2b5/s5+LNK8ySENRHt1GKZLud7tp+DP8GGe6C6wLP9iOCx6WG10rbgL/vPqTgEBRvBVO2xYYkgljgUxn0MSaOdbyP7Z/g2GfZ97r5xieCjaZAySTLTCCGEEP6k8zkwfb9Kv92S9Ktg/BeQNN7z4xLBw2CEA1+oDCOy0MAuEvQTQrhfVEeYughGv2ff8boOhxdAwXqPDksEiRVXwcLTVB1ITzGbYMl5sO52z/UhhD363QUnL4PYXr4eiQgUHc+CPnc4tutq/+fwy2DIW+qxYQnRrMiOKp124lBfj0T4O12Hwi2q1rObJRb+gLb1KdAlVZ0QQghhN81Qf+/RUkAmvC10vdC5skCidZs4G87Y4plSACuuhEUz3N+uD4X4egBCiCDV7gT7j9U0mHEAais8Nx4R8GJijqXZ7HUjtJsAYXGe60w3wZj/QagH+xCBr2C9Si/R8yZIOdEzfYTFQfJYz7QtglPbEeqPyURMzD77zkkaD/3vU2kahXDG3k9g91sw8XuVHsdRyePUHyFaYq6GuaMhYSicvNitTSeX/IiWtxsGeql+tBBCBLi6e3QhQGVs2PwwTJ4L0V2a/rw8WwUII1O9PjQRBNp081zblXlQ5flSA23atPF4HxYS9PMxo9HDhSiF8KXSPZAzD3pca9/xstJH2GA0GklPT1f/0W6i+uPRDsOg60We7UMEPnMtHPgakk/wTNDPXKsKncekgyHU/e2LoGY0GEhP6wT2XGsmjVJ/hHBW5WG1EKJ0LyQ6EfQTwl7GcBj2otsf/BiNRiJP/ll9loUQQrSo0T26EKBqhFflQeUR60G/7S/A9ufhjG0Q18f74xOBr3iHKgvQ+zb37vibNMfjaUONRiPdu3f3aB8NSXpPHzObJXWICGKbHoFV10HxTtvH6DpsuB/yVnlvXCLgmM1mcnJyGs+ZNSVQU+yhDk2eaVcEl4QhcEEh9P6HZ9ov2Qk/9YWND3qmfRG8ineiz06jZPXDcq0pvKPXTXB+ASQOc/zcisMwd5zaLSiEPXpcA6lT3dqk2Wwmp7AWc2xft7YrhBDByuo9umjdOp0FZ+1SWUesSZmkNgVI6QrhrIw3Yd0djtWvt5cn0oY2YDabOXzYe4vLJOjnY7oUnxTBrM8dMOkXaNPMSoaizbDlCdj7offGJQKOruvk5OTUz5l5K+CbJNj1jmc6XHIB/NBTgn+ieYYQCI31XPsh0dB3ln0F0YVoqE03CE+muKy25WvNkt0wuxvsftc7YxPByRiu0jU5o2w/FG+H6gL3jkkEN12Hihz3NVdTTsH+lehSbkAIIezS5B5dCKiv7Ve6T9VJqy2r/1nHM2HUm85fMwrR8zqVPjZ+oPvazF0OO19TKT49SNd1CfoJIYJEwmDocKp6MG5L/EA4fSP0vdN74xKBL34wpJ6s0h56QpvuEN8fDJKCWbSgtgL2/A+yf3N/29FdYegzkDLZ/W2L4GYIxTxtJbltL2n52OoCCGkDhnDPj0sEt+oilb1h32eOnZc0Cs47qm7ihbDX0ovgx95gqnZPe4V/0Xf3DLSMV93TnhBCCNGa7f1A3SeXZKj/luCwcIfY3tD+ZPeWP8n8BtbcBNX57mvTD0hNPyGE55XuhY0Pwcj/Wt8V484VGqJ1CImEST96rv1hz3mubRFczJWw6npV16/DKb4ejRCOazsCztjk61GIYKAZYOd/1XyYdrGD52qgSe1S4YD2p0BYItQUgrGd6+2FJ3O47d9JShrveltCCCFEazfgAfWsL2GI+u+Fp6kMI6dvVM9zhHCWrqvaflGdILSN6+31/afaVNBclroAJDv9fEzzcL5YIfxCzu+w7yPImtP49SOLIWce6JIDXjRP0zQSExOtz5myYkz4UlgCnPgDTPjKve3WlsEP6bD9Rfe2K1oNTTeTXvAyho33+nooorUIjYGTl8LE2Y6dl/EG5C71zJhE8Eq/Cka9ARFuCPgBWmwvqvo+gpY81i3tCSFEsGv2Hl0IzQCdz63/74hUiO4iAT/huj3vw099IWeue9qLTFULuJvLUucGmqaRkJDg0T4akp1+PmYwSNxVtALpV6t0jEmjG7++8UHIWwbnZEN4W9+MTQQEg8FAly5dGr+o67DkQhUcmfyz+zrL/gUO/aZqUkZ3afl4ITxRc688C4wRsihCOM0QEkpMxV9Q1cLuqc1PQFzfxjflQjgrvr9jx9eUwuobocsFkCw7rITvWL3WFEIIYZPMm8IhY//n6xGIYJEyGXr9A9q4odyPqRKq8iCyo8o84kEGg4HOnTt7tI9G/XmtJ2GV2SwP80QroBnqA35lB2DhGZC3Aka9BWPel4CfaJHZbObAgQON50xNA479t9nkvs4O/Q47XgbdjW2K4FddCLveBnONe9qL7Q1nbFHBZyGcYDabyer9IeaTV9g+qDIXNj0IB9y8U1W0buXZsOYf6lqvJcZwmLoI+v7L8+MSwWfTY+q+wg30xedTOn+m3J8LIYSdrN6jCyGEp7XpBiNehoTBrreVtxy+7ww7PV/T2Ww2k5mZ6fF+LGSnn4/pkpZOtDZHFsOhX6HnjZA0BmJ7+npEIgDouk5+fj4dO3Zs/IMTvlJBZXca9hz0vhmiu7q3XRHcMl6DDfdBeDJ0nuG+diVdjnCSruvklZhp39xBEclw9l4wVXhrWKI1qCmCna+ofyeNaf5YQyi0O8HzYxLBqWwPFKxXO0ZdrelSug9zdbjcnwshhJ1s3qMLIYS36Lprz0zC2kKP6yB5nPvGZIOu6xQUFHi8HwsJ+gkhvKvbJZAySeXzFsJVloCfbnZf8E8zBF0BX+EF3f4OEe0hdarrbZmq4K9/qnR37Sa63p5o3Y4sgqL10Od26z+XNMbC3eL6wumbIa5fy8dWHIbwJDAYPT8uEXxGvqF2i7qBedpK9mzcyEC3tCaEEEIIITzm6GpYdgn0vw+6X+F8OwmDVI3oICTpPYUQ3hfVUR7uCPcp3gG/jYKsOa63VXYAcuZBbbnrbYnWJaoDpF/p+k4DgPy1sPO/ame0EC4yZPwX1t0JlXmNf1CcodJ6mip9MzAR3OL727fqduHpMEeyPggnuSngV0d21wshhBBC+L+oLmCuBb3W1yPxWxL08zFNbiyEEKJFmqaRmppqfc4MaQOle6Fkh+sd7f8C/jhZ5fUWwhmVuZD9m2ttJI2Fs3ZB+v+5Z0yiVbLMm3r/++G0v5rWz939Fiy5UAWZhfCE/L9UwNlW3V1dh04zoOuFXh2WCDLZv8KeD11rI3cZ2r4PaZ8UJffnQghhp2bv0YUQwpMiU2D6Hki/yvk2qo7CL8Ng9/vuG1czNE0jJSXFK32BpPf0OYNB4q5CCNESg8FAaqqNlLBRHWH6XgiNdb2jjmepv5PGut6WaJ3+PBOKtsKMAxCW4FwbmgYx6e4dl2h16udNG3NnnzugTTokeb5+gWilsr6D7S9Ax7Mh5cSmP9c0GPiA98clgsumh6B0N3S7zPmdevs+xZDxKikzskDuz4UQwi7N3qMLIYS/KzsAlYehttQr3RkMBq8G/TRdKlX7RHFxMXFxceTn55OQ4ORDQSGEaCVMJhP79u0jLS0No7GZ1LCmaqjIhjZpXhubEI1k/wq1JdDpXOfSGJtrIWe+KiQdGuP+8YlWo9G8qZnh8EKI6QFtuvl6aKK1qMiBikOQONT6z3Vd0ikK1x1ZDCFRkDDM+c9T6T5MR9eyr2Ywad26NX+tKYQQAnDgHl0IITyhMhe2Pg1tR0HXi5xvx2zySgkqk8nEpk2bGDp0KEVFRcTGumHjQjNkGZsQQoiAUFJS0vwBZhP8Ph4WngamKsc7qC1XARchXNHhVOhygfMXjfnrYOGpsPVZ945LtEp18+a+z2DBNFh/j/rv3OVQW+a7gYnWITLVdsCvpgRmd4FtL3h3TCL4tJsAicNdCyC3SYNOMygp9c5KbyGECBYt3qMLIYSnGCNgx8uQ9YNr7Xgh4GdR6sVrTQn6CSGECA4GI6RdAl0vBt1G/aDmZLwBXydKfSvhHmX74fACx8+L7gzDXoQu57l/TKL16n45TPoVRryiFjgsmAYLT/f1qERroOtql2nmd41fL8+CsEQwhvtkWCLImKqh/KBz55proLrIveMRQgghhBCeFRoDZ26HcR87d/7WZ1XGiCAlNf2EEEIEjz63OX9uZCokDoOY3m4bjmilzCb4fSLotXD2Hsceake2d+1zLIQtHU5Rf9eWwZCnIcJ79QREK6abYNmloBmh03TQjq05jesLp28A3ezb8YnAp+swu7O6fjt5kePnF6yH30ahDX4WONndoxNCCCGEEJ4S08O58yoOwfq7IP0qlTUiCEnQz8c0qWMhhBAt0jSNzp072z9n1lZA8XbbaRy5/f4AADz2SURBVMWsSZup/gjhKoMRhj0HoXFgCLP/vOoilZ4s1LO53UXrYHPeDImGXjf5ZlCi9TGEwOh3VfpEzUqSGWuvCeEITYPuV0FYnHPnGyOh2+WQOIzOYQ5cawohRCvn8D26EEJ4Qu5SlWnJked54Ulw6hp1HeglmqbRqVMn7/Wn67rutd5EneLiYuLi4rxSuFEIIVoVXYffRkP5frXLKiTa1yMSwj6bHoVtz8LJSyFhsK9HI4QQ7mc2Qd4y2PM+9LsLYmV3vRBCCCGEEMJJvwyD0j1w7hEwOrDo2ge8GQ+SpZU+ZjI5UXdKCCFaGZPJxPbt2+2bMzUNev8D+t0Lmp0b2ne/D0sugrJM1wYqxPGKd8Cq68Bc2/KxcX2h3SSI6+/xYYng59C8KYQ3HP4Tfu4Pe95TQT975kUhvETmTCGEcIzMm0IIvzDsBZjyBxhC7T+nZJeqCe1FJpOJnTt3eq0/CfoJIYQICJWVlfYf3O1S6HOr/bXUCjdC1ncQFu/U2ISwac/7sPsdOPJny8d2uQAm/ajS4QnhBg7Nm0J4mrkKKnKgy0Vw9l6I6+frEYlgYaqCRefA0kscP3f539VOe2TOFEIIR8m8KYTwuZRJkDhMbQCwh67D3HEwf5InR2WVN+dMeaokhBAieFUXqoBeu4nNHzf8RRj0KITGeGVYohUZ+Ah0mgFJY5o/rrZMUtEKIYJb+2kwfZ8ssBHuZwxX36GVuVBbDiFR9p2nmyH7J0ga69nxCSGEEEIIz9HNqrZf29Etp/g010DPGyAixTtj8xHZ6SeEECJ4/TEV/pyugn+21JarvyXgJzzBGF4f8KsugMLNTY+pOATfpsDWf3t3bEII4W0S8BOeMvpdOOk3+wN+AJpB1X8Z/5nnxiWEEEIIITxr+4swbyIc/qPlY41hMOgR6HWj58flQxL08zGDQX4FQgjREoPBQPfu3R2fM/vfD8NfgpA21n9+4Gv4sbf1QIwQ7qTrsPgC+H1809qRVUchcSTE9PDN2ERQcnreFEKIQNQwpXt1kf3naRqERMucKYQQDpJ5UwjhNzrNgP73Qmyv5o/TdceuE93IYDCQlpbmtf4kvaePafbmmxVCiFZM0zRiY2MdP7HzjOZ/rptBM0paReF5mgYD7oPDCyC6s9phmvO7SimRNAamLvD1CEWQcXreFEKIQLbsMji6Es7YAobQ5o/NX6sW3rSbiGaMkDlTCCEcINeaQgi/EZMOg59o+bjcJbDwNBj9HnS90PPjasDbc6Ysx/Axk8nk6yEIIYTfM5lMbNq0yfk5syIHDv5Y/9+1ZervrhfCmTugTTfXBylES1Imq9qRAKYKWDQDdr3p0yGJ4OXyvCmEEIEotg8kDIGa4paP3flfWHAK1JbLnCmEEA6SeVMI4Xdqy1RtP1t0E8T0hoTB3hvTMSaTiS1btnitP9npJ4QQIiC4dDPx59lQtAWm74eN96vVPaf9pVaAN0wHJYS3hLeFsR9BwlBfj0QEMXkII4Rodfrfq3bX26PnTZA0DsITwWSSOVMIIRwk86YQwq8sPg+OLIYzNkGb7k1/njIJTlvr9WFZeHPOlKCfEEKI4DfkSbXiJ7wtRLaHqM5QXQgRyb4emWjNul3q6xEIIYQQwcUS8NN1qM5X1362tB2h/gghhBBCiMA34AE4vBCi05r+rKYUQtt4e0Q+I0E/IYQQwS91av2/+98PmsH+VeBCCCGEECJw6Lqq11KeCadvUtd9xyvLhMjUluv+CSGEEEKIwJA8Xv2x0HX17K8iB+b0ggH3Q79ZvhufF0lNPx8zGORXIIQQLTEYDPTu3ds9c6bBKAE/IUTQc+u8KYQQgUTToO0oSD4BasutH7PsYvihO5hVmiWZM4UQwjEybwoh/JapClZeA/MmqP+uzofEoRDT02dDMhgM9Ozpvf5lp58QQoiAEBYW5ushCCFEQJF5UwjRag161PbPdB06ngU1xWox2DEyZwohhGNk3hRC+CW9VpX0CU9S/x3XD6b+qa4Bfcibc6Ysx/Axs9ns6yEIIYTfM5vNbNq0SeZMIYSwk8ybQggB6GaoLmr8mqZBv7tg8BN1L8mcKYQQjpF5Uwjht0KiYcJXMPH7xq/7MOuX2Wxmy5YtXutPgn5CCCGEEEIIIYQILjUl8PNAWH1949d1eUAthBBCCCGClwT9hBBCCCGEEEIIEVxCYyC2H0Sn1adzqi6C79rDtud8OjQhhBBCCCE8RWr6CSGEEEIIIYQQIvhM+Kr+37oOFQchqguExvpuTEIIIYQQQniQpus+rmDYShUXFxMXF0dhYSFxcXG+Ho4QQvg1Xdcxm80YDAY0H+bgFkKIQCHzphBCNGCqhsXnQNeLodulKgDYYG6UOVMIIRwj86YQQthP13UKCwtJTEykqKiI2FjPLkCT9J5CCCECQnV1ta+HIIQQAUXmTSGEOKbyEBRugtzF6r+tPKCWOVMIIRwj86YQQtjPm3OmBP18zGyWIuJCCNESs9nMjh07ZM4UQgg7ybwphBANRHeFU1bDiFet/ljmTCGEcIzMm0IIYT+z2UxGRobX+pOafkIIIYQQQgghhAhukSm+HoEQQgghhBAeJzv9hBBCCCGEEEIIIYQQQgghhAhwEvQTQggREIxGo6+HIIQQAUXmTSGEsJ/MmUII4RiZN4UQwn7enDM1Xdd1r/Um6hQXFxMXF0dRURGxsbG+Ho4QQgghhBBCCCGEEEIIIYRwM2/Gg2Snn49JzFUIIVqm6zrFxcUyZwohhJ1k3hRCCPvJnCmEEI6ReVMIIexnmTO9RYJ+PmY2m309BCGE8Htms5k9e/bInCmEEHaSeVMIIewnc6YQQjhG5k0hhLCf2Wxm3759XutPgn5CCCGEEEIIIYQQQgghhBBCBDgJ+gkhhBBCCCGEEEIIIYQQQggR4CToJ4QQIiBERET4eghCCBFQZN4UQgj7yZwphBCOkXlTCCHs5805U9Ol4qpPFBcXExcXR1FREbGxsb4ejhBCCCGEEEIIIYQQQgghhHAzb8aDZKefj0nBWyGEaJnZbObo0aMyZwohhJ1k3hRCCPvJnCmEEI6ReVMIIexnNpvJz8/3Wn8S9PMx2WgphBAt03WdzMxMmTOFEMJOMm8KIYT9ZM4UQgjHyLwphBD203WdrKwsr/UnQT8hhBBCCCGEEEIIIYQQQgghApwE/YQQQgghhBBCCCGEEEIIIYQIcBL0E0IIERBiYmJ8PQQhhAgoMm8KIYT9ZM4UQgjHyLwphBD2a9Omjdf60nRJvuwTxcXFxMXFUVRURGxsrK+HI4QQQgghhBBCCCGEEEIIIdzMm/Eg2ennY2az2ddDEEIIv2c2m8nJyZE5Uwgh7CTzphBC2E/mTCGEcIzMm0IIYT+z2czhw4e91p8E/XxMNloKIUTLdF0nJydH5kwhhLCTzJtCCGE/mTOFEMIxMm8KIYT9dF2XoJ8QQgghhBBCCCGEEEIIIYQQwn4S9BNCCCGEEEIIIYQQQgghhBAiwEnQz8c0TfP1EIQQwu9pmkZiYqLMmUIIYSeZN4UQwn4yZwohhGNk3hRCCPtpmkZCQoL3+tMl+bJPFBcXExcXR1FREbGxsb4ejhBCCCGEEEIIIYQQQgghhHAzb8aDZKefj5nNZl8PQQgh/J7ZbObAgQMyZwohhJ1k3hRCCPvJnCmEEI6ReVMIIexnNpvJzMz0Wn8S9PMx2WgphBAt03Wd/Px8mTOFEMJOMm8KIYT9ZM4UQgjHyLwphBD203WdgoICr/UnQT8hhBBCCCGEEEIIIYQQQgghApwE/YQQQgghhBBCCCGEEEIIIYQIcBL08zFN03w9BCGE8HuappGamipzphDi/9u78ygpyzN/3HdVQwMKVdCoNAoGUSMuJG4JtkmIJgQ0TE5MHBOXGHWIjkadoEaj+WZckhn3nCSOccs5ESaTzBgzRydxwWFcj8q4EI3YCApoQKXB2HQXS1isen9/5EeNLWC/ZZqqLriuc/poP/V0PfdNeT50e/f7FinJTYD0ZCZAZeQmQHqZTCaGDRtWvfMSN1+uiUKhEPl8Pjo7OyOXy9W6HAAAAAAAAHpYNedBrvSrsWKxWOsSAHq9YrEYCxculJkAKclNgPRkJkBl5CZAesViMRYtWlS18wz9AKgLK1eurHUJAHVFbgKkJzMBKiM3AdJbtWpV1c4y9AMAAAAAAIA6Z+gHAAAAAAAAdc7Qr8YymUytSwDo9TKZTIwcOVJmAqQkNwHSk5kAlZGbAOllMpkYMWJE1c7rU7WT2Kxs1twVoDvZbDaGDh1a6zIA6obcBEhPZgJURm4CpJfNZqOpqal651XtJDarWCzWugSAXq9YLMa8efNkJkBKchMgPZkJUBm5CZBesViMl19+uWrnGfoBUBfWrl1b6xIA6orcBEhPZgJURm4CpFfNzDT0AwAAAAAAgDpn6AcAAAAAAAB1ztCvxrJZLwFAd7LZbIwePVpmAqQkNwHSk5kAlZGbAOlls9kYNWpU1c7rU7WT2KxMJlPrEgB6vUwmE7lcrtZlANQNuQmQnswEqIzcBEiv2pnp1zFqrFgs1roEgF6vWCzGnDlzZCZASnITID2ZCVAZuQmQXrFYjNbW1qqdZ+gHQF3wwwRAZeQmQHoyE6AychMgvWpmpqEfAAAAAAAA1DlDPwAAAAAAAKhzmSRJkloXsT0qFAqRz+ejo6Mj8vl8rcsB6NWSJIm1a9dG//79I5PJ1LocgF5PbgKkJzMBKiM3AdJLkiSWL18ezc3N0dnZGblcbque50o/AOpCY2NjrUsAqCtyEyA9mQlQGbkJkF41M9PQr8ZKpVKtSwDo9UqlUsyZM0dmAqQkNwHSk5kAlZGbAOmVSqVobW2t2nl1M/T753/+5zj88MNjhx12iMGDB292z+LFi2Py5Mmxww47xC677BIXXnhhvPPOO132PPLII3HwwQdHv379Yq+99opp06Zt8jw//elPY9SoUdG/f/8YN25cPP30010eX7t2bZx99tkxdOjQGDhwYBx77LGxbNmynmoVAAAAAAAAKlI3Q7/169fHcccdF2edddZmHy8WizF58uRYv359PPnkkzF9+vSYNm1aXHrppeU9r776akyePDmOPPLIeP7552Pq1KnxjW98Ix544IHynjvuuCPOP//8uOyyy+L3v/99fPSjH41JkybF8uXLy3vOO++8+N3vfhd33nlnPProo/Hmm2/Gl7/85a3XPAAAAAAAALyPTJIkSa2LqMS0adNi6tSp0dHR0WX9/vvvj7/5m7+JN998M4YNGxYREbfcckt85zvfibfeeisaGxvjO9/5Ttx7773x4osvlr/u+OOPj46OjpgxY0ZERIwbNy4+9rGPxY033hgRf7n0cuTIkXHuuefGxRdfHJ2dnbHzzjvHr371q/jbv/3biIiYN29e7LvvvjFr1qw47LDDUvVRKBQin89He3t7DBky5K/9YwHYphWLxZgzZ06MHTs2Ghoaal0OQK8nNwHSk5kAlZGbAOkVi8WYNWtWfOpTn4rOzs7I5XJb9bw+W/XZq2jWrFkxduzY8sAvImLSpElx1llnRWtraxx00EExa9asmDBhQpevmzRpUkydOjUi/nI14ezZs+OSSy4pP57NZmPChAkxa9asiIiYPXt2bNiwocvzjBkzJnbffff3HfqtW7cu1q1bV/68UChERESSJFEsFiMiIpPJRDabjVKpFO+exW5c37ivu/VsNhuZTGaz6xGbvo/gltYbGhoiSZLNrr+3xi2t60lPetJTT/SUJEnst99+5ce3hZ66W9eTnvSkp7+mp425ufF7zW2hp23xddKTnvTUO3pKkiTGjh0bEZt+n1mvPW2pdj3pSU966omeuvsZvR576q52PelJT3r6oD1ls9kYM2ZMVMs2M/Rra2vrMvCLiPLnbW1t77unUCjEn//851ixYkUUi8XN7pk3b175ORobGzd5X8Fhw4aVz9mcq666Kq644opN1ufOnRsDBw6MiIimpqbYfffd4/XXX4/29vbynubm5mhubo7XXnstVq5cWV4fOXJkDB06NF555ZVYu3ZteX306NGRy+Vi7ty5Xf4j22effaKxsTHmzJnTpYaxY8fG+vXrY/78+eW1hoaGGDt2bKxcuTIWLVpUXu/fv3+MGTMmVqxYEUuWLCmvDxo0KPbcc89Yvnx5lz8HPelJT3rqqZ6KxWJ86EMfip122mmb6Sli23ud9KQnPfWOnl588cUoFovl37zeFnraFl8nPelJT72jp379+sWoUaNi9erV8frrr28TPW2Lr5Oe9KSn3tVTsViMXXfdNYYPH77N9BSx7b1OetKTnmrf06BBg+Kll16Kaqnp7T0vvvjiuOaaa953z0svvdRlCrql23ueccYZ8cc//rHL+/OtWbMmdtxxx7jvvvvi6KOPjg9/+MNx2mmndbmS77777ovJkyfHmjVrYsWKFbHbbrvFk08+GS0tLeU9F110UTz66KPx1FNPxa9+9as47bTTuly1FxHx8Y9/PI488sgt9rO5K/1GjhwZb731Vvn2nibietKTnvS0+dqLxWK0trbGAQccEH379t0meupuXU960pOe/pqeNmzYEK2trbH//vtHQ0PDNtHTtvg66UlPeuodPRWLxZg7d27sv//+5a+r9562VLue9KQnPfVET939jF6PPXVXu570pCc9fdCeSqVSPPnkkzF+/Pht//aeF1xwQZx66qnvu2f06NGpnqu5uTmefvrpLmvLli0rP7bxnxvX3r0nl8vFgAEDoqGhIRoaGja7593PsX79+ujo6Ohytd+792xOv379ol+/fpusbzzz3d77Q8a791Z7PZPJbHZ9SzVWuq4nPW1pXU96eu+ZG//yrLTGSte9TnrqqRorXdeTnnqqxo3rG2t995567yntup701FM1Vrqup/rvqSd67W09bY6e9NRT63rafnuq1s/oW1r3Oumpp2qsdF1PevogNWYymc2ubw2br7hKdt555xgzZsz7fjQ2NqZ6rpaWlpgzZ04sX768vDZz5szI5XLle0y3tLTEgw8+2OXrZs6cWb6qr7GxMQ455JAue0qlUjz44IPlPYccckj07du3y5758+fH4sWLu1wdCAAAAAAAANVSN+/pt3jx4mhvb4/FixdHsViM559/PiIi9tprrxg4cGBMnDgx9ttvvzj55JPj2muvjba2tvje974XZ599dvkKuzPPPDNuvPHGuOiii+Lv/u7v4qGHHopf//rXce+995bPOf/88+OUU06JQw89ND7+8Y/Hj3/841i9enWcdtppERGRz+djypQpcf7550dTU1Pkcrk499xzo6WlJQ477LCq/7kAbC+29JsyAGye3ARIT2YCVEZuAqRXzcys6Xv6VeLUU0+N6dOnb7L+8MMPxxFHHBEREX/84x/jrLPOikceeSR23HHHOOWUU+Lqq6+OPn3+b7b5yCOPxHnnnRdz586NESNGxD/+4z9ucovRG2+8Ma677rpoa2uLAw88MG644YYYN25c+fG1a9fGBRdcEP/+7/8e69ati0mTJsVNN930vrf3fK9CoRD5fL4q93AFAAAAAACg+qo5D6qbod+2ZuOL3NHREfl8vtblAPRqSZLEypUrY9CgQVW9BzZAvZKbAOnJTIDKyE2A9JIkiTfeeCNGjhxZlaFfTd/Tj7+8ZyAA769UKsWiRYtkJkBKchMgPZkJUBm5CZBeqVSK1157rWrnGfoBAAAAAABAnTP0AwAAAAAAgDpn6AdAXejfv3+tSwCoK3ITID2ZCVAZuQmQXjUzM5MkSVK10ygrFAqRz+er8saNAAAAAAAAVF8150Gu9Ksxb3gL0L1SqRRvv/22zARISW4CpCczASojNwHSK5VK0d7eXrXzDP1qzIWWAN1LkiSWLFkiMwFSkpsA6clMgMrITYD0kiSJ119/vWrnGfoBAAAAAABAnTP0AwAAAAAAgDpn6AdAXRg0aFCtSwCoK3ITID2ZCVAZuQmQ3sCBA6t2ViZx8+WaKBQKkc/no7OzM3K5XK3LAQAAAAAAoIdVcx7kSr8aK5VKtS4BoNcrlUrR1tYmMwFSkpsA6clMgMrITYD0SqVSLFu2rGrnGfrVmAstAbqXJEm0tbXJTICU5CZAejIToDJyEyC9JEkM/QAAAAAAAID0DP0AAAAAAACgzhn61Vgmk6l1CQC9XiaTiaamJpkJkJLcBEhPZgJURm4CpJfJZGLIkCHVOy9x8+WaKBQKkc/no7OzM3K5XK3LAQAAAAAAoIdVcx7kSr8aK5VKtS4BoNcrlUqxePFimQmQktwESE9mAlRGbgKkVyqVYsmSJVU7z9CvxlxoCdC9JEmivb1dZgKkJDcB0pOZAJWRmwDpJUkSK1asqNp5hn4AAAAAAABQ5wz9AAAAAAAAoM4Z+tVYJpOpdQkAvV4mk4nm5maZCZCS3ARIT2YCVEZuAqSXyWRi2LBh1TsvcfPlmigUCpHP56OzszNyuVytywEAAAAAAKCHVXMe5Eq/GisWi7UuAaDXKxaLsXDhQpkJkJLcBEhPZgJURm4CpFcsFmPRokVVO8/QD4C6sHLlylqXAFBX5CZAejIToDJyEyC9VatWVe0sQz8AAAAAAACoc4Z+AAAAAAAAUOcM/Wosk8nUugSAXi+TycTIkSNlJkBKchMgPZkJUBm5CZBeJpOJESNGVO28PlU7ic3KZs1dAbqTzWZj6NChtS4DoG7ITYD0ZCZAZeQmQHrZbDaampqqd17VTmKzisVirUsA6PWKxWLMmzdPZgKkJDcB0pOZAJWRmwDpFYvFePnll6t2nqEfAHVh7dq1tS4BoK7ITYD0ZCZAZeQmQHrVzExDPwAAAAAAAKhzhn4AAAAAAABQ5wz9aiyb9RIAdCebzcbo0aNlJkBKchMgPZkJUBm5CZBeNpuNUaNGVe28PlU7ic3KZDK1LgGg18tkMpHL5WpdBkDdkJsA6clMgMrITYD0qp2Zfh2jxorFYq1LAOj1isVizJkzR2YCpCQ3AdKTmQCVkZsA6RWLxWhtba3aeYZ+ANQFP0wAVEZuAqQnMwEqIzcB0qtmZhr6AQAAAAAAQJ0z9AMAAAAAAIA6l0mSJKl1EdujQqEQ+Xw+Ojo6Ip/P17ocgF4tSZJYu3Zt9O/fPzKZTK3LAej15CZAejIToDJyEyC9JEli+fLl0dzcHJ2dnZHL5bbqea70A6AuNDY21roEgLoiNwHSk5kAlZGbAOlVMzMN/WqsVCrVugSAXq9UKsWcOXNkJkBKchMgPZkJUBm5CZBeqVSK1tbWqp1n6AcAAAAAAAB1ztAPAAAAAAAA6pyhHwAAAAAAANS5TJIkSa2L2B4VCoXI5/PR0dER+Xy+1uUA9GpJkkSpVIpsNhuZTKbW5QD0enITID2ZCVAZuQmQXpIk0dHREU1NTdHZ2Rm5XG6rnudKPwDqwvr162tdAkBdkZsA6clMgMrITYD0qpmZhn41ViqVal0CQK9XKpVi/vz5MhMgJbkJkJ7MBKiM3ARIr1QqxSuvvFK18wz9AAAAAAAAoM4Z+gEAAAAAAECdM/QDoC40NDTUugSAuiI3AdKTmQCVkZsA6VUzMzNJkiRVO42yQqEQ+Xw+Ojs7I5fL1bocAAAAAAAAelg150Gu9KsxM1eA7iVJEoVCQWYCpCQ3AdKTmQCVkZsA6W3MzGox9KuxUqlU6xIAer1SqRSLFi2SmQApyU2A9GQmQGXkJkB6pVIpXnvttaqdZ+gHAAAAAAAAdc7QDwAAAAAAAOqcoR8AdaF///61LgGgrshNgPRkJkBl5CZAetXMzEziHVdrolAoRD6fj87OzsjlcrUuBwAAAAAAgB5WzXmQK/1qzBveAnSvVCrF22+/LTMBUpKbAOnJTIDKyE2A9EqlUrS3t1ftPEO/GnOhJUD3kiSJJUuWyEyAlOQmQHoyE6AychMgvSRJ4vXXX6/aeYZ+AAAAAAAAUOcM/QAAAAAAAKDOGfoBUBcGDRpU6xIA6orcBEhPZgJURm4CpDdw4MCqnZVJ3Hy5JgqFQuTz+ejs7IxcLlfrcgAAAAAAAOhh1ZwHudKvxkqlUq1LAOj1SqVStLW1yUyAlOQmQHoyE6AychMgvVKpFMuWLavaeYZ+NeZCS4DuJUkSbW1tMhMgJbkJkJ7MBKiM3ARIL0kSQz8AAAAAAAAgPUM/AAAAAAAAqHOGfjWWyWRqXQJAr5fJZKKpqUlmAqQkNwHSk5kAlZGbAOllMpkYMmRI9c5L3Hy5JgqFQuTz+ejs7IxcLlfrcgAAAAAAAOhh1ZwHudKvxkqlUq1LAOj1SqVSLF68WGYCpCQ3AdKTmQCVkZsA6ZVKpViyZEnVzjP0qzEXWgJ0L0mSaG9vl5kAKclNgPRkJkBl5CZAekmSxIoVK6p2nqEfAAAAAAAA1DlDPwAAAAAAAKhzhn41lslkal0CQK+XyWSiublZZgKkJDcB0pOZAJWRmwDpZTKZGDZsWPXOS9x8uSYKhULk8/no7OyMXC5X63IAAAAAAADoYdWcB7nSr8aKxWKtSwDo9YrFYixcuFBmAqQkNwHSk5kAlZGbAOkVi8VYtGhR1c4z9AOgLqxcubLWJQDUFbkJkJ7MBKiM3ARIb9WqVVU7y9APAAAAAAAA6pyhHwAAAAAAANQ5Q78ay2QytS4BoNfLZDIxcuRImQmQktwESE9mAlRGbgKkl8lkYsSIEVU7r0/VTmKzsllzV4DuZLPZGDp0aK3LAKgbchMgPZkJUBm5CZBeNpuNpqam6p1XtZPYrGKxWOsSAHq9YrEY8+bNk5kAKclNgPRkJkBl5CZAesViMV5++eWqnWfoB0BdWLt2ba1LAKgrchMgPZkJUBm5CZBeNTPT0A8AAAAAAADqnKEfAAAAAAAA1DlDvxrLZr0EAN3JZrMxevRomQmQktwESE9mAlRGbgKkl81mY9SoUVU7r0/VTmKzMplMrUsA6PUymUzkcrlalwFQN+QmQHoyE6AychMgvWpnpl/HqLFisVjrEgB6vWKxGHPmzJGZACnJTYD0ZCZAZeQmQHrFYjFaW1urdp6hHwB1wQ8TAJWRmwDpyUyAyshNgPSqmZmGfgAAAAAAAFDnDP0AAAAAAACgzmWSJElqXcT2qFAoRD6fj46Ojsjn87UuB6BXS5Ik1q5dG/37949MJlPrcgB6PbkJkJ7MBKiM3ARIL0mSWL58eTQ3N0dnZ2fkcrmtep4r/QCoC42NjbUuAaCuyE2A9GQmQGXkJkB61cxMQ78aK5VKtS4BoNcrlUoxZ84cmQmQktwESE9mAlRGbgKkVyqVorW1tWrnGfoBAAAAAABAnTP0AwAAAAAAgDpn6AcAAAAAAAB1LpMkSVLrIrZHhUIh8vl8dHR0RD6fr3U5AL1akiRRKpUim81GJpOpdTkAvZ7cBEhPZgJURm4CpJckSXR0dERTU1N0dnZGLpfbque50g+AurB+/fpalwBQV+QmQHoyE6AychMgvWpmpqFfjZVKpVqXANDrlUqlmD9/vswESEluAqQnMwEqIzcB0iuVSvHKK69U7TxDPwAAAAAAAKhzhn4AAAAAAABQ5+pi6Pfaa6/FlClTYo899ogBAwbEnnvuGZdddtkm90F94YUX4lOf+lT0798/Ro4cGddee+0mz3XnnXfGmDFjon///jF27Ni47777ujyeJElceumlMXz48BgwYEBMmDBhk0sv29vb46STTopcLheDBw+OKVOmxKpVq3q+cQDKGhoaal0CQF2RmwDpyUyAyshNgPSqmZl1MfSbN29elEqluPXWW6O1tTV+9KMfxS233BLf/e53y3sKhUJMnDgxPvShD8Xs2bPjuuuui8svvzxuu+228p4nn3wyTjjhhJgyZUo899xzccwxx8QxxxwTL774YnnPtddeGzfccEPccsst8dRTT8WOO+4YkyZNirVr15b3nHTSSdHa2hozZ86Me+65Jx577LE444wzPlBv/oIE6F5DQ0OMHTtWZgKkJDcB0pOZAJWRmwDpNTQ0xP7771+18zJJkiRVO60HXXfddXHzzTfHokWLIiLi5ptvjv/3//5ftLW1RWNjY0REXHzxxXH33XfHvHnzIiLiq1/9aqxevTruueee8vMcdthhceCBB8Ytt9wSSZLErrvuGhdccEF8+9vfjoiIzs7OGDZsWEybNi2OP/74eOmll2K//faLZ555Jg499NCIiJgxY0Z8/vOfj9dffz123XXXzda7bt26WLduXfnzQqEQI0eOjLfffjvy+XxERGQymchms1EqleLdL8vG9WKx2OU5t7SezWYjk8lsdj0iNnmT3S2tNzQ0RJIkm11/b41bWteTnvSkp57oKUmSWLlyZeRyuWhoaNgmeupuXU960pOe/pqeisVirFy5MgYNGlR+jnrvaVt8nfSkJz31jp6SJIk1a9bEjjvuGO9Vrz1tqXY96UlPeuqJnrr7Gb0ee+qudj3pSU96+qA9RUQsXrw4Ro0aFZ2dnZHL5WJr6rNVn30r6uzsjKampvLns2bNivHjx5cHfhERkyZNimuuuSZWrFgRQ4YMiVmzZsX555/f5XkmTZoUd999d0REvPrqq9HW1hYTJkwoP57P52PcuHExa9asOP7442PWrFkxePDg8sAvImLChAmRzWbjqaeeii996Uubrfeqq66KK664YpP11tbWGDRoUERENDU1xe677x6vv/56tLe3l/c0NzdHc3NzvPbaa7Fy5cry+siRI2Po0KHxyiuvdLkScfTo0ZHL5WLu3Lld/iPbZ599orGxMebMmdOlhrFjx8b69etj/vz55bWNv7GzcuXK8mA1IqJ///4xZsyYWLFiRSxZsqS8PmjQoNhzzz1j+fLl0dbWVl7Xk570pKee6ClJkmhvb4+PfOQjsfPOO28TPW2Lr5Oe9KSn3tVTe3t7NDU1RSaT2WZ62hZfJz3pSU+176mxsTHWr18fu+22W7zxxhvbRE/b4uukJz3pqff0tPFn9H333Td23XXXbaKnbfF10pOe9NQ7etpxxx3jhRdeiGqpyyv9FixYEIccckhcf/31cfrpp0dExMSJE2OPPfaIW2+9tbxv7ty5sf/++8fcuXNj3333jcbGxpg+fXqccMIJ5T033XRTXHHFFbFs2bJ48skn4xOf+ES8+eabMXz48PKer3zlK5HJZOKOO+6IK6+8MqZPn97lP6SIiF122SWuuOKKOOusszZb85au9HvrrbdiyJAhEWEiric96UlPW6q9WCxGa2trHHDAAdG3b99toqfu1vWkJz3p6a/pacOGDdHa2hr7779/NDQ0bBM9bYuvk570pKfe0VOxWCz//4ONX1fvPW2pdj3pSU966omeuvsZvR576q52PelJT3r6oD2VSqV48sknY/z48dv+lX4XX3xxXHPNNe+756WXXooxY8aUP3/jjTfiqKOOiuOOO6488KsH/fr1i379+m2y3tDQEA0NXe9//d4fMt69t9rrmUxms+tbqrHSdT3paUvretLTe8/c+JdnpTVWuu510lNP1Vjpup701FM1blzfWOu799R7T2nX9aSnnqqx0nU91X9PPdFrb+tpc/Skp55a19P221O1fkbf0rrXSU89VWOl63rS0wepMZPJbHZ9a6jp0O+CCy6IU0899X33jB49uvzvb775Zhx55JFx+OGHx2233dZlX3NzcyxbtqzL2sbPm5ub33fPux/fuPbuK/2WLVsWBx54YHnP8uXLuzzHO++8E+3t7eWvB6Dn9e/fv9YlANQVuQmQnswEqIzcBEivmpm5+TFlley8884xZsyY9/3Y+B59b7zxRhxxxBFxyCGHxO23377JhLWlpSUee+yx2LBhQ3lt5syZsc8++5Rvn9nS0hIPPvhgl6+bOXNmtLS0RETEHnvsEc3NzV32FAqFeOqpp8p7WlpaoqOjI2bPnl3e89BDD0WpVIpx48ZV/GewpckvAP+noaEhxowZIzMBUpKbAOnJTIDKyE2A9BoaGuLDH/5w1c6r6dAvrY0Dv9133z2uv/76eOutt6Ktra3LGyueeOKJ0djYGFOmTInW1ta444474ic/+Umcf/755T3f+ta3YsaMGfHDH/4w5s2bF5dffnk8++yzcc4550TEXy6xnDp1avzTP/1T/Pa3v405c+bE17/+9dh1113jmGOOiYiIfffdN4466qg4/fTT4+mnn44nnngizjnnnDj++ONj1113rbi3995XFoBNlUqlePvtt2UmQEpyEyA9mQlQGbkJkF6pVIr29vaqnVfT23umNXPmzFiwYEEsWLAgRowY0eWxjW+imM/n47//+7/j7LPPjkMOOSR22mmnuPTSS+OMM84o7z388MPjV7/6VXzve9+L7373u7H33nvH3XffHQcccEB5z0UXXRSrV6+OM844Izo6OuKTn/xkzJgxo8vll7/85S/jnHPOic9+9rORzWbj2GOPjRtuuOED9fbeN40EYFNJksSSJUti8ODBtS4FoC7ITYD0ZCZAZeQmQHpJksTrr79etfMyialTTRQKhcjn89He3l6+/SgAm1csFmPOnDkxduxYtw8BSEFuAqQnMwEqIzcB0isWizFr1qz41Kc+FZ2dnZHL5bbqeXVxe08AAAAAAABgywz9AKgLgwYNqnUJAHVFbgKkJzMBKiM3AdIbOHBg1c5ye88a2Xh7z2pczgkAAAAAAED1VXMe5Eq/GiuVSrUuAaDXK5VK0dbWJjMBUpKbAOnJTIDKyE2A9EqlUixbtqxq5xn61ZgLLQG6lyRJtLW1yUyAlOQmQHoyE6AychMgvSRJDP0AAAAAAACA9Az9AAAAAAAAoM4Z+tVYJpOpdQkAvV4mk4mmpiaZCZCS3ARIT2YCVEZuAqSXyWRiyJAh1TsvcfPlmigUCpHP56OzszNyuVytywEAAAAAAKCHVXMe5Eq/GiuVSrUuAaDXK5VKsXjxYpkJkJLcBEhPZgJURm4CpFcqlWLJkiVVO8/Qr8ZcaAnQvSRJor29XWYCpCQ3AdKTmQCVkZsA6SVJEitWrKjaeYZ+AAAAAAAAUOf61LqA7dXG34QpFArR0NBQ42oAerdisRirVq2SmQApyU2A9GQmQGXkJkB6xWIxVq9eXbXzDP1q5O23346IiFGjRtW2EAAAAAAAAOqeoV+NNDU1RUTE4sWLI5/P17gagN6tUCjEyJEjY8mSJZHL5WpdDkCvJzcB0pOZAJWRmwDpbczMxYsXx6BBg7b6eYZ+NZLN/uXtFPP5vL8cAVLK5XIyE6ACchMgPZkJUBm5CZBePp+PTCaz1c/JbvUTAAAAAAAAgK3K0A8AAAAAAADqnKFfjfTr1y8uu+yy6NevX61LAej1ZCZAZeQmQHoyE6AychMgvWpnZiZJkqQqJwEAAAAAAABbhSv9AAAAAAAAoM4Z+gEAAAAAAECdM/QDAAAAAACAOmfoBwAAAAAAAHXO0O8Duuqqq+JjH/tYDBo0KHbZZZc45phjYv78+V32rF27Ns4+++wYOnRoDBw4MI499thYtmxZlz2LFy+OyZMnxw477BC77LJLXHjhhfHOO+902fPII4/EwQcfHP369Yu99torpk2btrXbA+hxaXLztttuiyOOOCJyuVxkMpno6OjY5Hna29vjpJNOilwuF4MHD44pU6bEqlWruux54YUX4lOf+lT0798/Ro4cGddee+3WbA2gx3WXme3t7XHuuefGPvvsEwMGDIjdd989/uEf/iE6Ozu7PI/vNYHtRZrvNf/+7/8+9txzzxgwYEDsvPPO8cUvfjHmzZvXZY/cBLYHaTJzoyRJ4uijj45MJhN33313l8dkJrC9SJObRxxxRGQymS4fZ555Zpc91chNQ78P6NFHH42zzz47/vd//zdmzpwZGzZsiIkTJ8bq1avLe84777z43e9+F3feeWc8+uij8eabb8aXv/zl8uPFYjEmT54c69evjyeffDKmT58e06ZNi0svvbS859VXX43JkyfHkUceGc8//3xMnTo1vvGNb8QDDzxQ1X4B/lppcnPNmjVx1FFHxXe/+90tPs9JJ50Ura2tMXPmzLjnnnviscceizPOOKP8eKFQiIkTJ8aHPvShmD17dlx33XVx+eWXx2233bZV+wPoSd1l5ptvvhlvvvlmXH/99fHiiy/GtGnTYsaMGTFlypTyc/heE9iepPle85BDDonbb789XnrppXjggQciSZKYOHFiFIvFiJCbwPYjTWZu9OMf/zgymcwm6zIT2J6kzc3TTz89li5dWv5494UIVcvNhB6xfPnyJCKSRx99NEmSJOno6Ej69u2b3HnnneU9L730UhIRyaxZs5IkSZL77rsvyWazSVtbW3nPzTffnORyuWTdunVJkiTJRRddlOy///5dzvrqV7+aTJo0aWu3BLBVvTc33+3hhx9OIiJZsWJFl/W5c+cmEZE888wz5bX7778/yWQyyRtvvJEkSZLcdNNNyZAhQ8o5miRJ8p3vfCfZZ599tk4jAFXwfpm50a9//euksbEx2bBhQ5IkvtcEtm9pcvMPf/hDEhHJggULkiSRm8D2a0uZ+dxzzyW77bZbsnTp0iQikrvuuqv8mMwEtmeby81Pf/rTybe+9a0tfk21ctOVfj1k462UmpqaIiJi9uzZsWHDhpgwYUJ5z5gxY2L33XePWbNmRUTErFmzYuzYsTFs2LDynkmTJkWhUIjW1tbynnc/x8Y9G58DoF69NzfTmDVrVgwePDgOPfTQ8tqECRMim83GU089Vd4zfvz4aGxsLO+ZNGlSzJ8/P1asWNFD1QNUV5rM7OzsjFwuF3369IkI32sC27fucnP16tVx++23xx577BEjR46MCLkJbL82l5lr1qyJE088MX76059Gc3PzJl8jM4Ht2Za+1/zlL38ZO+20UxxwwAFxySWXxJo1a8qPVSs3Df16QKlUiqlTp8YnPvGJOOCAAyIioq2tLRobG2Pw4MFd9g4bNiza2trKe979Am98fONj77enUCjEn//8563RDsBWt7ncTKOtrS122WWXLmt9+vSJpqamirIVoJ6kycw//elP8YMf/KDL7Y59rwlsr94vN2+66aYYOHBgDBw4MO6///6YOXNm+ZfF5CawPdpSZp533nlx+OGHxxe/+MXNfp3MBLZXW8rNE088Mf7t3/4tHn744bjkkkviF7/4RXzta18rP16t3Ozzgbqii7PPPjtefPHFePzxx2tdCkBdkJsA6XWXmYVCISZPnhz77bdfXH755dUtDqAXer/cPOmkk+Jzn/tcLF26NK6//vr4yle+Ek888UT079+/BpUC1N7mMvO3v/1tPPTQQ/Hcc8/VsDKA3mlL32u++5dwx44dG8OHD4/PfvazsXDhwthzzz2rVp8r/f5K55xzTtxzzz3x8MMPx4gRI8rrzc3NsX79+ujo6Oiyf9myZeVL4pubm2PZsmWbPL7xsffbk8vlYsCAAT3dDsBWt6XcTKO5uTmWL1/eZe2dd96J9vb2irIVoF50l5krV66Mo446KgYNGhR33XVX9O3bt/yY7zWB7VF3uZnP52PvvfeO8ePHx29+85uYN29e3HXXXREhN4Htz5Yy86GHHoqFCxfG4MGDo0+fPuXbxx977LFxxBFHRITMBLZPlfx/zXHjxkVExIIFCyKierlp6PcBJUkS55xzTtx1113x0EMPxR577NHl8UMOOST69u0bDz74YHlt/vz5sXjx4mhpaYmIiJaWlpgzZ06X/4E9c+bMyOVysd9++5X3vPs5Nu7Z+BwA9aK73EyjpaUlOjo6Yvbs2eW1hx56KEqlUvkv0paWlnjsscdiw4YN5T0zZ86MffbZJ4YMGfLXNwJQBWkys1AoxMSJE6OxsTF++9vfbnKViu81ge3JB/leM0mSSJIk1q1bFxFyE9h+dJeZF198cbzwwgvx/PPPlz8iIn70ox/F7bffHhEyE9i+fJDvNTdm5/DhwyOiirmZ8IGcddZZST6fTx555JFk6dKl5Y81a9aU95x55pnJ7rvvnjz00EPJs88+m7S0tCQtLS3lx995553kgAMOSCZOnJg8//zzyYwZM5Kdd945ueSSS8p7Fi1alOywww7JhRdemLz00kvJT3/606ShoSGZMWNGVfsF+Gulyc2lS5cmzz33XPKzn/0siYjkscceS5577rnk7bffLu856qijkoMOOih56qmnkscffzzZe++9kxNOOKH8eEdHRzJs2LDk5JNPTl588cXkP/7jP5IddtghufXWW6vaL8Bfo7vM7OzsTMaNG5eMHTs2WbBgQZc977zzTpIkvtcEti/d5ebChQuTK6+8Mnn22WeTP/7xj8kTTzyRfOELX0iampqSZcuWJUkiN4HtR5qfz98rIpK77rqr/LnMBLYn3eXmggULku9///vJs88+m7z66qvJf/3XfyWjR49Oxo8fX36OauWmod8HFBGb/bj99tvLe/785z8n3/zmN5MhQ4YkO+ywQ/KlL30pWbp0aZfnee2115Kjjz46GTBgQLLTTjslF1xwQbJhw4Yuex5++OHkwAMPTBobG5PRo0d3OQOgXqTJzcsuu6zbPW+//XZywgknJAMHDkxyuVxy2mmnJStXruxy1h/+8Ifkk5/8ZNKvX79kt912S66++uoqdQnQM7rLzIcffniLe1599dXy8/heE9hedJebb7zxRnL00Ucnu+yyS9K3b99kxIgRyYknnpjMmzevy/PITWB7kObn8819zbuHfkkiM4HtR3e5uXjx4mT8+PFJU1NT0q9fv2SvvfZKLrzwwqSzs7PL81QjNzP/f8EAAAAAAABAnfKefgAAAAAAAFDnDP0AAAAAAACgzhn6AQAAAAAAQJ0z9AMAAAAAAIA6Z+gHAAAAAAAAdc7QDwAAAAAAAOqcoR8AAAAAAADUOUM/AAAAAAAAqHOGfgAAANuhU089NY455phal/GB1Xv9AAAAPa1PrQsAAACgZ2Uymfd9/LLLLouf/OQnkSRJlSr6P4888kgceeSRsWLFihg8eHDVzwcAANhWGfoBAABsY5YuXVr+9zvuuCMuvfTSmD9/fnlt4MCBMXDgwFqUBgAAwFbi9p4AAADbmObm5vJHPp+PTCbTZW3gwIGb3B7ziCOOiHPPPTemTp0aQ4YMiWHDhsXPfvazWL16dZx22mkxaNCg2GuvveL+++/vctaLL74YRx99dAwcODCGDRsWJ598cvzpT39KXeu0adNi8ODB8cADD8S+++4bAwcOjKOOOqrL4LJYLMb5558fgwcPjqFDh8ZFF120yVWKpVIprrrqqthjjz1iwIAB8dGPfjR+85vfREREkiQxYcKEmDRpUvnr2tvbY8SIEXHppZdW+scLAADQKxn6AQAAEBER06dPj5122imefvrpOPfcc+Oss86K4447Lg4//PD4/e9/HxMnToyTTz451qxZExERHR0d8ZnPfCYOOuigePbZZ2PGjBmxbNmy+MpXvlLRuWvWrInrr78+fvGLX8Rjjz0Wixcvjm9/+9vlx3/4wx/GtGnT4uc//3k8/vjj0d7eHnfddVeX57jqqqviX//1X+OWW26J1tbWOO+88+JrX/taPProo5HJZGL69OnxzDPPxA033BAREWeeeWbstttuhn4AAMA2w+09AQAAiIiIj370o/G9730vIiIuueSSuPrqq2OnnXaK008/PSIiLr300rj55pvjhRdeiMMOOyxuvPHGOOigg+LKK68sP8fPf/7zGDlyZLz88svx4Q9/ONW5GzZsiFtuuSX23HPPiIg455xz4vvf/3758R//+MdxySWXxJe//OWIiLjlllvigQceKD++bt26uPLKK+N//ud/oqWlJSIiRo8eHY8//njceuut8elPfzp22223uPXWW+PrX/96tLW1xX333RfPPfdc9Onjx2IAAGDb4KcbAAAAIiLiIx/5SPnfGxoaYujQoTF27Njy2rBhwyIiYvny5RER8Yc//CEefvjhzb4/4MKFC1MP/XbYYYfywC8iYvjw4eUzOjs7Y+nSpTFu3Ljy43369IlDDz20fKvOBQsWxJo1a+Jzn/tcl+ddv359HHTQQeXPjzvuuLjrrrvi6quvjptvvjn23nvvVPUBAADUA0M/AAAAIiKib9++XT7PZDJd1jKZTET85f3zIiJWrVoVX/jCF+Kaa67Z5LmGDx/+V5373vfsez+rVq2KiIh77703dtttty6P9evXr/zva9asidmzZ0dDQ0O88sorqZ8fAACgHhj6AQAA8IEcfPDB8Z//+Z8xatSorXabzHw+H8OHD4+nnnoqxo8fHxER77zzTsyePTsOPvjgiIjYb7/9ol+/frF48eL49Kc/vcXnuuCCCyKbzcb9998fn//852Py5Mnxmc98ZqvUDQAAUG3ZWhcAAABAfTr77LOjvb09TjjhhHjmmWdi4cKF8cADD8Rpp50WxWKxx8751re+FVdffXXcfffdMW/evPjmN78ZHR0d5ccHDRoU3/72t+O8886L6dOnx8KFC+P3v/99/Mu//EtMnz49Iv5yFeDPf/7z+OUvfxmf+9zn4sILL4xTTjklVqxY0WN1AgAA1JKhHwAAAB/IrrvuGk888UQUi8WYOHFijB07NqZOnRqDBw+ObLbnfty84IIL4uSTT45TTjklWlpaYtCgQfGlL32py54f/OAH8Y//+I9x1VVXxb777htHHXVU3HvvvbHHHnvEW2+9FVOmTInLL7+8fHXgFVdcEcOGDYszzzyzx+oEAACopUxSyRslAAAAAAAAAL2OK/0AAAAAAACgzhn6AQAAAAAAQJ0z9AMAAAAAAIA6Z+gHAAAAAAAAdc7QDwAAAAAAAOqcoR8AAAAAAADUOUM/AAAAAAAAqHOGfgAAAAAAAFDnDP0AAAAAAACgzhn6AQAAAAAAQJ0z9AMAAAAAAIA69/8BR+JsJJhaNhIAAAAASUVORK5CYII=\n" + }, + "metadata": {} + }, + { + "output_type": "stream", + "name": "stdout", + "text": [ + "948\n" + ] + }, + { + "output_type": "display_data", + "data": { + "text/plain": [ + "
" + ], + "image/png": "iVBORw0KGgoAAAANSUhEUgAABv0AAAN5CAYAAAArSffsAAAAOnRFWHRTb2Z0d2FyZQBNYXRwbG90bGliIHZlcnNpb24zLjEwLjAsIGh0dHBzOi8vbWF0cGxvdGxpYi5vcmcvlHJYcgAAAAlwSFlzAAAPYQAAD2EBqD+naQABAABJREFUeJzs3XdcU9f7B/BPwgxbFBnKUhFQcS+sFjfWUfdEhTpbpW6rVuuou64O66h7a9VWbd0L97a4tyCoIC5ANiTn94c/8jUyDAoJVz/v14tXzbnnnvuc5D4JzcM9VyaEECAiIiIiIiIiIiIiIiIiyZLrOwAiIiIiIiIiIiIiIiIi+jAs+hERERERERERERERERFJHIt+RERERERERERERERERBLHoh8RERERERERERERERGRxLHoR0RERERERERERERERCRxLPoRERERERERERERERERSRyLfkREREREREREREREREQSx6IfERERERERERERERERkcSx6EdEREREREREREREREQkcSz6EREREdFHISgoCG5ubvoOQ28+9fkXpJUrV0ImkyE8PFzfoRR64eHhkMlkWLlypd5iaN68Ofr27au34xc2ISEhkMlkCAkJ0Xco9BGoXbs2vvvuO32HQUREREQ5YNGPiIiIiAotmUym1U9h/TI7PDwcX331FUqXLg1TU1M4ODjg888/x4QJE/QdWr6rWbMmZDIZFi5cqO9QCr2goCCN89fCwgKlSpVChw4dsHXrVqhUqvcee9euXZg4cWL+BZuD9evX4+effy7w4+TViRMnsG/fPowaNUrd5ubm9s73kKCgIP0FXUhkFrfPnz9f4MdKSUnB9OnTUa5cOZiZmaFEiRLo2LEjrl27ptHv6NGj+PLLL+Hs7Kx+D23WrBlOnDiR7bgnT55E3bp1YWZmBgcHBwwaNAgJCQlZ+qWmpmLUqFFwcnKCQqFArVq1sH//fq1iL8hzf/78+fD29oaJiQlKlCiBYcOGITExMUs/lUqFn376Ce7u7jA1NUXFihWxYcOGbMe8ceMGmjVrBgsLC9ja2qJHjx54+vTpe485atQo/P7774iOjv7wCRMRERFRvjPUdwBERERERDlZs2aNxuPVq1dj//79Wdq9vb2xZMmSDyqW5Le7d++iRo0aUCgU6NWrF9zc3BAVFYWLFy9i5syZmDRpUr4eT5/zv3PnDs6dOwc3NzesW7cO33zzjV7ikBITExMsXboUAJCcnIwHDx7gn3/+QYcOHVC/fn1s374dVlZWeR53165d+P333wu88Ld+/XpcvXoVQ4YM0Wh3dXVFcnIyjIyMCvT4OZk1axYaNWqEMmXKqNt+/vnnbAs/wOsiy5kzZ1C7dm1dhUgAAgICsGPHDvTt2xdVq1bF48eP8fvvv8PX1xdXrlyBq6srAOD27duQy+X4+uuv4eDggJcvX2Lt2rX4/PPPsXPnTjRr1kw9ZmhoKBo1agRvb2/MnTsXDx8+xOzZs3Hnzh3s3r1b4/hBQUHYsmULhgwZAg8PD6xcuRLNmzfH4cOHUbdu3Vxjz+nc/1CjRo3CTz/9hA4dOmDw4MG4fv06fvvtN1y7dg179+7V6Dt27FjMmDEDffv2RY0aNbB9+3Z069YNMpkMXbp0Ufd7+PAhPv/8c1hbW2PatGlISEjA7NmzceXKFZw9exbGxsZ5HrN169awsrLCggUL8OOPP+brc0BERERE+UAQEREREUnEwIEDhVR+hR0wYIAwNDQU4eHhWbY9efIk346TkJCQb2O9r/Hjx4vixYuLrVu3CplMJsLCwvQdUr5asWKFAJBv8woMDBTm5ubZbps+fboAIDp16vReY+sqR1q0aCFcXV0L/Dh58eTJE2FoaCiWLl2qVf+9e/cKmUwmvvzyywKOTL8OHz4sAIjDhw/n2i/zPD937lyBxvPw4UMBQIwYMUKj/dChQwKAmDt3bq77JyYmCnt7e+Hv76/R/sUXXwhHR0cRFxenbluyZIkAIPbu3atuO3PmjAAgZs2apW5LTk4WpUuXFr6+vu+MvyDO/cePHwtDQ0PRo0cPjfbffvtNABA7duxQtz18+FAYGRmJgQMHqttUKpWoV6+eKFmypMjIyFC3f/PNN0KhUIgHDx6o2/bv3y8AiMWLF7/XmEIIERwcLFxdXYVKpfrwyRMRERFRvuLynkRERET0UXj7nnaZ9xabPXs2fv/9d5QqVQpmZmZo2rQpIiMjIYTA5MmTUbJkSSgUCrRu3RovXrzIMu7u3btRr149mJubw9LSEi1atMiyBF127t27h5IlS6qvWHlT8eLF3+s4QUFBsLCwwL1799C8eXNYWloiICAg2/kDr5dr+/nnn1G+fHmYmprC3t4e/fv3x8uXLzX6nT9/Hv7+/ihWrBgUCgXc3d3Rq1evd84x0/r169GhQwe0bNkS1tbWWL9+fZY+EydOhEwmw927dxEUFAQbGxtYW1vjq6++QlJSkkbfjIwMTJ48GaVLl4aJiQnc3Nzw/fffIzU1VaOfm5sbWrZsiZCQEFSvXh0KhQI+Pj7q5V7/+usv+Pj4wNTUFNWqVcN///2nsf/ly5cRFBSEUqVKqZcO7NWrF54/f57rfAMDA1GsWDGkp6dn2da0aVN4enpq87Rla/To0WjatCk2b96M27dva2x71zkSFBSE33//HYDm0riZtD0fMo/l5+cHS0tLWFlZoUaNGurXtX79+ti5cycePHigPkbmuZfTPf0OHTqkjt3GxgatW7fGjRs3NPrk5RzJzs6dO5GRkYHGjRu/s290dDR69OiBEiVKYMWKFRrbtD3/AGDBggUoX748TExM4OTkhIEDByI2NlajT/369VGhQgVcvnwZfn5+MDMzQ5kyZbBlyxYAwJEjR1CrVi0oFAp4enriwIEDWY7z6NEj9OrVC/b29jAxMUH58uWxfPnyLP0ePnyINm3awNzcHMWLF8fQoUOzjVtbme85jx49Qps2bWBhYQE7OzuMGDECSqVSo29UVBRu3ryZbV686dWrVwAAe3t7jXZHR0cAgEKhyHV/MzMz2NnZaTzP8fHx2L9/P7p3765xhWzPnj1hYWGBP//8U922ZcsWGBgYoF+/fuo2U1NT9O7dG6dOnUJkZGSOx87t3AeAmJgY9O7dG/b29jA1NUWlSpWwatWqXOcDAKdOnUJGRobGFXUA1I83btyobtu+fTvS09MxYMAAdZtMJsM333yDhw8f4tSpU+r2rVu3omXLlnBxcVG3NW7cGGXLltV4TvIyJgA0adIEDx48QGho6DvnRkRERES6xaIfEREREX3U1q1bhwULFuDbb7/F8OHDceTIEXTq1Anjxo3Dnj17MGrUKPTr1w///PMPRowYobHvmjVr0KJFC1hYWGDmzJn44YcfcP36ddStWxfh4eG5HtfV1RWRkZE4dOjQO2PMy3EyMjLg7++P4sWLY/bs2Wjfvn2O4/bv3x8jR47EZ599hl9++QVfffUV1q1bB39/f/UX8zExMWjatCnCw8MxevRo/PbbbwgICMDp06ffGTcAnDlzBnfv3kXXrl1hbGyMdu3aYd26dTn279SpE169eoXp06ejU6dOWLlyZZalTvv06YPx48ejatWqmDdvHvz8/DB9+vQsX4gDr5dR7datG1q1aoXp06fj5cuXaNWqFdatW4ehQ4eie/fumDRpEu7du4dOnTppLIG6f/9+3L9/H1999RV+++03dOnSBRs3bkTz5s0hhMhxDj169MDz58+zLLkXHR2NQ4cOoXv37lo9d7mNL4TQuMeYNudI//790aRJE3X/zJ9M2pwPwOt7u7Vo0QIvXrzAmDFjMGPGDFSuXBl79uwB8HoZwMqVK6NYsWLqY+R2j7MDBw7A398fMTExmDhxIoYNG4aTJ0/is88+yzaPtDlHsnPy5EkULVo020L7m1QqFbp3747nz59j/fr1sLW11diu7fk3ceJEDBw4EE5OTpgzZw7at2+PxYsXo2nTplkKXy9fvkTLli1Rq1Yt/PTTTzAxMUGXLl2wadMmdOnSBc2bN8eMGTOQmJiIDh06qAtjAPDkyRPUrl0bBw4cQHBwMH755ReUKVMGvXv31njek5OT0ahRI+zduxfBwcEYO3Ysjh07hu++++6dz11ulEol/P39UbRoUcyePRt+fn6YM2cO/vjjD41+Y8aMgbe3Nx49epTreKVLl0bJkiUxZ84c/PPPP3j48CHOnj2Lr7/+Gu7u7tnmeXx8PJ49e4abN2/i+++/x9WrV9GoUSP19itXriAjIwPVq1fX2M/Y2BiVK1fWKPj/999/KFu2bJblc2vWrAkAuRaycjv3k5OTUb9+faxZswYBAQGYNWsWrK2tERQUhF9++SXX5ySzMPt2wdPMzAwAcOHCBY34zc3N4e3tnW38mXN99OgRYmJisjwnmX3ffk60GTNTtWrVACDHeysSERERkR7p+UpDIiIiIiKt5bZ0YWBgoMaSa2FhYQKAsLOzE7Gxser2MWPGCACiUqVKIj09Xd3etWtXYWxsLFJSUoQQQrx69UrY2NiIvn37ahwnOjpaWFtbZ2l/29WrV4VCoRAAROXKlcXgwYPFtm3bRGJioka/vBwnMDBQABCjR49+5/yPHTsmAIh169Zp9NuzZ49G+99///1BS/oFBwcLZ2dn9TJv+/btEwDEf//9p9FvwoQJAoDo1auXRnvbtm1F0aJF1Y9DQ0MFANGnTx+NfiNGjBAAxKFDh9Rtrq6uAoA4efKkum3v3r0CQJYl7RYvXpxlicOkpKQs89mwYYMAII4ePapue3t5T6VSKUqWLCk6d+6sse/cuXOFTCYT9+/fz+6pUstteU8hhPjvv/8EADF06FAhRN7OkZxyRNvzITY2VlhaWopatWqJ5ORkjb5vLuWX0xKHmXm3YsUKdVvlypVF8eLFxfPnz9Vtly5dEnK5XPTs2VPdpu05kpO6deuKatWqvbPfjz/+KACISZMmZdmm7fkXExMjjI2NRdOmTYVSqVT3mz9/vgAgli9frm7z8/MTAMT69evVbTdv3hQAhFwuF6dPn1a3Z56/bz5/vXv3Fo6OjuLZs2caMXXp0kVYW1urz+Off/5ZABB//vmnuk9iYqIoU6bMey/vmfme8+OPP2r0rVKlSpbnOrOvNsvgnjlzRpQuXVoAUP9Uq1ZNREVFZdvf399f3c/Y2Fj0799f4/zcvHlzlrzN1LFjR+Hg4KB+XL58edGwYcMs/a5duyYAiEWLFuUae07nfubzv3btWnVbWlqa8PX1FRYWFiI+Pj7HMS9cuCAAiMmTJ2u0Z+anhYWFxvFLlSqVZYzExESNz4dz584JAGL16tVZ+o4cOVIAUH/eaTvmm4yNjcU333yT45yIiIiISD94pR8RERERfdQ6duwIa2tr9eNatWoBALp37w5DQ0ON9rS0NPVVKvv370dsbCy6du2KZ8+eqX8MDAxQq1YtHD58ONfjli9fHqGhoejevTvCw8Pxyy+/oE2bNrC3t8eSJUvU/d7nON988807571582ZYW1ujSZMmGuNWq1YNFhYW6nFtbGwAAP/+++87l+V7W0ZGBjZt2oTOnTurl5Fs2LAhihcvnuPVfl9//bXG43r16uH58+eIj48HAOzatQsAMGzYMI1+w4cPB/B6Ccc3lStXDr6+vurHma9vw4YNNZa0y2y/f/++uu3Nq2pSUlLw7Nkz1K5dGwBw8eLFHOctl8sREBCAHTt2aFyRtW7dOtSpUwfu7u457qsNCwsLAP9bBvFDz0VA+/Nh//79ePXqFUaPHg1TU1ONMd5cKlRbUVFRCA0NRVBQkMYVdRUrVkSTJk3Ur/eb3nWO5OT58+coUqRIrn2OHTuGSZMmoX79+hg3blyW7dqefwcOHEBaWhqGDBkCufx//1vdt29fWFlZZTlPLSwsNK5g8/T0hI2NDby9vdXnJpD1PBVCYOvWrWjVqhWEEBqvnb+/P+Li4tTn6q5du+Do6IgOHTqoxzMzM9NYxvJ9ZfeavJlLwOsrRIUQWZYZzk6RIkVQuXJljB49Gtu2bcPs2bMRHh6Ojh07IiUlJUv/GTNmYN++fVi2bBlq166NtLQ0ZGRkqLcnJycDAExMTLLsa2pqqt6e2Tenfm+OlVe7du2Cg4MDunbtqm4zMjLCoEGDkJCQgCNHjuS4b9WqVVGrVi3MnDkTK1asQHh4OHbv3o3+/fvDyMjoveJ/13Pydt+8PidFihTBs2fPcpwTEREREemH4bu7EBERERFJ15uFHwDqAqCzs3O27Zn3N7tz5w6A18Wj7Ly9NFx2ypYtizVr1kCpVOL69ev4999/8dNPP6Ffv35wd3dH48aN83wcQ0NDlCxZ8p3HvnPnDuLi4rK9fyDwellPAPDz80P79u0xadIkzJs3D/Xr10ebNm3QrVu3bL8EftO+ffvw9OlT1KxZE3fv3lW3N2jQABs2bMDMmTM1CiJA1tcjs0jz8uVLWFlZ4cGDB5DL5ShTpoxGPwcHB9jY2ODBgwe5jqft6wsAL168wKRJk7Bx40b185EpLi4u17n37NkTM2fOxN9//42ePXvi1q1buHDhAhYtWpTrftpISEgAAFhaWgLIn3NR2/Ph3r17AIAKFSrkLegcZL5e2d3n0NvbG3v37kViYiLMzc3V7e86R3IjclmW9fnz5+jatSuKFCmCdevWZTk3M+PV5vzLaV7GxsYoVapUlvO0ZMmSWYqm1tbW7zxPnz59itjYWPzxxx9ZltPMlPnaPXjwAGXKlMlynA+5xyTwuvBjZ2en0VakSJFs7wWpjbi4ONSrVw8jR45UF1MBoHr16qhfvz5WrFiR5Q8bKleurP539+7dUbVqVQQFBanvi5hZwM/u/oUpKSkaBX6FQpFjvzfHyqsHDx7Aw8Mjy3mVuWTm2+fE27Zu3YrOnTur76dqYGCAYcOG4ciRI7h161ae43/Xc/J237w+J0KI9/pDACIiIiIqWCz6EREREdFHzcDAIE/tmUWDzHu/rVmzBg4ODln6vXmVoDYx+Pj4wMfHB76+vmjQoAHWrVuHxo0b5/k4JiYm2RYr3qZSqXK94i7zS3yZTIYtW7bg9OnT+Oeff7B371706tULc+bMwenTp9VXnWUnc+xOnTplu/3IkSNo0KCBRtu7nvdM2n6Z/L6vL/A67pMnT2LkyJGoXLkyLCwsoFKp0KxZM417/2WnXLlyqFatGtauXYuePXti7dq1MDY2zvG5yIurV68CgLrwlB/norbnQ2Gg7TnytqJFi+ZYiBJCIDAwEI8fP8Y///wDJyenXMfK72LGh74Pde/eHYGBgdn2rVixYj5EmLOcYnxfW7duxZMnT/Dll19qtPv5+cHKygonTpzI9WpmY2NjfPnll5gxYwaSk5OhUCjg6OgI4PWVpW+LiorSeL0dHR2zve9g5r7vOjcKSokSJXD8+HHcuXMH0dHR8PDwgIODA5ycnFC2bFl1P0dHRxw+fDhL0e3t+N/1nNja2qr/sEPbMd8UGxuLYsWKfei0iYiIiCifsehHRERERJSN0qVLAwCKFy+Oxo0b59u41atXB/C/L1ML6jilS5fGgQMH8Nlnn2l15Urt2rVRu3ZtTJ06FevXr0dAQAA2btyIPn36ZNs/MTER27dvR+fOnTWWE8w0aNAgrFu3LkvR711cXV2hUqlw584d9RUyAPDkyRPExsbC1dU1T+Pl5OXLlzh48CAmTZqE8ePHq9szr6rTRs+ePTFs2DBERUVh/fr1aNGixTuXl9TGmjVrIJPJ0KRJEwB5O0dyKlZpez5kHuvq1atZrnbT5jhvy3y93rxSKdPNmzdRrFgxjav8PoSXlxe2bt2a7ba5c+di586dGDp0KFq0aJFrvNqcf2/Oq1SpUup+aWlpCAsLy7dctrOzg6WlJZRK5TvHdHV1xdWrV7MUbrJ77vXpyZMnAAClUqnRLoSAUqnUWLYzJ8nJyRBC4NWrV1AoFKhQoQIMDQ1x/vx5jcJ7WloaQkNDNdoqV66Mw4cPIz4+XuPK0TNnzqi35yanc9/V1RWXL1+GSqXS+MOMmzdvqrdrw8PDAx4eHgCA69evIyoqCkFBQRrxL126FDdu3EC5cuVyjL9EiRKws7PD+fPnsxzj7NmzGvPUdsxMjx49QlpamkaOEBEREVHhwHv6ERERERFlw9/fH1ZWVpg2bVq297p7+vRprvsfO3Ys2/0y7xmWueTehx4nJ506dYJSqcTkyZOzbMvIyEBsbCyA18Wvt6+gyvyCN7vl3jL9/fffSExMxMCBA9GhQ4csPy1btsTWrVtzHSM7zZs3BwD8/PPPGu1z584FgFwLNnmRefXS23N/+7i56dq1K2QyGQYPHoz79++je/fuHxxX5r3LOnfurP7iPy/nSGYBLfP1zaTt+dC0aVNYWlpi+vTpWe6t9uZzZW5u/s4lUIHXVxBVrlwZq1at0ojp6tWr2Ldvn/r1zg++vr54+fJllnvNnTt3DmPGjEG1atUwY8aMXMfQ9vxr3LgxjI2N8euvv2o8L8uWLUNcXFy+nqft27fH1q1b1VeAvunN17558+Z4/PixeslLAEhKSspxWdD8FhUVhZs3b77z3qCZV61t3LhRo33Hjh1ITExElSpV1G1vL7sLvD63t27dCmdnZ/VytdbW1mjcuDHWrl2rcZ/NNWvWICEhAR07dlS3dejQAUqlUuN5SU1NxYoVK1CrVq0sS66+Ladzv3nz5oiOjsamTZvUbRkZGfjtt99gYWEBPz+/XMd9m0qlwnfffQczMzONeyq2bt0aRkZGWLBggbpNCIFFixahRIkSqFOnjrq9ffv2+PfffxEZGaluO3jwIG7fvq3xnORlTAC4cOECAGRpJyIiIiL945V+RERERETZsLKywsKFC9GjRw9UrVoVXbp0gZ2dHSIiIrBz50589tlnmD9/fo77z5w5ExcuXEC7du3Uy+9dvHgRq1evhq2tLYYMGZIvx8mJn58f+vfvj+nTpyM0NBRNmzaFkZER7ty5g82bN+OXX35Bhw4dsGrVKixYsABt27ZF6dKl8erVKyxZsgRWVla5FmTWrVuHokWL5vil75dffoklS5Zg586daNeundZxV6pUCYGBgfjjjz8QGxsLPz8/nD17FqtWrUKbNm3yfOVgTqysrPD555/jp59+Qnp6OkqUKIF9+/YhLCxM6zHs7OzQrFkzbN68GTY2Nnkq9GRkZGDt2rUAXt8368GDB9ixYwcuX76MBg0aaBQk8nKOVKtWDcDrKy39/f1hYGCALl26aH0+WFlZYd68eejTpw9q1KiBbt26oUiRIrh06RKSkpKwatUq9XE2bdqEYcOGoUaNGrCwsECrVq2yneusWbPwxRdfwNfXF71790ZycjJ+++03WFtbY+LEiVo/Z+/SokULGBoa4sCBA+jXrx+A10Wvzp07Iz09HS1btsSff/6Z7b729vZo0qSJ1uefnZ0dxowZg0mTJqFZs2b48ssvcevWLSxYsAA1atTIlwJwphkzZuDw4cOoVasW+vbti3LlyuHFixe4ePEiDhw4gBcvXgAA+vbti/nz56Nnz564cOECHB0dsWbNGpiZmeVbLLkZM2YMVq1ahbCwMLi5ueXYr1WrVihfvjx+/PFHPHjwALVr18bdu3cxf/58ODo6onfv3uq+X3zxBUqWLIlatWqhePHiiIiIwIoVK/D48WON4hoATJ06FXXq1IGfnx/69euHhw8fYs6cOWjatCmaNWum7lerVi107NgRY8aMQUxMDMqUKYNVq1YhPDwcy5Yte+c8czr3+/Xrh8WLFyMoKAgXLlyAm5sbtmzZghMnTuDnn39W36MzJ4MHD0ZKSgoqV66M9PR0rF+/Xn3uvXmfy5IlS2LIkCGYNWsW0tPTUaNGDWzbtg3Hjh3DunXrNJZj/f7777F582Y0aNAAgwcPRkJCAmbNmgUfHx989dVX7zUmAOzfvx8uLi4aBVoiIiIiKiQEEREREZFEDBw4UOT0K2xgYKBwdXVVPw4LCxMAxKxZszT6HT58WAAQmzdv1mhfsWKFACDOnTuXpb+/v7+wtrYWpqamonTp0iIoKEicP38+11hPnDghBg4cKCpUqCCsra2FkZGRcHFxEUFBQeLevXtZ+mtznMDAQGFubq7V/DP98ccfolq1akKhUAhLS0vh4+MjvvvuO/H48WMhhBAXL14UXbt2FS4uLsLExEQUL15ctGzZMtf5PXnyRBgaGooePXrk2CcpKUmYmZmJtm3bCiGEmDBhggAgnj59qtEv83kPCwtTt6Wnp4tJkyYJd3d3YWRkJJydncWYMWNESkqKxr6urq6iRYsWWY4NQAwcOFCjLbvz4eHDh6Jt27bCxsZGWFtbi44dO4rHjx8LAGLChAm5xpjpzz//FABEv379cnwu3hYYGCgAqH/MzMyEm5ubaN++vdiyZYtQKpXZ7qfNOZKRkSG+/fZbYWdnJ2QyWZZ8edf5kGnHjh2iTp06QqFQCCsrK1GzZk2xYcMG9faEhATRrVs3YWNjIwCoz73M53nFihUa4x04cEB89tln6vFatWolrl+/rtEnL+dITr788kvRqFEj9ePMeN714+fnp95H2/NPCCHmz58vvLy8hJGRkbC3txfffPONePnypUYfPz8/Ub58+Sz75uX8ffLkiRg4cKBwdnYWRkZGwsHBQTRq1Ej88ccfGv0ePHggvvzyS2FmZiaKFSsmBg8eLPbs2SMAiMOHD+f63GX3HpjTe07ma/WmzPNam9fpxYsXYujQoaJs2bLCxMREFCtWTHTp0kXcv39fo9/8+fNF3bp1RbFixYShoaGws7MTrVq1EkePHs123GPHjok6deoIU1NTYWdnJwYOHCji4+Oz9EtOThYjRowQDg4OwsTERNSoUUPs2bPnnXELkfO5L8Tr1+mrr74SxYoVE8bGxsLHxydLLuRkxYoVolKlSsLc3FxYWlqKRo0aiUOHDmXbV6lUimnTpglXV1dhbGwsypcvL9auXZtt36tXr4qmTZsKMzMzYWNjIwICAkR0dPR7j6lUKoWjo6MYN26cVvMiIiIiIt2SCfGOu6ETERERERFRtrZv3442bdrg6NGjqFevnr7D+eQdO3YM9evXx82bN9XLoxJR/tm2bRu6deuGe/fuwdHRUd/hEBEREdFbWPQjIiIiIiJ6Ty1btsSNGzdw9+5dyGQyfYdD+N+SkEuWLNF3KEQfHV9fX9SrVw8//fSTvkMhIiIiomzwnn5ERERERER5tHHjRly+fBk7d+7EL7/8woJfIbJ79259h0D00Tp16pS+QyAiIiKiXPBKPyIiIiIiojySyWSwsLBA586dsWjRIhga8u8piYiIiIiISL/4f6ZERERERER5xL+dJCIiIiIiosJGru8AiIiIiIiIiIiIiIiIiOjD8Eo/PVGpVHj8+DEsLS15/w8iIiIiIiIiIiIiIqKPjBACr169gpOTE+Tygr8Oj0U/PXn8+DGcnZ31HQYREREREREREREREREVoMjISJQsWbLAj8Oin55YWloCAMLDw1GkSBE9R0NE2lAqlbh27RrKly8PAwMDfYdDRFpg3hJJD/OWSJqYu0TSw7wlkh7mLZH0vHz5Em5ubuqaUEFj0U9PMpf0tLKygpWVlZ6jISJtKJVKWFhYwMrKir9YEUkE85ZIepi3RNLE3CWSHuYtkfQwb4mkR6lUAoDObvNW8AuIEhEREREREREREREREVGBYtFPz3RV3SWiDyeTyWBra8u8JZIQ5i2R9DBviaSJuUskPcxbIulh3hJJj67zVSaEEDo9IgEA4uPjYW1tjbi4OC7vSURERERERERERERE9JHRdS2I9/TTM5VKpe8QiEhLKpUKDx8+RMmSJSGX80JpIilg3hJJD/OWSJqYu0TSw7wlKVIqlUhPT9d3GHqjUqkQHR0NBwcH5i1RIWJsbJxjTuq6BsSin57xQksi6RBC4MWLFyhRooS+QyEiLTFviaSHeUskTcxdIulh3pKUCCEQHR2N2NhYfYeiV0IIpKenIyUlhUt8EhUicrkc7u7uMDY2zrJN1zUgFv2IiIiIiIiIiIiIqNDKLPgVL14cZmZmn2zBSwiBlJQUmJqafrLPAVFho1Kp8PjxY0RFRcHFxUXvucmiHxEREREREREREREVSkqlUl3wK1q0qL7D0SshBIQQLPoRFTJ2dnZ4/PgxMjIyYGRkpNdYuPCvnvHNmUg6ZDIZHBwcmLdEEsK8JZIe5i2RNDF3iaSHeUtSkXkPPzMzMz1HUjjou6BARFllLuupVCqzbNP15yyv9NMz3nCVSDrkcjkcHBz0HQYR5QHzlkh6mLdE0sTcJZIe5i1JDQvUr58DFv2ICp/c3p90XQNixUnPsqv8ElHhpFQqce/ePeYtkYQwb4mkh3lLJE3MXSLpYd4SSU/mPf2EEPoOhYi0pOvPWRb9iIjy4NWrV/oOgYjyiHlLJD3MWyJpYu4SSQ/zlkh6VCqVvkMgokKMRT8iIiIiIiIiIiIiokIkPDwcMpkMoaGhWu+zcuVK2NjY6D2Ot9WvXx9DhgzJt5gKg4kTJ6Jy5cr6DoMoCxb9iIiIiIiIiIiIiIjyWWRkJHr16gUnJycYGxvD1dUVgwcPxvPnz9+5r7OzM6KiolChQgWtj9e5c2fcvn37Q0LOk8yCYG4/K1euxF9//YXJkyfrLK5MSqUSM2bMgJeXFxQKBWxtbVGrVi0sXbr0g8ceMWIEDh48mA9REuUvQ30H8KnjDWiJpEMmk8HZ2Zl5SyQhzFsi6WHeEkkTc5dIepi3RAXr/v378PX1RdmyZbFhwwa4u7vj2rVrGDlyJHbv3o3Tp0/D1tY2233T0tJgbGwMBweHLNuMjY1zPKZCoYBCoci3ObxLZmEy0+zZs7Fnzx4cOHBA3WZtba3TmN40adIkLF68GPPnz0f16tURHx+P8+fP4+XLl+89phACSqUSFhYWsLCwyMdo6WOl689ZXumnZ3I5XwIiqZDL5ShatCjzlkhCmLdE0sO8JZIm5i6R9DBvScqEAFJSdP8jhPYxDhw4EMbGxti3bx/8/Pzg4uKCL774AgcOHMCjR48wduxYdV83NzdMnjwZPXv2hJWVFfr165ftspr//PMPvL29oVAo0KBBA6xatQoymQyxsbEAsi7vmbkE5Zo1a+Dm5gZra2t06dJF436ee/bsQd26dWFjY4OiRYuiZcuWuHfvnlZzNDAwgIODg/rHwsIChoaGGm0KhSLL8p5ubm6YMmUKevbsCQsLC7i6umLHjh14+vQpWrduDQsLC1SsWBHnz5/XON7x48dRr149KBQKODs7Y9CgQUhMTMwxvh07dmDAgAHo2LEj3N3dUalSJfTu3RsjRoxQ91GpVJg+fTrc3d2hUChQqVIlbNmyRb09JCQEMpkMu3fvRrVq1WBiYoLjx49nu7zn0qVL4e3tDVNTU3h5eWHBggXqbWlpaQgODoajoyNMTU3h6uqK6dOna/U8k7Tp+nOWV/rpmVKp1HcIRKQlpVKJO3fuwMPDAwYGBvoOh4i0wLwlkh7mLZE0MXeJpId5S1KWmgp07Kj7427eDJiavrvfixcvsHfvXkydOjXLVW4ODg4ICAjApk2bsGDBAvVVQLNnz8b48eMxYcKEbMcMCwtDhw4dMHDgQPTv3x+hoaEaxauc3Lt3D9u2bcO///6Lly9folOnTpgxYwamTp0KAEhMTMSwYcNQsWJFJCQkYPz48Wjbti1CQ0MLtFgxb948TJs2DT/88APmzZuHHj16oE6dOujVqxdmzZqFUaNGoWfPnrh27RpkMhnu3buHZs2aYcqUKVi+fDmePn2K4OBgBAcHY8WKFdkew8HBAYcOHcKAAQNgZ2eXbZ/p06dj7dq1WLRoETw8PHD06FF0794ddnZ28PPzU/cbPXo0Zs+ejVKlSqFIkSIICQnRGGfdunUYP3485s+fjypVquC///5D3759YW5ujsDAQPz666/YsWMH/vzzT7i4uCAyMhKRkZH59nxS4aXrGhCLfkREeZCSkqLvEIgoj5i3RNLDvCWSJuYukfQwb4kKxp07dyCEgLe3d7bbvb298fLlSzx9+hTFixcHADRs2BDDhw9X9wkPD9fYZ/HixfD09FQXEr28vHD16lV18S4nKpUKK1euhKWlJQCgR48eOHjwoHq/9u3ba/Rfvnw57OzscP369TzdTzCvmjdvjv79+wMAxo8fj4ULF6JGjRro+P/V3FGjRsHX1xdPnjyBg4MDpk+fjoCAAPUVgx4eHvj111/h5+eHhQsXwjSbauzcuXPRoUMHODg4oHz58qhTpw5at26NL774AgCQmpqKadOm4cCBA/D19QUAlCpVCsePH8fixYs1in4//vgjmjRpkuN8JkyYgDlz5qBdu3YAAHd3d1y/fh2LFy9GYGAgIiIi4OHhgbp160Imk8HV1fXDn0SibLDoR0RERERERERERESSYWLy+qo7fRw3L0Qe1gOtXr16rttv3bqVpU/NmjXfOa6bm5u64AcAjo6OiImJUT++c+cOxo8fjzNnzuDZs2dQqVQAgIiIiAIt+lWsWFH9b3t7ewCAj49PlraYmBg4ODjg0qVLuHz5MtatW6fuI4SASqVCWFhYtgXWcuXK4erVq7hw4QJOnDiBo0ePolWrVggKCsLSpUtx9+5dJCUlZSnmpaWloUqVKhptub0+iYmJuHfvHnr37o2+ffuq2zMyMmBtbQ0ACAoKQpMmTeDp6YlmzZqhZcuWaNq06TufJ6K8YtGPiIiIiIiIiIiIiCRDJtNumU19KVOmDGQyGW7cuIG2bdtm2X7jxg0UKVJEY8lJc3PzAonFyMhI47FMJlMX9gCgVatWcHV1xZIlS+Dk5ASVSoUKFSogLS2tQOLJLq7MJU6za8uMNSEhAf3798egQYOyjOXi4pLjceRyOWrUqIEaNWpgyJAhWLt2LXr06IGxY8ciISEBALBz506UKFFCYz+Ttyq8ub0+meMsWbIEtWrV0tiWuXxy1apVERYWht27d+PAgQPo1KkTGjdurHH/QKL8wKKfnvFmyUTSIZfLUapUKeYtkYQwb4mkh3lLJE3MXSLpYd4SFZyiRYuiSZMmWLBgAYYOHapxX7/o6GisW7cOPXv2VBe2tOHp6Yldu3ZpFKPOnTv3QXE+f/4ct27dwpIlS1CvXj0AwPHjxz9ozIJStWpVXL9+HWXKlPmgccqVKwfg9dV55cqVg4mJCSIiIjSW8swre3t7ODk54f79+wgICMixn5WVFTp37ozOnTujQ4cOaNasGV68eAFbW9v3PjYVfrr+nGXRT8/y8sZORPolk8lgZWWl7zCIKA+Yt0TSw7wlkibmLpH0MG+JCtb8+fNRp04d+Pv7Y8qUKXB3d8e1a9cwcuRIlChR4p334ntb//79MXfuXHz//ffo3bs3QkNDsXLlSgDv/x1zkSJFULRoUfzxxx9wdHREREQERo8e/V5jFbRRo0ahdu3aCA4ORp8+fWBubo7r169j//79mD9/frb7dOjQAZ999hnq1KkDBwcHhIWFYcyYMShbtiy8vLxgaGiIESNGYOjQoVCpVKhbty7i4uJw4sQJWFlZITAwUOv4Jk2ahEGDBsHa2hrNmjVDamoqzp8/j5cvX2LYsGGYO3cuHB0dUaVKFcjlcmzevBkODg6wsbHJp2eICitd14D4pzx6plQq9R0CEWlJqVTiypUrzFsiCWHeEkkP85ZImpi7RNLDvCUqWB4eHjh//jxKlSqFTp06oXTp0ujXrx8aNGiAU6dO5fnqLnd3d2zevBlbt25FxYoVsXDhQowdOxZA1qUotSWXy7Fx40ZcuHABFSpUwNChQzFr1qz3GqugVaxYEUeOHMHt27dRr149VKlSBePHj4eTk1OO+/j7++Off/5Bq1atULZsWQQGBsLLywv79u2DoeHr66EmT56MH374AdOnT4e3tzeaNWuGnTt3wt3dPU/x9enTB0uXLsWKFSvg4+MDPz8/rFy5Uj2OpaUlfvrpJ1SvXh01atRAeHg4du3axautPwG6/pyVibzcTZTyTXx8PKytrfHixQsUKVJE3+EQkRYy/4fIx8dHvR43ERVuzFsi6WHeEkkTc5dIepi3JBUpKSkICwuDu7s7TAvzjfx0QAiB5ORkKBQKyGQyTJ06FYsWLUJkZKS+QyP6pOX2PvXy5UvY2toiLi5OJ1fYc3lPIiIiIiIiIiIiIqJCbsGCBahYsSKcnJxw8uRJzJo1C8HBwfoOi4gKERb9iIiIiIiIiIiIiIgKuTt37mDKlCl4+fIlXFxcMHz4cIwZM0bfYRFRIcLlPfUkc3nP2NhYWFtb6zscItKCEAIpKSkwNTXV+Q1Yiej9MG+JpId5SyRNzF0i6WHeklRwec//EUJACAGZTMa8JSpEcnufiouLg42Njc6W9+RdIomI8sDY2FjfIRBRHjFviaSHeUskTcxdIulh3hJJD4t9RJQbFv30TKVS6TsEItKSSqXClStXmLdEEsK8JZIe5i2RNDF3iaSHeUskTcnJyfoOgYjyQNefsyz6EREREREREREREREREUkci35EREREREREREREREREEseiHxEREREREREREREREZHEseinZ3I5XwIiqZDL5fDx8WHeEkkI85ZIepi3RNLE3CWSHuYtkTQpFAp9h0BEeaDrz1l+qhMR5UFaWpq+QyCiPGLeEkkP85ZImpi7RNLDvCWSHiFElragoCC0adNG/bh+/foYMmSI7oL6fyEhIZDJZIiNjdX5sYnoNRb99EylUuk7BCLSkkqlwq1bt5i3RBLCvCWSHuYtkTQxd4mkh3lLVLCCgoIgk8kgk8lgbGyMMmXK4Mcff0RGRsYHjZuSkvLOPn/99RcmT56s1Xi6LtS5ublBJpNh48aNWbaVL18eMpkMK1eu1EksRLqg689ZFv2IiIiIiIiIiIiIiPJZs2bNEBUVhTt37mD48OGYOHEiZs2alW3f/Lzy1tbWFpaWlvk2Xn5zdnbGihUrNNpOnz6N6OhomJub6ykqoo8Di35EREREREREREREJDmJaYk5/qRkpGjdNzk9+Z1934eJiQkcHBzg6uqKb775Bo0bN8aOHTsA/G9JzqlTp8LJyQmenp4AgMjISHTq1Ak2NjawtbVF69atER4erh5TqVRi2LBhsLGxQdGiRfHdd99lWfLz7eU9U1NTMWrUKDg7O8PExARlypTBsmXLEB4ejgYNGgAAihQpAplMhqCgIACvr06aPn063N3doVAoUKlSJWzZskXjOLt27ULZsmWhUCjQoEEDjThzExAQgCNHjiAyMlLdtnz5cgQEBMDQ0FCjb2xsLPr06QM7OztYWVmhYcOGuHTpknr7vXv30Lp1a9jb28PCwgI1atTAgQMHNMZwc3PDtGnT0KtXL1haWsLFxQV//PGHVrESSY3hu7sQEVEmAwMDfYdARHnEvCWSHuYtkTQxd4mkh3lLUmcx3SLHbc09mmNnt53qx8VnF0dSelK2ff1c/RASFKJ+7PaLG54lPdPoIyZkvZdeXikUCjx//lz9+ODBg7CyssL+/fsBAOnp6fD394evry+OHTsGQ0NDTJkyBc2aNcPly5dhZGSEX375BatWrcLy5cvh7e2NOXPm4O+//0bDhg1zPG7Pnj1x6tQp/Prrr6hUqRLCwsLw7NkzODs7Y+vWrWjfvj1u3boFKysrKBQKAMD06dOxdu1aLFq0CB4eHjh69Ci6d+8OOzs7+Pn5ITIyEu3atcPAgQPRr18/nD9/HsOHD9fqebC3t4e/vz9WrVqFcePGISkpCZs2bcKRI0ewevVqjb4dO3aEQqHA7t27YW1tjcWLF6NRo0a4ffs2bG1tkZCQgObNm2Pq1KkwMTHB6tWr0apVK9y6dQsuLi7qcebMmYPJkyfj+++/x5YtW/DNN9/Az89PXWwl+liw6Kdn/OWKSDoMDAzg4+Oj7zCIKA+Yt0TSw7wlkibmLpH0MG+JdEcIgYMHD2Lv3r349ttv1e3m5uZYunQpjI2NAQBr166FSqXC0qVLIZPJAAArVqyAjY0NQkJC0LRpUyxYsABjxoxBu3btAACLFi3C3r17czz27du38eeff2L//v1o3LgxAKBUqVLq7ba2tgCA4sWLw8bGBsDrKwOnTZuGAwcOwNfXV73P8ePHsXjxYvj5+WHhwoUoXbo05syZAwDw9PTElStXMHPmTK2ek169emH48OEYO3YstmzZgtKlS6Ny5coafY4fP46zZ88iJiYGJiYmAIDZs2dj27Zt2LJlC/r164dKlSqhUqVK6n0mT56Mv//+Gzt27EBwcLC6vXnz5hgwYAAAYNSoUZg3bx4OHz7Moh8VOF3XgFj007O3L70mosJLCIFXr17B0tJS/YsXERVuzFsi6WHeEkkTc5dIepi39DFIGJOQ4zYDueYX7TEjYnLsK5dp3gUrfHD4B8WV6d9//4WFhQXS09OhUqnQrVs3TJw4Ub3dx8dHXfADgEuXLuHu3btZ7seXkpKCe/fuITY2FlFRUahZs6Z6m6GhIapXr57j98yhoaEwMDCAn5+f1nHfvXsXSUlJaNKkiUZ7WloaqlSpAgC4ceMGatWqpbE9s0CojRYtWqB///44evQoli9fjl69emXpc+nSJSQkJKBo0aIa7cnJybh37x4AICEhARMnTsTOnTsRFRWFjIwMJCcnIyIiQmOfihUrqv8tk8ng4OCAmJiczwmi/KLrGhCLfnqmUqn0HQIRaUmlUuH+/fvw8fHhVbpEEsG8JZIe5i2RNDF3iaSHeUsfA3Njc733zU2DBg2wcOFCGBsbw8nJKcv96szNNY+TkJCAatWqYd26dVnGsrOze6/iQeZynXmRkPC6mLpz506UKFFCY1vmFXcfytDQED169MCECRNw5swZ/P3339nG4ejoiJCQkCzbMq9KHDFiBPbv34/Zs2ejTJkyUCgU6NChA9LS0jT6GxkZaTyWyWT8bp50QtfnGYt+RERERERERERERET5zNzcHGXKlNG6f9WqVbFp0yYUL14cVlZWWbYLIeDg4IAzZ86or9zLyMjAhQsXULVq1WzH9PHxgUqlwpEjR9TLe74p80pDpVKpbitXrhxMTEwQERGR4xWC3t7e2LFjh0bb6dOntZvo/+vVqxdmz56Nzp07o0iRIlm2V61aFdHR0TA0NISbm1u2Y5w4cQJBQUFo27YtgNeFwvDw8DzFQfQxkb+7CxERERERERERERERFaSAgAAUK1YMrVu3xrFjxxAWFoaQkBAMGjQIDx8+BAAMGDAAM2fOxLZt23Dz5k0MGDAAsbGxOY7p5uaGwMBA9OrVC9u2bVOP+eeffwIAXF1dIZPJ8O+//+Lp06dISEiApaUlRowYgaFDh2LVqlW4d+8eLl68iN9++w2rVq0CAHz99de4c+cORo4ciVu3bmH9+vVYuXJlnubr7e2NZ8+eYcWKFdlub9y4MXx9fdGmTRvs27cP4eHhOHnyJMaOHYvz588DADw8PPDXX38hNDQUly5dQrdu3XgFH33SWPQjIsoDU1NTfYdARHnEvCWSHuYtkTQxd4mkh3lLVLiYmZnh6NGjcHFxQbt27eDt7Y3evXsjJSVFfeXfkCFD0L17dwQGBsLX1xeWlpbqq9xysnDhQnTo0AEDBgyAl5cX+vbti8TERABAiRIlMGnSJIwePRr29vYIDg4GAEyePBk//PADpk+fDm9vbzRr1gw7d+6Eu7s7AMDFxQVbt27Ftm3bUKlSJSxatAjTpk3L85yLFi2a4xKkMpkMu3btwueff46vvvoKZcuWRZcuXfDgwQPY29sDAObOnYsiRYqgTp06aNWqFfz9/XO86pHoUyATur6LIAEA4uPjYW1tjbi4uGwv1SYiIiIiIiIiIiL61KWkpCAsLAzu7u4sVBNRoZTb+5Sua0G80k/PeKkxkXSoVCo8f/6ceUskIcxbIulh3hJJE3OXSHqYt0TSI4RARkYGeB0PkXTo+nOWRT894xs0kXQIIRAZGcm8JZIQ5i2R9DBviaSJuUskPcxbImlKS0vTdwhElAe6/pxl0Y+IiIiIiIiIiIiIiIhI4lj0IyIiIiIiIiIiIiIiIpI4Fv2IiPLA0tJS3yEQUR4xb4mkh3lLJE3MXSLpYd4SSY9czq/0iShnhvoO4FNnYGCg7xCISEsGBgYoXbq0vsMgojxg3hJJD/OWSJqYu0TSw7wlkh6ZTAZTU1N9h0FEeaDrGhD/LEDPVCqVvkMgIi2pVCpER0czb4kkhHlLJD3MWyJpYu4SSQ/zlkh6hBBIT0+HEELfoRCRlnT9Ocuin57xDZpIOoQQiI6OZt4SSQjzlkh6mLdE0sTcJZIe5i2RNKWnp+s7BCLKA11/zrLoR0RERERERERERERERCRxLPoREREREREREREREdE7hYSEQCaTITY2VqfHXblyJWxsbD5ojPDwcMhkMoSGhubYR1/zI8ovLPrpmUwm03cIRKQlmUwGW1tb5i2RhDBviaSHeUskTcxdIulh3hIVrKCgILRp0ybfxzUwMMj3MTPJZLJcfyZOnFhgxyb6WOn6c9ZQp0ejLORy1l2JpEIul8PFxUXfYRBRHjBviaSHeUskTcxdIulh3hJJj0wmg4mJSb6MlZaWBmNjY422qKgo9b83bdqE8ePH49atW+o2CwsLnD9/Pl+ORfSp0HUNiBUnPVOpVPoOgYi0pFKpEBERwbwlkhDmLZH0MG+JpIm5SyQ9zFsi/Tpy5Ahq1qwJExMTODo6YvTo0cjIyAAA/Pvvv7CxsYFSqQQAhIaGQiaTYdSoUUhNTYUQAn369EH37t3V4x0/fhz16tWDQqGAs7MzBg0ahMTERPV2Nzc3TJ48GT179oSVlRX69euXJSYHBwf1j7W1NWQymUabhYWFuu+FCxdQvXp1mJmZoU6dOhrFwYkTJ6Jy5cpYunQp3N3dYWpqCgCIjY1Fnz59YGdnBysrKzRs2BCXLl1S73fp0iU0aNAAlpaWsLKyQrVq1bIUGffu3Qtvb29YWFigWbNmGoVKlUqFH3/8ESVLloSJiQkqV66MPXv25Po67Nq1C2XLloVCoUCDBg0QHh6ea3+ivNL15yyLfnomhNB3CESkJSEEXrx4wbwlkhDmLZH0MG+JpIm5SyQ9zFv6KOwsD1wY+r/HV6e8bkt98frxq7uvH9/+/X99TgUB++v+7/Hjva/7PN77v7b9dV/3y/Tm/vng0aNHaN68OWrUqIFLly5h4cKFWLZsGaZMmQIAqFevHl69eoX//vsPwOsCYbFixXDkyBF1IfDIkSOoX78+AODevXto1qwZ2rdvj8uXL2PTpk04fvw4goODNY47e/ZsVKpUCf/99x9++OGHD5rD2LFjMWfOHJw/fx6Ghobo1auXxva7d+9i69at+Ouvv9T30OvYsSNiYmKwe/duXLhwAVWrVkWjRo3w4sXr1ysgIAAlS5bEuXPncOHCBYwePRpGRkbqMZOSkjB79mysWbMGR48eRUREBEaMGKHe/ssvv2DOnDmYPXs2Ll++DH9/f3z55Ze4c+dOtnOIjIxEu3bt0KpVK4SGhqJPnz4YPXr0Bz0vRG/T9ecsl/ckIiIiIiIiIiIiItKRBQsWwNnZGfPnz4dMJoOXlxceP36MUaNGYfz48bC2tkblypUREhKC6tWrIyQkBEOHDsWkSZOQkJCAFy9e4O7du/Dz8wMATJ8+HQEBARgyZAgAwMPDA7/++iv8/PywcOFC9ZV2DRs2xPDhw/NlDlOnTlUff/To0WjRogVSUlLUx0pLS8Pq1athZ2cH4PWViGfPnkVMTIx6idLZs2dj27Zt2LJlC/r164eIiAiMHDkSXl5e6nm8KT09HYsWLULp0qUBAMHBwfjxxx/V22fPno1Ro0ahS5cuAICZM2fi8OHD+Pnnn/H771kLtwsXLkTp0qUxZ84cAICnpyeuXLmCmTNn5stzRKQPLPoRERERERERERERkfS0uKb5uMK41z+ZLMtk7eO7UvOxkz/g9FafJsc1H5cd+EFhvu3GjRvw9fWFTCZTt3322WdISEjAw4cP4eLiAj8/P4SEhGD48OE4duwYpk+fjj///BMnT55EYmIinJyc1EWxS5cu4fLly1i3bp16PCEEVCoVwsLC4O3tDQCoXr16vs2hYsWK6n87OjoCAGJiYtT3CnV1dVUX/DJjTEhIQNGiRTXGSU5Oxr179wAAw4YNQ58+fbBmzRo0btwYHTt2VBf4AMDMzEzjsaOjI2JiYgAA8fHxePz4MT777DON8T/77DONJUTfdOPGDdSqVUujzdfXV7sngKiQYtFPz958Yyeiwi1zHXPmLZF0MG+JpId5SyRNzF0i6WHeEhVu9evXx/Lly3Hp0iUYGRnBy8sLfn5+OHHiBOLi4tRX2QFAQkIC+vfvj0GDBmUZJ7MIBwDm5ub5Ft+by25mvo+8ee+yt4+VkJAAR0dHhISEZBnLxsYGwOt7AXbr1g07d+7E7t27MWHCBGzcuBFt27bNcszM43KJYirsdP05y6KfnsnlvK0ikVTI5XI4ODjoOwwiygPmLZH0MG+JpIm5SyQ9zFsi/fH29sbWrVshhFAXBE6cOAFLS0uULFkSwP/u6zdv3jx1ga9BgwaYMWMGXr58qbFMZ9WqVXH9+nWUKVNG95PRUtWqVREdHQ1DQ0O4ubnl2K9s2bIoW7Yshg4diq5du2LFihXqol9urKys4OTkhBMnTmgURE+cOIGaNWtmu4+3tzd27Nih0Xb69GntJkSkJV3XgFhx0rPMG68SUeGnVCpx79495i2RhDBviaSHeUskTcxdIulh3hIVvLi4OISGhmr8REZGYsCAAYiMjMS3336LmzdvYvv27ZgwYQKGDRumLhAUKVIEFStWxLp161C/fn0ArwuBFy9exO3btzUKW6NGjcLJkycRHByM0NBQ3LlzB9u3b0dwcLA+pp2txo0bw9fXF23atMG+ffsQHh6OkydPYuzYsTh//jySk5MRHByMkJAQPHjwACdOnMC5c+fUS5NqY+TIkZg5cyY2bdqEW7duYfTo0QgNDcXgwYOz7f/111/jzp07GDlyJG7duoX169dj5cqV+TRjotd0/TnLK/2IiPLg1atX+g6BiPKIeUskPcxbImli7hJJD/OWqGCFhISgSpUqGm29e/fG0qVLsWvXLowcORKVKlWCra0tevfujXHjxmn09fPzQ2hoqLroZ2trCy8vLzx9+hSenp7qfhUrVsSRI0cwduxY1KtXD0IIlC5dGp07dy7wOWpLJpNh165dGDt2LL766is8ffoUDg4O+Pzzz2Fvbw8DAwM8f/4cPXv2xJMnT1CsWDG0a9cOkyZN0voYgwYNQlxcHIYPH46YmBiUK1cOO3bsUN/78G0uLi7YunUrhg4dit9++w01a9bEtGnT0KtXr/yaNpHOyQQXvdWL+Ph4WFtb48WLFyhSpIi+wyEiLSiVSly5cgU+Pj4wMDDQdzhEpAXmLZH0MG+JpIm5SyQ9zFuSipSUFISFhcHd3R2mpqb6DkevhBBITk6GQqHg/TiJCpHc3qdevnwJW1tbxMXFwcrKqsBj4fKeRERERERERERERERERBLHop+e8S8yiKRDJpPB2dmZeUskIcxbIulh3hJJE3OXSHqYt0TSZGxsrO8QiCgPdP05y3v66VnmjVmJqPCTy+UoWrSovsMgojxg3hJJD/OWSJqYu0TSw7wlkh6ZTAZDQ36lTyQluq4BseKkZ0qlUt8hEJGWlEolbt68ybwlkhDmLZH0MG+JpIm5SyQ9hT1vhRB4ce8FhBD6DoWo0Mi8px/zgkg6dP05y6IfEVEepKSk6DsEIsoj5i2R9DBviaSJuUskPYU5b5OfJ2Pv0L3YO2wvCxxEb2A+EFFuWPQjIiIiIiIiIiKiQsW0iClkchliLsdAmVY4r0YkIiIqbLgAMBERERERERERERUacZFxsHa2Rru17SCTy2Bowq8wiYiItMEr/fRM1zdxJKL3J5fLUapUKeYtkYQwb4mkh3lLJE3MXSLpKUx5+yrqFX52+xmX115G0rMkLCi/AP9+/S+MLYxhZGYElVKFyFORXNaQCICJiYm+QyCiPND156z+P9U/cTKZTN8hEJGWZDIZrKysmLdEEsK8JZIe5i2RNDF3iaSnMOXt0+tPEfcgDtau1lCmK+HV2gvuDd3V2w+NO4TldZbjyaUneoySSP9kMhkMDAwKRd4SkXZ0na8s+umZUsk1yYmkQqlU4sqVK8xbIglh3hJJD/OWSJqYu0TSU5jytlSjUvg+6Xu4fOYCS0dLtF3TFuU7lVdv927njXpj60Fhq9BjlET6J4RAUlKS3q96rV+/PoYMGaLz47q5ueHnn3/+oDGCgoLQpk2bXPvoa370cdL15yyLfkREeVAY/meIiPKGeUskPcxbImli7hJJT2HI28zihZHCCDJ59ldDlKhRAg2nNIS1i7UuQyP6IDkVjlauXAkbGxudx6ONiRMnQiaT5fpDRIUbi35ERERERERERESkc8p0JZbUWIIzv53Rqr9KqUJGSkYBR0X0aVAqlVCpVBptI0aMQFRUlPqnZMmS+PHHHzXa3ld6evqHhkxEWmDRj4iIiIiIiIiIiHQuNiwWyc+TkfYq7Z19o0OjMddpLs4vPq+DyIh0J3O5yUmTJsHOzg5WVlb4+uuvkZb2v7yoX78+goODERwcDEdHR9jZ2eGHH37QWOYzNTUVI0aMQIkSJWBubo5atWohJCREvT3zCsMdO3agXLlyMDExQUREhEYsFhYWcHBwUP8YGBjA0tJSoy2TSqXCd999B1tbWzg4OGDixIkaY8lkMixcuBBffvklzM3NMXXqVADA9u3bUbVqVZiamqJUqVKYNGkSMjJeF/OFEJg4cSJcXFxgYmICJycnDBo0SGPcpKQk9OrVC5aWlnBxccEff/yhsf3KlSto2LAhFAoFihYtin79+iEhISHH5z8xMRE9e/aEhYUFHB0dMWfOnFxeLaLCj0U/PZPL+RIQSYVcLoenpyfzlkhCmLdE0sO8JZIm5i6R9BSGvC1atii+vfMtag+trVVfK2crmFiZ6CAykooF5Rdgz9A96sdHpxzFgvILkPwiGQDw4u4LLCi/AGd/P6vusy1oG5bXXa5+fHfvXSwovwB3995Vty2vuxzbgrapH7+5f0E4ePAgbty4gZCQEGzYsAF//fUXJk2apNFn1apVMDQ0xOnTp/Hzzz9j7ty5WLp0qXp7cHAwTp06hY0bN+Ly5cvo2LEjmjVrhjt37qj7JCUlYebMmVi6dCmuXbuG4sWLv3fMq1atgrm5Oc6cOYOffvoJP/74I/bv36/RZ+LEiWjbti2uXLmCXr164dixY+jZsycGDx6M69evY/HixVi5cqW6ILh161bMmzcPixcvxp07d7Bt2zb4+PhojDlnzhxUr14d//33HwYMGIBvvvkGt27dAvC6gOfv748iRYrg3Llz2Lx5Mw4cOIDg4OAc5zFy5EgcOXIE27dvx759+xASEoKLFy++9/NC9DZdf84a6vRoREQSZ2xsrO8QiCiPmLdE0sO8JZIm5i6R9BSGvJUbyiE3fPcXokZmRuh3vp8OIiLSPWNjYyxfvhxmZmYoX748fvzxR4wcORKTJ09WFwycnZ0xb948yGQyeHl54erVq5g3bx769u2LiIgIrFixAhEREXBycgLweqnOPXv2YMWKFZg2bRqA10tsLliwAJUqVfrgmCtWrIgJEyYAADw8PDB//nwcPHgQTZo0Uffp1q0bvvrqK/XjXr16YfTo0QgMDAQAlCpVCpMnT8Z3332HCRMmICIiAg4ODmjcuDGMjIzg4uKCmjVrahy3efPmGDBgAABg1KhRmDdvHg4fPgxPT0+sX78eKSkpWL16NczNzQEA8+fPR6tWrTBz5kzY29trjJWQkIBly5Zh7dq1aNSoEYDXxcySJUt+8PNDpC8s+unZ2+smE1HhpVKpcOXKFfj4+MDAwEDf4RCRFpi3RNLDvCWSJuYukfQUhrw9t/AczIqaoXyn8no5PknfgGsDNB5/Pu5zfD7uc/Vj2zK2Wfq0WdlG43EZ/zIoc62MRluv4700HtccqFl4ym+VKlWCmZmZ+rGvry8SEhIQGRkJV1dXAEDt2rUhk8mQnJwMhUIBX19fzJkzB0qlEleuXIFSqUTZsmU1xk1NTUXRokXVj42NjVGxYsV8ifntcRwdHRETE6PRVr16dY3Hly5dwokTJ9RX9gGv7y2YkpKCpKQkdOzYET///DNKlSqFZs2aoXnz5mjVqhUMDf9XxnjzuDKZDA4ODurj3rhxA5UqVVIX/ADgs88+g0qlwq1bt7IU/e7du4e0tDTUqlVL3WZrawtPT8+8Ph1EOdJ1DYhFPyIiIiIiIiIiItK5kPEhKOZVTOuiX8TxCPy37D/4jvBF8fLvvywhkS5YWVkhLi4uS3tsbCysra3z9VgJCQkwMDDAhQsXshTxLSws1P9WKBSQyWT5ckwjIyONxzKZLEtx483iW2ackyZNQrt27bKMZ2pqCmdnZ9y6dQsHDhzA/v37MWDAAMyaNQtHjhxRH0+b4xJ9ylj0IyIiIiIiIiIiIp3reagnhFJo3T8uIg6hK0NRtlVZFv2o0PP09MS+ffuytF+8eDHLFXmXLl1SX8EHAKdPn4aFhQWcnZ3Vfc6cOaOxz+nTp+Hh4QEDAwNUqVIFSqUSMTExqFevXgHMJn9UrVoVt27dQpkyZXLso1Ao0KpVK7Rq1QoDBw6El5cXrly5gqpVq75zfG9vb6xcuRKJiYnqguOJEyfU9zB9W+nSpWFkZIQzZ87AxcUFAPDy5Uvcvn0bfn5+7zlLIv1i0Y+IiIiIiIiIiIh0zt7H/t2d3uDV1guj40bD2FL/9yIkepdvvvkG8+fPx6BBg9CnTx+YmJhg586d2LBhA/755x+NvmlpaejduzfGjRuH8PBwTJgwAcHBwer7+QFAREQEhg0bhsDAQFy/fh2//fYb5syZAwAoW7YsAgIC0LNnT8yZMwdVqlTB06dPcfDgQVSsWBEtWrTQ6dxzMn78eLRs2RIuLi7o0KED5HI5Ll26hKtXr2LKlClYuXIllEolatWqBTMzM6xduxYKhUK9xOm7BAQEYMKECQgMDMTEiRPx9OlTfPvtt+jRo0eWpT2B11dB9u7dGyNHjkTRokVRvHhxjB07VuN5J5IaFv30jG8gRNIhl8vh4+PDvCWSEOYtkfQwb4mkiblLJD36ztu0xDSo0lUwsTbRerlBI4URoCjgwIjySalSpXD06FGMHTsWjRs3RlpaGry8vLB582Y0a9ZMo2+jRo3g4eGBzz//HKmpqejatSsmTpyo0adnz55ITk6Gn58fDAwMMHjwYPTr10+9fcWKFZgyZQqGDx+OR48eoVixYqhduzZatmypi+lqxd/fH//++y9+/PFHzJw5E0ZGRvDy8kKfPn0AADY2NpgxYwaGDRsGpVIJHx8f/PPPPxr3JcyNmZkZ9u7di8GDB6NGjRowMzND+/btMXfu3Bz3mTVrFhISEtCqVStYWlpi+PDh2S7LSvS+dP05KxNCaH8NPeWb+Ph4WFtbF8gazkRUMIQQSElJgampab6tf05EBYt5SyQ9zFsiaWLuEkmPvvP20upL2Ba4DV22d4Hnl1mX3cuOUAk8ufwEBsYGsCtnV8ARUmGRkpKCsLAwuLu7w9TUVN/h5LugoCDExsZi27ZtOfapX78+KleujHnz5kEIAZlMxs9bokIkt/epuLg42NjYIC4uDlZWVgUeC/8ET894k1Ei6VCpVLh16xbzlkhCmLdE0sO8JZIm5i6R9Og7b23cbFD5q8oo6qndFTyZ/qj+Bw6NPVRAUREVfikpKfoOgYjyQNefs1zek4iIiIiIiIiIiHTK9XNXuH6u3X26MsnkMjSe2Ri2ZWwLKCoiIiJpY9GPiIiIiIiIiIiIJKHO8Dr6DoEoX61cufKdfUJCQgC8XpaXiCg3XN6TiCgPDAwM9B0CEeUR85ZIepi3RNLE3CWSHn3m7T/9/8HB7w/q7fhEREQfIxb99Iz/U0QkHQYGBvDx8WHeEkkI85ZIepi3RNLE3CWSHn3n7d3ddxF5MjLP+x2bdgzzveYjLSGtAKIiKtxkMhnMzMwgk8n0HQoRaUnXn7Nc3lPPeEk2kXQIIfDq1StYWlrylysiiWDeEkkP85ZImpi7RNKj77wd8mAIMlIy8ryfKkMFuYEcqa9SYWxhXACRERVeQgioVCrI5XJ+3hJJhK5rQLzST89UKpW+QyAiLalUKty/f595SyQhzFsi6WHeEkkTc5dIevSdtzKZDEYKozzv5zfeDwOuDYClo2UBREVU+KWmpuo7BCLKA11/zrLoR0RERERERERERDoT/zAed3bfQdKzJH2HQkRE9FFh0Y+IiIiIiIiIiIh05t6+e1jffD0enn6Y530TYxJxftF5PD7/uAAiIyIikjYW/YiI8sDU1FTfIRBRHjFviaSHeUskTcxdIunRV9661HNByz9awrGqY573TYhOwM5vduL2v7cLIDL6WEVHR2PK5Mmo4e2NsiVKoIa3N6ZMnozo6Gh9h5ZnhfVefm5ubvj555/1HUaBql+/PoYMGaJ+rIs5Hzx4EN7e3lAqlQV6HMrZ6NGj8e233+o7DK2x6KdnBgYG+g6BiLRkYGAALy8v5i2RhDBviaSHeUskTcxdIunRZ94W9SiKan2rwdIp7/flsy1ji+77uqNqn6oFEBl9jObNmQPXkiVxfNo0DLl5EwsfP8aQmzdxbNo0uJYsiXlz5xbIcYOCgiCTyV7fv9LICPb29mjSpAmWL1+e53t8TZw4EZUrV4ZMJoNCoci3wp8+CnUPHz6EsbExKlSooNPjFoRz586hX79+BXqM7777DuPGjVO/VyuVSsyYMQNeXl5QKBSwtbVFrVq1sHTp0gKNQwquXbuG9u3bw83NDTKZTOtz+/Lly6hXrx5MTU3h7OyMn376SWP7iBEjsGrVKty/f/+94tL156xki34zZsyATCbTqKynpKRg4MCBKFq0KCwsLNC+fXs8efJEY7+IiAi0aNECZmZmKF68OEaOHImMjAyNPiEhIahatSpMTExQpkwZrFy5Msvxf//9d7i5ucHU1BS1atXC2bNn32sevMk5kXSoVCo8f/6ceUskIcxbIulh3hJJE3OXSHqkmrdGZkYo3aQ0rEpa6TsUkoB5c+Zg+tixOKpUYk9KCgIANAIQAGBvSgqOKpWY/v33BVb4a9asGaKiohAeHo7du3ejQYMGGDx4MFq2bJnlO2ltCCGQkZEBIUQBRKsbK1euRKdOnRAfH48zZ87oO5wPYmdnBzMzswIb//jx47h37x7at2+vbps0aRLmzZuHyZMn4/r16zh8+DD69euH2NjYAosjLS3tvfeNiIjIx0hyl5SUhFKlSmHGjBlwcHDQap/4+Hg0bdoUrq6uuHDhAmbNmoWJEyfijz/+UPcpVqwY/P39sXDhwveKS9efs5Is+p07dw6LFy9GxYoVNdqHDh2Kf/75B5s3b8aRI0fw+PFjtGvXTr1dqVSiRYsWSEtLw8mTJ7Fq1SqsXLkS48ePV/cJCwtDixYt0KBBA4SGhmLIkCHo06cP9u7dq+6zadMmDBs2DBMmTMDFixdRqVIl+Pv7IyYmJs9zkfIbNNGnRgiByMhI5i2RhDBviaSHeUskTcxdIunRZ94urbUU61usf+/9hRBIS3z/L6Hp0xAdHY3Ro0bhn9RU1MqhTy0A/6SmYvR33xXIUp8mJiZwcHBAiRIlULVqVXz//ffYvn07du/erXGhSWxsLPr06QM7OztYWVmhYcOGuHTpEoDXRbJJkybh0qVLkMvlMDIyUu+b236Z/vnnH9SoUQOmpqYoVqwY2rZtC+D1UpUPHjzA0KFD1VckZjp+/Djq1asHhUIBZ2dnDBo0CImJiertMTExaNWqFRQKBdzd3bFu3Tqtng8hBFasWIEePXqgW7duWLZsmcb28PBwyGQy/PXXX2jQoAHMzMxQqVIlnDp1SqPf1q1bUb58eZiYmMDNzQ1z5szR2O7m5oYpU6agZ8+esLCwgKurK3bs2IGnT5+idevWsLCwQMWKFXH+/Hn1Ps+fP0fXrl1RokQJmJmZwcfHBxs2bMh1Pm9fKfmu1+PSpUto0KABLC0tYWVlhWrVqmnE8LaNGzeiSZMmGksx79ixAwMGDEDHjh3h7u6OSpUqoXfv3hgxYoS6j0qlwk8//YQyZcrAxMQELi4umDp1qnr7lStX0LBhQygUChQtWhT9+vVDQkKCentQUBDatGmDqVOnwsnJCZ6engCAyMhIdOrUCTY2NrC1tUXr1q0RHh6e63Pk7u6Oxo0bY82aNUhKSsq174eqUaMGZs2ahS5dusDExESrfdatW4e0tDQsX74c5cuXR5cuXTBo0CDMfesPAVq1aoWNGze+V1y6/pw11OnR8kFCQgICAgKwZMkSTJkyRd0eFxeHZcuWYf369WjYsCEAYMWKFfD29sbp06dRu3Zt7Nu3D9evX8eBAwdgb2+PypUrY/LkyRg1ahQmTpwIY2NjLFq0CO7u7uo3Cm9vbxw/fhzz5s2Dv78/AGDu3Lno27cvvvrqKwDAokWLsHPnTixfvhyjR4/ONu7U1FSkpqaqH8fHxwN4XYjMXI9XJpNBLpdDpVJpnAiZ7W+v25tTu1wuh0wmy7YdyFpZzqndwMAAQohs29+OMad2zolz+pjmpFQqIYSAUqn8aOakTeycE+ck5TllHuvtvlKeU07tnBPn9DHNKfPz9mOakzaxc06ck5TnxM9czolzkuac9PWZa1nCEibWJlCpVO81pyU1liAlNgUDbw3MMidtYpfa6/SpzynzXM38kclk2X6J/nb7kj/+QAMjI9R6x73QagGob2yMpUuWYOy4cVqP/672TG9va9CgASpVqoS//voLvXv3BgB07NgRCoUCu3btgrW1NRYvXoxGjRrh9u3b6NSpE65cuYK9e/di3759SElJgYODA4QQWfb7448/0KhRI9y6dQu2trbYuXMn2rZti7Fjx2LVqlVIS0vDrl27IITA1q1bUblyZfTt2xd9+/ZVx3rv3j00a9YMkydPxrJly/D06VN8++23CA4OxvLlywG8Lgo9fvwYhw4dgpGREQYPHoyYmBj1a5TT83Lo0CEkJSWhUaNGcHJywmeffYa5c+fC3Nxco//YsWMxa9YseHh4YNy4cejatSvu3LkDQ0NDXLhwAZ06dcLEiRPRqVMnnDx5EgMHDoStra16SVUAmDdvHqZOnYpx48Zh3rx56NGjB+rUqYOvvvoKP/30E0aPHo2ePXvi6tWrkMlkSE5ORtWqVfHdd9/BysoKO3fuRI8ePVCqVCnUrFlT4/V8c16Z/9bm9QgICECVKlWwcOFCyOVyhIaGwtDQMMfn7NixY+jatavGcRwcHHDo0CF88803KF68eLbn3ujRo7F06VLMnTsXdevWRVRUFG7evAkhBBITE+Hv7w9fX1+cPXsWT58+RZ8+fRAcHIwVK1aoxzh48CCsrKywb98+AK+v9svc7+jRozA0NMTUqVPRrFkzXLp0CSYmJtnGcv36daxcuRLjxo3DgAED0KFDBwQGBsLPzy9L33Xr1uHrr7/O0v6mXbt2oV69ern2yTwH3n6tsjsnT506hc8//xxGRkbqbU2bNsXMmTPx8uVL2NjYAHhdUHz48CHCwsLg7u6eZZzMx2/We4D/vY/pkuSKfgMHDkSLFi3QuHFjjaLfhQsXkJ6ejsaNG6vbvLy84OLiglOnTqF27do4deoUfHx8YG9vr+7j7++Pb775BteuXUOVKlVw6tQpjTEy+2QuI5qWloYLFy5gzJgx6u1yuRyNGzfO8hcHb5o+fTomTZqUpf3GjRuwtHy9frmtrS1cXFzw8OFDvHjxQt3HwcEBDg4OCA8Px6tXr9Ttzs7OKFq0KO7cuYOUlBR1e6lSpWBlZYXr169rnGCenp4wNjbGlStXNGLw8fFBWloabt26pW4zMDCAj48PXr16pbFWrampKby8vPDy5UtERkaq2y0tLVG6dGnExMRo/FUM58Q5fUxzysjIwIsXL3Dt2jV4eXl9FHP6GF8nzolzenNO5ubmAICnT59qXJEv5Tl9jK8T58Q5vTmnGzduqD9vZTLZRzGnj/F14pw4J37mck6ck/TnpM/PXM/xr68aefjw4XvNqUiNIkh+mayO9WN+nTinNNy+fRtyuRwpKSkQQsDMzAwqlUrjAovMe90plUr1UoTb1q/HsDeOn5ueycmYu24dRo0eDSMjI6SmpmoUIY2NjWFoaKiOIZOJiQkMDAyQnJysMV7mlVlKpVJjm0KhgBACHh4euHr1KpKTk3Hy5EmcPXsWUVFR6n6TJ0/Gtm3bsGXLFvTq1QumpqaQy+WwsbGBUqmEQqHAkSNHcPbsWYSHh6vjmD17Nv7++29s2LABvXr1wpQpU9CpUydMmjQJKSkpUKlUKFu2LJKTk2FtbQ0DAwMoFApYW1sDAJKTkzF9+nQEBASgf//+AICSJUvip59+gr+/PxYsWIDbt29j9+7dOHr0KCpVqgSFQoElS5agfPnySE9PV883u9dpyZIl6NKlCwCgdOnScHNzw7p16xAYGAhTU1P1kqeDBg1Cw4YNYWBggEmTJqF8+fK4evUqPD09MWvWLDRs2BA//PADUlJS4OzsjMuXL2PWrFno0aOHuojWtGlT9OzZEwAwbtw4LFq0CJUrV0bLli0BAIMHD0aDBg3w5MkTWFtbw9bWFgMHDlS/TsHBwdi9ezfWr18PHx8fjdc3c45CCHXMb78ecrkcs2fPxrZt29SvR0REBIYMGQIvLy+kpqaiZMmS6vGMjIyynHsPHjxQL1OZee5NnToV3bt3h6OjI8qXL4+aNWuiRYsW6guW0tPT8euvv2Lu3Lno1KkTAMDJyQl169aFEAKrVq1CSkoKFi1aBHNzc/j4+OCXX35BmzZtMGHCBNjb20OpVMLc3ByLFi1Sz3nNmjVQKpVYunQpMjIykJ6ejt9//x1OTk7Yv38/WrZsibS0NI3cNjIygqenJyZMmIBx48bh6NGjWL9+Pb788ksUK1YMXbt2RUBAANzc3AAALVu2RGhoaJZ8MjExgUwmQ0pKCpycnNTbM/Mp5a08z1xy9c3zMbv3CAB4/PgxSpcurZ4TAHU+REdHw8zMDEqlEkWKFAEA3L9/H+7u7lneI958zd4cv1SpUtA5ISEbNmwQFSpUEMnJyUIIIfz8/MTgwYOFEEKsW7dOGBsbZ9mnRo0a4rvvvhNCCNG3b1/RtGlTje2JiYkCgNi1a5cQQggPDw8xbdo0jT47d+4UAERSUpJ49OiRACBOnjyp0WfkyJGiZs2aOcaekpIi4uLi1D+RkZECgHj69KnIyMgQGRkZQqlUCiGEUCqV6rY3299sy61dpVLl2K5SqbRuF0Lk2P52jDm1c06c08c0p9TUVHH79m2Rmpr60czpY3ydOCfO6c221NRUcffuXZGWlvbRzOljfJ04J87pzTm9+Xn7sczpY3ydOCfOiZ+5nBPnJP058TOXc5LKnBISEsS1a9dEUlKSeozMbW/+vN3u4eQkDgBCaPGzHxAeTk55Gv9d7YGBgaJ169bZbuvUqZMoV66cUKlU4rfffhNyuVyYm5tr/MjlcvHdd98JlUolxo8fLypVqiSUSqVITk5+534jR44UKpVKKBQKsWzZshxjdHV1FXPnztVoq169ujA2NtYY08zMTAAQ165dE3///bcwNDTUeI1UKpWwsbHRGOvtY7548UKYmpqK8+fPq9t++uknUbduXXX/+/fvCwDizJkzGvsBECEhIUKlUokqVaqICRMmaIz/999/CyMjI5Geni6EEMLV1VXMnDlTvV2pVAoAYtOmTeq2e/fuCQAiNDRUqFQqkZ6eLiZNmiQqVKggihQpIszNzYWhoaHo2LGjeh8/Pz8xaNCgLM+fEEKr12P8+PHC0NBQNGrUSEybNk3cuXMn13PJ2NhYbNq0KUt7RkaGOHPmjJg3b55o27atMDAwEL169RIqlUqcPn1aABD37t3L9jUfMmSIqF+/vsYxX758qfEcBwYGisaNG2vsN3z4cGFgYJBlfjKZTPz+++95yo/Y2FjRrVs3ASBLjuQ0Tl5+Ms+Bt8/t7MZu0qSJ6Nevn0bb1atXBQBx/fp1dVtqaqoAIHbu3JntOElJSeL69esiISEhy/tY5jkcFxcndEEyV/pFRkZi8ODB2L9/v8YatlJhYmKS7TqyxsbGMDAw0GjL6ZLPt/vpol0mk2XbnlOMeW3nnDinnNoL45wMDAzg4eHx3uMUxjl9aDvnxDnlV4x5bdd2TgYGBihdunS242bXPz9jzGv7p/w6vW875/RxzsnY2DjL521exylsc/oYXyfOiXPiZy7nVNDtnNPH+5kbfiQcYQfDUK1fNViVtMq1P18nzimzXSaTqX+A/y3f97Y3262trBD9+HG2/d72BK+v7snL+Nq057Ttxo0bcHd3h0wmQ2JiIhwdHRESEpKln42Njca85XK5+rtxbfZTKBRazenNbQkJCejfvz8GDRqUpa+Liwvu3LmT7X7Ztb357w0bNiAlJQW1av3vDovi/5eevXPnDsqWLavub2xsnCVu8f9Lu77ZltN/cxrjzbbM8zlz3NmzZ+PXX3/Fzz//DB8fH5ibm2PIkCFIS0vLMqfs5qjN6zFp0iQEBARg586d2L17NyZOnIiNGzeq77P49hyKFSuG2NjYLO0GBgaoWbMmatasiSFDhmDt2rXo0aMHxo0bp77KLbvXJ6fn6s22zH9nLrmaKTExEdWqVcv2/o12dnZZxszumBcvXsTq1auxYcMGyGQyDBs2DH369FFvX7dunfoK05zs3r37nct7vnnc7M7RNzk4OODJkyca7ZmrVjg4OKjbX758CQAoXrx4tuNkPjYwMMjyXpbTe1tBkUzR78KFC4iJiUHVqlXVbUqlEkePHsX8+fOxd+9epKWlITY2Vr3OKgA8efJEfQmsg4MDzp49qzHukydP1Nsy/5vZ9mYfKysrKBQK9YuWXZ/MMfJC9dZa1URUeKlUKsTExKB48eJ6WY+ZiPKOeUskPcxbImli7hJJj77y9u6euzgx4wQqdK3w3mOEHwlH6PJQfDb6M9h52+VjdPQxad2tG9ZMm4YALZb4XK1QoE23bjqI6vV97a5cuYKhQ4cCAKpWrYro6GgYGhqqlzl8m7GxsfrehhkZGTA0NNRqv4oVK+LgwYP46quvch33TVWrVsX169dRpkyZbPfx8vJCRkYGLly4gBo1agAAbt26pS5O5WTZsmUYPnw4goKCNNoHDBiA5cuXY8aMGbnun8nb2xsnTpzQaDtx4gTKli37QcWVEydOoHXr1ujevTuA1++Rt2/fRrly5bTaX5vXAwDKli2LsmXLYujQoejatStWrFihUfR7U5UqVXD9+vV3HjszxsTERHh4eEChUODgwYPo06dPlr7e3t5YuXIlEhMT1cuznzhxAnK5HJ6enrnOb9OmTShevDisrKzeGVOmhw8fYu3atVizZg3u3buHVq1aYdmyZWjWrBkMDTXLU19++aVGUTg7JUqU0PrY2vD19cXYsWORnp4OIyMjAMD+/fvh6empXtITAK5evQojIyOUL18+z8fQdQ1IMr+JN2rUCFeuXEFoaKj6p3r16ggICFD/28jICAcPHlTvc+vWLURERMDX1xfA6xfwypUrGvcX2L9/P6ysrNSJ4evrqzFGZp/MMYyNjVGtWjWNPiqVCgcPHlT3yQuRy01eiahwEUIgOjqaeUskIcxbIulh3hJJE3OXSHr0lbd+P/jh60tfw7aM7XuPERcRh0urL+HFnRfv7kyfrD59++JwejrOvKPfGQAhaWnonU2B5EOlpqYiOjoajx49wsWLFzFt2jS0bt0aLVu2VN9vrnHjxvD19UWbNm2wb98+hIeH4+TJkxg7dizOnz8PAHBzc0NYWBhCQ0MRFRWF1NRUrfabMGECNmzYgAkTJuDGjRu4cuUKZs6cqY7Pzc0NR48exaNHj/Ds2TMAwKhRo3Dy5EkEBwcjNDQUd+7cwfbt2xEcHAzg9T0YmzVrhv79++PMmTO4cOEC+vTpA4VCkePzEBoaiosXL6JPnz6oUKGCxk/Xrl2xatUq9b3x3mX48OE4ePAgJk+ejNu3b2PVqlWYP38+RowYkfcX6A0eHh7Yv38/Tp48iRs3bqB///5ZLvzJzbtej+TkZAQHByMkJAQPHjzAiRMncO7cOXh7e+c4pr+/P44fP67R1qFDB8ybNw9nzpzBgwcPEBISgoEDB6Js2bLw8vKCqakpRo0ahe+++w6rV6/GvXv3cPr0aSxbtgwAEBAQAFNTUwQGBuLq1as4fPgwvv32W/To0QP29vY5xhIQEIBixYqhdevWOHbsGMLCwhASEoJBgwbh4cOHOe7n6uqKv//+GwMHDkRUVBQ2b96Mli1bZin4Aa/vK1qmTJlcf3I7z9LS0tS1o7S0NDx69AihoaG4e/euus/8+fPRqFEj9eNu3brB2NgYvXv3xrVr17Bp0yb88ssvGDZsmMbYx44dQ7169XI9fk50/vuxThYRLSB+b9zTTwghvv76a+Hi4iIOHTokzp8/L3x9fYWvr696e0ZGhqhQoYJo2rSpCA0NFXv27BF2dnZizJgx6j73798XZmZmYuTIkeLGjRvi999/FwYGBmLPnj3qPhs3bhQmJiZi5cqV4vr166Jfv37CxsZGREdHax17XFycACBevHjxYU8CEelMRkaG+O+//9Rr3BNR4ce8JZIe5i2RNDF3iaRHynmblpQmUl+l6jsM0pHk5GRx/fp1kZycnOd9586eLexMTMTpHO7ldxoQdiYmYu6cOfked2BgoAAgAAhDQ0NhZ2cnGjduLJYvX66+72Gm+Ph48e233wonJydhZGQknJ2dRUBAgIiIiBBCCJGSkiLat28vbGxsBACxfPlyrfYTQoitW7eKypUrC2NjY1GsWDHRrl079bZTp06JihUrChMTE/FmqeDs2bOiSZMmwsLCQpibm4uKFSuKqVOnqrdHRUWJFi1aCBMTE+Hi4iJWr14tXF1dxbx587J9LoKDg0W5cuWy3RYVFSXkcrnYvn27CAsLEwDEf//9p96eeb+5w4cPq9u2bNkiypUrJ4yMjISLi4uYNWuWxpjZxQJA/P333+rHbx/r+fPnonXr1sLCwkIUL15cjBs3TvTs2VO0bt1avc/b9Yi3j5Pb65Gamiq6dOkinJ2dhbGxsXBychLBwcG5ntfPnz8Xpqam4ubNm+q2P/74QzRo0EDY2dkJY2Nj4eLiIoKCgkR4eLi6j1KpFFOmTBGurq7q52jatGnq7ZcvXxYNGjQQpqamwtbWVvTt21e8evVKvT3zfpRvi4qKEj179hTFihUTJiYmolSpUqJv37653qvuxo0bOW7Lb5mv6ds/fn5+6j4TJkwQrq6uGvtdunRJ1K1bV5iYmIgSJUqIGTNmZBnb09NTbNiwIcdj5/Q+lZGWIS7+dVGn9/STCSHdP8OrX78+KleujJ9//hkAkJKSguHDh2PDhg1ITU2Fv78/FixYoLHs5oMHD/DNN98gJCQE5ubmCAwMxIwZMzQqyyEhIRg6dCiuX7+OkiVL4ocffshy2fH8+fMxa9YsREdHo3Llyvj111/feenpm+Lj42FtbY0XL15oXCZKRIWXUqnElStX4OPjo/O1mIno/TBviaSHeUskTcxdIunRR94KIfDk0hPYlrGFsYWxTo5J0peSkoKwsDC4u7ur72eXF/PmzsXo775DfSMj9ExJgT2AaACrTU1xJD0dM376CUPfuqqnsBJCIDk5WeNeffRxGzlyJOLj47F48WJ9h/LJ2r17N4YPH47Lly9ne4UikPP71O7Bu3H016OYgRmIi4vL09Ko70vSRT8pyyz6vXz5UuMehERUeKlUKjx8+BAlS5bkfUqIJIJ5SyQ9zFsiaWLuEkmPPvI2IToBcxznoFr/ami5qOV7j6PKUOHJ5ScwMjdCMc9i+RghFUYfWvQDgOjoaCxbuhTb1q9HXFwcrK2t0aZbN/Tu00fjgpHCTgiBtLQ0GBsbs+j3iYiNjcWCBQswevRo/o6lJ1u2bIGzs3OuF33l9D4VdTEK//31H1pMbcGi38cus+inqxeaiIiIiIiIiIhIn5KeJeHs72dRokYJeDT3eO9x0pPSMc18Gip2r4i2a9rmY4RUGOVH0Y+IqCDl9j6l61oQS8N6plKp9B0CEWlJpVIhIiKCeUskIcxbIulh3hJJE3OXSHr0kbdmxcxQf0L9Dyr4AYCRmREaTW+ECt0q5FNkRNIghEBqaip4HQ9R4ZaRmoENrTbg4tKLOv/9mEU/PeMbNJF0CCHw4sUL5i2RhDBviaSHeUskTcxd0iehEkh9larvMCRH6nlbd3RdeHzxYcVDkhapnqv5TalU6jsEInrL2+9PiTGJuLv3Lp7eeKrz967s7zpIRERERERERERUyKUnpWNN0zUwL26Ozn911nc49A57huzBsxvPELA7ADI570dG2jEyMgIAJCUlQaFQ6DkaIqKs0tLSAAAGBgYAAGtna4xLHQdlmhKvkl7pNBYW/YiIiIiIiIiISHKiLkbBvpI9rJ2toSimgCpDBbkhF7UqzBKfJCIuIi5fCn57huxBxPEI9DvfLx8io8LMwMAANjY2iImJAQCYmZlBJvs0i8aZy3vKZLJP9jkgKmxUKhWePn0KMzMzGBr+r+Qmk8lgaGIIJOk2Hhb99IxvzkTSIZPJ4ODgwLwlkhDmLZH0MG+JpIm5S7oW/zAey+suh2crT7Tf0J5Xjb0HfeRt+w3t822Zs4zUDGSkZEClVEFuwGLvx87BwQEA1IW/T5UQAiqVCnK5nJ+5RIWIXC6Hi4uLOi+f3XyGV49foUStEjrPVZngYsh6ER8fD2tra8TFxcHKykrf4RARERERERERSUbqq1Qcn3Ecrp+7oox/GQCAKkOFxxceo2StknqOjogKilKpRHp6ur7DICpwr6JewdLRUt9hkJaMjY0hl//vD1AOjD6AEzNPIPh2MIzsjXRaC+KVfnrGG68SSYdSqUR4eDjc3NzU6zMTUeHGvCWSHuYtkTQxd0nXTCxN0GhqI422bUHbcH3zdXxz9RsU9Siqp8ikQ9d5+zLsJe7tvYfS/qVRxL1IgR+PPk4GBgaf9OcMP28/DQ/PPMSqBqvQYHID1BleR9/h0Hso16EcrF2sYVXSCgkpCTo9Nq99JyLKg1evdHvjVSL6cMxbIulh3hJJE3OXdEWlVEGosi5cVa1/NdQeVhuWTrwyQlu6zNvww+HY+c1OPLn0JF/Ge377Oc4tOIfY8Nh8GY9IKvh5+/GzcbWBU3UnuNZz1Xco9J6cqjuhxoAaMFIY6fzYLPoREREREREREZFk3N9/H9Mtp+Pan9c02l3ruaLx9MYwNjfWU2SUG88vPRGwOwAu9VzyZbzH5x9j18BdeHzhcb6MR0RUGNzdcxd7h+5Fi4UtUKJmCX2HQxLEoh8REREREREREUmGoakhnD9zhrWrdbbb05PTcW7hOWSkZOg4MsqNWTEzlGlWBmZFzfJlPPeG7uh5sCfc/NzyZTwiosIg5loMrm68iozkDCREJ+D+gftIS0zTd1iUR7+W/hU7+uzQy7F5Tz89k8lk+g6BiLQkk8ng7OzMvCWSEOYtkfQwb4mkiblLuuRW3w1u9d1y3P7fsv+w+9vdAIAa39TQUVTSo+u8TYxJhJmdWb4dz8LBAhYOFvkyFpFU8PP241dneB3UHFgTckM5Tsw6gUPfH0Kfs31Qogav+pMKZboSViWtYFrEFIDua0AyIUTWRdCpwMXHx8Pa2hpxcXGwsrLSdzhERERERERERB+F9KR0/Lf8P1TpXUUv99KhrNIS0zDdcjoq9ayENivb5Nu4Qggo05QwNOF1DUT08Yn6LwqRJyPh3c4blo68X61U6boWxOU99UypVOo7BCLSklKpxM2bN5m3RBLCvCWSHuYtkTQxd0lXhBDYFrgNl9ddzrGPkZkRagbXZMHvHXSZtxnJGajWrxrcGrjl25hpiWmYYjwF//b/N9/GJCrs+Hn78Xtw7AGiLkYBAByrOKLmwJos+EmcrvOVRT8iojxISUnRdwhElEfMWyLpKax5q8pQ4c/2f+L2v7f1HQpRoVRYc5c+Lq8evcKl1Zfw8PTDd/ZNiUvBsenHeC+kXOgqb82KmaHlopaoHFg538Y0MjOC55eecKzqmG9jEkkBP28/bjt67cC2oG36DoM+wOPzj3F0ylG8DHupl+Pz2nciIiIiIqJcpCWmwdjcGNGXonFn1x3YV7JH2ZZl9R0WvYeww2GwsLeAXTk7fYdCRO/JqqQVxrwag4yUjHf2vbT6Eg59fwhFShVBhc4VdBAd6ZJMJkOnrZ30HQYRUb5qMKWB+t9CCCypsQR25ezQdnVbPUZFefHg6AMc/uEw3Bq4oYh7EZ0fn0U/IiIiIiKiHFz78xoOjDqAoCNBcKrmhCERQ2BgZAAAyEjJwJnfzqDO8DqQyXV7c3bKu4yUDPwV8BdkMhm+vvw1Xtx9gZK1Suo7LCJ6D8YWxjC2MH5nv/KdysO2tC1K1Cyhg6goN/tH7Ufyi2S0+qMVZDJ+ZhIR5eTNP1KRyWQwNjeGoYJlHCmp0qsK3Oq7oahnUb0cn2eLnsnlXGGVSCrkcjlKlSrFvCWSEOYtkfQUtrx9ce8FDEwM1I/N7czV/z7z2xkc+O4AinoUhVcbL32ER3lgaGqITls6IfllMtY0XoP4R/EY8mAI7/mVTwpb7tLH69HZRzBUGMLex/6dfS3sLeDR3EMHUUmTLvM28ngkkl8m53vB79zCc3hy6QlaLmqZr+MSFVb8vP30BB0J0ncIlEemNqYaS0/rOl9lQgih0yMSACA+Ph7W1taIi4uDlZWVvsMhIiIiIqI8evX4Fe7uvYvyHctrdcUJFR5X1l9BWmIaKgdWhoGxwbt3IKJCY0nNJYiPjMfwqOFa9RdCIOVlChS2igKOjN4lPSkdRmb5+4cWmztuxvUt1/FD+g+QG7IIQkTSFnkyElu7bUXjGY1RoQuXpZaq+EfxMLUxhbH56/9H1HUtiJ+GeqZUKvUdAhFpSalU4sqVK8xbIglh3hJJj5Ty1tLJElW+qsKCnwREh0bj2uZrSIlLAQD4dPNBtb7VYGBsgLTEND1H93GQUu6StH026jON+x29y+aOmzG3xFyolKoCjEqadJ23+V3wA4CWf7TE94nfs+BHnwx+3n7cVEoVFLYKjffLqP+icGTyEcQ/itdjZJQXqxqswjLfZerHus5XfiISEeUBf6kikh7mLZH0FJa8TX6ZjAOjD+DB0Qe59kuMScTDMw91FBW9jysbrmBLpy1IfJKo0Z7wJAGz7Wfj+Izjeors41JYcpc+buXal0PV3lW17u/WwA0Ve1REelJ6AUYlXbrI26fXn+LmtpvqP7zIT4oiigIpJhIVZvy8/Xi51nNF/4v94fmlp7rt8bnHCBkfgieXn+gxMsoLn24+qNi9ot6Oz3v6ERERERERZeP5rec4MfMETIuYwvVz12z7CJXAH9X/gIGxAb69/S1k8vy9VxHljyq9qsC+oj1s3Gw02pOfJ6NEzRKwLWOrn8CIKE+EEHm+J1zNgTULKBrS1tVNV3H0x6PoH9ofDpUc8nXslNgUvLj7AkVKF4GiCJdwJaKPj2drTzjVcEIxz2L6DoW0VH9ifb0en0U/IiIiIiKibDhWc0TwrWCYWJnk2Ecml6H+xPowMjN6/WU0WPQrjIp5Fsv2ixK7cnYIPBSoh4iI6H2cmnMK5xacQ7ed3WDnbafvcEhLFQMqooh7kQL5wvrWP7ewrec2dP67M7zaeOX7+EREunRt8zW8evQKNb+tCbnB60UaLewtYGFvoefISEq4vKeeyeV8CYikQi6Xw9PTk3lLJCHMWyLpKUx5a2BkgKJli8LCIff/ya7SqwoqdKmg/h9zKlyEEFCm5b4MVvLLZFzddBVxkXE6iurjU5hylz5eRmZGMDIzgqWjpdb7KNOU2N5rO47P5DK+b9NV3hYtWxSVgyrD0DT/rz1wqu6EhtMaopgXr4ChTwM/bz9u/y37D4fGHsqyekhaYhpeRb3SU1SUF08uP8H6lutxd89ddZuu85XvDkREeWBsbKzvEIgoj5i3RNJTWPL2ZdhLJD1L0rq/UIkCjIbeV9KzJExVTMX+Uftz7BNxLAJbu2zFre23dBjZx6ew5C59vGoMqIEBVwfA1MZU633kRnLc2n4LEUcjCjAy6SrovE1PSkd6csHdT9HO2w71xtRj0Y8+Kfy8/Xi1XNwSPQ/1zLKU9S/uv2BLpy16iory4lXUK9zffx8JTxL0FgOLfnqmUqn0HQIRaUmlUuHKlSvMWyIJYd4SSU9hytttgdsw33O+Vn1XNViFxVUWF3BE9D4yUjLg1dYr16UA3eq7odXSVvBqy6Xh3ldhyl2iN8lkMgwOH4xuO7vpO5RCRxd5e3XTVcywnoH7B+4X2DGIPiX8vP242bjaoGStklnaq/WrBs/WnnqIiPKqjH8ZjE0Zi0o9KqnbdJ2vvKcfERERERFRNip0qYCk59pd6VesXDGkxacVcET0PqydrdFpS6dc+5hYmaBq76o6ioiI3kf8o3icmnMK5TuVR8naWb8QzY2JZc73ZqWCZeFggTL+ZWBXrmDuwZienI7VjVbDrYEbGk1tVCDHICLSBaESSHiSAPPi5lluG9BwSkM9RUXvQyaTQZ+3emfRj4iIiIiIKBs1BtTQum+L31sUYCSkK6nxqVApVVAUUeg7FCJ6S9TFKJyedxp25ezyXPRLep6EyJORcKjkAGsX6wKKkLLj8YUHPL7wKLDxDU0N8erRK6TGpxbYMYiIdCEhOgFzS8xFjYE10Hx+c32HQ+/pyZUnSIhKgOvnrgVyL1ttcHlPIiIiIiIi+mid/uU09g7fC5Uy92V1oi5GYabtTJxbcE5HkRFRXng098C3d7+FdzvvPO/78PRDbPxyI+7uvVsAkZE+yWQyDHkwBM1/4xfkRCRtMrkM1QdUh1t9tyzb7h+8j3XN1+Hxhce6D4zy5Pyi81jrv1avf4zCop+eyeV8CYikQi6Xw8fHh3lLJCHMWyLpKSx5G3Y4DCv9VuLB0Qda9X964yn2DtuLh2ceFnBklFc3tt7A5dWXsyyT9Da78nbwauOF4uWL6yiyj0thyV36eMkN5LAtbQuFbd6vxC1RowRaLW2FUo1KFUBk0lXQeXtn9x2sa74O0aHRBTI+0aeIn7cfLwsHC7T4vQXKdSiXZVvKyxSEHQpD/MN4PURGeVGpZyW0XNwSpkVM1W26zlcu70lElAdpaWkwNTV9d0ciKjSYt0TSUxjyNvFJImKuxrzz6rA3+5+edxrWrtYoWStvy85RwQo8FIjEmMR39jM0MXznvf8od4Uhd7UlVAKQ/f89V0gSHl94DBs3G5gVNcvzvubFzXnfzhwUZN4+v/0c9w/cR9M5TQtk/EwPjj5AzNWYPC3LTSRlUvq8pfzh1dYLY5PH8vcWCShZq6Te/3+QfxKgZyqVdl8iEJH+qVQq3Lp1i3lLJCHMWyLpKSx5W6FLBXz3/Ltsl9fJTolaJTDo/iB+4VgIyQ3lsHSy1HcYH73CkrvaurntJuaXnY/IU5H6DoW0oFKqsKLuCvwV8Je+Q/moFHTe1h5cG6PjRqOYV7ECGT/TxaUXsWvgLqQlpBXocYgKg9zyNi4yDod+OISIExF6iIw+1OW1l7GlyxbEP8p6NZ/cQM6Cn4Tp+vdjFv2IiIiIiIhyoO3/XBspjFDEvQgMjAwKOCLKi+QXyYg4EYGUuBSt+j++8BirG6/G7X9vF3BkpG8pcSl4cfcF/u33L65uuqrvcOgdVOkqNJjSAJW/qvzeY6z7Yh0WVVqUf0GRVowURgX+RbXvcF8EHg6EgTE/g+nTlhqfimNTjuHW9lv6DoXew5PLT3Bt07Uc3zPv7b+n9a0HSH+W1FyCTe026TUGFv2IiIiIiIjecmn1JYQdDsvTPgnRCXhy+UkBRUTvI/xIOFbUXYGb225q1V9uKMfD0w8RFxlXwJGRvlX5qgqGRAzBi7svEBseq+9w6B0MTQ1RZ3gdVOhc4b3HsHG3QdGyRfMxKspNXEQczi8+r5P8cqjkALf6biz60SfvwdEH8GrrhUbTGuk7FHoPTX5qgnGp42DhaJHt9i2dtiBkQohug6I8s7C3gKJo3u8/nJ94Tz8iojwwMOD/RBBJDfOWSHr0nbdCCPz79b9wrecK9wbuWu/3Z4c/8fz2c4yMGVmA0VFeFK9QHE1mNYFzHWet+tv72GNM/BjI5Fw+6X3oO3fzytrZGqNiR8HQhF+NfApaLGih7xAKpYLK24jjEdj59U6039geNm42BXKMNwmVgCpDxcIffRJyyttLqy4h/mE85Ia8zkeqcnsPa7GwBcyLm+swGnofXf/pqu8QIBNCCH0H8SmKj4+HtbU14uLiYGVlpe9wiIiIiIjo/wmVQNjhMBibG6Nkbe1vwh66KhRJT5PgO9yX99wgKqSESmCl30p4d/BG7cG19R0Oaemffv/g2c1n6HmwJ5dRloikZ0l4dPYRHKs5wsI++6tW8kvkqUis/HwlmsxuwrymT1piTCKSnich7VUaTG1MeXWzxNw/cB9mxczgUNlB36FQPtN1LYhlfz1jzZVIOoQQiI+PZ94SSQjzlkh6CkPeyuQylGpUKk8FPwCoHFgZdUbUYcFP4mKuxeDMb2eQlpCm71AkpTDkrjae336Ol/dfIiE6AcDr+z6enX8W4SHh+g2McpWelA6hEh9U8Is4EYGD3x/Eq8ev8jEyaSvIvDUrZgaP5h4FXvADAEsnS3i08ICNq02BH4tI33LLW/Pi5pDJZFhaaynO/n5WD9HRh9jSZQv2DNmj7zDoAyQ8ScCRyUcQeSpSo13Xvx+z6KdnKpVK3yEQkZZUKhXu37/PvCWSEOYtkfQUhrxVKVWFvnBB2lndaDU2d9qcp31u7biFPYP2IOZqTAFF9XEqDLmrjWJexTDs0TA0mvr6fkfpyenY/e1uXFp1Sc+RUW7arW2Hr4599UFjPD7/GMenH8eLey/yKSrpK8i8TXqepLPPUhtXG3TZ1gVebbx0cjwifcopb1UZKjy//RyWJSzRYEoDVAyoqKcI6X01+7kZfIf75rj95OyTmFlkJl7c5edYYRUbFouQ8SGIPKFZ9NP178dcuJ6IiIiIiOgN1zZdw7agbej8d2eUbVFW6/2iL0Vj59c7UXNQTfh09SnACElbMgMZ5AZ5+1vX8p3Kw6m6E+zK2xVQVFQYZN630aqEFXrs7wGnGk56joje5UOvovbp5gOPLzxg7WKdTxFRTjJSMzDbfjYqdKmAdmvb6Tscok9C/MN4zPecD9/hvmg6u6m+w6H3ULF77oVac3tzOFR2gCqjcP+B1afMvqI9vr70NczszPQaB4t+REREREREbzArZoYy/mVgVTJv91swMDZAbHgsUmJTCigyyqse+3rkeR/b0rawLW1bANFQYXBl/RUYWxrDs5Wnuq1U41J6jOjT9OzmMxydchS1BtdCiRolcu17ZPIRKIooUDO45gcd09zOHOZ25h80BmknPSkdVXpVQYmaub+2+en4jONIfpmMJjOb6OyYRIWJocIQdcfUhUs9FwCvlxPMSM6AkZmRniOj/FKpRyVU6lFJ32FQLozMjGBf0V7fYbDoR0SUF6ampvoOgYjyiHlLJD36ztvSTUujdNPSed7PztsOw6OGF0BEpGtCCCQ9S2KBII/0nbvaODD6AMyKmWkU/YRKIC4iDoamhrBwKPj7jxGQkZKBK+uuoHiF4rkW/YRK4MwvZ1DMs9gHF/1UShUSYxJhYGQAs2L6/Qv8wqQg8lZRRIFWf7TK93Fzc2fnHTy7+YxFP/okZJe3FvYWaDTt9dLVCU8SsKjSIvh084H/XH9dh0fvIexwGLYHbUfTuU1Rrn05fYdD7yk1PhVCJWBibaLX+7zznn56ZmDw/jehJiLdMjAwgJeXF/OWSEKYt0TSw7yl/PL0+lOcnH0SL++/zPO+65qtw+9ev/PejnkgldzttLUTmv3cTKPt8YXH+MX9F1xcelFPUX1anlx5gqKeRTHo3iDUHV03174yuQxDwoegzao2H3zcuIg4zHWai+Mzjn/wWB8LqeStNjr/3RlDI4fqOwyiAqdN3poVM4Odtx2XM5YY8+LmMP4/9u47oKmr/QP4NwkQ9t4bBBmKgBsF996rarVqrbNVq7a1rd19W1ttq1VbrXVVW6t11C3uCaIiKAoyZO89ws68vz/4SUVWxk1uAufzz/s2uTn3uYaTO845z2Oo0+r7/Co+bn19CwknE1QYFSGLiM0NdReL44ubvK7q8ywZ9GOYuhc5JwjiPxKJBKWlpaTfEoQGIf2WIDSPOvTb8I3huPXVLbk+m34jHVG/R9EbECGXzLBMXF13FWUpZTJ/1mOsB7q/3h1ivlgJkXVM6tB3peHQxwEug1yavGbla4W+7/aFY5AjQ1F1HhKRBIfHH8befnth6mYq1Wd0DHVg7qF4yl0DawP0WdEHzsHOCrfVUSir3974/AbOLjmr0okT+pb60NIlCc2Ijq+1fhtzIAaHxhxCeXo52Bw2FtxcgP5r+jMUJSErt6FuWPJwCTxGe7S6DYvNwu2vbpNBPzVmF2iHwEWBzTJHqPr6mJwNGUZmjhKE5qAoCtnZ2TA1NWU6FIIgpET6LUFoHnXot08PPYVEKMGQr4bI/NnIXyKRdDYJPRf1BFuLzLFkUreZ3WDjZwOrblYyf5Y8JJOdOvTd9tTz6qGlqwUtbtNHIToGOhi7bSxDUXUuErEEQe8HQUtXCywWC6lXU5F+PR0jNo5ocfukc0mw7mYNM3czhfetY6CDcb+OU7idjkRZ/TbzdiaqcqtUmtqsvqIeJUklsOhqAT0zPZXtlyBUrbV+y8viIfNOJtgccv3ZUekY6GD5k+Uy1x0nVMd7ije8p3g3e13VY0DkV4AgCIIgCIIgCOIlix8sxoKbC+T6bMhnIXjz9psAcyUciP+nZ6YHpwFO0DVR/zpzhGqEbQjDdwbfyZXylaCHFlcL/Vf3R+9lvQEAsX/H4u4Pd1GRUdFs29rSWhx/7TjOLz+v4igJRS28sxDLnyxX6T4TTydiX/99yArLUul+CUJdDP5iMD6t/RTGTg0DQmnX0nBu6TlUF1YzHBkhjWfHn+HelnuQiNteEWbTwwZ65mRiA9E2MuhHEARBEARBEATxEh0DHRjZG8n1Wfte9nAOdiazrNVAdWE1xEL50nNW5VXh3zn/4umhpzRHRTDJpocNvKd4t1jj6NmxZ9jlvwtFz4oYiKxzoCiq2Uz3wV8Mxuq01TB1NW22vZ65Hmb8MwMDPhhAWwxX1l3BqfmnaGuPaF1bdamUwaGfA4ZvHA5Lb0uV7pcg1M2LFbZFcUV4tOcRShJLGI6IkEbMHzG48ekNsNhtzxwUVAtQ+rxURVERsrq09hIurb3EdBgkvSdBEIQsjIzkewBIEARzSL8lCM3DZL8V1gpRnFAMM3czudODSUQSSMSSZikECdXaF7QPuqa6WPZomcyfZbFZiDsSB0NbQ/R4o4cSouuY1P2c2+ONHq1+nxRFoZ5Xj9riWhVH1XnwsnjY6bsTIZ+FIGR9CAC0mbaTxWK1mCJLEfnR+eBl8mhtU9PR3W9LEktQmlwKl0EuKl1pbeVjBSsf2dM5E03V8+qRfj0d3lO9VZqelZBNS/026WwSdM104RLSULc24M0A+M31g76lvqrDI+QwYdcEVBdWt9vvzi87j9jDsfik9hNo62mrKDpCWqmXU8HR5jAdBlgUKSrHiMrKSpiYmIDH48HYmOThJQiCIAiCIAh1kBuZi7399mL4xuEI/ihY5s9nhWfhwOADGLNtDPqu7KuECAlpXf3wKvTM9RD8sezfI0VREPPF0NIlA7edBUVR5AG3kpU+L8XFVRfh94Yf/Of5N3kv7Voa7nxzB6+ffx1cIy6idkWhy6gutNTyexn5npXv2IxjSPg3Ae/Ev0MG4TQMRVHY6rwVlTmVWJW8CuYe5kyHRMhgs91mmLqZYlHEIqZDIZQo7mgcCp8UInh9MLhGXKbDIVogFojB0Wk68KfqsSByB8MwiaTtPL0EQagPiUSCoqIiWFtbg80mKbsIQhOQfksQmofpfmtoa4hhG4bBdYirXJ83djJuSB/o0jx9IKFaI38YKfdnWSwWGfCTEdN9tz1Fz4pw/ePr6Le6H9xHuDd7nwwEKZ9FVwu8cfmNFt8rTS5FXnQe8h/lwzbAFtG7o/H8/HPMOT+H1hjI99yUMvrthF0T0HViV5UP+FEUhYNDDsLSxxITdk1Q6b47ChaLhQm7JyD7bjYZ8FNjrfXbiXsmNhtoKE0uhbBWCFt/W1WHSciAoihUF1RD31K/3VVi3Wd1R/dZ3VUUGSGPV/shoPoxIPW7Eu9kyEJLgtAcFEWhoKCA9FuC0CCk3xKE5mG635o4myDkkxA49nOU6/OmLqaY+e9MeE30ojkyQtWK4oqQejWV6TA0BtN9tz3lqeVIuZyCuvK6VrdJvZqKiM0RKoyKeKH3st5YmbQSroNdoWuiC9chrhj+3XDa91ORUYHE04lt/h10Jsrot/qW+ghYEEBbe9JisVgQVAsgrBGqfN8diedYTwz7dhjTYRBtaK3fdp3QFV1GdWny2sEhB3Fu8TlVhkfIgV/Jxxb7LTi/7DzToRAKENWLkHI5BeXp5c3eU/X1MRn0IwiCIAiCIAiCIDqUtOtpODb9GPIf5cvdxtV1V/HP5H/UdhCLkI3XJC98UvMJvCe3XiMuZn8Mrn5wFYJqgQoj6zwif43Era9vtdinWGwWjB3+S3c1esto2PSwoT2GhFMJODr1KIrji2lvuzO6/N5lpF1LAwDsCtiFsO/CQEmY+81cGr0UU/+aytj+Ndn9bffx7PgzUBIK0XuiEf9vPNMhETQI+TQEfd8l6ebVHSWh0GdlnxYzEbyq9Hkp/h77N+L+iVNBZIQsKnMq8feYvxH9ezTToZBBP4IgCIIgCIIgiBdufXULf4T8AX4lX+427v54FxffvUhjVISsSp+XIvF0IoR18q/46LuqL8b/Nh4gY34dBkeb02LKpRdCPg3Bsphl0NIjqV2V4cnBJ3i89zGjKTY9xnhg6l9TSepCGpSnlePhzoeIORADQY0AlJhC6uVUMlFCQ9387CYe7XkEFpuFax9eQ+QvkUyHRMgg804mNppuRMzBmCav93mnT7MaqoT60TPTw7hfxsFvjp9U22fczgAvi6fkqAhZ6VnoYcLuCfCe0voEM1UhV7IMI/nkCUJzsFgsmJubk35LEBqE9FuC0DxM99v6inqUpZRB20Bb7jbSr6UjOyIbY7aNIb8/DOnzdh/0XNxToX9/z3GeNEbU8THdd9sirBXi8f7H8BzvCTM3s1a3s+5urcKoOp/5N+ajpqiG0RisfKxUXmtOnSnSb83czbAyaSU42hzoGOjg7di3QVEUo78BeVF5yArPQuCiQHCNuIzFoWkoisJbEW+BxW747mafnQ0jOyOGoyJa01K/5ehwYN/LHoY2hgxGRqiCuac5Pqn5RC2vtzo7PTM99FrSq8X3VP19sSgyBYcRlZWVMDExAY/Hg7GxcfsfIAiCIAiCIAhCI9SV1UHHUKfNFUUEQahO6pVUHBp9CCN/HIkBHwxoc1tBjQDCWiEMrAxUFB1BEHS59dUt3P76NpY/XQ4bP/rTwxKEpskMy0ToO6EY8r8h8Jnqw3Q4RCuSzibh6V9PMfSbobD0tmQ6HEIJVD0WRNJ7MkwikTAdAkEQUpJIJMjKyiL9liA0COm3BKF5OkK/1TPXIwN+DEu5lIK8qDyF2ki7noZtbttIXSMpqXPfdQxyxOyzs+H7mm+b20nEEmw02Yjzy86rKLLOo668DgVPCiCslT/lLh0qcyrxs/PPuPnFTUbjUBfy9tvIXyMRvilc7VJ59nijB9688ybM3Ftf0Us0J6wToq68rvH7pCgKdWV1ENWLGI6MaIks/VZbXxvCOiH5LtVccUIx4v+Nh1gglmr7rPAsJJxKUHJUhKwe73+MX71+bbGmuKqvj8mgH8PU7QKJIIjWURSFsrIy0m8JQoOQfksQmofJfktRFCJ3RCLrbpZC7fAr+ci+l43qgmqaIiNkdWr+KVxac0mhNrhGXOia6oLNIbfN0lDncy7XiAuviV4wdTFtczs2h40+7/SB+0h31QTWiaReScXvAb8j8Uwio3FwjbkwtDGErpkuo3GoC3n7bdSuKDza/Ujt0suZe5jDJcQFOgY6TIeiUVIupuAH8x8QezgWAPBo7yP8YPEDMm5nMBsY0aKW+m3skVjc/PJms8E9+172eDflXfi9Ll2tOIIZwR8F43PB51KnGb/x6Q2cW3xOyVERsmKxWeDocFqsDa3q62NS048gCIIgCIIgCAIN9fwurryIwEWBcB7oLHc7mXcycWTiEUzaNwmBbwXSGCEhrXE7xkFbT/66jADg0NcByx4voykigimCGgFqS2rbHfB7Yez2scoNqJOy8rXCoC8GwaGPA6NxcI25WPJwCaMxdARv3X1LbSe2SMQSiOpE0DEkA3/SMnIwQuCiQFh3axhwsA2wRcBbAdC31Gc4MkJaCScSkHg6EYO/GMx0KISc2FrSTzIb9MUgiOrI6k11E/BmAALeDGA6DABk0I8gCIIgCIIgCAJAQwqk+TfmQ99CsYdcNv42GPnjSDj0ZfbhdmfW7bVuTIdAqImUiyk4/tpxTP1rKnq80YPpcDotGz8bUmOtA9E10YWuifqtlhTWCrHRZCO6v94dU/+cynQ4GsOxnyMc+zk2/rdDHwfGB+gJ2UzYPQFDvxnaYnaC5NBkFMcXt1vTlmBO5p1MaOlpSd3v3IeTjARE20ieEoapWyoEgiBax2KxYGtrS/otQWgQ0m8JQvMw2W+1uFpwG+oGmx6KPZg2cTLBgA8GSJ2ih1BfUbui8GjfI6bD0Ajqes41cTFBzyU94Rws3erd+H/j8fe4v8HL4ik5MoIpMQdjELE5gukw1II8/bYypxLF8cWQiNSvfqe2vja6zewG+972TIdCEErTUr/Vt9CHla9Vi9s/+fMJrq67ynhNVaJ155aew5mFZ2T+nDqmVO/MEk4mIPLXyBbfU/X1MRn0YxibTb4CgtAUbDYbtra2pN8ShAYh/ZYgNA+T/VYsFIOSkJtnTRd/Ih6b7TYj9Uqqwm1F/BSByO0t37wTTanrOdehjwMm7p4IU1dTqbavyqtC+o10VOZWKjewTubAkAMIXRnKdBgAgJj9Mbi78S7TYagFefpt9J5o7Oy2E0VxRUqMTH7T/p6Gfu/2YzoMjXLrq1s4u/hsk9eufXwN55aRmmHqqKV+W5JUgnpefYvbD/5iMJY/WQ4Ol6OqEAkZjdg4AiGfhki9fdK5JGw03YjEU8zWySWait4djevrr7f4nqqvj0l6T4aJxWKmQyAIQkpisRgZGRlwdXUFh0MulghCE5B+SxCah8l++3DnQ1x5/woWhi2EU5CTQm0dm34Mwloh5l6cS1N0hLQ4XA5MXU2ha6Z46rmZJ2ZCx4jUhZKGOp5z+VV86BjqyDS7uvfy3ui7sq/arVjUZBKxBPXl9eBX8pkOBQAwcc9EsNjk+wXk67duw9wg5oth4WWh5OgIVcm4lYGylLImr+XczwEvk6x4Vkev9luxQIwd3jvQfXZ3TD8yvdn2ra0AJNSH9xRvmbY3sDaAfS97co2qZsZuH4va0toW31P1GBAZ9CMIgpBBVVUV0yEQBCEj0m8JQvMw1W9NXUzRdUJXGNkZKdwWSbfDHK+JXvCa6EVLW7YBtrS001mo2zn39ILTqMiowKKIRdDSle7xB0dbPQYsOxI2h43lT5YzHUYji65ksOplsvZb18GucB3sqpxgaJBwKgFxR+Iw7tdxMLA2YDocjfDmrTchrGua+vGNS29I/btJqN7L/VYikiD4k+A266YKqgUQ1AhgaGOoivAIJXPs54j51+czHQbxCouuFrCAelxjqFfeDYIgCIIgCIIgCIZ4T/HG7NOzpU4D2JZZJ2eRVX4dgEQsQXVhNcQCkqFF01AUBRMXE1h6Wcr04JqSUEi7lobsiGwlRkcwSSwQoyqvCiK+iOlQCCUoSy5D/PF4UpdTRtp62k3+mwz4aQ5tfW0M3zAc3Wd3b/F9QbUA3xt9jyvvX1FxZIQ04o7GYYvjFqRdT2M6FEJB/Co+JGL1qHdLBv0IgiAIgiAIgiCIDuPB9gd4uPMhLW2FbwzHZtvNalu7imgdi8XCmJ/HYNrhaTJ+EDg84TDCNoQpJ7BOKP9RPqJ+j0J1YTXToQAAHvzyAFsctiD/UT7ToWicomdF+KXrL3j691OmQ2lV35V98Wn9p7Dvbc90KBpBVC9CcmgyKjIqmrxeX1GP5xeeN0v7SWgeHUMd9FreC65DXZkOhWgBR4cDI3sj6JnryfS58I3huL/tvpKiIuTxs9PPODT6ENNhACCDfowjdQIIQnOwWCw4OTmRfksQGoT0W4LQPEz22zsb7uDWV7doaaskqQQRmyNQnlZOS3uE9B5se4Do36NpacuxvyN6v9MbXGMuLe11ZOp0zhXxRY0zrWWNh8ViYeKeiRjw4QBlhNYpJV9MxoXlF1CVqx7pXx36OKDvqr7Qt9BnOhTGydpv68vrwWKz1DoNrra+NrS4ZJWatCoyK3B4/GFE72563iyOL8aRCUeQcCqBociI1rzab59feI5Dow8hLzqv1c9M+G0Cei7qqaoQCRn4TPXBksglsAu0k+lzj/Y8Qsz+GOUERcil26xu6DK6S4vvqfr6mEWRYhOMqKyshImJCXg8HoyNjZkOhyAIgiAIgiA6vR2+OwAAK+JXKNzWs2PPcGLWCUz/Zzq6z2o53RKhHJU5lRBUC2Dpbcl0KARDIndE4v7P9/H6uddh5WPFdDidXmVOJYrji+E0wAk6hjpMh0N0cBKxBHlReeDocGR+iN4Z1ZbWIvF0Imx62MChj0Pj6/UV9Yg/EQ/nYGdyPlVz0XuicXntZcy7Og9OQU5Mh0OoSFlqGXRNdKFvSSawaAJVjwWRQT+GvPiiy8rKYGZmxnQ4BEFIQSwWIzk5GZ6enuBw1HdmI0EQ/yH9liA0D5P9VlgrhKBaAANrA4XbqimuQVlyGSx9LKFnJlu6HoLQROp0zo3cEYkHWx9gWcwy6BiQQSaCaI069Vu6SMQSfMv9Fl0ndMXs07OZDocgaNdav6UoqtXVRBm3MhD2XRiG/m8oHPs7qipUQgq3vroFCy8L+L3ux3QohBKVl5fD3NxcZYN+JL0nQRCEDOrr65kOgSAIGZF+SxCah6l+q62vTcuAHwAYWBnAaYATGfBTMUGNABWZFRALxLS0x6/i48TsE3iw/QEt7XV06nLO7buiL1Y+Xyn3gN+DXx5gk9kmFMcX0xxZ5ySoEUCd5pvXFNfg2IxjiN5DTxpgTSdLvw3fGI7EM4lKjEZxbA4bY7aNQe+3ezMdCkEoTUv9tq30gcI6IbLvZoOXxVNmWISMJCIJ7nx7B/HH4mX+rKBagOL4YgjrhEqIjJBVZW4lTs07haSzSUyHAoAM+hEEQRAEQRAEQUDEFyH3YS5qimtoa1MsFKOepx6DIJ1FVlgWtrluQ8yBGFra09bTRvzxeORFtV4nh1BPitROMbA2gG2AbWNdQEIxu/x3YU+fPUyH0YitxUbCyQQUxRYxHYpGEdWLcOPTG3j651OmQ2lX3xV94THag+kwNMKdb+/gF89fwMtuPhh0cu5JbHXdqvqgCJmkXk1FyuWUNrfpMqoL1letR7eZ3VQUFSENFoeF1WmrMWrzKJk/G7kjEju77STnMjVRnV+Np4eeqs2EMVLZliAIgiAIgiCITq88rRx7++5FyKchGPbtMFra3GS2CS4hLph7cS4t7RHtM3IwQv+1/WHf256W9thabHxa/yk42h0j7V1ncHj8YTgPckbwR8Fyt9F9VndSi5NGHmM8oGumy3QYjXRNdfG58HOwOWQevCw4Ohy8E/8OKIn6rNokFKelqwUtPS3omjTvo0aORrD0toRELCH9RY1dX38dtcW1WJO5ptVtyPennlgsFkycTeT6rEuIC0I+DYG+Fanppw7setnhk9pPmA6jEanpx5AXNf0qKipgYiJf5yYIQrUoikJVVRWMjIwUmjlMEITqkH5LEJqHqX5bXVCNmAMxcBrgBJdBLrS0eWHFBZi6mGLghwNpaY8g6FaVXwVDG0Ow2Ir3NXU459YU12B3z93wnOCJCb9NYCQGgtAk6tBvleHhbw8R/l045l+fD4uuFkyHQxC0erXfpt9IB7+KD+/J3m1+LudBDkqTSuE/319FkRLtqS6shlgghrGDMS3XYoT64vF4MDU1VVlNPzLox5AXg36q+qIJgiAIgiAIgiAI2RU9K0JFegW6TujKdCi0EgvFOLfkHAxtDTFi4wimw6ENJaEgqBGAa8SVuw2JWIKwDWEwcTZBwJsB9AVHqI28qDzUV9TDfYQ706FoDF4WD1xjLnRN1WfVZmue/PkEkb9EYuLeibD1t2U6HIJQC8emH0Pi6UR8UvsJtLgk+Z86uP7pdYR/F453nr0DK18rpsMhFFCVXwVeFg9WPlbgGje/BlX1WBBZ28swsZieAvMEQSifWCxGbGws6bcEoUFIvyUIzUP6LaGIKx9cwYlZJ2ht8843d3Bk4hGIBR3rb5KtxQYvk4fH+x7TUntSXfoui81SaMAPaEiDFvFTBJ4eUv/aZequKK4Ip988jazwLKZDaeLy2ss4Ofck02EwTpZ+G7oyFD9a/wixUP1/C/3n+2PJwyVkwE8K4ZvCEf9vfIvvlaeX48ZnN5Adka3iqIi2yHu+DXo/CHNC53SoVb2azjnYGX1X9YWpq6nMnxXWCXFo9CHc+voW7XERsks4mYB9/fchL7rlOuCqvj4mw/oEQRAyYPohBkEQsiP9liA0DxP9NmpXFJ78+QTTDk2DmbsZLW0+O/YM8cfjMf638dC3JPU2VKH4WTFKk0tpbbPX0l7wHO9Ja5vqgMViYeLeidAx1GmxlpM8mDznSsQS3PnmDtyGu8ElRPEUvYsfLIahjSENkXVuJUkleHLwCbqM7sJ0KE0M+HAAhDVCpsNghEQkQeSvkej3bj8A0vdbr0lesPSxJDVOOxBKQuHGpzfQdUJX+E73bfZ+bUktwjaEQcdQB04DnBiIkGjNi35bllKG3b12I/iT4HZr2ZLvUP14jvWE51j5rjG1dLWQF50HYyeSQVAdOA1wwogfRqhNSmky6EcQBEEQBEEQRKdXz6tHRXoFOFz6HmaWJJYg/kQ8Bn0xiAz6qcjci3NBSeitYOE2zI3W9tRB5p1MXH7vMsZsHQPnYGcADQMBbC3NTQZUllKG21/fhrBWSMugn5UPSbNFB9/pvvi48mO1+9vymujFdAiMebz/MS6vvQy34W4wcTVBwe0CeHt4g2PY9vmv5+KeKopQcRKRBPe33Yepiyl8ZzQfzCL+s/zJ8lb7p42fDd6JfwcmziYqjoqQFkVRcOjrACN7I6k/IxFLwOao128yITsWi4UPSz5kOgzi/9kF2sEu0I7pMBqRHk4QBEEQBEEQRKcX/FEw3s9/H8YO9M2WHbBuAD4TfAYbPxva2iTax2KTtFXtKU8vR+nzUnBNGtJgJp5JxPYu21HwpIDhyORn5m6GZY+Xoffy3rS0J6gWoCSpRCNSGao7rhEX2nraTIdB/D/3ke7ov7Y/jB2MEf17NB6seYCM6xlMh0UrFoeFG5/cwJODT5gORa2x2CxYd7OGpZdli+9r6WrByscKOgY6Ko6MkJaFpwXmXZ0H/3n+7W5bmlyK7wy/w+2vb6sgMqI9gmoB9vTdg8hfI5kOheiAyKAfw9hs8hUQhKZgs9nw8vIi/ZYgNAjptwSheTpSv9XW0yZp0FSonlePuH/iUJZSRmu76TfTsdVlK+KOxtHaLpMCFgRgXdG6xgFpQ1tDiIVi1JXVyd0m032Xo82BbYAtbel572y4gx3eO1CRXkFLe51VUVwRCmIKQFH0rsBVVMyBGPzs9DNyH+YyHQptKjIrkBfVUEtIxBe1up2ZmxlGbxkNPXM9+E73xdBNQ2HXq+3VCfH/xuPAkAMoiNGMiQEsFgsLwxdi3M5xTIei1gQ1AtSW1rbZP+vK6lD6nN602YRi5D3fGlgZwDnYGaZupsoJjJBJTVENqnKrUFtSK3cb+Y/z8Xj/Y0hEEhojI+Rxbf017Om7p9Xzr6qvjzX/TpogCEKFdHTIDDeC0DSk3xKE5mGi38b9E4fn55/T2qaoXoSsu1nkYZmKlCSU4N/X/0XCqQRa2+UacaFvpd9hBnDrefUAGlZwvODYzxGr01bDbahiqUyZPOeWp5dDWEdfjTa3YW4IXh8MHUNyHaGIax9dw97+e5kOoxkdI52GdHjqNRapkLDvwrCnzx5EbI7AdvftyIvOa7ZNXlRek4fDpq6mGPj+wHZXuVcXVKPgcUGT3w1159DHASZOJC1lWxJPJeJHyx+ReCqx1W2OTDyCP0L+UGFUhDRenG+TQ5Nx4/MbUg0c6Zrq4o1LbyBwYaCywyOkYOZuhvdy38PgLwfL3caTg09wdtFZVOZU0hgZIQ9hrRC1xbXg6KjH/QIZ9GOYREJG4glCU0gkEsTGxiq130pEElz54IpGp1YiCHWiin5LdCy1JbXYH7wfzy/QO/hDSI+pfnt13VXc/h+96Y6qC6vxR/AfiNoVRWu7RMvMPcwx49gM2mt12fe2x9KopfCZ5kNru0xIu56GbW7bkHoltdl7Lx7m1xTVyDXrnOlz7t9j/sbuXrtpa6/LyC4Y/t1wmeokEc31XNITIzaNAIulXml3faf7YvGDxXDo68B0KLTpu6IvBn85GO7D3UFJKNQU1TR5v7qgGvsG7MOpeacaX3vRb/k1fNRX1LfZ9kcVH8HCy0Jp8dNNLBSjKq9K7VaZqhNTN1P0XNITlj4tp/cEAP8F/uizsg/5d1QjL59vUy6lIOzbMIgFJBW1plLk/BiwMABzLsyBvhWpHc60sdvGYnX66la/T1VfH2vOFB2CIIhOIPNOJu5tvge2FhtlyWXwme6jdjfIBEEQHQ2/kg+JWAI9Mz3kP8pHweMC8Hl8psMiVGza4Wm0t2lkZ4RRW0Z1qIfK6kzfUh/dXuvGdBhqzcTJBPoW+uBwW56FnHM/B/uC9mHYd8MQsj5ExdHJj6Io9JjXA1p65BGHuvGe4s10CJ2GTQ8b2PRoSNm7On1140A+JaHAYrPA0eEg5NMQOPZzbPK5ytRKbBm4BUHvB2Ho/4a22r6m3ZeeX3oeMQdi8FHFR9A10WU6HLXkPNAZzgOd29ym19JeKoqGkMfQb4ai9/LeMLAxkGr7qN+jUPq8FKM3j1ZyZER7su9lg5fJQ9eJXeWum2nrbwtbf1uaIyM6ArLSjyAIQo24DXPDkqglqMiowPHXjiPnfg7TIREEQXR4t7+5jSsfXIFYKEaXUV3wfsH78Jmu+St6CNm4hLjAJcSF1jY5OhwErQ1q94EaQQ9lrkKI/DUSMQdjlNa+qlh0tcCKhBVwHeza4vvW3a0RsDAAdoFt1/dSNywWC4M+G4QB7w+grc3qgmr8PfZvPNz5kLY2CfXBr+Tj7o93kXI5helQFCaoESDtelqT38AXA37VBdXYbL8ZFRkV0DPXw5Avh8BjjEeTzxu6GMJ1qGurq/gENQLc+/mexmWjcR/p3rBCTUJWqBEdl66JLqx8rcDmSPeIPyU0BZHbI0m/UAMxB2Lw7+v/QlTXeg1WaVAUBVG9Ym0Qint66CmSziYxHUYjMuhHEAShZux72WPIV0MwZvsYWHezZjocgiCIDq+2qBa5D3LB1mq4NOYacaHF1ULatTT8NfIv8KvIqr+OjpJQ5OFHB3DmzTP4yfYnpTz4CN8Yjqidmpumtba0Fs8vPEdNcU3jb11LdAx1MHn/5GaDAp0Rh8tBxu0MVGRWMB2KxqotqcWvXr8i4qcIpkNpRiwQ49qH15Bwkt4aoEzIf5SPv0b81WIq6dLnpRDVifDkryetfp6txcasM7PQY26PFt8vji/GlfeuqNXDTGn4zfHDuF/GQc9Mj+lQ1Na1j6/h3LJzbW6TF5WHg8MO0l73mKBHWWoZ6srqpN5+wu4JWFe8DtCshbsdUr93+2HG0RnQM5f/N0oiluAn65/w7+v/0hgZIY+rH15F+MZwpsNoRHJfMIzNJuOuBKEp2Gw2/Pz8lNZvE08nQs9cD84hzrD0toSld+t59QmCkI6y+y3RMUw5OAXCOmGztFVlqWXICs9C7oNcuI9wZyi6zoeJfpsXnYd9/fdh1JZR6L+6P61tn3/7PLLCsvBO3Du0tks0Z+puChs/m8YVLnSac34OuCZc2ttVlazwLBydchST9k1C4FuBStkHk+fc8I3hyLyTiWmHpin08Oxluqa6+KTmE41LaagKGbczUFtcC++p3m2uLuFX8cHRaTmVLNP0zPWw6P4imDibMB2KwozsjTD4y8EtppJ2GeSC9/Lea/V3UZp+a+Vrhfk35neIfyuiqczbme3WcJWIJch/lI/qgmoVRUW05+V+uy9oHyy9LbHwzkKpPmtoY6jk6AhpWXezVniiP5vDRpdRXWDe1ZymqAh5vXbstTYn1qn6+phFkUqsjKisrISJiQkqKipgYkIunAhCE1AUhfr6eujq6tJ+809RFLa5boNELMGajDWNJ4p6Xj1i/45F77d7kwcOBCEHZfZbQvNRFNXm34WIL0J9eT0MbcnNsSox0W+L4opw8/ObCFwUiK4TutLa9sXVF5F7PxdvRbwldeolgqAbL4uHlEspcB/hDjN3sza3Tb+RjuufXMfIH0bCZZD0KW+ZOudSFIXQlaGIOxKHdcXrSD9TgYNDDyL7XjbWZKwh50gN96LfcrlcnJp7ChKRBK8df43psGhRklSCG5/eQI83epDakm0Q8UXQ4rY+Waa962VC9V4+39766haM7IzQe3lvqT4rrBWiLLUMRvZG0LfQV3KkRFskYgm5ZulEeDweTE1NwePxYGxsrPT9kb8shkkkEqZDIAhCShKJBElJSUrrtzOOzcBrx5vODLm+/jpCV4Qi7VqaUvZJEB2dsvstodmSziTh4NCDKI4vbvF9La6WQg8zydw6+TDRb627W2PWqVm0D/gBwNhtY7H4wWJyU6/hxEIxqvKrIBFr5vnExNkEvZb2anfADwBYHBbKUspkXtXBRN8tTyvHvqB96LWkFz4s+ZD2fpZ9LxtxR+NobbMjmHF0Bmb8M0Mpq2pVSVQvQk1RDdNhMOpFv6UoCvW8etSV1zVLd12eVq6RKbDFAjESTiagJLGE6VDUWlsDfgDIgJ8aevl8O/TroVIP+AFA6tVU7OqxC8kXkpUYIdEeiViCDXobcGbhGaZDIWhAUe2XilD1Myly50kQBKEGWCwWHPs5winIqcnrw74dhol7JsJ9hDuy72Xjj5A/kPswl6EoCYIgOpb8x/nIf5Tfbiq4qF1RuLT2kkxtl6eVY6fvTlz7+JoiIRKE2hJUCyARqc8AmKBagIurLyI5VDkPsa6vv44t9lvAy+IppX1lk+WBvUuIC9YVr0O3md2UGJFixAIxACDnfg4KHhegtrQWLDb9D6bDvw/HqXmnNHLAQ5kMrA2QfDEZP1j8AEGNoNXtssKz8PC3h6grl77elCodHHYQu/x3MR2Gwg5POIzTb55WuJ3Xz76O+dfmN+lLdeV12N5lO84uOqtw+6pm3c0anws+R/DHwUyHopYE1QI8P/9cqrql6TfSEX8iXvlBEUpn42eDwV8OhrWfYmklCcUIa4XwmuRFy/eQ+zAXZ946g/zH+TRERsijMqcS/9P6H65+dJXpUBqRQT+CIAg1wK/iNz68eJmeuR56Lu4JFosFLa4W8qLyUBRXxECEBEEQHc/Qr4diXfG6dlfzpVxKQcz+mDYfbL6qurAaJYklcBvupmiYhArEHY3DpTWXlPJgujytHBE/RXSo8zdFUTg1/xQODjsIQbX0/UKZeNk8RG6PRNbdLKW07zLIBX1W9Gl3RYQ6kogl+MHiB1x454JU27PYLLVf2XFk4hEcn3kc3WZ1w+r01XAfrpy6qwPWDcDMf2d2ypXbFEWh9HkpRHxR42vZEdl4tPcRRHwRHPs7ose8Hm3+BiScTEDoO6HgV/JVEbLMfKb7wG+uH9NhKKyurA71FfUKt9NSLSKxQIz+a/vDY5yHwu2rGovNarO+UmdXmlyKIxOP4OlfT9vd9sZnNxC6MlQFURGyyLmfg79G/iVTZigzdzMM+WoI7ALtlBgZ0R6uERczT8xE0HtBCrdVU1SDmD9iUBBTQENkhDzYHDZ8pvooXKORTpp3x0IQBMEgDkc5hejvfHsH9zbfw6rnq1pNu2QbaIsPCj8A15gLoGFmkLa+NviVfAiqBTCyN1JKbASh6ZTVb4mOgaPT/t/H2F/GQt9CH9r62u1u+6LuiVOQE9aVrCO1MuSk6n6bdjUNj/c9xpCvhtDedmlyKa6uuwquMRfW3dXnRlARlJiCsZMxtHS1oG3Qfr9QBYuuFlibvVaqPi0Pr0le8JrkpZS2lY3P48MxyBHGTtLXDymIKUByaDL6ruoLrhFX6s+pou+K6kXQ1teGlq4W2By2Uq+BXUKkr2nY0fCyePhn8j+ozKnEh2UfgqPNQerVVNz+6jbsetohcGEgAhcGttlG/zX94TneU23vUwa8P4DpEGixKGKRQp9/ud8mhybj2bFnmLRvEtgcNgxtDDF6y2hFQ2RMbmQuhLVCuA5xZToUtWPsaIxJ+ydJNfgzbMMwtVrdTzT026q8KuTcz2kyOYPofNyGueHD0g/bzV5DKI+RvRFm/juT6TCaYFGdccqaGqisrISJiYnKijcSBKHenvz1BImnEjHj6AxwtNt+WCIWinF43GFwuBxMPzIdW523gqPDUeqDLoIgiI7mxuc3YOxojN7LpK+B0Z7ch7m4tPoSZp2aBUObhtWD/Eo+2NpsaOupx8AI0+p59ciOyIZFVwuYdzFnOpxGonoRaoprYOxoTPsKp/qKehQnFMPSy1Ljb8brefUI2xCG4PXB0DPTg0QsAZvDBkVRyIvKg0MfB6ZDJGhy98e7uPbhNSy4tQCug12ZDqdFEpFEZat4Xkzo6MgybmfANsAWuia6ja/lP87H4/2PMe6XcQCA4oRilCWXaewAONG+G5/fQNiGMCx7tAy2AbZMh6Ow33r8BlGdCKuSVzEdCkEoBUVRAAWZUlwfnXYUuia6mPzHZCVGRrQl7Xoaks4mof+a/jBza7/eMqH5VD0WRNa5M4yMuRKE5qAoCpWVlUrpt/7z/DHr5Kx2B/wAgKPNgYG1AfQt9aGlqwXP8Z4Y/9t4MuBHNBLUCNQm3RrTlNlvCc0l4osQ+UskEk4kSP2ZytxK3NtyDzXFNU1el4glEAsb0jPnR+ejIKYAJYklABpSRm402YiUSyn0Ba/hMm5m4PC4w3h27Fmr2zDRb7V0tWDiZKKUh/q6prpwCnLS+AE/AIg5EIOIHyOQeDoRQEMqGwB4uOMh9vbdi7ijcYzFVplbifL0ckjEylmJwMvi4fjM44g9HKuU9tVN99nd8VbEW3DoK/1Arqr7rioG/FIup2Cj6UbEHWHub1sV6srrcHrBafza9VcIa4UQ1gkBAHaBdo0DfgBg5WPVZMDv5pc3cWbRmRbbpCgKVflVan0NlnIpBcemH0NxQjHTocgt50EOonZFobakVq7Pv9pv+63qhw8KPoBtgC0oCYXdvXfjzoY7dIasUoO/HIzh3w9nOowOgaIoUt9UTbzcb1kslsw1bavzq1FTVNP+hoTUonZF4diMY1Kf87LvZiNyeyRt6a/LUsuQcSuDlrYI2eU+zEXoqlAUPi1sdRtVXw+RQT+GSSRkeTxBaAqJRIK0tDS16LdT/5qKKQemgKPNwbRD0+AzzYfpkAiG5TzIwf6B+5F9Lxtn3jyDfQP2obZUvpv/jkSd+i2hPrS4WlibvRYTfp8g9WfSb6TjyvtXGgc7gIYaOju8dyDlYsOgXu/lvbEqeVXjqhjr7tbwm+sHA2sDWuPXNC8/IPKa7IXJBybDZ5oPJGIJsu9lN9ueiX6b/zgfVXlVSmtfLBCjroz+eoGq1u/dfph9ZjYCFgQ0ed1rshe6zeoGj9HM1XwK/z4c2923o6ZQeQ+x4o/Ha2Rtxge/PMCtr27J9LDBxMkETkFOMq1SVkXfLU4oxp8j/kRyaLLS9vEyA2sD2Pe2h66ZbvsbazA9Mz0ELAxA4KJAiOpF2Oa2DeGbwtv9XN7DPKSEprQ42F6VW4Ut9ltw5f0rygiZFpU5lUg4lYCqXOX9/itbwskEXHj7gtzX/a/2WwNrg8brlurCavB5fNSXK14vkCm+033hO8OX6TDU0rX117DdY7tUgz8JJxPwnf53SDqXpILIiPa86LcJpxOQfiNd5s8vurcIcy7MUUJknU91QTWufXwNMQdikPsgV+rzSfD6YKzNXgsrHyta4ji/7DwOTzis1hNtOrKiuCI8/PUhKnMqW91G1c+kSE0/giAIhvGyebi46iICFwXCa6J0qXJencnFr+Lj+bnn8J7iLVXNKaLjKYorQv6jfGjra8O+jz0sfSxRU1RD6okRRCu4RlyZ6lR5TfLCrFOz0GV0l8bXytPKQUkolKWUNb5m7PBfqg7rbtaYdmgaPQFrsMhfIxH1WxReO/EarLtZNw4YnVt2Do92P8KqlFWMpvqkKAr7B+yH61BXzA2dq5R9bHXZCjN3M7x19y2ltK8qLBarxbR+Jk4mmPHPDAYi+k+XUV3A4XJgYKOcQXZjJ2N8JvhMqqwM6ubJgSeoKa6RuWYlJaGQ/zgfdoF2Mq8iUJaKjArk3MsBv4qemfHtsQu0w/xr81WyL6YN/mIwWCwWSpJKYOJs0uR81poZR2dAx1CnxVXSFEWh17JecBmkvnUR/Rf4I2BhQOOqZU3Uf3V/uA11a7UuvLySQ5Nxff11vBXxVodYqU40xzXmQsdQB1yT9q+HjRyM4DrEFVxj6a+dCeW78t4V6JroYnnMcqZD6bTSrqfh7qa7GPvLWPgv8Jf6/pKjzYGxI30pHvuu7Ivur3cHJabA0lKPa7bOxG+OHzzGeEDXVH0miZFBP4IgCIZVpFcg+UIyPMbIPzv+8f7HuLzmMl478Rp8p5OZjJ1Rz0U90e21buAac2Hrr/n1Nzqj8vRy1JXWwb63PdOhdGgivgiP9z2G9xRvGNkbSf05XRNdeE/xbvKafW97rEpeRWZUtoOiKIjqRTB1MW3yeo83esDIzkimwVdloMQUhvxvCEycTJS2D783/NTqJlAeiWcSYWRnBPs+9q2mQRXxRShNKoVNDxsVR9cwMK/MOmMsFksjB/wAYGH4QlTnV8v8uRuf30D4d+FYFrNMba4tPMd6Yn3VeqWlce2sHu58CEpCoe/KvrD0ssTiB4sBKU5tbf1+mziZYMIu6VfUM0FT+/TLjOyNZLqekVZ9RT14WTyUJJSo9cBte2IOxODGZzcw+/Rsco39ipD1IQhZHyLVto79HDH3onImRhHym/D7BKl+q19V9KwI2RHZ8J3uSwb1FeT3uh8svS1h5m7WeE6Upg5w4dNC6Jrp0nb/8ep9KqFaWlwtGNnRfy5WBBn0IwiCkIGuLv0P7FwGueCT2k9AieV/aNxtZjdwdDgafUNGKI7MvGyZMvqtMlz94CoSTydiVcoqUsxbiZLOJiF0RSiEtUIM+GCAzJ/nZfOQdjUNgW8FAmhYec1C6zd1UbuikHgqEXNC52j0SgJF9F/dH/3e7dfs5tclxAUuIS2ft1TZb9labAxcN1Cp+xj14yiltq9slITChbcvQNdEFysSVrS63V8j/kLRsyJ8UPhBh3iY/qr8R/moK6+D+3B3pkORibaetlyrgLwmeoGSUNA1kb4/qqLvstgscNiq+/sK+z4M2vra6L+6v8r2qWoPtj0AWA0rBYCGQe42Tm2NJGIJ0q+ng8VhaVy/AACxUIyc+znQNdFlZLKCoigJhZriGhhYGyhUk7alftv99e7oMrqLxmcN0THSgZG9EalFR3Q4urq6cBvuBg5H9vNhysUUXF13FdbdreEU5KSE6DoPFpsF+17/TSgI+y4McUfisOzxsjZrD/899m8YOxo3TLIhNF5lTiUENQKYdzFXSc1paahHFJ2YPD/OBEEwg8PhwNvbWyn9lqPNgZau/PMwjOyM0OftPjCw6tx1ozojiqJwYvYJPPnzSZPX/5n8D84vP89QVOpDmf2Wbr3f7o2eS3tKlU6LkJ/XJC9MOzwN/vP95fr8pdWXcHbRWRwacwgXVlyARNT2apPS5FJk3c2Sa5VNR9LWw0ixQIzqwv/+fTSp33Ymrx1/DaM2tz14Gbg4EAM/GgixQKyiqBpIxBLs7b8XYd+HKXU/oStDcWreKaXug27ZEdkoeFIg12cd+ztixPcjYOpqKtX2qui7j/Y+Ql5UntLab8njfY8R80eMSvepagvDFmLmiZkyf47FYuH4a8dx68tbzd47t+wcLq29REN0yiOqF+HAoAO4t/ke06HIpTK3EpttN+PKB/LXTWyt37JYLI0f8AMaavotiVwCh74OTIeidm7/7zYSTiVIvf2tr2/h3hbN7CsdDYfDQVfPrmCz5Xus7zPNB3MvzaWtnlxnVZlbiex72U3uB8VCMSiKarO2GwAMWDcAvZb1oi2WiswK7PDdgTsb7tDWpqbKvJOJ0wtOt3ufTqc7397BDu8dqOe1XgNX1fe2ZNCPYaou4kgQhPwkEglKS0tp77epV1OR+zCXlrbqK+pRnFBMS1uEZqjKq0LSmSRk38tu8jovi9fpBxkA5fVbuhTHF+PJn08gqhfBfYQ7Jvw2ARwdMtChTFpcLfi97gcDa/kmSQz5agjmXJiD+vJ6FD4pbHcm37Bvh2F91XpaazZokntb7uHax9cgqhe1+D6/ko8tjltwff31xtdU3W+fX3iOvf33Iis8S2n7iD0Si2MzjqGurE5p+1AmFpsF54HO8Bzn2eZ2AQsCEPxRMHQMdFQUWYP6inpU5VahprBGqfsZ+NFAjN4yWqn7oBNFUbi46iL2D9gPfqX8NfAoipJqlYyy+25deR3OLTmHB9sfKKX91sy7Mg/zr3fsun4G1gaw7m4t8+dYbBbG/jq2xTSeWWFZyI/OpyM8peEacTFm+xgEvBXAdChy67mkJ5yDneX+vLpfKxPKIaoX4daXtxB/LF7qzzz982mziaYEMyQSCe7vvY/vDb9HyqUUmT9v5m4Gj9HqVX9ME8UejsX+AfuRfiO98bWQ9SF4O/btdidM9V/TvzFzDB30LfVBiakOmWlDFpSEwsV3L4KSUCqtR+05zhPBnwS3mX1L1edZkt6TYaQGDEFoDoqikJ2dDVNTU1rbPb/0PLjGXCx/oljxZRFfhF+6/gJTV1MsfrBYoRQvhOYwdjDGRxUfQVgjbPL6ssfLGIpIvSir39LlwfYHiP49GpY+lnDo0zADuaa4BmnX0uD3uh/D0XU8KZdTYN3dWqHVlDY9bGDTwwYeYz1QX9H6TL4XtPW05d5XR5BwMgG8TB6Gfz+8xfe5xly4DXODjf9/adVU3W/5PD4qs9uejauokoQSJJxMwNBvhmpk7ZTi+GJY+lhKfW0h4ovA0eGo7FpE30Ifa7PXKn0/3pM1q14Ki8XChN0TUBRXJHcK8OTQZJxbcg6TD0xGl5Fd2ty2rb77/PxzmDibKJRCUVtfG/OuzlP5Q0p5UqNqksqcSoiFYpi6mMr1gMx/Xssr51fEr2h1woc66beqH9MhyM3EyQQTd09UqA11v1ZWlLBOiMhfI2HR1ULjfsOVia3NxoqEFTKloVtwcwF0zcggkTqgKApVwio4DXSCkYN8dcQoioJEJOn0g0SK8BjjAUG1AM4h/028YGoCr46BDlYmrWRk3+pEIpYg4M0AiAVilQ76SVNbXNVjQGTQjyAIgmGjt46Wq/jyq7S4Wgh6P6iheCwFqepwtEcsFLd4EZj7MBdmbmbQt9T8lC8dgRZXC1pcckrXRMO/Hw6nAU6NA34AcG7JOSSdTYJdTztYelkyGF3HIqwT4t/X/4WxozHefvq2wu2xWCzomUk3eJN+Ix0SkQRdRrX9wLwjWnhnISpzK9sc/JnxzwwVRtSc3xw/+M1R7iB7yKchGPzlYI2s61iVV4Wd3Xai1/JemPBb89U8r4o5GIOLqy5i+uHp6DqhqwoiJNpi38u+Sa0ZWRnaGULXVBeiOsUGb84tPQcLTwu8eftNudvQ4mrBfYTq68bxq/iozKmEmbtZh7zeevDLA0T8EIF34t+RO9WbRCRBXlQeHPs7NnldkfIFBEEHNoeNax9eQ7dZ3cig30vYHDYsvWW7zzBxNlFSNIQ8bIJtMOLtEXKlDKyvqMdm+83o8UYPhScOdGY2fjaw8Ws+mammqAbX1l+DYz9H9FraPIVn+s10XPvoGoZtGNbuhCpCNhxtDvqv6Q9+JR+P9z+G+0h3mDh1zt8uzbvrJAiC6GC8J3vDewo9NyDBHwXDf74/LTNaRPUi7PBpnhOcoiikXk7Fdo/tyLqrvFRoRPsoikL07miUJJY0e684vhjRe6I1NpVcZ6FnptesttyITSMw69QsMqhOM44OB1MOTsGwDcNUvu9T80/h+ifX29+wA2KxWZ32RutlWlwtjRzwAxpqLg5YNwC+M3yl2t6hjwMMbQybrN5UtoKYAsQejkVduXLPefH/xuNnp5+bpHFSV6XPS5EZlqnwrGK7QDu88+yddmcvt8fUxRSWPopNZKmvqGckU879n+9jp+/OFq+3OgK3YW7ot7ofLDwt5G7jwooL2Be0D+Xp5QCAvKg8JJ5OhKBGQFeYSnPj8xv4yfYnCKrVP9ZX3fziJs4sPEMySLWBo8PB4geL261J29nwq/ioKa6RKnXzC/W8ehTEFEBYK2x/Y0KtcU24cAlxgW2ALdOhaCxKQrX626tjpIOEfxOQeiW1xff5PD6q86tl6n/SyLiVgRuf3ei0fbQksaSxpl7mnUycXXQWTw6qJiXx6QWncXrBaZXsS1qaeedJEATBECMj+VIntIbuk/wLVflVqCmWva7N/W33cf3ThgfTlbmV4Bpxm9XlYbFY8J7iDbtAO9gF2gFoSOOlyiK5RIPytHKcX3a+xdo2yReTcX7p+Q77gEoWdPdbOlASCo/3Pwa/qnmNJUsvS3hP9oa+BRn0oxObw4bXRC94TVTswbU8xv4ytlM+bEq9ktrqze7LBDUCnJh9Aje/vNn4mir7bcKpBMT/K31NG3kIqgXIvJPZ+EBck5i6mmLkDyPhPly6FVZWvlZYkbCicbBX2QNxAPDs+DOcnHsS1QXKrWXLNebC0M5Qqfugy90f7uLAoAMojldtreeW+m5NcQ0W3VvUYt03WRwefxi/ePyiUBvycB3iipBPQ6Re3a1pPEZ7YMzWMTKl+XuV3+t+GPzlYGjrN6S0frTvEY5OPQo+T/5akqrCNeLCzM2sxWsydZcVloX0G+kKp1JWx2tlOjn0dVAotXtHFPdPHH6y/gmpV9u/Tnsh+vdo/B74OwpjC5UYGSGtlF0peLjjoVyfZbFYeOPyG+jzTh+ao+o8nh1/hh+tfmyxD2nraWNN5hrMPDGzxc96T/HG2uy18BjtQWtMadfSELYhTCPvNxRFURSOzzyOHT47IOKL0GV0F0zcOxF9V/ZVyf7LUstQnqZe/+4sikwJYkRlZSVMTEzA4/FgbEwuPgiiswrfFI77P9/H/GvzYd3dmpY2C54UYE+fPQh6LwgjNo6Q6bNHJh1Bzv0cvBP3DgysDRpmL0kosLXYqMypxLHpx7AwbGGzPOV3vr2D2MOxmH9tPozsO/ZNozoR1gqRfjMdxg7GzWbplaeVo/R5KRz6OXTYh1TqhKIoVOVVSf1AIfliMg6PO4yQz0Iw7JuWV54Ja4WND88IxdUU1cDA2oDpMDqV3b13o7qgGu/lvNfmdhRFYavzVjj0c2j15liZdgXsgqBKgHdT31XaPgpiCvB74O8Y+s1QDPpskNL2o24ywzJxdMpRjNs5Dt1ndVfafkqTS1H4pBCe4z07fR3NF7IjslEUV9RiWilZVWRUIHxjOLpO7Iqu42VL2crL4mGb2zYM+mIQhnw5RKE4rq2/BlGdCGO2jlGoHUL5ip4VIS8qryEDCakzrlTCOiH53WuHiC9CbUktGfh7ScbtDMQdicPADwdKXbc050EOUq+kwn++P0xdTJUbINEmiqLwvdH3cBrghHlX5jEdTqeUeCYRdzfdxcQ9E2HdjZ5neYriZfPAr+TDwtOCsdqCqlT4tBAVGRWN2Sge/vYQlJhS2UCfrFQ9FkQSrDNMIiErYwhCU0gkEhQVFcHa2hpsNj0LpfUt9WHqakrrrHEbPxv0XNwTdj3tpNo+enc0us/uDq4xF+N/Gw99C/3G+hssNqsxVeiDXx4gNzIXqVdTmz3wEQvEsOlhg/qKejLop0La+tqtPnwzczeT+gauI1NGv21J4dNCHBxyEBN2T0C317q1u71jP0dMPjC51ZUzB4cdRHlqOdZkrqE50s6purAam203o//a/hi9ZTQjMVASCsJaIXQMddrfuIMYtXkU6svr292OxWJh5fOVjQ8tVdVvX5jw+wSlp3UzdTXFmG1j4DTASan7UYZj04/B0M4Q434dJ/NnLb0sYWhnqPQH0haeFgqlJuyInAY40fb3xtZiI/r36DavO4CW+66wTgif6T4oTSzF00NP0eONHnLHMeJ72SazEe3LeZCD80vPY+g3QxVO4Qo0THC4+O5FDP1mKAIWBCgeINEuRX9fVX3OZcLJuSeR8G8CPhd9rrGptunmOtgVroNdZfqMYz9HOPZzbH9DQukoisIbMW/ARFf+FPpp19Pw5OATDP1mKBnElYP3ZO8264RSFIXE04kQ1Yma1Q6PORADA2sDeI7zpDWmzlZS4c43dxD/bzw+F3wOthYbfd5uvnK1uqAadWV1sPKVr2YxnVQ9BqQxZ7vffvsNPXr0gLGxMYyNjREUFISLFy82vl9fX48VK1bAwsIChoaGmD59OgoLmy45z8rKwvjx46Gvrw9ra2usW7cOIlHTguS3bt1Cz549weVy4eHhgQMHDjSLZceOHXB1dYWuri769euHyMhIuY+LLLQkVI1fyQe/ik/+9uRAURQKCgpo/bfruagnFt9fTGsaPxabhfE7x6PbzPYHHtKupeH8svO49vE1AICxg3HjgN+rhn49FCsSVrT4sGfo/4Zixj8z1OJE2pnwsnhtvk9RFMQCsYqiUU/K6LctMXY0hogvQlaYdHUu9cz1ELAgAMaOLc/wcg5xhtsIN6WlAO5sxHwxAhcFwmkgMwMu1QXV2GS2qdPV9XMd7Cp1zdqXH1qqqt++4NjPUerUlfLSNdVFv3f7wb63vVL3QzeKolCWUoaKjAq5Pm9gbYC3n74t02ACRVHIuZ8j035EfFH7G9Hk/tb7iDkYo7L9yUMiltDaf4wdjbE6Y3W7kyZa6ruWXpZ47dhryH+cj7ub7tIWkypVF1bjr1F/IfJX+e/71VVdWR3qefUKpfZ8WXF8MfKi8jRqdV9tSS3CN4VrRK3Ol9UU1SDpbJLCaY1Vfc5lguc4T/Rd1RcSIZl0T3QMFEWhrLpMocnjFekVePrXU1IORElYLBaurruKG5/daPI6RVG48v4V3Pn2jlL2W8+rR1VelVLaVje93+mNaX9Pa/WZCUVR+NX7V5xbek6pcQjrhHjy15N2Ux+r+jyrMYN+jo6O2LhxI6KjoxEVFYVhw4Zh8uTJePbsGQBg7dq1OHfuHI4fP47bt28jLy8P06ZNa/y8WCzG+PHjIRAIEBERgYMHD+LAgQP44osvGrdJT0/H+PHjMXToUMTExGDNmjVYvHgxLl++3LjN0aNH8d577+HLL7/Eo0eP4O/vj9GjR6OoqEh1/xgEIYOylDLc+vpW44/Lg+0PsNF4I6rzlVvzhNAMbsPdMGH3BAz5aki722rpasHS27Ld7UhtP9WoyKjAVpetuPrh1RbfF9WL8L3h9zi76KyKI+uc9C30sSZzDcZuH9vutlX5VagtqW1zm6FfD8XkfZMbV9oSijFxNsGkvZPgO92Xkf0b2BjAOcQZFl07z0okWQcdxEIxYg/HIuFkghKjao6SUJCIyXmrNSwWC8ufLMfrZ1+Xv43//x3jZfOwP3g/CmIK2tz+/s/3sS9oH54db7jPK04ohrBO2Or2EpEE3xt9j9NvnpY7Rlnc3XQX0b9Hq2Rf8ko6k4RNZpvw/MJz2tpUdBXAtEPTMOXPKXJ/PvVKKk6+cVLlNQoBQIurhZx7OeBltz3ZShN5jvXEmow1tK028Jvjh3XF6xibZCMPYa0Q1z++jqRzSUyHIpPsiGz8M/kfpFxOYToUtRf4ViDGbh/b6uTWzuji6osIXRkq02dE9SIcHHqw2SAGoXpV+VXgJfHavD5qT/fZ3fEx72Pa68p1BvwqPs4uPouUS23//k7aNwlvXHqj2T3R/OvzMeon+uu9S8QS/Gj5Iy68faHVbR7te4TdvXejMreS9v2riqheBIqi4DbUDX6v+7WaypTFYmHQZ4PQc3FPpcZTlVuF0/NPI/bvWKXuR1YaM+g3ceJEjBs3Dp6enujatSs2bNgAQ0ND3L9/HzweD/v27cOWLVswbNgw9OrVC3/88QciIiJw//59AMCVK1cQHx+PQ4cOISAgAGPHjsU333yDHTt2QCBoSOeza9cuuLm5YfPmzfDx8cHKlSsxY8YM/Pzzz41xbNmyBUuWLMHChQvh6+uLXbt2QV9fH/v372fk34Ug2nPl/Su4/dVt5D/KBwCYe5jDb64fKAmFel49rn54FVnh0q1MIejFr+Tj/NvnlXKjVldWh/3B+xtX8LWGxWKh15JetNS5qiurw95+e3H5/cvtb0wojKIo9FrWC65DXFt8X0tXC11GdYF1D/XIL9+RhW8Kx62vboFrzAXQ8N20Nfgd8WMEfrL5CaXPS1UVIsEwFouFOefnqG19AWVI+DcB3xt9L/WgA5vDxsVVF3H3B9WuBCp8Wohvdb5FxE8RSt/X4QmH8ffYv5W+H2WgYwJC9t1s5EbmtvuQodvMbujxRg94jPGAiC/CsenHcO2j1q9nBNUC+EzzgW2gbavb0GnuxbmY8c8MlexLXlp6WrANsKW1fpVYIEbyxWRk3smU+jPRu6Oxb8A+lCSVwL63PewCpUs935LC2ELE/h2r9FS8LdE11cX6qvUYuWmkyvetiXQMdDQqhaKRvREW3V+EkE9CmA5FJna97DD5j8lwGeTCdCiEBsq6kyV1hpIXOFwOSp+Xoqa4RklRdV4SsQSCGunPb/FH43Fr9i3kPcyTe586hjqN96+EbIrji/F43+N2J7K5DnaFRVeLJqvfWSwWbANslZLyn81ho9+afvAc3/pEnnOLzyE/Oh91ZXW0719Vbn55E9tct6Eyp/2BywEfDEDAmwFKjcfAxgCzz8yG31y/9jdWIY2c5iIWi3H8+HHU1NQgKCgI0dHREAqFGDHivxz/3t7ecHZ2xr1799C/f3/cu3cPfn5+sLGxadxm9OjRePvtt/Hs2TMEBgbi3r17Tdp4sc2aNWsAAAKBANHR0Vi/fn3j+2w2GyNGjMC9e/fajJnP54PP5zf+d2Vlwx+mRCKBWNyQeo3FYoHNZkMiaToz+sXrL7Zr73U2mw0Wi9Xi6wAgqBOAo8Np/NF58fqruWU5HE7Dg8sWXn81xtZeV9UxvRojOab/Yp9yaAoKnxTCJsAGYrEY3Wd3h+9M34Z0AMlluL/1PmqKauAc7Kwxx8TU9ySRSGBqagqJRELLMRUlFCF6VzSMHIzgNsKN1mPSNtZGTWENRHxRi8eUcjEFccfiMPLHkdAz12szdmmPSdtYGxKxBGyt5jHScUxtvd7R//Zait3Y2RhjdzSsKhOLxS0e04wTMzTqmJTxPUkkEpibmzekOn1pv3QdE4vFwrN/nkFQK0DwZ8Hg5fJwdPJRGDsaY9apWQ3tS6jGB+ZsNhvOIc6o59XDxN2ksb1Xj6mmuAa3vrgFlxCXhokaHfx7UuYxScQSHBp5CD7TfNBnZdM8/5p6TC9iVOfvSdtQG05BTjB2NJb6mGYcnwHzLuZNzrdKPyYO4D3FG2YeZkr5jXj5e2Jrs0GxqVZ/s2k7plZel+eYsu9moyytDN5TvBvrUcr7t+fzmg/s+9vDxKnht6+1YzJ2NMbkg5Mb3hNLYOxoDFM308a2X41d11QX0w5Pa/I7r8z+ZOVnBTab3er3oQ6/Ee6j3OExxoPW3whBnQD/TP4H7iPd4TjQUapzbm1pLUqTSqFnqQdKQkEkEIGt/d9gkCzH1G9NP/Re3htaXC21/d2T9ZhexMjUb7lYLEb4d+Fw6O8At2FuHeKY5PqeWIBdb7vGGDTlmIwcjOA3r+EBo1gslvt7Uuk5t51jUtbfXnZ4Nh5sf4AB6wbAvo99k+019ZgU/Z4WRy2GWChu3FbaY3ov9z1IJJIm8ajLMbUVe1vHxPT3lHw+GTc/vwm73naYtGcSWGxWu8dk398e/qv9YeVjpdAxlSSWoK6kDq6DXTvd95R8IRkug12gY6gj8zE59nPE+wXvg2I1fcbQUuzCWiF46TxYdbcCRVGozK2EvoU+tPW0lfK3N3zjcAD/pZJ8NXafGT6oK6uDpa9ls9jV8Xt69XWgoVSKkb0R9G3027yvUtUxaelrwWO8R2MMbR2TKmnUoF9sbCyCgoJQX18PQ0NDnDp1Cr6+voiJiYGOjg5MTU2bbG9jY4OCgoZR94KCgiYDfi/ef/FeW9tUVlairq4O5eXlEIvFLW6TmJjYZuzff/89vv7662avJyQkwNCwIQezubk5nJ2dkZOTg7KyssZtbG1tYWtri4yMDFRV/ZeX18nJCRYWFkhOTkZ9fX3j6+7u7jA2NkZ8fHyTPzIvLy/k3c3DkUlH4DHfA97LG2qs+Pn5QSAQICnpv3QWHA4Hfn5+qKqqQlpaWuPrurq68Pb2Rnl5ObKzsxtfNzIyQpcuXVBUVNT476mqY9LR0UFsbNMltJ39mIpuFsHOxg72o+0bjskQKI8tb/GYgvcHw21gw4CTOh+TOn1PFRUVtBwT15iL9/LeA6+K16Qduo7p9Xuvw9LSEomJic2OKeFkAuKPx8P2dVvo2ejR9j2F/BUCb29v3Nx0E3lxeej+QXfGvie6jkmd/vbIMcl/TAUFBUo7piVRSxB5NRJxcXGQiCSgDCgY2BlAIpHg8pbLSPw9EcF7g6FjqgM/Pz+4jnEF342PuLi4Vo9JWCPE4z2PQYkpOI5z7DTfkzKOqa6gDsXxxXDo68DoMdUV1CF+ezzcRrlh2JphHf97sgdG/jUStra2SE1Nle6YejccU2xsLPLu5KG0Vyk4uhylH9PMf2eioKCgSfvK6E/eXzZce8fGxqrP99TOMcX8EYOYP2IgvCmEjqlOs2OS62+vAhBXi+Fi6QIde53GY0r5MwWGNoYYvW50k2Pqvqk7jI0bVqypw++eRCiBm7UbtM20kZL2X7YGdfrdo+V7evWY0tMQ+L9AGHsaIzk5WapzrtFYI4wcMxKlNaU4Pu04Cp8VYvS1/+oCynVMWg2/Ear+3Ys5E4OClALYj7RX7+9JhmPKjM9smGA0zQXVVtUd4pjk7U9igRiCcgG69u6qMceUnZ2N8vJyhb+nxMREiMViVFRUMH5MyvrbQy6QeCoRRv2NUKr7X6YNTT6mjvg9ddZjqhRUora6FtZTrMHmsFFZWdn+MRkAfqv8YGRrpNB97u25tyEsF+KDvA861fcUeTkSVyZfgZGHEYYcGQKfbj4yH5NET9JwTHltH1P4gnDU5dfhjZg3UFhUiLvL7oKXyMPcJ3Ph4uqi8r+96f9Mh0AgaPK6un5PrR2T4WhD9BrdC3HP4tr9nsoKynBo1CGYdTNDj497KOWYKisrGwf02jqmF4OTqsKiVF1FUAECgQBZWVng8Xg4ceIE9u7di9u3byMmJgYLFy5sspIOAPr27YuhQ4di06ZNWLp0KTIzM5vU56utrYWBgQFCQ0MxduxYdO3aFQsXLmyyki80NBTjx49HbW0tysvL4eDggIiICAQFBTVu8+GHH+L27dt48OBBq7G3tNLPyckJJSUljYOVqhgRr8qvws8OP2N53PLG2lxMj4grekwtxd6Zj0lUL8KuHrsg5ovxTsI70NbX1vhjait2VR6TRCJBbm4uHBwcoKWlpdHHBAAlSSUw9zRvN3Z5jumvUX+hOKEYy2OXg2vEBSWhoKWtRf72aDomSkLhwJAD6DqxK4LeD2r1mBJOJiD1YirGbBsDLf2m83zU7Zhejb2t12Vd6Zefnw97e/smM6tUdUyPDzzGjfU3MOv0rIaZxRI0roZt75hqimtgZGMEiqI6/Pek7GNisViQCCRgaTedXafKY6qvqMdPVj+h37v9MGbrGPI9tXFMmfcycXDQQUz4fQL8F/h3iGPS1O+pIr0CBU8K4DXZi7ZjEtYKsd1tOyy6WmBh+EJQFAURX4Rf3H6BkaMRlkYtbfF3T1QjQva97CbZEVgsFh7teYSiuCIM+25Y43WvMr+n21/fRtg3YVj2ZBmsulk12V4d/vYkYgkuLLuAruO7wneGr9qcc8M2hIGXxcP4XeNlPiYAyH2QCy1dLdj3tGekPx0cfhD50flYV7quSYya/BshEohQFFcELV0tWHhZdIhjkrc//TPxH6RfT8f62vWNqyDV/Zj29N0DHSMdvHHljRaPqbVjffWYhEJh4z0um81W6+9J2mN69XVQDZ+hQDXbXlOPSZHvqb6iHllhWbDxt4G5m7lMx1TwqAC5kbnwf8sfHG2O2hxTe7G3dUzq8D1RFAUWiyX1Mb14NuXs7Nz4eXmOKf54PET1IgQsCGgxxo72PT375xmKYosQsDgAWWFZ8J7qDa4RV+Zjyr2XC2MnYxg5GrUbe8wfMeDz+Oj9Tm9wdDi4//N91BbXYvj3w5Xyt5d6KRX3Nt/DqM2jYBtg2yx2Skwh/lg89Cz14D7SvUns6vI90f23t7vXbjgNdMLoraOVckz3f76PsG/D8MaVN2Df277VY+LxeDAzMwOPx2uc0KhMGrXST0dHBx4eDcsle/XqhYcPH2Lbtm2YNWsWBAIBKioqmqz2KywshK1tQ20HW1tbREZGNmmvsLCw8b0X//vitZe3MTY2hp6eHjgcDjgcTovbvGijNVwuF1xu81zJbDYbHA6n2WsteXU7eV43tjfGF5IvWlxS2tL2L046LcXdEllfp+OYZH29ox8Tx4CDOefmgGvMha6RbottvBp7VngWEk8nYuBHA2Fg1by2G9PH9DKmv6eKigo4OTk19iFFjqkwthBcIy5MXU1lil3aYxLWChHxSwSMHY3RY24PAP9dTAKAlbdVszZai72111v7nib/MRl6Znrg6HBw7aNrqCmqwdQ/p2pcf5ImRiaOqaa4BuWp5ajOr272/sv/XfCoADF/xCD442BYdLVQOHZN/Z7Kysrg4OAg0/cqbSz3t96HQ1+HVnPyB74ZCN+pvtA1bfg9/lrra3hN8sLsM7ObbfvqMRnbGje+3pKO9j3J87ossbB1W59Zp4pjMrAwwEflH0HXRLfN7TvK93RpzSVY+Vqh19JeMsdobG8Ml2kusOhq0WQbZRxTzMEY5D/Kx/ANwxvTV7a1vTyxvHg9NzIXGbcyELgoEPoW+jK3w0R/Mu9iDvMu5i1sLf/fHseIg6D3g2DqYtp4w8zR52BF4gpU5VWBxWK1+Lt3YMwB1BTV4N2Ud5u8nnw+GWnX0jB229hmtQeV0W9cB7mC/y4fXCOu2nxPL7/OS+fhyYEn0LfUh+8MX9p/I0R8EerK6mBkZ9TmObcgugBp19PgP88fxo7GGPz54Bb3J20soW+HQlAjwLsp7zLyuzf488EQVAtUeu8u6+uyHpOWTsMgqrTba8IxydufvKd6w9LHEpSIAjiacUzmXcyhbaDd5r2ANK+z2ezGe1xln3PlfZ2u76M1mnxM8nxPpYmlODb1GEZtHoWg94Jkiv3poad4sO0Buk7sChMnE5XF3trrmvw9Pf37KdKupGHYhmEwcjDCjc9uQIurhcFfDG73mPb33w89Lz28fvD1Nvt2e7H4zfaj9Zhae11dvqfEk4l4fuE5Bn85GD0X9pSrHYqicHjCYdj42WBh2MJ2Y+y1uFeT/x74wcA2t5cllpZeF9YIkReVh+r8arACm/+7P9z9EKHvhMJrshc8xzSt/acu31Nbr59/+zy09bUx6qdRze4XWvvbW/ZomdQxynNMxg7GsO9lD0Nrw8btWopF1evuNGrQ71USiQR8Ph+9evWCtrY2rl+/junTpwMAkpKSkJWV1bgiLygoCBs2bEBRURGsra0BAFevXoWxsTF8fX0btwkNDW2yj6tXrza2oaOjg169euH69euYMmVKYwzXr1/HypUrVXHIcruw4gKsu1ujz9t9GmZz1QqRcSsDnuNaL+5JaJbKnErUldfBxs8GVr4tD+a0Ji8qD/c230PXCV1hMKT5oB+hHGcWnkF1QTXey3lPKe1zdDgI+zYM9n3sGwf9wr4LQ8GjAkz4fQL0LfWVsl8AMHZoGKygKAqlz0tRW1wLQbWgxQeqhOwMbQ3xft77jXUYWjPww4EI/jiYFOhWkurCalxeexmBiwNbHfRjsViNA34A0H12d3QZ3UXq9vMf5cN5oDP5DhUQtSsKVt2s4BLiwnQojQN+HZ1YIEbkL5HwmeaDXkt7tf+BVxg7GsN/vT8c/RyVEF1TKaEpeHbsGUb9OErp+0q7noYbn9yA2zC3xkE/dSYWiCGqFynl9ydkfUjj/0+7nga7nnbQM9ODnpleq5/xn+8PStJQ54/N+e/Ge/aZ2agurG424Kcs7iPc4T7Cvf0NGWLuYY4Pyz6ERCRpf2MZSUQS/GT9E5wGOmFu6Nw2t008k4jw78LhOdYTxo6Kz2YO/iQYYn7b1z3K5DrElbF9Kwsviwd9S/1mmWE6o15LZD9XMW36kelMh6AxxAIxsu9lQ99CH9bdrZkOh3HmXcwx+Y/JTeobSqvn4p7wHO+pEdcx6i7nfg6eHnqKMdvGgMVi4fnZ5xALxRj8ReuTZICGyTfCOiF06smzFVm9dvw1lKWUQVuv4bzHr+Tj0d5HcOjrAOdgZ6naoMQURv00ClwT2a6Pa0trAQpKfQ4HAD7TfeD7mm+rE4evvHcFADDs22FKjUMZKAmFrDtZMLAxaPX4mOD3uh/8Xvdrf0MV05j0nuvXr8fYsWPh7OyMqqoqHD58GJs2bcLly5cxcuRIvP322wgNDcWBAwdgbGyMVatWAQAiIiIANBQ2DggIgL29PX744QcUFBRg3rx5WLx4Mb777jsAQHp6Orp3744VK1bgrbfewo0bN/Duu+/iwoULGD26YQno0aNHsWDBAvz+++/o27cvtm7dimPHjiExMbFZrb+2VFZWwsTEBGVlZTAzM6P5X6upel49trltg9swN8w8MRMAcOatM4j5IwYrElfA0stSqfsnVOPa+mu4u/EuFt1bBMf+sj0kqymuAZ/Hh6mrKdhara+E6OzEYjFiY2Ph5+cn8+zBljz+4zGENUL0XdmXhuhaVhBTAIuuFo0382cWnkHGrQysSl6lsu+6nlcPbX3txvQfBKFKdPfbJm0LxMiLyoOOkQ5s/KS/BpDWvZ/v4cp7V7Dg5oIO+bBRFSoyKrDNbRu6v94d0w8z/3CsrrwOqZdTYe1nDetuHfuhE7+SD2GtEIa2hjJ/Vpn9ttm+hGLUFNU0TlZRJl4WD1X5VbDubg0dA/V/UJN+Ix1/Dv8TE36fINfgrTT4lXxsMtuE/u/1x7BvhkFLV6PnpHYKl9+7DGMnYwStDWr23st9V8KXIC86D05BTmBrsfFo3yPk3M/BxN8nqmyAVhlezpqh6fb23wteJg/v57/PdCgEg1R5zmUKv5KPjSYbEbAwAJP3T2Y6HIJoVFtS2zgIVJVXBQNrA6me09DVb8tSy/D3mL8RsDAAIZ+EtP+BDqY8vRzbu2yH3+t+mPb3NKXtpySxBLt77YawVgjP8Z6Y+tfUNie6KVPmnUxwjbmwDWg7Y6G6oigK9eX10DOX/t+vrqwOD3c+hLWfNbwneysxuraVl5fD3NycpPd8VVFREebPn4/8/HyYmJigR48ejQN+APDzzz+DzWZj+vTp4PP5GD16NHbu3Nn4eQ6Hg/Pnz+Ptt99GUFAQDAwMsGDBAvzvf/9r3MbNzQ0XLlzA2rVrsW3bNjg6OmLv3r2NA34AMGvWLBQXF+OLL75AQUEBAgICcOnSJZkG/F6mihsGXRNdvJfzHurK6xpf6728N2wDbeV6EEOoJ5+pPqDEFBz6Osj8WQMrgxbTehJNsVgs2Nra0tZvAxcG0tJOW16cyCUiCfhVfEz+YzL4VXyVDu52lpUtqhR7OBa6ZrrwHNv2am2xQIySxBLoGOnAzE25E0zUFd399mUcHU6rK/zo4DHaA9q/aTepvUnIxtTVFIsjF6vNKoay5DL8+/q/GLZhWIcf9OMac+VeIcZisVD3uA67F+zGlANTYBdoR3N0/+Foc1Qy4AcAJs4mMHE2aX9DNcE14cJvrh9setA/qeEFjg4Hk/ZPQlFskdQpbyQiSeN1TF15HQqfFMKmh41MN/+KEFQLcGbhGTgGOSLoveYDX0zLfZgLXRPdFtN602H0ltGtvvfyOVdbX7vJCuuMmxmI/TsW434dBy2uxjyGaPT00FNcePsCZp2apdYrPWXhO8MXghoB02GohaK4Itz47AYC3wqE1ySv9j/AsJwHOUg8nYiei3rC3EOx60RlXiurCx0jHYz9ZaxSz2edCSWhwK/kN8lmQsjn5VVfRvZGbWzZFF391sDaAGxtNrT0NO+8LKuM2xngaHPg2N+xcfKRmZsZ5l2dB+eBra/yE/FFuPPNHQz8cCDEAjGK4opknpBr4WUBr8leSL2SitzIXKU/I0u/kQ5+JR/eU5oPcLkMarg2E/FF4GhzNG4iFovFkvman8Vh4ebnN9F9dnelDPrd+/keJCIJBq4b2OZ2qj7PasxKv47mxUo/VY3uEkR7aoprUFtSCysf2VKDEuqPoiiceuMUylLLsODmgsZUBqqUeiUVT/58gnG/jiM3BzTY4rAFhraGWBq9tM3teFk8bHXZin5r+mHMz2NUFF3nUV1QDX1LfbJCmpAav4qPlEspsAu0U/ghnTorTyuHoFoASx9LuVd5J51NQuiKUEzcOxEeoz1ojvA/OfdzYOJsItODFkWI6kWQiCUasdJPHV1YcQFJZ5KwNmstWGwWkkOTcXj8YaWuRnyVRCzBBr0N6DG3Byb/oX4rRnYF7EJdWR3WZq1lLIbaklpUF1bDyseq8WESv4oPSkyBa8KV+aFHSVIJDo8/jIEfDlTZ9/yqtOtpuLvpLoZ8NUSpE34IZuQ/zseePnsw8oeRajmY/6q7P97FtQ+v4a27b5G/R0Jml9ZcQtLZJCx7vEzmwQexUIyfbH6C21A3zPx3ppIi7PiyI7JRlV8Fz7GejZMTxUIx8h7mQUtXC3Y9W5/wlvMgBwWPC+A7w1fpqSI7kv0D96Morggfln7Y7v27oEaA2MOx6Lm4J578+QRn3jyDYRuGoTi+GLGHY7E0ammb31FbXk1Trwy/9fgNojoRViWvavI6RVGgJBTCNoTh1pe3sPL5Slh4KmeSGN3EQjGuf3Id3Wd1h31v2VMT5z/Kh5WvlVKyiuzsthOUhMKKhBVtbldSXgIrcyuy0q+zEIuVW5cgKzwLWeFZ6LmkZ4s5twXVAlx4+wKG/G9Ip10F0hFU5lTCyMFIoVkDf4T8ARaL1e6PVGcmFouRkZEBV1dXhVOfPD30FPc238OkfZPkvliQFovFgrWfNaryqlCZU8nISb04oRixf8ei19JejTOLCPlQFIXZZ2ZDWCtsd1tDW0MM/mpwmzPXOjo6++2rDo05BGGNsNnFNKEe0q6lQUtXC04DndRm9jrXiItur3VjOgyle/DLAzzY+gDvpr0r1/WlWCwGx5eDdzPeVWqqMWGtEPuC9sFvrh+mHVJeSp8XSp+X4levXxG8PhjDvxuu9P11RIY2hrDubo26sjroW+rD0scSo7eOhstg1V1bsDlsfFr7qdpO+Bj44UCprhHkJagR4OSck7D2s25WD+bFObfsahlC3w7FrNOzGmdUc43krw0pqhdBx0AHHB3mUg+6D3eH+/COscKPaM7W3xaf8T9T+oNYuvRb1Q9ek7xoWT2uzGtlQj3pmulCz1xPrt9ljjYH3lO8lbaavLOI+i0KTw89xQeFHzQO+onqRNg/cD+6z+7eZs3OxNOJuLvxLthd2PAf5k/6rZRG/DACvExei9dv1QXVeLT3Efqv6Q8dQx3c+OwGHmx9AGMHY/jP94cWVwu+r/miIr0CFl0tYBsof2pMVZxnRv44ssX9VOZUYqvzVpi6maLbrG5qc48sjey72bj30z1wjblyDfop87nrgpsLpMqcsO3+NqXF0BIy6NfBPd7/GDF/xKDbrG4tDvplhWfh6d9PYe5p3m6xWEI9iYVi7ArYBeeBzph9Zrbc7fRZ0QeUmCz8bU9VVRUt7Qhrhagrq4OOkWpm+gd/HIxuM7vB1NVUJft7VcCCAPR4owcp+E0DFosl9UUOR4eDIV8OUW5AGoCufvuqrhO7Kj0dxtGpR1GRUYFlj5cpdT8dDUVRuPzeZfCyeHgv9z21W1VFSSiNS6UiC6+JXuAacWHiJP/DyOrqahojahlFURj500iVPbgysDGA3xzlpsuk0/HXjsOhvwMGvD+A6VAavXq/YuZmhv6r+6s8DnUd8AMAvzl+Sm1fW18b+Y/yW03fW1VVBbuedhjw4QA4Bf23AqkqvwrVBdWw7mYt8+Cdrb8tlj9ZrlDcRFNRu6KQeDoRE3dP1Ki0w8rCYrPAguacl7V0tWDpZUlbe8q6VlYnJ984idLnpVgSuYTpUBg35MshCt0jkrqIihv81WB4jveEgfV/ZXa4xlyM2zGu3evEviv6wjnEGVUm9PTb7IhsJJ5ORND7QTC06bgloJwHOgOtZF9MOJWAm5/fhJG9EQLfCsTgzwfD1NUUHmM9wGKx0H12dwCAuYdmPDtvLUsKR4eDbjO7wXO8J/zn+6s4KsW4DHbBssfLmvQZWQhrhShPK4epqyl0DOl9NmBgbQADtB9XV/OutO63PWTQr4ObsGsC/Bf4tzrL2mOMBxbfXwz7Pu0/QL719S3wsniYvI+c4NWJsEYIv7l+Cl/091vVj6aICGn0WtpL5emJzNyZW81LUnrSp7akFlxjLqOz3YkGw74Z1v5GCjKwMYBYKAZFURo1E48pErEEYr4Y2vramHF0Borji9VuwO/PEX+Cl8nr0CtE3Ya5wW2Ym8Lt5EbmIiU0BSHrQ5RSl1HHQEelA1q6JrqY9rfyVxTSQVAjQMqllE5R40UexfHFKHpWBN/pvh16AL8lLBYLq9NXtznwadfLDo59HZu8FvFTBO5vuY/VGath6mKq5CjpJ6oX4c63d2Dla6X0gVVV4GXxkHk7k1yjvyTnQQ7qSuvgOa7tmtnqoCiuCEb2RiqrZdoRaOlpqd01IdF5mXcxh3mX5qn++7zTp93PGjsaw8DOALGxsbTEUpxQjIgfI+DY3xE+03xoaVOd1JXXIfFUIgIWBrR6P+0/zx+GtobwmthQ01XPXI+RSWV0e/UZgqGNIWYcncFgRPJjsViwDZB/heWjfY9w6d1LmH99Pi33qS+IhWKUp5XDyM6o3Xr2w9yU//zoZeo7RZGgBUeHA9fBrm1u49DXofFHgJI0X+mVcSsDFEUhKywLSWeSlBEmoQBdU12M3TZWqosDZRDxRQCAmIMxODzhcIt/QwQBAJW5lXh2/Bn5G1FQ6MpQfG/8vdSpu+5vvY9fPH9BXVmdkiMjlGHCrgmYc34OGfCTQnVhNfYP3I+InyIAAFY+VvCd7stwVM3Z+NvAoZ8D02FohPTr6Qj7NgwFMQVyfT73YS5KEktojqpz0DHQwceVH2P8b+OZDqWZxNOJODz+MIoTirG7126cfOOkymN4sP0BTsw8gdrSWpXvuy0ZtzOww2cHEs8kKnU/bQ34iQUNE1Ve5TnWE8M2DGv3gUhL0q6l4cEvD8Cv4sv8WbqwtdkI+y4MCScTGIuBTsO/G4711evl+j46qrANYTgy8QgqcyqZDqVNFEVhT989+HfOv0yHonQURSGhOAEnE05CKFYsbfGkPZOw4OYCmiLTXBRF4cZnN5B0Vv5ne1X5Vfh3zr94tO8RjZF1HuXpDbWvW0NRFMTC1ktB1ZXVtXielZfPVB+sSFgB76netLWpTm5+cRNnF51F6uXUVrfRMdSBz1QfFMQU4OHOh23++2uCh789xHcG3yHvYV6L74sFYoSuDMXDnQ9VHJl88qLykHE7AxKxRO42nIOdMeiLQTB2pLeWHi+Lhx3eO3Bnw502t6sV1kJfW7WZz8igH8OU9RCvLLUMl9+/jOpC6VIj1RTV4NCYQ7j9v9tNXo//Nx4Hhx7Ew50PMffiXKwrXqeMcAk1UJJYgv3B+/F4/2OpP5N1Nwu/df8NtaW1KEksQVZ4FspSypQYJbNYLBacnBSvDSWsE+LW17eQcz+Hpsg0w73N93Bi5gmUPi9lOhSN5jTQCf4L/KVe9UJJKHC4HNSWqNeDSVWhq9++6t6Wezg24xgZTFUjOgY6ENWL1H7VzejNo1VSP44pZall2Oa2TabriVe96Lc95vXA0kdLpcpI8SqxQIzjrx3HgcEHWp0kEfZdGPb224vqAuWnEn3h1te3ELoyVGX7UwSLxVLLVREivghp19IgqhOBa8KlPUWPNPwX+GP6kelKWYGqCLFADLAaai4pk6BagEd7HyHlUkqT11ksFgqOFWCH5w5UZFQ0ec99hDtCPgmBnpnsK5Pijsbh0ruXIBHK/7BHUWwOGysSVmDinomMxUA3Talfpyohn4ZgTugcGNkbMR1KmyQiCUI+DYHfXHpWnCrrWlkRvHoePrn+Cdy2ucF3py+mH5uOz29+znRYHQKfx0fYhjCFJodwjbl4duwZ8h/l0xhZ5xG6IhSb7TdDWNf8+jQvOg8/Wv3Y6mCMWCDGD5Y/4PS807T1Wz1zPVh6W6rVb4CiKAnVODFr2LfDMGb7GHQZ3aXdz6VcTkHoilCUp5YrO0SlMnY0hnOIc7PsUMXxxbjwzgXkReXh8f7HeH7uOUMRyubye5fx57A/URRXJHcbdoF2GPr1UNrLOugY6CD4k+A2Vw9G50XD9idbfHLjE1r33R6Sr4VhbLZyLrRjDsTg/pb78JrkJVVOZj0LPVRkVDRL/+c62BUBbwbAd7qv0m8eWyKqF0FLl/yZtuXiuxchrBVi0t5JCrWjY6SDksQSmR5gF8QUoCqvCrxMHoI/CsaQL4d06O+LzWbDwkLxE0RJQgluf9UwwO7Y37GdrTsOv7l+sA2whYGNfDm4iQaypuINei8IQe8FKSka9UdXv31VcXwxUi6mKH2GfHVBNR7+9hDOwc7oMrL9G5XOTMdQB4vuLYK2nno9hO9s6krroGOoAw5X/uvGxn5rAcBFvjY4OhxMOTgFNYU1rQ7MCOuEqMyphK6Z6tLbZdzIQHFCMcb9Ok5l+5RH1t0siOpEcB3iqnb167rP6g6XEBcY2RthwQ1mVm04BTk1qVenLrqM7IIV8StUsq/zy86j++vd4THmv7oxbDYbZnZm0LfSh7ETfTOpB38+GD3e6MF4Kko6a6gxqaaoBgmnEuA2zA0WnqqpaaoJHPtpxn0ZR5uDQZ8Ooq09ZV0ry4uiKAT/EYy4ojgAgBZbCyKJCDse7sBHAz+CmZ58JSvyovKQfiMd/vMb0vh1VjpGOliZtFKhUhE6Bjr4qPwjcI3ISmF5eE/1ho2/TYv3LGbuZjBzN2s1da+oXoTARYFw6ONAa78V1AiQG5kLx/6OHeJeaovDFtj1tMOcC3Oga6Ir0zOUUVtGwdJbs8/3XhO9GlOVvqw8rRxRv0XBoZ8D1mSsgZ6FZqSIHr9zPOor6mHrL396T2UxtDXE8A3D29xmd/RuVAmqUFKr2gw0LIrONcGE1CorK2FiYoKysjKYmdFfZ4uSUMi8kwnXIa5Sf+blAbay1DKYuZk1mS1fnl6O8rRyOA90pmVgJ/9RPooTitFjbo9Wt7n28TWUJpVi3M5xMLJT7xl3TNkXtA/COiGWxzBT3L4zDcyKxWIkJyfD09MTHI78F8kivgjF8cXQM9fTyJomBKFJ6Oq3LRHxRdDiKvf3ryKzAttct6Hf6n4Ys3WMUvelqZ7+/RT15fUIXBSoETepedF5eHLwCXot7QXr7tZMh6OWXu63bBYbadfS4BjkKPXDpXs/30PewzyM3T4W+paqTaPSHkG1AFp6Wmq/wubIxCNIuZyCT2s/VbtBP0I9PPnzCXxn+DYZVG/rnBv3TxwebH+AibsnauxvX11ZHWqKajT+YWByaDIOjz+MCb9PUHmNcU1QHF8MEV8Eu0A7pkNRCWVeK8trT/QebLm/Bf8b8j+M9RyLoH1BiCuKw4ZhG/BJiHwrJe79fA9X3ruCheEL4TzQmeaICUK16O634RvDcX39dbx5+024DJJzxh2DiuKKUJJU0lja4cyiMzBzN6N1gkRHIBaKUV9RDx0DHbXLVvEqiqKQF5UHKx8r2rJ6nJh1AhwdDqb+NZWW9qRBURRMN5mikl+JM5PPYHLgZPB4PBgb05tmtCXkDq6DYrFZMg34AWgcuBHWCvGLxy+I/DWySZ7oqN+i8NeIv1CVV0VLjNc+uoZzi881W9KeG5nbWPPLbZgbUi6lSF27qjNadG8Rljxcwtj+Xx7wqyurQ/jGcKRebT1Xtqarr69XuA0trhbsAu067YBfW/nribYlnknEkUlHZE5r8GjfIzz564mSolJ/dPTblih7wA9oSM2xLGYZhn2r2qLPmiRyeyRufXVL7dN6vlCRXoHIXyJR+LSQ6VDU2ot++/TQUxwafQhxR+Kk/mzhk0KkXk5tXBUU9l0Yziw8o5Q4ZaVjqKP2A34AEPxJMCbtm0QG/FpRklSCLY5bEPZ9GNOhNPH8/HM8/E01tWj857ecary1cy6/kg9eJq/FdGbtqSurU4ua0OeWnMMO3x0K1ZRRB45Bjph7cS48xnq0v3EnU1deh10Bu3B13VWmQ2nVk7+e4MCQAyhOKKatTWVdK8vrrcC38HT5U7zW7TUY6hji44Efw8XEBQ5G8tdE7jazGxZHLlbLlSKqVM+rR3VhtcK/qbWltYj7J46U7pCRWEDf+ZnOftt1QleM+GEETF1NaWtTVSiKwrml53ByzknwsnkAgMn7JnfqAT9BjQDXP7mO2COxTV7naHNgYGUAbX1tVBdWI+N2BkT1IoaibFt1fjX29t2LK+uu0NZmVX4V7WUdnv7dcK/a2m9hSW0JKvkNtYL7OcqWtUtR5C6uA3r8x2OFaoUJ64QI+iAIhbGFTeomeE/1xvjfxtOWAmnQF4Mw99JccLQ5qK+oR150HrLvZWPfgH24/N5lAECXUV2wImEFzLuYA2ioMRi+MRwivnr+KDGFrtSreVF5iNgcIdWPfvimcNz7+V6Ti0URX4Qbn95AzB8xtMTTUZWnlavtiVXZrn18DRtNN5I6aHIqSy5DysUUmR/C3vnmDu5vua+kqDqfrLtZiDkYA34VX+n7YnPYsPW3ZaRmlaZYcHMB5l6cq5JBWDp4jPHA2uy18H3Nl+lQaEdRFK5+dBXPL9BXH8J7qjdCPgtBl1HSp7edcmAKVqevbvytzIvKw/Pzz5s8aKnn1ePuD3eRF9VygXtlqSmqQcatDNRXqNcD1lc5BTnBf54/02GoLV1TXRg7Greafospj/c/xsWVF1U2sCysFeLOt3ca7z1PLziN+F/jW9y219JeeC/3PTj0kf2h/c/OP+Pw+MMKxUoHn+k+CPk0hNHagnTQM9ODxxgPmDiZMB2K2tEz08PoLaMx5KshTIfSqpqiGhQ+LVQoPaO6KqguAABw2Bxoc/6bVDCr+ywkr0rGggD5UzobOxjDoY9Dp7+mjjkQg822m5EVnqVQO8XPivHv6/8i8bT8tQE7gvqKeiScTGj87/Qb6ciLbvnaUlAtwE82P+HmFzfbbLM8rRyn3zyNpLNJzd578ucTnF5wGtWF9A5cWHe3xsB1A2HirHnnBRaLhRlHZ2D6kenkvPb/2FpshH8fjudnm96T1RTXoOhZEYR1Qjzc+RAHhxxEabJ6DtyztdkY+u1Q+Ez1oa3NhXcWYt7VebS1BwBVuVXIjshGa4k086oafg+sDayhw1Ht+Uczno4QUhPUCHDh7QtwGeSCeVfk+0PWt9DHqB9HNXud7toVLiENS8YpCYV9QfvAr+Jj2aNlCFgY0CTNyMszTR7tfoScBznwm+tHfszRcDGQ/ygfLoNdYGCleJ20xNOJCNsQBo/RHm2m3aEoClE7o6Brpougtf/VCjOyM8LCsIWw722vcCwd2YEhB6BvqY9lj5YxHYrKWftZw3uKN+p59Wr3oEwTDPhgAPq920/mQb8Z/8xQuxR3mixqZxRiD8eiy8guKqllIagRoCylDNbdrTVihZCqaetry/UQmSk6hjod9oFTVW4VIn6IAL+Sj67ju9LSJteIi2HfyL7S9eV6m5P3TwbXmNtkNWhpUimufXQNwzYMU+l1S/yJeISuCMW8a/PgPtxdZfuVhUQkAYvDAoulGatnmWBoY4jF9xc3/jdFAerwzzVi0wj0e7efylY+87J4uPXVLZQklsChrwPYWmxUpdGTFeYFiViCHm/0gKUP8yk1/eb4MR0CLeor6sE14ZI+3oq+K/syHUKbBrw/AAPeH8B0GLTLrMiE6zZXdLfujsfLHkOL/d/jypf/vyKEdUKAgtqntVMm6+7W6P12b5i5K1ZmyK6nHab+NVUjU0EqKnJHJAxtDeE73Rfnl53Hs2PPsOThEogFYvw5/E/0mNcDU/9snj6wpqgGdr3sYOTQdukitjYbTw4+gaGdIbwmNa3LlnM/B0/+fIIRm0fQekwvSMQSsNiadw1o4mRCnhG/RIurhZVJK2Fg0/Q58dO/nuLK+1fwVsRb8BznCT1zPVqeJSuDgZWBRqzWHPjhQAz8cGCr7+dW5QIA7I1U/5yc1PRjyIuafhUVFTAxoe+HiZJQjekxnQaoX3F5AEi7loYbn93AiE0j4DrYFUBDigpKTMF/gX+bJxexQIySxBLY9LBRUbTqLXJHJC6uvIh5V+fBfYTiD45Kk0tRXVAN+1727V4IC+uEqMyuhEVX9Sn6rWwURaGqqgpGRkZyXwRRFIXbX9+Grpku+q/uT3OEBEG8io5++6qaohpkR2TDe4o3Le215/J7l3H/5/t4N/VdhW/QOxJ+FR9Ru6LgP98fhjaGTIcjk/L0clBiCuYe5kyHQiuKolCRXgGwADM3+f9WW+q3YqEYxfHF7abluvvDXZh7mrc7K1RYK0TBkwIY2RmpNJVRUVwR0m+mw2uSl9qm+Y47Godzi89hxrEZ8BzryXQ4aksgAK5dA06fBgwNgZ9+AtidcF5Gxq0MOA10Akebg4InBeCL+XAOdG52zq3IrEBWeBZcQlw0ciVBR/Kj9Y+w9belfbZ7R0NJKI1JHa4IZVwry2P/4/1YdHYRghyDELEoosVtimuKEZ0fjYFOA1aIQt4AAQAASURBVGHEbXvg5FVFz4rwW/ffMOiLQRj69VA6QiY6qR2+OyDmi7EiYQWqC6uR8G8C+q1uSNsX8VMELfcm1YXVrbbBr+RD21Ab1dXVtPbbqN+jcO3Da1h0fxGsfKxoaVPZqvKr8OzoM3QZ3UVjYmZSdkQ2Uq+kotfSXjCyl+03tCOoyKxA6pVUuA1za8woqGx7ovdg6fmlGO85Hn+P/xumpqakpl9nQfdFFYvNgmN/R6UM+KVeScWv3r8iOTRZoXZqimpQnlreJA2X/zx/BLwZ0O6/B0eH0zjgF/V7FCI2t3wx2Fl4jvPE1L+mwjaQnrz0Fp4WcAlxaXPATywUQ1grhLaedqsDfpW5lbj+yfVmqe/ijsZhu8f2xv+uLa1VKBWtqrFYLBgbGyvUb1ksFoZ8NYQM+BEy41fx8WjfI5SllMn1+drSWvArlZ+OUt3Q0W9fZWBtoLIBPwDwHO+JwV8OhpYeSdDwsvgT8bj24TUknWme+kbd/R7wO84vO890GLRjsVgwczdTaMDvRTuv9tsjE47gwKADbabHFtWLcHfTXUTvim72XtbdLMQd/a8uoLa+NpyCnFReu8S6uzX6reqntgN+QMPqSoe+DmTGdBvy84G1w2Jw6v0w5OcDyclASQnTUTU8CFT1nF7XIa6NpQZs/W3h0tOlxXNuzr0cnHrjFHIf5qo0PjplR2TjzxF/Iu1aGtOhyE0iksB7qjfcRrgxHYraqsiowCazTbjx2Q2mQ2nRo32PkBmWSVt7yrhWlsf19OsAgOFuw1vdpv++/hj791hE5kbK3L6hrSF6vNEDNn5kAjmdhHVCtai5qmyHJxzGnW/vAADG7xyPxQ8Wg6PDgYmTCfqv6Q8Wq2F13MB1A2mZjNhWG1xjLthsNu391sjeCDb+NhBUC2hrU9nyH+Xj8trLyL6bzXQoaqe2tBZFz4qavOY0wAlDvhqiEQN+R6cdxfGZx2lts/BpIc4vPY/MO/SdQ1MupbR5XWhtYI3RXUajr0NflZ9nyaAfw8Ri+oq4UhSFvKg8pZ1w2dpssLXYCrfvN8cPHxR9AId+iqXiitkfg7ub7qr8xladmLmZoccbPaBvQV/aPolY0uqgQl1ZHfYP3I/QlaFttpEcmozw78Px7OizJq/n3M9BdX5143d2d9Nd7Avah4zbGbTErmxisRixsbG09tvOKOFUAg4OO4jaklqmQ9EohU8KcW7xOSScSmh/41ekXknFj5Y/Nivk3BnQ3W/zH+ervCal+3D3hotzO/W/OFcl3+m+eP3c6+g2sxvTocgseH0weszrwXQYtCt4UkBL/2ip3wYuCkTQB0Ft1nXW0tXC0uilGPpN8xn8V9ddxYXlFxqvY2uKa5rU+CP+4znOE/Ovz28z1Xtnd+4coJv0FPapd2H4/8/liora/oyySUQSbDTZiH9n/8tYDG2dc52DnTHr1CyZy0Vk38vGybkn1WKwUFgrRN7DPNprKakSW4uNib9PRPBHwUyHorb0rfRhG2CrlitSxQIxzi05h8jtsg96tdqmGtzjUhSF62n/P+jn3vqgX2/73gCAqLwomfehb6GPqX9Nhe+MjldTWRbnlp7DxdUXaWnrzoY7+M7gO7knpWqKurI6FD8rBi+LB6BhsktbpTMe73+MS2svNXktOyIbRyYdkbqWNL+Kj4STCSh8Wtj4mkQsQdK5JJSnlyul33pN9MLCOws1qmyCc7AzFtxcAM9xJDPFqy4sv4Df/H6DRNRyHeK6sjr83vN33Pyy7RqTTBFUCyCooncA2inICXMvzqX17+XKB1fafEY+2XsyLr1xCV8M/kLl51kyZbwDKY4vxp4+ezDgwwEYuWkk7e27DXXDO3Hv0NIWHaPbUw5Ogb6VPuMz0pgkEUlkru3VnhOzTiDpbBI+5n0Mbb2mK/50zXRhYGUAA2uDNtOd+M/zh5G9UbOUUGN+HoNBnw1q/M4C3woERVGN9R01gaI/0re/uY3c+7mYfmR6k3pDnUltcS1yH+SiOKFYo757pll1s8Ks07Ng3U32h7BWvlYIeCsAFp6dJx3vy+i8uPr39X8hrBViTeaaTn3+UQdcYy66TqCnbpyqBX/cMR+2HplwBHrmelj+ZLnCbb3ab9sb3H1xXWLqatri6r1h3w5rcs10/LXjKH5WjHXF6xSOVRYivgh7++2Fy2AXjN02VqX7JuhBUUBEBFDtNwGrVrMhfgg8ecL8oJ9YIEbAWwGMP6xr7Zxr7GgMY0fZUxmVJJYg9nCsWkyUcB/hjo95HzMdBqFkOgY6WHBzAdNhtIwFzL82HzpG9NYGZnpS67PiZyisKYSelh6CHINa3a63XW8ce3YMUfmyD/oRDbLvZtP2HMK6uzW6vdatw6/00zPXw7tp70JYI5Rq+5SLKXh+/jmGfj208d+64EkBkkOTMehz6WqU8bJ4ODb9GPq/1x+jN48GAORH5+OfSf8g+JNgDPnfEMb7rTrQNdGF6xBXpsNQSz4zfGDV3QpiobjxHih8YzhSL6di9tnZ0DHSQX1FfauDgkybd4X+FOT6lvrwGOPR7PXzb5+HdTdr9F3ZF7kPc2HmbgZdU10kX0iGXU+7Nq9fR/88GmK+evZFMujXgXCNuBj48UB4TfRqf2OG1JXX4eHOh/Ca6KVwXT5Lb+aLuTOJklDYaLoRvtN9MeXgFNra9Z7qDbMuZhDViZoN+rFYLMw+OxtsTtsDjVq6Wug6vuFB7INfHiD3fi6m/jUVLDaryapES29LjPpxFG2xawJeFg+5kbm036hpkh7zeiBgYUBjKihCOnpmevCeLF9KSWNHY0zeN5nmiDofSkJhwLoBEAvEKh/wOzLpCAxsDDBpzySV7ldd1ZbUQlAjUOsUiZ0NJaEQ9H4QdAxVf36ryKzAodGHMPaXsegyskuL27gNa5rKrsvoLrDvrfqC6lpcLVBi9a0TJRFLcPats/AY54Hus7ozHQ6t0tOBTZuAhQuBfg2ldyAQAHl5gIsLIO3PenIyUFoK6FqYI2gs8OT/MwQxPeinra/dIc/1gQsD4fe6H1gc9ewzmubhbw+RF5mH0T+Phq6pLtPhEDLiaHOanc86gher/IKdg8HVan1ASpGVfgBw9cOrYHFYGPH9CLk+3xG88+wdSMT0POT3nuwt9/2ppmGxWFJf447aMgqT9k8C1+i/v+U+b/eB3+t+Ug+4WvlYYdK+SXAZ/N8kabMuZpi0f5JSJ/fc+fYODKwN0GtpL6Xtg061JbXQM9dT2+tqJrV0HV+VX4XC2EKwtdjgaHOwOm01A5Exi6IoCGuEjf1ZUC1A3OE4dBnVBQELA3Bg8AG4j3BH35V98c/kfzDw44FtnjNau/d8gS/it3leUyaS3rMDMXE2wYjvRyilnh/QUCMi8tdIhVIx5kXl4eZnN5FxS/42XqAoChWZFc1yFHcWghoBuozqAktfegc/e8ztgZGbRkLPXK/xtZqiGtz98S74lfx2B/xeJeaLkXU3CzEHY1rdJis8C/e33pc3ZI0yac8kfFD4QadeIaStp00G/OTwao1MQvVYbBZ6LuqJPm/3Ufm+q3KrUFNYo/L9qqunh55im+s2pF3XzLpKj/c/xp6+e1CZU8l0KLRhsVnov6Y/ei7uqbR9XFl3BTu772yW2j03Mhd1ZXXtpuukJFTjv3nI+hCM+omZiUdvx76NMT+PYWTf7anOr8aTP590yNoo168DubnAxZeymu3eDaxaBTx+LH079+41/G/vnmLUFfJgadyQeqi4mMZgO6Cs8CxsNN2I6N3Na262R0tXS22uHZPOJSH9ZjrTYcgt+242nh56Cm2D1mu4E0DskVhceOeC2pUSEQvEHXJV1a3MWwDarucHAD3tGq4xMioyUFIreyHV5NBkJJ3WvFrQr5KIJHj8x2O57w9lfabTWbR0XV4YW4hLay7JlMLUxMmkyYDfC7qmulIPTrHYLAS+FQjzLuaNr+lb6CNwYaBS069H/hKJJ38+UVr7dNsXtA+7e+9mOgyNMXbbWHxY8mGzBR7qJv1mOiJ/jURdOf1lVf4I+QM7u+9s/G8dQx18UPQBxmwfA0GVAIFvBSJ4fTDcR7hj5I8j0XdlX4X25/SzE8w3mSOpRPXnHvJLzzA2W3O+AkG1ABdXXUTCv7LXk3rBOdgZb0W8BZ/pPgrHQ4kp7PDZgctrLyvclibiGnEx6+QsldRjiN4TjWsfXkPqlVSZPzvggwFYk7EGgQsDW93m/tb7uPzeZdQUqfcDbTabDS8vL4X7LZmFBJSlluHelnukrp+UKIrCVuet+GfyP3K3EbkjEgeGHIBYqJ6pB5SFrn5LURSjNXyWRC3B62dfZ2z/6saulx16LukJx36OTIciF34lH9X51Uq5kWEKnQ9GW+u3lISCtr426ivqm7ze7bVueD///XbrMxwcehDb3LYh/1E+bbF2NEYORvio4iMM+WoI06HQLu3/5whkZv732tOnDf+bK2W5OIoC7t5t+P/uNXHY6rwVWlkNDTO90i8zLBOnF5xGQUwBYzG0dc7lmnDh0McBBtYGMrWZ/zgfOQ9y6ApRYafeOIWwDWFMhyG3aYemYV3JOrUZRFVX6TfSEfVbFOrL69vfWIUiforABv0NKIwtbH9jKdF1rayI13xfw7t938WoLm1PxjHRNUFXi4aMQvKs9lt8fzHeeUZPyRom1RTV4OGOhzjz5hmZPleZW4nE04moLqDvnub2N7dpqxHIpNv/u41t7ttQmlza5PXn557jwbYHMj+rqsypxMOdDyEWiJF6NRWP9j2CoFr2+mSCagFKkkogFopRz/vv90hZ/XZh2ELMOjmL1jaVqeukrvCe2jlWm8oq92EuDg47iKRzrQ82pVxOwf1t6rcAI+6fOFxcdVEpaTPdR7ijy+guTe5dtbhaMLIzgqGtIcb9Og5OQU5gsVkY8MEAGDu0ntqzPK0cG002InxjeIvv80V8FNcWo7y+HBb6Fio/z2rOiBPRpoqMCvzs/DOi98g+c1Ja+lb6WHBzAfqv7S93G9p62nAKcmqz00iLrcXG0P8NVeqM8s4q7Lsw7Oy+szG384APBmDa39OUdjId9PkgLHu8DPpWrRdDVhc6OvKnLStPL0f0nmhUZFbQF5CGSr2SiivvX0FupJRP2To5Ub0I3lO94TRQ/pXcvEweCp8W0nqTpykU6bcv5D7IxRaHLXKtUKBDZ14d3BKXEBdM3D2RkVSSdOi/pj/WZq+FjZ9iqc7VyeFxh3F4wmHaBv9a6rejfhqFJZFLoGem1+w9Nofdbj8Z+8tY9H23L+or6nF06lHGBhLSb6YjfGO4WtbQYLFY0DXRbZLxoSOgqIb0ngBQUgJUVwN1dUDB/4+P1Uj5LC8rC8jPB7S1gb4TbdBvdT/YepsCYH6lX1FcEZ78+QS1pcxOqGrtnGvjZ4N5V+fBe4ps9xNX113FodGH6AiNFpP2T9L4QXFdE5LWsz3Dvh2GD4o+gK6Zev1bmbqawn2EO4zsjGhtl45rZUXM8ZuDbWO3IdCu9YnCLyiS4lPHUKdDTMA1tDOEiZMJzLqYybTyM/NOJo5OPYrMsMz2N5ZSxs0MxB2JU7tVsbJyG+YGjzEezf4+gtcHY2n0Ujj2l22iYcyBGISuCEXK5RQ8/PUhzi87L/PkW4lIgu1dtuP0gtPICs/CD+Y/4PH+/1ITKKPfWnS1kHlyDpNGbx6NwZ8PZjoMtSQRSVAQU9BkwDr1SmqTQcDH+x7j8prLajcxfOjXQzHv6jwY2ND/tzjkqyGY+PtEsFgsiOpFiPgpAuVp5a1uL+KL8Pz88xbfoygK9n3sYWhn2OL7+dUNE011ODqw0LNQPHgZkZp+DJNI6LnZry2pha6pLrR0lfeVcrQ5ChdILUksgbmHeWMRUUUN+GAALe1ooqd/P0X+o3wM/nww7fUYhHXChhRYuZUwsjOCFlcLfnP8aN3Hy2z9bZXWNp0kEgliY2Ph5+cHDkf22bFZ4Vk4v/Q8Zp6c2enrUHlN9IKFpwXs+6i+npIm0tbTxuT9itXpGfbtMIzYNKLTDR4p2m9fYGuz0WVUFzj0U14NhbZU5lTi+YXncB3sSmraUlSn+ztWdxRFgaPDAVur/YE3abTWb1tqO+5oHIriihC0NqjdgSqbHjYYvXk0Hv/xGEnnktB3lWzpWjIygEOHgClTgO4KlLuLPx6PqN+iEPBmAAxtW75JZEpFZgX4PD4svS3B0ek4K4FKSxsG+l7IzATY7IbBQKDpe215kdozMBBw6WcLl35jGgYO/2oY9KMo6WsD0q338t7wn+fP6PdG1zn3ZX1X9VWr9Na+032ZDkFuNcU1yI7IhmN/RxjaqNdvj7pR138fvzl+tN+XK6PfKtOiwEUY4TYCg11lf9DPy+KhPK0cTgOcNP4cN/PfmTIPYDoNcMKUg1PgFERfSaCZJ2aCa8LVyGvz3Ie54BpxYeltCedgZzgHO6M8rRyP9j5qXFzAYrFg19NO5raD3g+Crqkuuk7oCqcgJ+RF57U4aa0tbC02+q3uB11TXWjracNrshdsAxqenSmr3/Ir+ajKr2p4bkvSwKoFkQjYuxeoqgLef7/h+lUaTkFO+Kjsoyav3fz8JqryquA10QtAwwKMoPeD1G4yhKGtoVLvkSiKQlFsESpzKnF13VUArY8vXPvoGh5se4DlT5c3m7Br3sUc86/Nb3U/eVV5AAB7I3uwWCzaxoCkRXpwB2Hf2x5vP30b/vP8lbofSkJBVC+S67N15XXY4bMDZxedpTkqelNKaYrk88m4v+U+OFz6L1aHfj0Uyx4vw5X3r+D0m6dVUjdAIpIg41YGBDWypzzQFB6jPTDnwhw4D3RmOhTGGTsaw32EO5lpLAVhnbDdOlXS4OhwNPJmTF3Y97LH3NC5jE1SKE4oxoXlF5B+Q3PrCNElfGM4fuvxW5sz8tSdsFaI2MOxyLqbxXQotGCxWJh9ZjZm/jtT6ft6duwZbnx+47///ucZIn6MkOl6KHBhID6t+xQug1yk/kxhIfDFF8CDB8C5czKF3Ez/Nf2x5OEStVxN93DHQ+zy3wVeFo/pUGiV9kr5z4yM/1b+AUCtlIvjov9/sXdQ0H+vWVo2DPQJBEAlg2U6WSwWdAx11PZBdm1JLW58dgMpl1Nk+pz3ZG/0WtpLSVF1LtkR2Tg65SiSQ5OZDkXtiQVilCSWoDK349TeVVcF1QWIzI1EWZ10NdOGuQ3DwsCFcDdzl3lf97fex8GhBzU68wkvi4fNdpvxaO8jAIBYKJb6d9XUxRT+8/1h7Kh45q0X9Mz1NHJwqDKnEn+N/AuHxx9ussrp7o93cW7JOZQklSD+RDyywrPket6oraeNviv7gsViQd9SHx6jPeSKM+STEPR5pw8c+zti1slZcg1AyuLa+mvY4b1DI8qwxJ+Ix4nZJ1CWKn29RU0jkQA//wxcuADcudNw/aqIEZtGYPyu8Y3/beNnA8d+jmrVh2uKa8DL5in1OX/079HYFbALdeV1mBM6B91ntz6b03++P0ZvHQ0DK9lXHeZWNmQ2czBiZuK4+nyrhEbY4rAFf474U67PSkQSBK8PRtdJXWmLp7qgGr/1+A3XP7lOW5uaYsqfU7A6fbVSCrCy2CywtdigJBQoMaWS9FPRe6JxcOhBpFyS7UGAJjGwNoDnOE+NSpegTBRFqX0dR3Vwb/M9/OL5C0qSZC9W/zKKopAcmkwe9MhB3skudHLo44C5F+fSUhNX07FYLAhrhDByoDe1lSqJ6kU4OfckYg7EMB2Kxok/Ho+wDWHgV/EBAK+deA1Lo5dCx0C2FEccbY7UmSd4vIYBv/L/H2cuLW17+/ZYdLWAfW97tRyc8RjrgcFfDtbo/tWS9FfmS7w66Cdtes8Xtf88PBrOq8dfO47bX16HmVnD60zW9St4UoD8x+pbr7KeV4+wDWHIuJnBdCgKufrhVXxn+F2z2qKawK6nHSbtn6Rw9p7OoDS5FDt8diD6d2bSurfmzFtnmqT36wjOJZ1Dv7398MbJN5S+L+8p3hizfQx0jFSXzjT1aiqKExryP6ddT0PoylCF2qsurIaRnRG4xlwAwPHXjuPcEulmIynjIbqoXoSM2xkoimO4sK2MuCZcvH72dcw4OqNJjdPey3tj9tnZMHY0xsVVF5WyaEEe8tQDlIfHaA8Erw+mLTubMhU+LcSzo8/UasCKThQF/Pprw2DfC1kyzhdNDk1G8sX/nv+4DnFF1/FNn8mL+CKI+Mw/73jh9PzT2Oq8FUlnWq9FqKguo7rAe7I3PMd5wnOsZ5sTIex62qH/6v4trjwM+z4M4ZtarucHALlV/z/oZ8zMoB9J79kBSMQSXH7vMrqO74ouo7oodV8+033knpVsYGWA4d8NpzUefSt9SEQStXxoomwcbQ5MXU2V1j6bw8b0I9PB0eaoZKl31/FdUfu/Wo1J9SkPfhW/oY4AWW0FADgy4QgywzLxMe/jTvlvknQ2Cfa97WFk3/aDVV0zXRjaGcLM3Uyh/bFYLJxbcg6GdobwHOepUFudiaBagL399sLCy6IhjQ5Df6u6prrwGCPfDNGOJvjjYAR/HMx0GArRNdPFaydeg5WPFdOh0OLax9dg6mqK3st7K31fw78fjjHbx4Br1PCwi81hw7qbtVL3+euvQF4ewOUCfD5QRsOE4hep1GUdrFQ2t6FucBvqxnQYtHsxwNelC5Ca2pDeU/zSInpp0nvW1DSkVgIAG5uG82p2RDZE9SJYezf8XRQVAZ4MnWIvr72MgscF+Kj8o/Y3ZoCJkwlWJq2EnoX095LCWiF+8/sNPeb1UJs6eibOJnAe6KxWD8ikZeJkgsCF7ddMIxpWRAWvD1arAdJ6Xj1i/ogBAAS+1XG+x9TyVACAh7n017m3Mm4huTQZw9yGoYu59M/AXAa5yLTKX1EUReHErBOw8rHCW3ffwv+xd9bhUVxdGH9nNxvfuLtAEiJIEtzdXQuFCtBCKaXQUkr1q9BSoy2lQJEK7u7u7hBCQtzdPbs73x+TibG+szuzS37P0yfp7sgJu3dm7n3Pec+zA8/weNtj9Py0J8pzyuEQ5AAjE9WWZd07uuPt+2/X/7/fAD+4RriClJAK123Wd1wPEysTvHbuNbX+HmlU5Ffgvz7/IXJuJIavHq54B45gIjSR+l1waecCl3YuIEkS045PQ3luOetrFYdmHcL9jfcx7cQ0tSsGlSVwVCACRwVq9RxM0ffrvuixtIfKY0hfuHcPOH2acpPw9qYS1lQV/Y7NOwZTW1O0Hir94TT2aCy2j9iOMf+NQbsZ2nUOVJYhvw9BwpkElXtAq4Ktny0m75+s0j7iGjF4gqatLO7+dRdmdmbosUT62gRd6edmyU5bI8McGXoET1kzXjnkPcvDrZW3YGRqpHXRb9iqYWrvq40ePDw+D/OezmP0mPpAdWk1sh9mwynMSav2iLq8eVp7WXO+AS+Px0NYWJja4/bPoD9hH2jP6EO2PuM/xB82vjYQVYm0UrHKZXKf5mLn2J2YfXs2hG5CSEQSSESSJn1Z6Wtmp3md0PGdjoxcP4evHQ5zB3ONj6NPaDpu+SZ82Afaw6W9C+sTPpIkUVVYxUlLwBZUgyAIve4L1RhRtQi3/rgF797ejIl+8satXSs7AJSDxG/ev2Hi7onw7MZcb5rmiETUpBsAFi4Eli+nKv406d2WfjsdGzptwIAfBqD7R92ZC7YFmdD2nn37UqJfUlJT0U8Ze8/sbOqntTVgVncZXpi6EASPQNpPwLNn7Fb6dZzXkfXed/LGLt+YD/sAe5WOV1NWAxMrExB87d5/RRJKwDPiKZ77dHq3Ezq9q1o/UK5gyD1x00vS8cPVH1BYVYjNYzdrfDxjS2PGE5Y1xdTaFJ+Uf8K4A4Wmz8qaEldAOf342yq/lrXs8jKcSTiDTWM2qST66RpSQmL4muH1c7xWQ1qh79d9cWftHZz75BzevPqmxs8wqlyPHAIdILBgdt4tdBNi8K+D4dHVg9Hjapvq0moIzAUyq8SKkopg629b30OPTbq83wUlqSVNEuTZHrdcwZDXkeKpfAj07g0EBgJ//aW66Dd01VAYWzYkGP7q+SvcO7tj0h6qJYOtry1CXwll1PJXU+wD7FV+XtQ2t1ffxunFpzH7zuwmSbvzoufJtYsOsA/AkFZD0MGVStTR9Xh9ua8OBoJjsCMWJC5A5/md2Q5FJsWpxVjhvqLed7wFzUi/lY5/ev6Dh5sesh0K40jEEk5Y6cmipkY9WwWSJNFqWCv49jO87Hl16Ty/M4atGmbQD2qyENeI4dnds94r//7f9/Fnmz+R9TALAJATlYN1EetQmkmVFDC1QBM4MpDRxu36gqrjNvtxNk59eAqZ9zPBF/AxcddE9Pq8l5aiU549k/fgZ5efdWK5zFUSzyXi7CdnUZxqGP3GuHy/UxYjEyN8mP0hhq9hNrtb0bgtiC+AqFqEi19dZPS8zUlOpnq1WVgAnerW1UQizXq3WXtaI2xaGByCHJgJkiFIkjRI2/yqKiCLur2iZ0+AzwcqK6nPlUYZe0/6GC6N1v/oqgrHuvl/bi4DAatJ8PhgdHynI3sB1CFr7JISEqUZpajIV75PkIWTBd6+/7ZWEwPnHpkL+x/tcSbhjNbOwRXWRazDlsFb2A5DK5TWlOKPW39g66OteJ5vuFb2AnOBVpK/1J3jMoE6lX5OFlSFf065apkWqddTsSZsDaJ2Ram0n7rw+DyETg5F0GiqYiVgeADMbM3qLRRVbftRmFCIQ7MOIf1WulrxjNs6DiPXjVRrX1kQBIEu73eBR2f9Ev0OvnEQy8yWSZ1XPdnxBCv9VuL5UW5cS5xCnfDqyVdfsBfUxrjNfZqLLUO24Omep4wfm0kkYgme7HyC/Ocaeu5zmLQ06qenJ+DlRf2uqugXMDwAPr196v/fuZ1zfQIlQOkJ47eNZ32dUlQtwtr2a3F79W1W45CFjY8NPLp6oLaitsnrAjMBbH1lO3LNjpiN49OOY0a7GdoOUSotoh/LSCSaL9wRBAEbHxudKPN31t7Bodmqe1qXZpTC3N6c8awigGrAe+6zc0g8l6h4YwPB1s8WA34cYHD2S+m30/Gz08+4t5Gb4rBEIkFMTIxa45YgCIxaPwq9PmNfOGiBfVzau+CNS280VGcTgJGpUb2FZ3FKMfJj85H1IIvxc3NdWGcadcZt7tNcXP/lOgrjqQZePCMeJzLjffr6oP3r7VFbWat4YwMl9mgsrnx/xSC+w8cXHMcy82WoLqlmOxSNMbY0ho23DWPHU2bcOgQ6YHHuYrx6Urs9gGLq2kkEBAACAVXlBTT091MHSxdLjNsyjnP2STVlNSDFpMFdY5KSqMpMOzvqP49G65IWdeutyth7Zta1y2ss+uU9y0PUrig42FDXJDYr/biAvLFbW1GLFe4rcPrD0yxE9iK302+jorYCNeIalFSX4GyCcmJ3UXIRzn56FilXVFx54wB2rexg7WPNdhiMc+DZAQh4AgxvPRwkSPx24zdGjnv207PY2HUjI8digpL0EmTczXhh0VFTNJnjagpJkg2VfipU7DmZqyf68Yx4ENeIdZZAJ+s8ruGu6P9d/yaL78qQei0V9zfeb5L8Jq4VY+uwrTix8IRGsb5seHbzRNjUMKm96/wG+MHayxriWrGUPbmBtsatRCRByuUUzidYlqSWYO+UvZwViZiA7iPt4dEg+mVlUa0GVIEkyfp+nlOPTMWA5QMYjJIZCuMLUVlQyfj9jSlaD2uNGWdmwC2iwaYz/XY6sh9lg5Qo3ytV1/fZFtHPAEi9lorKgkqdnCvxXCLub7gPiVi1L6pHZw/MfTwXoVNCGY+ptrIWl5ddRuyRWMaPzVVsfW3RfXF3OIVqt4eNrrEPsIetv61WLUtb4A6iahH2T9+PS8suKd7YwImYHYG5T+bW96hyi3DDu8/elem9ri4ZdzLwncV3Bv1wrAlFSUUAqF4GCxIXcK6HXse5HTFy3cj678nLyIDlAzDn4RzY+au2SMJFnNs6I2RiCGcnN8qSdCFJKwkKyqALMZ4W/QLr9DnbumTOfANMLDYRmuCdqHcw5NchbIfCKLS1p29drpyPT8N7wXUuuxUVlDAoD7rSz9W14bV7G+9hz+Q9sCSp0k+2Kv0qCyuxKnAVriy/wk4ASsA34aPTe53gN9BP6X2yH2Xjxu83GF98zC3PxeAtgxG6OhS+ttQX42yicqJfRW4Frnx3BcmXkxmNSRdM3DURI/9itsqHbSpqKzB9/3S0+qMV+vtSdpz/PPgH+RWaX6Qr8ytRkVfBmUSIx1sfY33kemQ/zmY7FMbIrchFWU0ZCBDwtVE+odnZ0hkAkFOhmujn3tEd78a8i7CpYSrtpy67xu/CqqBVKq+fyaLtq23xfsr7TeYofAEfRYlFCu2dM+9l4vSS08iNZv5GdevPW/jJ8Se9+m52XdQVY/4dI/U9cwdzvJ/8PtpN50aPM13iFOaET8o/QdeFXdkORS6mtqYYt22czsayriHJBtHP3Z1KOhQKqdfpCkBluPrTVXxn/h3ynuXJ3ObEwhO4+LV2nVMU4RjsiPeT3ken+fpjn37247PY2G1jvaDaHJIkUVFLuVv89BPw6adNWwvoghbRT8+RiCT4t8+/2D9jv07ON3L9SCwtXaqwQbAstLE4Y+dvh7fvv40BP3AvW6EF1TC1NsXsW7PR9tW2bIfCONH7o3HwzYMoSi5iOxTOYGRihLgTcci4lcF2KDol+3E2Ng/ajMTzTauTG/cTsHCy0Er1to2PDXz7+XLKs50tRFUiHJp1CNtHbQdJknh+7DlWtlqJh5seQmAmgI2PTRP/+xa4AV/Ah3NbZ7WfQ7hE+MxwTNg54QWrHn1j/4z9ODz7MNthaA1a9Auq6yVvX9dioqBAs+Ne+N8FHHv3mGYHaUEpEutut9JEv7C6tSKxWHHmtDR7z7BXwjBh5wS4t6ZKBtmq9KstrwVPwJO58MAF+AI+hv4+VKUFuqSLSTj5/kkUJRYxGsvmR5tRWFUIKxMrvNH+DQDAg6wHSglFTqFOmPdsHqdba7xMHI45jLKaMvjY+GBBlwXo4NIBlaJKrLmzRuNjj1g7AvOfz+dMKwLv3t7o910/g0h8oqGr/DytPWFipHxSm7r2nrrG1t8WTqFOUvvG3f/nPn7z/g0F8ao9UFh7WsPYoukc5Z2n72D8tvFy90u9noprP15DaUapSudTBnMHczi0cQC4ewtqQUm44G6jDKbWpgh7JQzuHd3ZDkUrFBdT1vMEQSWbEQTg7U29p4rFp7WXNXz6+oAgCFSXVuPM0jN4fqypbW3MwRg8O/CMwejVg+ARMDJR3FuZLRLPJ2LX+F0oTKTsXrp+0BWDfh4ksy9oSXUJLL6zgO1yW5y/XI1HjxrmErqiRfTTc8S1YgxYPgAd3uygk/OZWpvC2NJYpRtBbnQuDr99uL5XFdMQPAIu7V3AF/C1cnwusi5yHQ6+eZDtMF5K+Hz1vmdpN9Lw4J8HzAZjALyf/D6mHJzCdhg6JS86D8mXkiGu1r1diLmDOaYdm4aQSSE6PzebSBu3fBM+Ui6nwD7QHgRBwMrTCj69feDT10f3ASpJdWk1Dr5xELdW3WI7FFaoyKtA2o00zmTct0Ax8MeB6P5xd8aPq+79lklKSxuybAMCqJ92dWutmop+SeeTONcvJeNOBu78dQfluUo0uNMTxGLgad0/s19dgRm9aAIAbdoAvLoZsaK+fvRE3dm54TXXcFeETAqBux+1WF1WRvUL1DVWHlZ458k76Lm0p+5P3gwmx27o5FC8ful1OLdzVryxCjzKfgQAGN9mPNyt3BHiGAISJM4nnVe4r5GpERwCHfQuMSjlSgrOfnK2frHKUNj6eCsAYGroVPAIHj7o+gEAYOXNlfUZ9oaCR2cP9FzaE+YO5owfm617rpe1F34b/Fv956YstOiXXaZaZZlEJMGdv+4g7kScSvupy+AVgzFpzySp7wnMBDAWGqMiV/H3tLaiFtuGb0PSxSSp7yuzPhc+Kxzzn8+HRxfme++FTg7FG5fegHNbZq/V2qIsuwx7Ju9B9P5otkPRCG2N24SzCYg7qZsx0oJ06Go+JyfAuO5xQ52+fqGTQzHt2DQ4BDmgsqASV5dffaE11tv33sbs27MZiFo94k7E4eQHJ7WSkMAk5TnleHbgGbIfUved1sNaI3JOpMzt00upSSRJEuCT1DxB18mBLaIfy2h6kRaYCdB1UVe0GdeGoYjkU55Tjow7GagpV75hbMKZBNxbdw9lWUo0y1ATUZUI6bfSDWqRQhYSsQRGpkbgCQxz+FYWVuLAawdw/dfrbIfyAnw+H2FhYWqN24E/DMRHBR/B2tPw+mhogsCcG5mzuiRkUgg+LvoYvv0NqycnV2k8bisLKnFr1S2QJAmCIDDr1iwM+mkQAMA5zBkzzs7g9BgVmAvweNtjpF5NZTsUVog7EYeNXTci5mAM26EwgqhahOMLjuPWn/ot4oZOCUXw+GBGj6nJ/ZZJYuuc493dKUsdgDnRb+qxqfggQ7VFTm0TczgGR+ccVWgRpk9s2kQtjpiaAqF1XQboij8+n6r6o/v6VchZdxWLGybqje09aczNG47DlsUnF1A0dve+shfnv1QsrNFYOFnAu6c349b/T3KeAABCnagvRT/ffgCgdF+/8pxyvXPvSDyfiCvfX9FZWxBdUFZThhNxVB+zaW2nAQAmhUyCr40vcityse7uOo2OX55bjvv/3GfNwlpXsHnP9bDywIIuC/Be5/dU2k/tSj8CODrnKO5vvK/aflogdEoo3nnyjlIiXNrNNCScTZD5XawsqMSdtXfk9ho1MjGCXSu7F6oEX0ZKUksQtSsKBc81fJhjEW2O26Nzj3Km/64sLi27hBUeK1SulNUX6q09PRrKZ9UR/RojdBPi3dh30e3Dbk1eN7UxlVmtpgueHXiGG7/eYO38yhI4MhBLCpcgaEyQUn1hM0opRzM7QUMfwLw83d5nDVM10CO4bMEijft/38f6juvl+gE3p9O7nfB+yvvw6eOjtbhij8RiQ+cNL5QpGyI8Pg9vXnnT4Pox0JhYmSD2aCxyHnPPqoMkSZSUlKg9bs1szQzCko5JKvIq8HTvU4N9WJOFkakRa9XJsUdisXvSbpRlay8Rg0s0Hrdnlp7B8fnH67Pb9K1/KI/Pw+LcxRi/Xb59j6HiGu6Kvt/0hVcPL7ZDYQS+MR8P/n6A2EMvT09iZdH0fssUtLUnXeUHMCf6GVsYc+6ZIGJ2BF49+Sps/W3ZDoURrl4F9u2jfn///YZ+jA4OwMKFwEcfUWIgLdaVybkt5uYCEgkgEDR8BwAg/3k+fnH9BRe/uQinulbbbFh8Zt7LxO3Vt7WaZKkMisZu/Ol4ZNxW3ta9urSasV5YNGKJGE9zqfJPWvSje8GdSzqn1DHWd1qPXeN3MRqXtum+uDvmRc8zqJ7wT3OfolZSCxdLFwQ7UsknAr4A/+vzP3zY9UNMDpms0fFL00tx6M1DiDnEjWSj3RN3Y+e4nYwflyv3XFUItA/E9vHbsWXcFpX24/F5mH5mOvp83Uc7gTXiyY4nOLHwBCOJ6b59ffHus3fRaZ70flfVJdU4Ovconux8IvMYuU9ztZYkX1Neg0vLLiF6n35UzrlFuuGz6s/Q6V396R/WHG2O20G/DMKgXwYxflwmMbM1g5WHFUysDLPXfXo6UGbyHL/wHDDz4ExISIlaol9lYSXOfHwG0fujwRfwYd/a/oXWEqWZpUi6mARRlYjBv0B5hq8ZjrmP50LoJmTl/MoiMBfUf9/2vrIXXxFfoSJfdsYgLfpZkg2iX1aWbu+zLaIfy0gkmk1irv50Fes7rWe8ubksfPr4oP/y/ir1nyEIAtae1lr15vXs5ok+X/eBW4Sb4o1b4DQ8Pg9zH83F6L9Hsx3KC0gkEiQkJKg8bkmSROzRWBQlFWknMD0m62EWdk/YjedHDV+wB4D82HzcWnWLVeuCgvgCPN3zFAVxL4fQ2njcDvppEEZuGAm//n5sh6U2hjqxUQbHYEf0+qyXwfSkJAgC85/Px9RjU9kORW1iDsXgN5/fGLfJUvd+yzTN+/kBzIl+5TnlSDyXKHeyqGusPKzgP8ifM/2rNKG4GPjtN+r3sWOB7s0caPv1A7rVJTrTop88e8/G/fwau6iZ2ZnBxtcGFo4WcHSkXmND9Is7GYdj846hJK1E9ydvhKKx+1HeR5h2bJrSx9s5Zid+cvyJqfAAAIlFiagUVcLUyBR+ttTzQB+fPujt3RvTwqZBLFFsvx45J1LvepAbmRrBIciB0/1yVCU6lxIY2jg0dT2a0W4Gfhr0E1yFUspyVcCutR2mHJqCdq+10+g4TFFTVoOaMuUdl5SFzXvu6fjTuJNxB1WiKpX2sza1xpTQKejl3Uvlc/r194NjG0eV91OVuONxuPn7TRiZyh5zKVdScOzdY0o9C9j42IBnJH0J19rbGlOPTUWvz2T/e2wetBnbhm9THLga8Pg8nP/sPOdsy+XBN+brteuQNsdt4MhA+A/yZ/y4TNLxnY6YdWMWLBwt2A5FK6SlAXlWZ1BOFmDbk23gEbx60S87W3EfahpSQuLqD1cRdzwOtZW1KEkreaFVxu3Vt/Ffn/9Ys/8mCAJOIfqVkGTbyhaBowJhZmcmc5u8CqpYSlDTcL/RtehnOE98LymiShFK00thaqObagWPLh4qeYDXVtYi+VIy3CLctOI9TyN0E6L35721dnwuEXskFnkxeYiYHWGwi79cz/BQlfLscmwfsR0RcyIwYs0ItsPhFK7hrhizaYzBVO4o4tnBZzjz0Rk4BDmw9j0PnxmOiLciDGJRV1VMrEwQPjOc7TA0ojSzFBm3M+DV0wtmtrIfMg0NOotVX5rLK4sqSVRchOARMLMzM8jnEZJsEP0CAxteZ0r0iz0Si0MzD2Hq0aloPay1ZgdjiOqSahgLVevdzVXi4oCqKsqK87XX5G+rjL1nY9GvMeb25ph5bSYA4M5a6rU85Q1RGKPdjHbw6OwBhyAH3Z9ci/j084FDG2b/JtraM9gxGHwe5bpgbWqNC69fUPoYPT7uwWhMuiDrYRasvawN6tkhOk+66McUxhbGCBwZqHhDHTHtuPKCub7wyt5XkF+Zj/tv30d7l/Y6OSdJkhDXiLUugI/+ZzT6fNUHJkLZz0iFiYW4/edtePX0QujkUKnbxByKQW50LiLfjpS57kcQBFoPlf8s0Wl+J62NfyNTI7x9/20I3fVjHSf7UTZqK2rhFukmU0htoQU2SU8HKo2TAQBvhb8FALC2BqysgJISIDUVaNVK8XHMbM3wbuy7ELoKkXI5BVsGb8GIv0Yg4q2I+m1aD2sNC0cLmNtrb81eFhV5Fch+lA3nts5a1QyYZsD3AxRuQ4t+ZIV9w2s6niO0XN30nN5f9Mai9EVyHyTYJPtRNrYO2Yo7a+/o5HwkSeqVLYU6PNn+BKc/PM05SyimKUwoxM6xOxF7RP9tz4zMjDByw0iETQ1jOxTOYWZrhnbT28HW1zCsxBQRPiscrxx+BZ7dPFmLwdjS+KUT/GL+isGV765AXKM4e5/rPNv/DDtG71DJHs0QSDidgF9cfsG9DffYDoVRKgsqkXQhCVVFqmW4c4WAEQF4+97brF7TtEVGBlX5ZWwMeHs3vE6LfoWFlDCoLp7dPTF01VDGBQ1N+M37N2wdupXtMBihqIj66epK9e6ThzL2nrJEv8Y41H2UbIh+Vu5W8O3nC2NLbvdqSrmaIrfnVHN6fdoLw1YNYzSGMKcwrBi0Au9EvsPocbmMuFaMdeHrcHjWYbZDYZQPun6AszPOYk7kHKnv//fgP7x24DU8yn6k0XnEtfr//MhFiquKkV+ZDwDwt1W9quhMwhmsu7sOKcWqNbj6t/e/+N33d5XPpyoEj4CNj43cbYInBOOdp+/IFPwA4NGWRzi79CwIvuL1n5L0ElSXSi8B6rGkR5OFfqZxae+iN1VXl7+7jI1dNyrVl+tl5NrP17DcZjnyY/PZDkUmN36/gbvr7rIdhlYQiajnTlr087ZpmIioavGZmkbgyDV7iHjGsPKwQucFneHc1rnJNp5dPdH5vc6wcNL9+E25moJN/Tch9qj+r/s2J7+CGj+i0gbRLztbt+v4LaJfCyqRcScDG7psULps39rLGkNXDUXAyADFG2vIg/8e4BfXX5D9KFvr52KTIb8PwZvX3uT8pF5TjEyNEHciDmk30tgOpQmmpqpX1ZpamyJ8Zji8e3or3vglpaa85qV46DazNUPAiADWrUTyYvL0pueCpkhEEmSezUT0vmiDyOT0G+CHURtHwTFE+7ZEXMLWzxbGQmO4Rmhm1cU1onZF4b++/yHtJrfudVxAnfstk9Aij7s7YNSoGMDGhrJ3FIupTFt1cQh0QKd5nTiT9CIRSxA0Lgi+/X3ZDoURCuscimyV+OdV1d6zOQ83PcSFry7U23vm5iofJ1PUVtZyJvFR3tg9NPMQjs49qsNoXsTfzh8Luy7EzPCZTV4nSRKZpZl4nP1Y4TEebn6ITf03oSSdXTtVZRHXiNHnqz4ImRzCdiiM4mjhiH6+/RDmLD2xctfTXdj0cBMuJ19W+xzrItZhbdu1au/PFFVFVbi16hYy72dq5fhs3HNj86lFXhdLFwhNVK8Q+/z853j7yNu4m6Hawr9vP1+0Hq7dCvvkS8lIupAEUiL/uiwwEyi0Gh3992jMujlLYaJ/1O4o/OrxK2IPs7N4XlNeg8IEduwBVSV8VjgG/zZYrvWqPqCtcWvpYsn51kk3fjVc0S8zk+ojXWVWJ/pZe+PjMx8jYl0EKlzOAAASExUfRyIBli8HDmwtx9bfc+EY7Ighvw1Ryb1P2ziHOWPoH0MNMoE01CkUg3yHQlBEPXsRBOVCokv0f/VLz+ErSj2Vg6hahCs/XEHq9VQGI5IPKSFRml4qM3uoOUJXITrN6wSXdnLSYhnC1MYU5vbm4Bur/2+qD5g7mMOzq+FdEJsjdBNiQdIC9Pu2HwAg9mgs8p+zm2nE5/MRFBSk8rjlyiIMV7mz9g6WWy1Hxh3DrlyqLKhEcYpu+q8q4uzSs9g9aTdqK2oVb6znCEwEmB81H1MOTDGICmn7AHt0eLMDrNwNo6+dsti1ssO86Hlw7WBYop93b28M/nUw7APsFW/MQW6vuY2765mfcKt7v2USWrhxaFaIZ2REWesAQD53E6BVhsfnYfTG0ei+uLvijfUApkW/zLp1dmmiX9TOKFz5/grs7annPTa+F9uGb2O89506KBq7fb/pi97/U64lQ015Dfa+shePtmhWpaUs5xLPwW2FGybunqhw29L0UmTczUBlfqUOItMcYwtj9PqsF0ImGZbop4hI10gAwJ1M9V2HvPt4w6evD0MRqU9hQiGOzz+O58eY74PO1j03Jp/y0A60V89C1cmC6gGVU65aI9U+/+uDUetHqXVOZTn36TlsHboV1SWK181IksTz489lumMZWxrDvaO7wuO4d3RH+Oxw2PjavPBe1O4o/NPzH63Otw/POoyV/ishqhZp7RxM4TfAD10WdGE7DI3Q5rht+2pbzDg7g9Pzk9fOv4ZxW8exHYZWSE+nflabNlT6PS94jnuZ9yCyjQIAPFfiVnDxImUD6vtgP9K+WA+RjKFZnFqMte3W4uqPV5kIXyVs/WzR6d1OsG/N3e+auszvPB9/djsGt8IJsLEB7O1136pEv9MaDABNmq4WJxfj7Mdn0X1Jd52JQO6d3LEwdaHS25MSUmeLrAHDAxAwPMAgKjlkUV1ajfLsctj62xpEvxVFWDpTvY6qiqpw+sPTaD2iNQb9NIi1eCQSCQoLC2FrawseT/nv2YHXDiDzXibevv82+ALDFqXVwTHEEYGjAw167ALAkx1PcGzeMUw7Pg2thihhwK5FOs3vhLCpYQYhgsmjIq8CpnamKCopgq0nN6ppmIIkyZfiPnBy0Um4d3JH6JRQg7x+OrZxVJjhzWWu/3IdxhbGiJjNrF2UuvdbJqFFP0cpH4+9PVBc3CAsqYO4Rox1Eevg09cHQ1cOVf9ALUiFSdGPJOVX+g1dNRQ8Ix4q64r48/KofXR5ifbu7c2JxTlFYzdkovKiU0VeBZ7seAJLN0u0fbUtI/HViGuw5+kehDqFIswprMl9NMCecqaJL4xHrbgWAr5sV4buS7rrZV8/QyKpKAl/3fkLHVw7YFLIJKnbRLpRot/t9Ntqn2fwL4PV3pdJ7FrbYca5GbDxtmH82Gzdc2PyNBT9zNUT/XTByPUjkfUgS2YPvsYQBIELX1xAXkwe2r/evkn1WUkaZdfpEOigcN5m42ODketGSn2vJK0EWQ+ztNpTPmBkAKy8rCCplQDc7D5kUHDhWZlNuOKUoQ3S0gAJUYMKHpVx5m3tjQA76hml3JRS++LiKNcRWZqvSARs20b9XuAehjJbTxxb8QwV1x6g/3f94RjcMMExtjRGTVkNJGLDd97SNRl1eRZubgCPB2Rk6LYg5OW7MnAMTSqArDyt8Oa1NxE+K5zBiJhDIpbgZ+efcWz+MZ2cj2fEqxcNKvIqdHJOXZNwJgF/tP4DD/55wHYoOkVgIYBHFw94dGa3DJ0kSaSmpqo8bs3szGDuYG6QC9ZM4N3TG5P3TYZbJLctJDTFua0zwt8Kh1tH9v9O376+CJ4QrPeWJvLIepCF33x+w52/7qg1brnMtuHb8Ff7v9gOQ+uU55bj9p+3WbMpakEx009Nx9gtYxk/rrr3Wyah+7JJE/3ovn4FBeofn2/MB8EjOJPwknAmAQdeP4DcaBa8KbUAk6JfWRlQUTe1cHZ+8X1bX1tYe1rDwYEAQQC1tZpZv6pDny/7YMTaEbo9qRSYHLs23jb4tOpT9P26LwORUcTkxWDavmno+U/PF95zt3KHhcACIokICYUJco+jb0k3136+hv/6/oeybDmNK/WM2+m3sfzqcqy4vkLmNrToF50XjbIa/f7bTYQm8O3rC1s/5he62brn1lf6Oagn+jlbUhdkVUW/mMMxODTrkFbHg0OQA0KnyO7T15xhq4dh7qO5L8zN7m24h9XBq5F5TzNb164Lu2JJ4RKtin5hU8Mw8IeBnG9DU5Zdhl89f2WlqolJtDluy3PLcf6L84g7Gcf4sZlAXCtGSVqJXlSVqkN6OiDilyDEvD/CnMLgYO6A1vaUJXFGdSzMzYHqaiA5WfYxzpyhEtZsbIDOb7VDZkBv3D9fhNgjsS8495nZmuG9+PfQc+mLz0baZtuIbdg0YJPOz6sLasQ19U4hrq7S5xDahhuzzBbUQmAmgGdXT9i1stPZOcU1Yjza8gjJl+RcXeqoKqqCa7grhK7ae7CQxu3Vt7HCYwVynnAv40tTbH1t0XlBZ3j18GI7FJ3CF/Ax+p/RCJ4QzHYoajHktyF4/cLrbIfRAst49fDCyL9GwtzenO1Q6jFke08jUyM4BDkYnB0kAFj7WMOulZ1BCZnSsHC0wAeZH6Dfd/3YDkWr7Bq/C//2/pftMNTC1s8WzmEszGB0gLxKP1pI0tTGcc7DORi8ghtVJJn3M/Hwv4dKWZHpA0yKfml1LTft7AATKdUL4loxipKLICqrgo0N9Robff30gf3T92NdxDqltzcyMYKxBXMLyE9yngCg+qw0F+54BK9efHiW90zucapLq/Hs4DNkP9aPXvJlWWXIeZIDU2t2e6UySXQe1Zu6jWMbmdu4Cl3hLnSHhJTgfuZ9tc4Tfyoeh98+zHr/RnGNWGF/OH2DFv3oKltVqbf3rFBt3Sfrfhbub7yP8mw5ns5qIhFLEHcyTuVe9e4d3WHjY/PC6779fdH1w65wbqvcs1bskVisi1gnVSTk8VuWfwGgpqwGli6WMDIz3ORXTamtqMWlby4h/mQ826FIJT82H796/orL36nfr5XLZGUBxiIHrOp0Go/mPgJBEPXXyecFz9G6riVprIy8WJEI2LGD+n3SJGBUnZvxXUEXzMv+Au6dFNsF6wqBuYDziQLqQJIkhN8L8dozISoF6XBzk+4Wom1arvp6TGVhJcQ1Yp2ek5SQ2D99P+6tv6dwW3N7c7x68lX0/ES32QLundzhGu5qkKXJLu1dMOS3IZyw72mhBSaJ2h2FLYO3oKpIx51tX2K2DNmC1aGr2Q6DcSoLqP46DkEOmH17NicqK5lm+J/DMWnvJL2rNFAHMzszrVhZcQkjMyO9rLqtrahFaUapQT5vAQ2Vfs17+gGUvSegmb0n1+j2YTcsLV0KtwjDuGYyKfolJlI/fX2lvx9/Kh6/+/yO6H3R9d8X+vujCyRiCQ68dgAP/n2gu5OqiZG5kdKLOyXpJUi7kaZ0L3llqBf9HKVX4NA2g7QYIYvy7HLsHLNTZ/0GNWXQz4OwOHexXt5rZFEv+jnIFv0AoKN7RwDAnQz1+vrlPMnBvXX3WO/LfeGrC/jG+BvWe9wzyU8Df8Ivg35BhKt6FuG06Jddppr43mVhFywpXAKnUCe1ziuP+FPx2DpkK679fE3lfUmSxNO9T3H91+v1r3n39MagnwaBb6ykYxABlGaUojSztP6lhLMJuPrjVZTnMi9yNiYnKgfbRmzDswPykybYxs7fDrNvz0bn+Z3ZDoWzWLlbYe6Tuej9hXI9eHWNidAEnd7rBM9uumlzpWtoJxH7Rsu+re0opS+lOAV+AdRzkSzR78kTKjHRygoYMgQQxSejw8N/YZmbiLNnpbsVPDvwDHfXMd+nXRETd03ElANTdH5ebVNWU4YacQ2qUQaB2KZF9GtBdY7OOYpl5st0WtLMN+Fj8v7J6PphV52dU1XcIt3w5tU34dKOhRHVgtZ4svMJ/u3zL4pT2Z1wCYWqVa6m30rH2U/PoiBeAw+wl4DS9FIkX0pGfqzhTGQb82jLI6xttxYZd7XXPF1V3CLd4N3TG+Ja3SaPaJObK2/iN+/f6rNb6QdaVcdtC+xTmFCIu+vvorKwku1QtM64LePw6slX2Q5DZRLPJ2KF+wqtCQ1sjluSlF3pR5Jkvb2nppV+z48/x9WfuGEvRRAEjC2NOWM3qgnV1Q12nLoQ/RyDHdFlYRc4tHGo/77ostKvqqgKDzc9ROr1VN2dVA7yxu7Iv0bi9YuvK3Wcp3ueYmPXjci6n8VQZMCT3IZKP2kEOQQBUFzpJ3QXYsx/Y9B2GjO9BltQnehc5US/SNdIECCQXpqu1nnCZ4Vjcd5i1ttMOAQ5IHBkICwcLbRyfDbuuYP8B2FR10VwFarnylFf6aeivaeJ0ASmNqZa6W1u39oe/Zf3R+grylt70pASEpe/vYwr319BbUWtWo4erYe2xqKMRQgY3lA9+XjbY5xZcobqtadFJCIJEs4koCipSKvnaaEBbY1bnhEPTiFOSvWkZANrL2sM/X0oWg1uxXYojEOSlOhHQtJE9HOycILQWAgJKYG5O2VBHiMjP+laXc5Bly6AQACIqkQwKcqBMD8R5zYkoKr8xfWfmytv4szHZ5j+c15a8iupSSJPYgK+xByuruyIfoaT6qWn8GV13VQCr55eMBYaw8hEdx8jQRAIGhOk1LaXvr0EgbkAXRfpXiA0xOqHgvgCbB+xHT0/6/lSTjArciuQdT8LpRmlsPa0ZiUGPp8Pf39/lfZJOJuAK99dQeDIQNj5686KV9+IeCsCHed1NNi+hzXlNSjPLYelsyXbodTT71vDsEwsiCuot7m2D7SHV0+vJnYt6oxbtqmpoR7Wu3QBTKXMtUozSnFr1S149/Y2yMkOADzc9BAXv7oIWz9b+PX3YzucFqQgdBMi8p1IuIYzb6HL9rgtLqascQiioX8fAJxNOIsxO8egrXUvCEx/QkGBZrbjj7c8xuNtj9F5fmfWK3CyHmaBlJAGYYlcVET9NDYGzMwUb6+p6Gfra1tv0+pAaUo6rfQzszPD0rKlIMXsW/8xOXa9e3lj4M8DGXM4qRXX4moKJbK3d2kvdRtlRT+BmQDtZrRjJC5dcG/DPdgH2sO7pzfboTCCWCKur8aUZ+8JAPM6zcOCLgtgaazeM7iJlRRPXxZoN70d2k3XzneO7XuuurR1bosd43fAw0o1QbamrAYF8QUQugph4cSsiGrXyg49lvRQa18en4exW8bC3MEcAnMBzn95HrdX3cbk/ZPh3Uu5sStNyBz2xzB0eKODVvv5AVT/+k8rP+X8Wlzs0Vjkx+YjYnaEXtsKanvcVhZWorq4WqrtbAvao7ISqKoCHvrMRLstR/F9/+8xM3wmCIJAiFMICisLYedKFUKkplJJbuaNusdIJMD1umLhbt2on/6D/LE47yN8EnoIwtObcfLIEoye3HTdbdDPg1S2JdaUgrgCRO2KQtCYIDgGS+mnoMfkV1Cin7HIAQQIuLpSazy6vj7qfyqnniORqD+oOr3bCaM2jGIwGuaQiCS4v/E+ovdFs3L+irwKbOq/Cee/OM/K+bVBWVaZzu1cuUTknEgsKVrCapalRCJBVlaWSuO266KumPt4Llzat1SeykNgLjBYwQ8AIt+OxKL0RbDysGI7FIPi8fbHWBW0ClG7ogAArQa3wrRj0+DYpuGhUZ1xyzY7dwK//AJs3y79fVG1CFe+v4K4E9xsrs4EXRZ2wdjNY+HT24ftUDSiupoSj+RRkl6CK8uvIPUaN6p0lMW1gyuG/zlcKyIR2+OWrtKytQWM6rQ4CSnBolOLUFZThmu5x3AppC3OkJ9pdJ5en/fC7DuzOVFdd3z+cWwZtIXtMBihsbWnMnNryzotQJroJ5EAyXWtzP2UyD9gw96TIAgYWxhzQpxQNHafH3+Oaz9fU6o3mWsHV3T7oBssXZhJmDoVfwr5lflwtnBGV0/pSakRrhH4oOsHmNdxHiPn5AKiahEOzz6MO6vVs7fkIsnFyagSVcGEbwJfGxlqfB02pjZqC34A1UsvJyoHJWns9vTTJmzcc2+n38auqF1IKExQ+xgO5g6YHDoZ3b26q7Rf2o00/NX+L8bXqsS1Yo0tz51CnOqTRJ3bOsPa21rlxIfM+5m48L8LqCqm2mYIzAXw6uGlUVzKQBAE5wU/AIjaEYVTi04B3A9VLvLG7bnEc5iyZ4rKVbCN2TxwM2d7jj/Z8QQ7x+00yKpS2tqzxiwJuRW5MDFqeLa79uY1PHv3GQYFd4GjI1UVGNdsOSA6mkp+s7AA2jXKEzE2BjrObo+ksOHYd8QYNTVN93MNd9V5r7+MOxk49+k55DxR/3vKVfIqqImAQGQPa2tKmLW2BoyNdZugx/4M8yVHnZJ9ttnQeQPWtF0jdxueEQ9zH8/FyPUjdRRVU8zszFAQX2BQtmBe3b3wXvx7CJsaxnYorMAz4rH+EEmSJLKyslQat0YmRnAKdWI9g18fyHmSg3sbFPcL1VfY/v5K48zHZ3Bq8Sm2w1Abn94+8OnjI1dUV2fcss3Nm9TPezKGg7WXNd6JegcDvh+gu6B0jKm1Kdq+2pYTYoi65OYCb7wBfPIJNSmTRWV+Jc4uPYu4k4Yr4qoK2+NWmrVnSnEK8ivyITQWYqjvaJCEGI9tluFexgO1z+MQ5AC3CDdOfM87ze+Evt/0ZTsMRlClnx/QkCFdUwPU1jZ9LzOTyrg2Ngbc5LQ7PDLnCA68doAV0a8ivwIZdzLqF3jZRNHYfbLtCU4vPs1KIuO+6H0AgMkhk2HEk/5c7m/nj58H/YxpbacpPN7WoVsVzom5AMEjMP30dE6351AV2tozwD4AfJ52kwaLU4uxJnQNbq26pdXzKOLU4lNas4Nm45676eEmTN4zGX/d+Utn56SxD7BHn6/7wC2S2R62sYdj8aPdj3h+7DkjxwseH4y3772tcuJD8sVkXPzqIjJuZyAvJg+50bk6+2yTLyfj+XFm/n5t0W9ZP8y8MRPGFvpb5QfIH7dT907FzqiduJB0Qe3jt5vRDh1mdtAgQu2R9ywPMQdjDLKvON06oNKEyjjztm6o8m28nhRQ5+DbvK8fbe3ZqVND4iIAPDv4DB1Ca4GISBQU8XBKyhKQqEqk09YvrYe3xuw7s+HbX37yjj5C23sai+zr5w8EATg7t4h+LShBQVwBdo7dycoN1a2jGzy6KK62MrY0blJtoUsIHoH34t/DsD+GsXJ+bcJF4UBXxJ+OR8whGcbVHISUkEi/nY6ashrFG7eA6yuu4/Dsw6jIr2A7FEa5/899nPv8HKpLqtkO5QWSzicheg87FdlMIHQTYsaZGYxZf3GBvLyGqpKkJKBESmI5j8+DY7CjwSYTPNn5xCCuAzt3AqWlVMalLAEXoGxpZ96YiS7vd9FdcAxwctFJHJx5kO0wtAIt2DQW/XxsfBD3XhxOTz+Nw9MOwK1gEgDgj+trlTpmbCw1phtDkiRqymsgqtJdf25ZhEwMQeScSLbDYAR1RT+goRcgDW3t6e0N8OTMnHOjcpHzJIcV0S/hdALWd1yP+FPxujupmvT6vBdm3pgJnkDxMsS+V/dh04BNjJ179fDVODD5AOZ2nMvI8Wx8beAQ6MDIsbQJX8CH3wA/uEUwK3CwydDWQ5G0IAmbxir3/fjvwX/o/nd3/HbjN5XPZeliiV5f9ILfQHatxh/8/QDPj3JbTFEF2p410CFQo+OciDuBdXfXIatM+d6f1l7W6P15b8ZFP74xH05hTrD1V/LmoyVCJoVg1q1Z8O7ljUdbHmF18GqUZZYp3K+0uhRLzyzFmttr1BYJT7x3AsfmHVNrX11h7WXNeo9ObUNXh6lqfduYzu91Ru8vejMVEqP0+V8ffF77OWz92B1r2oDq5ydGOZ9ygPG2kW7tG1h36Wzc148kG0S/7s0KoA/NPIRrP1zGxInU/+/ejSbVfrf+vIVlZsuQdj2NiT9DKUyEJnCLcIO5vbnijfWMBntPezg7N7zevFe8tjHM1aKXgKKkIsQcjkHrEa11fu5hq+QLaRe/vgi+MR9dP+jKql0fj284mrZELMG5z86h9dDWSvu5GyLH5x8HAASO0mxyoCuKU4qxodMGdF7QGUN+G8J2OJwncm4k2oxvA4G5gO1QGCVqRxTSbqah9+fce2iefma63vYyKEwohJWnlcHZwjYXh548afDjb0x1STXyYvLgGu5qUPe7/Nh87J2yF8ETgjFx90S2w1Gb7GzgTKNe6AcOABER0rc1MjHSy8WH9FvpqCwwHEeFxtCVfg7N1vNNjUzR2aMzACCsZi4yyD0oKJXRCK4RKSnA4sWUVeSwYcCMGZTtTvTeaOyeuBvjt49H6JRQpv+MlxZVRT8ejxL+Kiooi0/rRq2jFfXzo3n90usgCKJe7MvPpz5veUIhUzi3dUb/7/vrhZW8Kkk6EpEEklrmsvhNjEwwOmi0wu0KKgvwNPcpHM0d5QoSw1cPZyw2bSIRS0Dw9MN2T1l4BA/eNt7whnLz4qyyLFxLvQYva9UtDo0tjNH3K/aroBemLkRtZa3iDfUEWvQLsA/Q6DiLTy/Gk5wn8LXxhYslu9fAgBEBCBih2d/DBEI3YX3/Pt++vsi6l6VUP7+koiQsv7oc9mb2aidH9P22L8Bxc5XKwkoIzAUwMjHM5fBqUTVSiynByN9W/3p1Kou0/pWGQH4+UCXIhIQQgU/w4SZsSE6IL4jHK3tfQaWoEjt7PwYAPHgApKcD7u6UAJiXB5iaAh2aFWmO/ns0js07BtE3G2EfOBP5+cCdOw3rDI7BjgibFgYTa91ZxZdmlsLUxhQCM8Na/wMowT3UZBhEFR2bzCcb94rXBYazSqSnqPvw7TfAD59VfYa2r7ZlOCLlyY3ORVFyEQAg8XwiSAkJiUiCJ9ufIGpXFOtWRSRJ4ubKm7j28zVW42CC3Ke5uLr8KmNWEfrK4F8HY8RfI1g7P0EQsLOzU3rcGpkaoe83fREwkv2Hf33AvaM7AoYHGNxNf+qxqZh1cxb4xtwTp0yEJnq7CLRpwCas77he4Xaqjlu2uXuX+mlcp8U+eSJ9u/NfnseGThtQnFKsm8B0hI2vDcZtG4fuS1Trz8I1du4ExGLA35+y8njwoEE8kIa4RoyCuAKdxccEb155E3MfMVMx0xy2x23zSr+0kjSIJU3tbsKEvdH/UQoWt9qs8HinTlECEAAcOwbMm0dVgNoH2KPdjHas93utLq3GqqBVuLTsEqtxMIWqoh/QUO3XvK8fPW4V9fOjv6t0H0GxmOqpogscgx3R4+MesG/NftW7orErEUtQW1mrVE+/CTsm4PWLrzMcoWK+vvg1ev7TE+vvKX7G0Ace/vcQy0yXIeGM+r3T9J22ztSayaPsRyxHoj4Cc4HWqiF0fc+tqK1ASnEKACDQXrNkXicLJwBQqXeZuEaMf3r+g3OfndPo3PqAbz9fTD06ValtU0sooYiuDqMrVVQhYDg3hE95/NXhL/zT4x+2w9AYWeM2sSgRJEiYGZkhOi8ax56rV3kZczgGmwdtRk4U9/qtJV1M0rte6MpSUNBg7elp7dnEjtzG1Aa3M27jSc4TeLeqQGgoZUG/fDnlFPTDD9R2nTo1rCXQBI4KhN8AP5hamaBXL+q1q40co337+mLclnFwaae75Iltw7fhzzZ/6ux8umRsm7GYIj6KVlkfNRH9hIrzLxilRfRjGZ4G6Z88Ix4r2SlRu6Nw4v0TOLnwJP5o9QcebXmELYO2IPlSMnhGPMx5OAeT901mfYGVIAg8+OcBbvx2Q696OUnDMdgRc5/MReRcw7BdUpfWQ1vDp7cPa+fn8Xjw8vJSetxaulii12e94NefXTsYfYIkSYPKYgWoqmOu2j9VFVch/nQ8ChML2Q5FJSRiCdrNaId2M9op3FbVccsmIhElDgHAyLqWuI8fS982YEQA+n7b1+AqY/kCPsJeCWPcckmXZGUBZ89Sv8+ZA/ToQf1+4IDsfQ7NOoQ/Wv+B6lLu2QDLQ1sJXmyP28aVfiRJImhVECy+s0BCYcOiuZMjAbNad4U2jrW1wLm6dcUpUwBXVyqL95NPgCe5zhjz3xh49VC9+oRJastrwePzlBJi9AF1RD/LupZJskQ/RZV+xSnFeLz9MSqyS2Ffp73R36OXCUVj98ySM/jO/DsUp+ouYSWxMBFt/myDz899DpFEsZVukEMQAOBZ3jO526XdSMOZj89wPvnG0sUS/oP8lar00QfOJpzFkC1D8M995Rft27lQz4sxeTGoEqne+/LQrEPYMXqHyvsxRXVJNdJva6+6Xtf33Of5VCKzraktHMw1myOpI/rxBDwUJhSiIo85K/mkC0nYN20fcp5wTyBRFro6zFxgjom7JyJsTRjKahTbguobwRODETQ2iO0wNEbWuI0voKy+K0WV6PtfX3x0+iO1jl+eU46062koz1HsaKFrjr1zDIdmHmI7DK1QUABUGr/Yzw8A7M3tYWdGlYrFF8Zh8WLKnSIpCXjvPSpp0d0dmDlT+rFH/T0K005Mq7f+vHWrqcWnrmkzvg3av9GevQC0DN2fsbHoZ2Wl27kt91fADByJRD3LkrgTcci4k8FwNMqReDYRN3+/ibavtkWfr/ogeGIwAkYEoDSjFADlZW7jY8NKbM0Zv3085kXPY12A1BQenwenECfYeNuwHQrrkBIS4hrdNZdtjEQiQUpKitrjtgX5kBISK9xXYPfE3WyHwhj3/7mPrAfK95jQNTmPc7Bl0BZE79Ovvn48Pg99/tcHXRd1VbitPo3bmBjKXs7KChgzhnpNVl8/v/5+6PVpL1g6W+oyRK2S9TBLpwvB2uLECaqqKzwcCAoCxo6lXr90iZrISSNgRAC6Le7GqJWdNqksrET0/mitJQywPW5pscbREUgvTUd5bTnEpBieVp712zTu3ZZQmNBEEGzMjRtUb0d7e+CVV4CVKykrHZEI+O034OhRLf8xSmDpYol3ot7hpA21Oqgj+llYUD8bi36lpQ1Vnz4+8vdPupiEfVP3If1Wer3ol696kYRanP/iPDZ23YiacvZ7SCsau26Rbmg7va3CnrTVpdW4+uNVZN7L1Cie4qpijNw+Es/ynuFc0rkmGfOyaG1Htc+IK4iTu13Wgyxc/eEq8mJ02MBRDVoPa41XDr8Cx2AdN5LREseeH8PJ+JO4mnpV8cZ1uFq6wt7MHmJSjKe5T1U+Z3l2OUozS1Xejyky72ViQ6cNeLxNRiaYhuj6ntu4n5+m6zRO5qqLfgRBYFH6IoxYy5yDUMadDDze9livk2foSr82Dm1wL/MeMssysfXRVpWO8XDzQ/xo/yOSLydrI0RGGPTTIPT8pCfbYWiMrHEbX0iJfv18+0HAEyAqN0phEos0OrzZAUtLl8K3r4KsJxbot6wf+nzdh+0wtEJBAWAktkGk3QB08Xix3zv9jBKbHws7O+CjjyiHCYkE8PQEvv9euoXkuc/PYZnpMpRmlCIggJrjVFU1uAxVl1Tj8FuHcf+f+9r885rQ69Ne6PNlH52dT5eIJKIm80kaMzPdzm1bRD+WUbcC7dCsQzgy5wjD0ShH32/6YmHaQoS+Eoqen/SEkYkRJu2bhJDJIazEIw+HIAeYCHXnSawt8mPz9S77XxukXk/Fdxbf4e66u6ycnyRJFBQUKD1ut4/ajj1T9mg5KsOB4BHwH+gPlw7c70mjDJWFlTjy1hFOW8c4hjhi2J/D0GpIK7ZD0Rqqjls2ofv5degA2NgA3nXJfbIsPg2NkwtPYlXAKr2v9n1e58RNV/i1bg0EB1Miz+nT0vcJmRSCgT8OhJmdmW6C1JC86DzsGrcLzw6ovoigDGyOW5GoQTRydGyo9vG39YeA31BZS4t+e7N+RKuVrbD2zlqpxzt1ivo5YADV383UFPj44zoxmCRx+cODBmFFzyWYsvekq/ycnRvel4VPbx9M3D0R7p3d6yf3uqr0qy6pRnFqMSfs0RWN3dApoRi7aazChJW8Z3k4s+QMnh9Xv7VBlagKE3ZPQFRuFFwtXbF9/Hal9vO2oW6+qSWpcq9BwROD8W7su/Du+fL2W2eDUwnURXWQ/yCl9yEIQiOLz1cOv4LZt2arvB9TWHtbo//3/eHZ3VPxxmqg63sufV/V1NoTAJwtnQGoJvppg24fdsPi3MVwCnViNQ5NoEW/VnatMDWUsgS9mX5TpWNYOFrAKcyJ9VY/LwOyxi2dsBLhGoH+fv0BAHuf7lX5+FwunAgaE4SQidxbf2aC/HzAuXgYNg88jeUDlr/wfmt7SvSjK6bbtgU++ICaZ3z3nexnX1s/W4hrxIjaFQWCQH2135Ur1E+eEQ/31t9D0vkkpv+kl5JO6ztju68l8oTnm1T6WVjodm7bciXWU4b9OQy9v2AnG9fcwRziajF4/IavD0EQTf6fK4hrxciJyqnvPaivbBqwCf/2+pftMFjH2tMa3r28YemiH5UtNWU1qK3Q78VrXTPmvzHo900/tsNQm+qSalz/9TpIkoSJlQlmnJuBXp/1YjssmZjZmqHjOx3hFKJfE9TNAzezlviiTehMu4gI6mdYGPVTmsUnSZLYMmQLDs0yHGuTzu91Rr9l/TixcK0uJAnEUwm28PdveH3IEOrn6dPUNvqOXSs7jN0y1iATBgoKqM/IyIiyzKEXJ2nLPxqnussmUeoJEiROxZ964VjZ2Q2WvQMHNrxOEMD48dQvgrhoxB5nt9dWzpMc3F59GyVpUsqK9QySbOilp6m9p7L9/ADA2ssawROCIXQVNqkC1QVDfhuCRWmLQPC4u0CnKk4hTnjz6psImxqm8r67onZh4OaBsPvBDmcSzsBCYIEjU4/Ay1o5G126n1VFbQUKKmX3WjW3N4d9a3uFVYtsc2X5FVz46gLbYTBCRmkGnuQ8AQEC/X37q7SvPvf1s/W1RY+Pe8C1gyvboTDC7PDZ2D95P96KeEvjY9Xbe1aoJvrFn47Hkx3MZtWZO5jr9XWYtvf0tPZEpBvVVuZOxh2VjtFqSCu8fuF1eHbVjkCtKRV5Fdg9aTfjnz2XGOg3EO91eg99ffphfJvxAIDjccdVPo6oWoTYo7HIuMuOw9zLCEk2uMJIq9YDAB9rHwANIj0A9O4NLFhAJQ3LosMbHWAsNEbccUoUppNTaYtPgbkAH+V/hDH/jtHsj1CStJtp2DZiG5IuJunkfLompywPYn45THkWTfr40c4iuoJ7Kk0LShE0OgiBozTPjFKHtOtpWOm/Ehf+d4GV86tCUVIR1oSuwe0/b7MditqQJInIOZHoMKsD26GwjpWHFV49+SqCJwSzHYpSvHbuNbxy6BW2w2hBh1z46gJOLTqFmIMx4PF58O7pDY8uHmyHZVCIa8WoKqpCdYlhVT9XVAAJdev+7epaFdKi3yMp61MEQaA8p9ygEguCxgQpZdnKZXJzKdGAz6csVmi6daMqhbKzpX+etRW12DFmBy59e0l3wWqAhZMF2k5rC8c2hmEX15jGViwEIVv0o4Udi6yBIEDgYfZDZJU1tXO+eJH62b49VS3WGGtrwMMDeNR/IYKWTWP6z1CJxPOJODbvGAriZAsc+kJZGVWtCchf/GiONHvPlBTqp7cKhVwkSepc9NMn4k/F4+CbBxV+1wTmAnh284StrwrKLYDS6lLM2D8DZxLOoFJUCVdLV+ydtBfhruFKH8PUyBTOFtSATSlOkbkdKSFRmlmKsmxu97x6uOkhonZEsR0GI5xJOAMAiHCLgL25vUr7tnNuBw8rD5gZqV5Rnxudi/t/32e0B9zLjKvQFWOCxqCbZzeNj0WLftll2Srtd/nbyzj27jGNzw8A6bfT8Xj7Y713ZkorSQMAeFo1iH5RuVGoqDWc731FXgWe7n6KnCj97b2oiJGBI9Gz/Hf8/ckQWFdQmaS0pa4qiCpF2D5iO26v5tZaanFKMVa4rzBIlwz6GVZC1MpMXKMTk+jxqgqvnX8Ng38dDAAICKDmMlVVDW5DZnZmOktcKE4pRvypeFTma6dXLdsUVFIe/85CezQumrXUcf1Ki+jHMuqUTLPtEy6qombS+pDdbe1ljZ6f9dSLWGVBEAR6ftITneZ1YjuUlx6CIODi4sJpqwN9Jzc6FwdnHkTi+US2Q1GLft/0w8j1IxE4mp2kDHU4vuA4fnT4ERKxfvQS4wv4mH17NsZtGafU9voybmNjqew+J6eGzL7guvyG1FTqgbw5b919C+O3jdddkFqkssAwHvhp4dbLCxA0Klg0MaGyMIEGu8fGGJkZIfliMvJjddQEjOOwOW5poYYWbmSJfrSFY1WBAzq4UILC6fim/q1P61pHde4s/VwhIYBEYIKnT9m9PgVPCMb0M9Ph0l7/7bVpa0+hkKrWVBZpol9a3XqKp5IFC3+0/gNbh2zVub3nw00PEXdSfv85XaFo7OZG5+LBPw9QliVfKCvLLoNEpPpzidBEiJh3Y7Ck+xJEvROF9EXpGNxqsMrHoasC5Yl+FfkVWOG2Ahe+vKDy8XXJ7FuzMePsDLbDYAS6onqQn/LWnjSvt38dqQtTsaz/MpX3jTsRh0MzD7HWv/HiNxexsetGVBZq51lJX56VpdHRrSN2jN+BlUNXqrRfv2X9MH47M8/Qd9fdxb6p+1CRq9/i2JkZZ3D5jcvo4NoBbkI3OFs4Q0JK8DDrodLHIEkSl769hPt/664vmCo4BDngc9HnnHbiURZ54/bkSWrueGoHtQ6aV5GHoqoilY5vYm2CURtHIfLtSCbCZQxxrRjWXtYwsdL/Nk7NoXtBn23vCrffHKX2Fva19UWgfSDche4qHbssqwxRu6JQnFwMgEps7FqXa0snpBbEFSD1eqqMIzBLyMQQfFb9GYLGBineWM+oElWhUkxNKNztmiYoCYW6vc+2iH4sw+Op/hFc/OYifnb5GfnP2VkY8u3niy8kX+hF9YqRiRH6fdMPvv2413y2BfV48O8DHHzjICt9fng8HlxcXJQat+m303Hzj5ucz/7lGrUVtXjw9wOk30pnOxSlISUkMu5QthcCcwHCZ4Xr1aTZwskCLu1d9K5yTtksNFXGLZvE1CVgBjV67rWxoezpSBJITn5xH336nsmjpqwGK9xX4Oi8o2yHojG06CfNDnBQ3Rrl9etUJmdjCILA4rzFGLtprHYDZIirP13FCo8VyHumnQVQNsctLdQoEv2EQsDYmPq9uwv14Z6MP1n/PklSYj4ABMrIAwkJAYzLCxFzPIHVvqNCVyH8+vvB1MaUtRiYQp1+foB80c9DySmPa4QrHEMcdS76HZ9/HNd/vq6bkylA0diNfDsSH5d8DI+u8v9RNw/cjNUhq9WKwdvGG8sHLEewY7Da98n3Or+HP4f9WW8JKQ0zOzN0nNeR8/NMY0tjCN2EijfkOGKJuEH0U6GfH40mz0xBo4Mw9dhUOAazU91eU1qDouQiGFsaa+X4mt5znz0D3ngDOKKE8/7DrIf45uI3uJB0Qa1zNcdV6IrJoZPRxaOLSvt59fCC/0B/xRsqQZ8v+2DctnGw9VPxxsMxvKy90MOrB6xMrEAQhFoWnwRB4MavN/DwP+WFQl3D4/NgZMJtW2ZlkDZuS6tLceLxLaTkUtX0WclC2AqohC5pApI8CIJAhzc7wL2TauKStrHzt8PM6zMR8VYE26EwTn4+ICYqUc3PR15FHuzNXqxoH+Q/CM/efYY1I9aodOzailpc+/Fak+R62oWkmNIBcfSdo9gyeIva8asKQRAGs57RmPwKSqshSD7c7a2bvCcU6nZuy+0VsJcAsVis8j6Wzpaw8bFh9eHdEAcmV7n28zVs6r9JYUbsy0LKlRQ8+PcBqgqllL1oGbFYjPj4eKXGbezhWJx47wTKc8oVbttCA85tnbE4dzF6LOnBdihyiT0Si0dbqZSoaz9fw/pO6xFzSHXbDC7Q69NemHFmBsxsVbc7YoOHmx7izl93lF4gV2XcsskzSldoIvoBgG/dWmKilOLXqqIq3PzjJuJPx2s3OC1TVVyF4AnBcIt0YzsUjZEn+vn7U59nbS1w/vyL73OxN7IsTG1MYeVhpbUFSDbHLV3p5+gISEgJ3uzwJiYET3hB9COIBmGwg5CqJDqdcBoSkqpOysoCSkupajNfGZpASAjg9vwizPdsRmk+e1a9oioRq6IjkzAl+pWUUP8BgLuS610TdkzA4BWD6xdRCgqoPinaZvKByej7bV/tn0gJFI1dI1MjmAhNFF7vAkcHInRqqDZCVIpX276Kdzq+A19b2YIej8/DsFXDEDIpRIeRqUZtZS0y7mZwtpq+RlyDzNJMpbYtqCxAuGs4nCycNLaFFEtUu7fY+tmi9dDWrD0rD/xxID7I+AB8AV8rx9fknisSAStXUvfObdsUX/NOxZ/CFxe+wJ+3/1QzWmZh4t5n5WGFsFdU7z/KdQb7D8b4NuPlXgel8caVNzBp7yQtRaUZpZmlSL6UrLWqWV0ibdzezriNofs642qbrvVuB+GJW3Blxl2EOrF3T21BOQoKgCpjKpncXGAOG1Mbxo5t7WWNNuPbwKt7Q39jKyvqJ/28G/FWBPp/118nc4Ksh1mIPx0PcQ2312jUIb/O2lMgsoejY1PthM/X7d+rP6sLLdQTOScSs27MgrGFdhZaDI2rP17FqqBVqCnTwaxbC5RllyHjbgZMbfU/+5oJBv44EEtLl8LMjp1JV2lpqVLbdZzXEdPPTId9a9X6Tbzs8AV8mDuYsx2GXKpLqrF36l4knU8CALQZ3wahU0Lh25/bmd6Gwo1fb+DysssqJZ8oO27ZgiSlV/oB8kU/iViCE++dwKPNUprE6RFW7lYYu3ksOryh/71r4+v0V2miH0EAAwZQv9+48eL7hYmFeLjpoV70DIqYHYFZN2bBysNKa+dga9zSE18rK4BH8PB136+xe+JuqRNvuqLLnewKC4EFcspz6m2w6DHt7y/bZtLRERAFhyE5dBji4tlLqNsxegd+tPuRtfMziaaiH53tnF5nOODoSNnzqoJQCJjWPbbrotrPt68vPDpzx4FF3titLKxUSoTq900/9Pmyj0rnXXlzJYZtHfaCze7LTF50HtZHrsftNdzqyUTz5sE34fGrB/598K/CbR0tHHHi1RNIW5gGAV+gcHtpLLu0DC4/u2DF9RVq7c92mxVtou4998AByoaeOgZw9ar87e9nUbaPHVyYe+Y7EnsEf935q766QhkuLbuEb02+RcFzzXrZFiYUoqpY98nITHMv8x6WnlmKvU/31r82v/N87Jm0ByMCRqh0LMc2jpydzz8/9hz/9v4XaTdU74fGRZqP2/gCaiJiXu2PUaMANzfALKs/Tv8Xjsf3TVHbKL+suhr49FPgTzn6+/7p+/Gr16/aCF1tUq+l4sryKyjN5Pb8Xh0KCoAqAfXddBe6M1pswzPiYdKeSQga07DY0Fz0C54QjE7vdtJJkc/NlTexZdAW1Fayl/SoLfIqqAxSY5F9fYIoW7SIfi0YPOJaMQiCQEU+9xfRpDHop0FYUrjEICwImMDMzkxrlQVMYulsCb/+fjAybfncVKU4tRgxh2M4O7E2tjTG2M1j0Woo5ZFv52+H8dvG620iRnVpNS5+fRFP9zxlOxSlmLx/MmezR9UlI4NaKDE2frEiiP5/uoKsMeb25phxbgYGr1C9XxFXkIglBlNhVFraUCUmq7KrfXvqZ0wMlR3fmLgTcTjw2gFk3lOu6qEF7UCvn1gpoWfSE7mifGN82+9bbB23FX62lOJLW3sGBMjenyAAvwH+yPXpiJgE9RaxmcCzhyfajG/D2vmZRF3Rz6su8TkxERCLVbf2BICkC0k4/t5xlGWW1lf7ZWWpFoeqkBJSr66h8afisT5yPRLPMd+7+b+H/+F43HGVLcykUV5TjqspV3Ey7qTc7S787wJ2jN6h8fm0hYWTBfp+25eTFqTVompsfbwVElKCNw6+obRYq67gR5Ndno1HOaolSxWnFOM7i+9wegk7gvKtVbeQdDGJlXPLIzcX2FH39afvdceOyd/nXuY9AEC4azhjcbx3/D3MOToHsfmxSu9j62sL/0H+4BlptiR67N1jWOG2AqIqkeKNOcz11OtYfnU5lh/fgi1bqPugutRW1iIvJo+T/yYeXTww5PchcAp1YjsUrfC87v5nUdUK4eHArFnUs+bdu8DXXwNvvdXgaHD3LtXL7cQJ4MED6ccTegjhEOTAqXWZhDMJOLv0rN730JQGVelHZZ25W8m2mRi1fRScfnLC1RQFWRYKaC766ZLwmeEY8dcImAgNrzej0FgIn5rhsCvt1SL6taAa5TnlODLniFYmSoZKr097YV70PNh427Aditq02Kk2IKoSIfN+JopTitkORSYkSaI0o1SvFmG4xLWfrmHHqB0oSWPh6UMJCB6BoNFBCB4fzHYojMAz4uHClxfwdLd+iH42PjZw78it3gKaQlt7tmr1YkUQXTGWlERVBDbHt68vZ7NpleHm7zexLmIdCuI0y7TmArQw6+LSUDXUHE9PqgqouvpFIbfVkFaYtG8SXDq4aDdQBri34R5u/nGT7TC0Ai36CYVU/5OU4pR6y87mNO7d9n6X9zE1bCqsTaneDXSln6x+fjQhdc6AUVGaRq4+vT/vjVEbRrEXAIPQlXrW1vK3a46HB2BuTlnTJSerJ/plP8rGrT9uIT82H05164k5OarFoSoZdzPwjdE3uPG7lPJhDuLSzgX9v+8vd8H18bbH2DluJwoTCxUeL6koCV9d+ApT9kzBvcx7MOIZYWLIRI3jfJb3DD3+6YE3Dr4hd7vcp7lIuZrCqQXRxlh5WKHXp73g2dWT7VBewMTIBFWfVsHV0hUAMH7XeDzKli7GFVcVI71E837f7VzaAUB9RbaymNqawruXNysOLtWl1Tg+/zge/P1A5+dWxObN1PNMSAhVMcTnU8+00hLVAKCspqxemGOy0s/Jgrqe5JQrf8ENmxqGVw6/onEfvjbj26DLwi56n+ibWkKVa+bFe2LnTuD776n7IUmSSChMQG658mXrN369gT+D/kT242xthas2TiFO6PxeZ1h7qviQoCdEZVCVfsJaf7RpA3TsCHz0TSbM+v2GVM+fkJcH3Kx7fL/TqFXjv/9Kn2cO+H4App+aDoLHnfXIyLmRmHVrFuxa27EdCuPk5wNVgjrRTyh7vaOoqgi5FblIK1GvYrW8phxVoipYWVEfekkJ9fk/O/AMa9quQcrVFLWOqwqe3TwR8VYEp75bTNHRvSO6JR1B25S19XNFtmgR/VhGVTEnJyoHd/+6y8kbaAvMU5pRilurqMWDFijyY/OxLnwd7v99X+fnJggCnp6eCsdtZUElVrivwJE5SnQzb+EFQqeEYuSGkTAWcq9yLvVaKvKfG9Z4FJgJMOfhHIxYp5p1CxtUFVehLLtMJUFd2XHLJrL6+QGULYtAAFRVSa8YIUkSRclFemEJKY3aylpUFVZp1SZSV8jr50dDEECbuoKqp810dltfW7QZ2wYWjjIUQw5xZ80d3PhVeyIDm+O2sej38ZmP4f2bN/64+YfUbRuLfo2prW34Psir9AMA+/IUhFxYjYyzT1+o/mxBdcrqWmALVWx9ThANn9WzZ+qJfm1fbYuFqQvh1dMLLnXafbaWp2wCcwECRwfCzp8bi1+Kxq5DkAN6fNwDDkGyU59zo3MRczBGKZeTqXun4n8X/4edUTsBAKMCR8HBXPO0ai9rqvQzsywT1aJqmdtN2DkBH+V9ZJCLVrrAxMgEiQsS0cenD0prSvHagdekJlnsjNoJj189FIqwigi0p7Iw4gtV64VsIjTBtOPTEPFWhEbnVwcjEyO8fvF1dFnURWvnUPee+/gx9XPqVMDODujalfr/48elb/8w6yFIkHATusHZ0lmDiJuijujHFOEzw9Hv2346Py/T0OKBWQ2VIHDzJvDll8Dk3VPhv9If2x5vU/pY3r280X1Jd5jb629Soj4gbdw+y6GubYHO/jCuW0qxdM7G7pKFiHf9AQBw7Rol8NCiH0FQ7QkuX9Zp+Gpj4WgB947uEJix55ChLZpU+skR/TysqIfT9FL1kmE8f/WE2TIzZImoDMWaGiqBAwQgqhRBVNkyIdGEmpqG6snmlX66ntu2iH4sw+Op9hH49PHBooxFaDe9nZYiMjxEVSLc23APz48/ZzsUlUm7mYbj848j9Voq26FwBhsfG/T5ug/8BshZVdUSPB4P9vb2CsetRCRBx3kd4ddf9zEaAp7dPBE+Mxxmtuz0bZQFSZI4/NZhrO+43uC8x53bOsPUmvt9Q5/seIJfXH5B/EnlF2uUHbdsQot+0iqC+HzA25v6XVpfv8Rzifjd53c82qqfff16fdoL8+Pm632GNKCc6AcAwXVFwrIqu2rKud+DeOKeiXjl0CtaOz6b47ax6PcsjxqcgQ7Sy/XoiRxt63ou8Rx+v/E77j8rhEhE2ea4KCjcdPfig4AEohqJTvq/NUdcK8aB1w7g4SbVKl+4Cm1bJavaliauIA7JRclNXqMTL2Ji1BP9zOzMYOVhBR6fV2/vqW3RzynECZP3TUbACAXqso5gYuz2+6YfPqn4BJaulnK3e5j1ENfTrsOIZ4Tl/Zdj/+T92DRmk9rnbYyDuQNMjajnInmLalxOKAKAK8uv4O8ef3OuzUWtuLY+gcvEyAS7J+6G0FiIB1kPmvQUozkSSyVStrJtpdF5aaGporYC5TXlGh1LV/CN+fDu5Q2XdtpzAVBn3FZVvWhpPmwY9fPsWeD8+Rf30Ya1J6Ce6Jf/PB+nPjzVss5SR1Ih9e9gWuOJ116jKt+fPAFMS6lMtTuZd+Tt3gSvHl4YsHyAxlWU2uDMx2ewtt1avXjWVkTzcUuSJNIqKHvPLq0brpWt7KjfyyT5qOEX4t49ag5SWEj1H55U1zVj82ZKrGhM9qNsnPv8HHKjWXhAlUFlYaVBfH7SKCgALKoC0M1lANo6t5W5HS0IqlvpV1hFOSlkViRDUKedlpQAQaODMP/5fJ2stW7ovAE7x+7U+nnYICeX8kc2MXlxPqLruS13V8BeEsQqmmUTBAGhqxBmdtxaDOcyBJ/AkbeP4P5G3VeGaYpvX19MPz0d/oP92Q6FM5hYmaD3573h1cNL5+cWi8V49uyZwnFr6WyJYauGIWRSiI4ia0FXDF89HEN+G2JwmWW1lbXIicqBqJrbWV0OQQ6IeDtCpT4Myo5btqispKzkAOmVfkDDYoo00c+1gysi34mEawdX7QSoA3h8w3gcja/TohWJfrSd49OnL1rpbB26FasCVzEfHMPY+tpqtR8KW+NWJKLGJACYWYjwvIBKGAtykD446Uo/euFz9uHZeP/k+zj9+AEAqnJMkSbg0dkdBVPfRaFbKPJZKCSvzK/Ew00PkXZTvYUDrqGM6BebH4uwNWFo/1d7ZJc1qHJ04kVUVENltSqiH0mSKEoqamLvqW3Rj2soGruZ9zOxocsGPNn5RO5xjEyMFApqBEFgVOAoTAyeiCU9lmBM0BhYGDNTKU0QRH21X0qxbJur0oxSRO+PRmlGKSPnZZrKwkrkx+Zzru/0urvr4LfSD3/d+QsAJbJ+0PUDAMCJuBP122WXZePAswM4m3gWADA8YLhG5xUaC+vFXFWrwm6uvImzn57V6PzqIKoWQSKWbjHNFOrcc9PrtHBr64bK6tBQyk6wthZYsQL47bemIsKDrAcAgHAX9kW/sqwyXP/lOjLuZKh93vNfnsff3f9GZUGl2sfgCnE5lOjnZ++J8eOB/v2p1+1qKBtWWda7+gZJkqitrDWIuXzzcZtXkY8aUHYH/cJ96rezNLaEiyWVNGDuEYfaWmDdOuq9du2ACRMAGxvquefTTynhiSYvJg+Xv72MrAdablCsAjtG7cBK/5Vsh8E4EgklxPrkvoPDE09jWttpMrfVtNKP5ujzI6z19TN3MIepHfeTztVh7sk3cLyDBbK91rwwD9T13NYwVlleIjLvcbuXGRfhC/h49eSrGPDDALZDURlTG1P4DfCD0FVFj6IWtEZVVRXbIRg8ErEEq0NW48DrB9gOpQkEQcC7lzfav96e7VAY59rP17AmdA1yn3Ini08aPr19MGLtCJWtILk8buPiKOHHyYmyRpKGPNHPzM4Mw/8cDu9e3toLUgtU5FdgTdgaRO+LZjsURiBJIDOT+t1LQU6Kvz9gbExNrjKarTV5dPWA/0B/iGu5KVIDDX1rtV3xzMa4pa0hCQLIrU1CjbgGpkam9Yv/zaFFv/JyoKICCHGkFN07yVQZp6J+fjR0xSAbop+FswWWli3FgO+5/5wcH98wzmRBf4aWMorESJLE/OPzUSWqQlFVEdbfW1//Hv15ZWdTiy9mZoCtioUKa9quwdG5R+sr/bTd0y/mcAyOLziO0kzuiE7yxq5EJEFpeilqyqRn6deU1+Dh5odK9Xlt69wWB6ccxJZxW9SOVR7KiH7Jl5Kxa9wuzlYLDfxhIBbnLOZcNf2OqB1IKkpCRW1DBeLCrgtxYtoJbBi1AU9ynmDIliFw+cUFY3eORUVtBTytPNHOWTO3I4Ig4GxBDU5VRb9nB57hzhrlq52Y4vbq2/jW5Fuk3dBuYoaq91xp1dAEAXz2GTBtGvX72bPA2rUN768dsRaP5jzCzPCZDETcQL3oV6H8Z+oW6YYFiQsQPkt9AbKmrAZFyUUwtdHvhWuxRILcako8GNHLEwTR8HxjVEpNQlSpKCIlJLaN2IbTS04zHqumDPxhIObHzjcYS+bG47YoX4A2qT8jMOtzhLVp+p1sbdcaAOASTFUC0nPKyEiq2m/xYipZ6tkzYOFCqpc8APgP9Mc7Ue8gcJSSD7Q6oPWI1mg3w/Cc74qLqWdPglDcl9rdSrNKP5qi6qImop+omnLJiz+tmgW2Okw9OhWjN47W+nnYIKcsD2J+BWwsTdgOpUX00zf2vboPmwdtZjsMvcNvgB9nel2oQnluuUq9q14WTrx/Aus7rle8IUtc+N8F7Ht1n9azMg0VHp9HZf5waAIlqhYZdC9V336+6PlpT85Zqr4M0AvCbm6yt6FFP9o+0hAoSixCaUYpStJ1nFaoJSoqUN+PzcZG/rYCQUPvsOZ9/Xp/0Ruj/xkNvoDPeIxMUV1SjRXuK3Ds3WNsh8I4tLWnpSWQUpIEAPCz9QOPkD5lMjVtqCi7fRsoiadEv/tp1AfburVy5xU+vwebzKf1FYO6hCAIGFsYw8RKNxNTiYTqJ1OuorNeTg7w4YfUgrK8R2Na9JNV6bc3ei9OxZ8CAHzR6wt80vOT+veEQsC9UQsVDw/FlZqNIQgCPT/tibYz2taLfsXFlA2etki+lIxbK2/pTf8V947uWJi6EOEzpS+050Xn4cCMA3i8/bHSx5Q1PjXFy0qx6OfZzRNjN4+FeyfZvXdaaEp2WTaupFwBAQKTQibVv25lYoXBrQaDIAgQIHAy/iQIEAh1CsXs8NnYM2kPI3aqXTy6oI9PH/B5qt1nx20dh/fi3tP4/Kpi42ODwFGBELpzKwlYlgUyjwdMmUL1gyMI4PRp6poPAAK+AGHOYTITadSFFv0aV24rQmAmgI2PDQTm6ld8Df5lMBalLdJ7AenyvVxIiBqA5GHsAMo5xN6eek9UQH3ABZUFTUR6eRA8AjmPc1CUUKSNcFuQQXGONfyzP8AA/tcwapbnQYt+xq5NWx5FRlI/27alqnM9PalKvy11uTSmNqZwDHbkVLV4jyU9MPDHgWyHwThFRQAJEhbWVeAruD3VV/qVqF7p13h9ubiquGmlHwkcnn0YD/55oPJxW2iAtk91ErKvQXAr5asFhXRZ2AUwAA1o/36qfHzOHNUm0+pCkiSqiqpgIjQBz0h/tO5Vgavg3skdr554le1QOIWoWgRxjRjiWjEnF0Yzbmcg816mwVjWscHrF19nO4QmPD/6HLvG78LI9SM1ygjlKl7dveDVXfeWuaogEUnwX9//0GZCG3RZ0IXtcBijkHomlFnlBwA+PtTP3FxqQbt5BcvzY89xdulZDFs9jPOfI41bpBsW5y0GKTaAhxo0WKKYmlL+/YoIDqZ6pURFAQP1cN7aaX4neHRRwfdQT6A/R6GwIXuWnljLwtGRErB+/hkotgsB/IBSsyi0aQOEhSl33toTZ+EscER+frAm4atFeU45ilOKYR9oDxOh9oW/s2eBlSsp67D331d+v/v3KWE9J4eqiKSrIxsjEjUIbNIq/cpqyrDw5EIAwJe9v8T/+vzvhW0CAxts61Sx9qTpsaRH/e8WFtR3Izu7oTcr0/T9ui+6LOgCSxf5/e/0BVt/W0zcPRGOwY5yt/vj5h8YHjAcfrba6zujTKWftZc12r4qu+8O29zbcA92re3g09uH7VDqoW0CA+wD6qsVmhPiFILVw1ZjcKvBjH/GOybsUGs/tpx32oxtgzZj27Bybnmk1hW3enpKfz8iAhg/HtizB1i1CuAb16AwzxgCAdCvH7NrQD29emLnhJ3wtfFVeh+SJFGcXAyegAcrd9UcRAyN5KcO6PcoCWFdsmAtpERQutKvJNcalq0sUVZThtTiVJk9jpuzIGkBJ3ue3v/nPkysTBA8XvfPW9qGFuKljUm6r18h4uDkRD1L+fg0fZZycwPmzwc++ohyogGocVKRVwFSTBrMcwZXKS4Gqo2ycbSVK8797ISMRRkyk1M8rTwR5BAEXxtfkCSp0lirlTQ4tRRWFTYR/YxMjTDtxDTY+Nho8qcopLKgErdX34ZPHx9W2jZpm5IaaoHHVVW7EC3QsiLNMqo2cYyYHYGItyK0FI1uKCkB/vkHOHZMd1ULl5ddxo92P3Leuq4xErEEIZNC4DdQ+01U9Y0Ra0ZgzsM5Ohf8eDwe/Pz8FI7bqUenYkHiAh1F1YIusA+wR8TbEWg1pJXijVvQCmVZZciLyUNJmmqVYcqOW7ag+ybIE/0sLRsm37TdSmMEFgJU5FWgIle5DFyuQBCEXiXiyKOoiPqpyI6FJrhureHxY6pKEKD63ty8LsHeN4/iyvIrjMfIFKbWphi6cijCpiqpaKkBW+OWrhJrIvoJ5Ss/To1aG/YMoj5YwikKP/xAwljJxOiAzycjOWw4K5V+MYdisL7jeqRckS1sMEkU5XyK27flV+w151GjdkJ0H9TmVDS6BJqbv/j+3/f/RlpJGnxtfLGk+5L616tEVUgopCYljS1Z1RH9GqMLi0+BmQBWHlacuZYqGrvVJdV4uPkhMu5K76NlZmuG4AnBckW/1OJUvHfiPQStCkJxlfbaXgwPGI4/h/2JNzu8qbVzaBOJSILDbx3G7VW32Q6lCdF5lK13G0f5QtbcjnO1KuqqSlVRFXKe5KC2QrvW1mygzj1XVqVfY6ZNoyzNC8pL0OuAJ946MhM/ryzDzp0aBtwMT2tPTAqZhI7uHZXehxST+N33d5xadEqtcxanFOPaz9eQ94yFGzfDxMfxYV7jjcGhnetfo8WgwgICnlaUiqSKlSAXBT8AOPfpOVz/5TrbYTBC83H7IDkBReZ3YO364neytT1V6fe84Dn69KFe6979xWPSiab5+dS6LSkm8bPTzzg+/7gW/gLVIUkSuyftxs2VN9kOhXGKioAqYyrrjE/w5Vaju1u5I3peNI5NO6byWKsVN9zDiqqKXujp12pwKzgESsmsY5Di1GKc//w8Es9L6V1iAJSJiwAA7nY2L7yn67ltS6Ufy3D1ZqhN7txpmOSnp1MPgtrGraMb2r/ZnnP9DOTB4/MwYu0ItsNooREEQcDKSrlMQH36rnGR9NvpiDkYg8i5kZzIvnQKdTL48bh70m4YC405661u5WGFxTmLIRGpZpuryrhlA7rST1EimI8PVemXnAyEhjZ9z7uXNxamLdSrZ4qHmx/Cyt0Kvv2Uz8rmMsV1686KrD1pgoKovn45OcCbb1IZ8XfvAuXlPLQ/FwOfDnbo8XEPxQcyUNgat40r/Xp69cTSHksR6RYpd5/x46kKz6FDAb/AIKz6jkB+ZT5yynPgbOms1Hl9enmh6hJYEf3cIt3Q77t+cApxUrwxA9B9ZEpKqN/9lFjTJ8mmol9KCjVmmkNbhpqZQao1UnQuJTbMCp8FMwFlZ305+TIm7p4IT2tP3Jp1C0FBDddRdUS/Zwef4dI3lzBi7Qg4O7shIYFyN9EWeTF54BvzYevLfjYxoHjslueW48CMA+i+pDvcIpr6WhfEFUAiksAhSP5iEy0atbJrBWtTJTMt1CDSLVLh+JeIJVjptxI+fX0w5t8xWotFXaafmg4zO25Zt9PjsI0D96rX5HF33V2cWXIGs27O0qmd68kPTsLS2RLdP5KyQs8Qqt5zJRLlKqKNjKheYRN+2IYaQQ7K7a+Bn2yBrVup/Xqw+JjDM+Kh+5LucG6r3H26OWk30nB68WkI3YQKr1lchiQbqrpaNcpttbOjqjFFImBSwGuoDiySWZkrjbyYPKRdT0PQmCBOteyYcmCK3tux0jQft8dz1+JW8E9wrV2IOVjRZNt+vv1wa9YttLJrBaEACAmhLD2bY2YGuLpS/ZMTE4F27XjosrALnEJ184yoCHGNGE/3PAXfmHtuX5pSXAxUCagLqypjTVUIgoCb0A0ZpRkoqS6BVd1HW9Ior1pUJdLqeqZ9gD3efvA2zB2kZOjpOdXVQKWkCOABQb42L7yv6/UabqQEvsSIxWKlt004k4C/Ovylk6aa2uTWrYbfM6QneTJOq8GtMHrjaNgH2OvmhC1oldKMUtz68xayHmhxFUUKYrEYjx8/ljtuSzNK8XTvU5RllekwMsMj824mLi+7jNwo9qtzq4qqXoremsXJxShNK2U7DIWoWs2gzLhlE2Uq/YAGa7gUKcU4BEHoleBHkiSOzTuGS99cYjsUxlC10s/CAli6lFr0Ki8HLl2ifhIE8LjHHJROeE1rsWpKytUU7Bq/C2k3NGseLw+2xi3d008oBHr79MZ3/b/DuDbj5O4THEwtaoaGAuYC8/rKlKe5T+Xu1xh7exK82mrk5+n+XuPS3gU9l/aEtZf2xBMakajpNez+feX2S0trGGOA7Eo/ulJTmrUnAKwZsQZZH2ThrYi36l8LcghCaU0p7mTcwemE0/D2pha9APUsOSUiCcpzylFZWKmTSr+9U/Zi8wDu9HtXNHaFrkJMOTgF7V5r98J7Zz85iz+D/0RRUpHcc8TmxwKg7CHZhsfnwdbfFkI3bvVbA6jnJb8BfnANd2U7lCY8zaOujWyJflsebYHLzy6Ytm+aSvt59fRC7y97w8JZRsNQLfHg7weIP6nd9R9V77nZ2dT13Ni4wYlCFm5uJCqD1wIAPh/6NsaMpp5Xf/21IQmECQ48O4C1d9aipFp5R5AByweo7VrQakgrvHH5Db13ZcrNBWL5exHjsRRJuFD/Op/fMDeZ5rME3w/4HkEOQUofN+ZQDA6+cRC50ezP5Rvj3skdbpFyGqnrEc3HbU4ltTbm6+TywrZ2Znbo6N4Rtma2MDICwsPxQt8/mua95AevGIwOb3ZgPH51MDIxwheiLzD6b24mKWtC40o/d6Hyop+q61TmAnOcm3GOOqeUSr8do3fgB7sfVDqmqgjMBHBp58KJ5H6muXy9EhJeNQCgQ5DNC+/rem7bUoqiR9SU1aCqqEqv+4SJRMC9ew3/n65639GXhrgTcXi0+RF6ftYTjm0UPE2/ZBSnFOP4u8cx4McBcGn/4kONNlF0kU6+lIy9r+zF+B3jETo5VO62Lcimzfg28OrpBTt/9pvfbhuxDbXltZh9Z7ZeX38VMfPGTE4LR/Gn41FbUYuAEQEqfw5cFfwA5Sv96P4MdA+V5uRG5+LRlkdo/3p72LfmeIILCUw7ptpiG9ehK/2UFf0AIDKSqla6fh149gxo355aQPvkE3OcPQt06wZ06qSVcDWiKKkI0fuj0f7N9lo9DxvjtrHopy5rhq+BlYkVwpyVX0h8tvI0wk9ex6P+70Mkspa5EKPvpKVRcwGaBw+oSklF0FV+RkbU/tJsjoGGSj8LOWvyzasvHS0c8Vb4W/jt5m/47vJ3GOQ/CJ98QllbuauRaB08Pri+V9DTw9Rr2dmqH0dZIt6OgKhapHhDHSJv7ArMBQgcJb0nVJeFXeAU5qSwl0xMXgwAINBeud5SmnAt9RqSi5IxMnAkLI2lq8mvneNmkgYpIQGCe85C4S7hqBJVqXSNZBI+wUd2eTYySlXLPvbs6gnPrjIa2GmRD7I+QG259i1FVbnn0taebm6AIqeyk/En8TD7IUyNTDGj3QxYd6SSP+7fB44coXqIMcFbh99CbkUuunt218l3y8TKxCB6UcXFATnWx5Dq+DduZVpiYOs+9e85OFD3wtxcoHVr1Y4bNCYIDoEOWrcJVAVSQkIikhhUlRg9bktKgHKSetho7aJe9SqNry9w7RqzojyTEDzCoD5DmqKiRpV+Soh+i04uwpZHW/BVn68wt+Nclc7lbuWOXwb9AkdzRwiLSABEvejn1dMLJtYmkIgkWrOOrymvgUQkgYnQxGAqb2kuXK6BU9EwWDsXw8qU/YQww129NECCxgRhQeICvbbCevIEqKxs+H9diX4kSeLQrEO49K3+VBVkPczC422PIari1kSeCziGOGL6meloN/3FLGG28ezmibGbx8Kru/5PAtjEwtECTiFOrNukSkQSuEW6wauXl0ELfgD3FoWac3X5Vex/db9BPRiSpPKVfrToJ63SD6Bs0a58dwUJZ3TULFcDCB4Brx5eBrFYQqOO6AdQlX3dulEWn+HhVLXYqKE1sMqJw9pvcps8M3GFttPa4gvRFwbZ47Sx6Hcj7QaSipIglqgmPg70H4jOHp1hLlDessa3hzvyvdqDREMigK449u4x/NPrH51UtNMLSPT17ulTqpelImjRr3dv6mdqKmUv1xxFlX6y+KDbBxDwBLiYfBFXU66ifXugf3/VjiENl7q8NG2KfpFzItFlQRftnUCHeHb1RO/PeyvcLrZAd5V+Y3aMwdR9UxFXEKf1czHN8+PP8a3xt3i09ZHijXXIr0N+xc1ZN9HepT0r53eyoLzMcsq1WILLIEYmRpyzaFWmnx8AFFcVY/bh2QCAORFzYGdmBz4f6NuXep9J62N1Ptdrv1zDxm4bIa5VPcmoJK0EErFqLQe4SHw8UC2gblLNk2Lovn7ZuSIkFyWr5mDQ2h6BowI59d0tTinGtybf4vSS02yHwjhpaUCVgBpQnrbSk+K3P96OBccX4Gaa/H54tO06/cx2/dfr2DxoM5VIwjJVRVVIPJ+I0gzuuxOpSnFxQ6Wfh5Vif/lacS1yK3JV6rUJUGvj5gJzLOq6CNPbTYeNDbW2Qot+3T7shrGbxmq1V/S99ffwg80PSL0mI5tZTykrA6LuWaNT3FGcfuUKeAT764fsR9DCSwVt7RlYl5iZkdHQ30+bEASB50efI/EsR9NVpNBjSQ98Uv4JnMM0y9QxREyEJvDr7wdLFxVXdXSAtZc12r7aFlYehleqrmsqCytRnFrMagw8Ix6G/DYEQ38fymocuqAkrQT3NtxD/vN8tkORyoAfB2DslrGcFydVobKS8n0HlK/0Ky5uEJga49vXF2/dewsRs6U0uuIYVcVVBrFQ0hhVe/rJY1i3YgTc2grj6IeIidH8eNqA4BEGmQhBi36mFtXourErfH/3RVFVkdbPGzIpBBUDR6PWzFrnff1qSmtQVVilk2srvYDUrRsl/NXUUMKfPEgSePyY+n3QIKoatqZG+mIxLfpJq/Rbc3sNhm4dil1Ru154z8PKA6+3fx0AsOLGihfeV5Wne58ialcUnOr6pGhT9NM3aspr8L3V9zj81uH610gJifzYfKWF5/pKPwftV/rRfXXkVYU9O/AMZz4+wzkreDNbM7Qe1lon1r36BC1sZJepNjCLkovwX9//cHf9XW2EJZWKvAqk3UhDVXGVzs6pDMqKfotOLkJaSRr8bf3xbb9v61+nLUFzGXR+VEf0K0krQVFiESoLVMuwIiUkVgWuwrbh21Taj4vExQHVdWKRi2VTsYgW/S6lnYHP7z54Ze8rKh9f1V7s2oQn4CH0lVC4duCW5TETpKUBNTLEW5p9z/Zh5a2VuJkuX/Sj7T1TU4HaWiDvWR5nrkPZj7Kxqd8mPN2rvACtLzSp9FOipx+9TVqpaqJffGE8+F/zYfsDtfjQ3N5TFzi0cUD7N9vDytOw1kyvX6ccSXx81GsRoA0Mb7auZ/AU+SE04uHmh4jaHaXFaLQLSTaIfqNGUdnt5eW6u7jMj5uPGedm6OZkDCEwF2g1w0KfIUlS5w8ePB4PgYGBcsct1yb8+syqgFXYN3Ufa+eXiCWcyGjTFTlPcnB49mEknuNmcoRbhBuCRivfS4JGmXHLFnRFj5kZYKqgx72pKeoXkGmLz9xcqtE6ABhbGsO1g6te3DNOvHcC31t+j5oyJUp89ARVe/rJwyXEHsTwoShwD9NZ72NVyH6UjaQLSWplxisLW+OWFv2qjal/eBO+CezMVLOZLq8px9o7a7H41GKVngnoxbV8HeddjPlvDOY+Vs0WSF3o/jC+vkCHuvYwDx/K3ycpifpcTE2BgADAy6vh9ebQ9p7SKv3OJZ3DibgTSC6S3hBwdjhVjXIh6YLGz3LnPjmHi19drO/pV17eEBuTlOeUY2PXjbi95jbzB1cTRWOXb8yHZzfPJn3W857lYVXgKpz79JzC41fWViKlmCp514W9J22xJU/0iz0Si6s/XEVVEfsLoo3x7OaJKQenwLsnR1aeAJRUl6BWrH2rSnnQ4lB+ZT5EEuUddQgegexH2SjP0cJglkHiuURs7LoRcce1W2mq6j2XFv085bidFlQW4HjccRAg8M/of2Bh3JCNQV8bc3OlV22rgzqi3+AVg/FB5gewdFYtkVhULULEnAgEjVF9XsIlSJIW/erEIotm9td14ixRQn3QqcXKV+VIxBL86PAj9kzZw0ywDGDlboXx28YjdIphtGBpPG6TU8SoNqJU9OafI42XFfUARd9DZeHgQD1HicXUnHPEmhFYWrIUZrbsV23a+tli2J/D4NPHh+1QGKe4GLAt74xuzgOVcjKgqwFVrfSj73tFVUU4GnsU5Tzq+aakhLom5Ebn4tCsQ1pdE2o1uBVGbxwNG28brZ2DDS5cAEiQ6NVL9ja6nttyf2WohXoufHkBl5ddZjsMtUlLozJdBQKqRw39EJGm2jVKbYwtjPWqQiThbAJyovTDdoQN/u7+N9aErtH5eY2NjeW+vzpkNbYM3qKjaAybju92RPDEYNbOH703Giv9VyL1umHZDsjCvZM7XjnyilrCmrYhJaRGAqyiccsWylp70tCL3SkpVKXLBx8A77/fIFSIa8RIu5GmcsayrnGNcEWbcW1gbMnNz0UdmKz04xnx4DGuEyqtnDnZ+/jaz9fwX9//IKnVbvY2G+OWHkvlRtTDqYeVh8rPjjyCh3nH5uHn6z8ju1y5SpLy3HJYnj0Iu7SHOq/00xUk2VDp5+sLtKtzaL9/X/5+tLVnSAjV04/OnJVmdSyv0u9Oxh0AQKRbpNTztHNpBxO+CQoqCzS2chyxbgTGbBoDU9OGDGptVPtVl1SjOKUYlfncuubLG7t8AR+vnngV3T7sVv+akakROr3XCX4D/RQem8/j4+jUo/hj6B9wMNd+ryg3oRsAIL1E9sW49xe9Mf/5fJhYmWg9Hn3nfxf+B/PvzLHs0jLWYrA3s6+33MotV77UzNrTGh/lf4Ren8pZzWMY57bOGPDjALhFumn9XKrcc5Wp9LMzs8OTd55gy7gt6Onds+l7dgCfT1VE0M/CmqKO6Kfu2pDATIDBvwxG5Bzp9xN9ITcXKC4hUW1E3aBkVfqJC6kPurCqEOU1yonePD4PPn184BTmxFzALbwAPW5j0/MAQgICBBwtHKVu62WtnOhHEA3VfomJ4FRrDSsPK3R8p6PBuaGRJJVAGpjxDfaNPYUuHopt2+mkJFVFv8aJNyO2j8CDQqoFllgMVFQANWU1uL/xPtJvc3ASymFKSylnkgy77Xg91hKTdk9iOyQALaIf60hUSG2atHcShq8ZrsVotAttzRMSQmXr0g+JuspiL0kvwfNjz1FdUq2bE2rIrnG7cHz+cbbD4Cyth7VGwCjt9/JojEQiwePHj+WOW+e2zrAPspf5fgvK0+fLPuj8XmfWzl9bWQtSQsLOX7UqD33FzM4MAcMDOGmbm/0oG9+afIvrv15XeV9lxi1b0JV+iqw9aWjRLzWVWggvLKQezulKmajdUdjYdSOeH3/OfLAM0vm9zhi3dRzbYTCKuj39ZOHuDoAkkZ7Kve9thzc7YOgfQ2Fkpr2eq2yNW9p9ophsEP1UxUxgVr+wklCoXI9NnhEP4jsPICxI0anoR5Ikbq++jeTL0qvfmKSwkPr3JQhKuGvfnno9IQFy/2ZaFKRFQlr0S5YSsqxKv7yKPCQVJQEAwl3DpZ7HmG+MnRN24tGcR/CzVSw+ycOntw/cIqhFejqpg64GZhK7VnZYlL4IvT7TnQihCHXGrq2fLYb+PhS+fRX3rTfmG2No66F4t9O7OknmVKbSz9rLGnat7Dhnefx422OcXHQS1aXcmftG50VDJBHpRLCVBZ/Hrz8/1/v6OQQ5oPvi7rBrpd25iCrjtqSk4V7ppkCLtDOzw9SwqS+8zuc3CEo5DH0E6oh+VUVViNodhayHDDYX1CPi4oBafiFIHiUC0P+GNPRnVJJrDaGxEIBqAsOkPZPQ58s+jMTKBClXU3D4rcPIfmwYntuNx212ugmCU3/BnDZfwIgn/flcWdEPaBD9EhKAyoJKxB6JRUE8Qwp9Cy9QVdXQ41rZuSSdlJRVptr1q1bStNq+sDq33nGopARwae+CxXmL0f2j7iodVxUuLbuEg28e1Nrx2SA3lxJvBcIiVIjKISalO+Loem7LrSfTFuTi2sEVnl3leChwnFiq53p9Pz/3OptiXWWxP972GNuGb9OL6jmSJDH0j6Ho8r7iDI+XlV6f9cLwP7kngk/YMeGl6P/2MtD+tfZYkLQAFk5SSgYMGC4tDtHwBDy0Htba4ARYTSr9bjZqx3DvHvXTp48Pev+vt0H2quAyJMm86Fd7+wE6nPgeeQ90ZIegAj59fNDp3U565Z6gLHSlWLFE+Z4a0vC0op7XlV0cM7UxRbvtS5AcNkKn9p7VJdU4Nu8YHvz7QOvnoq09PTyovny2tlQiIEkCly5J36empiFpMLxOq6NFP2n2nrIq/e5mUD24AuwDYG0qe5CODhqNMOcw8Hl8Jf4i+YiqRRBVi+qvCdJ6sb6sXPjfBZ32RdOE+kq/UtkTVolIgtKMUs4llsadiMONX29wSoyMzo0GALRxbMNqHF09uqKPTx+V72PPjz9H7JFYLUWlH9B9+GxtpVvT/3nrT/z74F+FNsm0ZT3Top+yFfYAUJZVhj2T9iBqp2otdK78cAW7J+7We5v6+PiGfn62prYwMWparUyLfgUFDc81qSX664CT/Sgb99bf06lFry6oqQGKsmzgl70I3w/+n8ztPK2pz1AZ0c+vLvcpMRHIicrB9pHbEXuY/Wvfnb/uYHXIauQ+ZbAhKAcoKgIkRC2MTKsUtvygoStzS6pLUFmrvONDc4vt3IrcJn39+AI+zO3NtTrPSzqXxInvE5PQyTA88yIAgI2JDWuxNEZ7KbotMIqoWgRSQkJgJmA7FLV5Xld40Lo19ZPODNOV6NdqSCuY2pjCxsdGNyfUAIIg0G5GO7bDaKEFVkk4m4Ar319Bv2X94NFZ9WoLJjDERW15bB+5HcmXkrGkaAmn/nanECdMOTiF7TAYhxb9VK30S05u6OsHUJUwJEn1quBSRq00Es8n4s7qO+jxSQ+DESdLS6l/f6DByk9TPEOtcc7WE/kFBEQiytawBe1SXd2QZZtXU1fpJ1Tv3qNqnw2CIODkZQoQ8qvemEZgJsCMszNg7mCu9XM1tvak6dsXiIoCzp8Hxkkp/o2Koj4Te/uG6x8t+mVkUO81dqSTVelHW3tGuEYw8JcoJmp3FPZM2oNJeyfB2poSN7Qh+uU+zUXWgyz4DfSDhaP+JCjdXHkTruGuiJgdgZL0Emzqtwk9lvZA+9fbK9z3wLMDqKytRG+f3vWCnDahzyGv0i/+dDy2DduGketHInyW9EpSNhi+Zjj6fdtPq1XZqlBeU47kYqpEt40Du6LfgSkH1Nrv5MKT4BnxEDBCN24z+2fsR2lGKWacmaGT8ykDvbApLcnpbsZdLDy5ELWSWrgL3THQf6DM4zAt+g3wG4DdE3fD39Zf6X2sva0xfvt4uLR3UbxxI3KjchF7JBYCc/1dmwOoSj+LqgD80zYJHboWvfC+rS1VlSkWA05mHniKpypV+j0//hwxB2PQb1k/mNtr/zlDERGzIxA6ORQCC/3+3JqTkUHNQywt5c9D6Eq/zLJMVIuqXxB5G9O40s+hjSNG/T2KGwUoJNX2g2+seXIWlyguBgosL+NGYH903dgF12cqdjeyMrFCe5f2sDOzQ3ltOcwEyvVcbF7pl1ueC2tr6lpMX98L4gtQmlGqtZ7A089Mh7hae73h2YD+tyNNKSsnWzMlF3i0DHfSvlqQS+yRWHxn/h2idqmWhcQVqqoaFihp0U/XlX7OYc6ImB0BoatQNydsQavkPMnBvlf3IeGsctZZuiD5UjJOLT6FwoRCtkMxCGrKapB+Kx2lGaU6P/f1Fddx/L3jqK2oVbyxAeHVywttxrfReq+uFihoe09lK/1oW+ziYmpfMzNKDMrL051VtqbkPs3F071PIa4xnAd9ejHf0pI5ca7taF+k9pmOMltPZHHMdWp9p/XYO3Uv22EwDt3Pj88HsirUt/dsvJ8qi2Mm5QUQ5iXqVPTjG/Ph288Xzm213xuFrvTza+Sc2b07NWaSkqRX7t2tKwYLD6dsQQHqemlhAUgkL84haNGveaXfnUz5/fxoRBIRVt9ejRn7Z6iUNd0cu1Z2CJ4YDHMH8/o+n9oQ/WKPxmLftH1699z51t23MG4LpfKWpJVAXKv8/eCHqz9g6r6puJpyVVvhNaGDawesHrYaywcsl7mNfYA9Os3vBIc27FlWSsPYwhjWXtacSeKKyY8BQPXUk9VziusM+X0Ihv6hO0cXcbWYcwuj9L2yubhQVlOGV/a+glpJLca1GYcBfgPkHse57rbDVL9TP1s/TAiegA6uHZTeR2AmQOiUUDgEqTZ2x24ai6VlSznV60wdsrMBHowQ2dob7VxeTDjn8RrmKPZGdZV+xal49AiYORO4fVv+8bPuZ+HuX3dRnMyNUneeEQ9mdmYwMuFGIgRTpKUB5cYJMPK+jfxK2Q+RjuaOMOFTQp+86nUA8PSknrvKy4Fqnjk6vNFB5XGiDSLnRGJe9DytWx7rmuJioEpAfSYWAuWSuAiCwP237+PsjLMqWWY3r/TLq8xrUukHAEfnHMXWoVsVVmyrC0EQMDI1rHFIP+dLjIsAADamNqzF0pgW0Y9leDzlPgJLF0uETQ2DfYB+9gqLi6OyT+ztGx4caNEvM5OauLfQwMNND/G73+9Ivaa/9gnaprayFo+3Pkb2Q915svN4PISFhckct0kXk3D95+uoKq7SWUyGTODIQHxc/DHajNV9NnDc8ThE7YriTHa0rui+uDtG/z2ac9lz9zbew6kPT0FULVJ5X0Xjlk1UrfQzMwMcG62TRUYCwcHU77TF560/b+Fnl59RmMjNReBO8zrhk7JP6vtdGQJ0ry6mrD0BaqKt6+QoZTERmmg9u52NcUsvZAqFwNSwqVjaYym6enZV61jqiH7Rf5xBwI1NKMiT6Oy5WCKWaG1C35i7d4EbN6jf6eQ/gBLKO3Wifj937sX96OtaeKPiKYJoqPprPjZk2XsCgIO5g8x+fjR8go9vLn2DzY82417mPbnbysO1gysm7poI717eWrX3bDOuDSbumQiHQPYX4miUGbu2vrb1/YM9OntgQcICpR1OYvMpO6hAh0DNg1UCN6Eb5naci0H+g2RuY+dvh6Erh8Kru5dOYlKWnKgclKSVsB1GPVyx9myMqte/VoNbKdV7kikm7JyANy6/ofXzqHLPpReFhc3yqOcfn4/nBc/hYeWB9SPXKxSbma700wRSovp9kEu2ueoir2qThrb47GA5DB93/xg9vXti927qc9uwQf46XuTcSHyY86HKlZTaojChENmPsyERG8biIz1uc3J4SHZajS1mnbD8iuwEFYIgcOetO8hdnAtfG/nXMYGAWrsFGix9W9AeRUVAlbFmCYfKYmtmi2Gth8HahBr4ueW5L4h+ke9EYvCKwWpdG5Uh4WwC8mJ0mOWoA+h/O5FREQDZop+u16T0/071kuDV3Qvjto7jzA1TVZpbewLUA4RAAIhEurmRkBISfwb/iQOvHdD+yTSEZ8SDiZUJTKxll9y/7Lh2cMXHJR+j6yL1FuTUpaZGtnd/jyU9MD9uPhyD9TN7lWsQPIK17ORXT72Kt++/zZns6JedmAMxuLnyptpipLxxyyaqVvoBDYvdANClC9ChLqH5wQPqp4mVCay9rFFdzK3eQo0RmAvAMzKcR1B6MZ+u6GEK5/R78Hp8lHNVnDPOzsCoDaO0fh5dj9vGot+4NuPwXf/vFFaGyYKesKvS+6bjWx2Q2nY4SLGkXkjWNk92PME3gm+02qPqyRPgu++o5/0ePYCwsKbv9+1L/bx4seniYW4u5RJCEED79k33oRMlmgtptOjX3N5z/+T9yF2ciz4+feTGShAEunhQ/bRvpN2Q/4cpCb2Yqo3P1M7fDsHjg2Fqo2QDGB2haOyWZpaiJL2pGKVMxUxeRR4KKqlsmVZ2rdQP8CVhU/9N2DNlD9th1ENX+rFt7QkAG+5tgPPPzph9eDbboXAGZe+50kS/48+P498H/4JH8LB13FbYmSl+sGVa9JOQEuyL3ofVt1erVKm9e+Ju/Oz8s/LnEUnwZOcTve8pJpFQ98wM2z345eHHuJh0Uep2dLJhCG8cvh/wPdpZ98HDh9RrGRkNCT05OVR/3sY6upmtGSwcLThTEXnhywtY23atQbmN1NTUoKSkoTcj3edNFqFOoXAwd1BqjYP+7HNygC1DtuCv8L80jldTnh14hrvr9KMnsCoUFwOVxnX9xIXq9RNXlvYu7XF06lHsnUS5tjTv6QcAbca2QcRbEVpJbpCIJdg8YDPOf36e8WOzCf1vV8srAkD1SeUChrPioqdIXpISN2miH48HuNa189FFFjvBI2DhZAFTW25NjKURNjUMcx7MgVOIE9uhcBaeEQ8mQt2KohKJBDExMTLHLd+YDzt/O4OzjGCThLMJeHbwmc7PSxCE3lgBFxQAb7wBrF2r+bHKsspwcOZBPN72WPODMcjE3RPxXtx7aomwisYtm9CVfuqIfnw+EBHRIPo9ekQtqreb3g6zb83mZJJQbWUtHm5+yNkqRHWhhQcmK/0AwDgjCY7Jd5CWwE3RWpuwMW4bi36a0tu7N67PvI59k/YpvU/giNaQhEeC5BvpzOLT0sUSgSMDYeXBUDPKZpSWAt98Q/Xe69gR+OCDBptOmogISqQrKAAeN7r10FV+QUEvinj04kRj0Y8kZff0U4Uu7nWiX7pmot+tP29hz+Q9WrX31EWVpqooM3Y39duErUO2oqq4Cqc+PIX0W8pNBGPyKNHIy9oL5gLd9Ye6lX4LO57skNvX7+AbB3Fy0UmdxaQMXT/oig5vKm91qG3CnMIwLWwaenv3ZjsU8Agecspz5H6m0jj3+TksM1+mk9YDlYWVuPHbDWTc1X7mjyr3XGn2nuvurQMAvNvxXfTy7qXUOWl7z5ycpkKRuhAgMH3/dMw7Nk+lz9Ux1BFePb0gESn3vFGWXYa9U/bi9hoF3pYcp7yc+nfPsT6K3+/+gGup16RuR1f60c8lV640/bz27gWysoBFi4CffmpIQgQogTTvWR5nnvuDJwajz1d9DMZWkB63paUkqgWU+5WzBXN27bQwn5sLWHtZw86ffUvN23/exumPTrMdBuMUFTXYe7pbKS/6fX3xazj95ISvL36t8jmDHYPxy6Bf8FWfr6Q+V2sLUkJi+NrhnHo+YQJa9PO1DEE3z24yKzZ1vSbVIvrpCac+PIWzn55lOwy1oUW/gGY9r3VtXfX6hdcx5LchujlZC1qnIK4Azw7oXhCSRcbdDJTnlrMdhkFx+sPTOPm+bhdRYg7HIOFMgtbsDJjm1i1qInb8uOaVBDwBDw/+fsA5a2EjUyNYezGsqLBMTU3DArWy9p5Aw320QwfKws7Pj1p4qaoCnnHnciiVnMc5ODDjAB5v5ZaorCnasPcEgLAPBuH+kKXIyDNm9sAaUJ5bjqs/XlV6kV6foBcyTSzLcTn5MpKKktQ+lr25Pbp4dIGr0FWl/Zovrmkbv/5+mLx/staSBJKTgYoKyiLq44+l97wUCIDOnanfnzxpeF2atSdN84xkgLqmiuocoKXZeypLZw8qmJtpN9U/CKheRjGHYmAioSpOtLGQsnngZvwR8AfzB9Yy7d9sj7Yz2iLtRhqu/3IdqdeVe+agrT0D7AMUbMks7594H6/sfUVu9WfqtVRk3OFWWXb3xd05tag2MWQitozbgmltp7EdSv3CeE65amVmNt428Onto5M5QmFCIU4uPIn4k/FaP5cq0Ndd+jpcUFmAo7FHAQCzI5SvnLS3p5LARaIG5wtNIAiivsopq0z5Zsh9vuyDyfsmK+1AYWptigm7JihtScxV6q3oTOVXiNHPJdk5EiQXJWP7FUocnDCBun/HxgKLFzfc4xr3aKwqrsKfbf7Ele+vaOVvUJXAUYHo/UVvg3PyKS8Hqo2Uq/S7nnod7594H2vvKM4WblyNO3LdSEzcPVHjWDVlyO9D8MrhV9gOg3Eoe0/VK/3EEjFyK3KRWZqp8jldha5Y1HURJgRPeOG5OvtxNlaHrNZKVSVfwEfk25FoNcSwHBvof7sPQn7D1TevorcP+wlOQIvopzdE741G4plEtsNQi9JSKvsHAFo1G9d0hleLT3RTrv1yjXOVNlzk8rLL2Dl2JyoLlLfw0BYSkQQbOm/AkbeOsB2KQTHghwEYuWGkTs959uOzOPD6AUBP5gNPn1I/JRLKVkUTzOzM8GHOhxi2apjmgTGERCxB+u10VORVsB0Ko9ALHAKBagvU3bsDS5YACxZQ/9/Y+u7yZerno62PcP5L7llm2LWyw4RdE9BmPPvWXkyirUo/3zBLSIyMOWXvWZRUhDNLziDxvH4+k8qDnqyVmUeh17+90POfnjo9f/rtdAj/+xN2aQ91JvppG3psODkBxnK0az8/6mdi3deKJKnqZaChmrkx0vrk0UkUPB5g2sjUY1/0PgSuCsSHpz5UKuZIt0jwCB5SS1JVrgJqzOAVg7G0bCmcfcxeiJUpnNs5w72Tdm2gtEH3xd3RfXF3+A/0x9wncxE6JVSp/eILKfGjtV1rBVsyi5uQ6kGbXiI72WFe9Dy8cUn7vddaYAYnC2o1O7tctd7w4bPCMe34NK1VRzfGIdABr196HWFTwxRvrEOa23vamtri4usX8U3fbxDqpNxYBijHClpQYsrikxY8MstUXwBXFmNLY4RMDIF7R/279jam3orOuK5CzFJ6hRgt/DyIKofP7z7Ybt4dYn4pRo4E+ven3mucdFraqAjWzM4MPT/ticDRuunB+rJSWoqGSj8ZnyNNdF40fr/5Ow7GHFR43Mb2nlzBMdgR3j292Q6DcYqL1av0q090KFc+0WFX1C6YLzPHqO0NrRqai34mViaQiCR6kwTPBZonxHCFFtFPT3gv/j28eupVtsNQC7rKz9X1Rbsd+v8rdLSWm3otFec+P8f5xeOLX13E/b/vsx0G52n/RnuM2TRG7T5f6sDnSz+XuFaMfsv6IWwatyZm+o7/IH/49ffT6TnH/DcGI/4aoTdZgI2ruy5c0OxYBEHAwlGDEgktUJZZhg2dNuDiN9J7TSiDrHHLJrS1p63ti3Z38iAIqi9W4/5xgwdTP8+coSZ+T3c9xeVllyGqFjEWLxOY2ZkhZGIIHNsYVt9TbfX0c3UhYVaSjcqkbFSyn9sCgJpsz7o1C22ntdX6uXQ9bul+cDUm1KRbli2Lsvz34D98dPqj+uokRRhbGIPHBwiSZKTqQRlur76NE++f0JpNJD0BViSI+/pSP5OSqJ9ZWdTnIRAA/v4vbk8fr3GlH/35WVg0vaY+z3+O2PxYpSt6LI0tEeQQBAB4mPVQqX2kYWJlAh6fVx9rVRVQzXCr1cG/DMa4LeOYPSgDKDt2CR4BpxAnWDor58eaVpIGAPC08lQ7NnWgs+7licBc6VlFkxeTh7+7/43H27mTRJpeko5acS3bYQBoWBjPKc/hpE0uQIlL3j29YeNjo5PzKTtum9t7EgSBrp5d8Vmvz1Q+J9N9/Vwtqep6VSr98mPzcWLhCaRcTWEmCD2Bvn9WKagQ69CBukdXlQhhJKI+dM+QdNjZAWPHUvdpoZCy8Aaain4EQaDft/0QMFy31dmy2D5yOw7NPsR2GIzC5/NRWi5CjRGVLaao0s/LmuoTkVKs+Pve2N4z60EWzn5yFnkx7Gal1VbWcvaarQmFRSScioehi+PA+s9IGdSpbq4SVaFSVIkacQ3uZtzFkdgjgGkRgIbrgo23Dd6NeReRc9TrbS6PlKspWBW4Ck/3PmX82GzSIvq1IBVVJkWm1tzvRScNaf38aMzr2jGU68gRMfVaKi5/exn5z/N1c0I1IEkSb919C0P/GMp2KJzHu5c32k1vB2NL3Vif8fl8hIWFSR23AjMBeizpgeAJwTqJ5WWCJEml+ywwgVukG2cmJ4ooKgIyM6kFTh6Put6mpWl2zLKsMsQeiUV1KcOrk2piZGaE/t/3R8AI9T4TeeOWTehFfVX6+ckiLIyqlKmpoWxeB/0yCAtTFuo0IUIZaiuUW+yrrgb+/pvqG6IPaKvSz8xUguAr6+Aec44z1X7GFsZw7+iu9SoHNsYtPVmrFFAXUU1Fv3X31uGnaz/hUfYjpbZ3DHZEwO/zkO/ZXic9NQDg2YFnuP/3fa0ludB/h6IJsI8P9TM7m0oEjIuj/t/XV7olqDR7T1n9/JKLkwFApUWUts5tQYBAYpFmFa0pV1OQeTWh/m/Q1efKJsqM3bOfnsXuSbuRfDlZ6fsCAHzW6zMceeUIxgePZyJUpaEr/TLKZF+IC+IKEL0vGjXl3OjBWl1cjcKEQlQXc+NZrkpUBY9fPWDyrQkyC4vw9tvAF18w08tNHehKvxpxDYqrlR+YBfEFOL3ktE5s8EVVIp3Nf1S55zLZ/5YWFbJVK7iUiToL4BV5Fbj5202k3VBuAnXhqwv4xfUXFMQXqBUjVygtBUhIUEFQiqssscjYGPjuO6q/rlktlXDh05b6/ru5AX/8AaxaRb1PH5erlGWVoSKX28n/qkCP27zKfIAgwSN4sDezl7tPY9FPkXjWWJTPj83Hle+vIOuB8mNLG/zi8gu2Dt3KagzaoKSYQLvk9dg+7FT9/UkZ6HGbXab8RZROvhHwBZi8ZzJGbh+JTDHlr9/4uVpbSEQS8I35Slsq6wMkSf3biXhlaLvDEh4rPFAlqpK6ra7XpAznX1lPUSZLoSyrDInnEjlhYagOsvr5AQ0Tczo7V9uETQvD3Cdz4dpBtR4ruoQgCNi3tje4SghtoqtsH5IkUVJSYnDZRSRJZdfrahyqQsrVFHxv+T3ubbink/PlROXoVGDUlOho6qe3NxARQf1+XkNXx0dbH2H7yO3Ius/uQz2Nub05enzcA/4DpZR7KAFXxy1d6ceE6EcQVLYtABw+DFh62UHoJuRUtSpJkvjF7RfsmbxH7nYiEfD998D+/cD69ToKTkNoWyOmK/34Aj5q+wxErleEznofK6K2sha1FdrPsmVj3NILVSUEtfCnSk8NadCiIV2dpAy0cKxpf1ZlmXJgCt599q7Wjq9spZ9Q2GDzlpzcMHdo3haARpq9J/0MI0v087ZW3g5qxaAVKFlagnc6vqP0PtLYN3UfTn1wsv7awKToJ6oS4fiC45zqbQ0oN3azH2Tj6e6n+LfXv7jwvwtKH7uVXSsMDxiu855+tNWWPHvPR1seYdf4XShKLNJRVPJx7+SODzI/0EqWvjrQ/YZMjEyQGG2NjAzg/v0GgV/XmBqZwsqEyh5Qpa9fWVYZrv14TWmBSBMufn0R3xh/o5NkZVXuuY2rGVZcX4G3D7+tdHJLc5iu9FNH9HMNd8X7ye+j8/zOSm1v4WgBG18bmNmZqRUjVygpAWqM8kESYgCAo7nstSdLS+CbbwAXc+q5xta74fvv7k7NZehknOai36VvL2Ft+7WcmF/Pvj0bUw5MYTsMxqDHbVWpCYJTf8Gi8C/B58kXFOhn07KaMhRVFcndlrb3rKgAXLr7Y170PASOYteqNWhsEHz6+rAaA9NIJI0q7GxU25euWs8qy1J6zlQrqRP9eAI4WlAfchWf6rdVWtrQHzv2aCyuLGc+A9entw/mPp6LoNFBjB+bLaqqqH+3Wn4hKkTlyCnPgQnfROq2ul6TahH9WEYiUXzzSziTgE39NyHpQpL2A9IC8ir96D5GurL3FLoK4RTiBCNTKWnDHKGyoBLlOeWcW6DmIhKRBH+0/gP7p+/XzfkkEiQkJEgdt9dXXMff3f9GSboO0mMYQiQCjhwB3nkHmD+fWmjnGlbuVvDq4QULJ+1bToqqRdjYZSO2jdim9XMxBS36BQUBfftSv1+4oFnmdOuhrTFi3QjYtWJAjeIA8sYtm9CVfra2zByvRw9q0byoCLhwgURxajHyY7lT1V5bUYuAEQFwjZCddEOSwK+/AnfreoYXFFDVi1xHW5V+AOA8uguKnQM4U+l3649b+M7iO2Tc1m5AbIxbeqGqFNTfRlf3qIuHUHXRr+LWYzgk39VZRZjAXAChGwPlGjJQxeqGrvZLTGwQAmSJfo0r/ej7XWN7z8YkF9WJfjbKi36uQldYGitnOSmPQb8MwqBfBtXHy+TnWpFXgVsrb3Guv6YyY3fq0alYlL4I/b/vj6Cx3F/0qa/0k2PvGTQ2COO2jtPqeNJn0kvrehUJ3fHoUUNC0rlzbEUEdPfsjr4+fSEhlb/PuIa7Yv7z+Yh4K0KLkVE4hjgieHywTuZAyt5za2upxU2Aug5ve7wN6+6tw4OsB2qd17mu/Ribop+RqRGsvayVdsfo+E5HzLw2E2a2+i36UX3gqH8nB3MHCPgCudubmgK921OVfjnVL1a60gk3zUW/6pJq1JTVoKpYetVLC+ojkUgQG5sAcbkt/LIX4YveXyjcx1xgDgdzKstKkcWnqWlDRW9JjSkcghwgMJP/PdE2Y/4dgx5LerAaA9OUlAAiogISXqXKFdTOFtRFtFJUidIa5cpsRRJK1TPiGdWL/WWS3HpXCnqN4vGWxzi79CxqK7lhy81l6iskzYoAALZmtjKTr3W9JsVd5aOFetw7uWPoH0PhFqnZ4gMb5OdTi3YEQVmPNUfX9p4kSaKqqAriajEsXTSfzGuDexvv4cxHZ/DGlTfg1V15K6KXEZ4RDzY+NrB0Zf+zrCquQn5svl7Z8P7zD3Coka3906eAWEw1VucKNj42ePWkbvqZiqvF6PphV9j6MaTC6AC6n1+bNkDnzoCZGTVxjo+XvViqCMdgRzgGc6fS+Pqv1xG9JxoTd080qMU0Jiv9AMoCb9Qoyhbz5DEJbo38A34D/DD16FRmTqAhxhbGCntPHTkCXLpEXYMIgkpMyMkBPDRzWdQqIlGD2KAN0c+lzm2JqQUxTXFo44C209tC6G44Y5GGXqgqFFEL+5pW+nlaU4tjqoh+aXtuwu15CbI6an9BGQAy7mZA6CaE0FU7n6ey9p4AZeV55w6QkNAg+klLGGx8PLGYShy0sJBu70mSpFqVfkxBW77b1CVKMyn6WbpaYmHaQvAFHHpoUwGhmxA9PlZ+4a68phx/3PoDHlYemBY2TaeV7PS1gBaupOHSzgUu7eT3UtIlWQ+ykPcsD62GtIKpDftzE1owdRO64eGthtcvXgRmzpRu46ttjk07pvI+AjOBzpLi2k5rq5P+uapA3yd5PMDEVIyo3CgAQFePrmodj2l7z/6+/bF74m60slNtElSSVoKq4io4hShvq6fvlJQAlpVt8Id/EnoNVO7mJM/BQFal38D/s3fe4VGUXRv/zW5674UUktB7BykiVcSO2EVFxY4CCijqq74W7L0rqCgiYEFEeu+9hg6BdNLbpmw22Z3vj8nsbpKtye4mvJ/3dXFtmH1m5nl25mnnPuc+745l7Ltjm1VXR0BTruH4kuNE94kmum/rVf2yF1VVhjiehk5P5hAfGE9BZQHppen0iuplsWxEhPRM8/Igwq+CWnUtgXFO2PD8P0ZJCaSHzedE/NM8uOI+Fty8wOZzfT18GdBmAAGeAVTWVOoj2C3BWN7T30Na/xdWFRAcLOVvLCqSojyveuUqhs0Z5vB0IalbU8k7nkeve3vhGWA6Gu5yg0z6efiXABDkFdRidWmIfyP9LgOEdgxl4NSBBMZffoOrHOXXtq3kKdIQrpb31NXqeDf0XVY/tdo1N2wConpF0ffhvq3K6N6ace/6e7n6vatbuhqM/O9IZuXPcll+weZCrYb166W/771X0uuvrXXcputyhGeAJyNeGUGvey0vflsLNBrDGNuli/QMO9Upbly40Pzrt5ZoY3WJmqKUItx9W9az0NFwRrLnXnWvbn6RkuEvD6fHPT0cd3EX4GRdPu877zQQfa19TJKfoyA4Jr9NI6Sl0WPTJ5TsSHbCxe1Hpxs6MeGnCQTEtLIs5Q6AbKgq1Dgo0q8J8p4j37+OswMnUVLi/FxXWo2W7/p/x7pn1zntHrbKe4Ih0m/PHqiqkua0uDjTZT08DPsKmUgzFelXoi6hXCN9IZOwtuKFjS8w/IfhHM87btd5DSGKIgG+tfXq6ggolAoCYgJcEgVkCU15T3OO5nBmxRlEne0np5emM2fjHKaumupy6eq4wDi+uu4rfp7wc6tZG1nDiaUn+OOuPyjPbR3a/TLpF+rZhsxMac4MDJTG3f37W7hydqIsq4zSjP8HCTpNQB7T/f0hrTQVda0aLzcvkoJNeHfbAFk+MD/fMXNeYnAit3a9ld5Rve06b9F1i1h842Kbym58YSPJi1rHmqw5KCsDBW60C21Lz0jbyGV5Hs0osz3Sr7VAla1ixZQVnPz9ZEtXxaGoqlJS5p2MOnQfFTW2/fhyXj9Tz7EhjCV4P+vwGcsnL29yXZuLsswylj+4nHOrzrVYHZyB0lJQu0tORUGeQXafv+/hfWy4b4PZvJwNYUreM78iX++ILDsmh3UOI7JnJAqlY2mjk7+dZPXU1XbldG7tkOdGN38pTPJf0u9fXDZIT4eZM+GXXyStYYBn1z5L/2/78/tJy3l5wLK0J7he3lPprmTQtEG0G9e03FCuQLur23HDtzdc9pIR/6vwMsVeX4bYuVMyqkVHw223GQzsGc7PS283jv50lOUPLXeqoUUUxVaRa8AepKRIRG1goCEaKDFR+mwu6ffb7b/xeafPm3cRB2Hkf0cyM2dms6JoW2O/rapL0ytHvDsC8oa7ogKGvzicHne3HtJvy3+3sPaZtZbzPNURfImJjpd8chaMpT2dYYcOTfBHp3CjykXrpNYEV/ZbUTSQRk/1ncWLV75Ip7Dm5S1pCunX/spo1AERaDRQXd2s21uFTqtj9Fuj6XZ7N6fdwx7nBnn+kvtUUpJl5QFjiU8wRPoZk36l1aX0i+5Hl7Au+LjbN9juzNjJ9vTtHL502K7zjFGWVcZ7Ye/Bho2AY3M1VhVVUXS+iFp1reMuagdqa2HaNJg925D/RYa1vrvpxU0svnExW1/bavP95Cg7uV+5Ej7uPjzW/zFu7HSjWcKxJK2ED2M/ZPMrzUys7CD0uq8Xt/12W6uJyJDzIbpXSlGT7dvDmDHSdy0p8Qn2O7l93vFzVjy8wkm1MWDNjDVOyadkDrbMucZjuhzl1zmss9U8YuYgS9y7Ys6zhP5P9Gfg0wOtltNqtOx4awdnlp9xQa2cC5mcs8dhbUCbAcwZNofJvSY3+s440s+4S1UWVHL4h8NcOnSp6ZV1APzb+HP3qrvpeW/rip5tLmprvbkQ9T4bEgfx6d5PbTrns/GfkTczj8f7P261rEz65efDoGmD6HJrl+ZUt1lQZas48sMR8k608s2hnSgtBbVHnQR2QPNURmxBbEAsw9sOp3NYZ0K9QwEorCrUk36yvCdI8rzqEsdK8w6ZOYT7N9+PT5gDjSAtDL0TsE8J8C/p9y+MoLRBR+/7od+z5JYlLqhNfWi18OGHcOYMLF4Mr7wiTeKnC09z8NJBVNXWPUnOnpU+zZF+srGzpsZ1eXuu+ega+k7p65qb/QunIzc5l/Wz15Ob7PxwEKVSSefOnRv1W025ht0f7m7xxaw92LBB+hwzRjJUt2bSL217Gke+P0JVUZXT7pG9P5v3o97n+JLmefS7EnI+v65dDWSDLKN8sZkpfgLiAgjtGHrZEaGmYK7ftjRkZxdHkn6ysVujaX258E79cYrzq89bjNCQCb6IiPqepa0ZMkFhb+J1WxHVNYQTI56kKKZ1ELhb/ruF1U87Xy3B1f1Wo5HWvQCPDHiQN0a94bBIvyxVls05ozw9RbwFNYK2tt6m2xlw93Zn2PPD6Hyzc3KqiaJ9pF+bNuBuFNBtbu8gQ44elPugKXnPhKAEDjxygJNP2u/Z3zNCMgweyz1m97ky/KL8iOwViV9cUL26OgInfjvBZx0+I21bmuMuagfy8iQHo9OnYeNGw3Fb+m6nmyRC3R6Dj0yeu8Ig1hR4BXkREBOAT2jrMGKFdQ6j661dcfdpHSoJ2eVSpF91oTSu9uoFo0ZJ3x04YJQPx4X4Yt8XRLwXwdRVU+06b/Czg/XSvc7E0QVHSVmX4vT7gO1zrjFRdDJfGle7hjf9t/D0NEi7OipCbNmpZXyx7wvKqm1/qfo/2p8rpl1htZzCXcGMjBmMfa/l5Sqbi7IyyA5eyvz059iaapsDRo/IHswdPZc7ut/R6DuZPNRqDY6NAOU55fz94N+cXn7aEdVuMjz8POgwvgPhXf531LSUSiUREUmUe0tGgS7hthFysQGxhPuG2xQ1L0fj5uVJjrgDHh/Q5Po2F236t+H50ucZ8ETL1cEZKCkxRPo1J7WArQ4sd/e4m62Tt/L8sOcJ8ZaYvqKqIr0TRmGh9Jl/Kp+3A99m57s7m1wnUwhKCCJhRAIKt/8dOkpP+nmVABDsZT5dkKttUv87v/JlCluSOHoGeuLp73ytW1GUDMWyge3vv6VIEh8faUF25Ag89xwUV5UAEOgVaJH4E0XrOTl8fAzGalfl9WvNKMssY/7g+Rz7pekGhv9vKE0vZdd7u8jcY7snfVOh0+koLCxs1G8LzxWy7tl1nFlxeXj9ZWfD8eNS35M33LKEVqbzf0a7MXruaJ4ve96phpRqVTWBcYEExF4+knVyNF/HjoZjcqTExYvNk8kZ98E47v7n7laxGDvw9QFSt6Y2+Xxz/balIW+IvR0Y1G08p6bty+Wbvt9w+PumR6k4Eo8deYwHdz1o9vvqaoNBPCLCEOl3uch7OkXak8bERkvj3MpznPnb+XOdq/utvAYVBNNy9E1BtF80ex7aQ+q0VARsCwM9/P1huq14B//Ci63mmTcV1dUG5wNb5D2VSikdgAxreWnla8p90JS8Z3Mgy50dy2v6mlyhVHD/pvtpd9cgwLH9OKpXFENmDyG0Y6jjLmoHCgoMfy9ZIjlwgm19t9/D/XhFfIWBU61H1siQI8Wam2uzqTiac5Rfk3/VEx0N4RXoxZS9Uxj09CAX1+zywFVtr+KeHvdQndoHkEi/+HjJWa22Fg63wFJFEATyK/O5VG6f0+bI10a6xIF4Zs5M7vizMbniDNg65xqveeRIv27hTY8WFwTzueCaikf/eZSpq6eSWpLqmAsaQRAEAmIDLsu0Ow1RVga5gf/w47l32Zu1t9nX8/AwOO4Yp+4JTgrmrhV30eeBPs2+R3NwuUgz2wOdTkdGRgnlXhKh2iXM8VF4xpF+LQ1BIeAZ4ImH7+WRTsdWqFTNi/R7c9ubhL0bxn82/8fucwfHDeajcR/x5IAnCa1bTspOhyHtQuhxTw+i+zk2B2ZNZY1d8u6XA+S5McwnjCFxQyz2RVfbpFremvf/HLZMPvesuoebF9zstDpoNPD77/D44/D00zBlCsydCwsXSt9PmQLvvy8ZQTIyoEBVAsDEpROZvHyy2evm5EgTvpubIU9HQwiCIcrBVRKfFzZeYMHIBWTubX3shipbRVFKEdVlLahvcZkh4aoEpp6dSu/JvZ1+L1EUycjIaNRvQzuGMnnbZHpOujzkIuQov759ISxM+rs1R/r5hvs63fEhaXQSjx5+lPih8U69jyMhGw9Djex9EVE14F5JVVXrJ0tsQU1VDSsfX8nBbw42+Rrm+m1LwxnynsZzqs7Ti4q8ilajly8oBIuy1bLDka+v9O9yifRzRsSmMQICwEuVT8je1aTvyXbOTezAlD1TeDzZuhxQc+Hqfis/R4V/HjvSt5Nemt7sayoVSgbFDiImIMbmHGQR3SPQdOtNraef00m/lPUpzB88n9QtqU65vrwBdneXnAdtgfF+wVqkn2wothTp1xzoSb9mRPrJcAZ5H3tFLGPfGUtQQpDjLmoHZE9wkIyBcp5oZ/XdlpT3BHhv13vc/efdrDy7skXuby/mD57PN32/aelq6PFIv0d4Z9BC3DNH4eYm5aIGg+NaevOHXLsR6St5F+VVtM6FhtJDiVeQa2Sube23MjEXECDlgILmRfqBwWnKUdGecl6rnPIcm8/JO57HvEHzOPyDZfa5qqiKktQStDXaZtWxpSGK0rOsdpd+I7kv2IL00nR2pO+gRF1S77hxbmvjZ+nu407H6zu22Fwl4+iCo7zp/abLomddAVEUOZZ6Aq2yHAVK2oXYlsKooLKAGWtmcP9f91staxzpl/xrMgtGLaAkraQZtW46ynPKuXT4EpqKViZn00xUVYn6SL+mrnEKqwq5pLJfdaxreFemXzGd8R3G6yP95Jx+Sg8ltyy8ha4THRvZPn/IfD7v3DrSyDgK8pg3JvJudj64k1dGvGK2rKttUv+Sfv+CP/+EBQsgK0vy0BFF2L1bIgN79pTk/xISDBEkcqQfwKpzq8xG+8n5/JKSDLINpiB75Ja7KM+4VqMl+2A2ZRktoCNiBTEDY5iVN4v+j/Zv6apcNvDw8yC0QyhK95aT7vPw9aDtlW0JaRfSYnWwFaJoyJ1x9dWG48aRfq2MG0FXqyP3WC6FZwutF/5/BFO5GL4+9DnrerShyj2rWRKf1apqNv1nE8m/tmyieoWbgvu33M+QWUNatB7OgEwyODLSD4yiXAICeSbzGbsiKZyFkrQSLmy8YNGhxVjaEy6fSD9nk35+fuBWW0Vk6j7Orm2mbq8DICgEl6hPuBrycywJ2cjwH4fbZAhxBmIHxeJ1x01UBkY7NP+bKVSXVlOaXuo042VT8l3KpJ+XF8RYcXa2JdLvvmX30enzTiw7tcy2ChihW0Q3BARyynOaRQrkn8rn1NvL8S+46PRn6krIkX7yHLZ0qXNlpfXyni0U6SffVyYfTeHYwmNse2Obq6pkEZG9I4nqHdXS1aiHo0elzy5dDI4A8XW+dmktoFIb4SstOHIr7Fto7HhnBwtGLWhWpEJJCezbJzlbak0MwepSNRm7M6gsaF0JfY0lm9dMWkPxc8WMazeuWdeU9zGOivSL9peiUuwxgHuHeFN8sdiq43XyomQ+SfyE9O2NWerq6sb5TVsrKitBpzMi/fxsJ/2uW3QdV/5wJfuy9jX6TnbGMWXXa6n8szL8ovxIHJWIb6SD5ABaCdIqJemfcGV7PJS2RcApBAUf7/2Yn47+RFWN5fQp8r6spARKsyrIOZJDZX7LjEsnlp7g277fcung5ZNSxxYUV5WiVUqea01Z48iODrbOZS9ufJHw98KZu31uveNyTj+Z9HMW2o1rp5d5dyYqNK6TErQnnYGr8S/p18px6fAltr25jeILzkvskVLn7HLttVJ032efwciR0oL86acNG3V5UV5WUwKAu8Idda2av8/8bfK61vL5yZA3566S92x3dTueL33eJVr8TYWgsNE68i8AKUT8woYL1Fa3zGKyqrgKnbZ1SQeaQ2WlwTu7Xz/D8TZtpL5eWYnT8wjZi2pVNV/3+pod7zgnmf3Od3ey/IHlDk9S7GyYkhXMUmXhrfQjN+gvvfxnU+Dm6caOt3Zwcqn9eZAcCaW7koSrEoju41hZiZaGTicZB8DxZJEc5eIqRxpbcHrZaX4e8zOXDpvfpDUk/Yw3ma0tP6ExZLLIUZKCDaFQgCI2huQRU2l/f8uS36JO5OLmixRfbGWThAMgP8dabymasrn5/GSsPreaWetmseb8GpvPkcksZxNEXW/tyjNZz9BurG2e4faiKRvg3r0lmc/+/aV33xIaRvrJY55xpN+ZwjOcLTxrc6SlMfw8/PRe88m5TXeA0dXouPjnEXxLMikrc5xj1frn1rP4psWOuVgTIJN+48dLqhGFhbDTsWlf6qGlI/30RIIFKcjkRclsf3N7q1AWuP6r67np+5tauhoA1GhryCjNIDVdUh7oZGTrk+0LLaE0IhMd9pL6xSnF5B7NbVa0yXvvweuvwxNPwO23w+bN9b+/dPAS3w/5vtXlGm+49wjyCsLbvXnea44m/ZoS6ecX7cfM3JlW8/pF9orkimeuaCSrXFQEjz4q2c5aWTYBk5CfY7WH/eOqXDajtHGnleffhlGbP1/9Mx8nfGx3PR2J9te05+6VdxPVy7IzxPHj8OWXoL5MzALZGmnDH+Npu7RnsFcw/h5Sx7OmbOHvb3DSSLhzEM8VPUeb/o5ZI9uL2MGxjHpzFMHtzOdLuxxRWV1DXMEDDAqc0KTxVJ7LbB3zSqtLKagsQF2rplZXy+6M3aw8u5KgYGnwMib9Lmy4wIJRC8g+4Di1mbHvjOXq9662XrAZOFt4lnaftuPHIz869T4y/iX9/kWTkbYtjc0vbaY0w3kaP/Iie/BgyVszIQGeeQbefdfgaQ/Solwn1KARJcvII/0eAWDJiSUmrytH+tlK+rlK3lOhVDRp8+8KHP3ZdQm7/5ew5+M9/Dz2Z7L2mve+dRT8TSRuWjpxKR9EfeD0ezsC8iTu61tfbsvdHaLq1sCtTeLTO9ibkW+MpPsd3Z1y/ez92Zxfex4P/8tLH95YYkdGmE8YpWIWeYFrmhXpp/RQ8sTxJ5j468TmVbKZ0Gq0dhHqoijlOmy44TbVb1sSxgnunRXpV1EhOQ5tfW0rFXktmzQ3aWwS13x6DZE9zHsSNyT9fH0NhGhrlvh0dqQfQGCIkmq/UMrKWnbtoi5V89Oon9jxlnMcMBrClf1WdjzTeEgG/TZ+jjForEtZx/u732fTxU02lRdFkZrflhN1fvtln9OvKRvgtm1h3jyYPt162YaRfqbkPdNKpPChtoFGyQLtQM/InrTxb9NIxswehHcLZ2raTHLaX4lG4zhDYtG5InKO2m7UdjRk0i86GkaMkP5OruNGndF3f57wMyvvXsmg2JbJmRftZz16aPxn43nixBOuqtJlg/NF54n/OJ6ZudIaIMRIGEUm/S5dcr2DjxzpV1ZdhrrW9o55w7c3MLtwdpOj3kXRYCdxd5favW5d/TLBScGMfX8sba9s2tjVFNjSb03tPZp/X+nTYfKevvaTfoIg2GQfantlW8Z9MK5RDvhvvpEcHzIyIDXVruq2CMrKQCtUoXGTDAP2RBfFBUjyQHL0tTHM5WeMHxZP+3HtW4VDhCXU1EiE/OrVsGVLS9fGNuSL0jonwdd20k8QBBKCEgCs5r4UBMPerKCgZfchMQNiuPKFKwmIaYXMSjOgVIfTK/V7Xu38Z5POt9fRoUYrOeC4K9yp1dUy5PshXP/r9bj5SoNwaakharm2upbs/dmUpl8+m5KiqiKuX3Q9uRW5fH3ga2p1zg8MkfdsLxy/jtgPY1l9brXT72kr/iX9WhhKpWVJwt6Te/PwgYdp08853hS1tdIiGww5vcwhPh60igr8axMJ9grm/l6S9NHWtK3oxPoWVq3WEEEoa/Wbg6vlPQGyD2Sz7/N9rWrhIYoia6atYdvrrUMW5nJC55s7M+bdMQQlBjn1Pkqlknbt2jXqt4mjE+lxTw+n3ttRkEm/EBNKpLLEZ2sj/QCGvzicdlc7Jxrhtt9u44kTT6BQXj5TYk2NwXDo5yfy1va3SC9NZ3z78QAU+m/i3MXmWRbDOofh5mVBm9kF2P/lft7wfIPMPbblYF22DKZNg7VrDcfM9duWhEz6ubkZkt47CrLBu6ICMvdksuWVLWQfbNlccBHdIhj01CC8Q8wznLKMp+xsZLzJbM0Sn86SaTVGQAAoajVc3JhCeU7LhXAq3ZWM/2w83e7o5vx7ubjfys9R7S71FTmqp7mQPeJNGcdMQRAEqo6cxr8g1emRfhc2XiD512SnKRU01es1LMy2HIDGkX5qteEZygZkda1aL3UUH9i0fL2/TvyVrGeymNi16Q4wCqWC0HhffZscRebe8ecdTE+d7piLNQEy6RcaCl3rxFNOnnRe3+0a3pVrO1xLmE+YQ69rK+To32yV+fk0pF0IwUnBLe5cWq2qZs2MNZz952yL1kOGHKXpo5XGVeNc1EFB0rpFFKUUA65EoGegXg7PlXn9ysqkdaAgwDvvSMdSUupHAQclBDHk2SFE9rRddrE5sLXfymTOhpKvGfvzWH4++nOz722OKGoq9AbwCvucIvJP5bPtzW125yvbtUv6JyO56YHhLoNKBeq6KD8fdx+CvIJsPlcm/TLKzEf6NXyWV718FTcvuLlFx8ZjC4+xfvZ6izKjGzcabCWXA3mrVCrpr3ycrukfMzzyRrvOtZX0A8N+LCezljN/n7F5X/4vbIOs/uPVxBSuxvKeDe3ypiCTYG4KN7zcvPBxlzxXa9wKkacAeQ/S/pr2PF/6PF1usZ1Utnjv6lr+uv8vjv501CHXa4hSdSm3LLmFc0XniA+M5687/8JN0Xx71sGDUnS+vPZtCHnPU6DJIkuVhUIwb1d0tU3q8rFw/o9CZyX+3yvQizb92uDh55wIlEuXpIgIb+/6C3BTiI8Hd20QI45eIGdGEb2jeuOp9KSsuoyUovrRaYWF0uDl5mY9J4ejIv2OH4eHHoK9e62XPTTvEKufWo0q20GrSwfh7pV3M/qt0S1djcsO4V3DGTprKIFxgU69j06nIycnp1G/Hf7icK75+Bqn3ttRkBeypvp7ayb9nA3vYCda7J0AeTMlCJBcvIcXNr1A7697kxicSJRvNFplJWeqdjRLNrmmsobsg9lUFVvW+ncmAtsG0umGTvjH2BY1sKMuAOnMGcMxc/22JeFMosg40q/LhC48vP9hEkcmOv5GdsAWBxs5mk9OGA8GAvByiPRzlrwnSBFNAfkpHHlmIWdWSC93Szgtefh5MHDqQJJGJzn9Xq7ut/JYWaFwrLynvaQfwNDfZ3Bu0CSnk377Pt3HX/f95TRJeeOcfs6AcaTfhQuSwTwkxGBAliWrfN19CfFuWs5lW/PjWENlQSVRxadwqy6/7CM4ZchS8WFh0Lmz9HdWFhQXt7451xEwlvc0N/5qa7SUZZY1S/bREajIq2Dvx3tJ29YCifJMQCZKPTXSuBpspMwmCC0n8SkIAlfGX8moxFF2RQKUZZVx6s9TlGU1LTQtu443DguDpCTw8JBIwOwW9M+ydc6VDZunyney4cIGq9KAtqA15PQDyDmSw+aXNpOxs/6LWFVcxd7P9lJbXcsfd/3Bqqmr9N+pVPDVV9LfMjFy7FjT6+4qlJWB2t0g7WkPGWdpXeNoAteROLviLLve24XCzbQJXKuF3383/P9yIP10Oh1ueV1IzH2aAdGWpWkbQlZAsIX0k53FS0pEFt+8mD0f7bG3qg7Bhjkb+H7o907LRd1SKKoqRKuobLJdQI5ar9XVUlRlPSFfja4u0k8peR6HekuGwWJ1kX5+lm2GCqXCofuE6tJqjv50lMy9jieOT+afZOC8gWxN24qfhx8r7lqBv4c/X+z7goeWP1SvbG2txB/Ykoe1oEBSQdy3D1aubPy9KBrGvPLaEgCLjhSuXh//S/q1MKwZbUrTS6mpqnHa/eXFdWysIXefOQQHS0YtUZQ2de5Kd3pF9QLg4KWD9crKG9qgIOs5ORyV02/bNskwaEso/oAnBnDfpvvwCXWiHpedEASBuMFxxA9rmjfyvwBdrQ5drfMGUVEUycnJaVURovZCNtIYb7hlyKSfqz1tbcGh+Yf4rMNnFJw2417TBIiiyJrpa7i4qRk6mC0E45wa6y5IYW1j240lwDOA8R0kAjo/cHWzJD5P/n6S7/p/16K/T5cJXbhj2R02EfoVFXD+vPR3fr7heGvst3Kkn7NJP78oP9r0b9OiEZu16lreDXmX9bPXWywnE3vGsuKyAaU1k37y2sWZ8p4BAVAe2pbwe68mYUQCqVtS+SDqAyoLXaSL3gJwdb+VydtyoeVJv5AoDxAEp5N+V716Fbf/ebvTPO+dnd/CONLPVEoBvbRnUNsWj7w6v+Y84ZuWElBw0SHPVRRFkhclc+mQfUZtR0GjMez1wsKktYhM3Jw86fi+m1KUwts73mb56eUOu6a9kOU9K2sqUWlMW7RPLDnBR3EftXiqhqC2QUxPn86QmS2bB1aGTPopq6RxtaHjofzupDefP7IbG+7bwMb7NpIUbLszS/r2dJZOXErGrqaxlDK5Fx0t5TBNrPPLkscxgM2vbOa7Ad+5LN+4rXOuPK6nVkg5v7tFND/y39HynsPih/H7bb/z4bgP7Tqv3dXteGj3Q3S+uXO94/888g/7v9iPQqkg73hevb3o2rVSRExcHMycKR07caL15/VTqSC4fCgveKWz7I5ldp0bF2g+0s8cgZt7LJcVj6wgY3fLeRZf/+31PH3habOk37ZtkrKIW92WKTXVcTl4nQVRFMnPl8YIY2lzW6CP9CtNtVpWryJT7c6EnyYwbM4w+27mIKhL1JRllqF0bz3qPY7AJveZrO7ry2+Z9o1ZMjyUHgyLH8aYpDE2SVXrST+FRPrJjnFFVUV6gtc4r9+lQ5ccppLnE+7DCxUvMPadsc2+ljHWp6xn4HcDOVt4lriAODbfv5mekT0pVhczbc00vj/yPUdzpOjCmhp49VWYMwfefFP6vzmIInzyiWGfeOhQ4zIVFYYxv1Qj5b0P9jafd9LVNql/Sb9Wju8GfsdPo39y2vVl0k829luCKU+827vezhP9n2i0UJY3tEFB1q/rKHlP2Rsny4a0bpE9I0kcmdji0nXGqC6rdiph9b+Ok3+c5J2Qd7i42bUERfqOdH677Tey9js/n6AjUCzNQyYj/WSJ39ZI+nn4euDm5ebQzW/R+SL2frKXU3+ectg1XQV5M+XvD1tStwAwOlGKEpYlPvOaSfrFDYljxH9HEN4l3GrZ1oDjxw2bM3PSC60FzswDJ2/M5Dm1trqWgjMt94OoS9VE9Y7CJ8x8YzUaw9gkE31gIABbs7ynTOA6m/Sr9fBBOXQwoR1CyT+ZT0VeBYVnCp13UxM4s+IMX3b/0uXzrCtQWQkiIqU655B+WaosmyR3ABRlxQTknae02LmezNF9oul0QyenXV8mhZxF+smRfmq1ZGSF+ikFLpZI76ls2GoK8iryGLlgJN2/7N6sTXriqES48QZUIfEOMWxryjX8ec+f7PtsX/Mv1gTIxiAPD8OcI0t8njrleIL14KWDzNk4hw92t1z+bF8PX7674TuW37ncbARoZM9IBj49kKC2Qa6tXAMo3BQExgXiG+HEEHQ7kFUm7ZE8qyX5n4b2gZYk/ZqCuCFx3PLLLcQOspIbxQzk1Cpt6qaZ9u2lT9lxDUBbraUir8Jpak9NgVYrGTdFdJwvlfZO3cKbT/o5OjosNiCWiV0n0je6r13n+YT6EHtFLO4+9XX3Bzw5gPhh8SjcFDye/Dj3rr9X/91Jiftk3Djo1ElaC1ZUSNHnrRllZaDAjfigOLqGd7XrXEvOTOZIv8qCSg59d6jFHFVAUlALTjRtiBdF+O036e/bbpOCFioqDM7SrRVpJWmc8/oTldeJJpN+tkTEGu8te07qSVTvKDtr6hhc/9X1TE+b3iL3diZKkAaM5qQW2P7Adtbfu17fPy1Bn9NPjvTzkQyDhVWFJkk/vUpeVvMHaUEQcPdxd/jctiN9BxU1FYxMGMnBRw7Sv01/QBqvbu16KwBf7P8CUYSPP4ajdeqiBw5IMtvmIv7WrIEjR6T1Lkhju2yzkCGv6718tJRVS/+xRzLZ2Wg9jMe/aARRFOnzYB/82zg+GboM2bhvC+kHUB61hu28RM3BKxk+/COeHfKsyXIy6WdO0ufD3R9ytvAsH1/zMb6+knhxc+Q9RdGwUcjKkv5vzalX1ImoS9QWcwy5Ehue38CheYeYkTEDv0g7Z+1/QUi7EKJ6R7ncm7vgdAEn/zjJgCcHuPS+TYWlnH4y6VdUJC10nSlXZy+639md7nd2d+g1Q9qHSB5/l1EuPxnyZso7oIrdmbsBGJEwAoAxSWMQUFDufYq/NmYxfnyM3mvRHoS0D+Gql69yUI2bhjUz1hAQE2CTt7q8eAOJ9LNlHmgpuCLSTyb9lk5cyoX1F5hTPqdFPCP9Iv24f/P9FsvIkZleXvU9VS+HSD9nErgyZIOYvKno+3Bf+j/e3+Xzna5Gh65GZ9ZL+nKGFLEpcm/ke0R1yHYY6RftH42AgEarIb8in0g/6/mZLi7eR8d9e0j2n4ZWG4SzUj+IoujUd8jZkX4+PlKUjFYLhw9Lx4wj/XzdfekX3Y+eET2bfI8AzwC9Y01hVWGT88n5t/EncGRfajbikEg/pYeS2/+83al7REswzucnv0Jdu0rGkZMnoX9/x95PNi7HBFjJGeFkTOk7xeL3kT0jGf/JeBfVxjyqiqtQF6vxj/HHzbPlzT3Z5QZ5Tz8/g/FMRmsg/ewZDwPjA+lxd9NzuTck/eRxK8UoQHTM22MY8/aYJt/DGZDXlZWeF6mqrcJT6WlXhKQ5OFresznQ1mgpPFtIeNdw/fuQMCKBhBEJ+jLycVGE06elY126SERRt26wf7+U108mc1sjmjM/xwfGM2fYHOIC4tCJunq5q8w9y7ihccwqmNWidrei80W4+7rjH9143szKkoIaPDzgppukdBEZGZCWJkWzt1ZsvLiJI+0eJax0DP7+lhVVGuKa9teQNzPPpnVNQ4dSWV2rNQVQXM4oVUqDf/uQdi65X7vgdvSL7kekr7QnsRbp1/+x/nSZ2AXv0Ob336riKorOFxHSLsSh48GrI16lS3gXbulySyPHrMm9J7PkxBI2p25mwQIpqlephDvvhKVLpfRg33wDTz5Z/5pqNfzwg/T3fffB5s3SPH34MIwaZSgnj6fegQavvtZE+v3v7dovM1haXAqCwOi5oxk4daDT7m8s72kLFEGZlPoeJE1lWbbEWN6zIXLKc3h23bOsTVmLKIoOkfcsLDScr9HYFuXxeefPWThuYdNv6mBE9Y6i802dW41X5uWGqN5RPLDtAZLGOC/XkCAIhISE1Ou3faf05cWqF4kbaiNz3sKwJO/p62sgA1tjtJ+jIQgCwYnBBMY7NxekMyAvLkr8dqPRamjj34YOIZLVINg7mJ+u/50bU9LIS4nhr79arp7NxeH5h22WyTpyxPC3RmP4jUz125aGM6PDGs6p3e/sztDnhqKtbr35D4ylPY0f0+WQ088V8p6yA5W8tlK6K1vkfe5ySxemnplK2yvbOv1eru63lZUgoGBC3OO8Pup1fVL75sJD6aEn+myV+Ox1V1dSe95ArZuXw+TOGkIURd70fpNl99kn62UP5Lo7K6efIBgMluo6EQBjI+s9Pe/hwCMHeGvMW02+h5ebF1F+kke7LBfaVAQFgaCtpaS4+bI+bp5udJnQpcmRRs2FcT4/Gd3qAn5SUgR8fBzbd+VIsRj/liX9LhccX3ycT9t9Svr25rNotbpa9mbutSvnXUPI8p5eNW1MOh3KpN+lS9L6zZX4YNcHhL8Xzsx1M112T2N5T4B2dXbelJSWkxO0Zc6Vx/TqEEnfrGdkT5SK5nulOFreE+Cv03/x+b7PKay0L1RrzfQ1fNX9KwpOF7Dh+Q3kn8o3WzY7WyK3PDyk3IwAPet8TJKTm1pz10ClggsRH/N76Wy95J2t8HH3Ye7ouTw+4PF6hB+YJ/3cPN3wCfVp0b3YgpELWHLzEpPfyQ4HbdtK+6i2dcvc1p7X72SexDr7qTvbHenn6+FLuG+4Tc9Efq7l5ZC2LY25fnM5+rN9740jkPxrMufXnLdesAWh1dm331bXqqmqy6/ZMbz5dkxb5up3xr7DgUcOMLHrRMCQ06+w0nSkX2TPSJJGJ+Hu7d7wUnYjfXs68wbO4+w/Z5t9LWMIgsCd3e80qcQwoI0UnHG+6DxLl0ub6WnTJNJv1iypzJYtjeff5GTJZhMRATfeCP36SccbSnzKc5d7gBQC6OPuYzEnuKvHwX9JvxaGwlrCOydCFO2P9PMIKAFAWxGkP1ZZU8nujN31koZakvf84+QfgJRw1Nvd2yHynmkN9uG2SHx2ntCZxDGJTb+pg9HvkX7c9tttrcow/S/qQ6FQEB8f36jfunm6XTba4pbkPQFi6uwpLZlM3hxOLz/N5lc2O+x62QeyqSqqctj1XAl5M5XlIf0eIxNG1hs7JvWbwLTJkhXl11+b/jw3vrCRzzt9jqhrGSvErPxZTPx1otVyxcWSE4sgGKLn5Ogxc/22JSFHhzkj0k+fd6GOjOo5qScjXxvZYhJRR386yrY3t6HVmN8EyaSesbQnQHidsmxxsesNgbZCfpbOjIw2zl0mI2tfFvu+aBlpP1fA1f3WmeTt+nvXkzkjk95RvW0q33ZYHDU9+qJ196r3zB0JbbWWDtd2IKJ7hPXCTYSz5T2hPqEYHW0wTDkSbQMl619qSWqzrqPdsYc+a94i74R5I7KtaOkctbJzpTHpFx4urS11OgG12rF9N0slbexska1yJk7mn2RR8iIOZB8w+b0oivx2229sftlxa9WmIKpXFINnDiakvQmGzQ7oRB23/XYbV8y/gvG/jKdUXUqpupRHVjzC9Yuut5kIvLnTzYyJmISfurNJ0i8oSFq7iKJte3hHo6CygJyKHJvL11TW8GHMh/zz2D9230sUDZF+MukXFycRR1VVhvX6/q/2c36t64zbtsy58t6jPGA/gF4+rbmQ54iKCily2xGYvmY6T61+inNF56wXNkLH6zsybM4wis4Xsev9XWx/Y7vZsnKUX/v2hjxwPeoCQI8fd1xbnIGyMsgO+ZU/c9/TS2E7ApaiNgvPFpJ9sOUMDAOeHECvyb1MfieTfrIDQkKC9NnQxtjacKKO9POv7oqXl/PuYxzpF9wumMRRiS0SqLB2+lq2zzXfJ1saT658kpB3QziWe8zmc1IKpf7npvUnOtCMgc4G/Hz0Z0LfDeW+ZffZfe7ELhP5aNxHXNvhWpOkH0jrG0fkkg/rHMaYd8bQpn/zFVVO5p/kvmX38feZvy2WC/UJ1cvZFnsfpm1bGDlS+m7AACnqT61uLOcrq4j07SvZmPr0MRw3XobL+x1ffy1D44YyMMZy0JarbVKtxwL2/xQ6C1l+Ty8/zaLrF5F33Dku7vn5UF0tLVKibJRlFrxLAKgtD9Ib4EYtGMWQ74ew4cIGfTlL8p5LTy4FpHyAAJ7etVR5pDdL3rOhF44tUUpj3xnLmLdal2zGv2ge8o7n8fsdv3Nhg3OE9HU6Henp6fX67ZkVZ8hNbsUJp4wgipblPcFA+rXEptsaTv1+im2vbXNIXj9RJ7Jg1AKWTDDt8dfaIW+m0pEWviMTRjYqM3Ik9O4tkSVfftm0+4iiiLuvO1XFLUOOyp6h1nCsbm2dmGiIXJeNkqb6bUvDlZF+LY2jC46y8+2dKNzNLznNkX5+fo1J3NaE2loDGekMAleGvJYy9oI/8PUBVk9dTUW+6x70udXnOPjtQZfkH3Z1v62shEqPNM5Wb7c5Is9WdI/oTkxAjF0REfIzd4QUpCm4eblxx593MHT2UKdcX6czOPM5k/QzvraxtCc4jhhrGySRfmmlzbP+RXYORBWWRK66+aGPR348wruh75K6JbXZ12oKTEX6CYIU7SeKItu2FTi07+rlPVs40u/7w99zz5/3sPj4YpPfC4JA2vY0co+27L4gbkgcV793NUEJQXafe6bgjD46qqCygFP5Uu62DRc2MOyHYfT+pjd/n/mb10a+hpvCNmm354Y9x6PhP+Ov7mpy/yEILSfxKUdi55bb/szcvN0ISgxqkryuSmVYn8mkn1IprV8Bzp2T9iirn1rNwa8P2n39psKWOVdeg3h6CAR7BTuM9DOOUGqOE7gx5NxYtuQrM0aH8R0YPXc0nW7oxEO7HmLCwglmy56qSwnfpYvhWGKitA6vqmrdef1UKlB7ND2C+pLqEtvTtpNSVF+NxTgirOEUvOSWJfxx1x9Nqq8jMOz5YQx43HQ6FnOkX2uP9DtbdAaACKFTk1JavLfzPSYsmcDujN0Wy8l9VKWCgJgA7ll1D51v6mz/DZuJib9OZPRbo11+X1ugqlbx5YEvKasu4+EVD9t83pl8aaDwqW6Hj0/Tgz983H0oqiriQrH9A8/YdmOZfsV0BsQM0M/RDfPWfd3zaxaMXNDk+skI7RjK0NlDCe8a3uxr/XX6L34+9jPfHfrOalk5x2upz0EGDTIcN+ZCGnIIB+umYDnCr3Nnab9fVlY/B688NyYGdGDHgzvYfL9lxy9X26T+Jf1aGJY2pMUXirmw/oLTvDllac/oaGzOF6IWSgBwqw3Sd4o+URLlfTDbsDA1F+mXrcpme5pkpL6t223klufyxJ4xbO/Sn0zN8Sa0QoLshSN7WbXGKCVLKDhdIOVdchJZ9f8GApz47QSXDjsnSbQoihQVFen7pE6rY+nEpWx6YZNT7udoVFQYDNSm5D3BkF+iNfahq165iqdTnsYzwLPZ19LWaBn5+kj6PdrPATVzPeTFxYuJK1k3aR3Xd7y+3veqahVvbHudlG4PgSBy9Ghjjy1bMOatMTx66FGbiDdHoyK/gqx9WVSXVVstK+fz69XLEB0mk0QN+21rgDMj/UxFz/95z58tttG+/Y/beWj3Qxaj2HPrbG2RDdKdCYL5hXhrQJURF+5M0s84p5/8Gg96ehD3b7nfIeOhrTj07SFWPr4SQeF8RQJX99vKSsgOWcKje4YzZ+Mcl9zTHIpSighf/DmRKbucFunnbKhUhnfVGdF3MsyRflU1VfjO9aXLF10o1zTPiixH+jVX3rPPHZ240HciWQWeiCKUZTVdy8472JuIHhH4hLt+bgbTkX4g5fUDOHas1qF9V470a+mcftF+dURCufl9xrPZz3Ln8jtdVSWHYkf6Djp/0Zk2H7ZhxZkVRPhGsO/hfXx57ZdE+UVxPO84qSWp+Lj7oK61zwHPmtNhS5F+Eb6St1Fehe1O1oIg8OCOB5uU91qO8gsLq5/bUB6/ZEPi5K2TueoV5+TVXrwYpk+Xcs/JsGXOlR0Or/d6h8LZhdzb816H1EepNKxdHZXXT5ZlttRXrSFmYIzFtatxPj8ZCoVhHJS/b40oKatF7S79Nk2JoH51y6sM/3E4Px/7ud5xeb4XxcYE7uBnBzP0Oec4GjUX8rgjq5/J8p4ZGZKDX2uEVqclq1xal0R6NE0WcmvaVv46/RfJeZb1aBvm9GspJI5KJH5ovM3l8/KgpsaJFTKCSqPiitgrANiXtY/Dlw7bdN65Apn0S8K9GeqZicGS54gtkbsTlkwg4eME1p5f2+g72UbYMOqt082dnJpGqSmQI/xu7Hij1bJ9IvvhpvVHq6hgYINAPNlh3DjoISdHsocqlQbZZjc3ydYEhihAMKyJzdlXG8LVNql/Sb9WjMEzBvNi1YtEdHOO9I690p4ApeoSANy1QXrSsF8byWh+8JKB9DOX0+/3k78jIjI4djDxgfG4K92p0qnQuOezOnIkybmWJ5ysLPjll8aezzLpJ4fc2mIcLEkt4Y+7/+DE0hPWCzsZhecKOf3XaZd67P8vIrxrOLPyZzF0lmsWlKJOZOKiiQx40rTXWGuDvOH286u/0TSGTPq1xki/kPYhBCcFO8Tg7ObpxhXTrqDH3T0cUDPXQ94Uhwf6MrbdWL2nsgx3pTuvbHmFX898T1SSZMg43nS/ihZBytoU5g2ax4WN1p0h5NwZpki/1ghnRvoZy3vKa8rynHJUlxxkSbETXkFeViUEzUX6gSFH11nHSv9bxMmTtr0/Mnnr6WlwOnIGZGKjttbw7kT1jiLhqgTcPJ144wYY/dZo7l1/r0tIP1ejogKq3SVvlzZ+zZecMcbB7IPMWjeLL/Z9YVN5T39PFO4KREHhtEi/nKM5rJ622mlOUrJjiq+vc/uGsaKIMemXVppGVW0VWWVZ+Lo3T4JKlgRqdqRftALR0wuNBnZ/fYyP237M8SVNm5g739yZyVsmO22PaA2ygaOhVLxs7L540cth0nY6UafP6RcX0LL5s22JHmoN4+PaZ9fy2+2/2X3exgsbAdBoNQyKlVzhAzwDeHzA4+x+aDdXt7uax/o9xpHHjjAkbggAq86tshglklmWycXiixQWSZ7t5oxiMum3d6+kROQqRPrWRfpVuCY6s2E+PxnyWiclRXqH4ofGE9XbRikmO/HPP9J9XnsN5s61nWiTx3V/f4n4dFc2P7+TDEuykE2BPFY4OnJfRkWFgSjq3CDgSSaMWuNeGqS9QX5VLgg6lIJST3zbA3leTCmuH+nn5oZeZrLhs+zzQB/6PtS3KVVuNspzylkwcgGH5h9q9J1Wa3hW8rOLjJTaUVtrIOpbG7JUWdSKNQg6d8K9muYQIz9Ha/Ll8t6yuloi0VLWpbDkliUUXyy2eJ4jIYqiXWTJzp3w0EMwb54TK2WENv5t2P3Qbu7qfhcAr2973abz2vp2Ji7/IWKqxjUpWlNGYpBE+uVV5FGhsWxTzirLIq00DY1WigZQVavYnbGbnek79eu6srL6hPeo10cx7sNxTa9gHfZ8sod5g+ZRltm8JK6XVJfYm7UXoJHzuymM8ZvBuMMlDKx6mY4d639nSulMztvXpUt9e03fuiHMmPSTuYh42/lol+KyIf3eeustBgwYgL+/PxEREdx8882cOXOmXhm1Ws2TTz5JaGgofn5+TJw4kdzc+gu49PR0rrvuOnx8fIiIiGDWrFnUNnDf2LJlC3379sXT05P27dvz448/NqrPF198QUJCAl5eXgwaNIh9+5yTV0VQCE7bPMiknT2kX0kd6edWG0RK3RzfL1oi/Q5dOqQfiOVw4IbynktP1El7dpOkPUO8Q1hx2wYCK/qhcSvgml+uobjK/OSxaJHknfb884YNp1ZraMvQOq7HpkWWAMd/PU7OEds1/J2FTjd04oXKF+gyoYv1wv/CLARBcGlEktJdSddbu9L+mvYuu2dzYM3LFurn9GtFgVF6VBZWUnjWvqTs/4uQN1LmZNO83Lz0C/mQjtJcKUfD2QOdVsf+L/eT/KvrM9JH9YlizDtjrBo9amsNkWLt2hlIP3mOaI1wRaSfsfTkvevvZfKWyY6/mRVUl1WTfyqf2mrLbrKWSL9OnaRPV3lMHzkCzz0H77xjvawz88AZw9NT+gf18/qJOpHyHNe53YZ1DiNxVOvJhexIVFai93pv4+9Y0u9s4Vne3/0+v5/63abyvhG+RL32BHlJVziN9MtLzmPfp/soSXXODWTjsDOlPcGwzxAEafyXIRuwEoISmp0rOzEokWi/aAK9mifL6eZmiGbWhoSRNCbpsu1PpuQ9QTKY+viIaDSCQ3MhHXzkICvvXqkn3VoKcqRftsq8HEbhuUKOLzlOTZWLwgtMoOBUATmH7d/fytEeTw54shERkBCUwNpJa/nq+q8I8JQ69sJjC7lu0XUWjZuf7v2UpE+T+KN8BmB+DzJ0qDRepKXBRx+5bg8it7OgsgCtznamOnlRMlv+u8Xu+8mkX5sG04w8fp0/DzXVWqfJaJeXG9YRSiXs3i3l/rYFZWUgonVK9LZ8zbLm2YH1kEm/9FLnhI6eOSO9o9HRje1dpgzIqmqVftzIVmWjqm4ZRzyQiJtyhUSGRvlF2yU9LqNzmMR0ni5ovDiX531HEbiOQHVZNbnHclFlN67UpUvSnsnT07CHFAQDAdhaJT4vFksRXZ7qWAL87H+GYLtTk68vekKqvBxUl1ScWX6GwjOus8kUpxTzhucbNo27Gg3Mny/9fcB0Cl6n4T/D/4OAwLLTy2zK7TcwdCy90ubRTf1Is+4b7B1MoKc0GFkjceV8vLLjxtHcowz5fgiTl08mIECaG0TROSkGNCoNqmyVxbQftuCfs1JO3YExA21aGx494I2AgoEDaUSuypF+MqcABtJPDiqS0b279Hn+vJTKQBQNY8Sfxf8h9sNY3t/1vp2tcS4uG9Jv69atPPnkk+zZs4f169dTU1PD1VdfTYVR0poZM2awYsUKfvvtN7Zu3Up2dja33HKL/nutVst1112HRqNh165dLFiwgB9//JGXX35ZX+bixYtcd911jBw5kiNHjjB9+nSmTJnC2rWG0NclS5bwzDPP8Morr3Do0CF69erFuHHjyMuzXRZChqWN6MnfT5K133kuQk0h/bzdvfF3C8ZdG6zXMe8W0Q0PpQfF6mJSilPQ6QwLNuNIv+KqYg5dknrPbV1v0x+PCQnmirMb8FV3JFuVzdNrnjZ7f3kDmZUFc+ZIhsKcHMnjxNPTwLzn5xuMneYQGBfICxUvMHpu69CFdvN0w83LdR77/6tQl6o5tvAY2Qccr08pCAJRUVH6fuuKvEaOhC2kX1SUNBGq1Y21vFsDvujyBcvuW9bs6yy6bhGLrlsESJP14sXwwQfS5H05oKwMjrV9lK8vPmNWcqxTmMSWKCOlDVlyE3g7hVLB5pc3s/+L/dYLOxgR3SIYOnsowYmWtRKKiqRn6OYmbbxlI6QcqdWw37YGyNFaziD9vLwkeSEwkFItFXmQsj6FL7t+yfHF5qNZRNEw1pgam2TS79w51/TPJXVpPi9etG50lMlbZ5N+YDqv37xB8/h+6PfOv3kdatWOleyzBFf2W5kg10f6OZj0k6WzMkozrJQ0QF4/O0ves9vt3ZiRMYN2V7ezXrgJkOttKre3IyEbF+PjDdEFYDB4yHJHzcH4DuPJfjabBTc3P5eJbOivCm7DpDWT8A2XvDQ05VY2LQ2w872d7PlkT7Pr0xTU1hrG7Iakn0IBXboIeHp6cvKkY/quQlDQI7IH13a41uYccs6CPtLPgmTg0Z+O8sedf1B8oeUW0fesuoenzj1l93ky6XdTp5tsKi/nyNl4caNZGd2T+ScB8CyXSAJze5CQEHjhBWktt3OnpOzjCoT7SlZ+naijqMp2HfwTS0+w/Y3tds+JctRQw0i/uDhpLaFWw475Z3nd/XWLa6emQiYdQ0LgiSekvy9etG3OLS+Hk3EzmZObwPxD8x1aL0cTRfGBUsiFs0g/2RGtYZQfGEg/WXlKJ+oY9sMwYj6Mof+3/Rm1YBRX/XgVvxz7hQeWP6B3incVyspA7S5HT9sv7QnQJVxyVD9dcLpRHzDO/2aMc6vP8VWPr2xScHE0QjuGMrtwNlf9p7FkrnE+P+PXvyVIv7yKPL4/bNu6vkdkD55v+xed0t5qsoOVrZF+CoVhr1NeDt1u68ac8jkudXxXuCnoML4DIe0tGLLqsGKFwQ6Ql+e8tbSMk/kn9QEwXcK76ANcPtz9odVz5ch22bmzObBV4rNGJzkluSsk0i/EW/pNi6qKEARDRL6xLTD/VD5/3vMn59ecpzkY/tJwZmTMwC/Sz3phC/j7rO3SnqIoqQgADBrUWF6zobxnba3BWV7O5ycjJkZ6Vmq1NJ8WF0tjnSCASsgkS5VFjday05erbVKXDcOwZs2aev//8ccfiYiI4ODBgwwfPpzS0lLmz5/PokWLGDVqFAA//PADXbp0Yc+ePVxxxRWsW7eOkydPsmHDBiIjI+nduzevv/46zz33HK+++ioeHh58/fXXJCYm8sEHHwDQpUsXduzYwUcffcS4cVI464cffsjDDz/MAw88AMDXX3/NypUr+f7773n++edN1r+6uppqI62KsjqrjSiKaOv0TwRBQKFQoNPp0Gl1/HnPnySOSeSuFXehUCj05WTI5RseVygUCIJg8jgYEkdmZgqIokBMjNQRGiaUVCqViKJY7/jiWxaTl6fk4YdFzqlEqqp0eHgoGRw7mK1pW1lwZAEzer+KTifdy89PBKQ2BXgEkD0jm12Zu/SeklqtFkEAX2UgvS78yJ5uw1h4bCETOk2ot+FQKBSIokBWlogoQlCQyKVLAi+9BLfeKtU/NlbE31/Ex0egslIgKwvi4y23SeEp/X5KpRKdTtdoADB13Pg5mTrelOd0ftV5/Nr4EdU7qtFzMvf8LD0nc3V3ZZtsqbsz2qTKUbHs3mX0f6I/bfq3cXibwsPD9fICv93xGzmHc3ji1BMo3BROa1PDOja1TQUFIIqKuonc9HNSKiEiQiQnRyAjQ0dgYOtq0+BnB+MZ4Nn8d89dAYI0Bm3cCAsXSuVuuEGgQ4fW359KytVktvuRn89reKHmEbRabaO2dgzpyBrWUO51BoVCJDsbcnJ0ei9GW9t059934h/jr792axsj8vKk9zokREQQFISE6BBFgfx8Ea1WqmtUVBQ6na7efVty3KuslOYtT0+dXgLNkf3Jx0dEpYKyMqkP62p1JC9Mxt3Xna63dXVKm0wdD+0QypDZQ4gZFFPvWsZ112hAq5XO9/ERGrUpNlbAy0tBZaVIaqpOvwl3xnM6eRKSk6X/azSSh2NAgPl3r7xceve8vAxrHWeNEQEBCnJzRYqLDe9M5wmdqSqqoramth6x66w5992wd2l7VVvuWXmP08c9URT1862p8c1RbQKZSFWiriP9In0j69anjmlTjL9kAcwsy6S2tla/2bPUptpjJ4lIKaOo3yD9OObIMUJwE/CNlggn47Y6qj8VF+sQRQV+ftI47Kx1RPfuEBqqYNy4+mNHSpEkR5IQmNCq1rCRkTr9fkZ+rjvf2snuD3bzxJkn9GoV1t69/V/uxzvYmwFTB7isTStXihw4IHLrrSI6nQI3NwgIaDxmd+kicOiQF6dO6bj++tYx55prk73vXqSPFKpZVl2GSq3C18O3UZs6T+hMWJcw/Nv4O7RNOTk5zJ83jz8XLiQzJwdBo8HN05OoyEgmTJrEQ1OmEBMT0+T+VFlTybnCc4BkTLblOXUM7khScBIXii+w9txabu58c6M2nciT0mgoC7siiiLBwabtDgqFgq5d4fHHdXz6qYIlS+C663QEBzt3r+GmcGNY3DA8lB5UairRepkfD42f05j3xzD63dF6O4L8nMw9PxmZmQKgICpKRKutX75HDyV79oikFXjT5dYuBCYEmrVTNLU/padrEUUF0dEisbHS2iUnx7Y5V6USKPU5QFFtGkpB2Whd15zn5Osr2aVUqsZjSlPGCJn0yyjN0H/vyDHi5EnJJtWhQ/11vE6nIzpaRBQV5OdDZaXI/rzt+ogfOR1OlF8UGy5u4McjP6KqVjGx80SrbXLUuFdWJlLlLjkhtfFv0yRbWLvgdrgp3CjXlJNWnEZcYJy+7n5+0jxXUiL9NvJxBKitrkVTrtG/Yy1tNwJITZXevbi4+u9eXJy0xk9NbVxHRz6nN7e/Sax/LBM6T+C59c/x49EfiQ+MZ3TiaIttCvQIpLNwA8nlAv7+5vuNpf4U5y9Ff1wsvohOp7PYJj8/HeXlAqWlOtq0UdTZZxv/7s56Tv5x/ty+7HarbSorgyVLpGeqUIAoCpw5o61H3jj63Xtr+1ssTF7IS1e+xKtXvcqrI15lYpeJDIsbZnb/CxIXcDznDFohCS8vr3pzQlPGvcSgRI7kHCGlKMXiuCeTUm4KN7RaLYEekodecVUxWp2WoCBp/MrP15FUl8ZPq9GSvCiZ8B7hJI41ONS1hC2sqraKDRc2AHB9h+utjvHp6Ury80VSoz9k0p5veVz7GE8PfFpf96goaczOy4PKSh0pKUqqqkQCA0XathXrj2PoSEiA06cFzp7VERgozenR0Toyq6QgsFDvUIv9Sa6nq3DZkH4NUVpH14fUuYsdPHiQmpoaxowZoy/TuXNn4uPj2b17N1dccQW7d++mR48eRMraKsC4ceN4/PHHOXHiBH369GH37t31riGXmT59OgAajYaDBw8yZ84c/fcKhYIxY8awe7d5Pfu33nqL//73v42OHz9+HP86PYOQkBDi4+PJzMykIK+Afu/0w93Pnby8PKKiokhNTUVl5DITFxdHaGgo586dQ602JNNOSkoiICCAkydP1nvJOnXqhIeHB8nJydTUCGRmJgAQFuaHWq2pJ5eqVCrp0aMHKpWKCxcM3jheXl506tQZb28NeXlqVq/OJimpmlvjb2Vr2la+PfgtA6tuRaVKwMdHR06OCu9QbyoLKimucxVoQ5tGbdJo4nEr6Mq9NzzFgpRPmL5qOvHqeL1HZ1JSElVVARQVqVAqdTzzTAaffNKGzMwgvvlGgUpVhoeHiuPHC/D0bENFRTgXLmgoLT1lsU3laeWo09SMfmo0xcXFZBjF9Pr7+9OuXTvy8vLVmWX/AAEAAElEQVTIyTFIpBg/pyI5dArJqNyU53TixAn+uecfAjoEMGz+sHrPyRg9evRAo7H9OXXu3LnF2mTp3XN2m0RRpN9b/UgaJc1QjmxTbW0tKpUKf39/OnfuTHBSMCXFJZw4ZcgL2ZqfU3JyKCpVAD4+HoC32efk5paPSuXNrl0FiKKqVbWpy5QuhIaGcvr06Wa9e7f9cRsajYb160/z/vsx1NQIgEBBQQBRUa27P3Xs2IkM7UF0Cg1B7sFUZVaRnJXc6Dn5qSUPqgulZ0iM13DsmJrly/MZNKjcvjb5AAIEE+zSMWL/rP34evoy4bcJFt+9w4d9UakiiIqqBUJxdy9BpVKiUsGRIxcJCJCMct7e3vUi8lty3FOpRFSqCrKzc0hOrnJ4fxKEClQqLYcOZVNSUk1kZCTrZ63HL8EPbWdDHZ0+lvfswaCEQVy4cIGs5CyTbVKpFKhUbVEq3fDy8iU3t3GbOnaMZ+9eNWvXZjN4sMppz+nrr6NQqbzrfuMA0tOrEUWDdFHD53T8uB8qVThqdQ0Q5tQxIjAwlIqKCo4ezcHLS4qo6DW1FwEBASQnJzt9zhV1IpEjIvHvLq1ZHdEmwUdg5f6VtPNph4+bT73ndPz4cUpLS/H390cQBKeuI6qqghBpi9pDIv3KsspILkl22BgREy+RftXaarYf3E6wZ7DV51Sw7iAxZzI4fy6J5ORLDh8jTu85TXFhMT4xPggKweH96cSJDFSqYCoqVCQnFzh1HfHaa1Kb0tMNz+lYmmRgTQhKaFVr2ICAclQqkUOHKujTJw9/f3+C2wUT3DmYI7uO4J/QeE9o6t0bs3gMpfml+jq5ok2LF9eQnl7F7t06qqoUhIeLCEIQRUUNn1MwFRVhHDwocOzYWX3kRFP708IdCzmQe4A+oX3oG9q3Rfca6lI1Xkov1Fo1Ww5uoVdcr8ZtUkDUqCi8g71JSUlxyLv33KxZfPbppyQCF7RargImA1EaDTkqFfNff503X3+dt999l8n3P8DOb3fi386fwI6BNvenE8UnEBEJ9Qol0jeSjIwMm57T2PixfFP8DYsOLqJdjSFyOCkpCaWXktTSVABqL8WhqinDx8cDnc78c4qIOIObWxzFxW5s25bLTTd1dPpe48erfkSlUlGcVkwxxXY9p6zjWXa9e2fOJODmFkRAQDnJySn12tSrV2e2b69hZ4YXT77YkUIK0aRqHLrXOHw4F5XKF0FQUVhYRG1tFwoL3Tl8+DiVlZbn3JSL4ZRGS6pNAeUB+u8cMZaXlYWiVodRVubtkHV51/CuvD/ofaI8o/T1dOQYkZ5ejUpVTU1NNsnJ1Y3apNW2pbJSwaFD5fxU+BMAV8dcTb/QfhwrOsbMwTPxDvDmxyM/svbcWg4fPYybws0lY3leXgWhFyZxnXokD43N5Ny5c03qT23925JSmsLqA6sZHDFY/5xqa4tRqdw4fryQsLAyfZu0CVquXHolatQkJye7dCyvyKqg9FQp8VfG031Q93pt2r8/HLU6hPj4+u+eKHqi0SRy6pQXFy+mUl7u+HVEbPtY3tj2BtXaarxKvTidLe05Vp1bxcCwgVb705kzoZSU+OHjE9ikMUJTJKkMXCq/RGpOKkltksy2qbq6CJXKnaNHc6itrSKMMIpOFaHr0tB5oWXtRgsWuJObG0hMjIaOHX04eNCX7dtz8PAwJGt35LtXWFLI8tPLAUgUE/X9KVwIJ+NCBrnkmm1TXlUet28bh9DHjSl5OSQnG36Dpox7V8ReQW5RLppCjcVxr6JKkgLS1eokbqAu8k9EpLS6FD8/H1Sqag4dKsDHR4WXlxcdu3Xk0bRHySnK0V+rKc+pZG8J2enZhFxpiNi0tz8RBh1COlBUXoQuR0dybrLF57R3bw9qa2vxCcrlePFZNp7cyPjg8fXePXnM3ru3hAsX2qLRVBMVVcjx4/mN2uTlJaBSBbB1aylt2/oDwQQEFJNeIIUNV+RXUFxcbLZNDYlAZ0MQXaXT40DodDpuvPFGSkpK2LFjBwCLFi3igQceqBdNBzBw4EBGjhzJO++8wyOPPEJaWlo9qc7Kykp8fX1ZtWoV48ePp2PHjjzwwAP1SL1Vq1Zx3XXXUVkpEVcxMTHs2rWLwYMH68vMnj2brVu3sleOG20AU5F+cXFx5OfnE1wXP+tKD8icHHj0UQUeHvD771I4qj3eGG++qWP3boH779dxyy2SLvA7u97hvp73UZwax3/+oyAuTuTTz7QMnD+QGzveyMvDXzZb96lTBTIyBF7+r5pvMqcye8hs2ocYwsUVCgWHDgm88opIfLzIZ5+JnD0LL7ygoLZW8hh86CEdN94IH38ssGWLwD33wG23WW7TysdWcnjeYWbmzcQ71LtFonhqa2o5/ddp3L3caX9te5d7NzmjTS3tseWsNmm1Wk6cOEG3bt1wd3e/7Nr0zjsCu3YJPPII3Hij+ef0zTci//wjMGGCyOTJzvPQd0SbjOtu77un0cAzz4ikpRkiZB5+WODGG1t3myorFQx85l1Oxz7PjR1v5s/bfzfZ1s2pmxm7cCztQ9rzUsBZfvsNRo4UmT7dEDViS5tEUaQipwIPXw88Az1d9pyW3LwEbbWWSWsnWXz3li+H779XcOWVIs89p0Cr1TFxokBtLcybpyMkRMvJkyfp1q1bPc+qlnz3HntMkpCYO1dHt26N29Swrfa+e9Oni5w/D//5j47+/aXjmbsz8Y/1JyA2wCltMnXcljbl5sIjjyjw8oLffzftqbpwoYKlS0XGjBF56inRYt2b2qbz53XMmKFAECA8XCQ/X2D2bBgyxHybVq2Cb75RMHiwyIsvOjfS7+OPFWzaJDJ5so4JE2xrU2udn3Siju+PfM8Lm16gqKoILzcvrml3DW+OepMu4V0QBAGNRqOfb5VKpVPblJICU58rZH3vSAQEKuZU4KH0cOgYEfVBFHkVeRyYcoDeUb0t1l2hULBvZT7vvyfimxDG/O/tb5O543Idf7vjN04uPcmcqjko3ZUO70/ffiuyYkXLrSMGfz+Y/dn7WXbHMm7ocEOz2/TYysdYn7KeT6/5lPHtx9drqz1t2rdPx2uvCSQkiHzyiai/jiiKrWZtZPq4kokTxXqpE7p2FXn33cZ1rKzUctNNlXh7B/Ddd6I+j6GtbcrKgpUrBVQqab3zj+YZDnp+RP+aGayZ8R4hIS077v2a/Cu+Hr6MaDuCAK8As20SBKHRtZvynD764APefukl7q6uZhGwAhhEY+wFbvD0ZMZzL6N5rYb+T/bnmk+usalNoiiyLW0bz6x7hmj/aFbds8rmd09ea4b7hJM5PVOfH0yhUHDw0kEGfDeAUK8Irtieg6+vyOLFdSkSLDynl14SSE4WmD5dx5gxrXOvUVtdS2V+Jd6h3nj6elptE0gSYJMmSddYulTEw6N++exsJY8/LuLuDosW6fDwcHyb3npLx86dAg88oOOmm+CuuxSo1QKffqqhpMTynHv/rBP8EtgLH6Ufxc8V1ssF19yxfMkSgUWLBK65RuCJJ1rHuGepTZMmiZSUwCefSFEfDdv03HMCp08LzJypY/Y5aT7adO8mhrcdrq+7iEjE+xEUVRWx9f6tDI0b6pI27dkj8sYb0LGjyHvv2bcnlOuuUCiYsHgCf535iw+v/pCnBz6tP/7FFzpWrxa44w6Ru+8WW8Ve49C8Q6x6bBV3rbyLjtd2rNemadMEUlMFXnlFoF8/w/HaWrj7bgUajcAnn2j1SiPGv0Fz2zTv8DweW/kYPSJ6cOjhQyw+sZh7/7qX/m36s2/KPottWnhsIWtWupO6aTCzn4rnppuEJo0RCZ8kUFpdyrbJ2+gd3dtsm158UcfRo9K4PHIk/P3A3xz7+RjP5j+Ld7B3vfJNfU6WjqduSeXcP+cY9NQgAhMCzbbp8ccFsrMF5szRkZcn8P33CgYM0PHii4byjnz3NqRsYOzCsUT4RpAxLQOlQmlzm3ak72DETyPwrk5gptsFXnmleZF+trYp8dNEMsoy2PvQXvpFSyGQQe8GUa4p5+zUs6xb3J7Vq9H3YVN1N9cm4zqaOj5/0Hwq8it4KuWpZreptKoUf09/i22troYHH1RSUSEy6tF/eObgjXQJ60LyY8lmx+yff1aSmyvy/PM6ZMrHuE3r14t89pmC7t2lNe6mTQruvFPHfwrac7HkItvu38bQ+KFm21RSUkJISAilpaUEODv5OZdppN+TTz7J8ePH9YTf5QBPT088TQj1KpVKvSSEDIVCgagTG+XgaViuOceLiyWiLywMZBuoqfKCIOiP12hrGPL9EIK8gnig0zL27PHjzBklSqV07stXSaReerJ07eBgge8Of82RnCOklqQya+gs/Dzqa/fK1/bzqyMea7yZf9N8k/XOzJTqExcnoFRCly7w6KPwxRfSuYmJUl3kHIXZ2dbb1HtybxKuSsDNy62eMdgY9h6393m4ubvR/bbuzbqOcZtsqaOz2+SI481tk7pETa26Fr8o03rRTa2jXC+hznX5cnpOJSVSXwkNtVw+NlZAECAnR+prlq7v6jaVZZXx+x2/0+nGTgydPdRqeVPH181aR0j7EMS+/UlPFwgMlPKBbtkiSaC29HOydry8HIr9dgIwvO2wRt/L/+8aIck4ZpRm0HVoLcLv7hw/LkldyJ73ttT9/Nrz/HLNL1z75bUMeHyAU9pk6vjdK+62WF5+TkVF1BE1cp9UEBYGublQXKzUy5kqFAq7nqszx7eqKqnOfn7Ken3MUe+en5/Uh9Vqw/Xjh8abLGtv3W09ri5R80X/Lxj41ECumHaF2bprNNJvIec3NNWmTp2k3+bcufpjkiPrvnGjEkGA4cNBoRDYskXKA2Hp3auuluru61tfrrEhHDFGBARI9y0vNzzT2upalt27jIgeESbzlLTG+alCU8F1i65ja9pWAPw8/CjXlPPXmb84W3SW448f15eX62RcL2e0Sa0Ghc6Lq8q+YdzNhXh7eFss35S6xAbEkleRx6WKS/RT9rNavt0VEagDoKaUemO2o55Tl1u6ENohFA8vjya3ydLxzExpDIqNbZl1RM/InmhFLe1D2jukTfkV+VwsuUhaaZrZOdcY5toUG6vQr68aPldrbZKhqdCgylIREBeAu7d7k9tkz/GKCqipkZ5pYKCUHyciwvS45+MDsbEaiooETp9W6PMYWmqTfE+dTsqvbBQIQHZSFnhC5aV43nlHydy5Ut63ltpr3NPrHovlNeUaPu/0Oe2vbc+N393YrDrm5OQw5/nnWabVMgHYhmnCj7rjK6qrGf7my6z7diMd+nSodz1r/WZk0kgOP3ZYb6Czddy7KuEqAj0Dya/M52DOQQbHGZyi5Xx+iX5dEQSB0FBB/85b+g3atIHjxyEvr/7ey1zdm3tcvrYoivX6oqU6Hll4hH8e/YdJayfpc6Nae66FhVKfDwoCb28BqF8+NhbCQiFw3W+seT2RCXMNa25H7TVycqQxKC5OSVpZCgc7PEtQ5p3k59+Gh4flOTdDK0X5dQ3pi4e7R6NrN+c5BQZKv41KdXnYI6qrhbr1X/11vFz3uDg4cwYuXVKyd8peDl46SN/oviiE+m27ut3VLD6+mA0XNzA8YbhL2lRZKdTtQQSTdW8Ic8cf6vsQ49qPY2TCyHr3CQyU3rGKisbz/7GFx0CAnvf0dGibrL177ce255ZFt9Cmb5t6x7VaydFEEKScfsZtVSqha1c4cgROnlTqZQ6bWkdTx386JkWB3tfrPtzc3LgqQVrLH750mHJNeT1Co2GbXtj8AtnabPr4bMLXN77JY8Thxw4T6h1q1bYVECA918pK6Z3v92g/Ot3YCU8fT5PnOHq9d2n/JfZ+vJced/cgOCnYZPnqaiWXLknPs3t3pT5HW0qKotH+0Vwd7T2+8txKQJKZNB4XL5Rc0DsJPTP4GZNtkiPhfavb4e1v+rdxxljQIbQDvh6++Hv664+HeodSrimnqKpIP1eXlNTvw0Xniig8W0inGzrVu7Y97974z8dTU1HjkLYG+QRZLb9zp7SGjYwUuPOqATxzUMpFWqWtwk/p12jM3rFDSV4eeHkJDBigbPTeKBSKOrsEXLwoUFUlHU9MVJCfLkUFRvpH6q9rru6uhOmn04oxdepU/vnnHzZv3kxsrCHxbFRUFBqNhpKSknrlc3NziYqK0pfJzc1t9L38naUyAQEBeHt7ExYWhlKpNFlGvoYjsOuDXbwb+i45R3OsF24CCgulz4YJ2C2htLqUA9kH2HBhAz27egFSEuOGsaIlJVDpcRG3gAJe2vQSAG+MfKMR4WcMXymtCBUV9Y+vOb+Glze/TH5Fvn7QNnrsjBsH99wDQ4ZIeT2Mv5fLW0L80Hh6TuqJp78DMqc2EaLusgu2bfUoTS/lneB32D53u9PucW71OVY/vZqStBKn3cPRkKPrQ0Isl5MNNLb0IVfDO8SbwrOFVKuqrRc2AU25hsPzD3P277PIKgTdu0P7usDiggIHVdSJKCnVUeS7C4Bh8cPMlov2i+bkEycpm1NGz27uuLlJ7du5k3re+tYQ2SOSfo/2I7xreHOr7hSYms9koi8/v3H51gB5gSgnRXc05Dm1vLz+8Yq8CkoznJzNvA7lOeUoPZRW5zj5t/D2Nl+mU93eIiOj8TrBUZCVX3v0gIiI+sfMobJS+pR/b2dCdgQ0Tkbv5unGxU0XyTtmpaIOQMHpAv6e8jdp29KadZ3Xtr7G1rSt+Hn48dG4jyh+rpjtD2znwd4PsuDmBQ6qrX2orAR3XQD9hUeYc+Uc6yc0AbEB0uI0ozTDpvKBgSJuNVXoNDU02No4BN3v6M6oN0Y5/sJI+4Lz56W/27e3XNZZmHfjPA4+cpDuEY2d6pqCtoGSi39aafPe/4gIicStroa6rAeAtBdYMHIBf9z1h9VrZO7J5PNOn3P4+8PNqos9kN9BX1944w3JOeKmm8yXT0qSpIxOnrTvPps2SYSfry9MmQLTp0NootRnghRxnDoFX37ZeO/ZmuDu605I+xAC4wKbfa15333HSHd3DgEjMU/4yRgEjPDwYEfudtr0b2OltGmYI6DNwV3pzjXtpYjC1edX1/tOzucX4yFJGgSbttM2QnS09JnjHDNII7y57U3C3g3T2y1sQVSfKAY+PRD/No2N8uYgr3fMrRkEAXq2ryQw7xwX9+SaLtQMiKJhbxcTA89teI6Lnss53O4u3jv0IjpRZ/H8S8IBAPpGmnYAbA7qst3U5dh1DLalbeOTPZ9wJOeI4y6K9DvKam3m1q4xkqq33mm9f5v+jQg/gHHtxgGSzctVqKyEE3Ez2Ooxm2xVdpOvc33H63ms/2N0CqtPAMjP0kiRUI8tr25h5zs7m3zPpiI4KZged/Vo5AyekyNF9Hl6Gtb+xujRQ/o8ftzxdTpfdJ5dGbtQCAru6SE5k8QFxpEQlIBW1LI703zqqKqaKv2z81K31f/mTUGYT5hN475f3U8n7y3jh8bT9dauuPu4mz/JgRg0bRDTUqcR2TPSbJmLF6XPsDDJkSApSRpXi4oMdjBHQhRFVpxdAUj9wRjnCs/x8paX+XL/l2bPv1AseTj5VCfh5eW4eqmqVY0i8oyx8b6NnHryFF3Cu+iPhXhLBsLCqkK9rdB4nQqw9dWtLL5xMTVVNU2uW+ygWBJHJVovaAaqahVVNVU2lRVFWClxsowfD9H+UbTxb4OIyOFL9dfQ8pgtCzf26yeNC6YQGwvu7tJYKr9zUbFqyjVS5wj3aV02s8uG9BNFkalTp7Js2TI2bdpEYmL9F6Vfv364u7uzceNG/bEzZ86Qnp6ul+EcPHgwycnJ9XL6rF+/noCAALp27aovY3wNuYx8DQ8PD/r161evjE6nY+PGjfXkPm2FuQHWL9KPiB4R+EY4x5IkG7btIf1K1CWA5JXdqYMb7u6S8enSJUOZU/mneOX0DWzqmcSnNT0pVhfTM7Inj/Z/1OK1TRkoN1/czLW/XMvr214n7qM4fsl5ARFR3yFBGsTvvBPmzJG8PsHQYe0hLFpS5XbDnA18GPOhywyx/x8QEBdAn4f6ED/MfGRLUyBFmkqJqi9uvMi+z/ZdNqStKBoWO3KknznIfSgnB3SW938uh7u3O7PyZjHq9aYZKz38PJiRPoNrv7xWb9CPiDCQRJcD6Zd86Qw17oUoRS/6RPcxW04QBLqEd8FD6YGnpxQdDfDOO3DXXfD999JGxxr82/hz/dfXk3BVgmMaYANU2Sr2fLyHvBPWyQxTpJ/8d35+/X7bGqDTWTcWNBemHGlUl1S8H/k+217f5pybNkBY5zCePPkkg2dYXhvZQvoFBUn9VBTh3DnH1dEYMpkWHGw/6ecs8tYYgXX244YGsWcvPcttv93m9PsXpRRxeP5hilKat2t+afhL3N7tdlbfs5rpV0zHTeHGsPhhzL9pPv3b9Nf3U1f2W7mfOJO8/fSaT8l6JotH+j1iU/kzf5yg99p3Cco9px/jLhfk5krreTc36slhXc5oGyQ1JLUktVnXcXNDL3eZbWRrFRQCCDRSeTGFgNgAhj4/lNhBsVbLOgqy4ScoCBISYNYs6NDBdFlBEBgyRBqw7CH91Gr4SQp64I47JFJx9GgoESXS75E7YhEEWL8eNm9uWjscgbOFZ/nl2C9sSzM9lwqCwOStk7nq5cbR1/Zi+aJF3KtWsxy418Zz7quq4q9Fi+y+V63OhgWhGcweOputk7fqVX9kXNvhWl4Y9gLdPa4DrO8/ZMh+1MY2BmejsKqQnHLbWcaYATGM/2Q8Ed1NMAVmIK/9LBl3+wzz5fA1c8joPMbm69qKoiLJ4UChAI13Bn+d/kv/3e+57/D2mbfNzrmiCPnuEuk3IKa/w+smOzY1dFZrDr45+A3T105nw4UNjrso0m8ow9yzjI2FWkUFaVmWjdMy6Xcg+wD5FfZ5KoqiyP6s/Xb33YoKkbTwr9mmfQ91rdr6CXbCEuk34ecJTFw00eH3bCrS06XPuDhD5L0x5KCC48cd72yyKFkap69udzXR/tH641fGXwnA9jTzzuuyA5K7zg8/ZQx+fs5fK8vPtWEf1Wpck5/M3dudoLZBuHmaFyuUVQLkqEwvLymCE5yzfzxbeJaU4hQ8lB6MSRrDypWSXbqoCIbEDUEhKEgpTiGrzLRh+kKJTPq1cwjpp9FqCH8vnIC3Ayiqsm+/Nm3QND4e9zGdwzrrSb+G+4++D/dlws8TmlXH5trdvz34LSHvhvCfTf+xWvbcOSmFg7s7jB0rHRvQRnJa2Z+9v15ZWS1QxtDGgmJ6uLmhl3UWRfDwgIDQSobFD6NreFeCvIIs1svVNqnLhvR78sknWbhwIYsWLcLf35+cnBxycnKoqrMWBQYG8tBDD/HMM8+wefNmDh48yAMPPMDgwYO54gpJVurqq6+ma9eu3HvvvRw9epS1a9fy0ksv8eSTT+qlNx977DEuXLjA7NmzOX36NF9++SVLly5lxowZ+ro888wzfPfddyxYsIBTp07x+OOPU1FRwQMPPGB3u8yFwva6rxeTt0zGP7oZbhsWIHdgWxffYCD9gryCcHc3eO+eOmUoExsQi4c2DEHnRqlOWql/Pv5z3BSWlWRlQ4tsQANJLuSnCT/Rv01/qrXV7FK+RYnvvnqknynIxt6KCuvRLNVl1XyS+AmrnlxluaAT4R8t5VgyJ0P5L+yHIAjcOO9Gut3ezaHXVSgUhIaGolAoGPveWKZdnEZQ2yCH3sNZKC+HmjqnnKAgy2XDw6XJrKbm8iDB7IWHnwdBbYPqkX7yuHE5tDe9KBtPTTQxDMRD2VhexxyefFKKjg4Lk8bGZctg9mzXeVLbg9zkXNbOWEvGTutRMfIzM57PjCP9jPtta0CV0f7fWWRRQ29MkOaaQdMGkTTGhD5NC8IWIxhA587S59mzzqmHHMkiE4zQukg/U5F+AEp318iGdBjfgedKnqP7Hc2LnPL39GfJrUssRimDa/ttZSWU+Bwgx2sbhZXOYdjaBrWljX+bejmQLCG8azjV3fqg8Q5winfygpEL2Pyyc5gTOcovMdHgkOdK1GhrHO7M56hIPzBEMTUkNO7fdD+3/HKL1fPDOoUx5q0xTY7kagrk8dGWSC2FQsEVVwQiCAIZGRIJbAv++EMiF6Oj4fo6h/laXa0+mmHsoDhuq/NvaMkMH8tOLWPSsknMP2w6HYUjUVpWRhRQCtiqJxQJZJ3JImOXbVHFAHkVefjO9aXvN32bRP71je7L8LbDG+33r0q4ijdHv0lCjZQH095IP1eRfpF+EhN/qdy5N7RlvdOrF6BQkJLpZZI0aQ5kh+ioKJh3+Gu0opYefiPofXEBAgK/nfmNs0WmF1mVlRBSPpSg8oEMTbw8Iv3iAiQrbnppuuMuiuE5CgL6vIsNERsL6eHf8k1AFO/seNfstaL9o+kV2YsekT3IrbAvuvP1ba8zcN5AZqyZYb2wEQoqitAppEa08W/ePLIncw8/HP6BsmrDgzO1B5ERNzjOLqLcUfjn8X/4IPoDairrRyjJfaKhwV9Ghw7SMy4tlaI2HQk5Quy2rvUd9/SkX7p50u9isRRe5FuThKeHJwEBTV8rZ5ZlMnHpREYuGGmxnKnn+k3fb1gwyjUqHcUXiylJLbFYJiVF+jSWYpVt1vLa1BiFlYV6GWpRFNmXtY8fDv9gc51kac8RCSPwVvqzcKFEEP/6KwR6BepzeJt7lilFUoV9qpMc4gjsofTAXSFFXl4suWjXuff3vp9pV0wjKTjJbKRf4qhEek7q2Uha3lboanW87vY6yx9Y3qTzQVIUUNeqCfWxTmSsWyd9Dhtm2EcPjBkINCb9jDkGNzfob8W3xVjJJD4ewnxD2P7Adk48ccIqqedqm1TrsIDZgK+++orS0lJGjBhBdHS0/t+SJUv0ZT766COuv/56Jk6cyPDhw4mKiuLPP//Uf69UKvnnn39QKpUMHjyYSZMmcd999/Haa6/pyyQmJrJy5UrWr19Pr169+OCDD5g3bx7jxo3Tl7njjjt4//33efnll+nduzdHjhxhzZo1REaaDzU2h4aJHV2F5pJ+YIgaMSb9/D39ub72B0YlX+CumP/w3Q3fcWXbK61e21RUgkJQcFe3Seybso87ukgh7+nh39ST9zQFHx9DnkJri2UPfw/8ov3wjXSBNpcZXDH9CqbsneIyw92/aDq0Wi2nT59Gq9UiCAJBCUE2eWW3BsiTtr+/+Q2KDIXCsOHObrrqh9OQdyKPne/tRHXJvt3whQ0X2PX+LtSl0kZHln40Jv0KCyV9/9aMDm6jGXMsiyk+K6yW3Zm+k8l/TebNbW8SEwNTp0oRfi++KC3ez52D55+37iBx5McjfDfgO8pzHeiCawGxV8QyedtkOlxnJpSgDjqd6QhWYxLXuN+2Bsikn5ub5HnmDMgbs4ZSmNd8fI3DnSHMYfeHuzn5h/UwD1si/cCwuL5o3x7GJohifdJvXurz5PtvIC/Psmdva4j0K8ssI/nXZLvHQ3shKAS8Ar2aLONzpuCMVSJmV8YuHlr+EJsubnJpv62shPPRc/lWcxWLjy92+v1sQWTPSHzuvJGK4FiHR/ppa7QUniukLMOB1lUjyEaXdu2ccnmr+Pbgt/jM9WHqqqkOu6Yc6ZdW4jjSrzWur8zBONLPGrRaLdnZp+nWTZKK+Phj66oRarXkiAQwebJhbswpz0En6nBTuBHpG8lAyVbD2bMtJ/EpR2RcUpkniM6vPc+a6WuoLmuaFL2MwIAAcoBAwFb/rFzAz9MXr2DbwwaSc5PRaDWoNCqrjrogyV999ZXBmdAYNdrGB21NLyBDjvQrLa3vKOUsyPLLWSrbZYLUJWqW3LKEvZ/ttfkcOULMEumXt/00SZ6ZiCIcO2bzpW2CsbTnrV1v5f5e93N/p+nEFt7HGNWPrLp2FR2CTa+7Kyqga+YHjEzZS5dIxw/uxtFhjurb8YFSiE9Gme0EuC2Q30lPT9PRYSC9w5mhP1GrLENRa9mxe9P9mzj62FG75KhTS1J5ZcsrAPx99m+r0qzGyCqX2Ct/ZRhebs0LL7rtt9t48O8HOZ5n0L80FxEmo1pV7bLoMBkBMQEEtwvGzbv++CbbCuU1dkO4uxucDpOTHVcfjVaDl5sXSkHJtR2urffduPbj+OSaT/h0/Kdmz5dlIb2qEikvL8fbu+m/p6+7L3+e+pMtqVsoVZtXHjNF+rXp38ai3KYj8c8j//BVz68sljG1/pRVCUxF+n24+0O6f9md/2z6D8fzjjNo3iAeX/m4xd/BGFMHTmXjfRt56cqXOH7c8Nts2CA5jg6Pl/J0mlIGqNHW6PuNn7qjw+Q9E4MlRUSZGDaFbl92o9uX3cgtN+1oIM/VJSWOtYtpNVo63dSJiJ5NI/7LNeV6AnV8+/FWy8s2g7oYMOnv2CvoGdmT9sH18w9ERRk4hD59rO/tjd8xe1VNXG2TumxIP1EUTf6bPHmyvoyXlxdffPEFRUVFVFRU8OeffzbKs9e2bVtWrVpFZWUl+fn5vP/++7g1cEMdMWIEhw8fprq6mpSUlHr3kDF16lTS0tKorq5m7969DBpkTWXfPqyZvoZjvzh4pWeE5sh7yqRfnSIqZ840KFcC3jVxTOvxGlP6TrHp2qbkPc+ehVtvhaVLBW6OfRyA7JDF1ChLLF5LEMx7xDcuK/DQrocY8coIm+r5Ly4fFJ4r5MerfuTgtwcdel21Wk11WTUXNl6gqtgFO1EHwd4Nd2vO65e5O5MNszeQfcA+i9neT/aycc5GvZefHMUTHi55HyuV0kazoVdTa0NZGQgIhMsDnQVkqbJYcHQB/5z7R39MEKTFz6efSkRLYaF146O6VE1ZZhmqbOcSCzK8Ar1oe2VbAmIst1FejApCfQ/yhjn91GrHy9c0FbaSXM2BuTy5roKuVsfGFzZy6LtDVsva+nvIz9QZ/bNEVa0nvgMD4Y+UH9nX8RrSPVZblJqSf19XRvo1JP0ubr7In3f/aVdUR1NQklpCztGcJhlqcstz6fZlNzp/0dniRnrx8cV8f+R7fQSNq/ptRQVUeUhkjkzuOBpZZVnMWjfLLo982ZHB0aSf0l3JM5nPcNMPFpKyNQMtnc/vYslF1LVquyLhrUE2HudW5FJd2zwiR15fNYxiqqmsYee7O63u/za+uJFfrv0FndZ1+uv2RPqB1HenThXx8pK83o18cE3i0CGJEImKAuNsGXIOzBj/GJQKJYmJ0lqttLTlcvbKkTGWcmFl7Mpg7yd7KU1vXuqGm+6+m5+9vLgJ+NnGc37y9ub+F54gvIvtOWWS8yRrdo+IHlbLiqKUV3HVKjhxwnC8VlfLzHUzifkwhkuqS5zMP8n6lPVkq7L187atexBfXwNx4Ao1CntzrgIoPZWcWX6G/BO2v4jylGYuT5Aoiqx4eAURB6TciI52cpL3dG3aQJ/oPvx484/c0VuaB4LSJxHlFm32XHkt5OckYSJ5jVNTU18+szmQx21nRfpZWreeKkymzOcIgs6dQb53WLyenEvLHsxePxuAUO9QUqelmswXaA6ZldIkHe3ZfPK2S5gUBXAq3xAFYCnSb/9X+3k74G3Sdzr2mVjD8JeG8+COBxtF4NiyB3FGXj8PpQfbH9hO4exCovzq26vjA+N5etDT9IzsafZ8OYrLW52ATqdtljx9sHcw0X5S3z9VcMpsOfm5GgdV3PDtDVz35XVNv7kd6H53d4bONq+5WFMj5X6H+oSMcaSfsUNBibqEz/d/johIvzb96B7RnW7h3ajWVvP7yd9tqpOH0oNRiaO4su2V7DZKwVhbC7/9BsPbmif9anW1zB09l95ud+Jf1cNxpF9QHelnJtJPFEVO5p/URzjKyKvIY1fGLo7nHScwUCLARLG+Pb0ir4JPkj5h3ax1Taqbu487d/x5h9XUH+aw6eImNFoNiUGJdAztaLW8vH8KN1oOjUocxdHHjvL6qNfrlXVzM6zPhwyxXhfjd0yW+mytuGxIv/9PqFZVs/eTvVxYd8Fp9zAlh2YNDUk/2VO1oTFC3hia85gxBVPynnv3SgPm0qXglTcE/6puaBVVLDy20Or1zHnEtzaUpJaw8omVZOx2rrFORq2ulvUp65m6aipzNsxxyT1bCj5hPuQcyaEi3/FW78w9mfw85meSFznQ5cvJkPuprQabpuTGdBU6XNeBB7Y/QMKIBLvOu3XprUzZOwX/aH80GsMiJiJCIo3k8bClDEm2QBRF/WLblqTdncMk98QzBWcafRcebpAxlMdtcxj01CCevfQs0X3MGwQciWpVtU0GTfm9DgmRDIEy5MVdbm7LRQOYgzzPyRtMnahj2allnC447bB7mCP9sg9mM3/wfE78dqLxSQ6EoBR4PPlxxr431mpZW/MbyhEmjpY6PF90np7fdSIn8G+8vCQP/J6RPREFLQfb3craE+a9+F0Z6Sf394qK+lEzCSMSmLh4InGDzWgTOQi7P9rNN72/oSLP/jl1UfIitKKWIK8gAr3MLw4n9ZwESPJ5lqJoHI3KSqjykAxQspHQ4feoqeT93e/z7aFvbZaerFmxhrjjqy+rnH6i2PKkn+wBnxCU4LBrhnqHkhScxKCYQZRWN4/IMRfpp/RUsvW1rRz5/ojF80sulJB7NBeF0nXbeHsi/WRER8OjdSndFy405NoBadw3lv3ct0/6HDSofuRMn+g+HHvsGL9O/BWQlCpk44qzpJ6tQTaOWpKCHPDEAKalTiOssx3etSYw5eGH2VxTQ19gM2AtpmwvsEWj4aEptjndykjOtZ30y8gwzMPG+2w3hRt7MveQX5nPy5tfZtzCcVy98Gqmr5lu9x4EXCvxKctAFquLqdDYNse5e7vzn5r/cP3X19t8H6vyniLcsvAW4u+XZPYcLSeYnQ1aoYqoNgbnHXkfUFkpUFkpjSmm8rydzUtFK6idlvvW09MgB+0oWVP5udpD5toCW2Rafz4m0fSRpddTnm+bwa1cU67vi5awNXUrv538DYWgYNP9m+zOD5VdLQ2e8b7WjebWoCf9jMgiY2f+hsudyB6R9LqvF97BTvR8tAO2kH5yXr/kZMfvKS2tiS1BT/pVJ6FQmHcksBXdIiQVmIZEkDEskbmuQJ8H+jD8peFmv09LkxyB/f3rB7bIUvOlpfWdSD7f9zll1WV0j+jOjZ1uRBAE7u0pZc+V+68lpJWk6dfzooie9Lv9dulzwwbo7CMp3p3IP8G+rH31zvd292bqwKlMFH9FQOFw0k9eCzeEVjSM/8aR/QuPLWTo90OZu31uPWdq4z2Id6g3XkFeLdJ/tTotX+z/ApCi/KyNezqdYe1qK+fx8MNwyy0wYoT1sm3bGuashAT4Yt8XxHwYw3Prn7PtZi7Ev6RfK4SHnwcz82Yy+u3RTrl+ba2hA9gT6acTdQR7Beu9keSBQKWSrgn1JbKaQvoZGyjlha5GA4sXCyTkPkUfj9vo38Z68mhzHvGmkLY9jXUz17lMts4Y2QezOfDVAYrOOyFhSwO8s+Md2nzQhqsXXs0X+7/ghyM/ODznSWuCd7A3zxU/x/AXzS8OmorQTqFc88k1rS43liXY62UrB0nbmovFlfCP9id+WDye/vatcN293YnuK1kRZGLP29sw/lwOef02XdzE8/nRHGv7CDYE+tEuWHJDKlYXU1zVOERKHsetRU+5Wsb2j7v+4C3/txB1lscoeSHacC5r00YyHFZUtD7nD3mDKRNFb+94m1uW3kLPr3ry9o630eqaL/lgKnoewNPfk6LzRVQVOTdKWRAEQjuEEtnDuuSL/HtY2+zY+q7aA61Oyw2/3kBmeRrn2rxOQJD026+8eyUJtePQKiuZsuk6swajhs/SmTC+h7HcWWBcIN3v6I5/G+fkgJbR6YZOjHx9JD5h9jf2p2M/AXB/r/stlhvQZgB9ovpQVVvFrb/fikZrRXfYQSgur0DjLg38cu42R0OWBKysqaRcY9t6syYlg4CCiw4n/coyy0helExJWoljL4y0ZigvlzbC9srdOAqyoUOe/xwBQRBIeTqFPVP2EOHbvHxE5iL9FEoFD+54kDuX32nx/Im/TmRGpn05nJoLeyP9ZIweLRF5Wi2sWWM4/tFH8MgjcPSo9J0x6WcMLzcvekT2YHCcwSu8Y52duqVIPzn/W1FVkUkpSwC/SD+C2gahcGueqSUqKoq333mHyZ6ePA7cgHniby9wg6cnT978FCk/p9h1nyO5RwAsRpbIOHzY8HfDNcb0K6YDMO/wPDLLMukU2omvrvuqSe+PvA9xRaRfgGcAfh6SRdseiU9718bWyCJBIdDu6nZ0vVHymMhwsE9wZpbIkcT7eDf9Rr0zt4eHwSB6/lIJd/5xJ50/79zo3Z626xbW9PGnILBpER7WYKzU5CjST3biya/Mp6rGcetea85qWp2WX5J/ASCm8F6byNttaduIfD+SW3+71aJ9RqvTMm3NNAAe7feovs/mlOfY5BgPkFcrDZ4J/g4g/cIbk34yOSSK9R36AeKHxXPzgpuJ6m1rllLHYOd7Ozk0r7H6iC2kX8eO0pqmpMQxjsE12hqKqizb/goqC5h/aD5v73jb5Pdvj36b70YtJ7L0Ory9tWZlZm1F1zBJwu1EnnmnUFOyrZUFlax6apVTFepshXE+P+Pfw8PDsG44elT6LNeU89GejwB4YdgL+kjZu3vcjYDA1rSt9eTcX9/6OgezDephO9N30vPrnry46UUATp+W9qc+PnDXXVJu1tpa2LEujEk9JxHqHUp+hemXxxbZZ3ugl/c0E+lnPLa7Kw0pG0K9pYlAfjdN5fVTKBU8euhRrnzBevouUyg8V8jqp1fbrU4jiiLT1kxjXco6PJWePNzvYavnlJRIxJ8gmHZY02g15FXk1TvWty888IBt+cjd3eHGG6FnT0kBMVuVTbYqm8qaSusna5rnPGgv/iX9WhimkjgKgoBvuC/+0c4x4JSUSJOwUmmfx+Yj/R6h6LkiFtwsJWv18zNEVsiL+epqQ24oexb2pgyUxgvdigpoW/AoL3dZyhWxV2ANMuFoTd4T4NLBS+z+YDcFp11v6e9ySxdmZMyg042dnHqfjRc28vzG58mvzCfMJ4xH+j7C19d/jU7UodVp2ZWxy6n3byk4mqhQKBQkJSURkhjCoKcHEdapeR68roS9G27Z8zMvz3K5loJOq6Ms03Y2J2NXBgWnC/QbKWNpT3lheDmQfkdyjqAiB42yyKZIP18PXyJ9JQOVqcWfPUTK+bXnOfzDYesFHYC2w9vS896eVvuwuah1Dw/DO5yVJfVbZydNvnRJypdo7bc0jvRLL03X5+So0dUwZ+McRiwYgaq6eRYPczn9QjqEMDNvJv0fte480xyUpJagLrFNmtFWeU9586FWGwwuzcW2tG2cLjiNn1sgA86tIDhQWtS4K9150O93Aiv6UlpTyKP/PGrSCONKeU93d0M+VlOyrdoa5+YHSBqTxPCXhuPmZcNOyAjHco9xJOcIHkoP7uxumcwQBIElty4h2CuYvVl7+eTCJ3Z7sDcFl6qkKD8fZUCTva6twc/DT29UthQhZIyrfryfEyOecDjpl7k3kz/vcY4krBzlJ3tWuxqiKBpIv5AWSipoBfJ8ZWosi+odhYefdVlSV/QLY9gT6SevlRUKBYIAo0ZJx+V0ELW1cOCAZIz56ScpN7xKJc1bcuoIS5CNd6by87gCId4hegNhQaXpBaOoEylNL6U0o/mGnRnPPsucuXP5SqkkECVXAmOBX4ANwEJghMKN4Uolc+bOpXNqF/Z/vt/m66tr1froIlsca48cMfzdkJy5ufPNeseJKL8o1kxaQ4B7qH6et8ch2JWRfoIgMDJhJFe3u9oux6vsA9mcX3Pe5vKycddcVE61qhpRFImrC9zPznZcPqXaWtghvs2lkN/Zk7++ngJIVFSdTL5Pd7albyOtNI2/z/yt/15dqyalPBlRUUust/NsFvK+xlHOekFeQfp5N7PMcWGTxjn9TGHTxU1kq7LxdwshovRam1Rz+kT1QRRFzhaeZX+2+f679MRSjuYeJcgriNdGvgZIxuakT5J4YPkDNpGbRWIqAO2CHRjpZyTv6eFhWK+2VFRYQ+z+YDdHfjzS6LgtznseHvY7Hmq0GrPG/21p2wh/L5xbl95q9vwSdQlTVkzhpU0vmSSLOoR2YEjojfhqkoiM9Gv2Hlcf6VdgW6SfvCVy83Jj/+f7napQB9Kc+sOVP7B97nazZWQ1gSQTPvm9e0ufMum37NQyiqqKSApO4vZut+vLxQXGMSJhBICeuF9zfg0vb3mZQfMGcc3Ca5i2ehpXL7yasuoydmbspLq2Wh/lN3CgtPYdO9ZQp4/HfcypJ09xXUeDDGpxVTHfHfyOtJI0h6f90Mt7msnpV6MzIv0UBtJPDuxpSPo5UmGn6HwR+z7bR/4p+9jzipoKdmfuRkDgl1t+oXdUb6vnGCsMNOwei5IXEfBWAA+vsE4eWsIDD8Cbb0pjRH6l1KZwX+vS6goPGzz3HYh/Sb8WhqlNW1lWGYVnC51mwDGWQ2vOnlEQDIt3mVCQJ0IPD/vCzBvKe2q1hkW+cWSSLDloDfZE+vW4pwdPnX+KuCHOlcYyBUEQCIgNwCvQQa4dJqATdcxaPwuAR/o+QvYz2Xxzwzfc3PlmKmsq6fxFZ6784UrOF9m+ablcUJZVxq73d3HpkGN2jIIgEGBLeFUrhDxh20r6RdYF6OTltT55RICF4xbyZfcvbY5WXTV1FT9c+QPUFZdJvwgjh31ZErI1k36HcyTSLbCqj02kH0BSsLT6NbX4kw14tmxidry1g7XT17okQnjo7KHc8M0NVsvJ85kp2QbZaJKZKfVbZxpJa2th7lxYtgxWr7Zc1niDGR8Yz993/s3TA59mwc0LCPAMYEf6Dh5b+VizfmdzEiyCIDjdWCyKIotvWsxH8R+hq7Uu0WrrZsfLy7CucFS036/HJcm4oUG34VUbVc+gHRfpR5+Lv+CGJ6vPr+anoz81Ol9eszhL7qohzMm2/nrDr3za7lPXVMJOLDgiOYrd0PEGm/LWdAjtwNLblqIUlCw6uYgfj/7o5BpCnloi/aK8nSPtKUMvC2ijdGlEjGQ1czTpFzc4jtv/uJ34YY5vr+xp3a6F+LaCygJUGomJcKS8pyMhywhD47FM1Inkn8qnKMW0laW2upYjC46Qe8y1MgzyPs8W0k9eK8tzTac6juDiRYnkvHjR4CB69izMmyf9PWBAfZlugO8Ofsdb29+qR1DIpN/58/Wljl0FhaAgzEfyEmvoJS6jWlXNx20/ZsvLWxxyzxnPPEPyyUyUHV8l0KcDh/38mObhwV3e/jzr0wHhyldIy8xkxjPPcNeKu7h3/b02Xzs5N5kaXQ2h3qFW5Y1ra+vntWq4xnBTuPHZ+M8YkzSGtZPWkhCUoHfAVSrtmytl0s8VkX4Af9/1N2snrdVHLtmC9bPX88ddf9hc3lqk3+93/M5HcR8RFiri4SH93rYorqhU9X+nhccWcuvSW1mXso5aXS3ppenMXvVfTrWRolI+Hf8Zg2INYbVSVKVAdUUID/Z+EDAYvEFy3tGKtXjUhNHGx3nzpLyvcVSknyAI/Hbbb+ydspe4QMfZeaxF+smE6fi4O1CKnjY50Pp7+jOhywQA5h2aZ7bc7d1u59vrv+WDqz/Qj0PRftH4e/pTq6vlaO5Rq/cal7OR0cfSGJtwjfWKWYHcX1JLUuuRXJae5conVrL1ta3Nvrc9eGjXQ9y84OZGx23dg8hzX8N0GGXVZTy16ql6DiB/nvqTqPej6P5ld31ErTFWnluJTtTh72l+I98+pD19o/uiFbX8ecp0UlxpLyAQHOzR7L1d13DJ48YWec/aWoMDg4efBzMyZnDTj87JES2jprKGwnOFFh1p5PWnKWl5Y9JPFGHxicUA3NvzXpSK+gsPWeLzxU0vohN19Ivux53d70QralmbspZP931KZU0lV7e7mtX3rMZD6cmuuvgJOSexsfN8qE9oPSJIq9Oy8eJGHvnnEcb/Mt6qM4i96BDagbFJY7mmven+bTbSz0cyphRWSZsO2WbYkPRL25bGulnrqCy0IaKtAZJGJ/FM1jN0u72bXef5efixdfJW/rzjTyZ2nWjTOXK9TdmI2ga2pVpbzb6sffXsLW9tf4th3w+rF+VpK/Skn48V0q8qF+HQ03Zfvzn4l/RrYWhNuG/t/2I/n3f6nOIUB2pYGUE2aNsj7WkODb1e5IV9UJB9hKJMHhYWGha4tbXS4He/kRpUbKw0Gb2w8QWLobP2kH6+4b6EtAtB6a60XtjBSN2aSnmOc12gfk3+lcM5hwnwDOCNUW/UG9z9Pf3pENIBnajjw90fOrUeLYHyS+Wsn7We08sdkytLq9Vy9NBR3gt/j7XPrHXINV0FuY/aSvrJBFhlpemokpZGl1u60HdKX7Qa25wjRvx3BGPeHaOPHDNF+smLgtZM+h3JOQJAQGVvm+Q9wSDzYErb3R7PxdFvjWbSukl64rQ1wNJ8JpN+6ek6kpOTTc63jsKKFZCaKv1tzUDVMKff+A7j+WT8J9zX6z5W3b1KIjuSF7HkxJIm18eYHGrIHeYm57J97vYm5WazBaJO5IoZVzDi1RE2SZvZkhsFqJdfwBGkX3WtIVF7f6+7gfrjY0QE+Kk7M1L4LwBb0+obKGpqDNLmjvLOtAbZE7nhmBzZK5K4IXFW5XCbg8U3Leav+/+y+7y/z0rGL3kTbQvGJI3htRGSB/vSE0vtup9WK0Xc7rc90IWCWmlzF+PrXD1KWeLT1kg/P/dqAnPPos3J1xsFHAH/Nv50uaULgXGOj2psLfn8Yvxj8HJzrEPd/EPzSfokiWmrpzX7WvJY09CAWJpeypddv2TPR3tMnleWWcbyycs5+rN1w66jYJy6wZY1pFarrTfnhoZKc7QoStF5J+tsivI+UTbUNZT2BPju0He8sOmFetJxsbHSfKFWO17+0FZ8ee2X/H3n32aJZa9ALwbPHEz7ax3XEUQxig4dX+L6289SoFJRUF3NnuQyBo46S1T0S0RGSnJ5/tH+hHa0MYENktHv9m63c1Onm6wajk+frh+daiqK54ZON7D+3vV62UF5Lx4QYJ9tQJb3lJ2AW6MD4pBZQ7juq+usF6yDtfVOzMAYksYkoVAKxMZKxyxJQ65fD889B/fcI8nlnqrrJouSF/HHqT/4+8zfjPhxBB0+68BHh18FQWSgdgaP9X+03nWioiSHrUOHsrm9qxT1surcKkrVkmHnQPYBAAIr++Pv7zzHMXMqFc3BNe2vYWDMQIfOB9ae41tj3uKGjjfweN+nAGmvYsv7+2g/6bnMPzyfY7mm5RKVCiUP93uYB/s8qD8mCAL9o6Uo3f1Z1hc/VZUKvDXxRAQ235E5wjeCMJ8wRMR60X6W8r+dW3mO86td62wenBRMSLvGjmcN92TmIJN+DVXEZq2bxef7P+eGX29AFEVmrpvJxKUTKVYXc7HkInO3z61XXqvT8tfpvwC4roPlseOObncANNoPnis8xzs73mF92j+IokhFRV6z97hdw7vi5eZFqHeoWWl9Ly+DY47xcw2Ida5jLdSlv8qZyfVfmc6hqtNJDkVgOtKvQwep/ioVnD6vZneGFJon/8bGmNh1IpG+kUT5RZGtyibcN5xfJ/7KwUcO8tV1X/FE/yd4buhz/H3n3/i4+5CZKdmuPTwkeUgw2NEKC+s7J/11+i96fd1L71Qxrt04h0f6xQbEsu7edXw63rQzaK2uVv+3UjDYvxtG+sl2sYakX+beTHa/v5vCM/Z7JCo9lPi38bc7RQ9IxN/NnW+2ubwlx/A+0X1QCkpyynP0kt7H847zwqYX2Jmxk5uX3Ex1rX0bLzki12qkX8EuuPiL5TIORgsIr/wLa0gclYioEwls6xyJIXNyaNbwzNpnSM5LZm7PmxhwYS6M3UlQXfiwvBk0Jv3sQVSU5BGkUklh0LJBLyYGhg+HTZukjUJkpMiQz64jtSSVHhE9uKvHXSavZw/pB1CeU46mXENIexsTnjkAmnINC0YuoNtt3bh1ifnw/uaiU1gnhsUP49r215ochGYPnc3q86v54cgPvD7ydb2Xx/8CIntFct/G+2gzoI3DrqkuURPZMxKfcBfouTkQ9pJ+np4SGV9aKhFk8uK9tWDAEwPsKt/phvpyNLImf7hRl5D/doRevzNQVVPF6QKJwA6o7G17pF9QEgKC3nPLGPaQKHGDXRMNnZucy7bXtzHgiQEkjEiwWNYS6ScbTDIyTDvYOAr5+fDLL/X/bwlVVVCjLAUvgPrz/ND4obw1+i3OF53npk5N95qUST+dTvLGNDZMpO9IZ9OLm4jqHUWHazs0+R7moFAq6D25t83l7dnsBAdLpKojSL+1KWspVhcT7RdNdPVwjlJ/7SI7BHQseJbpT/VgfPvx9c43zlPiKtLPXKTfqDdGOf3elYWVdpOK6lo1KUWSRd8WaXZjTB0wFe8Kb6aOnWrXeYcPSxG369bBggW2ec4Gl46gR9HX3D3cuTlm7I300+SX0GH/r2R3uJLCwlH6XHCtGZY8rV0BTzdPbulyC8FediafswFaUcvFkoukFNuXL80UgoIkMqPhWBbYNpArX7yShJEJJs/zjfDlzuV3EpQQ1Ow62IrycoODg63yjA3n3M6dYccOSeJTJoZvuknK86dWS3JYsrHMGLIkX1yAYf2hUEjv1/HjUqRgS+SOtMXT/Or3rnboPeWIryijYaptW+m3Ky+Xvo8I11GSWoJfpJ9NMrEAvaN6s+RW25yMZGlPpVJysLAlIku2Ddgj7QmGdubnS7aBV1+V3pHp0+27jr3Q6rSNIj/MocN4+9ZQ1nI3jXh1hP7v2Fip3ZmZkmRcQ+TmwqcNbLq7dkF0YgkbLmwA4K7ud/HX6b/QaDUotT70SP+KH5+7r9G1ZHWX/HwFPSJ60DW8KyfzT7Ls9DIm956sJ/2CKvo7VdnAXD7q1gZrkX5+Hn78fdff+uetVktrRmu/3fC2w7m16638fvJ3nlr9FFvu36InUyo0FXi7e+ulhUEalzdvht274ULOAIhcZVEaFCTy0dEkw5fXfkmwdzCdwgx7bUuk3+PJj+Phb9v45AjoanWU55TjHeKNu497ve9s/S0aqpsBrE9Zz7eHvgXgrdFvIQgCnUKl32BC5wksO72MT/Z+wmP9H9Or7iw+vpiLJRcJ8Q5pFIml0UhzmkYDffpIUZ3PbXiOrWlbySnPIcpPGhTnH57POzvfYXDQLYRwHV5etTQXId4hlM8ptzj2CYL0XEtLpecq773VpWqy92cT0SMCv8iWMRjl5Um/m7s7JtfKbm7Qo4fkEHgq2YusZ7LYmrbVZGR3gGcAZ586i0JQ6OWBAfpG96VvdOOFiiwr2q6dYWwPCZHWKlqt9M6EhEiSr7PXz+Zc0TlO5Eu5E69udzW/2Oj86kjI76MxWSvn9CtRl1CrqyU4WKKKGpJ+ve7rRZcJXZq0Dq3Iq0BdqiaobRBKD9vm2WxVNkFeQfi422d7tUT6+bj70C2iG8dyj7Evax+xAbG0C27HI30f4dtD33Ik5wjT1kzjgd4PUFlTycjEkVbvJ0f7yhHYZhE3Ae01R4FedrWnOfg30q8VImlMEmPeHoO7t7v1wk2A3AHsjfTbn72fDRc2EJS7GqouQc7GRgZje3I+GEMQDPIvZ84YvNpiY6VB+o034PXXQaEQ9N7iPx1rLLUlQ56YbSX9vurxFX9N/su+SjcToigy/rPx9LrfuR2+f5v+bJu8jVlDZ5n8/qq2V9EtvBvqWjWbUzc7tS6uhtJdSeKoxCZ5k5iDZ4gnk9ZP4so5TUtg21KQ+2aIHbx2a8/rZyuqyxp76piK9GvtOf1O5J9AK2rxqAnHq6aNzaTf7KGzqXqxirfHNE4Gbm/klFajparIer6I5qDoXBEnfz9JRb51V1/b5D0dWDkTmDdPMuTI854tpN/FiI95pTSWT/c29sKbOWQm39zwDd7uTd+Ne3gY8mk13HB3vrkzD+560Cqh2hRU5FXYHb1uL+kHjiH9ZGnPO7vfSVmptPEwNkrKRrCCPDeu7XBtIy9WY+9gJ6eL1MMc6ecKPLjjQe5aYdrRyhwEBJbdsYz3x75PhG+E9ROM4Ovhy8jokfUMXLZAzp1TUSGRDLbArbQjbQse5abOzpUn0pN+Nkb6hXUKpWT4DRS16eFQic+/p/zNu6Hvoik37c3dVFRXG0iAKOfyp2bRO6o3f9z+B/NuNC+P1lTIxFNGWfPDy8xF+gmCwKg3RpE02oSrOuDp70mnGzsR2TOy2XWwFXId/fwkg1pTIO/xTp82RCMNGgQ33ij93bt34zlAo9WQUy6FzjeU5pMlPs+ebVp9LkfIKgKRRo/ezU3KnwkSmVqeU85n7T9j44sbnVIHmfSTpdJsmYuaSvqFhEhrGZ0OXn5Zmve3bnVcTt+G+OPkH4S+G8pNi503D8h1t8UZxdhxzRTktW1UFEyZIv195AisPLuSGl0NXcO7MjR+KKvuXs0A3VMMO7WPO7vcRxcT6qXyeF1Q4I4gCNzVXZrrfz3+KzXaGnakS5NpYMUApzqBOiPS73TBaT7Z84ndqgGWYCqn3y/HfuHTvZ/Wywnp6WmQubR1b/nB1R/g7ebNtrRtLD6+WH/8v1v/S7tP29Vrx7JlEvG7fz/4l0mRfvsyD1i8/vJTK9kbfzsZoT84LB/1bd1uY0zSmHoEiSXSzzPA06V5aUvTS/ko7iO2vLql0Xf2RvrJ8+H5ovM89PdDgOSkJueBm9J3CoceOcQft//B2KSxaLQantvwHCA5FLy2TVKxeHbws/rfS04RcccdMGcOvPIKHDsmSZQPihmETtTplUlEUdS/F4N87qqru2N0rm1xdjD1XFPWpfDz2J9JWdd8hyhzKMsq4/iS45Smm5b3lJ+jv7/5PVmvOpPr0aPg7e5tVv4SJOLP+H22BJn0k+dikOog2yZku4CH0oOVd6/Uk2seSg+Gtx1us+KNvSjXlOujtY0R6RdJytMppDxd/3kFexuc5UrUJWZz+vlF+hHSPsQmNZ+G2PfFPj7v+DmF52zf1ExbM42AtwKYf2i+XfcyTmlmCoNjJS3Wp1Y/xZ7MPXi7e/PNDd+w+p7VCAh8c/Abrph/BTPXz7TpfjbLewL4ujat2L+k3/9DNFXeU9akTms/A+4Wof2URhNgU0k/kLxAQdoQygvcOBP94b5ekofaupR1ZKuyTV5LjvRrGIJvDoOfHUzPe3vaU91mw9Pfk4FPDnRKpAXUD90WBAE3henAXkEQ9AsVeVH/vwRRJ1JwpoCqYucSFS2Fs4VnmbZ6Go+ueBR1remdcHW1YTFkT980zuvX2qAp17D01qXseNv6O/ttv2/5eezP9Y5ZyulXUmLwam9NOHxJyucXUNkbdzfB5s2av6c/nm6mLQzyQqih4dEUqsuqmes71+nStl1u6cKLVS/S6cZOFsuJomXSTzaYFBQIVFc7Z3Op0aBP3v10nTx7QymPhiitqCYt4iuqxXKTRIgjNsKCYN5j2j/an7jBcY28XR2BHe/s4OO2H5N33PZBQzae2LLZcSTp98wVz/D0wKe5r9d9Jtcu8nhQUWEwQBVXFetzY8rHHGU0sQXmSL/y3HJWT1vNyd/N5+JoCXi6eXJT55t4dsizzXqv1bVqvfHfGi4Z8WnW8muC1FflDbezn+WcK+eQ/Uw2c0fPtV4YcPN0w3toX9T+4Q4l/ULahxDVOwp3X8eOAfKa283Ntf3CVZDznWWUNp/0syefrjFckVO3IexVijAFmfQ7fFgyICmVEnF3113w5JPwxBONz8lWZSMi4qH0aOQ53aFu63TuXNPr1BycLjjNL8d+sbhv2v/VfuYNmuew/YdM+jUk1I1/C6WHkiGzhpA0xjRp3BAarYbzRedteq8qKgwk67Bh0qctEVlNJf0khZ/616itNZDGjoavhy9FVUV2kfpHfjzCB20+IHOvbd5lloy7659bz98P/41OKy0grTmuyZGfbdvCiBHS36mpsPiYlGPwls63AFCT1YPIQ58SXNuNyZNNX0v+nYuL3dBqJWcogI0XNvLl/i85V3QOd3wIqRjiVNLPGZF+ezP3Mn3tdIt58uxFw0i//Ip8nl7zNNPWTGuU/9ne1BHxgfG8eOWLuCvc9e+iRqvhxyM/klqSiqfSsJ/bskX6vOYaiFVICjhni06jqjYfgrszbQ+XQn6jxG+vU1UqLJF+5bnlpKxLcZltxt3HnUHTBhF/Zf18lKJo+/rP2Ob524nf6PtNXzLKMmgX3I63xrylLycIAn2i+yAIAh9c/QEKQcHGCxvJLc9l8fHFnC08S4h3CFMHGlQsjh+X9pK1tY1lrxtKfO7O3E1aaRr+Hv50FK6rq7vrktvKz9U4yjtucBzjPxtP7KBYp903a28Wf9z5B+k70k1+b8tesndvENGSfFykpsZ8OXshk34NZUVlW7uxM3CH0A6suGsFod6hTOoxCW83X6sR4E3B9DXT8X/Ln4/3fGzzOW4KN94Y+QafXPMJnkpPvY3I1Dq1Ir+CgtP2e8m3vbItw+YMwz/aRs91YF/WPrSilnYh9iULt2QjAnjxyhfpHNaZbFU2T69+mtSSVECShH51xKsAeLt50zW8q1kbqwxRFOke0Z3OYZ0tO7lWZEDGX1DjoMS1NuJf0q+FoWjgiqCt0fJ1r69tMmTbg4zSDOZun8usdbNYXPYUhX7b7Jb3lEm/IK8g/bGGm1Z5ARrZBAdUYy9Q40i/hmgf0p6hcUPRiToWJS8yeS175T2HPT+M/o/2t7PGrRc6Ucfon0Yzc91Mi7kPZQyLl3Zv/4uk38nfT/JF5y84u6L57sAKhYLS1aXs/WivA2pmHj8e+ZE7f79Trw1tCsm5ydz46410+rwTn+77lG8PfcvLm182WVYmdDw8TC9qS9Wl3P7b7Tyy4hFO5J3QH5eN3q2R9HP3defC+gvkn7AcVqXVaGl7VVtirogxHNMaFgLGpF9AgGSsFMXGXk2tAUFeQfQPv5KQ8mF25001e80g6VOlwuoC2DPAkx539yBmUIzlgg6Am6cbbp6WFcjLyyXSDUx7cfn7S0YmiQDr1Gi+dQSys6X3xddXkp1SKKRNmyUSdY9qCdXuuYS4xzCxi2mJMHWtmje2vcHtv91eL+G2PbAUFaar1VF41oEsQh0SRyXS/a7uhHezwcutDtZkkozhSNJvQMwAPhn/Cb2jepuUJvfyMqwl8vJgwZEFRH0Qpff4kx0pWoL0q2wwrQsKgX2f7nOal21FfgUHvjlA3gnXTQYKhYJOnTqx6vwq2n7clidXPWnTedlGvmBnzhhyfJhDZSVcCvqDQr9tKD2dFEJShwjfCKL9o806YZlCaCgg6igocBzZM+z5Ydy38T6He9o3Na+3I1FcVew0YkyONitWF1OhaV4oirlIP4Bzq87xWcfPSFnfuD9vfGEj7wS/Q/FF5+R9NwW5jrY6jcl913jObddOWl/Jc3a7doaI9GuuqS+1LiO9VDLuxQbENor4lSP9UlMN13Qlfj/5O5OWTeLHIz+aLVNVWEVZZhmVBdb3YbbAGul36pSUp37su2MbSdqbw+FLh+nwWQc6f9HZatmUFGm9ExVlkFR1JukHEC0FR+PmZrATJCfbfx1bEBsgGR1kSVlb4BngSVDbIJvHUkukX+buTDJ3ZaJQSu+6caSfqSHN+H0IDJQMzrWKCtZfXAPALV0k0m+DpPTJddeZt88EB0u/sa+vH6WlCtqHtOfB3g/y/tXvc3/v+3lj5BvcoF6KR22YS+Q9HRnpJz9XOW+TI9DwOc7eMJuiqiJ6RvZkUs9J9co2RUVm5pCZJD+ezOyhswFYfno5+ZX5RPtFc11HiehJT5f+ubnB5MnQsU0kXtVxiIgcunTI7LXPFEj2kCBtR4fN0+WachYeW8jbOwyKMpZIv1N/nGLhuIVcOmSb6kFz4RflxzUfX9NoXDSOGrY10m972c/c/vvtqDQqhsUPY8vkLWYjwnpE9mDRLYs499Q5Iv0imX9YilSaOXgmAZ6GfIpyDupRo+B2KaWm3oHttm63AdAlrAuiKPJrsqRScnPnm6mp9EYQoF27SIfscf84+Qe9vu7FkyvNr7dNPdeA2AAGTh1oVy5ZexEzKIZbl95K/LB4k9/bohoTHw958d+wvkM33l7zg0PqJYrmcwmaSxszOG4wuTNzmX/T/Hr5uh1J+snRZhdLLGyCTn0A668EraEjvDj8RZ4e9DT+nv71SL+GzszzB89n6UT7o6eTxiQxeu5ovENs8zjIKc8hvTQdAYF+0f3supdsyzPHecQFxrF3yl4mdJ5AtbZa79QL8J/h/+Hwo4fJmZnDzxN+xsvNC61Oa3Z/IQgCWydv5dSTpyzn9Mv4HbZPQFHkXFtyQ/yb06+VobKgEk2FxqQcXXPwwPIH2HixTurDHYROX7Gn4meGYZtUU35Fvj6pZ0zpftgyAHq+QXDwi4BhQ9gc0q9jR8lIkJdnMOiZivQDKdpvZ8ZOlpxYwswhjUNujUk/UWw544Ml/HrDr9RU1nDfxsba+s3Fl/u/ZFvaNg5dOsTTg57Weyebw9C4oQAcyTlCuabc5nD2ywFxQ+IYNH0Q4V1tN0JbwuFvD+Pp78mQmUMccr2G0Gg1PLD8AQDOFJ5h75S9eCgNuvcFlQXM2TCH7498j07UISAwvO1wyjXlTO492eQ15UkvONh0X/g/9q47vGmr/R7ZibP3DglJCIEwAmHvvfcss6Wlk9KW0Q3d/brpgJbSlrYUKJRVaNmr7L1ngDCSEMje0xm29fvj5loeki3ZcuD7fj3Pw5Ngy5JiSfe+9z3vOe9Hhz/ChmsbAAA/n/8Zw+KG4dsh3yIkhEQv9Ll+mMAwDF7Le82qH7hSpcTIX0YavUaVWE5OxpXrDEMWZ9nZJEALluZG53A80uIRRFc+gg93Aj7Sip3w3NbncKPgBv4Y+wcaeHOknacn+R40GpKYsaYAH71itPQTl4jUA6lw9XVFWJswi9vRBbSPD0ke8iEykvxd2dnOaNFC5hMFZyUYEUGUCwEB5N7Jy+MnIlmWxcGqhQCAUaEvwlnJr7RRKVVYcHwBSqtL8XbPt9EqRLoS3ZJN0qZHNyFpfRLmlc2DysP+vho6jQ4KJwWaDGuCJsOaSPrsg7L3NIRQUjswkMQR+flAYmQiarQ12HZzG4qriqFWk43rk/SjxzK9pu6B7pidNhveEd7mH5IB+TfysX3GdgxaOAjBLcQPjFuSt0Cr06Jbw26S7T0BQKVSIcY3BrkVufjr+l+4nnedt/+GIWiiJDCIRX4eg127gOefF96+rFyL87ETwTJaFNfcg6er4yqVbYHTyaNou/Mg8rq+AED+PnVygGVZ7L6zG1vPXsCVhvcR6RmNGu1so9ilvtD6x9bIr8zH8aeOIzE0UdZ9e7t4w9vFG6XVpbhXeg/xgdaJEiFYUvqpPFVQOCmgqTK3HAiIC0BQiyC4B8o/8OhYHW7k30BmWSY0Og3cnd3RvWF3FBWRZKIUpZ/KZFJWqUhCjCrF+CwGTUGL0Pi+56Agru90SgrnFlNfoONZboVwIUTPt3ui59s9ZTumEOnXqi48uHmTzKdS1Du0V1tjf+tNOGlBRUQEZ1kopaeftw3TU6dOxLKSKkGTk4ntnSNA7XsL1YWorK0U1T+o2dhmaDZWxM1cB5rg5bP3nH54OqrLuPxPeDhZl1RUkO/QND4xzbm0bg0cL9yLap0a0b7R+vGPKobaW6htZhjyfOflKVBQQJ6vX0dxVmpv9XwLM9cC9+DYHu+OsPekfdByyuVbzBqSfkfTj2L5xeXEDm74T2ZxvS2kn4uTi1F/vJ/P/wwAeLLNk/qioSNHyHtt2xKyNCwMaHrhQ/Tq5oaWwS0F9327iAzCgYy0WN0SqjRVeOwv0n7nxY4vwlPlaXGMiO4TjWE/DHMoSSQGdP3BMMLrSAr981ceimm9p8HT2ROLhiyyWsQ1seVE/e87pu7AsgvL9K2KKCjp16kTd050vI/wjsCR6UfQvWF3aHQarL9GiJYpCVNwvk717O0tT0pfo9Pgcs5lI0LSFPS61nffTe8G3mjxiPBCXsxakmGAktAtKNddR/J9eXq5FBWR8ZlhzPsL0zwSX9sPaqVKxxIx96AUxPgRr1E+0u9G/g1M3jgZizyz0ZPJBmrLAaU540gLp1mW60tI0WlWp3pxnjidcRoA0DyoObxcxKsDAetKP4DE9X9O+BPbb27XzxUAyTMariHm75uP5ReXY8fUHfatLSLHAgoV4O+YPLIQ/lX6PWDoTGhzrzAvzLo9C/0+6SfbMW7k38C+1H1QMArM7fwyQktGgGW0eOPUo0jOT7b6+SVnliBqYRQqayuhZJTwq6qzvUj6xMzek6/fgFi4u5MKDIAoThiGq/AzRf9G/QEAl3Mu86og6MJCqzWviOdD6v5ULOu2DGkH06SfuI1QeapEN1mXgv2p+/Hy7pcBkKbC1gg/gFQ6fD/0exx78hhcneqxi2w9wDvCG4O/GYzw9jxdfSVCp9Oh2+/dMH79eBnOjB/3Su7pK5ovZl/Eizte1E+qLMti3Ppx+OXCL9CxOkxoMQE3XryBg08cxKmnT6F5UHPefVqyZrpTeAffniZ9xXpH9wYDBv+k/AONTmOxp9/90vtGFrIPAmIbAJuC/j1BQeYk6MPe149PkSQG+9P24/Ddw7hdeNvodYax3WbMUdjy5BZsnr7Z6nb0GlkK5iIiSLB68uR9s/lWDpiq0gODyDGE+votv7gcWbgApdYdY6KeEdyvglHog0pq6yoVliqm44bFofu87tBp7P9Osi9mY3H8YptsPoD6J/0uZl/EtL+m4dR9UmVXUyNsf0zHg4ICoFVIK7QIaoEabQ02Xtv4QOw9hRJiDMPAN8pXrxCQGyEJIXh0z6OIHyUts/6fw//B2PVjcSz9mORj6nQ6XLlyBc0Cm2FU01FgweKzY+Z9SQ2h0XCJ0JrubyEpci72HqxAVRUhNPiQWpAJltGCYZ30PfcchYLKAry8+2U8veVp0Z/xjfFFaWAjFOVrrW8sEv/M+wcXV1yUbX+LTi3CkNVDsOTmfNwNXoKj7q9j8KrB+mLB+kK1phr3S+9DrVEj3Mv+mI8PlBigKjRbYUnpF9UzCi9ce4FXrdXmyTZ48uiTsvaqpnhtz2tosaQFBvw+AENWD0Gv5b2w+PRiyUo/+uyazrlNDf6c5vzhqhGu5BJJV6tg86IXhnmwff3EkH5yQqPhYh5T0i84mLym1QJ7v72GP4b9gbxrVpoL1+FsFiH9xFTQ0yKnBg24uai62rodvj1Kv4EDgfXrgX79gIQE8tqtW1zcICcMezhJUftJgbXeTYbPtUrF5VP4LD5NSeDERACsAkFVXTCy6SgwDIOKCq4QJtZKsaC/P4uyslLk5fHPlTTuqA97T0eQfgXqAtRo5ZEFG1oJfnXiKwCEkOsc0dlsW8M40hZsvLYRe1P2AgCeakN6yLEsR/pRq92wMCCy4AnEVExEgDv/wohlWaSWkgEzWCkf6RfoHqgfE6/nESbKktIvqFkQ2s9oD59IGwYFG5DyTwrWjl5rpiyk19Hd3bpAgM5/XrkDsGL0Cnw/7HtJrg0A4OrkipkdZhqRFxkZ5Bl1cgLatOGeZ0OreurItT91P3IrcuGp8kS/mH6oqCD3Qm5uiixr3EZ+pNg7pShFcBuh67rp0U34IeEHu8/BVohZS1ZpqpDKHgIARKiHyHJcqvKLiDAn7cTklAznBDlFKjG+daRfkTnpV1ZdhovZF8HUFAHOPoArV/GdXpKO4/eO437pfSgUXKxq6oDVaVYndJ5tPt5Zw665u7BuzDrR21PSr2ODjpKOU13NzSNCPf0oFIwCI5qOsFhUerPgJrLKs/RKW5vhEQU0eQE6Zf32QPiX9Pt/gGPpx6BgFBgWNwzvdvoK7W79jai8Gfi835dGVURCiPGNgVqjRvvw9tgxdQdcW/8H6H8EGHrZKAFXW8sFNKYLErEwrNQMDRWueIj2jYaXygtOCifehbdKxQXUYiw+dVodCm8XymbBIgbj1ozDpM2TZN3n1dyrGLNuDGp1tZjQYgJmduBpkiGAmR1molNEJ8kBzP83OHs6wz/OyuxhB2L9Y6F9V4sVo1eAAYOfz/+MWTtnoaSqBAzDYMnQJejUoBOOTD+CdePXoUkACdoNmy+bknE0YcNH+r3xzxuo0dZgUOwgHHj8AJJfTMbKMSvRJKAJgoMBFiy2KKdh+cXlqNHWoFZbi1f3vIrIbyLR47ceNlsPygF1kRqXV11G9kXhXk9bn9uK/W/vN3qNEjJ8dlJCVgwPGupaNcpryvXXUmryhAZ/fIG8FCIl50oO1o5ai5vbHJdlG7BgAHq+Y706ni44LFWPUzIuJ0f+/nUAl4hR+5/G8D+G4xt3d1yLeAUZOeY2gdnl2Xh5DynIaJL5AUK9LVe4tgltAwC4kG0f6ce34G79WGv0+7gfXH3sL/LIuZwDdaEa5TnSyz51OkjqZSAH6bfkzBL8fvl3LDy1EAA3PhJrK+NtKaFcUECItakJUwEAq66s0hOFfFZXOlYn2HPYHggp/QBAXahG2sE01FbKPya7+roidkAsfKN9RX+GZVncyL8BgF+pIwVv9SCuEqsvr+ZdwFLk5pIESIXPeay7/wVSQxYiw+kwVh88g8QfE3k/eyefxJCe2kijedRR+ObkN/j1wq+o1ohz9Igb3RK3O05BkVJiI24BsDoWx784juS/rRf9iUFqUSre2k+uTwev0WiU/Spc4IkDaQfw+N+Py3IMsUgrTgMLFh7OHnprI7nRNqwtOjXoBCVj373ysBXbAEC78HZwd3ZHy+CWiPMnnpHfnf4OBYUkmWhPTz/AeI0nRpm3cPBCXJpxCc+1f473fUr6PYi+fjTBnVcpHCyqi9S4uvYqci7brzCiY5uLC3/817o1+XnnXAlS/kkRXdBDlX7tw623uKBKv/BwMu/RBKU1xQddh9tC+gHERQEg5GZICIkbrjmgfS3DMHpSX2zfzqriKhz97Chu77pt9l55djm0NcbFGkKkX9KGJNz4+4aZcsLQ4tMQLGtO+jVvDkRUjESnq8fxesJCAJzKLziYU+gIwTDe4QO9zv9tPf383PzgrCBrALlIehq3lisysDV5KwDg5S4v827L19eLDywLHDoEzJ4NHD5MXitUF2L8BlJo3Ca0jV69c/cuIYucnYk6DCDPJWBscW6KjLIMVGkrwbBKhLrGWD4hiWgRRFRYSXlEof2gFGF8KL5bjJvbbqKq2HhtRuN4MUWHhu0wrBU6SMFZMgSjZUtyHlTwkJdnfpwg9yDEB8bj0YRH4ax01n+3bm7yFLVS0i+zLBPqWv7KCiHSz9XPFR7BHmB1jlF/7ZqzCwuCF6CygD9XK4b0O3L3CGrZKrjWNACTL4/9DyX9YngeJzE5JWuFILaCjhX3S++bFTvU6sg6cVuNFxBvPG698c8b6LasG/689icAWOzrZwuK7hSJLkoCbCf96Dzm6ipPce7klsQdcW3SWl6F46brmxD+VTie3Pyk8E5YFhAoQHU0/iX9HjKk7EvBxeUXZU3aPNX2KdydcxcLBixAfj7AQIGeZUvware5oj4/uPFgHJl+BKefPo2BsQMBhRII7g54xeonwIoKrreR0IJEDAwXgXz9/CgUjAJJM5NQ+mapYFNPmgimFYaW0Kh/I7ya8yqajxdRevqQorymHENXD0VpdSl6RvXEitErzHpgyI3k/GTM2jnroe8FeGPzDSxuuhj3T9pXuVl4pxCld0r1TdYdiWmtp+HLgV8CABafWYyuy7pCx+rQIrgFTjx1Ql/1ZYiSqhI8t/U5xH4baxSsGdp7GkJdq0Z+ZT4UjEJ/rLiAOExoQQzlg4OBMtckpPn8jumbpyPuuzh0/627vqpxZJORgvaE9YGyjDL89dhfuLr2Ku/7rI7F7R23kXHauI8DDb747DsfVqXf+qT18P7UG5+lEG9/qWMsDeT5bB6kECmMgsGtHbcc0g+Oovn45mg+zvpYbFihKQRqEZ2T4xibOVr5nuF0CNtvbYcG1UgJ/Rpzb7TF/tT9RuqiMxlnUK2pRmBNO8TkzLG6yKSkn6XeHJYgxiZJDmuO1tNa45WsVxDdK1ryZ6X00wCM1TG2FLUWVxVj9ZXVAICZ7UlRjKU+ZKbjwZSEKWDA4GDaQZzI3WN23uezzuPFHS8i8ptINPi6Aebumiur/YmlKvhzS89hRZ8VsiSYTaGtFe5jIISs8iyU15RDySglN183RYcGHTCg0QBoWS2+OPaF4HaZmYAOGlyNeRpaVov2rhMRVDoYn55/GVdyr+CDQx+YfeZOIcmK+iDK7D254e/mr08+5lSIu07WErGSwQBz0udgyGJ5qpxrdbVoGdwSvaN741nvTWh+fwHeCj2ONqFt8M2gb/Tbbby2Ef+k/CPLMYVAi1pi/WNl71dIsXLMSpx8+iT6NbLPkcVwLON7tG5uu4mDHxw0eq3kXgk2P7lZNlcSlmVxMI07xvjm41HyZgmuPH8F5587D28Xb9wuvI2LJaRwSqrDgCkSEsgY1qyZ9cprgNhctwpphWjfaN73aS+7h1XpV3q/FBsnb8T1v67bfTyq+ggN5VcDUNIvJaQL3qp6C8EJ1u2UK2srcS2PsGdSST+G4WIva0l9e+w9TUGtTB1l8Sm1r5+mWoN98/YheYtxEUXu1VwsabnE7BkWKnI68M4B7J6722zcorkQU6VfeTlHWNC1jKsrZ5t76RL5SUm/xtbdW/VzTWGh+Q1WU8P1znRkTz9H2HsqGAVCPIlkMrtcuEhUCuj6Y0/er9CyWvSM6inotiNG6VdcDHz0EfDll8SueCvhEeHv5o+vB34NT5Unvh70tX57qvJr1457DilZdL7oAD4/+jmK1OaLupsFZLB0r24EL3d51+960q/OltkagftLp19s6glmC9o+1Rbv1L6D6N7RRq9LcRrx8gIKvA6gxO0CikvkywNRa09qv+vnR3KpOp05WaRUKNGzYU+80+sdADBwHJHnfPzd/OHjQhIMQr3g6DNqats69LuhpFe0wjGxl3ekNwLjAwVdDsRcyz13yLotqHQgcnPkOc+UulpqPtJPDOHvKNIvxCMEbk5uYMGaCWRowf4JjTfg3RSo4N73dyXBWUElGbBorGY6fhXcLMCy7stw7udzks5r8pbJeDH5RVHb6lgdzmSSB0Qq6Udzn/7+8igoh8YNhZfKC+kl6Thx/4TZ+1llWcgqz0JJtQXioboAWOsCXHzT/hOSiH9Jv4cMF5ddFGVrJhUR3hFoGthUb2sXGMDd/cVVxThxz/jmPXHvBDJKSTaTYRh0b9idC0TLU4C840DFXXi6VcGpThhGF13BwbY/XIbWL5ZIP4BYUlqqyqZJcTFKP0clB4Rwc/tNHP7osGC1ii3Yc2cP7pXeQ4R3BP6a+Jdkm84abQ1WXV6FF3e8KNq2ceWllfju9Hfo8VsPjFo7Sp88e9ig8lBBp9FBXWifH8yphadwYPwBlGdxEWxOjnHi2h7kVuSispa7J17u8jJ2TNmBJgFN8FLHl/QkrtD96qnyxK47u5Beko71SVwgLWTv6ebshgOPH8C5Z8/x+v+7uQH+roFomvERgtxCkV6SjtMZp+Gl8sKykcswr8c8O/9i+xDYLBDj1oxDhxc68L7PKBjMSZ+DRzY8YvQ6HQf5SL+HUelXo63BJ0c/AQsWflpSGSE1+WbJskMK6RfULAjzyuehy8tdpJ2AAyDGXpGSfvn5TrJWZgIkUUurr1/oPAMvdHgBc2OWw6U2BFma6+i3sh8aLWqEHbd2AABGNB2BK89fQdec36GAk9XKs7ZhbQEQO0oha0JLsLbgPvvjWXzd4GubFHqmcHKxTSFOx06FglQsWwOd13U6cb2ETLHy0kpU1laiZXBLfeGEJfWsKeES5RuFlzq+BAD4KfMZaBRl8PAASqtL8dKOl9B+aXt8f+Z7vcpv4amF2H5ru/QTFYAl0q/x4MYY9M0gh/T12z13Nz52+xgVueIzcVTl18ivkSy93ajab9nFZYIqyqwsICX0K+Q5XYC/mz/+0/VbMGDQLPMjAMC2m9ug1RkrL64WEFI9jGlt9zlaA8MwequxrLIsK1sT+PsDobePQnX8AC85ZMs5eDfwhncDee6TJgFNcPzJ4/jzkT9RWkpik5bBCTj37Dl9n7BVl1dh/IbxmPbXNH0iwRG4U0RiUDrfPcygc7ihvbAhktYl4dD7h4x6vOdeycXF3y6i8I48tqmfH/scfVb0wdv73wZASDbq9uGp8tT3HTpVQ3p72av08/MDli4FPvzQvv1QUKVfZqZt84Ee2irg1k9AhvixmpJ+5TXlRnG7Ifxi/PDIhkeQMDnBjpMjoLbFQk46lAxLSwNKSxlR69rLOZehY3UI8Qixaoer1XLKMqooEtvXzx57T1NQi88rV+zfFx86R3TGwNiBgvaIpnAPdMczZ55Br/d6Gb0e0CQA4e3DEd4uHJnnMnFh2QVoNJxyx7Sn34SNEzDyV+P+4wCXC7l82TifQa+Fvz+3r8raSsS3InP0ubo87O06AaI40o9MMHzkFI05DMleR8AR9p4AZ/EpF+lHY9duYf0xocUEvNDhBcFtxRSTLlkCnD7N5c9SUsgzBwBzu8xF2bwy9I7uDYCQRNu2kfd69OD2QUm/E4HP4s19b+Lk/ZNmx8kpz4ETo4JHVRPZryPNI1ClnyV7TwBw83eDi4/8NtVCYBjGjJCSQvopFMC16Fk40qIt1l3eZHX7PXuAf6zUOVVWAlfr6pY7dKDnyY3zpqrNViGt8NOIn/TjtdxKP4Zh9PGTUE7P2nV1FLq+0hXTD08XbO0i5lruvrMbABBYMpBXSWkLKOnXiCfspDmlkhKuaMIUUu5BKWAYhuvrZ+JyQpV+bVW1wLFJQB4n3qBzH7XnF1L6uXi7IP9GPqqKZEqC8qBWW4v3e72PR1s9ioRgaXGUmH5+UuDm7IaRTckcvfmGOVdDCzktuozoqoHwIYCniAlZZvxL+j1gKBTGl6DXe70wedtkOMtUfWNqT2Haf+hKzhXEfReHUWtHYV/KPuxL2YdJf05Ct2Xd0HZpW/5eQje+AfZ2AzZHgyk8o1+4Xq8rZLTV2pOeFw34aLLWVtCKQjGkHwDc2XMHl36/ZN9BReL6n9dx4J0Dsu6zSlOFRn6NMCZ+DPzdpNtPKhklXtzxIr4/8z0u54grofyo70f6/opbkrdg6B9D66Wpq1TE9IvBrDuzEDc0zq79xI+OR7d53fQe9EePAs88A3whLDyQhHf2v4OgBUFYem6p/rUhcUNw5fkreKatcP8vCqVCiRntZgAAlpxdon/dUk8/00a1pmjoH4q4rLfwZ48UfDfkO0xrPQ1nnz2L6W2m67fR6DTIq6h/lkyhVKDlpJYWewIwDGNmYWjJ7tRSH8MHhe9Pf4+bBTcR7BGMturXADw4e09GwdhM8IjBhd8u4OuIr3HvuHVrJZootbRwDQwkgbS7uw/y8uQNeQoLycJfoQDiGnph8dDFmNzscfRKuoqW1c/CS+WFuyV39aQfQBQoqhJSim0twI8PjIeL0gVlNWUW+ysIwVryxNndGe4B7ii9L3KSNEGtuhZL2y/Fhd9ssx8FjBc7YmpvnJy4ud0Wq5EVl1YAAJ5v/7w+KWqpXxWfyuqTfp8Q23NdGcrcrsLFTYPEHxOx+MxisGAxocUEbJ+yHYsGL8Kb3d7EsLhh0k9UAPRe5yMJQhND0XlOZ4eQfkEtgtCofyO4+Ytfldpr7alQKJCQkKCPk3tG9UTXyK6o0dZg562dvJ+5eO8Wboa/DwD4euDX6NU+GEoloLvbDb4ufihQF5hVaF4rJt5KjVytK13kQJgXycxllYsj/fz8AL/sGwhIuyDKucIaasprUHi7UFZHEaVCiQD3ACMC3ZB0GNtsLOID45FVnoUx68ZgwoYJaP1ja/t7Y5hAr/Tzs09ZKgb2xroqFfc88/X16/luT7yY/CKcPbj1YNzQOLyW9xpaPGK/LVVKUQreP/g+ACDKh1/l+kKHF/Dt4G/RNov06RFbbGT67BrC25tUtN8vvY/tN7cLFrQcTDuIJzc/ibVX1woex8uLW3NScsMmZO0FzswAisXLx7xUXvpiBqH4V+WpQvPxzRHQxP6sEyX9+IrVAPLMRUcDXvmpOLRc2ALZEBezLwIA2oS1sbotTZA6O3OJTDHJX42Gi0HkJP1u35afGAKAD/t8iN2P7sbwJsNFba9QKhDePhyeIcael0qVEo/uehTNxjbD7jm7sX3mdpQY2L6bqjqCWwQjpq+5VKR1azJWpKUBL7zAkXn0fqA9/wDgr+t/YfpNb1yIeQznzxMylir9rPXzA4CgIAW8vLxRUGAejNFrLKb3mT2gcWttrXCS3BZ8N+Q7nHzqJHpGWW8dIAY0du3coCvWjV+nd8jhA40j1Wrhe5ZaFL/zDomHa2rMLV0BYN06UjRRWUnsXLsY1F+6u5NnLLCsDwDgt4u/mX1+csJk/N60Eompv8tOMrQI5rf3FCoKmLpzKkYtGyXvSQgg40wGUvenms3bUgiX7PJslLhcBVgG8a69LW6bmwt89x3w7beWiyIuXiTkbng4V0wBcPNatgWOmmXJfcAwQLt28bzzrS2w1tdPyLa1LKsMhz86LJsTgVRYu5aZZZm4knsFDBiEV/UHy9pf4F1VxRGzfKSfpyc31guR/rSAwLQQRA6MjR+LZ9o+gyAPYyKKijtGqEoBhTMQxLmH0RxygdpY6Wfa088z1BOv57+Obq93k3ROSeuTRDuvuTi5YHbn2fh9zO+SncXkJv0A6OMCvmLec1lkcra45nVvAPTaAjR+WrbnVSz+Jf0eMgQ0CUCTYfI01r2YfRFRC6Mwau0o/SRnSvrFB8YjyD0IeZV56P97f/T/vT/WJa0DCxZ9ovvwWzJFjAEixwIxjwMqf/0i8AbJ79hF+jEMadrt50ea2VpCdnk2Jm+cjN7Le/O+L8XeEwCOfHwEu2bvEn+ydmDI4iF45uwzcA+Qr8xqSsIU3Jl1BwsGLLDp80qFEl0iSfRoza6T3k8Mw2DvY3tx/YXr8FJ54WbBTV7J84OGXErO2AGx6PZONzAMg/R0YNEiEnRdvcpvzyQFWp0Wf934C5W1lXqChkKlVInuNfRU26fgrHDG6YzT+l4dNDlOJ+68ijy8tuc1FFcVW90fXVCWFrjhxY4vYsXoFfo+ggCw49YORC+MxvPbnxd1fo5AVXEVqsvM+yPd3H4Td/beMQvyLZFFD5vSL68iT29J93Hfj1FVQgY2W5V+luw9+RKPfChKKcL5X8+b9UaQA06uTvAK84JbgPUVmBjSj2FI9atOp7PY58IWZGQA9wJWoDpqG6p1ZCUfFASoNIFodfcn5Lyag3Xj16FLBLcq1+m4AN9ala2z0hkJIQlwUbrYRPpZs0lq9VgrPH/leYS3s1zhb4is81n4c9KfuHv4LgqSC1B6vxTVJeJ6k/HBlgpHW/sLZJZl4nzWeTBgML75eP3rdD9iST8PlQc2TtiIl12T4FfRBV4eTmgT1gaN/Bph72N7sW78OgyNG4pZnWbh0/6fyuok4AjrKzHo8HwHTNk2BQon8cuG5Hxid9Y0wHrvaCHUGGT9GIbBosGLcPX5q3iq7VNm2+pYHX7NfQY6RRXa+gzEtNbT4OZGbOMVcEKiJ7GzpP13ALLwTakkpHWcRz2Rfp6E9BOrOHByAnL7TsCVvi/JYvF59/BdfBf3naAttljsvLUT7x98HznlnE2pkKLH3dkdq8euhrPCGUfSj2DDtQ24nHMZT215Sn+fyAE6TjpS6Xcu8xwaLWqENj9ZJ0qswVJfv4C4AAQ0CYBCafzMuQe6w8Xb/uzQq3teRbW2Gv1i+uHptk/zbtMsqBle7PgSKgvJiUpR+tVYydiPXz8ew9cMx/TN03l7Qx9MO4jfLv6mr8wXgix9/RoMBzqvAJrOEf0RhmGwYvQKbJu8DYHulvttylEMScd8SxaZrVsDEdf24NKH4hyD2oe3x7zu8zCxxUSr29L4ydBeVAzpRwtvGcZ6TzkxCAwkyXGW5SwsHzRYlkX6sXS9e8/VdVdRmc9V5gxYMABPHn1Sn/1VKKB3SQKA/OR8aKr4JSehocCCBaQQurgY+M9/iKLdtJ8fAJzJPAMdq0Oguz80GmDvXs6CXozSz9+fxMqWlH6O7OcHGBeAyakk6hzRGZ0iOsHbRZ6iKGrTKiZ2dXXlyEy+71ar5V6PjeWulWkhw61bwKpV5PehQ4GPPzZ3yAgPB6JziXXexusbcbf4rtnxqtRKqLR+stu0UnvP9JJ0lFaX6u8VtZpTLT4oHP7wMFYPWW32upQ1CLUn96lsA0WV5TGf9uljWSA9nX8blgU21QkGaV9GCqrazLJQG1ZZyeWenJ3lY8hbBLVAi6AWcHPm/1KExv2a8hoceOcAbu10TJPdo58dxbmlwlaS1q5llaYKj7d+HMOaDENkALl+OTn824rF3bvkGvj68q8hGca60pfmBOQm4QHgP33/g6UjlpoV99OYy0ehA7ziAI+G+vcC3IyVfjTuk2P9wbIs/pz0J44vOG7/zqzAEaTfwNiBpMVYXpLR2KpjdXrXxG6R0kjQ+sK/pN8Dhs6gKY1Oq0NNuXyD9rILy8CChYvSRZ90olVDVEXnrHTGslHL0LFBRzQLbIbG/o3xSPNHcGnGJawdv5Y/OArtC/TYCHRZDvi20A8GlFA0rDqzBU8+CaxcyQ2SQvBUeWLd1XU4dPcQb5WlFHtPAOj7SV9M2jypXpRqKg+VpESrFLg42Z4M6BVF7Ek2XTe3LVDXqvHegffQa3kv9F7R2yhhFR8Yj04+owEAf1z5w+bjOxIZZzKw57U9qMizPVuq0+mQnJyMsjIdPv6Ym6jVavMKGKk4lXEKeZV58HP101t42IJgj2A80oLYWf5whlRnmya15+2bhy9PfImJf1pf6FtTvkV6RyKjLAN/3/hb0HLNkbi18xY+9/sc1zZcM3tv37x92Pr0VrOku6VecJT0q6jgV9PUN947+B5KqkuQGJqI6YnTLVoRWgK1eFAySrOGzvS+EHsPJ29JxtantyLznPzXO2FyAp458wwCm1qZACCupx8A+PuzqKgoR0GBvL040+/pcD3iVewOGIEL2YQ4oPdPaSmg0LlhQosJmNpqqtk5A+IC/C2TtqBsXhnppysR1pR+tpBRhbcLkbQuCW7+bghNDMXc9Llo91w7yfuhoN+HlF4GYu/X48eBZ58Fzte1RKTqsA4NOuit2QDjnn6moIuFigpjG+c2YW3gpCarcnd3YNHgRUiamaRXvpuCZVkzW0lbQO91oWu6efpm/NDqB7uPIwduFNin9KPzrWGc3D68vb6a3BS3Cm4hi70IpdYd/+n0o/7+pgVkocXElmXrTY70Y8BgdsBuNE9fiEY+8hTcWQMl/cTaewJAQLQ3WKWz3QkKAPCJ8kG3N7ohtI0dFXoAPj36KT449AG+O/2d/jVLqtm2YW2xcsxKTGo5CZ/1+wy9onohMTTRyPq1WlONtOI0HE0/ih/O/IBntz6LDj93wHenvjPfIQ96RvXE6PjRaB3iOKtWLxcvpBan4nbhbbvXC9YKbtSFaiMldtrBNBSnCWwsAftS9uGvG39BySixaPAii3NBaSkpVmGhg7unOC8svmfXEJW1lfoq8pWXVmLc+nFGfagB6B1HrF1LSvrZ1Nev6BJQfAUACzSaBji5ARKstCe1nIRhTYbBQyWcPf+x9Y/4rbu54kYqxBQ5tW4NZMT3Q26iuHihfXh7fNLvEzyR+ITVbSl51KAB95oU0s/bWz6FWMe61j6nTsmzPz5Ua8QXM93cehO/df8NF369gPwb+dg4aSP+mcf5+kV0jkB4+3Cj3k2G38Ufw/7AT21/Etx/o0bAwoWkgEWrBU6c4Ff60d5HvZsQn8D1dV0egoLE9VP089PVxcrmhaz0Gjuynx9AvhdHWXzKCbUayPHZgf2Zf+mT45ZA1wZ8if/CQvJ9OzmROYGSfqaFDFS1mZgIPP+8MXFMERYGeKtboaV7X+hYHRafXqx/j85XYsYSW+Dn5ofdj+5G6uxUeKo8je4VvmuZdSELB98/iOK7xfKeCA86ze6EYT8ME8wHiFmP7U3ZCwAILB1gtUiWrjsAYdLv4EEgOZmMB6NHG79HST9LSj/6nTo7A6mpwvOtVHzQ5wNcnXkVz7Z7lvd9oXHfN9oXz5x9Bj3m9zD/kAw4+c1JXP5dWI3Pdy133d6FmEUx6L+yP9S1aiwfvRxbJ2/Vj5v2xtSpdXXUfCo/CmvF5I7q6WcJTgonhHiE4D1lD2BYktF7pko/rter+X6yLmTh+JfHRfMXrI7FhI0T0HluZ6vb5lbkYsGxBaL765rCsKefXPB388fklpMxq+Mso7HkRv4NFFUVwc3JzaJ7Gu6uAy68DtQUy/a8isW/pN9DhMJbhfjU61Ozps+2oEpThVWXSTnQk22eBEACCkrMGVpndo7ojFNPn8K1F67h1ku3sP6R9WgV0kr0sejingaI9pJ+YuGp8tQrEfnsKKUq/Rp2a4ioHlEO7+9XVVyFnCs50NbIV/aUV5Enug+fJTza6lEoGAUO3T2E63nGjefn7JqDDw9/iMN3D+Pw3cMYv3683pYnPR0oPTYZALA+ab0s5yI30o+m48SXJ5B3zTYJ162dt7Ck2RLknsjF+vUMMjMJMU3JaT4bDimg6sre0b0lS9hNMbP9TADA75d/x/W8G0Z2lnvv7MWvF0h/lvd6vWd1X9ZIv4SQBHRv2B1aVotfzv9i13nbgpBWIWg5uSV8osxZsOE/DcfQ74eavW4pyHdz4wLaB632SylKwc/nfwYAfDPoGygYpUWCwhL8XP2gfkuN+y/fN+uvJcXeEwCajmyKiX9NRFibMGknITPELlzpd1VcLO/YfjLtImqc8+HKeKFTg076c6H3Fd/inj6Lbm7EsskawrzCbB4PrPX0A4CcKznY+uxW5FwRXvlknMnQ9/1r/khzvHT7JQS1IKsYpUoJZzfbxytbKhzF3K8pKcBXX5Eq2X37yGvOSmfEB8ab2W1aIioMr6dplSO17PHyIn2Thfroztw+E/5f+PMW00iFNesrZw9nuPm7QVsrb1n1jhd34OxPZyV95tvB32LDIxsEiVB7kVuRa0S6xPo2Re+ka2ibsg4d4zi1fGIi+Vl9bTCcFE64nn9d36tEqVAitKY7GuXOhrdX/SyJpNp7AkBoCAu3kmzcOZxh9/GDWwSj/2f97Rq/L2RdwJH0I3BSOGFmBxJvsKz13l2TWk7CmnFr8Eb3N/DXxL9wePphxPjFIKUoBS2WtIDrx66IWRSDHr/1wMwdM/Hz+Z9xNvMsTmWIy+6/3OVl/DXxL3Rr6LhK2whvYpVSUVshyi3BEri5yfw9TbUGC4IWYNcc4kCi0+qweshqbJuxza5janVazN41GwCx7xQi0Slyc4Fc7104lpCIXy8KExNS4O7sjlsv3cLK0Svh6uSKrTe3Ytrf04ye50s5RMZlbT1KSb/kZBscN668D+xsA2jrgsIbi4C/I4CaYok7EkZYuzCEJNq/MBYT77RsCZSFNEaae3NZrIANQRUnhhZ0Ykg/WwvVLIGqYk6fll9BdLvwNgK+CED41+ILcxsPboyur3VF01FN4dXAC6NXjEb7GebK8dR/7sAvM8kouavT6tD2mbZWi6dUKqB3b/L7qVPmSj+NTqNvyTKpZ3swDEcKiLH2BLjkbk2NOUlTX0o/wDGk382Cm1h4cqE+N2YPdDryHV1tOBPTd4/F7ULr3sJ8rhEUdK0ZEEBIz7i6TiSmSj+aw4vid2MGwJFFXZi5AICfz/+M8hrygP5x5Q80+a4JthcTRyhH9GYcGDsQ0b7RUDAKKJXcMfgsLrMvZuPQB4eQc1mGaiYraNS/Edo8aa7Op+OqtTUIy7J6pV9Q6QCL6w+NxliFfNdcbImqKmD5cvL7hAnmxAR9ri0p/Tgivn7b6hiO+4ZzrtJZifB24WYtVeTCM2efweiVowXfN83t/HbhNwz/YzjSitNwIO2AkSJfLtIvLY38jI4W3obmCR8U6VdRU2HmFjQkbgiyX83G9kfWAJtCgDMz9e+Z9vSzNHbd2n4Le1/bi/wbFhqWGkChVKDZmGZo2L2h1W3XJ63H6/+8jrHrxoratykcofQDgFVjV2HRkEVo6MP9DcfvEeVixwYdLedsMrYD1xcAjOPa5AjhX9LvIYLCSYFWj7ZCWFv7E6mbb2xGUVURIrwjMKDRAADk5q+qApRK+yw4cfFNYGc74PijwMX5ZnYvdu1bImgFKF0cGkJqTz+ABN9CFhty4c7eO/ix1Y+4us4+ayVDzNwxEwFfBFjsfSEGEd4Rer9iw75yO2/txNLz5P8LBy3E5RmXcfCJg1AwZAgpLAQCy/rDoyoOw+NGorTath5RjkTClATMvDYTkV1saxapqdKAUTJw8nTSV9899hi3mLKX9Dt27xgAeWThXSO7YmjcUNTqavH2P+/rF8WsSxGmbya9+F7o8AK6Rna1ui8aGFnqcfd8e2LtufTc0nonfL0beGPcH+PQqJ95mVVkl0g0GW6u3rBW2UeJTkeSfh8f/hiPbnoUZzLOCG6TWpSKQPdADIwdiN7RvVFRwSU4xFTtGoJhGEFSwpBEEZM082vkh/jR8ZL6e4nFvvn7kLQhyfqG4BIC1hauUklNsTiRuwcA0Ma3jz7IYxjLVX2WCCa5ISYhV5lfifM/n8f9E/yVdPk38rGs6zJsn0H86xmGgX+sv2zFMbbYe1pTx5SVEdsjSorRBfe01tNw/YXrmN9jvtH21q6J0ILHUMFgCRqdBsVVxXo1qD0w7KfDp0Qeungonjj4BJTO4uygxUCn1eHMkjNI2SvNYrZpYFOMbz4eUb4WMlQ24rmtzyHi6wgcST+ify0vD1BVhyNSPdxokRcXR5KItWU+6BIyAANjB6KshstA0eejPpKZAJl7M1/OxA/DxCsyw0JZxB9fhvTf9jnwzMRj0alFAIBHmj+CcC+SHDecn8Qk+P3c/OCkIAvfcK9wfeLURemCaN9oDG48GPO6z8P68evxUd+PAADlNeVIyhU3PzgK7s7uevuje6X2BX6W7D2dXJzQ+eXOiBtGMsCslsWQxUPQ/nn7bGhP3D+BpLwk+Lj44P3e71vctqoKWLwYqHRJQbHLFXx69FNUaeSz9X6s9WPYNXUXnBRO+PPan/qEfFk118fWGunXqBGxSywutsF+Ku55oM0CwKmOZXD2BFyCgAoBaYYJruRcwarLqyzGcaOWjcKw7+3v6yrG2cDNjUseW1KIAKRYdM+dPcitENfAmtp7SiX96DwpJ+nXrBmZd8vLgSSZh4NA90AUqgtRqC7UkyXWoFQpMeCLAQhsGggXLxe0ntbazM1HU63BiflbEZm0C65Kzs5WoVSg+xvd0Xm2deUDVThev04KmwBujXYt7xrUGjW8XbzRMbYJWhk8NmKsPQHaZ5QU85o+S/U5TzqC9Luccxlzd8/Fj2d/tHtfNEmvY8h1dFZYL3yzlPinr9G1A71eqamEQKKgz6Ch2tYU9Pn0yx+KOP84lFSX4LcLRGm8N2UvbhXeQkktSdA7gvQzhaUxoumIpphxaQbv+r2+INYx5kruFWSWZULFuMGvvJvFoopr14ydQfiUfn/+SXJnISHAKJ62hoZKP6F1OX0+HKW+ZVmW182AXlPaU9AQtepa5CeLI4CkwifSB34xwh7jhuvJJWeW4MktT0LLavFoq0excvRKhHhyxTdieiaKAc39NbTAYVnLKdnieCMWV3OvwvNTT3T6pRP/BvkngapcwJ37A2J8Y/Buz3fxZrc3AXBr4JIS4/EIABKmJuCJw08gsJl1dyapWH2F2PFOSZhi0+cdRfrxIcQjBINiB2FQ7CDLG3ZaCoy4TWLNesa/pN9DBP/G/hjz+xg0HWF7/xOKZReXAQCeaP2EvhcYHZjCwvgtAURDUwnUFAJ5x4CCk2aJsvpS+gGWST+p9p4ZZzLwsevHOP39ablOjxeB8YHoPr87IrvaRj6ZQqvTYl/KPpRWlyLKx/7k2ox2MwAAyy8th7pWjUJ1IZ7aQvrnzOk0B7M7z0ZCSII+WQOQSV/BOqP31WR80vkXvTT8YYJniCeCmgVBqbItGdpsTDM8f/V5BCUG6SuvIiM51ex9/py5KOhYHY6lE9Kve8PuVra2DoZhsHjIYszuNBvvtSOV2V7eLGbveQEZZRloEtAEXwz4QtS+6CLEUuA5rtk4BLkHIaMsA9tu2leFLhfUhWrotALWUlYq++jfbInotAcsy+KXC79g9ZXVuFvCUwJYh36N+iFlVgp+HUmUmXSR4e4uTiUmFnQMr6kxtp+0hqoSeXv6VZVU4einR5G8WVyPJ/FKPxYAI7pnoVhcqyGkX79oYystS6SfVKWmVqfFlI1T0OHnDpJVJWISJw27NcTMazPR7ln+KvPA+ED0eKsHer3fS9KxxcIRPf0WLybPrqH1uOFCxXDuAmwn/QyVfpbQJpRUF8tB+jEM913Vl/UVo2DwZsmbGP7j8Po5oAGUSuH5ulZXizm75qCkqgRnM8/y9pwCCCFA1X4zvLZh96O79fYrXx7/EifVv6NWWSxLvykxCHAPQJhXmOg+vQDQIFKBe80HQd3WeqGONRz74hhWD13N2wdXDIrURVhzdQ0AYE7nOfrX6XPk4WHeZ8gaXJ1csfexvch6JQvqt9RInZ2KnVN34pN+n+CRFo8g2jcax9KPoeE3DTF+A+cyYYjk/GSbbYCkItKHBH73Suwj/awVMAxcMBBtppPxQ6lSou1TbRE/yja7XIpCdSGifKIwMHYg/NyEE2gsC3z5JSEXWtY+hTCPBsgoy8Cv538VdRyhZzerLMuoPUCv6F74oDfpW/zizheRVZaFq7mkKDLcK9xqvzwXF075YqqMsYqwgUD8XO7/jaYDQy8BfuLcblZcWoHH/noM65PWSzywdIhRpOQm5SJ63RcITj1lVcGwL3UfBq0ahF4/jMJRy23cAfATDmJIP2vqX1ugVHIE2MmT8u0XAHxdffX33M0CaZ6xFXkVuLWDv5+Vk4sT2n81CTe7PA4XT26AlGIRHBRESG6W5XIbNHlNe7i3C2sHBaNAnz7c58Qq/QDAz49Ubpg6VTiaYDCEI3oXh3qSL0psL11LoIQOS0k/EW4clvp6mZJ+oaF1hUq1xoQRzTNYIv30ZFGWArM7zUaAWwD83fzBsqzenjKimogBHEH6nc86j/cOvIeVl1YCsDxGuAe6I6RVCJzd7XM3sgaWZfFNw2+w46UdZu+JXYPsvk16yyb69oGSdbW4njxX13aOzkt8pN8uIuDH9On86/mgIBK71tQIr3UMC18txcpSwbIsOv7cEb6f+/I6UqhU3DmbXtftM7bj+/jvUV1qe693PmhrtChKLUJNhbCNJL2WClUV3jnwDgDg9a6vY+XolUZtNgD5lH70mYyIEN7Gmr0n7Q/qCNIv0pvEqvmV+aio4RlQz80CfFoALd7UvxTiGYIP+nyA59o/B4AU2Dg5kXnH9F70i/FDVI8oqDzEJaVS96fic7/PcXmVsE0rANwpvIOT909CwShE9Rw2Bcs6xt6TokZbg/2p+3Ejn7SxGNF0BHY9ugvzesyz/EGlK+AlYUKWEf+Sfg8Ycg7SFHeL72LvHTKxT28zXf+6aT8/m9H+W2BUKjA8Gei33yhR5uXlmEakQqAVoJbsPcWSfj4NfRA3NA6+0b4ynR0/QhJC0O/jfvCPlWcUOp91HkVVRfBx8UGHBh3s3t/A2IGI8olCs8BmyCzLxNNbnkZWeRbiA+PxSb9PeD9DF6MMGIcRJXJAXai2qx+KUqlEXFwCiorI0BkWJg/pl5yfjAJ1AVydXNEmzNx+QgiXLglXNsf4xWDh4IXQVfpABw3+bhyCNVfXQMkosXL0Srg7i4v2w8NJcFdWBsHEgIuTCx5v/TgAYF3SOtHnLxdS96diea/luHecS8BtnLwRC4IWmCl3WZZbsAkteByt9DudcRppxWnwVHmiW2Q3fHn8S6OKYpZl9QSPm7Ob3k7MXpukHbd2oM+KPnhl9ytGr7u6cuO2WDXcX9P+wpfBX8pqI+jq44q59+ei9we9RW0vtkIzIEAJb29vlJTYH/IcPw68/z5w4GgFclWEqB+TOMBoG0uksVSln1KhxP7U/TibeVaUhZAhaHLGsNm72f5VSgQ1C7K4n97v90Zoa8dI+G2xNaGJctoHxXCOZ1ngbJ0L5bx55DvQaoHDSTcFe/RYuyZ8yRqtllvsWlP6UYLpYvZFyxuKhCUyt1Zdi6OfH5XVSYBhGLh4ucA9UHyGaM+dPfjs6Gd2/c1KpRIJCQm8cfK8HvMQ6B6IC9kX0P7n9ujwcwfMODIcLFh90ssQ8XU8yb10bgyo1dbi7f1vY4fLNNQ45deb0s8WhIUB+VHtkOEqUrJhAYV3CpF2MM1mW979qftRo61Bs8Bm6Nigo/51e+ennlE9EeoZKqgibhXSCjpWhxv5N7Djlnny7s19byLym0gsObPEthOQAGrr40ilnyHk7DU+sulIpM5OxfLRyy1ut20bsRF0dgbee8sFb/ciCulPjn5iVe1n6dn98viXaPB1A3x8+GP9a693ex0DGg3A5/0/R6hnqL6QU2xvRqpukRSz8blSMNJiBNobNrdSeOFzc/tN7Jq7y+4kqKgiJxZwahCMWpWH1WQmHZvVqYn46it+u2gKjYZTRPAp/fis+ygo6SfVncIaOtcJ406dssHW1QqaBzUHQNRzUrBu9Dr8MewP3D/FvyB0aRiKKs9AuLoCJfdKUJlfie+bfY9dc3eJPkZHbsiFkxNXlETVpu3DiRK4SxeuQJDaRVqDUqlEkyaBYBjGrH9TfSr9xJDJUuEQ0k9BSD/TQjI+0DiyoID0x/zhB26NYEr6MQyn9qOFDBoNR1BYIhho/FNYCExt/hRuz7qNqa2m4nr+dWSWZcLVyRX+5aS42BGk34WsC/jw8Id65ylr17K6tBqlGY51h9JWa+ET6QMXbxez98SSfrM7z8b+afvxbDyZBy2RfrSf36hR5FqWlBi3Giov5/7fTsDV18mJux+ELD7pGsDbWyE439oChmFQoC5AaXWp3grfFEJjf5ORTdB9XnfBomtbUXCzAN82+hbHFxwX3IZey0N5f6JQXYiGPg3xSb9PeGNKWixhD+lXWckRS5aeSUuEP2Bb8atYeLt46/N9hgTu5hub0eO3HlilaA0kfmZxHwzDEWd8OUedVgd1obiKcSdXJ4S1DYNHsOXqkT+u/AEA6BvTV98SQQqys7liX0eQfrN2zkK/lf2kKcdZFig4A1STL9ERHJAl/Ev6PWAYLuQOf3wY257fZvfibsO1DWDBond0bzTy4yTzspF+FHV9oQztPevT2hMAWoeSBeG1vGuo1dYavSdV6ecZ4olJmyeh+bjmcp6iw0E9xvvG9BUVeFqDUqHE+efO4/hTxxHrH4upCVPh4+KDVWNWwc2Zf0YyTD7m5ZGk0A9nxFtX1RcWxy/GpqnS+yppqjXY8+oepOxPwa1bZQBYeHmRoIdO9PbYe/q6+uKzfp9hbue5Zv3WhHDxIvD226QS2xKKigAFnODNNICrkysWDl6IThECMn8euLoC48eT33/5hd9SDgDGNBsDgDRNNn0WHQ1XX1dknstE4R1ulRrdNxqtH28NJ1fjZ8LQcuNBKf3oYmhU01EYvmY4Xtv7Gn4+9zMO3z2MubvmotG3jTBhwwSzz9naz4+isrYSB9MO4ug9c/ZWqgVmRJcIJExJQE2ZuObNYuHdwFt0QYQUpZ9GU4uiIvszQ+vXkyrON348BJ2iBh61UWgTaZxRsVTVR79fKdcwLoDs/1YBfwW5ECg5xGfBYgiWZXFr5y1c+p1TzLM6Fnte24Os8+L7jtkCWxY79LvLzAReew149FGuf0Z5OZe8jI3lLFce3z0cAV8E4MS9E0b72rGDxAgMI2wBwqf0M0xgWEuCtQppBQYMssuzZUk4GZK5plA6K3HwvYO4tMLc/cBWqAvVyLqQJUkZtvbqWszbNw8bkjbYfFyWZVFaWsobE0f7RmPn1J3wVHnqyXB/TTMwYIyS0hSmhRzpJel4/O/HUa2thrPWB+7VsfVG+lXUVGDurrmYsnEKtDpxRRM0kZefD1SW2WehPeKnEZhfMR8KJ9uWgPtSicUobR1A4QhFjyG8XLzwXDtSefzFsS+M7gsdq8Phu4cBEKWLo0Grpx2t9Lt34h6W91qOLU9vweYnN+OHhB+gqbbfQp1hGKuFXzdIATPGjyek+VNtnkKEdwQyyzKx5soai58VenY1Og3WJq2FjtUhISRB/7qTwgm7H92NGe1ngGEYxPrFYmjcUNGkn7XvkRcX3wQ2RwNqkzmu8Bxw8kmSnLGCIHcy2VuyyEw/ko5TC0+hItc+2ZKYIqfglsGI+eAJFDVoaTV+paSfd2UbaDTESlAIOTkkjnB1NV7zU3V0fSv9AKLeVqlInG7p3G1B80DbSL+xq8di8KLBaNCBX4pFFR0umgr80ukXrOizAm7+bpLsuA1Jv5AQTtXeNbIrnkh8Qm8v5u4OfPEF8Nln4r97lmXh4aEGwP7P2XtS0q+itkK0basQzJR+Euw9798H3nuPxJ5//01eMyX9AI6ope1EsrNJL0HTZ9AUnp7cc1mc7wpfV18A0IsBejTsgeoKUmXnCNKPquDTS9L15wMIjxHfxn6LjZM2yn8iBnBydcKTx55Ev4/7mb0ntnhUpVShT0wf9IwhLViE5pr8fNLnjWFIYQLNixr29aOqaX9/ywWPhhaffKDfqbu7cKxsK2j+2LQXHIXQdW0+rjn6fdIPbn7yMlguPi7o8moXi73g6LW8XESIwafbPC3oqEGVfiUlxnkhQ5w5A3z/PfD668CsWYSsNwQt9vfzs6yANswJ8F0iR/b0YxgGYZ51fcTLuFjnful9HE0/ir/LtUBVDnD1Y6PP3Sq4heP3jqOsmrC6lvr6fdvoW6weulrU+UR2jcS0fdMQO9Cy2m3LzS0AgCktbbP2XF13Oq1b2+luKIDBjQcDID1TtyZvFbe2ry4AdncELpHCATmfVzH4l/R7wNDpuEqI2ztv49a2W3b3ypmaMBUrR6/E/O7GfWvESJBFIWMHkLqK9Dy4twl+HlyivT6tPQEgyicKAW4BaBLQBDkVxuUatKqwosLcg/hBoaaiBt82/hbHFhyTbZ/H75PJrXd0b9n2aWjPObbZWKTMTkG7cOFEiqEl4PG7Z9BvZT/M2jVLNmWDXOg0qxNaTGoh+XP5N/Jx4qsTSN2XinPnssCyXDBGn6eiItsXKGFeYXij+xuCSko+nKjLXZt6x5uCViE957kNpW+W4sWOL0o+v3HjyN9bWAj88Qf/Np0adMK7Pd/F9inbJdmWyYHQNqF4NedVtH6MSxB1f6M7Bn8z2Gxbvf2DQtiCzJoVgz3Q6rR6NeSklpPwTNtnAAAv73kZvZb3wsJTC5FWnIZTGaegrjWunLI3edIiiNz71/KumQUbUpNmHZ7vgFG/jZK1r9/9U/clJcbEkn7e3jpUVlbaTfppNNzCrdiTjLtxGGo2Z4vp6SflGsb515F+hdJIP0MLFmtj0+65u7H31b366sx7x+/hxJcncH3TdUnHlAp7evoVFgLJyWQRdb3uNOmCxMuL/O0xMUCm3wbcV99CjbYGLYNb6vdz9CjwY12R3qRJwjadfIsdWt3q4UGsxizBQ+WBpoHEtv1Clv0Wn5YSYgonBZ468RTGrrat8Tkf7uy5g6Vtl+LObv6KXz6cuE8mqC6RXWw+rk6nQ0pKilGcbIj24e2xZdIWqJQquDm5oXH+HAD8hW2GVj612lp0X9Zdb1HpW9keDJh6I/1UShUWnVqENVfXoEAtrgmZtzfg4VyDVv98g7+etN9C2551hovSBX6ufujXyDiBZm9RihjM6jQLKqUKR9KP6K3KANK3pFBdCA9nD7QNa+u4E6hDy+CW6NSgkz6RbCusKf1CE0Ph7OGM4JbBcPFxgYuPC5xcbM9glNeUiyaa6fxK5zMXJxe82IHEj9+d/s5iwoLv2a3SVOGRDY8gsywT/m7+Zr1PDO/JAbEDUKOtMbPkEoJYxaQRXIMA11DA1WThWpULpPwGFFkvnNAr/SyQfl1e6YI56XPscpFhWfFzpdheRXrST50IALhjYXg37OdnOHQ8qJ5+ALF1bVv3qJ86Je++WwSTWDkpT1rDQN9oX3Sa1QmMgn981Sd3AzzQelpr9HynJ546/hT6f95f9DEaN+aUC4aF1o+1fgy/jfrNaFyOihKv8gPIc6vR5BBBgsnUROON/1bSz1PlCQ9nsuOccvs8/eizqLPB3jM3l1MX0WeOj/QzVfpRwqFBA+NnkA+08Ik+tyzLYs7uOQBIsY5YossWmKrgrRUGtH++PeLH2mdZbQ/E2CYbgq4/Skr4CRxagNikCfnbaeGhocUnX39UPtDnm25vCvqdeniwFmNlW9DIl5B+d4r4JwYxBR9ywifSBwMXDESj/vz9Hw3nyG/6L8HF5y7q7Sn54OHBjTN8ar+KCuCjj4gN6/XrpLDENP9F8+rWxDT02a+q4v++HEn6AdAr5QyVfrU6g7ErbTVw4yujzwxZPQTdlnXTO+lZIv1aTGqB2EHyWVbWamv1x+0VLb21yK1bwKFDZJycPt369rZgeJPhGBQ7CJW1lRi5diTCvgrDG3vfsPwhhRJo/QkQMRoAZH1exeBf0u8hwvQj0zHj8gy79xPmFYbHWj+GAbHGVbhUiWQ36XfjS+DcbOD+FuDIOPgzXBKrvpV+DMMg+9VsXHn+it4Cj8LTkwuMxKr9Liy7gI2TNzqMfa/IqYCTqxNYnTz7Z1kWpzNID8JODcSrt6SAYRirPfoMA3PPkvYYEz8GGp0Gj/31mKCl2oNAz7d7otNL0r+n4JbBeDH5RbR9ti3y80lwT4M1d3duMrTH4lMqLtQ9djodcNNC2wmaBIkOaCBqYcIHlQp4ri522rqVX9WoVCjxQZ8P0DWyKxQSLZLsBcMwRn7iOo3wRGoY4AstnBxJ+h1NP4qs8iz4uvpiYOxAPN76cYR7kZvJz9UP01pPw18T/0LWK1lmylp7k6qN/RvDWeGM8ppyM1syqUo/ucGyLP4Y+gfWjLCsIKCorSX/AOsLV/q3qdWMvtLaFqSnE+LP0xPY99ZHeMf/Gj4d+YrZdoZ9ME1Br6GlSl1T2Er6AeKSJwzDYMTPI/DUiaegUJJnt2H3hnjuwnPoMNN+y2hLsKWBeUAAIe0B7trTZ9W0cbcm8AIuRhPr4Tmd58DLhaxUU1KAr74iC8UhQ4DJky0fz3DfABdTiLUsk7OvH/2bha5pWJswWatsg1sGo89/+iCktbiqrkJ1ob7PQeeIzrKdBx/6xPTBpRmXcGL6OWQmE3VFcx6zBqr0Ky4GWK0z3u31rv4973Jyj9dXTz9npbO+Z5Rh5a0lMAwQ2lAFtVcwWB9fu45/Zc0VZJzJsL6hAL4Z/A3yXsvD0LihRq/ba+8pBg28G+CFDi8AAN745w19b7+DaQcBAN0adrM5zpGCGe1n4OTTJ/FSp5fs2o9hsQ3fssPZzRmTt05Gl7ldMPibwXjy6JN2He8/h/6DwAWB+PbUt1a3NewXRPF026fh6uSKC9kXcCpDPNNSXlOO4X8Mx983/oaL0gUrRq+Ai5O51Zoh9j6216hIwxJsUvo1fwMYdNLc0jOkDzC+EIh9yuouxJB+HkEe8In0sVlZC5BYhxavWkpO3/j7Bgo2HYJCU23Rtiy7PJsUyrIKeKvJd2ypHyJd25gmqcX0XnOkArhl3e3B1zPLHthq72kNhsnd/p/1R4sJhFyUUoTBMECnumWspd5utsLHh7+nH0cwyH9MUziC9APks/ik11EH8Uo/QycJerlTU8m4T+NXGqcAHFmblkbcK8T086MwXcMmF3B90vs3GiC6YNIWUBV8aXUpSqpKrFoA9/mwD7rMtb04TAwKbhbgyKdHkHfNfFHPV0xxLvMcOv3SCe2Xtsfhu4fx0eGPMHvnbCTlJuljfkOLf0NQcpaStpT0M1T6UbtOPit6Q9DxVqiAw5F9NmP9CYkjRPoJFXxUl1bjj2F/4MinR+Q/KQuoqeFiKDc34gRH52chWCqQyckh+TUPD+Dpp8lrR44YE4RixTQqFVeowWfV6nDSj0fpRx25Pq3ZQxRoQ4zXpjTvW6gmygFLpN+Azwegzwd9zN/gQdL6JBx47wBq1cKOYHeK7kCj08DHxQcxvjGi9kvBssCvdW2n+/SR1s9WCpwUTlj/yHp9mzEAiA+0Uryg8gNazAPChzjmpKzgX9LvIQLDMLLLoSnKyrjA2257z8QvgO7rSQP0LivhGtxcL52tb6UfIOylzjDS+/plXcjCtT+voTzLMaUrfo38MPPqTHR/o7ss+7tbche5FblwVjjrrU4fBAxtxvLyGPw4/EcEuQfhau5VzNw+U3Rl8cMKhVKBgCYB8I7wRl4eCe4NgzV7LD4L1YVYc2WN3gpDDLKyjAOHaxbWpZTEkUIy8KFdO6BNGxIE0SbVDxtOfXsKa0auwYLgBTj3M/9JiqmWpgumggL5VcJUXTKu2TiiUHF2w/7JJ7F93FHkvJqDFaNXYHT8aF7rLXuTqs5KZzQJaAIASMo1rmC2hfQ7+MFB/PXYX7adjAl0Gh36ftIXnWaLI+UNxxxrFZpuboCTE1kNSEoKmoBW5TZqRObRD19qhsEdzSPKRo2I+is313wxIbWnH2C7vSfALQatVWNG9YiCXyM/ZJzJwKnvTqEirwKhiaHwCncsE0IXO1KUfu7uwKuvAi+/zC3GaHKDKpsDAsgC59O00dAp1QhXDzJSUu/bR57tdu2AGTMsV05bUvqJJf16RfXCwNiBiPaNFvcBC7CWENNpdMhPzhfdY8EaglsGo+fbPREQJ+B/aoJT9wkZ0Ni/sZ7cciTiA+PhVNQMGg15rviqpz09uQV1Xh7wZJsn0SuKVJAGlfaHQuG4BTcf+CpvrX4mDLjVaSq8RvS2+bjaGi02TdmEE1+dsL6xBSgVSrPYuz6UfgDwVo+34O3ijYvZF/VW2YfuHgIA9I7q7diDyww6l2s0ws8zLcSQA/vT9qO4qhh+rtYDQjrHGiYUA9wD8Fm/z7B18lZ0CBdfEDJ/33zsS90HD2cP7Ji6A8ObDJd66hYha9GS0pUkZ0QQMYakn1CxqKZag6KUIlQWWPDYtgJDNxVLc+X1TddxfclBAAzy8ki8zgeq8vOsago3JxJr8pF+Gg2wYQOwahX5P01gUzzInn4AN9aIXeOLRYugFugb0xfD4obJWgRMi87snWseewyYOJG4sABAQWUBbuTfQI3Wfrt9Hx+y6HkY7D3lVhHJSfqxYDFUswxLhy/VW2hagpsbl/ifMYM4zajVhPijY3+gQbgUFET+aTSkyJeSSWIK9w37BwIkRvp9zO/4ZtA3aO6fCG1dSsYRpJ+HykNPGKSXpDukP6NUZF3Iwv75+5GbZF6cwZcT8HLxwumM0ziXdQ4Dfx+Ir098jW9Pf4u04jSoVNz9ybeepLbKlMC1R+lHc6pCVs2c+lZ+oUKUTxQACOamhK6rylOF9KPpKE4tlvV8kjYk4Y/hf6DgJr87hloNsNCBBSt6fDV0ADEFLXoICyO9Gdu0IYQSteQFpIlp6LV+oKSfwXpDU9fTuFjhAQR2BjyijD4T4E7We9SNxBLpJwXXN13H4Q8PWyyCig+MR+mbpTj25DHJriRnzwJJSYRofewx+87VGrxdvLF9ynZEeEfAReli5n7ysOFf0u8hQWVBJe7svYOKPPvKmjYkbcCCYwtws8BY+kOrEQIDZRhUAtoDof0A7yZAzGNg3MN4rSbqG3yBuVTSr88HfTCvbJ7DE51ywUXpgvd7vY/n2z8PV6d6zFiZwDABn5tLFsK/jPwFDBgsu7gMUzdNlWUxYi/uHrmLFX1WIPWAtAYQuUm5qComs3JxMfmeDUk/SqTbQvodTDuIKZumYMSaEaI/Q0k3OhdSazs+yEX6AcSuAjD3NTfEzls7MWPbDEEfeEcidX8qbm69CfcAd3iG8K9MxZB+fn7EA5xlYdbM3l4khiaiSUATTG5JpEWXLwMfvBKJZe93Q0WZ5WpROSqmhWyLbLHHyrmYg5R/UqCttZ/UVzor0f659kiYkmB9Y3Bjjqsrp/oSAsPYaP9lApoUi2lk+e91d+fURqYEeX3aewLSK6bv7L6DXbN2oSzTQhZPRki11qHo0YNU8Zn2aqMLtXzvf9B2aVtkVabDo6oJEm6sRbWakBQsC5w8SbYbPNj6/UMTKEVFXBEAjSnEqsOea/8cdj+6G1MSbOtPYAhr1zRlXwq+j/8e55Y+mOqMk/fJl9slwv7qbVeRAWtS3XDWogV/rp5huHslNxdQMArsnLoTW4efRWBZP3h5icrxywaq7s4sE/Bt4vuMiWWXTWCAiX9NRMeXOlrflgd5FXmCSfD6UPoBJBnxZrc3wYDBtbxr0LE6HEojpJ8tVkD2QMfq7CIFrCUQ5USRugjnMsmYICY5IaQGmd15NoY3GW7Vxp0+uwWVBfjl/C8AgA2PbEDfmL4Sz9w6JM/v6RuBsy8BlQIPU2kykH/S6m6CPEiFWI22BmU1/HNmxqkMfBv7LS6vuizy5MwhNt4Z9PUgPHP+OTAuKmg0wvHrpWziQedd2RrDhpHX0tO5frgUH30ErFxJXm/VChhhskyhid/aWvPPUjhS6Sd1jS8WIZ4h2DdtHxYOXmh3yxVD0OSui2WRq1V4eZFexjT3siV5C5p930zSOlIIISEkTnqQpJ+jiKJvBn2DU0+fQv9G4u1U+VBVBTBg0MHpCTzT7hkzVxYhzJ8PzJtH3CWi6nLsp4lZEzw8jONg2hMOIO08DO09rYEvQf9oq0cxp/Mc/VjCMI4jGQwtPq1dy7RDafh94O+4d9y+/riW0HhwYzx77lk06mduDcmXE2js3xhrx63FyKYjUa2tRlFVEVyULvo2OnS+yc0F1q4lvd8oTEk/ep3T0zklmljSj649hFyHDNX4YmNlsdBfQ4G+xUIFH4yCwRvFb2DEUvvHIkMUpxYjZa9wvkGtBoo9TmNPGz+M3zBO1D7FkH70WaIFFnv2cPONWHtPgMsZPgjSr2dUTzzT9hkjRzhq7/mjx1Cg7TdAeRqg5ayQpCj97p+6jw0TNoh6hod+PxQzk2Za7WProfLQ56uk4OpV8rNvX+MiCkchwjsCV5+/iusvXNc/M4K4/hWwt7t5H+l6wr+k3wOGsq4ZzL3j97Bq4Crc3GrBp08Elp5fitf/eR177uwxep2SEXar/AQweTJJwFGrjfpEWnEaOvzcAY2+NZ/M6SJDbJLEzd8NTq4O6PhZh9OLT+PKmiuy7S/MKwzv9X4Pi4Yskm2ftsCQ9MvPJ9WlI5uOxLrx6+CscMa6pHX45AinsrA3UWIrdLU6ZF3IQnm2tJXEqoGrsLLfSiiVStTUBIJhGKNgjT5Xtth7Hksn/R27RXYT/Znz58nPHj3Iz+vXhSt6aSJJDtKPLjYs/Z0Lji/AT+d+wtbkrfYfUCKGLRmGt9Rv4aVbL6HJiCa824jpZcAwnNpPqMLOVsxoPwM3XriBvjF9sW0b8M47JGiuruYsW4Vgi0rMFM0D+W2LbKmUH/P7GLyc+bLV4E0MpI4HUnpSKJVKxMT4gmEYwcQqywI7dhAveCHcuQPooME7+Y0xYcME5FUI+7+2q2uBevas8ev0+5VyDakta4BbACpqpBUGibHfMkTraa3xxKEn4B9r2dJZLtii9DOEaYN0uiBJc9qN7PJstAhqgUH52+Gs9dXb69y9S55rlQpITLR+DB8frgiA3j9S7T3lBCUJKgWEIzF9YtBxVkfE9JVmiyKEnbN2YkXfFaKfUX0/PztJP6VSifj4eH2cbAmGpJ8QTKum3ZzdEO1CHtT66udHEe4pnfQLDQUU2lpk/7bTZuskpbMS8aPj0bCblcUpD1iWRcIPCYheFG1WWAjUn9IPIMTT5ecv46O+H+Fa3jUUqAvg7uyO9uHtHX9wkN68cd/FwfUjV4vWjmIgR0GKGBy6ewgsWMQHxutJZ0vgU/qZQmhMMHx2/d38sW3KNrzU8SUMbmzeZ1kOWOuzZIaSa8CtHwGFiv/9Y5PIPytwd3bHitErsH3Kdrgo+Zkcv0Z+6PpaV4S1teLlZgFi4x33QHeEtwnVz4tCFp/9I0ajddoyNMx/FsOHkzlOqyVWghRFRaRoiWGIqv6jj8zHSTc3joTkS+rrdNzrjiD96D7lJv0cBUcld+l4TAvEbIVSqUSnTrFgGAalpcZE7v+CvWeHBh3QsUFH+LjadzPa0osaAJo2Bbp2Jc9UTF14Rkk/w35+FF27ctvQPJ6tpB+FYTGHowqdqMWnGKVfbWUt0o+mo+ReiWNOBoCrjyvC2obx9qCn13LlzYX47OhnqNHWQMEoMLHlRGycsBHTE0lTsEGNB8FDRW5MOu589RWwejXw3Xfc/kxJP9qDsbycm+PFkn70nigshF6daQj6nXp7i4+VxSLKNwotglqgVUgr3nne0nWVs1CCotvr3fBW1VsIas7zoIBcx1K3K6hVlqC8RlyOjwpV+OZJ+uzQa9CqFbGKrKkBtm8nBaCUwJOi9OPLR9ta/CoW45qPw9IRSzGm2Rj9a9Te01nhDNz4GtgSA5RyCoIAtzqlXyX5ImiBCd+YUl1SjWt/XhNUYRrCPcBd8BrKAUrWOsL6Wgg+rj6I8ROx3q4pIgVlTqRaWM7nVQz+Jf0eMGgTx5CEEAxdMhQNe0hfiFNUa6r1BEKfaGNvXVlJv51tge0JZGW1LR44PAb9+5NFgZPj+DJBBLkH4VzmOaQVp5k1Z6aJ182bRS4EAeRczsHNbfaRr3yoVddiz6t7cPG3i7Lv+0HDMDDXarnq0kdaPIKtk7fC3dkdx+8dBwAk5yejz4o+elum+kR0n2i8WfwmEiaLUxMBAKtj0WlOJ7R7rh2qqnTIzKwBy7JGwZo99p7nswmD17GBuMr7mhqiDgOAsWNJkKBWG/vFG0LOKlv6d1oi/aht09ab9U/6eYV76Ul7oaBTbHDlyL5+DMPg5k0GP/1EkiI06XfFSj2AXEq/UM9QfUN7ChrQSUk8qjxVsgX3y7otw5qR4vr5AcIqBD7odDqoVJVgWVbw77t2DfjhB1KJSxNfmZmkyp3aZKWmAoVeh5BdlYb9qfvh5ybMpLevyz1fvswlTqqruaSPlMS4h8oDlW9V4uZLN/WLTtGflWiT5NPQB1E9o6DyFEiGygxbevoZglbyVVeTv5EuSJ5v8jE+6fsJTj9zGq0jSXMNel1pkqV1a3HHZRguqUz3T6tbpfaBK6gsQHFVsbQPmcDaNVWqlBiyaAgadJRn1aMuUKMit0LUs86yrL4Be5dI+0g/nU6HgoICq83OtVpO7W6J9OMr5ODskew4URvQwJtcm4xS8b31wsIAncIJuuvJSNlT/0r6pLwk5FTkIK8iT2/9ZIj6UvoBhGyh/d4ivSOxdtxafNz3Y6iU9TNuKRVKVNRUoFZXa9YfVyps6kdnAzZe3wgA6BdjXeXHspZjpfKacryz/x20+akNr4uH4bPLMAz6xvTFt0O+dUgyEDC2Sa1O+gU48yL3h5Tx9CRq+RbQbx/gKlAKHv8q0PJd/vdMMK31NAyNGyrYo9A7whsDvhiAqB7mz4xYiI13Cu8Uorqs2qKCAQCK7zRFZP50dAzqg+Bgru+NocUn7RXesCEp6hVSUFuaj8rKuLW3I3qmGir9HFFHWlpdKmmMtgZHkX60Zxu177cVOp0O1dUFejt8upavqeHuQTkKSK3BUaSfXKiqAnRMDe4ot2PPnT02FTFT0o8WGvKRfs2bk7GtrIyLOcUks03tPQ3haIIBAL4c+CVuvngTTyQ+oX/uheLV2IGxmF8xHy0nOk41UFVchZryGrPrpNEQlXKlKg2fnp6PefvmYUvyFv37Tgon/DryVxx+4jCWj1quf52u3+g1KSoif5+hupqSfioVR/ikp5PP0O/CWk8/X1/LrkOc0k9crCwF4V7huDrzKrZM3sI7b1u6rtpaLc7/eh4XV1yU7XwAkjsRiiHUaqDMjci8EoLF5fjoPHn3rvn8Yar0YxhO7bd1K/mMVkvGcsN+nUKg19qU9GNZx1pgC8FJ4QRPlScG114BKu8B8S8TW/M6WFL6mX5X0X2i8XbV20h8ItHqcQtvF+qd0/igY3Xov7I/Xtj+AkqqpBcC0DGvPlR+ktH6I2BcHuBMFpxyPq9i8C/p94BBJyDfaF90eL6D6J4pfDidcRpqjRrBHsH6JtQUtBpBFuY7qBsQ1J2MgJ5xZj7A9Q0PlYe+59GlnEtG7w0dSoLH+/eB48fF7W/HCzuw6dFNYHXyriCc3Zzx0q2XMPCrgbLsT6PTYPONzUaNWR8UTBUHhgvNQY0H4dZLt/DLyDp7n2sbcPjuYczeNVtfQVJfsCXhwCgYdHutG9o92w6ZmSyqqtTw8GCNkoSUDMvOFra44QPLsvreGomhiaI+c+0aSXD7+ZHeYfHx3Oum0Gq5oFSO6ns6fhQXCy/GRjYdCYDYllpSQj0oiK3QlJv0u114G79f+h3qWnICNKHSpg0wZw75/dIl/s9SyKH0G9dsHLJeycJ3Q78zet3WxGPKvhRc22ihqaQIsCwLj2APuAeKbzAhhfRjWRYsSwJXIdKPzpFVVcB//kPmi1deIf1sFi4k1jo1NUBmyG8AgDHxYwT7yQIkURYYSD5DyVwa2Ds7S++lYelYliBV6VffsLVimkKlIkkRFjosPPoDcvOJ/2ZokArzesyDu7O73l6Hkn6nSMs5dBLXPhKAeRLFFqXfk5ufROCCQKy8tFL8h3hA7x0hpR8Fy7IWF1diMXb1WMy8OlPUtgzDIH1uOk4/fVpPytgKlmVx7949qwm11FRyH7m7A9HRwtsZ2ntS0Pmx3pV+dUorKT39wsMBMAySuj6NKXum2XTcy6su44uAL3BnLw8RYgX7UvYBAHpE9eAlOOpT6WcIH1cfTGw5EXM6z6nX40b6kCpKIfsrsagPpV9uRS7WJ60HADze+nGr26vVXGKHT9mjUqrw64VfcSnnEjZe22j2Pn1266uft7NSq09Csvf+AjK3A7WlwKmngF3tiY1n6mrg/KvkD2MUQHBP4R3GTAVin6yXcxcDMYl6nVaHxU0WY/P0zVZJv9S6DgcJdbnRxqQuxoj0u3GD/Gza1PK50e+dr68fjSe9vEifY7lB51+NxrjvoRxYem4pfD7zwUs7X5Jtn7Snn732nqagSr+mAVYulhWwLIv79+8hIIA8/DTeoYSDSuWYPnCmcFTcervwNr458Q1+u/CbXfupqgJqlcX4uWI4Bq0aZNM+GpmYU/GRfgoF0NGgHtjfXxxhbClBL2XtZCuaBDRBXEAcXJ1crSr9FEqFw4pBKHbP3Y1PvT5FTblxgoaOGfcCl0GtUaNnVE+Ma2ZsDckwDHpE9TAq8qTja3Awd69mZhKiiGW5dQkFXYMkJ3OkT0CA9XGAYSxbfNLv1N1dXKwsJyxdV4WTAgffPYhjnx+T7XhpB9OQul+4NQ8h/chiW+zao2lTMqdmZwOHDhm/R0k/Q/Koa1dy7cvKgOXLyWsREeIUs0L2nlVV3LzgqIIKlmVRWl2K24XcBP9e7/dQNq8Mw6rOABX3gLZfGeXz9Uo/k55+NTXm11zprIRSZX2CZ1kW3zf/Hn899pfgNncK72Bf6j4su7hMcpEzwD0nYojYB436drz7l/T7H8KBtAMAgN7Rvc0mUBqwycJ8t/8O6PgD+b33VqDdQhl2ah9ahbQCwPUpoHB353oQrFsnrhKw+/zuGLNyjOykHwD4RPogJCFEln0l5SZh9LrRiP8+Hjq2fqsFTGFqA2RqiRjuFa73On692+toEdQCeZV5eOOfN+rxLAnSDqXhyh+2WazSyToszHiS9/MjfzvL8vt1CyG9JB3FVcVwVjibEfVCoNaebduSc6C9w/hIP1r9yjDyVNm6u3OKMKG+fo39G6NdWDtoWS02XNtg/0FlhlibJNNeYYY4lHYI/zn0H3x5/EszdbEQlpxZgml/T8NTW54y2m/DhkSdolSS50YoSWNI4NqjpBBaXNHEY3GxsFUsH3bN3oU9r+yxvqGVc5r09ySMWjZK9GcMexmIgbd3XdPqYv73aYAPkOvw6adcYHv5MrBzJ5Aa/C3SfVcDINX9lsAwnNqP9vWjSV0fn/rrH/bfUDEN2Fd1HBQEZASswrunZ2Kn87MAjAN+SgSlpJBYiBLuhskUa6D7o/eJLaQftTy6kG3Fx9cKxCTEWJbFLx1/warBq+w6li1QKVXo0KCDzUS1VFBrz+bNLfe74kuE02fcESoUS5jccjKyXsnCxgnmhIkQ/PxIMqlW5Ym8PNsGEBcfF4S0CpFUYEFxNot4FXeP7G72nkbjWBu/hxH0ef5vUPr9ev5X1Ghr0CG8Azo06GB1exrTK5XknjOFSqnCc+2eAwAsPrOYdx+1ulq0XdoWb/7zJkqrHey/uLcb3uo6BACQHvAlMPw64OwNRIwGIkcDSlfgzi9A+jqgWt5iw3OZ5/D7pd9xJUd4XbFhwgbsmrvL5mOISdRra7To+npXNB3V1CLpV6OtwY7sn5HvtR9BwSTYo0q/Owa1AMlEPKYvLBSCpeSvI/v5AeTepCRIiXRRgEXE+BIplqkVvj1whNJPq9Pqk7n2Kv0oTK3caA7J379+YlfDuFXOvOj1vOt4ec/LWHJ2iV37IUo/Yo/npHCyibQyLVDiI/0AzuITEGcjCHDXr7bWnIyvD9LPEEK93wxx7/g9JK1PEt7ATkR2j0TiE4lmDiY0H1DmcREA8EjzR0Rdy7FjgWefBb7+mlNsZmRwua+gIOPnhLqOnTwp3tqTguZtDdeoFPXlVMGXY7Rm7zlh0wRMPzJdtnP4541/8PcTfwu+X1nJ6kk/sUo/T09g/HjyO+1dS8FH+imVwJg6h8yLF8lPsQ56lPQrLTVeu9Gx1c3NcT39CtQF8PnMB3HfxZk7Mww8AbT/1uwzXSK74L1e72Fyy8kAyFxL10l8CuKsC1m4s8dyMaFOo0PXV7ui2bhmgtvQ9XFCcILkNaShIvahU/qxLJD8HZAnUoHkAPxL+j0k+GP4H1jea7ld+9ifuh+AubUnwD2gNBD4X0PrkNYAgMu55o3SR44kA2lqqnl/JT7EDYlD05FNoXCS7/FgdSxubL6B6rJq6xuLxOkM4lHWIbwDFMyDfZTpBEaDWEt90FRKFZaOWAoA+O3ib7iRf8OxJ2eCw/85jG3PbRNdYbFhwgasHroaLMsiO5u8ZmrJwDAcScQXmAmBTm4tgluItqWilbi0QtcS6WdoGWApGSoFVO0nRPoBwJSEKQCAP678Ic9BZYRUpR/fvfz1ya/x7sF38dre1xD5TSSmbJyC7PJswX1llGZgxaUVAICpCVON9hsSQsanJnVr9cvmQxgAjmSQi8AFjKuMKBGl01lenJli0DeDMH7t+HqvWJLS0w8APD2J4kBITUHnyG7duMC7Z0/yDwCWHtmEpMg5AIBP+32KHlE9rB6Tkn5nzhj3g7Olmm/7ze3o/EtnzNwuTnFFIdXes75hr70nAAQE6nA79DMAgKqclE8bkn60ojo5mag3AVLhKeU6mPZIscXes01YGwDAhSz7SD96z1si/RiGQWS3SIS3D4dOa3tRkLZGizM/nEHGafkszuSGmH5+AL96+0Ep/XxcfRDqGQqlQrz8hWG4RFHSznSb+vo1HdEUjx94HGFtpPcXo64E9D42hCPmp4cdetLvv0DpN631NLzT8x280U1coZ1hUY1QDvS59s/BWeGM4/eO43zWebP3d97fiat5V7H84nLH2q6yLOAeiUqGZMSz1c0IyQcAESOBzr8BLv5Az03AwJPClp6GyD0CbGsOpP9pddPFZxZj2t/TsP3WdsFtci7loOCG7WSjmES9s5sz+n/aH60fa22R9EstSsVf2mdxpvFIhASTi0uVfnfvksSnTsfZDlpT+llK/srhTmENhhafcoIWYt4uvI1qjTxrd0eQfukl6ajWVkOlVOmLa+2FaZETTaTWh7UnwN1TGo009xxrCPUkTbwsrdfEoKoKYBmDnlg2wMODK0QCuByCKVq35tarYt26nJw4ot00QS+mV6u9yK3IxXsH3sNre17TX8uaGuFrefC9g9j85GaHrSPbPtUWo34bZUbo0e+izJ0svKmAwBp8fYmgwMeHuyaZmVxsaUrgdupE5tHbtzlHH7Gkn1AuQqvl1k+Oupav730dvp/54psT35i9Z43MjegUAfcA+Zjl3h/0xqCvhVW1WaW5qHHOBwMGzYKESSVTjBpFxru8PGDbNvIay/KTfgDQv79xjCuWiHdz4+ZBQ3EAjfscObb6u/nrCTSzQnW/VuQPPjgCuLtO/3L78PZ4v/f7Rn0ALfUK3fniTosKPoAoAvt90s+iDShdH7cJNV9nWENJCZkzDFtyPDSozgfOzQJSVzywU/iX9HtI4OrjClc/26NAda0aJ+6fAGBO+ul03KAiC+l3+V3CVgNkYXThdUAtTu3iKFDSz1TpB5DBeehQ8vsuCYWWtZW1cpwaAFIBsW70Ohz+z2HZ9nkqg3iUie0F5yjodNxCRgzpBwBdI7tiZNOR0LE6fHjoQ4eenyl6vdcLE/+eCIiMLXUaHXQa0pckM5OBUunEG6zRZ4vPd10IUq09AY5so4FGkyaE0MvPNyccHbHgpgGupb5+k1pOAgMGx+4dQ1pxmnwHlwFiSb/gYEDtfB93881v5mFxwzA1YSq6RHRBra4Wa66uwaBVg3h7dWWXZ6Pvyr4oVBciPjAegxsPBmDe7LtV3VpDiPSTk8D95MgniPwmEl+d+Er/mlLJJU+kJB9jB8QionOEXfYsu1/ejVPfnZL0GanVqqGhxEdF6G+jz07btqQ5+/z5wKuvApMmAVVO2bgQ8yjAsBgeOkN04rR1a7L4zs4mC0J7nscabQ1OZZzCmcwzkj73/0Hpl+m1GeVu1+EKH0TnzIJKZUziREYCjz1GbFXpdZai8gO4hR99bm0i/eoWMUl5SXYlET08SJV5VrXlqsrBCwdj6OKhUChtHzAqciuwY+YO0er4x/9+HE9veRp3CqXbR/LBy8oXXF4OXCVtPKySfjTBVlBAFob080D9k362ghYcJa+5gP3z96M4rbhejquuVeN6HmmcyBevGPbzqy8V84OG3t7zv0Dp18C7AT7s8yHGNR9nfWOISwyHeoZifHNSJr/4tLHaT8fq8HvK7wCAOZ3nwNXJQSXsALnhemzAwcqfAViIX1R+gLvYrLkHAB2g01jdNNCNTA6W7OxfTH4RU3dOFXdsHkgtcrJE+t0pImOze3UsgutIv6AgMpdptYT4S08nc7Obm/WkpiXSjyYHHWm15SjSL9wrHN4u3tCyWr19pr2gNm5ykn703Br7N5ZURCIELy8vsyJWWXNIIuDiwq1z5IxdKemXU55jl0OSWs0p/ZyVtpF+gLHFp5AyxdmZU/vFxYnftylxS1EfPf2qNdX48PCHWHRqEVzddPprKVR82OOtHpjw5wSHOGxZgloN1CpLUOF8F4B4hZghaD7IUOlnSuD6+HAF2tRGUirpJ3QdATIvWIuVbUVJdQlvjGPNthUAKvIqcHO7PGNn48GN0Xy8sCNWcjFZowQwjeHuLJ5sdHEh60MAWL+e/D3l5RxBbTrmubhwDnKAeNIP4Lf4tKcYWCwUjEI/9tGWAl8d/wqDVw3GX5eWAbUlQPZuoFzYPhXgxhS+PGe3N7ph8LeD7Sbuz2eTArK2YW0lf5Y+I35+JPfyUMHZG+h3EGjy4gM7hX9JvwcMZZ3J/djVYzHp70k27+dG/g0oGAXCPMPM7B1KSjiLP1mS/7d+BNLrbPsKTgPXFwDlty1/xsGg1TnX86/zJtNaE07QKhlFsaTlElltsXwa+mDI4iFImCo9oBACVfqJJf3On+ds5uSEYR8FsaQfAHzQ+wMAwNqra3E196r8JyaAqB5RaNSvERiFuMzUxE0T8ejuRwEAOTkKeHh4oEED86HTUgWMEJ5p+wz+fORPPNv2WVHbV1Vxky0NGF1dOb/4lBTj7Q0TcXKBBjiWSL9wr3D0iemDOP84u6vg5YZY0q/E+QYOtmyKlYERGLVmFF7Z/Yo+mHm23bNYNXYVjj91HKefPo1Qz1BczrmMUWtHoUrD9dC6lncN/Vb2w82Cm2jo0xC7pu7SL8otkX58MZOc11Kj0+B+6X0k5RnbqdCgU6riQFOlwZU1tlnmaqo1uPDrBaTsSbG+sQGkkH5KpRItWxJiUiixapicatgQ6NKFzJmRkUBcu/twr4mBT0V7fDPgO9EEp6srZ411/bp9Vle0b+2tgluSguqHuadfbS1HvtiagGBZFnurPgEAxBW9CGedNwICzImHCROApUuBwYPJszZQYmtd08WaLfaeDX0aws/VDxqdxuzZkwIPD+BCzFT8Gd4Yx+9ZtwrR1mhtTqa4Bbjhsb2Poe0z1hdgGp0G666uw68XfgUrtqrGApRKJWJjY/VxsinUauD99wkBGxhoPSHm40MsagyreB8k6ff63tcx8c+Joi2iAe4+ZHp0xYxLM+ATJW0wObPkjE0KwaS8JGhZLQLdA9HAy5g4KS0FTpNwtN77+T1IUFWNvaSfobX2wwKx9tkvdST9zlZfWY2kXG5M23VnF26X3IaXygsz2s9w1Gkawdb4hRf+bYHhN4Bo6+vyQHeSrc9XS7D5kAgx8c7NbTexeuhqZF/M1pN++fncHEuRnEtIP4/qWH38yTBcrHLwINfPjxYVWsKDJv1oPCU36ccwDFoEkUoSuSw+aZGTnD39Yvxi8GHvD/FM22fs3hedc4ODyUWnyiVDe8/6AMM4pmAt2IPc8LW6WhSpbR8o5FD6AZw1JCBs7wkAzz0HzJsH9Osnft9COYn6sPcM8wqDglGgVleL3IocfXGc0DMa3TsajQc3tqtAzRL+fvxv3rhHrQZK3UilbaR3pFHfPrEwdD+iRRaGCk6KLl3ITzoeS7X3NG01QsdbV1fAxcVyrGwrLFmY02uqVpvPMRRbn9mKNSPWQFurRX5yPo58esRhak5djQuCSgajqUtfyZ/t04dcx4oKkoOhzwxdM5hi2DBuDLfUR9wU9Jobkn71paIO8ySLh6wycvDLuZex585ujEl6iuTxJ1YDLd7Ub6/RaXA977rRGtPU9tkQTUc2RcuJLS3mR9KPpWPVoFVIO5TG+z7LspzSj8dRxBroeT101p4AoHQBQnoBvhwPIPfzag3/kn4PGDopzZMsoE1YGxS9UYT9j+8X7Ofn5yeTxd/Qy0D3OtIvZhow4hbgb71HhCPR0Kch2oS2wYgmI1BSbW7sL1WFFdktEqFtQmU7P48gD3R8oSNCW8uzz8raSn3SsEM4+e4LC4GZM4EvvzTf/t49khz76CNjKf6dO9Ks/HjPpS6AdHLigh8xpF9iaCLGNRsHFiy+P/29fSchESzLQlujFb09faYyMlhUV1chJMT8ubVF6dfAuwHGNR+HLpFdRG1PveC9vY0VJjToSEsz3p6SDHIm4ijpZ8neEwA2TtiI5BeTRdkg1ifEVjkuuPAGtMpKsIpabLm5BV+f/Bpv7X/LbLsODTpg19Rd8HbxxumM03r15oakDWixpAWu5V1DA68G2D9tP6J8CTtbVcVdG5p0iY8nFZ2FhfzfrZzXktoWmSYybE2a/TPvH2yassmmij4nFye8kv0Khv0wTNLnpCxcdTodamtzwbIsior4SVUhKw8AmP1Ie/RKuore93ciuqG08jFq23rzpn1Kv1g/0nSnpLoE+ZXik4tiqjEfFKo4ftzmyvd9qftwW30WCq0bQu/OBiCcYAwMBF54Afj4Y+nXgC7WMjPJ/UPnTSmkH8Mwslh8ursD1c4ku3CvxEL1BYB7J+7h28bf4vqm6zYdy9nNGY36N0JwCwHvKQMk5yejWlsNT5UnGvk1srjt3bvW+9/qdDpkZ2fzxsk1NcCHHxLLVi8v4L33rFd2Moy5VdKD6ukHAGuursH6pPW4W3JX9Gfo+Ze5BCGkVYhkhfXF3y7i3I/Sq7+8Xbwxq+MsTGs1zeiYf/8NPPoosJq0OhW0KPtfRGP/xujUoBMSQxLt2o8j7T1rtbWYsW0GfjjzA2q14t1LxFrAdY7ojGFxw1CjrcHLe17Wv/75sc8BADPazYCvq6/U05aGS28DSZ89MPJUT/pZmJezL2bjyporNlstiylWK7lXgrQDadDWaOHryxU4mCaLr2WTAitvbSOj6ztyJPm5YwdwvC7PZ83aE3jwpJ+jlH4A0DSQfAG0Z569cIS9Z5OAJnin1zuY03mO3fuic25AALlP6b1THxZ0pnBE7Ori5AI/V/JH2GPxaUj62dO7mCr9GMYyoermRtR+UnJ4dC3zIEg/J4WTvjgovSRd1DPKsiy0teLzMmLBsiySNiTh/gnzWFmtBmqc8uDK+qF1aGub9m9o72nY088UXUzSPPYq/Qz7+VmKle2BvrCJp3jbcO4QIua7vtYVLSe1hMJJgZMLT2L//P3IOmcl8OeBulCNBUELsP+d/YLbRDE90OnWTkwP/FHy/hUKzink9m1u3BOat7y9yfrjjTfEX0fAeB1JUV9ja5hXHelXp/Sr1daCAXDFsxMQ0tesUrasugzNlzRHt2Xd9H0AbRE3GKI8qxz3jt9DdSm/001WeRbyKvOgZJQ2qW4t5XEeOHQawERdLvfzag3/kn4PGCzLoiilCIc+PIScK/ZZZLo6uSI+0LzjtuwVWm6hgFtdGYtrEODVGHBkvwYRYBgG5587j00TN+kruQxBB9PSUuGKFEOM+GkEhiwaIsu5aWu0kggmMbiccxk6VocQjxCEe4WDZYGFCwm5d+gQ1+OGYu1asvjTaLhG7XfuAHPmEBs7qePO5s1ckodO9ob+9Lm54ppvf9D7A/w47EcsGrJI2gnYgZzLOfjE/ROc+PqE1W3P/nQWJxeehE6rg0ZDJpTq6mqEhpr/cfZOhmJA1XWmvv60WjDVRJnvSHvPzEzL942vq69dlo+OgpjkycG0g9h2ewsYVok2d9ZgcNRYtAxuqW9obIrWoa2xZdIW7Ju2D50jOgMABsQOgI+LD8bEj8GhJw4h1j9Wvz0NKN3ducBZpQKa1dnQ81l80nFcjmtJ54nk/GSjqjtbSb+eb/dE34/7Im4Ikdoc/ewoLq64CAAoyyzD4Y8OWxwDnd2c4R0hgT2BtIUry7JQq0mgW1NjTDbR1yiJwxcsNm0KvPsOg0/eDpRsGWFI+tHv1ZZr6Obspq+6vFV4S/TnHmZ7T/osqlTEXtYW/HTuJwBAw/xn4KIhq2NHBPwhIWRNpFYbj31SySJq8Un7udoCT08u4QSt5djLv7E/FEoFatW22ZVra7SiK3NpwUOrkFaCfYZ1OmDNGuCll4h9rqV4jPTRzeY9/sqVxNbT3Z0svsVW21JSytSm9UEo/cK9SAYgsyzTypYc6NhRUkIU1unH0iXFl1O2T8Fj/zwm5TQBkMTyoiGL8NWgr4xeP3GCxHoNGgBjxhBFwv8XJIYm4uTTJ/H9MPuK1ui8S11Z5MS1vGv46dxPmL9/vqTktFilH8MwWDpiKaYkTMHyUctRWVuJ6Zun49i9Y3BWOGNWx1l2nL0IsCxw51cgc5u8Sj+A9PNL/tbqZkEeZN6xRPqd+eEMNk3ZhKqiKsFtLEFMsVqH5ztgfuV8hHcIB8MIW3zezCOLwFCXWKM8X+vWQMuWZEy+UDc9iSH96BxIC9MM8d9O+kV4kSrHjDJ5eto6gvSTE3TODQggAxFNotLrWF9KP8Bx/agpSV+ollCha4KqKkDHkODFHnvP+Hiup7vcog+hnER9xTyG9tfW1LgFtwrwmc9nsrbAoWAYBvMr5uOR9Y+YvadWA2HFY/E6CrBm3Bqb9h8SQkijqiquDypf8VNwsDHJGypSAyCk9KPfpaen5VjZHliyMFcquflI6Blt2K0hxv0xDgzDoNtr3fDE4ScQ1k56P2ltjRZBzYPgHigckIh1cRIC7Wt7+zb3zFhS3zZvDnTvLu0YfPae9Ub6mSj9NDoNdAAOhT4KxD0HZO8jLbvq4OPqAwYkQKBjpaU8Z8aZDHwT+Q3OLRUuKmw+vjnmlc1Dk+FNeN/PrchFiEcIon2j4eYs/ULS+cqR8YbNuPMLsFYF5BzUv+Qo1asQ/iX9HgJknc/CwfcOIveKSO9JiaAPp2wDSulNQF1XIcWygDoLqJQnIHYUvL25gKq+q0Bv77qNTzw/wbU/5bEHAYDzWZznMcMw2L6dW6QBxJeaIj0dOGLgakBJv4sXyc+0NODAAeFjHT4MvPMON0ldugT88gshEnNyjHtN+PuTYEajEbfwbhHcAs+1fw6qeiSNvRp4Ibp3NHwaWrfFOv3daZz85iQUSgVKSzmbXD57PqlKvys5V/DpkU9x+K74IJdWB5mSftaUfnLaewYHE0Vaba04RWdlbaWkv9HREBMYfn3iawBAy6oZaFA0CZ+03ogrz19BQohw5VGv6F7oGtlV/39fV1+kzk7FpombjAg/QNj3n9or3eURftDEjRxKisb+jcGAQUl1CXIquIyQrUkz9wB39JjfQ2+Ze/n3y9jzyh5oa7S49PslHHjnAG5svsH72fO/nEfhbemLb6l9KVxcWH2ipaiIzIumCQ1XV/Mk5+Ybm1FRU4GOHbmeDFJASb+0NG7RZutcbGjxKRb/DaSfPQmw1CJS6RBWxPWqckTAr1Jxi++bdYJWV1cyFkrBwNiBmNl+JgbFCjektwZnZ4BVkMpLXa3ludMjyAOz7sxC68dsq2I+9sUxfOz6MXKvWh/sL+WQnspCyqeKCuI48McfZC4tLTUvVBGDO3eALVvI76+9xi3WxcCwMAl4sEo/WgmfUSo+fqZzeXExcGzBMfzW/TdknhNPGnoEeyAgTr4HhI4rzz8PPPnk/y+ln1ygRK5GI3+CmxYXJIYmSirCMizms4Zwr3CsHrsaYV5hcHNyQ3J+MgDgqbin9NXlDgPDAKPuAt3Wyk/63foRuPSWVSZWjNKv7dNt8cifj8DZ3TaCQGxPP4Zh9NeZjnWmY2xaKVkENvQyjksZhqh2DSGG9KPrEdP1B1C/pB8f6WgvujfsjufbP48+0X1k2Z8jevr9k/IPbhXcglYnX3ExTXaXlJCiuPru6Qc4Lnb1dyN/hD2kn1oNuNVEYH7rxfpWJbbAxwf49Vfgk09s3oUghBL09WXVSlVihko/oWfUK9wL4e3DReVlbAHDMHByNS960Y+rbgw8VbaxoE5O3FhLSX2hOIiq/QID+W0j+aB3dygzLlh1RFG3KWihaXZ5Nm/rJBo3i4lb/Br5IapHlE3F4J6hnnji0BPoPLsz7/s6Voe8SrLAloP0s6b0sxUPqqcfYED6UaWfzsSe+PhU4NI8/fYKRqG3uy2oJIOIJdLPPcAdnmGeomIcoXsgMTQR2a9mI/nFZOt/EA8eantP9wigwQjATYI0VGY8bG0O/1+i8ZDGeO7Cc/COlKZ0oNh7Zy/m7p6LSS0n4e2eb5u9T4M12Qav7c2ABqOAnpsAbRXwVzgQNRno9odMB7AdLMvibOZZtA9vbzSoMAwZUPPzyfdhbUCoKq7C3jf2okGHBmj7tPRmooZQeakQOzAWAU3kmz1GNBkBHxcf+Lv54/594LffyOtjxhAV3vnzZOJq3JhU1bMsR9RQ0i/ZYExdvRro2dM8gcmywPLlZAL89FPg88+Bn37i3s/PN16MOjmR7zYvjyTV6nOBIBbuAe6YunOqqG2fPPYkSu6SKJVWVXl4aM36RQGWG9zyYW/KXszfPx9j4segZ1RPUZ+hto9CpF9GBlmg0WDSEUGhQkECl/R0cjxL1Wr3Su4h8adEVNZW4t7ce/rkyIOEmOTJ2vFrsejkIhTuewo3YG6rIRZC/QGESD/6XZpWZht+hq9XgFS4Orkixi8GKUUpSM5P1jd4litpNnXXVLh4uUCpUqLdM+0QGB/IW9lVeLsQW5/ZinbPtcPwH4dLOoZY+zFD+PmxyM5mkJYGLF5M5oVly4yrwwyf7au5VzF63WgEugcibXYaPFQSDlaHwEDy/BUXc2SRrSR8nH8c9qfut0npV1lJVFayWHzLBHsrMwHgzDNncL80Ay9MD9J3kXNUgjEsjMxttN+RFGtPioGxAzEwVmJDQRMwDFDsTqopX/zncUxun2d5+zoyvrq0Gk6uTlCqxJeU+8b4otGARvAMs54QoUo/IZukX34hxUkqFXkmcnPJd2mtF58hdDry7LIsiVnatxf/WUDY3vO/UenX5NkmYBQMvMLFMZbaWi2KU4vhFe4Flaf4Qisdq8Pxe8fROqQ1vFyMjyWFHPpfhY7VQcfqbLZ5c3IiybOyMjJPyElA0wJBqjAWC1vmV4Akc74b8h2qNdVwL3Sgf5whlCrAPUIfv8hW2Nn2a0BAsWwIMaRfgw4N0KBDA8H3rUGMs8Ht3behVCkR04dYfyQmAmfPkiKLzp3J/MWyLLKqiL1nnEkxGkAsztq2JWvI0FBxsQpNlt6/T5LSlNBiWW499N+q9BvUeBAGNba9QMcQOh1ZnwHy9fRT16ox8PeBYMEi99VcverUXnh6knOsribxcX31nTKEo0i/xUMXQ8fq0DRABKMtgKoqwEUTgidavCApfuGDLXGkGAjZe9Yb6efNWUPGWVH6qTxUeHz/4w45D3WRGlnnsxDUPAheYcaTqxxrEIDkZCiRo1AIf7f9+gH79pHYVSw8PMj5qdXkWaQtVhzRvsUUge6BcHVyRZWmChllGWa2/Z6eJJaWUqyUeS4TJeklaDammWznmVachvcrYuGWEI2ZrikApBOL0dEkFisrA67XdUOQmzyipF9RUV3hgFv9ja1dIrvg2bbPond0bwDE3tONAfpn/wbcVgLtFgHOvkaf8XfzR6G6UJTSz6+RH545bbmv7L3j91BVXIXYQbEW+3cqFbbJnh9qe88Gw8m/B4iHKP3z/xMMw0DloUJoYijcA2xbIO1P3Y+kvCTBRKCsEzzLAs3fBCLrquqd3ID4V4Bwaf2YHAGWZZH4UyI6/tIRJ++fNHufDqhiSBlnd2dc+OUCUvfbUIZugpg+MZiybQpCWsmQra9DpE8kpraaiiFxQ7BpE1lIJCYC06cDvXqRbVauJGTe0aPk/9OmkZ+3b5PLSEk/JyeSyNy50/w4qalcxUtqKjB3LrEQpSgqMk8O0MFWLFFSpanC2/vfRvdl3VGlsc36xlFw9XHVXzcarPr7O/NWqdDnq6gI0IoouKQJUinJGEr6mXqI+/qSxTnLEjKOwlGVYDTopHajgtt5RyDKJwpVmiosPbdU3pOwEWIUYu7O7pjXYx4i/QkrJ7dlqxCBR0lAPtKPPoeW7CakgC54kws49l+upJlPpA9cfUnmx83fDfGj4nkDPK8GXpi0ZRLaz5CYuYc0e0+GYeDv7w9fX/LcLl9OAvvSUlKdLhQovnfwPQBAz6ieNhF+5Nic2o8KBmx9HpsFNkOMbwzcncXHCmL6Ljwo0KpVexbcDMMg0icCIQFcFs1RCUY67tK580Gow0xRWC1uor175C4WNVqEvx//W9L+W01thSnbpliNT1mW1c9piaGJZu8nJwP//EN+/+ADYMAA7nUh0Of21i0GH3wAvPwysQS9fZvc108/LelPAWBs78myXLLiQZBWeqWfBOs4moQvKwOCEsLQ862e8I3yFfXZkvQSLG66GIf+c0jSed4quIUev/VA2FdhZmqSB/n9PQyYsGECXD9yxabrm+zaj6P6+lGln62kny19n9qFt0PniM7w9/d3vMV71l6gkPyNhoS4LK1S/FoBvi3Net2YIsI7AstHLccfY/9wmF2TmOux5+U92P78dv3/hw8nJF5VFbBgAVGSsmDxtOtOtEr7BU1CG/LuZ/p0EgsNEsl1+fmR9Q/LGqsKaTsNWnTrKDiS9JMT1QZCGbmUfilFKWDBwsfFR5aiSjrnKhSMPh7OyuIsIf8XlH7tw9ujY4OO8HG1XVX2sNu0AlwcbJqLkUzE2zimUWvI9FLrSj9HIut8Fn7v/ztu/G3uNnO39A72JzTCqqpJdh3DsBA7MFDYqjUoCPj5Z+AxiQ7rfH396Drdx4d7buWebxmGQd+YvhjSeAivklhq301Wx2LD+A3Y+eJOSXNl5rlMHPrPIRTe4U/gXsm5AgBw1vjBw8O278DZGWhYNyXSFklyk0ceHtx8RUni+rL3HBg7ED+N+AmTE0ibmlpdLVwZILb0FFBwGoiaCIQbT/oBbmSQKFAbK/1KSoiIRCqOfXEMa0au0Reiyo2H2t6TB/XdAulf0u8BQ6FQoCyzDDXlNTbv40Aa8WbsG92X931ZvdgZBmj9MRBjoJRq+6Xx/x8QGIbRN/5cl7TO7H0pKhalSolXsl/B2NVj5TxF2cGywLk6++SxY8nleaTOtvzCBWLBCQB9+gB9626PrCxC3BUWkoqkJ54gr69fbx5cn6hrexcRQfZNSSe68CwsNCdRhKrLhOCidMFvF3/DsXvHcOTuEesfkAFX/riCTY9ugk4jnBnIvpiNwjuF+sCktJTcYxERXlDwyGV8fMj3ybLiSBNLCVI+sKywvSfDAFFR5HdDix1H2HsCHOmXYSVXyTCMvrn8kjNLUKu1ra+UnBCq7MsozcB7B96DRsc1mZJ6L4uFULNv04Q0BctyRKAcSj8A6NSgE7o37A4fF+7mcFTiESCqvovLLxq95uzmjKYjmiI0UWRzAwNISUoqFAo0bNgQ/v4kwDK01khJ4beg2n17NzZd3wQlo8R7vd6TfH6GaGIicrSV9JvdeTZSZqdgfo/5oj/j5MRVlT9spJ9cVbaA8eLM0aQfHWNtrdCurK3Eyfsncbvwtu0nw3KLBTGL58gukWg6qilaP2Gbzac1lFaXwtvFG0pGiZbBLY3eY1nOIaBfP9I3iloZ3zDPw+hRU6PApk0N8dprCpw9S/ql0J4pTzxh2yLZsM9VZSU3zv63KP28vDgOgia5WR0r6h5wdndGtze7IaZvjKTzpLFKy+CWRhW4Wi2X/HwQ39/DACeFE2p1tUgvSbe+sQXQOUHO9gM6Vqe/dm3DpDmW2Kr0o6BzLl+sLCvOPA8cIwlbkvwkz7RsyeWaYqC2zOIm7s7ueDzxcQxrMkwwiZPyTwq+CPwCl1fzNGwWATHFagO/HogBXwzQ/1+pJEUSnp5k3PzjD2LZ5VPUGw3zn0J4CL8NV3Q0cY8ZP178+cXWiQZvG0xpNK7y8YHkXshSQNc3jiAUWJZFcVUxknKT7LbPpGMlw4i39rOGtOI0AECMX4wsCUTD55auT+icSxXJ9QWphEJ9oqoKqFEW4HzhQf0Y+7CB5vwqKjjCWaPhnhOLOUFWBxRfBfb2AO6uten4k1pOws0Xb2LVmFWiiPnkLcn4+4m/UVUib/F3QJMADF0yFNG9os3eu11+GZUuqcjTiXdP4YNhTsYRFuf0WTTs62eo9HPkfLt9ynbsmLpD317CEPQZLbM8RerBKBiM+GUExq+XMLkAuH/iPg6+exCl9/hvoKQ8wtJ5qVvaRcRT1TotGnKEYszQ4lOn465jfaqoKYp0wLqE34EO/H2pgz3IzZxbQZJW3t6cIxyfgCZpfRL2zd8neLyur3bF6BWjBeeqN/95E31X9MW2m9sk/BUELPuQK/2ufwVceMPoJYfHxyb4l/R7wNDpdFg7ei0WN11s0+fLqstwNvMsAKBPDL/nfH1J+R8GTGpJFn8brm2AjjUmdKT2XPMI8rA7iK7IrcDaUWuRvMU2f2I+pBSl4JsT3+DEvRO4e5f8PSoVqegEgMhIYPBgMjB36ADMmQPMmkUGaxo4UFVfdDQwbBgJWEpKONssipN1gskJE4BJdYVQjRuTxB1gTPqZKv3EEiUMw2Bw7GAAwK7buyR9F7Yi82wmrqy+gtIM/gCCZVlsmLABv3b5VU8Mkp5+LHS6Yuh4yogNLR2s3WMVNRW4lkd6PLYJE1eBXVpKgneG4YIGQ8TU5fFoQtqQfJRb6UcDXGukHwBMbDERIR4hyCjLwMbrG+U9ERvARzRodBqM3zAeHx7+EHN3zdW/bsnKwB4I9ecLCiLXt6bGOIFRUcGdt1xKv/d6v4cj049gYsuJ+tdk74ljgO0zt2P789tRU16D6rJqrBu7TlIvKlNI6emn0+mQnp4OHx/z5zYlxTxQrNZU46WdLwEAXur4ElqFtLL5PAFj0o9hHGfnIwS6MHvYSD96DW1dpD266VGMXDMS57POGz0XjrT3BDglt60JsJd3v4wuv3bBr+d/tflc3MGtaizZylEonBQY9esoNB4koQEegG0ztuHsT2etbufj6oPbs26j+M1iMyXq3r0kcejuzhUZxcWRZyEnR5jo2LBBh23b1GBZFn37kt7CL78MvPWWeBWKKSjpRy3IARI/yZWElYIG3mQizS7PFv0Zw/GjpAQ49/M5LAhagIJk65OUV5gX+n/aX/I9IKQWMxxPbFGE/S+A9ry5V3LPypaW4Yi593bhbZTXlMPVyRVNA6VZ2dFra+t1pXMuX6wsK9p+DSR+BoDE4JQAkuV7zDkI/OkHpCy3e1du/m4IbhEMF2/bfB3FFDk1HtQYTUcaX+fAQOCFF8jvO3eSJKNQ0Zk9oKQfbR8B1E8/P4BH6ceyQK08TBELFsELgtHyh5aSFNl8oKSfi4tV8aho3C0hDcCjfaNl2Z/hc0vvD2pL7+cn33mLgaPi1gtZF/DV8a+wJXmLTZ/XaIjKpcjzBEb/3QfPbn1W3hOUCe7uXGxNn0U6LlolcIuvADsSgLyjQKVtBS2B7oGIC4iDm7ObKGI+53IOLq24hKI78i5AfSJ90OH5Dghqbj7gpalJH+oYd/vWeA+C9DPM79TbfGsCW4j5Rv0aoWG3hpLyq60ebYXnLj6HsHb8PYJpX2z3mii7ikhN+4M7gjyixaP375NryLJkXJW7ON8ULMuipKoEN/JvQMfqsPvR3dC9q8OEllMBhTNw/hVgUwig5STptPVLVhmplGYY7jsxvBcpkjcn4+inRwWFTA27N0SrqcLP2pnMMziQdgAlVdIreMrKOPXhQ8l3pG8A0lYbvVTfz+u/pN8DBsuyaD6+Odo+Y1vfuPNZ56FltWjo01DfNNcUsjZgri0D9vUHbiziXru9lFQD1RTLcAD7MDB2IHxdfZFZlomj6UeN3pO6qC7PKcftXbftqjrKT87HrR23UHJPvhLEA6kH8PKel/H2gbf1Kr+EBOOk1QsvABs3Au++Swg6WmVJF2b76gox4uPJe3PnkqrQo0eB3bvJe1lZhEBSKgl5OHky8P775J9h/zrT5ICQpYQlDG5cR/rdqR/Sr+c7PTGvfJ6wLRYLDPxyIAYsGAClM6lspwtKhaJcsKpeLOl3KuMUtKwWkd6RiPCOEHXO1EozKIg/QUn7+lF7naoqrn+E3MEETX5ni8hVuji5YEb7GQCAlZdWynsiEqHRkH+AMVn02dHPcPL+Sfi6+mJuF8eTfjRYMl0cODlxxzS0+KRJGh8f+XqB8IGOkeXltlk3WMLgRYMxdMlQqDxVyL+ej9s7b+Pan9ds3p+Y3owULMuisLDQiPweOZL8NCT9AgIIAfzugXdxq/AWQjxC8H7v920+RwrDhQRVI9gLKdYojrJJshf2qElYlsX2W9ux9eZWANyi2JFWYqa2yraOq5REvpp31eZzGa36Tv/7/VIrPssmyDybiXVj1kFdpLa4naZag3M/nUPagTTR+/ZUGUu+WJZzHJgyhStA8fAgBUqAsMXnjRtAbW0NnnpKh7lzgY4diWtB5862P0P+/rS3FVfU9KBsWrtFdkP2K9m4OOOipM8Z2hh6BHvAP85f9up4Q+hJvzB+0s/NTdjO6n8ddN1Fk++2whFKv+t5pDFNq5BWkvsN2qv0o3Ouo+wu9YgYCUSO0f9X1r5+XnFAo+mAt3XC9MjdI1h5aaWg4jOsbRieOPQEmo6wrY+YlHjHFF26kOtYXg4sP/wPzrO/otz1hqykH41xHiTpV15eV5Bz83vg0jybbQkNoWAUCPMiCx6aWLYVVG0lZwxPlX5RPlGy7M/wuaXJXUr61Xci1VFx6+G7h/Hq3lfxx5U/bPo8vY46hiySnJX8itkHDYYxz8cYigAsx1AsEDYE6L4eaP6GpQ1FQYzSr8MLHfBG0RsIa8tP7DgC92uJ8rqxl32kn+HawBGkH1/bHEN7z/qYb/mcmmjsLFWNq9PqUJEn/sF29XVFaOtQuHjxD575avLFOGsCZSX9HDF30QL9lBRj4tbRoi8tq4Xf535o9n0z5FWQJBSjqwGTdwwoTwNU/oBXE0DLrSVGNBmB93q9ZyQqMnSkMkWfj/pg7r25cPawbUykrifUBUUK6LPh48OpER8q9NsPDDYuoHV4fGyCf0m/hwDdXu+G3u/3tumztB9Ti6AWvO9rtTKTftpqoPAsUJHGvVaZCZTeAET2lnEkVEoVxsSTBeC6q8YWn4Y918Tg6pqrWD1kNbLOZ1nfWABRPaIwr2weEp9ItHkfpjifdR4A0Da0Lc6TX9Gunfl2fAEdJf3oApIqUJo25Xr+/fwz8bOmCbGEBFLNwzDkOD4+xt+l6WLUFtKvf6P+UDAKXMu7ZrdNkhi4+blB5SFc2s8oGDQd2RSJjyfqX6PBqoeHsMUL/V6skUSUkO7esLuo8wWErT0pDEk/Q5Wfq6v8fQdC69wY8/I4Es0S6DN5+O7hB2rxqTbIcdPA8HzWeXxw6AMAwOIhi40aVUvtTykGNTXcwotvcUCVKIYBFf3dEYuJKk2V3rbIw4MrEJAz+QgAQc2C0GY6SRo36NgAz55/Fv0+6WfTvjQajtCWkpSMjiYBVqdOROHMQofLWUm4WEiex8BAYPvN7fji+BcAgC8HfmlX3w8KLy9uUWiv6nbinxMRvCDYrKjFEuh39LDZJNlj73m78DaKq4rhonRBQnCCPoHp5+c4AoKOexS2kkV0jLlbbDtR0MN/InwqSLGYVAVC/o18JG9JRso/KRa3c3JxwvyK+Rj6/VCbz/PmTTJPuLoCQ4YYv2fN4pP2D25qW55cEImJ5OeROjfxB2VN6ebshhDPECgYaUsxw6r5+FHxePrk04joZL146MKyC1g9ZDWK7xZLOt7lHJIYax1ibA37/72fHwDE+pOg2i6rXjhG6TcqfhSK3ijCqjGrJH/WXqVfvYDHblFWi3L3BkDnZUDYQKubvn3gbTz+9+O8/eTtBctaJ/3Sj6Xjc7/Pce7nc2bvKZVA67pH94eTv+Fi1NPI9dsiK4lD15bp6VxsVl+kn6HlcVkZSDV95g5AJ89aw5beq3xwRB84SvrJpfQzBI2pHpT9nKPiVn83cuMXqkXaPpmAPosKJ3J/SS2oqE+YFq6Kbvfjlwj02QE0fMSu43965FNM3zwdlU6kMM0S6efm56bvBS8nDv3nEJa0WIKyLHMPymwdsYVs6ptg1zECAjgy/0HYezoSf177E76f+WLMujFm79mi9GNZFouiF2H92PWiP1ORV4Gq4ipBkoS6nbhoguwi/aKiuPWjt7djHEAMrbDrq58fQMapIA9yI2WV1+W1q7KBf3oANxcDLd8CBhwBVFzOY0TTEXi/9/voGdVT/5ol0s8vxg/eEd68Kk6dVofP/T/Hjhd3CJ6jPaQfHdseSmtPAHByB9ykt7KRE/+Sfv/lSM4npF/TAP6sSEmJzNJh10DgkWKg3TfcawnvAePyAC9plkGOwsQWxK5u4/WNRhMEHVTF2nvGDozF0CVDERBn34rFydXJIsEkFeezCdPXMqAtrtUJZfhIPz7QyYaCJt0AYMwYoG1bsmB7802uOr9zZ/P9GH6XpskBW/qg+bn5oXMEOVB9WHzqNDpkXchC3jVzfXptZS0Kb5vfJDTAkoP0o5a83SK7iTthcFaaQqRfw4bkOS8rI4GEo/r5ASTIVKnI2MIn8TdFQkgCAtwCUFFboSetHwRo9bpKRQK7gsoCTPtrGjQ6DcY1G4cpCVOMtqcLptJS+ZRvlEB0ceG3eaQBFZ/ST+7FRLul7eD+sbveatZQJeUIi09DBDULstk+mV5HQBph1LEj8N7HZYgcuhrPHRyB3W188U/TljioeBsAGbtGNB2BPtF9sHDQQkxNkK9XLS2wsHeBVqQuQl5lHm4Viu9B8bDae9LzsYU0OJ1xGgDpVeWsdNb3NKXqMUdApTJeUNhK+tEEXVpxms2Vft7egGsNIXqkKv1aTm6JF66/gBaP8BeLGcLZ3RnuAdYz/11+7YLey3vjRr4xg3e0jpvu2NF8AW2J9CsvBwoLyfgg9zWlpB8lFR+U0s9W2KoKK75bjLRDaVA4iV/6FamL9PajzYOaG71Hn9//r/38AKBJABnYbxfeNmspIAWOUPoBgK+rL28vHmv4ryD9Ls0HNoUCFVyhYH3FL6YIdCcTg5DVMsuyOPzRYVz47YLkffMVq5ntX8cipFUIAuP5M15t6kS6KUVEiheqipVVWRAQQNYaOh3nNlJfpJ9CwY1BpaUAOv8KtPkSUDgBqb8DOhGViRZAbZjtVfr9t5J+FPWt9KPzsth+YWJBSb+iKtsGCar0U6rqlH6Kh1FWQmCaj7Gp3U/6n8Thy8D2TyxWXFqB5ReXI09L1iukTYrw9uU55Tj/y3notPLZ3imUCrA6ljcPV8mS8Zo+47aCYTgFlyPWIKaWio5s32IKT5UnSqpLeAvybSH9GIZBi4ktENNPfG/pzU9sxldhXwnmDKhyTWUn6adSQb+WdBR51Kiurjw3lxTJAPVXUBHmSVS0WWVZmL1zNh7d9hIy414jjgkiYYn0Y1kWpRmlKLxjnkOtraxFWNsweIXzL7jKa8pRWk2qAuxR+j2UpJ9OCxScAdTiWzk4Av+Sfg8YORdzsHrIaqsV10Lwd/NHs8BmaBHMn7yhE72fnwOlw/Vp8i4CvaN7w1nhjJyKHKNJSqrSL6h5EDo83wHeEbY3YLryxxVeYslWaHVaXMomHuRuxW2h0RCrKlPbMSEYStc9PIw/xzDA668TO1CG4RLrnTqZ78fQxtI0cWto/SklnzmwEammPZB2QPyHbIS2RoulbZfi8EeHzd67vuk6vov7Dknrk4xepxVqDRv6CgYehn+7JWyauAnnnz2P8c3FNzO2RvqpVNx7aWmOrQJjGE6RZkhOCUHBKLB2/FrcmXUHHRt0lP+ERMJQWXQo7RDiv49HUl4SQjxC8MOwH8yuq6cnl6gWWyxgDYYEHt9tVJ9KP2eFM1iwesU48OCSZlJg2AtOzLzGMAxCQ0NxMecC+u+IxBNbH8W2W9ugUZZBqXWHexUZGAMCyL26//H9mN15tt09XQ2RUFdIKnasFkKcP0ng3ioQT/o9rEo/MX2KhEBJPzqexMcTO+s5c2Q6OQEYXj9bezNSS8CymjIUVxXbtI9kdjsqXG/CE9IHBYVSgYAm1jOxpRmlyDyXidpKyxUPNdoanM44jUN3D8HbhftSWJYj/Xr0MP8cJf1u3gSuXgV+/JH8BLjFcHi4Mzw85I0xExKMx94HqVT77OhnmLBhAi5kiScDTPvjpB1Mw9+P/43ybMsPeJ8P+uCtyrcEF918oCRuA68G8HIx/pw9pP3/CqJ8ouCscEa1ttquvn4P27xrr70nnXPlnEPN4BYOeDYC3Dg7OFntPQHg+pfAoVFWNwt0I5kmmnw0BcMwOPH1CVxeeVnyKdBroVQK21ZF9YjCE4eeQFQPfptHSvpVOBF1eUNv8QlXMWAYc4vP+iL9ABP7QJ/mxPL19lLgxDTg5ncWP2sNcin9KFkkJ+n3erfX8UHvD5AYmijL/gyf2wdN+omxhLQFfm5kkLBX6efkTMjkh9XeEzBX+tF1rNVn8uI84EZdcX/5HSD/OFB0SfLxaaxboCEBnUZjXLRpitOLT2PrM1tx77h9PXIN0WN+D7xw/QWzfqo6VodqphgAEOTla/dxXn6ZFMzL7UwBGCv9WJZ8h9RlibSMcNx8S6/hvVLza0JJP6nE/MAvB0pyuGs8pDHaP99e8P3+0YMQUjQSbtUN7SL9AE4c4ah5y9OTy/PQFk31RvrVWVVnlWdhT8oerL6xFTeDhgLBPYGSa8DVj4ASrgqzVluL5PxknLh3Qv8an+qUQlujxcKGC/HP6/+Yvefi5YJp/0xDj/k8i0FwfQM9VZ5maw0xMGzT8tChphDY3RG48oHRyw6Nj3nw8GrS/5+gPKscaQfTbO7p91bPt/BWz7cE36cLSNkegtpyIHsv4B0P+DQjr2kqgKzdgHskENBBpgPZDhcnF/ww7AdE+kTqpcyA8aKaqh8dCXWhGpumbkKrx1phzEpzWbwtSC5IhlqjhqfKE7k3SPK3rYRbx8+PBO6FhSQwMf0OPDxI0nTkSODPP4l6jK9qggb/5eVc4okmbmmzb42GvCeWdOoR1QPBHsHwc3X87Ofs7oz+X/RHSEKI2XsBTQOQMDXBrAqptJQM0NHR/oJEg1jSz0nhZNYfxxqskX4Asfi8f59U2tJgzFHNgUNDiVJCTF8/gFi4PmgYWiTFBcShWlONlsEtsWL0CqOxgoL2RMjKIgFFiPntIhnWCLz6VPrFB8bjVMYpI3XOw5Z85INUskihUCA0NBTPrnkWJdUliPGNwWOtHoPm6mhc3JMABZzg5GQ7iSMG/fqRsbCFdXGVRVDVhhSl38Pe088m0i/TmPRjGNJ71tEICwMu1+Vtbb1f3J3dEewRjNyKXNwtuatPQomFVqfFh7eHA27AnJp8zGhvW4B37c9ruLn1JkYtH8W7+Li65ir2vrYXTx57EpFdhcuXM0ozoGN1cFG66CtJAULm5eeTJCdfnBIRQe7Nigpg3jzy2uXLwJIlhPRjGAZNm7rLXrDm5UUS1Lducf9/UNh1excO3T2EMfFjRMcEpqqwknsluLTyEpqObopmY5pZ/byUhWZDn4ZYNHgRryL1X3tPQKlQYmTTkXBSOEHLCrtAWIPcSr+M0gw89tdj6BDeAZ/1/0zSNadJRcB2pR+dcx2K+NnknwFkj1+KrwI5B8ja11lY0mpN6QcATx57UpRq2hSGcaut69aQECAkTINqZxJYRvvbp2zhQ2wsSWLernO6rU/Sz8eHrJF0OceA5h0BhTMQMw1glEBIH+s7sAC57T3l7OknpXBUDAyfW9O1f32TfnTtWloKsGnrwNTkA01esHu/9tp76kl4l4df6Wepp58gWBa4tQTwbw/EzwWavAjEv0yeKYmI9CZxY1blPbi6kmegtFQ4Zkh8PBFBzYIQmuh4G7wqTRV8a1tAzRYjxNvX7v2FhZF/jgB9FmtqCMFGSTY3N1qY7Lj5ll7D4qpilFWXGREytij9bEHHFy0Xi7/bZQFuLSG/21tU0aEDsHcv0Mx6KG0zYmNJjiepTlvwIJR+mjoFvH78KrkGXH4H8IwFfEhFZmZZJuK/j4dKqULVW1VgGMai0s/JxQk93uqBwGbS5Xb2WHsCXLwhZ69i2aBQAYmfAb7GLRIUjm7kaIJ/Sb8HjNghsZhfOR+szjHNHG2S8ltC5T3gyFigxdtA6/+Q12rLgSPjgMYzHgrSDwCeavuU2Wt0Ua3VkqBDDBmyZuQaVORU4OlTT0s+Byc3J0z8eyI8guXLiNBqi8TQRFw4SgYLsdaeFLGxHOknhEaNiOpPCB4epOK0tpYjo2hywMmJTGCFhSTQFEv69Y7ujexXsuut8qHba/zWmg06NMDYVWPNXie2FCyKi+9Bq20AJU/jKLH2nlKh0xHiCbBM+sXFEWXFlStcwOIo6wcaX4ol/R4G0MWamxsJLA48fgCtQlpZrNSkpJ9c15SSeUIEHp+Ckv4uB+loiPhAEtjxkX5y24zJCcPrKAZarRanr53G9lvbAQA7p+5E08Cm2FMLXN5DtgkMdGwhiFJJLA7thV7p9z9g72lrYrlGW6NXRtW3cthwUW8PWRTlE4XcilykFadJrtKv0dbof68ss906/N6Je7i86jJ6vN2D18Y8qmcU+nzUx6oqkFYAR3hHGM3flqw9AfK8tWwJnDpF4gatlhSSFBYCd++S+dbNLQ9abQDvfGsPWrfmSL8HaU+pt46TkFA2Vfo1G9sMMX1irLpS3Nh8A27+boJqIKHzm9VpFu97/9p7Evw54U+792E478pRlHgl9woOpB1ATkUOPh/wuaTPVleTmBOwndDVarVIS0tDdHS07M+uJdDvUbYYvONPQOffrF4QPemnFib9gprZlo2qrATAsnBzBQDz88i7nodD7x9Cx1kd0bBbQ8H9xLbKAQpYMKwSMSHyZ8aoQuJBKf1CPFLQKqc7cPZZct2c3IHGz9i97whvYqP9MNp7yg3D59bFRQkvL45gqO+efrSoqqYG0GYfhlPKEiB8GP6PvbOOb/Jc3/g3SZu6e4uUFrfhNmywwWACjLkwY3omZ3rO5Jy5nrkrc/nNGNsYc3RsMNwpWqEtLXVv7PfH0zdJ28j7Jm/StOP6fPopJK88zZvH7uu+rpvITK+uK5F+lY2VmMwmdFpl45O0bg2W7D0DWOkn7RmlOI2smn4aDcwrguZK8f8gz2NYkkosryqP6Ggb6eeMHIvvHU98b3XZ5fUvrycqLYoBZ7VmccKDw5mes43GRkjwUXK0WggOFuNoWZmIR5ha8otscU3fzbdRIVHEhMRQ1VRFfnV+K5t3aQ+klPQzNhlZctkS4rLjmPbwNK/bKCXGhIZ6v3YaPx4+/NC3ScBZWbB2rU2t6a+x1T6BxWAykB0MwzdfBs13iPqdszZDhE3wkBIpBpBmUzOVjZXEhcW1U522/bxPetBxkk3R5iL2LN7DkIuGkNivPSlYb6gnNTLVOt8qRUAr/fQxMPBf7V42mTxPFPQEx+09AwAajQatTvmjMJlNbuvB2Nt7qoKwdBj/IfSYb3stNAnGfwB9rlPpJr6BvZJDbhZoeFI4ESmeLXiCw4LpP6c/3cerZ/C9Kk/YUY5JOomSEhFIlmzj5OLii+GUU+CMMzxvh33tL2lDYB+4bWspIQdajdbvUue2cNafLBabxYhG49xrxN721Bnu+OkOLl9yuSI7r337xOIgIsK1X/WoFveDbdtsRJGvSD9pM6GE9Ptg6wec9X9nsSZvjW8a5Qb29p4AI9NHut2wefJddgaLBQ62ODm7U/pJCyrp36B+BpNUC9be3lP6vnQlpR9AmCmMTVdt4rmZz9EvUfzdvWxr28D0gXeA3vHCQ2t/+X7Z9eBa1bsJIHhK+pXUlTC++3i6RXcjOy7b/QkqQg17T4DrR1/PszOfZXDyYMXn2pN+dVWek34T/zWRO8vudFq3OGNMBpPvmUx4ousHJNkado+xrXUsFvj995b7THR+7rXXwg03wNtv22pdbN9us/eMjfXNl1ayu4OOVfqlR4ovlJTlKgfSGC2RfvoIvSwb+m+v+pbf7v5NaROd4ri9p3qQxhKzWZ1xWrJKkoKuSiA9V63WO1VSjdrFuOxRvRfWXwvH1rV6WRqfj3jHz9igC5EVRZScIlwp/ZrrminbV6a4XlVNWTNZm74gYe9ah+8Xbylm5+c7aShvcPi+hJQ+YowJMaSRmqJ++Eey98zNFXsgKRDsL9KvyRTBds1DkNmmFrOpUfx4iKEpQ7l+1PVcMPgCr9qoNumXV5XHTwd+stb1Uwv2/dZ+z+FvpV9IiC1ZqDrhUpj8NYR7FhC2h72bkCf26tL4mBUyjidOfoKLh1zsdZt8BWlNVVAgyFPZQoCgcAi3W+xW7Ya9LyqujymtCfOr81spN92htriWmkJ15o/f7v6Nv17+q93rZrOtTwZ07doWSHNbYaHjen6+nG+tFp9tLMw9VfoFhQSR/0c+JTscyMXawGK28MkZn/Dn8386fN9gMnCsuhYLFtWeo7BMVedajpDdZtvqN9KvJcmwsKYQg9lAEKA1N4OpGfRxEDdMEFQtCA0KtY6XRbViTSklSBsMtj2IHBxZf4RVD62iKs/xSbP6zKLotiJ+uaS9Nag7VFTYagkHJOkXIDhO+nUwDv92mEPLD3l07tJ9S4l9IpZLFl/i9BjV7T31MdDrIjEwSNBoodfFEDdUpZt4jwZDA5/v/JxHVj3S6nU5pIw95rw9hwu+8Wyhb2pWn8F/64y3WH35aoZrLgeEskvpBiIrC266yftAV9tFo7eknwSLxUJFg+8Zhy3vbuH5rOcp2WlbdKz931peHfIq5ftbf0GamsSCGSAiwvlzlf7umhrb8W3x6c5PeXfLu1Q1yZ8tN7fwgyec4LqGWffugjQyGOCPFgtuX9p7gryafhJ+OvgTi/cs5of9P/imUW4gkX77wj7mf7//j73H9ro+Afnf5aYmUZNqxQrH75tM8MILsGGD+P/AgY6PkxZUzc1iYd/QYCPWfWHvCULpJxFISsfIjoCn9YYGJw/m5nE2O7CePW39qbMsFHvF9UKr0VJvqJdNFEgBZV/GYD2Bp8+xW3Q3ll+6nIM3HfR7oohaSr/Lhl3GP8f900riKoE96bcseRpDX/Vs7RWRHEForPcRSEnpJwUG9uwRdp2lpWJ94sqNIDERZs4U487Qlj9j61YROAZITXUykXqJ/v1tAcXOqvSzV2Mbm4zs/GwnB34+4PS8ee/PY+LdLhhYB1i8ezEbCzdiMLWv63jc3tMGk9nktJ6bHAQF2cYTNVT2xbUiGys1Urnll30yRoCVbLehfCPsfx3q81q93K2FE6isVMlyzFgPhcvc1rOSY+/5850/81Lfl6grUSa5r6+HsJoSQsqLHSb6DLlgCHeW3Un2Ka4TYCJSxXohtDndJxZYSUmiJITJBF9+KV4LCfFPQD0mBiobU/ij5l5Rn0hCwRL4LAqOfOvxtQclD+Ll017mmlHXeNVGqaafWvaeS3OWMvPDmfzzh3+qc0EHsE+G87fST4OFy4bfQ1xoEWWMgW5zQOu9QVmwLpifLv6JdQvXeVQ7SiL9ekcM584T72TeAHXKt/gCcXGib1gsNjIe3Ox3avZD+SZBBEg4/CFsvAkqtyu6f1ulH7gnCo7tOcbT6U+z9inHSQ5KcdnKy5jxzIx2r9vXFuwMaxhHpJ+v4jttIZG3eVWt51tp7dzc7Dzm5Qw37L2B878+3+1xzbXNHPj5ACXbHROEG4s2MvTTKFYMGuB1PT9/QSLjJfhrbB2RNoJrRl7DGX3PEPX6DLBvwnfQ7wYxSDRXiR87SGtIaU0ZFGSLEzmy+Mxdncubo99k3/et3YiGXDiE63deT7dxrhM3lO7pGxrggQdsCmKpVnxAIe9L+HE8lLVPPvAnjpN+HYzl9y3n24WeLUb3HttLdVO1w824BNXtPV3Boix70ZcwmA2c+8W53Lv83lYbcX/Wq3pn0ju8dsJrql4zWBfMxB4TObZPSFQGKxcJqIa23yn7RZO0UTjmfP/rEH/k/0H6M+mc9J53NRjkQBeiIyQqBEOdXf/RiAVGVHrrjYCUmRYcDHq9c3VNRIQtkOiINMmryqOguoAgbZAiW7pNm8Rvd/UbNRqbNkZu1AABAABJREFUhaAU8Agke89pmcLG4bdD6qkNlEAi/TZrX+fOX+5ke4n7DYz0XXZH+n36KSxdCi++2D5oZ7HAk0/CL7+IZ3Tjjc5Jv6Ag2z1LSmyLqshI9YMn2fHZ6DQ6aptrrQRSV6zpZzI7Jur1ekGUQ+dR+ul1eib2mMjJWSdTb6h3fwK2YHKgKv083ah1hK1St25i8923r7q1eZTAnvQrj1zL9pLt1DR5xujWFNaw5d0tGJvaZ3C/M/kdvr3a/fpUCgR0j+7OypVwxx2iVoVeDwsXOrb2dASJ9Fu3TgSGNBpITXW+vvUGej0MGyb+rbZtshLY19iQi7b2ngCmJhOLFyzm9yd+d3pe71N702dWH9n3aTA0MP+z+Yx6c5TD+kfH7T0F1hWsI/zRcCYsmuDVddSce4/WiWyslAjlX27puQa0+qHnBTCvENJmtXo5NNQ2lxcUqHCf5nJYMRsOvOXysGGpw3hv7ns8M+MZp8f0ntmbCXdOQBcs335tzeNr2PnUMvaNvhDduWc5DYiFxYURFOqaEDml7yRuif+VaeYnfBIY02hgVsvjWLZM/E5I8A9xHB1lASzt1zjRAyB9tlAxdDDUVvpJCr/M2Ex1LugAEjms1fqPYLCiYgtnZD3K+YMftD3Xmv1QscXrS5+SfQpjMsag1yl3SuhMCneNxkYw7Nljiwu4jAnufRF+GAkNdkmFmZfA1GUQ3VfR/aV6cPlV+URFidiJu31IQr8Ext48lm7jvVd1AqQOSyX1hPbJL9/v/YkVgwawo9dVBHWCYldSwmFhoW3t56v4TluM7zaeWb1nWe0eJdgnBnmi9pODkOgQ7mm4h9NfO93h+1KMN8gc1WlIv/j41kSfv0i/MRljeO3017hyxJUYzG3siZuOwRexsPWuVuekRbXfo7ir61dTWENDRWvngZCoEJIGJhESpd7G2WyGxx4TluIxMYL8k7vf9CsMVVCf29Gt8Kymn9FoZMWKFRw4cIALL7yQqKgoCgsLiY6OJvLvvvtTiJMePIkwrWejVE5ZDmCzaHME1e09j62D1fNg6MOQfYXt9fXXwcF34OwKCOr4UTc6JJo+8X3YV76PzcWbmZEtsnykhY7cTXVNUQ0b39hIj4k9yJqe5f4EO6SPTlds4yIX21u4CqXWnmrCF0q/zNhMimuLOVp7lKrGKmJCfbfLGHLBEIZc0PoDPPGOEx3W+pMWWDEx0KNHd6cbb41GfC7FxYL0a1tXeW2+yFwbnjqc8GB5EZXaWrFYB/ekHwiLz+++s/3f1/aetbXiR87QP62XIP3WH1lPXXMdEXr/7pok0s+iEUFzOQXY2xZCd4T8fFi8WPy7uRm++QYWLLC9X1Ag/NuDguDf/4axY13fMyVFqGSOHrURImqr/EAQSGcNOIvw4HBMFkGMdSbST84C32Q2MebtMfSN7cvTmU/TPba13fLAgSIDtnt3JxcIQKy8bKWi4wNV6edJ8KTeUE+9od6qrPA3goLglVfEWO9NMLPR2Mj2o9upbKzklOxTFJ0rkX46UwQaiw5jUDVHao7QP0R5JHfLu1v47Z7fiM2MJXNqpvV1i8WCscGIqcm9Y0GUPooeMT3oFduL5d+I18aNg2uuUUamDxokgotSUCglBXr37uYzNedNN8H+/fLmVV9B2lBLWbRyIM3pjY3iJzRUBEbO/eJc0kY6LpZjNpnRaDWKPsucshwsWIgLjSM5ov0EdFzpJ9AtuhvNpmYOVRzCYDJ4nIwQGytsbQNF6efNc9VoNHTv7nyt7DU0Gghz/F3v1k2s1woKVMj6DksX9eHiXderT41MZcEJC1we0+/MfvQ700UxdQco/KuQir8KaB5/JuER4rM0GUxW4rC2uJaDvxwk65QsIlNcL8Ljw+J55kbv6ye5wkknwbvv2lRt/nJQSI3Yw0dnTWRlxcOAXamR6L4wZYnX169srKSguoCeMT09UoeBD0i/qsOAqA+sFtr2W2n+jo117TLjE8QM5I3Da/lrTzIXTkYoUJYOhqQTYfqvfm6MDdK6tSmkgL+OFJEamdrK2jzQkJUl3IIklxm93k1CR/ezICQBIuysoWP6ix+l947LIueGHLpFd+Pj98V3yp3ST6PRcOqzpyq+lyMYGgyYjWaHRENeRRG1YXuINmeqci9fQ1L6FRXZ4i3SWtDX8+29k+91+LpGI9pSUyPWg0oEJs21zexZsofYnrH0mOjahlyj0aAJcvy3Sep6vTEpoOultkVWFmzcKP7tbxU1gNFsJEUH8Ue+hKA5ENUXsq6AxNbJa22VfiCSQXbvtpWesUfGmAxuPXJru9cLNxaSNCCJ4HDH6+NLFl9CQXUBj057lPHdx8v6G/buFWObXg/33ee8VmiHI/uK1pxJC/ztVKR4Cs/NzWXIkCHMmTOHf/zjH5S2PPEnnniC22+/XfUGdnVkn5xN/7me7UqkOkx9E5xn3siS8iuBVg8RmRDcpn5I7BDIOA0MlSrdyHsMTxNFW+xrp0kDq1zrOkO9gZX3r2wnU5aD2S/N5vRXHWemeIKrvrmK65dez7p9+ykpEQtwZ2ohf6DtJGUfgJdDlDhCWlQaWXFZWLDwZ4Fj/+6OgBSEjInRkJCQgNbF7seVPeK2o9sAIbGXi61bhVKse3d5Nd2GDGmtQPFVdmZoqO3aci0+e8b2JDkiGZPFxO5ju33TMBeQSD+zVgTN5WR5uiOwLRZBAphMNiJ06VLbxhBg1y7xe8AA94QftM6ikjKpfEH6AXx2zme8O/ddqw2L/Rgps2Sc3yE9RzlKhE92fMLWo1v5JfcXYsLad4ZLLxWLxSlTVG5kAEEi/QJJ6WexeFbT77uc70j+XzIXfnmhbxomAzqd9wGwfWX7GPPWGM7/0r29TVtYST/0hBpENnRBtWeylkHnDuKsj88i5YTWGbwajYar/rqKue/NdXuNJ055gtx/5nLliCutdRXmz1eung0LEwpKCZmZ7udbbxATI6xHO9LCUNpQl9bLt4YMDbWrdWTXp/ue3peoNMcB6fUvreeJuCco3CC/dqA0Rw9IGuBwc3pc6SeQHpVuTZw5VOlZuQYIHKWfGqSfVqv1ad/l6Aqod2yJKyXwqKL002ih99UQP9z9sT7AuV+eS+aT14FGQ5jexMsDXmbxJYut7x/4+QCLL1nMoV89/96piYiI1mspf5F+MRH1FFQPoLTGB76lwKR3JjHk1SHWxE1P0LamuLfwhdKvbb+V9h0dYn+vC6EqeDzFtdlintPHwLAnYNDdXl/6x/0/8vTap9lSvEXxudK893vTG4x5awyPr3nc6/b4Eva1kkGG+jZlCgz5rxj77GGxQJ0ytUqwLpg+CX0ICw7rkH1Izrc5PB79ODs/29nuvdIaMdGGaWP91yAv4Mre0+fzrQt4WtfP0GBg8cWL2fzOZpfH1RbXcuCnA05tsaW1s96Q2GmUfmCr6xcaqt6cIAfVTdXsLt1N4a2FHL7yZ5J33gvFv4JOD+PeFuW67GB1I6ltr/RzRPo5QmNlI2+OepNvr3LuHPNnwZ+sOLwCo4K6oVKctXdvUe6qs8Hf/VXx3W6++WZGjRpFRUUFYXbf0nnz5vHrrx2XedNZYTJ5XvfNqvRLdJw5aDLZMmpUs/eMHw4z1kKPs1u/3vd6mPSl08zLjsDw1BbSr7g96Sd3Ux2bGcvVm67mpAd8bzfpCo3GRj7c/iGvbniVnXtEwM+Ten5qwv47FRraOggq1xLRESb2EDVn1uSt8aJ18rDprU1sXiS+H4dXHubXe36lKr99Gpq0SI2MNLNnzx6X/dYVSSTZSQ5Jli/RlGvtKcHetgx8a//gSV2/wcnCk3ZHyQ4ftMg1pGCWRSNsDZSQfs5IsFWrYMcO8bk/8ogIONXX2+yNQGREgSD95EBaUB096nvSry2kfm00Bp4yTIJcsshoNvLAygcAuKzPZUQEtY9iRkQIdWxnsHdpi0Zjo6zjpM12Y6Pyugu+QlOTrT8pIf1+OvATFiykR6X7pmF+Qs9YkZ1f3lCu2JozKSKJl2a9xImNjxLW7B3pF987niEXDCEszvtdZ02NbRPW00PxgWTxCdCtm/v5trMjKy6LkttLKLtT/mJJo3Fc1w+goaKBwo3tib2I5AjShqe1sy53hd2lLaRfouOJqzPZnPkSGo2GPvEi4rCvTHmCoARpraaG0q+6SSxaPVH6qWHvaTKZfNd3jQ3w6zTYeLPDt6W6fvn5Kt7TYnGbBfXTgZ94b8t7VDZWOny//EA5H83+iG0fbVN066YWN6CIaB1x2XFEd7Ml3WbPyGbeB/PIOtm9E81H2z7irU1vkV+l5gfTHpLFJ/iPLApKHsm/flnD6sNnt3+z6CdYcRpUe943u0WLL5WS2qttofZ4mVspCBg1Sb+2/Xb0aJg8Gc47T7VbyEf1XmKiRTusRFH/myF1uteXfnvz29z+8+2szl2t+FzpOeqCRHC6I2zmlUAi/QwtTukexwM33QZLMqHes7WmI1tyZyg/UM6bo9/kj2f/8OheEqK7RTPssmEkDmifgVZeXwlARCch/SQVU12dbW6T1gw+nW/t4KikhET6KY0XRCRFcO6X5zL5nskujzu88jAfzvyQ3FWOCWdJ6RdiTOqUpJ+/VX4DXx7IwFcGsufYHkITRsJJP0H3uU6Pn5k9k/un3M+s3raJXRIgOLL3BMhbk8fqx1ZjMYs1k8Vs4aSHT2LQ+YMcHm+xWKwlZpTs7aXvXJRn4nv/4cj3kPtZu5f9vbdVTPqtXr2ae++9F30b09TMzEyOHPF8MfR3xfOZz7Pj/5QHv6saq6yZnM6Ufrt3iz1KREQHeLEHAByRfq5UWI6g1WlJG56GPlKZSXDO0hy+u+47Kg9XKjrPGdbkraHR2Eh6VDpV+0UApiOtPaH1wrHtJsae9FOqFjqxu7DX/D3feW0atfD7E7/zxzNiUZnzXQ5rHl3TusZfC6QNR3Q0NDa6DrRLn4tD0u9oC+mXIu/hWSw20m+4gkRjqa6fRuPbydCTun6DkzqO9LMq/TTylX5xceJzNJkcB+NWt+wZzzpLKP3Obok5fP21jWBRSvpJisHcXDh4UPzbl6Rfs6nZShoEBdnmC09Ie39AbvDk/a3vs798P0nhSZzdNlGlE2ND4QbSn05n+OvyBoXwcKFOg8BR+0nErUYjvzaexWLhpwM/AVgtuzsrokOiiQsVu73cKmXZ0/Fh8fxjzD+YGHotoYYMAI5Ue7f+riutw1Bvm/tKdpaw/qX1VOXJiNC04PBh8TslxfPMVXvSr0cPi9v5trMjSBtEUkQS2rZZ9W4gBXvaBtA+OPkDPp79MZY2C68hFwzh0uWXKiP9jh0n/eRC2odJyZieQE2l38arN1JzVw0n9mhvV+8Oaij9wP1a2XOYYfTLkHW5w3cl0k8VpR/A9gfgs0hodBLhasHCbxZy2ZLL2F++3+kxh347JHtf+Nu9v3Hg5wOt7Mwv/O5CZjxlm/siUyIZevFQIpLdP6wn1z7JVd9exa7SXbLu7ynss+79laxmryJqt+dsKofin6Fmr8fXz4jyfp5Vc7xsMDRYY0Bq1/Sz77dhYaJGrxyHElVhqIHvBjAj6hLAwbq1crsg/z1EfJjYqFc0Kh9speeoDWqpiSWjTERHIj299RrbJelXuQO+7gkH323/Xuo06HM9WJQFqj/d8SmXL7mcrY1C5SNnDxKZEkl1QTXGBvmqH0foPqE7c96ZQ8qQ9or38nrx7CODO77epxzo9bbYWlvSD3w534qkwtjHY0l4MqHd2lKKMSlV+gEMOGsAcVmuP//0Uemc/sbppI9yTAZZlX7GxMCuQ9wGo0bBxIkKEiosFlFv88j3Xt1XItUKawpFrdu0UyCiJUtzy7/hrxtaHX9K9incN/U+pmfZki1c1fQD2PF/O/jt7t+oLhCdPSw+jMn3TKbfGY5FStVN1VZCWSp5IAedhvTb9ThsuMH9cT6GYtLPbDY7ZCYLCgqICvhPPfCQMjSFyFTl3jjSxjItMo3okGiHx6xtcaEYN05F+6KqPbDjEahsL5Xn4Huw4nRQIM31JSR7z31l+6htFrORJ5vqpuomSnfJt14CyF2Vy8bXNmI2qlPT7+cDPwNwStYp7NghHmYgkX5tJ1opu7Ox0RZAkItJPSYBgug8VOFby5r5n87ngm8uAODkx07mms3XkNC3fWqqPennDlIGTFvZe5OxidCgULQarVXt5g4FBcIiVa9X9rzHjhVt7d/ft3UYPCL9kgej1+mpa3Zs1eBLSKSfiZaafjIyNHU627jhiASTSLkTThC/J08W34GqKvj9d/G7sEV4Ibe+jLSg2r0btmwR/25bH1ItbCjcQPgj4UxcNNH6mr26MRAhp6Zfk7GJh1Y9BMAdE+4gPKgT7QbcIDUylaLaIvaX77daPbqCRhN4Fp/2ak2565O9ZXvJr84nRBdinSc6M6RgnZSxrxTR0RDqpdIPxAbtqeSn2LfMpoQ4vPwwy25cRvl+14NATlkO3Z/tzqkfnmol/TIzPW4KAwbYrCu9uU5XhzOl35ibxjD5P5NVWXva23s6ghTk+bvbewJWpZ83pJ8UwFOrnm6kPlJWYlNbqKH08ymCIqDPdaKkhANIpF9xsU3d4hXCMiBpIphckwxSndnSOsf7xbisOO5puMetsgFEPfnVj6xm+0fbndqZ1xbXUn9M/gbLk0x6T3HbbXDOOTDNtyUErUjIv4tpvd6judlWO8+K7nPhnBrI8LzchpX080Lp54mduTNIiUJR+ihiQ2O9v2CgwdwMA/9NWfg8oM269fAn8P1QyP/C48tLCVflDco3OdL4qJFIvwBX+mm1rddSLkk/UyOExIPWQSZexuki2SJCmY3D73m/8+6Wd8lpEEFJOXsQfaSeWwtvZdLdvlvnVzZVAhAdHOuze6iN9DZDtyxBh8UiSHQvkBSeRFVTFY3GRsoaWgdCPLX3lNBU3dQq4bAt4rPjGXnVSGIzYx2+b1/TrzMp/fR6+Ne/YLpc4XLlNth4Exx616v7ZkSLuWzu/81lweIFNBjs1jVHV0Lhd26v4Y70G3PDGBauWyib35DWJrGhsYQHy58gOw3pN+JpGP9BR7dCOek3Y8YMnnvuOev/NRoNtbW13HfffcyePVvNtv0tcP4355M5JVPxeVqNltP6nMbJWSc7fN9isZF+JypP9HSOqu2w7V6ocqDSqcmBo79BbWDUFkiOSCY9Kh0LFrYWbwVaK/3kKtC+v+F7Xhn0Ck01TbLvffLjJ3PrkVuJ7RWrsNWOsfzwcgBGJ0y31vOTqxryFewl6W03MXq9bRBWWtdvQNIATsk6BYPZwIOrHvSukW6QPjLdmmVkNppJHZaKRts+Aq2E9HNGhIUEhbDnhj3U3lVrzTJ0h3XrxO9Bg2yBUDmIiYHXX4eHH5Z/jieQFGlKSL9LTriEurvrePX0V33TKBewkX7y7T3BRuS+807r73NNjY3c7dVL/A4Kghktidi//gp79oh/d+8uPzianS3GKp1ObNZmz1am9FSCzNhMTBYTuVW51kwrd3UMOxpyMqYfWPkAhysPkxqZyrUjr/VPw/yEjKgMovRRGM1Gl6oCe0jjcaCQfp5kvX+XIzYjk3pOIiy4E+3unECy+FSq9CtvKGfF4RVUhm0mrLkn8bruROg9lw9kjMlg8PmDW23QBp07iAW/LiBthOusy9zKXAqqCyioLiC35c/w1NoTxDx3222wcKF31+lMeO7P5zj383NZlbtK9jnOrLKGXTqMMTeMQRess75mNpn5ZuE37PzcQbKeExjNRiuB5UjpZzDYlOzHlX42pd++cs8tBKU1tRr2nt5ALaVfRyEuTuxJzGYoKnJ/vFv0XgjTfoTITJeHSaSfFIRsC41G47A2piNEJEdww94bmHr/1FZkkdlo5qfbf2LN42tY+/Rankp5imN73G+ymoxN1nb5g/TLyIAFC/xUgsLURNC+/zGhxxLAAWmuCwWdTDsBJ5ACpWrYe6qRJJEckcx7c9/j8ZMfl/2d6lQISYBhj9KQeA7QZt2aNlOofGNP8Pjy0h7cG9IPXedQ+oHN4hPcWO4mjIJZmyHzAtXuLdWLP2bIA+TZewKqfK9/vftXlt20zOF71S2kX0xIrNf38Rfakn6yyrcceBvWXwsWM/x2Cmz8p+L7hgSFWOsD51XltXrPG9Jv15e7eDzmcXK+8zxZakK3CQwJnkNEY9+unYAWdwKcvgcG3SvU6yVrBEmvEOmRti9Ryf4PCP0iyqbsPXklnNk6hm80G9lzbE+r/YlE+tXVORZ2JPZLJGNMBjq92Ie8N+09lly+xGmbPE1I6jSJhwmjIX1mR7cCxVVsnn76aWbOnMnAgQNpbGzkwgsvZN++fSQmJvLJJ5/4oo1dGp4WcRyZPpLvLnTOxu/dKwK2YWGt63t5jdRTxIIgvEf79wb+C4bcDwG0APq/s/+P9Kh0esWKiLy0qW5qEtmAcrJC+s3pR0zPGKs3sRxoNBpFNkquUNNUw6Yi4fGYWDsVEPYpHZ3REh0tSAmTyXHmYkKCIEWOHVMevHv85Mfp+VdP7p96vyptdYdf/v0L615Yx53H7iQ4vP33V9pwxMRoyMrKctlvJSLMWZ27tsHqLVvgww/h8ssFuSfBYoGfhJMdkzxIdvPHJOhJTb/QoI4rRCmRfk+M/ILM3vXW7Hx3OO88ePxx2LYNbrgB7r5b2NAdalkbpaS0DpBNmwYffSSOl4jigQPltzMiQhCMZrPva80lhieSEJZAWUMZOWU5DEsdptgG2d9wFDwxmo2YLWb0Oj0NhgY+2SHWIy/PfpnIkEi3/bYzQaPRMCBpAOuPrGdX6S4GJrn/cknfw0Cp0+hMveAKH277EID5A+b7oEX+R88YMTEerjys6LwNhRuY+eFMeoacwJBjW7hq5JXc4IXbaVyvOOZ/0vozjUiOoNe0Xm7PlQIA3WO6c7jFSb2X+9NcYsIE8dti0XapfusMq3JXsXjPYk7KPInJPd2rgMC50k+CxWyxJjDVHKlh89ub0UfqGXSO45oa7c63WFh83mJ2l+62ktP2sKodNAGsCPMjhqUO4+yBZzMuY5zH15DmXW+VfrtKd3Hnz3cyKGkQT5zyhOLz1VD6abU+7Lt/3QCla2DGHxDUfiOk0Qi1X06OcMvo4WC76gu4I/0Aclfn0lTdRN/THJflkKDVaa2uI9JcGRYG2iAtu7/cTURyBBPvnkhDWQMJ/dwXziuuFZl5ep1edtJhp4EuBOYVs/gO8cWtqGgfHKdqt/jOZF0BWl37a7iBmvaeaoyX8WHxLDhhgfcXagOf9lsPIK1bWxFFIfEwbpFX11WF9NN2DqUftCb9PK7pB8L+r2Y/TJKvspRIv5LGfOIRJIHR6H5vW5VXxZb3ttD71N5kjM7wqLn7f9jvsGQLQKglntCm7sTHJ3l07Y6A/bim09n2wC77bekayP8SzO9AfT6EePb3do/pztG6o+RX5TMibYT1dW9Iv+TByQy+YLBLRdiym5dx4IcDXLPlGoLD2ve1uybdBWtgbW3nTVSSjegWe8ztD8D2+0U8Pm6YoktICSwApSawpM9GE95dvOAgEb66qZoBL4vEv8Z7GgkJCiE0VCQSS4nvjmK8ZpOZ/cv202t6L0zNJoxNzl0APSX9OoXSz2IBLOCghIO/51nFd+vWrRtbt27l7rvv5pZbbmH48OE8/vjjbN68mWR/mbd3Iax7YZ1Priup/MaMgWA11yP6WDHAhDhYNQRHBxThBzCxx0Sy4rKsGUOhobYJSi5RMXD+QKY9NI3QGHlkhclg4sBPB6gt9lDr3gZ/FPyByWIiMzaT8sNiYFZCIPgKGo2NRHU00Ure454QByPSRvDmmW+2mpx8iehu0ZiaTBRvcSxZk0i/2FgN0dHRLjPQJCKspsZuc+AEFgu88YYg6R95pLVUfvt2kaUcGuoZ6ecPSH9rSYkgqAIdUkbS6LSxnNTrJKJC5K0URo+GF16Avn3FM333XfG6RPrZb6hAZEENHSqer1TzT6kyV6v1PeEnoV+iWEjuPSZqnnQ2pV9hTSEjXh9BzOMxPLX2KcKCw9hyzRbeOP0NzhpwFhqN+37b2SARfXLr8wSqvafc5JXtR7ez9ehW9Do95w4613cN8yOs9p4KlX6SpWtIkNigqf1MzUYzR9YfaVe7wxHyq0VxkW7R3VVR+tmjK/ZbR0iNFBOpFJiXA2c1/QC+uvgrXh/+uvX/MT1iuLP8TibdI38hEawLZnaf2dw24TaH9QbtA9hd/PHIwgmpJ/D5OZ9z24TbPL6GtJ6urhZBUU9xsOIgS/ct5bfDv3l0vhq1x2T13eZKqNii/OLaYPGlc0D4SejeEsNSpa6f2QDb/gsHXBMNcki/n2//me+ucW+fVZVfRf2xeiwWSztbyCt+v4Ir/7iS/nP6M2fRHFnjoxRUS4tM65rjaWgilnAx8TgkzQ8ugvVXQ41nStykCBEob2ttJxcWS+eogRowc+7Pk2Dzne7XrUaF9UNa4CnpZ/8cO6vSzyXpV7gM9r7g/HOt2g3lG4RqTCa6x4jB+EhtnrXciJw1a11JHSv+u8IrFdjVG6/m6o1XO3zv/LB3OHl7HpMS53l8fX8jzc54IybGtvZy2W/HvwtzCwShc/oeOPFjj+4tkbfSml+CFFP1JKE0sV8i8z+eT8/JzjcN+gg9wRHBBIU6D4ioqaIOWBQus9UVTpsFw//nEYFrT6xtagLN5CWQ2uIxWnsIin5qpSCMC42zumJJdWTB5oDlzOJz4xsb+eSMT9j1xS6uWHMF8z92nqxrsphIi0yjW3Q3RX+LRDQHNOlnqIZPg2HDje3e8vc86xHFGBQUxMUXX8yTTz7JK6+8wsKFCwnraNlRJ8W2D7d5dF55Q7nTgIzFImpJgS1jWjWYGqG5wnndvsqdkPOKfO/MDoC0GczPd32cpyjfX86HMz/kz+f/VOV6tc21ZMdlM6nHJHaLEisdbu0pQVo8Our+EnGg1N6zIzDmhjHcXXc33Sd0d/i+FFiLiDCxfft2h3VNJYSG2rLw7YnlOZ/O4cRFJ/Jnge17sXmz7XtYUwOPPWazzPrxR/F76lQ/WeR4gIQEQUwZjcqe8zN/PMOI10ewaLN32ZpKYZ8xrRQZGXDPPeLf+/eLTYtUz68t6QftfdoDpc86Qv8EUWxwzzHhRSr1685A+uVW5jL5nclsL9lOo7GR7tGiD8eExnDVyKsAMJnc99vOhoGJnZv0UxoAy47P5sN5H/Lfyf/tMkqFab2m8ezMZ7l+1PWKzvMF6Ve6q5RFJy5i86LN5CzN4a2xb7Hh1Q1uz8uvEhNYnLY7jY0iyaydysJDdMV+6wgS6VdUK9+L0Jm9J0BobCiRqZGtMmvD4sKISFIv2ixttgM5gN3ZEB0tkn0sFvkWaI4gkceSHZdSqFF7TFbf3XQLLBsOtYeVXXzksyLD3QWkun6q7PM0QbDnWTjsOlhqrelX77wG/NQHpnLaq45rEdrjh5t/4KnUpzAbze3WrVHpUYqcZ8C/9fz8jqo9UHuQ+HjxmTgk/XotgMnfQLhniaQ9Ynpw05ibFM/VEpqbhSsOqDNm/p73Oz/u/1FRoogcBMSca6yD5nJormjlUNEurLRsOPwiTxnfFtIasqJRmay6qcmW4Dq3/xz+O/m/nNhDzRo6vkHPnlgJN5f2noc/ho03O39/0lcw57BD1YozSGRRQXUBEVHieyVnzZoyNIUr/7iSSXd5nvWs0WjQRzou46FmjU1/wX5tbV/Pz22/DU30+t7S3lpa80vwtqafO0x/dDrXbLrGIUFitpipba6lplYMDl12PdpQBCtmw5a7xP8Tx8CA2z2azyTVOoBOo2v9ue57FZbPhIZC60sajca2R6mx7VEk0q/UyXJn8PmDmXTvJFmOMZcNu4zC2wpZdKaymKBENAc02Ws2QMYciGnvsuLveVaxjuD99993+f6CBerbDXRlzP9UuU2V0Wwk9alUokKi2HX9LlIiW2/sDhwQzHtICIwcqVZLW3DwXfjrOpj2M6Q6qCe4/zXIeUnYgEbLs87zJcrqy3h1w6uU1JXwwqwXAEH67d4NeXluTm5BVV4V3171LQPOHsDIq9x/oOEJ4cx+eTZpI13XwpGLswacxVkDzqKyppkF74jX+vdX5dJeQ47SzxvSb3Xuap7981kGJQ3ioWkPeX4hGXCVRWRf06+mxv0gnZoqgjbFxYIQslgsrMlbQ3lDeas6cosXi98TJ8LWrYJM+t//4LLLbGpdqT5cIEKrFWRYbq74kSv2Plp7lM3Fm9lUtIkrhl/h20baQQqefHLgRRLKg1hwwgJFtbDi40WNvcOHhS2rRPo5srObMAFefVXYCMfEtM7QCzT0TxQDyt6ywFf6mc02wqhBW8L0dyeTV5VHr9hevDjrRab1mubwvK5GHAxIEizy7mO7ZR0faKSf0g13eHA4Fw29yHcN6gAMTRnK0JShis9rMor6wqHBeixYeF83lc+ez2fNFWs8DupGpERQuquU+mP19JjYg1HXjaL/XPcLDSnrN6RRBHa6dxe2Q2qhq/VbR/BE6eeK9Jv9Uuv66sf2HMNisZDYL9FhzWJHWJW7iuLaYkanj6ZXXPsJ7m+RWa0QZouZopoiIvQRxIbGKj5foxEKzvJyQVy4DM66gPQ9kr5XSqGWDaHbvps8RRA2RvWjhRLpp4rST6OBUzdCmOtFnBzSr/epvWXdst+Z/Ujom4AuWNdOFW8xW3go+CGCwoK4p/4eWdeTEgq6JOm35U4o+oGEuFpA79jyOHaI+PEQqZGpPD/reY/Pt7dDViOJ85HVj7Bs/zIWnbmIy4df7v0F7dDhc25QBJy2EyxmolryVqR1f6v5JmkyWDxr69CUofxw0Q8kRyhzKJOeo04H5wyZy7mauR7d398ICRElREpK3OxFT3gEel8NQU4Gfw+scdMi09BpdJgsJoJii6EqQ1ZSi06vo9s4ZcofezRVN5H/Rz4pQ1IcltvpDMrbtkhNFWOIxdKa9AMn/Xb/mxCaAt3OFP+v2gMly6H7fAhV9t2XSL+86tbBU0ll5Snpt+2jbWx+ezPnLT5PtqOahOLaYjKeySAoJoKZVBMZGRi2xKpDFwZjXoeofl5fqm9CX2b1nsWy/csYHaaDDTdD9uXCxa/bPIjqDfq4VuekRqaSV5XXao8ixfucxXjD4sKY9tA0tn24jcaqRkZdMwptkOvno1T51insPUMTYfJXHd0KwAPS7+abW2eAGAwG6uvr0ev1hIeHHyf9FCI2M1bxOQfKD2AwG2gyNlktJ+yxfbv4PXy4mOhVRXR/yF4IYU4m4t5XQ/ppEO75RK0mLFj4z/L/APDY9MeI0EcoVvrp9Dryfs8jY5y8jIqI5AhGXz/ak+a6xOEDeiwWMdB65cmuIqSso0QHSUSu7KfkorS+lMV7FrPt6DYePOnBDrEcsVhsE4sg/dyfk5IiLDslpV9xbTHlDeVoNVoGJIpgvUQcaTSC5Csqgvvugz//hPXrxSYnK0vUbwxkZGUJwu/gQWGDKQeDkwcDsKNkhw9b1hoWiyDgLFj4zx83ATB/4HxFpB/AiBHi2a1bZxtDHCn9QkMFmfvLL4Kk72i3HFeQ7D0lpV8gk372RaM/2vM6eVV5ZMdls+KyFYqtITozBicPZkrPKQxLHSbr+M5O+h2HDZLSL0yvR4OGSs0BGiqPUFhT6HFQNzwhnDuO3YFWJzZlp73iXo0Ctpp+lkqxsMrM9Oj2f2t4Y+/prKafPX679zf2LN7DPQ33oNPLC9q9uuFVPt3xKU+d8pRDy0opwHOc9LNh/mfz+XrP17x62qtcO+paj64RF2cj/TzF0Vqx8PRW6efzYGjWZeJHCWoPw6H3ofs8lySOPelnsaiw/op2XYMPYHqv6bw39z1rEpU3GHbZMEC0vV1f04haSN3Gy1/vnDPwHAYlDSJS3wU7bK8FkDiB6J0iodJl3zFUi1IkfobadshWNW+kZ328U0CjJThYkN0NDWLt2mq+GeU5CRsXFsfM3jMVn2dPFAXyns4R5s6VcVBED/HjDIZqYf8XmQ3xw2XdV6fVkRyRTFFtEbroo0CGon3IygdXUrqzlNNfP53QWPmkUMmOEj469SNOeeoUJtzW2vbMaDbykmUo5v6x3KT/AfD/mOAJ9HoRbysttSXdO4XZCJvvhKg+NtLv6G+w4R8Q2RvSTlF07yEpQ5jVexYj01qLH7xV+tUU1lD4VyGVhypJHdY6UclsNLP8vuVkTc9yqBiTrLR15gg0aDsVgasI+lgRX7fHitOF1fnkxYou1TO2Jy/MeoE+L/ZhgF4LOS9AyhRB+iWNFz9t4GiPIsWh3ZVwyvk2h/0/7PdJXLxT2HsGEBSTfhUOVlP79u3juuuu44477lClUX8nGBoM4G7gbgPJzqt/Yn+HtTakDugTZUnKVPHjDF5m06mNxPBEEsMTOVZ/jJyyHIanDVdM+kWkRHBXzV0dQjjVNdcREhRCkDaIPSIeH1A2geecI4J8jmxk1Qgyz8ieQYguhAMVB9h9bLe1jpU/UVdns/OQO7FIte6KW+bH7SWCie8T34ewYJGuu2SJeG/CBEESpqTAE0/ASy9hrY00c2bgbyx69YLly22qNzmwJ/0sFotf+lZTkwigWDS2ot56B0WL3WHECPjqK2GhbDKJBa8j0hvgoovEd2fOHE9b7R8MTRnKhUMuZFjKMMC2mKuqkldw3Z+QNt36EAsf7hDOA/+d8t+/FeEHoh7cistWyD6+M5N+96+4n9CgUC4fdnmXC3JtKd7CvrJ9TM2c6jCJyxEk0i88RIxfIc3pNAQfsVq4eQqtTkvFoQpie8bKVoRlxmbSaGykqUQEio6TfsrhrdLPEamx8Y2NlO4u5dRnT2XoJUNJH5Uum/ADYZsMIkDgCJ0xS97X6BEt+sDhysMeXyM+Xri1eEP6Fdd5p/TzW0JG4TKo2Q99/yHfLq58A2y/TwSdXew1U1PFuqWxEXJyoF8/Mfc9/rjYQ11yicK2GutFPbjILAh2vBHol9jPmkDlDFvf38oP//yB878+32UdIwkNDTZbSGkO12g0XLf9OkXNT4lM6XJzpxU9zgYgrmX6c9p3fj0ZqvfAPM/kn0drj1LWUEZmbCbhwco6h9pEurdq3oDGwXdBo4NeopNGR4t+UFWlnnW4p7Cf9w5VHKLB2EBGVAYxoTGuT+wsaDwmVD7OFH2NpbDmHOh/m2zSD+Cvq/4iPiye554K4xjKksGP7TlG/h/5hEQrUzHEZsZy2mun0X18+7ItVY1VlGl3QyTERXWu8lTp6YL0a6v0awdtEJy6AZrspFgZpwsSMGGU4vvOyJ7BjOz21lPekn5jbxzLhNsmONxzVOZWsubRNZiaTA5Jv9I6oaoPNoggTJddj5oNguBrBUvLj3IYTCIO9mNTGMzd307Z1xZpkYJQsC9BILcMTOLARJIGJbmM8y1YvIDcqlwen/4447u3Jx0dobnZVg4poJMPS3+H3M+gz3UQ07E2faroYPv06cPjjz/eTgV4HO6xd/FexedIdl7OCBBpwes2C8SXqNwhPIgDAFLWpaRi6dGSxFRYKALa7qDRaBSREh/M+IAPZnyguJ2O8NyfzxH/RDyPrX4s4Or5gSDBpk1zbFeiRpA5Uh/J9CxRHG3JniWeX8gLSO0PC4OQEC39+vVDq3U9dLYj/Y4K0k8iuxobYeVK8Z599l3//vDcc3DVVYIoOtmBg26gQVK5HTok/xwpYaGsoaxVYWBfQlJoaoO9I/0GDhQKaikQk5XlnJhNTIRbbnGsBAwkZMZm8tFZH3HHiSJxJzraRvR5E3z0BaRNtzl2H3lVeUQER3DWgLNcnqPVyuu3XRmdlfQzmAw89+dz3PXrXewr3+f7hvkZCxYv4NwvzmVDofv6eRLak35iQ+Yt6ddU08QLWS/weMzjss/54eIfOPzPw9TkCkm6mqTf36XfSoHbsoYyp7W620JS+plMjt0Hclflsv3D7dQeraX/nP5M/PdERW3KrWoh/WKOk35ykRmbCXhH+knPVRWlnwckj9msDkEhq+8efAc23gRb75Z/4fTZcOomSHOt0gkKgskt5b7eeUcQ44sWCSecxYvl7f1a4dB7sGwYHPOuVnt4UjipJ6S6JODz/8jnncnvcODnA9a+rdeLn+NoA2OdtdibFPNw2neSp0D6qUIB4wHGvT2OQa8MYtvRbYrPVXO8NFvMlNSVAJ6reZ0hIObcHY/Azket/3W6dm08BuuvESShB/hg6wc8tfYpyhvcSFXsYP8cr1t6HYNeGcTXe7726P4BB4sZFqfCmrOdHxPRE8Z/IBy/FCAjOoOw4LBWNRrlYv7H8/nH7n/ITkSTEJUexahrRpEytH0fqWysBEBniiAmsi2ZEtiQ3J8y7MzHnPbbqGxIHGv7f0QPofBzQ/IogUS4OKy7KQNBoUFOn21Mjxiu33k9Y24Y4/B9SemnNya1akuXgqkZvkwUNpz2mLoUJn/t0SVTI1NZe8VaVi38S9QFlOx8S9bAd/0FSdXmePBM6Tf1vqlM+e8Ul8dsKNzAqtxVNBobZf8N0hii09mszwMSZRuEmrKppN1b/p5nVbtbUFAQhYXeBRz+jkgboVyOJyn9JJvAtvAp6Ze/GH6/COpcFMQr/R2+HyIKggYA+ie0Jv0SEwVJZTTaSBl3KFhXwNb3t8o6NjIlkuAwdRYRq/JWUdNcQ6Q+KiCVfq6gVpD5zL7CluCbnG+8bJFnkDLSpL9HL2PnndKyxpS+XztKhY3lkGSRmbx5MxgMQo3br01icFAQnHkmLFzYOTb5Uj27oiLbhsgdwoLD6B0vVq7+sviUvofhUc3W1zwh/YKDYYhdgrmjen6dHRqN/Cwuf0PKJkzT96XotiKWnL9ElmWVnH7bGVHXXGcNALlCZyX91uStoaqpiqTwJMZ3k5cB2Jkg1Us7VCk/a2J89/E8cfITXDz0IsLDIdQg0t+9Jf10eh29Z/Xm9DdOV3SeySSSqMCWVKUWumq/tUdGVAald5RSc1eN7ASzoCCb84CjTffMZ2Zy7dZriUxRHgVpNjVTVCOS9o4r/eRDDdJPbiDFFWqaRTTEExXQwYNibxQe7v0e0m3fHfKgqDe060lB3shBULhQmIQ6sVewwyWXiDX0zp3w5pvw66/idYNBWLQrQtJEGHyfS+s7i8XCsn3LWLR5EfWGeofH9JnVh0uXX+qyVlVtcS2lO0sxNZus87W39lUvr3+ZNze+aQ2SdhlsuAGWnQDNle5JvyH/gbFvCQWMB4gLFTeoaFDOyKs5XpbVl2FqqWWntCadHHT4nDv1exj3rvW/TteuQeGiZtnR5R7d5l+//Is7fr7DalEuB/Y2rQazSCAN1nUu0sgpzM2QfZXrhAptEPS62GPFiqeqMH2EnsKNhXyz8BsqDnmfiSqRfsGm2E63hjn/fHj44fZJ4e36bcUWaHCQVG2xgMHzOrrVTdVWpRjYnqnZLBLaPcHRbUdZ98K6dq/rgnUkDUxyWgpLqp+rNyQSHNw5YmaK0VwOSZMgMlO1S879v7lMWDSBLfkroDpHOBkA6EJA0348m95rOg9MfYB5/edZX5PKwHizVpUgJV7Eh8mvXSWRfpGRAe6I1udamJsPCY6Ja39CMen3zTfftPpZsmQJr732GhdffDEnnniiL9rYpZHQX3mldrlKP5/UfavcBrkfg8FFmk7COOh3s6jtFwCwKv3KBGum0aDY4vPPZ/7k60u/xtjkPkNw3gfzOH/J+R611R5Gs5G1+WsB6KOfTH29ICt7uneECQjYF/eV7DE9wRn9zgBgXcE6RfZXakHaaMTEgNlsZvv27Zjd/EGS0q+kRPztO0t2Ajal37qWtc2YMQE+WclAdLTN3lJJIGVQ0iDA9tn4GtICITzSRvrpNMqLkoOw+JQQ6Co+uTCajewv32+1dVNzQacm7OvbxIfFW5XAriC333Y2vLDuBSIfi+T2n253e2xnJf2+2SuSPU7rexo6Z5ZDnRi9YltIvwr5pN+ItBHceeKdzBswj+hoCFGJ9AsKCeKi7y9iyAXKLNqlDF+NxqZUUgMB22+rdkHVHtUup9PqSAxPdGjX7wrSGt9RXb+I5Aii0qM4sv4Ij0U9xvZPtsu+bkF1ARYshAaFkhTu2HL2eE2/9lCD9JOICzm1Gp1h8zWbqb2r1qMkCakm/KBBIoPaU8jquzH9YcafcFYxBMmMvNYesgWo3CAx0eak8e234re03s7JkXc7K2KHwND7Idq5fadGo+H8L8/nym+uVEQktMWAeQO4s+xO+szuo1rNmv8s/w9Xf3e1rAShToXQFAjrBvrYVn3HE8WJO0gBSSXKMAlqkn6SO0pCWILqhFNAzLnRfVqpkyQbQ4ek31nFrQhCJYgLE18YT5R+kZE2i7zgdrZ7nRS6UBjzqghSu4OpWVEn+/Xgr1y+5HJWN70IKFP6SajKq2Lz25vJWy1vbP3+xu95ecDLooxSG5TVi0BpsDGu05F+oaFwwgmty26067cWC/w4Dv68vP0FlvSAlZ7FZ4e+OpSYx2NaOZOEhNja4qnF5/qX1vPDzT9QfqB1X6zKr6KxyjmTKCWxhBiTuu5aNCwVpn4H/W9p/XpdPux+Cso3Kr5kRpSQiUYV/wDf9RNiHYCE0XDaduh5bqvjJ/WcxH+n/LdVHVRp/1Fba7PZ9AQWi4WKRtEfpTFZDqQxJODr+elCILybGF/bwN/zrGLSb+7cua1+zjrrLO6//36GDh3KokWLfNHG47CD2WJmd6kg/QYkdYDSb/B/4LwmiHEhN9PqYORzrSXlHQiJ9JM+N1BO+o27ZRwXLr1Qsb2AN9hfvp/a5loigiMwFwuyqG9f7zbi/oQ0EFss8hVgjpAelc7o9NFYsPBdznfqNE4BPMm0TUwUz8loFKRJz9ie9IzpyYCkAZjNsKFlvTSm4xM/VIFEfCmp6zckeQj9Evp5pLbzBG2Vfnqd3uNagl2R9Lv3t3vp82Ifnv7jaSBwlX51dWDSNHXdBb4CdI8WE5mUCOQKEunX3CzqW3Y05ATBLBYL3+aIaO0Zfc/wQ6v8Dyvpp0DpZ4/oaJvSz77egj+waPMiuj/bnX/9cicgAmGdZX3iFbbeA98P6nALe2mN7yoxo7m2mebaZsLi5PvfSIkfPWJ6OJ0jpeBOZwuY+RIS6Xe07igNhgaPruFWrSQTEfoIjwgBifQbOtS7+8uC2SCUc6EK1Eo/T4Rfpso+/OyzbYkICQlwRss0opj0kwmp9o2zBEVTs4kVD6xg+8fuSXiNRmMNbEnztydoMDRYg2rpUR1cFE1tDHtc2JwhPiONRiRaOkxuMhth3dWw42GPbiUFJKXPUgnUrJPZpev5NZZCc+vP12XCWmiyx5mznpC49uvWLqf0k4ut/4HPwqBBfpLZ/vL9vLvlXXY2/gJ4Rg71ntmbmw7exAkLTpB1fHB4MEGhQQSFtlf2llRXimNMsb6vXdsRMBtg6APWupitkHGmUI55AKl2pX1Si0bT2uLTE4y5cQyXLr+UmB6tCxV+c8U3PJPxjFPbe5u9Z+Lfby3aUAib74CinxWfWtssOuC9m7+AgXdBpHLLqvBwm7LSm+TwekO9tWSFJ0q/gCf9ag5ArYLgqA+h2OMg4DJu/2ZoNDZyzchr2FO2h6y49tHm5mbbZKpmxrUVGi3IDdJbLFCXq6ok2RNI5OihykOYLWa0Gq1i0s+VFYs9/njmD4yNRib+e6LXBOHeY6LeY5/4fqxYLvj5zmLtCSLzJyJCLJKrq70bmC8achER+gg0+F8WJ/UnJZturRaSk4XlZXExvD79c/bsgf4JsHevsAyNiBD14boCsrJg/Xpldf0eOOkBHjjpAd81qg2sSr8oAxg8s/aUkJ4urDXq69W3s+soZMdlA1jrpgWq0q+uDrb0uoRdmnxmFjzH2G6BkVzSEZDU/rtLd1vnNmcIDRVjstEoxuMkxwIev0FOEGzPsT0cqDiAXqd3WEC+K8ATe8+8qjxK6kpIi0wjOjqDsOLuJOt7kBju3vJOTRytPUpBdQFHa8TG2ydrzkBE/9sgcTzUF8CBt2HwvV5f8qX1L7EydyXXjLyGk7PkFfOVQ/plnpTJNVuucVjXxhnc1fOD1oqH4xCIDY0lOiSa6qZqcqtyrQmHSiDnmfoKJhPsaHFbH6JM7OsZfjsFqnbAqRvBUC3UdK5gsUD2lRAif/IKC4Prrxd1/a6/Xlh7fvONh6TfuoVQXwgnfe/0kLSoNPaW7bXa47aFNkjLqodW0ff0vgy50PHfu+W9LcT0iKHXSb1UsfeUkkHCgsKICYlxc3QHw9QMW/4Fva+CGBcbJFMzaHQiybiF9AkKEvu0qipBmse0/VO1QVD4PUT39WjMVsPeU43xcmDSQN6d867fEib9it3/Ez+n5wjFH25Iv6ZyKFsHccMgTFm5HG9Iv/DwLqj0O7oc9r8BA/8lPk9niBkAGXPALD97MC1KPJsqUxGJeEb6BYcHE9dLvprhlCdOgSccvyeRfnpzLMFd5PG1gk4vnqMjjH7Z48tKiab51a2Dp1FRQmHtqdIvZYjj9Wn/ef3JGJvhNPlsaMpQJifPpfbACfjA6TgwsPspsT4a+mDr12OHwMmrXc+TTmBBkKh/NQHDbPVTMRth32uidmc3W6KtxWIhpyyH/Op8JvWYREhQCBqNiBMVFYn1aqqHOShSEk2wNpiIYPnMbadxG1l/DZRvgHMqO7olykm/4+hYhAeH8+ypzzp9X7KECQryUUeoOQCNxRA/2j35t/5qUah99g6P/b/VQM+Ynuy6fhfZ8dnWoKhE+uUpcGCxWCxYzBa0OueB1U1vbsJitjDpbs+yaOyxt0yQfrWH+7F7t6glNnGi15f1K6KibKSffdFhpbh53M3cPO5m9wf6APa+0UqQmiomw6IiUU/k4EGYPdtWcHbUqNb2DJ0ZUl07JUo/f0N6jmlRqfww6wfroscTaDRwc8d8HX2GPglig72/fD9gI/0CTelXWws1YTupNe+iqqmqo5vTociOzyZEF0KdoY6cshyXQWaNRgRPyssDg/RraBHCuCL9JJXfSZknyarb2Bnhib3ny+tf5sm1T3Lb+NvIiH6KpOoZ3NE3l/lzfdRIJyhrEINDmCWBWhwEWbsqkieKn1Vz4ch3kHmRR5my9libv5Yvdn3BhG4TZJN+ruw9JWg0GlJPULYbn5E9g6/P+9plnzte0689NBoNV424Cp1GR3iwZ/IBe6WfZJmrBNuPbueuX+9icPJgHj/5cUXnHjggxuWICD/VKk45Sdhl/jASovrCjLWuj9do2ge+ZGD8ePEDthrdBQXiO6zo+9tcAU3HXD4YSX3lTHWt0Wpc1tq0mC0svW4pPSf3pNdJvVTJZpdsn9Oi0jx2t/AbNFoo/lnUozrZRa22g2/Drv/B5K8hziZLjYuzkX6ZmQ7OO20HBHs2UUmknzf2nmqoitKj0rl02KXeXygQkTgOsi5rNZ+6JP2O/gZrzoFx70HWAkW38ob0i4wEQ2UXU/pV74HcT6HPda6Py7xQ/CiApIAuNxSRjefkkKHewMFfDxLXK47kwZ6zPCaDjtDmbkTSxZTPPkaPGJHlnF/VmvTztFajPSwWCyXbS0jom2BVZ46+frTLc64eeTXZlVfz3PcQ2UVcl9rh0AdgrG2/9gkKF/sQDyD1x3bQaGHjTdBtbivSD2D0m6Opaa5hzz/20C9R2JzHx9tIP08hjb9xYXGK1iedRunX6xJIdV+Gxh+QFXK+9dZbZV/wmWee8bgxf0dotYodVl3C3trTJ2v7vc9Bzkswr0j4DLtC9pWABvSxPmiIfOi0unZWqJI6p6BA3sa6aFMR7055lyn3T2HCbROcHnfN5muoLlCnaNKuElEzRlvej8REuPtuJ5uYAEZ0tFC6BUodKU9gn02i1WoZMmSIrH6b0pK49OViE0fyhefZ99/b5PBdxdoTbBaXublCSaSUzLRYLD4PRkjfwYTo8Fa+5Mch0Du+NyDqERlMBuLjxUY20Ei/mloz9SEHAFub3UFJv+1MCNIGMaH7BJYfXs5vh35zqyyRSD9PLVjUhJwgWE1TDRHBEV3W2hNsSr+KxgqqGqus9jmuIFmh6HX6Dq3VKJF+epPIEFCb9Au4fms2ibrW8cPF/0c8A0Mf8ZrwA/dkgSP4ShWWHpXOnP5zXB5znPRzjKdmPOXV+dIzbW4Wamiln+/BioMs3bfUanulBNu2id+DBwu3Cm8gq+8OuU/8jhkEel/Uo2iPmBixNj96FPbvF7WRZGPSl24PcWfvCZA8yHmw2mKxcNGyi9AFiz2DGoGt0rpSAFIi5Kt9/Q5Ts9iIa4Oh16UQP8L18RaLqJUTld3qZUlt7tQe14t4hEQSeWLv2VnGyw6fc7ufJX7sIK1xqhzl+CVOgNGvehT8jg9teZ4KlJutlH5lXUzp1+c6yLpCKGhVhqT0K2ssxoKZmhrPvl8VByv49MxPGXvzWE597lSnx5maTax+bDU9JvYga3p7Nmh2+uWs2nY56V2E82vXbw++J1SbY99srwQr/R32vQ4DboM4JROgTemXV91aMaEG6ffXy3+x7MZlXPj9hfSZ1Uf2eV3ean7GH9DsZJFvNkLDEaHMU4AHpj7A9/u+56l+Y4TjwsgXhIJXo4WTV0Jo6/i+RqOhR0wPdpbuJK8qrxXpB97FiRqNjaRGpipen6hV79jnyHKeoOPveVZWaHbz5s2yLhbwGWRdAAcrDhIfFk9saKzD931azw+g+3yRThEsw+swcZz4CUCkpAjlXHMzlJTYCBpniEiJoNu4bkSlux5dgkKDiO8t35PYFfpGjiK5spSk5jE8+1zntM5SOyBZ01SDTut5FrUnaCshb25uJjS0fUHWtpCk7j9yK4UnfMok032E7bie5mZR92jkSB81uAOQkiI2QfX1gkiXS05Pfmcy20u2s/HqjQ7titWEGrVRujLSo9IJDQql0dhIblUuCQmCUAs00q+wtgCztokgTbA161AO5Pbbzobpvaaz/PByfj30K9ePvt7lsR1JENnDYpGn9Hto2kPcM/keTGaTfxrWAYjUR/LSrJfIiM6QbdcVMKRfvRgcgg2+If0gwPpt0Q+w8nQY9RL0/YeqqcUS6eeKLGgLteq/eYIuH2jpIISE2NZSlZXKP1/p+5MSqZzgUbuen+y+2+8meRc8sAjyPhdBfi/KRvTtK0i/nByFpJ8MyCHvm2qaqC2qJaFvQrv3tDotmVMyrf9Xg/STkjMSwtvfL2Cw82Gh7pv4GQy8o/V7xnqharBH3+sFSdEm7uR2TDTWwdGVEJ6hOOA9tttYbh57M+O6KY9rqEn6rc5dTZ2hjuGpwz3q5+4QUHMubtat4enQ51qPrivVaPS0pt8Vw6+guLZY0T4k4KELcX+MxSIseEOS2vdVJ0iJSEGDBqPFSHNQGbW1SR4p2ZMGJXHaq6eRdYrrtVdtcS0r71/J2H+OdUj6SeUFutL6pVW/ba6A2v2gdfA8G4rg8AeQPls56RfTYu/pA6Vf9sxsxt0yjtiesQDs/2E/a/+3lmmPTqPbWMclluoN9dTUhAGawLd59BRB4e3nPwl/XAq5H8O5tRAk/8ucFpVG/i35aHJehG3/BVO97c1kx0513WO6W0k/CRLp503i4ZiMMRTdprw+uqcubH9nyKIYly9fLuvnt99+83V7uxyU1ki86KuLiHsijsW7Fzt83+ekX8pU6H+L8wHIESxmYYHkpBCrP7A6dzWXfX0Zj68Rdjdarc1uUo7FZ3RGNJf8fAlDLnBecyJvTR5Htx91WnBWKeZmXM+Y/d8yUD+rUxJ+YNuoqhGQnP/ZfKIfj+brPV97fzEFsCf9zGYze/fuldVvJdKvLnQPzcElnD03hDNaBCvDhnWtxaZGY7ODUlLXr6yhjMrGSg6UH/BNw+wgfQcNIcW8veltv3+PAh1ajdaqnNtfvl+VDC5fIL9O1BxMDc0iSCtPUqqk33Y2TM8SthHLDy13S465zJj2I+rt9hfu7K5Cg0KJ0HehwdIB/jHmH8ztP5ew4DBZxzsi/V6vnE+v53uxoXCDr5rZDlIwmUYxWKhN+gVcvw1JggG3Q4ad8tRsgvzFwoLHC3hC+kljtNqk3xsb3+CznZ9R3eR44WaxHK/p5wxmi5nCmkJyyjwpGifgDZl7tO4oAKkRyixdjUbYtUv8Ww3ST1bf3fZf2K7ArrMuF0pWKNt/OkA/kaTO3r0KT2w8JhQSx9Y7PURS+jmr6Qew7IZlvNTvJZqq29fEMtQbMJtsn5kayWqS6tPfNV8VoaEQqnaBxm5NV7Vb/CzpCbn/Z3td2mM7YAzc9p2GYlh5miCQFWJq5lSeO/U5zh98vuJz5dQwlosHVz3IrI9m8dOBn7y/WBt06JxbtgF+ngyFP7Z6WVZikweJYecNOo8fL/6RO0+8U/Y59vPe7RNu56kZT5Edn+36pM6CY+ug7C/3x2k0wgY07//cH9uCYF2wdfxpCi7CaBRJ90qh0WgYde0o4rNdJ9dHpkZy7dZrGXezY4JeTbvdQEC7ftv/n3DW0XZKaAAyTodzaiBT+Thmtfesdkz6eeMik9AngZnPzCRpoKg9UVNYw5H1R1yf82QClx0IoUGf1zXXooZqKN8kfjtCxmnQ/1Ywya+vKUGj0YiEq3MqId5OhWAxg6H9g+wR3f7ZS2VgOqIGdaew9zQ2wI/jRF1GB/D3PBsgnjnHIQcWi4XdpbsBW/2ltvA56ecJdj4KK8+A4l86rAn51fm8t/U9vt9nK8Bub/GpBn64+Qfen/6+OhfDVqelsxJ+oK6yJCFMzC67Snd5fzEF8LRYrET61YYKm9Yhaf256iq4/3645Rb12hco8IT0y44TC1KpjpwvIS0QyjR7WfjtQu769S6f37OzwZ70kxZzjY02VVYgoLBJkH49IuRZe3Z1jEofxSVDL+GJk5/AaDa6PDZQlH7S9ykoSCju22Jj4UaW5iz1b6M6EZrNNtJP2vBUmo5wuPIwBdUqLWhkQMqQ1zT4TukXUEgcA8P/BxF2mf0aDWy6FbbeLTbLHsIbpZ+aG26zxcyNy27kvC/Oc6qAaG4WJBEcJ/3aYsmeJWQ8k8Eliy/x+BrePFdPlX7794u5PioKeipzivIchz+B/C8Fmff9CbDzMdfHD30Azq0T5LsX6NtX/M7JUZiL2lQKf10LBV87PWRSz0m8N/c9HjrpIafH9JvTj0n3TMJsbD9e/HLXLzwa/ihV+SIzR41s9gUnLOC3Bb9xy7gA3niMfUvU25MSufY8D0sHQsFiCIoErR5+vwD2PAvfZMOBdxxexi3pF5kFo1+D3ler/ze4gJpKv6O1gtj3hcqvQ9FwRNhnWwytXpbWrXV1tnmnFbbeC59HOg+MO0GfhD7MyJ7hNJ7mCF2NLGqFjTfBqnnyjp25Dmb8qejyksWnIaQE8I4gMjQYrGOkI+j0OlKGphCbGevw/f/tuo7f+0+gMEx94jzgoQuFYM8mlJ4xPZnVexZn9T8Ls916V9qHeKP0k2AymKjMrWT4FcP5d/W/yRid4fC4RmMjjcZGTBgIMsV0qUR6K0r/EDWPD3/i+P3MC2HE0xCijsMcAL9Oh2/alyyQCF+1lX6eolPYexqqob4AGks6uiWATHvPttiwYQOfffYZeXl5NLdJ1fjqq69UadhxtEdRbRFVTVXoNDr6xHcQ6bf5X1D8E8z8y7Y4d4fMi8DcDLHOVXK+hkQuHKiwKYqSWvaNcpUsG9/YSFlOGTOemuHw/akPTqX+WL0qNrfVTdUcLm3EQhKxsZ3XNldarKtRQ2pQ0iCg40g/pRNL9+6Q3K2OhhAxQfZP7I9G07VsPe0hKWcLC+WfI5FM9v3SV5CIDn2oLWB+HK1xzsBzGJI8hLEZYwkNFQGKujoxRnZz7K7hd5SYBEGcFSN/o96VEaQN4v158pJNAoX0s896bztdmswmrv7uajYVbeL5U5/nprEyrd86MQqqC1hXsI7Y0FirctMVWin9Wja6IYZ00EFhjYIB2Ev0iu2F0WzEckQEH7s86ecIGi2M/0DU9dN4nkfpDenX0CAIGzUc2UrqSmg2NaPVaMmIchxskdZEWq2wozwOGzJjMwFRG9dTqKH0U1ofJTdX/O7Xz0f14B1h1iawmARZbqxvrfJyBi/6mITsbPHdragQa5tEuQK4yGyY+oPLvWxmbKb1O+AMA84awICzBjh8L3lQMr1n9Sa6m5is1VD6pUelkx7VCYpX6ewGsPRTofIy6DYXBtwhbDnXLYTqPYDZqQ2hFISUkmbbQaOBPtd41DyT2URJXQnVTdXWmkZyoSbpJ80R0pzRZdBtDpxd0S55JipKPDaLRfSHdvGt0BRImiRUFXJKz3gB++d4qOIQOq2OtMg0gnVdoK7fwH85VPg4RFia4sv/dPFPRIdEc9XlYVQh1hGyx147GJuMPJ32NN0ndOei7y9yeEzl4UrCE8PRRzre5++v3UpF5B9o9HXKG9AZsP8NiOoDKSc5fr9yOxhqIWm8osvGhcXx/UXft3tdDXtPALPJzFtj38JsMHPNlmvQ6rTgZD0i1eLUoCXIFNU1E9Ais2Dow5A0wTfXr9oF1TmQOh2CW4KcaTMhum+7Qx2pPKXkcG8coV5Y9wJf7PqCBScsYOGIhbLP6xRKv7AUmOe/RFx3ULx6/vTTT5kwYQK7d+9m8eLFGAwGdu7cyW+//UbM33LH7z9IKr/s+GxCghwveKVNYryKpH8rmJvEokBJod/IXjD0QQjruAWqZL9QWFNIg0HIDJQGQPct3ce659dhMTtODe17Wl+GXTrM26YC8MWuL7hoSwobsud16kCamkHmgUmiGHFHK/10Onnffb0ebvivUCUlhicGdj0NFSAVxFZC+nWE0i84rIsVX1cRFw65kAdPepDRGaMBdYo0q46mKMIbs+iboCzoIrffdmWoabfsDVxlSy/avIhNRZuICYnhvEHn+bdhHYSlOUs5+/OzefbPZ2Ud32QUVi56nd46LwU3igHYn6Tf9xd9z74b9xFUKQLYvlirBEy/PbocvukDRxwoUJMnihpRXkAK4FY1VblV7EoIC7ORbmpZfOZWCvYnPSrdaSBTyuqNi/MjQdRJ0CtOZEiX1JVQb6h3c7RjeGPbKlk5JkUoU8NJ6yM1+7DbvhscBfpYkaV+5j7X9aEka80a79eKer1NzXhASb6ZTg/pM0UdMR9h5NUjOf/r863Jo9JcHdCBLW/QWAq/zYTiNuVhovvBuHcgZiBog8X35LSdMGsznHEAepzr8HKSM47bvmM2CJJIAfKr80l/Jp1hrw9TdB6oV0PMaDZa+7hSYl8uOnTO1WhA2/r+Wq3t++/Qmr7fjTDtJxFgVYDKxkoWbV7Eaxtek32OPek38o2R9Hyup1+SVv2C7mdB1qXyjm2uEiqkJvmbw5TIFMKCw7wmiIJCghixcAS9Zzl3e/nywi95vtfzTsvt1BorAYgLj/WsEQEIa781G+Gv6yDnJecHrzkX1l2u2r3VIv20Oi0D5g9gwNkD+Ovlv1zae1Y0ikE+xBKLBm3XVPpF94HB9zhPNDLUwur5sOsJz65/+BNYPQ/q7T7nQf+GMa+3O9SV0s+bGNGu0l2szlvNkWrXVq5tcbymn3IoJv0effRRnn32Wb799lv0ej3PP/88e/bs4dxzz6VHjx7uL3AcraBkcSWRHRL54Qj2m3GfYORzcOZ+z3b6piZ5fuE+QEJYAtEhgoE6WHEQsG1u5QZAT3vtNG4vud1h1ompWbmfvCvsOSYsIcOaux+392yB9L3fX77fGvT0NSwW2yImIkL01yFDhsjut3vLxHPsn9jfV00MGKS1JP4VF8u3TPKX0s9otG26g/THlX5yIWVhHjvWse2wR58j9zFtxwGuGn6t7HOU9tvOBovFwq7SXby0/iWXY2MgKv3a4q3NbwFw7+R7u559lRNIRMGhSnneyOcNOo//TP4PI9NGWp+prs7/pJ8EKRCn9loloPqtoVaojIKdsCLGBjj8saIgmD2SwpM4dscxGu5pkF2rVKNRv65fbpUg/XrGOPd4lPYZCV07j8kjxIbGEhMiviMSgar4GrHityfPtK5ZRKWV1m9Tu0ajrL5bsQ1qD8q7YPkGYa1ZpI4dW/fu4vcRZXEmsbhtLHG5yP3pwE8s2rzIqkRodwmzhU/O+IQfb/3R4fsSTCbbc/GG9Htn8zu8vuH1Dpkb3OLYn1CyHBqPuj9WShzW6pw6DclSyZasgs+i4LCyOqxxoeLikq2cXFgs6pF+pXWlWLCg1Wh9UqOxQ+fcnJdFXTkHkOI1atajLqsv48pvruT2n26Xdbx9HbqICDCYHSSQuiOS7ccNQ41QXHVGHPkWfp4AJSsVn6oGQTTjqRmMvXGs0/cHnTuIMTeNceq8VWeqBCA+PJBqIXmOdv12+nIYdLfzEwbdK348RHVTNVWNts6oFukHMPmeyYy9aSw/3PwDG151XqNcml/15lhAHRV1p0NQOBT9KBR7nqDHOTD+Q1mJTP0T+3P/lPv57+T/Wl+T9h/elIGRygjEhylTK3UKe8/qfXDoI2hwXOPZ3/OsYtLvwIEDnHbaaQDo9Xrq6urQaDTccsstvPHGG6o3sKvDWRaKI+w+JpR+AxId24KAbaEbkETRyjPg12kiQ8jP0Gg0VlWRRPpJwTK5i8iotCjC4sIcLiKkrCKTQR3yb2+ZqDAf0dgvMJ+lTKgZZE6PSicmJAaTxUROWY73F5SB+nrbGj0yUvTX6upq2f1WIm/7KVQldUYkJ4NOBwaDfJJIUuAeKD+gaCxUCmlxoNGANljs2rqEHYvKsFgsHKo4xM8HfsZoNlpJv9LSjm2XBKNRLC5B2QJfab/tjJj23jRuXHYjfxY4r7Ohpt2yN3BG+uVW5rL+yHo0aLh46MX+b1gHoVesIP0OVx6W9R09b/B5VkWutOEJaRZZF/4O7BqNtvHVGws6RwioftvtDDhjr1D1OcKh92DtRU6Dlu6g0WhICE9Aq9C+UFofqlVTQyKqesY6J/2krN7jpJ9jeGvx6Y2954arN1B/dz2TekxSdJ6ntaudQVbf/WUSrG+xWiz6GXY87JxMSxwLU5ZCxumqtE+yK1dc033DDfBVCjQ5X+Qu/GYhV35zpdN9ikar4djeY1Tnt94YbX1/K5+f8zmVuZWAjfAD757Lo2se5dql11r3vgGFbmfA3ELoLrOWmBtIfae2VuxFHCKqL6RMFbaQChAVEmUdn50Ruo5gv4/0thacZN+bFJ6ETqt+wLDD5tzGUtG39juOI0rznMN4TV2+qOt3VBkBJQWZ6wx1shKJ7ftjeDgYTAbmR0LigReFumrbfbBsmLCibYv6I/DLSa0VOWsvgtVne1ULWDXUHoQlvWDvi/KOTxwn6hvHDJZ9iz/y/+Cyry9jU+TDgDr7EGff03H/HMeU/0xxel69uRKAxIhY7xsRAGjVb7VBkDwZ4l3Ukul1EfTybI91/dLriXk8hhfX274r0vyk1t4yODyYS365hDE3jnF6TGVjpTjWGNeqDV0KaxfAChdrHo0Wzq6E8e95dv24oeK7YG+LfGSpuG9dXqtDUyJTuG/qfVw01GapGxpqm9M83YNIik0lpJ/BYIsFBTTpd/RX+ONiqNrp8G1/z7OKSb+4uDhqWnp1RkYGO3bsAKCyspL6es+sTP7OMJvlT/bulH4Wi83H3mf2nke+g4MeDi59roXB/xXWoIYaIUn24xfeSjC0qIqUZo5ZzBZKd5VSurt9BDyhbwKpw1PRBauzCN97TJB+kU39OrW9p5p2chqNxu8Wn1IgRK8XP2azmYMHD8rutz1iejCt1zTGZjjPSOsq0OkgtSURV67FZ8+YnvRP7M+UzCnUNquQIuYE0kI0IgJMLUXijyv92sOChYGvDGTGhzPIq8qz1j0NFKVfba1tvlBC+intt50NGo2Gk3qJ2g2rclc5PS7QlX5f7PoCgMk9J3e9ejUu0COmBxo01BvqKalTVvA7OFhsvEIM/lX6bSraRM/nejL7Q5EEqNGov/nqVP02bQaMfUtsov0ItZV+knVPj2jnzi3SfHCc9HMMifSTq9xtC2+faVhwmOKkJntHCzUgq+8Ouhd6XSb+nfcZbPsPNDuJHOnjIGM2RKjjKCTVoFas9EueDL2vBotzC960KJGAUVTrOLsb4Ia9N3DO5+e0eq1kRwm7F+8mJFp49tqvW71JCJfsIH2hDFMFoYmt6/l5gYgICGoRATqt6xeWCif9IGrIKYBWo7Wq/SR1ghxIZFFQkNhHegNf1/PrsDk3OBqmr4B+jms4u1Q/N1fAzkegZIWiW8aExqBpsW6SAs+uID3HsDBhOWowGzgrEmL2PQ9YhAVtcAw0OfhuhCZD7QFotnMCCEuDAbdBXa4I7Oe8oqj9qsLUCPp4+f0wqjcMuN1h7S9nOFJzhPe2vsch3TLAe1XY0uuX8sZI5WKTJmMTBoQsKSkq1rtGBAha9VtTI5jVdR+zR3JEMgD5VbbabtLaXw2lHwgL16zpWaSNcF47UuqzOoMYk7uk0s9QBYZK18fIdAaRjeo9QgVfL29x5K3FpzSXxoXJV91KayONxvtEGp8ifRZM+hJiT3D4tr/nWdmkn0TuTZ48mZ9//hmAc845h5tvvpmrrrqKCy64gOnTp/umlccBwGXDLuO6UdcxKn2Uw/dra0XWNfhQ6bfnGdh0q2fndj9L1GwIjoTyjZD/FRS2LwjrK2THZaNBY90AKVX6NVQ08MqgV1jz6Jp2701/dDrnfaVO/SGDyWAlJiO7iNKvthbMZmGNkeuZ4xEAc/rNYeHwhXSP6a5OA93A2+znK4Zfwa8LfuWqkVep16gAhtK6fsG6YHb/YzfLLlpGVIjv0nWkPh4VBc2m4/aezqDVaK2K6H1l+wLO3vOHfb/y0wlJbOtzIVrFKUtdG1JiwabiTU6PsSf9OlI85czq6rNdnwFw7iDH9Xq6KkKCQsiIFlFoOUTB/vL97C7dbbXyi4qCsOZuZET0pFt0N5+2VcLR2qPkVeVxpIVkjI6m6/bJxmOw8VbXKr7ILMi+EsI9//wXbV7E/M/mW8lvOVCb9MuvFoEcV2us40o/15CUu54qq7xR+nkKtZV+sjDwDpFlDjDgDlGvzT7j3B5NZapOWvZKv+LiYh5+6CFGDxhA34wMRg8YwMMPPURxcXH7E3ueJ+rdhDkPRqZFivckgsYRHDnGnPLkKfy76t+ExYUB6tTzM5qNVkVEwJF+dblC2VXnxaawDTQa3/YfKTAphySSoJa1J8CQ5CG8O+dd7p7kwrqvM0IXAilTIM5xcNSl0i+6H5y2C/rfouiWWo3W+jzlkLj29fzMFjNmi5nbS6Fi2hrQBEG/m2HG7xDhYO7UBsOZB4Q6TsKY10UCQWgKlP2pSr1SjxEzEGZthN6+i1VI42KtRiRDeE0QaYQirKmmtUpz/Uvref/k96nKdxzck8ZDLBqS1LanCATsfQH+L0TEWZ2h4BtRo7ptLVUZ6B4tvt/SWhFs64b6ehHr8wfSItOY03cusVUTW7WhS2HKEjilfcy5FWoPitICzZXKr7/1P7A4HRrs1ip9rodz64Satw1yK3P55eAvrazrpT2Ip0o/T+w97derAV1XPKKn4D5CldXY9hVkb9GHDh3K2LFjGTJkCOecI7LT7rnnHm699VaOHj3K/Pnzefvtt33W0OMQpN8rp73itD6YtMCNjBTZ3z7B8P/BJPkBCaeI6gsjnoOUk7y/lkzcPelu6u+p5+FpwlpAUtA1N0OTjBJx4QnhnPTwSQy52ElBVZVwqPIQRrMRnTmM0OZunZr0kzarFotYML/7LtxwA/zlYWnHf038F2+e+SYTuk9QrY2u0CGBkE4MpaSfvyBlBUVHw7Re0/jy3C/594n/7thGBSikOov7y/dblX6BYu+ZU7qf5uBjWII72J8yADEibQQgFFjOIO1vDQZ5c56vIAXBwsJsr9U111HVWIUGDWcNOKtjGtaBkIiCQxXuSb/zvzifga8MZGWusLOKioKoxkEsOekw31/kn0SqsgbB/ERqBfPTmR0J3KJsPex9Fiqc9y0rvMiw3lq8la92f8XGQhfBmjZQ297z0emPsuT8JczqPcvpMcdJP9eYkT2D28ffzilZp3h0vkRaVFfbEjnlIK8qj9M/Pp1/LP2H4nuqXdNPMaL7QtwwERxvC7MRFmfA7+okVoJtrbp189P07NaNNY8+yj/37OHVwkL+uWcPqx99lJ7duvHsM88ovrakwiqqca70qy6oZtNbmyjLaZ0er4+wJaOpUbNGCqhp0FhVagGDoyuFvWuZ87pNnkAW6Vf6B6yaB2XKNqPSZ6jE3tOeLPIWGdEZXDrs0q6XGNVU5tLmUlpfOFRv6kIgZoDzhAEXkALNSki/yEiRnA1QZAJt7GARedZobeNX4TKoPSS+23ueE7X+HI1tIOpynXkYRiofazoUa86FX+WLPSQFdJW5CAsWr0m/2S/N5oo1VxASFdLq9ZqiGo6sO2JNnmiLBmMDkaZuhBrSiYzogplqkb2EDXaYiwQ0XSgEhblUrDuDlBAmuUKAbd0gxfr8gelZ03lv9mL6Ft0HBLjiy5fIXyysgit3KD83JAEiMluPTUFhYkxywKbd/vPtnPLBKSzZu8T6mrQP8CfpJ8X0AtraEzo2u9oBZGtCV65cyTvvvMNjjz3GI488wvz581m4cCH//vfxwGmgQFrgxvlyXe/KI1oJwtOh/83qXEsmokNaLwhDQwU5ajCI7LHkZPfXmHzP5HavbftoG/lr85l631Qikr1f1UvWnhGN/dCg7dSkX1CQmIjr60UAY1NLzGzrVhg9umPbJgfSxGIfCAkNlWd/0WBooNnUTExoV46GtoY3pJ/BZPBZnT170q9nbE+X9Yr+7ugT3weAfeX7mNNSriFQlH77yvYBEK/prfhcuf22s2JY6jBAbMSO1R9zmNUfEmKb86qrxRzYEXBk7xmhj2D3P3ZzsOLg38raU0KvuF6szlttVfm7Qlu1spo22nJRVi+C1RE+Jv0Cot+mnwqn7xH2V67w1/Vw6EM4u9wjyx1J7VlQI7/QmJRl69TKTiEGJg10WkJAwnHSzzVm9ZnFrD7OSVN3iI4Wdo4mk9ibyP2cC6oLWLpvqTWBQAl8keDmsu+aDfDLFEibBUP+I14z1IogpD629bGmeui1QJCCqrUNSoqf5sjee1hlNjHW1Jqsv6ixkXXAGXcLRdUtt7Y43JiaYf1VopbVwDscXltStLiy9yzZWcK3V33L7Fdmk9A3gaPbj1K8pZg+s/sQniAmRjWUfpKzTVxYnE9qwHmF9Fkw5TtIUHczKIv0M9XBkW+h2zxF91eiDJOgJunnD3TInLtqjqghNTfP4dtSHMTpPNd4DIw1gvBQAE9Iv/BwYe0JEKuFYEMF2O/z6/LE35MwTig9Dn8EaTMFMekMwR2cWVz8G1Rug6zL2o+/TqFp+ZEHaVw0WBox6qqorZV7Hyd3dyLxmf7IdKY9NA2N1vH7mbGZnJ2fz7FjEH6pV00IKFj7bY9zxI8rpM2AtG0e3ceR0i8oSMypjY1iLeEvMsbectcbC+yARH0B5H8NaacINbMzZJwubM+jHQuCXKL/P8WPPYx1ULVLuBm0cS6RbP/tCV9vlH5mi5lIfSSNxkZFSUlqJET5BSvPhIrNME9p8WjfQHaKw6RJk1i0aBFFRUW8+OKLHD58mClTptC3b1+eeOIJxzYYx+EWOpmj1L6yfWwo3OCy7pXU4XxK+qmN5ko48E6H3FqjUV7XT4J98c2DPx9kw6sbCA5Xh7DoHtOdq4feTHr5eeh0nWej4AySuuToURsZdMizUicANBob2Vq81Zpp50u0nVh0Oh39+/eX1W+X7V9G7BOxnPnJmT5sYWAhrcXxqMh5rKMdPtz2IQlPJnDxYs+KSsuBGsGTvwvslX6SvWd9vf+y91zhULUg/VKC+ig6T0m/7ayIDom2ErabizY7PEajUW5r7Qs4C4JpNBpr7d2/G64ZeQ1fnfsVC05Y4PZYifQL0YkMZ7XraciBFCQLNfuO9AuYfqvRik23O4uWiF6QPEXU4fAAGVGC9DtSLb/QmLTeV0vpJwfHST/fwtNx2pvabWqTfm77rtkobB2bWmwEag7A51Gw87H2xwZHw9g3oO/16jQOYem5aeO/+MHchLOK22OBb5ua+Pedd9piHNpgKPgaSp1bbkmKFlf2nhljMrjg2wsYME8QATs/28nXC76mKtf2wNXIZg/oen6hSZBxmqh3piJkkX5Jk+GcashyP9/aY06/Odw89mb6JboIwraBmqTfysMr+WH/DxytPer9xRygw+bclOkuiQqX9p4Av54Ey09VfFtPSL+ICNBpdNww+gZW9OtJ2NI2de0iesCYN2Ds2+L31KWuCT8JRT/DbzOgarfSP8N75H8Jm24BU4P8cyb+H0z/RfbhYcFhxISIhWJjcJF1fPMGW97dwsoHV7Z73RnhJ0FNy91AgD/7raT0q26qpqrR1iGltYO/9iHNpmZqaiyt7t2lUL4RNt4Ix/50fVx0PzF2hqo0x1ftgh/HiGSFNnCk8vSmpp9Wo+XIrUdovKdR0RrFkSAjIBE72KFNqgR/z7OKdc0RERFcfvnlrFy5kpycHM455xxefvllevTowZln/n2C22pBbhHHl/96mdFvjua+5fc5PUbKgPIp6ff9MFg6SL3rbfk3rLsCWoK5vsZ1313HlHenUFAtWHf7GkdyUL6/nBd6v8DqR1dbX5vzzhxuybsFfaQ6NcKGpQ7jjiHP0bv438TEBLhfsQxIn/HWrTal86FDnqmeLRYLqU+lMuz1YbIUEd6ibSDEbDZTVlYmq99uKd4CQFJ4YHg5+wOS0q+oSL6ve5Q+ivKGcnLKcnzWLvvgyY6SHXyy/RM2FKprKdRVIJF++8r3ERpq++4Hgtovr1bUvEgLUUb6Kem3nRmSxefGIuf2gNJ4rMaG21M0tMQVJKXfluItlNSVdFyDAgATuk9g3oB59Ijp4fbYtko/6Zm+dOBmej7Xk093fOqzdkqQ7D2Djb4j/QKm35asAoOMDjPwDpj6rbDM8QCS0u9ITceQflWNVTzzxzN8tvMzp8c0NNj673HSzzmKa4tZk7fGZaKmKyjdm4CN4EmKULbmtFhsa121gqBu+25QGMw7AqNeEP8Pz4DMS9Rzk3GDt958kylBwU4JPwljgal6PW+/9ZZ4QaOBeYWi1o4TWO09XSj9wuLC6Ht6XyJTxQJr1LWjmP/JfJKH2AgwNUg/SZEdkKSfynUaJUgEkUvST6cXFmYKcf3o63nu1OcY1815IK8tHDkbeIr7V97PrI9m8euhX72/mAN02Jw79AEY8bTTt13aewJkXyF+FOK/k//Ljxf/yMzsmW6PtSf9woLDeHH2i5ww/C40va9rf3DWZRDdR4xz6TJV3xqtSCao2iX/D1ALQ+6Hab+I+oI+hJQQ0RRcpAo5tOOTHax5bA1mo/i+VuZWsu7FdVQcdN75LZb2e5DOjlb99s8rIOcV9yftfwMOfaD4XpH6SKsqy1FdP3+Rfud+fi6D/y+E/IT3ugx52wpJk2Dar0KVKQeelBYo+AZ2PAImu3ofEZkw7HFIntrucGl/qpbST4JGo3Gq3HUEe/eugMawx1yWRPP3POuVmXHv3r25++67uffee4mKimLp0qVqtetvA4vMBe/uYyLzZ0CS82whaYErdUCfIHkKJKtYh2/QPWKxEa0siOspfjv8G6tyV1lt4pRm00ZlRBEUGtSq7oJGoyG6m7ojj7Sw7czWnhKkDetmO/FJTY1nRdY1Gg294oR9h/QMfYm2dU4sFgv5+fmy+q1E+km2e38HJCUJmwejUT5JNDxtOCDIuHpDvU/aZb9A+Gr3V1z41YW8ufFNn9yrs6NPghiLD1WI2qKS2q+jST+T2URRkyD6u0cos/dU0m87M24Zdws/X/Iz14y8xukxngST1YZ9Tb8mYxPnfn4ufV/sy+rc1a5PPA6gPeknzU/VTVXkVeXJqgvoLSTST9csFpy+IP0Cot82Vwkbwo03+fxW3aKFlc6R6iOy/2ZpvV9dLewgvcHBioPc9tNt3PyDc+t9KZs3PLzj7IE7A8a9NY5J70yyrgOVQupPnpB+Sgme+nob96JW5rTivqsLhQnvQ08Htcq23A3rr1WVIFry8cdcbmiUdeyChga+/vhj2wtBriOMo9NH88G8D3h25rNur91QIaLP0RnRDD5/MLpgW+a3GqTf5J6TWX7pcp48+UnPL+ILGOvhyySR9KsyZCn9AMo3iaCnj6Gm0k9S+PnKAj0g5lwHcKv0638LDPyX4uuO7z6eGdkzrGSUKzh8jn2ugVHPK76vQ6ROhzMPQY/56lxPCUKTxP01CsLCdXmw+2mo2CL7FMniszmoVBVyaPbLs/ln3j/RBol2567K5YebfqBwo/MaIx9s+ow1/SawL+3hLkMWWfutoR4Ovgulv7s/accjot6kB7hwyIVcM/IawoJsdRP9TfpVNFZgtBjQmcMCX/HlCULiIXWasNl0h6VD4Df59TWtyP8Stt3burZjaJIYSxPbp0RJpJ892Ssl/3mi9PMUvrCj7wj4e571mPRbtWoVl112Gampqdxxxx2cddZZ/P67jEHmODzCrlKR+eOq1oZf7D1HPQ+jX1LvehHdYUiLerH+CBz5Tr1rO0B2nLAOk1RiSu09g8OCuX7H9Yz7p8jyqzhUwf4f99NU0+TmTPnYWLiRvGNi9OwKpJ8UZG5r6empxadkYbe/fL8XrZIHbyTkm4sFyymRWn8HaLWQ2rIXlVvXr3t0d9Kj0jGajT5T39nbe0q2sFLA/Dhao1t0N+6ddC9vnPEGJrOJpBbRQGlpx7brSM0RjJZmNOZguke5V0P9HTG221hOzjrZWnfGEQKB9GtqmS5DQ+H5dc+zr3wfYcFhf6sECUdYsmcJT/7+ZCvLHEdwpvSLMLbPwvQVksOT6RPfB32jUKf5qqZfh0OjgWFPQPez5R2/7X7Y5VmAXbL3rDPUUdUkb1EaHS3mXYvFe8vewhoxaadHpTs95ri1pzxINsUHKw56dL5X9p5hykg/KZit14sfv8BYJ+pflju2om6FkhXiR0Xbk6rqauTSJilAlf2DqD8CBd+KhAAHSItK4+KhFzOxx0SX1/360q95Mv5JVj60koby9rZ6apB+CeEJTM2cyok9TvT8Ir6AsR6yFwqbTZUhm/Tb9h9Yc64iMtlgMlBUU6TIgllN0k+yjE2J8K0iy6849AGsOR/q8p0eIsVCmppE3bCOgP1zNJqNFNcWK6rtKAthLc+18Zg84kYNmJoEgac0AF2fD5tvF/UAZeLzcz4nZ0ED6RXnqUIOxfeOJyLJ1rH6nt6Xi3+6mMypmU7P2XfsIBWRf9AQepBgdaryBA6CwuC8Bhj9svtjJ30JJ3rmCvLS7Jd47fTXWpVjkOJk/nKRqWgQA3ywKa7LkLetYKgBi0wlWPxIiB2i/B5DH4ZZm0EX5v5YbKRfUU2RdR8qJYaXeSDcX5O3hknvTOLWH29VdF6nKNljNsKGmyDPudLP31BE+hUWFvLoo4/St29fpk6dyv79+3nhhRcoLCzkzTffZNw4+XYHxyEf1U3VVjvKAYnulX6dlihafy2smgcG30Ujs+KyADhQ3pr08zQAuuvzXXx06kcUb1GnpmW9oZ5Rb45i/rpEmnVlnfdZ2qGt/FpaGHhL+u0r973Sz9NskmP1x6x9dmjKUJVbFdiQLD7lkn4ajYYJ3ScAsDZ/rU/aZB88kRYqwbquttpXB1qNloemPcRlwy4jJCgkYJR+TcYm+gVPJ752ElGRXbc2n68RSKRfjaWYh1Y9BMDj0x8nKiSQV/C+xz++/wf/+uVfVmcHZ2hL+kkbn7DmFtKv2vek3/Oznifnxhx61AhlTpcl/YKjYeCdovaUHOR9Cgc9q1MdFhxGXGgcep1ett2tVmtb83tr8SnZER4n/byHNcGw3DMbek/G6dJ6kZmjVOnXIVnTjaXwxyWQa6eg2/caLJ8tgiX2mLEWZqqbEGYyGpG7azsKxNgPcHmfw6ozFSlcHKHHpB5EJEew4r8rWP1Ye5W79OwD3sLKE4QmijqN2ZerfmnZpF//W2D8+/IDq8AH2z4g/Zl0rvr2KtnnqFU/rMnYREWj+KN8pfTrEFRshbzPXCpoQ0JsCQkOLT4LlsCvJ0PlTkW33l++n0WbF/FdjvuEc8kSMiwMDlceJu3pNF5/Jw3+uFTRPWXh1ymwen77sdAXKN8ES3rC7v8pOy/2BDhlDWTJ//sTwhNIiBEWAbW13ou3LRYLVXlVHN0uFLBhcWFkn5Ldighsi9Ja0YfCtLGdvnyOQ+hCQB/r/riEUaq6rPm7trg0FgYbYzu94sshls+EJZnyjh3/Lox6Ufk9IrpD3LD2Ct9fpoqYfBskhScRogvBgsWa+BIfL/KxjEblDm65lbmsyVvDtqPbZJ9TUQGrW5ZL6c63Kh2P5grIeRGKfWPF7QmC5B44a9YsfvnlFxITE1mwYAFXXHEF/frJL2R8HJ5jz7E9gFjkOcvgt1hsQfZEZfs9Zdh6L4QkQX/n9j8eo//NkHkBaGR/LRXDW6UfQMXBCta9sI7+c/vT78x+BIUFkTpMnQW4pF4L18ShNyV0iUBa2w3rlCmwdCkcPuzZ9exrjvkajoIhUTJSS7YWbwXE9y06pCvu2J1DKekHMKHbBL7Y9YXPSD/74ElzbeuA+XG4RqCQfn0S+nBl8C+syoGIKcrPl9NvuwKW7VvGytyVnDfoPIcqY+ljCATSb3XpYmqbaxmRNoJLTrik4xoUIOib0JcjNUfIKctxWTPoprE3MbR6LZkbFsCkL4lqIWmC63tApH+UfhKktZOv1iqdrt+e9JNXdXEO3HSA2NBYRfUt4uIE4eeJZbo9JKWfZMHlCMdJP3mwJhh6WHvak71JTZPIbvKU9FM7W95l3w1NgkmLIcqmFKB6D5Qsh6bS9pZWwepF9Z59+mmOlpXxHnCRjOPfDwtj7oUX2l5ImwljF0F0X6fn/HzgZ/Kq8jij3xkkRyQ7PGbEwhEMv3I4Oz7dQZ9Z7YOvapCx3+z9hsKaQqZmTqV/Yn/PL9SJIFkeV1aK2IjToTT1ZOXXDhMXlwLOcqBW/5ISQYK1wS7dHLyF3+fcEU+Jmn4uSD+NRiS3lJSIMTG1bcil6RiU/yXGDgVYm7+WK7+5khnZMzi97+kuj5XWrSEhNseYE0PNvnGoGnQvWEyAH+zf9HHQ90ZRQ0wJgiMhSbmC2FYuRRDi3vaLV4e+SsqQFOa8Owez0UxiP9fzX3l9pWiHLta7GwcYoqKiRCKKxgxxw0HrJpZqsYi6qsHRosapQlQ3VVPbXGtNEvO7vaed0q9Lkn4p08HoY9mkoUaMM21JYkM1GNs/SI1Gw5OnPEmUPorYUHGOTif2A8eOCUcoJSXGpHlUyXz21ltCdZ2dDRNdmyl0LEISYG6BTzkNpZCt9AsODuaLL76goKCAJ5544jjhpxJ0OveKhd2lLfX8XKj8Dh8WHU6vB58+mv1vCA9gXyD1ZMi80KPi2nIhSdGljbgn2bSGegPrnl+HxWwhsX8iY28cS0hUiCrtk+rUJWjEBtCnVq1+gj3pFxMDI0eKf3tK+kk1x/xR06/tplun05Gdne223/4drT0lpLXEahSRfi1Kvz8K/vCJx7V9TT+D+bi9pzuU1Zfx68FfWZu/NmDsPcFzmyS5/bYr4K3Nb/HE70/w6yHH2WWBpPTbVb0egNP6nIZWSS2RLoq+CSKInFOW4/K4B096kLOzphBcdxAsRusz1VT7z95Tgi9Jv4Dot7ufhp8mQL3MCS2ih8i09hBxYXGKCD+wbbK9ralx3N5TPbRNMFQKT+w9vzj3CxruaVCcQOELpZ/bvhsUAd3ntrakGvYEnFvfmvAr+0tYaRrVqfdcXFzMv//1Lz4zmVgJrHNz/DpgRXMzVy5caHsxZoBQqLmotXPzDzez8NuFbD+63eX1NRoNQy4YQmhs+wKZaij93tj4BtctvY7f8wKs/MqWu2D9darWaZQgKZ+bm20qO7UQFyo25VLAWQ6kNoR7GdqQrD2TI5J9tl7qsDnXTZ1MsD1Xh0q/rCvgnCpImarothKJK8em096WXtpHnlueAPN9kBGZeQH0uhi0fnCkiekPo16ApPHKzzUbhd2xTORX5XPtsivYki2SKLwliDQaDVMfmMqo60ax5rE1vNz/Zcr3u36WUt+N1HWBAFsLrP12z5Pw41gwyfDA3fM0fJUEFZsU3++T7Z8Q83gMF391sfU1f9p7GkwG6gwiIBBs7KL2nic8BCOfk3dsxTbYcCOUb1R2jz8uhS8csHSzNsGEDx2ectPYm7h8+OWtiDopTqQ0OVwad+ND5TGFmzbBqlUiCeSGGwTh6BYNRfL6g9rQaCE8w2bZ7AD+nmdlrxq++eYb5syZ87cImvkTZrN7W4m9ZXsBXGbprRexM4YN83FNhtnbhQ+0L9Hku2qg9pY7FovFo4110qAkpj06jdjMWNXbJ6nXYoyC2OpqSr/evaFXL/HvggIwGJRfT7L3zKvKo8moXi1FR2hbU8NsNlNcXOy23w5LHcZVI67itD4yLcG6EDxR+g1PG860XtNYOHwhTSZ1n6nF4tje8zjp5xxf7PqCkz84mUdXPxowSj+T2eRxcFJuv+0KGJkmsio2FTneyElzir/qLjiCFDzZXi4WLqPTR3dcYwIIckk/ADJOh5SToe6wLXO6sjsgMnDd1QX0Bs2mZno+15Nhrw2nqkF0Sl9YkQdEvzVUQe0BCJapfjA1QdkGqPF9zWEJyS1CohJ5jqBOcZz0Uw/WBEMP7T09LT0QGhRKaFB7AskVfEH6edR3dSHtZVn73xBWmiqVfXjrzTc5KTiY2cDjwBk4J/7WAbN1Oh5/8klS20mLXCMlUgR6jtYd9bitatT0s9Z5VKj+9DmOLoeSlarWaZSg19sSwxwSRBKKf4OvUuHwxy4Oag0p0KlE6adWTT/pu+RLa0+/z7mNpXD4U1nEkTQmOnymHn6PrMpNGSSuI6VfsC7YJ99hKxqKwdi+3mfAYMUs+K6/bPJeo9HwzpZ3OBL7OWaMquxDxt08jiEXDuGEBScw6d5JxPd2TSBUNVUCEBEU6/3NAwTWfpt9tSCK5Cjj40dB76shWHmgMSNa1J8+XHnY+pq35ZKUwH78DTLFdE3STwkaiyHnJTjmLo2pDdJOgd7yraqdwdPkcGnclaP0s1jgjTfEv888U8ST3aK5Cn6dBitmi7ql6xb6lGNofe8KsQ90QTj6e28bOJrDvynkqFrmD5hPYngiJ6Sc4PQYifQbM0atljmBC8ZaFex8VFiInpEDUXJ6tDJkxWWhQUNMaAx1hjpiYsTEqIT002g0TLpLoQ2CTEjqtfAGQWx1hZp+9hvW7GwRLIqMFIGG/HzIylJ2veSIZK4ecTU9Y3tiMBsIQR2VpSNImzWbHYWF4uJikqQZzglOzjqZk7OUW8d0BUixkdJSN9Y6dtDr9Py6wDe+1/X1IM2rrWr6+SODspNCstDdX76/Fekn93n6ArM/ns3q8C0MjnmLiIgzFJ0rt992BYxIGwE4J/0CReln1NZxoEpYl4/OOE76gTzSz2wxc6jiEFEVe0nO/RiSJxOdOhkAQ30EQ5KHEh0SRXVTNTGhvskaKqsvI68qj4LqArqZw9FqfVMPLCD67dAHxY9c1B2GH0dD/9uEbZlCLNu3jLc2v8XYjLHceeKdss5JaVmWH/WcXwCOk35qQkowLK0vpaapRnG9Uk8SEj2FL0g/t323bAP8Nh2GPgz9bhSvGRugdA2EpdoUgP3+CYkTxGsqYMnHH/PPRhGEuaXltcnAVGABkAIUA+8DK4DUhARuufXW9hf6cSwEx8K0Hx3eJyVCdEq5tTnbornZRjJ4Q/qVNYgOmxAeYB125p9g8J0PXGys2L9VVEBGhpODQuIhMgt08iV49sowi8UiS5WtFuk3LHUY7819j/Bg37kh+X3OLVkBay8Qdrlu6jtK8RCHY6KpGYp+hNBkSBwr+/aSclOJ0i8kxKb0G6E3Q+laSJog+56ykb8Y1pwN4z8Uyj9fYfmpkDBG2TpHQre5EDMYzAZZFpHpUemE6EJoMjXRoM+jtlZhAMgFek7uSc/JPd0eV22oBCCyC5F+1n47ZCqkTZd3UspUxcpYCdL6Jq8qD4PJQLAu2K9rFovFwtz+c9mxuwktQV3P3rPmAGz7r6iXmTbD/fFJE2FuPoQpLHLX5zrHrx9dAVW7oO/17d4qqSth29FthAeHW126PCX9yhtblH5h7pV+TU1wpCU35LzzZN4gOApSZ4h5vvR3OPA2xI1w+HepjvzFsO5KmLIUMmY7PMQXzmaucJz06wQYmT6Skekjnb5fUQE5LTGi0b6OnVXvFf7PLmxNvELcSOhxjlhA+ABhwWHU3FVDhF6svv2ZmSIHktJPX9t1SD97pV92tiANMjNhxw5h8amU9NNoNLx+xutqNtEhzOb2pN9xuIcUDGxuFsGkji7JJPVtqRD8NSOvYXqv6X9L61W5kEi/gxUHiY03AkE0N4vMc2+sprzB4crDNOhK0JmjjvdHFxieKr7XOWU51DXXWec6CR1N+lksYmwIIoL9V5dyqGGLTzPXOxMk0m9f+T6nAcWaphp6v9ibcaGwav6bBPe6mAiNmFctFlh1wVafrxukAFmMPg4NWqKjOy4ZIOAQ2RsG3wfpjjd57pBfnc9Xu7+i2dTsd9Jv0ZxF5FbmMibDefagRPopqdvxd0RMaAx3T7zbmhGv+HyFexODycC8/5tHYngir5z2iiJiwBekn1sEhYtAsz2ZZ6iE5TOgzz9g9EvitdhB4kclVFVXYz/b3AJcALwNPAdUATHAXGAhcG+QkzBJRE8Icr64lUi/o7WedUrpmWi13tlCBqzSD1St09gWcXEiQOiyzmncMJihrI64RBIZzUbqDHVE6t3/DZK9p7ekX7fobiw4YYF3Fwk0JE+F8R9AykluD3Vp72kxCkVw5sWKSD8p2FzZWInJbEKnde5mZk/61bQo/R6MLIPfz4e5PrBUTzoR0k8XFnG+grFBBPdDHNcddYu+/1B0uFajJSsui93HdlMfekAV0q+upI6X+r3EiKtHcMoTp7g9PphQdKZIooJjvb53QMFi8tut0qLSCA0KpdHYSG5VLr3je/s1npoSmcLi8xZz222Qg/r1iDscdYegJalTFoLC1S2NdeAtOPwRZF0OQWGt3vpm7zdc9e1VzO4zm6UXLgW8IP0a5JN+0jyq0ShYq2q0MOp58W+LBcK7Q7KfCgHGDIaB/xL2yQGC40VUugA2bBC/+/Txw0Z86UD4S9kkrwjpM2Hi/4maCT6CfRBUmqTq6sBo9NktZUMi/TQVXdfeE2wWn57W9fMHJMIPlC0oSupK2FS0yefWo4GK4GDb91apJWRlYyWrclep2p62Fknju4/nkhMuYXDyYFXv05XQPaY7IboQDGYDxfX51s12R1l8mi1mcitzAQhv6tn1FvgqIiUyhdTIVCxY2F7Svp6Q1A+qq31STsct7C2dU2PimdZrmv8bEaDoFdsLnUZHvaHeqrpqC0mpfGk0BK+/Cgw1rZR2/rBtldQj0UEiw6MrrFMcwmKGPc+JDFG50Opg6P2Q6JntRkaUCPAdqZZfJ0dS1xcXe3RLK4amDOWMfmeQHOE4AGgy2YLox5V+7vHI9Ee4fvT1ilV+YFs319TIG6fLG8pZum8p7299nxCFNSU7hPSLGQjTfhZJnhJCU2D0KyK7HURAumW8U+220dG07SapwD3AX4gA4l8t/zcAMc4Gt4mfwbi3nd7HW3tPKXAaFeV5QoXRbKSysRIIMNKveq+o09hc6bNbxLW4hZW7F3ApQnhwuNUlRG5dP2kv6W1Nvy6J0CRRu04GseXS3jMoHMa/D31vVHR7yVbOgoWqJtcSJUdKv3eak4Ra2RcITYYpS+QH/j1BUJggLMe947t7tIFkfV0fcsDrmn4A4YnhNFY2cmyXvA3qw9mrmLW5hn6hvnHs6igkVnyBdkk3+XXdzCZYezHsfEzxvSTyFmwW5lL/9IfST4L0/elyMYHUk+HcOuilIMmj9rCo7acEOx+FdQ7sPQfcAdN/A237pKceMe1rx3taBkar0RKiC7Em07hCQ4vLcViYjDVR5Q7Ydl/r9aNG4z/CD8QecNjjQmUYIDhO+nUw3FlDlNWX8dnOz9hRssPpMetaLHx9bu1pscCge1pv0nx5L4vvvW4jI22DR0er/SwWC3dPvJurh91AeIPI+O8KwbTYWBg5EsaNs2WDZGaK34cOeXbNJmMTu0t3s/1o+6C2WpAWE6GhICX7ajQa4uPjXfbbJXuWMPKNkZz56Zk+a1ugw5MFQFl9GfFPxDP13amq1qOSguAdpVDrjLBf0O8r3+dxFpdaKKkrEbUeLVpCDd0UByfl9NuuhGGpwwDYWry13XtSPzAabYtof6LJLhfCp/WHOyGCdcF8ff7XbL5ms1PiRSL93q7WwYDboVZMolKfkNYxvrQNKasXpF+EVmSZ+Wqd0uH9trEUNt0ChxwXtHcJi8UjwqJbdDcAjtTIJ/0kpV9VFTT6sF59ZaX4s7TaruFCEciQxmmzGVmBUUnRFR8W71Kt4gi+IP086rsarbCbSmixrMn9BD6PgmL1rN/nXHghH4TKq3n4flgYcy+80KP7WJV+HpJ+atTzk7LoNWhkBdX8hvwvhSqrZp/PbiGRfi5r+gHsfRH2viD7uhqNhqtHXs3NY2+WVRfcYrGtecLCXB/rDssPLef7fd97rB6VA7/OucY6aJLPykrP1Cmp0OsSxck2ep3eqtZ0Z/FpT/qlRqZy+bDLichaAFl+UF86iocdfA9WnyOUet5C4ZxhRe1hWH02HPpI9imSNWRdyAFVktQ0Wg23l9zO/E/myzpeeo4yp4FOAY1GQ1hUMoRlQFg3eSdpdXBkqbKkNjtIz/FAhSD9pDVLfb3vRRRmixmLxaKadXJAIii8ncrOJVbNhdXzlN2j+BcxH7dF3AlCfe2gDE73aFE73p70k2JESuuKLzl/CY33NnLWgLPcHivtbWTNo7ufgp0PQ5UD7iT3/8S8HwDw9972OOnXwdBqXT+Cvwr/4rwvzuP8L853+H5zM2zZIv7tc9JPoxGe3770Fgeo2i3qJRz6wCeXX5u/lmnvTeOiry5Co+l4uzMJGo2GG8feyD3DXyTYFENkpI1s6szQaOD+++Gee2wEa88W2/U8Dx0xPtnxCQNfGcgtP97i/mAP4SgQotVq6dGjh8t+u6tULMAHJalnSdTZIJF+khWYHCSEJ5AZm4kFi9N6ZJ7APmMa4Pe83/l277eKlBR/Rzir69cRkFR+oYZ0dAQr3qzJ6bddCVL9352lO9u9J9ncQsfMedKGe3P2+dyz/N/WYPVxCJze93SGpQ4jWOe45qhE+u0yhUDOy7D9PsC2jvli78f0fK4nly25zGdtlJR+EVoh9/LVhrvD+60+Fk5eBX1vUHZe9V74OgN2/0/xLSU7yJK6EtluARERtmfgqcVnXlUeT699mm/2fuP0GHtrz7/JUOoVqhqrWJu/lnUF6xSfGxRke6ZyxunSepGRkxShvBaXL0g/t323ajdsuRsqtrR/z1ANdflC6ZJ6CkT1Ua1dC6+6iuUGA+6eyDpgRXMzVy5c6PiAyp2w5S6RUe4AVqWfl/ae3jwTaW6NC4tTTAT7FN3OgrFvQ7TvbK8kgkhSJlssTgLR+9+AnJcUXful2S/x3KnPWZ+xK9gnOYV4WX7+vhX3cdrHp7Eyd6V3F3IBv865+Yvhy0QocD7n2ENKLnJp2eoBPpz3IT9e/CNpka7L1tiTfgOTBrJoziIemf6Iuo1pC1MT/DwZ/nRQ7zB+JOR/IV/Z5QiFP0Le556X1NEGQ/5XUC2feJTIIrWUfgARSRHoI+VlEdo/x64CrVZLwuhb0MzaAGHuxyUrziqCqd95dE8r6dei9IuMtK0Lfb23fH3D64Q8HMKaxMuArkXgAmJdUbZBqDHlou8N0N9B/WFXOOlnmOfEIsRidnj/7jGC9KtuqrYm6EukX1WV4CWUQg75Za/0c4sxb8BJP0L8iPbv5bwM2/7je6ujzXfCijNc3sffe9vj27YOhtnsWs2299heAPol9nP4/qFDYgKLjbVZJnZ6hKVB3WFo9F022/LDy1mduxrAr8Vn5UDKTOzKmdTdWhKRystb22jKRZ94EQSQ7FB9AUebbrPZTF5enst+K2U9SW38O0Ky/lJKEg1JGQI4Jis8RduM6ftW3MeZn57p041zh6BklQiiHV0OK06HivYqLyUIJNLvcOVhAMKbMgkJUW53JaffdiXcMOYGDt18iOdPfd7h+/bWcf5GUxM0BR3lSNz/8eTvTyq2ovu7QyL9QnQhMOJp6CMKkkvjW3OTjryqPA5WHPRZG2xKPzHQ+yp40uH9VhcCyZOU1xSLyBQZ1yHK/fYTwhKsfaKotkj2ed7W9dtavJXbf76dh1Y95PQYifQ7bu0pD1/v+ZoTF53Ivcvv9eh8JXsTb2q3+YL0c9t3q/fArsfaq1SayuHrnrDxJsg4XQQkI3qo1q7U1FQef+IJzggJcUr8rQPOCAnh8SefJDXVSb3ZukOw63EoW+/w7ZFpI/lw3oc8d+pzHrVTUVa7E/SM6cnyS5fz4TwPlMq+REx/yL4Cgn1X8Lst6XfffbBwoQMl9KQvYPpyn7VDTdKvuFYEZ31ZA9mvc254N+EcFT9S1uFu7QPXXwOfxyh2iZrTfw4zsme0q3/dFg7JohVniP2Wr6Br2fBoHJD2sYNh1lZRx9BT7HkG/liAx+HgsHQ4rxFOkE9+SvaeRm2taqSfXBTXFvNAwQT+yp7TpUg/j/utznO27KReJ3H1iKuZ2ENYJmo0tn2Ir+OpFY0VGMwGzEbRL7oc6bfjQfhxDKDgefZeqLjGJlodOFKsH/oIPg2GomXt3orUR1pr8OVX54vXIm3PwFdxIlmkn2QZrtMLi1RHGPUinLpBzaY5Rs1+qNzqMmDl771tF9ARdW64s2DaW9ZC+iU4Jv2kTXhamue+/7JhqIFVcyDjTOj/T9/dRx8LcwscD0QqYECiqBeYX51PdVM1MTHR5Od3vNJvZ8lOGowNHDvWD4jq0qRfRITIFi8vh4IC6Of46+0UfRIEoZZflU+jsZHQIPVnfEeBEIvFQnl5ORkZzusP7C/fD9gWtn9HeEoSDU4azDd7v2FniXqkX9vgiRQ0l2PN02lgaoZ1C6GhSCRNFP8EWZcJiwYPce6gcxmUNIgxGWPY10LMd5jSr0oo/cKae3q0uJfTb7sSJItAZ4iOFs+yo5R+lRF/ATAgaYBH9a66Mg5XHuar3V8RogvhH2Pab+Ck8eu5+EbYei/ME7X/pM12pFEEyPOr8n3Wxgh9BH3i+xCv6UkzviP9OrzfGutBG6Lc+koXAqf+5dEtNRoN6VHpHKo8xJHqI2TGZso6LzUVDh70nPSTakimR6U7PeY46acMUuKMlLypFNHRUFQkb5wONNLPbd9NmwGn57RXJYTEe712cYdbbrsNNBom33knU4ODWdDYSApQDLypDWWtxcCTjz7KLbe6yJpPngKn7YbITIdvp0WlcdHQizxuo7Ru9SaYGaGPYGrmVM8v4CtYzMLK1YeQ9s8VFSIAvXmz+H9xsa28BADRCjefiPISZQ1lhAWFWWvCOYP0HPV672M0klWsZB3rC/h1zk2ZKn5kQiJya2pEfVld22k5oickjhfzdrC6BUpNJptSNCRErMOajE1EIMoh+BTTV7T+8lTtgdr9kH4axA0VrzUUwYabRC0uJRanI54WFvGeKoE1GsWxupOzTuarkfUsej2MWuXCdK9wrP4YB5r/QB+Z1KVIP0v1XsJ3/AdL6G2QPF7+ibWHhKos9WRlVpLAmf3O5Mx+rcvYxMSI8dbXe0upnmqwKRbw3jo54JB1uRjLHNhrqoqKLWA2QsKo1q9HZkLGHNA7Xuz3iOlBeUM5eVV5DE4ejEYj4n4FBaIMTLrzbYQVzaZmTvngFKJDovm/s/+P8GDXRW/dkn71BbBsBAy4DQb+y/mFfLi2bIXJX7k9xJdlOBzhuNIvwCGX9ItXnlCsHKYmMUDUeejJqAQ+DMjHhcVZbRz2HNsTMEq//639H6PfHM2H+4U6oyuTfmBT+xUUKD83KTyJ6JBoLFh8pmjwpKaG2WK2tkcK+PwdIQUFldh7AgxKFoqKHaXOa5gqRdvgiVSAPdjXiyl/QqeH0/fCnMPQ+xo4pxZ6nO3VJcd1G8eVI65kSMoQa4ZtRyVGpEelMz5lOjF1o7rURq2j0JGW1oL0E+qIMRm+9iTvfDhQfoDbfrqNF9Y7rjMkkX5FZr3YvLRkgEvPNKxZkH4F1QWYlFjDKMANY24g58Yczox6EOhaNkmtsO0/8H8h/lnz2qFbdDf0Or3bGkP28FbpZyX9Ip3v1v8OLhRqYmDSQMCWYKgUbpUtdrCSfmGBQfq5RVAERPeBYAfFlkc+C4knwso5UPSzT25/y623kltQwOR77uG5AQO4Pj2dp7IHcLT3PcydX+Ca8AOhUovp75VSwhXUIP0CEqYm+CwSNtzs09vYK/322nHu9fUO2lN7CIzyCxxft/Q6Mp7J4LUNr7k9Vq36YfWGeusY4kuln+qoPQiF7dUiniAqSnBMFouTteugu+GkHxQTfusK1vH2prfZXLTZ6TH2lnUhIfDFri+IfjyaGXkNMEWePanHaMsWb7sXVp4BVXbJsU3lcGQJFHyt7Nqxg6HbGd61r2oXHP5Utl2eXqcnIVpE7v2t9KtsrAQEWdSVxlZN+UYSK7+EeoXJfgcWifqqterE0fwVT61olEg/MdB3uT1I+izor7B8UcES+GEUHHPsPuAQ666G1Q5qYSadKEirJMcEsqu6fnKTw6ubqlmVu4rvcr6T5fgjzd1OST+zQbhCRMlI5GkqFxbyfzMcJ/0CHFKGaN+Evg7fL2+JCfiF9AtNhLPLYeQzfrgZcOQ7+H6oYO9VxoAkofbbVbpL0cbal5CsKqOahYotznUCYadHdzFnkO/BuKvRaKyk2r4y31h8ehIIKawppMnURJA2iB4x6lkSdTZ4rPRLHgwI1ataGTBtgyddUukHYmMYkiAIQJX/to6ue3rx0It5dfwvZJX8s0tt1HyJ97a8x9mfnc2yfe0DLh1P+glrjdHpo/3fgACHtNY7WHEQo7l9EaKE8ARuGH0DNX1uEJmg3/UHs8E6T2nqUgnSBmGymBTZQ3qCrlgbpRViBguLwVAPgqwNRcJu7PDHik/9/qLvabynkTP6yQ/GSaRfsZPyHO4gR+knbbr9Sg51YsSFxZERJRQznrgXKAmgSbVVlCr9LBabxb5fn6uhFuqPCNLFEWr2QuH30OQ7e4HU1FTuufde/tq1i5wjR/hp7S769L0XkylVXvy6scRlsPTH/T/y1qa3KK0rVdw2NUi/3/N+59W/XuWvI56pjn0CQw2knORUIakWpP1zVRXssnOQbUf67XsNvsmCcvmfkWRtJtW2dQXpOXo7R0q1IUODQokOcUCUByIsFlh7sXCIahtgPfiuqFXX1t7XBbRa35AKb29+m4XfLuS7HOe1zextWoODwWBqSR51UntZdRx8F36/QHymI1+AMa8Lwk5C7CCYvQOGPSr/msY6RWS3U+x5FtZeAM3yk5SkerX+LjEgkX5BptgutW619DifHX1+hNSZyk7sPg/GvQNhniUS1DbXsu3oNuoNYmD1VzzVSvoZ41RRUXcJmA3QWApGBRv7/rfCkPsU3+raUdey6MxFTO813fqaRPqVylzuSGvWSH2krJrDbi3PI3vBjHXQfa7rCxnrRC3ZLXfKa6gnMBvh0Adel9lRG8dJvw6Gq+KVtc21HKk5Ajiv6edXpV9HoPYwNKtctRkYmCgycO1Jv46298wpywEguEaQfim+c/AICHhD+oGtZp70uakNR6SfRqMhNTXVab+VrD0zYzMJ0v593ZPta/op4e76JfRDp9FR0VihWsD6b0H67XkeStfa/l+1B/a/KRYeXmBN3hre2vQWphAReOvIMdKbIJi7ftsV8UfBH3y5+0tW5a5q915Hkn7NzVATth2AE1L8ZLPRiZARnUFYUBhGs9Fay9IembGZvDj7RR6e9jCghaBIMNZZn2l9nc5q72qfhekLqKVicIYO77fZl8Pkrz1LogiKgANvwdEVik+N1Ecq/pu9VvrVuif9JHIo3LULz3HYwepeUKLcvUDJ3uR/M/5Hwz0N3DtZWf3AhgaQyopIgVg14Lbv5n4MX3cTtYgdIeN0OLdGBCX9BMnVw2Borexxil+muKznddMPN3HVt1d5VKNaDdLvq91fcf331/P5rs89v4jaCE2EqUuVqxgUIibGpgpbZ1e8sR3plzhWBD5DkmVfOyFMbHDkKLHVUmxK1p6pkb6dD1Wbcyu2wa4nYOiDMPZtiOje+v2QJChbp+hzB5vKXFKdt0L1PlHXXInSBRuJKxEJjmCf4KTR2BxjZmiKIPczRffzCOUboakMDNUQng69r25/TLRjcYBTHHwPPo+Eo17Wt8+6AiZ+pkj1/PmRp/mj70nsMC327t4KYbWFNHYt0k+j1ZLYfSgavcJyCfEjhJ12iGee7UNfHcoJr53AxsKNgLI1izewKTbjul4isKEavsmGHfLrZALC3WlurvNado6Qeb6or+uoDRtuFrX9HGB2n9lcPvxya6klsJF+JSXybl3VJEg/uUksLu09pUCjHJvioAjo909ImyXrvh6hqVTUSt3/psvD/L23PU76dTC0WuePQCIzEsMTrYuStpCUfn6psWGogbwvFWVmeYX00+DUjRA7RPVLS7Y7u4/tDgh7z7L6Mqs9j6VUELzHST/XkJ7hrmO++T46Iv20Wi2pqalO+23PmJ48cfIT3DD6Bp+0qbNAGo8aGx1ssl0gJCiE+6bcx2unvUaYQn95Z2irRvF7hqav0VgKm/4JOS/aXju4CNZfDTXeEeJXfXsVV317FblNW4COIYksFgsNhgavVEXu+m1XxLDUYQBsPdo+06wjSb+yuioa9UK9LwXEj8MGrUZr3Ui5TGg59BFog2D2FtDHWgPW1dVYVea+Iv1Ofv9khr8+nIP1wg7LV8GTTt1vg6NFbeqxb/jldvaknycieUnplxaV5vQYifRTkxzq6hic1OJe4AHxo3ScDg0KJUKv7OFI69ygIFF3TC247bsxg6DvTaIOlzPoQn1mn+kIoaFCTQQybed6X+04YNYCqfaapNJSAjUUYscaxJ5SIqn+TtBqbQFo+xIS7Um/caKuWUx/2ddOCBefpxyln1pq+OJaIeH2ZT0/UHHOLfwett4lauL2ukS8ZmyA/W+JCSrjNJi1VZDACuBSSdRQCLseg2N/KLqmFF9zReJKSQBt95EXkwO7n1J0P48w4A7h7KCPcX3cke9g9dnykuUjekK3ucrJwrZIGg89zhHBdJkoaNhLWfQKiixbvLu3QtiTRV2G9DMb0BYuJTU2yPN+66GzUlZcFgAHKg4A/ttbSuRtUBezaQVErD0oyud1b11DAzkvQLF8e3WlSj/JrjomxM2Y1gKJ9HP4vDfcCL+cJD47ORj5DGQtkHesJwiOFgmj2Ze7PMzfe9u/rxQlQGAyOa+5khWXxeLzFlPT5PxL7FfSr74A1pwNg+6FEx7y/f00GlHzAaC+EHY8BGNeVeXSA5MGkhKRQmxobEAo/aTajd2ju1OxTyyckpUlwHU6SDX9iovFglppwGFm9kw0aJjUc5L6jcMx6WcymTh8+DCZmZno2lURh15xvbjzRB9KxjsJQkPF51ZbK9TISoKE/5nyH1Xb4kjpFwxk5b8PNath8D2q3s/v0MfDzL8EASCh1yWQNAnCuzs/Twb6xPdhz7E9FDXvA06msdGzvuoNKhorSHgygbjgFMZr8ggJUX5zd/22K0JS0W0p3tLuvY4k/Qqq8wkyxhAWFEFsaKz/G9AJ0DehL9uObiOnLIfZfWa3eq/B0EBVUxUJhz4guGSF1ZpFIv1qamBk2khMZpPbwuieYkfJDo7WHeVEndiw+Cp40qH91mwQ1mTpp3m+OQxzTqC5wuaizTy06iGSwpN4/YzXZZ0jkX4NDWLeVVKLGJTZex5X+smHZFnua6Wfp7Bf56qZdOy27yadKH4CCBqN6DdVVWIcdbuvdqNWS4kUnbKkTmbqux3UUIhJ902OCKDN5KEPoXq3qL+mgCTwBPHx7RVhUuDQq+tK9p718kk/bwPTI9NG8v7c94nU+9aDV7U5t9+Non/Hj7K9tuNB2PMMpJ8K4d0UEa0SXCr9EkbB6XvEtRUgLlR4wboi/aT+KO19JOv154Im8MiYJxXdzyNE9IChD7g/ruYAFCwWY5O78TXjNPGjFiwW2ZNIv+Rs2AM1QQdoavKfRXyXrOlXuQNWnUlF+nVET3pReb/9YRSEJIp6mAqRHZfNr4d+5UB5a9LP1yKKMRlj0BqiCduZQag8zqjzIDxDJHMqhcUMhz8Rqs30U+Wds3IOaINh0hetXw+KhLlHQO+4zlSDoYHf83+nrL6M8wafByiv6SfZeypV+jncf5ibwFAlai0HAoIioNsct4e54oB8geOkXwAjNjSWuf3nujzGrzX9wtJhwicQM9APN2uDzXdA3qcw/AnHhd8VYnLPyRTfLjLntmwRr3Wk0k+q3dgnrp+1HV1d6RcXJ8igujooLITMTGXnj+02lrHdxvqkbeC8pl+Nv03oOykSE8VneOwY9OjA8oZtgyePTn8UfflGuuU9BSWpIvjQmW0ftTqx2bVH7BBVFNJS3cy82v1otcIGTFYwTEVIFocWC2gteo83an+3fjskZQgaNBTV/j975x0eRbX+8c/spvdOCCSU0HtHaYKKKFiwd8V6vTauvWDh2nvXn9feFSvYUFFQEJBeAoQOgZDe6ybZ3fn9cTK7KVtmdmc3CeTzPDwhs1POZPacOect3zeX/Kp8mwES2tbplxIwhBmbSxk1QX/Z7qOFfnEi8tpRpt/ivYs598tzOTttLN/O+E5I+3adTlSUmJdVVsKHM3xXd1mWZVuGg7FODAS+NNi0Wb+tzYVDX0JoN8BDp19tPpSsh/hxEJKo/jBzLd/t/I6eMT1VHxMUJOZUpaUikEqr02/xpYvJqcyxyaY74qjM9Nv7DgSEQc9LfHL6E3qewMunvszorqM1H6vFgDZn4RxkZB6d9qimetKe1K5WS0d850ZEiL+3qkw/N9gy/ao9z/RzWr9GBe3S6Xf4a5GNNFSFA8NLYh3YK5UxzIbVDCvOFTJ3KmsbaZH31CvTLzU6lcuHX+7dSVSiS78NCIekFgG5g+8XhtnKPZodcwpKIIRDp19AOEQ5LoXjCjWZfq0UYxrlPbMCksV3p72QfhWkXwOBfizQaiqCxcMh9VwY84qqQwYkpQNQHbyPykr/Of2sspVAa8TRJe8Z1g3r6NcoKInDI+toRG8IjPHo0ulx4jkqmX7+qun35ulvsmYNPPar78oLdDwkWHsdJJ2g3ulnMQlnYatTSUJG2AnFtcVM/3g6gYZAzht0HkaDsVmmnxr/vy3TT6XXVgk6dDgnGv+24/twRulmWH+rCE5JO1/9cWrREADhTzqgZk4nCiaTfQLrF6dfULTQ/40d5oeLtWDg7XDKP2DUx9rQVEdXMWAXFHic4e41O4t2ApAaJiLfwsOPMsOKAyTJnu3XVH6lveCJgWvpgaVk5GfY6sYdyyj9qth9MGwzTGYT/2T/w8KdC3VpR8tI24uGXMQ5MY0Pddqv7fLFrImSTc5r99WXeXVqxQC8t2RPM/lAf5JVlgVAQoCQAeuc4KsjIijCJhPZUuKzrWv6SUjEBB+thYi9p1+8cPrtKdnT6jPl3VJujAQkIe1buMpmtK+s9O08prK+0hblbqgTz/CoMZ40JTwNLqxTF13vjNxf4K/ToXClpsO6RXYDRPadVcNC1pu6fqO6juL0fqe7lIc86jL9qg8LI0nWAp9dondsb24dfysT07RntWlx+n2942s+2vKRTXZOLb50+rkkawEsny3qtrcjlL+DKqdf1gIhKVW13+HH3sh76uEsapdOvwmfw6zM5soUPkLJCgNQkl9ayXsaAqDwbyjPVH1eW6afCnlPPWRaOxwNFaIMTMt3V2AkjH0Dukzz+NRus59NhVB1QNM5PXL6NY6zQQZj2xmOHBEYpc7hV30Yls2E7O+9v2ZwHIT3FBlKKkmPE7KQtcEH/LoOeXjqw1x2uJIBR546evpkSBJynxuoDR3o2fGTvvRYhj491rHTzx/P1Nc1xduMwpWw6xXhTNeCJMHkb2HE0+qPOfFXmPqD48+qs0QWqQO6RnTFKBlpsDbYgpoUm19dnbr5U72lnmBjsOpMP7eBUFrkUI3hULZVBGb6gu1PwNdxmuYV/qDT6deOeW/Te3yb+a0tBbYlpY2B8iEh3kUDdgjiRkP8WHVFOjWSnCxjNIoBRauDQi8uGHwBL5zyAuOjzgGO/iw/BW/r+mVXZPPDrh9smZJ64jKV3AGyLHP2grMZ9uYw9pbs1b09HY2ExlINalP9FQ6UHuD4d4/n8u8uR9ZhMeVw0Z1+DUz/GyL7eH3+NqUmG34ZBetvav3Zn6fD9729WpAqmX57S/baDJD+Dt5XMv3ijJ1OP60MTRLZnhn5Gc22t5UDF/SLfD+amdl3JuuvW883F3zT6jPF6RdvRETonrwcup9l658NDfa/cW1DrS5jaFMUSbOQgBCsdeLleNQ+S2OQd3IxSZNh/DsQN1LTYUpdvXpLva3Wsxq8cfqpQTGYHzUBaeGpMGMtjHmtrVviELUGtNqGWqobRJRaQpi2GllKcJvfnX6Ve+DIT2BumXrVtjSVSXaLKR/Ktjg1zinZ9d5k+nk635FluX06/QJC7WU7fEzTTL+BjfZwhzXGzy2CSV+oPm9KZAqXD7ucK4a5zwDXQ6YV4I/9f/DT7p88kor1O7m/wU+D4cBHup/abcDab8fBinM0nVNx+pWanKtPtJy39k/oz/mDzucN0zfwx4marudzKvcJmT9XVO2D/KViDPMWyQCnrIRB96g+RMlGrwvMp6i0zvs2aKCuTgQetrt5a/5fwtljqRcOcyfBJK2wagv00RNbpp8f5T1lWUaW5aM3oCJ7EWyYK7KitZJyqn6JOX9fAH+e5vAjo8FItyjh5FdqxwcF2eesBSpeU9eMugbTAya+OFfdu9dpTb9/roL9H6o6h43IPnBeKfS/WdtxaglNgZhhQmq1HdGhnH7Lly/njDPOICUlBUmSWLhwYbPPZVnmoYceomvXroSGhnLyySezZ0/zSOmSkhIuvfRSoqKiiImJ4ZprrqGqhUt669atTJ48mZCQEFJTU3nmmdZ63V999RUDBgwgJCSEoUOH8vPPP3t0T5KTLBNZlpn7y1zO/fJcW62NligOqrg4PyWrFK2Bb5Nh37t+uJgDrGYRPaYTr6x5hbQX0/jvigdJacxi9tT55C2jU0Zz2/G3kWYREXBHez0/BW+dfvOWzuPML87ky+1f6teoRhylkkuSRGpqqsN+W1RTREVdBRKSrbjxsYynTr8+cX0INARSVV9lm0x4Q8tF9+/7f2dpwR5MsaPBlAeb7oGaI15fp02QAmDIg0JWpSVJJ4jtFs+LmChOv32l+4iMFtrj/nYUHa4Qg0OMJBaJnkzwXfXbo5mhSUMJMgbZ6lgoNDWc+DtIef7BE1iXfjbVxjZ62XYAEsMTGZ0y2mEEZJ1ZGEgelLbAkgnCsRSSSHAwBAaKfUrLLPR/rT/hT4STW5Wra9uU7Ib40HifO3DbtN+WbRNzXm8MKhG9RYBJeA9NhwUZg2xZQkcq1L+bPHX67SvZx/OrnmfRzkVO95HloyjTT5bB3Hgz8WOF889qgU13+yTyd2/JXj7e8jGrDq/SdJwyTtfV2Y3OjlCcSlqiphV8lenntu8OeQAuboCYwfpe2Es0Zfr1uwXOK4GEcQ4/9kbe06mBSyUVdRW2AJHEcPXSwj7FXAMlG0QmmB9o6vQb2Rh34bCmn8b3S2J4Ih+d/RHPnvKs2331ykZ5cNmDnP756fx96G/vTuQGXd65kf1g4F2QNEW/hjXiNhCiz7+glzY57l6xvfjk7E9YcJ7zjO+Wc53ZA2bz5flfEtLzQq8yF31C5jOw6hLX69ouU+GCKlH/vQ2ID40nQBadYl+Rf6We9HLE607ms6KUUW0O7HoVfh4mAntd0VAJX0Vj2PqA5/22Jhs23wu5SzQfqmT6FdcWU24qt/VPXyqO7C/dT/BjwVy2vifQDp+jt/SfCyctg7BUz4631Kv/4x/42Hm2b59/wYA7nB6qOO4Pl9vX8snJ4meuhmWnUWUyj/LubpbkVJsrFBeKtM2tkSTfOk7Sr4KT/4QQ18Z8f69tO5TTr7q6muHDh/P66687/PyZZ57hlVde4c0332TNmjWEh4czY8YMTMoID1x66aVs376dJUuW8OOPP7J8+XKuv/562+cVFRWccsop9OjRgw0bNvDss88yf/583nrLnvq8atUqLr74Yq655ho2bdrE7NmzmT17Ntu2aS/UbjA4fgQ5lTlU1VdhlIy2SIqW+LWeH4AxWHjHPdR+9pq/L4DFI+yLdR04XHGY7YXbbc6nQ977GLxCMdZ0ZvqpQ8lk2Vaove+5w1Gmn8FgID4+3mG/VeQNukV1IyTgaJuFaMdTec9AYyD9E0Rdhm0F3j/XlhP8mZ/OZMZHJ1FclSt0vTOfcSph0O4JTYZhj0DXU1p/NuguoXMe4LmFNi06jUBDIPWWeqwRYvHRVk6/KNlzp5+rfns0c+eEO6m+v5pHT3y02XbFmGy1Ool89xFlpjJ21S0nP3YhUSHtpOB2B0Mx5K4zdIf064SzwlyNJNnlzCorjCIaFpnMQn3lRZRMv/gw3zv92rTfZj4nsgZ0nG9qQYmiPVKp3umnLLi1Ov025W3iziV38uwq50bshgYwN6pId/hMv4K/4Oehzd/7OT8Jo9vh1tm13vLuxne5YuEVfLzlY03HhYZCQKMKoqv3riIf2SWii2Yjgq+cfh31natk+qly+rn5W4/tNpZPz/mUF2e8qLkd3jqLQgNDWT5nOd9c8A1hge3ES1+6GX4ZA3v+55fLKbaRbt3sgbQO5ztl2+HgZ8JYqjN6ZaPkVeUBkByR7GWLXKNLv40dBiOfEUEvOuM2k2jQ3TDgNm3nDI7i0mGXMqWHcyel07nOce/C0Ic0Xc/n9L4aJi4QUp+uMASAUSdbRe4SWD8X6tXV6pYkiVhDD4Lrk8kr81997zkL57Cix0wqQre0vwyxSV/C1J8goieEdIGup4HFTRZkfQkkTUWK6Ol5v7WYYMfTQo5eI5HBkcwdP5cnT3oSGbnZ2lKPuriOKDOV0WBtwCyLgLyjzukX1k045Y1B2o/deCcsCAaTyozwTXcJG5gj0q+GAf9xemhqlDDgZpVn2bZ1a1T4PeKDOHqHTr/QrnBOgWc1guuKYdtjkPe7Lu3zBH/PjzvUbPy0007jscce4+yzz271mSzLvPTSSzzwwAOcddZZDBs2jI8++oicnBxbRmBmZia//PIL77zzDuPHj2fSpEm8+uqrfPHFF+TkiGy6Tz/9lPr6et577z0GDx7MRRddxK233soLL7xgu9bLL7/Mqaeeyl133cXAgQN59NFHGTVqFK+9pl0ixmKxONy+q1jIFfaK7UWQk47vd6df7AghiZfmIKvEH/S4EAbdB1Z9pAAGJQ4CILMw02vnkzcUVBfwxbYvyMjPsKVEHyuZfkpNvyNHxCRBK0OShgCt5eu8xWIRtaeg+QvGYrGwc+dOh/1WkfRUsqOOdTzN9AP7c91euN3rdjRdrMmyTIO1gacSoNvPvUX6/dj/g5QZXl/H7zRU+VzWw2gw8tHZH7H0iqV0jRKDkr+dfkq2Z7hFDNKeTPBd9dujmfCgcAIc1M4JCrL/Hf35PHcUikz9kPpuxIXF+O/CHZCvtn/FzT/fzIqsFc22K06/P4IGw/An4MswWHkJYM9sKC21z2+Uv7leSJJEv/h+pMem+7ymRpv2295XwsjnRC1rb1h6Cvx6vObDlLp+nmT65eVpu5YiIeoqI0iRgZSko8DIYsoXspLmJlapLtOEodQHmQ/KfEZrcJokqZPL8sYh4Cunn9u+W75DGIrdGTX9jOLQVmWwNFfD4YWirrIDUiJTuGToJUxInaC5Hd5mowQZg5jcYzLnDNQmdehTQlNg2KN+y4waPRqmToU5c+zBmw6dfvvfg1WXigwbldSZ68ipzKG2wbWShh6BMbIs27JFlexRX9He58ptVY+65XO0WC2a6u36lYTx0OMC19LkB79wOm55RPFa2P2KkBZVyUPx25m+NZculjH6tcMNfx78i8LoxVgMte3P6RcQBskni//3vAgmfwWRjpM+bIT3gGk/Y+l9ref9NryXqLM64intxwIvnfoS9066l5iQGAIC7GOtr/qoIsMbJokFT4efj7akrqR1PVS1xAyDtAtBVvk9mPQVjNIelATQK6YXIDIvFZradN3x6F+PcubnZ/LLXnXOZqcllwIjRAC8ViwmyHgY9n+g/Vh3bHkA9r7jvgl+fs/6vpKynzhw4AB5eXmcfPLJtm3R0dGMHz+e1atXc9FFF7F69WpiYmIYM8b+gjn55JMxGAysWbOGs88+m9WrVzNlyhSCguyOthkzZvD0009TWlpKbGwsq1ev5vbbb292/RkzZrSSG21KXV0ddU30WSoaR0OLxWJ76JIkYTAYsFqttujsfnH9sFqtGAyGVl+OoiIJMBAba8VisafyGgwGJElqtb/iUba28K442240imhxR9utVmurWjGOtje9J0fbW7bR2XaDwYDU40L79saf3txTv9h+gHDWJA+pQ5aDyMqSm/0tfX5PksSqQ6u4+JuLGdFlBCfkbQAkkpLAYumgz0nDdy8pyUBgoHCw5eRY6dpV2z0NjBeFGnYX76a2vraZg9ybe6qqAlkWbRYTCsnWV2tra7FYLK3uaU+xkBLuHdvbads76nPy5LsXF2dFliUKC+19Su09DYgfAAinnzf3JMtQVyfaHhBgxdQgDOYb66Cu+3kEBcZg7X2dGE+sFqTNd2LofgZy8snt/zntfAF2v471xKUQNcC+HftzkrbNh4qdGCZ/5fF37/yB5wOQFSmi2svKmr9vfP3dm5Q6icigSCIOplMkywQGWrFYtPUni8WCyWRqtW9H6k96f/ciIyVqayXKymS6dvXPPW3LF0bviNrBBAfTvvqTh/fkru2e3tP3u77nk4xP6BrRlQndJ9i2m8zCGhwoBWKxWjH0vBQpehAWi4XoaAlZligqsjIwYSCLdi1ie0HzMdTbezqp50ns+PcOjEYj5yyQkWUxtlosvnlOyvvWVdt98pwSpkDCFKTG+ben9ySFdoPAaiRZ1vTdS4lMIdgYTGVdZbNjXN1TcrJ45+blQX29FaNR3XdPyRSLDxXp+Y7uqabGgCzLhIXJWK2yre0dpT81a2P385C6n9d8uyEMup8r7snB38CbexqYIOap2wu22+rRqL2n6GgDxcUyJSVWZdnT6p5yK4WWUlKYCMzRMu4pMlxhYfbz6/Gc3L1z5V2vIO39H5azciEksd2M5eHhYu5fWSm5/+7VFmNccTZy35uR4l7VedwTY3lgoNW2Fmm3/UnlPRGaCgPvE9sbz+nLewoOtvKf/4jt27aJ51pT0/q5Sj0uwZBwPJaAGJttwd09jXtnHFvzt/LzxT9zSvopTtteWytBY/2wlmt6tfdUWVdJTYPwViaGJTZrT7t755ZtQl55IfLQx5DTLnB6T662u7qn6GjxHqqshIYGKwZDi3vKXohhz2tYR72CIXaI6jFi1eFV7CjaweS0yfSP79+qjSaTmOsEBor17G2/3sar615l5bBRHDfoauQ+N3h8Tz4b98zVGIIiW2+31GFcfRlytzOwTvy61b169Jx6XQM9LoOQrrY5k7t7ioyQkGW5MaDFP/MIpdRBgDmaoKDWa5A2eU5HfsBQvhX63oS1hZKa0WhENhU73N60jUq/Vc6t6Z5kIKIvyIDF4vU9RUVJVFdLlJVBSor+76eiahGkFirFIcsyQUF2G1N7mUdovaembZR/6AMxw7FO+137PfW4FEOvy9XfU/wEsd1R27M+Rt7zJtbjPm6Wta20vXeM2LaneA8WiwWj0UjXrmINcuiQeCauvnv/HPmHn/f8zJn9zmzlB3HUdjEnkgkKapyrlmVgKPwTel6KNTCu1f6t7pUWzyk4GU5cDnGjMaKjPcJsxpD5HHSZhrXXVS7vyd8cNU6/vMbQ1i4tdBG7dOli+ywvL4+kFilUAQEBxMXFNdunV69erc6hfBYbG0teXp7L6zjiySef5L//bZ1+mpmZSWSjnkhcXBxpaWlkZ2ezapfQp42T4ygoKCA5OZmDBw9S2aS6eHZ2XyAckymHjAx7Ok3v3r2Jiopix44dzb5k/fv3JygoiIyM5llRQ4cOpb6+nl27dtm2GY1Ghg4dSmVlJfv3Cy9+cN1+Emr+JHHkvyi1JHO4SVpcZGQk6enpFBQUNPs7NL2nEiU1EUhOTnZ4T6mpqcTHx7Nnz55msqy+uCdZlokIiKDKXEVD5B4aGvqRkVHN1q1ZSJL/7mn5juUAJAUksXt3GRBNQgIePyeAkJAQBgwYQGlpabt/Tl27ymzbVs2yZfkMHVqj6Z5kWSYqKIqK+gpW71lNXIN94PfmnnJyGqisTCMgQKa21kBgoLgns9lMSUkJ27dvZ8CAAc3uacP+DQCkx6RjMpmOuuek9btnMJRSWRlAZSWsX3+QhIQI1fcUWi3SK/cU7/HqnurrJczmIRgMBvbu3QaBYuH8WSW8Oe49rHIAuxrvNaJ6Lb0P/Q8Cw6gMH9/un1NwcDI1xu7sPVgLUobD59Qjey3htVsIMtdSWdPg1T3V13cDEjl8uJyMDLucg6+/e8+c/Az19fXMnVtOZWUFOTmF7NhRq6k/hTeG7xcWFlLQpMJ0R+pPnn73/rfzfyzLW8a9x93LnOPm2O6pvj6FyspgsrNlBg6M8cs9Lc8U77qQir4EB9Ou+lNbP6eW95RoEFlXa/euJSMqw3ZPsQ2xnJV2Ftc37Kb0t4sJmviOuKeMDGprY6msjGTr1lL6Tu8LwPqs9c3uS6976t07naoqEcy2d28W+flW3Z9TZmam7X0rSVK7fE5u7yl0rrinykpN370Xpr/ACye+wO7du22fqXnn1tYaMZslli8/TK9eoaruaddh8XcLlcV719E9VVfHU11dTUBAHRkZh5s9p47Qn9R+99K7hhAZAjsONeh2Tw2WBgwYKDWVklOZQ0hDiOp7io5Ow2QysWXLEYKCqhzeU+ZBESQaGygi37WMe1VVwVRWVlJQUEhGRpVuz8ndO7cgdDr1XRMo2ZWFbMhrN2N5QUEE1dXJVFVFuP/uyQ3EpjxKUPgYuoLD797mqs1s2LuBCbETiA2OVX1PBQVpWK2wf/8hEhP7aL6nv/b8xa/bfqV/TH9GxI1ol3NYf44R2dlBmEy9qKkJdXBP3UhOG83BffuorDyo6p7iQsVac8vuLXSt6er0ng4eTEKWuyJJ9WRk2KW2tdzToSqhdhERFEFNWQ378+z7t7t3bnwhstXC3tw6asozND8nd/eUmJhMTU0NFouZf/7JIjLS2uyewnK3klK0kQOZa+gytIfq7978tfNZmruUF05+gfocu8yr8pzKy01UVtZTXFxBRkYxRaXC5jaudjOVu77iQO1Ej+/JF/2pR/Y9RFb9g/W8IuobzM2eU4BkZciU76mpN7Cnyd9An+dUqvqeSkqiqKyMp6goGAjx+RiRdSiLijqRMi/VBWMwQF5e2497PQ+/TnTVn5jSrmHXTvt2o9HI0JJ5WMsyyUhbAJISiN54TyUlSGuupipsFMUxZ9na5cl3r7osl5C6fdSGDKB7jz6a7qlXn14crDzI5szNDI8bTl2dWFuWlUVgMuk/lmccEH+jUDkOk8lEQUE+GRllPn9O4Ic5bGp3quJmUSEnUth47Ta7p6AK5OrD7MnciCmkutU9dTF14eERD9Mnqg8HDx4kPT2dkJAiKiuDycy0snVrFvHxzr975SbRF0vzS21tcnZPvXr1xmSKavwbZ1FUZCEl7wWSSj7GEjeejJymmp9anlM4IaUHmz2nyMoVRFWtpKLf4x6P5bV9fkKSG2jIyHD5nPyO3EEB5O+++872+8qVK2VAzsnJabbf+eefL19wwQWyLMvy448/Lvfr16/VuRITE+U33nhDlmVZnj59unz99dc3+3z79u0yIO/YsUOWZVkODAyUP/vss2b7vP7663JSUpLT9ppMJrm8vNz27/DhwzIgFxYWymazWTabzbLFYpFlWZYtFos84+MZMvOR/2/t/9m2K/sp/+65xyqffrosL1vWfLvVanW4v9Vqla1Wq+rtsiw3337gC1n+FFk++IVssVgc7t9ye9N7crS95TmcbVfuybL1Udm6sKdsrqvW5Z7Gvz1eZj7yp5sXyKefbpVnzbLKRUX+vaerF14tMx/5vl8flGfNssqnn26VKyu9eE4u2u6ve9Ly3XvuOfF3//xzi0f3NPHdiTLzkT/e/LFu97Rvn1meNcsqX3yxpdk91dXVyRs3bpTr6upa3dNx7xwnMx95wbYFR+Vz8uSezjvPIs+aZZWzsrTd0/rs9TLzkROeSfDqnoqLzfLpp4s+VV9vlgsrC2XmIzMfua6hrnXbKw7Icl2J2F5XI1tWXSWbSzM77nOqLZPNDQ1effeySrLkt9e/Ld/9yQfy6afL8oMPWtvknu64Q3yXli/X3p/q6urkTZs2yfX19e3zOansT5589+Z8N0dmPvKDfzzYrI0PPCD+nr/95r97OvnDk2XmIw+98i156dIO2J98+Jxabv9sy2cy85GPf+d4h/dk/Xm0bF2U3uyePv5YPNNXX7XI64+IMTTxmUSf3FNtrSzPmiXe3VVVvnlOTd+3fn1ORRtl67cpsmX32x3uu3fDDeI7sG6d+u/exV9dLDMf+bmVzzm9p02bxPO+6SaL3+9Jj/5k21ayTbZsfkC2lGa2bntNsWz9LECW/zpL93sa8voQmfnIX2RoWz8984z4u3/7revvWLWpWi6tKdX83bvhBtGH16zR9zl11Hfu6tVi7n/bbfp899JfTpeZj/zn/j9V35PJZLaNreXlnt3T/GXzZeYjX7voWt/2Jw3PybL8PNn61zm6PCet95SdLf6m553n4p4aGlTf07kLzpWZj/zKP6+4bPuDD1rk00+3yr/95vk9/bn/T5n5yOkvp/v8OenyzrVYWv0t9fzuXXiheMcdOODgnppcV8sYcdOPN8nMR753yb0O2/jWW6I/vveeaNO1i66VmY/8zB/3yOaa4jbpT67uybLhLmEva6j1zzu3oUE2l++XzWW7VN/T818vleNvnyYPf+hq1c/J7XfPxT0VVxXb1v/nX1zdLp6T2WyWzfW1srVoneO2b3lYtq6+SjbXlrS+p8os2fpFqGxZc6Ot3ypt0npPli3zZflTZHPBP5rv6addP8nMRx7y+hDZbDbL8+eL/rl4sW/G8kf+fERmPvKYR69WNTfS7Tlp+e41NMjm+jpdxz1V91R5WLau/bcsH/pWVdutCyJl67LTdXtOJpPFZksvKHB9T0PfGCozH3nx7sVu77WmRvg7mq43zaYK2XroO/G+8fQ5mc2yuWyXLG+dL1uUNqy6SrZ+HiSby3b7fIwoKSmRAbm8vFz2B0dNpl9yYwX7/Px8uioagY2/jxgxwrZP04hDwJa5oxyfnJxMfn5+s32U393to3zuiODgYIIdCEgHBgZiNBqbbTMYDOwu3g3AwMSBthTQlvspDubERCMtPnK4vyfbJUmyb085FU7bAuGpTtNStW7X2kaDIQCCYjDW57VKN1Z7nqb3NDhxMGuOrGF36Q66dpXIzYWcHCPx8f67J6V+Y3LgQDIkiYgIpb6Gh89JRRt9fU9atvfsKWqXHDokNfseq72noUlDWXl4JdsLt2v6G7hqY329aFNYmIQk2bcbDAbS09MJDAxEavxAOc/DJzzMxtyNjE0Z67TtHfk5Odvu6p4SE0WdzJISI2lp9u2OaLp9YNJAXjvtNfrF97PJL3jSRrNZ/AwMhMBAI5Y6EfV0eSQErv830uiXMQY2KWYT2VPcE2AsWg4H3he/HPee6rar2e6352S016Py9Lu3s2Qn1/14Hb0iBjGYK6msdHweX9yTyWwCC4QEhNDQIPpieLj9faf2ngwGA7179yYgIMDhd6nNn1MT9B4jhnUZBsD2ou3NrhkbK8a4iorm45g3bXe2XbmnHUWivlyUaQhBQR2wPzXB12P5gEQh2bunZE+z69j+f9p68XPvO5C/FOOET4iPNyBJUFYmMSBBHF9YU0hpXSkJYQm63NPcxXP5K+svbhs9D0kS0r9hYUaadiu9nlNgYGCr963W83j0nOQGCE1BCo6Bxv08vqfKvaJmRPezIH6sz797KSmQnQ0FBfZx0t13r6i2eU0/R+eurhbXDQ9vPk/T2nZP7slV29Vut5274A/Y/hgkToSYAc2vGRoHQx6CmKG639PMvjPZVriN73d/z4VDLlTd9qgocZ6qqtbrvKbXDDOGEUaYpjaaTKL2iiRB376uz+/only1vaO+c6Ojxd+jslKf716XiC7sK91HUW1Rq2s7a2NDg308bTq2armnwppCQNR5bPq5x/cky7D/ffH/9Ks9+7ubK0G2OH6fqTiPN2NEZKR4riLQ3tD8u151AH47DmP69TD8UVVtUTL9ykxlLp9rQ2PZ7dBQz+8pp0rUGuwW1c3ncyNd3rkGA0YH7dFrLI+ONlBdTasx0ZuxoHu0KEaVU5XjcP/6erH+CA0V7z+zLBaXlqAYjKFxrfZv8zns0IdEzaqAEKjYg7Hwb0i/SnxWuhVihur7zpWAnweImnRTf1LVxuBwM8VRy5DN+d7daxNc3VNFQ4X4vzWE8OAw9/fkZVtUbzcaIX6M4/2HzRf2CEdtjEiDcwqQzNW2fuvMXgJu7qnbTDAGYwxLsc151d7TwMTGMjslu5ElmZgYY+PaUoy5eo/likRrsCW+cU6qbu7i6+3N7mn7k7DzRThjN4agGIfn8Ml3zwDs+T+QAjGmnu1+/5SZED0YdLJZBgcbSE4WdcXz8owkJjrfv7wx6zY2NNbt3Eip52cwSPY5kTESUmeL/R22RuV3b//bkPkchv5zIShGzAFGPI4xtCtU7sNQ8CekX9PqPE6fk6VS1A0P7wFGe8FJLX9HX+F/QVEf0atXL5KTk/njjz9s2yoqKlizZg3HH388AMcffzxlZWVs2LDBts/SpUuxWq2MHz/ets/y5ctpUGZqwJIlS+jfvz+xsbG2fZpeR9lHuY4WnA3O31/8PV+d/xUjkkc4/FyW7U6/uNbzDd8QFA2xwyAo1k8XdMDge+G0Tc0cft4wtttYJqZOJCUyhdRUsa1JFrZfUJx+cRZhpGuhQHvU07On+HnwoGfHXzniSj4951OuHXWtXk2yvWBCm2eMI0kSUVFRDvvtqX1O5f7J99Mrtlerz45VEhrtzEVFrvdrSVhgGDeNu4np6dOdjpFqUMqoKoWeG6xiXD8hzIC0/z1h3HVGygyYfdjm8Gs3yFb4aShsfUjd/gV/w45nPb5cn7g+AByp2YeM1WfFuR3xbea3hD4eyllfnIWiiuBJ0W5X/fZoZ2iXoQBk5Gc02x4TI36Kehq+p85cR9eIrgTIYUSYBuEgBqqTJvSNF/KcRTVFlNaW2rbXNNRQVV+F2doY0VC8BrI+h4ZyGqeolJVBeFA4Z/U/iyuHX0mduQ692FW8iy35WyivFXIvQUHgq27VZv02YRycug7SzvP+XLV5sP1xKPxb02H5Vfmcs+AcTvzwRE3HKTGPubnqjymqES/olo7hptQIZWwaVRs7Ln3/DTPWQtJkx58PfdBmSNCTM/ufCcDv+3/HKlvd7G0nTNgmbXNSPTlwQKwl4+KwjR164bbvrrsJfugvGtCOiGiMAauqUnnA8tnwfR+nH3cJF6VA8qvzne7TEmWuExAg/nlCQbUIcE4K92JBaa6BJZPh0NdgqYFtj8COp0F592hl2i9w4hLP2+MFTddyrfpSUBxE9oXQrqhFqX9aXFvscj9lDeLNfGdS2iQ+OfsT7jj+Ds9PohKv3rk7nob1t4K5Wv+GNSG6MZbR4VrE2gDZ30PhSk3n7BbZDYAjFUccfl7fqPipPMcGi1g7djUXQbWfjUZqCIyAkMb3+fqbYO11UHNEOPwWD4etD+p7PckAg+dBz0tVH9InUUQCVxgOtaql5QsUZ1GgOab9rD/2fwDlO9zvJ8viu92SwAgI7eL9XDl+rLCvhqdqPrRHTA9CAkKot9RzsOyg6/6pA71iezG151RizMJm6olNwOcc/hbqCiEw2v2+TTn0Day8WAQLekJIVzg7F0Y9r27/SV+IOa8j6orhwKdQluH4c2DdkXW8s/EddhbttG3rJoZSjjgeSm1U1IkvSHSI+79RU5usJAHZi8R4pgcD74IBd4DU6JQL62afC6y5Bjb8BxrUTgiBIz/BjwPETzf4e23boZx+VVVVbN68mc2bNwNw4MABNm/ezKFDh5Akif/85z889thjfP/992RkZHDFFVeQkpLC7NmzARg4cCCnnnoq1113HWvXrmXlypXcfPPNXHTRRaSkpABwySWXEBQUxDXXXMP27dtZsGABL7/8MrfffrutHXPnzuWXX37h+eefZ+fOncyfP5/169dz8803a74nS4vCjgpDkoZw3qDznHaGmhr7ZNJvTj9zrRgEHL14Oig3jLmBv6/+m+tHX29z+h065L/rF9cU2wwuIdX9AGhRLvKop0cP8TMnxx4VqYXjuh/HJUMvIT0uXbc2KS8YxeCiYLFYyMjIcNpvO2mO4sAuLGyb6yvGE2WCHxUcxcunvkzNsCfhvFL3E7Kw7r5toCeYCsUCy6LSkL/vbdh8N9RqsAI3IS06jQBDAPXWOkxB2X51+h0uF4vpqOAor4wnx3K/HZoknH57S/ZS01Bj2644/crK/NOO4IBg1l+/nmsLKgm0RLefRXc7JSIowmaI2lawzbb9rt/uIvLJSD78/V9QtBZGvQgXmiAo1ma4L230ES68aCEfzP6AblHddGuXYuSMMAqjpy+f41HRb+NGwazt0Odfmg4LCQjhu53fsezgMqrr1RtRG5cympx+75/1Pj9f8jNjU8Y63ae6sQkt50QdDkOgMHAFuPFe6myEPK77cXx53pfsvnk3Bkn90ltxVtTUON9nzsI5XLnwSrLKspzv5IB9+8TPdP2mzjbc9l1jCARE+C5iwEMiI8XP6mqVX4GIdIgZ4vRjxemmOOHU4E2Ak4LiZPTK6Ve2VfyrzRP9ZcoimLEGDAFQvF4EoHUQhNqH+H+rvhQUDdP/hn43qj5ffJh4/5XUlrjcr+UaxBNSo1O5dNiltsABX+LVOzf3NzjyPRh9+5KIihI/Ha9FJOGI3/mSpnMqc6QjlY4NyS3XHw3WBgKAK3Oeh833aLqW3xn+BJz4B4SmQGAU9L1JKA/ozdCHoOclqnfv10UY3cyGKlvWjy+paaghzBhBoCXWf+sP5SViKoRvEqByn/0zU6FwKmy+z/U5qvbDz8Mg8zn7trJtsPNlMAn7YVvOlQ2Sgf7x/QHILMz0udPv5nE3s+zKZfSvuRrwwRqkdCt8102MZ55y6jq4oEb7/KZ0E2Qt8HwMNRghNFm8o72l+hCsvkwEUTjhyb+f5LofruP3/b/btnVvNJtlZzs/tSzLdqdfsHunn/IeDQ0F6stg5UXinx6EJMGo5yAwsvVn496C0a8I57pouPvzRQ+EQfdCzDC3u/q7v3Yop9/69esZOXIkI0eOBOD2229n5MiRPPSQyHi4++67ueWWW7j++usZO3YsVVVV/PLLL4Q0mTl/+umnDBgwgJNOOomZM2cyadIk3nrrLdvn0dHR/Pbbbxw4cIDRo0dzxx138NBDD3H99dfb9pkwYQKfffYZb731FsOHD+frr79m4cKFDBnifOKvN0qWX3i4b40uzTjwoXhpFfzlpws6IXuR+5ekB7RFpl9mkSjs3T2qOxXFwghxrGX6xceL77HF4j4yxF84y/QDx4P0xtyNLNq5yOak6ESgpPZ74vTbX7qfT7Z+wh/7/3C/sxNaGk+igqO4dfyt3DLhbpHGr8b4VrQGVl2uLdLHl4R2gZlbYMRT6vYfeBecugGCPRtYAgwB9I4VmdXVwXsxmexRr77mcIXoT2lRaV4bwjq048ALukR0ITEsERmZHYX2iFJlYVZa6uRAH1FfJ/pcUJB/r9sRmZA6AQmJ7YXbbdvqLaLznV22SERvB0aAUUwCmzr9fBU4XVwjnH7hku+dftBG/XbXq7D3Lff7qSEgDKIHiZ8aiAqOIiJILDSdGSIdoVQZ0OL0G548nNP6nmaT93SE4vTr0Jl+dcXCWeEuU+nX8bB0uq6XNhqMnD/4fFVRzU1Rk+n39Y6v+WjLRzYlA7X40ukHbvruqOfhtA3OP28D8vLyeOXlR1n350BW/taNMQMG8tijj5KXl+f8oFHPw5SFTj9ODBN9qrBa/SRYD6efLpl+CcfB7Gy7tFXscDFvzl4Ev46DnS+IF42auXHZNtj1WptmRSl9yZUDXS2KvKfaTL92mY3iBI/fuSf+Dqf843NHvkunnyEAJnwm1j0acJfppzxHZd7aYGlABtbGnQ6p52q6lt+JHwNdThDPJaInjH1NBL60McnxYQQ2iHnkvkLfjwsTUiew5MRKTtie4Zt5a8kGsDb2ndLN8OtxcOQH8XtdoZh/5Pxs3z8oFqb9BoPd2DNDu4O1HmgyqT/wEWz8j3AINuL1XHnN9fDLOI8OVSQ+dxbttPVPX6vI6PGedEh4KtTmQMEK785jDIHcJSLrTy3DH4PzSiAsxfPrVmdB8Tr3+8lWWH8L7P/Q8eeR6TD5W+jhWI4e7EpQe0vsmYmK08+VPbemoYZAg4jCiQqOcttUZf4bEoIIFpv4BQx92O1xXhPVzy6L3FABf1/g/pi4UTDiSYjq69u2eUCHcvpNnToVWZZb/fvggw8AkSb5yCOPkJeXh8lk4vfff6dfv37NzhEXF8dnn31GZWUl5eXlvPfee0Qoeh6NDBs2jBUrVmAymcjOzuaee1pH8Zx//vns2rWLuro6tm3bxsyZM3W7z5/3/MyzK59lY+5Gp/v4XdoTIGog9LkewrSnf+tK9kLY8RSY1EdOuqO2oZbkbmJW50+n37Auw/jx4h956qSnUMpEHmuZfpJkz/bzVOJze8F2nl35LL/t8yIypwnKotCR088R7216j9kLZvPa2td0uf7RguL0K/Cgqy7cuZDLv7uc/234n8fXdzoprDoAJZvURSqXboaDn0D+Mo/b4RPULqxjhohJiMGZ6rl7lIldTegeQNS78QeK0y81OlUXmaRjFUcSn/7O9FPkezqfo3qePvlpCu8q5IYxN9i21VuF029VzAwhp1RfBoWrwVRge6YNDXYnTb2lnkPl+skXKEbOMD85/dqEzOdgz5v6na++DMozNR0iSZLNEJld4SJktgWKvGdenr6OX2VO1KEz/Y78BL+OhUNfud4voreox9EOcJfpV1VfRXWD6OzJEc7ryjvC106/jsSLzz9Pj+7dWfXUEzxftZMv63K4ffdOVjzxBD26d+fFF17w6LyKI12psacGPbLDdHH6gYh+D2ixEEqeDj0uhsQp8F0ybLzd8bFNyfsDNtzSzEjtb1w6/Q58oimYeFDiIK4YfgUz0me43E+PZ/nRlo9YvGdxM6WGdokkiSwTH+M2k6jnRUKiWwNKpl9lfaUtA6UpLeeto7uOZkbfmRSn3whp7dzpp5D7m8usHa8p2QC/jIGDX6jaPSQEwhqExOeuPP9IbNXVgYRR/3lr5V5YMglWXyF+D06EikxhawCIGgCn74T+t9iPMQRA8kkiuMIVxiBx7OD77duGzhfBJno6byVJtMkDNbcB8UJq0x9OP2UtqbvTT5kwB8XCBdUO67u6P4cV/rkKjvwsHLWrrxDvRy1Z8U5qAKrmn2tg6cnu95MtsPs1yP3F8eeBUZB6NkQ6ly9PjxWTx6ZOPzXynuFB4ZgeMGGaZyIs0P2iopn6miFAZConq7hHPdnxLBz+2qXcaXtHh/zPTvRmwfYFfLTlIx6b9hijuo5yuI/i9IuP92PDupwg/rU1Qx6EoY+Il6oOnL3gbBbtXMTnZ38LzKa0VNRzaOEL9glRwVHM6jcLgP80lg471px+IJx+O3ZAljZ1Ihufb/ucx1c8zuXDLueU9FO8bo+rTD9HKBk0gxIHeX3towlv5D37xYuAjd3Fuz2+fsso28q6Srbkb2Hw7seILfgVLqp3n+3X81IxMXcx8fEbsgyb7hSTnZTT1B9nrhaFhT2shdo3TkQsNUTuhQKx2PbHu0fJnE0JT8XcmJzRkSKm2wsjuozgSMWRZrJySlaYv2r6nf756WSVZREf8ApRnHh0Oot0xlF9WCXTb3/0eOhxgci6WD4bjvuQoN5XEB4uHH6lpbC3ajNj3x5LXGgceXfkeV0/oMHSYDOIhXEUO/2m/y2iOvVixXlQvBbOL9eUBdEtqhu7inc5zT5wRFISGI0iG7u42F5X1xkltSW8v+l9ukZ25ZKhzqW5jopMv7hRMPgBSHKzjpn4uc+a8Nyq5/hqx1e8d+Z7DE4a7HZ/d5l++VUiWjAsMMyWGaqG+np7KYM2cfod/ALMVdBHv1rcnvLi88/z5Lx5LLdYGN8iW+JSk4k1wBn3C4Prbbe3cHAVrxdO5D7XOZwj2jL9PHD6eTrXabA02GQnPXb6rb0BovrDgNtafxYQBhM/Ff+PG6dubtzzksYAtJGetUcHXDrQsxfB4W9g2KOqpNGO634cx3V3Y6zH+2dptpq5etHVWGQLh287rMpA2ibse1/IpHWb5fNLuZb3bEQx4Kt830YERfD5uZ+THJFMSEDrh9XS6TdvyjyVrW0nWM2wrNFBfVGDPvJ/LQmIEGUkzOqiQiUJIuVUytnE3iL/Of3AB/PW0BTo82+7bGpYN1FbTVF4kAxiPFXIWyqcO3GObbytkCTxnd54m1jH979Vf4nWcZ4HWA9IEE6/zKJMokXSn8/kPbu/2J0GSwPjLMuRGKCfTWDbo1BfAiOf1azMYaNsm8jCDEmGbjNhwscQ0UedqlTpZlHfMWWmd46/9Guh2+ni++Jq/JMC4OwcMLjpDC7O4yjTT3H65eWJAFRFVtsRwQHqOqIy/+0alQWVZpGF6G96XS7sbtFuVB3X3gD1pTBpgX/apYEOlel3NGIwtH4Eu4p2AdA/oX+rzxSKGxUl/Jrp116I6C3Sr3WSkIgMikRGZl/5DpuBxJ/ZfgpKpt+xJu8J0LOn+Omp0++EHsKI8+fBP3UpCO2spp/BYKB///6t+m2n088xTeU9tT4Wxem3p2SPx8+0ZZTttoJtTH5/Mvfu3QzDH7cX7nVFYITdqLH7DRHFVZPjUXu8pmq/kFTKWaztuD9nwi+eRwQqE7vaUOGA9XemX5dQe3a5J4s1Z/32WOG5U55j5807uXLElbZtSlZYebnvpCCbsil3E9sLt2OtE4PqUeks8gOK0y/I2KgzFTNcLFLjRgPNJT6VOhsF1QW6ZPuVmoQWrIREkFVcyJfPsc36bXgqxLh3yKim12Uw8E4RWasBm+SYBnlPo9E+h3SlSqhwoPQAdy65k7uX3O1yv6Mi0y9miIje9kY6yUuWHljK2iNr+W7nd6r2d5fpl1clHrLWLL+sLCGpHxnp3jHsCW777q6XIWO+/hfWSF5eHvfecw8/1NUx3sk+44Ef6uq49+67W0t9lm+HzGeEsc4BE9Mm8tk5n/HkSU+qblOz+jUeIEkSK65awTcXfGOTodREfZlwgKmRN5v6AwxyPXYAEJIogucC3Ut5+QqXDvRRL8Dsw+rWBCqRZbsUvqeG6eyKbCyyhSBjECmRvh+3PHrnKsGI25/wXcOa4DaTaOOd8GWYCCrQwEVDLmJqz6n2uVUTHDqLGqpgyWSRAdLeMQTAjHUw7VffOPxAOLXOPiICIFQSZ+hBcH0yNbW+rw/6zsZ3mLdjJofj39c/eDQgDEa/0DwxwpHjKPc32PY4rP0X/HESmDVk70qSkGLMXtjqo7Ze447vPp7Hpj3GvMnzbJm4vlhbyrJMQXUBhTWFWEwiyEm3Z1m0Cg59af99/4ew7iZt54gdBrNzoP9/xO/JJwtJXTUc+BhWXSqkRb2h50Uw4D/ubeSSBKFdIdjFHOH7dJdS94pt6EDZASyN0raxsWLuIsvaygy4Qpn/npj8FPzQB8p36nNiLUT1g8QJ7v+uVQegco+qU/q7vx6bFrB2jCzL7CpudPrFO3f6tYm856Fv4O+LhF5wW2MqFLW2dEBx1Owo3GFzPh04oMupXVJmKuOhZQ/x4+4fqayUqWqcnyZrW7sfFXgr7zkhdQIBhgAOVxzmYJmHJ2mCK3nPoBbFqEpqS8ivFh5bJdqpE0F8vHg/NjRozyjqFdMLo2SkpqGGnErPJkEto2wVg/lyS6yQylATfdWUon+ELKhJhTXVF0Smw1kHYeAd2o7rcQn0vUGbxEQTzux/JksuX8JM+Q3AdxF8TaltqKWoRhQpTwoWEjAGAwR4uF5t2W+PJRxleCmGE6vV98+ztLaU3Cox+w+vEe/bTqefOr7N/JYTPjiBx5c/DkCdWVifzj78PKy6TCwoB95pc1Ipc8LSUggNDGVE8ggA/sn+x+u21DbU0i++H+lx6ZjrhXHU18/R7/22rljUnfJwrHRI7zkw9CHNxrbuUaI4hpZMP7BLfKpZcCtjbEKYa8/PUZHpp5aabNh0j6jHojMXDBY1QV5d+yrV9dVu93eX6eep06+ptKevSnC57Lvj34Yp6hyfvuSdt99mWmCgU4efwnhgalAQ777zTvMPup8Jp++Gro4VRtKi07h46MWqssIUvM1GCTAEMCltEucMPAejJ7LuQTEiA2CcTnVNQazZ/RFd5AKX8p7hqSI7R0NnMJlN5FTmOA1KVJ4jeP4sD5QKY0TPmJ7NlBp8iUfv3BN/h5HP6d8YB7jN9AvvCUnTwOKiEKpGHPZJuUEYd035ul3Hp8SPcTpOtRWzjC8zfWsuMxM0Olc8YGv+VjZWLqY6ZK9+89baXMh8ASx17vcFIaeYMR/GvyXegVozyk7PhKmOg369nitbG2DXK3DwM82H9ozpybwp85jVb5YtoLS+3nUtYk+orK/E3FiTWaoRaiO6Of2m/QKnbYXGWnPk/gb734eKXbDqCtclpepKhEQ0QGgX8a/pZ0Vr3V+//3/g+E+EFKw/sFqE86zWxfgVfxzEjnD6cfeo7gQZg6i31NvKEEiSva5ftpPKBOuOrOPMz8/kgaUPqGqqYsvbW3+hWO9GOfeP+BRZFr6Hsu3O9znxVzjNeXm2tqTT6dfGWK3NDQuFNYWUmcqQkGwedEfkNNrA/eogKt8GhxaoK9rta5afBctO1cUwozj9thdup3ej+t1+P5QdWHdkHY8uf5S5v8wlP18sNGJijk35OsXpV1hoNy5pITwonLEpIpPpr6y/vG6Ps0w/q9VKRkZGs36bWShq9aRFpxEZHOn1tY8mAgLsRmitEp+BxkB6x4oO6anEZ0unX0OjVr2jSE5VTPgITt2gXpLDF4T30F5vqO+/GjMbPXvlp0WncXLvk211L/zh9DOZTcwZMYdZfWcRZBWhgyEhnhkoHfXbYxGrbKXBIvpAQIDI8gDfS3xuLxQT5O6RqQRahcWm0+mnjuKaYpZnLeeXfaLughK4gBQg/rVAWXCXiqQ8jusmjM16OP16xPRg18272HPLHr/UZmyTfnvgY1iUBgXezyO8pVtkN4KNwbb3lloUp1+OilgZxemn1B5zRofP9LPUwaLesHW+un0zn3Fe68QLLh16Kemx6RRUF6iqAe3SUQG2gLMu4drqAihOvz4+Ui1323djhuhbj8hDFn32GZcrE0U3XFFby8LPWhhEg2Ihqi8Y9Vu46V6ryBMMgRCiIgVUtsKa62GDi7p+shUWpgoZ6jbEZV+y1EPlPmGgVUG9pZ7Qx0Pp9kI3WwZ8S3Rx+pUJp1+vmNZS377Ao3euJAnZ1sTjfdewJrh1+vW/Gab9LORGNbAlbwv/W/8/lmctb/VZy/nOlPenEPpsCj8OeQdG+cfZ2SEoXgcZj6q2E0ZHiQWdP8oMlJnKAAg0x+g3b818HjbdAfnL1O0/4ik4Yzd0mQZp52m/XmgyGFs3Xpe5shQAWx+CvZ7LfILoI8pYW+p4aPSY4hohcxcSEIJRDrVdTzeavvNGPg3n5EPpFjj4MRxslH13FOSx4VZYfTmUbm392apL4I8ThMSuK8JTodelHttpbBSthcWjIOtL1/s1lMNPAyHjIef7TPzU5fhmNBht76amEp+K089ZXb+DZQf5YfcPDsdaRyg22SLjVKFs46tINXdU7YffjoMdT+lyOn/bpDqdfu0MRdozLTqN0EDn2h5KR1K0c/3CkIeEFrgi2NyW9LsZhj/mfhBVwZAkoc+7o3AHqT0b6+X4w+mXsw6Acd3G2WSYFGPNsUZEhL1GmFJnRCuKxKceTj9XmX4t6ZT2dE1TiU+teFvXr+VCTTGY3xmSD79P9eicKJHT5Zn6ZoOoIfc3Ib3URqiqpaETsaGxvH/W+/x4yY/U1YkJ3rEYEKEX1/9wPdFPRbNo1yLbNsVBVFbm22srY2T/WJGNZjAIGcJO3DOt1zQA1mSvoaahhhN6nMAFgy9g76h34PgPxAL01+NtMjRN5T0BW4bJP0e8d/o1pWW91KOG2BHQ7xaI0vF9XrlXSDnte0/TYf8a8y9q59Xy5ulvajpOmUeqkfdUao2pzfTrsE6/+hIIilZnUAnvCbMyhYFOZwKNgcyfOh+Ap1c+TbnJtbVTmYPW1jq2NZXWio7uTaZfm2Ax6bJ+85byigrU/uW6AOUtrdOyVci91zpPq120cxFvb3hbVWYn2A1cno6tW/O38vra11Ub1JpRnQUZ/4Wqg+r2lwxQsg6KXSjvWEyQfjV0naG9PTri0umXv0zIhh36StW5goxBthqaSv3ElijvyKAgz22U+0uFMcJfTj+PMBWIZ+wn3Mp7eshXO77ihp9uYMG21rWYWq4la821mMwmJNrI+Nxeyf1NOBGcyB23xJ9ryuJa4TAKNMfr5yga/pjIzlI7tkUPgoh22pclSWTsTlI3BrbkcPlhFu1cxOa8zbZ1SIm6GArVKGNtbHC8bZsuz3LP/1orO4R1h8BIUTt9+koYMFdkrC89WSg+NWXQPTDhU/F8W5J+DQx/UmRSOqOuGOp1GtCMweJ8ZjfzDWMIDJ4nagh6wUunvsTSK5YyJmWMbVtKoxK1M7WR8jpxr9Eh0aquUVsLyRH7PJY8143IdJHRPsBJkFN9qZCFbQv5URV0Ov3aGTZpTxf1/Boa7PXf/Or0kyQhT+QniQmX9LwE+t0EnmbsNKFXTC9iQ2Kpt9RTF7MNEDKTFm3lVzSz9ohI9x6bMtZmnDkWpT0VlGw/j+v69Wx0+h3UL9NPk9MvodPp5wjF6VfgQhnBGd46/ZzJe8YbrFDjReHOg1/AT4Mh6wvPz6GVqoOiEPumu7QfW74Tlp0GWZ4XFv55z8/8Zn6AsrANflmgNcUfWUXHAlX1VWzMtctOKLUXfO30214gMv36NEpQBge3XaBeRyM9Np3uUd1psDaw8tBK5k2Zx4LzFjAxbaLYQZLAagKr6CTOnH4bczfapEH14Kjtk12mwphXmsvzeIsxVETAa5SFDjAEOJTmdYey4NaS6ZcQ6trppxjKO6y8Z2hXOG2TkFl1h8EI0QPsMk86c/GQixmYMJBSUykv/vOiy30VR4UsN88cUpg3ZR6meSaePvlp1dc3m+1S+m3m9PtpMPza9pl+0VFRqO2V+UB0dAsjlaUWFnYTcrBOuHLhlVz/4/W2GsXu8Dag4vf9v3Pz4pt5c722YAEADn8n5OfKMtQfc/JyOGWl888DwmDsG9DvRu3t0RGXTr/oAY0y2cNUn0+pl6hkn7SkZU1xT7Bl+sW2U0cBwPqb4atovzn+lC5YV+d4TKQmBzbfJxxQGnBVQ7flfKfB0kCoBKllK1s7AI5lel0OM9aK2mYqMISVsbrfidx+oL+tJpivUPppkCVOv2A1Y0hjdtZRsqCJH6M5Q1bhpX9eYvaC2Xy4+cNW6xC9UJx+MY116DxV/2mGxQQbb4PtjzvfJ3GC+Fm5Fwr/hiPfN/88ZqiwSzuS8E87X9TYC3BhUNz9BnwdCyU6yELGDofZWZB+lev9AsKE07r7Wc73OfQVrJ8rpECdcGqfU5nWa1ozB57izK9ykvBbUSeMSNHB6px+IXWZvH1GH8aFPKxqf58y8A6R2e6I8p3wzxw48oNfm6SWduC96aQpSqafq3p+eXliARgWZo/U9wsVe0Rhb7Xa1R0ESZIYnTIagIN16wkNFY5VZ2nJeqFk+nU6/QRKPcW9e13u5pSJqRMxSkZyq3IprPYgrawJzuQ9HXHzuJv54twvuGToJV5d82glqXH+6Emm35wRc/j5kp+5/XgX0kEuaCXv2Sht+Ix1EJy5z6NzAqIuQvezHEd1+YrgOBj9qphYakUyQsGfXtVj/WDzB/xQ/jglkX/5xelXZirDZBYP8Kh1MPiRkcliktrU6eevTD9F3jM9Qjj9juHyipqRJIlpPUW2358H/7R/sPsNyG5ceJ62CcaLWlNNa/oB9I7tTUJYAvWWejbnbfaqLW9teIvhbw7nyRVP6mLQPGYITYHzy0UdWT+gzCNzc92X0dJa06/DZvpppaECClf7ZL1jNBh5ZNojALz4z4tU1TuXQgsKEpnR4FziMzggWJO0fH6+WOMEB7fhmqPbmZAyq40ubuesSy7hY5UW4I9CQ5l9SYv5V0C4qMWTcqrT4xTpXLXrEm/lPQuqRYSdVslXQKjonPi7ttpfgR2jrIESxOm4pl8PIRumQaIyPlRkm7jL9PPGwaDU9GvXmX5JU0U2i44St64IDbXX9q6sdLCDuVrIr+Uv1XReWw3dFk4/s9keBG5z+lkbSDDCsL1PCknwTgThaUK2WeV3ISkmkpLI5RRYdtvq0/oKXTP98pYKKUx32VQdDVkWAcau6tc5YWCiUIHLLMpstQ7RC+UZRgeJsVeX9YchGGasgxHPuN838Xg46xAMbXQ+HfoaNvzHdV08NcSPgdSzIXqwd+fRm7zfYfcrQgpUA0pwoLNyTYrCRVRwlKrzVdcG8vv+OZSGTNfUDp8hy1DkQD0nqj9MWeTakdqGdDr92hiDofkjmD91Phuv38gt425xekxTaU+/Bpfsehl+nyLSV9sDS6fDn/osGs/sdyZXDr+SvvF96NU4t/alxOeRiiPkVOZgkAyM6jrKlgJ9LDv9hgiVVbY6kMRWQ2RwJGuvW0vZPWVu69O4w1mmn8FgYOjQoc36bXpcOhcOudDmOO6kOd7Iew7rMozT+p5mqyWnlZaLbiXTz+OafgrBcTDlO5cFjnUnMErUqugyTfuxkX3gghoYdLfHl1cCUapCdjleaOvMXb/dRejjoTz999NeG8Ec9dtjjVFdRR3KjbkbkRs9AUo0pq+dfv3j+zMkaQi9IoYCnY4irUxMFVl9G3I3UG+pxypbRR2RvW+12rdlhK0kSfxn/H948qQnSYlM8aod+0v3szV/K/nV+b5zxDcxovi939aXw+KRQupHTyTJ48n6A0sfYNj/DePXvb+qPqZLF3G52lr3sllq5D0bGoThEzpwpl/297D7dfXZKJkvwJIJqmXKtHLOwHOYO34uiy9dTHig8z+qJLlxVniA8p2IjfXdGtJt3x39oogyb2Ouve46ljU04EKcEoA1wJ/19Vxz7bWtPxz9ostgrMSwRqdfjX+dfknhHmRsGAIg+SSHdaNcX/Rv2Hyv4yiDve/AykuFNFobooxdyvrOW+LDhOFZMUS3RI/ajK+c9gofn/0xE1IneH4SDXj0zu13o8jk9BOS5EbiM6InnL4Lhjyo6bzKOvNIRXOnn6PajA2WBooskDngUeh9pabrHPVY6lTLA8dGGwmpF87WrHLPg1LVoDjng/Rw+uX8DNsehaoD3jdMB3SbKxetgu97wf4PNB86IGEAADuLdvpM3jMmJIapPacyIEqsZ3XJ2JQkiBkMCePU7d9UBST7e9jzpnvVuaUzYPk5zj9POQ0mf6P9veuMIz86XB82ozZflB1wtd+wR2H2EQiKcbpLSW0J7296n5f/edm2zV0tapu8p8pMv7zqPry85n1qwiep2t/nbLkffju+dWZmcBx0PxOi+qk6jb9tUseuBaydEh4UzsiuI+kb39fpPtnZ4qdfpT1BpCiPflkYn9sDAREiylIHbhl/Cx/M/oATe51I795imy+dfkqW3+DEwYQHhR/zNf0ABg8WEc25uZ5JQYIwbAcHeP/SdFXTr76+3uvzH0t4I+/pLS2zUYYnD+fxEx/n3vRRIkJLD/L/EtJtvqQmxzu9dy+MzgqK1Gp18G6/ZPopUliJ4Ym6ZBUd6/12WJdhGCUjhTWF5FQK3T9/Zfq9Put1Mv6dQb9wofnf6fTTxvDk4QBsyd/CsP8bhvERI1sGPW+vOZb/F+x8EWSr7Zk2jbCdN2Ue9066l9ToVK/aocgjxYfG+8bpJ1vhx4Gw6jLbJr/225psqCsSWV56U7QW9vyf+9S7FhwoO0BGQQYbcjeoPiYoyG4UdRdp/fiJj7P40sXM6uc8gK7pwr3Na2p4yv73YcNckfWuhpTTRB2WEO8CyJxhkAy8dOpLTEid4FbCVTGgOHJWXP7d5Vz+3eUcKldfDFt5f7dUqtSbjvDOTU5O5qmnn+aM4GCnjr81wBnBwTz1zDMkexCZ6Wmmn6djq8dOv8LVQtZT4xgFCKn7HU87dpKXrIesz4ScWBui9CNn2Qf8cw2sVu/A8Ye855iUMVw27DKPAx89oSP0W5e14AyBwuiq0T6kyHvmVeXZVGEAlD+HJNkzDBusDdTKUJE0XcjpdWJnyST4fbKqXaOiIKxOGN32lngo86QCq2wVwXLo5PQb/hhM/9u/aj9u0KXfxgwVNa3jtUtvK06/rPIswqLFpFHvTL9T+5zKsiuXMXfg84BOTr+Cv7UrOpRsgiVTRGb8zAwIinW9v2R0XhrLk/etO3a+BBvvcH1uqwlKN0GNC1m7kCQIS3FZ1quwupCrv7+aB5c9aAsoduf0U+Q9VWX6yVZNJZf8QtoFMPCu5lK4VgtU+dBxoAOdTr82xmq1aj6maaafX+lyAvS/tc0n7jamfAeTvtT9tP5w+q3PWQ8IaU+z2Z4F1UXHMjKqcVVc1o+EhUG/xuCILVvati3OXjBWq5Vdu3bZ+u3ekr08v+r55rJrnTTDG3lPgF/3/sqDSx9kZ5H2wrgtI22HJA3h/sn3c2L1KlinQ32R2nxRZ2/djb6ZuClsfRC+6yqcf55SvA4OfOLx4Uqd2aqQXX51+qVGpXotk9Sy3x6LhAaG2uRXFIlPf9X0U1DWpLrV0zhGGJo0lG6R3RiRPMK2WDLFjoaYxvT4g5/CxtuhvswWYVtRYc/O0gsloyE+zEdOv4ZySDgeIsVEwO/9NmYwzD4sajbozb53xHuirkjTYcO7CIPi1nxtEgiuHEVNGZAwgFP7nErPmJ5O91GM5KGhdqnJDsfoV+CkP9XX6UsYD4PvhbDuvmyVDdnF/MGZAUWWZb7e8TWfbP1EUz0k5f0d5cP4Tbd9d92NsOsV3zVAA7fdcQf3PfEEU4xGZoSE8CnwO/AJMD0khClGI/c98QS33e5EZn7b47DUuRymp5l+nhq48quF1Jhmp9+Op0Sms8WDVLhBd8HpO8UY2pJxb8IF1boF6XqKO0MklbugQv06wx/ynv5G8zv34OdCcalil28b1gKXTj8QWaXl2taMieGJBBoCkZGbSU02neso8RmKUzDQ6Ju6rx2anpdBrytFEJcboqIgvE4kOewp3uOzJhkkA6X3lHJnbR1B5iTv+6QxBBInunSG+BPd5sqBUaKmtQeKQglhCTbFiKoQMR7o7fRT0COLGhBOmt8nC4lOLRgCoXiNCGiJcp6kY2PazzDZSaD5qkuEcp1VxwXbyKfhJDfyxuE94LwSGPZf5/tYTKKOYX2Z013SotMAqKyvpNQkHrg7eU9F0r5pHUCnbH2IW9MHkRB2uP04/eJGwshnmq8PStbB9+mw41nVp/G3Tap9jFadAJBVlsW/f/y32+Lbbeb0O8oxW81k5GcQnSIWZvv3+86O/9+p/yXzpkzum3wfhYXiOkFBdmkunyHLsOY62Pe++P/6W+DHQS6LtPqT4Y0Bc5s3e3a8LMvMXTyXIW8MaSXRof4c9gmFu/o1v+z9hTuX3MnTK5/26FrHAkqmX0WFk6Lrbnjxnxd5bMVjHjlWnU4Mhz8Bx3+ovTEtCe0Cx70HE7/wrdZylxPFQirMC3m+zOdh9eXq5c1aoMh71gXlUlzlW6+fLMscLDsIQM+Ynp31w3RCqeu3KW8T4J9Mv+r6aluErdL/O2v6aSM8KJzs27NZfOliDI1GhiDJYJ+gDPgPnLwCAiKIigJjYzJT0+eaVZbFgm0LnGYlqMHm9GuS6ef1oluWYdXlsOs1CIyBSQtg6EPiM3M1BkulfT9/4QtDTr8bYdpvmutfKU6/LfnaIqH0lIRUFu4dVtoTIDwVktqJNFATcitzueXnWzjj8zOc7uPsWVbUVdjq3naJUB8xqEji+dLp55Z970Hub23YgObcdvvtZGVnM2XePF4aOJCLQ1O4M2Igw2+cR1Z2tnOHH0Dlbijd6HRuZXP6+TnTT8t3AoD+c2HUS54F9ob3EPVsnNEOgoXdjoknr4AZ7oRe7UxIncCVw6+0ZeK3xNvAmMzCTP5v3f+x6vAqz07gD6r2Q94Svzt0Xcp7Aqw4R5Sj0YBBMtA1UsgtNa3r5+g5TkidwCU9xjL87xNh+1OarnPUM2CuyIRTMY+KioJwU6PTr8R3Tj+FBlMQEpJ3a8maI1C6td3YzdoTSrZfsSQc7nrLeyro5vQzhgt7UI+LtB0XM0TIXvbTIXgcSaxvDAE6nKuRuNEiW9Nbu1TuEvihL2QvcrpLaGCorX6wYrdxF3S44LwF1M6r5ZqR17hvQ0AYFquBktqu7cfpp2A126WMA6Mg/TroMrUtW+SSTqdfO2JL/hbe3PAm/9vgup5Imzn9Nt0FPw/TNxrBG+qKYfsTkLNYl9PN/mI2w94cxrqqbzEaRYHoYs/tYy4xGowMSBhAn7g+zer5+bxGo6kAcn+BI9+L34NiIaIX1PvoRjWiOP22bvXMxidJEn8f/pvthdtZnrXcozbU1dmv7e4Fs/bIWgDGdxvv0bWOBcLC7H9HT7L9mtYi00rLxVpuZS6bcjeRHdJTyHfpQc9LIDJdn3M5o9elMN6NPrs7+s+FE37E09d+dEg0SWFiYlfMbo8cuGopqC6gpqEGCYm06LQOGTHdHpnWcxqz+s6ib5xYZPvD6fefX/5DxBMRvPzPy76rA3cMUW+pxwiM/GscrLxYbIweJBwaxiBRniJGbG4aZXvG52dw0TcXeZWVrmQ06JrpV1cIhSuhcEXz7aZCDL+OITX3CfF77q+w5novL+aGve/AkZ99c+7YEdB1uogQ18CwLsMA2F28m9oG9Rk4ajL9rLKV51c9z4ebP6TO7HxAV4zk7oKg2i2WOqjN1T6pXHUF/O5BDV0NWGUrb254k5/2/MSm3E0O93H2LJVslKjgKMIC1T8cf2T6ueXcQhEs1Y5ITk5m3gMPsG7HDi657ghjp+7goosfcC/pedwHcG6R075tk/dUmennzXxHlmXP5T2TTxR1oz3FahbjZ9E/9m2mAjj0jeoaX77E7ZiocQF+2bDL+GD2B5wz0HGtJm+dt38c+IMbf76R51Y959kJ/MGQeXB+FYT61yClqFQ4zfTrdaVQptI45r91+lv8eeWfDE60Z6w6mut8ef6XfHr+VxjjRkOodsnfTgRNM/12F/ne6afLvPXAx7B4OBSt1KVN7Y79H8Hi0R5JFQ6IF06/PHMmoH+m39kLzibp2SSWHBFZc17bBEK7wOD7hJKdVkKc18FuRUOFyAA7vLD1ZxM/gxOXaL++OywmMLtYANSXw+FvXWdERw+EQfdCtIMM/iYoSiEtnX719Y4VZyRJIiQgRF05psH3c//f27DKAe1rDSJb4dfx8OsYMc+JHiTsdB5I4/qLTqdfO2JXkUiHVjIqHFFZaZ/kpHiR9OERVrMYRNTWxPA1shW2zIPD3+hyOsW4sjl/PamNpW98KfGp4Nd6fqFd4LQtMP5dscAZ+l848bfmusRtyIABIgukrAwOqS9R0owTeoiX919Zf3l0vGLgkiTHGSlGo/37rzj9xnVTWQD4GESSvJP4HN11NICmmkYKLaPBPtzyIaPeGsVDyx7S3hBXWC2Q/YOI/tObOp0c8onHQ7dZ7gtOu2BAE4nP/Hx9muWIA2WiOHq3qG4EBwTrkunXtN8eq1w18ip+vORHLh4qnEVKZnlZme8SqVYeXkmtuZaeMT07nX46UGYqA6AyeRYkTrB/YDWDReinNn2uCsp70RunX9Oafrpl34YkwcwtMO5/zY2uwQnIsSOpDR8qvpzFa+HgJ0JS2RfIMmy6AzKf8c35lWtozLROjkgmMSwRq2xlW8E21ccpgTaunH5lpjLuXHIncxbNcVlXrsNn+pVshO9SIFOj8Vy2AFafZpl2i+rG2QPOBuDzbZ873MeZLKHi9FOirNXiL6efy3duYCQERvi2AV6Q0GjPK1KjxuvGWTSz70w+P/dz7pxwp6prK33WU4Pm8jnL+eycz+gaoWFRaanz/nteVwjLz4BdL9u37XwR/j5P1A5qY9zKe1YdgAOf6vaO8TZYbX+pMED0iumlS3vUonmubAzyQ8Ryc9zKe/a5FoY8oLldM/rM4ISeJxAZbM/IdzpvDe8BJ/0BvedousZRT305/HmGCMp3Q3AwRJv7EtSQRGyQ7+xQKw+t5LRPT2N1yEO263pMl2kw8G6Ia1/Gfd3WuJZaIUNfpz1N76qRV/H5uZ9zzeg5gLBZN+hYQSi/Kp/CmkKsZpEV59VztJr9V95ICoAt94o1jMPPdR4/8/6ABaFw4CPn+1QfhBXnwqGvnO8T2QdGPAnxY1xezpnTD1zU0NWAt3MinyAZhMO431yhUtMB6HT6tTFNB+ldxe6dfjmNJZ0SEtrgyz/6RThjt98nd04JSYQZ64QciQ6MSRGD2obcDT6t67cmew2Xfnsp7216D7A7/TyoD68NxdgUHCf+Qft5lo0EBsKgxrrIntb189a4qbxcwsJa/3mMRiNDhw7FaDRSZiqz9dlOp59rFIlPj5x+KcLpl5Gf4TIbwREtnX5KHYb7TItF1rJeVO6C5WdqNxg3VLiue1CyUdTy2+tlll9TvJAkeX3W61xZto9uJRfZxi1fcKBUOP0UY4e3xpOm/bYTO0q0dEOD+9pfnlBSW0JmkYj4nJA6wVbTr9Ppp5012WtIeT6FBmsDFqBk1Osikh2EBMsXgXDwY8Du9GsqrTO151QAlh1c5nEbEsMTSQhL0CfTr2AFFDVKqQVGQlBM888lCcOkL0iZ9gzGgABIPhmmLBLzPnfIsqgXduBjmyNUxUFw0jIY4SOpblMRfBkOG7XVC5QkySYfp0XiU428Z1GN8GhEBUcR5CIYpMNn+gXFQN9/Q8Jx2o6b+Cmc/JfP58kXDL4AgG8zv3VY28/Zs1RqtyVHaFs8+MPp5/KdK1uheD1UexjZ5wcUp58qtZfaXBExX33Y4ccDEgZw0ZCLbKoV7vBmviNJEmO7jeXioReri6JX2HgbLOwuxilPCe0KI5+D0U2cfkMeguPeh+6zPT+vTjTN9HPo38z7HVZfpslBWWeusxk6W+JtYMzu4t0A9I7t7dkJPEDTXLk2V6xN2iCL0628p450BqtpJDASCv9WlSkmSdA9eBCnbMnnf1P0Ue1yxL7Sffyy9xfyA8Wc0yv7acJ4UTMtoP1oDeq6xu37L5id5dbR44gJqRO4aMhFDO3em4BGtUo9lWSUEgNBFmHD9Oo55i+FbxIdZ9/pTUAYTF8N49+xbzPXworz4KAPFA8iekPahSIwwRnhaTDhM0g92+vL9YgW11HehQaD/dk4WoPMWTiHK767wn0Zpr1vI2+6B8ksyjy0O3nPtPNg6INwaAH8NBRKtCUn+Nsm1en0a2OaLvBsTr8E504/RdrT71l+7ZX4MbpFiypOv4yCDLr1ELP1Awd0OXUzVh5eyWcZn/HTnp8APzn9KnbBN0mw793Wn9Vkw7KZwkjWDhgxQvz01Ok3ucdkJCR2Fe9qVoxbLcoLytHLRZZlKioqkGWZdUfWAWJBphQv7sQxitOvoED7sT2iexAbEkuDtYHthds1HdvSeFLfaADOC0iE2JHaG+OM6EFiMqfFYJzxX5F94EIrHdkKiZOFPru3VO6Dr2IgY77HpxiSNIT+Sb2RMPrU6ZcancpVI65iVt9ZgPf6/U37bSeQU5lDflU+wcH2v6kvJD6VWjT94vuRGJ7YWdPPC7pGdiW3Ktf2ezMnTXgapJ0PoaKouOL0ayqtc0JPEQyzvXC7Tf5NK1tu2ELhXYXNJHc9MoRZTLDyQlg+26X8jAz2fps4Qchjqqm3Z6mFnF/gn6vVO2wkA8SNEgYdXxAcL2qzRg3UfOio5FEMTBhIgIaaH2rkPZUaY+7mLx0+0y96IIx9A5Imt3VLHHJqn1MJCQhhX+k+tua3Vgtw9ixzK8V4oLV2mz+cfi7fueZq+HWsV3MRXxMfL36qyvQrWi0i5ltKFHuIbvWKtBCWKmryBcd7d54Bt9mVY2RZGMZ7z2kXAaZKP2pat70ZydNh8jeq1waltaWEPB5Cr5d72WprNsWb5yjLMutz1gOodhbrgaa5cuFKWPsvKPBMVccb3Mp75v8Jv08VPzXyWcZnXPf9dbZ5kqO5TsIzCQx5Lpaq9bdBwd+ar3FUIxmE3HFTB4cLlGeptxRkUxSVioB6Mb557MB1FaTbhrS3Na4kOQ4+9BalxECwRTxHr96RsgxRA3xfnkUhYZw9uFGWoWKHqGtc4zhYyCsiesGkLyDlVOf7BMVCz4tFfUJnyLLI2t10j8vLKZl+WeVZtm2uMuu/2vEVH2/92GaTc8rhb2HP/2EyC4Nsu3P6KQRGiXlt4xpcLf7ur51OvzbGarW/QNTIeypOv+7avlf6kP097Hu/DS7sAks9lG0XGTNekhqVSmJYImarmdrozQAc9sFYrERrD+8iorf94vSrLxULuvCerT8LToCS9e2i5gLAsMYErO3bPVObiQuNs0m1elLXTzGsOHq5WK1W9u/fj9Vq7aznpwFF3tMTp58kSbZsvw052qJoWi7WlAnGwojpcPyH2hvjivRrIExDXYugWEiaBt3OcL5P/BghH6OH0y+kizBqh6d5dRpFhtiXTr9JaZN476z3uGeSmGh6GzHdtN8e69z88810e6GbrXawIylIvVCcfhNTJwKdEdPekBqVSkxIDAC9I5OJz7gPDjRKxUT1h0lfQsoMAOIaE/mbLrYTwhIYmjQUgL8Oem+k8yr71hgCJ/wgamG5iJZ22G/LtkGpm4iggDCY8Imok2EIFNv2/B/ses250cZUqFl6UxOSBFN/9Khm1lMnP8WOm3YwZ8Qc1ce4lbLDnumXGOY6e7LDZ/p5isUkguEOfe3Ty0QERXBqH2Gc+Tbz21afO8v0U6R+u0dqWxD6w+nn8p1rCIThT0Lqub5rgJdokveMHwfHfwIJExx+XG+pZ+HOhby94W1VRh5vpKx+2/cbr6x5hS15GqMmB98HJy3VzzmX+Rx8boCGSn3OpwOBgaAE1jscFyN6Quo5ogyGCmJCYghtfH9lV2S3+tyb+c7hisPkV+cTYAhgRPII7SfwEE1z5S7TxHu86ym+b1gL3Mp7WkxQliEkZzXy1N9P8c6md2z2g5bPUZZlimuLCTOXEbH7JeH076Q5BvUZLEpQsKqx1kOUDDFjg5gce7wG2fcuLOwBxet0apk+6LrGlWUhc5z9g0eH/77/d15c/SLB8cJIoJcz1ypbbU6/gAYdMv1SZsCMfyBmqA6tU0lDFayeAzufF3admVtgoDb1D78iSVCeAVX7XO52er/T+eOKP3j1tFdt25QgwZbyng2WBmoaxAs4OiTa9fWn/kT5hC1Y5QAkqR3bDkK6CjlplXMHBX/bpDqdfu2E0tpSW5HvfvH9nO7Xppl+u16CTepqEviNAx/Bz0Mg33sjliRJTEgVi7Y9ZiGBlZPjuAipNyiRvMO7DEeWIbcxeN+nNf0SjoNT14lI85YYQ+CsgzD6BR82QD09e4r3THW155MFReLzj/1/aD62qbynK9bniijMTmlP9yhR06qkkhyg1PXTIm9mNtv7bstMP1dSZl5htQjjsiMHuqkA1t0s6h0A9L0JpiwEZ9kbehugAyOEUafPdR6forq+mt/N89nU63Jycv03Wel0FumHMr9YlyMWrTExYrsvnH4rD4tC951OP++RJMlm/Ht8ygMEHfzQaWaJMt62jLCd1nMa4F1dPwWPn6Vi9I4bbXNSqsZUAItHwOb7nJ877w/xMzgOukwV260W2P2aWGhXOlm8brgVvowEswsvWRvhqt6eM9TIeyprjvgw19k9yqK9wzr91t0I627SfpwUAJvvgX3qMha84ZwB5wDwTWbrGuXOMv0envowdQ/U8ci0RzRdy181/ZxiDIHB94oaw+0UTfKeYd2h16XCaeQAi9XC2QvO5vofr6eiznWAqix79578fNvnzP1lLj/s9sxgqxuGIBFoVn2wbdvRBElSFwyhVgJfkiRSo1MBOFzeOkLYm+eoBJUOTRpKaGA7TW8IjodupwtZVz/jNjss5VQ4r1goIGikZYmQls+xobEOWEY9VExfA72v0nyNo56aI8JB5my+1YT4eDiQ9CpXbuzNQ8se8klzFGdRkNnLTD9DIASEQ5h3wbPtGkmC9TeL+bIH3PHbHdz+2+1URQkbmV6ZfhV1FVgbg/aM9To4/doCSYLiNUL+VpZFRp4a9RJP2P4UrLne+edFa+DbZNjrZn571kGY7DrwLTU6lRN7nUhatL1fOJu3Np0DRQZF4hLJQI1BlHkJDW0XggGOSRgH6Ve3dSvc0un0ayfsLdkLQLfIbs0KCLekTTP9Rr0AU75rgwu7IHEiDLpP6BfrwPTe0wFYlbeEkBCwWOxOOT1osDSwo3AHAMO6DKOiQmSxSJI9G8qnOBsxAxpHZ6tFZAW2IYGB9qzH7NbBk6qY2XcmXSO6khKp3TvuSt6zKQvOW8Daa9fa6rF04hxNBhQH3Dj2RnbdvItXTlMvQass1KD1Ym1q7XrfyNkWrxEGxm2NRrimGcglm2DP65D1mfjdYBSTvT1vQsajzc9jKoAf+omIsHZEcEAwX+U+xZH4T9hbdNBn19lXsq+ZXJK3Nf06sTM2RRSfX3dkHbIs24wnejv96i31Nu9/LscAAQAASURBVMPVxLROp58eKOoA64r2wrnFMLLJonzjHWKRhz3Tr+V4601dv28zv2XwG4O5Z4kX2bc1R2DJRFGr1BNCksQ9D3lQ/F65D36bAAWNGf2Hv4GlJwsHX1MMRjjlH5i1DaL6Oj530hSRrR3gQ89W9g+w6gqo8+xFaJWtmK3qotDUyHvmV4macF3CXUenKnOiDivvWbTas8h8Q4Co83j8x/q3qQWn9zud/vH9Oa3PabbawwquHLhBxiCXa8aWmM12J26bOf06AMqctbDQM8WRpoQGhhIeKDqP4mh3RkOD/XqeSFntKxFG9j5xfTQc9D6s/bd39fxa0v9WODvHv1kUKnAbDLGoN/yl3hmdGtXo9Ktw7vTzZN6qlI9o10GlJs9kwvVAsZdUVjqRavUCZZ70V5YIJm/l9Gscn00yGKMHQ0hneY9WlG2FNdeKOpluSEwEWTJTaD5gqwGuN7ZacN46/XrPgdN3aM7o6XBM+hJGe2YjGZggJOyrQ3cC+mX6KRKt4YHhmOvEA/TYJnDwM1h5qdM6vD4jIBxO+hMmf+t7D1bhCsj63PkExhgCkX1b11PXCWUN0jLTT3H6hQWGEWgMdH6CguVQuJLaGhGE02n/8Z5Op187YWy3sZTdU8bvVzh/QcqyyDwDP2f6yVYoXAUxw4VhpD0RPRBGPAExg3U53cy+M3lg8gM8Ou1R0hoDFg7pWGt+Z9FO6i31RAVH0TOmp82hGB8vnF0+Ye0N4uXmTjvZXAtLJsDKS7xf5XqJ4tT21Ok3PX062bdn8/DUhzUf60reEyCk8c0TZAxibLexHjkWjzWaZvp58tVKi06jX3w/DBoiopSFoMGAraC0kuk3tmYj7HtPe0PckThByDyNfkVokS9Msxsb48eKzL4+N9j3lyQ4+CnsfqV5/wyKF5OxhOP1bd+hr8RCTKXhuCUBhgD6xQ4CYF9Vhk+GCbPVTP/X+hP6eCg5leKFp0eNm5DOGSMAw5OHY5SM5Ffnk12R7bNMvzpzHfMmz+O8QefZJMs7nX7eoTj9NhdsFZlsTesZH/oSskVQliN5T4Bpvabx+bmf88cV2jPg9xTvYUfhDnKqcpBlqG8crjQ9y8KVotB5dZb7fRtp1W8HzIXExnGxZD2UboLGYA6Sp0OPi6DX5a1PFBgpFtwg6hi3HLz6/hvGvam6XR5Rvh0OfgxV+zUfOmfhHKKejOL7Xd+r2l9NRotS87hrhOtMjQ6f6XfaJjhlpWfHJhwHIa7lT/UgNjSWnTfv5NlTnm1lCFHjwFVLZaPaoiRBhD7l0J3i9J1bkw2/HidUEdopypzVZHKTFQYiWHJhqnDoOyExXHyHlDqazmjqwPDkPakEEKfHaqhTlP8H7H+3+ftED3yVweAFbvtSwvEQM0z1+Vxl+nkzb3146sP8fdXf3DxOuxy0t6iaK9eVwLddPMug1oGwMHsQitOyEdnfi9q+GpnSQ9i5thVso6imyGmmX5AEgfXF7VIdoM2JHy/Wu93Pcr9rPISbRDDWnuI9PmmO4jAKNMcTFNSOM4a8QNc1btfpEKt+HGzKgIQBAJQF6Ov0k5GZ2nMqE1IneG8TKN0iArCNbbAYDe3in3fjxC/g/HLnX/bY4TB9BaSd5/o8JRtFKQk39SwXbFvAQ8se4mDZQcC5vGd5nVC7igp2E3W261VYMhlTjbBXtdt6fh2I9jcjO8YwKgLzCG1bZbB0RG2t3dAS71qNR18ynxXR2cVr/XjRtqFXbC8ePfFRJqZNJFXM5XWt66dIew7rMgxJktjfaPvp0UO/azRDloW8SvUBcCdpGBAqojJjhoLc6BRoKnMiWyFnsSaDnacof3tPnX4GyaDJQdQUV/KeRqORAQMGNOu3nbhHkwFFJ5pOCpU5z8y+M7l/0v3sHP4anLDINxfudakwnkT2FZkpikE6OE4sgFpOwMa/B2ceAFM+7H5dbDMYYdpv0Pdf+ratcJWQXKk94vEpRqSIyO2SwK0+KbyeXZGNRbYQZAwiOUKk/Ho7we/st3bCAsMYkiQKd6/PWW9z+pWX63udyOBIHpjyAF+d/5VNnrDT6ecdXSOFc2bFgaVi0VrbRIbgtM1wspD7VMbb0lKhVqAQExLDRUMusp1HC4fKRfRTWlSabR4KGp9ljwtg9mFVhiBQ0W97XAizs+2y5UHRMPFz15GrR36CH/pC1hf2bZY65/vrSb8b4fxKEQCikUBDINUN1bbsWXcoC2RXjqLbjr+NxZcu5rJhl7k8V4fP9AN7bUetyLKQ6zZprw2lF46yk6yylekfT+eK765wKxnZFEXaMzJSBET5Cpd911oPtTm61GL3FaWleRzKepR1fw5kWHo3xg4cyGOPPkqeo2LGBiNE9hMyn05Q6ma6y/RT5jpBQdqfT01DDblV4p2QHqfB6Xf8x3DWIRH5f5TjLPvAxsRPYeQzqs/nKtPPm1rUYYFhTEybaJur+QvVc2WLCdKvhcTJ/mmYA5Rsv/x8Jzus/Rdk/FfzeRPDExmcKALJl2ctd5rpNyEEgr7vAfs/0HyNox5lvRua7HbXxES7029vyV5VdU+1oijHBJnjPF9/FK+H9bdA+Q79GqYTPlnjWkx2+4UGFDt2vlU4/fSS9+wT14dlVy7jt8t/82psBWDk03BeqbDRHK0ERurjXNzzBqy+3O187fnVz/Po8kfZnLcZcL4GKTcJY0N0sJt6fv3nwvi3qWnM6uywQYcu8LdNqtPp18ZoKeKoGOVCQ8WCwKfse9cu85h0Agy6t01021Wx7XH4caBHLydXKJl+ejr9cipzMEpGW8S+4vTrrY86aWskCab9AieqjOwf97ZY8BgCob4MfugD+z8Un+1/H/6cKRwHIJyAh77xSVagkunn7d/eYrWw9MBSW4aXGlxl+lmtVoqLi3n5n5e5/ofrWXnIw+jxY4zgYHtUuacSn1/v+JqLv7mYbzO/VbW/I2md2QNm8/hJjzNhwMUQ7itPeyMxQ2FWpsj+c0VUX+Ek3PgfoaNftk1s11AIXTWD7xOygF7c+4hkEf1XGbpVV+ljhQOlBwDoGdPT5rj31lmk9Ft/F01ur4xJGQOIun6+rOnXEo+ywzqxMb33dM4ecDb3j54jattlPmf/MDjeFtgTHS0MxrKs33PNKhfBPj1iejiUTlZNSJLqhaiqfhscry1sO348xI2BsMbIIlmGJZNElo6vFQ4CozzOphnffTwAa46sUbW/mky/3rG9ObXPqQxMHOjyXB06069su8i6ry/z7PiCP+H7XiIj3w/UNtSyImsFdWZ7J3OUnVRYXcjv+3/nk62fEBqgPgTaX/X8XPbdiN4w+xAMuse3jfCQF59/nh7du5Ow4wmer9rJW/k5/GfnTlY88QQ9unfnxRcc1D8/6Q+hOuMErZl+nrwj95eKBWVMSAxxoXHqD5QkVcb5owFVNf004Ct5z7ZC9Vw5LAXGvw09L/JPwxygOP0KnXWp8W/DqOecfOgaReLzs4zPWs1blUy/PIsB+t3c7iRs2xUqpMwTEiCsvhfIBqobqm2BC3ry99V/s/2iOhIqT/Z8/VHwl5COr9PJi6Ujuq9xd78BC0JF3TeNKPKeh2u3Y5HqfBIcrIytXmV/+UjWst3QUAH5f0K1E7m6ij2itEzpFtfnSb8eJn/nNiioZ0xPALeZftUNYoPbTL+kSZB+jVv1tY6Mv21SnU6/NkaJaJmzcA63Lr6VIxXOMzAU441ipPMZRf8ICbiNd4jfE46DEU9CeDstXGupFUYknSJxTWYTi3Yu4leTqMulp7znXRPvour+Kh6ZJs7tU6dfUwk/tUaBpsaz6oONGX+N23peBkMehsRJ4vfdr8Hf58Fe/SWxvJX3VBj91mhO+ugkft/vXldewVVNP1mWOXToEG+uf5O3N75te7l14p6mEp+esCZ7DV9s+4I/9qtzYLs0ntTk+CdyX4vjbuSzMHUxxPgwsjckSURgesGwLsLpVxG2FUdB796iGK56xfSybfPW6SfLMocPH/ZJBKlXWOqFbIYSSOEnbHX9ctYRGyu26bkwq22o5dU1r1JmKmu2vTPTzzuMBiPfXvgt8095EYY+Al1PtX9YkyPkM61mDAZsz7VllG1lXSXPrHyG8748T1N/sGX6RafZnmNgoMpsFFmG1VfCAW2OE5/025AEIWmT1DiPaaiAkGQxLvpa80m2igW2u0W2A8Z3E06/9TnrsTRVYHCC29pVGlAW7R0y0+/QAlhxrpjPekLMMOh3C8SN0rVZjpBlmT6v9mHKB1PYlLfJtt2Ro0JxMnSN7Oq6LkoL/OX0a7fvXDe8+PzzPDlvHsstFv6wmLgUOAm4FPjVZGK5xcKT99/v2PHnAq2Zfn6r51e2HY78DA1V2i/YAXHr9CtcLepyV+xWdb4RySO4YvgVzOwzs9VnnjpwF+9ZzM0/38yve3/VdqAOdKR+qzj9nMp7djsdEid6dO4z+58JiOfbct5qlIwc3/144rscD2NehaS2y3Zs16yfC98kuK0VGh8PBjmIsLqegO8kPmVzEAY50PP1x4Db4cx9Hik1+Brd+230ICGVH6B90jcocRApkSlUmss4nPCuT5x+XmX6VeyCg194XFu7w1CyEf6YJoLeHFG5CzIegtLNrs+TMA5SZ7t1+vWIFsHkLZ1+Ld+1M/vOxDTPxE+X/OT8ZE2+x1WNU5Oj0enn7/dsp9OvHVBnruPjrR/z6tpXXUoSKk6/aDcZsV4TP07IfQx5wMcX0onhj8Gs7SLyTAeq66s5e8HZvLXnYUyBORw50lwiy1tCAkKIC43DbIaDB8W2dA1KLKrZMBf+PMNzCZ3YEXDGPuh5ifjdGAzD5kN4Y4R876tg4N3Q60o9WtsMxelXVORdHZNJacKwt2D7AtXHuIsq2VOxh53FOwk2BnNG/zM8b9wxRkJjrfMi1/N/p4xOGQ3AhtwNqvZ3JAl5qPwQu4t3Y108Elac7VlDfEVEb0g51f1+3iDLUL7Tnk3oAYrTrzp4D1k5+mu1HigTmX5NnX561PRrlzRUCNmM/U3qS1Yd9PllT+h5AreOu5UbRt9g65dOo6U9YMH2Bdz6y61MfG9is0mtYjzxuVLB0U5QDAx9UNTdUNjxlMhYMwkLmLMgiwBDAPP/nM83md+QUZCh+pK2TL/oHtoX3DWHxMKz8G/V1/MLZduFM2jqDzBKmxHfMyT47XjYrD3DaVDiICKCIqiqryKzKNPt/u5qV8myzLMrn+XDzR9S2+B8knXwoKglLkmQ3BGTgXpeBsd9AJH9PTs+OB7GvOKXeuaSJDG6q5jnrDpsDwRx5MDNrhARcd2jnEtKOsJfTj+XmApg/0diLtKOyMvL49577uGHujrGO9lnPPBDXR333n13c6nPQ9/A5vucZgvbnH4+zPTzqJ7fgY/gr1lijD4GcFvTr2q/qDVZtlXV+cZ2G8uHsz/kpnGta9t5GuT0056feH3d6/y27zdtB/qT1VfClnlt2gS3Tj8Q/dEDw+op6aew+NLFzJs8z/Ycvy97nJf+eYnokGhWXbOKv69uZ/OZ9kbC8cJOZHUtnx4YKOya4XWNdf1KfOP08zroUJLEOr0t6sD5my5ThVR+3EjNhwYaA5k3WYwNJZHLKSvTR0Tj8eWPk/RsEg8ve9g7m8Dhb2HVxVDufh7doYkeAqNeguSTHH+eNAVO2wLd9LFjtsz0cxV4GBwQbFM/cEjpZvgqFvb8z6b4lqKPif+YptPp1w7YXbwbq2wlJiTGVsPIEX7L9JMM0Osy8XI7BokPi7fJn5XGLaGhwYVmvBdkZ0NDgxgYdTemyLIwKNeXQECk5+cxBoEhwPFngZFCFztAf82nyEi7c/uI5+XHuHDwhQAs3LlQVXQ8uHf6/XZELMRO63ua+/T0TmzENSaYeZrpN6qriLTfkr8Fc9MsVic4mhTe8OMN9H+tP1sjj4e0Cz1rSIdGhl9G2bO4PaBLRBeijIkY5EAy8/br2DaBzekXK5x+snyUOf2q9sOvx0H2IiH1N+lLGHSf+GzHM/DTQJ9n/g1IGMDLp73MuYPOJbFx3l1cDGb33UoVb6x7A4DLh11uq+cHnZl+PqX7WTDyOVtWvzLetsz0Cw0M5cReogbeT7tdRFo2odxUbqsb1jTTT/VzDO8hau8Nna/yAD9QXwq/T4YdT4tBRo/aF+6QJBjxNPT9t+ZDjQajbV66Jtu95JK7jJaq+iru/v1u5iyag0V2Pjf6tjFIeOJEe+BOhyKqH/S+Ur3aRRszMVVkpjR1+inPsq4OFDWgDu30q9gJ/1wJee3LqfHO228zLTDQqcNPYTwwNSiId995x77xyPci8MLiuMNdPPRivjj3C64eebXLc3sz17l21LX8c80/3DfpPvUH9b4SxrzuuVO8g6GUGaisdLJD9zPh7BxIPcfra3kq76nUbR3XbZzXbfAJshXyl7rPEPExbp1+O56BBSEis8cDTu1zKpIkUVcHMjI/Fb3Mbb/exvU/XC92KF4Py88REnqdtKbnRXDcexDWze2uCQkQXT2GITHjiQjyTALdGYXVhZz26Wk8sH4O4OH6o74McpdAvc7Fz49Srh11Lb9d+gejDnyOxWKfd3hDfnU+hTWFmK1m72wCPS+F4z/yi3pDmxKSAAPmOpcfDoyC2GHu1Z/y/oCv40SglgsUp58SIOpM3lMdVogdDiFdbMkxPXt6cp5OmtLp9GsH7CgURWEHJgxsZiBriVLTT7PTT5YhZ7GoeZf/p9DwVcIuNt0Dh74W/y9cBetuEjJRHY0DH+taTHl6bxFBX5O0DNCnrt+nWz9l9FujeXXNq0BzaU/dVaUkCSZ8LOpM+FqyylwLWx+CTH0j5VMbEwq9kfickDqBiKAIKuoqVEXHg93p56h+jSzLLMldAtgdip2ow1t5zz5xfYgMisRkNpFZ6P5ZOjJMK7UdM7teCP1v8awhHRnJAMMfhz7XeXWad8ev49RNVVCgvxSpUtNPyfQzm+2GTm+cRZGRXgQ/6EnlPmGEsNYLuYy08yGyMTI/+SRImAiR/fzWnJgYCAgQUwI9Cq6vO7KOdTnrCDIGcc3Ia5p91lnTTyeq9ousvn3v27clnwQD74Agoevparyd1XcWIDIK1FBmKmNYl2H0ju1NeFC4Z87boGgI7aLhAIHP+m1QLAy4Q0Shy36sq9D/FuGg9QBF4lMxCrtCCVoym0VwWUvyqkSWUkRQhFMjW0EB/PWX+P+552pvb5sjW4WEsrcc/Bx+Hq5a8s8bJqSKGsArD6+0ZUk3DUBT5qeK00+pKaYWfzr9nPbd6MEwZZGQ32tHLPrsMy5XrIluuKK2loWffWbfMOxROGMvGBxbIUd1HcWFQy5kcNJgl+f1xpgZHRLN+O7jGZ48XMNBg6Dfjb6pId0OUewnTiXnAiMhtKumIJA6cx37S/dTXd/cuunJs6ysq2RLvpB/biunn9t3rmSAsw7BpK/80yAnKAFrTp1+Eb0gebrXNhDh9LNwdsotGCQDH2/9mB93/wi1uZC9EGp0MBAd4yQkwICcx3i67z9cNETfOpH51fn8svcXVhWJ+a5H64+C5bDsFMj6Qte26Ynuc+VNd8HW+R4dGmQMYnqfE4mOkpCxsmjbb7bAQU8pqRUL1LjQOO+cfuFp0OtynyQsdCgsdaI+pbs5cnCCUH5rXFs6o2Wmn7PAw0f/epQrF17J6sOrnZ8sbjSc/Cdy99kcECYhevVyvnsn6uh0+rUxRqPR5vQblDjI5b4ey3sWrYY/Z8LWB2HbY1C4Qhg6TAVw4EPY84aw+BUsh/3vQ/E/2m+krdn+OGx3XkRdK0pWUVWoeDZ61PVbemApG3M3cqRSpK75rJ6fudbu1HWjwawLhgA4+Bkc+kqfHP5GFIlPbxyuRoPRJpekxlAGrjP9thRs4VDVIUIDQjm9X/syWLR3lCwBT51+Bslg65dqJD4dTQoVp5+WGjhHHQNug7TzvDrFiF49MMiBPqnpd+HgC5kzYg5Du4joNMXBAJ47i4xGI+np6RiN7cCw1XU6nFsE3We3/ixuNJy4REToga7jaUsaLA38fehvvt+1yGY80UPi8431IsvvwsEXtpLv6Mz00wmLSTiP650Xy3CVWT2rn3D6rc5eTXGN+wG5R0wPttywhX23ippRmp5j+U7Y9QrUah8sfN5vh8yDiV90GIP31J5TmdV3lipjsCNHUVNyq3IBXKqLLFwoAi5GjIA+GsqEtRuq9sOXYbD9Se/OI1tEpL/JBy+8FoxJGUOAIYC8qjxbxHRgoAjMALsBRanppzXTTwke9bXTz2XfDY4XGVXtTE2mvKICtaIrXYDy8iZZH+FpInjHy7HE0+wwj7DUiWDgYwhVNYyrDmiSwB/z9hjSX0lvlp0ry57Ndz7a8hH1lnr6x/e3GVH9iep3riS1udG8S2MMUWmpPaCsGWnnw9QfIcq7LNa6OjAQwDV9H+T2424H4IzPz2DAwrvgEqtwIHTSGlmGNddBxiNud22qOKI3yhw30igi4Txaf8QMa5RKPFm/humIT+bKR36Aw994dYrYWKgKyeTK32cQ+3QsL65+0eNzKfVw40MTbYFsmt+T5lpocJbmfRSy/lb4rrtjW8Lhb+GbeMj50fU5YofDSUuhu2sZ0B4xoqZfmamMMlOZU6ff4r2L+WjLR7Y1iCuKikSmoNFotwkfTfjbJtXp9GtjrFarLQNJrdNPc6ZfZB8Y/oSovTZloUhplq0QkgQz1sHEL8UEbtDdMDNDF1kLv3PcRzD1Z91O1z9BTBKL2IWMrEum3/JDywGY0kPUBtkn7Gf61/Nbe72QrfJXYXZDoDBUT1+ha1ahMsB7k+kHMDZFFF1ed2Sdqv2VF5Qjp98X20SU16y+s3SXoDja8TbTD+zO+A053jn9Jhx8ETL+63lDjnEUOeKyMvvfWS/mHjeX9896nwEJAwD7+QMC7IZPrVitVvLy8rBa/ZjR4wqDUYybjlDG0CM/wtKTxCLFByw9sJTJ70/mlsW3kJAoFgTeOv2KaopsY+SNY29s9XlnTT+diB4E5+TCwNvt20q3wi9jbBIsynjrKHszLTqNIUlDsMpWj+oGaTJMH/pS1Bf2QGLLL/3WWT/0FYe+gu/7QKGLKFcnnNrnVH685EeuGXWN230NBrtxy5HEp5Lp1zWiq8PjKyvh11/F/8/zLkak7bCaRb0SbzOne10Gs7P8UtcvNDDUNs9xJPGpPEslq6i9ynu2u3euCqKjolDr1s0HoptG4FrqoSbbqUGxur6aRTsX8eHmD12e19MMBrPVzO2/3s5ra1+zzXPdcvgb+DIScn7RdrEOjBIM49Lp98c0UWtZJUofVBzx0NwJpfZZWmUrr64VSkC3jLvFpfKTr1DVb/d/BHlL/dcoJ0RG2t9xntaKV0NT5+0j0+wOrF3FnsmGHjNIEuT8DPl/uN1Vma8WFgrp8co6/RwzxbXC6BDR6PTzKKAioqeQSozU22CnDz55305fBadt9uoUCQlQF1hAl+BeWGUr9/1xH7mV7p09jlDq4UYH2oNJNT/LnJ/hq2i7wt3RTmC0CK4yO+hPEemQfh2E65NCFxEUwbIrl7H3lr1EBkU6lfdUE3DIzpcg8zlbll9qqgh+O9rw9/y40+nXxsiyrDrTz2N5z5AkGHwfRA8UNYRGPGWP0ApPtWcVSIZ2+0JzS8I44dzUiT5xfTBIBmqtFdQF5Hud6ZdTmcPekr1ISExMnYgs45uUZVmGgAgx0Af60SkV0ct57T8P0c3p163R6ZejzunnKtOve1R30sLTuGDQBd416hhEmdR7szgb3XU0AYYAKurdy0Q4Mkw3NEY1J1ZuhRL3jsOjkvxl8ONAyP7e41PUSoVs6zOH1f2mkZvru2w00Cc7TJZl8vLybHJpbUbVftj2uMjSckfhSijf7rPM+8k9JhNkDBKGqrg94pJeOv3u/+N+TGYTY1LG2KQIm9KuM/3MNf4LlPEFhgAwFdpqSrly+oF2ic+mKIZpVc9xwO1Cyi9xgubrtJt+qyfGMDCGglXnaAkHKI4iR5l+itPP2cJ7505huO7WDYYN81ULfUz0AJjyHaR1LG3SCd1FX3Hk9FOe5cKLFlL3QB1nDzhb07n95fRz2XezFwln0yHvsgj05qxLLuFjlVbEj0JDmX3JJfYNOT/CwlSRHeGAUlMpsxfM5tofrsXqQk5Yeb5ajZmHyg/x4j8vcudvdxKgdi0WFAtdpkFkX20X68AomX4upcwH3g39blZ9TkVi93C53enXNBhO7Xxnyb4l7CreRVRwFFcMv0L19fXE7TvXaoYNt8IWDXUjfYQkuanr11AhSthkLfDqOk3nO6GBoSy9YimBhkAu6XuSqHdlcqYv2gmn74KT/nS7m5Lp92nZTUQ/Fc1HW1zXD9OCkukXJgmPv+b1hyyD1Xnd4/aAT+bKwXFeZ6736QMJldO4K3AfE1InUGep47lVz3l0roJq0c+iA0SnlyQNjqDSrVB9WEhVpp7rvM7d0cbwR2H6clG/ryUJ42D8WxA30v15Mh6FvW+73W1qz6mkx6VjNBgdrj9kWXYbcAjA3jdh37tHfT0/f69tO51+bYxVtlJmKgPcO/2UyDRN8p4Vu32WKdDuqCsRRi8dCAkIsUlrVIfs4vBh75TWVmStAGBE8giiQ6LJzxfRDwEBkJamQ4MVJAnG/R+c4Hjh6VMsJlh/C2y4TZfTKTX9cnLA4sV8a3LaZJ6d/iwvnfqSqv1d1fS7eezNLDxpIbMHzPa8QccoirxnRYXjGkNqOHfQuVTeV8mHs11HS4Njw7QSAb187EI4wXOnV4cmIKKx1pHnRueIoAiyoj+mOOpPdhzST/LscPlhMgszsTRZYGlyMLR3jvwMWx+A8h3u9x32KJydK4xyPiAsMIyJqRMByAn9HXBRG0UFsiwTEhCCUTLy4owXW0WpWyyivhi0w2dZmw+/jBIZaSDmEbte86m8qleYioSkdtPsuehBIhup7w2Aa3lPEE6/sMAwVVLH135/LUPeGMK3md8CGp23gRFCys/fGXXtlW6zYFaGV/36SMUR9pbsdbufM3kdcO/0U4Jzunf3fVnoDkH2D6qMHnpwweALeOGUF7h+9PW2bY6eZZAxiOAAbYOp4vTTXCZCT4LiIHEihCS639ePXHvddSxraGCNm/3WAH/W13PNtdfaN0YNhH63QoTj4NMu4V2QkDBbzRTVOI9881TeUxkPesf2xqC2Hl3KaTBtcccN9vUAxelXW+tCpaLfjZDuPptaweb0a5Lp11TVQO34aZWtDEwYyNUjriYyuJ3UoG6JZIST/4KRz7d1SwA3Tj/JCJnPCNUML2g535nWaxrZt2fz4XFXwtKTofBvr85/VBMYoaoDKEFqVCdhla2sPLxStyYoteDC8FDes/Bv+DpayNQfS9SVQP6fXtlVBwjBHnbtknhwyoMAvLnhTVvWnlpkWbbJe0YaxLwhOFjl2Fq8DhYPF/UYu5wAk7/yWvL3mGP3q3DwE02HNJ2zKkvp8rpyTGbx4nWZ6Td9JUz5vrOen850Ov3aGINkIPv2bAruLHBbkN2jTL9Vl8JPA9t9lIrXVOwR2sSZz+p2ygXnLWDXjfuIq5pEfT1UeZEEsDxLSHue0OMEwF7PLy3Nc8m6VjSNHtVQhFw3DMFQvB5K1uvyfUtMFAsmsxny8z0/T9fIrtw54U4mpU1yu68su870A5AkSf2iuhMbERH2qCyXUbYuCAkIISRAnTXElbxnkPEY1heMHwtn7IIenmerhgaGkmAQkmlrsrbq1TLe2fgOg94YxA0/3mDb5tcaN76mz3Ui6rXLVPf7GgJ8Po6f3FvUp9hjWQJ4l+knSRKvnPYK+27d53CsbSp31e6cfiFJENkfogaIl8CW+2HDLbDvnbZumWMqd4m5Xc5ip7soTr+qKsf1biakTqD47mLeP+t9t5fbUbiD7YXbbRkqqp1+FXug6qDb83einlfXvEr3F7tz15K73O6rzGG8cfopwTodkk336Ffre+dzsOluvwQCHJ96PLcdfxvDuthTLF09Sy34K9PPJUmTYdovfpFL1UJycjJPPf00ZwQHO3X8rQHOCA7mqWeeITm5Sd+JHghjXhbR8w4INAaSFC48FDmVOU7b4Km8p1KzvG/8sZO15wmhofb3lkuJTw2kRjt3+gUHwwebP+CtDW81O6bOXNcq4/O0vqex/cbtPH7S4/o0zBdIkqjxlOR+Pe0PXDr9AsLhjL0w7k2vruFovpMUnkRAwnEw9g2IHeHV+Y9qanLEPLW+3OVuSqZfWJEIRNTT6afIe4Z66vQzBEPUIOg6Q7c2dQhyFgup44LlHp+if6NvLTcXjkuYwZiUMdQ01PDiP9pq+9VZ6piUNomhSUMJR3xZVL8jI9Jh8Lx2V0PYL9SXi5rW2Q4SQfJ+h5UXQ1mG+/NMXwmTvnK729b8rTy87GHe3vC2Td6zqV1VWXdEB0cTGujEyAqi7nNUX5vT72jN9PM3nZbrNkaJhk8MT3Sp3242251Oqp1+sgxpF0CfG7xO0W73RPSC3ldDwnG6nXJMyhj6JfYmIlz87cpdz1lc8lfWX4CP6/ntfx9+HibS2NsCSRJFs09ersv3TZLsEp9KirevaWiwZ6O0dPqtyV5DnaWOuLi4Nqm10NGRJH0kPtXiyHhy1YirmDt+Ln3r9kHpFt834igmPWwEABvz3cXFq2dz/mYAhiQNsW3z1AjWFEmSfN9vD34Be/5nD77IXyYWvE0xBotIw0CVUdxVB2DLg1C6WdemKkzvPR2ArRXLsGL22OlX22BXE1CKebdEMZxokmTxF5Ik6h0Pukv8f9QLcPIKSL/W7aFtQtQAmPwNpMxqvj3rS1EzDggPt9dOdBRkYTQYVQdQHCoX+uY9osWzVe302/Zf+L6XyKT0AL/027Zg/0ewxzND5Lhuwqnw18G/XMoEgmt5z3mT5/HLpb9w/qDzHR6rZIh2aKffgQ/0q1c28jk4qe3qWDV9ltsLtnPihyfyn1/+o+kcdXX2vutrp19H7bu33XEH9z3xBFOMRmaEhPAp8DvwCXBKSAhTjEbue+IJbrv9djdnak1KZAogMnWd4cl8x2w125xK5ww4R91BFbtg+TlinnIMIUkq6vrlLoHFI4V0owqcyXvKyGxLvperFl3VTL3i5p9vJvyJcE744IRmcyfRPomwQAcyM76koUIYhmty3Pfb4vVg9VCqxQe4dPqByGINCPfqGk7nO1F9oe+/j01ngloOfQV/znS7hlFsAxFl4zFKRg6VHyK7wsvaLo1U1QvjaYjVw5p+CeNgxpp2nR3mk/dtwnEw+mXh5PeQiAi7HW/PHokHJj8AwGtrX9NUtzEkIIQ/rviDrf/eimQRxjnVzzE4DoY/1uFk3nVBkkQQ6yEHDrvKPSL7UY08cVRfERzrhoz8DB5Z/gifZHxCYKA9qUUJVlPqObrM8qsrgfJM6mrqyGk0oRytmX7+nh93Ov3aGINB3SNQHE5GoxhEVSFJwog1+F7PGteRMATAce9CqsoFjwaUxbGnTj+L1cLw5OF0i+xmy4BQ6tT1cGwf9QxzNdQViTqNbUVwvPjeyTLsfccrCUGAwYPFz40bvWtWQXUBn2V8xmcZn7ncr6lxrKnTr7S2lInvTSTxuUTCEsJU99sOR9E/wnBscZAaogOKAdGZ5Jwavtr+FWPfHss9S+5xuZ+jhdq9k+7lpRkvkLz+KtjejqNpfU3OL5DxX69OcVzXqQBsrVJnGFHDljzhiB2RPMK2TQ95T4PBQFpamu/6rWyFHU9CxsPCgAKw7TFYfpY9M6R0i/Y6krW5sP0xr+ovumJU11HEhsRSZS6nPHw9BQXaE1leWfMKo94a5dKQCfZJf0hIO5ILrMmBzOfAUte8UYGRIpK93TS0BcHxYq4T1SKrY8s88b2jeZCFu/FWkVtxRL2l3paZkhYttMhVZ9/2uASGPAihXdzs6Bif99u2YtdLwsjqAaNTRhMZFEmpqdQ2XjrDVXZYelw6M/rMcJoZpATm2GS3OiKzs2Hy1/qcK36sqH3ipzHhQOkBPtryEX8dFAGDTZ/l3pK9LDu4rFnNPzVUNtrYAgKcK1nohcu+W7gaNt8raty2Q267/XaysrOZMm8e85MHcmFwCo90HcgJ8+aRlZ3t2OEny7DiPJfzKsXppybTT8t858fdP3K44jDxofFcOORCdQdV7ITshR4HZHRklMBpp04/yQj1pdCgziitZPrtLdlrK9dSa7KyLe1GMqKfBqBXrN1q2SW8CxbZwt+H/ubKhVfyyppXeOSvRzBbzZ7cjvfkLhGG4QMfuO63tXnw61hYc53/2+gEt04/U6FXQXOy3M5rUbd3uk6HcW9BpGPZY4WAANEvA6yRDGh0Mq08pE+23xuz3qD+gXqOk8W4rdpZ1FBpz4Jqr2uBRnwyV45Mh/63un127lAkPnfuhDP7n8n/Tv8ff835y+PgBk11b4vXC9vosUpgFMxYB6Nfav1ZnxvgogZ1pQYaKkS5MDcGAmU9sad4D5LUeg2iSJt3jXRRzy/nJ/hpEMXbfkaWhRS9JoXDDoS/17ZH2Uq643Hf7/cx67NZLD3gOoJUcThFR2t497TXejQdhILqAub/OZ8NiaLOj6dOP6PByKfnfMrh2w6TGC7S0pWMii6e2cIc0/9WYeQIitXxpB6S/R2svQ4y5nt1mrFjxc9167z7Oq89spZLv72Ux1e4dvQok4ngYGg6Fi/ZvwSLbKFndE9qimqwWl1H2HdY9rwJKy8Eq2+cfu7qTKnBZDaxPmc9q7JdG7xcRkyPexv6/MvzRnR0Dn0p+mad5w9iZn8hDXlEWk11vfeT6tLaUrLKswCayZrpseC2Wq0cOnTIN/22aK1YVExfASf8CEExYntkXyHnqWTjrL8ZfjteGE7UEj8Opq+Cwffr3mwQ76YTe50IQHHkMkwmbfJx/1v/P+b+MpedRTv5aodr6Q+P5Ml9za4XYdNdzrMdSjbC5vs7zlzquHdh/Lu2X5Xx1pmc8sbcjfR9tS9j3x7r9JRHKo4gIxNsDLbNX1T3yW4zYdgjalvfCp/227bkuPfgxN89OjTAEMDkHpMBWHbQdZaOq0w/dxwV8p6GQFXRyaox10J9mX7nc8HHWz/myoVX8t7m94Dmz1LJgOge1V3TOZtKe/rahumy75ashx1PQ43rQJG2JDk5mXkPPMBzb+5gwvQjXHfbDuY98EBzSc+mSJJ4j7hwMKhx+rkrL+CI19e9DsA1I69Rnb1N97PgwhpIna3+QkcJ7t6LJJ8IZx1U/bfpE9eHaT2n8d5Z7xEdLIplvrjlAbKS3gRZ4n+n/49T+5xq2/+60dfxzQXfEGgI5KsdXzH3l7k8/OfDfJf5nec35Q1p58LEL2DQfa77rWSAoY9Aj4v830YnKLKQTp1+628WWZseBrJWVNinf62yow99BT8OgILOmn5OiR4k1kFh3dzuqjzLwZH6S3wGGgOpLhdjo+o1yKa7YPEIKFihWzt8hU/nyl6uf5o6/SRJ4vrR1zOy60iMHiqClZWJn27rElvqRJbpkvYlI+534seIbMeWSJL6MiIbb4cf+0ODa0N43zjh9MutyqWqvsom8VndaCI6f/D5mOaZ+Pp8F8F40YNg0H3sKxkBiCy/du5z9xh/r207nX5tzIqsFfy852db+rkzVA9yTfllNKy7yeO2dThyfxODe/E6XU5nsVr471//ZVPQa1ikOtuC2VOapvEqE9QkHe0R4iLtpEt3OwtGvQiDvMsyHTJEOG1KSux1ED1hbIowamYWZrqUFFAM3uEt1EB+3vMzAKf2OZWSkhLkjmIEdofVLDIOlIjW/rfCxAUQ4Bt5GbWZJ64Y1XUUAJtyN7mUN3OUjXKo/BA5VXmYe8+B5JM8b0RHZ9B9MHMbBGp5oTRnfL/ehNb1wCo18Mde7xdFW/OFLHGP6B7EhtoDF/So6SfLsm/6raUe/jodfj9BRNTFj7F/Nu5N6HO9Xep47Bsw+hUIdSFr0RJDACQeL376iPsn38+aa9cwsvZuwH1dvwZLA7Is89jyx7jhJ1F78a4JdzF3/FyXx3k0h/E1wx6FCZ86r9Wx/0ORwVnqZaq53uT9AV/FwoGPm29PmtLsO+huvE2JTGFvyV62F2ynos7xBEdxxKdFp9lq2apy+rmRnlSDz/ptWxM7onWWpgam9RSRue6cfs4y/SxWC0+ueJIPNn9gq3PbFFk+CuQ9y7YL5QK9VAuq9sOXYfrVCHSDMs/ZmCvGnqbPUg+nn69x2Xd7XylqbcWNaf1ZO8OtU6Ep5xbCFOeOGzVOP+U92dQwnVWW5TSrt95ST1hgGAGGAP499t8qGtkEY4j4d4zhVt5TIwbJwB9X/MFlwy5DkiQ+z/icD/aJTO6ZDe9x/ejrm+2fHJHMOQPP4a0z7HX+zuh3BucNOk+fBqmlZIMIZADocSFIErLVQllhluN+G5IEQx+ElFNbf9ZGKIHTxcVgsTjYIe0CGPYYyJ5lUSrvwehou1SdHQkk39fePlZQ5hq9jPo7/cDe32PVxsX3ulzUgkuYoGs7fIHP5srrb4WF3b2azyt1/XbvdtJHVfDuxndJfDaRG3+6UcPcVIZh/4UBt3l20aMFS53I0mspy1x9WDi01WS0p8wUdiNce99iQ2NJCBMPZm/JXluwWtM1SHBAMPFhLiRE4kbDiCfIPCSy44/men7+Xtt2vqnamH2lorhb/3jXWtGOFgIuaagCo4/1W9obVjOUbRWSaDqQHJFMZFAkMlZqgvd5nOm3s2hnM+dEXZ0960FZUHrNmuth+1M6nUwHDEYY8B971otZQ/pIEwIDYeRI8f91Xvhyu0R0ITUqFRmZDbnOJfYcGaatspXFexcDcFqf0zxvRHtk/wdC1iXzOfF73CjocYFYxBSvd1t8Wyt6yHsOSBhAWGAY1Q3V7C7e7XQ/R5l+A18fSLcXuulWK6DDEtUXYgZ75UyKipJIMZ1MRO0gcgq8k/EF2JIvjFrDk5vXD9Cjpp/vsArH0QAX9X1MhaIvxQyFvjd4cAmLCGQp3+F5M10wqusoxnUbR5dE4Zx0Z9y8cuGVjPjfCB5c9iAA90y8h6dPftqtNn1TtYJ2gzEEel7iPIxw4J1w2maxCGlPBESKNgU7WPVa6sV3BvcZDckRyfSM6YmMzLojjl+wSj0/RdoTVDr9/rkKfhpiNyp2YkeWxbjQ4FkkmeL0W5613KUknLNMv8KaQu5fej/XfH8NRql1tHVNjX3c7bDynrteEpnV9c7SeTQSlgppF0LsSH3O5wbF6bejcAc1DTXNM/0qxfxFqSWmFn86/VwSGNVYa6v9r1GVNZqqerdujP/nDjyXBect4NbxtzrdR5kbK2N3TUMNx797vC3wsCVBxiAWXbSII7cfoWdMTxWNbCRncdvVf29j3Mp7Ahz+TqyPVKLMf3YW7eSy7y4DID33biaGz3F6zJwRc3h95utcMPgC3j3zXf/W95FlWHU5LD/Tvs1Sh+HP6aTlzHe8vw6BPHoTGyuccVark3Vl2rkwZJ7HgazK3MnhezDtPJi1DRLbv1OoTVk8ClY4rh3cFMU+kFw/iUlpk3h02qO6XP7iby7m0m8v5UilyCx36fTb8iDsek3MoRMnCqUKDzPSjgqCYiCyn8dzVYC0NBG0ZDJBVpaoQfv+pve5etHVLksLNCWvKo+imiLqzHXqpeeNIaLmZq/LPG77UcG2R0SWXuWe5tuzPoffpwipb3ekngMjnoAg9wt4JdtvT/Eeh04/tRw8KH4ezU4/f9Pp9GtjahpqMEgGt5N1zdJYgRFwykoY85o3zetYdD0FziuB7me631cFkiTRP0E4Y6tCdnnk9MupzLE5GpSC3criMSREQ31GV5hr4cgPUPyPDifzAdufhB/6iuKsHqBIfK5d610zRqcIw+3mvM1O93Hk9NuYu5GC6gIigiKYmDrRu0a0N9KvhsnftJYPPPKjqN2w9y3Hx3mIMklTJm2eYDQYbTXfNuQ4d+AqcgJNZZLqLfWES9D9z6mw+T7PG3E0UF8uapp5weyAN5m6fTuDjbO9bo7SL0d0GdFsu/Icw3yTfOodxhDo+y/niwqrGb5Ngo1eRBrWZsOv4yDzec/PoYLERKgI2cbbGS82iz5bdXgVb6x7A1mW2V28my+3f8nW/K0YJAOvnfYaT538lCpjVbvK9LNaYOdL7r//4aleFbH3GQnj4KTfIaVFEMr2J2BBsG0Rp0ZO+bjuxwHwT7bj+UOQMYhhXYYxOHGwbZsqp19wAoR06RCGfb+T+4sYF7IWeHT4iOQRxITEUFFXwabcTU73c5bpl1clJIYTwxIdSiwp7+fISAgK8qiJbU/vOTDyOW2Z1a4wBMKkL6Dnxfqczw1dI7rSJbwLVtnK1vytzZ7l4fLDgPZMP2UN0+ZOv/pSEWXeMvK8HaI4/UpKwOwuWagsAw5941QObWiXoVww+AIGJw12+LnFYndEKWP32xveJrcql+2F223vZUfR4UnhGmRjZBlWXgQbXGfnH62oyvTb/gRsfVDzufcU7yEpPInhoTMZcOQJt1lFN469kQXnLbBJZ/sNawP0uBi6nGjfZgxGDknGYoxs7eCr2CneWfve82873SBJGrNxNaLMnTps8Et7IKSLY3nBFijPsaGkGyuuWsHMvjO9vrQsy3y942s+y/iM6mrxnXbaJ801cPBT8e9o1RPUyrBH4ORl9gB+DzAY7Nl+u3aBUTIyb+k83t/8PmuPqDPsFdYIw2lieKK6Plmd5bHN8aijy0kw6B4IaGFwTj4JRr0EYWkOD/MUW12/kj021TRlDXLPknu44rsrbAoWDvnjJNhwO/mN5YZTUnRt3jFNp9OvHZAalUpwgOviKMrkVLPB7Fh6canVJtaAkoFZHbLTI6ffsgNCfqlbZDdCA8Wqvam0py6PJyAUZh8WxZLbI2HdIbSrxxHXYxoVgPbs8U6ORTFa7ih0njHjyLmu1Ns8qddJBAcEk5yc7N+ITF8iGUQEj7HF+JM8Hfre1NqoXH1YZJR6GCGsh7wnwKjk5tJXjmgp5SHLMmarGQmQA6OOvUzopljq4OtY2Pgfr07TPUVkCh7RoTTPlcOvZP4J8zmtb/PvnOYsdwdIkqR/v60rdr+oMATA+HcgINxzmbnwHjD6VZ9HKxpiD7Fy4Hjeyb6dsW+P5Z2N73DjTzcy67NZ3PTzTbyz8R36xfdj7XVruXL4lfx0yU/cNE69fLiSZdIuavrl/yEcsXv/535fWYaC5bD7dd+3y1uiBws5K0MgYB9vndYuAsZ3Gw/AmiNrHH5+0ZCL2HLDFl4+7WXbNlVOv1HPw0l/qG66I3zSb9sD0YMg/TpR99MDjAYjT5z4BF+c+wX94vs53c9Zpl9upVDD6BrZ1eFxR0U9v8SJMPCOtm6Fx0iS1EziU3mW1TUWmxR2ely6pnMq71J/OP1c9t3MF2BRGlTu9X1DvCQ6WiiOyLLrcRQQihl/nydq/HpAebm4jsEg3pP1lnqeXfUsAJPTJiNJEt9lfsfQ/xvK6sOrWbhzIQdKD2i/kGwV85KBd3nUzo6OsiZw+TzHvAaTtdfYS45I5rpR13FJ8OdIGNuvs8gYJKQ6B7cIfpzwGQ2j3kRqGQxSXwbhvSDYz85JFShlNnMdiTyVbRdlXw586tG5XWb6lW6GXa/qpi511DJtMYxzP9dWjPt6rCcVimuLbWoIQeZEAgJal2+xERAGM7fCpAUdTrK1vc+VW9b1U+pSL89arur4gmphOE0KT1Ln9Nt4p5hjmLyIMD9aSD4RRjwF4S2ce3GjYcBcCO3i/hwlG2HZqSIhwA22TL+S1pl+P+35iY+3fkxJrZOXryxDTTayqcD2nHVTxGuH+Lu/+q5QTCeqUbNw05TpJ1th7fXQ7QxRrPtYonyHyHpLvxaCvZ9tK06/qpBdHtX0++OAMHqd2MseTeeTen6GAKG33x7peRn0uMRjiYTYWOjbVzj91q+H6dM9a4Yap5/iKGraz5RIpImpEzEYDCQn6xQ53paUZ0Lms0Ka0FGBbWMwjG3MEpatonhvUKzI3jj8jehjp2jX21eMiCUl4t3u6ftOydp0JtVqMtkNnUpUb0NjVHmVDDUnrSA6pD2kHLURxmDoPxf+n72zDo+jWv/4Z3fj7kmTNE3d3d1pKRSKuztcnB8OFy1yL3bxi8tFilOgFKh7SzV1b1NJ0qaN++7+/jiZbNKszO7O7qzM53nyJN3MnHOmkzNzzivfN6GXW81kNf7pHDxcy/Gq8iYtd1cYmzuWsbljW32uhCykR+bt9leEkW/KakjsY/u4jteJL3fo+g/3zpdB98wcOq97nD3ZT7H26FpumH1D0+8GZQ7ikt4iw2VAmwF8PONjp9v3qUy/tHEw+nv5sp3r7xdZHLmXy5I38Tglm4U0S7uLhWysRPbZLdZ80rPPnjRd80w/s9ksaxMiPVvtOv0UIGDet6cS3Q6GuhekJad+l6NMv4wY6/+3Xq/nV7ZDGJVThirTnskojHZKb6gP/Qy73hb1WWPaK9u2FQa0GcCc3XNYd3QdfRu3M0XVR0lLTsNoNjY5BeWyvVHFqV07hQdqBbtzN3UEdL3bujyxjyFlEh05IvZudvdtHW8Q9WFtyKY3mBr4ZecvHCk/wo0DbyTklOOkeZeYKBx/n637jMPlh2kT04ar+l0FCKPZlmNbuO/P+9hUuInq+mrW3LCG/m2ckJ3VGyDHsdxeoCI5/ewGkbr4LBqcNZjBWYN5YlHLvnyK+jLQh1mt52hz3qYOh6lu1NjwIFlZsH69DWeRPgTKtkKta8Z/yemXZC1RrWA+rL9X1CaNtB5AoyEfaT95+HBjkEV1MW+sfoOIkAgeGPWAS21KWfHJ4WkYzBEkJjpYFoTGiC8/w2NrZbNZ2IpCY4VUpot0aYxN290Y5zMmZwyztsyS7fSTMv3SotPYIGd92vZc4eSK8P01hl9grIFjS8X/qwOu638dF/e6mHbx7fjgPfGZpNh0tEIESNjae6DTwfQdnGis0arX23j2Bgh6vXeDC/wrlCFA6ZiosNOvYi/s/UhM0GCjaBFseBCOr1CkOXfkPc1mc5PTb2L7iU2fSwY4RZx+DZWQ9ySUytBkVgudzuLwK1wkHE5OIkl8rrOTEe6IHqk9AFrI5JyKNQfDw6Mf5rWprzGt8zSMRiN79uzB6Go1Yl8h/zvxjKiSUdtuz4cwb4KQgwyJhIH/gb7PWn5vlF/PTTJmGI3uZfsNbDOQ3IRc2idaN7xJm/mICIvhs7ahtun3oYZQ1zsPFAa+4rYzKisLDqS8yyPliTz0l2fkUpXI9PPIvE0aJKSk461LdXkEDxZ9Tk2FTgUPcGPlfh4e9TD9M/pzXf/rmHPZHJZfu5yYMPc2wk5LlHsSQxi0Pad15KM1dDph5J+20TccfiCCm7bMdFiLQTLuFxZic/3SP6M/YYYwjlUdY19Jy6wRk9lktWacteCYFux8U8gnu1nPL2DetyphK9PPkdNPds0UVzn8C6xtJnlcugX+GAb7PlOm/f2fwffpULREmfYkaoqgcIGQjvICklNv7dG1TeuYkKpstv9jO/vv3N/KaWSP+nqL0693b/vHKoHduZt5Ogx8GSL8I4Rbdl2/tFGiRqwVZwqADh3nzzqf2367jcKKwla/b55V1GBq4Pllok77vcPvJSJEtPnomEcJ0YewPH85FXUV9Err1SR3LxsPriP8AcmIWFoq9iI2Mda4rNBg11mkNtv+DT9mW7UbGI1Gjqz9EPNv/Vzaq6uBlCF2xJpae1xXOO+4yGhxgVNrbLag3YUwaZGoj65hm+K/YdPjQinIDhkZwj5QUyPmz7L8ZTyx6AlmLp1JaY0LUltAfpnoMzVc1L+16YRfe4/48sG6lXLw2FpZp4Ndb4lgJzfo0EF8P3QI6upgTLsxgCgfYa8utYSU6ZcUntpkE7C7Ps29RKiNaAiWXAB/n1JLeMcb8FsfqJChFpAyHC6sgE43Ojy0TWwbOiV1ItQQ2kLes7ahtinDr02M/SAJaQ+SlCSeCYGKt/e2Afxf6R/EhcfJcvo5Je8Z2wnOOSoyOYKN7BliEZY+0eGhcpAy/arC9zrt9Nt7ci8HSw8Sqg9lVM6ops+lTD9FUpYL5kHeE3B0rgKNeZjqo7BgCqy82ulNZ8fGKVJQ4Hr33VK68eulv7L+Jts1cKTFRPOF4YA2A7hj6B1NdTjKy8tdH4QnKV4Di2fIkxrp+QictkpeNGtYIlQXwInGKM/U4ZA+Tvy87zP4pZvDxbyEwWCRYrG6QZNJ7/Te7LtzHx+d/ZHV30sb7ub3cWPhRgByopKI3PUWFC50fQAagHD6RdW1p0FXzaytsyiucs2Tm1eYx+wds5tk55qjVIaY4vO27QwYNcs7Rd4PfgvfZwiDs4eQ3kcVRak8O/FZ1t20jvfPep+pnaYq4iT3mUy/4jXyNjnNSRog1lW+QubpcMY2kVXSnLqTsOZW2Pc5IGqySY6/LVusNxUeEs5FPS/ixgGtN3ObCjcR+1wskz+zpNebzTLq3BycBXvet2n8dgaffd+6y+ZnYZF79af/PvI3M5fMZHPRZqu/d5jpF23f6adYpl/hAiGFJnHoJ9j5H4vzLGUEdLpZOWUSQ6TIxItyruadQ9pfCRdWWtY/HmZsu7H8cskv/Hrpr61kkhIjnUsh2rlTGNvi4yFb4f8WWwTC3C0oKGD9uqdZs7A7V56XxeDu3Xnm6acpcGEzYtAbmhztR8pbL4CbOxj+3PMnu0/sJikyiZsG3dR0TG5CLtf1twRrPTbmMeclojY+BD+2lb1uDzTi4oQt22y2HQzDng/h60iX11w+7fSLzoXE/jblpatqGsSzuWyH+KBsF6y4Go7Lq7/lbTwhCylhd60TlQ1pYyBU7SKpPs7J9bD5aSi1vk6RCAmBNo2+gMOH4cwuZ9IztSdltWW8sOwFl7o+WHoQgCSDCPCz6vQzm4Xkf/FKv5P1bI7H3rfj5sAk9wKokpLEfsRkgoMHoWdaTxIjEqmsr7Rbl1riWKWIuIkwpmE2i78Vq3tJswnqK9waa0BStg3Kd7b8zNwgSr0gY/3gompG8z1IYaUIdArVh5IUaePFWHUI9nxE2dG9QGBLe6qB/z7dAoR9d+zj3hGO6044HSUfkab8htcfiGwjFmEhytTr6p7aneWXbGfSpkOUlTnnq5Ky/Ia3HU50mEVEXFF5z8xpwsnZ7iIFGvMwkW2ErvuwT5x+gciOtLVDeEg40zpPIzch1+Ym2WcM065QdVjITxXMt32MFFGl00HKEHnt5pwHZ++HtNbSi+jDxPcG+Yus5hIenuLUen4Ac3cLx/iZ7Uei2/B/wvAYzBT8JQomF7su2dOmDaSWTyKuqq9bG7OPN3zMWV+dxTOLn2n1O5/KEJOocy3q1GXCk4WhxoNIz9gTJ6DBceCl0ygh0+o2pgZYdgn8Psi1LLSjf4poSbUzJUJjIb5ba2OTziAicgv+avqoZ2MQ+mY79pZPz/mUd6e/S4fEDi0+31S4iZqGGuqN9U2fVVQI5wHYcfpNmA9T1gRXTWlnKd0Kx5c3brpd47mlz/HI/Ef4abv1d9mpjiIJSWLHVk0/xeU9194Nyy+zzJtej8GMQ0LmFCAyA4a8Lf6ezWbxN+yOQ6LdRTBllfISnIYw7wR5NJIclcwZXc4gMzZTGMxo4GSZa5lHeXnie+/ePjAtd7wuHN71vu0UfOWll2iXnU3dXzN5qWI7H588wl3bt7Nk5kzaZWfzyssvtzyh4C/4IQsOfG2zzcxY4aFw5PSTZM9mdJ3RKsv+kdGPkBKVwoi2Izin+znOX1hEOkRkiO9BiF4vQ+Izrhu0uwTCnffaNTRY1js+6fTreC1M+NPms6wiahCmsw+JwDYQzpB9n0CNb9auk/aUR4/aWJoVzHO7pp/V+2g2NX4Fd+asQ7JnwLQ8IavvAOleHjoEep2eZyaIPeFLK15iz4k9TnctyXvGme04/XQ6OH0DjJ3tdPtBQXx3t1VOdDpLtt++feLeSskQciQ++2X0o096Hww14p2VlGRjHVO2Hb5NEHWDNSxMy4Pxv7f8rNtdMH0HxOTKa6NgPhz53fFxwGsrX+Py7y+nWC/KKVVWtlQYsRmoVPw3rLoW3bHlgOb0UxrN6acyOp3OoURL82g0h4bPsl2w91Oh2R6smE2ieLOMlHFHhBnCGJjbFb05lIaG1sYTe8zfJ5wvE3IntPhcUaefPkQ4OSP9pO5Nh6uEsdJJpAd/aanF4OgJTpUSnL1jNh9v+Lhp4ehz1BZDbeOuJGs6nLUH2l8mJGmsyVSsu1dkAzr7fDCEg95Ktk+7i0TGSXx32U0pGZVpNputyn5IG/nmG7XTO5/OnUPv5PSeV8Hk5dD1jlbnBRUNVcLhJ0fi1QZhYZCWqqfr4ZkAvL76dQ6XOXdj6431fLn5S4BWNf3q6izPXJ9x+lUdFtJxW57zXp/p42HKSlGQ20PEx0NoqFhvSJk+SmEy0VQTV1Wnn84AQ/4LA19zLTBo11uw71Oo3K/40JyioUpkdJ/qMAqJFSoPQ99v+kiS8rOV6WePTYWbAOiTbqlZKRmmY2PF/LeK3iB/IxmsDP9EyI4ZXC+MOD53PAAL9lvPRrEl7zlz4kzmXj6Xs7pazzR0W97TbIaD34jaegB9Z0L3/wNT499rdI7tGkjFq0S26sZHXOzcwxxfDfnfe73b0OhyTsQu5qduSVz1/bVOn9/c6ac6pVsb1Ul811j+yksv8dwjj7DYaOTP+houAyYClwFza2pYbDTy3MMPt3T8hcZDVI7INLWBPadfc3nPyR0nc8eQO5jRbUar49rGt+XgXQdZeNVC9K5kpnS7W9RnM9h6gAc+kvFf+j9vReoIGPkFJA92um1pH2kwiKxCn+HIXHnrfZ3ekqVftBiyz4EZhyFjsv3zVCI1VWT+1NfbCAzeMhP+vs3pdps7b62+Czc/C18aoGST020HFRGpon68jDV3c6cfwNldz2Zyh8nUGeu4e+7dtk+0QWmtuIGRdQ7kPXU6EVyp0RpTg5ACdsNWANC+MQZrX6PQilRP/ECpY7n03y77jY03b8RQJd6fNtempgbIOhPie7g11oBDiUivv28VNUxl8N227/hf3v84ZNwACDuOpORks54fCBnRsbPZWTpO/FMryagomtNPZeTIclRWWiLvHRrM9n4EK6+CkxvdH5y/kvcE/NbLoZSAXMLCRG0wsBgu5XBNv2t4fMzjLSIxGxosRjO3nX6VB4Vkkr9FmZkaRLRIjXzLcnS05R64k+2XV5jHEwuf4L2177X6nTXn+uurX+ean67ht12/AWK+tm3b1nk5HU/QUAV/jYNFZwonn94gItv3fQ7fp8KJtS2PN5shKlMUNA9RsFC1k8ZzaVHvjrwnwJxdc0h+MZlzvm4d6WxN3nNE2xG8OvVVzux+npAoVToLwN/IOhMuKBW1zdwgMxPSSk+nd9xoahpqeGrRU06d/8P2HzhacZSMmIxWBi7peRsSYjFgu4Ki87a+TDjh4nu535YPodMpI71rjfJyy2tKVSOYTiccp+0vd+38YR/B1L/Vf3Yc+BJ+yITCU7K6dToRANQskKxH49533z6RpWeLemM9y/OXN0Vjgn2nn81Nd8lmYSh0I4NNwqfet0rjRD02W0xoL4IAluUva1GzVqK5tE7zZWKnpE6c1vE0chNyrbbrdqbf7v/C0gtha2NgRNY06HC1PLnXlGEw/HMY/Ib4d0O1c2vcin2w6gY4tszpYctiw//Bymu9tu42m828vOJler3XkcPp72M0VAr5PSfwdj0/cDB3h7wNF9f6rCxeQUEBDz7wALNra7ElgD8UmF1by4P332+R+kweDFNWiFq/NpCT6ZecLOb2a6e/xvSu0622ExkaqdWmdpGCggI2bRSSreef7p5kqzWa7z985tVlrIHt/4aF0+zWLWsxb8v3wIKpsPR8sXcMcWMR7kEclo3o8zSM+cHpZ3ZJCU1SglbXrfE9IOdCUQJDwz51JbKcRpL0tBQUrNPpeG3qa4ToQ5i9czZzds1xqtt3znyH2kdr6VVzC2AlY7OhWtTyO+FYYtKX8ehauXQz/Noddre2mzmDlOm3Vyg3cvOgmzn5wEn+c/p/ZLfhcG2a2AfG/AiZU10eZ0BSsVckBFU3qyVctBR2vQsNlfLa6P9vGPCqrEM7Jwnp6ML6XYDYgxRXi5tnS2EEgMh0yDqTA8fEgyDQM/28vbfVnH4qc/OvNzs8RnJEREeLSHy7dLsHhn4IqSPdH5y/0mYKdL9PRL0rwOwds1nf/krykz9yqq7flE5TeHL8ky0MZidOWBaRNiOO5LL7vzCnv4iM9icOfgsLT4eDtiVwTkWnU0biM68ojycXPclnmz5r9buKCktR9/h4YWxZc0RIHw7OEtGeer2e5ORk9L5QWVanF5l2WdNbRuxG50Jcd+EUNJvh7ztFRpdOB+0uhhFfKqtbX18G86fAhodkHa6UvGeb2DacrDnJuqPrMJ+ymbOW6deE2SQ2wFIWQrCi0ytikcjKAh06zooSBt731r3HS8stBbQfmfcIb65+0+b5r69+HYCbBt5E2CmR583ldt0Zqtvz1myGo3+Iv5347jB+DmRbN8Z5jH2fw8bHPNrFqRtupZDem7GxwkCjCkWLoSTPvTbCEiy1/Y6vErKFdbb0wTxIXA/ocrt1ydfKg3B8ZdM/k5KEY95shm3bbDd5ztfnMPLDkXy79dumz+w5/Wxuune+AX+NFUYeN/Gp963S1JUKOe4SF1IwG+me0p306HRqGmpYdbj1OlAKlDCZhONHDtXVItAQ3Mj0a38F9Hte/I26dP5lFofQxkfgm3j5BtsT60Q9SWfrdsql12Mw8iu7hnMlqTfVM2vLLI5VHSM/QWTE90+Y5FQbatTz8+e5+/577zE+NNSmw09iKDAuLIwP3n/fwZEWsmLFAthepp9HJSGrjsDft0ORezWa/BVJslW/TEi2flhsQ7LVbIY1t8HWfzndR/OMTVWpr4A/R4usa0OEWC8MeNXu/q/FvI3pAL3/Cb2f8NaIXcZuMGnqCBGo5+QmQlrr2HTe5pwHo74Wmesa9vmpPay40uFhp2b6gSizc8cQoczzzJLWJSAcEWYIo6pULIZa2d2KFsKOVzxaL90bePR9G9dNKDWkj3ermeaZfmYzJEUmkRCR4FQbbqtQBCsFfzUmBDVzbh/4CtbcLF9mPetMaCMv27trSlcA9lWJPWRlJVzb/1pqHqnh47M/tn1i47pasvMGutPP2+tj/1uNBxi7T+x2eIxkwJYlixWRAh2v8etitG6TOhL6/wtiOyrS3OaizeyJ/ozjsQuccvpZQ5L2TE1VwN6eOU0YIJKclx9Rlezp0PspyDzDqdOkzEh3nH49UkXaw5ZjW1o5ik51ru8+sZuSmhLCDeH0ThPh0Uajke3bt2M0+oDDyBABvR6Fnqc429JGCSnA9LFwch3sekNI0oGooRPdVtlxhMSKAsG18jI3JXnPggL3aof1SO1BuCGc0trSVs/RUzP9Zm2Zxfx980U2RNkO+DoSNj/peueBQtES2P+lW01Im7ToEyO5fcjtmDEzvO3wpt8XVhbyjzn/4MftP7Y6d0PBBpYeXEqIPoQbB97Y6vdK1YFze94W/AkLpsBm5zecipH/vcicMXpO31gyCB9yT8WlFafKJqvC+geEAazBCY1ue+z9CHa+3jJy0lukDodB/7Euq7z2DvhzVAunRK/GpFR7df2k+hp/7v0TgKLKIgorC9Gho2dqz6bjHG66218FA14RdaXdxKfet0pTdQgWn+1U8NOp6HQ6xuWOA2DBvtZGq4gIyzpTkkk+UX2CZxY/w/82Wa9xJBk6o6MtmYJOExIFPR5wuw4MIBztSQMtf8+HfhJRy7bIOU9I3Gaf7X7f1siYJKLIvVTbL8wQxrcXfktqlMX60T1solNtqFHPz+7cLf5bBNH4KD998QVX1NTIOvbK6mp+/OILywc7Xodd79g8vinTr8J2pt9h898sPrCYqnqF3lXNKd0sAjMUUsLxJ5pLtv7lSLJVp4P8b+Gw83W+rCmNqEJJnpCelPZmQ951KBHfYt7qdOI5njLMC4N1D4dlI8wmp0u+OFQ10JBPl38ImVgHSHuQY8dalnK5Z/g9PDXuKV467SXrJzpAmpOt9iBtpsLp60Wgkh/j0bWyIQL6vwjp49xqJjtbJDxUVVlsoXL4cfuPpLyYwiXfXWI/6LD6KMw/TQTTabQk4zQY/QMkDbB81vV2Uds1zMkoIxkBeKNzRgOw5tgCzBib9h/hIeEkRtp5Ma64GmbFUlosTgh0eU9v722D2DPkG3RMcOyYkl5WDg2fJ9aqE3Ue4HRKEpH9VRG7Zct7bizYyK87f23SMJZo7vRzm9QR0OcprxkfFCMkGno/5nTNHyUy/bomd0WHjhPVJyiqbLnqONUwvfrwagAGtBnQQkanRqYxwKPkfy/PeJ00EE7fKJzDnkKnE7UEh8qTfkhOhvBwkX1Q6Ia9PMwQxsDMgYCQN2uOFCiRmCgyNu/6/S4mfjqRJQeXiAyC3Mshoa/rnQcKm5+C1Te4JVXWfLP9n9P/w45/7GBE2xGtjrv2p2tb1cZ8Y7WQcDuv+3lNxrDmKOkscmveJvSBng9De8eRqh5jwEswI996bU2FUCoL91SUct66Rb+ZjbX8FJKoGvwWnLbCpRq1HqX9VUKGpZmBS47Tb3IHEcG5YN8C6o315BUKT0GnpE5Eh0U3Hecw0y91uCgQr5B3wSfet54gpj0M+1hk37uBvbp+Op1FFl3adO8q3sVjCx7joXnWM/PdiqTe86Goc6RkYELvx2HSArHOLdsFS86F1TfZPycyA0KVUfqwitksZMG8RHZcNl+f/zUGwkisGEFIdZZT50tOvz597B+nNDbnbt6TsMi2BKbalJaVIbdKejpQ2jwadOfrsNO2ssH49uOZdf4sZk6Y2eLzujqL/PLHO//N2I/H8vKKl6204CYZk+DsA5BzkfJt+zAuSbaesQ0mLXS6L69kbMohdTicWyjWBCA7GNwf37nSPsRqpt/h3+DrCDjgXICNw/uY/wOsvhlq3DBKBAt9nxZOBgfExYmAI7O55b3MisvisbGPNdWBk8OR8iOM/2Q81/18vW31H50OEvuJuoN+jq/P25AQyGlMipXq+n2w7gMmfDKBTzZ8YvO8wopCiquLqaqvsr8+LdsuFF1kBqAHFTG50HZGy2DMuK5iPSC3tu+eD4XqxvHlDg8dnDWYuPA4SutOUhq1vrHEgAw7U0JvTCljKDohIg4DPdPP22hOP5WxVVOjOStWiO9dujg68Gr4ra//1XjzBJv+Cb8PUuT/QnL6VYbvkZ3p99GGjzjzyzP51/KW0iCS08/ten6mev+/z8Y6qHRcwFdCCadfZGgkHRKFsPjWY1tb/K65lCBYnH5Dsoa43qEnOL4alpwHa261fUzlQVh2Cez7TBTQViDrwi5OZBbrdA42aE4wsq2QMV6e33IR0nyzlleUx9GKo0SGRIpslqgsGPGZyAgIdno9JvTvcd/pd/SoeCR1SW75onrrjLcYnDmYkzUnuez7y6g3WnTm4sPFZLt9iPXN4KlzUjUiM6Dvs04HKihKTHuIbOPRVA1PZ/p59T6Wbhcyk2U7xb/Tx0OHq5RrX6cXNZwAak8ol0Eoh0OzYfG54hpPpe05wunWbCMnOf127wZbdoF+Gf1IikyivK6cNUfWEB0Wzbndz+X0Tqe3OM5u9Lu/r0m8SUiU+HuM7+FWM+PbC6ff5qLNNFjJZJAkPqsbfVT5ZSLwIifeuiSZy/X8TA2w/SXY9TaYPGR8iusMQz8QJQxOxWwWBlg36844xFgL36fByqs9288pjG8/nplp+xi6c27T+kYOZjPsEmVVmup7qk7XO2CI7Ww4tYmPi0NudbdCIL75i230D0L+2wa5Cblc0PMC+rfp3+Jz6Z6GhcHqo2I9K61vFUWnF3KE4Wp7pLyLS5Kt4UkuqSb5jNMPRJaOj9biUxK7AWvRbSFjitOOHYeZfsdXwO53ocFOsWQNp9DplCszsL9kPwv3L2TennlNqkKtAkjLdkC1MrU8A5pd78Av3WXVZbRHc4lPgL0n97Jg/wKWHlxq85xjVcLolxaVZn9Opo+HC0qh3SVujTGgcTLbuQURGUJZTuc48DhEH9IUkHg87k8qzIXkvNKOe+beg9FeaZ0e/0dh918BHWFhoiSIhnJoTj+VaZ/Q3u7vKythVWOpjvH25JTNZuh6J3S/34cqR6tI3UlRM6XeTT1OoGOSyMasCy2i4KS8VL+8IhFaK8lCSkhOv/R0NweV9wT8lOOU08ynMJthdmdYcoHsU5Rw+gH0TBNSZac6/SSHrrQoXHJQ1LwYnOlj8qkJvWDwO9DjQdvHhMYLve5yx/LBitBQDTvegAOzZB2uVEaRZBRpnunX0EBTRm5iIny1+SsAxuWOIyIkwr0OA420MSLSyw056LQ0Uaetrg6rxsgwQxhfnvclsWGxLDm4hMmfTW7Ksn1pykvMuWyO1cxAaD0nVaFoqUclNWVjNkHFfo8+86V5WVxs2znkCqpk+tUWwbHlomaHJylaCj+2hT0feLaf5lTsgcM/yVZ2SE0V89Rkgh07rB9j0BuY2F7IBv6w7QcGtBnAdxd+x2unv9biOLub7tKt8F2KcPxoyMNNR2nnpM6svG4lR+89Sog+pNXvJYlOKdPvYOlBANrGW5f5djnTTx8Cp2+CKass9fg8QYerLRLl+d/DpifEz7XFQoY5/3vP1tszhEP6BIjv5bk+bNA+JZMQU0xT1oIcysqEw7d5sJXqtJks7qOPcvall/JZhLy14qeRkcy49FLLBwk9Icr5wonSczU8LZ/8snwMOkNTLXFFKd0mggKDDJckW6uOiFpfRucWQz7h9Dv8C2x5PmjUn6RnW2GhlbIRCb1h3Gxoc5pTbTq8j72fgPOOQ5RW088h+T/C/MmWIDw7WKvrB1BVX8W3W7/lyYXySnNIa530SLFeiIkRpVtaMG8iLJ4hq72gxmwCzG4/TySn395GhXYp+GV9wXobZ9BkL0iJSm2akzaD0gzhEOKqLn2A81MHWDDV8u+V18CXBmiolHd+1jSY+BekyEuEmNxhMmGGMIgq5kjiNxwqz2dZ/jIMDtTppD1ISormzlAazemnMh2SOtj9/bJlUF8vUqI72DtUp4NO10PXfyg7QH9l0H/grF2iHoibxIXHEWcQHqe9JXtknSNJY/VOb+n0U6w4aXgKRLSBSOc3lz6BTgcdrhHp5jKNXko5/XqkiHBnW5l+CQlQUFHAlmNbMOgMnNbRslHQ6/V06NDB68VXWxASBZ1vsi8rFxYvIo7bTLV9jJLoQ2Hjg7KNvQ7rL8hEchZtPbaVE9ViNSjdR4MBjKEnmyQkbxhwg/hFxT5YeS0c+d29zgMFs1lkL7iIwWDJXD561PoxHZM6MuuCWcSExbDowCJu/fVWKupEdOzUTlPR2VjZKSXv6fK8rT4K8yfAch+IHKwthp/be7SuYEyMxTGnpMSnKs7btDFCdrhT61qRipI0QEhtR+eKuVS2wxJN6ebcskm3u+ASo5DvOpVjy2DOQGFkaUbHRiX5/fttNzupwyQA/r3i37y15i2rx9h1+pnqIK4bhClTBMcn3reeZNX18G2iW44/nU7H0OyhLSTImyNl+klOP0liOSfOuqGy+YbbafQGkUnvDcxm2PYS7Gh0SkekwOTlMOobz9c0H/W1kKj3MpLx2Rmnn/ROTkmxYvD0IP48d6+/4QYW1NezysFxq4CFdXVcd/31lg9N9SIwx4786+wds3lz9ZsUVxU3fSYZMyuThLRPn/Q+xITFuHgFdlh+Kfw1Tvl2fRyXJFt3vQnzJohgKyfwCaff3o9g02NOv1v8dd4mJSlTNqI5DjP9QqIgPNn/SqyoQW0RHF8FNY6z6mxl+lXVV3HBNxfwxKInKKxwfJOltU5KmFjrWK2x2fV26Hitw7Z8HY/P2y63wpnbhQPdDSQ7tpTp1z9DOP3yivJaqAA1R3L6xRnSaGhoVGQ99V7WV4jAc28FuvsjaaNb1vRLHABZZ8nK3HOFq/pdxYn7TzBF92+OJIu6x5f2utT+SX/fTshBIfUaDNKe3n7P+tdbPQCRpAZtsaCxTMf48Q483p4wLGk0kR0lJD7zKx2/UAorCjlWdQwdOnqkttTTUUzes9vdMHW1fy82+zwh6mTJDOVo7vRzJzheuie7T7a8l80dDBkxGRTcW8DsS2aTGm158+h0OuLi4mw6KTzO0T+F8V8ObWdYNwp7An0ITPgLxnwv63Apks9dec/U6FQu73M5D496uEnarHk9v/+sfo3yunJ6p/Xm7G5ni1/UFIkNcamdAlfBQsU+mBUjMofdoLnEpy2mdprKqutX0TmpMydrTlInI3tOqQwxl+etPgz6zoTOt7k3ACUIT4Hu/weZZ3i0G6WkdZrj1Uy/A19bMhmivRCBHRIliqFnTxe1nH7pJrILqwvgp1zY8IDnx9AcnQHqToCxpdxobq74bs/pd1HPi5jaaSpx4XH0TW9d87SuDsrLxc9WDWFJ/WHyUmh3oUtDPxXV37eeJro9pI5sda+U5FR5z4Nl9jP9pEzQttZ/bZ3y3bDhQaeN426h04k1x7jfLJ9FpkOoBxwlPoIUNOGMvKf0Tm7TRvHh2MXu3F08A/4c7d0BOUFGRgbPv/AC08PDbTr+VgHTw8N5/sUXycho5k7a+ZZ47hevttn+Hb/fwT/m/IMtx7Y0fSY5GE5ECWlPW+oHbtPpZrF/DDJckmzNPBMG/sdpKVSfcPqN+FK8i50cu7++cx2Wjdj8rCUrXCYO72PVYSjJc08yL1joeD1cWCaC8RxgK9MvJSqFfhn9AJi/b77DdqRMvwSdWMxYdfr1eMDzgYFewF/mrZTpV1golOzaJ7YnLjyOOmMd245vs3qOJO8ZbhS2uIQEUR+wBSfWwNrb4dCPnhl4IDD8E+j/ouXfXW+HMT/Ir+lnMopn6J6PZB0eExZDdFg0Ial7ORmzAj16LuxpZ29orIGdbxBV9gfgYuChn+Ht+ao5/VQmKcL2gqyoCDZvFouZcePsNGKqhx+z4G/HRXKDhoZq2PsJFPylSHPt4jqBWc/xasfRRZK0Z6ekTkSFWrT0GxosWWpuO/0CCZkevORkMRfq6y3yja4wrfM08m7JY85lLetunFp3KjEykdM7t6xnZDQaycvLw2i0o0mtNPu/gvI9Qq52ybmw8Ezv9e0MKcMgzNqqujVKyXsCfHbOZzw78VnSosWkkjZqUYmlvLryVQAeG/MYein6P2mQ0H3vomVFE5kJqaMgpqNbzUgGRUdO3B6pPVh9w2rO7HxmU2amPZTK9HN53oYnQ/f7IGOCewNQAp1OLNjbzvBoN9LczM9Xrk2v1fSrPgorrhT1TNUgY4JwEIenQEQ6RLcTdRCUpmKvWNvUW6klkzIMzt4HuS0jKiWn3wE76rDxEfHMuWwOJQ+UMDZ3bKvfN0nQhUN0tItjdwJV3rfepNcjMO5XCHHvP/NQ2SGu++k6zvn6nFa/O1XesynTz0pNv7IySwR2nz5ODCD/O9j6AlR4Oco6JNJ7gU3Nqdgr1AIOzfZqt65k+knvZG87/ezO3ZA4IUHvw9x97708NHMmYwwGpkRE8D/gL+Bz4LSICMYYDDw0cyZ333NPyxNThkLXu+zW0ZYM16sOWVyK0rr1kE5I1XvM6df5JmHsCzJckmxNHS7+r5yoid7QYAlyUtXpZwgTf4tO4s/vXLvBpPnfw76PnWpPmpM2M/22zITf+oggKw37OJF93zzw8FTzkCRBP2/fPIftSPWLY412Mv0CBI/PW7MZ9nwIB79xq5mYGItzfvNm0Ov0Te/D9UetS3xKmX6hdeI5bHU+xveGcXMg5yK3xqdhB71B1O0+8KVTp+2JEiV2uoVPoE2snYWoPhzOO868k68AwZHp5+33rOb0Uxl7Xl4py69PHwce79piSB4uDEwaAp1O6BXvfk+R5p4Y+hrT1lXT/phjR4Etac+NG4XDKiHBzYfZng9gzW1QGwALzU3/FJJ1Jutp/c0JCbEs2qSMSVdIjkqmV1ovixOoEWmjFhNnP2rPKw/p8t2W1e6OV+DPkUJCc8h/odejnu/fVSoPQskWh4dJiz6la4eBxSCmTzhEu4R29EjtwXk9zrMcoDeImkMGrb4fhnCYMFdIQ7uBZFC0l+knkRCRwN3D76ZTUieHxyrpLHJ63jZUOl3LJRDw60y/yDYwdjb0/7eHO7JBfA8Y/AYk9hNrkMmLoaed2quusu9zUR+lcp/sU9o1Lg8PHBASWPbQ6XSt3o/QUu7K6tJ120tCYkdB/NH46G0iQiL4cMOH/Lj9R8pry1v8rlWmn1TTL651Kt+mTeJ7bq6Tc7X7/8GkxZBmr/B4gLH3Iyhe6dUuJSdCWZmVulU2UCvTD+zM3RGfwrhfvDsYF7j7nns4cOgQYx55hH937s5F4Zn8X2x3xjz8CAcOHWrt8AMRdDHwFYjvbrPdUW1HAbA0f2nTZ8XF0KCv4GC9MHyOzvHdTEh/xC3JVidoXl4gzoOlTe1y8DshM+4i/vrOlfaVp2aIASK45gzrmUTWqKkRmUhgx3mbdSb0ftLtoJ2gwGSEw7+JGtgOyMwU9p6qqtZSrc44/aS1TnitjUy/k5vgzzGQ/4Pj8fsBHp23Oh1sfAi2v+J2U/2Foifr1jX+O8N+Xb+eqT3pm96XkGoxwa06/SJSIHOqpd6zRmuKlsDqm0USAQj7+Nq7nGtj6lohby+TDQUb+LHsEQC6Gx1Ie+p0EJ5M/jHh3A0Gp5+30Zx+PsyaNeL72NYB1y2JzBBFirvf5/Ex+Q2GCBj7C/RRpv5RTmoienNYk/HSHlKmX++0lk6/5UK1hREjwC0Z36N/wN4PISQAZIz0oRCeJjI0ZKBUXT9rSJu1t/ffydD3h/LHnj+U70QO1YXwS1dYc4v4d9/nYMCrQkYu9xLI8qy8n8sYa2B2Z1jv+DkUG2vZELsr8Qlwsvokv+36jTpjXZPTr1tyT9bftJ4/r/izpQG7vgxOrIWa4+53rAE45/STi9msUi04iT0fwnepcGyFCp3b4OA3MGeA2Kx6CCWzcCW8eh/bnKZO9o83yToDBr8NkTbqp+3/qlVEbps2EBYmJDpdnaeS089mENqO18TaREMeVUeELOaRuW41kxKVQnac8NZvLNzY4neS008yYC64agG/X/Y7XVO6tmpHcvo5leUHIoo/bbR/y807Q3QunFsEfZ/1ardxcWLv0Pzd6IiCRk1DNZx+gUBGRgaPPPooa7ZuZdSUwwwau5Vbb3u0paSnk4zKEU6/ZQeXYTKLCIwTJ8BgiuStgSt4+4y3bcrvukXxGvh9CBz6Wfm2fRyXJFtriuC3fpD3pOx+pOywxETZlSuUpaEKVlwhjLtBhhTYtHevlV9GZojMcJlI9zEiwvIObUXm6dD7cc3pJwedHpbMEIoADggJschA7trV8nej240mVB/K/pL9HCixI1sB1DcGk4dWiUy/Vs7buhNQth3qZb5Mg51R38IwedKO9pCcfusbfXz9M/qTEpVCqN56bbmvzv+KDTdvIKaqJ2Crnni9e3V/goHyXbD7XSjdKv59+BdRksIZ4rrIVvQCyIzNbPo5rbi1EkkL6k5CyRZKi8VmJRjkPb2N5vTzURoaLDI7PXuqOxa/JWsaxHVWpCkp6rmuznFm0gMjH+DTGZ9ybvdzmz4zGmFFo+14hLuqLSO/grP2ytdh9mV6PiJqE8qsvaSU02/d0XVc8cMV3DnnzqbPSkrApKtn7qGvWX14NWa1FhDmBuj8D8icJv6dMQFyL1ZnLM5giIA+TwvtfhkoVdfPbDbT5Y0unPHFGWwo2NBi063X6VssOoBGw8cgyP/WvY4DhSO/w7LLhAHaRZrX0lBq2lRWiucmqBQxHZkBSQPdLlyuKKYGkdnvQTmh5k4/Je5lfb3F4eDRTL9Ds+HkBg924AImI/x9B2x4WNl2kwZC55tt1+vZ+LCQnmqGXt8y288VjjfGSdiUu5q6VqxPNORhqhVGsII/3W6qKVr6FIkkaeMsRcx3T+3OlE5TWkjPS2xs9Bf2bV3O0Tb534t3ajAZXHR6iPB+GLJO53xdP2l9lZlp/zivsusd2Pc/tUfhFCEhlrkkOVKtYjbDkvNh4yM2D+nfpj+RIZEUVxez47jIyCouBh0GRuYO5uZBHnLY1J2E6iOylFUCEaclW0PjoKHcqWebFHSomrSnPhRGfw+9HlNpAOrRudHcs3evWHe2oK4Uiv+GuhJZbTVXNdBQAJ0Ohn0CPR+Sdbh0L3fubPl5TFgMPdOEUXRDwQa7beTdkkfto7WEl4njWwUdpo+D84qgw9WyxhT0pI2GuNbBYs7Su7fIhD56VLxLL+9zOUX3FfGv0/5l9zy7QYcrroZZMWCsdXt8AUvO+XBOgSVxYPgnMOOgc23Ulwunocw6pmnRafw4bQ1jtmykrCjB/sFHfoffetHWIEovaZl+yqM5/VRGbyPlKz9fOJiiohxEaNaXw+Jz4ID8dNugor5MkQ2OLqSWTR2uYXnX0RSeqLZ7bNeUrlzR9wr6pFvCpbdsgfJykeHUq5e7g9EJGbNAwMlQSKWcfhV1FXy+6XO+2foNZrOZujohf3Usbi4na4tJj05nYoeJrc7T6/V07drV5rxVhKgsGPQaZJ/luT48RY/7Iec8x8dhcS64aoCW0Ol0DMseBsArK19h/4lD7M54kaj4KusnxHSEvjMh2fl6FwFJxV448AWUuy4HlJYmpnJNjfwMBEdI7URFiQwld3Bp3uZcAJMWQqgPZVTnXgIzDojNqodITxcGztpai5PHHaT7aDCIeg4ewdQAa24SayGzA+1Kb6I3QNFCRZw6TjH8ExjaOuNOcvrt3+9asw4NYRGpIhJUIbzyvlWTqLZw5nbo85TbTdmSSJLkeq1KnjXj+HHhINLpnAg0NDXA6puElL4qaS0qUnVIRErXlzs+VkEkZ4KkTGGPykqx7wBwKTGt5rioXShFhoP4+ajj51nT3G0og8pTFnmbn4Gdr7swIHWR/g9PlZxrgU4Hx1dASZ7NQ8IMYQzNFuvPpQeXYjZ7ycnQ5jQ455DsNXog0lyy9ZVu3bk4IpP7Yroz6C4rkq2GCDhrD/R5Qnb70n1UrX6YPlTI3LlYh9qf37kZGSJAsKHBSrZf/rcwdzAcWy6rLSmowq7zdvOz8OdoaLBvE9JoJPcSSJUX9d6lcRl5aqYf0GRb23psa+tfnkKYIYySE0KBIJBr+nlt3taVuv33HhUF3bqJnzdsAIPeYLPUVXltOUaTiP6V7H5WnX6J/aDNFFGyRMM6oXEQmW6prxmW4LwtOe8J+LWnWP/KZFKPQcRV96G0VNgVbBLfg/quj7G9UMzvYMj08/Z71v/e6kHC7t3ie8eODvbSFfvg8GyRtqvRku2vwTfxIrrLTcJDwjia8AMnYpey5bA17Qj7LBO12Rk+XBg/XcJshrX3wPHVLjbgoxTMg5XXyapRqJTTb1DmIEL0IRytOMqB0gNNBpQjKZ8DcHGviwnRh1g9N8xdD4Q9AkWiQEYUUIcO4ruU0ewONw+8Gb1Oz1ebv+KNhr5sz36Alw7YyI6MyRXRhkn93e84EGh/BVxQBumu12MKDbXMTaUkPqU5qZQkpM15azaLWmRFi8W/60rgz1FQttP68QFOSIjFuKmExGdZmfgeF+dBv4DOAKN/hEFvWDY0vsLEBTBF4Xf21hdgdlfbG6+00Vafb7m54rurgRZ2DdM1RaIWrcwIULl49H2rNvoQETkdYks/TD7921h3+rVtVAk8dAiWH1zBM4uf4c89rZ02UpZfly4QLVetTGcQNTQHvOzqsP2X/V/AoulQ6riGsZJIhks5mX7SuzgxUcjUOU14MhSvFutzsxkOzIK/xskOpAwLC4NlF8FP7aF0u+UX436BocrUW/cm6eniu12nH8CMfBhrX0JTquu38tBKqqqgpr6OTe1u4tfDn1BvDM5MPG8hSbb+smA+nfrcDJj58oO3mT5+PM88/TQFdlM57SNl+qmSIWZqUGTd6q/vXJ3O4iw6NUOMlOEi2DPWcS1xsNxHu46imkIoD859gsuYzbJsHNJ93LOndQ3qJ8c9yaG7D/HgKHn1sm1m3x5fJepfBkjtdo/P272fwrcJUPCX202dKvEJQrXpeFXLSNPHFjxG4guJvL7q9Sa7n9UMsB7/B2O+d3tcAU/ZTksQV+UBKHPSd5BxGvR8FAzypZKjoy17iqIi28fVRffl1flPcaS8C/HxECm/Cw2Z+Jh1JPgwnfo2a0Ry+nVytD5J7AMXVkHXuxQdV0CQ2A9yr1BEb12n05Fg6gjAtsLdNo97cdmL/GvZvzhWafFKmUwt6/m5zMl1sOMVOPClG434ICWbRR2gEsd1qtJEfVe3nX5RoVEMaDMAEHU1SkqgXl9GQfxPgJAbsIbJZCIvL8/mvHWbPe/Dzx3h5EbHx/oiZjPMHSaMYQ6QNPv37HG/2zO6nMGPF/1IVGgUNXphDbuh913uNxwMhMaKLzdpLvGpBFKGmBKSkC3mbX05rH8ASreJX+p0Qopry/Pi3yfWiij9bf/yTQf8we+EwdmDSFm4jrKD5CA5bz0q7anTQcoQ36x3Gp6svLdTFyIiWnV2IoiMtUJetBmS088jmX77PhU1XY+vdK1xK3j8fesL1BYr8r6XMv22FG2hzljX9HlamnDk19XBz1vm8diCx/hyc+s1pEv1/HQ6SBkmMoiCjcxpMPQDUd/Pi0hGaMmQaQ/J6edUPT9jDex6Vzj2dDoY8i4M+0D8nD5eSAvHtG95Tv4PogZuM5rm7sC3IGs6xDeG9pfvhpBY35LNlokUDOPQJyQj8OS6Adfx9w1/8+70dzlxAkqj1nIw9b88tOA+mwGHbnNotqj3qsErL71Eu+xsYtbN5KWK7bx/7Ah3bd/OkpkzaZedzSsvNwYyHJkLW+3LzjVH1Uy/48tFPfjtr7jchL+/c7s2qg+2cvrF9xDBnjKVCGQ5/Qb9B84tdKpWYFCz+Vn4Jhaq8h0empUlAlVqaoTyWXNyE3LJisuymR0G8O3Wbxn/yXheXPrvpmz3VgGku/8LS88PCElIr8zbhN5CCjUize2mBgjzGxs3ijIe646uI/e1XEZ/NLrFcasOr6K8rpzEyKQmu1+a+90HL38MgzW3iJ9XXgN/OKl4lTkF+j4tMgadQHLU2nL6lZbCgw/C4sUiMeZ6eVWC/B5vv2c1p5+PIhnCHTr9QNR28yUJMl8hfSyM+FQ4RhUgTS9Wk5uPWZdtqa6v5rmlz3H/X/ez7ui6ps+3bROGz+hoJ2ulnErSQJi2WUgoBhLtrxA60zIk65TK9AMY2XYkAMvzl1NaCgWJP2DU19AluQsD2wx0vwOX0Ino/5gOKvXvJjodRLeDSMcFZKRMv+PHLRJU7jC963QWXLmQ2Oqe5By7gWndbcjbFC2GuUPFZl5DULoNChe51YRkWFTK6ad0pl8TpVth24stC1gPeQdGN9Z4zJgoaqb2etQ3Jevy/gmb/unRLnIaS6y2kkhyAcl5q/h9lKguFJJNvuiglShcJAzpStH9Xpi2ybY0y5aZ8HUElG1r8bHk9Dt61HFtYmvYramRPAS63wfx3Z1vOJhZfTPM6Q/NHHWukBOfQ1JkEu0S2nG03JJubTBYAjJ2FuQ3Hdscs9ni9JO9RjU1CNUJX553niShF3S8VtR+9SJStoLHnH673oE1N1uceKkjhcEchHzv+Dlijlfshd2N2XrR7YQhae8n4t/Fa+BE4x4opj2M/cnS/rp7YfWNovSCnyFL3hNE9Pz+r6C+wuYhuQm5DMwcSIg+hJMnoTh2CQCjckbZNWa7xdbnYd2djo8LcF556SWee+QRFhuN/NVQw2XAROAyYG5NDYuNRp57+GHh+Nv/OWy4HxpslAs4BVmykJ4iPAU63QTprUtTBAtShtgO16sVAB7cfwQzUdmQMlJWprheb7F/tnLgymDZwWUs3L+Q3UUicjEkRJTXaUHX22HEF0L2UMMxSf1h2EeQ4n5plI4dxf2orBT3t0NiBw6XHWb78e3sL9kPQG1DbZMttVvMUBoaxJa81bPVZBRqBH5WJ1gVej0OHW8UP+deLvZsXkBSSbDl9PvmGxgZfT8zJ03m6afMjBvnlWEFHZrTzwcxGi2Sdx07Oji4cIHYYGl4nC7RQwDYeHyV1d9/v+17SmpKyInPYVKHSU2fb94svg8YIBYebpHQM3Dq+UmEJ8mOGpGcfiUlInLdHUa0FWmXy/JFpt/hJLFguLz35Z7bdDui881w5g5FMq9UY9TXIjLcAc3rlSrhXADoFjeYsVs20+fAf21v1hqqoPoIGLU6DE2suAKWX+ZWE9K9VFreU/EMsZShMHE+DHzN8lmHq1tK7IUnC0OmLzL4HTHHPIgULb19u/3j5ODxTL/9n8OfI+Ho7x7qQAG2PAtr74SGSu/0F98L2l0M+pb1LeLjxZfZ3Dp62hEmk8WgaTXTL20M9P+XmDsa8ml3EfR5GszuyaLqdDoO33OYXbfvol1Cy2eXJPF54ORB8e+4ti1+f+yYCL4xGKC7XJ9t0WIRJbxNfhaMhvs4I+8pBeA45fTrdBMMeAU6XGP/uFU3woGvhPMuOhc63wI5FwqD7uJz0C88jZB6KxaebvcII2v+D04MyjfQ6wvYtfNpPnu7O12yshjcvbt1OcgDX8HyS2SV3ahtqOVgUQknYoTTb3TOaAdnuMGg12H4Z55r3w8oKCjgwQceYHZtLbZM10OB2bW1PHj//RSk3ABT1oBennSeTSlBbxDfQwSwKRTo7I9ITr+jR60Ek86fDKvkpZDIyvQrWqplzjpDh6tgwlyIdWTYFNir6/fyipeZ8dUMNhRssHrupiIRxZQbIaKYkpKsxHAm9hN1Bn0xuDPA0estAWYbN0JCRALD2w4HYM6uOeLzwo3UGetIjkwmpk78zSQnW7Gj1h4XamHHl3lr+P5Lt7ugfaOtp+O10PNh586vLoT5p8EO52oyS9mZtpx+GzZAcuRhOmfspncfbT56Cs3p54Pk5wuHRlSUJULXJn//A5Zd6pVx+SUbH4U1/1CkqWHZwwDYUbUSs5Xo5vfXvw/Adf2vw6C3yG5JUq1d5KlKWGfrv+Dwb2404ONUHYEjcxweFhMjakOB5f/VVSSn38bCjew+lk+7YzfTK+RsLu2t0nyS/qaCaAEqSXwq5fSTDGFxcXYc7JlTRc2VtjOU6TQQ6PEA9HvBrawN6V2llNNP6Qyx0PpCIe0JQqas2TPar0gbBUkDPNqFZPjPz3c/C1cynngsYjp9AvR6DNLGeqgDBej7LExZCQb3a7cBog7u9tegwUbgQvZZMPJLiOvc6lfSM9dZic+SEuH40+u16HdFyTkfej2iSF2/iBDrhduys8X3o1XWM/2kSPr27UF2WZjoHBElnHWWK0MNDOYMhIWOpcyVxBl5T8kXJdvpZzYLqbpudwkFGXsMegP0oaALFYF7A18V5+pDYegHmCYspCHUig5X+lgY84MwAPsRr7z0EhPHZZO5eyYvlG7n7SM25CAB2p4LI792GDi08tBKbv31VqYvS6Qo4RfAw06/pAHBKcXbjPffe4/xoaE2HX4SQ4FxYWF8MGsxJA8SCiwykIKcVJH31CAmxiJP3ypDrK7UsgdwgKxMv+0vw3LN/uYpOjcuX605/ebumctPO35izeHWSQ9ms5mNBUIyPV0nHOBWA9WCVaXAHTY9LrLqFKBnT/FdCi49vdPpAMzZLWyBqw6JBIuh2UM5flzYxazW84tIg/NPQp9nFBmXhh1CIoWMtK168jaw5/QrLRV15l9a8T/qTt+nwCA1bKE5/VRGr299CyRnRseOMuz/fZ8XBiUN6xxbAod/cnycDMZ17Y/eFEYVx9l7sqWXYlfxLhbuX4hep+eafi0jZGXXZ7RFXSnkPS7q+QUqGx6EhdNEfRs76HSWmjMb3SyDkxmbyfDs4fTL6Ie+PJs2JedyX/aPdEyyHYWm1+vp3bu31XnrFsY6IfHlZPSMz7L5WVE3zQFSJrNSTj9Jfk6VKFt/JucCEf3lhsO5eaafEnspJTPE9Ho9PcpfQv9bd/E89XeMNbYdPgoQF2cxnLgrkyQrYtodkvpDn6cUcZp4jOTBIqpYqYCOA1/DurvA5LxGZ7tGO7SzTj9ps5acLBx/LZBquW56wunx2MNj79sAp+4UqVAp06+4oTHTL75lpp80x50KTIvtJDI7pVptwUhUtteVNzwi72k2Qd5TsPlp+e+V+G4w/nfr9awyp6BP6hMwc7e5HOR8kwM5SBCKLO0uFM5QO4Qbwlmwf0HTvxMNWfTL6OeZizAZ3ZYQDgR++uILrpCpbX1ldTU/fvGFmBN2pFolzGYVnX5lu+DX3m5nngXCO9emxOfU1bJVMmStW3vcD6O+cXp8QUt9uQjEPyjv/0y6j/v3t1Z26pMmDEGbCje1Ou9I+RGKq4sx6Awk1AnPklWn3w8ZsPRCuaP3abw2b4+vhCO/KbLJb15/02y2OP3m75tPbUMtKw+L+uBDs4bar+en00FYgsP3rQZwYBb81hdObhClBZY5qfAUEgsXlEP/F5w6Tbpv1qTR8xorZuXmWpI6ggVvv2f9960ewDjlJMqeLjYXGtYZ+wucfVCRpjq1DyeuagDRNZ3IL2kp5/Lxho8BmNppaguDSmmpkE3S6Sw1zJwmLB7O3CkkdwKVDlfDkP+KKGEHSJIAm1qv9ZxmwVUL+OaCbygsFMbYdBkqo3Xu6opao3If1Jd6T/7N0xz6AQ7OcniYlHUi1TB1F7vycxIVe2HfZ05HKmnYR6p3U1mpTI1GRTP9zGbqM86G3EvF89SfOfoHfB0JB77waDfdGm3527bZP84RHq1xY6r3n2hdU72Qg1JivD3uh0mLxQbMGg2VsPoWS82tZkiGlLVrnRuK5PSz+o5sqISaQqhVoNjuKXjkfetLVOyHv8aKWmpuUtNQw5TPp5D0QhInqy1eoexsqDeUUqcTD+ZT5T2ljAjZTj+T0X/mnScZ+xMM/a9Xu2wu72nvFtTUWJ69jp1+ZhEgeegHqLMfeOcMgTB3nZaDbC716WCO9G/Tn7137uXJyBJGbV3NS13WEmpwvAdyiZKNos7rtpc8076fUFpWhtwqnOlA6cnjMCsKdr3t8PiyMpENDyoYL2sKGutkmtxuyt/nbXNngiuYTDL3HynDIOc81zoJRnQGIXV/SF4gfmqqmEcNDZaSRxJ90hudfkWtDUGSI7BrSlcqSoT6Qav9h9kEyUMhtqtz1+DDeGXejvkRzjmiSABj+/YQGirsBUePQr+MfmTEZFBZX8kLy16wZPplDW3af1h1+lUXwMlNHg2EDRjMDcL5Xl8u5MfLtjp3vk7n0r2X7tsxK1tE4fQzc37/D0WdcA2PoTn9VMZkar1Ak+30M7u/uAt4QmMVi65PTITJhxcwfvMuchjZ9LnZbOabrSJy6fLel7c4R7qXmZlCrtVlottCQi83GvBxMiZApxtkFVSWnH7btwvDhjuEh4TTKalTkwxShoPdoMlkYseOHVbnrVvEdYXpu4WsUiAw9heY7rieiZTpd+iQ+zUawZLpZ9fpV7QUVlwJJ9a732GgUHkAfu7kVqZOWJjFIaCEE1fJTD+T2czWupGY+r7ofmNqE9sJci6CqLaOj3UDSeLT3bp+Hq1xs+d9+KGNf9Q13vgI/DUaSvLcbyu2E6SNti05pg+HPf+Fo3Nb/WrQILHRPnwYDjoRDyW9I606/UJj4Ox9MPhN+Q3KwGPvW1/CEAklm0VdEjeJCIngYOlBKusrmbdvXtPnWVlQFS4eykkRyUSHRTf9rqHBsk7tKtf+te9j+DFbRH1reBXJCN3QABU2ko/MZljVWHo8NlZI3tlFb4BRs2DyMpG9qACBMnedloN8/31hUPu+Day+SVYfNSXxJFQNpoOcqENX0YdD2/MgLogzc4H4uDgKHB8GQCEQH58A7S4VezQHSGsdu+UFPEXaaJhxANpd4lYzgTBvpeAVKYOoiZObhJqOA0WhsjJxnk7nwVrUwUhIFJyxFQa/Jetwna7lvWxOk9OvcFOrcjsbC4UMVN/0vrZtAjo9jP0Z+j7t1CX4Kl6btyFRitlUQ0Isdu4dO0Rd6ifGPgHAa6te4/ROpzOl4xSGZA1pchZZlffc/wXM6QsnNZuOQ3IvhbP3ivfFxHlwugv/Z8dXisB5J5CWNidOtLb15eVBdGgJYyOuE/vWIMLb71nN6ecjlJTAf/8LL7xgMZhKBnGbbPu32FiUbPb08PwXU4Oof1O40O2mdDpo31ZEDTU3llXWV9I3oy/Jkcmc2eXMFue4Le15bJmQ8goWWRYZjuyMDPHib2iArU4GqVjt0mxJOffkntshegMYrNfk8TsiM2TVwEhMFJsqs9l5uTlryHL6pY8X0WrJg93vMFCISBfG51BH1kH79GqMS3A3C9dstkSEpaS41xZmU2BFAMZ0gFFfebw2T48e4vuOHeJZ6yoezfQzRENkpvg/8XXange9n1BGgsbUIL5sZZLoQ+Cco8KQfwpRUTCgsSTk0qXyu7Sb6afhOpHpcH4x9HpUkeamdJwCwNzdFodvRAR0jO7P+E17eHHIly2OP3hQbMKjoy2Svg7Rh0F4MkTnKjJmv6UkTzjzS92MjHCCsDCLE8+axOf69XDddfDvf4t/S3K+VqnYC/s+F8+SsETflkhWCZfkIENihJPIQU0/Cekd6VFJyISeMPobyDrDg534PmdfeimfRcjbZ30aGcmMy6+Ekf8TdXId4BP1/IKoJrwt2rcX77zy8lP2lQV/wto7oMy+Zn1z563BXunvJRfAj/LmuEYj8d1lBXdL2FIc6ZbSjRB9CCU1JRwqa6nao0NHZmwmfdL7aCU/lKahEo7MVczufKoU702DbuKl015i2bXLeH3a6/x++e8kRibaz/RLHSH2VjGOjOYairD9ZRE4b6qXfUpMjHgmQ8tsv5ISyM+HelMkVUN+hc63KjtWjRZoTj8f4YcfYPZsYYRpaBAvqMxMByeFJUF0jjB6aVjHbIJF02GzMgVem+rhHDBhNBkBiAmL4ZsLvuHovUeJDW8pt+W202/zs7D8Mo/IZvkcSy+CX7o7PEyns2T7uVvXD8QCv65O1CmyGkXkafZ+Amtug5oAusemeji23GE2XXPZWyXq+sly+kW3heyzhWNSQ2CIgDPyoPt9bjUj1dt01+lXXm6JBnPb6XdiHfofUkg6+b2bDQUX2dnCEVBb67pDvq5OyL2ChwxhHa6E09cJB4SvkzIUev9TmUyaFVfAV6FCqsUWEWkimtkKIxuFCpYtk9+lFBhjddN9ciPs/RRq3M9W03CPJqffnrktIuDbZuuIrutATsPkFsdLEfSdOzthL25/BUzbpL1Dy3fBlplwwruZxpIB85AVhfKvvhJGlehomDYN7rrLTkNb/yWeJVrGpk2cloMsLRUTadJC6PWIrPN8wlkUJFx/ww0sqK9nlYPjVgEL6+q47vrrZbft8frFtjCbYNPjULTEyx37JiEh0Lu3+Hl98y1o2/Ng4gKHqkmy52NMB0jo4+owg5O6UuEwkikPLimOnOr0Cw8Jp1uK8AieWtfvgVEPcPiew9w/8n7bNoHSrbD+/+DEOmevILipLoCFU2HPh4o0Jzl1m9ffvGf4PU33VsJupl/KMLG3itQiEh1iNgub46HZwnlb8JfzbXS9W6h5OYFOZ13iU6rnl5UTQVSnaZA0wPnxaMhGc/r5CJsbgyamTIE774Tnn5exAe90PUxZpRUvtYchDIZ+AP2eU6S5nBxY3/5ybt6fyLL8lhYza7UY3Hb6DXkHxv4KUXJDsP2YqLYiOlZGVqPkXFDC6SfJlqWmypNkMdgN/XNlAH/CnvcgJNrxsf5CQxX8OVJkIztAymhW0unntqNIwyWkeblrl8XZ4wrHG30HCQlCitAtzA2QOpr6SD/IBpPL9tdg6cUe7UKnc7+un5TBEBYmjNAaCpEyQki86uy8i6oOCYUDK9nzQ4aId11+vnyJT7vZ8Id+gpVXQbXydVIVf9/6IkVLrdZfdIWxuWMJN4STX5bP9uOWDLTsRl9zfn7L452u56dhIX2ikCvLucCr3fbvL74vOcXGX1truZ+vvAK33OIgM7ffTBjyHqSN8sg4A2HuWpODLACeAQYDXRq/PwPsAOKd1ANsHhjj0WyUnW/C33c4FZ0fiGRkZPD8Cy8wPTzcpuNvFTA9PJznX3yRjIwM2PoiLL/cxtEWVHP6le2EzU/LrpXmiECYt9Izcl1zn05MLqSPc5hpJt1Hh/XE+78A42a7NsBgZd3d8FtvqC+VdXiXLiIg+/hxy75Qok96H5IikwixoSqk1+ltK42UbBb2ifLdTl6A7+KVeRvTXsizdrhakeakdee+feJdaDa3VpaprrZImasSmB9I6HTw9+2w4zVYezusu9f5NlKHC8UAvXPGGal8kuToa/5znz5ajXBvoDn9VMZgMFBTY5H0vPBCmDRJRuF1DfnkXqKYlF+7dmDUV1NLGSsPraSgoqCFYaU5paVikaLTyZBqtUV0DmROcX3A/sSAfwuNdUOYw0OlTL+9e0VWkDscPSq+O6rnB2K+9u7dW9nF1fDP4Kx9gSWtFBYPA1+Hzrc4PFTK9NvluASgQ2Rl+u1+H2bFQdFi9zsMJI6vhHX3QdURl5tISRHvLrPZPeldu1F9Tg9qGLqJf9Jx2NUBYcwA4MRaOPSDkFrxIO7W9WtuBFNcderwL7DqeqjYp3DDHmTPh/B9hvvRxV1vFxKvNjL5AJGBNG881BS1+lV0tHMSn83ldq06EdpfAaN/gNjOjhtzAo+8b32RnW/A6hvB6GaRYiAqNIrR7UYDItsPYM+JPXxccw77U99slR0mRVjLdvod/QNW3wzlChRu9XfC4oVcmZdl2SdMEN9XrWpZ12/bNmEwS0mRt54lLFEEj3qAQJm7p8pBvgK0A5YCdwFvN35f0vg9NSdHHHjwO1h7j8OSBc0DY9yq++6IQz/D3g9B5+1ic77H3ffey0MzZzLGYGBKRAT/A/4CPgfGh0QwxmDgoZkzufuee8QJxash/zshg2sH2c4ipYntBKdvgC7uS6MFyryV1jdbtkALdV6zyeG6Wcu89SBZZ0Gvf8o+PCLCYiM4NfjwsTGPcfCug0zpZLGRmZo9b6urxRdYsQlknw1n7YXMqc6M3mfx2rzV6YVdJ1GZDNfUVDHPjEaxz3z6abj0UjjSzAwh7T1iYiAy0kojSy6AZZcpMp6gYMwPMPA1GPAK9Hve9XZkZutKTJwovs+eLeqm1tXB2rXiswntPoKvo6Fgvuvj8UO8/Z7VnH4qYzab2blTPPBSUpwwctYcg9W3BN0EcRmTEWpPuN1MTg4kVgwD4L2173Prr7fS/c3u3PJLa+eGlOWXmWnjReWI0u1QV+L6YAOYpCRo21a8c5pHjbiClOknx0hiNpspKytrVTjaLXS6wMzk7PoPWRHkPXuK/4Ldu1um/TtLfb1wtIMDp19EOqQMh9AE1zsLRE5uhO0vQYl76bNKSHwq6vTDQ/NWTQa/BRdVezw7WHL6bdni9Poe8HA9v6IlsOcD+9luvkZYkpCE8ka2Rc4FMPA/Nh0SoxofzXKcfsXFwplgMNh4tsa0h7YzFP97DLh5a4se/wcT5in2t9xc4hNg0YFFrDj5I0eSvmL/fstcrqqyZP7JdvoVLYLd78qqvxwU1J6AMgUilpygfXsRgFhf33L+Su/cPn0cBFkcmQN5T7v2UJdJoMzd5nKQrwDPAYuB34HLgImN3+ciHIF/L1jAKy+/DEfnwo5XHO7hPBoY05xxv8L03VrNt0buvuceDhw6xJhHHuHV7t25OSOT+2K6U9jpEXbsPmRx+AEM/xQurHJYp1y1TD99CCT2VaS2caDM28xMISfX0CDWr4AIqvkqHFbdaPdc2c7b3e8JiWQN+bSdAX2egLAE2afYkvjsltKN6LCWa85PNnxCm5fa8OBfDzbtP6KiLPXEmjCEi3WrE/UFfRmvz1uzSdhV3USns6w9//UvWLNGOGp/aaYeabeeH4jyR7VaaQHZZEwUNX6zzoDM050/v3CBCJzf+5FTpw0fLhz41dXw/ffwwQfi3sbHQ1bHNEgdJexzQYS337Oa009lTCZT04KkRw8n1uOlW2H3O24baIOChmr4LhnWOM46ckR8PLQ3TUVnCmH3yV38sP0HAIZkDWl1rJS55LK056pr4eeOwWVc2fEfIUEjAymSb+FC97qUZMvkOP1MJhN79+7FZFLonhxfLXS1FYjw90nMZocLw6Qk4fgD52pMnYq0wA8NFRFhNsmeDhPmKhapFjDknA9n7oSMyY6PtYOSTj+3ZVoLF8GiszAd/1vZeas2oTH2s7wUols3sVkuLnZNftejTr/+L8CMQyIb3l9oOwNOWy7q+7nDjtdhzT/sH5M+XmQE2jCuNJf4dHRvpU13aqqQWmqFDEluV1D8feurJA2EjAlOy+XY4vROp3Nax9OY3mU6AEsOCh3I1JrRHDtmeTbv3i1e0VK0tSz6Pgtn7xfZJRowbwIsOM2rXep0MH68+HnBAsvnzZ1+NjGbhVzhjleg2vWsfkcEytyV5CDPCAvjQWA2YOvpPRSYXVfHg/ffT0HqrXD2AQi1L/fp0Xdkc/QhWg3OU8jIyOCRRx9lzdat7D56mDMv2krHTo9y/Pgp/08hUbKMM6pliJVuFXYOBQiUeavTWZH4NESIfY4D5SfZ93HvR7DjVTdGqSEHW04/CbPZzLy98zCZTWws3EhBRQG1DbVNyj9Wn61Vh6AyP2Dsa16dt3s+hFkxIgNaAaQyEtK8A5g/X8iVg4wg4EkLhU1HQz4NVa47bSPSRR3FcOcMNDodXN6okv3TT/Dbb+Lne+6B8A5ninuY0NO1Mfkp3n7Pak4/H0CSQevpzN962hiYcRjaX+mRMQUUIZHQ9lxIbu2Yc4V+bXozfvNOrs55im4p3eid1ptzu5/b4pjjxy0ODJedfu2vgu7/5xXjrs9QOB/2fSLLCXZao51l1SqLrKMrOCPvqTg7/yOKIte7qVHqixxbBt+nifvpAGeyTmzRXNpTC2Z2gfBkiOvsMJrZEb17i+/79rkuvSvVbnA70690Mxz5LfCeocYaUa+tZLNHuwkLswRXrFjh/PkeN2gGYoa0HI7MEVJtbhAdLSIvAf5yUMtdCoyxGWn7U1tYMM2t8QQ9ZpNiztOeaT2Ze/lcbh0s5N6WHBBOv0mdxYv255/Fcb/+Kr5LRjXZRLfTXrISHa+DTvYzRzzBuHHiFmzdKtaw1dWWQEPpHWwVnQ7G/SIyS4P1+ekkd997L0PGj2cYth1+EkOBcWFhfPDFLyIgRW8/e1d6R3rUUWSsE3VDq0+tTqjRnH79xPcNG075RX2ZUFVyICWuSqaf2Qx/DIdFZ3qxU/9AWruuX9/sw5FfQre77J4nO9Nv5FcwcaFLYwtaGqph/mmw4SHZp0jrk717T5FqRTj8Rn80mkmfTeKm2Tfx6cZPAeib0dd+uY9198FPOWB2P1st6IhuB6mjFVsDSk4/gLvuEiUEKist9iCHmX4azpH3FMyKhlmRsOhs58+P7wET/oDss5w+ddAg6NrVUrfxggssz2kNzxNgljD/Q9IxBpHpJxudDqIyhaFWwzHDPoTuLhQstUJODkTVtWe0+TG23baNTbdsIj7CEs25YAHcdhvs3w/h4TDU1aD+zjdBzwcVGbPfMPhtOLdIVo2UnBzhKDeZYK4bQT7OyHsqTo8HYOgHEBGA1YmjskWNJxmSbyNGiEfajh2WBZ6zyKrnByL7a8ODItJPoyW1xVDinl5uYqL70ruKyXt2uQ3OPwEJAZbVWV8h6rVt87y00DChZs3Klc6f6zEj2PZXRW0xf2TnmyJTzx1GzRJZjvaozIff+omsHhtMmiS+L1gg6ivYQnL6Wa3nZzaL7OCUYfbHo2GbE+vh60iRfaUwR8uPsufkHnTouHX6CEBIKM2dC8uXi/fuBRfIbKx8Dxz+DepKFR+n39L1dugp34CpFMnJltrWv/4qnH8mk5ijNo1jkpRQSDQk9ffKOAOFYwcOINe1e2V1NT/+73PhJJIp7+nRTL+qfPhrNGx/2YOd+D/SfGq1bi3ZDPMnirp+dlDH6WeErndB7qVe7NQ/kGSO8/MtgYRykH0fo3NEoKSGfAwRULYDagpln5KSIr5MJti5s+XvdDodw7NF9Nr769/nZM1JeqX14pxu59gPOsw+S9hfFFJXCCoyJoqsLIXW/D17woUXwr33irpvUxpLNM6ZI77btQfUFMGud0W2s4Y8EvtB7mUQ20XY6byITgdXXy0UY3r3hsukUow7XodNT3h1LMGI5vRTmf37ReRKTIyo0SALsxkO/yLq+ml4Hek+7d/f+ne7dsHLL4taKV27wmuvQZs2LnTi53r6LhPZRmity2RaY3LB3LnCge4sNTUWSQG59ymilTi8GyT0ho7XKteeLxHdTkjZtbvI4aGJidCrl/jZVYlP2U6/4ytg6wtObTqChoVniChMN5GMJy0ibJ1AMXlPEDUbdHpl563aRKTAoDeg860e72rwYFHL7cABS1a0XDyS6VdfBhsfgq1uFCBXk13vCBlrdwiNgXAH/6mhseL/yk4kc79+Yo5VVMBqO0o9UiCGVaefTgcjPofejzsctisE1Ly1RWQmZJwG0bmKNms0GTnzS5EBkh6TTo8OCQwcKJaXb7whjpk2DXLldpv/HSw6A0rc0G7WUIzTG8ux/PQTfPyx+NmutOeCqQ7rWSlJIM3d0rIyTo0LLACeAQYDXRq/PwOEAqUnj8PPHeDgt3bb9UqmX2g89H8JsrRsMHtIakv79olMkybiuon1VobttXFDg0XZwqtOP30I9HlSZBwrRKDM25gYYYcBIRcIwIFZsOxSsTaygWQTcJjpV3NMs8M5i04n5MGHOadUYU/i8+7hd5MRk0FWbBbvTX+P9TetJz4i3r5NIPdS6Oenewgb+Ou81engiiuEegGIYESDQQSB79/fsrxAK0q3wpqboWCel0YbAGSfJfZsZ2yGwW+61sbWF1yuZ9qrF3z4ITz1lLjPAOz/HPa879pYNGSjOf1UZtcu8RffvbsTmdKV+2HRdMj7p8fGFXDUV8DSi0XxejeRUtE3bWotK/n99+L7iBHw4ouQ5ap6zrxxsOQ8V4fo35Rug32fyTp0+HBRZ/HECftGS1tIWX4xMULuzBEGg4Fu3bphMNiX7JFFXQmY6t1vJ0CQJD6XLHHtfNlOv843w/RdEB9c2uGy6HQDdLvH7ToHgwaJ76tXOx+/YDRa7qXLmX5mM6y8Bg58DSg8b32FLre5XxtOBjExFrk4Z7P9PJLFEBoHZ2yFAcpnRXmFUbNg8mL32ijf7TiyNSwBzt5rNwtJr4cJE8TP9iQ+Hcp7eoiAnLfWiEyHcbNlBcg4Q4Opgb0nRcHGC3tcCMD06Zbfx8Y2i7SVQ/bZQo0hsa+Co/RzTm6Av8bDQftZQJ5gxAhLlqYUhGhT2tNYCw0V0OAdKflAm7vxcXE0F8d8BWgHLAXuAt5u/L4EuByoMZmh270O15leyfSLSIHu94iyIBo2SUqCzMzGspfNX6/hSWK9ZacOuOQoMhgc1BT3cQJt3kqBwV9/3Ri0VpIHB76EKuv1TBsaoKzRH+jQeTt/Evw5UrGxBg0uyEJKTr+NG1v/LjM2kwN3HSD/7nyuH3A9IY0lKrxWL9UH8Pq83fMBrL7ZI00nJloUZp55RgScgg17QGJfGD8XsqZb+aWGx9j3Gez9wOXTk5NFTfkmxv8Ok92o7+OnePs9qzn9VGbRImERdUraMzQeBr8DuZd7ZlCBSEgUHP0dync6PtYBubkiUsFohF9+sXxeUGDJUrr4YmFQcwmzCUJiwBDp7lD9k7wnYcVVQmrQAaGhMHmy+FmSAnAGyZgpV9rTZDJRXFysTPHVjY/ANwlQ7WT6jD9xbJlwvlTsd3ioJPG5a5flvjiDbKdfWALEdpIlIRt0dLwOerhfR7RvX4iKEpuuHTucO/fECWF0CQmRGTFtrBEGTRC1awoXQOUByP8eCoQnQ9F560uYzYrVAbOHtAFztq6fNCcVj3yPae+/jof47iKj3R1WXivkXRVAkvhctw4O2VAMtfueLFoqNv8lWxQZT3MCdt56ifCQcPJuyePxMY/z8OiHAVE/o21b8fvLLxeOP9nEdRVBM6Fxyg/WXzFEQvFqqDuhSvdXXGExbIMdp58hHE5bBsPlBdS5S6DN3bMvvZTPGjMpXgGeAxYDvwOXARMbv89FOP4qT5TyysJMSB1ut11VJCE1bCJl+212slxy8zpwXi13euBroc6hkLxdoM3bceOEokFdnchuN3d/AC6qhvhuVo+XHH56PcQ5es21uwRyr1ByuMFBZT7sfAtKt8s+ZfBgMa/y8iy1a5sTZghDd8rEs2kTMJvhr3GwZaZz4/ZhvD5vCxfC7v/azZh1h4svFmvTwkKhnAY2gg7DEqHNaRCT65FxBCQNVbD6JvixHez/yrU2xv0Kk5crN6awxKC8h95+z2pOP5XZ11gX2m7h9VMJTxL13lJHeGRMAYlOL2rFjVBmw3t2Y+3T33+3FBb+6SexlujfH9q3d6NxnV48UEd87vY4/ZJud8HYX4TjUwaS/veGDZZoS7lIcnVynX5ms5n8/HzMSsivJg2CjEkQoUYxQS9ReQD2fgwn1jo8NCHB8hx0JdtPttOvrkRsOrQsS48REmLJ9nPWUSTV3khOlmE8qSuBOQMsdcvW3wfzJops+LP2QV+xqVN03voKJ9bCt0mw622PdyXVpd2+Xf4ztrnclWKRthV74dBP/l1TzNTQWOvppOttdLweuj/g+Lgjc2HbS3YPadNGBDGZzXD77fD221Da7L/XaLTMSaub7hNrYPe7HnF6BOS8tcWut2GN8nK92XHZPDn+SdJjhDarTgePPw4PPWSRh5SF2ST+djVaEtsFzi8WWfIqoNPBzTeLjM3rr5chia0PcXCAMgTa3L3+hhtYUF/Pb8CDwGzAVp79UOA3o5EH77+fgoICG0cJvCLvuf1V+H2QWI9r2EXag2w5NYZl+RXwq21DjbQu8rrztuowHF8OOmUyBgJt3up0cNttEBYm1Jn+WhRjN+DTKedtzweh92OKjTVoKN8Jf98GBX/KPiUjwyL/+OWX8s6Rnq2tbAINlVC2LaCeh16ft/3/DReWeywALDdXSEDefLMIUuvTx4bcrrEueMshuYo+XGRqVh0UspquEN3OcYkJuZiMom6uO3tiP8Xb71nN6acyEyeaue026NLFiZOkrAYN5zCEKdbUkCFiEVJRIbTiS0vhz8b1y3lBqsqpGCnDIGua7Np+GRnQubN47y93MvBE2o/LdfopSsdrYOxPXg4L9TJZZ8G5hZAjb1KMHi2+L3Uhy7+5s8gu2/4NP+UIB4JGS6oL4M/RsMX9WgfDGwPcV6xwbk1ut2j3qdSXi0LU5kZjdLe7oesdkDpKLEgjXNUH9QOi20N8Dwh39AfvPikplmesXBllyXgSEuJkNpE9Dn4Li2fAyXUKNagCR38XtZ7yf3C9jQ5XCrk2R+z7VDjCG6rtHnbHHcLY2dAAv/0Gzz5r+V1xsXD8hYTYcN52vRPOKYDkIc5dg0ZLCuYL56kXglEyMiyZ9bIp2wGzYmC7n8rqegqdTnXVAJ1ORMZLwYitqDwISy+E407qM2s0kZGRwfMvvMCFBgNjse3wkxgKjDPAB0+db/MYo9GSWeRRCTpjtag9ZpBRwyDIkTL9du2yBPQCQiEkItXmYla1jM3u98AF5SL4QMMqGRkiqx3gs0/qqD60Bsqsqz7Jruen4TpJg2Dc79DuYqdOu+gi8a5bswZ277Z/rNlsR94zNEbYJYa861T/Gs2ITIcQz75PIiLgjDPgrbfEnsTqevXv22BWlAgA1pCH3iD+/qflwaDXXWujvkw46hzsLWVRUwi/9YZNnqkLr2FBc/qpzA03mJk61YnNd20xfBOvTQ5XqCmCfZ/bXOw5g14PZ50lfv7kE7j2WqithQ4dRESKW+z/EjY/q8zD1J8p3S7bWyA5i5zNEFPV6RcMhMZAhPxCUMOHi2fhnj2WLEw5NF/gO3T6pY6ErndDWBAI/TtLWKIw7ta7n0k1aJCQ3z16FA4elH+eU06/6LZCz7/3E+LfMe1h4Ktey2ZQlfAkIdfW3jsy385KfDbPYFAsriHnfCFt7s8OpoTe4vkT74ymu4v0fBimrAF9qN3D2rSBmTPFV0gIbNsGextjIoqKxPe0NBv3UacXBgCZQToaNhj8tjDeOrhXqmGqh/TxIthAoyUNVbDtZdj3P7VHYp1jS0XARLX9rDMN+9x9772kJydzlczjr6wz8uOvG2z+vqRErF31elGb3GP0fAhmHBC1/TTskpYm1p4mk1A2aGLQ6zBxvs3FTPMMMa+j0wV28KgCnHVWo6x1bTGRi4fAjv9YPc4p5+3qW2DdfYqNMWgIi4fMKU4HZWZlWbL9vnKgSlhWJoLYQJNO9ghmE5zcBCfWqzuO+F6QMVmUvdKQT3gyJPQSNhNX2PWOcNSVbHJ/LIZw6PU4tHFGekTDFTSnn79RVyIecDEd1R6J/1G6FVZcAYd/cXysDCZNguhooTddVyeyIW65RYG19/7/weanQK9cZqLfsfVF+LW7qJUig5GNtbS3bLEYm+UgOf3aOFFiKVaJtJVFZ4m6TMFAdaGoryYjQzk+XtSDA+ey/Zov8B1GTGeeDgNfDuwsMFcxhMN5RdDvObebiogQtTTAOYlPWU4/sxmK/xY/yzR4KDJvgxgpc3PDBqiWEY8iGU8UzWCI6SCkzT0cYepRotuJ50/KMNfbWHE1LJURJZ3QE5IHyXaC9+5tuc+//y6+S/X8rEp7ApxYB+V7ZLXvCkEzbyNSRO1pXyWxD4yfA21nqD0S30MfCpufht3vqD0S6+ReCjPyxdrHiwTi3DWEhCA3RjAdKG2wbYyU9iperwOnYZdevcR3Z+r6qSbvue9/cMxJDX0HBOS8NcB110FpbRqfbXqO41HWM3CdyvQr+AuKFio0wiDDbIaqI05LM0rZfqtWwVo7VUOkZ2t8vAhka0HlATgwS/QfQHh13prNMHcQ5D3hvT6t0e1OGPuz9gJ1lqpDYg64WiojdTT0esypoH6bhCdDnyeFwpuGR9GcfipjMDipwx7bEcbNhg5yYw01mkgaACO/hnYXKtJcZCQ8/TTceiu8+abQn+5mvTa0c4z4rDE6XxmNfr8kYxJ0ulF29E5aGnTtKtYhy5bJ66K0FI40rvmysuSdYzAY6Nixo/PztjmmBlH/SIFsKr9g97uw5Dw4uUHW4aNGie/OOP2ken5WF/gaqiE5EJYtk7+3k2Ra7dYmOvIrzB0MO9+U1aYi89YXObEWll8pq2amu2RnQ2amcK7b22w3DU3pWkUNVWCscXxcMFB9WNRjkENDtahhIpOpU8X3hQuFvNm6RiVVm9nwS84Xz3cPELDz1hoNVWIeVx1WeyQazqIPFVlA4/9QeyS2icryajZuoM7d+Lg45OZLFgLxdlL4PBIYY43d/4XDv3m4k8DBqtOvYr9Q4JGCzU5BFXlPkxFWXgXbXlCsyUCdtwADB0L/AQZmbXmQ/3w9jh07LHWnJZy6j2ftgqnW/x40HPD37fBjFtQec+q0rCw480zx8yuviPtVWgrvvQdz51qOk2wCVpV/ihbDsotkB5X7A16ft3qDqOunUi1jDTfZMlPMgUVnunZ+6nDo85TrmYIagAs+IDfRnH4qYzKZ1B5C8BAaJxx+UdmKNdm5M5x+OuTkKBhoEpYooqqDmaQBQm89Xr4X1dl6cCtXCkdEx44yJCEbMZlMFBQUuDdv9SEwealwQAcDbc+Foe/LXhwMHy7kjvbuhcMy7Z92F/insvu/sHC6kErWaE3pVtj2ElQ7oa9qg2HDICwM9u8X800OsjL9wlOg+/2iZqQMFJm3vkjdSdj/mci28jA6nUXiU869tFlPw1UOfAnfxEHBPIUaVJGNj8K8ia6fP+FPOE1GAdvjK0W9i13yM5B69xaZ79XVIqhp8WJx78eOtXFCzweh212y23eGgJ231ji5Hn4fBAcc6FapxZpbYZdWA8cmSf0hJFLtUbSmuhAOfC2+e5FAnbtnX3opn0XIq+H4aUQ4M07vZbNOp+KBMdYwm2HtXbDTxdo9QYjk9Nuxo1ldv+rDsOlRKFpk9Rx1avqZYewv0P3/FGsxUOetxHXXif3l+vVw332i1t9PP1l+r6pMazCRPg463SyCoJ3k6qshN1c4+6TA+59/hjfeEBmAYPlu1SaQNhZGfuXfZQJOQZV52/UOyHLRaaQUa24TNh0N52jbmOmcbasQtBc58LXYD5dsUXskXsfb71nN6acyZmdS2+vLYcE0yP/RY+MJCurLhB61L1JfAaXbZEkhBgVmMxjrZB0qSXxu3WpxAtlDygiUzpM3HDMFBQXOzVtbBEPdMRC64R2vky0DEBdnkfiUW6PRKadf2Q4o+MOmISboObYc1t9nM6LZGWJj4ZxzxM8ff2yRYLXbvRynX8ow6P+CqOknA0XnrS+ROhrOO+61aEspc/Pvvx3fS8WdfhFtIOM079TC8zRVh6Bij+czF6PbQbtLIbaL7FN0OpgyRfy8qbFcw7XXWgyhreh0I3S42q1h2iJg5601YrtA32chbYzaI2lNQzXsfg8KF6g9Et+mYi/k/6D2KFpSuACWXQwFf3q120Cdu9ffcAML6utZ5eC4VcDC+jqu6/gN1FjPZpFkzzMzFR1iaybME88WDVm0aWNRNWjagyT2E1ldHa+3eo4qTj99CGROFXXKFSJQ561E27bwxg0P8vG5nUhJMWMywQcfCNn6I0cgL08cJ2svWbTYKyobAUnO+TDkbYhy/uEXFgb33y++79olyntENyr+v/oq/O9/Fnn6060pWkfnQLuLXOrbVwn0eWsVUz3seiswAkG9TcYEuNQM3V2sSVpfDvMmiYxBd6kpbAxc9lG7vAfx9nzVnH7+ROkWKJwn9Kg1XCPvKfgm3qM1aNzi+HL4tQfs+UDtkahP9VHxf7HpEVmHp6RA9+7iZ0fZfuXlsHGj+NkZp5/bGGthxVXBuUhxwsAtZZXMny9PFtIpp9+Al+DiWoiUW5klyMg6AyYtgnRbqT3Ocd55Qnb1yJGW8ivWqKmxyO3YlffUEBjChR6+l+jaVURAV1ZajCO2UFy6LGsajPsFIp0owOqrDPsIzt4PBnkZI6049DMc+d3xcZFtYOT/IHu6U81PmmSRSR4/Hs72gWDQgCciFXo+DMmD1R5Ja0Ii4YIyGPia2iPxbVZcKWo1+5LhLW0MDP8U0serPZKAICMjg+dfeIHp4eE2HX+rgDNCwnn+n3eSceY3QmXmFLZtE9LJBoNFrs4j6HRCiitpgAc7CSx0Opg8Wfz8h6TYGxINSQMhzLpcq2rynr70rPET2mYZSU6P5qP3qpk8WfwXvvgiPPSQqOnXti0MGiSjoaUXwNq7PT1cDSu0bQt33in2iRdeKIJKO3eGigr4qlEs4YorYLAPLqcChj0fws8dhTqQGuhC4PwSGPyWOv0HM4YoOLlOmXIEXe+AC05CQm/329Kwi+b08ydShsH5J6HjtWqPxH9J7C8i3/HRhXJUW+jxEKSOUHsk6hORITbLIfKLE8uV+Fy5EkwmaN/eC1G2zSleBfs+E1JewcSmx2FWrM2I51MZORIiIuDoUZG5aQ+zGdasET+3CQB/gOpEthGGQiuGKpeai4RLLxU/f/GFcBjZoqCxWE5UlCVysxX1ZfBLd9iuGaABEXG3739Qsc/jXTWX+PzXv+D554U00qZNrWujeEW6zF9xVwt87V2wUV4wjCvEx8Mtt8D06fCPf9gZbtFi+K2fVi8qGAiJhMh0tUfh2/R4EIa+51tKIlGZ0P4KUdNPQxHuvvdeHpo5kzEGA1MiIvgf8BfwOTApPIJROgMZXWYy8exXREZLaEyrNj7/XHyfNMlOvVQlMNbJVkvRsDBxonDIbt8OB6XyuXWlVvcwNTUWGVCvrne2vywkz2XWS9dopP+/YNpGCIni5ptFiY/ycrFmzcmB554T+0+H9HsRut/r8eEGLKuuh/UPuHz6mDHw0UfCuRcRAQ88YNk3jhsHF1xg48Rll8EPmcJpruE6+nAIiRH1qNVApxNBGF4MfA0YynbBFzr4Kde18/UGOP8EDH5T0WFpeBbN6acyOmeNPyFRECrfCaJxCtnTReR7nHy5K68S3x36zRRSIsGOTgdTVkHvx2WfMnKkOG37dotMoDVckfYUQ9KRlJTk/LyVSBsDMw5ChyBz3Md2gcxpwmEjg4gIiwP3TweqVMuWwZ494pyJcspkleSJLBkXagkEDSajojWATjtNFGAvK4O337YEJ1dVic9AbLpffln83N5e+cfqQmFUNcmXQHZ73voyxatgxeUimMALnHaa2FiXl4u59/778MgjcOWVFkmdVatETU5QKKjiyO+w8IzAMW7Vl8H+L0TNPVcY8o4wXMlh17uw8EynHRGnnQY33igklGxSXwEN5XYOcI+AnrfWWH0T/OFN6QGZFP8t5p4vObN8kawzhZNHb1B7JAKzCRrsRNl4kECfu3ffcw8HDh1izCOP8Gr37tyamclr3bsz/tFHePKZQ7TvcA8vvWQJhjl2TGSm3H8/fPihCJQJCYGLLvLwQA//DF+Hw4FZHu4osEhMhCGNJb+asv1+6QZLzqGmpmUJifnzxff4eJnOIqWIyoKU4SJAViECfd6eSliYyPBLTxdKFjNnivsoiw5X+UZNLH+laLFQt1KI9HThsL3xRrj9djvBajHtIb6377ynFUCVedv+MuE8T5aTFusB6k6KdalMu5JGM6LbQeoo6Pu82iMRQaNH5qg9ClXw9ntWZw4qAWDfoaysjPj4eEpLS4mLk5FRUXsCDnwlNpXROZ4foIaGn/LQQ7B5s6hDJNUTa2iAt96CBQtEAej9+8Vnb78N2dlqjlbDFtu2CQNJeDh89pnIGDsVoxFuuw0OHxbZZJdcIqPhldfA3o+FXJkWQGGducOg+jDMyFesyW3b4MEHRYbtXXeJ2o2vvCLkWPr1E86/PXuEfOTzzwsnoYYMTA2w9yNhfJBZN9NdGhpELY1Nm2D3buHgKyoSv7v8cvjhB5HROX262IC7ze734O/bYOp6SOipQIMqU1sM36VA7uUwwsPO2rV3we53YfqegKphEpCsvEYEpUxZ4342qJL8NRZOrIcLSkCnxYo6xGT0DYNi2U74tTv0eQZ6PqT2aIKGmhqxxomuWcVTE6cTMfhJHvrwFrZta3nctGkio9qjFC2BnW9Cj/s1iU8n+ftvePJJUZv6k08gdOsjHC1O5IGP76OsDP7v/6BPH7jpJuHcvekmD0u1aihD2Q4RSJZ9lnAAIfYlOp1vvXYDHmMdGOxFlWlo2OHgt0Jid8SXkHux2qMJPo6vEmXHOlzj3oPzt74iW/SsXcqNzU9w2hfkJiEe70HDLiaTzMjZwvnC6KUPhU43eHZQgc6W56FsGwz/RO2RtGbhGRCdq6VMS9SegLwnIXmIiCqSwejRwum3dKlw+lVVieivDRvE73fvFt9zcpx3+JlMJg4dOkR2djZ6vZPGryO/Q30pZM8Qtbg07NKtm3D8HD4s7qVUY+PAAaHZHx4uvg4fFg6kGTNkNtzhGkgZ6Xo9rWCg7blQXSBS8hTaBXfvLhxCn34Kb7whHEcS6xvVbmNj4ZlnlHf4uTVvfR19iNfXBCEh4n5KNVTNZnjvPZg92yJb1rWrCLxQhE43iL/JsADRCg1PhlHfeiejv++zMOBlv3TWBPS8tcawj9QegXW63y+CQPzwb8ir1FeIOtQZk2GYD9TlNjdA9rkQ38vrXQfd3G1GRIQIWHvpn4nsK+7Ghu+S2LZNyJaff77IhK+t9UKWH0DaaPGl4TQDBoiaYcePiyzNDh2eZfFii1LFv/8t9inl5SKY9PTTVR2uIgTFvD2xFtbdBdFtm5x+Ll3qnP6QNBiG/lfR4QUNmsNPMVSZt2Yz7HobwhIg91Lv9NmcuO7Q52lI6u/9vjXEvd/3CeRc4F4A/cDXwFit3Lj8CNk+IIXQnH4qIzvRMmMyjPlROD803KN4JRQu9J1oXAmzGaoOac6I5hgiYOfr0O4S2U6/ESPgnXdg50749lv46y/hGIqIgDvuED6M/fvFcc5iNps5ceIEWa54JbbMhBN/w7nTgtPpd/g32P4SDP9MVsaJTifqnXzyCXz3HaSlQXU1vPSSpX6GxIUXWs8EtEraGPGlYZse93uk2fPPF9lhkgN++nRhKFm6VGSOXX45tGvnoJG9n4LOIPt5AG7OW3+hrgSq8lUphq3TwQ03CEfunDnCCf/gg8I5qBiBVrch5zzXz/25M8R1hXG/OD42xFZxTAU49JOQ2Gl/lUdC5INi3voDWWeoPQL/IDQGYjuL2ty+QHwPGP2NKl0H+9zt0AEmnduFhz5c3PTZP/4hghJt1prS8Cn0eiFb/sorkJ8vvkCUEairgyVLRIApiCw/g7fNCWtug6RB0PEaxZoMinmbPhEmLXI/GEIfJoLuNFyj6hAUrxH7cW+t781m2PAAJA8WzooAQZV5q9NB3hOiXJIaTr+EnoGh/OKvdPkHtLtIPAfdIX2cIsPxR7wttqm9rfyFsHhNO1wphn8uDGG+puOg0wl9bE1x10JIFJy9H6Lkp+QlJEDv3sK58Mknls/++U/o1En8e9QopQcqg1HfwMl1wSspWV8i9PtL8mTLzE2YIBy3hw/Do49aPu/TB7p0gbw84WAIhAjbYECng/vug6+/hr59YehQ8bksWVaJLc+Khpxw+gU8ZpPIMAlPFe8QFdDphFTZwIEi6j0lRaGGt70snpkdrwu8TKP6cnHvwuQWkWkksa+oySAHs0nUTtHplQ922P4qlGyEDlcr226wUpkvHKnpY1Vx3lvF14LjfJ2J89QegYaPMGOGUDFYvx6mTLHUqfYqW1+A6qMw8FUVOvd/xo8Xa5qNG0G/+w26JfxB8tnf02AKoa5OZG2OGwe9vJ1M21AJu94SEuEKOv2Cgsh08eUuU1a530Ywc3g2rLkVxs+FNqd5p8+GStj2L2h/ZUA5/VRj7GyvlZXQ8DGUqOWooJqUhmM0p58/0FAlovgj22iTQwlCY9QegX20e9wSF2pYnnGGcPq1aSPqZkyaBDFq3/bIdIgMYu9U23Mh+xwIkZuSB0lJIrPv559h/nyR4Td9upANdDmLaM1tcPR3mL5bm2u2qNgL6x+AnPNFJJeCxMe7WedtzI/ifahhQacXEnwh0cLJo5JzTKezOHIVwWQUG/TIjMCTNT++Cv4YDv1egB7/59y5o7914mAdLD5HOAonLXSuH0cMeh1qjynbZjBTsRfW3g4DXvUdp9/6e+HQj3DaKmUMpRrewWyCRWeLYNFO16s9mqBEp4PHLniZI6PDyZ5wmzqDyP9RKABoTj+XiYtrdNiGbYJ9v0NDOSFhiTz0kFCU6dJFhUGFRMP5JWCqVaHzAMBkBHO9pqykJumTYNgn3pWflgLJdVogkyKkKLnhc5IND8LxFTBhnpZxqxZmM5jqXFcvq9gLv/WGXo9Cz4eVHZtGK7RZojI6OUbngnmw+CwY+qEW0aUEDVXiRRGRDgner3Vhk+MroXQbtJ0ROLWLlKCuVGTIxfeCiFRZp4wYAV98IRx9Svp1dDodGRkZ8uZtc0q3Qmg8RAWwZIojXNxcZWWJDKKrrxa1Ndq6q5wVEg2hCZrDzx46A+R/B/Hd1R5Ja1wYk8vz1p/odpfaI1AevQHO3C5kgAKN+B7Q9jwh0+lJdDoY/JYIGlMaD6+fgmLeNiexH0ycD/E+JFkUkQERbbRobrmUbhO1TtpdAqnD1RtH9VEomAsxHVTpPujmrg1C971Fu5BoMKjk9Ju8RNQS13CfwW/BkHeb9g4Gg6WusSo4qxAgg6CYtzXH4Ps06HyLuKeusv01IW2YGcTBvO4Q11l8eROdXr5Khh+h2rw1m0XgX2ic9x3otceh8oDm8FOLsl3w+wDoehf0fdr1dtLGQpTzyR2BgLfnq87sbUFRDQDKysqIj4+ntLSUuLg4+wefWAu7/wtd7tD0i5Wg8gD8lCv+Pwe9pvZoLKy9G3a8CmftbSourQHs/wKWXwYjv1I868hrzBkgFigzDqo9EnWpLoADX0PGRN9yuGu0xGwWEcS+FgVbdxLQe8TYETBochmBz6bHRe2w9leoNwazCYy1TmVua2gEPMdWwJ8jGjN4PVMbVzZmExhrRHaDhjqU7YSQGNmS9hoasqjMFyUTYrsEZ414dzDVC5tCxmTXFSTMJvjSADkXwqivlR2fhueoLxP7yIgMbd4owY7/wNo7YeJCIUuvETw0VMOiM0U9x47XqT0av8QpX5ACBFiBFP/DaDQ6PihpoIgu0xx+yhDVFga+Dh2uVHskLel2N4z+IWgjHmySOhIGvAKJA9QeCUajkT179sibt83pfDN0u9czg/InKvbCurvg4Cy1R6JhD53O9xx+ADvfgm8TRCCME7g8b/2NhdNh/kS1R6EMtcWw4w2oOa72SHyPrc/DwW+cO8dsFpJWSlF5EGZFCYkdDxE087Y5ZlNjcIOGX5I0QMiHdXdSstcT6PSqOfyCcu5aI66Leg4/Yy0ULRFZnxruU10AR/+AmiK1RwJ73oPf+kD5LkWbDYp5qw+FUbPclIzXwaQl0PsJpUYVnPzaCxad5b3+Dv8igv4P/+K9Pr2AavM2aRB0ugnCk7zbr4b6hESKGtaaw89lvD1fNaefRvCh00PXfwhnqi8RnSOkPfWa1ngLotsJ6Tpvy0DYoLy83PmTOt0I3e5UfjD+RvIQGPOzqD+mFge/gx2vq9e/v1C2A/Z/JYzQvkJCb8i9QmQ5OYlL89bfCI2FkFjh4PF3Dn4j6psd+U3tkXiOY8th3kQoXODcedO2OCdLVbQYvk2EfZ841489dHrIvczjwThBMW+bM3eYUAbwBY7Mhb9vh4r9ao/EfzCEizWr2tnWJzdC4SIw1qk2hKCbu9ZoqBLBbqYG7/ddeQD+GiOyMTTcp+BPWDBFlAdRm4xJ0OtxiM5VvGlt3spAp4O0Ub5ZAsGfiO3k3UD32C7Q9W4hrx9gqDJvU0fAkHfUqUG9+z0oWur9fjWU48hc2PCQCKjR8Dia08/XqSuB3wfDng/UHkngYTb7jnHU1KBFdzvCV+6VhuvoQyB7OoTGqDeG3e/CxkfU699f2PE6LL/Et6LEs8+CEZ+K+gEarRn5BYz9SX2DsxK0vwKGfw5tz1F7JB5EJ2r5Vh5w7rS4zhCVLf/4yCxI7CvqyipFdA6M+BzaXahcmxqQNR2yvBj5bo/CebDzDcCHAj/8gZrjoha7Go4eiR2vwrxxYKxWbwwakPdP+LkjVKkg7R+WKFRSss70ft+BSMrwRtWlPmqPBNLGQJ8n1d1L+TNbnoO197h+vtksnu++FBTpj4z5EQa/4b3+kgfBwJc1Z62/01AFq28Upa801OPIHFh+JVQXunZ+4XyhXNNQqey4NKyiOf18ncqDUJWvOYSUZtvL8E28iMD0BUo3w7dJsPUFtUfim2x4GL5LFvVJ/I29H8PvQ6AkT+2R+AZmkzByN1Sp0//A12DCX+r07U+0v1LU0dQcbBpqEBIN7S8T2YuBSvIQuKAMOlwt/xyzGaoOi4AwucR2hEmLIOc8Z0eo4W16P+Y7tab7vSBqTEdrNaadYtsLMH+SuvuLjtfDoDe0+rdqkz5BSL0aVKh9GpEqVFJSR3q/70AktpNQbYnRnod+z9HfYf+nrp/fUAFfhcKq65Ubk4aGP7L6Jlh3n3f71IfC+D/E+01DPcp2wP7PXJeZ7vUYnLlTBJFqeBzN6acyOkcR+Yl94NwC6HqXV8YTNERmiIgfX3Ei6cOh/VW+JznqK4QniejK2mJVh6HT6Wjbtq3jeduc+nLhuA8JYOO1M+x+V2jqH1NJliG+O6QMUadvfyJlCLS7yHecLrXFQgpx32dOn+rSvPVHTPUignn3+2qPxD2Or/StDFNPoTc4L+dtqocfs2HNbZ4Zk1z2fyk2+zXHPNZF0MxbX0WnEwZu7f/fObLPEcFFYQnqjSF1JHRR7xmhzd1GMk+H/i9CZBu1R6IRKJhNop7fxkcVbzpo5u2Yn2DGYdfP1xmg3cUicEvDdY6vhPUPQGW+d/rLexIWzwi4DE1V523REu/bc/Sh0GayqKGsoR4dr4ULSoXUsSuExgjlGn2osuPyE7w9XzWnn8ro9TJvgT7EswMJNnIvhYnzIaGn2iMRxHeH4R8LnX6N1nS/DyYthKgsVYeh1+tJTk6WP28But4O5x6FmFyPjcuvSB4GXe6AiAx1+jfWqlrnxu/wFVndqsNQvBqqjzh9qkvz1h/RhcD2l/1bDtxshhVXwe8DA25jbpUTa2HX2/KlAHU66HonZE51rp/8H2HlNeL5pwRFixqldTy3aQmaeducE2th2aXq1yqpK4GC+UKqUsM5UkdA1zsgIk3tkahGUM5dX2PLTJgzMDgCaLxBdSHM7gqbnlB3HPXlYr3gAbWUoJm3YQmi/qqrhETByC+h882KDSkoObkBtr0IZdu801/pFuGg0gXW37eq8/b0DTBlpXf7NJt8xzYRzITGuacGVbEvqOv5eXu+BtZTzw8xGo32D9j1jth4a2hoqI7RaGT79u2O562GbZL6C/myRJXqYswbDz/nqtO3P2Gsgx9zYMUVao9EkNhHSCF2c74OR9DMW51OyDhO9Gf5WjP0nQl9ngm4jblV9n0Ga26FaplR5/pQGPiqqHnoDMWrhNR0xT5nR2idwW/BuccgPFmZ9qwQNPO2OXWlcOBLKNuq7jiK18D8iXBwlrrj0HCeygPwXQpse0m1IQTl3LVGxT5YOB32fe79vk0NUHtc2VquwUxIJOjD3HMWKUFYPJy5VdQmU5igmbfVhSLLTK0yExqCnAtg+i5IG+ed/kbNEuvWAEPVeWsI836f+7+EryPg8K/e71ujJaXb4NgK185deAbMm6DsePwIb89XLX3MlzHWwt+3Q+Y0yAjeSeExtv5LSDR0d6OYs1KsuBqic6HPEyoPxIfZ/xUU/AFDP1BVbqqmxglJWGMNbH9F1PVIGeq5QWnIp81UYQjRsI8hDOJ7iOeSr6DTgc41GQin5q0/E99D7RG4h04fXLXnOl4HmWdAeKpn++l2r6gtFZ6kTHs6PUSkKNOWHYJm3kqkjYYLq4SBWU3iusDA/0D6eHXH4a8svVDUZPd2BDxAQzXE9fCoQ14OQTd3bVHwByQP9n6/vR8XXxrKEBoHZwR+bfagmLe7/wt5j8Pp6yGxn/Pn15eJOmYZE0UZBA3XCE/2/nsqQKVrVZu31QUiqDB5iPdkrCPSIOM01dW/NBByuTo9nOlCtm7H64IjuNdH0Jx+vowuBE5brk0IT7H3IzA3qO/0M5vg6BxI0rTh7XJsiciK6P1PiG6n9mjkUb4LNj4MPR/VnH7N2fQ4HF8FE+Z6v2/NCCKf8b+rPQILBX+JyPU2UwJ206YIJiOUbRfR6LGd1B6NczRUQ01hcEkhJ/QWX3KpL4fll4taUc5ISyntoDu2DCIzRc03DeXQh/pGfYvodkKaXMM1QmJFhpXZ7P33VXw3mLzYu31qWCc6Fy6q0dYsGspxbDmUbhZZUmGJao/GP2kzWQTWRKS7dn59Bex5DwyRmtPPHcxmseY31XrHrnPwG4jpJBSHNJShaBEsuxhGfuW9udBmsvjSUJ+ej7hegqz7vcqORcMumtPPl9Eb1IkODBbG/gzhno9Ud4hOD+cUgFGTmbBLr8eE7FuYH0nVxHaG01ZChIezOPyNygPCMWGsAUOE2qPR8Ac2/VM40c8rUnskvk31YfitF3S+RUgw+hMHv4GVV8Po76DtOWqPxnuYzcLoIedZaKyBI79BVLbzfZTtAHO9c05GW23NGw8ZU2DcbPfa0mjNyQ3CuZs2Wu2RaLjKMD+uq6qhHGo5+8xm2PYvYUPQsnWVY++n4h3a8Tr1xnDwW9jxish00Zx+rpEyTHy5SmQGnFcs5F413OPnDpA62vNBwA3VIgM/9zIYoYLccqCSMgKGfezefNLwXzpcqfYINGSiM5u1SphqUFZWRnx8PCUlJcTH23BiVB8VTilfiPrV0NDAbDZTXl5ObGwsOi1y13XMJvUymDc9LjLG+s1Up39/4vhq2P8/6HoHxHZUdyxFS6D2GLQ91+lTg2rems2Q9wSkjRHSQ/7E8ZWw/VUY8nbwGLPqy+H7dGh3MQz70HP9mM3wTRwkD3W/5qOpAXa/C5FZ0HaGIsOzRlDN2+b81kfUGjprt3pj+GOkyIIY8716Y9Bwjb2fiP1j9/8TwaMqELRz1xpFS4UcYNY07/VZcwy+T4NON8KQd73Xb6DzW9/GZ/Mu9cZQeRBKt4r1ncL2IW3eanidTf+EqLbQ6XrP9mOshUM/CYWKtFGe7cvLBN283fmmeA72e17LovdXqgtg2SWiPn3Ha9UejSqUlpaSkJBAaWkpcXFxHu9Py/RTGbsP56UXQPkeOOeI9lDzBGYzlG0Dfbi6Bu3iNVB7AtLHallPjqgugMOzG2VNErzevU6nc+7BXF0AYUnqFDr2ZdSULM7/Xsj6ak4/x5TvhJ3/EQ4ktZ1+bmS9OD1v/RmdDvo8qfYoXCNlGIz6Su1ReJfQWGhzmvvZd47Q6aDP08rUwNCHQJfb3G/HAUE1b5vT8xGxPlWTsAT/UlXwNeorYOfrQu0h53zv9r33Izi5EXo+6N1+mxG0c9ca6+4WEnZZB73XZ2g8nLZK1KHTUI5hH6sfiB2dI748QNDM29JtsOJKIZHuStamsVao1USki6w/Ddfx1n7FEA7tLvROX17GJ+atN6XMD34jJI77v+Cd/jRsc3Kj8Fd0vQu63Cr/vLoSKNkE1Uc8NTKfx9sOeq1YnBu8+eab5ObmEhERwdChQ1m9erXTbRiNRtu/zDwDOlyjOfw8RflO+LUn7HhN3XHsfAMWThWyXRr2OfwzrL4RjqpQCw4xX/Py8uzP2+YsPltIV2i0xFgLB2ZB4QLv9z1lNUxZ4/1+/ZGs6XD2fsieoe44TEa3jOBOz9tAQW3HgTPUHFN7BOox5kfodre8Yxuq4egfUOZCpkG3u0TAjJ8QtPO23UWQe7G6Yxj3Kwz7SN0x+DP6UNj0mMiU9zajvlG9pl/Qzl1r9H4SBr3h3T4NYZAyRNR31FCOpP6Q0EvdMdSVeGxtFzTzVhcCNUUia9MVKg/CnH6w4z+KDktDwxVUn7cLpsKcvt7rb+wvMHW99/rTsI2kSOhsXb/4bnB+MfR61DPj8gO8PV81p5+LfP3119xzzz3885//ZN26dfTt25cpU6ZQVKRgvaGeD2nZKJ4ktouIqFbbCNb1Dhj6gSqZa35H1tkw4ktoM0W1ITj1kG57PuRe4rnB+DPLLxESDd4mJEpk12g4JixeFFdXSSKsicM/wawYOPSzy00EvBGjOcVr4OeOsNeDcpFKcmwZ/JAJqzws7xMI1BTAgimw5z31xpD/g5CgLFri8a6Cat5qBA6GcBFcNPwT7/cdker5zGEZaHO3kaxpkH2Wd/usrxDBdRrKYjZBXan4rgbGWvg2UWSpeaqLYJi3cZ1hxgHoertr54cnCeWEjEnKjisYOTIH/hwFJ9Z6tp+D38H3GaK/AETVeRvTCeK8GGASGgPRbb3Xn4ZtorLgjC1CSlzDp9HkPV3k5Zdf5oYbbuCaa64B4J133uHXX3/lww8/5MEHW0uq1NbWUltrWYCXlZUB4iEtPah1Oh16vR6TyUTzUovS56c+0G19rtfr0el0Vj8HMJlMsj43GAyYzWarn586Rluf+/w19RKyAobGY1W5pvh+EN8PGn+v3Sc71xSZjinnQjHGZn1465qMRiNmsxmj0SjvmrreIz40GoPrPjm6JkJgxDcQ2wmdyeTdayrJA30o+oQeyl5TIN4nsxmqDkNdMbrEvupdkyEGUkZgCm/TNO+duSapr1OPDaj71HyMkW0w68Mxm3WYT1lf+OQ1xXRDl30O5s53WH0XB+x9kq7p8K/oDs7C3Pd5dFFt7F9TSAK6QW9jju8Nje8h2ddU8Bf69XdDvxcxtTnd5WvSNVSjb6jGpAtt+vtqdU0K3SfpfWvzmux87rfP8qNz0K+9HQa+hqnNGd6/pqML0B1firn91RCZ6X/zyVeeEYn9xOcO1qqKXpOpBmNFPkTlNMnKq3Gfgu6d62vXtGUmuq3PYZy2VQS4BsI1+cB90q29E/3uN+GcIxjD0rx/TfWV0P4aSB6K2WjU3rkyr0nO2J26ppAEDL0eFWM8pX2/vSYbY/f4NTVUoyvbgbHysLCFeeqa9OHoYzpiMsQ12WQC5T5J81Zqw+vXNOj1Vmsdj/3tmU3oSjehj87BFJbsV/fJ3uf++rfn8jVV7MZ4Mg9SR0N4cmBck5P3ydtoTj8XqKurY+3atTz00ENNn+n1eiZNmsSKFSusnvPcc8/x5JOtdau3bdtGbKzIOklKSiInJ4dDhw5h2PUaMVXrONjmcVKzu5ORkcH+/fspLy9vOrdt27YkJyeza9cuamos0pAdOnQgLi6OrVu3tvgj69q1K2FhYeTl5bUYQ+/evamrq2PHjh1NnxkMBnr37k15eTl79+5t+jwiIoJu3bpx8uRJ8vPzmz6PjY2lY8eOFBUVUVBQ0PR582s6ceJE0+cZGRm+dU0dszhZVuf9a4qJZOuWLRibPX+0++T4mo7s24RJH41ZH+bVa2poaODEiRNs2bKFbt26affJrWvqACdNJFUc8uo19dg1jQZDAuFn52n3ScY1dd1zIXpTFYVDl6t3TYmj2ZGcBoeBw3lOX1N0dDQAx44da5GNH0j36dRryu/zu7imxr59/ppiHyb2ZDQdkwjo+WTtmgy7FpFV+Dm7dBOJ7TDV/jXtL6SmZhhUAUfznLqmqKoCOpuN1NWUs63Z585f01A6nrWLooICCpq1o/R92rZtW9P7VqfTqX6fvPW3F1ldSpfQeOpqq928T65dU9n2r0g8+l+2VfWiNry9380nn3lGlJ7g8I5FNIQkYzTEe+eaGvIwzBvPofT/43jypardp2B859q6puLV/yLp4AvsbfsaVVF9vHJN1eGdqI2bQv7ek5j0eYExn3zgGZFQnUVm28sJ0xnUuaaSOgoi7xDv/7w87Z0r45ps3af8pf/CrAunNG58wFyTX96n9hOJO+8YW/PyMBZbxq/4NR3PhrQ34SjEVuwPqPtkNpubxhUo1wTW//YMxjJ67xgLHa/jUJvHA+KawL/vU3rD37QJPUBR2q0UFBbKu6aTP2NY/3/szP2Eqqg+PndN3rhP3kZnPtWtqeGQI0eOkJWVxfLlyxk+fHjT5/fffz+LFi1i1apVrc6xlunXtm1biouLiY+PB1p6j1lzK7oD/8N0zjF0hrDA9PL7wjWZTejnDkIXkYJp/J/ev6b87zGvuALT8C9E/SwlrsnO2L1yTZ7+29v5Fvq/b8M4bi6kT/TqNZnNZmpqaoiIiMBgMNi9JvPqm9HVFmMa8SXo9MF3n+RcU0M1Or0BfWik165Jt/sdMESi63iNZ66JwLpPuj3vi3qjXW/322sym83U19cTFhZmdYz+eE22PveraypaDqVbMLe9oEly1++vyZX7VCeUHwiNDZxrUuA+NTQ0NL1vpWP9/Zr84j5Vn4CKXZDQF/ShgXFNatynA9+gW3YhpiEfYW5/hXeuqXwnpj0fYM4+D5IHK39N2jvX+ft08Ef0O17B1O9FSBoUGNcUiPdJu6ageufyY1uIzMQ0eYXz11S+C8Oa6zF1uB5z7hU+c02BeJ+0a3J8TWazmdraWqKiojCbzd6/pvI9mHe/izlzOqSOVOSabH5eX4Zu15vok/phajPNr+6Tvc/99W8PQLf6RvT7PsR0ThHmsCR511SxC9Ox1ZjbTBVyyT52TTavVcH7VFZWRkJCAqWlpcTFxeFpNKefC7ji9DuVsrIy4uPjKSkpaXL6tcJUD/pQpYatYYtVN4IhAga+Bjqdd/s+Mhe2vwQDXoGEnt7t218p/ht2vQVd/gFJA7zatfSykF5AdlkwDWqPw9TV3hmcv7H/C1h+uSjInDVN7dFo+DJr74b47i5rxjs1bwOFkxth78fQ+WaI66r2aKyz/ErY/xmctRdi2qs9Gv+gdDssmg7d7oYut6ozhv1fgNkM7S/zaDdBOW81AofKA7Drbci50OtrVbXR5q6GhgfY9z8o/Av6PgeRGYo3H1TztmA+hMZB8iDnzy3Jg/mnQY8Hodudyo8t2ChcCBV7oON1nutj32dQlQ89HvK+rc/DqD5vi9fA3CHQdyb0fMjx8RqBRcVeMNZBbCfQayKSciktLfWq08/7gqIBQEpKCgaDgcJmKawAhYWFZGQ4twg71dvcAs3h5x2G/hcG/UedRUDmFJjwh+bwc4bkQTDsQ1WMKCaTiby8PPvzVmL8b3Dacs8Pyl+J6QQ550NYotoj0fBlTPWw83U4Otf1JpyZt4FCxT7Y8SocW6b2SGwz6DUY/YPm8DObhZP25AbHx+p0EBLt+vrw8C+w5XnXzpXYMhM2P+VeGzIIynkrkf8jbHzU+/2azVC6TWR3a7hHdDvo93zQOfwgyOeu2pjNsOR82PmW2iMJPE5uEMFKRUvV6f/4ChHMpfOMYTWo5m3GBNccfgAJveHco5rDTym2/RvW3AKmBs/1sfcT2PpCwDn8wAfmbUJvOHMHdLtHnf411CWmA8R30xx+TuLt+ao5/VwgLCyMgQMHMm/evKbPTCYT8+bNa5H55zKlW2HPh1Bd6PhYDQ0N30V7AdomZQiMmgWpCjwznWH+FFhzm3f79GdOboK5Q2Hvp+r0rw+F80/CoDfV6d9fyZgI03dBh6vVHoltwhKh7Qy1R+Eb/Dka1t3n+Li4rjBtA3S6wbV+9n0GGx+ChmrXzgcY8T8Y9pHr52s45tAPsOVZqCv1br81RfBrD1irGTP9kg0PwsLpwvGj4Rs0VEHe05D/vXf6qy+FQz/ByfXe6S+YqCkS6gRlW9Xpf/AbcH4JhCer03+gYTJqz0pfoNdjMGE+4EGH3IjPYLIPB0H6M4YIiOsChnDP95X/vdgvnVjr+b405GE2Q81xqDkm/5xFZ8MvPTw3Jo1WaE4/F7nnnnt47733+OSTT9i2bRu33HILlZWVXHPNNe43fugnWHWdSHXX8A7bX4G/xoHZi1736qOw5AI4Msd7fQYKO98U98uTUWHusOdD2PORtpnwRWqOQm2x2qPwH0KihCRKfZl6YwiN9YiUUUATGiukNnQ+uswr+EtsEjRE5HG/56CLF4IRej8B0zaDPszhoTZJ7AupIxQbkoYVev9TyN421rr0Gjq9kL/KPNO7/QYq21+F2V289/4s3w0lGwIym8Fv0ekh73E4+J13+gtLgItqYOCr3ukvmEgfDxdWQkcXg26UICxem99KsPoW+CoEGiqcP7fmmJA5L9up/LiCkZShkDYK9AbP9RHZBhJ6ea79YKe2WCiWeLyfE1C+07v2Wg37VB6A71OFCoxconOFo1jDa2hpKC5y0UUXcezYMR5//HEKCgro168fv//+O+np6e433uFqiOsGif3cb0tDHtVHxEuk6jBEt/VOn6XbIP87kZGh4RyVB4SmfvURiM5RezSt2TIT0EFHBYIAApktM6HqiIhe9RbTNnmvr0AgthOcc0S9/iv2C4NpXDcwuOGoCEbqTopMzdQRviUX3lApovySBsLkxWqPxjeQ6/CrOgwHZ0HaOEjq73w/8d2dP6c5pgYw1QqJUQ3PEdNBnX4jUqGfExt3DfuY6kFnEMotoZ6v2cHob7VgM1/DEAGnb4DITO/1qTeAXntGK44+VN21VMF8iMrWjKVKkNgXci4AswvBw2XbYPllMPA17V4ohakBjFWeeU+azcJuFJEmAlk1lGfFVVDwJ1xU7dlg007Xiy8N3yEqWwTCpI6Uf86g1zw3Hg2r6MxmbXegBmVlZcTHx1NSUkJ8fLzaw9FoqAJDpPej5+obI8xCY7zbr79jrBWZCl6+X7KLJVcdgapDQsJSwzbzT4MTf8N5xVrkqoZ11j8A214U9QJc3FyrXuRcLTY8DFufE5ldvlQ31lgDB78R8p5ZWkaRUxQthr/GCmNT1ztca6O+DGqPu+ZYKv4b5g6Gfi9Cj/9zrX+ZBO28hUa5nEIw1flmYJOGhh2Ceu6qTdkuqD4MyYO14AylMRmFrFxonKhh5E3MJvgqHDKnwdifPNOFNm/lUXMcji0RtcxiO6k9Gv+nthh+yIT2V8HQ/yrffn0ZfBMvkioCUJreJ+bt/i+hcr+o6+cNmU8NDT+ntLSUhIQESktLiYvzfFCgj+o+BTHGGuc0cTWUISRKHadDaIzm8HMFQ7hqTqK6ujrHB0Vlag4/OYz6Gs495t17ue8zKFzgvf4CgaLFsPVf6mQQZE6DXv+E6HZuNSNr3gYa2WdB/5d8r/6LIQLaX6E5/Jpz8Bv4uTMcW27/uIS+MHkptD3ftX5MDfBDFqy+ybXzQ6Ih9wpI6OPa+U4SlPMWRMT7D21EjTZvsv5+WHSWMG5r+BfGWrG+KclTeyRAEM9da9SXQ8U+7/S17xOYN15ktmgoi7ke/hgKm59WoW8TDH4LOt3o0W60eSuDiBRoe47m8FOKsCTIOss19Qo5mM3Q7V5oM9Uz7fsAqs/b3Eug50Oed/gd/g3yf/BsHxqexWQUe5tDngle0bCO5vRTGZPpFE3ioiXwfRrsfl+dAQUzJVtgw0Pec7oe/EZEZGq4xtE/vVcjoxGTycSOHTtaz9vmVB0W9Rq1JGrHhCV6VsPfGiuvFXV2NOSz/3+w4X6oVSEgJX0s9HnCrY2ErHkbiKQMg+73+F49RO3Z2JqQGPE3bqy2f1xYvJBQiXJRKk4fAt3udt1pGN8dRnwKmVNcO98JgnbegnCudv8/7zvGK/YJp5G338uBzP6vYNe7nu+n+iisuBL2fuz5vhwQ1HPXGssvF7UdvfHuy54hMsHdDJTSsIIhAvr/SwQteRt9CHS6AbLO8FgXQTVvT26ENbdB8Rq1R6Kh08Hob6DzLZ5pPyweBvwb2l3kmfZVJqjm7ean4O9/qD0KjVPZ9S78NU6U73BEfQlsfQGO/ObpUfk03p6vWk0/XyM8SaSfJw9SeyTBR/Fq2Pq8kGvIvdSzfdUWw9ILof2VMPwTz/YVqKy7RxTgzjlP7ZG0ZPPTsPtdOKcAIhWo8RnINFQKqZzILIjt6J0+x/wI4ane6StQ6HondLgGQhPUHolGIPBbb1HfaMIfao/Ed8g8XXw5wmwS2Xr6ENfrZvR5yrXzNLxL/xe93+fobzSnvNJsf0nILXZ2MbtWLuHJMPp7iGnv2X40nCd7hqhLbKr3fG3i5EGaDcGTdL9P7RFoKEH1Udj1FiT2E1K4zlDwFyy5AAa9Ae0v88jwNDT8iprjsPR8SJ8IvR/zXD8DXob6Us+1r+EaVQdFIEV1gWN7XmgCTN8lgmg0vIbm9PM1kgYGpN60X5B9NsSv8s5myRABI78Szg4N1xjwsqjD6GtkTBb1BiPS1B6J71O2U9Sn6v2UZxeJzfFglGzAEt9DnX7rK2BOf1G0u8cD6ozB31lxFZRugal/qz0SC/G9IEJzvLvEkd9h0Rkw9EPoeI17bZnNzksr73pX1GEd+JqQRdcIPLRaTsoy5B2RyetpQmOF5JyG7+Hus1ouZjOYG0Af6p3+NLxH/g9CEm3Iu5A+Tu3R+D9pY+Gco64FgYbECXudr0nn+zOFi2DnG9D7cRF8rySHf4Pd70CfpyGxr7JtawjCEqFsByQO8Gw/qSM8276Ga/R+Cvo+K+9YvUGTRlYBTd5TQ0MiPEnUYXM1et4ZQqKFzEDaKM/3Fai0mazK/5/B4ED2Kuc8GPQfzXAmh9jOwnmrOeJ8H2Md1JV4t8+6E0Ly0Ox+fSmH8zZQMURCSKxv1ega9RUMel3tUfgeB2bBtn/bPyayDbS7BGI6uN6P2Szqti2e4fy5hfNg74eg93DdjkaCdt4CHJkLf4yA4yu901/lQdjzkVYLTGmSBkJcV8/342MZmkE9d9WibDt8m+gdOdlgZeW18Nd47/drNglnroeDXYNm3oZECul7V6SsU4bAxL8gM3BrxHmd2uNw6Aco36N825X74OjvjuXz/RjV563eAOccgYEve64Ps9nn1jkajTjzHK2vEKWQjFr9WG+iOf1UpsVDujIf5g4Xtd401MFsFgaWY8s8209D4C48vIrJCPXlXuvOYDDQu3dv9RdXgUJojKgvleThyDAJYx3MioFV13unv0ChvhxmRcK6u73bb3QOnLEZej7sVjNBPW+HvAOTFmg1uvyBvR/CpseEcc8WSf1h5Bei1qWr6HSNfbiwgR75NZx33Ct/T0E9b0Fk7FTsgZoi7/R3bCmsuhaOr/JOf8FEXQlUF3q2j83PwLfJULrNs/3IIOjn7qkcWwELz4DCBZ7tp6FCOJmjczzbTzBjrFXHeZBznlgPpwz1WBdBN2/L90DJFrVHoQGQOQ0uKIO2M5Rvu8ttcFEtJA9Rvm0fwGfmraeD3WsK4etw2PCQZ/vRcB5jLRyaLdY6jjg8G37MFt+DGG/PV83ppzLm5gaXyv1QvlNeEUwNz1BfAvPGw/r/82w/szu7FmWvYaHmGHwTA+vv91qXZrOZsrKylvO2OWU7NMe9r5M6GmK7qD0K/yI0FtpdBsmeMzZ4EofzVsN7FMwX71ctm6g1A16G0zcBXsgSHzsbxv7s/CZdpxMyPl4g6Odt5jQ4txCyz/JOf+kTYczPkDbGO/0FC1VHRPbVZg/X0ozKErJoPiA5F/Rz91QaKqBwPlQd8mw/yYNh0iJ59WE1XGPk/2CKl7KvvUzQzdu/xsJKF6R3SzbDxkegJE/5MQUrIZGelYzX6byj5KUCPjNvqw7DtpfhxDrPtG82QpupmjSkL2I2wuKzYKcMFZ/YTvx/e3ceHlV1/3H8M5OdZSaELey7bCKbyuKKIqDUfa+1LlR/Wm3rUq1a17buba3aKmpdaGutWhUt4oIiKoogKLIjq6xhSzIDhJBk5vz+uBIJZJlJZu7NSd6v5+GR3Llz7/c4fDIz99xzjg75pbPOcSPmdl4b5m8/i0Sj+93R3eYY5w7qrj/xrqDGLr2FdOTT0pHPJO8ckWJnPv7sQck7R2OQ0cq5KJboud+rEY1GtXr16oq53V/xFmn3amfoOmLzzW+l19o4uUi2lHRp1DtSP/c6ihuMkf+Qel3l7jk3TpFWTKzzv40ac9uQRYqlJQ9La1/yuhLHlg+dKSz5HXmwYD8p0Kv6jrgdXzojlfPn1e1ctbkj1xgp70Np19q6nTtGjTq3kvtThGe1lTqe6kx5hsTJaif1uMK54SiZelwujZ5RL9aTbvTZPVDuidJ5RVK3i72uBLZa8y9p5d+TeopGl9t+v5F6XR3/80JLpMX3OdPpInF2rZFWvyBFSxN73K2fJH+UtYfqTW73bJa+vjF5I7iadHBuVuwxITnHR+2lNpFG/Evqe2PN+7Y8Qjr8USm7f/Lrqsfczmuqq2dDzXw+ycfL4qlkfylLyZRG/iu552gMfD7pmNe8rqKiNsc6d+Z7fbeVTdKCUuAQqaTAuTgG7LPqWefLQw+mY601f7q08C6p9dFS1wu9rkY69C6p+2VS0y5eV1L/GCPt3SEpWvWF+50rnVy0O9mZxq0uVj7tfEkfcFds+5ftlKaPlrpfKg1/vm7nRmy2z5a2zpD63pz8TsCy3c5600gsn08a9rTXVcBLbowwKVggLf6D1OcGqdXw5J+vsSpY4HQidDnP3Q72ZY9IJTuknnweTpjev6jd89qfLI1fKjVpn9h6GrtVz0qL75VaDHL+JMrXv5GK1klnbkzcMXGw7AHSCR+6t2QL6pduF3ldAarBSL/6IhqRFj9Q97u3kRjGSJvfT/4aHGiY3L5D32b9bpZOmulOh19ZkTMd7Po3kn+uhmbjFGdK4l1r3DvnkD9LJ0yT/NwIU2s+v3TSZ9LRL3tdiSMl3Znaw5/mdSX1z+7vpNdbS4vvr3qfTmdL5+RLHU6t+/lWT5KWPxr7TSq+FOmIJ6QuP677uRGbtf+W5t8i7Vqd/HO91UP6aFzyz9OYRUqSd+wFd0srnkre8VE3Wz5yRkonS8FX0rr/urrOeaO0Zbo07xfOcg5uGvmidNQr7p4TlUtrLgX7SGkBrytpWLpcKB39qtS0a2KPO+BuacgjiT0mDpaSIeWeIKVnJ+f4edOlr250bbYR1IKJ1vydcslD0idnJffzMA5Cp199EV4qfXMra4HVF5velj4aK61LwoXSub+SFtyZ+OM2RgXzpZnnuTptQ2ZmZtUPfveytHGqa7UgTmW7paUPOx36iM+eTU7H3+617p2zWTep7aiEHKra3DZ0OYNdW4etRtvnfD+aDQdp0un7aQCPrnqflHTntUxJr/v5RvxDOm117DeppDZ1psJqd1Ldzx2jRp1byfn/PWaW828jmaIRZ/RCsqegbKyipdKHJ0qfXZC8cyz7s7T+v8k7fpwafXYP9PnFTgd+snS/1FkipM1xyTsHpE5nSid+JGUf5u55g32kVkcm/TSNKrdrXpTeGSKFlsb3vLIiZ5aEyN7k1NVYZfeXOp+T+E6j9mOdkbkNWL3JrYk6M5KU7Un8sbfNdD7nlIYSf2zU3ZKHpZebSDtXVL9fwTfSpqnc/Osyn/F81c/GKRwOKxgMKhQKKRAIOF+4d3zhXMwJ9vO6PJQUSKuek3JPklok8IO9MdKbnaVmPaXRDXd+cddsnyO9P1wa+mjtp+lIpMldpIyW0slJWsS4IYoUO6NNmnZL/ofyaEQq+k5KbS5ltk7uuRqaSLEz9bRbo+6ipVJJyMkTI2frJhqRdq1yPl94+e++JCT9N1vqdok04gXv6rBZ8XZpzwapWQ/nbnMAdvjkTOezx4gXkjPd4548KVoiNe2c+GOj7ta9JqUHpdzRXlcC20TLpNKw8xmOz8OJs+p5adHvpKP+I7UaFt/zZl8uHfe21OGU5NXXWJWEnN+VsM/yx6R5v5JO+MBZ8iaRHTtle6TiLc7MUCkZiTsuEmPdf6XVz0uDH665L8NE3Zn2vB47qC8oyej088i+F7qgoEDZ2dlelwM3majTqZjR0utK7Bctk6J7XVuHJhqNqqCgQC1atJDfX8mb1bZZzkWXttxpG7NoRHo1ILU+SjqBEXj4Xv7X0rtDpIH3Sv1vq9OhasxtQ7d5mvTRGGnoY97eHFESklY+JQX7Sx3Ge1eHzVY+I825Uhr1ntRuTN2Ote+OXBkp0DuGc/9dWv4XaeS/E3szVBUafW73iZY50yoHenldCerCmEZzwZ7suqxwsTO9Z4dTkze1Gn5gos6NaW5deA4tld7uJ/W/XRr4+6SdhtzGaPsXztTbva7iRv1Em3OVtOrv0rmhxFzb2bNZmjpQ6v1L6dDb6368eqhe5bZgvrN0QHqOtOYf0ugZUpMOzuuQli2lZnlbH1BPFBYWqkWLFq51+vGO7jFjjDNNwI65zodI1C/GxL7eTax8fjr8EsWf6lqHn+Tkdf369aryXonWI+jwi5c/RTrpU3fWHIuWOetmlRQm/1wN0bZZ0pYZ7pwrtYnU42dSy7pPZ1Rjbhu6nCFSnxuknMO9rSM96KzhSYdf1b57RZp2jLRzVeWP5wyVDvu9sy5iXZXtkqb0lhbeE+MTvr/Qmdqk7ueOQaPP7T6fniVNHZDc9S/W/keafaVzYQbJsX+HX6L/TZfulMLfJmdKrVogu9VIxv+TNf+QZv205mm1UHe7v5P+kyYtcLEDISXT+Twcz2i0WiC3MWo1XDr8MTr8kqHlEc660YlamzRaIgX6OJ1QDVS9ym2LQdLQR6TAIc53jOItzvXtL6+WFt5Vt2PvXCkVbUhImfDQxrelfGZEczuvdPrVB3kfSO8dIa2Y6HUl2N+Sh6XXWib2Isim95yL5vXhjbmh2LXGmZff64sdZXucC6KIX84Qd9YcK1ovvdlVWnxv8s/VEH1+ofTV9e6cK9BbGvYMU2ElQkZLacifnJsSUL+VhqTQkqo/d+QMce5Wbta97udKCzgdiF1/HNv+Pa+UTl2emA5HxK7TOU6nfbQ4eefY9om06hnJn4C1IlG1ggXSu0dI619L7HG3THc68FkXvv7a8D/p1Wxpw+TEH7vPDc4arTlDE39sVJTeUupwujNjgVuadXM+D3f4kXvnbAwiJdLqfzjX4VA/9JggjfyHlJWbmOM17SKd9Il0yM8TczzEpsv50qnfOt9Z5HNGK9d1DcyZ50nTjk1IeUiSRX+QFj9Q/T6fni0tuMOdelDOpcV5UK3mh0h9b5baj/O6Euwvo6Xzob40JKl9Yo45/yZpb750JneqJMyaf0gL75aCc739wrviCWn+LdJJn7my2HqDYqLOyJa0gJTVNnnnSQs4F0da86GxVgY9KKU287oK2Gr+rdK2T6Xj35XS+HdUqR4TpJ5XuHe+BjrdUYPS/afJP8fhf5MG3N2g74avFzJynNFYe/ISe9xm3aV+t0g5gxN7XCROVjupxWAppZYjpY2RCr6Wsgc6M2RUOHZbqdvFda8RNUtrJh37utdVIBF8PumLS6VOZ8d3g+HGqc5NMof9Qcp2sfO3sSjeLn12vtRqhDTwD15Xg9rat+64zyf9aGnd13DrMcEZuYn667uXnAEQ/W+p/HFjpOHPSRmt3K0LdPrVC8E+0uAHva4CB+pxufMnkY54UiremthjNnadzpECfaWm3Vw5XfPmzSt/oGlXZ9q6Zl1dqaNB2TZT+uA4adBDUr+bkneefSOeUDtdznfvXPNvk0p2SEc+lZDDVZnbxmL9ZOmbW6Thk5I+RVSVSndKRRtdnZLZOjV9KV77kvTtX51cZB/qTk37rHzaGZHd+VzXTtnoc+sWn0/KbON1FQ1fk47S2dudqekTKXuANOj+xB6zjsjuAVoeLo3+qPbPL/hK+vAEKXeMdMx+IzrDK5zPthl02DdYq1+Q8qZLQ/4sZSb3Ymmjyq0/TTpuijMaLB67Vksb33JuIkXiZeQ4/48TMX3q1k+lrR9L3S+XmiToBv56qN7ntq4dfpJ0yDV1PwaS6/h3q+/Q8/lin10GCcX0nh5LURlTPTYmrY+SOp3pdRUNS3Z/qct5rnzhTUlJUY8ePZSSknLwg53Plo6dzIWz2sge6Hx5ajXc60oQCzfes7bOcC5yJEC1uW0sUjKlaMTb9SyP+Kt0+pqKa1vhYHnTnSmrK1MaknavSdzdrtu/kKb0rfp8+1twp7TskcScNwbkdj9zfyVNH5O84+d96EyVjuRLdIdfPUR2k6DFEGdayQNvupj3K2lyR++XOGhMljwkfXWje+fb/oW09p9J/93RKHPb4ZT4R+v1vla6MCK1OSY5NTV2Pr906krp8Meldf+VFt9X+2PlfehMJViSn7j66hlrcrt6kvTpOVz3bsiadpJSs7yuwgpu55VOP49Fl/9VmtzJmesY9c+KidLiBN09W7zdmcYQyVESSvopotGo8vLyFI3yOiZUetAZgZfsL1C71kozfiSt/U9yz9NQbZ4mvZ7rzrpBYz6XTlmQkEORWznTh5+2Qmo/1utKUJOFd0lzr6n8i3Gvq6QzN32/TkYCpOdIJiKZspr3PeFDaeijiTlvDMjtfvZuk/bucNYgSrTSXdL00dLCexJ/bBysbLfzvWL1pMQd8+vfOJ9t6snFNLJbhRVPSd/8Nv7nRSPOzTIj/+Hc5ChJxduk/HlS1wulvjdxoc1Nm95xRt+55ciJ0nm7pLRgUk/TaHMbKa43vzvxvX1TGK96zvlsUlZUu+P0/qV08tcNei1qa3JbMF/a/K60+7v4n1u0UXr/KOffA+qvsiLnc0lV69JveEt6o720cYq7ddVDbueVTj+v+TOc6V6adfe6ElRm7b+kpX9MzIfBmWdLb3blg2UyzLlaer117T8UxsgYo7y8PJkDX8OCb6SPT5O2fpLU86OOynZKWz50RsogfhmtnPeqlEx3zpegi1hV5hbu2T7beS9N9FpWDdHA+5wpp+TCv9fAIdKp30rdL6l53+z+Ussjkl/T98jtfkZMkk6eJ6WkJ/7YPp90xBNS14sSf2wczJ8hLXlQWp3Ai1e7VkmhRfVmFDXZrcKGN6Rlf47ve+CuNdJb3aX1b1Tc/vlF0rRjpNbHSIfRYe+qY16Tztjg7jlTmyY9340yt19eI72cJZWGY3/OrjVS3gfxPQe1M+RP0mlrpNRaroWakSO1GOTe91YPWJPbQ2+XztpWu2VwSgqdzzl7dyS6KiTS9s+ldw+X1lex7q0/TWrS2VkqopFzO68Nf46Res70ulpqcZvXZaAqw19wfjEl4oN2m+Od6VnqyZfyBiVnqNOhUxqq/QfDuggtde787HW1++duKL57xRnhctTLUovDknOO7AHS+UyBVGs5g50ReMm2Z7O0Y67U8kgpq23yz9dYbJ3prEMy4B73RwWs/6/T6dfuZCkr191z26a6Ec8FC6TChc76senZrpWkkpAz5VJaPV83pKHypyXv2KlN+eziJn+qdNKnUvNDEnfMY/6buGMheY58Jv41bXetlqJ7D/59f9gfpLz3WUfcC26vn5g3XcpqLwX7uHvexiBniNTlgvimTP/uP9I3t0nj5iVu1gVULti3bs/fs1lKbcZn1/ogo2Xtn5vdXzqLm0brvezDpAG/k1qNqPzx9ic7f+A6RvoB1Wnes25vUvs77B5pqHvr4TQqPX8mjfyXlNXOm/N3vUA6Nyy1PdGb8zcE/lQpWtqg591HjLZ+In1ymrQlMWv64XtbpktLH5YKEzNtalwOvcuZHrKuX+Abi2hZ5esvrv+vNOsn0p5NiTvX9i+c6QGru4N2xd+kVwPOtC3wxuZpzqgEpom3X/YAKSXD6yrgtqadnA6jmm7+jJZKG6c6f8890Rnp0nZUxX1aHemMnID79u6Qts9xpkZONmOkGeOkr29K/rkaox4TpKNekjJbx/6cduOkIyY6I1aQfCUF0ob/1W6mrOknOSOPUD/sXuesDb57vdeVIBky20gD7uBmiHqITj+P+b++kfX86rNoRAqvcNYCQ6Pn8/mUk5MjX2Vf2FOzkjP1VmPR6SzptJVS2+OTd46SAudCCnmuve9ekeZdl9xztBwmDXtOan1UQg5XbW4bkx4TpPFLpBwPvgCnNZNyT3BGi6F6ZUVOB9uX1xz8WJcLpWNel5p0Stz58r+Slj4kbZ9V9T7BQ6WuP5GaubcuCrk9wJYPpRVPSIWLEnvcVc9Lbx8q5X+d2OOieqGliVnXpKTQeQ3Dy+t+rAQhu1WIlkqhZVJRDTdtbJvp3NwRWub8zHp99cuaf0rvD3PWqEo2E5EOf8JZzzfJyG2McgZLvf5PymzldSWNw/zbnBtBd66M/7ldLoht+nqLWZXbHV9KX90gbXo7vucVfONcf6jsZkjYY8WT0rd/87qKesHtvHL1xWO+1c9LpTu9LgNVKc6TphzijI6oiy9/Ln1xGev5JdPKv0sfHO98qU4Sv9+vzp07y+/f71dnSYG08hk6kmwQWiJ9PN5ZVwW1s/kdafmjUvG25J2jWVepx2VS08TcRVtpbhujJh2ckXb+FHfPuyfPuSDN+19sUptInc6pfP28YF+p05mJnaqo87nSKYuk9uOr3qfjadLIf0rpwcSdtwbk9gCH/EI6fW3ip782EcmUeTM1emP2+U+kLy6t+8jN0GJp9uXSxv8lpKxEILtVKFwgvd1XWv189fv5Up3pzaN73akL8WlzrDToAWfkZrL5U53ZbDpU8/6cqFM1xtwWbZDmXC1teNPrSlCV7pc6IytrM/PWobdL/Rv2MkpW5TZ3tHTc21K3i+N73pp/Sp+dz2xQNvjmdumtns7AmQN9+1fp28fdr6kecjuvFvx2aNiipyxyFphF/ZTVXur3G6nDj+p2nMKFUuFi1vNLpqINzp1Au1Yn7RTRaFTr1q1TNLrfRZptn0tzrpQ2TU3aeRuNzdOkBXcnr3OgWU9p2LNS7knJOX5jMOB30pl58U2FE68Ev/6V5raxKiuStlUzoisZ1kySpvRxpm1FbEb+Q+pznTvnymztrJdRzz6fkNsDNOkgNe2S+OP2/Jn0o2VSoHfij42qHXaPNPwfdX+/C/SVjn1T6nB6YupKALJbhWY9pX63Vr9uq+Q8PupdqcVAd+pCfHKGONcGkvH7+EBlRck/x/caZW6jpdLKidLWT2N/zrJHpP/1lnatSV5d+EGrYc7ISrfX0rSEVblND0odTol/bdseP5NG/lvK6picupBYaQFnUMSBRr3nzFYD1/NKp5/HTFZ7pgSsz3w+526+ui46etKnzh8kT7+bpXN2JPXClTFG+fn5MvtfpGk9Ujrmjbp3DENa/5q06B6nAzcZstpKPS6Xsg9NzvEbg6adnP+PyfTOYGlG4u5qrjS3jdXsn0nTRla/fluitT5a6v2rykeuIT7f/FZ6NTvxvyMjJVLedKlo48GPlRRKH46WVv8jseesAbmtROlOaf1kpjhqCDr8yLn4VdeR1xk5zkjcQK/E1JUAZLcK6UFp0H3OSDEgFlMHSB+f5sqpGmVum3R2bmQc/FDsz/GlOmuyMl29u0p3xjeb09aZznfJbZ8lr6Z6wMrc7lxV8zTX+wv2kbpeyDVzGwz8g3TyV5VPf9ykoxTs535N9ZDbeeXdymt7GabcaKRkeF1Bw5baxJsP4OktpE5nJGwqwkatz43S+MXOCFvUX7vXSfnzknf85j3cuYO6Mep2sTToQUkujupqfZQ09C9MHxiPwsXSp+dKm96puL1JJ6fz1J+Z2PPlfSBNP1H67uWDHyvaIO34QtqTpJsxELvvXpY+PVPa/H7ijrny79La/yTueIjPni11e34Sp7SHB9b8y7lQHf7W60pQlT1bpPdGSEvi6CiqjbI9UquRUvaA5J6nMfOnODcyxnP9oPcvpFMW8D3FTWtelF5rKW39OPbnFK2T8j6USnclry7Eb/ts6X89pVXPxP6caFny6oE7yvZIu79zbjKF6+j085hv3Utel4CabP1UmtJPWl/LdcBWPl375yI+BfOlJQ/XfY2UWJmos14VEiPQy7kDKFlrjm39RHqzm7MYNGrvwxOlzy9K3vGPeU064onkHb8xa3+yMyrarWlybLrztF4xzsjnwkUVN/e6SjphWuV3UNZF2+OlgfdWvm5Q9qHSuTulPr9O7DkRv/anOGvbtDw8ccdcdI+07M+JOx5i99lF0lvdpEgd1m6b8SPpjY78rrXF4vuktw91Rq1UZtcq50J1OlPZ1VupWVLReqksyZ0JqVnOWroD703ueRq7XWvdn/Ye8cke4CzNEc8Nb11/LJ2/x1lHDvVHzhDpkGulNsfHtn9kr/RyljTn/5JaFhIkWiqtePLgm0h3zJbe7CqtfMqTshq7VK8LaPTasrZUvZfeQooU125efROV5t8qNeshdToz8bWhorUvSUsfcj7g5QxO+OF9Pp9yc3Pl27f2Uf486b0jpYH3Sf1vTfj5GqXSXdLutcmZgtOX6iwEzqjbuul/i1wdKVZHB+UWzgLbyepc39+yR5z1Uo5+lfWJ4hHoK523y73RkalNpP63Vf24z+f6tDrkthJN2jtr2yTScVMkw13Unmh7vJTW3Ok8qO3nkpwhUmaberUmJ9mtRkYrJ2+711Y+gmvAXVLfmxgZX5+lBaQzG97I90ab2y+vcm4KPW93bL9Ht3zs3GTcY4KU1izp5UFSi8Ok49+O/3k+n+Rz4buOh6zLrT9NOvzx2PePFDlTmAf7J68mJI4v1bn23WKg1OX8H7ZntpV6X8dSH99zO68+Y9UEwA1HOBxWMBhUKBRSIBDwuhwkizHSrtXO+kmtjvS6moYvvEIq3iy1HO7OBcrCRdKyP0k9r3IWmkbdTTtaCi2Rzt5Rry5iwSVbZ0qb3pZ6Xik16+Z1NQ3TkoelxfdKp66QMlsn91zLHpVWPCGd8o2UkuApKRuj9a9LO+ZKh97hjAJItGiZ0wGRnv3Dts3vO1Nfcbd0/eFWpz2AxIqWOReh+XyL6kQj0oxxUudznc/DSJ51/3Vm7el1leSPYTzE3F9K3z4unblJymqX/PpQO5vekVKbSW2O8boSVMUY3gsbou1fOOulNmG5nqq43RfE9J4ei0QiXpeAZPL5nPWp6PBzR6CX1ObYpHX4RSIRrVq16ofcZh8qDX+eDr9E6j5B6vcbKcqc343S1hnSkgekksStd3tQbhu7zNbO766925J/rj6/kn60jA6/2tizWVr1vFS8/YdtG9+WltyfnN+PezZLr7WSvrm94vZvbpNmXZr489WA3FZh/q3Sq82k0nDdjxUpcaYZ5P5PJBDZrYY/teqLnKsnSSufYc0bG2ycKq3+R/KOv3utc4PPrrXJO8cBGm1uO58j9b42tg4/SepznTT6Eym9ZVLLwgHC30ofjZPW/DO2/ede6/xp4KzM7a7V0pS+znTXaHhaDafDrwZu55VOPyAW+fOcCy17Nsf3vB1zWUDYbSYqFSxw5pROgp07v1+Hg4tkydHjMqfTLxlTcIaXO6OcwssTf+zGpHSX9M5g6ctrEn/sPjdK45ckfBqP8txC6n6pdNJMZ/1MN3AXZ+1seleafbmU98EP2wbdL/1ouXP3cqJl5kptRx3872Lwn6TD/5r488WA3FaiaRepzSipJFT3Y22bKb0acKbghTdWPS9NHVi71zO0zFnnZtvnia+rjshuNaJl0pyrDl6jaOnD0sK7Yu98gHeWPix9fUPyjt+8h3T29uqn3U4CchuDZt2d0WMuT3ne6GW0lLZ+7HQYxWLo441mPUzrcpvZzrleF8u1nnWvSfOul4pduFEViWGMVLTJmelun29+K316DtdPPUKnHxCL/K+d0SfbZ8f+nGip9OEo6eNTk1cXDrbkIemdgdKOOck9z5pJTsdHwTfJPQ8Sp+Abaf7NUuECryuxW1ozZ3qqZIzeSs2Sgn0ZGdYQfHoOd3HWRfuTpZEvSe32W/s5s40UOCQ5Uzv6fNKxb0iH/Lzi9rbHSZ3OSPz5UDu9rpJGTZWadqr7sTJynJsAsg+r+7FQO5EiZ9T1rlXxPze0UFr5tLT7u8TXheTxpzo3nxUudKZx3OekmdIxk53plFG/DXpAOub15J7Dn8KacW7I/0qaOkha86/Y9o+WOX+4eO2ujJbSOQXOuqex6HCK1OFHya0JtZOaJZ26XOr765r33fyetPwvkriB1BrbPpUmd5BWv/DDtsKFzrSf3AjsCW4lA2LR8QxnqHKgT+zPiZZKh/3OuZsF7mk3Vtq7XcpI8lpVpTulPZukrA7JPU9j9OnZkolIx05O7HFzT5TGzJKa90rscRujcXOTc9zChVKTjlJ6i+QcH4686dKqZ6ShjzodSYlWEpJ2zE7OiLTGIitX6npBxW17d0iRYimrvTtfnCIlkj+NL2kNVYtBzhTl8E7PK6VeP69dxjqdI521VUpJwvqeSK7j3pRSm1d83dOzWQ7CFsle1mHl351p2FsNT+554NxkWBp2PlvFYs7/Saufk84NSWnJX4sJ+4n1hlDWims4jvibM+I5g+l0rRE8VOp5lfMdY5/j3nJGd8IT3ErmMR9vSHbIbOV8+I5nypXUJlKf6w++aIfkyhksDfmjMxoiwXw+nzp16uTktvcvpDM3O/82kFiR4ti/fMUjo6XzBZoPjvVTtEx6Z4j0xWUJPWyF3MKxc4X03X+kgvnJOX56UDr9O6dTEXVTvM25yUSSvvy5NLmjpCTdYW6Mk7/PL3Z+nnutM/1j4cLknK8a5LYaSx6UFv7O6yqQCHXpVPf5nDVa69loILIbg7TAD697NCKteVEq3uptTYiPiVYcqZkopbukOVdKy/6c+GNXo9HmNthPOn211PNnse3f8kipywWSLy25deFgu9ZK3/5N2rmy+v02vCH9N0faOMWVsrxkbW5DS6Wvf1Pz9wt/mtSsK524NsnIkY580rnZfn/MYlDO7bzyf95jfj8vgTXKdsc5vadFC+o2RMYkfOoNv9+vli1byl8w7/u7yMhvUhw3RRr1buKPGy2TyvZwp1EilO6SFt7j3I2cKNG9zujozucn7pjaL7e83/6g64XS2TukdmOSdw6f3+n8Q+1tfFt6vY20/jXn5/YnS4f8MnnvPT6ftHu9s2aKMc5ohg6nOndtuozcVmPda9LKp+p+nKV/lr643HlvhHcKF0vzb4t/3fD8r6XwiuTUVAdkN0aRYumzH0vTRkqzfiItuMPrihCrb5+QXkqVdnyR+GP706UTp0t9Ypj6LpGnJbex6fV/0lEvOVMUwl2FC52b0bZ+XP1+KU2dacszkjCTST1jbW6L1ktLH5K2VPNaGuOsWcwNMXYr3SWt/Y8U/tbrSuoNt/Nq2W+HhicSoWPIGvOuk94fHtuX8rI90uutpW9uT3pZqMSmd6XXWkprY5yfP0aRSERbPr5Feu9IafP7CT029pOsu1/W/EN6pYkzPzzqJiVTWv5YYjOW2lTqf6vTIZVAkUhEy5Yt4/12f2kB5068ZChYIH11o3NHLuom53Cpy4VS067Oz90vlQ5P8ujJ46dKYz5zfg/3mCAd9W9P7rAlt9U45jXp1BrudI/FlunSulfim8UCiVfwlbTkfinvw/ieN+tiacYpyampDshujPwZznIEKU2k4S9IPf/P64oQq+Y9pU5nJmcK85R0qe3xrk/12qhzu+FNacVEr6tATVqPlEa9L3U4vfr92o+VRs9oFNMlW5vb1kdJ4xc761RXpaRAmnaU9M1t7tWFxFj7H+ndI6Sdq5zZhT6/UPruZa+rqjfczivf8oBYdTxTatbd+ZJWk71bnXmMk7FWEmrWvKczEqJZj4QfenuT49W62zb5A6wLlzRle6SVTztru3U+O3HHbdZD6nox6zAmgj9VOukzJ2sWKC5OwnSxtivaJG39ROp0lnORKVHWv+5MS9X5PGdKFtReVlun081NKenOF20T9XwqZHJbhaadEnOc4/4nle1KzLFQex1+JI39UmoxOL7n9blRMvVzlCbZjYHP56xdndrE60oQr3ZjajdTQtFG5ybiPtc7HRiVKd7u3JTlwWwyjTa3yx+XdsypvgNin5XPSPlzpSOeZMYft2W0lNqd5HUV9Y6VuU1t6kytWx1fijT4YSnY352akDiRImnPRmewTLCfdNTLNb/eSBo6/YBYdTjF+ROLpl2cqTngjeY9pbO3O9MCfPs36buXpBOm1+3C9vdTQpam5coMeVpKSUlQsTiIP02af7PU9sTEdvq1Pc75g8QI9kns8T49x8nZsa8n9rio3MqnpUX3SGNmJ/Zu2AF3OdNQtmz4d9i6buE9zjoYR/8nuef56tfS6uekUxY66xmjfomWSoWLnM6CQO/aH8fnk9KaJ64u1E56C6nl4fE/r0di17+FB+jwa1y2f+5M192lmmnsPxrjXDD90TKW2l9nAABOBklEQVT36mrsBj8k6ftlQWqa2SBvmrTuVekIRgZ6whhnhHRm66r3WXy/06nU+5fu1YX4le6SChdILY9wrv0cKD0o9XV3mmMkSPfLpB6XO3+PlkpdzvO2nkaO21OAeEXLEr5WHJLE53Puqgx/K+3ZVLdjzThFvtmX8dq7wZ8qjf5YGv6815WgJoULpVXPJuZYZUXOhQ64o/O50vBJUvMEj4j2+Zy14Fh0PTFCS6T3R0qrJ0k75rozPXH2YVKnc7grs74q3ia9O0Ra9pfaHyMacaYpZxre+iFaJm3/QtqT53UlAGKx8u/SjFPjWye887nSmZucmzWqmk6y3Vip41mJqRGxyRki5QyN7XPr8OedG4v5jOuNmedKkztUvxbxt3+V1r7oXk2oncX3OtN3hpd7XQkSzeeTInulD0+UZv3U62oaPTr9PGbdoquN3cq/S6+1ksLV3H1XEpI+PdtZVw7eO+x30tlb6zbNXKRYMhH5TJm69+hBbt3QargztV0i5X0gfXE5Cwkn0sK7pdlXJGaR7VFTpVGJ/73p9/vVvXt3cnug7P5S958mdgrHlX+Xts9J3PEg+dKc9RBKd0rH/086tyD55+zzK+mYVz2duorcViOrnXTYH6ofLVKTvVulj8ZKS/+YuLpQe5vekd4f4YwgicX616W3B0hbZya3rlogu2gUwsukrTOkovXxPS8rV1pwp/Tl1dKu1Qc/Puh+adB9CSkxHo0+t2VFzvISNUlt6vnU541a7klS9wlSpJrX6uT50lFJnhGjnrA6tx1OlQbe78x2UJnF90vvDZf2bHG3LiRGSoazLJY/nUETB3A7r0zv6TEfdwnZpUkHKdBXKg1VvU/B19KGyVKbUa6VhWr4E/BrLiVTOmGafNFSBSqbfgCJZ4y0+zspLeCsbZEIhYuk1c9LPSZIOiQxx2zs+v5GOuRaKT1Br1ES+Hw+BQIBr8uov0p3JmaKv9Kw9NV1zrSeTG+dOM17Smdv87oK15Hbavh80qG/rdsxUpo405MF+yamJtRN2+Olfrc4FzRjESl2RsanNk1qWbVBdtEoHHqH00EX6/fCTe9KJfnOOsoD7pZ6/0pq2i2pJcajUed20zvSjFOkYc/VPG3yrtVOByFTn3uj1//VvE9ma0nVTP/ZgFid29Yjq17bVJLKdktF65xpPmGn46ew9mkl3O4D8hlDt6sXwuGwgsGg8vPz1aJFFXc3wF4lhZJ8vEnVFxvflkxE6nhanQ4TiUS0ZMkS9evXTyms6ZdcG9+WPv6RdORTUs8rE3PMaKlzoSwlKzGdwUic9ZOdu6V7XJ7wi5jkthoL7pYW/U46/Tupaae6H2/brO+n9xxe92PhYNvnOHc3N4K1ScltjEyUL9SoV8guUIkPT5C2z5LO2lrxRqu9+T/c3LjkQangG+nIp6W0Zq6W16hzG17hzFzS4zIpd3T1+84YL235SDqf5QjqpV2rpUiJM4VuIxhc0SByu2eLM9ovJd3rSgBXFBQUKCcnR6FQyJVOe74lAsmQnk2HX30y91ppwR21e27edOmTM6T8ryU5H67gghaDpUN+IQX7J+6Y/jTnizYdfolljFQw37lwUVurn5O+ukHyJee1IbdVyB4gdTq7+mlyamLMD1MitR5Bh18y7PhSWvKwNPca6bMLvK7GNeS2GjvmSpM7SSuf8boSJFI00iCmsiK7aBRCS6Slf3ZyW5miDdKaF50ZFUa+JB3zesUOvz150tv9pG9ud37O/1ra/K5nI3gbbW4DvaSjXqy5w0+Sul4sDbgr+TWhcmVF0uc/rXpq8qV/lt7uK+1c6W5dHrI6t1s/kf7XQ1rFZ1kgWej0A+JVuFD67KLK19HY+on07d+chUtRfxzxpHPXZG2EFksbpyS2HtSsSXvp8Mek1kcl7phFm5wLpWXcnZlQG96U3hksffdS7Y8x7Flp9Axn/ne4p/PZztptgUOcUWSb349/3v0vLpPeO1zatSY5NcJZv2v+zVLnc6TBD3tdDeqDJp2ktGxnrYzaWPg7aUpf56I06odoRJrSW/rsvOr3M8Z5/fI+cKcuAJVb8y/p6xulwvmVP/7lNdKsnzjTema1ldqfXPFxE5Ga95KyD3N+Pvo/0hkbG8UIJWt1vUDq9xuvq2i8UrKkDa9L27+o/PHOZ0v9b3Omxkf9lzNUajlMata94vYvr5EW3+fMZgGgThjuAMQrWupc3G57vNTm6IqPLf2jc4de+/FSs65eVIfKtB9X++f2/oXU/TLnQyaTIdtt9XPOiM+T50stBnpdTcORO9oZlVmXDtrM1t+vwQDPLLlf2viWdOqq+N6/sg911l3IaJm00hq97pdLHX4ktRgipWZ5XQ3qg6y20viFtX++P02SqdfrsTY6/hSp4+nOKB9jqr7wX7xVWniXM/V5LCNTACRH90ul3BOkQL/KHz/6Vee7R0abyh9v0kEa/XHFKZp5j/fGgrulkh3S4Y97XQmq4/NJZ22pejRs21HOH9ghtal04ocVt5Xtlja/59wQ0f82b+oCGhDW9PPIvjX9CgsLFQwyDaRVTFTaucqZCuJAZbulbZ9J7ca4XxeqFy2ToiVSapNaH8IYo+LiYmVmZrq+AGujtPLv0rd/lU54X8qs4gtzPLZ97ozG7XF5Yo6HxCje6kwv2aRzUu5uJrcx2rXayUf3S2vet6RQ8mc4F6equzgN1BK5BaoQLZPCSyV/ZuXfRTxGdtHo7Vxp3UijRp/bD46Twt9KZ22ufr8vr5V2r5WOZxageidS0ujWhWtQuV35jLNcyGG/k1KbSyUFzg1uQAMTCoWUnZ3Nmn5AveXzV/0lO7UpHX71UcF86ZWm0vLH4nteeIW06rkK66ukpzeuD5OeKtvtfODbU8MXsFi1Hin1v4UOv2QxRireFv/z5v5SerOrlD8v4SXtQ25j0Kz7Dx1+Wz+Rpp8klYQq33fxfdKbnZwLW7Z/ybRFtEx6b7g0fazXlbiG3NYgtNSZAmn7bK8rQaJVd0+uP9VZj7UedvjtQ3bRqOz40vksa6LSiqek/x0irXre66ri1qhze9wU6cyNNe+3Z5O0+7vk14Oq7clzpr0/cHry2ROk//V2rh80Ig0mt5vfd2ZTS23qdN7S4QckBJ1+HotGmafYWls+kj49W4oUOxdAF90rle7yuipUpmk3KfckqWnX+J638U3nA2RokSQnrwsXLiS3bun9S+mM75iK0wbGSO8dKU07Ov414fr+Wup3q9Ty8KSURm5rYd+ov22fOT8f+JoGeks5Rx68BgOS561u0o7ZjWbqL3Ibg5JCacUTTlbjYaLOVGab309CUaizb5+Q3uwi7c2v/PE9m+v19w2yi0Zn2SPSqr9LoSVSh1OljmccvH5fPdfoc5vWvOI0q1U59vW6Ta2Nutv2mXMNbu2LFbc36SQ161H11J8NUIPK7dGvSKcskFIyva4ESCq388qafkBt5X8tbXjLudiy8W1nerpeV3tdFSqTHqzdNBzdfup0FLYanvCSEINEjyBa/ldp7T+lY1531tFA4vh8Uruxkin74XUrKZDSsmt+HVsenrQOP9RS14ultic6o2Ln/lLypUhDH/nh8R4TnD9wT9eLnLVlB9zldSWoL3KGSqetkZp2ie95xVukRfc4n1mZnaL+8ac6F6CL1kkZlay5+OXV0qap0nlFzr4AvDXs71Lfm5w1jiWnYwh2MVFp20ypdKfUYbzX1aA6nc6UBv9JOuTnzs+lYWfU36D7vK0LdePzSU06el0F0ODwTQGorZ5XSl3Oc96cWh4hye90LqHhyGwjdT7H6yoaty0fS9tnOdNy1lXZTucOedMA7oarjw77vWQizt+jEen1ts7dzke/Uvn+616VmnSRWh3pWomIkT9FatrJeR0LvnI6/SIlUsHX0je3ScdPlVIyvK6ycRn0gNcVoL5JSZeadY3/eRmtvr+buvZrHCOJelzhfMeoSrtxUtPudPgB9UVqEylnsNdVoK5mni+lZ1ff6bfpHSlaKnU8zbWycACfX+p7ww8/L3tEWniPdOJ0qe3xnpUFAPUR3xaA2kpr5vyRpPQW3taCmm18W1r9nDTkL87F7JqUhp2pW1n/zVurnnGm7+j2U6lJ+7odq/+tzh8kh88n+b7/WBEpkrpdIrXY7yLIni0/zM8f2eusRZXaRDp1JRcv6yt/inTsm86ITX+Ks4bGlunOtIAdT/W6usZn1bPOaL+uP/a6EtQXJSHnxpicoVJm69ie409z1oRD/VTT6PheV7lTBwA0Fj6/dORTzs3cxlT9e/ib3zpTa9PpV3+0P0XavU5qfbTXlQBAveMzJt7Fd5AI4XBYwWBQhYWFCgYZHQYk3eoXpNlXOAt1tx9b8/6rnpdmXy4d+1b5xW1jjKLRqPx+v3yJnnoSlQstcb58BfslfrpPuKdwkfT+CGnU+1LrEc620BJnmrm2o5J6anKbQMZI+XOdzlw6at0VXi5N6eP8/ccN/6M7uY3Rmn9Ks34qjfy31PXC2J6zN9+5wJkW5H21vsr/Wlr5tNTvJuvWTiW7gH3IbYy2fOTMfBHLtQQgycgtYJ9QKKTs7GyFQiEFAoGkny+G1WoBoAHofJ50bij2D+ktBjlTE+YMqbC5pKQk4aWhGsF+Unb/xFyY3P6FtHqSM4IT7jIRZ0R02c4ftgX7Jb3Dbx9ymyA+nzOdNR1+7vOlOP/tfK63dbiI3Mag7Shp6GPxrT28+D7pvy2k3WuSVxfqZvd30sqJzhTn+9syQ5o+Rtr2mSdlxYrsAvYht3Kmtc//uurH246iww/1CrkFUB06/TwWjbK2FOCK1CbOn1jlDJaOfUNq0qF8UzQa1fLly8mt2yIlUt4HUumuuh1n9STpi0vrfhzEr8VAZxrPdmOcUbSb3nNGjbmA3KJBaNZdOr+46jUyGxhyG6MmHaXev5CadYv9Oa2GSd0vlbI61LgrPNLuJOc9s8dlFbcXrXc6/KJl3tQVA7IL2Ifcfm/OFdK7Q6U9eV5XAtSI3AL2cTuvdPoBaDx2rZVW/l0q2139fnt3uFIOYrTqGWn6SVLe+3U7ziE/d6ZrTUv+MHpUIiVdMlEpb5o09+fO3wHExueXUjK8rgL1VbTMGaEQi87nSsOf599TfZbaVGre4+Dt3S6WztvJ2kUAkAydz5MOvV1SFTPMvHuENI3fvwAAO9DpB6DxWPeKcwffjjlV7xMpkd7uL33+E/fqQvU6ni4d9gdnHbG6yB7grM+Ykp6YuhA/YyR/mjTs75I/xetqAMB+q56VXg1IBV95XQkSqaxI2vSOFFpWcbvPz/snACRD+3HSYb+Tsto61wQOvEEx2F8K9POmNgAA4sSiLAAaj05nS817Vt95VBqW2o2tcp+UFC60uK5JR+nQ33pdBRLBnyKNmOT6acktYB9yG6Om3aTWRzlrp9YkWibNGC91PE065Jrk14ba2/2dNOMU57PrMf+VNk6VQouknldK6dleV1ctsgvYh9zux0SlWRc765EfOfGH7SNe8KwkoDLkFkB1fMa4tKgOKgiHwwoGgwqFQgoEmGoOAGoULZOiJfGtzbi/2VdI370knbVNSs1KbG0AANR3e/Kk//WUuk+QDn/U62pQk1XPOzeiNWkvfXKWtGmKdMYmKbOV15UBQMMV2St9NEZKDUjHviH5GSsBAKg7t/uCmN7TY/S5Ah7Ys+Xg6TqMkdb9t9o1cYwxCofD5NYLhQul11pKy+twkbL5IVLrY/ji1siQW8A+5DZJsnKlc3dKQ/7odSWIRY/LnA4/STrqJenEj+t9hx/ZBexDbg+QkiGd+JF0/P8qfm9c8pC08mnv6gL2Q24B+7idVzr9PBaNRmveCUDiLLpXeiNXCi2tuP27/0gzz5UW/6HKp0ajUa1evZrceqF5L2fK1awOtT9Gv5ukUe84a8qh0SC3gH3IbZw2TpU+PVsq3l7zvj4f74M2iZZKa/4l7d0utR7hdTU1IruAfchtJXyVXCpd/hdptfvLFACVIbeAfdzOK51+ABqXViOk7pdKvgPmP+90ttT/Nqn3rzwpCzVIyZRGz5C6/9TrSgAAqF92rZY2TJZCi6vfr2CBtOkdqazIlbKQAHkfOGtLbXzL60oAoHHZ9rk0/SRnxhlJOmmmdNS/va0JAIAY0ekHoHHJPUEa/rwU7CPtXCkt+n5kX0q6NPBeKT3b0/KQRKtfkObf6nUVAAAkVvdLpHNCUtvjqt9v5dPSjFOkvTvcqQt1126s1ONnUkot1zMGANSOiUhbP5byv3J+btZdatrF25oAAIgRCxsBaLyW/lFa+ZTU4TSpxWExPSUzMzPJRaFKxdukeb+UWo2Uev8i/uevf0Pa/I406P7E14Z6jdwC9iG3cUhrHtt+vf5PCvaVmnRMbj1IHJ9fGvaM11XEhewC9iG3lWh9lHRmnpSR4/xctMmZfWbfz4DHyC2A6vgMq356IhwOKxgMKhQKKRAIeF0O0Djlfy2VFkptR3ldCWIRKZFeayl1OV8a9vf4n79nixTZ7dylCQBAQ7InT9r4P2e6ci5IAgCQWK8GpZbDpBPe97oSAICF3O4LYqSfx1h0FfBQzuC4do9GoyooKFCLFi3k9zM7sutS0qUzN8U+ouFAWW0TWw+sQG4B+5DbWtg4RZpzpZTaTOp64cGPR/Y66xn7+fqH5CG7gH3IbTWiEWnVM5KJSj2vYnpP1BvkFrCP231A/GbwGAMtAXsYY7R+/Xpy66XadvhJziiIog2JqwVWILeAfchtLXQ4VTrqP1L7cZU//t1/pFcDUt6H7taFRoXsAvYht9Xw+aVlf3aWBRn0gHTIz72uCJBEbgEbuZ1XbvUEANgjUiKtf01KbSJ1PD2+537+Y6lgvnROflJKAwDAM1ltnemvq5LewlmfqHlP92oCAMBmPp909H+lZl2dvwMAYAk6/QAA9vD5pS+vkrIHxN/p1/k8qdVRyakLAID6YE+eM41nZuuK2zue5vwBAACxa3GY899Pz5ZaHin1+4239QAAEAM6/QAgDs2b12F6SdSdP1U65jWpea/4n9vrqsTXAyuQW8A+5LYWdnwpvXekNOB30oA7vK4GjRTZBexDbmuwJ09a/7okRvuh/iC3AKrjM0wA7IlwOKxgMKhQKKRAIOB1OQAAAABsFi2VZv9M6nJhxbX9QkulBXdKvX8htTnWu/oAALDRVzc6a/uN/VJqebjX1QAALOR2X5A/6WdAtaLRqNclAIhRNBpVXl4eua0PSkJSaFl8z5l3vfTF5cmpB/UWuQXsQ25ryZ8mjZhUscNPkkKLnPVw9273pi40GmQXsA+5jUGPCdJR/5ECfbyuBJBEbgEbuZ1XOv08xkBLwB7GGOXl5ZFbrxkj/a+X9MUl8T0v/0tp++fJqQn1FrkF7ENuE2DX2h/+3vlc6dyw1O5kz8pB40B2AfuQ2xgE+0ldzpfSmnldCSCJ3AI2cjuvrOkHALCLzyf1/bWUFucc9ifNTE49AADUJwvvkRb9Thq/RAr0lkyUC5UAAAAA0EjQ6QcAsE+/m72uAACA+in3JGcdPxORZl8p+VKkI5/0uioAAAAAgAvo9POYz+fzugQAMfL5fMrJySG39Ykxzsi/WGz5WPL5pTbHJLcm1CvkFrAPua2j1iOdPyYq7VrldPpFSqSUdK8rQwNHdgH7kFvAPuQWsI/befUZJgD2RDgcVjAYVCgUUiAQ8LocALDPzAuk3d9JY2fFtv//DpH86dL4RcmtCwCA+qIk5EyH7WMpdwAAAADwgtt9QXz781g0GvW6BAAxikajWrduHbmtL1KbSOktnNELsTjmdan7pUktCfUPuQXsQ24TKD1Ihx9cQ3YB+5BbwD7kFrCP23nlG6DHGGgJ2MMYo/z8fHJbXwx7Vho1NfbpyrIPlfr+Ork1od4ht4B9yC1gJ7IL2IfcAvYht4B93M4rnX4AADvFOh92aImUNz25tQAAAAAAAACAx+j0AwDYa8Ob0idnSJHiqvdZcIc0/UQpvNy1sgAAAAAAAADAbXT6ecwX60gVAJ7z+XzKzc0lt/VJeLm0aapUsKDi9uKtUtEm5++D/yQd/jcp0Nv9+uA5cgvYh9wCdiK7gH3ILWAfcgvYx+28+gwTAHsiHA4rGAwqFAopEAh4XQ4A2KmkUPKlSGnNf9hWuFiadpR0yLXSwD94VhoAAAAAAACAxs3tviBG+nksEol4XQKAGEUiEa1atYrc1ifp2U6HX7RUKilwtgX6SLknSq2Ge1oa6gdyC9iH3AJ2IruAfcgtYB9yC9jH7bymuno2ALDczp07vS4BByreKn32Yym9hXTMq5I/RTrmNa+rQj1CbgH7kFvATmQXsA+5BexDbgFUh5F+AAC7FW+VIkWM7AMAAAAAAADQqDHSDwBgt+xDpTGfe10FAAAAAAAAAHiKkX4e8/l8XpcAIEY+n0+dOnUit4BFyC1gH3IL2InsAvYht4B9yC1gH7fz6jPGGFfPCElSOBxWMBhUKBRSIBDwuhwAAAAAAAAAAAAkkNt9QYz081gkEvG6BAAxikQiWrZsGbkFLEJuAfuQW8BOZBewD7kF7ENuAfu4nVc6/QAgDsXFxV6XACBO5BawD7kF7ER2AfuQW8A+5BZAdej0AwAAAAAAAAAAACxHpx8AAAAAAAAAAABgOTr9POb38xIAtvD7/erevTu5BSxCbgH7kFvATmQXsA+5BexDbgH7uJ3XVFfPhoP4fD6vSwAQI5/Pp0Ag4HUZAOJAbgH7kFvATmQXsA+5BexDbgH7uN0HxC0BHotEIl6XACBGkUhECxcuJLeARcgtYB9yC9iJ7AL2IbeAfcgtYB+380qnHwDEgQ9VgH3ILWAfcgvYiewC9iG3gH3ILYDq0OkHAAAAAAAAAAAAWI5OPwAAAAAAAAAAAMByPmOM8bqIxigcDisYDKqwsFDBYNDrcgDEwBij4uJiZWZmur4AK4DaIbeAfcgtYCeyC9iH3AL2IbeAfUKhkLKzsxUKhRQIBJJ+Pkb6AUAc0tPTvS4BQJzILWAfcgvYiewC9iG3gH3ILYDq0OnnsWg06nUJAGIUjUa1cOFCcgtYhNwC9iG3gJ3ILmAfcgvYh9wC9nE7r9Z0+t17770aOXKkmjRpouzs7Er3WbduncaPH68mTZqoTZs2uummm1RWVlZhnxkzZmjIkCHKyMhQz5499cILLxx0nL/97W/q2rWrMjMzNWzYMM2ZM6fC48XFxbrmmmvUsmVLNWvWTGeffba2bNmSqKYCAAAAAAAAAAAAcbGm06+kpETnnnuurr766kofj0QiGj9+vEpKSvT5559r0qRJeuGFF3TnnXeW77NmzRqNHz9eo0aN0vz583XdddfpZz/7md57773yfV5++WXdcMMNuuuuu/TVV19p4MCBGjt2rLZu3Vq+z/XXX6///e9/evXVV/Xxxx9r06ZNOuuss5LXeAAAAAAAAAAAAKAa1nT63XPPPbr++us1YMCASh9///33tWTJEv3rX//SoEGDdPLJJ+v3v/+9/va3v6mkpESSNHHiRHXr1k1/+tOf1LdvX1177bU655xz9Mgjj5Qf589//rOuuOIKXXbZZerXr58mTpyoJk2a6LnnnpPkLLr47LPP6s9//rNOOOEEDR06VM8//7w+//xzffHFF8n/HwEAAAAAAAAAAAAcINXrAhJl1qxZGjBggNq2bVu+bezYsbr66qu1ePFiDR48WLNmzdLo0aMrPG/s2LG67rrrJDmjCefNm6dbb721/HG/36/Ro0dr1qxZkqR58+aptLS0wnH69Omjzp07a9asWRo+fHil9e3du1d79+4t/zkcDkuSjDGKRCKSJJ/PJ7/fr2g0KmNM+b77tu/br6btfr9fPp+v0u3SwXPIVrU9JSVFxphKtx9YY1XbaRNtakhtMsaoX79+MsaUH9P2NsVSO22iTTa3yRhTfsPQ/ue1uU1VbadNtKmhtGn/99tIJNIg2tQQXyfaRJt4z6VNtMn+NvGeS5tok31t2pfb6mq3rU3VbadNtKkhtGlfnW5pMJ1+eXl5FTr8JJX/nJeXV+0+4XBYe/bsUUFBgSKRSKX7LFu2rPwY6enpB60r2LZt2/LzVOb+++/XPffcc9D2JUuWqFmzZpKknJwcde7cWRs2bFB+fn75Prm5ucrNzdXatWu1c+fO8u2dOnVSy5YttWLFChUXF5dv7969uwKBgJYsWVLhH1nv3r2Vnp6uhQsXVqhhwIABKikp0fLly8u3paSkaMCAAdq5c6dWr15dvj0zM1N9+vRRQUGB1q9fX769efPm6tGjh7Zu3Vrh/wNtok0NrU2RSEQpKSkNqk0N8XWiTbRp/za1b99ehYWFFdbftb1NDfF1ok20af82lZaWKiUlpUG1qSG+TrSJNvGeS5tok/1t4j2XNtEm+9qUmZmp3r17N6g2NcTXiTbRpn1tcpvPHNit6aJbbrlFDz74YLX7LF26VH369Cn/+YUXXtB1112nwsLCCvtdeeWV+u677yqsz1dUVKSmTZtq6tSpOvnkk3XIIYfosssuqzCSb+rUqRo/fryKiopUUFCgDh066PPPP9eIESPK97n55pv18ccfa/bs2fr3v/+tyy67rMKoPUk68sgjNWrUqCrbU9lIv06dOmnbtm1q0aKFJHrEaRNtqu9tikQiWrx4sfr376+0tLQG0aZYaqdNtMnmNkUiES1ZskT9+/evcGeVzW2qajttok0NpU0lJSXl77cpKSkNok0N8XWiTbSJ91zaRJvsbxPvubSJNtnXpn3Xpg477DD5fL4G0abqttMm2tQQ2lRYWKicnByFQiEFAgElm6cj/W688UZdeuml1e4Ta09obm6u5syZU2HbvrsLc3Nzy/+7/x2H+/YJBALKyspSSkqKUlJSKt1n/2OUlJSosLCwwmi//fepTEZGhjIyMg7avu+c+9v/C9KB+7q93efzVbq9qhrj3U6baFNV2+trm/bV5fP54j5OfW1TXbbTJtqUqBrj3V6bNsWzvy1timc7baJNVW2vj23aV9P++9jeprpup020KVE1xrud99yG+zrFs502Ndw28Z5LmypDm+p3m/Zdk2pIbartdtpEm6raXt/a5KbKK3ZJ69at1adPn2r/pKenx3SsESNGaOHChdq6dWv5tmnTpikQCKhfv37l+3z44YcVnjdt2rTyUX3p6ekaOnRohX2i0ag+/PDD8n2GDh2qtLS0CvssX75c69atqzA6EAAAAAAAAAAAAHCLNWv6rVu3Tvn5+Vq3bp0ikYjmz58vSerZs6eaNWumMWPGqF+/frr44ov10EMPKS8vT7fffruuueaa8hF2V111lf7617/q5ptv1uWXX67p06frlVde0dtvv11+nhtuuEGXXHKJDj/8cB155JH6y1/+ot27d+uyyy6TJAWDQU2YMEE33HCDcnJyFAgE9Itf/EIjRozQ8OHDXf//AsBd9eFuDQDxIbeAfcgtYCeyC9iH3AL2IbcAquPpmn7xuPTSSzVp0qSDtn/00Uc6/vjjJUnfffedrr76as2YMUNNmzbVJZdcogceeECpqT/0bc6YMUPXX3+9lixZoo4dO+qOO+44aIrRv/71r3r44YeVl5enQYMG6bHHHtOwYcPKHy8uLtaNN96ol156SXv37tXYsWP1xBNPVDu954HC4bCCwaBr87gCAAAAAAAAAADAPW73BVnT6dfQ7HuhCwsLFQwGvS4HQAyMMdq5c6eaN29ePn86gPqN3AL2IbeAncguYB9yC9iH3AL2CYVCys7Odq3Tz9M1/eCsGQjADtFoVKtXrya3gEXILWAfcgvYiewC9iG3gH3ILWAft/NKpx8AAAAAAAAAAABgOTr9AAAAAAAAAAAAAMvR6QcAccjMzPS6BABxIreAfcgtYCeyC9iH3AL2IbcAquMzxhivi2iMwuGwgsGga4s3AgAAAAAAAAAAwD1u9wUx0s9jLLoK2CMajWrHjh3kFrAIuQXsQ24BO5FdwD7kFrAPuQXs43Ze6fTzGAMtAXsYY7R+/XpyC1iE3AL2IbeAncguYB9yC9iH3AL2cTuvdPoBAAAAAAAAAAAAlqPTDwAAAAAAAAAAALAcnX4AEIfmzZt7XQKAOJFbwD7kFrAT2QXsQ24B+5BbANXxGSYA9kQ4HFYwGFQoFFIgEPC6HAAAAAAAAAAAACSQ231BjPTzWDQa9boEADGKRqPKy8sjt4BFyC1gH3IL2InsAvYht4B9yC1gH7fzSqefxxhoCdjDGKO8vDxyC1iE3AL2IbeAncguYB9yC9iH3AL2cTuvdPoBAAAAAAAAAAAAlqPTDwAAAAAAAAAAALAcnX4e8/l8XpcAIEY+n085OTnkFrAIuQXsQ24BO5FdwD7kFrAPuQXs43ZefYYJgD0RDocVDAYVCoUUCAS8LgcAAAAAAAAAAAAJ5HZfECP9PBaNRr0uAUCMotGo1q1bR24Bi5BbwD7kFrAT2QXsQ24B+5BbwD5u55VOP48x0BKwhzFG+fn55BawCLkF7ENuATuRXcA+5BawD7kF7ON2Xun0AwAAAAAAAAAAACxHpx8AAAAAAAAAAABgOTr9PObz+bwuAUCMfD6fcnNzyS1gEXIL2IfcAnYiu4B9yC1gH3IL2MftvPoMEwB7IhwOKxgMKhQKKRAIeF0OAAAAAAAAAAAAEsjtviBG+nksEol4XQKAGEUiEa1atYrcAhYht4B9yC1gJ7IL2IfcAvYht4B93M4rnX4AEIedO3d6XQKAOJFbwD7kFrAT2QXsQ24B+5BbANWh0w8AAAAAAAAAAACwHJ1+AAAAAAAAAAAAgOXo9POYz+fzugQAMfL5fOrUqRO5BSxCbgH7kFvATmQXsA+5BexDbgH7uJ1XnzHGuHpGSJLC4bCCwaBCoZACgYDX5QAAAAAAAAAAACCB3O4LYqSfxyKRiNclAIhRJBLRsmXLyC1gEXIL2IfcAnYiu4B9yC1gH3IL2MftvNLpBwBxKC4u9roEAHEit4B9yC1gJ7IL2IfcAvYhtwCqQ6cfAAAAAAAAAAAAYDk6/QAAAAAAAAAAAADL0ennMb+flwCwhd/vV/fu3cktYBFyC9iH3AJ2IruAfcgtYB9yC9jH7bymuno2HMTn83ldAoAY+Xw+BQIBr8sAEAdyC9iH3AJ2IruAfcgtYB9yC9jH7T4gbgnwWCQS8boEADGKRCJauHAhuQUsQm4B+5BbwE5kF7APuQXsQ24B+7idVzr9ACAOfKgC7ENuAfuQW8BOZBewD7kF7ENuAVSHTj8AAAAAAAAAAADAcnT6AQAAAAAAAAAAAJbzGWOM10U0RuFwWMFgUIWFhQoGg16XAyAGxhgVFxcrMzPT9QVYAdQOuQXsQ24BO5FdwD7kFrAPuQXsEwqFlJ2drVAopEAgkPTzMdIPAOKQnp7udQkA4kRuAfuQW8BOZBewD7kF7ENuAVSHTj+PRaNRr0sAEKNoNKqFCxeSW8Ai5BawD7kF7ER2AfuQW8A+5Bawj9t5pdMPAAAAAAAAAAAAsBydfgAAAAAAAAAAAIDl6PQDAAAAAAAAAAAALOczxhivi2iMwuGwgsGgCgsLFQwGvS4HQAyMMYpGo/L7/fL5fF6XAyAG5BawD7kF7ER2AfuQW8A+5BawTygUUnZ2tkKhkAKBQNLPx0g/AIhDSUmJ1yUAiBO5BexDbgE7kV3APuQWsA+5BVAdOv08Fo1GvS4BQIyi0aiWL19ObgGLkFvAPuQWsBPZBexDbgH7kFvAPm7nlU4/AAAAAAAAAAAAwHJ0+gEAAAAAAAAAAACWo9MPAOKQkpLidQkA4kRuAfuQW8BOZBewD7kF7ENuAVTHZ4wxXhfRGIXDYQWDQYVCIQUCAa/LAQAAAAAAAAAAQAK53RfESD+P0ecK2MMYo3A4TG4Bi5BbwD7kFrAT2QXsQ24B+5BbwD5u55VOP49Fo1GvSwAQo2g0qtWrV5NbwCLkFrAPuQXsRHYB+5BbwD7kFrCP23ml0w8AAAAAAAAAAACwHJ1+AAAAAAAAAAAAgOXo9AOAOGRmZnpdAoA4kVvAPuQWsBPZBexDbgH7kFsA1fEZVv30RDgcVjAYVCgUUiAQ8LocAAAAAAAAAAAAJJDbfUGM9PMYi64C9ohGo9qxYwe5BSxCbgH7kFvATmQXsA+5BexDbgH7uJ1XOv08xkBLwB7GGK1fv57cAhYht4B9yC1gJ7IL2IfcAvYht4B93M4rnX4AAAAAAAAAAACA5ej0AwAAAAAAAAAAACxHpx8AxKF58+ZelwAgTuQWsA+5BexEdgH7kFvAPuQWQHV8hgmAPREOhxUMBhUKhRQIBLwuBwAAAAAAAAAAAAnkdl8QI/08Fo1GvS4BQIyi0ajy8vLILWARcgvYh9wCdiK7gH3ILWAfcgvYx+280unnMQZaAvYwxigvL4/cAhYht4B9yC1gJ7IL2IfcAvYht4B93M4rnX4AAAAAAAAAAACA5ej0AwAAAAAAAAAAACxHp5/HfD6f1yUAiJHP51NOTg65BSxCbgH7kFvATmQXsA+5BexDbgH7uJ1Xn2ECYE+Ew2EFg0GFQiEFAgGvywEAAAAAAAAAAEACud0XxEg/j0WjUa9LABCjaDSqdevWkVvAIuQWsA+5BexEdgH7kFvAPuQWsI/beaXTz2MMtATsYYxRfn4+uQUsQm4B+5BbwE5kF7APuQXsQ24B+7idVzr9AAAAAAAAAAAAAMvR6QcAAAAAAAAAAABYjk4/j/l8Pq9LABAjn8+n3NxccgtYhNwC9iG3gJ3ILmAfcgvYh9wC9nE7rz7DBMCeCIfDCgaDCoVCCgQCXpcDAAAAAAAAAACABHK7L4iRfh6LRCJelwAgRpFIRKtWrSK3gEXILWAfcgvYiewC9iG3gH3ILWAft/NKpx8AxGHnzp1elwAgTuQWsA+5BexEdgH7kFvAPuQWQHXo9AMAAAAAAAAAAAAsR6cfAAAAAAAAAAAAYDk6/Tzm8/m8LgFAjHw+nzp16kRuAYuQW8A+5BawE9kF7ENuAfuQW8A+bufVZ4wxrp4RkqRwOKxgMKhQKKRAIOB1OQAAAAAAAAAAAEggt/uCGOnnsUgk4nUJAGIUiUS0bNkycgtYhNwC9iG3gJ3ILmAfcgvYh9wC9nE7r3T6AUAciouLvS4BQJzILWAfcgvYiewC9iG3gH3ILYDq0OkHAAAAAAAAAAAAWI5OPwAAAAAAAAAAAMBydPp5zO/nJQBs4ff71b17d3ILWITcAvYht4CdyC5gH3IL2IfcAvZxO6+prp4NB/H5fF6XACBGPp9PgUDA6zIAxIHcAvYht4CdyC5gH3IL2IfcAvZxuw+IWwI8FolEvC4BQIwikYgWLlxIbgGLkFvAPuQWsBPZBexDbgH7kFvAPm7nlU4/AIgDH6oA+5BbwD7kFrAT2QXsQ24B+5BbANWh0w8AAAAAAAAAAACwHJ1+AAAAAAAAAAAAgOV8xhjjdRGNUTgcVjAYVGFhoYLBoNflAIiBMUbFxcXKzMx0fQFWALVDbgH7kFvATmQXsA+5BexDbgH7hEIhZWdnKxQKKRAIJP18jPQDgDikp6d7XQKAOJFbwD7kFrAT2QXsQ24B+5BbANWh089j0WjU6xIAxCgajWrhwoXkFrAIuQXsQ24BO5FdwD7kFrAPuQXs43Ze6fQDAAAAAAAAAAAALEenHwAAAAAAAAAAAGA5Ov0AAAAAAAAAAAAAy/mMMcbrIhqjcDisYDCowsJCBYNBr8sBEANjjKLRqPx+v3w+n9flAIgBuQXsQ24BO5FdwD7kFrAPuQXsEwqFlJ2drVAopEAgkPTzMdIPAOJQUlLidQkA4kRuAfuQW8BOZBewD7kF7ENuAVSHTj+PRaNRr0sAEKNoNKrly5eTW8Ai5BawD7kF7ER2AfuQW8A+5Bawj9t5pdMPAAAAAAAAAAAAsBydfgAAAAAAAAAAAIDlrOj0W7t2rSZMmKBu3bopKytLPXr00F133XXQ/MULFizQMccco8zMTHXq1EkPPfTQQcd69dVX1adPH2VmZmrAgAGaOnVqhceNMbrzzjvVrl07ZWVlafTo0VqxYkWFffLz83XRRRcpEAgoOztbEyZM0K5duxLfcAD1TkpKitclAIgTuQXsQ24BO5FdwD7kFrAPuQVQHSs6/ZYtW6ZoNKqnnnpKixcv1iOPPKKJEyfqtttuK98nHA5rzJgx6tKli+bNm6eHH35Yd999t55++unyfT7//HNdeOGFmjBhgr7++mudccYZOuOMM7Ro0aLyfR566CE99thjmjhxombPnq2mTZtq7NixKi4uLt/noosu0uLFizVt2jRNmTJFn3zyia688spatY1f0oA9UlJSNGDAAHILWITcAvYht4CdyC5gH3IL2IfcAvZxO68+Y4xx9YwJ8vDDD+vJJ5/U6tWrJUlPPvmkfvvb3yovL0/p6emSpFtuuUWTJ0/WsmXLJEnnn3++du/erSlTppQfZ/jw4Ro0aJAmTpwoY4zat2+vG2+8Ub/+9a8lSaFQSG3bttULL7ygCy64QEuXLlW/fv305Zdf6vDDD5ckvfvuuzrllFO0YcMGtW/fvtJ69+7dq71795b/HA6H1alTJ+3YsUPBYFCS5PP55Pf7FY1Gtf/Lsm97JBKpcMyqtvv9fvl8vkq3SwcvHFnV9pSUFBljKt1+YI1VbadNtKkhtckYo507d6p58+blv6xtb1MstdMm2mRzm4wxKioqUtOmTSut0cY2VbWdNtGmhtKmsrKy8vfbffva3qaG+DrRJtrEey5tok32t4n3XNpEm+xrkzFGu3btUjAYlDGmQbSpuu20iTY1hDaFw2FlZ2crFAopEAgo2VKTfoYkCYVCysnJKf951qxZOvbYY8s7/CRp7NixevDBB1VQUKAWLVpo1qxZuuGGGyocZ+zYsZo8ebIkac2aNcrLy9Po0aPLHw8Ggxo2bJhmzZqlCy64QLNmzVJ2dnZ5h58kjR49Wn6/X7Nnz9aZZ55Zab3333+/7rnnnoO2L168WM2bN5ck5eTkqHPnztqwYYPy8/PL98nNzVVubq7Wrl2rnTt3lm/v1KmTWrZsqRUrVlQYidi9e3cFAgEtWbKkwj+y3r17Kz09XQsXLqxQw4ABA1RSUqLly5eXb9t318jOnTvLO1YlKTMzU3369FFBQYHWr19fvr158+bq0aOHtm7dqry8vPLttIk2NaQ2lZWVKT8/Xzk5OerTp0+DaFNDfJ1oE23av01NmzbV7t271aZNG23durVBtKkhvk60iTbt36bFixdr+/btysnJkc/naxBtaoivE22iTbzn0ibaZH+beM+lTbTJvjbtuyH9qKOO0rZt2xpEm6SG9zrRJtq0f5sO7HxMNitH+q1cuVJDhw7VH//4R11xxRWSpDFjxqhbt2566qmnyvdbsmSJ+vfvryVLlqhv375KT0/XpEmTdOGFF5bv88QTT+iee+7Rli1b9Pnnn+uoo47Spk2b1K5du/J9zjvvPPl8Pr388su67777NGnSpAr/kCSpTZs2uueee3T11VdXWnNVI/22bdumFi1aSKJHnDbRpvrepkgkosWLF6t///5KS0trEG2KpXbaRJtsblMkEin/PLDveLa3qarttIk2NZQ2lZSUlL/fpqSkNIg2NcTXiTbRJt5zaRNtsr9NvOfSJtpkX5v2XZs67LDD5PP5GkSbqttOm2hTQ2hTYWGhcnJyGsdIv1tuuUUPPvhgtfssXbpUffr0Kf9548aNGjdunM4999zyDj8bZGRkKCMj46DtKSkpSkmpOKfr/l+QDtzX7e0+n6/S7VXVGO922kSbqtpeX9u0ry6fzxf3ceprm+qynTbRpkTVGO/22rQpnv1taVM822kTbapqe31s076a9t/H9jbVdTttok2JqjHe7bznNtzXKZ7ttKnhton3XNpUGdpUv9u075pUQ2pTbbfTJtpU1fb61iY3edrpd+ONN+rSSy+tdp/u3buX/33Tpk0aNWqURo4cqaeffrrCfrm5udqyZUuFbft+zs3NrXaf/R/ft23/kX5btmzRoEGDyvfZf6oSSeVT/u17PoCGKzMz0+sSAMSJ3AL2IbeAncguYB9yC9iH3AKoTuXdlC5p3bq1+vTpU+2ffWv0bdy4Uccff7yGDh2q559//qAe1hEjRuiTTz5RaWlp+bZp06apd+/e5dNnjhgxQh9++GGF502bNk0jRoyQJHXr1k25ubkV9gmHw5o9e3b5PiNGjFBhYaHmzZtXvs/06dMVjUY1bNiwuP8f1IeeXwCxSUlJUZ8+fcgtYBFyC9iH3AJ2IruAfcgtYB9yC9jH7bx62ukXq30dfp07d9Yf//jH8kVK919Y8cc//rHS09M1YcIELV68WC+//LIeffRR3XDDDeX7/OpXv9K7776rP/3pT1q2bJnuvvtuzZ07V9dee60kZ8jnddddpz/84Q966623tHDhQv30pz9V+/btdcYZZ0iS+vbtq3HjxumKK67QnDlz9Nlnn+naa6/VBRdcoPbt28fdtgPnlQVQf0WjUe3YsYPcAhYht4B9yC1gJ7IL2IfcAvYht4B93M6rp9N7xmratGlauXKlVq5cqY4dO1Z4bN8iisFgUO+//76uueYaDR06VK1atdKdd96pK6+8snzfkSNH6t///rduv/123XbbberVq5cmT56sQw89tHyfm2++Wbt379aVV16pwsJCHX300Xr33XcrDJt+8cUXde211+rEE0+U3+/X2Wefrccee6xWbTtw0UgA9ZcxRuvXr1d2drbXpQCIEbkF7ENuATuRXcA+5BawD7kF7ON2H5DP0OvkiXA4rGAwqPz8/PLpRwHUb5FIRAsXLtSAAQOYRgGwBLkF7ENuATuRXcA+5BawD7kF7FNQUKCcnByFQiEFAoGkn8+K6T0BAAAAAAAAAAAAVI1OPwCIQ/Pmzb0uAUCcyC1gH3IL2InsAvYht4B9yC2A6jC9p0f2Te/p1pBOAAAAAAAAAAAAuMftviBG+nksGo16XQKAGEWjUeXl5ZFbwCLkFrAPuQXsRHYB+5BbwD7kFrCP23ml089jDLQE7GGMUV5eHrkFLEJuAfuQW8BOZBewD7kF7ENuAfu4nVc6/QAAAAAAAAAAAADL0ekHAAAAAAAAAAAAWI5OP4/5fD6vSwAQI5/Pp5ycHHILWITcAvYht4CdyC5gH3IL2IfcAvZxO68+wwTAngiHwwoGgwqFQgoEAl6XAwAAAAAAAAAAgARyuy+IkX4ei0ajXpcAIEbRaFTr1q0jt4BFyC1gH3IL2InsAvYht4B9yC1gH7fzSqefxxhoCdjDGKP8/HxyC1iE3AL2IbeAncguYB9yC9iH3AL2cTuvdPoBAAAAAAAAAAAAlkv1uoDGal/vbjgcVkpKisfVAIhFJBLRrl27yC1gEXIL2IfcAnYiu4B9yC1gH3IL2CccDktyb8QfnX4e2bFjhySpa9eu3hYCAAAAAAAAAACApNmxY4eCwWDSz0Onn0dycnIkSevWrXPlhQZQd+FwWJ06ddL69esVCAS8LgdADMgtYB9yC9iJ7AL2IbeAfcgtYJ9QKKTOnTuX9wklG51+HvH7neUUg8Egv6ABywQCAXILWIbcAvYht4CdyC5gH3IL2IfcAvbZ1yeU9PO4chYAAAAAAAAAAAAASUOnHwAAAAAAAAAAAGA5Ov08kpGRobvuuksZGRlelwIgRuQWsA+5BexDbgE7kV3APuQWsA+5Bezjdm59xhjjypkAAAAAAAAAAAAAJAUj/QAAAAAAAAAAAADL0ekHAAAAAAAAAAAAWI5OPwAAAAAAAAAAAMBydPoBAAAAAAAAAAAAlqPTr5YikYjuuOMOdevWTVlZWerRo4d+//vfyxhTvs/dd9+tPn36qGnTpmrRooVGjx6t2bNnVzhOfn6+LrroIgUCAWVnZ2vChAnatWtXhX0WLFigY445RpmZmerUqZMeeughV9oINDSx5HZ/V111lXw+n/7yl79U2E5uAXfFkt1LL71UPp+vwp9x48ZVOA7ZBdwT63vu0qVLddpppykYDKpp06Y64ogjtG7duvLHi4uLdc0116hly5Zq1qyZzj77bG3ZsqXCMdatW6fx48erSZMmatOmjW666SaVlZW50k6gIYkltwe+1+778/DDD5fvw/st4J5Ycrtr1y5de+216tixo7KystSvXz9NnDixwnF4vwXcFUt2t2zZoksvvVTt27dXkyZNNG7cOK1YsaLCccgu4K6dO3fquuuuU5cuXZSVlaWRI0fqyy+/LH/cGKM777xT7dq1U1ZWlkaPHn1Qbl35rGxQK/fee69p2bKlmTJlilmzZo159dVXTbNmzcyjjz5avs+LL75opk2bZlatWmUWLVpkJkyYYAKBgNm6dWv5PuPGjTMDBw40X3zxhfn0009Nz549zYUXXlj+eCgUMm3btjUXXXSRWbRokXnppZdMVlaWeeqpp1xtL9AQxJLbfV5//XUzcOBA0759e/PII49UeIzcAu6KJbuXXHKJGTdunNm8eXP5n/z8/ArHIbuAe2LJ7cqVK01OTo656aabzFdffWVWrlxp3nzzTbNly5byfa666irTqVMn8+GHH5q5c+ea4cOHm5EjR5Y/XlZWZg499FAzevRo8/XXX5upU6eaVq1amVtvvdXV9gINQSy53f99dvPmzea5554zPp/PrFq1qnwf3m8B98SS2yuuuML06NHDfPTRR2bNmjXmqaeeMikpKebNN98s34f3W8BdNWU3Go2a4cOHm2OOOcbMmTPHLFu2zFx55ZWmc+fOZteuXeXHIbuAu8477zzTr18/8/HHH5sVK1aYu+66ywQCAbNhwwZjjDEPPPCACQaDZvLkyeabb74xp512munWrZvZs2dP+THc+KxMp18tjR8/3lx++eUVtp111lnmoosuqvI5oVDISDIffPCBMcaYJUuWGEnmyy+/LN/nnXfeMT6fz2zcuNEYY8wTTzxhWrRoYfbu3Vu+z29+8xvTu3fvRDYHaBRize2GDRtMhw4dzKJFi0yXLl0qdPqRW8B9sWT3kksuMaeffnqVxyC7gLtiye35559vfvKTn1R5jMLCQpOWlmZeffXV8m1Lly41ksysWbOMMcZMnTrV+P1+k5eXV77Pk08+aQKBQIUsA6hZbb7jnn766eaEE04o/5n3W8BdseS2f//+5ne/+12FfYYMGWJ++9vfGmN4vwW8UFN2ly9fbiSZRYsWlT8eiURM69atzTPPPGOMIbuA24qKikxKSoqZMmVKhe373lOj0ajJzc01Dz/8cPljhYWFJiMjw7z00kvGGPc+KzO9Zy2NHDlSH374ob799ltJ0jfffKOZM2fq5JNPrnT/kpISPf300woGgxo4cKAkadasWcrOztbhhx9evt/o0aPl9/vLpwGdNWuWjj32WKWnp5fvM3bsWC1fvlwFBQXJah7QIMWS22g0qosvvlg33XST+vfvf9AxyC3gvljfc2fMmKE2bdqod+/euvrqq7Vjx47yx8gu4K6achuNRvX222/rkEMO0dixY9WmTRsNGzZMkydPLj/GvHnzVFpaqtGjR5dv69Onjzp37qxZs2ZJcnI7YMAAtW3btnyfsWPHKhwOa/HixS60FGg44v2Ou2XLFr399tuaMGFC+TbebwF3xZLbkSNH6q233tLGjRtljNFHH32kb7/9VmPGjJHE+y3ghZqyu3fvXklSZmZm+XP8fr8yMjI0c+ZMSWQXcFtZWZkikUiFXEpSVlaWZs6cqTVr1igvL69CJoPBoIYNG1Yhk258Vk6tdSsbuVtuuUXhcFh9+vRRSkqKIpGI7r33Xl100UUV9psyZYouuOACFRUVqV27dpo2bZpatWolScrLy1ObNm0q7J+amqqcnBzl5eWV79OtW7cK++z7RZ2Xl6cWLVokq4lAgxNLbh988EGlpqbql7/8ZaXHILeA+2LJ7rhx43TWWWepW7duWrVqlW677TadfPLJmjVrllJSUsgu4LKacrt161bt2rVLDzzwgP7whz/owQcf1LvvvquzzjpLH330kY477jjl5eUpPT1d2dnZFY7dtm3bCrnd/yLGvsf3PQYgdrF+x91n0qRJat68uc4666zybbzfAu6KJbePP/64rrzySnXs2FGpqany+/165plndOyxx0oS77eAB2rK7r7Ou1tvvVVPPfWUmjZtqkceeUQbNmzQ5s2bJZFdwG3NmzfXiBEj9Pvf/159+/ZV27Zt9dJLL2nWrFnq2bNneaYqy9z+mXTjszKdfrX0yiuv6MUXX9S///1v9e/fX/Pnz9d1112n9u3b65JLLinfb9SoUZo/f762b9+uZ555Ruedd55mz5590IsLIPlqyu28efP06KOP6quvvpLP5/O6XADfi+U994ILLijff8CAATrssMPUo0cPzZgxQyeeeKJXpQONVk25jUajkqTTTz9d119/vSRp0KBB+vzzzzVx4kQdd9xxXpYPNEqxfsfd57nnntNFF1100N3OANwTS24ff/xxffHFF3rrrbfUpUsXffLJJ7rmmmvUvn37CqMRALinpuympaXp9ddf14QJE5STk6OUlBSNHj1aJ598sowxXpcPNFr//Oc/dfnll6tDhw5KSUnRkCFDdOGFF2revHlel1YBnX61dNNNN+mWW24pv8g4YMAAfffdd7r//vsrfCFq2rSpevbsqZ49e2r48OHq1auXnn32Wd16663Kzc3V1q1bKxy3rKxM+fn5ys3NlSTl5uZqy5YtFfbZ9/O+fQDEpqbcfvrpp9q6das6d+5c/pxIJKIbb7xRf/nLX7R27VpyC3gg1vfc/XXv3l2tWrXSypUrdeKJJ5JdwGU15bZVq1ZKTU1Vv379Kjyvb9++5VMW5ebmqqSkRIWFhRXuYN6yZUuF3M6ZM6fCMcgtUDvxvN9++umnWr58uV5++eUK23m/BdxVU2737Nmj2267TW+88YbGjx8vSTrssMM0f/58/fGPf9To0aN5vwU8EMt77tChQzV//nyFQiGVlJSodevWGjZsWPm0gGQXcF+PHj308ccfa/fu3QqHw2rXrp3OP/98de/evTxTW7ZsUbt27cqfs2XLFg0aNEiSe5+VWdOvloqKiuT3V/zfl5KSUn7XclWi0Wj5vMwjRoxQYWFhhZ7g6dOnKxqNatiwYeX7fPLJJyotLS3fZ9q0aerduzfTngBxqim3F198sRYsWKD58+eX/2nfvr1uuukmvffee5LILeCF2rznbtiwQTt27Cj/oEV2AXfVlNv09HQdccQRWr58eYV9vv32W3Xp0kWSc6EjLS1NH374Yfnjy5cv17p16zRixAhJTm4XLlxY4YvTtGnTFAgEDupQBFC9eN5vn332WQ0dOrR8vfp9eL8F3FVTbktLS1VaWlrtPrzfAu6L5z03GAyqdevWWrFihebOnavTTz9dEtkFvNS0aVO1a9dOBQUFeu+993T66aerW7duys3NrZDJcDis2bNnV8ikK5+VDWrlkksuMR06dDBTpkwxa9asMa+//rpp1aqVufnmm40xxuzatcvceuutZtasWWbt2rVm7ty55rLLLjMZGRlm0aJF5ccZN26cGTx4sJk9e7aZOXOm6dWrl7nwwgvLHy8sLDRt27Y1F198sVm0aJH5z3/+Y5o0aWKeeuop19sM2K6m3FamS5cu5pFHHqmwjdwC7qopuzt37jS//vWvzaxZs8yaNWvMBx98YIYMGWJ69epliouLy49DdgH3xPKe+/rrr5u0tDTz9NNPmxUrVpjHH3/cpKSkmE8//bR8n6uuusp07tzZTJ8+3cydO9eMGDHCjBgxovzxsrIyc+ihh5oxY8aY+fPnm3fffde0bt3a3Hrrra62F2gIYv2sHAqFTJMmTcyTTz5Z6XF4vwXcE0tujzvuONO/f3/z0UcfmdWrV5vnn3/eZGZmmieeeKJ8H95vAXfFkt1XXnnFfPTRR2bVqlVm8uTJpkuXLuass86qcByyC7jr3XffNe+8845ZvXq1ef/9983AgQPNsGHDTElJiTHGmAceeMBkZ2ebN9980yxYsMCcfvrpplu3bmbPnj3lx3DjszKdfrUUDofNr371K9O5c2eTmZlpunfvbn7729+avXv3GmOM2bNnjznzzDNN+/btTXp6umnXrp057bTTzJw5cyocZ8eOHebCCy80zZo1M4FAwFx22WVm586dFfb55ptvzNFHH20yMjJMhw4dzAMPPOBaO4GGpKbcVqayTj9yC7irpuwWFRWZMWPGmNatW5u0tDTTpUsXc8UVV5i8vLwKxyG7gHtifc999tlnTc+ePU1mZqYZOHCgmTx5coXH9+zZY37+85+bFi1amCZNmpgzzzzTbN68ucI+a9euNSeffLLJysoyrVq1MjfeeKMpLS1NehuBhibW3D711FMmKyvLFBYWVnoc3m8B98SS282bN5tLL73UtG/f3mRmZprevXubP/3pTyYajZbvw/st4K5Ysvvoo4+ajh07mrS0NNO5c2dz++23H/SeTHYBd7388sume/fuJj093eTm5pprrrmmwmfiaDRq7rjjDtO2bVuTkZFhTjzxRLN8+fIKx3Djs7LPGFb/BAAAAAAAAAAAAGzGmn4AAAAAAAAAAACA5ej0AwAAAAAAAAAAACxHpx8AAAAAAAAAAABgOTr9AAAAAAAAAAAAAMvR6QcAAAAAAAAAAABYjk4/AAAAAAAAAAAAwHJ0+gEAAAAAAAAAAACWo9MPAAAAAAAAAAAAsBydfgAAAADQCF166aU644wzvC6j1myvHwAAAAASLdXrAgAAAAAAieXz+ap9/K677tKjjz4qY4xLFf1gxowZGjVqlAoKCpSdne36+QEAAACgoaLTDwAAAAAamM2bN5f//eWXX9add96p5cuXl29r1qyZmjVr5kVpAAAAAIAkYXpPAAAAAGhgcnNzy/8Eg0H5fL4K25o1a3bQ9JjHH3+8fvGLX+i6665TixYt1LZtWz3zzDPavXu3LrvsMjVv3lw9e/bUO++8U+FcixYt0sknn6xmzZqpbdu2uvjii7V9+/aYa33hhReUnZ2t9957T3379lWzZs00bty4Ch2XkUhEN9xwg7Kzs9WyZUvdfPPNB41SjEajuv/++9WtWzdlZWVp4MCB+u9//ytJMsZo9OjRGjt2bPnz8vPz1bFjR915553x/u8FAAAAgHqJTj8AAAAAgCRp0qRJatWqlebMmaNf/OIXuvrqq3Xuuedq5MiR+uqrrzRmzBhdfPHFKioqkiQVFhbqhBNO0ODBgzV37ly9++672rJli84777y4zltUVKQ//vGP+uc//6lPPvlE69at069//evyx//0pz/phRde0HPPPaeZM2cqPz9fb7zxRoVj3H///frHP/6hiRMnavHixbr++uv1k5/8RB9//LF8Pp8mTZqkL7/8Uo899pgk6aqrrlKHDh3o9AMAAADQYDC9JwAAAABAkjRw4EDdfvvtkqRbb71VDzzwgFq1aqUrrrhCknTnnXfqySef1IIFCzR8+HD99a9/1eDBg3XfffeVH+O5555Tp06d9O233+qQQw6J6bylpaWaOHGievToIUm69tpr9bvf/a788b/85S+69dZbddZZZ0mSJk6cqPfee6/88b179+q+++7TBx98oBEjRkiSunfvrpkzZ+qpp57Scccdpw4dOuipp57ST3/6U+Xl5Wnq1Kn6+uuvlZrK12IAAAAADQPfbgAAAAAAkqTDDjus/O8pKSlq2bKlBgwYUL6tbdu2kqStW7dKkr755ht99NFHla4PuGrVqpg7/Zo0aVLe4SdJ7dq1Kz9HKBTS5s2bNWzYsPLHU1NTdfjhh5dP1bly5UoVFRXppJNOqnDckpISDR48uPznc889V2+88YYeeOABPfnkk+rVq1dM9QEAAACADej0AwAAAABIktLS0ir87PP5Kmzz+XySnPXzJGnXrl069dRT9eCDDx50rHbt2tXpvAeu2VedXbt2SZLefvttdejQocJjGRkZ5X8vKirSvHnzlJKSohUrVsR8fAAAAACwAZ1+AAAAAIBaGTJkiF577TV17do1adNkBoNBtWvXTrNnz9axxx4rSSorK9O8efM0ZMgQSVK/fv2UkZGhdevW6bjjjqvyWDfeeKP8fr/eeecdnXLKKRo/frxOOOGEpNQNAAAAAG7ze10AAAAAAMBO11xzjfLz83XhhRfqyy+/1KpVq/Tee+/psssuUyQSSdh5fvWrX+mBBx7Q5MmTtWzZMv385z9XYWFh+ePNmzfXr3/9a11//fWaNGmSVq1apa+++kqPP/64Jk2aJMkZBfjcc8/pxRdf1EknnaSbbrpJl1xyiQoKChJWJwAAAAB4iU4/AAAAAECttG/fXp999pkikYjGjBmjAQMG6LrrrlN2drb8/sR93bzxxht18cUX65JLLtGIESPUvHlznXnmmRX2+f3vf6877rhD999/v/r27atx48bp7bffVrdu3bRt2zZNmDBBd999d/nowHvuuUdt27bVVVddlbA6AQAAAMBLPhPPQgkAAAAAAAAAAAAA6h1G+gEAAAAAAAAAAACWo9MPAAAAAAAAAAAAsBydfgAAAAAAAAAAAIDl6PQDAAAAAAAAAAAALEenHwAAAAAAAAAAAGA5Ov0AAAAAAAAAAAAAy9HpBwAAAAAAAAAAAFiOTj8AAAAAAAAAAADAcnT6AQAAAAAAAAAAAJaj0w8AAAAAAAAAAACwHJ1+AAAAAAAAAAAAgOX+H9r/l9g1Qs3SAAAAAElFTkSuQmCC\n" + }, + "metadata": {} + }, + { + "output_type": "stream", + "name": "stdout", + "text": [ + "948\n" + ] + }, + { + "output_type": "display_data", + "data": { + "text/plain": [ + "
" + ], + "image/png": "iVBORw0KGgoAAAANSUhEUgAABv4AAAN5CAYAAADAfkzvAAAAOnRFWHRTb2Z0d2FyZQBNYXRwbG90bGliIHZlcnNpb24zLjEwLjAsIGh0dHBzOi8vbWF0cGxvdGxpYi5vcmcvlHJYcgAAAAlwSFlzAAAPYQAAD2EBqD+naQABAABJREFUeJzs3XdYFFfbBvB7lrp0QQRRmqKIir3GKHaNvceusb5RYjfqq7HEHmsSY6+xx5LExK4Re4kFxY4oYkNUBKTD7vn+8GNfV4qgAyPr/bsurmTPnDnznJ3hYeVhzkhCCAEiIiIiIiIiIiIiIiIiytdUSgdARERERERERERERERERB+OhT8iIiIiIiIiIiIiIiIiA8DCHxEREREREREREREREZEBYOGPiIiIiIiIiIiIiIiIyACw8EdERERERERERERERERkAFj4IyIiIiIiIiIiIiIiIjIALPwRERERERERERERERERGQAW/oiIiIiIiIiIiIiIiIgMAAt/RERERERERERERERERAaAhT8iIiIiMgi9e/eGh4eH0mEo5lOff25au3YtJElCaGio0qF89EJDQyFJEtauXatYDM2aNUP//v0VO/7HJiAgAJIkISAgQOlQKJ9LSUmBq6srFi9erHQoRERERJQFFv6IiIiI6KMlSVK2vj7WX2iHhobiq6++QvHixWFubg5nZ2fUqVMHkyZNUjo02VWrVg2SJGHJkiVKh/LR6927t971a2VlhWLFiqFDhw7YsWMHtFrte4+9Z88eTJ48Wb5gM7Fp0yYsXLgw14+TUydPnsSBAwcwZswYXZuHh8c7c0jv3r2VC/ojkVbgPn/+fK4fKzY2FsOGDUPRokVhZmYGHx+fTHNHVFQUBgwYAEdHR1haWqJevXq4ePFihn137dqFSpUqwdzcHG5ubpg0aRJSU1M/aMy3LV68OFcK2ykpKZgyZQqKFSsGMzMzFCtWDNOmTcswfgC4ePEiWrVqBXt7e1hYWKBs2bL46aef0vU7deoUPv/8c1hYWMDZ2RlDhgxBbGxsun5JSUkYM2YMXFxcoFarUb16dRw8eFCvj4mJCUaMGIHp06cjMTFRnokTERERkeyMlQ6AiIiIiCgz69ev13v966+/4uDBg+nafXx8sGLFig8qmMjtzp07qFq1KtRqNfr06QMPDw88efIEFy9exOzZszFlyhRZj6fk/IODg/Hvv//Cw8MDGzduxNdff61IHPmJmZkZVq5cCQBISEjA/fv38ddff6FDhw6oW7cu/vzzT9jY2OR43D179uCXX37J9eLfpk2bcPXqVQwbNkyv3d3dHQkJCTAxMcnV42dmzpw5aNCgAby8vHRtCxcuzLDQAQCLFi3C2bNnUaNGjbwK8ZOn0WjQpEkTnD9/HoMHD0aJEiWwf/9+DBo0CC9fvsR///tfXV+tVovmzZvj8uXLGD16NAoWLIjFixejbt26uHDhAkqUKKHru3fvXrRp0wZ169bFzz//jKCgIEybNg0RERF6RcWcjJmRxYsXo2DBgrIXi7t3745t27ahT58+qFKlCs6cOYPvvvsOYWFhWL58uV7fAwcOoGXLlqhYsSK+++47WFlZISQkBA8fPtTrFxgYiAYNGsDHxwfz58/Hw4cPMXfuXAQHB2Pv3r16fXv37o3t27dj2LBhKFGiBNauXYtmzZrhyJEj+Pzzz3X9vvrqK4wdOxabNm1Cnz59ZH0PiIiIiEgmgoiIiIgonxg8eLDILx9hBw0aJIyNjUVoaGi6bU+fPpXtOLGxsbKN9b4mTpwoChUqJHbs2CEkSRL37t1TOiRZrVmzRgCQbV69evUSlpaWGW6bOXOmACA6der0XmPn1fdI8+bNhbu7e64fJyeePn0qjI2NxcqVK7PVf//+/UKSJNGqVatcjkxZR44cEQDEkSNHsuyXdp3/+++/uRrPb7/9JgCIVatW6bW3b99emJub6+XHrVu3CgBi27ZturaIiAhhZ2cnunTpord/6dKlRfny5UVKSoqubfz48UKSJHHjxo33GjMjZcqUEX5+ftmeb3acO3dOABDfffedXvvIkSOFJEni8uXLurbo6Gjh5OQk2rZtKzQaTZbjfvHFF6Jw4cIiOjpa17ZixQoBQOzfv1/XdvbsWQFAzJkzR9eWkJAgihcvLmrWrJlu3BYtWojatWvneJ5ERERElDe41CcRERERGYS3n3GX9qyxuXPn4pdffkGxYsVgYWGBxo0b48GDBxBCYOrUqShatCjUajVat26NyMjIdOPu3bsXtWvXhqWlJaytrdG8eXNcu3btnfGEhISgaNGicHd3T7etUKFC73Wc3r176+7saNasGaytrdGtW7cM5w+8vrNl4cKFKFOmDMzNzeHk5ISBAwfi5cuXev3Onz+PJk2aoGDBglCr1fD09MzRnRybNm1Chw4d0KJFC9ja2mLTpk3p+kyePBmSJOHOnTvo3bs37OzsYGtri6+++grx8fF6fVNTUzF16lQUL14cZmZm8PDwwH//+18kJSXp9fPw8ECLFi0QEBCAKlWqQK1Ww9fXV7f0686dO+Hr6wtzc3NUrlwZly5d0tv/ypUr6N27N4oVK6ZbirVPnz548eJFlvPt1asXChYsiJSUlHTbGjduDG9v7+y8bRkaO3YsGjdujG3btuH27dt62951jfTu3Ru//PILAP1lctNk93pIO5afnx+sra1hY2ODqlWr6s5r3bp1sXv3bty/f193jLRrL7Nn/P3zzz+62O3s7NC6dWvcuHFDr09OrpGM7N69G6mpqWjYsOE7+4aHh6NHjx4oUqQI1qxZo7ctu9cf8PrurzJlysDMzAwuLi4YPHgwoqKi9PrUrVsXZcuWxZUrV+Dn5wcLCwt4eXlh+/btAICjR4+ievXqUKvV8Pb2xqFDh9Id59GjR+jTpw+cnJxgZmaGMmXKYPXq1en6PXz4EG3atIGlpSUKFSqE4cOHZxh3dqXlnEePHqFNmzawsrKCo6MjRo0aBY1Go9f3yZMnuHnzZobfF286fvw4AKBz58567Z07d0ZiYiL+/PNPXdv27dvh5OSEdu3a6docHR3RqVMn/Pnnn7q5Xb9+HdevX8eAAQNgbPy/hY0GDRoEIYTuvc7JmBnx8PDAtWvXcPToUd21X7duXd32u3fvomPHjrrlN2vUqIHdu3dn+X686z0RQmDr1q26tk2bNuHp06eYPn06VCoV4uLiMrzbOyYmBgcPHkT37t317h7u2bMnrKys8Ntvv+m9J0ZGRhgwYICuzdzcHH379sXp06fx4MEDvbEbNWqEEydOZPgzk4iIiIiUx8IfERERERm0jRs3YvHixfjmm28wcuRIHD16FJ06dcKECROwb98+jBkzBgMGDMBff/2FUaNG6e27fv16NG/eHFZWVpg9eza+++47XL9+HZ9//jlCQ0OzPK67uzsePHiAf/75550x5uQ4qampaNKkCQoVKoS5c+eiffv2mY47cOBAjB49GrVq1cKPP/6Ir776Chs3bkSTJk10v5yPiIhA48aNERoairFjx+Lnn39Gt27dcObMmXfGDQBnz57FnTt30KVLF5iamqJdu3bYuHFjpv07deqEV69eYebMmejUqRPWrl2bbtnTfv36YeLEiahUqRIWLFgAPz8/zJw5M90vxYHXS6p27doVLVu2xMyZM/Hy5Uu0bNkSGzduxPDhw9G9e3dMmTIFISEh6NSpk94vyA8ePIi7d+/iq6++ws8//4zOnTtjy5YtaNasGYQQmc6hR48eePHiBfbv36/XHh4ejn/++Qfdu3fP1nuX1fhCCL3na2XnGhk4cCAaNWqk65/2lSY71wPw+llvzZs3R2RkJMaNG4dZs2ahQoUK2LdvHwBg/PjxqFChAgoWLKg7RlbP+zt06BCaNGmCiIgITJ48GSNGjMCpU6dQq1atDL+PsnONZOTUqVNwcHDIsNj+Jq1Wi+7du+PFixfYtGkT7O3t9bZn9/qbPHkyBg8eDBcXF8ybNw/t27fHsmXL0Lhx43TFr5cvX6JFixaoXr06fvjhB5iZmaFz587YunUrOnfujGbNmmHWrFmIi4tDhw4d8OrVK92+T58+RY0aNXDo0CH4+/vjxx9/hJeXF/r27av3vickJKBBgwbYv38//P39MX78eBw/fhzffvvtO9+7rKQtzeng4IC5c+fCz88P8+bNS7f85Lhx4+Dj44NHjx5lOV5SUhKMjIxgamqq125hYQEAuHDhgq7t0qVLqFSpElQq/V9dVKtWDfHx8brieFpRv0qVKnr9XFxcULRoUb2if3bHzMjChQtRtGhRlCpVSnftjx8/HsDr8/TZZ5/pli1New5eq1at8Pvvv7/zPQEAtVr9zvfk0KFDsLGxwaNHj+Dt7Q0rKyvY2Njg66+/1nvuXlBQEFJTU9O9J6ampqhQoUK696RkyZLplheuVq0agNdLhr6pcuXKEELg1KlTWc6LiIiIiBSi5O2GREREREQ5kdUyhr169dJbevDevXsCgHB0dBRRUVG69nHjxgkA6ZaE69KlizA1NRWJiYlCCCFevXol7OzsRP/+/fWOEx4eLmxtbdO1v+3q1atCrVYLAKJChQpi6NCh4o8//hBxcXF6/XJynF69egkAYuzYse+c//HjxwUAsXHjRr1++/bt02v//fffP2h5P39/f+Hq6iq0Wq0QQogDBw4IAOLSpUt6/SZNmiQAiD59+ui1t23bVjg4OOheBwYGCgCiX79+ev1GjRolAIh//vlH1+bu7i4AiFOnTuna9u/fLwAItVot7t+/r2tftmxZuuUO4+Pj081n8+bNAoA4duyYru3tpT41Go0oWrSo+PLLL/X2nT9/vpAkSdy9ezejt0onq6U+hRDi0qVLAoAYPny4ECJn10hm3yPZvR6ioqKEtbW1qF69ukhISNDrm3aOhch8qc+077s1a9bo2ipUqCAKFSokXrx4oWu7fPmyUKlUomfPnrq27F4jmfn8889F5cqV39nv+++/FwDElClT0m3L7vUXEREhTE1NRePGjfWWW1y0aJEAIFavXq1r8/PzEwDEpk2bdG03b94UAIRKpRJnzpzRtaddv2++f3379hWFCxcWz58/14upc+fOwtbWVncdL1y4UAAQv/32m65PXFyc8PLyeu+lPtNyzvfff6/Xt2LFiune67S+71oSd968eQKAOH78uF772LFjBQDRokULXZulpWW660EIIXbv3i0AiH379gkhhJgzZ44AIMLCwtL1rVq1qqhRo0aOx8xMZkt9Dhs2LN28Xr16JTw9PYWHh0eWy3Lu2LFDABDr16/Xa1+6dKkAIMqWLatrK1eunLCwsBAWFhbim2++ETt27BDffPONACA6d+6s67dt27Z0uSxNx44dhbOzs96c6tevn67ftWvXBACxdOlSvfbHjx8LAGL27NmZzomIiIiIlMM7/oiIiIjIoHXs2BG2tra619WrVwcAdO/eXW9JuOrVqyM5OVl3t8rBgwcRFRWFLl264Pnz57ovIyMjVK9eHUeOHMnyuGXKlEFgYCC6d++O0NBQ/Pjjj2jTpg2cnJywYsUKXb/3Oc7XX3/9znlv27YNtra2aNSokd64lStXhpWVlW5cOzs7AMDff//9ziX63paamoqtW7fiyy+/1C0pWb9+fRQqVCjTu/7+85//6L2uXbs2Xrx4gZiYGADAnj17AAAjRozQ6zdy5EgASLdsXunSpVGzZk3d67TzW79+fbi5uaVrv3v3rq7tzbtrEhMT8fz5c9SoUQMAcPHixUznrVKp0K1bN+zatUvvzqyNGzfis88+g6enZ6b7ZoeVlRUA6Mb+0GsRyP71cPDgQbx69Qpjx46Fubm53hhvLhuaXU+ePEFgYCB69+6td2dduXLl0KhRI935ftO7rpHMvHjxAgUKFMiyz/HjxzFlyhTUrVsXEyZMSLc9u9ffoUOHkJycjGHDhundOda/f3/Y2Niku06trKz07hj09vaGnZ0dfHx8dNcmkP46FUJgx44daNmyJYQQeueuSZMmiI6O1l2re/bsQeHChdGhQwfdeBYWFnrLN76vjM7Jm99LwOs7RYUQ6ZYcflvXrl1ha2uLPn364ODBgwgNDcXy5cuxePFiAK/vXEyTkJAAMzOzdGOkXZtpfdP+m1nf9xkzp/bs2YNq1arh888/17VZWVlhwIABCA0NxfXr1zPdt1mzZnB3d8eoUaOwc+dO3L9/H7/99hvGjx8PY2NjvZhiY2MRHx+Pnj174qeffkK7du3w008/YeDAgdiyZQuCg4P15pEb70na99nz58/f+b4QERERUd5j4Y+IiIiIDNqbxR8AuiKgq6trhu1pzztL++Vp/fr14ejoqPd14MABREREvPPYJUuWxPr16/H8+XNcuXIFM2bMgLGxMQYMGKB7jldOj2NsbIyiRYu+89jBwcGIjo5GoUKF0o0bGxurG9fPzw/t27fHlClTULBgQbRu3Rpr1qzJ1nPBDhw4gGfPnqFatWq4c+cO7ty5g3v37qFevXrYvHlzhs+devt8pP0COe19v3//PlQqFby8vPT6OTs7w87ODvfv389yvOyeXwCIjIzE0KFD4eTkBLVaDUdHR13RLjo6Osu59+zZEwkJCbol/G7duoULFy6gR48eWe6XHbGxsQAAa2trAPJci9m9HkJCQgAAZcuW/eB5ANCdr4yee+jj44Pnz58jLi5Or/1d10hWRBZLtL548QJdunRBgQIFsHHjxnRLPabFm53rL7N5mZqaolixYumu06JFi6YrnNra2r7zOn327BmioqKwfPnydOftq6++AgDdubt//z68vLzSHedDnjkJvC7+ODo66rUVKFAgW+cjI87Ozti1axeSkpLQuHFjeHp6YvTo0fj5558B/K/wDbwuzmeUi9KWtEwr3qf9N7O+bxb5sztmTt2/fz/T6zxte2bMzc2xe/duODg4oH379vDw8EDPnj0xceJE2Nvbp3tPAKBLly56Y3Tt2hUAcPr0ab1+ufGepH2fvc8fAxARERFR7jN+dxciIiIiovzLyMgoR+1pv9BMK1qtX78ezs7O6fq9ebdgdmLw9fWFr68vatasiXr16mHjxo1o2LBhjo9jZmaWYcHibVqtNss779J+kS9JErZv344zZ87gr7/+wv79+9GnTx/MmzcPZ86c0fuF89vSxu7UqVOG248ePYp69erptb3rfU+T3V8ov+/5BV7HferUKYwePRoVKlSAlZUVtFotmjZtmmHR8k2lS5dG5cqVsWHDBvTs2RMbNmyAqalppu9FTly9ehUAdMUnOa7F7F4PH4PsXiNvc3BwyLQYJYRAr1698PjxY/z1119wcXHJciy5Cxofmoe6d++OXr16Zdi3XLlyMkSYucxi/BB16tTB3bt3ERQUhLi4OJQvXx6PHz8G8PoPJtIULlwYT548Sbd/WlvaeSxcuLCu/e1i6pMnT3TPqsvJmHmtTJkyuHr1Kq5fv46XL1+idOnSUKvVGD58OPz8/HT9XFxccO3aNTg5OentX6hQIQD/Kxq/+Z687cmTJ3rzLFy4cIbPZszsPUk7RsGCBXM8TyIiIiLKfSz8ERERERFloHjx4gBe/zK1YcOGso1bpUoVAP/7hWpuHad48eI4dOgQatWqla07WGrUqIEaNWpg+vTp2LRpE7p164YtW7agX79+GfaPi4vDn3/+iS+//FJvacE0Q4YMwcaNG9MV/t7F3d0dWq0WwcHBujtlAODp06eIioqCu7t7jsbLzMuXL3H48GFMmTIFEydO1LWn3V2XHT179sSIESPw5MkTbNq0Cc2bN3/nUpPZsX79ekiShEaNGgHI2TWSWcEqu9dD2rGuXr2a7q637BznbWnn69atW+m23bx5EwULFoSlpWW2xnqXUqVKYceOHRlumz9/Pnbv3o3hw4ejefPmWcabnevvzXkVK1ZM1y85ORn37t2T7XvZ0dER1tbW0Gg07xzT3d0dV69ehRBC7/xk9N5/DIyMjFChQgXd67S7oN+cZ4UKFXD8+HFotVq9P3g4e/YsLCwsdEXCtHHOnz+vV+R7/PgxHj58qLfcaXbHzExm1767u3um13na9neRJAllypTRvd6zZw+0Wq3ee1K5cmUcPHgQjx490rvDMK1wmlbEL1u2LIyNjXH+/Hm9P0hITk5GYGCgXluFChVw5MgRxMTEwMbGRtd+9uxZ3fY33bt3DwD0vkeIiIiI6OPBpT6JiIiIiDLQpEkT2NjYYMaMGRk+++7Zs2dZ7n/8+PEM90t7hljaL2w/9DiZ6dSpEzQaDaZOnZpuW2pqKqKiogC8LoC9fSdV2i95s1ru8/fff0dcXBwGDx6MDh06pPtq0aIFduzYka0lQ9/UrFkzAMDChQv12ufPnw8AWRZtciLtLqa35/72cbPSpUsXSJKEoUOH4u7du+jevfsHxzVr1iwcOHAAX375JUqUKAEgZ9dIWhEt7fymye710LhxY1hbW2PmzJm6Zf7SvPleWVpavnM5VOD1nUQVKlTAunXr9GK6evUqDhw4oDvfcqhZsyZevnyZ7tlz//77L8aNG4fKlStj1qxZWY6R3euvYcOGMDU1xU8//aT3vqxatQrR0dGyXqft27fHjh07dHeCvunNc9+sWTM8fvwY27dv17XFx8dj+fLlssTyLk+ePMHNmzdz/KxQ4PU8Zs+ejXLlyukVuTp06ICnT59i586durbnz59j27ZtaNmype65dGXKlEGpUqWwfPlyaDQaXd8lS5ZAkiS9P07I7piZsbS0TPf9Bbx+/8+dO6dbahN4/QcSy5cvh4eHB0qXLp39NwSvn6v33XffoXDhwnrLeqYV7FatWqXXf+XKlTA2NkbdunUBvF42tmHDhtiwYYPes0jXr1+P2NhYdOzYUdfWoUMHaDQavWslKSkJa9asQfXq1dPdRXnhwgVIkqT3fFUiIiIi+njwjj8iIiIiogzY2NhgyZIl6NGjBypVqoTOnTvD0dERYWFh2L17N2rVqoVFixZluv/s2bNx4cIFtGvXTrcU38WLF/Hrr7/C3t4ew4YNk+U4mfHz88PAgQMxc+ZMBAYGonHjxjAxMUFwcDC2bduGH3/8ER06dMC6deuwePFitG3bFsWLF8erV6+wYsUK2NjYZFmU2bhxIxwcHPDZZ59luL1Vq1ZYsWIFdu/ejXbt2mU77vLly6NXr15Yvnw5oqKi4Ofnh3PnzmHdunVo06ZNju8gzIyNjQ3q1KmDH374ASkpKShSpAgOHDigu5MlOxwdHdG0aVNs27YNdnZ2OSr2pKamYsOGDQBeP0fr/v372LVrF65cuYJ69erp/QI+J9dI5cqVAby+47JJkyYwMjJC586ds3092NjYYMGCBejXrx+qVq2Krl27okCBArh8+TLi4+Oxbt063XG2bt2KESNGoGrVqrCyskLLli0znOucOXPwxRdfoGbNmujbty8SEhLw888/w9bWFpMnT872e/YuzZs3h7GxMQ4dOqS7wys+Ph5ffvklUlJS0KJFC/z2228Z7uvk5IRGjRpl+/pzdHTEuHHjMGXKFDRt2hStWrXCrVu3sHjxYlStWlWWInCaWbNm4ciRI6hevTr69++P0qVLIzIyEhcvXsShQ4cQGRkJAOjfvz8WLVqEnj174sKFCyhcuDDWr18PCwsL2WLJyrhx47Bu3Trcu3cPHh4eWfb18/NDzZo14eXlhfDwcCxfvhyxsbH4+++/9e7C69ChA2rUqIGvvvoK169fR8GCBbF48WJoNBpMmTJFb8w5c+agVatWaNy4MTp37oyrV69i0aJF6Nevn96daTkZMyOVK1fGkiVLMG3aNHh5eaFQoUKoX78+xo4di82bN+OLL77AkCFDYG9vr3s/duzY8c4lmjt16gQXFxeULl0aMTExWL16Ne7evYvdu3frnvcJABUrVkSfPn2wevVqpKamws/PDwEBAdi2bRvGjRuntyzn9OnT8dlnn8HPzw8DBgzAw4cPMW/ePDRu3BhNmzbV9atevTo6duyIcePGISIiAl5eXli3bh1CQ0PTFRgB4ODBg6hVqxYcHBze+X4RERERkQIEEREREVE+MXjwYJHZR9hevXoJd3d33et79+4JAGLOnDl6/Y4cOSIAiG3btum1r1mzRgAQ//77b7r+TZo0Eba2tsLc3FwUL15c9O7dW5w/fz7LWE+ePCkGDx4sypYtK2xtbYWJiYlwc3MTvXv3FiEhIen6Z+c4vXr1EpaWltmaf5rly5eLypUrC7VaLaytrYWvr6/49ttvxePHj4UQQly8eFF06dJFuLm5CTMzM1GoUCHRokWLLOf39OlTYWxsLHr06JFpn/j4eGFhYSHatm0rhBBi0qRJAoB49uyZXr+09/3evXu6tpSUFDFlyhTh6ekpTExMhKurqxg3bpxITEzU29fd3V00b9483bEBiMGDB+u1ZXQ9PHz4ULRt21bY2dkJW1tb0bFjR/H48WMBQEyaNCnLGNP89ttvAoAYMGBApu/F23r16iUA6L4sLCyEh4eHaN++vdi+fbvQaDQZ7pedayQ1NVV88803wtHRUUiSlO775V3XQ5pdu3aJzz77TKjVamFjYyOqVasmNm/erNseGxsrunbtKuzs7AQA3bWX9j6vWbNGb7xDhw6JWrVq6cZr2bKluH79ul6fnFwjmWnVqpVo0KCB7nVaPO/68vPz0+2T3etPCCEWLVokSpUqJUxMTISTk5P4+uuvxcuXL/X6+Pn5iTJlyqTbNyfX79OnT8XgwYOFq6urMDExEc7OzqJBgwZi+fLlev3u378vWrVqJSwsLETBggXF0KFDxb59+wQAceTIkSzfu4xyYGY5J+1cvSntus7OeRo+fLgoVqyYMDMzE46OjqJr164Z5kUhhIiMjBR9+/YVDg4OwsLCQvj5+aXL02l+//13UaFCBWFmZiaKFi0qJkyYIJKTkz9ozLeFh4eL5s2bC2tr63TXTkhIiOjQoYOws7MT5ubmolq1auLvv//O1rizZ88WpUqVEubm5qJAgQKiVatW4tKlSxn2TU5OFpMnTxbu7u7CxMREeHl5iQULFmTY9/jx4+Kzzz4T5ubmwtHRUQwePFjExMSk65eQkCBGjRolnJ2dhZmZmahatarYt29fun5RUVHC1NRUrFy5MlvzIiIiIqK8JwnxjiekExERERERUYb+/PNPtGnTBseOHUPt2rWVDueTd/z4cdStWxc3b97ULZVKRPJZuHAhfvjhB4SEhGTr+bFERERElPdY+CMiIiIiInpPLVq0wI0bN3Dnzh1IkqR0OATgiy++QNGiRbFixQqlQyEyKCkpKShevDjGjh2LQYMGKR0OEREREWWCz/gjIiIiIiLKoS1btuDKlSvYvXs3fvzxRxb9PiJ79+5VOgQig2RiYoKwsDClwyAiIiKid+Adf0RERERERDkkSRKsrKzw5ZdfYunSpTA25t9UEhERERERkfL4r1MiIiIiIqIc4t9PEhERERER0cdIpXQARERERERERERERERERPTheMefQrRaLR4/fgxra2s+D4SIiIiIiIiIiIiIiMjACCHw6tUruLi4QKXKm3vxWPhTyOPHj+Hq6qp0GERERERERERERERERJSLHjx4gKJFi+bJsVj4U4i1tTUAIDQ0FAUKFFA4GiIyBBqNBteuXUOZMmVgZGSkdDhEZACYV4hIbswrRCQ35hUikhNzChHJ7eXLl/Dw8NDVhPICC38KSVve08bGBjY2NgpHQ0SGQKPRwMrKCjY2NvxwSkSyYF4hIrkxrxCR3JhXiEhOzClEJDeNRgMAefrIt7xZUJSIiIiIiIiIiIiIiIiIchULfwrLyyovERk2SZJgb2/PvEJEsmFeISK5Ma8QkdyYV4hITswpRCQ3JfKJJIQQeX5UQkxMDGxtbREdHc2lPomIiIiIiIiIiIiIiAyMErUgPuNPYVqtVukQiMhAaLVaPHz4EEWLFoVKxRu6iejDMa8QkdyYV4hIbswrRJ8WjUaDlJSUXBtfq9UiPDwczs7OzClElG2mpqaZ5gwlakAs/CmMN1wSkVyEEIiMjESRIkWUDoWIDATzChHJjXmFiOTGvEL0aRBCIDw8HFFRUbl+nJSUFCQmJnK5TyLKNpVKBU9PT5iamqbbpkQNiIU/IiIiIiIiIiIiIvpopRX9ChUqBAsLi1wrygkhkJiYCHNzcxb+iChbtFotHj9+jCdPnsDNze2jyB0s/BERERERERERERHRR0mj0eiKfg4ODrl6LCEEhBAs/BFRjjg6OuLx48dITU2FiYmJ0uGACxUrjD9AiEgukiTB2dmZeYWIZMO8QkRyY14hIrkxrxAZvrRn+llYWOTJ8T6GX9oTUf6StsSnRqNJt02Jzyi8409hfEgsEclFpVLB2dlZ6TCIyIAwrxCR3JhXiEhuzCtEn468+OW5JEks/BFRjmWVn5SoAbHqpLCMKsBERO9Do9EgJCSEeYWIZMO8QkRyY14hIrkxrxCRnNKe8SeEUDoUIjIQSnxGYeGPiMiAvHr1SukQiMjAMK8QkdyYV4hIbswrRCQnrVardAhERB+EhT8iIiIiIiIiIiIioo9IaGgoJElCYGBgtvdZu3Yt7OzsFI/jbXXr1sWwYcNki+ljMHnyZFSoUEHpMIgyxMIfEREREREREREREZHMHjx4gD59+sDFxQWmpqZwd3fH0KFD8eLFi3fu6+rqiidPnqBs2bLZPt6XX36J27dvf0jIOZJWFMzqa+3atdi5cyemTp2aZ3Gl0Wg0mDVrFkqVKgW1Wg17e3tUr14dK1eu/OCxR40ahcOHD8sQJZH8jJUO4FOXFw+lJaJPgyRJcHV1ZV4hItkwrxCR3JhXiEhuzCtEJDdTU1NZxrl79y5q1qyJkiVLYvPmzfD09MS1a9cwevRo7N27F2fOnIG9vX2G+yYnJ8PU1BTOzs45OqZarYZarZYj/GxJK06mmTt3Lvbt24dDhw7p2mxtbfM0pjdNmTIFy5Ytw6JFi1ClShXExMTg/PnzePny5XuPKYSARqOBlZUVrKysZIyWDJUSn1F4x5/CVCqeAiKSh0qlgoODA/MKEcmGeYWI5Ma8QkRyY14h+vQIASQm5s5XUpKE1FRjJCVJGW4XIvtxDh48GKampjhw4AD8/Pzg5uaGL774AocOHcKjR48wfvx4XV8PDw9MnToVPXv2hI2NDQYMGJDhEpu7du1CiRIlYG5ujnr16mHdunWQJAlRUVEA0i/1mbYc5fr16+Hh4QFbW1t07txZ79mo+/btw+effw47Ozs4ODigRYsWCAkJydYcjYyM4OzsrPuysrKCsbGxXptarU631KeHhwemTZuGnj17wsrKCu7u7ti1axeePXuG1q1bw8rKCuXKlcP58+f1jnfixAnUrl0barUarq6uGDJkCOLi4jKNb9euXRg0aBA6duwIT09PlC9fHn379sWoUaN0fbRaLWbOnAlPT0+o1WqUL18e27dv120PCAiAJEnYu3cvKleuDDMzM5w4cSLDpT5XrlwJHx8fmJubo1SpUli8eLFuW3JyMvz9/VG4cGGYm5vD3d0dM2fOzNb7TPmbEp9ReMefwjQajdIhEJGB0Gg0CA4ORokSJWBkZKR0OERkAJhXiEhuzCtEJDfmFaJPT1IS0LFjbo0uoNFoYWSkApD+Lp1t2wBz83ePEhkZif3792P69Onp7nZzdnZGt27dsHXrVixevFh3N9DcuXMxceJETJo0KcMx7927hw4dOmDo0KHo168fLl26pFfAykxISAj++OMP/P3333j58iU6deqEWbNmYfr06QCAuLg4jBgxAuXKlUNsbCwmTpyItm3bIjAwMFcLFgsWLMCMGTPw3XffYcGCBejRowc+++wz9OnTB3PmzMGYMWPQs2dPXLt2DZIkISQkBE2bNsW0adOwevVqPHv2DP7+/vD398eaNWsyPIazszP++ecfDBo0CI6Ojhn2mTlzJjZs2IClS5eiRIkSOHbsGLp37w5HR0f4+fnp+o0dOxZz585FsWLFUKBAAQQEBOiNs3HjRkycOBGLFi1CxYoVcenSJfTv3x+Wlpbo1asXfvrpJ+zatQu//fYb3Nzc8ODBAzx48EC295M+XkrUgFj4IyIyIImJiUqHQEQGhnmFiOTGvEJEcmNeIaKPTXBwMIQQ8PHxyXC7j48PXr58iWfPnqFQoUIAgPr162PkyJG6PqGhoXr7LFu2DN7e3pgzZw4AwNvbG1evXtUV8DKj1Wqxdu1aWFtbAwB69OiBw4cP6/Zr3769Xv/Vq1fD0dER169fz9HzBXOqWbNmGDhwIABg4sSJWLJkCapWrYqO/1/VHTNmDGrWrImnT5/C2dkZM2fORLdu3XR3DpYoUQI//fQT/Pz8sGTJEphnUJGdP38+OnToAGdnZ5QpUwafffYZWrdujS+++AIAkJSUhBkzZuDQoUOoWbMmAKBYsWI4ceIEli1bplf4+/7779GoUaNM5zNp0iTMmzcP7dq1AwB4enri+vXrWLZsGXr16oWwsDCUKFECn3/+OSRJgru7+4e/iUSZYOGPiIiIiIiIiIiIiPIFM7PXd97lBiGAhIQkqNVqZPRYLjOznI6X/bVBq1SpkuX2W7duoWrVqnpt1apVe+e4Hh4euqIfABQuXBgRERG618HBwZg4cSLOnj2L58+fQ6vVAgDCwsJytfBXrlw53f87OTkBAHx9fdO1RUREwNnZGZcvX8aVK1ewceNGXR8hBLRaLe7du5dhkbV06dK4evUqLly4gJMnT+LYsWNo2bIlevfujZUrV+LOnTuIj49PV9BLTk5GxYoV9dqyOj9xcXEICQlB37590b9/f117amoqbG1tAQC9e/dGo0aN4O3tjaZNm6JFixZo3LjxO98novfBwh8RERERERERERER5QuSlL3lNt+HEK+/zM2RYeEvu7y8vCBJEm7cuIG2bdum237jxg0UKFBAb/lJS0vL9z9gFkxMTPReS5KkK+4BQMuWLeHu7o4VK1bAxcUFWq0WZcuWRXJycq7Ek1FcacudZtSWFmtsbCwGDhyIIUOGpBvLzc0t0+OoVCpUrVoVVatWxbBhw7Bhwwb06NED48ePR2xsLABg9+7dKFKkiN5+Zm9VebM6P2njrFixAtWrV9fblrYMdaVKlXDv3j3s3bsXhw4dQqdOndCwYUO95wkSyYWFP4Xx4dNEJBeVSoVixYoxrxCRbJhXiEhuzCtEJDfmFSKS29sFn/fh4OCARo0aYfHixRg+fLjec/7Cw8OxceNG9OzZU1fcyg5vb2/s2bNHr+3ff//9oDhfvHiBW7duYcWKFahduzYA4MSJEx80Zm6pVKkSrl+/Di8vrw8ap3Tp0gBe36VXunRpmJmZISwsTG9Zz5xycnKCi4sL7t69i27dumXaz8bGBl9++SW+/PJLdOjQAU2bNkVkZCTs7e3f+9j08VPiMwoLfwrLSXInIsqKJEmwsbFROgwiMiDMK0QkN+YVIpIb8woRyUmSJN0dWh9q0aJF+Oyzz9CkSRNMmzYNnp6euHbtGkaPHo0iRYq889l8bxs4cCDmz5+PMWPGoG/fvggMDMTatWt1cb+PAgUKwMHBAcuXL0fhwoURFhaGsWPHvtdYuW3MmDGoUaMG/P390a9fP1haWuL69es4ePAgFi1alOE+HTp0QK1atfDZZ5/B2dkZ9+7dw7hx41CyZEmUKlUKxsbGGDVqFIYPHw6tVovPP/8c0dHROHnyJGxsbNCrV69sxzdlyhQMGTIEtra2aNq0KZKSknD+/Hm8fPkSI0aMwPz581G4cGFUrFgRKpUK27Ztg7OzM+zs7GR6h+hjpUQNiH8OpTCNRqN0CERkIDQaDYKCgphXiEg2zCtEJDfmFSKSG/MKEclJCIH4+PgcPZsvMyVKlMD58+dRrFgxdOrUCcWLF8eAAQNQr149nD59Osd3eXl6emL79u3YuXMnypUrhyVLlmD8+PEA3v8uRZVKhS1btuDChQsoW7Yshg8fjjlz5rzXWLmtXLlyOHr0KG7fvo3atWujYsWKmDhxIlxcXDLdp0mTJvjrr7/QsmVLlCxZEr169UKpUqVw4MABGBu/vidq6tSp+O677zBz5kz4+PigadOm2L17Nzw9PXMUX79+/bBy5UqsWbMGvr6+8PPzw9q1a3XjWFtb44cffkCVKlVQtWpVhIaGYs+ePbxj/ROgxGcUSciRxSjHYmJiYGtri8jISBQoUEDpcIjIAKT9g9fX11e2v04jok8b8woRyY15hYjkxrxCZPgSExNx7949eHp6wjy3Hu73/4QQSEhIgFqtzhcrtU2fPh1Lly7FgwcPlA6F6JOWVZ56+fIl7O3tER0dnWerFHCpTyIiIiIiIiIiIiKij9zixYtRtWpVODg44OTJk5gzZw78/f2VDouIPjIs/BERERERERERERERfeSCg4Mxbdo0REZGws3NDSNHjsS4ceOUDouIPjJc6lMhaUt9RkVFwdbWVulwiMgACCGQmJgIc3PzfLEcBRF9/JhXiEhuzCtEJDfmFSLDl9dLfQohIEkScwoRZVtWeSo6Ohp2dnZ5utQnnxxJRGRATE1NlQ6BiAwM8woRyY15hYjkxrxCRHJiwY+I8jsW/hSm1WqVDoGIDIRWq0VQUBDzChHJhnmFiOTGvEJEcmNeISK5JSQkKB0CERkQJT6jsPBHREREREREREREREREZABY+CMiIiIiIiIiIiIiIiIyACz8ERERERERERERERERERkAFv4UplLxFBCRPFQqFXx9fZlXiEg2zCtEJDfmFSKSG/MKEclNrVYrHQIRGRAlPqPwUxERkQFJTk5WOgQiMjDMK0QkN+YVIpIb8woRyUkIoXQIOda7d2+0adNG97pu3boYNmxYnscREBAASZIQFRWV58cmov9h4U9hWq1W6RCIyEBotVrcunWLeYWIZMO8QkRyY14hIrkxrxCR3BITE2UZp3fv3pAkCZIkwdTUFF5eXvj++++Rmpoqy/hZ2blzJ6ZOnZqtvnldrPPw8IAkSdiyZUu6bWXKlIEkSVi7dm2exEKUF5T4jMLCHxERERERERERERGRzJo2bYonT54gODgYI0eOxOTJkzFnzpwM+8p597K9vT2sra1lG09urq6uWLNmjV7bmTNnEB4eDktLS4WiIjIcLPwRERERERERERERUb4SlxyX6VdiamK2+yakJGSr7/swMzODs7Mz3N3d8fXXX6Nhw4bYtWsXgP8tzzl9+nS4uLjA29sbAPDgwQN06tQJdnZ2sLe3R+vWrREaGqobU6PRYMSIEbCzs4ODgwO+/fbbdMuTvr3UZ1JSEsaMGQNXV1eYmZnBy8sLq1atQmhoKOrVqwcAKFCgACRJQu/evQG8vktp5syZ8PT0hFqtRvny5bF9+3a94+zZswclS5aEWq1GvXr19OLMSrdu3XD06FE8ePBA17Z69Wp069YNxsbGen2joqLQr18/ODo6wsbGBvXr18fly5d120NCQtC6dWs4OTnBysoKVatWxaFDh/TG8PDwwIwZM9CnTx9YW1vDzc0Ny5cvz1asRPmR8bu7EBFRfmFkZKR0CERkYJhXiEhuzCtEJDfmFaJPk9VMq0y3NSvRDLu77ta9LjS3EOJT4jPs6+fuh4DeAbrXpZeXxvOE5+n6iUkf/uw/tVqNFy9e6F4fPnwYNjY2OHjwIAAgJSUFTZo0Qc2aNXH8+HEYGxtj2rRpaNq0Ka5cuQJTU1PMmzcPa9euxerVq+Hj44N58+bh999/R/369TM9bs+ePXH69Gn89NNPKF++PO7du4fnz5/D1dUVO3bsQPv27XHr1i3Y2NhArVYDAGbOnIkNGzZg6dKlKFGiBI4dO4bu3bvD0dERfn5+ePDgAdq1a4fBgwdjwIABOH/+PEaOHJmt98HJyQlNmjTBunXrMGHCBMTHx2Pr1q04evQofv31V72+HTt2hFqtxt69e2Fra4tly5ahQYMGuH37Nuzt7REbG4tmzZph+vTpMDMzw6+//oqWLVvi1q1bcHNz040zb948TJ06Ff/973+xfft2fP311/Dz89MVXIkMCQt/CuOHUyKSi5GREXx9fZUOg4gMCPMKEcmNeYWI5Ma8QkRykiQJkOQfVwiBw4cPY//+/fjmm2907ZaWlli5ciVMTU0BABs2bIBWq8XKlStfxwJgzZo1sLOzQ0BAABo3boyFCxdi3LhxaNeuHQBg6dKl2L9/f6bHvn37Nn777TccPHgQDRs2BAAUK1ZMt93e3h4AUKhQIdjZ2QF4fYfgjBkzcOjQIdSsWVO3z4kTJ7Bs2TL4+flhyZIlKF68OObNmwcA8Pb2RlBQEGbPnp2t96RPnz4YOXIkxo8fj+3bt6N48eKoUKGCXp8TJ07g3LlziIiIgJmZGQBg7ty5+OOPP7B9+3YMGDAA5cuXR/ny5XX7TJ06Fb///jt27doFf39/XXuzZs0waNAgAMCYMWOwYMECHDlyhIU/ynVK1IBY+FPY27dhExG9LyEEXr16BWtra92HQyKiD8G8QkRyY14hIrkxrxB9umLHxWa6zUil/4v2iFERmfZVSf97GpYQAiH+IVCpVLLklL///htWVlZISUmBVqtF165dMXnyZN12X19fXdEPAC5fvow7d+6kez5fYmIiQkJCEB0djSdPnqB69eq6bcbGxqhSpUqmv2cODAyEkZER/Pz8sh33nTt3EB8fj0aNGum1Jycno2LFigCAGzdu6MUBQFckzI7mzZtj4MCBOHbsGFavXo0+ffqk63P58mXExsbCwcFBrz0hIQEhISEAgNjYWEyePBm7d+/GkydPkJqaioSEBISFhentU65cOd3/S5IEZ2dnRERkfl0QyUWJGhALfwrTarVKh0BEBkKr1eLu3bvw9fXl3cREJAvmFSKSG/MKEcmNeYXo02VpapkrfY2FMdSmalkKf/Xq1cOSJUtgamoKFxeXdM+vs7TUjys2NhaVK1fGxo0b043l6Oj4XjGkLd2ZE7Gxr4uqu3fvRpEiRfS2pd1596GMjY3Ro0cPTJo0CWfPnsXvv/+eYRyFCxdGQEBAum1pdyeOGjUKBw8exNy5c+Hl5QW1Wo0OHTogOTlZr7+JiYnea0mS+Lt5yhNKXGcs/BERERERERERERERyczS0hJeXl7Z7l+pUiVs3boVhQoVgo2NTYZ9ChcujLNnz6JOnToAgNTUVFy4cAGVKlXKsL+vry+0Wi2OHj2qW+rzTWl3HGo0Gl1b6dKlYWZmhrCwsEzvFPTx8cGuXbv02s6cOfPuSb6hT58+mDt3Lr788ksUKFAg3fZKlSohPDwcxsbG8PDwyHCMkydPonfv3mjbti2A18XC0NDQHMVBZGhU7+5CRERERERERERERES5qVu3bihYsCBat26N48eP4969ewgICMCQIUPw8OFDAMDQoUMxa9Ys/PHHH7h58yYGDRqEqKioTMf08PBAr1690KdPH/zxxx+6MX/77TcAgLu7OyRJwt9//41nz54hNjYW1tbWGDVqFIYPH45169YhJCQEFy9exM8//4x169YBAP7zn/8gODgYo0ePxq1bt7Bp0yasXbs2R/P18fHB8+fPsWbNmgy3N2zYEDVr1kSbNm1w4MABhIaG4tSpUxg/fjzOnz8PAChRogR27tyJwMBAXL58GV27duWdfPTJY+GPiMiAmJubKx0CERkY5hUikhvzChHJjXmFiOSk5PNCLSwscOzYMbi5uaFdu3bw8fFB3759kZiYqLsDcOTIkejRowd69eqFmjVrwtraWne3W2aWLFmCDh06YNCgQShVqhT69++PuLg4AECRIkUwZcoUjB07Fk5OTvD39wcATJ06Fd999x1mzpwJHx8fNG3aFLt374anpycAwM3NDTt27MAff/yB8uXLY+nSpZgxY0aO5+zg4JDpcqSSJGHPnj2oU6cOvvrqK5QsWRKdO3fG/fv34eTkBACYP38+ChQogM8++wwtW7ZEkyZNMr37kehTIQklnixIiImJga2tLaKjozO9bZuIiIiIiIiIiIjoU5aYmIh79+7B09OThX4i+ihllaeUqAXxjj+F8bZjIpKLVqvFixcvmFeISDbMK0QkN+YVIpIb8woRyUkIgdTUVPBeGSKSixKfUVj4Uxh/iBCRXIQQePDgAfMKEcmGeYWI5Ma8QkRyY14hIrklJycrHQIRGRAlPqOw8EdERERERERERERERERkAFj4IyIiIiIiIiIiIiIiIjIALPwRERkQa2trpUMgIgPDvEJEcmNeISK5Ma8QkZxUKv7KnIjyN2OlA/jUGRkZKR0CERkIIyMjFC9eXOkwiMiAMK8QkdyYV4hIbswrRCQnSZJgbm6udBhEZECUqAHxzxcUptVqlQ6BiAyEVqtFeHg48woRyYZ5hYjkxrxCRHJjXiEiOQkhkJKSAiGE0qEQkYFQ4jMKC38K4w8RIpKLEALh4eHMK0QkG+YVIpIb8woRyY15hYjklpKSonQIRGRAlPiMwsIfERERERERERERERERkQFg4Y+IiIiIiIiIiIiIiN4pICAAkiQhKioqT4+7du1a2NnZfdAYoaGhkCQJgYGBmfZRan5EcmLhT2GSJCkdAhEZCEmSYG9vz7xCRLJhXiEiuTGvEJHcmFeISG5GRkayjNO7d2+0adNGlrHyiiRJWX5NnjxZ6RCJ8h0lPqMY5/kRSY9KxdorEclDpVLBzc1N6TCIyIAwrxCR3JhXiEhuzCtEJCdJkmBmZqZ0GHkiOTkZpqamem1PnjzR/f/WrVsxceJE3Lp1S9dmZWWF8+fPy3Isok+FEjUgVp0UptVqlQ6BiAyEVqtFWFgY8woRyYZ5hYjkxrxCRHJjXiEiOQkhkJSUBCFErh/r6NGjqFatGszMzFC4cGGMHTsWqampAIC///4bdnZ20Gg0AIDAwEBIkoSxY8fq9u/Xrx+6d++ue33ixAnUrl0barUarq6uGDJkCOLi4nTbPTw8MHXqVPTs2RM2NjYYMGBAupicnZ11X7a2tpAkSa/NyspK1/fChQuoUqUKLCws8Nlnn+kVCCdPnowKFSpg5cqV8PT0hLm5OQAgKioK/fr1g6OjI2xsbFC/fn1cvnxZt9/ly5dRr149WFtbw8bGBpUrV05XaNy/fz98fHxgZWWFpk2b6hUrtVotvv/+exQtWhRmZmaoUKEC9u3bl+V52LNnD0qWLAm1Wo169eohNDQ0y/5EOaXEZxQW/hSWFz9EiOjTIIRAZGQk8woRyYZ5hYjkxrxCRHJjXiH6hO0uA1wY/r/XV6e9bkuKfP361Z3Xr2//8r8+p3sDBz//3+vH+1/3ebxf12R8pB5w5qv/9bn9y+s+Mnr06BGaNWuGqlWr4vLly1iyZAlWrVqFadOmAQBq166NV69e4dKlSwBeFwkLFiyIgIAA3RhHjx5F3bp1AQAhISFo2rQp2rdvjytXrmDr1q04ceIE/P399Y47d+5clC9fHpcuXcJ33333QXMYP3485s2bh/Pnz8PY2Bh9+vTR237nzh3s2LEDO3fu1D1Tr2PHjoiIiMDevXtx4cIFVKpUCQ0aNEBk5Otz1q1bNxQtWhT//vsvLly4gLFjx8LExEQ3Znx8PObOnYv169fj2LFjCAsLw6hRo3Tbf/zxR8ybNw9z587FlStX0KRJE7Rq1QrBwcEZzuHBgwdo164dWrZsicDAQPTr10+vuEokByU+o3CpTyIiIiIiIiIiIiKiPLJ48WK4urpi0aJFkCQJpUqVwuPHjzFmzBhMnDgRtra2qFChAgICAlClShUEBARg+PDhmDJlCmJjYxEdHY07d+7Az88PADBz5kx069YNw4YNAwCUKFECP/30E/z8/LBkyRLdHXf169fHyJEjZZnD9OnTdccfO3YsmjdvjsTERN2xkpOT8euvv8LR0RHA6zsSz507h4iICN1yqnPnzsUff/yB7du3Y8CAAQgLC8Po0aNRqlQp3TzelJKSgqVLl6J48eIAAH9/f3z//fe67XPnzsWYMWPQuXNnAMDs2bNx5MgRLFy4EL/88gvetmTJEhQvXhzz5s0DAHh7eyMoKAizZ8+W5T0iUgoLf0RERERERERERESUvzS/pv+67ITXX2msvdL3qblW/7VLE8BFv09SnUNQq9X/ayg5+PWXjG7cuIGaNWtCkiRdW61atRAbG4uHDx/Czc0Nfn5+CAgIwMiRI3H8+HHMnDkTv/32G06cOIHIyEi4uLjoCmOXL1/GlStXsHHjRt14QghotVrcu3cPPj4+AIAqVarINody5crp/r9w4cIAgIiICN1zV93d3XVFv7QYY2Nj4eDgoDdOQkICQkJCAAAjRoxAv379sH79ejRs2BAdO3bUFfkAwMLCQu914cKFERERAQCIiYnB48ePUatWLb3xa9Wqpbec6Jtu3LiB6tWr67XVrFkze28A0UeMhT+FvZnciYg+RNq668wrRCQX5hUikhvzChHJjXmFiOT25tKSSqpbty5Wr16Ny5cvw8TEBKVKlULdunUREBCAly9f6u62A4DY2FgMHDgQQ4YMSTdOWiEOACwtLWWL7833KS0Hv/kss7ePFRsbi8KFC+stV5rGzs4OwOtnA3bt2hW7d+/G3r17MWnSJGzZsgVt27ZNd8y043KpZ/rYKfEZhYU/halUfMwiEclDpVLB2dlZ6TCIyIAwrxCR3JhXiEhuzCtEJCdJkvKk8Ofj44MdO3ZACKErCpw8eRLW1tYoWrQogP8952/BggW6Il/dunUxa9YsvHz5Um/JzkqVKuH69evw8vLK9djfV6VKlRAeHg5jY2N4eHhk2q9kyZIoWbIkhg8fji5dumDNmjW6wl9WbGxs4OLigpMnT+oVRU+ePIlq1apluI+Pjw927dql13bmzJnsTYgom5SoAbHqpDCNRqN0CERkIDQaDUJCQphXiEg2zCtEJDfmFSKSG/MKEclJCIHExETZ7iKLjo5GYGCg3teDBw8waNAgPHjwAN988w1u3ryJP//8E5MmTcKIESN0RYICBQqgXLly2LhxI+rWrQsAqFOnDi5evIjbt2/rFbfGjBmDU6dOwd/fH4GBgQgODsaff/4Jf39/WeYhh4YNG6JmzZpo06YNDhw4gNDQUJw6dQrjx4/H+fPnkZCQAH9/fwQEBOD+/fs4efIk/v33X90ypdkxevRozJ49G1u3bsWtW7cwduxYBAYGYujQoRn2/89//oPg4GCMHj0at27dwqZNm7B27VqZZkz0mhKfUXjHHxGRAXn16pXSIRCRgWFeISK5Ma8QkdyYV4hITm8uV/mhAgICULFiRb22vn37YuXKldizZw9Gjx6N8uXLw97eHn379sWECRP0+vr5+SEwMFBX+LO3t0fp0qXx9OlTeHt76/qVK1cOR48exfjx41G7dm0IIVC8eHF8+eWXss3lQ0mShD179mD8+PH46quv8OzZMzg7O6NOnTpwcnKCkZERXrx4gZ49e+Lp06coWLAg2rVrhylTpmT7GEOGDEF0dDRGjhyJiIgIlC5dGrt27dI9C/Ftbm5u2LFjB4YPH46ff/4Z1apVw4wZM9CnTx+5pk2kCElwEVxFxMTEwNbWFpGRkShQoIDS4RCRAdBoNAgKCoKvry+MjIyUDoeIDADzChHJjXmFiOTGvEJk+BITE3Hv3j14enrC3Nw8V48lhEBCQgLUajWfHUpE2ZZVnnr58iXs7e0RHR0NGxubPImHS30SERERERERERERERERGQAW/hTGvxwhIrlIkgRXV1fmFSKSDfMKEcmNeYWI5Ma8QkRyMzU1VToEIjIgSnxG4TP+FJb2sFYiog+lUqng4OCgdBhEZECYV4hIbswrRCQ35hUikpMkSTA25q/MiUg+StSAWHVSmEajUToEIjIQGo0GN2/eZF4hItkwrxCR3JhXiEhuzCtEJKe0Z/wJIZQOhYgMhBKfUVj4IyIyIImJiUqHQEQGhnmFiOTGvEJEcmNeISI5sehHRPkdC39EREREREREREREREREBoCFPyIiIiIiIiIiIiIiIiIDwMKfwpR4sCMRGSaVSoVixYoxrxCRbJhXiEhuzCtEJDfmFSKSm5mZmdIhEJEBUeIzinGeH5H0SJKkdAhEZCAkSYKNjY3SYRCRAWFeISK5Ma8QkdyYV4hITpIkwcjISOkwiMiAKFED4p9DKUyj0SgdAhEZCI1Gg6CgIOYVIpIN8woRyY15hYjkxrxCRHISQiA+Ph5CCKVD+ajVrVsXw4YNy/Pjenh4YOHChR80Ru/evdGmTZss+yg1PzJMSnxGYeGPiMiA8B+7RCQ35hUikhvzChHJjXmFiD5GmRWP1q5dCzs7uzyPJzsmT54MSZKy/CKijx8Lf0REREREREREREREnxCNRgOtVqvXNmrUKDx58kT3VbRoUXz//fd6be8rJSXlQ0Mmomxi4Y+IiIiIiIiIiIiISAFpS09OmTIFjo6OsLGxwX/+8x8kJyfr+tStWxf+/v7w9/eHra0tChYsiO+++05vSdKkpCSMGjUKRYoUgaWlJapXr46AgADd9rQ7DXft2oXSpUvDzMwMYWFherFYWVnB2dlZ92VkZARra2u9tjRarRbffvst7O3t4ezsjMmTJ+uNJUkSlixZglatWsHS0hLTp08HAPz555+oVKkSzM3NUaxYMUyZMgWpqakAXi+1OnnyZLi5ucHMzAwuLi4YMmSI3rjx8fHo06cPrK2t4ebmhuXLl+ttDwoKQv369aFWq+Hg4IABAwYgNjY20/c/Li4OPXv2hJWVFQoXLox58+ZlcbaI8gcW/hSmUvEUEJE8VCoVvL29mVeISDbMK0QkN+YVIpIb8wrRp2txmcXYN3yf7vWxacewuMxiJEQmAAAi70RicZnFOPfLOV2fP3r/gdWfr9a9vrP/DhaXWYw7++/o2rY03oI/v/pT9/rcL+ewuMzi3JwKDh8+jBs3biAgIACbN2/Gzp07MWXKFL0+69atg7GxMc6dO4cff/wR8+fPx8qVK3Xb/f39cfr0aWzZsgVXrlxBx44d0bRpUwQHB+v6xMfHY/bs2Vi5ciWuXbuGQoUKvXfM69atg6WlJc6ePYsffvgB33//PQ4ePKjXZ/LkyWjbti2CgoLQp08fHD9+HD179sTQoUNx/fp1LFu2DGvXrtUVBXfs2IEFCxZg2bJlCA4Oxh9//AFfX1+9MefNm4cqVarg0qVLGDRoEL7++mvcunULwOsiXpMmTVCgQAH8+++/2LZtGw4dOgR/f/9M5zF69GgcPXoUf/75Jw4cOICAgABcvHjxvd8Xorcp8RnFOM+PSEREucbU1FTpEIjIwDCvEJHcmFeISG7MK0SU35mammL16tWwsLBAmTJl8P3332P06NGYOnWqrmjg6uqKBQsWQJIkeHt7IygoCAsWLED//v0RFhaGNWvWICwsDC4uLgBeL9u5b98+rFmzBjNmzADwernNxYsXo3z58h8cc7ly5TBp0iQAQIkSJbBo0SIcPnwYjRo10vXp2rUrvvrqK93rPn36YOzYsejVqxcAoFixYpg6dSq+/fZbTJo0CWFhYXB2dkbDhg1hYmICNzc3VKtWTe+4zZo1w6BBgwAAY8aMwYIFC3DkyBF4e3tj06ZNSExMxK+//gpLS0sAwKJFi9CyZUvMnj0bTk5OemPFxsZi1apV2LBhAxo0aADgdUGzaNGiH/z+ECmJhT+Fvb2OMhHR+9JqtQgKCoKvry+MjIyUDoeIDADzChHJjXmFiOTGvEL06Rp0bZDe6zoT6qDOhDq61/Ze9un6tFnbRu+1VxMveF3z0mvrfKAz1Gq17nW1wdVQbbB+8Ulu5cuXh4WFhe51zZo1ERsbiwcPHsDd3R0AUKNGDUiSpNdn3rx50Gg0CAoKgkajQcmSJfXGTUpKgoODg+61qakpypUrJ0vMb49TuHBhRERE6LVVqVJF7/Xly5dx8uRJ3R1+wOtnDSYmJiI+Ph4dO3bEwoULUaxYMTRt2hTNmjVDy5YtYWz8vzLGm8eVJAnOzs664964cQPly5fXFf0AoFatWtBqtbh161a6wl9ISAiSk5NRvXp1XZu9vT28vb1z+nYQZUqJGhALf0REREREREREREREMrKxsUF0dHS69qioKNja2sp6rNjYWBgZGeHChQvp/gjCyspK9/9qtVqvePghTExM9F5LkpSuwPFmAS4tzilTpqBdu3bpxjM3N4erqytu3bqFQ4cO4eDBgxg0aBDmzJmDo0eP6o6XneMSfepY+CMiIiIiIiIiIiIikpG3tzcOHDiQrv3ixYvp7sy7fPkyEhISdHcanjlzBlZWVnB1ddX1OXv2rN4+Z86cQYkSJWBkZISKFStCo9EgIiICtWvXzoXZyKNSpUq4desWvLy8Mu2jVqvRsmVLtGzZEoMHD0apUqUQFBSESpUqvXN8Hx8frF27FnFxcbqi48mTJ3XPg31b8eLFYWJigrNnz8LNzQ0A8PLlS9y+fRt+fn7vOUsi5bHwR0REREREREREREQko6+//hqLFi3CkCFD0K9fP5iZmWH37t3YvHkz/vrrL72+ycnJ6Nu3LyZMmIDQ0FBMmjQJ/v7+uuf7AUBYWBhGjBiBgQMH4uLFi/j5558xb948AEDJkiXRrVs39OzZE/PmzUPFihXx7NkzHD58GOXKlUPz5s3zdO6ZmThxIlq0aAE3Nzd06NABKpUKly9fxtWrVzFt2jSsXbsWGo0G1atXh4WFBTZs2AC1Wq1b7vRdunXrhkmTJqFXr16YPHkynj17hm+++QY9evRIt8wn8PpuyL59+2L06NFwcHBAoUKFMH78eL33nSg/YuFPYUwiRCQXlUoFX19f5hUikg3zChHJjXmFiOTGvEJEcnvz+X4folixYjh27BjGjx+Phg0bIjk5GaVKlcK2bdvQtGlTvb4NGjRAiRIlUKdOHSQlJaFLly6YPHmyXp+ePXsiISEB1apVg5GREYYOHYoBAwbotq9ZswbTpk3DyJEj8ejRIxQsWBA1atRAixYtZJmPHJo0aYK///4b33//PWbPng0TExOUKlUK/fr1AwDY2dlh1qxZGDFiBDQaDXx9ffHXX3/pPacwKxYWFti/fz+GDh2KqlWrwsLCAu3bt8f8+fMz3WfOnDmIjY1Fy5YtYW1tjZEjR2a4RCvR+1LiM4okhBB5flRCTEwMbG1tc2VNZyL6NAkhkJiYCHNzc9nWayeiTxvzChHJjXmFiOTGvEJk+BITE3Hv3j14enrC3Nw8V48lhIAQApIk5VlO6d27N6KiovDHH39k2qdu3bqoUKECFi5cmCcxEVHOZJWnoqOjYWdnh+joaNjY2ORJPPxzKIXxwaNEJBetVotbt24xrxCRbJhXiEhuzCtEJDfmFSKSW2JiotIhEJEBUeIzCgt/RERERERERERERERERAaAz/gjIiIiIiIiIiIiIlLA2rVr39knICAg1+MgIsPBO/6IiAyIkZGR0iEQkYFhXiEiuTGvEJHcmFeIiIiI/od3/CmMH06JSC5GRkbw9fVVOgwiMiDMK0QkN+YVIpIb8woRyUmSJFhYWCgdBhEZECVqQLzjT2FCCKVDICIDIYRATEwM8woRyYZ5hYjkxrxCRHJjXiEiOQkhoNFomFOISDZK5BMW/hSm1WqVDoGIDIRWq8Xdu3eZV4hINswrRCQ35hUikhvzChHJLSkpSekQiMiAKPEZhYU/IiIiIiIiIiIiIiIiIgPAwh8RERERERERERERERGRAWDhj4jIgJibmysdAhEZGOYVIpIb8woRyY15hYjeJTw8HNOmTkVVHx+ULFIEVX18MG3qVISHh6frK0mSAhHmfx4eHli4cKHSYeSqunXrYtiwYbrXeTHnw4cPw8fHBxqNJlePQ5kbO3YsvvnmG6XDyBEW/hRmZGSkdAhEZCCMjIxQqlQp5hUikg3zChHJjXmFiOTGvEJE77Jg3jy4Fy2KEzNmYNjNm1jy+DGG3byJ4zNmwL1oUSyYP1/XV5IkqNVqWYp/vXv3hiRJkCQJJiYmcHJyQqNGjbB69eocP/Nr8uTJqFChwgfH9DYlinUPHz6EqakpypYtm6fHzQ3//vsvBgwYkKvH+PbbbzFhwgTdzzmNRoNZs2ahVKlSUKvVsLe3R/Xq1bFy5cpcjSM/uHbtGtq3bw8PDw9IkpTta/vKlSuoXbs2zM3N4erqih9++EFv+6hRo7Bu3TrcvXv3veJS4jNKvi38zZo1C5Ik6VXYExMTMXjwYDg4OMDKygrt27fH06dP9fYLCwtD8+bNYWFhgUKFCmH06NFITU3V6xMQEIBKlSrBzMwMXl5eWLt2bbrj//LLL/Dw8IC5uTmqV6+Oc+fOvdc8+PBpIpKLVqvFixcvmFeISDbMK0QkN+YVIpIb8woRZWXBvHmYOX48jmk02JeYiG4AGgDoBmB/YiKOaTSY+d//6op/QgikpqZCCCHL8Zs2bYonT54gNDQUe/fuRb169TB06FC0aNEi3e+kPxVr165Fp06dEBMTg7NnzyodzgdxdHSEhYVFro1/4sQJhISEoH379rq2KVOmYMGCBZg6dSquX7+OI0eOYMCAAYiKisq1OJKTk99737CwMBkjyVp8fDyKFSuGWbNmwdnZOVv7xMTEoHHjxnB3d8eFCxcwZ84cTJ48GcuXL9f1KViwIJo0aYIlS5a8V1xKfEbJl4W/f//9F8uWLUO5cuX02ocPH46//voL27Ztw9GjR/H48WO0a9dOt12j0aB58+ZITk7GqVOnsG7dOqxduxYTJ07U9bl37x6aN2+OevXqITAwEMOGDUO/fv2wf/9+XZ+tW7dixIgRmDRpEi5evIjy5cujSZMmiIiIyPFc5PohQkQkhMCDBw+YV4hINswrRCQ35hUikhvzChFlJjw8HGPHjMFfSUmonkmf6gD+SkrC2G+/1S37+SFFjreZmZnB2dkZRYoUQaVKlfDf//4Xf/75J/bu3at3s0lUVBT69esHR0dH2NjYoH79+rh8+TKA14WyKVOm4PLly7o7CNP2zWq/NH/99ReqVq0Kc3NzFCxYEG3btgXwetnK+/fvY/jw4bpx05w4cQK1a9eGWq2Gq6srhgwZgri4ON32iIgItGzZEmq1Gp6enti4cWO23g8hBNasWYMePXqga9euWLVqld720NBQSJKEnTt3ol69erCwsED58uVx+vRpvX47duxAmTJlYGZmBg8PD8ybN09vu4eHB6ZNm4aePXvCysoK7u7u2LVrF549e4bWrVvDysoK5cqVw/nz53X7vHjxAl26dEGRIkVgYWEBX19fbN68Ocv5vH3H5LvOx+XLl1GvXj1YW1vDxsYGlStX1ovhbVu2bEGjRo30lrTetWsXBg0ahI4dO8LT0xPly5dH3759MWrUKF0frVaLH374AV5eXjAzM4ObmxumT5+u2x4UFIT69etDrVbDwcEBAwYMQGxsrG5779690aZNG0yfPh0uLi7w9vYGADx48ACdOnWCnZ0d7O3t0bp1a4SGhmb5Hnl6eqJhw4ZYv3494uPjs+z7oapWrYo5c+agc+fOMDMzy9Y+GzduRHJyMlavXo0yZcqgc+fOGDJkCOa/cScwALRs2RJbtmx5r7iU+IxinOdH/ECxsbHo1q0bVqxYgWnTpunao6OjsWrVKmzatAn169cHAKxZswY+Pj44c+YMatSogQMHDuD69es4dOgQnJycUKFCBUydOhVjxozB5MmTYWpqiqVLl8LT01OXLHx8fHDixAksWLAATZo0AQDMnz8f/fv3x1dffQUAWLp0KXbv3o3Vq1dj7NixGcadlJSEpKQk3euYmBgAr4uRaevzSpIElUoFrVardzGktb+9jm9m7SqVCpIkZdgOpK8wZ9ZuZGQEIUSG7W/HmFk755T9OT04+QBXNlxBgxkNYOFgYRBzelfsnJP8cxJCpMsp+X1Ob+KcOCfOKW/n9GZeMZQ5ZSd2zolz4pxyb05peQWAwczpzRg5J86Jc8r7OaUd6+2++XlOmbVzTpzTpzqntM8PaV+SJGX4i/S321csX456Jiao/o5no1UHUNfUFCtXrMB/x48HkPEv6rN73Le9va1evXooX748du7cib59+wIAOnbsCLVajT179sDW1hbLli1DgwYNcPv2bXTq1AlBQUHYv38/Dh48CACws7ODECLdfsuXL0eDBg1w69Yt2NvbY/fu3Wjbti3Gjx+PdevWITk5GXv27IEQAjt27ECFChXQv39/9O/fXxdrSEgImjZtiqlTp2LVqlV49uwZvvnmG/j7+2P16tUAXheGHj9+jH/++QcmJiYYOnQoIiIidOcos/fln3/+QXx8PBo0aAAXFxfUqlUL8+fPh6WlpV7/8ePHY86cOShRogQmTJiALl26IDg4GMbGxrhw4QI6deqEyZMno1OnTjh16hQGDx4Me3t73fKqALBgwQJMnz4dEyZMwIIFC9CjRw989tln+Oqrr/DDDz9g7Nix6NmzJ65evQpJkpCQkIBKlSrh22+/hY2NDXbv3o0ePXqgWLFiqFatmt75fHNeb362fdf56NatGypWrIglS5ZApVIhMDAQxsbGmb5nx48fR5cuXfSO4+zsjH/++Qdff/01ChUqlOG1N3bsWKxcuRLz58/H559/jidPnuDmzZsQQiAuLg5NmjRBzZo1ce7cOTx79gz9+vWDv78/1qxZoxvj8OHDsLGxwYEDBwC8Loin7Xfs2DEYGxtj+vTpaNq0KS5fvgwzM7MMY7l+/TrWrl2LCRMmYNCgQejQoQN69eoFPz+/dH03btyI//znP+na37Rnzx7Url07yz5p18Db5yqja/L06dOoU6cOTExMdNsaN26M2bNn4+XLl7CzswPwuqj48OFD3Lt3D56enunGSXv9Zr0H+F8ey2v5rvA3ePBgNG/eHA0bNtQr/F24cAEpKSlo2LChrq1UqVJwc3PD6dOnUaNGDZw+fRq+vr5wcnLS9WnSpAm+/vprXLt2DRUrVsTp06f1xkjrk7akaHJyMi5cuIBx48bptqtUKjRs2DDdXx68aebMmZgyZUq69hs3bsDa2hoAYG9vDzc3Nzx8+BCRkZG6Ps7OznB2dkZoaChevXqla3d1dYWDgwOCg4ORmJioay9WrBhsbGxw/fp1vYvM29sbpqamCAoK0ovB19cXycnJuHXrlq7NyMgIvr6+ePXqld7atebm5ihVqhRevnyJBw8e6Nqtra1RvHhxRERE6D0Ul3PK/pzu7b+HKyuuoNgXxeDTyscg5mSI5+ljntP9+/cRGRmJa9euQZIkg5iTIZ4nzolzyk9zSk1N1eWVUqVKGcScDPE8cU6cU36akxACL1++BACDmRNgeOeJc+Kc8tOcLC0tAQDPnj3TW4kpP8/JEM8T58Q5fcicbt++DZVKhcTERAghYGHx+g/m37zJIu3ZfBqNRnfH3h+bNmHEG8fPSs+EBMzfuBHfjhkD4PXvgN8sRJqamsLY2FgXQxozMzMYGRkhISFBb7y0O7Q0Go3eNrVaDSEESpQogatXryIhIQGnTp3CuXPn8OTJE12/qVOn4o8//sD27dvRp08fmJubQ6VSwdbWFiqVCubm5ggICMC5c+cQGhqqi2Pu3Ln4/fffsXnzZvTp0wfTpk1Dp06dMGXKFCQmJkKr1aJkyZJISEiAra0tjIyMoFarYWtrCwBISEjAzJkz0a1bNwwcOBAAULRoUfzwww9o0qQJFi9ejNu3b2Pv3r04duwYypcvD7VajRUrVqBMmTJISUnRzTej87RixQp07twZAFC8eHF4eHhg48aN6NWrF8zNzXXLnw4ZMgT169eHkZERpkyZgjJlyuDq1avw9vbGnDlzUL9+fXz33XdITEyEq6srrly5gjlz5qBHjx66Qlrjxo3Rs2dPAMCECROwdOlSVKhQAS1atAAADB06FPXq1cPTp09ha2sLe3t7DB48WHee/P39sXfvXmzatAm+vr565zdtjkIIXcxHjx7VOx8qlQpz587FH3/8oTsfYWFhGDZsGEqVKoWkpCQULVpUN56JiQlMTEyQlJSku/bu37+vW7Iy7dqbPn06unfvjsKFC6NMmTKoVq0amjdvrrtpKSUlBT/99BPmz5+PTp06AQBcXFzw+eefQwiBdevWITExEUuXLoWlpSV8fX3x448/ok2bNpg0aRKcnJyg0WhgaWmJpUuX6ua8fv16aDQarFy5EqmpqUhJScEvv/wCFxcXHDx4EC1atEBycrLe97aJiQm8vb0xadIkTJgwAceOHcOmTZvQqlUrFCxYEF26dEG3bt3g4eEBAGjRogUCAwPTfT+ZmZlBkiQkJibCxcVFtz3t+ynxre/ztOVX37weM8oRAPD48WMUL15cNycAuu+H8PBwWFhYQKPRoECBAgCAu3fvwtPTU+88AdA7Z2+OX6xYMShC5CObN28WZcuWFQkJCUIIIfz8/MTQoUOFEEJs3LhRmJqaptunatWq4ttvvxVCCNG/f3/RuHFjve1xcXECgNizZ48QQogSJUqIGTNm6PXZvXu3ACDi4+PFo0ePBABx6tQpvT6jR48W1apVyzT2xMREER0drft68OCBACCePXsmUlNTRWpqqtBoNEIIITQaja7tzfY327Jq12q1mbZrtdpstwshMm1/O8bM2jmn7M3pxp83RNSjKPE85LnBzMkQz9PHPqekpCRx+/ZtkZSUZDBzMsTzxDlxTvlpTm/mFUOZkyGeJ86Jc8pPc0rLK2nbDGFOhnieOCfOKT/NKSkpSdy5c0ckJycbzJwM8TxxTpzTh8wpNjZWXLt2TcTHx+vGSNv25tfb7SVcXMQhQIhsfB0ERAkXF6HRaERCQoLQaDTvHP9d7b169RKtW7fOcFunTp1E6dKlhVarFT///LNQqVTC0tJS70ulUolvv/1WaLVaMXHiRFG+fHm9Y2a13+jRo4VWqxVqtVqsWrUq0xjd3d3F/Pnz9dqqVKkiTE1N9ca0sLAQAMS1a9fE77//LoyNjfXOkVarFXZ2dnpjvX3MyMhIYW5uLs6fP69r++GHH8Tnn3+u63/37l0BQJw9e1ZvPwAiICBAaLVaUbFiRTFp0iS98X///XdhYmIiUlJShBBCuLu7i9mzZ+u2azQaAUBs3bpV1xYSEiIAiMDAQKHVakVKSoqYMmWKKFu2rChQoICwtLQUxsbGomPHjrp9/Pz8xJAhQ9K9f9k9HxMnThTGxsaiQYMGYsaMGSI4ODjLa8nU1FRs3bo1XXtqaqo4e/asWLBggWjbtq0wMjISffr0EVqtVpw5c0YAECEhIRme82HDhom6devqHfPly5d673GvXr1Ew4YN9fYbOXKkMDIySjc/SZLEL7/8kqPvj6ioKNG1a1cBIN33SGbj5OQr7Rp4+9rOaOxGjRqJAQMG6LVdvXpVABDXr1/XtSUlJQkAYvfu3RmOEx8fL65fvy5iY2PT5bG0azg6OlrklXxzx9+DBw8wdOhQHDx4UG9N2/zCzMwsw3VlTU1NYWRkpNeW2e2fb/fLi3ZJkjJszyzGnLZzTkD4hXBsbb0VVQdXRbNFzd5rnI9tToZ4nvLDnExNTVGiRIkPijGn7TxPnJNcMea0nXPKmzkZGRmlyyv5fU4f2s45cU5yxZjTdkOZ09t5xRDm9CZDOU9v4pw4J7lizGl7dudkZGSE4sWLZzhuRv3ljDGn7Z/yeXrfds6Jc0prlyRJ9wX8bym/t73Zbmtjg/DHjzPs97angN7ddJnJznGzs+3GjRvw9PSEJEmIi4tD4cKFERAQkK6fnZ1dpvPOzn5qtTpb79mb22JjYzFw4EAMGTIkXV83NzcEBwdnuF9GbW/+/+bNm5GYmIjq1f/3xEXx/8vQBgcHo2TJkrr+pqam6eIW/7/M65ttmf03szHebEu7ntPGnTt3Ln766ScsXLgQvr6+sLS0xLBhw5CcnJxuThnNMTvnY8qUKejWrRt2796NvXv3YvLkydiyZYvuuYtvz6FgwYKIiopK125kZIRq1aqhWrVqGDZsGDZs2IAePXpgwoQJurvdMjo/mb1Xb7al/X/a8qtp4uLiULly5Qyf5+jo6JhuzIyOefHiRfz666/YvHkzJEnCiBEj0K9fP932jRs36u40zczevXvfudTnm8fN6Bp9k7OzM54+farXnraCgLOzs649bdWSQoUKZThO2uu036W8KbPclpvyTeHvwoULiIiIQKVKlXRtGo0Gx44dw6JFi7B//34kJycjKipKt+4qADx9+lR3O6yzszPOnTunN+7Tp09129L+m9b2Zh8bGxuo1WrdicuoT9oYOaF9a+1q+vQULFUQDWY1QMkWJZEYnYiw42Fwq+0Gc9v8V+AmZWm1WkRERKBQoUKKrR9NRIaFeYWI5Ma8QkRyY14hosy07toV62fMQLdsLPf5q1qNNl276pZuNDY2zrKY9yH++ecfBAUFYfjw4QCASpUqITw8HMbGxrolD99mamqa7tmI2dmvXLlyOHz4ML766qscjXv9+nV4eXlluE+pUqWQmpqKCxcuoGrVqgCAW7du6QpUmVm1ahVGjhyJ3r1767UPGjQIq1evxqxZs7LcP42Pjw9Onjyp13by5EmULFnygwosJ0+eROvWrdG9e3cAr3++3L59G6VLl87W/tk5HwBQsmRJlCxZEsOHD0eXLl2wZs0avcLfmypWrIjr16+/89hpMcbFxaFEiRJQq9U4fPgw+vXrl66vj48P1q5di7i4ON1y2SdPnoRKpYK3t3eW89u6dSsKFSoEGxubd8aU5uHDh9iwYQPWr1+PkJAQtGzZEqtWrULTpk1hbKxfnmrVqpVeYTgjRYoUyfaxs6NmzZoYP348UlJSYGJiAgA4ePAgvL29dct7AsDVq1dhYmKCMmXK5PgYStSA8s0nogYNGiAoKAiBgYG6rypVqqBbt266/zcxMcHhw4d1+9y6dQthYWGoWbMmgNcnMSgoSG/N94MHD8LGxkb3zVGzZk29MdL6pI1hamqKypUr6/XRarU4fPiwrk9OiCwe/EqfBjMbM3w+5nMUKlMIQZuCsLnlZoQGhCodFuVDQgiEh4czrxCRbJhXiEhuzCtEJDfmFSLKTL/+/XEkJQVn39HvLICA5GT0/f8iSdpzvuSQlJSE8PBwPHr0CBcvXsSMGTPQunVrtGjRQvf8uYYNG6JmzZpo06YNDhw4gNDQUJw6dQrjx4/H+fPnAQAeHh64d+8eAgMD8fz5cyQlJWVrv0mTJmHz5s2YNGkSbty4gaCgIMyePVsXn4eHB44dO4ZHjx7h+fPnAIAxY8bg1KlT8Pf3R2BgIIKDg/Hnn3/C398fwOtnMjZt2hQDBw7E2bNnceHCBfTr1w9qtTrT9yEwMBAXL15Ev379ULZsWb2vLl26YN26dbpn5b3LyJEjcfjwYUydOhW3b9/GunXrsGjRIowaNSrnJ+gNJUqUwMGDB3Hq1CncuHEDAwcOTHfzT1bedT4SEhLg7++PgIAA3L9/HydPnsS///4LHx+fTMds0qQJTpw4odfWoUMHLFiwAGfPnsX9+/cREBCAwYMHo2TJkihVqhTMzc0xZswYfPvtt/j1118REhKCM2fOYNWqVQCAbt26wdzcHL169cLVq1dx5MgRfPPNN+jRowecnJwyjaVbt24oWLAgWrdujePHj+PevXsICAjAkCFD8PDhw0z3c3d3x++//47BgwfjyZMn2LZtG1q0aJGu6Ae8fs6ol5dXll9ZXWfJycm62lFycjIePXqEwMBA3LlzR9dn0aJFaNCgge51165dYWpqir59++LatWvYunUrfvzxR4wYMUJv7OPHj6N27dpZHj8zinxGybNFRXOB3xvP+BNCiP/85z/Czc1N/PPPP+L8+fOiZs2aombNmrrtqampomzZsqJx48YiMDBQ7Nu3Tzg6Oopx48bp+ty9e1dYWFiI0aNHixs3bohffvlFGBkZiX379un6bNmyRZiZmYm1a9eK69eviwEDBgg7OzsRHh6e7dijo6MFABEZGflhbwLla3HP4kT8i/+tT/4y9KU4+/NZERUWpXBklB+lpqaKS5cu6dbkJyL6UMwrRCQ35hUikhvzCpHhS0hIENevXxcJCQk53nf+3LnC0cxMnMnk2X5nAOFoZibmz5snhHj93K64uDjd7+o+RK9evQQAAUAYGxsLR0dH0bBhQ7F69WrdcxDTxMTEiG+++Ua4uLgIExMT4erqKrp16ybCwsKEEEIkJiaK9u3bCzs7OwFArFmzJlv7CSHEjh07RIUKFYSpqakoWLCgaNeunW7b6dOnRbly5YSZmZl4s1Rw7tw50ahRI2FlZSUsLS1FuXLlxPTp03Xbnzx5Ipo3by7MzMyEm5ub+PXXX4W7u7tYsGBBhu+Fv7+/KF26dIbbnjx5IlQqlfjzzz/FvXv3BABx6dIl3fa0588dOXJE17Z9+3ZRunRpYWJiItzc3MScOXP0xswoFgDi999/171++1gvXrwQrVu3FlZWVqJQoUJiwoQJomfPnqJ169a6fd6uR7x9nKzOR1JSkujcubNwdXUVpqamwsXFRfj7+2d5Xb948UKYm5uLmzdv6tqWL18u6tWrJxwdHYWpqalwc3MTvXv3FqGhobo+Go1GTJs2Tbi7u+veoxkzZui2X7lyRdSrV0+Ym5sLe3t70b9/f/Hq1Svd9rTnU77tyZMnomfPnqJgwYLCzMxMFCtWTPTv3z/LZ9fduHEj021ySzunb3/5+fnp+kyaNEm4u7vr7Xf58mXx+eefCzMzM1GkSBExa9asdGN7e3uLzZs3Z3rsrPKUEs/4k4TIv38SVbduXVSoUAELFy4EACQmJmLkyJHYvHkzkpKS0KRJEyxevFhvCc779+/j66+/RkBAACwtLdGrVy/MmjVLr8IcEBCA4cOH4/r16yhatCi+++67dLcgL1q0CHPmzEF4eDgqVKiAn3766Z23ob4pJiYGtra2iIyM1LtllD4te4fuxbmfzmFY2DDYutoqHQ7lcxqNBkFBQfD19VVk7WgiMjzMK0QkN+YVIpIb8wqR4UtMTMS9e/fg6emZ5fP3MrNg/nyM/fZb1DUxQc/ERDgBCAfwq7k5jqakYNYPP2D4/9/dI4RAQkKC3rPxiJQ0evRoxMTEYNmyZUqH8snau3cvRo4ciStXrmR4pyKQdZ56+fIl7O3tER0dnaNlUj9Evi785Wdphb/rh6/Dp37mt/OSYbv22zWEHAxBiyUtoDL+38q7qUmprx+8bMp/tFD2abVaPHz4EEWLFuWzLYhIFswrRCQ35hUikhvzCpHh+9DCHwCEh4dj1cqV+GPTJkRHR8PW1hZtunZF33799G4aEUIgOTkZpqamLPzRRyEqKgqLFy/G2LFj+XNOIdu3b4erq2uWN35llaeioqJQoEABFv4+BWmFv9/6/oaOKzsqHQ59RG7+cRM7uuxA+y3tUap1KaXDISIiIiIiIiIiUowchT8iotyUVZ5KqwXlZeGPJWKF1Z9ZX+kQ6CNT0Kcg3Gq7wcTCROlQKJ/RarUICwuDVqtVOhQiMhDMK0QkN+YVIpIb8woRyUkIgaSkJPBeGSKSixKfUVj4U9ibyzvSp0WbqsWvDX/F2Z/P6rUX9C6IHgd6oHij4gpFRvmVEAKRkZH8cEpEsmFeISK5Ma8QkdyYV4g+HXn1fa7RaPLkOERkOLLKT0p8RmHVSWHRYdE4OvUotKn8y7RPTfyLeDz+9zEigyMz7XN+6Xnc2XcnD6MiIiIiIiIiIiL6eJiYvF4VKz4+XuFIiIgylpycDAAwMjJSOJLXjJUO4FN3dfNVXJh9AUWqFoFXUy+lw6E8ZOVkhbHRYzMt+iZGJeKfCf/AurA1ijUsxrtDiYiIiIiIiIjok2NkZAQ7OztEREQAACwsLCBJUq4cK22pT0mScu0YRGRYtFotnj17BgsLCxgbfxwlt48jik9YhT4VUKJWCRRvwmUdP1WZFfTM7czx1bGvYGJpwqIfZYskSXB2duYHUyKSDfMKEcmNeYWI5Ma8QvRpcHZ2BgBd8S+3CCGg1WqhUqmYV4go21QqFdzc3DLMG0rkEklwEXRFxMTEwNbWFtHR0bCxsVE6HFLA06CniA6LhoefB0ytTJUOh4iIiIiIiIiI6KOm0WiQkpKidBhERHpMTU2hUmV8844StSDe8aewtIfFRt6JxNmfz6LxnMYwMv041oGl3HVl/RWcmnMK39z5BvZW9pn2S4xOxPHpx2HvZY/KAyrnYYSU32g0GoSGhsLDw+OjWU+aiPI35hUikhvzChHJjXmF6NNiZGSUq9/rzClEJLe0GlBe4vqBH4lr267h3E/nELw3WOlQKI+U614OrVa1gk3RrKv8JmoTXFp1CTd/v5lHkVF+9urVK6VDICIDw7xCRHJjXiEiuTGvEJGcmFOIKL/jHX8fiepDqsOtlhvc67grHQrlEadyTnAq5/TOfkamRuh/vj/sPOxyPygiIiIiIiIiIiIiIsq3eMffR8LU0pRFv09MTh6vWcCzAB8oTEREREREREREREREWWLhT2FvF3Oe33qOTS024eXdlwpFRHlBCIEfHH7Arv67sr1PeGA4jkw6kqOCIX1aJEmCq6sri8REJBvmFSKSG/MKEcmNeYWI5MScQkRyUyKfsPCnMJVK/xS8evwKwXuCcWf/HYUioryQmpCKItWKwNbNNtv7BG0KwrHvjyEiKCIXI6P8TKVSwcHBIV1eISJ6X8wrRCQ35hUikhvzChHJiTmFiOSmRD6RBG8fUkRMTAxsbW0RGRmJAgUK6G17EfwCDiUcFIqMPlaRdyIRFxEH189clQ6FPlIajQbBwcEoUaIEjIyMlA6HiAwA8woRyY15hYjkxrxCRHJiTiEiub18+RL29vaIjo6GjY1NnhzTOE+OQjnCoh9lxN7LHvZe9kqHQR+5xMREpUMgIgPDvEJEcmNeISK5Ma8QkZyYU4gov+M9yx+pO/vvYEPTDYgKjVI6FMoFoQGhODz+MKIfROd438cXHmPv0L181h8REREREREREREREelh4e8jlfgyEaFHQhFxjc9zM0T3/rmHEzNOICkmKcf7Xv71Ms79fA6Pzz/OhciIiIiIiIiIiIiIiCi/4jP+FJL2jL+oqCjY2tqm256amAoAMDbnaqyGKDk2GdEPomFf3B5GpjlbLzzuWRxiw2Ph5OuUS9FRfiWEwKtXr2BtbQ1JkpQOh4gMAPMKEcmNeYWI5Ma8QkRyYk4hIrlFR0fDzs4uT5/xx8KfQtIKf3l5somIiIiIiIgAIPpBNC6uvIjK/SvDpij/TUpERERElBuUqAVxqU+FaTSaTLdFP4jGv4v/RWx4bB5GRHnh+c3nSIhMeO/9E6MSEbQpCE8uPpExKsrvNBoNgoKCsswrREQ5wbxCRHJjXvk4JMUkYVmFZUiKSYLaQa10OEQfhHmFiOTEnEJEclMin7Dw9xG7f+w+9gzeg5CDIUqHQjJbWX0ltnXc9t77xz6Nxc5uO3F161UZoyJDwA+mRCQ35hUikhvzivLMbMxQ57s68GriBRO1idLhEH0w5hUikhNzChHld3yA3EfMq4kXOv/ZGe513JUOhWQktAI1RtSArWv6Zztml31xe7Tf0h5FqxeVMTIiIiIiIjJkaU/6kCQJNYbVgBACz248g5WTFdT2vPOPiIiIiMgQ8I6/j5hFQQt4t/KGuZ250qGQjCSVhLqT6qJin4rvPYbKWIWyX5aFnYedfIEREREREZFBe3HrBWZYzsDJOScBADf/uInFpRfj+o7rCkdGRERERERyYeFPYSrVu09B9INoJL1KyoNoKL9JepUETQqXH6DXVCoVvL29s5VXiIiyg3mFiOTGvKI8Dz8P2Lq9Xn3EvY47qn1TDYUrFlY4KqL3x7xCRHJiTiEiuSmRT5jBPnLXtl3DQreFuLXrltKhkEzu7LuDdfXX4eGZhx80zrlfzmGWzSw8/vexTJGRITA1NVU6BCIyMMwrRCQ35hXlFCxVEN32dkPZL8sCACwcLPDFT1/ApYqLwpERfRjmFSKSE3MKEeV3LPwpTKvVZrm9aI2iqNivIuyL2+dRRJTbYp/GIvxSOFKTUj9oHKdyTijbpSyM1XxUJ72m1WoRFBT0zrxCRJRdzCtEJDfmlY+XNpXnhPIn5hUikhNzChHJTYl8worBR87W1RatVrRSOgySUYVeFVChV4UPHse9tjvca7t/eEBERERERPRJuLLhCp5df4Y6E+rAxMIEAPDy7ktsaLoB5XuWR50JdRSOkIiIiIiIPhTv+CMiIiIiIiL6BNzYeQMnZ5+EkamRrs2mqA2MTI24kggRERERkYHgJ/t8IORgCA6NOYSmC5vCvQ7v8MrvQg6EQJOsQckWJT94rLM/nUVoQCi+3PmlDJEREREREZEha7OuDWLDY6Ey/t/fABuZGmHQ1UEKRkVERERERHLiHX8KU6nefQqMzYwR+yQWcc/i8iAiym0BkwLw14C/ZBnradBThBwIQcLLBFnGo/xNpVLB19c3W3mFiCg7mFeISG7MK8oyszaDQwkHpcMgkhXzChHJiTmFiOSmRD6RhBAiz49KiImJga2tLaKiomBra5tl37RTJElSXoRGuez+8ftIjEqEd0vvDx4rOTYZJhYmkFS8Nuh1rkhMTIS5uTnzBRHJgnmFiOTGvKIcoRV4euUpbN1toS6g1tsWGRKJsz+dRanWpeBZ31OhCIneD/MKEcmJOYWI5BYdHQ07OztER0fDxsYmT47JP11QmFarfWcfSZL4g8aAuNd2l6XoBwCmVqYs+pGOVqvFrVu3spVXiIiyg3mFiOTGvKKcuIg4LKu4DP9M+CfdtqSYJJz76RzCToQpEBnRh2FeISI5MacQkdyUyCd8xl8+8fTKU9zYeQNVB1WFZSFLpcOh9yT33ZtCCIQGhAIC/MtcIiIiIiLKlMpEhbpT6sKlqku6bYXKFMKwsGGwLmyd94EREREREZGseMdfPvHw7EMcnXIUD049UDoU+gDR96Mxw3IGjk07Jst4kiRhe6ft+Gd8+r/aJSIiIiIiSmPhYAG/iX4o8UWJdNuMTI1g62oLlTF/RUBERERElN/xjr98olTrUihcsTCcyjkpHQp9AKEV8PDzgK1b1s91zIlmvzSDpRPvAqXXjIyMlA6BiAwM8woRyY155eMU8zAGsU9j4VI5/R2BRB875hUikhNzChHld5JIW3uQ8lRMTAxsbW3z9IGORERERERE9Gk6+9NZ3Pz9JtptbAdrl/RLev7a8Fc8OvcI42LGKRAdEREREZFhUqIWxHU8FJaTuqsmRYPHFx5DaFmrJX1CCGg1fOjwp04IgZiYmBzlFSKirDCvEJHcmFeUExcRh/DL4TCxNMlwe8W+FVF3cl3+u4LyHeYVIpITcwoRyU2JfMLCn8K02uz/o+qf8f9gRZUVeH7reS5GRLnp9u7bODTuEOIi4mQbM+RACGbZzsK1rddkG5PyJ61Wi7t37+YorxARZYV5hYjkxryinPrT6mNM5BiY25pnuN23iy9qjqgJlRF/TUD5C/MKEcmJOYWI5KZEPuEn+nykZMuSqDOxDkytTJUOhd7T3YN3cXLWSWiSNbKNae1iDZcqLjCzMZNtTCIiIiLK/+JfxOPQuEN4cumJ0qEQEVE+kpqUinX11+HGzhtKh0I5lBidiD3+e3D30F2lQyEiIgWx8JePuNd2R70p9WDraqt0KPSe6k+rD//b/rAqbCXbmIXKFkKvf3qhZIuSso1JRERERPlf8J5gnJx1Enf23VE6FPoIXFhxAWEnwzLdHh4YjhXVVuDKxit5GBURfYy0qVpEXI3ApdWXlA6Fcuj5zef495d/8ejcI6VDISIiBRkrHQDRp8TUyhQOJRyUDoMMmLl5xks3ERG9L+YVovyrfI/yKFKtCOyL2wMAhFZACKH4Uo7MK3kvJT4Ffw/4G77dfOFWyy3DPkZmRnj1+BWSXyXncXREH455RV6mlqao9309FPItpHQolENFqhZBlUFVuFrYB2JOIaL8ThJ8UqkiYmJiYGtri+joaNjY2GR7v/0j9+P+0fsYcH5ALkZHueX5redQF1DDspClrOMGrg3Ek4tP8MVPX8g6LhEREREZhqtbr+Kvfn/hyz++RLEGxZQOh/KYJlmD+8fuw9zOHC5VXJQOh4g+Ys+uP8OVDVdQ5T9VYOvGFafyo+kW0+FZ3xNd/+6qdChERIT3rwV9CC71qbCcPtgxNSEV2lQtUuJTcikiyk1r66zFb+1/k33ckAMhOPfzOV4XnzitVosXL17wAdREJBvmFaL87c7+O7h35B4AwNbNFi5VXCCpJEVjYl5RhpGpEYo1LMaiHxkk5hV5Xdt2DSdmnkB0WDSEEEh6laR0SJQDT4OeotOOTmi2qJnSoeRbzClEJDcl8gmX+lRYTm+4bL64eS5FQnmh2pBqst/tBwCN5zVG88XNYWJhIvvYlH8IIfDgwQPY2dkpHQoRGQjmFaL87cDIA9CmaOF/yx+uNV3R60gvpUNiXlGIJkUDlbEKkpR14ffOvjuIuh+FKgOr5FFkRB+OeUVeft/5wbOeJ5zKO2Gu01x4NfVC21/bKh0WZdPObjuR/CoZQ+8NVTqUfIs5hYjkpsSimyz8EeWhOuPr5Mq41oWtc2VcIiIiIsq/msxvgtSkVKXDoI/AmYVnEDAxAH1O9kHhSoUz7Xf2p7O49889VB5Q+Z1FQiIyTJJKgnsddwCAex13FPQpqHBElBPV/KshLiIOidGJMLflc+qIiD5VXOozn9GmanFhxQVc++2a0qHQR0SbqkXEtQi8vPtS6VCIiIiI6CNRvHFxeLf01r2++edN7Oy2EykJXB7+U2PrZgvP+p6wcrbKsl+9qfVe3xma93+UTEQfgatbriI8MFz3utP2Tqg9rraCEVFOVR5QGWcWnMHWtluVDoWIiBTEwl8+IxlJOPTtIZz7+ZzSoVAOPTz7EOvqrcOd/XdkHzsuIg5Lyi7BqbmnZB+b8hdra979SUTyYl4hyp8yWk4mPDAcQZuC8OLWCwUi+h/mlbxX9suy6Lq7K6xdsn7vXSq7wLWmq+LPgiTKKeaVD5ccl4y/+v+Fvwf+rXQo9IHK9y6Pki1KKh1GvsacQkT5HZf6VJiRkVGO+kuShC5/d4Gtq20uRUS5Jf5ZPJ5eeYrk2GTZx7YqbIXaE2rD7XM32cem/MPIyAjFixdXOgwig3fyh5NwruCM4o0N//uNeYUo/3pw8gE2tdiEpgubokLvCgCA6t9UR41hNRRd+ot55eOnSdZAq9HCRM3nh1P+wLwiDxMLE3Tf3x2aZI2uLTUxFbu/3g07Tzv4TfRTMDrKjourLuL8kvNos7YNCpUtpHQ4+RZzChHJLac1IDnwjj+FabXaHO/jVssNtm4s/OU3JVuUxLcvvkXp9qVlH1uSJNSfWh9eTbxkH5vyD61Wi/Dw8PfKK0SUPZpkDQ6NOYSTP5xE0qskpcPJdcwrRPmXykQFl8ousHSy1LWp7dWKP++HeUUZB789iDMLz7yzX9iJMEwzn4aLKy7mQVRE8mBekYckSXD9zBUedT10bUZmRgg5GIJHZx8pFxhlW0pcCmKfxMLM1kzpUPI15hQikpsS+YSFP4VltARPdvaJfRqLhMiEXIiIiPIrIQTCw8PfK68QUeaEENjUYhPOLToHyUhC/3/7o/u+7jCzNvx/UDOvEOVfRasXRc/DPVHiixJ67ZF3InH30F2FomJeUcqFZRcQvDv4nf1s3W1RukNp2Hna5X5QRDJhXvlw0WHReHjmYbr3UJIkDL4+GF13d1UoMsqJ6kOqY8SjETgx6wT+mfCP0uHkW8wpRCQ3JfIJC3/5UNiJMMxznofAdYFKh0I5EHo0FDd+vwGtJncq/BdWXMCiUovw8t7LXBmfiOhT9erRKzw69wiRdyKhMlLBpYoLVMavP0KlJqYqHB0RUc781f8vbG27FULLX2Z9SoY/HI52G9u9s5+tqy06/tYR3i298yAqIvpYnFt0DqtqrsLD0w/TbTOzMfw/djM0t/+6jbsHlfsjHyIiUh6f8ZcPFSpTCBX7VeR63fnM6bmnEXIgBOMTx+faMSRJQmJUYq6NT0T0KbIpaoORj0ciOe5/z2gVWoHfOvyGpOgk9DzcU8HoiIgydmn1JUQ/iIbfd36QVJKuvcbwGkh6lQStRgsjVd4/a4KUYWZt9kncqU5E76dcj3IwNjdG0ZpF021Ljk3GnX13YONqg6LV02+nj0NKQgpOzz+NYg2KYejdoXo/+4mI6NPDwp/CJCnnP4jV9mq0WtEqF6Kh3FR7Qm2U713+vc55dlTuXxmV+1fOlbEpf5AkCfb29rl2jRF9ylTGKr3nYkkqCea25pBUElISUmCiNlEwutzDvEKUfwVtDMKTi09Qd1JdvXbvVsreycW8kvcSoxPx8u5L2Be3z9adO6fnn8az68/QaiX/zUn5A/PKh3PydYKTr1OG25LjkrGt4zZUGlCJhb+PWGRwJI5MOILUxFQUrcHz9CGYU4hIbkrkE0lwwWJFxMTEwNbWFtHR0bCxsVE6HCIiIvp/17ZdQ0p8Csr3KI8Hpx/gzr47qDqoKqwLW+v102q0UBlx1XQi+jglRCYg7lkcCnoXzHC70AreDfCJCN4TjE3NN6HVqlao2KfiO/tvbbsVwXuD8d+4//LnHNEnIC4iDmoHdabf70IIXN18Fc4VneHo45jH0VF2pSam4umVp7AoaIGEyAQAgEsVF4WjIiIiQJlaED/FK0yrfb/nvV3dchWraq5C1P0oeQOifC1wXSAur7+sdBikEK1Wi7CwsPfOK0T0mpGpEQ6PPYyTP5zE1S1XcXzacSS/Sk7XL+2XI0IIPDj9IK/DzBPMK0T5l9penWHRTwiBpeWXYkubLQpExbyihALFC6De1HooUq1Itvq3Xtsa/41l0Y/yD+aVD7O61mqsqLIi0+2SJMG3qy+Lfh85Y3NjFKlWBAWKFcDObjvxZ58/lQ4p32JOISK5KZFP+EleYe97w2VSTBIiQyIR8zBG5ogoN6TEp2C6ejr2Dtmbq8c5Pfc0jnx3JFePQR8vIQQiIyPfO68Q0Wue9T1Rvld5VB5YGU0XNkXf033hUNIh0/6Hxx3G6lqrERoQmndB5hHmFaL8SZOiwbMbz5Acm/6PFiRJgn0Je1gXsc5gz9zHvJL3CnoXRJ0JdbL9jHhzW3OojPmrAso/PuW8IoRA2MkwBO8Nfu/9vVt7o1S7Uu/sm5qUCm0qCyEfq1dPXiE1KRXA60fNfD72c4Ujyr8+5ZxCRLlDiXzCZ/zlU5X6VULlAXyeW36RmpQKz/qeKFCsQK4ep9WqVlA7qHP1GEREhiwxKhGm1qZoOKuhru1dz8io1K8S4l/EcykdIvpoRIVGYXHpxag1ppZePkvTaXsnBaKi/CI5LhlPLj6BTVEbFPDM3X+/ENGH0SRpsLXNVpjZmsHrtleOl3CWJAmN5zZ+Z79zv5zDviH70Pd032zfPUx5a1OzTYh/Ho/hD4ajfI/ySodDREQKY+Evn+LzOPIXdQE1uu7umuvH4QdwIqIPs3fIXtz84yZGPBoBM2uzbO1j72WPVita5XJkRETZZ2ppitrja8Pdz13pUOgjcHDMQTw8/RA9D/eEkYnRO/u/uPUCa+usRb2p9VBnQp08iJCI3kd0WDROzTuFmiNrwru1d67+nqigd0GUalsKRqbvziGU94QQ8OngA6HlHWpERPQaC38Kk6T3/2AWvDcYSTFJKPtlWRkjovwuMSoRSTFJsHWzVToUymOSJMHZ2fmD8grRp65wpcIQGpHtot+bkuOSEbQxCMUbF4edh538wSmAeYUof7J2sUb9af/H3l0GRnGtfQD/r2Tj7u7ESJBA8ODu7lCKS6HFWkrlrRdaoNDSltLiUNyKB5eggQgJCcTdk41u1ub9kEtoSmR9Vs7vy72dnTnzDJuczMxzznP6Nft5aUoponZEwW+UH9x6uqkwMtKv0IFXxkNFVoVEST+gfkDLoE2DSOKY0Bi62q+kXEnBo22PMOnEJJnX34veE43kS8kYvHkwTJ2aLwHtNcALXgO8ZA2VUDIGg4Hw9W8GahyffBz8Kr5KBqBrI13tUwiCUB46+hOS+KMZkyn72gnX119HTXENSfxpgNwnuYg/Fo8OczrAxt9GaecR1Aiw0XojAsYHYOLRiUo7D6GemEwmHBwc6A6DIDRa1/e7ynxs9v1snFt4Dt3XdsfADQMVGBV9SL9CENqJV8ZD5A+R4JhyVJ74I/2K6o38Y6RU++ub6aPbym5KioYgFE9X+5UOczvAqbMTrH2tQYkpJF9OhpmLGeyD7SVuoyC2APFH4zHi9xFKjJRQNZFABBFfRHcYGktX+xSCIJRHnhyQrBgUWamUFhUVFTA3N0dpaSksLWVbNyH9ZjqYeky49VDtwzohvce/PcaFJRcwM2Km0kfJXVlzBXZt7dB+dnulnodQPyKRCOnp6fDw8ACLRUqwEISqURSFmH0xCBgXINOMQXVE+hWC0Ez3N99HyuUUjDs0DkbWRm99LqwTgpvJhaWnJZhs1T6Ekn6FIAhFI/0KUJxUjO3+2xE0KQgTjkyQ6lhBrQB6hnqt7ndv4z2UZ5Rj+PbhsoZJKAFFUdjZeSfcerlhyJYhdIejFUifQhCEopWVlcHKygpcLhdmZmYqOSeZ8afBPPp40B0CIaH2s9vDe5A3TBxMlH6uQT+0vjA3ob0qKyvpDoEgNNbDnx8i614Whv48FMa2xlIfz2AwtHLQBelXCELzcDO5yLyXCY4xp8nP2fpsWPtaqziqN0i/ojqUmMKTHU/g0M4Brt1dJT7u8srLSDiegPfT3yfryxMaQdf6ldwnuairrIN7L3cw2UzY+NlgxI4R8BniI3VbkiT9gPrB51n3sjDs52GkX1AjdRV1YDAZEAvEdIeiVXStTyEIQvuofo4hoVBioRiCWgHdYRCt0DPSg5W3VbMvXwiCIAj6FcQW4OU/L2FgbiBXO7VltYj6IwqkqAJBEHQZ8tMQrKtcB7ZB8+M8a0trkRWZRfoqLVdbVosLSy4gek+0VMfpGenB3NUcdZV1ygmMIAi53NtwD/v670NNSU3DttAFoTB3M5e4DR6Xh+eHn6M8o1yi/cfuH4u1JWtJ0k/NGJgbYP6j+Rj689CGbYmnExG9N5q+oAiCIAjakcSfBsuPzsc3Rt/g4daHdIdCtIKbxUVFdoVKXqyUpZZhd/huPP71sdLPRRAEoU1G7RyF1YWrweLIV87l7nd3cW7hOWTeyVRQZARBENJrbQH5ax9fw64eu1BdUK2iiAg6cEw4mHl1JsKWhUl1XL+v++Hde+/KPRiGIAjlCP80HCN2jICJfeOqQkKeEFUFVRK1kR+djxNTT+DFyRcS7W9kbaTy8tCE5P79dz/yx0hc++gajdEQBEEQdCOlPmnW2gN5S8zdzeEzxAcWnhaKC4hQivOLziP1WirW165X+rmMbIxQGFeImgE1re9MaBUGgwFXV1e5+hWC0HWKmJndeWlnWHhYwK2n5q/BS/oVgtA8FEUhek807Nrawbmzc7P7BYwPgKWXJZh6qn2JS/oV1WLrs+HVX7lrjBME3XSxX7EPsYd9iH2jbYJaATZYboDfKD9MPDqx1TZsA20x/u/xcOzoKNE5hXVC5D7OhYGlAeyC7GSKm1AssUiM84vPw3+sP3yH+jZsH/LTEAh5Qhoj02y62KcQBKFcdPQnDIrUdqFFRUUFzM3NVbqgI0GfZ7ueoSytDP2+6qeS81EURW5QCIIgpJD9IBulKaXwG+kHfTN9usMhCIKQGY/LwwaLDWj/TnuM3j2a7nAImomFYjBYDKmfDWpLa/H0z6dw6OAA74HeSoqOIAhZVBdVw9DKEEzW2wM3zi85D9sgW4QtlW6WryQq8yqx2WkzQheFYsRvIxTePiG9/Jh87Gi/Az0/7on+3/SnOxyCIAiiGXTkgsgcfZqJRCK6QyBUoMO7HVSW9APoGUVA0E8kEiExMZH0KwQhg+g90Tg14xTqKhS3lhGvnId/Fv6DgtgChbWpaqRfIQjNw9ZnY+q5qei0pBPdoTSJ9Cuq9eT3J/ha/2tk3pOu/LSIL8LVD68i8VSikiIjCMXRtX7l9OzT2OSwCWKh+K3Phv86XOKkn0gg3b+XiYMJBm0ahHYz20l1HKE8Du0csDJ3pdTlnImW6VqfQhCE8tHRn5BSnxou/lg84o/EY9Rfo8j6C0SD8oxyJBxLgM9QH1KCQ8fweDy6QyAIjdT1/a7wHe4LMxfFjbwqTSnFsz+fgWPMweDNgxXWrqqRfoUgNAvbgI02w9u0uh9FUdg/cD/MnM0wZu8Y5Qf2L6RfUR0zFzP4DPGBqaOpVMcZ2xvj3ch3YeVjpaTICEKxdKlfcevlVl+qWc719rYHbIeVtxVmXJ4h0f4MBgPdVnaT65yE4jXVvx8YcgDZD7LxUflHNESkHXSpTyEIQjuRxJ+GK0kqQeKpRIR/Gg6Hdg50h0M0QSwSY1//ffAd7osea3qo5JxlKWWIWBMBBotBEn8EQRASsPG3gY2/jULbdAp1wryH8+AYKtm6KQShaEKeEGwDcruvayQt+c5gMCDkCckaQFrOf4w//Mf4S30cg8GAazdXJUREEIS8eq3r1exnFdkVOLfoHHyH+aLzks7N7kdRFJzDnGUa9EZRFEABDCapNES31KupsA20halT4+Sfa3dXGFkb0RQVQRAEoQ7ImwAN1/X9rui+pjvY+uSrVFf8Sj6K4otgG2irsnM6hzljzp05sAlQ7EtsgiAIbaWsBIlTJyeFt0kQLanMrQRLn4Xr668j/mg81hSukXtGAKFZonZEIWJtBKadnwb3Xu4t7vvu3XdVFBWhiUR8ESqyK2DhaUGWEiAIDWFgaYC062mtvn9gMBgYf2i81O0/P/IcF5ZewOSTk+Ee3vLfGEK56irqsH/gfrSd0hbj/278Xfb+rDdNUREEQRDqgmSLaMZkyvcihmPCUVAkhLIYWBhgTdEalZ6TY8KBW083lZ6ToB+TyYSXl5fc/QpB6Bp+FR/fmX6Hjgs6YuSOkQpvP+NOBpLOJGHgxoEaNzKa9CuaRSwU4+iEo6jMrUS72e3g0ccDtaW1MLYzpjs0QoWM7Y3h0sUFxrbq+b2TfkW1bn11CwDQ+1PpXwKfX3oez/58hjXFa8jMEUKt6VK/cmXNFVRkVWDcgXFNDuzhGHPwEfcjsPRYSjm/iYMJrNtYQyx6e31BQsUYwIgdI2DhYUF3JFpHl/oUgiBUg47+hCT+aKaIkZM5j3Mg4ovg1oMkeog3xEIxqgur3yr5QGgvBoMBMzPFrU9GELpCWCdE8LRgOIUqZ3bey39e4v6m+wiZEQKH9ppVlpv0K5qFwWKgw9wOKE8rR98v+tIdDkGTgLEBCBgbING+3CwuEo4lwLOfp8r6J9KvqFbsvljoGenJlPjzGeIDfTN9gFJCYAShQLrUrxS/KEZZSlmLs/klSfrF7I9B3tM8hK8Ph5GN5Il9j94emBs5V+L9CeXRN9VH6ILQJj97+udTFD4vxJCfhqg4Ku2gS30KQRCqQUf1DAZFUeQ2ngYVFRUwNzdHaWkpLC0t5Wprk9MmmLmYYf6j+QqKjlCk8oxy5D7JhWt31yYXXVaWg0MPIv1WOj6u/piU5tERIpEICQkJCAwMBIulnBGeBEFIj5vJhaBWABs/zSi/XJ5RDiNrI3BMOKRfIQgtlxWZhV09dmHAhgHosVY1a1GTfkW1+NV88Cv5MHEwoTsUglAaXetXhHXCFpd7qSqoQtKZJLh0c4F9sH2T+xwcdhA5D3OwKn+V0mYHEsrV0pq+R8YeQdI/SfhM+JmKo9IOutanEAShfGVlZbCysgKXy1XZwAIyZ1kLDNgwAOGfhtMdBtGM9JvpODbhGHKf5Kr0vH6j/RC6MBSiOpFKz0vQSyQi3zdBqBtzN3ONSfoJ64TY0WEHSl6WAADqKuvATeXSHBUhCYqiwK/mN9qWcDwB+wbsQ11lHU1REXR48NMD3P7mtkT72rezx+ybs9FxXkclR9UYuV9RHY4xhyT9CJ2gS/1KS0k/AChPK8e5heeQdCap2X2mnZ+G+U/my5T0e3nuJc7MPQMRX3f+zdXRnt57sH/Q/iY/G7VrFFbmrFRxRNpFl/oUgiC0E0n8aYF2M9vBb6Qf3WEQzfDo7YFxB8cprYRcczot6oQhW4aAbUAq+hIEQbQk8XQizi0+h4qcCqWdQ1gnRPLlZFTlVyntHIpQnl4Otj4bry6+glgoxs+ePyP6y2i6wyIkwCvj4TuT73Bh2YWGbRU5FciKzEJRfBGNkRGqFrM3BtG7oiXal2PMgUdvDxhaGSo3KIIWghoB8p7moba0VqbjeVwejow9gshNkQqOjFAnYqEY1z6+hoPDDkLIE9IdDtGChBMJiD0YC5Gg5YSEQ3sHTDk7pdkykEB9yTFLT9mqT+U8ykH0rmgUJxbLdDyhGCYOJjCxb3pgh6GlYbOfEQRBELqBZAS0CCWmwGCSko7qxsLDgiy2TBCE1Ooq65D/LB/u4e50h6L1Mu5kIOr3KPT8qKfSzvHqwiscHXcUQ7YNQZf3uijtPPKy8bPBytyVEAvEYLKZCFsRhnJBOd1hERIQCUQImRkCp85vBhp1nNcRnRZ1anVmAKFdZl6dCX4lv/Ud/0csFKO2tBbGdsZKjIqgQ1FCEXZ23on+3/WX6W8cx5iDVxdfwdCGJIa1GYPFQNyhOJi5mNWXkCQDR9VW5MZIlCaXInhqcIv7sQ3YzQ4OpygKt764Bb9RfnDs6ChTHGHLwhD2XhiMbcnfDTpNPDqx2c8qcyvB4/Jg429Dln4hCILQUWSNP5q8XuOvvLwc5ubmcrVVEFeAg0MPousHXdF9VXcFRUhouorsClx6/xLajGiD9u+0pzscQgUoigKPx4OBgQG5udcC97fcRx23DmHLwmBkY4Qrq6+g/3f9yRocSkCJKdQU18DQ2hBMlnKKIfCr+Hj0yyMETgyElbeVUs6hDKRfIQjtt6f3HhQlFGFN0RqVnI/0K6rDzeIidn8sPPt5wqWri0xtiEVipf1tJNRHXUUd2IZsjb3P1JV+pTy9HGVpZfDs6ynR/twsLvhVfNgG2DZsy3uahz9C/0CX97tgyJYhygqVoNnJ6ScRdygOn/A/0djfazrpSp9CEITqcLlcWFhYkDX+COmYOZvB2NYY+mb6dIdCNOHUzFPY4roFlFi1OXaWPgsvTr5AYXyhSs9L0IvD4dAdAqEg3Ewu7n5/FywOC5n3MvF4+2PkPMqhOyytxGAyYGxnrNQXmxwTDnp+1FOtk37VRdW4+tFVFMQVNNrO4XAgFolpioqQV3lGOR789ABVBepdZpZQDJFAhML4QvC4PImP8Rvjh7bT2kKV40HJ/YpqmLuao9fHvWRO+gEgST8t9vSvpzgz9wyqi6qhb6YPlh5Lpf2AoulCv2LhYSFx0q+2rBbbvLfh6tqrjbY7dHDAvEfz5K5Awc3kIuN2hlxtELKLPRiLG5/fAL+q6Rn+fmP8EP5pOElayUEX+hSCILQbuYunmVgs/4s0QytDLHy2EKHzm6/fTtDHzNWsvryCisuwGtkY4RPeJxj0wyCVnpegj1gsRlxcnEL6FYJ+Q7YMwdritdA30wfHhIMu73eBiQNZp0EZsu5noSytTCXnosRUq+uy0CXtehrubbiH3Me5DdvEYjEOTD2AX/x+0eiXgbogZn8Mzs4/i5rimkbbM25l4PIHl5F6NZWmyAhV4mZy8Vvb33DnmzsSH9Ptg24YunWoyl4OkvsVzVKRU4GYfTGoyFbeOrgEPbIisxC7LxYck/qX2ze/uIlNjpsgqBXQHJn0dKFfqcytRG2Z5Ot1GloaovfnvRG6sPF7IgaDAefOzrD0km19v9dOzTqFQyMOqXyAM1Hv+d/Pce/7e2DpNz2bL2hiEPp+2RdMNnntKwtd6FMIglAtOvoT8heAIJSs/7f9MTNipsrPy2AwwOKQkg4Eoclev4hxaOeAAd8NUOvZYpqKoijs67cPF5ZeUPq5Xp5/iW9NvkXiqUSln0sWQRODsCBqAfzH+DfarmeiB0tPS/DKJJ9BRKhexq0MPPvzGZh6jW/vfYf7YsaVGfAZ4kNTZIQqcUw4CP80HF4DvegOhVADd7+/i109d701IEAaWfeycHr2aaTdSFNgZIQ6GP3XaKwuWA09Qz0AgL6pPiy9LFGVT2aIq6Mbn93ARquNqC6slviY8PXhaDOiTcN/Pz/8HCkRKQqJJ+y9MAzeMhhiIUmM0GHS8UlY+GwhKeNJEARBNIus2qwlsh9mI2ZvDLqt7AYrH/JimKhXnFiMvGd5CJoURMr0EIQGyY/JR+HzQvgO84WhpWHDdoqiQIkp8vusQJSIwsAfBsLEUfmzKS3cLeDey11tS3MzmAw4dnR8a3vg8kAEBweDxSIvFtTZyJ0jMWDDABiYGzTabmRtBO+B3jRFRaiaib0J+n7ZV6pjcqNyce/7e+i8tDM8+ngoJzCCFrWltSh5WQKOqezlytx7u2PisYlw6+mmwMgIdWFo9eY+s9vKbui2spvCz5H3NA8pESno+WFPhbetSzz7eYLFYcHYzljqY7lZXBhaGeLS+5fAMeZg6Yulcg8SDhwfKNfxhHzYBmzYBto2+3nkpki8Ov8KU89ObRhMShAEQegW8uZQS5SnlePJb0+Q+yS39Z0Jlbq67irij8XTcu7Hvz7GyWknUZVHRm0ShKo82fEEN//vJkQCkcwjpuMOxeHUjFOozKls2JZ6NRWbnTfj5T8vFRUqAYDJZiJsWZhKXl7YtbXDjMvqOfOqMrcSWZFZZNS2BmMwGDCyNmryM4qiyBp/RLP4lXwknEhAycsSukMhFGzgxoFYU7gGbH3Zx/ua2JsgcEIgKTeuZQriCpB4OhF1lXVKPQ8lpnD5g8u4vv46il4U4dSsU7i67mrrBxJvCZ4WjOG/Dpf6uDvf3sFPbj+hPL0cs6/PxpSzUxRaGUiaUvDZD7KReS8TwJs1aQnpVeZWIu9pHoR1wmb3qciuQEFMAbm3JwiC0GEk8UczJlMxX4HvMF98kPUBgiYHKaQ9QjEENQLc+/4eXp6l50V9yMwQTDgyAfrm6jm7hFAsJpOJ4OBghfUrROsoMYVrH1/D7vDdAOofYJ/99QxxB+Ow3X87dvfaLVO7YcvCMO7gONgE2DRsM3czh6GlIcQi8vBGKN7zI8+xq8cupFxpXP7pdb+ScjkFh0cfBo9Lyn2qI4qikBKRgvKM8iY/PzXzFDY7bYaQ1/wLIkI7RO+Jxr4B+1CaUirxMW493fBJ3ScIXaCa9cLJ/Ypm4pWT/l+bxO6PxZGxR8DN5DbannwpGadmnpL778Xrdd8YTAYmHp+IhU8XwsbfBrmPc1H0vEiutptC+pXmuYe7I3haMBhMBmwDbWEXZKeQdimKws6wnTgy9ohE+/O4PBwYfABn556FWCTGsYnHsLvX7mbvXYjmPT/8HH+E/oGsyKxm9xmyZQjWlqyFgYVBs/sQzSN9CkEQikZHf0JKfWoJfTN9tS0dpsvYBmwsT1n+1no7quLc2RnOnZ1pOTdBDz6fDwMDcnOvKhRFofTVm5erLD0Wpp2bBgB4+tdT1FXINora3NUcwdOCG22zbmONJfFLZA+WaFL8sXjc+foORuwYAZeuLko/3/PDz5F0Nglj940Fk60+D5LeA70R/lk43Hu7v/UZn89HWXIZks4mIS8qD579PGmIkGgJr5yHA4MOoP277TH6r9Fvfe41wAv65voQ1AjANiC3/9qsMq8SuY9zwWAyJD6Gjr6I3K8oH0VReLz9MRw6OMCth3xlOiM+jEDkxkisyltFZv5piY7zOsLK1wo2/jaNtuc9y0PsgVh0XtYZLl1kuy8SCUQ4MOgAuqzoAv8x/jC2NYaxbX15yvmP5yut7KA29yvRe6IRvTsaw38fDtuA5ss7NsWtp5tSSvUyGAwY2RhJ3CcYmBtg9O7RsPG3AZPFRKdFnWDhYQEzZzOFx6bt3MPdEf5ZOBzaO9AdilbT5j6FIAjdwKCkmZdPKExFRQXMzc1RWloKS0tLhbRZVVCFwueF8OrvpZD2CO1BURQYDMlfABGaSSQSIS4ujqzFpWLCOiEoMQU9Qz2FtFdXUQdKTJHRmSoSsz8Gt764hYnHJsKxw9vr2ynalTVXcP/H+1ieshyWXor5+69Mr/sVf29/sNgshf2cE4pVV1mHuINxsPK1IveBhExyHuVAyBPCPfzt5L+ikfsV1agprsEPtj+gw7wOGLVzlFxtxR+Lx8uzL9H3676wcLdQTICEWqoprgFFUQ2JOlkUxhdi/4D98BvjhxG/jVBgdM3T9n7l3sZ7uPPNHSxLWqZxyXdKTEHEF5GBRyqWH5OP0lel8B3uS+7fZaDtfQpBEKpXVlYGKysrcLlcmJmpZtCL+gw1J+R27aNr2D9gP2qKa+gOhfgffhUf3CwuRHwRLecX1Aiw2Xkzzs47S8v5CUJbxeyPwe2vb6Ousg5sfbZCH6biDsVhg+UGpF5NfeuziuwKnFt0DgknEhR2Pl3XbmY7LE9erpKkHwD0WtcL66rWqU3SLz86Hw9/fthQkqs5ekZ65KWBGtM31UenRZ1I0o+Q2enZp3Fu0Tm6wyAUiGPKwazrs9DlvS5ytxU0MQhj948lST8tIagRNFvK08jGSK6kHwDYBdnhg+wPMHjz4CY/T72WinOLz7V670G80WNtD3xY9qHGJf0A4NmuZ/gt5Ldm1/OryK7Ai5MvVByV9nu26xmOTTyG2tJaukMhCIIgaEISf1okeHowhv4yVK1Kh+m6Vxde4Se3n/DiFD03snpGerD2s4a5mzkt5ycIbRW7LxaRP0SCxWl+9F9xUjGOjDsidZLO0tsSbae0hX2I/VufsQ3YiPojCmnX06SOmVAPhlaG4Bgrp8SVLG58egNXVl5BycuSVvfNe5qHrPvNryVCqC9KTOH0O6dx/ZPrdIdCKFnS2aQmB460ps8XfdDv636KD4igDVufDc++nk3eTxC67fmR5/jO7Ltm+4rK3EoknEiQa11pJovZ7ICh1KupiPo9Cvkx+TK3r4ukKeGsSne+vdPi/YWwTghQgJlL07Mbrqy+gqMTjqK6qFpZIWqVgtgCbHHdgph9MS3u1352e4w/PB6GloYqiowgCIJQN2SuvRbxGuAFrwFkpLc6sfS2RJcVXRS2gLYsZl+fTdu5CdUjZShUY9qFaShJKgFbv/k/owwmA0lnkuDUyUmqtr0HesN7oHeTnxnZGOH9jPdh7kqS+YrybPczcEw4CJoYpLJzFsQWQFAjUMmagq0Zd2gcMu9kvrXGz7+97lcOjz4MY3tjLHiyQFXhERK6u+EuYvfHYtr5aU3OyGEwGci4laE2M00J5bm04hIMLA2w8OlCqY4LmqS6PhAg9yuqIKwTgqXHUliy4PY3t1H6qhRj9oxRSHsEfUzsTeAzxKfZv/0Ptj5A5MZILH6+WOrn2BenXiDnUQ66rezW7MzBLsu71K/vpuAZpNrar5S8KkHSmSQEjA+Apaf6/R1/ee4luJlc9P2qb5PLi4QtDUPoglCw9Jr+fjov7Qz/Mf7QMyKVJSRRV1kHIxsj6Jvpt7ifY0dHOHZUTUUTbaWtfQpBELqDrPFHk9dr/CmjritZz40gCIJ+lJiCWCRu9iGXUA9b3LbAxN4E8x/PV9k5NztvhomjicYl0KL3RkPfVB8B4wLoDoX4j3sb7+Hpn08x/9H8ZtcHFdQKSLlWHZByJQUA4D2o6QEkhO649dUt3P7yNhZGL1TIIMQjY48g404GVuevJhVmtFze0zwUxBagzcg2MLI2kurYE9NOIOF4Albnr4ahFZlppAiPf32MC0svYNr5afAd5kt3OG+pyK6Asb3xW888RQlFMHczB8dEfSpdEARBEARdlJkLag5J/NHk9ZddXl4Oc3PFzdz4e9TfqOPW4Z1b7yisTUKzZdzOQMLxBPT8qCdMnUzpDodQIoqiUFlZCVNTU5L8V6KoP6Lg1ssNtgG2Cm8772keItZEoPva7vAZ7NPkPoIaAdJupMHM2QwO7R0UHoOuyXuWB7FADOcwZ5Wd89nuZ9A300fg+ECZjqcoCqDkK/nE4/IQ9UcUAsYGwMrHqsVzkX6FILTf88PPEbEmAuMOjYN7L3elnov0K6oR93cc4g/HY9SuUVInb5pSV1kHjgmHfGdEiygxhaKEIti1bTnZXJFTgYKYAnj292yxgobE59XifqWusg55UXlw6OAAA/OmB/eoG0GtAL+1/Q1sAzYWxS4Ck9X6YIHqomoY2Rhp3fdHlzvf3sHDrQ8x98FctZwpqu60uU8hCIIeXC4XFhYWKk38kaF6NBOLZa+b3xR9U30YWGrGzaAuuPPtHRwZewQigYi2GPJj8vHo50coTiymLQZCNcRiMVJTUxXerxBvcDO5OLfwHO58fUei/YsSihB3KA6SjrGpzKtE7pNc8Kv4ze5TVVCFv0f8jaidURK1SbTMsYOjSpN+ANBhTge5kn4Xl1/E+SXnUZlXiZh9MRL/fP1b2vU0XF17tWF2UHNIv6I9aktr8eLkC4nWcyQ0kzzjOfXN9esHiKlgSCjpV1QjeGowppyZopCkH1D/nElefmq+4sRiHB59GMmXk1vdV5ZnWAaT0WrSDwCe/PYEh4YfQumrUqnP0RRt7lf0TfXh0cdDrZN++TH5iD8W3/DfLA4LXVd2RedlnSVK+t3dcBc/2v1I3lm0gqIoXFl9BUlnk1rd18jGCNZ+1mSGtoy0uU8hCIIedPQnZI0/LTPu4Di6QyD+pfB5IVIiUmi92QqZEQL/Mf4wcTChLQaC0BamTqaYGTFT4tJFD356gKc7n8JrgBeM7Zpe5+Tf2gxvgw/LP2zxxauFhwVG/DECbj3dJA2baIawTghKRNG2pogspblFfBFKkkogFohx/ZPriN4VDYcODrAPtpeqHZ8hPphxeQbsgiUv/5Z8ORnnFp7D0G1D4TfKT6rzEcpDURTufHsHTp2cmp0pDABFL4pwdPxRDNg4AD3W9FBhhISq5D3Nw57eezDg+wEIWxYm1bG+Q33hO1T9SsgR6kMsEiPtehpYHBY8envQHQ4ho7K0Mry68AqBE1segPTPgn8QfzQea0vWSpS4AYDITZHw6u8lUUUK/zH+MHczh7F96/fHukxQIwA3kwtrP2u1Trxffv8ych7lwH+0PxhMBphsJsKWSv53yKmTE9pOaQtKTAqStaQytxL3N90Hv5rf6r146IJQhC4IVVFkBEEQhDoiiT+CUKLxh8bTvuaioaUhDC3J+goEoQhMNhNeA7wk3r/D3A7wHuQNPWPJE0sMBgNooctgMBgInU8e4hQh9Woq/h7xN0bvGY32s9ur7LwFcQU4POowuqzogq7vd5XqWLY+G1NOTwEAlCaXwmuAFyy9pC/fo2eoJ/UaYAbmBuTviRqq49bhxic30P7d9i0m/uxD7DFm7xi49nBVYXSEKrH0WHDr4QZTZ1LanQDOLT4H+2B7dF7SWSHtMRgMHBlzBK49XEniT4P5DvXF+tr1rSZYrHyt4NnXE3XcOokGvFXkVCBidQSCpwdj3IHWByM7dXKCUycniePWVek303Fo+CEM/204Oi3qRHc4zQr/NBxioRg3/+8msiKzMDNiplTrnHv194JXf8mfsXSVqZMp3s94X64Z/gRBEITuIIk/LVOcWIyE4wkIGB+glPWnNMXtr2/D3N0c8Yfj0ffrvnDs4EhbLHSPzBOLxOBmcsHisGDmrJoawgR9DAzUtwSMJqPEFGL2xcC+nT0c2jtI/Hvt0sUF6CL5eRLPJIJjwpHowVfEF0FYJ4S+qb7kJyAaMbY1RvD0YJX/vTSxNwHHlAO2oXS3YSKBCCw9VsMMRfsQe9iHSDfTD6gv+VhTXAMrXyuJfpZf9ysuXV2w8NlCqc9HKBfHhIOFzxa2OnNV31Qf7Wa1U1FUBB3sQ+wx4/IMmY6lKAoPtjyAka0R2s1U/s8JuV9RLrFQjKgdUQgcH6i4xB+TgZF/joSFu4VC2iPoI0k1Gmlnhps4mGDu/bm0VVEAtLNfMXM1Q5f3u8C9t3LXXpWXZz9PAMCrC6/Ar+SDV8aTqNoJIR0GgwFzN3OJ9s24k4H0G+kIXRgKE3tS/UkW2tinEAShWxgUGSpCi4qKCpibmyt8QcfEM4k4MuYIRu8ejfbvtFdYu5qkKKEIvwb9Cvt29iiMK8TgLYPRZbkUb94VhKIoJBxLgG2grUTrHChLTXENfrD9AaELQzHi9xG0xUEQmoxfzcevgb9CxBdh9o3ZsPG3kep4SkyBwWw9wbLVayv0TfWxKGZRi/sVxBZgZ9hOhH8SjvBPwqWKhdBcp2aeQm5ULhY8WdDoxVplbiUYLIbED/VPdjzB+UXnMeXMFFKyUweJ+CIw9Zi0D0wi1M8Ptj/AJsAGc27PoTsUQgEEtQIIa4USlycndEPKlRTom+nDpasL3aFgd/hu6BnqyTxggVA/IoEIlIgC20D6OQYZdzJw9cOrGLhxIFnSoBkFsQUwtjeW6J7/9je3ceOTG1gYvRAO7Vovv0sQBEEol7JyQS0hM/5opuiFHT16e2BB1AJY+VoptF1NYhtoi/lP5sPAwgBsAzZts9x4ZTwcn3wcHeZ1wKido2iJAQAMrQzRbVU3UtpLB4jFYpSVlcHS0hJMJlnEW5E4xhxMOTMFFp4WMDCXfOSfkCfEVs+t8B7kjTF7x7S6/6g/R0EkELW6n3Uba3j284Slt/QlHgnNZexgDHM380ZJv9yoXOzsXJ8E7vtlX4nacezgiE5LOsGtV+svVf7br8QdigM3i4ueH/aU+ToIxeJxeWCymOCYcFrd9/qn13H3u7v4IOsDmDqScpDa5sWpF8h9nIsea3vAwEL6UeozI2bC0Fr5SSJyv6IaeoZ60DNU/OwriqIgFoqlKuNHqI9zi85JNMgMAO5uuIvqwmoM3jS4xf2EPCEK4grgFOok0UC310zsTWRKEDWF9CvqgaXHAmTsdkR8EYoSilCZW6nYoLTIsUnHQIkovPfqvVb3DZ0fCv/R/uR5UUakTyEIQtEUnQOSBEn80UzREy4NLAzg2JG+spbqwimU/vUC2AZsjD88XuJSDMrCYDIw6MdBtMZAF7rXV1Q1iqKQlZUFCwsLukPRGuk301FTUgO/kX5waC/9SEm2ARv2Ifaw8LKQaP/XZXIkaXf6helSxXL3+7sQC8XovKQzGf3/P9c+vgYRX0RLH5l6NRWvLr5C3y/7gmPcesIGAAb98HacDu0d0P6d9lKNjHYOc4ZzmLNE+/63X4nZG4PMu5nosbaHTvWv6uz+5vu4/eVtLI5b3GqFAdsAW/iN9IOwVqii6AhVSr6YjKc7n6L76u4yHS/L3zlZkPsV5avMrURFTgVsA2wlGhQgqbTrafh71N8Ytn2YStfGJRRnyNYhra7v91rqlVTkR+dj0A+DWkzopV1Pw6HhhzBk6xCpKu1MPDZR4n1bo439Ss7jHJycdhIDNg5AwNgAusNROs++nviw7ENyf9mCLsu7tLgW/L8Z2xmTcqty0MY+hSAIetFRdJMk/rQQv7q+prqZi+6t5xa1Mwo2fjZw6+XWcMOYdiMNd7+7iwlHJsDQUnUvu/WM9NB2cluVnY+ol/MoBy9OvUCnRZ3wz7x/0H5OewRPC6Y7LEJD3f3uLtJupGFV7ioY2RjJ1Iak5YvEQjEYLIZSHnYpioKesR4uLb+EoMlBJPH3P0lnkupfZP2o+nOn30rHg80P0GFOB7nKQTNZTIzeNVri/WtKamBgYQAmS7aRq0O2DSGzPNSMfYg92s1uBzPX1u/7gqcFk7+JWmzQj4PQbVU3mWb7AfUl2qryq2DiYEJ+zzVcwokEXFp+CbOuz4JnX8kGFUnC3M0cLl1dVPpMRSiW30jJS3yPOzQOhlaGrc7is/KxQo8Pe8B3mK+84RH/UltaC4qiJB4gpumkmS2qq6RZs1VQK4CgWgB9c33yN50gCEJHkfnKWmhH+x04OPQg3WGoHI/Lw7WPruHS+5cabedmcpF+Mx2ZdzNpiox+1z+9jn0D9tEdhko83PoQDzY/ADeDi4K4AuQ9zQNQP/K5PL2c3uAIjTN692hMODJB5qSfNGL2x+Bb42+Rei1Vov2Lk4pxeMxhxB+Nb3VfBoOBzos7453b78Da1xpA/ToaD7Y+gFio+nID6mLx88WYc5eetaw6L+mM5SnLJV4v8vzS87j5xc0W96kuqm61ndOzTuOPjn9AUCuQ6Lz/ZeNnA0svSzIaW40Ejg/EmD1jpCpDTGgnfTN92PjZyPzy9NaXt/CT208oeVmi4MgIVXPr4YZ+3/aDbYCtQtu18rHCrKuzJFof9tjEY8iNylXo+XVdVUEV0q6nQSyS7d5N2pHmJvaSDQKwbmONAd8PgJWPdMuNlKeX48ZnN5AVmSXVcbrCZ7APlicvh9dAL7pDUZnsh9l4uO0hLbMi1J20v/dPfnuCH2x/QO4T0g8TBEHoKpL400Lt57RH0OQgusNQOQNzA0w+NRlTTk9p9EKy7eS2WF2wWqrRjYrwYOsDbHLahPzofJWetykVmRUoTizWiRf8o/eMxqzrs+Ae7o4lz5dg0I+DUJlbia2eW3H1o6t0h6d0pqZkzSZFMnUylbu0Ttb9LJxfeh6lKaUt7mdkYwT3cHeJZ2tzTDh4ee4luFncFveL3huNqoIqMNlMuPdyb9h+97u7uPrh1VaP12YMBoO2ZImpoyksvSzBZLd+KyYWivHy7EtkR2Y3u8/FFRex1XMrastqm29HJIZdiB2cOjtJte7Tv/sViqJQmVeJmuIaiY8n1AdFUYj4MAL3frhHdyiEEpS8KpHrd9O9lzu6rOiiktkl5H5FuRw7OqLXul4wcTCh5fyUmELa9TQ8/fMpLefXVjH7YrCv/z781e0vXFxxUerjo3ZE4Qe7H5B1X7JEGyWmkPMoBxl3MprdR8gTypykqSmpwe2vbks86K012tqv6NJgq2e7nuHSikuoLmh9MJuu2d1rNw6PPizx/vbt7BG6KBQm9vT8HdAG2tqnEAShOxgUGUpDi4qKCpibm4PL5cLMTPdKciqKSCDC+SXn0X11d9j4STZrQlWi90Yj6vcoTDgygfZ1/nRBa+v5RayNgHu4O9qMaANKTJFSIkSLKIpC0pkkeA/yhp6RjCvU/0/Mvhicnn0ak05OUvj6HDXFNS3ORixKKMKvbX9FmxFtMPXs1Eaf8av5yHmYI/G6gtqmuqgaBbEFcGjnoJIZnf9FURSqC6ohqBXA0tOy9f3FFGpLa5uNNfZgLF7+8xIDfxgIc1fl/c3JfZKLnZ13ou/XfRG+Plxp5yEkd3r2aVj5WiH8E8m+j588foKRtREWRC1QcmSEqm2w2gCHdg6YfWM23aEQWizhRAISTyZi1F+jwDZoeuUQsVCMu9/fhXUbawSMC8CVNVfQ86Oe5AW0nEpelSDuYBzSrqehMqcSS+KXNPsdNOX54ed49PMjjN0/FpZeEtx7UBS2uG4Bi8PC8pTlTT5r3fn2Dh5vf4xZ12ZJXMXgNWGdECVJJbDysZL7flvbcLO4ePDTA7Sb2U5l66+qg4LYAtSW1cKlqwvY+mRlotfEIjGOjjsKQytDjN4teYl/giAIQn3QkQsiiT+avP6yy8rKyGKxckiJSMGBwQfQc11P9P+mf7P7CWoEiD0QC2M7Y/iP8VdhhISqnJxxEgYWBhiydUiza1eJhWLs7bsXps6mmHB4goojVD6xWIzCwkLY2dmBySQTuuWR8zgHf4b9ia4fdMXgzYPlaotXzgO/mg9TJ1OVj9ilKAovTr6AbaCtwst9aboXJ1/g6PijGHtgLEKmh6j8/JSYwrfG38KjjwemX5yu9POVp5fD1MkULI50a3z8t1/hlfNw7eNr8BvtB5/BPkqKlpAURVHYYLEB7uHumPrP1NYPAFCRUwFjW2OpfxYI9UZRFG5/fRumjqboOK8j3eG0iNyvKN+BwQdgZGuEcQfGKbzt659ex52v72BJ/BLYBrZ+b5FwPAHHJh5Dny/6oPdnvRUejybJe5qH84vPY9CmQXDr6SZzO1X5VTCyMZKoaoC8Ek4kwNjWGG693Jq8j43eE43IHyKx8NlCWv+uaFu/EnswFqdmnMK4Q+MQPJWszUsQqqZtfQpBEPQrLy+HpaWlShN/GtN7/fbbbwgJCYGZmRnMzMzQrVs3XLz4prwFj8fD0qVLYW1tDRMTE4wfPx4FBQWN2sjMzMTw4cNhZGQEOzs7rFmzBkKhsNE+N2/eRMeOHaGvrw8fHx/s2bPnrVi2b98ODw8PGBgYoEuXLnj06JHM16WMvGvG7QwcGn5I4hIemsx7oDfm3p+LPp/3aXXfyysv48GWB8oPSg1VZFcg7u84lGeU0x2KUgh5QpSlloGbwW026QcATDYTbAO2VCXuNAlFUcjPzydrIiiAlbcVhmwbgg7vdpC7LQMLA5g5m7Wa9Lu4/CKi/oiSuv3nR57jyuorTX7GYDAQOD6wxaTfsUnHcGzSManPq+ns29lj6C9D4drNlZbzM5gM9PioBwInBba676Ptj6Ran4NfzW/035SYwpGxR/B7+9+lLvn8337FwMIAw38dTpJ+aoLBYODD8g8x8dhEiY8xczYjST8txGAw0PvT3nIn/a6suYJDww8pKKqmkfsV5RPWCSHii5TSdreV3fBRxUcSJf0AoM3INpj7YC66LO+ilHg0iW2QLcxczHBmzhmIBNJ9P6XJpQ1/w00cTFSS9APq15F1D3dv9j62/Tvtsfj5Ypn/rvCr+MiPkX9pDG3rV4KnBmNJ/BL4DvWlOxSVo8QUakpISXl5pV5NxfHJx1EQV9D6zsRbtK1PIQiCfnT0JxqT+HNxccH333+PqKgoPHnyBP369cPo0aMRHx8PAPjggw/wzz//4NixY7h16xZyc3MxbtybEY4ikQjDhw8Hn89HZGQk9u7diz179uCzzz5r2CctLQ3Dhw9H3759ER0djffffx/z5s3D5cuXG/Y5cuQIVq5cic8//xxPnz5Fu3btMHjwYBQWFqruH6MVdZV1SL2WCm6m9q7bJOQJGx6WXLq4tPqgoWekh+kXpmPK2SmqCA8AcHfDXTz9Sz3Wtch5nIOT004i824m3aEoBduAjXfvvYtxB1sf1TzjygxSHoNolaGVIbq81wV2be0U0l5FTgWyHzS/PpuIL8KjXx4hNUL6NU4STyXi0S+PUF30Zi2MjNsZ+HvU36jMq2z1eF45D7xyntTn1XRW3lYIWxomUakrZenzeR90mNNycrmqoAoXl13Eg59aH7giEoiwq8cuHJtw7K3tgZMCETIjRGUvCgnVYTAYUpV6q6usQ9b9LFQXkvVziLdVZleiNKUUwjph6zsTauudm+9g4lHJBwRIw9DSEPqm+i3uU5xUjN29diPuUBzY+my4dHGBgQU9a+qqE7Y+G73W98KkE5PA0pM8UUaJqYa1/V6rLatFxNoIxB6IlawNisKF9y4g/mi81HFTFIW8p3mNBg9RFAV+Vf1AI3kqWlxccRE72u8giZ7/KUsrQ21ZLRhMBmwDbXXy92aL2xYcGXOE7jDUypl3z0j9bqk0pRTxR+PJeokEQRA6TGPe/owcORLDhg2Dr68v2rRpg2+++QYmJiZ48OABuFwu/vrrL2zevBn9+vVDaGgodu/ejcjISDx4UP+i7MqVK0hISMCBAwfQvn17DB06FF999RW2b98OPr/+hvX333+Hp6cnNm3ahICAACxbtgwTJkzAli1bGuLYvHkz5s+fjzlz5iAwMBC///47jIyMsGvXLlr+XZriM8QH62vXo+3ktnSHojT3N9/HX93+kuqllXu4OwzMVXfjHPlDJGL3SfYgpmwuXV0w8dhEuIe70x2Kwr1+AGUwGNA3a/klxOv9CKIl5enlqKuoU2ibp2aewv6B+5sd4cPisPAR9yMM/WWo1G0P+H4AVuashLGtccO25EvJSL2aCn4lv4Uj6824PAMzr8yU+ryEahhaGWLOnTnotqpbq/uy9FgwdzOHpY8lKPGbnzW2Phu91vVCr497KSSmBz89wK6eu6SerUAoXlV+FVKupDRK/Lcm9WoqdnXfheTLyUqMjFC1F6deYG+/vch7lidXO2P2jcGyxGVkbSWiRYXxhS0OaOJX8lHyqqRhYJGwToiSVyWqCk8t8bg83N9yH2xDNuxD7AHUr9X8OnnWEhFfhA7zOiB4xpuSjyw9Fh79/AgvTr6Q6Pw1xTV4/MtjpF6VfpDZw60P8UfoH0i7ntawLeVKCra4bcGrC6+kbu/f/Mf4o/f/6XYJ2NdqSmqwt89eHBh8QOoKDdokaHIQPPvr5hrkTakprkHcoThkRUpX0St0fig+FX5K/i0JgiB0mEY+0YlEIhw7dgzV1dXo1q0boqKiIBAIMGDAgIZ9/P394ebmhvv376Nr1664f/8+goODYW9v37DP4MGDsXjxYsTHx6NDhw64f/9+ozZe7/P+++8DAPh8PqKiorBu3bqGz5lMJgYMGID79++3GHNdXR3q6t68SK6oqABQXzdaJKp/ccZgMMBkMiEWixu9HH69/fV+rW1nMplgMBhNbn99Tkm2s1gsUBTV5Pb/xtjcdmVdk6BWgLrKOuhb6DcZY3PXVFtci/Sb6QiYEKD0a5r3eB5EfBHEYjHt35ORnRH8xvo1fKaq70mZ1/TasQnHoG+qj5G7RoLBfJPUa+maonZEIT8mH8O2D1PLa2pue2vfk1gshoWFRcM5tOGa6PjZu7jiItKvp2NF5oqGZLK819RxQUf4j/Wvf4j/z5Cb1/uzjdhgG7EbrkHSazJ1NQVQ/1KttrQWRnZG6PNVH7Sf2x4WHhYAoJXfk7zXdGjIITCYDEy7OI22a0qNSMXtL2+j/4b+8Ojl0fQ16bHg0t0FFEU1Okdz39O4g+NAgYKYEgOi+tmmhpaGMDA1kOma/t2vvI6dm8VFWUoZKnIrYOZipnN9hDpdU/qtdJyYcgLj/h6HwImBEl2TfXt79Pu2H+zb2UMkEqndNWnj96SKa6rMrUReVB7EInHD9yrTNTHx1rOJoq/pdb/CYDB07ntSxTWVJpci+UIy2oxoA2tfa6Vc098j/wbHhIOF0QubvCaHjg74IOcDAPU/T2fnnUXcgTisLVsLjilH6mv6d4ya+j1lP8zGlZVXQFEUrNpYIeVKCg6PPIzhvw5H6ILQFq+JocdAz497AnhTJoplyMKi54tg7mYuUV+ub6mP93PeByWipO4jvId6Iyw1DKYupg1xCuuEMHEwgV1bO7m+J59hPmgzog0AvLW/NN+TWCyGlZXVW/dLmtRH6Fvoo+OCjjB1NgXFkP57UsdrkuX3acDG+ndyIpFIa65Jnu/J0NoQH5Z9iJqymkaftXpNVP3214MB1emaNOF7oigKVlZWb7Wtydekjd8TuSZyTZp2TaqmUYm/uLg4dOvWDTweDyYmJjh16hQCAwMRHR0NDocDCwuLRvvb29sjP7++Xnx+fn6jpN/rz19/1tI+FRUVqK2tRVlZGUQiUZP7JCYmthj7d999hy+++OKt7S9evICJiQkAwMrKCm5ubsjOzkZpaWnDPg4ODnBwcEB6ejoqK9+UbXN1dYW1tTVevXoFHu9NmTYvLy+UPitFcmIybLrYNGz38/MDh8NBXFxcoxiCg4PB5/ORlJTUsI3FYiE4OBiVlZVITX0zKtDAwAD+/v4oKytDVtabEUempqbw9vZGYWFhw7+noq/JzMwMCQkJEIlEsBpvhZ5jekIgEoDBYkh8TckbkxF7IBaDLg+CoZ2hSq6JWcaU6JpU8T2ZmJjAx8dHZd+Tsq+Joihwi7gwFZqiuKRY4mvKvJ2JxLOJcJjhAD0TPbW6JkD2n73MzExUVlaivLxca66Jjp89j8EeEFuK8TLjpeKuyR9w7+MOlh4LKSkpb10Tq5KFF89egOPMaShdLM01CaoFSN1cX+I5bGdYw/bchNxWv6e0l2nI+icLhg6G8B3uqzHfk7w/e2KOGEZGRrReU8GrAhQmFiLpaRI8enk0eU1WQisw7ZnIznkzs6K1n72szCyUFJWgOqca8T/Fozy2HCuSVyCnJEfmayovL2+4Jrtpdug/sz8yyjKAMt3rI9Tpmpw6OaH9p+1RaVbZcA2tXVMmNxOmQ02RL85HUUKR2l2TNn5PqrgmTg8OhtwagkKqELx0nlzXlH87H9VZ1ejzUR+lXhOTyURFRYVOfU+quKbMs5l49vkz6Nvrw9rXWinX1GZBG7h6ukp8TdbdrdHdqTuyM7NRLX4zQ1mXvid7P3vMvDoTZYwyxMXFQWAigH1Pe3Ds6xOhTV0Tv4yP4/OOI/ijYHDMOE1eU1Z8lvTXxJf+moZuG4rk5GQknUmCgZ0B9Nz0MPn2ZJjbmCMxMVFtvqf8/HyN6yM4bA4CggJQXl4Os+FmAOrffZF+j1zTv68pMzkTokLJrykmMgZV6VUw8zaDobWhWl6Tun9Pbm5uTT67a/I1aeP3RK6JXJOmXNPrBKUqMSgNWqmUz+cjMzMTXC4Xx48fx59//olbt24hOjoac+bMaTSjDgDCwsLQt29fbNiwAQsWLEBGRkaj9fpqampgbGyMCxcuYOjQoWjTpg3mzJnTaEbfhQsXMHz4cNTU1KCsrAzOzs6IjIxEt25vSm6tXbsWt27dwsOHD5uNvakZf66uriguLm5IWCoyi7w9YDsYTAYWxS1qtB3Q7My4vNn+vMd5KE0phc8wn0ZlIRV9TfwqPmpKamDqaAq2Ppv2EQwURWGbxzY4hjpi6pmpav89Sfuzx2QyG65TkmuqKayBnqleozWR1O2aZPnZEwgEyMnJgbOzc0Pcmn5N6v6zp6hrur7+Ou59fw+Lni+Cjb+N1NfEzeLi7Jyz8B7ojW5rujWa/draNQlqBdhgvgFtRrTBxBMTyfekwmuiKKph1FdT11SeUY5fvH9B56WdMXjrYImvSSQSYf+A/ShNLkWXFV3ATedi2C/DZJ7x97pfYbPZrV7Tf2PUhu+JXBO5Jl27pr9H/I306+n1s7OMOEqZ8ZeTkwM3NzcwGAzyPSn4mmpLa1EQWwCHdg4wtjGm5ZqKkoqQ9zQPHn08YGxnTL4nGa8pakcUzi8+j1G7RyFkZshb1yQSiJAdmQ09Iz24dnVt8ZpKU0tBiSlYelqCwWTIdE3R+6NxZtYZjDkwBm2ntFXI9yQWiXF45GHY+tti4OaBb8UuzYy/vLw8ODk5NRpRrwk/e+cXnQclpDDst2Fg6jHf2l/X+r2ytDJc//g62oxqg+CpwVpxTbJ+T0KeEI9/fowOczvAwKrxsjWtXVP03micnXMWk05NQpuRbdTmml7HqO7fE0VRyM3NhaOjY8P5Nf2atPF7ItdErkmTronL5cLS0hJcLhdmZmZQBY2a8cfhcODj4wMACA0NxePHj7F161ZMnjwZfD4f5eXljWb9FRQUwMHBAUB9JvbRo0eN2isoKGj47PX/vt72733MzMxgaGgIFosFFovV5D6v22iOvr4+9PXfXn+MyWSCxWK9ta0p/92vpe0DNgwAW5/d5GfStMNgMJrc3lyM0m6XJhYAqCmswcEhB9Hnyz7wH+0vdTsuXV3g0tVFqhhluabM25k4POowRu8ejfbvtJcqRlm2S/I9OYc5w9rPusXYFfU9qeqa/rt/U5ra38TBpMl9pY1R2u3K/n1iMpkoLy+vn0H2r/No8jWp+mePV86DgYWBwq+prrIOe/vuhXtvdwzeNPitfX2H+oKtz4a1t/Vb55YkdisPK7xz450m9wNa/p70jfUx+/psWLd5c251/54A7fvZA96+Jj19PXRf2x1eA7ykulYWi4W2U9qCm8FF58WdwTHmyBXj637ldT/LZDKRdCYJTD0m2gxvI9U1tRa7Jn5PrcWojtd0YdkFpN9Mx5LnSwBoxzX9l65d0/PDz2HiYAKPPh6NtjeltWsa8P0A6BnqQc9QT2Gx/3f7636lqWchWWLXlO+pKYq+JhNbE5j0N5F4f0m2S3tNaRFpuPjeRcy+ORtmjmat7q/t3xNFUShLLYOll2WT5xSLxE1u77SoExzaOzT5/MpisSCoEmD/gP0InBAI16OuLV7TvW/uIXpPND4s+xAGFgYyXZNXPy+EvRcGu0A7hT1rsFgslKeVw9jWWO7vqbS0FM7OzlLfM6l6+7+vqa6yDmUpZWBxWGCyFNMf0n1NksTY0naOEQcJRxNg6WXZsJ+mX5OsscSfjse1ddfA4rDQbWXT6303145rV1cM2TYEDiEODfuowzUperuyrkkkErXYp2jiNSkyRmm3k2si16SoGKXdrk7XRMfcO41K/P2XWCxGXV0dQkNDoaenh2vXrmH8+PEAgKSkJGRmZjbMzOvWrRu++eYbFBYWws7ODgAQEREBMzMzBAYGNuxz4cKFRueIiIhoaIPD4SA0NBTXrl3DmDFjGmK4du0ali1bpopLlti/k2LapPB5IbhZ3IY65bIS8oSNZntJojKvEiYOJs0mmP7N3M0cXT/oCof2LSeEVWnyycl0h6Bw1z+5DpsAG4RMD5H62LxneShLKUPghEAlREZoosLnhdjRcQeG/DQEnZd0VmjbHBMOBDWCZj93D3eHe7i7Qs8pDTrPTYfixGLEH4tH4IRA2AbY0hpLypUUVOVXod2sdm99ZuZshoEbBjZxVOs6Lewkb2jNYjAY+GfBP7DysWqU+CNU78S0EyhOLMbCpwulOo7JZoJjzJHpfohQT+cWnoNTJ6dGiT9ZObRTn/tXQnpVBVUwtjVuNPtf0QriCnBk7BF0fb8rwpaFvfW5z1AfTDw2EXZB9c/dgloBjk8+DsdQR/T5vI/S4lJXFdkV+NnnZ3RZ0QVDfhrSsJ2iKPw94m+IRWLMuDSjYTu/mg89Iz0wGIxmB60CgIG5AUb+MVKiZ06/0X4wcTRpSPrJwszZDEO3DZX5+Oa89/I9hbepKfRN6wfgEW+YOJhgXdW6hoFruixoUhBYeix4D/KW+lgbf5uGSjIEQRCEbtKYJ/1169Zh6NChcHNzQ2VlJQ4dOoSbN2/i8uXLMDc3x9y5c7Fy5UpYWVnBzMwM7733Hrp164auXbsCAAYNGoTAwEDMnDkTGzduRH5+Pj755BMsXbq0YSbeokWL8Msvv2Dt2rV49913cf36dRw9ehTnz59viGPlypWYPXs2OnXqhLCwMPz000+orq7GnDlzaPl3ac2/y4hpA++B3liVuwpMdtOZdkkcn3IcadfTsLpgdYv/NoJaAUpelsChnQPu/XAPV9dexcLohRK9DHFo5wCHzeSliTLxq/m4t+Ee2oxoI1Pi7+ZnN5F8ORn+Y/zl+nkitIeFpwVsA21hF2yn8LYZDAaWJixVeLuKQlEUKnMrwTZgw8jaiO5wlC77YTZufnYT9iH2tCf+7nx7BwUxBW8l/l6PBlPXv+Fj9oyBsb0x3WHoPH0zfRhaGkp93L9fPBOaj6IoTDw+EWx9xT3a8av4KE0uVatBbETrKIrCL36/wLmzM2ZGzFTaeQwtDaFnpNewLvF/WXlbwcrbquG/2QZsZEVmwcBc9qSTpgtbHgavgV6NtjEYDHBMOBCLxKDEVEOy9vzi8yhOLMasq7MaLU/RlI7zOkp0fv8x/vAfo52DgwntwmAwSNLvf1h6LARNCqI7DIIgCEJDaUzir7CwELNmzUJeXh7Mzc0REhKCy5cvY+DA+pHwW7ZsAZPJxPjx41FXV4fBgwfj119/bTiexWLh3LlzWLx4Mbp16wZjY2PMnj0bX375ZcM+np6eOH/+PD744ANs3boVLi4u+PPPPzF48JuybJMnT0ZRURE+++wz5Ofno3379rh06RLs7e1lui5lvdBLOJ6Af+b/g3GHxsF3qK9SzkEXeUem2wXbgcFgQFAjaPGG8sZnN5B0JglLni+BS1cXBE8PBpOluQmixDOJSLue1lC+SdNxjDlYlbcKPC6v9Z2b0HVlV3Sc31HmqdZioRi8ch7Yhmy1eTBhMBhwcHBQ20SBOqIoClmRWXDr4QaOMQczr8yEka1qE1/cLC7+6vYXwj8NV+osrZZk3MrA3r57MXjLYHR9vystMahSwLgA2IfYw9LTku5Q0O+bfgD19kCdkpcl2D9wP/p/2x8hM6Qf3KAozfUrvsO0695CU434fQTdIRBqgMFgwHug9LMBWnJw6EEUxBXgw9IPFT5zjNyvKI9YIEa7We1g5WPV+s5yMHMxw+LYxRLvz2AwsKZwjVJnIaozc1dzDN3a9Ey58YfHN/pdoCgKhtaGMLE3aTXp95pYJAa/iq+xidWiF0XIuJ2BwPGBMLKR7T5cE/sVsUiM07NOI2B8AALGBdAdjlqpKqhCyuUU+AzxgbGdbg40iz0QC7debrBwt5Dp+JSIFFz+4DIG/TgIPkN8FBucDtDEPoUgCPVGR3+iMYm/v/76q8XPDQwMsH37dmzfvr3Zfdzd3d8q5flfffr0wbNnz1rcZ9myZQor7dlcjVh5Gdsbw6G9g1aVb4o/Go+C2AJ0W9VNptHtr4WvD5doPytvK1h4WIDBYsC9lzvce0leCu/qR1dRmlyKiccmqs2NQsatDDza9gjdPugGCw8LusORWt7TPFj5WkHf9M0DsJGNkcwPh559PeWKJ+dxDnZ134VBmwY1W29f1ZhMZqvrjRKNJZ5KxLGJxzDn7hy4dnNV6oNlycsSxB+tLy/577Ir1YXVMLQypDUhbx9ij9BFobBvJ9sgFk2jb6oPxw6OdIcBAHDr4dbk9urCanCMOeCY0DuwoKV+RSQQgRJRWnWvoSt4XB6e7XoG20Bb+AwmL4M0nVgoBoPFUOg9Z8cFHVFdWA0RX6Tw33Fyv6I8LA5LKaUYpXV+yXkkHEvA8tTlDffuupr0a83r39us+1l4sPkBwj8Lx5AtQyQenFhTUoNt3tsQNCkII/8Y2eQ+KVdScGXVFQzeMhheA7ya3IdOqRGpuLTiEmz8bGQuV6yJ/UpJUgmeH34Ocw9zkvj7j5QrKTg9+zTGHhgrU3UfTVeRU4HT75yG1wCvRmWApULV3x/Iu0yOrtLEPoUgCPWmrBxQi+dU+RmJRoRCoVLade/ljtk3Zsud3FAnCccScPf7u82WlFG0Tos6YeaVmTLN8iuKL0LOoxy1SfoBQM+PeuKD7A9g5mpGdyhSo8QUDo85jD/D/oSQJ0R5ejmS/klCZV6l3G2LBCKpjxELxbj52U1YelkqpSykrEQiEVJSUiASSX9NusrYzhh+o/1g5qz834uSlyW48ekNZNzJaLTdKdQJi2MXN7nGm6oYWhlixG8jtOpvRksqsivAr+bTHUYDsVAMQW3jNSDde7lj6Yul8BvtR1NU9ZrrV9JvpuNbo28Rsy+GpsiIuso63PryFjJuZ7S+839QYgpXVl5B/JF4JURGqNqz3c/wrfG3SL+ZrrA2281sh+6ruislsU/uV1qWciUF0Xuj6Q6jVUn/JOHa+mtNJqjM3c1hG2TbaFBTWWoZovdGo7qwWpVh0k4sEuOPTn/g3sZ7Le6XFZmFhOMJDS/pJX2ONLI2gld/L9gGNV+6nMfloa6yjvbBRM3xG+WH6ZemyzUATRP7FdtAW6wtWasT1Tak5T3IG+MOjlP4bHZNYepoimnnpqHPF31kbsN7kDeWJS4jVTpkpIl9CkEQ6o2O/oQM0aZZFb8KVlBuKRZtMe7QOJQmlyqkrOLN/7sJfjUfg34Y9NZn/Go+KBH1VmmV3Ce5OL/kPHp82AOB4wNbbH/qP1NlLiGpLJpcIkMsFKPX+l4ABezsvBM1JTWoyqtC26ltMf7QeJnarKuswzbvbWgzsg1G/zVaqmPzY/KRejUV4Z+Fq93DSGWl/MlQXeLW0w1uPZuecaXwc/Vyw/zH82ETQBZZp9v+QftBiSgsS1LM7H155MfkY2ennejzRR/0+rjXW5+rwwCSpvoVSy9L+A73hamTKQ0REUB9Avvm5zfRa30vuIdLXpUAqF+fa86dObBuY62k6AhVMrE3gUdvD5g6K/73UcgTKiX5R+5Xmsav4uPEtBNg6bHQfnZ7AGi09ltronZGIe1aGoZsHQITexMlRgoknUnCs7+eofuq7jC0alyNpeeHPdHzw56NtqVeS8W5Becw7fw0nXoRXZVXhdrSWtSU1LS4X/dV3dF2SluZBqNNOjGpxc+DJgYhaKL6rhNm4WGhkIo0mtivGFhoZnlWZTOxN0HwtGC6w6ANg8kg5TnVgCb2KQRBEP9GEn80M9VX3guzuxvugmPMQdiyMKWdQ5VYeizYBjQ/klEaadfSUJ5ejoEbB771UvXRL49w7/t7mHVtFhw7vikFZ2RrhNJXpRKPUlWHl7X/JuKLUJFdAY4pB8a2mpUEZHFYDWuf1ZbVQlgrhIWHBSy9ZV+fS99UH06hTjKtgeIU6oSVOSvBYCnvO5bmJQ8hG7FQDCZbdRPfDcwN4NTJqdE2QY0Al1ddRtCkINpn28Ufi0fkxkiM2TdGYX2tumo7tS1YeqqZPd4aS09LOIc5wzH0zd+bxDOJSL6UjF4f94K5qzmN0TXP3M0cU05PoTsMnWbpZYmF0QtlLn+uqkEPhPL5jfKD3yjFzw5+/NtjRKyOwMLohbD2JUliVeCYcPB+xvsNszef/P4E9zbew6LoRRKt95YfnY/4o/EY9dcoJUcK9FrfCz0+7AF9c8nWofMe5I3JpybDqbNT6ztrETMXM6xIXQGxSNz6viqoQKGuKIqCsFYIPSPNX4teEoJaAeKPxMN7kDcZRNWC2tJasA3YOvNzAdS/ByhPL69fdkaO9wHcTC5enn8Jjz4eWv9sRxAEQTSNlPrUYlE7ovDsr5bXK9QUxUnFyHmUI1NZxqZMOjkJK9JXNJmcs/SyhGNHR9i1bVzC0cLdAmuK16Dz4s4tti2oEeD54ecoTixWSKyKkh+dj23e2xC9O5ruUCRGURQEtQLUltU2bOu1rhf6ftkXHd7tAI/eHnK1P/3idPRa9/YMG0noGenh0opLiNoZJVcMTRGLxDg49CDubbyndjNHtcnRCUext+9eiIWtv4hRFGGdEEUvihr+O+9ZHqJ+j0LOwxyVxdAcIU+IytxKnSjB1fvT3uj5Uc/Wd1QBfTN9vHvv3UbrrKVGpCJqRxTY+mR8FtE8tj4bDu0cYO4mW3KYoiiUJpeqVdlbQr1YelnCqZMT6rh1dIeiViiKanRvqmgcYw7aDG8DoH7wG4PBQGlKqUTHDt8+HB9Xf6yQCimtsfS0hLWvdZPLIjz46QEebnvYaJuFuwX8x/hr3ABERZFl+QhJifginJ59GhdXXHzrs9rSWlxbfw05j+m/12zJJsdNODzmMN1hqEzWvSycmXMGMftJyfTmJJxIwEabjXh57iXdoahUeUY5tnlvw5U1V+RqpzC+EBeWXED2/WwFRUYQBEFoGpL4o9mr0lf45dEvSml75pWZmHFFxoWA1czjXx/jzy5/gpvBVUh7xrbGzT58BU0Mwqxrs5pcS1CSBzZuJhcnpp5Qu5t4Cw8LdF/THc5hznSHIpHb39zG+SXnEX8kHpscNiH5cjLdIQEAuFlcxB+NB6+ch/gj8ciLylP4OaoLq8HN5KI8vVyqmaMMBgOurq5qN9tUHVEUBX1Tfeib6at01t+ZOWfwa+CvDS/aXbu7YkX6CrSf015lMTQnZEYIVuaslDuhTsju9c/FsF+GYUXqCrUo0dxSvxL3dxyOjD0CQY2giSMJZeOV88Dj8mQeIBL1RxR+9v0ZGbekXyOQUC9X1lx5K8miCD6DffDOrXfemq0uL02/X7n43kX81e0v1JbWQlAjQEVOhULazXuWh7vf3200ACdkZgjee/UeHDs4tnBkY/9eV0+ZKIpCdVE1qgqq3vrsyW9P8PTPp00eJ6xTzhr36irtRhpiD8RCyFPedbM4LBTGF6I4ofitvwm5T3Jx99u7yH6g3i//24xsA5duLjIfr2n9imNHR4w7OA6BE1peQkSXOXZ0RNCkIJg4KLdssbphspjosqILvAZ4ydWOc5gzZl2bRUqGykjT+hSCINQfHf0JgyLTSWhRUVEBc3NzMNcxIdYXI2FJAgJsA+gOS21lP8hG2vU09FzXUyG/KJSYQlZkFigx1bAujlgkBq+MByMbo2aPq6usw8NtD2HhYYGQ6SFN7sMr5yElIgXWbazh0M5B7lh1Td6zPIACItZGoKa4BuGfhiNqRxQmHJkgczmz5uRG5SLqjyh0XtJZ4u8qclMkIlZHYOq5qfAZ4qPw0bsvTr2AoEaANiPagMVhqezlja6iKEqlf3wf//oYvHIeuq3qRmZz0SQrMgt3vrmDHh/2kHpdNGW6/fVtRP4QieUpy1v8O6RObnx+A3e+uYPFsYthG0hKCKna1XVXce/7e1iauBQ2ftKvHVqUUIQnO56gw7sdyP2KBqMoCt+ZfAe3nm6YcVk7Bvypu4fbHuLlPy8xdv9Y7O61GyaOJnjn5juNSrKJRWKp7xEvvX8JD7c+xIKnC6RK9L3Gr+Lj1YVXcA5zVsh6aa2pyK7AFtct6LysM4b9PKzRZ9wsLkR8Eay8G5fU/6v7X+CV87A0YanS41MXJ6adwPO/n+Pjmo+Vel/Pr+Y3OdNTJBCh+EUxjO2MdS6BQhAEQRAEoS5e54K4XC7MzFRT3p3M+KPZEJ8hAIB119YpvKQfv5qP4qRirRhV6dLVBb0+7qXQF/SHhh/CnW/uNPx30tkkbHHdgsQzic0ew9Zn497391osoWpgYYCgiUHkJZqMbn5+EzvDdmLs/rGYfX02AscHYuaVmQpP+gFAVX4Vnv7xVKpZe8FTgzH8t+Hw7OeplJI9kT9E4uKyi9A31YeeoR5KU0rx4tQLiY4ViURITEyESKSYkri6QNUjbjov6YxeH/cCW58NiqKQfClZqeXCpJV8KbnZEfraoiKnAqnXUsHj8ugOpRErHys4hjri3g/3kHo1le5wGrTUr/RY0wPra9aTpB9NnDo5of2c9jKvDWQbaIuhW4eS+xUNx2AwsLpgNcbsHaOU9sszynFk7BE826W45QM0/X6ly/IumH5pOkwcTOA/zh8B4wLeeo67+N5FnJ13VqrnsIEbB2LG5RlwaN/4d5KiKFxccRGnZp5q8fiihCIcn3wccYfiJL8YOZg4miB0YWiTg2jMXc3fSvoBgGsPV51bXzT803BMOTNF6YP5mivvytJjwT7EXuuTfprUr1AURaolEISa06Q+hSAIzUBHf0ISfzRb13Md2Ew2ziSdwZH4Iwpt+/7m+9juv13t1pqTlkggUnhSlMFkoPOyzhj558iGbQbmBnAMdYRLl+ZLjLA4LMx7NA/TL0xXaDyqcnbeWZya1fJLA7p1/aArBm8eDFNHUxhaKT7Z92+efT2xumC1VGUWTZ1M0WlRJ+gZ6iHzXibyo/MVGtP4Q+Mx6cSkhlHjl1ZcwvFJx8GvkmwNJh5PvZIZ6qiusg67w3cj6WwSbTHwq/l4/OtjHBx6ELe+uEVbHP8V+WMkLn9wWavXlgyaGIT1tesb1k9SF0GTgzD94nQ8+e0Jrn9yne5wGmmuX+GYcJosi02oRuD4QIzeNRr6pvp0h0LQjGPCUdpLfUNLQyRfTkZRQlHrO0tB0+5XKIpC5I+RqMqvL2v5evDXwA0D0fX9rmCymKgqqAIlpiDii1CWWobytHLUltYiYm2ERGUeWRwWvAd5vzUoicFgoDy1HIXxhS0mEi29LDHu0Dj4jfaT40olx2QxMeL3EQiaGNRoO0VRKM8oR13F22tDDvphEEb+MfKt7drMNsAWfqNU851kP8jGyRknGw0qy3uaB0Gt+ieZ0m+l4/Ts03K9u9CUfoVXzsO3xt/i4vK312QkGst5nIPd4bt1ap2/v0f+jYgPI+RuJ/1WOr5gfoFHvzxSQFS6SVP6FIIgiOaQxB/NgmyDsL7XegDAsgvLUFhdqLC2PXp7oOe6njCwMFBYm3R49PMjbHLchPwYxSZY+n/TH+au5gDq15rw7OeJd+++2+qLE9sAW7ANmi/Rd/Wjq/jR4UdwsxSzHqEilSSVoCSphO4wWuTZ1xNdlndRybn0jPRgbGcs8ayvnEc5jV5iHBx6EFc/vKrQmCw8LODZz7Phv7ut7IYJRyaodB06bcfN5CLnYY7C+xRpHJ98HBeXXUTXlV0RNDmo9QNUpN83/TDr2ixAe/N+AOpfov67JJs6YDAYYOuzMe/hPAzeMpjucCSW8ygH6bfS6Q6DkNHzI8+xPXA7il4oNqlDqE5VQRXyo/OVNntE30wfqwtWY9CPg5TSvqZIu56GiDURuPv93SY/p8QUDg07hN29doPBYmDa+WmYfGoynh9+jsgfIvH8yPMW24/ZF4Oy1LJmPx93cBwWRC1osUy4kY0RgqcGwy7ITrKLUhJhrRBbPbbiwrILtMahDiiKUmn1neLEYsQdjEP6zXQAQG1ZLf4I/QPnFpxTWQyyqsiqQMy+GI0ftCwJUZ0Ibae2hWNH6Uv66hp9U33kP8tHRbZi1lJVdyKBCEUvisBNl/9dkrGtMQLGBsDSy1IBkREEQRCaiCwwpAY+7vUxTiWeQmxBLJZdWIajE48qpF33cHe1Wr9IVgYWBjBzMYOlp3JuWDJuZ+DvUX9jwZMFsPJ5uyTNf4lFYhTGFYKpx2zywdrEwQQWHhZqWU5lzp05dIfQotqyWhhYGKi0/CI3i4vSV6WNkm1NEQlEODzmMAytDLE4bjEYDAYGbRqk0O+5LK0MhpaGjZL1rcVFSM8uyA4f13wMsUBMWww9PuwBnyE+CF0YCpae+syYamnGs7bIiswCv4oPr4FearlYu22AZpXNPDrhKIysjbDw2UK6Q9E5J2echI2/DcI/CZe5DQaTAWGtENUF1Rr3s0fUe3HiBS4svYAZV2bAe6C3Us5BZpXW349NPj0ZTp2cmvxcWCeEex93sPXZDbMB9c30EbYsDFY+VmgzovlZ5hU5FTgz5wzajGiDKWemNLmPvlnr3wElplQ+qOXZrmeI3hONyacmw8j6zfq03VZ1azKpUfSiCA+3PkTwtGCteE5tTU1RDX60/7GhoomyBU4IhHtv94bnZkpMoe9Xfd8qH6uOAsYFwG+Un0Q/65rOxMEE4w+NpzsMjWDtZ40Pyz7UmUGwLD0Wlicvh1gk/3OqbaAtJp2YpICoCIIgCE3FoLS5npcae72gY3l5OczNzfE07ynCdoZBRIlwecZlDPLW7VG1qpQVmYV9A/Zh4MaBCFsW1ur+NcU1+MH2B4TMDMHYfWNVEKHu+L3d72Ab1M94UZVjE48h4UQC1tesb3Emp4gvQsz+GLD0WGg3q51SYjk04hBSr6biI+5Hb43ori6shrGdcYvHUxSFyspKmJqaqmVCg9AMwjohhLVCjZ8t3pyDQw8i43YGPq7+mO5QNEJr/crzw8+hZ6wHv5GqKWNG1KMoChssNsCjj0eziQJJ2yF/LzRbblQuXp1/hY7zOsq83mNrKrIrkHw5Ge7h7rD2tZa7PV29X6EoCpSIeusFtlgkRurVVBhaGsI5zLnZ4zPvZSL2QCwGbx7c5Hpxu3rugqhOhPmP5ys89ubc/uY2Hmx5gHduvSPRTMOsyCzs6rELA38ciO6ruqsgQnpV5lbiyqor8Bnqo7Tnh+Y82PoAFh4W8B/tr9Lz0kVX+xWCIJSD9CkEQSgal8uFhYUFuFwuzMzMVHJOkvijyevE37+/7E2Rm8BmsjE9ZDpsjGzkPkdtWS1OTD0Br4FeGvtgJRaKVTK6q7qoGvpm+i2Wz/m3exvvwamzEzz7atZsrNKUUmTfz4bPEB8Y2Ri1foAKiUViXFpxCRwTDgZ8P0Bl53118RW4mVy0m9kOekZvv0RRpWe7nqE0pRT9v+nfaPuVNVdw/8f7WJW3Si1nkmqarMgsiAQiuPdyV7tyj3Sryq/CZpfNCF0YiuHbh9MdjlKk30xHRU4FQqaH0B0KQciFourXEpP03oUgZJV8KRkHhx7E8N+Ho9PCTnSHo3J3v7+LoElBcpVL41fxcXLGSVh4WmDIliEytRH5YyQi1kRg1rVZTVaE+GfBPxALxBi9e7TMcUpL2lmGwjoh6irqYGRtRO7BlEjIE2KD1QY4hznjnZvv0B2OxAriCiCqEzU7s1ZbRP0RhfyYfAzcMBAcEw7d4ai9mpIaPNv1DA7tHOA9SDkz29XFy3MvIagRIGBcgNzvwcozyhH5YyT8RvkprSIAQRAEIbmmckHKphvz5dWYSCRq+P+ruq/Ciq4rFJL0A+rXL8u8kwluhvqtNScJsUiMnWE7cXWdYtdQa4qxrbFUL856rO3RZNKvIK4A55ecR97TPEWGpzDJF5NxauYpFMQW0B3KW5gsJob9MkylST8A8B3qi04LO7WY9KvMq0R1UfVb249OOIoDQw4oLJYO73Z4K+kHAK7dXdFudjsIaltev0ckEiEuLq5Rv0K87ebnN3Fo+CGAvG96i7G9MQInBGr1miMefTxI0k8KpF9RX6/XhZRX9oNsRHwYAX4VXwFREdrIOcwZU/+ZCr9RipnZq0n9SlFCEa6tu4bbX92Wqx2WPgu1JbWoKappVMKtLLUMRQmSrbHZblY7vPfqvWbLwI/8Y6RKk34AmkzeFT4vxKERh/Dy/Mu3PmPrs2Fsa0ySfkrGNmBj6YulGPKTbElmuhwafgjnFsm2HqEm9Sspl1Pw9I+nLVabId4QC8W4uvYqYvbG0B2K0kX+GImz886CwZK/j6wpqsHjXx6r7bspdadJfQpBEJqBjv6E3GloMbY+G+uq1mnstPTqwmqw9Fhq/WD43xmJOQ9z8OS3J/Ad7gtHqN+Lc5+hPph0YhLsglsvxaNrxCJxw5os/3X769t49uczLIlf0mgdSAaDoZLfr4CxAQgYG9BoG0VR4GZywa/kw65t/fcpqBUg/34+AgMDAdSvGcM2YDdac4UAeq7riYrsCo3tG5WJwWBgwuEJdIdBqJmWblBfnHqBC0suYPSe0fAZ7KPCqHRbZV4lCmIK4BjqCGPblstAtybjdgYiN0bCd5gvPHp7KCZAQmV2dt4J156uMs8ik4ShlWGLa9TJQlNepNkE2GDOnTlyV8pg6bEw/eJ06BnrNbr/uLfxHqJ2RGHpi6Ww8W958KexnXGrZd9VTSwU49WFV+CYchoGRVYXViP1air8RjedKC5NLkVdZR0cO6jfs5KixR6MRVZkFvp91Q+GVoYqPbeFuwWgYcso9v68t1wz4DSlX5l4bCJqimt0Zt06eZnYm2DO3TlaPTDxteG/Dgc3k6uQ51T7EHusylsFjimZVSorTelTCIIgmkMSf2qmml+Nw88P42neU2wfvl3u9jT5xbapoynmPpgLsUD+hY0VrSy1DHt670H7d9uj7xd9G7Z3mNsBPkN9YGip2gc7SVl5W8HK26r1HWlwful5AFB5eUGKorCv/z4wWUzMjJjZ5D6e/TxBiShYejcu8TTx2ESFxXF+yXkUvyjG9EvTm53B8e9EHiWm8GvQr3Dp4oJZ12YBAO59dw/3N95Hp8GdYO5ijhNTTqA8vRyLny+Ggbl2rtcmi+ZGyhPaj1fOwzbvbQhdFNrk7FpCeoZWhrDwsFDrQTraKP1GOk5OP4mJxycicHygXG2FzAiB7zBf2AQopuIEoTpCnhAigQhCnlAl56urrIO+qb5KzqUuGAwG3Hq6KaStfyc0ItZGwNTZFCEzQ2BkY9Rq0u81Qa0A6TfS4dDeodGajg9/fghuBhd9vugDjrHqXvIymAwcm3QMXv29GhJ/nv088QnvEzS3osjhMYch4ovw3sv3VBYnXVIjUhGzN0blFU00Vce5HekOQSUYTIbaJfHVnVsPxfTD6s420Ba2gbYKaYvFYZFlQgiCIHQcGWKkZqr4VVh4biF+ffIrkoqT5G4v53EOYg/EKiAy5ch5lINNTpuQdiOt0XZeOQ9A/cM2i8OiI7QWmbmYwdje+K2ZVAwGA2bOZrSvFdcadVzaMzsyGzkPclR+XgaDARN7ExjZGjX77xI4PhAjfh+h1ER6XUUdBLWCZpN+SWeTsMVlCx5vfwygvjRqny/6oOP8Nw/IJg4mCPogCAaWBmAwGegwtwNCZoaQpN+/qOPPvrpJvZaKv0f9rZYlgeUl5AlhH2Iv9wwp4g2P3h6Ye38uWTtExZy7OGPEHyPg3NlZ7rZMnUxh19au2VnvhPJQYgr50fkyJ+7YBmwsil6EEb+NUHBkbzs6/ig2OWzSqb+jGbczkPskV+Ht8qv4SDqThBcnXsCliwv6fd1P4mNTI1JxaPghJBxPaLQ96UwSYvbGQM9Qtc8gDCYD4w6MQ+/Pe7/9WTP3zd1WdkP3NZq5/ry0Ru8ajVV5q3QuYU40T1ArQNr1NFTlV9EdikahKArFScXIfpBNdyhKw+PywK9WXNl1YZ0QxYnFTS5ZQhAEQegGBqVLT29q5PWCjuXl5TA3N2/02YhDI3D+1Xl83PNjfNP/G7nOc3L6ScQdisP62vVqWUM+9mAsTs04hXduvwP3XvW1SPKe5mF3r93o8n4XjZqRQVEUUq+mwj7EHib2so+sEolFYDGVk+wUCUTY7LwZ3gO9Me7gOKWcQx4tldukC6+cBwOLphNnyZeSUfKqBF3e66L0OMpSy/Bg6wP4j/Fvcn1JoP5nkMfjwcDAQKNn+ypT6tVUHJ98HMN+HYa2k9vSHY5aSjiRgBNTTmDC0QlvlZgldA/pV3QDv5qP4hfFcOrkRHcoOuXhzw9xafklrEhfUV+WT409+OkBCuIKMOznYXIPcNOUfuWPTn+gJKmkvlSaHOUHm/J6wJe0zwz8Kj6e7HiCgLEBsPR6U4mCElOoyKmAuat5C0erRmlKKUqTS+HSxaXZe2iCaErSP0m4svIKRuwYIXWFDk3pV/Jj8rGj/Q70/r/e6PN5H7rD0RhioRg/2P4Aaz9rzHswj+5wlOLuhru4tu4a5j+ar5D7saKEIvwa9Ct6f94bff6vj/wB6hhN6VMIgtAcXC4XFhYW4HK5MDMzU8k51esNOwEAmN1uNgBgf+x+iCn5ylyGvReGKWenqG0JrpDpIfic+rwh6QcAxvbGcA5zhv9ofxojk15VfhUODDqAG5/dkPrY2IJYZFdkY8n5JQj6NQgCkUAJEdavL+Lc2RnWftZKaV9e6pb0q8ytxI/2P+LWV7ea/Pzpn09xafkllYx+t/SyxNCtQ5tN+r3G4bz9YqoytxLHJx/Hy3MvlRWexmCymbANspUrOa/t/Eb64eOaj0nSj2jQVL/yb49/e4wHWx+oKBpCGS4suYCdnXeitrSW7lB0ikM7B9gG2cLY1hgVORW4t/EeCp8XSnz8q4uv8Pi3xwqdIdCcru93xei/RiusqkVr/Yo6GPnHSAz/fbjCk34AoG+mL9O9CMeEg+6rujdK+gH1M+/oTPpRFAVKXH8/nHA8AQeHHETJqxLa4lEHFEUh/VY6KnIq6A5FY+gZ6oGlz4KIL9u6WprQr5jYm2Doz0PhO8yX7lA0CpPNxMAfBiL803C6Q1EaGz8bBE4IhJWvYpZmMbI1Qvin4XAP17DFPtWIJvQpBEEQLSEz/mjyesZfaWkpLC0bP7jxhDw4bXJCGa8MZ6acwSi/UTRFqToURSH2QCz0jPQQOD4QFEWp/aia8oxy3Nt4D77DfNFmeBvUltXi+d/PYRtoC48+HhK3I6bE8N7mjbzKPNSJ6gAARyccxcQgxa0fp+7qKuqQdj0N9u3sYelp2foBSvDw54coflGM4b++WWOwOKkYEWsi0OHdDvAf83YiujC+ELwyHly7u8qVXOdmchF/NB6+w3zlqukvEokQFxeH4OBgsFhvZo1W5lbi5zY/o+O8jhjy0xCZ2ycITZdxJwNp19PQcW5HmLmoZoSVpmuuX/m3HR13gFfGw4q0FSqOTncdm3QMpcmlWPh0oULae3n+JQpiCtBpUScYWqnnOsXaLvNuJnb32i3VLJDjU44j/kg8Pq7+WO3LzP+bJP0KXRJPJyLtehqGbhtKdygtEtQKUPyiGI4dHfHq4ivoGenBvZc7LYM9H/78EBGrIzD3/lw4dnREQWwBch7lwH+s/1vLIgBA4plE3PzsJob/Nhyu3V1VHq+q8Mp52GC5Ae3fbY/Rf42mOxytp879CkEQmof0KQRBKFpZWRmsrKzIjD9dZ8A2wILQBQCAHyJ/UEib6pjfzbiTgfNLz6PkVQnquHU4v/g80m+mA2h+TQh1wmAy8OTXJ8h5VL8unaGlITov6SxV0g8ArqZeRXp5OgzYBljdbTUAYNujbYoOV60VJxXjyNgjiD8ST1sMGbcyEHsgFoKaN7MtbfxsMPXs1CaTfgBgF2QHt55ucr9kyY3KRcSaCKWsIwPUr9+0NGEpSfoREiuIK0Dc33F0h6Fw6TfScev/bpFZTQo2/tB4zLkzh+4wdIqBpQGMbN5+oS6rNsPboNfHvUjST4XK0soa/bdLVxdMvzQdPdb2kLiNgT8MxKxrs1SS9KstrcXZ+Wfx9K+nSj+XKlXmVuLluZcQCepnGCUcT0DUH1GoKamhObKW7Qnfg4NDD0LIE+Lq2qs4MfUEQNPjk4WHBbwHeYPJrn+1YB9ij47zOjaZ9APqn/P41XyVzFSlE4PFwJCtQ0hpeYJQMLI+IkEQBEFIhiT+1NTyLsuhx9TD3cy7eJAte/mszLuZ+N7ie0T9EaXA6BQj7Xoanvz6BMJaIQwsDDAzYiYMzA0aysSoO3NXcyyMXoi+X/SVq52dT3cCAGaEzMAH3T4Am8nG3cy7iM6PVkCUb0v6JwkXll1Qq4dtCw8LjN0/Fm1GtKEthmG/DMOaojUNL88k/TkUC8VyJ9Y9+3lizt058B7sLVc7LTF3o3/NF3Vw++vbePL7E7rDUHu3v7qNk9NPoq6yju5QFKrL8i5YFLMI1m3Us9yxprLxtyEzKFVs5I6RmHllJt1hEDLKeZyDbV7bcH/z/YZtTDYTPoN9oGcoeRLP3NVc6nWwZKVnpIdnfz1D5u1MlZxPVWL2xeDvkX8j+VIyAKDXx72wumB1s0krdRG2PAy9PukFABi5cySG/zqctoGTfiP9MPWfqbAPsZds/1F+WJ68HN4DlXffqw70TfXRZXkXeA/S7utUtMQzic0us6ANTr9zGjs771TLgdma4NTMU/jZ92cIapWzNApdytLKcGDwASSdTVJYm1X5Vdjbby959iUIgtBhJPGnppxMnTAzZCYGeg0EhyV7XWkjWyM4hTopdFS4ovT+rDfee/Ue7NraAQBcu7mi39f91HY9wqY4tHNo+P97+uzBoRGHpDq+sLoQZxLPAADmd5wPJ1MnTAicAADY8mCLUh4Isu5l4fH2x6jMrVR427IytjVGyIyQhp8FOpg4mICtzwYACOuEODD4APb22wteOa/ZY66suYKv9L4CN5Mr17kNzA3g1sNN6evOZT/MxuExh5Efna/U86iz+5vv4/nh53SHofa6rOiCKWemgKWnXWVNDCwMYB9iD7YBm+5QtIpYKEZ5ejnqKrQrUaxrzsw5g7399tIdhk7gGHMQOCEQnv0bJ+0oikJxUjGKk4pbbUPIE6Iqv0plL4/ZBmysLV6L0Xu0q2Rh26lt0f/7/vAZ7AMAsA20hYG5Ac1Rta7dzHbo8l4XsA3YcOnq0mx1CjqcW3wO23y2QSySb616Qjc9//s5bn52U60GqSoSi8MCS5+lERWO1JF7b3e0ndoW/Ert+vkoTytHxu0Mhc42FwlEKIgpQGWe+rz3IQiCIFSLrPFHk9dr/JWXl8PcvOmZOAKRAHoszVmvQ1eVZ5Tj0vJLeHn+JdpOaYtxB8ZJfOyPkT9iTcQadHbqjEfzHwEA7mfdR/dd3QEAXV264tqsazDSU1zitrqoGmKhGCb2JhqVZFWFohdFyL6fjfZz2uPcwnOgKAoj/xjZ7INZ7MFYvPznJQZvGQxTR1OZz1tTXAMDC4OGEkmyoigKYrEYTCazyZjTb6Zj34B9GPnHSHR4t4Nc59JU1UXV4FfxaVtLkqBXRXYFOKYcjXipqy5a61cAIP5oPI5PPo7xf49H2ymkpJmy8bg8PNjyAJ79POEe7q6wdk9OP4nyjHLMuT2H3B/QpDK3EpudN6PdrHYYs3dMi/um3UjDvn77MGTbEHR5r4tqAlQQSfoVQjKF8YVgspiw8behLQaRQITrn1yHta81Os7riIgPI5B+Ix3zH81vcn+KohCzNwYGFgZqlbBUtMhNkYg7GIfJpybDwt2C7nA0RsnLEjCYDFh6WUr1t4j0K4SmE4vEEAvFDYORCXqRPoUgCEXjcrmwsLBQ6Rp/5C+KGtPmpF9xUjHK08vh3stdJeuSKBOLw0LqtVR0X90dA74fINWxdzPvAgCmtp3asK2rS1d82edLfHv3W1gZWik06QfUz65TN9c/uY7Y/bGYe38uTJ1kT6DJ69739xCzLwbeg70x4vcRAFpebzJkeghCpofIfd79g/aDV87DitQVcrfF5/NhYNB0UsOtpxvWFq+FgYXuJj2MbY3V8ndAXdVV1kHfVJ/uMBRm/8D9AANYmrCU7lA0Skv9CgDYt7NH1w+6wsrHSoVR6a6KrArc+uIWKIpSaOJv3EHJBy4RssuPzodDe4cmPzN1MkWfL/rAtYdrq+0YWRshdGEonMOcFRxh8yrzKpH/LB/u4e7gmMhekQRovV9RhcL4Qlh5W2nsLHBBjQC/tf0NAPA59TltcTDZTDz6+RHcw93RcV5HDNwwsMX9GQwGrqy6ArtgO61O/IkFYvDKeFp1H6UK8pRjV4d+hVAdSkxp1UAlJosJJosUZVMnpE8hCELTkRl/NHk946+0tBSWls3PPBFTYsTkx+Bp3lO80/4dsJjSl127v+U+RHwRen7YU56QFerququ49/09LHi6AI4dHOkOR25V+VUwcZC+TGOHHR0QnR+Nf6b+gxFtRjT6LL8qH9X8anhbKXZdiNcl2diGbJg5q8eaTJGbIhF/OB6zb84Gx1i+F0nyyH2Si5riGnj08VDpS6Abn90AJabQ7+t+crUjEokQFxeH4OBgsFjaVaJREaoKqlBXUQcLDwutK2GpDH91/wu8cp5WJcluf3MbLD0WeqztQXcoGoP0K+pHWCdEycsSGFoakrUVNUxWZBZ29diFPl/0Qe/PetMdjtTubriLax9dw9z7c+HS1UXmdtShX6HEFH7y+AmGVoZYFL2IlhgU4cXJF9A314dXfy9a4yhPL4eRrZHE9/FpN9JgbGtMa5l/Qn3VltairrJOqpmS6tCvtIabxUXMvhi0GdGm0ZIhhHRyHufgzDtnEP5ZONpO1o5KE88PP4eltyWcOytuMI+QJ8TL8y9h6WWpFe/cVE0T+hSCIDRLWVkZrKysyIw/4g2KotBzd0/UCGrQ3bU7AmwDpG4j7kAcaktr1Srx125WO5g4mEi8CLy6kyXpBwA7R+5EcmkyOjt1fuszB5M3DwO1glrEFsSii4v8pZzKM8rxs+/P6L6mOwZubHlErqp0X9Ud3Vd1pzsMOHVykmr/jDsZiD0Qi24fdJOrxFLfL/vKfKy06irqkHA8AdZ+1nDr4aay86qD6D3R9S8sH8yFSxfZX1jqCvfe7uCV8UBRlNaUNwlfH053CAQhN7Y+G/bBir9/EglEiNoRBUMrQwRPC1Z4+wRg7m6OdrPbIWhyUKv78qv5tA6GaorvMF8YWhnCwsOC7lDkJhKI0GV5F+gZa3blkYBx0j8bKsO/fyYe//oY+mb6CJnRfFUMz76ezX5G6DaxUIwfHX6EzxAfTD07tfUDNEh+dD5ufHIDpk6mJPEnBzMXM9SU1GjN2tIigQinZ5+G10AvTDs3TWHt1lXU4diEY+i0uBOG/zpcYe0SBEEQmoMk/tQci8lCB4cOuJd1D09yn8iU+Jt0cpLavTiwDbCFbYAt3WHQrpNTJ3Ry6tTiPq9KXqHbX90gFAuRvTIbJhzZkoyvmTqZovva7vDsRx645VWSVIKnfzxF4PhAWtdWkUZdRR3Ozj2LkJkhOpf4c+3miu5ru8PaV/YSQrpkwHfSlS4mdNuFZRdQW1qL8YfG0x2K1qstqwUAGFgYKDQpz2QzceOzG7ANsCWJPyUxczbDmD1jWtyHoijs7LwTekZ6mHN7TrP7HZ9yHJbeluj/TX8FR9k8+2B7pSSd6cDWZ6P7avoHndFJKBbiaPxRTG07Ve6+pLa0FmVpZbAPscedb+/A3NW8xcQfUF+qlKXP0trSdpGbImEfYg/vgYqt3qLtmGwmuq/urpXlwz37eWL+k/kwdzWnOxSNZupoilW5q7SqzOeUM1MUvgSOvrk+xv89Xit/lwiCIAjJaOddtpZ5nRh6kvtEpuMt3C1gZKPYdeLkQYkpkAqzkvO28oa1kTW4dVzsfrZb7vb0DPUwcMNA+Az2UUB0inHnuzuI+zuO7jCkFjwtGGuK18Cjr4fMbWTezcTR8UeR/TBbITG1VobCzMUMU85MwaBNgxRyPk3iHu6OgRsGwtDKkO5QCBqUvCzBwaEHkXg6ke5QNI4k5W2KE4tREFuggmiIu9/dxUarjShLLVNouwwGAzMuz8Ckk5MU2q62EfFFEPFFUh1DURSyH2RDyBO2ui+DwYBjqCPs2to1e79MiSlk3s1E0fOiVturrgZu3QKErZ9apUjZLHoV1xRjyIEhmH5yOn568JPc7d39/i52dtoJbiYX0y9Ox8idI1vc/+b/3cS3xt+iNLlU7nOrI341HxGrIxC7L5buUDRS/2/7o8O7HaQ+Tt37FY4xB06hTjC2I+uNy0ubkn4sPRZ8hvgodN1moH6AS9spbaWuakS8oe59CkEQRGtI4o9mkvwheZ34i8qLkukc/Co+il4UgV/Nl+l4RUu7kYZvDL9B9N5oukOhVWJxIn559AtuZ9xucT8mg4kVXVYAALY+3AqRWLqXTeqOoijc+r9biDugeYk/PSM9GFkbybVeXMmrEiSeTgS/Uv7fTxaLJVENer9RfjC2JQ+cRMvEIjHOLT6Hm1/cpDsUhajKr0L6rXRU5VfRHYpGkbRfmRkxE0ueL1FRVLrNOcwZHeZ2kLnMeIttd3aGib3i29UWPC4PP3n8hJ2dd0IsFEt8XFlKGf7q9hciPoyQaP+RO0Zi+K/Dm52FxWAysDJ7JSYem9hqW0eOAD/+CJw4IXG4LdrTZw/2D9ovVxuS9ivKUltWiw1WG3Dry1u0nJ9uNYIadPurG66lXYOxnjFczV3lbtNnqA/6fdMPHBMO7IPtW127z6G9A0JmhGjtmstsfTbmPZyHnuvUZ6kNbUd3vyKJqvwqCGoFdIehFSiKwvVPr+Pquqt0hyI3QY2ADIxXQ5rQpxAEoVno6E9I4o9mkvyBD3UMBQA8y38mU9Ln2a5n+DXwV+Q+yZX6WGXgmHDg1d9L50tc3Ey/ifcuvodN9ze1uu/sdrNhaWCJlLIUnH91Xu5zX1t/DXt675G7HUVZ/HwxBv80mO4wpMav4qMgtgA1xTUyt9FhTges562He2/5R/hRFIWKigqJ+pXqomoUPi+U+5yaQiwSY0fHHbj9dcuJduINJouJlMspSLuaRncoCuEe7o6Pqz9G6IJQukPRKJL2K9qyDqQmCJwQiFF/jlJKGXeKosDN4qIyt1LhbWsDA3MDuHR1QdjyMDDZkj9G6Rnpoe9XfRW+HhuL0/rDY2pq/f9GRirmnEY2RjB1NJWrDWnuV5SBV86DXVs7GFrrZgWAh9kPkVyaDCtDKzyY9wBhzmHYcn8LDsQekLlNz76e6PVxLxjbGoNXzms1Me4/xh9j94+FpZelzOdUZ0w2E85hzrANJEtbyCI3Khe7w3fjxckXEh9Dd78iicNjDmOb9za6w9AKDAYDaVfTkHAsQa2/c0mcnn0aP9r9KHU1gcTqc4wAAQAASURBVNbwq/n40f5HXFx+UaHt6gpN6FMIgtAsdPQnJPFHM7G49dHCbazbwIRjghpBDRKLpS9R5tLNBb3W94Kpk3wP6Yri0sUF085P0/k15jLKMwAA7uatJ3yMOcZYELoAALDlwRa5z12VV4XyjHKF31zKgsFgwNrXWiPXXcu6n4Xf2/0u1UNpU1h6LIWMeBaLxUhNTW21XxGLxNjmvQ0X39OdhwBeGQ913LqGtbEIySyIWoB3br9DdxgKw2AwtKo0kCpI2q9U5Vch7u84lKZoZ9k2XVEUX4Sf3H7Co+2P6A5FrcQejIWgpn6WxuSTk9FxbkcA9evmVhdWt3q8qZMpwj8Jh0dvD4nPeXfDXRwZd6TJz3Kf5CLxTKJE1Tzy8ur/NzUVKC6W+PTNmnR8EsbsHSNXG5L2K8pi6WmJObfnIGxpGC3np1taef2Ans5OndHWri0iUiKw8spKbHsof0KipqQGGyw36PyLZkGtQKLSvkTT2AZsFMYVStS/vkZ3vyIJ/7H+6DBX+hKmRNMmnZyEZUnLNH7wmV2IHTz7e0o0mEcaTBYT1n7WSqkQoQs0oU8hCEKz0NGfkMSfBmAxWejgUH+DKMs6f86dndHv634amVjRZhnc+sSfm7mbRPsv7bwULAYLN9NvIjo/Wq5zj941Gu+nv6/wm0tZCGoEqMyrlKpklrqw9rVGny/6wDHUUeY2Uq6kIDdKtbNxmSwmen3cC+3eaafS89LJyMYIy1OWY/AmzZtZSidDS0ONf5h+Le9pHpIvJUNYR17EKUNBXAFOTjuJ1IhUukPResenHFfa7GVLb0t0eb8L3HpKdm+iC9JupOHUjFNvlekUCUQ4MOQAzi9uvRKDLKM7S5JKkHUvCzwu763PHm57iCNjjrSaWBAKgaJ/LQP4+LHUYRBaKLWsvp/2svQCAAzzHQYAeJz7GPlV+TK1WVNSg93huxH5QyQ6zOsA1x4tlw+tq6jDPwv+wdO/nsp0PnX3dOdTfGP4DTLuZNAdikayDbTF2tK16LSoE92hKFTPD3ui31f96A5Da5g6moLJ0vxXmr0/7Y0JhycovF22ARtzbs9Br497KbxtgiAIQjNo/l9JHfFZ789wecZljPYfTXcocoveE42IDyN0/uXr68SfJDP+AMDV3BUTg+rXcrmedl1pcalaSkQKNjttRtzfmrfGn4WHBXp/1htOobIvmH169mlcXKb6UdE9P+qJ9rPbq/y8hGYR8oR4deEVch7l0B2K3B798ggHhx4kI/CVxLGDIyYcmQCfoT50h6LVKDGF5EvJyHuap5T29Qz1MGTLEPgO9VVK+5rIrYcbhmwdgu6ruzfaztJjwdLTEla+VqDEzSf2BLUCbHHZInWydsjWIViVtwoG5gZvfdbzo54Yu38sjKyNWmyjsBD4d87xkQImcuY8ysGNz2+gIqdC/sZo8vi3x7j3wz2dLd/138Sfo6ljw5ryF1/Jdk/KMeGg+EUxKIrCqJ2jEDI9pMX9WRwWnu58ivTr6TKdT91Z+VgheFowLNwt6A5FIzEYDK0ZeEYoV35MPu5+f1dn+3OCIAiCaAmb7gAIyQzwGiDzsYJaAQ6PPgy3nm7o/VlvBUYlm8TTiXh1/hUGfCf7NWmDTG4mAMDdQvK13b7s8yU+7/05/G385Tp3VX4VUiJS4BzmDBs/G7nakpe5qzk6L+sM+2B7WuOgy5BtQ8A2UFxXbGDw9gvCllAUpRMP1pn3MlGaXIqAcQHQN9WnOxyNwa/m49DwQwiZGYKx+8bSHY5cQheEwrW7K/TNyPcvLUn6FSMbIwRNClJBNLqNwWTgo/KPIBLQX6pbV7A4LHRZ3qXJz8YeGNvq39DqgmqYOpuCpS9dlYWW/lbZBtpKtHbY6zKfRkZATQ0QEwPweICUtwqN5DzOwe0vb8M93B1mzmYytyPt/YoiRe2IAq+chx5retAWA518rXzRxbkLAm0DG7aN8B2BJ7lPcO7VOczpMEfqNtn6bKwpWiP5/gZsrCleA0Mr7Vxn0XeYL3yHkQEU8sh7loe062kIWxom8bMSnf1Ka/Jj8nF9/XV0WdEF3gO96Q5Hazzb9QyPtj2C7zBf2Ido3vuE9FvpiNkTg26rusGurZ1C26YoCre/ug3bQFsETghs/QDiLercpxAEQUiCzPijGYul/FKLbAM28qLywM3iKv1ckhi7fyyWvVym0+ssCUQC5FbWl3eUdMYfAPha+8qd9AOA4sRinJ51GilXUuRuS16OHR0x7OdhcGjvQHcoUqvIqcDu8N14/KvstbOCJgbBb6SfQuJhsVjw9/eXqF8R8oTYGbYT5xadU8i51V3s/liceedMwxpNhGSMrI0wevfot2a6aCKXri7oOK+jTiS6FUmafgWAzs/mVxVFrAvbnHs/3MOfXf7UyBLciiSoEeDmFzfBK3+71OZrr/sTkUCEmuKaJvex8LDA/EfzZUoy5T3Nw62vbjWaUVhXUSfx71n+/6o2BgcDdnaAQFCf/JNH0MQgLI5bDNfuLZdybIm0/Yqizbo2C9MvTqfl3Orgi75f4MG8Bw0lPgFgRJsRAIArKVdQJ6yTue386HycnHFSohKXRtZG5G8y0azEU4mIWB2B4iTJFielu19pDTeTi9SIVNSWkvXGFanzks6Yc2cObINaHwyjjnKf5CJ6T7TSnlFvfn4T8UfjldK2tlP3PoUgCM1DR39CEn80k2Zhx3+S/sFHVz9CSql0yRoGg4G1JWsxaucoacNTCn1TfVh6WtIdBq2yK7IhpsTQZ+nDzli2kV1Z3CzwhM2/jGqJfYg9Jp+aDP/R8icRdRpVvwZPcy/7VE0sFqOkpESifoVtwAaDwQCTrRt/Brqv7o7JpybD2M6Y7lA0Tvt32mvkCFpCMaTpV05OP4mN1htbLHtIyKcipwKvLr5CdVG10s5RlVeFytxKnX85Gb0nGrf+7xZiD8a2uB+Py8Mvfr9ItNaftJ4ffo6bn91EfvSbddcebH2A782+R0FcQavHv57x5+gIdPnfpEV5y30a2xnDrq0d9Az1ZG5Dmn5FGYysjWAboJkviRVBLAa2bwcuX36zrYNjBziaOKKKX4VbGbdkajcrMgtXVl9B3ME4VGS1Xgq2PKMcadfTtLJE36X3L+HG5zfoDkOjtZvVDu/cegdWPlYS7U93v9Iav5F+WF+7HoHjycwrRbLxs4FbTzeNXeuv+6ruWFu6Fg4dFD8ImsFgYFHMIgz6cZDC29YF6t6nEASheejoTzTzr6MWkeZBZ9P9TdhwbwPuZN5RYkTKlx+Tr9QXVprA0dQR9969h6MTj8o00nX1ldXw3OqJcy9lm61laGUI/zH+MHczl+l4Rbr99W0cm3QMYpHm3VCZuZhhdcFqmUvoPtv1DJucNiErMksh8VAUhaysLIn7lXkP52H49uEKObe6s/Kxgv8YfzKyXEYigUijZ0tSFIVNTptwdt5ZukPRONL0K/bt7eE30k+jf1bUXdr1NBwadgiZdzOVdo5Bmwbhg6wPdH6gRMf5HTH59GR0eLdDi/sZmBvAPdwdjqGOTf6eXF55GS9OvpAphtAFoZh7f26jwRfWvtbwHuQNa1/rVo9/PePP0REIC6v//48fN173T1oURaG2rBZVBVVytSHN/YoiVRdVo+Rlic7OaBWIBEhMEuHSJeDPP9/8LDAZTAz0HggWg4WEogSZ2o7eE420a2n4IPsDicrK3fz8Jvb134e6CtlnGKqrhGMJSLuWRncYGs3Kxwru4e7gGHMk2p/OfkVSDKbuDLpUJYqikBuVqzYVrqRlaGmotEoO9iH2avHORxNpQp9CEIRmoaM/IXcdGiTUMRQA8CT3idTH5jzKQfTeaAVHJD1BrQA72u/A5fcvt76zFjNgG6C7a3eM8pNtFmaNoAYiSoSH2Q/likMdkm15UXlIuZKisaP05KFnpAdzV3MYWmvn+ibqgqIo8Kv5dIehsbIfZOM70+/w9M+ndIciMxFfBLsgOxjb63YiQ9l6rOmB8X+PB8dEspd0hPRcu7ti5J8j4dTJSWnnIAMk6rH0WPAf7S/RzLYxe8ag50c93/q3q8qvwoMtD/Dq4iuZYrDysYJLV5dGL4rbTmmLqf9MlWjNq7w8gIIIqcxLuFCxAbE+M3HTbC4SXspWMQIAxAIxNlptxKUVl2Rug04JxxLwi98vSLuum0mZC68uoN0xQzz2HgseD8jNffPZz0N/Bu8THt7v+r5MbYctC8Osa7NgZGMEFqf1l9jB04MxbPswrXwG+CDrA50uJ6soYpEYNSXqUVlFXuk305F+M53uMLRS5t1M7Oy0E+cWat4SFq8uvkJZWpnS2q8trQWPK/vffIIgCEKzSbZKMqEWQp3qE3/P8p9Jfezj7Y8Rsy8GgeMDaX0hR4kp9P2qL2wDdbe8jiJ0cuoEAHiSJ30S+LWdYTsh5AmxOHaxosKSyeRTk9UiASkLiqIQfzQepk6mcO8l+VqNr7Wd0hZtp7RVQmSS4Vfx8eCnB7D0skTwtGDa4lC22pJa/GD7A8KWh2Ho1qF0h6NxrNtYw7OvJ0ydTekORWZsfTZmRsykOwyCkJuVtxWsvCUre0ZRgCw5PH4VHy/PvYSFpwVcurhI34AWeHnuJWwCbCT+t35NxBc1mtFh4mCCVXmrIOKLZI6FoihkP8iGoZUhbPxspDiufsbfS6evcP7hF/UbLer/5+Cj4fjWb5xM8bA4LIQtD9PYEtAOHRzQfU132AXLVmpf06WWpUJICcCk6p8Hk5MBZ+f6z8z0zeRq2z7EHuUZ5ch/lg+Wkz1++EkPvr7A4mYeNbwHesN7oLdc51RXDCYD+qb6dIeh8X72+RlGtkaY/2g+3aHI7eqHV1GRXYGVOSvpDkXruPdyx/DfhsNniA/doUiFx+Xh0LBDaDerHcbsHaOUc2x23gzP/p6Ydm6aUtonCIIg1BtJ/GmQINsgAEBCUQIoipJqRHbnZZ0ROCkQTD16R1RyjDkI/ySc1hjUwbH4YyioLsAg70FoY91G6uNfJ/6icqMgpsRgMqT/Xl27u0LIE0p9nDJo8kjfE1NPIGBcgEyJP2UwNZU8OcPisHDnmzvw6OOh1Yk/kUCE4OnBSp0ho80MrQzJqHUdJ2m/wq/m49rH12AXZIfQBaFKjopoSWEh8OGHQMeOwHvvSXcsv4qPE1NPoNOSTjqZ+BPyhDg18xTMXM2kGhyVEpGCk9NOYuSfIxutoWziYCJXPMWJxdjVfRfaz2mPgPEBiNoRhX5f92s18VZWBvD5QIVxFACgl1sv8ErtYPDwE9iatZcrJkUMopHmfkWRXLu5wrWbKy3nVgepZakAAKM6TwBASgrQW7aK9U16uvMp7nxzB+UzliC53BbJycCMGQBNXzctakpqUBhXCLu2djCyMaI7HI0WNDlIqvcXdPUrkuj7VV/wysnMK2XptKhTw/+nxBQYTPWvXsBkMTHyz5Gw9LRU2jk6Lugo1aAhojF17lMIgiAkoblv27UEiyV5LW8/Gz8wGUyU88qRV5Un1XmcOzujzfA2YOuTXK86+OPpH3jv4nt4kP1ApuMDbQNhwDZAJb8SyaXJMrUx5KchGPH7CJmOVRRKTOHFyRcoSiiiNQ5ZMRgMTDo+CT3W9pD6WIqicHHFRSSeTlRYPCwWC97e3hL3KywOC/MezcPkU5MVFoM6MnU0xbgD49BuZju6QyFoknY9DTf/7yYqcyvpDkXjSNOvsA3YiNoRhZfnXqogMt10dPxR7Oi4o9X9Dh0CiouBW7ekX8/N2M4Y4w+PR9iyMBmj1GxMPSYmHJmAAd8PkOo42wBbGNsbgxLX/4Pzq/m4890duQdZ2QbYYsDGAei2shtKkkqQfDFZojKKef97VOAbpgMA1vdaj197H4d5bXu8pPlXVNr7FUJxUstfJ/68ANTP+HutVlCL6Seno8euHqgTSr/uXlFCEe58cwfQ10daUf3LUooCYmOb3r80pRS/hfyG+5vvS30udZZ9Pxt7++5F0tkkukPReAO+H4B+X/WTaF9171e8B3kjaFIQ3WFovZQrKfizy58asS4bx4SDjnM7wrOfp9LOMXTrUHRe0llp7Wszde9TCILQPHT0JyTxRzOxWPIShwZsA3hb1pdDkXXRdbpvgBLPJGJP7z3IfZLb+s5aLKM8AwDgZu4m0/FsJhsdHDoAkG3NR3XBK+fh6PijePCTbAlQdRAwLgDOYc5SH1eVV4VH2x4hJSJFYbGIxWLk5+dL1a/YB9tLtE4QoduyIrNwZNwR5EZpZt+dfDkZt764RdZ6lIE0/QqTxcSK1BWYcnqKCiLTTYY2hjB1bHn0cVYWcP16/f+vq6sv9ygNBpOBtpPbwjZAN8uyM1lMeA/yhu8wX6mOM3Mxw5LnSxAwNgBAfZn96x9fR+zBZrIeUuixpgfs2tqh28puWFe5DtZtrFs9Jj8foEChWq/+ntPDwgO+/7uknBygRo5ls57++RT7B+1HXaX0ySFAtvsVRRDWCfFbyG+4t/GeSs+rTtLK6tc2fJ34S0l5MzjAgG2A04mnEZkViUxuptRtmziYwKBdG6QGDAPL2ADt/jfeKjq66f31zfQhrBXS/nyqaLZBthj681C49tDdmaV0oKtfIdTL88PPkR+dT2ZXEnIjfQpBEIpGR39CEn80k/ZBJ8juTblPaWTczsB3Zt/h6c6nUh2naDXFNShKKGoYDa2LxJS44WHa3Vz28pAN6/zJmPjLe5qH80vOIz9ayjeCCsQ2YGPsgbFoN1v3ZmKZOJrgg+wP0GtdL4W1SVEU8vPzpepXxCIxCmILUJxUrLA41E3UH1E4t+iczC8oifrSf0lnklAUr5mzc8PXh2NRzCJYeFjQHYrGkbZfMXUy1YjySppq5I6RmHa+5XVaDh1qPMsvU/r39wAg17p0mqqqoAqFzwvlboeiKHRb2Q2jdo1ChzkdFBBZfZWEjDsZYOmzJPody8sDxIw6+HB6IsAmAG7mbjA3B8o9d+GR9yj8ff+mzLGUpZYh+0E2aopkyx7Kcr+iCDVFNRDWCnX2fkBMiZFW3jjxV1PzZnAAg8GAl2X99tclQaVRQxninttUlLqE4L0VQkS6jscrh2/x9GnTM4+NbY3x3qv30H1Vd9kuSE1ZeloibFkYKa+nACUvS3B88nGJKqTQ1a9IIvNeJjZYbUD03mi6Q9F6Q7cNxXreehhaGtIdSqvOzjuL7YHbIaxT3vIrZ+aewc0vbiqtfW2mzn0KQRCaiY7+hCT+NMz3/b9H6vJULAtb1uq+xcX1a3sA9WWbXLq4wMiW3nUGOs7tiDVFa2SaIaUtCqsLUSeqAwMMuJjJvnZOf8/+mBQ0CWHOspXiqsipwJPfniDvmXRlYxVJz0gPIdND4NZDtpmP6uCPTn9gd/huqY9jMBgwczaDmYuZEqKSXFVeFX5v9zsebNHcWZetSY1IxdM/n0LPUI/uUDSWRx8PrKtch3azNDNJr2+mD/sQe7D0SKkWZastq0X6zXTUlMgxnYiQWWoqcPcuwGAAPj712zIypG/n4vKL+Nrga/C4ujVi/tHPj/Bb8G/IuCPDP9r/XPrgEg4MPgAmm4kOczooLBF+bvE57Anfg8y7kmVy8/IAFmWAr/wvIGFpAgz16l+C1tjeQoHFP7iYeE3mWPp+1RfrKtbB0kt56xIpg5mLGd579Z7EpQO1TX5VPnhCHkAxYSRwhVd9jg8p/yo+4WlRX3LudYJQGjduAK6x59Hh6V/Q83iIG/knkeSyHnlFPKlnHhPEa/HH4hUyIINOTBYT9sH2MLY1pjsUrccx4YDJ0ozXnAYWBjC0MlTqcjxJZ5KQfiNdae0TBEEQ6o3Ud9MwfjZ+re7D5wP79wOnTwOhocD//R9g42+DmREzlR4f0brXZT6dTJ2gx5I9ETHafzRG+4+W+Xiv/l5Ylb+KPIDIyS7IDmxD6bvSklclYOuzYeZqBgaDvtkxps6m6PNlH7j3kn32qbqbcHQCaktqwWRrxkOgOmJxWBKtKaWuihOLYepsCn1TfbpD0XqJpxNx9t2zmHxqMvzH+NMdjlbhlfMQuSkSXgP+n72zDo/qeNvwvbtxdyFEkBCCu7sWKKW0BVqkpYVSLxXaUmq/rwI1oA51WqRCsVKKFHd31zhx92R3vz9OTnw1q2Hv6+LasDtnziTZzM7M877P25yIgRH1tlmxQnjs3x+aNRPqd+kj/AW0D6DNA22E+nSe+o/Z2mh9b2tKC0obFJCUl5AnZJYVlxvUSrvvq31JOZOCk6eTVu1FoSUoqObzPYL7cTLxV05m7Nd7LNZyqGqjJuWKckaFTuLk2SIC/OyJihKCBa5fh379hDb6ZvwplfDff+CVFY8sNwUPx6rAxByXk5w61Yfg4LrXXfv3GqkXUun7iu71si2VDTM2kHI6hcePP27WNX5jwKelD/MK5ll98F7TXk2Zvme6uYdxR1BeXE7i0UScfZ0JaBtg7uGoZcSnI4x+jzkpc2xOHDZs2LBxB2MT/sxMQzcDpaXw889w5oxwwNOqFWzfDjExwuvnzwsbMUvZc1zddBWlQknUWM0CZmOl0ubTy7xCi72LPfYu5t1EnfzhJHv+bw8T/ppA0576Zz+ak3t/uVev67a+uJWb/91kXuE8JDLD/IFKJBJ8fHx0mlckEgkD3xpokPtbKhKJBBc/82Y7NwaybmURuyeWthPbmn3u0AV5qZyv23xNm/vbMGH1BHMPx+rQdV4J7x/OyMUjCWhv2Yct1kjm9Uz2vb8POye7eoW/vXvh2DGQSmHyZCHjC6rWhLrQ9fGudH28a4PGa6kc/uwwl9ZeYvru6XUOw5p0a0KTbk0a1P/ob0YTu0ew5DQkPi18mHl4ptbtb98GJXKCg2uO467ofixNhDj5EUrlpTjIHDT2FRMjvLfy8oR/oUGlRDnG4tfCg8D2gbp+K3qtVwzBpXWXKM0vpcOUDnfkQWiYZxhzW/7OJ+sgsF1VVnD1jD9R+NM14+/SJUhKgvxeD/DuvGJaBDZlXNQ4NlzZQJbbIU6d6sPo0XWvO7v8LOd/P0/P53o2mprTMnshWMom+jUciVSitehnrnnFhmVRlFXEsoHL6PZUN8Z8M8bcwzE7tkAd/bHNKTZs2DA05phPbJ8CZkah0P1X8MHeD5j01yTOx6Tw6qvwzz8QHy8c+Pzwg7A59/QEOzsoKYHUCmeM/R/t5+DCg4b9BnRk99u72fzcZrOOwdwk5CYAEOrR8ILvCqWCaxnXuJWlux0PQOaNTFLOpjR4HPpi52yHW7Cb1tHrjYn2k9sz4K0BBl2MS6VSwsLCkEr167Mx1t4sLy7nxn83yEvKM/dQrJ7zv51nw6MbzFoXVB/kZXIGvDmA1vfZss/0Qdd5xaelD71e6IVPCx8jj+zOI7BDIE9feLremnGpqfDNN8LXkyZBSAiEV8QXJSZCuZryMaWl8H//B199ZYRBWyD5yflk3cgiNyEXpULJ+d/Po1QoSTmbYpC6C67+rrR5oI1ZD4oKCgSB7kLoi3T904fPD39e+drwzq2xL/dFLi1i77VTGvtSKoX3x6+/wrp1QoDhyu8K+Wv8Kta+dVrte0sVDV2v6MvxJcfZ9NSmO1L0E0mpWPYHBECLFsLX169X1eATrT51zfjbvl147DHajxYDhWC+PqFC7b4st4OcPQvyesqGDnh7ALNOzkJq33iOJe5eejczDs0w9zAaDZnXMzn/+3kU5Qq17cw1r2jD8W+Ps2/BPnMP447ALciNkYtH0mFaB3MPRS3pV9LZ8cYOUs4Z9ywm4UgCt0+ar7SLNWPJc4oNGzasE3PMJ7YZzMzk5qpfwNbHL2d+4c8Lf/L8++e4cQM8PGD2bJgyRbD2HD5cOLxpWpFAFVdRCuTMsjOc+lHzBt+Y3PX5Xdy99G6zjsHcTOs4jQOPHWBe/3kN7uv17a/T6qtWLD68WK/rV4xcwdopaxs8Dn3pMKUDjx99HL/WfmYbQ0M5vew0e97do/N17Se3Z8CbAww6FoVCQVxcHAqFbvNK0vEkPgv/jBPfnTDoeCyBzBuZrBixgmNLjpl7KFZP9P3RTPhrAn7R1vX36uDqwOB3B9P+ofbmHopVou+8Yq3Iy+QcWnyIq/9cNXjfebfzyE3I1ft6mYMM/zb+uDdxr/G8QgGLFgliT1SUIPwB+PmBi4tw2J6YqLrfHTvg+HHYuhXS0oTnirKK2Dx7M2dXnNV7vJbKwLcH8mLCi3iGeXL488OseWgNm5/fzNKOS9n19i5zD88giDafZa4xZBVn4WhXZXPs4iIhpFywVfznjGa7z6wsoW64RALjx8ODD4J3mDsxHcayK7ODXoKxueaVkYtH8tDGh0x6T0siqyiL5BRBfQsIEIID7OwgP7/qb7+5d3NkEplOInhxMeyr0DSGD696vnfT3gBkux+koFDJ1XqmVf9of4I7B9uyUmyo5PjS46x5aA1Zt7LUtrPk9crpn05z7GvbXsQUSCQSer3Qi9DeDQ+yNiaJRxLZP38/mdcyjXqf1Q+sZtPTm4x6j8aKJc8pNmzYsE7MMZ80Dj8NKyY7W/fI4jb+bbiWeY1U5UWGNh/Gm2+Cv3/ddmFhQvZfbCx07w5TtkzBwU2znY8xCeunf82UxoKfix9+LoY5OO8Y1BGA40nH9bq+zyt9bBvtBnL+9/PE7Y9j4Nvmt8tUKpVkZmYSEhKi03UeTT1w9HQ0uC2ZJeDq78ror0c32LrNBvhF+eEXZV2in42Go8+88vfjfxO3N45nrzxrxJEZh9RzqWx/dTu9XuxFq7tbUVpQSvrldJp0bfgccnzJcY4vOc7zN57H0UP3epO5CbnYu9rj5OVUI5ts/Xq4cAGcnGDOHJBVTOUSiXCwf+mSsBYMr8dhXC6HNWuq/n/6tHBwL3OQcfSLo7Sf0p4OUy07al5bziw/Q8SgCDxDq4oWdp3VleKsYtpOaotSoaT1uMaRGSzavJY4xwAQ4RVR4/X2Xv2IKf6b/fH7gJfV9iVaxTZtCo89Jnw9aZKMLVu68O23sHOnYC0boIO7r77rlYYS0DYA2pr0lhbFuN/HcSD9MF091xAYOBY7O2FeuHFDyPoLCIBo/2iK3ijSqQ75/v2C+NekCURHC8/tvLWTpzY9BUCpXRol9rc5dapJ5evVKc0vBTD7PtUQlOaXcuTLI4QPCG9QrVAbVbSf3J4m3ZtorEtvrnlFGyZvmkxhRqG5h2HDgmg7qS1NujXBPcRdc+MGMOjdQY1ibjUHljyn2LBhwzoxhLuMrthO/M1Mnh7uc239hR1rnvMFpkypX/QDQfiDqow/r3AvXHzNV+dKqVSa5U3emOnWpBsAp5JPUa7Q3Wup2xPd6DKzi6GHpTWnfjrFkS+OmO3+hmDs92N56uxTOl1zfct1vuv6HTG7Y4wzKB1xC3LjqbNP1WsfZ+24BrjS/enuhPSwLdgNhSarJUvj+LfHWT5iOTnxOeYeyh2Do7sjbkFulJfo4QFoZoK7BDNt+zT6ze0HwLIBy1g1ehXysno86nTEM9wTpVJJ6oVUva7fOGsjC4MXQq2l1K6KJLXHHoOgoJqviWJfbGz9fe7fX2X9B3DypPDo4OrAi/Evcu+ye/Uaq6WRE5fDhkc3sHHmxhrPixnBAW0DGPPNmEYTJJKQAEqU5MligLrC38Bm/ZAqHCnI17wVFIW/iGpd2NnB3XdDx46CReQmK0gmUCqVlBWVmXsYZuVm1k0UkjIcygMrhdradf6kEqlOoh9U2XwOG1ZVVz6jMIOrGVdxtnNmbfccnMqacOFC3WsTjyaywH1Bo3FmyInLYee8nVzffN3cQ2k0BHcJpt2kdjh5WW9pCBc/F1vwnAnZ/9F+PvH/hLzbllvqwc7RDv82/kYvedL50c60nXAHR7zYsGHDxh2OTfgzM/oIf619hQ/ufKeLNTbhtakt/JXklpB6IZXyYvMcxGXdyOJ9x/fZ+8Fes9zfUvjq6Fd8ceQLbuc13Gu9pU9L3B3cKS4v5mqG4W3JjM3xpcc5+Il56042FM9QT7ybe+t0TXF2MQWpBcgcGl+GnY3GzXddv+OHXj+Yexg6kZuQS/yBeOxddDvItKE/IxeNZPqe6dg5WoexRE58DqsnrK6sBRoxMAJnH2cAes7uyYC3B6CUNzxwqdP0Tjx//Xm97aeixkXR68VeNeqTKZVVto4dOkBxeTFbrm+prCcsrgXrE/6USvjrL+Hr7t2FxzNnBOtQELLBpXaNY6vgEerB5H8mM/j9weYeikmIj4cyWRYlCO/pMM+amUdjOvZg5KkcOl9di6aYvPqEP4D109fjtnQhINjEFhcbYOBGpDS/lPku89k4a6Pmxo2QMnkZSXlJALiUhBMYKDwv/l7F/aKuJCYKGccSCQwZUvV8qVzI4usb1pf2rYVMrRs3qPN+8wz3pMO0DlZt+18d7+bePH7scbMGVjZWFHLrCjwTKSsqI+Nqxh0feGBKnL2d8Y3yrcwmtkTSLqZRnGPhH5w2bNiwYcPqaRy7eSsmP1/34vK+ijbCtS4X8PNTvVsXD3vi44VN1uHPD7Ok3RLSL6frNdYGI4GWI1vqLJI0NhbsX8DsLbMrN98NQSqREu0veOZcSruk8/VXN13l287fErtPRSqAkZnw5wQm/zvZLPc2FCW5JeTE56BUaH8o3O7BdrwY/yKhfQxbe0AikRAUFFTDAk5bUs6m8O9z/5Jy1rgFxk3Nxic2srTTUqvLUrNUQnqF0KS7dWXEDHlvCK/nv14p5NjQjYbMK9bCpbWXuPjXRZJPJ9d5rePDHenxTA/snBomYu56ZxdbX9xaabdUkluicx/dnujGsAXDajyXl1cluPj5KenxfQ9GrRzFnxf+BKoO9esT/o4fF0QdZ2d44QWhHmBeXlXmT35KPrH7Yq32sLU6EomElne1JKT7nZH9HRcHRY4xAAS4BuBiX9Pxo0VzGU52juTlVdmCquLWLeGxWbOaz3s09SCkSwDBAXIKCgTLT20xx7yiKFPQYWqHO9YBIDk/GSVKJAp7HOX++PoKz3tWON/m51e1/ebYN/T9qS/fHv9WY787dgiPXbpQ2SdUCX8OMgfCwoQs0YICSK2V8OwW6Mb4X8cTNTZK32/NorBzsqNJtyZ4RXiZeyiNiiUdlrBi5Aq1bSx1vXL7xG2+ivqK40v0K81hQ3e6zurKY/sfwzfSV3NjM6BUKvmh5w/8ed+fRr/Xb2N/44/xfxj9Po0RS51TbNiwYb2YYz6xCX9mJi9P91+BLCsKlFJKZVmkFqo+pA8OBnt7KC0VosEjBkUw4K0BZjv89Gnhw0MbH6L9Q+3Ncn9LQKFUkFog7HgD3QIN0me0nyD8XUy7qPO1EomE4pxiSvPMEw3nFeFFYHvD/BzMxX+v/sdnYZ9RlFlk7qEglUoJCgpCKtV9Xsm7ncexr44Rt1/PkG8LReYgw87RrtFkrZibMV+P4e4ld5t7GDojkUhsmzY90WdeKSssY/9H+7nwZz2+bhZIr9m9eOL0E0SOjlTZpjinmNIC/T8rb267yc3/biK1k3Jj2w0+i/iM61sbbgUn2nT6+EB2WSrnUs8hQYJMImSUu/lnEOO/hOTkuhlZa9cKj6NHg4eHkDEIcOqU8Lhv/j6WDVhWmQlprWTdzCLxWKLZHC9MjVwuZGEVOsQAdW0+QdgfiBaPl9TEjZWXC7ahUDfjb8j7Q5i2bRr3jBfea3//XTebSxUNWa/oi7OPM+OXj79jM7HEgEOnsmD8/aTYVcQyuLkJj9WFv6S8JA7GH+RMyhm1fSoUVcLf8OE1XxOFP4A521/gcLvelEsLKgMLGisluSW2zC4jENQxSGNWqDnmFW1wDXSl79y+hPY1bMCnDetFUa6g72t96fCw8WsoKxVKnQKUbVRhqXOKDRs2rBdzzCe2GczMZGfrHkWdFOeMS0lzZNhzK+uWynZSKTRtKnwdFwfh/cMZ/O5gPMM89R2ujQaSVZRVWYsvwDXAIH1WCn/pugt/kaMjmX1zttrDTmOhVCjJS8qzyhpQ1Wk2tBm9Xuylk23nwU8Pcmun6r9dfZHL5dy4cQO5XPdaVOH9w5l9azbdnupm8HGZk9FfjmbmkZnmHoYNM6FUKjn5w0mSTjQ8w/pORZ95RWovZddbuzi7/KwRR9ZwchNzKw9DgjoGqWwXszuGhUELObtC/+/nsYOP8ei+RwHwi/bDM8xTp6y/5NPJ/DLkF65tvlbjeVH4CwyEW9nC50pTj6bM7jWbwrJCeq1ow/nwp0nz2FbDyq+ggMp6W3dXaPmdK8q8nj4tPLa+tzUjFo2wepvckz+e5IceP5B5I9PcQzEJKSlQVgbOUk/uanEX/UL71dsuPfRn9rRtx0cn5qnsKyFBEP9cXcFPxZn7sGHC64mJcEzLMm0NWa/Y0I/EvEQAHMuaVNp8Qv3CXzMvIb3zZtZNtX2eOgWZmeDuDj161HxNFP7cHNz46+JfJNsdJsfleL3C3/4P97Nu2jrdviELZc+7e5jvMp/smGxzD6VRMX75eEZ/NVptG0udV3wjfRm2YBhNezY191DuGBTlCvZ/tJ8zy9UHL5gLmb2MAW8OoNMjnYx+r8mbJvPghgeNfp/GiKXOKTZs2LBezDGf2IQ/M6NPjb+YGOh9ZTfruxfQO7S32ra16/yZkysbr7DtlW0UpBaYeyhmIzlfsBHzcfbBQeZgkD6HNh/KWwPeYnrH6Qbpz1QUZhSyKGQRW2ZvMfdQGkTbCW0ZuWgkjh6OWrUvzi7mv1f+48wvxtmI5OkzqQD2LvZ4RXjZsqJsqCX1QiqbntlE/KF4cw9FK4oyitj4+EZOfHvC3EOxanSdV2T2MmYemcn45eONNKKGo1QqWf3Aan4f9ztKDWlKwV2DCesfhkdTD73vJ5FIcPET7BY9Qz158vSTtJ3QVuvr85LyuH3idp16NWJ9v6AgKoPBmnkLh/Yu9i5MbifYaV9q+iq3YqqCzS5fFrKzgoOrBB1R+Lt0ScgObDa4Gb1f7I2Lb02bSGuj1d2tGPz+YHxa+Jh7KCZBXPN39R7K5qmbWThyYb3t/INKyHO+wNXscyr7ql7fr/byIPNGJjvf3En6mQRGjhSe+/577Wv96bte0ZdbO2/x9+N/m6/kgZmpyvhrQkC12MP6hL/m3s0BzcLf9u3C46BBQhZpdapbffYJ7QNAltvBeoW/xKOJXFp3qVHYCgd3Cab9lPa4BbmZeyh3JKaeV2xYJhKZhH0f7OPMMssU/mxYD7Y5xYYNG9aOTfgzM7m5ul8TEwPOZSFEttAcgV1d+CtILeDXYb9yaPEh3W9qAG5uv8mhTw8hL7tzI2ZSCoTQ/EBXw9lbdmvSjXcHv8vIliP1uv7cb+c4//t5g41HWyRSCT2e60GzIc00N25E2Lva8/ixx+nzah9zD6UORVlF3PjvBmWFjcOiKD8lnz3v7iHpuC3by1AUZRRx/JvjJB5JNPdQtMLe1Z7J/06m25ONK5PVGgjuHIyTl5O5h6ESeYmcgA4BeDXTHPDg6O7ItG3TaDWmlV73OrP8jMosb23rj0aOjuS17Ndoc3+bGs/Xl/FX3drxzQFv4izxJNflDH9cWln5/PmKj/221bTHoCChn/LyqtcbA6G9QxnwxoAG12m0FkThT9wDqKJvtPB+TlderSH6VEes71fb5hMgPzmffR/sI/5APJMmCQJycjKsUF+Gy2wknUji1A+nKM7RUplsZDTzakYXh0n45A2oV/grLhb+9qEqeCA2Jxa5ov59W24uHD4sfF3b5hPAyc6JILcgfJx86NW0FwDZrse4fr2uJex9K+/j9bzXkcqs/2ii/eT23LfivjtmvjEVuQm5bJ69mSsbr5h7KDqzbc42rYKMbBgOiUTCY/sf4/7f7zf3UOrlyJdH+GXIL+Qm6nEYqCMXVl9okGOFDRs2bNiwbqx/dW3lqNpoqyInR7BUAQgP19xebBMXB3bOdiQdSzJbrZah84fy/I3n7+gIyJT8CuHPQPX9DMGut3axb/4+k9/XxdeFUV+Mou1E7TMeLJHL6y/z2z2/kXEtQ6v2MnsZTbo1IaCtYaxeDcmJb0+wYsQKkk8nm3soBiHtQhq739lNwuEEcw+l0RDSI4SXEl+i5/M9zT0UrbB3tidyVCTBXYLNPZQ7DkW5gvTL6RZR/7Q+7JzsGPvtWO76/C6drru+9TrX/r2muWEFCrmCLc9vYcfrO+q8tv/D/XwW8ZnWtQMlEgkSaU2RUhT+AgKqZfx5VQXU+Lr48mD4SwDsSP6z8sBdtPls1656/1VZf6dOCXUNv+/xPTveqDt2G5ZLfEVCdkBIodp2XcIE4a/Q4SbnL9Yf8FM94682wZ2DefrC03Sd1RUXF3j2WeH5v/8WMkotjT4v9+HVjFcJ7nxnfh6MaTWGu4t/p3nq7BrCn6tr1dcFFaYsTT2aYie1o1ReWpkpWJv9+wWhsHlzaFZPDN8zPZ7h9su3+XzU57TwbgFAsUNCjb2siL2zvc1xwoZGjn5xlJv/qc9CtUSybmSReiHV9h43MYEdAnH1d9Xc0AwUpBRw+8RtrR2DGsL+BfvZ/c5uo9/Hhg0bNmxYJjbhz8zk5uq2AIyNFR59mmTxxJZH6PdTP7XRY2K0b0IC2Ls6MjdnLiM+GaHvcBuEg6sD3s29G0U0p74YI+MPBPuerde3arTkqY97l91r0XZslk7WrSxubL1BUYZ2h9uF6YWUFxunrqFEIiE0NFTvjWXLUS2564u7Gk0d0Ka9mjLr5Cyi748291AaDXZOdrg3ca8jPlgq8jK5LcK6geg7r1zZeIWvo7/myt+WF51fnF1cWdtPl++rvLicv2f8zcbHN1JWpF1mtEQi4eEdDzP807opMQ7uDrgGuJKboDni+/KGy9zcXvczPjVVeAwMhJicGKCm8AfwYK/BQlvJGeLioLQUrlVol21rxd507Cg8XrwoZDoWphUiL7Vep4ac+BwWhy7myJdHzD0UkxEXB0qUPHk9GN+PfVXWA2/i3gR7nFFKy9l3PqbeNqLwV5+wY+9ij38bfxzcBOv6rl1hyBAhm+vzz4U6g6po6HpFHyRSCc4+zjrVZG5sVM8QFpFKwaXCzVcMSLWT2hHmKWwixUzi2iRUxFR16aL5viEeIQCUOQluAbXtPouzi7m5/SbZsdmaO7NglEolf97/5x0135gK9xB3nr/xPHd9pjpYxxzzijZMWjeJ5649Z+5h3HHIS+VkXs+0SCebIe8P4bXs13B0N57wV64oZ8aGGZTNLmP8CttZjz5Y6pxiw4YN68Uc88mdq8BYCPn5kjp2J+oQLXdahrnx27nfOBB/gNicWJXtAwPBwUE45Ek2cxJP2sU08m7f2R7ZUztM5eBjB3mj/xsG7felrS9x18q7WHtprc7XhvULI6hjkEHHow3Xt17nz/v/JPmMdWeX9XqhF2+WvEnTXtoVbP/3mX9Z4L6A8hLDi39SqRRfX1+kUv2m9qCOQfR8rmeD6lhZEvYu9gR3DsY92N3cQ2lU5MTlWE0W5fbXtvOhx4cmsdIxNQqFYMdYZOSEOn3nlSbdmtDnlT74t/U30sj05+8Zf7O041KtM+1E7JzsmLxpMpM3TcbeWbPdelFmEaX5pQR3CSa8f12bhm5PdmPWiVn4Rflp7Gvby9vY+uLWGs8plVUH+UFB8M9D/3Dl2SuMjRpbo13P8A4AFDvGs3VvJleuCJk6vr41BQCoEngSEgCJhNm3ZpstYMwQlOSW4BrgqtXvqzGgUAi/uzJZNgXluWQWZap0mZBKpDR1iQTg2M2rdV7XxmWkKLOIrFtZlf+fORM8PYUxnFNdOrDB6xV9SDmbQtrFNJPdz9JIyUsnJVUQ8QNqmU7UV+evhXcLwjzDKCqr/0Mmq+LX7u2t+d5N3JsAUChLRom8jvCXfDqZ5cOXc2WD5QWK6EJpXilX/7lK8inr3ttYIhKJBO/m3moDz8wxr2iLTTgwPceWHOPLyC9JPGqZ5QmM/Z748eSP/HT6J56LeY7Q3qFGvVdjxZLnFBs2bFgn5phPbDOYmSkrU1Ko3omnBmLkbYtm9rT2aw3A+VTVhVikUmhaoUfExUHcgTjOrVKzEzcivwz5hdUPrDbLvS0FPxc/eof2pn1ge4P2G+0nZDRdTLuo1/VlhWUmj+jPvJ7JpXWXKM3X7eDV0tB10R4+MJzOMzpj52j42h9yuZzLly8jl1tvdoYhybudp/PBvg3NbHhsA78O+9UqMul8WvrQtFdTXAMs0+pHX0pL4aOP4PXX4bvvjHsvfecVz1BPhn88nJDuIUYamX4olUp8WvkQ2CEQB1cHna8P6hhEUCfNwTK3T93ms/DPOPz5YZVtpDKp1p8hY78fWydrMDNTEPCkUkHEc7RzpJVvK3ycfWq083TyJMgxAonCnm3Hr1fafLZtK9h71vj+gsDOTqj3ldYINJKAtgHMOjGLLjO1SEtqBKSmCvNDmWsMAAGuAbjYu6hsHx0g2H1eSbtaWd9NRHQZCQ4GJxXlOn/u/zMrR1XVjnR3h0hBSyRDjQO6OdYrf8/8mxUjLbQAoQlo9VUkGzo4Uu55tY7gX5/wt3XqVmJfiFVZQzwnR3j08qr/fgsPLqTfT/346dRPBLoGYie1w8POj1K7zDrCn39bf0Z/M5rmw5rr/o1ZEI4ejrxR9Aajvx5t7qE0Soqzi7m16xbF2fXX6bTEfVBRZhGnl50m46p2JSFsGI7QPqH0ntMb10DL2gOUFZZx9KujRg9+FpMDnOwst962pWOJc4oNGzasG3PMJzbhzwLI1SERoXqtjXYBQmGWcynqhbzqdf4OfHSA9dPXm+XAttcLvej0WCeT3/dOINpfEP4upV/S+dqTP5xkvut8Yveqzhw1Bj2e6cFbZW9pnSlnqRSkFnB963Wta2d2f7o7dy+922jjKS6ufzOsLStHrWTFXY3jYGztlLUsCllk7mE0Oro+0ZVhHw5DKbd84a/7092Z9t80ZPaNx9qtoADeeQcOHhT+f/QoOjkH6END5xVLQiKRMGzBMO5beZ/efSiVSg5+epBtr2xT2SawfSDhA8M11nMtzCjkrwf/4tCiQ2rbNRvcjJYjW9Z4Tsz28/cHmYa3+M6HdzP2XD4lN3uwfbvwXG2bTxD6aSIk5xAfDwmHEzjw8QFbEIWVINb3cw6OASDcU31B8F4RHfEq7oCy1LXSVURE/H999f1EOs/oTKdHO9V4TswAUyf8gennlR7P9aDfvH4mvaelUFhWSG5pNkqJnNahgXUEf1H4E2v8gebAtuxs4VGV8Hc14yoH4g+QmJuITCqjcF4hB8en4FjuX0f4c/V3pftT3fFvY3kZ4roikUrumAxjU3P+9/P8OuRX4g/Gq2xjaeuVtItpbHh0A1c31c2qtmFcQrqHMOKTEfhHW9a8knEtg83PbebyOuMWw43JjgHg9T9fZ2HwQqPeqzFjaXOKDRs2bOiKTfizALQV/uRyQbwDwYqpfYCQNXYuVb3wF1qR2Z+QAH1f68uD6x8EM5zX9pvbjy4zrCfiurhYsClSKAzX51dHv+Lzw5+TlJdkuE6BNv5tALiUdklnUde/jT/tJ7fHycv00WBSmdTqaz7GHYhj5V0ridkdY+6hGAR7V3u9smAskVZjW9HtqW7mHkajo+2EtvR4tgdSO+v+27VGiouFLL/z54WaTA4Owhqi9oG9pXD488N81forSnJLzD0UAPKS8gxisyyRSLix9QbnVpxTmS0vtZMy+Z/JRN+nvsaok6cT17dcJ/Vcqso2inJFvZ/t1et1nU4+zaMbHuWbY9/U20d0k3C6dHSocV27dvXfT6wPHRcHVzddZftr28mOyVb7fVgqx5ce58R3J8w9DJMh7hPsfQRrs1BP9fZebw18kxedzxCePovLtc4gr1acU6sT/nq/1Jt+r9UU03x9hcesrHouMCMdp3Wk+1PdzT0MsyDuO2RyF9q1rGvnXl/GnyZE4c9TRVnoUoUQLOAgE+Yde5k9zZoJWcbp6VUZg42JjGsZ3Np5y+rdTCyVZkOaMWLRCPyiNdtjWwr+bfyZtH4SUWOjzD0UnVl5diXphekAKJT1r0Ns6I5PCx+m/TeN9pMN6wBVm1PJpwAI7htMy9EtNbS2YcOGDRuNFdupnQWgrfCXlCTY9zg5CVZMol2kOqtPECx6QKjxF9Y3jMjRkWr98W0I/PorzJsHe/cars8P93/IC1tfMLjwF+kTiVQiJackh+R83WwjQvuEct/K+2jSrYlBx6SJxKOJJs8yNAbBXYIZ+/1YQnpotrNLOJzAb2N/I25/nAlGph8T/5rIxDUTzT0Mg9D7xd4MWzDM3MOwYSbykvJY9/A6bvx3Q3NjK2HTJkHk8/SEDz+Ejh2F58+cMe+4VCEvkaNUKMlP1uE02UgolUrWTF7DknZLKCsqa3B/434ex+yY2cgcaqbalReX8++z/xJ3QLt5Xmon5aXElxj38ziVbc6tOsd81/l13svVhb9Tt0+x7PQy1l9er7Kfvn2rvnZ3rwoMq434fHw8dH60MzMOzcC7uRaFvCyQg58e5MgXR8w9DJMhZvxJPYS1YLBbsMZrWguVA2rU5Dtzpmr926GDbmPQNuPPhukQ9x2OZU2IjKy7B3StcMKrLvydST5Dv5/6MWrlqDrty8shr8LoQlXGX6lcEL8c7Rwrn3N2rsoorp31t3rCar7t8q3mb8aCObviLL8O/bVG3UsbhsO3lS+9X+yNdzPr+Txy9nGm9bjW+LT00dy4ASiUisoML0Ow6twqpq6bSp8f+3D3qrvx+9iPC2kXDNa/qdjw6AY2PLbB3MOogYObA82HNce3la/R7lFQWsDldCGaR/GYgnE/ql5j2rBhw4aNxo1N+DMzEon2EY9iRH94uHCdaPV5Of0yZXLVh1hiHQfxgEipVCIvM62v7O1Tt/m5/89c3mBcSwNDcv16zceGolAqSC0QIvoDXQM1tNYNRztHWni3APSv82dqdszbwW9jfzP3MBqMV7gXXWZ20WpDl3UzixvbblBW2PBD5/qQSqU0b97cVoDahlHJuJbBt12+5ehXR809FLWkXUrj7PKzZF7PNPdQDEJxMaxdK3z96KNC5r8o/J0+bbz7NmRe6ftqX567+pxRDze0RalQEjk6klb3tDKIDZtHU496a7UmHEng2NfHdArw0JRl7ezrTPiAcDya1szUSa6I8wkMhFvZwiKxmVezevvIL83nl+zHOBDdC4WkrN76fiJixl98PHg396Zpr6ZWa133yK5HuH/V/eYehskQM/7KHIU3R5Cb5nqUnTqBEgUHDyn4809B0Fm8WHjtrrtUZ4YCXN96nV+H/UrSiaqANm0y/ky9XslNzGVJ+yUc/dqyP7eMRXy28PtxKguhZT2JH/Vl/NlJ7TgQf4AjCXWFczFoVSIBj7oJhECV8Cdm/P1y+heG/DKE5LAvAbh2rWZ7Zz9nXANcrTqrqPW9rRn15SirEqasEVXvEUvcBynKDWgdpIb39rxH1FdRHIw/2OC+bmXd4tENjwIwJnIMJfISsoqz2BOzp8F9m5r0K+lkXLGsKJSSvBKjz3NnUqoiAl/e9rJR79WYscQ5xYYNG9aNOeYT2wxmdiRaC3+i5Y64YQv3DMfLyYsIrwi1WV5BFXv+zEy4tT+RDz0+5PiS4w0Ys+4UZxWTcTWD0jzrsT4RD9SSDJScl1WURZlCEHwCXNXX/NGHhtT52//hftY/st7AI1JPnzl9uOvzu0x6T3PTfnJ75hXMo9mQ+g9mG4pEIsHDw0NjXRZ1ZMdms2/+PpKOGzYr1dQkn05m5aiVjSrby1Jw9HCkKLPIIJaJxqT50Oa8lvUaHabomK5iofz7r3DYGhwMgwYJz3XqJDyePy84AhgDQ8wrloBUJqXvq30ZuXCkwfrMicth7/t7a9R4DR8QzrNXn6XT9E5a91NeUs7ZlWe58veVel9vNaYVU7dMrVOnJrXCHbSG8Odd/+eLi70LG679RZbrEfIdr6jN4qqe8adUgrxUTnG2ddY48Qz1JLCDYYOtLBWlsirjr3tYB0ZHjq4MElTHE4cG8193N/Kcz7N8ObzwgpCtFxICM2aov7Y4u5ikY0k1snrFjL9MNTEXpp5XSvNLKS8pV2nN29i5mCCs6dwUTSqdYKpTn/An2sRmFWeRX1oza7u6zaeqX2FJuWDxLAp/CbkJ7IrZRYH7aQCu1Jru7l5yN1O3TLXqz5rgzsH0eLYHDm7WaZm//eZ2Xv3v1UrR1hLZ9so2Pg34tN4gZktcr6yeuJpP/D9BqTCe0FOuKGf+/vn4u/gjoeHf++LDiymVlzI4YjALRy5kYPhAAPbGGdAGyUQ8duAxHjvwmLmHUYM/7/uThUELjSr+yRVyQj2EObzjoY5sm6O6JrUN1VjinGLDhg3rxhzziU34MzNKpVJrq0+x9oZoySORSEiZk8LV566qreHh5ibUAgIodfQgpGcIroGuDRi17jQb0ow5KXPoMNU6DmBLSqoilQ0l/KUUCCmXXk5eNWxvDMVT3Z5i2bhljIkco/O1SceTuLLxikkP8lve1VKng1FLJe1iGl+0/ELr7CepndRotdHkcjnnzp1DLtf/YCv/dj4739jJzR03DTgy05MTn0PM7hirPay2ZNwC3Xgh5gX6vNzH3EPRiJOXE44ehp9vTU31bL9Jk0BW4S4ZFiYcspeWUqc+l6Fo6Lxy/o/znPzhpIFHpRt5t/MMYu9Zm8Sjiex6axcX11Rl2kskEnwjfXELdNO6H6lMyqYnN3Fo4SGd7l/d6vNWlvqMP6lESodAYQ3W/e4zjKrr3ldJkybCYX5BAaQllfG+0/v8++y/Oo3NEijMKCQ3MdeqM4h0IT1dmCvs7GDe0OfZNHkT97a+V+N1xeXFlCqL6DFKiDBMTRX6eOUVobyAOtpObMvcnLm0GtOq8jmfCgOErCxBjKwPQ6xXdMEvyo/nrj5H7xd7m+R+lsbVis1MsFuTeoW6+oQ/D0cPPByFdL74nPga7UXhT5XNJ9TN+AvxECzxix2E+pOXL6t+f9gwPceTjjN8+XDOppwlITfB3MNRibOPMwHtAijKLKrzmqnnFW0I7BBIxKAIo5ZZ2XlrJ6XyUorLi+nWpGG1zTOLMvnx1I8AvNH/DaQSKQPCBwCwN3av1X2eWqJgE9ovlNbjWxt1bP3D+3NohrCmbHGpxR1V69iQWOKcYsOGDevGHPOJTfizALTJ+Csrq6qFEFWtNrS4mVKHRFKV9ZeLOw9vf5h2kzRHAN/JiIdpIGT+GeJvMyVf6NTQNp8id7W8i0c6PaIy2l8d9/x4D6+mv1qvbZkN9cgcZDi4OdSp81Qf5/84z+1Tt406noZ+kAR2CGTm0Zl0f6q7gUZkHqLGRjGvcB7R46PNPRQbZiJmdwwZ1yzL3kdfNm8W1grVs/1A+HwXs/6MWeevIfPKwY8Psvud3YYbjB5se3kbC4MX1ntQ2BAix0Qy+d/JdHtSOGgrSCvg+tbrOouMUjspE9dOZPzy8fW+/u9z/3J8aU2nhvJySEsTvtYm4w+gY6DgDSttcgY7NR/39vZVdbiS0uzp+HBHQvuqDjCzVE4vO83ipostuq6uIUlPFx79/FD7+61NK19BtPNueZVnnxWEuyeegBYtNF9b38Gll5cwN8nl6uuY2w7STIdrXieaZE6ic0CPel+vT/gDCPMUfH/jc3UX/pzsnHB3cMfZzhmAJu7CpJKtSMLeXrCUvV1tSZwdm82ed/cQfyi+vu4sHqVSyWcRn7HxiY3mHorOpBakMv4P4fPHQeZAhFeEeQekhv6v9+eRXY+oDK6xtHll0P8GMWH1BKPeY+W5lQBMbDsRe5k96YXpeve19PhSCssK6RTUiSHNhgDQI6QHDjIHkvOTuZ5poBooJiIvKY8zy89YlO3/oHcGcffSu41+H5lUOJv46/6/eP7G80a/X2PF0uYUGzashZwcmD0b5syBI0dswV7mxCb8WQDaZPzduCEc8nh6Vol4uiBek6zaEdSoxB+K58LqC5QXW7Y1nEj1n5NcXmWn1RDEjL9AN8uznHLydDJqJGJtykvKWRy6mO1zt5vsnsbCp6UPT55+kq6zuqptV15SztrJa9n3wT4TjUw/7F3sCeke0iiypCQSidGyK+90YnbHsPeDvUa1LmoISqWSP+77g78f+9vcQzEI69cLj9Wz/UTEOn+nTpl0SFozZukYHt7xsFnHED4gnLYT2+Ls42zQfu2d7YkcFYnMXoZSqeTqxqusvGsll9frnn7ZYngLPMM86zyfn5LPye9O1rFfTk8XNlD29uDsXkxSnvC6qow/gI5Bwpuleu0XVVS3+7x32b1WGQwS1CmIbk93w7+Nv+bGjQDRWtPLW0lhWaHW17XyEYS/qxlXGTkSli0Tavtpg1Kp5Oo/V7m161blc3Z2VXXf1Nl9mpLEY4mc/OGkwcV/a8EnYSpdbv7OlE4T631dlfAnWsXF5dQUz7UR/tY/uJ7c13OZ0FYQPUThLykvsbJsRfVM9cK0Qna/s5tbO2/V7soqkJfI8Qr3wtnbsJ8zxqZMXsaE1RNIyE0gyjeK5eOXI5XY1s7WQmFZIWsvCZYQQ5sNpfePvWn1ZSuKy3V3PCkpL+GLI18AMKf3nMrADic7J3qG9ASErD9rIuVsCusfXm/1Tja6IFfIyS/NRyYRNgxFLkU4+1nXvGTDhg3rRqmEb76BmzcFa/f33xdEwBO25GOzYFvVWQDaCH/VbT6rB9dey7hG35/60vnbzmqvD6zQmlJS4OKai6ybts6kItzJ70/y18S/rKa2xu1aSVmGsPs0dsYfwP64/Xxx5Au9Iv1i98aqrC9kaMoKy3APccfe1d4k97MEJBIJk9ZPoufsnuYeikbKS8otKjJSH65vvU7MnhhzD6NBJCYKCybxgM2SuLD6Arve3FWjrpMloVQoGbl4JD1fsPy/N00UFVUdnvepx11VzPi7fr3uoa0lENI9BL/WfmYdQ7cnuzH2u7FG678oq4jVD6wmJz6H4Z8Mp8VwLVKl6qE4u7hOlqpboBuvZrzKkPeH1HhedCYICIDEPMGWzdXeFT8X1T9rMePvTLJm4S9MSPQhzoqT5ZoPbc6Yr8fg4uti7qGYBHGecPHOxnW+Kx4LPLSq1RXpGwnAlQxhDaiL+5hEImHtlLXse79mUJNo92kpwt/ldZfZ+PhGCtO1F0QbC+XlcKtCSxMFt9pozPjTw+qzNiHugtVnZlEmLaIEUaJ6nT//tv7MOjmLns9Z5+e2nZMd0/dMZ9iHw8w9FJ1YfHgxe2P34u7gzvoH1+Pu6M7p5NMWbfd5+LPDHFqkmzW2Oci7ncfGJzYatd74xisbyS/NJ8Irgntb30tSXhJZxVmsu7RO574KygoY33o8rXxbMbFtzSCBSrtPK6vz16RbEyaumVjDjtqcJBxJYO3UtSQeSzTaPc6nnsdjgQf9fu4HgEeOB8nnzRT9b8OGjTuSffvg4EEhYHn0aHB2Ftai//ufIALWl5AklwslC2yZgYbHJvyZGYlEO+FP3BhVt/kE8Hb25mD8QU4nnyavJE/l9dUz/m6fuM3ZFWfJjsnWb9B60PP5ntz/2/04uFtHsfPqVp9gGOFvaoepHJpxiLcGvNXwzlTwxD9PMHvLbI4maldvrjr/PPkPW2ZvMcKo6uLs7czMwzMZ+NZAk9zPmJSXlHP4s8Nc36Le+kTmICNqbBTh/cONNhapVEpUVBRSacOm9nVT1/Flqy9NWvPR0Gx7aRv/zPrH3MNoED//LFg8/vmnuUdSl25PdOOJ00/gFqR9HTNTIpVJ6fRIJ9rc38bcQ2kw4sG5s7Pwrza+vkJ2llJpHLtPQ8wrJbklZMdmG25QOmCM2n61USqUJBxOIDchlz5z+uDip7vQpJAr+Cz8s3rnLQc3hzp/a+I6JSgIWvq0pOiNIs49dU5tzZh2Ae2QICGlIKUyGEkV1TP+Lq65yB/j/7BYod+GgFibWuoh7KZlUplWJQEifQTh70amfofT9/x0D4P+b1CN5zQJf4Zar2hL11ldmfzv5Hqzahs7MTFK8iW3cXGTVwaC1kaV8NfcuzlhnmF1apPrI/x5OXnhZCcUjfQOEzZW1TP+7J3tCe4c3CgcJ6yJ38//DsDHwz+mtV9rHtvwGJ2/7cyy08vMOzA1nP75NEe+OFLneVPPK5rIvJbJye9OknreANZBKlhxbgUAk9tNRiaVMb3jdAB+Ov2Tzn35OPuw5O4lXHj6AvaymsG5Q5sNZUD4ALoEdWnwmE2Ji58L0fdF49HUw9xDASDtQhrnVp4zag36k7dPokSJu4M7AKM2j+LHLj8a7X6NGUubUyyR5ORk3n/vPbpHR9MqJITu0dG8/957JJvLas6G2cnMhCVLhK8nTYKnnoIff4Tx4wUh8MgRePppWLAAdu2Cs2fhq69gyhSYMAEeeAAeewzWrjXv92EszDGf2GYwC0CbGn/VM/6q4+fiR5CboOpdSLug8npxo5ecDH3m9GFuzlyTRuAHdQqi3YPtLLLAcn2In1PiRrh2BqA++Lr40qtpL9oHtm94ZyroFNQJgFO3dfd8G/HpCMb9PM7qinabG6VcydYXt3L+t/Nq28nLTJPt6uDQcHG99fjW9H2tL/IS68jQrY8Ri0Yw7GPrirquTllZlYhz8qR5x1IfgR0CCeoYZFKLYF1oTPOYeHAuHqTXR9cKp+H9+40zhobMK+Ul5Xzi/wlbnjdNYEl1Mq5l8In/Jxz/9rjmxg3AxdeFWSdmNSirUCqT0ueVPrSbXFWDWV4q5/jS4+TE1V0oisKfuL5zsnPSWOPX1cGVKL8oWvq0JDlf/YZczPiLj4fsW9lc2XiF3AQtItUshNL8Ur7v8T3Hlhwz91BMhjhXKFyF3624P9BEc+/mAKQVpqkNIlRFm/vbENYvrMZz2mT8GWK9oi1eEV5EjorEzunOq2V95koO2zs2YXWUIyXy+g+bXV2Fx6KimnXNX+37KrEvxDKv/7wa7bUR/qavn85dK+7ifKqwPpZIJIR6hBLoGoh/qDCn3bolRHeLlBaUWq3jRNyBOPZ+sJfcROuZJ+Ny4jiVfAqpRMr90fcD0LtpbwC23thqzqGpZcLqCTx5+sl6XzPlvKKJsP5hzEmdQ+dH1Tsz6YtCqQBAKpEypcMUAB7t/CgA229uJzY7Vq9+7aR158nBzQazZ/oeZvearedozYulBLN2fqwzr+e9TvgA4wUCn0oWzoH6hPbh+7Hf0+/hfvR5vR7LEBtaYUlziqWxeOFCwps2Zf/8+bxw+TJLkpJ44fJl9s2fT3jTpixetMjcQ7zjyI7NpjjHeIEF2vDtt0IgWYsWgpAH4O4uiHlffgkdOghnXQcPwqJF8MYbsHUrFBQIbUtLhTr2y5dXBTXaaBg24c/MKJVQWCjYsKgiPV34J5FAZGTd1zsEdgDgbMpZlX1Uz/hz8nY2eTSltR3CisKfaKFmiIw/U9A5SNhYnE45rfO1kaMjiRgUYRJxNvV8Kvs/2m+1m/vq2DnZ8ej+Rxnw9gC17f554h8+8f+EkrwSo41FoVBw7tw5FApFg/ppP7k9wxYMs+qo65YjW9J6XGvNDS2UCxeqDsMSE81Xn1Ud8lI5yWcscGDA1he38lXUV42inpM2wt+gQcLjkSNVi2ZD0dB5xc7Rjl4v9qLVWNPbLBVnFxPYIRDv5t5Gv5dbkFuDPz8HvDmAro9X1YuNPxjPpqc2cWZ53VTOjApHUD8dY7jOPHmGa89dq6z3p4qQkCpXiqhpPXiz5E2adGui283MSFFmETlxOeQl6S5kWSvi5rjMUTfhz9PJk35h/RgXNY78Uv2yOpVKZY21vibhz1DrFW0pLy63ur2IoTh1XdjEuMg8KjPuaiMKf6DdZ4gYtKpO+NsXt4+tN7bWEJMvPXOJ5DnJDG7dGT8/YR987VrVNasnrGZJ+yVW+bu6+d9Ndr25y6rWHcXlxTzU7iFGR47G31WohTqy5UgADsUfIqdYi+hkM+Dbyhcnr7rvZVPPK5qQSCS4+rvWO1ZDIJVI2fjQRlLmpNDGX3C4iPCKYGC44Kiz4coGrfu6kHqBIwlHKJMb3yXB1Ky6exWfhX1m7mFU4uDmgJ2j8YJQROGve5PuzOwyk2dff5ah7w412v0aM5Y2p1gSixcuZMEbb7BXLmdLcTFTgKHAFGBrcTF75XIWzJtnE/8aSEleCTvf2qlV5vjZlWf5ovkX3D4pZK0o5Ao2zNhAwhHTWXeXlwvnEQDPPSfU/a5OaKhg9fnZZ0I2YHg4eHrC8OEwfz788Qf88IPgdFheDv/+a7KhmwxzzCc24c/MiFme6uw+RZvPZs3AqZ51ozY1WwIChAOckhLIyVGSej6V26cMkMamJZ8GfsqfD1iOX51SCStWCJNK7b2lUlkVSd+lws3CEMLfV0e/4rPDn5GYazxPdzHj73Tyab2uVyqUFGUZf8OacDiBHXN3NArhTyKVENY3DJ8Wak7lETapTbo1wdHdesU0a8EaD4xqc7xWgpIlZv2tmbyG77p+R2m+5hpSpsbJywl7V3ucvI1z2GJKtBH+mjcXMrTKyuDAAdOMSxeGfTiMLjNNbw8V0j2EGQdn6F1zz1xkXs9EXiYnuEswE/6aQLsH29VtU+198X+7/49HNzzKkYS61me10cb6EcDRUVg7AtxOs0Mqs64tg2eYJ3OS5zDkvSGaGzcSxPdEiZ3uNaX3PbqP9Q+uJ9g9WOf77luwjwVuC8i+lV35nKXV+FvSYQnfd//e3MMwC5cThU1MgLNq4d7OrmqPqU2tWDHjz1ONc6pYX7L6nCOTyiq/Fl1sqtt9tpnQht4v97aamvDV6fVCLx4//ji+kb7mHorWtPJtxar7V7HxoY2Vz0V4RdDKtxVypZwdt3aYcXSqUcgVpF9OJ+umZacCZFzNIP1KOkqFcfcltWv7jm0luA9surZJ6z4+O/wZvX7sxZs731TbLrMok5jsGJ3HaE5CeoTQfHhzFHLzizc3d9wk7WKa0fqXK+SV50Cdg42TaWrDRnJyMnNfe42NJSWoqsrbE9hYUsLcV1+12X42gAMfH2Df+/u0cg8L7x9O095N8YrwAoQz1zPLznB141Ujj7KKxETBOcLFRTifqA+JRMgGnDpVsPhcsQKefx7atxeuCwyE++4T2m7aJGQA2mgY1rWLb4S4uwsLQXXCnyqbT5FKsUdNlpe9vVAHCITMkZ/7/8zWF01j4aFUKgntE4p/G3+T3K82CgWcP19zwli9Wogm2LABdu+u2T4rS2grlULHioD4lBT1WZna8PGBj3lx64sk5hlf+LueeZ3cEt2sZhRyBQuDF7Juqu7FwHUl+r5oZh6ZSdNeTY1+L1MgL5NrrCPVb24/pmyeYqIRNYzinGJ+G/sb+z8ykm+gkbm8/jIfen3IlY1XzD0UvTlW4U7XqiJJ6sQJ841FFe0nt2fgOwNNZmOrC4P+N4gnTj5hNfbS6tBG+JNIYEiFvrFzp/HHZMN4XNl4ha9af8WZX8/g6OFIm/vb1BtYUv19senaJpadXsbtfMMGdIl2n7G3FMTuiyXpuJXYH9yhiBl/BRLdMv4aimeoJ2H9w2ocqorzlaVY9DQf3pzmw1ScQDRilEqIzxb+bkM9Q9S2ra/On0KpoP/P/QlbHEZ6YXpln9pk/NUn/FWnPuGv86OdGfL+EKNmwxgLJy8nmnRt0ijsZEe2ELL+tl63TLvPvKQ8vo7+moMLD5p7KGr579X/+KbtN0YJSCwoLVBp2T2m1RgAdsfs1jqLe0/sHgD6h/dX2ebXM7/i+7EvT/5Tv82qpTLw7YHct+I+swcwKZVK/hj/B5uf22y0e5xPPU9+aT7uDu5E+0Wz7cY2fnjpB34e9LNF7tdsWCc/fP89g+3tVYp+Ij2BQQ4O/PjDD6YYVqNk4FsDufvbuwnqJKzpq3+eFGcXs+ahNRz4WIj69Qzz5LH9j+HdTHC6CesbxlPnn6Lvq31NNt64OOExLEw4n9CXXr0EATAvz3a2YQhswp+Z8aioM6yuzp8m4U/M+DubcrbS670+RLvPlBQJQ+YPoefzmqZqwyCRSHhw/YMMfnewSe5Xm2++gddfhxdegBs34PBhwS9YZMWKmqKgWM8vIAD8/cHBQRAPUxtQl1upVJJSoHsEtq74ufjR1EMQ09RZv9aHVCYlalwUIb3UHwwYAmcfZ0J6hBjN+sTULGqyiJWjVqp83doy0BzcHIjZHUPWDQs5sdMRRw9HgjoG4RrgqrmxBZKUJPyzs4NHhVIdnD0rZHNZEtH3RTPwrYE4ezubeyiNGlHg8dWQRDBokLDAvnDB8qxhizKL+GXwL+x8y3Qr97gDcfz5wJ8kHjNesI0xiBgYQau7W+EV7qW2VpT4vvD2hqQ84WA/xF3z53dMdgx9f+pL+yWa6w2HV5SguXlLwq9Df2XPu3s0fwMWQsbVDM7/cZ6CVAN731oo5eVVe4lchX7Cn1KppKBU959Xh6kdmLplao1MJ+8Kd11Lyfgb8/UYhn1ovXV/9SUrC3KVwhwY4afeqrc+4U8qkXI98zrxufHE58RXvi7WAdRG+HO0q3K6+Pfavwz+ZTBzts2p3NdeuVLXfcUayY7JpqzQwhZqariYdpHzqefr3aNUCn83tlrkHsajqQf9Xu9n8Zb+HaZ2YPB7g40iOK2/vJ4mC5swff30Oq9F+UYxu+dslo1bhkwiq3txLZLykriWeQ2pREq/sH4q27XwFtwTLqVf0nvcdzRKGP3VaHrONt4Z3P44IWi3d2hvZFIZo1eOZsPeDaSeTzV65qmNO4cNq1YxrXqBXjU8XFTE+lWrjDyixkfKOeHsWOYgo+usrkgkElIvpPKR90fE7hPqtzq4O3B963VSz6k+qPaP9sfB3YHcxFyybhn/bK+68NcQpFIYKySvs35941gnmhOb8Gdm3N0FGVxVxl9ZmSBWgeBzWx9RflE0927OgPABarO8Aiv0puRk6P5Ud6Lvi9Z32FbDxYtCoVCA+HiYMwc++UT4/113CQepqak1vYPFA9OgIOEQtUnFPrkhdp/ZxdmVG+BAN+MJf1BV5+/U7VM6Xzv2u7EMfGugoYdUh+LsYo0ZctZE20lt1UaS731/L+sfWU9xtnEL7UqlUtq3b49U2rCpXSqTMjdnLmO/G2ugkZmW5kObM33PdJr2tM6MUjG7r00baNtWOEAtLhbmM0vE0g6F0i6msX3udq288K2B6gKPOnx9q7LUa2eyK5WC374+ASyGmFecvJzIvJFp9DmwOmkX0ri09hLyEuuKcHb0cOTB9Q+Sm5DL4qaLubzhcp02ZWVCBCSAt4+iMuq/ibvmGnzeTt4cjD/I+dTzGp0BxHXn5SsSxnwzxqiHVYbm2uZrrHlwDemX0809FJMgWi/KZNA9tBOjI0fT1r+t1tevvbQWjw89uP/P+w0ynuoZf/V9RBhqvWJDPQkJUGwvbGCaeugu/AGEeoQCEJcjnOaI7zVX17q1W6pTUi7UtK6e8ZdbksvumN0cSzpG8+bC9Tk5VXuvrFtZrBy1ktPLTmv+5iyMpR2X8ts9v5l7GFqzYP8C2i9pzwf7Pqjz2qCIQdhL7YnNieVW9i0zjE49EomEofOH0mJETRtvS5tX2jzQhv6vq86gawhrL69FibLy77M6EomEz+76jIfaP4SzvebgvD0xQlBP56DOeDl5qWwX7S+cHcXlxOldD9Yc5MTnsOmZTWZ3gpFIJXR8uCNR96g41DMAbfzbML3TdMa3Hg8I9sr/jP2HyTcmW2UmtbmxtDnFUsjJzUXb0LJAIEddlouNOpTklfBdl+848X1Ny6fMa5l4hXtRmFYICGd2LyW8xPjl49X2V5xVzOKmi9n99m5jDbkSQwl/INT9c3ER7ENrl8GxZswxn9hmMDOjKePv5k3hgMfDA4JVlN2wk9px4/kbbJq8Se1irSrjT//x6kPapTS2vryVpBOmtYgqL4evvxa+HjgQ+vQRnisthU6d4MknYUqF8+Iff1RtdKsLf2AY4U/M9vNw9MDJzrhZbm8OeJMDjx3g0c6P6t2HsT3wNzy2gQVuCxpN5Nnor0Yz4M0B9b6mVCpJOZ1CwuEEHNy0q63UEEoNZIItkVq/RaK1Ii5sunUTgg/EWqOWaPe5b/4+FjVZRGmB5Zivxx+M58BHByy+9ou2aGP1KVLd7rP6YfuhQ0Ih7fff1y9irqHzikQq4YXYFxj95egG9aMLXWd15fW81wnpafwsdmPgE+lDx4c7Etq77qGeaJ9oZwdFkjTkSjkSJFoFFnk6eVY6D1zNUF/zQczIiYuDVg92oflQ67FKbHV3Kx744wH825rHZt7UiO8JLy94sfcLbJq8ibFR2gfv+Dj7kF+az42sGzrfuyS3hB1v7ODiX1XRKWKgQnl5lUhdG0OtVzSRHZPNhhkbuLn9pknuZ0nEx0Oxg7CB0RQYoFL48xTmoPhcIeNPFP7UZftB/Vaf4hiS8pKwt4eICOF58aDIzsmOmN0xZMdmq+/cwlDIFXR7qhttJrQx91C0olxRzqarQv23QRGD6rzu6uDKV6O/YuvUrQS76V73UxUl5SUcTTzKgTjjFSM21bxiTgrLCtl8TbCLvC/6vgb3tztmN1D/e6E6Ps4+leuHy+l1g5IsFUWZguPfHCd2T6y5h2J0BjcbzM/jfubJboIdq51UEPvKFQ2sWXMHcyfMKboiLy+nPnOZZOB9oDvQquLxW8DF1TpdmMxFcXYxre5uVef51ve25skzT9ZI4LF3sdfYn7OPM31f60vUvcYLOhCJrZhmRdeYhuDiIoh/AHusx3TGIrEJf2ZGrPGnSvgTbT6johrmkQtVQlZyMsTsjuHLyC+5uMb4KSRpF9I4vOgwmddM6/ezfr2wkfTwgCeegLlz4eWXhZTh114ToqKHDBGiEfLz4a+/hOtEYVTMkDSE8JdaIKRZGNPmU6RHSA/6hPbBzcFNr+u3z93ODz1/MKooFzE4gi6Pd7kjxCWJRMLENRN57OBjSO2MO+UqFAquXLmCQtFw4TbzeianfjpFYXqhAUZmOpQKJRse28CF1RfMPRS9KC6Gc+eEr7t1Ex67dhUeLVH4s3O2wyPUg/zblhP52/GRjjxz6RkiBkWYeygNRqnU3uoToHdvcHISLKsvVPsT2LZNeLx1q8pFQFsMNa8Ys95iWVEZ1/69RsrZmpFNDq4OyOw121xZImF9w7j3l3vrtSwWRR5vb7idLyxOAlwDKg94NNHKV9hMahL+vLyqgs6uWFnJVJ8WPrSd2BYXXxdzD8Uk6BIgUB+ihVtMdgxyhW5ZshKphP3z93N1Y9X7yc6uKrixPrtPQ65XNJF1M4vTP50m42qG0e9laSQkgG/uYHq6PEiHwA5q26oS/sI8hLDt2hl/6rLQFUpF5XxUXfgT7YgTcxNRKpWVe1OxzIJbkBvzCucx6J1B6r8xC0MqkzLsw2F0e6KbuYeiFQfjD5JVnIWvsy+9m/aut82srrMY0WKEVhljmjidfJreP/bGfYE7PX/oSb+f+7H95vYG9Xlj2w2+av0V17der3zOlPOKJrJjs1nSYQknfzhp8L63Xt9KUXkR4Z7hdArqpLLd+dTzzN83n0tp6q05d8fuBmBguGbXHzHr72KahdqQ1INnuCcvJb3E8I+Hm3Ucx745xhctviD5jOn8+GUSGcFJwdxcfxN5qXU5YFgCljSnWAqLFy4kJSODX2o/D4QD+4EXgCUVj2nAxevXWbxokSmHadV4hnoyad0kuj7e1WB9DvtwGG3uN25wUmlp1Zm5ITL+oOoczNr2oeowx3xiE/7MjLpNMVS9wVXV96uNOtum6sKfo6cjMgcZSrnxM65ajW3F7FuziRwdafR7iaSnw28VbiszZoC7uyCcDhoEs2ZVbW5lMpg+Xfj677+F68SMP/Gwy5DCX4BrgP6dmBCJREJ+ivEO8ns+15O7l95ttP5Nzb75+9j60tY6zyvKqyZ1azt8vL71On/P+Jvbp26beyg6kR2bzemfT5NwKMHcQ9GLc+eELO/AQGha4VTaqZMwf8XFCXOUJdH7xd48fvRxfFrqedpsBGT2Mvxa++Ho4ai5sYVTVCSIwaDZ6hME0W/QIOHr9euFx4wMOFnt3GnHDkOOUHuKsoo4tOgQN7bpnlGkiexb2awas4rj3wrpsmVFZZxZfsYktQzMQUaFfuHjA7fzhTk62F37rIwoXyHi80q65l1Um4o94qEvjvJp0Kd3jHWmtVGV8aeksEz3gJ0m7k1wkDlQriivzOzSFgc3B5659AyjvhpV43lRhDR3nb+IwRG8lvUaHR/uaN6BmIH4eGiW9izvd/6N/uHqLQd1zfjz9FTdl1QipfjNYhRvK/B1ropaEeepovIickpyKvda4t5LIpEYNUjEhsCxxGOAkOElkxo3OKakvIRJf03icMJhyhRllTXn1l5a26B+HdwcUCqUFltXsTi7mLLCMqOILesurwOEbD91fy9v7HyDN3a+wZpLa1S2uZ13m6sZV5Eg0ThHALTxExYFmsRES0Iqk+Ie7G72gGOZgwwHNwej1Ua/kn6F08mnawTvyKQyuh3vxv7H9luUO4sN6yQ5OZm5r73Gn3I5e4AjFc8vBhYAe4EtwBRgaMXjbmCfUsmCefNs4l8jJzFRCFp2ddXu3EIbWlUkPiYnq06WsqEZm/BnZpo1E4Q3McOjNmLGnybh71zKOYI+DaLDEtXRnKLwl54O/u2DefrC07SdqH39D32xc7TDK8LLpIewZ84IEQctWsDgwerbdusG7doJh+0rVlRFnYo/L3FT2hDhL60gDTCd8Pf7+d95fvPzXMu4pvO1g/5vEDOPzsQ92N3wA2ukXPv3GudW1f0j3jhrI7/f+7vFbkrVETk6kknrJxHc2XAWP6bAu5k3r2W9Rr+5qovTWzLi/BMZWZXl7e5e9Rlw6JB5xmUtFGcXE7M7hvKSxmFpIx6Yu7gIop42jBsnPB49Knxuibaf7hVT+p49wuedqVHKlWx7eRtnl581eN+rJ6wGhKASgJSzKax/eD3nVqpYXFk5osjj41O1vtCmvp9IlF+F8JehWfiLrnCTSc5ywru5t9VEjP92z2980eILcw/DZIhzhb13Kq7zXfH80FOnzD2ZVEYzr2YA3MzS3RLTr7Ufju411/mWIvxJJBKcvJxMYrduaSRUxECF1nUMroOuNf40WX1CXSHPxd6lsixFYm5iHeEP4PbJ25z6+ZTF1Q9Wx+X1l1k1ZpXV1BY+n3YeQG0WqEKpYN2ldbz636t6BROIxGTHUFJeQqBrIFefvcr6B9cDsPn65gb9jkP7hPLc1eeIHh+tubEZCOoYxPPXn6f7090N2m+pvJSNVzcCmm0+744Ugmw3Xdukso2fix/7Ht3H0ruXqi0ZI1KZ8ZduPRl/ALmJudzaZd56lV1mduHJM0/iGaYmaqIBfH7kczp/25m52+dWPmcnteNE1xP0/K6nVpaANmyo44fvv2ewvT2jgQ+BscC/wFxgI6CqEnhPYGNJCXNffZXkZNNlvFojSoWSP8b/wamfTxm03/TL6XzX9TuOfn3UoP1Wp3p9P0PFcLm6QkhF1Y6r6o1qbKjBJvyZmbZtBTuc27erDnxFMjMhLU34o2lV1+K3Bk09mpJSkEJsTizZxdn1tvH0BEdH4QAw1YT7ktyEXHLickxaz03MimnWTPOkI5FUZf3t3Fm1oa1d4y81VahVog+T2k3i8IzD/G/Q//TrQEeWHF/Cl0e/5GD8QZ2vtXO0q9ykG+OAT6lUsnriao5+ZbwPHVMzdctUZt+aXeM5pVJJUWYRZQVlJl1oy2SGidz1buZN63GtcfGzrkxFACcvp3rt8awBMZKp9oFa377C4wHjlUXRm5g9MaydspaS3BJzD4XrW6/zy+BfjCIumQN97PuaNoXu3YXP+g0bYHuFm9b06UI/eXlw7Jhu4zDEvOLi58L0vdPrZAUZgtB+ofR6qRd+rf0A8I305YE/HyD6fss8EGwo4vvC2xse6fQIRW8UsXz8cq2v19bqE6qEv8sOHXhk7wwCOxjfstwQ+LbyJahTkLmHYTJEMVjiJtjdOsocdc7kae4t1HC8kal7Vm5RVlEd0UOT8Geo9YomcuJySD6TbDWitaEoLITUzBKK7BMJbqL5exeFv4KCms9HeEUQ7hleGVygi/BXH6LdZ1JeUmVZhep74KNfH+Xvx/6mOKtYvxuYgeyYbG5uv2k1tcvPpwrCX7uAdirbSJDwzL/P8MnBTziedFzve0X5RXHxmYtsnrKZSN9IBkcM5p6oe5jTew5ypeH/Jk01r5iLY4nHyC7Oxs/FT6VNq8joSKGu8pGEI5XCfW3sZfb0C+vHrK6ztLp/n9A+PN3taSa1naTbwM3Mrrd28euQXynKKjL3UIzG/rj9APQOrXpfyCQykkKSCB4bjJ2jdnbwNmrS2OcUXdiwahXTKqxoXgReB+4B+qBa9BPpCQxycODHH34w6hitnZz4HK5vuU7ahTSD9usa6Ere7TzKi40XHC0Kf4ao71edqIrShDbhT39swp+ZcXOTVdoo1a7fJGb7RURojvT3dvYm3FP4CzubUv+Bp0RSVbcuJQWubrrKzrd2Gj2icvtr2/ks/DOTbrhF4c/PT7v2UVHQr59wUArC5lesQevtLfz8lcqaEam64OPsQ8+mPTXW1zAUnYM6A0JNBX3558l/+KGX4Wv9lRWWcWnNJZKONSCF0sJwcHPA3rmmuCeRSHhw/YNM3TbVZOOQyWS0b9/eoAvUsiLrylZMPJpIxjXrreMjCn+1LbRE4e/iRfNnT9Qm7UIa51adI/6QbvZwxqBJ1yYMmT+Elne1NPdQDIK+dbvuvVd43LxZyPpzcoIBA4S6tqCb3ach55Xw/uE4eWqZuqgDY78dy8iFIykvKacgrQBnH2faTmiLf7S/we9lCdSu++hk54SPs/ZvktZ+rQnzDCPcS/POLDRUWA+VlAg1Iq2FEZ+OYOKaieYehskQ3xNyZ2GhGuSmu+gp1vnTJ+Pv36f/ZUn7JTUOFNQJf8ZYr6ji8OeH+bbTt+TdzjP6vSyJxETIcTnBjo5N6fJzlMb2qjL+uod0J+aFmMrMam2Ev/TCdMb+NpZJf9UVB0I9QwlyC6KovKgy4y81FcSSJ92e7MZD/zyEnbP1HFL3eqEXbxS/QUA7yy/poFAquJAqFAFWK/xJJJUCwuGEww26p4u9C52Dhb2pq4MrGx7cwDM9ntG6Lq0qbu26xX+v/Ye8TDhjMOW8oonrW65zetlpg59/tPJtxc/jfub/Bv2fxuCOEI8QBkcMRomSn079ZJD7dwnuwtdjvmZy+8kG6c9UtHuoHaO+HIVUZr7jz11v7+LyhstG6TurKKtS0O8b2rfy+feHvM9Xo77SyRXCRhWWNKdYAjm5uVRfXb4ItAdmann9w0VFrF+1yvADa0R4hXsxN2cuA94cYNB+nb2deTnpZfq83Meg/VYnNlZ4NFR9PxFR+Gssdf7MMZ/YhD8zo1Qq6dJF+Lp6DR6oEv6iNO/VAOgYJNSuUCf2VK/zd+3fa+x7fx85scY1y201thV9Xu2DnZPpNnC6Cn8ADz8s1PyDqp8TCIJpQMU+Ls2wgRdGo2Og8F44m6p/1ouDuwMuvi4UZxs24tbB1YG3yt7i7m8bT42/7JhsEo4k1Cuim7JWiVKpJDc312Bi/o99fuTbTt8apC9TseGxDay8a6W5h6E3ovAn1n8V8fMT7D6VSsvL+ms/uT0vJrxIy5HmF9t8WvrQ//X+eDT10NzYCqgt8GhL+/ZCxrs4FfTtK4h/Q4cK/z9+vOrwVhOGnFeUSiWZ1zPJTzZ8Ddni7GI+9vmYbS9va/R1TMTsLn3rJ7TybUXsC7Gsm7ROY1uJRMj6kyjk7P7gAOf/OK/fTW0YFfE9Ueqgv/DXs2lP7om6p9LKTRda39eage8MrFHbWBT+xLFVx9DrFXVEjo5k0LuDrNLBoCHEx0Oxg+D1qU0NUFXCX21UORNUJ68kj3+u/sOmq3UtBv+d/C+3X77NPVH34OsrON+Ul1ft3UK6h9BqTKs6AXWWjkQiMVkNsTJ5GaVy/T7nFEoFfzzwB/OHzK8U+1UhZpQdStDdZ353zG6WHFuik+WwrtzYdoODHx8k46oQ8GfKeUUTx5ceZ+PjG5HIDPue8Hf1Z3qn6Tzd/Wmt2ouZfD+e+pFyRc1Mj2+Pf8vkNZMbFChsLbQY3oIez/YwW/3vktwS9r63l8trjSP8HUo4hBIlkT6RBLpVOTPM7DKT6L+jWdZsmVHW3o0dS5pTLAFPDw9q50EUANquOAOBHFuhNo3IHGQ4eRk+UNbYmCLjz1r/FK9svMKGRzeQeSPTLPOJTfgzMwqFgq5dha/FunQi2tb3ExHFnjPJZ1S2qS789ZnTh+euPYdnuHF8xkXaPdiO4R8NN+o9aqOP8BccDHfdJXxduxaGeOiqb6bN9ye+Z/GhxcRkx+jXgY6IhzZX0vUPixg6fyjT/puGs4/hC1BLpBKTCsHGZuebO/mx14+VUZ3yUjl/jP+DS+tMW/hcoVBw8+ZNFAqF5sZaENY/jLABBg7ZMTL9Xu9H/zc1F6e3VNQdqPWrKFu4f7/JhqMVTl5OeISYX2grLylHITfMe99SqG7pqAsSCYwfX/X/YcOEx6ZNhcWzQiHU+tMGQ84r8Qfj+TLyS04vO93gvkRu7brFxic2kp+cT9uJbQnpEcKngZ+ydspag93D0qieCTp782ymr5/OxTTj1duJjgalRErSil1WYaOrKFew9eWtXFpr2s9gcyK+J4plgtWnPsLf1A5T2fDgBh7u+LDO17ad0JZB/xtUo46eOG9l1JOEb+j1ijqaD23OwLcG4uB6Z9X4S0iAIgchE1+s06cObYS/UnlppZBb25mgdjsAB1ndn3n1gDiptMqNprarSnUR2dK5vP4yCYcTTHKvorIiOi7tSOuvWpNTrPsBqp3UjrFRY3m9/+saM8ZE4e9wwmGdD6o+O/wZT//7NB8d+Kje12OzY1l6fCn5pfqLEd2f6s4zl56ptPk25byiiWEfDmPypslmzTADGN96PL7OviTkJrDl+pbK5xVKBZ8e+pTfzv/GgTjdIgrzS/M5nnTcZOcajQF7V3uevfosA942bBaPiPg77BdWt8a9a6Ar/tH+Bheh7wQsaU6xBMZNnszyWlZ0nlBHDFRFCuCpbvFggzO/njFaveDM65ns/r/dpJxLMXjfJSVV6zhDZ/yFh4ODg2BFn5ho2L5NReKRRE4vO41EKjHLfGIT/iyA8HDh8Ka0VLBxAyHy8fp14Wudhb8U7YQ/72be+LT0MWlGkqnQR/gDePRRmDULptZyZxSFv/oOL7ThsyOf8dK2l/SyT9KHKF8hLCIxL5G8Ev2sjWT2VZvBgrQCNS11Iz8ln5jdMRSm618o3tJo92A7hn8yvPJvKeVcClc3XSX9UrqZR9Ywhn80nHu+v8fcw9CJDlM60PnRzuYeht6oyviDKrvPS5f0n4uMRWlBKRf+vEDyGfMV7D674iwf+35M3P7665gYiuqHXxdSLxg1ml1fq0+A/v2hUyfo3VuoJywycKDweNQMZVaDOgXR4/keNO3d1GB9JhxO4OR3JynNL2Xcz+PoMK0Dre9tTXA3zRku1kr198W6y+v45cwveh+eKpSaNx/R0YBEQuKoGYy1gs+EoqwiDi86zI1tuteqs0aUyqoM3jylMAcHupq/FqO4dq4v48+G8ame8dfUQ/Ocq074W35mOSGLQnjm32e1svoskQs1f+sT/moj7k3FOn95t/P40OtDts3ZpvFaS2H99PXsfGOnSe719bGvuZR+iVvZt1h8eLFR79UluAt2UjuS85OJzYnV+rrE3ET+ufoPIAhP9TH4l8E8tekpdt3apff4PMM88WvtZ3ZxrT78WvvRYoT6jEpdSStI44sjX3AsUftCzY52jjzS8RE8HT25nVdVTPOfq/9wPfM6Xk5ePNLpEZ3GMXvzbLp/352fT/2s03XmpCS3hG+7fGu2eUUqk+Ib6YtPCz0W9FpwMOEgUNPmE+BE0gmKxxZz79Z7cfV3Ncq9bdw5zHz8cXaVlXGk2nPjAG2rjP/q7My9k63LJtiUFGUWsf6R9eybv88o/efE5bDnf3uMsj9KSBD2I+7u6gPD9MHODlpUfJxaq93nkPeHMCd1Dl4RXma5v+Wtku5AJBIqs/7EOn+3bkFZmfCH00RLS27R6vN86vk6Vg4i1Wv8ARTnFJN03Li11n4Z8gt73tUytcAAFBdXbVp1Ff4cHWHs2CprTxHx0FXfw/bUAiFqI8DVNLUfvJ29Kw99rmQ0bHb856l/+KbtN5TklhhiaMTsiuGXwb9wa5cVFQvSQKu7W9FnTh9kDoJY2qRrE15Nf5VuT3Uz88hsWBuqavyBMJ9FRwuLqoMHTTsuTeTE5vDXpL848d0JzY2NhKOHI/7R/vhG6eiLqSPTN0xn6tqpjFwxkvZL2rPi7Aqj3Utfq08QFsnvvQfz5gnrDBFxvXHxovB5aUocXB0Y9fkoIgZGGKzPfnP7MSdlDoEdhM88J08n7ltxH71f7G2we1gS5eXVMoO9FdzOFw7ygt10Ezq/Pvo1wQuDeXnryxrbtmolZObcJpgimZvOYzY1zj7OvBD7AgPfGWjuoZiE3FyQy4W/86xy/a0+QQhsSCtIo0yuW33frJtZ/Dr0V07+UFW3QMz4y8w0rzXPumnr+PP+P803ADORkABF9rpn/BUWVtXbE3F1cCUpL4njiccr3WnUZaKry/g7FH+IQcsG8fA6IbNUrPMnRoq7BrgS3CUYn5bGOSA3NEqlkvG/jqff63UzbQxNTnEOC/YvqPx/az8to4OrsfbSWtZdWkd6oebgRGd758q68Yfitbf7/OnUT8iVcvqH9VdpHTyq5SgA/r32r9b91kdheiE58ZZlHadUKI1iOX4g/gCzt8xmxt8zdLrujQFvkPRyEo93fbzyOVE0ntVlFm4Oun2ut/FvA8CldOvJqndwd0BRpkCpMM+HUUFqAfkp+UaxeFMqlZxNEdwYugR3qfHatHXTGPTLILWOYDZsaEtQUBAffvQRYx0dK8W/mcAuqCEG1scRYHdpKTNmalsR8M7D3sWeyf9OpufzPY3Sf9NeTZl5dCY9nulh8L6r23waI6+out2nteLq72q2pCub8GchiHX+ROGven0/bd8bzb2bc0/UPUzvNJ2isqJ621TP+AP48/4/WTZwmdHsVMqLy0m7mEZ2TLZR+q8PMdvP2RlcDRTY1JCMv3JFORmFwoX+Lv6GGZAWiJvBqxkNmx1DuocQMTDCYBuY4C7BjPpyFE26Ne4i044ejjh7G94mVRNOTobzA8+Jy2HjrI0mtyzVl13v7OLr6K8t7gBAW8rLBQsDUB1JL2b97TNOIJje+EX7MW7ZOPrPM5/NatsJbZlxaIZRI1rjc+JZeXYlK8+txNfZFyVK3tn9DiXlhgmMqE1DMv5UERwsBLeUl8O5c9pdY8h5xdBIJBJcA1yROchQKpSsnriaDTM2mHtYRkPMtpHJoMwuozLQS1ehx15mT3J+MlczNa8RHB0hIgIk8nLO78+yeAs+qUyKZ5gn7sHu5h6KSRDnCQ8P6NakC2Mix1QezOpK1FdRBHwaoNY9pD7sXexJOp5Uo4aQj4+whykvrz/rz1TzSkFagUGdK6yB8nIhg06XjL/qe6aCWj+ubk2EQLbzqeeQS4pxcBDmBVWIwp+jXd1GpfJS9sTu4UiicExYO+NPKpPyyM5H6PGs4Q+mjIFEIiHqniiaD2tu9HstOrSIzKJMonyjKJxXyIPtHtS5j3d2v8N9f97H0UTt0v5Fu09t68DJFXK+P/k9AE90fUJlu1GRgvC3+frmBokhX0Z+yfqH11f+3xLWK9kx2SxwW8CON3YYtN/DCYcB6NW0l07X+Tj74GIv1Dj96uhXzPx7JrtjdiOTyHi2x7M6j0P8fDGmxbihkUgkPHXuKUYuGmmW+x/4+AALgxaSddPwKfBKlCwfv5yPhn1UR2iXSWU0u9mMm5/dpCiz/vNBG+qxhDnFUkhOTqagsBAnDw/6AgORsAN4DBiDavHvCDDW0ZEPP/6YoCD9AtPuBOyc7IgcFUnTXoZzxqmOvYs9Id1DjFJySRT+DG3zKSIKf9aY8Zefks+ldZfMuhdpPEW2rBSZTMgQ6tRJ2BzHx8PWrVUWXNrafAJIJVI2PKj+sEvM+CsoELLiujzehcgxkcjL5EjtDK8D2znZMSd5jsH7VYe+Np/qaEiNv4zCDJQokSDB18W4mSjV+X7s93g4ejQ4y7DzY53p/JjhrBN9W/ni28p0PwdTcOK7E5z47gQT/5pIeXE5t3bdou2Etrj4uZh0HDKZjNa6TBpacPL7kzi4OxA9vv6IXUtCXMS4BVp+Rkp95OYKj1JpVeR9bfr2hR9/FOw+t26FkebZv9ZBIpHQ6ZFOZrt/WWEZds52Ro+i+vLol8iVcgZHDOaHe35gd8xuYnNi+eHkDzzT4xmD3kup1L/GnzokEiHYaMsWOHUKundX397Q88rNHTfZ/NxmRi4aScu7Wja4v9i9sXiGe+IV7oVEKiH+QDw0PgfzSqq/J27nC44NAa4B2MvsdeqnlW8rQPtawD4+UHplJ/vuP0Sn688Zza7KEBTnFFOUWYR7E3fsHBv/Vqf6e+LlPi/zch/NWZyq8Hf151rmNW5m3awUe7TBNdCV17JfqzEH29kJriWJicKBQPUABmOsV1QxdctUzY0aGcnJQhZosWNFxp+n5ow/OztwcqpyTnGvppuHeoTi7+JPWmEauS5nCXbtoTYwVV3GnxikkJwvRKHWzvizNpRKpUkiuNMK0lh0eBEAHwz5AGd73YMLS+WlXE4Xoovb+rfV0Frgxd4v8nzP52nurZ2wueX6FuJz4/Fx9uH+NverbDc4YjD2Untic2KJy4kj3Ctcq/5r03N2T1wDBNXalPOKOiRSCR2mdiC4i2Etx/UV/qqz7vI6dt4SbGkntp2o1dxQG1FcuppxlXJFOXbSxv8521BC+4ZSkleCZ6jh65tJJVJGR45mdOToOq/JJDKa32xO3P44Cp8pxNnH9EHJ1oylzCmWwOKFC5n72msMtrdnQXExDsBqlLwkkVCgVOLk6kr/wkIGOTrySHEx9sCfwF4kFKIk0NOTgoICkpOTbeKfCvKS8nALdjPqmkKpVJJwKAFHT0cC2hrOjS5eWG4SqvtHilaIwl9MjFAizcGKynbH7IphzUNrGL9iPB2mdKjUgEyJLePPzIiFHd3cqkS+r76C06eFr8U3uKFwdKw6QExOhnaT2tH7xd7YO+t2YGTJGEP4a4jVZ1phGgC+Lr4mXRhH+kYS6BZosA8OpVLJ2RVnKcqyRYvVpiCtgMCOgqqedCKJve/tJf2y6ev7KRQKMjIyDFYw1qOpBy8nv8yIT0cYpD9j0//1/jxz6ZlKy1VrQ7Tvc3dXnent5weTJglff/MNHNO+zIdJKMkrMVpBalXIS+UsH7GcjY9vNKqFT35pPt+d+A6Al3q/hIu9C28OeBOAd/e+q7WAoi2FhUKhbDBsxh9UuQycPKm+HRh+XnH0cKS8uJySvIZnSSrKFfwy5Be2zN5S+dzDOx/mgd8faHDfloqYOSUIf/rZfEJVLeBb2be0ylh1dYVcv2b4j+ttlEhRQ3J5/WW+aP7FHVPjT3xPGGKeaOEtFNG4kanbz04ikdS73gyvOMuPrVUezNDzio2aiAcw7eTTeLDdg4R7aieqqKrzJ5FIKoXgHJfjGt9r2gh/uSW5FJYV1nCjERO/YvbEsOGxDWTdsvwCkaeXneZj34+J2R1j1PscjD9ImbyMrsFduS/6PgDK5GV8f+J73tvznlZ9XMu4RrmiHDcHN8I8tQvLj/CKoIVPC633k9+e+BaARzo+gpOd6kwZVwfXygCUC2kXtOq7Pgb9bxDdnxYimCxlXvGK8GL88vG0uV+/zOv6KFeUczzpOAA9Q/S3gZvafirvDHyHRSMW8eWoL/XqI8wzDBd7F8oUZdzMuqn3WExN1s0s9ry3h9unbmtubGCix0cz9tuxJt+j2kntONLzCO02tcMzzPCiY2PHUuYUc7N44UIWvPEGe+VythQXMwWYgCDspSiV7ACk5eW88fbbDHzjDeYFBDAZSEfCYpRsAN5NTWXf/PmEN23K4kWLzPntWCRlRWV83vxz1k1bZ9T7ZN3M4qe+P7Hnf4YtxZUmHHljLE3Xz0/Y+8rlcMPKtnehfUO558d7iBgUAWCW+cQm/JmZ6tYWTz4JQ4dChw5C9GOnTtBWu2C8Gv3lluSSXZytsk1tu09jkhOXw/k/zpObmGv8m1VgzIy/rCxhstEFsb6fKW0+jcGtnbdYN20d2+dub1A/G2dt5NvO3xrF495ctJ3Qlm5PdsMrwosOUzrg28qXpr2Nk6KvDqVSSXx8vMF+thKpBLdA40Yd2aiism6Xl/p2kycLnxUKBXz0EVy7ZvShaYVSqeSbtt+w5qE1Jr1veXE5ju6OOHo6IpEa7736y+lfyCnJoZVvq8qo2pldZhLtF01qQSq9f+zNnhjDLaLFLB5XV/WWavrQoYOQWZqYCKkadFpDzysh3UOYfXM2bSfouMCpB0W5glFfjKqRle4X5UdYPyP5jFgA1e1fk/KEjL8m7rpbZwe5BeHu4I5CqeB65nWN7d3cIDcgEo8HRuAR4qHz/UyJf7Q/vV/ujV+UAReCFoz4nvD0UpBfmq++sQZa+ghZuJczLut8beKxRI4vPV5jrhCFv5iYmm0NPa+oQiFXcOL7E8QfijfqfSyNBMHhk4f8P+S3+3/D31W7PYgq4Q+ga7BQIDbH9bjGiO4RLUZQ/lY5h2ccrvOah6NHpSCUkp9SuS8V3WhAOJg6/fNpkwcS6YOTlxMB7QMqs86MxbjW47j23DV+GvdT5br8QPwBZv0zi08OflIptqpDFNjaBbTTa20vV6jfBMsVcsoUQn3QWV1naeyvbYCwDjCUZaSp5hVzcCH1AgVlBXg4eqism6gNj3Z+lP8N+h8v9n5RbyciqURaafdpTbXj8pLy2P32bm5utx6xUhvWXVrH6gurSclPqfOaTCoj3z0fhxYOd4QDgqFpzHOKtiQnJzP3tdfYWFKCqpCDnsDGkhLmv/8+5eXlFOXkcADYjZIpwFBgCrC1uJi9cjkL5s2ziX+1KM0vpfNjnWk2tJlR7+PTwoe7Pr+LQf83yKD9imfwvkYydpNIqhKlzljPxw4AnqGedH6sc+X+2RzziU34syCaN4cXXoAPPoDvvoP33gN7HRPx5mybg+eHnnx84GOVbUS7z5SKtcGayWv4qd9P+g1aA7H7Ylnz4BoSjyYapf/6ECcdfwPqbF5ewmSjVFYdzmuLKPw11HJTVwrLCnn1v1cZ/8d4jRs1bWg2pBkjFo1g6PyhDepHaidF5iBrVGKSbytfQrqHVP7/kZ2PIJU1juk1NzGXa5uvIS9r+HvImKRfTmf73O2knKu76bEWxLnFU0NApkQCzz4LnTsLGWGff14VJW9OJBIJvV/qTdcnuxo18642jh6OPPTPQwz/eLhR7/P7hd8BeKrbU0glwt+3g8yB3dN306tpL7KKs5j410QKywoNcj9DZvHUxtW1avGsTdafpWLnZEf3p7sTdY+B7REsmOrCX2aR8B99Mv4kEgkdAjsAcPK25jeBWP+rdu0vSySkRwgjPh3R6GzFVSHOFVLP27gvcMfrQy8USv2iScWsrkPxh3S+9tSPp9j01CYKUqveJKoy/kxFSU4J/8z6h1M/nTLPAMyEmPHXVMcYNHXCn/jeyHY5TkSE5r5kUlm9Nf4kEkkNu08Hh6rPObHOX5sH2vBqxqtEjbX8uT16fDTTd0/Hv43hAzy/P/E9y04vq/x/qGdo5bwNMCB8AIGugeSV5rE3dq/G/s6nngegnX87ncaRkJvAhNUTaL+kvdoDK5lUxuYpm7n23LXKWvPqaOPX8FpxF/+6yC9DfiHzuh71OIzEsSXH2DhrI6UFmsVYbRFtPnuE9Khcg5qTbsHCfHAsycKsR9QQ3CWYx489To9nTFs/tKywjJ/7/8yxJcb5Wc3fP5+Jf03kQPyBOq/ZSe2wL7WnOL3Y4usz27BMfvj+ewbb26sU/UR6Ar3t7fng3Xe1Egnnvvoqydbq8W0EXP1dGfPNGDo/argSS6ro+XxPg65ZSkurStYYMvmmNmJ5ksN1Y8osFqVSSXlxubmHYRP+GhtiRKcYBV4ftTP+ZA4y7J3tjXJQG94/nAf+eICQHiGaGxsIY2T8SaVVFqm62n3e1fIujsw8wsIRCw03IC1wlDny5dEvWX95PTHZMQ3uTyKR0PvF3rj4Nqxu3ZhvxjDzyMwGj8eSMWbWkak58vkRVo1eRfatbHMPRS0xu2M48NEBcuJ0VOYtCFH489AimcbODl59VXiMja15qPrvvzBjBnzyCezaBXl5xhlvffR6oRc9nulhkr+BsyvOcuK7EwBIZVKjiu0p+SkciBM21PdH16xZE+AawM6HdzK5/WR+u/83XOyFOfJ65nUG/zKY+Bz9Mk3EzxpjCH9QZfd5ygzn4Uknktg+dzv5yQ3LTroTqS78zekzh+I3ilk0Ur+o2R4hwuHX0cSjGtu6uYG0rIS0L3/n4KcH9bqfDeMgvidKnYUPAg9HD70Phns37Q3AtcxrlYFr2tLtqW5M+28ajh5VYo8oEMXGmidAxd7VnilbptDjWdMe9JqbhAQolWXi6JeoU/CfKPCrEv4CS/rinzuc8PCG/TJr1/mrvTd1dHdstLWokvKS+OnUT8TlxKltt/bSWp745wlmb5lNemH95QOkEiljW40F4O8rf2u8d6XwF6Cb8Oft5M2W61u4lH6Jg/Ga538xc1gTD7V/iM1TNvPBkA90Gk91CtIKuH3yNvkplrOeuLX9Fqd/Pm1QW+wjiUeAhtl8GpJJ7Sbx6fBPmdx+srmHojX2LvY06dYEexfTlrjJT8kn7VIauQmGd8CSK+RcSK3K5K2NTCKj96HeXOx3kdQLlp9BbcPy2LBqFdOKi7Vq619cTB/QSiQc5ODAjz/80NDh2dATpULJ9S3XDVJ6Q9yHODhUBZAZgx49hAD4GzeqrEUtnZy4HBa4L2Dv+5qDs4yJTfhrZIS4CwJbYp7qDLvam6t7l93LtP+mGeWg1jPMk7YT25rUFkqcBAwdbSCmLWfqGFDo5eRFj5AedG3S1bAD0oBMKqusnSAWcjcEyaeTubBa/1oMNoyLu7u7QfuLvi+aMUvGWPwBTJeZXXj6wtNEDIww91D0RlurTxE3N+gmBNyyb5/wWFoKK1YI9o1798KiRTBrlmnFPxDq7hmT8pJy9ry7h/0L9lNWWFbjtS1b4I03DPs920ntmD90Po92epRQz7oeZ872zqy8byVDmg2pfG7G3zPYHbObvj/11WsOri7wGANR+DtzRrOFtaHnleRTyRz46AAJhxMa1M++BftY2nEpOfHWK/jrSvUafwCOdo54OulXt6V/WH8GhA8gyk9zVo2bGyjs7JFfuW6WGra6sOvtXfx2z293jDWT+J7ItRN8nyN9I/Xuy9vZu/LwUAx20JagjkE0H9a8Rt3w4GDBvaS0tG6JAUPPK/Vh52hHy5EtCepopKIjFohSKQh/8X4/c+++pkxbN03ra8XPm/qC8H0dQuh+bj9tEhYSEaF+z7g7ZjcPrXmIhQfrD3oMcQ8h2C240hYyOLjufTOvZxK3X704Zgns/r/dHF96XOv2z/77LDP+nkHEZxG8tfOtetvsidnD5DWTUaLkwbYP4uusOnt5bFSV8KdpztNX+HN1cK0Melp+dnm9bfbG7q0UcrWllW8r7mp5F8Huumeti3R7shtzs+cS1lew+DbFvKKJCX9N4OXklw0akPblqC/ZM30P0ztNN1ifDWFQxCBe7vMynYI6mXsoOlFeUk7K2RQUctNlv3k38+bV9FcZ8t4QzY115Fb2LYrKi3Cyc6qs0VudJ7o+wcjRI2nxaIsGB2/fqVjCnGJOcnJz0XYFdQF4XMu198NFRaxftUrvcTUmirKKWNJ+iUndKc79do6Vo1ZyetnpBvdV3ebTmKZunp7QpqJ0rrVk/ZUXlxM5JhLfKPO60NiEPzMjkxm2wK9Y50Vdxl9tq8/GhpglYWjhT9wM65rxZ06ifIXDvCsZVwzW5775+/hr0l+Ul+iesqyQK9j/4X5u7mhc3vqWgkwmo0WLFgadV5r2akq3J7vh4mfZmwWpnRT/Nv44uDmYeyh6o63VZ3X69xce9+4VDvv27hUEL39/mDAB3N2FyP2LhimfohU75u3g08BPDWpxVBs7RzumbZvG9D3Ta0TuyuXw669w9iwcOWK4+/m6+DK331x+Gqe9Lfby8cuJ8o0iPjeefj/141qGbsUYReHPWF75LVoI74+CArh6VXU7Y8wrre9tzVPnnqLV2FYN60gJJXklFh+YYEgMmQk6Pno8e6bv4dkez2ps6+oKSKQUvPAG9/xwT8NvbkTSL6UTfzC+UVmKq0OcK9KVwhzTyqdhf1ePdHyEl3u/rHXWTnVqW9pIpVTWg6uemW6MeUXVeO40MjOhqAhKHITAiqYe2vt9RlXEAFy6VPc18ffn5aV5nXIt4xq/n/+dvXH1RzivnrCapJeTmNh2IlAVlCpafQKse3gdf4z/Q+uxm4vDiw9z4U/tAiIVSgW7YnYBoERZw7YzJT+FxYcWM2XtFMb+NpYSeQn3tr6Xb8Z8o3YuG9Z8GE52TsTmxHIu9Zza+294cAN/PvCnXsGoD3d8GIA/LvxBcXnN7I9SeSkPrXmI8M/CdQ4YaCjVfzammlc0IZFIDC6yuDq4MiB8gF7zso0qdr65k6Udl5J1I8vk9zZGkP25FOFvvo1/G2TSuu/7KR2mMG/OPKb+NBWPppZdn9kSsZQ5xZx4enigbUhHHmgtEgYCObrWUGqkZN3MoiirSK/zVX2Jvi+aYR8NI2JQRIP7MvaZRXV69RIerUX484vy48H1D9J2QtvK58wxn9iEPzOjUBg22kgU/hJzNWf8paaCQiFsig9/dpjDnxv+r2fNQ2tY2GShyeo9FRVV1Z8xVsafrsLf8jPLWXRoEVfSDSe+aYtYX8GQGX89nu3BxL8m6nVtUUYRO17fwcXVJlQh7iAUCgXJyckGn1csHXmZnLgDcZTmG09oMgW6WH2K9OgBjo5ClPz16/DPP8Lzo0fDww9XLY4uG24K0Ih7iDtNujehMM0wte6qoyhXVNpDekV44RlW8/Tx8uWqTL84MycKhHmGse/RfXQO6kxGUQZfH/tap+uNnfEnlULbijXoNTWapDHmFRc/FwLaBTQ4Gr7/vP7MvjkbB1frFfx1pXrtx6lrp/LI+kf0tpPVhcraXwWWL6ZNWD2BV9JeMfcwTIb42ZFUIij4Dcn4A8FC9tMRn9I+sL1O1xVmFDLfdT5bX9pa43nR7jMmpuo5U61Xrv5zlfed3ufcb+oFkcaEWN9P6SF8EepRN0tdFdU/E0prLanEz9Tg8HyNgSylcuFiR1ndGn9AHSGrvoy/Xi/0YuiHQ01aM1gfnrv6HON+HqdV20tpl8guzsbF3oWLT19kXOuq6344+QMvbXuJVedWkVeax4DwAay6b1W9h/nVcbF3YXhzocaxJrvPaP9oJrSdgJeTl1bjrc6giEGEeoSSXZzNpqubarz2+/nfScpLwtfZt7IWpLZsub6Ft3e9rVWt2fpQlCu4svEKsftiLWIfJC+Vc3P7TaPYOloaMdkxrDi7giMJBoy0MzKRoyPp/2Z/k9p9xh2I4/KGy0Y51BezeNsH6PZ5bUM7LGFOMTfjJk9muZOTVm1LJRKtRcIUwFOXaOdGTJOuTXgx/kW6Pm46hzh7Z3v6vtqXwPaBDe7LGKW2VCGebZ0/b3pHK0NhjvnEJvyZGUNHooZ4CFafOSU5FJQW1NvGx0eoCyWXC3+kEomEY98c4/gS7W1KtMW7hTdBHYNMVvNMnHRcXcHZwAkA+gp/3574lpe3vawxCtMYGEP4Cx8QTvR90dg56l63wMnLicePPU7vl3sbbDw2qlAqlSQnJxt8Xvmxz4/8fu/vBu3TkKRdTOPnfj+z/8P9ZhvDscRjPPXPU4QtDqPnDz1Jydc9pVpXq08AJydB/AP46SfB89zeHkaMEJ5rLUwBajO6DE2PZ3owbds0vCK8DN530okkFgYv5NDiQ/W+Xj3Lr3p2SUPYH7eflWdXkl2crfO1/q7+vDPwHQDWX16v099mdsXtREtHYxAeLjyqE0mNNa+U5peSeEx1kJKhyckxT40xQyKXV39fKPnzwp/8euZXlDTsG8spzlEbMAZVtb9KE9M4t+oc8jLj2vk2lDsl26+sDMTSK7H5FVafPg0T/vTF2ceZZkOa4d/Gv8bz1ev8iRhrXqlvTM2HNTdpyQFzk1DhoFzmrHvGX2CgsE8sL6+7boiJgXT3HXxq78EDqx9Q248o/DnItAvKqF2GAqDtxLZ0mdHF4utmuwa44hXupVXbA/FCNlzPkJ5E+0fX+Pkk5ydzd6u7eXfQu2ydupUdD+/A2V67zew9Ufcgk8i4nXdbc2M9kUqkTO0wFYAP9n1AuUIQMYrKipi/bz4Az/d8Hke7+sVeVfx06ife2/seu27t0mtcEqmEP+/7k0OfHjLZvKKOnPgclg9fbtCA6mWnlzF782yTZ1Nq4ssjXzJt3TR+PfOruYeiNc0GN2PIe0NMmv124KMDrH5gtVHWJeL5kir73huZN/jnl39Ycd8KMq5akW2VhWAJc4q5mfn44+wqK0OTvH8EyAB+1VIk/NXZmXsnW0+NUGMjkUiQ2plenlHIFQ2u82dK4S8oSNhXKBRw7Jjx79dQ1k5dW8cO3hzziU34a2S4O7jjai+czqiy+5RKq+w+xQ3WQ38/xIxDMww+niHvD2HK5ikG71cVxpx09K3xl1ogFFIOcA0w8Ig0YwzhT0SfCUvmIKNJtyb4RprX49iGbrj4uuDkpd0izhy4+LkwdMFQWt3dQNvABnA6+TRLTywlPjeeo4lHGfTLIJ0PYPTJ+AMYMEB4PH++6v9iH60qfiRXrwoLJGvH3tmejo90pGmvugeZSmVN2wdDZfx9ceQLpq6bykf7P9Lr+uEthuNs50xsTixnU85qfV2+kNho1CLZYUJJHLNkR25+fjM/9PiBwnT9MkPLisrY/X+7id2nWeH96y+YNg2+/16vW1kMongpkUC5Q0ZlfawgN/3rl31++HO8PvJi3s55atuJ70OniydYO2Ut+bfz9b6nsbmw+gK3TxrvANySqIx2lSi5kdXwGn+V/Zbksf3mdm5maW/NLpFImPzPZHo826PG8+I8Y6hgDF0I6xvG5H8mEz4g3PQ3NxNixl+BTHfhTyKpqp9S2yI8JgacSyNQouRK+hXkCtXif4lcOERSJfwdTzrOoGWDmPTXJKAq4y89HUoadv5kUooyi0i/kl7D3lYdovDXN7Rvnde+HP0lGx/ayFsD32JEixHYSbUPsHyw3YOkvpLK12NUOws8v/l5Fh5cSEah/of/s3vOxtvJm1PJp1h2ehkAr+94nSsZVwh0DeTJbk/q3Gcbf+ENdzFNPzcYiVTC+BXj6Tevn17XGxonLydGfzOa6PHRBuvzr4t/8cXRLzhx+4TB+jQE3UO6A3AsyQpOX83IoP8bxH0r70PmYHh7N00Zfy9ve5kPfvuAG+tuUJBWf1KADRvqCAoK4sOPPmKso6NK8e8IMNbRkbfefpvdGkTCZOAJ4GBRET9//TXdo6N5/733SK6vuPAdQN7tPLbP3U7KOdPX4cqOyeZjn4/Z+179tuzaYsgyFNpgLXafxdnFXPjjAolHTRforAqb8NfIkEgkTGk/hSe6PqE2ylKMrBTr/Pm19sPZ2/pr5JhC+NM14y+tMA0AfxchAlqpFA4/yk1g4dzKVzj1LyovIq/EMLnQWbey+DTwU3a9pXtkZllRGWWFZQYZhw3T8dDGh7h32b3mHoZKPEI86De3X71ikLHIL82vYd97d6u7ebLrk/x2/2+EeoRyOf0yA5cN1JhFUx19avwBdOkCLtVKiYwZU/V1WJiQFVhcbFpxJ/5gPKvuXmXwRWxgh0DuXXYvob3rWpclJAj1gewqzsrS06usn/WluLyYzdc3A0I9NH1wsXdhRAshBVOs7aMN4tjFTCtjUF34M3XwWZsJbRjywRDQMwA6OyabPf/bw/XN11W2USqFmo+//CJ8feaMnoO1EMTAI29vSC4Qgrv8XPy0zqqpD1EkOpp4VG078X2YFtyR+36fgJO3ZQaDyEvl/DXxLw5+ctDcQzEJovDn6lHCo50eZWSLkTT3bt7gfmdunMnw5cP57dxvDe5LzPhLTBQyFG0Yl4QEUFBOjlwQv0M9tbf6BNXCX2wsuJRE4CB1pEReQkx2jMo+NGX8yRVy9sTuqbQI9PCoWvuIa5XcxFy+bvM1u/+3W6fxm5LLGy7zdeuvufHfDa3a55fmI5PI6BtWV/hrCG4Obvg4qz5xyyjM4KujXzHnvzmVvxt9CHQL5MtRX/LxsI+Z3mk6225s4/MjnwPw87if9bIQbesv+MteSNOuTmJ9tJvUjqY9TbcHUIeLrwvdn+pOaB/d/u5UoVQqOZwgnG72DOlpkD4NRfcmgvB3JuVMg95XpmbT05tYMXKFye4X3DmYthPbam6oBxse3MCaiWvoEdKj3tdlUhlHeh7B/4g/YX3DjDIGG42fF19+mdfnz2eATMZQeydWAtuBFcAIJycGyGS8Pn8+7/zvf2pFwsVAOHAZ+BDon5xMzuXLLH77bSKDgxl91113nAB4a8ctDnx0gJQzphf+PMM8Ce0TineLhtkLiefjpsj4A+hdYR534oRlB4s5eTkxr2AeIxaOMPdQbMKfuTFGyv+3Y79l6d1LCfdSHd1aO+NPUa4g42oGuYmG9aPf9fYuziw33UlbmqCxGWXSESMYdMn4K5WXVtrDiRl/R4/Cs8/CsmWGHV99uDm4kfRSErlzc3F3dDdIn64Brng398YtWPc0lNM/n2a+63xu7bxlkLHYqIlEIsHHx+eOsTgzF2XyMsb9Po5xv4+juFzwWAt2D2bJ3Ut4sN2D7Jm+h3DPcK5lXmPG39plUpeXVwk9ulh9Ajg4VC2AIiOFfyJSadX/TWn3WVZUxvUt10k8YroIJ9Hms2PHqs+AhoqdK86uIL80nxD3EJ3r1lTnvcHvcfmZy7zQ6wWtrzGF8BcSIrxHCgpUf7YZa16JHBVJ/3n9cfF10dy4Hrybe/PEqSfoOkt1PYTvv4fVq6v+n5RkmqAbY1G97qOYUSzWdtYX8eDucvplcopzVLZzdRWygQo9gwkd2QZHd93s3EyGBB748wG6P9Pd3CMxCaLw5+XmxJejv2TL1C0NEoJFxIwkMUNJW65svMK6h9fVyOT18RHePwpFlQ2lqdYrl9ZeYtucbRRlFRn1PpZEQgKU2N9GgQJ7qb3OjiNinb9Ll6qcAnJyhH9SiYwo3yjh9fRLKvvQVONPzFJOzq+yUGvWTHhNrAXp4ueCUqE0SpaMoQhoF0DfuX0JaKfdz3jNxDVkz81mSLMhRhuT6DRTnR23dqBESbuAdgS7Bzeo/ykdpvBK31fIKc7hkfWPAPBM92cYFTlKr/6qZ/w1xP5KvLax7YNuZt0koygDB5kDnYI6mXs4NWju3RwfZx9K5aU6OVqYm6KMIgpSC/SqH6qQ62afUl5cblRr9EjfSO6Lvg9v5/oP7mUSGUqpEgWNwPbFDNjOVqp48aWXiE1IQDrgDV52i2aGbxM+j45m4BtvEJuQwIsvvSS0qyYSjnSqEgmnAu8Ce4F7gZeA28A7wJ/AUiB/61bCQkJYvGiRGb5D89B+cntmnZxlFucqiVTClM1T6PaE/mccYFqrTxDWix4eQi1qcV9hqcgcZHUSrMwxn9iEPzMjlZrnV1C7lkLK2RS+ivqKkz/oV1i7PpRKJfvm7+PSGtUbQ0MjTjrlHtfp/3N/lh5farC+xYy//Py6Be9VkVYgKJEyiaxyQXa9IjnBVJNUsHuwQScXB1cHZhyaQY9n6o8sU4dPSx86TO1glNpfNoT5JCwszODzSlFWEbv/t5uLa/Sz4TEmSoWSzyI+Y9sr20xzP6WS5zc/z85bO0nMS6w34r2ZdzPWP7gegH1x+9TaYYnkVsRcSKX6WTtOmgQ9e8KT9bgsRQlndFy5Uvc1YxExMILXc1+ny8wuBusz42oG3/f4nsvr67cuFoW/nj0NY2F5K+sWL20VNjHP93weqUT/v6v2ge2J8ovSur1CAYUV5+bGFP7s7aFJhW6k6mdlrHmlodg52hHUKUjl58nNm7BxoyBWPf20kPlaXl6zhpS1Ic4Tnp5wO18Q/oLdGnaI6+/qT4RXBCDY76lCIqnKLC4oME99Am2Q2ctoO6EtYf3ujMh20RLY3TCxXZWIwt/B+IMolNofGKZdTOPs8rNk3cqqfE4iqcr6E0UdU80rN/67waGFh1CU3RmHnmIQh0RpzwvdX+GJrk/o/NkVESHUSS8srPp9iY9BQdAmQLAwVFdGQFPGX6CbEIFaIi8hpySn8r4AtypiA+0c7Xj28rP0n9dfp/GbkpDuIQxbMAzvZtpHy7s5uBlEnK9NSXkJg5YNosnCJnXcJrbdENbII5obLurc08mTZ7s/S4fADnwy/BO9+4n0jcROakdeaR4Jufptjve8u4f5LvMpuF1g9vXK9rnbWdJhCcXZxQbpT8z26xzUWef6icZGIpFUBg8dS7Qeu8/7f7+fJ049oXP90FM/neKrVl/pVA/r9C+nWeC2gJjdMTqO0jDYSe1wz3Wn+HjxHRUAYygsdQ9kLoKCgujc5U26D7rIb38ncuziRd54802CgmqWHBBFwgFvvMFn0dE85OrH78AW4CCwAEEA3AJMAYZWPO4F9ikULHj99TtG/JNIJQR3DrbosjrqkMshq2LJ72uiak4SSdW9srNNc099SDicQMKRhDpBJuaYT2wzmJlRGKHoklKpJKc4p96IPxEx46+61WefV/oQMTDCoGN5/sbzjPpSvwhAfUhPB7mkhAVJQ9kft5+nNj1VmZHTUFxchMwa0N7uU7T59HPxq9x4i1mJ+ZZbHsdotBjRgvHLx+PdvGHp5DbqR6FQEBcXZ/B5RWYvY8+7e7j4p+UJf0WZRbg3ccfOSftaKA3h62Nfs/TEUiRIWHXfqso6mrXpENiB7dO2k/BiAjKp5mh10ebT3V1YzOhKcDC8+WZVTb/qiMKfKTP+pHZS7F3sDdpndkw2WTey6o22zc6uEjZ79IDwioR3fWtKyRVyHln/CHmlefQN7cvLvV/Wr6N60EYwKaxKljGq8AeaRVJjzSsAayavYeXolXpdm34lnbIi1b6BJypK4XTvDqNGQWiF65Y56hkaCnHd4OpaVce5oRl/QKVFlKY6Pa6uYF+cx/KOn/LfK/81+L42Go6Y8ad0S1KbsakrHYM64mrvSk5JDhdStbfg6/5Ud17Pe52Q7iE1nq89JxtzXqnO8I+HM/vWbJx9rb+cgTaIQYVNPIJYPPpjvhz9pc59SKUQXVGeTLT7FIW/8HCI9hNevJSmOrDz/SHvkzM3hwVON2D74DqvO9k5VdpCJucL0Rii8GeOWpCmQBcBXR8c7RyRK+XIlXJWnqv6XFUqlfx3U5ivh7cYbrD72UnteGPAGxx//DjO9vr/fTnIHIj0Eawp9K3z5xnuScTgCMpKykwyr6hDIpMgL5Xj6GEYke5IohDV1qtpL4P0Z2hE4e9oknq7cEtCn4BopVLJwU8O4uDmgL2z9vsb92B3mg9r3mArvfr44/wfLNi3gHMp51S2kUllRF2JouCZAm6fuDNqHxsSU61VrAlRaNHkUBQUFMQbb77J5BkzyCrIoD+CzedcYCNQ3bg4GXgf6A5MAwJKS3llzhzOnrWeTGJ9SDqeRNyBOL2yjw1FSW4Ja6esZd+CfXpdn50tBCxLpbq7VjUE0R4+x3BbH4OzY94Olg9fXqesiTnmE5vwZ2aMETH91dGv8PrIi+c2P6eyTe2MP3sXe4Z/PJyIQREGG4dEIsEr3AvPUB0LVjWA9HSQS/MJc6+qb7L+8nqD9F09skBb4U8UX6vb7Jha+Pv32r+M/2M8H+7/0GB9Xtl4hU3PbNK6mL0N06BUKsnMzDT4vOLg5sCTZ57k3l/uNWi/hsDFz4UZB2cw5D3j2SaJbLuxjdlbZgPw0bCPGBs1VmVbqUTK0OZDVVqv1EZctBhjwSSKgbGxUGTCYM+cuBxO/XTKYMXkW4xowStpr9B6XF2x9dgxoYZbZKQwTzc04++LI1+wL24fbg5uLB+/XCvxVhMX0y7ywJ8PMO73cRrbijafjo5VNQuNhfizUnXYaqx5BQQLpLLCMp37ViqVLBuwjF8G/6KyzalTwmOXiqRTQ2SBmhtx3eDmRqWNuEGEvyaC8Kepzp+bG5Q7uODS1AePph4Nvq8xuLLxCp8EfMLlDaqzkRoTovC3ze45vD7yMpjThZ3UrvKg+VDCIa2vc/RwxMGtbjZTbeHPmPNKjfG4O+IV4YVUZt4tb1wcTJ4MM2fCF1/Avn3GsR0Whb+mDSx3Jtp9XqjQfMXfW0QERPtXCH9qrD4dZA54OLhjl7AOUnfX26a63SdUWX3eulVVczbxWCI73thB3m3D1Ck3NH8+8CcbZ23Uqu2gZYPo+UNPTiSdMNp4Hu7wMADLzy6v/Nu6lnmNuJw4HGQODAgfYPB72ssaHuTVNkB4w+kr/HV6pBNT/p2CV4SXSeYVdQz9YCjPXn5W52wyVVhqfT+R7iHWl/EHcG7VOY58WV8VsvqRSCTMOjmLcT+PQ2on1fo9FnVPFJM3TTbKedjKcyuZt3Me++JUH9jLJDLiwuKwe8EO3ygTpeM0Iky1VrEmxDMLTy3e0osXLmTBG2/QGiUzgR+AwdQU/cS6f/uBF4AlwOtAP6WSbp06NerMv73v7WXZgGU17PFNjYObAzf+u0HqWdVJQ+oQHfd8fATxz1SIZ2aWLPz1m9uPuz6/q06wiTnmE9OkSNgwKeJGqrbNR3XEjL+cHCguFiywDE15cTn5yfm4BrgaPPNDFenp4CD3Zc0923nt6FR+P/87P536iQfbPWiQ/n194fZt7ev89W7amyMzj9SI8hSFvwLDnIVrJDE3kfWX11MmV50ZoStx++M4/s1xer3QC99I7ReRu97eRWl+KSMXjTTYWGyYhsD2geYeglm5nH6ZiasnolAqeKTjI8zpM8eg/euyiNYVHx/w9xfmnmvXoEMHw9+jPm78d4ONMzcycc1Eou+LNkifEqmk3sOUcxXBrl0ryr01NOPvofYPsePWDu6Pvp9m3s3066QWjjJH1lxag0wiI7MoEx9nH5VtTVHfT8ScgtiktZP0uk5RrqDHcz1w8a+/PmBxsVCfCqqEPzHjLz5er1taBNWFv4+Hf8x7g9+jTNHwz3Yx408b4U8pldHpi8foZfjzY4Ng72yPX2u/OvUU6kOpFP5Zs4OTKPxlcg2AcE/V9b11pVuTbuy4tYNTt09pfY1SoSTlnGAnEtSxyvpJnGdMXYsj62YWMgeZ2YXq338Xfld5efDff8K/KVPgQcNsTyoRf752QZdJyvMg2E0/u/82Qtk1Ll4U/kaqZ/wFBXdhZueZmuveKiuUzfD6v8kgtyAup1+uFP6aNhX+FvPyBNsoHx8hGn7//P2E9w/HPdjAfrYGIDsmm9J8zfUfisqKOJxwmDJFmdrP/oYyoe0Entv8HOdTz3Mm5Qydgjrx3w0h269fWD9c7PWrqWts3h/8PguGLqCZl2HWW40FuUJOeqFwqmqpGX/9w/qzftL6SgHQWjjy+REyrmXQ/enuWgeG2DvbE9wlmEOLD3Fj6w2mbJ5i1tpvF9KEyAyxTmZ92EntSA1MRTJEYtJgfBuNk9LSKlcaTcHKycnJzH3tNfbK5UwDgoDPEMQ9kcVU2X7WDm2YAhxRKhk7bx5AZQ3BxsSg/xtE5JhIXANMsOFXgUQqYfat2Ti46mdBLibEmKq+n4j4/rNkq88WI1qYewiVWM1Wd8GCBXTv3h13d3cCAgK49957uVKrYFFxcTHPPPMMvr6+uLm5cf/995MiellWEBcXx5gxY3BxcSEgIIBXXnmF8lohl7t376ZLly44OjrSsmVLli1bVmc8X3/9NRERETg5OdGzZ0+OHrUcewMx+lu0gaoPV1ehICZUHYyeWX6Gr9t8TcZVLdPZNJB0IonPm33Oie+MF9lYnfx8ZWU2S4C/jPlD5gOw/eZ2YrMN4xvjU7FX0zbjz93RnR4hPSoX60plVVSEqYQ/sYi7WA/IEPR+qTcvJb2ETwvdNq9X/r7CtU3XDDYOG6ZDXirn9snb5CdblkftgU8OcOJ7488xT/zzBDklOfQN7cu3d3+r1UYvNjuWV7a9wnP/qs6+FhGFPw8jnU2aw+6z5V0tmbhmIuEDGn4QXZBawN7395J2Ma3e18XaqWJ2oyjyZGdX1UXThSC3IDY+tJHpnabrfrEKWvi0oI1/G+RKOXti9qhtaw7hLz6+KsvC0pHZyxjw5gCVxcjPnhWyaQIDqxwOGlvGHwjWbm4OehQFrUWX4C482ulR3hzwptqapOL70ZKtypsPa86jex/VOO+cPg1PPQVPPCEIxdZKXh4oUZAqF9ZWkb6RBuu7c1BnAE4m61D/WwI/9vqR7a9tr/F0QIXxRXq6aeeZP8b/oTYzuCEolUp23tpZeTCvipQU2L9f+PqZZ2BwhfPl7t2G/1mIgQ1rS58kZFEIq86t0qufVq2EbPPMTJg2reozNiICWvq05Pt7vueJbk+ovH7p8aXM/PsJjrb7Gjq8V2+bph5NCXYLplwh7MMdHCCkwiFWrPPX5v42PHX+KSIGR+j1fRibWcdnMXXLVI3tjiUdo0xRRpBbUGVNVWPg5eTFPVH3APDrmV8BoeyEo8yR4c0NZ/NpaKL8omjp01Jvd4WS3BJ2zNvBxdXmLUlQVljG7v/tJm6/YRYaMqmMm7NvEv9ivFHfNw3B29mbca3H6eU+8O6ed5m4eqJaq0pjcfe3d/P0hae1Ev0urb3Ennf3UJIr1PbLjskm40oGeYnqM5Hzbufxx/g/uPqP4TdfhWWF3MoSJkp1wt/YVmN5e8Db9AvrZ/Ax2LjzEPfUdnZVdb9V8cP33zPY3p6egCeCnWcOggBIxf/rs/2sTk9gY0kJc199lWRrLtKugqBOQXSd1dXcw9Bb9IOqc3FT1fcTsXSrT3Pat9aH1Qh/e/bs4ZlnnuHw4cP8999/lJWVMWLECAqqqScvvvgiGzduZPXq1ezZs4ekpCTuu+++ytflcjljxoyhtLSUgwcP8ssvv7Bs2TLefvvtyja3bt1izJgxDB48mNOnT/PCCy8wc+ZMtm7dWtnmjz/+4KWXXuKdd97h5MmTdOzYkZEjR5Kaqnt6rDGihEI8hF1TUl6S2jRS0cblzJmq5xTlCgozDJNq7BrgSu+Xe9Oke8NtqLTh+X9e4Xzoc7j4ZuDkBM28mzGk2RCUKPnljGE2/bpafdYmNxfKKoLzy8qEqBljo40QrCtugW64B7vrbGPyxMknePz44wYbh42aSCQSgoKCjDKvxOyO4buu33HxL8up86dUKtm74CBHvzujuXEDWXnfSsa3Hs/aSWtxtNOubkdxeTGfHvqUH0/9WHmwpQpjWn1ClSB22YTudx4hHkTfF42LX8MjzGP3xbLrrV3EH6ybrlVcXJXl0LKl8OjkVJXZrq/QI5FIDP631D+sPwAH4g+obWdK4S8kBGQyIYKzvs82Y84rZYVlHPnyiMFtGavbfIrDFsXgxEShELk1Ulv4MxSuDq78NO4nnuz2pNqDV/G+t3dd5t9n/0VRbn01T3Jz4ZNP4K23hPdCcnKVnaE1kpcHxfaJlCmLsZPaGfRwuH94f769+1uWjFmi9TUSiYRhHw+j21M1RXkfH+FvsbxcCMgw5rxSnc4zO9PtaQ2ZaXryxZEvGPrrUFp80YIF+xZQVFa/l/aGDYLA17kz3HUXPPmkcGiWmGj4DMiEBFCiJL5EOEhXdyCsDgcH6NNH+DonRxi/j49QT1gbdt7ayY9nfuZSXhrEr4OsunV6lo9fTtLLSUztUCWciXX+xAxD1wBXAtoGYOdo3SZFYjZ139C+Rn/PT+swDRBsAMsV5fxv0P/Iei2Lp7s/bdT7mhOJTML+Bfu59s81k8wrqsi8kcme/9vD9a3XDdpvU4+mZs0sMxbPdH+GE7dP0HFpR6auncrtPNPVoAvqFKR1FvGpH09x8NODlWueIe8P4ZnLz2jMJM+4msGVjVfIiTP8yfSV9CsoUeLr7Iu/i7/KduNaj2Ny/mTOjz7PrV23DD6Oxo6p1irWgphd5elZtb9SxYZVq5hWEVk3DlhOlQAI9dt+1kdPYJCDAz/+8IN+g7ZQUs6moJBbxj5KqVBy8a+LnP7ltM7Xikkttoy/mhz/9jifBn1K4tG6LozmmE+sZhW9ZcuWGv9ftmwZAQEBnDhxggEDBpCTk8OPP/7IqlWrGDJEqPX0888/Ex0dzeHDh+nVqxfbtm3j4sWLbN++ncDAQDp16sR7773Ha6+9xv/+9z8cHBxYunQpzZo1Y+HChQBER0ezf/9+Fi9ezMiRgj3hokWLePzxx3n00UcBWLp0KZs2beKnn35i7ty59Y6/pKSEkpKSyv/nVoRLKJVK5BUnUBKJBKlUikKhqCHYic/La51UqXo+yFWIoyiRl5CWn4avi6BWSSu8jMRikh06wMGDUk6ehAkToN3kdrSb3A4QRFKZTIZSqaxTfFImk9UZY33PezX3YtjHwwzyPUmlUiQSSb3PA1xOu8yvVz9HHlhOH/lYlEohqvGprk/RyqcVY1qOMcj35O0NSqWUzEyJVt/T2ktric2JZUTLEbT1a8vt28L1QnvIz5fg6Vn/91R7jKqe1/Q9BTgLYdapBamUlpXiYO+gcuza/p5QQm5cLnK5HO/mVTXMNP2eFEoFdi52la/r+z1p+j3p8z3p+94z1O+p9vd0I/sGzb2aI6lWDVbT96RUKvH396+cVwz5PQV1CaL/m/1p0rNJ5XXm/j2dPQuHuz1DE7dcFAolUqnxfk9NPZry14S/aszZmr6nFt4tcHdwJ680j/Mp5+kQ2EHl95STI/z+3N2V/8/edYdHUb3dM7ubbHrvhIQUEgiE0HvvXQSkSRFFaYoIggp+/rAXQMWGAkqR3qvU0DsJIYQEQkjvvWzqZnfn++NmtmTbbE1QzvPwbJi5O3Pvzp1b3vO+54VYTEvraKy+FxJCg6aBp09l1zfXc6oqrIKNqw0oDqV3m4JGBWHO1TlwCXZRGsuTkgCJhAMXFxrOzrI6+vpSyMujkJoqQfv27NpE0zQ+vfoperXshUH+g2DJtVQqb8j71Nu3N/6I/gPXM64rlZdvU0UFmSusrWkAyr+7MZ8Tj8eBt7cEmZnkt3J2Vm4TM64w1zTWuCeBBOeWnUPw6GC0HtuadZuOzDwCSb0EUw5OUdmm+/fJ+xQRIZGSfO7uHFhaUqiro5GTI4GPj+a6N8f5qaJCApqmYGUlxsidY+Fh64GfR/0MOwtFJtBUbbK2pkDTFMoepCLrwj30fL+nVDqqucy5CQcSUPy0GH0/6AuuJVep/PffcxEVRYOiADc3GoWFFB4+pNGli/q6N3WbGKh6TuXlFKqsSLRfgFMAKJpSmKMNaZOnjSfmdZqnc5u6v90dEomk0VzJgYsLMRDk50vg4EDGFWbtYqr3qcc7PSAWixXqYqznVFVPPDQq6iqw6uIq/HbvN+ydtBd9/PtI2yQQAGfPckDTwMSJZO/A59OIiKAQFUXh2jUaM2YYp+9JJBzk5NCo5eWivL4EHIqDUFcS7q9P33vvPWDOHEIuV1dz4eNDAyBjak19DZJKkmBvZY9Ap0Cl5yQUC0EBcBcWAA9+g4SyABzba22Tnx8ZY1JSaEgkpD41FTUoelwEn64+ej0nU43lpSmlyLyWicBhgbBvYa/xOSUVk3e0rRuRPTflGDEiaAScrZzhbOWMtJI0BDgHwNrCGnwJX+W6o7mMe9/e/BYJhQlYO3QtPGw9FOqo7TlxrbhYmLAQ9j72sHa0Vhp/zNUmpyAnzIuaBytnK4jFYoP7XpWwCraWts3qOak6XlJTgj/u/4EnRU+w/SVFZ+vGdX+Y/xAdPDuAy+WiSliFLt5dkFKagl1xu5AjyMHFORfN1qaakhrk3s9F8PBglW1lntMrh19BQVwBLB0tIRaLwbfnK/UxVW1t2bclPqz4EKBh9DY9yn8EQObcoamtErGE1FdEyvzXbSy6tsnLy8tk64jmuNfQ1KaSErI/dXCgQdOUxjaVV1RIo/vmAfgcwGsgBOCrAI5BUfZTE2bX1ODHXbvw0apV/4q+V11SjW0DtiF4dDBe/vvlJm8TTdOIXBWJ+up6RMyOUNtWVW0qLia/gbOzbM9tjjHCwYE4pZWWKtrO1NXd3O+Tha0FnPydYO1hrXCOw+FI62lOPDfEX2OUN4RHuDRoL0ZHR6O+vh5Dhw6VlmnTpg38/Pxw69Yt9OzZE7du3UJ4eDg8PWW5qkaMGIGFCxciPj4enTp1wq1btxSuwZRZunQpAEAoFCI6OhofffSR9DyHw8HQoUNx69YttfX9+uuv8emnnyodf/ToEezt7aVt8fPzQ1ZWFkrkksh5eXnBy8sLaWlpEAhkkgItW7aEq6srkpKSUCunUxQYGAg3GzcUVRfhSswVtHYgsj+hoaGwtLREXEMyJD6fB4GgJZ48cUBpaS0yMmTSqVwuF+Hh4RAIBEhJSZEet7KyQps2bVBaWopMuUQ59vb2CAoKQkFBgUIYtjHb5ODggISEBIUXh2nTwiMLIaZFcCkagVD3NpBIJBAKhQiuD0awbzBQCCSUJBjcprIyW9TW+qK42JpVm3658wuu5l2Fg5UDLEstcecOBwIB6X82NjaoqrJAdrbqNjHPiUF4eDiEQqGCxC2b55SfkQ8OOJDQEkQ/iUav8F4GPycvVy/8EfQH/Mb4odMXnVg9J46Eg6u7rsLO3w7WntYGtak59T1jPSf5Nq19vBZ7nu7B70N/R3fb7qzblJKSgtzcXNjb24OiKKO2iWPLgcskFxShCEVxRc3iOf3yizPK6qxQVmeNtLRKBAbaG/050TQNa2trvdqUk52DEPsQRBdH4/i94/Dqrr5N5eWuqKqqQnFxPuLiBHo/J3VtsrMTQCCgIRAAsbGpsLExz3M6svQI4n+Jx+Ajg2Hfyl7/Nj1NBOyBsvwycIsUn9Plyw4QCFwRGCgE4C5tE4fjDIHACTExQowb586qTWmCNHxx7QtYci0R/Uo0RDWySE1jvE+uVcQR537ufdyNuQtrnrXCc2LaFB9P2lRTUwfAw+TPycamCAIBH9evl8DColyhTSKRCAKBAPb29mjTpo3Rx73em3vDzs8OaWlprNtUkF4ACw7JHdy4TXy+H3JyXFBTUwkOJxVxcRLpc/L1dcCDBwJcvpyH8PBqtc/J0DaZ6jnl5JRDIOAgKSsBZ5OJEsXmcZuNMkaUlpfifOx55NXkYYDXAJVtKi52QnW1OyyG9MXMtSFIL0kHVUYZ1CZjz7l3N99F7qVc9F/dX+k5CQS2iI5uDZGoHgsWJCM/3wK7d7vj5k0ac+c6PZfriKwsf1TxCangZeElvVZzXBs5OUmQmlqN27fzUVNThaqqKvTq1QtVVVXP5Xrvg94f4I3wN7Dxykb8kfgHsgRZGL5zOObbnUbm9V4IDCTpJoqK7OHvL0ZEhAtKSkibvLzsIBC44+xZGjNmGKfvVVe7QiCoQoU72Y/62fpBVCsCLGGUvldRIUBcHHlOP8b/iO3PtuPtbm9jTY81Ss9JKBbClQuMzv0N1VahSK7uBMesLK1tEotFEAi8EB0tRGkpB66urtgzbQ8yT2di9LXRsLCzaJJ1uaq+l34sHQ/WPMC0Y9NQTpdrfE5xmaROHhaE0NLU96qrJbh50x7+/nUYNaqVzm2qra7F3KC52PZsG/JT8yEqEj0X79NfMX8huTQZAx0GoqtbV72ek22BLThFHFhbWysoMZmrTY+fPga4ACqAzLhMg/peSV0Jxl8YjyG+Q7B3xl6UF5c3i+ekqk3VkmqsubwGYlqMGV4z4GPjo7JNF3MuYmXUSnza/VOsHrkalrWWWNV6Fca5jsPc63NxKe0SHhU8gkOtg1nadHf5XeRezsXy/OXg2nI1PyceUBhXKG1TSXEJTr97GhKRBBGrIrS+TxkZGUZt05UEkjIgxDkEEolE7XO69fAWylzL0GZrG9TYkqj0/7KNRdc2eXh4oKamBhKJREFprrm1qT6uHklXk+A701eqyGWK5xQTYw2BwB11dTWorXXU2CY+ny+N7vMC8A2ALwEIANyBouynNngCKCouRhaLdURz7Xt2dnawzLOEZbAlcnNy4TXcC9YdrZtNm9qtagfbFragKAoVFRWs+15xcRCEwjqUlWUiLo68I+YYIxwcLCEQVCItTYS4uEyjPSejjXtjWqLj7I548uQJMuJk8lOBgYHKQTXmAP0cQiwW02PGjKH79OkjPbZr1y7a0tJSqWy3bt3olStX0jRN02+++SY9fPhwhfNVVVU0APqff/6haZqmW7duTX/11VcKZU6dOkUDoKurq+ns7GwaAH3z5k2FMitWrKC7d++uts61tbV0eXm59F9mZiYNgC4sLKRFIhEtEolosVgsbR9zTP64/DFNxyUSCd1hYwcaa0CfSjylcFwikSiUfeMNMT12rIS+c4ccjz8cT9/84SYtEolomqaVyjPHG9dR1fH72+7Tu8bsokvTSo3SJnXHLyRfoLEGNPU/Lj1gUgIdGam6rcZo08OHInrMGAk9bx6759R9c3caa0AfeXyEFolE9JEj5PtjxkjosWMl9OPHqtukqo6GtMlrnReNNaDvZt7V2Md0eU4X/+8iHbc/jvVzKnhcQK/BGvrcB+eM0iZtx435Pqk7buznxPz75OInNNaAbvdrO1pYL2Tdprq6Ovr+/ft0XV1ds2uTsZ9TbKyIntozlZ4wpJweM0ZCX7limjaN2z2O7rapG3017apebVp6eimNNaAXnVyksU0rVtAN7TDNcxIKJdKxp6TEfM8p8WQifXj2YTo/Pt+gNhUkFtC1VbUqn9PatWJ6zBgJvXOnYh3Pnyfj7cqVmusuf3zDrQ001oAevH2w2jYZ8j7V19fTPut9aKwBHZkcqbbv/f03adMvv6iuu7GfE3O/H34QK7VJflxpTmMEU77xsVOnxPTYsTT9/vtipTquW0fes717n89xb8YM8pxO3o2hsQa067euRhsjYnLINR2+dpDOO43rePQoeae+/rr5zrllWWV0dnS2yue0fTvpGx9/TI5nZ5P2jB8voWtqmm+bNPW92bPFdOCC5TTWgF7yzxKj973k4mT693u/0/vi9rFuU/zBePrHwB/plEspCse//prMQYcPy8YV5pwp3qeaihp6a/+t9I31N0z+nMqry+kh24fQWAOa+4k13W9KlHTOHTNGQl+8qFj30lIRPW4cOZeTY6R1+UUyvvX/YB2NNaAn7p1osr63OWozjTWgh+4YqvI5Ddk+hHb4DPSzU/1o8bNtKtsUlRVF9/2zLz1u9zhpHXNzZe9kXR0pH384nr7yxRW6sqjS5O+TLn2vNLOUTjiSQFfmV2p9Tq1/ai2d9+WP79ghpj/5REwnJpI2FRSI6CVLxNJ+89lnEjo9/fmYnwx9n8bsGkNjDejf7vymV5sEBQI6424Gff/+fVooFDZJm8qyyujq8mqjPKcvrnxBYw3obpu6NavnpO547z9701gD+o97f6hs0+mnp2nLzy1prAH9xtE3lOr+8t6XaawBvfjUYrO1KfGfRPruxrt0TXmNyjbdXH+TPv3uabq6rFplm7YO2Er/3ul3Wlines0Ufziezo3NNclzemnPSzTWgP7x1o8an9OyM8torAG97Mwynd4nTcebW98zZZuEQiEdExMjta001zbtGb+HXoM1dOFT09iWmeMHDpA5eu1asdY2fbpmDT3CyoqmSWAWTQP09wDNA2gngG4H0Dvlzmn6txOgu7Zp81z3vaSzSfRPwT/RggIBLRKJ6KqSqn/F+zRvHll7Pnxo3jEiL4+skyZMkND19U03RujTppKSEhoAXV5eTpsLz2XE3+LFi/Ho0SNcZzKlPwfg8/ng85XzQnG5XHC5ivlU1IV+Ni6n6fjENhPR27c3fBx8lM7L/79zZ+D0aeDBA6B7dy7u/3Ef6VfS0eMdorZMUZTK66uro/zx8tRyJJ9LBi2hjdImVcfFEjGWn1sOAAgsWQj7urZo3VqmOc2Uv597H+tvrceaAWvQ2rW13m3y8CDXLi4GKIoDVV+Rv3ZRNRE9drdxB5fLbfierGxVleG/AaD9OXnbeaO0phQVwgqlNqkqz+aegz4bpLKsuvK2brYY9csoeHf21tgnGRjS99gcN8bvrutxbW3aHL0Z8YXxmNlhJjbc2YD4wngcTTyKV9q9orK8qnsy95C/j7Ha9Gj3I1xcdRHTT06HV4QXqzYZelxdXQ7s56B19H7U2Trjcd83kZgI9O9v3OcklohxNf0qyuvKYWNho1eburXoBgC4n3dfWk7VPcvLyX1dXLhofNoYbbKwoGBnR8acqiounJ21112X4+rqEjImBCFjQliXV3VcLBTjtza/oe3EtphycIr0OPOcUlPJuBoSQinUMTCQHM/MpEDT7NoUmRYJABgWOMxkY8GQgCFIKU0BKOVzTJtqa0nd7ewU26Sp7myOq6tjq1YcUBSQlUUp9D+mPFMvRpPemOMeTdOoyKyApZ0lrF2sDW7Tgwfks2tXjtK71LIlqU92tmHvWVOMezQNVFeT51TFIZEM3vbeOtdd3fH2nu1hzbNGRV0FksuS0catjVIdHRxIv6wSSFCeXg4OjyOV+tSnTaY47tjCEY4tZPKjzHGJBLh0iZQdMYIc9/YG3N2J/OSTJ0DHjs/fOqKyEnAR90e/wdUYGjiU9Z6CbZuuZlzFglMLMLDVQExpP0VreQDg2/NhaWMJSb1E4Tyzhi4pIe8fM56Y6n2qqa5B4eNC+HRT3gupqzub41kVWUgrS5Pma+NyuXCwdsDJGScxcd8k3I/iwr4mHH37Urh3D3BvWYb+/Z0U6ujkRFIuxMYCt24BEyfK6i6oE2D6oenwsffBuuHr4MB3YFXHe/caJD/dHwGVQAfPDnqN2eqOyz+ndh4kUfzjwscqn0eduA4VEuBBq3cRFDAeqM0HbGRSncw9rmdeh6etp/T/np6QrlVycii0agWEvRwGvGxY3dm0SR5s+piTrxOcfJ1Ulmt8z3DPcFhwLRDsGiw9/vgxsH8/KRcTA4wYAURFcVFURH6Dmhrg7l3yXD/5hIuuKlJVGtImoRDgcJrP/inUNRSnkk7haclTtWsjTXW8sOICYrfHYvS10eBwODo9V2O16dRbp5ASmYLV1aulUTfqymtqk0giwh/RfwAA3un+jsa6N5f5aVjgMNzMvInItEi81fUthXM3s25i0oFJEIqFmBw2GX+M+0Op7ou7LcaRJ0ewI3YHvh7yNez59iave8ioEJXHAbI2TfonCWWpZRj6req5deqRqbByspKOs/JtEteLcXjaYQQODcSMUzOM/pz2Tt6LpOIkuNu6q+1LAGDJtYRziTN4J3goCyyDa2vX/6SNxdDjjW0r+tRR1+O6tGnyvsmoyKqAS7ALq/L61rGigqzjXFwoJXtrY7w1fz6+/Pxz3IEsj997AKYDeB3AWZA8f6+q/LYidlhbY8Krr0rb8jz2PWGFEDWVIuQk1yHE3Q42zjYay5uj7o2PCyuFSL6cjJCxIazaRNOMXZyCp6fy3tqUbXJykuUOFwq5sLHRXN6c415NaQ0iP4pE2CthCBwSqLK8uWF+cVED8fbbb+PkyZO4dOkSfH19pce9vLwgFApR1ii7Y35+Pry8vKRl8vPzlc4z5zSVcXBwgLW1Ndzc3MDlclWWYa7RHPC/gf/DxrEb0dGro8ZynRpUGmNiyOfQb4firei3FBYw+mLgmoH4uO5jOLVyMvha6vB71O+IzY+Fo6UTgjL/BwsLSPP2yOOTS59gd9xu/HD7B4Pu5+oK8HhAfT3QqAuoREEVMc6525Kky0zyUwaVlQZVhzWuzb2GmtU1GBI4xDw3VAEbNxt0X9wdLXu1bLI6NGdIaAnW3lyLDXc2IDonGkt7LgUAfHb1M0hoieYvmwl8ez6sXa0hFAibtB6JiUBsjATZ7Ucg5LXe0mPGxqOCRyivK4edpR0ivCL0ukZXH2KpeZD3ACKJSG25BvVqOMrZ0Iuri7E5ejMyyjNUf0lHODTYDhtSzD43ENWJ0HtFb7SZ0EbpXG0twKgzBAUpnvP1JQtCgQCQU2RQi3pxPS6lEmZgaOBQLaX1x/YJ23H99esYHDBYbRlmbrC1NVk1FODvTz4zMsgi3px4duYZfvT/EY/2PWJVPv5APKJ+j4KoTvX7lJBAPjt2VD4n387nDXV1ZHMDAOWSXACAj72KBY+e4HF46OzdGQBwN/uuyjJ2DakEqwqr8VPgT7j+TfNzwCvPLEddRZ3S8ZgYsgaztwd6NFggKAoIDyd/N1LAeS4gFJJ/XuXj8duY3zAudJzR79HJi2wUYnJjlHJgqEPQ8CAsjFuIoGGKg7KbG/lsvBY2Few87bCiYAWGrxtu1Ov+fOdn9NvaD4tOLVI4bsWzwvZRR9A5eT84tCVWrAA+/+0pzrTqiBqx8oK/N1m+4OZNxePbY7fjVNIpbL6/GV03dUV8QbzWOolEQHQ0+bvMkoyl7T3a6944lmjrTnLVZQuyUVGnvKgQisk60ZJrCUQOBk53VCrjZUf2zYXVhRBLiOwRRQGtWpHzqanGr7cxIapVv6ZrjENTDiF+UTz8HP0AkHl2e0MqNHd38v8zZ8i70aIF8MMPwK+/knmMpoGDB41b96QkYNo0YONG417XEDDOJk+Kn+j3/ZfbYMCaAYCZ1zDyCB4VjO5vd1cg/fTBicQTyKzIhLuNu5LzZ3PFsMBhAIDIlEiFfevppNMY/vdwVNdXY2TwSOyauAtcjrLxdXDAYIS6hqJbi25S+4m5UFNSA4lYca9NURRmnZuFOZfngMdXHSth7Wyt3mZGA5P2TELP93oau7oAyHwT7hkuHUfVgcvhwivPCza/2iA3OtckdXmBpkNVYRVoCQ2eFU9K+pWmlCL/IQtDpR5QZa9QBy8vL3zz7bcYx+fjjvxxAP8AiAFwHVA4pwp3AFwWCvHGvHl61Lj5gG4bhmtdlmLF1674+WegwLzDHCucX3kee8btQeHjQlblBQJiFwcAF2XO2aTg8wErK/J3IwqoyVEYX4joP6KR9yBPe2Ez4bkh/miaxttvv40jR47g4sWLCAgIUDjfpUsXWFhYIDIyUnosMTERGRkZ6NWrFwCgV69eiIuLU9B8P3/+PBwcHBAWFiYtI38NpgxzDUtLS3Tp0kWhjEQiQWRkpLSMLjAGwWYIOnRAg5c/UFgIeEV4wT3M3eAFKwOKokzWRpFEhPW31gMA5gV+CUuRG/z9oeRpAADv934fALD1wVYUVrEbyFSBxwP8yJ4NaWmaywrqBKgUko2+tx3xyi8slF0HIB6t5oCtpa3Rn0Ps37H4o/MfKHnGwqL+AloRmRKJpJIk2Fva49UOr+LdHu/Cge+ARwWPcPjxYVbXYPL6meqdazOhDRY8WAC/vn4muT5b7NsH0BwuIuZEYOwHxOs8OZkYQI2JaxnXAAC9W/YGj6NfgHywSzDsLe1ha2GLzPJMlWVEItlYwCyka+prMHzncLx18i0EbAjA5P2TcSdL29JYM5qK+Iv6Iwpb+2+FuF4/PXO+PR/Dvh2GDjM7KJ1LTSVGMRcX5QWnpaWM6ElK0n6fu9l3IRAK4GLtIjV2mwJs3k+mPzBEi6nh7U3mzpoaZZLU1OOKdydvdH+nOzzaebAqH/VbFM6vOA+uheqoOIY0dXVV/m7LBr+TzEzzE5yGgmkXhwMU1uYAkK0tjIXuLUhOWW3EXyVti94reiN4ZLBR728M/Nr2VxyYckDp+Pnz5HPgQMDCQnacIf4eseOdmxWYVBJcLmBtbZp7hLmHwZJrifK6cqSWGcbEuBMfOBQWmn5cMRVqRbX4M+ZPAMDI4JHK56sswaWtYG9Pnssf939Fenk6frz9o1LZnj3JHiwxUUaG0jSNzfc3AyCkWX5VPvg8ZcWYxoiPB6qryRrio4HLsLzXcqnjkSngZOUkNTgnFil7XgnFQnhxgS7pPwEUBwiYBTRyYnO3cQeHIjnIC6tlezOG+EtPl5U9NvcYtg3cZuxmGIRf2vyid52iosgzs7QEvvsO+PxzICAA6NoVWLuWzMm+vsDSpaSPxMcDuUa02R85Qox1p08DcilvmhRS4q9IT+LvpTbo/3/9ERgW2GTjSve3uxvF0eCXe78AAOZ1ngcrnpXB1zMHurfoDntLexTXFCMml3iU73q4C+P3jkeNqAYjg0fi0JRDxBlABSiKQtRbUYicHYkglyCVZUyBqN+jsM5zHU4vOQ2AECk3190ETdOgOJSSqkFjlDwrwfE3jyP7XjYAoL66Hg93PQTHgoO2E9sicGjTRntwKS7S/dNR8lUJWg1q1aR1eR7RnNcqNE1j/6T92NJzC8RCsseur6nHX33+wskFJ01yT4b4c3JiV/695cvx0VdfoT+XixFWVtgF4AKAnQDet7ICKAqjuVy15N8dAOP4fHzz3XfNKshGH0RFAaAoSCTAuXPA/PkylZrmgo5zO2LUL6Ng68HO85hZuzo6yuzb5gTTD5l+2VzQsk9LLMteho6vdVR5vinGk+eG+Fu8eDF27tyJ3bt3w97eHnl5ecjLy0NNDUlS6+joiDfeeAPLli3DpUuXEB0djblz56JXr17o2ZN42gwfPhxhYWGYNWsWYmNjcfbsWXz88cdYvHixVIZzwYIFSElJwcqVK/HkyRP89ttv2L9/P9577z1pXZYtW4bNmzdj+/btePz4MRYuXIiqqirMnTtX53apCxU1FDRNo6y2DFkVWRrL2doCoaHkb2bgqRPUoTLP8FC0tCtpSL1oOndNHoeHO/PuYM2ANYionw+AbJpUYYD/AHTx7oJaUS02Rhnm3sgYkrURf9kCsgB04DtI5SoY4o8xPpor4s8UENWKUFNSg5qSGlbl7/x8BxvDN6I4qdjENXs+sfbKbwCAORFzYGdpB2drZ6m8y7YH21hdg8PhwNXV1WTjSnNARQVZOFFiEV55BfD0JIsNkcj4xguG+Ovn10/va3AoDp4teYbCFYUIcFY9QDFEHEWRaBSaprHon0W4n3sffC4fElqCQ48PYcrBKcir1N9zqKmIP0G2AIXxhSjPMP6q7Nkz8tk42o9B69bk8+lT7dc6n0KYgSEBQ1R6Ixsb5bXlENQJVJ5jiD9zRfzxeLL+0XjxbOpxxc7LDqN+GgX//v6syk/aOwkzz85U6aBUXw8w+bJVESGenoT0EQqbp6elJjDrBTs7IK+SWICbjPirojDsu2EIHRdq1PsbCpqm0XVhV7R5WTE6uLwcuNNgVRjeyCbbviEo6ulTElVpbty7ByxZwm6MagyBABByS1HnGo0aUbXxKwfAgmuBcA/CjjLGXDZIOJiA698qRoQyEX+FheZZr1RkVSD+QDwqsow36R2IP4DimmK0dGiJMSFjlM4z4ycznvb0JXvQtTfXorhacf3r4gK0aeiqdxteuejcaDzMfwg+l4+4hXE4Mf0Egl20E+xM/+7eHZgRPh3rhq+DvxO7MVVftHUjUX+Pix4rnbs4+yJiX78En6ILgMdAoPN6QgDKgcvhwt2GsMHyaxtVEX8SkQQSkQS0pHl4bNA0jYDBAawc4RqrPdA0sGMH+XvsWPJedOwI/PQT8L//kXUgA1dXmTrPxYvGqXt5OZGXZcBEHjY1GOIvvSwd1fX6jWf/hn1QQmECLqZeBIfiYEHXBU1dHdaw4FpgUABJA3Iu+RwAILcyFyKJCDPCZ+D4tOOwsbDRdAnYWZrJ200OEXMi0H1Jd/RYQqQALq+5jPMrzrO2YQkrhYjZEoNnZ8iG5MG2Bzgy8wiSzyWbrM6RKZGYd3we9j3ap7Usj8NDtW01atvXws7T/L/v847mPKZI6iXw7ekL316+4FqSfauFtQX6ftQXPd/ryVqlQRcwkVVsIv4YvLdsGdKzstB/9Wr82LYtFvn4YEPbthiwejUyc3Lw8XffqSQGB/Os0J/LxUdffYX3li0zelvMBZqmsbXfVjz9k9iVxo8naz+RiDjfNCe06NYC3Rd3h42r5rGaQXHDspZZ35sbTD9sbhF/FEXB3sde7e/YFONJ8xvB1GDjxo0oLy/HwIED4e3tLf23b59swvvhhx8wduxYTJo0Cf3794eXlxcOH5ZFynC5XJw8eRJcLhe9evXCzJkzMXv2bHz22WfSMgEBATh16hTOnz+PiIgIrF+/Hlu2bMGIESOkZaZOnYp169bhk08+QceOHfHgwQOcOXMGnp6eOrdLLNYvAkIbTiWdgvO3zpiwd4LWssyG4v59snj51ulbqdeTITi/4jyOvnbU4OtogrutO/438H9ITyOTnTrij6IoadTfL3d/Qa2oVu97qvJEVYXsCkL8tbBvAYAM7qWlitcwV8Tf1fSrmLB3Aj44/4HRrtnlzS5YmrYULbq3YFWeltAQ1Ylgaava0++/jPtx1biQcQIA8Gob2SZvUttJAICbmTelMkiaIBaL8eTJE5ONKwCQciEFJxeeVCuzZwpkXM+QelLGxgK0RIJOF7/HvY+PgaJkhrMn+jkJqwRN07iWThZoff36GnQtD1sPjZ498oZCiiISxtsebAOH4uDUjFOIWxiHA68cQNq7aVolXTShqYi/fqv7YUXRCrgE6acBcev7W9gzfo9KJ4Pkhn21OuIvpCF9B5uIv9j8WAAyuSJTYv6J+XD+1hm743arPG9u4g+QI3UaOaSYY1zRBXaedmjZW7VkdI1cF1FF/HG5REYNeP7kPuWJP0Zaz5hSn4CM+HuQ9wB1ImUWjOmPlZXNM2KSoigMXzscXecrRjpdv07WYK1by9ZfDLy8iHFdJDKNZLQ27NlDCI7ff9f9NxUIgGL7Szjj1xUDtw00Sf0Amdzn/dz7rL/zYNsDXP7ksoJ8GhPxV1oK1NWZflzJvJmJg1MOIvOm6mh7ffDrvV8BAPO7zFepBMDMr8x8O7X9VER4RqCirgLfXP9GqTyTt+1+w0+75f4WAMCksEkIcQ1Bf//+WutE0zLikJGxNQekxF+hMvHnbO0MD+/+wJRqoN0qtddg1jS5Alk4G+McmZMjK/fy3y/j9euvG02RxlBQFIWX/noJg79QL9nN4MurX8LpGyd8doXYGy5fJs6jtrbA5Mna7zWkIUtDZKRxxt0LF8h45+ND5sT794GHDw2/rqFws3GDi7ULOBQHaWVpOn+/LL0M2wZuw8k1J5tkvZITlYOdI3ca7PS8KXoTAGB86HipNOzzgmGBw2BnaSclbt/v/T6OTTuGv1/+GxZcCy3fliFXkItHBeYJw7ewtsCI9SPgFkos14M+G4TJ+yYjYLAao1IjeHX0wqKERRjwfwMAAJ4dPDHyp5Hw62O6Z3cl/Qr+jPkTkamRWssyjowiscgkRNC/Hc1tDyQPriUXw74bhlEbRikc77GkB9q90s4kUUW6SH3Kw8vLC6s//hj3EhLwNDsb9xISsPrjj+Hl5aVEDL7p7oP37dqirP1qpGVmPdekHwBU5lZCkFuJslyySR07Fpgzh5xrin0HGwgrhRBWapfSamrir7lG/GXdyUJ5pvpKNcV48twQfzRNq/z32muvSctYWVnh119/RUlJCaqqqnD48GGlkGB/f3/8888/qK6uRmFhIdatWwdeo7jUgQMHIiYmBnV1dUhOTla4B4O3334b6enpqKurw507d9DDnDstFpBupCq164IwxF9cHGBpZ4nuS7obRb5p4KcDMXy9cfNqMMivVNStZrxC1RF/ADA5bDL8HP1QWF2Ik0/1D39nG/HX07cnot6MwpbxZBNfXEw2bPJ5CM0V8VdaU4pjicdwOf2yeW6oAj3f7Yl3nr4Dex/lZN3/ZZSUAKt/igNNicGv94Qop530XIRXBE7NOIW0pWmsI5Bqa/Untdkg82Ymon+PRn6sabTjG6OqoAo7hu7A6bdPg6Zp3L9TD9uybNi2D4BbW7LKMAXxl1qWitzKXFhwLNCjhWnHd3lDYVJxEpaeXQoA+GbINxgSOATtPdpjcthkgxfwTUX88fg8g+pe8qwEKRdSYGmv7DTARPwFq5my5Ik/bfvdw1MOI2FRAiaFTdK7rmzhZecFGjSuZ6rOkWbuHH+AeuIPMP24cuv7W9jcfbNWOdiytDIIclRHSQIy4o/PJ5KYqsDIdWtz3mlukJd/3TZhG+o+rsO8zsbNdxHgFABXa1fUS+rxMF/ZEsz0EZoGbnx/B5u6bEJtuWn7hjHASOQxsp7yoChZ1J+58/wVFMicEpKSZFFbbCEQAJXWJKllmHuYkWsnQyfvhjx/eewj/oatHYZF8YsUiBpHR7IGpmmy9jH1uOLbyxeT9k6Cby9f7YVZIDonGney78CCY6H23Wsc8cehOPhqyFcAiHxfYyWWLl3IZ2wsIWPK68rBoTiY10l2/ZvxmZj52zqM+WAvpk4Fjh1TnM8yMkjecQsLQOQRhavpV1FWW2aUNmvCxLYT8c2Qb/BSm5dUF6A4AM8aqEwBrk4EMo8oFWnhQDwxGJUUgERmA0RCStI8UlwbhNSyVJTXlcOCQ4iPE8TPDxMnKkb3qUPPnmQtUFho+BhF08DZs+TvyZOBkQ1qtTt2NL0zB0VRiH4rGtWrq/UazyysLZAfm4+qfDN51TZCeUY50q+kszKWasKHfT/E+uHrsbzXciPVzHx4o9MbKFlZgs8Hfy49Nj50PDgUe5Pj/vj98PneB/NPzjdFFbXCxtUG7aboRpq4t3WX/u3X1w893ukBSzvTOTonFLKf93kcHoKTguH3sh8ebH1gsjr9m2HqtYo+yInKkcp7qkNlvnENjTQti6xiK/XJFvLE4JPMbPQamoAWvh+Dop5veU8AsPexx6DDbyOr9SA4OxOHw+Bgsv8oLpaRZ80F6VfT8Z3bd4jbo33BwdTd2dnElVKD5hjxV19Tj+0Dt+PonKNNXRUFPDfE3wvoBkb+Kb8yX2ukEEOWlZcTw9LIH0ai0+uG5zdqPao12r3STntBHVFaUwq/H/3QdVNXVNRVoKpKJtmlifjjcXgY23osAOBW5i31BbWA8RbPydGcU8zW0hZdfLqgd8veAGQyn+7uMuOZuSL+vO1Jf5D3qDUUNE0j9u9YPD2lhz6VmVAprERsXiwiU7R7xDUFRCLg66+BbNEDAIBDdUeFTT2H4mB069Fw4Ds0TQVVoMtbXbA0fSl8uhk30kQdbD1sMX7LeLy882XQNJD7/R4ER+1Dv69Hoc+KPgBkxJ8xvaaEYiFeCXsFY0LGwNrCsORJBVUFmLB3Ajr90UmltyVDxDk6Aq1dWyNveR4uzLogjVKWB03TkND6WcLUSTmaA8/OPsOTY/oxs2N+G4OPBB8p5XSrq5NFbakj/vz8SA6dqirF6AFVoCgKbd3bwsXa9Nmp+/iRvns9QzXx15QRf+aal+RRmV8JQY5Aq8HuymdX8H2L71FVoLocQ/xpynfG9BVjOgqYA/IRfwDJ/2Xo2NQYFEXh51E/49zMcyoNShYWshwOgsJaVBdXo6aYndy3OVCYUIg94/YorUuYfmGjRrmGIf7Y5vnLEeQYJLvM4OZN8snYF3fu1I3oEAgAgVU8ABMTf166E3/ubd3hEuyiYDylKEW5T1PDsaUj2k9trzVHE1swqQImh02Gp51qlRf5+ZzBqOBR6OfXD7WiWnx6+VOF8oGBpGxtLfD4MbBn0h6kL03HgFYDQNPArl3A/PUnsKtwBW5KNqC6GtiyBfjmG9lYzUT7dewI/By9DgO2DcDm6M1GabMmDAkcgg/6fiCVM5XHB+c/wMp/FqIo4xRQkwtkHwMEyvsFf0d/+Nj7KKxrXFxIJJpIJMs5S0to3P31Lh5se2Cq5uiE5PPJODH/BKv0BSmlRIe+lVMr5OQQkp/DAeTEhDTC0hLo16A4f+GCvjUmiI0ljhA2NuSaU6cSR5nERFk/akq0cmqlNgecNth62OL9ovcROr9pJKjbTmyLVdWr0HpMa4Ou42XnhWW9lhmsNtIUsLaw1imyTxU6enUEQCLM68X1RqjVvw/xhezn/W4+3TCh1wTYDbODo79x5sIXaFpU5lVi+6Dt2PeyeqnXq19exXrv9Sh5VqK2jK6oribzMqB7xJ8u4PNlqTqex/zbqpDwmIKEa4F27cha2MpKFkyij9S/KeHezh2tBrSCvbd2zyRmb+rQRKbKZhnxRwOjfhmFnkuV18ZNiRfEX1NDYhq5PE87T1CgIKbFKKou0ljWyko2eOebJ4jHIBx9chRCsRBCsRAOfAdptJ+Hh3YjaV+/vuji3QUtHVXLhLGBiwsxvEkkQJbmFIoKYIwcbm6yepqN+GsggvMq84wm80BRFP5Z/A/u/MjORf3+lvuI3x9vlHtrw+2s2wj6KQj2X9uj4x8dMfXgVLPcV1ds306Mz3w+B96WwXCo7oTY2KaulWbYednB0c/RrElpO8zsANfWrsjOppDt0xWlfhHo3E/2sgcHEyNKUZEsybChaOPWBvtf2Y8jU5U91HWFI98Rp5JO4UHeA5V5VxtLgzlbO2NI4BCl3/i9M+/Be703LqVe0q8ejor3MyfOvHsGF1bqb7HicJWXK2lpxPvQyYmMy6rA48lkQJvTwpqJIk0rS0N5reJqlabJ5gpoPhF/psbQb4ZiWdYyOPhq3jmEjAtBr/d7qU06zob4C2uwkyQkKEc4lNWWYdnZZWpz3DUlGhN/psL08OkYFjQMtpbKvzFFye7fdsEALE1bCufAJnLzVIHKvEokn0tWigplnLXV9YuICPL5+LH28fHI4yMI2BCAsF/DkFFumF7sjRtApus23OvZGkXee5CeTmRJ2UIgAARmiPjr6NURF2ZdQPwi9ms4mqZRVVillDPcnMSfMSGWiHEnm6x3F3VbpLacKhksiqLw9ZCvAQB/PfgL8QXxcueAzp3J34zcp6+DLyhwsHUrsHcv4FU6ERQolNndxkuzMsHjEdJ44ULg22+B8yQ9Lbp3B+IKiPdYe4/2Rmi1/tgSswU34n6H2/WxQGkMMLUWCFNON/Dr6F+RvSwbb3V5S3qMwyHyu4Csn1AcCte/uo47P+kYFmsiZN3Kwv1N9yGq0b6PTy0jG9UA5wBcvUqOdeqkm+F06FDyeeOGbH2gD86cIZ8DB5L9v7OzjIC8d0//674AAUVRKterL8AerV1aw9nKGbWiWpXKA/91CMVCJBUTqQA28/6ggEFYu3gtlp9bjsAhgaau3guYAXxHPvp82AddF3ZVW8a3hy/CJodpVVLRBcz6xtqaOKSYEu0aYkeed+Iv/Vo6/nn7H8RfIwRsO7mYmNAGH5XmZJ8ASNTzzLMzETI2RGvZxjYsc6M5RvxZ2Fig8xudETq+aZyQ1OHFyqSJweGZZtTkcXhwtyWyA2zkPhlF1Lw8oKqwCjuG7MDVL6/qff/6mnqs81yHcyvO6X0NddifsB8AMKXdFADsZD4ZTA+fjqi3orC051K9709Rsqg/TXKff8X8hfU31+NZCdGia8qIP8YzuV5Sj+Ia48WTTzk4hbWc6+X/Xcat7/WPtGSLOlEdZh+ZLfWwdbV2RRefLtLzuYJc3M66bfJ6aINQKNuA/zTnTSQtSULbnK+Qna0Y8l8rqsWqyFXov7W/ypxL8uBwOAgMDDR5wtjq4mqkRKaY9B4AcP6D8zj86mEpWR0TA5R5h8Fl6jDwrWSkmJWV7J1sjlE8fB4foa5k8meMcvJoLA2mDiW1JcivymeV00EVmkrqEwBGbhiJ8X+O1/l7dRV1eLD9AYqfKo9bTASfn58sYkYVGLlPTQvra+nXMPXgVPxy9xed66gPHK0cpZLcSSWKCQhramSEVFMQfyUVdZhyYApWnl8JkURklnGFrSNB25fbYvha9XMOG+IvKIhsWAUCReedXQ93IeTnEPxw+wcsOb1EOu7Uimp1ym1mKjDEH8emDEN3DMXsI7P1jv41BPJ5/pobAgYHYHXtanR+o7PCcW39wtubRF6JxcSwrg7bH2zH5AOTIRQLUVpbqtKRgy2KioCHSaV46P8m8uqf4XaLGUh324Rdu0g92KCsQoQqKxLq3s7d+AobDKwtrDEkcIhO0dCCbAHWeaxT2kswef5KSkw/rpxfeR7rvNYZRe6Ky+EidkEsLs6+iD4t+6gtp24+7+PXB290egNrh61FsItiiHqXLkCe43GcefAAABn/N20CjjT4HS153Qv9/EnIV6XfIXz7LXF2LC0lRDEjZduhcx2eFpOJLtxTha6tCfCo4BGOPTmGSqHib1wnqkO6CCgO/QDwGABwVEcBqRv7GblPRtEFAKafmI6ZZ2Yapd6Gov//9cf7Be9LJefVoU5UJ8353soxAJcvN3xfe+pGBYSEkPy0QiEQHa1HhUHmvNsN259RcimhGNUMbSkszIHkkmS8dvQ1zD4yW6/vZ93KQtW1KpPvg1ThydEnyLihvzNIcXUxhv09DNsebGuSub25gKIoab5hxtniBWRIKk6CmBbDge+AFvYtmro6/3qYy7aiCyysLdB/dX+NxEzg0EC8sv8VBRlaQ8Gsb4wt86kKuipxNFckn0vGvV/vIeUpcRJSRfw11zx/bKCvUypN0yiIL9BeUAuaY8RfXYVmey2AJhlPms8I9h+FKaNmmCgvNvKOzAYrPx+wdrFGQXwBqov0dykU1YrgGuIKaxfjylAVVxfjQgqJGtGH+DMW2BB/G6M24v3z70t12FURf+YynFlyLeFmQzanxpT7DBoeBM8OquWOGmP6yekY/etoo91bHdbeXIukkiR42Xkhd3kuilYW4exMktDiXPI5+P7gi9lHZjd5guuoKBKF4OEhy9/ROpiMBw/lHBz5XD7+ivkL1zKuad0AURQFBwcHk0fjnVt+Dn8P/RtVhaZlrvPu5yHrTpa0PTENKmOdOyuXNWaev5TSFKMb+hkjXFy+MvHHEHGUbREGbx+MledXquyfg1sNBgBcTL2oVx2akvgLHhEMv766J7ovTCjEsdeO4fGRx0rn2MpLyOf5U4eYvBjsj9+Py2mXda6jvghxJRVjDLUMGIcQCwvTe1TKg5mXigSkg6y9uRYT9k5AdX21yccViViC+P3xeHbmmUHX0SbpCJAoUGa8iG8IvDkQfwAzj8xEYXUh2ri1wacDPwVFUXhS9AStfmyFETtHoLregDALI4Dp70KrbESmRuJU0imd8uawRZ2oDrse7sKnlz9VaXxk+klFcT0e7nqItMtpRq+DIaAoSiGvHMCOEB4wgHxeVePztjl6M1479hoktARDAoagcEWhVMpdH9y6BViKnTGrThZdGtdqPq6Lv0dCArtrpJWnQMKpgyVlDX8nf73rYgrY+9ijy/wuaDWglcJxJuKvqMj06xU7bzu4BLsYLdcSh+JgUMAgjXVWJfXJYMv4LVjWaxn4PL7CcdfADDwInIV9Dt1xJv4mTpwATp4kDi2LFwPjxgGT204GABxIOICQEOC334DPPwfmzCEk0ptvAsX0U4gkIjjyHc1mEB7+93BM2DdButdhIBQLkS0Calq/Dbh2BUpjVeb4UwcPD/Ipr0Tj3dlbbbS3uUFRFGzdbZUkyBsjozwDNGjYWNigqsAD2dlkXu+powIURcnyQeqb5y8piTgV+PjI9rGA7O/09KbP8yehJdgeux0HEw7qRX5F/RaFC0suQFRrGkUleRQVEeKd+c1OvHXCIGWLPY/24ELKBWy4s8Ekc/vzBEYV4wXxpwwmCjLMPYzV/FkprERcdBwOzj+ItCtpJq7dvw/msq2wgVgoxs31N1Ffo5sELi0xzsDORFWZUuaTAZOWIT+fvTNcc8SgzwZh7JnFKLd0h62tTN4TkMmZJiU1v3zGFdkV2D12t1aVBUGDwAqbfMXyOLf8HDa234iUC4YFEjS3iL/S1FJ86/Itrn+rWb6lKcaT//aqohlAnHcFyDlrkmtL87rpGPHH4XKwPGc5Rv4wUu97WztbY+61uej3UT+9r6EKR54cgUgiQkevjlKjqT7EX019DfIr9dc1ZQZtTcQf4+Hp6+ALQCZB6O7eNB7zUiKYRX/QBaJaEasFhXcnb/h0MW1euNTSVHx57UsAwPrh66URNQx6t+wNa541kkqScCvL9NGHmsAYF3v3FQEgv194g4O2/KaeoigMDmBH+IjFYsTFxUFs4hVSxOwIjPpllFaDh6GYdX4WFicsBgDU18t+l04qUpC2bUs+2RpMNWHdzXXosqkLPjivLEulLzp4dAAAPCxQlq1hDIUF3GhcSruEY4nHVC4ImH5wL+eekjwkGzQl8QcQckdXY4xriCumHpmKti+3VTrHEGTavMyYhXVysiw3QWOkl6UDIPmGzAUmCjSxSNHVj2mXJvLKFGB+R6raHTM7zIQVzwqnkk5h0PZBuBp11aTjCsWhcOz1Y7i1Xv24fHnNZWzquklJxlEebAgeQCb3GR8PxOTGYM7ROQCARV0X4eGChxgRTLTPgl2CYWNhg6LqIvwV8xf7BrGAUKibkZVZL9RakFBXZk43NjgUB68dew1rrqyRrmPkIe+4dGTmEdzf3PTRkAwKHxci+VwyhFWKCZjZ9AsmAufRI2XJaEGdAMvOLQMALO2xFOdmnZM6U+kLJrJwYq9OkHwiwQd9yHyT0HI5Lj1lJzWbWkkmPD/rtiY3FMfmxWLFuRX44dYPrMpTHApjfx+LsMmKUmRMxF9+vsTk65Ve7/XC69dfh6WtYcRfnaiONQnBVvaoXlwPsUQMCS3B0kuvQ8StgGN1Fzy70h1bt5Iy8+YBIxu2YpPCJgEAbmbeRHZFNvh8ktNv8mRgxQpg/HhZztgIrwizGRWCXIiWNqOyARBP7noJMUzyuQ0kZ9ynwPVXlFJcZJZnov/W/ui2uZvCcaafyEvCiuvFKEsvQ21ZrZFboTvSrqShLK1MazlG5rOVUytcuUKeSbdu+s3vHcgyUsE5UBc8a/CraZwT2dubOMTU1jZ9yo8A5wBYcCxQI6pBZnmmzt/vsbQH+m7qCxqmZTBTU4F33yVSu/fukT4/YfsEDFgzQO9r7ojdAQCYEzHHWNV8btHDt4H4y3pB/DVGWlkaKFDo6q1e5lEee+L2YNwv4xC/KR75D5+DnD7NDOayrbBBzF8xOP/+eUT9HsWqfEV2Bf7s9SeufX3NKPdXJWVuKsjv7Zsi97yxQFEUsmrdAIpCWBiRMmfg50dUq2prgUzdpzuTwsbVBhnXMrSuc/Ql/gKGBMA5yBmtBrbSq34MmlvEX31VPVqPbg2P9h4ayzXFePKC+GticG7NAu6/axIXuzGtx2BBlwVo7aI9ybR8xB8AJW/p5oJ98SSJ7dR2spxtjAyMD0tOaVP0Jjh+44jl55brXQ9570hVqBfXI68yDwCkXrfyEX/yOf7M5V3pY+8DK54VKuqMZ/W/8tkVfGn9pdbk9qI6EeoE2sOeDUV6eTqcrJwwqNUgTG8/Xem8naUdJocRr+mtMVtNXh91qK6W5dKo9j8M52+dsejUIummvnGeP7bEH2CeiSRgcAC6L+4OKycrk9+La0nIxSdPgLo6MsH7q+BmGEmI5GSZkVcf1NTXYHfcbgDAsKBh+l+oEdhE/GVJiH5TF+8uSmUAoKVjS7R2aQ0JLcHVdN2lmBlDpHxybnMhPy4fX9t9rbPcr7WLNdpMaAPXEFelcwwRok0O08uLbCBEIpmjSGOklzcQf2aMmhkcMBivd3wdXX0UN+/myuXWGPKEzvjQ8YicHQkXaxfcy7mHtQ/XmvTeFEXhlf2vaJSOpiU0aktrNUZ8MO++lZahiZFaiX6Sj5f2voQaUQ1GBI3AhlEbYMGVSdLxODys6L0CAHEIqBfr5mWrDo8eAbNmAevWsf8O0y9quMR5h3HuMjYsuBYIdCZ5YBpHowKy961GZIGpR6ei38fGdfAyBDF/xWDniJ2oyle0ErAh/tzcZP3iWiMbyZ5He1AprESIawi+H/G9lGSjaRo7YnfoLEt36MF53EolE33v3qT/fzP0G3Tmk/XJmcwDrK5jX9MebTPXYaLfQp3urw+SS5Ox7tY67H6026DrMIROcXHTbHz1we9Rv8N7vTe+u/Gd1rJsiL+zz84i8KdA8L/gI+zXMESmRsKSskbH1B04e5oHkQjo25dE+jHwsfeRSoyqc0I48oRE1I1pPYZdw4yAIGdC/CWXJEuPMaRfXyvA+coIIPsUEPoO0Gcv0IiQsbGwwbWMa4jKiUKtSEboMRF/8lKfz848w4ZWG1QqAJgT4noxdgzegXPLtaezsLWwxejWozHAf6B0XNFV5pNBu3Yk8i87Gygp0f37DPHXupFZgMcjBkhA/b7WXOBxeGjtSir4pEh3CQ/vLt5w7eYq3TuYAklJwKpVsnc9JYWM4a1HtUbQsCC9rvmk6Anu5dwDj8PDjPAZRqzt8wlG6jOxOBGlNaVNXJvmhY/6fYSyD8vwcf+PWZXncXjI8s1CzG8xSjLoL8AO+qxVysqIsoMxI7k6z+uMsX+MRffF3VmVt3W3RXlmOcR1xllrMVFV5pD65HJlDjIC9f6ezRqpl1KRfD4Zjx6SThCm6AcHDkc2Hze3PH88Kx7eL3gfI74fobGcvsRfyJgQLHm2BByeYXQU0xcFguYRGerR3gPTj09HyBjt+RHNjRfEXxND0vU3YMBJzQmK9MTb3d/GxrEbpXkhNKEx8VddXI3bG24j9ZIaK6kWFD4uxNUvrqIwoVB7YbbXrCqUEh+MzCdNs4/6YNDKqRXqJfUG5XljiIeSEtWTUV5lHmjQsOBYSHMtqiL+JBJCZpgDR6cdRfWqainxZQx4hHug/bT2WhOZp19JxzcO3+D+FtNGBgxsNRBPFj/B9gnb1Xo7z+04FwAhkZtKuu3OHRLt0aIFkCN5gPK6coglYrRrRxY6BQWKXrcM8Xc76zaqhM3H7clY0hGqUJRYhPj98VLJYUbms1Mn1cOlmxsZxyQSw6L+Dj0+hPK6crRyaiX93Y2BDp6E1X1c9BhCsWI0CmM8SK3VTPwBwJCAIQCgV54/OzvZb2fuqD9HP0e0GtgKjv66uQlqSkrOduynKO1yn1Liz4wRf9PaT8OfL/2JcaHjFI4z7TJnfj+A/I6ltnfwuOYSxBIxerfsjX2TibPNjfwbJpdHbj26tUbp6EGfDcKSZM2bBLYRf23akA3XJdtFyKzIRIhrCPZO3gseh6dU9rWOr8HD1gPp5enYH7+fVVs0IT0d+OILQsDfvcve+Ych/gQgEX8+9qaLoFcnQwsoEsRtXmpj1PwhhqL91PYY+8dY2HkpDgpsCWFG7vPKFcXjVjwrtHJqhfld5iusLTIrMjH32Fz8/fBvqfyWNpTVluHNf2bjWtsusGx7TkqEAcBgL7K2fSJgF/HHLQ9GUP5yTA+dx6q8IWjvQbxr4gviIZaw210nHErAX33/QlGiLIRSJvVp9Coq4faG27j7C7vfUhPOpZxDQVWB1qhKmmbnEZ9cmoysiiyIaTESi0nE98qOa2FXRyxA3t7A228rr3UWdVsES66lyujD8tpyXEq7BACY0GYCu4YZAVLir1RG/DH5qPkUwBGWABIh4DkI8JuslOvPxdpFGhWYI8iRHleV48+jnQd6LO3R5GMOLaYx6pdR6Di3o9ayffz64NSMU1jk/yuKioghsyu7QB0l2NnJ1G30yXvErH8aR/wB7JRszIU2bkSLWx/iDyAOGZrWjrpCIpK9b2lpwMcfk/mPkWLPzCRlDFkjnU8+DwAY1GoQPGw1Rwr8F+Bm44b/6/9/2DFhh4Iz1gsQOPAd4GnHLtUKl8OFmCdGrX0tLGxe/Jbmwu+/A199BezYYbxrcngcdHmrC2vHBq4lF+9lvodBnw0yyv3NGfEHmD81krFx9bOr2PvSXjx+ROYjxkldHox9orkRfwDA4yvvieUhkchsFmyIv9ryWhx/8zjyYklwDE3TyLyZifRr+nsc2duTtTJNP78EsbnwgvhraviMAuwbVuAlMYDIgFAVAyBP/NE0kW88u/Qs4nbrl0gg70EeLv3fJRQ+Nh7xZ8+3x77J+7Cy90qpN3pdnYzdZ0v8MV5kyaXJKKzSr37W1jJvVFWbpGwBkcfysPZG5AUOLl4kRj6AGD34fELwAOabzKx4VkaX/mn7cltM2jMJLsEuGstZu1ojYnaE1rBnY8DRyhEtHVuqPd/Pvx8CnAIgEApw+PFhk9dHFRiv3379gNj8BwCAjl4dYWUlWwDIS/kEOAXAz9EP9ZJ6HE88bt7KqsGpxaew3me9yci/pyee4uDUgyh6QqyDTOJjRg5VFYyRCPrPmD8BEILYmNJpLR1awsfeB528OqGoWtHiySykEwWE+Ovsrd4jU5foz8agqKaT+7RytMKrp19Fh1c76PS9v4f+jV/a/KLyHNuIP0D7wpqR+mzl1Eqn+rFBTg5w7BiR8mADZq5oCuLvmfeXOGg3GN9c/wYA0M+vH6x51igVluJxkemjLER1Iojq9A9HZUv8WVkBQUFAh7RNmOP/PxydehROVk4qy1pbWGNJ9yUAgO9ufmeQca+oCFizRrZRqq1lT4Aw/b2Cboj4M5HUJyAnQ1usnHFeXrEAYC/3bQ74dPVBl7e6KBm32PaLPn3I2iw5mUTVMJgdMRvJS5KxuNtihfJ+jn54JewVAGAVDQYAqyNXo7Q+DzZ1QZjYWVEWblTwKPRNuIeXSi+zuhbbPKfGQJBzEKx4VqgR1UjlC7VBKBCi6EkRKnNlC12G6BQIKNTVmVZd5N4v9xC9Kdqga9SJ6qS5X4cHqY9IBsiepL4hKFjTM1nUbRHyluch+q1onJpxCudnncf/Ri+ChwchEz78UPX4P639NDx75xn+N/B/Succ+A54uOAhfh39q5S4NwdUSX0yzk2RNQA9Phlo+bLsC43GT4qi0MKBKKPISwsz/aSgQPYV50BnjPxhJHx7+hq7GTqBZ8VDt4XdEDKW/e/MrOm7dzcsdy+zBtZV7rOsjMw1FEXmvsZgk7veXGjjSog/fdYc+bH5ONnzJG5/r79zL0CkXBNPJOLoa0dxYIosAvv4cbJGa9uWkPMAmStub7iNb52+Re59/dJpXM0gKh4D/PWXCv234bNBn2FWxCzYWZpZ/uJfBi7FBa+eB8ssS6kz7QuYFjQts0UcPmyYXQIAytLLcPH/LqIgvkB74UYwpu3PnBF/wPNP/I36eRSG/ToB5dUWoCjVTjeMfSJRebvV5BALxYjeFI2HO1UvOOSfi50dUFNag0f7HqG6WDbOlGeUS3NSioVixPwZg7s/E4c8YaUQO4bswMXVutu0GHA4svV2U+f5S7uchh1DdyD7rnKajOaAF8RfE4PDCP1WpQORg4CLQwCxcULAaJpGaU0p0srStJZ1cyMvTn09UFoKOLRwwOzI2Rj2nX5SdyFjQjD/wXwEDNYh8Z4WWPGsMDlsMr4d9q30GDPg8HiETGMDJysntHUj+aIMSRqtaZOUVZEFAKgr8sVPPwE/NKREsbcnBkeKkk1mz7NuNVv4dPHBhO0TTLZZP/n0JPbE7WGVg4VDcfBax9cAAFsfmF/uUyAA7jcEPvbvD8TmE7mvjl4dAajO4UFRFCa2mQgAmHN0joKBRR4cDgehoaGyccWEsPOyg1dHL9SUmsZZoe2ktpi4a6I0Aoh51100cMyMQSQ+Xr97Jpck43LaZVCgpH3EWKAoClnvZeHum3cVInVompBwQm4xsqsI+aSJ+BsUMAjdfLphRNAI1jmH5NHUef50hXcXb/j19VN5Tpdob01SGtX11SisJk4gxpb6vH8feO89YMsWIFJFkGatqBbxBfEK0ce6EJrGhNiiDAUOZwDIIkb4PD56t+wNALiTY9pcK/EH4vGl9Zd4dvqZ0rmnJ5/i8qeXtRot2BI8AJFcsRS7onv1GrR1V84hKY9F3RbBztIOD/Mf4myyfnmZaZpE+hUVAb6+MuehjAx232f6RWl984n4i1wdiS+tv0RFdvMdUGhaRrpry6vl4CDLIds46o9DccDnKS80V/ZZCQDY+2iv1vX23ey72Bi1EQAQnv47enVXvF6grx2cqrsiP0+7oUYkFiOBsw8V1nGwsTWilpQacDlchLkTnSJVktWqEDE7AiuLVirk8LCxIf8oCnBxCTHpemXWhVmYcmiKQde4lXUL1fXV8LT1RLiHBs8jyJx4LC2170k87TzR2bszRrcejaGBQ8HjUfj+e2DTJiAwUPV3OBRHrWMbRVFo694Wi7ot0tYko4JxxJSP+HOyckLau2lIfDsRXE6Dl2P5Y+CwF5DwjdI1mJQIjNMkQIg/iiLqGM/LekUVKoVk4M4jDu4qpep1AbNHiNPRL5eR+fT1VT0/Nifij4kufpD3QOfvOrRwQKtBreDcytmgOlz97CoOTD4AQY4AdRV1qK8mhkvmd3/lFdm6MjsbsPO0g3cXb9j76Kh3BmKvuZZOPEL7++upA/sC/wn8ef9P9P2rL7bc38L6OzwODx4FHujyURfE/BVjwtr9O6GPbaWwULYeoGng++8Ns/elXUrDtS+u6aWkVlVYhatfXsWzs8p7K13RVBF/z2skl0d7D3gPJTkEbG2JvboxQomfJdLT2TsImwscHgcXP76oNk0L81xsbIjT5M11N3Fo2iFkXCcb2yfHnmBDwAbE7yOGOVt3W8y+MBtjfx8LAODb8zF201iM+mmUyuunRKZga/+tWvf+TH9s6jx/teW1yLieAS5fe0SuOWy1Svc0+x1fQDVsWgIdvwHCPwVgHM+M21m34fKdCwZv1y5Xx+PJ5HcYicGAwQGwdmZhPVMBvgMfXhFeen+fLeQNpLo4tPT07QkABsl9Mpukx4+VNYUZr1WLmhawsiIbtdBQYOZMWRnGqGsuL5aY3BhM2DsBbx5/02jXFFYJcWrRKURvNsyj2qA6iIV498y7mHF4BjZFb2L1nVkdZgEArqZflW7KzYVbt0h/CQgA+C4FyBHkgAIlzQHHbOobe4d9PfRrTGk3BR/2/VBqaFEFS0NciXXAgP8bgJlnZsLGVYslVU84BzgjfEY4+A7EesZG/pDJz5SUpJ+ELhPtNzxoOPwcVZNNhkCV111tLck9V25L3qFgl2A4WqlfUbvZuOHum3exdvhavSISm5L4e3qKRHFW5rF/50Z8PwLjt4xXeU4XgowxpObkyCIyGOQKcsHj8ODAd1Ab9aUPTp0CPv1UFsGXrcIBLOL3CLTf2B53s2VydLrKVxsLVwqOgubUw76mHcLc20mPrxu2DkmLk/B6x9dNen/XEFe0ndgW1i7K64b4/fG4suaK1uURQ/xpI3hq6msQFkbCSNg4CjhbO2NeJyKnyBjodEVWFokks7Qk/YIxGrJJ6E7Tsv5eD9KhzBHxpynHX1UV4NnBE+2ntW82EX9/D/8bO4Yo6irJb6S1SX0CQI8e5PPJE6C0phT7Hu1TkmeWR2f7b5SqAAEAAElEQVTvzhgaOBRiWowfbv2gtpxYIsb8k/NBg0aLolkIsRgkle1jwKhvCARAZZVmMi+xIBVRrabhetvusLc3z+/PGOQfFbBzX1eXL5zZb1RUmHa94uTvBNfWyvlhdcG5ZJLHbVjQMK2e8/L5/fRxsnd0BFxZVvde9j18fuVz3W9iZDBSn9kV2dIcfVwOF/5O/gixoIDkrUBVJmDpBNgFAVbKyh+qIv54PMC5gbuRl/s8Mf8Ejr1+zDSNYYnr31zHxg4bUZZeprVswIYAOH3jhIR8MpZ6eRl2bybPX04OyZPJFgzxpyriAJDtaXNyCNnalOjWohsoUKgT1+kcYW/rYYsZp2ag/XQVmmo6YNyWcXj575cx7eg0zDo/CxY2FigsJAQuh0Oeg5cX+bu2FmgxqgPmXJyjJDPNBuV15Qh1C4Uj3xHdWnQzqN7/JogkItzIuIGf7/xscqn55wVX0q/gRuYNZJazWDg2gMvhQmAvQMr4FLTsrV4R6QXUQ1fbCuNk2rIlke4uLCTSn/qiw8wOeCv6Lb1yiNISGpc+viQlXwyBuSP+GPnI5zHiT5ArgFgoltZd3Z7e1ZU4tUskZI/YnEBxKEw/Ph2zzs9SeZ5pG/OcBn8xGH0+6IPgkWSh0WpAK4S+FAqX1jKv/YDBAQopOyJmRcCro+LCKPbvWJxbcQ6CbAFy7uUg606Wxnoy/bGpib82L7XB4seLNaYtaUq8IP6aGBIm4yvFAVovALyHAVzjbIS97MhLlFuZy2rBxGxGGK9EiUiC0pRS1JToHtFTW1aL2vJaoy3UnhQ9wdfXvsb9XMUccdoGU3VgiD9DIv4YQ/K1a8DrrwNbt8rIhtc6voav/KMRkvM/dO0KfPklsG4dMHq07PuN5bJMjer6ahxLPKZXbjB14PF5iP4jGslnNM9UF//vIk4vOW20+8pjU/QmpJSmwMvOS0roaUOAcwBOzTiFvOV5ZpcQYXLV9e4NxOaRaL/Wrq2l9WAkeIqKFD2crHhW2DNpDz4d+Kn0WHV9tcI7JpFIEBcXJxtXnmOIahXl/tjIH3p6EoOiSESMtrqApmlcyyAG/be6vKXbl3WEfH4kxlAo4ZfA1dpVY34/Y6Apib/SlFLEH4hHcZIO1ioN0CUXnqsrIYPEYmUCLsglCLWra/H0beMJ7N+8STZ5EonqXEUMgl3I4lieYGHapY28MjZOpJB8ft4lU6XvGwCEe4SjMqvS5IYXrwgvTDk4Bf79lUMixm8Zj7fuv6XV0YCpt7aIv7dOvoXl8f1QZnMPGRnsvEnf7/0+EhYl4MshX2ovrAJMpEDbtiTar2WD/YVNxJ9QSMY1ADg29RTqPq7DS21e0qsebMBE/KWWpSqRXvIRf+2ntsekPZPg5O9ksrroAhtXG1i7Kj58hgymKHYSez4NgZQFBcDOhzsx7dA0jNw5UuN3VvYmUX9bYraguFr1+PZP0j94kPcANpQTwrLWoUsXZXLI2hqwdxQj1v8NtPrFG7kC9bJx0ZnEiGNf2wY21uzyvRgKJuItroB9uFHy+WQ82qtIFLq7EzL73r1Uk61XaAmN8oxyCKsMYzEY4m94oGaZT0BmdDC19Gp2RTZ6/dkLn1z+BF9f+xrnks9h6sGpOPn0pGlvrAJuNm74buh32Dt5r/LJohvAndeBsoeAtTcw/AYQ9IZSMVURf4By/nkAKIgrQH5sPpoa9dX1Wp1bK4WVKKouQnldOaryySbb00B7kK2tbJ8QF0ccRz79lMiJa4I24s/ZmRjvJBLipNKUCHIOQtmHZYiZH6OzTJ2x9kHOAc5oN6UdLGwspHW4f50sMIKDyfqMx5PZTgz5zZysnHDltSsoWlkEKx4L75T/CMQSMQZtH4QlZ5awlpf+t4OxW/Xw7cH6O1yKC4GDAEnjk9QqqLyAeugzpjD5VNu3B5YtI2u9y5fZK3w0BofHgXdnb1g56T4+2HnaYd6deRjz2xj9bi4Hhvh7keNPO/a9vA8bAjYokWOq0IaoW+Ox6TNq6Azfnr7SvXf8/ngIK2VrasaWxLSNoigM/WaoNDeglZMVph6eCr8+mscdWkLj0d5HSL+WDpqmEbczDnE74xA8KhhL05ciZIxmWXWmPzaV1Gd9Tb1UFcA5wJnVuqUpbLUviL/mBpoGqnO0l2MBb3viDV4rqkV5nXYKvPEGK+VCCn4K+gnxB3T3ELnw4QV86/QtaoqNIwN4PPE4Vl1chTWX1ygc18XwK48eLciC6UHeA73k8gCgWzfgpZfIYFdSQjS8tzYoRzpaOaIisTPsa9shIkL1981N/DH9gS0RzAYcHgfLc5fjlQOvaCz37PQzPDtjuMRAY1QJq/D5VeLx/En/T2Bryb4jjG49Gq42unmC0zTwzz+EVBLUCfDj7R8Rlx+HBw+AtWvZeZowGt5hYTIZmwhPWSextSXeYYCy5w+H4kgnk0phJYb9PQyLTi1CUXUR/oj6A1MOTsGmxE0oqy3TqV36IuqPKJxZesbo160uqsaXNl/i/EqS7J6mZe+JJoM+Remf54+iKFyacwl7Ju2RyhwaG0+Ln6L9b+3R+ufW0mPMoqkdPQ2FKwrx10t/sbpWlbBKr4hlZnHUFMRfp7mdsKpyFfz7sdO6yrqThZMLTiL/oWojny6OHxSlmWjhcrjwtDOeh9apU+RzzBhgwQLytyriL8RFWVJR33nNENzNvovzKUTC0qdkarOToOZacuHdSXuEGxupz9TSVOyJ24PbOTfg4U6Wwc9YTE8tHFpolQTVBIb4YySJGck3NoYApq9zOKRtllxLWBrJSUwVvOy8cGrGKSS+nQgeR1GbxtxrF10wac8kvLJfcT0iHwXKxobM5BYrLAQOJhwEALwUqplkHRo4FJ28OqG6vhq/3vtVZRkmojxI8Dr4Ig907ar6Wj5eXAis41FcW4ATT0+ovWdcbgIAwEXcTq/oMn3ARPyxSSHA4NLHl3D2PUV5XOY3Li1VoXtkJFQXVeNH/x9x7v1zel+jsKpQ6nA4NHCo1vLMvGpqo1gLhxb4uP/HAIBVF1dh5uGZ2B+/H2ef6SdDbAgoisKKPiswpd0UKWmRI8jBinMr8Ev6E2DAKcBVcxSTv6M/Wti3gDVPceCWfxcZvH7jdbwVbVrnLG3o+2FfLHm2RKpIoQ6ppYSscLFyQWUxYYO9jRCozcwhx48Dy5cDUVHAn39qjh5nDNGtW6s+T1GyqL/UJuZYKIqCA19/9jztYBpOv62fsylN03h25plSruGTC0/izqxfwBEJFXKN+/oCXs9u4MrbB1BVaNik2Hiu/a+Dz+NLUx9cz7jexLVpepTWlEr3Ct1bdGf9vUDnQCzosgAT2040VdVeoBHkx9s2bYCOHcn/Y2N1v5awUojsu9kQC8XaC6tBi+4twLMybHypr5c5SWpKu2JMPM8Rf6HjQ9FxbkdW6kRhREUfCQmmr5e+yLyViYNTD+L4G8elx6Sk5uM7ODL7iF7BQgBw95e7ODT9EFIupICiKMw4NQOvXXkNtu62sPXQbgxhIv6aivi79MklbAxnpwLRlHhB/DU3XJ8CnAgCNMgKsYUVz0oqW6bJa5hB44g/j/Ye6LmsJzzDdTeG+vX1Q6c3OoHvyDLxnhYwUWpDAoYoHNc34q+dRzsEOgfiwCsHQMlph9E0jZTSFKSWpqKstgzX0q/hn6R/VF7D0hKYNw/YsUNm2L16lXjl19XJIo4Y6cbGMLcXCyMLxpYIZgtbD1u1ck4M3rz3pkk26r/e+xUFVQUIdA7EvM7zjH79xrh3D9i4Efi//wPmHFiI986+h45/dMTkrYtw4Uax1NivDiUlxIBBUWQx6O/kj1HBo5SSuTPRpCmqU/kBAK6kXcGtzFv4Pfp3eKz1wIJTC3D4yWFsfLIRrX9pbRbyL+VcCqI2RkmT9hoL9TX1CJscBo/2RBJKKCReyIB2MkRf4g8gm+5p7afpJaHJBp62nogvjEdqWSpKa0oBKEYIUBQFGwvtYV6FVYVw+tYJff7qg4o63Ri8poz4s7SzhIWNBevy2XezEf1HtEptd4lEFt3FdvzXhWgxBIWFMpJn4kTNEX+qcqmZW+pTJBFhwckFoEEjUDAbdnUhSvPSxZyLmHRgEnbE7lB9ESPhzk9k88CApmlc/eIqip+yixLVRvxFpkRi+M7hENNijAgagQgPEmGbr2MASX5lPupE7PWEaVqZ+GOI6MxMcl4T9JU11xcURWF069EIdglWGg/l8xPX19Tjn7f/wb2N90xfKT3B9Ak2Mp+ATIaySlSBm1k3AQBjQ8Zq/A5FUfigzwcYGTwS/fz6qSzTy7cX/OwD4JL2BrhcmSGoMTw9Aa8yQjQefXJU7T0TiolTnjvCNNbNmOjv3x+Z72Xizjz2ahlDvxuKyfsnKxxjfmNTEn8cHgc93+uJwKHqpdG1QUyLsaL3CkxtN1XqPKcJ5or4A4A1A9fg51E/g0NxpDlqX277sulvzAJ5lXlYd2sdvn2wC2gxWibvmfQHkPCdUvl3eryDrGVZStHUTB5U+blT1wiwpkR6Ocnb7GNLFh+2tsaZ15l9ZVISGd8sLckcsnOn6vIlJeQfRUFJXlgeDPGXnm54HY0FfRxVC+8VIvr3aL0M5bn3c7Fr1C5ErlJUx/Hu7I1Kj0BwRXUKxF+LFoBVZRHKH2XqFY0DwGzOms8jBrUaBABGVSt6XsGkBAhyDoKbjRvr70V4ReDr9l/DfbU7Hmx7YKLavQADiURG/IU0BCrpm5sVANKvpWNLjy2I+iNK/zqJJch/mG8QMVFKzBawsDDf/pSx+TyPOf76reqHwV8MZmWrZoi/x4+17wdNhaQkYMYM4OhR1ed9e/pi5IaRGPHjCOkx5rlY5mUh7XKa3uRy93e6Y1HCInR5k+zJOTwOXENkwRnx++OxbeA2JYccBk0d8efY0hGOfo5w8DXD4t8AvCD+mhu8RwBB8wCxcVypGbInt1I78dc44s/B1wEj1o/QSw+8w8wOGL9lPLgWhssP1YnqpDl1Gnvc6hsZwePwELcwDgNbDQRFUagSVmFz9GZ03tQZQT8FIfCnQDh/64z+2/pj3vF5ENQJQNM08irzlK/FA0aNIjIpAgEQHQ28d/RzJLquhZ1HoVrvTnnjmTlgbWENRz4ZGdkQwWxRkV2BtMtpkIjVR05SFAW+vXFIYAaCOgG+u0EMCP8b8D9YcNkTCgAgoSX47MpnGLR9EEpqSlh958oV8lkpFOBSYpT0OskuG3EpPAQXYjRLBTLRfv7+xDA9pd0U/PPqP1jcfbFCOYb406T1PSZkDHZN3AUuxQUNGhGeEVjVdxWC7IMwPHC4UXOVqcPIn0ZiZclKWFjr9ttrg2NLR7yy/xVEzCaRkMw7QlHajbcM8ZeYyC5PiUgiwq93f5XmpjElHK0c4e9IDECMVFpFBSCm6mDrwN7Zw93WHS0dWkJCS3ArU3XCZXVgDJJNpYOeH5eP1Ivs3Mm7L+6O9wveR8s+ynOQ/LjJdvz3a1CaaEz8fXr5U0w9OBVX0q6wu5AWXLpEFu7h4cRwyUQtVFUpj/eaiD9t7YpMiUTLH1ril7u/GFTfkpoSWFtYw8nKCf1q1gJQdkhJFiTjWOIxnErS4t1gILLvZCN+X7xUMiP7TjYu/d8l3P3lrpZvEqgj/qrrqzHn6BwM/XsonpU8g4+9D9YNX6cyokQblp1dhpY/tMSBhAOsv5OVRd45S0tZxIWPD4ngq64mRllNYJ6H2PEZhuwYgkWnFrGvsJEhn5+YZ8XD/S338ewf40f06wphlRBXPr+iNL4wOf60yb8ysLAg3szFDpcgkogQ5ByEIBfteVVG+0/Fq/RphNsPUnn+g74f4OfgZNjXhqFdO/VSvl5egGcD8ReZGil1EpEHTdO4X0giIPwsOrFrmBFgY2EDXwdfnciXVgNaKUV5M17cNTWm24Zau1hjxPcjEDZJd2K0oKoAqaWp8LLzwrfDvlUtY6kC5or4Y/B297dxdOpR2FjYwN/RXy3pbGrkCnJxIvEELqddBgCpPLBSVHLKNiDxR9bXZYg/eceM8oxyPNr7SKdcwcaEuF6My59eRtrlNK1lmZyFzlyyhjE0vx+DsDAyTgEkhcS6dWR9fPOmzOAsD2Yv0bKl5jU04xyVlmacehqCRwWP0H9rf/Tbqnuf7vBhB6wsXwmupe42CEc/Rwz9bqh0/8Gg5Utd8LjdZIht7KX5xAES8ZfW8SWI5i/Wy+aRI8iBy7cu6Ph7R9SLjetE+W8AY/e5kHLhP5/nTx+ZTwZioRj5cfmozH8OQ6eeM2Rnk3WnlZXMwY9xFnj0SHdixyXYBQP+NwBBw3XP78cg/2E+fo/4HdGbovW+BrNPcXY2jwMi8HxH/DFgQ/wFBpL+UllpeudkdTh0iNixr6sJrqYoCj2W9IC9N3koKRdSUJpJGufy1kS8ee9NnZy7G1/bva27WuKsPKMceQ/y1CpAMWvuprJt9VjSA7MvzgaH27ypteZdu/8AOJxGjyB4HtD1Z8DS2SjXl8o7siB6VOVSaA64lXULNaIaeNl5IcxdcQOvb8QfAIXImtj8WLx18i08yHsAC46FVG6GQ3EwNmQsYvJi4PejH3r/2VvlwpPDAfr3J39fukRj85Mv8bjlSgS1E6idHOWNZ+aCjz1JXpMjMI6cLABc//o6tg/ajqp81QxmTWkNks8lG32x+dOdn1BcU4wQ1xDMCJ+h8/c5FAf74vfhctplVgb/2lrgToOTu52lPXrFxmJk0T/omXgJ9nWhqOeV4GrlFo1eSQzxx2h5qwOTv0NTxB8ATA+fjkeLHiFhUQIeLHiALwZ/gcdLHuP3sbIM0tX11Uo5mowFhxYOsLQ1ndwcAyayi41Um48PWZTW18uSa2vC9gfb8fbpt9F/a3+zbCoZ2RpGkquiAsh2/RsbeD5Ye2Mt6+v09ycDztX0qzrdvykj/gDg2NxjChFdmkBxKNi620q14uXBkGN8PnG+YAN1EX/nU85jf/x+5FcZPvnRNHDxIvl7SEOAupWVbFHaOOov1C0UAJBcmgyRhHiysSH+npU8wysHXkFWRRZWRa5SSQ6whYetB67NvYbbb9yGhy2xtMqPYxwOB1O6TwEAXE67bNL3ZPRvo7GqapV089CiRwu8duU19Frei9X31RF/K8+vxI7YHaBA4e1ub+Px4sdo79FeL+LPxdoF9ZJ6/HbvN9bfYbx85Q22FhayfHKaJNoAWZ8Q2aXhYupFXEk3DkmtCY8KHuHzK59jy/0tCseZ9VZtLSAWU1iavhTTjk0zeX20oaqgCpc/uYykfxQt32zkXxvD3R0odCBj9IigEVpKE2zZAuzbR/6pQ3Q0mcDUyXwChBywq20LDzocQrEQ++KVL/i0+CnyatPBkViirU1/VvVrSkjEEoW8vXZ2ZC63s/NS3gc1MTZHb4b3em98cvkTnb/LGB3MRfwBwLjQcUhfmo7YBbE6O8AZC8cSj2H83vFYd3MdAEijoV+3qQT22QBFDbLkvXcCw9lHizLEn/z4nHoxFYemH0L2vWzVXzIxyjPKcWXNFSSeSNRaNquCJH6zk5AchsYi/mxsSF6/Tz4BFi4kUXyDB5NzO1QE5WuT+WTARPw1B+LPke+IaxnXcDvrNqrrlVUf1IHD4aBr/66wstcv+s7W3RZ9VvSBV4Tiw2Lm8NbBNCRVsvq0II8WWYX6ObdeS78GGjQoimqy97c5o3fL3uBz+cgR5OBJkY7J2/9lkBJ/LXQj/urF9YAv8EbGG+j7QV9TVO1fDQ6Hg/DwcNZrFcb2EBRE7IPM31ZWZG+lq5Sya2tXDFwzEG6h7KM8G8Mz3BN9P+qL1qO1TAIawBB/5pL5BGT7ject4i/+QDz2jN+DoidFrGzVXC4QSswBTSL3WVoK3G5YpuWwMBEXJhTi7+F/I283icR2cKBg52m6MNBui7phWdYytOjWQuV5RupTFfFH0zRSLqSwWrPpClpCQyIigS+6qlE0xd6nee22XsDo0Cfir6iIyFUCwOU1l/FX3790NvSdXHgSFz66oNN31CEyRSbz2filMkYuJJFEhPkn5yPYJRjrh69H3vt5qF5djdrVtRB8JMCmcZvQ2bsz8QAuS1W78BzU4OR9/X4JRCCb3n4RPmrv2xR5clo4kAEzW2C8DXPbSW0xcsNI8KxVW99zonKwc8ROPD5s3Iy1vVv2Ro8WPfC/Af/TOy8CIyFyKe2S0rmMDODCBZnE5L17RMLVywtYvRrggQ9e2ii4CQZiScT/AACFDucQE6P+fgzxFxICJBYlqiVgmYg/xmtME9q4tVHIOyWqF8HRilidKuoqMHLnSMw4NANiif7a8JpQ+LgQKRe0MJQ64vKnlxXGD+YdURchIQ+KgtQbN5HFHL8zjmgjTWo7ySwSUtPbTwcAbIvdBpFEhIoKINN1G6pRrFO+Ucaz/1rGNZ3u39TEX79V/TB8/XBWc0regzyUPFMdCqWPHCYT8ZeToxgNyshxMdGYhuDpU/Le8vlA796y46okywDijGFjYQORRCTNB6RtXhPUCfDS3pdQWkvIPoFQgJ/u/GRQvTkUB6FuoWq9KyPcImDNs0ZBVYFJjS9Wjlbg8DgQ1Ypw7v1zqMqvgn9/fzj5O7H6viqSRygW4mH+QwDA8enH8fPon6X5g1QZlrVhXud5sOBY4FbWLel1taGxzCcDpk9qk1Zjnke9DWEIGSceUyIuPw6fXP4E22O3KxyX75dVVYCdp51WuW9zwN7HHvMfzEf3dxRz3xhM/AVrJ/7Ky4nUOwBEJxEy/k4WMdI9zH+IA/EHUF5ZJ+0Hmog/T0+AAoVAwRwAUPr9AeBcMslb51LZD64OZkwGCuDk05N4ed/L+P7W96zK597PxVc2X+HOzzLCh5nLKytNl9w+JTIFe1/aqxNBFJ0TjcX/LIaEliC/UndHEGZeNYfUpzzcbNyk676mQJAz8VZLKSVrQcbZrAjWgOcgwMKJFLQPAmyVI/gltAQDtw1E0E9BKKoukh5XNW+2GtQKk/dPhk9X04+BquDk74QFsQvQdYGGl7gBzF7Lss4XgPGIP4DMJd3kUifOmEGcoB48AFatIikoZs4EVq4EIhtUEoODNV+TcY4qLW26NSIDXwdfeNp6QkyLEZunW2KsqvIq5MbkoiJbt0aIakVq5UEfPgRA0/C4sBtb+20FLaEhyBEgatlu2JZkorCQncpIYzBr+P5+zd+BoylgbWGNvn6ErLqQYhy70vMKT1tPeNp6oqdvT52+dzPzJtzWuqHv1hekn74Q6vByM8QfI/MJkLGZsU3oIvdJ07RRnC05PA6GfDUEfn389L5GUxB/z2vEX8mzEjw7/QxcSy7rIBWmf8THm7ZuqhAZCYgbpj6BQDvR6hrqimFrh4EOJQE5zHMyFSxsLGBppz7QQJPUp6hGhIPTDuLcsnOgJeRdOrv8LDJuGB5amXU7C+u81iHhUDNOziiHF8RfE0MiUbHpTfgWONcb0MEIrA7DAodhYdeFiPCM0FrWyUmWJ4AxgglyBajIqoCwUrfVbNKpJGRcM06s8oVUstBrnN8PMCzijwGPw0PsglgkvZOEZb2WwcWazGh8Hl8aFWhnaYeBrQYCgNqcf4GBRPJDALLRs6h3RdeO6j0Om4T4s28Ba541qoTGu2nAoAD0WNID1s6qrWpubdwwZuMYtBrYyuB7CeoEUkProIBBuPXGLUxrr3+kAUP8XUy9qHBcIgE++wzYsAE40KDmdvUqILBKQHWHH9AuvB7TGm47YADw7thhsOU4w6Y2GHejVEu1iMUyr9s2bYAPLnyAFt+3wO9RvyuVdXYm/2haN89biUSCxMRE6bjyMP8h7mTfwaHHh1RGDRgDh2ccxpHZR4waBRS/Lx5PT8jC9RjDLVuCn4miUZVTTR6FVYXSiLkp7aboWk0p0tMVF02aMD50PFytXZEjyMHZZ2eRXPoMpfY3QIGDmR1msr4nE/F3J/uOTjKlTU38tZ3YFu2ntmdFsh6cdhB7xu9ReY5NsuzGcHYm5WmaSC8CxBOWIeD9nQwn/hgDW+/eikSDKskygBBu7/V8D18O/hJ2lmQi09a2gwkHkVCYAG87b/ww4gcAwI93ftQ532NZbRnW31yPgirZi6Iq96xEIkFachp6+ZKoO0bOzZRIOJSAmD9jkBvDXpaaplWTPJZcS1x+7TIuzLqglKtNn4g/Lzsv6XX+jv2bVb3UEX/yef40gXkeAksyLoa4hGgobRyokqEFiAezfN6NqsIqpF5K1XmdaGzw+Dx4RXgpkcT6EH9wTkG1VQq4sJCuEzTh7FmZw1ykeA2+vv41vr/9PXIFufjk0ieYcnAK3jq0DPX1JL+dr6/6azHkgFPGq+BSXNzOuq1Etr/W8TUscTuBoNwPTb7hboyM8gwcfXKUdc4lh5YOCBoepPBcmHE4J6dM9T7ICChLK0PSP0ms+2VFXQWmHpyKekk9Xm7zMs7OPKvzPZuK+GtqMFK4KaUpkNASKfF3nXYHBp4CHBtkLsS1QGUaIKpR+D6H4iChMAEppSlSeUxANj5XV8v2Sk7+Tmj3Sjup3JQxUFVQhRNvnUB5hnadKA6PA88OnnBt7aq1bIRnBEa3Hg1bAUnwZEzirzE8PEjaCYDMN9nZxCHh8WPZWjhEy7RhZSWrY1NH/VEUhW4tCLN5L4d9DlmJRIKoE1HY3GUz4nbrllArfn88vnX+VsmZkaYbiD+Kgl9PH/j19YOwSgiappEflQV7TmXDeKbT7QDIiL9+/k0j0/s8gJH7vJqhm8LJvw1/vfQXcpfnoptPN+2F5cDlcGEhtIBPtA/yYpVT1ryAZjS2rWiDughrJs/fQ3b+ggCAvJg8rPdaj4e7dPiSBtA0raC+oAuYHH9NEfH3vBF//T7qh1XVq+AU4MTaVs3k+TN3xB9Nkz2MPHK1bL05XA56L++Nal/Syc21D7n1/S2ceOuE0nFVUp91AhKEY2FjgYm7JmLW+VmgOBRKU0px79d7uPyJ4QpGwkoh7L3t4dTKSefvmmrvowmmy6r+AvqjtgCoyQFq8wFr7cnkNWFOxzmY03EOq7IURTyNMzOJcdLbGxj7+1i9omCWpi+FpN7wDl0rqpUaPIYEmob4A8imUxvGtB6Dc8nncCrpFJb3Xq50nqKAgQOB6BPEouwAX7hq2BeaO8cfAPwx9g9sfWmrWSKbGDi2dGTlGasNm6M3Y/E/i+Fh64HM9zJBURT5B/3bMqDVAABAfGE8CqoKpDJ39+7JDPR79gBt2wJXHibjdshQ1JXkwvafx/hj+ib07k0Mt1yuG66NL8QnH3MRW0Im0cY/cXo6iRi0sQE8veulZKO6BXxgIMkXmZKiXRpUHfr69cWHfT7EZ1c/w58xf+oliaoN/T5u2KzSgAGPQgEL4xaiplhmGNIl4g+QGYuKijSXO/rkKCS0BJ29OyPAOUCPmhKsW0eMJJGRxLuakRxQBT6Pj1kdZuHHOz/iz5g/UVpOXLw6Ow6XRuSyQbBLMLzsvJBXmYd72fdYGw0cHIBqyzTcst6CmNxJ6ORtvvxQuqL3+73V9il9Iv4oini0JySQiN7AQCLFJaEl4HP50vdfXwiFsqifIY2mK3URfwDwxeAvpH/TtPaIv7md5oIGjTD3MHTz6YZLaZcwue1kBflqNtj7aC/eP/8+dsXtwv359wFo3mT19O2Ji2kXEZWjf4J5tgifEQ6+Ax+lKaWgaZrVnFVXJ8ud0Xis4FAclWsIZqwoLiYOH2yVN2ZHzMaRJ0ew+9FufDP0G3A56nP7qMrvx0Bd3snGYJ5HGaeB+HM1PfHX2pVUtqCqAGW1ZQp5Yx0cSD8VCID0PQ9wYeUFvH7jdb1yQhsL9dX1EAvF4DvwFSIQ9SH+wrwDMPD8Y3hFxMGer3lHKxIBp+RSX7bKX4IMtz+xP34/9sfvl11TPANRIJt7Td3ZzY3I/qDWCzPD5sHP1UOan5mBPd8eAfVjkSIw34abQbgHYa/j8tkZ1W3dbTH9xHSFY8w4U11teB5wdej8Rmd0er0TWZtoAU3TWHhqIZJLk+Hn6Ic/x/+p1zqZMTr814i/lg4twaW4qBPXIUeQoz7HX9JG4P4yYNh1wL2PwqkWDi1QWF2IrIosRHgRZ1VGJru8nKzJGTUMAKznBTa4/eNt3N98Hy16tEDnNzprLFv0pAiOfo6sctm82/NdvNvzXSxaBFTAtMQfAMyZQ/buVlZkP29jQ8iotDTyO2qT+gTIniYvj8xbjKG6qdDVuytOPj2pE/EHAHb+dhjw6QD499fNmcvKyQo+3XzgHuaucDw/n+wneDxgwoZB0jyJfHs+eq/ojYKCUOQlkd+MkUtlA0GdQDqOMlFtL6CMV8NfRT+/fujeorv2wv9y6DPm8Tg82FbZov/W/njo8lBJxvYFjIf6epmUZ2NHC8bxLz5e+36jPLMcVo5WEFYK4eDrABtX3fZ3qlCSXIIdQ3ag49yOGPi/gTp/v7iYfDYV8afKttacweR8ZfZv2tbqoaGkTxQWkn/u7prLGwuxsWTOt7UljvNJSWTdoM1RCGDfNmMh80Ymsm5noa6iDnwHOXntigqE3D6KlE4TIRLZIWbTPVz57AreuPUGnAOcETxCJnfgHOiMhQ8XwinAyeA1ZNDwICyMW2jQNcyJFxF/zREdvwFeSjOY9NMHzKYkr8EhSN8XgqIovZJqN4YVzwr57+fj9hu34eeoHJ5uDKlPthjdejQA4p1XXqvaK3TAAKDWgniretlqNuI3RY4/Po9vdNKvIqsCf3T6A9e/UZMN1ghg5JfqJfXgcXhSiTtD4Wbjhg6eZGcrH8Vy8iT5tLYmUVwffJWJ60FDUGeZi/Ye7fH1kK9BUWSDx23o5uHtuLCyImHmqnLzyct83sm+DYFQADcbN7XEC5PnLznZsDa+3ul1UKBwMfUi0srSDLuYCoRNCkPYpDCjSr1xuBzYesheaibHH9v3nG0Uz+EnhwEQmU99IRTKjPZxccDSpdolRt/o/AamtZ+GhV0XIlpMZNzGtXxNp/tSFCWV+2Sb569WVItfH32Oy+3bItHzS4zcORKFVTqEOhkBebF5+NH/RwXZN3XoPK+zWiOcPhF/gDLRwsh8+jn6sXIA0YTTp8mc5OamHNnFSGlri0Ktq5PJC2tq2+udXkdP357gcrg4Nu0YZkXM0lny+K+YvwAAszrMkh7TRPx18iJjVUyeBj1jI4GiKISOC0X3xd1Zz1kMwUNRRGo1ozwDy84u0xjh7uxMNloikWqJEHUIkoyGg4ULcgQ5ShHjjSGf369xPkqmP2ZmykhLVWCeRyFNBhdzEH8OfAepXHzjqD9mk1dRQTY+IzeMhKNf00kNAkDc7jh86/wtks8pTppMv7BSL8CgBA8PCna1beBV/IrWsrdvEwkkJyegY0fAoaYDOtoPA0AkO7v6dMWmsZsgTiNEB+PVqw4cjsxR4P02v+OzQZ9J83XLw9wbbgZM38uqyNI7fzAzttXUcDT2e0NBURSrtUlkaiR2x+0Gl+Ji98TdcLbWL886E/Fnzhx/zQEWXAtpxHxySTLqxMTTug9PAMSsIA6tAODaA2izHOArW7Ra2KtORaBKjnlD4AbsGadaDUAfDPlqCBbELiBEsQbQNI1tA7Zh28BtrK9N0zJHQmYdYCrw+cC4ccCwYYS0Cw4m+ednzwZeeomd0ZSJRs5umhSKCmAi/nR1NrJ0skS/1f3g20NDaLUKhI4PxWuXX4O9j+KgykQ/BgQozyN9P+wLXz+ydmSUJNgisTgRNGh42XnBy+4FGaMOLR1boo9fn/90DsRcQa7e0Slciosq2ypcmn0JHWY1MZv/L0daGtlP2NvL5i4GQUHEGaOqSrWdSB6RH0Zinec6uLdzx1vRbyF4pBadZhZw8neCrbstrBz1y3/alBF/9fX6SSk3BUS1IsRsjUHRE+J9zshmanNUtrKS2f7MGfV35gz5HDSIzHEA++h1Zs1rrn3I6N9GY0nKEkXSD0DcpluwLc0CzeGhspIojdi42oAWqx4zXUNcpcQsG6WHfwteEH/NERzjLWxomkZJTQmelTxjVZ7ZlDCbFGGVEHG745Bxnb1sZ31NPZLPJ6MsrUzH2qqGJdcSPXxVJzI2VsQfGwS7BCPENQQiiQjnU86rLOPlBTi2JLul1p6aib/nNXy9MfiOfNSW1UqTmzbGwWkHsanrJr2vL6gTYNqhaVL5pdR3U6VyrMbA4FaDAchySWZmkhwZFAV89RXg4FSP661GooafDi+L1jg/6zxcbZRDOXk8oEMHGlX8Z7gXpfxbMGRQmzbA2WQSUz8scJhasoGZ/LUtDhuDy1Uk3P2d/KWRLtsebNPtYjpAVKefdERjVBVUIf1aOmpKDY/400T8ldWWSZ/5xLYT9akqAGIckUgISezrSzziPv1U8wK1vUd77Jm0BxyKAwEnEzyRI8YEvaTzvV/v9Dp+GvkTK5nSU09Pof1v7fH59U8g4dSCI+FDJBErGfRNDRtXG1i7WsPSVr1WOxvoE/EHKBN/DBluqMxnVRWwr0FNd+pUZU9OdVKfDDLLM7H30V5pu7hcYsSTx63MWyiuLjaongzi8uNwL+ceeByegsSsunmJy+Wio2dHWPGsYGdpZ1RpX2NBnuChKGDfo3344fYPmHtsrtrvcLmQRuazlfuMiQE+WW0J5+ypAIC/H2qW+2TyvrZrL8G6m+vw9bWvpb+fjw+pa2WlZuKxshKgIUaBiKzlQt1C2VXWQKiT+5SXDPaK8EKPJT3g4Nu0YU4uwS7o+HpHOAU4KRzXJ+JPU4RuY5xoUJ0ZOZKoAwDAJOzFPzP+QcGKAtx78x7e6PSmdA3AlNEExglP1Xix8d5GrIpcheQKYhkwN8nkbusOS64laNBq8xQ3RtzuOByYckC6TmCIP5qmUFdnmnpm3spESiS7BdTJp8Tba07EHPTx66OltGqIRLJx878W8QcAgc4kHC+1LBWjW4/Go4WP8GFIX+DxOkBYRgq59wY6rwMclB0XpMRfhSLjxKzn5N9F787ecGvrZnCd5fMneXbw1OpkQotpdF/SXStBCABiiRgVdRUoLSXrQYoynwe/IWjRsHVtFsRfgyJKYlGiTlLmjfdBhoIx3Koba/UlSxlFo1BX88znL/B8QiwRI+y3MLT8oaU0F7gu4HF4qLesR1JE0otoPz3BdkxhZD5DQpQdLTgc9nn+gkYEodO8TkaJ9JPen8fBm/feRM+luuWIZMDk+HPWzy9KL1hby/bTz4u9tDipGMdfPy6VZ9XFXmFuuc+CAuK8CAAjRhDFAIA98WduB0Q7Tzvw+DwIcgXYPWY3Hu17BAAY+s1Q5PSdArGFFQQCIHRcKOY/mA+XYM324ru/3sVPQT8h86aWfBsqcOXzKzj2+rEmT3OhC14Qf00MtRNJ/mXg4SdKeRB0RWJxIly/c0XXTeykFhnij4n4E9eJcfjVw4j5i72Hf3lGOXYO34nozdG6VlcBiUWJWr2JzRnxBxC5T0B9nj8ACIwgK/9OQZo9DZsix19WRRYm7J2AETtHGO2afHs+3k19F/0/Vp2YnO/Ah5WTft5F55PP49XDr+JZyTO0dGiJLeO3GD1isafHMFjQtshKtVWQ7OrenXjLtpm0H5XWCbCsd8PxyRfUemXSNI2/+BG4FN4a5x4or+ieNKToCQ0FziWfAwCMCFL/HBhJI8Z7jA24XC7Cw8OVxpXXO74OANj6YCskRsgd2hibumzC30O157pig5TIFGzrv00hxwYT8ceW+HNrsAdVVgK1atLfXUu/hnpJPcLcw9DGTU8tVcgIpFatgPXriWFAIACesfC12Ba7DQDQomQ63J11f0dGBo/EOz3ekUryqcPcY3Mxds9YJJcmw9vOG/0KdqNfQgxOjn6kt5FTXzj4OmD+/flaDWeZtzLxe8ff8eTYE5Xn9XX68G/g95jnVlZbBh6HB39Hw4i//fvJc2/ZknjbN4aqqAUGeZV5aLWhFV49/CpSCslq29ZWcdNYXV+NSfsnIfCnQNzLVpa9qhJWYcPtDRi7eyyrd3zrg60ASM5Jd1uZNVIV8ceMK0GuQRB8JMDVuVfNKhfNFo0Jnv0JRGZxcMBgjd/TJc9fYiLw5ZdkTPYvnI9XXTZg3fB1astHR5NNFQ0JjooWYMX5FVh1cRV+u/cbACL/yWy0NMl9VlYCQl4RHC3cwefyDe6vbMEYJDVF/DUXtBrYCi/9+RLcQhUJAaZfsJ0/bmbexPLbU5DjvB8VFdBITKWnkw06l0vyazFSejnPXDCq9Si42ZC6ZGSQeczKSjYGaYL8WrxeXI/jicfx3pn3UFFXgc33N+Pr618jpYasy83pfQ0Q2VxfB7K+zSxnt1nOf5iPhIMJUs9aS0vAwoKCg4MDampMI/d56f8uYf+k/doLgjj/vNvjXYNyRjNjJkWZPwqzOaClA5H5zarIggPfAe082sGl81fA+FTATruUOiN1ri7iT54En3JwCoavHW5wnZP+ScKOwTtQnFSM7HvZOL3kNMoz1Xt/c3gc9F/dn1X6gqSSJDh+44iwLeRdcXdXjvhujmCIP12j10wBd1t3dG/RHaNbj0ZpDTulF2a9cuadM/irz1+s7xW7Ixb7J+1X6bSsTWFCX7LU39EfczvOldoWXkA9kkuSsejUIrx29LWmrorZcT/3Pspqy1AprFSpfqUNjBS9WMIiEf0LKEGdbUUVmH2Ej4/q89ry/JWmlKK6uBoRsyMw+ufRetTWdGCIP3OuOeXXU4wDRnOHo58jph6divbT2gPQzV5hTuKPpoFffyXKZh06EDsW02+15fgDyPf0dcI2FFX5VUiJTJHO11xLLjghJCqW+b2ZiD5NCBgUAL++fgoqY9pAS4izWM69HKReTIWFrX4BW8Z2UGKD52AJ+u+GWq/5olvAoy8Al66A73i9r8/INJXXlaO6vlprDiBmg8XkxrJ2scaUQ1Pg3o69m6Ktuy3G/jEWnhH6a5rUi+sxctdIiCVinJxxUirH2BjmjPgDgCntpoCmaUwKUy8N+MPob/Fu74VSg486MBuI6mrz6VZbci1xLPEYKFAQioXK+TdMgHGbxun93c+vfo5rGdfAoTjYNXGXUSP9GFApwzE0pgAciQ2WJ8u8XMaOJe/noby1AICZrZeiW4j6BTdFUQj28ENaehzuFJ2DQBAhXaxUVso2hG5+RYg6RWRrhgWpYAga4OlJ+khVFfu8ETRNQyAQwN7eXsEwP6HNBDhZOSGjPAORKZEa76sPvLt4G03q07uzN0b8MAI+XWWrZl2lPm1sZL9dYSEhYxpjXOg4pC9NV/Iu1xWMsd7fn9w3LAy4dYss2rRJun3Q+yPcOucN75KpBkcIiCVitXnGuvl0w86HO7G0x1J8MuATrF5hj9RagF9v2D1NCWGlEMJKoVqCSV+pT8bonpdHDPoda5diWfU7mN9ZfyebggJZ1M/cuTL5X3kwc6tAQPqzPAnhZeeFnr49cTPzJg4+2QtgmVK7frrzE3Irc9HKqZXK+ZAGjU8uf4KKugqt77hQLJRGqTFOAQxUEX/y44qucqLmhDzxl1ySjKicKHAojlYpX7bEX2YmsGYN6TdubgCKImD7KALuapZVZWXADz+QZ1PZ/22cerZZeo7JLwuQKNScHDKWRESovlZlJcAXeeLYgEx06lajMaegMdHKqRUAIpsqD2a8Yjbi2wdvh5WTFaYenmqWeukCXaU+jz05hiNPD8DPlQ+f0ikoLJRFdDQGI8Xdrh0xhjDDVVYWcTph7vn4MfkMDVU9PjSGvOx+YnEiJu2fBJFEhGOJx5BaRjz+bfPIO25O72sGLR1aIqU0BZkV7Ii/vh/1xcA1A8GzIuMHRQE2NjRKSkSorOTBzc34C+A+H/RBVT47z7r+/v3R31+18xpbMPn97OzY5wr9N+GNTm9geNBwdPHuIjto6UT+MRCWA7dmAZ6DgDbvKXxfndSnfB5WYyPvQR5yonNgYWOBrNtZuPvzXfj19YNjS8PDaJm1pRVFBktT5/czFpixrqCARCpamn6bqBF35mmXhZcHs14R1YpQX10PWkKz2p8UPCpA4olEjP9T2eaizbjJ/GZZWbrt5/v592Odn/u/DjEtxsaojbDkWuKHET/oLcf8POJCygUAxIlNn7Ufl+LCpsoGc9bOwfmC8xj2nXFtAP92qLOtqIK2qH+G+EtIUD1WnF95Hk9PPMXyvOWwdtZBpkIHXPv6GpLPJGPO5TmsnThFIpmjn7mdzezsyPrqeYn4s3K0QpuXiEO5UChTgGJjr2DsRunppL2mtG9fvgzcvw9YWACLFpFjDPHHJuJP/nmYm/jz6uiF1TWrFaQ8mTroQhC7h7ljzqU5CsdomsbtH27j4d8PMe34NOl6UFglxO7Ru9FmYhv0fLcnph+fjtqyWr0doZtCOek/uDVpXpBI1Hjmt10JDLthEOkHkBwt1jwyceQKtNP3jAGhVM6xru3Etkoe1Jpg7WKNLm910VlbXx674nYhrSwNdeI6BLuo1rUWiWTRPOYacHr69sQPI3/QmITbxdoFnb07a/XKkkkdyYgNU8PNxg0WHAvQoFn1B7ZIuZCCG2tvGOVal1Iv4U7WHQjFQnT27oww9zD8NPInk22OHsfzwJUQy21KCulTvr7EAJtUkoSkkiTYWthi7RTtyVvHtSUeyAUO53D0qOx4bCz59PYG7hREggaNcI9w+NircQkDWQwyWtts8/xJJBKkpKQojSvWFtb4YtAX2D1xt0kSyI/bNA5jfx9rlGu5hbqh59KecA6Qbep0lfoE2Bnz/Rz90KtlLz1qKUN6esO1Gl53RsaNMfRqgp91GNpmfQen6i4GRQgcfXIU7Te2x6XUSyrPz+8yH48WPsLa4Wthz7dXkOnLq8zD/vj9OPPsjP4V0BFPjj3B+Q/Oa1z0BA0LwpJnSxA6XrUEkr5OH46OZENG00Sa8/vvgYRHXETd1H8S+ftvkn+gQwegq5pAAGtrmdeiKvnAmeFEbvN46i4AihuEu9l38cXVLwAAnw38DHweX+n7dpZ2mBNBFq+/Rf2msb4nEk+gqLoI3nbeGBGsGHWsivhTNa6YInLYUMgTf/vjZdF+8hGNqqBKSk4Vdu8mv0toKCH0LCzI5ig9Xfn3oGngxx+B3Jp0JIZPw5XqjaBAYfuE7ahdXYv2Hu2lZRvLz6qCfH+3tjCNMUAV5naai8S3E/HbGMU+JT+GAACPz5OSOk2FG2tv4MisI1JvTAbMOpGt1OepJBL2H0qRKAxNcwgThcQY9Z2dCSlM04rzNjMfsJH5lL9eXh6Rhr485zL8HP2kpF+ERydwa4k3gZMTu2saEy0dW8KaZw1BHbsdtpWjlVL/sLGhUV1djcpK04wlQcOC0GGm+fIZ/Vfz+zHo49cH09pPQ2vX1riRcQNrLq/B6bgdQGUawESacK2A3DNAhXIiZD9HP/g6+MLVWlFKn1FwYBxSASD1YipOv3salfmGWQL7r+6Pd1PfhUMLB4SOC8U7Se8g7BX1HlvH5x3HnnF7WBlsGALTAWQv/LwQf46OZP1B0+y8/psbmPXK2M1jMT9mPmunxGHfDcPK4pUqFWq0rTe9vAjZX1Mji4p5AeOitUtrhHuEQygWYsv9LU1dHbPiQioh/oYEDNHr+87WzpjcYTJ4ATzYeZvZQv8vgDrbiipoy+fm50civ6urFec0BmGTw9BzWU+TkX4ASalSnlGOyjz28ydjF+bxzK9o8LylRhLVymS65JUg2NiwnJyIrZCmZbKxpkB5ObC5wRd12jRZ1DqjQFNZqZ1Ak3fAboLgNVAUBQ5PRmUZGhmaejEViSfI2jTjWgY4PI5CCovK3EqUZ5SjvkrmNa+voh2ggQMyIV4Qf80VHC7g3mCQFgsBAbscfY1BURS87clbnFupG/Env6+pr6mHRGyeDiqWiPHVta8AAO/3el9tlKK8RKa5pD6NCUtLmSelueQ+ORRHrZyOIXj490NcWHkBdQJFTazKvEpc/vQysu+xv9eK8yvQ88+eOJRwCD+O/BHxi+KxuPtio9VVHhKJLJz+k09o+HWPBc2tw6uvkkk6xDUEGUszcGjKIVbRhsODCPFXYncNe0/mo6SERIdsJap66NWLRPntnrgbq/ut1no9Js8fW+JPExZ3X4zp4dPNajQ2FnSN+AN0k+8zBPIRf4DMW+vxY8UxVBUYQ6GtrWESUBdSLuBJ0ROsurhKmrNm7Y21KKwijedyuAo5weSN9seeHMPUg1Pxw+0f9K+Ajnh68ilufncTNcX6R9kZIvPMEC0HDsiekTrvNpomkizq8q+JRMC1a+Tv117T7OmtSe7zlXavgMfh4UnFfQisHkvbFV8Qj1G7RqGqvgpDA4diRvgMtddf2JU4JxxPPK4UoSWP9PJ0WHIt8VrH15Qi+LRtsG5k3ED4xnAM2aGfAcKUkCf+9sWThItT22mPQNP0XOTBGEGnTiWbs86dyf9/OXcS7X5rh9tZt6VlIyOB2w/KcbVdezzj7wcFCn+O/xOzI2YrELc0TbPKD9RUcipedl4IcQ1RWoc1Jv5ePf0qJu3WHFlpaqRfScfjI4+VDL265PhLL0tHfGE8OBQHnRwbHHk0EMIM8ecpJ3AR3OCrJi/3bAjxBxBS5cH8B5gcNhkAMNqPSFLa2TVNRM7mcZtRtaoK87vOZ1WeltDIvpeN3PuyfQgzxjW1MedQwiFcSr2EOpFhyQaZiL//Yn6/xriReQOfXvkUlo+/Ao4HAMIGNoTLB6bWAd1/V/rOsKBhyHwvEzsn7lQ4zuRglTeS5t7Pxd2f7qI0mZ38oyYwOZSsnKzgEuyi0Xu7uqgaVQVVrDy8syqIVqZVPdlzPS/EH0U1rzx/DMpqy0x+D769slMVoF1hwsKCnWS3PCS0BAmFCQaPO/8VUBSF93qSKOGf7v6EenEzliwxIqrrq3E94zoAYGjgUL2u4WPvg+0zt+PLuC/R6z3DnF1fQDMY0kEdOcbjyaKqVI0V7ae1x9Cv9XvObDHkyyFYkrIE9t7sGTz5/H7mzvSgTyRXU2Jrv634vSNZ48jv3dj+bkzKgKdPNZczBFu2kN8zIACYOFF2nM+Xrbm0Rf1pI7nNDUMI4vrqehyafgin3z4NiUiCSXsnYdaFWaAoCjRNQyKWwCXYBfNj5qPvR8YPoDAXXhB/zR00DVwaDtx9S+9LMHKfbCK8mPBtoVBmbL/65VV8ZfMVSpLYubE93PkQv4T+gqzb+iUI2Be/D0klSXC1dsXCbuqjrJgX28bGvLI6IokIkSmR0ggMeex8uBNvHHsDF1MvsrpWUxg+GDkdZlNqDPT5oA/m3Z2n5NFdmFCIK2uuIPsOu91jRV0FYvJI3hpzyJ+kpclk9z5OHIXfJB3x2hen0VduTHe1cVWKilGHUNdQhLqGQsKpw6WgHli/8yH27CHGQTc34lXjYu2C6eHTMbW9doM0Q/yZ0uvHWLj0v0s4ueCkwdc58MoB/D1MMV8gMxaxjdgANBN/2x9sx7g947Dz4U7lkzqgrk5mmGXIpKAgYogVCLQvmhijuaGGwtX9VsOaZ43bWbfRfUt3hG8Mx8oLK9Hzz56oFSknOZQ32nf1ISFqUTlRZpMd6P9xfyxJWQIrDXkNHx95jAfbH6itkyEyz8yzklBC3AnvhfsBM5Ceozrs+uxZYPVqYO1a1dfKyyM691ZWMoO/OjDkgHyuIgZuNm4YFTwKAJDtsgu2tkBqaSqG7xyOkpoS9GjRA0emHtEo89PWvS0GtRoECS3BpuhNasst67UMGUszsKzXMqVzzO9ZVaWauHaycsKjgkeIyolqdlF/DMFTZZWI2PxY8Dg8vNzmZa3fY+skwEjNMZuiXg32k1Mph/Gk6AmWnV0m7a/R0YCF2BGDXOZgcMBg3Jl3B3M7zZVeS1AnwLzj8xD4UyBcPUnFNRlZKyuB+wGvYlbkINzIME50vSFojjk3pp+YjhWFK5SO60L8Mfmbe7fsjVYeZEHMJuJPnvhrvGkvLSXjBEWRaFE2YEiCsjJZ/Z2tnbF/8n5kL8vGlBaknU0h8wkAVjwrneVttvbbiosfy9bGpsxzXSeowzrPdYhcFamxHE3TWHp2KQbvGIwr6VcMuud/PeKvVlSL44nH8UfUH9L87E95fkDb9wGeHGOiY79hIv5KSoizHgB0nNsR72W+hxbdW+hd38QTibj+7XXUlsnWSLXltUi5kAJxvepcWNOOTsMbt99gdX1G6pNX/XxF/AHNi/jLFeTCc50nvNd765SjrKa0Bvd+u4e0K2lay6ZEpiBmawyElUKV59k43jDpGNK03w4Akc9u91s7OH/r/CL3GkvMCJ8BT1tPZFVk4WDCwaaujllwI+MGhGIhfB18EeIa0tTVeQEtYLM3ZRyFGcUgBurGH2PDwsZC5/VbU+T3YyCfNud5QIseLeDXjxga9CHHQhpec1MRf+XlwJWG5e477yg7nrOV+2Ta1lyc3Qwh/ixsLDB532TMvjgbXAsueHwerBytIKoTYfvA7bjwIYm6tnLSfe/TnPCC+GvuoCjAMQxw7qz3JXSJ+LO0lG3GmbBuz3BPtJ/WnrVcBsWlwLHggMvXPe5XQkvw5bUvAQDv9XwPdpbqR0pDIj4MgaBOgJG7RuL/Lv0fnpUoRmIeenwIfz34C3ez77K6likNH+rg60A2oYbmNpOHe5g7WnRroZRI1benL+Y/mI+wyVqSnTXgVuYtSGgJApwCpPU0JeLiyGdYGBDeIL22//FuFFcX4+OLH0MkEWn4tjIoisLRaUfhZxuMGn46vi3qjV0niUvXwoW6EVeAzEiYnEwii9jASkMio5TSFHx17Sv8fOdn3SrCAtl3spH0TxIkIsPIAIpLgeIqjjX6vOuq5KEY3Mi8gZNPT+JpsWGrKiafh729zNjH48kMv9qSMxuL+PO295Z6wkblRCG+MB4AsLTHUljxlPuDPPHX3qM9LLmWKKkpkcrImRpO/k5wDnAGh6t+CXL7h9s4//55tQssQ8b/8HDy6ReegUL+beQ5HUV+tvLLWV4ObNtG/o6LUy3hxCyMfXy02zO1SUq+Gv4qACDbdRdsbGj8/fBv5Ahy0M69HU7NOKVxPmSwuBuJjt58f7PU+KoKnnaeKvPQMgvnxhLUzLgS6hYKK54VKoWVSvNfU4MhSIos78OCY4GhgUPhauOq+UtgR/yJRLKIHob469GDyJu0ePo5rHk2uJV1C4ceH4KEliCrwa9mTa/vcWHWBXRr0U3henaWdjiXfA5pZWlIEJ4FQPqXKtlvJkdEqd113My5rLU9xgRN0/j08qeYd3yeQsRF44i/4qfFuPb1NRQ9UTHgmgkURcHCWjnJuk7E3zNC/I0OHs1KApZNxN+TJ+TT35/9eGVrKyP1suR8tCiKgo+9D0pLyWDTVMSfrqA4FEb8MALd3+kuPWZrC3A4XJOsf0U1IriGuMLaRfNDTypJQlZFFiy5lgbLoBtrPn9eUS+ux0t7X8KCUwtQXE28JBKsQoFOawGeXMRwcRSQeZj1dZ2diXOnRCKLvLdxtYGDr4OCzJOuiN8bj8gPIxX2trfW38Lfw/5GQZz6l56t0SdLQF5cSdnzFfEHNC/iz9POE4I6AWpFtazXHFZWVhDViPDP4n8Qvy9ea/mYLTE4/vpxtcpGbIz5DPHX2JivDolFRFIswDnAbDl7n3fweXzpGvf72983SY4kc4PJ7zc0cKhBuaQqqypx7pNzeHL0iTGr95+BJtuKPPQl/uoEdVjvvR7nVpzTs4a6oSC+AGeXn2W9Zm9K4u95k/oc/ctojP55NAD9nJQZ29/Tp9qVo/TBrVvkusHBMnuVPHQl/ppLxJ+hDqmtBraCS5BiB6c4FLh8LhKPJkIsfP4ddF4Qf00MLhtR3G6/AZ3X6X0PXSL+AFm+EIb4Cx0fikl7JsE1RLsBDQDCp4dj0aNF8O7krWtVcfjxYSQUJsCR74i3u7+tsawhER+GwNnaGf39+wMgkmoMJLQEV9KIC8WgVoNYXaspiD8m4s+YUp8AkYMtTVWU3bGwsYBXhBfsvNg9pKvpVwGYJ9oPAOIb9oPt2kEqoXfi6QlE/B6BL699iS+vfqnzNdu4tUHM4jsIwBD4Fb0Jqzo/9OkDdO8OvHfmPXx97WsUVGlJJtUAb28ykdXXA6ksOBkul4s2bdqoHVfi8uOw+uJq/HzX+MTfpN2TsDRtqUGGGACYvHcyZp6ZqXCMMYYbK8dfYjHZcIe6sgy/UAN5mU/5/RjbPH/GNBR+OuhTHJl6BIemHMKZV8/gyeIneKfHOyrLyhvt+Tw+OniSPEhROVGGV4QFaAkNQY4Aghz1q7Mxv43B5P2T1Z43hPjr0wf49Vdg/JwUAICNMACCCkppsbh1q+w+NA3cvg0lyBN/2sCQA+qIhHGh42BF2aOOl48qqyR82PdDzI6YjXOzzrEisABgfOh4+Nj7oKCqAFtjtiqcK6wqRGxerMbvW1jIpAOZOVZ+XOFxeNL+EpMbw6pO5gJD8PS2n4789/Px8yh24xwzVggEsnxwjVFWRvoAjyd7f+zsSF5H6/oWGOP0PgBg0alF6PNnHzwrJDv6AD9LlcYaiqLwStgrAIDjyful6y5VhtbKSkDMqUYNnww48rK9pgZFUfjl3i/4M+ZPBfnYxhus4qfFuLjqIjJvZpqtbo2ReikVhY+VB3y2xF+tqBaRKSRCbEzIGK0SsCKRzLFEFfGXk0PGD2YeaNOGTStkaNmSfKqSg2LW501hhAGAHEEOJuydoJPkb7eF3dB6lMzKYG/PgZ2dHWprjW/0tvWwxdxrc9H7/d4ayzFG1T4t+6hNK8AW//WIP3u+PRz5pPHJpUSf3pKrQoc2/gvg+hRARcT4q4dfReCGQIWoZg5H1s+Z942W0ChLK0N5Rrne9R3962jMuzMPfAeZvGPoS6EY/v1wlXuWpNNJuLfxnlJKA3VgnCufZ+JP3umgqcChOGjn0Q4A8KjgkdbyzHrF0dcRM8/ORP//66/1O0O/G4oZp2bAylG1cV+b1CcgU5JgG/FnrH3Ifw0Lui4An8tHVE4UbmQ2vfqBqTE+dDze6/medL2oD/Iq8+DylQtufX7rBfGnB7TZVuShTeoTkI0V8sRfTUkNAgYHwC1U2SHTFChPL8ft728j9SI7h98XxJ9+0MdWHRhInErLylQ7rxuKW7fIZ281y2NdiT9z53xUB1Mo0XAtuJh5diYWxi0E19K4exVWHJCR8YL4a2KYI7HjwFYDsajrIvRuqXkDzIAZ1EtLNZczBa6lk2RJS3osgaOV5t1zU0X8AcBLoS8BAI4lHpMei82LRWltKewt7dHFpwur6zTFZObr4AtrnrVR9fFpCY0f/X/EkVlHFI5X5ley3iQDwLUM8vz7+2nfqBkKmgYeNewhw8OBTl6dEOoailpRLbIF2Qh1DcX40PF6XdvF2gVnZ51Gu+y1sLUF3noLKK4uxi/3fsGqi6uk3tDaQFG6hfxLJBIUFxerHVcGBQwCl+IiqSQJaWVpLFvDDtYu1qyjgnWFPu+6RuKvwdPWUAM6s2hnFvEMGOJPW8SfMXMC8Tg8TGgzARPbTsSI4BEa28aQDIz3fDcfEo1kLuKvqqAK37f4Hlc+Vy+t5tHeAwGDAtSeN8Txg6LIM0uvIMSfEx0IQHGR++gRydNGUUC/Bj+EGypsDLoQfwyRoI74s7Gwwev80xgem4+2HiGw5Fpi+4Tt8LFncfEGWHAt8GGfDzE+dDz6+PVROPfrvV/R8Y+OePf0uxqv0TifQuNxpZNXJwCQyjI3F8gTPM7Wzgh20aK92gAbG9nYoo7kYWQ+G+e3YDZOHs9WwMvOC4XVhbidfRuJzhvA48meuSq80o4Yck48PQFPX/Vyn2VlQBWfRDo4WznD1ZodCWwsMNH38vLg8s4DNA349fXDvLvzWEf3Gxs0TWPniJ248MEFpXNMv9DmsJ0jyEEHzw7wc/RDuEe41oi/oiLSdktL2ZgKkN+GIQKXLgXOnyd/h+n40zDzSqYKLpVZnzel1OexxGO4mHoRNfX65Wq1tpZAKBSioqLpJIPloykMAU0DiWRZodAX/mto6UjYaiYya1htNHBpNCCvnhG6FOizV6Ure3ZFNlLLUpVy1DZWcKivqceGgA0K0rG6wsrJSkkq1KeLD3q91wv2PsqWrAd/PcA/i/9hFXUjEgGdHIchwnosbOqCYWPTfLzi2UA+72xzCKoK9yAyDXEFcVrLMusVmqYRNDyIVR4rx5aOaD1aRejD/7N31uFRnGsfvmd3465EcYfgxWmhWCktVeqleqieek+9pz1f3d3dFdrSQltaKFCsuAcJloS4e7Ly/fFmsrvJysxKdgO5rytXwu7M7LA7+877Pr/n+T3NqLH6zM42W9I6IrNYCDD941VmhJzgJIQlcOuYW7lv4n2K53gdmQldJ/DCzBc4vc/pLh9Dq9Gi1+l544Y3mPKYssT0Tsw4i63I6PXm+aYjMUSu+LMcK6K7RXPRjxcx4lrXHd7U0H1yd27YeQOjbhilaHt/EP78qbWAPTJ/yOTn639uSUpyJVYRGGi+n8jzSk9RXQ3bmvN/7Ql/cr/aPCf1Qv4m/Hkrpi5JUptWVp6gPTSg1nQKfz5GkU2BoRHWz4edbXvKKeHcAefy+uzXObPfmYq2lwMJlrZmv9/1O389+pei/bd/vp3N729WeZaCl2e9zLpr1nHb2Nucbuurij8wC39/H/2b4lqxEl1+eDkgqtV0GmUDhBxsbM+b2b/H/Jua+2t4edbLHjumpJE46caTGHDuAKvHF127iGfinsFkdH6d1+vrWZ+7Hmifir/sbPG+BwWJvmySJHHPhHvQaXRcO/xaNs3fxPDk4S4fv0/PAF5+UceLL0J4VCMvrXsJvVHPsKRhDEgY4PwA8nFUNPk1mUxkZ2fbHVcigyIZly4aUy3NWqr4HJSSsy6HDW9ucHn/6vxqVj+zmvyt+VaPu1vxZ/l2VNRXUFAj/Nnc7ZdgWfFniSz85eaaqwBs4asKgdbCn2Wfv/YgrEsYo/89mp7Tetp83mgw0lRrPzGhsVFUwYJ7iR8Hy4TwlxRkLfwZjfCW6MvNzJlwxRXi7x07zGKtjCvCn60efzLRlRPQGSPcCujfPPpmfrzoRwY32xcD1DXV8ebGNwGcJgG17qfQelzxd+FPzTgh48zus3V/P5nRzc6F2VnhPDLpCQAGRI2kf+4TpKSIrE17jEkdQ9eorlQ3VlOZ8CtgW/grLobqYHED6Bffr937C6RHimB+doVZgZKFP5NJBEWDo4NJPSnVqnqmXTEJe52R11knXZlMyiv+esb0ZN2168i8KRNJklquiZIS24Fc+XucmNjW5lfu/5ifL75HkiScBdTgqOJPnp/7SviLCY5pqZBT2i96z8I9vJj+Igf/EONuWBjU19d5xfEid0MuKx9fSfmRcrvbGIyGlnm7u8LfkiVijhYcDGPHunWoDo08VsgVf8nGMij8CySLcEOXydD1fLBhb5gaaduRpLXwFxgWyMT7JtJvjmvJWw2VDRTvLbZrTW9rzTLrtVlcsewKAsNtVDFa7muCO++E/M+eJH3VIiLqB9Czp+rWhj5Fti6vrnY8h20v1Ah/lvMVk0m4Sziy6Korq6O2xHaPZxkl8YakJBGwbWx0HjCFzoo/d3h6+tM8MfUJksI7UBmtD9FpdJg0Jgq7FBKe0oEyEPwEZ7EVGcu5jKO1qeVYsfb93Wx8q33W3pYEhAaQOChR8XrCly4THanH3+G/DrPp7U1oAsScx9VYtSt9/vLyHMcXAP75BwwGkViYmmp7G8uKP0eXvL8Jf/7Ye94RvrCq7hT+OgLaQMhbAoXuNZ5XihxIsKz42/vTXjIXKrMHWPfiOlb+b6WibbMrsrn6x6u54ocrWh4bkzaG2BDndxZfVvx1i+7G0C5DMZqM/LLvF8As/Cm1+QTfVFfqNDqvBA4nPzKZcXeMs3qs96zejP73aEWVYBtyN9BoaKRLWBf6xNrPvPQUcrVf//7mxrZXDb+KugfqeHfOu4QFun9h9ewJhdI2Mt7M4LFVQri/ePDFqo4he317KutnRs8ZAPx+0PM+8mueW8PimxZTV+ZaBUDR7iL+uOcPK7s4vV5MjkFdQD82VgQuLHtzgXmxnRyeTGSQe6V29ir+IiLMGdOO7D591ROo9RgvC3+b8jZhtGG/5WkkSWLWK7MYeJ7tEpiSvSU8EfYEK/5n+54nT6QlyTWRR0YW/rpFCuFPFl0OHhSfbUgIzJsnqnd69RIT4PXrrY8hB3jUCH+VlfYtJWWByZ3FVevxPbsim1M+OoXCmkLSI9M5b+B5Dvd3ljUnJ0RsydviVz1WamvBhJGnj81kzpdzqGxQHrF0VfiLjRVBaZMJJoRexeqrV/Nwt+VoTcEtY4A9JEni/AHCzjZT+y1g21qtuBhqgsW45W6ygivYqvjT6cwVdPI4pq/XU5HtuvWeO0gaiZHzR9J3tvX709hoXri2HiuqGqo488sz6f1Kby5feDnf7hKfQUiAUAhjY4VwazAIoW3FCvj0U7MIaKu/n8zVV8M778Djj4uqv8cfd1z9aQslFX++svqUJMksCFcqs3cNjg4mLDGsRVQJDRW/bfW1dJcjK4+w/MHlDu2kM4szKa8vJzwwnBHJrmfZFxebe8HOm2ceS05E5GtC7i/7S/yFcGGttfDngJZWBK16kMvjrqXt1dQnprpcYXxo2SFe7/862z5ta3294n8reDrmaerLrW/S4V3C6T65u9NjV1eLOQQIW/EbboD77nPpNH1GYKD5OvaHPn8ZXZqFvwLnwp8la55dwwupL5C3xb4St/XDrTwb/yxHVtluzqc00UyjsW3hZw9POY90cvyyp2gPyw4tU+wSZA+tpAUThNaEUl3WAdSTDoosOIj+xfa302hEYpe2qZ6V9/zCiv+toL7CzqLQi+jr9RxecdjhPEnG0vWkvelIVp8zX5zJHbl3tNiFuyv87d+vbPuSErjlFrHeqHMQgluzRvy2V+0H5oq/mhrHIpr8f/MX4a8jXSe+olP46yicvgNOda1Cx2QyUVJbwp4iJw2nmrEl/F2z9hqu23ydov3P/exc5n6rzIv8iVVP8OHWD/l217c06JVbQoJvK/6AFhvIF9a9QGldaUt/OjXCX+ss1uONk248iZnPz1S07ejU0ay8ciWvn/56u1Q0yMLf4MHWjyut1lRKSkQKBdXmFJyLBl+kan/55p+b65mb2YxeQvj74+AfGIyebVQ78b6JXL36aoIiXKv6SB2dyjXrrqH/2WbrG8ugoBqRR6czB0Utg/myvY4ri+2GBvj2W9Fvsb7ebAHXWvgDZX3+fCX8yRV/tbXi/zQwYSDfnP8N267fhoTv09K1gVqGXDaEpGG2s3ktkz7cGSoOlYveBn0TrCv+ZIvWQYPME9oJza6ZlnafjY3ma0ueKDsiLMwcOMrPt72NJwP6uZW5DHlzCF1f6sqGYxuIDYnl03M+dTrGOZs8ZyRmMDhxMNN7Tade3/6LVXvU1YFRamBr1e8s2rdI1bXsTPhzZHPTq5f4nZUlqilL88RF40z4A7hg0AUAbKz6CYNUZ1P4KymBmqDmij+l1QGGeijdpGxbJ8jCX2uBRx635IXhR5M/4r0x73nkNT2F5QI4yOK2VF5fzozPZvDzvp/JKsvis+2f8cv+X6z21WjMgsMzz8Bzz8E338CW5kJXR8KfJIkxYcgQmDpV2ImrRa74KygQ47QlblX8GfXQ6H62WdcoceOzrAR1RI8pPZi/aT69ZogvjDwWemORPvzq4czfPJ8uQ2x8OM3I/cIyEjNcnveZTPDmm+I669cPZs926TDHDbLV56k9TmXdNeuYN3Re241yf4HvEyF7YZun7PUgl9dKJe7Fv1uI6RnD+LvHkz4uvc1zYV3CSB6RTF2pefDIXptNVZ6yNPLKSjBIdejCKrn3Xjj99Paf43kCOZnJL4S/5oq/A6UHVFkLp49PZ9QNoxxWoicMSmDovKF0ybA9VqhJNJPdP5z1+atqqGq5xjsr/lyjQd/AT3t/4olVT/j6VLzGh1s/ZOonU3l4+cNuHUfbXF39n2f/w5Lrl3ji1DqxgZq4ZNeuYAgIpsertzN/03y7/UW9ycE/DvLx5I/Zs8B5fFhem7ZOfmwPOlLFnyRJRKREtMQxXY1Vy0n/+/eL5ENnLFwo4lLV1bBune1t6uvN6xc5pmGLwEDznMtRnz85huUvwl9Huk58Rafw11EIjHZ51+LaYuKfjWfgGwMV9XWzVYUWGhequH9XfP/4Nn0T7LEuV4xOj05+tGViohRfC39XDruSyKBI0iPTqWmsITUilejgaIYlDVN8DFtZrN7GZDJxztfnMPKdkRTV2IlyusiCyxbw+azPAWisblS1b5AuiEndJjmtRvEEJhPs2iX+bi38eZqEsAQemfwIIIIhcrBMKZGRZlFBSeZPhJM78KiUUUQHR1NeX+5xa8eUkSmkj0tHo3Pt1hIYHkjamDSr/iqyyBMc7Ng2zxa2gvlVDVWEB4bTP059X42VK+GTT+Duu+HH5vae0dG2gzpyPyd/FP5CQ8XEDoTdp06jY+6guXSP7t5uNoJbP9rK++PetxlMi+0dyzmfnmPXxsuTY79Oo2NImnXFnzw2WFrzydlx27aZX7+gQIwlISHK7Vpl697WlYMggvuy0O2JrMq1OWtbrLGGJQ1j4782ckr3U5zuZ0v4sxxXQgJC2HHDDj4/9/OW6ih/oK4OjBqzQhKkU56A0LqfW+tCRnsVfwC9m9vMHBBtrVrEO3s2KpaMTh3N1B5TmT/kNoyaBpvWKsXFoDWGEa1LUl7xt2yG+GksEwfc9hBUZSnbtxVyFU9rS0fLPn8Agy8ezNArhvqkCjRnfQ6v9XuNnV/vtHrcsr+fPLSV1pUy7ZNprMtZR0xwDB+f/TEPTnqQCwdd2Oa4cpWe5Tgu/+1I+PMEUVHiPTaZ2laCyjbNLo0Ta+fBgiRocs8PR23FX2vCw0Gr1XnF6jMkJoTk4ckEhtm3ZTyr/1lsnr+Z52c87/LrbN4srJN0OpFt7SjL/0RAviY0koYxaWNIa8yBvFbuEoGxENkPtG3vHUqtPgH+ef0f3hn1jksOE12GdGH6M9OJ7x/f5rmR80dyxfIriOkpvlwmk4mFly3kvTHvKRrbqqqgIHoRPwyIYsanM1Sfm78gJ67YSkZpbxLDEpnTbw43j76Z2ibnJcLyfKXrxK7MfmM2CQPsl+H2ntmbsz8+m+Bo24F3NYlmcl8mZxV/RpORx6Y8xnUjryMmxEd+zR2c3KpczvrqLB5a/pBVcu3xxPaC7QAMTRrq1nF0Gh1IsGHUBtJPbZvs0IlznMVWQJ31YUufv2M6RX1IvUHXiV2Z+uRUekzt4XA7S+ckX1b8+buFY3VBNfsX77dKGnI1XpGaKmILDQ22XT8sqagQdvMyK+wYBG7cKBKWk5PbtqhpjaXdpz18HYdvjWXswo/MiPwKz3cq7EQVWqWRbEODWDwFRkOiuv5ncaFx6DQ69EY9BTUFLdnb9rDV46++op68zXnE94u32fRcxmgw0lDZQHB0sNMAcm1TbYttx0WDL1KdcetLq08Q/WC2XLeFsIAwuoR3YfdNu6lsqFQlYPqi4k+SJNZmr6WgpoCcyhwSwjznS6Sv16Nv0NNQ2cBL3V9ixL9GMP3p6R47vqfYsUNc3zqduaLOm9w65lZ6x/ZmZPJI5xvboG9fYSm4dy8Md9B2UKvV0ksuP7G3jUbL1B5T+S3rNw6WHWRM2hiXzske+no91QXVRHeLVr1vfXk9uhAduiDzWOBKfz+ZhATIzLQW/m4afRM3nnSjS5VK8uSroQE++0z8bavaD8wCz6FDYgJiaziUJ9LtLfxJkhAsCwtFgoe3AteOqC2ppexQGbVFtaoXPZ4a+zfN34TeqCfvmIbPMfvZyxV/Ay1cxFJTRVDn8GEh2k2dat3fT6leOmUKbN0Ky5bBBRdY7yffcwMD3bMwlTlvwHncP/F+TJh46OSHFIt08uRZfp+VjCv+QF0dGCTxvZaQCNAEKN5XFniKioQVyhtvwKRJcF2z0YEa4U8WkJVU/EmSxB/z/sBggPM/NFeRWtpCFhdDxtE3uHPuG5wyQOFqpu/NUJsN2lAo2QC7HofaozDuY2X7W+Cs4k8W/sbe6rvmZsYmI9pAbZukE1t9H+cvms+mvE3Eh8az9PKlDpO1UlKEO0BsLIwaBb//3n7CH4j7y86dos+f/BVsbDR/N1VXBptM0FAE4T0hwL1gk1zdpbTiDyB/az5rnl3DyQ+fTGRkPGFhYQ5tiVylprAGXYjOoftAsC7YrT7OIOaSAKeean8u0CFoqgRdhNuN6CZ3n8xX531Fn7jmCdD2h8T4M7fcvFHCOJi+yub+aqw+68vqqSmooba4lpAYzyWgWK5dTSYTJqOJqU9Npam2SVFiVFUV1AcKtUxJ2wp/RU5c8YeKP0mS+PGiHxVt6+n5iprgplLhLyo4igdOfsCt8zrR6RnTk9Gpo/kn9x++2/0dN42+yden5HG2FQgr4iFdhrh1HK0k4lK/nPELn175qdvndaKhdExRM1YkR9WSdGAz2TGDgWi3zs9VgqODmXjvRKfbyYlmOp1vqtct16T24in+wKE/D7Hg0gWc9+V5DL5IVBW4Ko5pNCKOtH27SPqX7y22+PFHsS5IShJuQlu2iPhS64Rk2bFo/Hjn72FKinhtJRV//uJoIL/Hcu95fxEk7aFYA/IgJ3huou8xys1CnG7YBKvOhr0vqX4NjaShS5iITORVOe84bcvq8+iqo3xy6ifsX+K45Kj8cDnPxD7DH/f+4fR1tuRtwWAykBye7FSMtIU/ZBr0jOlJl3Bz1EdtzzBZ+CstNfeNaQ9s9ezxBHO/ncsVy66grqyOtLFpxPZStuhtNDTy0LKHeO2f19Ab9R49p9bo9cKaCWDGDHPlkzeRJIkz+p5BcoQCP0AbyCX/zpr8Go1G8vPznY4rr5/+OiX/KeHiDHX9BpXwzXnf8Frf11yq+lg0fxFPhD6Bocnsa+CO8Cd/v1rb90mS5FKlkmzPaBn8t5c1lZoqJsl1deYqIktMJrPwp7RazJO0HuePVR3jyVVP8sCf7ROMGHfHOO7Kv8umFdumdzax6LpFdquGPV3xl5ykQaMRNhi7djVXQerMgk7LOTe3MP3nH/HbUvhTyvjxovro2LG2fTstbT49sbCRJInHpz7OE1OfUHW9t86utDeuGIwGjlU5WBW0M5YVf0G6IFXVq3LFX2YmPPmk+G7+9Zc5a9BR70X5OsnNFdvJAq6Sij8ZrdZc2d060Cq/dnx82/6Ndul2AQy4E7RBED9aWMWPel35CVkwOnU0e2/ey+b5m60e96dG6l0nduWGHTe06RtqWfEH0GRoYneRUPaXXLrEqUPDxRfDtdfCK6/AHOHuzr59wnpHHtfV9u5Tg2z3aZnx61aCgCSJa2H2LvfPLTKdEF2Iqr6wtcW17PhiB0dWHCE42EhDQz3V1Z5Pzf363K95tc+rHj9ua+RqqB6OE+b9n9WXwu9jhUWwG3SL7saFgy/kQOkBnlvzHEdSLoST3lC8f1pkGmmRaXSL7mY1h7RcK8kPn/zgydyefTtxfdR5jxn1Rt4c8iarnrAtPgI01jTy1Vlf8cuNv6DRahg0dxDDrhim6PiVlVAfkNvy/+moyIkr/iD8qaH1fOXPB/7k2wu+tblt9tps3h//Pgf/OGj3eGrt+0DM7xptT1878SAXDRKtM77c+aWPz8TzFNYUkl+dj4TE4ET3rIm0Gi3nDjiXuQPnerydyYmA0tiKmoo/Q+Z+0jL/pGbnYfTeDXs5xWgwoq+3fxLynDM62jeiW2tBx19JG5fG6W+cTtdJ5iwwd+IVcmGCo9hfdTX8/LP4+9prxXrUaLRuSwIiviHHLhzZfMrIcY08B7KBP8ThLQkIMK/1/GFd6gzFGpAH6RT+fIzi4HhAOIz7HDIedel1ZMEhr9q58CcHtqqqaLkZJY9MZuZLM0kf79giQBuoZfi1w232TWjNP7liBBqdOtolezl58PeXAccVYmLETdRgMGfUtAfyYrS1nY67yJ9jdLdoLl18KSPnK6twK6gu4LFVj3H7b7e3ZKZ5iwULRLAmKgrm2Wg/4o/IN/+9ex2Xr5tMJvLz852OK13CuxCo9Y7i2f/c/oy+ZTSGRvX9A7ud3I1hVw9DG2C+BmThz5XqLmd9u9QiT4BuugnOOUcEXcfaKXLR6cxBgEOH2j5fWysmYuAbz/zWwl9FfQX3L7ufl9a/5HXxHRwLGAf/OMjmdzejC7a9QPV0tbdOZ67a+aM5Z6VPn7ZJASedJH5v2SLuja4If8HBZtvQP/+0fs5RH7n2pLXVp61xZXPeZiKfimTCBwpWEO2E6PEnvlRBWnV9RuWxwnIBXl1t/n44qviLjhaBaZNJ2AGD+H6puT6NJiMVyT9wsMsLVtZqJhPklVZiwqRsnKg+BHtfgepWgcykqWIe6QJhgWH0jetLWKD1f6h1xV/JvhK+PPPLNnabvkQeY0Oade8AbQC7btzFtuu3MSpllNP94+PhrLPEfKFrVzHm19eLDFz5+5pkuxWpR5CFv6NHzY/J16Q8f1SMsck8gWiqgPw/HW/vhCuHXUnN/TW8O+ddxft0n9ydWw7ewsj5IwkLM9HQ0NCSxe1J+p/dn+FX26/mq22qZf6i+by6/lW3eh3L31Ul1b1+i9EA4T0grAeUe+a7++7md7l76d383RgC3S+xftJkgp2Pw8GP2uyXHpVO9u3ZrLhyhdUcQb7WLS3HXKWmqAZ9vZ6GKvt95QPDAqnOr6aupI6Ko+peUFT8ibWVXMHYEZETV/LzlfUYag8qGyrZU+S4H1Xr+UrRriJy1uZgNLQNslUdq6J4TzGO2gHL8yAl9/OYGBH0N5msx+zWbM7bzL6Sfe0y1z6euWDQBUhIrM5ezdEKB294B0R2xOoV24vwQPcCXRpJw/cXfM9FP17E1he2euDsTiyUxlbUCH9jrx3E4YmXUpLQz6G44i3WrYNXX4VDf+fwTNwzbPlgi91t5bmuL2IVIAQduUe3P/dvi+kRw0k3nERkqrkIxB1xTHaO2rHDfnHIzz+LtW+3bjB6NJzS3M3jr7+st/vnH7PNZ+ukZls4s/rU683xOX/p8Qe2W5X4K75oi9Ep/HUkul8E0a5l/SSHNwt/Cir+wsNFIBTMAYaI5AjG3jrWoU8+QFR6FHPenUP/s533z1qfK5ocjUl1zWrQ3zINXEGrNQd5PdW0XgnyYtTTFX8A2z7ZxrIHl6nap7BGpM4nhiV6tcdYQQF8/bX4+9prfWcTq5YePcR3srLSbC/mKZT0/VTDiGtGMOPZGVZ2nUoZffNo5rw7x+oxWeRxpeLP0r4P4Ej5EQa9MYiLv1df6WgymSv+UlLg6qvhq69giAMHFtma4fDhts/J5xQRYZ7Qtiethb9+8f0IDwyntqmWzOJMr7++yWhi9/e7OfDbgTbPnf/V+dxddLfdXpGeEP7e2PAG494fx5sbRPmvPMn9+2/x27K/n0zv3kLkqasTdqCuCH8gbEIBVq2yzgr3V+HPFj1jelLbVMvh8sOU1ZXZ37Adsaz4C9bZ7tdjj9hYMV6EhcGDD5oD+UeOKBPpZRcgubeCmmo/EIlQXxjPITP1fvYcNQ/yNTWwKekm/hiSxoqi75wfqHAFbLoVyra2fa5oLayfD/WeyYSQhT852KEN0nLg1wOU7i+1v5OXOLLyCP+89g/1FdYVS3LFX4hFwaskSS5ZZ0mSufpeFnhDQrw7B23pA2NR8Wcp/Kli3+uwOAMq98GGG2HZNGhwfeKp1WhVz9c0Og0xPcSJy+O3Xg9Nnp2GMP6u8Ux9Yqrd53cX7ebdze/y2KrHVPcXl9HrzXOCDi38abQw6hXRSuK3k6DWvXXBbwd+44+DIoPGZoKZJEHm83DoE8XH1OnM17ts96lv0LPjyx0cWm4js8oBEckR/Hvfvx1eHwBXrbqKUdeP4qVuL7HpnU2Kj19VBXUB4j3syBV/8fEiUckyycmXbDq2iainopj6iePPrTVzv53L7dm3o9G2nU8OPG8g/yn9Dz2m2C/ZVRNrkCRldp/X/HQN/V7rx68HfnV+0E7skhqZysndTgbgm13f+PhsPEtLf78u7vX3s2TPgj0cXXV8CaT+hJqxIiBYR9yY3hgCQ5xaA3sak0k4X/3+O+wvjSNxUCLBMfbXTPLa1Bf9/WT8yWHEFiaTCaO+rTrnTqw6I0Pcg3Nz4VvbRessaw63nn++uP9MmiR+79lj7TYlr01PPllZwqDsQGOr7zxYV176Uxze368TX9Mp/HU0DA3QWK56txbhT0HFn9z/CaztPj3N6X1O56LBFzG5+2SX9leThefPyBY21XkH4NCnoHfeuNxdvFXxB7Dy/1ay6vFVNNUqj+QU1IhAp2xJ6y3ee08E2ocMMWfFdAQCA81WUq3tAV3lm13f0O+1ftz+2+2eOaCXcEfkkat45EBRZnEmu4t2tyyoQIxxn3ziPLBRUSGC/5JkFhSdTZ7kz8xWxZ8s/CU4zqXwGvIEXq401kialh6UG3I3eP8EJPjp6p9Y88yatk9pJELj7Cu9nkj62Ja/jXU568ivFpFbWahpaC4CGDiw7T6SBCObi5g3bnRd+MvIEON+TY3ZegPcCOh7GCXCX3RwND2ixQW+Jd9+pmh7YTKJ76dRakRCIkinTk3XaOD11+Gjj2DMGLPgcuSIOSknLMxsI9IaOYMyK0v8VisEjEkdw4CIsRg1DSwqfK3l8cIiI0WRv9IQeIzkSAWDRdpZwsox0cYNrmIXZL0LxWvVnRzwzqZ3uPana1ucGqBtxV9U1yjur72fkx88WfXx3WX397tZ8u8lbeyBLYW/yoZKl3q7WjJggPgtC39dunjX+kiu+MvLMycJuJwgYGwSa4iwrtDrGmH9KrW/7ZfRYGTPwj3s/35Hy2PtnZ0rV1O4Y6EmV0IFB/suE95tGkpExR9A+nnCVcbNa+K+P+9r+XvSoadhqY0eQtNWwfjPVR1Xfo/l8ViSJBZcuoCNb2x06TydidbaQC3B0cEMunCQlXWXM6wq/iI7bsWfpYhlaw7b3vSNE9YnedV5lNQqT1iwdBCxhSRJSBr714JadyFHCX8yh8vFkz1jeio7aCd2uXDQhQB8u9tOZLyD4qn+fpbcW3Uvl/xyifMNO3EJpRV/JftLKN5bbLXOaE8OHTLPI7NyQ7h69dVkXJxhd3t/SEpt3Xve36jOq+aJ8CdY+fjKlscaG81Jba7EKyIi4MYbxd9ffGHuL255fDn5bNgw8TsuTsQYwLxOqaqCzc2dGk5WuDxLShJzgNpa8xrPEvlaDwsT62d/oSNV/PkCP/qoTkxUZctWHYBvI2H306pfp8XqU0HFH9ju87f434t5Mf1Fh6WpO7/eycJ5C6nMsTFKtGLe0Hl8ed6XjEsfp+icLLH0efanTANXkBezYYUfw9p5kPOT119TXox6o+Lvst8u44IFFxAQGqB4n4JqIfwlhnmvWY7JBJuak3avucZ/mwPbQxYitm+3v40kScTGxioaV4K0Qewr2cfvWb976AwFxZnFfHXWV+z+freq/erK6vj63K/Z8cUO68dtVGwoRRbVysrg4EH46g+hmvaL69eyzQ8/iEyq++5zXE1p2d8vQOGl7ShoIouRvhb+LMd42fpu4zHXAmlqkCSJcz47h2nPTLN63GQycfDPgw7ttTwh/B0sF1aIcuCldYWWHOBvzahmd8A1a8yfoZwZpxRJglNPFX8vsyiO9ofFFbSdONsbV0YkjwBEv15fI48T0bWjqL3HQNYtWaqPERxsFvYsF+RKPpfW1ilqhT9Jkrgu4y4ANpjeoKZRTHBWZW2kMaCYQGMk49PHOz9QYAwkTYMgG0pE1/PhrMOQNqftc074ed/PvL/lfbbmb215TA5uyItCSZKcBlm9xbg7xjFv2TzCEq0zRCzvH8+veZ6k55J47Z/XbBxBGf2bzSxku8Eu3s1VIjpafB9NJnOigcsJAgPvhjMyQRssrpG+N0Kgew1mr/jhCka+M5K9xcozkiSNxO93/M5fDy0nOkpcL54M5tSV1vHlmV+y7ZNtdrfZWSgsLTMS7Qe7nCHbfKamdrz5ZAvrrhZVoPoaSJ4OGQ9DiHvetelR5lYP+oBoCLIxyYkeBCG2b5z3/nEvPV7uwQdbPrB6XE6SlO+72kAtFy68kFP+qy6Lb8cXO9j+mYOJtAXJI5I5/6vznbrdWFJRaTwuevyB4+S19iYiKKIl2WhH4Q6727WerzTVNbFn4R6y12RbbddY08jfT/1N/rZ8h6+rNslYtvi3F8yvaqiivL4cEH1SO3GPcweci0bSUK+vp7bJ+4nT7cVd4+/i3TPf5ax+Z3nkeFFPRaF7TMfBMvv9LDuxjdLYitK16d9P/s3r/V8nKbgccGwL7A02WizxlSST+8PatHXveX+jsbqRHlN6EN0tuuUx+Vw1GtdiWABTpogfoxGefdZa0MrJEWuD8HDRjkBGLmz48UeRZL5unajc797dfH9yRmCgec5lKzFeja1teyKfj78KfwUFIhadna1SA/IQncKfj9GokcnDukHK6RBlw4PMCRPSJ3DTSTcxrec05xtjHtwtg8LB0cHE9Ippk1FtybGNx9j+6Xav+9bW1Zn9jo+Xir+11f8VGdhdzxcPlO+E38fDMc9bgaRFphEaEEqARrk4p5SYnjEMOMdO1NwOstVnl3DvRdHKykTmjUZjDup2JORsnq1b7ffD0Wg0dO3aVdG4MqXHFLSSlv2l+zlU5rlVvclkYv/i/ZRlqSsXrs6vZu9PeyneW2z1uDsVf+Hh5j5tt94Ky3cIC8uekWbhT570lpbCQw/Zr3KWPfjViDxy0CQ/32wVKCNX/Mnf//bGlvB3UopoYrcxz/vCH0C/M/uRMtK6XK6utI5Pp31qlTXXGk8kfciLX1n4s6za69bN/rGHDxcWzbJIHBZmrnxSw5Qp4vfmzW0reXxd8dd64mxvXBmeJHpo+UPFnyzwSBIEBUnoNO5Vrdiq+HNU1eOu8AdwxeizCa3vTYO2lJfWvg7AH0eWiONrphOgVXC/rjlq/wYRGC3mkS4gB7CzK8zBU1uWKsV7i9nywRYaa+zPE71BdLdoekzp0UZ4lK+L4GATn+34jIqGCuJCXC/P6tvXWuTxtvAnSW37/LlVGdza1tJgv8+ZErbkbWFz3mYOlSufQ0iSxNmfnM3Vq68msUs4kiR5VPirLa7lwG8HKNlvvypoZ5EQ/typ+Ovw/f1MJgjrDskzQee5hZSlmLGv7/1w8sK2G+lrRB9SU1trrIr6Cg6XH26pipJpLfwB9D+rP4mD1SUM/v3U3yx/aLmqfdRQVtVAevHVjIs9o8Vtp6PiT8IfmL+vcsWuLVrPVwwNBr459xs2vW1t15qzLoc/7/uTA7+2tZu3RG2imZzwZy+Yn10p7qExwTFEBPlZ1LQD0iW8C4dvPcy267cRGuBCTwg/ZXDiYK4dcS1Dkzxj9SkhkX40neLtxc437sQKpbEVeaxwJoYMuXwIp/z3FLoPiwbMicXthaXwd+gQ1NUY+eGKH/jpX7YLDzqFP+fE9Y3j0iWXMuQyc4WuZdKIOxrPDTeI2FNREXxuYZQgz0HT062Pf8op4j5UXg7/+x8sXSoeV1rtJ+Ooz5+8DrEUHP0Bf79O1q2DRx6BL79UqQF5iE7hz8cY7XXrtIUmQCygelym+nWm95rOa6e/xtxBcxVtLwcU5MEe4NT/O5Ur/7qSoAj7Floznp3BfdX3WTU2tcW2/G1kFmditLHoU4IcJNDpzIH9joq8mC0q1okMbDlgWZ0lrLnwvIg6uftkqu+rZvGliz1+bFeQrT4TQ71X8Sd7XcfHi8B9R2PwYHG9FxbanyQajUaOHj2qaFyJDIpsqbZdenCpx84zvl88DzY8yIT/TFC1X8KABB5seJBJ902yelxuHuyK8GdpVaTTQW2IUPnCG4XwZzDAgeY1f2SkEPceegiWLIH9+0WGlIz8niepSIaPihJjqcnUNvvX1xV/tuyc5Yq/bfnbaDS0T+Be36C3Egm0AVpmvTaLQRfYT3Bx1+ZZb9RzpFx8ILaEP1s2nzJhYdbPp6S4NqFPTRUTVINB+PeD+bPwtW2cpaWKwWB/XBme7H/CX2ioZ6pvZOHv6FHzd9XR5xIdbS3iuyIGREdpGVL+IABPrX6MguoC1haJe/Tw8NOdH6C+EH7sJvq32aOpCnJ/Vt3nTxb+cqrMLgGtrT4Bdn+7m5+u+YniPe0bXGqoasBkbDtXkq+LXM1aDpYdJDwwnLP6u55BHxZmnTiU6L0pSwtyhq4cSHYpCLPjf7DhZmtReNeT8G0U1ClzArGFXN1lKQgrodukbkSkRmAwVGEymTwq/MX1jePBhgeZ/N/JdrfxhNVnhxf+JAlGvQwjXzQ/tvMx+KErGFy3xLUU/mz2+APYfAf81Asa2vYDlR1JciutWxHYEv5AzCFsffftcf7X53Pu5+cq3l4t9VUhDDn6Fq+OX6QsWcOP6dnsROkvwp9coStX7Nqi9XwlODqYcz8/l/F3W1fMd5/cnSv+uoKMSxxX/apNNJPXCKWl1msImaMVYiC3rIztxD0630vn6DQ6Lv7yYjY9qLxfaScCpbEVWWxwNlb0mNKDyY9MbtOOpD2oqoJMkQNNaKgoosg6pKHsUBllWWU2Czd83ZoE/L+SyxZKhWBnhITA1VeLv7dZGFnIvb/TWw1/QUHw8MNiXXr4sNkiVK3wZ9nnrzW+jmHZw997/FnGNFVpQB6iU/jzMd6ujHMVW9UgSgkMC3Tolw9w75/3MuD1Abyz6R0Xzs46A6/DWuw0ExcHlw+5n+6G962fSDsLziuBpOkef02NpPFJibE9Wnr8ebHiTxb+2iNQ5w2Cg802Y1u32t7GZDJRWlqqeFyZ0XMG4FnhT9I47pfhCI1Wgy7YulJHXnSHupjIeffdwsbzk0+gMbK5vK9ICH/Z2aKnW0gIPPOMmCQdOQJvvAF33AHz55sr9Vyp+AP7dp++nkhbjvHy5dIzpifRwdEA7WIHc3T1UZ4Mf5ItH5iFo6DIIEbfNJqeU+33PnHX6jO7IhuDyUCQNqjFBjshwWzh6kj4A7PdJ6i/HmQkiTb9Hfyl4s/yfa2psT+uyBV/mcWZPrdYkgWeyujVnPfNefxvxf/cOl5SkrgeGhpgd7NrsTNBtlcv8TsgwPXv9cSIy4mqGUV1UxXX/XwdBxtEv82JSac539lkgL43C8s+e+QughVnwjF1ST9yMN/SHlwW/qqqzGPIwPMHcuEPFxLbu31Tg98a8hbvjXmvzePydbFL/yMA5/Q/x+2qAPk+DOoSQVxFXtTLi3yXKv6OLYa8JdYT5vCe0GWyS33DW86t+bqQq1jUYDQYCT+wmeiCvR7v2yJJEhqd7SVuSW1JS7/zgQlOBnsHdHjhzxaSDoIToc71EgTLIHxawSI4+HHbjZJnwoC7QGr7GaVGNAt/Vc6FvxX/t4LHgx+nNKutgGiPhAEJpI/3nlAgB5xccQLwN7p1E0NGaanZ3tiXZHQRIp0jq09b85WMSzLaVIZqtBq6n9KdqHTHJQtqE82iokSioclknTwtIwt/XaOU943sRBk1jTVUN3YgZcAOG3I38PbGtx0K3GrRarT8Me0Pel7T2VdSLUpjK87sD40GI7n/mO9r8jqhstLcX97bbNkixqZu3cxOUnv3ijY9Vyy7wmZsUL7n+sqhCPy/d9svN/3Cxret3ZI80ZZEpl+zUVVOjjkuJScD2rLvTEgQ4p9cHNOvn3qHEkcVf0oSYn2Bv18nli5mvtCAOoW/jkbxelgxBwr/VrWbyWSiuLaYnYU7FVXZ2bL6bKxuZNWTq9izYI/tnYAjq45QsMNBo6zmc/kn9x/AXGGiFnfs//yNhNhazuj7Kv1Cvm/7pEYnfg5/CRtuav+TayeenvY0f1/1NxcMusBrryFb83nbmsubWNp9eoIZvYTw98fBPzAYDZ45KJC7IZfMHzNV7VO0u4gjK4+gr7dOkZWzY1wV/pKSYPx40ATVUGESE+7qI2IGtX+/2KZ3b1GB9dxzMHeusHMMDBTi3I7m+IIrFX9gtko6fNj6cV9bfcoVf3q9+T2WJIkN/9pA5X2V9I/vb3dfTxHXJ47es3pbBV6UZO67O/7LomaPmB5omgOPkiSy4bp0gZEjHe9/0knmvy0rBdViKfw1NZkXjb7u8afVmr9vjrLmkiOSuWzIZTw46cF2qxC1hyzwNIYeZsGeBaw6usqt42m1ZsFFHgOcfS6y3WdqquvNznv20DAo+yVSdRlEB0djwkRk7TD6JCm40EKSYdSrkO6gmiVpGox8RQg+KnBk9anXm9//+P7x9D+rP8HRwaqO7y595/Slzxl92jwun1e2XiQXnNxNZcqrDSyFv/aYT8jJIzt2CFtgl4S/GWvhtFYWzt0uhCm/QpQ6a3ZL3BH+6srqiPlnDSn7V1JV5bkFcOmBUrKWZtFQaTuStq9kHwDdoroRGeSaOmMymSu1O6zwt+EmUQVqyaB7xXUS3t3lw1pW/HXN/RL22eipmX4uDH8WgtoOqi0Vf62EPznAJFsvAyQOSmTg3IGKExkbaxrtXheeori6nCZtBeHh/pnYq4bgYHNykz9U/ckVujsLd6oOmjVUNZC7QVxTxXuLyduSp+gYaiv+JMk8r5cTTi1pEf4iO4U/T/LIX4+Q8GwC729+3/nGfs73e77n+l+u540Nb3jsmFpJy+aRm0ma1Q7ZSicgJpNzoWfPgj28N+Y9Nr0rqi5DQ82939qr6m9Tc8HnqFHmuezevRAQYrs6vabGHCPoFP5s01TbxOZ3NnN4+WGrxz0p/MXEiPffZIKs5hb2cjKgvTlonz7wn/+Ie/hcZYZ/VjgS/nydvG4Pf6/487V+0Sn8dTSMjXBsCVQp6MZqgd6oJ/HZRDLezKC41vndxVbFn0anYdkDy9j5lf0MpO8u/I6fr/vZ4bGzyrIorSslUBvIkC5DHG5rD08Opr4mLjGUK3/I5fV/XrPbmofcRXD4M5uWTIsWwccf22/rY49bltzCyHdGsuqIewFST5AWmcaErhO8mgHZ0Sv+wCz8bdtm7nHpDqNSRhEdHE15fTkbj3mur9uyB5ax4NIFqhbmG97YwEenfERdWZ3V4+4KfzIldSX0i8kgpKErRzJjMRphn4j/0ac5XtylC8ybJzzR5R5s27eL364Kf3LQ1lL4M5nMwStfTZoCA80TD8txvndsb/v2XB4mLDGMi3+6mP5nmyPpH03+iE9nfOpwP3fH/3p9PT2ie9A3rq/V47fdBu+959yWIy3NPI6kprp2DmAt/MmfgU7nH/c1pYusT8/5lEenPNpSKeorZIFHGyiCukFa+5bkSpE/H7kHo7PMxgkTxHdq0iTH2zli8GCIrZ7AuYVbuXPcnQytuZP0oms8t+AOToR+/1bd66/F6rMyp2VcDwoyZ5O2XmQ11Ta5fapqmPXyLJvWjnV1YMLE4fqtAAzt4n6/nAEWOll7zCcGDxavU1UFy5aJvh2gUviTJAj0fCmxq1afAKFxoTScO529Yy6nrs5zDhS7vtnFZzM+o/SA7SqwcenjqLi3gl8vc71/dmWlGBslyb3kD59SvAbKNnv8sPI1EagNhFMWwdiPVO3fUvHnwOpTnloOOHcAc7+Zq7jCeM/3e3gq6in2LlK3hlZKYyNkxj7Hb8OjeWzj7V55jfbGn/r89Yvrx/wR83n81MdpMiq/x5hMJj6c+CFfzfkKQ6OBNc+t4Z0R79gdIyxxZb5pz5YWYGavmTw6+VHO6HuG8gN24pSY4Bjq9HV8t+c7X5+K2+wq2gW4Z0XdGrnvtSeTfDsxU1trvi/ZGyuShiYx/NrhLWteSTLHAIrUue+7hMlkW/jLzBTP5azP4fuLv6e6wLzwk8ew8HCRCOIr/Ll3W0BoAPdW3stpL1k7s3g6Vi27yhw4IBIuZUGutdWnJWPGwDvviN9qkee2eXltY8z+UAVqC3+3hHWnfZEn0DnfpBNvotpuMX4cXFANKoNaAdoA4kPjKaotIq8qj8Qwx9EKWz3+dME65m+aT3T3aLv7nfr4qQSGOw4ab8kTmdfDkoa5HGBWm4Hnz8TGQr0hktyKSCor7TRKHf4cjHqtTXZsTQ28+64YkCdNMvdjUMLekr1szttMVlkWk7q5EansIBwPFX+9e4ubRU2NyPjp06rAQZIkkpKSFI8rWo2WeUPmUa+v92iT+Qn/mUBDVYNoT6lwiMu4JIPY3rGEJVrfDT2VHdM1qiu7b97OhReZqK8XYoss/PXt23b7IUPgt9+E8FdfbxZm1Fo7Wlb8mUxiol9eLiZtkuTb6q6YGPH+lpX5R9WCyWQiPCm8jd2r9TbuT5xm953N7L6zXbZZkCTRbHvNGlFN6iqyPYel8BcT4x/21ZGRIlmiqkr9uOILZOFPIwt/OveFv9b2Kc6Ev/R0+Oor915zUHNry8OHNPQMz6DfkeeorVW4uFpzGQQlWPfssodRDyYjKJyDycJfTVMNFQ0VLUJvZKRYAFZWmu+tH0z4gNqSWm7OvNnO0dqP+npoCMijQl+ERtJ4JJCWnAyXXiqET3cTUpSg1cIZZ8AHH4jrS76PyFXbTqk+BFX7IW4MBLaaYB76HHJ/hPFfmPtLq8Cdij9JkkgY2Yedy4NbLIU9Mcb0md2HkLgQYnraFzojgyJdrvYDc6Z1QkIH7jM+awvobXisZv8AhStgxAsu3YxSI1L56ryvhAAYnWHTzpPKfbD5Tug5D7pap6LLFX9l9WXUNdUREiBKIuS5klwd74qVZlTXKIbOG0p8f+9EqyoroT5QCJYpUR04y9CCnj1h9Wo46H33d6cEaAN4+8y3HW5ja74iSRLj7hxHU10TJpOJEdeOILp7NHF9nPuUudJT2lHvrkndJp0Qa+725ryB53Hbb7ex+uhqjlUdIyWio2ZkCPt8gAHxrlfjt0ar0TLv43ls3LyRkT86sTXpxAolayBZkLJMiJP5+6m/qciuYNpT05jz7hyr5xIShGVjewh/Bw4Iy+bQUCH6GY1ifllWJl6//HA5O7/aSb+z+zH4QjFX9heBx98FnYCQgDZVk54W/vr0gfXrxeeYnw8GgxBjvZVA3qWLmALW1YnrxnLN4S/XRWv8WSAG8zURGuqCBuQBOiv+fIxGrR+URqda9JORexnJvS0cIQt/5eXWKn/y8GRCYkLs7jf8quEMmjvI4bFlm4ueMa77jLsyEfdLmirRHXyDrolClbJb6h+aYtMSR87SAXEjUENahAjktc6qbW+MJiMPLXuIV9e/SoPeexY8x0PFn1YLGc196Ldsafu8RqMhKSlJ1bjy8qyXefvMt93qddOantN6MuCcAap6/aWPT2fsbWPRaK3P3VMVfyDs9/r3E+e0fbu5t1prARVElQWILGf5uxUern4Cl5YmqrhqaswTe/l3bKz4TH2FrcruuqY6rvnxGka8PYK6pjrbO3qQA78d4Mszv6T0QCmSJDH3m7mc88k5dre3zKp0d/x3Z9I1ahTccot7QV+5oqyw0Jy552ubTxnLybOjccVkMpFbmcvKIyvb+QytkYU/KUA0PwjWuZ+a2q1VUVx79DKIjRVZlnJmrjz+OX1tkwlKN0HFbucvkvMTfBMGuT8pPq+QgBD2/3s/NffXWFV3ysH3ykrztl0ndaXbyd3arX9Byf4SFly2gKylWW2eq6sDyaTl2t6PcMOoG1pEBHeQJLjoIjjH/jDlcWbMEAt8uVI8OlqFnWz2Qlg+E8q3t32ubAsc/Q5q1Qt3IKq7QnQhLvVN1Gg0pKZGIUkSRb9u5PNZn2M0uG9lkDQ0iVHXjfKq3axs8+ko07pDoLNxE81bAntfgjrX1gYB2gAuHHwh49PHo9HXgKG+7UYmA+T/DtVt1aSooCj6xfVjbNpYKhvMA0tAgDkxUl4rGQ1GltyyhPWvrFd0bt0nd+fsj89WJPi4QnU11AeK5o/pUX6QTeUB/KniTwn25itD5w1l1HWj0AXpSBuTxskPOLd9NhjMcws1c//2rOLpRJAWmca4tHGYMPHtrm99fTou06Bv4FCZ+LL1i+/nseNO6T6F+NB4AjS2LR07sY+S2EprkcfQaGDHFzuor6jn8F+HOfj7QZt2mvJYYcsW2NP8I7osMWyYiEsEBpoLBvbuhX5z+nHr4VtbRD/wH0tHea3hD71mW5O9Npv8bflt1jyeFv7kdhIHDpj7+6WleS9ZODDQ/Llb2n0ajebCIF9fF63xZ0tYsE5cV60BeYBO4c/HGAwulNxX7oes920vphyQHN4s/FU5F/4s+z9ZfnkMjQaKM4tpqnPdxknu2yDbubjCcWP1eWwJbLyJU3r+ADjx+DbqYc9zsPvZlod2W8T41Ap/clZtTmWOuh09TEltCY+teoxbfr2lpd+WpzGZzJOqjlzxB6L/HAi7z9YYDAaysrJcG1e8gCcCv54U/sBs07Z4sVjUR0XZnrjExgobR5MJ/vxTPKbW5hPE5FquppPtPuXvua8nTPI4byn8BeuCWbRvEVvyt7C9wEag2MPUFtVy4NcDlOwvcb4x5grQwMAOXGnRTESEWeiT+3b6i/Bn6ZPvaFw5VH6ItBfTmPHpDPRGfZvn24sW4U/neatPEAsrm9X4XkBOOvjrL/E7LEyBxY4kwRl7YLJjq3UAwntC0gwIjFZ1Xr1je7cReGwJf9OemsaZ75zZbtmM5YfK2fH5DsoPlbd5rrYWgvRd+HfGf3ntdBu9xjoIYWEwfbr536psPlNmwajXRfVVawY/BBfWQngPl86rV0wvau6vYct1NjKRnGAwGKioEH226rOLKM4spjK70vmObjJv4TyuW3QdBdWO+5E7Iqd52uyO1bNPyV4IOT/a7hEw6AE4OxdCPSBc/ZAm+tK3JrI/XFgPA+9p85QkSWTenMnaa9bSJdx6wi5nl8sCuEarYfun29n7o3esO9VSWQn1Ae6vcf0JWfjLyTHbXvuSBn0DW/K22G1T4WwdlLclj4qjyqLH8voD1CWayddpa+HPaDKyNGspmcWZGE0e6NfQiRWXZlwKwEfbPvLtibhBVlkWBpOBiMCIlridJ/jgrA94fufzXL3wao8d80RBSWxFjkvKa6eDfx5kwaUL2PD6Bi5dcilXrboKja5tjKu9kgRMJlixQvw9dqz5cUu7z4CQAKK7RVvt5y+VXfL6q9L7U0TV/HrLr3w247M2j3tL+MvNhf37xd/eTj6z1eevtNRcLarYeaSdsIxdtFPuqSosXcx8EavtFP46Ike+hPXXQsUuVbvJFX/51flOtw0IMH95LO0+17+6ntcHvE7eprbiYc66HF7t+yo7v7bfAxA8I/wdN1afiSfDyT+RLZ0HWDetb4OkhYMfwoG3W0azXRaXgOqKv2brLvnz8BWFNUKRiw2JJUDrnUy0sjJhD6TRtE/VhjcZ2tyiaPduaLBRIFnlQn273qhnbfZa9hTtcfPsBEdWHeHZhGfZ+uFWRdsbmgw8n/w8S/+z1Opxk8kc0He3uuvcr89lyJtDqIwXKp48ienb13621JDmFqSrmuMLrgh/0DZjWp7g+3oibVnZLSNJEqNSRgF4tO+jPQaeP5D7a+6nz6w+rHpyFSsfW+lQMPZEtfegNwYx4YMJLvWl8jSynaQs/KkK6HuR1g2y7Y0r3aO7E6AJoMHQwLEqGx3A2wl5nEDnuYq/hASz4BYT037VubLwt7m5/ZaqcUJJNnf0YJi8CJKmqT631vhDI/VeM3pxf+39DLm8bc/o+ub8uBD3C/18zpw55nuVqnEiagD0vdG20BsYBVrXvyuSJLkp8IrJfNPJ07hh+w0OWwko5bOZn/HJtE9sPqc36vls+2e8s/kdt15DFv78wSLbJXb8FzbebHvyE9ZVuIx4gu6XQvLMto9Lkktp6vL83XKtdFPmTVz2+2WK9v927resfna16tdVSlWVueJPXmN1dOLixDhvMJgtbn3J0oNLGfHOCG5cfKPdbezNV6ryqnhnxDu8NewtRa8lzzeDg0USn1LsWX0WVBcw47MZDHpjUKfw5wUuzriYQG0gW/O3sjV/q69PxyX2Foskhn7x/fzaXv9Ew1lsRX5ajkumnpTK7Ldmk3FJBpIkEZ5kO2DZXsLfvn2iV1tQEIwbZ368X3NR6d7m3JnGmkZ2f7ebYxvFes4fhT+jnw2dJz90MtOentbm++pp4S8qSnwOliJuewl/eRZhf/maiItT4TzSTshrUr3edozU13iqfZGr+NnH1Ykiul4AJ/8A4b1V7ZYUJqLWSqw+wbYNXLdJ3Zhwz4Q2fbhABO91wTq0gY6jY1cPu5onTn2CU7qfouzEbXDcWH2GJEPamYTFijuqw4o/SYJJP8DsnSBJNDWZe5SBEBX0KgouZOHV1xV/BTUi67pLmPdK8eT+fvHxvrVW9AQpKWKiqNdbC7/ucNfvdzH+g/G89o9nKiJC40OJ7x9PUJSyqpuGygZiesUQFGm9vaWto7sVf9sLtrOjcAc9ugZYxZts9feTkYU/efKgtr+fjNyQeU+zruov1hm2xnjALPzleV/4s7xnbHl/C7u+2eVwsevuRLq8vpzdRbtZk73GyrLQV8hVZXKCjb9U/Cm1y9BIGrpFi/+EbE/kC8xWn56r+JMk8+fTngkjsvAnJwMqWnAXLBeWjSqdINTw876fuebHa/hkm1lUkSv+LGMiNYU1LJq/iO2feb9iWMZWfw0Q10VRxB+UGbLbzXrUWyQlmTO1VV2PjgLMJiOUbISSDW6dm6uEhopzq2kMaLn/N1Q2uPVZBUcH27X5LK4txoQJjaQhPlRdJGvZMvj2W7Fwl60+O6zwN/5zGPOB/edrc6DMhq2EWk56Awbcafu5otVw7FdVh5Pvj5YJqeFdwttYxNtCX6/nwK8HKNjmeqWnM/LLKtFrxWAou6p0dCTJv+w+MxJF5XJmcSaNBnUliBHJEZz53plc8sslirZ3db4p37Nbr+nlViepEanoXOip2oljYkNiOavfWQB8tPUj356Mi8j9/frFec7mU2bHFzvY+ZXj5PxOXEOeA8vCQ2h8KKOuG+U0mam9hL9ly8Tv8eOtHURk4S8rS1R01xTU8O3cb9n0ziar8/J1vEJ+X00m/+vf1m9OP4ZdOazN495wp5Nb08ifS+te9J5GjntZVvxZCn/+RlCQOUnH3+w+TSbfC3+ds46OSFR/8aMSNT3+QASFjx61Dgqnjk4ldbTtxUy3Sd24YfsNTo87s/dMZva2kQGqAnnQ8ZfqCJcx1IM2uGXwdCj8AUSam5Ht3y+q2KKjxe+aGvF5yX7dzvCXij/ZbikxzHvN946H/n4ykiT82ZcuFVVCI0a4f8wp3afw8vqXWXpwqfONFZAwIIGrVl2lePvQuFCu/rut/YhssyN70buK3qjnSIVo6Dc4pRdduzru7yeT0coZzVXhTxYQd+0Sgq2/WH06Ff7aoeIPoHBnIflb87lx141U5zmeqbk7aZLFqYTQBCKCIlw7iAdp3UfOX4Q/NZVc3aO7c6D0AIfLD3MKrif0uIM8VlyW/Djf3vSwx47brZvIhG3PzyUhQdyr5PuWosXVwY/g0Ccwt1JZBdehz4TV34QvRe9oBWzL38YHWz/AhIl5Q+cB5uvE0n5HF6xj87ubMRlNDLmsbRWepynZV0J9RT1Jw5LQBpgze0wmqGqoZv2QGYz52kT+nfltrAM7GtdeK5Jg5thwT7RJQwn80BUG3AVDHrWxgQR/Toa4sTD1D5fO6am/n+Lb3d9y65hbW64LpYSECOFPXqBv+2QbP1/3M1csv4K0sa6paud/fb7d5+T5ZnxoPFqN8iwwoxFefVXcvxcsMN+HOqzVZ3SGbetXmT9OAW2ISDT0FptuhfoiOPtIm6fe2PAGz6x+hrkD5/LsDHN7A1tzltriWop2F5E8IpnAcPsTRV2wjnsr70Vf7z1L6iNlYj0VTCThgR3dksZMjx6iL7Y/CH9do7oSERhBVWMV+0r2MThxsPOdLBhxjfJFk6vuQvLcvqpKVJ3LgXZZ+EuP6ujNQf2Xm0ffzMjkkVw2RFkVsL8xf+R8xqWPIyLQs+uTkz88mWH/GUZabBqDL1L3nenEOZYiT9WxKgJCAxT1GZbjUsXFYs7qjSJPvd7sXHTqqdbPdeki4ojl5UL8GzAghrM/OZvup3RvOS/wfcWfTife2+pqsd5or9YLzjAZTUga2x+aN4S/3r1h7Vrzv72dfGbL6tNfxGBbSJJ4v8vLxfvv6+vWksZGc0JvWJjZkaY9caniT6/X88cff/D222+3lD4fO3aMan+TVjsAbpXxN6kzOj4p5SRuPulm5vRVFjGwlVnpL8jZth120S3zc3/4Y7Ld7ECb1ByB3U+zZ6cocRg0yFxRpMbuMzUyldCAUGKCY3zam0m2+vRmQE6u+Ovo/f1khg0Tv2V7QBlJkkhPT1c9rkzpMQWtpGV/6X6fVu20xlP9/Y5WHEVv1BOsCyY5Irmlzx84Fv6ioqyzqVy9frp3F8eqrxcigr9MpG31+ANxrwDYXbSb6kbv39fXPLuGhZcvpKmmyWl2pLsT6UPl4vruGaMwQ8LL+LvwV13tfFzpHtUdgMPlh9vn5GwgV/yFhmgICQghJMAz3o7yWNvP88nXDhlsEZdRNE4MuAsmfgsBCoNF5dsgZwHUHlV8TnKwMrvS7Pdmq8dfUGQQdxXcxZnvnqn42O6w+tnVvDf6PRqrrKs/9HooD9wBkokuYUkdXvQDESS67ba244ZdmqogYQKE2pksSxKMeAH63+7yOeVW5rI5b3NLpYJSJEmiZ0/xmcj3+sSMRFJOSsFo8I6Xk6sOEzU1ZkeN6moRoAsN7aDJh7XHnFcGD7gb+t3q3uuYTLDmctj/pu3nMx6FUa/afMpgNHCk4kjL/VrGlvC39aOtfHTKRxRsd17JJ0mSzcpgT9FYG0y3wusZG3ap117DF8gVf3JfIV8iSVKL2LejYIfN511ZB9nCVXeh0FCzvbTlul6+d3aN8nKZxgnMyd1O5p6J93TYitu40Dgmd5/MyJSRHj1uvb6ehWctpNvjSicPncgoGVMsK/7+euQvnk18lspc53Ha2FgxDWtqggplrUdVs3GjOL/YWHMisowkwcCB4m/ZRWro5UOJ6hqFyeQ/8Qowi32W7Ul8zR/3/sFL3V+y+Vm3rgL1BHKfPxBiqKstaJRiKfzJRhz+dE3YQo4P+VtlqJxIJEkiGcgXVs6qhb8jR46QkZHBWWedxU033URRs+z79NNPc9ddd3n8BI93NK6a4646H35Id2zh04px6eN49fRXuXzo5Yq2t1eFtmj+Ir6d+22b7ff9so9/Xv8HfYN9Eam2qZYl+5fYnKwrpb7eLEameKgNhU8wmaDLVEiYpE74y14IW+/FcPQXQAh/8o1AjfAXFxJH9X3VZN6c6VPLETkQkxjaWfGnFLnP36FD1hMgjUZDXFyc6nElMiiSsWnCQ8xTVX+b3tnEysdWKtr2wG8H+Pvpv6krrbN6XA4GulsSn1WaBQixRyNpWppZd+liDlzbw3KS7GrFnySZj7Ntm//1+Gst/CVHJNM1qitGk5H1Oeu9fh6jbhjF+P+Mpziz2KnFm7sVfwfLDgL+I/y1tunwl0CyZcWfs3GlR4yICh6uONxOZ9eWFuHPzSSB1kyaBB9+COfbLyLyCqqFv+gM6KriJAc9ABfUQLjy74HsEmBpD25L+AMISwxrt0XN4AsHM/XJqW2yq+vqoDJU2BUOSxrWLufid4R3h1N/h97z7W/Tez6kznb5JWwJwkrQaDSkpcUgSVLLuJ48PJmrVl5F1wmuBcarjlXx16N/cWyT7X6jrjpMWPb6uu020Vdl5kzvZOd7nX+ug+8TwNhkf5s+10Pvf7n3OiY9HP4MCv+2/XzqbEiznYgqB+1bO5LYSkjtOa0nM1+cSWS648lc/rZ8Dvx2gKY6B/9vNwmu60HG0Te5qfsbXnsNXzBggLjW9+wRbiO+Rrb73FnYtiLV1XWQLVxNNJMk23afcsVf10jXxrcO7lbdiQ/RarTkpOcQPNj9/tcnGkrGFEuRp/uU7gy7chiRqU4CDAjxRl73ecvuc/ly8fuUU2z3ZJOFv927zY81VjeSva2UxuZ8Ol/HK8As/HlLIHWF0IRQwhLC2vRwNJnM6yJncSY1WAp/qaneb1/UpYu4n9XXm+ON/i78+UPveVtYxq8kyQ0NyA1Uv+Ktt97KqFGjKCsrIyTEnFF9zjnn8Oeff3r05E4EDHLNp1rix0P6eaCv9ewJWSBnEeTnWz9edayKsoNlbQK02z7expKbl9gteQbYV7KP0784nWmfTnP5vORy48hIz5ZPtzuSBGPfh6H/1zJ4lpQomNh3vwTjpEX8sFEsmAcOdE34kyTJLxpHy4GYzoo/5URFmS1dt1u0UTIYDGRmZro0rszoNQOA37N+98QpsvPLnax9fq3zDYHMhZn8ee+fGPXWiQzyTdLdYH5WmRD+esWI0tiJE2HqVLi6rbtoG2S7T53OPT9zWazdvNkstPlaiJYXGxUVbZtlT+w6kb5xfalpqvH6eaSNTePIiiN8eeaXTrd1t+LP34S/4GDrccnfKv6qqpyPK92juwO+rfiTJ/i/lr7GvIXzWJrluehkfHz7B/gthT9F445R5ZgfGK3MEtSC9MhmgafC3C/P3gKrpqiG/Uv2U1Pk/fGj57SeTLx3Ypu5Z10dVIRsBWD4iSr8tQOW14UaDAYDOTl7MZlMNDbSEmCScaXPX+GuQlY8soLcf2xb2LdU/Kmcb1red6ZOhTfeUDZ/8EuSThXCnsZ7lW8ASDq4sE6sc1TSLUpUpbS+p9hKVkoalsTY28YSle7Y+2vT25v4/LTPqS/3nr+SHOjzZIa/P5CSApc2FzG+8YZwrvAlGV3ExHxHYdskYnfWQa1xteIPbPfuahH+XKj4a2qCBx+EG27wjUVYR8JkMvHlji+Z9fksimuVZFT7ByW1Jdyz9B4+3vqxx48tJ3g3GbyX+HC8omRMsZwjZFycwZnvKHe88Gafv+pq+Ocf8Xdrm08ZWfjbs0fEII16Iy+kvcDiGxYBwh0owMvTBSXIwl/rRENfMuHuCfxrw7/a9BmWnSHAs8JfZKQ5dpTeDo7RAQHm18tunuL7S7sae8jxIX8zomztYuaJOYpaVAt/q1at4sEHHySwVcOl7t27k5tre6HViRcYcAeM/QAC1EU/i2uL2VGwQ1FDbDkY2Vr4u+jHi5i/aX4b0WjaU9O4csWVVj1WWpNbKa6R1AjXLRiOG5tPC+Rgb2Nj24Hq2LFWk/zgRA43nUFVTSAhIcKCRRb+Dh0y2xF1FP435X+svnq1V/34j7eKPzBb0G3ZYv14vYsrQln4+/PQnxjUBpFtMOeDOdy4+0ZF25762Klcs+4aQhOsFT5vVPyBaP57222iybUzRoyA/v3htNNsZ8opRRb+9u4Vk8GAAM9OBl0hKkoIGiZT2wy6j8/+mL0372VOP6XNpNxj8iOTmfHCDKfJCMdbxR+Ybfs0Gv/pW9DaKsPRuDIyeST3T7yfG0Y57/HrLeSF4I6qFXy6/VP2lezz2bl4gqQkMcfR6RQs7oxN8HUwrLtG3YuUbrZfjWMDuQqnpqmGigYxYMhjWOvx49CyQ3xx+hesfnq1unPyIKLibytwAlf87XkBdj3peJu834WDyNHvXXoJVyv+ACSprkVUl+/3RoORb+d+y4JLFqg+XtcJXblu63UMOHeAzeflILBaq095Xn5cCDr9b4fhzzrepjYXfh0FOx9z/XUkSSQX2EswOPQpfB8PhavaPCXfn/Or86ltMie4ysJfebn66qfh1wxn9puz22Tme5L8mnyatBWEhx9/pVkXXABjx4o15pNPtnWKaE9arD5tCH/g+jqoNa72+AMnFX8uCH+ffSYSPXNyYNMm9edzIiFJEv/967/8euBXtuVv8/XpKGZX0S6eWfMMj66w1Y/XPbSSlvlvz2ffaR17buwrnI0p8hwhJEC9sOpN4W/FCjFmd+8ufmzRo4eIi1RXi/FFo9Mw9raxxJ08yOr8fI0/VvzZQ16ThoaKdZwnkVvUKLb8d5O+fcVvuSK0s+LPNbzR81EtqsOYRqPRpkKZk5NDxHGxIjq+6fVKL4a8NaQl+OkIueKvsNB6gdU6q0EmpmcM3U52PArJti3ueK/LFX8d2uYTIP9PWD8fKvcSGGi+oVkKrdu3w/XXw9NPW++6a6eJ1Ii9jMnIQaMRn1VYmLi5H1Xesofn1jzHiLdH8M6md9z//7hIamQq49PHt1SNeBqTySz8ea3ir3Al/DYWGkq89AJtsezz5wn7l1Epo4gOjqa8vpwNxza4fbyYHjFEJCu7J4TGh5I2Jq2N6CMHAkPcbNcVFxrHoIRBDEwYqHrf4GB49lm47jr3ziEpyVp49kUVUWsshabWQZz2tv/tfVpvhl0xzOl27k6cksKT6B7dvaX60x+QJ+/R0b6/JmTk6VxNTdtq0Nb0i+/H41Mf54JBF3j/xOwgT/CNklicB+s6tp2RJMH//R8895yCRbehDtLOEnafalh5Fmy6RfHmoQGhxIWI8kO5ukvuE1pZaX0fGnTBIE5//XQmPzpZ3Tm5wDsj32HxzYvbPF5dY6AqRASGT1jh78DbcOgTx9sExUFQIkiueQbJFX85lTkYVbQfAHGdh4aKC0cOsmu0GhoqG2iobMBkVDe5CQgNIGloEuFdbN8gnpz6JBX3VvDwKQ+rOq48vnRolxE1BCVAfRG40//b2ARFa6H6sJ3XiIeoQaIysBUxITFEB0cD1lV/svCn11sHdT48+UO+v8SxcJ0yMoVR14/yqtPJYs18fhsezdLS97z2Gr5CkuD22yEtTbjTfPGF785lWNIwnpn2DK/Nes2rr+POfNNWMP8/E/7Do5MfZUiXITb3aWy0Pd/avh0WLjT/e4P7S7Tjnr5xIlq9v9QPGlMqZG+xKKXtF+/5ptI6jY5DPQ4ROsHDfvidAM1tEfSNrD73BZbeo85xxFvCn8kEv/4q/p4xw/52Op25j7nc52/yI5MJPXkU4D8Cj7/1+Mtem82SW5ZQnNm2qtgbNp8yl18OZ5wBs1136FeF7Hy1Y4eYe8nxIn8RhFsjxy/8veLPF6gW/mbMmMFLL73U8m9Jkqiurua///0vp59+uifPrRNnbHsA1qvrwZAcLppU5VXlOd02Pl4EhhsbrYPCJpOJ7Z9tZ9sn1llUdaV1ThfpnRV/FpT8A1nvgl6MTHLfsd9+M2/y9dfixr1xo/WEoHj/Zt46oz+n9RaCnSRBr+Y4thq7z4LqArbkbyGzONOd/4lfU1YmblQajXtWjQ7Z9xqUrIfK9sukGzRIVI0VF5vFcHfQaXS8Nfst1ly9hlEpo9w+nr5BT+mBUqe2SiaTidKsUpu9Qd2t7pK5d+K97LxxJ/NHOuhz1A7IVX/gPxMme33+ZPRGPXVNdbaf9AHuZGADfHT2Rxy69RCndD/FcyflJnIWpr8srsD8/ppM5vfcX9Hrzeeobxb+gnRBPjwjz5CQYL6vOyQgEiZ9B/1vU/cCQ5+EjP+q2kXu85dXLeaQ8qJWrzcvakCsDU668SQCw4Q7SHW+d1ZgJqMJo8GI0dA2Wrq/9AAGbS06Uwi9Y3vb2PsE4LRNMPkXx9vEjoRZmyD9bJdeIiUiBQmJRkMjRTXqI1fy/d1ykX7xzxdzyS+XOGwdYIuqvCoaqhrsPi9JEpFBkS2iklL8IVPXI1Tshr9mw7EljrfTBsLZR2DII66/VkMpLB0Pe+xUF6bMgmkrIGGczad7RIvesZaJqjqdOahj2edPF6wDB8tPk8nkknWsWioR/U/To11sCO0PlGyAo9/afCo01Gxxu82HhVTRwdHcPeFuZvf1buTT01afFwy6gIdPeZhu0W2TpPfvh0sugYsvhkcfhe++g9WrYedOeOEFMReTKz02buzs9+eMPrHizdpf0nGEPzkW0y/O88KfVqNl6YylxN3nrUDIiU11Negaa0kYlkZMD3XN2r0l/GVmwuHDEBgIU6Y43tZWn7/iYggryyaqvsCzJ+Yi/mb1eWTFEf559R+bcS5vCn+pqSIZvb3qrWThb88e0TrJZBJzMV+7VtmjtWORv+CpmKY7qBb+nn/+eVavXs3AgQOpr6/nkksuabH5fLp1WVInTnGrsWPJRmHRoyLDNjmiWfirdi786XTmm5FlFZokSfz1yF+setxsz2I0GHkm/hm+u/A7h8dsqfhzQ/iTRY4OL/wNvBfOLYJokfl3zjni4T//FAvarCzr/m0rVojf5eXwy+ohLMy8g+j+U1ued6XPnxzEy6nMcfV/4RYmk4mHlz/MK+tfoabRc9Hl8nJ47z0x4ZH7+8XHe7EJ7qjXYNZWiB3upRdoS2AgDGh2s9q6VfzWaDT07NnT5XHlwsEXMi59nEeqvQ4sOcCrfV5l70+Om4HUl9fzau9X+eXGtsFJf7hJehK5ShP8R+SxtM5qzf1/3k/M0zG8t9l/stfdCcT4K2PHwumniyw+f0GnE9WuALW1zseV3Mpc/jr8l6KkIk8jXxOSBHqTCPp39Iq/dqHHZaJSUAWLL11Mzf01LdbQgYHmimx79jtbP9rKq31epanO871lJI3E9Vuv54w3z2jznKYxiiGH3meG/nW0Gm/d/P2cgHAI966tcYA2gN6xvekX16/FAlYJ8nwlLEyIe5bCsaOWAY5YePlCXkh5weMiz3Ej/NVkQ/4f0NAOfa90YTD8eUg/16XdT0o5ifHp4wlo1YtQbo1gOWe5/PfLOe/L8+weq/xwOU9GPMnqZ7xnPWwyQY1WrHF7J6Z57XW8Stl2+G20sIA22BbQBw4U99q8PN/afdrD3XWQJe58721ZfdrDYIDXXoOGBjEObtwIH38MTz0F990nKixTU4ULQEiIuNfu7zh6lk/oE9cs/HWkir8SsV7uH9/f48ce2mUop3Q7hcSw46jnSTvhbEwxmYTI0BgazVnfXsqo69UlT8tuQJ4W/uRqv0mTnI9hcjzJUvgrPFpPn/WfU/XBtzaT69obf7P6nHDPBG45eAtJw5PaPOdN4a+9SU0VsaKmJvi7uTuEP7hW2cNfK/5axzQ9MUdRi+pXTEtLY9u2bdx///3cfvvtDB8+nKeeeootW7aQeDw10Gon3LIcmfQtnHUYJOUfo+wr37phuj1ku8/Wff7O+eQcLl1yacu/DQ0GRvxrBD2m9XB4PFlgctXq02QS/tNwHFh9ShIEx0PzonbQIHHj1evhhx9gQXN7E/lGJwt/v/8ODU0BrK59ntTh5qoVORNQjfAnfw6yINveVDVW8X8r/49bf73Vo8ddsgR+/FEsmOSmxl4dnoITIWYoLgypbjG8WWeU+/xJkkRkZKRXrYyUkpiRyPi7xxM/wLHCZTKaGHfXOPrM6tPmOTmw487EyWgytkumtxIsK/78TfizzJ6XCQ0IpbqxmtXZvuvT1Rp3AjH+ch20JjAQbrjBWhj2B+TvXXW183Hl6p+uZsrHU/j1wK/tdHZm5Ky+sDBoMDRX/Gk7fsWfYgqWw8ZboSrLtf1VfC9SIlIIDbD2KXG2GK8tqSUxI5GawvYtHQ1oSKJrydVMCL2qXV/Xb6gvgvIdoFdQsZ29ALbc4/JL7fv3PjJvzmyxV1OCPF8JDxfjSuvK4swfM/n+4u8xNCrvOdz3zL6cdNNJdseqK3+4kvmL5qtOUDhuhL+UmXBhHXS7yPm2FZmw4/+g0sXAeUC46EefNNX284ZGcfzDX9p8+u0z32b11auZ2Xum1eOO5iz2MDQYSB+XTkSK91LkSysaaAwQfQX6dOmgmakxQyDjf9D3RrBzDw0LM9uT79nTjufWitzKXL7b/R3LDi2zetyT6yB3HCbkxOniYnGLPVZ1jKVZS222Wvn5Zzh4ULzOE0/AtdeKYP2AAeI4CQlw993ivR8xQuzTaffpmJaKvw4o/Hmj4u+Z6c/wZOmTaN87QZOg3MDZmNLYKEQRcK0KyxsVf1VVZpFm1izn2/fvL8KSBQXme2tJTTBHhswh45Fz7bZ5ak/8zepTkiRiesSgC2qbLH88CX+SZK76W75c/PaXGJYt/LXiT05ulIU/X8RqXfoW63Q6LrvsMp555hneeOMNrr32WkLcbcJ0gmKrX6JiAiJVy+29Y0RZ2IFSZeqQPeEvfXw6MT3NpewBoQGc+faZjLrOcZaLuxV/VVXmiXhyB3ZSAURD+2rrBcDcueL3kiXmG/a994rqi8OHxcJgSbNDT4u3c3PQrmdzUveRI8rjeL6u+CusEYvksIAwwgI9V8Jz+LD4XV0t7FLAi/39ABor4Pt4+EvB7MqDyELSxo1CKNbrDezYscOtceWrnV9x8+Kb3b4mYnvFMv2Z6aSe5Pi7HhoXyoxnZzDw/Lb99+TJndxHyhXWZq8l8qlIZn/RTmboDoiKMts6+ovVpyPrjAnpEwD4++jffiOauVMF+tHWj0h5PoU7frvDsyd1nCJPnsvLnY8r3aO6A8qTijyJLDhFRkJDc5XC8WD1qZiiNbDvFWhSmQZbvgt+SLdvxacQZ8Lf+DvHc82aa4juFu3W69ii7GAZ619dT8n+tv115UXfCdt+PGchLB4ChSucb5u7CPY8A43tV8JjMIhxJSREZJK3vgflb81n59c7KdqtPBo29taxTHtqms3njCYjn23/jHc3v6u6F+FxI/yBSBZtVUVnk8o9sONhKPrbO+eh0cHO/8ER28KfPWwJfw2VDfz99N9k/mC7bUF8/3guX3o5Qy6z3VvNE2QVCjFZYwwkOcqPo2K2yPvdLPBmPASR/cBosLuYtGUL194szFzI3G/n8sr6V6wel8cVt+IrzbjjMCEHRuvrxXF+z/qdGZ/N4IZfbrDarrgYPvtM/H3VVSLAetZZ8J//wDPPwAcfiB/Z9vukk8TvjRtd+A+dQMgVfwfLDmIwun8teJsGfUOLKOyNij+AzIWZbP1oq1eOfTzjbEypqoLEQ+vpu/5TGgrVl6PJ8YDyciEimkxi3GjNwYPm4gdnLFsmjtWjB/RVkI8VGiq2BfO4XlQEZSkD6TlBWaXFsU3HqC4Qg2ZtcS1VxzyrvPiT1WfhzkL2LNhjt8XV8ST8gVn4k9tt+UsMyxbye+4P14klrecTnpijqEW1n9snnzhuEj9v3jyXT6YTF8hbCvoqxVYqcp8TpRlQ9oQ/gLqyOurL64npEYNRb0Sjc64j//eU/3Kg9ACDEwcrev3WyANOfDwEdeS4nskEy2dA4hSYsrjl4VGjhDAgC1fDhsHgwWKiv3YtvPiiWCRERcHEwdth0fnQ71boexNdugiBsLFRbKNkUJYF2GNVxzCajGhUVI96goJq4cPpaeuJ7GzxOzzcPNB6VfjbdAs0lECE8mx3T9C7t/BtX74cPvwQNm6UOO009475/Nrn2XhsIyd3O5kLBl3gmRN1ETmQHBPjeDtHZJVlUd1Y7Td96q65BpYuFdm8/oA8QbIVtB+dOhqdRkduVS5HK47a7EvS3rgTgM0qyyKvOs9vrgV/x9IuIzDQ8QS1e3R3AA5XHPbuSdnAUuCp14vV8gll9Tngbuh5BQSpvI+GpkBQAgREKd5lW/42Xln/ColhiTw57UnAnJjhiyzc3A25/HrLr5z/9fnE9bHuXbOrZAuFkYU0hQ4E0tv/5HxNzAgY/DBEK5hvZzwCGY+quhY8gcFgaLFubG0bOPrm0Yy9dSzB0Z75LpfWlWIwiXFM7ZzzuBH+CpYDEnSZ7HzbLlNg5gaIapuUpYjqw7DqXOj9L+hzQ9vnJQ3M2gzBjifnBqPByqrX1vWi0WlYdv8y+p/dn/5neydo7oysQrFADTWk+oXrhmJMRlg7D4LiYPYu8djR72DjzTB5ic0WBgMHwuLFvhX+BsQLb7o9xW3LDj0VUHPnex8YKObXlZViTX604igAXSO7Wm339tsiyD9gAEyf7vy4I0eK3wcOCPFb/j50Yk16ZDqB2kAaDY3kVOb4xfrFEVllWRhNRiICI0gKb2sd6Aku+vEiu0JFJ45xNKZUV0NgXSWhlfmEJYba3c4e4eEiptnQIJL333tPfL9fecXc1mj/frjzThE+7NoVJk4ULSKibEzZTCazzeesWcprRAYOFOLi7t0wfrywGAYRT8z8IZM9C/Zw9sdn27y/VR2r4qNTPiJpWBLz/pjHR5M/wmQ0MX/TfAJCFCQaKcBS0DGZfGs1uerxVez8aifXb7+eLhlt5zDHq/An488Vf3Lswt+EP7niL1T9EOExVAt/t95qbcnX1NREbW0tgYGBhIaGdgp/7c2mfwu7FIXCn5wB5W7Fn75Bz/NJz9NrRi8uXHghbw9/m35n9ePUx051eLzzB56v6HXtcdz09zPpYcSLEGw9uZMkOP98eO458e/zmltWTJ4shD9ZEJw5EwIi08SCrVms02rF55WTIwRSJcJfUngSEhJ6o57CmkKvTTbtIVf8dQn3nCqn15uvk//7P/Fe5uaaK628QuJk0ASJXn/tiCTB7bcLcfidd2D7domCgkQmTHD9mGNSx7Dx2EbW5axzW/j7+YafqSupY+43c+1us/6V9Rxadog5788hNM76buiJir+sUmF/1yuml+sH8SDDhvmXpaOjzKiwwDCGJw1nw7EN/H30b58vnJuaRGIDuJaBLWfT9ozxbs+r4wXLybOz4FKPGJEq6ouKP1n4i4yEjf/aSL2+nqjg9hUwfIo2EEJd6CkVGCMC7yooqSvhg60f0D++f4vwpyS7cudXO8lZn8NpL7qZmdKKntN6csXyK0gY2HbC82vZK2zo+xF9mp4E7vXo63YI4kaJHyWEuTe2/7T3Jx756xFGp47mrTPeUrWvPLa0tm5sPR9wRuGuQn6/83dG/3s0fWe3TcKSE81iQ2IJ0KoLRB03wt+We6DuGJyjoGwgMFr59WMLY5OoQjbYKF2Qic6w+1RuZS4TP5xIaV0p5feUtwQb5UQwS+EvIDSAa9dfS1y/OBtHgqX3LCW6WzQn3XiS6v+GUg6Vivc0kg7W389kgjHvic9LJjhJJAE02K62lSv+srKEaBXsgzybAQlC+DtQeoAGfYPHq/xNJvesPkEER1sLf+lR5iSUvXth3Tqxfr/pJmWB7OhoUcGzbx9s2qRMLDwR0Wq0bL9+O2mRaR51FPIW/eP7c/S2oxyrOuaVxIHrf76e73Z/x+OnPs518dd5/PgnMlVVkDNwOqYpUwgIUR1aR5JEzC4nBx591JyIu3w5XHaZ+HvVKnMB9tGj8MUX8NtvcM895v58Mps3i2MFB8Mpp6CYAQOE7fDatTBtmug9Kkninrtp5RF2fbOLSfdPIr6/WfUxmUxIkkRESgSTH51M+rh0dME6hl8jEkY8JfqBea0h91T0pag286WZ9JzR06boB8ef8JeSIubq8jzdn4U/+T2vqvK9QGyJu/MJT6C6vKesrMzqp7q6mr179zJx4kS+/FKdXUcnHmDkKzBB+fveJ7YPN4y6gf+M/48i6za5SqqgwPpxXZCOcXeNo/85/anOr0YXrFNU8ecux43wpwmAPtdD+tltnpo4EU4+GWbMMFs5jhpl6QmMqOoKioU5B6wyaeX3Ra6MdEaANoDesb3pF9ePyob2T40oqBEXVpcwzwl/eXlishISIqxRnn8e/vc/GDvWYy/Rll5XwZh3QNP+3vmSJK6V558Xi8cDB4LJcrHVE8DYNPFGrctZ5/a5lR8qp2RfWws2S4r3FrN/8X4CwwOtHjeZPCT8lTULf7H+Ifz5G86C9hO7TgTwiz5/lj2gXMmY6hT+1CFPTmtqnM+aWyr+fCD8WS6wIoIiSAhLIFAb6Hin44nynVBztF1eKj1SBC2zK7Jb5pBKKv4OLDnA+pfWU1fq2Wrb0LhQuk/uTlhi28Behb4YgMRwP16h+gsmE9TmiiotF2g0NLIlfws7Cneo3jc2VlxHtnq2VRytYMeXOxStV6pyqziy8ojda8yd+eZxI/wNfQxGvKB8e0ODGF9cIbIPzMmC/rfb36apGqoO2LSUTAhL4Ej5ESobKluSBMG28AeQMiqFoIi24o+h0cCmtzaxb9E+l/4bSok2dadb4fUM0Mzx6ut4HI0WUs+A9HPMjyVMgDMyIXmGzV0SEkTgz2gU4hWIvuovvijWX+1BcngyUUFRGE1Gr/Rxq68X/z9wLdEMrHt37S8UVjTJoeaKv2+/Fb+nTDH3TVSCbPcp97DvxDb94vt1CNEPQCNpSI9KZ0zaGK8cv7qxGn2ensr9flYGcxwgr0Gi4tSLfjLyWFFRAZrmcOrateK3yQTr14u/b74ZbrtNxPtKSuC++8TYK99CTSaQw/GzZqlbK48aJc6juBgeflg8FhcnYksT753IHTl3WIl+m97dxMeTP8aoFwPl+DvHkz5erBHG3T6OcbePU/s2OESnM8/B7LUWaC/Cu4Qz/Kq21fAyx5vwJ0miyEDGn60+5aTlpiZRResvyDEsX1b8eUSp6dOnD0899VSbasBOnKPRuPkRJM+AeOWThKjgKN6Y/Qa3j7tdUUaRXPFXWtr2yzP18akMv3o4kWmRXPvPtUy637F3XV5VHov3Lyaz2HYPBiXIglaKMrvpDolWK5p4//vf5iyFwEBaqrjGjLE/4KoV/gD2/XsfmTdn0jeufW0qwVzx50mrT9nmMz1dvH9hYTB8eDtkfFTuhX+ub7ZSan+6dRPXSFhYOL/84vq4Igt/m/M206B374556ZJLuX7r9Q63mf36bB6oe6BNc+S6OnN1lzvCnyz2+EvFn7/hzDNf7vP31+G/2ueEHGDZ38+VW2en8KcOs9WnRL9+/RzOV2ThL6cyhyZDk93tvIF87Z6wvdxWngUrXAw2F/8DG25SLPikRopJRk1TDRUNYuXtrMcfwJT/m8LdRXcTEuvZfuBNtU12rauqDEL4S4o8QYW/P6fBPzYsFm1hqBP9Hrf+x6WXshSElaLRaOjXrx/x8WJcsSX8rXl+DQsuWUDpfhtPtqLXjF7cX3M/GZfYriJzx1r+uBH+kmdANxVODuv/BYszoMmzvXpa2PRvWNRHtKxoRaA2sKU6Sr53g/0KUaPBSNGeIqrzq60e1wZquTPvTma/5d0+z2nSGDKOvsmM8Lu8+joep8bGd1aSnC6a5Kq/PXtg+3ZhT7dsGeza5YVztIEkSS1Vf3uKzHaf8rjibnxFTmQJDHS9tYi8Vv/mG9h8QCTnrP+9KyaTqNpZv97s9KOGESPE7/Z6rzvp+Gg1Ws5cdCa1/6r19al0OJyNKTmrj9Alaw1RuhqbzytBLrSIjoannhKxwKNHRcFDTo74rdOJwoCpU0WSxaRJItHivffg66/F/tu2iWSMwEA45xy7L2eT0FB4/HFxj5Xn83JlV1hiGKHx1opFWVYZZQfFjz1K9pewf7HnEjOUrDe8SWlWKVs+2IKhyXGGy/Em/AEMsWiR7M8Vf8HB4rsCZkcgf8AyhgUe0IBcwGOvqNPpOCaXY3XSvhj1UFfgfDsXCA83X6CFhW2fb6oTAT5JktAGOq52+uvwX8z+YjbX/+xYCHCELGh1+Iq/fa/DLxlQ0bY3gT3mzYMLLoAbLOM4NUdg812QvwyAtGaHGaXNf32NHIjxZMXf0ebCh/T2bOmz7QHRI+PA21C8th1f2Jo5c8SNZOVK1/st9YrpRVxIHA2GBrYVbHPrfJTalWi0bW9F8vkHB7vXz/NQ+SGgU+yxh6MefwCn9jiVk7udzLUjrsVgbP9GxJa0boyshqqGKopqhW1V57WgDOsef44r6LqEdSFYF4zRZCS7Unnw3xNYLrBu+PkGbvrlJkrrnAsFxw39boe+N7m2b/UB2P8GlCgrHQgNCCUuRNjp5VSKiYaShXhU16g2QQNP8MMVP/BY0GMtGceW1JhEtXlylG37v+MakwkaS6CpXNn2ulAYdD+kKWsb0BpZoDlWdUzVfSIwMLBFyCmxYQ4wdN5Qzv38XMK6KBv0JUmyOZ8Ai4o/F6zlLfuIdlgUVE22IW2OuC5MLtz76wog630od6BOpJwOA/5j99x6RAsLaUvhz17F35GVR3hj4Bvs+KJt1WlAaADR3aJVnb5aOuQ1UnMEfuwKW21YITdVicSB3c/a3HXQIPF7yxZ4zaLLQXuuPwfGC/Vxd5F1s0Fn8xUlyONRXJzriaNycLS4xERdoFicHtzSlSVL4LvvxHPjxqmPaSQni99VVaKqoRPbbMvfxvxF83ngzwd8fSpOuf/P+3lw2YMcKT/ilePrJB07MnagvaT9nYmOBxyNKfkr9pK+ZynhAQ5srZ1wzjlw1lnw9NPCclPuqbZunbnab+hQ4WYF4vfdd8NVV4l/f/45rFkDX30l/n3aaeZ7pRqSk+Gxx8yxActCA6PeyJrn17D2BRHnmvrEVG7cdSNxfe3Psb859xsWzltoc47uCs5iFt5m9dOr+eman8hd77jC4ngU/iz7/Pmz8CdJylpQtDethT9foLom+aeffrL6t8lkIi8vj9dee40J7jSWOkExGt0cCE0m+CFNNF+fukzRLjWNNewv3U+wLpj+8Y6boEuSyEI5eFD0+bMUU2pLank2/lmCIoO4t8J5/5TcKjFIyhnjajGZzFafHb7iz2QSfS8ClPchioqCyy9v9aChATKfFx9U0qkuVfz5kgdPfpDLhlzm0R5/csVf166Ot/MoeUuhLhfOyYNgz/1f1NKnj5GYmBLKyhL49Ve46CL1x5AkiTFpY1i8fzHrc9YzOnW0y+dTnV9N1tIsUkenEt+v7SzB0GRg19e7SB6R3KZHkyz8uTJxlanX15NfLRqU+ro/nb8iT47q60WFZeu1TUxIDCuuXNH+J2YDd6ouZAE4NiT2xOr/5gbmHn8mduzYQUZGBlqt7aCBJEk8furjRARGEB0c3X4niTngGh5u4p0N72A0GXnw5Afb9Rx8Sr+bXd839Qw46wiEKs+USYtMo6SuhOyKbAYnDlacgVuRXUH54XK6TfLcWJw2Ls2u1Xy9JCr+UmP9eIXqLSQJZm1Rt8/Qx1x+uS5hXdBpdOiNevKq80iLdN7nzGg0smPHDrp2zQC0VFSIHs06i5VpysgUUkYqm/Af/PMgJqOJXtNtV/cX1xa3nKsa9Hpxf4QOXvFXsBz+nit6UXe/WNk+Xc8XP65QfQDWXwsjXoLoQXaOP1f82KFnTE9WHFlhs+Kvrs66v1zy8GTG3z2etHHW197Wj7bS7eRuxPR0YzKpgINlh2jSxhAeHgX4SVMZZ5gM0OsaSLTRCEoXBrk/QdQgGHh3m6flir/d1ppbuwp/LRV/xeYkWnlccTRfUYJcURrnRt7IyJGweDGk9S3ll3pRaRXcmMa775ptRNVW+4EYhwIChOhXVgaJnjPNOa4ori3m3c3v0jeuL49PfdzXp2MXk8nEmxvfpLy+nAsGqajIVoFWo2XbsG2cP8XF8fwExtmYops8kcyyfsx24x6TkgLXXmv+97hxsHWr2e4TYHSrcIwkwbnnirHqxx/h2WfNc6hzXcvhAkSs9/HHhYg4x8JMRNJKbHhtA1Fdoxh3xzgkjURQpOPM7FOfOBVtgNZjt0RfV/xNfWIqPaf1pOtE+0FGg8EcrziehL/kZBFX1On8P8EpIkJ8L/yp4q+2udhaFv7c1oBcQLXwd/bZZ1v9W5IkEhISOPXUU3n++ec9dV6dKEWSoPtlot+bQl5Y+wIP//UwVw27ig/O+sDp9klJZuHPkpCYEIZcPoSe05VVT8jZ4akRrgl/JSUiMK3VHgeT3H43uxesk4noA7N3QaQQcGXhr6jIdhDfFt/t/o4nVj3BpK6TeHnWy+6fkwqSI5JJjkj26DF9UvE3cz2Y9KJ3o4855ZQKfvghgcWLxYJSp3qUh7GpY1m8fzF7S/a6dS5Fe4r4Yd4PnPbyaTaFv8qcShZevpCxd4xl5vMzrZ7zRH+/6sZqZveZTUFNQUuVSifWyLaZRqOYILkT5PA27mRLGYwGJnefTGTQcTQL9zLyxF7pxPmOcXd472QcIJ9fWIQeo0lMpIN0bpQJn0gERIofFaRFprGtYJuqij+Ab+d+S+n+Uu4uvltxNbgzxt1hu4dIo15Po1aUBHXz59TU4wStRktqRCpHKo6QXZGtSPiTiYwU8xS9Xtz3bX1c9RX1BEcFOzzOsgeWUXG0gjuP3Wnz+f+b8n/cM+EeDCqr1yx7y/oyU9dtNAEQnQHB7bSAihoIk5e0rE9cQa7OP1huFv5kF4iGBiF6yNVPwdHBTH9mutX+pVml/HjVj4yYP4Iz3z7T5fNQwgulkykffpRTtWsBbzYV9yDhPWHMe7afkzQwezcE2k6U6tpVfB/k78eYMaIypT2Fv7P6nUXPmJ4M7TLU48eWhb9Y5eGVNnTvDu+/D9WNQfTY/SEF1YU0aYNbKniGDYM+fdQfV5JEUmRhoTjPDh8T8RJ94sSbe6jsEHqjHp3G9R5s3qSwppDy+nIkJPrEunBBKEArCcHK184txyOV+lCq47oR48H185gx8OabkJlprjgeY6ez01VXiXF30ybx7xkz3F/Ld+8O97aq6ZAkiUt+uYSiPUWYTCZF8/h+Z/Zz70RaIceE7K03jHojP9/wM6FxoUx7appHXxsgND6UQRfYSWRqRhb9JKmDJ4u1QpLg0kt9fRbK8LeKP6OxrfDnC1TfAX2hTnbihBHPqdq8d2xvAA6UHlC0vew7XdDKTVTSSJzziXID6ZaKPxeFP7mKLSnJNTHjuESSxOK6mchI80Ls2DFx43ZGXVMdW/K3EBPi3WzY9sBgMF8n7VrxJ0kgBUBtLlTsgqTp7dBU0DbDhtWwcqWJsjKJv/+GyZPVH+O6UddxzYhrSIlwr7Q2aVgSFyy4gJRRto8TGh/KBd9fQHSP6DbPeUL4iw+N5+dLfnb9ACcAsiVCebmYINlbLJTUlrBo3yJm9prpcbFeKe5U/A1PHs7yK3zTf7OjIr/PloFvf0Se2AeHm3uSBusciwTHDcX/wMabYfCDwpbPFeqLoXQTJE8XwV4nyP3c5B69lgtxk8n+rW/cneNoqmnCZDAh6bx7f8wpbvYBNEmkxXf8uY1qyncI2/HUMyFE4Xhduln0Ke5/B3RXbxeQHpUuhL/KbMZhW5C1hRzELioSCX6thb8/7v2D1c+s5q6CuwhLsL9invrEVOrK6hy8jkREkPo0Zfm+ExrqWm9ZvyFxEkz7S90+dfmw7ipIma0+WTEwBlJOc7xNxR7YfCf0uspm5V+L8GdR8SdJQozJyxOiR7KDyzsiOYILF15IVFfvVvnrjXoqjLkgQY/Y9lx8eBk7oh+I78LAgbBhA0ycKCpD2lv46xPXp0Xc8TSWVp/uEh4YzpXDrgSgaijccgsUF4v2Ha4SG2sW/jqxTVpkGkHaIBoMDRytOOq3Nv9ykm236G6EBHi2D7KMTqNjyrIpGH8ywiavvMQJiVFvpPJIBZiiiIry3AQhLg769oV9+8S8uk8f+2ORVitsP++7T8yjXKkiVkrCwIQ2Dk1KKN5bTGzvWJtW7IYmAwsvX0j6+HTG3GJH3WzGmdWnRqche3U2YYmeV1eylmaRMiqFkBjH31F5TRoeLj6bTtofs2ORb89Dps5iaRLq+a4XiunIS5hOXESeJO8vVdZsNSlJ/G5d8aeW3Er3rD6PG5tPgJ2Pw7HfPHMsfR0UrIDqw0iSuc+fUrvPHjGih8a+kn2eOR8V/N+K/+PldS9TUe+Zmv2CAmF9EhjYzhmQhX+LQNvO/8HymVBvoyFmO6HTCW93gEWLXDtGYlii26IfiKrgAecMICrddvAgKCKIAecOIHl428iN3L/FHeGvE2Uo8cw/5+tzuOrHq/h+z/ftc1I28Ad/9BMJtRPnsroyVhxewaojq7x3UjaQzy8wxCz8BWlPkIq/pgqozwNjo+vH2PUE/HUaVGUp2vyJqU9Qc38ND5ws+ubI44fRaBZJbDFo7iCGXTnMpi2nK5iMJhZctoDN729u85y+PoShhz5geMGLBAWcgJlix36Ff66D6sPK99GFQ222uKZcoF9cP/rF9UNywdNJrqqxFcROGpbE4AsH01jt+BrvcWoPBp430OE2ruBOwkmHRxcGhX9BrXf6TmEyQsEy0WvOBv3j+zM2bSzDk4ZbPW6vz9/u73fzap9XyVkv1KeA0AD6n92f5BHeTVbKq8rDJBmQjAF0i0vy6mt5jLzfYfEwKPjL/jYmI+T8BNkLbD591VXC+uumm8xrz6IiszVuR8YTFX+2iIiAF16A55+37pmkFvm8Wn8HOjGjkTT0ihXWz/tLlMW8fMHeYiH8OWu/4w7do7vTs6knuiMn4HzIi5QeKCX47VdIzVzm8XjFWIvCcXvVfjJhYWJc+fBD6758/sC6l9fxev/XObrqqM3nm2qb2PX1Loozi1seW/XkKrJ+b7smsecwsu+XfRgNojjp2nXXcuVfVwJQmVtJ2SH3B8m6sjq+PONLvr/YeQxEPrfjyeazoyG/9/5i9SnHrwIDhU23r1A0+t9xh3L7phdeeMHlkzkR0XgifbQuHzbeBEnToM8NTjfvFSMmQfnV+VQ3VhMe6Hg16zHhz82KP7kCyJ+t6BTRWA7bH4SeV0LKTGdbO6d0I/w5GYY+AYPuIzUV9u5VLvzJFik5lTkU1hSSGNY+ipneqOfhvx4G4OIMhf1GnCD390tLa+eCu5VnQcxQGPwQxAwDHwWdNRoNGRkZdO8u8e23IlNs717o51mnBdXoG/TogtrebkxGE5LG9gfliYq/JkMTAVrf26/6O0osEc7ufzarjq5iwZ4F3DzaAzbFLuBOANafbX78FXPFn8TgwRlO5ytLDy7lwu8uZEL6BP6++u92OENrsSkgVEQbdRodWs0JkmaZPB3OznbvGF3nCsu3wGhFm7d2B9DpzE4DFRXOez8YDUabWb9qqS+vZ8fnOwgIDWDENSOsnjPUhZNeclXL/PWEo/ulwtYxerDyfSL6wLl5Lr/ke3PsWAbaQZ6vaDSalnm9LeFv8EWDGXyR4/+HEtupa368Bo2k4ZHJj6hKQDxuhL/Ml4SQM0CFJXNABFxQ69qEOvcXWHUejHkfetjxh4oaCBfW2T3+iOQRrL1mbZvH7YkeAaEBSFqJhsoGDE0GqvOqvV7tB3CoTAiXIY3ppKd1kJzqhhKoL3DSa16CDTeKe0N626ZR6enW1l9RUeIekJsLvWy32vQ4Kw6vYG3OWmb0msGI5BFW44o7eKLHn8w/uf9QUV9BRpcMksKTiIlxr385mPfvrPhzTJ/YPuwu2s3+0v3MxAMxFy+QWZwJiOQZb3Hn+DthjdcOf1zjaEzRhego7TuGqpgeLaKUpxg3Dj75RPztTPgDMRf3R0e0ntN6MuJfIwhNsC51+u6i7+hxag9Gzh/JQ00PtQh3VXlVrHl2DQkDE+g5vafV3K7FYSSvhuoCE+Fdwtm/eD9fnvElE+6ZwLSnprX0HayvqOedke8QmRrJtf9c69a6QxekY/abs4lIce4aIcdSOoU/3+Gvwp9ltZ9HNCCVKBoetmxR1iDeUz07OlFJQITIyAtR1lMjJiSGuJA4SupKOFB6gGFJwxxubyn8ObJxcoTRZORYlSjZc7Xi77hZfOvCYdZW0HrIiix2FAx9ElJnA+Y+f0rtViKCIugb15d9JfvYkreFmb3bZ2JcXCsyeyQkt/qvffed6PVxySXm/n7tavMJMOxpCIqDLlPEjw9pbGwkKiqYU06BP/+En34SFhBqWbhnIe9ufpfpPadz+7jbXT6fL8/8kpz1Odxd2PYkfr7+Z/b+uJeb993cpn+PJ4S/fy36FwszF/Ls9GeZP3K+6wc6zpEXK46Ev3P6n8Odv9/JiiMrKK4tJj60/ftmuVPxN+TNIVQ0VPDjRT8yKmWUZ0/sOEUWcEwmKC9vJC7O8T1LthFvz+rxmhpxfgABwaLi74Sp9vMUCePEjxtERZmFvzQHU9FPpn1CTUEN12+73m7Sh1JCYkN4oO4BDI1te9bIiz1/b0DvNUJTxI8afLCGa2xsJDg42GHFnxKOrDzCV3O+YtZrsxh6edt+XyaTic92fEajoZEHT35Q1bGPm7XHgbdA0qoT/sD16yIoTtjeO7KadfHY9kSPPrP60GeWcLXJ35bP28PeZtZrsxh902iXXkcpO5oXH6H6ri3tMfye7hdDNyeWvpIEYz+AEGVjSVqauAfk5LSf8Pf2prf5cueXSEiMSBYJIPK44g6y1acnKv5eWPsCX+/6mhdmvODWesoSd8fMEwW5Z55fV/yVeL/irxP3sDemhKdGc7CvsFnydMVfWppIrDAYoFs3zx67PUkclMiZ71j32K06VkXWb1kEx4j3VKPTtLiBRCRHcMXyK4jqGtVGW5AFHf3f63kh9W/+teFfdJ/SnfF3j29jExocFczY28fatRhVQ0BoAMOvHu58QzqFP39AXvs56z3fXviLY5Ui4W/58s6+ON7CIz0TdWFwQbWqKqM+cX0oyVEm/CUmirl/Q4P4ArlyYzOajHx41ofkVuaSHO6a5cpxE8jR6ER1mKfQhcAgcwdeWfiTrVGVMCJ5BPtK9rE5b3O7CX9yf6C40DiXqzPKy+Hjj8XfcXHmir/0dA+coBp6X9vOL2gbo9HI3r17ycjIYM4cLX/+CatXw9VXq89azanMYcmBJWgkjVsL1ZTRKQRFBtms8ojtHUv8gPiW7CxLPCH8Hak4QmVDJWEBnd6QjlBi9dkjpgfDkoaxNX8ri/Yu4qrhV7XPyVkgB2DVTpyMJiMHyw7SYGhwK8ngRCMgAIKDhTf91q1ZTJkyAK2DhgV94/oCUFRbREltCXGh3n+v5QVWaCgYaBb+dC4IfyYTVB2AyOZ+QY0VYk7lqQQdb3HsN2gsEQFcBf35HGIyKjpGWV0Zd/1+F0W1Rfx40Y9IkkRUlJhzOFtkpY5OpTKnksbqRpvjvlp0wTp0wW2XMpmFByiMzKJ7ZE/AOz2g/JqmStBFqBdWSrfAscXCPSTIwx53rbCcr8TGinFFDra3Ztun29j6wVYuWXwJASFtq/i1AVpSRqUQnmRbnatoqKDRIKxCu4SrU2aOG+Fv2irhOKKWsq2i/2PPK9WNMfFjYbICv/nClaCvgZRZdjfRG/UYjIaWsd2e1aclwdHBRHWLotcM7ytQO3OE8JcQ2LVj9YFUMj4kz1B8uLQ02LWrffv8DUwQ9r67i3cD1uOKo/mKI0wmzwp/RyvE9ZEe5bnFaafVpzLk9jZFtUU+PhP7HCo/BHi34g8gZ30OFUcqGHj+QLcTr04kHI0p8hpEo/HOHOEi9e2W/RaTyUThjkK6DOlCREoEd+bdadfCPWmo2a7j8IrDhCeFE98vviVRuTwinXEXZ9BlSBc0Wg3Tn5lu8zgT75nY8reh0YAmQKO6UKmhsgFDo4HQeGXN2TqFP98jawX+VvFnGb/yiAakko40Pe3EESoz3OXMfCUZUDodxDcXd+S56AKk0+i4bMhl3DPxHpet9+TFd4cX/poqxY9cpuApTEYwNFpV/Cl9iRFJIktyc37bXjneoqhGTMLdsRY9YtEW5P33xYITfFDxJ2Mywa+jYbUdW6N2pGdPGDRIZIotXqx+f7kqasOxDZjcuFZPeegUzv38XJvZVhP+M4Er/7rS5iRMFv7cscM5Ui4ukG7RHThVrh1QYvUJcHa/swH4LctD/UlVIk+c1C6u8qryaDA0oJW0Hg28nAiY7T6dTxfDA8NJjxTvr5zB7G3kazYiQgR4cu/IZct1ylwqrNj9FCzOEEFogLxfYflpYNR77mS9wd6XYP214EJfNSs23AgLU8U8wglBuiA+2PoBi/YtoqJBKH0ti/Fyx/ue+vipnPPJOR4R/arzq8lZn0NDZUOb55bmfsc/fU9jXcATbr9Oh+THHuL6VUvhCmFFX7ZV9a6Hyg4x4u0RDHxdfZ89Z9UrVblVHNt0jNIDtjdIH5/OvD/n0Wu6bZGnoLoAgMigSIJ16sT840b4C04wJzao4cA7sP4a7/Wu3nSbGH/scN2i6wh5PIQPtnzQ8pgS0SO6WzS3Hb6NuD7eT0A5UCSEnbSIdlh8VB8yL+5qsqFkg/pjlG6BbQ9B9UFl2xvqxes6Qa72znbTfVoNcpWUJ50GamqgsTke7QmrT1n46xrlueuj0+pTGZdkXELlvZV8cd4Xvj4Vu2y/fjtZt2QxNm2s841d5KV1L3H/jffz3YXfYWhq65LQiWssvu5Hum1fRFSUT0wTOhSL/rWId096t6Xnni5Y51RMqy6o5vNZn7Pg0gUYmgzse28lmqYG8iP6cPYn5yiu5Ksvr+eTqZ+w6nH1Peg3vbuJ55Of5/CKw4q27xT+fI+/WX3W1orfHaLirzUbN27km2++4ejRozQ2Wiv1CxbYbgDdiZepyYaC5aLPnwJ7nwsHXciQxCFM7TlV0eETE0XDbnvZuO2B/OXt8Ivvva/A9ofgtE0QO8L59koo/BtWnAHDnyOl67VIkli4VFaiyHN8ZMpI+sT2aQnatgdyxZ87wt/hw+a/6+vNDeXbteKvqRJ+GwPdLoaMh4X1rk5ZVpC3mTNHiKG//goXXiiayiplWNIwdBodhTWF5FTmtLtgIgeQXfXMNxgNZFeK6IMnF9vHI0qFP3lRuiXfBWHFA7gagD1YJoJb3aK7dfb5U0lEhLj319Upy5zvH9+f7MpsMoszGZ8+3stnZ54XREaKBKOUCJX2hjIps4ToFz1E/Lv6oKgArDkCEe3kWeYKw56E2mPuRxyCEkVPuKYKCHScbREaEEp8aDzFtcUcKT9CdFJ0S2W2szFETvIwmUzUFtUSluj6Kijzx0x+uf4XLv31UnrP7G31nGwlHhPU/pbEPsdogPRzIKKv+n27zoX4cebvgQoigyJb7g31+npVApujHn8AY24Zw/i7x7ts11RQI4S/LmHqfRiPC7eRpkqoy4Ow7up7UPe6FlJOhwCVUaySDXDkK+h1jejlZ48hjwH2k8tCA0LRG/VklWW1PCaPN/4ienSpOZVuhTB2zETnG6ulJhvCmuffBX/Bsmkw4UtIOxv+PBX0lXDmAbH2UMqxX2DXY+K+F97T8bYmEyzqI+4RszY53FQW/tqz4k9et+ZUeu5F5esqPFzduskWTYYm8qpF1rQn1yKdVp/KCA/0/6CRVqOlZ4yT76GbVDZUsnz4cmKnx7ZYKnbiPiV7Cgmu0iJ5v5Vshyfj0gwCQgM4tuEYO77YwajrRxEa5zheFt4lnDPeOoPkEcns/Gon655YTmJ/ifzek6iqUi6uyRWuFdkV6Ov1Np1C7JE2Jo2e03qSOlpZq6pO4c/3KI1rtReuOlZ5GtUj/1dffcX48ePZs2cPCxcupKmpiV27drFs2TKiPN3VtBPlFK2CdVeI3wo4o+8Z3D3h7hY/fGfImWWuCn+Hyw+zeP9i9hTtce0AHEfCX/QQ6HEFhHpQjIjoDdGDITCKwEBISBAPK118ndrjVPb9ex8vzHzBc+fkBNl2IyE0weVjyBV/U6YISzoQFapJSfb38TjGJtE3RWbqnzDm3XY8AWssbSjGjBHVupWVsG2buuOEBIQwOHEwIKr+XKX8cDk/3/Az+xdbVxcf/OMgP1//MyX72w4qliKuq1afedV56I16tJLWdTHgBEFJjz+A4cnC335/yX6qG6u9fFZtcbXiTxb+vL2wPh6Rg911dcoWSXL2fWZxprdOyQpL4c8tYobBlCUQGC3+3f9OmLXFv0U/EOederr7xxnyCJz6u1PRT6ZblKiiPlIhbsJKK/5AiH6fnPoJn0z7BJPR9Wry9HHpTH1qKgkD284hyhqE8HdCWvtqtDDmPRj4H/X7hqZC/BhhIa+S2JBYQgNEEEdpEF6erzgLYgeEBjgU/da9tI6/n/rb7vNyxZ9am084Tir+8v+En/vD0W/V7xs7AlLPUJ/QVr4TMl+A6sOOt0s9vaVHuS1kC+k9xeb1o7/ZHEbmnE/G0Tc4e7CHWyVsvAUW9YI6cf0S3AW6TBUCriYARjwPI19VJ/oBDLwXZm4QPeKdIUnQ+3rodqFTGxk56TI3F9rLxSo1UgRj86ryMBhFJZOrFp8y8jjkCZvPY1XHMJqMBGoD3Up0bY18bhUVoJeNCZRWcHZywqGVtOSk51AzvsbtfmcnIvbGlCGv/ou946/0eH+/45EeU3ow65VZ7P1pL389/BeGBmWVp0PnDSVxcCJDLh3COZ+dQ/Vg0ZNcTf+2oMggrlxxJWe+faYq0Q+g68SuXLrkUps287boFP58jxy78Bfhz18q/lSP/E888QQvvvgiixYtIjAwkJdffpnMzEwuuOACuvrMX6/j4u7ktIUuU2Did5A42TPHa4WzbFxnLN6/mNlfzOaBZQ+4fA7HjdVn2hwY9xEEezATPSQJpv8tsrUx9/nLzfXcS3gaT1T8ycLf6NFwzTXi7169wFNfK0UExcHsnaLaz8dotVorD3qtVrwfAMXF6o83KlkEBTYe2+jyOekb9Gx6axNHVh2xevzIqiNsensTxqa20QE5eBwYCCHq44+A2VonLTKts8rLCUozoxLDElly6RKO3n7UJ30TXc2YahH+ojuFP7WEh4sqrcTEXormK+0t/Flafe4q3MWtS27l5XUvK9tZXwd/zYa839s+pw0U1ngAlXudB699gckkEk98QPfo7oBI6gKz8KdkIS5JEukT0+k+uTv6etetVLsM6cLEeyYSld426bC8UdzwEsJOwIo/dzE0impXlUiS1FJ9I99/HWE5X5GD2FVVZou91hRsL2DtC2ttisUb39rI1o+22n0tueLPlfnmcSH8hfeCgfe45zJiVGkP1+0COOuoWJ+6gZyAtrNwZ8tj8vVSWWkheviIpibzWqubp13lu5wK3S8DY7OdcdQAOPU3iDtJ/Dttjnif1aLRQdwocZ9TwuAHRCKBk8ryhAQxb9froaDAxgY12ZD5kurTdURSeBIaSYPBZKCwprDNOsgV5CRnT9p8pkemo3G3D68FkZHmtW55OXD4K1jUD45847HXOF7434r/Mf3T6azJXuPrU2nDR1s/4uLvL2bhnoVefR15HWwwddp8qsXRmFJRAUgal92JTkTO+vAsrl59NREp6gK6kkZiyKVDiIwV17Ia4U/eH0Tyob7B+cShaE8RBdtt3cgc0yn8+R5ZK6ivF3M0XyMnroda5M95TANSgeoZSFZWFrNni8y8wMBAampqkCSJ22+/nXfeecfjJ3i8407vLCtCkqHreRCiLJvVZDKxvWA7C/YsoK6pzun27mZXyrZLrlZ3mUzHyeK7nZDtVtQKf0aTkcqG9kmPuGXMLay9Zi23jLnFpf1NJjjaHFvq3h1mzoT//hfuustz5+gS1Ydhx6NQ/E+7v7TJZKKystJqXHHnu3tSqgguuFPxF9srljvz7mTq49a2wpMfmcztObcT16/tyloW/qKjXXew6+zvpxw1lgin9T6NtMg01c2x3cWde8DB8s6KP1cRk2cTBQW1iuYr03pO463Zb/HgyQ96/dzAeoF1oPQAr/zzCl/t+krZziX/iEqYSgciZX0R/DYa1l7u+b687lJfAF8FwZa7PXO8zJdh+yOKNnVH+AM49f9OZdYrswgIda3nszOqDCJy2yXiBBT+cn+BdVe5JN4BsHIOLB6qXujBbGWXXeG8yZflfCUszGypZ2+usv2z7fx+5+8U7S5q89z8TfO56MeL7L5WSW3z9eCC1aerleZ+RcwQGPaUY8tNe9Tlw7cxsOVOdfvpwoRFpbPq0YOfwHexUGjbsWZQ4iBACCjy+iQyEjQaMSSrDf55moNHGygP2kFARIVHKsSozRXJJgDpZ8PYDyDMSWJ1xW5YcRZUKEi4qc0V6xNXkkac7KPRQEqzwYZNx5ktd8Lm26HUc/3kdRpdy/c6tyrX5jpILbLw54nPs0X483C7BEkyuzGVlSHE4ISJwqr52G8efY87Outz1/PHwT/YUbDD16fShuWHl/PVzq/YVbTLq6+j1WiZ+etM0ualUV9Rr2rfwkJ46y3YudP5tscj9saU/K355P62A21jXWfFnwq0AVrSxqa5vL9SlyJblB0q442Bb7DmWedJAL/c8AvvjHqHqmPqGsV1Cn++RyQti7/9oc+frXWExzQgFagW/mJiYqhqfgdTU1PZ2XwXKC8vp1auY+xEMUZPe2E0Kbu6JUli6idTOe+b86zsU+zhrtVnUY1YqMeHuhaEqa8HQ3MMosNX/K29EnY+5vnjlu+CNZdB/rIW4W//fse7WPLx1o+JfiqaG3+50fPnZoPEsETGpo1tsfFRS0GBuC4CAiA5WQzwo0a1s80nCAueva9BWbOXZn0h7HgECv5o5xMR48nBgwetxhWrhaFKTko5ifDAcLequzQ6DeFJ4S2ZVjKSJBGZGmnTcsRS+HOVuNA4ZveZzaSuk1w/yAmC5STa37QNmfp687mprfgbGD+QU7qdQkaXDM+f2HFORIR43w8fLlY0X+kb15frRl3H6NTR7XB21hV/DQZRFRGktIdVl1PgrCOih5U9ghNg4H0w5H/u99HzNCYDdD0fogZ75ng5C2Dvy2By/jnLwp8rVp+e4sOTP+SHK3+w+VyNSSSbpUSfgMJfyQY4+BEY7ZTOOaPrhdD/NnOVkQrUVPxZzlckybnd5/CrhzPvz3nE9GprRxsYFkh8P/uf9X8n/5eq+6p4YuoTzv8TrThu2gy4SlC8qBQMV2l73FgGVVlgcBJkDk6A6KHCutIGsSGxJIcnA7C7aDeA1fXiipuFJ1mTuY+Vg4awpG9Pz9wi1v8LlgyHGuffoRYaKyD3J8i3Ub3emsOfw+9j7Aqtdtl8F/w8EIyOKyVs9vkzGqBiD5z0Fkxa4Ln+9s18ff7XbJ6/mUEJg2yug9Qij0GeqPib0HUCH571IbeOudX9g7VCXt+VliJsyactF/OCFafDtvs9/nodlV4xYuyS3T/8ib3FQuTvF9fPq6+jlbRURVTRkN6gOnHzl1/Ez333wbPPuh4L7KjYG1N2fbuL4rcWENBQ1Vnx1464s96ISo9CE6BBE+BcApnx/AymPzNddWVip/DneyTJv+w+bVX8eVwDUoBi/7OdO3cyePBgTj75ZJYuXUpGRgZz587l1ltvZdmyZSxdupSpU6c6P1An3mP9v+DgB3BBDWiDnW6ekZjB8sPL2V6w3WmvP3ny63LFX51Ymbkq/MkL74AA9xtt+xSTCXJ+gAQviBEmg1jQRWcwatSpgMjOKi1VlrWYEJZAVWMVm/M6RpagbPOZnt7O1p6tqc6CTf+GES9AzFCRWX3aJojs78OTMmO1MFTJ0KShlN9Tjlbj3htcmVNJ2aEyuk0S1Xd1ZXXkrM0hdUyqzcbOnhD+ZvSawYxeM1w/wAmEPDnS66Guznpi0pq8qjze3vQ2FfUVvHjai+1zgpir/XQ69feA+ybdx32T7vP8SZ0AyNdGba0vB1n7WPb4K9aLAHOwzvn8pwUlLgmD7nXhzNqB0FSY6EFbr7EfiyC8AjsyWfirqBflNq5k4G7/bDurHl/Fpb9eSnS3aJUnCyaDCZPBdqZCHWLOmRpzAgp/Qx6B/reLiitX6HWVyy/dUvFX6bzirzWxsZCfbz+oGN8/nvj+bT/Pgh0FaHQa4vvFt0kwsiQ80DXl7rhwG1k1V/QvdaX/tEYneler5eBHsPkOmL4GEsbZ3y5llvhxwODEweRV57GrcBdj08YCkJgoRL+iIujn3Zi5Q7Y3W48kBHjIXWLQ/cKG01mVnyUJ42BOFoQrcDVImg6DqyF+rLrzCoiA0BSR3Bhqv2+2LPxlWw4BpRvh97Ew9AkYZDEXMxkV3W+cMambeU1tkLOE3cCTVp/do7tz5bAr3T+QDWJjYUL6twQWaME4R3xXw7vDmPeFTWwngNntI6ssy8dnYo3JZGJvSbPwF+/dQUyn0bFmwhrSB6UTFKkwOa4ZS/eolSthwwZ4/nlzT88TlRHXjODv/Uk0VMd2Vvy1I+5U/Gl0Gq7fdr0i8TtlZAopI+3f62zR1CTiKNAp/PmaiAhxjfhTxV+H6fE3ZMgQxowZ0yL4ATzwwAPccccdFBQUcN555/H+++977UQ7UUDcSdDtEsVVf8OShgGwNX+r023dEQ/AbPXpqvBn2d/P35LuVSFJcH4ZTPrW88eOGgTnFsHAe0hMFAthkwn+/lvZ7rL4u7dkLzWNNZ4/v1Y8t+Y5Xlr3UkuvP7XIwp/He2qoJWoQTP0L0s8T/9YGi2xWnQP1pB1xx+pTI2ncFv0Afr/zdz46+aMWT/Xs1dl8MfsLdn+32+b2nhD+OlFOUJD4Aee2WfX6eh5d8Sivb3idRoOLFSUuYDlp6tD3gA6GLPxVVysfB3YU7ODDLR+yvWC7l87KjKXw16BvrvjT2QhqbLwVNjRXs5tM8PeFcPgLdS9WuV/spz9O3S3CuysWi6b3nE7N/TX8MU9UtlsuxJUmMZpMJvT1eqpyXVuVXb36as759Bwbx4XBuS8yIPtZeiemunTsDk9glAgCtzO9Y3vTL64fcSHqI+bOKv5AXDMl+62VweUPLefNjDcV9WxxheNC+Ks5BLXqxVi3iB0FA/4jEhTc5LTep3Hx4ItbhGUQwh8IGzpfsr9QCH+p4SqEOkckThQV5mpRIvoBxA4Xx1e7Rhn8EExb4VD0A7MYYFXxFxgNfW8WoiNA9SH4dRTseU7dObQT8hjkEetWLxIbC5dmPEz/qhusn+h5pTrh+DjHXyv+imqLKK8vR0KiT2wf77yIUQ8/dGVS9d8MiB9ASoQ6IQPM/TqvuAJ69hTCxvffe/g8OyAxPWOoSBuESavrrPhrR+TYkKvFKJIkYTKZWPviWv68v21S044vd5C91rX5krwm1Wh8L/Kc6KhpY+NtZFNMX18TioW/FStWMGjQIJ588kkGDBjAFVdcwerVq7n33nv56aefeP7554mJaWu/0kk70ns+jP9UZGwrYGiXoYAy4U+e/NbUQIN69x9zj78w13r8yQtvX39hPIIkKarIVI1GC8FmYfWUU8TvlSuV7Z4UnkRSeBJGk7FdAraPrXyM23+7ndI619RkvxH+AqOEZZzlIktfJ6w/FVimeZrgYOtryx2rT0uaDK53xx0ybwizXpvVUp3RZWgXTn/jdHpOsx2o8ITwV91Y7frOJyBKJ0jdo7sTHRxNk7GpxXarPXA1+NpoaKRer66fRSdmzPd+5WWWz619jqt/uppFexd56azMyEK1U6vP7O9Evz6AhhIxNhevV/diOT/A0W+gYJnrJ+xJDn4sBM1GDzW5MplEr6ii1U43DdIFERpgDhzL44fJpDy7cshlQ7j10K2kj/ds2nhDA6QUXkmvgrtIie/o3vAukP+nEKndYf2/RM8wlVw65FIyb87kyWlPKtrecr4iV9c4Ev6W3LKE1/q+RmWu+UY1cv5Ipj09jYAQ+/0ib11yK9f8eA37S9S9L01N5jVPhxb+TtsIk5e4vn/2AnFNOLPttCRxEgx/2rkAYdTDjv/BoU/tbnLHuDv44rwvmN5restjCc3LSV8Lf9nNtrY949wUWjJfhDI3116lm+CPKeYega0pWuPUqtMulpV5DjzhbVp9RvaDUa+KSkaAkFRxH1ZzPTlgZ+FOnln9DF/sEMk8rddBavFkxd+CPQtYmrXUK2uS2Fh4cPkf/Fr1VdtED6MBjnzT2esP6BUrhL+ssiyf9FWyh2zz2TWqKyEBTnqhukpdHtRmM8yQzU+9fuLMNWfSUKU8kGcyiUp8gHHj4Mbm/LmVK33fX7U9aT2m1JXVUVtc2/IedAp/7YeSJDFnZP2exfqX11O4Q0wgyg6W8fmszyk/Us6Sm5ew8PKFGPXqY3iWa9LOJGXfIq9L/aniz5GjVnugWPibNGkSH3zwAXl5ebz66qscPnyYU045hb59+/L000+TL98VOlGF1ocehZYVf84mQqGhZns1VwZad3v8yV/aDt/fr/YY5P0O9V5qSlFzBI5+B4Z6Jk4UN529e82TNmfIVX/etvts0DdQ0SDujolhiS4d4/Bh8dvnwp/R0Fbg23YfLBmmrkeHB9BqtfTv399qXLEU/lxZ76w6soq+r/Zl2qfTXD6vvrP7Mvqm0QSEisBcVHoUJ91wErG9bKfTuiv8mUwmkp5LIuqpKA6XH3btICcYSq0zJElquXdsydvi3ZOywFZjZCX8nvU7oY+Hctpnp3n+pE4A4uLEZ240xiqer/SPEzbHSvoHu4tlxV+9pdVnzRFYdb7owQowexdM+Er8HRwvbJnVWnj2uR7O2AupZ3jo7N3k2GLY/5rnEokkCZbPhA03ON+2FVqteX6mNBikts+MJflb81n30joqstu+mHxN6HTmSuYTBkOD+Ay3/se949QXQn2+V5OXWs9XlARzekzpwZjbxqANNI9FfU7vw/g7xzt8re/2fMcHWz+gqlFdBEBOOJGk4yDx0J0oVPF6yHoParxQNShpYfeTcPhLVbvJFX9FRZ4/JaXU1UGxXmQhDkhxQ/irOQpb7hY/7mCoh6K/bSdvVB2AP06Bda5b+aKvgZVnw8ab7G6S0lxQVFXVPBbbGkO0gcKaNONh18/Fgk3HNnHPH/fw0daPbK6D1GA0mhMlPVHxd81P1zDjsxmK+p6qJSYGSutS2VE4pe2T1Vmw+iLY9bjHX7ej0SO6BwCVDZUuJxt7A9l6tHdsb++9SFg6XGKC6X9z4NcDrPzfShoqlQt/FRWix7ok0eIm1aePSIr57TfvnbY/YWtM2fLBFp7r8hyNB8T3ulP4az+UJIk5o8eUHszfNJ+LF10MwJGVR8j6PYvc9blcvfpqzv7obDQ69TbUnf39/Ad/7PFnGcPyhQak+ooOCwvjqquuYsWKFezbt4+5c+fy+uuv07VrV+bMmeONczyu8Xhjx013wLYHFG06IGEAAZoAKhoqnE5IJcm9gfaZ6c/w3PTnWiZfajluhL/8P0RQpniNd46f9QH8PRcq9xITA0OGiIeVVv2NShbZmMsPL/fO+TVTVCtW6jqNjujgaNX76/Vmz/nu3T13Xi5x5Ev4Ugs5P5kfS5ktbHFsVZ54EaPRSElJidW4Eh0tvr8Gg2s3v7jQOPaX7mfTsU0Y3QgCGhoN5KwXKcDOsqhk4c/VIvKy+jJqmmqobKikS5iCHl6dqLJEGJ40HIAt+e0n/Lla9X2w7CAmTC73dzrRiYsTQnpxcSMNDcq+/wMSBgCQWZzpzVOzqi6LiLCw+tQGiQqV7O9FbyEQVmOWtsWDH4SQZHUvGBABkX3dP3FPMf4LOOuIZ+8zQx6HjEcUbfr4yseZ/ul0lh8S8wU58CGP30rI/SeX3+/+nfoKdZUfB/88yG+3/0bF0bbC36HCAgojf8UQt/PEy7g1GWHMe9D7OveOM2kBzFzvkf5b9mg9X5GD7PZ6/AEMOHcAp714GmEJ4kZgMirLZpKDvbEh6iL5lvedDnstVewW/fbq3EjQHfgfOK8EIlQEqA9+BMtnORcLJUn0xR73icPNDEYD+0v2U9ckGuj4Q8Xf0aNQFyjWz327uCH8hXWFmf/AyJfdO6GECXD2Ueh1ddvngruI/oG957t+fF2YuI4aSuxmEgYHm7/L+fmISs5FfaFko/WGHhxbUiOFnWxuVa7NdZAaKiqE+CdJ7rcbqGyopLy+HID0SA83RDM20TvgK4K0tbYdXSL7wvjP4KS3PPu6HZCQgBBSIlJICE0gv9p/ChVK60rRaXQtVqReRdIw4Z4J3LjrRsISlS+k5MTx+HgIaC6qP/NM8XvJEhGPOd6xNaYkDkqk79kDqAgT64jO1iTth5K5ojO0gVpC48zlV8OuHMYNO29g0AWDiO8fT9eJrt3P5RhKpxDse2TNwF8r/jyuASnArVlX7969uf/++3nwwQeJiIjgl19+8dR5nTB43HIg/zdhR6WAQG0ggxIHAersPl2xDLwk4xLuHH8ncaGu+WYcFz02AOLHiUl47AjvHD/9XLF4DhVeK2rtPs/ufzYAP+/7mYp673k4yBWgCaEJaFxY/OXkCCErLMwzVixuEZoGXee2vOcAJE8XPTTUBpbdxGQykZ2dbTWu6HRmUceV727fuL4EaYOoaarhSPkRl8/ttzt/48OJH7Lv5308GfEkyx+2Ly67W/Enn2diWKL37FOOMzqK8Kf2HiD39OgZo7D/TSdWhIeLxX59fR2lpcrmK/3jRcVfZnGmV22VamvFfQDE9XvT6JvYd/M+Hp3yKPS/HWZthdTZnn1Rkwlyf4b9b3r2uK6g0VrfdzxBz3liHqGATXmb+OPgH+wq2gWYx2s1CSbZa7JZ+9xactblON/YgqHzhnLliivpMqRtYsffR9bwT99ZbEhyI8DdUdGFiP5OKW5WOLvR23f6p9NJeDbBqWV86/mKGvsmk9FEY00jyx5cxqt9XrVZ+SlT11TXUg38/+yddXQbV9qHn5FkjhkTO3Ych5mhwYbLzLDldsvtFrbttlvYr7vlLXfbLdO2TRmSpqGGqWEm2zHEdsxMkr4/rseSbcGMNIIkes7JSTK6mrmjmblz7wu/11XH33G99ihcILK8ag66vo+QeAiJU+f9rMsV2WcmBdkl0YM6lCqwxeA3BtPvtX5sKhQOJH/I+LN2/GVEuyA/UrQUDrc5PONGQfQA9zslrzs6v3uDImHYE0KC1R1m/Q6Tv3B4L3Rv68LRo4CpBUzNtiVfD70PS2YJ5RQ3SI1sc/xVF9hcB6lBNibHxopMdnfIqxJO75jQGCJDNI5cLlxIVtFlnNXvFftjZq/LFZd/ccjWv8KvE4RU7HHKwTsOUnJ/Sbvdyx+4d+K9NDzSwHNznvPMAcr/gDVXwXfp5C88lfHfjOevB/6KPkj5jX30qPg7JcWybfJk4dgoLYV16zTusx9ia0zpM68P01+/CLM+iODgk1BdwofIdr+KCuU1xZWQOND9sTKQ8ec/+EuNv+ZmkSENHYPXfSE77bLjb8WKFVxzzTWkpKRw//33c/7557N6tfO6IAE8zJz1cPpOxc3/Pu3vfHvJt0xKn+S0rZx9406EhavIi+/jPuMvqi/0vVl7Y51M7HDIvEos0hF67AaDqIeXq8BnMyJlBBcOupCnTn3KM/1ro6ROhOi6WvPxSFuCanq6H0RhJ0+HyV96zpmrAe7U+TPoDPRP6A/gVk23sbeOZeT1I4nvH0+v6b2ITrcfDuW2469K3OwuGWJOUlQ5/roLx9/Woq1uZYGqwZ2MPwg4/lxFkiAuTkxOlb77s2KzMOgM1LXUUVBT4LG+yVF8ISFCijwmNIa+8X3pEdmmNRY7XPuDShJs/ztsfUjIKvqKlloo/l1kXngCk/Oarr1iegGWQAt5DFGT8TfksiH8ecefyZqtLuI9IjGCjKkZhER2tbYUVQsp9W4616Tlj2u0XEjm/A/2/lv118rqyyitL1UtbadUVaQqr4qXe7/M70/8Tkh0CEERQUR2t784kLP99JKeyGB1i4gTwvGXcQlM/Q5ihrq+D1MrVO2BmkPKvzP073BxjbIswZYaUZfOwXxCrtO1s0SsceWMv7o6SzS1t8nNhcySO5ge8ef2/qliz7NCWllrA1DJSvh1rKj1WbkL1t/kem2/zijIMJcdf4WFQJ8b4OxsCLVR1qEuFyo2Q417NUnljL+qpirqmt27GeTxRwuZT3kMTI92s/6jLRImUjfwVZbnXuHYAG5sgoPvQL2K4BqzCXY8YZlftNZB7AgI0cCJ6CP8NQjUoDMQFeIhL0HJSsj5BBqLMbfWcbDwIAV5BapqlxW3qeVbO/6CgmBeW2zRTz9p2N/jBFlpQJa1l5WVAniH6Gjxe5tM/ldnMuD48x/8pcZffb34W5KOoxp/AIWFhTz99NP069eP6dOnc/DgQV555RUKCwt55513mDBhgqf6GUApQd1UvX3OHXAu5w44V1HtPesICzWU1pfyy4Ff2Fa0Td0XrZAf2uN68e1N2hbP3brBqDZ/lJKoLEmS+Oqir7h/0v1Eh3ouT12W+jzu6/s5Ys1VsPIiX/cCsDj+XNVDH5Q4CHDP8Zc4MJEz3zqT+L7xXLHgCkbdYNtR2txsMeK4m/HnkcX2CYosS6FkEj0gYQChhlBMZhMF1Z5z7Fjjao2/gOPPfdSOH0H6oPaaJXuOea7On7zA6hIQtOufsOlOjx2X8W/D6Tu8LuXcgcptsGQ6HH5f2/2aWuHH/rDqEqdN5cCKnKocwDJeq1mId0vuRtKQJCSdOqtJ3bE6TEbbxquSWuH4iwrytRyAD1h1EfwyXJvafAffhO2PqnYYyO9dOdtFKbKhvb5e1BSyR1RaFDEZMUQkRTD5wcncsvUWh7VYrGU+1daVPCEcf+FpkHYOBLsxp2+php8HiVp8nmDrg/DTAGiyX/98SOIQwOL4Cw21jP2+yvorK4Os4vu5p98brtWwH/J3mPYjoHXkt1lIvFZshkPviPdEqYapOfk/wIrzobXB5sey46/kaNuDbO+5G/gXOK/I7UzHqJCodjn3wtpCt/YlBzhpoSYjO/40l/kECE0gbPjtlDX0xGx28N4tXgobboL9ryvfd8GPsONx2P2s+P+YV2HcWyJ4OcDxw4C7hRz8JQ2szbyPqSumMvGOiVTmVCrehSz1ae34AzjtNNDpYNcu38ot+4L5l8zno5kfUV4q5lkBWUfvote7b9fyFAHHn//gLzX+rGU+fR0goNjxd9ppp5GRkcGrr77Keeedx549e1i1ahXXXnstEcd9xfMTiIYiUUeuuVLzXbs6yG4s2MgZn53Btd+7XlD8hMn4W34m/DbVs8f4bbL408ZIkZzDrl2ePawa5Iw/Vx1/cvaiz+v7AeT/CBtvg4ajHbc3lkBjsde7E2njIXFHphdgUIJw/MmSbp5EXrwaDOqzu2QCGX/qUZPxZ9AZ2HPbHqr/Wk3PaA8YNGwgT5zU3BNmszng+NOA+HjQ6w2q3v3Wcp+eQg4Iku/d+bvn88iSR6jK/h8c+cJjxyVuNER45763S3g6jHoJUmZru1+dAaIHQzfn9ZjljL+cyhzAch3URuC2NLSQuzJXVRT6G4Pe4KMZtmuCldYLy21s8EmY8ReaLBw9WtTPGvManLFb3BMqkI3cSjL+rOcrYWHCmQOOs4slSeKa36/hlPtOUdQfV+v7wQni+LPjnFFFcKyoW512nvLvVGwT82MlmdHd58Ggh0CyL0E3JEk4/qznoXLWn68cfy6pU7TWwfIzIO9bSJwIKTO0r6WZNBXOzRfZniOegTlrIGmy8+8ppXovFPwAlbblfLt3h6SIHK6Ny7BImdoiKBL0wZp0Sc70L6gusLkOUoqWGX/yWkR+V2pGcxW01qHTKVB06T4Pxr0j6jsqJfVsoWTTud6vqRVa613psc/ZWLCRuZ/M5apvr/J1VwBR/3HiuxO54psraNUqG9cWEekg6TDoDOSn5ZN3ah4hUcqD1mxJfYJYF2S1JTnv26dRX/0Y6zFFH6xHF6Sjpk6M24H6ft5HjTS8NwkkqvgP/iL1KWf8+TrbD1Q4/oKCgpg/fz75+fk888wz9O/f35P9OmnQuysg35kjX8LS2VCxVVFzs9nMT/t/4h8r/kF1k+Mnw9VBtrReRHC6FA3ZxgkzkAbHijoZniSyn/jTxuA2Ofs9eyz1kJxR01TDx9s+5od9P3igg3D18KtZd/06Hp6sYiHSRl0dbG9ba2a5oKyjOaVr4cAbQnrNmlMXwmyFxRU1Qq/Xk5WV1WVccTcySq6J4E7Gn1KsDSmuRsbsL9sPBJw9alA7QeoV0wu9G3Wg1OKKAbawppCG1gb0kj7gBHaDxEQdERERVFYqv94PnPIAv131G5cNvcxj/eocWfnj/h95etXTvBtzCZylQpLOFVpq4Mh8qPdOxmsXInqKaG5PyJlO/QZGveC0WWfHn/yeUSP1CbDqn6v4YOoHFG0tUtTebDIz5LIh9D/H9jqkvFHMOePCTkLH39jXYbpG9dZjhrrk4G7P+Kt2nPFna76S3FaysUjBrXBo0SEW3r2QytxKh+1Oasef2QxfJ8DKC9zbjySJutVqaqYeeAtWnC3GSmeknQ0jnm4vU2ALeR66s2Rne20UX9f5K6gupDpsB4YIFZalyp0iQLc223MdA8taUx8C8WO13Xefm+CCMkgYb/Pj7t0hOqSE+pZuEN7D8b4qd8Kuf4lgSTeQ6/wV1RXZXAcpRcuMP48FIR58C76KgbJNzsuwSJKQWw1S4Aw1NosxQ5JE7XqDlTxmxVYxlhx4w83O+waj2ciiQ4tYnrPc110B4FD5Idblr2PRoUUYVAbXKCLvW8j7Rsi2F/xERvlK9g3Yx64rdhGRpDyC0l7GH0C/NlPT/v0a9NeP6TxXOf/T87liwRXtc91Axp/3kW3Svig/5QhZrSLMP5WFTyr8ReqzoS32rrPjT3MfkAIUv2l++MEzDoCTHZOWVUkBkmfAmNehmzKPiCRJ3PbLbRypOsK0jGlMybBf9Ftp/Y3OyLKOrtZzA8tDe9xn/J3yseePMeG9Dv/NyBCDTX09ZGdDHwXlNj7Y+gF3LryTcanjOLv/2Zp3MSE8wWVH8OLF4sXasyf4RfzBkEdhwD3CqWuND/K5TSYTJSUlJCUlodNZ4jrcqfEHItJ6dPfRjOkxRoNeOkaLifSU9Ck0G5sZlzpOkz6dDLgaGWU0GTFj9szC1QpXMv6MZiPXjLiG+pZ6gvRBnunYSUBMjImmpmZKS4NRGi82sedEz3aKro6/plaRWWII6iZkzz1J8VIhqzjq3zDgLs8ey0/JiBHGzNL6Uuqa64iJEQ+n2vdM/7P7Ywgz0C1F2TWTdBKnvXKa3c8rm4XjLzHiJHT8aU1TOdQehnjl7345C9xZxp+t+UpqqlB0yM+H0aMdH2f317vZ/PZmxt9p2/kgc+6Ac6l9qJYGFzLfjnvHn6kJel4AscO8f+ze10DCRGUOBwUMTBiIhERZQxnFdcWkdEtpz/jzldTcNj5i++CHeHHP1Zwy+kNlX0oYDxdWgFlhJKY/Ehzj8OPu3eFA+Thu+XEPn10RjEMb6LFVsO0hiBkGqae73KUX574IQO+Y3hQVFXVZBylFy4y/v0z8C7N7z9Z+7RTZD1LPhOhBxMXBoUMK3rvNVZA3H3pfZ399uulWIbc78ZOuc6jIfiIYJDRZk1PwNlmxwi5WUF1AY2sjoYZQn/bnUIUITpP7pTm7n4Ga/XB+Cex+liEVOwBUZRc2N1ueB1m+15p+/eDnn098x5+tuYokSe1rkEDGn/fx14w/2ckTcPz5HtlnUFcnEl984GcDLPdEaKdXjuY+IAVorC0RQC1mrQt6xwyBfreqitIdniwixrcWbXXYztWsofaMPzeir4/7xbcP0elg4EDx790KE7YuHnwxOknHhoINHCw/6LnOqcRkgh9/FP8++2zfayUDIiIyNLGrHFZzpSioXrLSa10xm80UFRV1GVfclfrsF9+PTTdt4o0zPB/pKb8g3VGQfnDygyy6ahFjUzWOcj6BkR2tahx//1z5T3q+1JNv93zrmU5Z4co7ID06nffPeZ8vLvSg7ONJQFycmaamJr+LrOxc97GpTVIupbVUZBF4kpRZIsgq/ULPHsceS+fA79oH5QBCiu6Pe5zWBIoJjSE6JJrE8ESK64pdXoj3GNODKQ9NISpNm6IY1a1izpnUzcM1/lrrPLt/tRQt+lz6RAABAABJREFUha0PQZ262noOWToLfj9DZIEoRGnGn635Slqa+LtAQSLttEencfHXFxPbO9ZhO0mSiAiOcCnY7Lhfe+hD4ZSPYOB97u8r71tRh09prbiE8dD7amW1UGtzYNlpcPgDu03CgsLaa8fuKhFyn77M+GtthSpERldmvMrMWEO4Zg5Rn9FQDAf/C/WdaupV7SWiYQNRUdBqCm6XCrRL2jkwezUkT3OrOyNSRjAiZQQRQRE210FKkec5Wjj+RnUfxTUjrmmXqdWMnufB1G/BEK7cNrPraVh/AxQvsf25ySjeaa11YLCxADOEw+yVkOkfUplqSQhPoFtwN8yY21UKfIlchiArzkOOvwkfwOT5wjYx8jm29XmIgbsHMvKtkYpr/BW3VSsJD7f9DpQz/g4eFOOhI+qO1VG2XzxcRduK+OGGH2hpaFF2Lj7Geq6y5OElrH1xLaBNoHIA13A1GcXTNLUpm3d28gTwPvKYZTZb5vK+wF4WqOY+IAUEHH8BGJEyAnDu+JMnwXV1loFNCcfqxIrMHanP437xDWJxtOMJKNvo2ePU5cGW++Hob+2bZLlPpXX+krslMyNzBgA/7f9J6x7yxsY3eGntS+RX56v63saNYiIaGQmnnqp5t1yjLheqdoO5U+RGa50oqJ77uW/6ZYUcjeZvEyRbyC/IwKTJu8hZU7W1yiWBq5qqOFp7lC92ed6x5krGXwBtcNWh89mOz3h4ycPtUnta03ky3dgqNswq+gBWnu+RY7ZjiBBBVuGpnj2OPXQhoNOmNlIX9GFw+H0ocP7uL7qviJL7S+gd27tDZrkr6xmT0YTZ5PyL+3/az/xL5lO6t9Tm56MaHmRg3vMMS9JYBrW1Ho4uEv8+8jV8mwqlG7Q9hjscXQC7/wVGDWq6yfS/G4b8HczKswQyojPoH9+fgQkDVS9sU9seJyWOv6i0KAaeP1DV/tVywpQZ0AJ9mJjntnigYIqkh5LlYj7tgFvH3sojUx4hpZvQnZMdf77I+KushPoQYbwfkKxQVr7xGOx/Q1vnvK84tgI23AgFP1q2mVphxTmwfB6ZaZUAzh1/Yd0h8RTbziYfID/zx4sxX3FgZ98/w/j3IGGS7c91epj0OUz9wU+iarVFkqT27LpD5R6WgleA3AePZfxFDxD1QwESxtMUP5E+tX1I3ZxKQ4WyOYK1zKetWyI1VazJmpvhiIME/6aaJj6Y9gGL/7oYgNzfc9ny7hYOLfL9dVCD2Wxm05ub2PeDKGoo17M+XsaKEwl/z/gL2LB8j8FgsRn5Uu7Tn+4Jz2pzBfA+phZYMFIU9h6rLDtHdvxtK97msF14OAQHW1L/baX926K0QZZdck3qs6XFYuA7rqU+q/fCjschJEH7egvWGBthz/OABN1nAx0df7J8vzNmZs5k8eHFrDyykrsn3K1pF59f8zzZldmMTxtPWlSa4u/JisNz50KI8trUnmXLA6K25qUtIFnFUoT1gGk/eaYGk0rczfiTaTG2UNdSR0xojNt9soe7L8j86nzCDGHEh3s42+MEo3NklJKFzCWDL+GZ1c/w84GfqWmqITLEcwO0PY10R+RX55MYnkiIwV8Gi+MTSy0FdcagR5Y+Qk5lDnOz5jKtl3vR/LboPFbIUp8HEk9nfI9Rmh/PJi210FLlfQfg9B+dt3EVSQdn7BLvMCdYy2XJjr+WFiEtrsZJv+2jbSy8ayFXL72a7iMdTy5LdpWw68tdTH1sqs3PUyrOw1wMA2zUpHGLzffAoffEbxM1QEietfq4eIQ1w5+G3tcqlvpXRO+rVX8lNSqVvbfvdelwsuMvX11MmEPe2vQWGws2cumQS5mdNVvVdxU7/goXinrPgx8SWXb+Qu6XIsNn6OPCweIOPeZBDxWabhtvFcEDZ2cLp4IjwtPg4nqni5POaxFfSn0Kx5+o05cVp9Dxd3QRbLpNOLl6/8lznfMGKbPhlM+h+xzLNp0Bxr8LDYXE5sbAbgWOPxAOw7pciHR97DpSdYTPd3yOXtIzK2KWy/vRSqqttL6U7/Z+R5+4PkzvNd29nVlz+API+RzGvAZRfZXXuurWC7pda/uzqr0Q1V88fwYHJ95cARtvg9gRMOgB9X33MVlxWWwr3taebedLZKlPj9SibyoTgWFWWcVTMqYw+afJSCqcuvKza6u+H4jbpW9f2LpVyH32tnMqwRHBDDx/IEHhouTCyOtHkjE1g5QRlh2bzWZVffMFkiRxb+G9NJSLQULO+AtIfXofOePP35RoAsHr/kVkpAgc96Xjz5/qPgYy/nyM5i85XZBYdOqUGzplqc+dJTsdan9LkmWgVeNAcDfjT872k6TjPNsjYQKctg16elgarFsmnLlfGIHa6NsXgoJEdFJhoYPvWjEtQxhqV+Su0DwduaROrNKTIpIUfycnB7ZvF9KlZ5yhaXfcI/1iERHfWepTkiD1DGHQ8BKSJBEXF9dlXJEXho2NlheQWl7b8BoRT0fwl1//4mYvHePuC/LhJQ+T8FwCL659UbtOnQTo9ZbACqVynyNSRtA3ri+NrY0eyQyWMZuFIwHU3RczPpxB+NPhrD6y2jMdO0lISJAICgqmsdFiFFPCoMRBAOw+plBjWiWdF1hyxt/RpHnQ5yaPHLMDTeXwdQJs/avnj+VtwlNVR/0HB1vmaGqjcCN7RJIwIIGWOufST5MfnMwjjY+QMMD2nFIevzQPFOt1FQx+RDjWYgbDmXshZabGB3EDXRBED3LuZPETbM1XZKnP8nJ1Y40jlmQv4b2t77GvbJ+q75nNkNeWmOU00LHwF9j5pBgT/ImS3+Hg2yKjztsEx4ngAUmBqUGSXMoykjP+KiqcS81pTVmFkfrgHECF8T71TJjyLfRwvZad3xAcA70uFTVAF59qyYZOmgwZF7c/M4ocf2uugJ/6gQt1OGWO1hzlr0v+yqsbX7W5DlKC2ayd42978XZu/PFGbvnpFvd21Jm6XDi2EkLEwi65reSenJ3llMZjcNiqHmV9PiwYARtudv5dQxQULYIKx4Hi/krvGPGcyk43X+LRGn97noevoqC67Z239hr43ICkInMfOmb82UOW+3RU50/SScz4xwymPDwFEI7Azk6/X277hRX/t0JV/7yF9VwlKCyIqFQhkRPI+PMdigMevEygxp9/odau5QnsOYN9EegQcPz5GFcKTztl3iYY/ZLi5pmxmYQZwmgyNpFdke2wrSup1Q9MeoAX57zIyJSRyr9khbXMp58HAznGEA6xwyDMw4WxdQaI6tvBERUUJJx/oLzO3+geowkzhFFaX8qe0j2ada++pZ66FqHbp8bxt6StLMEpp0CC66qx2pN+AQx73P7nDUUiStIL6HQ60tPTu4wroaGWF46rsghJEUm0mFrYXeoZI76Mu9FScuayXAsmgHJkuU95MeMMSZK4ePDFAB6V+2xpsUgHKr0vWowtHK44jMlsIiMmw2N9OxkID9cRHx+GJEmqFlmDE0Wquaccf10y/tpq/IUoqSmlBSFxkHU9JM/wzvFkStfDrn8Jg52nMDaJmnHlmx02++3Qb8z+eDb3/novQAe5TzX0ntWb69deT/rkdEXtDSEGdPqu8+faxgay9QupDN+oveMvabJ418uONXlC6qX3u0NaG0TttRaNC1mYzfDbVFh1iUtfN3WWQLfC1nwlIsJiRFMi96kEWWo4Lkxd0a7SUhEprNdDT2cl3AbeB6fvFPWe/Ykxr8H5JRCiUb+KFsO2R5Rp+Q7/B8xdp3zhdmw15H3jsIn8Xt9evB0Qc5bgYNGdUtvKvx7jUEkBZl0LOnMQqZEKM76Do6Hnuf53n7hDUBSUrYfajs4UVY6/tHNgwH1gcjEyEegRKTLUC2sKSeuZ5pJ9pbnZcmu7a7jNrRSytb1ierm3o84M/TtcWAkhIhLb+ndWFKO74WZYdw1U7hD/1wWLTPGeCuTRdXo4OwcmfepCx31PVlwWsaGx3psj2sFkNhGkC0Iv6T1T4y9uFGRcbsn+jx0OqWdRdaSC7GXZNFYpe85UOf52t/D6oNdZ/+p6AEytJt4Z+w4/3/ozplbb84Cc33P49S+/0lzbTP7afHKW5WBsUVhrwovIc5XSPaUc3XIUU6sJs9myVg5k/Hkf2R5dVeX9oB9HBGr8+ReyXcuXjj97SmYe8QE5IeD48zEmk/1FsbfQSToGJoo6Gc4MdLJRR43x7+z+Z3PPxHtcntycEPX9QETZNVe5VvxGLc0VYhFtlcGpts5fsD6YiT0nArD5qGPjnxrkDNAQfQiRwcotc7KUzxCNa6R7lKLF8G13yPnMcTtTK5RYRbod+UpIr6jEZDJx5MgRm+OK4gLwdrDO3vFkQVp3Mv6aWpvaxzBZwjiAclwx2l8yWBiEFxxcQFWjQo+hSqwzP5ROprMrszGajYQHhSs3ygWwiclkIji4BrPZrGr8aB8zPBQsIC+w5LHiywu/ZPNNm5mV/yqs9ZKM2tjXIcuOfJanOPorbHtIzCk8RUsVLJ0J+1912Ky2uZbFhxezJm8NoJ2stCP2/7yf4h3FNj/bW5TLhn6nsb7fHG3njM12Chce/gi+7QFlmzQ8mAtUbIFFE+HA69ruV5JAr76e5BPLnyDh2QT+tepfdtvYm6/IWX++dvxlt8VB9uwpgudssuw02PEURKSLLFBTq5B/3/53N3qsIZIknExaRU3mfw+7noZ6D9So2/4YrLvOYZNVR1aR9UoWl8wX8w5J8p3c54FSIRcYp89AryTLtqncs8EaviKqH1xQKmrIWdGjTSlakeOv1+Uw8hkIjnW5GyndUpCQaDW1smX/FpfsK67MNe2RU5kDiJqnmqO3jMeJiUIJp7lZ4Xt35LNwymcQ3baYDk2CcW8KKV8lBB2/xpgbRt1A2QNlPDP7GZ/2Qyfp2H/HfhoeaaB7NzclmG2RfpFwzsoB4APuYdeAf3DjA7fz0YyPOLZL2dxRdvw5yniXg8pL9lXS0tCKsUk47nQGHZGpkTTXNtNU09ThOwUFYt87Pt3B+n+vp6aghj8t+xOXfn8pAGX7/SuNS56rrPrnKt4e9TbNdc3U14M8xBzXZYiOU6KiRA03sEiu+hqjUYzDEMj48xdkx58/Sn36wgcUcPz5GI8Y0IuXi4WoCsmM109/ne23bOe0vqc5bOeK1Ke7KK6x4e/8cSfMjwFTs+ePtetp+G0y1FoyOGXHn9KMP4DXTnuNor8UceWwKzXrmizzmRiRqCrNWb7nYl1fF3qGddfD6stsfxY7CnpfI+S3HJH9ESyeJuoHgTCAbFQvDyMM8+U2xxV3DbJ94/qil/RUN1VTUKORRc4G8sLblRqOe0r30GpqJSY0hp5RzkL0A3RGHt/VRM4PSRpCv/h+NBubWZq91CP9sr4nlAZI7SsVEjf94vv5fd0If8dsNhMaKrK01QT9eFrqs3MUXVZcFiO7jySopcr7knsOMps0p/+dMGctRA/03DFCk0Sd6H63O2wmZ9PKRk53AkwKNhbwzZXfULLTvgXfbDLzxXlfsPRh22PN0Upx4GBTHHot1Q2XzhX1szu/W+NGQfRgUV/bl4SnCnn3FHU17BQx4zc45WNVX5EkibKGMoe1lOzNV7Su8+eu4y8z006D5kqo2g31Ryzb9CFC8vDoIjD5OHPBbBL90NLZNOAeOGMPhCkIpsn5HPYqV59h8CMw8WOHwZFyTfC8qrz2+0Z2/B3zYByELYIbMhiQ/09mRd2m7AtHF8J3PSH/B892zBcYuhZflp0FpaUWY6gnCdIHkdxNKOrsP7rfJfuK9ZzC3WljbpUHMv5KVsL+10UNtzYMBovkrSIna2Qf6HWZOMFjq9WPU8ZGyP4ECn9V9z0/wKAz+NV6IEgf5LX+NLY2sjJuJasvWE1MZozT9mYzFLfFVyU7EKoKNdWTEGukoVsiM76/k/F3jW//7NLvLuW8j84jLNZi8S4thbvugrvvhlF3T+X2/beTMCCB0JhQgsKDeO+U9/hk7id+lfknz1WGXjWUWc/MIjQ6tN2YbzA4CAwK4DEkybVkFE9iXUrHFRtWAO2RHX++dA7by/jzZBKFPQzOmwQ47ij8Weh797pccaHsCWkTFLVTa9RpaGlgafZSEiMSGZc6TtmXOiE7/o77iJqUWaLmhTckJlLPEtI+VoWdBwwQL8qjR8X1i1NgA5EzQbXkWL1YnauR+QQ/LqJclwutduS1QuJgwvvO95E8Q0TLprfVf+x7K0T21a6PuC7BJhNiCKFvfF/2lu5l97Hd7QYYrXEn429bkZD5HJEywq8Wd8cLsoSumkm0JEncOOpGimqL6Buv7T0r44pm/v4yUXCiX3w/D/To5CMqysjRo+rujYEJ4v1RVFtEeUO5asO7M+zKAs/xck3H9TdB8TI4Y1eHSHyPERwjagZ7mk4ZHLZIjxbSnMV1xTS1NhEbK+Y3rrxnGsob2PHpDjJnZJI0xPb8wGQ0ce6H5xIe39XQDFBcLSanoSYN7zWzCRImiuj5zu+VmCEwd6PvdegjMmDwQ77tgxVyzTNXain5W8afXcdfcAyckwNGqyBLSQenLoDQZGW17TxJw1FYNhf63gZjX9Nmn90U1rIDOPQulP8hnIVKSHEumZwaJRyOdS11VDdVEx0a3e708LbjL6g2kz5Ff+VipeX6IjKhzy0Q5cGADT8iMlJI99bVCQeCU7ncddeJjNlTPnL5mKmRqRTVFlHS6Fr6pzzXDLf9elFFe8afllLzOZ+Imp2pZ7VLfYKQYiwqEn/kIF+n1OWKAOFeV6n8zSVYfx2kzIUec1V1P4AXKFkJO54QkrBJoqYeBT+TfuhTWtOK2NEXIrs7N6hVVAiHvU5nCa6wxQ83/EDm6lLKR9/IgQMhDBvmOOLqm2+EWkdTEyzfEs0VV1g+kySJYVcPo6m6CVOribL9ZRhCDcRliXf3Vxd9RWtjKxd/czH6IO/Xre09qzd954p1rrtlSQK4T1yceO+7qmSlNfI9odMFnMH+ghzQ7st7xB27ptYEMv5ORPreCqdtgXDtDfNqa/zlVOZw5udnMvcT1yeHJ4zUZ9b1QhbMGyRNhUEPQJhFmD0iwrLwOmw/CNvjyBl/ah1/siHR7xx/MxeLOibu0K2XyK4IagtN6X+HctkVhbjr+APPZ/CAe5PprUVbARiePFy7Dp1EuJLxB3DfKffx/JznGZLkGR1eVyZN7Y6/uIDjTwuio4VstJrJc2RIZLtjaFeJQo1pFXSOontm1TM8vfJpyuq9HP4Z2Qe6ZUKTl4pM1RwEoxdSKEA4vVrr7X4cHxZPqEFcgIKaArcyy3tN78V9xfcx8jr79aD1QXqGXjaUrDm2g9pKasUNGiZp6PiTdDDmZRj1gp3P25x+DUrSLTyEJzNOm8qEikTBL4q/Ijv+HGX82UPO+NPC8ddsbKa2WSwiNHf8gbj2nbOdwrpbnH4V21QdU1MMEWJemXGxtvttqYbq/c7bjXsLZi1Tv38HUdDhQeHt1zG/WmQy+krqU3UwYuJEIakY5ZkAKX9Dkiy1wQoLFXyh9hDUHHDrmLJjuKTBtZuhvu1Vp4WBTs7401Tqc8S/YPpCIS1shZxdqeh3tibtXPUBI/oQmPq9eCceh/x92d/p80ofPtj6gc/68NTvT3HKu6fw6XYP1Eqsy4WydYDVOFq6jsT8z4nXgVFhhqecPZqYaJFU7IzZZCZ1fCoxw3piCgphv5PXQmUl/GqVKPrjjx3ldQHG3zGeqY9MpTqvmjeHvMnvT/ze/pkhzEBjZaNPnH6ds3PksSLg+PMd/uDUscbaVuHrWMAAAvke8WVWqL2MP18QcPz5GI9kpXTLhNgRqjLLGlsbeWHNC9z0400OJwVqHX+l9cIQlhCeoLgvnZEdf8d9xp8fIEdSK5IDaeOjbR8x++PZfL7jc036cE7/c1h/w3r+OfOfir/T2Gh5ofqd1Kczjq2BheOg4Keun7XWQ9539g0dKqNWJUkiJSXF5rii9tm1xaAEzzv+XMnuktlWbMn4C6AeOeNPrePP07hijNlfHsj40wpJkujVSwQlqJ08f3XRV2Tflc2k9Ema96tzjb//W/l/PLL0EVoOfygk7rzFwPthxiII7+H5Y7VUw499Yf0Nnj9W1R74Kgb22HF4Ie4NWVb5SNURt6Q+DSEGIpIiHLZpqXcsqVlWKzyOEVo6/pSw/w0h41euXT1kxbTUwDfJsEv5nEoVZjNsewSOfKn4K7LjL68qj2Y7Tmp78xVrx5+7SjgVDeJ+kJCIDolW/L3GRss82abjz9gkavk5qu144C1YMELUbPYFwTEiazdpqrb7XTAaVpzrvF1kH7EWVUr+92K8yf/OYTNZbUJ2/Pkq42933QqqQ3fSLdpLQRjHIbJDStGac8ZSmLvWrePJ9ZzrDfUu2VfcWX9Y02pqbb8/NZX6DI61mWUn/85yTTZFRGTA1G9dkwzvcZq67F8/oqqpikMVh9hevN1nfdhctJm1+WupaPRA3ZzMK+GiakiwmnMP/AsHpiwneMtgLnv+MvLWOq/RKgeIp9nJIWiua0bSSUx5aAozXj4HgO3bHdfS+u47kUXYv794z9fVwYIFttvG94tn2t+nMfa2se3bzvvoPP60XNTvrsypZOHdC2ltbHV6Lu4iSRINfzTwau9XObxE/DD+lMVzsqKFXUtLAlmg/oc/3CP2xgpfKJMFHH8+Rqe0YJFaWmqhuUpx8yBdEI8sfYR3Nr/TLk9hC1cdf4nhDnQCnHBCSH221sGyeXDwv9475vobYMGoDpvk6Es1i4O9pXtZfHgxiw5rY0iNDYtlXOo4Vc4ZObI2ONgPX6i5X0Lx7/Y/N3SDumzb2SAH3oKV50GOjai/NVfBD71FPQWF6HQ6UlJSbI4rWmT8Tew5kYsHX8zUDI0NSVa4M3G6buR1/HnMn5mYNlHbTp0kuCL1KdPQ0sBvh35jbZ57hhtbuHJPnD/gfK4cdiUju9vPHgqgDJ1OR58+sUiSpHryPC51HL1ieqHzgOxd5yi6xlZxoyTtfhz2PKf58exiPXk3Nnn2WKZWGPgApJ7h2eOACCKLHwMRjjXa5KzOvKo8t98zdcfq2PXVLqoLqm1+/vnZn/Pm0Dft1kYoa5N1jNBr5PgztcLCsbD3347bJU6GpGm+kXdsOCqkwYOUO7ZUEZoAc9bDGOVykckRyYQHhWPGbHdNYW++kpwMer1w7LsbpZsUkUTtQ7Xk3ZOHXqc8SyA3VzgdY2Ig2tbPemy1GGOKFtvfSeqZIqMmabrKXmuEp7JA+90GWdc5b9dSI9Y+SglNhrjRYs7sAHuOP29n/C3odhErhgyl2KQgm91sgt+mijIcJxE92mJhFDn+VDyf9rh34r1svXkrT8x5wiX7ilaOP4Bfr/yVd89+l5RuKc4bK6E+H+qO2PxIlYNVK5qroNkDjisPIyuT7CzZ6bM+HCoXEthZscrK8ahG0nV8noJjILwHJsCMuUMyoD327hV/DxjQcbvZZObbq7/lkzmfYGoV75iBAyE9XQTqf/CB7f3V1MDPP4t/X3IJXHSR+Pe339qvATr98emkje/oedTpxXO959s9rH95PXu/3+v8ZNxEp9MRExtDaGwo4Qkiwz/g5PE9/pDNZY0/ZXYFEFj7LXxQUg+wP1Z4zAfkgIDjz8cYjR4oXlt9AL6KhN3/UvwVvU7PgATxdneUzSM/QHV1lmh7R8j13NzJ+JMdf8e11GfjMaG77qaMiSqkILGAtjJEurI4kJ08K3JXaNk7VVhL6vhd+vy6ax0/azFD4fwS6H1N18+yrhXZIukXdf0saSpkXCaMJwoxGo0cOnTI5riihePv9L6n88WFX3DlsCtd34kT3Imiu3LYlbxxxhv0T+ivbadOEqxlM9S+ml5Y+wJzPpnD82u1N2y5ck/cMf4OPj7vY4/Jj55MGI1G6uqOtBW493VvBCaTxVgQGgoms4kWk8gGqx77Dgx7yssdMsLvZ4s/niQkDkY+AxmXePY4APpQmLnU9rvLivTodBLCE2gyNrkl9QlwZOUR5l88n0O/2q4NlzgokR5jetiNlJTruUUaNJIGqM+DxmLnBs7YYTBziboMJ62I6gdz1kC/Wz13jIRxEKR8Ei5JklO5T3vzFYPBEqSWn+9ad637EREc0S4BqBSnMp/Jp4qSCplX299JeJrIqAl1PfDRLRaOVZaZp5YBd8PA+5y3Wzwdfh6qfL8JE8Qz1H22w2ZpkR0df/LcVl4neIOqhloaDcLTODRNQeZTU6lY/9lx3JyoyGvOI0pOu7Uesj+GoiUuH69PXB+GJA7hWP4xl+wrWmXxGHQGZmTO4LqR16kKOHDI3n/D9xkiE78TXnf8lW2C+TEiePU4Y2iSGJN2lOzwyfHNZnP7OzErTmPHn7EJ9r0ClZ2cmi3VhNbnkj10F+/d9R49T3FWcNPi+BvYKSFU0kmEJ4YTFh/WrsCg18Ptt4vPFy2C3TbMiD/8IJ6v3r1hzBiYNk3IiFZWwmIH8TP2GHPLGK5ecjVDLvH8Gs9oNBIyIoQb/7iRlOFicqJlkEAA1/CHbC5rAs5g/0O+Rxobu8oKewt7DmGP+ICcYEe1OcBxTXgPsRiNG+W8rRWDEgexrXgbu4/t5qz+Z9nedbgoWmoyCedfiBM1US2lPo9rx1+3XnBxLZi9+JCPe7PLJjn6Uk0dgFN6noJO0nG44jA5lTluy5a888c71DTXcN6A88iMdVQ8xYJsRPRLmc9Jn0Owg+wCR57K4FgY+aztz/rcKP6opMaOzoa/TZDsIU+cnI0tAbQnJkYs4IxGsRiTHYFKmJM1h0eXPcqSw0toNbVi0Gk3vdCy7koA19Drq4Ce7VFzSgMwKhoqeGX9KxTUFPD2WW9r1h95nAAxmbaWE9T1OA1CojQ7liJ0elGjVdck6u/pg717fB/y37P/257RKc/XamuFYzZY5c+QPiWdc94/h8yZtucGp71ymsPvj4u4kIN5vRjSb7y6A9ujWyackwsmFXJ+DUUd6is7xWTUJNvFo5haoHofhPUQzmcFTEmfQvdu3Qkz2B+47c1X0tKE1GdBAYwY4UqH3cOp40+SlDt5TUah6tBSCf3v1KB3CjCbRKZuqEbZRq6Qdg4Y7dcHdZUz+p1BUkQS03tNByzzgoYGde8md9iZL26QoNZY0hIVZNqGJsH5R8W9cBIxeLD4e9cusfaIczh0mGHtnyD9QkiZ6dZx7Y0rzvDrjI2U2WIcjuoa2CgHStTWij8et5dEDxSBqdHHX2CdXKu+qLaI0vpSt+xTrlBQU0BdSx0GnYHMGGU2EMVU7YY/7oIhj0KM1bXJ+ZyeG29hXkwUf5icG1LKykQGtSRBPxvVEmY/MxtJL3UIwBo4EObMEY6/11+Hl1+21AasqoLvvxf/vuiittK4BjjvPHj7beH4O/10dacaFBZE5gyNfz8HdB5TAk4e3+Nvdq2A/Kv/ERoKERHCZ1FWJvwY3saf7otAxt+JiCECJn5oO4vIAYMTxQx91zH7siWSZHlo6hWs5wJSn1ZIEmhoDHcFeXFQXCyct0qIColqz/r7YucXbvfh5fUv85dFf+FQhe2IfltYZ/z5HWlnQ9Jkx21qDsKOp4ThDGD3M3DgP17NO5edptXV0OqGJL7ZbOZQ+SGOVHkmctnVKLrVR1azoWADLUbHNaAC2Eens0yk1db5G919NLGhsVQ1VbGp0EHdIxdQu8A6WnOUA2UHAveChkRFCYNla6sYQ5Si1+l5/PfHeWfzO1Q2VmrWH/me0OkgKMgi8wkQavDRSnziRzDlq65Ov22PwA99odZ25pMqNt4Kqy/z3rujarc4noOaidYyrhER4nqAaxk4EYkRjLhmBNE9XZOtzNRNI6v4LwyOdPJOVoMkKa+ZvfVhkZVRrzC6KvcLWDQRarNFvd8GlSkbdbmw+nIhPelJ8r6BX4ZCoZ2CPDZ444w3WHTVIqb1mqb6cNZ1/txhWfYyrvv+Ov67WZ3MvkPHX2MJlKwQRnil7HlGSIPaqXeoOZIOpn4H4zyQkdNUDktmwI4nHLcb+hiMUK48A8DO/xP1Mh1wdv+zeWrGU+33lbwmNZvtS8Zpza6jYiyPbO2NXo3P3t8d/BrTvbtwCJjNsGyZk8aGCJj2A4ywEwypgPqWep5d8yzPbH/Grhy0I+T1h7vGwd9zfufdze+yq0SBDKxSesyFMS/blJMOCbHM3b2S9WeIgEmfQZrtIHF/JjIksj2A2Rdyn3uOiYzNPnF9CNIHabvzyL5w6iLo1UmVJ24UDH6Y/0xaypdBX1KR7VjBQM7269Wr41q87EAZzXXN6Aw6m6oL11wjpLGPHIGvrErbfvKJeLb69oVJVqUHR48Wf+fmKrdJdaZkVwmv9nuVY7uFwpjZbKaxSnmJFCW0NLSw/ZntHP7NMof36yCBkwRrlSJ/IOAM9k987SD2p7Ei4PgL0I4cBeVI6hMskwA1jj8tMv6Oa8df5S4oXKiu3oUWHHgLdlsWUQkJIsqqtVWdYf/yIZcD8NnOz9zuUlGtKDCopu6BX2f8KaF6Pxx6W8iGGRvh4Ntw4E3nhqPCX+HnIVC6we0uREXRbqBwR+7zL4v+Qp9X+/Dyupfd7lNnzGbXJ05/W/Y3xv93PB9u+1Dzfp1MuFrnT6/TM7O3iNJedEibeqAyap3B/938X/q91o+bf7pZ036czBgMEB0tDGlq7o2okKj2mkzO5hZqsB4nJAmaWi2S1kHzY0WNVG8jG+TMZvjjbiheLv6fOAVqDwr5bXepOQDVe72ned1UBrn/UyxTLknayEo31zW314+R+fnWn1nysGMZOK0Mt4B4P299GI6pqFuaeAqkng0mhYan2kNiXlCxTdT73f6ouj6WbYDcz6HeTQ+ZMxImwKAHIXqQZ4/TRlpbWR93pT63Fm3l/a3vszR7qeLvmM2QkyP+bdPxl/ctLJ4G+d8p26FOD5O+gNN3qM8Erjmorr03CI4RcoNNHiisc/AtOPyBqq+EhFiGQ29JOe0/JgzAsSiQ+QQROJH37UmX8Qcwsy15b8kSBfEqqWcKhRwX0Ut6Hln6CP/L/l+7/UENWqlLfLL9E2748Qbm757v3o5kFDgxfVLn7zhFlvv0ieOvVDj+5PI6mhLUTUglR3VK04sfC8P/j4LdBhbcvoCSHY4LotqT+fz2ym95te+rGFtsj2ORkXDDDeLfn30Gq1eLd+mvv4pt11/fceqakiJUIZqbRUC6K5TsKKH8YDlFW4swNhv5YOoHzL94vkuOf3uU7ikl+3/Z5K3Ja9/mT1k8JyuyQ6emxntBP47wJwdPAAu+dPxZlyXxh7HiuHL8rVixgrPOOosePUR9j++++67D52azmccee4zu3bsTFhbGrFmzOHCgo7GivLycK664gqioKGJiYrj++uuplT1LbWzfvp0pU6YQGhpKz549efbZrtFnX331FQMGDCA0NJShQ4fyyy+/uHRO9uqUuM3u52CVuhowsuNvT+keTA6KwssGFSULrGtGXMOLc15kRuYMVX2x5oSQ+sz+EJafJiSgvMnh90WUcdsESKezZP2pWRxcMOgCgnRBbC/e7tZEucXYQlmDMBaocfz5bcZfYwl8FQ1b/+q4XdIUSJ4BKbNE3aR5m2HqN86NQIYIYVypV5ZdJ0kSPXv2tDmuSJIlOkptNpc18oJpc9Fm13dih+Zmy/pWzQuy2djM+vz1AExO1zDT4yTEnXtkTu85gPaOP7ULrH1lIrO2X7wNjZoAqpHHlfh4Ma6onTzLdRa3F2/XrE+dAwTkjL9QQwhS8nQhReUrag/DofdgX1twRI95cGmrkN5zlxm/wVxtM2odkjABLq6HfrfZbVJUW8Ssj2Yx+m0Rvi07/lxdZG14fQP/ivoXRzdbJilmk5mDCw5SuNFxJt32qhVURGxAH6KBB6BiK+z+JxxdqPw7qWeKrM9uCh0Cgx+Gs/YJWcSRL8Cwf6jrY/pFcH6xOK4nicgQ2VtxI1V/taHF9rVwNF/RKuNPrvkYF6ZMnhSEAbKhQQQ7yP3oQPJ0GPqEkN5TSswQ4TBTw96X4af+oj64Wnb9E3Y97ZnMYEkH5xXCmFcct/vjbtj+mLp9z1wGpzp+3owmI4fKD7EyV/wukmR5D3jL8XeoXDj+koIUPue7n4W1V9nM1jrRmTxZGPfz8uCgEj92c6XIKnWBEEMIyd2SAciryXPSuita1e3KrcoFICMmw70dyez8B/wyXGR420Fe2xd5y8xQvEzU8VQTGOMnjEsdx9geY4kM9n5EuYRERnRGu8qWpjQUOxzzs2Zn8adlf3Ja429PWxnJAVa+SbPJzPA/DWfsbWPRB9nPXJ4+Hc5qSwR98UV46SXRpVNOsUj/yuh00LOtK7n2b22HDLl0CI82P8rQy4eiD9YT1y+OuL5xmFpcTCG0QfdR3bl6y9WMvnl0+7ZAdpfvCQ+3lBPwh6y/wD3hn8h2LbUB7VrQuSyJNR7zATnguJqB1tXVMXz4cF5//XWbnz/77LO88sorvPXWW6xfv56IiAjmzp1Lo9WvfsUVV7Br1y5+++03fvrpJ1asWMFNN93U/nl1dTVz5swhIyODP/74g+eee47HH3+ct9+21KZZs2YNl112Gddffz1btmzh3HPP5dxzz2XnTvUOEZ3OQ5eg/A8o+EGVFE1WXBZBuiDqW+rbi6bbQo3U54zMGdwz8R5G9xjtvLENzOYTJOMv4zIY9zaE27IieJCJH8FZ+zuEWLkSFRgXFsfZ/c/mrH5nYXQjYrWkTkSZGXQGVYYY2fHnfxl/EsSPh3AnBt2gSCG/KxMcrcwwmDhJGFjSL1TUG51OR3x8vN1xRYuX36juonbolqNbNI2og44vSDU1/rYc3UJDawPxYfH0j+9a/yKAcuSMP1ccf7OzhCF0Xf46qptU6EE6QW0UtuxgCtwL2iCPKwkJrjn+RncX7/+NBRs161NnA11KtxTW37CeJVcvhek/C4eKr4jMgrnrYOLHlm2yxFvREpFF5g7elIvTBYGDGm0AkcGRLMlewuajm6lqrHI74y9pSBIDzh2AzmB5j0k6iTsP38mFXzp+F75bez6rB46nSq+BrGrsKDhtK2Rd59r3mx38AHnfWGQfg6LEHG3gvepqA8qEJoHBB4UrnJBbmUvSc0kkPZ9kc67gaL4iGwNLStxz5rji+JNlPtPTLTWKOhDVX8hYqnXkmU2w71XhkHPYzizujR6nQcpc19YNB98RMrKeMiwo2W/et3D0V3X7jezjtIZkXUsdfV7tw9QPplLTJGpBWNf58wZ5NeIm6RGm0PE38jk45TPvZWr7ERERMGGC+LdTuc+yTTA/Fg7+x+XjpUenA6KWmlq0cvzlVOYAtEtKuo2kg9ZaCO1ut4m8ti9UqDLtNmaTsDUpDEz1J/429W9suHEDfxrxJ68f+47xd5Bzdw5PnfqUtjs2m+GnAbBsXtfPyjbSungGT/46kauyryIo1r76RHMzHGqrxGKd8SfpJMbeOpapj0x12pUbboCxY8W+Dh8W79FrrrHdNl08rhxx4zayniue/d+zOf2109EHazdP1uv1ZI7IJDrVIkGv1VgRwHWsA9r9yfEXuCf8C19m/Mn3hF7fdT3hMR+QA44rx99pp53GP/7xD84777wun5nNZv7973/zt7/9jXPOOYdhw4bx0UcfUVhY2J4ZuGfPHhYuXMh///tfxo8fz+TJk3n11Vf53//+R2HbTOnTTz+lubmZ9957j8GDB3PppZdy55138uKLL7Yf6+WXX2bevHncf//9DBw4kKeeeopRo0bx2muvqT4no9FDsh8TPxBR2jrl0lIGnYHNN2+m9qHa9omzLdRIfbpLXZ0leOm4zviLGwl9bhTZXt4kqj8Ed/SWuSoH8tVFX/HDZT8wPGW4y92RZT6TIpI61AVyhmxA9LuMv9BEmLHIYTaEW0iSKkOB0Whk7969dseVxLZSm+5k/A1KHESwPpiqpiqyK7Nd35ENOsv3KWXVkVWAyPbzRQRNOyajyPQp+Nl3fXATV6U+QRg5esf2xmg2tl8TLVATRVdYU8iOkh1ISIHsT42Qx5WYGBFBq/beGNtjLAAbC7Vz/HW+J0IMIYxLHccpPU/R7BhuET1IyC5ZYzaLzI8Db0Cd+mwEyv+Awx+6nA3hMhXbRE0zO0QER7Q7VvKq89oXWa46/npN68XFX19M91EdjZySJBEWa39FbTKbaDCLgyZ2U+7osYtOD7HDRbabGsxmWHY6/DbZdvR94a+w8gLY+kDXz4zNkP2JcBA7o/oA5PxPZMh4g4PvwPeZiu/d5G7JlNaXUttca1N2z9F8JTLSsliXZTddoaJR3A9qHH8OZT5batyQa5REBmneN46bHVsp6kTWZsOpvyjPHrXmzN0w9XvXuqmEujyRkVjpIOD17IMwy/64YZPWOiEjarQvlRsVEtWeqSM7d7y5LgWYGno7A/KfZmjMJOeNQawB0872bKf8mBltoj/LlzupMR49CDIuh+ghLh8rLVLoBOdU5Kj+rhbGfJPZ1F4DPSNao4y/IY/A2YccqsTIa3uvZfwlTYeLqiBDnbpUAIHma1VTE2ReKQJGOtNSg75iM8VlO1mVu8ph/fNDh8QzGh0NySJ5lobyBupLlQ+uOh3cf7/lHXrWWZb7szMZbY+Iqxl/nZF/V1Orid+f/F1Vv+1xZM0Rtq3d1mGuEsju8g98Xb/NmoDUp38i3yO+yPizvic6D/ke8wE5wFYs43FJdnY2RUVFzJo1q31bdHQ048ePZ+3atVx66aWsXbuWmJgYxowZ095m1qxZ6HQ61q9fz3nnncfatWuZOnUqwcGWydXcuXN55plnqKioIDY2lrVr13Lvvfd2OP7cuXO7SI9a09TURFOTpQZNdbXIhjAaje0XXpIkdDodJpOpQ3SsvL3zDWJvu06nQ9KH2twOYOpUQdd6+8D4ge390uv1mM3mLu3Dw8X2mhoT1ofQ6/Vd+r7g4AISIxIZkTwCg85yuyk9p6oqMJt1hIWBwSC5dE7W2DsnW323t13T6yR5+JyMRsy12dBSBbEjAOjeXbQvKDBjNJq9ek4F1WKRnhKRgtFoVHxOFRUSZrNEbKx0Yl4nR+dUvR8p52PM6ZcixQx2ek4NDQ1dxhT5/7Gx4nc8dgzAtXPSoWNY0jA2Hd3E5qObyYzJ1Ox5qq01YzbrCA42YzKh+DqtPCIknyanT/btGHF0Afr112Ma8gS61DOOy3tPONf1HDtmxmi0tFf6PL1+2uskRyQzNFlIwmpxTvX1wn4eHGx559g7p18PigyD0T1GExca136cE3qM8PA5yeNKbKwJkCgro8O94eycZMffrmO7qG6oJiI4wu1zEkFBOkJDO/3uxkZ0Ox5DSpyIMbVjkJhfXKcJn0BTCVJYKrq2fSu+TrlfwJ7nMJ62CwzRXjsn87probkC05kHO2y37nvPqJ6UN5STV5VHdPQgzGaJ0lIxx9Diedr6/lbi+8WTMSXD7jlV1FdglsRn8eFRAK5fJ7MZytYixY1BFxSm+nkyRQ+GsB6Ym+vAENbxnBKnIw1+FHPWzeja9tnex8ZSdOtvgMTJkDzD4XWSsj9Bt+tJjKcuRd/9VI+PESazDkkfhqm+EEJ7OL33gqQgUiNTya/JZ3/p/vZa3/I5yeOKveuUmamnrMzMoUMm+lmpNqs5J1laPiYkpkM/Hd17xcUSZrOZlBTLHFk+J/P2x5GyP8A0ZyNEZKh/nmYsxRSegbnTcTucU2MZOl0w5oheljGicAGEJEDcGIXPUxCE9YS2ebbmY0T1AfSb78Y0/Fl0MUPs3HtBYruN393u9dvzb3Q7/oZx1lqIH2v3XNOi0thTuofcilz6xvYlJETMbeW5gqfH8tSGOWQdncvAeJOYFzt6nprrRcZWWyDuyTiPGDbMSGysRHm5xPr1Jk45xc456cPglE/Edqt9qTkn2fGXV5Vn85l3dE51deI+Cg42YTK5dp2OVh+lydiETtLRPaJ7+7E8fZ2SksT6qbBQ3fpJyTnZvPckXdt2166TknPy9Ly8uS3rPlgf7JXnSWnfXTonXQimkf8WG9o+bz+nxGnUn5XHxj+N5/EvL2F3n92MvmK0zXPavduE2SzRv78Zk0mcw9eXf03uilwu+e4SMmdmduijvXMNDjby5JOwZYuQ+bT3PPXsqQMkcnNN7e/bDn13cdz74twvOPDzAfrM60NoXKjL957JaOKT2Z8QPSSagSstKZDyWBEWZvt6nKhjub+dU3Q0besN+2O2t86pvt7c9v4Qc8fAdfKPc4qJEfaC8nLJ6+dUXy8BOkJDbY9v3uaEcfwVtYU4JcvhKW0kJye3f1ZUVERSUlKHzw0GA3FxcR3aZHYK85T3WVRURGxsLEVFRQ6PY4t//vOfPPHEE12279mzh8g2Dcu4uDjS09PJz8+n3Cp0ISUlhZSUFHJycqipqWnfLmruxHPgwIEOcqa9e/cmKtTIkU1fUh+UTkuQCLPp378/wcHB7Nixo0Mfhg4dSnNzM/v27WvfptfrGTp0KDU1NRw+bJFMCg0NJSxsAC0tLezfX8iOHZUAREZGkpWVRUlJSfvv0Ghs5KyfhND3jit20FprCfNTek65ucHU1KQSExMGBLN79+4OD5VW5zRgwAAqKirIy7NEMts6J5ev06rR1IaMIDvl8Y7XKSrKw+dUTvTiUTSG9OZg5gdERkbSvXsWzc1N7NxZw44dBarPqaCugIbwBi4ceaHte8/BOf2x7w8Aws3h7NixQ/E55eb2orXVQExMN89eJ3vPk71zaimkdO3TVEVOoz58mBvXyf451R5cTp8jT1NQ1oix710Ozyk3N5fy8nJ27dplVZfLck51dVHU1MRTUBAChLp872UEZ7AJ4fibkzZHs+dp+/Y6amp6EBzcSklJi6LrZDab+T37d0A4/jz6PJUVE7TmfJqDelCY8hciopPJyujBsaPZHC1rBnMqcd0fQ4q8mJ7g2XvPQ+NeU1MckE5+fgM7duy3eZ0cnVMKKVAHVSFVmp1Tba2Jmpp6CguL2bGj3uE5LTi4AIBRkaPa9+XTMeJ4eD85OafW1lbKy8uprs7BaMyktFRSdU7d6EZSaBLVLdUs2LCA4cnD3T6nPXu60dycRmhoKDk5Oewv3s+igkX0CAnn/toXofVGDtQP8ePrlE96goGiI3spabRkIzm8Tv1uI78xjdKcepB2eO2catNvp6y0mEoHz1O0JByRR6qOEG4spaYmhL1769mxo9ile+/IL0dY+epKxj43FrPRzK93/ErPST258tcr7Z7T2q2i1pDOGE55SSHQ1+XrFNKUzcBD51Pd43qipv9X9fO0P+w6GnVNsOeg5TpFhLB79/6263Q+HDxG//5xXa5TZOpz9BpzGc2NjQ6vk6F1KlHdH6W+MpEB3fH4GLHfOInGtNFQABTsUHTvJQUnkU8+K3asYELahA73ntlspqItLdTWdcrMHMDatS2sWVNMWlqZS+dUUi3k5VuqWzr8xo7uvcrKKGpqaqisLGHHjtoO51RYFUJ0UD8OH6oEqdrFeUQZeXmW9l3PKRMyviWuKoz0aDh6aAOpG8+kLOZc8nr83el1yts6n6ZmMw2h/UDSeWSM0JlCCU9/C5N5OP2wc+9Fl1JW1URBTZTT6ySf01FjX/QJ11OeU0Zz4Q6716l7RHf2lO5h476NJNclU1vbnZqaUKqqQjGZDB4fy/fvh5qaMCoqjpGfH+zwnEq3/IekQw+Q3fMFqiOnnpTziL17d9OrVzS5udH88ks1o0YleOycIowRABw4dqDD/pWcU35+JDU1IRQWFlFRkeDSdfp1kwg+Sw5NZu/uvW6fU3T1b8SYcoid+CQlFS12r1NZWSU1NRnU1EBeXgMZGZ6fw+Zu+4nIuvWUxl6ALqibX9579s5p9n9ns7J4JS+Of5EpyVO88jx9v/l7blh8A2MTx/LMmGe8utZoMbVQHVXNrkG7uLD7hXbPaevWBmpqWgmTitixo4HIyEjG3zWe7tO7U5NQ034MpdcpNlbUDLR3TvHxWUAke/bUsGVLdrscnrvjXu+7e5NxVQaJgxNZ9/Y6/vjPH0x8YyKGMIOqe8/YZGTMQ2NoCW4hNzeXuro6AI4cSaGlJY6wsOCTaiz3t3Oqr5eoqYli+/ZKJkwI8ek55eTUUlMTQXFxGQcONAeuk5+cU1lZCDU1qZSXR3n9nGprewExNDZWsWOHJa25d28XlDw0QDJrXaTJS0iSxLfffsu5554LiLp7kyZNorCwkO5W+eQXX3wxkiTxxRdf8PTTT/Phhx92uLkBkpKSeOKJJ/jzn//MnDlzyMzM5D//sejL7969m8GDB7N7924GDhxIcHAwH374IZdddll7mzfeeIMnnniC4uJim/21lfHXs2dPjh07RmxbQRRNPePFS2HpLEyjXsXc98/t28GxZ/xI1RGeX/s81U3VfHz+xzY94++/r+e778xccIGZq6+29LOzZzyvKo/MVzMx6Aw0PtxRtkXpOa1cCc8/r2PgQHjuOQXe/qMLwGxCl3aWzXP1SQSD2YTu93mYYkdhHv6vDn33RlQG+1+DkETM6RcDUFys56abzISEwBdfmNrUJJWd06ojq5j+0XTSotLIuyfP9r3n4JzK68s5UH6AYH0ww5OHKzqnxka45BLRbv58EYnpN5EmxcuQls5U9Jy5fO+11EP1LogZgaQPcnhOzc3N7Nq1i8GDB6PX67uc05o18MwzOvr3hxdecP3ee/uPt7l1wa3MzZrLgisWaPY8bdli5u9/19Grl5lXXlEWsbqvbB+D3xxMqCGUqr9Woaejrr/mz1Pu/5A23YZp9joIiUf/20TMMUMwTZrfsY+mJkxSEGakjtv9PHKrtBRuuEGPXm9m/nwxPlj30ReRW3fcIeTXHn/cxMiR9s+p1dRKyospVDZWsvKalUxMm+jwXP0lGs26j/4YYWc0Gtm1axeNjUN4+mkDmZnw0kvqzim7PJvUqNT2zH93z+mnn+Cdd3RMmSJx331GFh9ezLzP5jEscTBbrvwBXXAkxqCO8n5+dZ1MTeh+7o85biymSV91Odfj7d67Y8EdvPnHmzwy5RHOiniSp56SyMoy8+KLrmX8rXluDcv/vpxrVl5DysgUCjcVYjaa6Tmxp91z2pC/gQnvTSC0OZXNV+YycKAb59RQiHT4fUieji55quvPk9kEldvQxQxGWjIdU9KpmIf+o13z5UQZI+z1/YYfb+CDbR/wxLQneHTaox3OSR5Xhg0b1t4fa1at0vPcc2YGDDDzzDP21xqOzqn/6/05VHGIFX9a0UEG2NE53XWXxOHDZh57zMTo0V3Pyd65WuPwOpVvw9xwFFJmd9ze0oC5dB0kTe1yTux/A3PSNIge5PQ6mRdPh/JNmM4rB53ed2PE17GYEyZgmragSx/dvfdu/OlG3t/6Pk9Of5KHJz/MP/4hsXGjxB13wJw5nn2e8irzuPWpLdTmDOCZB7MYM8bJORUsRHfwLUzD/gFRA/1+LLfuu5ZjxKJF8PrrOkaNMvPEE5L9czq2GrY9jGnw3yB5pupzmr9rPpd+eykTUiew6lqL5LySc7r1VomCAon/+z8TQ4e6dp1e3/A6dyy8g9P7nM4Pl/7g9nWS1l6BlPcV0oUVmPTdHF6nK66QqK2VePVVM5mZnp9HmHc/h27bgxinLYCU2X5779nq+6XzL+XL3V/yf6f+Hw9OetArz9N/N/+XG3+8kVmZs1h4xUJtz2nLA5gbj2Ee99+u84umSkxFixn++cXsaYbS+0uJC4uzcV31XH21mebsAoZv/4Qz3zqdwZcM9uh1kiQdl1wi0dBg5rXXTO31fbUc99a+tJbfn/idKxddSffR3du3Kz0nk8nErl27GDRoEHq9sDE8+KDE3r0SjzwiMW7cyTWW+9M5ffWVmY8/1jFjhpm77/ZCprODc3riCTObNkncfruJ2bMD18lfzunYMbjhBh1BQRJff23GbPbeOf3xh8RTT+nIyjLxwgsdM/4qKyuJi4ujqqqKqKgovMEJk/GXkpICQHFxcQfHX3FxMSNGjGhvU1JS0uF7ciS7/P2UlJQuzjv5/87ayJ/bIiQkhJCQkC7bg4KC2l8iMvIN25nO7Rxujx4CY99ElzQNOn3uaD8GvYE3Nr2BXtLz9llvExYU1qV9WJi4wRsbpc677tD3iiYRyZsQnqCu71bbi4rE/CUtTUH7xmOw6kKhcd7zbLG9IQ+69erQVpIkm/ux97ur3d5133qYucRuQU1XfxtrHJ7TgDs7bEtKAr1eorkZqqv17drH7e0dHHN49+EA5FfnU9tcS7dg24UX7fU9PiKe+Ih4VedUXS3ugZAQWTfbU9fJhe0JE+HM/ehC4hU9Zy7deyHdIHG8ovZBQUFkZWURFBSEJFkcTvIxk5PFbynrXLv6G5za+1QemvwQk3pO0vR5amkR/QsPl5CbOetLVlwWK65ZQU5lDsEO6l9o9jz1uhRih6OP6S82Jk1Gih6MXqdrX2xRuBDWXIHulE9s1lvQ5N5TuV3pdYqPF6dhNErU1uq71NVU8jz9sO8Hvtz1JZcNuYwz+p3hdt+bmkSfIiL0HR6zLuckwdcXf83iw4uZ0HMCep2yd6tHxwiV273/fnK+XafTkZWVxbFjBkCiokL9OfWO7xrd5s45yWNFaKg4ZqtZKAoEB4WhixLHst1DP7lO+nBIvwQpaoDN/XTZZmqB5hr0wdFu993ednfOKSMmAxAZf/HpOiQJKis7zhHV9GXsbWOZcM8EdAax4Os5vqfT/VQ1VwEQ1BpHZKTevXPq1hOGPea0vdNzWnsd5H4Bs1cCkqht3LnCu639tNbB4Q/Qp57ZpcagJEnoaw9AWHewuh+8MkYcXQRFi2HEM+3vO0fts+KyAMiuym6fk8jt5XFFXtR33k9mpjjX3FwxH7Ca0ig+p223bKOiscLuOsTWtooKcdyEhI7vG2fn2hm7996qC8HcAmdndzgp3bqroOAnOG0rRA/oeE4Dbld0TABp0F+h/gj6oGBF7V0+J7MZ6nIhPA2dzoY5YcQ/kcK6q3r+lPalZ5QYDwprCtHr9UREiJ+ysVH87clxb+GhhfwUdRMJvWYSH7+4fa5q95xS50HqvC7vo5NtHpGaKq5NcbHUftvbbK8Lgspt6BuPdlhXKT2n2X1ms+rKVQzsPlD1vSffP9266RWvQTpz5fArGZw0mBBDSIc2Ll+nUz6BmscgKMquHUFu36MHHDgAJSUSmZmen8NKvS6D+NHoEye1Xyt/vPdsMS51HF/u/pI1+Wu0uU4Ktu8rFYkHgxIHOT2m6u3lG5Eai2zPL5qK0K+5mMu6wWPlIkjS1jkVFYl5W7eQYKJ6RBKXFdfhfW0LLa5TRgbs2yeRn6+nVy/X92PvOo25eQzj7xiPztCxr0rPSacTmfPWthV5XRoWdvKN5e5u1/KcEhLEdaiuVm438sR2SZJoapK62CoC18n35yTfI62tUFMjER3tvXNqFmrShIfruqwn7O3Dk3j/iB4iMzOTlJQUlixZ0r6turqa9evXM3GiiPqfOHEilZWV/PHHH+1tli5dislkYvz48e1tVqxYQUuLpfDtb7/9Rv/+/dsz8yZOnNjhOHIb+ThqsDbOa0pYMvS9BaIHOm9rRVpUGindUjCajWwp2mKzTXi4+NtZEfXS+lIAEsMTVfXBmvx88XdqqoLGIfEw+UtImSP+X7wMfh4I+15x+fgnIgYDJLZdkqNH1X03JjSmvU7LwfKDTlprQ2Wl+Lvt8fMvDGEQ1RdC4py3dQdTC5RugAbbGcUykiQRFRVld1xJEJeO8vIO5TNUMyBhAE/PfNquU8dVXCmMHGIIYUrGFK4afpWmfenC7udgw5+hpQZiBlu2T3gfBt7X0SoZPQjCUsDU2nU/fo7BYHnWSktd28fS7KV8uuNTftj3gyZ9ku+LsDDH7fQ6PTMyZ/D0zKc71JQN4B7yuJKQIO7xqioxgfYlsgqIPFY0tooN4fpgqM2G5gof9UwFo56HPjcoa1uxFebHwJ4XPdkj2xT8At9nir/tkB6dTmJ4ImGGsPbxo7ISOgViKiY4Ipiq3CreO+U9irbal9G3pqxeyK4EGeOcjhVO0UoMpfc10PdWiOwHs5bD8H8o+96x1bDpdjjwlu3P118HP/T2/jum8BfY8xzUHnbeFugdK5zwhyu6tnc2X+nRQ7yPGhvBjpiKUyKCI0iLSiPUoGxSYTSK8Q1szDk33ALrrhdZnO4w4p8w5o2u2wfeD4Mfgsg+tr9nalH2u/eYB31ucq+PStj+GPyQCdX7bH/e7zboeb66fRqbYdk82Paow2ZpUSIaNL9GLBLl512eK3iSxYcXAxBbO5lo23EYAWzQo4f4u6TEyfojfixcWAG9/+TSceLC45iUNYm4cPXrMqVzTUfEhMZwauapHTKM3UJnEGsKBcjx54WF2hzaKRE9IWUm6FUs2vyEGZkzAPg993dajC1OWmvDntI9AAxMVGebU8Ss5TDPtt2O8FSY+Am/HU7mjJ/OoHCd7Rtk8/JqAHqOTuLWXX8mbXya9v20QXq6+Ds313E7VwkKD+ri9FPKJ/M+4bMzPusyV+m8BgngG+TgZCvVRZ8hC/sF7gn/wmCgfa7m7fvEkV3TYz4gBxxXjr/a2lq2bt3K1q1bAcjOzmbr1q0cOXIESZK4++67+cc//sEPP/zAjh07uPrqq+nRo0e7HOjAgQOZN28eN954Ixs2bGD16tXcfvvtXHrppfRom5FefvnlBAcHc/3117Nr1y6++OILXn75Ze699972ftx1110sXLiQF154gb179/L444+zadMmbr+9a0SmMzqnhPoaSZIYlzoOgA0FG2y2kR1/zhZYx+qPAbQ7ilxBnrymKZl7SDpIOxsyhKQlsSMgfoLIyvIl5X/A9r9D9QHfHL/6APw0sIPBUE6KdVCW0i594oRRwhXH3xsb3+DFtS+SW6l8dtdWAqZL5pFf0FIL9YVgbHLe1h0Kf4FF4yH/W4fNjEYjO3bssDuuxMSIoEyz2fK7+hN+PZEu+V1ICRsinLeNSIfTd0Kb5PDxhuwgljND1TIzcyYAS3OWatIfLYwxAVxHHlciIowYDGL8kAMylGIym7ju++sY/Mbg9qAgd+g8VjS1jcGpBoRDZKdCB4s/oMTJpA+DXldB3EjP96czQZEQHAPY7+elQy6l5P4S/nPWf4iJEXEQJhNYlTpQTe7KXMoPltNY2ei8MTAofjgD854n/diN7fNUl6gvgG8SYd9rbuykjeRTYfSLIjNPHyrmqUpImQ0TP4YhNhwgZjP0vlY4irwd4ND/bjjrAHTrmsFri0GJg5jVexaTe07u8pmz+YrBYDEI5uS42F+VVFWJn1eS6OrUqdoFVTuVX0N7pF8IqaeLgzQUQV2e2J4wHob+3f41XTgals51PF5UbNfOae2M5FOh3x3aGv31wVCxDeqPOGw2Pm08f5vyN64Zfg1gmRs4C0h1F6PJyKJDvwGQVDXX+brEbIKVF8Gh9zzbseOA+HgIChJBQw6DyiSdW8+Ys3HFHmazZV7hN3PNqr0i6NOkzDElO/46iVt5FrMZqvaILPXjiOEpw4kNjaW2uZY/jv7h/AsaIDv+BiQMcNLSRQx2btygKMi8guqqFMZuGkv53q7W78V/XczGa14nuL6SwYNBp/eeiTijTdTgiONh3y0qcytZ+fRKyg+qs/wHhQdhCDV0GVNcCVQOoD2yapk/2LQCtgr/JT5e/O1tx5+jOYUvfEDHleNv06ZNjBw5kpFtRX7uvfdeRo4cyWOPCTmeBx54gDvuuIObbrqJsWPHUltby8KFCwm1GpU//fRTBgwYwMyZMzn99NOZPHkyb7/9dvvn0dHRLFq0iOzsbEaPHs1f/vIXHnvsMW66yRI9ecopp/DZZ5/x9ttvM3z4cObPn893333HkCFDvPRLKGTZPFg8XfXXxvVw7PhTusCSjXuuOv7MZhUZf6UbxATZmuBYmLlURA/6kmNrYOeT0JDvm+OHxAFmkCw5xnLkpStRgX3j+gKuOf7+ve7f/GXRX8itUu74kw3Mfun4O/IFfJcqsks9SeIUGPIYJDiPIHX0ItHpLJMkV7O5ZMobyvn14K+szF3p3o6sUOv4K64t5o5f7uCrXV85b+wu036EOeuUG1rlSB53swN8gOz4c/UemdZrGnpJz8Hygxypcm8lZzJZpBIcTaZL6kq4e+Hd/HrwV7eOF8A2RqMRSbJkwah1CuskHavzVrP72G42FW5yuz+dF1hyxp/REC4cIknT3T6GxzGb4fdzYMW5ztvGDIFTPhLGdm+TNAVO2wKp9jO8O0pLg1yuwJ1FVnTPaOa+NJde03spap8RPpCs4r+QVn6Fe8aYplLoltVBRtPrSJKQrTfY8GBKEvS5EQb/1fv96tZLZKQpjFQdkTKC3676jf+b+X82P3e28M3MFH9nZ6vppKCwppDrvr+OR5c6zh6zxjrQrIsKz+yVMGuF+o7Yo7EUNtwEvwyBegXrg8yrxR+znSzPqr2wYDhsfUC7PjoiZQaMeQUis7p+ZjLCLyNg60Pq93v+UZj4ocMmw5KH8dSMp7hg0AWA9zL+NhZupLKpgqDWGFKlsbZU9TrSeAzyvxEBoCc5kiTKDYACtZmqvbDnBfH7ucBXh77ijgV3sPvYbsXfaWqy+MxdNdwerTnKA789wP92/s+1HXTmwOsi6LM+T1HzpCTxt1cdf4fehZ8HCQno4widpOPUTDGfWpqtTZCiIxpbG8muEC+ygQkaZ/yVbYQjX4tgZAdsenMT95Xcx7QbpnX5LHNmJo1JGZglHd42Z8qOP09l/AGU7ill6SNL2fvdXueNrbjkm0u48KsLu8xV/C5I4CRFXpNWVbmnZKUF8j1ho7JXAB8jO/5cDWh3FX8LEDiutLCmT5/epfCiNZIk8eSTT/Lkk0/abRMXF8dnn33m8DjDhg1j5UrHxuyLLrqIiy66yHGHfU1wPJjUZyKNTRWOMmcZf56W+qyoEA+MJFmi2GxiNgtJpKpdcH4xBFnVnZMk8fmxlaIWRaaH5QBt0fsaSJ4OEZnePzYICdQzO0505N9TrdQnWDL+DpSpz2AsqhUphindHF3QjsiGGL+U+owaBP3uhG4evrYhcTDsCU12lZAgCt266/j7fMfn3L7gdub1mceCjAWa9E1ttNS6/HW8tvE1luUs46LBHh6PJUnId6phxxOQ/TGcuUfULTlOkCdIrt4jUSFRjOkxhvUF61mavZRrRlzjcl+sDXmOJk6/HvyVl9e/zIrcFcztM9fl4wVwTHy8GD9cmTyP7TGW/WX72ViwkXl95rnVjy4Zf61irtNqiIKRz7q1b68hSWBuW6nKaUYnCLGxYiFeXm5x3qil9yxlWWUy1u8Pt37K2OEwd70bO9CQxmNQvBQyLrFsa6237RD0Fs2VUHsI4kZ7/FByvR9XMv7yq/N5f+v7ZERn8NSMpxR9R3ZU251v6jWy6Cw/C8o3wPj3ILIvhCuQNRl4n+PPg7pBv9shTaW8picwG0UGkFFZtq67eMvxt/DgQgASqmcTF6PAhBKWDJc0eu138He6dxcBvU7VZkp+hy33CYlkF5Qzfs7/mc1lm5mSMYVBicpkMmWbhlxT3hU2FW7iuTXPMTRpKJcOudS1nVjT6woIS1VsP/CJ4y/5VOh/l+9sHG4wo9cMvtnzDUuzl/LwlIc9eqz9ZfsxYyY2NJakiCRtd37oXTj4Hzg3v6MNTKa1Dn7sh6HnBRjG2C5/Ezs6i53Ds5AkGOChhER7yJn9hYUi0DM42HF7V+g1vRdXL7ma9Mnpbu/LZArIOvoLUVEWU29VlSW43RcEMv78F/m+8KeMP19wXGX8BVDJpE9hynzVXxvTYwwAhyoOUVbf1bqnVOrzjL5n8NLclzh/oGuL0IIC8XdKipAHccioF2H0v21PeAA23gab7hT1I7xNUCTEDLXfNx8gZ/y5JfVZoS7jr76lnppmofulxvHn1xl/iRNhzMsQ1d87x2upFtKibuCujKPMhLQJgHC+mTTKalOb8SdnDsnyxB6h7gisvwFK16n/ri5ISOTVF2jeLU+ixT0iy30uyV7ipKVj5HtCr7f/HjhWd4xHlj4CiPdOAM/hzuR5bI+2oKJC20FFaug8mZalPkO0Msx7i2k/wvSfHHuqzGZYdTHs+pf3+tWZg++IPw648psrGfDaADYf3dx+n6iVhHWHLQU7qIjYgBTh5ZBOT/LHnbD6Uqg5JP7fWApfJ8L2x33Xp5Xnw+JpIqtLITVNNVQ2Vqo+lDsZf+UNYpCKC1NuCZLv1y7Go+xPIPcL7WQ0k6dB2nmQNA1GvaDuu/Z+9/A0GPOqmJd6i22PCnWZzuiD4ewDMPol9fus3CGkMVsdLzJzKnNYlr2MqsYqrzn+fj0kFAUSqxXIfMrogsQ6MEB7mQmnajOpZ8HMZS5nuHcPEwfKq1aWKQfaBI7sKNkBiIxUTUiYIDK7FXZIzqgsLvae4i+RWcL+EqvROXuRWb1ncdGgi7h86OWK2h+pOsLWoq0uHavV1MqMzBlM6zVN+9pOA+4R0uBhPWx/rguGiAyajQmU7Cqhvkx4uc0mMz/c+AO5K3LZ3ZYcm5kJEQoqWmhJbCx06ybu2QIPLZcNoQYyZ2SiD9Y7b9zGto+2sepfq2ht7Jhl32SVU+EvBv2TFZ3OEqjla7nPgDPYf5Hn9N7O+PO3EkYBx5+P0XXRkvE9cWFx9I3rS3p0us1Js9KMv/Fp47l7wt3M7D3TpX7IL/8eduYx7UgSJE0W0kf2Ph/7OsxZIxaj3qYuT0RI+5KipbDtb+11AqwXX2oXB+NTx/PUqU9x30Qn0cedKK4tBiDMEEZksPJFsGyI8cuMP2/SUg2/DIM1V9iVj9TpdPTv39/huCI7dY65pqDTzrDkYYQZwqhsrGRf6T73dtaG2siYjYUbAYtDwSOUrhXRlLU56r878EGYt0nIox1HuCv1CbRL6Kw6ssqtvljLJNhaK7eaWrn060vJq86jX3w/7p90v1vHC9AV63HFHbkM2UG/sWCjQ/UGJXSWVLlw0IUsuXoJj42+CpadDjkaSW15GiUGoMZiKF6uWO7LI+x9Cfa+6LDJoYpD7CvbR3ZFtttZw67w3Oa/sXrgeIpiv3Z9J2YTbLgF8n/UrmPuMOBeOOVzCG/Tu9cZIG6UbXlFb5F1Iwx7CkzKAunuWXgPUf+K4uV1L3fYrmS+Imf8HT1qeeaVIgcuqnH82c342/E4bH9Uu6zcgffBuLfUBwQeeBO+Te46H6nN8Y2seF0uVO916qRTRc7nsP56p3X+Tvv0NGZ8NIM/jv5hkXz2YGJdVWMVGwvEnDOxam57dpXjL+0R5R48XQf8OEFWm3EadBreQyjluBAwq9PpGJgmpBTVSM1rka0hO/6GJg11fScyxmbVC/TENoGlhgao80XJPa95G7Whf0J/vrzoS64beZ2i9n9b+jdG/mckz61+joUHF3LPwnsUH2tU91EsuXoJ317yravdtU9UfyENbu/9pAuCOWv486Jc3hzyJr+/+zsARduK2PbBNvZ8u4edO0VTX1QtkiRL1p8n5T4Bcpbn8Msdv9BU43xM3vLuFlb9cxVBoUEd5iryWCFJChITAngcOQjHl46/1lbxBwLOYH/EV44/R1KfvvAB+Z/XKYB2VGyFHU+6ZLDefPNmcu/OZUTKiC6feauIuuz4S3OkgFObI7TNnZE0FaI11lRXyu9nwMIxvjm2zNEFsOv/RPYSYvElSWJhUF2tbldZcVn8berfOKu/OvkVWeYzuVuyqmg365orfsfhD2DFedDggmaqWoKiIP1iyLjYYbNgJxoZWmX8BemD2rOD1+W7kA1nAzWRMWaz2eL4S/Wg4y/jEjjniMP6VnbRKY8s9Ce0MNqPSx2HQWcgKiSKumbXLRDOjDEPL3mYpdlLiQiK4NtLviUqJMrlYwWwjzyuuJPxNyJlBAadgeK6YrdrP3a+L9Ki0piROYMB0T2ETFi9h60HWnL4A9jqoF5bWAqcdxSGP+21LnVh0ucw/WeHTdKjheUmrzqv3QDpTcmxykYxWYgKckPrp+aAkMw6pmEtN3eIHwu9LgV920sxOAZm/CYk4HxFr8tEhoFBmXWje6SINNtb1rW2jrP5SnS0ZcxRK/fpSsafXcff9AVCltPXdMuC4Dgh3SbTUCTqgC3uWrfJ40x4H87J6XovtDaIDOFja9Tvs9cVMO0n+9krbcjjzZGqI17J+IsOjebIPUe4OvQrwlrSnNeeB9j/Kvw2CZq9rC/lp8iBvIrKTJjNULkLGopVHyczVqQKu5Lx505k/vbi7QAMTdbA8bf3efg2RTiPFRIcbFkre1Xu89ha+KEP5H7uxYN6l10lu/hk+yeAUD466/Oz+Pf6f/PNnm982zFjo+LAi98bf2f5tOWEDhE3efeR3blt721M/dtUdu0SbQYP9lRHHSPX+durrgSfara+v5Ut726hrsT5uvTyny/n2lXXIumkDnMV6yDlE0ih/7hFniP60vFnPfcI1PjzP2S7VkDqM4BPMZk8GKFZsRV2/F3IpqikW7D9KDs546+52XEh1d9zfmdd/jrqW1zzECrK+Nv5FPw6Diq2O9+hyQiFC92WSlRNrysg6wbvHrMz/W6H03dChFgoBwdbIgPz873TBVfq+4GfS31W74WCH9ozKT3OyGeh759Bsj10m0wmduzY4XBc0SKbS2ZimpCUWpu/1v2doW7hnV2ZTXlDOcH6YO1kdewR0dN1qab6Alh9BRx6X9s+eRBr57CrAbxRIVFUPljJtlu2ERHsum6MI8ffB1s/4Lk1z4l/n/uB4louAdRhPa64k/EXFhTWHky0Js8Fg7AVdoMEYofDJXUw6EG39u9V8r8TGXWtDuZKOj0ER3utS12IHQ7dHNfc6xnVE4C8qjyf1BqqbBIruuhgNxx/Uf3hvCKRaedPbP6LRepVH2p3DuCPDEwQQXd7jnU0YCuZr4Drdf7kOuOuSH12cfxF9RXKIr6m+xw4fQfEWFlnQ5Mg4wpRZ8vb2AtuaqmGDTdBzmfq9xkzWARaOZlzZUQLS3FuZa7bAall9WU8sfwJcisdB4z0iOxBQsmFgJOg1PZOXgYjnoXQZNc6doJhnfHndG5ZvAR+GQLZH6k6hslkwlgujBNqAozkOUW4i+VTm1qb2tVPNFmThCRARC/xRwXWcp9eIzxNZBx7ay2sIWazmf1l+5068B5d9ihmzFww8ALOG3geD04Sc8w///xnqpucR1CX1HloMlTwE3zVDfKcZBLueZEzUstZPWk1x9YeY+sHWwGIy4rDGBzenmnnK8ff+PHi719/VRgY4CLnvH8O16+9nrgs5/OC4G7BJA9N7jJX8Tf5vpMdf5D6lO8Jg0H8CeBf+Mrx58iu6VEfkB2On5VjAPX0OBNO2wopM1zehdls7iLJZW2AdRRdeenXlzLx3YkcKDvg0rEVZfwNuBeGPqlMW75oESw/Tcj2eZNBDwqNfl8SkSEW0zqLJkFPYaMjzwX1sNzKXH458AsHy5XX+XPF8Wc2W17kfin1OeJfcGkrhPf07nGbK4UR0AVpJy0l2Kzr/GmBmsm0LLk0PHk4wZ6S8G0qg/wf3JPqDYqCgu+hXEFmsp8gR881N0Ntrev7ccfhJ+PonliRK7Jy/jrpr1w46EK3j+Uy2x+H3c/67vhexN3J8+SekxmQMMDtuqCdJ9Mrclfw5sY328eF44pR/4bzi8Fgw+JYe1jU0ao97PVudcBsEplFDiLLZUP84crD7YFF7kpKq6G6RdyUMaFuThbCkqGtRpRf0FgKB96A7A9U1dXzKCvOheVnKmo6MFE4/vaV7cPoQv9lx5/aOn+yoTU5QrnTRR7XOtT4q94PLTXqDu5J5Dqm5X/AsdXCCTz6RUj3wTvQ1Aq5X0Lhgo7bg2Pg1EXQ71Y39u34XrHO+FNae94ewfpgdh3bxW2/3OawndlsCZZU5PhLmgKD7j+uHPWeJDlZZMg0Niqo/5o4GfrdKepgqiQlXKwz86pcq/HnCntL92I0G4kJjSE1Ukk6qBP63ARz1yvOrJbxRdANET3hnMPQ+09ePKg25FTm0P+1/lwy/xK76iSrj6zm273fopN0PHnqkwA8OvVR+sT1oaSuhPm75zs8RnFtMcnPJ9Pv1X40tWos+xuSIOrFRvZx3G73v7gsvB6j3sjB1w/yx3/+aP9ozx4xtqWliSx7XzBqFIwcKeQS3/WgmU7SSaQMT8HYYuTALwc4vNj23Dp3RS5lB2xHOGohCxxAO2T7oLedOtb4W2ZXgI5Y1513lLSkNf52XwRmoicyoQkiStug3vhqNps5+/OzSXguoUvEnMFg0bS2F11pNpvbo20TwhNUH7+11VIDwKGcSsxgGPqosp2mzIbh/yd00E9Gmqs6SFLKi1ZXMv4eXvowZ3x2hiqJi8uGXsbGGzfy5PQnFX+nsVE4H8BPM/5ArGK9rfWw+xnY9hAUqK9DJBtky8vdf/nJjr+dJTsVRTw6Q81ketcxoUvi0fp+RUtgxTnOIykdERQJZ+fA2Dc065anCQqCqDbFTC300FtNrc4b2cHRPfHfs//L5xd8zv/N/D+X9+8yB/8LBW3Sh8WLYf/r0JZxJKyD3wujaOMxWDZPSFadALirk//C3BfYc9serhjmnlRh5yLqX+z8glt/uZUle78U18WVmpy+olsvYSgHKN3Q8bP872HXP6DKw/pHzjj4NnzbHY7Zr9nZP6E/IAyg1o4/b5T9MZvN1LSK5y821I2Mv+Jl3leFcEZoApyxG+Zt8R/5aGOTxQHlhMyYTIL1wTS2NpJbpV6CN1Oo9nFYpe+7pF5YvpMilBRjE9gMNFtzBfzY17/qV7XWiRICW//qm9p+MpIO1l3btf6nPgS6z4ZoF7LwG0vgq2jYfLfDZrLjL7cqt/094Krj72jtUX4+8DM/H/iZ7IquHubNRzcz5+M5vLDiDerrxZS/ux/FBhwvGAyWNYjTOn/6UBjzMiSMU32clDDh+CtrKFMsNS/bM1w10MlrkqFJQ1WVs9Aanzj+jmN6xfQiIzqDVlMrH23rml16uOIw5395PgBXD7+6XVkkxBDCtSOuBeCzHY4zm1fnrQYg1BBKiEFjHcDk6TBlPsQ4kZeduZS/1iZh0psY8/kY/rTc4qT1ZX0/GUmCG28EvR7Wr4ctWzx7PLPRzJcXfsmKf3SVdTebzfx080+8P/l9TMau79dAxp9/4U8Zf4F7wj+JjhZji3VCiTfQQkJcSwKOvxMZs1nIrbigjy9JEoU1hZQ3lLOhYEOXz+XoSnuOv6qmqnZjryuOv6IiMJnEgxJny4bTeAyO/qZuIa4zwOCHoVum6v64TMV2WDrHYiD2FcZmmB8Lm+5s3yRn/Lni+OsTKyLL1GT8xYTGMKbHGFW1D+SI0NBQ/xk0O1C+GYoWe98gNPghOOVTSD1b9VdjYkCnE8+X04hbJ3SP7M5H537Etlu2OZQHVkpnY74jnpj+BPn35PPgZA9K+iVMFA677rPd20+o+jHQ12hRCLmqsYrJ700m/tl4GlsbXdqHI8efTtJx6ZBL0Xk7mt5sgi33wa62mmtTvoUzd0NI249W+IvIiNnxOJQsF2NEhYdXsV5Cvi/q6y0LHTVoda063xdNRjF4pJkq4fczhfzR8Ubed6JW154XLNv63QmzVkDKTJ91C4C40dD3VoeSdbKk48Hyg8TEtSBJInBHbR1hV2hobaDFLO6BhHAXHX/GJuGk33Cjhj3TiG6ZqjM/PMqpC2DK14qa6nV6+sX3A4RTWC1ZWeLv7Gwxb1HKZ+d/xpG7j3D50MsVtberMJF5FfS/27+K+RgiYOZS6HG6bx2Skg4mfwWjXtJun8FxED8OIvs5bGarxl9jo/Kfo7yhnPsX3Y/JbKJffD8m9ZwEwFub3urS9r0t7/Hb4d9YclAYiZOTLQGwdmmugG97wE4fBCb5MbLDtFBpfIXZ3LGmpQK6Gbrxx41/UP5AOeFByrQ73c3iuWzIZRy5+whvnvGmazuwJvdLWH8D1KtfoMuOP69KfQLU5QpJ6qIlXj6we0iSxF3jhUzyfb/d1+Ed1WJs4czPzqSkroSRKSN5Zd4rHb576ZBLAViWs4yjNfb1KVfmrgRgcroP5aJjhlBoFovr8IHhGEIseoRyfb9BPq6W0LMnnNkmJPDOOyIJwFMYQg2c/8n5nPmWbeWCuS/NZc4Lc9Dpu65ZAhl//oU/OP78zcEToCOS5JvMUH9zCAccfz5Gp/PkJTDD1wkuGzHG9BgDwJairgZLZ7IqcrZfZHCkS9FN1vX9bK61D/0Xls2Bo7+q3jfGZu/JZjUWQeka3xd21weL+h/d57ZvkjP+XJH67BOn3vHnCn4t8wmixuSy07xvEAqKgl6XdzmuTqdj6NChDscVnc5ivNdC7vOq4VcxNHmoJgZ9NRMnSZJIjUptN/54hIieoqZiuBJNJweYzXD4A8j+VJNueQMt9NCjQqI4WH6Q6qZq/ij8w/kXbNBZJmH1kdXc9vNtLDiwwP6XPI4Es1fBqDYHTWiiJbO+tV5IUw15VDhK0i8S2TrHcaa59bgSHm55Pt25N1qMLdQ0uSaf19pqMQjIfZEdy7WhqTDhA987ylwhaSpkXA4Zl1q26fRCLk5hdpXHiB8LY193KKueFpVGWlQao7uPpqqlrP297Y3Mg/IGcTNKJgMx4S4GoZhNMOY16HOLhj0LALbr/CmZr4BQ/QgOFoFBip0FiIyMntE9iQ1TNoGsr7coTHSYc/a/0/dy/bZIPlUEgfk6CzT1dIjplCpStRc+D4IdT6jfn84AM36D/nc4bCZLCx+pOkJIqPAIm82WADJn/Lz/Z55f+zz3LboPgNvGCpnPd7e82yFQqbG1sT2jZ3KEyPBxqEQj01ItygC4oLpzIiM7/pxm/IEIxvh5IKxVLiGp0+kYNmwYI7uPJDYsVnH2nbUxf3vxdq79/lqqm6o5VneM67+/nuJax540SZLoGd2TwUkaFEk7thIOvQd69Z4Fucaf1zP+jM0i87ez7O9xwF0T7mJW71nUt9Rz+deXt8txBumDeHb2swxOHMwvV/xCZEjHuqO9Y3szIW0CJrOJL3d9aXf/q/KEUsKU9Cnad37tn2DfK87btdQQrxfPgrUKS2UlHGiryDNMg9KU7nLZZUJxJi8PNnpYuX/g+QNJGNA1MFeSJPrM68OwK8UP0nmu4m/yfSc7sk3LHzL+AveE/5LQ9qh7swSFI8efZ31Atgk4/k5kJB30vR26z3Pp63KE7uGKrk4yZ4XUj9WJp8qVbD9QUN8v7RwY+gQkq6xfaDaLYuErznWpX6rpPgcuroVe7smaacLol6DPDe3/lX/bY8eUL5Rl+sb3BdQ5/l5Z/wrPr3leVc0FOSPNb2U++90GYzWI7nSVxlJYfRlkf9y+qVm2XDlAltrRwvGnJX41cTI2O6xnpQpJgu2PwZ7jpw6cFhl/kiRxSs9TAFiTt8alfXR2Bv984Gfe2PQGX+62v8j2OJIkDJ0JEzpuP/QufJ8OTaUw7EkI7yG2R7VlLpiM/lOjSyXW44rsFHb13vjXqn8R80wMz615zqXvW2cayveFnPFnDEkUdWaiB7rWOV8SEgeTPoXwNqvy9sdg/xv+JTHoAEmSyLsnj3U3rCOlW4pXJce6BXfjnNAX6Hf0cSIiXAzEMYRBnxsh7SxtO3ciUrUX9jyvOIjutD6nceOoG7soPiiZr+h00Lu3+PehQ6p7qhjZaBQeDiGyn71im+cOeCLR2tBxvmQIh9QzILKvxw6ZGpXKI1Me4dXTXsUQZGyPg1Mq9ymr2ch17M/sdybp0emUNZR1MOL/uO9HKhorSItKI6l2FqCwvl9EhqjRNuBupad0UiA7/o7aT5CyoA+BmGHQrbeqYygZVzoj3ze6kHounX8pH2z9gAd/e5Arv72S97a+x00/3dR+r3TGbDbb/cwlxrwK5xVASLzqr/pM6jOyD5yxB0a6Nq/zJTpJx4fnfkh8WDxbirYw8HXL/PHMfmey7ZZtpHRLsfndK4deyczMmWTFZdn8vLa5li1HRQC95hl/rQ2Q+zmUrnPedvE0vozMRifpOtTY3rhRTDH79rXM7X1JRASMEXkHLgWmq6W1qZWSXZaHxWw2U7a/6+LGekwJZHf5F7KNsKLCd8slf8vsCtCVlLYhXNHcQyP8LTs44PjzMSY1ujWuMPpFl4urZ8YISczsyq71DpRm/Lnq+JPlJ3v0sNMgehAMfUxksqlBkqD3tZB+iXcNsH5Y2D06GiIjxUtSdrQqRc74y6vOo6FF2Sr7xbUvcv9v91NQo/xgckRogr8qJabM6uBM9TqmZhFd2SQmqabKPRh/nYSpxrExTp7ca+H4q2+p542Nb3Dzjze7vfBVOpmev3s+Z352ps16DJpxdCHMj4EjyuTMnDLpfzDVjVqBXkaLjD/A4vjLd8/xJ0+aVh3xYOSsUqoP2HYKR/SCkARotuERK14OP/QSMqDHGSaTiX379rXPV2SnsKv3RmxoLPUt9e3XUi3yAstgEH/AkvGnef0UX2EywoG3YNvDYLQTYeVt1l4DW5RLK8sGSG9EV8aExjDedC99jz7SPj9VjS9rpR1vVG6HLfdDmbKQ/GtHXsvbZ73NnKw57ds6jyuOkOU+lTr+mo3N/Om7P3H/ovvbszecIY9n7dl+Of+DBSPg4DvKDnqyUrgAvoyAI1bBOBHpMPU7oUzhCrlfwPobHa7TgvXB/GPGP7hx9I0EG4La5wiKHX+FwvE3Pm08ICRpbx59MwCvb3y9vd37W98H4OphV3O0UGRXKsr4C2AT1ca3yV/CSOVBc/K48nvO79z28238Z9N/FH1Pvm++rX2QPaV76N6tO0/NeIrnZj9HkC6IH/b9wIfbPrT53UWHFjH2nbHM3z1fcT+dEuZaEUk5sLOuTvzxGpIE0QP8SxJZBT0ie/DeOe8BUFhTSEmdxRmkd5BVfdu421h89WLO7GdbMnJd/jqMZiMZ0Rn0jO6pbacNYXBxvbIA5PQLSRt6H8bHjMzItATNr18v/h6nvpSmx1CVFewmH0z7gI9mfITZJOwXRVuKeK3/a6x8emV7m85zlYDjz7+Q52xNTa7X+XWXgOPP//HmuALCvu7ovvC4D8gG/ueNCOA39I4VEXauZPy56/hzmPHXWOKe027wQzDkEe/I45SsgoJf/MOgVPALLJkFlTvaN8m/r9o6f/Fh8USHRAO274/OmM1mimrFSGsvas4Wubni74wMdf07aQjvAfM2QWrbgqNmPxH1O5COOXayaJnxZ9AZ+Muiv/D25rc5UH7A5f2YTBaZLWcTp2XZy/j5wM9sL97u8vGcYogQGcXRGhU9SDxFddSyL9Hc8Ze3xiXHsLXjr6m1qT1K36e1Mn4/AxaO6ro9ZSacvlPUQ+tMt94g6VXXqvFH3M34k6/d+oL1tBhbVH/f1kQ6t1K8LIYYj8J3GZB3/DjZbaLTw5l74LSt/iMVV7ICytYramoym9rfM97KPJDnoy5HVi4Y6T01iOOd5Bkwe43LiiJqUev4K60v5aNtH/HSupcI0jsrxiaQFSba64onT4e08yzzqwC2iewHPS+AMHuRmi5QslKUdGhSPniocfw1tTaxtWgrAONSLRbvG0bdQLA+mO3F2ymtL6WguoBfD4mSEteMuKZ9raTI8Zf3najX2uongRt+gqqMPzfYfWw3b2x6gx/3/6iofUMDGKVGfq8Wzp/3z3mfhPAEhiUP46lTnwLgwcUPUtfcdQ738vqX+ePoH6w+str9jh9dBPk/CNURFwgNFYG94Au5z0YR5Fbj2TIgnuLs/mfz4bkf8vj0xzHoDM6/oACP1/fTGSA42nm7wQ8jjXi6w6amJtjSVs1nwgQb3/ERslytNwz0I68fyYR7JtDaJORPgyKCGHXjKHrPtr9e9yt1ogCEhlquha/kPgPOYP/HW3MPmeZmSwaqv4wVAcffiU72x7BsnpAEVElmrMj4K60vpba5tsNnzjL+xqWO46W5L3HdyOtUHxcsjj+bi6u1V8P3GWBSbzDsgDeccbuehpXnA34QAddSJYx29ZYiKT3bgs/UyilIkqSqzl9VU1W7FFtyRLLi48iOv3QPlnFzi2XzYOlc5+08SWQf8Qcg9Sx29/kes5Moay0z/oL1wYzuLhwd6/IVyI3YwZZ8X5c2rY2szF3JkmxRPH5sj7EuH88pKTPh1AXaSgY2lUODFzUG3EALqU+AUd1HEawPpqSuhEMV6nXarBdYGws30mRsIikiib5xnpMQc4jZJLLG+9xk+3N7xoKIdDg7G3pdavvz4wh3M/4GJg5sz/qTja9q6LzAajG2sK9sHwAZcX0hNAn0J8DqKyQeuvXydS8snLUPZi132GRd/jr6v9afSe9N8qrkWH51Poeb1tMQVOBaxp+pFeJGQbC/FhT2M0ITIHGiMoNjGw0tDWwt2mrTcO4Ma8efkvgROVsjMSJRcf3hLhl/YSkw9RuXM29OGiKzYMpX0H22ZVttNvxxNxT/7to+hz4OF5RBqONAwaLaIpbnLGf3sd3t7wMljr/txdtpNjYTHxbfrm4DkBSRxLUjruWpU58iITyBT7Z/gslsYlLPSfSK6ktxW5k3RVKfOZ+IrFhJGwfCiYJsfKupUZGRtvk+2HSXquPIc8T9ZfsVtW9ogPJuq2gy1dO9W/cO2cn3TryXrNgsSupKeG3Dax2+t690HwsOLkBC4vZxt6vqo012PS1KOOC6gorP5D4rd8KSUyFboRqL2SwCioqWdNzmQ64efjV/nfxX4sLinDe24mjNUd7a9FaH+qAAUzOmcsvoWzhvwHladlNQuUMEmBtV1mtpY+tWYZxOSvKvAGtvGuhH3ziayX+djNlkpjq/moT+CZz19lmkjrUf3RHI7vI/5Hmbrxx/AWew/yOrDXgr48/arhniJ2JEAcffiU5ttphUqYialIkKiWJY8jCmZUyjqrGqw2eyYcVext/gpMHcPeFuLhx0oerjNjRAVdvhuttabydOhp7ngU5ZFK9Ndj8H36RAc5Xztu4w5G8w/l3/kL5IvxguqoYeFkeVqxl/AA9NfohPz/+UsanOnS9yUfSokCjCgpS9FU0mi0PSnyakHTBE+E82RhumsDZvrrHRru6/LJ2qVY2/8alCKknOxnIF+QWp00FQp0e7vKGcG364geh/RTP1g6ntRn5Zoum4oD4fvo6Hnf/n654oQquMv1BDaLtj2JU6f9ZOHmuZT8lXY6qkE1njA+5x4bt+8B5wEb3ekiHvbsafTtK1Z4KuzlMfId95gaXX6Vl/w3o+Oe8TkntfDPM2Qo/TXOtcAPsomHNFh0Szv2w/O4p3kJAgDHjekPr8387/8VnEBPamPuTawltngAnvw9i3NO/bCYupBRqVX9whbw5h5H9GsqlwU/s263HFEenpQta3rk6ZMVt2/CVFJCnun/yum5H4JBx82+cG6OOa+nzY9zJUulgjMTRB1Dx18s58ae1LnPrhqbz9x9tOA1Ktkeeq41LHdZlLvHraq9w94W4AhqcMZ27WXG4YdQNFRWJdEhpq5Rx2xJjXYM5a9SUpTnBCQy01mRQb9kvXQNGvip9JvV5P//j+gFClaVaQPdfQAMeiRXbnvD7zOtwXQfogHpv2GADPrnmW6qbq9s9kR+CZ/c60W+dNFeP+A5M+E/UNXcRnjr/YETDiGUi/SFn71hpYfjrsfsaybc+zsPhUqPNCgTeNaGhpYPhbw/nzz3/m+h+u76BuMrP3TN48800uGHSB9gfe9wosngIt1c7b7nmR7d8M4YIvL2gP0pVlPseP96/liWygLyuDFjdj/JXQWNnIvzP+zYI7FthtYz1X8be6XQEsAam+dvwFnMH+i+xTOHbMO+OKtf3KX8bXgOPPxyhd9LrM4Ifh4jqX5eq23bKN5dcsJzWqY+SLM6lPd5AnqhER4k8XhvxNFL52h6AoUY+podBpU7dIPAUyr/DsMZSi03cZedxx/F0w6AIuH3o5PSKdy/scrRWrOzXZfsXFIhItONgyCfQ7pnwtIsL9BL1ez9ChQ8W4suJcWDob6o50aSdLsGkV9SJLJWnh+AsL6/qC/GzHZ7y75V2ajc2kdEvhgoEX8Nn5n9ErppfLx3NI7hew/Eyo3qfdPsNSoc/NkDxNu316EOtJtNHNcqjnDjiXcwecqzqCFiwTp/BwWHnEw5I53mDHE7B0jvN2fkSHcQVtskHla+hKnb/OCyydpGNEygiuGHaF4uyeAC5QdwTyvhGZy3boE9cHg85AXUsdxm5iYuEN42N5g+hTkDFWnTHGbBLnJCtAuGFsPen4aSAsUz6WyYb4vaV7ga7jiiMMBksAmBK5T1ccf5WVYNA1MTj4NZGx4i8r9eOBwx/A8jMsmSfx4+DcfJEd7wrGZqjaLRyIDkiPFnIgR6qOqJL6lOv7Wct8ygTpg9pl/ub1mcfCKxd2kPlMS1N4a4SlQMJxFJzmRWQDXKHSJfj0X+CMPYp+eHlc6RnTk4igCIxmI9kV2U6/19AAx6IWAuK6d+aKoVfQP74/5Q3lvLpe2CCqGqv4YNsHANw1Xl1Gol2i+kPaOW7tQnb8yRmqXkNngEEPQMxQ+22aq6CirUxDUBRM+BCGPGb5vKkMavZDqHJ7ga8JCwrjfxf+D4POwGc7PuP/VnopwLP3NTDqJVFb3BmlaxnWuItv9nxDXlUeJhNsaFuy+5PMJwip2tDQtoRQL8wfQ2NCGXrFULLm2Xbcd56rBJw8/occTOJusLKrBKQ+/R9vjyvOxgmP+4BsELCQ+BhXah6pQmfwyOLVWcbfhoINrM1bS2Vjpep9yw9jkq31ula/V5+bYN4GbWX8OmM2+1/EcNFSyP++/b+y1GdBgYhk9RT51WLFnBalRB9HcKTNX5WWJrLAAjjHbDZTXV0txpWB94nMJBt1V9LTxbBQWanNJEk2nmwt2kpTq2uSI/IL0lY6/J/H/Jkrh13JimtWUHhvIfMvns9lQy9ztbvOqd4PRb9BUIx2+5QkGPeW8khYHxMTI547s9lS+8hVHpj0AN9e8i1n9lNfK8l6Mn20RgQQTEmf4l6H3OH3c9pkmFykvgBqDzl0nvgbHcYVtMkGndRzEuBaFqjDyXTlLtj97HFbY8avyf8BVl4AFVvtNgnSB7XLgB8z7wGgtlaZMd4dZMdfcGucOqnPg/8R57TvZc907ESm1+WQerbi5gMTxHx7T6m4LzqPK87o3VZyx1OOv/JyaDWFsCH5EIx/X/H3AtA2Z1oC9W1ZOvoQCE+FoEjX9ld7CH4eDAfedNgsI0Z4g3OrclUFpD5wygO8ecabquT3HJag6IypFepyxd8BuqA66DQ4RrEtQx5XAPrF9wNoVwlxRF2DkcTq0+gXPYxZvWd1+Vyv0/P0zKf566S/csuYWwA447MzqG2uZXDiYGZkzlB4Mg6oPgCt7r8sZfuJN7LtbWIy2j+PNVfCb6dA7WHx//QLIMkqmG/ks3BOznGXKTsjcwavnSayPx9d9iiT35tMn1f6sPDgQkUZpy6ROAkG3K3s2Zj0GTNbpgNgNBvZt0+oa0VEwCCNStlrhSR5X5bvtJdPY8zNY2x+1nmuEsj48z/kLHx37RWuEnAG+z+S5F0ZYWfjhMd9QDYImNN9jMmT3hYQE6/ChVDhotxKG51vTmeSKnctvItT3juF5TnLVR9Lnqh2cfwZm+Gn/rDrX6r32QXrSZIGk2ybVO6AryJhn5vZiVqy+R74wyJRl5QkZBVbWtRHP9S31LPgwAI+3Pqh07Z5VcIY0DO6p+L9y/X9/FbmE2DvSx0cqb7GZDJx+PBhMa6kzBIZvzZqjoWGWhbeSoxozugV04uE8ARaTC1sK3ZtrHH0gtTr9Hx83sdMyfCSxOPQR+HCcgg7fiJOtUaS3K/lpgXWk+nNN28m/558hqcM901nzGYhTdSqvkZVO6NfhrMOChmz44QO4wodHX+uzltHdh/JeQPO48ZRN2I0qUsp7bzAevuPt3ll/SvkVOZA+SbY+iBU73WtYwHs030OTPzYqYLEgIQBAGTX7GlXbfC0AdKS8RenzhiTfgkMewqybvBMx05khj0Jw55Q3HxgonD8yRl/nccVZ/RpK2WsyvEXrtzxJ0tERcVHQpSPasgerwz5m1CXketNt9QKJ0ZLjWv7i0iHwX8T81gH2Mr4s66rYo/BSYO5ZcwtquYS1hl/Tqk9BN/3gu2PKt7/yYTq+vJmMxz9DfK+c9rUelzpnyCyjJXU+Wtq0DMo/1kWnL3NrjrF+QPP55+z/kl8uJgElTUI2YM7x9+pzdpk5bnwyxC3d5PctnTxesYfQPkWmB8N+9tsH2YTFP5qUZ8Z9CD0ux0iMu3vQxckgrj2vep/AdQOuHnMze2Zn6vzVnOo4hCnfXoaX+760sc9A3RB6NrsAa2mVtauFZvHjBEZ9f6G7PjzhoHeGZ3nKgEnj//ha3tFoMbf8YE3HX/OxgmP+4BsEHD8nei0VMLy00S9Chf4fu/39HypJ+d+cW6H7c4iK4/VCStPQrgC+YFO2M34aygAXQig0YNiNonIs+WnW2SetCbhFBH16i+MeBbG/7f9vzqdJXpV8QKsjcrGSk7/7HSu/+F6WoyOxZL/PPbPbLpxEw9OelDx/mXHX3q6un51oaUGtj0Ku/5p2aZV9N2W+4XEkT/TWg9/3NulFo8aI5ozJEkStVKQ2Ffqmjxm5xek2Wzm3l/v5YOtH1Df4gFNYWd4onZj2Sb4bbIi44U/4G4tN2vMZjOFNYVOx4rOdHYIp0altstweR1JgplLYaobzn6DDS3b4ww5srKlRWRzuUK34G58c8k3PHHqE+h16uQuOt8TL69/mbsW3iUcCj3OgDnrRSR0AG2J6geZVwoJOwfImV17S/d6rdZQaV2b409txl9InHBauJqZFEAxw5KHASLLt65ZffBEVpsKl6cy/lJ0yzl3wAvERXq49veJiCFclBOQKV4GP/WDvG9d3F8EDH8Kkk912Ex2/JXWl6ILFfeUp7KLVWX86cOg/12QNN0znTnOUe34kyTYeIsI6lFBvziR8ScHnzrClSyeu8bfxVtnvMX1I69X1S+bmIzQ6wpx37iJz2r8AUT2BclgyQZvKILl8yx1/JImw4h/OZ8HH3gd/rjzuFNveGHOC3x6/qd8ev6nrLx2JQX3FnDlsCu1P1DlTvg6CQ78R1n72mxG6qoIlqCp2ciSJWLzJD+dKssGem9l/KkhIOvof8jr0kCNvwCO8GZAgT+OE34Y4xFAU0KSRKHo2JEufT08KJz86nyiQqI6bnci9VlaXwq45viTI9S6OP66ZcIZO7STTpF0Ql8+qFpkcGht+IkdBjMWabtPd+kxt8umtDTIyRHRrGPHKt9VSrcUQg2hNLY2kledR+/Y3nbbxoTGMLrHaFVdlaU+3c74ayqDo78CJhj0V7F4rNgOc1aLe8BVzGaYvcb/DYa5X8C+lyBhAmRc3L45KwuWLYODGq2p3jj9DeLC4ogMce336BwtlVOZw0vrXiJIF8SlQy7VppNKyP0CWqog80/a13syhIso1kY/XMnYQMsIut6v9CanModtt2xrNwArwTJxMgN+4jBz13FXsRUOvQdDHoXQRE265E2CgiAqCqqrhVM40stDoLUscLOxuT2af3DiYAhNEH8CeA6z2eEzYC3pOCsJsrM9b4Asq2+T+jTG2ZSL7kJDkajtl3XDcScp5jccWwN7noeB90PiRKfNx/QYQ2ZMJtmV2czfPZ8rh6ozhvbq1VGiPM5B0vSbZ7zJk6c+SZhBmRW/pQWm93iTU3p+TUPkJUC0qr4FAEpWgakZUmZAZBYMeshxrS8NiAmNISokiuqmauoNecAAp46/Xw78Qn51PrN6z3K4bumMKsdfRDqM/rfifZ9syAGdBQWihrSiUjdjXodgdUoJd46/k7sn3E1sWKzDdi3GVo4ELya+aRrh4co9f7Lkpybo9EKlRQNk+0lNjZhDezULJagbnH3YsjY2hMPYt9SXVul7K/Q8XzxLxxF6nZ7Lh17u+QOZmoWTVal6yL5XeFbayGd62L3XSHW1uE/GdS1z6hd4W+pTDYHsLv/D144/f3TyBOiKNwMK/HGcCGT8nejo9KKeXbwKj44VmbFCiiG7IruD3Kcjqc8WYwtVTSJi1hXHn12pTxktsz1GvyyyN/zdeaMlZnOHDEdZtkZtxp9O0pEZI+6PwxWHteodAK2tFlkdtx1/3XrB7BUiC0SSRCSuIUI4d9xBkiBhnGfrRLpAaOdZR+oZcOoiSO1YX01N9LwSMmIyXHb6QddoqXX56wAhCRhq8OJMau+LsPWvImJVa6IGCgnRvhoaCzyIbFjVIuOvezcx25Jl3pRgNlvui7nfDeOMz85QFLntMbY+DDmfu7+fim1CBqloifv78hKdx5XENn+lO5Nns9lMXlUeu0p2qfqe9WR6f9l+Wk2tRIVEiRqyxmYRyKNSPjSAQn4ZBsvPcNhkSNIQRqaMZGjSUK/VGpIl16IMccr88ntfgk23QZGfBWcdT7RUQ8H3UKssekgn6dozY97b+h5gY77igJAQS6aQs3lLiCGEtKi0dkk+Z1RWwkvrPuTvKxYTkai8FnUAK9ZcBlvuE/+OHgQjnoY414JOAdj2CCwc41TqT876qzUImRBnjr93Nr/DzT/dzHd7v1Pclepq4UQB6NG1bHYAlSQmiue5tVWFHGWPeWLNpQB5XIkPj3fq9ANYcWg9G/qdxrKhfduCzI5vwsIsAVk+yfoLiRNynSDqM/a9GZKmqttHzBAh9at1AOaJQtwoEcCstG58z/N5WxpKtQk2bxUB9GeeqdDp7gP8SeoTOs5VAtld/oevHX+Be+L4wJ+kPn1BwPHnY/T++sZtIz06HZ2ko6G1geI6y+zckdSnbIDRSTpiQ51PuDtjU+rz6G+w7npLMWit0AUJB47ZLKRptNSS3/8GbH/cv/Tp87+HL7tB/nftm2RDiuIi61bI0bKHyh1bYR787UFeWPMCNU3K6n0cPSoWhKGhFgOzamoOQfFy8W99qEWGaORzMOM3CFZ/b3bAbBIGZk/JxLqAXq9nwIABHceV0CToPltEXVrRuy3QubRUFPj2NZ2jpWTH34TUCd7tyKkLYdrPHWWrtEKSjiuZRy2lPuVaK2qkYFtaRDR4Q1ABe8p3svDgQkVGHI/QXAV7noO8r93fV9q5cPoOyLjE/X15AVvjimqpLhv8b+f/SP93Orf8rM4Rbj2Zlp2GgxIHiRo7B98S77jS1a53LIB9IvsJ9QUHjOw+ks03b+bV019tf3972vh4w4AH6XP0YeL1jvvWztDHhOx5D8dOzAAOSJkJlzRB5lWKv3LNiGt4fNrjfHTuR7bnK06QA5ays9V21jHl5dBiCqWgZfrx9Ir2L0a9KCT8tKK5Skj1Gx1Lvd9/yv28febbZEWJ2qP2lGhkNhRsAGBcqvJUF9k5FRen0Iiz/XFYeaF2CjUnGJLkYpkJU6vI1naAK+PKggMLAYirnUJwsI8GgPU3iKAajdaUsuNEzlQ9LjGboWpP4DnSgqQp/KrvS40JSsuNhIbC7Nm+7pR9rDP+fG1G6zymuCIL7A1+OfAL/1r1L7IrNJ4gHQfIjr/qamE/9Db+mN0VoCvWGX+eLrHnbJzwhQ8o4PjzMV4p7Ljhz/BNsktvzmB9sIiip2NWl6OMP7m+X1xYnOraPc3NlmiNDo6/4mVw+D08dsseeAOWzIDsj7XbZ/aHcOgd/zL0h6VC0hQhcdqGLLly5Ij6WyQrVlhhHGX81TTV8OyaZ7nvt/swo+wAcn2/jAw3fr6Nt8DSWVDbaQIkO3tByC66OqOszxcG5i33u9hB7TGZTJSVlXUdV8xmqN4vDClthIdbIpcPa+RPf3b1s0x5fwrLspep/m7nyJi1+aLy+IQ0Lzv+gmMVyZa5TNVu2P0cNCgNc/YdWmb8DYgfAMDeMuUZf/I9URWxERBSjt2Cu7nfGVcIjobzjsKIZ7TZV8wQ/3o3OMDWuKKF429w0mAAdpbs7KAo4AzryfTOkp0ADEkcIjZGDRQyvaGO69AFcJEp82Hs64qby44/T2f8nd/zVgYU/B9xIQpruhkiIOv64+YZ9Et0QaoVOFKjUvn79L+TEZNhf77igORk8bejd5LZbOa676/jvkX3Ud1UrWi/pvyf6RG5v914FMAF0i+C7nPEv48ugqVzoHS96/sb+xqctc9pveWrh1/NjaNvJD1avJQcZfyV1JVQWFOIhMTIFOXZiLLcebyyBFKo2gElK7RVqDnBUD2HMJvh+3RYdbHDZp3HlSeWP8Fpn57G9uLtdr+zOEc4/tIa5/nuldBcAU2l7pWgsMJ6bX/csudZ+HkQlG3wdU/8j70vwX7lczGAT8//lEelejKO3cTMmdDNR8spJSQliemZtV3QV1iPKdZKNP6UybMufx3n/O8cHlryEFmvZDH3k7ntAdQnA9HRoNOJ10RlpfeP74/3RICuJCSAwSCcw1rYthzh7J7wig+oEwHHn49RY+xymdBkiB4CRtcqnstZXdYRJNYZf51PwZ36fqXiq4SEdKobNOJpODtbyDZ6gl6XC0nUHvO02+epi2DW79rtTwvix4iMppRZ7ZvS0oTUQ12d5fdXinxvHK607znKqxaruuiQ6C61Iu0hO/7khYtLjH0TxrxqPzth68PwyxDY97Jr+9eFQObVEDfG9T5qjNlsJi8vr+u4cuQr+Kk/FPzQYbMcPa9Vnb+tRVtZdWQVq/PUZ9tYG/MbWhrYUrQFgIk9PeiE60zREs875Mo2wdYHoOg3zx5HA2QjlxY1/uSMPzVSn/I9UR0ljIfjU8e73xF3CE0QtYu0wNQCRUuhcqc2+/MgtsYVWYJZHqtdoX98f/SSnsrGSgprChV/r0PG3zGR8Sc7Eek+GyZ+AFH9XO9YAE1oam1CHyVS/Tyd8SePFeHhjtthNov6mvXK77cADijdAEWLXfqq3fmKA6LbSu85MuzUtdTx/tb3eWHtC+glBcGHphayii7nngl/ape+D+AGZjM0HhPGencl9VUgr0sdOf5kxYGMmAwigh07FK2RDUSO6kp2YMrXcF5gjHGEvL5T7PiTJOh1FXTvWqvems7jyrKcZSw8uJAdxTtsti+pK2Fb6SYAehnnKOyMB5jyNczRzlCvxRzN56TMgn53qK7teFKw999w6F3l7bM/Rv/TaKoO7kNCz1lneaxnmmAwWALHFMsBewjrMaW52WL39JfsroqGCi6dfymtplZ6RvXEjJlFhxbxyNJHvGNn9gMkSTh1wDfyxv6aBRqgIzqdJbHI03Kfzu4JXzybAcffycCwx2Hmki5Sf0qxVcdNNq4YjUKOzZqsuCz+Pfff3DvhXtXHspb57BJ15ymnH4gsn3H/EbKImu0zGiL7aLc/D2EwWOr85eSo+267489Bxp9ck0uuw6EEOUJRdX0/s9niuInsA33/bL/toAdhwF8g6zqVB2kjLBkmfgi9LnPt+94kaQr0u11kw1ihdZ0/2TEjSympoalJ/B0aCluKttBqaiU5IpmMaHeLPCrE2AQrzoOV53n2OKlnwMxlyusy+BAtHX8DEkTG377SfYonO/KkqaqbemkuTTG1QvYnwpipFbXZsHQm7H9Nu316EWuJaFeD1kIMIfSLFw46OXNPCTYdf4mDXetEAHWUbRJZ7jWOo0Xe3fwuMc/E8MpekRFfXu45+Z3yhnI2Fq2lPjjX+aK7YjOsvx72vuCZzpxsbLgJ1t+k+msLDy7krP+dxcqilaq+FxMj/nYkT15SJxYS4UHhip078ws+5otdf2PAAFXdCWBN0VKYHw85n0DmFXBRpSUD0BUaS4Vhu/wPh82qm6pZnrOcLTUia0t+P9hiX5lw/PWP76+qK7LjT3HGHwSy/ZzgUn35kc/AkEdUHUeeY8jXvjO/HRJBeFH1I4gP7q5q35qjYbrhCeH4ixsNY16B6MDA3IW5G2CSiprjZjNVVSYkjIwZY5Ha9Wf8rc4fWNalkiSSFHyN2Wzm+h+uJ7cql6zYLHbeupNDdx5i+Z+Ws+TqJaIEwkmCrAjhbUexdRaoP9wTARxjLffpSaztmv5CwPEXwCmjuo9iSvoUekRaKppbG1c6R1emR6dz14S7uHH0jaqPZbO+3/7XoXCB6n25hNksDLwFP7m3n5YaqNwhasD5G7v+Cbs61uFwdYEwNnUsH537EW+e8abdNkeqhBevZ5sMjxJczvjb+Q/4ZaiQ8HRGcDSMet4ie1q5077sZ8Evwth5bK1lm8kIRgcWBn8irLvIfozvmJ3Yp80vrZXjT3bMbCjYoDqSxbrGn+wEmJA2wbuT1rFvCoewJwmJh+Tpx0XBejm6vbZWyK24Q1ZsFgadgbqWOgpqlBUdaWgAMybKQ4XUp88cf6XrYO1VqmV1HBLVD8a9A1k3aLdPL5KSAkFB4r5wJ7pySJKQ6HTV8bfhhg2suW6NRRI4/3tYew3UuaFBGsA+1Xthz/NQaV82DaBXTC8aWxtZVbCUoGAzZrN6RQGlrD6ymju2nsIfWRc6z/iL7A+T/idkPgO4z9C/wyj1TtTFhxez4OACvsr5StX3ZMefo4w/2fGXFKEskM8sBfHjlrPZVHgG/QKJwq4TmiwkrIOUKXs4pbFI1D3L/95hsx3FOzj1w1N5dpcI9FOS8edRx19rPeR+IeT1A9jFWurTk4Hv8rW25/hbeEg4jBOr5vouW6PgFzG/bKnRbJfyur6wsGuA9nGH2SzW3AEshCVDlPJxbEPp1cxcfh9fx/6bHqf+4PwLfoC3DPRqkN8vISG+U4rPrsimrlnYFysaKzhYfpAgXRBfXPgFUSFR9I7tzbRe03zTOR8iO/68nfHX0uJ/WaAB7COPK97K+As4/gJ4l7pcUWjc2mmhgtvH3c6Ka1dw/SiLoUSSLDeys0Lqauji+GttgC0PwK6ntTuII1oqYeOtwoHkDqVr4ZdhkP2RJt3SlJxPu8hD9Ool/s5WWQ84pVsKVw2/yqFBXpb6TI9S5sVrbhYLFXAh4y92uDCoRyh3MgKw50XhMDxmJ/o8eTocWy1qQcosHAUrL4ASdRHrniayg0auDaxW2L1FwiZFRcK54y4jUkagl/QU1xWrku6DjoWRbxr9/+ydd3gUVduH791N7430EDqE3pHeRVAE7GJFwNeugB199bNgRewVFRv2V1DBRu+9Sye0kN57293vj5PJJmT7zu6mzH1duTbZnZlzIJMz55zneX6/O8l8OJOFl7qwKkPjLbLU46c4vy29XgSayy/aCa/IgWPv1ngyHoPz/3N+X8zg52fIXnNUC91T48kdve9g3uB5qK30MCkvh2KfY1Spi/Dz9DPIObqakG5wyRL5q3s7zGoQjG+sXDyuqNWGjH1HPGRqA39Z1gf+6spnBPsEMzhhMIHeNf3L2yf8dV0oMdeiiLsCJp+E2CvMHjYkYQheGi9SClPwjRPVgc6qPMgqFZW43lWRlhfdngGQeD0Ed3VOZ1oaCdPEl43c2U9UCW7K2MSZ/DNWn2eN1Ketgb/002kUF4OXl2EurGAHId2EvUH8FCg8AeeXQYUDcgEB7WHkCovJMR3CRAZbaulZtOpS84E/qeIvwomBv+Jk2HyDWGspmCQmRthMlJfbML8sPg0broLT5v9v685XpIq/4zkNA7E6vY6/Tv4FQKvCy9y3aZv8Oey6H5AvkhAWBv7+QpnpgnW5do2T7B3wW0c4+527e9J4KD4jxhkrI+bl5fDhh5Dnv5WUsK/JxLjsbWNDqvhrDIE/aUypu1fhar479B3DPx9Ou7fb8eNhkTQV5hvG9lnb+fPmP+kX26/BOQXlBeSWySDb0wRwV8Vf3TmHUvHX+HHVuOLOscIUSuDPzWg0VvhPOEp5Fhz6P8hcK+tlpczqiwN/hzIPseX8FrJKbJdFaxD48/CFSQeg7yL7O2oLXqEw4n8waqVj1/FPhO7/hYgh8vRLTkb9ARPrS+dImx3O2JizteIvNVXMZf39ITTUykb0NVpz8VfCuI22ZxzHTYY2N4FPdP33pQxDjS/0fxe6Pm74rM1NonrAhow7Z6PRaGjfvr3xcSV3D/zepV7QNzDQ8LeWbFqt1Wp8PX1pE9IGgJO5thkHXpwZ08q/Fe3DZPJTs4ReD+UuTBG78JsINJ+/qNrhwH9h9/2Q9jdsug623CKvvKSNqFSGqj85TJA/mvwRr1/6er3qcXOUlYFWXUpr7WhGtRmFh7uks7xCod1tzvlbbwKVw6bGFSljX5bAn50Vfw3o9iRcV9xA1lhBJrxChM+lxsvsYb6evgxJEPOf6oTVABw+7JwuSYEer+pI8xV/5dmKt18joVN4J8a3G48ePZ/s/cTq86SKv5IS09KxNgX+yrOI3hbH7L4P0r69kL5XkIELvwrZ9CIHqt48fCFuEvibTxqMCogi0j8SPXqKfP41m4y6+MrFrLttHVd2vtKmrtgU+POLg6HfQ8LVNrXR0sjOTict9Xl2rktiQNc4BiQl8cLzz5NubjdO7S3urULTXtEXz1ekIO/xnOPo9PV1yVWoWHvbWmYlLCSseIj7NugGvCcsADwDZLukSmVInnVkjuZ2AtoIT2xdhbt70ng4/Ar82h5KU6w6fOlSUJcmc0f8MTp4glbfNKonG0vgr+6YYnb94URWnljJjT/fyKZzm1Ch4kjWkdrPfD19GdN2TINzXt/yOtELo1m01UV7qG5G2s9ydeCvrsynWomsNHqkir9UJy8HLVX8uSQGdBHK7elmdPaa49hCcDeYuF8YJDtApbay3qRZ2mC5OLtywcYFDP1sKF8f+NrmNrJq9rjrSX0GtndtVUT0OPCuYySt18PRt2DPPOv1SII6Q8//ExVojQ3/hAaBMWlxkJJiuw/P9pTtfLDzA/al7zP6eW3Fn5Uefyk189j4eCtlFM4vg1UjoTJP/GyP9kJQRxjytagWlNBpYd1lIhgDENZXZDRLdH0U+r0hry+kg+h0OtLT042PK34JIsBw0YRfbrlPKVhnzvfRGO6aTAPC9+l/0XDsbde0FzkcOs+BsAH13+/9CgxaDLETYPCXMGIZ+LRyTZ9MIKfPn62UlUFIaT9mqNewYvoK13cAoKrQsaoFc2TvgB+D4OTHzrm+TJgaVyQpZps8ei5iQOwAHh3yKA8Pftjqc6Sx4pvkN7h/5f3sSdtj+FDtCR7+oHb9hLrFUJYuKnosMKaN2IhI91sDwJEj5o62H0Pgr5X5jdtTi2FZHGRtcU5HWiJpf8P/YuCcbZKdAHf3F9KMn+79lPJq65IfAgJElRCY9vmrDfz5WTE305ZxtPIO/s0arsh8ysHJxWLOHDsJBn8FATL4nFthmdAzqicAhX4HKC83vVSL9I9kZJuRxAfF29QFaf5jVeDPKxQSr4PQnja10ZJYtHAhifHxBO5dwMLioyzOSuWho0fZuGABifHxLHrjDeMn+sbAdaXQ63mT1754vtI2pC2eak9Kq0o5kFFfolqlUtEtshsTQ+ai1nu5L/DnEwlR8kvzNQufP59ImHIG2t/h7p40HuImQ9Ij4Gd5HDt3DpYvh7ah+3m29SYG+oC2icimNhaPv7pjSl3FEVeycKtQQLqh+w2cn3OeV8a/YvGchKAEyqvL+WL/F03md+4I0t6xq6U+G6Oko4Jp6koIO1Nm3FLFn0tiQBehBP7cjK0+WHbh4SsWIJ4WJADNkPReEj4v+NTbzJdu5IuzK9OKxRM6KiDK5nakLI3ISMTmUspy0DpoLmUP2kqht3/6a5FptuchSF3ROD37bKWqCPL2iw3tGlq1EoFcrdYQeLOW93a+xz0r7+HPk38a/fz7a75n1+xdTGg/warr1Q38WUXWJig6IWQSHUWvN/gDVuaKa5ZnuU/I3Ub0ej3p6enGxxWfVmLx1PE/9d5uX1NUJ0fFH0CH0A7EBsZSrbMtgiw9IPcU/sH4r8bz4a4P5emQNag0Ijv74kCcs/AKFUFjKaFB+r/yDDB4T4X2gpjx4vvSFFhzKWRvd03/6iBnxZ9eryerJMvq6q5GMZk+8y38rxWkr5L/2gHtIOIS4Y3UiDE1rtT16LGXhOAEXhn/Ctd3v97qc6Sx4tezX/HuznfrS3iVnIWcnaBVssOdxl8DYYtl2VspA/lQ8Vr06DhxwnGvUGNIgR7vKgsVf+H9oe2tENY05HWbBN7h4N9GVOLYyKQOk4j2jSa7NJufDv9k1TkqlWW5T5sq/vxb8/H+xWw5fw2dG494Q9Pl/E/CAzSoC7S9GXwiHLveuskiKcvCWrlnZE3gz1cEdsqtiyNbRXm5qDAFw3xIwX4WLVzIS/Pns0GrZVV1OTcBY4GbgL/Ky9mg1fLSk08aD/6pVBarzS+er3hqPJnaZSoxATGkFBpf4LprMx8Qa8zSFKfsQDaLwB80mTW4y4ibBH1eter/5bPPQKcD/8RhvBswjVWl2Lw+dxfSBn1+vrxjuq3UHVPckaS8P30/a06vQaPS8Mq4V4gLirPqvCldphDiE8L5wvOsOb3Gyb10P5LUZ1aWuOddRUXNclMJ/DUNoqLE0Fla6tykdkt7WC6JAV2EEvhrKVQVQZFt0nt1UavU6NFzKtdQFmRK6lOaWNuaUanVGjaXIyOB01/Bhqki4OZq9FVw6Hk4+oaoIJiWDhN2WS/DseUWIdXXGDmzFP7oXc/zsa4kiK0LhHahwiiu7r1Rlwi/CPrF9qOVv3WVS5IXQZx18xro+zpMPg6BMmQW734QVnYXAV6fVnDpVujXvCUSpACrXBl17056lwtzLzC732ybzpMm0ydLd7MqeRVbU+zzJLWL0N4w/EdoNdh1bUpUFsAffeDYO6aPyVgHZRdEdYWLJwpyVvxtS9lG5OuRXL70cquOLy6rolKT597JdEA7iJvinGCBTwSMXS08x5ogdWWkXHVb6vViMl2pyeHfnH0AjGozynDAsbdFYMqV0r0tjc4PQLsZFg8bEDcAf09/csuzIfIQ1dVw0v5pqEkkjz+LFX/R42DwFxY3jhVsIKwfTNgqZNZtxEPtwdWJQg7x/Z3vWzjagKXA36vjXyVlTgpzB8+1eK3KSoO3tRL4k4GBH8MUGaMMUSNFUpauyuxhUsVfkZ8I/Bnz+duesp1H/3mUX4/9alMXpLmPjw/mEwskdj8Ev8S6Vaa9sZKens7jjz3GbxUVDDJxzCDgt4oKHn/0UeOyn4UnIPlLqDaj6XoR7056lzMPneGKTgZv2oLyAqb/PJ0v9n1BSanYJXZL4C95CSxLMO0x7wDNJvCn18HeR2Hf45aPVahlzx7YvVtIWF9/WyuSvduQqW06Up/+/sKSBBqPT6U7kgTe2v4WAFd3vdpq9SwAHw8fbuwukvSW7F/ijK41KsLDhSKEVutalSK3Jo4o2IyXF7RtK753lhINuFnJzARK4K+lsP5K+NP+apaOYR2B+r5dxgJ/er3e7sBfTo7I0PDwqPF2a3Mj9H5VSMa4Gg9/GLUCxm8UUTHfKNu098vToczN2gSmiLgEuj8jdPPrIPn8nTlj2+WkwF9yvjwlY1YH/oqTDd5+tnr6maLtLRDS0+C5pfEGTSMasR2lqgj2PgYnDNV0kpSGXFrXKjszM6WJ0+kSUXHZNaKrPB1qrBQcheXtYO8joKtsIMFaj7Y3iyB0j6ddnvkqZ+CvU7jQUjtXcI6SSsvV04fzd/F3nzDeLR3meOP2EjNe+L56hbivD42U6GjxvK6oMMh020NeWR5rT69l54WdFo+tqhLzhJzA9QB0bdWV6IA63qxxVwjJXOX35TySHoZO91o8zEvjxdzBc3l13Kv0aC8Sf5zh82dVxZ8bMisVLDMtcRoD4wYys89Mq7NfJZ8/U1KfXhov4oLiLCeblaWhXdGf4QlfExIilC8UHMS/tUiaO/Iq/BLvUMIpIMaawUssButrA3++B9CjNxr4W392Pa9teY3vDn1nUxekhNSwMCunX75xogrWy1qT8pbD4k8+YbSnp8mgn8QgYJSXF58uXtzww7NLYdttUHjM6nYj/SPxuuge+vrA13x76Fte2vQSFeViO8wtG7dh/aDDXRAivzWIt3c6J44/z2/fJdEx1kofxcaISi0SIC+sUJ7lyV/Aiu6Qu9vsYTqdqPYDuPxyUT3nodKgoulU/IH9e1POwtWb+RXVFfx96m8AHhr0kM3n39brNgCWH11OaZX1yRJNEbXaMI9zpc9fYwzwKJina80Wo7O858Gy1Kc7UAJ/bsbeTXKbaTMdOt1nCJTYiLRhW1dSy5jHX155Xq1XR2xgrE1tSJrMrVrVLK784qHrIyL44g7C+okAoET6KlGZY2GyBcCYf0TQsDES2gt6Pit8COsgTa7srfgz5ul2MvckD/7xIJ/s/sSqa+n1Vkp9VhXCPyNg3RXyLgLCB8Ck/UK+qgmiUqkICwszPa5o/ITP0fn/1b4lSWkUFRnkjNyB9IBMLhJP4W6R3cwcLSMnPoR/hkPhccvHyol/Inj4QUgPmLhXVNGYQ67gto1IgT85pD7D/cKJ8BPSXydyLXuEHSveAUCgh5s20FyxwZC1BTbfCPkHnd+WnZgaVzQaQ4LGuXP2X//j3R8z5ssxvLHNhK9PHaRxIjtIyNaMbjO6/gFRo4X/qgPS5gry8dzo53hk6CMM6ioeNM7Irrx/4P0MrHiSgPIk04G/XffDH/1E8ouCvCR/AQdNe26ZQqVS0TG2I1vv2MrMvjOtXg9ZCvxZTcl5VGXn8fEopnNnRU1OFrSVYi6lqxJBQDskYO0hqVUS7016jzGZy8FE4O9YtggUdQ63rbRTmvtY5e8HYt166RZQe9jUTktg+dKl3GKlZt+tZWUsW7q04QcJ18Cwnxokr0qYWwdpdVp+OfIL3xz4hgf/fFC00+vW2rEk0B3ThugxMPAD8AqW9bKLFi6ke1I8saeEj+KHaVb6KDZWRv0Okw4oA7WuAqqLwdt8pso//4j9nIAAuP56IHc3L2e/ziOhTcfjD+zfm5KTumOKqy0ovD28OX7/cX689kcGJ9iuSjQwbiBtQtpQUlXCiuNuUFBzMe7w+WsUtiQKNpGUJF6dWfFn6b5wWQyoDsqs1M2o1S6KvXawTXbvYmoDf7mGzXFjHn9StV+EXwQ+HraNgFLFQGQkkHcAAjsKf8LGgsZP+AcVnxZBwWaGJAkiyR5ZS/tQYRJ3ruAcVdoqPDWetZ8dzDjI2zveZlDcIKukH/PyxECpUhkCUsZRQ+trIbibsgiog1qtpnVrMzIQag1ctltsyNTg6yukswoKhNGt5PlnL9W6ai796lJO5Z1i/137CfEJseq88nLQUc2p/KOAqORxCRU5UHAYfKMtHysnHr5wuXVed4AIQm25WWSRD3jXef26CCl7Ti4p2C4RXdh0bhOHsw7TO7q32WNPlYsKsE4BLvJevJi9D0PuHhjxi/MqyCpy4Ox3EDdZBIEbIebGlYQEsSA/fx7626mGKgX5D2dZTr2T9gxzg9YCBh85BReSvgoOPgu9XoLI4VadImVXHjkihjI5H9t39LmDnR9DSqWZzEopgUwJCMvP2W8hcwN0f8qmX6yxcUWv11tcDEtSn3l5xj+/d8W9+Hj48Piwx81X/UUM5K3z6Ww5peMmN6h8N0uyt8LqUdB3kQh+OUrhCTj8MrS+DmJN+4T7ePhwz4B7OPgZpGBc6vNYTk3gL8K2wJ+kdmB14E/BJAWFhVg7044CCoxF90O6iS8TmJuvjPlyDBvObqj9eUbvGTw+7HHm1FiMNpeq37o+ioOoH+S5qbyc7cDkJ58EYM5cy5LIjQIfKzxbWwId7hRfZtDpQIqZ33BDTUC7OBxd3JU8NfAqNInXOr+fMiHtTbmz4q/umOKOKh4/Tz+u6XqNXeeqVCqu73Y9r2x+he/+/Y5ruzWd3709SD5/zbnib3/6fj7a/RHFlcVU6aoI9w3ngUEP1O7V55XlsfLESi4UXSC/PB+A2X1n0za0rWs62ASQ1qTJyeL3J/fvrqwMqmsKqwNMCAa6LAZUt02Xt6hQD50r3UcdQMqQtFTxJwX+4gKtNWgzIA3SUZFaWDcR/hliX2edRcQlwuuvtYWHb/FpOPERFJ9xSbfsYt3lsPOeem9Jk6vsbNsqv6IDovHx8EGn13GuoH7Zh/RzQnCCVdeSZD6josDT08gBRadApxWyq/0WQYdZ1ne0BaDT6Th37pz5cSWgjZBNqYMUZJUjuOOh9uBw1mHOFZwz6ft4MdXV4qvUO5lKXSW+Hr60CWnjeGesoft8uCrTbRV1VqNSQeFhKHRiepIREhNF07m58sh9ShKuR7Is/zvSq8QxHQLll0CyCm0ZlGeAp7yZ2PWIGQ/XFoqq/EaKuXFFDg8ZKch/LPuYRQmi8nIo90inyOcwKlSMTBxZ/4CDzwtZc6Wyy3lIVT0V2VYdnleWx9qCxZQG7aeoyFDVLydSECjY1J9q34Uw0Qq1BgXb6f8eTLZcwX0xdceVal017+98nwGfDKCsykjUpg7mKv50eh0f7v6QN7a9YZWc2fHjKnR6jeLvJxdBXaDrY0I9Qw60ZZD8GeRst+pwaTPWbODPmRV/eh3snltPVUPBQHBQENaKTGYAwSYHdMRzyAjm5iuXdzT4S9/Q/QY+mfwJapW6nuKQS8nZBX/0hZTfZLukLD6KjZXiZNhxN1Tmu7snjZojR8R6LSAAJkluOQFt8Bi5nMAOt+HnaY1ZaeOgMUh91h1TXBXkqdRWsnDLQipNjHO2cFuv21h46ULenPCm4x1r5EgVf64M/EnrD2lu6kyWHV3GkM+G8MGuD/jqwFd8d+g73tv5HlklonqmtKqU1m+25uZfbuaxVY/x0qaXeGnTSwz9bGg9u66WTkSEeN7rdHDMetVwq5HmFIGBpscKd8SAlMCfm7HW08Jh8g/Cmgl2L0akLIIz+WeoqK4AjFf8dW3VlXcmvsODgx60uQ3pjySqVZXwdeh4j/kTXI1KbdFnAoCsTbDzLsjb4/w+2UvpBSivXwcfECAGQrBtE1elUpmU+zxfeB6A1kHWmRGb9fcrOgkre8Ch/7O+cy0MvV5Pbm6u5XElexskf1n7o5yBPzAv/2oMaZOmyFf4+yW1SkKtcuHjSa1xXVuOMGEHjF3t0iZ9fEBKnj4pw5wxqZXQVziSbT7wp9frydKJRJMurTo53rA9DHhfVGU6s6pY49Poq5DMjSsJNTkd58/bf/3E4ER8PXyp0FZwOs98yXlZGZR5nyOgOpFe0b0I97toN7a6qObZplSCO424SXB1JiRMs+rwuX/P5a6Vs8lv9zEgr7RKUUUR605uJatK3DfSHEbBhQS2B784m8fJuuNKRXUFL216id1pu1m4daHZ88wF/nJKc9DVWBpIstKmKNv7BhH6jahU0LGjTV1XMIVvFPR+Wcjx/7sAqh3Ujw/uCtfkQvenLR6aWpTKcf/PSQn7ukHgL6c0h+xSkaggrWetpa7Hn0UqcuDYIkj726Y2WgpTpk/nKyt3zL/09WXqdBMJUX/2h3+GGv3I3HzlP/3+w9CEodzR+w6+nPolGrWGsjIoLhafR7q6qKw8U3zJOMeUxUexsZK5CU5+2HID60UnYd+TkG9eLWZ7TZ7EgAEmkqibEFJyYV4eFBa6pw91xxTp2eLMij+9Xs99K+/j4X8e5pof7Kv0q0tSqyTmDp5rdRJ+U0aq+HOl1Gd2TQ6kM1UB9Ho9C7cs5Krvr6K0qpRx7cbx+vjXWTRhEY8PfZwhCaJYxs/TjzFtx9A9sju39bqNBwc9SFJEEmnFaYz5YozFNXZLwplyn9YkE7ksBlQHJfDXksjaBCX27c5F+kcytu1Ybul5CyVVYiFnrOKvTUgb7ht4HzP6zLC9ezVSn+GRPtBljsPypE5BWy4qCo6ZkdqLmQCj/oBWw1zXL1uZtA+G/9TgbXszq96Z+A4bZ2zkkvhL6r1va8VfPX8/vQ6OvAGlqeJN/0Rhfh59qW2dU2jIvsdFcLpa/PFG12jvyBX4ax8m9EJP5VlX8SdVknn45xPsHUy3Vi7y9zv+Hhx5XfjRNAXU7lnBdeggXk/YXtTRAKm6y5KsY3pxOpWqYtCr6RHXzvGG7cUVAejiM3BmKVhRodLYkILC587Zb4moUWvoEtEFsHxflJVBaMlAbs09w7rb1jU8oM+rMPWsqApXaBTc2P1GAE54/YBOVSWrmfqBjAOM/mYI2zqNw98f4x5/536Cg/8HlSa0IRUcQ6cVY1i5/Tst/l7+vDruVQBe2vQSmSWmr2VO6lM6L8w3rJ7sfAPKM/E9Mo9x7T4nOtrEfaNgPynLYf98sWZyBLWHkDe3IjCyN20vKzzu4FT0Kw0Cf1K1X3xQPP5e/kbONo1NFX/e4TDlnFWBypbIrNmzWVtVhaX6ze3AuspKZs4yoeoSPgjCbNcWD/YJZtMdm/h0yqe144O0QWfy+eFM4ibBtBSIvdzysVZir49ieno6Lzz/PAOSkugUF8eApCReeP75xlUR2GY6jNsI7e9wd0/cQ+YGOPwSFJlejOn1sG2b+H7gwDofVBVxbvUUPv1lPF/s+8K5/ZQRHx/DHoU7q/4kXFHx9/Huj/lkzyeoUHFX/7uc11AzxB0Vf9IcwZmJh9tStvH46sfRo+fu/nfzx01/MG/IPB665CFeGvdSPYn8pVct5eDdB1kydQlvXvYma29bS5eILpRUlZBXrqyDJCS5z3//lf/aUkyjscmHK4G/lkJwd7iuGLrYXokHoqpr1a2rWDJ1CWG+Iu3RWMWfI6Smgoe6gtjIYnku6AzUXnDqEzjzteljfCIh9rImqUcvBf5srewZ03YMw1oPI9C7fuVKbcVfsHUVf/UCf1mbYO88OPiMeFPtCRO2QmQjDqg2FXq+AGPX1voeSRV/cq3vJN9Ha6U+pYX3JT4zyHssj4+u+EiejljixAci+KdqIna32go4/Q1cWOnSZqVqCDkq/npG9eSBgQ8wb/A8s8cdzxELW7+KNsRGeTvesK3sfQxOfOiatk5+CFtugkIn6E04mZgY0GjEYlha/NiDtQFhqY3QULGJp+AmLqyAcz9adeiYtmOI9I+kSJdNduAqWQN/UqDHqzrS9ALr7Pdw6Dkxf1OQn7y98GtbOOVY5cgN3W+gb0xfSqtK+fbgtyaPkwJ/xir+0opF9lKUf5T5xrxC2ei3nWVH52LOFlnBDvY+JnxAL9sjj0x24QnI3GjxsJ5RPQEo9jlKYUlFvc8kmwopwcQWbPL4U6nBP0FUwCo0IDo6mpdfeYXJ3t4mg3/bgcne3rz86qtESzv+FzPgPRj4gSx9ktYfLq/2q4uMFX/2+CguWriQxPh4Ni1YwENHj/JBaioPHT3KxgULSIyPZ9Ebb8jWP4dQe7TsPYC2t8LE/RBl2ts6JUUk8Xp4QL9+dT7QVdE641e06atYe2at8/sqI41B7lPC2RV/1bpq/rvuvwC8OOZFJnWcZOEM69Dr9Xyx7wuuWHoFuWUy+HY0UqSKv+xs0GrNHysXNiUH2cnghMGkz0tnxfQVvDfpPTzUpvetLk5uigqIYvWtq1l32zr6xvR1XiebGFLg79gxIfkpJ1Lgz63zCiMogT83Y8nEXsaGZJcrkzLj6gb+Np7dyJbzWyiqsM1fp7xcTL6HJvxMl8NRkPqXjD2VEZUaRv8F400sQrO2QoGMu1rOIv9fsaldUX+ntkcP8bp3r/3VG3WprfgLss3jLy4OiBwBQ76Ffm853pEWgkqlIjo62vK4EjkMIgbVVjPJLfVZG/izsuKv1t8zSvwbfD1d5Jo9YSeMWuFcKUc5Ualh+0w4/rZLm61b8efouBAdEM1bE99iZt+ZZo/z1IaQkDWL6PyrrZPYkhNtBZx4X1QtuILEG2DIN+AX65r2bMTcuOLhYcjGdcS7LSlCaG4czjb//EzLrESP1vRkOn0NJC+R5wGmYJr9T4jKcSvwUHtwfbfrAbgQvpS0NCiSyYJRCvx5V0WazrYd8jVcthc8bKv0UbCSwA7Q+SFRhWMDF48rKpWKGb2FWshXB74yeV5oqHgtKGj4Z342X+jUJ4Ykmm9c7cmBCwM5V9C9VkpMQSbKayZ0Id3FRr2j7LwL1k+2eFh8UDy+qhD06mpOFR6t99ltvW7j/JzzvDfpPZua1uttDPyVZ0HJ2SZZve8q5sybxxMLFjBCo2GCjw/fAKuAr4HxPj6M0Gh4YsEC5syda9f1rV4H1eC2DTq9Hg69IOQrZcRWH8WK6mpemj+fDVotf5aXcxMwFrgJ+Ku8nA1aLS89+WTjCf6BsK/ZcjNUGsn+aM6oPSC0J3iZTqiQZD579booOOUVwgfx/2VuFlb53zYm3B34qzumODvw9/epv8ksyaSVXyseHvKwbNdVqVS8se0NVpxYwf+ONF+p3LAwsS7Vag3PbmcjSX0622og3C+cSR0n2RU7iA2MpUdUDyf0qumSmChiGWVl8o8t1iQUuSwGVAcl8Odm1GoX/gqyNovMZweo1FaSUSwWdVLgr6SOhcM9K+9h6GdD2ZayzabrptaoOeo9g1GF9YWwRpyREJxkXHJPrxcL1L8ugapGXLUIkPYX7LwbLlocd+8OXl7iIWarZ9Pmc5u56X838dnezwDQ6XW1BtLWVPxVVUkBIL3B46/NDeChaDBZi1qtJjo62rpxRaeFvH2AIfCXkwOVjvtI2yz16baMWw9f4SHTVFB7wojlwnvOhbRtK6q6Cgocq+qyhWhVT3qd/YTBRa/i4eqCTI03XJUOA11UeRraW0gYeYW6pj0bsTSuxMeLV0cCf5M7T+bjKz62WAn6R8pS/uodzmqViaDT8Xdh2x1NJ5jfVOn7Jgz+0uJhEtN7CK+mjNBf0KpLOWXdo8EiWaVi19ZsxZ/GW2yWKTgHrxDotwiix9p0mrFx5YbuN+Ch9mB32m6OZBk33ggKEq/V1Q0VR84W1AT+gi1E80rOcv6skPhWKv5kZvASmLgPqkvlScDoeI+QcNabT8lWqVS09uoFwKmiAw0+iw+Kt9nfr7BQ3GdgCDib5cSHsLwNFJj34GrpzJk7l7MpKYyYP583k5K40TeWhwOS6HHXfM6mpFgO+pWchZ33GlW/sGkdhBsDfyVn4MDTVlfOW4stPoqf+PiQlpXFbxUV9TwB04EXgAHALUBkRQWPPPwwBw4cMHodl5OzE858Axd+dXdPXIdeJ/7dFvaXJJnPQRfn4ajUVHqFUaIHrd5FpVAyISXnnD3rnvbrjinOlvr8cr+YV9/Y/UbzcuV2cF3X6wCadeBPrTbIK7pC7rO83LAP7qyKvyqtfHY0JZUl/Hz4Zy4UXpDtmk0VtRo6dxbfy6lEA9Z5/Lk0BiS16fIWFeqhdVUdMsCh52HbDLsXYsuPLsf3RV+u+0k8OKQBLjvbcMmUQrHzFxdkm8yJFGTK8b5cVNP5NDJR3IspzxZyNmn/GN5TqWDgx9D/ncbvL5QwTVQuBtf3UvPyEsE/gN27bbvk9gvbWXpwKYv3CLkntUrNobsP8f6k94n0t7yiSksT99GAhDWErotpuebdDqDVajl16pR148qeufBHHyhOJihIZK/p9fJMlNqHtic6IJo2IW3QWdisAfGALPY+wdNZHbnx5xsd74A1nPtZSEg1NWInQIBrPe+8vAwLLzl8/ooqitiWso196ftMHuMKzXyzePiDv4t3hEtTLW5uugNL40pCTUG3I4G/nlE9md1vNr2je5s9bn/x31R7FBDob2JB3P0pGPm7/R1RsI7oMdBqqNWHD4obRNuQtlSrS8gI/k0W2WCoU/FX3cr4WFF0CnL3KBU4jRBj40qEXwQTO0wE4OsDxiX1vbwMiYf5+fU/k1Qm2oS0Mdu2ftVo/tNWGCApgT8nsH0m/BQCyBD4a301dLjTKr/dNr4iwH+mzBCg0DsQfJTmIcHBWJeA1GowdJ4jPMkVzBIdHc38p55i5+HDzH7oAgNGHeaKyU+Zlvesi65KqDJkrm/wkU3rIKzboHMKfvFw2S7odK+slzXlo1g3mNcJ6A5sLC9nlJdXvaDfIiAR2AQ8BHwAPAEM0+vp37t346j8a3sLXH5YvLYUSs7BXwPhwFMmD8nLg+NC1bhh4A9oq6kk0cMgfdxUaNtWvJ496x4xj7pjSmGheC/ACdt8BeUFLD8mlGZu6SX/vT0taRoAq0+vprCiUPbrNxZc6fMnzRF8fJznETvt+2l2FdUY45ofr+GaH6/h20Om5fRbEpLc5xHjeYZ2Y01CkUtjQDUogb+WRLcnYdhP2LsQiw+KR6fX1U4YpElyRYXIiiypLCG/PL/2WFs4fx7CfFNrKwcaPbpyOPqGyDiT0OuFfGK729zXL2sJaAsxl4pM7YvoW1NsuWePbZe8sfuNqFVqtqZs5WSu2NXz9vDm7gF3W1XOLG0aR0ZqUfklgHcjD/42Uoqs1VCLnwJdnwC1FyqVvD5/UQFRpM1LY/3t61FbsVmTmQnFvv+SVnGSY9ku8DmrLoEtN8Lu+53fljOoLIDyTJc2Kcl9yrFh/8GuDxj86WBe2fyKyWP2pxxHqypzqma+SU5/A8XJrm3z2DuwLA5ydrm2XSsxN67IUfFnDTq9jlN6kWwzvu0E4weF9YU4eTwxFCyg04LWuhJxlUrF9B7TUaOmyPdf2QN/XlUmKv5Ofgh/9oOiprXR1eQ4/Cr8OdDmAKuxcWVW31nM7DOTKztfafI8yefv4sDfp1d+SsqcFGb1nWW6Ub2O8qjprEm+GbUag8KEgjxU5EDpeQjta1WwTk46BvYG4FDFytqks2VHlzHgkwH88O8PNl/PZu+e6HHQ741GW73fWJGSh6xWmvFvC1NToPfLRj+2eh2EGxVH1J4Q1g+CbKtCtYQxH0VTwbwI4LYKgx/mIuAlYAPwJ9ST/VwHbNTrG4fsp9pTqC+1JDS+0PMFiDMte7xjh9iK6tgRoxYJk5Of4f1I2JO2h/MFNso6uZGYGJHwU1Ehzx6FPRQVFdku/WwjhRWFTOsyjX4x/egX08/yCTaSFJFEp/BOVGor+ePEH7Jfv7EgjeWZLtgmcXaScl5ZHn+d+ost57cQ7O24Z/LkTmL8+OnwTw5fqzkgBf6OHjV/nC1UVxvGCcXjT8F9RI4QG2J2LsQ6hncEIL04ncKKQjw9DROLzEy4UCTKhgO8AgjyDrLp2hUZB/j4ig6Min7drr65HL94mLADLvkM8g/B70lw3DbvCLej14O2vMHbkhn0oUPUShpYQ0xgDOPbjQfghp9usFlDXvL3Kwm8FC7bAZHDbTpfwUaix0DvBeJexuDTJcnuupKMDCjyEXX23SK7WThaBlQaGPqDCHw2NYpOiUz6w6aDZs6goxj+Zan469pKzLQOZxnXVtDqtNx1oDt/9PVHE+rkaNLFlKbC1pvhwLOubTdiMLS+XsgSNjHkCvztTdvLJ7s/MXlf7E3bR7kmGw9tIGM62eYnpiAzqX/A9z5wzvrN9AcGPcDfE1PonPqcbIE/SerT25TUZ/w06P4MBHWRp0EF45RnQHl6A99oe7iy85UsvnIxg+JN/42HhIjXgossnjRqDXFBcUT4mdmFUak55vMCy47Oq91QVJCRsjTI3gpxl8tzvezt8HtXOG28ArQuY2OvwqM6iDzd+doksvd3vc+u1F1mFQZMYXPgT8EubA78qTXgFyeLpLfbAn/FZ4SftBOo66OYpNHwIsaDeYGAVF+ZDjwO/AaYGnkHAb9VVPD4o4+S7q4ITF3S/oH9T7u7F67BNwq6zzcpqZ2aCr/8Ir43Vu0HoOo2n12eYjH367GmI5OqVhvGCHf5/IFB+lmlslL62UYSghNYevVSdsze4RT/L5VKxdTOUwFYdmyZ7NdvLERFiVdXVvw5K/D3y9FfqNZV0yOyB0mtHE92mNZlGipUbL+wvValryUjVRNnZta3LnMESQnRy8tgTdBYUAJ/LQ29XmRp20GQdxDRAWKKeCJH7P7WHVwlvWBbq/0AzlwI5mRuPzyih9nVN7cQ1kcEUf0SjAbQGjW6KvghALbe3uCjuDixAKquFsE/W7ilp5Am2J22m34f96PSymoAMGwaK9nXLkavB72e2Fjxo9xrOUsyS+XlYjJd5PsvAN1auSDwp/GBhKkQNdL5bclNQFtoextEXOLSZutW/DkqtSIF/o5lH0Nr5Hl0tuAsWqpQ671o1yrWscZsxSsEhv9Pdvkli4T3h2HfQWgv17YrA9KYnZNDrfG9Pby8+WXu/P1Ok5mov/77NwDhRWOIiTIh9bl2EvyiPEScjn8biJ1kU2V+pH8kl3QTpeUZGWBDUYZJZvaZReeMpwgq7W088NdqCPR81uWVRy2OPq/D1HNic9IFSIG/iyv+rEXyCkpUFBnlJ6C98CJud4c81/MKBX21WLdYIDEqhP6nlnFLzlmSWiVxPOc4q5JXoULFnf3utLlpmwN/6ybDPhP+swomsTnwB0L1Iu0fq6vOjVFdLeQRwQ2BvzVj4c/+Trv8nLlz2blnD6d0OlbQMJi3CMhABPwAFgOjjRx3MYOAUV5efLp4saz9tYvkz+HwAiht2RvY27bBnDkigTokBMaastvtPh/fjrPxVHuSWuSGLF8HaNNGvLrL5w/skH62E2uUkuxFkvtccXwFFdXOSTxwN66s+MvOFq/OSg6SlAqu63adLNeLCYxhaGth09CcvR6tJTDQELQ9d06ea0oyn61ayZKbJCvKStjNOCOjwyRZW+HHYCF9ZCeSMbok91l3cK319wu0bdNNq4VDpxN5fPUGwjq5djNbFryCYcpp6Hyfu3tiPWpPiLtCyKJdhEplv9zn1C5Ta7/vH9MfL431qdQXLoBaVc1Iv7vhguLRZA8qlYqEhATrx5VzP8GyeMjZWVvxl5YmT18+3PUhbd9qyyP/PGL2OGliVuovqnykoJBTsWCO3qhRqWHwEmh9rUubTUwUC52iIscn04nBifh6+FKhreB0/ukGn0vPF//yjrSKcPE0xcNPeKBGuLGizB0mFmawNK4EBBg24h2p+usaYb4S9M+TfwHQpvpS04vusD7QSqkWdzrBSTByufActQF/fyHbVKnJ4dQpx7txeesb6Xj+eYIquipVOe7EjrWMuXFFr9ez48IO5v01z2hyiDGpz9SiVK7/6XqeWmPaAwmA3XPplns1apVW8fdzBh6+UHAI/n1RnusFdYLJx6H9DIuHtmoFEUWjKcwIRa8X81CAyztdbtH30Rg2Bf70eig4KFQZFGzCyyudE8efZ/nSJDrFxTEgKYkXnn/efFXZ0Tdh7aVQVF+GwpZ1UFaWmzLz9Xpocwu0u92pzfy6fDljvL0bBPP+D+H3Nx34qua95YC1rmK3lpWxbOlSmXrpAD2fh8mnapVrmjUbpsKmGxq8vXIlvPgilJZCUhK89Zb56qM7+91J9qPZvDhWpvHZRUiBv9MNl4xORxpTcnLEmOKMueYfJ/7gUKaN2fZ2MDBuIPFB8fSJ6UNGiQtK4tyA3PtZ5nCmKkBWSRarklcB8gX+AK5OuhpQ5D4l5B5brPUNdmkMqAYl8Odm1GoX/gp8oyF8IPjYn9bWKax+4K9exV+RHRV/BUfJP/IXGn0JXl4q15trt2SGfQ9dHzX6kST3uXu3bZf09/Ln4ys+5rpu17FwwkKrz0tOFl+xASeJKf4QMjfa1rACIMaT8PBw68cV71aiwqmqQFaPPxAbd2fyz1g0Ec/IAD1airyFwLZLKv7+6A2rxzi/nWaEp6dBEsFRuU+NWkPniM6A8SBPbeCvopNrN/P1eii94MIGL6L0Avx1CRz8P/f1wQjWjCtyyH3WSsBmN7wniiuL2ZO1GYAefpeavkivF0XlpEKjpFJbydqEcfzTO4ptRxz3mJEWWKGhRjKwc3bCstZw+qsG5ynITHUZnPkWMjdYfYq5cUWn1zHuy3G8se0NDmQcaPC5ManPk7kn+eHfH/jukIW//7IUwtiNTq9RAn/OImMdXHC9lJy04V1WBtkFpSzatgiAu/vfbdf1bPJzUqlgyhkYZruXYEtm0cKF9O4RT+ypBSwsPsoHqak8dPQoGxcsIDE+3rSfXPxU6P8ueNePctiyDnJbZr5KJSrRk+Y5tZnlS5dyy0WeHf8HPA+srPl+LbAdKMAg+2mJKKDgYp1ldxDYHgLauLsXrqGqGKrryyQcPgwffyy+nzwZFiww7u1Xy9bbCd77oM12PI0Bd1b8SWNKXp4YU+SWdTyafZRp30+jxwc9nO69p1apOXH/Cdbfvp7Wwc1zAiTN67Kz5VEWMYdU8ecMqc93d7yLVq9lQOyA2sIbObgq6SoANp3bpMh9In/gT5pXWFIRcGkMSGrT5S0q1EOrtU920y4C2sLYVQ5VitRW/OU2rPib2GEi70x8h+k9plt/wdNfEn7gMqIDkomPb3wlsS2Vnj1BoxGa8bZmzMzuN5vvr/meEJ8Qq44vKIAXXoCqKojq1Bn91FTo8pDNfVYQ48nRo0etH1eiRsLl/0LM+NoMqYwM0Okc70v7sPYAnMozn/2cmQkl3qfQqsrx9fC1KyvbJnRaiBwpPNWaKqUXYNUol/v8SXKfzvb5M1T8dXKabr5RCg6LCthDbsqE9YmCiizQ2S9d5QysGVdkDfxlHW4gEVylreLqyPnE5F5Dx/D29jeiIB9H34Rd99t0ipfGC0+fSvQqLb8lf+9Q8+XV5aw9uYUS75PGk8b0WvE35RHgUDsKVqDXwpbpcPITq08xN65o1BqGtRbS/xvONgwmGpP6PJN/BoDEEPP6nfqhP3D3n+JYJfDnJNrcLCTJ5SJ9DeyfDxa8w318hHSTHh2Rb/nXvj+hvW2VyRJSNr/ZzfSLURayVrNo4UJemj+fDVota7Tl9fzn/iovZ4NWy0tPPmk8+BcxUEiyXyQvbMs6yG3+fi6ioLCwXjBvEfAaMAYh2RkNvAxMBrwwyH5aIgMIlsqu3Y2uWqjXXFjp7p44l7GrYNSK2h9zc+Hll4Vq1ogRMHu2FfKTxSfFVw0F5Y0geGslkix3aqrYL3Il0piSlSU2R+RMSNXqtNyx/A4qtBVM7DCRyzpcJt/FTeDj4eP0NtyJv7+hMCU52bltOavir7iymHd2vAPAI0PMK2fZSuvg1oxMHIkefW1FYUtGSmqXyz/U2oo/l8aAalACfwo2MSh+EDf3vJkxbUS1TN2Kv17Rvbhv4H1c2t5MNv7FtJnOPo+3SSnsUrtxqOAiLqyELbdCWcOpvp+fkIwAONAw2Vo2qqvhlVdEdkRMDDz8sAqVXwz4xjiv0WZO+UXZndYSESEWDdXVhgwmR+gcLiq6TuScMOv1mJEB1ZpCEj37MyBuABq1xvHGzaHWwCWfisqgpop3Kyg8ApV5Lm1WCvzJIdEnyToeyT7S4LOjWSLwF1Du4oo/lUZIL7Ua4sJG66D2gMknoPcC97RvBkvjihyBv47hHdGoNBRWFDbwHwn1DWWM+hn6Jf9IVJSZjdWjb8K/L9nfCQXrSV8tAj162zJFruooksO2lzomFXY67zT/2TGUTUkDjCcIRFwCl+0U0r0KzsUzQHijdnvCptPMjSsjEkcAsOFcw8CfManPs/miFKBNcBuzbebkCFk0jUbxlHYa7W6FPq/Kd73UlfDvAig5Y/HQyEhQoebaRGG/8NZlb9k1r0xLE5vMYHkDB4DiZFH1WuYCfbFmQHp6Oo8/9hi/VVSY9JUbBPxWUcHjjz5qXvbzIqxdB7kt8Hfgv7DuCqgudWozwUFBtcG8dOBxoA1QNyQ/B3gCOIrw+bOGL319mTrdhiRvZ6Itg623Ca+/FkJBgQj65eWJgNj991uZbzB+E4zfRFpRGgM+GUDim4lm1+eNiZAQEdDR64U1jKspLy93SpDn7e1vszVlK4FegXx0xUculf/LL88ntyzXZe25kvY1+aGuCvzJnaTs5+nHkqlLuKH7DbUVenKyaMIijt57lNt73y77tZsaUsXfmTPyOK1YW/HnDpTAX0vj5Mew1/7MgRGJI/hq2lfM7DsTqF/xZ9cfS0h31qXej1bvWWvureAiCo/Ama/EYtUIXbqIVzmqe0zx449w8KDI0n3qKQio2OVeqb2WSOEJ2HEX6vQ/awP5cuiitw5uTZB3EFW6Ko5lHzN5XGYmhJT25+3uO1l32zrHG24JaLxgWhr0ftmlzdadSDs6OZrUcRJvXfYW9w64t8Fnx7JF4C+cTvj6OtaOTQR3gUs+h6jRLmz0Ipxo6u5M5Aj8eWm86BjeETBeCWpVFt3pr+CUtdtXCg4xeAlcW2DzPTt76LWodB7keO1l19mGgX9rySwRN4RXdaQiE98YSJgGwfJ59NYG/s5uaFABbEzq82yBCPyZrfirLqPowBe0Dj5EbKwVFRIKjYNO98GkQ+BvvpoTDM+HmyMWsWPWDu4faFtVMogEuNdeg8pK6N4d69an6atF1WueE7MlmxGLP/mE0Z6eJoN+EoOAUV5efLrYyHN93RVC/cJO3Bb4K02BnO2gce4Ed8r06XzlI6p7FgOjgUoaSnrOAfYAmxGyn+bYDqyrrGTmrFnydtZePANh+E8wrBn7VWWsg2PvkpOWxyefwB13wJEjIhA2f77YQ7GFqIAozhecp6CigHVn1jmjx7KjUhkq9M87rhJvF1JStFyBv1O5p5i/Zj4Ar1/6OgnBrtsInffXPMJeCePj3R+7rE1X0q6deJUjUdkU1dWG5DO5k5TVKjVXdr6Sb6/+1ikJ8X1i+tRarrR04uLEWqC8XBQjOEpjVhJomjtMCvZz4Xc49pbNGdqmkBZYFRXw7d7lbD63mYrqCutO1lWBTlu7UahU/LmYDnfCtYUmq1s61chJHzdv0eYQu3aJ15kzayZ0G6+GdZOc16BCQ9QekLwEsjbV+vylppo9wypUKhU9o3oCGPXokZAekFFRLjK63fcE7H5InrQed+KGAFHr1mJyVFRkyGiylz4xfXhg0AMMjBtY7329Xs/Vre8mPvt2Ev27ONZIU+XcT/DXIKhysjmBKXRasZm24y6r5wrSxmhqqpAeshdJ7vN8oWFlv+PCDj7Y+QFpmSIz2exketTvMG69/R1QsB7vcNB423xaYqtw4iuEnNHHW+yv+pMCf95Vkcazbc8vgyMLQWvlnFTBcbTlsq0v+sf2x8fDh+zSbI5mH633mVmpz2AzwaHS87RNv51Rbb6plQ5TaAIEtIGQbqD2tHiotC7NzfZgQNwAu+aV33wjkh79/WHePCuraWIuFf5+YX1tbq8lYsx/zhS3lpWxbKmRZ4VnsPApt5O6Hn8u5ZLP4KpMp8vCzpo9m7VVVWwHlgO3AMEYl/TsCbyCkP2sG/xLB14ABiCqBSeoVIwdN86Z3bad2Inga61DYRPk7Hew+36eeLScX38VCQkdO8Izz1C7breKzI1w7mfUKjXTugglhHd3vOucPjsBaZ1x7px72rfJ89UKHvzzQcqqyxjTdgyz+86W56JW0jq4NXr0bDy30aXtugop8OfMir/cXLGV5OEBQTLaZl6c6OZsLhRecHmbjQmNxpBU4KjPn17vxnmFFSiBPzfjcmPHAR/CVRmA/ZNNvV7P4azD7E3bi6en8D7Qqaq4+bdpDPt8GPnl+dZdKPUP9D8GEVP9C6AE/lyOZ6D4MkFHUXjB2bMisOsMpMqyLl0Qo2W3J6DLHOc01gJQq9W0a9fOtnEloC1M3As9n6/dCJNrotQz0rrAn05V5brMmNSVInuyqfuwVJfAkTfg7A8ua9LT0zA5clYWnUqlYnLYY/Q+8zlxYS7U+azMh78ugVOfu65NU1TmQeFxyHdT5YBaA2H94NyPUFVo1bjSqhV4eYkMSCmYbw+39bqNP276gxm9Z9S+9+TqJ7ln5T2sRmTGmh0rfGPAT5lMuITqMsjeASVnbT51aNCNAPx1drndzWeVitWVyYq/M18LhQuVk+WjFQSHXoDvfaHIOpkIS+OKl8aLwfHCi/dinz9J6rOkRIw5YKj4M+sT7BvNr/nLWH9muuLv19SoKoQSyzu+0vPB3uSkAwfg55/F9w88YIOEl38itL4WfBrhbk8j5GL/OXNEAQV1y3slhn4DI5bVe8uWdZBbM/NdsAaJjo7m5VdeYbK3N1mISr8pwFcmjpdkP0cAExCBwtbAeuAh4FPgPb0e7fr1JMbHG/dedBfVJXD6G5FU3szYr3+W/9v4D2n50XTqBM8/DwsXGixZrObfBbBNCL3OGTwHtUrNb8d/Y2/aXvk77QTcVfEnjSl5eeJvVo7A35rTa1hxYgWeak/en/S+SyU+AYYnDgdg07lNaHWu9xpzNpJCUUqK8/Yw60q/yvXrq9ZVM3LJSF7Y8AJFFc5P/p316ywS30xkzek1Tm+rMSOXz19BgfAgVaksjxMujwGhBP7cjqsHevxiwSvUoRHq/Z3v0+39brXl6VFRUOGZhh49nmpPWvlbuejR+FIdMoQzOe1QqSA21u4uKdiDXgd5+yH/oNGPw8Nrgro652TMFBWJL4DoaMQ92fEu4bGlYBcqlYqgoCDbx5XgJFCp6NBez1VJr5J6Rh7vuIFxA7kk/hLig4xvxJeXQ3ZRIX/2CeCKlT0oqyqTpV2zTNwH49Y6vx1no/KEA0/BqU9sP7f4DCxvB2dsr7aRJtNyBP4uFF5gyb4lrE5eXe99Z2nmm6XkHBSfgsocFzZqgra3wtTz0Gqo69uWqnV6vgBTzoJXiFXjikpl8MtyRO7zys5XclmHy2rbWn9mPatPr8ZT7UncBeHZZHaTruQclMpQsqxgmcLD8PcgOP21zaeOaycqBs5VHCSn1L6/uboVf0YDf/3ehEu3iqp2BecT0gNaX4e1iYXWjCuS3Oe2C9vqvR8QILJ0QVT96fV60otFHYtZqU/PINadnMLZgh5K4K8podfDsgTYdrvFQ6WxwN7A3/ffi+YuvRSG2GL32ww3T51JXf85S2QAwVK03wLWroP0eoN0n0sDf5UFcOpTKLBf5toW5sybxxMLFpCDqN6bBazFtKTnHOAsUAT8CmwE/gFuAsbWvP5VXs4GrZaXnnyy8QT/jr0FW2+G1D/c3RNZ2b4dnn05ml0p4xgwQMVLL0Hv3nZu43V7AgaLsG+n8E5c3+16AF7Y+IJ8HXYiUoGAqwN/KpUKL68gSkrkC/yNTBzJB5d/wDMjn3GL5GKvqF4EegVSWFHIwUzj+4BNmbAwoQyh1zsezDGFM/YqluxbwsZzG1m0bRF6nF+F5+vhi1av5Y1tjWQcdxN1ff4cQUomCg+3bCXg8hgQSuDP7Wgd0cSyB10V5B+CEvufmqPajAJg7Zm1lFWVERkJ5Z7Cly02MBa1tRJ0MeM5EvMPZ/J7ER0tqgUUXMxfA4X0oRFUKkPVnzPkPiWf9rAw2/XpFYyj1Wo5ePCg3eNK17B/mNH7Mbp7vV2bRe8It/W+ja0zt3L/IOMeK1lZUOR7CJ26koKKPHw9XWDoplKJ5IemjsZLSBoO/8X2c8tSoeS0kGWzkQ4dxKscgb/P933OjOUz+HD3h7XvrT+znkMZYkNEbs18s4T2FNJLnR9yYaMm0HiDZ4B1x2or5ZOtzVgrngnFyeDhW9sH7YU/SN76mcVxRQ6fv7pkl2Yz+zchf3N9x1n4VSYSGGjhebF6DKy/XJ4OKJgnsCP0egmix9t86qDukbRLf5iBGR+jUVmW7zNGepHk8dfKeODPLx4iLDlIKchG/BQY9j0EdbLqcGvmK3f0uYO9/9nL4sn1/b1UKkPVX0GBWEDnP5bP+TnnSQgy7ZOTm6OvfXZJSSwKTQCVCjo/CPFXWTxUGgvsqTyvqoKjNaqyU6faePLK7rDKjf7ATYy6/nOW+NLXl6nTpzf8oDxLVBqn/V37lrXroLw8US2sVot1qMsoPAbbZ0HKMpc1OWfuXO555BE+9/AgGniZhpKeddkD7AD+hloPxrqSn52A+4BpFRU89sgjpKdbG8J1Im1vhf7vQuRwd/dENrKy4K03ygn2TGHYMD1PPungXlnkCOHFW8P84fNRoeJ/R/7HocxDjnfYyUjJOqmpyLJHYS1arZZNm46g1+vx8QE/P8evqVFruKv/XcwfMd/xi9nZ/pAEkdmy8WzzlPuUM1HZGHUr/uSgpLKE/679LwBPj3iaIG8Z9UNN8MCgB1Cr1Kw8sZK/Tv7l9PYaK1Lgz1GpT2neaY3Mp8tjQCiBv5ZH4TFY2QNO2m/m2rVVVxKCEiivLmfdmXVERUGZl9jpM1XZYwrF38+NqNTQ+1XoYFpX3Jk+f5LMZ60+/a4H4Y9+7vO1aiY48iAJ7TKWJQff5pfDczhru3qbzWRkQKHffoBaP0DnNrgeUv9qPlIw4QOsDxDVpdUQuKEa2t9h86lyGmaPbiM2ydadWYeuptLsnpX38HxuV9JC/ufaij8Qm4uNqTro6Fuw5lLjn+l1cOhF+GeYIfXX0QBg3gEoOimCiRIVOag3XElQ3kqLp8uZjfvT4Z9o9VorTuSewFvjzfWxTwJWTKY7zIa2tzneAQXLeAZBt8chYqDlYy+iSxcYkPcakednk3ravsXtmVyRcOarjcZoMUhZWvMZ65spluYrrYNb0zu6Nxp1Q7lWyedP8t3RqDXEB8UbPVYi858n+GZaKIN7nhVKEwpNh57PQef7LB5W6/GXa/vm8MmTwj8rONiOdWnYAAjtY+NJLZe6/nPm2A6sq6xk5qxZDT/UVcKBpxsE0axZB0kbdBERhuphlxDUCUb+DglXu7BREfzbqNeznYaSnt8Aq4CvgXGIoOBID4/aoN8iIBHYhJD8/KDm9Qyg1+m44/bbXfXPMI1fPHS6t3kkdiKm8x9+CK39d7BkagKPTH7LYgWJrXSL7MbVXcV9+Nnez+S9uBOIiBCJf9XVhuRxV5Gb63i1n16v56v9X7E7dbdMvXKM4a1FkFzx+bMPqWJcrsDfm9veJK04jTYhbbi7/93yXNQCHcM78sDABwC4d+W9rlHeaoRIUp/p6UKNzF4as78fKIG/lkdAe+g2H6LtN2ZWqVRM6jgJgJUnVhIVBfn+OwEbAn/lmbDpOuG3hcGwV8HFdHlQZGmbQKr4O2GdZYtNNAj8qVSAHjzsCGQoyIJKo+Gk5n7KqoM4eVK+65ZXl1NcWdzg/cxMKPIVPma9onrJ16ApDr8EG0zf700OvV4Eao4sFJnPppA2v/U6KK5JZ5I2R6ttm+S1aSP+VPPyDBuu9jIgbgB+nn5kl2ZzOOswx3OOczjrMGq9B+FFo11b8XfqM1Hx1pgoPAyVuYaAXsZ6OPGh8CPU6yFjDYR0F79XvR523iPuB3vp8iBMPgnBXQzveYWh77OQnFDLm1RyBv46hHWo/X5mn5moi8XFLU6muz4GXR5yvAMKTsXDAwYMEN9v22b+WFNMjLmdDmlP0MF7SEPZK205/BILW5UgsMvQVcOu++HY2y5pThpvrE1S0uth15E2nMrrw5BRrny4KLiSkBDhR6zX2z5HOVijeNatmx1SekO+hH4tWy7LFur6z5kK/m0HJnt78/KrrxJtLFLvGwsTdkDvV2xu35bMfFnxCoG4y62ujJaLi/+/JUnPEcCbwD2ISsBtGg2tIyK4oyZqvgh4CdgA/MlFkp+IYODWv/9uPJKflQVQluHuXjjM1q2wYweUacMoirkHdaQtusMm2PMw/BQm1hA1PDvyWb6/5nteG/+a49d3MiqVYb/wnGW7V1kpKBBRV3sTUnNKc7jup+u4ddmt3PzLzZRWlcrYO/uQfP42ntuIXi7lmEZEUwr8ZZVk8cpm8RxbMGYB3h7ejl/USp4b/RxxgXGcyjvFS5teclm7jYngYAgNFfNGRwofpMCfW3yDrUAJ/LU0PHyh1wsQNdKhy0zsMBGAlSdXUuRzmNORbwJwbddrrbtAwRE49xPl2WITWhqcFRoXkqxfWprBj08uGgT++r0JE/e4xPBcwTQdOoC3phT/84ugwsHIDnDfyvvwX+BvNJtQVPyJwJ9LKv66PwNDvgG1fdJyjQ5dlUig2PcY6I2ktpemwB99YdtM8fPpr+D3zpDymwgW/T0E1l9hU5M+PoYNV0er/rw0XgxNED52a0+v5ZcjQrY0qmw0XtpQ1wX+dNUiaHb4VRc1aCX93oH+7xvGxFOLYefdoC0TgdthPwhpI5Ua8vZC8ueQ8qvt7Zz7GbQ17uc+F61qVSr0He+lzDfJ4mUkWZXkZMdleHpH9+a5Uc8xvPVwnhn1TO0mXWOdTLdY9j4Gv7Y3eEPawKBBUOhziM8OvUdBeYHN5/fymkaXCwvoEtK74Ye6Suh0H8RcZvN1FexE7QGnv7RvDDLDzgs7uX3Z7cxfXV8Sq+6mztcHvub6n67n+0Pfm7zOwYPw/Z67WLBtDZcMVxLMmhy5e2D1WIv3l0pl2Jy1Ve7z33/Fa/fudvRPwWYk/7kRGg0TfHzqVZ6N9/ZhhEbDEwsWMGfuXOMXUKlqlC8CbW7bbXOKuooKLubi/+/VCCnPB4E2Pj6c0Gh4/tVX8fTyIhoh7/k48BsGyc+LGQT8qdfz+KOPul/ys/gM/BQKh192bz8cpKQEPvpIfD/w0u4Ejn7PLmWFBvjGQmjvem91i+zGdd2uM1sp35iQAn9yWQpYS36++P+xZ136b+a/9P24Lz8d/gkPtQc397gZL437/Y0Gxg3k1l638n+j/g+tvvl51Epr0jNnnCMNK6fH34sbX6Sosoh+Mf24vvv1jl/QBgK9A3nrsrcAeHnTyxzNPurS9hsLcsh9ui2hyEqUwJ+bUaub5q9gbLuxeKo9Sc5L5veM99GrqokpnMy0Lpb9FwCIGon2qkJ+3nsrYAgwKbiY5C/g147C99EIgYGGwJzcVX+pqeK1NvCn4DBqtZrOnTs7NK506ACD4pczzG8unPzI4T6F+4aj0+s4kHGgwWfpGTqKfEWKtUsCf60GQ2vXyus4FY0XTNgJl24D35o/pPO/CP+QilwheaOvFtnFAIGdIGqM8MBQqSGoMwS0tVkiUhqv5ciik+Q+155Zyy9HReAvPEt4ULiu4k8FY9dAj2dc1aB1aLzqL/b7vw2j/jD8rr3DwaPGbCKwEwz5Wvic2MKF32HTNbD3YZOHqNVqOnfqiFpvfsMqNlY8Myor5bk3nh75NBtmbCDSP9L6TbpN18O+xx1vXME61B5inKmyPXDXty/s7jiNzcH3sWzPJpvPl7JtjS66PYOg/zvQzsa/BwXHmHwCRlvnE2LtfCW7NJsv9n/BNwe/qZeVXjfwt/ncZn749wezPkV/1XRr1CjFV7pJolJDzg7hUWwBadMly4wQwsVotXD4sPje5sBf5gbY9QAUOsEXoZkzZ+5czqakMGL+fN5MSmK6fxR3e0Zw2DuAhFatWPrJJ7zw/POmg0raCrGGrfGstnZcyagpCouKkvNfYwWbroGfI+1KlpGDi/+/74mN5a2kJEbOn8/ZlBTmzJ1LcFAQ6cBiYDSmg34Sg4BRXl58unixhSOdjH+imANHXOLefjjI99+LauW4OLjWynx6q0iaK9Y6XiFGP67WVVOlbdzy6O6o+FOr1fj4xIk8AxvXpauTVzPksyGcKzhHh7AObJu5jfkj5uPRCGwlfDx8+GLqF9zZ785G0R+5iY4GX1/h3euMQLFcgb/SqlI+3fspAAvGLkCtcn1s4Kqkq5jUcRIhPiGsSl7l8vYbA1Lgz5Gkdlsq/twRA2qaUScFxzj2LvzWCcrsz8wK8ApgROIIALrHdmDosY10Pf0eRUXWV2udTw+gsDQQX1+xYajgBtSeoPGG6hKTh0g+f3IH/upV/FUVCoP2rM3yNtIC8XLI+VsEdTadu453dn5GdYc5DvdHCugZC/ydzDpLtaYIT5UXncKdLHujq3bbQtupqDUQ3t/w8/lfRAWXrgo8/GHCLkisyR5rNRhG/2lY9F3yOQxabHOVrZyG2aPajALgl6O/sP2CEHyKzp+ChwcEOd/XWqDWCN/Dxr5Z4BUKsSYqmDwDoPU1DSv2LBE9Dvougm5Pmj4mZxc+v8cImVEzqFTCuw3gyBHbumEJqyfT2Vshb7+8jSuYpteLcNkuu3x1/Pygk5dQn/hl73qbzk0rSmPThbWUeV5otJmVLRKfSIOMtBVYM18ZkTgCT7UnZwvOcjLXIGUseXKkpsLpPKHNkxiSaPQaRUWwZQvc0WceV/dZYnX/FBoRIb3g2kLoeJfFQ+0J/CUnC28Xf39INH4bmSZrMxx/p56EnoL1REdHM/+pp5g+cyb5ZdkM0BbzamE2H6Wn89DRo2xcsIDE+HjjcpJH34CVPYRHcQ3WjCtuC/yF9hXzLjds7kpI/987Dx/m+IUL7Dx8mPlPPVUrpzpl+nS+8vFhOXCLlde8tayMZUuXOq3PVqFSweAlhjVPE6S6GlbV7LvPvEOP14aR8K/z5ffe3fEuiW8m8v2/pqvmGwOtW4tXOSwFbCE/XwTGbAn8LTu6jMu+uYzCikKGtR7Gtpnb6Bfbz0k9VLgYlaq+Eo2c1JUSdzRJ2c/Tjx2zdvDsyGcZ3268452zA5VKxY/X/kjGwxncN9Cyl3JzpEcP8bptG+js3C5s7OpESuDPzejsvbMcReUhvIMc4MnhT7Jy+kruGfgfOngPxbcqoXYibZFzP3PhqIgktW+vqDu6jTbT4fJDEGE6n08K/B2XMZG1rAzy88X3MTEIeY4DT0N6y8wykQudTsfBgwcdGleiosDPX8PfJ2dwNsXxtPgeUeJJeijzEFpdfSmJrNxqYnNuYGzClXhqnCy/ef5/8IO/qHBqzgz+QmzE+9bsZjhBTkTOwF//2P74e/rX/twr/BJ8qmKJiHDhc6E8WwSGmwNlGbZJ9Gp8hCeer+nSa51/O0q8OqL3thxhSapRBD0qs1KI1fIZU8/B6D/kbVzBaYxuJxLIdmTYFvj78+SfvJU/hv1tbzd+T1xYARuvhYLDMvRSwWrKM0X1U7Vl/xpr5yv+Xv4MbS0kof8+9Xft+yEhEBYmNmBOZp8BoE1IG6PXWLsW0FUyrcsbRFavtOZfotDYUKmsnhRImy62BP4O1RSLdusGNidiJz0KU85BqAu8qpspixYu5KX589mo07JGV17fS668nA1aLS89+WTD4F/UGJG45C12X60dV9y2QdfzWRjq5gCZBWbNns3aqiqyACPOiqQDLwADgE41r6uAHEeNvxXYuVMkqoSFQb8euVB0Stg2yEHmRtj/tNHE/7yyPFKLUvlwl/kEP3dTV+rTVVuoOp2OY8ey0ettC/L8d+1/qdZVc03Xa/jnln8I92t83sKSItOSfUvc3RWnIClDyLFfUZeCAhGkV6nEXNQWLt4LA0hqlcQzo55B5cYNcT9PP7e272769BGqRfn5cKBhrYJFysqguFh8b01CqjtiQErgryXS+T644jAEd3XoMmPajmFix4n4evrWTpyt8lOozINN19Aq/TlAkfls7HTsKF6PH7dZEdAkkmJLUJDIriWoE1y2G9rNkKcBBbupmyF17nimqPJx4BffPrQ9vh6+lFWXcSrPMPOqqgJ9dkf6nv6W76/+0dFuW8YzGCJHCSmY5oxKBYHtrT/+zHfw5wCbMtWlSovMTMe9Pz01nvxx0x9cEi+q7QaHulrmE9h6M/zcqulXhGasg1+ihZejJapL4d8FUGmFRKNXCCfbfIo+8UaLh0qBv8OH5XteVFdDXp74vrFm0bVYKgvg6FuQ9o9dp988TFT8pat3cyGr2OrzpGeJX3l74xUbhUfh/E+18m8KLuLkx7BqpPj/l5FL210KwD/J9e+zdu1Aj46U4pqKv2Djz/dt26Ba58lf/hnQ901Z+6bgQnJ3w/H3LT5c7Kn4c8jfT60B/wShoKJgM+np6Tz+2GP8VlFh1kvut4qKhl5yEYNE5bkN81693rBfEW0sstXCiY6O5uVXXiFfpeLiENEiIBHYBDwEfFDzeh44l5FhvCrTlZSch9Vj4Ogi9/bDTlavFq+jR4PaNxympUC/t+S5eNZm+PcFKLvQ4KOZfWeiUWnYfH4zBzMOytOeE4iKAk9PYSlgq4erIxQU2F7xt+mOTXxw+Qd8MfULfDwap754UUURfT7qw4zlMzhf4OIyShfQubN43b1bvjUpGOYWoaHgYYNK6r70fcS+EcuU76ZQXFmMrhHuO+j1ert815s6Hh4wVOQYst62XFTAMB4FBAiJ2caIEvhTkAVp48Wqij+VJwxZyl+n7wYMgSUFN6DXw4mP4My3Jg9p3x40GpEBIZfcZz2ZTxBVJ2F9wb+1PA0oOIQUjA9PewF23g3ZW+y+lkatoXuk2EmpK/cp+TN5eYkMG6cTO0FUAoX0cEFjTYjqQig+ZdLn0xj+/oa/XTmy6IYnDmftbWv5/cbf6ec5HXBx4C9qNLS92a3SS7IQ1hfa3AIhVuxcJi+B/fPhzDeydqFjR/G8yM21bdPVHJmZ4lHl7W2F/OuF3yFnlzwNK1hGVwV7HoJzP9h1ep92iQRqE9GrtHy5zvrnzKlcMfD4V5gI/CXNgxuqhDygguuImSCkg33k3U0f317IH605vaaeB1G7dlDgt5sKXSkBXgFGpT61WkmxQkVSn0jwU7wFmiynPoNd91r0+ZMCf9ZuDOv1hsBft2529Ct7B5Q23ExXsI7Fn3zCaE9Pl3nJ5eWJ5EO12sVzzaJTwoPbxgp3dzBn3jyGXHopH9d5bxHwErAB+BPqVWWuAjaD8apMV+IdAbl7oVymCagLKSiAXTXT17Fj63wgl/9a+5lw+WEIbjjIxQbGMqXLFAA+2v2RPO05AbUa4uPF966S+9RqobBQSJjbMl4EeQdxV/+78PP0c1LPHCfYJ5gBsQMAmqW3W//+IlB84QKcPSvfdW3xcqvLE6ufILMkk1+P/cq4L8cx7stxTP95eu2axt2sOL6C2Ddimf6/6e7uilsYKXJR2bpVJBfYgr33hCtp4rtcCnaTvMS6qgArkW5yU97b9fAMQJtwI+sODQEM1UUKbkClgoPPCG8KE3h5wfDh4vv33hPVF46SWrNmr/V2rMwz6zOo4FqkwN/vJ+6HYT9A+ECHrif5/O1PN3hvpWZUkBbyP8IjdIrUrztpeztcnQ2Rw2w6TUrY2C+TnZqPhw+Xd7ocXb5Y0bl04tT1MehvegxsMngGwZAvIXqs5WM73AmDv4L2s6y6tG/5MVRbb4bMTWaP8/Y2SKvIJfcpJRRFRlqh9LZhKvz7ojwNK1jGOxzGbYAe/2f3JfqEipXWykMbrT7neHZN4K+yvWlJFbWHTX5zCjIQPkBIB8scXOsT3Ydw33CKKovYcWFH7ftt20JGiJDvvrT9pXgZkbY+exYqKiAqJIeE4KNKFWhTpuNdMPpvi56idaU+rcnyP3tWSDT5+NixJtXrYfVI2D7bxhMVJJYvXcot5db9XRr1ktt0A2yzXjFGmlNERIhEJZdReBROfSqS7ZoAny1ZwjaNhu0Iec/Hgd/A9qpMV+LhK9Y0vRe4p30HWL9eBJk6dqyRtDz7A6T8Kl8DPq0gOEkkWxvh7v4iIX/xnsX8dPgn+dqVGUnu89w517QnWdOo1dbJOlZpq9DLWV7mZC5tLxQV/k7+28KRTQ8/P+jbV3y/ebN81627LrWFr6d9zcA4sae2/cJ21p5Zy/f/fo+mkaxVYgNjSS9OZ8PZDfWS7FoK3bqJ4H5JiagStQWrLUnciBL4czNqm40EZOLgc3DkNdkuJ230SRmTZtHrOX9eRNJ9fesEfxTcw/BfYPCXZg+ZNUtUZSUnw/LljjfZoOJv7yPwQ4Bt3lQKDVCr1fTo0cPhcUUK/O082pHq2GtB7Zj/Xp/oPkzsMJFbe91a+95X/37K7g5Xszr6coeubTXbZ4kFt0J9NF52VboNHixeN22SVz6jKUycmgVqD1HlaIUHpFqtpkO7BNTnvoO8vRaPl+Q+jxxxtJMCaYFllSTXJZ9Dp/vlaVjBMioVRA53KNBzWU+xCD5euLe2EtwSktRnnG974zI7GetFFY5Co8WW+YpGrWFcu3H0iOxBcaVBErZdO9Cqi1HrfLi8w2Sj50oJCJN7/4xqRZKQRFZomoT0gJjx4GG+giIiQryWlxs8V8whbQgmJdkRCNJVQa+XoL1iVWAvBYWFRr3kjBEFFBRcJENWcrbWB82acUWaUxitFncmsZPg6hxofZ2LG7YPSfJzsrc3zwCjMR30k5CrKtMhGskmuq1IMp9jxtS8sX8+7HtMvga0FcLfr7rM6Mdj2o5hSucpVGgruPbHa3lrm0wSozLTukYcylWBv9xcNYGBQYSHW+f/+vyG5+n3cT9WHF/h/M7JgBT4++fUP41SetJRJPlGOfcrpL0KW58h4X7hbJ+1nUN3HyI2UKybrux8pUl/alfTK7oXYb5hFFcWsyu15annqFQwQljP2yz3aWvFnztiQErgr6UydCkM+1m2y/XqJf5Yzp+3Qt5r171Eb2uNn2cB7dtb7dWu4CxaDYZA80aLwcEwc6b4fulSQ+DOXhoE/loNg3a3W8ziVbBMpa216UaIihIa1dXVIthLyVmHpGnu7HcnK6avoGO4KBMrqyrjyzMvANDH7wqH+2uRygIR9MsyX63UYik4AodegIocq0/p319UA6en19wjMmHvZNpukr+AjVdDiYtWkM6m4Cj8NUh4cxqjLAN2z4Vy28wxKv17oL86FzpbDqo5K/Bn1T3R9haIHmP5OAX50FVDmf2TglsGTOHGyjX0Pv0Fa9daPj6/PJ+CSpEk1D7MRHnOzv/AdmUj3uXotMIzdsfdVh1uy3zlq2lfceDuA0zoMKH2vZgY6Ju9kAn7chgeer3R86TAn09Mb+j2pKh4UGi66PViA9sMXl5i3QKW16SHDsH334vvR4+2oz8aL1Hl2vpaO05WAAgOCmrgJWeKDCBY+uVKTNgKYwz+n5bGFZfPMyVUKvAOA88AFzdsP3PmzeOJBQtYCtxi5TlGqzJdSUUuHH0T0te4rw82cuaMWEt5eBjk5hj2Awx4X75Gzv0Av8RAuvHKLrVKzU/X/cSDgx7ES+NFv9h+Ro/LLcvl3hX3UlhRKF/fbEAqNpBrjWGJnBzQ6XSEhVk+tkpbxeI9i9mbvpeSqqahZDUobhCBXoHklOWwN81ycmdTY+BA8XeVkiJfsFh6hlgb5Nmesr1eFWi3yG5suWMLTw1/ircve1ueTsmAWqVmdBsxEVpzuumMn3Iijb87d0JpqfXnNYXEdSXw52Z0OjdlVkRcAkHymesFBECXLuL7PXssHOwTQ35lAqVVQYq/X2NAr4fybIsL6TFjRIC3shLef9+xrJkGgb92t4tKDSUK7BA6nY5jx445PK6oVNC9xiZs7x4d/DVQVGXaiafGE1Wd3+37O98nrzoN34rWXBFnndSgQ3gFw3Ul0Fu+KudmReYGOPA0ZFof3PXxEcE/EFl0snXFxsm0wxQeg5RloPZ2UYNOxjcaipOhysSC/PQSOLYIsqz3U9PpdBw7cQqdhyWDPYE0Fzh9WlRcOIqkGGVVxZ+C69l6K/wSa7eEYnxQPHeMHo2XNpzVqy3PLSQvDO+qKBKiTGyg9nwRejxrV38UHECtASszxm2dr3hqGioPqFRC7lOj8yP1nK/R86TAX6suA6HXi+Df0AdQoYlQkQs/+MPuByweWlfu0xS5ufDKK2LMGTMGRo2Sp5sKtjFl+nS+8jEuP3gxX/r6MnW6af8ha8YVaU7h8sBf3n6RnNXEmDN3LlHR0Y5VZboSXQXsmQNnvnZfH2xAq4UPa3L1Bg6s43sf1kd4kMtFUBJ0vNfsM9BD7cGbl73JwbsPMqx1QwuIvWl7GfDJAN7f9T53rxAJPpXaynpWHs6mWzdReZeWhtUqEY6QlaWjpKSYsDDLG1+/HvuVtOI0ovyjmNplqvM7JwOeGk/GtBUJk3+fan5yn/7+BrlPufYrbNmrOJBxgEs+vYQrvr0CrU5b+35iSCLPj3mehOAEeTolE9K9sOZMywz8tWsHcXFiv3vbNuvPs7Xizx0xICXw15KpyDW9OWgH0qBqMfDX42leP7AZUNVKCiq4kePvwv9aQbb50U2lgnvvFVkz+/bZrn0sUVlpmKjVBv4UGh39ahL9du9RQ6+Xoav9gT+J7NJspv88nYf/eRiAjmlPE9PKRQEXDz/wiXBNW02N+KkwbiPE2ia7Knl/yiWfUVIivsCFGVO9F8B1ZeDTiN2YbcErBK7KhK6PGv886VHhkRQ/xfZrl6XDmW+h3HwJRUSE+NLp4Phx25u5GKsr/qrLYFkC7HnY8UYVrCdmAnS6D3T2V5sPHSr8IS9csHzPxAfFc73fx3RM/a/pYHDrq5UKHHcxcTcM/MBply+rKuNC4QVAzCnathXvG6s8LygwJJp17uy0Lim4Cq9QiBoLwd0tHirNIaTnx8VotfDaa8K/qU0buOceO3MPj78nqlyLTtpxsgLArNmzWVtVxXYLx20H1lVWMnPWRQmDRSfh1GdWV567reJvx52wpmkqEoSGhDhWlelKfGNg7Fro96b7+mAD330nrHJ8feH222verMi1mJBtM+H9YcC7ENrb4qGdwjvVfn8y9yRrTq/hxQ0vMmjxIJLzkmkb0pZHhjxCVkkWI5eMZNQXo2qTspyNn5/BkuTgQee3l5cnHgzh4ZaP/ebgNwDM6D3DqOdwY0WS+1x9erWbe+IchtXEsOXw+dPrbVOieX3L6wAEeAU0Gi8/c0iBv83nNlNe3fI8sVUqQ9WfLXKfSsWfQuMl7W/4ORzOfifbJfv0Ea/794sFlSm0WlEJAHaYqCvIT1g/aD/bKpnNmBiYXGOjsmSJ2Ni1Felh6e9fk9VWXQbrLlf81xoZUiD/6FEojpohyyZqSmEK3x76FgBvXQjxObfVerE4lYLDkLtXSNIpNMQ3CiKHgca2IKzccp9StlRQkKgodBkar+ZVbSz9W0rO1q++0evEZzHj7fr3qjLXwpbpcP4ni8d27Spe5ZDisX6BpQPfWPB044ZTS6TdbdD/HfC0riLUGDsy1pPT91HSQ5azxkKSaVRAFO3zZ9Mm6x7Xb9wquJUPd31I6CuhDPhkAAczDpKwKIHXintS6ZFt9BkkVfslJID//lthi7VidQqNEpUKRv1mleS0FBA2lYy6Z4+Q+fT1hccfF4kHdlFdCuXpDo1/LZ26XnJS8C8deAEYAHQCugNjNRqefOopoi/O+MjcANtnQq6lzGOBNKdwmbKEROcHofvTLm5UHuSsynQJUaOaxN/kgQMGqeH77quTEL3/SVHdbKMsv9wczDjIoMWDGPvlWJ5a+xRVuiqmdpnKrjt30Tu6N8E+wahQkV+ez7U/XkuVtsol/erZU7zud0GhobShHxFhPsO1qKKIlSdWAnBD9xuc3S1ZmdZlGn/c9Ae/3viru7viFCS5z/PnHZf7LC6GshqbTEtBnrr7Xg8PbhpJqZ3DOxMTEEOFtoKt57e6uztuQfL527dPJBBaorpaKEiAG+YVNqAE/loqQUnQbgYEyFdy17GjCOSUlJjJ2C7PpHjDXDoFr8fXF2JjZWtewV5aDYFBH0NoT6sOv+46Ie169iwWN+iMkZoqXmNiavaeKzIhfZWQ3FNwGI1GnmyiyEixWabXyzex7h3dm8eHPo4KFf1TP0Wt93TNA/LQi/BnX7ul6FoEep2QIaoqtvoUueU+bfJyk4PqMjj9NRSecFGDLiRvP6zsDfueEH/EW28Xkox2ehlqNBr0sZNh4MeQeKPF47t1E6+Ojh1lZVBYI0xg8b7w8IcJ26FH09xYa8msPr2adVWvkRH8G+vXiyocc5gdK3L3wi/xogJEwfXk7IJ/X7bKM9bW+cp13a6jY3hH0orTuOTTSyivLqdSVYhndTinTzesPJcCf126AKVnoeSMTe0pNF2kjO09e4yPJ1JAcPRoIetkN10fgannm49qgJuQvORGaDQkaTS0BtYDDwEfAE8Ag1QqFrzwAoveeKP+ydHjYeTvED4QMD+u6HSGJDOXJ460mQ4drfM/bWw4XJXpavQ6KDolrEwaKSUl8Prr4rk1frxhsxmA8AGQeAN4y1g+UnBYeJpfWGH1KZ3CO9E5XJTLB3sH8+XUL/nfdf8jzFcY3nlpvPjx2h8J9w1nb/peXtn8inz9NUOPHuLVFRV/Yr6psjhe/Hb8Nyq0FXQK70TPKOv21BoLMYExXNbhMvw8/dzdFafg728oUHG06k8KBIeGiuRnc7y17S2qddWMTBzJgLgBjjXsIlQqFdd0vYZHhzxKUquW6YkdFyfiGjqddftbOTliHPf0NPhLN0aUwJ+bkWuT3mb8E+CSzyBaPskJtRp69xbfm5SBLDxKcNoi2oftoWPH5lVk0VIICBDBP4Cvv4YKG5UoLgiFJkNWm38iXF8OPZ+XrY8tFY1GQ48ePWQbV6Sqv4wDa+DX9nD+fw5fc8HYBaQ9lEdo+lUArqn4a3cb9HkNPE34QSnAyU/gj96QYVs0X5LPkEPu0+UyCUXHYestTcYHxCaCOoukjuix4kHrGQyVeaCyfWyoHVd8gqHDbCEnagFpgXX4sCEz0h6kAE9goJD3UWiEFJ+BTdfD2R/svoS0SVIatJ+SErjlFnjwQfjll4bH/nZsJUfLNlCtLja+EaPXgl8ceAYa+VDB6WSshf1PWEzmsme+EuYbxj+3/EOn8E6UVpUCcGWXyXh6qCgogJMXqS1Kgb+kJGDcehi/0ZZ/iUJjJGW5qNysyDV7WEyMkHfV62HDhoaf790rXqV5roL7mTN3Lk8+9RSZwEbgH+AmYGzN6+rqajZotbz05JP1g3/+CRB3Ofi0sjiu5OQI5SEPDwgLc/o/qdlgrCrzYrYDk729efnVVxtWZbqa9FXwWwdZla3kZtUqyMsTSfD/+c9FH7afCUO+lnejrLJAeJqXnrf6FG8Pb1betJIPLv+AQ/cc4pZet6C6qE9xQXG8PfFtAJ5b/xz/Zv4rX59N0LUraDRi3WhKzlkuMjLUBAUFERdnfq7yw79iDnxd1+sa/B8puJ/Bg8WrLb5txrC2YrygvICPdn8EwCNDHLfLcSVvT3ybV8a/QnSAm8dxN2KL3Gfd/Str//TdEQNSAn9uRi+HMVIjwpLP3+niIcz67TRrT9/CtYr9SuNh72Ow7Q6rD7/8cvHAy8mBV14RpdDWyn5ur1kx1PNbUalslhlUaIher6ewsFC2cUXy+dt/OBi9xl8WIzeVSkVloUiH8fcXMktOJ+ZSSGoaEgtuI3osJD0CAbbpLw8YIJ/cpy1m2bLgnwjDfmqeXmAaHxi1Qtz7AH1fFxnxfraXNjQYVypyIXuH2XNiYsSXViukjOzFpirQqmJRaZTePD0qGi96OPcD5Nufet0rqhcAxb6HaNNOaMUnJ8Nnn8Gpi2xj7v79LjZ3Gklp4EHjG7fh/UXlZ3P8u24KJN4Al26H0F5mD7N3vhIdEM2qW1aRGJwIwA09rq31m122zHCcVgsnaoq5u3SxqQmFxkz+IZGsY0X15qhR4nXduvrvZ2aKJES12lA5Yhd6PRx+FTJsMIJRMEl6ejoLXniBlVotg0wcMwj4raKCxx99lPT0i1zntJUWxxVpTtGqlfj9u4yMdbCiG1z43YWNykvdqswJPj58A6wCvgZGqn0YptIw44EFzJk71809BUL7QOeHIKyPu3tiFL0e/vhDfD91qgNSw7YQ3l94mne8y6bTQnxCuKv/XcQHxZs85sbuNzK502SqdFXMWD6Daidba/j4QKcaG0JH1hiWKC2FwkI91dVVREWZn6vM6D2D67pdx/Xdr3deh5xIZkkmj/7zKNO+n+burjiFgQPFdmNysmG/wR6s9Yj9ZM8nFFUWkRSRxMSOE+1vUMEtDB8u7pcjRyzfL/bsX7kjBqQE/tyMzh6TNLk49i6sHgM6+fS4pcDfyZMGeS4JvR4++NCDjOI29BgQUVsdqNAIyNsjsrStHIS8vOCOmjjhzp3w9NMwa5bQzjZHVpaoAFGpDJVC5B8SmXnVDpSFKABiPElOTpZtXOnWTfyu95zpx9luB6D11bJcV5LZcUm1n4J1BHaAPq9CSDebTpNT7tPlgT+vEHFPh3R3UYNuRO1pd+ZwvXFFr4e/BsGma+v7BxpBShwwqQBgBdK+nnWBv3xRadSEN9aaJP6JcH0F9LK/ar9daDv8PP2o0JVz79Mn+Pprw/1TN9uyorqC1OIUANoEtVdUIxoj/gkQMVBI75rBkflKQnACe/+zl20ztzEicQRTp4r3N22C7BpltzNnhCKFvz/ER2RB8hIoNOVDoNBk6Hw/XFsIYZZL9YYPF8GdEycMaiNgqPbr3NnBSvLyTNj3WKOuKmpKLP7kE0Z7epoM+kkMAkZ5efHp4sWGN3+Jg/VXWBxXrN20lR1tmfAZV3m6uGF5mTN3LmdTUhgxfz5vJiVxT2wsb3ZJonLAfEaNTeHosbn88IMseaKO4dMK+i2CVkPd3BHjHDwoxiQfH0OCQi0XVsK6KyBvn7yNqj2Fp7kTUKlUfHjFhwR7B7MzdSdvbXvLKe3URUracGbgLyND3MsqVRE+PubnKlO6TOH7a76ne2TTXFP6ePjw5rY3WXZ0Gceym5/9TnCwwYZiu6myZSuwdq/iy/1fAjBv8DzUqqYXctHr9fxz6h/uWH4HWp3W3d1xOWFhhjFm/XpYuxZmzID//rfh803a17RFscodMaCmdxcqyEfZBTGpKL1g8VBrCQuDNm1qgnwfCP1yibVrIf/8CaKDU3G39LvCRYz4FaactmljeOhQeOMNmDRJyLBlZcGPP5o/Z2ONylK3bhAeXvPmyU9gzXioaLw6/C0VLy+DgbYjm/cXI23MuSTwl70NlreBs9+7oLFmgo0rdrnkPl0e+HNyRmqzRKWC7vOFNLPe/EJASgTavdv++8Kmij/vSLhsF3SZY19jCvahUju8maRRa+gRKVZYBzIOEBwME2sSZDdsMNw/p/NPo0ePRhtAoqkVVsqvcPg1JZnInWgrLEoxOkqobyiD4kWIoF07MVfR6eC338T9sqLGxqhzZ1AVHIBtMyD9H6f2ScEFeAZZLeMbHGxIIKhb9SebzKdXKFy6DTo/4OCFFACWL13KLeXWeXHfWlbGsqVLDW9Ejxe+aBawVqZNdmInwuRjEDvBxQ3LT3R0NPOfeoqdhw9z/MIFdh05zJq1T3HZZdHodPDVV/DOO+7uZeNGqvYbM8aI8k3RCUj/2znR07wDYo7kBGIDY3ljwhuMTBzJ6LajndJGXaT9iQMHnBdolpIPw8Ob/3oxyDuo9vf26zHn3CPu5pJLxKsjcp/WPkN2zt7Jxhkbm2wFaHl1OTf8fAOf7/u82d4PlpDkPr/5Rux5Z2eL+aNkIyDh8v0rO1ECfy2ZHs/C1TkQ0EbWy159tdgb3LQJ7rsPfv4ZXn5ZBAIfGDSLdyf1UCp9Ghse9uktduwId98tKv5AZNBUmSkglTL3pYEUgDY3Qf/3wDfG6DkK7kXaGKk48TMcelGWa9qTGWM32nLwDAEPxd/PIql/wrJESPvbptPkkvt0eSb26lGwsmkZsDcK2t0O7W4V2cNm6NFD+OhkZkJqqn1NSQssq+xiNF4Q1g/8W9vXmIL95O0XWeoOIMl97k/fD4hnj7+/kBT/t8Yy5lSu0P30r2hPdJSJRKWz38K+R0VAUsH16LTwQwBsn+nSZqWqv7/+go8+gn/+EWuRyy8HQnrByN8gdpJL+6TgBPQ6Md7kH7LqcKmaZm2NqIlOB/vFEFPrRWs3Gi+IGATBSQ5eSAGgoLAQax2FooCCggLDG4OXQLIPlN4AAMYESURBVC/LaxSbkokUrMbXF+bOhQdqYuCrVjkmpycLZ74V8qoFRy0f60Jyc2HrVvH9RGMKgF0eFJKcFuSy7WL/E0Kxw0kVPDN6z2DtbWvpG+N889SkJLHGyM21f41hCSnwFxFhenMrvzyf59Y/x9HsxnWf2cOVna4E4NfjzTPQM6imnPzQISgqsu8a1gZ5vD28GdZ6GAFeTXP/ydfTl//0E+ajb213fgVvY2TIEDHGaLVin6t1zfbCPxflELp0X9MBlFVxS0bjLa9pcA2jRgnft5gYERlfsgQ2b4bycjhcdisePZqWwWmLQFsh5NGyNtt1epcuooKvtNSQSXsxFy6IoIBGIwbSWiIGQqd7QO1hV9sK9fHx8ZH1epKMY1vVN+gPPCOLNLBU8eeSB2TUKJi0D+Iud0FjTRyfVkKeTWtbpYwccp/l5QZ5aJdNnEL7QPhAFzXWtDE6rlQWmB0PfHwMsiqmfH8tYdMmnU4LVUWgrbSvMQX72fcEbLzKobTrnlEiCH8iVxizeXoKZQEwJA0dyxHyQ37lHUzfE30WiiocxTfYPag10GE2RI6yeKic85X+/SEuTiiNSNV+998vfF3wiYC4KyCgrWztKbgJvQ7+7AcHnrbq8EGDxLMoI0NU2Zw8CcXFIqmgQwcH+1KRozxvZCQ4KIh0I++nAy8AA4BONa8fAX7+xuWEzY0rbgv8nfpMBKKaMSoVjB8vKrH0epGE4V70ovK/IsvdHanHP/+IjeSkJKGSZRS1xjnJS50fgiFfA86RmVOpVKjq7C3qLFgCOIKXl8G/d98+57QhBf6io03vl645vYZn1j3TLLzxruwsAn9bzm8hq6Rx/d3IQXS0+JvT6WCHeat6o+j1bpSLdgP3DLgHjUrD+rPra5MyWxIBAXDPPXDZZfDee6LYBYSCXV1xAinwp1T8KZhFo9G4twPpq+Dcz7JfNikJ3n4brrpKZG3fdBO8+ipc/ehMND0el709BQfRa2H9lXDMvowOlcqwQWdq41/auOvdG4KC7GpGwQIajYYuXbrIOq7ExAgpxyX7X+bF/UfQ6hy/tkulPhWsJ6wfXHEYEqbafKqjcp/SpMnfX3y5hP7vwKDFlo9r4RgdV05+DD+HCSldMzji86fX2+jxV3gUfgyCf+WpTFawgc4PwiWfWfR9NMeNPW7k3EPn+P4agyyzpA6waRNUV8PO1J0ABJf2NX1P+MWKKhwF9zHgfVG1YAa55ysqFUyZYvj5P/8Rm9CAQ/elQiND7QF9XoN2d1h1uLc3XHON+P7DD0GyhevVSyQiOsSu++EHf6gudfBCCgBTpk/nq4uCdouARGAT8BDwQc1rFnD45EkWvfGGODB9NWyfhaY8xey44rZN20PPw5FXXdyoe7i8Js/yr7/MqwA5ncQbYUoyRA53Yyfqo9MZAqJGq/0ATn8tv7+fRMx4aH2tRcUOR8kry2PuX3OdHgwbWJO7uWqVc66fni6CmX37xpocU7ac3wLA6DbOlzd1NgnBCfSJ7oNOr2PFiRXu7o5TGDxYvNoj91lSIoocwHyQ5/qfrufeFfdyvuC87Y00IuKD4rm669UAfLjrQzf3xj2MHw/33iuCxt26iX3R8nJR1ARir8Keij93xICUwJ+bcYexYz32zIM9c51yaR8fYYL5f/8HN9wggoHujnMqmMDDD4Z8A93m230JaePfmNynXi98euAimU+dFn6OhF3mN4gUrEOn05GTkyP7uPKf/0ChvhPbD3fk198cf2xID0iXBP6OvA5nllo+TsEhHJX7dJvvioJFjI4rIb0gfhpozFfsSFJqBw9CpY2FEYWFUFEhNvStui+8gqHdDFHJqeBaYidAm+kiS91OwnzDSAhOqJct3r278I4uLhZVozsuiBTdkNIBpjduy9KExLNCo8YZ85Vx42DyZHjoIbjiijofbJsBPwQpAZrmQpc5ED/Z6sOvu07cD3o9HDki3nNY5hOg1TBod5tYQyk4zKzZs1lbVcX2mp8XAS8BG4A/gZuAsTWv64CNej0vPfmkCP4VHYdTn6IrOGZyXKmudmNm/sjfYeDHLm7UPQwcKJ7bBQUGSUu34ARVK0c5cEDcg/7+hoTpelTmw9Zb4Oibzu2Is0zxasgpy+HdHe/y67Ff+fuUbfYRtjBmjJDiO3lSfMlNWhro9Xp8fPJNzlWkwN/g+MHyd8ANSFV/zdXXTfL527NHrDFtQUocCQkRex7GyC/P58d/f+T9Xe/jqXFugN0V3N1flLl9ffBriirs1EdtJqhUYp0BhmSDggKxv6FS2bav6Y4YkBL4czN6Jz94LdJ3YU3JvwvI3Ah/DYa0fywfq+B62tzokJ68ObnPY8eE1KeXl+GBC0B1MQR3BW+l9EsO9Ho958+fl31cCQmBmXfoCfbO5NcfskhLs/9a9mbG2N3YwWeFxI6CdeQfElnseQdsOs1RuU+XGyNnrIWd90LhCRc12HQxOq5EDILhP0H4ALPnJiaKDaDKSti1y7Z2pWq/sDAh+2gRv3hRdWZHxapC40K619RqGF6TrP/33/DF5KV0O/c2ISUDjfs+6vWwLAE23+i6zio0JGU5rJ8CJedMHuKM+YqnJ9x5J4wde9EHwd1EkEZjn5+1QtNGpRL3hbRhA0J9xGE63aOoBshIdHQ0L7/yCpO9vVkJPA78Bpiq3x4E/FZRweOPPkq6z3i4Jhd95BiT40p2tnhEeHlBaKjz/h1GCelmcb7UXPDwENJoACsds/51nPRVsvnTy8GaNeJ1xAgTgQO1N4z4FTre7ZwOlF6AX+Jg32POuX4NHcI6cM+AewB4edPLTmsnONhgH/Pnn/JeW6czrE2rqoyPKeXV5exOE5ImQxKGNPi8KXJl5ytp5deK+KB4d3fFKbRtKyq+Kyttv2esSVLefG4zevR0DOtIdIC1rrWNl5GJI+kc3pniymK+OfiNu7vjdsaMEXPKQ4dEYoC0pxkWJp591uKOGJAS+GvpRI9znQRCRQ6UnBGykgqNE12V8PuzA1Nyn3o9fFYTdxk+XBiA1+IVDOPWQQ/rvDoU3MfYQSf5+qoorujwWu3v0x6Kiw2VP+Hh8vTNLJMOCElHBesoz4Dj70LmOptPlap+N2wQCyZbcHngL3MTnHhfJB8oOA2VSvj+AnzyiZBJsRa3efEo2E7mJhFwS/7CoctsOreJK5ZewZOrn6x9b8wY8bp9O/yxeBBtM+8nyCuYgAAjF9BVQcd7IHaSQ/1QcJCSc5C6EkobicxR10dh9MpGWQGiYAenPodfO0DBEatPUamE5+P118Ntt2E8cUDB7cyZN48nFizgSpWKIZgO+kkMAkZ5efHpF9+BV6jZv3Gp+qxNGxcPBdWlwhO5BTFhgkjc+fdfOHPGjR05sxQOPAXl7vcrKyuDLaI4rGFyioSHr6hmdpZcuU+kSJLzcf4AOG/wPDzUHqw9s5bdqXbo/VuJJJm6fr1BhlEOsrOFF6OHB4SEGN+73JO2h0ptJZH+kbQLbSdf426kT3Qf0ual8fbEt93dFaegUol5AMB334l9KWuxZq9i47mNAAxv3Xgkhh1BpVJxV/+76B7ZnQg/pVAjIsKgGPH114ZK46agWKUE/hSE3GKVC0p3E6bCVWkQe5nz21KwnbS/4YcAOPeD3ZeoK/cpBXc2bxbSOl5ecPPNMvRTwS2o/BMpjprN0awh7NghStvtQcqMCQ42LZMgGyoVBLSD4CQnN9SMaDUUJp+ETvfbfOqAAcIIOTPToH1uLS4P/HV/CqamQEh3FzXYDElfA6vHQO4es4fdeKPQxM/ONvgrWYPNgb/CE7DlVkj9y/pGFOTBK1RsJFmQfrVEdmk2K06s4KsDX6Gr8WVr1w7uu09IxR+oKUSOijKxcavxgv5vQ4fZDvVDwUE6/AeuLxfPEwUFuVF7iurNahsySRCBiJtvNnj+OUTmRuGNnr3d8rEKNjFn7lx6tG/PLCuPv7WsjGXffAP5/0Kh8WCwTgcraiyrLnP1NsT5X+CnEPHaQggLM3hp/e08pUfLdH0MJh0ErzA3dkKwebOQFoyLg06dTBykq3ZuJ9SeMGE7JDnH5qcuCcEJ3ND9BgAWbl3otHa6dYP4eOG7tX69fNc1eIzrUZvYMZdkPockDKknU9+UUalUaByQ7W8KjB0rFGmKi+EHG7Y9rVmXbjgrvI1GJI5woIeNi/sG3seBuw5wTVc5Jk9Nn0k1uaUbNsD774vvrVExq9Ta6HkiM0rgr6VTngU/BsLeR93dEwV3E9Aeosc7JLvZpYvIhCgtFd6Oubnw+efis6uvNqJ9nLEODj4vPHkUZCEwMNA5F9Z4ETD2Y3L8pqLTwcaN9l0mO1u8Ol3mE6AsA8rSne5l0KzQ+EBge7vSoX18YMoU8f3339v23+7ywJ9KBX5xTje4by4YHVf0WrHpWWReLtXHR3huqVRCE3/nTuvatDnwV54BZ76CwqNWnqAgGyHd4LKdkHi9Q5eZ2GEiwd7BXCi6wKZzBumACROg2+3vkxX3ORUeWcTEONphBaei8bLK79Fp85W66PWw8x44rUgUNRva3gyXH4Tw/u7rQ9FJSP0DdPappCiYp6S0FGtrkqKAgsJC+LMv6v2PGx1Xdu8Wc4qAACGz6FL8W0Pb2yCoZSUhjh4tXrdvd+MyLKizSPBrBIGM1avFqyQVZ5Qdd8LPEa5JyHcB8wbPA+CHf3/gbP5Zp7ShUhmC+X/+Kd+9Zgj8mZ6rbEvZBsCQ+OYh81kXvV7PyhMrOVdgWrK9qaJWw4wZ4vvffjOsNy1haa+irKqMXanC12J4YvOo+APwUHs0m8C2HAwaBE88AR06GN6zRkXiYMZBXtn0CquTVzuvc2ZQAn9uRqNx80TEOwJiJ0JID+e3dfprsUhSaJwEtodRv4v7wU4kKR0fH5GZf+ed4iEZFgZXXWXkhLS/4eB/W5wEirPQaDS0b9/eqePKyJHidd06+86XAn+2GODazdE34JcYKDrugsaaEVXF4m+zLN3mUydPBj8/OHsWtm2z/jyXBv70elGtpiQcWIXJcSVqNFybb1Wwp2tXQ1D4nXcgJcVyu9JCzGpJtlZD4Poy4buk0CTx9vDmqiQxWfj24Le17+v1ehaf/D+2x9xB37GnuOEGExfI3gEbrxbSowruQ6+DjPVmq6FcMV8BoKoATnwAGWuc245Cy6L9DLi+FCKa34ZvYyA4KAhrZ6AZQHBwMPR6EVW7GUbHld9/F6/jx4O3t6xdtUzkcBi8BIK7uLhh99Krl/Bdzcy0bs7nNKpLoDjZjR0Q89lDh8Q+iSRfbpTAThA2ADydmBSTsU54uZdnOq+NGnpH92Zcu3Fo9Vre2v4WIDa/T+TI668+Zoy415KTxf+zHEiBv9hYtcm5ylfTvmLjjI1c392xpLfGyP1/3M/lSy/npY0vubsrTqFvX+H1W10NS5ZYFzC2lJC6/cJ2qnRVxAbG0jakrVxdbTRUait5YtUTHMw46O6uuJ0hQ+CNN+Cll+C66+CKKyyf0y+2H48Ne4yx7ca6JQakBP7cjM5WIyS5Ualg+M+u2Sjb+zAcfM757Si4lb594fXXxUOxoiYR9rbbRDCwAV0fgYl7hRyjgsPodDrS09OdN66c+oxJXsMJ8s7h2DFhamsrktSnSyr+Wg0VkmP+zW/y5VTS/oK1E+DC7zaf6u9vmPxYW/VXWQl5eeJ7l/i5VWTDmrFw4L8uaKzpY3JcUXvYVDF5yy3QurX4XT/8sGFxfuECrFxpCP6CWLwfrFlXtG5tZQMqtahYVao43cOpz+DIGw5f5sbuNwLw4+EfqdJWAXCu4ByZJZl4qD149q7etDM1ZSg5I+TUKtzv59OyUcG6y+Dg/5k8wunzFQnPYLgqC3o3z82rFoleDyc+hDPfWj7Wmag9xXNQQXamTJ/OV0YXjg350teXqdOnQ9LD6OKnNRhXUlNhzx6x5SFJdCk4Hx8f6NlTfL9jhxs78ntX2HitGzsAa9eK1549LSS+dnscRjs5ST5vn/ByLzjs3HZqeHzo48wbPI85l8wBYOWJlQz4ZAAF5fIlfQcGiqA+CEsBOaYVUuAvMtL0XMXX05dhrYfROtjahUrT4bpu1wHw2b7PuFB4wc29kR+VSlT9qVSwaZPw+zNHSUnde8L4MQXlBbQNacuIxBHNskJu/ur5vLz5Za7/6XpKKm2TWm+OqFTQvbvY3wgNte1cd8SAlMCfm9G3JAm6YT9CX+dpfCvIwPlfYMM0qMxz6DKJibBokZD5uOwyg9xHA7xCIbS3kIVScBi9Xk96errzxpXyDDxLjzC4j5j52KOlf/q0eHVJ4C/+Shj4oXJ/2UrkSOj7BkSNsuv0KVPEgv/UKSGvZInUVPHq4yNkmJyO2hP6vQ2JpsqGFOpidlwpOQdHF1mVTe3lBQsWCEnokhJ4+mm491646y744AN45BGRTFBdLZ4fWq3IqKsrpWGWyjzI3iYkzBVcz6nFcOQ1hy8zuu1oIv0jySnLYVXyKgB2pgp92J5RPfHxMLMZnHgd3FgNcZMd7oeCA6hU0P99SJpn8hCnz1fq9sUnAnxcpSOt4HRUKjj4LBx/xz3t6/WQ/AXkK1nvzmLW7NmsrarCkoPidmBdZSUzZwlHQGlc0en0pKXB0aPwbU18uH9/GxQE5GTbzBabaNahQzonjj/P3DuT6BQXx4CkJF54/nnS021XFLGbjv+BNje6rr2L0OthTU3BudlqP1fR9haYeh4iXaN5O7bdWF6/9HUSghMA2HhuIwUVBXx94GtZ25k+XSSfJicLWwFHqevx55K5SiNjROIIhrceTqW2kte3vO7u7jiFdu1gdo0l+NKlsGyZ6WMXLxY+knFxEBtr/JgpXaaQ/GAyn0/5XPa+NgYeHfoosYGxHMk+wgN/PODu7jQpTuedZvO5zVRUi6oYd4wnSuBPAXL3wqbrRem/M4kcLqS4FBovRSfhwm/i1UECA2HuXLGxazLppeRss9GxbxF0fQyuzqbbkG6AyGC05bmVnS2ybgEGDnRC/xTkwScCusyBQGsjLvUJCoKJNYrBH34I+fmGz3JyRDCwusa/PjsbXn5ZfN+unV3WgrbjFQKd74fosS5orJmTfwD2zLVaxjs4GF58EYYOFffAuXOg0Yj3c3PhySfFPXPmjLiP7rnHhnsiayv8PRhSV9j9z1FwgEGfwaVbHL6Mh9qDa7uK7PxvD4kd2x0XRLnAwFgrHhwqtVKF0xhoP6NxjLHlWaKyobrM3T1RkJMRy2DwV+5puyILtt0Ox952T/stgOjoaF5+5RUme3ubDP5tByZ7e/Pyq68SHR0NJz9G/UdPKEnluedU3HmnSCiSrAkuv9xFna+LXg+pv5uVPW6uLFq4kJm3xxN7cgELco/yQWoqDx09ysYFC0iMj2fRG44rBFhFtych6WHXtGWEkydFUpuXl0hmM0lxMmy/0/lS5d7h4Bcv5kpuYEL7CQB8tPsjWTe/g4OplYH/8ksoLXXselLgz1SywGubX+P+lfezO9WKDNcmylMjngLE7+p03mk398Y5TJ4sKrYAPv0UHn1U7F/OnQv//COG8N27RTBZpYIHHxTrVnOYTVBswrTyb8U3V32DWqXms32f8eX+L93dpSbDkn1LGPb5MGb/NtttfVACfwqgq4RzP0K+TKLYRtvQii+Fxk3Hu+G6Yggf4Jr2/hwA6xTdlSZDzSJh8GDhkZGaKhY01vL332IC1aOHyJhyKqUp8PcQ4S2qYB/VJVBonz/itddCTIzQw3/2WSgrgy1bRCLAs8/CzJkiC/uxx4TUY0SEmEy7BOVZJB+RI2D0X9D2VqtP8fISv/cHHoCHHoKvv4Z334X4eBEI/usvcdzdd4uFvNUEd4E+r0FYf5v+CQoyEdwFAuSRVZ7eYzpDE4Yyuo2QC6gN/MVZCPzl7RNJbLpqWfqh0Aw4/xOs6AbZjgelFRoREZcIb3J34BEgAo/t3beB0xKYM28eTyxYwAiNhgk+PnwDrAK+BkaqfRiu1nD7/QuYdPlcysoAvQ69TsuyH3zYs0eFh4fYtO/UCa68UlhRuByVCqali/ulBbFo4UJemj+fDVota3Tl3ASMBW4C/iovZ4NWy0tPPum64J8bkdRxLrnEhO2JRP5BOPWJa/wIS847d9/PDLf0ugVfD18OZh5kW4oNZvBWcMUVYn+hoEDYTdhLSQkU1eSlmwr8LT20lHd3vktynnv9I53J+HbjGRg3kLLqMrq9341n1z1bK8FvCqmiqSlx7bVwzTXi+yNH4MQJ8fX22/DMM2KNCiJImJRk/BpFFUVoW8D+wqg2o3h6xNMA3L7sdl7d/KrZAH5GcQYbz250VfcaLZvOi4SOoQlD3dYHlb6l1S03EgoLCwkODiYvL4+QkBD3dkZXDboK8PB3XhuZG2D1aOj/HnS8y3ntKDQd9HrYPx/84qDTve7uTbNAp9ORkpJCfHw8arWT8jpS/wRtOa99N5UNG2DkSOHXZQmtFmbNEhv7jzwCI5ytMJKzSwSVuz8tqrsUbENXBb+2F/JoE3baVYqXliZ+1wUFYuEkZU96eBgq/kAECF980UXyr3odLEuAuCtg4EcuaLDp45JxBVEN+vjj4j4ZPlxkXSo0IXTVUJ4OniHgKZ9mr1anJeClAMqryzl490G6R3Y3ffDmG+Hsd3B9hSLx7G6Ovy/kGMeuhZBuDT521bhC9g6hZNHhTvBPcF47Cq5Frxd+vZ6BwttVodmSnp7Op4sXs2zpUgoKCvD2DqbKczoJrWfh4yN25T08oFcvKCnRs3dvOSEhPjz/vMrkJq2C80hPTycxPp4NWi2DzBy3HRih0XA2JUVUbDqL3L2w71HodB/ET3FeO0bQ6eD224W39dNPW1C7kcY0jY8Y15zJ8jZirjZpn3PbMcGM5TNYsm8Jt/W6jSVTl8h67Z074bnnxJjw2ms22AXUITlZJKMGB8OXXzacq+SU5hD5eiQ6vY6UOSnEBTk7m9l9nMo9xe3Lb2fTuU1cEn8JW+7YYtS/bl/6Ph7++2FWn17NN1d9w/Qe093QW/vR60XQLy9PJKieOSMSlKtq4pzR0fDOO6aD94/98xgf7f6I50c/z/2Dmve+k1an5d6V9/LRbrGPMm/wPF6/VMjBrjuzjvMF57ml1y0UVRQx6otRHMo8xI/X/siVna90Y6/dR5W2ipBXQiitKuXQ3YfoFtmN/Px8QkNDKSgoICgoyCX9ULRw3IxTF7tWd8LD+bJIHoEQP9Vu6TgFF1JwWHy1vsa57ahU0HuBc9toYajValq3drLB9K57Qe3NtGlT2bhRZDJecYXw7TLH7t0i6BcYKCoGnU54f7g6UwR6FGxH7SkqgL1CxP+hyoKuhRFiYkSm3BNPGIJ+V10FN94oqv9WrABPT1H9Zaspst1UFUBIT/Bxh9FL08TiuKLXCa8/31iHgi3h4WKRvns3DBtm92UU3MXZ72HrzTD8Z0i4SrbLFlcWM7btWNadWUdShIVd3PazodVwJejXGPAOh6BOoDeeHe6S+QpAxEDxpdC8OPE+7LoPxqyGaBcbZ2krQOPt2jZbMNHR0cx/6inmPyVk5/R6eP99OHZMbMqWlgqpcOEprSI01JfnnjNdmeFSik9D4TFRoeoV4u7euITFn3zCaE9PBmnNV78MAkZ5efHp4sW1v1unoFJD9laxD+ViDh4UgYSAACsqTlUq8HFFBiTQ5WG3jmH/6fcfluxbwvf/fs+iCYsI9ZVvEdi/v5BU3bIFXn8d3nzTQqWlEVJSxGt0tPG5yreHvkWn19E3pm+zDvoBtA9rz4bbN/DT4Z9oG9q2NuiXUZzB9P9NJ8w3jEptJb8d+w09op7o1c2vNrnAn0oFXbsafh4wQNxH77wjAsEPPWT+Plp1ehUFFQWE+4U7va/uRqPW8MHlH9AzqiePr3q89nedXpzOdT9eR1ZpFjsu7OBE7gn2pO0hwi+idv1WWFFIaVUp0QEtZx9mb/peSqtKCfMNI6mV+H9wRwxICfy5GZ2ukWxKF56A3N2QeJ1zNL/D+ojNIIXGz/6n4MJyiC0CDz9390bBBlySQd//PfAKpUMEjBsn9M8//hgWLjRfFPbnn+J13DgR7HEZbvIwaBZ0e8LhS3TsKLJcf/lF+KsMqFERHjPGTSb3XqEw2jo/OgWBxXHl0Itw8L9w2S4I6+dQWyEhMNZeW7DUP0UVeZ/XXL8RrAChvaDT/eCfKOtlg7yDCPMN48UxL6JRW0hAiB6j/O4bC4nXiy8TuKziT6F5EtZPBPq93bDJtn4yFJ2AK5NdZEysUBeVSsjGS+j1YqN++zYdvmmf0rqjms6dZ9AoHG1Slgkf5HEbIHK4u3vjEpYvXcpD5eVWHXtrWRlvLl3q3MBfSE+4tsgtf6uSzOewYaICzSzZO8R45goJ4873Ob8NMwyKG0TPqJ4cyDjAVwe+4oFBD8h2bZUK7rsPjh4VVhKfflp/vLCGf/4Rr126GJ+rfLH/CwBu63WbbP1uzKhUKq7tdm299z7c9SFrTq+p997EDhNJDE7kmVHPuLJ7TiMuDl5+WagUmfv7zS7NZm/aXgDGtG0ZaxCVSsU9A+7hhu43EOYbBkCkfyR397+b5zY8x7s7hT6qn6cfK6avoGN4R8qry5ny3RTKq8vZNGOT5TVdM2HTOYPMp7pmX9IdMSAl8OdmGo3S6vG34fi70GqoIoXT0un8ALS9xfkBk8wN8O/L0H2+uO8UHEav15Obm0ucMw30Yi+r/fbWW2HTJqGDvnq1COpJpKaKgGBhoZBw3LVLvD9hgvO6Vo/z/wO1l5B0VHCMyjwh/ekTadfpvXqJL4WmicVxJWokVD4IHq6RqjCJrgIqc0Gv+Lu5hZDu0P9t2S+rUqn4cppiIN/ccMl8BWDbHVBdDMN+cG47Cq4l4hLx5Q5Ce4NvjBL0aySoVJCQAAnxKvQ/PUxxRWf0+tvd3S1BzAQY8CEEN5Q7bq4UFBZibS1HFFBQUODM7rjt77SyUlSdgbDFsMjm68Ez2G3ym65EpVIxb/A89qTt4bIOl1k+wUYCA2HuXJF4+uefogpwkDnd2TokJ8O+faBWC2/Qi+cqh7MOsyt1Fx5qD27sfqPsfW8qTOkyhfZh7ckryxPKHO3GWvbhbqJYCtqvPb0WPXq6R3ZvUZVsQG3QD0CtUvN/o/+P/rH9ufmXmympLOGHa36ovS8ySzLZk7aHwopC3tz2JvOGzHNXt13KxnPC43B4a0PyjztiQErgT0HQ5hZoNQy8gp1z/eQvIXcX9Fogq/eLghOIGuWadkrOQ+ZaqH7INe0pyIeuGqqLCAkJ5cYb4bPP4IsvRICvRw/Ys0fIa5SUiMNPnBCvPXqI7CmXsP9JkQasBP4co+gkrOwFcZc3j43T7XdCxCBoP9PdPWk+RI4QX+4mforL/VsUGhm/d4HIUTDwQ3f3REGvg8OvCgngdre6rx+lKVBV6L72FZoffV51dw8UjKFSoRv2Cynniujk7r5IBHcVXy2I4KAg0lNTrTo2AwgOdtLeU11ydkHePugwy/lt1bB7t1gHR0RAN2vivt2eBLWL5DcLDsPWW4X3bYc7XdPmRdza61Zu7eW8uUGvXjBtGvzvf/D220Ie2Jpb7ecagbJhwyAyEi5WrP1yv0hEm9RxEq38XSTN2gjpHd2b3tG9zR6j1+uN+gE2N1YlrwJgXNtxFo5sGUzuPJlTD5yiqKKItqFta99vHdyaNy59g1m/zeKptU8xufNkOoU3mqe1U9Dr9bUVf8Nau9fHpBFoICg0CiIGCkkeTydl7Kf9CcffEb5RCk2D6lLnXr/tTXBdKUQrD8kmRWU+/BgIe0SWzuTJIpiXnw9PPQUzZghT7ZIS4a/xxBNw551www1wvyu9ji9ZAgPec2GDzZSA9tBmutsWhrJSmQenl0DmRnf3REGhebJ/Pqx3k3m7TgueIaBRJMobBSo1HH5ZjLnuZMzfMGGbe/ug4Bx23gu757i7FwqNiciRVHjLKzetYBtTpk/nKytN1b709WXqdBd4gZ14H3bMFusAF/FHjavAiBFWFh12mO26JBnvCJEUU13imvYsoNPrWLhlIUUVRbJe95ZboE0boT70/vsiH9gcmZlCyQjg6quNHxMdEE3r4NYtRubTHv448QdjvhjD4j2L3d0Vl7DqdE3gr52ypykR4RdRL+gncUefO7i0/aWUV5dzx/I70OrMe8E2dfTo+e3G33ht/Gv0jbFk9OpcVPpGozXZsigsLCQ4OJi8vDxCQkLc3R0Dep1zJB61FVCRDX7N2wC32fD3UKgugkkH3N0TBRvQ6XRkZmYSGRnpXM+cLTdDxBDodA8gJso//ggbNxqq/C67DP7zHys8DRSaFqUXRFJAUEd398Q+qktF9Ydvy5LicASrxpW9j0HhURi53LWdq0vBEcjeCrEThQybguvZdD1kb4MrTyqJXgqQdwB8osA3qsFHLpuvKDRfVvYElSdM3O26NvfPFwlwfReBxst17SpYhU6nIyv1JK2iElB7+rq7O7C8DUSOhMFfuLsnLiM9PZ3E+Hg2aLWYU1fcDozQaDibkkJ0tJPn5Lm7oSJHqFNorAtKOsKZMyLZVaWCTz6BqIaPQIU63LPiHj7Y9QFXdLqCZdcvk9X7KzlZyH5qtTBvHowaZfisshJWrYL0dOjQAfbuFT/37g3PPy+OMTZX0el16PX6FuNRZisLtyzk4X8eZkDsAHbM3uHu7jiV5Lxk2r/dHg+1B7mP5hLoHejuLjV6zhWco/v73SmqLOL18a+3GMnPuuTn5xMaGkpBQQFBQa6xSlFWWm6m0Sx29Xr4tSOsd5JMlsZbCfo1JSIugfBBllOjHCF9DaSvdt71WyBqtZro6GjnjytDvq4N+oGQwrj3XvjyS5g/H/77X/Gz24J+2nIhR6ogL1XFoprnZBOW0fPwU4J+NmLVuFJ8CgoOuffvLmMNbJ8pApAK7mHIUph6Vgn6KQhCexoN+oGL5iuVBXDqM8g/5Lw2FNzHhJ2uDfoBZK4XHuVK0K9Roj71MVEbOqPO2erurghv7MBO4NOy5pzR0dG8/MorTPb2ZruJY7YDk729efnVV50f9AMI6wcxl7ok6Afwyy/iddgwK4N+x94WiQyFx5zar8bK7b1vx8fDh9+P/878NfONHpNWlEZuWa7N127XDm6sseL78EPYsAEOHRLef3feCR98IH5fr70mgn4AV11lON/YXEWtUitBPzPc0usWPNWe7Ezdyd60ve7ujlPx9fDl2ZHPcmffO5Wgn5W0Dm7NwksXAvDkmic5kNHyCk3cEQNqJFGnlov2YuFod6FSQWgvCHRSFUfubiEpoNA06LsQBn3iXEPsfY/Bttudd/0WiFar5dSpU24bV7y84JJLYMAAtzRv4OTH8L0vZG12c0eaGR7+4BMJAe3c3RP7OP+/FruodgSrxpWh38OVp0DtxhLfuCtg1EoI6em+PrR03LkRUpwMh1+B/H/d1weF+mgroOiUeL34I1fMV4pPiWSAFDdWIis4D42L/LDqMm4jjFvn+nYVrEIb1J3CiKvRergmg94sak8hNdznFXf3xOXMmTePJxYsYIRGwwQfH74BfgCuASJRMRYIDA6mpKSE9PR013RKr4fqMqc3k50N69eL76dNs/IkvVYkV3pHOK1fDSg4DAefh+IzrmvTBAPjBvLZlZ8B8MrmV/hq/1cNjvl498fELIxh5vKZHMw4aNP1r7kGOnYUqkSvvSZsSN57D3JyhAfjZZeJzzUa6NlTVPxJSHOV8spyViev/n/27js+qir94/jnzqSSMiG0UEKXDqIg1QKKILr2rlvsruu6P9u6uva1oq66rn1tuGvvDVRExQKCgEqk955QkkxCQtrM/f1xJRBJIOXOnJnJ9/165ZXMnTvnPocJT+7c555zKK/a+3xGamqb0pZT+zlzpd7zzT2Gowmt9mntuXXMrTx2nJaXaYiLDr6I43sdzwGZB2ARm+tAllaWcsF7F/Dqz6/uNaWpiWu1murTkF1Tfebn59OyZUvT4YSWbcNridD+GDjifdPRSKTI/cy5G7tzHZOoS4MFAgFycnIYOHAgXm8IL8AWLXcKt13Ogi5nhO44jbX+HVj5LAx/ts7RBtLMBMrh9VTn79CYD0xHE1XCllck+lWVwqaPILkDtBkd3mOvfwe+PgVG/s9ZQ1jM+/kuWHATHDPXGXGxh7DklYpC2DoT0npAeu/QHEPM2ZkH22dD5lBo0cF0NBIBdL4SWXJzc3n2mWd4+t//ZuOWLRyKxcXYZAG5wItJSXxZWcm9993HVVdfHbpAqkrg7SzIPiXk066+8AK89RYMGAD3RHLNY82rMPPsiDpnunH6jdz9zd14LA+tW7Tm2AOO5fkTnwecKRV7PtITG+fS9aVDLuXfE/9NvLd+M0xs3QovveRM61lY6Gw77jin6Bf/SxOBAHg8Ne9735VT8tPzOeq/R9Etoxsr/7ISK5Q3x8eAnLwcBj05CAuLnMty6N+2v+mQJMIU7CwgKS6J5EiYljsEpiyfwnEvH0d2ejZrr1xbI2cUFBSQmZkZ1qk+tfqShJ4dgEF3QWpX05FIfQWrYN6VzvSs/W8IzTGytABu1IprARvfi9yRNdknO18iexr9MiS0Mh1FbAqUwYb3IakNtBtrOhoxJVgB35wB3X4f/sJfuyNh/GxI3XsxeTGk7WHQ91pIMHSDY0IGdDzWzLEl9LbMgG/PhFGvQNezQnusYBX88FfocqazHILI/mx4H/LnQ++/QGKm6WiMyMrKokVyMjv9fr4FhlNzvMG5ZWXOtJ9//ztA6Ip/cSmQdTRkDAxN+78oLYWpU52f95wuMiK1Hw8T5oBvgOlIqt1x5B2sLlzNKz+/wpaSLeyo2FH9XPeW3fn2gm956LuHeHPRmzw17ynWFK7hjdPfqNcUi23awJVX7nuffd0r8PHKjwEY3Xm0in71MLDdQE7teypvLX6LO7++k1dOfcV0SK6buX4mG4o2MLHnRE3z2Qgtk2t+NsjJy2Fgu9Dm6HCautz5YzCx58SIyBma6lN28y9yij3bXF6E1RMH/f4KnU93t10JHU+cU9jZNNV0JBKJkjvA6cUw6DbTkYgJcy6Db0J8kc1t3kTnb1DWkaYjiU22Dd+eBcufMBfDovucKX4LfjQXQ3OXkAGjX4V+fzNwbB+0HuYUnyUytD0cDrrf3PTQdtDMcSU8Wo+E4c+FpxC3fTYsfRg2fRz6Y0mTtM97GGve/5kOw5le/ufbQ7tsRoTLzc3l+r/9jQ/Kyxlexz7DgQ/Ky7n+uutCO+3n4W87N6KE0JQpTvGvUycYOrSeLyrfDgtucf/62/4kZkKrQyAuckbbeCwPL5/6MuuuXEfOZTnce9S9NZ4fmT2S109/nffOeo8W8S34ZOUnHPr8ofy8JfTr+H68wsn9E3tODPmxYsVNh98EwGs/v8ay7csMR+O+h797mDPfPDPmpzMNh8e/f5xBTw7iX9/9y3Qorpm64pfC3wGRkTNU+DMsEqq/1cq2wtJ/wbaZpiORSHDMDzBuRmjaLsyBN1rC0kdD034zZVkW2dnZoc8rluWM+otEJevgq5Ng4xTTkcSu0vVQvNy5Az5aBCNkPd0oVK+8EpcMh70NA28PX2C/ltIV2h0F8RGwtk9z1uVM8PUL/3Eri8Kyfo+4IyznKz9cB6+nQcn60B1DzEnJhh7nh2dGmTaj4biF0POS0B9LGs2yLDKrFmLlfmI6FDjkMZj4k7kRzxHgmf/8h7Hx8XUW/XYZDoxJSODZZ54JR1ghUVoKb7/t/HzGGQ2o9xb+DD/fAdsMrEsfqICCBc7NexEk25fNgLYD6JHZo9bnj+99PDPOm0HblLYsyFvApuJNIYvFsiy8Lb3kbMnBwmJCjwkhO1asGZw1mGtHXsurp71Kj5a1v5fRqqyqrLqwc3IfzTLVVGsK1wBw5SdXcvfXd5sNxgXLty9nZcFK4j3xHNXtqL2eN1EDUuHPMI8ngt6CVsPghNXQ6wp3282dDh8Pc75L9EhqHbq7FG0bMg/S+msu83g8tGrVKjx5pXQTrJoceRfUipfBxg+hbLPpSGLXYW/BxHnOyOBoMeu38G4X50OuNEi980r2SeDrG5aYatXlDBjzobnRRbJboDz8NwbMuwpeb+GsHSyRwQ46N+L8+Pe9ngrL+UraAdB6VLOdZq/ZCNdFa18/rSUY4TweD/ETZmAdHwGjS+JSoGWELokQJu+9/DK/Kyur176/37mTd19+OXTBbPkKZv4eipaGpPn334fiYsjOhiOOaMALW49wCsRdDMykMu8vMPVA2Bl9n5mHdhjK9xd/z99G/42jux9dvb2sqn6/b/Xl8Xj4btt3AAzvNJxWLbRkREPcP/5+zuh/Bl5PbK25+tmqz9hRsYNO6Z0Y2qG+w3ulLpPGTeK2I24D4MbPb+Sur+4yG1AT7SoKH9r50FqngTVRA4qgqlPzFAhE0CiEuGTnrkm3E3NlkXMR3o6gvsr+Ve2EzZ+GZtq0loPgqM81/avLAoEAS5YsCU9e2TYLvjsP8iKsoJ81Ds4ogS7nmI4kdnkTTUfQcKndIWMAeBNMRxJ1GpRXqnY66/1J87X8SacAt312eI/bZhR0/a1GfEYSywP586B47wutYTlfOeBSOPIT5wK8xKbPx8PHIb7olvclbJ4WcaNiZG+BQIAlK9YRCBqe5reqFAp+avaj0P1FRWTVsj0XuBM4BOj1y/fPgO35+aELpnQDrPmvs7SNy3bsgHffdX4+5xxo0DVdb6JzXSS5vetx7VenE6H/Tc7f6ijU2deZe8fdWz165sfcH+n9aG8+WPqBa8cIBAK89sNrABzbU2sGi+Ptxc7w3pN6nxRZM/hFKcuyuHXMrdxzlDNt6k1f3MSkbyYZjqrxqqf5rGNqYBM1oOjM8hI6O/Ngq8tTfWafDCetdxYRluhRkQ9fTHAu4knUKKvnnZVN1vZwOPQNaB8Z81bX4E2MqDULYlLelzD3iuiZQvPAu2DMR6ajiFr1yivr3oDXU2CToWl2N34I86911ksRc9L7QKeTwZsU3uP2uBBG/bdZr6cUkU5c54wSr0XYzlckdqV2g7SeoT3GjzfAVydCRUFojyOuqCzZAps/hpK15oLInwdTB8Pyx83FEAF86en8etW+h4AuwDfAlcBdQFdgCrAlL48e7dpx5x13uL/eX6eT4HS/c13KZe+8AyUl0LUrjB7dwBcXr4CqEtdjqpcOE+HAOyC5tvJs9Hn+h+dZ51/H2W+dzQ+bf3ClzYpABbNyZwGRs1ZXtNlWuo07v7qT8949z3QorqgKVvH+0vcBOLmvpvl00/WHXs8dY+9wfp5+fVSu+WfbNqkJqUBk5QwV/qSmeX+BaaOb/R1qAiR3gEOegAP+6H7ba1+DnNudOyIlOiW1gc6nRd50revfdtZLkNDK/QyWPQqFP5mORCJFel8nJyQYmgZnywxY8k+oLDZzfHG0GwOHvQmZQ0xHIpHAZCH2h7/C8qfMHV9Cb9hTcOhroT3G6Jdh5GRNGRslkncuxvvVb2CDe6N+Gh5Eexhwi/P3sBk78Zxz+G/S7puAHgLuAb4CPga2AL8FioFbgNMB75YtPHDLLXRp356s1FR6tm/PIX37Nr0YGNciJDMCLF/uTPMJzmi/Bv/Jm34UTDvU9biaowfGP8DR3Y+mpLKE37zyGzYUbWhym/GeeJ477DkmHTWJg9sf7EKUzU9FoIJbvriFyT9NZp1/nelwmmzq8qls37mdzORMDu9yuOlwYs5Nh9/ErUfcChCVvy+WZfHG6W/w7QXf0r9Nf9PhVFPhT2rq+ls4+CF3p+Xc9AmsfslZ60Oih2U5Rb+Wg91ve92bTuHPE+9+2xJe5fmR8387UA7fnAELbjYdSew74DI4fjlkRsGHoG3fOaMTQzC9j+whYwAc+jq0a8jiJi7qfyOcuBZadDJzfDHru/Nh0X2mo5BfK9sKq14Mf/61bVjyMGzSSG9potRuWpogiuxM6kVw6BPQ/uj97xwqaT1h0O3N/gaYiy6+mC8qK5mNM73n9cAHwHBqFgEnAFcDm395rhQ4HPhnSQn35ObSdckS/n3rrXRv355WKSl0a9OGrm3b0qlNG7q1aVP/4mDRMsj93LX+LV4MN90EZWUwaBCMGNHABmwbelwE3S9wLaYGm3sFTB9n7vguivfG88bpb9CvTT82FW/iNy//huLypt0MaFkWvX29uWbkNXiidEpU0zqkdagukL2+8HXD0TTd7I3OUgYXHnQhcZ44w9HEpluPuJW5F8/lnxP+aTqURhuVPSqipoFV9jLMxMKO+9TpeOhzJcSnutfm0odgzsVRO394sxcMQKDC3TaHPQXH5qjw5zKPx0P37t3Dl1dy7oC3WkHx8vAcrz4OfQN6/8V0FLGvRcfQT6/llq3fOqMTNU1Xo4Q9rzRWQgakdAZ9CDNv0X3w/Z/CdzzbhjWvOP/XJbIUL4fv/gAb3q+xOSx55ZQ8GPZ06NoX84qWOVNxbv/e/bbLtsC6tyBY5X7bEhIej4cuvYZgHXAppPc2HU6zl5WVxb2TJnF8YiK3AmNxCnt7FgFnUrMA+AnwNTCNmiMCx9s2VcDBpaUcum0bm7Zupe+2bdy5bRtP5eZy5ZIlfH333XTp1ImHHnyw9oDmXApfneTKep3z58Mtt0BpKQwYADff3IjRfpYFA2+G3lc0OZ5GK8+H8m0xk+d8ST4+Oucj2qa05ae8nzjrrbOoakLfouYzUIQ7s/+ZALy2MMQj9MPgziPvZNGfFnH1yKtNhxKzLMtiSIfdN84EggHsCF9n2bZtJn0zibwdefvd10Q+sexI/xeMUUVFRfh8Pvx+P+np7k87EFHyf4CyXGcecYkumz91TpCHPQ3dfms6Gok0m6Y6F1sH3KgP2M1RsMpZz82TCB0mmI6mbnYQSjdAUtvwrzvW3Cx/AjZPc9b0CvddbmVboGoHtOgCHm94jy01ffkbZ+rV0/3hu+nLDjqjvrW+a2SpKoFNH0PrEc4NIyJuyvsSpo91Zqvpc2XT2wsGdv/9WP82fH0qHPZ2SNYFkxCzbXNTDU8/CtJ6wbAnzBw/wjz04IPccs01PAmcC9yJs8bfCzjr/X31y/ddP+85InDP4uCvfx5ey7FmA8cnJnLD3Xdz1dW/ujC//l2oLIKu5zTqJjHbhjlz4K23nNF+AIMHO6P+EhMb3FxkMPn/JITmbJzDES8cQVlVGQ+Of5CrRl7V4DbyduRx3WfXMaHHBM4ZeE4Iomw+tpRsof0/2xO0g6y4YgU9MnuYDkmixHr/es59+1yGdxzO3UfdTbw3MgeOvLHwDc548wyyUrNY/X+rSYqr+5qTiVqQbl1ogscee4yuXbuSlJTE8OHDmTNnToPbCARcnFLTDcFK+HS0c0eUWzIPUtEvWqV2hzajnZEUbrFtKFqqdZhCIBAIkJOTE7680mEijHoxcop+bo9MlX0L7ISZ50DOraYj2TfL44wEU9GvURqUVwoWOMXg8m2hD+zXFtwM7/eASn/4jy01jXwRTisI70wPlkdFv0gUlwKdT92r6Bfy85WKAme936qS0LQvkaHVMPjNksavRx4odwoB4ExL+0572DrTedzpJGg/ETr+xpVQJfR25ZXgzPPgXUPTfttBqCjc/XslXHX11bTLyiLrl8fvAb8DnmH3KMA9f95zRGCXOn6urejHL9s/KC/n+uuu23vaz+yToPvvG1z0q6iATz+Fyy6DO+90in5xcXDssc5Iv0YX/ZY8DJ9PgJ1NWLuwqWKw6AcwrOMw/nfy/xjbdSyXDLmkUW1MWzWNF396kTum3xF512yjTNuUthzZ7UgAbv1y93WDz1Z9xm/f/i07KnaYCq3e8nbksd6/3nQYzc77S9/n63Vf88CsBxj57EgWb13savtVwaomjyasCFRw/fTrAfjjkD/us+gHZmpAKvw10muvvcbVV1/Nrbfeyvz58znwwAOZMGECW7ZsMR1a03jiIVjh7ppdQf2hjFppPeHIae5+6K0sgg/7wDxNxxgKzfrE9KsT4P2erkzhIvUQnwajXnGmV41Utu2s51EW5X+bDat3XjnofjijBJLahDag2mSNh75/Ba+KP8YlZoZ3ytXyfGfkz879T68ihlQUOjcX7iGk5yu5n8GUgbDxw9AdQ8yLa+HcfNaYG3tsG767AKYdCmXbYOdGZ43Y9W85z1seGPORliWIMoFAAJLbg28ABMrCH4DlgYnzYPRL4T92BGuZkcGu8pYfyGJ3AZBf/VxXQXDPn/dlODAmIYFnn3mm9h1q+Zxo285afRs3wvffw3vvwRNPOIW988+Hf//beS4lBU4/HZ591ikEJiTUr/+1Kt0A22a6e4N1QwUqYOWzzmjIGHNqv1P57PefkZKQUr2tIRf4p62aBsCINg1dvFFqc0a/MwB4KeclbNumrKqM37/ze17KeYnhzwyvV1Etb0cez//wPH/99K+c8MoJvPrzqzWe3/P9tW2bFfkrCLh0LXrSt5Po/kh37vtW64mH0+XDLue1016jZVJL5m2ex8FPH8zcTXMb1Ma3677l2k+v5d5v7uXFn16s/j2xbZvz3zufP0/5c5OmBH5q7lOsKlhF+9T2XDvq2ka3E0paCKWRHnzwQS6++GLOP/98AJ588kk++ugjnnvuOa6//vq99i8vL6e8vLz6cVGRcxdYIBCo/uBrWRYej4dgMFgjae3a/usPyHVt93g8WJZV63aAYDC47+1HfweAF+c/w6/393q9e8VY13YrUIrnTR92z0sJDnnUXJ/2iLHJfYqU9yla+xSowup7PXbLg/H8cqyo71M9toerT7Zt75VTQtqnpY/Axo8IHv5R9V2Dxt4n3wCCCa2w94gzUt+nevcp0n/3OhwX2X3auQnv50dBzz8SHPpY832fmtCnPfPKfvvkSQYb+KW9sPap40nQ8SRnO3sXFWL9fYqoPgUqYPsc8CZjtRoS+j5tn4P15USCQ5/E7nFRaPoUi+9TmPpkLXsEzw9XYx89i2DmIQDVeQVc+KxRW59SDsDT/2asloP1PsV6nyoKoXQdnszBDeuTx4OdMQCqSgl60yF9IIz/Hm8tMYa9T/uJPSrfpzD0adexAgP+gb1rDZ1AIKr7VNf2aOvT8WedxX/vvZdzy8rw4Yzq21UA5Fc/vwdcuZ+f9+f3O3dyw0MvsT3/7wSDEAxColXItQNHsjB/PK8s/xeBgLO9qsqipAQqa96bgmVZgI1tQ+vWNieeaHP00ZCa6vQpEGji+3TwAwQG3QN4qs+bw/4+2Raeef+H1WoEwY4nxNzvHjbVhZ9/fPUPkuOSuf6w6/fbJ9u2mbbyl8Jf2xER1adozRFnDzybKcunkJmcSXFZMSkJKbxyyiuc/fbZLNq6iJNfO5kZf5hRPVpqV592lO9g+urpTF4wmY+Wf1SjQHPB4AsI/JLjZ2+czZ8++hM3HHoD20q38fjcx1m4dSGjs0fz+qmv0y61XaP7tLF4I0/Pe5qqYBX92/Svfl0svk+R2KdT+5zKiA4j+O27v+XrdV9z25e38cHZH9SrT1tLtnL8K8dTUFZQva1jakfGdB3DnE1zeGnBS9g4ReLJJ06mTUqbBvWpuKKYf3z1DwBuPuxmkrxJ1b+TdfXJBBX+GqGiooJ58+Zxww03VG/zeDyMGzeOWbNm1fqae+65h9tvv32v7YsXLyYtLQ2AzMxMOnfuzIYNG8jPz6/eJysri6ysLNasWUNx8e7pEbOzs2nVqhXLly+nrGz3HW3du3cnPT2dRYsW1fhF6927NwkJCeTk5NSIYeDAgVRUVLB06dLqbV6vl4EDB1JcXMyqVauqtyclJdGnTx8KCgpYv373XRlpaWn06NGDLVu21JhWoXWah07Zp1Jgd2LdHseN5j7F4vu0zz7ZC9j508OsbfUXKuOz3OmTdSYUwsDsoN4nF/u0du1a8vPzWbhwIZZlhaVPFdsXEbdlJot//JKquNZm36eDH2BhTg6BPeKPxPcp5n73Wqez9YcXyI07BKz4iOqTN+CnfZebad1lrN6nRvapqqqqOq/06dNn331asoiU0h+wrQTK0w6K2D7F4vsUSX0q3LaBgUvGUJh+NOVDJ4e8T5UJXdmW9TeKC1pTnpOj9ynC+pRS4qNHl99SEfCw5Jfttm1TUOB8CA9dn/5EVnoWa1au1PsUw33qvvZyUkvnUXbidhISkxrWp+zLWVUxARYuqtmn/Hy9T1HYp5QUZ2TP1q1ba8zCFM4+pe6YTWLlegrSj6H/4JF6n37p06jRo7mrspLZwInAf6G6AMivfq6rILjnz/vTDtixw8+8ebv/DVJS0inqmcb6vGSWLdu9PS0tnWAwSEnJDhISbNq0qaRt2wAHH5xFenopgcB6unQpIy4ONmxw+X1au874+5TZ7XE69z08Zn/31q9fz/zt8/nHN/+gZWJLrhh+BcX5xfvs04qiFWzesZkkbxKDMwezdu1aSkp2Tx0eCX2Kxvdp8rGTWbVqFauWOv1ql9SOWRfO4uCnDmbe5nmc+/K53Dr4VtLT0+nRowef/vwpx797PFX27mLf0A5DGeAbQPv49mQUZ5CTk0NWVhb3z7yfH/N+5My3zqwRz7frv+XMV8/k38P/3eg+/WPpPyipLGFw5mA6lHQgR581jPTpmgOu4Zt13/DR8o/4Oe9nArm7+2nbNmWtyvhg6Qf0j+tP/5b9Abjth9soKCugR0YP+qb1ZVvZNqb+OJWOlR0Z2Wckk4+dzCWfXMKnqz6l8786c0znY/jb2L/RI6FHvfr03/X/ZVvpNrqldWNo3NDqf+d99ckEy27qhKbN0KZNm+jYsSMzZ85k5MiR1duvu+46ZsyYwezZs/d6TW0j/rKzs9m+fTs+nw+IoGr/zlysNS/iaXsodptDI6raH4t3MER8n1a/gD37IoKHfwjtJ8RGn2LxffJ4qKqqoqysjKSkJCzLCk+fKksIWok11gjQ+9TM+rT4XvjpRgJHTHGmW4yFPsXi+9TIPtm2XZ1XvF7vvvtUVYHnrTRodzTBw98Pa5+sRfdgbf0G+/AP8MTFN7v3KdL6ZK14EtvXH6vt4THTp/3Frj7Vv0+78squC/Wx0Kc9Y4yV9yka+mStfQVK1mL1vRq8SfXrU8EPeNMPwI5Pj8g+7TP2fWyP5PcpHH2ybZvKykoS2Im17CFs3yDIPiWsfbJmX4BnzYsETt6KJ6lVk/tU1/ZofJ8efvBBJt18My+Ul3MycB6wFvgYuBP45pefD8EZ2XfuPn7en5eAu7v04T8v/4zHAx4PxMd7cMIP4vVSvT0x0UNqKrRoESRxj4+0IX2fyrfi2fgugTZHQFqvvfaPhP9PDe5TBP/uVQWr6P9Ef1YWrOShCQ/xl2F/2WefHvruIf762V8Z33087572LvHx8Vh7XOuIhD7tb3s0vU+frviUiS9PJGgH+fMhf+bhCQ/j9XopKS+h1f2tyErN4qQ+J3HB4AsYlDWo1j4Vlhfy4MwHeXr+07RMaskfh/6RkZ1GctWnV/HCCS/QM7Nno/r00fKPOPG1E/FaXuZePJeBbQfWq0+x+D5FQp9Of/N03lnyDpcMuYTHJz5O0A7yr9n/4j/z/8Oy/GUAZCZnsvLPK/kx70fGvjgWgG/P/5bhHWtOFL0r9nmb5vGnqX+qMYXoq6e+yml9T9tnn3J35NL7sd6UVJbw+qmvc0rfU+rVp6KiIjIyMvD7/aSnpxMOKvw1QmMKf79WVFSEz+ejsLCwuvAXMYqWw4e9oM81cPADpqMR06pKnInv41PdaW/VC7D2VTjkCUjt5k6bAuyeLmvXH8lmZd1bkPc59L8RWnQwHU3zsmM1bHgfupwFye32v79ElQbnlZXPQ2p3aHdE6IPb06w/OGsznV5c40YEEYk8IT9fmXOZs37SmA/cb1uim23D+z2cvxPHr9DfixhSnVeCO7HeSIMuZ8Pol8MbRMl68C+CDhPCe9wo8dCDD3L9ddfRHcgLBCgBvgK6/PL1FTCN3UXAO+v4eX8mJCdz+N//zo033RSCXrhg01T48lg45Ek44FLT0TjrnHq8kNDSdCQh8/S8p7n0w0vplN6JlX9ZSYK37gUaJ740kY9XfMwDRz/AlcOvbJ7XVsLsvm/v42+f/Y3UhFQK/lZA3C9rhW8q3kT71PaN/ve3bbvRry2tLKX/4/1ZU7iGa0dey/3j729UO+KeuZvm8tXar7j44ItJjk/movcvYvJPkwFoEd+CykAl75z5DscecCyDnxrMgrwFXHzwxTx9/NP7bXvepnnc9fVdvLPkHQZnDWb+JfP3+buTvzOff8z4B/M3z2fGeTPq/Xvm9/vDXvgzM8FolGvdujVer5e8vLwa2/Py8sjKqu8EBI5fV9IjQloPOPobGHhL09va8hXMvQKKljW9LTEjLsW9oh9AyTrn98IT716bAjj5JCcnJ/x5JW8GrHsjvMfcK4bpsPxx/V6ZkNoN+vxfZBb9ZpwI08eZjiKqNTiv9Dg//EU/gJGT4YwduogbSWwbgoH979dUC26BKYOgfHvojyWNs+wx+Hw82E4eCfn5StlmKF2///2k+bEDMOBG6Pc3/b2IMdV5xZMMx+bA8P+EP4iUbBX99uGqq69m7YYN/Pa220ht25Yq4BickX/3AscDBwNfALOBi+r4eV9mA19WVHDhRRft/WRhDvzwN6c4a1LrETDmY+hwrNk4wLku83Yb5+bsGPaHA/9A+9T2bCjawP8W/K/O/WzbprCsEICjuh1l5tpKM/TXUX/lT0P/RO9WvVnnX1e9vUNahyYVXfd87VuL3uKi9y/aa9RaXe6YcQdrCteQnZ7NrWNubXQM4p6hHYZy9cirSYxL5Jy3zmHyT5PxWl7+dcy/yL0ml4qbKziu13FYlsVzJzzHxJ4TuXfcvfVqe0iHIbx48ovcdNhNfPrbT/f6vbNtm5s/v5nZG5y/QpnJmTx8zMN88YcvGvQ7aiKfqPDXCAkJCQwZMoTp06dXbwsGg0yfPr3GCMCoZXmgzWiId6H6vH0OLHtUF2OiXfEKWPu6O20NvAXOKIHkju60J+b9eB18f7lzkdeUoY/BSeshqY25GJq7qhLYudl0FDV5k8CbbDqK5se2qy/wSzO15Wt4uy2sfjEMB/M4+SfOxZuUxF3FK2H7bNi5KTzHO/xdOPbH8BxLzKrww2djYUE9L8p54qDHhdDzktDGJWZlDHBuXg2n8u1QujG8x4xCWVlZ3HjTTazLy2Pj5s2MnDCB0cC7wHjgJKArcBw1C4J7/lxX8W82cHxiIvfed1/tN+TvWA2L74Nts1ztU4MltHQKxCnZZuMA8PWH7uc732NYYlwiV4+8GoBJ304iUMeNaZZlMevCWWy+ZjMD2gwIZ4jNmmVZPHbcY8y9ZC7dW7q/Dto6/zrOfutsnv3hWf4zv343hfRu3ZsW8S341zH/IjVBnzEiybLty5iyfArxnnjeOP0N/jL8L6QlptXYZ0iHIUw5dwqZyZn1bjc1IZU7jryDNik1rylWBau48P0LufPrOznu5eMo2FlQ/ZzX421aZ8JAhb9Guvrqq/nPf/7D5MmTWbx4MZdddhklJSWcf/75pkNzR7AK/IudCylN0fv/4KSNkHmwO3GJGYvug2/PdKZNcoNl6S7bWHLgPXDYW2ZjsCxo0clsDM1Z2VZ4qy38eL3pSGo69DVN9RZuGz+Et1o507+G09ZZztRJEhladIK0A8Jz4XXQbXDCSvAmhv5Y0jgH3gWnFervtLgvPg2Kl0JF/v73DZTrZtTmoqoUCn5yvofL6hfh3U6Q90X4jhnlsrKymPLxx2zYvJnxd9zB0r596dSuHSWtW2OnpDAaeM+yGA8cBnwE1T+Pw1nL7zPgf8D4pCQO93q54e67uerqq2s/YLux8Jsl0O334ehe3SoKzd4wu6fEVjDiOWg/3nQkIXfpkEtpmdSSZduX8caifc9WlJWapek9Y0hnX2fuOeoeAP700Z847PnDuO3L21hVsKrO15w3+DxWXLGCk/ueHK4wpZ4GtB3AB2d/wPtnvx/S92dT8SamLJ/CMf87hud/fB6P5eG+o++jZXJ0TYscZzqAaHXmmWeydetWbrnlFnJzcxk8eDAff/wx7dpF4FRnjbHiKZj7Zxj7SdNOAjzxWm8rFvS4ELLGQbwL61GuecWZGrD1iKa3JZEh60izxy/bAvk/QOthMb02QURLagNdz9X/a4GkLOeu4XAXYXJuc+7gPqMovMeV2qV2g/EzTUchkSIujCOvK4ucWSpaD4eMgeE7rphheeDkeo4k3fgBzDwHRr8O2SeFNCwxbPnj8MNfYdzX0PbQ8Byz5WDofh5kHhKe48WQXaMAf70mX25uLs8+8wzvvvwynfLzWRIIsNKy6AAssW1usiy8Xi8tW7bkpHPO4cWLLtr30jvxaRDfO7SdqY8pAyGlGxz9lelImpW0xDSuGnEVszbMYkDbvUfzPTDzAU7ofQK9WvUyEJ2E2lUjr2LOpjm8vvB1vln3Dd+s+4ZH5zzKir+sICMpA3CmcywqL8KX5Fz3bJ/W3mDEsi9ju40NWdsbijbw27d/y4y1M6q3JcUl8dppr3FC7xNCdtxQsez6TnArrioqKsLn81FYWIjP50IxxW2FObD6f9D9D+Dr1/h2/IucadZSu7kXm0SvYABeS4SOx8Ph75iOJuZUL2pvYgFqOwiVfjOFtzUvw8xzYdTL0PXs8B9fItP2uZD7KXT9LaR0Nh1N1DKaVxoi9zNn5KlyQPOz+AGn2NxhoulIZF92rIYlD8JB92N7EkOXVwp+hKkHwcB/wMCb3W1bolvel7DoXhjxPCTrYl6sqXG+UjAfNnwA3X4HaT1MhyaRpKIQdqyEzCFmjm8HnSUyWnSEATftf/9wWPcmLHkIRr4Y8/9fgnYQj7X3xHfvLnmXk187mdSEVFb+ZSVtU9pGz2cgqTfbtllduJrpq6Zzzzf3sLpwNZPGTeK60dcB8OTcJ7n767t5+dSXObRzmG4akYhTVlVGn0f7sNa/lnYp7Tij/xlcMuSSWm8YaCi/309GRgZ+v5/0dBeWV6sHjfiT2mUMhIMmNb2dmb+Diu1w4pqmtyXmVe6A+KbMb23D4e+5M3JQalVRUUFSUlJ4DxqsctZyanMYHPFeeI8N0Go4DPk3tD0s/MeWyJU7DX66EdqNU+GviYzklYbKGmc6Avm13M9hzUtw4N2QHKIZMQLlzqiOLmep8BfpNrzrrPvd5lDofEbo8kpqdxgz1fkuzUPRMmeq5+xT9r1mVrsxzpfErOq8kjkkvIUd29YyFtFiziVOoeuMkvCORt/F8sCwJ8J/3H2p2gFFS5xlXWK88Fdb0e/TlZ9y4fsXAnDZ0Mtom9K2+rmo+Awk9WZZFt1bdqf7kO4kxyczdcVUJvSYAED+znxu/PxG8nfmM2/TPBX+mrGkuCRmXjiTNYVrGNZxGHGe6C6dacSfIbtG/OXn59OyZQxPTbfiGQiUQu+/mI5Emmr2Rc7IqtMKwZtgOhqpRSAQICcnh4EDB+L1hnmR2dkXOxfZ+t8Q3uNKZJl9MWz/Ho790XQkznRv/iXOjSwmPtjHiEbllTUvw7bvYOgjoQ1OItuyx2Hu5XDktNAVZoMBKPzRmV2iKTNUSOgFyp3peNuNMXu+IrFn9f9g1u/g0Deg82mmoxFDjOaVnH/AxvfhsHf2XXwW89a/A8XL4IA/OVN/inMuZXmaVfE6b0cet8+4nbmb5vL9pu8BGNphKN+c/w2Jcc5yBTpXaV7+b+r/8cicRxjQdgA/XPpD1Bd7JDIVFBSQmZmpEX8SIRZNglWTYeKPjS/09LzI1ZDEoJYHQ6DMmc7R26ZxbdhB56RSYs/w/5g7djAAHp2MRwRPAsSnO7nCa/juyPh0Z91HCb/cabDqBRh0e/im/51xgrPe54TvwnM82b+u5zgX4ZPa7n/fxvJ4zU3XJQ3jTQzPaKtgFVjeZnUBs9nLOhqOnA6ZB9e9z5pXYeUzMORfkNE/fLGJOd9dADtzYeyU0B/LDkB5vqaQjQbZJ5s9/qrJkPcFDJ4UutkQGqoZfo7+au1XPDF398jLPx/yZyYdPam66CfNy6Kti3js+8cAeHjCwyr6SUzRFXipW9VOIAhluaYjkUjQ608w6n+Q1MiiH8DyJ+ANnzMSRMQNZVvgjTT4+U7TkQjAIY85C9WbLvqBM/VXoMJ0FM3ToLvglK3hXfMzsQ0kZ4XveLJ/CRmhLfoBVJU6XxI9/EvwfHMqCRUbQ9P+/Kvh9VTngr80D8ntIOtIJ+fUpXS9M+I0Pjx3V0sEqCiEinxnGs5QG3Q7nLASdLE4ethBM8fd9h2snhx5s5EULICcO8z9u4TZaf1O49S+p9K3dV8+/e2n/PvYf9MivoXpsMSAJduW0P/x/gTsACf2PpGjuh9lOiQRV6nwJ3UbeCv8Zknj10Yq/Bk+HgprX3M3LoleiW0gc2joLwQ2Y8amodi5Gb4925neN5wq/NBmNLToFN7jSmSrLIIPeztTFEuTNTivtOgASa1DE0xdRjwLh78b3mPK/pWsd9b6C5XV/4XXUyD3s9AdQ9y1cwPWxvfwlc4MTfvpfZw1fxNbhaZ9iUx2EHbm1f18v7/C6X6dL8a4Gucrh7/tzALQ0NG/pRvgrTYw548Ne51GGUeP+dfCe12cEeLhNuwJZ+mUSLsJYe3LkHMLbJ9rOpKwsCyLN894k0WXL+LoHkfXuZ+m+Ix9j8zevTTFA+MfMBiJSGjoliTDIvoPSVNPXiv9zmicQLk78Yh5Sx6Crd/CYW827vVdznC+JCS8Xi8DBw40c/D4dFj3BiSG+WJ/+gHO+lESOZY9BsEK6HOVuRiCVdDvek0B6IJG5RXbhh2rnDV+MwzlJIkM866ADe87IyFSu7nffmp36Ppb57tEh3ZHwfEr6JjWIzTt9/qT8yXNyxcTnAvWp+XX/RlWo7FiWpM+B9k2FC5wzlmseEjKgi5n1u+1yx6D8u3Q72/OlMYS+eJ9kNoDyreamZ41wRf+Y+5Pz0ug4wn67LQHo9dWJGxuPeJWlm1fxql9T6VnZk/T4UiMM1EDsmw7HHMfyK8VFRXh8/koLCzE54vAP/y7rHnZuYDa/femI5FI8N2FsP4tOHF1eKdwk3qxbZvi4mLS0tKwTNx1WrkD4lPDf1yJLFMGQ8V2OGm96UjEBY3KK3bQmda51SFwVAhHe+1pxdOAR2sLR5ptc6DgB+eCkkZDyC+Mn69I7FnyL9ixAg76595r0/uXwOZPIPsUSMk2E5+E3F55pTwf1r0OGYOgzah9v7homTNTRJ9r4OBfjfjY1xr1tg2fjoKdG+HEtfo7J/tWWeTMgtDqEGjR0XQ0sh86VxERt/n9fjIyMvD7/aSnh2fkt6b6NCwYjPA5tH++AxbeZToKiRRDHnLupG1s0e+nm5x1/iQkgsEgq1atMpdXTBT95l8Dy58M/3GlbqNfgWMXmI5CXNKovGJ5YNAd0PPS0AX2a4v/Ccse2f9+El6th8EBlzoXQ3WvofwiGAyyJedV7Fl/gGDA3cbnXAarXnC3TYl8ff4Phv5776IfwKYpMP9KKF4W9rAkfPY6X6n0w/eXOTOS7I83CfpeCx2Pr7m9ZB18Mhy2fFX76ywLxs2AMVNV9JP9K/gJvj4Z1r5iOpLa2UHY/r0za5eYv7YiIjHHRD5R4U/2bcTzcPh7jXtt7mew+n8QrHQ3JjEnPr3uOx73x7Zh6cOw/h1XQ5IIUrYN1rwCRcvDc7xAOSx/HDZ/HJ7jSf34+pofEfzj32HGCfr7Y1KfK+s/TZYbDnsbRkXohRRx1mP99ixnTT43zb8Wfrze3TYlLFJL5mOtfRnyXVxPqHIHrHgytOtKSvTpcQGM/QRaDTcdiYRTi85OQa7vtfvfN6UzHHQ/tDui5vaqHVC0FLZ8XfdrvQmQ0b9psUr45X4GX5/qjAwNl7QDYPhz0OHY8B2zITZNgU+GObN+iYhITNBE97JvrUc0/rXLHoON70OXs92LR8zb+i2UrIeuZzX8tSeuhcBO92OSyOD/GWae40yzlH516I/nTYRTt4X3A5vsn21DyWoIlIGvn5kYdqx0phf0xJs5voSfLrpFtmAl5H3hrP/Z7XfutbvhPeempMH3utemhMXWzLNoM/J6vGld3Ws0LgVOK9BNH81RoAzm/BFaZMOBd9R8LiED2o83EpYY5PFCh2P2v18w4NzYWtuIPV8/OGWzk1t+bem/nSk++9+kpQ6i0Y5VsOFd6PlHaH90eI6ZnAU9zg/PsRqj3ZHQ9zrnu4iIxAQV/mT/KvxgByAxs2GvG3QH9LjIOemW2PHT3yH/B+hyRsNG/1kWJLYKXVwCQFJSkrmDZw6BUS9Bm8PCd8y4lNo/jIs5wQr4sA9kTYAxH5iJ4dDXnOlqxBWNyis7VjmjLrv+FvqHYURWZRFY8RCXHPpjScMltXZu/nH7/Tl+KVSVuNumhEV8apZTpHGTZTlFHml+PImw9WtI71Nze1WJM21dSldNxdgM1Hq+UrIW4tLqvpax8X2Y9xcY+SK0G7v387V9zrBtWP+2c64z8PamBS1mdDkbsk9r+DWuprDtyM5DcS3goEmmo4goRq+tiIi4QFN9Gub1RnhRrPBneDMDlv6r4a/NGAAdj3M9JDFs4O1wxPsNX6unfDv4F0GVRvyFitfrpU+fPubySnwadD0HUly+kFeXTVOhSOu1RBxvIgy60/wdrY2dllhqaHReScpyRmBYYcpHH/SCL8J0x7Y0TiiKspbH+dsjUaU6r1g4U4otf8qdhnfmQcECnWs2R5YFE3+EMR/V3J73BbzfHVY+ayQsCZ9az1c2fwrvdYU1L9X9wmAFeJMhpUvd+2ycAtMOg4oC57FlwVHTYdyXznmvRJ/4fRSDQ+WjvvDl8fvfzzTb1jp/RMC1FRGJOSbyia6KGRbxC8Wm9nDu1m95UMNeZ9sQqAhNTGJWuzHOV0NHcm54Hz7qD3nTQxGV4OST7du3m88rlcWhn2YrWAXfnAFzLg7tcaRx+l0H2aeYOXbJOlj5vPNdmqzReSWuBZywAvr9NTSB/VqXs6HDb8JzLGkc23bWgV35vDvtVRRC7nTYmetOexI21XnFBn7+h/Plxijtda/D1ANh++ymtyXRp7abAFp0gl5/hjajwx+PhFWt5yutR0H386Dl4Lpf2OVMOH4ZpHave5/ybZA/H/LnOTPfgHPjSWo3N0IXUwJlsOIZ2DYnDMeqAN8AZy30SFa1Ez7sDe/3aPafpSLm2oqIxAwT+USFP8Psho6aCre4ZBj1X8g+qWGvq8iH1xJh3pWhiEpMs4MNX1et5SBnDQTfgNDEJNi2zfr1683mleVPwBs+2B7qD1A2jHjBWYdAIpeJ38UtX8HsC8LwO9g8REReqY8hD4VnSlFpPMtyCjw5t7iTGwp+hM/Hwbo3m96WhFV1XgEY+T+Y+JM7o7RbDYeBt0Far6a3JdFpw3uw9NHdj1sOhqH/jvyL7dJktZ6vxKfCiOehbROXIehyJpy80fn8+/HBzucdiX6lG2DOJbDkn6E/ljcBDnsTDrov9MdqirhkGP4M9LkKUjqbjsaoqPkMJCJRw0Q+0Rp/EhrBSuh8BmQcaDoScZttw7udIb0XHPV5/V+XOcT5ktiWMcj5v+8N8Xz4nnjofGpojyGNV7oBvjgGOp0EB94Z3mNnjYPD34VWI8J7XNmbfzGs/q8zBXCGbvoQnAuwyR3cWeMmtQcMfQzaHtH0tsScVkPda6v1MOdLmq/F/4SC+dDzEqgshMQ2kb2mloRPsNL5/LCnJQ9B8Uo48A5IaFn3a72Jzle7MZB9KnQ8MaShSpik9XQ+M9S2tuOe7KCzvISvz773ixVtD3e+wLn2s20WtBllNiYREWkUjfiT/dvwHnw2tmFraSVnwaGvmV/jSdxnWc4Ufm2aeOekxKY2o+HQV0Nf5K3cEdr2pWmS2jvfPQnhP3ZyFnQ6EZLbhf/YUlPJWlh0D2yZEfpjfXc+LH4w9MeRpmk9wr07yFOyodefIKO/O+2JOZXFzhpsmrZVmmrIw3DsAqfAM+ME+OQQCAZMRyUmBcpg6sEw67y9n9v4EWyeCvG++rWV1NYZtdWig6shikGdTtj/WsEzz4VPR+z+G7VjjbPsREPMuxIWRfhov9r8dANMOxRyG3DDt4iIRAwV/mT/KoucOydL15uORCLF0Edg0O0Ne803Z9T+gUtclZa2nw8uscC24cNeMEN320YsjxeO+xkG3hLe4wbKnNGG4qpG55U2hzoXYHte4m5AtVnzMmz5MvTHkaar8DvrJEmzViOvbP4UZl8Eq19sWqNfnwbfntu0NiS6ZR7srNUWrHSmd2w7puHrkkvUqvV8xZvkFOwSM/d+7shpcPQ37kw1LNGrssgZLbyrsFe6Eda/vfv5zmdCt98761dXFsP0I52ZTeo7ZVsw4MyAkReFxbOel0CPi6D1cNORGNEsrq2ISEyzbE1YbERRURE+nw+/3096errpcPYtWAWWt2HTpKx/B3I/gwE3QXL70MUm0ePTURCXCkd+ajoSCbVVk2HN/+CID0Iz5WdVKXx/GaQd4OQYkV02TYUvj3XWpuhxoeloJJwC5c53b6LZOGT/Ph4KJevglNymXWydewXkfQnHzNX7Hu0C5c5F0c6nQ0I9R97U5rMjwJMER37iXmwSfapKYPv3zrSMIiL7s/4d+PoUGPgPGHgzfH0qbHgXTs6FpDY19w2UwYKbIbUn9Lhg7+lj6xKsdNaIjOYZSap2OqPzs0+CFp1MRyMiEnVM1IJ0a5NhwWDQdAj754lr+NoIW2bA8sfB1tQqMami0LmjesnD9X/N+Jkq+oVYMBgkNzfXfF4pWeOsBbBjdWjaj2sBIyer6BfpgpXw042w6P7wHTOpLXT7A7QeHb5jxrgm55WqEtg2O/RTre1af0ciX6+/wIBbGj5N1q95Epwb0/S+R5298oo3EXpe1LSiH8C4GSr6Ccz6A0wfq6ljm5l6na/4Fzl/ewp/hu8uaNhSJhK7Op4Ao16Cftc5j3v9GUa94ty0/GveJDjofjjg0voX/cDZN5qLfuCMWJx3Bax9zXQkYREx11ZEJGaYyCcq/BkWNQMut37r3IlbXwf9E07etHudJ4ktcWmw8T1N1RVhbNsmNzfXfF7p+1c4zQ++vqFpv0TTDkcFTzyseQXWhfHDYeYQGPkC+PqE75gxrsl5Jed2Z12U4uXuBrYnOwhbZ0LxitAdQ9zT/ffQ+8/gbeIaoAf/E4790ZWQJLzqzCtl28C/xExQEju6/QFSezg3Bkizsd/zlZK18Olo+PYs2DQFVj0P5dvDG6REJo8Xup6z+0aidmOhyxkQl7zv11UUODc67s/Gj2D73KbHaVrbI+Cwt6DzGaYjCYuIubYiIjHDRD5R4U/q5+e7YPbF9b9j3+N1pvjUmgqxyeOFU/JgVD2Lwf7FsOIZKN0U2rgkMsS1CN3//UWT4MM+kD8/NO2Lu8Z9AeNnh+dY+lAWmToeD4PuhPgQTmURrIRpo50io0SP+lwsk+YjUAHvd4W5lzfu9f4lsPxJrfMq0Ol4OG7h3lP0SfOW1B66nutMBd/vOjg2B1qPMB2VRKt1b8HbWbDp4/3vO+dSmH1B6GMKtfhUyD4FUrJhZ55uuBMRiQIq/En9DLgJxkwB6nlhddsc5646iV1xKfXfd/MnMOdiKNZ0Ks3Gtjmw5mX32217BLQaBskd3G9b3JfSJXw3gKx6DqYOgcKc8BxP6qftYTDgRmgRwv+zlseZaaDL2aE7hrhr/rXOBbNdazM2VLDKuSkt93N34xJzvAnQ/0bofn7jXp87zVn/t2ipu3FJdNIUwPJr3gQ45FHoMNF5nDGg4cuZiOzS6hBocxjEp+17P9t21h4fdGd44gqHikJ4twP89HfTkYiIyH7EmQ6gubOi5WSzzaj672vbzroKbQ+DsfW4A0qiU0WhU9BL7wstB+173y5nQlpPaHlQWEJrrizLIjMzMzLySs6tkPcFdD69Yesf1Ma2nXXC4lOdO3OP+lwf1KPJtjlQuh46nxra41QWO8dRUdhVEZVX6uKJh75Xm45CGiK5PWQOhfJt0KJjw19ftgUW3AS9roCsI92PT0KqzrzS/4bGN9r5DEg7wJnyWUSanag4X5HYkdIZjvps//tZFnQ4JvTxhFNCBvS7AVoONh1JSCmniIjbTOQTy9aExUYUFRXh8/nw+/2kp4dw+is32UEI7Nz/SK9gAJb927n42qV5zP/dLBUsgKkHQv+/w4F3mY5GIs2WryBYAW3HgKeJ95gsfhCWP+GMOk4/wJXwJIw+Hgo7VsMpW0I/+s8OOqO/JLJ8ezZU7oAxH5iORGJFoAL8C50pZNN6mI5G3BYo14gtERGJDsFA3Z9xdm6GpCzdtCoiIkZqQbo6ZlgwGDQdQv1UFMDraTDvqv3v6/FCnytV9It16X1g5P+cdRL2p6Iw5OGIk0/WrVsXGXml7eGQNa7pRT8Ab5KzuHpy+6a3JeF34N1w2Nuh/cBbWex8V9HPda7klapSqNrhXlC/Vp4PUw9y1gCV5sGbAJkHqegXpfaZV767ED7o6dzI0RBl27TWq0gzFlGfg6R5sG34+lT4YkLdz38yDD4/KrxxhVMMr9esnCIibjORT3SFzLCoGXAZnwHtxztz4YuAc9Gt27mQ2n3f+wUq4K1WMPP34YmrGbNtm/z8/MjJK7btTtG315/gmPnOVJ8SfdqPh3ZHhK4oV/Cjc5F4zauhab+ZcyWvHPEejPvCvaB+zQ44xd+qnaE7hrhv08fw5W+gbGvDX1tZ7Iwilai0z7yS0sWZrrMh5w+Bcni7Lcz6nWsxikh0ibjPQRL7LAu8KeBNrn3N4kApZJ8G7SeGP7Zw+OJYmHaY6ShCRjlFRNxmIp9ojT+pH8uCw9+p374b3oOcf8DQR6DN6NDGJeZVlYInoe6RXYES6HYetB4e1rAkAnzY27lp4Jg5jXv9tjng6+ssmu7GyEExx7adEV/xae637U2BuDRIaOl+2xIdktrACStMRyENtXOjs1ZwwY/Q/uiGvXbRJFh4Fxybo5vSYs3AWxr+mkAp9LwYWg1zPx4REZG6jHi+7mk+41JgyEPhjSecWnRy+rivqU5FRMQoXUkV9wUroHwLoHnMY96yx2DuFTB+JrQeUfs+CS1hxLPhjUsiQ6eTnO+23fBpHgNl8NXxkNQeJv6gdRGimW3D2+2cERxjp7rffvoBcNwiZxSyRKaKAlj9klOgaTfGdDQSKbqcBZ3PaNwNAS0HQ5dz9j/rgDQPCS1h2FOmoxARkeZmz4LXnp95d43qiOXPsMOfNh2BiIjsh6b6NMyKphOBrd/Ct+dAwYJ979f5dDhpPbQZFZ64xBzfAOh8GngSTUciOPkkKysrcvLKQfc5X42Kx4JBd0Lfv8b2B6bmwLKcInBdNwc0hm3D939yRoWCin4h5EpeqdoJ866Ada+7F9ieKvyw4hnInxea9iU04lIaPwq482kw+iWIa+FuTBIW+80r696CzyfsXr9VRGQ/Iu5zkDQfpRtg+pGw+P7d24oWw/vdYN2b5uKSJjGaU/xLdn/OFZGYYSKfqPBnmMcTRW9B2RZY+wr4F5qORCJFuyPg0Nch86C691nyMMy+SGsvhYHH4yErKyu68kpdvInOtF3dzjUdibhh+NMw8Fb32itcACufhZX/ca9NqZUreSW5PRz5GQxw8XdgTzs3w5yLYX09pySXyLEzD1Y8DTvWmI5Ewmi/eaV0HWyZAf5F9Wtw0SSYdZ6zrrSINEsx9TlIoktiWyhe6Vwv26V0o/M93mcmpnBZ/V/48jhnus8YYyyn2DZ8exZ8OhxKN4X32CISUibOUXRWZFggEEV/IDtMhNOLoevZde9j2/DzXZA3I3xxSWTL+wLWvgbeJNORxLxAIMDKlSsjK6/M/T+Y+buGvaZoOZRtDU08EhtaHgjHzIOD/2k6kpjnSl6xLMg6CpLbuRfYnlp0gjEfQ7ffh6Z9CZ3t38GcSyFvev1fU7YFPh8Pa14NXVwSUvvNKz0uhtML678+9JavYeMHGv0t0oxF5OcgaR68CXD8Mjj4AedxsMpZu/iE1c75bywrWubcqFO8zHQkrjOWUyzLWR6j39+gRQdn27wr4d3OEKwMbywi4ioT5ygq/En9eZMgPnXf++zcCAtuCt10XhJ5lj0GX59e9/OHvwsnbdB0jWFSXBxh02IVLYGCH8EO1v81c/8M73d3pu+T2OBfAjN/C5tcWONv15oZGQMgPr3p7cl+uZJXbNuZCilQ3vS2fi0+FTpMgPRe7rctodV6tHOe0OG4+r+mdL0z/fzOjSELS0Jvn3klPrVhN4yN+RBOXNv0oEQkqkXc5yBpPry/LH2yMw/e7QTLn3Kuf1gxfsm177Vw6jbw9TUdSUgYyynJ7WHwvbsfx6c723ZdU9mZqyKgiNRLjP8VEtcVLd33hdukds4ojN5/CV9MYlbBT7DpQyjPr/15y4KEGJ/iQup2xHtwXE7DPvT0uBD6XKPfm1hiV8Gal5wicFPkz4cpgyDvSzeiknBa/AC8m+28hyK7JLWGTidCclb9X5M5BM7YAb2uCF1cYt6O1bD00fqv87e/mxNFRERCrXQ9JGRCYivTkYRHgk8zO7nJvwgW3OYUkPc06B8wYbZTYK4qgS/GO8vp7FKy1pk1KQanXBWRplHhTxrmh7/CVyc60xfUxhMPmQdDeu/wxiXmHPwAnL4DEjP3fq5sK2z8qOZ899K8NOaDQJczYNBtrociBqX3hdP90P+GprVTtMxZ+0kj/aJPm9FwwOWQkOF+29u/h1cTYclD7rct4VFR2LD9LUvTOsa6De/DvCtg6zf73m/nZmdfnWuKiIhprYbCcQsh+xTTkYRPyXpY/E8oWWc6kui3ajL8fDuUrK57H08CtDsSvMm7ty24BT7sBYHS0McoIlElznQAzZ0VbdMfHnA5dD4T7AC1/vqUboDE1rrrpznZ1wX4rd/C1yfDiOeh+3lhC6m5siyL7OzsyMsrGz90Fjg/4NJ971dVAhUFznpdEls8XvC4UKzrehZ0OgHiWjS9LakX1/JKm1HOVyjEp0P7YyCla2jal9CafTGsfNYZxVef/9ubP3VuNGs3NvSxSUjUK69knwJpPaDNoftuLO8LmHkujH7NuXFIRJqliP0cJM2PZQHN6Pcwfx78cK1zbhZDM38ZySkH3gVZ46DVPtY49sTDkIdrju7rdBIkd4Q4zX4gEslMnKNYtr1rsRwJp6KiInw+H36/n/T0GBq58PEwZ82Vk7XuSrNh287d2MFKyDqy5nMl62DzJ86i1qndzcQn5k07zJkS9rQCpwBUl+VPwNwr4IiPnPW6JLbsWAP53zsfTDzxpqMRkUix8jnnPGLwJEhqs//9pxwIlUVw4j7uhpbmo2Qt5H4O7Y/WjUMiIiLhVlXiLMPQ7kiIS97v7hJiqyZD7mcw8sVfitAiEilM1II01adhgUCUzsFcV72486nQ85LwxiJmWRZ8czr8eP3ez6V0hp4Xq+gXJoFAgCVLlkReXjnonzDhu/2v85feDzqe4EwJKLFnxVPwzRlQvKLhrw0GYPo4WPG0+3HJPrmaV378O3x1UtPbkdjS4wIY8Vz9in4ABz/kfEnUalBeKVkLVTvrfj6lC/Q4X0U/kWYuYj8HicS6uBToeFzMFf3CnlNWPAOFPze9nS0zYPNUZ0CGiEQUE+comupTGqZyB0w90Bl+PuypvZ/v97fwxyTmDXmk9gWsbVt3GYVZWVmZ6RD21npY/fZrd4TzJbGp8+ng6wdJ7Rr+2pLV4M+BkhHuxyX75Vpe2bkRCnOgshji09xpE6BoKSx/ErqcCa31OxLzfj27gESleuWVFc/AnIudmQA6Hrv388Eq2LEK0nu5H6CIRJ2I/Bwk0lwUrwA7GFN/k8OWU3bmwZxLIPtUOOyNprU1eJIzFei+luQRkWZDI/6kYeJTIbk9JLY1HYlEki5nONN57qmqFN7Jgpm/NxOTRJZAuTMFSF3KtoQtFDEk82Do9jtIzGz4a9N6wonrdHNJtDvkCTh+hbtFP4Adq2Hpw+Bf6G67Eh62Dd//CRbcuv99g1V1zzohsaf1CGetv8yDan9+1XPwUT/Y8EF44xIREZHddubCh72dtf6k4RJbwZGfQb/rmt5WUpvdRT9dYxFp9lT4k4Y7+hs48I69t6/4D3x5nDMljzQ/waqa731FPrQeqdEX4ph7OUwf61yg/7WKQni3c+3TxUrssYONe5030f2CkYRXXIvQjAJvN8YpDHc+0/22JfQsCzZ/6nztz6rn4M0M2PJ1yMOSCODrD4e95dx0WJuWBzlThLcbE9awREREZA/JWXDg3dD7StORRCdPnDOjRatD3Gtz0yfwXjdY96Z7bYpI1NFUn4Z5PDFUey1ZA7nTId5nOhIJN9uGj4eAHYBjc5yLeC06weHvmo6sWfF4PHTv3j0y80q38yC9T+1TTlQWQaeTnAt8Ets+G+MsAH/M9/V/zfq3If8H6HNl7VMKS0i5nlfy58PqF2HQHe4Vcr1JkJLtTltixjHz6jclUUIryByi9dyiXL3zyq4bBWwb1r8JWeMhYY/PGa0OgcPfDl2gIhI1IvpzkEhzEGMzs4Qtp1SVwo6V4Bvg7g2SqV2dWXPSYmfqVZFoZ+IcxbJtzZdjQlFRET6fD7/fT3p6FM69vOA2Z82lkZNrbg9WOXerSPPz810QrIT+fwdPvNb2E5G9zbnU+XAz6r/1f803Z8CG9+CUPEjICFloEibLHoO5f4bD34NOJ7jTZmUxlOVBUpYzJbmIxJZNH8OXE52RBEMecmaY2LlZs0qIiIhEmsIcSO8HHq/pSKLDurfgm9Ng+LPQ4wJ327aDYOlmCJFIYaIWpAxgWCAQMB1C4xQugC1fQdXOmttV9Gu+BtwIg24DbwKseAo+O8JZ4FnCJhAIkJOTE9l5pWonVPh3P9a9J83LsKcaVvQDGPUKTPheRT9DXM8rXc6CYxdAx+PdaQ9g01T44ADYPNW9NiW8Kouc91HnDc1Cg/NK+wnOFGL9b3QeL/03fDoSts4KXZAiElWi4nOQSKxb/iRMGQSbPzEdSZOFLaek94JeV0DW0e63bXmc4t/6d2HNK+63LyINYuIcRYU/aZzhz8AJKyEu2Xlcsg6WPwXFK83GJeYFyp2pCvwLndEXElYR/WG38Gd4uy0s/dfubaueh09Ggn+Rubgksnm80HKQ6SiaNVfzSmIryBjo7qjw9D7Q73rnu0Qn/yL48lhY90bd+9hB+O4CWPta+OKSkGlQXrEs6H8DJLV2HrefAIMnacSfiNQQ0Z+DRJqDDsdC9/MgpYvpSFwRlpySMRCGPhLCZQssmH8l/HSjbroWaYZU+JPGScysOWR81Qvw/R+dkYDSfBUuhHc7QXwGnLRJU65JTel9oM2hzlzzu5Rvg+JlkNzeXFwSPpVF8MN1sPL5+u2/7DFnuhiJLcFKyP0cipa7017LQTD4HueDs0QnX3845Alnvde6lG50bhbZOjNsYUmEan809LtO08qLiIhEkpTOMOJ5yOhvOpLosOUrZ8mCULIsGDEZjpqu8yaRZkiFP2m84pUw/xooWupMvXPom/u+YCOxL70X+Po5RRxvguloJNJ44mDsVOh6jvO4qtS5cHdKLiS0NBubhIc3GZY+Ahve2f++JeucteAW3hP6uCS8ChfA50c500KLAMSnwQF/BF/fuvdJyYbTCmDATeGLS0REREQaLlhlOoLIVvATTD8KZl8U+mO1OwJSu4X+OCIScSzb1lhfE3Yt6FhYWIjP5zMdTuPkfg7fnQ+HPA4djzMdjUizZ9s2ZWVlJCUlYUX63Vybpjr5Y/Sr0G6M6WgknHashhad97/gux10RvbEp0HLA8MTm+wlJHnFtmHFk9ByCLQe1vT2Nk6BpQ87o/4yhzS9PTEnGHBmlIj0v2HSJFF1viIiUUF5RSRCVBTAx0OdNeuGPWk6mkYLSU5ZcBuUroVDngRPAiy42bkp2tfPnfb3paoUts/RtRcRg/x+PxkZGfj9ftLT08NyTI34k8ZrewRkjYOUrqYjEZFfJCREyUjLqlJn9Nee035K85Dabf9FP3Au/rc9VEW/COB6XrEsOOAyd4p+AGV5sO07ZypZiV6LJsEbqZA/t/bn173ljASWmBA15ysiEjWUV0QiQHwGtOgESVmmI2myJueUTR/DT3vMVLF9DuR9Ad5E5/PQgXeGp+gHMOcSmD4Wdm4Oz/FEJCKo8GdYMBg0HULjebww4lnN3y0SIYLBIDk5OdGRVzqfCsevcD4USPNStRO2fgs71tS9T7DKmf7EjoLf5RgX0rxi2+68xz3OhzOKoN3Yprcl5rSfCC0PAitu7+fKtsA3p8FPfw9/XOK6qDpfEZGooLwiEiEsC8bNgEG3mY6kSVzJKT/d4MxKUlXqPB47BY41tH599wvg4IfAm2Tm+CJi5BxFhT8RETGjPqO+JPYU/AjTDoW1L9e9T/58mDoYfr4rXFFJuK2aDG/4YNss05FIpGg5CMbPhMyD9n7O2wJGvQw9/xj+uEREREREGmr8d85XXIvd2+LTzMSSdST0uRISWpo5vogYocKfiIiIhE/LQXDgPc7onrokZkLf66DDMeGLS8IruQO0Ggq4sGZG0XLY8L6zpojEhsIcZ72/XeJToevZzvS/IiIiIhK5Kgpg3lWw6gXTkZjlTYSMAaajqClYVfMcW0Rimgp/IiIiEj5xKdD/+tpH9eyS1hMOmgStDglfXBJe7Y+Goz6HNqOa3taGd+CrE6F4RdPbEvPWvg5TDoQ1/9u9rTzfXDwiIiIiUn/eFrD8cdj4oelIzChaDjm3w85c05HUtGoyvJEO278zHYmIhIkKf4Z5PHoLRMQdHo+HgQMHKq9IdKhrfTfbdr4kIkRFXulwHIx8EVK7m45E3NB+PHQ+zVnvD6B0E7zVCn66yWxc4pqoyCsiElWUV0QiiDcRjl8Oh75hOpJGa1JOWfNfyLkNipa4HleTpHaHNqNNRyHSbJk4R9FZkYhIDKmoqDAdgsj+rXkV3moNW7/Z+7lt38F7XWD9O+GPS2oVsryy4mn44W9NbyejP3T7HSS2anpbYl5CBhz6ujMtMECgFLqcDZlDjIYl7tL5ioi4TXlFJIKkdAbLhSn9DWp0ThlwC4yZCm2PcDegpmp7GBw5TcU/kWZEhT/DgsFaRjuIiDRCMBhk6dKlyisS+ZLbQ3qf2kf8VfqdBdATMsMfl+wlpHll44ew9CFnrYmmqCpxJx6JPLnTYdljMOolyD7ZdDTiEp2viIjblFdEIkygHHI/h4IfTUfSKE3KKZ44Z636KC98ioi7TJyjqPAnIiIi4dXuCBj/LbQbs/dzHY6B3yxx9pHYdsjjcMpW58NxU3w8BL6Y6E5MElmWPgKrnoeStaYjEREREZH6qsiHz4+Cpf82HUn4BKvgp5uhZJ3pSOq24QP4+nSoKDQdiYiEgQp/IiIiIhJ+LTpBgq9pbQTKIXMYtDzQnZgksox8ESbMgdSupiMRERERkfpKbu/c5Nfrz6YjCZ/NH8PCO2Hls6YjqVvxctjwNvgXm45ERMKgibdYi4hIJPF6vaZDEKmf/Pmw8C4Y+A9njTaArTNhzcvQ63Lw9TUbn1QLWV4JBqB4KXgSIK1n49rwJsKoF92NSyJHgq/pxWGJSDpfERG3Ka+IRJgDLjMdQZM0OKd0OA7GfAytDglNQG7ocSH0vATiU01HIiJhoBF/hunkVETc4vV6GThwoPKKRIfKYlj/Dmz9Zve2je/D8sfA0u9wpAhpXqnIh4/6w6JJ7rctIhFL5ysi4jblFZEIFax01vqLMo3KKZYFHSZAYgSvVZ/gU9FPxBAT5ygq/Blm27bpEEQkRti2TVFRkfKKRIe2h8GJa+GAS3dvG3wvHLcY0nuZi0tqCGleSWwNA26F7FMb9/qqEvhkJKz+r7txiUhI6XxFRNymvCISoeZcCp+Pi7qpJRuUU+wg/HAd+BeFPjA3FK+AdW+YjkKk2TFxjqLCn2HBYNB0CCISI4LBIKtWrVJekehgeSAle+/tvj7hj0XqFNK8Ylkw6DbocEzjXl+01JkqtHy7q2GJSGjpfEVE3Ka8IhKhev4RBt0BLTqajqRBGpRTtsyAxffDqsmhD8wNP98J35wBZVtNRyLSrJg4R9EafyIiImKGbcP6N6FkLexYBd3Og9bDTEcl0SLzYDhlizOFkIiIiIiIRJbWw2L/8127sTDhe2hRy02tkajnJdDhWIhrYToSEQkxFf5ERETEDMuCxQ/C9u+cx0ntY/+DodS07g346UYY+b/GvfeeOOdLREREREQiU1UpbPoIsk9zPgPGmlZDTUdQf21GmY5ARMJEV0pERGJIUlKS6RBEGmbYU5DUDirynTXfJOKENK94ksDyQtWOhr3OvxjWvQndfgepXUMSmoiEjs5XRMRtyisiEWzeX2DlszD6Nehyhulo6qVeOWXVZEjtDm0Ojb6CZtVOiEs2HYWIhJBla/VjI4qKivD5fPj9ftLT002HIyIiIhI9fvgbLL4Pxs+C1iNMRyMiIiIiInWpKoWVz0GvPzlrvceCQBm8nQUpXWDij9FV+PvqFNj6DZySGzvvh0iEM1EL0og/w7T4tIi4JRgMUlBQQMuWLfF4dPImIk0XsXll8D3QfoKKfiJRKGLziohELeUVkQgX1wJ6/3n342BVRE/XX6+c4k2C8d9BxfboKvqB8xkqPg0qiyHBZzoakWbBRA1IZ0SGacCliLjFtm3Wr1+vvCIirglLXsn7AuZfA8HK+r/G8kDWkaGLSURCRucrIuI25RWRKPLlb2D6GNNR7FO9c4qvD7QZHZ6g3NTvOhg5WUU/kTAycY6iwp+IiIiImJP7GSx5EAp/3v++RUthzh+hdEPo4xIREREREXcltYOk9s6ov2hVvBI2fQyBCtORiIjUSYU/ERERETGn5x/hN0uh5eD977v2VVjxFJSsDXlYIiIiIiLishHPwmFvRPRUn/u18hn4ciLkf286ksZb/V/4ZARU7jAdiYiESBRnWRER+bW0tDTTIYhIjAl5XknJrv++A26BDsdCq0NCF4+IhJzOV0TEbcorIuKmfeaUHhdBYmtoPTJ8AbmtPB+KlzlfmQebjkZEQsCyNQm6EUVFRfh8Pvx+P+np6abDERERETGnssiZ6rPNKNORiIiIiIhIKK1/G5Y9Boe/C/Eq2hsRKAMrHjxe05GINAsmakGa6tOwYDBoOgQRiRHBYJDc3FzlFRFxTdjyyuxLYNqhUOGve5+NHzlraQQDoY1FREJK5ysi4jblFZEos3MzbJsFhTmmI6nVPnNK0XIIlIc/KLd5k1T0EwkjE+coKvwZpgGXIuIW27bJzc1VXhER14Qtr3T7PRx0P7CP4+TcCrN+B5ZOX0Wimc5XRMRtyisiUab7eXDqtoid7aPOnGLbMOM3MPVA5+doV7Qccv4BpRtMRyIS80yco2iNPxERERExq+Oxzte+DPuP86HUssITk4iIiIiIuC8upelt7LqIHs7PBnYAelwAljc2PpMU/ODcXJncAXpeZDoaEXGZCn8iIiIiEhlsu+4P0ZkHOV8iIiIiIhLdyrbAmpeg9WhoPax+r9nzs0JlIbydBS0PhqO/Ak98yEKt5omDfn8L/XHCpf0EGPcVtB5hOhIRCQHNlWSYFQt3iIhIRLAsi8zMTOUVEXFNWPPK95fDBwfUPm1O6QYIVIQ+BhEJOZ2viIjblFdEolBlEfzwV1j6r7r3mX0RfDQA/IsgWAXfnQdLHnKeC1ZCuyPhkEddL/rVmlNsG6pKXT2OcQk+aHtYeIqmIs2ciXMUFf4M83j0FoiIOzweD507d1ZeERHXhDWvxKVCi07ORYBf+/ZseK9LbKylIdLM6XxFRNymvCIShdJ6wthPYcTzNbdv+Xr3OX/3C6Gi0PmMUFUM+XMh93Owg5DUFsZOhcwhrodWa07ZPscZYbjmVdePZ5RtQ8FPWudPJMRMnKPorMiwYDBoOgQRiRHBYJB169Ypr4iIa8KaVw6aBOO+dO48/bUOx0L382NjLQ2RZk7nKyLiNuUVkSiVdSR4E3Y/3jQVpo+FTR85j9uMhBNXQ3w6JLSEo76Aw98Ga4/L2bYNeTNg2xzXwqo1pwR2Qnov8PVx7TgRYeu3MHUwrHzOdCQiMc3EOYoKf4bZunNdRFxi2zb5+fnKKyLimojJK/1vgMF3m41BRFwRMXlFRGKG8opIFKvww483wPp3IONAaDsG0nrtfn7PaSiT2u49LWVZLnw+Dn76u2sh1ZpT2o2BY+ZCy8GuHScitB4Ofa+D9seYjkQkppk4R4kL+xFFRERERGqz4X1Y8xKM+p/WmhARERERiXXBClj2KGQdDdknw1GfNez1ye1h+H+g9ejQxAfOqMJYnXnEE+/MvCIiMUcj/kREREQkMhT8BOvfgsIFu7d9ezZ8e67W9xMRERERiTVJbWD8TDj0jca30f08SD/AtZD2MuME+O6C2P48YttQtdN0FCLiIhX+DLNi9Y4REQk7y7LIyspSXhER14Q9r/T6E5y6FTKHOI+DlVC2BSqLYvcuW5FmRucrIuI25RWRKJcxEDzeprVh27D+XShe2eRwauSUQDlUFUHVjtj9PFKeD+91hflXm45EJGaZOEexbE2CbkRRURE+nw+/3096errpcEREREQiV1UJxKWYjkJERERERCLRtjnw6XDofj6MeM799gMV4E1wv91I8fnR0PYIGHCT6Uikvsq3g+WBhJamI5F6MFEL0og/wwKBgOkQRCRGBAIBVq5cqbwiIq4xklfKtsLK56B0w+5tKvqJxAydr4iI25RXRITWw2Doo3Dg3bu3Fa+Az8eDf0mDmqrOKfk5u6f3jOWiH8CR01T0iyZlW+H9HjDtMLCDpqORejBxjqLCn4hIDCkuLjYdgojEmLDnlS0zYPaF8G42fHUyVO4I7/FFJOR0viIiblNeERF6XQ7JWbsfexIhdxqsfKbBTVVtnYtn2nCYf5WLAYq4JKmNM0JzyMPOqD+RWug3Q0REREQiR/vxMPJ/kDUOyvI02k9EREREROonWAU/3eys9deiE4yfDYMnNbiZ8oRs7A7HQ5dzQhBkhPrhbzD7ItNRSF2KlsKcP0Kw0nl8xHvOZ2bYPTJVZA8q/ImIiIhI5IhPh27nOtPNHPU5GFgEW0REREREotD22bDwTlj2qPM5ovUw8Hgb3EzQ0wJ71MvO65uLwgWw5SuneCqRZ+2rsOIpyJ1ec3veDJh6EOzMdR4X/AirXgh3dBKB4kwH0NxZupglIi6xLIvs7GzlFRFxjfG84k0yc1wRCRnjeUVEYo7yiohUazMaDnsbfAOcx3YQ8uc5I6LqU8QrWIC17TuyO57Y/HLK6FedmzCbW7+jRd/roP0EaD2i5vbyrVC8DLzJzu/5dxc4owM7HAtJbc3EKnsxkU8s29ZYUBOKiorw+Xz4/X7S09NNhyMiIiIiIiIiIiISOyqL4a1W0G4cjJ2y//2/PRvWvgbH/gQZA0Mfn8i+7MyDFU9C7yshwVf3fhs+gNYjIak1+JdARQG0GRm2MGX/TNSCNNWnYYFAwHQIIhIjAoEAS5YsUV4REdcor4iI25RXRMRtyisiUqf4NBj6KAy4qX77D3uKwKhXWZIb3zxzyvbv4YfrNN1npFjxFOTcBnmf73u/Tsc7RT8AX5/dRb/SjU4RUIwzkU9U+BMRiSFlZWWmQxCRGKO8IiJuU14REbcpr4hInXpeAm1G7X4cKINts3c/Lt0EP14PW75xprrMPrX55pRNU2Dx/bBtlulIBKD/3+Gwt6DTSQ1/baAMvpwIn452fpZmR2v8iYiIiIiIiIiIiEhssm0oy4Xk9vD1abDpIzitABIyYMdKWDQJkrKg7aGmIzWrx4XQ6UTIONB0JALgiYPsUxr3Wm8SdL8QghXOz9LsqPAnIiIiIiIiIiIiIrFp2mgoy4PjV8Cgf0DLAwHbea7VMDh+OcRnmIwwMrTo5HyVbYGStZA5BCxNGBh2O9bAkoecEX/J7RrfTp//cy0kiT76n2uYx6O3QETc4fF46N69u/KKiLhGeUVE3Ka8IiJuU14Rkf3q+BtnusRgBWQeDAfeBQktnee8iZDWs3qNNOUUYMEt8MkwKN/uPLZt50vCY+MHsOwR2D57//vWh38xfHEsbHjfnfakwUzkE434M8yyLNMhiEiMsCyL9PR002GISAxRXhERtymviIjblFdEZL/6/73euyqn4BRJU7qCJ955nHM7lK6FQ54CbwJs/BAS2zijJXVt230H/MkpULc8yJ324tMhbzq0PQw6neBOm9IgJmpAzfjWhcgQCARMhyAiMSIQCJCTk6O8IiKuUV4REbcpr4iI25RXRMRNyilAh2Og//XOGojBAOTPhcIcsCud59e9CdNGQfk2o2HGLI8X2oyGuBbutNeiI5yyBfrf4E570mAm8olG/ImIxJBmfWIqIiGhvCIiblNeERG3Ka+IiJuUU/bg8cLh70FlIcSlONv6/hVaj4CkNkZDi0kFP4FdBS0Pdnc0ZYLPvbYkKmjEn4iIiIiIiIiIiIiI7M3jhcRWux9n9IcD/uj8vOg++OFvZuKKRQvvho8PgbI899te8zLMu8r9diUiqfAnIiIiIiIiIiIiIiINs2kqrH4RglWmI4kNva+Egx+C5Cz32974ASx/Aso0RWtzYNm2bZsOojkqKirC5/NRWFiIz6ehtiLSdLZtU1ZWRlJSkpFFY0Uk9iiviIjblFdExG3KKyLiJuWUBirdAIltwZtgOhLZn5K1EJ8OCS2hcoczdat+x8PC7/eTkZGB3+8nPT09LMfUiD8RkRiSkKATLRFxl/KKiLhNeUVE3Ka8IiJuUk5pgBadVPRzS+kmCOUYrZQuTtEP4Lvz4P3usH1u6I4nRqnwZ1gwGDQdgojEiGAwSE5OjvKKiLhGeUVE3Ka8IiJuU14RETcppzRC0VJY8q/QFq1iXWUxfHwQTB8TnuOl9oD+N0Baz/Acr5kzkU+ipvB31113MWrUKFq0aEFGRkat+6xbt47jjjuOFi1a0LZtW/76179SVVVzfuEvv/ySgw8+mMTERHr27MkLL7ywVzuPPfYYXbt2JSkpieHDhzNnzpwaz5eVlXH55ZfTqlUrUlNTOfXUU8nLC8GCmyIiIiIiIiIiIiIikWr5UzD/SihcYDqS6BWXCl3OgS5nh+d4B02CnpdAQkZ4jidhFzWFv4qKCk4//XQuu+yyWp8PBAIcd9xxVFRUMHPmTCZPnswLL7zALbfcUr3P6tWrOe644xg7diw//vgjV155JRdddBGffPJJ9T6vvfYaV199Nbfeeivz58/nwAMPZMKECWzZsqV6n6uuuooPPviAN954gxkzZrBp0yZOOeWU0HVeRERERERERERERCTS9LwExn4C6X1MRxIZ1rzqrKdXHxUFzkhJy4KDH4QD/hja2H6tdBPszA3vMSUsoqbwd/vtt3PVVVcxcODAWp//9NNPWbRoEf/73/8YPHgwEydO5I477uCxxx6joqICgCeffJJu3brxz3/+k759+/LnP/+Z0047jYceeqi6nQcffJCLL76Y888/n379+vHkk0/SokULnnvuOcBZiPHZZ5/lwQcf5Mgjj2TIkCE8//zzzJw5k++++y70/xAiIiIiIiIiIiIiIpHA1wfajwdvoulIzKsogHl/hrJfBhEVr4BVk2ufBrVkHXw8FH74q/PYssIXJ0DBj/BuR1j2aHiPK2ERZzoAt8yaNYuBAwfSrl276m0TJkzgsssuY+HChRx00EHMmjWLcePG1XjdhAkTuPLKKwFnVOG8efO44YYbqp/3eDyMGzeOWbNmATBv3jwqKytrtNOnTx86d+7MrFmzGDFiRK3xlZeXU15eXv24qKgIANu2CQQCAFiWhcfjIRgMYu+RDHZt37Xf/rZ7PB4sy6p1O+w9p2xd271eL7Zt17r91zHWtV19Up/Up/D1ybZt+vXrV51XYqFPsfg+qU/qUzT1ac+8sqvNaO9TfWJXn9Qn9Sl0fdqVV3adu8RCn/aMUX1Sn9Sn8PfJtu3qm8T3PG4096mu7eqT+qQ+hb5PAAMHDqxxzTba+xS292n7D5CcBUntYqdPDX2fSjbj8Q3ELvgZO+NgvOvfgh+vJ5DQBtpPqNmn+Eys5A7Yab2x93EdL2R9Su2H1f1CPG0Oa37vU5j7tCvOcIqZwl9ubm6Noh9Q/Tg3N3ef+xQVFbFz504KCgoIBAK17rNkyZLqNhISEvZaZ7Bdu3bVx6nNPffcw+23377X9kWLFpGamgpAZmYmnTt3ZsOGDeTn51fvk5WVRVZWFmvWrKG4uLh6e3Z2Nq1atWL58uWUlZVVb+/evTvp6eksWrSoxi9a7969SUhIICcnp0YMAwcOpKKigqVLl1Zv83q9DBw4kOLiYlatWlW9PSkpiT59+lBQUMD69eurt6elpdGjRw+2bNlS499BfVKf1Kfw9Wnt2rUUFhbi9Xpjpk+x+D6pT+pTtPUpEAjg9Xpjqk+x+D6pT+pTNPXJtm0GDx4cU32KxfdJfVKfoqlPHTp0oLCwkLy8vJjpUyy+T+qT+hQNfWrXrh0ZGRls2rQpZvoUjvepatOXJH49jrxWF7C53RUx0afGvU8V0Poh2AFpa9bQo8dFFBfvYOXW9rAth1YFb9GqYh4txr7Chlw/+a0fhhIv5OSY6VPK/zGww0CKi4qa2fsU3j6ZYNm/Lm+G0fXXX8+kSZP2uc/ixYvp02f3/MAvvPACV155JYWFhTX2u+SSS1i7dm2N9fpKS0tJSUlhypQpTJw4kV69enH++efXGNE3ZcoUjjvuOEpLSykoKKBjx47MnDmTkSNHVu9z3XXXMWPGDGbPns3LL7/M+eefX2P0HsCwYcMYO3Zsnf2pbcRfdnY2W7dupWXLloAq4+qT+qQ+Na1PFRUVLFy4kP79++P1emOiT7H4PqlP6lM09SkQCFTnlfj4+JjoU31iV5/UJ/UpdH3alVcGDRpUHU+092nPGGPlfVKf1Kdo6lMgEGDRokX079+/xh310dynurarT+qT+hT6PgWDQRYuXEi/fv2qb6yO9j6F5X2yg9jzr8buci5kDomNPtWyfZ99CgT4daHl132y5v4Zz8on4ZStBBMyI6dPgSqCWHttj8n3yUCfCgsLyczMxO/3k56eTjgYHfF3zTXXcN555+1zn/pWRLOyspgzZ06Nbbvu9MrKyqr+vufdX7v2SU9PJzk5Ga/Xi9frrXWfPduoqKigsLCwxqi/PfepTWJiIomJe89zvOuYe9rzRPXX+4Z7u2VZtW6vK8aGblef1Ke6tqtPjevTrmPseZxo71O4t6tP6pNbMTZ0e6T2aVdclmU1uJ1I7VNTtqtP6pNbMTZ0eyz1aVc+iaU+7aI+qU9uxdjQ7eqTs70h+0dLnxqyXX1Sn+rarj41rk+1tR/tfQrZdsuLNfRfe2+O5j7Vsb3OPuXcCIULYdSLkNBy9/Y9Yx/+hPNl23is2tf0C2ufAhXwySis1O54R7+y174x+T4Z6lO41R51mLRp04Y+ffrs8yshIaFebY0cOZKcnBy2bNlSvW3atGmkp6fTr1+/6n2mT59e43XTpk2rHt2XkJDAkCFDauwTDAaZPn169T5DhgwhPj6+xj5Lly5l3bp1NUYJioiIiIiIiIiIiIg0G7YN2+Y435ubsjzYsRziM/a/bx1Fv7DzJjjxxodnFJqET9Ss8bdu3Try8/NZt24dgUCAH3/8EYCePXuSmprK+PHj6devH7/73e+47777yM3N5aabbuLyyy+vHmn3xz/+kUcffZTrrruOCy64gM8//5zXX3+djz76qPo4V199NX/4wx8YOnQow4YN4+GHH6akpITzzz8fAJ/Px4UXXsjVV19NZmYm6enpXHHFFYwcOZIRI0aE/d9FRGRPkXJXiYjEDuUVEXGb8oqIuE15RUTcpJzSBIvvhx//Bkd9Du3Gmo4mvEY8D4HyyCnq1deRu5dOY/61kNoNel1uLh5xhdE1/hrivPPOY/LkyXtt/+KLLxgzZgwAa9eu5bLLLuPLL78kJSWFP/zhD9x7773Exe2ub3755ZdcddVVLFq0iE6dOnHzzTfvNd3oo48+yv33309ubi6DBw/mkUceYfjw4dXPl5WVcc011/DKK69QXl7OhAkTePzxx/c51eevFRUV4fP5wjqvq4iIiIiIiIiIiIhISBSvgCUPQt9rIbV+S3hJhLCD8IoXMg+Bo6ZDfJrpiGKGiVpQ1BT+Ys2uN7uwsBCfz2c6HBGJAbZtU1xcTFpaWvXaOSIiTaG8IiJuU14REbcpr4iIm5RTpMHy58Hcv8DQRyBziOlomiZQAZ746Bu1GOH8fj8ZGRlhLfwZXeNPnDUERUTcEAwGWbVqlfKKiLhGeUVE3Ka8IiJuU14RETcpp7jEDkJhjukowmP7XMifC1a86Uiazpugol8ImMgnKvyJiIiIiIiIiIiIiIg7Zp4LnwyDsq2mIwm9Ay6FkzdBy0GmI3HHyudh8YOmo5Amitv/LiIiIiIiIiIiIiIiIvXQ5WxI6QJWjI87sm1nhFxiK9ORuGflf6BoKfS5SqP/oliM/88TEWlekpKSTIcgIjFGeUVE3Ka8IiJuU14RETcpp7ig0wkw+N7YKoj9WtEymHoQ5H1pOhJ3DX8Gjluool+Us2zbtk0H0RwVFRXh8/nCuqCjiIiIiIiIiIiIiEhYBCuhZC2k9TQdifs2fgjfnAFHToM2o01HIxHMRC1II/4M00KxIuKWYDDI9u3blVdExDXKKyLiNuUVEXGb8oqIuEk5xUW2DR8Pha9PcX6ONR1/Ayesjs2iX+lGKFxoOoqYYSKfqPBnmAZciohbbNtm/fr1yisi4hrlFRFxm/KKiLhNeUVE3KSc4iLLgs5nOF92lelo3LN5GlQUOD8ntzMbSyjYQfiwL8y93HQkMcNEPokL+xFFRERERERERERERCS2DbjRdATuKlkLX50ArUbAuC9MRxMalgcG3gJJWaYjkSZQ4U9EREREREREREREREIjUA5Y4E0wHUnTpHSBgx6AVsNMRxJafa81HYE0kab6FBGJIWlpaaZDEJEYo7wiIm5TXhERtymviIiblFNcljcD3ukA694wHUnjBcqcKTABel0OrQ4xG4/IfqjwZ5jX6zUdgojECK/XS48ePZRXRMQ1yisi4jblFRFxm/KKiLhJOSUEfP2dkXLeJNORNN6s38O3Z/0ycrEZKM+HqUNgvkb+ucFEPlHhz7BgMGg6BBGJEcFgkNzcXOUVEXGN8oqIuE15RUTcprwiIm5STgmBpNYwcT50PjW0xynbArPOg8Kf3W23aicEK5wRf54on6q0vhJaQrDMWe9PmsxEPtE7Z5ht26ZDEJEYYds2ubm5yisi4hrlFRFxm/KKiLhNeUVE3KScEmKBivrtl/8DfHOWM8UmQOlG+PlOqCis+zU//R3WvgpJWU0Os4a4ZDj8XTj0DbAsd9uOVJYFxy2Eg+4zHUlMMJFPVPgTEREREREREREREZHQWfwAvNcFKvx7P1e+3Rmt51/sPM77HNa/AdvnOI9XPA0Lboat39bd/kEPwNBHnBGGUHNaztzPYWde0+JvLkW/XwuUOaMeJaqo8CciIiIiIiIiIiIiIqGT2AaSO8DOjXs/t+F9WD0ZCn5yHnc6EU5cD20Pdx4PuAkOexs6HLv3a+1fplFMyICelzg//3wXTB0Mtu1M/fn5OJj7p4bHvOVr+OwI2Dan4a+NBcUr4MO+sOhe05FIA6nwZ5jVXO8UEBHXWZZFZmam8oqIuEZ5RUTcprwiIm5TXhERNymnhFC338Exc8HXb+/nepzvPNf5dOdxWk9o0WH38554yD7ZGXUXKIdNU53tRcudAt+K/9Rsz7KgZC2Ub4OMATD4Hhg8qeExFy12Rh3GpTT8tbEgpQskt3eKttJoJvKJZWvCYiOKiorw+Xz4/X7S09NNhyMiIiIiIiIiIiIiEnqBCvAmNO61s86D1S/C+O8grQdMGQj9b4JevxrRF6wCT9zer7fthk3bWVkM8WmNizUWNPTfS/ZiohakEX+GBYNB0yGISIwIBoOsW7dOeUVEXKO8IiJuU14REbcpr4iIm5RTwmDeVfBRPwgGoKoUvj0Xtn9f/9f3/zsMvA0yD4bEVnD88r2LflB70S/vS/hyolPMqq/mXPSD3UW/QHnNdROl3kzkExX+DNOASxFxi23b5OfnK6+IiGuUV0TEbcorIuI25RURcZNyShgktYHUHmB5nELc2ldg23f1f316Lxh4y+7CXkOm4Vz7GmyZAaXr971f6Ub45kxY/mT9245lW2fCOx1gzcumI4lKJvJJLWVvERERERERERERERERl/W7Abqc44wk63gsHLcIUjqH59gDb4WDJkH8/qZbtCBvurPWoICvP6R0hbgWpiORelLhT0REREREREREREREQs+yILXr7se+PuE7dnJW3c8FKqB4OWT0hxYd4LjFzuhEgQQfTJxnOgppAE31aZilhTFFxCWWZZGVlaW8IiKuUV4REbcpr4iI25RXRMRNyinNQEUBLH0Uts+tuf2b0+Czw6FknfNYRb/a2Vr/sqFM5BMV/gzzePQWiIg7PB4PWVlZyisi4hrlFRFxm/KKiLhNeUVE3KSc0gzs3AzzroCV/6m5veel0O0PkNzBTFzRYMGt8FF/CAbq3mfjFPjqFGcEpQBmakDKYIYFAvv4TyIi0gCBQICVK1cqr4iIa5RXRMRtyisi4jblFRFxk3JKM+DrB4e/Dwc/CGtfcwqBAB2PgyEPgkero9XJ8oA3GXZu2vu5ymJnNGDhAtj0ERTMD398EcpEPlHhT0QkhhQXF5sOQURijPKKiLhNeUVE3Ka8IiJuUk5pBjod76zn9+1ZMPcK09FEj/43wsT5kJLtPA6UO99tG749G6YfCd3Pg5PWQ+sRxsIUFf5ERERERERERERERKQ5aTkYhj0FQx81HUn02HM05OqX4L0usGMVBCshpSu06AzJWZDU1liI4tC4VRERERERERERERERaV56XmI6guhVVQxxKc7Un94EOOTR3Wv/2UFY9TyUb4N+fzMbZzOlEX+GWZZlOgQRiRGWZZGdna28IiKuUV4REbcpr4iI25RXRMRNyiki9XTAH+GElZDcfvc2j9f5bnlg+VOw5CEIVJiJL4KYyCeWbdt22I8qFBUV4fP58Pv9pKenmw5HRERERERERERERESk6fxLoEUHiFftw0QtSCP+DAsEAqZDEJEYEQgEWLJkifKKiLhGeUVE3Ka8IiJuU14RETcpp4i4xNdHRb9fmMgnKvyJiMSQsrIy0yGISIxRXhERtymviIjblFdExE3KKSIuKdsCS/8NJetNR9LsqPAnIiIiIiIiIiIiIiIi7tn2Hcz7C6x91XQkzU6c6QBEREREREREREREREQkhrSfAKNfhY6/MR1Js6PCn2EejwZdiog7PB4P3bt3V14REdcor4iI25RXRMRtyisi4iblFBEXeROhy5mmozDORD5RBjPMsizTIYhIjLAsi/T0dOUVEXGN8oqIuE15RUTcprwiIm5SThEJgdINsP1701EYYyKfqPBnWCAQMB2CiMSIQCBATk6O8oqIuEZ5RUTcprwiIm5TXhERNymniLgsWAVTDoQ5l5iOxBgT+URTfYqIxBCdmIqI25RXRMRtyisi4jblFRFxk3KKiIs8cXDg3ZDUBmwbNJo2LFT4ExEREREREREREREREfcdcKnpCJodTfUpIiIiIiIiIiIiIiIioVOYAwtuMx1Fs6DCn2Eej94CEXGHx+Ohd+/eyisi4hrlFRFxm/KKiLhNeUVE3KScIhJCC26FhXeBf7HpSMLKRD7RVJ8iIjEkISHBdAgiEmOUV0TEbcorIuI25RURcZNyikiIDHsadqwAX1/TkcQ83bpgWDAYNB2CiMSIYDBITk6O8oqIuEZ5RUTcprwiIm5TXhERNymniIRQUmtoPcJ0FGFnIp+o8CciIiIiIiIiIiIiIiKhl/cFTDsMyraYjiRmqfAnIiIiIiIiIiIiIiIioVeeD/lzYdts05HELK3xJyIiIiIiIiIiIiIiIqGXfTK0WQPJ7UxHErMs27Zt00E0R0VFRfh8PgoLC/H5fKbDEZEYYNs2wWAQj8eDZVmmwxGRGKC8IiJuU14REbcpr4iIm5RTRMRtfr+fjIwM/H4/6enpYTmmpvoUEYkhFRUVpkMQkRijvCIiblNeERG3Ka+IiJuUU0TCZP3bzlp/dtB0JDFHhT/DgkH9UouIO4LBIEuXLlVeERHXKK+IiNuUV0TEbcorIuIm5RSRMNo2Gwp+hB2rTEcSUibyiQp/IiIiIiIiIiIiIiIiEj79b4BTt0FaT9ORuG/Hatg4BapKjBxehT8REREREREREREREREJn4QM8CaajiI0cqfDjN/A9jlGDq/Cn4hIDPF6vaZDEJEYo7wiIm5TXhERtymviIiblFNEwqhkLSy6H3bmhu+Y4VhTsOdFcMoWaHNo6I9VC8u2bdvIkZu5oqIifD4ffr+f9PR00+GIiIiIiIiIiIiIiIiEz5qXYea5MOw/TrEslGwb5lwCJevgyE9Ce6w9mKgFacSfYaq7iohbbNumqKhIeUVEXKO8IiJuU14REbcpr4iIm5RTRMKsw7EwZip0+23ojrHr/7NlQWCn81VVCoEy2Jnn/vEKc5yCZoX/l8OHP5+o8GdYMBiGYaUi0iwEg0FWrVqlvCIirlFeERG3Ka+IiNuUV0TETcopImGWkAEdjgFvUmjaX/ksfD5u9/Sew5+Do78COwAf9oPZF7h/zNX/c0YxlqwGzNSA4sJ+RBERERERERERERERERGAomXgiYfUbu61aduw5SsoWgo7N0GLTuBNcJ6LT4MOEyG1u1MUtFwcI9f7/yBjAGQMcq/NBlLhT0RERERERERERERERMLPvwQ+6gtdfwejXnSvXcuCkZMhULG74LenQx5z71h7atEBuv0uNG3Xk6b6FBGJIUlJIRoWLyLNlvKKiLhNeUVE3Ka8IiJuUk4RCbP03tDrChhw497PbZ0FW75ueJvBwO61/Wor+u2paieUbWv4MWpTuhEqCt1pqwksWyuVGlFUVITP58Pv95Oenm46HBEREREREREREREREbMW3AJpB0DH4+GdDuDrB8fMbVgbiybBhvdh9CuQ0rnu/Uo3wId9oPv5MPTfTYsbYPbFsOo5OGkjJGcBZmpBGvFnmBaKFRG3BINBtm/frrwiIq5RXhERtymviIjblFdExE3KKSKGlW6EFU/Dymch3gfDn4GRjZj+s8IPZXmQlLXv/ZI7QoffQMuDGhfvr7U7CnpeWl30AzM1IBX+DNOASxFxi23brF+/XnlFRFyjvCIiblNeERG3Ka+IiJuUU0QMS24Ph70NYz9x1ujreo4z4m+Xsq3wxbEw9//23c7gu+G4Rfuf5tOy4NBXoccFjY+5YAEsedj5uetZcMjjNZ42kU9U+BMRERERERERERERERGzLA+0GQXexN3bbBs2fQz+ReCJg81TIb1X7a+3g/Vf22+v1zaiQGfbMPsC+OGvULal4a8PERX+REREREREREREREREJPIULYYvJ8LPd0BCSzi9GHpd7jxXUViz4LbhfZh6IOTPa9gx5lwGn46q/bny/LqLgpbljE4c9yUktW3YMUNIhT8RkRiSlpZmOgQRiTHKKyLiNuUVEXGb8oqIuEk5RSTC+PrBIU/A4EnO4/hU53swAF+dBN+c4fwMUJYLOzdBcoeGHcMOOKMJq0pqbi9cCNNGwY6VNbcHq8C/2Pk5sRW0Gd2w44WYZWvCYiOKiorw+Xz4/X7S09NNhyMiIiIiIiIiIiIiIhI95v4fxLWAQf8AT7yzLVBec6rQ+rCDzjSjv5ZzOyyaBEd/DZlDdm+fdxWseAqO+gJaD99n0yZqQRrxZ1gwGDQdgojEiGAwSG5urvKKiLhGeUVE3Ka8IiJuU14RETcpp4hEmSEPw+B7nKJf9dp+DSz6we6iX6C85vaBt8L4mU7Rz78ItnzlbO96DnQ8HloetN+mTeQTFf4M04BLEXGLbdvk5uYqr4iIa5RXRMRtyisi4jblFRFxk3KKSJSxLOd7yXp4xQPr3258W2VbYdpoWPxgze0tBzsFwc/Hw+oXnWlFWx0Ch74G3oT9Nmsin6jwJyIiIiIiIiIiIiIiItEpfx5YcZDYuvFtWF4I7ITKItixBr45c/c6ft5EGP4M7MzbXWyMYHGmAxARERERERERERERERFplOyT4IwdjZvmc5fETJgwB+JSYNljsO516Hkx+Po6z3c4xvmKAir8GWZFQXVYRKKDZVlkZmYqr4iIa5RXRMRtyisi4jblFRFxk3KKSBRrStFvl7gU53uvy6HdWEjv2+QmTeQTy9aExUYUFRXh8/nw+/2kp6ebDkdERERERERERERERERcZKIWpDX+DAsGg6ZDEJEYEQwGWbdunfKKiLhGeUVE3Ka8IiJuU14RETcpp4iI20zkExX+DNOASxFxi23b5OfnK6+IiGuUV0TEbcorIuI25RURcZNyioi4zUQ+UeFPREREREREREREREREJAao8CciIiIiIiIiIiIiIiISA1T4M8yyLNMhiEiMsCyLrKws5RURcY3yioi4TXlFRNymvCIiblJOERG3mcgnlq0Ji40oKirC5/Ph9/tJT083HY6IiIiIiIiIiIiIiIi4yEQtSCP+DAsEAqZDEJEYEQgEWLlypfKKiLhGeUVE3Ka8IiJuU14RETcpp4iI20zkExX+RERiSHFxsekQRCTGKK+IiNuUV0TEbcorIuIm5RQRiXYq/ImIiIiIiIiIiIiIiIjEABX+RERERERERERERERERGKACn+GWZZlOgQRiRGWZZGdna28IiKuUV4REbcpr4iI25RXRMRNyiki4jYT+SQu7EeUGjwe1V5FxB0ej4dWrVqZDkNEYojyioi4TXlFRNymvCIiblJOERG3magBqepkWCAQMB2CiMSIQCDAkiVLlFdExDXKKyLiNuUVEXGb8oqIuEk5RUTcZiKfqPAnIhJDysrKTIcgIjFGeUVE3Ka8IiJuU14RETcpp4hItFPhT0RERERERERERERERCQGqPAnIiIiIiIiIiIiIiIiEgNU+DPMxMKOIhKbPB4P3bt3V14REdcor4iI25RXRMRtyisi4iblFBFxm4l8Ehf2I0oNlmWZDkFEYoRlWaSnp5sOQ0RiiPKKiLhNeUVE3Ka8IiJuUk4REbeZqAHp1gXDAoGA6RBEJEYEAgFycnKUV0TENcorIuI25RURcZvyioi4STlFRNxmIp+o8CciEkN0YioiblNeERG3Ka+IiNuUV0TETcopIhLtVPgTERERERERERERERERiQEq/ImIiIiIiIiIiIiIiIjEAMu2bdt0EM1RUVERPp+PwsJCfD6f6XBEJAbYtk1ZWRlJSUlGFo0VkdijvCIiblNeERG3Ka+IiJuUU0TEbX6/n4yMDPx+P+np6WE5pkb8iYjEkISEBNMhiEiMUV4REbcpr4iI25RXRMRNyikiEu1U+DMsGAyaDkFEYkQwGCQnJ0d5RURco7wiIm5TXhERtymviIiblFNExG0m8okKfyIiIiIiIiIiIiIiIiIxQIU/ERERERERERERERERkRigwp+IiIiIiIiIiIiIiIhIDLBs27ZNB9EcFRUV4fP5KCwsxOfzmQ5HRGKAbdsEg0E8Hg+WZZkOR0RigPKKiLhNeUVE3Ka8IiJuUk4REbf5/X4yMjLw+/2kp6eH5Zga8SciEkMqKipMhyAiMUZ5RUTcprwiIm5TXhERNymniEi0U+HPsGAwaDoEEYkRwWCQpUuXKq+IiGuUV0TEbcorIuI25RURcZNyioi4zUQ+UeFPREREREREREREREREJAao8CciIiIiIiIiIiIiIiISA6Ki8LdmzRouvPBCunXrRnJyMj169ODWW2/da77lBQsWcNhhh5GUlER2djb33XffXm298cYb9OnTh6SkJAYOHMiUKVNqPG/bNrfccgvt27cnOTmZcePGsXz58hr75Ofnc+6555Kenk5GRgYXXnghO3bscL/jIiIN5PV6TYcgIjFGeUVE3Ka8IiJuU14RETcpp4hItIuKwt+SJUsIBoM89dRTLFy4kIceeognn3ySv//979X7FBUVMX78eLp06cK8efO4//77ue2223j66aer95k5cyZnn302F154IT/88AMnnXQSJ510Ej///HP1Pvfddx+PPPIITz75JLNnzyYlJYUJEyZQVlZWvc+5557LwoULmTZtGh9++CFfffUVl1xySaP6pj8kIuIWr9fLwIEDlVdExDXKKyLiNuUVEXGb8oqIuEk5RUTcZiKfWLZt22E/qgvuv/9+nnjiCVatWgXAE088wY033khubi4JCQkAXH/99bz77rssWbIEgDPPPJOSkhI+/PDD6nZGjBjB4MGDefLJJ7Ftmw4dOnDNNddw7bXXAuD3+2nXrh0vvPACZ511FosXL6Zfv358//33DB06FICPP/6YY489lg0bNtChQ4da4y0vL6e8vLz6cVFREdnZ2Wzfvh2fzweAZVl4PB6CwSB7vi27tgcCgRpt1rXd4/FgWVat22HvxSTr2u71erFtu9btv46xru3qk/qkPoWvT1VVVRQXF5OWloZlWTHRp1h8n9Qn9Sma+mTbdnVe2XWiGu19qk/s6pP6pD6Frk+78kpGRkadfY22Pu0ZY6y8T+qT+hRNfbJtm9LSUlJSUmqNMRr7VNd29Ul9Up9C3yeAkpISWrRogWVZMdGnWHyf1Cf1KZr6VFRUREZGBn6/n/T0dMIhLixHCQG/309mZmb141mzZnH44YdXF/0AJkyYwKRJkygoKKBly5bMmjWLq6++ukY7EyZM4N133wVg9erV5ObmMm7cuOrnfT4fw4cPZ9asWZx11lnMmjWLjIyM6qIfwLhx4/B4PMyePZuTTz651njvuecebr/99r22L1y4kLS0NAAyMzPp3LkzGzZsID8/v3qfrKwssrKyWLNmDcXFxdXbs7OzadWqFcuXL68xIrF79+6kp6ezaNGiGr9ovXv3JiEhgZycnBoxDBw4kIqKCpYuXVq9bdfdLcXFxdXFVYCkpCT69OlDQUEB69evr96elpZGjx492LJlC7m5udXb1Sf1SX0KX59Wr17NunXryMzMxLKsmOhTLL5P6pP6FE19qqqqIj8/n8zMTPr06RMTfYrF90l9Up+iqU+2bVNQUMARRxxBSUlJTPQJYu99Up/Up2jqU0pKCiUlJbRt25YtW7bERJ9i8X1Sn9SnaOnTrlyyK7fEQp9i8X1Sn9SnaOrTrwuQ4RCVI/5WrFjBkCFDeOCBB7j44osBGD9+PN26deOpp56q3m/RokX079+fRYsW0bdvXxISEpg8eTJnn3129T6PP/44t99+O3l5ecycOZPRo0ezadMm2rdvX73PGWecgWVZvPbaa9x9991Mnjy5xi8TOH8Ubr/9di677LJaY65rxN/WrVtp2bIloMq4+qQ+qU9N61NFRQULFy6kf//+eL3emOhTLL5P6pP6FE19CgQC1XklPj4+JvpUn9jVJ/VJfQpdn3bllUGDBlXHE+192jPGWHmf1Cf1KZr6FAgEqq//7Gov2vtU13b1SX1Sn0Lfp2AwyMKFC+nXrx9er3ev/aOxT7H4PqlP6lM09amwsJDMzMzmM+Lv+uuvZ9KkSfvcZ/HixfTp06f68caNGznmmGM4/fTTq4t+0SAxMZHExMS9tnu93hp/RGD3L2xt+4Z7u2VZtW6vK8aGblef1Ke6tqtPjevTrmPseZxo71O4t6tP6pNbMTZ0e6T2aVdclmU1uJ1I7VNTtqtP6pNbMTZ0eyz1aVc+iaU+7aI+qU9uxdjQ7eqTs70h+0dLnxqyXX1Sn+rarj41rk+1tR/tfQr3dvVJfXIrxoZuj7Q+hZvRwt8111zDeeedt899unfvXv3zpk2bGDt2LKNGjeLpp5+usV9WVhZ5eXk1tu16nJWVtc999nx+17Y9R/zl5eUxePDg6n32nDoCqJ4Ga9frRURMSUpKMh2CiMQY5RURcZvyioi4TXlFRNyknCIi0a72cmWYtGnThj59+uzza9eafRs3bmTMmDEMGTKE559/fq9K68iRI/nqq6+orKys3jZt2jR69+5dPZXmyJEjmT59eo3XTZs2jZEjRwLQrVs3srKyauxTVFTE7Nmzq/cZOXIkhYWFzJs3r3qfzz//nGAwyPDhwxv8bxApFWARiX5er5c+ffoor4iIa5RXRMRtyisi4jblFRFxk3KKiLjNRD4xWvirr11Fv86dO/PAAw+wdetWcnNzayy2eM4555CQkMCFF17IwoULee211/jXv/7F1Vdf/f/t3XtwlNX9x/HPJiEh5E4iuQEB5GpNKaiDYahWjUkobbg4UhkNyFgYFKxQQEorgdaOBK2DolMCHSiRWnVsgbYgCUEliKZcAjGBagSKBCUBm5CEECCXPb8/HJ5flksS8GFX1vdrZmeS55yc5zzJzCdnn++cZ60+Tz31lHJzc/Xiiy/q008/1aJFi7Rnzx7NmDFD0tfbP2fOnKnf//73+uc//6nS0lJNnDhRcXFxGjNmjCRp0KBBSktL05QpU7Rr1y59+OGHmjFjhh566CHFxcVd9bVd/JxZALhWTqdTVVVV5AoA25ArAOxGrgCwG7kCwE5kCgC7eSJPPPqoz47Kz8/XoUOHdOjQIXXv3t2l7cIHK4aFhWnLli2aPn26brvtNkVFRSkzM1NTp061+g4fPlx//etf9cwzz+jXv/61+vXrpw0bNujWW2+1+jz99NM6c+aMpk6dqpqaGo0YMUK5ubkuW7xff/11zZgxQ/fdd598fHz0wAMPaNmyZdd0bRd/kCQAXCtjjI4dO6bw8HBPTwWAlyBXANiNXAFgN3IFgJ3IFAB280QNyGGoPHlEXV2dwsLCVF1dbT2KFAC+iZaWFpWWlioxMZFHUgCwBbkCwG7kCgC7kSsA7ESmALDbqVOn1LVrV9XW1io0NNQt57whHvUJAAAAAAAAAAAAoG0U/gDAi4SEhHh6CgC8DLkCwG7kCgC7kSsA7ESmALjR8ahPD7nwqE93bu8EAAAAAAAAAACAe3iiFsSOPw9zOp2engIAL+F0OlVZWUmuALANuQLAbuQKALuRKwDsRKYAsJsn8oTCn4ex4RKAXYwxqqysJFcA2IZcAWA3cgWA3cgVAHYiUwDYzRN5QuEPAAAAAAAAAAAA8AIU/gAAAAAAAAAAAAAvQOHPwxwOh6enAMBLOBwOde3alVwBYBtyBYDdyBUAdiNXANiJTAFgN0/kicPwwGKPqKurU1hYmGpraxUaGurp6QAAAAAAAAAAAMBGnqgFsePPw5xOp6enAMBLOJ1OlZeXkysAbEOuALAbuQLAbuQKADuRKQDs5ok8ofDnYWy4BGAXY4yqq6vJFQC2IVcA2I1cAWA3cgWAncgUAHbzRJ5Q+AMAAAAAAAAAAAC8gJ+nJ/BddaHKW1dXJ19fXw/PBoA3aGlpUX19PbkCwDbkCgC7kSsA7EauALATmQLAbnV1dZLcu/OPwp+HVFVVSZJ69erl2YkAAAAAAAAAAADguqmqqlJYWJhbzkXhz0O6du0qSSovL3fbHxuAd6urq1OPHj107NgxhYaGeno6ALwAuQLAbuQKALuRKwDsRKYAsFttba169uxp1YTcgcKfh/j4fP3ximFhYfwTAWCr0NBQcgWArcgVAHYjVwDYjVwBYCcyBYDdLtSE3HIut50JAAAAAAAAAAAAwHVD4Q8AAAAAAAAAAADwAhT+PCQgIEALFy5UQECAp6cCwEuQKwDsRq4AsBu5AsBu5AoAO5EpAOzmiVxxGGOM284GAAAAAAAAAAAA4Lpgxx8AAAAAAAAAAADgBSj8AQAAAAAAAAAAAF6Awh8AAAAAAAAAAADgBSj8AQAAAAAAAAAAAF6Awt83cPr0ac2cOVMJCQkKDAzU8OHDtXv3bqvdGKPMzEzFxsYqMDBQycnJOnjwoMsY1dXVevjhhxUaGqrw8HA99thjqq+vd+lTUlKiH/7wh+rcubN69Oih559/3i3XB8D92suVdevWKSUlRZGRkXI4HCouLr5kjHPnzmn69OmKjIxUcHCwHnjgAZ04ccKlT3l5uUaNGqUuXbqoW7dumjt3rpqbm6/35QHwgLZypampSfPmzVNiYqKCgoIUFxeniRMn6vjx4y5jsF4B0Fp765VFixZp4MCBCgoKUkREhJKTk7Vz506XMcgVAK21lyutTZs2TQ6HQy+99JLLcXIFQGvt5cqjjz4qh8Ph8kpLS3MZg1wB0FpH1iuffPKJ0tPTFRYWpqCgIN1xxx0qLy+32t1135bC3zfw85//XPn5+Vq7dq1KS0uVkpKi5ORkffnll5Kk559/XsuWLVN2drZ27typoKAgpaam6ty5c9YYDz/8sA4cOKD8/Hxt3LhR27dv19SpU632uro6paSkKCEhQUVFRXrhhRe0aNEirVy50u3XC+D6ay9Xzpw5oxEjRmjJkiVXHGPWrFn617/+pbffflsFBQU6fvy4xo0bZ7W3tLRo1KhRamxs1EcffaScnBytWbNGmZmZ1/36ALhfW7nS0NCgvXv3asGCBdq7d6/WrVunsrIypaenu4zBegVAa+2tV/r3769XX31VpaWl2rFjh3r16qWUlBR99dVX1hjkCoDW2suVC9avX69///vfiouLu2QMcgVAax3JlbS0NFVUVFivN954w2UMcgVAa+3lyuHDhzVixAgNHDhQ27ZtU0lJiRYsWKDOnTtbY7jtvq3BNWloaDC+vr5m48aNLseHDh1qfvOb3xin02liYmLMCy+8YLXV1NSYgIAA88YbbxhjjPnPf/5jJJndu3dbfTZv3mwcDof58ssvjTHG/PGPfzQRERHm/PnzVp958+aZAQMGXM/LA+AB7eVKa0eOHDGSzL59+1yO19TUmE6dOpm3337bOvbJJ58YSaawsNAYY8w777xjfHx8TGVlpdVn+fLlJjQ01CVrANz4riZXLti1a5eRZI4ePWqMYb0CwNW15Eptba2RZLZu3WqMIVcAuOpornzxxRcmPj7e7N+/3yQkJJilS5dabeQKgNY6kiuTJk0yo0ePvuIY5AqA1jqSKz/72c/MI488csUx3Hnflh1/16i5uVktLS0u1VpJCgwM1I4dO3TkyBFVVlYqOTnZagsLC9OwYcNUWFgoSSosLFR4eLhuv/12q09ycrJ8fHysR+EUFhbqrrvukr+/v9UnNTVVZWVlOnXq1PW8RABu1l6udERRUZGamppcsmfgwIHq2bOnS/YkJiYqOjra6pOamqq6ujodOHDAhisB8G1xLblSW1srh8Oh8PBwSaxXALi62lxpbGzUypUrFRYWpsGDB0siVwC46kiuOJ1OZWRkaO7cufre9753yRjkCoDWOrpe2bZtm7p166YBAwbo8ccfV1VVldVGrgBorb1ccTqd2rRpk/r376/U1FR169ZNw4YN04YNG6y+7rxvS+HvGoWEhCgpKUnPPvusjh8/rpaWFv3lL39RYWGhKioqVFlZKUkuf6AL319oq6ysVLdu3Vza/fz81LVrV5c+lxvjQhsA79FernREZWWl/P39rRv2F1ycPeQK8N1wtbly7tw5zZs3TxMmTFBoaKgk1isAXHU0VzZu3Kjg4GB17txZS5cuVX5+vqKioiSRKwBcdSRXlixZIj8/P/3iF7+47BjkCoDWOpIraWlpeu211/Tuu+9qyZIlKigo0MiRI9XS0iKJXAHgqr1cOXnypOrr65WVlaW0tDRt2bJFY8eO1bhx41RQUCDJvfdtKfx9A2vXrpUxRvHx8QoICNCyZcs0YcIE+fjwawVwbcgVAHbraK40NTVp/PjxMsZo+fLlHpotgBtBR3LlnnvuUXFxsT766COlpaVp/PjxOnnypAdnDeDbrK1cKSoq0ssvv6w1a9bI4XB4eqoAbhDtrVceeughpaenKzExUWPGjNHGjRu1e/dubdu2zbMTB/Ct1VauOJ1OSdLo0aM1a9Ys/eAHP9CvfvUr/eQnP1F2drbb58qd5G/g5ptvVkFBgerr63Xs2DHt2rVLTU1N6tOnj2JiYiRJJ06ccPmZEydOWG0xMTGXvPltbm5WdXW1S5/LjXGhDYB3aStXOiImJkaNjY2qqalxOX5x9pArwHdHR3LlQtHv6NGjys/Pt3b7SaxXAFyqI7kSFBSkvn376s4779SqVavk5+enVatWSSJXAFyqrVz54IMPdPLkSfXs2VN+fn7y8/PT0aNHNXv2bPXq1UsSuQLgUld7f6VPnz6KiorSoUOHJJErAC7VVq5ERUXJz89Pt9xyi8vPDBo0SOXl5ZLce9+Wwp8NgoKCFBsbq1OnTikvL0+jR49W7969FRMTo3fffdfqV1dXp507dyopKUmSlJSUpJqaGhUVFVl93nvvPTmdTg0bNszqs337djU1NVl98vPzNWDAAEVERLjpCgG42+VypSNuu+02derUySV7ysrKVF5e7pI9paWlLgvYCzf6L/7nBMB7XClXLhT9Dh48qK1btyoyMtLl51ivALiSq1mvOJ1OnT9/XhK5AuDKLpcrGRkZKikpUXFxsfWKi4vT3LlzlZeXJ4lcAXBlHV2vfPHFF6qqqlJsbKwkcgXAlV0uV/z9/XXHHXeorKzMpe9nn32mhIQESW6+b2twzXJzc83mzZvNf//7X7NlyxYzePBgM2zYMNPY2GiMMSYrK8uEh4ebf/zjH6akpMSMHj3a9O7d25w9e9YaIy0tzQwZMsTs3LnT7Nixw/Tr189MmDDBaq+pqTHR0dEmIyPD7N+/37z55pumS5cuZsWKFW6/XgDXX3u5UlVVZfbt22c2bdpkJJk333zT7Nu3z1RUVFhjTJs2zfTs2dO89957Zs+ePSYpKckkJSVZ7c3NzebWW281KSkppri42OTm5pqbbrrJzJ8/3+3XC+D6aytXGhsbTXp6uunevbspLi42FRUV1uv8+fPWGKxXALTWVq7U19eb+fPnm8LCQvP555+bPXv2mMmTJ5uAgACzf/9+awxyBUBr7b0PulhCQoJZunSpyzFyBUBrbeXK6dOnzZw5c0xhYaE5cuSI2bp1qxk6dKjp16+fOXfunDUGuQKgtfbWK+vWrTOdOnUyK1euNAcPHjSvvPKK8fX1NR988IE1hrvu21L4+wbeeust06dPH+Pv729iYmLM9OnTTU1NjdXudDrNggULTHR0tAkICDD33XefKSsrcxmjqqrKTJgwwQQHB5vQ0FAzefJkc/r0aZc+H3/8sRkxYoQJCAgw8fHxJisryy3XB8D92suVP//5z0bSJa+FCxdafc6ePWueeOIJExERYbp06WLGjh3rUhg0xpjPP//cjBw50gQGBpqoqCgze/Zs09TU5K7LBOBGbeXKkSNHLpspksz7779vjcF6BUBrbeXK2bNnzdixY01cXJzx9/c3sbGxJj093ezatctlDHIFQGvtvQ+62OUKf+QKgNbaypWGhgaTkpJibrrpJtOpUyeTkJBgpkyZYiorK13GIFcAtNaR9cqqVatM3759TefOnc3gwYPNhg0bXNrddd/WYYwxV7mTEQAAAAAAAAAAAMC3DJ/xBwAAAAAAAAAAAHgBCn8AAAAAAAAAAACAF6DwBwAAAAAAAAAAAHgBCn8AAAAAAAAAAACAF6DwBwAAAAAAAAAAAHgBCn8AAAAAAAAAAACAF6DwBwAAAAAAAAAAAHgBCn8AAAAAAAAAAACAF6DwBwAAAADfQY8++qjGjBnj6Wlcsxt9/gAAAABwPfh5egIAAAAAAHs5HI422xcuXKiXX35Zxhg3zej/bdu2Tffcc49OnTql8PBwt58fAAAAALwZhT8AAAAA8DIVFRXW12+99ZYyMzNVVlZmHQsODlZwcLAnpgYAAAAAuI541CcAAAAAeJmYmBjrFRYWJofD4XIsODj4kkdl/uhHP9KTTz6pmTNnKiIiQtHR0frTn/6kM2fOaPLkyQoJCVHfvn21efNml3Pt379fI0eOVHBwsKKjo5WRkaH//e9/HZ7rmjVrFB4erry8PA0aNEjBwcFKS0tzKV62tLTol7/8pcLDwxUZGamnn376kt2KTqdTixcvVu/evRUYGKjBgwfrb3/7myTJGKPk5GSlpqZaP1ddXa3u3bsrMzPzan+9AAAAAPCtReEPAAAAACBJysnJUVRUlHbt2qUnn3xSjz/+uB588EENHz5ce/fuVUpKijIyMtTQ0CBJqqmp0b333qshQ4Zoz549ys3N1YkTJzR+/PirOm9DQ4P+8Ic/aO3atdq+fbvKy8s1Z84cq/3FF1/UmjVrtHr1au3YsUPV1dVav369yxiLFy/Wa6+9puzsbB04cECzZs3SI488ooKCAjkcDuXk5Gj37t1atmyZJGnatGmKj4+n8AcAAADAq/CoTwAAAACAJGnw4MF65plnJEnz589XVlaWoqKiNGXKFElSZmamli9frpKSEt1555169dVXNWTIED333HPWGKtXr1aPHj302WefqX///h06b1NTk7Kzs3XzzTdLkmbMmKHf/e53VvtLL72k+fPna9y4cZKk7Oxs5eXlWe3nz5/Xc889p61btyopKUmS1KdPH+3YsUMrVqzQ3Xffrfj4eK1YsUITJ05UZWWl3nnnHe3bt09+frwtBgAAAOA9eIcDAAAAAJAkff/737e+9vX1VWRkpBITE61j0dHRkqSTJ09Kkj7++GO9//77l/28wMOHD3e48NelSxer6CdJsbGx1jlqa2tVUVGhYcOGWe1+fn66/fbbrcd2Hjp0SA0NDbr//vtdxm1sbNSQIUOs7x988EGtX79eWVlZWr58ufr169eh+QEAAADAjYLCHwAAAABAktSpUyeX7x0Oh8sxh8Mh6evP05Ok+vp6/fSnP9WSJUsuGSs2NvYbnffiz/BrS319vSRp06ZNio+Pd2kLCAiwvm5oaFBRUZF8fX118ODBDo8PAAAAADcKCn8AAAAAgGsydOhQ/f3vf1evXr2u2yMzw8LCFBsbq507d+quu+6SJDU3N6uoqEhDhw6VJN1yyy0KCAhQeXm57r777iuONXv2bPn4+Gjz5s368Y9/rFGjRunee++9LvMGAAAAAE/w8fQEAAAAAAA3punTp6u6uloTJkzQ7t27dfjwYeXl5Wny5MlqaWmx7TxPPfWUsrKytGHDBn366ad64oknVFNTY7WHhIRozpw5mjVrlnJycnT48GHt3btXr7zyinJyciR9vRtw9erVev3113X//fdr7ty5mjRpkk6dOmXbPAEAAADA0yj8AQAAAACuSVxcnD788EO1tLQoJSVFiYmJmjlzpsLDw+XjY9/bzdmzZysjI0OTJk1SUlKSQkJCNHbsWJc+zz77rBYsWKDFixdr0KBBSktL06ZNm9S7d2999dVXeuyxx7Ro0SJrl+Bvf/tbRUdHa9q0abbNEwAAAAA8zWGu5oMTAAAAAAAAAAAAAHwrseMPAAAAAAAAAAAA8AIU/gAAAAAAAAAAAAAvQOEPAAAAAAAAAAAA8AIU/gAAAAAAAAAAAAAvQOEPAAAAAAAAAAAA8AIU/gAAAAAAAAAAAAAvQOEPAAAAAAAAAAAA8AIU/gAAAAAAAAAAAAAvQOEPAAAAAAAAAAAA8AIU/gAAAAAAAAAAAAAvQOEPAAAAAAAAAAAA8AL/B5dVLZann5lPAAAAAElFTkSuQmCC\n" + }, + "metadata": {} + }, + { + "output_type": "stream", + "name": "stdout", + "text": [ + "Plots have been generated and saved with the corrected original data.\n" + ] + } + ] + }, + { + "cell_type": "markdown", + "source": [ + "# Finetuned Model Predictions" + ], + "metadata": { + "id": "_KxBniHGqS3M" + } + }, + { + "cell_type": "code", + "source": [ + "import apache_beam as beam\n", + "from apache_beam.options.pipeline_options import PipelineOptions\n", + "from apache_beam.pvalue import AsDict, AsSingleton\n", + "from apache_beam.transforms.periodicsequence import PeriodicImpulse\n", + "import logging\n", + "import os\n", + "import json\n", + "import timesfm\n", + "from apache_beam.utils.timestamp import Timestamp\n", + "import csv\n", + "from apache_beam.ml.inference.base import RunInference\n", + "from apache_beam.ml.inference.utils import WatchFilePattern\n", + "import typing\n", + "from google.colab import userdata\n", + "import apache_beam.transforms.window as window\n", + "\n", + "logging.getLogger().setLevel(logging.INFO)\n", + "\n", + "# --- Pipeline Configuration ---\n", + "PROJECT_ID = os.environ.get(\"GCP_PROJECT\", \"apache-beam-testing\")\n", + "REGION = os.environ.get(\"GCP_REGION\", \"us-central1\")\n", + "TEMP_LOCATION = \"gs://apache-beam-testing-temp/timesfm_anomaly_detection/temp\"\n", + "STAGING_LOCATION = \"gs://apache-beam-testing-temp/timesfm_anomaly_detection/staging\"\n", + "FINETUNED_MODEL_BUCKET = \"apache-beam-testing-temp\"\n", + "FINETUNED_MODEL_PREFIX = \"timesfm_anomaly_detection/finetuned-models/timesfm/checkpoints\"\n", + "\n", + "# --- Model & Window Parameters ---\n", + "CONTEXT_LEN = 512\n", + "HORIZON_LEN = 128\n", + "WINDOW_SIZE = CONTEXT_LEN + HORIZON_LEN\n", + "SLIDE_INTERVAL = HORIZON_LEN\n", + "EXPECTED_INTERVAL = 1\n", + "# go to checkpoints bucket and select the correct model path\n", + "INITIAL_MODEL = \"gs://apache-beam-testing-temp/timesfm_anomaly_detection/finetuned-models/timesfm/checkpoints/timesfm_finetuned_20250814192006.pth\"\n", + "\n", + "MODEL_CHECK_INTERVAL_SECONDS = 10 # Check for a new model every 5 seconds\n", + "FINETUNING_BATCH_SIZE = 7680 # 9600 # make larger later. minimum is WINDOW_SIZE for validation and training\n", + "FINETUNE_CONFIG = FinetuningConfig(\n", + " batch_size=128,\n", + " num_epochs=5,\n", + " learning_rate=1e-4,\n", + " use_wandb=False,\n", + " freq_type=0, # should change based on your data\n", + " log_every_n_steps=10,\n", + " val_check_interval=0.5,\n", + " use_quantile_loss=True\n", + " )\n", + "\n", + "\n", + "options = PipelineOptions([\n", + " \"--streaming\",\n", + " \"--environment_type=LOOPBACK\",\n", + " \"--runner=PrismRunner\",\n", + " \"--logging_level=INFO\",\n", + " \"--job_server_timeout=3600\"\n", + "])\n", + "\n", + "\n", + "\n", + "# HParams for the model\n", + "hparams = timesfm.TimesFmHparams(\n", + " backend=\"gpu\",\n", + " per_core_batch_size=32,\n", + " horizon_len=HORIZON_LEN,\n", + " context_len=CONTEXT_LEN,\n", + ")\n", + "model_handler = DynamicTimesFmModelHandler(model_uri=INITIAL_MODEL, hparams=hparams)\n", + "\n", + "def print_and_pass_through(label):\n", + " def logger(element):\n", + " print(f\"--- {label} --- \\nELEMENT: %s\", element)\n", + " return element\n", + " return logger\n", + "\n", + "\n", + "class CustomJsonEncoder(json.JSONEncoder):\n", + " \"\"\"A custom JSON encoder that knows how to handle Beam's Timestamp objects.\"\"\"\n", + " def default(self, obj):\n", + " if isinstance(obj, Timestamp):\n", + " # Convert Timestamp to a standard, readable ISO 8601 string format\n", + " return obj.micros // 1e6\n", + " # For all other types, fall back to the default behavior\n", + " if isinstance(obj, np.integer):\n", + " return int(obj)\n", + "\n", + " # 3. Handle NumPy float types (this will fix your float32 error)\n", + " if isinstance(obj, np.floating):\n", + " return float(obj)\n", + "\n", + " # 4. Handle NumPy arrays\n", + " if isinstance(obj, np.ndarray):\n", + " return obj.tolist()\n", + "\n", + " # For all other types, fall back to the default behavior\n", + " return super().default(obj)\n", + " return json.JSONEncoder.default(self, obj)\n", + "\n", + "class WritePlotDataAndPassThrough(beam.DoFn):\n", + " \"\"\"\n", + " A DoFn that writes plotting data to a file as a side effect\n", + " and then passes the original, unmodified element downstream.\n", + " \"\"\"\n", + " def __init__(self, output_path):\n", + " self._output_path = output_path\n", + " self._file_handle = None\n", + "\n", + " def setup(self):\n", + " self._file_handle = open(self._output_path, 'a')\n", + "\n", + " def process(self, element):\n", + " _original_window, payload_dict = element\n", + "\n", + " # ✅ FIX: Use the custom encoder to handle Timestamp objects\n", + " json_record = json.dumps(payload_dict, cls=CustomJsonEncoder)\n", + " self._file_handle.write(json_record + '\\n')\n", + "\n", + " # Pass the original element through, with the Timestamp object intact\n", + " yield element\n", + "\n", + " def teardown(self):\n", + " if self._file_handle:\n", + " self._file_handle.close()\n", + "\n", + "\n", + "# =================================================================\n", + "# 1. Get Latest Model Path (Side Input) - WatchFilePattern is not\n", + "# currently supported on Prism. Uncomment the following to run\n", + "# on Dataflow\n", + "# =================================================================\n", + "# model_pattern = os.path.join(\n", + "# f\"gs://{FINETUNED_MODEL_BUCKET}\", FINETUNED_MODEL_PREFIX, \"*.pth\"\n", + "# )\n", + "\n", + "# model_metadata_pcoll = (\n", + "# \"WatchForNewModels\" >> WatchFilePattern(\n", + "# file_pattern=model_pattern,\n", + "# interval=MODEL_CHECK_INTERVAL_SECONDS\n", + "# )\n", + "# | \"PrintModelLocation\" >> beam.Map(print_and_pass_through(\"Model Location\"))\n", + "\n", + "# )\n", + "\n", + "# =================================================================\n", + "# Ingest and Window Raw Data\n", + "# =================================================================\n", + "\n", + "\n", + "windowed_data = (\n", + " PeriodicImpulse(data=input_data, fire_interval=0.01)\n", + " | \"AddKey\" >> beam.WithKeys(lambda x: 0)\n", + " | \"ApplySlidingWindow\" >> beam.ParDo(\n", + " OrderedSlidingWindowFn(window_size=WINDOW_SIZE, slide_interval=SLIDE_INTERVAL))\n", + " | \"FillGaps\" >> beam.ParDo(FillGapsFn(expected_interval=EXPECTED_INTERVAL)).with_output_types(\n", + " typing.Tuple[int, typing.Tuple[Timestamp, Timestamp, typing.List[float]]])\n", + " | \"Skip NaN Values for now\" >> beam.Filter(\n", + " lambda batch: 'NaN' not in batch[1][2])\n", + " | \"PrintWindowedData\" >> beam.Map(print_and_pass_through(\"Windowed Data\"))\n", + "\n", + ")\n", + "\n", + "# =================================================================\n", + "# Detect Anomalies using the Latest Model\n", + "# =================================================================\n", + "\n", + "inference_results = (\n", + " \"DetectAnomalies\" >> RunInference(\n", + " model_handler=model_handler,\n", + " # model_metadata_pcoll=model_metadata_pcoll\n", + " )\n", + " | \"PrintInference\" >> beam.Map(print_and_pass_through(\"Inference Results\"))\n", + ")\n", + "\n", + "\n", + "# NEW BRANCH: For plotting. It takes the payload dictionary, converts\n", + "# it to JSON, and writes it to a file.\n", + "plotting_data_output = (\n", + " \"WritePlotDataAsSideEffect\" >> beam.ParDo(\n", + " WritePlotDataAndPassThrough('plot_data_finetuned.jsonl'))\n", + ")\n", + "\n" + ], + "metadata": { + "id": "LRRg2QhPfEZr" + }, + "execution_count": 19, + "outputs": [] + }, + { + "cell_type": "code", + "source": [ + "with beam.Pipeline(options=options) as p:\n", + " (p\n", + " | windowed_data\n", + " | inference_results\n", + " | plotting_data_output\n", + " )\n" + ], + "metadata": { + "id": "qAJDAbdGqW_V" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "source": [ + "# Plot Data (After Finetuning)" + ], + "metadata": { + "id": "_fT-AjbrUOl5" + } + }, + { + "cell_type": "code", + "source": [ + "import json\n", + "import numpy as np\n", + "import matplotlib.pyplot as plt\n", + "\n", + "def plot_anomalies_and_forecast(\n", + " values_array,\n", + " all_anomalies,\n", + " all_predicted_values,\n", + " all_q20_values,\n", + " all_q30_values,\n", + " all_q70_values,\n", + " all_q80_values,\n", + " title_suffix=\"\",\n", + " x_lims=None,\n", + " min_outlier_score_for_plot=0,\n", + " context_len=512,\n", + " output_filename=\"plot.png\"\n", + "):\n", + " # The key from your file is 'outlier_score'\n", + " filtered_anomalies = [a for a in all_anomalies if a['outlier_score'] >= min_outlier_score_for_plot]\n", + " # The key from your file is 'timestamp'\n", + " anomaly_indices = [(a['timestamp'] - 128) for a in filtered_anomalies]\n", + " anomaly_values = [a['actual_value'] for a in filtered_anomalies]\n", + "\n", + " Q1 = np.nanmean([all_q20_values, all_q30_values], axis=0)\n", + " Q3 = np.nanmean([all_q70_values, all_q80_values], axis=0)\n", + " IQR = Q3 - Q1\n", + " upper_thresh = Q3 + 1.5 * IQR\n", + " lower_thresh = Q1 - 1.5 * IQR\n", + "\n", + " plt.figure(figsize=(18, 9))\n", + " # This now plots the correct original data for the horizon\n", + " plt.plot(values_array[context_len:], label='Original Time Series', color='blue', alpha=0.7, linewidth=1.5)\n", + "\n", + " plt.plot(all_predicted_values, label='Predicted Mean', color='green', linestyle='--', linewidth=1.5)\n", + " plt.plot(lower_thresh, label='Lower Threshold', color='orange', linestyle=':', linewidth=1.2)\n", + " plt.plot(upper_thresh, label='Upper Threshold', color='purple', linestyle=':', linewidth=1.2)\n", + "\n", + " plt.scatter([i - context_len for i in anomaly_indices], anomaly_values,\n", + " color='red', s=70, zorder=5,\n", + " label=f'Detected Anomalies (Score >= {min_outlier_score_for_plot:.1f})',\n", + " marker='o', edgecolors='black', linewidths=0.8)\n", + "\n", + " plt.title(f'Time Series Anomaly Detection {title_suffix}')\n", + " plt.xlabel('Time Index')\n", + " plt.ylabel('Value')\n", + " if x_lims:\n", + " plt.xlim(x_lims[0], x_lims[1])\n", + " plt.legend()\n", + " plt.grid(True, linestyle='--', alpha=0.6)\n", + " plt.tight_layout()\n", + " # plt.savefig(output_filename) # Save the plot to a file\n", + " plt.show()\n", + " plt.close() # Close the figure to free memory\n", + "\n", + "# --- Main Script Logic ---\n", + "\n", + "# 1. Read and parse the data from the Beam output file\n", + "all_window_data = []\n", + "# Make sure 'plot_data.jsonl' is in the same directory as this script\n", + "try:\n", + " with open('plot_data_finetuned.jsonl', 'r') as f:\n", + " for line in f:\n", + " # Check for empty lines that might have been added\n", + " if line.strip():\n", + " all_window_data.append(json.loads(line))\n", + "except FileNotFoundError:\n", + " print(\"Error: 'plot_data_finetuned.jsonl' not found. Please make sure the file is in the correct directory.\")\n", + " exit()\n", + "\n", + "\n", + "# 2. Sort data by timestamp to ensure the correct order\n", + "all_window_data.sort(key=lambda x: x['start_ts_micros'])\n", + "\n", + "# 3. Reconstruct the full data arrays\n", + "all_anomalies = []\n", + "all_predicted_values = []\n", + "all_q20_values = []\n", + "all_q30_values = []\n", + "all_q70_values = []\n", + "all_q80_values = []\n", + "all_actual_horizon_values = [] # This will hold the real \"blue line\" data\n", + "\n", + "for window_data in all_window_data:\n", + " all_predicted_values.extend(window_data['predicted_values'])\n", + " all_q20_values.extend(window_data['q20_values'])\n", + " all_q30_values.extend(window_data['q30_values'])\n", + " all_q70_values.extend(window_data['q70_values'])\n", + " all_q80_values.extend(window_data['q80_values'])\n", + " # Populate the list with the actual values from the file\n", + " all_actual_horizon_values.extend(window_data.get('actual_horizon_values', []))\n", + " all_anomalies.extend(window_data.get('anomalies', []))\n", + "\n", + "# 4. Convert lists to NumPy arrays\n", + "all_predicted_values = np.array(all_predicted_values)\n", + "print(len(all_predicted_values))\n", + "all_q20_values = np.array(all_q20_values)\n", + "all_q30_values = np.array(all_q30_values)\n", + "all_q70_values = np.array(all_q70_values)\n", + "all_q80_values = np.array(all_q80_values)\n", + "\n", + "# 5. Construct the `values_array` using the REAL data from your file\n", + "context_len = 512\n", + "# Create a dummy context so the array has the right shape for the plotting function.\n", + "# The first real value is used to make the context visually seamless.\n", + "if all_actual_horizon_values:\n", + " dummy_context = [all_actual_horizon_values[0]] * context_len\n", + " values_array = np.array(dummy_context + all_actual_horizon_values)\n", + "else:\n", + " # Fallback in case the file is empty or missing the actual_horizon_values key\n", + " print(\"Warning: 'actual_horizon_values' not found. The original time series plot will be empty.\")\n", + " total_len = context_len + len(all_predicted_values)\n", + " values_array = np.zeros(total_len)\n", + "\n", + "# 6. Call the plotting functions\n", + "if values_array.any():\n", + " # Plotting function for full graph\n", + " plot_anomalies_and_forecast(\n", + " values_array, all_anomalies, all_predicted_values,\n", + " all_q20_values, all_q30_values, all_q70_values, all_q80_values,\n", + " title_suffix=\"(Full Graph with Correct Data)\",\n", + " min_outlier_score_for_plot=5, # Set a score threshold\n", + " context_len=context_len,\n", + " output_filename=\"full_graph_correct.png\"\n", + " )\n", + "\n", + " # Plotting function for zoomed-in graphs - feel free to change\n", + " zoom_ranges = [(2000, 2500), (8300, 9000), (9000, 9600)]\n", + " for i, (start_idx, end_idx) in enumerate(zoom_ranges):\n", + " # Adjust x_lims for the fact that the plotted array is sliced by context_len\n", + " plot_x_start = max(0, start_idx)\n", + " plot_x_end = end_idx\n", + "\n", + " plot_anomalies_and_forecast(\n", + " values_array, all_anomalies, all_predicted_values,\n", + " all_q20_values, all_q30_values, all_q70_values, all_q80_values,\n", + " title_suffix=f\"(Zoomed In: {start_idx} to {end_idx})\",\n", + " x_lims=(plot_x_start, plot_x_end),\n", + " min_outlier_score_for_plot=5,\n", + " context_len=context_len,\n", + " output_filename=f\"zoomed_graph_correct_{i}.png\"\n", + " )\n", + " print(\"Plots have been generated and saved with the corrected original data.\")\n", + "else:\n", + " print(\"No data found to plot.\")" + ], + "metadata": { + "colab": { + "base_uri": "https://localhost:8080/", + "height": 1000 + }, + "id": "RZwpgtAr8nlD", + "outputId": "d27b08f1-1e77-4109-bfa6-c709edf4b5e8" + }, + "execution_count": 21, + "outputs": [ + { + "output_type": "stream", + "name": "stdout", + "text": [ + "9600\n" + ] + }, + { + "output_type": "display_data", + "data": { + "text/plain": [ + "
" + ], + "image/png": "iVBORw0KGgoAAAANSUhEUgAABvsAAAN5CAYAAAAmV4erAAAAOnRFWHRTb2Z0d2FyZQBNYXRwbG90bGliIHZlcnNpb24zLjEwLjAsIGh0dHBzOi8vbWF0cGxvdGxpYi5vcmcvlHJYcgAAAAlwSFlzAAAPYQAAD2EBqD+naQABAABJREFUeJzs3XdcU9f7B/BPEvYWFHGwRBFsUcRVtA5cWJWvo466cX+11m3Vap2to2rVDkfddVTrqNqqdePeCqIoKoIbcSCKEEZyf3/wS75GEgSFXOB+3q8Xrzbnntz7nOTylObJOUcmCIIAIiIiIiIiIiIiIiIiIipy5GIHQERERERERERERERERETvh8U+IiIiIiIiIiIiIiIioiKKxT4iIiIiIiIiIiIiIiKiIorFPiIiIiIiIiIiIiIiIqIiisU+IiIiIiIiIiIiIiIioiKKxT4iIiIiIiIiIiIiIiKiIorFPiIiIiIiIiIiIiIiIqIiisU+IiIiIiIiIiIiIiIioiKKxT4iIiIiIiIiIiIiIiKiIorFPiIiIiIqNEJDQ+Hh4SF2GKKR+vgL0urVqyGTyRAXFyd2KIVeXFwcZDIZVq9eLVoMLVu2RP/+/Qvk3PrGN2XKFMhksgK5njFp7vPz588b9boeHh4IDQ3Ndd/WrVsXbEAkqnHjxqFOnTpih0FEREQkKSz2EREREVGBkslkufoJCwsTO1S94uLi0Lt3b3h5ecHCwgIuLi5o0KABJk+eLHZo+a527dqQyWRYvHix2KEUeqGhoTr3r42NDSpUqIAOHTpg69atUKvV733u3bt3Y8qUKfkXrAEbNmzAggULCvw6eXXixAns27cPY8eO1baFhYUZzB1ffPGF0WM8duwYOnXqhHLlysHMzAz29vaoU6cOpk2bhsePHxs9nsImKioKU6ZMKbDiulKpxPz581GnTh3Y29vDwsIC3t7eGDJkCG7cuFEg1yxoM2bMwPbt23PVV1Ow1vyYmpqiZMmSqFu3Lr755hvcvXv3veN4+PAhpkyZgvDw8Pc+x/DhwxEREYGdO3e+9zmIiIiIKG9MxA6AiIiIiIq3tWvX6jz+/fffsX///mztvr6+WLZs2QcVSfLbrVu3UKtWLVhaWqJPnz7w8PDAo0ePcPHiRcyePRtTp07N1+uJOf6bN2/i3Llz8PDwwPr16zFo0CBR4ihKzM3NsXz5cgBAamoq7ty5g7///hsdOnRAo0aNsGPHDtjZ2eX5vLt378avv/5a4AW/DRs24MqVKxg+fLhOu7u7O1JTU2Fqalqg1zdkzpw5aNKkCSpWrJjt2NChQ1GrVi2dNmPPhp00aRKmT5+OChUqIDQ0FBUqVIBSqcSFCxcwb948rFmzBjExMUaNSWzR0dGQy//3XeKoqChMnToVjRo1yvf35+nTp2jRogUuXLiA1q1bo2vXrrCxsUF0dDQ2btyI3377Denp6fl6TWOYMWMGOnTogLZt2+b6OV26dEHLli2hVquRmJiIc+fOYcGCBVi4cCFWrFjxXoXwhw8fYurUqfDw8IC/v3+enw8ALi4uaNOmDebOnYv//Oc/73UOIiIiIsobFvuIiIiIqEB1795d5/Hp06exf//+bO2F0fz585GcnIzw8HC4u7vrHEtISMi367x+/RrW1taiFVcAYN26dXB2dsa8efPQoUMHxMXFcUnRdzAxMcl2H3/33XeYNWsWxo8fj/79+2PTpk0iRff+ZDIZLCwsRLl2QkICdu3ahSVLlug9Xr9+fXTo0MHIUf3Ppk2bMH36dHTq1Alr166FmZmZzvH58+dj/vz5OZ5DEAQolUpYWloWZKhGZW5ubrRrhYaG4tKlS9iyZQs+//xznWPTp0/HhAkT8uU6mrz8tsL0/gUEBGTLQXfu3EHz5s3Rq1cv+Pr6olq1aqLE1qlTJ3Ts2BG3b99GhQoVRImBiIiISEq4jCcRERERFRpv71mnWaps7ty5+PXXX1GhQgVYWVmhefPmuHfvHgRBwPTp01G+fHlYWlqiTZs2eP78ebbz7tmzB/Xr14e1tTVsbW3RqlUrXL169Z3xxMTEoHz58tkKfQDg7Oz8XtcJDQ2FjY0NYmJi0LJlS9ja2qJbt256xw8AarUaCxYswEcffQQLCwuULl0aAwcORGJiok6/8+fPIzg4GCVLloSlpSU8PT3Rp0+fd45RY8OGDejQoQNat24Ne3t7bNiwIVsfzb5mt27dQmhoKBwcHGBvb4/evXsjJSVFp29mZiamT58OLy8vmJubw8PDA9988w3S0tJ0+mn27woLC0PNmjVhaWkJPz8/7bKu27Ztg5+fHywsLFCjRg1cunRJ5/mXL1/Wzq7SLLPap08fPHv2LMfx9urVCyVLlkRGRka2Y82bN0flypVz87LpNW7cODRv3hybN2/OtqTgu+6R0NBQ/PrrrwB0l8DVyO39oLlWw4YNYWtrCzs7O9SqVUv7vjZq1Ai7du3CnTt3tNfQ3HuG9uw7dOiQNnYHBwe0adMG165d0+mTl3tEn127diEzMxNNmzZ9Z9+3Gdo3rlGjRmjUqFGez6fPpEmTULJkSaxYsSJboQ8A7O3ts83I1Nzje/fu1d7jS5cuBQCsWrUKjRs3hrOzM8zNzVGlShW9y+hqzrFv3z74+/vDwsICVapUwbZt2/TGmZaWhpEjR6JUqVKwtrZGu3bt8OTJkxzHtnPnTshkMly+fFnbtnXrVshkMrRv316nr6+vLzp37qwTn+a1X716NTp27AgACAoKMrhU8/Hjx1G7dm1YWFigQoUK+P3333OMDwDOnDmDXbt2oW/fvtkKfUBW0XHu3Lk6bXm5b6OiotC1a1eUKFECn376qXZsht6/Fy9eYPjw4XB1dYW5uTkqVqyI2bNnZ5uhrVarsXDhQm0uK1WqFFq0aKHdW1Emk+H169dYs2aN9vXK7R6Ib3N3d8fq1auRnp6OH374Qdv+/PlzjB49Gn5+frCxsYGdnR0+++wzREREaPuEhYVpZ8727t1bG4smFxw7dgwdO3aEm5sbzM3N4erqihEjRiA1NTVbHJrf4R07drzXOIiIiIgob1jsIyIiIqJCb/369Vi0aBG++uorjBo1CkeOHEGnTp0wceJE/Pvvvxg7diwGDBiAv//+G6NHj9Z57tq1a9GqVSvY2Nhg9uzZ+PbbbxEVFYVPP/30nftJubu74969ezh06NA7Y8zLdTIzMxEcHAxnZ2fMnTtX74fWGgMHDsSYMWNQr149LFy4EL1798b69esRHBysLVQlJCSgefPmiIuLw7hx4/Dzzz+jW7duOH369DvjBrI+QL916xa6dOkCMzMztG/fHuvXrzfYv1OnTnj16hVmzpyJTp06YfXq1dmWNO3Xrx8mTZqEgIAAzJ8/Hw0bNsTMmTP1Lit369YtdO3aFSEhIZg5cyYSExMREhKC9evXY8SIEejevTumTp2KmJgYdOrUSeeD9P379+P27dvo3bs3fv75Z3zxxRfYuHEjWrZsCUEQDI6hR48eePbsGfbu3avTHh8fj0OHDn3wzNMePXpAEATs379f25abe2TgwIFo1qyZtr/mRyM39wOQVXBp1aoVnj9/jvHjx2PWrFnw9/fHv//+CwCYMGEC/P39UbJkSe01ctq/78CBAwgODkZCQgKmTJmCkSNH4uTJk6hXr57e36Pc3CP6nDx5Ek5OTnoL7ADw6tUrPH36VOfHWEvf3rhxAzdu3EDbtm1hY2OTp+dGR0ejS5cuaNasGRYuXKhdHnHx4sVwd3fHN998g3nz5sHV1RWDBw/WFnzfdPPmTXTu3BmfffYZZs6cCRMTE3Ts2FHnHtP46quvEBERgcmTJ2PQoEH4+++/MWTIkBxj/PTTTyGTyXD06FFt27FjxyCXy3H8+HFt25MnT3D9+nU0aNBA73kaNGiAoUOHAgC++eYb7f3l6+ur7XPr1i106NABzZo1w7x581CiRAmEhoa+80sYmj3gevTokWM/jbzetx07dkRKSgpmzJiB/v37a9v1vX8pKSlo2LAh1q1bh549e+Knn35CvXr1MH78eIwcOVLnvH379tUWBWfPno1x48bBwsJCm6PXrl0Lc3Nz1K9fX/t6DRw4MFdj1CcwMBBeXl4698bt27exfft2tG7dGj/++CPGjBmDyMhINGzYEA8fPgSQVcSdNm0aAGDAgAHaWDTv9ebNm5GSkoJBgwbh559/RnBwMH7++Wf07NkzWwz29vbw8vLCiRMn3nscRERERJQHAhERERGREX355ZeCoT9De/XqJbi7u2sfx8bGCgCEUqVKCS9evNC2jx8/XgAgVKtWTcjIyNC2d+nSRTAzMxOUSqUgCILw6tUrwcHBQejfv7/OdeLj4wV7e/ts7W+7cuWKYGlpKQAQ/P39hWHDhgnbt28XXr9+rdMvL9fp1auXAEAYN27cO8d/7NgxAYCwfv16nX7//vuvTvtff/0lABDOnTuX43gMGTJkiODq6iqo1WpBEARh3759AgDh0qVLOv0mT54sABD69Omj096uXTvByclJ+zg8PFwAIPTr10+n3+jRowUAwqFDh7Rt7u7uAgDh5MmT2ra9e/cKAARLS0vhzp072valS5cKAITDhw9r21JSUrKN548//hAACEePHtW2rVq1SgAgxMbGCoIgCCqVSihfvrzQuXNnnef++OOPgkwmE27fvq3vpdLq1auXYG1tbfD4pUuXBADCiBEjBEHI2z1i6Hckt/fDixcvBFtbW6FOnTpCamqqTl/NeywIgtCqVSud+01D83u3atUqbZu/v7/g7OwsPHv2TNsWEREhyOVyoWfPntq23N4jhnz66adCjRo1srUfPnxYAKD3R/Oeuru7C7169cr23IYNGwoNGzbMcXyauHOyY8cOAYCwYMECnXa1Wi08efJE5+fNvKS5x//9999s59R3/wYHBwsVKlTQadOcY+vWrdq2pKQkoUyZMkL16tW1bZr7vGnTpjrv9YgRIwSFQqGTR/X56KOPhE6dOmkfBwQECB07dhQACNeuXRMEQRC2bdsmABAiIiJ04nvztd+8eXO239W3x/Lm72dCQoJgbm4ujBo1Ksf42rVrJwAQEhMTc+ynkdf7tkuXLgbjffv9mz59umBtbS3cuHFDp33cuHGCQqEQ7t69KwiCIBw6dEgAIAwdOjTbud98j6ytrfXev/po7uE5c+YY7NOmTRsBgJCUlCQIgiAolUpBpVJlO4+5ubkwbdo0bdu5c+ey/X5o6LtfZ86cKchkMp1crdG8eXPB19c3V2MiIiIiog/DmX1EREREVOh17NgR9vb22sd16tQBkLUfoImJiU57eno6Hjx4ACBr1teLFy/QpUsXnZlACoUCderUweHDh3O87kcffYTw8HB0794dcXFxWLhwIdq2bYvSpUtj2bJl2n7vc51Bgwa9c9ybN2+Gvb09mjVrpnPeGjVqwMbGRnteBwcHAMA///yjd1nKnGRmZmLTpk3o3LmzdrlIzbKChmb3/fe//9V5XL9+fTx79gwvX74EAOzevRsAss1uGTVqFICspRrfVKVKFQQGBmofa97fxo0bw83NLVv77du3tW1v7pulVCrx9OlTfPLJJwCAixcvGhy3XC5Ht27dsHPnTrx69Urbvn79etStWxeenp4Gn5sbmplfmnN/6L0I5P5+2L9/P169eqWdPfSmN5cEza1Hjx4hPDwcoaGhcHR01LZXrVoVzZo1077fb3rXPWLIs2fPUKJECYPHJ02ahP379+v8uLi45HFE70cT+9uz+pKSklCqVCmdn/DwcJ0+np6eCA4OznbON+/fpKQkPH36FA0bNsTt27eRlJSk07ds2bJo166d9rGdnR169uyJS5cuIT4+XqfvgAEDdN7r+vXrQ6VS4c6dOzmOsX79+jh27BiArHs3IiICAwYMQMmSJbXtx44dg4ODAz7++OMcz5WTKlWqoH79+trHpUqVQuXKlXV+t/XRvAe2trbvvEZ+3Lca+t6/zZs3o379+ihRooTO72PTpk2hUqm0MyQ1S6FOnjw523nf5/cxt97OQebm5pDLsz4CUqlUePbsGWxsbFC5cuUcc+Wb3rxfX79+jadPn6Ju3boQBCHbEssAtK8NERERERU8k3d3ISIiIiIS15sFHwDawp+rq6veds3+ZTdv3gSQVTTSx87O7p3X9vb2xtq1a6FSqRAVFYV//vkHP/zwAwYMGABPT080bdo0z9cxMTFB+fLl33ntmzdvIikpSe/+gEDW8p0A0LBhQ3z++eeYOnUq5s+fj0aNGqFt27bo2rUrzM3Nc7zGvn378OTJE9SuXRu3bt3StgcFBeGPP/7A7NmztR8Qa7z9fmiKM4mJibCzs8OdO3cgl8tRsWJFnX4uLi5wcHDIVnB43/cXyNqHaurUqdi4caP29dB4u1jytp49e2L27Nn466+/0LNnT0RHR+PChQtYsmRJjs/LjeTkZAD/K0rkx72Y2/shJiYGAD6oGPMmzfulbx9DX19f7N27F69fv4a1tbW2/V33SE6EHJZf9fPze6/9/PKD5r3UvLcaNjY22uUS9+3bhzlz5mR7rqHi8YkTJzB58mScOnUq256GSUlJOl9yqFixYrbikLe3N4CsfRbfLHrm9PrnpH79+liyZAlu3bqFmJgYyGQyBAYGaouA/fv3x7Fjx1CvXr1seSEv3o5PE+O74tPcO69evdJ+ycGQ97lvDb1P+tpv3ryJy5cvo1SpUnqf8+bvY9myZXUKjsbwdg7S7Bu4aNEixMbGQqVSafs6OTnl6px3797FpEmTsHPnzmzvlb58KwhCgRY0iYiIiOh/WOwjIiIiokJPoVDkqV1TLNDs5bV27Vq9s3/enBWYmxj8/Pzg5+eHwMBABAUFYf369WjatGmer/PmDIucqNXqHGfYaT5klslk2LJlC06fPo2///4be/fuRZ8+fTBv3jycPn06x/3FNOfu1KmT3uNHjhxBUFCQTtu7XneN3H7I+77vL5AV98mTJzFmzBj4+/vDxsYGarUaLVq0eOdeblWqVEGNGjW0e26tW7cOZmZmBl+LvLhy5QoAaAue+XEv5vZ+KAxye4+8zcnJ6Z0FH0MM3W8qlcpgPHnh4+MD4H/vrYaJiYm2AHn//n29z31zRpRGTEwMmjRpAh8fH/z4449wdXWFmZkZdu/ejfnz53/QXoTv+/p/+umnAICjR4/i9u3bCAgIgLW1NerXr4+ffvoJycnJuHTpEr7//vv3ju1D4tO8B5GRkTozA/OLvvfJULtarUazZs3w9ddf632OphArlitXrsDZ2VlbIJ0xYwa+/fZb9OnTB9OnT4ejoyPkcjmGDx+eq3tNpVKhWbNmeP78OcaOHQsfHx9YW1vjwYMHCA0N1XuOxMRElCxZMt/HRkRERETZsdhHRERERMWWl5cXAMDZ2TlfZwPVrFkTQNYycQV5HS8vLxw4cAD16tUz+CH0mz755BN88skn+P7777FhwwZ069YNGzduRL9+/fT2f/36NXbs2IHOnTujQ4cO2Y4PHToU69evz1bsexd3d3eo1WrcvHkTvr6+2vbHjx/jxYsXcHd3z9P5DElMTMTBgwcxdepUTJo0SduumUWXGz179sTIkSPx6NEjbNiwAa1atcpxGcncWrt2LWQyGZo1awYgb/eIoaJVbu8HzbWuXLmSbXZlbq7zNs37FR0dne3Y9evXUbJkSZ3ZUR/Cx8cHW7dufa/nlihRAi9evMjWfufOHVSoUOEDI8uaIVapUiVs374dCxYs+OAx//3330hLS8POnTt1ZroZWtL11q1b2WZK3bhxAwDg4eHxQbFouLm5wc3NDceOHcPt27e1BbUGDRpg5MiR2Lx5M1QqFRo0aJDjeQpqNldISAhmzpyJdevWvbPYV9D3rZeXF5KTk9/5++zl5YW9e/fi+fPnOc7uy8/X7NSpU4iJiUH37t21bVu2bEFQUBBWrFih0/fFixc6BTlDcURGRuLGjRtYs2YNevbsqW3XzGrVJzY2FtWqVXvfYRARERFRHnDPPiIiIiIqtoKDg2FnZ4cZM2bo3cvuyZMnOT7/2LFjep+n2etJszzch17HkE6dOkGlUmH69OnZjmVmZmoLG4mJidlmxPj7+wMA0tLSDJ7/r7/+wuvXr/Hll1+iQ4cO2X5at26NrVu35ngOfVq2bAkAWLBggU77jz/+CABo1apVns5niGZ20Ntjf/u6OenSpQtkMhmGDRuG27dv63w4/r5mzZqFffv2oXPnzqhUqRKAvN0jmgLE24Wr3N4PzZs3h62tLWbOnAmlUqnT783Xytra+p1LnQJAmTJl4O/vjzVr1ujEdOXKFezbt0/7fueHwMBAJCYmvnPvNn28vLxw+vRppKena9v++ecf3Lt3L9/imzJlCp4+fYr+/fvrfR/fNTPtTfru36SkJKxatUpv/4cPH+Kvv/7SPn758iV+//13+Pv75+u+hfXr18ehQ4dw9uxZbUHN398ftra2mDVrFiwtLVGjRo0cz2HoHv5QgYGBaNGiBZYvX47t27dnO56eno7Ro0cDKPj7tlOnTjh16hT27t2b7diLFy+QmZkJAPj8888hCAKmTp2ard/bv4/58XrduXMHoaGhMDMzw5gxY7TtCoUi2/25efNm7R63b8ahGcOb9N2vgiBg4cKFeuNISkpCTEwM6tat+95jISIiIqLc48w+IiIiIiq27OzssHjxYvTo0QMBAQH44osvUKpUKdy9exe7du1CvXr18Msvvxh8/uzZs3HhwgW0b98eVatWBQBcvHgRv//+OxwdHTF8+PB8uY4hDRs2xMCBAzFz5kyEh4ejefPmMDU1xc2bN7F582YsXLgQHTp0wJo1a7Bo0SK0a9cOXl5eePXqFZYtWwY7O7scP9Bev349nJycDH4Y+5///AfLli3Drl270L59+1zHXa1aNfTq1Qu//fYbXrx4gYYNG+Ls2bNYs2YN2rZtm+eZgobY2dmhQYMG+OGHH5CRkYFy5cph3759iI2NzfU5SpUqhRYtWmDz5s1wcHDIUyEyMzMT69atAwAolUrcuXMHO3fuxOXLlxEUFITffvtNJ9bc3iOaQsrQoUMRHBwMhUKBL774Itf3g52dHebPn49+/fqhVq1a6Nq1K0qUKIGIiAikpKRgzZo12uts2rQJI0eORK1atWBjY4OQkBC9Y50zZw4+++wzBAYGom/fvkhNTcXPP/8Me3t7TJkyJdev2bu0atUKJiYmOHDgAAYMGJCn5/br1w9btmxBixYt0KlTJ8TExGDdunXamY75oWvXrrhy5QpmzpyJs2fP4osvvoCnpydev36NK1eu4I8//oCtrW2uZoc2b94cZmZmCAkJwcCBA5GcnIxly5bB2dlZO2v4Td7e3ujbty/OnTuH0qVLY+XKlXj8+LHB4uD7ql+/PtavXw+ZTKZd1lOhUKBu3brYu3cvGjVqBDMzsxzP4e/vD4VCgdmzZyMpKQnm5uZo3Lixwf0m8+L3339H8+bN0b59e4SEhKBJkyawtrbGzZs3sXHjRjx69Ahz584FULD37ZgxY7Bz5060bt0aoaGhqFGjBl6/fo3IyEhs2bIFcXFxKFmyJIKCgtCjRw/89NNPuHnzpnaJ4WPHjiEoKAhDhgwBkPX7eODAAfz4448oW7YsPD09UadOnRxjuHjxItatWwe1Wo0XL17g3Llz2Lp1K2QyGdauXav97xYAtG7dGtOmTUPv3r1Rt25dREZGYv369dlmvXp5ecHBwQFLliyBra0trK2tUadOHfj4+MDLywujR4/GgwcPYGdnh61btxpcdvfAgQMQBAFt2rT5oNeZiIiIiHJJICIiIiIyoi+//FIw9Gdor169BHd3d+3j2NhYAYAwZ84cnX6HDx8WAAibN2/WaV+1apUAQDh37ly2/sHBwYK9vb1gYWEheHl5CaGhocL58+dzjPXEiRPCl19+KXz88ceCvb29YGpqKri5uQmhoaFCTExMtv65uU6vXr0Ea2vrXI1f47fffhNq1KghWFpaCra2toKfn5/w9ddfCw8fPhQEQRAuXrwodOnSRXBzcxPMzc0FZ2dnoXXr1jmO7/Hjx4KJiYnQo0cPg31SUlIEKysroV27doIgCMLkyZMFAMKTJ090+mle99jYWG1bRkaGMHXqVMHT01MwNTUVXF1dhfHjxwtKpVLnue7u7kKrVq2yXRuA8OWXX+q06bsf7t+/L7Rr105wcHAQ7O3thY4dOwoPHz4UAAiTJ0/OMUaNP//8UwAgDBgwwOBr8bZevXoJALQ/VlZWgoeHh/D5558LW7ZsEVQqld7n5eYeyczMFL766iuhVKlSgkwmy/b78q77QWPnzp1C3bp1BUtLS8HOzk6oXbu28Mcff2iPJycnC127dhUcHBwEANp7T/M6r1q1Sud8Bw4cEOrVq6c9X0hIiBAVFaXTJy/3iCH/+c9/hCZNmmR73fT9zr9t3rx5Qrly5QRzc3OhXr16wvnz54WGDRsKDRs21PbRNz5N3LkVFhYmdOjQQShTpoxgamoq2NnZCTVr1hQmT54sPHr0SKevoXtcELLeo6pVqwoWFhaCh4eHMHv2bGHlypXZXivNOfbu3StUrVpVMDc3F3x8fPKUAwEIhw8ffufYrl69KgAQfH19ddq/++47AYDw7bffZnuOu7u70KtXL522ZcuWCRUqVBAUCoXOtQ29Hm+/TzlJSUkR5s6dK9SqVUuwsbERzMzMhEqVKglfffWVcOvWLZ2+H3Lf5hSvIAjCq1evhPHjxwsVK1YUzMzMhJIlSwp169YV5s6dK6Snp2v7ZWZmCnPmzBF8fHwEMzMzoVSpUsJnn30mXLhwQdvn+vXrQoMGDQRLS0sBQLbX802ae1jzY2JiIjg6Ogp16tQRxo8fL9y5cyfbc5RKpTBq1CihTJkygqWlpVCvXj3h1KlTel/3HTt2CFWqVBFMTEx0fleioqKEpk2bCjY2NkLJkiWF/v37CxEREXrzRefOnYVPP/3U4BiIiIiIKH/JBCEP64wQEREREREVMzt27EDbtm1x9OjRd+4DRgXv2LFjaNSoEa5fv65dBlXqPDw88PHHH+Off/4ROxSid4qPj4enpyc2btzImX1ERERERsI9+4iIiIiISNKWLVuGChUqaJcsJHHVr18fzZs3xw8//CB2KET0HhYsWAA/Pz8W+oiIiIiMiHv2ERERERGRJG3cuBGXL1/Grl27sHDhQshkMrFDov+3Z88esUMgovc0a9YssUMgIiIikhwW+4iIiIiISJK6dOkCGxsb9O3bF4MHDxY7HCIiIiIiIqL3wj37iIiIiIiIiIiIiIiIiIoo7tlHREREREREREREREREVERxGc98olar8fDhQ9ja2nKvDyIiIiIiIiIiIiIiItJLEAS8evUKZcuWhVz+4fPyWOzLJw8fPoSrq6vYYRAREREREREREREREVERcO/ePZQvX/6Dz8NiXz6xtbUFkPXG2NnZiRxN0aBSqXD16lV89NFHUCgUYodDRGQUzH1EJEXMfUQkRcx9RCRVzH9EJEV5zX0vX76Eq6urtrb0oVjsyyeapTvt7OxY7MsllUoFGxsb2NnZ8T/8RCQZzH1EJEXMfUQkRcx9RCRVzH9EJEXvm/vya1u4D18IlIiIiIiIiIiIiIiIiIhEwWIfiUYmk8HR0THfKtdEREUBcx8RSRFzHxFJEXMfEUkV8x8RSZHYuU8mCIIgypWLmZcvX8Le3h5JSUlcxpOIiIiIiIiIiIiIiIj0yu+aEvfsI9Go1Wrcv38f5cuXh1zOSaZEJA3MfUQkRcx9RCRFzH1EJFVi5D+VSoWMjAyjXIuISB+1Wo34+Hi4uLhoc5+pqanR9i5lsY9EIwgCnj9/jnLlyokdChGR0TD3EZEUMfcRkRQx9xGRVBkz/wmCgPj4eLx48aLAr0VElBNBEJCRkQGlUqmzlKeDgwNcXFwKfHlPFvuIiIiIiIiIiIiIqMjRFPqcnZ1hZWXFfQKJSDSCIECpVMLCwgIymQyCICAlJQUJCQkAgDJlyhTo9VnsIyIiIiIiIiIiIqIiRaVSaQt9Tk5OYodDRBInCAIEQdAW+wDA0tISAJCQkABnZ+cCXdKTi8aTaGQymVGmrxIRFSbMfUQkRcx9RCRFzH1EJFXGyn+aPfqsrKwK9DpERLllamqarU2Towp6X1HO7CPRyOVyuLi4iB0GEZFRMfcRkRQx9xGRFDH3EZFUGTv/8UsVRFQYyGQyvcU+Y+Uozuwj0ahUKsTExEClUokdChGR0TD3EZEUMfcRkRQx9xGRVDH/EZEUafbsEwRBlOuz2EeievXqldghEBEZHXMfEUkRcx8RSRFzHxFJFfMfEUmRWq0W7dos9hERERERERERERERFRFxcXGQyWQIDw/P9XNWr14NBwcH0eN4W6NGjTB8+PB8i6kwmDJlCvz9/cUOgySGxT4iIiIiIiIiIiIiIiO6d+8e+vTpg7Jly8LMzAzu7u4YNmwYnj179s7nurq64tGjR/j4449zfb3OnTvjxo0bHxJynmgKgTn9rF69Gtu2bcP06dONFpeGSqXCrFmz4OPjA0tLSzg6OqJOnTpYvnz5B5979OjROHjwYD5ESZR7JmIHQNIlk8ng6urKTXSJSFKY+4hIipj7iEiKmPuISKqY/97t9u3bCAwMhLe3N/744w94enri6tWrGDNmDPbs2YPTp0/D0dFR73PT09NhZmYGFxeXPF3T0tISlpaW+RF+rmgKkhpz587Fv//+iwMHDmjb7O3tjRrTm6ZOnYqlS5fil19+Qc2aNfHy5UucP38eiYmJ731OQRCgUqlgY2MDGxubfIyWigozMzPRrs2ZfSQauVwOJycnyOW8DYlIOpj7iEiKmPuISIqY+4hIqsTMf4IAKJXi/AhC7uP88ssvYWZmhn379qFhw4Zwc3PDZ599hgMHDuDBgweYMGGCtq+HhwemT5+Onj17ws7ODgMGDNC7fObOnTtRqVIlWFhYICgoCGvWrIFMJsOLFy8AZF/GU7PU5Nq1a+Hh4QF7e3t88cUXOvst/vvvv/j000/h4OAAJycntG7dGjExMbkao0KhgIuLi/bHxsYGJiYmOm2WlpbZlvH08PDAd999h549e8LGxgbu7u7YuXMnnjx5gjZt2sDGxgZVq1bF+fPnda53/Phx1K9fH5aWlnB1dcXQoUPx+vVrg/Ht3LkTgwcPRseOHeHp6Ylq1aqhb9++GD16tLaPWq3GzJkz4enpCUtLS1SrVg1btmzRHg8LC4NMJsOePXtQo0YNmJub4/jx43qX8Vy+fDl8fX1hYWEBHx8fLFq0SHssPT0dQ4YMQZkyZWBhYQF3d3fMnDkzV68zFR4ymQwmJiaifdGBM/tINCqVCjdv3kSlSpWgUCjEDoeIyCiY+4hIipj7iEiKmPuISKrEzH9paUDHjka9pNbmzYCFxbv7PX/+HHv37sX333+fbVabi4sLunXrhk2bNmHRokXaosHcuXMxadIkTJ48We85Y2Nj0aFDBwwbNgz9+vXDpUuXdIpWhsTExGD79u34559/kJiYiE6dOmHWrFn4/vvvAQCvX7/GyJEjUbVqVSQnJ2PSpElo164dwsPDC7SYO3/+fMyYMQPffvst5s+fjx49eqBu3bro06cP5syZg7Fjx6Jnz564evUqZDIZYmJi0KJFC3z33XdYuXIlnjx5giFDhmDIkCFYtWqV3mu4uLjg0KFDGDx4MEqVKqW3z8yZM7Fu3TosWbIElSpVwtGjR9G9e3eUKlUKDRs21PYbN24c5s6diwoVKqBEiRIICwvTOc/69esxadIk/PLLL6hevTouXbqE/v37w9raGr169cJPP/2EnTt34s8//4Sbmxvu3buHe/fu5dvrScYhCAKUSiUsLCxEKfix2EeiUiqVYodARGR0zH1EJEXMfUQkRcx9RCRVzH+G3bx5E4IgwNfXV+9xX19fJCYm4smTJ3B2dgYANG7cGKNGjdL2iYuL03nO0qVLUblyZcyZMwcAULlyZVy5ckVbtDNErVZj9erVsLW1BQD06NEDBw8e1D7v888/1+m/cuVKlCpVClFRUXnaLzCvWrZsiYEDBwIAJk2ahMWLF6NWrVro+P+V3LFjxyIwMBCPHz+Gi4sLZs6ciW7dumlnCFaqVAk//fQTGjZsiMWLF8NCTxX2xx9/RIcOHeDi4oKPPvoIdevWRZs2bfDZZ58BANLS0jBjxgwcOHAAgYGBAIAKFSrg+PHjWLp0qU6xb9q0aWjWrJnB8UyePBnz5s1D+/btAQCenp6IiorC0qVL0atXL9y9exeVKlXCp59+CplMBnd39w9/EUkUQl6m+OazQlPsmzVrFsaPH49hw4ZhwYIFALL+ozBq1Chs3LgRaWlpCA4OxqJFi1C6dGnt8+7evYtBgwbh8OHDsLGxQa9evTBz5kyYmPxvaGFhYRg5ciSuXr0KV1dXTJw4EaGhoTrX//XXXzFnzhzEx8ejWrVq+Pnnn1G7dm1jDJ2IiIiIiIiIiIiIPpC5edYMO7GunRd5KQrUrFkzx+PR0dGoVauWTltuPtv28PDQFvoAoEyZMkhISNA+vnnzJiZNmoQzZ87g6dOnUKvVALI+ky/IYl/VqlW1/66pBfj5+WVrS0hIgIuLCyIiInD58mWsX79e20cQBKjVasTGxuotrFapUgVXrlzBhQsXcOLECRw9ehQhISEIDQ3F8uXLcevWLaSkpGQr4qWnp6N69eo6bTm9P69fv0ZMTAz69u2L/v37a9szMzNhb28PAAgNDUWzZs1QuXJltGjRAq1bt0bz5s3f+ToRvalQFPvOnTuHpUuX6vwSA8CIESOwa9cubN68Gfb29hgyZAjat2+PEydOAMiaEt6qVSu4uLjg5MmTePToEXr27AlTU1PMmDEDQNYU5latWuG///0v1q9fj4MHD6Jfv34oU6YMgoODAQCbNm3CyJEjsWTJEtSpUwcLFixAcHAwoqOjtd+eICIiIiIiIiIiIqLCSybL3VKaYqpYsSJkMhmuXbuGdu3aZTt+7do1lChRQmdpSWtr6wKJxdTUVOexTCbTFvQAICQkBO7u7li2bBnKli0LtVqNjz/+GOnp6QUSj764NMsh6mvTxJqcnIyBAwdi6NCh2c7l5uZm8DpyuRy1atVCrVq1MHz4cKxbtw49evTAhAkTkJycDADYtWsXypUrp/M887cquzm9P5rzLFu2DHXq1NE5plnmNiAgALGxsdizZw8OHDiATp06oWnTpjr7AxK9i+jFvuTkZHTr1g3Lli3Dd999p21PSkrCihUrsGHDBjRu3BgAsGrVKvj6+uL06dP45JNPsG/fPkRFReHAgQMoXbo0/P39MX36dIwdOxZTpkyBmZkZlixZAk9PT8ybNw9A1jTo48ePY/78+dpi348//oj+/fujd+/eAIAlS5Zg165dWLlyJcaNG6c37rS0NKSlpWkfv3z5EkBWAVKlUgHISjpyuRxqtVrnmxqadk2/d7XL5XLIZDK97QB0EnBO7QqFQvuNhrfb347RUHt+jkkul8Pd3R2CIGiPF/UxFcf3iWPimDim/B2TIAg6ua84jKk4vk8cE8fEMeXvmN7MfZpzFvUx5SZ2jolj4pikPSZN7tP8DVgcxvSu2Dkmjolj4pg0/T09PXU+8yuIMWn+ztT8aPLt2/KrPS9yOrejoyOaNWuGRYsWYfjw4bC0tNT2j4+Px/r169GjRw+d5735d7S+dm9vb+zZs0fbBgBnz57Vefyuf77d9uzZM0RHR+O3335D/fr1IZPJcOzYMZ3rvtn/7fjefg0M/buhmDT/rinsvX29N9sCAgIQFRUFLy+vbK+RvusZilEzA/D169fw9fWFubk57ty5gwYNGmTr//aYDf27s7MzypYti5iYGHTt2lXveQDA1tYWnTt3RqdOnfD555/js88+w7Nnz+Do6Kg3dkMK+vegsP0+FaZ2ADAzMwOQ/X7Q5ENNTtTkvfwkerHvyy+/RKtWrdC0aVOdYt+FCxeQkZGBpk2batt8fHzg5uaGU6dO4ZNPPsGpU6fg5+ens6xncHAwBg0ahKtXr6J69eo4deqUzjk0fTTr96anp+PChQsYP3689rhcLkfTpk1x6tQpg3HPnDkTU6dOzdZ+9epV2NjYAAAcHR3h5uaG+/fv4/nz59o+Li4ucHFxQVxcHF69eqVtd3V1hZOTE27evKmzrnWFChVgZ2eHqKgonf9AVq5cGWZmZoiMjNSJwc/PD+np6YiOjta2KRQK+Pn54dWrV7h9+7a23cLCAj4+PkhMTNTZ9NPW1hZeXl5ISEhAfHy8tj2/x3T//v1iN6bi+D5xTBwTx5R/Y7p27VqxG1NxfJ84Jo6JY+KYOCaOiWPimDgmjolj4pg4pvcfk42NTYGP6dGjR5DL5VAqlZDL5TA3N0d6erpOLKampjA1NUVaWppOMdHMzAwmJiZQKpU6H7ibm5tDoVAgNTVVJ3YLCwvIZLJs7ZaWlhAEIdsehVZWVlCr1TqTRWQyGSwtLaFSqTBnzhw0adIEzZs3x5QpU1C5cmVERERg7NixKFu2LCZOnIj09HSYm5tDEARkZGRor/3mDDelUonU1FT06tUL8+fPx9ixY9GtWzdERERg9erVAP5XdNDMxnt7DG8+zsjI0D7HwsICTk5OWLx4MUqUKIEnT55g7Nix2nOlpqZqx6dSqXTOI5fLYWFhgczMTO05MzMzdWLRvE9vFnTT0tJ0xqt5n968puY5b45/6NChCAoKwpAhQ9CvXz8oFApcv34dhw4dwo8//qj3ferWrRsaNGiAOnXqwNHREXFxcZg8eTIqVaoEHx8fAMCwYcMwcuRIKJVKfPrpp1AqlTh69CisrKzQvXt3nff3zTG9Oda0tDRMmDABo0ePhpWVFVq2bAmVSoWTJ08iMTERQ4cOxU8//QRXV1fUqFEDaWlp2LhxI0qXLq19//Pz3ntzVqa+9wnI+r0sSr9PhWlMb8961YwpIyMDN27cgEwm08l7+Ukm5Hf5MA82btyI77//HufOnYOFhQUaNWoEf39/LFiwABs2bEDv3r113kAga63hoKAgzJ49GwMGDMCdO3ewd+9e7fGUlBRYW1tj9+7d+Oyzz+Dt7Y3evXvrFPN2796NVq1aISUlBYmJiShXrhxOnjyp3WgTAL7++mscOXIEZ86c0Ru7vpl9rq6ueP78Oezs7ADwmz/vGpNarcaVK1fg6+urnbJc1MdUHN8njolj4pjyd0zp6em4du2aNvcVhzEVx/eJY+KYOKb8HZNKpdLmPs2HE0V9TLmJnWPimDgmaY9Jk/s++ugj7XWL+pjeFTvHxDFxTBwTkFUoioqKgo+Pj/Yzv4IYU2pqKu7cuQNPT0/tB+15mYUj9kykO3fuYMqUKfj333/x/PlzuLi4oE2bNpg8eTKcnJy0/T08PDBs2DDt5BXNcz09PXHx4kX4+/sDAHbu3InRo0fj3r17CAwMRKdOnTB48GCkpKTA0tISq1atwogRI5CYmAgAmDJlCnbs2IFLly5pz7tgwQIsXLgQsbGxAIADBw5g2LBhuH37NipXroyFCxciKCgI27ZtQ9u2bREXF4cKFSroxGHoNdBcLzw8XKc9KCgI1apVw8KFCyEIAjw9PXXGK5PJIJPJtNcEoPe6586dw8SJE3Hq1CkIggAvLy906tQJ33zzjd73aNmyZdi4cSOuXLmCpKQkuLi4oHHjxpg8eTI8PDwgCFmzsX766ScsWbIEt2/fhoODAwICAjB+/Hg0aNAAYWFhaNy4MRITE7X77xka64YNGzB37lxERUXB2toafn5+GDZsGNq1a4dly5Zh8eLFuHnzJhQKBWrVqoUffvgh296AuVHYZrsZ6/epMLQDWcVzTT7SUCqViI2Nhbu7Oyz+f51hhUKBpKQkODg4ICkpSVtT+hCiFfvu3buHmjVrYv/+/dq9+hoVoWLf216+fAl7e/t8e2OkQKVSITIyEn5+fjr/4SciKs6Y+4hIipj7iEiKmPuISKqMlf80H6Brin2k6/vvv8eSJUt0ZkYSUcERBAGpqanapXk1DOWq/K4pyT/4DO/pwoULSEhIQEBAAExMTGBiYoIjR47gp59+gomJCUqXLo309HS8ePFC53mPHz+Gi4sLgKyp5Y8fP852XHMspz52dnawtLREyZIloVAo9PbRnIOIiIiIiIiIiIiIqLBatGgRzp07h9u3b2Pt2rWYM2cOevXqJXZYRGQkohX7mjRpgsjISISHh2t/atasiW7dumn/3dTUFAcPHtQ+Jzo6Gnfv3tXOwAsMDERkZCQSEhK0ffbv3w87OztUqVJF2+fNc2j6aM5hZmaGGjVq6PRRq9U4ePCgzkw/IiIiIiIiIiIiIqLC6ObNm2jTpg2qVKmC6dOnY9SoUZgyZYrYYRGRkYi6Z9/b3lzGEwAGDRqE3bt3Y/Xq1bCzs8NXX30FADh58iSArCnh/v7+KFu2LH744QfEx8ejR48e6NevH2bMmAEAiI2Nxccff4wvv/wSffr0waFDhzB06FDs2rULwcHBAIBNmzahV69eWLp0KWrXro0FCxbgzz//xPXr11G6dOlcxc5lPPNOELI223x7DVsiouKMuY+IpIi5j4ikiLmPiKTKWPmPy3gSUWGi2edRs8+khrGW8TT54DMUoPnz50Mul+Pzzz9HWloagoODsWjRIu1xhUKBf/75B4MGDUJgYCCsra3Rq1cvTJs2TdvH09MTu3btwogRI7Bw4UKUL18ey5cv1xb6AKBz58548uQJJk2ahPj4ePj7++Pff//NdaGP3p+ZmZnYIRARGR1zHxFJEXMfEUkRcx8RSRXzHxFJkZhf8CpUM/uKMs7syztuVk5EUsTcR0RSxNxHRFLE3EdEUmWs/MeZfURUmAiCgNTUVFhaWooys0+0PfuIiIiIiIiIKGfx8fH4bvp01PL1hXe5cqjl64vvpk9HfHy82KEREREREVEhwWIfERERERERUSE0f948uJcvj+MzZmD49etY/PAhhl+/jmMzZsC9fHnM//FHsUMkIiIiIqJCoFDv2UdEREREREQkRfPnzcPMCRNwVKVCHZVK51g3pRJnAIR88w0AYMTIkSJESEREREREhQX37Msn3LMv7wRBgFqthlwuF3XjSiIiY2LuIyIpYu4jypv4+Hi4ly+fVejLod8ZAA0UCty5fx8uLi7GCo9yibmPiKTKWPmPe/YRUWHyZqmNe/aR5KSnp4sdAhGR0TH3EZEUMfcR5d7yZcsQZGqaY6EPAOoAaGRmhhXLlxsjLHoPzH1EJFXMf4VHaGgo2rZtq33cqFEjDB8+3OhxhIWFQSaT4cWLF0a/NpGxiDm3jsU+Eo1arUZ0dDTUarXYoRARGQ1zHxFJEXMfUd7s2LABPZTKXPXtmZqK7Rs2FHBE9D6Y+4hIqpj/3i00NBQymQwymQxmZmaoWLEipk2bhszMzAK/9rZt2zB9+vRc9TV2gc7DwwMymQwbN27Mduyjjz6CTCbD6tWrjRIL0ftQ5vJv+ILAYh8RERERERFRIZL08iVyuyhnaQBJSUkFGQ4REREVgBYtWuDRo0e4efMmRo0ahSlTpmDOnDl6++bnTElHR0fY2trm2/nym6urK1atWqXTdvr0acTHx8Pa2lqkqIgKPxb7iIiIiIiIiAoRezs7xOey72MA9vb2BRkOERFRkfM6/bXBH2WmMtd9UzNSc9X3fZibm8PFxQXu7u4YNGgQmjZtip07dwL439Kb33//PcqWLYvKlSsDAO7du4dOnTrBwcEBjo6OaNOmDeLi4rTnVKlUGDlyJBwcHODk5ISvv/4627KCby/jmZaWhrFjx8LV1RXm5uaoWLEiVqxYgbi4OAQFBQEASpQoAZlMhtDQUABZszdnzpwJT09PWFpaolq1atiyZYvOdXbv3g1vb29YWloiKChIJ86cdOvWDUeOHMG9e/e0bStXrkS3bt1gYmKi0/fFixfo168fSpUqBTs7OzRu3BgRERHa4zExMWjTpg1Kly4NGxsb1KpVCwcOHNA5h4eHB2bMmIE+ffrA1tYWbm5u+O2333IVK1FhwmIfiUqhUIgdAhGR0TH3EZEUMfcR5V6brl2x1sIiV31/t7RE265dCzgiel/MfUQkVWLnP5uZNgZ/Pv/zc52+znOdDfb9bP1nOn09Fnro7ZcfLC0tdWbwHTx4ENHR0di/fz/++ecfZGRkIDg4GLa2tjh27BhOnDgBGxsbtGjRQvu8efPmYfXq1Vi5ciWOHz+O58+f46+//srxuj179sQff/yBn376CdeuXcPSpUthY2MDV1dXbN26FQAQHR2NR48eYeHChQCAmTNn4vfff8eSJUtw9epVjBgxAt27d8eRI0cAZBUl27dvj5CQEISHh6Nfv34YN25crl6H0qVLIzg4GGvWrAEApKSkYNOmTejTp0+2vh07dkRCQgL27NmDCxcuICAgAE2aNMHz588BAMnJyWjZsiUOHjyIS5cuoUWLFggJCcHdu3d1zjNv3jzUrFkTly5dwuDBgzFo0CBER0fnKl6iwsLk3V2ICoZCoYCfn5/YYRARGRVzHxFJEXMfUd70698f06dOxRkAdXLodwZAWHo61vTrZ6TIKC+Y+4hIqpj/8kYQBBw8eBB79+7FV199pW23trbG8uXLYWZmBgBYt24d1Go1li9fDplMBgBYtWoVHBwcEBYWhubNm2PBggUYP3482rdvDwBYsmQJ9u7da/DaN27cwJ9//on9+/ejadOmAIAKFSpojzs6OgIAnJ2d4eDgACBrJuCMGTNw4MABBAYGap9z/PhxLF26FA0bNsTixYvh5eWFefPmAQAqV66MyMhIzJ49O1evSZ8+fTBq1ChMmDABW7ZsgZeXF/z9/XX6HD9+HGfPnkVCQgLMzc0BAHPnzsX27duxZcsWDBgwANWqVUO1atW0z5k+fTr++usv7Ny5E0OGDNG2t2zZEoMHDwYAjB07FvPnz8fhw4e1MyqJckMmk8HKykq067PYR6IRBAGvXr2Cra2t9j9QRETFHXMfEUkRcx9R3ri4uGDW7NkImTABf6el6S34nQEQYm6OWTNmwMUltzv8kTEx9xGRVBWG/Jc8PtngMYVcd9ZhwugEg33lMt2F8eKGxX1QXG/6559/YGNjg4yMDKjVanTt2hVTpkzRHvfz89MW+gAgIiICt27dyrbfnlKpRExMDJKSkvDo0SPUqfO/vxxMTExQs2bNbEt5aoSHh0OhUKBhw4a5jvvWrVtISUlBs2bNdNrT09NRvXp1AMC1a9d04gCgLQzmRqtWrTBw4EAcPXoUK1eu1DurLyIiAsnJyXByctJpT01NRUxMDICsmX1TpkzBrl278OjRI2RmZiI1NTXbzL6qVatq/10mk8HFxQUJCYbvCyJ9BEGAWq2GXC4XJfex2EeiUavVuH37Nvz8/ESf2k9EZCzMfUQkRcx9RHk3YtQoQCZDg6+/RiNTU/RUKlEaQDyA3y0scCQjA7NmzMCIkSPFDpUMYO4jIqkqDPnP2sxa9L7vEhQUhMWLF8PMzAxly5bNth+dtbXutZKTk1GjRg2sX78+27lKlSr1XjFYWlrm+TnJyVmF1F27dqFcuXI6xzQz7D6UiYkJevTogcmTJ+PMmTN6lyJNTk5GmTJlEBYWlu2YZhbi6NGjsX//fsydOxcVK1aEpaUlOnTooLNcKgCYmprqPJbJZFCr1fkyFpKWtLS09/q9yg8s9hEREREREREVQiNGjkSXrl2xYvlyjJq9ARkZSShTxh5d+nbF7/36cUYfERFREWZtbY2KFSvmun9AQAA2bdoEZ2dn2NnZ6e1TpkwZnDlzBg0aNAAAZGZmavey08fPzw9qtRpHjhzRLuP5Js3MQpVKpW2rUqUKzM3NcffuXYMzAn19fbFz506dttOnT797kG/o06cP5s6di86dO6NEiRLZjgcEBCA+Ph4mJibw8PDQe44TJ04gNDQU7dq1A5BVIIyLi8tTHERFhfzdXYiIiIiIiIhIDC4uLpgwcSJqNYpC3WYPsOrPKEyYOJGFPiIiIonp1q0bSpYsiTZt2uDYsWOIjY1FWFgYhg4divv37wMAhg0bhlmzZmH79u24fv06Bg8ejBcvXhg8p4eHB3r16oU+ffpg+/bt2nP++eefAAB3d3fIZDL8888/ePLkCZKTk2Fra4vRo0djxIgRWLNmDWJiYnDx4kX8/PPPWLNmDQDgv//9L27evIkxY8YgOjoaGzZswOrVq/M0Xl9fXzx9+hSrVq3Se7xp06YIDAxE27ZtsW/fPsTFxeHkyZOYMGECzp8/DwCoVKkStm3bhvDwcERERKBr166csUfFFot9JCoLCwuxQyAiMjrmPiKSIuY+ovxhYMsdKqSY+4hIqpj/8p+VlRWOHj0KNzc3tG/fHr6+vujbty+USqV2pt+oUaPQo0cP9OrVC4GBgbC1tdXOajNk8eLF6NChAwYPHgwfHx/0798fr1+/BgCUK1cOU6dOxbhx41C6dGkMGTIEADB9+nR8++23mDlzJnx9fdGiRQvs2rULnp6eAAA3Nzds3boV27dvR7Vq1bBkyRLMmDEjz2N2cnIyuCSiTCbD7t270aBBA/Tu3Rve3t744osvcOfOHZQuXRoA8OOPP6JEiRKoW7cuQkJCEBwcbHCWI1F+EHOfZplgaHdOypOXL1/C3t4eSUlJBqdRExEREREREb2PkJCsf06cCNSpI24sREREhYFSqURsbCw8PT1ZXCSiQstQrsrvmhJn9pFo1Go1nj17xqnTRCQpzH1EJEXMfUQkRcx9RCRVzH9EJEWCICAzMxNiza9jsY9EIwgC7t27J9rNT0QkBuY+IpIi5j4ikiLmPiKSKuY/IpKq9PR00a7NYh8RERERERERERERERFREcViHxEREREREREREREREVERxWIficrW1lbsEIiIjI65j4ikiLmPKH9wRbSihbmPiKSK+Y+IpEguF6/kZiLalUnyFAoFvLy8xA6DiMiomPuISIqY+4hIipj7iEiqmP+ISIpkMhksLCxEuz5n9pFo1Go14uPjoVarxQ6FiMhomPuISIqY+4jyD2f2FR3MfUQkVcx/RCRFgiAgIyMDgkh/sLPYR6IRBAHx8fGi3fxERGJg7iMiKWLuIyIpYu4jIqli/iMiqcrIyBDt2iz2ERERERERERUR/NyUiIiIiIjexmIfERERERERERERERF9kLCwMMhkMrx48cKo1129ejUcHBw+6BxxcXGQyWQIDw832Ees8RHlBot9JBqZTAZHR0fIZDKxQyEiMhrmPiKSIuY+IpIi5j4ikirmv3cLDQ1F27ZtxQ4jT2QyWY4/U6ZMETtEItEpFArRrm0i2pVJ8uRyOdzc3MQOg4jIqJj7iEiKmPuISIqY+4hIqpj/ir709HSYmZnptD169Ej775s2bcKkSZMQHR2tbbOxscH58+fz5VpERZFMJoO5ublo1+fMPhKNWq3G3bt3oVarxQ6FiMhomPuISIqY+4hIipj7iEiqmP8+3JEjR1C7dm2Ym5ujTJkyGDduHDIzMwEA//zzDxwcHKBSqQAA4eHhkMlkGDdunPb5/fr1Q/fu3bWPjx8/jvr168PS0hKurq4YOnQoXr9+rT3u4eGB6dOno2fPnrCzs8OAAQOyxeTi4qL9sbe3h0wm02mzsbHR9r1w4QJq1qwJKysr1K1bV6coOGXKFPj7+2P58uXw9PSEhYUFAODFixfo168fSpUqBTs7OzRu3BgRERHa50VERCAoKAi2traws7NDjRo1shUX9+7dC19fX9jY2KBFixY6BUq1Wo1p06ahfPnyMDc3h7+/P/79998c34fdu3fD29sblpaWCAoKQlxcXI79SdoEQUBaWhoEkTbZZrGPRCMIAp4/fy7azU9EJAbmPiKSIuY+ovzDX6Oig7mPiKSqUOS/XR8BF0b87/GV77La0p5nPX51K+vxjV//1+dUKLD/0/89frg3q8/Dvf9r2/9pVj+NG79m9clHDx48QMuWLVGrVi1ERERg8eLFWLFiBb777jsAQP369fHq1StcunQJQFZhsGTJkggLC9Oe48iRI2jUqBEAICYmBi1atMDnn3+Oy5cvY9OmTTh+/DiGDBmic925c+eiWrVquHTpEr799tsPGsOECRMwb948nD9/HiYmJujTp4/O8Vu3bmHr1q3Ytm2bdo+8jh07IiEhAXv27MGFCxcQEBCAJk2a4PnzrPesW7duKF++PM6dO4cLFy5g3LhxMDU11Z4zJSUFc+fOxdq1a3H06FHcvXsXo0eP1h5fuHAh5s2bh7lz5+Ly5csIDg7Gf/7zH9y8eVPvGO7du4f27dsjJCQE4eHh6Nevn05BlUgfTRFeDFzGk4iIiIiIiKiIYN2IiIioeFu0aBFcXV3xyy+/QCaTwcfHBw8fPsTYsWMxadIk2Nvbw9/fH2FhYahZsybCwsIwYsQITJ06FcnJyUhKSsKtW7fQsGFDAMDMmTPRrVs3DB8+HABQqVIl/PTTT2jYsCEWL16snVnXuHFjjBo1Kl/G8P3332uvP27cOLRq1QpKpVJ7rfT0dPz+++8oVaoUgKyZh2fPnkVCQoJ2GcS5c+di+/bt2LJlCwYMGIC7d+9izJgx8PHx0Y7jTRkZGViyZAm8vLwAAEOGDMG0adO0x+fOnYuxY8fiiy++AADMnj0bhw8fxoIFC/Drr7/ibYsXL4aXlxfmzZsHAKhcuTIiIyMxe/bsfHmNiPIbi31EREREREREREREVHy0uqr7+OOJWT8athWz9wlcrfu4bDBQ9q0+zY7rPvb+MusnH127dg2BgYGQyWTatnr16iE5ORn379+Hm5sbGjZsiLCwMIwaNQrHjh3DzJkz8eeff+L48eN4/vw5ypYtqy2GRURE4PLly1i/fr32fIIgQK1WIzY2Fr6+vgCAmjVr5tsYqlatqv33MmXKAAASEhK0ezm6u7trC32aGJOTk+Hk5KRzntTUVMTExAAARo4ciX79+mHt2rVo2rQpOnbsqC3sAYCVlZXO4zJlyiAhIQEA8PLlSzx8+BD16tXTOX+9evV0lgp907Vr11CnTh2dtsDAwNy9AEQiYLGPRKNZ1/nN/3ARERV3zH1EJEXMfUQkRcx9RCRVzH8Fr1GjRli5ciUiIiJgamoKHx8fNGrUCGFhYUhMTNTOqgOA5ORkDBw4EEOHDs12Hk3xDQCsra3zLb43l9fU3Adv7uH49rWSk5NRpkwZnaVINRwcHABk7fXXtWtX7Nq1C3v27MHkyZOxceNGtGvXLts1NdflUtpkbG/fh8bEYh+JRi6Xw8XFRewwiIiMirmPiKSIuY+IpIi5j4ikivnvw/j6+mLr1q0QBEFbKDtx4gRsbW1Rvnx5AP/bt2/+/Pnawl6jRo0wa9YsJCYm6izHGRAQgKioKFSsWNH4g8mlgIAAxMfHw8TEBB4eHgb7eXt7w9vbGyNGjECXLl2watUqbbEvJ3Z2dihbtixOnDihUwg9ceIEateurfc5vr6+2Llzp07b6dOnczcgkiSZTCZqsU8u2pVJ8lQqFWJiYkTdtJKIyNiY+4hIipj7iEiKmPuISKqY/3InKSkJ4eHhOj/37t3D4MGDce/ePXz11Ve4fv06duzYgcmTJ2PkyJGQy7M+zi9RogSqVq2K9evXo1GjRgCABg0a4OLFi7hx44ZOQWvs2LE4efIkhgwZgvDwcNy8eRM7duzAkCFDxBi2Xk2bNkVgYCDatm2Lffv2IS4uDidPnsSECRNw/vx5pKamYsiQIQgLC8OdO3dw4sQJnDt3TrsEaW6MGTMGs2fPxqZNmxAdHY1x48YhPDwcw4YN09v/v//9L27evIkxY8YgOjoaGzZswOrVq/NpxFQcCYIApVIp2oxSzuwjUb169UrsEIiIjI65j4ikiLmPKH9wNaqihbmPiKSK+e/dwsLCUL16dZ22vn37Yvny5di9ezfGjBmDatWqwdHREX379sXEiRN1+jZs2BDh4eHaYp+joyOqVKmCx48fo3Llytp+VatWxZEjRzBhwgTUr18fgiDAy8sLnTt3LvAx5pZMJsPu3bsxYcIE9O7dG0+ePIGLiwsaNGiA0qVLQ6FQ4NmzZ+jZsyceP36MkiVLon379pg6dWqurzF06FAkJSVh1KhRSEhIQJUqVbBz507t3oZvc3Nzw9atWzFixAj8/PPPqF27NmbMmIE+ffrk17CpGHpzuVpjkwlcuDZfvHz5Evb29khKSoKdnZ3Y4RQJKpUKkZGR8PPzg0KhEDscIiKjYO4jIili7iP6cCEhWf/8+mugfn1xY6HcYe4jIqkyVv5TKpWIjY2Fp6cnLCwsCuw6RES5IQgCUlNTYWlpqbNnqaFcld81JS7jSURERERERFRE8Ou6RERERET0Nhb7SDQymQyurq46VW4iouKOuY+IpIi5j4ikiLmPiKSK+Y+IpMrMzEy0a3PPPhKNXC6Hk5OT2GEQERkVcx8RSRFzHxFJEXMfEUkV8x8RSZFMJoOJiXglN87sI9GoVCpcv34dKpVK7FCIiIyGuY+IpIi5j4ikiLmPiKSK+Y+IpEizZ58g0rr7LPaRqJRKpdghEBEZHXMfEUkRcx8RSRFzHxFJFfMfEUmRWIU+gMU+IiIiIiIioiJDxM8PiIiIiIiokGKxj4iIiIiIiIiIiIiIiKiIYrGPRCOXy1GhQgXI5bwNiUg6mPuISIqY+4jyD2f2FR3MfUQkVcx/RCRV5ubmol3bRLQrk+TJZDLY2dmJHQYRkVEx9xGRFDH3EeUfFvuKDuY+IpIq5j8ikiKZTAaFQiHa9fn1ChKNSqVCZGQkVCqV2KEQERkNcx8RSRFzHxFJEXMfEUkV85+0NWrUCMOHDzf6dT08PLBgwYIPOkdoaCjatm2bYx+xxkeFnyAISElJgSDSt/NY7CNR8T/6RCRFzH1EJEXMfUQkRcx9RCRVzH85M1QwWr16NRwcHIweT25MmTIFMpksxx8iEg+LfUREREREREREREREBCCrWKtWq3XaRo8ejUePHml/ypcvj2nTpum0va+MjIwPDZlI8ljsIyIiIiIiIiIiIiIqZDTLSk6dOhWlSpWCnZ0d/vvf/yI9PV3bp1GjRhgyZAiGDBkCe3t7lCxZEt9++63OUoJpaWkYPXo0ypUrB2tra9SpUwdhYWHa45oZhTt37kSVKlVgbm6Ou3fv6sRiY2MDFxcX7Y9CoYCtra1Om4ZarcbXX38NR0dHuLi4YMqUKTrnkslkWLx4Mf7zn//A2toa33//PQBgx44dCAgIgIWFBSpUqICpU6ciMzMTQNYSiVOmTIGbmxvMzc1RtmxZDB06VOe8KSkp6NOnD2xtbeHm5obffvtN53hkZCQaN24MS0tLODk5YcCAAUhOTjb4+r9+/Ro9e/aEjY0NypQpg3nz5uXwbhGJi8U+Eo1cLkflypUhl/M2JCLpYO4jIili7iPKPyJtAULvgbmPiKSqMOS/RR8twr8j/tU+PvrdUSz6aBFSn6cCAJ7feo5FHy3C2V/PavtsD92OlZ+u1D6+tfcWFn20CLf23tK2rfx0JbaHbtc+PvvrWSz6aFEBjgQ4ePAgrl27hrCwMPzxxx/Ytm0bpk6dqtNnzZo1MDExwdmzZ7Fw4UL8+OOPWL58ufb4kCFDcOrUKWzcuBGXL19Gx44d0aJFC9y8eVPbJyUlBbNnz8by5ctx9epVODs7v3fMa9asgbW1Nc6cOYMffvgB06ZNw/79+3X6TJkyBe3atUNkZCT69OmDY8eOoWfPnhg2bBiioqKwdOlSrF69WlsI3Lp1K+bPn4+lS5fi5s2b2L59O/z8/HTOOW/ePNSsWROXLl3C4MGDMWjQIERHRwPIKtwFBwejRIkSOHfuHDZv3owDBw5gyJAhBscxZswYHDlyBDt27MC+ffsQFhaGixcvvvfrQsWfhYWFaNc2Ee3KRADMzMzEDoGIyOiY+4hIipj7iEiKmPuISKqY//KPmZkZVq5cCSsrK3z00UeYNm0axowZg+nTp2sLqq6urpg/fz5kMhkqV66MyMhIzJ8/H/3798fdu3exatUq3L17F2XLlgWQtSTnv//+i1WrVmHGjBkAspbSXLRoEapVq/bBMVetWhWTJ08GAFSqVAm//PILDh48iGbNmmn7dO3aFb1799Y+7tOnD8aNG4devXoBACpUqIDp06fj66+/xuTJk3H37l24uLigadOmMDU1hZubG2rXrq1z3ZYtW2Lw4MEAgLFjx2L+/Pk4fPgwKleujA0bNkCpVOL333+HtbU1AOCXX35BSEgIZs+ejdKlS+ucKzk5GStWrMC6devQpEkTAFlFzPLly3/w60PFl5h7V7LYR6JRq9WIjIyEn58fFAqF2OEQERkFcx8RSRFzHxFJEXMfEUlVYch/g68O1nncYGIDNJjYQPvYsaJjtj5tV7fVeVwxuCIqXq2o09bneB+dx7W/rI3aX+oWnPJbtWrVYGVlpX0cGBiI5ORk3Lt3D+7u7gCATz75RKfIEBgYiHnz5kGlUiEyMhIqlQre3t46501LS4OTk5P2sZmZGapWrZovMb99njJlyiAhIUGnrWbNmjqPIyIicOLECe1MPiBr70ClUomUlBR07NgRCxYsQIUKFdCiRQu0bNkSISEhMDH5X4njzevKZDK4uLhor3vt2jVUq1ZNW+gDgHr16kGtViM6OjpbsS8mJgbp6emoU6eOts3R0RGVK1fO68tBEpKamgpLS0tRrs1iHxERERERERERERGRkdjZ2SEpKSlb+4sXL2Bvb5+v10pOToZCocCFCxeyFV9tbGy0/25paZlvs5JMTU11HstkMqjVap22N4tumjinTp2K9u3bZzufhYUFXF1dER0djQMHDmD//v0YPHgw5syZgyNHjmivl5vrEhVXLPYRERERERERFRHcs4+IiKjoq1y5Mvbt25et/eLFi9lm4EVEROjMFjp9+jRsbGzg6uqq7XPmzBmd55w+fRqVKlWCQqFA9erVoVKpkJCQgPr16xfAaPJHQEAAoqOjUbFiRYN9LC0tERISgpCQEHz55Zfw8fFBZGQkAgIC3nl+X19frF69Gq9fv9YWGk+cOKHdY/JtXl5eMDU1xZkzZ+Dm5gYASExMxI0bN9CwYcP3HCVRwWGxj4iIiIiIiIiIiIjISAYNGoRffvkFQ4cORb9+/WBubo5du3bhjz/+wN9//63TNz09HX379sXEiRMRFxeHyZMnY8iQIdr9+gDg7t27GDlyJAYOHIiLFy/i559/xrx58wAA3t7e6NatG3r27Il58+ahevXqePLkCQ4ePIiqVauiVatWRh27IZMmTULr1q3h5uaGDh06QC6XIyIiAleuXMF3332H1atXQ6VSoU6dOrCyssK6detgaWmpXcr0Xbp164bJkyejV69emDJlCp48eYKvvvoKPXr0yLaEJ5A167Fv374YM2YMnJyc4OzsjAkTJui87kSFCYt9JBq5XA4/Pz8mSCKSFOY+IpIi5j4ikiLmPiKSKua/d6tQoQKOHj2KCRMmoGnTpkhPT4ePjw82b96MFi1a6PRt0qQJKlWqhAYNGiAtLQ1dunTBlClTdPr07NkTqampqF27NhQKBYYNG4YBAwZoj69atQrfffcdRo0ahQcPHqBkyZL45JNP0Lp1a2MMN1eCg4Pxzz//YNq0aZg9ezZMTU3h4+ODfv36AQAcHBwwa9YsjBw5EiqVCn5+fvj777919h3MiZWVFfbu3Ythw4ahVq1asLKywueff44ff/zR4HPmzJmD5ORkhISEwNbWFqNGjdK7/CqRhlj79QGATBC4CEh+ePnyJezt7ZGUlAQ7OzuxwykSBEGAUqmEhYVFvq0HTURU2DH3EZEUMfcRfbiQkKx/DhsGNG0qbiyUO8x9RCRVxsp/SqUSsbGx8PT0hIWFRYFdR0yhoaF48eIFtm/fbrBPo0aN4O/vjwULFhgtLiLKThAECIIAmUymk/sM5ar8rinx6xUkGrVajejoaG6SSkSSwtxHRFLE3EdEUsTcR0RSxfxHRFKlVCpFuzaLfURERERERERERERERERFFPfsIyIiIiIiIioiuBEHERGRdKxevfqdfcLCwgo8DiIq/Dizj0SlUCjEDoGIyOiY+4hIipj7iEiKmPuISKqY/4iIjIsz+0g0CoUCfn5+YodBRGRUzH1EJEXMfUQkRcx9RCRVzH9EJEUymQxWVlaiXZ8z+0g0giDg5cuXELgODRFJCHMfEUkRcx8RSRFzHxFJFfMfEUmRIAhQqVSi5T4W+0g0arUat2/fhlqtFjsUIiKjYe4jIili7iPKP/zctOhg7iMiqWL+IyKpSktLE+3aLPYRERERERERFREs9hERERER0dtY7CMiIiIiIiIiIiIiIiIqoljsI1FZWFiIHQIRkdEx9xGRFDH3EZEUMfcRkVQVpfwXHx+P76ZPRy1fX3iXK4davr74bvp0xMfHix1aseHh4YEFCxaIHUaBatSoEYYPH659bIwxHzx4EL6+vlCpVAV6HTLsiy++wLx587SPZTKZaLGw2EeiUSgU8PHxgUKhEDsUIiKjYe4jIili7iMiKWLuIyKpKkr5b/68eXAvXx7HZ8zA8OvXsfjhQwy/fh3HZsyAe/nymP/jjwVy3dDQUMhkMshkMpiamqJ06dJo1qwZVq5cmee9DqdMmQJ/f/98j1GMAt39+/dhZmaGjz/+2KjXLQjnzp3DgAEDCvQaX3/9NSZOnKj9XVOpVJg1axZ8fHxgaWkJR0dH1KlTB8uXLy/QOIqC1atXa3/nND+5+VJCWFgYAgICYG5ujooVK2L16tU6xydOnIjvv/8eSUlJkMlksLS0FK3gx2IfiUatVuPZs2fcrJeIJIW5j4ikiLmPKP9wz76ig7mPiKSqqOS/+fPmYeaECTiqUuFfpRLdADQB0A3AXqUSR1UqzPzmmwIr+LVo0QKPHj1CXFwc9uzZg6CgIAwbNgytW7dGZmZmgVyzsFu9ejU6deqEly9f4syZM2KH80FKlSoFKyurAjv/8ePHERMTg88//1zbNnXqVMyfPx/Tp09HVFQUDh8+jAEDBuDFixcFFkd6evp7P/fu3bv5GMm72dnZ4dGjR9qfO3fu5Ng/NjYWrVq1QlBQEMLDwzF8+HD069cPe/fu1fb5+OOP4eXlhXXr1kEQBGRmZkIQ6Q92FvtINIIg4N69e6Ld/EREYmDuIyIpYu4jIili7iMiqSoK+S8+Ph7jxo7F32lpqGOgTx0Af6elYdzXXxfIkp7m5uZwcXFBuXLlEBAQgG+++QY7duzAnj17dGYPvXjxAv369UOpUqVgZ2eHxo0bIyIiAkBWcWzq1KmIiIjQzlbSPDen52n8/fffqFWrFiwsLFCyZEm0a9cOQNaSlHfu3MGIESO059U4fvw46tevD0tLS7i6umLo0KF4/fq19nhCQgJCQkJgaWkJT09PrF+/PlevhyAIWLVqFXr06IGuXbtixYoVOsfj4uIgk8mwbds2BAUFwcrKCtWqVcOpU6d0+m3duhUfffQRzM3N4eHhobPEIpA1Y/G7775Dz549YWNjA3d3d+zcuRNPnjxBmzZtYGNjg6pVq+L8+fPa5zx79gxdunRBuXLlYGVlBT8/P/zxxx85juftmZHvej8iIiIQFBQEW1tb2NnZoUaNGjoxvG3jxo1o1qyZzuy0nTt3YvDgwejYsSM8PT1RrVo19O3bF6NHj9b2UavV+OGHH1CxYkWYm5vDzc0N33//vfZ4ZGQkGjduDEtLSzg5OWHAgAFITk7WHg8NDUXbtm3x/fffo2zZsqhcuTIA4N69e+jUqRMcHBzg6OiINm3aIC4uLsfXyNPTE02bNsXatWuRkpKSY9/8IJPJ4OLiov0pXbp0jv2XLFkCT09PzJs3D76+vhgyZAg6dOiA+fPn6/QLCQnBxo0bAXxY8fNDsdhHRERERERERERERJKxfNkyBJmaGiz0adQB0MjMDCuMtAxi48aNUa1aNWzbtk3b1rFjRyQkJGDPnj24cOECAgIC0KRJEzx//hydO3fGqFGj8NFHH2lnK3Xu3PmdzwOAXbt2oV27dmjZsiUuXbqEgwcPonbt2gCAbdu2oXz58pg2bZr2vAAQExODFi1a4PPPP8fly5exadMmHD9+HEOGDNHGGxoainv37uHw4cPYsmULFi1ahISEhHeO/fDhw0hJSUHTpk3RvXt3bNy4UaeIqDFhwgSMHj0a4eHh8Pb2RpcuXbQzIS9cuIBOnTrhiy++QGRkJKZMmYJvv/0229KL8+fPR7169XDp0iW0atUKPXr0QM+ePdG9e3dcvHgRXl5e6Nmzp7ZgrVQqUaNGDezatQtXrlzBgAED0KNHD5w9eza3b+07349u3bqhfPnyOHfuHC5cuIBx48bB1NTU4PmOHTuGmjVr6rS5uLjg0KFDePLkicHnjR8/HrNmzcK3336LqKgobNiwQVv0ev36NYKDg1GiRAmcO3cOmzdvxoEDB3TeXyBrr8Do6Gjs378f//zzDzIyMhAcHAxbW1scO3YMJ06cgI2NDVq0aJFj8SsqKgq1a9fGxIkTUbp0afTp0wdHjhzR+0WB9evXw8bGJsefY8eOGbwWACQnJ8Pd3R2urq5o06YNrl69mmP/U6dOoWnTpjptwcHB2QrMtWvXxtmzZ5GWlpbj+QqcQPkiKSlJACAkJSWJHUqRkZmZKVy6dEnIzMwUOxQiIqNh7iMiKWLuI/pwrVtn/ezeLXYklFvMfUQkVcbKf6mpqUJUVJSQmpqa5+fW9PER1mWtjv3On3WAUNPXN19j79Wrl9CmTRu9xzp37iz4/v/1jh07JtjZ2QlKpVKnj5eXl7B06VJBEARh8uTJQrVq1XSO5+Z5gYGBQrdu3QzG6O7uLsyfP1+nrW/fvsKAAQOyXUsulwupqalCdHS0AEA4e/as9vi1a9cEANnO9bauXbsKw4cP1z6uVq2asGrVKu3j2NhYAYCwfPlybdvVq1cFAMK1a9e052jWrJnOeceMGSNUqVJFZ1zdu3fXPn706JEAQPj222+1badOnRIACI8ePTIYb6tWrYRRo0ZpHzds2FAYNmyYznU0Y87N+2FrayusXr3a4PXeZm9vL/z+++86bVevXhV8fX0FuVwu+Pn5CQMHDhR2v/HH48uXLwVzc3Nh2bJles/522+/CSVKlBCSk5O1bbt27RLkcrkQHx8vCELWvVu6dGkhLS1N22ft2rVC5cqVBbVarW1LS0sTLC0thb17975zLGq1Wjh06JAQGhoq2NraCp6ensLkyZOF27dv68R+8+bNHH9SUlIMXuPkyZPCmjVrhEuXLglhYWFC69atBTs7O+HevXsGn1OpUiVhxowZOm27du0SAOhcKyIiQgAgxMbGCq9fv9Z5HQTBcK7K75qSiVhFRiIAsLW1FTsEIiKjY+4jIili7iMiKWLuIyKpKuz5L+nlS7jksm9pAElJSQUZjg5BELTLZkZERCA5ORlOTk46fVJTUxETE2PwHLl5Xnh4OPr375+n2CIiInD58mWdpTkFQYBarUZsbCxu3LgBExMT1KhRQ3vcx8cHDg4OOZ73xYsX2LZtG44fP65t6969O1asWIHQ0FCdvlWrVtX+e5kyZQBkLR3q4+ODa9euoU2bNjr969WrhwULFkClUkGhUGQ7h2ZWm5+fX7a2hIQEuLi4QKVSYcaMGfjzzz/x4MEDpKenIy0tLdd78uXm/Rg5ciT69euHtWvXomnTpujYsSO8vLwMnjM1NVVnCU8AqFKlCq5cuYILFy7gxIkTOHr0KEJCQhAaGorly5fj2rVrSEtLQ5MmTfSe89q1a6hWrRqsra21bfXq1YNarUZ0dLTOa2VmZqYzvlu3bmX7vVcqlTnepxoymQxBQUEICgrCwoULMWjQIEydOhXh4eHYvn07gKyc8iF5JTAwEIGBgdrHdevWha+vL5YuXYrp06e/93kBwNLSEgCQkpICuVy8xTRZ7CPRKBSKHBMWEVFxxNxHRFLE3EdEUsTcR0RSVRTyn72dHeIfPsxV38cA7O3tCzagN1y7dg2enp4AspYdLFOmDMLCwrL1y6mAlpvnaQoUeZGcnIyBAwdi6NCh2Y65ubnhxo0beT4nAGzYsAFKpRJ16vxvYVVNEfHGjRvw9vbWtr+5tKWmKKpWq/N0PX3nyOm8c+bMwcKFC7FgwQL4+fnB2toaw4cPz/X+bLl5P6ZMmYKuXbti165d2LNnDyZPnoyNGzdq91F8W8mSJZGYmJitXS6Xo1atWqhVqxaGDx+OdevWoUePHpgwYcJ7vef6vFkMBLLGV6NGDb37M5YqVSpX57x48SJ+//13/PHHH5DJZNrip8b69esxcODAHM+xZ88e1K9fP1fXMzU1RfXq1XHr1i2DfVxcXPD48WOdtsePH8POzk7ntdQsxers7JytAGtMLPaRaNRqNRISEuDs7CxqxZuIyJiY+4hIipj7iEiKmPuISKqKQv5r07Ur1s6YgW5K5Tv7/m5pibZduxohKuDQoUOIjIzEiBEjAAABAQGIj4+HiYkJPDw89D7HzMwMKpVKpy03z6tatSoOHjyI3r175+m8UVFRqFixot7n+Pj4IDMzExcuXECtWrUAANHR0Xjx4oWBEWdZsWIFRo0alW0W3+DBg7Fy5UrMmjUrx+dr+Pr64sSJEzptJ06cgLe3t3ZW3/s4ceIE2rRpg+7duwOAtghZpUqVXD0/N+8HAHh7e8Pb2xsjRoxAly5dsGrVKoPFvurVqyMqKuqd19bE+Pr1a1SqVAmWlpY4ePCgTiFNw9fXF6tXr8br16+1Bb0TJ05ALpejcuXKOY5v06ZNcHZ2hp2d3Ttj0rh//z7WrVuHtWvXIiYmBiEhIVixYgVatGgBExPd0tV//vMfnWKwPuXKlcv1tVUqFSIjI9GyZUuDfQIDA7F7926dtv379+vMEASAK1euoHz58nByckJGRgZMTEy0BWNjKpzZliRBEATEx8fr3XCTiKi4Yu4jIili7iMiKWLuIyKpKgr5r1///jickYEz7+h3BkBYejr66imMfKi0tDTEx8fjwYMHuHjxImbMmIE2bdqgdevW6NmzJwCgadOmCAwMRNu2bbFv3z7ExcXh5MmTmDBhAs6fPw8A8PDwQGxsLMLDw/H06VOkpaXl6nmTJ0/GH3/8gcmTJ+PatWuIjIzE7NmztfF5eHjg6NGjePDgAZ4+fQoAGDt2LE6ePIkhQ4YgPDwcN2/exI4dOzBkyBAAQOXKldGiRQsMHDgQZ86cwYULF9CvX78cZ5SFh4fj4sWL6NevHz7++GOdny5dumDNmjXIzMzM1Ws6atQoHDx4ENOnT8eNGzewZs0a/PLLLxg9enTe36A3VKpUCfv378fJkydx7do1DBw4MNuMr5y86/1ITU3FkCFDEBYWhjt37uDEiRM4d+4cfH19DZ4zODhYZ9lTAOjQoQPmz5+PM2fO4M6dOwgLC8OXX34Jb29v+Pj4wMLCAmPHjsXXX3+N33//HTExMTh9+jRWrFgBAOjWrRssLCzQq1cvXLlyBYcPH8ZXX32FHj16aJfw1Kdbt24oWbIk2rRpg2PHjiE2NhZhYWEYOnQo7t+/b/B57u7u+Ouvv/Dll1/i0aNH2Lx5M1q3bp2t0AdkLeNZsWLFHH9yus+mTZuGffv24fbt27h48SK6d++OO3fu6BQ9x48fr/3dA4D//ve/uH37Nr7++mtcv34dixYtwp9//qktxmscO3YMzZs3BwBkZGQYjKGgsdhHRERERERERERERJLh4uKCWbNnI8Tc3GDB7wyAEHNzzPrhB7i45HaHv9z7999/UaZMGXh4eKBFixY4fPgwfvrpJ+zYsUM7C00mk2H37t1o0KABevfuDW9vb3zxxRe4c+eOtvjy+eefo0WLFggKCkKpUqW0yyC+63mNGjXC5s2bsXPnTvj7+6Nx48Y4e/asNr5p06YhLi4OXl5e2qUYq1atiiNHjuDGjRuoX78+qlevjkmTJqFs2bLa561atQply5ZFw4YN0b59ewwYMADOzs4GX4cVK1agSpUq8PHxyXasXbt2SEhIyDa7ypCAgAD8+eef2LhxIz7++GNMmjQJ06ZNyzZjMK8mTpyIgIAABAcHo1GjRnBxcUHbtm1z/fx3vR8KhQLPnj1Dz5494e3tjU6dOuGzzz7D1KlTDZ6zW7duuHr1KqKjo7VtwcHB+PvvvxESEgJvb2/06tULPj4+2Ldvn7aA9u2332LUqFGYNGkSfH190blzZyQkJAAArKyssHfvXjx//hy1atVChw4d0KRJE/zyyy85js/KygpHjx6Fm5sb2rdvD19fX/Tt2xdKpTLHmX5Xr17FmTNnMHjwYJQoUSLXr+f7SExMRP/+/eHr64uWLVvi5cuXOHnypM7szEePHuHu3bvax56enti1axf279+PatWqYd68eVi+fDmCg4O1fZRKJbZv357n/S8LgkwozF+xKEJevnwJe3t7JCUl5WmqqpRppsr6+fl90DRqIqKihLmPiKSIuY/ow4WEZP1z8GDgs8/EjYVyh7mPiKTKWPlPqVQiNjYWnp6e771P1vwff8S4r79GI1NT9FQqURpAPIDfLSxwJCMDs374ASNGjszXuInyy5gxY/Dy5UssXbpU7FAka/Hixfjrr7+wb98+CIKA1NRUWFpa6izjaShX5XdNiTP7SDQymQyOjo6irF9LRCQW5j4ikiLmPiKSIuY+IpKqopT/RowciTv376PBhAlY4OuLwWXLYqGvLxpOmIA79++z0EeF2oQJE+Du7g61Wi12KJJlamqKn3/+WftYzC94cWZfPuHMPiIiIiIiIioompl9X34JtGghbixERESFQX7M7CMiKmic2UfFnlqtxt27d/nNAyKSFOY+IpIi5j4ikiLmPiKSKuY/IpIiQRCQlpYGsebXsdhHohEEAc+fPxft5iciEgNzHxFJEXMfEUkRcx8RSZWx8x/zLBEVFiqVKlubsXIUi31ERERERERERQQ/zyQiIspiamoKAEhJSRE5EiIiwzQ5SpOzCopJgZ6diIiIiIiIiIiIiCifKRQKODg4ICEhAQBgZWUFmUwmclREJFWaZTxlMhlkMhkEQUBKSgoSEhLg4OAAhUJRoNdnsY9EI5PJ4OLiwv8IE5GkMPcRkRQx9xGRFDH3EZFUGTP/ubi4AIC24EdEJBZBEKBWqyGXy3Xyn4ODgzZXFSQW+0g0crncKDc5EVFhwtxHRFLE3EdEUsTcR0RSZcz8J5PJUKZMGTg7OyMjI8Mo1yQiyi1TU9MCn9GnwWIfiUalUiEuLg4eHh5Gu+GJiMTG3EdEUsTcR5R/OEms6GDuIyKpEiP/KRQK5loiEpXYf/vJjX5Foje8evVK7BCIiIyOuY+IpIi5j4ikiLmPiKSK+Y+IpEjM3MdiHxEREREREREREREREVERxWIfERERERERERERERERURHFYh+JRiaTwdXVFTJuOkFEEsLcR0RSxNxHRFLE3EdEUsX8R0RSJHbuMxHlqkQA5HI5nJycxA6DiMiomPuISIqY+4jyjyCIHQHlFnMfEUkV8x8RSZHYuY8z+0g0KpUK169fh0qlEjsUIiKjYe4jIili7iMiKWLuIyKpYv4jIikSO/ex2EeiUiqVYodARGR0zH1EJEXMfUT5gyuiFS3MfUQkVcx/RCRFYuY+FvuIiIiIiIiIiIiIiIiIiigW+4iIiIiIiIiIiIiIiIiKKBb7SDRyuRwVKlSAXM7bkIikg7mPiKSIuY+IpIi5j4ikivmPiKRI7NxnIspViQDIZDLY2dmJHQYRkVEx9xGRFDH3EZEUMfcRkVQx/xGRFImd+/j1ChKNSqVCZGQkVCqV2KEQERkNcx8RSRFzHxFJEXMfEUkV8x8RSZHYuY/FPhIV/6NPRFLE3EdEUsTcR0RSxNxHRFLF/EdEUiRm7mOxj4iIiIiIiIiIiIiIiKiIYrGPiIiIiIiIiIiIiIiIqIhisY9EI5fLUblyZcjlvA2JSDqY+4hIipj7iEiKmPuISKqY/4hIisTOfcy4JCozMzOxQyAiMjrmPiKSIuY+IpIi5j4ikirmPyKSIjFzH4t9JBq1Wo3IyEio1WqxQyEiMhrmPiKSIuY+IpIi5j4ikirmPyKSIrFzH4t9REREREREREREREREREUUi31ERERERERERERERERERRSLfURERERERERERERERERFFIt9JBq5XA4/Pz/I5bwNiUg6mPuISIqY+4hIipj7iEiqmP+ISIrEzn3MuCSq9PR0sUMgIjI65j4ikiLmPiKSIuY+IpIq5j8ikiIxcx+LfSQatVqN6OhoqNVqsUMhIjIa5j4ikiLmPiKSIuY+IpIq5j8ikiKxcx+LfURERERERERERERERERFFIt9RERERERERIWYIIgdQcGIOxKH80vOIyM1Q+xQiIiIiIiKNBb7SFQKhULsEIiIjI65j4ikiLmPiN525Y8r2DVoFzJTM8UOpcAw9xGRVDH/EZEUiZn7ZIJQXL8jaFwvX76Evb09kpKSYGdnJ3Y4REREREREVEwIAvCf/2T9+5dfAi1aiBtPfnka/RQv77+EewN3KEz5oTARERERSUd+15Q4s49EIwgCXr58CdabiUhKmPuISIqY+4hIn9iDsTg6/SiUiUqxQykQzH1EJFXMf0QkRWLnPhb7SDRqtRq3b9+GWq0WOxQiIqNh7iMiKWLuIyJ9Xj16hSdXn0CVrhI7lALB3EdEUsX8R0RSJHbuY7GPiIiIiIiIiIzu+c3nSHmWApsyNmKHQkRERERUpJmIHQARERERERERSU/ZWmWhSldBJpeJHQoRERERUZHGYh+JysLCQuwQiIiMjrmPiKSIuY+I3ubV3Av2bvZIT06Hua252OEUCOY+IpIq5j8ikiIxcx+X8STRKBQK+Pj4QKFQiB0KEZHRMPcRkRQx9xGRPlf+uIItnbYgOT5Z7FAKBHMfEUkV8x8RSZHYuY/FPhKNWq3Gs2fPuFkvEUkKcx8RSRFzHxHpk/IsBfZu9rBxKZ579jH3EZFUMf8RkRSJnftY7CPRCIKAe/fuQRAEsUMhIjIa5j4ikiLmPiLS58XtF1AmKYvtEp7MfUQkVcx/RCRFYuc+7tlHREREREREREbXcXNHZCozxQ6DiIiIiKjI48w+IiIiIiIiIjK6iysuYnHVxXhy7YnYoRARERERFWks9pGobG1txQ6BiMjomPuISIqY+4jeX3FdBS09OR1yEzkUZgqxQykwzH1EJFXMf0QkRWLmPhb7SDQKhQJeXl5QKIrv/9gREb2NuY8K2vNbz/Hg7AOoVeJsCE2kD3Mf0YdJvJ0Ir/N/wiQtGTKZ2NHkn9iDsRBUAhy9HMUOpUAw9xGRVDH/EZEUiZ37WOwj0ajVasTHx0Ot5oeRRCQdzH1U0I5MPYLldZZDncF7jAoP5j6iD3NkahhKxF+Dc9xZsUPJV9V6VUPgqECxwygwzH1EJFXMf0QkRWLnPhb7SDSCICA+Ph5CcV2ThohID+Y+Kmgfd/kYTX9oCrkp/8yjwoO5j+jDtF3TDudbT8bDyo2L1ZKeJX1KIiMlAy8fvBQ7lALB3EdEUsX8R0RSJHbu46dARERERMXIvZP3cH7xeaS/Shc7FCIiohzdO3EPR6YewauHr8QOhYiIiIioSGOxj4iIiKgYMbE0gWUJS7HDICKifBR3OBaVzqyDzfO7YoeSrx5deIRSVUrB+WNnsUMhIhLF91bfY6psqthhEBFRMWAidgAkXTKZDI6OjpAVpx3miYjegbmP3kWVroLC7P03c06OT4aptSksHCzyMSqiD8PcR/RhTs09CfsnMcgwtwHgJnY4+UaZpIRMIYOppanYoRQI5j4iepfM1EyxQygQzH9EJEVi5z4W+0g0crkcbm7F539UiYhyg7mPchK5IRLbum1Dr8O94NHI473Okf4qHanPU/M3MKIPxNxH9GHqfdMAp6/a4LFHHbFDyVed/+qM9OT0D/6iS2HF3EdE79LzUE+oM9Vih5HvmP+ISIrEzn1cxpNEo1arcffuXajVxe+PGiIiQ5j7KCfKJCUAIO1Vmt7jMftisKn9phw/EKjWsxr8Q/2Rnsw9+6jwYO4j+jCudV0RV60NUu1dUJwmSVz47QLmOs/F/TP3xQ6lQDD3EdG7eAZ5wquZl9hh5DvmPyKSIrFzH4t9JBpBEPD8+XMIgiB2KERERsPcRznxD/XHqPhR8Gqu/3/4w6aE4fpf15EYm2jwHNd3XMf+MfuR9lJ/wZBIDMx9RB9GUAuQqTKBYvahacqTFDh4OMDGxUbsUAoEcx8Rvcu1bddw5uczYoeR75j/iEiKxM59XMaTiIiIqJBQpanw6sErKEwVMDHP/mdayLIQJN5OhL2rvcFzOFZ0xCcjPoFVSauCDJWIqNA6MO4AVBkqBM8LFjuUfPN3vx2osScCT139AbQRO5x8c3P3TUAGOFVyEjsUIiJR/Pn5nwCAOl8Vr2WaiYjI+Dizj4iIiKiQiNoahd9q/IZr267pPe78kTMqh1SGiYXh72tFrotE1JaoYrn3ERFRbpyYfQKnfzwtdhh5cvfEXYNLOAOAvVvWlzyU1sWrKNZ0VlN89tNnYodBRCSa8p+UR+lqpcUOg4iIigHO7CPRyGQyuLi4QFacNp0gInoH5j7KydNrTwEY3rPvwm8XcOKHE/hi+xdw/thZb59mc5sh/VV61pJvct5nVDgw95Ex2bvZw9zOXOwwci3xdiJWfboKdYbVQYsFLfT2aTglCPPDgwAAxWlFNJlchojfI2DvZo/SVYvfh93MfUT0Ln1P9RU7hALB/EdEUiR27mOxj0Qjl8vh4uIidhhEREbF3Ec5+bjLxzC1MkXlkMp6j/87/F9kpmbmuB/fjb9v4OSckxgaMxQlKpQoqFCJ8oS5j4yp255uQBH6bFGdqYbCXAELBwuxQzG6xNuJiNochYB+AcWy2MfcR0Tvos5UQ5Whgqmlqdih5CvmPyKSIrFzH5fxJNGoVCrExMRApVKJHQoRkdEw91FOytYoi6BpQXCs6Kj3eOPvG6Nii4pwqW74j8cyAWVQtXtVmFoXrw8MqGhj7iNjUqvUOX4porBx8nbCROVENJrSyGCfqC1RqPnPVDjdu2S8wIwgcn0kXOu6wqu5l9ihFAjmPiJ6l+mm0zHDaobYYeQ75j8ikiKxcx+LfSSqV69eiR0CEZHRMfeRIY8uPsKKwBWI2hKl93jgiEB029Mtx2/+nl5wGo8uPYJNaZuCCpPovTD3kbFs67YNa5uuFTuMXEt9nop/Bv1jMPcDQPT2rL1cHeJvoDitiObg6VDsZ6Ez9xFRbgjFaY3m/8f8R0RSJGbu4zKeRERERIVE1JYo3D99H3eO3kGVDlWyHX8c+Ri399+Gb3tfOHg46D1HudrlkKnMLOBIiYgKr4TIBLFDyJPkx8m4sOQCXj14pTf3A0Djmc3w1+2qSLEva+ToClbTWU3x9PpTpDxNgVVJK7HDISIyuqG3h0Iml3FvOyIi+mCc2UdERERUSDhWcoSlkyWqdNT/Ye+eIXuwb9Q+PL3+1OA5PJt4Qq1S49VDfpOWiKSp7pi6CPouSOwwck2VnrXMT0nfkgb72JW3w0vnSsg0tzZWWEYRsy8Ga4LW4N6pe2KHQkQkihKeJeDg7iB2GEREVAxwZh+JRiaTwdXVld9eIiJJYe6jnFTvXR3Ve1c3eFxQZy3vU65OOYN9Hl18hPCV4QgcGQjbsrb5HiPR+2DuI2Nq9kMzsUPIE3s3e4QsD0GZgDIG+6Qnp8M6MQHplvYAik9uf3j+IZw/dobzx85ih1IgmPuI6F2OzzqOeyfvoeOfHWFiUXw+pmX+IyIpEjv3FZ//ilCRI5fL4eTkJHYYRERGxdxHOUlNTMXNXTdRumpplK5aOtvxTls7Ie1VGixLWBo8h8JUgfoT66Okj+EZIkTGxtxHxrR7yG48PPcQ/c70EzuUXDG3NUf5OuVhUcLCYJ+9Q3fD90QEkkpVBNDNeMEVsJi9MVCYK1DCs3ju28fcR0TvcnD8QQBZs7yLU7GP+Y+IpEjs3MdlPEk0KpUK169fh0qlEjsUIiKjYe6jnMQdjsNfPf7CxeUX9R63dLKEXXk7qFVqg+e48scVXNt6DXIF/8yjwoO5j4zp3K/n8ODsA7HDyDVlkhKL/Rbj6PSjBvu4N/IAADwv+5GRojKOLn93QactncQOo8Aw9xHRu9QdUxeNpjaCma2Z2KHkK+Y/IpIisXNf8fnKCBVJSqVS7BCIiIyOuY8M0ezFZ2g5s1PzTuHA2APo8ncXeLf21tun3dp2eHn/JTJSMmBqZVpgsRLlFXMfGYu1szWcKhed2QQv4l4AAF7ef2mwT9We/pj8l79xAjKixNhE7Bu5D81/bI6KwRXFDqdAMPcRUU6K2tLTecH8R0RSJGbuY7GPiIiIqJCo0rEKSniVgGeQp97jR6YdAQDYljO8X9Otvbdw6JtDGHBhQI77PxERFVcj7o0QO4Q8MbM2Q5mAMvBt7yt2KEanTFRC+UIJCGJHQkQkjpf3XyL5cTJK+5WGwkwhdjhERnFg/AHEHYorMkuuExUVXN+JiIiIqJBwquSEjzt/DGtna73HWyxogfKB5XF+8Xk8u/lMbx9HL0fUGlILtmUNFwSJiAAg4UoC9o/dj/iIeLFDyVd3j9/FpZWXIKiLRgWppE9JDLgwANX7VDfYJ3zVJdT8ZyrKR+0zYmQF7+D4g7BxsUHFFsVzVh8R0bvMd52PZTWX4fWT12KHQmQ0J2adwIOzDyAIReNvNaKigsU+Eo1cLkeFChUgl/M2JCLpYO6jnLy8/xKz7GfhwLgDeo8H9AtAytMUXFx2EVf/vKq3z6EJh3D32F3YuNgUZKhEecLcVzg9jX6Kkz+c1C4hXFz8PeBv7Bq0C+pMw/ubFiaZaZlY1WAVjkw/YrBP/KVHAADz14nGCsso/Lr5wbdD8Z3RyNxHRLllbmsudgj5ivmPchI0PQgfdf4IMplM7/GY/TG4c/SOkaMi+nBi5z4u40mikclksLOzEzsMIiKjYu6jnFxedxlpL9Pw+rH+b/beP3Mfqc9TYelkicohlfX2CRgQAIUplwCiwoW5r3AytzOHwlyBjNcZYoeSrxJjsgpicpOi8QFjcnwy7h67i0xlJhp+21Bvn6ZzmmP57cZQKcyMHF3BCugXgJu7b+Jp9FOUrFxS7HDyXWHKfS/iXsDEwoRfBiIqZCYLk8UOoUAUpvxHhU+DiQ0MHkt/nY51zdcBeP/fj9cJr5EYm4hSvqVgble8CulUuImd+4rG//1QsaRSqRAZGQmVSiV2KERERsPcRzlxrOgIt/pu+GTkJ3qPb+2yFanPUhGyLASlq5bW28euvB2ubLyCx5cfF2SoRHnC3Fc4mdmYoUSFEjC1NhU7lHzV8teW6Li5I2Ry/d8WL2yS45MBZO3baoiJuQlUphZAMZsh8TjyMXZ/uRsPzj4QO5QCUZhy3xL/Jfirx196jwmCgBM/nEDClQQjR0VExVVhyn9U+Nw/fR/ha8KhSs9+f2j+fqvQrMJ7nz98TThWfLIC90/ff+9zEL0PsXMfZ/aRqPgffSKSIuY+MqRKhyqo0sHwh72OFR3x+vFreLf2Ntgn43UGXj18pfd/nIjExNxX+JT2K43u/3aHVSkrsUPJV7UG1xI7hDxx/sgZfU/1hb2bvcE+ibEvUP7qGSSVrgzAw2ixFbTLay/DtZ4rvJp5iR1KgSksuS8tKQ23D9zWe+xZ9DMcGHsAp348hdHxo40cGZG0be2yFVFbovDVza/g4OEgdjj5qrDkPyp8VgSuAAB4t/aGlZPu36GmlqYfPOP1cUTWF1+VScoPOg/R+xAz9xWvrwUSERERFWGZykwcGHcAUVui9B7vsa8HPII88J3Zdzj63VG9fe4euwufdj4oW7NsQYZKREXcisAVmF1iNha4L8Dt/foLAEXVri93YZpiGlKepYgdSq6Y2Zgh6W4SEm8b3o/v3M+n4RJ7GuWiD8LA9jZF0r0T9yCoBC4taQT+ffzhEeSh91iJCiXQ+3hv9D3Z17hBERGubLwCdaaaX9QjSbF2tgaQ9TdQQfDv7Y9aQ2qhXK1yBXJ+osKKM/uIiIhIstTqwrUi2p2jd3Bi9glUaFZB7wy/1MRUvHrwCgBgZqv/f4wenHsA61LWBRonERUNGakZODzpMOqPrw9LR0udY4m3E6HOVENhrkDayzSRIiwY5xedB1B09uxTq9TY0nkLvEO84fapm94+lUIq4/Da+3jo3ci4weXg2l/XsLPvTrRZ1QY+bXze6xwj7o2AIAgQBAGy4lTFLITarGhj8JjCTAG3evrvPSIqWG1WtYFVKSs4eTuJHQqR0QyOGgxVugom5tlLExmpGZhhNQPA++/ZV6FJBVRo8v7LgBZWsYdikRibiOq9qxeZ5erJuIrG//1QsSSXy1G5cmXIC/hT1qS7Sbj1760i881eIirejJX76N327gU6dwYiI8WO5H/iI+IBALWH1NZ7/Nj3xxAfHo/PN36OT4bp39ev81+dUb1vdbx69KrA4iTKK+Y+cVz/6zpOzT2F8NXh2Y41mdUEn37zKVRpKrx88NL4wRUg54+dUb1vdVjYW4gdSq7cP5W1n4yppeG9Ez0aeeL6p/3wspQXBMFYkeUsens0lIlKRG3WPxs9N24fuI3pJtNxcfnFfIys8ChMuW99y/UGVwVIjE3EkelH8PDCQyNHRUT+of7wbmV4if6iqjDlPyp8rJysYFvGVu8xVdqHz3KN2R+D32r8htsHi9fqFeeXnMff/f4WOwzKgdi5jxmXRGVmVjDTtd+0Lngd1n+2Hmd/Plvg1yIiyg1j5D56t19+AZRKYPlysSP5n+q9q2NQ5CBUaKr/W4iX110GkLXPliGxh2Kxvdd2PL32tEBiJHqX/WP34/emv2drZ+4zPuePnREwIADlA8tnO+bg4QCPhh7odbgX/Lr4iRBdwRkUOQj/Wf4fscPINWtna1TtURX+ffzFDiVP6o2rh/oT66PumLrvfY7E2EQ4VnJECc8S+RhZ4VJYct+tPbdw+NvDeo/dO3kPYZPCsLyO4T+KUhNTcfXPqxAKS7WZqJhIuJKAiN8jkPo8VexQ8l1hyX9U+OzovQNTZVORGJt9CXMzWzN8desrjH78/nvIXt10FY8uPip2/0/s9qkbKrWsxGV/Czkxcx+LfSQatVqNyMhIqNXqAr1OQP8AAIBbfS5LQkTiM1buo9wzNTyRwugsnSxhbm+OjJQMvcfbrm6Lip9VxMp6K3H7gP5vKcpN5Kg3th6c/ZwLMtQ8uX/mfrH7ViUZ9urBKzy/9VynjblPHKWrlkbI0hC4BrpmO/ZXj7+wLngdTs07hdTE4vUBY/Tf0dgzdA9SnhaNlT2cvJ3Qdk1bXF57GZEb9E83P7/4LGr+MxXep7MX0sVSyrcUGk9vDJdqLu99jsPfHoZtGVuDX3Ip6gpT7qvYoiJMrfX/0ePZ2BNOlZ0QND3I4PMv/HYBW7tsxbPoZwUVIpEkLfZbjO29tuNF3AuxQ8lXYuW/zExgyRLg9GmjXpbeknQ3CTH7YgwWsTWrTphYZF/GU66Qw9HLUbuv3/twrOgImVwG13rZ/wYuyp5ce4Kbu28iPTld7FDIALH/9mOxj4q9wJGBmCxMLpZrNRMRUfGS9jINC9wWYNegXXqPV2xREWlJaVC+UOLGPzf09gmbFIa7x+/med++80vOY2uXrUh/nf//47B/zH5s7bI1389LhVPDyQ3RbU83scMgAM9jnuMXn19wav6pbMc0+3/G7I9B6rPiVezb2Wcnzv58FsoXSrFDybWFHgsRuT4S5349p/d4xv9/qJNm6WDEqHKWeDsRl9dfxsv7hpeBzUzLRNyROAhq/bPBmsxsgjrD6hRUiPSG5vOao+eBnnqP2ZaxxZDrQ1B/fH2DzxfUAuSmcs4mIMpnFiUsYO/+f+ydd1hU19bG35mh9yZVerWAoBSx9y5q1Ni7Ro0l0RjLTTGkqbHFWGLvsSv2joqgKCooRar03mHow8x8f8w3I3j2QQcHBvT8nuc+18w6ZW/OzD5777XWu7Sha/PpZjg3J3fuANeuAX/8Ie+WfN48+fsJjg8+jgPdDhDtPgd80Pf3vkQpTwFfgEd/PcK9n+4Rzy3LLkPctThwM+nLVvRY3QM/83+GiZtJ4zrQQvFe5o2ZD2dCWUtZ3k1haKHI1dn377//wsXFBVpaWtDS0oK3tzdu3LghsVdVVWHRokXQ19eHhoYGxo4di5ycnHrXSE1NxfDhw6GmpgZDQ0N8//33qK2trXfMgwcP0LlzZygrK8POzg6HDx+mtGXnzp2wsrKCiooKvLy8EBLCSD4yMDAwMDAwNC/Pdok2eTVMNIj2lIcpyI/JR7sv2sH7O2/iMUO2DUHPH+g36+gI3R+KyFORqC6tlvrc96GgooCKvApUc2V/bYaWx8XpF3Fs4DF5N4MBQFZoFgpiC5AZQq3DZd7NHObdzcGv5n9ymbcV+RUw8zKDjpWOvJvyQeTH5KMktQQA4DrLlXhM1xU98HzEWqR0ajnypE+2PYHfVD8E/BpAe4z/Gn8c6XMEESfJGYtWva2Q6J+IpPtJUt076mwULs68iLKcMqnO+1wR8AUofFMIBVVqBgUA8Gv4qOZWQ8Cnj0LvuaYnfqz6EUYu9FLiDAwM0rOqcBW+Tf4WKjqto85sSyf/01JtbLVoW2gDAG02uNtsN/T6oRfRVlVUhbur7iLw90CiPTUoFSdHnERKYArt/bmZXIQdDEN+zKf1hbiz8g5OjjgJjhJH3k1haKHI1dnXtm1brF+/Hi9evMDz58/Rr18/jBo1ClFRUQCAZcuW4cqVKzh79iwCAgKQmZmJL774QnI+n8/H8OHDUVNTg8ePH+PIkSM4fPgwfv75Z8kxSUlJGD58OPr27YuXL1/i22+/xdy5c3Hr1i3JMadPn8by5cuxdu1ahIaGolOnThg8eDByc3Ob74/B0GRcW3QNvixfhB0Mk3dTGBgYGBgYGsS0iym6LOgC7+VkR97JkSdRWVgJ5ynO0DbXJh7Dr+Hj3Jfn8Ob2G6nubTfUDgCapF6Isasx1A3VZVJsnaHlY+JuAk1TTdRW177/YIYmRZxN1W5sO4pt9qPZGH9mPNqPaw/Dji1H9lcWTL09FcP/HQ62QusQsuFmiSLTPRZ5wGGEw3uPZ7GaukUfhpmnGQDQZqPkx+aDrcBGuy/awdiVLPXJzeLi+a7nyAnPIdrpOPflObw68grlueXSNfozpbayFqd8TuHxX4+J9pAdIVivtR6XZl2ivUbyg2T8Y/sPos5GNVUzGRgYGD6aFqCazADAZaoLtMy1MPXWVKL9zZ03uLPqDvE9rqCiAIeRDrQBrKYepvA56AMzDzPa+8ddi8PlOZcRd42shtNaMelsAut+1kz9XAZayGFdzcTIkSPr/fcff/yBf//9F0+ePEHbtm1x4MABnDhxAv369QMAHDp0CO3atcOTJ0/QtWtX3L59G69fv8bdu3dhZGQEV1dX/Pbbb1i1ahV++eUXKCkpYffu3bC2tsbmzZsBAO3atUNQUBC2bt2KwYMHAwC2bNmCefPmYdasWQCA3bt349q1azh48CBWr15NbHt1dTWqq99Gp5eWiqRL+Hw++HzRRhaLxQKbzYZAIKj3IxR/Lj7ufZ+z2WywWCzi5wAoGrB0n3M4HAiFQuLn77aR7nNZ9onNZqN9+/YQCoUSe1P0KeeVaOHIzeZKrtFUffoUnxPTJ6ZPTJ9k2yehUFhv7PsU+tQSn1NNWQ1yXuVAx1oHOuY6xD4BbAiFQtTWCsHnC1tEn6z6W8Gqv5Xkmu8+p3bj26EgtgBmXmaoLq+uV+OAxWIBQkBZRxltu7WFkpaS5Dv2IX1qP749jDoZQdNMk7btjX1O/db1Q791/f7/707/Lv4UvntMn0QOpsxnmagsqoSmsSb4fH69sU98zdbUp7ptbE3PyWmsE/5X9T+w2CzJmCJu+9G+R1FVXIUBGwbA3Nu81fSpLnSfG7sZo7KgEtUV1VBQfjtOttQ+te3WFkuTl+Ifq39QW1ULn/0+lD5lv8qF891TyLbtDj6/C8Rmefap46SO6DCxAwDROvjd5xG6PxTBm4KxKG4RDOwNiN+xW8tuwX6EPVxnu0q1JuzzWx9kPc+Crp3I0dhSvnvvjhHisU88B5TXd0+88xNxIgKjj42mtF38dyyILaDtU2FiIYoSi8Cr4jFzWKZPTJ9k2KcdDjtQnFiMuSFzYdzZmHJ8a+yT+PiOHTvW2/Nrjj4JhWzJZ/z/z1ZmvnvN3ydlXWUsTVpK26fjg44DAFymuUBFX6Ve2zmqHHzp9yUA8vwiMzQTz3Y9g76jPnRtdIl9Eu8F6zvqv3d+0ZqeU211LWIuxiD3dS4MnAw+iT59ar8nFov1wf4OcZ9kiVydfXXh8/k4e/YsysvL4e3tjRcvXoDH42HAgAGSY5ycnGBhYYHg4GB07doVwcHBcHZ2hpHRWxmJwYMHY+HChYiKioKbmxuCg4PrXUN8zLfffgsAqKmpwYsXL7BmzRqJnc1mY8CAAQgOpta2ELNu3Tr4+vpSPo+KioKGhkh6S09PDxYWFkhPT0dhYaHkGGNjYxgbGyM5ORlc7lt9YXNzc+jr6yM+Ph5VVW/rS9jY2EBLSwuvX7+u9yVydHSEkpISIiLqS6I4OzujpqYGsbGxks84HA6cnZ3B5XKRmPhWpkdFRQVOTk4oKipCWlqa5HNNTU3Y2toiNzcX2dnZks9l2SdNTU1ERUVJNv6aqk+dtnSCc7UzdAx1AKBJ+/QpPiemT0yfmD7Jvk88Hg8cDueT6lNLe05FUUV4OPUhvH7wwpDfhxD7BOijvLwc+fkViIjIbDF9errsKXQ76qLr910pz8lysSWyv8rG1rZb4TjDEU7fOtXr0xGXI+BmcmE2yAwFKgUoiCj44D6VxJaAX8WHk6YTqqqqZNqnkBUhqMqtwuiroz/57x7Tpypo9NKAh40HeCweAEj6xOfzweFwWmWfWutzKswuROrFVGg7acNttFu9PqUGpgIATgw7gTnBc1CkXtQq+vQhzyn7VDaebniKAX4DoG71tn5pS+5T7B7RvV+ffw2f/T6UPmWcyoZyVSkMkp4iOVkfERHcFt8nRVdFtF/aHqmhqVDVUUVaQRrl98TmsFGjUIOYNzFSPSftEdrQHqGN2ITYFvXdI40RfD4f7du3l/tz6riiI4qji5Gbm0vpk9BciNmvZyPrXhb8t/nDsN/bbF9xn+KeiLIjuJpcRERENNt3L+p6FPjVfBh6G362YznTp0+7T8WJxQAABTWFT6ZPgOg58fl8JCQkNGufBAILVFVVgcerQUREksz79Cl995qyT/FH4vH679eYensq1J3VKX1q690WGc8ykFGZgayILEqfinKLIOQLoaihSOlTemw6itOLUVYskvIm9cl5mjOqtatRolki6dun8JyqtKtgNsgMiWmJyOJlfRJ9+tR+T4qKioiIiJDs+X1In2QJSyhr96GUREREwNvbW7QxoKGBEydOYNiwYThx4gRmzZpVL3sOADw9PdG3b19s2LABX331FVJSUupJclZUVEBdXR3Xr1/H0KFD4eDggFmzZtVz5l2/fh3Dhw9HRUUFioqKYGZmhsePH8Pb+61k1sqVKxEQEICnT58S203K7DM3N0dhYSG0tLQAMB7w9/VJIBAgPDwcHTp0kPwAWnufPsXnxPSJ6RPTJ9n2qaamBlFRUZKx71PoU0t8TlmhWTjgeQA9/tcD/f/oT+zTqFGiqE87OyE2b24ZmX0xF2NwduxZmLibYO7TuZTnVJpRipPDTiIvKg+jjo6C82Tnen26sfgGnv/7HF0WdMHQHUOl6tPxgceRHZaNlYUrZdonoVCI/Z77kR2ajVWlq6CiqfJJf/eYPgF+U/3A4rDwxbEvJH3i8/mSsU9RUbHV9aluG1vLc4o8G4nzX54HALjMcMGog6Pq9Wmj/kbwynlgsVkYun0oXOe4tvg+fchzEgqE+EPpDwDAssxlUDd86+xrqX2qLq3GRr2NAACLXhaYFTCLmtkXWYC1kxJQZOSIuSt1MGiQ/PsUvDUYt5ffxpDtQ+C+0J3yPMIOhuHaV9cAAH1/64vua7pT2lhbVYuChAKoG6pDzUDtg/tUza1GZUEl1I3UoaKh0iK+e+I21h0jxGOfs7Oz5L50fXq3jc3dp9zIXOx13QsA+LH2R0qf0kPSkROeA6cvnKCirdJs372D3gdRmlmKb5K/+SzHcqZPn36f0h6lQb2NOgycDD6ZPgGizOaIiIh6e37N0aeDB9m4eFH02aVLApn26VP77jVln55ue4o7391BW++2mBU0S6o+lReUY5PBJgCi99G7fcqPzsfJESfRdVlXdP2m62f1nEJ2hoCbyUXf3/p+Mn361J4Tn8+njH0N9amkpAQ6OjooKSmR+JQ+Brln9jk6OuLly5coKSnBuXPnMGPGDAQE0Bf4bikoKytDWVmZ8jmHw6n3EgPePlDSsc39OYvFIn5O10ZpP5e2jeL2vGuXZZ9SH6Ui7GAYun7TFUYuRk3ep0/xOTF9YvrE9Em2fSKNfa29Tx/7uaz7pKwhekezOewG28Jisf7/3o1vO93njemTuID5yD0jJcfVPf7+mvvIi8qDz0EfuE5zpVyj3RftYNLFBIXxhSiIKYBhh7eR+e9ru7KmMqqKRdk+lr0sZfqcrPtaIzs0G1UFVVDRVPmkv3tN0UZpP5d3n7jpXLA4ot9W3c/F7Xr38w+5vrz71BSfN3Wfasvf1kw0czeTHCe+5+Srk1GRX4HTY06jMKGwVfTpQ9oigACOPo4w72EOLRPygrml9SnrmSiivecPPdHv937EtrRpZ4gcW8P/vxY+6r0lqz5Vl4iCX9MC0+C12ItyT1VtVclnmqaaxHvmRuZiv+d+DNgwAN1X1ncGNtSnnfY7UZFfgen3psO6r3WL+O7RfS6ea9C1ke46suxTRUEF9rjtQafpnYjfsYRbCfhvyH8Ntke7rTbSgtJQklgCdXf1Bo+VZZ+yX2ZDSUOp3nmf01jO9OnT75NVL6tGXacl9wl4K7/47rq3oevIok8CAWjnmsx3r/n65L3MGxCgntRkXapKqlBVXAVNE01wlOpfS1j71glD2jNRVFWEeht1KKop0rYx+2U2DngfQO+1vdFjdQ+Z9OljPpfV84g5H4PcyFwM+HPAe49vLX361H5Pded8HzIGiccrWSF3Z5+SkhLs7OwAAF26dMGzZ8+wbds2TJgwATU1NSguLoaOjo7k+JycHBgbizSsjY2NERISUu96OTk5Epv4/8Wf1T1GS0sLqqqqkj886RjxNRhaNzeW3EB2WDbUDdVh5GL0/hMYGBgYGFo9+g76WF26mrJwICFfjYP69FjdAz1W9QBo5ntpwWkAAJv+NkT7mbFnUF0q2nw18zKr5+x7H25z3FBTVgMlTSWKTcAXoCyrDIrqilDVVSWc3TCdpndCW++2UGujJvW5DK0Pq35WCNkegpK0Emiba8u7OZ8tFfkVAADnKc6wH25PsRfEFQAsYEXuCiiqKjZ385oMNoeNiZcmyrsZUqFnr4du33fDs53PoG6kDq8lXsTjFCtLIWRzAKgT7c2N5xJPGDgZwNiVvG7OCc+BtqU2Zj+aDQ1jDeIx8dfjYepuCvNu5lLdu6asBgCg1fbjI6A/B1gsFkrTShH4RyD6/NIHbIX6m1x5r/Mk/+62shvxGiWpJbj93W0M3DQQpu6mTdreuhi5GNF+fxgYWjOZzzNxqNch2A60hZ6DHjwXeULHSkfezWr1tKS13ecMi8VCtxXk9wkAnB1/Fol3EjE/bD5lHqFhpIG1wrW05wp4Angs8oDtYFvaY8IOhqG2qpZ2Xd1a6TyvMyJPRqI4uZgZLxiIkN2YckQgEKC6uhpdunSBoqIi/P39JbbY2FikpqZK5Da9vb0RERGB3NxcyTF37tyBlpYW2rdvLzmm7jXEx4ivoaSkhC5dutQ7RiAQwN/fv56sJ0PTQOcZlyW91/aG7WBbuExxafJ7MTAwMHwIzTH2fe5kvcjCoR6HEHs59r3HtqQFIYvFQqJ/IrJeZBHtk69ORrsv2uFvy7/x6tgril1ZW5TR2H9df9gOol/8kHD0ccR0/+kwcTOh2CoLKrHVfCv8pvlJdU0xORE5qCyohJI61ZHI8OmhaaoJEzcTcBQ/LJpa3gj4AuzquAvBW+nrdbdGjDoZocuCLujj2wfaFlSna9C6IFyefRkHuh4AN4vb/A1sQsL/C8fRAUdR+Kbw/Qe3AHStddFpeidUFVchdG8o8ZjA3x+gk/9WOD062Myto0dNXw0dJ3aEgZMB0a6gqgBVXVUoaypLMu3f5fFfj6HWRg0WPSykuvcPlT/gZ8HP0LfXl7rdzU1LGPtU9VThPEUk/S3gCyj2ug7mLvO6EK8Re0U0p7LsadkELaRn3rN5mHRlUrPek4GhOSjNKEVtZS1iL8cieFMwSjNK5d0kmSOP8U9AHeIY5EDKwxQEbwlGVUkV0Z54R1R3rTHBoEn3knBp1iXkR+fTHqNnpwfDjoboOLGj1NdvyZSmlyLuahzKc8vl3RSGBpDn3E+uzr41a9bg4cOHSE5ORkREBNasWYMHDx5gypQp0NbWxpw5c7B8+XLcv38fL168wKxZs+Dt7Y2uXbsCAAYNGoT27dtj2rRpePXqFW7duoUff/wRixYtkkhsLliwAImJiVi5ciViYmKwa9cunDlzBsuWLZO0Y/ny5di3bx+OHDmC6OhoLFy4EOXl5Zg1a5Zc/i6fCxwOR1K7oClxGuWEqTenwrDjh2c3MDAwMDQVzTX2fe7UlNcgJzwHxUnF8m6KVBSnFOP4oOO4tfwW0d6mfRvJxD7zWSbFLt5MTb6fLLVj7eLMizjU6xDRxuKIQiLFi7J3SfRPxJlxZ5D9Mptof7rtKQL/CJSqPQytF3VDdfRY06NeJkZLHvtquDXIi8pD+NFweTdFpuja6ELDWAPb7bbj5jc3KfbCBJEjrDS9VJIF+CnA5/HhN9UPSf5J4FXw5N2cD+bUqFMAIIpCJ6BlJspgKzTrCBmr/TSa9Kfp8P+fP61T1Wm0E9iKbKzXXo+wQ2HEYyZfn4z+f/Zv1P1lLXvUFDTn2Hf6i9PY2W4nrX3EnhFYVbyKqHrAVmCj0/ROAIDt9tuJ56u3UYdJZ5Nmz6Y8++VZ7O60u1nvySAfasprELQhCG/uvJF3U5oFBZW3Ymte33rBtEvzZcw2B/Ka+7WkQM7Pmdgrsbj93W1cnn2ZaJ96eyq+vPClZH5Tl9qqWhwfchz7vfYTz7XsZQnLXpYoz6N3eHkt9cLCiIXQsdRpVPvlQUlqCaIvRDd4jOMoR3z14iuYdKYG6DK0DOS97pWrjGdubi6mT5+OrKwsaGtrw8XFBbdu3cLAgQMBAFu3bgWbzcbYsWNRXV2NwYMHY9euXZLzORwOrl69ioULF8Lb2xvq6uqYMWMGfv31V8kx1tbWuHbtGpYtW4Zt27ahbdu22L9/PwYPHiw5ZsKECcjLy8PPP/+M7OxsuLq64ubNmzAyYiQfmxKhUAgulwtNTc0mX6iJi262hgUhAwPDp01zjn2fM7rWugDQ6jawXx0VZes5jHQg2uNvxCM1KBWDNg+C52JPin1+2HykP0kHW4GNmvIaqRx+GSEZyI/OR0F8ASVTQlVXFb1+7kUrh50fk4/o89HwWkqWnjNwMkDms0zkx+TTZoAwfDrcWHwD2pbasBnwVm62JY99SppKWBi5EMpa1HrcrZnM55kI+EVUC52UyeMyzQWK6op4sfsFos5EwdxbOhnFloqAJ+prp+mdYOTcOtZzaY/TUJRYBAAYun0o8ZhOszrj16udm7NZ7yXyZCSebnuK7LBsTLkxhWLPeZUjCUzJCc+h2AFA21wbl2ZfgussV7jOcP3gez/a+AhBfwZhys0paOvVtlHtbw6ac+yL8YsBAIQdCoPbLLd6tpqyGoTsDIFpF9N6Y7MYbiaX9hmJ8frGC+4L3cFiN98YLhQKEX89njYzlOHTIiMkA/6r/WHY0RALIxbKuzlNjk1/G6wqXgUFFQUoKMu9ypLMkdfcj8nsaxl4LvZE8KZgYokIALAdSK9CU5FfgTe36J3+qvqqSHmYAqNORug4gZy5V1lUicA/AmHR0wJOo5yka7ycONj9IErTS7E8Yzk0TTWJx1ycfhHcLC6WpS4j2hnkj7zXvXJ9mxw4cKBBu4qKCnbu3ImdO+mj0ywtLXH9+vUGr9OnTx+EhZEjCcUsXrwYixcvbvAYBtkiEAiQmJjY5N7u02NOI+ZiDHr91At9f+3bZPdhYGBg+BCaa+z73FHRVUH/df1h5mVGe4xmXiJsX5wBd/J8ALrN17gGsBtiB1VdVVq5kZMjTwIA9B31iZH5+zz2oZpbjfKccnxx4gs4T3L+4Ht7L/fGlXlXUJRYRHH2sdgs9PWlf4d6LvKE+wJ32smsdT9rlKaXgqPMfOc/BzpM7IDMkExU5FdAzUAkzdOSxz42hy1VfcvWAjfjrTSn60xXin3kvpGoLq2GhpEG2nZtuc4SaVFQUcCytGVQVGs9dQhrq2qhqqcKm4E20DQjb+60RFxnuSLyVCSxJiQACGoF0HfQR7ux7eC5hBqgAgC11bXIi8qTWo7q7sq7AFp+UI+sx77MF5kwcjGiyCQDgHl3c6Q9SiM67apLq+G/2h8WPSxg1deK4jwL/y9ckp1v0I4clJMfk49d7Xc167qaX80Hr5wHl6lMSY7PAXE2tuNoRzm3pHmoKq7Co78ewaSLCcy9zaFuqF4v26+1I6+5H+PsaxnoWOo0WHcv2i8aYQfCMGjzIBg41n/vqOiqYPi/w2nfR2oGapj7dG6D9VwTbiYgeLNIRrS1OPtGHxmNnPAcqOqr0h5jP8IeBTEFqCmrgZIGUyKjJSLvde+n8xZhYKBBPPi1poUzAwMDA8PHUVVUBVU9VWibU+tUiVEpz4dCbTXYpSVoKc4+Mw8zmHnQOyh7/dQL6Y/TUZhQSCzKzc3iQlFVEe4L3SmLpvfRflx7WPSwgJY5VUqlsqgSl2Zegv1we3T5ilrLpyS1BDvb74THIg8M3DCQYu84sSM6Tuz4SW1gMNCjoKKA1KBUVBZWSpx9LRl+DR+/K/+ODhM6YNypcfJujswQO0+WvlkKXRvqGLe3817kvc7DoM2DYNFTunppLRkWmwVeBQ8ZIRmw6msFVV36DZOWgnU/a8x7Ng//2P4DbUtt4jj65mY83K+eQJ65GwCf5m8kAeNOxliRvYLWHn8tHgVxBej3ez/abLA9rntgN9QO3b/vLtW9x58bD145Dw7DyZnwnyKxl2NxatQpdF/dHQPWDaDYJ12ehNzIXOjZ6VFsagZqMHY1RmpQKmq4NVDRUalnt+pjJfn3qIOjiPfPChXVE9a1bb45E1uBjUlXJjW4ocvw6WA32A6rS1Z/NsFhJWkleLzxseS/Z9yfUe+3yMDQmhHUCiDgC8BR4hADQs98cQYA0Pvn3hSbkroS3Be40147dH8o7qy4g6m3pxLrUr/Y+wJX518FIApKbS1Y97OGdT/rBo9R0VFBxIkIdJnfBZa9mreGLkPrgNFCYPjkGXNsDNYK18J9Pv2LgoGBgYHh0yIrLAtX519FRkgG7TFluuZIazcQfO2W4egTs7fLXlyYeoFo67O2D3gVPNz69hYeb35MsWuZaaEivwKlaaVS6/gn3U9CRkgGUfqTX81H7OVYyaLpXbhZXAj5QmLNBQA40O0AdnXYRbR9qvB5fHCzuKjmVsu7Kc1O12+7YlHMIoozWp7wKngoyy4j2rhZogy4qNNRzdmkJmfA+gFYXboaQeuDEH6cWo8w73UeAOD2d7dbVW279yHgC/B402OcGXtGUpewOYi7GoeAXwNQW02uufc+gtYHAQAe/0Ud2wEg5UESAEArP6lxDWwCaspqUJpRStvn3r/0xugjoxGyM4RWItJptBPaekufWdp+bHtJjbnPBU1TTbQb2w52Q+yI9tfnXiP2cixRkpijxEG/P/qh35/9iI4UMw8ztPuiHQDgwdoHxOtnPBXNqWwH0UuvyRqxLHnyg+RmuyeD/GBxWKLAgM+k5lp+TH69/27uepgMDE3J7e9v4w+VP/Dy0Eui3W2uG9qNbQczT3Kga0FcAdIep0nKMtVF01QTph6mUFQlqzi8PvcaADDz4cxWVUIiKzQLL/a9QFVxFe0xbb3aoo9vH2hb0gc1M3zeMM4+BrmioqLy/oMYGBgYPjGYsa/pEU/qM19k0h5Tq6SGKg19CFuQpOCzXc+QFZoFbiaXaC9KLJLYnEZT5Ui8vhHVzLMe0HBEIImHvz7ExZkXUVtF3bTVMNaArq0u2o9rTzw3/Hg4aqtq0eHLDkS7qq4qihKLUJpRKnW7WispASnYYroFESci5N2UZueA9wHcWXGHIjUrz7HvpM9JbDbZTLQpaSjB/Wt3jD83vplb1bTE34jHeq31CN0Xiujz0RQ7R5kDTVNNcJQ4eHXklRxa2DTUcGsQui8UiuqKxIzGpuLsl2fxYO0DVJdK7+AvSStB6L5QAKAdR72Wd0dEn8WI7NtySk8EbwnG1rZbEfhHINGe9zoPF2dcxM2lNxGyM4R4zMj9I6Fro4uciIbrxb0LN5OLuGtxEmd9S0ZWY5+puym+PPclrHpbEe1X519F8OZgieO4LkKBEJa9LOG93Jt2c7TDBNF3781tcp0k7+XemBM8B2r6zZux/erwKzz45UGz3pNBPhQnF2Od5jraoLe7a+7Cl+WLQ70ONXPLmgZjV2P0/b0vZj+ejTVla4hZua0decz9Wlhp6M8W0y6mAERZdiR89vngy3NfEm2lGaXY4bgDB7sfJNpt+tsgJSAFIdvJcwtBrUjLVdNUEzVlNdI2XW7s7bIXV7+6SrsXAAChB0JRnFIMHUud5msYg9TIc93LOPsY5AaHw4GTk1OT69dGno7EOq11iLsW16T3YfgwBLUCyYuXgeFzpLnGvs8d9Tbq0GqrhSdbnsCX5Us8Ris/EfbPTkEhh94h2NwIBUIYOBlgxJ4RRPul2ZdQlFiEoduHwqa/TT0bn8cHr5KHfn/0Q9LdJCTeTZTq3ha9LAAh8OYOeZNvacJSjD9LdoZ0nNARw/8dTswmAADbIaIsgLo1xD51dKx04LHYA23at5F3U5odPVs9KGkqgc/jSz6T99inrKVMKyOopq+G4TuHo/1YsjO7tVI3KthzKVXCaHn6cswMmAllLeUWX/dMGjjKHPRY0wNjjo5pVqeEWIKZlB0NiObAyQHJxHlw1guRPOKY42Mw7jRZSlatjTqqNfQhZLec+YNYzpEua69utne7Me2Ix5Rll+Hcl+cQeTJSqntvMduCkyNOIu1RmlTnNTeyHPv4NXxkPMtAUWJRg8eR/ialGaVYp7kO/mv8iee8PPwS5yaca/C6ShpKeLHvBWIuxXx4oz+SkrQSJNxMoHVwMnxaiLNHKwsriXaxM8zU3bTZ2tSU6Dvow2WKC/Ts9GjfHa0Zec39GGdfy8Blqgum3pqKYTuHEe3FKcWIvxFP/L2LZT9J5SUAgK3IRqcZnWDe3Zxot+hpAZMuJthut71VBYuYdxP1hyRNKiY3PBc5r6QLkGJoXuS97mUKpzDIDYFAgKKiIujq6oLNbjq/89O/n6KGW4PEu4mfVU2HlsoWsy3gVfCwhrtG3k1hYJALzTX2fe6oGahhWdoyWkcfAChXFAMAhKyW8xw8F3vCczF9XYHyHFENLpdpLhQbv5qPOyvuQNNUExUFFWg3jryxSofLFBcIBUKi9GJtVS2SA5KhY6VDrAVo7GaMmIsxiL8RT3SYuM1yg8MIhxYl69jUVJVUobq4GmxOy/l+NRdtu7VFwC8B6LGmB4ycjQDIf+zz2e8DwW76YCP/H/xh4GSATtM+HVnA5PvJAIBxp8cRZX1DdoSgmluNb1O//aTqaSqqKqL/n/2b/b5fXvgSvHIeFFTJf8vw/8JxaeYljD46mvI9a9u1LQZuGgi/qX7gZnKJ9euqiquglx6HGlVtAC2jRkuHLzvAyMUI6m3UifagdUFgK7LxU81PtNcI/D0QtoNt0XFSR6nubexqjOyX2S1+01+WY1/k6UhcnH4RtoNsMfXWVIp99JHRuLv6Lvr49qHYxI6EJ1ufoOf/elLqqfIq30r5TvCbQLx/ZVElXh58CVVdVXQYT85AlTUcJQ5sB9nCcZRjs9yPQb6YdDFB/3X94TCCvG/TeU5ndJ7TuZlb1XRUl1Zjm/U2AIBRJyOMPjwaxq7Gcm6V7JD33I9B/jQk+3zK5xRywnMwM2Ampfacpqkm1grX0p6b8TQDam3UYDPQhmjv8lUXSeBLmw6tJ/By9qPZ7z2m54898Wj9I6Q/TUdbL+ll0BmaHnmPfcxoyyA3hEIh0tLI+suyZPju4Ri5byR6/dCrSe/D8GGo6qtC01RT3s1gYJAbzTX2fe5EnIzAb4q/YdyZcfix5kfiMTUqmqhW0wFfrWWNSS/2vkDESbL041ehX6HjxI7YoLMBQRvqy3QpqCpA30Ef3EwuBm4cCNcZrlLd18zTDMO2D5M4Z+pSnleO/4b8h7PjzhLPra2qRfDmYCTeIWcTPtn2BK+OvqKVDmut/Ov8L/ym+RFtJaklCD8ejqKkhjMwPkXMPM3gucQTqnqqks/kPfYl3EpA5Cly5lBxSjGC/gzCxekXm7dRTUzHSR0xZNsQ6FjpoLqEKi0Z+Ecgnmx5gj/V/gS/mk+4QsuEX8PHuQnn8JfBX5TvU01ZDe6uuYuA3wKw2WQzMp7R122VNYpqilBUV4RQQP6Oq2irQNNUk5htqGGsIZFjCtsfRjw/6PcHsHnpB5vQhrOvmhOOIgdGzkbQMNYg2i37WMJlCjU4RYxQKMSro6+grKlMfPc0xPyw+VgrXNvig0hkOfaVpJQAAMx7kDMZHH0c8XXU15RNUwBQ1VNFr596QdNMk5hd6rHQA85TnAGAWN8oZEcIdjjsAAB4f+fd6D5Ii4aRBiZcnCCRGGX4tNG310eP1T1g2NFQ3k2RKWmP0xDwWwDl87oqHDmvclBR8Olk2QPym/sxmX0tg4iTEbg89zKqSsj15/JjRTUr9eyll6/NeJaB4E3BKMsi1+O+MPkCkvyT0GFCB3ScKF0wUUunurQaBfEF4JV/OvW2PzXkve5lnH0MnzzGnYzReW5nSvQig3yYcW8GZjyYIe9mMDAwfOKot1GHoFaA0vRScBTJ8gn5lu6I6PcNavWl22BsSjJCMnB1/lU8/PUh0c5WYEvq3r07wWdz2JJgipQHKVLfe5/nPvxt9TfRpqorctrkRuYS7U//eQqOMod2AzDqVBSe73qO2mpqPcDWjJqBGpS1ydKlbb3awm6IXatyosiKzOeZMOlsUk/CT94E/h6Im9/chIBP3eQWywd5LqHPqm2N6FjqIPtlNvZ77SdKGIk3/DnKHGS/zG7m1jWevwz+QtSZKFQWVELAq/88oy9E49H6R3jw8wOUZZdBQbn5MhavfnUVGw02ojSdXJvUabQTlmcsh/0we6I9eHMwAHr5OvOelqhR1kBGuwGyabAMiPaLxsEeB5H5nCyHbd3XGi8Pv4Qvyxd3V98lHvNd9ncYuW9kUzZTpvBr+Lg87zJC94c2+717rO6BNWVr0PN/PYn27fbb8ZfeX4j2o9boBIC+v/bF8vTltM7ZdmPbwWGEA7EOongMcZnqQivZ3VScn3Qef+n/1az3ZJAPtVW1uDjjIvH35cvylfwvZAe5TldL5Wj/o3jw8wOU55XX+1zAE0BVTxU2A20w+dpkikQ/Q+NgnH0tg7THaQg7EIYjfY4Q7QvDF+Lr119D04QaeFtVXIXfVX6HL8u3XlkAMW6z3WDZyxJpwWQpb7GTZdypcZJ1bGvg+JDj8GX5IiWQfi2v3kYd406Ng3U/62ZsGUNrgnH2MXzy8Cp4yI3K/aRqkbRmyrLLJFGpDAwMDE2FmZeodtLt5bdpNwS0c2LhcncrlFJaTk3X2CuxAIC+v/cl2iNPRiI1MBX9/uyHvr/WP0YoFGLKjSn4JvkbuEx3QUmqdGMtR5GDkpQSpD9Jp9iUNJQw6eokTLs7jfZcfXt92vpYXeZ3QXVpNdKDqdeuLKzEH6p/4NrX16Rqb0vAZpANrXwMv4aPN3feoCCuoJlbJX8eb3wsdf2tpka8uV1VRI0uNnEzwc+CnzFk25DmblaTws3i4uWhlwDIUdN9fu2DoTuGgl/NR9ghcjaZPIj2i8Y+z320WXk13BoAonEF72zoWfa2lAQdeH/nDSOX5gvmEGdk02Uw58fm496P95AVmkWxRV+IRvqTdBi7GWP2Y7KEk+Oodggf+B0K2rYcqdms0CykPUrD5TmXiXZxcAogeo+8C4vFgqBWgO3223Hvp3tS3fvC1AvwZfki9nKsdI3+SHiVPITtD8O1hc3/zmIrsKGkrkQrDy3e+Ct6Q80oL88rh990P0SeJo/N2S+zUVVUhbKcMpzyOUWxD981HEO2DYHXt16oLCA7pJuCysJKJPknMcownwkJtxLw6ugrBPxKzYKrS0ubY7yPBa8WYPaj2VDRUan3ufNkZ6wsWIlpt6fRBoIwMLRWBv41EHZD7WA31I5o13fQR5t2ZInNioIKScAkSTFBWUsZKQ9TJHU+30XDSDTvPz7kOJ7+87QxzZcLb269AYAGHZRXF1zFrWW3mqtJDK2QT6c4A0OrRFOz6SftJ31OIsk/CZ5LPTF029Amvx9Dw5wYfgLcTG6D+tsNUZZdhrircTDvZo427VuP9jYDQ12aY+z73FFUVcQEvwk4PeY0biy5QayDp8rNh1JVKcp5LSfbzG2WG6z7Wkucle8intiTahRV5Fdgk+EmGDgZID8mH8N2DYPHQo8PvrfnEk+kP0kHN5Ma0Q+gwbq3Pf/XE85TnIlZU4DI+eqxyIO8WccCWBwWOEryKWDdWIQCIe797x46fNkBTqOcKHYlTSXMDprdKGma1k6vH3vh9dnXKEwohJ7d2/7Lc+zz+tYLFj0taGvTlWWVga3Ipq091hqJ+E/kfLIZaEOUUuwwvgNqymsw7sw46NroNnfzaEm8k4jMZ5kojC+EmQd1LHSZJgpmGLF7BMWmY6mDQZsGYdCmQc3R1HqMOToGOeE5UGtDDnqIvxaPwD8CoaylTKmhqGagBiMXI6jqqaI8txz69vrN0eSPpuf/eqIirwIWPSyI9oLYAlj0sED3Vd1h1ceKYhcKhShMKISKjgptthkd4u93s787/n/PscOXHy4rKauxLz82HzuddqLHmh7EupTjTo/DuNPjiOfWlNUg/Fg4Eu8mwn6YPZQ162fnxV6JxYOfH6D7qu5o9wW15m/7ce1RllOGLaZb4LnEE0P/aZ51dWVRJWrKauCx6MPnMwytFxaLBVU9VWLwzbxn81BTXgMWi9Xq5lb6DvrQd6CO6/waPk6NPgV9B33YDbGDSReTT2oeAshn7sdk9rUMFFUVMeX6FFr7jaU3ELI9BL3X9kafX/rUs2mZaWFO8BxommoSVRpqymrwddTX0DInq4gM3T4U2pbaCN4cTMwcbKnMezYPvEpeg1LGXku9kPksE6XppdBq23JUVBjqI891L+PsY5AbHA4Htrb0xVplhUUPCyT5J8G6L5Pi3BKg20T+UJIfJOPKvCvoNL0TRh8ZLZtGMTA0I8019n3uZIVmIeleEvTs9eAwkuykYvNFNax4ui2nLoiujS40zegnhgM3DUTsxVgE/BIAZU1ltO36tig3R4kDNQM15Mfmo8eaHsSN1YboOLEjbU2D0vRSHO5zGJ3ndkaP1T0o9pqyGvxj8w9cZ7li1MFRFLuqripUdFWgqEbNeKmtrAWvnIeashpUc6spG5AtFhbQZUEXWpnwpHtJOPflOYw7PU6qTeFPApboN1hZ9Db7Q95jn3U/a1j3s4aSOjW7qLq0GlvMtsCwoyEWRiyUQ+saR/arbOxx3YM2Hdrg68ivKXaOEgdt2rfBtNvkjFy/aX7IfJ4J11mu8Nnv09TN/WB0rHUAALq2ZAfkmKNjEHkqEs/3PIf7fPd6trTgNDzb8Qzm3c1RUVCBTtM6NVtNN+fJznCe7ExrF8tzkjZwLHtZwueAD/Z57IOhsyEsulOdZ2H7X8D96lWU6ZoDS8jZf82Noqoi0ekqJvFuIrJCs+AwgvweFvAEONLnCJwnO8NzkXQyunOfzgWLw4JpF2rwS1OirKWMb1O+haL6h9WgleXYF7pPJG2Y8ypH6nN1LHXgudQTIf+EoDynnPKudZ7kDFN3U1h0tyDKdO732i+Ra7Ud1HxjubaFNpbEL6FkRDHIBz6Pj+jz0Wg/vj1thunH4OjjiJUFK4k2UqBbayH6QjRywnPQ66de9f5uBfEFSLiRgIQbCXi67SkmX58M+6GfToafvOd+DPKlJLUEFfkVMHIxAluBOl6EbBep75D2CBVUFOqtc9/l+b/P4b/GH/OezSOODUHrg/B0myijb8BfLUf+/H18yDin1kYN4cfD4TTGiXH2tVDkPfYxMp4MckMgECA7OxsCATkLQFb0+aUP1grXwmk0Ner+U6fwTSGKkqgyLvJk0OZBGLSl8dHW4oWekiZ1s46BoTXQXGPf507e6zyEbA/B4K2DMXjzYOIxBWadEOc5GQKNljVJ3tV+Fw54HyDa3Ga5obKwEmmP0xBxIqKeTUVbRZQ5JwSKk4ppZVHoCNoQhAe+D8hGlkgWzH+NP9Gc9jgNHGUOrYNxn8c+BP4eiHMTzlFsytrKGLpjKMIOhGG91nqp2ixPWCwWkvyTEHeZLANr4GgAuyF2UFD9/GLrXKa64Pu87+tlZcl77Dva/yh2Ou0k2vKi8wCg1dWUrC4RBSwUxhcS7QM2DMDYU2Oxw2kHURJNvHEvlvpsKXSe2xmLYxfTSnBeW3QN5yedx7UF18CrqF+7NPFOIiJOROD6out48PODZp0Hx1yMwbmJ51CcXEy0hx0QSaW+m9UnRlyz7+nfZLmp/NeimqkCNqfFZC0UJRXhze03qC6tJtpHHRyF4f8Ox9kvzyLuGnWsZLFZ6PVTLziNkX6dZuZp1uyOPkDU5qriKlQVUyWBSchy7HMa44QBGwZg9NHRRPuNpTeww3EHihKp33sWmwX3+e6YcHECMYtSz04P9kPtaevx1a3L2Jx1gjiKHCTcSqCtucrQvPj/zx/nJ53H402Pm+T6QoEQ6U/TUZhAfa89/ecpHvzyAEn3koh1JVsqhW8KcWbsGQT4BiDvdV49W2pgquTf6kbqDWbztEbkPfdjkC+B6wKxt8tePPydXIu+j28fdF/dHcN2DqPYhAIh3tx+g+e7nxNr9pl5msGytyXt/CP5QTIA4Pu872nLTLRE0p+k49byW8iPyac9xqKHBcafG0+rBMQgf+Q99jHOPga5IRQKkZ2dLSmcyiB7ttttxz82/8i7GfXwXu4N72XejT7foocF5obMRc8fyIXpGRhaOszY1zyINw7FMl9EWCyoleaAU9pygiL8f/BHUWIRUeoHAMpzy1GYUAhFdUVRvap3GLR5EGwG2MB5Cn12CR0h/4Qg4JcAlGWXUWxaZlroNL0TOs0g14qKuRgDfjUf9sPJ0cjiunbm3c0pNkU1RRQnFQNoXVHbQqEQPgd8MOowNZMRAFT1VZFwMwFJ/knN3LKmhVfBQ3leeYPHbLPahoszLtb7TN5jX3lOOUrTS1GaXkqx6VjpYOj2oRh3iix/11Kx7GWJtcK1+LH6R6K9ILYAu112oyC2gLLBCADqhupwGOkAjjIHN5fdbOrmfjC8ch7SgtPAzSBv5j7f9Vzyb35N/Q2gutnJ1v2tiTKgTcWFKRcQdTpKksH3Lu3GtQNHmQMWh+qpy36ZjchTohpUXZd3JZ4/YPNQPB+xFnHeM2TX6I8k/Hg4jg8+jhtLbhDtuZG5uDjjIl6ffY3IE9QaW2wFNrqv6o7cyFyiM7AhCuIL8OivR8iNym1U2xuLgC/A7k674TfN74OOl+XYZ9HdAt1XdqfduHyx5wUK4goQ/l84xVZbVYtqbjVMOpsQ6yeK8f/BH4d7H6Z87jrLFXp2elhdsrpZg1j4PD6iTkch4kQEsW4TQ/PSeW5nDP93OJwnST/P/BAK4gpwoOsBXJp9iWK7+c1NBPgG4Gj/o8TgsZbKdrvtAAADJwNKRq3tYFtMuDgBK3JXYEX2Cmiba8ujiU2GvOZ+bGanu0UgLnMQd4X8fnca7YQnW57g8UZq8EBBfAGODz6OawuvSWr31cW6nzXSHqXR1uMz7CBynCc/SEb2q+zGdqHZOeB9AE+2PqENHAOAM1+cQfjRcGiZtayAZYa3yHvdywyBDJ88wVuD4cvyRciOEHk3pdnpvro7+q+n1nOQJ1cXXMV+r/2NPl9JQwlmHmatSnebgYGh+VHSUIKZlxkiT0bCl+VLPEalLB9tY/yhkJ/VzK2jp037Nmg3th0GbyVnI54ZewZl2WVwX+guWcSIqSioQMTJCJi4m+DG0hsIOxgm1b17r+0N4G0k5LuMPjIaow+PJtq8v/PGzICZtBkB7ce1BwBY9KRK07FYLDzZ+gROY5ww79k8qdosT6pLqnG412E82/mMaFfRUcHw3cM/OQnPA90OYJPhpgYXL46jHKGgqvDBmS/NgdtcN6jqqxKz9zSMNOC52JM246q1kv40XfJvUn2vFTkrMO7UOLTt2rZFbRikPU7DpZmXkOifSLSbdDEBW4GNn2p/grJ2/TFH30Efa8rWYMyxMRiwYUCDTg1Z4+jjCCUNJRi7GRPt7ce2B79a5Lh4F7HDasTeEej/R8uauzeEWG6uNIPqRAcA/L9fk63ARh/fPsRDeBU8BPgGIP5avFT33uGwA3dX3a2XcdYciMe1hqLumwoBX4Cwg2FIDkgm2pdnLMewXcOINXZL0kpwoOsBPP/3OeFM4OHvD/G7yu9IvpeMgvgCin3UwVH4KvQrHBt0DEHrgz6qH9JQlFiE1MBUdPu+G1EGjqF5MXA0gPsCd2hbNI1TKu1xGgCgTQeqOoV1f2uJHHzHSWTZ+ZbI6COjMXDjQCyKXkSRlda11gVHkYOSlBL5NO4TpaVkv3/u2A2xw7K0ZZh8fTLRrqylDNtBtsS60cpayjB2NYbbXDfaetv91/WnDUS1G2YHhxEOODv+rEQ5oTXgsdgDdkPsGizHoWmqCTXD1pOtyND8fH66QgyfHeKFY0OREZ8qA9a1PG3qF3tefNT5cdficHLESfgc8IHbbDcZtYqBgeFTo6asBsN2DMM+j320x+jkxAIAeHrkjVl54DLFBS5TXGjtHGUOrPtbY9BGqhxyeU45Xh58CdtBtlDVVQVHmSPVvW0G2GD4v8OJ2XVVxVU46XMSiqqKmHRlEjhK9a+tY6mDm9/chHU/a3Rb0Y1yvt1QO/T5tQ9UtKk1d7hZXCioKkDXVhdCoRCsVrJCZyuy4TzZmdZBlBqYiuf/Poeavhp8Dvg0W92wpka8IdXQczLvZo5by27B6xsvYu0xedDrh17o9UMvWvuFKRegbaXd4hwtV766gsKEQsy4R83mKs0oxfWvr6P9+PZwmUodN16feQ0Wm4Vl6cug3kadYr887zLSgtIw5vgYucgh0pEWLNrspQvs8jngg6qiKgj5QkrNKG4mF0VJRbAdbAs2hw0+jw+OonRjYWMZe3Jsg3attlowdDYkSk45jXLC6COjcXHGRfAqeOj6DTW7Lz86D9ahD8HVt4JQSM3slgcmXUzwM/9nsNjk8eDR+kdQ1lbG6uLVRHttVS3OjD0Dl2ku6PUj/e+ThMtUF3AzuUTH1seSFZaF0vRSOI50pNgUVBTQfVV3uchnPd74GP5r/GHd3xpWva0odjUDNXgs9CCeq2agBrU2aghaFwTnKc6UgCEdax3RPGDXcKIj54HvA+RG5CLjaQYse1nKpD8fgqquKrou6wqrvlatZn7wKVNZVIncyFzo2ek1SfCtzUAbTLw8EWae1N/X9LvTZX6/5qDT9E6iutSl1cSguBPDT0j+Pe3ONNgMsGnO5jEwNCkN1ZQL2RECfSd9Ys14TRNNzA+bT3tu5KlIJN9PRrsv2tHeN/ulKKOvoXrKLY1h26mSpu/SZX4XXJh6AbaDbNFh/KcVVMogG5jQKAa5wWKxoKen1+ST9nGnxmFJ/BIMWN94x1d+bD5iL8fKsFUi4q7GoTy3YSmsj6GioAIlaS0rSszYzRhtvekL7b4PsfxW5EmqFBADQ2uguca+z52oM1HY57EPU29NxVrhWuIx5TpmKDZ0AF+95WS0AMD9n+/j3o/3iLbpd6eDX8OHL8sXt1fclnxemFCIXR12AQDe3H4D+xH2DToN6bAZaAM9Oz3K5+V55UgNTMWb229QkV9BsQsFQiTfT5bUPnuXg90O4sHPD3D6i9NEu7aFNoI3BeNX9q8Q1LaOuh5K6kooiBPJyJHg1/CR8yoHSfeS8OTvJ83cuqajqrjqvXVzrftZY/ju4fUideU99t1dcxd3V98l2rJfZiPiRASC/my+bJUPhVfOQw23hmjLjchF7OVYWjnBgZsGYsaDGUi+n4z0J+n1bEKhEGH7w5Afk/9RigtNgf0we3gubSDTUggc6XsEvyv/TpHxDD8ejkM9DuFg94P4S/8vJN9PbvoG/z/cTC5SHqagqoSc0Xpx5kXkRuTi3g/U8V1JQ0ki/xn4RyDx/Oc7n0A/MxKmcQ9k1uaPhcVi0Tr6AMBtjht6/q8n8mPziWuelIcpSA1MhbK2sqjmrBSMOTYG0/2nQ81A9tHtp0adwimfU0SbkroSBqwfgHZjyBuM7yLLsU+cmdzte2pQDQBkhWYhIySDaFPVVcXgrYNhO8gWiqqKFLvLFBdMvjqZNmMr4JcARJ+PBiDKpmgu1A3V4bHIAyraKkzNvhbA+YnncbjXYYRsl71qUtSZKOxw2AEhXwgNI2pdyerSatrxtSWTFZqFdZrrsF57vcT5IObFvvqB0GzFT2uLVt5zPwb5ErQ+CFvMttCuD+OvxSPhekKjrl2cUoyk+0moKSfPkW9+cxOl6aXo49sH1v2br87sx1LNrUZhQiFqysj9AkT17o1djaGqq9qMLWOQBnmPfZ/Wm4ShVcFms2FhYQF2EwtqqxmoQc9O772yHw3JUe102olTo07JdIFRllOGkyNPIniL7FPKeRU8xF2Nw0aDjfjb4m+ZX/9jmB86H3Mez2n0+XaD7dBpRid4feMlw1YxMDQfzTX2fe4Yu4qy9VIfpdIek2/uisTOX4CvQJaelAcxF2Pw8LeHeLGXnAXN5/ElWSF69m+dcu8uCHjlPKnvfdLnZL3o4rrUjcp8N6sPAG4svYHaqloM3kyWHxVvyJOyizRNNDHlxhTJf7emzTz7EfYSidJ3sehhIcmuJNUqbK3YDRVJyzTklH3wywNUl1TXi/qX99j3aMMjPNrwiLhR2JJrqHp/542+v/cl2kw6m8B5sjO+OPEF0a5hpIGr86/iwpQLeLL1HYdznS7zhRxc2xQtqyZ/NLYDbTF021DaaPC6Gdvv1scz7CjKViqMLwQAaJo1n+z7nZV3cLj3YeS8yiHaxVKpPgd8KLaashq8PPwSAIiZf6H7Q5HxJB25lu544z5Rdo3+SEJ2hsCX5YvX514T7fqO+ri76i52Ou3EvZ+oTs7jg48DAHr/3JvY74YQ8AWoKa8Bn0et5fOxuC90h8s0+oCZE8NP4NKcSw1uxomR5djXZ20frBWuhd1gO6L9SL8j2O+1H8FbyetLlykumHprKlEyTUxacBoiTlLr41n1tQIgcvQ15OBtCgL/CMTB7gfBq5B+bsMgW4xcjQCI5jiyJvlBMmqranF6zGlicM567fXYoLMBvixf4njSUtnbZa/k30WJ9euEqxuqw6KHBfr49sGC8AXEjN3WjLzmfoxvsWVQWVgJbiYXp8eQgz1nBsyEdX9rYmJFaUYpfFm+8GX5orKIWgu5x6oeMPMwQ+i+UOK1NYxFAQPdVnSDgnLzixo2dm1xqOchbLffjsS7ZCl7AMgIyUDnuZ2ZLOAWjLzXvcxOI4PcEAgESE1NhUAgQElqCa59fQ3VXOkWeR9CSWoJHm9+3GBR1peHX+JX9q+SrLF3mXxtMgZtGUSRCvoYxEVmm2KBWp5XjpMjT8r8urLgzZ03eLaLXN/oQ9Cx0sHow6PhMEL2kj0MDM1B3bGPoekw8xDJ/zz89SEuzblEPEazIAWdb66HWix5kSAPxLVKpvuTpYrCj4Uj51UOOs/rDPf57pLPjV2N8UPlD/jqxVeYHzYfGiYa9ep1fQgGjgYoiC1AzMUYik1RVRFLEpZgcdxiYhaFqbspOkzoQBuRPHDTQABv6wK+y94ue2HS2QQrclcQnYktkYqCChTGF8KkC032EUtUP6zv730/KYkVDWMNxF2JA6+SftM17kocpY6WvMe+nv/rCQDEujgmbiZYK1xLmwUsTx6sfUCbYaRuqI4v/vsCzpPI8kR8Hh/50aK6YpSaJixg/NnxiO06A2xBLYL2U3/38iI5IBl/qv9JyXgQI3Y0D/lnCCXL1LqfNYbtGgZlLWUM2zWMIlXYlKQ8TAEA6NqSHSn9/ugHp9FOUNGhyhm/PvcaOa9y0H1Vd/xY9SPFfmXeFeRH5SK93QCU6za/fCQdYolUumx0If/tZldbL6qyR/91/dHn1z7YZLgJN7+9KdW9D3Q9gHUa6yROUlnSc01PjDk6hmirLKpE/PV4vDz4Eus01733Ws059vVY0wMAiEGuxcnF2Ou+F8/+Ja/D4m/E49Z3t/Dg5we4MPkCJfBm8JbBmHxtMjTNNCVzleagJLUEEf9FwNTdtNXMDz5lBm4YiLXCtbAfZi/za9cdz9+nqpQbkSvz+zc1q4pWwdGnvjSw0ygnzAqchd4/94aRs5GcWtZ0yGvuxzj7WgYD/xqI/uv6w3OxJ9HOq+AhZHuIZP5Ul7p1v+kSNyoKKlBdQt5D1rHWAVuBjc0mm3F98fVGtL5x8Cp4+EP1D1yaRd5/eB+5kaKxrW5Q77sE/h6I0P0tZ/+CgYq8171MzT4GuSEUClFYWAgzMzME/BaAsP1hMHY1RpevZFuD4vKcy0i8m4jO8zpj5N6RxGPE0kb5sfnEKGL7YfYyn9BqtdXCmrI1TVJoXM1ADaMOjYJhR0Ni7SV5cn7ieVQWVsJ9gXujokKzwrIQfiwcrjNdYeTy6U2IGT596o59DE3LlJtT8N+Q//Dy4EuMOjCKYlcvEo39fLWWI+PZ66de8P7OG2r6ZFmygF8DAICYcVCcXIy9XfbCYYQD4q7God+f/Yibq3R4LvFEQVwBcWwWCoRQ0lCifWd1ntcZmqaaKIgtIErvGbkYoe9vfWHoTN14ryqpAluBDSMXI2Lmn5iUwBQoqik2a22xwjeFyAnPIcq11ZTVIOK/CGgYaxCdLRX5FdA218b9H+/DzNMMtgNtm6PJTU72y2xY9rZsMEp29NHRCPglAOlP0tG2q+g7KO+xr+OkjjB0NqSVqEt+kAwFVQWpfjPNgWZbTdgMbFzk7sXpFwEAPX/oKcnKEcNisaCspQyFmjJE9l4IEyuqA0pevLn1BrwKHgQ88gK535/9UBBTAK8lVJUHBRUFeCz0oK1b1pSMPjwa3EyuJIPvXapLqxFzMQZJ95MoNWbadGgDmwE2iL8WD5uBNrDpX/+Z9/yxJ6L9YsESCMDi14LFahnL+C5fdYGgVkD7uwo7FAYzTzOMOzOO+HfxXu6NqLNRMPMyo3xH34c4oEDfXl/qdr+Pl0de4tLMS/A56AO3WfVrhAt4AihpKKGmrOaDpEdlOfalP03Hga4H4DTaCRP8JlDsPVb1QI9VPYjnCvgCZL3IQm5ELhxGOEDbvP4zSw1KxZMtTzDm2Bi4L6Su0xRUFaBkqIMTw0/AaYJLs9VjFQcDO/g4yCU7g6H56PdbP5h0NkHYgTAM2EAtwbKqeBVYbBaUNVuOIseH8L5Aol0dd0FRTREdJnSAo49jk4xp8kLecz8G+dNjNfmdBIjUziZemgjL3tQ6sAZOBlhduhocJQ5x7C+IK8CADQNoa8iO+HcENE018eDnB1BQbb53h4KKAqz7WTd6r3JF9goIBUKoG9KviUfuG4moM1HIfJ7Z4vZ7GUTIe+xjMvsYWgS9fuyFSVcnwWmMk8yv7TzVGVptteA22432mCF/D8GasjWw7kvWcs58kYnX517LtI5QZVElDnQ9gODNZJmVgrgCJN1PalTmX01ZDRLvJlJkIloCErklgp9PwBegqrgKtVW1tOe/uf0GT7Y+aRL5UwaGD0EoFCL8v3CU5zVdvU2Gjyf2SiyC1gWh6/KumHiJLHmmVMUFAFS3aTkLUGVNZVQVVSE/Jp9oH314NJxGO+Fwr8OI9nsru5f+JB072+0EIKoHO3DTQHSa3ol4DTqs+lhhYcRCStQxIAqG2Wy8GRsNNpJly4TAyZEn8WgDuX5d6L5Q3P/pPh5vfEyxVeRXoDynHC8Pv4TfdD9aGc+TI0/i7ipyzbWmYrvddpz54gxRXk7bQhs9f+hJm+GW9jgN8dfjAYjeXZ8K2WHZUG+j3mCGBZvDFkmYtqDIan4NH4YdDIlZVdxMLo70PYIDXQ/IoWUNk/44XSJJ+S6J/onwZfniN6XfiHbLPpZwneWKfr/3I26OBv4RCNvQ87CMuNaiapcqqCqArcAmbv4AoqwrLXMt3P/5PsX26ugr7HbdjZAdITg/6TyywrKaurkSrPtZw2UqvfSjOHKdFIVu5mEG7++8kRuZi7D9YRR79x/7wbSPA9xu/wXne//IrtEywONrD1rFjYq8CtRW1ULHUocYLBK6PxR+U/1g5GKETtOke2etLFiJlYUrYdXHqjHNbpBLM0UR+beW3aLY1A3VsYa7BmuFa7E8Y7nM790Q4cfCAQBaFtL/XvVs9TBw40Dwa/jgZnIp9u7fd8e3Kd+iw4QOaPdFO4qizbGBx3DYQ1Qb+JVq10a0vnEYdjDEj9U/ovdPZGUAWXDt62u4sfRGk13/U+LF3hfwZfni8SbqfO5jKU4uRuazTPT9tS9M3KiBYyraKq3O0Sfmzso7ONj9IGX9mPooFXlRech8lok7K+60yozFlgiT2Sc92dlAcbFsr5kWnIbw4+G0e5p+U/1w/6f7UNGmzs3ZHDaUNZVpgzzC/wvHyREnUZpGVmc70vcIHvz8AOpG6uj3W7/Gd0JKasproGmmKZERlRY1A7UGHX2AqM5z1OmoZp3jMrQuGGcfQ4tAx1IHDsMdGozobyyuM1yxLG2ZJLKchIKKApTUlWgzzfa578PZ8WdlWieAX8NHbmQuipLIDrkdjjtwtN9RlKRS5abeR3VpNSL+i8C5Cefwm+JvLar+0QS/CZh0dRKxUGninURs0N2A8P/CieeW55VLinVrmDTu5cnA8LFkPs+E31S/Znc6MEhHeW45UgJSYN7NnOi8AoAMx7541f9bCJTJWXTyoCynDHu77MV/Q/8j2q36WEkkp+tq+b9btyo/Jp82u4SOizMu0kqeKmu93Vwh1U0IPx4OjjIHrrNcieeLA1teHXlFsWmaamL6PZFsafixcOK7VigQorqkGsVJxe/rhkxZEL4AEy9PhKKaIsXGYrGQdC8JMX5k+UPLXpbQMhc9A7HU3afAd1nfYcTeEbR2oVAIFV0VjD0xtlmz5Pg8PlIeptDOeU6POU0rqVMQX9CUTfsohv87HCP3kZUpVHVVAbytUfouPVb3gMMIB/iyfOE3za+ejV/DlzifVEtzgCpqLUN50fun3viJ9xOtBOfBHgcR+EcgHv72EKUZ9Td5KgoqkPMqBzeW3EDkqUhwM6hOjaYi5mIMdnXcRZSjAkTOVQC0UdhiOcrIU5EU26IpxXhwQeT0LTYmv9PkQVpwGp78/QQVBRVE+7Cdw2AzyAa+LF88/ecpxX59kUhaq/O8zlLfW1VPVfIbkJbUoFScn3ReUk/2Xdp6i8auaXemEe2xl2MRtCGowTIRTYH7QndMujoJA/8aSLQfHXAUvixfpASSv4Nuc9ywJH4JjDtRxwwVHRVoW2jTvq/EG6o1ypqIyKeRr24igtYHYY/bHuL8QxY8//c5QraHNMm1PzXir4mCmMpzZR/0mPooFcGbg3FjyQ1ivapTo05ho+FGXF1wFVmhrWeTe5PRJjze+Bhpj9Mo9U3rSp67zXWjDXJhkA7G2ScdxcXAvHnANPIrr9GE7g2F3zQ/XJ5zmWjvsaYHnL5wQllOGcXGq+DB/3/+ODfxHDHQ1Gm0E2wH2eLG0ht4tJEabFpRUAEVHRUsS10GBZXmy+yrraxF6L5QpD5KbdT54f+FY7v9diTdT6I9xrCjIb568dUnVSaCQbYwzj4GucFisWBsbAwWi4XkB8m4uvCqXDZbYi7GYGObjfBl+dIOyEadRCnYsnxJ1FaKstdU9ciL1I4TOwKov8H6oagZqEkiSQS1AqBxtWGbBKfRTnAYTo7+FWdH0Mk2hewIwaVZlzDv2Tz0/bVvk7WRgaEhVPVU4TDCAbaDGyfJV3fsY2g6Os8RbRw+/PUhMSsLABR4lTCLuQfl9JaTdfXor0fgVfDgPJVcg4tXyUN+bD7aeretNw7aD7PHWuFaLElYgiHbhqDLV12kzgx/c/sNXh58idwoalSxlpkWFsctxrS704iBOckByeBX82HezZx4bXGtPucp1H4pqioi/Gg4lDSU4PWNFzFjTCgUwtjNGO3Ht5eqTx+LkbMRHEc6EjNSaqtq0e+PfphxbwbxXFU9VcnmqLhe3KfAi30vsLvTbpRlUxfmgGjecXzQcQT+GVjv86Ye+57tfIbDvQ8j4r8Iol1VVxWZzzOJG4TGnYzRY00PjNxPdqrJk6gzUbg44yLRiWnSWVRrcF7IPOK5vHIezow9AwBQN3rnd8sSBQ9kOPSGQm01NAOuybztDRF/PZ72O1ScXIw7K+/QOizSHonqhWmaaVLmuO7z3SWSkj1/7An74bKvK0WH3zQ/5EXlQSggT7xn3J+BxXGLMe8Z9Xm9uf0GUaejAIhqEQqF9a9h5rcDelmvsXvRI0R6SiefyOfxUc2tlqlCiZiEGwm4tewWTgw/QbQXJxcjeJMo2CMnPId4jJKGEh7/9RhPtj2R6t4xF2NwqNchJN2j3xCj4+qCq4g8FYnMF5lE+5QbU/B93veS+r914VXycGrUKfiv9sce1z3vvZcsxz7DDoZwGE4vZ1mRL3K6kuqdVRZVIsYvBtXcauKatrKwEkWJRbj29TVs0Nsgkc8UM2jzIBQZOSKi39KP7oc0iOVvs19l0/62PhYdax0Yu5GDJhjqM3T7UMx+PLtJ1uLpwSJ5/cznmbi94jbFHns5FhV5FXix5wVljtGSETtGe/3UC2ae9ccU1xmuWBK/BP+r+B989vnQyvi3Vph1b+sgponKNnsuFdXqo1OscRzliIBfAohKDQXxBQhaF4So01HEQFATNxO8uf0GSf5JxCBsi54WqCquwt01dxF9IZpibyrUDNTQ88ee9WoWS4PfVD8UJhRK3ucktttvx70f7xHVShhaBvIe+xhnH4PcYLPZMDY2BpvNxu0Vt/Fi9wu8OkqN+P9Y/H/why/Ll7bo++ONjyUDKV2x8QUvF2CtcK1Mi4Kr6qliwIYBsB9K3oQYe3Is1grXNirbka3Ahpa5FmwG2GBuyNwmqQvYWC7OuAhfli9R9kz8QlTRJb+0tC20oWGigdKM0ibZsGBg+BD0bPUw6cokdJzQsVHn1x37GJoW6/7WyAnPwXrt9US7Slk+DDLCoVhE3oCUB7YDbdHrp160NXd2d9qNGm4NrPtbUzYE8mPzcWHKBXAzudjvuR8Pf38o1b1HHRLVNRTXsX0XfXt92PS3IW4SDt02FN/nfw8lDSXiuTYDbNCmQxtaue7M55kwdjXGkL+HEDcx2Rw25ofOx4D11BouH0Pa4zQc7H5QIrdJadeLTNq/R2lGKY72OyrJxnkXJfW3fwu6rJeqkqpW9U4TCoXgZnAb3HBlsVnwXOqJmrIaFCcXSz5v6rFP30EfJp1NaGuHdV/dHZa9LInfXxUdFfT/s78kSKAloaiuCFV91UZtcr/Y90Lyb7HDXQxHkYMZ92cgy743ci3dUdW2cXUBG0NRYhFODD+BfZ77iPbkgGQ83viYNkPO0UfkgF+evpxSa1tRTRGLYhZhfth8eHzt0ayLbLe5brDqY0UrK2nY0RCappqoLKykOPPEGYo9f+yJDl92oLS7VlEUHFih9BRPHchZXXTEXIzBeq31tJtdLw+/xN4ue5EfS96Ma4jOc0W/mXfrv4kpSiyCqr4q2o9rjyHbhlDsa4VrsapoFV6fe42MJxlS3fv0mNNIDUyl3URsiLlP5mJF7grKxruYysJKSTDou+UF3s2kfx+yHvvurr6LsINUqVcAmPt0LmYFzYLnIk+KjZvBxeU5l/Fs1zNiyYSgDUH4x1YkEWviZkL5Dnov98Yb9wlwvb0R1mF+lPObiqLEImQ8zUDf3/o2mSNk4F8DKWMkAxltC22Ye5s3SaZMhy87SAJT3OZQS7AM2DAAbnPdYN3PGh6Lmr8ua2OZdGUSZgbMRN9f+1LqTqvoqCDhVgJen32N2qraFqXIJAvkte5lfIvSwW0iEQQTt4aD0hTVFNF9VXfYDbGj2DSMNdDt+24Ye2osVPXJCRIcZdH+LCmo0mWKC1xnu+LJlidNss9MB4vNQtqjNESdiWrU+UN3DMXgrYMbzNrrOLkjjDoZoaacUFqDoUUg7z0/psIxg9zg8/lITk6GlZUVev/cG4F/BMJ+mOyjb8V1TtQMyIuDIduGIPtVNioLKmkzzpoCBVWFRmXtiQneGozKwkqi/jSbw8aQv4dAy1yLdvEtL8QvWpJkqs0AG8wPmw9tS5o2C4GyrDKcHn0aow6NgutM1yZsKQMDmbKcMlGdo8G2jRoz6o59HM6nI+3X0ih8UwjHUY5I8k+CvgO50L1OjijyvczOtRlb1jB2Q+yICx4xFj0s0KZdG1j3tUZFQYVk46sivwKXZl1CxtMMtOnQBi7TXIgyXQ1h5mWGmQ9nQt+e+vcqzSjFg7UP4OjjCIeRDpRNQCUNJezutBvW/a0xci81O8qihwW+jvyaeN+CuALkRuai3RftUJRUBF1rXeJxJ0eeBJ/Hx9SbU6XqV0PUlNWgKLGIXIcQIhlvQCRd+W7tBTV9NbjNdaOVlX517O3CMu1xGnQsdSjHBK0LwqMNj7Akfgn07PQa2Qvp2e26GyZuJhIH74ci5AvxYs8LdJzYkbYWBZvDhnEnY4T8EwL3he7QsdIB0PRjn/0w+wbnkR0ndGwwSCP2SiwKYgvQbUU3ii3gtwA8+PkBvsv+TiIn3lxEnYqCppkmUVqvIK4AOxx3oNv33YiyfvFX46FupI4V2SuI1z416hTcr8XijdtYqDg0LoClMWiaamL4v8Nh5GJEtIszwGwHkTPoxxwbg+yX2ciLzkObdm3q2bJfZSMjJAMW3S1QmFAIjiKHdg0ga4ZspTqz6lJZVAm/aX6IvxaPn/k/16tp6TbLDap6qjg9+jQMHA0otf+ie36FktoFWLRzEWLbV0DY68PbJa7hnRWWhQ5fUjePaqtqUVlY2WDNbDq0LbSxVriW1h7yTwg4ihyMPzueaC/PK8eu9rvQfXV3DFgnXTDHwI0DwVHiNEoCVElDiTY4BQBufvM2SJRfw6/n2FDVVcXoI6NRnlsODRMNCIXCBp3Kshz7biy9gZDtIWjr3ZZYj15BWQEW3cmZn9qW2rAbaoew/WFwmeoCq95W9exiJ3X3ld2JTrWTPifR/mExOPwaVGmQ51VNgZa5FkbuGwmTLk0nHappqklxwDOQqSyqRPixcJh0MaH9rjUWi54WmHpzKgR8Adq0b0Oxd1/ZXab3ay4cRjggJzwHmS8yKc6+am41biwW1Yu8OOMixp4a2+iA0qZAKBCCV8FrcLxsCHmtexlnn3QI5ORjPjP2DJS1lInBnBpGGrSS1QDwaOMj8KtFKjZOo6nBpDnhOXh58CUANHgdWcPn8dHHtw+tgtv7IAXrvEun6Z1wuNdhaJpowmupV6Puw9C0yHvPj0krYJAr3P8PIXH0ccTcp3Nh7k2W//oYxp8Zj7XCtej1I3lVnHAzAYG/B8JlmgtxUgkAe7vshS/Ll6gl3Vj41XxcW3it3mZgXfa47YEvyxeFbwqJ9ns/3EPg72T5iuLkYhzsfhA3v7mJhJsJLSpCzHGUI1xnuxIzN2qravF442Mk3Ewgnmvdzxp2Q0Wb4Il3qDr+DAzNQfqTdIRsD8HTbdTaMx8Kt6nC5xgkxF2Nw82lNzHz4Uwsjl1MPKakjR2ybLtDwGncArapuDL/Cq0k2qiDo1CSVoKj/Y/i3g/3JJ9nhWVJ5I9eHnwJVT1VtPuinVT3zXiaAX4Nn+jEqSysRNiBMJwadUqycVyX0vRSlGWXQUmT/Lfc2W4nfFm+ODPuDMWmqK6I9uPaI/pCNP6x+Ycou8rn8RF3NQ7J95Ol6tP7MHQ2BL+Gj+yX5LpLYmcDKbNfRUcFhfGFePgbOYOybs1EOpm5knTRWMDNoZdqaQqUNJSgoNqImD+WKPOILkNTjP0we8x8OJNSs68px77cyFw83/2cUsNNzNEBR3Fl/hWiLdE/Ead8TuHO93eIdnGdTG5m84/dlr0sYeZFzj4SZ04+3viYaJ90ZRK+jvwaD355QJlv8nl8idSfZcRVNOc+t4KKAtwXuNPK/nac2BEj9owgBmoIhULEXo7F4d6Hsav9LkoNr/hr8bj61VXs6bwHh3sdppUCbQqKEovwfPdzFCaQ5+6bjTcj/lo8rXMq55XIyek3zY8iw8xT0YRJVlso1yjDPpYcEEGHeTdzGLsZ0/69dax00GVBl0YFHAhqBeDX8GkdJQM2DMCIvSMQfSEaWWFUCd1NhptQkV/RKCd6txXd4LXUq1E1UZPuJcGX5YugDUFEe9yVOMm/3828VlRTRKfpndBtRTe4THGhdfTFxgLZ//9qkdXYp++gD8OOhhhzbAzRHnkqEuHHw4nZ4sqayuixpge8vvUi1vS1H2qPgRsG0mbPxV2Jgxo3F0IAWfZSeJs/EjV9NRg6GyLrRRZtYM7HcmXeFVyZR34/fG4IBUL4/+BPO45dW3gNN7+5iSdbpJPd/RByI3Kxx20PEu8mQlGVWitZHMzR0nj6z1P4snwRdZacyRN3NQ67O+3GPvd9CDtUPys3dF9ovf8mBYbJkyP9jmCd5rqP2k9i1r0tn6aaA1756gp8Wb60Ciq1VbUShx2JivwKFCYUEt9pdeup7/OgKkU8+ktUx2/8ufHQs2++gMry3HIc7nUYL/a+eP/BNOfHXIpBcUox7TGapprwWOxBGzTH0DKQ59jHOPsYWgTleeXIicghajE3NYpqiihOLsbt727TymyJ67uQstEai/hedWW+6iLeeKwuIdea8jnggxF7RxBtagZqUDdSR4xfDP4b+l+DL9DmZuLFiRh1gJxJUFVShYgTEbST+OQHyeBmcuG51BPuC92bspkMDPT8/2TY0cdRvu1gaBC7waLAgMiTkbTHlBraodjYCeyaquZq1nsJ2RGC0L2hyI2k1s0DRJswVcWi9toMfCu7Z9jBENb9rKFrI9oE1neUPur+2tfXcGvZLaJcYJv2bTBs1zA4jHQgvrfu/3QfvAoebQ0XU3dRJHNVEfVvrWWmBY/Fb+WYSAs6fo3oPeY8mVzLsLFwFDkwcjGCpqkm0T711lSsFa6ljc50X+COPr/0Idoser6NeG8/llxrMKbYCHyOIq7eaL6Iv8rCStgPsyfWT3wfbA4bdkPsUJRURFtLglfBw6FehxB/Lb7RUa2N4fnu57i28BrthkKSfxJC94bWkxYVo26oDkNnQ/T5tQ/RaTFyz0isFa6FiVvTZZbQ0XFSR2gYaaC2mpp1ZdHTAhP8JmBB+ALiuYrqitjYZiMCfAMk9eDEiOsjV6nrgyUUQv0V2WHYFPAqeShKLKLUBBNj3Mm4wbq4ftPeSgi+W/dPnLnGr+ZDw0QDRs7NtxHiv8Yf1xZeQ8Yz8jzWdbYrhm4fipF7R1LWFCWpJZKNIa22WpRahBYR12D1RgXHpxzH1S/ZUmUtWPa0xPzQ+XAcSZ63RPtFw3+1f6PWC/fX3sfvyr/T1s/SttTGKZ9TODP2TIPOAWM3Y9r3Hh3V3GpkPMtoVDDms13PAIB27elzwEfyb5JM1pG+R+DL8sW5CeeI52dlASu+E2L+TPJ3vLF4LvbEwoiF0LMlb1xeW3gNftP8cHMZuXyFZU9LDNk6pEHHbtzVONz78R7Fsdb5K5GTOsl1dOMa/xFEnYnClXlXpJZQ/VDyXuchP1p6OdhPkYRbCQj6MwhH+h4h2m0H20rk9WSN2BHmv9of/w37j2Lf47oH+732w5fli+uLr8v8/o0l9pIocKboDTUYDhApU4gRz+HFmHQ2gftCd0y9PRXf532Ptl3bvnu6XBHP41pbzb1W1ly501Q+CfG75saSG0T7tNvTwKvgIXAddQ6RH5uPjW02Yrv9duJ73nORJ4b8Q6+oIA44NHIxQm2l9MoFjUVZSxluc9xoywq8jz1ue3B69Gkk3qVPbgj4JQB6dnq0svEMDIyMJ0OL4OY3NxF5MhI+B3yIkiQfQ8azDNxcehPdVnZDuzHULIeIExEARBvCph6m8F7mTTnmqxdfoaqkqlH18xpC21IbWubUyEoADUriAIDzJPpNOlU9VZTniApB61jpgK3Ycvz6EScjkPYoDQM3DqRE7ImdqnQZntxMLkpSS+C11It2kcvA0NTYDrbFtynf0taWZGgZGDgZAACe//sc2S+zMefxHMoxyuWFaPfoAApd+wBoGbVauFlcqOqrYs4TansBIHBdIIqTitFhQod6DiRNU02MPTkWCbcSoGOpg5eHXyLKIKpBvf93cRrthCdbn+D5nufwWFi/Fgqbw4bHQg/K52IcRztCx1qHNsNizLExqCyqpF243FhyA3r2elgSt4RoV1BRwJzgOTKX46vmVqPzvM6w7GUp9bnFKcV4vPExXGe7Eu116+XWdczW5QmrGzC0G/AKWCR1CxpHeV457v1wDz1/6NkoCa43t97g4W8P4TjSkfZ5KKgogMVmQSgQyjRQqiF0rHUA0G9IDds5DNcXXUdRUpFEWlSMkbMRFoYvpL12bXUthAKhqF8y3kFKf5KOV0dfYci2IcTfz+tzrxG6LxRuc9woqgiKqopE6SIxJSklkn+P2FM/QExBRQFfR32N+UuV0cl/K5TzyLUpm4LM55k43OswAPJ8Nyc8B/s89qH/+v7E+qVsRTYEPAGm3Z1G2UzRtdXF9HvT8XTbU3SZ37hstcaSF50HALDua020j/iXHKQHiBws3Awuxp8dj3ZftKP8bgxTngNgo1inGCylR1K1Ky04Dc//fQ6vpV6SwIu6uExxAZvDRmVRpdQOerFc9LOdz9DrB2q2V93gENdZrhT7mGNjUMlRx9H+R+Ho44gJFyZ88L3/sfkHFfkVGLx1MLp+27WeTcAXIO5KHLQttGHSmeqk7/JVFxh2NITXN2T5K7fZbijPLUdtdS00TeoHg5SmlyL5QTIAkRNq5L6RlNIMSUmA0+ODUKwuA0BWF2gsxcnF4ChzKO0CgLGnxuLawmsUqUAAyInIwW6X3TDpIpJwftcR/mLfC0Sfj4ayljJen30NzyWe9aT7BqwbgENvesMw5Rn0014CcJVpv+jIicjBky1P4DTGCWptmkaS136YPdQNZbvOb63YDbbDjzU/0r673Wa5wW2WbPdrxChrvv0dvW/uQKckIQ8GbRmEjKcZtIoaBk4GUNVTxaygWZQ5hFUfK0nt0LqZSi0FacbklopQyDj/GsLPDzh6tGmu3X1ldxg4GdDLJLNEpQ40zajvM3Ggp7KWMq2MrNcSL0SfjyYGa5h6mIrk7h12wH64PSZfndz4jkiBsqYyyrLKEHkyEt2+kz4oQrx+fFdquy7hx8PBq+Ch6zddaY9h+LxhnH0McoPFYsHc3BwsFgvK2qKJHSlq+WMJWheE9CfpeH3mNdHZZ9XHCuW55ShNL6XdtCItEj8WXWtdfJv8baPP/9f5XwDAwgjq5hSLw8LckLlQ01eTZHm8i1AoxMXpF+Hg4yDVZvDHcmHyBQBA39/6Upx9enZ6cJvrRttmU3dTVBVVYbvddnz9+mtKnRYGhuaArcAGr5IHBRWFeovSD6Xu2MfQtEy7Ow3HBhyTyFu+i26WKNOl2qDlRNH2/6M/+v/Rn9Yeulck90PKoCuIK8DF6RfhudQTLw+9hJqBmlTju9tsN3AzucTNcV4FD/mx+dA00STKfDqNckJRYhFir8TSZrHxq/ngV/NRU15TbwOYm8VFVXFVg5k8bA4bxcnFyI/Jl+nmfWpQKi5Ov4iJlyaKsmne4U/1P8Gr4GFF7gpKwI9QIERlUSVttGjms8x6/yZFaxukhkI7Jw7Znj4AmqeumLaFNry+8apXg+pD4fP4iDoTBacxThLn2rsoqimi+8ru8JvmBzNPM4kzqqnHPtcZrrDobkFbo7PDhA6w7GVJcfSJ4fP4ENQKiA6985POI8YvBvOezyNuon8MgX8GIu5KHLqt6Eac//AqeHCZ6kJ83/AqeCiIK4CGsQbxd+k3XZQBN/7ceIpsH4vNQvrTdLRJLcLz4T/DyLj53kliJ4VVXyuiXVzfmeTMYLFYGHtiLPJe58G6nzXlWbFYLFj3taZ1uL2P8OPhqC6thsfX1MCGjJAM3FhyAz3W9CA6Wb88/yV45TzaepaAKKMsNTAVo4+MricPbNXXCu3Ht8fV+VehrK0M24H1x8N4j0koEXwH7RJtlGm8lKpPyQ+SEX4sHLo2ukRnHzeLi+f/Pod1P2upg+k6fNkBGiYatBtxZ8aegUE7Ayx6TQ5ncPRxxBKPYKgZOMB6rHS198SychY9qEEL/Go+To85jU7TO2H0kdEUu+0gW9qakADw7N9n8F/jDwDos7ZPPRuLw4KZlxlKUksalNzTKBLNPWQ59r0+9xpnx5+FRQ8LzAqcRbHbDbbDN4nfEM8Vv3ezXmTh9ne3Me32tHr24qRipAalYsb9Gej7a1+KnGfyg2QoV6jBKDEYpQa2aC5nH5vDhrGbMRx9HInSjrJg8rXm2QRuDbDYLHDY71cbeF+tysYwYP0A2Ay0Qfz1eGIA9vuCoeWFcSfjButkL4puOJxru8N2cDO4aD++PbyXe7eo7L6qkirJe60xwVvyWvcyy+wP5+DB+v8ta+doQ2pIkaciMebYGOKcysjZqMHffHJAMjJCMjDx4kSo6FCDsLt81QXpwenICs2CzQBy0GVT4TzVuZ7CizR8m/Lte49ZFL0Id1beQbRfNHGPm0H+yHvPr+Wk+zB8drDZbOjr64PNZmPgXwOxNHEpXGe4yvw+Xt94oeuyrui/jrx52u6LdvBa6oXv875Hp2mdiMck3U9CwK8BtDKfjYFfw8evnF9xYeoFoj3aLxrXFl2jvWduZC6t3E1hQiGufnUVif70qd+8Ch7Cj4fjxe7GaUk3FkV1Reja6kJFm/pCNnEzgetMV0p0rJi6EXx09WlaI7FXYvFkm+zrHjA0Dfkx+djptBMPfB806vy6Yx9D0/F893P4TfPD5OuTsTxzOfEYDq8KfI4SqvTJ9bDkReLdRESeJsuPzgqaBYeRDtjhuKPeuJEVmoVDPQ8BAEL+CcGYY2PQ9zeypCYdhh0NMe7UOMomMyCSUtnbeS82m2ymyOaJufe/e3h56CXR9vCPh5J30rvSYJWFlShNK0XY/jD8bfk3aquozjOhUIiHvz3E/Z/vS9Wn92HmYQZDZ0PaPokl3oqTiik2XWtd2A+3R/w1smxkTniO5N8RJyOIx2jlJUI3JxZqfHKduaZAUVURyQ+SJQ4VaRAKhCjPLYdaG7UGN111rHTQcWLHepG6TT32qeqrom3XtrSZSXFX48DN4hKdEnmv8/C70u/4U+1PibwliboOXFnRfWV3DN0xlDar5M3tNyhJLamXKSom+UEy9rjtwWaTzcRz3ea4YdDmQbDobkGZTwoFQjzb8Qym8YFwvretWTfG9Oz0sFa4FjPuzSDadax1YORiBOv+ZIdd+3Htkf0yG5fnXKbYAtcFwpfli9vf38a/Lv8i4Ra5DjQdftP8cH0RWR4u4qRIav7RBnJmnb69Poxd6Td8L0y9gOuLriPyVCSl/lGbdm1gO8gWlYWVCN4cTKnZV2LkAJUqNUw7Pg3TjtV30rwPsYOPLqjR1N0UY0+NhXn3xtVOt+xpSStxa+hiSHQwAoBAAOwc/wBmcQGoVtMF38ZeqvuuKlyFtcK1xOtzlDhwHOVIW6eHV8HDw98fIu5qHNF+/eu33wFxdoEYTRNNzH0yF99lfoc5wXOI6xahUFQXONO+l0zHPnFdc8fR0kvJ69roYsg2keQZKdu8/5/98b+y/8HMwwwGTgaUMefM2DNwCj4MtoCPFBf6LFVZ06Z9G8wPnQ/Xma6NOr80vRT5MQ1LdJ4ceRLHBh5r1PU/NWrKauD/gz9iLsUQ7S8Pv4QvyxenfE7J/N7FKcUI3hQMY1djYgBWS4WbyUXwlmBkviDPEQS1AhwbdAy+LF+JkpGYaL9ocDNEGoqvz75G4RtyrUR58a/zv9hitoUoZ/whyGvdW3dO05w1iRnqE3kqEnfX3KWt+ei/2h9B68m1c99H3NU43F15Fxt0NxBlf3c47sDrc69h3d+akv3flJRll+HZjme0QVCygFfBQ/y1eBTEFTTZPRg+Dnnv+TE7jQxyg8/nIyYmBnw+H8qaytC11m0S6QKr3lYYvGUwbST3mztvcOf7O0gNSiXWCQKAo/2O4sHaB6jIk52zr6asBkKBEGwO+Wd45oszeL7rOXIjyA49t7luaNOenNkm5AvBzeLi6ldX4cvyRVUJtUYSR4mDPr594LnEs9F9aAxzgudg9JHRxMgwXiUPh3ocon3hK2spY+CmgVAzUINBO4OmbuoHU1Vche322xF/g7zh+z5eHnyJ29/dlnGrGJqK1MBUAOS6Yh9C3bGPoelQ1hJJaJRllxGzQwAgreMwhA1dA75iy5FkzQrLwrGBx3Bp5iWiXdtcW+KYKoh9O8F/d1Mu80UmRfLvfeztshfHhxwn2jRNNCX3INXMefjHQ7DYLAxYP4B4/rMdz6Bro4vFcYsp2QIGjgZYkbMCgKhu1bv1TMT3zHudJ3OZLbU2asiNyKVsvohpP16UpRi8OZhory6upq1dV7cm3rubxWJKDUTODMXq8g9u88ciFAgxbOcwjDk2RupzFZQVsCJ7BQZvHky7cK+tqkXm80x0ntcZZh5vHelNPfY9/P0hfFm+yHxO3my7NPMS7v90n/gsSlLfyl2C4PQa+s9QjDk+psHs08Zi0cMCnos8aTcFvnr+FcYcG0OUQKqbCXjjG2o9lM5zOoNfw8dmk80SZQUxNeU1b2tSC/hQLMqhnC8vvJd5Y8GrBZRsRECUgbleez1i/GLw8tBLym9XnJkWvCkYuRG5EPKl2+Wb7j8dU25MIdo6z+2Mrsu7Yuj2oUR7zKUY+LJ88WIfOZAu4kQEHH0c8RPvJ2JmbU6E6Bm8ufWGUj9PMz8JbQpdAACvukhXo87M0wyzH89G+3HkrOu7q+7i/MTztO/Khnh9/jUuz7sMbia54M+gTYOQG5kLX5YvpY5cYKAQ3NuioJUc226olGE5NrYCG0n+SZI527s8+fsJ7v90X1In8V3qZguSNt9D94did6fdtM5CoRCI95qCTMe+Mh37BqwfgG+SviFmPQHAVout8GX54tUxcjBHx4kdYexG75AGRPLWZdlllHmutqVIMrdM1xw8Fem/Kx+DuDZkQbz0G5tbzbdiZ7udxHrEYuKuxjVYH+lzoiCuAEF/BuH06NNEu/gd29i1eObzTFo1p8Q7iUi4mYCL0y/i+Z7nFPuf6n/Cl+WLDXobEHsltlH3bwouz7mM29/dxutzr4n23xR/Q+Id0fcr8lT9YL6iRFGdPyMPIww8PlBSd7alUJomCkajk+l/H/Ja9woE5H8zvB9ZOkdjL8Xi0fpHONjtINH+xYkvYDvYFulPqCo8FfkVODboGLaYbSHW2/Za6iWZjyXcoAZ2abXVgqqeKsafHf9xnZASPo+PosSiRieKPPB9AF+WL14efkm0CwVC8Cp5WBSziCh1z9AykPeeH+PsY5ArVVWiDb0Xe1/gb8u/kRVG3mz7GIRCURR6dSm5QHpZlmjBfHr0aUn9hXdxnekKU3dTWsmqxiDerNW2IhduHbZrGCx7W0LfkSxH5bPPB19HfU20GTgZoLKg4RUzR5ED7+XezZ7SnnAzAYd6HEL48XCKLTssGwBoo3BfHX2FOyvuYG7IXHT/vnuTtlMa8mPzUZhQiJxXjdso07HWkXmtSoamo613W3Rb2Q1eS8h1Xj4E8djH0HQ4TxY5Wi7Pvoy0x2nEYxQrS9AucC+0Y55Kde3a6lrEXY1rtMO3IcIOhAEAhv87nGgvTS9F5rNMOI12wvBdb48xchFJnfzE+wlL4pfAYbgDStJKiNegQygQ4s2tN0i6l0SxaRhrYFXxKqzIXUGUSSzLLoOSphJ0bckyzHOezMHMgJnQt9enOCbZCmxcXXAVYAHjzoyTSHvXhaPEgctUF3gsItcMbCwqOir4Oupr9F5LrtnY1lskpdS2G1VSqaq4CpZ9LIkScUD9WlWquuRsM66+FZI6jUKVVvPJUue9zsOhHockGSLSErQhCOs01+E3hd+I3zFeBQ+3lt0iZqc25diX/1qUuUG3YdthQgdkPM0gqh5Y9bXCipwV+F/F/4gbWlptteAyxQXa5uQ528cQeyUW26y30QYMPd78GH9b/i2Zr9bFwMkASxOXAgCqish/W7EU4buZQBxFDjwWeSDRdQwUaiqg9Vy2WbMNkfkiE74sX/iyfInjaHFyMU4MP0HMiGWxWJI5vcNIB8p40eHLDtDuInpON4bcgOkA6WRXrftZw26IHdFm2MEQgzcPps1UEwdp0Ek7/lj9I748/yXYCmyKrM/Lwy8R8k8I9B31MenqJCio1ncGOj45CqOsWvz9zd+IcJNuvsmv4YPFYoGjTN6ste5vTVt79H1kPstE2P4wbDHbQrQLBULJ/P5dVY/8PKBKTQ95Fp1h9+wUYjdfleregX+KsjhJawpAtInY+xfy2C7O2u75Q0+ifVbgLEy5MQVe33hRsoUriypxZd4V5ITn4OTIk7QZL8YJj9D29R0Ashv71AzUoGOlQyunJ64/m/IwhWIrTS/F7RW3kR2WTZSfzo3MReyVWPiv8cdmk80oTa+fcT5sxzBk2fVAsstIKDRjgAo3i4u4K3FQM1D7qMBgUqCSGHVDdaIc7OeIVlstGLsaY8AGcvDWoE2DsLpkNW1wV0MUvinEPo99CPg1gGivuxdDCrIS/26riqoQcZysmCAPTD1E7wTnSc4NHjf+7Hh0mlFfSarbd93ws+BnLBy+EN0TuqNSKMOoBxkw9+lcTL42uVHS72Kae90bHAycO9est/ykkKWzb/DWwTDvZk6syQeIlAEe//UYD397SLEVvilE4p1EcDO5xPestrk2unzVBW3atyG+E637W6OysBJnx51F0IbGZQ82Bm1zbbjNdUPSXep6+kMI+EU0Piqqk993gloBDvU4hABf8jjK0HKQ554fU7OPoUXw8tBLlKSWIPZSLK0MTGO5sfQGnu14hnZj2+HLc19S7Mn3kyX/Tn2USnR+jTo0SqZtAgAtcy1MujqJtvaQx0IPeCyk39S8//N9KGsr0xZ9dZ3lCmVtZfRY3YMomQkAm4w2wbK3JaZcJ0cwN4aC+AKUZZfBzMOMOCm8/5NoM4n0whYvKg2cyJGCFt0tELwpGCHbQ9DHt0+j6qU1BTpWOhj8t2gi0xiebBVFNY/cO1KWzWJoIkzcTGQ+TjE0DY4+joi9HIuD3Q8SNf/VSrKhXpKFSp50WvcPfnmAR+sfYfDWwTKXBXGd5QqLnhZwGkWtXQAA+7vuBwBY9KJuSiXdT8Ipn1Povro77v94H55LPTF0GzkDhcSQf4bgcK/DyInIgXU/qnyekrpSPQdWXYZtH9bgxo+OpQ6qudUoTS+FupF6PYdKbVUtynPLYeZhRltjUFlTGaMOjwJkLMUTdjAMd1fexaSrk4hZLYqqiui9tjfc57tTbNxMLq7MvYJeP/ci1vat5r4NMgpaF4T+f1LlxFlCAXgqmmApyzZjsSFUdFTgPMW5UXVvBbUCFL0pkvx3QWwBxQGmpKmEIduGIPZSLDKfZ9I6RmRN1+VdwVZgw7K3JdHutdQLKroqRIedgrICFAzpl0Uv9r1AyoMUFKcUo+OkjvBcJDtVhCdbn6A4uZiY0QoAavpqsO5vTXH+iNG11qWtaXJ1gch5YjPAhjKnVFBRQJ9f+mD/RB7SOgyBprlO4zshJXU33XkVPIoMYvyNeMRfj4dRJyNgUv1z2QpsuH/tjuhz0Rh/djwxg9njvAd67+8NIUuICl4F1JU+/Pd1Y+kNqOiqoK8vVQaZV8FDfkw+NEw0iONF99Xdwc3k0taC4yhyUJhQCG4WF+be5vUCHzjKHLAV2LDqYwWTziYU5Y+SNnbQzkuAaqUq+Gq8D+4PIMpaujz7MiZcnEB8t2i11cLt5bdhM8DmvZvV7yIOlCBJHQMiSe1qPWPkGrtAZdA7QVIsFiL7LoJCTSUcgw9LvcN474d7AMhriurSalydfxXOk51h7k2dn/f7vR/6/d6P9tp5r/Pw31CRLNiQv4fUs73rNKopq6G8G4VCIdrG3BXf7b19+VDK88pxyucU2nZri8GbB1PsY46OwdB/hhIdu2U5ZQg/Fi5p87u82PcCIf+EYNjOYfBc4lmvfAIAOIxwQMYeB7jeXI8KHTMA0snJNpaSlBJkh2Vj4KaBxGzf9+E21w05L3Pq1ch8lyk3p0ithvCpom6ojvlh82ntCioKjXb8KGsqg8Vm0QbLeX/njYK4AmQ+y8TIfdR18cRLE1FTXoPqkmpamWd50PfXvsRa2mIWxy6GgooCtC3IAUN3/3cXw+8OR2jnUCQkJcDV0bWJWio9Zp4tq9TBh/Dnn/X/m5HxlA5Z/r00jDUw+9HsBo/xOeBDrHWs76CPiZcnok27NsQ909rqWghqBei/rj/K86gBKO4L3FFdWo1nO5+BxWY1axZcWXYZCt8UNqq26aQrk8DisGA/lCxtzmKz0Pe3vqgqrkJRYhGx3jcDAzOjYWgRDP93OML/C0e7L2RfXFS8gUCXwTYzYCbKcspQklICw46GxGNqq2pRU1YDZW3lRksYUNqlqYw3t9+gLKsMBo7Sy2CIo1+6ftOVkiVRkV8BM08zmHqYUuTSxFRzq8Gr4DU6G42O8xPPIys0C4M2D4L3cqrEjFiWSJx1U5eOEzui/fj2tNKm4sK7T7Y+gbGrMTpNJ9dYbG40jDRg3s28wUUkHWIZE622Wk1S6JxB9lTkV+DchHNw+sJJphu+DLIl7XEa1AzVoG2hDZtB5PFfN0dUj6TYUbrn2GF8BwgFQtoN3Y/BtIspTLvQO0fcF7gj81kmSlJLkP0qG8adRHJcZTllONrvKAAgJSAF3VZ2I9bea4i2Xdvi+/zviQ69vOg8nBx5Eu3Ht0fvn3sT67VtNtmMtl3bYtpt6gZgaXopAn4NQOi+UCyKXlQvqCM/Jh9pj9LA1zfEjpWpWPwXObr+YPeDKE0vxfJ0cg3GxqBmoCaqC0OzsL228BoAkeP4XQ4l5UEAAQAASURBVIeetoU2vL/zpj23bk08uqwZw+SnMEx5gQSz7wGQ39eyRqutFgoTCnH7u9tSy0Xxa/h4deQVHEc5Yug/Q6FpSnV4cBQ50DTVRNK9JBS+KWw2Z5+Zh1mD0qTm3cxpg3KqudUI+DUAxYnF+OK/LygbmSH/hEjqJBs6k+eKjcVmgA3Kssto56nBW4Jh6m5KzA5Nf5KOA94HYDvIFlNvTaXYi5OKYexqjGl3yJvyl+dcRif/WGQ49kW1hWyzZhvCdqAtJvhNQFFiEXHTWCxR7PE1uU3Ddw6HdT9rJN1LomyGJNxMQMyZOJiwTWCUY4T82EK06fzhju2Q7SEAQHT2he4Pxc1vbsKqrxWx3mBDG0lCoRBpj9Pgv9ofqUGpWFW8ql4wnvMkZ5TnlOPWsltwnuIMDWONenPCeM/JyBP0w/y98xFvFw8M++AuSaRME+8kEp196m3UYd3fGhpG1M2296GoqthgoMerw68grKpGjo03gp8CXevEKAqFgGZBChyfHEWWbTf0+3agVPeeeHkilDWVYdXHimpkiYLx6OoUCoVCVBZWgqPEIQYPXpxxkfa+Wm21MO/5PPBr+FBQViDWCRX/zUvakLNEG8ud7+8g/Uk6bWYfi82irVtq5GwE19mueHnwJQrjqdKkLlNcYNrFFHqD9dBBrwPUFOv/7Tabbob7/2cYF5nIfs1OR5sObTD78WzajNn34bPP573H5L3OA7+aT1sio7VSUgJoaAAc2WxfiK6ZWoJLsy7BYaSD1AFvam3U8GP1j8QatIBI1r33z72ha6NLfBaOPtLXqmwOhAKhKIOawyLuFek76OP1+ddIf5JOmXcVJRbh8frH8IAHPJ57oMK9AmimbpZmlKIirwJtOrSh3eN6se8FEq4nYNShUZK9mNbG5+7sEwqBbdsARUVg0aLmvXdFQQVqK2uhaaZJ3Ofaar4VOtY6mPVwFsWmqqsKx5H0P4YLX99F9MGnsJjWC/2/oq55A9cF4sVukVT35GuTP6IX0lHNrYZ1X2t4Lvak3dt7vuc5tM21YT+M6tBzGOHQ4PXZCmw4jXHCvx3/haK6InG+ysDAyHgyyA02mw0bGxuw2WwYuxpj0MZBMHIxkvl9+v/RH2uFa+G+gBqVD4jqZ+xx3QNVfVXaqIgtbbdgY5uNMneMPdvxDPHXyLJNm003w5flSytHJX4xkDTvi1OKcWXeFZwecxp3V9+VSF7URVArgLqhOjpO7vgRPaDiMl1UT0RVn7zQ9FjsAe8V3sRJPovNwqEeh3BjKbXuDCCSyOj7m+hllvqIXINDXhztfxR3vr8j9Xni+hGl6aW0Uf0MLYv46/FIupeEGD9y4fr3UXfsY2g6Uh6mIGx/GL688CXtRk+eeWckuo6GgC1d7JNJZxMM3DCwyTaFzn55FhsNNxJtvX7shdzIXDz9+yme/v1WflRcUwOApC6ItDLNIdtDEHs5lrj5zq/ho+hNER6tf1SvVqCYtOA0sDls2kjrPW57ELovFF7felE2CzSMNWA1sxc4Bbko2HiIIh0GiGTTMp5mEKXHPgabATbIjcyVyKe+i7juHqmWhJKGEvKj8/F442PiuaZdTCXSTnQOQb6i6F2pltM4qZfGYjfEDk5jyNmjDcFR4mDsybHosboHtC20ie9yoVAIu6F2+D7ve7Qb83ZDuKnHvtjLsbi++DoxulcoEOJ3ld/hN82PeG5KQAqCNwUj+kI0sl9lU+xjjo+BvoM+HEY4oMu8LjJtd8//9cSi14ug3oacfdZ1WVd0nESeq4nrgby5/QZ73fdS7FNvTcX8sPnwm+aHez/dq2erKa9B7GVRzaM2yc/Q3LFGTqOd4L3cmxgo5fWNF2YFzYJaG6qjhs/jw/9//jg77ixODDshqXckJv1JOtIPpWLugbkYeXUkSp9LN7dy/9odDiPJGy3iTCdSJi8AFCUV4fqS60QpZKFAiEM9DiHvdR4G/z2YmEUkfp6Hex2myrKyWLBIEb1zNMqkc8pZ9bWCy1QX4qYSIBq/1fTViBndQqEQMRdjaGthcjO5yH6ZTVuDa9KVSUjtOAwWEddQG1W/xpaAx4fjE1GQSrkOVSr5fTiOdCQ7+iAKqqytqqWtqZfxNAMbDTbS1set29+CuPrvPAVlBeha60LbXBsmnU2Im+RCFgsRfRYjuZOPTMc+u6F28F7hjRn3qc5mQFTb7sKUC/XrkP4/HCUOvJd5w8zLTFKPti5mnmbQHaULi90WsNhKDbqpKyWcZ0leVzcFyprKqCmrwaONj2hrQ34sD397KFGf+VRISQGmTgVWrZLuPHGNTV+WL9EetCEISfeS8PLQS6nbVJpWipvf3sSbO2+I9vLccpwceRIBvgHEZx13LQ4hO0JQll1GWwtZHjze/Bh/qP5BlCIEgKD1QTg77izOTTiHu2vu1rPVlTwvVyt/b01NWRL4ZyD2uO2hlQEHgKtfXUXMxRjwKqXLKBfDrHvlT0EB4O8P3LwJ1JBVp+shyxqH1xZew1bzrXi28xnRbjPQBubd6dWxihKLEHMxhlgf/UmKGSo12iD12EMc6nmIYo88GQktcy18/frrRgXlN5bynHJcmHKBUp9TDJ/Hx7UF13Bi+AminZvJReC6QOLaU4ymqSaG/zv8vY5BBvkh77GPGXEZ5AaLxYKWlhZYLBZyI3MReToSVSXkicaDXx5QNilkhVjm5OL0i7R1/cT17+iiQxtDbmQuhAIhbU0+dUPRpg9bkfwz7b++P6bemkrcLNC314e6kTqKk4rxaMOjelJiYlR1VbEiZwUGbRz0Eb2g0vWbrlgrXAvXGa5Eu7a5NoI3BWOn006KrTy3HBkhGcS6NADwaMMjBP4ZiJH7RtJGEkaeisSJESekrlX1MWS/zEYNt4YiQ/UhKKoqwueAD1ymutBmNDK0PBTVFdF9VePqRtYd+xiaDtdZruAocSS69yQqtY3BEgigXEDexKSjprwGpRmlqK2SreMJAO6suoPXZ19D3578bgDeBnm4zXlb69PAyQDD/x0Oj8UeUNRSQaaildSRrAG/BuDmNzeJzhLjTsaYGTATvX/pTZRaebbzGaqKq2jHZvev3TF0x1AM2TqEcr6GsQbelIqypQRsDvG9J5YcEweUyAqOkqhuGZ2T0n2hO1T1VWlrXfX+pTe+vECVCAdEmWSZz0TfLbr6eKUGNijXNgVfsflkqYtTilGUWASbgdLX7GUrsKGkqYQD3gfgy/LFq2OvKMeUZZdhncY6PPB9UG9x3dRj37Ndz/Bs5zPi31ooFEJJXQnhx8OR/pS6eNZ30Ee370UpR6TAHeNOxhh9ZDTMuppB3Ug6ydWMZxnYaLgRz3c/J9qLk4vxaOMjopMREP2204LSiBsdDsMd8NWLrwAAWS/o616/uf0G6cH1+y3eKC02tAdLwIfWQ+nqpX0MZTlliLsaR3TsA6JMs+qSahQnF1NsgloBgta9rb3Czaq/IVxXVSLGKRUWfaX7ng/fORyTLk8i2txmuWGtcC0GbSLPnW8sFpUOyAqlPgsWi4XBWwdjzLEx6PpNV0pQRebzTMlmmKm7KWUcbB/wLzq87oCt327F5S+pARcNoWerhzHHxtA6+1IepCDiBLn+lVAgxOkxp7HPYx/RHrw1GHvc9uDMF2eIdk0zTTiE/AfDlOfghUfXswlq3r5DFXiVKHwsXRBVRUEFXh5+iZxwcjCmkYsRrWqLuM6fkSs50HTKzSmwG2oHRTVFyvpQwBfg7Piz2Gq+Ffu77pcE7tWFxWIBLBY4NZUyHfs6TuiIQRsH0W5chu4LRcSJCNz7kbpurq2qRW1VLTKeZhBr+gHAuVB/dIzoiAGHBlB+n318+wAAYrs2j3ynGKFAiJSAFDz9+ylxfvI+xI6rd8eKuhTEFhCDY1sz9/7/KxAb2/Bx7yLgi3b66aSjnUY5wXOpJyZflz5TpiynDM92PkPYfnKA1c1vbgIAos5E4aTPSYr95IiTuLHkBjabbIbfdHLwjjxICRD9nrQtyTKd4tq5AChSnvZD7THo30H46/u/sHHlxiYJfqfDdqAtuq/qTvusAZG0qvNkZ6J09YfQEta9n3tmX6Ucy0DaDxfNO4LWk2vmjTo4Cm9uvsGZsdQ5RPqTdPxj+w9OjzmNvNd5FHu+qTPivKiqFmLcF7qjNE1U6z43KreRPZAeDWMNdPu+m0iymE/1nIrXht1Xk/eStphtwb3/3UNKIPk9XVNWgyN9j6AsR1Q6iaFlIu+xj9lZZpAbfD4fERER4PP5uLXsFs5PPI+I/8gLzQDfAAT+Htio+8TfiIcvy5e2KOvjv0QR+UWJRXj4Bzkaa0XuCnyX9R10rHQa1QYSiuqKsBtqRzuhW/ByAdYK18KqtxXRzlHiwKCdATGqXllLWbJg8f7Omyj9BAC3V9yG/w/+RFtTcXeVKJqNFHGacEu0Qfdu4WoxKroqqK2sRebzTNpaQy/2vED8tXjaDaSmQMAXSCL+G4PbbDeMOTamUc5CebKr4y78qvCrvJvR7HSa3gn/K/uf1BKJYuqOfQxNh4aRBvg1fMRdjcMm403EYzi8KliHX4ZWMvndQ8fFGRexte1W3FkpfTbv+9Aw0oBFTwtauZELUy+Am8GF3RA7WPR4G3mvpKEEy16WUFJXQmT7CYjcGYCjX0n33hyybQhquDV4dYTqwAEAy16W6LO2D9HZ57XUC+PPjaeVZurr25dW9jYlBSgLfAk+RxGhw34kSslpmWlhdelqYt27jyHtcRpUdFRonX0W3S2wMn8lOs/pTLHlRubiUM9DyA4jO2kASOSa6IJYuAbWiO45D+XGspV6a4jKwkqEHwtHbkTjFr51zyPJuSqoKMB+mD3YCux6ta2aeuxzmSZyBIvl8+rC5rAxbKdI97BuFqwYfQd9DPxrICx7WxKztgoTCvHk7ye4/+N9cDOkyyypKqpCRV4FMp5lEO2hB0Jxd+VdJD9IJtozn2Xi5eGXxBpbgMihsTxzOVaXrKbY3tx5g/jr8ViRswLT706vZ1PVVcWKsrVI8JiEGjUdyHopGu0XjbTHaURb5rNMnBx5ElvNtxI370tSS3Bi+Il6UrhiFJQVJFnLC14toGx0KGspY9rLxUju2BPVJmugYSzdBmXqo1Tkx+ZLdY6Y8txygCXaYHoXFpuFrt92pXW4JT9IRlVRFeY8mYN5z+ZRpCXVuKLfnSZXE3plvaRqV7RfNPZ22Yu0YPLz6PdHP3SZ3wUZIdTvqHiTgs65ZDdENHbFXycrlZRlvx371IbXl5piKSvidY+5CO+7FG2j7yD7csj7O1OHjQYbcWnWJcReoXozeBU86NnrwWk0OYO589zOGHdmHG3tc7HU7OqS1RQp4sKEQkn2ZsbTDJRmUMcUAV8I+5D/0OHhbpmPfYn+iUgNIiuczA+dD69vvdB+HDVzL/tVtsRpa9KFOs7d++keCkdnwyLVAk4xTpRg0d4/98bzEWuhn/YKZtGyn//QkfooFYF/BMJ7hXejas2KqS4hB/UCot8s3cZra6WxDg7jTsZYK1yLHyp+INptB9li6LahtPUTby67iX/s/qG9NkCuGQngrRoCQJyX1c32tuxFrs8rD4b+MxSzgmah0zTyHob7Qnf0X98fPwt+ptTPNXY1hklXE2iViv6eUXlRTd5eMfHX4/FowyPaGooAMGjTIHzx3xeNvkdLWPd+7s6+6jpD34f8LWT593Kd4YpF0Ysw5foU2mM0zTSJSg5KmkrQttRGuy/aERMkamoAnqoWci3diRmxZp5msB9mj4szLuL619c/riNSoKShhLKsMgT9GQQBj/rbEgfb0Umn2w+zh7EbfckioUAIXjmvSYKOGWSHvMc+pmYfg1wRf/Gt+loh8W4ibbFnDRONRsulRZ4UpU+nBqYCBBkLzyWeqCioQM6rHBi7kmUT6KSVPgZda10M+XtIg5FUDbHfaz/MPMxo67CsyF4BFodFW2ycV8lD8OZggCWSOpUVV+ZfQejeUEz3n06UAxLjc5Aqq2fqborBWwejTQfys7bsKZrUv9jzAiN2jyAe06ZDGyQ/SG7WQrWmXUyxOHZxo84V1AoQ7RcNXWvdBusa1ZTXgK3AblHF4/OiqBFWnwO11bWIvxYPHWsdmLiRpbzeB+Poa3pqq2rhc8AHl+dcRnkOORLcJF4U4FFs7SrVtY06GSH6fDSxWPjH4r3cm1jvVEzaozSYupviy/P1s8mEAiFSg1LxaMMjqHTQglIVFwUZ9BtbJGwG2KDPr32IcioV+RVIuJkAky4mxM02M08zRJ6KRG5kLvqs7UO8fvSFaLw6+gqDNg2q97dLDSuAamkO8izpa4ax2Cy82PMCZdlltFk1jSE1SLSJ2GFCB2LU8q6OuyDkC7EomlrkQklDCbYDbWnfN9EXolFVXAV9R33a2nhm0XehmxWN5CHzAVBrJTYFxp2M0efXPo3KgK/mVuPRX4+gZqCGTjM7SSJ266Kqq4ouC7rglM8pmLiZ1FusNuXY1+HLDnD0caSdS3b4sgM6fNmBttZVcXIxUgJSiLWOL868iLRHIkdJyPaQBmsDvou4rqG2OTniv+iNSIaSLoAkNzIXnks9KdkAgMiRErIzBBY9LGA3mOowvrvqLqpLqokOpqoq4Fv3ILTLikaCxySYOjYuap+OM1+cgaaZJrHGplEnIxg6G4KbySVGvAb+KQpUEM/76sJiszBgwwCUpJVAz16P4oSqKKgAqxa41v93lKumITznPLprfnhG8KEeIimotcK1FFtGSAb2e+1Ht++7YeBf1Ppys4JmAULQfgfFfYs8FYmZD2bWq63mPMUZFj0sYNiBmokmFAJRPb9CbdmfaP+6PdjK0mUgFb0pQlZoFmL8YmDuTR3fSzNK8WznMxi7GcPMs77zlMVmYVXRKtrsZpv+NlhVvAr8aj6x9vSxgcdQqa6PqL6LMfDdoZLFBk9FC+ZRN1Fg5owuX1GDKhrCqo8VuJlc4vhaU1aD+z/eh/tCd4lDsi7Grsa06z5AVE/n+tfX4TDSgZLpqaKtAtdZruBV8KCorghFNWrQQ3VBGVTK39bFk9XY93jTY9z5/g6MXY0xP2w+xa5hrIEhW4cQz637jos8EUmRJFbWUgY0NXC/733cGfwEv7T7pZ79/tr7MExUhV5mJMr0yLV1mwJ1Q3U4T3GGzQCbRkmx/VjzIwDQBiMBomAnOrnt1oospfjeRSyfTJr/vNjzglZynaPEgc8BH+hY6xDtvX7sBdvBtsgJzyH+rr/P/b7xjW5CdG10G9x7GL5reIPnXxp/CQsSFoCrwUWKSQo8v2qemvDm3cwhqBU0uL9QmFCI4uRimHc3JwZ4fQjyXvd+7s6+utKdze3sA1CvTvu7+P/gD9vBtvBa4kWxGXYwxLfJ39Keq5YQjjY5iUjrMAQ7b1LrSWqZaUnqbbt/3XzS0wDgudQTTl84EdVqxEkZUWeiiIo476svqKyljGl3puH44ONQb6Pe4L4Bg3yR59jHZPYxtAh6/q8n1grXwm22G9H+XeZ3lIjkD6X3z70x+9FsTLw0kWg36WyC8pxyDN4yGM6TnInHvNj7Avu77pd5+vcetz24Op8smxS8JRgb22ykvWd1STUS7yaKoojfISMkA+t11iNkewhqq2shJLyxxdH2dLUMG0tVoUiKla5GhqqeKqz6WsFtFvVZt2nXBupG6rS1EdUN1SXZc3dX3yUe031ld8wNmUtbnL6pODPuDE6NOiX1ebVVtTj35Tns89hHlOgSs99zP64var6IpA9hYeRCzHw4k/j9+pTJjczFmbFnEPQnOVuYoWXwaOMjXJ5zGQteLSBu2gIAT1kDZTpmqNEmy3zR0fun3lgrXAuvpdSFiSwIPx5OO8Z9k/QNyrl8/Kn+J/zm35R8nvk8U/I+sYy6gQzHPlDzGSDVfRVVFdH1m67EzeCC+AL4TfPDrva7kBdNdvS/Pveatg7t+UnncX7yecRfi6dIdguqeeDwq2Gc+BjuV31RUUDVm6mtqsWDtQ9EQSoyxH2BOww7GtJmpZi6m6KaW03M9NGx0oGShhKuzLtCPJebyUXSvSQUxBZIanG9i2J1OVQqCqFSTJ8dKGtYbBZSHqQg6rT00eMsNguGHQzh/Z03Bm0cRLvxY+BogL6/9W1wM13WVBZW0mb183l83P7+Nm39jNSgVGyz3gaArC5Q11mmZy+dk1/fUR/L0pbB+zvyYrzPL30w4/4M2uCBnFc54GZwiU7KtMdpCPw9EP8N+a9e3R8xAzYMwNDtQ5HyMIWSWZiUwIdR0hOol2TC6dEBmdfsaz+uvaTm5btom2tjYfhCrMxfSZTIt+5vjY6TOsK8G3UsEgqEUDdUx5mxZ7DTaSdqyutniDzZ+gRH3Xeg320LrNgwGUmnyb9tEu+bz4jrRb+5Ra43BYicxqRsRT6Pj92dduPeD/dQW1VLkX7UNNFE265tEXc1DlcXXKVIR1Zqm0CDK0C34G7o9KR+ncL3YTtY5EhW0iAHFKgbqmPcmXHoOIFcG1JFR6XBTV4VbRWoG6oTHbedZnRCgbmrSM6SV/9ZCfhCWL26BL2s1+ArqkDDTrrxYsb9GVgcu5goe62iqwKLnhZ4/u9z4nMVCoU4N/EcgreS3yniLIC4K3H1MpQBkUNt1MFRGHdqHEYdGEUJEMh4loEn47cAABLdGp8RQ0L8/u31Ezm7szChEEVJ5O+HtoU2JlycALYCG53nUR2r3b/vjqLZg1GpVgkhqBtUD399CIvXtyBQUEacN7lmYFNg4GiAYTuGQc9Wr1GZDBxFDjiKnAaltA52P4jt9ts/ppktjsYuz8qyy7DHbQ/tPDRofRD+sf0HxwcfJ9p/qPiBft5dyUNb77a0ykYVBRU4O+4sipOKKdnNLZmStBL4TfNDzEWyFHFJaolETvbdTOSQnSEoTigGAGiWaUJY1Xzral4lDwWxBagqpq/Zt7PdThwbeIxWoaIl8u5P/TPbqqBQK+WwKcu/V+C6QBwffJxYVggAQveGIvp8NNH2PlQL0v+PvbOOjur8vv5nJu7uCQmEJAQIBA/u7hQoFAoUSqlRoU6Fpq6UUsOLU6y4uwQNIQRIiBB3d53MvH9MI8N9Jk0olP7eL3utrkVz5s7cO3PvI+fssze2yTfocORrFrsulsTXDVhHQWIB3d/srnV98zCQE5XDxiEbyY7IFtr01BD875XXr4GyWklFYUXD851MTZ6Q6Ty2hXkMMR4X+x7j/wRy7+aSHnp/iTDrlta49XDT6oeWfSeb0N9DubbimlYJg/3z9pNyOUVrd8j9oDClkKrSqlpvvntxYuEJSrNLhb4fACOXjcRnjI+QLWJgYYCekR7H3jrG54afCyWrjKyNmLRjEp2e6ySJ/ROM3zie2Rdm0+1VcRJ85umZxJ+K53Ojz4XxkwtP1jK670VWeBaWzS1x7+OOlaeYPadrqEtZTpmwCPqwUJhSSMTOCK2/VUPQMdCp1ffX5hlZ8xkFCf+eD2FjYN/GHvfe7v9z3nM1PifOXbV3Yj7Go0eOnrrr8vhvYq80gDTvvtzp9SxK+X+nYzZyXyS7nt5F0NdBWl+TmaBOPF4Nq0uEGNloEhwMSvOpaqL9zM8+P7NltNQjBdTJNs8h6oSxiLG9d+5eSrNKmXlKnACMPx1P8wHN+bDqQ5w7aT47Fj6OhA18vfb/i7M0E6sZYRl8bvQ5VaVVdJr3YOcsI2sjMm9laiWZdH6+M0UpRVoJLHZt7PDo5yGMdX25KwtLFgIQdyJO+JpsN3VhySCv6fPH/aKyuJJeC3vx7KVnm3ysvok+E7dNxKqFFfue20f8mXjJa8pyy7jw3QVsW9n+q94zJz84ya+tfxX6d6mqVVxafIkT752gOEOasKo//9Z4edVH9wXdmR00m5ciXqLPB02TUCxMLiRkdQg5kWKfNeuW1pJuqvqYGzyXXu/2Em76nTo51RZwdk7ZKYl7DvbEa4QX2ydv58jrRzSDZWXoVdStlfRjm+aX9neYtH0Sg7+Wdr81Bh2e6cATm58QyptXllTyg9sPqKpVFCQWkHxJM1ni3lfdDegb0RyVTIWeg5TtrQ0ymYzXEl7j5SixWkPria2ZtH2SVtb1nd13+MX3l1rvPQ2o1GzugAUBzI+aLylyVldVU1VWRcK5BK4tv6bhH6ZUgm1CMPrlCiK9I9n5RNPIZXat7Xi34F2tXsOnPjjF7pm7hcVARYWCQFkg6weuFx57eelllrgvIe5UnLCoZjmiJ653TtDh6DeUbNUkOCryi7HIUhdOU737P9AEo46ejlrVhbru2fo4+f5Jbm+9zZ1d4vu+7dS6xKCIeHnpx0sEygK58otUerS6Ql0oS/IdRK6L3wO9rrGrx7JItQjfCb7C+Lr+61jaYimnPjoljLcY2AKZXEbcKfGcpFQpscyzxC3RWpKczXFWF+8LbbUrtzwshKwK4Sevn7R6mzaE7ZO2s+eZPVqlI0EtLSyymPi/jPu973Jj1HkXbevQmsJQfd/oxiLjRga/tv6V0LWhwnjYhjAKEgs49/k5Dr9+WBKvKZgFygK58nPTZH8fJnZN30XYxjBij8cK40vcl9T++16yiI6+DiYOJtwZeIdfX/gV+zFNIyD+E5TllpEXl1crKyhCTX7s3n3Gfxnye9Jj/+vFvvrX/2939qVfT+fu0bssbSGW9n3h1gt49PcgdF2oJJYXl8cSjyUEygKFc1Zq2yFE9JgNIJTYd+/tjq65Lrt8dpFW9O/ts3QNdXHq5FSr7HEvanz8tHWq73tuH19ZfEXQN+Ix+AvTL/jR40dGLR9FwKvSzsDHeAx4XOx7jEcIuVyOj48PcrmcQ68cIlAWqNUs/KeWP7G8w/L7+pzqqmruHrur1bw95pA6EXzph0tE7hU7WA/5fghDfxjaoCylCH9O+5NVAauEsZrCoY23lI0KMPv8bIb/PByv4WJvj87zOjNlzxShH5+tj62GGaxIXkbPSA9rT2setEmLroEubt3dsHS3FMa3T9oOIExaRe6LJD8+X2ODXR9R+6O48tMVOj7XUSI9U4O4U3FsGr5Ja2L1YSD7jrrjo/f7vZt8rI6eDi9FvMQ7+e806Ak5ecdkBnwx4H5P8aEg5UoKyZeTJez0/9/hNdyLcevGafVl+DvUH/se4+Hh9yB1F3DMsuOErAoRvkauqKDjwc9wvCruzNKGq79eJVAWqLVL6J8gap+6qDQ3eK4wcRp7PBaD8kIKbZoT7lDnf2Ttac17xe/xWuJrhAx9lwoTa8oixZ4+2uDWw42Eswnc2CD1yTKyNmL6keksUi0SeppZulvi2t1Vq3zdG2lvaPVrkMnAPUz9G9zq+wImzppyhTXFGRN7E9pOebDMzMqiShakLNAqy2jtZc3ErROFnqyFKYWU5ZVpJbeAeiPn1sNNq1dQqYUT4b2eJd/l32OcZoVnsWHQBq0M9L/Dzc032fHkDkJWhtSazNdHRVEFIStDJEWYhz321XQeidaSOgY6DFs6jILEAmEyzmuEF4tUi3Dr6VabqK8PXUNdIvdF8ovvL7XyZY1FTlQOZz4+o7V7NC0kjS9MvtDaYXTl5yus7LJSKLtq6W7J/Jj5OHV0YtDX4k7e/PJ8On/SmQ6vaSZmjSz1SWg7gjvdZ6FTVY7hTUGB6h9g//P72TN7jzAWcySGQFkgqwJWCTsKCpML+bH5j0Lyl46eDoZW6gJekXUzMistNeKegz2xGKeWGt42eRumXZsm627RzELYKQbqjsTWE1tj7ir2qjrxrtoHW+QlpaOvw/zo+Qz9fqjw2AvfXeAL4y+4XNKWuTFvaZABVSrwuHkAt2RnzvQ9Q4yXdhKLCDlROdzeflvSoVYDv+l+dHquk6Truj7iTsYJfSXLcssoSCxg/YD1Qm+aE/WswfW8PDSDerpku/oT6z+e9se+49abaxtxNXXY9fQuAmWBQs91lVJFv0/64TXCS+gFZOpoiltPN57880nhez+x+QleDH+RsWvHSvZqhcmFHHlNXTw/9PIhCYGg0LIZwaMWoVdejNelDf/quq9mPhKRGrLCs9g4bCPVldXCTuKYwzGogk7T6VonZq6fKRnr0lv2JNZ/HLkufpjkiv0fHwayI7O5tuIaLt1chHLbf4fwHeGErg3VqkoAoGeiV9sB+/8L7lfG076tPYO+GcS0w+I126CvBrFItYhe7/YSxld2WUmgLFAjF1EDMxf173d91XXhsfX377HHNOfqe9fENb6Z/wX4TffDe7Q3/T/p3+DrBn41kL6L+mr8rdPcTryZ/iY3J9wk0yETpappP1zmrUzJequxMHUwpcWgFpg4aLeseT35dV5LfA1Di8YTZ+rjUex7/8d4yH+L+o9OY8aFphT78hPyWT9oPdd/Fz/Tk7ZNYuBXA2n7lHifY+pgSsjKEIJ/C5bECpPqyO6i/aVKR5cSazfyHH2FOU33vu4oChWYvmbK+4PFHqQPAxbNLLBva8+R148I11U1eVBt0qKha0IBhD6ENX59ADe33HxAZ/wYDwOPOuf3ONP4GI8U+vpqBml+fD6gliN60NjzzB42DtnI0TePCuM1hRpAaEwPaka3SE/575B+I52Uy+L3tGtjx6txr9LlRbFHkXNnZ7q+1FUobwSwacQmTiw8IYwBDP1hKOPWjWORapHW99jx5A52Td/1N1fRNMQcieHiDxe1MsRqmO1P7pJurpUKJdYtrYWJZAD/Wf4ADZ7zjbXqJLWtr3Zt8AcNpw5OPH3saWEyuDHQM9JDpVSRF5untV0/KyLrviUOHhZWdVvF6oDVQrmq/59h28qW9jPaY+ooNlVuDGrGvsd4uMh1Vvt9aJNZtMiMQa6spsJMnNjVhpqkfUNSbveLvov6YmBuwMrOK7m05JIkvmPKDgCym3XgXhn4qH1RLGm2BKuMOzQP3Y3isrjIqQ39P+uPiYOJ1oRwQ+jzQR+G/zSczJva5a7z4/O5e+yuZONTWVCKbmUpRdbNKDe1Ax1NpqNzZ2dmnJjBpB2TtM5n94szn55hsctirfKPV3+5Sk50DrY+0jmlJLOES4svaV075MbkErk3kr6L+jJx60RJ/PBrh2l7+hfMchOpNvj3GNPmbuZ0er4ThpbixE3s8Vg1i1aQSKsqqyLhTF0xTaQOYOFmwbTD04g7ESeRq3qYY1+3V7sx8MuBtBjYQhKTyWR4j/Rm+E/DtfrjlmSWkBOZg21r6W99+LXDBH2lZthqI4ZpQ43Xnqi7CKhdy5m7iAtIyiolrca3EnoJgjpR8ty15+j5trRra22/tfzQ8Qf6p/Xnd5PfNWIGpvoUW7shr64kpstUynrdXxeeNlxbfo3Q30OFsRp/oJTLKUKJ29vbb9fuDSTHGuoyYeMECm2bk9R6KPvOS4sWtu+58/HHHxPZKhKFIOGsDeX55fzg9gOHXjkkjCvKFdzYcENrYnXA5wMYtXxUg+TApItJBC8PlnQZ2be1J9ulHbeTLPhjj7FEtjXFpz/Kv/5mWibu6tKGmEMx7Ht2n9bkuJmTGZd/vCzsatU10K2VYxXd+/0+7le7/o0+JC1oRy7eT4FdS8J7zcWop6Z0pNzYiMS2w6k0MqfM3AEzP48mXVfN+YoIhkWpRZz+6DTWXtbCJHW3V7ox+/zsBucUO187/Gf6Y+qgud4ryblnDivS/C3few/kVRU4xl3CIjsWpfLBjX1p19MIlAWyYcgGYbzHmz1YpFrE6OWjJbGKwopa71GRWkjErgicTmUQ1zyOo0OvaIxJKpWKKgMTLnZag230cuyj1jyQ66lBZUmlVh/ZotQicqNzaT+jvdC79O8wdf9U+nzUR2uRHuDNjDeFe9P/y7jf7hxDS0N6vtVT6APbGKQGq0lAolyAhZsFbj3ctBZWR68cTce5HbH0s2TW+VmS+MuRL/Nawmu8lf0WEzY9WIncf4JOczsxde9UrTYiI5eNxMzZjOb9m2NiJy2s7X5uN30/7kv70PaU3G3avvrQ/ENsHrX5vs476UISYRvCtKpbgXptos1zuLF41Pveh+lf+X8BD7OzUVmlJO1aGkWp0s66GvR6pxfDfxwujOVE5zB65WjGrRsniTl3cWZ+zHzeyX9HaDFhUJqPcX4Kqd59mHV6liQe8HoAFwMuYlpiSmmBdruchwELdwucOjkJFbBq1MBcu7kKj30x/EVejnoZn9E+kphMLuP98veZtH0SqmoVieebRux9jH8Xj3Lse1zse4xHBqVSyc2bN1EqlYz4eQRj1oyh9aTWwtdm9H+SyICZlNxHTcGtp3piMLQwFHZJzLk4h9kXZvP8jee1ytsUZxQTfyZeq+eONoxaPorpR6cLY7oGuhx5/QhnPjkjjFeVVlGSVaK1aBZzKIbzX54XnlNebJ5a8rKBTrG8uDxyonLuy/ugIQR9HcTRBUe1JkpqIEqE+I73ZX70fJr3FydJDCzq5Jy+NP9S+Jo2T7bBf5Y/9m3+PQkMI2sjqkqr7othWFFYQaAskG+sv1HL02iRq72z6w4hK5uWuH/YcO7sjJ6JntZOnv9fUVlcyfdO3wvlZRqD+mPfYzw8WKWGo1tRQqZ7F8asHiN8jWWGOnGZ69m1Se89fr1arnjQN03zxGsMzF3MadarGYAwQTn8p+FEd5mKflkhFhl1ide82Dx2TlXL+LnePkJyq4EY9G2ap6CDnwNvpr8plARJuphEoCyQdf3XaS3w75q+i21PbBPG7h67y/F3jqs9DO5o+t+VJuZinhOPWW4itokhVJZrPhsqpQpDS0OOLjjK7701Cxb/FM16NcPez16rTPLNzTe5/YfY286+jT0DvhhAXlyecH0RvjOcP8b+gaGVobDTJ+VyCvrlRbiFH226ocY/gJmTGYWJhRx+TTyG1SRcc+/mSmKKcgVR+6LIataRV3IXCWUWZXIZuoa6FKcXa8hjNmbsK80uva9iM6i7W3u920soHVrTgd7myTbCwm1hciG/9/md0uxS5LrS7VH9IvbtPU3rqjJ3Mafnuz1pPVm8xvWd4Evria1pNb6VMH75x8uoqlXCJGLUgSgCZYF8bfutMJlj3dIapbP6+9a9R65YpQLHmCC8r2zGJjkUpf2D9VfUMdDRSoLy6OfB60mvMzd4rrDImRWu7sDp/IKY9dxyuBdF1u4YF6She88S5MqyECJnReIR58Hwg8MpDGl8J6ZSoaQwuZArP4nl4ULXhbJ7xu5alYp74feUn1Z5/Oqqaq6tuMbBlw5y4PkDlOVp3uc+o32I7zAelVyHvMgMjfW/UglpXn1I8HbluZXPMe8XaSGnITi0Vz8T9Qv19WHRzIIeb/fAqYOYbNesdzON97kXbaa0Ufv6CZRETHOT0a0sodTSWdJtIZOBdcpNWl1cR5G1O85P9WvkFanx0p2XeDX+VdpNbyeJ6Zmoz+Xyj5eFXW6g9mDU5o1+4fsLXFtxTRi7nWnHteHvc234Qp65+77EWsAq9TYdj3wFQIZHF6qrH9y6r0Za8X68yV0DXBn8rbqor6iQzjm9F/YmfmZn4j3iudEhRqMQWl2lxP/Y90xbbY1BhQG3O4jlx+4XW0ZtYUmzJUKpTffe7rxb8O59yUYCeI/0pn9g/wa7Ai8vvaxV+vT/Kv5Jgr+qtErih1qDpItJfKL7CbueFpNvn7/xPNMOT0NHT3yPzA6azbAlw4QxI2sjYo1i+cXvF969+K5GTCaTYeNtg0UzC4xtjBv0Ef23oahQcH3NdY4sOCIsWudE5VCUWsS5z89Jxv7kS8ncWHkD+yx7xu8eT/6F/CZ9dpeXuzDg8/tT/+k6vyujV41usNh3/N3jfGn+JTnRYinyv8Oj2Pf+r8t23oumyng25aeqrqzGzNlMQoqpQWpwaoM5slVdV3Hx+4vCtXmNEpm2rlL76HO0Pr+KSkNz4T5rbZ+1dL/UnWKTYrZO2drIK/rnKE4vpjS7lN7v9xbuk2pIK8feOiY83s7XTqvCBKjzyE6dnAj+LZg7ex6sBP9jPDg86pzf42LfY/wnYNHMgg7PdFDLSgqQZNKKIlsPogUKSOUF5eyYsoPra8St411e6MLIZSMJ3xEuZN/f2X2HNT3WUJRWJJTEBPje8XvW9VtXy4ZsLEwdTbVek6JCQdT+KK0+b0vcl/Cd/XdaN5o1hcnyPIH8UUohl5dcZm3ftawKWCXc5Oro6+DR34Meb/do7OU0CgO/HIiZs5lW/5nhPw/HqoUV4TvChfFP9T4lUBZI0LdSjWozJzOeD3sekLJoa+A/y5+xv48VJhweJi58d0HrhN0g6iU+PPp5aNXDjz8V/692KzYGc6/OZWHxwvuW9fi/iutrrlOcXiz0wnyMB4vYE7Fc+vGSUAro72BUlIF5Tjw5ru3oMFucIErx6U94r7mo5E1LWhlZG+HW3U3Izv2nqCiqqN1013RT1Ee2ox8Fdi1xvXMcm5Q6GdH6PqV6VWXoVJWj6yZO3GrDntl7OPOpmIBS4+UUfzpe6D12e5taIk7bnLJtwjYSziUwasUorJprJkZNPayJ9xsJgMfN/WTf8/7RB6NZ0WkFqcGp/6ijVgTf8b5k3c4SdlECjF4xmqzwLKHEoo6+DhmhGWo5KsHm2XukN+PWjZNcbw2m7JlCUushANhGiz//YaH9zPb0C+wnjHV4pgOLVIuEctkG5gbc7vM86Z49OfVnriRpBer1jaGlIfOuz6PdNGkSviGs7rGatf3WNumYGlz5+YrWYnRVaRVLPZdy/O3jwmOTLyXX3tei4tPknZMx7NmBVK8+HNJtWqFF11CXHm/0EHYcgtoXctL2SeibiNmf49aNI+B1sbpETYdceU4pn8gDJfExq8ZQ+lYps36fhcnnmuNVcWohNqnqMcQ0LwWZ6Cb+B3gp4iVGrRilNW7uao5zJ2chYWj4j8N5O/dt4fqisqSSD1v+gUvUaTxu7keeqtlld2pPIboRRcxaN4tuV7pRGq2dbX4vjG2NGfnbSLq/0V0Yr+lu6PWeWL6uNLuU9QPXc/mny5JYdUU1++ftR9dQl6ePPa11/nC4ewHdFcsoTtdcu+tWlmJSqCb4pTk3Lenq1NGJgV8OxO8p6ZwC6jVsUlASbj2kzPnqymriT8Xj2t2VZj2bSeI1XTwLUhbgOVjarRPeVz1etD/yDWWnNb+X6sJiPG6qffwK7Tyb3H1h62Or1Tag/p6uNEtKjIw6EMXvvX7n0HwxOfHyj5fZP28/S9yXkHRRc/+XkyvDoCwf26RQ3nyhRMLc161Sj4tlprYktR3xQBPPnV/ozNjfx/LE5ieE8e2Tt/Ot3bdarTHaz2yPa4Ar9m2lpEhLd0vKHU1R6iiRqTTTRDKZDOVfayWjciOu+T9YZRi3Xm7YtrIVEmfkunLiT8fzx5g/tNpyNISsiCyt3fs1OPPxGS5+J5ZS/i8i924unxt/zskPTmp9zf3ed/Fn4vnC5Au+NBUTa0NWhaCqVmn1T3Ro50DLoS3FxJlbmazrv47wneI8QElWCflL8xm7ZywV1zXlz1UqFWc+OcOqbqu4+tvV+y4+PQyc/+o8e+fs5dIPl8iNlhKl8uPyAXV39PF3NdchMYfrCESh7UOx6tc06emSzJIG/SgbQtT+KPY9u0/rnrYgqYCgr4O05l3+q7j33v9fL/41tdjXtPdW1f4nwon3TrB+4Hr2zROr7HR/ozs+Y3yE97CiXEH0oWhOLDxBTpT0eS9wbk1yq0EodfRQVisl55AZkUNpMwVLX1nKWJ+x93F194eSzBLOf3G+1jf4Xrh0cWlQESz7TjZbxmzh1lapXUhpTimBskCWtljKrLOz6PKCWCXuMR7jcbHvMf4TiDkSw6mPTgmTRgDOd07S6vxq4SZQUa7g9tbbpF6T+sbUQN9EH31TfaEvSM1metOwTRrJ0vqQ66kfFdHGqCFsHbeVTSM2CWMZYRkoFUq8Rog9+Wp0rW18xKyOXu/2ovWk1qwfuF4Sc+nqUitVknI5RYNZXwNzF3Nmnpyp1fvufuHazZUFKQu0Jte7vtSVgqQCIvdIpYCyI7Nrk9zJF6QSSUffPMqydssAdVu/CGnX01jquVSrR9fDQPTBaBLPJeI9uukyngZmBsw6MwvXAFfiT8eTfFEsDeX7hO//d14S/1dhZGOES1cX+n3c71Gfyv/3uLnxJkdeO0JVaVWTj0317kuZmR1eVzZr9ZWsNLLEOvUmFvFSj7qGkB+fT+S+SEqzH7wkyOFXD3P36F06zu0okXEDWLwYEuyXs2rORu50qpPVce7szFvZbzHz9Eyy3DqQ69ymyZ4Vob+Hcvqj02TeknY6OPg58HLUy0zcNhHbVlLiQcTOCAoSC7TOKSN+HcHY38fSaW4nDS8qAAPLuu6BAruW6JhqJvhV1XW/X8fnNGXgHgRG/DJCazeOmYsZzXo1E0ooKsoVBLwewLzr84ReEfZt7Wk/oz27Z+7ma6uvJfGt47fiFn6UbFd/yizEHTMPAxlhGQR9E6TVOL4hyHXkGBZl4XfqJ24++5OQkFSQWMBy/+VcXiotePwd2k1vd9++jHd23yH+dLxQ8lquJ6dZr2aErg0Vko1curkwcetE9Ez0uPKztKvL0NKQPBM3DEpyUeg1raMm9ngs39p9q9VboySrhD3P7BFu6kHdTX78nePkxUk71FoMasHtPvMAUOpKSU5HYo6w+NJiFLoKwvM1r1ulgjIzO5J8B6NbUYzpznVNuq6/Q3FasVZCW3ZkNmc/P0vK1RQhmUPPWI8b624IE8LVldXoxdatIeVVmmv7K8b9av99cPhBbEdK2d4NofPznRny3RBhrMbbUZsE/86pO4k7GUdRirTAqGuoy/Sj0xm9YjQtBrWQFDkj90biEbqHMjN7VP36aahZqFTQ/ui32KfmsPj1xWx+anWTrsnIyohe7/bCo5+HMF5VWqU1mauoUO+zki8mC73vbqy/wa7pu1jmv0zrXO0ZsgO9qjKqYjWLZqoy9f6k3MQGy/QI0v680ISrgszbmeydu1coH1ofIn+6GjluEakGYMaJGQz6ZhAmDibo6GsSguSKSppf34X7rYO0ObpEct1Z7p0JHrWIBL9RWKXeeqDJVY++HvjP8heuD0DdFVuaXcrFxdLCVWlOKYnnE0m+lEzqFem+WVGuoKwig07BnZi1ahDpN+sVc+Ry4tuPJck1iV9f+JUCkwe7zxrw6QBeingJAzNpF0RFYQXRh6K5e1QqA94Y/Nr6V35w+0HreATqcUXb8/FfhL6pPs37N5d0ldbH/TYT1HSNarPV6Da/G1P2TGFeyDxhPO5kHPFn4oWxypJK4k/HE/yr1J8L6mT3TUtMsV6t+dwqFUpOLzpNypUUDr54kKMLxBYtjwJZt9Td6IaWhlg2t5TE7+xSd9/omehJiD8d5nRg+OHhfL7wc45MOsKoPtpJMiJcX3Wdq7/cn+euU0cner7TE3M3scTthe/UY7KesV6DnUb/dfyvF/vqozHfRVO+LxsvGwZ8PkDreBGwQE1Wu7VZvMbt80EfLv1wSUj0S7qQxOYRmzn/5flaxYf6KHH0JL1lTzxu7OVT3U8lHap33fpinKjLgJMDcAx9sOoVDcHGx4Yh3w+hKLWIiiKxZ/vUfVN5Pel1YewX31+I2hdF+nUpoaL+euPcZ+ewatE0csBj/O/gcbHvMf4TOPbuSc5+epYQLWbNzjHnMM1PRlktnXlqNmDa5C6Dlwez6+ld2LayFWrPX/y+bjNUs6C5Fx9Wfsgi1aImD6aZtzLJicwRJjJMHU3p/mZ37R1wPw5nkWqRkCUL6o6X8O3hFCRKpSJ0DXRrz3XmqZlaz3v75O1sHf/vtbTXYH70fEavkjLj63dONh8klfJ06fbXdyVTs+xFCF0bSl6sWFLtYcHE3oRW41ppZUyrVCohG6kG7n3cGb1qNB2e7SDcIAAMXzqc1hPFEmCPCoGyQAJlgVqL5P8/Iu5kHOmh6UzdP7XJxf/HaDpqPJvuSypWJseoKAvdqjI+0flE+BK5UoFj7CXM0qKa9NYHXz7IH2P+4OgbDz7RUJNsClkZIuxGb31mGd7BZ0h2iyG8RZ1njlxXTvr1dC5+d5HsZh1pdWEtJZt2N+mzazqntTGubbxsaDOpjdDjaMjiIbwU8ZKw6AXQ/un2Wv1fZDIwyVdfa2zHJzCw0SxCufdxx/cJXybvnEz318UdN/eLK79c4e6Ru7XSqfdCpVQx9vextJ/RXhLLvJXJ6u6riTkSI/RkqIFLNxdaDpNeu7mrOUXWzUhoN4pCR6kvw8NCdVU1ZTllWmW8j719jEBZIDfWS4vgKqUK/fI6BrjIb8LI2oiu87tSml3a4NwnQkFigVb/5urKajYN38SF78Vrtb4f9cXc1VwomaNroMvArwZq7RyxcLPAc4gnzp2dhXKad4/dxfDMEWxSb6FX0fhOMaD2eYneLy5IXFt+jdC1oVp9NyqKKihKKxLKbKWkQJm5I8GjFhH/1EJJfMm7S+h6uSsbn97In0/8qXlejubc7vsiGZ49yHf0QeHq0aTr+jvsnrWbLaO2CGMZNzI49cEpVnVdRXZEtiRekFjAkdePELJCWlAwtDQiustUAOLaj0WvleazJZfD0SdasWH6BlLcTengIpboF6GiqIJrK69p9eT7O+ga6uLe151BX0klnuW6cjwHe2pdO6SGpGGbHEqppTO6A/tqdDWqVFBppCZ32OQ0PeF6+afLfGb4maRDrQY93+mJmbMZEX9KC+V6xnq1kpgn35d2EXV8Vk3AyI3OZf+8/ZK4aU48ABVGlljO0exG03e0JmT4QoKGdcYy9QL5l5omR/Vb29+4vuq6kD1fklVC26ltmbB5gnAN0ef9Prxw6wX8Z/oL39vGy4aeb/Vk7pW5OHfSJBgqMnMxKagrlokK8bLqKrwvb8AzZOcDTzRfX3OdayvFyi8v3nqRJ3c/Sa93pd2nWbez2DZBLbXtM1Y65xx46QAdlkSgV6VHtU41SlXdmKNUQq6LH6ufXc3QI0OZtnHaA7oaNRLOJnD3qNgHOflyMteWXWPU8lG4925a8R7U3klAg1KFg78bTNf5TZN0f5QoL1IQcl3GDz/pcfCg+DX3e985+DmwSLWI5649J4w7+jviM8ZH2LkHsH7getb1WyfcH7p2c1X7BWtZMvk+UedHmjNZc/0gk8to93Q7XANcsfSwpP1M6brsUWHcunF0e7Ub5fnlQtJDDalg2JJhtJncRiNm4WZBZUklrcNbI5c1PT1r6WEp7F5uDG5vvU3Q10Fa91mKMvU68bkQ8b3wX8Xj4p4m6hf+H3Sxr6qsim0TthH8m7iA7zXci/fL3+et7Le0vkfrya2Fc5K5mzn+s/wZ/vPwBskYJVautJncRrIXK7Z2I8fZk4DLAbQ9eH9EwvuBroEu+Qn5XFt+TUikSgxKZM8ze7Ta93R/ozv9P+3PgM+k8rwWbhYMW6qWQc4Iy9BKaH6Mx/jfMlp6jP8U5HI5fn5+yOVyQkz74KTzJ5mVlsLX5tt7UW5qi6hNQUdfB58xPji2F7M1Ek6rZUxSg1PJicrBxltzk1xTdEq+lPzAGX2Dvh5EwtkEdVfCPSpxFm4WNO/f/L6LUtsnqv1C/Gf5U11ZrcE6rSqtYszqMRjbGWuVWCzOKCZ8e7jWhfr9YsuYLUTti2LU8lHCTok1Pdfg2MGRET+PkMQ8h3oy/eh0nDo4CZPJtclSlVp2QtS2XpGvZs80VTrsn8C5s3ODpu6Xllzi6IKjTNo+SVKwqyqtIujbIDLDMqksrtQq93bolUNE7IzgI+VHDSaVHwUqCisknTr/v+LoG0dJD00n504O/rP98R3v+/cH3YP6Y9+/harSKiqKKjCyMpIw1P/LsPezp6KwQqvvB6hZ7HrGehKPUqPKfDKad8Mh7jLmLcQSuO439gKQ6i8djxpCu+ntiD4QjeewB99t235Ge85+dpbc6FziT8VLZNMU+kboyFzRq9TDsqiOYFBZUsnJj86QcjER3S6dKLZ2w9S+aUlhS3dLJu2YhJ2vnSSWF5vHtZXXaDW2Fa4B0gKPuYs5+57bR35cPk8fe1r4/td/v875L84zcetEDQZoblgy1qnhpLXshUomk9jXWbWwYvKOyRx98yhXf73KjOMzmnRdDSHzZiZR+6O0JgE3Dd+EgZmBunvvHpg6mtLh2Q6Y2JugUqoknRaXf7rMhW8uMO3wNKGPbObtTMxys+h48DNujFv0YC6oEXDu5Ezbp9oSsjKEjnOknZI1ElQHXjggKXKW5pTiFnGMchMbfJ/vK1w3GdsY4zvBl3X91+HQzqF23dWYsa8sp0xr0kqpUJJwLgEzV3FHonsfd60sWYBmPZupC9JakBqcSsKZBBTG5ty7vT4TeAZZhXp90eL6n0DjE192re0wMDfAxEE8T9YURPt/0l8SU6lUXFp8Ce/R3kJW/d7VWXgGn6TKwJTS7gMBzTVfpyudyC/L50o3abeiSgWu4UdxjL1IZLenMeghlhm9H6hUKqGUWQ1aDGpB7w96k3kzU6ODrQY13lkiX1SVCopsPIgMmEGlkYXEs8+oKBPL8hJCu6bhmO31l2pI47oxy3LL2P+cumC1SCV9JpMvJbO6+2rc+7gz68wsSXzqvql/+xknFp4g6JsgXo58WUPqv+uC3nxx3YG7Tl9QZeQEaPpV3ek5h7KSeQw4OYDMCANoggq/nrEe1RXVXF9zHbfuUqnOyuJKEs4l0HK4lJQg15Ez8reR7J6xu7boVx8O7RwIWBDApcWXkOlI16ceN/ZSYu5IRJ95eNzz6Mt1ZOiX5uEXvIHzPW/QZuSUxl8Uar8pHX0derwl/TIqiyu5teUWxenFtBrbSiLvb+Zs1mB3c0lWCXIdudAbz9jBjCTfwbX70jK5pry0YXE27mH7kCurKTe2RiZ7cOu+/S/s59qya9j42GjtpG81Vuz/aeNjQ/OBzYk7EUdGWIaki7pZr2Ycv5JAcOdgLnW/xOLWi2tjVeUKPIP/ZFLsJBwyHEhwF8uE3i/W9l0LwMLShRIvNuuW1gz4YoBWguzf4bX41/72NT5jfB64j/3DxJXzlZhk3MXAwpVNm2CEYBn7MG2CYk/EoqxSColMLYe1JOZITK0y0r14+tjTtdLw96LjnI68kP4C6XfTSTRMZAUramNyHTnj149/MBfwgKFnrEe3V7rRYnALoRS497sTOMkAMs2kY6RSoeTSgkuMTxgPuyHYOpjOz4v9akUwdTLFzMUMlUrV5BxB26ltcWjvoHWflRujnscjdkbQcW7H+7IveBT73nvxiCyz/jNoarqxKa+vLK7Eradbg+NzjeS8CFvGbMGutR19P+wridl42TD2d+3ym/YRZzBLjSKy+0wmbtX0qVepVBgXZmCTqiaReH/adAWu+0V1ZTUdnulA+xnthYoIuTG5hK4NRaYrY/SK0ZLnVpu6RA3aTFITBg6/cpgjC45o9UB9jEeLRz32Pe7se4xHispKNdMh3bIV14cvJMFE3LkU0/UpklsPEU48cl057n3dOTT/kFAyaszqMbUeQoUpUj1yPWM9LnxzAZ8xPlolNY++eZRAWSBRB5rW/dHz7Z48tf8prcn1fc/t4+RCsdb+0bfUn6nNdLVmwx26NlTiQ5B8OZmfvH7i4vcXSQtJo6pMyjCrYdv1+bBPo6+nMajpvos/HS+MVxRVkBWeRVGalBlv7mJOwtkEvrX7VijJY2hhSN9F6oXAwRfFNMbRK0ezsGQhukb/Lpfhj3F/8Lnx58KYbStbnDs7Cyf7qrIqznx8hog/I7h79K7WDtXki8lqM9//EHnnw6oP+bDqwwYlZP6vIT8+nwvfXSDjptgTpMYvLGp/1H1LpkDd2PdvYdOITXzv+D1xp7QbZP8XMezHYTx75dkGSRG/tvmVH5v/KPm7XUIwDnGXudnvZa73FCf4i2zcyXLzp0q/aZvXtlPaski1CL+p4m7efwqvkV61n3MvorrP5FKvYt7/4n26nq67jzJuZJByUd3d4HV1C0U27pgM692kz80Iy8DG2wa71tJiX35CPkFfBbG6+2oSzkmTfOX55eRE5WiV4/6l9S8ceOEAeiZ6kqKYjqEeVYamOMWcp+PhryhIzNeIh64L5Sdv9ZwWd+LB3sOjlo3Cvq09x94Re676PeVHcXqx0Fze3NUcZaWSvbP3CpOEBmYGmLmYaV0D9P+kP+XGVsgA8xRxN+XDQml2KUUpRcJn64kt6u4bETtd11CXzGadSPXui3U/7fe/XRs7Ju+cLCke/N3YZ+pkintfceeGnrEeU/dOpdv8bsJ42vU0wneEC3+LyuJKNg7byKUfxd6IEbsi2DB4AwDnU6VFr74f1SUh8pya1mWvb6rPuwXvMvLXkcL4iJ9H8GbGm+puBwEsmllo9fOtSkrHKv0O9gnB2J2XemhlvlvAlqlb8L/uj1+Y5u9VlleGbZJaUcP95v4my/42CJWamTxu3Thh2MjaiAGfDmDK7im1Pnj14fuEL/0/6y/sgqsqr8Y0NxGfS+vxO/UT8mLNNbBlfChdzyQxb81iZvweQNzusEaftom9ibAztAYiieP6UFYrifgzQtiVXZ5fzqd6n3L5x8v4jPaRFJ909HUoNknDOiUIh5+SNfYsKhVUGZrhkuKAW7Ib7UOaJvvrM8YHc1dzzJzExa2SzBJGLRtFt1ekz5ZKqaI8rxwjayOtRaSh3w9lkWoR49aOk8RSWg0kzbsvprmJKNI1ZbiU5ZX4nl+FfZY91TrVJOhIOwsbwvClwxny3RCh53oNASj+VDy5d6WF54qiCpa2XMq+58Q+QsvaL+MHtx8I+iaI/IR8jZiRrQkZnj3IaNGdjBbd0bPQXN+bZ8Zglquej+M6jFd3Zj6gdZ+hhSG6RrpM3iFWOInaH6VVQtHUwZSnjz7N+I3j6fyCtJjQ4ZkOZA4OQKGnHkOrlXV7EkWFEqv0CNqEt8G0xJTtk7f/84uph5q9hKgLwqq5Fe2mtSM/IV/oQ/8gsHf23to54P8C5HY2ZHp0pdLIkkItdoT3292UHZlNoCyQn7x/Esb3zdvHhkEb2DtnrzA+7dA0FikXCZ/LkqwS8uPztRYCAe7o3CHRXdzpXpRaRPYdaTf4o0aNjciWUVuI3Ce1KnnvPbDYsZrLU5ZIPF1PB56mJKGuCzJFTzp/NIS+H/Vl9vnZ90UGzo/L5+ynZ0m7Lpa4jT8VD6i7ukXy1I3Fv73vvRf/651+TfXsa8r3VZBYQFJQkla5ym0Tt/G90/daVTvSQ9MbJIdVV1VTXlAuzJHpVFeiW6lFYUoFHmH7qNZRsmvcLlKaNe25+ifIuJnB8g7LuXv0rrCQ3m5aO3zG+HB91XWUVdJK9LYtCr4ZF8S3dt9KVNyqSqu4ufkminIFbZ5sg2OHf0+e9DGajkc59j0u9j3GI4NSqSQyMhKlUom8qgLdimJUWmg3JnlJWCeHoRAM8qVZpbVSaqKklZ6xHp5D1N0XOZFSOSlFmYKUKymc/fSs1nOtkfpUCWREG8KOJ3dw4KUDwlhWRBZFKUVaB+jI3eqForb27voSQfcmJiyaWaBrqMvZT8+yotMK8mKl8jI2Xja8FPGScLP3T9B7YW/eyX+HMavHCOMvhL1AypUU9jyzRxg//8V5QK0/fy9ubLhB2jXtfgsA5QXlXP3t6n1LMN0P0m+kE7knUitD2Gu4F3OvzsWjr4ckZmhhSMe56s4KR39HrRuYotQinLs4a/XoeBSQ68qR68r/c52G/wTZkdkce+sYKZfFC8Kxa8fS7ml116goKdYY1B/7/i2U5agLMPomYibtfxVnAs/wqe6nZN2W6vTXoEbq4l6UuniT2HooCgMT8qRDIADZ7p1J8+qDbvmDTR6FrA5hdY/VQrPxv8OVX65weYk6ESDqcAYoM4gntH0oqW51GwCLZhYYutV1MN52/4wSVdMkFDcO28jxt48LY67dXBny/RAc/R2F99G+uftIOJPAc8HijidDS0M6zevE86HP4+ivOe9ZeDtwt1Nd4rI0V9OTJ+9uXu1GcOBXA5t0TY2BXFeOXEe8JG43vZ3WYh9AyxEt6fF2D+HY7D/Ln2cvPUvC2QQOv35YEk8PTUepq56/jfLFc/3DQHF6Ma7dXJl+dLpw/FZWK2k7pa3wuzYwMyCzRQCGxdmEz/+Nm5s1feiqSqtY6rmU7+y/ozy/XKNzqTFjX8zhmAYLuusHref0x6eFsXOfnWP7pO2kXJWO30qFkrtH7nL0jaOEn5KSOQzMDTB2tgTANUJa+G0+qAVFM17kTvdZpLdoQksVagLW9snbSQwSJy/1jPUoSCwgPy5fEpPJZIxaPgobbxvKcqWFdMO2nqQ3V/uhmCRLCWlnEgvJtcmlz9k+9L2kyZouyypBt6qcMlM7kMnRD26aX1pDkMllDPluiFD+FtTJm8riSq0dtT6jfej+endhvCyvHO8rdX7YlsWa673i5uqipkVOOhkOeWQ4N34c1DPS492Cd4VdfQCtJ7bmxdsvMnW/uIMvfHs4257YxuUfpeRDuZ4cr5FeDPpmEE/uelJSeCtILsQ0LxuLAgvMso2oqqgrtqtU4HbrIHpVcvaO3supMU2TMjSxM+H1pNeF3aOg3uccflU6RoG6SHn41cMoyhW49ZB2BdYQIq/8cgVFhbTQnufchpbBW2l14XeK9pzSiFVl5KBTrb7OeI94DLObRtRTKpTkx+cLn436Y5upg6kkfuytY+TdzdMqRe//jD9mzmYcf+d4bXdLDeRycLh7gc77A2kWtl/SXVrg4E2+vTd3O06kxMqV6urGr/tUShWKcoXQAgLU+7/3S9/XKge7d85e1vVbx7YntgnjMrmM3TN2a/29bRTtcExzpNulbhSk1q0xdI31udN9FgBxHnHoKaQ+iP8ETx14ihknZmglPdw9dpet47YKPYwagrJayffO37PEfYmEHFsfiecTqSgQJ6v/i9CRq3CMvYhFpna/yvstcBQkqH93bQn4Gg/M4T8Nb/J7Z93OYvuk7bUedvfi5IcnGblpJDPXzuTp9ZoqEdWV1Sx2Wcwvvr8QKAvk8Gvie/hRoL7EsUUzKYGl/dFv0a9Q7zUywzRJI47tHXEc4Mi2Sdv45q1voPHK0wCs6bWGdQPuz3PXxMEEly4uEvJJDZ698mztv0Xep43Bo9j3/q8X9+5Fdb0U6oMu9tm1tmPGyRm13Wb3Qt9En+L0Ytb0XCPM1b6e+DqWzS058f4JSSztehqf6X/G15ZfE7FLSgZK9xvMzYGvYZUWzv7n92uuQWSQ1awjKpRk2WWx9fa/Z11kYm+C/zP+mLuYC9excl053d/szvgN44X7x0uvbqVsz3FKs0uJPRGrEStMKeToG0c5/vZxJv4xUasU+WM8ejyKsa8+Hhf7HuM/geahu/A/9j2KO7HCuG/QGlqE7qJawNauLykjKpTkxeVxc/NNnDo60XyA1Afu2NvqpE7ssVhCVonNxiftmMT0o9PxGdM0T53b224T/GuwMOFbUViBjr6OVvnD5288zxvpb9B7obgzo+vLdb4C9yaErT2ta2VnrDythJ+ha6hLdmR2LWPrQUEmk2FoYSiRYKmP7gu603qSlBkfsjqkVnfaob2UtZxyJYWo/VHoGuoyZo24mJgUlMSxN49p9X14GKjZDA34XKqrDWr2fFGq2G9Hritn9IrRzL4wm/TQdLIixEWN1xJfY+Rv4q6ARwFltZJzX57j/Nfn76ug8V+FsY0xJvYmVFeJOyz1TfXp8lIXnjn3DC0GPTjJs4eNWWdm8XLUy0L5xRqc/OAkK7uu/BfP6u9Rk2DTN9NepBzy3RD6fCDtUK5yakZmiwDMsuNwjDmn9fg2Z37DNUTMTtaGY++oPc0uLr4ojJfnlZMdkX1f7ONzR9TnWrW8SiiZZh97CbuUJHaP382J3mtr/27uak6nxdOI6jqNr975CuOSam6fW9qkz+78QmdiDsdw9jMp+UXPWI/uC7oz7/o8oQm75zBPurwslVauwZwLcxj+ozgxpFKBbaJ6/r0x8HUsvDWLgf6z/AF1l3TrJx6sd2nm7UwGfDGA4UvF52bhbsHzYc/T7VVpcT/7Tjahv4fi0tWlQV/J6P3RXPlJKqOYGpyKXlkB14YvJNVbKl/zsJAdmc3eOXu1FtXCd4Rz649bnHj3hHBjbpYdh3P0WcriMyWM08SgxFqCkbYuz4YwZvUY4XcN6kSf1wgvrf6KLgFqZYHUq6mSmKGlIfkTZqGqVvH9/HhJvMXAFgw7OJ8c57YU2rbQSIwArFgho/zURVpdXItOddPmvIKEAsK3h3Nntzi5mROVw8ouK4XPHUDk3kj2zNpDQZLAo9nMmHSv3iS3GkjKkJmSeJJqO0alRuyYuIPYpzSL6MZOFtzpPpPorlORKyrQjWmaX9rfYd+8fXzv/L0wFr49nC/NvuRrq6+FXVcAS9yXsGn4JsnfdYz1yXVWJ5WKrN2xbK0pHVVl60Ri66EUWJSwZ8x2FN4PbrtraGmolmU1E3f/1fyGorFQ30SfKbun0PUlsS9Y6Ipgeh2OIKxdGN+99R3m7nVScCoVOMRfxTHNnELzQhKdtc9pIsSdiuPIG0eEPt+gJuq1n9menGhpYVTHQM1Kryqt4tISaWdsjR/aoZcPSYr/gEYhzKSXv0ZMz0oth7lqziqe2vwULddrl9UUYW3ftfzY/EfOfSH9PqrKqph5aibPhz0v3Ae593Wn84udmbBxgvC9B34+kNlBs3n2yrO4dNG8x8oSs3D7ixRgn3iNnHvm+goTa2K6TsUiMxr/I183KXGadDGJz40+F84Z9a+tskQ8Do1aPgo9Yz2svaWJ+aQLSfzQ7AfMnM3wnSCVob+28hrmR/fjFe3F8MPDKUzQ7C4ttXDi2BgrIn0i6XlR/L3dL2x9bGk+oLmwEz7pYhLH3zlOm8ltsPdrol+2CorTiilILCArXDtxzMLd4j/lAfe3UChI8BtJxl9kj3txZ/cdyjb9iV6Z+JlvCO593Hnl7itaPbZ6vtWTRapFwnsI6jzdS7Olktw1yhFB3wQJj80IzcAlzpVS41JKjTWPv3cfLSryPyrUJ0/faxkDoFepvpbE1kMZ/NNojVjria2ZvH8y4W3CKTUpRdVEGR9Ld0sqCiu07l0bQlpIGmW5ZcJzBnDp4sIi1SIWqRZplV79v4D/9eLf33X23UtYaQoMLQz5c9qfnPnkjDA+bt04puydwpDvh2hVqIrcE0n0ASlxoYasDAitbmo4PWY58Vxbfk2jS04mk1Fk3YwSkxKeW/kcHm95NP6i/iEs3CwwtjNm98zdwnVXSWYJekZ6+IwVe5+a5aj3Zz1XzqDDMx00YvWldGOOxDzgM3+M/5/w2LPvMf4TUBioB63qtAxAqv1eCz3pLVu/qOT3lFRS6vg7xwnfrpbHqpHgqw+rFlYYWhqSdi1Na8LhfpOLXV7uwtWfr1JZXClZILl2c8W5szNnAs/Q671eEi1rPWM9rSwrgMUui7XGACZun8Sa5QqKfZpjIlVko7qymoMvHkSpUEqMov8JgpcFc+uPW4xZNUbIALv++3Wa9WpW221ZHxZuFrgGuNKsTzPaTpXK1w3/cTiha0JRlCskE18NQn8PBeqSw/8GvEd5syB1gVY26p/T/iTmUAzjN4yn3XRNL0GVUkVpTilKhZJJ2yfhPUqsJx6+PZyo/VFM3Tv1P7HYrq6srpWgbTmspcQzszijmPTQdNz7uDdY+P2vQaYjU5url0jl60Dtv3jqg1NM3DpRWJD+r8LI2kjoO1MfGWEZZNwQy5c+KpRmlWJgboClu6Uwnp+QT2FSIdZe1hLmvs5f+SKblDAs0+8AUuKEXdwVdKqryHdu2hhfs/nQVszrMLsD7aa3E845f4d3Or3DS+dfwnqeNUFZQfR5X7OQ6RJ5io4VHQlvE45KpkmAyTkfgfeVo9j62dLjQg+KWjXNnzHg1QDSQ9IxdZKet6JcQWl2KYaWhsIxqOOcjsQciSFsY5hknKtBemg68afjaT2ptYanSVFSPsaFGeQ4t6VaT5pEt2hmwRvpb5AWkkZqcOp9s4tFuPj9RUJ/D+XdwneFCfw9s/agb67P6OWjJbGKogqSgpK0SoAnnk8k4VwCA78ayIhfpIY61ZXV6FWV4RZ+lKR2/x6Zw8HPgd7v96YsrwxltVLS1ZgUVE9qRwXUI52W55djk6yWRTRu6Uyvd3tpHOs52JNFqkUkX05m07BN6Bnp0eVF7UXge3H5x8tE7Y8SdoSV55cTfSC6trhwLzo80wHbVrZaO17iShwwDphBuYk4qVWSmINN6i1SvfpQVVU3hgBELtqM3V8dFK7hRwEx4UiEVuPU0ofaJLDOBKoTJCJCmbJaSd7dPDo820HiSwqgUlSjlOuS0bwbFlbS533O6jlU61Tz24u/IcvW7FrRMdTDLuEaNqm3iOk0CacH6NmnKFcQskJdwBf5WVq1+Euyr7iS0qxSjQ5QgNMfn6Y0u5TmA6UkPR0DPVK8+1NqpCDTzYkAG801hlwOUW11OD/5WwCqm8CozYvLY2mLpTh3dubZK89KOl+ryqo4NP8Qrt1dhX6Xo5aPQqVUCX3xahC1P4q7x+7S5/0+GkUo9wGerDkeQYmJutNMqdI878iAGVQXLEQpV1Kgvx1Y1ejrSgpK4tLiS1g0syDgVWlxwMjGiEs/XMLY1lhCMtQ30Wf0ytHsm7uP6IPRBLymeXy3+d24ufEmKVdSCFkZIlmft7rwu7rLrdNExvhp/lb61qbkOfriHzGPdJcCWneXfqcNXtdfsmAi79CCxALW9V9Hr4W9cPCTrtf8pvr9rRS3ib2JmJSpVFJhbEm1jgHy6kqqBV14+iV52CbfAJqWaK6ZhyL3REq+a1BLDm+bsA1LD0tejXtVEm81rhULSxYK31tHXwczJzN6vN1DuLeNOpaEUeRtwgeHk+CewPO+z9fGqiqqMclPIctJF/9Qf4wqLBt/UY3A5pGbKc0u5dnLz0piijIFuoa6tBrfSuiH1hDkunJeT3qdrPAsIVEJoCCpgFdiXnngPvYPExVpubjfPEByq4GUWjpL4tue2IZKqcKsgxe5Lk2TnNc11K0do/8JyvPLJYRkE3sTur7SVatyxdR9U3l1wQayLA5hr9BUH9Iz1tPadf2o4TPah9ErRxN9MFqSv1GpIHTwm5hn3aXU0pmyMjC8J2Vwcs5JPt7yMUE9gihqUQRixWQhzJuZE3s8FkWZokGPcxGqSqooyyvTqtKTfCkZua4cR39HZDqy/7NqPo+LfeJ/1+Ber/SmNiJZt7QW7h1r4DNae8PEnT13GL1qtFA5oMWgFg0+8ybZCZjk5ZHUZhhfhw6RkGRzXdtTVpmMRbgFya7/nuoXqH3CFa8ohITl8B3hHHzpIDNOzqB5f+k6N2S4eg4f21k6JxlaGjL96HTOfXaOUx+coiy37KHZijzG/208LvY9xiOFzl+ZlGTfwWS7daRzZ0vh6271fRGljh6DdaULGB0DHaYdmsatLbeERYW2U9vWFvvKC8olkpejV4wm4WwC7n3dhQnl2BOxVBRUUJxRjHsfd+zbNJ5R2G9RP7ov6K51QVuzSRUtnDJvZXJ5WwIOvb3pOlia2CpKrUsa5cXlabBd0kLSOP7+aWJKAijIhfJy6aIy/UY6RalFtd5QDwrRB6JJOJPAkQVHmLpXKnN04IUDeA72FBb7PId4YuNjQ358PsY20u9MplP3PQXKAoWTf8e5HXHr5Sb0gHlY0DXUJe5EHMmXkxm6eKhkoV2T8Bd1OZRml/Kdw3e1//92ztuSDhGVSkX86XjiT8XfF2vvYUBHT4fe7/emILFAKF+aE5nDpmGbGPbjsPuWu3wUcGjnwDv572jdLF1arGa173hyB74TfJm8U+yZ8nfQ0WnaZuyf4va221z47gKjlo/CqYM42dF+RnutHTOPCkO+H0J5QbkwplKp+NGjzqvv3vHA6s5FHO/cILHtCFK9xN6kZrlq77lcdzF5QBtGLRvF0MVDtXZz7XxqJ0lBSbxX9F6T3hfUEluh/qEMODVAmKCM7D6Tk+3bM3b3WDIc6rowMsIyuPOjWtL62dXPsnf0XgZ2/bZJn23uas6MEzOEsYRzCWwcshFTR1NmnZ2FjZe0YHLx+4ukXE4RFvsuLr5I9IFo4k7G4dTJSSNhV5ycV/tbKHV0qSgcBtTN1anBqWRH5XLgyzCqY+KEXob3i/Yz25N5M5OYQzFC4kt2ZDao1OsHQwvNidSliwvDfx7O7T9u03pia8lYGHs8ljOBZ/AZ4yMcJ2uSuvYJwaR79gQsH9h1NQQjayMKEgsI2xBGjzd6IDfW3EwGvB6Az1gfmg9oLinSKMoVmOYnk+HRhWHfSwuYNdA31cepo5MkUf53Y59Hfw+cOonHKH0zfcauHYutj60wbmxrrDWhoKioxjQ3gUojS6qMpMnijLAMTj2tlsHSKy+iqkpz3VRuagt/FfvMs5vmGynXlTN+w3gsm1sK422ebIOtr62Q7KOqVhG6NhS/p/wk9x9A6ZVbdDy8G4D0HuMBzWfvpt9NlHIl8mo5RuWa3UlKpQqLLDUruHnoHioGNn280galQomuoS7tnm4nlChyDXDlg8oPUFWrhJ08JVklmDqZSorJoE5UVZjaUKx3lEJVHHdLPqO+7pn1jVO43jlPpo43AZcCKKvMgcGNO++afURqcCqqahUyXc1zD9sYxvXV17m5+aaw2Nesp/Y5tLK4khPvn+Du4bvkROXQ9aWuGs+HS093ov2sMSsyw/Oup3rPYmPAhg1w8yYU2TbHKtOEGRtmUC2vhiYo6PnP8ufUh6dICxZL4Vu6WzJu3Tit1gI161eRZDbAyN9GErw8mE7PdZLE5Eb6KBSGqHT0JAlEmQzcwo9ilVHN3Y4z8RrWtLG9wSSgnQlOHZ04/8V5/J7yE+7fQteFYmhpKPQi3DNnD6YOpvR+vze6BroahSBTTwduDqgrtFnes5WyTQjG46baxiGh7QhUqsav+6y91Oostq3F49yNdeoCokgh5e/g3NlZWEyrwVWn0QRNUZFjk0OObQ5GFnUksYrCSnwurccHqDBsQWT3sU3+/IYQfVA9vhYkFUj2cM0HNGdB8oL7fm9zV3NCVoewcehGei3sxcDP6ySqCxILWOK+BFCTF6cdmnbfn/NvwtjJglSv3pRaiOfL8RvHs/vTWxRbaSceaENlSSUJZxKQ6cjwHOIpyVPc+uMWO6fuxN7PnhfCXpAc/07eO1QWV2olvGlTeaiBabk3WRaHcK8e1ODr/mvw6OdB8qVkUq6k4Dm4Ls+hVKqJ7RZZMThHnSHz1mSsetet7yP+jODWllsA9LzQk9Ke0o7IhtDjjR74PeWHrlHTU7t+0/zQM9YjPz5fSKTbMHhDrYLP3Ktzce4sLSw3Bv/2vvcxNFF/7n3Qhc+Ecwkknkuk9UTxnBS6NhRkaJWb3DllJ80HNuep/U9JYkqVkh8u/kCAawA9m0n1ba3iQrBMCOOacxuMbTTXyIpyBR0PfoVc2YYYzxgOjBJbKz0M5ETlcH3Ndfxn+Wt04tXANcAVmVzG+gHreTvnbSkhWybHNfwIezpcwiHkOUnuxnOwJw7tHPje8XsSzyc+Lvb9h/Eox77/O/Slx/j/Djo6Ovj5+aGjo0O1niElVi7IzcSSluVmdlQaWwpZJsVpxWwavokb62+QEyWVn/Ed78vUfeqiU8ROqdZz8uVkds/crdWPZ8OgDWx7YhsHXzxIwtmEJlyhWs6qMLlQyBQsyy1D30wfj/4ewkTHzqf+JOTTg/z+/GVKBHYS9aVGqis0C0Dl+eUknY7F6+pmOu8PJCtC2n1iYmdCl5e7CFmj/wTjN4yn+xvdtfq0TN03layILI69I/XEAfjR40fW9Vsn9JIoSCxgyp4pDX6+10gvus3vptUH5mGgorCCKz9d4erPV4Umu6OWjeLt3LfpPE/qj1h/Ye7UyUkoSVJdWU3Uvij8pvkJzc4fBeS6cgZ8NoDx68cLFzGWHpZ0fK7jA+3AeVCorqwWyqGB2i/qC+MvCNsYJowP/LIuQWDerGns4hrUH/v+Lex4cgepV1MpTNLuV2JoZai1Y+ZRoTijmLANYULfUZVSpbWDB0CHauSKKiqMLCnTkgyJ6zCBa8PFDPiGINeVY2BuIBy7Qf0sm9ibaJAyGovRBqNJcE/gxuYbwk6fUktnKkycaX+jPR3i6hIh93olWedaI7No2nixotMKDr58UBizaGaBgbkBxenFEv8iUEsy5UTmMGn7JOHxx985TkVRBc+cfwaHdppFTMs2LkR3Vo/tdkmhFMZpzlnX11xnz8xdKO7cJd9KyoD8J/Do60Hu3VyuLb8mjD9z9hnSQ9M58toRYTw/Pp/E84lCSbVO8zoxN3guesZ6ZEdmSyQx+3zUh8TWQwGwj9cu2fYwEPBaAFP2TBHew3a+dniP9BYSqEwcTAge+SEpPgMpuBwp6W4tySzhK8uv2DpuK8OWDtPY/Ddm7Lv43UWhlA+oCzFXll7h2krxb3X0zaN85/CdkFhTkluB19U/0Mn+AZVKOk+n30insuAvb9OyAu71U0/2HUxi66FEd5nKzYHSbpqGUJJVgrmbudYipe8EX/p93E+4VpTrynnu2nP0/0xcZNF3rCN6OV7YJYmf6XeGc33O8eKvLzJmh2bBOS88A92qGjKFCnm+WN1CGxQVCvLixIao+qb6vF/2PqNXSDtia6Cjp4Ouoa6wGDjyl5G8kfqGkHhTnFlC25NLaXXHnX5n+lFyUXP9rjQ0Qr9CyVNbnqJFXAsUeeJOfRFMHU15+tjTjFw2Unhedq3t8BzqyVMHpEmpGvzY4kd2TNkh+XtVWRVXll7BuYszC1IXSDpnVCpQoaTN7TZM3DmRgmT1OmXbNoiIANPcRKwL1Wvr4C63G31NNdc1Zc8Uur/RXRjfPmk7J98/KSSYVBRWELEjAp+xPkKptzu77xC+I5yhi4fi3EmaCM4bMQ2VXE7n/YHk/6G5ti+LTcMqQy0fW2ru9ECTkEbWRnR8riMWzSyoFni+By8PZs+sPVo926MPRHP+y/N8afqlZI8ok4FJXjLeF9dhnhkjOe8qQ/X6sMDOkyyPLsjljV/36Rnp8XbO24z8RdztPfjbwTx75VkGfyOuYNf4mWmzpgDYMmYLJz88Kfl7aoYOKh1dZCoZVoXdMNOvGzPkBnpkeKi7tA3KS0lx0e4Vdz8Yu3YsvhN8tZKoInZFsNh1MTGHmyZbpqxWEr4jnOj96vNVlN3TwiJTFwOBJr/3o4SuhQmOMUHYxV8Vxv2m+qE/cyqVxpZNfu/40/FsHrmZTcM2CffTkXsjAYTd5qDuPDF3NRfOaQnnEvjO4TtC14UKj02+nIxjkhz/6/40C9W8F5QKJev6ryNQFsjmkZv/UxJ2pz8+zU9eP3F99XUyb2l68imqVNjHXcYm5SaGpbmELNX0x00NVkuPZ9hn8MeTf2Axvml7sQvfXSDuZFyTu/oA0q+nc/7L81rncv/Z/rX/NrQSKxj9HR7FvvdePCLLrP8Mmnr9TZmLjW2M8X3CV2vOJ+jrIPbM2sPyDsupKJL6oo5bN44Wg1uQHir1Y/3j/B/s/mg3q/qsIuGcNA+b492d6C5T0a0sJSMsQ3O8kqkltdMc09g3el/jL+gBoDy/nKj9UcL8BYBTRycGfjUQl24uGuopNbDIiMYxVk0w11BbQV1IDJQF8r3j97yV/ZbWtcBjPHo86rHvcbHvMR4ZVCoVhYWFqFQq7OKv4nV5E6oSMZPJ48YeOu8PpDy7WPo+yrrZKO6UuGBn0cwCez97jGykic+aAXTv7L1k3s6UxGvg0M6hyZ59W8dtZe/svcJurMzbmVQWVWLX2k5owp7jHUCVvgkF9l7CYt+Q74cwcdtEXrrzEratNBNIzQc0x6SepF1VhfTzLT0sGfHTiAfuO2ZoaUi/j/vRrHczodeP52BPlFVKsm5JfRMSgxJr/518Wdpqf+7zc/wx9g8mbJrAq/HiZFtebB6fG33OqY9O/YOraBoi90WSciWFkb+NFLLqdPR1yInMIepAlCRmYGbAy1EvM+DzAaRdSxMWmeQ6cnq+2xPv0WKJz4eFHU/uIFAWyK0/bjX5WH0zfUJWhAjNlB81VnVbxbp+YiNzY1tjTBxMhIkhgE7PdWLyn5Pxf8afXu9IOw4ag/pj37+F8RvGE/B6QIOdvBe+ucC+Z//dxfDfIf16OteWX6M4XTr2y3XkvHDzhVoviXtR3rkXtwbMp1rfCJ2qcuH3rZLr0Dx0N063xOQDbYg/E8+uGbvIuCmWPbVvY09ebJ5wHPs7eB304pm1z9DWUtxtKK+uwrjMmS/f+5KEMXWFL5cuLgw8t4ib/V5i87RD3G5zG3mx1Oe2IZRklXD1l6vChJetjy2vxr/Ki7dfxL23uySecjmFwpRCYdc2wOyg2YzfMJ5mPZtJOpTkBvrol6sL0ektemDsZKkRb/NkXcddlruUNPFPMXXfVIb+MFQY0zXUpdO8Trj3k15zWW4ZTh2deP7G88JORzMnMxw7OLNj3gl+afWLxnoF4MzHZ2gWfoQk38EU2P9743vcqTjW9FxDSWaJVtmy6IPRnP/6vGT9IpPJsEkOo+ORr4j94g/CNmnOWTnRuVQUVJAbk1ubEKxBY8a+AZ8PoMdbPbTG5XpyreecF5tHSWaJMMmt0tPnYsBFWoUbYZ68VhL3Ge3DkN0vUGpmj1FRpjQxIpOhW1mKy52TyKobXzwCdYfnun7rSDyfqPU1qwJWcfSto5K/y+Qy4s/E85vfb6ReE3gRejXjdu/nAChxlK7nLErUyfko7ygynDUVE3RM655DneoqVPt/b9wF/YWNQzaytMVSSrIEi1TU99DZz85KiAigXu9tGr6J4GXBWru3jyw4IvSAUSrV3Zc1yNPRXLtfCwilyFQdXztzLbojxLKt2tBiUAs6z+ssLPY169mM6YenC2WXAJa1X0Z+XH5t8aA+jKyNeD35dYb/NBwzJzPJfRy64gp9DiaR2CyRrZO3YuKs2RXT6sLvWGXn8tvzv3Fq4OUmXVNFUQV2beyw9RUXnJ27OuPR30MYqyqrIuVKCpF7IskWkAdjDsfUFsUKk6WEIpVKTeIAteysBpTVKOU6hHbMwSj5Y+4ul3o0NoTwneH84PYDwcuDhfHO8zrzWsJrQkWDtGvqLkdtibI3099k5umZ+M/yl3RmV+WX4Bm8DfOceLyvbKI4Q/MZKHDwJnjUIrLdOuBy5wTVCmWj133ZkdnsmbNHKwnVxstG4iFYHzUJ15ubpP6J+fH5nP/6PFH7osi5IyXI6hfnYpFdRNcrXXl18XCNMUvHQJe0ln246ZfEtknbQPlg/br9Z/ozeedkIYGwMLmQO3/eQVmlbHL3krJKyfZJ20kNTmXEryMk/uoWbha1cqgPUjXgYUOlggzPHuQ7atd7vN8Ch3VLaxz9HfF7yk84Dg79YSjzo+drJeCe/ewspxadEo79esZ6lGSWcPE7sef1yYUn6XvMlO4Xu9PssubxVaVVxJ+OB9TzS9DXYt+/R4H68+C95N7qahXNbquJDiXmjvhM1Vzf9/2oL4PDB7Ps+WXc8b2D3LxpKdq7R++SdD7p718ogLGdMT3f6SmUUAQY8OkA/Gf5M2bNGIncdmPxKPa9j6GJ+mNBY8aFpvxUdq3tcO7sTHm+eC33xJYn8BrhRUVhBapq6Ru3mdyGa8uu8ef0PyWx6vBqBp0YhEeCh7BwVmntSJJ7JVX501nWfhkVhXXFRF0DXdI8e+GU7sSEPyfQ+6zUzuNhwaWrCxM2TSDxXKJWr+Seb/Xk2UvPCon8Xlc3A2Dd0V3SMVn/OaosrkTf5NHb+zyGGI967Htc7HuMRwalUklsbCxKpRL7+CtYZMVQGSRmp9nWbBJlAgNTl7oNmEji6MTCEyxrv4zMm5lCT7+QlXVJoWsrpGzxmkTy8zeeb7JPgG0rW3JjcsmPy5fEatixV3+5SnmedHK8XOHPjSFvUmQrTSioVCqu/noVmVymlSnuM7kd5cZWhA5+AysfsbfY2r5rWeq5tAlX9PcoSCxgTc81fO/4vdbuutcSXhOyojNv1iVs7vUMAWj3dDuG/TiMNpPbaPXwqmHoapMiehiwb2tP7/d7497HXSjJevbzs6zuvpptE7YJj7fxsqHz850Z+sNQWg6TelbKdeW0fbItch05ivKmJe//CWo6NkTdhuX55bUG7GnXpbJQekZ69P24b5ML5P8G/Kb5aRQP6sPcxZySjBKyIqTFaFAzUkPXhNJhTof78mMDzbHv30K76e0YunioxJurPmRyGZXFlcLN+aPC7W23QQbOXcSyMRG7Irjys7gjqsZs3OPGHjoc+RqlYINhUJyDdVo4hoXaiR4iXPrhEmEbwjj/5Xlh3K2HGxM2T7gvuZvMLqVcDLhI4tsqYbLP/8g3DN/vTZV+FdwjbZ249RJeVzaTbR3NrLWzUKwTdyxow+nXTgNwac8lYdzIygi71nZCz77JOyfzkeIjre/t0tUFGy8bqquqJUWv6spq9MsLqTQwI8u9E/pWmsk+t+5uNH+mD9FdppLv2ErSdfVPcOztY+yZtQdrL3ESI+ZIDK0ntRbKz2RFZPHHmD8kRa0aKMoV7NlWwflsH4o79JLMDzoGOlQYWVBo50mRrcc/vZRGw8jKiOYDm2usn+rj8OuH2TxyMyfePSEhPigqFBiU1nWA+U7w1Ygnl9mQ5tmTHGc/0kPSNZi4jRn74s/Es3vGbiHhIj8hn6LUIq1eQv0/6Y/nUE9h95GOgS6Xu10m3DecQgvpextaGiLT16Na14C0lr0liQ7bhGCcY85hXJSBaV7Tivg148CVn8Rj1ZlPz5ByOUU416pUKgzMDHDu7Cx87mQyKLNwInjUIpL7Py2J9zvemsFHB3N06FGS+mkWPExdLQnv9SxXut3lSpcrxPlLSRUNofmg5ti1sRPOK9WV1ex7bh+nPjwl7aQBcmNyiTkcw4EXDpAWIl1DJJxL4NIPl4RdnobWJvw+ayVKmZLDQw9T0UZTnv+u8WaWvrKU7xd8T7JrMtXKxm+yS7JKOPzaYaL2S8lZNShKLaIkU1zgdO3hSpeXujDkuyGSmFxHjrmLOapqFdmR2ZK5tqKoEpMSPbJts4loHYGuqWZnbYGdmkhhmW+JiqYlDq7+epWfWv4kLNaB2mc2/nQ8FxdLE/Am9iYM/EqtanDyA2k3WP9P6rpONw2XFusMbwVzu4Mlf8zIIGGs5jWZeLtyfdh77BvxKybFVWSnisdSbdg+cTuFyYXCbvPcmFyW+S/TWggcvWI0i1SLtBY5VSpI1fOg25djJSoC5Wl56FfUFZyLEqVJSN2KEjxDduAUc56qyupGr/uK04oJXRPK9TXXhXFFuYITC09w6JVDtX/Lic6hqkxNQpi6byrzo+czbv04ybE50TmcePcEo5aPEnbh2906Ta8j0eRb5hPpk6ZhQ6FUgsLQlD/Hr6PHhR702ives98vQlaHcPU38XumhajJkH2+6oNHX48mva9cT06neZ0wsVfLut7bsa5SqagorGDC5gkNEk3uFw8ryVdwJxXr1Ftas/I/tviRyoWBWKU2nbBp62PLvOvzmLBpgnB8N3UwxbqltVb/tlMfnuLsJ2eFikvOnZy1Ep0B2s5vy+mBt9n5xE5OP6E5/usa6jJq+SjS2g4irWUv7KYMEL7Ho8DIX0bWenPpGGiuzauVMuL9RqHQMySjRQCW/h4acV1DXTJPZTLs8DCsc6xRFDVtH+Y7wZfE84kUZzRtDge4teUWQV8HSXwGa2BgbsDY38dKvFibgkex770X/+t1xqZef1Nff/azs4RtEKsiOfo78tSBp3jl7isYWoq7Q3u+05Oeb0tlOq3bWLN39F6Of3Bcqx98uX4SUd5RnOp3StIZXmTjQbynDPdEdwJuPlg1s79DTmQOkfsiNQqQNQjbGMaqbqu0Npok+I0kptNkui+fJck32frYMugbtbJP7LFYIcnqMf4beNRj3+Ni32P8J3C385Nku7ZH7i5mFWU070ZC2xHoWQh83GQyus7vSr/AfvT7uJ8kXppT1y0oKqrNvTqXF269wNjfx9JumngSuV8MWTyE7m92F05spvU677RJljSE0x+dZvvE7Vz68ZJEsqo4oxjHLm5EdZ+JwsCUakGTUlZ4FglnE4TSY/8E++ftJyPsr24XwULhC9Mv2D5pu/DYdk+3Y86lOTx7+VnhhO452JNur3Tj0o+XhPIzoJbFtPK0ou2T/x4707G9o9qXToawwBmyQruUTlleGdsnb+c7x++48N0FdTu/ANfXXGf7pO1CebKHhedvPM8i1SK6vNhFEqvvnyhiaaXfSCd8ezhVpU3rgPg3EHMoRut5GVoZMnrlaK33z97Ze4naH8XGIRu58N0F4Wv+iyjJKiHuZFyDG0GbVjaYu5kLf89HBe9R3nR8tqNQmkapULJtwjYOzT/E6u6rJXGDzCRsE65RYulCllsH4cbF9c5xVMiI6dY0j5beC3vjNdKLDnPEm9/rv18n5mDMfXmHtmz+GmU2Q2h+S0X6DamkSY6LH9Etw3BOcSY7/0Tt30syS4j++QiGpXn4RvXndpvbKNs3jfSwP38/P7/0Mxd6Su/ttJA0NgzZQNimMK2kg+2TtvOV5VfCWFVZFZd+vMRn+p9JJLGzr8XjFHOeYutmqATJIz1jPVrM7o9VWjj+R76WGMn/E+gY6KiTHFpu+wPPH+Dk++L5xsbLhj4f9aG6qlrIaD3z6RnCpn5FiYUzd1wGShjyKZdTMCgrwOfi2n96GU2Co78j1i2t2TF5h7DAZONlg20rW54+9rRkfVKaXYpz9DkK7DzxWfK8pGMmu9SYFN9BZDbvRviOcJIvNq0wZutji3tfaRclqAkJpo6mwqIXqIk30w9PF/pvKZWQb5XPtie3kSYgCynKFUTsvoNZXhJmuQmS8cImpS5h2vy6lH3cEGx9bHHr4YZzV3HxvzCpEGNbY6HkZVVJFfvm7sPSw1JI7iqPSsD33ApanV+FUZa0c9A93poWseqOv+v2mp5XSiW0ClpD18ueWOVZEdRZLFWrDX0/7MuLt16Ueo2gJgQVpRTh1tNN+Hu1m9aO2UGzGfjVQGG3wJWl6sLorDOzJDGlElJdMtk0bRMJ7gncq4HkHzmQgEsByFQyvKK9kCU33uu4PL+cyz9eZsvoLbXFk/qIORLDYpfF/NbuN+Hxo34bxYifxV6WSoWSnKgcTn10il9a/ULeXc0CUadXexE66CUKLNUs8HuLlDGdp3CxRzRP7HyCCTv6NvqaoK7gfPzd48K4jp4OZk5mWgrKdd9vy+FSQpqJvQlDFquLmz7jpAQvs2tnMMsO406L3zhQ9oH03KJO8+FnH7Jv9D5iXmzanmTkspE8deAphnwrLa6qlCoybmQQsjJE69pHpVJplds++FsCS95NY+5caczE3ZaortOIDJhBVNdpmDXXfDZNcpPwP6b24852bQ+C7ihtqPFOriwSfxfbJm7j/Jfna/3oK4oq+LX1r5z74lzta6xbWgvXHy5dXZh7dS6txou7wQrd23K7oyNR3lFsmbocVb1teUlWKW1O/8LrP7yCQYUBt9uIi5H3i33P7uPgiweFhXTTdqZsmbKFJ2OebPL7ynXkjFo2ipmnZhL0VZCEnFOYVMg3Nt/w51N/CovG/xRfmn7Jnjl7Hvj7IpOj0DNCpSPOIVg0U//+1XoP3v6hNLuUs5+d5cL34n3Q0B+G0mpcK41icX3MuTSHeSHzhLH5ufO51PVPMh0yybXXvBd09HXo9FwnUjx6ktJqIBFFTfcjfJiYHz2fqfunUnJPp69SKSPbvROxHZ6g0siS1fdsWQpTCrnx0g26XenGKz+9QrMzTfNPt/ezx2eMD7oGTc8nBbwewOiVo7Wu69f0XMNi18XsnbtXqyTh/wX8rxf7murZ15TvKzsyG+dOzrR9Spw7qa6s1qqYBPCD2w8knEkQWgDpOuoS0imEYtdiIfHA6fIehmwNJqZlDGf6ndEoWleVVeEQexGPuyoq9CsIfS+08Rf1D1GeX45rd1dejX1VaDlSWVxJypUUzn56VlgMzHLvTIWJNdnX4oUKcZ2f78y86/PYN3efUAHjMR4DoOkzwmM8xkNAuakt8f7jcNeiMJfUZhggnnjK88vJCMsg4UwC9m3tJSzz0ctHY+tjy9E3jpJyNQWv4ZofkheXx86pOxmzegwuXTULLcpqJZ/qflr7/4O/HUyPNxvP+vMa7iX5PBFEm2uP0N3YJt8gyXcwIP1Mp45OpIWkceS1I3j09dBoAU8NTuXgjK04NetIiaULRemtsK234FZUKNRmuUCfD/o0+noag7ZPta2VgBN5AXkO9qQ0u5TE84m1G9oa6JvoszpAvQK+10C9PsK3h1OYXMiAT6WMvlG/jfqnl3BfOPPpGa7+fJU30t6QMHBmnprJis4r8H/GX3KcolxRu2EvSi0SsiSrSqu4ufkmzl2chd2rjwIGZgZC6cRaqNSJrXs7eP4LKEwuxNRJ3JWXE5VDUlASdm3shHEzFzNyY3KpKq0i4s+IJo0HjxKbR24m9WoqEzZP0GriPOyHYQz7Ydi/fGYNo+v8rhQmF1JRVIGBmWb3Rv3Fb/IlaUHBIPo2Hjcvc2Pg61S1MBdq4me7+lNs3bRNNagTZiIj8drzuZgs7FZpDNLKYwnq8DZ3PDqy8Hmpn2BC+zGEtVnIa9/OIbFlHSMwO7KuY2PwET8SmiVQMlPcAaUNHUM6km+Zj8pS+tyWZJUQeyyW2GOx6JvqSwoqGWEZlOaUapW3+8L4CwwsDPB7yg8Te83OPQNbc4qsm2GddhvrtNvk3n4O2tUVkQ68dIDgX4OxBap19VEqVQh/UAEU5QryE/IxdTQVjp8DPh1AyMoQNgzZwOzzsyXxkb+NZM/sPZz/+rxEutfE3gQ9Yz2Ov32c5v2bSzo5nTs5U9nGn2pdzXu3Bk/ufpLPxwVjlxiCY0wQIGW1PixYtbDCraebBnGjBl1e7CIkeYBaQSGxzTBKzR0xdJeqBqhUYFiUhUom46U7Lwkl2RpC1P4oLJpZCNcPFm4WdJrXSes8GLk3krTrafR6t5ck6VWaXcLri18n1D+USifpM39q6U0SV6hlNMtNbSUyR7Edn6DtqZ9RynVIadX0boLZQdJ7qwajV4zW6m0n15XTYU4HrRJbVem5mBSoxxrT478DmvPyz/N/QSmrpO/pvugq9TXCxcl5yP/yL3TIcEBGfOMv6G+gb6bPmDVjsPWx1epF6NbDTet19XirB22faiskw+Xm5NL1SnuGHVHPVxl2mkU5h7vQ/MZABp5UryGdrLR7u94Lq+ZW+M/yV9sCCJYvNclO9z7igjTAntl7KEgsYMbxGRp/L8ks4Wefn7FrbUfv93tLEuEllaVEO33ByCPv0f6aEQUjCrHuUqcootLRpVm8KfpV+njeFcsla0OLQS3wHOqJRz8PYfzOnju0HNGSTs91ksQUFQoKEgqw9rKmzSSpKkJZXhltJrch4NUAoeRf5uCp7HfpQIeQDpSaaRbdKrMLcYpRd8jrVOtQw0NWKVUoFUqtvrg1EHlh18DG24bhPw3n0PxDZN/J1iBZglq94vfev6NUKHkn7x3J8dfe+gPfskpyXfzICOuu4TWrb25IoZ0nMmU1KrkcXVPNe9y4oG7+T/EZQGPnK1A/G09seQJzN7GaTItBLdA10GXitomAeo+gVChrCa1B3wQh05HRfkZ7yfhraGGIc2dnog9FY2hhKHn+ipy8iTG4hkquvvkr68muqlQgV1RiXm4ORfDLy780+poaA98JvkT8GUF5frlkjZCsm0y2bTa+l32JvR5Liw5Nt6C48N0F7uy+Q1VplYbqSH0PqX3P7aPNZLHyx/2iqrSKlMspD/Q9AUw8Hak0ssSoSNwZMuv0LD77DAqbpvgLQNKFJNb0XAPAe8XvSWTiDrx4gPDt4RjbGdPjDek+KOC1AAJeE3fR5N7NJWxjGD6jfXDqKJXXlf31rOhW6aJTJX1uUq+lYplWQL6DD0VF/52+hcTziUQdiCLoqyD6ftyXfov61cYUChWoVHjc2IN+RTEZ6d3g87r91uUf636klDYpDcr0imDtaU3fRX21dk01hLzYPPbN3YexrTGtxklJAEkX1PKg11ddp+OcjlqVFf7r+F8v9tXHgy72FSYXknA2gbZTxcW+X9v8Sm5MLqNXjabtlLaS8cSyhSVGdmJSQk5ZDoZlhuRcyaF4XLFkHq+0sCG3whyVTHrC1RXVOMZdotjCjuaBXsycNrPxF/UPkXkrk41DNjJ0yVACXpWOhZ2f70xOdA6XFl9i8LeDMTDX3CfKlNW0ObuMy2fBOKu/Rr62OKOYvXP2om+iT7/AfsJx9DEeAx539j3GI4ahoXpRYliUhVl2HCotkg7mWXdxiThORZ7U0684o5iEM+ougaxwsfTehWy1F07UMSnrWSaXUVVSxelFpyUxZZXm+WgzntWGH5v/yM6ndgpjiX8tngLeFutH61SpN206CinbQyaT0fNddVKw+cDmEoNsR39H5Po62CWG4BG2j6D9mkzFqH1RXPj2AuM3jKfry12bdE2KcgXXVlyjsljMOG3/dHutHloAT+56kuzIbI6/I2UX1++MEOnwH3/3OL/5/caU3VOYd13MCCzPL2fPnD0SH6GHibtH73L156u0GNRCKINh1cKK8rxyLi2WSuOZOpoyeedk7P3skcllBH0rvW6lQkl1ZTXOXZy1ymw8DETsimDHlB3kx+c3+ViH9g6Y2JsIJWwfNdo93U6YkAIoSCggdG2ohHFfg1mnZ9H9je4ATNg04b7PoWbs+7dQw4ZrSFZyy+gtLHFf8i+dUeMQviOcZe2W1W4260NHX6fWX6XmN6kPVZfORHWbjkJfnVAVKSgUOPpQZmaPSY52Ly0RlNXqZ1KbPFPA6wHo6OkQ8WfTPSuN9iQwb/k8yo3KJHJTNXDNfobjg46T6ld3H9v52uH2dB9yHdWEl9ttbqOiabIRow6Mou+ZvuhXSAkoLYe25NnLz9L5xc5YNZdu9k99dIr4U/HYtrYVMjjbz2hP/0/7M2HTBOxaaxbTjd3tSPXuLzmmBvU7FJJ8h6BSNT5xmnwpmV9a/cLXll9rfU2rca1oPkBcpPQe5U15frnEIL0GXiO8GLt2rGQeBnXysmzIWGyTb+B9ab1Etu/GuhtYp96i0tAcefWD7bJvCNl3silILKBfYD+thbPqymoqiysl97ieiT7FVm443g3ixvhAjW4SgNKYVNqe+ZXW51eScDZB0vX1d2OfTC7TOj4DnPrgFJeWiGVmLy+9zJmPzxB7PFYSU6pAJdOj79m+uMVJCwgJpbbqDhzAKfqcJNFRZWhGTJcppLQaSI6rlH3cECL+jODXNr9qSJrei1tbbxF9SCpZqWuoS6ZTe/avyyZPMBeb925P6OA3Aagwl5JUWiV/DTJoHd6a1rc0yXBlmerCS5lhGWVGZfTa11pyfEP4zOAzAmWB5ERLpdr0jPTwm+qHra+tUK6tJKuElCspWj1eXLq6YO1pLfRFTUiNqy30AejHa4430d3qPu9m+xxsujdeTlyuK2fs72N5Lf414XqrywtdWKRaxKRtUglEgHNfnCP091DiTkglmPVN9em1sBeDvh7EgM8GSAoaSzd/iUnmftJtjlJgrYehcV0xUK+sEP8j3+CS6sSBEQdY/ZJ2mVHhdenImX54Or3eFXsN3/7jtoatQX2UZpUS/FswudG5GFhIiQvH3j7GD64/CNnpAJU2zmQ6ZDLi4Ag6XemoESuLU3evV+pVYpVnhdkd9bO/zH8Znxl89rfXpahQEHUgisxb4oKH10gvpu6bKmTWB30bRGl2qVbvcpPxQyiw98I2+QaFKZoyWSoVOMacp9Ohz2kV9LtkfZHj2p5Y/3FEBswAmQylUtXodV9FUQU23jZCOWJQF1Im75xcu64zsjZizsU5teSzc1+c49ibx/hj7B+SY5UKJZXFlWyftF3obV5SApEu7+Me786ofaPIjaxbCxtaGxPe53kAEpppH8vuF2PXjuWN9DeExQRTfVPsM+0ZdGIQ6aFSxYOGUFVaRaAskNDfQ7H1rZM/q4GNtw0vR70M8FC8j16Nf5VZp2cJY+E7wvnc+PP78jdXKsEkL1lS7Ks/d9XnjzYleV9fRUZE2vQa6YXPGB+t11VZXCnsjAa1vO6Zj8+wotMKDrx4QBIPWBPAc8ufY/aa2YxeoellXF5QzsrOK2l5bRudD36Kav2Gxl/UQ8blpZcJ+kq9h793Pakor6bzgU/Rr1DPuSWGmgTFlsNa4vG6Bz++8iPnnjundT2qDVtGb+HEeyf+/oUC2LaypesrXbUW8Z4PUz/zFu4W92VNUIN/e997Lx4X++rwoL8L997uvBz5srBYDOA9Rv0c73t2n1BNZOe8naw/vJ6NYzdKYhlnM3j363eZs2YOkXukMt95rXtxrX9rul7pyjNrntF4fwNzA+Laj8G0IIuYoIuUJf976ljWLa3p9V4vVEqV1rVRwGsBPHftOUkBE6D12WW1/753/VKQUED0gWhub7tNz7d74j3q3/N8f4ym41GOfY+LfY/xyKCjo0OrVq3Q0dHB/eZ+fC6tR3H7jvC13pc34nQ3iKpCabFPYWSOSq6+leNPxUvi8afjuXkgkfDez5HdQlrY2jp+K6DWVb5XGlKuJ2f60emM3zCeV2Jf0TqJaUN+fD63ttwSJkJuR6iTTZe+OSeUl7nbZQrBoxaR6iNNgKpUKpoPaM57Re8x4/gMCZPL3MUcKy+1pEyGR1dup2tuGF26ujB65WhSg1OFm72GELErgv3z9nNtpdTfsAZhm8LY9sQ2rZ4mw5cOp8+H0o7C+gW6bq92k8Rril5GNkZauwUi90USuiZU2OnzsFCaU4qhpSHd3+guYeYAJF/Wfi4ymQzfCb5MPzIdq+ZWkgIzqBcr7xW+x8hfRj7Q8/477J65m9tbbxN7Qpo4rSypZOuErWweuZmCJIHxsEqdUC5KE8sjPUoknkvkzi7xWOMa4IqJvYlWz76SrBI8+nnwctTLWn0j/w71x75/C3MuzmGRahE2XuLkEag35wWJBVo36I8CccfVCVORzJtcR06Pt3vQa2EvOj8vZffrOthSaOeJbXIozUP+RFEhlhDxCN2Na4RY2kwb9szaw2cGn3H0jaPCuL6pPlaeVlrlBhtCetVt9Kr0MCrOEo6h7jf24n27ios9LpLcqu75MrY1xtTLmWLrZnzy4Se4Jruis1d8n2tDzPMxuCW74f6nuGvFpasLI38ZqdHhUIOa3+D8F+eJORIjiY9bN45u86XjOqg3nsb56mLaymdXIm+hOb4HvB6A3ECXAjtP8pzbNMpcvgY1vnSeQ8SdMDGHY7BtZUvfRdpl8T6o+ICpe6dK/p58KZnfe//Onll7Gpxz9MvyMC5Ik0jkyuQyKowsuTngFeFc/7CQF5fHxe8vak2Q3952m88MPuNLsy8lG3OVCoyKs7DMVBemJHOevG5roVQoNbomGjP2dZ3fFVNHU+E4VJxRjGuAK+2eFkuu19yDIuKXkbUJiZ3eBcAyWyrVZtjSjaQ2w8hx9iPbrb0kGaJbXoxz1Gk8bu5Hdh+ZkqzwLLUHqQBJF5LYOWWnVg/QS3+mojx7gdCz0vlUritHYWBC8KhFxI14URK3yVTilOrEmtlrWDd3i0bMsrUzNwYt4LcXfkMlU2GWd38bUm2+TSc/OMk31t8I/UQi90Syqtsqvrb6WmsRdNsT29j19C7J33Us9TkwQp0kvtXmFqZdNdfmydbBrH96PZe6XaLM8SWM3Bvf2VddWU1OdI6GBUBTUOOzJvLDNDA3YODnA7UnZk5XM373eGJaxrBt6hnMPdTznkleMu1P/IBulfpZbBfWDrfsWU06r7CNYWwctlEreWvcunH4PuFL/Ol46XlbGCDXUz/Xf4yRFpBaDGxBp+c7ceiVQ4SuC5W+eZUCHYUOm5/azKkBmnsOI08Xors8xc8v/0y/0/1w3afu5NTm+X0vlrgvYcuoLVz5ReqHWZpdSuyxWMxdzTG2kcoJtpvWjuE/D9dK3NLv1oG7nSYTMnyhZP7Iv5mEa6R6z2ian0zmVc17WKlnQK5LO+wSr9H++GKUFapGr/tSLqewotMKInZqLwDlxeaRGJRIdVU1lUWVBH0TVOvxO+fCHNo93U7YoXb32F2+NPuSZj2bCQu/LhHHmb52MPaZ9nS+1pmiFM39abWeIZ9+8CnRXtEMPzj8b6+lSVCpJbtFncAph1N4ctuTBA8IpsO4JnqH1Rue+i7qi+M98uY6ejpYuFnQcnhLAhY8WE8nlUqFvqm+VqKmTEcmlKlvDErTC8hoEUBKPaKUUgkLFsCnn8L5r86jWPQZRn91mTZl2vIe6V1L3L1XVQPAf6Y/U/ZMkRC3avCl2Zd8YfxFrfd7fbj1cMPCXU3gEq2bFIYKyozKiPSJJLx1eIPnqbSTrkUfFer7Pbr3vmcNLZdRZqrOy5zsf5ITg05rhE27mbLEZwl51nncybvDqbim5WaaD2hOTnSOkDxTWVzJqY9OEblP7Id64ZsLXF91XbiuB3Dwc2CRahGvxb8mfDYbg0ex770X/+vFvvrX35jvoin7LB19Hdb0WsOOJ3cI40O/H8qL4S8y7dA0obTvgegDUAZpmVJFHLmunFKjUmKbx+La3VV8AioZpsWm2OTYaEpeymSUmTmQ4J6FxXYVh0YcEh//EGDqaIqBhQFHFxwVepemh6YTfyoeqxZWQgUDlUxOhaE5PTa+IMk/27ayxfAvIuX9EPIf49/Dox77Hst4PsYjg1KpJC8vDysrKwrtPDHLTUSVI+2mqZmQyszsMHSUso52H9RHoWeEUq7LsB+lEnSXf7yM+80oQkZ+iK6AtNRxbkdM7EyI2BkhWdDKdeR4Dm6aVE59DPl+CCc/OClkdNzKrWNHKcoaNiC6N4+iVCj51vZb2k5pyxNbnhAeM2zDNN5+G6oMTBk3QvMN0kLSSLqQROjvoQD0/6TxSUarFlZ49PPAzle8wD/61lEufncRgIFfDZQwl48sOIJDOwdaT5QyyJ07OdPlpS60GNxCKJE05LshDPluCAWJBSjKFULW660tt2pf+2/Bb6qfVmlEoJZdW2PcXR+KCgXJl5JJOJNA5xc70/11aYcSqI2Pr/x8hWcvP3vfRaamYty6cdzceFMo+1RVWlVbMMu8lSnxBinNKaVfYD88+kqPfdRoN6Mdvd4Ts9t19HUwczETbm4Brv56lTMfn8F/tj/dX+8uZIv/HeqPfXL5f4dzY+9nT/zpeMrzy7V2lP3byIvLw9zNXNhVrVKp2DxyM8VpxRTEF0gSdrp/rXBMcpOwSb1JtWI0ULfY2vnUTjrvV48X8e3HNum8bHzUY49IAhHUXS0DvxioVert5k3IyoIBAjXAg8P2outfzIu/vUiwUzB9P9IsQtkk36BMR72hr5DVFdqTkiDotzCapYWjmqjCPcEdmRYPDm3oN7UfCWkJePeTJqNLskpIC0nDwc8BM2dpIrvlsJZ0f6M7IStDcA0Qb8jiz8QTtjGM7gu6a8whBXfSsEq7zfme58mxySGjLBmomyN0DXTptvd9Nr53G8e7QVQr+tNYvpqNl02DksNhG8K4ufmmWrZSsBb/ze83LNwthLKtOvo6VBSo5/ezn56VJPFvbb2F0aFo4ryHkOg3Cr179ri3ttzCuKgYz2vbiOkiLSY+LHj082DELyNIC0mjfGK5pLuvRorb9wlfySa0ILkI58jTANiODpCoAxi3cCR41CKMCtLhxeWU5ZbR5301uacxY9/do3cJWRFCr/d6ScahopQi7uy+g3MXMbvce5Q382PmCxP7KhUUmBUSNHIceio3VCpp14NxQSo2qTdJ8ekvSXR4XtuGWZ66IK32WJwjPAcRfCf4omukq7XT/eJi9Zpp8DeDJbHKkkrMsuNIa9mL0Awn7l2tKQpKMMnLp9zUBpVKWqwLOF1CK/NRrHxuJSYqzUWwTFcHs5x4FvywgB1P7CCnddMkVz+okPqv1aAwpZCL36uvS9Qt49LVBdcAV5IvJQs7gf+cVuefVZRapDnm6Mq51fYWSrmSVOdUBtks1ThWqSgh1TmVuOZxWJY+w9XcQPoh/W5FyIvL45dWv6Bvps9bmW9JZEQLkwtZ03MNnkM9hdKrz5x9BmRg5iQdI2twY8MNbqy7wegVozW6KdpN78pnpu9QYlJCgd5hyiorMDExQK5Qd/1muXWgQnmcJLck7FILUOuMNq7LuSCxgLtH7hK8PJhBXw6SxOW6cq4svYKuoa5kzWdgZsAbqW+wzH+ZsIO57ZS2eI/25kvTL5HJZfjP9NeIu25fzHSn6aybta5Woq8GupYmFDh4UWhRyKHhh+hiMRmAl8JfatR1GVoYUlVaJfT6KUgsYP+8/Qz4YgCO/lL/2uYDmjfYQVNdDcjlKJEjv2du0DU1JN/emxi3zZgVmdFJT7OgrVNVjkVGFNap6gK/SgY5OTmNWvfVrHcuL70sJDJd+P4Cx948BsCbGW9SXVXNnV13MHM2o8PsDti1tmP8+vHC9zZ3Mcd1sitG442EHY02ZpVUZDoS6h/Krba3eLLLM7WxiqIK7OND8bRoi0+kDzY52slj94OfvH6iJLOE98velzx3+lb6xHjGkOKdomFb0RjoGemxsGQhxRnFVJVWUZZbptFxrihXsHnUZuza2AklKf8JKosr+db2WzyHeDL9yHRJXCaTUVFYcV/+5mXJubhFHCOh7QjKLNT3d3w8xPzFtZIdPQ3V1RgVZVFm4YRSqcHF+ceoKKyguqpaON/K5DJ1R0uRNP9hYGbApG2TyLydKcwF/DnyT5IKxUoKhhaGLFItYvRfQ6+vr/BlDwWKcgUH5x/E0sOydk1THy5dXBi7dqyQ3CLX1SG893MEu7lTZFaESbXmnnlt6FpC00OxyrVi2OFhhBuF0//txudmrL2sidofRUVRhYQAnn0nm7OfnsU1wBWf0dIud4f2Dg2SPENWhSDTkeHU0QlrT+v7IjH+V/e9/0toarGvKagoqsC5s3ODcpJ2vnZac4d9zvTheofrjPxQSmrvPqo73dLVRNHf2/4uiRdmBWJeGsSxwSc4MegE3zp8WxtTKpRU6xlim6Ve2yb3//caAUCtiGNoaVjrn1ofEbsiOPvJWeZcnINzF2cNP0KVCsL7vgDAaIHwQEq2AWEtJ+CpjGLr+K20m9GO3u+JleIe49HiUY99j0fbx3hkUKlUJCUloVKpSGvZi5Bh76HTS8yoCx61iNt9X0QuYL/J5RDXfhwVJjZUlkhlsHq914vYTpPQqSpHWSldzHSb343C3Cpi/cZS1VXql3Nt5TVOLDzBnjl7iDrQNNmc7gu6837p+zh3kiamVCqo+kteTrR5tsiIwvvSekxzxdKjALf+uEWgLJC065pMmJgjMewasRKTvGSQydC/Z10Wezy2ttDX92Pt3QwiuHZzZeapmbQc1lIYr5/MEnURXfrhEld/vSr0s3INcCU7Iput47ZSkCDoFvsLu2fuZnWP1cJY/0/6M+3QtPsyqf4nCF0byvqB6ylIlJ53TZdiZVEld/ZodtqUZpWyrt86Ti86zdEF4i4hRYWCpKAkFOWK+2bVaUNmJlRqUY/zHe/L5J2ThV1VRtZGzDw9kwmbJwi9BbLCszj44kGh/OKjxp5n9tQmVyWQqf2TRL41ACEr1DJXoWtCOfzq4fv6/Ppj37+Fq79eJVAWyK0/bgnjaSFpeI3wYsreKU1OpDxMzDg+g+dvPC+MVRZVUpymZp3f3HxT+oJDh+h44FNSWg3k2vD3ketrjgk1xIByExsK7cXjmTb0+aAPi1SLGPKtmFRw9I2jbJu4TevxCxfCDz+oEzP3QoaMEpMSTvc9jXtfabEwdOg7rJ/2Dc+seQafY3V/D5yTiHWamgWtkqv4YcEPRM1rvNwlwJw+c/hk9yeMmiv1Pk2+lMymYZtY7LKYpIvi57ogoYCKwgoJc1OlVLHnmT0cee0I11ddpyhVs0OpPLsY04IMegX1Yua6mRSma7LBb229Rewvh3CIvYhTzHmqqx7cs9Pnoz549PPgwncXhHEbbxsKkwvJCJPKCTp1dGLM6jFYNLNg+M/SLoe0a2no3wmrTdbfC9+J6kyVZUaUcK5/WNAz0iM3JpfgX4OFZKTB3w7m1bhXmbRtkoT4oFKqUMp1SG41EOdnhmr9DIW+MV3nd9WYHxoz9lk1t2LoD0OFXfK2rWx59vKzWjv7qquqMbQwFEoNVhRVIMt7mxib8WRbXpEkO0puxeIarn6gysykJI5cZ7EXSWMx5+Ichi8Vd8J0X9CdiVsnCovk1RXVWGZGo1tZysmz0nVNacgdfINW4RhzHqMs6T10ZPAOgnoGYZNtQ/MMzY6YqpJKZNVqQsCYvWNQ0jRyQEOQyWQ4dXJi6JKhQia3QzsHyqbNIeLJRVh3kmYzzDzqimD3sqIV1Uqq9Krod7ofA09PQ79K8/3HHJ7Ku1+/S7fL3Zi9rAdFuxu/dq8pAlQWVQoT8BF/RlCQWEDIKrHkpZmzGee+OMdi18WaDHPUhIkNgzdw5uMzpF1LkyRYLb3tiPKJwjnVmf4n+5Maq/b5KrJrQfDID0loNwrXFBt6XOxBl9ORkk7hhtB7YW909HXIDpd22oD6uZ60Y5LW7mtjW2MWJC9g/AZxEalSpcf4i28x/KcRklixextiW6gVIlT3GCHWfw6jvaMp8m7a2P5y5Mu8V/gebt2l3o/WLa3xe8qPkwtPCqV9Qb022v/CfmGsaPFKPK/+gUmeVG7WqJkdMV2ncnDECbY8tQU9D83irm3SdVqEqrtSo7pOQ6Yrb/S6z9LDkhaDW2jtAI09pr6WoT8MRc9Er5YUUVM0L80p1SobZudnx7Otn2Vi5ETSi6VymLYzRrBrWhlV+lWUGZch061bQ1Tml9Ps9mGe2jwe22xbVs87Jjn+n6BGyaCmyF8fVl2s2DRtEzkuOSiaSGICdcfgjXU3+K3tbxx86aBGLCs8i7gTcVxZeoXFLovv7+S1oCaBe/foXWHc3s+eoUuGCvMEfwcDDydSvPtRZVAnAVefwDL74hxkT4wn30Fd4GlKp05hSiHH3j7GmU/OCL/v81+d5yuLr1jaYqngaPio+iMWqRYJ94fVldVYtbDC7yk/IbFSW6FPhGqxYMdDQVleGddXXefUB9q77szaexKxJ5rLP2kaJapUoNLRY8zeMbz5/Zu4xWnK9t1ados5q+ZgnWuN511PqpKbVvztu6gvC1IWCIl4NV3qnsPE5PUWg1ogk8uIPxMvjB9ZcIS9s/ey3H95kyV0a/Ao9r3Sc3hkH/2fQFOvvymvTzyfSMyhGCybWwrjx987zplPz2g9fsCpAbS91RZHUykpp5VtXVdbVbX0ubCPL6XnhZ61vKf691hxZil+p37CpNSYC90v8Kv3r428on+O5MvJbJ+8HX1TfUnjA6iVBez97FndfbVW6xjvi+s52iOQ8J2aHc6HDqkotGtBmHV/yvLKKMkQK6k9xqPHox77Hhf7HuO/AZkcpa4+MkGLq9oQvAK9sgKUVdJVnSItE+8rmzDPjqUsR6rF7NLVhVJzBzoc+ZrcHdIFWl5sHtd/ukDJxVC+/14zVppdyv7n9nP+y/P3JQ159vOzWpPr1Tn5ZDQPIKLns8K4U8w5zLPjsEkKlcTkOnK6vVa3Ib+X/a5npEdpepFa035/IPlhmgmgGmNyl64uTTYir5m8EoPEicnJOyYzeedkntz9pDD+etLrpF5NZUWnFcJ4jazYgRekOv53j90leFkw7Z5uJ/ToArVfobGdMfkJ+Y24mgeD/Ph8rv56lbiTcUJ2XO/3emPvp04envtM09/I0NIQuzZqppOtr63we60oqCDmcAx+0/wwdzF/YOcdHQ1z5sArrzT9WLmOHI++HvhN9RMm8pw7OdNpXicMrR6tRr8I/T/tj1tPaWIIIO5EHCu7rOTuMfHGfMSvI2qvqc2TTXt2HiXOfX6uwfiKTivYNHwTmTczJYzqR4mUqyn8MeYPobSYXFeOYTt1MsxIcA/qONpRYO+FUlcflY4u93ZADPh8AAV2LbnVr3EdBE1Bq/GtMLQwlBAx7kWeYH3fLMYZ3whfTvc7LeyMVerqo9BTYFJigmFFXaKksEKfQtsWxHSuG3v1ZI1//lQqFYGyQOZ1nUdyoXSuc2jnQIvB6qR8YZKUubz/hf2E7wjHZ4wPebGaF6ZSqQhdG8plxRUG3Bog6Xi0CfAirO8sAJzSnShNzNeIR++PJmPvFUzzU0hr2QtZE0gPqddSCZQFsnvmbmHc1seW/Ph8rf6Kk3dOJjc6l+PviqVe9Yz1MHEwEUpxDfxiILkvvo+uohzLtAhJB5PPaB9SvdQMcdFc/7BQWVJJ2yltmRc6D1NHqVeEsY0xlh6WQu88U2dzIno/R4G9F+lbTklkniuzCui8P5D2J37gZOuTWPUW+8Bow4VvLxC5N1KD5VoDPWM9dkzZIfSiArUfyLd23wrlb7MyM+h1tjU9LvRAr7xYkvisyszDpDCdIutm2CZdl8SzmnclMmAm4b3mcqdn47v6ADJvZ5IUlCSU6gG1rJm2tZihlSHXh7xFumfP2sJcfRi1dCHJdxBOd4NwCJUShsLahxHeJpwndj5B/z80i2p54Wk0D9sLQLpjOsPuvNmk6zr72Vl+8f2F0myp5KWZsxnPBT9HwKvaZfGOHFF7hB0SKCvdtutPcit1B9q9EnjlKSV88PkHmBWb0TLKlKKTNzTipU5qubQaXz+lUePHCxM7E16OeplZZ2cJ1RhqumIXJC8QHp+fkI+iTKFWO7hnb19dUU3K1RTaTm3LO3nvYN9Gs6isI1PPvW5JbvQ925fitGJUKnWXmG3iddzD6opSe8bskcp+/A1eDH+RUculRA6AHVN2cOzNY5i7SteYRalFLPNfxsUfxCSpyz9d5rO2W/nwEz2Cw6TfWXa3kZzrc453v3yXaRunacTyL0bQ8eDnDDr3GQ55Y/E2VI+HgbJAAmWBTbq+e2FgbkC7Ge1oOaylpNsF1DKzB186yLVlYlsCFTIssmLwDVolWZurVGCeGcNHn75DwMUAKu9JQpZYqIs3WW4dKLRv2eRE69NHnxZ2+gJM2T2F94rfI+C1APRN9NEx0KHd9Ha1ZIEl7kv4yuIrjrxxRHJsRrGatDJv2TwOTjkoiavPU8lAlRE9cxyoKKojqhjYmHCn+0wAjMqNqDQQFxTvFzNOzqDPh30wtpOu5xxNHWmW2IzZH87m2NKmFRkV5QqOvXOsdh3s1Emz+8TEwYQuL3cBkBCR/in0jPXo/2l/Rv4mtmEozSqlKKVIq0pEgzA0xD7hKg5xdR629YcE+3ZOyP3bodRTrxObcg8mXUjiwrcXOL3otLA7L/26uujTYU4TJVVR7+W/tfuWzw0/51v7byVxvzA/Ol/tTN/TfZl0QJPsV1VaxSe6n9B5fyCd9wdSeX1fkz//fmFoaYj/LH8Gfyt+Lrc+uYuNHRaTeLNAQg6orqzG/cZenNLV9167G5qFfP0yfSwKLMi0z+SzDz/DbJb2zvB7oVKqWNllJUHfBgnXTW7d3ZgbPFetXiFAcUYxUfujhOt6gKcOPMWoFaPo/2l/YYfS/xX8rxf76qMx30VTvi/rltb0+agPTh3EnX23t97m9Een+czgM/LipBvfNS+tIbFZIrLT0nGwOLaYJ3Y8wQeffkDYljBJfO/oHSx5dQmOaY4MP/mjBklL11iPHOe2ZNlmEekjlrH9J2iogCOTy9AzEstSg5pI2u/jfvjP9peQG1UqsIu/inmOWpo79WqqRlwRk0DnA5/S4eg3vBr7KsOWSJXtHuMx4HGx7zH+I2h28wCd9wdSnZMvialU4HltO+1PLKEkTuovIzc0QClT38qnPhSzrZS6BmS5dcCghZRhdm25epPnEH8V80xNnyG5rrzWd6P/p/3p9Y5Y+k8bTn1wij+n/yk0o1Vm5eAaeRLnqNNCBmZS6yFkeHQhs7mYZdv/k/68cOsF3s59G9tWthox9z7u2HdpVvv/Okaak4ixrTGO/o60mdKmyR1wKZdTCN8eLu6k+QtOHZ0wMDcQdlqKEgk1uLP7Tm2SrihDmjwKWRHCgRcOoNvFny6vi9vVK4oqWNl55d8WNx4kYk/Ekno1lWmHpmHrY9vga4cu0eyE0DfVZ/b52Txz7hmyI7Jru41qUJRaxNVfr9J1flehTNE/wbm/vqKUFHH8xxY/EigLFP7WymolhcmFQn8AUHu8XFt+rUHPkUeF1CupnHhXbGRu7WWNsZ0x8bdKKBdcWquxrZi0fRKDvx2M3zTt0q3/NYxbN44xq8fQdoq4O6X3++rn6fLSy8L4o0JaSBqJ5xOFY6SesR7nm00leNQiAna+JYnrBnTmbpcp6FaWYpKbhKJSM3tf0rE30d2m0SpoNS2vbJEc3xBurL/BZ4afEX0oWhi3a21HbkwuqcGpkpiiXr7+3q5rgDbXvBh1YJRWdTbj/BSsc6z5ef7P7Btbl6wrM3ckKuBp8h1bIa82wiPOg46pTfCBU0Fs81icrzpz+TfpfWDpbsn0I9NZpFokLEwoZervN3JvJMkXNYuFMrmML977gg1j17MlOFRSGFMqwTxP3b1zps8ZDDw154kh39d1UOY5tkKlanxirGaMurH+hjCuVCiZc2kOM07M0PoeQxYPofMLUjm1/IR87h69S+rVVKEfilxXDrq62MVfpeW1bZI58fa22zhHn+Vuhye0zvUPA3En41jVbRUZNzKERcqSzBLOfXGOXU/vkniXKZVgkRlFm7PLyNx+lqj9ml1TFWl1XRkHTx7kZob2tYII49aPo9sr3YSeXUqFEo/+Hlql92qSxCErpV1XOlYGrH96PV4xXvTbFyFJZJj07kTwyI+oMjDFNCdBmOiwSwimVZBaVaBJrOdziRx86aBwPKjBL76/8HtvqUSRTCbDOuWWmp2cLy3CpzgkcWTgSoIGGJLZWiotVoOrXa5yt6vm/WfkaEGKd1+OjtHFosgF11PS7tWGcOrDU2TfyRb6TgPc2XOHnU/tFJKvwneG0/7INzQL248iS5r8OXMGTPKSKLTxkEhDqfR1SHKt6/6It9Fcuxf4NKuNr5u5DkU/aUG7Idh42eDe2134bDi0c6DLi12EHRQAa/usJSMsgzkX50iKu+au5ryb/y4DPhPoNwO73lnLws8XEuUdxc8v/Yypt7pQblSUicfN/dglXSe2eQIff/wx1zteFxbjtSH7TjZZ4VlaJdjaz2xPu6fbCbt4VEoVVSVVXF91XUhgzAjLwDAxCqOCdM7slapbqFTQIv1Nor2iqXDVTBbrmBtTaNsc27KneXrdaGyXpN1zbMMP2vmvzhMoC+TYO+ICUMuhLZl2aBrOnUUKKyrM3cxZkCIu3Oq98CzhvZ4jyXeQZJ9VlpKL95VNgLqoXBSn2X1abONO8KhFlFo44nn1D+F+SBuSLyWzKmAVkXvFyUldQ10NaVy5jpz2M9vTrLd639fx2Y6Aety5F0nXkhj/53is86yxd5V2MCuiYvGM1OHLZF8G//QCedfqrkvHQJdiGw/i2w4gul1HBgZvavQ1NQbN+zen/yf9MXUQPK+x0Du4NynOKRi1kKpPVJVVceztY8L9tqJCwYVvLqBUKBm3bhw93tSU6jR3Ma99Ju+nePV36PNBH6EcK6h9my58e0FroaUhKCqVJLcaRLpnnSpRfc50WX4FqpzcWmWBpsxZrt1c6bWwF6OWjxJ22U/cOpFFqkUMXSzu7t80fBOru68WEkHqF4xKs6Tx3td70/dMX1rGtKRlhGbepqKoQqOjufov38x/A3pGeoz9fazk/qmBjp0VBXaehA1aQJ8PNFWTqquU2CWp/VwLbZtT4PGyRjz/yXwWv7GYInN1sfneDuiGUDM+58fmCyWx9c30MXc11+oNWZpVSvc3utNuulgxwb23O53mdqLPB30eF/v+P8GDLvbZeNlQnldO+A6xx+bzN55n2NJheA7xFNqEpDqk0v5Ge65/el0Siw+Jx++WH7rVuhSmSsfJHItY8q3yaRfWjm5n8zTGHH1TAzKbd8Mu245n1j7DxO0TG39RjcCu6bv4yesn4X7FpYsL3d/szumPTmtd+/tO8GXs6rES0qVKBc1uqVlwHjP70O/jfhpxuVldp6C2Lv7HeAx4XOx7jEcMMzP1ZtmwWC0ro7whTgpZZKm7bHTNpWw/fTsLlDrqiaPFEKkU0I4pO2hz+hcS2o/BtJs0QVk/QWeVrlmYMLQ0ZEHyAhapFtHngz5N1invOLcjqmoVmbelRUpcXci398IiK0aYCCmxciOp7QjKzKWGydVV1eybu4/40/FaJff8X+9PeovuhA14BeMWmu9h5mLGpB2TiN4fzfKOy5t0TW491B1R93q01SD+TDzbntjG+gHrJaaxKpWK3JhcXk9+nQ8VH0qOLU6vSxild5CyVPp81IfmHz3NwoUyArUQfrdP2g6oO2v+LXj082DC5glaza1PB54m82Ym/4+9twyM6ly7v38zE3d3F0JCIMHd3Z0CpVDaAvWeupdSd29pKVAoUNyluLuEkBDi7u6ZTEb2+2GahMneoUlLT8/z/lmfSq+Zydb7vnStGVtmNF2/22FmZ4ZTRydmbZ8l0j/KvZLLqWWnuPztZVIPpVJbfPdG9f/MkWtMbkh1t9cW1fKl95d8bP+xpHMnV8iZuXUmff5zd8Xu7wa6LerGuO/HSSaRbH1sqSuu49p7B3m302aRPT8qn32L92FmZyapQ9RWNK59/y0EjAig60OtJzGGvTeM7o92p7awVlJI+t9C7PpYHIIdmqaR/wrcUs4Qen61QYc6wOaXrhF4dQtWFbloTMR7y52QtDcJrUrbRIfcEl59vVgSvYTQaWJBEaVSwLi+GqOGOsliX/LQKrZP286ilf/hyg9XRPbQsysZdVjfJd6/4V3pA5RpmbpzKrrt4u+3BplcxrYZ29DJdFw6I130ld1hkuT3cb9zoY9+8qOliLpMJqPBtAFjtTFJBxtEwUlDRR2mtWWUOZqgsh+Hu1Owgd3SxRKf52eQ3HMuajMbdO2grwsYHsCS6CU8lfyUpH3HvB187vZ5q/v7uU/PoTBRSGqdlKeWNz0DJ986KbJXZFRQeC2bUq8I0rpOE00o1eTXoJUbIcjlknv9PwWHQAe6L+mOqY2piGoQ4OAzBzn++nFi1segrjWcWFFWqLAuzWz6d9/nDKfszQPcSez9ANunbafH1R5k7zSk5PqztS/tSBqbp26WLCBln88menU0tr7S/seAVwbQ+5ne+AzwEdlkxnIy/DIAMFbr0KgMz1sQAJmMtO4ziR7zimiyz+vWIRzy45Ah4JQV1a5ESIeJ+i7+0++elrTvnL+TkoQS0cQJ6P09uU5NuVtH1GbiZqkCdRI5Tmu5HnmEajcx7d+CNQuYsXUG17tdxyZygYHNzM2OSpdgelwNodZpCA2DpKe+WsOcvXMY/PZg7P3F05t1pXXsXribmxtvSifglRqM1Upcsq7RkC7uOLLPisa+MBGb0gxRUUthZ8K2GdsocC3gwNgDKEMM390Msx2semQVb7/9NiXOJeiEtvPXVWRW8OvwX7m++nqrhabMM5lknZVmt+jxWI8/bcwqSSwhfme8qFlK46IlyyeLGqsaSpxLkJko0Omg43l9EbjcrSOlgYsBMNbYtfmcQN/ssGnSJgpuSNOwhU4L5fS7pyWLZjZeNjyV/BTqOjXnPxXTHU/6eRJRY14j9PxqNIfEjVROlw8w+PQEqgN/JnDAZwY2y1BfUnrNpcbBB5WlA0bO+mfpTfWbvNHwxh33HIBjr+r/nhQdYO6VXN41fpfzn0tTNA9/fzjTNkzj/GfnKUsVU0dqtVBv40JhYH8Rnb2y0DDpWJUr9ptMq4vwvfk79oWJaBt0bfb7NPUaci/lSu7/oC/c7py/kx8jf6Q6r5raIj097NUfrwIw5qsxLBWWsujyItF3K3MqiYiJ4MzkM0z4Ufy+N5y/Sr8T5uxwyKN2UD4BYc0U54IggCCwa/SLKKr3EH5KzMDyd3D4hcMcfVV6gr44vpigm0F6SuKBYomIqJVRnP/0POc+PSeymViZMGPLDLo80KVVne3ytHKmb5rO8A+H/72TaIHa4lq2zd7WaoOsjbcNpjamfym2q76Rhv+N3RirmvfK2yWBvvX9HN1X3+KQpy/Qt4fG09bHluHvD6f74u5/SRYj5WAKORdzKIwVN5C4dnZlwooJOIU68WiMmKbf710/fpv7G2sXrOXaQj8Dm7m9OQvPLmTlwyv5+ZGfOTtVmpb430C354aQ3Fuvy6hsseUpzIy4Ofhxyjw6UeTXC5WVYfPAF6O+4Pm+z2OiMiEoOYiGrLY3B8iN5HSe15nEPYlUZoubLRqqG/jc7XNJxiTQN1pf+uqfb/T8b8e9LfF/pdh3/M3jLJMta2K6ultor2Zfe69X0r4kUg9JsyKZWpvS+6nezNk7R6KwJRA3L477vrmP8avEE9BpvmmsfHgln7z4CWGPizU+LWsssa2w5UbEDVK6zzRgmxIEqLNxI6azfiLQI6/9dMl3gqWrJQ5BDq1O78kVcj1bkoQbc+z1Y3xg9YGk/I8gQFKfB4gbtITAR4aKGJeM3J0p8O+DTm7EO7NvkXryf08y5x6a8W+uffeKfffwr0GhUBAYGIhCoSC96zRyOo5AHiQu1gkCZIaPI6nX/Zi6iBM8MhkUBA2k0K+XKOkE4BjiSK2deKKvEa8rXyd2yBMk9ZpLfpB4Wkyj0nDxlIo1nxZTnS/dvdwa+vynD5N/mSypXSeYmJHbcThpXafdcdoNxBuuoBWI2xzHvtcu8sWE46KNoiShhKKr2ZR4d6XBQpyEOfz8Yb4N+pb04+nYBYj12O4E507OPB73ON0WdZO0n3r7FPlR+Yz7fpx4Q9cJfBv8LYefPyxJNdHj0R5cnbCUqxOWcksdLLK7dnblXEEAngnHUK0Sd5QW3CjA1MaUsJlhBI8Vf/+fgkOgAx49PMi+kC2aggAwMjXCpbML/kP9RYmL6rxqvg74mk+dP2Xfkn2i7mGHoOb7c3LpSUl6sr+KP3Pk7j9wP0uFpZJJbhNLEzx66p0mK3dxF27y78lsnbmV4lvFd+VY7ya2ztpKzsUcySSSqbUpurl6mimTtASRfeusrZSnlbN30V72PLLnL/3929e+/xZqi2q5/N3lO06WePf1JmhMEArT/95x/Rl6PN6jSfOyJRpqG/CN2UvwpQ1krD0psquj4/COO0ilawjZYaOQt0hY+MXuw74gHp1MTkbk5HYd1/jl45mydgqDl0prnu5bvI/fn/wdC0dxEVFZKxBx9At8bh5A6hFocDMjwy8DxxI7yYJHYqQ30ZFX6BzTGb/U5uYBk9pyPBOOYVmeg4COw6MOYzymfV3qdZZ1vLP0HaoeFXdPph5J5SO7jzj47EGDxoxG/HjtRyzq9Ofbkm5Y0Al45HrQ90JfBm6qEWn+VcRm4556DmOdLc5VY3E2NQzI6krqMA30wj4/joijX9AgoaV1J7hFuBmspbfDu783Hj08KEsWJ3tBTyt5Y430VKBHDw9GfDwCcwdzSerqS99cIvTcalQWdpR5dhYlzbIvZKPQafCOv7vaR6DXWTnwlJimDfSTp+aO5myeulnUlAMQPCGY3s/05tXqV7HxNryXdSV1uGRepdC/N9fGvk6pusWkjqUZ1c4BFLgV4JfhR316c0GjLWuf3xA/+j7fV7KZwtLVksgHI3EOc5b8ro2XDWO+GoPfED+RTavRYqQxYu38tVwe7CGiQNSUVWFdkoGiQVyYArCs0E8cyQQBr1uH25UIsfG0oddTvQifKz1ZbeNtg/9wf0kqHmWZEu/4o6hNrVFZip9h2dViFq1YhFuei2Qy1wgLBJnwx7EbPn+CACb11TjkxWFWfpIbQe1jROgwoQNDlg4RFbEBagtrqS+vx/v+AVgGinVYuszrQvSI50nqNRercLE2qV12XNN/t9TOczb3wK32RX587Ecu976MWmd4zxzjcpm+bTq2FbYMOjUIWZzYL2sNGqWG9OPp7Hl4j+Q6F70mmjWD1khOYYK+4GwfaM+ZD8+gURlOyTXUNhC7NZ71D59iy7QtonfPYqoD6x9Yj8ZIg1W1FQ1KtcFzVuHagZ5HD/H222/z+HcL0TW0XbssbKY+SdaaNrSJpQmRD0bi1VusG9mI6ZumM/HniZI2QSYnt8MQZGHiBhfr9Bjsi7Iw0dqj0Bk2KAoCCOiI8r+f7dO2Yb5AP40uN5K3OolyOwbvGky3g90IfFqsR2Vma4a1hzVRK6Kkmy6B9aPXc/HLi8SsF9ODaS9dxbZQerrOJsyLayMe5rPnP+OLZ79AEWaoyWOfF0fnU8sB2DthL+b2lm32+/yG+GHhbIFGKX1/Dzx5gJh1+kYMnVaHkbn+vW5Lh7/DEAfee/09MnpkSNqNhw/k0jAf3u5SzMQpUdj6N685Venl9Nj/Dr3Pe3Ct+wXSO93dOOvC5xc499E5ShLFBRzPkZ5889Q3pASloBXEDSodp3Rk8prJTVONt0OukNNpZie6PtSVFd1XcPGriwb24lvFrOi2gu2zt3P528t374TQ35O4zXHsuH+HpL2JBrwViuc7QW5rjYAM+7zmtfL2Yl/YXH3TgdJav1/ezWJHflQ+60auY8c86fN69MajTFw5EedQ6b26+6LuPHHrCVw7i5ucTAJNyPfIR2OsocTccJJYYaLAp78PSttgcr1ycVZI06P+EyhJLGmiF5ZqBJHLQa5V45Bzg/Rzhg0sMpmMemtnKh2ckKnLQGdYzFMkKwi/EY5zsTPzNsyj7kDb9ywA/2H+DHx9oCRdceZpfXNWVa709OjIT0cy7odxkoVCgPct3meZbBmrB6z+U2mC1vBvxL0t8X+l2HfuY33DQks5hEbkXMxplankTvgni33Z57P1E9LvS7MWFN0sklzXQR8fbgzYyA/LfiDTJ1P8ARvI8c7ByMEIRwtxPnXi3ok8/c3TFLoVsn5iJ7QmzftDTWEtQVc30iW2CwWuBXzzjLTG6F9F0JggvPp6Se69lVmV1BbVMnX9VElNVnN3e9S1arbN3ytZ8Kt28seisoCUn45RlSN+d3M6jeb66JexPPU7+99ue2PvPfx38W+vffeKfffwr0Gn01FQUIBOp0NtZk1BUH/k3tJFuWK/nlS5BEnadBVVeCQexzXjsmQXzNBlQ0mPnIr/9R1UnxMHc8pSJZ1Pfo9tUQoNFnaGtnIl75u9z6EhH5H50g9sf+5su87R0sWS0Omhkpo4Oo0OlYU95e5hktN5HS6uo8e+Zfje2CPacBWmCvr+/iZ5nj2o3n9GtAkU3Swi+ovjeN3ciEfcGuqyDTtOPXs1X+dr1kPadU4N1Q3UFtci6KS9gEFvDWLWjln0fLynKNEtk8no/0p/HDs4NjmfUuh08nv8YqV5+BUKMK0rx7xa3C34U+RPJOxMYOaWme04o7uDhF0JbJ2xVURpBvrkj88AHz51/pTfJvxmYBMEgYr0CgDMHcXPgXOoM08mPQkyCJsRJuou/jv4O46vqY0piy4vYqmwFJ/+4ikKK1crQqeFtjqB8W/Cu683jh3EDiNA+ol0ZEcOU2vrTmo3Md3D7e+OlEboF15fsGXGljv+/dvXvv8WNozbwO9P/U7qYemuu0ZNs0mrJ2Hna9fqZ/6uhk57ETIpBAsnC8mgR1OvwTkrCtviFDLWiddmXUoarumXqHbwpTCgL3Jjw0R3o6ZOao9Z7T4uCycLIuZHiDSXmiDTJ6ilKNHq66HewoEGMxu04pwVANU21ey535qhy8Q0nJumPExCaAIT9k3A6Upz4G2qrMA95SwWlfkIcg1x4XHEh0hTqkhBp9Exbv84IqKlp1LkCjmqShWXvrok0mkD6JgQhmuhKwkhCaIkt1atZfHPi+mY0JH43iaiwpu5rwvFnn5YlxfT9cQ2ii8a/v7qGQdIfvwrnHJuUGvjBrK2u681hTXcWHej1caD3k/1RlWtapoKb4mFpxdSklDCnkXi4r6pjSmmNqYoy5SSk4Ehk0LI6zwSrbG0duIzac+Q2nU6pnUV+EXvbvM5tQUXv7zIle+uSE7uAfj096HPs30wsxUfW5f7uzDmqzGYWJmImiLMXaxI7PMARb49kem0FGQbFmIErQ6HnBsEpgby/uvvYzGvuXjQlrVv5/ydelpAiaSVU4gTtn62kvR0oC+ubrtvGzqt+Pdrcqt57cPXCEoJIs/XHFmLhqP6G4mEXFyLU/Z1bAuT0LaYHk3s9yDFPt30upg957R7/xz7zVj6PS9N/zX8/eHMPypNI2tiZUJ22CjK3cVFFID82kQsay2ZtjkYv3O/ieybHtjD9hnbmbB3Ai47DSecKhLycU3TT+NWWmeRZCz9DrQGnUaHWqmW9Aft/O2weOYR9hX05N13paezNGZWVLkEY+Yk7nrN6DIRpZU+UZwfZZhgNK+2oNOlBPqd68eMrTMwvWG4F7vk2dD5Zmee/epZhp0YhuKmNN24FJw6OjH5l8mEzw6/o35ta40eAPHb4zn+2nHR+l9TUMOOWVvITFZRP2GayD/S/fFQdb/WnRc+f4HSG8XodKA20ReSnLKuU22rb+CrN2v7OQG4d3VnwGsDiFwYKWk/9pp+Qq7zXDE9eX1lPddWXEMml4koVUEfc9iUppMfPBCho/g5TZ/6LMdH12KW8xpVV7YZ2OqSc/G6dZgG3WHyHH/jQNVHgL6oevTVo62uX414s+xNJl2cxL6kfSKbYwdHhn0wjNKkUslmjrSjaU0FNakGQeHQYYKvbCL0zM+imEVmbITGWI5jqSM6uQ6dkeEzbqJsTt5FR0aj1bbP73vw5INM/kW6Can74u6M+XoMTyU/ha23bdNzKmj0z8/OB3ayacomss+Lu/3VqNEYa3A/6c6J5WLpC7mHO0YWD3E0ZQlx2fMN1jmFuTFl7mHkuacQFx7HiSHb23QubcXIz0ZiYm0iWeRMVibTYNLA+P3jRdPioGebiVwQKRkjCTqBupK6piJfy+tye6K2tenrvwr7AHvcIt0k3xvQv5cPHH6A4HHtL5waeboiQ8C2pNlfur3Y5/bweK5OWEqtvb4prD0hR8KuhCafXyrRfPLtk6QdTePWVmkf07WLK90e7iaZ/8i7msemKZtYP3o9sRvFE49F5UUYqY1wKnZCVWKopykIAol7Ehm7JwT/NH9Mq/46w0p7cXuzltQzmrL9Bt43DxAQvYtbmwzPS6vRYV5ViP/NUwTeOIJjC33cmPUxZL6UidpYze5Ju2no1vbz0qg0lMSX4DfUT7LB0M7PjsiFka3SR1dlV7H3kb0k75eWJQidGopDsAPlqeWtNiD8Gf6NuPf/Kh46+xATVkyQbFoDWNV3Vasa5G3F3S58qqpUFMcXt9pwsmHcBr7v+D2HnjskalqXyWRkDMwgLSANjU78fGk1WsyUZkyNm0ruaTELRELHBC70bdYTvp3JQaPUYFmeTGKHRDbN3oSr5d1lUEk5mMLJpSeprxT7YyUJJRx69pBI1qIRp6q7kRsylJxTqSL2i8b7439jN5nrz3LhC0O9ZE1+MT6x+7GsyCWl+0ycJ/z3ZCDuoX34t9e+e8W+e/jXIAgCBQUFes2EohRc0i9JUnMJAtgWJhF0ZaOkZp+uphb5Hwt77AZpmgy5Vo1jbiyqdMOpFkEQKIjWU9q4ZlxGpjXcZFSVhpuWSai4c/RO+NTlU3Y/KJ3AE1JT6XbwQxxyxAVIgFpbfSd0vZWjaFOWyWTokFPi3ZWbQ57ANcKwa9p/uD8uA/2xKy3HIz2T6jzDJKfvYH0XdW7IUApsxLRPd0La0TTWDllL0l5xUQv0mguhU6WTUjK5jBEfjqD4VjFrBq8R2ctSyrArSECm0yIYiTvFt0zfgstP75HWbQYxI6Q1NgBW9l7JyWUn23Q+dwM31t3g6Et66pnWNOquLtfT6ziFGk7u2XjasPjaYro80IXSxFL2PWaYsBB0AjKZPsHiPcD7jsmn9uLPnL0zH57hHcU7Te9Ie+DRw4OMUxnE/Cr9fP+bcO/hTuyGWNRK8XSQuk6NrK6OvODBlHuIaX+n/zad/i/3x87Pjim/ThHZq3Or/1Sn8Pa1778Ft0j9GtGaJkMjVvZa+d84nDaj4HoB22dvJ+NkhshmZmfO9VEvcnPIE/RfL6YCUowZyY3hzzYVWlpe7uRec4kZ9gwmyiocs8U6AXdCdV41+dfzRUWtRvR8vCf5UflcW3FNZKspb8CsrgyFul7yHZywaThvLXsbE/Wdg5Its7ZwfXBz0qrW3ovYIU9S5tkZt/KpAOi4c6L0dui0Onpd6cXIIyNRZIk70PyH+fN02tOM+34cnj3FjTn9LozHrdCNjokdRcUYuULOkRFHODriKDHhRtj4GibkTN3syQvQTxCrTFSorQ2d4quVzYmwC/2ikZu0fR3Mu5rHrvm7+KHTD61+ptdTvejxuLSmjlNHJ31SXuJeNWrITVw5UZLa2m+IH6Ud+uGYsY8uRz4TUXUdf/M4gde3U2vrjtL6znqv7UX/l/vrKcskjjv7fDaXv71MyKQQLF0sxR9A35V65YcroiKpkZkxKgt7Op/8jm6HPiLpO8NJoeroVAKidzHm0Bi8s70NCtptWftCp4biO0g86dWIW1tuEbclTtJ2aUsWcVviOPi6uPgvNzfieuR18jzyEGRaUeLTLMSHzE5jsStIIPjKRkl/tNbOkzobF2ocfduVOL383WWWyZaRcSqj1c+c/+w85z8T0w2aWJpQZ+OGa9oFLMvFSYOCntV89exXJHVIQmMibhiqVeiTI9bV1tTVGO5Nmjo1ZjUlHBh7AFOVKaPW+LX9pIB3jd/lA4sPJBvtjM2NuXDLFlNlBcmx4kRI8a1iHLOvY1RfY5CgbkSNtYpLg0zICRkoKoqpqlR0iQkiMDWQ8LhwTAoM17lbI8y4FapPQq9YtALNyPY1HUU+GMn0jdMlG/EiH4wkYkEE5z87L1nk3HG/Xqd78bXFoulUSxdL0iKnkhsylJt0Fv2+ySU5I46MoNSxlEu9LmHsYIogQNzgx8joMhHr8mxKHVJ4e+nb/PTochTtpNcb/v7wVrXDCm8Uturr1RbWsm/JPhJ2JkgW0s9/ep4Ol9Yj04nfK9Drpt8IepLABB2qFMNiiiqrEPe0S9hV2NHzck9czuofht0Ld3Puo3Oiqc7bIQgCF9Iu0P9sf+L3SftdIZNCeDr1aQJGipljGqcjBrw2QJJCVzN7HrkdhmDUUCfa6zVKNf43T7FwzUKe/fJZFA2G96IwsB8HZ4WzYtEKfDN9Uddr2uz3VeVUkXctr9XPdprVCbf5bk1JTRMrE16pfIVxP4wDIG5rHIm7E9m7SNw0qavU4ZnjycgjI7n5s1h/MVazg3Md+2FaeZjS3TFEHWuedLNwtSat+0xiu+hj7Tz7Oze2tRf9nu/Hq1WvNvmrt0Ndq8ax1JGImAjKY8XNX5XZlex5ZI+kzqGyXMmnzp+SuDuR8NnhTFo1ycDu1ceLF4r0xRep5+TvQCaTseT6EhZfWyxpv/rjVb72//ovUfZptXqKujprlyYHt7E3R65V8/67hi9je0IOY0t9DG5ibSLJ9tHrqV5M/Hkirytfl/x+7pVcShKkJ3mqcqtI3J1I6uFUdswVTwYWLSriqW+fYuLeiYzf3NPAVp1XzabJmwhKcWPBrwsIk2iw/adg52dHvxf1DTtSrBsZB27hnB1Nco/ZeE0yZNXQ1KnpdPrHpn8XBhpem4y+GWycvZEyhzLSQhTMHv1Em49LXafm0LOHiNss7Re5hLswefXkVv0qM3szBrw6oFX7tA3TeCrpKZ7Pf15SiqQt+DfiXvEx/Gt/ul3w7OVJ90XdJbUyBUHAPtjxT6nCpdDe82/P54PGBDHv4DzsA8VsYqCPR2y8bLj45UWU5YaFLZVOxZrha4iMjiR6RLTouzXHanjl41fwX+/PzY3iPSu6azRHRx5l4OmBvPTxS5SlNDf22PjYsmLRT4QkhTB+/3iCU+7uNLqRmRE9Hu8hqTPr0dOD8T+Opzy9XFKu6cYNKAjsz7Wxr4tkgAQBuu1vlulo2QyiLavEJfMqZrWlVLiHYRXSOoPdPfy7+LfXvnvFvnv4n4BX/FF84g6ii4oW2XQ6geArG7ErTEJTLaY1MPVyaep6zTiRIbJHr4nGOesq18a+juOcUYZGAX4b39wJ7R9vyGdu42XDs9nPEj3qRa6OexP7nu3bJMzszIjfEU/mGYkpNis9NZZ/zB5Jar3c0JFcnbCUwsD+og1Xp9VRdTMTo4Y66q2cuH7TsDBmbm9O6NuDiQ2PZdVDqzCVoIb0fechTOvKCbqysV3n1Dh91loHGMC1n6/xfdj3rdLmdH2oK2O+GSNa+BL3JhJ0dTPpkVPJ6yKms/Lo5YE6oHXtrmcynqH3M73JvZwr0kL5J2FqY9qkKSOlFZR2LA07PzuCxwVLdta5d3Nn3PfjCJkcItrwM89k8m3wt7h3c6fLvC6SIsD/FM5+cBZBJ5B3Tfx81pXU8U3QN3xo/SE5l6S7lly7uBK9Jprfn/79nz7U9kHQF56lEnUBIwKgtpbgq5twyBU3D9SV1GHhbMHU9VMlNWL6vtCXMd+In91/G5NWTmKpsLRVcfWlwlJCp4VSlVMlSSfR+JmlwtJ/8jBFuPDFBYwtjPEf5i+yaXUytCYW1Fs5YRsgUSgxM0NtboNbylnCTv+IstxwTRBkcjQmFrimXcA1vX3UTYeeO8SKbivYvVC6mcPKzYpui7vh3ElMYaTBmPSIyZR6R0oGUw0+1iQHJyJU/yCaaBEEgUUrFjH86HBSg1K57t2czNMpjFFZOeqLm4Kc+Wvn47FCmppSCgoTBd8++S1WtVZY7pYuANn729Pz8Z6Sk7GX+zcnLyxdDb8vN5JzbsA5kkKSEGQ6ClsMZut0YNygL4TtnrybbBfDYmGpVwTF7vac73ues31+bFehxS3SDQtnC7otlqaevvrjVQqiC1pNgteV1PHA4QeYtHKSyJZ+PJ3vO37P3kf2tlpMrDK7SYLP5xQ5p4i1x4wV1Nh5kdRnPoWB/dt+Um3AiI9G8FjsY5L0YHUldWSeyWyVFvrMB2f4JvAbDjxxQOS/CALINc0JeMdIw+SPkX3zlJZjqSOqrPbtxf7D/cm5kCNJOVUQXYB9oD0jPx0p+d0jdtPRyRVcOiDWzzK2M2X3lN24FroyfmOuiKLRzMeVYv9e5HYcRnrEZATB8F5ZlmXjdeswLn+sFe2J2xoTII1NPy0R+1ssR1480qoGqLGqBpuSdIxU4vulQH9/N83ZRGa3qSL7wKtTCL0Vysa5G9n0iKGemn2EDzeHPEmBWwG1lrWUO0qv/a3Bs7c+wWBqK94LdVodDjkxdDz/CxaVYuqvtGNp+N/YQ+TRzyk/EQ3ApW8vNRVEE9wWcjX0WY4O24Wtj53BdwVnBR++8iEb52xk2VvLqBzRgjJeLrDlvi28vfRtjI0nMyzooTafk6pKRcKuhFYT1QD15fVNFIotkXslF1WVCvdu7iIdF1NrU8q8uqC0dcdSYomVX9My4NwAcrxyyAmbgluQH4IA/tE78YvZS521KykB0ZL6L3+G0++d5kvvL1ularv/4P24dHbh+i/ixhdrT2vmHZrHhS8u8EOYeJ2LXBhJZufxhFz4FU6cFNlNSvOxL7Pl+ye+Z/1Mw4Yi677hXB++hEzfTHpd7oXLef3z3Oe5PgSMDMDYXNz41wQB3nzvTUYeHYnJKfEkTHl6OSfePEFlVqUkLfCAVwew4MQChr4zVFJaQOvpQ36HwcQOf4bAkYbNnlVx2bhk6wsNCp0ClyRpusKxv49l/rr51Ja3XQaiILqAXfN3tRpn7U3cS9+X+7L0+aUoy5VU51bzc8+fufydfm16ve51pm+cLjl96pDswKKVi7jc6zI9vhXveU6/pfDoN0PQ1JuiLbZFWdOcmL19zRt5eCQPrWr7e9UW5Eflt6opWXiokIVrFrJ1xlYCXxY33l786iLXV11v0i28HUamRjiG6H2WgFEBkr67sbkxvZ/pTdBYaRahv4qG2gYyz2S26ldbe1hjYm3yl2JWVWYBFlUFWFQXIWuhSxp+/Ft6HHiXHvuWYVuof07b4zcFjgxkqbCUV6texdJZvFgFjgyk2yPdRD5NI1b2Wsn3od9LskCETAph8Nv6Z1MqZi4ILyA+NJ4rPa9wdaBhMa/lenAt4u5qza3ovoKNE1vPi4TNCGPS6kmSE3Q9lk4gdsiTVLqFYOZjGMvLjRWUeEUCcK7fOZIGG/q4hzWnSeyYiMZYg0JniYy2U76ZWJnQ9eGupB5KbfU52zpzK+c+EetZApx48wSXvr7UKj36/1/wf6XYV5ZSRtyWOEn/fOd2gfLk0r9N49mWtaC912v73O3seUha3qTXE714LPYx/pP1H+wDDAuCKo1+sCLHKwfzHuIGK7mbnNjwWA6PPEzHxzuK7D1L9NSctZa1FLoWIjNuXpN0OqizqCMpOInglGDGbLy7+Zlb226RfTZbMs4ytzdHrpBz7qNzlCaKYxKL7AQ8E44jE7SS62ila4heomrni/rc1G0w7ehP9MgXKPXUMzFINczdwz0A3L0RkXu4h7+B/OCBBEZtQ2iQ6OAUQGnlTKVLIP26+InMChMFtfaeaI1MefnaOJH91tZbuKamkxcyDFnLtVgGY74ew3e/2mBXmEi9o6HTKTeSY+Nlg+aPGLG9dLuTV0/mzPtnJHUnBFdX0rpNxzX98p9qZLXccDVKDfEvrsHTPYysTmOpKDWH2xxDbYMWjOTsnLoTnUKHZQuO64tfXSTvu+s4FVU0/b6EfJkkPHp40POJnqKNpxEbJ20kaW+S3mlsWaTU6Pht/G/4DfVjwCsDRN8NHhvMj+us9Lo0Eo7IgJcHsKsYLC/mYKKsAgyFeu187Si4XoCFswVjvvzvFVw6Tu7I5F8ms2vBLpHmHuifwYqMChZdXSTSbKqvrCd2Qyw3N97Eq58XPR8z7GK0dtcnTqN+jiLq5ygWXV0kyf39V/Bnjtwjlx+h8EahZKerWqmmPFXfXVscVyzSeSmMKcQ5zJmMExmUJomdnH8LOq0OQScw6M1BkskfuZEc/HwhI5OA6zsAQ0qrC19e4OwHZ7Hzs2P0l6PpOMXQ8Rz1aYuGgv8hCIKAoBMkk1qg1y6L3xFPdV51q0XB/za0Ki0ePTyw87MT2VRKLeZVJQgyOQ2lZoAhFZxQXYNpjQq5Vo1c0yCa1Ik8/CmyPxapm4Mfb9dxhUwOIW5zHG5dxR3ooJ8m8+rjRcBwiXVSocA+Px7zmiIq0+ZBiOH6rB7ckb1he3nh8xe47nndkP5JAMtaS0xV+mSVja7592VaDUYNdWiNzZChQGWqQivNHikJmUxGhV0F+8bvo0//PiJ7aVIpSfuSCB4fjFOIeJ2rsbVk+WOb6JT1JG92laasCk4KZvhRFSVTsvH0bC4SVUSlERAbxf5x+0kOTpbsgDszygrT4nLu23QfqlkNQNtojmw8bXix6MVW7enH0rm17RaTfhYX8wC+C/kO1whXFhxfILJZe1rj2MGR0qRSuswXT82e++QcnQ6dYMXCG1zod4GfHX80sF9bcQ2rWjUdLq4jfuCiNp1PW5FzMQdkSGpwhUwKYfLqycSsj8Gjpwf2/obBd/GtYnQaHQ8cnY9zmOG9Lk0qpdPpH9EYm5PZeRzTJxvq0Jn7u3F5wutc8nPjqe+eotquGMRyhq2iPK2c9BPpqGvF/mBldiVJe5MInS7NHoBMTtS4N/CUaHJVyIywruuCxsSOemtPhBZOT2OwXOPoR40jooKKT9zvGP2RlOh64H0EQXqiQQrBY4OxcrOSpEODZlaKB448ILLVFNbglnqe3JChVLqJm52sCs2IjI/8470R//aQ4wMotAskPiweC50h7bAggGVFLg8dfoiz/c9ycZhYp/ZOeOTiI63aCqIL8I4/QoOplaTWYMjEEFZ9V49tcQqmTtaoqlUcfPogEQsi8Bvsx9BDjnjkPclHr36EWv0NJre97um1yajMmpk3dDJDh9GiHLxLvcn1zMWkXoFGqQbuUDS6DRWZFWyeuhmAF0teFCV186Py8e7vzajPR4mKeQBPJT2Fpl5DfWU9ptamkkkc5/TLeKaeoeDG/bjdxsxRPLOKtZ1WUm9Wj4nGGXO5NTod2Bbr6bfTu07lUp83CEwJJCh9NLpeWmhjUtjUxpSqnCqOv3acqevERWFtg5aYdTEYmRvRdaHhVIqJpQmBowLp9VQvdGqxc+4z2IfUkAo8ksoRqqvF9qOrGZs6lg3zxFrbMlMTVJbWqE3UbJyzkUGWSwAY/fnoNp1XXFgcOrkOm1li7fPqvGoufX0JZPou+5Y+n3OoM3sX7SX7XDYTV06k28OGDSE6raCPYyQCJBNnW7I7dOZG8IfYVTjQ18PQhzCtKaXPpTLsir0pdTHF1MoCCTIJSXj00Pv45z45R++nxfRct169xZKd+utU/XA1ptamlCaVUpWtX19kchnhs6X1QZ07OXN+/Hlu+N7AKkg8jaB2syZffZ2gLnHUhsbh2L/5Ha8trMYn9hQdrDtgrjTHpurOevPtxYruKwB4sfhFLJwM3zvLAEsu9LlAsXMxRgpx+sot0g2njk6SDZUmViY8mfAkGpWGtCNpFN0s0k+9N55XcS1f+3/dtOeE3xeOtYeYWvivoDytnDWD1uAz0IeFpxeK7JYuljRUN0hq5/4ZNCXi7zTuAXJtsyacSZ3+c3ebRaw4vpjaolr8BvuJbIGjAkk9nCrJRiOTyeixpAfBY4NFbDcAJdPLuJinn3KXC4aTmuYO5iwVlmLxmi9K0yw868benZP5AzbeNndkO0jck0jwuGDJqStTZ2tUf7xSLSn65SZGZERMYvWcuWgVWjwxjPWTEuWgAPsyexas7cNN4/P4fyhN+90SCmMF9gH2qKpUkow1BdEF3Np2i4yTGfR/SdxUFjYzrFWaWYDfn/4dhakC967u+A/zl6Rm/b+A/yvFvtX9V1NbVMuMzTPoNMuQYeiXtXKs+8zHvLoIjUojyilpNFBbC7Z3IYRvz/WqyqnCqaMT3v1bn/w0szOTpOdvaGhg+rbpZPlk4b5M/xxWZFaQfjydrgu7YhRmxPYZespopbtYPiX4cj0BVfexefZmorpH8aXXl002VXUDvpmd6ZCsZzEb/HXr9Ot/BbO2zaKhtgGdVifKrQg6gcDRgTx46kFcOoslPywL0nBKu0KNgzeqan+DJhRBaJYYMZa4lzIjBRpTSxB0RB76mAxVRxgvTft9D/9v414d+B7+NchkMhwcHJDJZJR7dOLqhKUo+vYSf04uI27I4+SESQd+glpNvYUD9vnxBqPbjRj/83iuDR+HZVk2DYWG1B8ymYxuj3TDOSsKpY0rZb6GAa62QcvWmVvpfOwrOh/7ivztYoqlO6HjlI4surIIrz7iZJtOB+Ue4ST0f0hSoNop8yo99i3DISdGrNlnosD30THIdRoij35ObaLhZNWt7bc4NXg14TfDsa2wR9eCfqYkvgT1H4W+lO6z2rWhWzhaMO67caIiRyNsfWwJmRzC43GPGwRToN/4si9kS94n0NOllXl2xj4/DqtsaUoeuRxcU88TGCXWlvmp609oVJpW6VL+DpbJ76xX1nluZ14seZHAUeKO0/4v9afrw1351OlTYtYb0lrWldRx4IkDZJ3N4vwn4ufLqaMTA17TF0YdQxxFAfDfwZ/dd+dQZ8Jnh0t2MNp62/K68nXC54Sz5+E9Il2V8rRyrnx/Bb+hfvR6Svxe/1sQtALnPj4nSfUDehpOmbf+fS327iqyN+pTVGRUcPYjMVXcpW8utfrbjbh97ftv4dyn53hH/k6TLs/tEHQCy2TLOLXsFE+nPi1JoQR6WteWvPH/NBaeWciDpx6UtNUUKel0+kfCT/3AkTFfiApEmkPH6HzyOwqC+nNz2NOY2Bp2DDYW+iqdA6m3bl9Ha+c5nVkqLKX/i9LTWBc+v8Cpt09J2gQB7IqSMK2rIO+UuHM/SrOOerN6tszcQteHDJ9BmVzGxXGTiepnynOfP8eITc3Jf6vyLCKOfYl9bhT5DlvYPHszafPafk6CTuBxz8dJDEnEoY84OV9wo4DDzx/m+47fE79DvD57lTyA1syTFN9YURJcrVTz5A9vMf732dhUWfyRfL/9mgiY1dUx/sB4XvnoFdTphgljt+QzDN1XSP9z/QlNCNU3tNwlTF4zmY5TOrJ9rrT+UORDkdSV1JGwS1wIcenkQv9X9M9A6DRxAUqtVGNar08wW6jE06lD3xmKxsgUy8o8HLOj/8ZZiLGq7ypW9VnV6rUqTysn9XCqpMZG2BvTuDphKc985Y+xvWHSU2FuQoVbCFmdxlDuES7ZBCVDQY1VDXsn7MV+cLP/05a1T1muZOSnIyWnR4PHBfNozKOtNrxo1KfJNX2AYp34+bSotmDBhrfxLX+HpL4LMHc0TOgpL8XQ6eQPWFToJ9lb7o95wc1Jggq30HYnjp7JeIYFJ8QFY4Dxy8fzeNzjkglmQScg16iQS2iZADgkWTBl9xRGHR6FQ1a0yF42YhaJXUMJSA3AN8Hwva4vqcGiSj9mG34znL80MtYKzB3MKfGKILPLRBrMxdkKW1878jsMJqH/wzj0DERhomD+sfkMeFXv78SHxlNlU8UjPz9F1gVDnRaNUoNbvhsOpQ74ZvjiXmH4bgVdFHh49cM4lThx/2oncjecbPNx23rbYuWuT2ZKvRupR1I5+vJR6ivqJZ/j1FRYOeswH9t9LOrML0sto8uRz3HOiqLByl6UqOvoG0m5Q7menvTCOUrjiwyeM6/4IxjrbAi/GU7fcy7oGtqun9T76d7Y+du1OtlXeKOQOfvmMO5bcdNkI0Z9OooxX4kb6V7Y8z6nw7uycWEhuvETRfaSzkOJjozGodQB2wrDZ0FbV49pbTlGaiPKHMtQObb9GZTJZWydtZXtM7ajdBEnAT17ejLmmzFc+upSq9ID2ef0VNhalXidNP/ifTqdXo5j9nURDZeZpyNp4ZFc7XmVE8POYeFj6EM45MViV6xfSwo6LsTIzKTNfp+VmxX9X+4vWegDUNQZoTJRcenxWOx87Zom6Rt1fzLPZLbKrOIY7Ejs0FgqbStpqGkQ2Q8N2sWmOZuYaQ1DzTHwq+rLlbhkXsMjz4M9k/fw1bNf/em5tAeNCdGSRPFUrXUXa46MPIK/qT9BSvH0XcQDETwR/8QdixYHnjzAxokbRbp8hTGFBs0lrRV7/goaGzZb05n16OHBvMPzCBrT/olC84hmGQ5Brt+IG29XavdZJPR9kKgxr1Ds38vA1hYUxhSyuv9qNk7cKKIfB9j98G5+CPuBtUPWSn5/3qF5rWq61xbXUpJYgp2fneSUZZ2y+UB1Mmm5haCcRch0Miy0d5e+zrufNxbO0nF2VU4VZ94/0yqbTV1JHeaVBXQ9+CG3vjwsspdan+HJ757k1Y9eJfS8YVGk00FnXvnwFcyV5tSZV5Ba13Z6UkEQ6P9Sf14seVGyEa+RJaE1RgT3ru7EbYnj+mppOYOrP17lwmcX2HH/jr9ENwv/TtzbEv9Xin0dJurfa6kCEUDIxV/xiTtIkgSF7ZNPwrx5UCwhUX77+bflWrTneuVczCFpbxJ2vnaS9o2TNrJlxhbKUstElNiCVqDzzc562n9Bvw//Nu439jy0h+L4Yj1dtQAynQyNRuzzuBW6EZjeXBS9XbOvOq+amdtGozJRsXvSbsxGtaMDtg2I/S2W1f1WU5FeIbKlHEzhK5+vKEstk6SEzw8ZQpFvT4KubqY4zvCGNV77HvuWcXrYMs5/bpgfFOpVmFUXoVCrqLNxx9TV7m6d0j3cZfzba9+9Yt89/GuQy+X4+Pggl8v1q1oru4oggLGyCpuiFBrKxQ6nOqcIt/SLyHUaytPFPP4vXH2BUxEDCT2/msrfxcUUQRCwLU7BNVVsK08v59a2W5gqKzFVVtJQIt2VLX3cAltnbW0SBBfZ8wvwi96FVZl0AOCQp+detysUFw4UJgocx/WmIKAfhX49MXcxTBDZB9jjMjEYjzwPnv3qGSqj0w3sQ98ZCkCNnSc1Dj7t6vZL2p/ED+E/SFJzAIz7bhyzd82WtClMFLxa9SoWzhb8EP6DqEDUCK/4YzhnibWurv50FdOTh8gP6EZqt+mi5H5BdAG5l3LJOJFB5mkJ6tS/gzs4PXlX8zj70Vm+D/2ewy+IHXz7AHuur9I70S3FzK09rBn67tCmf0uJlUc+GAnok8mtOVJ/BX/myNUW11KSUNKqLpmRmVFTsqqlrkrQmCB6PN4DG08byWPe88ieVoXf/0nIjeVMXDkR+wB7NPXi88o6l4Vw5hzJPWaTGSGe9Jm5ZWZTJ+qoz8VTfAefOcimyZvufAy3r33/JcRt0q8nLWligSYqMlWVirMfnRV14qqqVJSnlXP8teMcfl78fP+TyLmYwyeOn3D1Jwn6OxMTcjsMocw9DP/ZPUXvqCwkmPygAQgywyRII1K7zaAgoC/pXach14iTXn8HA14bgNxITsJucYFIVdGclHTsKk6EKC4mMuLoCG51uoV7iwk5QQCn6iHY1fYm2zubEtfmxG2DmS2F/r2ps2lO6OuEthfF1Eo1jo858mTMk/TxEk/2+Q/zZ/hHw0UUnY3ocvY8T345kBH7jETJEEEnYF1lR71df2JHvIB7P8OJR+uIQC6P1K+DCp0CbZEhpZVFVSH2pQ3YVdqxcfZGFFbiBFFrSDmUwjLZMn4Il6bZNLE0ob6yXjJYA32Cuzq3mkvfSNNFOQY7EvlgJDZe4imHIUuHcH3mENwK3OhzrrNIxN3Wx5a0IL3fYlF6dwvpRub691hqnavOq8Yl3IVn0p4xmCxqxOu3Da1FRxvaLN2sSY+ciiBXEHpmBRk7ogzsdal59Nz3Dq9+9Co+ymP0Hdjced+WtS9qRRQpB1IkbXKFnF0LdrW6xtpm/sTk7UHEqB8U2RpqGnDKjcGkJhmNvFq0HhgZAQh4xR+h87GvUNcargmVbiHcGriY+P4Pkd5tWrv8pvTj6Rx67lDT5I3ouH1sMbM3kyzEWLtbkzDgEWpt3TFRiu0RA+ayYe4GgpODcUkT+5sNrt5UOpox+NRg+h00fO8qkwrwSjhKkXMR5fbldLsk1qm9E7bM2MIy2TLK08S+t72/PRmRU6h0ldaFvj1nI5frqfZurL3BzU16TZbqvo6cGXgGp1Jn6qsM70VDvpJHf3qUkUdGsnDNQlyuGE5spXcq5+Dog9RZ1BEXXkCup5hGtDWY2ZnxVPJTvFT6kuSEe+e5nZny6xQaahokaTzfmptCbAx4TIgU7aVyIzkNFvYU+fUka8RDIiaILlVzsK2wxb7cHr9UKyoKS9Hp9PraoJ/w63I9iNODTnN5eB/kpm2bVmzEM2nP8ODJByVtvz/1O0dePCJJR1V8q5h3jd/l1DvSDSxxH5/lqW+eItPpS0l7aUg/4sLjeHjVw0zdaThVWHnqBl2PrcIjzwMzpRmKP8K8ZTJ9g11rtHRNEMC4wRhBJXZoFSYK/Ib40ff5vk00jrdjx/16vTBzB3NJPWNthzAEmRz/G3skKLVBkOkfYq2ilhvlhs1fFa4dqbH3JjdkKEobV2Sy9vl9Iz4aITmFA1A6uwMfvvYh5y2zMLEyQW4kZ/iHw+l0n/79XTt0LcvDl/PriF9F3z2VcYr8mnyW/LSE05NOi+y5On3stTTZlg0nwqnLb6YftQ1wInrUi1zuLa0N9ncxc+tMpm+aLtnooRN0GGmMGPvVWM5+IG6002l15F/Pl1yL6ivr+XXEr1xfqY/DIhdGGtidw5wZ8/UYbH1tGfnZSMmJ3b8KCycL5uybw6ztsyTtiXsTufTVpb+WCJTLKfLtQZFvMx1r475mXlOMaV05OiNTka0tyL+eT/b5bJL2JVFfLqYYNbbQrz1Skhx/hszTmawdspbPXD+TbKJ13ubIyMMjGb9vPPPXzjew1ZXW8YHlB0xfp2XpO0vpfuzuTpfG74hvVWverasbvZ/pzam3T0kW06+8upNOZ36ixt4bUzc7A5uqWkXn0xuxUOoLiR7Jhmt/ra2WXM9cyhzK+OnRnygYKn6OW0NdSR3vGr/L709JS2YEjQnideXrrWq2yxQyjM2NW33un8t9jsVRi5m1fZZkDNkW/Btxb0v8Xyn2NcpuOIeKm1Bl2uZcS2Ojyu3I/aMv6sqVO/+Nu13s8+rjxYzNM1rVdKzOqyZ+ezzfBn0rYnuSmch45813yPPIo2S5vtFj3A/jGPXFKGy8bAguDGbm1pksfWcp158TF6QPTbvBhy+/g1+6H0OPDzWk6LeXc3jkYVSmKgSZwC/Xf2n7SbUBDsEOhM8Ol5z0tfawptN9nbD1sZWUjWkwsqDEO5LssFEiX1MQBFwTmvfvxkaepu8mphN+ajm2Rckk9Z2P97y7O7F4D3cP//bad6/Ydw//GnQ6HVlZWeh0OkIurKXH/nfQ5Ym5+nVagaCrm+lweQM1ibkiu7GDNZo/nNkjLx4R2X87+xumKlOOjjiLRQ9D2kedRseqPqsAMFHV4JBuWGAydzCn68NdqXL0I7HPfIKebDs1pKATuLX1FoeePSQ5ySarrsQp5wZ+N3ZLBidp3aaTFzSQKudAyQ13xQqocfQlO3wcVj6G3dpevb3wfbUvMV1iuNDnAgU2hpprpjam2A7oTI29N5YVOe1KWtWX11McV0zUiqhWP1Nwo4DL31+mOl9M5wP6667T6ETJ+as/XaXLkc/JCR1BtsQkZ9LeJORRl9g6Kpgr3faKgqNRX4yi68Nd2bVgF1d++BNPp514OvVpllxfIhnU5FzK4cSbJ6grrpMsYGrVWkIm6ydwRnwywsBmbG7MoDcG8UrlKwCkHTEsohbGFnL89eOM+nwUPZ8wpP34p/FNwDd8H/q9aBoR9MW9hN0JuHRxYfae2SJ6JCMzI6J+jiJmfQwnl54Ufb+x+CnVWfx3Ebsxlv1P7KehVvzbMpmMvCt5nHn/jKTdd5AvWFpiU5qOQi0OcF0jXJn8y2Tu23kf3n2lndo/m768fe37b2Hq+qk8fPFhutwvDvbkRnIej3scIzMjYtbHiGhgEvck8k3gN8iN5a0Gi/8UMk5moCxTSl5TuZkJ+R0Gk9Z9Jp1fGieiS5OFhZHbcTgWVfk4ZV5DdduzJghQ7tGJnLBRhJxbTei5lS1//o44/f5plsmWce1ncVMCgJ2fHeVp5RTfErdX3u7zSz3/Pslu9LgqrR+n1Qg4Zkej1Cxn66ytHBjZPOGssnIkq9MoYkK3ARARHYHDpbZHawpjBX1f6IvpUVOMVoqplywcLRjw8gBeKHhBcoqt3MUWrVyLS5EtqkrDiRgTSxNiRzxLTif9ut6S4kinE8jwucZ7r7/HJy9+gizCMIGT1n1G038XuBXQoBV3e7eGxqRUy87JpuNOL2f0F6N5+MLDrf7G7N2zGf2FeE8quFHAvkf3Eb0mmjPvnxHZ6+qgQlVKUEoQg053E12XnEt5dEiwYM2CNeydIK118VfxaPSjPHjqwabzvx2ZZzLZOHEjma1MGxgV5eGedIoe+5aRu8kwsarT6TXkAqO2YVmZT+kNw8JuI2sAgFyrMbjXbVn7Hjz1IJ3ndZacqqorqcO9mzsRCyIkv5sboO/yH3FMfK+svO1Z/Vgqp3u8hHX2d9S1CJ4dh3bh8qixRIWvo9QuWdIv8kw4ToeL61o99tZQEF3A1R+ucmvbLUl7fUU9q/qsYs3gNZJ286oCOlz+Ddsi8SSwpYsnyR2SWbvgV9Iip4nsm52DiQ6Yx/Fhxzk13rB5zDrIjdRuM9k4ZyNm9WZ0uxIm+v6dEL9dP0HZmvajdXEaYad+xKZIXLy9/O0leuxbRsi51VTH6Z/DxL2JTRMwGhqI7RLLoVlBuPU1nNyT2RhzbNgxbkTc4PjwaMzCDfW0/Vze4GLfi9RY17B1xo8URbavqcjE0gRzB3NJymtbb1syTmawduhaSapZ/+idmFcXYTZrsoiyys7XjoT+D1Hi212SOv/sWyd59qtniekSw7K3l6EJVyAIUGPnRW6HIfpzS3ej3KGcTJ9kZK1Qcksh7WgaZz86i6pa/F4BDHt/GB2ndqQsVRyvmFiZEDwumIRdCRx95ajI3sVuJGpjNb2uTUGeI72mAFzoe4EbEYZaQ6Z+7uQGdaPKpoqFvyyk0/eG65VM0XoRRKfVMXHvRF7/4HU8vhdP+6rr1BiZGjHojUGixhnQ64CHzQjj+YLnJZN1DROnkdxzLsk954r8veqkXDpe2M6IIyOYt24elTGG66DS1o0bg8eT4vYJAedeIC87q81+X9K+JD60+ZC4LdJFtSOF+jWo0vIKubnQUN2AsYVxEwvHmK/HYGxpLEmhmH0ym8U/LaberB7XMeLEfZdrnRh+dDgLS3wI2jiDihvNz4PcSK9z3CvtNL4ZvnSLahvNYFvhFOJE+H3hkhpxpedKmbl1JpndMgmfI6YoTT+ezopuKzjwxAGRTVOvIf2Yvul1zNdjRFN01u7WhM0MozKzkltbbknGBn8HHcZ3kPSZoJm2urWmyjtBW6NEkMkxVtWIGtbcUs7hG7sPt5RzmNTpcwztCTlCJoaw4MQCHo15FDt/O5F93LfjWCospfdTvSULlR/bf8wy2TKK48V+l1uEm6TOayMCUt3pFNcJs3ozzJWGEzEtfWbHgnbqq/wJAkYG0O8l6efa2NwYt0g3HIKlpzRch3SkIKAfyb3n4T3dkNFGpxHwyNM3Vl2PvM7p2YaNvanda1k3fx315vq4Uyu0/WYZmRrhFulGZValqKEM9E1SR146wuZpmyWZHm5uuolDkAMR86X9KktnS9y7uhM6LfQvT73+G3Hv/yrUSjXRa6MlKVf/DHJt8zrR78XW118p/+L29NX5NhCVtafYZ+NlQ+xvsfz+tHTBWbtcy9r5aymeXCy5vusUOjrFdaJuRx0AfoP96PtsX0ytTXGucqbTrU5o5VpsI8QNWBXGtxDkGnwzfRl8ejCq0mYfp96knrhOcVjU6RkwFPPv7npx4o0TVGZXSr4XbpFudJjQgXUj1pG4Vzy4IVfVoTGxoNC/t4jVQ6cV8E7R71mqxY4Mf3+4gd3IzZG84MEobVwR0N7T7Psfxr+99t17NO7hX4MgCJSVlSEIAioLOwB0t8RTEDqtDstKfbHKMkAcmJg42aA11gfUfZ/vK7JP2zGNJT8t4VKfa5iFGiYMBEFAq9ai/IPCzaLUsEvG0tmSscsnkdR3AdVO/u1aTGVyGZPX6PmTpRK+suBgssJGY1ZbRmmyWNNMY2qFR8oZ/GL2ijZcVXUDYaeW45F4Qv9bEpu6WqshzzOPQ2MOIfM03ITK08qxHRCOWV0ZgVe3tE+0e3QgyMDcSTySDnB99XU2TdrE70/+Lipi6rQ6EnYn0HluZ55MeFLUQWxqY0qDhT3Vjr6StHpT101l4zPHAUjwek1k7/ts3yaKiTs5QX8F9gH2uEW6STr44feFsyR6Ca9Wv8rEn8QURvse3Ufi7kSeTHoSx2BxxyroaWvu//1+0bRYdW41t7be4vDzhzn8/GFKEsT0Nn8Vf+bIdVus1y+R1OcqrGHzlM0cfu4wcZvjRB2BqioVMzbrE/TJB8QJygUnFjD+x/E4BInpAv8uMk5mcPWHq5K0TIJOIGxmGLN2zJKkj7H2sIbaWlzTLxF69meRvSKjglV9VpFxMkNSi+ct3Vu8WNy6PhgYrn3/LTiHOktqd4G+AOoc5syYb8agUWpEU7FW7lb4DvJl/tH5klo//ySiVkbhGuFK2HRxEvr2yZA7rWEOuTfxi91HfUX9bZ8XCIjaRqdTP2BeW0q5mzQtcWtonNBJPZgqaXcJd+H1+tcZ8LJYm9TY2pyMzuOpcAnGwkv8/J+cdp0fH/2RFz95kcMvGk5SajU6/G/sbioGtnyCimwPkOilH8vqd74fbifbUewzUfBJyCfUeNXQIIiTXTqtDo1KI9mhCHBg1A+8+9a7fPTKByI6Y0EQuOn9FDFuwzAu/U000RudfADL/I3YVNlQZ1mHTiIeOzrFk/X3r8ex1JHyMjFlW2vwHejL6/Wv80bDG5L2w88fbtIKksKuBbvIOJkhSW9bnVfdVESUWue+eykLdc3rXOl5heWPLhdpnZxaqw/mXAtdkct923xObYFjB0d8B/lKdmtbd9QXDzasVBq8F41wTz6NZ9JJytzDsAsy3LNqC2twT9YXNustHen+1ngDu02vjlwZ+yw75zTge/1bbq5oLoK2Ze1LO5rGjrk7JIsOaUfTuL7qOs6dpGl383x07Ji6g4Ru4iRKoTKfLNd1+GX40el6g+i8Gyd1Tg49xuqHN6AwN2xgCbr8G7bFKSi0arxv/t4uv6nHoz1wi3TjxFsnJO3b7ttGZVYl3RZ1E9lU1SqsyrIp8u1OtaOfyG6EOQ7VA9BYhGIXIr4uj341nQd/eZBMv0yyggyvqbG9FbX2nty3+T6yvbPJmThc9P074dmcZ3n81uN49BQXWkqTSvGP3oVFdSEyicSlhZsNWoUJ1uXZ1KUVUJZaRn1FPZ699AXbLke8mbJzCggy1C1vp7UxZwadISE0gSrPGehasAeYCC26pFtZs6RQFFfEO0bvsPvh3ZIJeEEQCJ0ayrAPhklOwWWFjyM/aAANd6gXWJbn4HDzdBO9WiM8BgZxvu95tAq976LTCQiCXi/SM+kkv87fz86pW5Br5SR6PNsuHyJxbyLHXj1Gyu/SU7MBIwM4+8FZEcUh6CdPZ++ejbG5MTd+vSGyy2cG8+NjPzL6987ILomnS32PrWbk4ZGcHXiW690MO/PNOviQ13E8brUvk9qhivpu+vVmqbCUpcJSbDxbn9zRaXR0j+oOIKkblnsll+9CvpNsWAPo80wffAf78rX/1xTGFkp+Rm1uQ6VrsGjtvlVwGvuSapyLnfHO9kZXJX5WZDX7GXyqJw7l1igb6trs95nZm9FQ3dAqbbp1vIZFKxbx5jtvknc1j9riWg4+00zr1uuJXrxW8xpz980VfVer0mKuNOfU4FMEPxcssgcnBNDnYh82uGSSPX8z1p2b9VzVSjVmVUUY1zfQ63IvJu0ZdVf92A1jN7Bz/k5Jm1WlFf7p/pzteJaCjuLG4NxL+mZgr35iP9fSxZLn859nyq9TJKUWAHIv52IfYE/u5dxWp/z/CgpjC1neZTnRa6Il7X5D/NA2aEWNnm1B/a00XDMuY1+QgK2VYbxjoqpGrtPilXC0iSmoPbfK3MEcvyF+uHZ2RWHc/gR54/4qRfvoEOTA3H1ziZgfweIosezGLwt/4fsnvmf7jO2cf9Dwb9t62/J02jN8//j3vPvGu/zyyG/tPrY74dqP15qaWFpCo9I0aXA5h4n3Wq/J3ckJ01Nltmxo01po+eDVDzg06hA3Im7gYm6oBy+X6f00M6UZfc/3RXaz7YUgUxtTOk7rSPL+ZEntR2WZksvfXtZTGUvki4pvFpN9QTwl1oiiuKJW6Z/bin8j7hUfw7/2pw1w+t3T7H5wt2SDIMDeJXtZJltG/E7xc6g1NiUzfBwp3Wc2UQRL4c+KfVu3/vn1aO/10ql1kpq+AG+eeJP0gHS+7/q9aC/VarX4ZPpwZOQRnoh6AtDv3dd+voaqSkXYjDA2fLSB919/H6/54vXdtCSHwJRArvS8wqGpDgR0bmaw0AkClXaVpATp/Z5qa+khhL+KiAURrcoagb7xN3y2tAasZ8Jxuhz/WrKxHJmMz57/jI9e/ojsXuIcsrG7M3khQ7jY5VVUFZPIPvbflVi5h7bj3177xC1f93AP/wKywsdS7h5GaIS4ECJXyEnouwCNiSV9ncUdHTIZ5AcNxKo8m04zxRREugE6rqRcwb18hsj5UhgreDLhSeb1SkJrZIraRcz9XhJfgnmVBrOaMqribWCwdMJcfFwyOozvwCOXHsEhWJzQlSnklHl1odrRD+++9iK7cX3zhtRyfRAEAZlOi21RCubVRdSkDoEBzYXQrLNZFLx/HXcfd/I98tG1+IHT750ma28SdkCpR+d2Ja0snS1Zqlvaqj3q5ygqsypZcHKBSLNPq9KyecpmIuZHMGXtFNF3O8/pTMJvnZHptMh0WsDQyTe3N8ffdgLB+ywJv9kNzSuG4sTRa6Ox8bQh5P2QVjV9/iqurbgGMuj2SDdRwc/CyeKO01xefbwoSyqj+FYxFo4WmDs0F0pLk0r5LuS7pt9pWSgKGBmAa4QrhTcKidscR88neoqon/4q/mzfGf35aEZ/Lq2VaeFkweClgzm17BSxG2KZum6qwXVJOZTCtln6CaOQSSGi7/sN8cNviN9fOu76ynoOPH6AwW8PliyejvpsFMPeHSYpBK2uU7NuxDoiFkQQOlXcaSs3kmOy5EEaflqDQiPugP9lgJ4C4tLXl8i5kMMjlx4xsOvUOmRy2V2lAbobUJYrOfz8YfyH+4um+wSdQMqhFOrL6+nzXB8RnYS2QUvm6UzKUsr0k4//RYz7flyr17KmsJaQ81swVtVypcaZyEMzDJKvutNn6XA9neywUVS6dsDEtvkd1WqEpgQIQF7IsPYd13fj6PdCP8lnDGBlr5WoqlU8nfK0yCYI4H3rMAqtmryjt2C0IWWmxlxHtVBNsXMxVm4tAgSZnA1zN1BtXc2QE0Oos2zOKJtXFtA9Oo0yWSCpQansmLaDl+2kJw9bw7HcYxx75BiLnxQnYOK2xLFj7g78hvox9puxovW91rR1jRG1So1t/hm8s72JuJZMSWw+9G1O4uYmXGP8gfHsmbgHI40ROn/DxKlFRR4qMx0dcjvQ60ovSidLBEd3QEttrNtRN6iOxJuJfPr2pzz/5vOiaZ6EXQl49pbWhQkcFcjcA3M5/e5phr0nfoay98cwOWMyH7/0MeUO5aLigFCr72IdcnIIhxwkVNj/BtaNXIexhTH37bpPtGfF59uhMzLB7Oh+im+5iqh3ivx7U+HWkVLvSDxa1H+U5UqccqLJCx5EXshQZC0urUwhR2OiI839W/qZzqNELU7M3gl2fnYMWTZEkkLRLdKNkZ+OlNQ5BpAJCmIiYuhQOkJkU9bW4ZXtxc3wm1S4D+YxN8MiQl1aAe5piVj5WaEyV4j2R+1tdGjOmVcRhLG0FcYWxgxZNqRVtoMOEzvg3MmZga8OFNnqiuvwTDpJfmA/6q3Ee3/6vl08vm04l4b6YBuqAQwno1KCsyiz01MQmmkNE5SCAFpZA26FbtTa2CK3kNYIaw02njbQimRSTUENJqpqsjqNkaTyDBgfSvSoYIwbahk8zgxjiwZCp4U2FXJ9cwKxTdOR7fMLFVnjcHdvvl+3B83RAfM4WfsZT/B80/+zu3iJR7MfZf289Yw+NBqz3vXQ+uCuARQmCgStQPTqaPq/1F/U7HTm/TOcePMES64vkZycKvfohH3uTYpX7KR20iiDLvba4lrck69hWZGLXWESVTkBBsWsA6ErOOJ1GOsqa5yynagvVyIIoFPo76lroT1pASom7J1Aj2s90HarB6Qb71qi3wv9uPzNZU4tO0WnWeJYSWGiYOSnI1stpAPM3T9XMmFhJtcn7/ZMOc8kJ/GFNlJW41raFziCZ8NQkV2GnKCCV8AOOvzxWqmqVESvicajp0erLAoKEwUF6wuopZZPJn8istv62BIwMoCTS0/iGOJI4EhxoaeR+i79WLpoXTE5sh8njSslvuJJ+6IOFSx/7cOmf78TutvA7pp6Hu94fQJ9w9wNDHJcALRt3/Lp74Otr62eAUUCYSd9ccmzIyEkAUsH06ZEYmXmnyfkrQdZ8/V/vsamtitmFd1E7++hyadQacpID6giyrMK89vWycqUEsJPryA7ZBAFXo8Ta9/+yZQ7IeWgPiHbfXF3fAYYUpwPf3Q4rxu9TnJuMqVKcYNs14e7EjQmiPrKetR1aoOJdplMhpWbFXIjOd+Hfk/vZ3obaE+mHUtj85TNgH7yr2Uy+u9A0AoUxRaxe+HuJkmG22HpbEnIxBBsfdu//xt7uZIXPIhKl2DkRnpftPH1LHMPw6osC62xGZXO+knG9sT6GpWG6rxqtA1a7PzsRD7Ure232P3gbmx9bVlyfYmoIPhS2UtUpFdITgUC+AzwEd3jRihwRGWiL76H1Rn6ojK5DBsfO4pd9AlwoWVi529iyq9TWi1Y5F7KZc3gNYz5Zgy9nxLvlWo1yDUNuCefpvCUFwxsLgLUaWppMG3AqcSJiIQHCfSYYfBd7yQr3IuGczP8JqMPj6bevH3nFTo1FIcgB0m/KfOMvoHTyt1Kskl10qpJxKyPIediDl59xDmu5eHLAT3F9cytMwkYESD6zP8F/K8U+7ot6oZ9oD3+Q8U63gCph/RNpJJFdpmcUq8ITOoqqCuta5rm/iuorwfztrkPf4r4nfHkXs5l8i+TJe1e172wM7IjNUjcIKut1/LQLw8RHRHdtJet7KVn23Hv6o5RqBHJ9cmgMNTja0T/k5F45Xjx4Wsf4qh+CqPbwvKK5DKe/eJZbKtsSeyQyMa5G/mFu0fl6dHDg1vbblGVUyWScii4UcCFVReIfCwSz55iR7nKOQDb4hQ6nltN3tVpePS4PW8po8a6BoVGgdk5NTmdpN/NbKdfeOjsUjJV5wHxwMs93MP/VjbyHv6fhc7IVJ8McJSYepLJqHH0k5z0AlDlFOEXuw+nnBuGPM1/wH6WPWcGnmHKJg8q94o7VgGCr2zEJf0SgsLQmS1PK2dFxPd0Ov0TgVFbydkmrdkjBUEnUBRXhKmNqaQwK0olprXlNJjbSNLHBERt1x+Da0d0LbqSTaxMiRv6JMU+XbErSECoMaRQKk8rp+BgPF2jBzDntzmoEw27uW/vHi8I7NsuB6gkoYQrP1wRidU3YtqGaTwR/wR+g/0wszVMhCtMFExdNxXHEEeuLL8iSScBEHHkc4IvrRf9/8rsSsxL1VRbV1NpL+6i3f3gbuJ3xDPwNXHC7O9i35J97Fu8T1L/SFOvQVmm5Pov1yU1urov6o6yTMnmKZs58pIh1ayJlQl2fnaAvhDVMhkoV8ibpqkGvTWo1eDovw1Ta1OGvD0EG2+9c1OdZ3jczmHOdF/SnQdPP9g04Xc7CmMKiVoVRV1JXbv/duapTGJ/i22V4ihxdyIre6+UFFGXG8vp82wfAkcHSiatYtbH0PDTGsrcO5HUa57I3nleczemzyDxvXjP9D2+7fBte07nv4J1I9cR/Us0GSczRDZNvYbfxv3G0ZePEjo1VKQT4NTRiZGfjWTPw3sk9TX+SbhFuJF5OlPyuBvqdZjVlmJWW0rhqQRRYkwoK8eiIpdctxiKPMyRmzSv7wIyCv30NDu3Bixq93HJjeQ4BDm0WuT3GeSDmZ2ZZJeturahqXBgZC3eG5wq3bGqsWL3jGT6PmfovAvISO6QTIF7AT2v9KRvTPNUlVFDHS65FdhW6oP9QrdCjDzF1GWtQVWtYsGaBfQ935ePzn4kstv52mHlZkXGiQyyzoqp2npejmTW5lmE3goVrQcNdQ1M3TWr2mD9AAEAAElEQVQVuwo7To5SYxdm2IyhC7FhzYI1GKuNeXz549gnGCbbPJJO0v9oPr2u9OJGlxsIEhM1raEkoYQtM7YQ+5tYDxXggOcB6izqqFtWJzkN/ErlKxTGFPLLQHGAKFfIUVWqyLmQg7JcPG1Y0aEXByZlo5PrsK30FWl8JfR7iOOjKzGvN6fnmfYVMP8MaUfTSNyTSF2xeI0VBKh0CiSjy0TsA8XNRtVO/pR6RwLiBKG1ryPRI56nMKAvluXZVGUY+hfqqjrCzm3jgXUPsGLxCphk167j3rVgF/lR+ZJJDKeOTpSllnH0ZTGVIEDw9VgeXf4ouVZiStS63BoeWfUIHRM6UumgRd4ikZITdYmQ61F0v9adXhdDRVRu6d2mk9l5PPmB/YkZ8Vy7/Ka6kro70mT1erKXJE0sgJWbFUm97qfEWzz1B5BnmUWafzL9j2RjsUE8jX5w4hlODj3J4yuWM/cHw/Wk4GI8Iec+43KvBGq9X8Gsrn2Zn4LoAm5tuyVJDend35uoMa9SLFEoAf1ktqAwosHcFoW5Kdbu1vT5T5+mZqayh3qwfdp2Ju7tR0GUIYW/KreWh1c+zNDjQ3nsh8ewP2349yuFeIzVxhirjel8szNGmW2nyHMMdmTBiQUMeWeI5DPoHOZM2MywVhs9ACyq8lFdjqGh2vAZqi2sxTPxBA1mNmSMWSJqmGhMZAWlBLHg1wVU3yxBpwOTev0E4OhD/bCptCHHK4cbnWNFz/CdYOtty4wtMxj2vnRjy7oR60g9nErwWPG0V1VuFQf/c5Dcy7mS8UzKxd/pc6EP17scQe4jTkgljX2GkpAXeeLntdy39ykDW/WFmwRf2oBprX4daXyvdszbwcFnDt5RMkAmk7H8/uX8ev+vuFmJJ6/t/e3p+3xfvW+dJy60X//lOuaO5lh7WksXQKOv4Re7n64H3hdNWZjLDRN7Lf3JxgKtykRFcodktO2kcHom7RkWXZH2TaLHpPPrA7+yac4m7AIdMbYwxsbLBhMb/TTy8s7L+T7se85/rudqq6+sJ/+6vuDf+IwNOhXBz9OPiX7b1boXlXaVvFYK+1IeJsi+uVnPzNmK3A6DOdt9KSd7PEq1S5d2ac1pVBoufn1Rcq8EuG/nffgM9GnSm20JI7kRi1YsIvt98RSStbs1gk5g3Yh1bJ+zXfR3M09ncv4T/fUwtjRsiGgs2rt2cSVgZMCfUvG3B26RbnRf0l1EHXo7ui3q1mri/04wcnMiL2QotfZe6AT9fWh8DDMiJnFzyJPEDXkCpa3+3WjPIxizLkYv5dDxe0oTxcXVaz9do6GmAUErSGram9ub497NXXq9OJTC5x6fs0y2jIP/OSiyd0l4ApdCFwJTArGJN5Qh0ag0xKyP4aWPX2Lmlpl0TRaz6fwdXPnuCttmb5O0WXtY4zvIl7QjaZJaook/nSDw6mbcU89RdtlwgrpeqSQ8NpzuUd1xy8qnfus+A3vXlMEMPDsQpbmSlQ+vpH5U21O0qioVl77V6z5KXW+XTvp9pia/RpJyu6aghkPPHiLlkPTU9+Clg4lYEIF7d3fJfNX/FfyvFPvs/e3p9nA37APEvjfA/GPzeejcQ9Jrhk6HQ24s4aeXt0oLD9KTfS1RK82+3oT2XC9TG1Ocw5xbfT5G7R3FA+sfYN66eSJpIxc7F3p+0JNFDy2iIFnfHNi4H1t7WLP5/Ga8sr2Yvm062ZvEa//5fufZP34/xg3GGNXXGMjpCHKZXrs5LI6jI6Tjhr+Dwlh9I74UfW7izUQS1iTw3OrnJL9b4R5GbsgwZIJOJAEk6AScip2wL7fH6xdTrrTwg1QJ6YScX0NI0gC+f+J77JdIxxb3cA/3in338K9BJpPh5qanRXTOuIz3TWmeZ51OwKYohfDj31B+UTw5oKtvDqaPvSoOXBQyfTBsrAZdg2HAr9PouPrjVQAcCuIxqTHcgG5P9tTauOExxZCD/U7QqrWsHbKWY68dk+7OzMgg9NxKHPJuSha9yjz1mgT2hQmUJxjSYDRuwCW+Pbg2YSkeAwy7rMLndsbxy9cpdCslMDUQXYVhIqSxYFTs3RWlrXu7AoCsc1kceOJAq4UW+wB7nDo6IQiCKPiVG+k1v8pSyzjw+AFR4Sz7QjbuyaepdvCh2kkc+OxdtJcuPzZwZtAZDkwtanVa42P7j9k+d7uk7a+i2+JuuHV1k+yKu/rTVT5x/IQ9D+3hwmfSo/T+w/XnEzzOMJFi7WHNoiuLGPvdWNR1ao6+ZOiMKMuVlCaVEjJJP634l0TcW8GfOXI7H9jJMtky0k+kt/qZYe8Po/PczqIOd5dOLkT/Es2aQWs48rJYS/PHiB/Z+8jeVovGd4JjiCM9n+wpSdsEesqSmsIayWtlZGqETqNjx9wd1OSLmwMsXSzBypJy945NAfLtGPnxSPq+0heH8Q4M/EC6qPxnFEC3r313C5XZlWwYt4GYDdJ0VaHTQwmdFmrQzdwIuZG8qZN5zyPiJHnOxRwDiqH2UKL9XdSV1HH6ndNknMoQ2YwdrLkx8gWuTljKmAtLRdpkskkTOTwrgvNhozjaxdvgeRcEGdnhY7k6YSn2BfF43RI/o3dC7uVczn58VkTD1oiej/Uk/1o+UavEicqGqnpMVPpnz7azeGKh194gHv3xUTzL7hdNo99+DisWr6BwRPPYVbVzAL/P7UFUd/3fVGgU0EpDhRQ0ag1uBW6MPDIS0yhxwGbf1YXLo8ZRNm+E5LRucEoAYfFh3LflPlGh3cjCiN/m/MaRkUfI9ldj4mg4sejhHE6Gfwbp/ulkh/TDO8hw+rTIv7mLOicwAGOztuuGFMUVEb89nh3375C0D/QZyKXel8h+KVuSlg+gw4QO+A4RT7Uqy5WY2ZnR6+leouYWAMHZhUIvEwacHcCzXy4UUVs7ZV1j2CFb4jvGkx4kXo/+DiatmkSn+zpJnlP5+QSCL2+gwcwGK1fxFEPv3mCsrMIndj+5pwzpSVVqOcjldD30MaHnVnNruWETVU10KrYlubgUu2CsNkZ32yPYlrVv0FuDCJ8t1mRqRHVONeXpYp1jALkgw63QjUnbxb6akb0ph0YdIssnC1OlBk2DoeNzMmg/axaswbHUkZFHR6KqFSekTWvLMaspRWNq2a5ESNSqKH7o9IMkpVkjjrx0RPIZNbYwRm1mTYdL63HKvCqyV3TR8dv9vxHTtRZ1oBSdkP5AldZWNDgY7mk5pFPoWkhS8FU8Yr/CfscXbT8p4KeuP7F15lbyo/JFNrlCjkKjwjE7GrMq8XkfX5WO980DWJVm0FClv9Zbpm/h0H8OAaDQmZHlk8XuKeewDTPsinaQueFQ7oZZvRkKrUKUxT438ibfPv0t5fblvPPmO5TPMKRk/TP4DfFj8JuDJZP+odNC8eztybbZ26grFRfSw04tx7SuAquPXxdNtdgH2hM3cAn5wYNQ2buJtI49Tpkxd8Nc8jzy2Dd+H8Z+NggC3Bj5AjcHPQbAyCMjie4aza5pu0RUs3eCpl5D0JigJv3olrB0tZRcC0A/XXrp60vEbYmTfIZNz9cw5pDetzBtJRdspLPCqrIa0wrDa6YuqcSqLIMUlzcxzXmJmi16zbWkvfp4rzXJAND7ItHno/lu0Hcc/ERcNAAIGh3E68rXiVwQKbKlHU5DWarkP5n/kaTZqn/yBZJ7zqHayV+kM+SvCiMkIQTPHE9CEkLQlRjGWcV+PdmxwJsN92+g98XeqKtVbfb7CmMLOfbaMYpipdcLmwhH0gLTkAnNE8jPpD/D+O/1zT86jY6S+BKOvHAEbYOWE2+eYEW3FaiqVNRn1dM1qitdrnthmSOOqU2r5TiU2/FMVghO37iRdqC5gdHc2Zr8DkNIDD5HrXkiyNrnCybsSuDQfw5x8Usx1StAxykdWXh6oSQzS0FaAarzKuwq7KSbLlUats7Saxh3mW/oPyhLlawZvIaC6AK6PtyVYe8aFrx9+vvwTPozFMYUNumJ301M+HEC9/9+v6Tt8neX+W38b39Jv1wQ9Pu0RWU+gsbQ1ws7s4KO51eLPt9WOIY4Yh9gT4cJHSTXQbdX3LjyxhX6Hu8r6WNcW3GN+J3xkvkNQSc0xV+XvhY3UQ84bsrsTbPpd74frqcM97yq7Cr2PLgTC6UFnW51YtBB8dT430Hw+GD6/KePpM0hyIHwueEk7U2iNElcAK28lYdlZR6xQ5/CZ4HhBLOqSsWM7TOI6hpFYkQAip6RBnbtgJEcnOZEtXU1Od45aNpB4KNWqolaEUXWOWm9VIcgB8Z+N5awmWEYm4s1nGsKaxj05iC6PtRV8vtD3h7ClDVTmH90fhPNdnvxT8S97cX/SrEv51IO11Zca1Ub1CHQAe9+3pKsNib1VfjF6gvFLdfJ289P6jLrdAIWFbmY1Or957tZ7AsYHkDPJ3u2GhMHfhJIQkgC/rn+orXO2MSYNUFr2Pb5Nlb01ksqzNg8g6XCUqw9rGk418Ajqx6h883OCFHig0oNSiUmIobel3oTefRzbl642WSz8LFi/bz1dLrViQVrF9D/bP+2n1QbUJVdxaA3B0nS+iYHJbN2/lpMok3IvZIr8W0o8+rCzWFP49zVMBegUWl58vsnGXJyCCdGKzEfZBjTCPX1mNUUY9pgQolzCTKne2SN/6v4t9e+e8W+e/jXIJfLcXNzQy6X45p2EdeMy2iPivVMNPVaOlzegFldOdnrTorsFsFe1FvqaTJrCsXJsuDdHZiz9WUujB2F1SRDB1+r1rL/sf1N/3aNN/x9186uvFi7lKsTlhI/aAk2naSpZCTPz0iOz0AfEnYmEL9DzLstc3Gh1sYN35u/Swq3GnRDt6AVUys1OGdexapUT83QckM+fkLOnlPlXOt2nPfefA95T0Ov0czOjE7rXkYmCHQ5+gVabdt39Mbuw8yTmZJ2ZbmSMx+c4R35O2ScyJD8jO2Yfozf/IAoOX9zfxaeiSfIDx5EfvAg0ffC54Rzrb++yJjtvFpkXyospeeTPamvqL9rVJeNmPjTRJZELZEsMLqEu9BtcTcmrZrE6C/F3fnXV19HXafm9frXJUXaLZws6LGkBwNeG0CHSYaBS9aZLLbO2EpdaR1mdmatdsT+FfyZI9f4XLZMUANUZFbwofWH7Jq/C5+BPpId8I0TpNGro0W2xo61v6LZ5xTixLhvx7U65VgQXYBGqZF0vgDcuroRNjNMkoIraHQQ1NQSGLUd92TxJLCmXsPBywf5RfELi/aKu64nr5nMzK0z73j8t699dwuqShUZJzMkJ8kABr46kFnbZ4mSi6CfuH0+/3n6PNeH0sTSJsqXRlRlV5F9LptHYx5lqbBUsuD9T2HrzK1YuVnR6wlx8v7251eqYUEQoMTmGMOPDufNd1+nKquiyaZRC1iVZWFdko5zxhXsCsVr8J1w6ZtLHHvlGAcePyBpt3S1ZMzXY0TFfQBjO0tKPTojIDMohDQiu3MFF/peIOx6DYktJoVVtWpee/81Ju2eRKVdJWpzw/dObdSs6fnQ6ocofFdMbdYazOzM+PTFT5ELcixPiItpOy5dYUtAGCu9HpRMjG6bvo+fFv/EtunbWtCR6Cdqk0KSKHQrRJDpREVMLxN9cafItYgtk5/D3Mfw3a1yDiS3w2ByQobhqv4MU3nbO+8DhgfQ6+lejPlaXOgG0CzXMPrQaAojCyWD68wzmfR+urcoQQh6yuwNYzdw+ZvLrBu5TmSXyfQ6cLmeuaR0lIm6Xk3qqylwLeD4sONc7CWdrP6r6PpQV2ZsmiE5gSQ3NUJjaoVOIR0gGh/YTcSxL3HJvEp5nGEhZ9MGTdMUDoDXWMPCnJmfGzW2TqQEptAtqhuqmIrmv9uGtc/Gy4ajrxwl72qeyJa0P4nq/GrGfiNNoXlg/AESQhJQmYonzRTWJlzodwGfLB+mr1dQkWbY3FVjV0eGfwbHhx1nxaIVYGl4bRyzo3FLO49dUSJm1UXtapLy6a/fq34bL60vdOqdU5z/9Lxksk6vJShHY2KOIBffr8ZJnR2TP6W0bxeRfcjRXvS+2JuULp0oHj7bwGbcwY61D64lJTiF6xFHudmldSpeKXR/tDvGlsbY+4s71FVVKhzy9HqpNiViParDv+TimnGFjhfWkrc3iuq8ahpqG3AK0/tumVnbsKuw43rEMYwdDKeovDt04qcn9vP7uN/57qnvKBxouCfJGsNbGegUOnTtKErUFNRw+v3TZJ9vXcOovqKe6txqyUY+jakVGhNLBLmRKLg3NjdGaeuGxtgceYNS9H2LQgUBaQGUOpZytedVFE5mTftcvbUTJ4ac4FaYvptfkBqnuQMOPnuQj2w+apVNYfrG6RTHF3PsNXHTpHMnZ57NeZbiuGJW9VslsheNl7HqoVW8+OnraLeIC9Z2mTeo0X7FjrkFpE80pL+zG9+f85MWcCP0B5wKqii/qU/+T1o9CdBPQ7YGdZ2a3f13U3qmlLN7zors+dfzWTdyXZOWXUtM+GkCLxS90KpPI5hbUOnagZRec/Hub+hvWmYZMWfTHAaeGcicTXPQxYvjT4tqFTO2zWDswbGoqxra7PeVJJRw7uNzrWoNOpl445Ppw4ydz1ORVkZJQgnvGr/LybdPAvBE/BM8lfwUD51/CJlC1vScaVQalDeVTN4zmb0Td3NptJhStcdOD5Ysf4xYlTdyZzOMrZv3D0H447mT6Zi0exI99/7YKkOLFFw6udBlXpdWKfxjNsRIxsMAxzYfY8KqCeycuhPXt8Q0zmc/OktlZiWOIY4ijWdTW1N6PKaPqf2G+EnebwsnC3wH+3Lh8wut6jf+FVTnVXP1p6sUxUkXbn0G+GBibdI0edke1N9MIeLYl4SdWYFDomGjqVltGRbVRfTYtwyXdP071Z7kve9AX55OfZo5e+dI+nujzoxiv9F+xqyX9qv2LdnHlmlbuLnppsgWPDa4Scqj033iidoiv76c63+O48OOs2u6oW61uYM5fV8aQEpQJje63KCwt1jP++/g6CtHyb/W+r3oOKUjD198WFKnNvCt+4ke/TIqSwcUNoY+tNzSiM2zNnO512U2Tp3PxrB3Db9saUmNrQxBJmCkNkKnbvt7ZelsydB3h3Jj7Q0KoqVp039/8ndubb0l6Q+eef8MV76/gq333aWSvx3/RNz7fxWr+qxi35J97Jq/S9JeFFfE0VePSq4JWiNTCgL6kdxzjijOul3HXrLYp9YRdnYlXU58A9zdYh/otQhPLj0paZu5cCYvn3yZhXkLJfXPjeRGxHSJwWSWOEch7yDn6PCjpL+XztQVU1v9+05++aT1vYSxQ3NuUScIKM2VHB1+lDqLOoYfa58u9Z8hcU8imaczJQs5xqbGmNWbEXkjkrJksQa5S/olgi7/hlzTwD7DQV8EGZwadIpbYbfI9tegdjS8ZuZdQ7kx6kViOp3EqtoKVdXd1SK8h7uHf3vtu7fi3sO/Bq1WS2pqKlqtlpSecwAQToqT6zK5DJWZPtCvTRFvfDIZVLiGUOjfW5LyJONSBs75Siosr4iSMwoTBXkjHmDr3GSODj9FUohYBNVwGqTt5ydXyBn1+SgiFkRIcvHLnB3J7DKBYu+uTTSOLVHg34dyt47YBhomPhtqG/CN3Y9b2nmsS9JQlRsG71dO1GBcfQ0zZSOHv+GB73xgJ3EPfIxTTjRyrUYy4dwabH1tGffDOPo+L80NvX7Ueo6/fpywmWFYOBsmZBtqGvjE7Qs2v3KdZesCRIlVZYcI4gYuaZWyNXJBJBeHXKNDYgcGnxyMWinWiyiIKsAl3IXBbw1u+0m1AXcSVg0YHsDEnybS9aGuIucL9LST11ddl0wM1RTUsP+J/bxr/C51xXUi3cnGglX2uWzWDF5zxyCkvdA1aLAqyzLQh7wdL5W+xOv1r0vqTMhksiaqneT9ySJ72rE0rnx/BQCFqbjr8/7f72epsFRyGubPkHs5l3eM3mmiKGqJkMkhDHprkOTfbahpIGFHAr6DfQ20E2+HUVd98tozUdx8cOy1Y9get2Xynsnk/iju1IpcEEnYjDsHn7evfXcLLuEuLIla0upEjCAIbBi3gWWyZSIKjUb4DPBBYaoQ0SEHjAzAsYPjHadS/im4d3MnZEqIZGdxQ00DDrmx2OfdpORympgGIycX3zQLSh1LSQ5ORmbcnCjXNmjpeP4XQi7+ilynJW7Qo+06rl5P9cLO347QGeLiPcDlby9TV1onrTWkUKBTKJAhUJ0h7g7u0fsDyr2nERZdx629hjQtAjIyfTMpcSrBptKGQk1zs4pRQx1OeTXYVOr3y+TgZEo6tZ2+TkBAJ9exeuFqKu8TUxQ13Kpk0u5J2JaqRNTSAJaaHuR75KOwekGUNGhcP8Piwpi92pL804bJV835bN5++206xnek2uIm1eoK0e/f6FFPjuOvdDi9jIaytk/BmdmZMfbrsfR+WlqPrKqwCrsKO24VS1PibBi7oVXaSOcw5yZqxnHfjxPbL+5jwXJ30v3TOTzmCpYuzZMzgiDgnBWFW6Ebk3dPJiz7szafU1sQvSaauK3SE/h2PYIoDOiD3409ZEpQsmJmSo2dJ9EjX8B/rmGXe0FcGaHnVlFj70XcwMW49DOkGjLzdiZ62DR2T97NqCOjaDjfvN60Ze0zNjfG0tkSmUIcPGtVWpSlStG73ghLZQib5mwipp94eqmxKJbrmUtCJ7VBEhugu7WeDqzCvoI8zzyEFhGSa9oFlJaOlLt2JPzU8nb5gz4DfOj5RM9WtaAyTmZgZm/GfzL+I7KVJJYQdvpHKl2Cm6hVb4dlMow8PBKbShvUgrhTvHN0CCGJIZwL7ctZB0MfuVFrDeDcgHOcHirNTNAaJiyfwGs1r0nqBOVeycX71mHK3TpS4S5eJ4t9e5DQbyGF/r2xCvGkrqQOda26SffE76CK6dung0yHuoWrJwiglTdnqnQYPg9uGZZ0v9odhUaBT6YPiqK2V2arcqs48cYJVvdfTVmqeK9M2pdEXUkdj1x6RHISLqnPA+QHDUCdno26zvDAdRodigYlrukXCN7+CbmXDX2IxJlVvPfme2iM9eu2Vieg1QrY593CpiSdS70vER8WT+eYzkzePRFNrbio3RoadTlbKzjLZDKUpUrJCSOFsQIbTxt6PtFTkia/wUVBtk82lQ4KjF3sRHavK7twybhAssd7XLJ+2cAmCIBMf39+WvITl17U+6Od53Rm+qbprU7ZgL6J5MSQE2yftp2Ex8QU+qpKFWlH07ix9oYki4SpjSnZ57L50OpDSUo0WXEhJsrK5uO8DZb+3pweXsbFPhfZMXUH8iDD87YuyWDMjiJsqm04M+AMcjvjNvt9wWODQQbx28WNogAen8JDvzxEpxgLavKqmppIGidNU8tSKXcox62XG3KFnGHvD+PZ7GcxdzDHtpcd6+9fT7p/OkpTccxrHt6f692zcOp6FK8Xv6AmtDm+rEguJvjicsJjwyl1LCXTJxXa0fvlEu7C1HVT8R8mTVm5c95ONk3aRGmy2C+y7mnNnol7KHIpQqMT+zXuXd2JXBjJ/KPzRTYTSxPG/zCeVypfoTi+WFRQLEko4UPrD8k8lYljB8e7yl5RklDC/kf3N2kCtoStjy0N1Q2UJJRI2u8EoaF5fbFP1zM6ND6nwm3JZ0Gm38xa5kByLuaQc1Esd9AWKDQKAlMC8TvpJ1nwHf7RcNy6ukk2dF7OvcyY/DEc+e4I09ZPE9n3jnyaaz2ukeuVS0aA4Rpp7mBO5Ct92DhnMzun7aRcQrf97yDywUiCxkpTrhbGFrL9/l38/vIpCq6Li2oNDYAgYKysRFVh2JgrN5ETHxZPgbv+e0rBsIk2Xr2aLLtXsauw443336DHAekpOynI5DLs/O1w7eIqGfemH9ez80hRFQP0f7k/o78aLZmjEHQCa4asYfPUzRx/4zglie1/TuGfiXvbi/+Vyb5G+I+QXgd/GfAL5z46J/IPALQm5uSEjZTUQb79/ZY618YmB+Uf2s93s9hXcKMAG0+bVmV0HMwdiHSLxN9ef86lSaWkHk5FXaemuKQYu8ftsK62xmi+Pk5fO2wty2TLKLpZhCJQwdmBZ6kPqcfUWkwdMH3bdJb8uISdw1JZ++TvWHjeppNcWEu/8/3I8Mtg45yNrHp4Q9tPqg1YfG0xQ94eIuk39bXvS5VNFV8//TUdponvl1lNCXZFyThnXaMq2zDmlhnJODHsBLc66f0SZWt9/jId//nqP1R+KS1TcQ//Pv7tte9ese8e/lVUV+uDunprZ9QmFuArntSRGyu4NfgxckOG0vGdOSK7prIW26IUXNMvSdIqnJqdzefPf05AYgOqxAzD31bIyTMLoMAtl7MDT5DrZxiYK8uVfOnwHj32LaPHvmUkfSamuLsTPHt6MmXNFLx6izUsAOrsPMmMmCRJWeJ16whu6RdRaFTUFRkmNk2sTUns8wD1lo6EXFxH2U3DDnghKZlexy7RMaEjgSmBmFYadsrcXuRI7T4LoR3Rmlwhp+djPVsN1kJnhDLgtQHM3DJTJHaPDLC1RWssXeBR2FqhtHXDI/EEbilnJD9jgycdkjow9ORQNMrmgE+n1bFMtozKrEoei32szefTVrwjf4dlsmV/iWpl/PLxPHbzMXbcv4PYjYYbsrJcydUf9DQlUT+LKf8cghyaBI8DRwXi2OHuBTbasko6nv8Fl4zLkna5Qo6RqRFyhXirsPWx5cWiFxn4+kCMzIxEWoP1Fc385fftuO+uHTPopwoFrdDqFJtjB0dMrEwkNRW0ai2ph1MpviVOcoC+kCjUq9AqTEiPEAtN394hPjjVsKCs0+rYs2gPUStb15hpROPadzexvMtyDj93WNJ27pNzpPyu12NomUhR16lZJlvGlmlbeKP+DVHBWdAKKMuV7Ji7g7XD1krqM/1TmPLrFIa8PUSSD19VVkvA9R0ERm0n6j/rxO/mhQsMPqzgRsQNDk3MwsypOTErU8ibkiGlXl1Eeq1/Bq/eXgg6gbTD4okVgLgtcdzaIl080mkFjBr051N8Xlwot5a5Euf3Jt8//j3eTxmuswoTIzbM28D5/udZvGIxHQ82T2FaVOQx4EgxQSn6RMXJoSfJmND2JLdaqSbsVhgNJg00eIrXuZTEG3S73o0lPz/Mpa/Fa4ZNhTHORc7IWlZJ0CchX3v/NQafGkyJiwYjyxZ6ro6WZPspcChz4MFfHuT6uV0Gdr/rOxn32yXCb9hh0tCAVmi706xt0FJfUd/qBMLy4ctJ6JjArJdmSU5Oj/5iNNV51Zz/TNxcYONvw8rRKxFOCPR8vKfIrrJ1JsdHiU6uo8hun2iiMbnnbFID8/HK9SIk+q8l3VrD7oW72TZrG8oy6ShRJ1egNTJFJtFtaDF1DAkDHkFjaonc1NB/kFtbkBc8kNyQYShtxZqQggAKVS1GGiNWL1yN0QTD5p0/W/uuLr9K0Lgg3LuKfzt0WiiTVk0y2F9uR8fELnS/2h0rpbjhoi67isU/Lca62pqkbpGYOxsWapyPKHj77bfxzvLGRGWCWmPoD6Z3nUZa9xmUe3QiL3iwZMH7Thj33Tgei5H2TRYcX8BLpS9J2owtTKhyCaLeUnrvt8iE/uf7M2nPJGzPiunxdi7MYsusLXSO6UjQJcPrZlloztDjQ3EtaPTV7t7UtkOQAzkdh5MXPIgGc3ExUGtsRo2DD9mdxmAX6YdzmDPPFzxP32f1jWTn+p8j0zeTlz96mexDhmtpeWkRgYnmeGd50+NKDywyDNc5/xsWTNw3EYs6Cx765SG6npNOckrBOfSPAr4MSbrAvKt5XPvx2h1ZFpyzrtHww2qRj1IYW0jXw59gU5xGsU83zJwMpz8ap/UCUgP4z1evI7/cgFatIzBqKx0urWfxisUAeGd70zW6K7oGccNba4h4IILw2eGSUzqgp6Mf9NYgyalZTb2G0qRSgscHM+CVASK7qZGAQqPg+DgBhxniCej0vrM5O+AsQclBkH/DwKbKLsI+PwsjtRHIQEB/L7POZbF99nbSjkrvsaCnZT815BSxXWLRGYn3Or8hfsw/Pp9b225JNqVVZldSU1iD7yBfUXOiIAiY//IjwRfX4XXrMHlRhsn9Ott60jvI9PTTwQI9gw2LFrZFzb7ipd6XEIxlbfb7TKxMmLJ2CkPfE0/eAdT7yIjvGM+vD8fgGOGFjacNMrmMmrwaBJ3AAw8+wJinx5BVnIUgCNzceJMDTx5Ao9Qwrd9MCr1MsaizwKZMT7l28D8HOfOBPuaq7ubGwdEbMJOBwrgcpbr5OdeqNBjXZWCuNOfcgHOsf2A9inboRuZdzWPn/J2tFpgaG2ekpiDM/c2J6h6Ff7o/1afE1zFkUggBIwP40vtLvvT+UvL3t8/dztkPzhK3ybAJprGRTW4s58nEJ3GLEE+e/FW4RerlH1prsvPq68Vzuc9JNlX+Gcy6hlIQoF8vq10CgeYEfcywZ4j+g+a+2K+nga0Rq/quYlVf8aQu6DV/3zV5ly+8vpAs8MxbP48H1j/A+APjJdfJAS8PYEnUkqYmg9tx4MQBvM54kXIzRbJp11AE0PCgVRoVnp/bYV1lgonKhAMO0lq3fxVm9maSbEygpzPOOJZK3qkUTn4nnlisSsjDrjCRiGNfkbnKUBrgdn9h7oa59NprWFB0PpnCc18+h1wnJ7FjCQ6h0sw1UtCqtbiEuzBj8wyR5jrQ5Psm7E6gKkdMs+jYwZFd83dJTnULgkBxXDFJ+5M48/4ZyXezrfgn4t724H+l2BcyKQQ7fzt6PiaOGQD6v9Ift0i3VhuHTWtKCTm/hhvrDPfS289Psq4gl3N1wlLihjzR+mda+b0/Q+GNQlIOpmDuKN1E/a7Hu7zq+Sq/fPcLynIl11ZcY/3o9Zx+/zQ1qhoUWgVynbypkSPzlD62VZgq9M2iApw4foJbceK42lU+GKW5fp+6qATNbWxlZuVmjDg2At9MX6y1M/Gq+6HtJ9UGxG6IZe3QtZINKtXnq1m0chGeuZ7oFGL/JKvzeNIjJuF96zDqXMNm6kaNXyO1EXNX2pG3wnD0T1NSjn1eHFbVVlzpeQVN1/ZR1d/Dfxf/5tp3r9h3D/8TMKkrJztsNLLJ0kLLCnU9noknSHhro8hWn12MeY0+cZ9xMkNkj67dj5HGiB7n6lBeuCGyA5TanAQg3u0tw/+fVGqQIGypM3QnqJVqfgj/geNvHJf+QFoaoWd+xqY4VdJsUaUPLG1K0lEWGxb75EYKqp0CKPHuSmbn8Vj5Gzp3cm9PEiIDMVea88D6B7BLMQxiG6kmq5wCqHHwbteGHrUyimWyZa0KOQ94eQDD35cekzexNCHkk4cxrq+h+75lIqFoTb0Ghboex5wY7ArFtDuHXzzM9E2TOT3oNOsXXDakRPvjHKpyqnh36HFOf333dRcaj7Eljv12jAe7P8gy2TJW9lkpsqdkGrN9h1zP899C6Nwx2JGHzj+EfaCeBuv0++LpVudO+vvrN8xPsoP+L8NYT3VwOyXb7cg8k8mpd09J0uM2Qm4k59a2W6KE8u00OlK89L+N/41lsmV3TOS0BlNrU2RymWQyGPQ0nkdfOiqpd2Jub870TdNBQFJvpyC6AG18Mqk9ZklOUSy+thidlY4K2wqEzw1fHEErcH3ldfYu2tvuc/q7qC2qxaOHBz6DpAPElAP697Xz3M6i4Pv2bk6prn8Ta5Mmrb+MExnUl0sn2v8J1BbW8rnb55x4SzxlaepgSUr3mWSGjyXoP+NFWhRCz15c6++JIBOQCUaGegYKOfH9HyY9YjIl3pGYV7ZvYlan1VGZWdkqzdas7bNoqG2QLBAp8yuwL0yg0jkIt/vEdMUN+48wc8tkil2Kyag0LELefg5Xel4hJbQ5CaO0dia6jztZPs2TWi0nXu4EZbmSWVtn0S2qG4IEtbN8kDk/P/Iz8R3jsfEXd2sPPdKBJ354gjFbM0loQT8qU+gnEq91v8bhidU4djV8ThO841j14JuUOJXgXOyMrtbwuHVG+kDGqtaK3+asp9ZKmopOCvE74/nY/mPeM31P0q4TdFTYVZDrmSvZ2NB9cXfqSuskpyxiCmPYn7yfd069I/nblR17k9HtUYJSgpix9T6D7k2ZTAYCWNTp9zHru0jRDM3avFINKnUZRZjVlpHWfSau3cXNSIIAMq0Gs+oilCWG+7SRrRX5wYOxKs+mx75l3PzaMEFUdTWJvgc28Mw3z5DlmwWu7QtCSxJKqM5tPTA6/e5p9i3ZJ2nrfMONifsm4pckTmgpdEZY1FljVeeLR/kMkd9j7GbPrdBb9D/Xn9c+fI2aAsPOe6WNK0obN32xL2QI7SmMxayPaTUh0Ii8K3mkHBT7VdaeNqRHTMEpJ7qJju12FPXR8NUzX2Ffbo9FmphevWfVBsJzNxF+M5yul/wMbA3ZVQw+PRjXQleGHRvGyP3S06+t4ZvAb1gmW0b2BTHlpa2PHQVBAyQLwqD3603qypHptOh0el9i+5zt7HlY31SX1DGFKz2vkO+ej8zcsDkgJf0K03ZOIzI6kgn7J2AXa3gz4/tWsW7eOurN6rnWV4Z1n7ZTvRlbGDNl7RSW6pbi0slFZO//Un/mHZpH7uVcyXfLNfU8Co0K+dhReg3g22DuYEGxd1cKA/qS2WUi1j6G66hlroKg5CAEmQAyEyyMbRGQkdptOlVO/jiUOzDgzAAOjzrM2sUZGNu2XbsU9FSds3fNlrSd//Q8V767ImkrjC3ku5DvJBvSAHxWmfHme2+S7Cm9Dla7h5DnmcfEvRMZfsywgFV94irhF45hVm+Ge547Vpn6NXjdCD0tcuOk553gkeuBSZ70OuPU0YlJqybhN9RPZDv8/GH2P7qfuQfmSmpAqwcMQWnjhlvaBVHBI7H2IumuXwMgE4xEU1OF/r1JigznzHA1QUXf4mFuqK1+JzTUNuA3xI+Ok6U0OMHvoYlsnr2ZXNeLyE30PsJ9u+5jwGsDEASB0XtHM2/DPNa7r0dVpSLjRAaJuxNRK9X64xTkTNk1he4X9InT2A2xJO7RT7vJUaAArmQ6cWX7UKoTKpr+rn2YO5++9ClXekk/J3+G1COpxKyLkdQyBhj91Wgei30Mn4FiP7ZxMnvs72Op3iC9P+Rd0Te+tqTwr86v5jPXz5oKvi2LqAEjApi1YxbG5sac+/Rc+07qT2DuYM4T8U+wOGqxpP3Ii0dY1XdVq3rBd4IgQKlnZ4AmHa5GuGRexT3lDMb11djn6QtTLZ/RER+PYOSnIyV/uzS5FJ1aR3VuNQ3V4nUuLSCNDN8MTj9zWiTJ8afHnSAwYf8EHvvxMT6y+UhkH7u3B7M2z+K+Tffx+BeG61XczTjeeP8N/vP1f3jtw9eYv2Zuu/522tE0Ti472WqzRnVuNaWJpZLTnf7D/EmLnAJAbp0+Zi+4UUBxvD4HVbT+CP7Xd1Dg3werLobve0Opkpc/epkRR0ZgU2WDWa3hNSvx13KhzwWqras5NiYdj9Gd23xOdcV1/BT5E+c+kX52g8YEMeitQWhVWsnGSRMrE8LnhEvSK8oVcl4sfpEXi1/kiYQnWqXg/b+A/5Vi3+zds3km7ZlW7QNeHsCS60sk5VFM6irocGkd1mWZrVJyg3Qhr+X5/xkVfXuuV8ikEB67+ZjkPgqQ75mPWZ4ZWU9lUZpUSlW23keP2xSHsbUx3z31HXKdnIY39GvNkugl3H/wfhwCHVCfVbPo50U8vvxxjr0kLkhnjvDh1wW/ErKvA19+PovypIomm3tHb3574AI3Im7QIM/CqbzbHZm62gtjS2O6PtJV0k9x7eTK2f5nkQkyqkuk96wqp0CSet2Pqb/h0IdaqWHRypcZc/Rxau29MHI2pMpvSMwgMGob7vnuHBpzCNWw9vmC9/D/Du4V++7hfwKBV7cQEL0T4ao4AFDXa+hw8ddWv2vu44zqj67h0++KCyUuBTZ45Xixbt46TIcaUk821DTQfd87TN41mUUrFjHkeD8Du0OgA8O+HE9WpzGSgst3hADFccWcef+MpA6LTKfFsjKPgGtbyTwtTtAk9XkA5R90llq19I5cb+1MsW8PzFwNC0BGHi5khHoSHxrPoTFXMA8y3ERkchnOs4ehMrfBNfU8mlZ+XwqNlKNn3peevAPIvpDN/sf3S9L+rVwJdTauVLiFIjMyDG6K916g66GPSes2g6TeD4i+W55SjmmBiirbKsqcqgxoQOVGcuYemEuJVwS6k2c4/ZN0Z95fxRsNb/BGwxuSXUvr963HP8qfQpdCfAf5ir/7ZAUnjqjx/OlNhrw9xMAmN5Lj3debp5KewtTWVKSPl3oklX1L9jFn7xz6PW/4fP5dyCz1jqROQgcI9IWfk2+dJGmvuPCqLFPy9uNvs05Yx4tlL95RIzHmV3FBJPmAPug2s28/jWfQmCDe0r7Vqph43tU8jMyN8OorPVGbfT6bq8uvoqoST6iFzw5HZmuNZ/xRzCvFNC2m1qZUPF/Bjmk7MLI3vG5yYzlefbya9Aj/m8i9nEvOhRzJ4AD0tKmvVL3CtA3TRNSppjamLBWW4jPAh8wzmaKJyNjfYtk0eROD3hrE5DWTsXT97zmVV3+8itxYLqnPqDA3ocI9jGK/XtiP7CHSYBS8fcgJsCI4OZhBxztQV9xctBAE/WR1qXckPrEHCLq2pV3Hdfh5/QTlsA/EUwwAZ6+YUlZtTIkEfZzc3IQSrwhKvCIQLMQ0cKWl0TgX22FZY8nJI4bBXINSwwObf6Df5fn6iYbuGU02tbkt9S5zKXHWJyUHnBmA3962FySsHa3RvqGl15VeeH7pCcCRl46wTLaMhN0JKEwV5Hrlsnn2ZnxHiLUIaz1GkelXSYVdnUgb0tLekpuzbLnc+zLlVuep1xgmHBoTeUkhSXz60qdoexhel6zO41Ga68+l0LUQQWj7nmUfoA+SWqPtdUt2o9q6mnXz1zXToZXUGSR7Ho1+lLkHxEml4nPFLP5pMUvfXsruhbslf1+GHJciF8LjQqlrUSiX6zRY1CpYO38tu2dIT1j/Vdz/+/28WPIi1p7iJqX0uHN4JRwjyXG6iCIRQBUVh8/NA4SfWk7mNsPEbnAwyARdE82xRmmYCNSp9D+YGpCGXCtHq2kfdckDRx7AMcSR8nSxVmxuTAnJxXbYTpV+744MPwRAYJxY7y8oLIKUgZ/hXfkuAVe3UNmC1q8+0pwt920htnMs1yOvIxgbhkgyrQaZtrnZp11dz2m1ZJzM4NSH0jSZ+dfz+XX4r2yduVVkEwRIc3oZm5J0fOLEuo5qMx0V9hUsf2w5Jx4QJ2WNK4sxr1azd+JeNi00TAa6dgsnZsgikjok4Zfhh3+yZ9tPimY9X6kmKK1WT1PUbf+7eEhQYjtlnKfL8W/odPwjCo/EoFaqKU0qbZraNMaSLN8sNs39HesIw8SpzMWMjbM3crnXZdbOX4usBW1rpWsDqUGpqE3UlHuPw7KTNBOFFHQaHfUV9ZLnBPpiYNqxNPY8tEfUsAbgkXQSs5oSMtz7iuinrTxtyYyY1ETB1fLdCznpybwN88j0zWTDg+fIMO6ATC6n3COclB6zqTPXYFVjhcZYg3/JW+3Sz41aFcXOB3ZKNl8BzNk7h4CRAZINWNbu1gx4dQDxO+JZN0qsTVrsV05seCyDTg2i5ry4qbLxXTkw7gAnhxrGaeW6PIqci6g3q2fCvgmEb9Kv/d0W6zWfM05ltHpOqmoVC1cvZPHPi/HYIGZIqSmsIfdSLv7D/SUnXsJmhjF4qTTlv0wmQ9N/MNf7ubPnPhN8RwUa2mMrefaLZxm3fxyLlo+k9LThtJjGwpYqr+kEZg5jyrpMlBJME60hcXciX/l8JVn8BxB0CkzrTbEvs0ZV3UB1XjVJ+5KoLaxFJpexdv5aLvS5gHyUXE/B6q1PRGpVWm5uuskzX0yhzKEM7QT9+64sUzbFU9WHf2PGlpmMqnHC7sjg/4+9tw6v4t7avz9b4+7uCSFCSAju7lqgXlqoUj11pbTn1N2dCpSWIi3urgFCICGEuLvr9v3+MU3CZHZo6OnznPP83t7X1V4ka8/OzN4zX1nrXvdNe243ccJ4RXY4JiOGiXsmXpNnX6d3aW/7BRsXGzxjPS1KtdVsqeHB9x+kYEoBN398sySeuT6Tk++eRCaXcfOuHnEzXc/q2JVjJR6j1s7W+Cb5YtQbufDDBVrK/1oWvlukW6/kRIVagaZR02u3+tVgqBZ8+XRW9mjsxMQBj8LTeBWcYsDedwhL3YCqo1kyZ414YgTDH7O8rxx09yCe0z3HC8YXLNpTPP/F8wSuCuTx+x+XWHJ0Ku2slK0kf590PDEPNHOxv2WJcQCbDjW27bZUeVVRGthjH9ZjzNRLb5Wr4seZP3LoxUM05EnXFyB0wQ1/YnivY2vnb7VFwnl9nvA5n/QXuoVsJw6jOG4GpTFTcB4htlTwcwqhwzGQZsdmPrv3Mw4vFkvJVvQ3sGvqLvRqPdXO2zhY2XclKbWDmsiZkVScrbDYuQcwdsVYXjC9YJHAsv2+7dRm1RJ3Y+8FRmsna9yj3FHb/9/tIPpvKfa96fEmK2Ur2f+85WYAo95IW3WbxblaYdCi1LVT2m9ClwpCJ/6os8/YoWPQ1pXEHBLu17+y2GftbM1vS37j88TPLcZ/WfQLH9/3MRvnbcQ1zBX3aGEOGP549/jjUeOBOUv4o15xXoRPCUcml2HSmHBocaAwqBDnqc6AoB7V+fn46EcTXfIWrvWuRF+Kpr2meyxV2amp9Wyn1aGVkfvtSd7x8V8q03xwxUGaCpuw85DmRdr92hk4dSALNi6gbIdUktWmuRLb5iqa3UNROYuPV6LAq8qFwMrJ5AxZgtP0EaK4VVQweYnXUeEjEJUTnK4hP/03/n+Fv4t9f+M/BplMRkBAADKZjEbv3zfqx6WJEJPehHW75UUZgNLJDqNSSF7P+nK2JD5pzySuW38deeF5yLzEixyZQvjbFT4V2LXZYaUTM61s3W2Jv30Q1SFD0Nq5XtPEp7JVccMWQXbUknyHLDKCnOQbUBq0kgJP12t+N9NzCBRvTFqr2kja+hIBGdsByxO2WWag0aWRjPgSrLzFm4CMtRmYdHrsGsvxv7wfg7bvidPQiaF4DfDCJ9HyxmXPk3tYPXk1Zz49I/HIMGgMeOceQWfjSN6gRagdexQdgrypCUxEa+fS1clxJRZvWkzRo9GodCqsOuSSCTtiWgR2jeWY5Aqqh0rvhX8HCpUChUph0YS3fGo5Lz3/Ep/e+ymT3pCyJP2y9hNz5AsK86Wfs0FjoCazhmNvHGPhuoXM+VosHWnUGak8V8naWWv5fODnFrtX/yzMCgVnZq6gcKBlw2P/pVMwubrhHCNNALZWtSL7VEbBwQLOt56XdMQ0FjUy/AlhEZf2bZrk+HvT7+X2I7dz+OXD13xNrZWt7HhwB9nbpEVIAEOHAZWtChsXaXLfoDXgP8SfJYeWWOyStHK0wtzWjl1zJVEnvpXEqy9W47rSlWk7pvHuJrFUkEwmY+mJpdy046arnv+VY9+14OjrR7mwxnInmVe8F7O/mW2xKAbCeNRa0Urx0WKLvgwAo18Yja5FR84OsdxV8JhggsYEETIuhITbElBaXZvk5b+Dc1+fwzfJVyItCuKNyH4LeyaT2YzcrCKoKIyEc/4iDw1dq47+hz5j0NaV2LZUURFu2WegNzgHOxM6MZTBywdbjG/8UculhBs4qZRKnqmc7CgcMJsWj1DaG6WbuZ2LjvDJfZ/w+FuPY7NZLPtr0hkJu1RNVLawKTf36NyTXbGsi8iJwON839niKhsVSTcl0T6qHb/hwjNffkYommgaNdBsxqfcByuNlcgIvhMV4S6sWvIu3y1ZQ+hEcXJepVAxvuNjvCq9SDrpyG+HvxSfd14r07dNx7NKmKMtMS+3XO/AN7d/Q0JaAvrWvkvJ+iX7scK8oleZxElnJjH317k8MVyIN5U08abHm10dRp8nfs6BFQcsjic6rQ6bDuH3lopqiuI9OBS9xLERx/jns6/jFt29BjHqjASf/w2nZidCCkJosv1ru9GVNkpsXGwsdiueCT3OprmbcK+WUWtBlkl78hweJecojxiD0wAxgcXbtgLPjKfIjqrn3IR7iHlULPvnNDyGMzNX0BT+Hi+8/ALua7rHi76MfWUpZex9Yq9FWfbdXxShupjG4XQXC0dCmX82n9z7CWtu+EbicWE2g1GmRd1WimvlJbRNmq5idkN+A7kdQlHzYuxFfpv7GzJH8Xqw/5HP6X/0C3yyDxF1/NtrIkn9UjqMep/+pK86a/He3vnQTnStOmZ8NkMSaylvISRL2NDX+PpKCkSTFQ/j3OCMWWbmpNtTkuMDdn5FfEoJzU7NNLmIi1OVsmI2jvVDY6Phmzu+4YuHfu3zNQGsMK9ghXkFIeOkxbSSk6VEnvwBudlkUYI0P/g8WVFZWGsMdFQ2UJ1eTUtZS5ePzaKfF3Lbt7eRmL9OOt7YKrjc7zJV3lUUhBYQ49uzEN/9GZ8PXsqxJst+WZZQca6C111e5+vhX1vs/m+tasU3yZeFvyy0SHzJGn47xTHTqLZgcdvpkeNYnUtI6gaqLoq7xYLH3cBvs3/DLDNT53AYa6eWrnnOpFSzZlkBO6ftxLHJEYeGBky9eFdaQumJUi6svkDKR5ZJBZ4xnpx45wQpH1qQaPZ3ZMIrE3AJdbEo41Y0uo4N121g1JFRtJ8Sy9uZjCZit7zK3E1zudzvMvlhBaK4tqwKx2ZHDCoDx0YcI3eiUGiZ8ckMgsdeXc3CZDDhVueGGTN1o6TjReW5Sn6e93OvChIxC2OIWRTD3if3Un5WShAAOBI3ktToZ/gxb7X4byuhxaGFDpsOGp3rKDCKr1tmMuJZcArf3CPorB1Q2Fj1ed3XWQzb/ZhlWXbdjgLu/vxu7vnsBqpOF9NR30HqF6mUny1HJpNREFrArqm7MD9jRmWrImRCCCOeGoHaXk2GMYP8kDxODj2JZrgwjg5YMqDLp0xZ3YxvqT+feeVT9MRHOI7snrPaapuIyYjBrdaNiJwIRh4biUHXd1/ggOEBPNX0FEMesNxB/Lrr63w+8HMMWul7ypQytFZaavrXWLS+qDgnjJHTPpTK0Dr4OvBM+zPcdfYuYm+IlUhHGnVG0temYzKYqE6v/tO+ZJaQsz2HlbKVpHxs+bnrN7cf2mYt6T9eu+eSNruIkLRfUWtbcS0V7j+zWfif0tBzjWSW5Aoy12dKrCU6IZPLhD1vL0UvG5UNLx1+iQd3PGgx3unpbmm8ULmr2D15N3sm7mHZ2WWS+IZFR/j29m85OO4gm64Ty2E6hjny4osvdv23Y+FZi3+/N8QuFopwltZrABk/ZVB82IKPMYIajNyoR2ftSJtH95znFiXMb/J+kdT5C1K0PT9rK2cbvrttFSlDhPtA1kMZoPNn6w5r5vw6h+Idfb8uKwcrgscHU3KsxKJ9RUN+A+dWnevV+sLGzcai/ywInn0Xf7lI4cFCWspbepFd/WP82X3vX4n/lmJfJ2neEqkBYOONG3nL6y2LnfQdjl6cm/YMleHSveWV12cxL/j7C2xaanp9zZ+FvkOPR38PgsZ07xfy9+ZTekqQbJbL5NR41nBhwAVs3W0Z/fxoHsx/kP4L+6Pv0JN0Jonjw49j+FEY+/N257HzkZ20lLeQuHAU7zz6Dt/e/i2OkwTiyusur/O68+sAlBe8gn35ek7cepIX16zEOVEg9mibtVSVVdKmSkFhUFAYXEh+5F97/016YxLJ91uWY71Uc4mvq78me262xZypV/5JIk7/iNyCB63SVs2xWQs5N8yRdnWhpHir9HChwTeGVvt25m+YT/vPllXiekPJiZJec2h/46/Ff3rs+9/L2P2Nv9EDcrkcNzdhgVQZPgq/7EMWX6e0VZE26TF8sw8wdIG06CCTQVXYMJyqcth3yZfIHrmS48OPdyXjJIOltYq8QYtJGXQ9KUNSUBqlm8rCfXm4lrVj1VZP9X43mBQreU1vCJ8aztOtT/eaHG/yDOfs9Od44VZpMs6uvoQmj3BqAxyQ9fDMUagVNHpHYdXeSOyBj6ibNhOGBHfFTRmZDN+RRqmtH2X+F0hvOcg8xnbFU79Mpe5gIbZAUex0SYfdH+GetHt6jaWtz6fNaMXNZx4kLE5czNN36PHP2k9NYCLNnhGShZdtfARF8RFYt9Rg3VoLSNmfB42vMHHvRIakDKHlmVYcf1+wG3VG9j2zjzYnT3LiQ/C0/mu7jzbcsAGj3sj8NfMl32d/z/7sLrC8KQeo941Ba+eM9YUC6nNdRYbl9bn1fBr3KSB0uT1Z/6To2MgZkQy4dQDnvz9PU3HTX8pIMhaVEX14GxURowCpzNW63EQYnojPebhHTCDDzt+Or+/4GqPCSGNGI3o3vUjKpehQEcffECQM42+Ol7y3Z6wnZSllXP7tMkat8ZqkQepy6kj5MAV9h57IGVLT42kfTmPyW5ORKaQTq6ZBwy8LfyH5/mSLXZg1pVry4+YQem4jHY5SSZOf5vwEgE+lDzeuuRHe646ZTWYqz1di62Z71QTVlWPfteDIv47gP9Sf+Jukn6dToBN5O/MoOVbC7K+khe7arFo+jv4YgJt33UzYFSx1g9bAnif2YOgwMOurWV3s607YuNpQdKiI3F25+A7yRWWn+l9btCw5tETCGu5ES1EdCbu+QmYyoj/gSNuTt4vYdY1b32Z0fjmf3HeA08mnmef3XFfMbAbblioATHIVtYGJ13ReQx8eytCHh/Ya9zv8Eya5goZ+Uo8ukwlUmlYG7HuXKsUQuGWq5DVGhZGU5BR83ZaIfq+0VXNh/IPsTPJn3sZ5aGy7d2v2dUUEX/iNIqv+ZMZksvrm1YxSPQzAd+O+o6GggYcLH77qdc3pN4c5h7sJB1PemULpyVIipkcg++de7v7ibhqdGslNzGXQwp4drML3ZEY6RmlbtFgdPsxI7UjiMuJwHC0e25WVBgafHkyVV5XA7nQQJ6pcyi8SXqtFb4phSMoQjAv+OsnLhgUN5MhzGLl6JNohWow6IzKFDHuf35MfZqjPrqchv6GrS7ATtsNsef9hQcpN/7w0CdLR8CMDM4LZPkWGXt1BT+nHVtdAnGryGH1kNKWBUnmafwcvK1/GytGKx6ofk65BHJWYZWbGHB5DzcUaggeKCUHW86Zx+qiWdmdfnBPEh+o1GtxqVFwcfQJnq6tI7MlsqfONJTqqOzH7R2Of2WRG36Fn6vtT8R8q7cy2jg0nJ/lG2i2MzQBDG9/lpNcj+NQvor4e/K5YMmobOpA1vs32MR+jmjCDJf292TxdYCG3VrZifVHH9JTpHB15lGanZtzU4s15o5dASlNrmrFurbEoddsbKirAOmIMNUGDLJ/3I0NJWJJgcWxvqGwgMTWWA2MPYPZ4n6IiCL/i0TMfzOXh3x5m47yNVHlVSY4/NvIEtS5F2LTbYNsqLmDqNTrsWu3QWmkxqAySBOS/A32HETOQP3A+9X7SjoE6bycODBPm05+SKnHwtWH086MJGCbITBuURtybHYk4X0hzVjlM6r6PriyYqgzOkrXkrA0TWJDrzxtPvMHN345Al5wFS/t23p3deJXnKmnIb5B0y5/64BRHXznK8qzlkg5mgA4nH9xKzhGaup7qzPl4XiEpWJbRQEjqfpS6dpxq82muHMSV61xVRDDnnM7h2ORIv6x+VMhPoW0ZzYDdH6DSteHUL5I118OwE8MYeHILumVhQN+k3Wd8OoOKsxWcev8UI5+UJgq1LVoWrF3QlYi0hAU/LrD4+5u8XieleTOf3fMZSzylnX+NHo7UuXUW48T3mE2RkZwgQd0kMyYTR5VwbnKFnNsO3HbVa7JxseGtx98CYHaUdN3jFe/FsEeHsWXZFjBD4jLpPF+fV8/xN4/jGespKiKZjCbUa75jYtBEzgw6w+WSLLhiyjdE2fDVnYJs/4HxB3jQX8ys98vciVeB4Me9af4RRjssxM1N6l9mCT6JPkRMl3bPd6Lu0BlcG1w5MvII0/2W497PFZlC1iVj2QmdUSATuYS4oFArUNuraYvu4MebBLl2vVmYZ4c/OhyTUVhL/HrTblooo9QHitxrqbEXPJM2L9tMTZWWhdsWsmPqDvZM2sOhMYd41vrZPl0TCB2EDQUNuEe5W5St1DRqqGyoJGd7DtHzokUxl6kufF7/OUk+SRbfe+yLYwmbFMZ34wRf6Z73uMpGRe6uXPY/s5/+C/uzcN3CrtjFdRfZ95Qw/96y55a/1LOvU8p3x/07LJLDHAMcGfXsKPySr62rGsAqOpTqoCQ8i85ilskxm7sT/hVhI1Dq2tFbO1AbMBC9jVNXcl/XquP0p6fZ+8ReAOJukI7PLeUtFOwXCvPh08Il42Dxp8W8+NqLALRd3yZae8sVcp5ueRqTwWRx/a6QK2hybuLYyGN4DpB2msnk3QN6p4dnJ2xV4vMwyix3KveGud/NZe53c3uN33bgNhoLGy2ee/mZcoLStwGgKysGEllhXtEV12pBZtQTlrqeao9QmNdd1Dab6fKzjs6MxkMtXksGnVYRVnQ9u6bsYmDaQJo8rk0RIf7meELGh4jyC50oO13GlmVbSF6ezIRXJogtUBAKFrsf282FNRckaxCj3sj6Reu7fl68aTH95lqWF74a/uy+9/9F6Fp12HnZ9bqH7CQueMV7WYzLjXqcqrKpOOcq6hj+o84+hY0VF0fdhUkhrFv+ys6+9DXpnP/+vIjsvPHmjbhFuHH7kdtJ3pNMuXU55xIFUuO+p/dx/M3jRM6MJOmDJGZtncWZpDPEewn33+opArkmYUkC9ZeH4dI6nAb7410qMJ5xnjj6CfuPgPN6+tePhJsFz2qZWdjvvOYkSAQ/zdPsnLKTk8NOcib5DN8pxJZN/w4UVgoOPHcAlxAXyffVfLyZub/OpXJxpUWJ3NqAgai0bQzc+Rpt0Yvgpu7nymQyk+vzL0rdVrFg83O0nFgIS7rHacHG0AQyE5HZkXRYWWCXXQXfDP8GQDR+/Y3/Gfynx76/O/v+xn8MRqORrKwsjEYjZrmCivCRMMpSd4UMg5UdBrUdrZekpt6tFwVmm2vFRfbukCbbciJzuBhzkedfeh7NT7/+wVmJZ7bSk6X8umA1oec24pd9kNqDUkPm3mDQGjj7xVnKT5dbTlY3NOBWegGVpsUicy7g0h68ClPwz9qHUSO+Lisna/IGLaYmMAmTXIGk/mM0otQbCcsL48H3H8RwWiyDMfPzmV3/bvSKwnwNCZ6iI0VsW76tV+bj4f53c378I3yxxk6ymbNysGLzdY3kBRwgMH077T183joXFhEpPxJybqPkvQsPFeKX5UpxYDGnB51GeYVHl1Fn5MTbJ3Avu8jeIddzyfBen6+pL8j4KYNLGy7RViWVbZrrNpfAokAWHF7A0dePSuJN3lHU+Sfgum0NqV+JmVq2HrbE/V4M0zRoulhQV8IrQVhARM2JImR83+Wo/ghmoxG75krs/sCvrN2CLLxWqaUksISQghBSpqdIJFuDRgcRf0s8Y1eOZe73cyXHV6RWcObzM0x4dQJT35cWO66GTjme3rpLc3fm8ordK6SvkTJWrRytGP/KeLzivSyyh3965gKh5zZS2m8C+YnSxNawR4dRt7yOHVN30DFNfP/qO/R8kfgFH0Z8eNXzv3LsuxYsPb5U9Oz2RENBg0VWJ9DVpQRIfIR0LTpSPkgh9ctUlNZKicSSS6gLizYu4thrx3jV4dVe5W/+J+AU5MTux3Zz4l1p17dcpaTd0Ru9tQMmuaLn8E2pfRqNzo1orbU0OzaDons8UtmpKYqdTpuTLxfHSAtyfcGmWzbx21LL0o01QUmYZXIUpVKJZm1dK4EZ29GrbbGKkCb/3AvsCM8NZ/uM7dT1F38XMrmME7F30W7Xjn+pP94VV0iTyWRo1RpMcmEzpFfrMaiEe6zwYCFNRVeXEasuqOY5p+e4a+FdXTKb3gneDLpnEHYedijC7ciKysK5yZmaC1KJW5dLn3L/h/cz+GQIVReqaCxs5N2Adyk6UoSmRYP12eMYFUY+eOADlInO4s8k2Z43H3uTDpsOFq5fiCxLPOh4Fp4m8aSWISlD2DllJ7JrkP8tPlbMK3avsOvRXRbj7VHtGJQGNN9r0LXqcA1z5QXDC0x4RfCevfvc3dTl1LF+8XrJsVfKkbbrpQPl1uk/884j72DTYUNgUSCa5u7ElEKtoGDAXDbPEhJu4/ZbZof+O9A2ay2OCQqDiuLAYr65/RvcEy10A7u50e4sJL4lDPUgV15/6nVODD+BR8kFqk+Ku2a0ZbU4lR+lyO0t9k0txWd6t9zyH419JqOJnQ/upPRkqURuGEDl7oRbaRr9j3xh8fikje08+89nUeql81ZubgaDjxiJyInALDNgNgvEh3Evj8Mz1hN1sVBwTjqbxNxNc9HWir1qy6InUhY9kaL4WZyf/Pg1kaSsW6qxaa2h3cnHIlEiel50r7LUpkAz7zzyDv0z+xN7/CQyGaSvTWfV6FWYDCaUIf4cG36MKbumcNNP0orW0VFHuTDgAnN/ncuSz8R+yq0pRTz+1uNEXY7Cu8KbyeffE8WLjxbzbsC7vXYEnFt1jk23brJ4j/kMDSJ94iMWC30AyLoH7HpdJY7+jriGu3bJkO2af5wd03YQlllEW554vNHnt/DUq08xafcknvnnw7Ru2CaKmz2CyIkQOtSVBiXXYF2KS4gL91++n0UbFllMnIZOCGXMi2MkEp2dkOu1KPRaFAYt2nbxw9NW045beQYt7iGcmf68xLsUYFDOb7jWuzJ9x3TM2fWYAY29kCiIyDLjW+ZLTkQOeyfsBZu+34MKlYJ5P8zjxq2WPa4+6f8Jx984brFrqiazhh8m/UDmhkyLx8r2VTNzy0zq3OvQ9ag9yhVy1i86wJHRR1j25TJuWt3twWU2mzHay2lyEu4fO00kHjKhqH74n4evyT9tVqTU893B14EBtw3AJ8lHIvMNsP2JA/y8/AhTdzxI9HxxcQkz6BvySUhL4OH3H8Y6RZwgN5nFN1XPjt28YKHQV+ZbxuWwzdR2VF3Tuu/GbTdy4zbL39XxJZd455F32DdxH/bB7siVciJnROKT6INBa2DFiyt48cUX4Q0hsXz87eOsnrwafbse4+8bxjm/zsHmHaEb4JdFv3RLCJuFcuw/qxW8U2SPocOA2WQm7Zs0NDWtbJt1ntJAA232bTS4Ngj+kn3ExXUX+Xro11zaaNlmYdnJZSTdk2Sx4BzsHAzAqCdG8dagtyRxhUpBbZawLz32uvi+0TZrSfk4hf3PCBIQPcmFLmEuBI0OImR8iNAR34vk95+B7yBfpn4wlYHLLI/vlWmVWDla9Wo7cDXIXZyoDh6C1saJ8sgxGI2/76NlMur84qkNTERr60L46Z+wbqnu2mO31bR1Ffp6K+5nrs9k0y2b2HTLJmou1kjinT7gmdGZFgu3coUcpZXSorKAeZeZF198keUfLWfvk3sl8cg8H6KyohhxYiFzty4XxVyNrvyQ8SsvvvgiD733EBHnLHf59wZ9u5722vZeFU5Sv0hl082bLO4PPfp7UB0oFJvlRmm87fsNRB/9CseaPDRFYuJNcVU+iafsiciOYNqOaQw8KJYFdqiRE5ETQbttO68+9Sp1C/veMdtW3cbP836m6HCRRQJK0OggQieGcvrj09RkSr9Lg9ZA6peplByTeu/KlXKu+/k6pn04jcEPDJYQ3vqKP7vv/Svx39LZN/mdycz+anavPp33ZdzHE/VPWMz3KHQduJWeJyx1Pee/l0pmd8JyZ59ARtL+vp74K4t97tHuDLp3kIjoPPbFsV1y3PGH45mzeQ7/ePsflJ0u61I8y96ajdJOyfe3fE91YDUDCoXO2JFPj8R3kC8e0R7oq+qIzvDj/g/vp/x9gdBy74V7uwqL22fsYfXNqwlMdWT6mldwNwgEgn5z+6H2siF1YCrVntdWDOsrzEYzRr3RIhHf2GbEpcGFnPIcatqkz12TuyelkYk0eoWh7OG/XN9Sj33NCXzLfYlNU9BxTiz723b0HAm73yKwKJDXnn4N+ZNS4vvVsPjXxVclPfyNvw7/6bHv72Lf3/iPQqMREmWB6dtxqsqGYcMkrzEZTVg3V+NWdoGGk5clcblaiVEhFH0iT0kZpf61t2GWmSkKrkfmLWZdaNv0BGRsJzY9loRzCYTnirubFFbdE3FZ5BgC7up7UULfrmf78u3sfHinRV8PSksJOf8bnoWnLPoDXCkr15AlXjR2TsBN3lFkjrkX14HBorg6KY5zk26jKKgIrZUWc4/1hFukGyofN+q9ozEp1ZiuoVusOr2aM5+c4cxnZ3p9jcKgxVBRLfFE6zB1kBr7HmZjJp5Fp9E2iRl5TScvEZK6gXrf/lQHSxOfB54/wITNQ7kYe5HsxEFYXZEMVNmqiF4QTYuDnvs+uY/oXX+dDAvAog2LGP3CaKydpQnI6t+quWPVHcTtj+Pst5YLwnq1HYVxM4icLWbE2Xnac9JjFiy9A5BuUutz69n9D6FrsKP+r+tmAVCEBGFGhnWL5c8q+sgXDNq6Ev15qbdCh64DhUFBcWAxIY9ESMyJnYOdmfjaRA6uOMjOh6U+Q18kfUHaN2k4BjjiFnltjBffJF9mfDqD0AmhFuMtZS0orZU4hzhLYipbFfo2PVvv2moxgVms80ZnZY9BZYPeWirzknxvMl6LvDg19BSy68RJW4VKeND64mHSOfZdC74a+hUHnpd6H4FQ4GyrbmPw/ZZlJZPuSWLqB1NZYV4hYZipHdQk3SNsYE++c1Jy7JnPzrDtnu5k6v+mZ0N7TTuXNlyi4oy0IG3j7UT2sNvIGPcAmWPulRQxU8Zns+qOVdi02+DS4NC1ef9u3HccfXEvNcHJXBp1J77ZB4k4teaazuvSxktcWH3BYhIEoM5/AHbNlVjnSgvOuhYNLlWXqQ1MxHaQVJ60/0Ef5v46FwBTD4MSk9FMh2wvNu02fPjgh/x4y69dsVbXQA7M8iUrOgsA78pwplc/BsCIJ0cw+AHL90bXe8tNlDuX47velxMfCsXV4mPFbLtvGzWXalCFOLBu0Tq+vieFmCVSGS6r9g7c69wZf2A8lWmV5O3Jo7m0mbRVaZidzLz/4Pvsnryberd6TNbiXaZJCW32bZQElLBu4Tr0keLEalFcd8v+5ajLoOo7QaWxsBF9u97ivQ2QU5fDkVFH2PDsBuw87TBoDOTtyRMRWoY8NITEO6VdIdoSLQNTBzJr8yyL5AI72QCanZqJvxDPHavuoO6K9zSbwbqtltlbJlIYVMiFQX/t+H7r/lsZ+9JYi4lL5+NKHn7/YVR6FSYradyk02PdWkvU8W8p2yomqJiuYNqHpO2kaL14LdCWUUBE6j6alV+TGfgw6Y3HRfGrjX1yhZwFPy0g8c7EXpNxWlsX2h0tM54bfdSoDCpu/iZSSlLxVrL6ptWUBJTgXaZC06QlbHIYo58bjZWjFXVjZLz52JvITXISziega776GH0tiRDX8ouEpa5HpbHspwOw5e4tvB/6vuT3JrkZpUGJV7UXnmVC0SvlwxSKjxTTXNbMpaAM9kzew55Jeygc6Cw9z99ZEJejLpM+UNz5I/OyJSU5hXrXemZtvYmoveLkUe6uXJpLmyk5IU0CAmy+YzMXfrhgUQLRZAK5QUdAxg5cyqTrooCCFpasWkJySjINBYJs2qGXDnWtgUZp3qA4sJiPln+E84geygNqOWV+ZTS4NHApOpN0d7GHtC55JOtv+AlXOy2rHvqQqhnXplflFulG9Pxoi9K9IeNDUKgVvB/8PjWXpON/wp43cawtIGP8g7hGi+9Tl1hfTs94gkMjd1Li8R26HlKwqd8/yYhtB6j0rmTVklWYBrqjtFFzefjtpE16FIDE1ETyw/I5OuooCguJ3d5Qn1ePrlWH1wDLz07M9TFEzrKcMNK16ihLKSPzl0wu/iJdD+rSK0hMTcSu1Y59NW9K4hUuGwBocmpCY9+9IZHJZDj+6xYOjc/DuXUod3z/CslfC+vB89+dZ+8Te3uVLgdBBn9pw1Kef+153L6yvI70ivPirjN3WVSYOL2nGV1VA//82EWyrpEr5bz2+D/5/tbvOTb8GAT1SJmUaxh5ZCQR2RGMPDISWaF4/D4ydBUvvvgih8YcYsa2Gehq2vu87is4UMDaWWspS5H6/ABoHU00OwljickkEM3GvTyO5OXJmGVmLkULxTTVLhUGraGrC7mjvgPtRQ2Td00moCQceaOQuG8sbOxav/qWeBGVF8YjlWHEvfwYLXtrkMll+A32w9CgRe+yACfjjQzJXMuEs/uQX0MqqfMztkRqBPAb7MfMT2da9LdzLXPlvoL7qHOrQ99PSuxtKm7iyCvCOLDk4BJRrL22nR337wCELurk+8T7y4BhAUz/ZDoF+wvI231tcmh9wZAHhjDbgs0IwIUfLrD3yb1/Si3DbAaj0gqrjias2+pEc1JQxjbCT6/FvqEE25YqFAZ9d+7gd+LXkIeH8FDBQxbfO2h0EEl3JzH+lfEW92jvzX+PV596lXWL10mIOUa9kS13bWH7A9st5jfkXnIyozPxqPUgb4f08x6zfwRTdy9g4IVJRF4Qj5HNpc3krU8DwKXRhRFHLHd69oaf5/3Mmx5vdhWGe8IjzguHOePIyJQWYpwCnGhxF/adelcPjHojK2UreUn5EgAmuRKD2o7U6c/hdaf4+25uaGD0kdGE5oeSMTgK18l3ieIXpnfw8gsvo7XWorXWYlT1fYFh1BupyayhPqceowVpZwcfBya9OYlpH03DKUjaCZ67I5dxL49j8tuTJTG5Qk7MohgG3z+YaR9M67XbrC/4M/vevxL/LcW+oNFBOPg50FrVajGutFZaXHsA2LTWEJS+jRaXABKWJIhiZrPgLe1RdAaDVnofGLUGPApP41Bb0PX6q+FaPq/AEYG4hLmQ8VP3Wu/UB6c49pqwnvvlwU3snLKTZsdmFCoFrhECkSpiRgQKtYL8sHyiL0Sz8UaB6D/hlQncefpOFGoFDRkXmbwzBvc6d/RV0rG/xDuTSp9KvC9FMWS7jtoMIWe6eNNiBu+ZzuY5m8kPyyfpTBLzNs67Jp/ZP8KZz84QPDbYYueewwQHfrzxR0YdGUX6eun+MD3oHn6Z1J9ds8qwjRITf3VtWhavW0xiaiKbbmjBYaE4/yy3s6HDwQOdWsijVmjE8uh/hH5z+jHg1gHXdMzf+PP4T459f8t4/o3/CgiL0Wp44w14XdxSrG/VEXv4U2oCEhj1uXSxbBvhR4eDJ/aNZbQ7SBchk7dHodAsZet11fT3FRcTDRojXoWnGaheTGi2gaIwcXbIZ6APy2tWcPvtws8qy57iFqG2VzPy6ZEcffUoWb9mkXSXeEEqDw2mLHQ43vnHubTRRZKkb/KKwCyTITObkduJJ31No4aAjP20uAXT6NNfws6RycBa70Ojhxuf3/M5zwTOFcV1bToCXr6Lihd3MHDX6+jbngb6ljRIWJLA9uXbe01y2zaW4VaWjlvBKfIXLhKxZQ16M1YaK46OPIrM5UMe9REXU/RlNbiVZ5A+eilaRynLccyKMWz8Mk24RrNCtBCRyWUsWr+IA37n8S63piLsLxQjB6LnR0uZv79j8KzBbNnsSIt7CM4eUikW38v7cawpIGvEHXgMFG/oysvhUo4SCOCfP8zHOVBcNLsyyeYc4kx9Xj2uYVK2+Z+ByQTFsdPQ2lpm6pllv8vyWUi6KooVPP/P5ynoH8dPYdO51VdaBO0slp/55AyTXp8kYlaPeHIERYeKOPjCQRoLGhn93Og+n7eDnwPB44KxcrKsd194oBCDxiCRo+xEyPgQ5Eq5RJYGwOQXgFrbSnD6VhzqCgFpd59CLlxXp5xE1+/VCpadWtZTpeovgdlsxnuAd7e0YM9zUiuwdbcVERSuxIBbel/UKa2UzPx0Jr6DfNmybAtnvzxL0p3d45WVoxXOIc7cefrOq8qT/k/g84GfEzgykPlr5l/zsZ3+dYNTBjPu4DhaR9VBsheFBwspPFhIf4ccGnz641x1GY29xx+8mxjZWwWt+1v23GIxblRacXnILdj7SGUObfzcSJ36NGa5gtxc6bEXx5TT0HyRm368Bzdf8W5L26zl4fcf5mziWbbM3sLAmrd7XHT3PTll1xg05d/B+08x8bWJf3hNdt52rLpjFc+/9Dzbf9zOuEfGsXnpZuou1+GT5IN/SDj+9cuw10UjU6skx/+y6DeUegPOjc48MesJbFxsuu6jZm0zDa4NKPVKHJodMPT0/Wg14FHtQZNTE43e8QQEJIjCGgcPcpMWcslnKU1OTRhNfd+sxd0Qh1OgU68SyNN+mMa4C+N459F3kCvl1OfWs3ryaqJmR3H9b9dz+tPTuPdzJ2KaVFatJaOFOZsF2dPsn7MZcYfYRN2zsT/Gjsvkh+azn1TuvGLO03UYcCu9QLudjFNDTtHiZVni8c8iZFyIRS81AJOPknMJKWisNXz4ZhvvfSYulGtXrSU2twC92hZdvZispG9qJzozmmrPag5NUfH0DWK5NPuBEZwfPpx613eZtXkWVbXpMGt4n85ZJpdh0pv4fvz3FuWi6vecwavgFNlDLPuibpj8FomOYbjX+ks8+2Q2SnIjchl+bDiT9/jTdFMN5QY5ZSllxCyMQWalos2+jYNjD3J49GHWed0tOt7v0l7McgVNnpGoOxox6qPpK2ey3icG77xjxB76FIPmWUmX0YYbN5CxNgO/wdL1g65Dh8KooMG5gbLIkchkMH/1fNqq23DwceCyTpCfSxuYhk3lw5Ljr/9xLhXepRi8P0IZEiWKKUNc2D5D8H5u8VuMwl98XkMeGELkzEhJt3cn6obPxFRSilu8dL3WVNyMe3EmXoUpyI16QCyBH2Kvw70omOCiYPSycionVtJa2UrsDcLrgi44E20dzYUBGchtxOsLma81P9wqkPvOJJ9hjPoRUTxLtZZYKzgXCJ83we5rkFytz63n4IqDxN4Ya1EmHISkb8CIAIvdYg2BA2izFeaTnkRes1lGlv+LVNl/gmu9K+1N8wDnrrhebaDDph2NjYai4CJMTlZd61yDUs26hetocOnurLckmdwbDr90mPPfn+eJuicsEgAmvzmZt7zeouxUWZffeCf8BvvxVNNTrJ21lk23bJJ46BbdYuLL8W/wyLuPUBpQBi93x0xGExP3TKTSu5Ktc04xvVJMaHNVBDHuYtbvP23o6tC//ejtvO39NhlrMyzK28Lve6H3hSSZrE268Mrfm8+mWzcx9f2pFn1/K4fOocxHj7q9EW2LjUUPpWqvavZM3sNzk18X/V5epmHivomkDkwl8VwiTd5Sj7TQvFBuXCt05xma+u4z21LeQvbWbKwcrSyufawbVcRnxjPg/ACqg4txM6j5POFzxr08jpHPjmTd4nUo9Ur+Megf2Lja4BzijJWjFSajCV2ejuEnhrN9Vj6eZqFDTiaTdXmYjds3BocaJWse+hbirYkZkIxMJsMjxgNZuY7MgPvRqaqZtXkWSalJ6FuGo3DuW4dp+NRwxr8yvlfZ/r1P78U5yJlB90jnwuKjxXh+58m2JduIv1N6P6R8lEJzSTPu0e6S8crex54FPy3g7Gdne/W1VlopCZ0UyuGXDuM32K/XZ/9aUXGugrRVaSTcnmCxiJmwJIH8Pflc/OWixXv0aug4lkrc/m0YlFYotW3duQCTCYd6gUDhUSyQdWwbyzCZhPnFwc+BoY8MJXqB5T0tCKoOMz/rXUnEZGVCi+V72mQwdfmNuYS6MOwRce5l2sJptMe0E+YUxvVx10uPH7WE6krocPCgPFooUnbWQp0CnUh6cxG3F45GbpIT0ja313O0hJLjwp7a0v7RbDaz9/Hd1Pv0Z8VLCrZskR7f5BVBefgoiErE+HtBpbOYrp8+h+zf+TI98zIKVzUf3v8hGmsN4TU3QJDYRmK8/i2qmuopd/kBryov5HZ9H9sd/RwZ9/I4dty/g+j50RafL+8Eb4sFCRAKFk1FTde0D/+/iP+WYl/VhSq+Gf4Nk9+ezLB/SJsccrbnsOfxPYxdOZb+14mJTho7VwoGzKHN2Q/vAeJ9q9kM8fveQ6Vro/msHdwifr5NHVqCMoT13pmZK/7Szj6AzHWZtFS0MO4lQdI69obYrvm8VNFG7rAsTg47yZcJXyJXyvEd5Ev4lHDkdnJyHsihLLYMZ50zAB0NHbTXtuMU4MT50JMcvuFHyvzK+PJWwe99pWwlIMhQKvXCOmxxQhFmvQmnUCFHpu/Qo2vpHqMCiwMZcGEAJqMJBddmX9QbzEZzr3tLMwJRz6XBhY4mYUNSeqoUQ4eB4LHBJJzUsiD3IT6572VA7Ckvt1ayfsF6GlwasFJGgL0492Od0I/skn7o7b4nqNAKpeLaCpjZW7PRd+ivec75G//38Hex72/8VyB76C0k7JayQUHo3CuLHEO7ow+9ZdEbvaLocPCgOE66MDUq5BistJhlRsnEprK3Im3io1S57CcnahtWcqncUOdkp9C1o2+RA32TD1OoFCTdlYS1s7XFBI7M0YGqkCHorB2QB1vefBTET8WuqQ7bIPGErmvT41V4Guu2etSaFtpKI4DuApC5ugb34hLsAqxpspMWJX6Y9AOlJ0pxB1pcg0Ded2amylbF7Udut9jhBhCS9hs2rTU0DBiDe7R4w9Va3cHTrz3N6UGnMfr6IOvxdwsm1vNe9Eu4th5haI5U2iN0Qijlx1qJOx/HoHNmWqvacPYTkpQmo4mmoibMyLgQd4HcuCjJ8f8OarNqUdoocQ5ylsR2K3fz8+xn8W6cR3Ll55K4UteOWiMwKXU6sLkiz9JcWE/UsV9xaCgh2y6WhT+Ji0tXyjic/ug0QaOC/rpiX30DZrkCjZ1lRnTWSME4fYwF9RlrZ2t8psSRWxuNwiy9F9LXpncxtEwGE/oOvSgxNvG1iVzadIl189dx5rMz17TJaKtq4+N+HzPk4SFMfVfabTvkoSH0m9fPojxuS3kLm5dtZuDSgRZluK68Jd3KM+hZ7Fu3YB151XnM7JiJbKcM5oiPt/Ss/xWQyWS0lLdQeKDQYjxkfAgTX58o6aa9Ep2L4+VZy3GPkiZv/Yf445MolbsKnRjKzod3cvbLs4x/efyfv4g/gYHLBnL+u/PseHAH0z6YJop11LbiezkFs0yBvYc12pYEUbJuQvrN5LUK3adHRh5houM/RMfbtlRj21KNXm1H1og7rum8xv1rAsE3j0Rla5kkEXJuE0alipaoOZJYp2xyyLlNFJeEwEoxEaQquJl8XRbXbbgOrVxMQFGoFRwbfowyvzJC8kMIq+xOeqnbGwitaqTMzoMazxrOD8gmor8g65f6dSr1ufVMfLX3ol9n4vitx95ifryQYBxw6wBOvX+KkPEhlG+oYtHqOMpDo2ksbyYgQFrI1FprqfKukrBSjXojTo1OhOWFMXvLbLSKetGj5Z3hyPJVy1l902r8NSsJtREn2mRGA02ekQSWj2bW1qkoP+07mUMmlxE0SurP2Ymw+DCOlB4hUSl0k2l/3xzq24WC5I4HdhA1K8pisc9vuB//vOmfGBVGdj8p9W0ddMiJ0KyHeOn5l/DULsTOs7vYp9cY8SgR/CuSziaxb+a1dfaZjCaMOiMKlcKiVPjux3bj4OcgSbQBGKKs2KrayvP/fJ5avx3AdaK4LCKMar0bxXEzGL1YfKymspHF6xazd8JeDo56llmuCUymW/pP7elMnZ8nWistSalJmJwt6EBfBS6hLgy4dQCOFu4vlasjze4hGFWWWc8aRRXHRwis3idNP4tinfd3fmg+R8aWM8/XmdWTP6GjvgP3KHfkLSY8qj1ocGlAr9bTbhSzrl0qLmFSqlC3N+JedgGj7hn6WuzTOHpSGj0J+7oii8tYmUxG0JggSUcKQENWPcs/Wc7eCXsxegjPpUuoS5eclvs5JTeduYltM7ZxwnsJIPY5c691pd2mhTpvmSh5Y9AaMDVoUelU6NV6UhLX4WGOAbonfDtPO0nH9JUocE0C1yTKW6Cn81LVhUoCM3dRGjWeGgtKDXWT1XwR/zzRl6KZnfgUulad0JH/+xrL/vR5xhnGkReWh6FDx5WENB+bECLKnyPf612MijaJhKLs4mHimidjvHMvAScHYuvY96xVe2076T+mk/5jOktPLsV/iLiQeeazM6T/mM51P12Hg6+0+7922Cz05dW4F6fSVBaJo2N3kkbXokWv+4nBKYMZfWQ0tWMrIMm5K541rZ5T478VpBTNMkxGI9pWHT7ZxzEqFJwf1kKFbwWDTw0m/kI8ushmQNgfmAwm1l+/nsCRgRa9gGKuj+H89+d5L+g9nm552uK1B40Jwi2qd6WFkU+PJOGOBMnvL+j3orHRcGbQGdw8xfsok8HEyGMjuRB3gYqgRmwN3Z+nyWii7Uga6iYD52LXku3jyrKI5wGw97Lnvsz7rqokYOVkxZQfpqD31hOcHCyJK22UqGxVnP3sLE4BThZ9QB3qiohMWUPq97MYtry7e9tkNBGTEUOTUxNq1SJi3cULYf/Ekbxx1xTabds5nXyaOeEvieKx6WHM2DoGgG9v+5bnAv6YcNOJ+JviOf7GccpOW+7sG7IqHMfyGDRWGnRNHdh7u4JMKFTLZXLuT76fj05/hNpRjUwmY+yKsYxdMRaAAh816x5+FwfdBDyFehAPFz3cNYekjyymUZvFr5E16H3WYTPwSUxGE4PvH0za3hoefvF2dk/eRX5oPv6Ny3uVorMEe297Rj1tya5DQGcXiGecp4SsFzwnmHPV56jUV0r2tACRMyNRWCkImxSGrk2Hjbp7jlDZqIhdHItHfw823byJppImhj7U/YwUHCjg+/HfA4L029XGvGtFXbbgMZ76ZSpPtz4tkbW097ZHJpdZtIj4I8gd7TGobVFrW3Erz8BkWvC7N5wJo1KNwnCFbLhR1zX+p61Kw8HPgVUjVwF/zrfJtdaVhLMJ2LXZoXlII8oJKK2ULD25lMu/XbbojZ7gnUCCd0Kv7631DabRqKXdKh8ZMoxGZ5S/b0vqZHXMauvfOewxsezlXt/HEm7ZcwuNRY0SNZpO2F83hZwiy2Ng4cFC+h/eitygpd3LAZXtIB4pfaRrz6TTAWYTjjX5dBQ5AFcQ0BUy6twF31JzrUFSSJHX1BBaGURjf1vu/uJuGsbq4bG+X5dnjCeJdyVaJIRm/JTBr0t+ZeG6hUTNluZGZn81m/Iz5TQWNUpyHPp2Pe8FvYdToBNOgU4Mf2J4l6/u/zX8txT7Nt28CYCAEZY/xx9nCJ6qlpQDjNb21PUgQ14JlU4YRywMkchsrOiwc6PdSSAd/JGq4LV8Xnm781DZqZj77VygWxq2/0KhWDns4jecipqDWSnkHupz61m/aD2T3pxE/C3xrPZdTcqgFEJeCOFjPubL5C9pyGtgWcoyTB4ysqOyURlcGB0kzRUt+XYJDi0OPPrpuxDzEqdtbiUIR16xfQWAefHzSEtIY/PszWyZtYUXrP86z745386h+GgxLeUtkvWgtkSLc6MzX975JbcsEojBu/+xm4b8Bh6teJQO6wYcWiKYuHcUWvtyuGIfhUpGRpzQJRmbK8PQ3A5056y6vxs51/90PcrQKniu7+e9dtZaAGLMfxf7/l/H3zKef+M/BrlcTmhoKHK5HIPalhbXQMx20gW2wkrJ9ilfcTLmHhrS8yVxXWU9/pf341GShkIvTZZlDAnn+MQApu2YgTFVLFEkk8swWNvj1jEbd/PnuMjuF8Ub8hv4xHMlg7auZODuNyl8R+ojdzU4Bzsz4okRFtlUMhnobRypDh3K97ulce/0l1DXfsm5YdYYTeLskK2nHWkTH6XRK4rAiztpzRXLfOqzCwi+sAWPijaGHR8GJeJNROTM7kRqUdx0FFZ9r/t31Hdg42rTq257RcQo8hPmUd1/LB7R4iKl3EpB6sBU6tzqsG6pR68RrzRqTUWYFCbG7/S16NkHUGpOwbvSm6ACM9qW7o2MpkHDB2Ef4FOuYtO8TTSq++6v2Bd8HP0x7we/T0OB1K+sXd+OTlWDdWsH6jZpvDhuJpdG3knswY848abY08+kM+DQIDANM3+WnrOjnyP/KP8Hd6Xexa37biVoTO9J62uFsbic4AtbsGuSSnBdCUsLPqcAJxJemo/MbKTfsa+pShffg1fKujzZ8KSkOKzv0INZSC7N/sqyvE1v6Ox27E3mzcbNhtwdueTvlY4XyIRCpcpG2pkEoMzrlgrOS7xOEm8ua6ajsgOPGg+UReLnRteq4+vhX3P0Nalv45W4cuy7FsTdFNdrdynAwRUH2X7fdouxK73lGgsaRbHWqlZWylbyadynLD25VNIFqLJTETA8gCP/PMLL6pdpqZDK8vxPYcK/JtBS3kLKhymSmLauDd+cI/hlH8Tp2E40jWKZhKDzDow7OJGC0AL2TdyHlZuwEJ/7/VyibxrYVeSuDrm6vKUl/PibPW8808jXz1iWfFJ3NKLuaLa4mTJo9Ng3lOBacRG7RsvJPIPSwKbbgrC+YZ7o90obFXsm7yEzJpM5v83B92y3l6FNSy3xpxvwKxMKzmkJR0gdJowpW5Zt6Uqk9Yb2mnbGHByDV5VXl8zmqGdG8VjVY7iEuKBp0WLfVEfkuaOsf+G45PiQ/CD6X+yPTbsNBq2B3J25rJStJPXrVForW3nkvUeIS4/j5JCTmH3Ez45f+FAOjjmIY7Mjg7d/S91+8VwddfI7knb8i9jzzjS7BePsIPWV6g1t1W0UHyvuVQY54skISv1Lmf3cbBqLGnENd2Xm5zMZ/4pQ2L5136005DewedlmybETkyeSH5lPQWgBdgHS9UtReANHRxzFLDMjQyEiHCms1VwY9yC7J+0mPC+cSVuuTc44e0s2r9i+IpLNuRIn3j7B7n/spj6vXhKTy5SYZWbOJp6lyV36WSpGjaD4d+lUCUnKy571C9YLcqrAoUop/d2MCa2VltefeB3dLd3klD8a+/Qder6dvJa8EpVF7zCHpEjq/eJwqbDsHTb41GAm7hGS6j3nrbasBp57+TlBRmtANdZu9oRNEXxznEOccTqiZ/knywkqCsKv1A9jj+7TzNF3kzX8DmqCk4W5QXFtzODqkCHkD1qE0sJ6a/6a+RYLfQBKVxVHRh7BsdkR7yJhrt5400be8n4Ls8mMVb2MoKIgFq1bxA0/SjseP3zwSzZct4GAy5cIPC4UQGuzavkg7AMa7trBs688i3+JP/neb3HZ5V3RsSkfpbBStpLCQ4VXvTZLawTXOF9ykm+gzj+efglSQpDBpEdnpeN8wnn0wSoCRwTyeM3jXd6FZ2d0cGrIKR57+zFq95wTv3eTO+P3jWH4mYeYvm06Hhd6jCe5dgQfH8LL1UpOfz8Ll5S+t9v7DfZjwdoFBI8Ntuh/1FbTRm1WbZdcmrZFy/kfzlORWtH1WTjW5BN8YQsNPZ69hsuC5LtjsyM7pu7Axl+cHOosWvqX+rPipRXI9xajb9Xhl32IwEv7uWmN8P1aaa1waHEQrYH07XoubbjErkcse5NGTItgwqsTiLvJsofi9vu34xnnaZHQ017bTtavWdh72xM9T7oGUdcq8Kr0Ys/kPTAyXBRTqBW8/+D77Jqyi9CsFqxyuqXITXoTjd9vwarmBwq83iPb7wU2aAUP3Yb8BtR2apwCelcTUNmoOBh8kNHHRrPioLRgETgikEXrF1Gwv4D8fdL1oKqyBNumCipDh1GvFJerTXoTC9cvZN6meYzfbkPxIbFMlr9LHBW+FTQ5N1HhW4HRXjymReZ1d3nXeNSAQnZN675ZX85i3vfzLMZqBhnZM3EP65fI8Rodhb2XPWp7NS3lLRh1Rny+DGRxylJ8SgMx6o3sfXovH0Z+KBBYbGQ0OTdh05iOqlZIKu97Zl+XJ21pWD0XYy9yoWo8WdXjMJmEte0XSV9w8ZtDVHlV0mHTwcXYixwadxGTuu/Em/Qf0/li0BdUpkk9fwGmfSgQuix5gB5rPcZvit8YnDIY3RqdJB40Ogg7Dzu+HfMtn/T/xOL7b7lzC1UXqig6KPZSrs8RnlNjqJH0+9LxS/7rSHv95vQjeGwwBo3BYgeIR4wHj5Q+0qsEf3tdexcBqSfUsZHkDLkZnbUjxTFTMZmE8cesUHJu6tOcmbmCMzNXkDr1aapChnTN5UdfPcqex/Zc9bzP/3CelbKVrJStpOiw1Ht68i+TGXdwHINPD5acn0wuw3+IPxNemWBxHi85UcL2+7eT/mM6FeekEv06WmmxyeBMcDIZXlNEaxC9sXteVhgUyPTXVsHxH+pP7OJYi7KpMpkMY0MzIWmbUOgskJRkYJIrUGtbsSrNQyaX4ejn2KUSo7iUgUfhGSJT1tCwRbxGNhlN2LfYY6WxYsT2s9SuekYUV6WcYPwOOSa5ib0T9tKcIJUr7A26Nh0tFS0MumeQRSKnvbc9Rq2Rn+b8RPHRYkncOdiZX2/7lYMvHLR4zR79PTCbzVzectmiLGtf8Gf3vX8l/luKff5D/Ym/JV5CJOrEjM9mMPOLmYx5fozFuNyoZ8DuN9l2Xw+vYjOcmfECZ2auwHqgdJ6Wq5RcHHc/BYkC0/Kv7Oyrz6sXk5HNAlmscw01clsKD3/wLAu2vEljUSMXfxbkwI++ehSllRLZcBlVXlUYf/fCba0QyHb23vbI5cKJDDlzPZX7BDuOFeYVXSSF7MhsMvtnsqkV3mkAnVE8Hg24MAC3OjeMSiMGVd+9MPuCwoOFbL9ve5cH4ZVoT21nwcYF+FT4oDcJz/OIp0Yw6a1JABwec5ZfFv7CkJQh1Bdni469ktCyYLU9LT+K91navBJ8Lu9Hqalh15RdaCc5X9N5d3rE9taV+Df+Ovynx76/i31/4z8GmUyGo6OjIB9Sm091UDI8/LDkdWYzlLmtYeShIHJeXiuJa8vruv7tWSBNCJe6fU+ezysEZZdgysoRxQw6E9attaT7zmRPvA/VdvtE8bqcOtHP9gniTezV0F7bzkrZSrYt32YxbjqfzsDt/8KpKhuDhbmnxiUV13pXXAo+pblQPInI5HIM1vY0+PQna9gSHOPEBSCr+CiOT/TCqDAyZfcUZPlihvqoZ0ahcHGg1dkfjYPnNU3oF1Zf4JOYT3plnJYHeZAef5FK232SmNJOxeY5m7FvtSfhwCqaS8X+Na5N7gQUB+BW74FKK9UyXztrLdd9Np69E/ey8oWVIk+2K7uR5m+cz4Cj0q6tfwedbDmzBTmokm9auPeTe7n++xCCjln2/jIjwyRX0q4RD7uu/Tw5N/kJ6vyEBMyvt/0qObYuu46ylDL8h/lj72VZxvHPQB4i3DfuJWkW4+5FZ+l/+HNMlVJj44qWCobskJHpuwRVR22XnEknIqZH8Fj1YyTdk0ThwULJxurneT+zbsE6+i/sT/jUvj9XIBQSPWM9RUXrK9Fe007at2lUZ0jP28HHgSEPDeHyb5dpyJcWZmXNTZjkSrKG306Dr5TxtOzkMtbcvIZVd6zi13/8KoqZjCZKT5Sy72npvS/6G1eMfX2FUW/EoDXgEma5yF6bVYvc1hq7ySNobJTG26uFjeusL2dJmLZXMo2/Hf2t5NhOGUcQEmCahv897fFvv4XelMrsgtzIGLucSyOWUjNzCXYe4mKLcdIsds3uvi9NJjMGjQG5Qk7skkFcHnormSOXobFzw7VMqqd/Nezc1EFo6nryPpb6UQJcHn47jnUFeB6VkhY6yhqIPLWa8ogxFA2QFroX/rSQR95fQZ3DQZo8xInbK8fq3ZN3cympu+DQ6uLPwWnu5ER0z3OlJkFKyc7TDqXN1UkdLVXtQvImZTCqKqEYrmnUUH2xGm2LFofRUbz16FscGHuAjHapZPb4gxNZ9MsinnzjSS6svtBVIEj9MhWVnYrjw45zcuhJ9L5fMHroItGxVsE+HBl9hkrvSmo86tDbiHehTR7CGKEw6jg5WofaTdpR0xtyduSwauQq3nB7w2I8ryGP4sBizo44i5WjlSA/eldSV8IveEwwCisFDXnS8QJALpNj024jKTYDEDSVvZP2Epcex6itWVSmdRMrzMiQYWbMobEA2Bql3U9Xg1OgEwNuHWDRmxSgI3kkJrmC+ippxTmqoB+3rH2BJr/raAgQJDY/G/AZn8Z/CnT7rbmUX6Q1T0zkwF5NRlwGNZ41PPXqUwz8TDxO1u1IYcymbfiV+dFh24H5CrWsPxr7mptlNFp7kVXhSG8WBy7lF/HOtUyoiMkcwMhjI7l1zQsSL2K5rYK8sDwanRsxywRm/YIfF7DCvAKXEBd8YoZxcMxBInIiuPOrO+koESecTUo1JqWaNhd/GnxjkCn6voXyzjnCgN1vYtUmTQh0IntbNsfeOIbJKL731d5W7Ju4j6CiICIvCEzv9B/Taatqo6mkifwx7bzy7Cu0OLRg325pfhA+B51xIw5VgpdXY1EjLWXdSTuHFgdm/zabxd+NFR1ZelLw1io8WGjxnAdtFchwpbulPm7WbvY0eUUiNxqw0Uu9Cp1rnIi8HIl9mx+hNkKB75OYT/g0TrgHDwS/wuWoy5wafAp1oJg4pq1twSfvGF4VcgafHoxzgXhsOzWjgvcefo+XWvR8f8v3VI67hgSPDL5x+QbD2wY8Y3v2K8KY58ewaP0iznx6hqaSJtqq2/j11l+5uE74DLxP/opVRyPZg2/COUJ83ra+zuwff45TQ05xaugp1F7iOcsz04ahJ4bSbttORkwGRk8bVI42ZIy+l7KIEVhrrZm+bwJO046w5cl3UXp0rweV1kriboxj8jtS36VOjHxqZK/SfJc2XCJnZ47FWPXFan6e9zM52y3HI39z4a4vBA+qSzqxKodMJqPBtYE2+zaGnhyKbW73+kiuklN3dyD7x/4CQHRmNN5nhbHhu/HfsWr0qqt6VZvNZp7e9zSx6bFoT1guiLhGuLL05FKJvxGAXdpx/LIPUNp/MrIAceJVrpKzbuE60uPS8awwU10mLlCZDUZ8q6ej1Cux0lgh04mL/+3hK9h4Qwvf3P4Njs2OGDsMfV73NZc201Hf0et6r3RkB8dGHgOzrCthe8/5e5j73Vz0Oj36X9uJ3h5AzW3ldNR30FjQSH1OPdoWLe42ClzqXRh9eDT9zgnfZ9Ghoi5/wOtM65mWno9dcxkXfq3j+I5dKK2UuPdzR+Gk5rsl33G5n0D0yAx4lA5D3zvSay7VUHG2gjOfW/Z8973ZlxuqbyBkvlR+2mQwodQrGXhuIMY9lltSGosaAYi+TpzorsupY6VsJWWnyvBN9uW6n8VEvsRliczcPBNFvoK97+ylTXftXXa9QWmt5OZdN/Nsx7MWO/A3XL+B111et3CkgDfd3+Q1x9csxsxm6HD04sLER6i+opgHEJCxnX5HvwKzGblRj9zUrW408ulu6e3bDt6GJZh+9xNVqBUWuzcLRxeyb/w+dr25S9LRYjaZqc+rp63G8ueYk5rD6Y9Ps/Gmjfxy/S+SuHrDs/Q79hkzts1g+ce3i66r/mI9T736FMOPDefW728l5KfvLP6N3pD1axa/LPrF4v4PQO7s+LsnsPQ5DR4TTOaYezk3+XEax8/HZDRx7I1jnP3yLACqs6fwyT1CYfws7EaJO4G1lW089vZjjDk0hkrvSuo9xeuLywkVbJ61GY21huw4Z+ZNt9x9bQntte1svHEj6T9a3scEjw1m0YZFuEe7S7rgQdhfjnhqhEVZV5WNiiWHlnB36t28YHiB/gv6S17TF/yZfe//q5i/Zn6vRA6AQXcPEllpXAnHugJi93+AStcuUb8wm8En5zBxe99FX9MoObbnV/9XevYl35vMQ4UPdZ2T0lpJ+NTwrjGgOngwBrUr4RmXaMhrwNpFIIDZutvSpm7jxYkvElQUhPtdQrH64aKHeaT0EZwCnLA/qufpV55m4k4v1j8sJZoeGX2UXVN38dze/hSsuJ+6M0Lu54n6J0g4OJV/PvtPUhNTcal3wb/EH+MftTReA9qq2xj26DA846RrxYnzJ7J51mZc611pvNwICF7Eh148BIDCbEN+aD7vPPIO6kHi9UdHXTuPvfkYE/dMJDNeiyJanO/S55fil3MElbaBtIFp6BIsq530hqXHl7LCvMKiAtbf+Gvxnx77/i72/Y3/GIxGI+np6RiNRoIubCXs3AZYKy3m6Zq1LFm1BLlJjvyGYEncvn8gze7CxsCjSLqBCMt2JjE1kQ8f+I6qmT0G0wYNsQc/ZmCKGw+/cxtxR8ULpYBhAczdtoyLo+8hderTuEzpeweIXCXHJcyFM5+cscy8t7fHYGVPxOm1WF1MlYR/m/sb+8fvZ8jJGFrLxItCg86IdatgLt3qFoTSUVzYUro5UeVnRW54Ll8t/Qr1QGk3mPvSOXQ4uBNybiO6NilLsjd0yhRuvXurxXih1wfUy5+m344Dks4q0+8rh4KQAgqio7DqYeptc7yFpd8sZc/UDLKH3ip5bwd/BzqcDZjlZsxyM1cuxtX2apZnLed8/HniMuLwrvhrpC478Wj5o6wwr8A1XPq+JUVmFEYFZ5POUh8uZU3b1xXhWJtP5ph78Zon9i6Sy8GotqFg4HwCRgVj1yP5c27VOb4b+x3b7tmGoeOvZSTJHOxpdfFHZ205ce6TexTb5kr0KdL7s7awltm/zabZsZldC8PwscDe1DRqOPvZWc5+fhaDVnzunczeslNl5O2x3B3VG8Imh3Fv+r2ET7FcJDz7+VmUNkoGP9ALU7a2nZpLNRg00s9TlzCY1OnPYtXegHMv3SNTwqYA8Nakt0S/t3ayZthjw3r9u524cuzrK4w6IyfeOkHuDgsmbwifZ9GeHI5ddORf/5LGb9hyAyvMK0hclojKVtzVaOtuywrzCmIWxVCdUU3pqVJRPGdHDifeOsGyU8t4tOLRq8p8/dU4+/yvvcbkKiUae3faXPzRegdJ5Efz3c5wJu5fxGTEsOzrJ2grrkfTpBESDJO+xKqjkXZnPzwLThGQabkbojcEXtyBUq+hLNqyNJdZJqfRM4oOZ6lHi9LZjtJ+E2n2CLV4rMrJh2YnHcqWQ5QWnhDFtM1ablxzI8kpyWTGZLJ/UPeXbVTbUOcl+I4BTN41maFfCePV/Zfv5x9lYhnTnuiwdeCTez+h/6X+uH8ubFx2PrSTT2M/JXdnLtvKvqXVoZVDYw/R5CHd2BwfdZk9E/dQFO6Oe5Q7Qx8eyuyvZzP769mondXsnrKby/0uY6X3QS0Xj/0aUyuBtcso9ytn1R1fkBYmnssrIseQG2Wk3qWeYt/DGM19Hwv9kv1QWisJHGVZLnvVq6vwrvRmy6Qt2HnY0V7bzocRH3LwxYNdr7kz5U5uOyBNil385SJPvfQUT77xJHtukjLlrVTCMltukmOmAY2uOyFt0JtQ6jqoc6/nt9m/kZU8QnL81VCfV0/JiRKLzEyzGS56TSB1+nMUdUj9KF1qXQnJBb+abkm1wNGBXZ3jrQd/wyf9LcJS11O7/4L4WKU3bs1C909+aD6yQGdRXOnqQKO7Czq1Dq9KL8w13euLPxr7zAolpdGTsG6tk4xDAG2ZRZgVSnKTb7BwNOyal0GrXSuhOXLaS8XJPIWvLWtvXIvSoOTmVcnUXhB3tAfGj6Mu5AYyYjPYPWk3Cpce65PmKqxbumWVeiZC0temk7nB8pxhUNui0rXjn7lb6Grved5f7WLtzLXsfXKvhFBk+N2fcue0YxQmCAmi+WvmE39zPCpbFcbfM6Frb1zLzzcfoCcGpc0gsCiQnxd/z7uPbwAE0ot3gjc24b58fN/HXOovFAGRif/2kIeGEH9LfK+EHK21kNhRO0u7Wju/4tiDH8P2HZJ4VFoYN669kcfevBPjzwW0VrWib9N3zS9yo4JG50Z2TN+BdbR47a4IsefklGmkDirirUffImeyuAtDa2PA062FDX5mpvuMZVZs36Xe9mXsY/269Tyy5pFeX1N6qpRjrx+jtaIVe297rv/tegbcKnTEOxVdwGjIoMErAKWDOAFj5e7A5TgZFb5CN0tP71H/Mw5M2jMJrZ0bGcMScU4YhEypQOPoSWVIIgXBBfi4tbDFF15zhyuZMAq1gvlr5luU7QXY9+w+vkj6oqsjsScqvqxgrWItv/xLmnx37+fOvNXzuPDDBd7yeksSL09s4cC4A8zdNJfkfeI1sNlkxqnRCZt2G9besJafFncn5+UKOdpoe6q8BULB0JNDid4rfP+D7x9MU1ETP88Xy/FeicbSRp54/Qmu23Ad3lukCim1l2s5+d5J1HZqHP2ksn1tA4aRP1DoctD3eCzlCjmZMZkcHHeQV559hYxYcUG77Ngx7vpkMJP2TOLp154m9rTYX9SsUNLsrCOwOJC7v7gbTWFTn9d9ebvzWDNtDeVnLKtuyMwq7Fvs8SytobW8kbrsOjbdsomCfQXolDrefOxN1i1cR/2g0ajt1F2WCu017SRmJfPQBw+RG55Lxjyh66KxqLFL4tTulz3E7duFu84EFwNpLqlDaa3E1t2WjrzGrnMYdHoQN/x4A5qmvpO/Btw6AEd/R4v7KIDrP7yeQa8N4kjxEUmsdk0tz/3rOfZM2gMWXD+Ov32ck++cBGDcynGimNpejd8QYe8aMCJAUrySyWUoPZVUelWSdDaJqrQeBJd/A5pGDQ35DZgMJovJPu9Eb6wcrKjLrrNw9NWhzyvG79IeHGvysGmuEnn2uZZnYt9YxqBtL5Gw5y38L+3pmrN8k32Rq+Qs+GkBwWOCLb73wDsGssK8gue0z1mUv20d18qR0Ud4bMZjEmlSXauOD8M/5C3Pt9j0vHT/eLb/Wd587E02z9pM7EOxkniVTx0VPhVcjrrMiWGnu+aStjZoN8gp8yujxaGF3PBcLsRK3/9q+Hnez2T+kklHg7RIbdQZ0Zw4R5uLP0Z178lzo9oW1Go0DRr2PrmXrXcJ+ZD2CTPJGDaWlEFHMIWLO+xktgpODT5FcWAxm+Zv4vR0cYddbvBFUpNSMSlMmGRa7BTOfb4mOw87ku9PJndHLlUXLN+70fOjWZ653KKk/Q+TfiB9TbpFic+/Cn9m3/tX44862f63UJdTx9fDvubcqnMW4/p2PQUHCiwXpBUK9NaO5CTfYFES2S/7IFaaZowt0s5UfV1TF0ELxJ/HX9H1uPHGjXw75tuunwv2FZC9WehYOzImgw2LTnB8SgK+yb5ducSJb0ykQSNcZ7ttOwZXYW9n627bNWfLXKEkoIRDow+T+TtJfcONG9hyt9DtNqtaGPfNMjNGhbHrWmxcbAj0CsegMmBSmJi47zqWfb0MQ9tfl0s7/sZx6rLrJNYVACFRIVhFWjFl9xSq9grPpVOgE26RwhonJNeD5NPJtDi0gI14DDXJzNS51dFm10ZxzATkSWLygPWQAZydMI8qL+F9r8XH/m/87+I/Pfb97dn3N/6j6Lzxq8KGE5S+DVm+VGbFZDThUePB+QHniR4vZfspbK3osHfHtrGcy8Nvl8Sj021xbhzL24PepoJsoFunX26lpDxiFJf6rcW6YyBVXqnAw11xK0crPBL86Ph9j3gtCwVrJ2uu++k61l23rsv7R3Te4SHkDL6R8NM/WZQjdGxPID0uHZntYkb0Ey+0m8sbiT34MRfjNWgDn8NotKUnC80kM6Cx0aC3TyTKSyzLc+CFAzRtL8GlsgKlXoNJJ/bCuhr8h/qTsCQB1wjLm7Vx263wKr8Fg1om9Q9rkTF/w3zywvIoiI1G7SSeHCv71XFJt5Nyzwv0s7DvmfnpTDav9MKpcQ2eNZHo2vRYWXUXLtyj3AkuDKbUr5RfbjzC132+qn8PlUnWrBv+EQDzcr6VxL0KTuFceYl6/3hJt4KuqQO30mxcy9KJf3MQg24Td0lcyZh8w+0N5nw7h4TbEvp8biUnSlDZqvAeIE2EmM2QNWJpr8fmJi/GufIyIcPiJbGOug4SzyXS4NJAq78Ok0msalZyqpz1twtdrbk7c2kqbsItortI9EDOA5QcK2HNtDVUplVy74V7+3xNmiYN6+avI2JmhMXEllOwE37JfihUUjZqZ/fNog2L8OgvTYIbDOBReLrLxBrE0lDp69OpO1RHsD4YzXENiBUvmfxm76z6K3Gtk77KRkXcjXG9FnxDJ4VyacQyNHauNGZZfg99u5722nZs3GwsypMNf2I4F9ddpPBAoUheJGFJAmWnymjIb/gf8yTsDU7VAvNcNW2CJGbUGVFp2jEq1ZgMKsxmmSiZkq3/CSuTIHfmWWWNoUOHtVO3JFi/44JfiV6tJi9xseT9r4Ymj3AMaluavS13lzpXXaYmeBBa/zBJTOloR2XYcOzri34vHvS4D6dOZX/FNpZ8voS6BBNcQfo26oyEFITQ4CJsjhSmK4geZjNKgzUykwyz3Iy1xhqrFmEDYdQb0TZrsXa27pVdZuOgoNqrmj0T92BrJ7B3OxNQjn6OmA7qiD8fT35oPmaZ9D6sCGghM/IMOq9nCBwpFNY6Jfnq2oX38S/xJ+L0L9SlDoch3UmHtGPfMWZHI9rJvpT7lWOy0M55YqwORfNOBp6IIj8+Fegb+cajvwfPdjzbazzyXCTuxe4cHnO465rrc+s59/U5xr44lpWylUTPj2bRhkWSY0+3nyYvNA+bDhvGzR4niQedreXm8ptZffNq0gam8Ujsq12x9joN0ce+ArzxK/OjNfjaZCFlchkqW5XFcU6vB+uWGkwKFVZWzpK4x9QkNhUkEHR+C7ZNFcBwtE1a1A7CuFCdsxWfKi8OTzRz52jx+N+eV8nSz4ZyYGwL6xav45VAsRyVQ3I050qaaHRu5JlXn0E/qhEe7I5fbewzmQQJXPfS8zTmh8GYHgSt/HJcyy9SFTzE4vEaay2HRx9m+o7ptJfWc6WXsQIVVjpvrHWBqEzumM2w8eaNpK9J59Z9t2I2h1DruI8K11JKA0pZ6iweI8NPr8WgtqPVNQDXsnS0zQ8C3W2LnR61lvyPaoOSsGssw6PkHEatUSIjveNfO3DBhZq3ayTdHzZlNtz7xb8oiHCkzUX4POJujCPuRqGoYl0tI6IugqKgIsxI15KTt8VS6zOCT+58sCten1dPZVolgY/Mo85B6PDbPGczjpr+fHbFsX7JfldloKdPFApi0xOlscLd2QzcsR6tnQtW0dKOAZvE/mQW2uFRdAa5rTVV56toKm5i1HNCAmvZ5zdjkhn4ZPknXQXPrr/bfoKdw6Z3/axXiK/bpkWBk9aVecH1zIj7np/NfV8NNl5qZOk3S9GpdDTc34BLiLi7qjKtEitHK+65cA9uEW40Fjbyy8JfGPXsKMa8MIa3nvgKjaqciEoTRuMTomNNJgiouw238mxGHx5NXWg5xHUnhTOn1pI3cjeBNQ8TVvUYrnIwGkwodRoMCiu+W/Idc+2gotCb2oxEfK7r+6ZE36anIrWCrXdvZc43Ui/Zt8++zWOpj3Gi6gQLn10oitl72RN/Uzx1l+soOV4iObYyro0T4Ud54vUn6GgVkwf1HQYeee8R0gak8eu8X1GaxIVh8xV7l51TdxJlFPYjwx8bTnttu0UPqk7IlIL0l8Zag2mR9LOoyazhwHMHsPeyxy3STVLk0fsEommoIjzlR+pjB8Ecy/M50FVU70SO/XnKEs5RElCCU2s/RvuI53GTLgvr5gKKA4vJTHDhjqAhGI1Xl8zvROf8+dOcn3he97wkPnbzcBLT5Ni32VOTXIAh0JeSYyUEjwsmiCDa7NvIjMkkWDMCtb2a6HnRuIa54uDngG2IJxcSm7gYc5Eoe8HzeswLY3AK+n1tZGOLTm1ku08Tbe+9RGii0Ik5Z9Uctm1IY9z2cWT1y8KjxoOwvDD0mr7LDbpFuPFISe9F9EkvTGK0ejRFk4ugh0WuOkTNhbgLVHtWI3ORrmH07XrU9mpu2XMLVk5WopiDjwNLTyzFqDNSkVpBbVYt7v26n7uWihZSnk3Bu0rYJzXlNvV1efGHyPo1i99u/41Rz45i7MqxksJY5IxIDr14iEubLjHyyZGS42/dfyv6Nsufsb64HJ+84/jkHUdn5YDJ9A/MZkEBQdWjO7HFNagrh5G2Ko3YxbG0VbWx75l9THhFur7+I5gxY9tmizZPizHMKHq2FGoFpohI5DnZbPnVyLyeXAs1tNm3kZqUSuiNUtLb7pknaFEK+SCFMRejUejsvGmBjhZrW/bf+kPXax31Ui/lq2H448OpTKu0KC9qNplpazIgs7a8RmnIb8CzIBudtQMKe2+UNvb4D/PvKoYa3b3YHC7cQ780ZrGcT7uOlTuq2DG9m/Qi7bATflZr1Vy3Np7ipiMwvnd/yyuhslXhFe/F6Y9O01LRgle8WHmj/Ew56T+mk7gs0eK+N2p2VK9Fe4PGwNHXjuLo74hzsDOesZ7Ye/85haH/ZKHvvwmn3j9F6clSYhZb9kv7MvlLajJrGPXsKMb/U5y/M8vk6KwdaHELlhx35S3VkZ6LyAMOJNLzf6WMZ3NpM84hzgSNFfZ1Rp2RqLlRXbmnSpdN1DscptotDSuHOUQviMbKyQq3SDeqG6uZtn0ahcGFmMYJ9+epD09x6r1TXL/5evRJClbbrQYgrvB6YDgZa4Umilmfz8InJYdJVpPYvXwPzLnEgYH3AIJ9SdHxChLibqM8ZD3DR54lTzFHQgz+d3Dzrpu7uhQtocOrg58W/8RbEwSSVGNhY1eDRVxGNP0zI8nql4WxXZwvdXB1YufcVmQEkpg/UaQAZzabwdqaNkd79Go9S79aip1fDdzR9/NeKRMKvs92PPuXfh5/wzL+k2Pf39/u3/ivQE1gIh6Fp7FtkcruWTnb8MEj36NVVTFDJqXzmfRGyiPHURo9GbNCektvmbURpUGJa50r1ipxXGWrpjxqPJf7XeZyv8sojD1k4HRG0j46QXhKMWaZglo7X1jct8UXgO8gXx4ufNhiTCYDjYMHGeMfwF0qsU78+Rg0SiV6Ny9aOnr4i9nJOTT6EI7NjgzZ+ha1CdfBxO5FQ8fBU8zbWE7jkgBMNqrfi4ndG4ycbTloMitQAhljl3OTY+8TlSXMWSVNFHRCoyomN7ydbfNTeXW+2LAeHcSnx9Nh00FrkFmy0MjyOc9Ju5NEZEfgXJkFiJmyADKzksTURMYcHkPTo404uAoLg/badjbcuIH80HKOD9/PwJpV13RNf4S3fd9G36bnwbwHsXUXd1LKr2iDN8mlUkKVYSNo8InGM/8kNae9YUJwV6y5uIGQtF8BOP1uu6TYFz4lnBmfziB3Ry5ypRynwN79Syzhm+HfAJYTkIYz50jcvu33LglpUaLD0ZsOR2/CLDRyOfV34tWnXsW50ZlBF+tprmjDxb/7+dn8WTnNl4TExriXx2Hj2oPh7mDVpf1viXF9NdReqqVgfwFqe7XFYt/Ip0YSNSuK5rJmyXu317az57E9jHhqhEWGo6m+oavQ1+osLWz9dPdPDLAdgMKooGZPDVxRozRoDGRuyMQt0u0v9fwAIbFfn1vfq1+erZstHsV7cK7MIm3yE/Qs/mdvyyZ9dToZP2Vw3brriFnYfZ9pmjR8PexrVDYqlp5YKrnHbFxtKDpURFNREyo7FaETQ3v1PPyrcX7y48Tu/xBZWgYgToQ051YzYO8XXT83Pv+gyEs0eZ0jk6of4o0n36AsPJbZkT4orWHe6nmc/TaD4r1CIbHeuYT0/lu5siKhb9fz1dCviJ4fzdgXx0rOq94/nnr/eJS9rKICMnagt3Ygy0P6XAHIzCb6nfiOOt84YL74ms2rKfT6mG3TBxHqO0kUs3a1ZfsNsZR4nGLJqiWYlCr4vX7kVJ3DoG0FmObGcz7hPJvnbCZJJoyPP07/kfIz5Tynfc6iJBOAg9yJhJw3OTXkaTybhWTjvB/m0VLRgm+yL6pPdMzfJJxrVuJlQPz8DC/dwFn3R6h0WQ9Mp7msmcpzlfgk+WA0GLn7u+doURXhVHOZ7AI10684Xt4mkHo8ajyIzI5EMUw8jvpkH8LtkgqtMpF+l/thbPnrpGQPXX8In6M+XL/2eppub8LG1QaveC+S7xdkNftf1x9No4bcXbmSbmLlQCVrb1zLpNBJjL5Fahzf1HSCsJKArsfRdMWkJ1crqQwdinf+SeLS4zjj8AUWWxZ6gXuUOz4DfSRd08LfgdhDgm9Rx4OPc6WxO0CzqZ598SP5x67FyEytgJAA6/RW/f7mb5GZZVgZw7g/sEersFJJq4OaDhuBGX+q5Rd6sh4cOmKQm53ZP24/QxKk3qe9wag1YNdUQX7CPCLmShMhLuMT2V8QglFphclkFs29AMq2esLyhDlB1mO959sWwO0/fcuFmMNsXWjg5v6+pPyjEACFlYKS/TuZctCBTfOcaHJu6lIi6ERl2AhMChVKXRsaew9RkaITlmTaOlHSfwpl/SZg5Wglia2bvA6FUYGDlYNEVsfWYIdLjZ78sHyabM4ikyVx/vvztNW0MfzR4fidt2Xc4ZvYPWk3WlupfHlR0nyqnHLwrPLEo84Tk9FMwLAAxv1zHEfe3M0yt+9ZvWAJ7XbtmHsU2QsPFnJ5y2WGPjT0qmsPS0khpaMNra5BVIYNJyJOStRTBviQ17+eiwP6ETo9AK94Vxb8tADfQUKCKi+iCMcGaxb/dCMtM3NgYXfR2aw34djkiNZKS2jpdQT6iQvSg3f44Fl4L6WfvsvXdz2O/bBNcEvvRcsrIbcPpM71MG71QiGvZ7Hv/PfnOfnuSR4ufhiltRKVrYqwKWFdkottNiUMTB3IzG1tlI7NJ+Cm7mR2/cVyxv9WSbHvAJQGJQaj+Nm18wqjQxFKSKEXQRe2oAmLpjXIjYTdHwAQcvo5Vj3wTx47EEPkwcH4TOouAtTl1PFRpEA6s7Tem/reVBoLGrmw+oLFYp9vmS+b5m2i/7DeZdrGvSQlNAB0Nhe/8eQbDFffJYrJlXLODMqmMCgXK40VSmP3/a9p1OD7WCbTB05n+4ztVPhW4C7rLlJMfM1y53wn7Dzt+G6J0Ck4PWK6JB4yLoQFaxew4YYNNBY1SvwIi9TbyQt4mX7HpqOtFRf6dG06nnr9GXJDszk/4Dy6SPGcpA1W8ttcwQu5IsiDSdHiooV18dtcv30Qrzy9m3b/B7H3MAJ9K/a5Rbox/PHhvXq1mYtrkJllbJi/gWGx9+AV74WNmw0lR0sYYBiAe407GmsN6YHLgS9pqWihvbYdtb0ax35+nB7WTK1HLRHtwnzkk+RDe43QCXJsZg55dQXsca2m1QznzWbaatrYePNGms1GxqSMwdGjkXG37mDV3B084P5An64JoPJ8JRWpFUTNjuryObsS5T7l+Fb4CkS6KeKY/Uh7NlZuZLLdZB4MelBy7Jjnx+Ae5c7Xw75m8AODmfaBmMQqk8nYeNNGLm24RMSMCG7ceiMgPM9X2ie8/Y+3eXZ+7+Sga0Vn4eXIv44w4skRWDmIx3/nEGfmr5mPd4KUkAnCPdwbrIYkcLI+CsfavC6ZTrMZvPKFTpeawESqgwejsXPFrFB1jdPZW4Rumwurha59S8W+inMV7P14L64Orgy9b6iIrAlww7YbmLxxMtlvZtOU0yTq1lRaK2mcdgP5vTSKGZuNuNe40+TU1EtHiln0b6MRDr50mIG7DtHi7ML+K5pc5GYpcfFqmPTGpF5jSmslhfGzcKzJR6ltBcRFraoLVQReFKT7tZWhqO1uYenxbsJss6EUpV7JfZ/cR3G/Sni8+9ho54GMuHSSY9FDGXZ8GK4ysTpGzF47RmU/yFfLvsKm3URhTTrQ93xT4tJEBt4+EJlCui6pvljNyXdPUnuplqkfTJV8l6OfG803I7+xWPg1aA0cWnkIa2drNI0aFqxdQOz10m7M/wv4b/Hsy92Ri3eCN0MfHmox3qnWETlLSj7xy9iFXX0pqpPfc2H1YOJvvmJdZIZzU57EsSaPIQOlVgsKR3vOTnsWucmAQteByXR16cdr+bwyN2SSviadJYeXAMJ9s++pfSTcnkD41HCmbA2n0kvB8eHHMRlMbLplE5d/u0zsDbGEPRPGkJQhmGVmOn6vau18UHjOdC26HmtxYc4a8eSIrlycS2ET/Y39AUFdpfPzS/smDYC5WSG03xDBwzNT2RPwE0qrGX2/sD9AXXYdW+/Zyi17biF0gngNsPOzncxZOQfHVx2ZOFpYy6hsVV22N/kxcVR4b+fBDx9Ed0cBLOsmxblYuZGUvw6AsDPr0Hd4w9PCHjN9TTq1D27CK3Ew8lg5RoURNyvLyjV/BF2b7u9i3//j+Pvb/Rv/HZDJKY8cS6hroyRkNgsFntm/zaZ59QFYLN7Ut6TmMHD3zxTHTKXeJ4aei7NqL6GA+MLKFyiNOgVv3yh67x5/TfRTzo4cTr+6D+fff27L1NPXxZemScP+Z/cTNCZIlFjvhKm0HP/MDGoDBmJwljKt4tK8sG/x5ehUqKvSA93JdYW9igPjD+Bf4o9v3Qzi3MUFDbmrM5V+KryqvJi5LY6m1rNwS7cf0B3H7+BfNq+A2YzO2hGzue86whd/uciZT84w7aNpeMZIpdx+uPWbXo+19bTnrce/JeqSGyMOHaWlPBQ3t+5zl/8+JE3cOxGntr30LPalfp2K/akisiOz0VrLud+jmzVt0BjI35PPQKK5FH0ON6lFzL+FTrPgxqJGSbHPsdJIUlUSIQUh+HTsA27l+wnfY+dpx4K1C2hz8UNn48iAve9QezgZngruOtYhwIW8xOtwqCskKkpP+tp04m4QyyAl3Z3EoHsG/anzHvvSWIlsYydkjo7ITUYcaguwVOzDbAIzmAxmoIfsjUKG1lpLUFEQyUfqqM6sFRX7UrUxGBJUqLRtJN01QLKxz96azd4nBW+X3J2WpSl7Q32e4HuUcHuCxXhDfgNfJn/JmBVjJIUaR39HZn8zm/badjrqOyRFSFWOIGdWFDuNOv8ebXvAz9N/xqA0oDQo8VJ78QzdXS3aZi2bbt4EWE62/TswGU3MXzO/VwZZW4sJvdqOdkfv3wc28TN94PkDVJ6rZOgj0k27plFD7SVBFrjwUKGEXay2V3Pz7ptZPXk1P83+ibvT7rbYKfo/CUtEDpWTLVXBySgMOmyUBlR24vu8JKKWIh/huuocDqA36QA18TfF4zkhnsdm59Cme4JDYw7R6ghXFvtMBhNGrRGjrnc2VuSJ71GatMCdklhR/Cz8Lu/Ho/A0IPZi6yippd+xjbQ5+VAXIL3HWs7tYKhmKCeGn8DFTppc9W1YjMroQofNKkyK7jlLZ+1AnW8cDS7dY3Bn8r5LDuwqQ319Vg1z17ThN/RuCqOFsdc13LUridMeoyBl0m4m7ZmESieViAk7uIWkin5kxFzi4pZ8Ko7nc+y1Y4RNDmPmFzPxKramYqA9L72wksejPxMd2zrUlq/D3iSoMIh5v86jwUfcHWJbexy/eiXQj4+Wf8TzfpZ9pyzh0qZLrJu/jsBRgdx+WNr9HxgeiP6onqi8KLQtWjxjPLnn/D1d8YW/LOSjfh+x+9HdkmJfJzu7t27JdfMFBrpHtQeBxYG0TGuGAKFbW26tpqzfRHJ93mLksZFEpWuE9U4fp+PmsmbSvk3Df7i/RIrLaASDygalvoO20gZ6FvuOFPyIc10rbz32JsOzhVbghCUJXcnlThN7PVn0JCRqfU38uugy5W4ZTNg7AVvvVlHnXmt6AUEXc5nccQ6toy1jpvTdS0LXYcAn9yg1AQkgl3aUK2yt8Mk5imvFRYyG55GrxR/WjN/GEVQcRKtdK+oe66KO6la8809yPvwAFwNPAA9y49YbaShowC/Zj9LXvsOz2o2EtASiLkehfVzsr1cT3P0cV4WNQNkjV/28/nmLfjgAjjV52DWUUB08xKJHRoVvBQqDAocOB8xms+h+8kny47MH9zPnlzA8z+yE15O6ktPxN8XjHXkLp037SExNRK2XftYNPv1psq1l7JFR9M+Iw/SDicaiRg48J0h++jZlElASgFlmxqNenKxI+zaN89+dx97LnhFPSGVmAzJ24FqeQeP8G6FH17dTnA/f3/IVttqDhJu/lBy7v+1DDsUKN05s8wfc5f0ADXkN1FysYdxL4zg8IRWnKiN3fnUXpiQxycWU08o/3v0Hx0dkMfxYBPIk8Vq0KLqJXN9zfJaZzMyQNly9BUnYlooWMCPxmLoSzXIrvrzzS1wag3hmyDOS+IDbBuAV74W+XS94wCrlFB4oFPyQbjTjX+KPSq+iNECLwl78fZiMgn9Wu9tYjg92ZmmyeI07Qf4kyswnkRtP41G8m7YiG5S2vlQHJeFekoZbLXjlhLE75jy0fcCQK7xLlVbCHGmpmNyJBWsXWJT9Bbh59c1UeVXROKNREis6UsQvi35h/GvjSbxN2sYZtc2VfiW38d2S7zAiLmDKVQrwXkNGnIzlHy3HLJPB7/apMoUMTaQdte61Xa/vnLM23LABWw9bSdHmSlxZnLb07Fk7WxM0JogBtw6wuGaxPvgVMwuG8/LzL7A6XNxBJ5PLMLj54NBSwU0/3oTRRwtXqAebzd3EjTrHA6S2beYOuj14L0XnUuZ9CaNCGECvVULuqkWJh27iTWHRgspbuAcG3TsIBx8HdK067v/4fgCqvNppW9FG+pp0LvxwgaS7kzhQ+SslHl9z/drrCS3Sw+tw/M3jlJ0qI+7GOLLkv1DldZIPymyoLvJmooMGU5iJ2qxa7EYKsr+zfVt4yAXOacFk7vuFZazN4Njrx5j99eyurv9OmM1m1i1aR2R2JLfGSG0cOr/r+PfiWf3Vau7Pul/ymppMQWY5e2u26L5prWzl9WWvo9wmPCM9//aVcG5x/kuTnz6JPty08yaqM6otEq3Svk2j7nJdV6d2T2y8aSPeid4Mf3S4JCaztkZnZ02tnbA/7Cz2aW0EYobMZKTDwbNrUdH5iHjGeVKdLuRGBt1neW+5ZfUWKr6uIJ98wseFS/YNNm3C2HZ07FEed3lccvxuhxupjionvvBzQFz1M+w3cP/H99Nu087+s/sJ2ynefyYfj6FN7YlKryK4MBTD/Wacf5fF7LB3ZMLeCeSG5+JX5odnfQhYsC7oDc2lzdTn1eOd4I21k3Q/Zd9Qgm/uERq9+9EznxQ4KpCsYUtwrrqMTYh0PHH7ZS0PdDwgyBEqxeOgpraV0LNraNHFM+jMIGQq8d82KsxorbRorbS8+493me0tvb97Q2NhI58P/JzBDw6WSNjC74S1Bg27HtlFxdkKyXfZ+R5Xdrt2Qm2v5t6Me9E0CP7dnWSc/4v4byn23bjtxqt6pS2/tLzXWN7Ypeh0MHDHK6SvSZcU+4wqaxp8Y5BL03OYzcJeOmb/+6i1rZjuWiGKWXp9XxEyLoRpH03DNUzYL6psVcz6alaX8khQoTdRlwMZc3gM+WPzu8bYjLUZ9H+3P289+hYRORFE7wqB+2H2N7NpKmrCf6g/Ecf6MXHPREYeG0lJsKBqcyURaNuN2WQ5fMKa4+6E5ofynEsrxMDo50dT2aDlw8svU+lazoFisC8YxniTEYX82pRUeoNzsDPR86IleUGAYnMxBZ4FNNc18w+lYKMRMSOiS57Vu+MpTkV/jJX2KNODxMQmXasOt/xtpPb/hAG1ozCVdX8ZzsHOAESlpuCZ7MmaWzbz44Bra3B4IOcBWqtar7pe/Bv/b+Bvz76/8R+DXC4nKioKuVxOxKnV+GYfxDRE2qWj69Ax8chLhBUORyWTLszUns7UBCbil7Wf2EMfS+IKgwKFQcHRkUfJis4WxdrrO4g88T2DTw1m/L7xDEgTL7adg5xxCndHa+vMpRFL8X7spj5fn75Nz+mPT7N56Waq0qUa6uaqarzzT+CXtQ9VdZkkfnRcGxdjspiyMR/t5UJRzGASFpGlAaVkDEvCPjpAFFfHRXF0kh2FwYVciLtAoYv4upVWSuxGJtDsHopNc9VVE9o90V7TTuHBQo6+crTX19i32BOTEUNDQQOaRg0XVl+gKr0KnVFPq10hdm122Dc2o+8hSRhwzJV7Pr2Hk0NPUhQrZeme/ews8hMplPmXcXz4Pqyv0Mh29Hdk5uczyYko5rr11xF6KqfP19QXPJj/IDfvvtniItkn14ZZW2cRmh+KdaMwiRfsLxB5NRrUtmQNW4LrdDGTy8rZhgbfGIrjZpC5IZNjH58XxYuPFvOS/KWulvtrxZjnx1jcJALIwsPQWTviUFdoMR6R8iODtr+M/gupAbq+VY9PuQ/lvuUcG++AS2SPgrWNDR2O3gRc2sOZz6Remj/P+5n2WqFg0NtmszeETQpjyeElXVJHPVF4sLCrA60nlNZKcs61sfeJvRb18Nt9wymIn02DTwwmpXQRlBeeR1FwEXnheZwKOiWK9ZQO6g1Xjn19RUddBx9GfMiBF6SeTADvL0vHJ+8YVSFDBSPIHpj63lRu2XMLU96ZImEQO/g4cMMWIYt1acMliafPgecPsHryarwTvAkZH/KnJVz+DBxqCzHLFVBTK4lZezlREjudwoS51Iy9Dnsv8XllDCtj19RduNS7EFigoqOxlebSZl53eZ3PfVbiVJ3Nr/N+Zca2GTz4nljO1srRitaqVpqKxH6pnfDKO45jXQF6OxeL8WbPcKxba3GuypbETAYTSl07tQEDaf698++3239j71NC8Ts8zYsRx4TEek/Ws0FjwKHmAgXOt/Dz9T+zdc6+rliHkw8FifMpDhK8QIIKg/A7J4yR0z+ezvRPpluUfOyEtYsN1UFJJJ/0YPRBoaiVtzuPVaNWUXSkCL2bjOMjjrPyxZUU95MyTrUOzrTbtBN7MZojmxu6ns+AkQE4BzlTeNOzpIwQvDp7JmY7fy73Lefj+z6mLVnM1v7srn9hUAjzRYtDC1hgL/eGTpns4iPFFuP2HfakJqYSlBKEZ4wnBo2Bk++dFHnOTnl3CpPflkr0dmR2MHnXZPye9WPnUzt7PYfgwmBmbZ1FQ063PnVn8nfksZFUeVaxdc7JPl8TgNpOzbiXxxE5U/pdmExwacQd5CYtROUlldy2TtFy51d34lHjwcBBwmef8VMGaavSAKE4GXk5ksffeJyKtXvF12xsodxN8FcenDIY7/QeRbWcUnzyjmOtkaEyOqOWd4+LfzT2KWzUXBpxB01eUbRWt0nixjYNGns3agITMZmk90BLiCDvad9mj8xKfA85Rnnz2/VKSv1LGXxqMG0VzXgneBM9LxqFWkHVBDlvPPkGTU5N2HTY4CEPtniOneiZCJEr5b0+X441efhlH8K+ocSibDvA9O3TWfz4YrRN4m6eRk0j9fZHKAoqosrPFpmMrsSwplGDtecAWrwns39SPocn1ErOa09MCBOHj+HVWamcn3kMMzLsPLuJOecSzlHlVUXy6WTG7hcnwaPmCElahZX0usxm8CpMQaVrpzlXqsqR15hDneNBRu1ux7BP6vHssKeOR996lOnbpqNKE7pEL667SPqPgne2GTPlfuXsWzgD11niQqPZRcWJoSeo8GmkPGIU8mhxMjkoagW2fv9kio0d++54i4wRgi/wO77v8I7fOxi0BppKLI/tJrMJjY2GercWi0Qp7wHe6Fp1fNzvY0pPlaKwUhAwPEDoADTDsq+XEVQUxOWhs3DpL/Zsde7vx7EZo2jwG4xn0wysFbZsumUTL6texmQw0fHNzyTsfoPDA5bz+hOvUzZCj8rJluK4mWSMERKASUUh5AypZezEb8C6+3txCnRihXkFTzU9ZfG6CvYXkL42XUKK6UTpwlKyI7OJlUm7NppMTWQps3jvnY/59SHpGiRMk4xbvT+BRYHYl4qfbbMZ2tTCWHox5iKXoi93xawcrKhbFkjKkBQAlqxawqzXhfGs8nwlKR+msPvx3RbPF6C9rp1x+8fx4osvEvWatI3IqBMkc2d9OYv+10k7Fpsdm6nxEApEBvF2BJWNirz5iWxYsIFNczdhjBaPN6o8HfM2ziMmI4b5G+ajuSAmrRWGVXJs5DEW5EUTfPYJci+d7vO6L31tOh+EfUDxUctz1hHth13/NhhNtNe149Hfg+BxwaCCQ6MPUelViW2bHLPR3EVqq82qpe5ULjetFvaydb8315ccK8E5xBmA8PM+DD2TzCPt3kR/eRttR+tw8HXAb7AfjQdLqPGsoUmpoqbMHbfL4eg6+u753rkmLzst3fOaMdPo0kjKkBTkIdLPqOVoC7N/m01j/0aS7k6SxEuOl3DyXWEOvWmHeK/e0dDRVeiLuD2C6PndHRSOAY7EXh9Lwt0JADwT8UyX3P9fhfAp4Qx/dHhXQf5KFOwtIH1Neq/Hpv+Yzp7H9lgsZps6tKg7mpD93iHc+ZK6wIHU+8TgXnqe8DM/MWjrSjzzT3bFhz4i7EMXbVzEjI8td7kcDD3I+gXr+fa2bwkaLVVBmbpuKi+++CKHxh+SkDm1zVrGfOvCotXR2DQXSd88UrhH2+zaLEqUxqdGkpCWgHutO4HF/pjNZkJmxXJm5gpKw+MYdXQU/qX+hOeG0++i9NyuhrWz1vLd2O+ouVgjiRl1RmRmM4VxM4UiaQ/YutnS6hZEaf/JaCLj6WjoYKVsZdf+vDHYnkvRl/j8ns/JmSpmHOvb9YTkueJR48EPt/zA/uvFfguZ41r4/J7PMSo7yQF9L6IrrBQ4+jvSWNBIR73Ui1Btpybupjhu3X+rMEb0wP7n9xOzOIbZX82WxOQKOZ4xngSODGTQ3YN69dv8I/yZfe//q3DwdaAspYyKcxXXfGwnD+zSyDuZ+YWU9OhUdRnHmjyL5BJjcxsRp9ag1raisXURrRX/3WKfV7wX9bn1bL9fUEaSK+RkrstkxwOCdO3HD//GqiWrSB2YiqOfIx4xQs4oZnEMdecEb7p+Wf2I2Cl0JA68fWAXWduvwpeRx0aiU+losxc+AKPeiEEjjHtl9gfRq/X4lfkxfcd0tPlCbmncS+NQ3OjIiRHHKfCoxnrPKCK3XaC1obXvF/YHWDN9DYBFMpHNcBt2Tt1J7JuxHHvzGAAZP2Zw5J+CJ63MaKDVTsfeSXtRRrhiMprI3JCJrk2HtllLSOYF/ItV/Ou5FRiv67YasXK0omGgkoyYDNpt25GZlddMJnINdyVwROBVcwJ/46/Bf3rs+7uz72/8R6FWC8kYra0LRqU1lsQYmhoaGXSkhJTkFIY+vV4S7/A3cmRcDRGZ7rjXS+Uulny7BKcmJ9559B28zOIuCpPehG1zBY7NjgxOGUxRkNiLwjvBm3l7l3N/3wlWXbDzsuOGrTewduZacrbn4BUnbqlXxPXnYqmZmOObMRR7AGJWdLOrkvP2qThoBjLW1VkUa6ls4tbvbuVizEXMtiaLg7xZZqDWo5aNCzay3FPsR1V5vhLD6ECqancQffwbtI2P0pPB1huS70tm/3P7e5UTjD8fT0hBCAPTBlI4vRDfJF823bKJsS+Nxf+2aPxL/LkQfwGd94s8GCReNPqqg9DrS2hy98VgJS3UHL71MDsvdydUey5Eku5Kwucfrqj1ao4PK+zT9fQVLiEuEjmnTnjG3cJa36kUBKcxsfo4IDDpOlmcIakbcKzN5/zkx1H22D9ceQ1ZI5aiShI/BZVplV3/3v7AdmIXx/Za5LKE05+cxtrZ2iJr1GyGvKSFmHphOGWHnSW5xpnKEGlRrDG7kbu/uJs9E/dQFgJq5x7SpkZhA6ezcuDgioPELI7BPaqbNTjzi5k0FTcRf1M8jgHXJuNp42pDe007jbbSLkuA/N35GLVGi59TWamZ9efDsR7oLJJ87IRbtCcXm21J2PMWOit74NFez2Oov7hwq7RS8kT9E70y569E59jXVyhtlIRPDe+VcZxe6oK7/wB0NpY/S0ub9U4o1AoiZ0Zy+9HbWTVyFUdfO8qY58d0xX2TfQkaE8SMT2fgES3tQv6fRPiZtSgMOgjpXcoILLPmZb+3sfXL6seU3VNoG1OHPNEHnyQfCvYV4Fl0hrs+vwuzzExRUKnk+IDhAbj1s6BhC9g2VWKSySkfNt9iHCBj3AOY5NLvyzrQk/QJDwMQ+PstmvZtGiCwFY/MuUyVKYXlHy1HG6WCK2yfNI0a+p36lSbzoN/9P8QJyDKXn7r+PThlMDGZHpjNZpLvE3cXWoLRDTbN+YFFP4QR7CqM7+uvX4+mQUNzSbPIDyyy09/nCmybsYkSh5+QmWV8H/k8kTPkXR2uRpORFkUJGKoIzQvF7CHOrCpq9PTP7E9hcCE1njUYejzWWus2Pr/7czyrPXFpcMF4DT5BMQtjCK4O7rXIEvpBKFEFUchuFj7LxsJGdj2yC++B3tx55k7un3I/BZ4FrHxDSrjQFegYfkIgU+TuyYXXxHG/Uj/s2uywibvMZo86ZkYt64q113UQdmYTJ4amcWTUblTEXFNnX/7efA6tPES/ef1w8BF3KplMoLV3R2vvjsLC1K4NV7B70m7s2uxwaKsFghj8wOAued7Fm+/AtcyK4sBi/HvIjumqWxh9aDT5ofl8vPxjJrn9QxR3nJDM9kZXSj1/YsxOHcU/DIVJ3ePJ1cY+M3KUug7Cz/xM4a45+EUkiOL1e87gm3OYzJF3WpTRrI7wI61xJwnnE2g8V0BNrDdb79rKoHsHUeil4Fy/50hOSWb6jum0zK+j5EQT6WvSSb4vuSuZmjYwjfT4XKZGvSd67/CUH9Fb2dPo3Q+7xjL07SO5UnGhM+Fnqau7MnQ4do3lRJxeS3vdoxJiwt2f3Y1PpQ/ZydnIVeINYerl00RdjuTcwHPUhwidqbO/ns2MT2egslNhNIKjJhZHtln8TJd/cB3mmFxU9/1KnsqETC7v6vRVLo/nN/cXQQZ7J+5lSO63omOjZkXxdMvTvc47mSOXodL+f+ydZXhU59q2z9Fk4u7uQhIgBHfXQnFoKaUtNerupd3trruXtrRQWqTF3d1CIARixN11kkzGvx+LJAxr6Ib97leO7+j1Z+9yZ80sm+e59bo6GDVSvNe6VLgybs94ovKi0MqsJFaVZjQqDf1T+6OW6ik6WERdVh0TPhH4+4YdSkajDMPg3YnBZKDibA3bH9zOsJeGIfFWsmfiHpzbUwjqHEXgNXWeuuw6nDQeuNkW8W78Tra3w2dqyz155bCVVKZW8kLrCyJavaacMh79/FHO9TuHpl2Dyl48LenT24eBTw7Eyd8Jo9aIZ7ynQOMpgUOjU6nyzsPTJBPtSyYTnIu4HfvWNkaf/or2ygko7BU4+l+d9o0IpbHDEaR/olFpMMkl3T6ixl7Kl8u+ZI63BhcZJIR/RKXxZcC6X3o9zn1zjqw/svBJ8sE32Vc0jfzQPx7iQPAB2AtcJ2F70eUiP973I3PXzeXi50eZ8Znl9IjzHQtZd+kT5q6di9bT0p/UdeiJSP2Njo4BHB51GLnxurXqmsmw4pBilG5C8+KyrGV8HvE5xQeLb3hNmkYNI44Ka4tfb/HESc6WHP6Y+wez180mfq6YYeXC8AZOGk4SeSUSg0MzdPO4CA0o5+3epdW2lYu9LzLVx3LaTFZnJCkjiVanVhIvJVITfD2dspQIBTxra8v+DhN1reUolf96HwbBJ2sqbOLIm0dYtHeRyG7IL2Z48XD6netHjS6TJlUAGxduZOz7Y4l4KIJDow9xaPQhXNoG8KHPO/gm+xIwMAC5rRxJk4ngkmB+X/A77t7CD8fO0647WRmXGoRtWwjf9PuB5pnb8UoSplsjp0TSYifFuy6FUnsp351xw+nIADRPaUSywzdC4OBA5m+Zb7VgYDabuePXO6jwr8A0SbxXO5Q50PdCX1bct4Iho4YwCMvm4IxfM9C2akm6K8mimQEErcBvnv6GmLQYxt1mOTHZVNgk0Ntvm80X0i8Y9twwMtwy6H/vf0a0L3drLmc+O8O4D8fh28dXZB/11ihWjV7F6c9OM/BxMa3fiNdHUHKkBLPJLKJo1J44R+KB/bS5BGBU2GA03gmAXUsVblWZAN0NZ/YtVd1rSUddBxGTInDwcaDhSgPuUWI/V+mv5HKC0LDaRe99LY6VCglrvUnsixn1RnwrhGC3vcJKQSOa7nf0236nROa1i3bRrihHIenLpYEmnpBKu/Xuzaokzo9x5nzf9zjb/yyOukjeuYXRvq4mj67i9rUwdBrwzz1EXWBv6oPFBWWzySxwFkskgAST3vI9LR3swR5nIUcRILWcgG5waeDN13t8yACj5W+ga3JVapSScCkBu/CbT8I7+joy6JlBbLl7C/Fz40UNYNpWLdpWLQEDAqw2sOTtyKOzuZMJH00Q2cwmMx0NHcht5aK98lZxq3HvfwxXHev/K5N99bn1bFq0iaEvDrW6Jpz+9DR7ntzD2PfHMuRZy0YnZUcTyqZm2l38cQ60vJ9mM4Re2ITcoEVtkwLLLJvmjW0anOuEppTLox+j73+w2AfQlN9EfU791WPN9L6ndw8VqVlKSUgJJSElbOm1BXtve+JmxaHv0PPz6J8Znjyck4PTmbz0Y+GzipoE2twhQfhOSuLVuq8oCPydqCpB++4t5VuA4G871puRy734bVQ+aeEr6TtCmDRuLGik7WIjEpOEeFszY+QK2rS2mIy3WBn7C8TOjMUvxfq0q1ZnRq/QU+pdSr2tcF/8+vt15xVCL/zJc7se4uPn/kQmUZC5PpONCzeS8kgKrg9E882D39BhJ2bRyd2ai+sFA+seOE6rcyt9zoeT27gV5oqL9TfC8feOg1mgQ70RM83f+M/hf23t4+9i39/4X4TJZOLSpUskJCRQmjAF3ytHcP3he/QvLrFwRqR2UtbPWU+TaxNLEDs/ZdpMMkLvJSMUBuUcEdmbvP2p9EsFBDrQa2Hrbk/6hOfZ3+8FUlNSkZrFCcy2qlYcGpsB6CxWAjdHXyeVSQkaEsSC7QusUiNUKNPZPHIohV5zidKINQ8UejmVXrWkDTYw9jrhd12nHt8qX3RKHQ76i6gH+cLEnmJhQ84ZEs82U+koaM9cr8Oy6c5NaC/XEgkcHnGY2XYv3tQ1deGx/MduqE0z6tAoZEYZ6+es57GRj6FyVTFzzUy8enlRW9/BfT/ex6mBp2gONYoSIbrxTnwZ/QXeTTMwWWm63qLehtqzheRzyYw4PpOGqfU4JAv3Vteuo/hwMS1utpweuJ/C6FsrIHXUd9Bc0oxbhJtVao/L67PYf9qBAXOCGHT9AKqrK+UOTehsdBilQlfdnA1zus1ae3c69J1gMiE0YPbcu7qMKpL2/kpl5AjqQvvTcZ0vnbQ4iYCBAZQeL2XPk3u6O+xuFjuXCV1W1op9ppIyXCuzqAu2Pll3bkAhOyZuZYbsO5HNzteOA6MPUBJcgoNEJ6J6c8i/QMAJoaNLYmfbzVHehT5L+nD8veN8EfkFS88ttSqWfiPoNXrWz1pP0uIkZvw8Q2Sf9MUkRv9ztPhAIPdEPfFHv6MycjgqN3EHe1AQqH8U7plSK+7+evrrp6lxrMEoMxKoDRQJIqtc/zVl3bVrn0x2cwGdjaMNZpOZ8yvOWw3I2tyCkOm1uFVlUWXnBoiDssq0So4sP0L/x/oTPq6HNsdkNNFSKvzgEhclisTdo6dFs3HhRk6+f/Iv9Tr/O1AaNxGDjb3V6aW2kgbCz+3HqLBFqZLRXjvKIsnT51Ao8TpbUlNSabdvZ0DAwzh4O3DX/ru4ktrC7/0/xa/Kj7yIPM6MtEwOdTZ3Ej09+obvZVHfmRQxE/sbNGrFH/4KncqFvAE3ngaPO/IthAQDkxjw+ACUDoIzqHbRUC8TAgPTdWu30tGGzdM3U+9RT9+0vshNLoAQzMceW0UfdSV1SwIoDywnrX8pMUlC4SH161Ryt+Yye+1sq4kbAI1BQ5Xbej57HFZFCt36Y98dS/GhYmJmxOCwdAev//Y6mUnNDJwndh/rVScxS82YMSNRGLl2nausq+SSbhxh5V6M33cXGkfLaQBVVidzN8xl1aJV1HjXgLPl5uBZK+yBrk2uzPljDobgW+Nptve0v6GtLKkMs9TM2R/OMv/t+d2d0f79/TGZTHjv96Yuvo7sumz6+1smAG1H2vLJE5/QYdfBpSfF3fmjj00k9Io/b/z6JuperRQ49Tj8eo0Bp/pCPD1GoVfswFHtjMlkdTDXKuLnxmPrYovSXhxEmEwQdXo17c4+nD8/jokTLe26IBknZSd5/t3nMQelwj+T2f/cflzDXYmbHYdN8iR2JT5Lav9Ueocstzy2ro3Rh0ajU+o4Peg0BnvLZyW1s6XBrYrLIc8wTvMklTU5wIir5/XXa59Bb8agUFEZMQyPRHESxC4ygOqwwehtHER7DoAEGWnJaSj17iT6u2PUGmnIa0DTpKHNUY9roytFoUWsvHslk6KeY9XojzF0GggeEYxtpYm4kjjyI/IJqX9OlOhQaNsxSSWY2n/BL0+JQTOAa4t9fwWDrQPV4YNRuwdbfV6FYYWUBpVyYe4Fkb3tipoFaxewc9JOtF5CQcGoM2I2mZHKpDSdXUFsaiXnRw5BxViMRrq1RM1mqPOsp79HC4EKKNLaYzIJrBW95vciq7YDDzyo96ynw8kbib3li6JVa+ls6sTR31Hk85nN0HFV11ZqZdtrK2pkyKnBbJi9gaFRf4rs1UPM7E76Bp9qH8YFPorCXkHAwIBuiqLeGYPoUFSwfepdFJT+xLDAMNQVavQdeky2ZmSA3iGVHf2k1Jju5xW+w9Bp4Ou4r3EE2n2DMY8BNkzCy1V4TvO3zEeqkFJypITOpk6rSQ4nqQvuje5M2DuBwsOFxE+xLBLtf2E/qStSWZa5DCcfJ6rTqzn9yWlUbirCx4VzduAlXOsMDDmSS1NuGET3dHh1VLcSn+GLe4MTcWnZ1ObG0lLaQnttO1K5lK2DVnGh9yZMNOJd7Q0SLe1VrYSl7UWvlFJnDuGE3zk+XTeMht3DiP28ga5iX1tNG5+Hf07s7bHcvlqsTzj0paEU7C1gRcoKXjW8KiocDA0aygEOiI4DCHIWfM59405g9PpCZDeZhPhq37h9+NlZ+pMmvQnvSi2eXsIabmvoiaG0rVpCt9mS4D6IgsgaFK7r8R/dE+88mvco2hYtx989Tv9H+nfvkV1wCXYh/b10DPYG1t2/TnRerqGuxNwew5E3j2A2mUVaU/GtT1HS/gp3/HYHGmkuBm0yB146IHyXp5Kwk23Y+kWTG5MrmqxqT1HwVsBbmKQmDo88zO1OH1rYhxxJIKh0CJuf/pH0uHMMC912035f3Kw4AgYGdNNSXo+kbZ441vtTFlAGNjLcItxQOigpP1lO2IM9jZJmibBIJixIoNf8XkjlUtqHyfk08J+4qYczqv0JAAtt+UPTL9JkzuZQsBpz0HnOhDmhbdWiclPh4evPQ19N4dIIOzI8WpANDMbXR7xO3wgKOwXRt1kXcjOZTUTmRxKZH4lmhwauq7XNenUWL/d7marzVVyqEe+1KctScIt0Q12pprm42cIXl8ql6Dx1NLg3sO/2fTQ93NQ90eYS4oJnog/rFu2ncnwlR4cd5Y74m2fx+Vdor22n6GAR3/f9nucanhPrlzvZoHJX3bChwppmdBdkQX5Uhw3Eoakcpaalm8bToySt+2/qA3tTGj8Jk1wp0PzpjD0SCruExL+1BpV7+tzD6vOrkRvkGPVG0QTIl599ybzj87DttKXpriaLZliFi4K3X3qboNIg/NX/FH32tb+lTTvaeOC64UIXxSgkhjpiKt7BXhsuaJGtz6Pf9t+ojBxOUXw8GjvBTzNw88xEAPen3Y++Qy9qkALh/cxLWYBJJkditJRPAbi0Lot+O4Wm846E/th7TbK4dxpJI5hh5NGZjAyz7Jho1vawOvQqeJEpzpYU0d75SgY2D+RCnwvcvvl26gaIk/x/Bf/+/kz4dEL3xNS1yNyQybb7trFw50IiJkaI9r17T91L5vpMig4WETrasrFS06jhQ68PhQK9BMZ/OP6Gv+G/wr8T9/4n4FKdQ8S5deQMXoLJ9O/pmv2nsWaiMA12LQXntdjz5B7AujyAW/EFPLOPkTN4CR0NHhZTtWYzFCdNJyJtPRIncQ5M7ulKRdRITFdlMf6Txc8LKy/QWNDInD+E3Fdncyd/zv+TvkuFondwkTf1bnVU+woN7OdXnOfgyweJmxOHXq1n1OFRnB54HvvbBP/g28Rv0bXpWHxoMVobV+o8iwR5AYm4UDd56xDkehlxX3xBnEc7Fa52GHVGvogQfJVH3B7h0Xt3Ebn4ICfmHkTpcgu8v/8Cg54eRMavGdRm1oqkjdL/6KTPhT5sn7qdCSOFvE3WhiyMWiPjPxhPVvQBghRG7lizAJWxE9+lviQ/mEzc7Diq5CZqfARWuLCCMLSKK4CQB3GPckfraESlEfaSUYdG0eGRAdx8se/AC4Kvl7goESf/W8uX/o1bw//W2teFv2ep/8b/GUiNeujUiLrvTQrIis/CUe1I+a7DouMMeU3M3jCbgLIATsWMENkrooeya/IuFq5ZSP9DsRa2ro2ud+FqWlxakMksE8rlp8v5Y9AnxJxcSczJlTT+vuemr8eoN6KuVOPXz6+bw/panGr7DPt2ey7Gr0OjEtvHb5Hz0DcPoZM1iGxKXxXvvfAe6b3TCSgsp7PSUlumuTid2EtygkqDuG3LbcizLKfwBj3TU606PvQ4crub7zioz6mn8EDhDacktk87wdr5a9H4D8Q11BVNo4aNd2wkY00GcgcFe8bvodGtEb+SdpH4fJcI721/hhOx7xvRZ+tbtdhqbJGYJUjMYLpmgqqtqo3fp/6Of5mcs/3Pigq7/wo5W3JY0W8FJUetUI4Af87bQMsnK/n8aev2AVf2cPuB04QUCc8y/ed0sjcJ+m+V0SMp6nM7/Xb+g5qfdlocJ7WR0+ngiVGpQqbvxKixvCc2jjb49fOj30P9eLrqaZIWizW+/grOQeICdhfMVdX4FJ1GqbFOZ2WQCQl1JeKgKCywD3KXnxl88Qtm/2ym7LjlfTG5Xe3qdPbj3Iin8Yi3LCBp1Vo66oRgJu37NG4FXfSozlami0Cg6tx6z1ZOfSzuGJWqbGjwT6TDSSxeDVB9cDdu1cJzK+wjntrySPRAG6BF7ahG62H5rNSVat6QvMG6meKk038C/R7ux7gPbqzj4lhfiG/+cWQGMQXRDwN/YEW/FRTsLaC13LJQ0lzUzOdhn7Ny6Eqip0cTM91SR0imlJF4ZyLpP6fzhuQN6rKtJ6D+O9AQ1IcWb3GhD0DX2olLdQ5uFZdwupImol4KKHQlJieGes96MpIykDn1FLlUXo5Uhwld1DKH50gq+cHiWHWVmh0P7iB3ay43gkfpeVxzxO+YyQQaRy/MEgnKjmZRQKVrbse97AJ26pruTWjipxMZ/ZZQoFZ2ylB1qNh3uxdhsy2npqQ2ctL7pFMeWM6QE0MYelJ4H1orWrFvqURmArdGYQ1q8DQijYpHIpGwc9lOCvYUiNbca9Fa1MzsDbOJyo3i80ohUZF8fzKzfp+Fwk6Bs3sgaic9vS66oj4pTrb1OR/HqIOjCCoJQtfUyJkvzvCG5A2Ov3ec9qp27lxzJz7VPmydthV9uOX63B6rYMPsDSj0Cp798FmCTll2ms/+Yzbz1s2jJLiEg6MOYna5+T2rLruOw28cpvhwsVV73NI4qn2qGfnVSBrzGvFK8GLJ8SUMf2U4SOCDZz7Ao96Dirli6jGJrYQWlxb0Sr1V7aK8gc1smbGZb5rhldJolNdURWw9HTk/+RWykn2456d7WLhK3En+V2iva2fPk3u4skNMFWs0glN9Ib4FJ8nYWSayd+21B8YcwDhEKPwPfXFod1OIIm44qf2FJqkN9a9ZHCsPceKbB78hIzEDlyYXTE09tFHqKjVtF/JwaFEjMUv46JmPKJ8vppW64TXVa4g9+RNyXQeuMeI12j4+BJ3KiaDLu9Bprp+kAbfz6xl4eiB/zP4V+0g/HHwdmLdxHol3JNKZ28Ljnz9OeEE4JSEltCs1DH1pKF69vAgbG4bLeTNzN8wluCSYgIJayiuzLD47e9hSdk/LZ/vEL/jq4a+QO1gWzad8M4W5f861el1Sg452F38qo0aIChYA+8bvY9fkXd20vSVHS9j2wDbqc+pRhtiy8faNeNZ5Epcm+IPfJ3/Pe67voe/QU628RLNTBUlHT5Jw8DNRE9WqxavZNvIQ7336Bsu+eBBdhx5Hf0fyduZh2pDPI189QnBxMH2KVomm0k+8d4LPwz+n5IjY7zGZQK5tx1Zdh75dvK64D4rgm4d/JS8yD6NETPVnMpswS81U+VWhczMTMCCA6Sund/vL5eMfZ+/ESyxetRjdiSLUVWraa9uR2cgwlXbw7sb5nGuK4e5fFhN+VCjmm81mYmcJfr6kpZg7rqSQtm8ArjnCPY++LZrISZGMfXcsj+Y9avVZhPUewA/3/sCZ/mew8xczB5zpPEOuUy4vHRaStS4hLiTckdDNUBBe+zT+VYkEXylGXXqdb15YzdQdE9ApdaxatApduJneS3oz6h/CpJzG3IxOUYt7gzsPffsQtqdq0bZ04laViXfJJabumEqu2oaVshaKA0uRXEOv2l4rUOJl/JohOmcA3z6+zFg1gzHvjrFq/0fvf2AeaObO3DtFNm2ZlmFHh+Gs9kdrhd4uO+tnZE2/cbH3RSrimi1sSkcb3nr1LbZP2864veNYsLFnuqWzuRPj4XSGpr3K6MsF2Bh6EtV5u/IoPV7KsX8e48CLB0j9OlX0vSqVik3PbWLbsm3YKcTPyq+fHxM/nUhdZp1V6kjfbC1xWXHsm5CNfUIYRQeLOP3xaU5/ehqtRsuIoyOYsGcCT3zyBP5plo0/ZqmgbWqSmTDKjVwvfW6jU2DXZs87TbCj/da07QAWbF/AAxce6P5vXbuum/o+a0wza+et5cf7fsRhYDAqNxV2HnZ0Nnci1UhZsvJ+Zv45k5hMX3TtOjYv3sxbyrcw6U09ce+FBNyOC/SpG+Zu4D239wBo9lRT41PDrvylHL/8MQO9xqCuVLP5rs3kfneEorBq/Pp8xvqZbxMWH4ST880nCg+9doi3VW/TmN8ospkx8+0D31LnUYevvbiAKLeRI3OUMWnXJGw/EDcsecV7oVPrOPXhKf5cYNlcYNAa8G7wZvoWoVlNp+5Zj3K35FKXUQ3qNhrdGjk45iBefawIXv2b6HtfX4a/NhzXMFerrB8mg4nmomZOfyym8TabzRQfKb6hzy2PCKU8bgI5Q+4lc+Sy7mJf19B7XspCipOmY5Irr34e3dR3/wp139fx2j9e46V3XiJnU47I3ndXX2JzYvGu8UbXYbm+dxo60Sv1FEQU0HF1Guz0Z6f5eeTPdNR3EJvbiwe+f4l5mz4mIF3cDD2o5B1mlj3PNMUefBtTMRrMNGYKE4Lu5RdRdLai1Cpxa3Ajunis6Pi/gkuwyw0ZSqRyKVKjnujTq61S8OPsTKOf0Pyh0IjZjQJP1zPmwBhGHkrEnGp5vF6jJ6wgDNdGV8ZuV2Dzu6U8Rv+Lw5i4ZyIGuYHf5/9O5cibpxrsaOjg0ppLeMZ6WmUg8k70xjfZl98m/2Y13pbbyNm5bCcn3jshttnK6fdgPy76XaSovUjUPP5/HTYdAjORXCumhf/fQt/7+zL6n6PxjLP+Ht6x+w6WHFvC4GfEEixqv2hKek0m8syv/DblNwtbxYlitPZunJv6OjZjhoo/WCajKmoEgdn76bf9DYwdPX6btcLfrVJDWnyVUkbSXUkEDAoA4PY/B7P4l8XM/HMmVReqKD5UDED+7nwWHV9ERVgo9pJlmDqEON4pQNhXPGI82JD9OUbjJUYeGklwnmB/3fx6d6H9zIBznBhygkVVsDgvEbPZbDG9p9ArMBkFX2mdmv/oO1yfXU/ql6k0FzeLbKZ8LcOPDcel2QX9VbarKV9PYdIXgp7s5d717JiyA/caA4aGVjyiPZj6zVRCRoRgNBhRdahQ6BTMXzsf6ZGeRiyJTIKNWsbiVYtxanFi3bx11M67eW10gDHvjiF4ePB/VJ/2b/zfxN/Fvr/xfwKeJecw2Nijf/hxkVio0SwkP1JSU2j55Yzo2IKyU/TK7MVtW29j5p/i5HxX0ce/wh+PBksHXteux6UqC68aN8JL5uBXazkuf+3iXRPSH/tRN0/r0VbVxtfxX/POw+9YtUdeDuWJz54guCTYaod6VmwaGpWGEXtAW2BJMWe6ek/yIvN477n3MCRbBludg3z5+MmPUTuq6XuhL7JKy0RM78W9aY9RopfrhWD1Fjb0Llqcgn0FVu3Nng5U+lfi0y4koW2cbEhanIRfsh8yewWnBp/Coc2BpJPptF9HBarJqyb5XDL1boVorBRjFv+ymPu/v59zKec4eFsYbtfoxNl72XdPhN77472M3yimAf0rdD3r2sti7RkAuyG9MShUGBTiDbXq8PsM3fQzMWd24XlSoPDYsmQL62eu7/4bk1RGg18vbEItEwYuEZ7kDr6bRv8EEg5+hs0my0JRZ3OnoC1nBgcfh26atZvFEyVPWO3aBDAnJmGUKQi+ZJ3+K+qyluXLlyPbKy5o6PVgrwtDIYmlwyUAmcoyaaYPCOHyiIcxyRS4VOfQfp2f/ev4Xzn1kfC5hfsKuRW4R7oTPy+emBkxVu2dzZ3UXKqxWthQuDtRH5hESMY2Lq6+KLIfN/xIpW8lXy77kkZ/8TTkC/teoOW+FrZP287h+w5b2Lro16wFxv9VtNW0cf778zfkV3doKEHVVk9+8hwMNmLePp/ePvS+pzevaF+hzxJLXaZr19yNd2wUHXt53WWaS5q7//v6Kc3/brhWZmI+cFBEveES70/a1NdJH/8cBbOfFWjUrkH1wnF8/tjn3f9tMgt0NAdfOcjx57fR5hrE+YkvIjNo8c+xnGhwDnQmaGjQDfVMVS3VhGRsw/2KOEHT3g6FyXOwa6km7MKfovVVW9VE6MWtlMWOo3WoQLNSmVZJfa4wzTfu9wQe+uYhHDqjscdyz7o2SFk/dz2liQKtX9f6tX/MfjLjBfomiVnaHcB1BU5ZGywLGNeis7GDXpm9GHJiCO5XhOR5S1kLmeszaSlroffwO8gZ+hS5KZOwGRArOj4hPZIRR0dwz8p7aE/L6S6+lh4rxdbTli23beH0wNOcTz6P0cfS/fTySyKzVyY13jUURDsQFGEZ5J4eeJpTg05RFlRGQa9AnD1ufhK45GgJR5Yf4ZdRYu1RgEVJiygIL2Dv7XtxDnZG6aDEP8UfBx8HzJhpd2inMKwQWbBMNN3hbnbHtdGVkKIQ6s+LdSWrg5u5mJTBij19cHllIY0nemiZTSbheXYqKsmNziUvqvqWOm2dApwY9Y9RBAwMENlMJshPnkureygmmbigYXdGx/3f3U9pUCmmJOFeZv+ZzeHlhwGo4AxKrZKRh0Yiv3xdM4iNlBqfGjrsO7j/+/sJW9MTLGauz6T6o98YtjOju+h8S0luuZza4BTU7iFW/SKzWaDQdanJxaAVf65dqx7POk9MUq1wD3bn89OQnyg5VoLE3YbjQ45T5VsFZmjXqxnx6ggeuvQQKlcVjX1gw+wNeNR7MPygB02FxaLPb7FLQ+2kps6rTjSC2e/BfhZaUNfCP/cgvfd9iKLTegJPgoSEjATu33s/miYNdVl1nP/+PK0VrcjdlWQkZeBZ50lInrCRttcJ/9twpYHCfmpW3L+Cy/EnKAjJsFhvzGbhHeswg6tdAzKlUZju0BrRtl6T6JGYCU/9nd473rQ4ry59t2v1K6/97N77PqTXka+p3i0uMJXpSqnxyiekOARZiZjKzTEfhhwfgmOrIyatkAjZuHAjq8evBsAgt6XJTcf2KdvR9rLDJdiFpMVJuIa64q7zpiMzkuZ6F0JrvFG1CPuiQqVg7h9zWf1KGu8/9wabDJV89NRH5M4S3uE9T+1hy71bMGgNaNVaqzpYcomC8sBydk3ehWOUuNFpuftyjg07RuPrjVSmVWI2mbm05hLlp8oxGUyM3tyIW72Z9Ysa8BwQbnGsbYQHqxatIi05jcLwQiSOckwGYbLe0Gkg8pgbk3dMps2hjUMjD9EZao9diCfnprxGbj9h6nLUySFMGplB1f2/ovDqOT/3KHfmbZrHvact9WevRcz0GIY+PxSpTBz+V1ZXktGRwVntWZGtraCNMQfHEFzig7yzTXTf5IczmbJjFM7t/RimeNzCdu2fRuXF417RcPXfzXwZ/SUyPy/yEvzJ8XsFed3HmHYLGn1b7t7C6nGr8UrwYtqKacTPE9Nwmgwmqkur+fOpP9n3wT6r1+wU4MQLrS8w5p/iImdgjoKUc0OwtfsQmwBPPGM9GfzsYOJmxSF3kPPlsi/ZN24fOhsVsd6WtHyRuhSGpr2Da6MfAWUByJotiyiHx15m1dOfMl7jytDLCRiab9xocz2q06tJ/zndojDzgecHfOD5AQBVMR3kxAp+phyh8PV40eMsPrQYJ4UTvlVBJF5KZNLOZDrqO7p9U3WlGmmdkajcKKKyHbG/JPgvtZdq6XItZDop7gYFU2SHaPy5gaLNF3H0FzT7MINJYqApUCicRriJ9bj/CkoHJYZOA4dfPyyySZDQb0Q/ij8rZsYTM0T2hvIG2i+341/hj7TIevpKrxHWkesnZmpzapn/wXwUBgXOw5y5fVXP5GuvBb2QTxjN6YkTuicprK0L/xWMemMUjxU8ZlV2oEvTKmR0iPhAM/wy8he+jvvaajxz/Wl2rf21oQMxKGyJTP0NqUGLY30RqtYazGbhGXQ1/EZOieSxwsesnrNLhAutjq2UhpV2T1tfi3O3n2PVolW8//z7uMVYNixrOjRE50TjVdPjv7ZVt3Xr5HVkh+BZ50zsxVa8L4oZmfx2/0RM2iqCSw/gf3InRp2RwDkDuTRyGbl9+9J/9xqS05KZvHMyk9bfvC8IcO7bc3zo8yHlZ8TU/QCdDu7UhKSgtRMXzXz6BVDYdzbnpr6OctFcDJ0Gfhn9S3dx2feKjPjMeH649wcKx1rSmxrqNNy1+i76XOhDYZQOQ1zPO7rxzo3g6MK2mbUYZUZyY3JpC7j59aKzqZNjbx+7YUObf4o/01dOJ252nNViYE1GDUOeH8KI18UN80oHJaM+H8WPo3/kk/s/QTH01nIQ/9uoCRvEuamv0+wb+3+GxnPsO2MZ9uKwG9ojJkTckMFJ6+FPXUgKNWGDRDq02+av6Y5lrcr7GIwoO5q7/9tk7LkhR4/ewgVYQZ8lfZj63dRuliClvRKlk5Iznwp528K+Q8iLzCPxUiKt5a34DxRYIXRqHUdfPkpJmBF500ouLRK0EJZlL+N18+s4+DigOmLmyU+fZOSRkUTmiN/fjN5ZnE8+T/qpXoQtn8nxdXtQqBS8bn6d0CMj+Pjpj4l1yiXkYADHD/URNSf8uzCbzZQcLWH026O79WivRUuCHT/e8yOReZGoTwr5ztOfnubXCb8Kx2OmxqeGrYu88L59MOpKNRdXX6Qxv5HO6g6ef/95Rh0axc7JO2lJ6ZFiiZ4WTd0gIe5S6BVUBFTQ6X9rVJxDnx/K3UfuFumt/o3///B3Ofdv/J+AR+l55DoNXMfBD9BW3cpz7z1HeUA5xqfFyf2G3p28+eqbLPxtIdFXxMFgaFYeTsUT+eC5D4g3WGofaJo0RKRtwDF0AP0P9O2mJOpC/Nx4bHrH8tQzEpBI6WOZI/9L2DjZkBmXSfzGeDa8tIE5/5xjYXf2TyQjYR2Lfl1ETehuwJI+6fCow9R71DNj8+0YWy0rJW4mDxIuDaDau5A6rzoudx4HejaCZmUtrc6ttDm08c4L77DYZ4Xo/GrmulO3+xgPfPsAHSNbIfbmujNjpsdw4IUDHP3HUXrNE9MgmiVmgkqCGPlbPumD0/Hp7UPOphy8k7xxNQsdfFlxWQSoP8fW0zKREpoVQvD2aXzxyBcMLhEXHbLispAbhGXrQvgCYH63zcbJhmfrnuXOlAeIywomN9a6I38jRE+LprmwmZCRIVbtrndP5+gNpFFa3bWoo3Op8/MmVjJfZHcvS8emo4mivrNIGW5pu9b5bAhIwinQclrt/I/n2ffMPu46eBeOvo7Ye9mLaGD+Crp2HVKZ1Gr3jsRGSV1Q8g1FoqKuCBNV0cfc0bZqLYpCdZm1xBzfRm1of64MXITLdTlOmQykJgOOjSVIzCbaakNxchJT6S3at0hEG/mvEDIy5IbPCWD3E0LBdeTrI0U2oxGMcls6nHytOjk5fYrYl/AHIw+NxKfmGGDpkL968FX2FgjJqCsNlp2b9p723PbjbVYF0v+r0HfoKT5cjHdv6/fKtqMB57p8akIHYLaiwTj1W7GQdxfsvey5/dfb2XTnJoxaI9mbsom9veeBVqZWUnqslKcqn7JKffPfibij32LXejUBYxrBtUzOXb8dk1yJUank+sv2Nw/DWTOEoPxWRh8cTYt7BRo/L469LWiNhChyuDhiJs7V53FobgR6koFKByW1l2uR2VgvrvrlCQmKouF3iWw9CZcBGBS2ogDTxt+dvJQFaBw88bxqW9FPWKNfN79OcXQd+s5qQnKj0NnkAZHdx7ZVq3nikye40OcCR0YeobNceB98+/qye0ox2ZGXMMqFKsmwg96YL72B4cWXWbBtAd/1+Q59x4217pyTvFj++nKe/eBZHPYJJ7bjoR3k7chjxi8zwC6JI4mDaLfNI9TzGOBvoVO2Y8YZpJ3V+FT7cF/MXAY/FEH4+HCcApxQO6q50PdC93ddn0yLthV+Z82uzeyaspuJ8ZZamdce26foV/xuPOAqQtSUKMruLEPpZH0aMOPtDAaeHsi+ufuw97RH167jHYd3SLwzkckrJ+NZ68mpQaeYMX+GiFYn5HIIj38uJLfTUtMYlmm5XpgkQrK2w6WFBj8PbFx71kBtux7H2jxqQt4ia1QDjpr4W0pGnHj/BOe/P0+/h8Q0zCYTNPvG0uxrvfiEzoRKo0JqknY3VCUuSqS9VvA11DvfY0nNEkFHTm85EWPQG1B1qNAr9JweeJog957CrKOvIxJbJZWedmhUGhIyElBobl5fUWqrpD4wieBLOynYLsN3qeX5Nx29hELbxqXRjyK1FT/PNXeuwbfSl9u23IbaqZy4/m5CkhqQetmyf9x+ks8lc8/Ke2h0KKMz0Rt1pRqnACc0fmYyHTNxbnam3qOepcE9VNBmM7hWXsZPKaPJTil0ChtMXNszeerjU3S2dDLqjVHXnxZtroF4F50hMvV32mruEGn2fdHwBfUbhWKxrk2HR6zgK1VfqMY8Xngp1s1bR2zZezwggTt23kHJ0ZLugjTA3gl7sdH5WiR5TCYYeWgESSG13H//57x8cD8yGwUtte24R7vT4KngnaGPo7XVciXiOB2qCIvz6nNPH2RKmYjaq+ueNHlH41qTi12YeF+qbarAudmZuevnogvIBcZb2F2zJfQ6Po5x+8fR3MdMw9QG1FXq7u+q1/+MQZLDuZRGBnrJcI9yJ2hYEBKphOiUkXw7dQBPj3uPvCGn2VLTU+AqO1mGe7kdPjGwMqiGlxoN1F0tOJ/+RChuZKzOwKQ38XS1WD9RU9dK37S+lAWWWS1U2ynscFQ7El4QTkddB56xntyx+w5hcsdsRqEV1og2p06kNtclRu0VFIb3FE4NJiOF+wtJ/ymd0f8YjVe+PU6VCbjN30nblCNoeRKzWQISaHX342zKWXwiynnXA45roOqa5g+5jfyGDVAgTG/l78rnRbV12v5vH/qWoJIgbL62YcE3Cyxsjn0d+faBbxm/dzxhRR+B+TULudjskfVUtp/kztV34eNdY6Eza9QbiciLoNWplW8e/JzJl5uFf9cZcY92p9LBnvNRy2h0PM7gfXehr2oGxjPlmymsn7Weo28e5dE8sdQBCMXu7+IFmvn28HbGPWu5MZSdLOPsl2cZ+MTA7nXgWuyedJhaxW68tM5IJI/iEuLCuPeFz2jpbKHes556z3pavfrx2ChL6jqPLDvGbtPiMmAW/c64UznRcp1z1aQQ75HOq/nh7P5jMqb4TrDOmC9C6YlS9j2zDxsnG5KXChPfERMjuifiTGYzzi3OjDn2HEYvNRWprfw64VdGvzWalIdTODZjElWqV/CuC+MlT3t8+vhw8ZeLtFa0YpNhZOHvC9kwewOhjkJxvzG/kcgpgp8x4+chKLSDqbjvgqA5LJFg42jTTf9d7V1CnQ4ObB5K2b4Wyu4vJzBC3HBiDX2X9qXoQBEBg8V/r5Ap+ED1AY7+jtgrxbHCmvfX4PaFG9/d/x233yamqd3x8A7OfSMUH7ume7ug8lRxYvAJhpwcwg7TDsbUjCHBW2jmy9+Vj2HPURwHD8Snyofxe8eTH5dP8pxbm7S/ERryGqi9XEvIiBCrsVvM7THUXqolYYG4uRAJSKQSzCYz2hatSDNNd+EyEWczaPBPRGoyYNQnAlJ0tk7detFBl3biXnGJZp9oYB4SqQQ7Dzv8+vkx89eZN6R1NwwzoFurY1LIJPzjxL+duqg6ChuFtcxoNqK4hvJS06hhwVphDcmPE2Jxl2AXXMNd0XfosRvSh/T8Ptg3lWEjF6+xrZF9SApOJTnxHF/7FHKv7FWktnK0Dh7ojDacHnCaKt8qWp1acXR82Or53wg7Hrra3GrF19I0aog79j11Qcl0uIiLiNf6Z2azkEPqmlCa9fssmqc8xmpfN7S2Wi5K9wI9OSeJi4JtU7dR7VNNRcBBivUNvM2w7mYRSUQ0tf07u9fVrr3nl9G/4BnvyeQvLPXXroVzkDMTPpnAifdOEDomlLAx4sKDd4K3hbzItdh450YMnQaGvzzcqv1af/1WJ5T/t9B9ymYzwsOWCHvp/wGoK9V8FfsV/R7qx9h3xZOpunYd6SvT8U70Jnh4sJVPgMroUQx+xvLfbN3scKnMo9/2N9DaTgMsG1S0FXUkHhT2y7TJrxBtL8S3LS3w6afi77jV4uiR5UeoOl/F803PX/2AHltHyEiq5FAVC69MikCmlBFzeww+fXw4/NphVIN8KQwrwMbKpLgu2MzJQSdRu8fg2/E4ZrOZDzw/QGmv5ImSJ5CYBT/c1aOFwl6XiXLv0T41XnWIFRJIupRI37P96WzpBBfQaKCoCGJjb14r3QJmSF+ZTvzceOQ24hybTimh3b6dQacHoYkR8kKho0OxdbXFbDaTkO7BgNbb2Dr9IYo7hpK9sYNdj+4i+rZovJ5P4FzyOcoDysmKz8LOcAoQ1oDDyw/jmSbhxJTx6FR/IDE1YjLcWhN2S1kLUpkUR7//2ZzO3/ifx9+TfX/jfw1SqZSEhASkUikFyXOoiB6JNP18d2deF8xSMzXeNeRH5GOMEgcAZokZk8zEr4t+5bd7xZ31zjVXiM+M7fowC5vSSUVBn1kcGPopZ/unUhXsYmGXSCVC1UIiHHcrG5+tiy17x+8lLyIPo6d4ETYGObNz8k7yI6rQWKHFAbiccJnPntmIbZJlkNlZq2HWn5MYfHo83tXeuGotjy9tPYFbgxtSkxStrRaTzPLE192+Ds+tLbQ4t6DQK9DfQpeLR4wH4z8az+BnxfQCAPd/PYW7Vi+mwaOVFp0KhZ0C//7+OPo60l7VxgPfPkBoUShN/r2Q21sGLl5j+7Fq0SraHLRCR/p19/vY8GMcGn0I72pv+qX2o63GskteYacgtNCLwtBCNs/cddPXBALf/cw1MwkcFGjV3vYXjBq1iRLWzV/HweGfUx4p0Ow8nPkwD2Y8CIBbdRbeBScB8TWpK1rwvXIY+6YyyuIn0pFkWfD2T/Fn8HODaatu46vYr7i4SjyN9ld4x+Ed3la9bbVT1dzRSV1IChXR1vXtfl/wOycHCeddnV5tYatoLMNkzCbTfwm1TrtFUxjKkjzijn0PgENTGU3X0fbce+peZqyawYn3T4h+8/8KRr2RD30+ZPtD263ao6ZGkXy/9UC9vbIFz5JUasIGEjEpUmQfYyt4z73TB+B+VeD+WtR+W0vftL4kpScx5py4U7zPPX2sUm9ci2vXvpuFa6grKctSyN2ca1VcuiEgifRxz6Bx9ERisu70aZo05O/Ot0o3kXhHIg9nCUFz9QXLZz3+o/EEjwjm/A/nb/p8/1MwS2Q0ecfAI8tEmlG61k4c6wuxa65E0VQjoidS1rQSXNYLrY2WFucWJHIpLiEuLMtZRvJzo5HrO0ne/xt5Idv45qHvRd99//n7rWofAdQF96M4YSpaB3erdp/8YxiUKuqDk0UdljJ7FS3eUTg1FKEsEtOEevV5kYqYCfRKraLu5DYLm0QqBA86pQ65Xk6jai9mMzh4O9BuX09gWSC2GiF50+DeQG2QWkgYySRM/GwiiYusa0QAV0XsYfOMzWROEBI0XffUL8WP1qwihh1OxLnZGYOVsatWlw5KQkoweryIb0wMCjsFAQMCcPJ36i5IhBSF8NwHH9HviuU6pz50jhffeYuogpFEVr5Gh9Fy6rtrQr/XpV6Enf4ITan1CWxrcApw4vbVt3dr9FyPslNlhBeE064XCl3lp4VrrzxXiUFnYNnXyxi3b1x3UexaePXy4sKoCxwddpSox8R0s7M2zePxzx8lZ0IhX76/DIdr9Hs7atuJObuBvueFgLz/lV235GP4p/jTa0EvC43jLphMwkSsQ2Op1WN9xg9h6wK4fduTOP2aBwjFqpqLQmHdttMWk9TEVw9/hf0ky2elz2vg+fefJzktmaMjjjJg7KvdtpKjJZg7dVxOCaTNoY1ZG2fhcKDHv/hXa5/RCFKTEbmuHYOVfUFX14JDYwlSg95q57LC6IxzizN9L/Sls6oJpYOSirMV1Fys6d4Dm1ybyIzLROqiYNOiTXwd9zWF+wrppZxKr5KvaXFpIS8qD5ODZfAedGknyamBjDkwhmc/fBZNg2UT1t6n93L0Tevt0U1+8VRGDkehabE6LSw9JKxNL7W/hFOAEw7eDkRNjcItwo228y089dFTxOT2xqtdmOQNHBzI0BeG4uDjgEeJDUOODxF+9xKT6L6MPDqCoMtxXOqw43e71UKRrrCJhtwGlH0C0doKEwQnhp5g64wjFu+gk78TQ58fin+KONlrNkNBynzOTX0dp17iLvSO0zU8+emTZMdm0zI4QmRvGubE90u/51KvHJzjYmnMa6S1rLW7kSdx/2Vm/TkLAJ2pk/badrbdt42czTmYzRDsfBl7uY67nMBOIkwRa5o0/DTkJyb/HIt7TiSRNga+xR2lwORFzIwYkh9IxqQXbpK1SfnsnGPctu02ln29jOpz1SL74o7FuDW68dHT31DjGEFtZi1rJq4hZ1MOMoWMj55ezqFRh1B0VKJrs/StDSZhPU1KT+K1N16j9VQVnU2d+Cb7onRQcnxhMZ88+QmrfWCTn1DU0am1ODSUIDEb2TllJ6b4KxRlhrL9t7HoGnocU6POSF1WnYimuwveid7o2nT8MOCHGzZ9DDo1iOjvxHpMEgcJ1b7VZCRmUBEmttcHdnAl+gomCULMdA10bTruXHMnA84MwCQzkef1ESCwn9RcrKGjpQqNTKCWWz93PVsfFaZEY2fGMnv9bBIXJXLq41Pdk+/Xwmhv5GzKWbZP2U7mY2J/rbmkmcu/XyZnS44FO0EX2l00yI0SJq9vpuXQeRrzG/lx0I9c+OkCRqMR+zZ7bDptqPBYTYfeUkNLGeZHWew4KgN17Bm3D3W45XWPSH8J+aH5RPbOI2XZWgiwuWm/L25WHFK5lN2P7e7+t3kb5/FQxkMADPvFh0e+fITEND0tWeUo7BRoW7Q0FjRiMpuw04YT0bQOV+mXKOwUJCxI4P7z9+OT5ENnjIxNMzZRElxCsavg389eP5shzwvMNqqwUbR6DqHKp4bQV7+lcZCwzi3csRDfTwbh2uRKQ14gr7QaqFDVU68RP5cbQeWq4s49d9J/mXWWnHW3r+OH/j+QdyJPZLNJtmH/mP2oHdVWCw4esR5ETonkxbYXRTSNdl527Bu/j+XLl1PjXcN9L98HCHv/0X8Ia7ZTYyPOLc74Vvl2Swz8J5CzOYf1M9ez/4X9ovUAIGhIEO217VYp4yUSCU+WP8mynGXYe4vzH6aGZpzqC/EuOk3oxS1Xp5UFqssuzXE7dQ2lCZOpC0rGbBamYVO/TKWpqIkj/zjCL6OtMx4cLTnKl6lfcqzkmFW7UqbEvd6d8WXj0V43tSpzlLFn/B7a7Ntou0qfr23Voq5QY9QbaTZVkOP/EkeSn6QgRMyEsmLqfOb1+Y1IWSkHEtKRSKXomjtwq7iE1GRg96TdFIcWk9krk8ZocZP4X2H8x0L+whojglQhpdkr+oY5merzFQRnbMMv9xDmgkJsXWwZ+/5Y5m0SaO8lSju0tlpkBhnFOktfQGInI61fGhUBQvNUo6TneY94fQRFsXVUuv6CzCDjpbdfZeJmoTmmPqeesuNiKvZrIVMKup02TjaYjWInMm9XHr9O/JWq8+IJexB0x4a+YIX2EYEpZ8uCLYw6OIoBpwdgW269OPyv8O/Evf8J+OccoN+Of+BRduFf//Et4MAByM7+947d8+QetK1aqxOzAO86vcuuR3eR9YeYicXtyiniD39lMaHXBXufnsKN4Zw4VyRz7FlDzFJZt6/3ovX+n1uKR6rOV2HrasuYd4S8iFatRdemY9DTwu/TRq7Eo2087h3jkcllOHg7ILeV43E1JupzqgqjzEjqbGEf3/nITt6QvEHxkWK0MUIzW2WQDp2dC2aTGU2DpnuKcOaftzF7w2yODi/jlxf/QJXghEFr4Pvk7ykddYzZG2ZT4+zCD7NSCX/0VxTOQsz0wgvw/POw5+YVmiwhgUeuPMKAJwZYnbyO0I9Ganbkmwe/QTlfaE488/kZKk5XIJFI8C9zoO+FvkTlRqEprcUl1AUAxwBH5E5Ktk/bTla88A7IzT2/OzsPOyS2NlS6/Ija/jKPf/Y4wR/fWpP5p0Gf8rH/x90Nnn/jvw//W2tf9/f/r3zr3/gbV6HTCY6vzs4Vh6YK5Du3WVALAchdbfjl7l84O+AsRrOYa17eKiGoJAi7djskZnHgvnHueb5a9hWJl8cwodBSe0Zmq6DJvxclgRnsHb+Rxl6WhZ722nbWxv+DftvfIO7ItzT9tPmWrq/FpYU1d67BYYqYVg/MdKo62TLzBPVBfUXW0QdGM3bfWMyYRBuunbcDBX1m0W5n4KFvH0J/XUIi+Kwzj33xGH6VfvhW+mKntvz+zpZOpJ0mTg45yZePfokqwOWWrmvQU4Povbi3VVt1gJJTA0/y/dIP0ISF4OjniKO/I3qNHqPRhI3WBrlBTpXLn+KJF193CsML6XUpBp/8Yzd0NMILwpm6YyrNxU3d/9ZwpYE35W9SEF7A2vlrsdq29xfIWJPBDwN/sErjqddDy5Nv0G/7G9i2iQNbibTnu1qUwsbsGeeJd4LQ7d4+aTg7Z2lwzl+O+rilA9Ze2Yr/lSM4NAkJ5uu7i4KHBzPuvXEEDQli+KvDrXYn3wysBQDm9HQSDn2BfUul1WN0NjraHNowSo2ihLIkSs7HT39ElW8V/jkHaSqw1JaUt/T8d21wP5zDxEWR1vJWCvcV3pA69UYoO1FGe007nU1ibToQuod9+vhQelyc6NY1d+BZlo5dq3XKPLsSE4NPDGbrzDTyhywW2V22u5BwKYG+5/sy+LhlUU/TpOHASwfI353/L6+ha+27FZiMJsxms1XdD6lchkdZOkkHPkXVKk5Qnvr4FNvv386aSWvI2WwZXKsr1XwR9QXrZqzjgQsPMOxly+kkmUJGyZESDr92mH3P7eumkPufQPawpRSkzEPi6SGaqlIX1BJ9ejVRZ1YTuutbge72Gkj3rGLofhnZcdkcmxSEd1w0MqUMj2gPohb2FIOFYmCjxftQmVbJF5Ff3FD/qNUznCa/eIxqcVLIZALvglO4lwvHXv+edf13QNZe7DMFyrRXdK/wql4omrgZ4wAzvyz+mdTRlsGejbsdK+5fwanBp3jguwfoc6hHA3TW+n7M/nM2kXlCEfvkkJMcuaMEuY2cTXduYvfju7snW6xB29JJcHEw1T7VNIQJxbb5W+bzXMNzeMR4kJbxGwNPJ/Dkp08i2yrQOD9T+wzPNQpjHDYdCuza7ZAbHZDJJFSmVXL4jcM0FTbRUtjC458+zoAzg0Bmh63Cck8y2snpdPDCtTmYwItfcubIlxb2xWufYOafM7HttEXV3oZB+5+hYQH4euHX7B+7n0c/f5TKc5U4+TvRa0EvJnwyAbPUzOERh1HqlKjfV4u0av2S/Tg7OZWDYw7iN13cCW5UKVA7CveywQjV7T1TcnJHFQW9x5MVl8WIwyOITE+/tWLfAH+aCprI3ijOOBiNEH7+D2JOrsSmXayPFGc/jPiyT1FpvTFf7Qh1DnTuTiiuXbCWFfevoM6rDr2jZagQ6J3EpcRyaryFwqD0GpKQ2JmxuMyfgN4GkMC6uetoHm/plxUdLeL0J6fpaBD/djqbO/EuPE1V5HBCp4sL085ThpI97H4U2jZqq8RFs8j8BMwSMwdmjcRlcBwuIS7M+WMOcbPj0BWpuXP1ncgNcjbM3YAizInmomYAXMNccd/TxvQ1bXjXdE1Y9DwMsxmK+s6iImo8BochlER6Irtuasvey56Y2288WVUZPYqLE57D+brpfYA+P0yh8N5YjhZnIJEIkxdd1EBmhYQW5xac2nyxb+1EIoFDrx9ix8PChIJvvh3j9o9j2LFhDDna1+IdMpng53v24z57DwXZPgRk1aDTGIiZEcPMNTPRfZHKq2++inNz1zlZvoAXV11kzaQ1tJSJdX2vn3IAKNhXwMY7N1KXXYfRW8HJQSc5368G58BJouOHeD1L36oKtCG/EzlpHkFDg3gw40GipwvFpDODLpAfkc8jXzyC7YF6Ggsasfeyx6+fH+rGBqY5vExLvTNZZ+LwKBfewWt1Wwesn0mnUcFXzy1j5HYhkTlv0zymfjuVF9Uv8mz9s9i6ihOXshAHLiYKflpbtXivC8sJY8TREbipk9m+XWi0SLgjAbdIt+77EHUlikUrU6g5aekLNJ4p4rU3XiO8IJzc6FykLkrMRjNSuRSJVIJJYUJno2PZpVheef5ZfFIVqIsbiDn1M/32/sLy5ctZV6/kvtOBqA4OxdDc8xuquVTD1/Ff80ngJ6JzBhj+ynAGPzsYdZXaasNQUnoS1T7VZD8iXk8S3ROx1dhyKeESeX36CI2Q16CrkL5hwWEq51hqcMlVCrbctoWLSRfxqfJBo18LgK2rLSnLUnC8rGHoceH5dKo66bTrKUTGz4mn7GQZe5/eS/rKdNF52fnYsXPKTs6lnEPnKt4Tes3rxZNlT3L8n8c58b5Yj0oqL8NGa0OrQx3I5VRdqKL8dDmX1lyio6GDZz98ljkb5jDk+BB+2/uZxbEdPkZyEmU4m57D3eY4A6Oes7C7FZylz84Y3FubmOKZg8lRetN+n4OPA1O+ncKI5WJaPQCdrYmC8AK2zVbjPCQar3gvHP0dKT1WSnFtMWXKwWR49UyK5WzJ4dw355Cr5NyW9AJt3gKTjH2T8Oz0Hfpu+nnJkNuojZpDhEMFkQ5VNGnrqM2s5T2396h68hRx2XFMMjjy5uzTHHjke5ReN5/8z9mcw95n9gqTFVZQO1OIBTb9tElks0my4fiw47g3uCM/L56iGPDoAOZvmc/qcas59NohC1tXs9EzHzzDrI2ziD0gNABfy1YSknOFFucW3nvhPeKXiFmC/l1ETY3C0c+R8yvOW93zumKrLlmD6+Ho64hHtIfVxgSbMUM5P/kVShKmkJeyAIlSIUyhV13Vi48cQdbwB6kL7kerVwRms0Cn2VreiqZBw+mPT3dPpl0P02kTd6+8G6eXnazGUmM/HMujXz7K4B8H01FqeV0ylYxTg0/x4TMfY/AQpq4DBgUw/uPxuIW7cTD7a5SNW2i12UWjnTWfVHhiegCJsL9UbL9A2IWNJB07afGXO8NuwF5wAwx6clD39O71sHG0oSJ2DE71RTjXiIuvLUWNeJaexy/vKBw+gkKlYMizQ3omqhtr8Kjz4OW3X2baH5ZsTddS8E/YPYHBO4XmMIlUQq8FvfA+Wc2rb72KWWKmMqAeRz+hGHnvqXu5Y/cdf3lNJoMJ32Rflp5bSvj4cJG9rbqNgj0FfJ/8PVuXbhXZey/uzYn3TvDrxF9FNkOngbyNeSRmJDJp9yRq0mr+8lz+Cv9O3Pvvomsvtm8WfG6TVPZf0qC7FtnZwiTcc8/9yz+1ipKjJQQODqTfg9ZHvSMmRRAwMID+j4qbIiRIAAkBOfvZscxSgmXG5rtJm/wKl0YuQz5PLGskc3Hk3NTXKY8ZQ+SZNRjahAJR2Q1qybcSj+TvySf7z+zuHJVBY+DiLxe7mycDdr2JW84LmDv20VylYddju7j8+2XSvkvDe6wPAI5qR0wIfn3qV4JGr9xW3v3LaVHuQiNNRyKVcPeRu3kgXdC0DWoci0urMAGpNoHJZEan1nUXt3tl9qK40BfPgDo0XrE4OQjTg4VXSRYOHrz567wWEomEytRKfhr8k1Wqe8cLtTzy5RJUGhVGJ+G6oqZFdU/R75tcyDcPfsPC3xeiP1ZO5ORIXjO9xpSvplisFw98+wBx1/xs3aPdMbe2EXm5AbleTm50Lurof29q9UZ78d/4z+J/cu27Hn8X+/7G/xpMJhO5ubmYTCZkug4a/eLRz10oopSwkwtJwfm/z8d2abroc5xzbbhn5T0MODMAOsSTPq0OeehsdEzZ2h/VSesdalqlsCEcCrKccMrfnd/dAWyWyjDfgqegrlKz+OfFJJ9Lpl0nThiUXd7IHb/egX2L9a7I6NxYUlJTGHzuCQzNlmNlTZJmTvZfzYXe+zk08hD6AMukU0uQgWNDj6HSqHjg+wcIPW9JsbT44GIq7u3pfDTeYBLIGk59cooPvD64YYGmJHEKB8YKnOEGOjDqjVz85SJlJ8vwDvLn88c/xyQ1Me6PbJryLa+9XCMEm0kZiXgVnRU5ZuP2jmP8nvFkxmfyy12/4BzRoxMgkUkwG830yuzFwNMD6XVRPLX1Vyg/XU7FmQqrnPdN19QQrHVTuV8xMWnnJJb8tIR+a4Sg6B3Hd/iur0CX0OFg4HL4h0TmSGjKumxxrEuMD5eHP8S+Ye9B1TTa0j4TfT7AmRxnjitG4dnX+uThjXDXgbuY9sM0C8qlbvgJyWnPEuu6G24NbsRnxiMzybq1e7pgulpw8qn2Iem8PS3XTYvp+gygJkRwVDUOnihdLCkzz/9wnoMvCR7WjgetawbeCPU5wnvT9z5xkRwg47yRLXdvIe178SSaQ7g3l4c/hEzfSckxcQBrm9fO+H3jkZiMGJViyp0VS1ew6fZNbJ6xmUEbLDtKO5s7Of7OcdZMWvOX53/t2nez0Kq1hI4OZf6W+VaDfoWuHTBTH9Abg1JMT3p+xXmKjxQz7YdphI2zpHjpbOmkMa+RhisN/LnwTxEdRVNRE9NWTAPg5AcnacwTFw/+OyHTd2JWq0XTqXb+rpQkTKEiahR1vUZi52l53RmDSjk2TFjzuybDjHqjoFsjl5E5/AH2jttLTkwOLs2W1CG2zrb4JfuhtLdO/QgQkfo78Ye/Fv272Qw5Q+/DJJURkr5ZFDC1xZ4DOwABAABJREFUZ5fSZ+fbtHhF0jJASDrIFLLuyUXlsQMkn9BTElJCi5fgiLfXtWMymjBd82E5MTnkRRVjNkPutp7ERFcRBsCM8I7VZAj/Vn7qxvTGLdkNLPl5CTE5Md2d80p7JSo3FRKJhJaBUn649wfK/csxOgnnau9pj8pV+J3MWjeM5z54jiE7z9N4OIP0lekcWX6EzXdvBiloVBoKw4rIGv4g7oMtiyJnYo7yyaP3U+q7l7jsOJQ1lr8N7+ZeuDWFkz/gHB88v5xCb0sK3b9C6jepvCF5g7ds3rJq13RIMMqM6JQ6JFIJHjEezPptFhETI5BIJRwedRidUofuT50oSf71yZ+paq9EalYQ6CRemxvGRvDTkp8JOOTPnV9NpPVKTxOEzM6G2uA4KgIqiMiPwK3q5q8JhARPW3Ub+nbxlI7JBC2eQtLHpqNJZO8oqsG99AKFyXOwXzIXAN9k3+7iF4DEJMG+zZ4gfW+LY1V+nhwdU05xaDGz/phF24aeSHnLPVtoXruHMRuzcGx1JDsum86Qns3HZDKxZvQa9j69l7IT4gyDrsOAa3U2qtZqq4kZkwm8C08RfWY1G9aIg6dhh3szadckTHIlyGQUHSzi6D+OomnSQJuRwLIgHNVC97PRZOTOvXeyLGcZ3oneSBzs6XTwJDo3hJffehn9+VqL7231DEemmoyN3cvURT+MzN7SV32m5hnmbZwnPmnAvfQCYWkbkOmtB9Z3rvqMlU2zeerj59Br9Fxed5lvk76l8lwlCUP6cmJiFPFZ/Yk9/gMSCRx98yjnvjlH7eVa8lKa+X7p9wSXBJOcliC6b71qj7E2czNbjvZizIGBGDQG9Bp9tz6rzCTDvcGd5HPJjNsz0OL4/N355O/OJ+27NNE5m80QffJn+m1/g7pjQnI5/ad0Lq25RMXZCozBNuydsJdGT43VpJHZYEBq0IHZjNksJN7Tvktj6z1CVqN8UDorpp/HQ2nEKDXRWtZKe207CnsF6RnbOfr1EK5ciGLD53PxOCH4s85BzjxT8wxld6Qx/b5t2Mr0VISPxL6fUPioOFtB4YFClA5K7NztRA0kADjI2DRzE8uXLydiitiPdL07hLXz9xJQFoy8sw2Vu4pLay6RuyUXk8FEn/N9sOuw48yAXOwC3THqjGgaNRi0BmydnakIkpDeO51189dhE+XMtBXTuH3V7ZgMJhzqlXhXe5NX9RDUeeHQ4YLSw4mS+Em0eAjJ3+icGDJSUtl2Zx2Owb7d52XvJRTqb0T7BTDu/XE8WfqkiA4QYMqOKQSUB1DfTxyTlO1o4oX3XiA+Mx69vFlkH7Q2kCc+eYJ6p/3s1r1qYZMp5Vzoe4HS4FKmb5nO7RuFwq+ts233hGFxSDEAzs3OONYKe+7X8V+zYc4GwicI61jkZPGz6NqnHFsdUTaJ92qJVILKTcWYd8cQM11ciB++IZYFvy/guwe/wHFIYrf2d/y8eKQ2Us6mnEXtqGbc/nFIrlj+sE6q13AidiBX/JYL53Ld7642ZiCbZgoFnN6Fq+jjNPam/T6TwUTiHYkMebZHR/4NyRvdtNlOU99nw9ztpPX6iAabq1q/H4xj+CvD0bZqmbthLvf/sIQ+u96hqaiJ/F35nF9xHpNemPyVIGXi7olMWumI2WQmd0suJz84KTSTmcFG1sHhCl++2ToA3ZU2FCpF9z6vdlCTMCCLcXYQJO+hSbsZXNlxhVMfnbLaRKXWqlkRs4Jf7voF00TxZ3b5f+P3jsflSxerny+RSGgubhY1pDXmNXL3yrtxaBfyCYalQuOwV7ygBykN9CNjUD+aXIW9UmI1YPr34BHjSc6Au6gcfQcqd/F03oEXD1g5SoBRb+Sz0M/YfPdmjHpxnN61rmqcfWnxjkIiF/zcRn9BYkOlrkWqt2ygVtgrLCaKZqyaYfW7TZkmQkpCkF+QU3tFHO83BzRTGlhK1GdRogmlrt/ltQ3Ypz48xcaFwp7jmCll7oa5PPfBc0xeI37WU7aO4tnD49l9ZjyvfPYE+g49jnGBmJHQ4OvN/N/nE5Ubxfg945m3Wkyb/Veoy64j9etUq5O+ZjPI9Bpcq7OxtdIkFTIxlgvjnyM/eS6mUUK+qK26rXvCyP7w58xbN49LCZdoD76OcrWig0e+eITxqQMYUBGEb6lLt23t9LU4NSjJSMjAJDOxYd4BPG8XGiBUbqobarV3obGgkY/9PubEe+KGBhD01Lq0XC/8YH3CzaePT/eU1bVw8HHg4baH+f7+7/lpyU9EWmHEuRn8O3HvfwVdX1PaazJ5/ebT7HNrReG/QlmR4YYMOjeDJ8ue5O4jd9/QvnD7Qu49dS/ukeIm6ebYgWSOfBib9kYK9hR0/7vJYGL9mO/wzzmI1sGD4ibLprLGgkZMV5skA3IO4FyXj7Htrws9N1vsa6tuwy/Zj0X7F+EWKeTl7DztmPvnXNyu5uk0qhY8qw2kHDxJ6p8FuIYJujhFB4rwmxnER099hEalYeBvvTCbzSzat4jZ62cTMCAA21QzC35bwBOfLiH2/BYkEgnBw4PxSRKKhBWjF7N7eh2+R31Y+OlU2i83o3RUMnfjXIJn9CY9uYPXnUtZXu3KjrN/YiMX+z//DsxmMxKZhN739MYjWvzbkfp5k5nQgklqwqQWXkiXYBeaCpow6owY5CaaXJvYePtGJP29kUgk3T6ptFnK/LX3kZCRgE6pwyTv2ZPco4T3ovfF3ti327Nr8i6qptxaSeel9pd4rvG57ufwN/778D+99l2Pv4t9f+P/BKJPrSLk4lZM4ZGiRLODzp5Juxdjo7XB0FfMLawM82LXxF30utyLhb8tFNkdWjrxqvFi621b0faznFhRV7SStO9DRh4ayewNsxl5wHJSx6+fH71fmMClkY+QPWwpjovFnTI3glFrJLQ4lGnbpxFTIA4yvdtCCC8IZ84fS3ArFwc+v995hJ2TdzL4QBW6vBILW4OmngLf92nwaODIyCOUe1smzZrDDBwYe4AK/wou9DegjBRTLIXLR+BeLyR42iqbb/q67Nzt6Kjr6NZFux41LltxbnZm5KGRtGSXoFPr6P9YfxIWJKCSC8l4jUpDjVchUrmlA1u75hivvvkqB8YcIGvonaLgOT57MJF5kbS4tFAUVoTMoacA5RbuxqJ9i8iOyWb0odGMOnZjjTJriJoidNnJVVa07SRwfuKLXBzzJGr3EJHdqdTMgLMD0Cv0dF5NguvadN10iCdlS1HqlLz33HvsmbTP4li5SkGnkxelvluIyYlBWVpsYb+89jJrJq3h+3cbOXsWTlj35W+I0NGh9L23L1KZeLk3BwbR7ux3Q6q3+esfw79S6NJyC7cUYO+s15B4MZEa7xq+eOQL3HtbJrolUgnNPlffewkims9Drwqdtx4xHvR/zDqlz43Q++7ePFv/7A2TWt/cdRKDQoWu30CRTSqXIjNq8Sk8RcWZCpH9QOJmVty3ArVdJoo2cbDX5NZEq3MrTW5NuMS4WNicApzw6uV1S5qKN4vm4mZ+n/o7l367ZNXuUplFQM4BGgIS0FkRlr9z753cf+5++t7bF694S4oajxgPnqkV6Es71Z0iGrAT759g29JtDHt5GGPfG4t7tHXqyv80zGbwLDpLr4Ofw4cfi6jHlO6O1AX3oy60P3VxI7D3tEyk5CYUk94nHZ8qH+LTzXRUNFGfU8/H/h+zpte7BGds5+SQkww/OpwHvl1sEdi4RbihrlJz8RfrtLnBGdtwbCylPjBJZDOZoDBgL1pZPrZtdZhMAm3euW+ForrMzpYmL1dqAwPRuwnByj8d/snPI34GwFB+Ds/KJjCDyWyks6WTD70+5Lcpv2FsMzBn61ckZN3GgbEHODJSaIpQuaqo8bPh58U/U+stJGZis2JJ2ueNXqOn1wIh+ZPySMoN73doTAwXBjgwZecUhv4gTNekfZ/GG5I3uLzuMkY7KA8s54elP6Ae6gLApkWbuvcCnessSoM7MUpa6DC00HdpXzxiPei9pDe+kb7smyUjo08R2f4vkNF8XePN1Ztf5VvF8uXLqR1nuV5Vjl9M6rxcmsLhCRcsip7/Ci5X6bmNOqNVOmPnfA/Ujmq+e/A7fPv6YjKa2Lp0K2kr0pBL5TyS8gj+r/jzWMVjyJSWe1bh1kbuXH0nT334HHvmiLlg9sueBMCj3oOBZwbSfs1vy2TqKcb+svgXNi5ou6VO2srUSsLGhdHnXrGYsMkEhX1mcXnkMtpcxUXI+nMZhGZs5Yr7ndQYhCJjY14jlWnClHfv7CEkpyXz7IfPYrdVXCB20AiJE79KP+pKhUKQXqOnpURIetlojd36uubrKNfiHovD0d8RrwQxVZbcxYH0cU/T4exHY26dyN5ZUY9RbkNZ7Dj0VzWCcrfl0lggrNWHRh/lwJgD6PU70TV3YNKb0LXpMBvN9B0ylUvj36bJzcCU7VPoLGrqnpqQSCW0DXbjxIRAVKGXqQ4rwmx/a7LmLWUtNOQ1WLXZt1TiVpWFW0WGVSq3JtnPjNs3jpk/Dqe5uBknf6H5oLGgkWDnUOw7w7kS3UjB1bV7yjdTcPBxwKA10OFkoNK/krXz1/LjvWstPtdoNCPXtjPY5RgfLjhOwQMbkNspMWp7NuSLg1Oo9KskvCCcvudjLd7BoS8OxTnIGUd/se9tMoFjo+CbdtYIz92nr7CeuYW7dRcB7lw5AptN60THazbvpe/udwg5/zXl23dg1BmpvlDd3UgWJDcQ7tPC6x9+AyPD6HNPH15se5HQUaGYfJV0zttKSGwxsx75g4b+wn2RyqTYe9lzMbKVS3FZHKntjbSfL7b9hSnRHwb8wOqxqzn69lG+6/OdxSRg93VdfV8dNLG42YoTODISiCl9lEnb/ZFXliJTyBj45EDCJ4Sj7zQwfet0/Cr9ODO0Eocwby79don33d/n/A/n8YwLpyZhOZWBPQn4w28c5svoL9E0ahi4NZLFq+7h7vjDxD3ZhnLUEOQuDtSF9qc8Vuj2jlA7U5eo4eO4ehxVPX6Zc6Azr5tfv2ES8fyP59n/wn6rNoC189dycPRBq2uk3t6ZtL5pRF2JIvjC96IJZ7W7lirfKnpf6E3wZbF/oNQLxdgTQ05wfFhq978XHypG4y8nL0qgbZy2bRqTfhT2KudgZ7L+yCL7z2xeN79u1d9rKW5h7rq5PP3x08R9HCeyt9e2U3WhiuSlyVYnXgqjijnfV2gKM5shZEQIL7W/RPL9ydg723NpuB27J+7m+6Xfo0uxLADZp+t54NsHiMx1JO7od2hPWDbNHUl8Br+Be7hQ2ZtZ6/PRXbK+NljD2a/O8rbqbcpOWh+5MJlAapRhq7HFoNPRXNxM7aVa7DztUDgo2DB7A6n9UikLLENuIydwiLAHlJ4opX5/Orf/qqTNoY2sFA0SiYSsDVkEDRPixIaT76IqfZM7pBqCtkxCn96Ga5hrt5ZmjXcNnzRDSU4Qo08NRmvlN3QjdGmCVZ0T0wmazCb0Sj1FYUVI/YT9P3NDJr9P+53GgkbUm9Xct+I+tGO1zP9GrI1+ee1l9j67l8eLHmfqN5bxn16jx6daWJsu9bqEc1JPItw5yBmMRoxyOVKTlMHFgzGV/eeScm1tUKX3oNIugjatmHJbKpdi62rLU5VPiQ82C77/xV8usudXcRG+viYNneY7mm2EgmFXzFwf2Jv6gCRcq7Nxrssn4dT3BF/citks+IpukQLl4+vm10laJPZhATpmdvDxkx/zzxf/Sfg88W/ntm9vI+CXAEJuD0HpYllo11RreOHdl3j9zZeJOiXQhEZMiiBqahSaJg2tcQY2zN5AelI6lYHNos/2L/MhpMoPG5WWTod2TCYzTr2CSJv6GtVBfkTmReLS7IJtpy327bcWb60eu5qdy3ZSl2XFv2jV4lZxmYK+c6gJE9ODShRyjEoVzb6xmIOCUVep+cj3Iz4N/hSA7L5NnBlwhk0zN5E30PK6Yp2TURj9uNfZxBvvrkD7nNDUom3V0pDbQIuHjo2zhGKoTl7PiaYNgND08L7H+395TbYutkgGSTh37Fy3ruf16CpG+PUTM1BsmLMBk8HExE8nimwg6Glq7DSUBpdSKrWeL/i/ik5HT1p8ojHJlbesQXcjnJryNrHHxNIPNwujzsjBVw9aZeXogtB4IT7hrv6k7KFLeSz/sZ6/N5lpzq/Hp/Ak3gUnqS1q72kGaNTwRcQXlD/9KX13/AOAqoihyFzF+niW53Bz13Pg5QP8OuFXTn98mu/6fHf1PCVc/v0ye5/ei9lsZsPSw/yy+Bf2jN+DXbgrnnGCT6B0VHLh4bP4V/gTXBJM6IUAMEPY2DDi5wgT1vJ6M+EF4RSGFlITKdyX6vRqyk6VXXOeElyaXeiX1g9taQdyGzmxt8fS943pnB5SwFZZOyf/7EfSjvdorbBOdX6rMOlN/Dn/T4ydxu7ruRaNoR3k9XLmnpX3ELVdyDFWnK0g648sTEYTgRX9cG9wJyMpA1OQPVl/ZPGG5A2OvXMMF5MbMTkBeNd4s/KelVyZ1LPGbr9fGGzZN3YfGpUwnTnQ9q5bOneFndDAYy03+Df+/8LfT/hv/J9Ao388DQEJGMQsnWhbtQw4HUq1TzXq+31EdrmvM2cGnmH3xN0cHCue5Jm5cSYLfl9AZq9MTkT8ZmGTyqV02rvTbt9OYFkg3jWWi7VnnCdRdw1kx6A+bO8nIb1zy01fk0uIC4/mC6LydafETmVtfzPvvvAu7g02ONUXiewGhYGS4BIup0SjCLZ0zjpK2njmg2cYclzo+ry+W8BVKgR2bY5tZPWWoAyzTPZlrs/EMVPH6DPfMW37NDryxZ3/N0LSXUn49PaxKkYLEHrpIhP2TGDkkZG051SgrlRz9vOzVF2oQtuqpW9aX2q8a1h5z0rs/C2LEgYfKXmReTS4N9Bp7yAq9qWNmcaRKT2FU73R8oUJGxtGeEE4rY6tlAyZcNPXBBA6JpTnm5+3GvhIJGCSK1Ho2q0+q7LhZj58+kN+W/gbF2cIQeTY98eSskxIrA/eLuhraOw0GBSWF2XQGpFr25Hr5Xz81MfsXGRZXGgtb6X0eCmq1mqiTv1C3UHrxZ4bYdsD2zj50UmrNrMZrgy4k6zhD1q1F0b3BKbXJx06itTM3DSTgPIAGjwaQGkZ7EnULZiN1dR5SQi+vIuqM5YBwsw1M5nwyQTGvDuGlIdvXICwBrmtnHPfnCNni1jrAcChsQzMJpptxOuFqVOHxGym1T0UuZWpLYmTAxUBFcxdP5+wM2tFdsdWR+za7VBqlTyy7hELm0wh46FLD/Fcw7/J7fEXcPJ3ovc9vVFXqEXadAAd7oGUx4xFa+dm5WghEegcJKaPA8Ept/e0Z9Mrm2grb+PwJ4ct7HGz4ggdE0ro6FCGPDcEO3fx5OC/i7KTZfwx7w/Kz4gLCmYzBGbtQaHXwID+Is0+azRylhCiIr9KPxLPGWgoLsHO3a67uOzQXMHy5ctpdmnm9MBzovWm79K+3UWy62GSymlz8aci1jodUJ7XUlYu/oBjk4Mxm+H3235nx0M7qMuuo8SjkC8fXMb6CUO6KVn17XpKjgqJ89/u2canT3zEK2+9wsA1/pgMJhT2Ctyj3TG0aok/X0fKxblXL1E46aChQeyaXkVxaHH3OUTkRxB3yhuj1sjAJwYy8MmBf0kD7ODrQk1YX/KiGpF4uwBw/J3jgBAYSDQmnFqckBlkdJqEafOMXzM489kZAGoihvHLXV/y6RNvYzPSD58kH5ZlLaPPkj64qdxIKv+a4PIJeBYfoSDHkrrJtkjPyEMjcWoVgs/rg9w6RRpat63UV7rjdD4B8y1Qj0ROjuR18+u8bn7d6iTPsN39mP3HbMKqhYJ3fXY9F364wOHXDyMzykh+P5k5BXNw9XMVH99sIrAsEButjJIS8UvokaVn2KlBpN6WRa/3PsIxpWfKvrWkib77fmbQyUEYFAYqvDbcUjKi6EARF3603qVtMoFRqaLTwUOYcrsOF2LOsWbhGlTqUjovCHuKX4pfd7JnyJEBjDo0isIoJ+Thlkn2svwLDD5oIDw/nC8e+4Kz9woFwi66HoB/vPIPmtyaeOmf79JnjaVeW+SSSB4veRzXUHFTgtEItu2NhF7cQtXxApG9/WQGPoWnaPGOov9gOYZOA2tvW8sXEV8AcCUyA41Kw+SNcprP5OKd5I2mUUPJsZLu37dHvTsp51LQ1naQ9WcWX8V+RcnREk5ofiI/biy7Rxfxw+u/YQrrKXCZzZC47yMCL3xBjXksrrlPoLmGks1sNvNp0Kd8GWVJP9uF7H7RpPYvJPjyLtRVapF9we8LSLyUyN5xe7HzsMc5WFir26rbaC5sxC/vFJdifiGtv7AP93uwH09XPY1fsh8D216h75WfaLdvR+1s+dlmo5ne+z4k7nwG7j6NXPCtA6m0m3LI/FgCm8ZP4Vn/Tj5+fAtfPr7K4h30TvDmiZInSHlIvEebzZA26WXSJr+C7wyhsSZ5aTKP5j2Kb7IvxkutLFyzELsOGc2O4kLHFf8M0pPS8aiqp/LySfL35FN2sozhrwoFgYhV0/nHN1N49sT39LadQVtNG++7vc/+F/ZjdpPzbux5Dqok+PWuJyVISHToNXq+iPyCQd+ksFYNI7zSeWlsCic7frD47kOvHKI6vdqqdp0+o5Hly5czbfNg1FZoPHcVfs+52Gc5NVyDIsiXxvxG6jLrkNvKkSrk/HrHr5wadAqTRJho7GIiaClt6b63gy78gzvXfYhDpT2GDgPeid7IVXLcUuZQH3obC/03sjDhDTqMrd3vrcbZhzUP6Amcegxthw3tTbaYrEz73Ag5G3M48d4J0n9Jt6qTXBheSEpqCkMeGiKaYG4P6GTbbdswSU1EXgkUUYlnjK1m3fx1jDw8kviTls2FHQ0anvrkJUYdHMXlhMtkJAqai40FjUgVUpr69HTZZyRmcHmI0IR1x847SLorCYlUgkFrsEo92tncSVx2HFqllpohYnq5/N35rBy60ioNIcD5gRmcGHKCwScGo8mvQNemI+PXDCrTKnG2dWZo5Qa81HdQ6V+J8fp6t1HQNm22O4leUkJ5p2Wxr1NZgQSwc2onKLwYmermpwm6Jmy2LOmJOV81vMrLnS8DIE8/x6TdY3nhvRdQ77xCS1kLx94+RmVqJVJbKZm9MtkxdQdrF27D0c8Rv2Q/+tzbB0dfR/L0aTS5VJDeO52LQ9uQSCWEjQ0jdHSo4Avm1+Ja0sE7NnVULvsJ+QhhLUpanIQi2omywDLULXZ8ciIS4+bxaOpvfi92DnLm0bxHGfveWJHNaDDy1EdPMXnH5O6Y9o+5f3Bl+xVaSlowtZtwanVCk6AhfpaYZjN/dz6nPz4tKkQDePTy4N0X3mX1nas5PvQ4epPw/qd+ncqvE37FVFkDGHFrdGP8z+NvGFv8O0j78hSJ+z6yKgEBMHvdbJ5vfB5HX3FDhVQh40r/O6gJSeGHNeKiVvHZXxh8oBq/rO/ps/sd2iuENcalOhePciGW9M07ChIpSCTd68+Ur6ew+NBiys+Uk/WnWBcMQOYgo9W5FZ2NDgPimGNm7EzePPomg38aTH2H5bX5OPlgthXed6eGYkBgp8jZnENreSs6HxOZvTLZfPtmNs34Q/TZ3y37lR33/szm4Uf48N4VSGzk3etgi6cb/3jtH5wdcJat07ey4iHx8X+FroYta76wrl2Pd/FZHK+e8/XobNagaq1Bqhe0EbsS5V100Nm9aziXIqwD1+tKOgV5su6uizwam0pSCexVuwBC4cfO046qiHakgClcws6LfSk5JBRwW8ta/6UqSY2ihrWBa9Ec1VB2Vtwg0FTURNGhIu7ccyez180W2bWtWrL/zGb3E7tFa7tBa6D5dDO99b2RGqW0dv5nCiX/3eh611WtNQRlbMe+qfw/VuyTyKUYbKzJ89wcio8Uc+LdEzeU/Njx8A7elL7J9gfFbGW2taV4Fp0VWBGugUwpw29wCACB2fuIOr3KIqcaNDQIVe9opFffy4qYMSJ93X8Xmnqh4OQU5NQ9KWbUGYmcEsmMX2Z0/129Zz2nBp9C4evMoKcH8WLbi9x3WtBPHb93PEeHH2XV25tAIrDRHPvnMRrzG/Gd04+3Xn2LVYtXUZhswqAx8F2f7/hp8E8A1LW8gGPtRS5NymPpV+/hMFrI5VakVrB39le4VxZiIwFHnwZa/RUYZTfvN5nNZipSK6xqbUukEoa/OvyG9P1FpiPkBL3LpaQ6IkcKE7EOvg6MfHMkCpWCsScjuOcnYeLWaDZ1U2rm7cjDIdCVs1Nf4MCYq40c16zBXQxkF/pcQGejY+Ke+ch3iWlE/wqrx69m9bjVVn3Bv/H/F26tbfVv/I3/MGRXN5rqiGH45exB9dHbtDz6iIWmiY2HimNTeqOxjSbebqToM/xtY3DQxJAXlUOFVuyQnu1/FoVeKFpUyc5Y2FSeDuQOXkJqv3tI7Z+K0uDGquuOr6iATpsyktKTaLNPA6bf9PU5BznzeNHj2DhbC/JM6BV61i6uIrr6eZE1vGQUdU67yU6oY7THdUkxGTS4N+DW6MaSn5bgcptlEt/ngg2LUhexbdo28vz+wdmOOO6jpxNy37P70GKPIWo+Vwb0ZnKCuCjyV3jgwgM3tMVf8qfZpYGvHv6KT8ftxTvJm5m/zcQtwo2GqkZu23Ybx4Yeo9K/UpRcVw9UsDZkLXbtdig0LZhMlpM6zp3J5AS+TMrZFCbvnExFfDFBs4Vu3rbqNi6uukhudC5ZcVlEym6t4FJ7qZbazFpipseIpjfMZjMBWfvwKezSVHjdwm5n70mb8irVqkx4B6+l38mOzabapxrvam/c3CwLyvUXK+i9byVVsmTODDwjckYHPzOYAU8OZsHoGuxaazC2WFK6/iucv0pl2ffeviKKXDIzCbtwgdL4iYBYd6Mk0oZq1w3ctm06uuuSYjahDvw+/3eaXJtwbHWko70d6HleitzLxKTupyygjDoPe2Qqy4Rz6OhQ8nfns+fJPTxW+BjK0BvTJV6PjvoODr16iPDx4d2dX9cib+AiMJsRy6+DprSOmJMrASg+qGLgsmQLu6spCIVOQWGMPT4tYj79ZV8to8K/AolZgl+1HzzeYzMZTLTVtGHjaGOhBWINslt0slVuKqHI+fU5Rr89Grmt5fatdfXBpNESmLmH6oghcN3VqyvV1GXXsW3pNgY/O9gieatr11F+qhx1iZrDIw7j38syCA4bG8baGWs58f6J7u7u/xRqMmrIXJ+J/wB/AgZYnrPJBFcGLcagsCVhtBfXM2+0ZJbT69BmtHYuyE16GvOnd1OGAIzaEEE/+Rx2TNlBpV8ln0bdjaOfI5M+m4Tvbf3ZMlZIzNd415Dex1Kfo6moiY76DoKHWZ8eLU+YdMPA0WSCR758hAr/CtKHC0WGqd9OpTq9GudAZzIOC7SHT37yGErVSuAeZq6ZieHq/mXGjFlqJisuC7w8sXO346W2lwAoyjOQPeQ+WtzKGXVwFAqDI+ary9GClZ7YdC5n1aJVFIYXcmDMAcoGK3nVUYl/f39Ubirqc+spP11O+PhwUXLJbAYP9ShaokYxXGBtZcInE6i5VEPk5EhcfjLx1OanyIvIQz5QSO6kPJJiUUSWmIXkh9FkwqgzYtQZkavkdDZ14nElnWh7Z5JTJ9LsZxk42RYZGHlkJCXBJbg1uiEPtLRXtzyDNCuSpS1u9N4+CXOAWPvm38XloWXYlLbSN3UWzSXNtFULa+zgZwZjNpmpy6zDzt2Ouqw6PGI8LPSqOocq+Mj3HeQGZ0ZfKuJ6nfs+F/oQWB6I4q5TzArQs/UaKhYzEgwKJQa5AY86D5zbnG8pGTH+o/EEDA6gLrMO70RLqm6DwUyfXe+gdg+hIHku17v7ek/IM+fx2GePYWcrNLAceu0Q4ePCiZ8bz8nxV2iSXEZiN4PIKMvpptaGKiKvuHMlwo0CCjBe1diwJvSucfbHzsdygu+v1j5dhwG5to3SuAncNjpaZG+IN7NbuoeSsK0sVaUhVUiZs2FOd3J8WOF+LrvP4dDIk9wf9gAKlQLfPr44+jpS09SOc005SodfOT/enmeH2vHnuHcw6U10NHTgnAPzykei9TmFjZ2WKFXPZLjZDK1unuSG/YJbg4LwPFeMnT17ojVN3GuRHnI3Ss98ioM8eMHnBZE9NSUVs8RMav9UatR2BMfb8ETpE9g623J2SxbJZxwp9/ajzUe419Xp1Rh1Rvz7++NzrJjg86WoF41AZ2vZva43GVAMvUhYbDEmk4QCjRyj0Yxvsi9j3xvL7r2XifKKInTYFfwdtUgcOyyOr8uqo7GgkbAxYSLdXrMZzDLhvery5Yw6I+117ajcVZiadIQWhbJhzgbCgl4SXXNhYi3H4jazfep2Rjo9gEuwC8kPJndTFKlanCjQSTGovQm2L6VzkC22rrY4BThhNjfzuSfMcxGmAB1k1YA3dZl1NOY34o4Dbu8uh8+Xc+yL25C4XIIn4aHLDyG3kVNxtoLazFocfMUJO7OD8H5G5wZSerwE1/mWE2Ne6xTcd/w+fl+iZoyXK9rWcspOlRF1WxRSuZSC8GKCSv25bccYqoKvMGhRIkoHJXGz4zh+shzvnMNIzDYEFLQhrTZTerwUhb0CG0cbJIkJNGj1FLQ4sTU1iEL7jSQ4TiD+yCY67dxwLrDjqI+SN/anYLM9lPigSkBo5qvPreermK9wCXXh8cLHr78sgt8Npqy9jC13byF8fDgKleXznBc/j6ojVWgjtKJjMxuFpoKNszZybKyG5df5yGPN72OsTGLDnO9wl4t/twaFEr1C+L1IEIoOJoOJxrxG/PIgThVHVnwWGUkZtJp6fOQZv8xg/wv7edv2bcZ/PJ5BT1pO23gkerB8+XIARoWI6fz8+vkx6q1RbL13KwMeH8DwV4Zb2M2YcW51Yvy+8Whci6nLFqbV4+fGM/W7qfge20p0uAPVQxRgsPyNq/vK+CFEoN1P75POROWbFvaJm5PwLxnG0jfew3NGECN9q4mS3VzBL2JCBHGz46hI7WGfkMqk3cUFadpOYnURpPVNY2TAMPyS/XD0c2T/C/vpVdsL53ZnWlxaMGMUij8hLvR/pD8esR6UJFXxu49Q/HZqEX64s36fJUxAm82sv/cgjdJMTnsZcPUt5U93Ga0VreTvycfYoGVU7ig8ffU09L/Cn6G5TPC5eRaVjvoOJFJBk/R6mMwmjDIj/VP7Y/jMAMNg6vdTKT1aSsCgAOyd7fnY8WOmB1iPwcd/NJ5Rb44iZ3MOTgFOhIwI6bZJJBKQCHrvg08NpulIE0yzPL7PsVRaHF7A+105UVOjbvqaADbdtQmDxsCcDXNENhsXW5TaNnod/orm/GW4u4unhdWVaiRSCQ4+lutRRwe0ekXQ6hUhOgagILaEHKfjqDQq/OungVyG2Qzu5T1No5XRo7DtHUN1tbBmNxc3s/ORnejb9d1F8NdMr4kamaTNUvzL/ZEb5DRXNeMZYRm7nv7kNMu+XIZKo6J6RDXeI3t8EAcfB7Yty8T7WB2OeoG2t9e8Xvj09sEl2AX52Z7gX2UW3w8zZnZ1wM6rbp7RZKL+aC79tq+nzteXY3GWf3sreLzocYF+z0rjl8rdjksjH0Glrr2qdWzZPFm67wrxRzcDYLoShv1bi3jd3JMPMEqEZP3UbVOReoinpvwaF3DObR3K4gA8dcI+Z+dhx7O1z3LbpIe5bds0zE9s58S2oXj2Fopqg54ZJPLvrKE0qJRVi1Yxv5946jV/Vz47l+1k9D9HW8RIXbhzz528IXmDM5+dYey7Yy3iy466Dn4Z9QujvEcxo2YGFfYVIE5Z3RRuNe79r6DLjwlJ34x9azV6WwfMZmtZgVtH/22v8tVX//7xG+/YiKO/I6PfHm3Vfu4boWDslyyewnQoz8E1+xR6lRPFR2wt1rm4u/pxodIbqUGH1t4VnQ4UCiF/sOTYEn76CVJJQdnZiqJTjdlgB/Q8E4fGUhzri6gLTsZgI264P/bOMU5/cpp7T95r8R51MWKlfZvGgxlC83hHfQdblmyh/6P9SVyUSNKpcIq91ORF5aHpNLLxjo1krstk4JOCn+3WJMjG2CSGI5FI+DbpWwC8E70Z7D6NkJrHKPb+HDMmJDIJNk423f5b/CkDEcqByKen4+eiQWGjpLW8lR/6C/vcnLzpxPq1MXfWef456jw6h7tv8klB2YkyVg5byZh3xzD0+aEWNqlcStjYMA4vP4zKXUXoKMv8iGNWJzPOzCC9fyOxMwU2lMx1mdh52DHi1RGoI/tQ6reDpz56CvPoQmJ/modBY8C/vz96iQQpNgQ03It3cToeCnFRLrgkmOy4bKKzA9CUZgPWNX6toXCfUBxszG/EO+Ffry9/47+G/8m173r8Xez7G/9rkMlkJCQIOhpFXp9TKVnFwI5Foomx8s4yDqTMoF/qIDJfd4KFky3sPpfduPfHOWyZtpoaP3HnRWmEA42Ox3n4q4dpd5fB2z22f5VY2/1xFofeOErAnADGHBgDLjd/fR0tHQy4fwC17rXseW0PveltYW9uyCGiIoKSgFVENHwrOn7UHl/0sslsWnAKs/lRC5uNrx0r71lJSFEIs/+YTZDZ8rNlLQYCyoXx79l/zEY+rAp6pv2ZsWoGK/6ooKBtHSpdEEpXMUXHjVB8uJjiI8WkPJTSrRFyLVYvOUazQxoaOw0qV3tkShkbF24kfl48AS/1ZfWdq3FqdWLE4RF0jFZDdE/CWS4Rkg8L1i7Ft/pXTCbLgl2h0+3I2zNocrUnOzabaa49BarW8lb2P7+fBBI4NOoQUt2XwD9u+rrSf07n7BdnuefEPQQOtpyENJvM3YW+Rj9xcWmcfAl52Wko1elEaISE595n9+Ic6MyAxwaQ1i8NuV7OK2+/gqafq8U7aOPuQEVYGDXeNfhV+OGO+J7q9aBx8iZ9wnNMEjPV/iWGvTyM7I3ZVukgULfiXFeAfXMlIA64qlz/YGh6IFKThAqTH9eGvzInBbkxufS+0JsZW2ZQ5p/FgIE9BSRDYBj1Pq4ElsOmGZt4Kvojy2vS6Gm+qvO3c9lO7tj51wLk16KLyjLhzoQb/k3vPe/RXBMBD1h2Mdp6OVERNRKZQcu8peKkVNAhBS8fepmtczqRuorpRVNTUmlxbkFukGMMtVxvGgsa+SrmK+S+HjyRtwx78aMELNe+W8GAxwYQNzsOW2dxYVarBbtONS61uTQEJIrsK4evpLW8Fa94L7EmX0ETq8et5i7uIrVfKrbJ4s/v91A/Tn14ijckb7Dk2BKChoqpgf8deMYLCYRrCyhdMJmgze2vvkeCSSpDpu9EqWnuLpZ1Qa6XojAr6LDvoMO+A8k1yWqnUHfOT3qJQru+ZMdmA7YW+0HXRLJLiAshI0NE36xQgPvlI9h0NHN9A4jZDOf6ncOl2QWviiZMJqGrsuueRbf2ZdzecWhUGsy+wmcnLOx5H9yrHJHJ/Nk4ayMx1322SQ65EQfosM9kzJkY5AYhuV99sRqbTuF9VOqEwnmHfQettp1IZVK+7vU1dZk9E+Z3HbxLVOzLPXaR8JPruZCiwWz+GICYGTHEzBC6FjVBcHrAaQaeGUinnZB0nPxFz37sWvsyt2+cQGFYIRqXRo68Wcaxt4/R/9H+9L6vNwFZ+6jvo+PHe35kcqLlVGxzipw/Aj5GoVdw/4r7qZ5gGWWOOJCERhWF+30bKHVvxc/vXdEzuRGKjxRz6NVDxMyIYcDjA0S0JaXRtXioNQzdXkPDlQZCRobwXONzKFQKJEoJHRs6OPDqAbLjs3mh9QULvasu7VKDvIV22xzAMhm9c/JOlDolZ/NtaGt24GmPnikIG28XLo28k+yYr5i9dipetT6YrQ+GWUVHfQd7ntjDqLdGiZJBJiPIjHpcavNwqi8ELJOXXTSom27fxG22QsJ60ueT0KqFRL/ZJ5piu+3Ap2RoY7i8bgh/zv+TWb/PQhrnyhuvCdpRYQVhuFwN9P1T/JEpZRh1Rvpc6EN673ROjPNgxNyeJLxMJuPI7CNsvLKRJceXEDTE8vfd2dhORNoGqkMHiib/AU65b8ehTcHC1f3oHNiOVGZP3OyezF/y7lyiFbNYdfcWHgvxQddWR/z8eMLHh3P0hxNEpp7ixIhmbJSfIFXAtBXTqDpfRcz0GFw+lOJ5ciR3O0YSkDWahCVNgNAI1V7Xwc93vECbQxWYYc/EPUz26knySaQSFu1bZNUnArBTt6LQqsiOzbaql3Z2wNnu/6/XC9SkZz49w6QvJ1EUUMK3D3xLSmoKPheAV+DnkT+jbdHydNXTGL39qA9I4vYt3ig1LZjfu+Y5Ax1zNtHHA579agrPn0xBc3cHAbGefNPrG+TAQhbyp8PPHPFpJPnClxiNZrqmok99cooLP1xg3qZ53etAF8xmcKorQKWuRVMRDbhx6bdL7HlyD/O3zCdx7ELuV42myfEKQRqhC/3M52coPlTMrN9ndSdpDQoDRpkJ70Rves3rhbpSmE5cufRX5hulRL4TSqe3ivxd9rTXtBMwMADJtmzMPz5G5rz97PjlNtqD8+G1JBx8HRjx+giOfZtOmI+QMLtY6INtsHA9XRTW1hKe3c8yxI7PH/2c0KJQpoTeLrJneOdgm6KhwG8vMtmH+Pf3J2RECPp2vZAck5hRaVR41EBTQw1e8VHd31u99jCB+QXsnHSAhtAdTB2kIiZpavf+ZzaD0azgtktDmbuyP47DWzFFm8FsxqUmF9caOFTpwVq3CiIHnSTYvqe40NkkrC3X6m5ei2fPPktBXAH9E/qjclVh1BlRV6lRuaqwcbKh1929SEpJYsAW8Rqpy25j1h+zONv/LJ1WCAJczzeTXOrH7zMrMF7nS9q4qDg1sR8nYpcx649Z2HU4YX5DoJWb/vN0tty9BYe2niKHs0lImJ344ARO/k7derPWGqiuTfTPiJkhsnvGeeIc7Ezad2ndXfPXYuKR6UirW1lx3wre6PUUrWXCdLJjgCMGrQHH8mzi2x0ZeuRlWu8xwTX+9/VFhuuppRs92zDQxFENoDhAiCaLRxJunmrr+sLR9ge301reysLtC9k/K49S1R7qvOoYHfUWCjsFfil+5G7JJf3tdJ7kSSr8KigJrkFd086+Z/eR9UcWr5tf744FhhwfQlyeA8Z3jex5ag8ZqzN4qeMljHIjRpmBz5vAYJKS5GairaqNE+/2aAgsiK5mekQZ8nxAefP6dvtf2E/6ynQeK3xMNNktkUn49sFvmb5lOonhgg+bvDSZ5KVCQ16EWwSedp6olqt46/W3eCX3FYvj7dztwB02L95M9LRoy2Jfm4R7Wu8h6JSw34QPFGLelIdTqEytJP3ndFrdQ8kP/Imf3DqYGzL3pq8JoL2m3eqUMEDcwj58+70Mt6pMNAYxjeeVHVf4fervBA4O5J4T91jYOtqMeBafp9PBA7VHTyI5b2cepz89TfOgDq54C4V4N5sTKN2uriEKYQowL2UBrZ7hqK4+IrNZ0P8uO1HWTSdtjaIZQLlZydJ1SwEoSioi8glLrbYT35zAs96TkqASJDbid0AiUXBy8EkiaqYAQjNR+alynih5Apcjcp7Z+AwF4QV0OotTkfNrvHnLX0VmoytbMwZinG1E1yTE1q71zcw4sIeD/ebg3uiGe2MIAA1XGig9UUqfJWJac/G53eCdlUiRmE1EpK2nMnI4YNlA4BrrQ0XUSFyqc7B1E9MVj9waQofydnpd7kV1hGVDWk7JedxLDjDU0Z8heyZhUvccv2XJFpJ3e2OSeOKYeRmbxz4j0U0oko7/YPy/vJ6Oig6mbp9KZnwmdt7iQnromFCGvjSUgy8d5OBLBy0KlNdD16azKPbZutgy6ctJvHn4TWxzbOkd1Ptfno81/Ltx77+LruW4PqgvlKXTEND7P/bZJpOwJyP59wjyxr43FpWbSiQ90YXHCh5DrpJbnfZtiepHjUsU/rmHWDu9hheaheYxvUZPe20bTT4xtF2Vm9FfsyTlbs2l7XgHHS59iNu+AgDdqIeAnmY8h4YS/K8cxnNIFBl14kYodYWajroO0fTn1O+mEjgkkNKjpd35CRsnG8a+Nxbfvr5ghiH74+mjCqXFuYUahyLayoRi9tkvzzLk3ESefOc+pGZnXu18FbPJTPy8eEqOlBA0LIjCnU34lypJOTIHuZMEuY2cF1p6mub2TTpDu7KMubvW4F4Xw2PvxCJVGHH0c+z2JzvthPVjdzvccwsNAiajwKxzI3mWFUdXID8nJ7Y6llAsi32KZgOBV6I4MWQlBwoPMCZsDHP/nEtbVRsmg4mOkDgylG34VmhxsrNF5aai/yNC3mnP6ctcdp+JWl7HHQcexXDNu+Ie7U5DbgPz1s/j/Wff57sHvmSZq3VppRvh7qN3U3K05IYxy7+L6upqnnxiBeeO/oZE0oqzkxPTFy7kvqVL8fG5tcGS/1/wP732XY+/aTz/xv8azGYzra2tmM1mYtKK8Kr1ZPdtFaKFx2QwYt9mT2x2NK75qaLPadM3YpQ0MW3bNO7+cYrIHln1Gp4t42l1akVjbzn2rm3W4HvlCKGFoYQVhBFQZtn1cznDhNSoR2KWsGHOBq5ME3e83ggt5S3MXj+bISeGcCFXcMYL9xey85GdNJc045Vtxx2/3UFsdiwKjZgWIS9eRY13DfN+CaXzouXUSVdQWRxazIfPfojjYMsClMPYMbz3wkc0ujXi1OqEpFP4+wMvHWDt9LWEjAghPyCHjJClnImagMaguenrKjpUxJHlR8hYI9YZBOiw70Rro2VIxiYClcGY9Cb6P9qf6OnRSG1kFEQU4Kh2ZNThUbTXWV636qKBibsmUuNvpC44WdRVNHpPADM2zyA/Mp+qXi8S8P/Ye8vAqs51bfeaHptxdyEuJEGDu1sptFgpLVB36qWl3tWuttS72lIKNaC4a3EJFoEQD3F3nz7Oj0ESJjN007332fv7zln3j7VK3jl8jPd95H7uJya4e8w10pXoe6MxSuGxbx5j/KY7l5YBuvsHFZ0oshgTkFCQOJd6nzgqQ0RmT/GpYrYt3IbJYKL90Bnu+TWQkSdG4n1KfEfOfXyOA0/3LL4mqYmjo4+ijTMPMql81fyweDFFQUXM2ziPIfvMg7J12XXk7MpBahD3+3fJIWPeHcPjmY9j7WRpqBgHJNHiEojftd6NhFFHQul/WXS0C8vNnVTjDWOv1q2W5EHJyD3NE0QmDy/yY0WHWqvS0qE3f8e+T/yezM2ifExjwZ3LyAL4DvZl1Nujbtuzz7q5CiRSZF6WGupWbmqaPCPwvH6OTZPXWYw3eraRHpdOg+O1Xvedd3celwZcIjkpmfz55hIcXffYUFnHDz/c/vxvnvvuFFVpVfw68Vca8hssKk8BnCoy8Lx+jqxhy2nysmxEnvBgAmPeHcNDlx8i4UFzZ1jtrSb+gXgAwnPCLeRnTrxzoptpCKLG/n8XXMJcmPqvqQSPC7YYM5nE/3ErukjHT5ssel2pI33IHPko2cOWkTX1eYtehH/OL+T3hb8jM4i9bUx6Iw35DWy6axMH5q7FvfA8pcFKhp0exuJfHjZL9vkM8CF8ZjjV6ZbSYAAOlVn45B7HsdpS7slkgqNjT6FuVTPglCj3U3a+jPNfnKezoRPbJiuGnh1KSmImjQljEASB1J9Su6U5Ru6PYs4WMUltEox0NnTyluQt3lW9i1av5ZrPMgqcP2HNsjWsWbYBQaA7kbfx3o1kR4rn5FE3hBnV32Iymrr7CQx7dRjzd8/vlSncYWjHIG0m9FobbcfEtbY+r57jbx6nMrWS9lAJByYfYNu8ajznLgHEPmXNJWK/LvvidqIzo5m+ZzqdaZXdFX/NJc20OLfyw7IfODpqA6X+pRjtzYMtBlsJte61NDo1snfKXppueYV3T9/N4YkHeT+ikbuHZiGo73wirEqrouRUCYdWHMKkt5T4MqAlJzyHjQvO4p7ojkwho7mkmdbKVpo0Tbx54k1O+pzEtMiETGF+XEmdkbCcMPpf7I9nkWUPUK2TiiqvKmrSw5i8+klaLvS8TyYTKIwOxBX/wMUBFzkzLP1vywyNfmc00fdYElD0JiP77tKSHW1A6MVJdT5k5LV3X0On1KH1E22ugoMFHH5B7Ck7pO1jfKvnMPePuTgkt5O1RewtkrU1CwEQpGL16YxdMwjdKzLYHYMcu6Vupu2dhnWnNRfCJnOpaX/3cQVBoK1arJy89V4CSG2sKY6Z0uscBtBirMEkNWGQGzDoxf1VX6nu7r+jsbOj0akRQWLEZBLtlT0P7aEup45ObwPbZ22nyC8Fq5YaDO1a4u+PZ/Lnk5FIJZSO1PPNo99Q3eKEXXYO2vIeabKvAj9h2u4brNle4oRdUni3Y+D3vxDFkvVLuhPxvSEuPY6HvnuI1us1dNR2UHK6hNbyVgRrKVVeVai0KhwbxYN79hWd5tqsWnJiSoh59CscB5dTE2hOUDEJJv7ZBLMqYFBYBfVx2UgUcjNyRY1bDTXW7djumMbgvRlmVYpdBIXeeocJAoSd/xW/zEPUn8gA6J4v9R16ZCix0nszKHkQrldEWy9nVw7ZO7Ix6ox4psiZtX0WQdeDaM8U3dGT75xk1zKxl9H02vf4PqyVxSvXcqV/LgEjAhj30Tgc/B0QpAKotJzUmshyrCFHLcoN2/vYM+rNUbg+msDCF3/jYE1fPnruUy4vEHuEbZq9ic8CPqO5pJmy82UWBBEAJ4U7DS4NXO5/GaWPZeD0YkgeWZFZPPbNY5iuZYEECg4VUH2lGk2zhtfefYWQghA+ePUDbIZ4U32lmi3ztnD9z+uo+rryyXOfkJqQTJnHKdolbTTkNXDq3VO0VbdRtPMFHLOWIXU7wpkFm2juq8amjzfXRj1OxhBRYrdPYRBvJRUSMPcQMtcef8l3sC8rdSt53fB6L28XZGRmoFPq2OO6B7mVnIpLFXwe+DkHnxN7jYZOCGXo2KGMCbKsNjBUa4nNiCWkIITQ/FhLSc2rVwnKrkJikiAxWU5iaUGLAJAZZcgNiu557twn5xDcHEhPKMXG5M4DGzYw47eR3e/CtoXb8Iz3ZOm5pcQttCQxGduMTG+fzhNbniD4J0sbQhAEfs75mRHJI3oNmkeWJBB8PZg6dwGZgxqfQT7MXDeT/o/0p9WmlbfefJPf539PanwqBi/zbRWVRhJSEnio05VN1xOxLzVP9lweXkD6/Rt5ssWLWcfGQKXmju2+/IP57Fy6s7sXKcDl7y6Tt1fsbdjg3kmtuzjnGQUjBq2Bqd9OZdGhRd2/d61zZci5RDrrO7sTOpWplaiKBYaeHkqfgng8y50QTAKlZ0pR+6iRyqS4VtkTXuPBBxovwl58A8OWetQh7lTd1MPsSvmXSExS1hrvJkB952oPQWPF33atMzfDJJjQWmn5494/kE0X14b22nYqUyrRtmqJbInkpfaX0Cv0NKotfYb2mnYarzdyz5Z7GPryULOxxoJG/D8V57IDEw/g80KPekXknEishvYjP34oZd4nuFhxEb3x70mbeQ/wvo2CjyhN3eAbR/6A+fy0zTJTfupdsX9x9DzLdby92UBAxj7Ck39G1dYjhXzmozNcP3wddcMtfteNz7IsciyNHhGEXtyAsqMJ29wUHCszEQRRmvmV1ldYeGAh9x2+j6cKnuo1+XXXvXeRHpfOqWGncEiwPO8Nizew+pnV/PTgTzhEmo9nF2XjdqqcV/7xCsMPdmAyidWlXeQ+g7WJepd6wnPCiU61/G6DNs7g8O8T0F0JwuO4D4JewG1Kf6qChyDXdxJ/6hyvVYTzj8yBTNs9ChCrK3c9uIv2WkuFgZtxcMVB3pK8RdZ2y35pRr0RmUFLedgomjwsqzsdwz2oDBtJ1oiH0Y+fgqZZw1uSt7p76tm2KbBvsWfbfTLk0x8327a6opikUy68Xh3KkrmnCB4vVm1pW7WkrUvDJBH46MWPABmNzo1o1OI7ePaTs2xbuI0Tb5/gq/Cv6KizVLXQteoIzwnHu9EbPwfLHs2u4a4MfkasoOotaZG5JZP+j/Xn7g13W7zHSjslrgtc2Ruzl61ztuI0xJKEdSf4z/i9/xV0HaY2cABZw5ejs3G0iCn9Z6Br03Fx6tv033vnZPJbkbgsEd9BvnTU965Q4hTs1GuiD8Bo70ybSyBVIUOY+GlPuxpdq45zrx/AvfACMr0Y/9LdcJk76jrYOHMjjT/twqE6r3sbiY35u9DsEUZB4hwkLmKc6tZH1fXsblWJSluXRs6OHILGBXXHBpV2SqqvVPPL+F8w6o1sW3KGEyNPoNKqMOqNRMwWyWMmvYkLE47SbttOeHYQ155bi8lgYs7GOayoXIGVgxXlf6Yyfq8d0ZnRuJZYzlWlATUUBxajLbbB8+BuSg/nYOdhx3Plz2G1PYE333yT+60qCN8Ths2Oibe9770hcGQgr7a92k06uRkGjYHKtZX8OfBP3Cdb9iBvHG7Nhy99yJCzQzj9lWijXv7uMhumb0DbquWI23TKPA9xcVwSodPmUZNRw5GXj1CVXkVHRRtL1i9kwKWhXO//MJIx4tpem1lLfY64FtS61iJIBDTWGq5K/l6yL2B4ACNeG4Gdx39ejvZWrP7kEwJ8fane/D5vVmbzbUUFz2Rnc+r99wnw9WX1p5/+tx3r/yb8T899t+LflX3/xv8aTCYT169fJzY2Fp9SW4zSEArjHDAYQH7Tm9lZ0c4LH7/A5cTLnJn2hMV+ZP08+crxK6bsnYJPuaUG+4CT7cTJxvPDAy/gqxtrNqZp7MQn9zhD5S/iWVxAk5N5Mk/VL4aM+hhK/cWyuBDFnX+oNl42HJxwkImHJtL4fSOMhMqUSi5+fZG+i/tSF9LJjpk7mLVzFg1eBwFzFmdGXBGS9kt41AYiuZVl26BhxIkRFAcUUxxYjEpiPlk7S4MwSXXUutfy6YpPmWcv6g3U59RTcUnsryMIJmKvxHLX9rsotb7K4H6WVUy9YeDjAzn59klSf0y1kNQBcGqwwbs1nPHb0in0dcXN4MeFLy/g3McZQRCNl8v9LpMdkc00vxfMtlXlGRh8fjAb7i8E17EWhsbJEScRJOIfpYLCbFxpq2TOxjmcOVKCY30r+aGWgde/Qp9JfRjz3phekw6CIKHRO4p2R2+kRtF62nTXJjrrOxn41EAU4cHklHVQ7etKXIS44PsM9KH8gngOY/4cg2OTI9vu3sYEJ/O+kS3apu7/Pj7qOGqFuVzL1d+vcvKdk1gPXYpC00J7kRsMskxi3Q4VlypQOahwCbVkIQLU+/ZFruvdOVK3WtFp1UlZn0h8mlqBnn14Xndn5bvvkZ0QSG1APIFRURbbtzq7sGbpGsJyw6jPr4bQHuPVN8kXmUrGvJ3zcAxwvOPrAfHeOoc637axsF/mQQTAboqlpIFUCpWuJ4kGvAZYSnpciUyhKPwMM3fMxK9uB4Iwi5v94Ef7P8qvV37lWu01i6SYrbst2UlLkOs6cC68/fnfPPfdaWm/RCb21VPYKHqVoJEaDch17UhMvWvRD391+G33beNqw8y1M3m77m1m7J6BYbMB+vaMGzoN2HnY8XDqw/+hPOl/Bm6RbmZyaka9sTsR0H+f6FAJSHrVzO9CrzaUIN6jmIwY7tpxF62rStB4B5GzOwfBKOBLKeM8D+KdewKHWvPErUwpo6mwyaJPYBdcCi9jlClIm2ApFywI4NI6ihMjTxBeNgCTCX4c/CMgyopZhfnw3tOfoVVJiKwBBNj1oBjgXiWsIm1wIe0Ucu+WVfj5DkLbIq5LRp2RttImXnv/NU4PPU15xAQUBhcx6D49jC0La7nu36PbP/RUPI7pa+l4awVz/xDXl6sbrrLprk3M3TKXiJnmlTrqeDe+fuJrnv7saUpObgIGsG3hNiouVqD2UeOlCMejcRYm61gk9mKC5zP/z7rP+/eH99OpLcG7wpvnBm0icVkisQtikSllFLYWUe5bjsQkQWqUItzqdWtFUs/zPp14uXaSYd2TkK5Kr6LFvoVAXzH5UqgHQXnna3Hi0kTU3mp0bTqkCsvnedfGGbR1RrPugXVoZFKULVq+i/+OgJEBTN45mahrUTQ5NqG/V28hn6u8omfBBrHco9PGsseco9YPk7QeX98aCgdX0DeoZ87RNHTgUZCGybuFs8OvYaP9e9Kkx988Tt7ePAtpOhD72V7o+wEX+sKC4hcsxnVOAoVBhWhVWoyC+F31XdKXPlPE9cdpyxrmdYRj16agzkbPxO0TiV0Yi/cAb77Zl4p/sT8Nzg0cnHiQYOtRAJSd6+nZd2TsETRWGkYeH0l1QDLcL7LUTSYTk45Puu3cJ7VS0ubkS8TZn8haO5rwD81tjJDjTvhc9GDtg2uZaP8N+nZdt9zPKmEVu2fswdh5hvt/mk393DSGLfHHZ6AP7dXt4KigfUA6sanxxJz8lpzkYcRHDqAsuQyfQT5o1QIe6hpejF7Nnogp2Ebf3X1cha2CiJwI4lPjKQ4oxqHZ4UYlRw8J5tgqsXLhvkP3WVxXWZAtgUWNPPPZM9QtqeuWHe3C7K2zibsaR6NjI1LBhN9IkchSdr4M41gBlUbFttnbCKj140lg5rqZ1FytwT3Gnawjr7Aj8E+qPS+xbPs7ZnOhTmNg4ea5zIwrZubUC+RYvYrCTkXOrhwUtgraxrnxTcKbbPSEWNL4XLPMjPMcOTsSGxebbmnNmyEIUBE6EpuWSiL6iYHRuMVxqBxV+Cb5cu1QCwGFTow/FEGbp/huu4S7UPhnIfpOPQ6lcvzS44lPj6c0QEPFpQpqM2tJWCp++/5ZQZQ6+3E+oJC21k7cot3I2JhByekSjKHWPP3gd1hLwHZZNl7ankqcrG1ZXDibRb5uPp9Eb+Bx4LxGnG+yt4tEiM8CPgNESTfHQEez6/Jq8GHyvhlcjU3B2Et0UIIUk9SEVqVFppTRWd/J3M1zcY91RyKT0uoaRp2r2GdFZ9ST8mMK1zZdQ2GrQLgrhFb7VlQaFdedFnO9ehf+JRqxJ4zWiLRDj61GytVwHYRncc91RXeAst3BmT1T9zAwooz77CFaCRm3LHy9JdC7MGvHLHzLfPng1Q8Aunv/1lwVpVCLVxRTcLSA9pfamfjeRPO1b7At76x8R6y6OmGDUWdEat0znjNLxYXWszz55ZMYVVJ4q2dTXZuOuBRXqjwN/HHvHwzLvIQgiMkZo86I3NWdCWmlhEXK0LVuRdIqJmYfPPMgm+duJvmzZMa+b+67dcFQZKDfP8UgXKVVpcX47//6nctvXeadKe9QvLbYYrxj+TBWtydg0+mDSW8UeyPfHw9ARavoK9W613J5mAPDBvU129Ym38TMXTOxs91N1obpuI40TxAJCAy3ggfaPNh1YgQtCbo7tvtqrtaQtjYNK0crJvxzAhKphFFvjUJpJ5IFJDpwa3Vj+KnhaMdVUOrlyM9jfmbM+2MYs2YMS06vpNL9InZtHrwYsBLPeJEc0Freik2+wPgj49lzVx3aQa8gUwk0Xm8kam4UMqWMidsHIUgSOLG4GlkfNX3jRlPbKKfDXtxHg1cUFbUj2PbbfEoOhmKcbbhV7fC2iJkXQ+XlSrHa4xYY9AaGnxxOjXsNfReI93rtkLU05Ddw35H7KD5RTNs7bZx87CQjRlqueQefPcjV36+yUrvSghDnFOxE+bPlbNdtp86tDplEHC9LLmPDtA0AFN31Gq61rty//n7S5GlMfO3O+77X59ZTedny/QPIP3Qdr7wyagIGkJ9vmWiZ8MkE9J16gsda+p5GiYIm91Aca/KQ30SYnP7DdCovV1L5WiOjalZwbPQxfLKO0FmfhNrDFoW2DacbRLTAK7uw6axHZu0C9PhpfSb2Lg3aBZfYQeydfgqdoo7PIj+zGG93aqfZJBK9bvWFmmuaGXViFACeFSZqa0Vpz4tfX6TicgXBY0axTnIaz9ok+tpZ+mk1sXczOOKfDIg5xRbhY6Qq5Y15UJzvdEop0ZFFDIkp5ydfcQ0e8NgA3KLdLGy0W5H8qdgvWu1tmUxprWwn8syPVAcNosPRMp50a59wTZOYVOmsF5/NtoUXqLM+R//87ahl5klwmZcVax9Yi8lLxYToPHYpRNumrVIkP2kHD2RsQQbzByzGZKzhu1ZR6vjw82JifPS7o8X3uheykUOEAxvnbWTexnmk/5JO3/v6WvzG1s2W1w2v9+o3H33tKG3VbSQ9m9TrGnJz8vs/G7D+z/i9/xV0nabUoEPV0YjOSg381/vNd/ljIPYDvPJbNeHTw/+Wb2zUGlntt5q4RXHc9Yu5eoAgCDQWNHL0taOETg2l72LzZylBAAGa3cNIeLDnZbBytMI5ygMys3CuyqImoD96vVgAcXNyK/SiGIO6NG0Vt4ZeXMrS8bx+DkPSU4DKwq9OeDAB/6H+Ftdadq6MwqOiXVd5uZLhr4jxBp+BPhi1RiRSCdW+TVQHXuT84PNMGLySYEcjU76egtxazq4HdzEg80O0Lnb4L3dEIpNg1BvRd+hRWCs443eUtBn7yAnPoY/pMfSdet63eR+AN4xvdFfZD4k4g7azD44Rnt330nBTTCSgOICk5CR0zebE4b9CS3kLKT+kEDw+2EKJRDAJuFa5UuVaxdnSswQ6BpqPY0JqkhKeE44QJp6je6w7A58ciMJawaA/vbFrmEf6CBOCIJITz3woEjm8Pu7P/kn7qfFoJ7reB+MNE1xh2+NzfP/Q9+iVelzqXFBZ/b3vsuBQASp7Fb6D/3ukbVd/8gkfvPYaJ41GBmEeq1mo0XAemP6qKOf/7HPP/bcc8/8W/E/Pfbfi38m+f+P/COyYX4uy6Sj9khdxYGs70+7tYatKbORcGHCB68HXkdskA+bsiv31XwKwb+o+AL7ne7NxhbYNyY2J51YjxcbTnmsjHqHOMYeCwJPIZOYlxreS3f6OjaNUK0lJTEHdqmZmf1GKbcBjA+h7f1+xfP9oCKedfsO5wRnfttkW2xe7iwm6H5av4xPPf5qNaeu0jDk2hov9L6JuVXNa8iPzWdZz3jWthBeFUxhUiE6l677u+AficY1y5WPPjwmw1XNhRAu5Ybn0dbqbO4Wtuy0L9y/stboIYN6vw2i3baU41J2kCC/U3moGPT0Iz3hPcgubWPnOSk4PO83x0ceR3ionOFnJj7H/oN22Hd+WdzAazRlqV/qKjCH/Yn/CUtJpzPPA01MMmpmMYtNelUbH5cTLnBjbe+Pj28HBz+G2CRGjUSDD616W/NDFvFzF89XPd/eBOHPqcw7O+gHf+sUMcBPF7JecXNItdeBd4Y1nlfhundf+DDd1dmrJr2fuH3NJTUglpV8KAUZHs2NHzo5E7u5M1kboc3kzlYdHwL2WvUluhx8GiCVmLzW9ZCn/WFePQWFNk2fvzYV/ue8H+uT1YdFvi9DmlXBzsk9uZ0W7ox8qoR9qjaUcpiIjhaTzZ7CLiiYpOYnWBU1m4zPXzuTwK0f5Mn4tk3c8TP+Rf09K4FOfTwkaHcSCvZa6pjVBg5DrOvDp5XvNTT9P/z/PAo64J1jq4UtuFLw7NzhjdROTtgt9vurDSGEkTvVOhLSFwHLz8TYXMUh7h7GPO4ZHrAchk0LYvmg7PgN8LIKv9X7xNHmEY9NcibKzGbBk4rbXtpP+czr+Q/17NfD2f7Gf39J+I0RlLus79v2xFBwqYMeSHdy77d7/1usqOFjAjvt3MG/XPMKnh9NY2MgXwV8w9oOxJD41jHrvGDocvakOTsL6lpvaWdWMe2E2epUamdREZ0OIGWt1bOpSFA6ZNDg3kJKQgr/LTLz7e/OG4Q2Of5/LiYc34FSZRWGiOP/dHNfVd+iZu3nubSUuahMn0tHcYblIIK4TczfPp83eluo+ExAE6Lu4L+k/p+MU7ERDbjpNTk2MOjYK+9ZTIBmGykHVHXzNi6ykVpXO9D13I+1owN7PnhdqX0BuJedqRhNpfdOo9KoksuBeZCbRIbNSq7BvaGL+mfnsmrGLRudGKvzaiZAnmfXasvexJ3RKaK/XJdwI2Py8+Gf8FGK/SpdQFyouVhA5O5Kot62Zt2kuFX0Goe/TAlj2JNFYa7gech1crJBIJd3BSb3ehFwvJ7AokEW/LaLhPiMs7dku6KwzL+x+gYfe/Q6VZx0yYUb32Hfx3/GQ4iF+W/U+qw/E07h5KnYr7tx8Vdope+3t2QVvdRiZ+hxkBhmNTXqaMsUgr9JWib5Tzz2b7xErO+ZaVgEpot3YPutLNFYaAuvftxifvGkALtWj+Meaj2DQ96REfNc91lHTil/WYTrl4rcoYPxbzOPYhbH4DLIMSAEY9EZGHx1NtUc1rQ5FgHlVTHM/CZsDf+fRbx7FSXIVXpnMxhkbibw7ksSliejcvCklg01zP2eG/Evsfe2x9xWft/F6Mw/+9CC7p+3mcv/L2N54V+pze+bL08NFFuuw08PQ1HLHEAQwyZS0OQdg7Wn5fumVRlrVrZikJoxGuhMSXv3E4HG542Z8W31Rt6opbkpDaTeD8gvlVKVVYeonkBcIVVQxr+4Ys9wGsGn2JiouVnD3hruRaAS8BBsm+nUyKWknL7c/BsQAoHKwQtusJSozCtc6V4adGYZmehsgstx1bTpOvn1SvAaTYCFLbC97kdzYbYTkyHu1nVzq3SgMLOToVBkrYj2xUbUz9KWh+A/z59qZM7zyj1fYdtc2Wm707HMKcuqWwQvMsGfl4VmUzjzAiai+QE+/JpPRREBGNCYH8d1NLpvDgyZR8snG1YbmIFskJgl2UoGoQZnU6y8hkfbYFyq16rb9q0wmqAgfBUCX0nHQ6KDuviXXL+5k0m5vDo0/hHWQWJUWMTMCB38HlLZKsia38fP4T+h3uR9ySX9ayltoq2rrrlq0PpfBhwX38MPSj5hhI/b/TF6dTOLyRDzH9mG5YQLfRx4CYE6hWOlTdr6MP+7+g0AgS3kdouGpsjBWtInzRfC4YK4f6SFEWPQxBjTVzQy6kMigC4lUJpWRsNT8+uMvB+NZIWfNsjV86f0vqq8WsnHmRqZ8PYXYBwdQMOA+8oJXEZ4dTmdQMz4DfbBxsyFuYRyHahtwqXMhqDCIaXun0eFSR/rPBdh52HHhqwtcG1pJXfAfADQYRRWPzqpm3IpyaXYSKIorItGjmTPJEfx2KJHEp5q6z6uzsZPUH1NxDHQ0k7btQkZMBiHXQ3j1vVdpXNyIV6IXk7+cjHd/0Q566chL3LXtLvRX9Ix/a7xZss+ECaPcSGpCKiX+JbyuMK8eTHHYQKHTPoqD7sfPuj+CIHBt0zVsPWyx9nNj4qGJnB94nsLgQhw7+iEIouy5yFCvR+58mAuqFNrHhjHW81lAtHnmbp5L6ZlSti3cRsKyBIueOI6Bjkz4cgL2CfZ4JlhKRFU0VqDUKUlITaD4ZLGFGoSLJJRJ5w4z6PBpWmafpfPu/nwW+BnxD8QTvTKaoOtBtKjbiC9ez/BbiPtuCYM51nqGyWGVVC7/hTKb+Wbjkek+VHaMpv6uswx8p5S9zu+ajecfzCd9XTp3/XqXRQB+0FODKDlVQvKnYqJTrpIz8o2epMjsNeHYtEUjFaQookzY+9ijVCupy6pj0qOTCdj7PdcDI9ArG5DIZcQtiiNmfgwyhYymDNjp/D3WzES4Ia7yePbj3Wv15aQc2qUV3OtjRf+nfqdlyHO0txhodQmgsO9MPAvOIK+uZ6fuVZx8clDY3XmQWyqTmlWj3AyZUcbYo2JSt11oR/hd6CbnOPg7EDkvksvqyzQ3NncTVG5G2PQw1L7qXiXhdTY6fnAQfaERJ0bwe9bvzNw4k9w9ud2/Cc8Jp861jkanRpSOf0+9InZhLHH3WVaeAhz5rgCfnLPYNFVSGj2RW/uB3No2wgxSKfkDLX0cl1AXMjZk4HXdCZPEhH+JP14FZ9A2xmPnbot1S4+CgNbGGf3IcVTWiwTZjvoOSs+W4tzHmZ1LdlJ+oZzXOl+zSJIdOAASQfybwWRp+6iaVYTlhOFV6UXzzOaupRAApaeS7x76jum7pyPFC4NBlAv1G+qH0k5Jov1kkstFApCxl/xLe2Ak2zvf4l9n36fDyR6JTIK2thW7hlJaHR347d4PqaIWjDDT9Csg2ti3JkZ6w5w/5mDoNFj0CAfRVqzsM4w2J/GZCIK5iV96NI/wM6fQq+yQaYNQeyXycOrD2LiJF+HaOQSJzhGXGmuwuqXnn0pKSUAJn3X0JWWDgN+Nx27nacecP+bw2wYZfi1NLPQ9y1uLVhEX3YzpZRNShZTY+bEEjgzEysHKog0DgKOVIzMGz0AoEbrttJuRujaV46uOc8/We/AZaGkzTv9hOutGruPL0C95w/iG2TfUUt7CvrH7mGEzA0Ei0BjRCIHQ2dBJxqYMvPt74zOgdzv0fxNd8TrvnGN4FiZTFjEWQRj21xvdAWzcbIj4/GE27bKh75+r2X4SlpxYclulod6wbuQ6AEIm9tJCR4AvQ8W4ptxKbvFOu1w8QHD2BdLHPsvNPphMKcO1rzcNmeK3b193HeONadIp2IlnSp7h93U6ij/chEQQZUgFwXyubHUJRGo04GYtzn+3xjz17Xrq8+rRNGuws+oh6OpaxeSZlYMVC/aJ81VbdRtXfr1C7IJY1gxcw32ml/h05j2YZCYEwUTNtRoOPH2Ae7aJRK2oS5kcHneC10PKeUCWw2/TfiP/QD73bL0HrZ+JVFKRGW0JrJ9jptAimAQe+H4qbbbNvPTBxzAKtnv0p7PBgY9cxIrbN3mTipe/5LMlp1g1/gLWPkv+4umYoyy5jBNvnUDXrrNI9ilsFHzz2Df4lfpRkVcBtyg1KhsNeFV68eFLH/LiDWLw6Q9O4xjoiNxKjkojx7nBltCrTbR5FtBnWR+kCil9pvSh007B+cHnkRvsCWuoQ2dQAvZm/oNzgzPVntXM2zgPG6kSzEPFf4lfJ4rz5mOZj+EWeecFBL2hqqqKl1966Uair3cMAnZrtYx48UXmL1jw/1tJz/8N/FvG89/4PwIlbt/jX+JP5FWoKzOXX1Q4Kdk3dR8V3hWom3SWCbsqBf0v9se+2Z6Isg8s9p08NoCt89IZlDyIPlfNJzSZSk6nvQe2phG4SH/HVWZeYmysrMG1+DJ2rXY8/dnT9PvWMjBwOwgIaK20HJp4COtRYgA6bX2ayFTMayBRIS6Ip0am0OBjqeU794+5zNk8B0FislhwI6Pj+faRbynxL2HO1jnIsszlMDsuXWD+xvm41boRlx6HslxkZeXuzeXUu6fwSvSiw0dOcWAxG+dvxDbqzid6XbsOr35eBIzs3bC5OCiPU8NP8dPCx8gKzMY5xJnyC+Vk78xGopJREFJAi30Ljs3+GG8h1/R3n02rfSvDTw3HK+PT2wY+3Wvc8bmeR3lRD1O37FwZH6g/oM2mnrNDziJI/p4My/6n9/OB/Qe9lve3tDXflOgTIZVJkUglSCQSWozVtFlnYhSqEDpEhl/j9Ubaa8SKuV/v+5VPn/uUB398EM+D5uelbeggOjMap0YnokpWM9Fo3rTJM96TPnf3pdPOjYKEu/EYbRnE+St0Ocu9SX9Isq4Remkjys6m225f4V3BhnnbsY0zZ506RXiSN/g+THIVIRc30nD1lkrKG5r2mVGZfPnEl9j1NY+UCIJAcakUU1MrXz7z9xKzGRszMHQasPe3dGoAmjwjUHa20HnyksVYXVs5Nh02GGQGHOMs+8GFZ4Yzc8dMNs/dTPawpRbfXtb+LJpSmwgsCsQnzdzBaS5pJuTiRpzKM/7W9dwp3KLciJoTZcbs6oKdHdi0VBN+/hccqnMsxnct28XW+Vs5/PxhCg6bVyDVZtXyluQtvg36llnrZjHydUumrUFjIHt7NutGrqMhv8Fi/D+LLnmprqBLR634nrZWtopMt8S7qQ62rCAGaC+qxf/aAXyyjxB4aSuNhebSTh6ncxh+ajSl/qXsmrkL+U0VHC7xflxPmE2dtw86WQMmiflkVH6hnK/Cv7qtXHGe/yFORQ9Hp91oMSYI4FRdjHVLDhIkCALMWj+LVcIq7DztqG8rxrHRkX6X+2FbmIFEIuHlppdZqRF70HSxFI/NjMYw626kMik2rjYo7ZSonG3YcdcOpg2/xoTC94lKXYfJJM7LE/aHElQURFChGBBN7r+ec+NyUNmrOP3had63e591I9eJcnK9sFA7ylrpf7E/UpOUNifxOcz+bTarhFXYuNigr2nEvjaX6DO/Ur7uM0Cspurq/+FR6twthW00mcjdk8s30d9QeLSQxmuNrHxvJX3T+3Il9gr6WwKnvsGjqAnoz5dnv+XzFc/Seuic2bhSr2Tkj4tZb+pAow7G3/E/Dup0obOxk5KMEnQdPc/4LclbvCURy0+8n5rF1dirvP7u61zffx2XMBdGvzuaMe+PQaqSsnnOZuQGOR1PdFj0fRoTu5TiPipyInJocLf8LotCyrkWJ/YYvVQ5xmw+sfZ2JmXUBPaMe47J+yaz4NfJd3xNIDauP/XeKU6+e9JiTK81MPLkSO7ZfA/yWsuKQxPi4loUWERHkC2CIJCwLKG7sv23GR+zaa7Yy88kmNA0a6hKq6KzoROJly37J+2n1E+UdpTe4A12BTtafXoCsf965F+UzjcPzBb+Uci6Yeu65V9vhra2hT6XNtLiGoTPeMsE7fVBjRyceJCozCi0LVrkVnJWCat46NJDAPS/2B+vSi9WP7ca0zA3HPwceDzrcZJWJGG81szqJ5+ltsKVE6NOgLeqO6Hhm+RL8EEFQ955kXfyHXgn1xFTZ89aPXXt3Wy6ZxO7ZuwiOyKbvVP2IrfvIRZIpBIi7opg8peTew0222nDkTm8QtHAF3AKtpTBSp7Qj/PjIgmreAuJRHxnPfp64JXghcRNTkpCCrbttvhflyEIsHX+Vn6b/BsAzuVWKC7G07R1AlN2xZjtV6KU4fXNm8x6dDsbdz9CQPouNE0aYu6NYdDTg7D5LI9Vb69iW6Ez5zWQ6/m+2Tt6/M3jrPZf3S29ejt0BZXy9ufxXeJ3FB4rpCGwjZ0zdpIRk0Gzk2gX2XrY4t3PG6WdEqPchF6pJzkpmeKQDsJnhPOG8Q1iF4r28KXhbpwf8Cer3l6F114dObtysPWwpe/9ffFv6cMDRS3UlLlxaucwZKli4OBm1v2A02Jgb8M3swk8IsqrjvtwHEq1kslfTmaVsKrXZJ86xp+dMw7R4NSAqZcqCs8qJ6KyopCapEgkYmV63KI4rJx6+r76lvkwf+N82q5UEbcwjhdqXiBoTBCatGqe/OpJHJscOTn8JApfG3wH+yK3knP2n2dxrLGizADjrtrz8cMv4X6kgY6SWgIy9hGZ/CdPfP0Em8sceSHHDZf8YEztPddbc7WGwy8cZvPczb0+o9TEVHbO2EmJfwlylRwrRysGPjGwm/gz8cBEUZb9g2SLhLS+UY9fiR9VnlVkJJRZVLyr2mXYtdpRGDcMtwlPoO/Qs3X+Vn4e8zNWTtasWbqGc0nniMiKwLlMlCu+uaL/Sp8POKV8jZSQuexRib6RpkmDU7ATKT+kcPX3qxQctJzHbN1tqRhTQcyRGGZtmmUx7jLZha+e+IqEtAQu/cvSHtxTuIQK21cpDXZE7u1O0fEidK06qtOr6azv5P6f72fygUkEpu2g9vx1s21Dvcahlq/i+dBKJvYvoMPV3FgMywwg5mwSk+rkTDHWY7plyd04cyMZGzO6K3tuhkwpY9grw5izaU53IrBLphKgMKyJEyNPsO0+L/xmTsUlzAV7H3uu/HKFj5w+JPHYEfqnTMeneggGnZFjq47xy7hfkEglGOwFKnwqaJdup7npnwhGgYJDBex5aA+6Nh2Z8YWkJaRRq3GhrsNPTMxmVNL3z88ISt+JdVsdMwbezXsPj+PU2DY0DndeJXHqg1P8Nvk3SxlYwM3JjaXJIgPo2qZrmIwmHs98nFXCKlxCXdjStoUXOl4gIjsCl72WlcYx82IY/+F4VvuttvgGbk4O9rvcD8fTjgDdVcQA0ZnRdFp3snbpWsIXWxIY/wpHXjrC0VeP9jpWHTyYOr94nKqzUWhae/3Nnkf3sO/JfRZ/vx3B+OjrRzm+6jhlYfV89sxnHJx4kKujn8TKy0nspVonSoukTnyJoviZGL396LT3FCXfM2vZOGMj30R9060+0xsKjv/Ms58+xEsfvovkT8uJcNzn41iwYQGjj4+mpcg8BiFRSqj0ruT7h7/nyAzx/TB0GvAf7o9ruCvNGaV45R7HtrEcabOlJGt/z518Ny2SpCHhpEX70q7roDWjCLumMtRNzVR5VXX/titOkLYujV3LdtHZ2ElbVVs3ee5WRM+Nvm1SUKG2ojpoML6Zh/DK68WuatNi1dGAc1UWsqtpyJQyPOM9sfcRfdFhKcsYevEpBvyZjO7YWrNtTYKA1Cil1eoK9rmPIt0lVrmo7FVYOVih2r4J98P7+LpgOm0jz1EZ1YxcJed13evMWj+LrG1Z7H9yP52Nlm1X7E32lFSUcGTiEbSxN/oumwRMBhOCIKC0U6L2UbP30b2c/fisxfY3J6oaCxtprWjl6oar3fONYBKwb7GnX0o/OgvE4xccLmDfY/tIXp3c673830bXtyOSX0Fj6/y3ZfJ7g0whwzrQE721PbV+4vxxqzrAfwST0UT8knjiFvVOEBj0zCDGfzyeyV9Z+gRaZy8avKLwyT3GV+FfdZPKBUFg6AfTuTRtFZemrSJj9JPd1ytTyHDwc0Dh5ca10U/Q4eBJv73voKs0JzTLde24F19EUtGlAGZ+7OJTxZx48wRtVeZrVpcEcf6B/O5vQTAKtFe3o23VUpVWhd2VDB7/7kPu2v8jkkoTp98/jclg4uw/zxL/pGirddg00W4Qn9f1P8X11jHQsVttol/Gw3iX+aFUK3mt8zVWalcilUup9Kmj0quSyxrY1w7NukaL72ShwQFrWw22Ls1I5HfeZ7brWrtIaGZjbSKZ/94/7sWYZklAsb+iY/Evi3Gtc+2ufh728jD6PSwWrRydlsOaZWuITm2g6OIBXEJdeF33OmPeGdPdrskgbyH26IdIt30MiNW+Xbj/1+fwqpvB+UHnKUu683UYer75282TfwdrfviB0QrFbRN9XRgEjFIq+XHNmv/yMf+NO8e/k33/xv8qrKxER9u31JfigGLWPHwYWx9Hs990JfcmHJrAxO0tFo1h7YtUTNs7jaFnhhJ6zdKByPZ9mQqXDQw/NZyINHPpEKPBhGCo5nhkICei4tDIaszG9bmFBF7dg3ODM/l98mkLunO2X11WHS989AJDzgzpnuT1HXpaK1ox6ozILhTxzOpn8Kx07XV7K40V3hXeDEzui6HC/Lzs7NT0L6ukzTmKTfdsoj3cnOmVGZXFtru2ATB7+2zUV0TjL2BEABM/m8j8XfMpn9QTqLpVguOvcOr9U3zs/vHt+6y5fEpOlDjWphcX7Y66DrQtWlTu1mxYsAG7NieeWf0gbfnmhoZzpwcudS74lvniW2Jjkexb8NsC5myeQ3rfdD5e8TE2fR27x2zdbbHzscOt1o7FPy9m5J//cZPumyEYBXStOq5tsuzVpjXqqPASjZ/MG3JVl767xC8TfqG9ph2PZCmL1y9m7B4Nxs/FBfmbqG+65cXG1+1EkAg4NTph3W4u42Kf4MWbq97k4oCLjNurxnNHpsXxTSZoss+hILwZdZ/eewLdDk/mPcnSc0u7DTCz/YaGo7NSE5i2s9dt++T1wa/Uj/zQfFRu5pViBc05pATPI8N7Bva1WRSWmN83fVw/zk4dTKl/KfWu9XBL8OjsP89S/tsJAPyvHfxb19TZ2Imth22vGupdcCu+iOayZdJNHu7A+6+9T/LgZIpOWso6JVRNICEtAdsO8bu81dg17DDw+8Lf+X3B7+z5eI/ZmLZVi1N1DiGpW1HmZnQ3hu4NXXPfnaKpuImW0haGvza812cZ5VqDbVMp5eGjaXUOtBgvOVVCR10HS88ttejZd/Oc+vPYn9EazOWMi08W493Pu/u/b02q/VfgHuOOR5wHruHi/fYZ6MMqYRWTP+9xclRtdbi1Xrfor2QX4UN20hKK42ZQ0G+uRRC9dcJszo7oCQoICGiaNFw/cp220kbaHbzZMTKEqx4xFKtGmD1rB38HAkcFWvQJ7EKy22ImHB5N/+NXLcYEAT545X0a7NOJ+3MNJpO4jpmMJgSTgG2ehGc+f4bUfiVUTlhisf3CjQ/y8A/vUel2hitWmzFoDPwx5w+Ov3m82wEYaAWDJiXjOEIM1nTNW00OTVwP7glK5ivEb/vcJ+fQt4vJC/9h/t0s/pvRltvEtL3TCM0LRdEmfq9GnZH2mnb0nXocpw5l3aNlnE06S2Okpfk4fvdglv24jFVvvYlyXzNFJ4qozazlyEtHkNkrSI9LJyMmg7qINxg2fJnZtqqoPpTETiVLOwiNyRoDPYmWGWvFKr8a9xr2Tsjlvjc+xzrYUgbsdtj/+X5+iv2JD2w/MEsGdKFRX0WjUyNXYq8gcbTGuY8zI14bgWdfTyRyCddirtGqbkVdpjZztEBkfUsFMYprlFru2ytxFXsm7yfwcAC/fHQ3lRd6iA1SlYIWVydaHFpQaVU4N0f9rWCEwkZBwPCA7t6IN8MklXJs1DEAlG2W/WtDC8KZveMZaoOm4jL9ISQSCTKljBNvnRDlcyTtSI1SAgsDUdUa2PfYPr5L+I6dD+wEZxXnB5+nxqOGJT8tYfp6sRKs6zzU5TLefPNNrDusqXetR+dgfs+ufHCFsuQymoqaLM5LMN2g0vdSMQvidxSRHcHkA5PRt1gSWAZeGMTAC12S5FIufXeJb2O/pSGvAWSQad3K3Hr9jX2ZWHp2KauEVTgGOGITFE9+hJzNGb6Y3noG6eUe++TyN+dxanShTd1GmV8ZFwdeRG7bM4crbBTcu+1eBj7Ruxy6R85+ok58gWCyvGaAgLrlxOQ9iO/1Wtqr28jfn8+2Bduoz61H2seaXTN3EVQYyqhDYjA3Y2MG+QfyKThcQOa4cl758T3uFmzxK/W5RWpMYFsbfNsMGcVtuJWm3pAfhUPPHer+3ZEqO174bTj9f5xrNs+217TTUtrSa0LZZIL4A/+g/563qNkpJucz/8ikKrWK2sxaTB5KUhNT0aq0KLWijXrirRP8MuEXABwbHfAp80Ghc8CzaRYSiYStC7Z2S2xeGv44Zydl4jUggw4vMVissFagUqvQlNZy6NdJVBT4cHjbSBzSRPZAyPgQVgmrKHl+H/e9/As6k4Q/pu0ie6QoAZy2Pg1dq+62yhQAya37SU08yxdPf4HzCEt760pSCKufWU3/S/2R1lRh52lHxqYMrm28RkdzJ6rS57BpN7Dtrm0owh3RNGsoPlVMS3kLMh87Tg4/ydXYqxwdexR5gB2j3hpF0ook5u2ch8QgIehaJMkGIxUeVejVUqyCvckZvJgaPzERH54bimTkJTa8/S7K8B7/wTnUmeDxwQx+bvBtry01MZVf7/sVtbea2sxafp/2Ozm7RHLQ4OTBeFd40+HZgUQqIWtbFl+Ff4W2VYuQ1s7StUsJLArE2mCpiDB8czCPfvsoXRpzUrkUubWcuEVxSBUyyvzKaHJqYuiZoXjl7hKr0R2tWFGxguT3lBQEie+XX4kfzlnifPWJ9ydsuWcLA58cSNj0MAY9bRlKqm6rZs7mOfgX++OY7Wgx7mTlhElq4vf5vzPoOcvt++7zYcKhkaQO9cSqbzjWztZ4D/Bm2MvDUNor2TN1D8UBRbiWpdNc2Hv/3oezp7G3HWTCLbZC0pvkDHse/7bxqFvUmLRGM7tvpWYlq4RVvVbitFW1IVVICZsW1p1Y/djjY74I+QJBEEgeXSoSFmRSkIjjd2+4u7tXnbqpjmm7+vHAD4Opz6mnLrOOktMlIplRD0qtkqRzSQw+2IFBY6D4eDF5+/Iw6oxMa93BlNzToCtj9xYHzuzbitzJjuqgQRjkUlISUqgPKsLftpLA2MVcb7pzsl59dj35B/I5/cFpizGJVIJzgjMe/Ty6/30zumIBiSmJeO64fVWAd39vnMPMZSCqUqp4/p/P0/9if35c+iPbFon+cVd1NMDqZ1bT7NBsdqw7RcLSBAJGBfSaxJSo1ZRETSJ97DN0OFjaLWuHruXyvy6byWB3QdvUSf89bxFz9Etkus7uub0rweKb60KHTQedNp1obZ27m7q3uAZhUFjT5+IGVG31N6T/xHNzCXMh6LlZ2PiJCdP7j9/fq/RlfUMGVhopKo0eQW95PwoGFnBq2CnWP3CIgIGBZmMGvQGVRoXMIMMkMSAIcOXXK5z9SEw01VzJxCf3BKHJ32G/c7XFvps3n2TLl3OI3jOZx997FW1TB1Z9eirxXvzwRTZVRLPj6EwS9r+HyWBizyN7SP0xlfOfn+cTr0+657VbUZVWxd7H93a3NLGARAJSKYJEYhGD8J8SQ/r457k6+kk00+diMpooOFzQ3f/PPe0QgZkpHB1ziCt9zX0DQ0E7b7zzBj9dGcZcjyqsrHtIk13ra8qgVn7KfYBk9/cZMuE1AOpy6ig+WYxLmAuRsyN7Jeo1FTUR9WEUyqNK2m+05cjdm8s7indIX59O9D3RLEteRm1WLUXHinq97C51BIWNgrT1aWxbsI0jLx3B3see0SdG8/uC31n39jrmrhBbAnQ2iMmUvyMD+Hf93v8Kur6VkrhpXB39JM0efy+BfzuYjCaqt58lMG0HxXHTWSWs6tUG/ys8dOkhZv40s9cxiVTCpNWTGLJiCCq15bNu6xPP9X5zMShskKlk3XNOY0EjPwW+jU/Wke7fdt0Dg8bA11FfU75cJDg6V2YiQcDYZp4Q09i5UuPfDxzszbbvQpdUsXOI+fzq2deTWT/P4rHMx7BytEIQBNTeasa8N4a2yjaWX1yO3jsAueBJ3/OldGTVEjxetGdKTpXQIXSwaXE5jk2OLPh6DgaNgccyHuORK4/gleiFw59Snv7saaZst8fp7F4kEglyqx6VjAPTznFg8gFmnvNhy7sP0H66TiT6ZT+OcWUg6xev5y2FPeOy7Pk836nXXs23Q8iEEGaum0ng6ECLsYpSI06NTpwZcgZZX0ub0id2LAcnnsO5wRlTgXgztS1azn50lo76DgQEOmw6+O6h76gdLUPfoae5tBl9px5NaSuPfvMoAy4MIDMqk5rAHp+4i5Rp09ZJg91BLg24RPkgSz/vr7DkxBJWCavwSrhzX/p22Pn779ynubPjL+7sZMfvv//HP/z/GP4n575b8W8Zz3/jfw0ymYyICFE+cNaOWRhlRn6//zwqG/MJU1PbwT2b7kEiSLg0qMKiV1VzhIGf7/uZUcdH4VNuWc0VVOCKuyqcS6P7Em5t3qC9paiRAQf+RWd7CMHXg7E27Qfu7x63io8gtdQVR70NBfEG+gVa9gy8HSQKCa3qViYcnkBrQiskiiX5Bo0BbasWiUJBu20792yaSavXMcBcmvGXxb8QlhPGgg0LMHhUAj2lEEa9CblOg14JhVFZ9HM2vycdXhKuOF9BqVWyYd4GhvivAODCFxdoLm0mZMFg8gqMeCo8iU+Lp9WvBobdmQxDl/TFhukbeCLH8n5Y633xqo6j/7lBGNpbqA6sxi3SjcjZkVTcsByqPGuo8e+Hwt5cv6N+x2me3Psknz77KXorZ542vWc2LjPKkBvk6JV69Eo9wk2FFC5hLtxz9h5emPgCkdmRRGT9vemt/6P9yd6e3auzZpRK+P5hUR42KVtMUO19ZC8gVkUpm8Gt0ouUxBQajT3WkbZZDP76lgYS2BzMJ89/wt1W31ge/MbjM9IARvP+i6c+OMX5ry6Rfs9bNDs2U6epBO68/N0xwPG2PfFMbh60uwRhX1/U6/iYo2Ow7rTmy9DvLWKvlXnFRKbUkR3RwDtvvM2/+puzWiUSUHfGItfLse60RtthHghJWZMCQIN3NA1ef69accCjAxjw6IDbjgembsckU2D/wByLMSni/NInvw/VhXkW44Yx8XyStIiA4lDsa/IQhFCz8bdPvg2ASWbi4qMXzcbco92p7DMMr/zT2B3YSsXleYR7WzoYN899d4qGvAaOvHQElYOqu//KzZAU5OObfZSsoUvR2Fs2iu7tW735vN8wvcGygGXYtdmxfft25s2d1z2eujaV9J/TWXJyCQprBW5/oxL4P4Kdhx2PpD/S65jJKBBwZQ8uZVeQmgxoGldg59nzfSjU1t2yqRIJWN9SMJMTkMI1h+34lvoy8eiz2DkrqXGu4ZfxYrA5Wipj6yhRajA6MxrTOoGuj9Ep2AlNk4aUH1IY9pKl9Mt9v9xHUFEQ+yed4N1bpH5MJjBJtbTYt9DqoMdkgt8m/UbBoQIeOP0ARlcFZ4acoczPiI/EGn2Hnvdt30eqkPK67nWUBkf0plZaZD+T33qVhvyJZG3NImtrFpPGhjNr+yy+icphaFI6u/zyWGr6ArdoN3JiZBwe+StNTk0AxKfGk3gtirYn2njw9IPIlDJ07TpSfkihvbrdYl6wi3Nh3f3ruHfTvWhSDfCBSO448dYJpn47le26fZS6/UjpRJikEHvmfOrzKXJrOU/lP0XyiCtYNxqIzruXQE9XRrwShMJagd9QPzq8jWyfvR2FwZHQtL443lIEZxT0GCUGlkwbyd0P5jKzcgUvIB4j4YEEJmQ+i5vzCXzkUGayrHb/KxxRHsFN4karuif5/rq+R4Zuz8GncNY4s+3ubTwTLhI01o9ej1d/L2LeECulDk08RPmicj5wMVcPqPsznQe+G4Zd2xg01tnws/mxXY1x+Nbdj3XneaxbSiipygbEXjmCSUBi1CE1Stk77RQT07f/rev6/oFktCVGFvTSn8UEnBh1ghOjTjC6fL/FuHexB1Zpjkgd7kESKz4MWzfb7oR5/Lkg9CYbRp0YRW2ChuCngrn6+1VCp4Uite6pHmu3bYcbcj71eebkHZPUhF2rHZKb+ivKZDLm/DGHsnNlvfZtUro5kDNoMZ7Xz1F1poCYGHOZI/cCB1TaDn5b8Btv2L9m1rtjlbCKbXMOodBqiEuPQy414hDrgFeilxgwD7VjzfI1hOSHsOSnJegeboKb4i2ecXOp09UR472D9GFnCPGO7B7L3XaNGK9Y0uJTwFpNbP4vmG5pI1Z9tZqGvAb6TOpjJp0LUGu/D99cV+TVK2mt/AC1l3m/IHXNNZzKT+JeXkNT/v3d0oolp0sw9hfvX3piPTK1KAW06OAizn16DitHKwJtm1Fa6Vn8wgbk2Ta8JHzUvV+D3kRZ2hCy/XP5ZPkmnpxsi5XrG9TliMkvmYMtHzy0ivd8NTgWxZBZ54TpJhtm+GvDaSpq6j6fmyEIIL3Ry0eiEO0tt2g3pAopvoN8EdKLAHj020fRW7fDasSWSwK0lLUw5MQgklJUdKg9aHOW0VHXQXNJM7ZutggChKuM2Kh1zH5yCzvK4+j3UD+CxwWjVCtRhlnBXV58Ih/CoeU/4CPteRiCSeCKlcBWdQ2JjVEMsfmcwOhEQLSBATI3Z7Ln4T08XfS0xTxo6jDgUudCm10bxl7sQdeW8Wh1PzDx0EQkgWWAJ9P+NQ17P3t0nTpi0x3QKFxITkpG4mZF5uZMdi/fzfDXhiPt78DRsT22kkkQuPr7VU69ewqAQQQR7uDE0A9+w/bZM2xrfAyJjQ2trkG0W9cjbzuBt72GrYGd7GuH6psmDLWXutd+kV0Yd3gcTo1ObL5HrHqqz60nb28eValVhE0PY/Wzq5Eb5MTXJiKYBCouVVCfW09bVRtR8cNZN/RnYjJiGH/EH91KHUrbHsJISVQDne3FBGR20lj0K/I3FvFahxisbmoSkOvlmKQm9k/ej7XGjTdMPfOvcFN/l6FnhhKaJxLqEh5M4OLXF8nbl3fbXlPVqdU8/tXjuNW50e7RDh+ajyfqEpmSM4WamBrcEyztorSkLLQdNRR6nEIQ7iVwVCDLL4ja7CVNpVwacAmJSULy4GQWhQgspEdloObIURL2naGPjS8l21YSkpQIN6mbGq3tmJL4Cas6Ctj+zQpaHu8k4rE7s/vS1qXx5yt/svzS8m6i1U03DEEQUOgUONTV0V5dT/UVAyfePkHY9DAc4505LD9IlWczA66tROVkQ/S90WTvyKb4RDEjL44j+jcJByYeoDSwidfkr5G1LYuYeTFYO1vjtzsVd62OpNgq0i8F0RRajmqKE6XRk0hJknE+7A2OV/RlXIkLo5Ij6RzYCaG9X8etmLV+Ftc2X6P6imXiVBAEXtrxEs2NzQQrgpHKpJx87yTHVh5j2flldPzYwSvrX2Hz3M3ExVhWxJx6/xTFJ4tZuG+hRaJQppRR71KPxkpDi0MLdkq77mN2Ifh6MFmRWQy8MJDyhHJ8Zty5NGFtRi3pP6cz/sPxSG3M31OpUYdEMKG3su+VxOLV3wuXCBdm/thL8F8q7suqowGblmqMxkDkcrENwraF2xAcHFiy9Rj1PnFAMwadHdjIaPDti0LXgV/mIVxLU3G6nIG11BbhmeXUddqxObcv9O3L7pLbX1P+2DZ+GflPQitW8daSeIvx9LEZFLUUMCTrDM5qc5WUlustvPKPV6j0qkXjsRBBENuIKGwUNBY28mfUPpIf3UhQYRDWWstkd41SQ19rLX5hpexploFMhsLdiUaPCJyqs7HptCHG2kSDdx0NgUWAuP/L/7rcnZDoqkq7Fd8liFLq/kP9Lda11so2glO2Ue8dS61/Pwt7rCv5p7V1xmQPzSWN/DpBlMNbJayiqt9UilWHODvkLMGGWxwSWxlXY64y0q+WUXNPsbdMvGcVlysoOVVCWVgzeyatJr7kczxaJ+Iut8ZkMLF2yFo6GzqJXRhL1rYspq+ZbnFNVh5WJA9KJqQghLpLdTADFNYK1D5qM1vkpcaXelUe+Drqa6wcrLpVOpz7iMkc51Dx/3VGHSaZCezplhGNuTcG38G+vSoV9Ib/jN/7X0F3oktpg0EpxppuTd7e3B/+TtFU1ETFusO4AkVxM+i1ieIdYMf9O7D1tGX8h+MtxgRBoPZaLSoHFQ5+5onErimkLGo8b/04HtmNx6uwVaBytsGr4AyqziZqAgZgMon+cfXVauqyRJuv/x4x4Zc95AF8gs0TtXaNZVi11yN1coBSy3N2j3WnvbrdQl0oda2YZA8cHUjRsSJe7XgVhbWCwqOFpP2Uxrh/jOPoglqaaqqILJjO0MQg3NrEB2TvZ0/uVyl03n8YuSEUhVaBIAhmrUoEuUCndSfZkZV42iRhMpg4/8V5tK1ahr86nHEFqXRqjJTY3IVLvQvGNoMoaxruin9iEtaVT9JU5kNYWglPX21Htlx2c0j1L+EU5ISVg1Wvikr6Th2TD0wmJSEFiZelfeIZ3J+6mmgW/piEvsUAy8VnK5gEEMC30BE3UyQZsRl0OujJ2JTBrgd3MfCpgQjD1MiMMiSChH1T9xGvERfZtsq2bin69YvXo1eINnhv0tb/U2huabnjqKQH0Nxsqery/2X8T899t+Lfyb5/438NJpOJxsZGnJycODHyBEqdEvsma6SCgZtfTZXWiqisKE4NO8Wf4zYhkX5nth+do8D1kOu0qlux0ljxNqvMxpNOhSIR/EmerMZoZT67K+xUXOp3iXKfcjyrPGmwPszNyT6ZswMtbg74N70g/v7OJblxCHZg3ZJ1rPhkBfoicTL2GeDDiDdG4ODnwOGoTSSH/sDz/3weVbulUSoz2lLiX8Lxye7MiDRvpJ2fnknCoa006ispc++R5OqCcOPfOpUOQT2ffv6i8zrkxSG0VbXx9sgjhDh1IvF2YfD5wXROvvOJN3xGOH0X90Wq7L0wOPT8b7jZhBBQ7IjQV0tHfQd5+/LoM6UP1vZeLNj8PoVBVVwd5IvC3Tyx1R4u5WTLSXRKHQ4dfS0Ms18Wi0F6q04r7NrsLKpuFI4KIrMjyQnLYf+My3zMR9wp3KPdea6896axxhvBrzfffBM4hvDbcOZumUv+/nzcIt0omqBj7Uixoi/wRgTw3u33djsbHsmXuadlPh++9C6mWxZkbbOGgKIAGpwb+PrRF5ljMpefUalVWLvboW5Vs/yH5eTmH2PqSPPeIH+F7/t/j98QPyZ9NqkXpiwUJdx1my3h0IRDhOaF8soHK2h+MBXm9VSE6So6GX56OE2OTZT7lndLD3ZBWlmBd0E5w1ruYtSxaFp8qmB4T7+V+w7dx7qVeZRd0tDs3ntPoL/CruW7cPB36FVyUgLorB2Q2NtZjPlp+jDownQOTbnOZ3M/tRiXKVToTDrGHB2Gun0TsNJsPLggmA6bDowyI1XJVRYa7uURY6nzjSeY67hH925N3jz3SaU39cUxmNA0abBxtWxi4T3Am3EfjaPwSCHB44ItmHWymEiy61zR2FrKG3VBMAkYtAakcmmvDs7muZtZ8ekKSraWwNyevyetSELboqUuq45+D92+mvI/g3NrMzn19nHmrJ9G8Eh/yi+Us2bQGiJmRTDpp7m4laSgs7KnJmK4BaPVdEMqVmxZbulwFSrFykvHJkf8ClvRVDfjHOLPtO+mceHHDGouFNHvUj8uDLxAXmghK4Q3zLYfuWokRn3vBnSbtzeXnS9TG2h5PwQBgq4HkRafRlZiIK8LUHpW9JysnawRvKw4POEw/uUjkFfXg0QMdnT1IagefS/XTNtZtmYZdd7t2LxmQ8KyBELGh9DUpiU+PZ6T9i283ZCFzHhDhnCQL1cHuFDnVtd9Hs4NzngXOWLoNHQ7Trl7czn/+Xk8+npY9O6QOyopCiri7JCzOEhFJzBzi1hp7BbthvyQkZCiEEr9SjEpxHO9uXo1N7qYZlkBBvcPGDMgEitHGP22SGJJK8sGwKXOmYD0XTSnx8KUnvng/Jl/4FBwlbFjxSDgzTH2daPWcW/Zy5QtqqeisJ4tx91h9K/czaJen82tCBsTxrOrxD5Qn9qL3/zNMnTRJ4NBE0BaQhpGk4nik8UUHS+iLruOkCdDeOGjF0jvm87Ad8Wqrfq8eqqvVBN1dxSpmqM0O9RS42HA2jjG4tiq5NPcXTWDz95ZT9WUbPZ59PTHbcmpZOT27ehbhnBhsChje6fJPkGA0ux2bJtaKC8H31vI1QaDEbtWO/QKvYV9AOAwKYnjzQm4lF/BeDQL44tDqc+tJ3ZRLDKFjMgrvgiCO8fGZhHnv5j4JfHE3wj4Zb1fwqPfPMrxUcfZfM9mBtiIPcaKb6qUFtdKeO7T59AEyrvb1JpMJjzHeBJ5d6TZ3Hfzdcl1HXheP0t9mgIwT/YF5vjge9mHkvAEJAo5Rp05m7TBuQnPCmtmb5+NxKoDryVelF8oJ2NjBsJAOOYDbg0q1te5YezQc+ajMxx56Qhz/piDyRRNqMtF3hx8ifaB8Frrk937dY/3gjSYvns6jr6TcEzOpWNGf7oaFWlbtfwrTkwUP1P8jAXT+8i4Q9Q5ujH+yHhaK1otkn1+13YhCI38tmAH94a9gL2zDqWdEm2rFvdid+7/dRVFYe40ekcjCCLbOGSCeG+KfnLlnWYf9jtXWtxPK6MVc3+fQN/h7ihid3BEMPCqRELR8SIA3B6eTYfNCwy3g37L9hEw8gyvW/VkTOx97Fm4b6HFfruQMlXMbiy6Ea8a8vwQhjw/BAD59808vflpmh2aqQq6QXxK8qUypRKjzogkJpqSTlfcSlJxstNRfKqYsnNlzP5d7GGd9I8neCGgjL3z/8APIwatgS9DvyTuvjjOjm5jTfzD3KuGb23hX1ViormpqInPgz5nGlO5/P4VHg3IJGHsNNY0mCcXuoIlgtHygzNda+XJr56k0rMS3Qgd3MLXUTdpsVN7sHHBKVYNWEHRiSKOvXGM8R+Nx6GfSiSrKfSEVryB20B/GvLFqms7Tzsc5R4EVz2PqXMLI06OQGvXQFVKM2ofNR6xHmxrOwYRx1jtUAvUcr5hYLcd3O7oya9LTvGmdz1NtQ5cThlCyCTLANTt4F0ZTPB1b0w7A2h/tB3/Yf64RrgSuygWk2CixaGFmTtmkpCWgO4RHYGjAkECNi42BITH0eg/DY3sbTxqxXVj/Zj1KKwVLNi7gOwBNRTIjvLEl1EYZZfhprm5raKVle+t5NzgcxycdBClXidKdB8r5JeJvyEdbIQbudozQ89QEtPJ28CUr6bgHOpM4ZFCKlMqcfB3wM7D3KYTBAFBIpAZmUnrUEtmed35OgZuHMiiQ4tQyiyr2TMjrkJzG2MOj6GzPRuTIYyjrx/Fs68n6kk3qhukAnqlHpPM/F3JsDqFsU8OHbZaXP19MLqaJw6kHW2UawOYHFnHwInJVAa8R319fbfdt2XeFrQtWu7Zeg8Ka/PnGDAyACTwQ/8fWFG1AjsPu+5APMDdux6hVuZKRLaWQulu2mNGkL09m8BRgYzYPZZnfnwKhd4FLyZi4w6eCZ5M+HQC7rHuWF9SUBnoT054Dp1qK2RKGQOfHNhNvpBotZi0Mt7RuaN470Mcgr/BcKMQwrapjYHnB1IWIOO9HD/6nE1A3/T35MO6ksAgzpsN+Q3Y+9pTo6nB9V5XOn07MfwoHvDYSrE6XdemQ+IuodSvlGqPatp8LaVP63PqKTnde/bKKcqJnx78CQD7Znv6eYt22/FVx7t/E1L5FMUBKxhzbAxlsWUMnNF7lXZv6PdIP0ImhvRaMaxOPkLCpYtkD3mADntPwPw9vFnF4lbIbVVkjHgUdWMJGlvn7rkg+p5o5NZyNs7YiG/9n6g6GnErSaHtvkdQx4uVkdXBSXjmncK2qRx9cBgtbUoEAUpuukW5e3KpTKlk6ItDLar7ulQk8rzfoqDhPkKcb+0vJibxlVo9jXVG1Oqea5fZyUiNTyUhLQFD7X4EoS8dtR3k7MwhaUUSOjsTNR411HjUIDdZ9o/+ZuFm0mw6aDbBFc90ltqI/QZLoydg1VLIqWG7+dg5CxtXaA3J4QvZD8QtiqP2Wi2eCZ7Yedvdttdsn8l9aK9pJ3J2pMWYQWvEprkS+7pCHGvyMJkeNBs/uqUB5/JyOtXu0G6HSq3CIcChm4C5a9AS6mxO37g75naXzEPF1jlb2Q5I88BB78APiMmzgJEBnHE9SR8FpI59mk2rz1Kzcyn62SO6K+iGvzqcxOWJvapylFFGfp98Bp8fTHN2M8wQ/b3W8lY0zRrKL5RTeLSQuPvielWG8RvqZ0beiJ4bTbQgSjNqmjVc/+k6wQXBuHq70lHXgY2rDdbO1mY90v8j3M7v/X8LXba0XX0xQWnbKY8YiyD0tMwpOl7E+tHrmfPHnL/s6X0rVGoVDkOjSW8NJuz8r7wlKWRp8tJee0DeDik/ppD+c7q4xt4Cg9bAprs2kb8/n5h5Mdy94W6zcbvCDGyLiyiLGM+1ayrcb4QZ1F5qvJICKdqbiXPFNZwrrqFtfAGwwc7TjqEvD+VKjhWt2/8EuDEXmUOhacW6pRqpRLx5t8bghr08jIFPDMRkMJnFECpTRNvTLcoNxyBHJFIJ2lYtnvGezN89n384/IOwPpWsX/QdrfZ1LHfoT8TYCJadX4bKXsXXkV+zZP0S1j6wltOTzvGZ9WoKjxaS/Fkyw14ZRtNwI5uCRcJ9eIsb2tbhHFohKlMkPZeEV0YqGlQcePU4TDvOGu/9mAwmsrZnUX7PFqY4+VIwZCqJQetpxgVXj94V1XpDzbUavov/jlFvj7KIN0lVcjbP2UyTYxPDhBEW20qlYJJJ2T1tN2OGjQOg7GwZQWODsHG1IeG8Hy41oeSF5oHWhEOU6DdUXKzA776R/L7kCg1qkaBmukGKco3sOffCoEKQwKzts/AQ7ODVO74s3paLxPUVFSuwdbecg3tDVVUVa374gZ2//05zSwsO9vbMXLAAWxsbqv7jzQGoBhwc/l4l7P/t+J+e+27Fv5N9/8b/GgRBoLS0FEdHR67GXWXEiREs/WEMmr71iLl/EcGRUUxc+SkGeUuv+7GRiZNGrXttr+MnxmTRbpWFu3Y0gkTHzYa2tZsd+6YexiTVkhORg0pnvvjdTMLzyj2BSSOFlcPv6PrslHbsfmg3zYubGR4gbiOYBFT2KpRqJXYSsTpmy2INMaWzLbaPzeiPRlmN3j4Ria35RNykauRSv0vIDXJe+sdLNM/UQE9cCp/j8NzZ51j74FpA2m30VKZUcurdU7gCMjcP9o/O4ZPnPuGnwXeePAKRnXk7yDsKaHMr5803P2NlzE8EjQ5i5rqZ2Lja0NqmIOyajgrXKxzvn0CHvpmbGwynB1/mqP1RPCs98a12sgh89i1cy5WgB+mXHcX4nTNoiqmBAaLBXptZy8anjnJmyBmuxl5lUMXxv3VNV369Qk1GDWPfH2uRFNNrdMz9oyf7YTKaiLo7iqi7xYq0mxNdxhvycxGzelgcl5KyqJEcJOZqDNbO5lJvhtwOHlj3AHum7uHSgEukSn4AeirSBj4xEJepibz89XPUu9Tj30uPmb9C5eVKKi9XEv9AvEW5vuz8WeLOnydn8GLAMknkavyMq7GP4F0VireNeZBFFWPPF09+gUSQEJIfgiHM/LpkhXkEZBxHiHuZ6sBqxgWZO4qOgY7oiyrwzUmjJnAgf2c5qs2sJXVNKq6Rrr0m+woT7kLR2YJ/UwtgT0NBA1d/u0rYtDAk5Tom7+tHccwUTDMtHZXTde/i3eLNsdHHsNVEWryDCzYsoCC4gEjXSH7+8efuPmsgyjPY15Zh3VqDbeYhSs/d1Svr8ea572b8PPZnik8Wd7PiboaVgxUKawWZWzIZ+ORAi2Sf3M2JTjVEnPmRuoB+cBMDHUTHpvF6I7uW7mLEGyMY/VZPJXFbdRspa1OIuxLHnql7WDTTPIniEetBxcUK8vbmETBSlA289fz+s1j3gxHv4lp2f5rP0yP9aa0Uk0fZO7KZtE5C6oQXEaRSUKpQ3JIDbUjOo//eDbQ7eKPqaKT80iJ8bmLqzvjam4Fui/nlvl9Qql7l3SGR2HlK6PdQP3RuPmxdug+NlYbiwGJK/avpYmjuWr4LfZue9tp2kp7rvV9gzkATGY67Carqg8nUrZ4EiE7mol8Xkd8nnzPjxRdo2flltFW14RrpinBW/NuUXVE4dv6MwvpZll9a3i0xWWi9hRKbNZwe1o7U1gM7Dztm/CBKWWZma9kzL5T4wDKGn5ZQdy0U43NiwjPxTAtzfnqT/ZP2c37weY6OPUr2CAPvBL5Ne007lSmVpP6YSsy8GPpMMieQAASqw4krWsOpEcvwk4qVszN+nEFreSu2brb4/iEhKvc+Kj0r0Uea4FWYu2Vut6OwoCWVg9o3KXf5hSqNN62VLrSUtuAW5YakUsqCjStodCjCrTSV63ntQE+yT1UjEJQZzcHKZFJPhaNyF+eDmms1FJ8oxo1irpyNZLp1PjOOjKAt2lIy83bo790fgFDnnvKD/U/vR6aQMeHjCZyek4aksIUZO2fQFFCFzsMGOy87pnw1BVsrW+TBchKtEpnfMB99p55dS3dRcqqEVztepS6+nfUua/CvXU5csWVVTVvJIewaxOftKb+RoL4Bub01lQG+1LjX4Ftih771DQTh7Tu+rpLoyXgWnKbwYB6+S81LK3RtOp7/5Hma7Zu5OtUyCK6z1nMl9CMmpwvoap0xGZLI2JCBVC5lwKMD2D4vmWblVcJrNuAePMVsW8EkYK31RCKI38tR3Qe8wCv0mdgHp2An8ow9cmSX+l/C26t/z7aCQPH1YqyCrbCyt7IgHRjaNDjU5nM9YTZTFliWi/gmPUOmvI3ARneMRtGeeqnxpe7etI4aJT7OrVwZMoQZA/ujUqsY8MQA/If509nkgfzIMFzjc/nqxX/yRsxuzj4syoqp7FW0XjiE9koO2kFKbK11cNO67j8qmJq0SnLCc3DRaOnPODN6S5c0po2rDdYuluuKRJCSmpBKblgur0ZZeuQ7ZqzHINdzPeQ6KkcbHANseKVVzJAeWn2NoHwJZQFVVHhvAu4lc0smRr2R2PmxxCa7YUpdzuDlv2F9Qy4zb38eNi42KAM88Z/SSlzfdCrq1GjK3NBrjIRODmXOH3PYs/IEM2xnsGH+fvr56PGTG8zWvPyD+VSlVTHoqUEWc/7Nv+sKCjUVNXH9yHUCRwdikkOndSeHxx9G5ynOy0NfHMrQF0WpQSHEidIWR/Ji1EQEOeARp2bSF5Pw7ueNIEC5ZxUtqnZCN9+Na4QDdVl12Lrb4pvkC6ZM1jjLuNfRhKbDiv1aMdDWlcQE8P7wcfjmU3589UFanHbC04t5sf5FVvuvxjXClZFvjOx1fTZ6KKn1qMKrygtFngxuicUnVuYz6NAkTs18ELmjHXJVE/a+9ijtlAgSaHFoISQ/hDlb7BG8mkhcnojPQB8CRgRQsr6GWVtiyQ0sxavSC2OrnoJDBZgMJlrLW6lfpqDTx8jJ6pEc+XkxkohcmltN9N+ziWbXYPqefB7d/L58m5GK6XgJypsKnErPlrJ2qNgn6ubEUBcK+z+M0foysakVdDZ04hrhyuNZjwOgN+pxrnem2qOatIQSZAoZth5ipa8gCBiNIBUUHBtzjItDS/nSdjV1WXU3EXDEl2HNsjU40YeP23V8YCdWQT9S+CLpsVep9Kq88d4ICIJIfGl0DqHdrkdur9S/FL2hx04d/PRgGvIaWDNwDSNeH9FNHumCc19nvnlcVMuIVFsmZ0KnhLLo4CJ+nfBrr4FTkGLdaU18ejx6/2pqs5w4848zeCV6MSZhPE998RJXYy6QG5aLxMHcGGyJlpAXsoHfAmC7byDXlOYyofLT79BeL+Plh39A4RjN1GAfSkuvddt9XdLbbZVtFu+hX5IfE1dPJHtbdq8VjQ6l2UhsXDgw8QCjIl8keGww7rHuHHj6AN5/+hHoFEi5TzlH4vwwmUpor25H7aXGJdQFu3gXsvO0NDo3ojS4IpFK6P9IfxoLGzFoDJya10pVcz3rvAoIUnXyjVJCU14NfS7+iWN1LjCF5qX7kE+4wPuDD7Er+E+L87sdyi+U01HXQeiU0O5//zLuF6Z+OxWnGU6kJKSQmJoIi0E3W8eiQ4toyGsgYGQACmsFvyp+ZYrvFH6Z+IvFvmetn8Ws9bM48c4J1N5qEpcmdo/dXMG3ZN0S8b19XiQxdSHmYjF/JgmceOMEzz3SO/Hzdti9fDc2LjbELoi1GOszxp/8lBQizv5E3oD5gGUSqvRsKQatgaDRQRZjGnv3brWOrnk2b18ep94Tq4Gz4j3Q2aowyQYhU9sgCD3VO5nDlmOSKfBLdKM8T5yz61OKiT2yDZlRx4Y9om0w6KlBFsk+u0oZ/Ur6ITPKyBucR8hEcx9u6eFH0RxpR6E/TtEMT/yDelgRdl52HJt1iZxoDQn53yAIEDY9DJ+BPnj09UByXsCm3QaNlQZBYkmoi1Aaed0Fvm+GKzox8diUnEXcUZFYdHboWRCgzdgT0Tnz4RlKTpUgGAUePPOgxT678FfkFbWvA2kTX8I7+yhGpbVFsuPS5kKCr4okQkOWJzbfPswzRc90jxslYmJu8frFaFyt4Z2ebfs5j2F8WhX16uNoNG/j2iY+a/cYd5YcX8KBIRcJvrwE3lvH9YxgWvVpyK3GMPOnmTj3cSb953TOfHiGJ/Oe7K68uxlFgUV8vOJjds/YDYBrhCuTPp+EX5IfBYcK+POVP5FbyQmZEGKhzjLjhxnkH8xn1/JdjHhthFkPuo7aDnLfyGW072j8yvw4YH+A2a/Opi6njq8jvmbws4OZ+OnE297TLtzO7/1/C12fvE/OUVSdzVi1mscINU3iu/9X7TZ6g627LQEr5vDnWvDPEPts9iaD+1fY/+R++kzu0+u7aDKYuslIkXdbJqRVNcXYF1+m1SWIsq1t6AcnorBW0FbVhteQYI5UReNReAF1QzGNubXQLwAHPwfGfTCO69/DyfZErNrrsWssxdjmDfTYq/W+fcWEX2UlEGxx7ItfX+TwC4ctkptdcqMXv77IUwVPIVfJaS1v5cDTBxj8rCgrHpTvRXxqPGkJP9Osf5O1Qw9Qk1Ej9ka3kWHqMBJQHEBVoPicfh4ryqTELog1m78FwYTCWkHswlisnKyQW8lxvn4JraonlmgSBBryG9hyzxZArFhcGTGOxMRkjlxfgq3bA3f0nABydvYuBwwgV8oxKByYv2Eycl8ZDDUfL/tzNzP3qEkes4RHl4q2R0tZS3fCPnOQgnb9AV75xys0DOsg4JkAFh9djNpbTT2uDMk5QY73mwRmnMVBLsb+jNqe+TLpXBLnhpzDodkBd+HOK9Ghh+hWm1V7R8m+1Z98wssvvcRohYJnNBo8gaqKCn5+/32uarX8pFCwUG+prncrfra2ZtaCBX/rXP9vx//03Hcr/p3s+zf+j4BDeyKlfqUURHrR196yqsUk1TH85HBCCkIwvmI0Y8+5nVbwxh9v8OuiXykKLMIkmJBKepyjMv96muwKueuTsxhsrwBvdo8Jgig12Gx7yexvXUm+jiNnSdz7J1vmpRNU5017sydwZ8m+juoOsl+UIw3pj/uHoqGefzCfk2+fJHRyKLalMDRvKA3evWf5R5xIxCDTcnWkEcFo4uYWmwo3K/ZM34NHlQd+pX4obM0dRYNKoNmhGfsWe+b+KtA+9hg8MRrfQb4MeHwA69Pj0Vnb0/96BCDgqva7o2sCuPLbFXJ25jD5y8kWLFuAb5/YSJu1qF9vMImL0r7H9uEzyIekLxeQOu4xGtTbmLN5Dh2RjZDUs0B3Jc1GHR9FWF4Awpfm+w7P8iKo/H2+HvUlG3QnUPlO6x5rrWil+c88hjIUhyYnrLz/HnPkyi9XKDhUQPS90RZJMYPOSHSmyPxqcLNDppCha9Nh1BmxcrLCplJCZFUkHtUeeDX1gbfg10m/4hnvybh/jCMt8hh1qiusenMVTdHmgVfnYF9Oji6kxL+E6IxoPCSWjCO9UU+buo2fHvyJMYNf+FvXNW/nPDI3Z2LVS5JQUKpQaVpwqspCMA21SHJKBDlVXlXsuPsi88feEnhVSmlwaSDmagxzts5B529eHaqN6EueXEV+0ElkEg8e7WNe5dZW1YauSOyZEHXqO+CpO76m7B1ildDs3yyT5F0IP7eOlhwVvPYwjQWNHF91HDsvO+xCIsjrPw+JYKI+vQxmmrPxfJMVjDvzEJ8/9Tm5zrkWyb6Td52kXFVOriyXgVHmwaWG/AbCzouSLprYfriE3r7Krje4RLhQmVrJLUWSgOgARN8TTeTsSGzcLOdImQwEiRRBKkOQWM4p2xZuQyKTELcozqLRdFNhE8dePcZ4xlPsX4w02HL7wc8O5tCKQ3wd8TXz98wnbOrfr8bsDZXOUXizjaqsJgDCp4ezUrsSiVRCa7sEo/KGI2Iyn5sBlC5qGryjMShsMMqVyG8JRlcHdlLmVIYgFahw3Uq9fiQgBp0dwj3JGfog12KW3vi10P2sU9ekAqByUNFe297reUuRMXnfZEILbC3mKZNJYO/UvfiX+BN32QtBEBmPXQ62pKCVhVsWUu9Sj9ZOdPhvlurSVq3DQ17O6eEZeBlFR6m5pBmJTIJSbo9n2wIe9vqRWpmS81orTAYTefuv41TbQX7IddL7pt98NgB8Ff5Vt3MLMP5jS+mY8v3ZzF5XhvW0e5Elie9ul0P3luQt7JGQ1jeN+PR4Gm3Em9VFeABw2rGLiRo9RQHFNFpncDHHhlPvniJiVgT9nhlGWLaay4P8+cdL/2COn3nCo3KchO+TVuHQ5MCzW5fiPFysFvg25tvu3/TJsSP4iUzU8SU0enze63PpDfV763nzzTfJ65+H7gGxYqpLzm/CxxNo8GrBpqqTxNREpFUmQpeEsqJiRff2b6W8z9tJ+9k8dzPPlT9H8PhgBKOARCq5iejRu5TPxsU7aJLksCZFjWeZD49O6ukzZ+3lxLVB8eSG5zJvwzxC8/5eZZ/MoMGr4CyVycAtyT5rpQONjo04NTnhXGZZBbGv9EPqVf9i3f0Gxpg+Q24l556t91B9pRqjzkibupMWqxYk1RIEAS5+e5F9j+1j5JsjaQ71ZduCeqqcmkm8nIhCUgiIBI4BTwyg8blG5myew/a7tqP3+oq+483tk90Dd7Ob3Sw6uKi7Oq0LusY2/LIOU9lnGHI7y8CpYO+AY3UK3qfXoBv+GBKJm9na9tQvs3FWGjg7TovC3YnKlEKkMile/bxI/amGPzeN41edLZ19DyIIcNevd9Fc0kyfiX2ofW0t/kUqYk97M/j0UFzHNt241wKB4/qwRD//RuVsGtciJEz2fLz7uDauNjxX8RwKa4UZQ74LAUWe2LSqudzvcnfSLHNrJjauNgSODCQv9DqCtKuXIFz4+gLHXj/GooOL8Bobxdb7nQhJO0a/swK8fS+b54oKAB6xHoRFRjA24nfyTg/FvsAbPoHfp/yOU7ATCy88hX2iFP/wUub/awzLzwyh475WHIY7dvfGSySRrQMuM7ZIQFflhUFn5C3Ju4DY/6riUgVqLzV9F5vLxZqMJjwKklHXF9EWNwQIJGtbFodWHGLqt1PRJ9jxvYfIxnbTiwS+nQ/upDq9mocuP8RJ4QP+jBfngVr9ApxDfsPG1YarG64yfOUoNs3bwiNSFe5vrkBuG0H+gXzaa9rxHeyL4sfz5Hz9Olcf3caOH2fj4RkM70HgqEDmbJrDD5+9x1Rnscr5sswJW1ux73X+gXz07XqGvjiU8Bm99/AxeihYf/96QorHcd/oFRbj6f1TqPY7R5szGPWv4zvYF0OngaxtWQwa7ItTgxNKnRKdtRqZSoFziHM3OcfQqsGmqQKj3Sj2zDfw4ehBDFk8ktJzpVRerkRvGkr65W/ZG1aMTdo69IpspBGJNHpGomqvx6a1hpayAVjHTKWooYx+Pj02hkT213JiDh2JtPlEkuGpwSnYnvbadiouVuAe647KQ8VTXz5FRnQGRyflIreSk7c3j6OvHcUrwYusIxcYcOgEddLBSBzFhNvMn2Z2HzP+T19iOmayc9ZOdILeLCClcrRGH7AFZ1k2C357B/+SAISXwT3Ok/wB86n0Nq+o76qI2b54Ox59PWirFOev3mR/byba3VpJA5CsTeaV6yuZ2H8crlGibX3+i/Oo7FXEL4ln8rZR2DQpeO/V9/jcayW110TfJf7BeJytXLHW+xNUImXkyZHULjBPSggIdJrggzoV5wxXUcp/A3qCt1nReUg0jdgKI5G0B5rJlQLM2zWPmqs1vZIDQEx0Dn66p//it7HfUpNRw2ua18ib8hSHAwbT4JDCUA/RHgybFkbN1RoqdpWyhCU0OTRR7VFN/ex6Ln56jry9ecTMi+Fc8xbSgj5gwsEJxGTEoX1Ry6V/XeLClxd4ruI50q0/p1Odx8/1tjTWeBAaZUDT3ol9TU9vvrkxeYQ5N/O1zoRJeufa0yfePkHe3jyWX1wu9tbr48yY98bgM9AHrY2WXTN3ick+8QYTMj6EkPEh3fcbIObzGL5f8T0vN73c6zGSVyfjleBlluxrKW1h5PGR5PfJ53K/y4R6iutlzL0xbJ23FYD8KA86bDpo92u3qLz+j+DdzxulveW8D+A2OoY9Z51xqsxEa2NJLjj7yVkOP38YhwAHs8QRgK5Vi//VP2l1CaDROxrjjVdo72N7aS5uRmdtYNOsRwGYdklAcUsOSNnZTJtLQJcaqCi9rJSht1KjahL7Pccvie+1WswnxZrhyaJkZF18HdySzzGct0Khb+di/4uEW4/g5hLoft79mCX/lsMuv3JFuQ9BeILdy3ZTmVrJSw0v4XHcmhf3vohtRAGX0MNb5vuelZKAt08rK/Uy/C6GoB2tAWlP3CfpbBLuMflE17tzJTcI/Upjt3pG4JhANs3exOBnBhMwwlKOSdemo7mkGTsvO6ydzL+9LvurImKM2b+70OIWzNbZh/ArdcTfyjJgvWDtCNpsY1C3qhFszDc2VHcSlnIB1wFXsL0ykIbyHj9s0+xNRJ5zp8GpAefUITS+/AExqkm8r3ihW1GhqaiJyNmRvcoJNuc288i/HuH8oPNwwxxqq26jNrOWgBEBxC6MJWBkAD/0/+G2SabqK9Wkrkml3/J+lJwpYfui7Yz9x1gGPTWITUs30WnoxL/En9hI0S7L2JABiJV//yei69lVByfR7uBNdXCS2fMMnRLKy80vW0iu3wn0jW1Yt7STMfJxvl6txSPOsrfvX2HB3gW9xmQAlLZKXte93usYQFP/8Vz3G4V37nGuf30J7cpoFNYKzn58lnOfnMMmdDjFsVOxbSpHHdQTT0pdm0rz75UY1VOIPPRPAHRJ93FzUk+u68ClIgOhzh8Itnj/z39xHhAl/2+9noLDBZScLsHKSbwuO087pv8wHSsnK7F9xPcpjDk6hml7ptH4TH53gunoq0fps3U4LxxchFOjE3GXYzDqjIx5bwytla1EzY0i9N1Iks4mEXs1FoOtA/JP5cz+tSf+s3duDZ3SbNZc9MG31ImHZ3ZiFWXFwCcHcmldOqZWLXmmUDzai1lXOoQkXQe2N6RdpQYdqs1/UHHvGEvJbEQpW+8B3kTMtJRhlEgkKAQ/6l3q8bb1txgvN2Ugta6k2aYYd2uxlcrUf02lOr0aXZuOAPk/Oeu8iJSEFNwDhiJTyLoJH+XnOnArzqRWISUmIwbDjTm6i9gIMPHQRC4MvMDuuwt40W2dxfH/Ck/mP0n1lerbKlDdjNWffMIHr73GSaORQUZzO2ahRsM+YIZez3kgAFgD7ASaAQfEbgnLgGLguE7H+mXL/ta5/hv/Nfw72fdv/B+Bu/4YTIdNHTkJ/ihdzI1sXYeWsLwEAooDcKt1s2icrXb3pzDkMlP3TsWp0QnjKuMtTEjx91djryCVm7OZ2ipamLIrkfS4Dqw7rUFivoDJ3Zxocg+myvUzflyqYZDkmTu+poL0Vuq3nqDN6Tq506YQNtwTtbcauZWcjvoOHAolJBwZT3piJyqrEsB8obgwQsCu9jpjtxxGL7cGeiQUu+Q1qj2rWb9kPctczBsFVQ42cSTxR+xa7Sj2z8POzgkYzdmPzyKRSOhwFBc098YhKDQtyDV3XlZcnV5N5uZM1D5qJq2e1MsvxF4SPuU+CC46Ouo6SFyeiN9QPzRoqXa7irpWTcy1GLTN5s2BHU8auO/SfaQmpJITnsMy4U2z8ZCrmTQ71LJndiVp4yqJ9+t5XgGjArkwcQ6uuR8RkxmFIf8X4M4TY8NXDqfgUAGZmzMtkn0SKzlvvvkm0RnR+DWJXskHapG1vOz8Mvwuqhlw7l6qPKpwbhSNq4KDBRQcLGDsB2O7ndRN925isJP5Odl4OnA1vppahxru3XQvUrm583H9yHWu7r+OUqVEp9LhZWXJ/PwrhM8Iv21QSx/Xj/ZrRfhm/4nJmIRMal5lkXQoh/hjy9m08BLKW3xBo9aIfbM9tW617J62m/tCEs3GdTaOlPk3cyXkWew74hEE8+rRtcPWYigXZU57c4D/CpGzI3EJd8EpqPftnCoyEaRyVAPF4KRRb8Q10hWnICfyDLUk9/uJKVttKf/cDt541mzbujAdh60P02nd2duuyRyQSV2HGERscjJPcN4s3WZ19TKCKf5vXdf076Yz/TvLfgwgEgU2TNvAzJ9mdjt/N0N/4iwxx4+TPeQBOh0sg2IDV46nRaNkyrOW74JrpCuTvp/EgYcOYKWxwiSYB812PrCTtHVpeA/wRiqTmrE+/6sQJFKyhzyAnaf4LUukkm4yh2ASsG6pwa34EvZ1hbRW3I+9T8/6YNvHi+uJPVWwrrfY4uem1pBrfRTrDmsUbRdora+l9JyRHffvoLzeilCpFYo4BWP/HEtkVhTGp42ADK9+Xsit5Nh52nH5X5eJvz/e4rzVGScYdGE8ta4tFuxfg9FESr8UIrMiCbxmJ0onnS4h/Zd0MXjXoiewKJCdM3fiqw7EoDFw9PWj+Az0IXpuNAPPhoMpiIzYDARBoOZaTXfS654rr2LTVMOfuXfz0uLH2TXhCBLZappLmnFolrFzxmk01qLz7VzvjE+NM7o2HYnLE2mtaMVnkA/OIc7dfTduhrWHPQ2ekQw/7Ytjgxe8KlbRHl91HMcgR5oKmzg54iQ77trBaKnojJacKUHXpqPPxD4gkaLSqBh0YRAm32YCHonkFKdQOahw6+tFyqRXuBr4JBprDaZb/OuuOXK6ZzvOz6wjXTkTgOUXl9Ne087kbc+jczpGiY8WnaDlhd4/z17RXikmbEMvhaJt1aK0U5KwLKEnoGeEvNA8Ds1JZMY4sVTm2uZr2LjYEDQmiDNnINcmHlWiH1aOVox8fWR3RbGyyMSws8NwaGnBs3EbMJu3JOIasUpYhVFmxCAz4Jrvw5xN89D5NZhf9w1W+4mRJ0hJvMpT3FllnyCAVVsdZRHjmPlgosW4TKFi670peJZ14mL1ksW4+zEpTx95mi+f+BKtWnRac3fnkrYujSHPD8G2WQUdTgzddxFNsY7KSDGoX5VSRX0gFHl8AcDg5JlYCT2EnUPPidI6Mddi2DVjF7baPthgbl+oQ9S0FrSi9rEMqCpcHMgZfB8gQdPYyc2sY4AOQzWpYe/QJPfBR6nAZDBx7tNzKGwUYhV8lIRI12voHcSAVMWlCs5/fp7YBbHYRfmTOWw5hoZFLNhTT1xECH0m9iRLMqa1s3baBwzOWk5goT8OHWL1q8lgYsu09YyMGcnWOWJguMppu9l3L5VJ/zJAHH01iKjMAHLCc7r7xOxYvAOnECcevfIoEiTEpyQwZd8UygMLsLdXoWnU0FTYRHOwHS22uTQ7NNPkpEUQYNDTgzj/+Xnqc+uReI9jaMITeAZUkVfk0x2gabzeSLOmmQd0B0ksVfH50HzWq/Uo1Obvw9mkswS7tdDn0BjUFxMwfGboHtN3ignIypRKy2Sf3oRf1mEA2lNdgUCaS8Q10d7XHlOFeCITD0xEKRHvc31uPZUplXTUd+B2rJaHch/iUv9L2NuIz/napmvk7c1j2GujGGJt5CvfNjr/9SFf1TURfU8bnvGeOAU7YXKW4jrwCrskdlT5DELt5wiICWfHQEecr45kfOQjHG6JZ+OD7xMonQE8z7aF2wDobOgkdW0qEbMiLKTITIJAp00nFb7NWLtaEtrqQrRUSapYuhYajOdhfhIqBxUKWwUdNe08/cXTnB94ntOTghme4E7NtRq2zt/K4GcGYzsoirMz5iERFATV+uBqBbl7znPg6QNixXxJMlH2HjQM/YXSF3Np079FePAMCvrfg31tAWHnf0Xa2crSxAWcchyN1FGUj/9hwA9IpBJeN67qrS0YANLWs3QortHmFoZMOZKyc2VsnLmRmPkxzPplFumJzeQFZSM1ifZDV5W9jZsN+U1XMFFPWEEcXp0zMWgNOAQ4YDKIH0FoaRzGhjqc60/hYLLDyslK7IsqgeYboiwOikYiwsrIUIo9VytvqM7enKSbuWMm4XlR8I4YTL7yyxX63t+XFVUrLAKMAK2loqxk0rkk9E4muKUgK6+8jpTqi6RMu8izz4m23pmPzmDvay/aUYIEQSLKdCKTETIhhGUXluEc4ozMzpprIx/jutPTZIeeICLEXKbZNl8gLGUi7SPSWJQmZY/zBbqSfb9M+IWAInfqlm9ncbWGC1nHyE0bQUhITzAxfHo44dN7t81T1qaw/9P9DP9uOCOGivJgNRliwlowCmR6fEqDg9j32mA00lHfQdCYIMKmh7F2iFjd2W7bTkhBCJ0tGnwG+ZCzM4e8/Xk0XE5nzLkADHIDLfat4v62ZOLR1wMrBytCr/mhsJLydt8Gvln9KMa7W7B/LICUqa/TLmwhI+AxttX5ckap5zedPYZ2Q6/X0BuSnksSk8grj7LowCKai5u5+vtV3KLcUAaJTkZhYCGB9YEo7ZQ0FDRQeraUoDFB6M/oGXd4HJpwDX1H9bXYd/nFctqr21l+YblFpU1HWQejj49GY6XhzLAzaL0tlQFODvoAnVLHExFPoGvT9ZoAux1sPWy58ssVxrwzxiKIbzJBh6M3HY7eePSSE0hfL5Kzxn9kScDSd+hxL76Ie/FFslV23T24wmeGc+GLCyg75cj1cgxyQ/exACpDhuBReJ4+l/8gZ9B9KM+V4FUHgjAKx2hfsoctQ9HZwvffg2uwpawjQHnfDjJdjtNu284nkz6xGM+d+AQ7AoPoVBWx2N88OdFc2oz95is4RFyhyV0BPEHQuCBs3GwwaAy0eevpN/YiZXl+eDVbJrtl+0dyObgSV59aHC73Rd+hwzYhjEbPCJyqspl4aCIzAjspKghCea4vRp0JtbeazvpOSs+UUnSsCJdwl16TfXse2cPV364yY+0MEh5IMBvTtOhwrCxAY+eCRu1uYdvrbJy4GneWq3EwuHMQbdVtfOIp3ptVwirq3Fpota1l/ZL1+BnMm/rqmjpwK0lh7JAThC3N4/VSL+AD6nPryd6eTZujli+e/gLn1htKUDds4hNvnyBzcybOoc5kb89m1s+zLB+WBCSCBKdGJ7SNWggU19rL312mz6Q+eMZ7Yudhx+QvJ/dqd5189yRSuZSXGl9CqVZSek5MnBb+Wciwl4aR7ZeNgEBRUBGvxooEnYARASQsS2DoC0Mt9ncrBEGgs6ETfdt/XPnz34UuO6jJM4Imzwizv4Eol9le046dp93f+tabS5vJWvoZ0UD62GctyLN3gqDRQZScKaH4ZHGv7+hfQVAoMaiUVAcnMfbpmG77pYus7XE9GZ2VA7ZN5XTUeAG2lJ4rZdfSXQD4BovOl15pi+wWOUuTTEFB4t1EDYqCc5bJbpW9Cr+hfhaV6CffPcnFry/S/9H+7Hl4D9O+m4a1kzUVlyu4/K/LPF/zPMvsX8OqvpmhZ4bi5zSB8FnhFB4ViYL5d59C/qCcPvl9SExNxKAxMPzVnsKKoPxAJh6aSLN9M022luSSXN8NdChLCMkPY87WOeh9W7G7z47JX0zmz/71fFJ4H67N4/HbE0D0tZPkXOuPm6v47ds2lSEvKmDPQx08dPkhi33Hzo8ldr4l8RBA19LJzG39qPPtyyOPT7AYbx4i44+w73j6i1f48/n9zPh2MplbMjn/2XnCpochCEr0SoFdM3cxVz2qW7p0xMoRtFfKCLi6lyallG8e+4YwnagcU5UmCma2uxgp8shGkAjcrOB2p7iZhPZXqKqq4uWXXhITfbf5zRTgCWAUIr14NPAMiNV/iK3s3wGUMhn/+OgjPD3//jfzb/zn8e9k37/xvwq1WjQ6TDItgkRAq6iymLAysi4zb8NETow4wa/3/cpH/w97fxkf1dm2fcP/cYm7u0KEENzd3VpoobS0pUKFunvPKm1pKTVqUAdKgeLuEAghISTE3T2ZyPh6PyxIGCbt2fN93+u+nt/9nNsnyDFrZq2ZtY5jP/Zt37dNZevDljh4KWtbVahr1uFf5Y/ZakYh68kk3vrTMKySfmxYuQFPvW0njlGnJybXn2pfb5LSk+jSGm26R7QD+pBXFUSThyhrZ/oPJtMGhYZNd+zmjk0zOPzhHqJHrUChVaBwEBNU5f1aORT0Oys3rKTF6zhwkwSY8wQyfX/BozmQAe62C2tXRQcLtywkMyGT3Nhchmlvlg8TT7TdqZ0fl/3ITHU8AE7+TkgkEpQ1zbRp81Hq00k+Vk3t2Hkw3t7wvDdMfHciebvyKNxXCB/Zj0fm+SOxaJizcw50WKmKrOLsB2fxjPWkpqmIPM9FdAV1cWb4GfZEP2RzrLxZIKAygO1zt2NW2RM52+fuReeYTV0nlBvU3NhNYTJLMahVBJcFY5KbyB74n5nV+vX349btt/bajWW5Jr3W7NpGclEfLCYLQcODKD9TjlQuJWLoPZyWHGPPmLdwlfblX7zBoIcGkb87HwSY+kcyqo5+ZI5eQHSwrSSawWykTZN+7fq2EygZ3j0mCAJZm7PI3pDGtPFZeFTl0eZQDmPs5c16g2AVSN+Yjlu4G6FjQnt9TV3YEJr9+vRq2l3pdwWXJinz/nyQlsByGNfTAdqZ1szjHz3Ojtk7uDjwIst9bf3UxOIfK8GlwQw/E0tW2FkSE3skEZPuTOL8KSOnheGYFf9c9x9EeRKzwUxDTgOBQ+118n2KziAzG1CNEauiO2o7aLjagMVkobgzm0K/dzk8ri8JxfYb2PrQTjJCTrP0h6VoO7UINxUP31hs0NRsuxPUemrJGX4XsWdEbxCZ6q+Nv6/PfTfC2G7E1GVC66G1+z2cA52JnReLxWhB36pH7WKbUJC6udDuHoJVbvs7XMe7e8RgNXACJN70qKtd1PS9oy/zL8xn5YaVdK7thF97xr3ivQgbH8aiLYv+I4+GfwK1rh5NWy1GbzHp1dXcRconKQQNC8I5IYS4E6IPVqeTt42PG4gbEQGBrKDVqMzeCMILtuPXkoihJaHcuvlW9M41SOI8MAoKNE1VaICAygD0aj06Jx3XGpFZmSoG/Dvu2oHRpXc/mvicwbQ7tHN22mB70/drZOnuGbsJrl3MA8B3o8R7os+8PoQPmckD3uImy7/UiNlg5uyaswDECXEcmpZGu6KAxb8sxuiiwjRb3CB7xXnRUlpH31Nfc9moQuIFUquMfwkw8L6BPHy8jZIwUb7No208iRkCY04k0/ZqW3ciqfRkKd+P/p4pa6fYdA0AmGOl7Jh/gjm/eWLJFwnDH6f8SFtFG1M/mcqb5j9pahPJKk9BlJf5bqR4Xa8Ir7Dv1nTSlR+i7dTyWOifREyK6JaTa26xYlSawGrAt9oXqYPtl6ZoEoivCeWbkZUog0o4XCkm4/yS/cjYlAHGEvr56DAZ5dxfrMHB558nGDNHX+Gzx9aj7dTyoqcou3tdFhVgzroxmK1DuTBZiUQuo6W0ha23bMUjxoN7Uu/h0LqvSA9cT12AGwrtOVK/SKXoYBFzN85FlQ8TD0/EKrFiUlwBbLuNPWpdcJH4cW98Ffny3/GI7dnAdpQ3EX8ulw7zbNITdlJrbf6POvt8C8+g0dUhd+o92RJXvQvkAo7SXpLk4RaKhp9DaVQiazRgNVsJnxxO1Mwo5Bo5i34Zj0k2AKO8jjYuM/PzF5j2yTRkShmvr93PwAsDKQktYcuiLURIxKRW1mZRls4S5MmP4z7ApDARcGUDreZouK9Hfm/O/jmEhoYik9nPjzK1AqPGlYSj6yjcNIJh4yfajLfs/4IZqWN4/8n3mez2PYY2A4eeOQSIktfyASFEuvzOybfKqaz5iplrFnP196tUXqik0KsS3egn6Z8TgvJ4JJ25t9BUICHz58xrG2+B8Q5weM4GNkc+T1uYuBY35ol+cAlXEqjxraE4rJiAygBM/ToBsTrYYrKwa+UuMjZl8FDeQ3ab6PPD8pBaunj8o8epXFBJ8Ihg/JL9cA5yRhBg5Zd341PrTX5UPgoXNeGDxWrbstNlpJsuYdXv5NzQBrL7O/MsMO6NcQx+eDCOPo7UHlHy5ontTIv6nK6qNxAEmPrJVJQOSrp0ndzx7SgaEzyov+0oRR4VyB3VpH4pqlm03BvAgYADHA2A8JkXeFgzC5lazpO1T1J7uRbPWE8sJovdeiP+WFLyBy5GYWhn5ESRGE26MwnveG8ChwaSsEHBgj8/ok96Iy0eYndW9Mxoyk+X01LSgtwgQdupZfqe6TQEGSg8WEjJsRImvS/OVcEHx3AkopGrrvMRPJS4RbixZ9UeqtOqMYRoeGj6NpKN45k8To3g0lPpkb4xna5tfnw2YR0PDn2YZ83we6vtfLH7gd2A6IV785omye3gjl/v4PSI02TUpOHnZ0umt1gr6HDoICMxgxl+t9NU0ETktEiiZ0bTqZVzasQpyoPKyY2ZREH7MSRrW8WOq4tVFIzVcyRxON4NcUw+u4Wu2gCkbQYkUkk3Uar0kbIh+AwEw7Sr0u71pcHLwsXbDvJ+4gb8VZXU+WXQ2nUv0JeqVFElYflycX1/6lpNWdmpMlrLWkm4LQG//D/wrFPx3d1fYzGe6fZxbS1tRSaT0Ri+gtAqT0akqDG8ZkAql6JyUaFyUmEarGadyzpu2fIQAVn7MOv7se32bRhaDTxS+AjWJas4VXaSxb/WIRdUSD6VIJGLMUxnYyehGQcZOGwPy287wcoqDYIAP31UR1BWGnVK4XrDPW3ObbR6iGvu853Ps2vlLtK/T2fKR1N6jVGb81uZvleMqbuuNSh0NnbSUtKCe6Q7RV828dKvL/HDsh8wWMQ15Zbfb+n2bE1f0EqhaQ/+lf6Y5K1o3F1w1jtjtViRXPveOx0FskecISTQVkJUUykw7Nww2vuWcvXAMBwH93RQFx0sIhw/wtUQb24lp1hFfe1FnJx6yL3GvEbkarmdvyfAmdwzmLPMrLtvHYNODELjruGOw3eIpLJWgaX9HJFtkUw8NBHF6BqqIqv4aepPTP9sOoO3j+PWEzPROYtk7co+z6MxGXCLcMNqsqKugvDL/fhy5Zc0eHfxhXY9HXUdhE8MR6FVMPJIfwyaKD6M3ULD5CN4x9/SvS51OjvT5iJH0h7El3t9aN81ktiEDhhkdwm9Imy8aOvg6CuS6IIgIFgEBEHAqDMy//f5mBQmDHeKv9WGQRvQN+tZvHMxzledGXl6JGsfXYvfQj8G5Qzi4oaLxN0SR+CQQE69fYrcHbm8bHnZ7nP7DO/DN59+Q2ZpJgDfzhYJ0bzded2v8an1odGjkaJRRexcuZOFXy60e5+/gtZLi1uEW68N/rW7UwlLK6W032xCQmyrnE6/d5qJ707EJ9GnVy81lbuW0vjp+BaIPnDX54IJb00gcWkiXw/+mhf/9SK7ZuwiMvUXOium4ObmTmWfSTQE9Sfh2HoCrx5EobDgbhCAsd35DZPGGYWzic7GTjRuGrvnq8XfyEU/cZ1QBdnvKxqUqXSpSgAw3uRtrW/W45lhZnHGYgxKC8JjoHHXkPVbFsn3JNMUayRo2G6iJLCqRMFH2Pqnf3P7LxhUncgUCajDipns5Yq1EVp8onGryaHJw8IvQfksj8vn9QEXeF79MjM+m0FFSgUWg4WSoyWET7SXIQS48qvYkdYbSdNRoyPyoigVWhM2FKvVtp3RSs86YkSHUWe7P9g3K5VWTUb3q29ErW89P62op9jPzAyXRtK6xDW+IVcsHr0yrgofGZzsm0rV1XDebFChb9F3+0rG3xaPTCmz8Zu+DscIR3bN3MXyTctp3NUI/cE1xBUnfycMOgOGLgPtHe3ErIjBRWs/36RtSMPBx4GRz44EIHFpIipnVXenk20HtUBFSgWV5ytt4ui/Q2VKJd8M+4b+z/S/2WHifwzX5yyZsQvn+kK6nH0QhJ6C/5w/cti+fDtTPprC0NVD/+Jd7KFvFvdGVokUh5ZKdt1/guFPDu9VWvXvsPv+3VhMFh7Ksc2DWUwWig8Xs/3O7STfk8z4N22LTGQdbajb9OidPHFJdOe6Fa13vHg/ySwmQq9JzXZWxwI+dNaLiiKK2HB8c0Tp+qwx9xPoalvM5F1yHo/KTJgXS28E0oNXHuz1WmrSa2ivaafuSh3ZW7KZ8uEUNG4aomdEo3RQcuSFIwQ1eXMxIZOykDKmJL1MnyR/fBJ96GzoZMutW4nNieVKYg0LZt6OQqugq6mLhpwG3KPc8ZqSzHdtpzkf/ynBXXegb9HzrpvoLf1k3ZM4tmmRqhz5Osqf0kXt9JnaUzRtFQSedgMHTSlHLoUSVB6EqcNE3rW6S51nOB1PvcLK93q7MlE1bN+j+4idH2tH+pkEE83uHVR5n8ZgGofGzt5E9BQ2KJsp6yzDrDdjaDMw+JHBojXG0R9Y0hnBpw/7okBLc1EzeX/mUXaqjPDfxvPzbdup8yrBo2MBnkoxf+0c6Mzwp4eztmk/ewJFhQ+32koqa38HbpYp/2ucWXMGt3C3Xr1Lb8TXGzYwTqGw6+i7GSGACtgPdqTg7UAKIin4/1b0lvP7P4X/kn3/xf8aZDIZERGiPMfhGQZcarIYfjQS/bAGoKfaRO6uZPf03VT5V/X6PibBwKXw27o70b8XvrcZNzpE0KkSA/obAxYApwgPXntZrMK/2ucqCouX3eJ2vfLerckNzV9IdfUGs8JMSchFts8xsWjMcgDiF8cTtygOmUqG4YxAlaaKLQu3EFf5u/0bCDJqfWvZNyONwb6hNkOWNhPxWfF0abrwr/LHMKIc6CFiEoqHQs4iTo7YiUFt6CYo4m6N4/KmyyQe+YSqIPhj1nr0lgTiA279x9clkUi4bddf6y3P3DmCyoAKrg6IYenEUXjFOTLzq5kEjwymOC2PR9Y9wpFxRzgx5oTdd107S8r6Ee8AoDV42o1fjTmEgwTero/gz80LcYrRwrUCz84mPR71Aq3uKlL77yF9UBnQu8xLbzB1mvBL9sMlyD4Q9pT7cMuBLAJzzuDctAVT5zM2ngCy41BT44NRZcRiETcD09dNh2vyfiaFBanChESQ2F1T0YV07t4wn2Njj5Edl42WnpaV85+eJ21DGiM/WcTl7R4E51ygKUUK/DOyz2qxsnOFWM216uoqPGNtq7ikpcU4NVZQHzywV1+QXbP24F+uZfmmINoLarjxHpP5qjg79Cx13teqjW+6MMWFnUxPu8TRsWFE50XTVW/ruTn6xdGcHbyNpAvvc3H6i8BfE2O9Yc+qPTQXNvNEtb3MVln8dCRWC97XTilxWSLRM6NROCgwv5PCc+89x57pe6gOtScKrdc2aC2uLaKfxE2/1+1rbqcwpBCLzEJYSRTcZI1zfVMOYjLn5i5RsJ37bsQnkZ/QUdvBk3VP2lWx+/bzJX5JPFtv2YrKRUX8rfE244qkOArqY/EqTUXv4AHYerIJCEiMNRx86DzyR6Ns5BfNVjO3bL2FRo9GCiIK6Ovf1+bY4U8MJ/PHTDYM3sAjBf9cbvWfwKmxhJCsvZR5uAPOVKdVc/zV4wCsqn2J3ERfWt3kmF3n4+hrGyx1FNYQkHWYBnMK2k4tLSWr8Ix07R5POuaOi+dAisKL2DdlH7eFTiFwSCADv76fr54RuySC6hM4Nu5PMgdIWHOto7AipQKL0UL07OheZT0Adt+SQb3yNHGVm+zuESkyHv7kCQois7k6QIEgwOiXR1N5rpKwCWE4bZah1Ucy9nA04XUXUTkNInllMk3Xdh81/s20qCvwbPCkDTMBgwK6SbOTJ8o5OPEg9cHlxJcHo6zzwmwUOxJvlHp2bx+Ok2wO9eN13Yk1ECViEm5P6FUSrVFfT6n3er68z4/XnMX1tt+d/ahMqWTIw0PwnreLR44/QnaSkviYsT2fFSVucvOVv2GVWWl3aseqtH2eL5dkcNlnJq7Nrty55X5qb7H9bM/zAv0O30lr7Hq2OtbTahZ3sI35jey4awfLmM+vzxXw3KVIQn6eT8sDnfxTaPWRtLm0oRZGd3vEWc1WBEFAppBRHlODqUNHB9/SUNcP+bXkS+S0SAqqC1BsayZqYBS1gWJF5fHXjtNe086ML2bQNlBgm/NXdGo7Ca97ibew9cuasHsQTroRPP75hxCeyXrHnqSZsbkDv9IiEpzvJz1hJwKW/6g6syp6LJr2eurLuiDelrDQ6ZoJuPwJtQES8LOXBGrtY+Wg/0FWfrkS944GDO8Y2HbbNpLvTabvgr60+0+j3G0r+yatZZzwBlK5tDvJJFR0MHP3TLbP2U56/3Q8JWLyQ66REzAkgLLEURQHiAkM5/oCWuQdiLWefz33XYcgCFjkKmrChzEwyb7iuSqwjjpzJWa5GbNgRq6REz4xXPTCFMBgdkCpNmANLcfgHIxMKaMqtQr/wf40ORn4IzyFTq8rPFe1iyGxAWwc9w1tFW04BTihbZDiVBOK1a+MW+Le4uvOfwHw09Sfuj/fu84buVkudonMa+c62aer1JH+fbp4jQr7tazJo4OUISnonHQ87CsaLN91UvQNMZuhyb2Z8qAyds/czfOx36N2FT2oPPt4cvGX3dz97d38euuvNPmJnnoqJ1W3T4rh0jfU59Xw9Ih30VrEgGjIw+J2O/NqJWqpgKcMpleCxSrlfUEs/IqcFkmquQO3JjdKfZoZG1lFbEk0UrkMBzcHwieGY9absZgsvVa+S6RSWn1F8kJ2Lc/lm+SLb5KYRLXkN5BwsZVNy36mJVDDOqDvwr54x3vjHuFO5uQmMqZ+T3hhON6ykQhWQfQVi/ZAECDi2Bh+KK7i8ITN9EeBRPIc5WfLcfBxAD+BR11hrdcRiDvCsqKJwEHOrz/P3of2ogYqDQEwFCaeH0JWhysAU9ZOYf/q/YB4v/bWpe7Q5UJ4cTjhxeFUBRdBf1uyL2l3AO5Fi/ls1Wcsit5G/dVcjjx/BCd/JzymRnJo0iHcG90ZeXIkFkcdYePDaCluYejqoVw9lEHs1Vic2xyIvLiZqrRp5L9+FATR0+qdfocY5/QlglXCe6UOSLrMdBVWEZyZTr2vEpmmmVShnfpD/Sn8YzQRz4hzVeKyRKqrJbTu/5Hi057wlKi6kfJxCtlbs0m4LYFD4/9g8oHJrPxyOg2rGvDq6yX6w1zrSHXWxxNcWYpX5UUsRgtTP5rard5hvebvkja4BMHtfpQOSmou1XR/J4IAEiScH3weB8Ebi9HCnyv/xCnAiYhFA/AsT+dgQCt7q+Fkh5T3BLDWN+JTnEKo52gux27EomymK+AFVEkTup+jqZ9MJWJKBGu81zDjixk20owAHvG+bF58ijZXMzvuPQpA3p957LhrB7ftvg2Tm4T8qHyGpAwh+/tsAh8J5I+lf6BwUHB/+v0sErbwZVOoWHQ55RgW00w+DPiQyGmRxH84kAbTPVgMpeI1Wm0n54bBAqV9PuKpwE7W+nyIztKzJ3hFeIXZQ25H9c08mpYfJPST16mSv2sz930a8ykgymndXBxQOqGUS2cuMerUKJGMcdcQNr5H1WPW1jiMijDMcjOC2YJXXy+cApzI+D4Dn8ciMCp7SAizxULE5Iju2K2uyMq3iW/i3DWUYbmfIlXIeNksEmSfxX+Ga4sT38zfzCbfNuKXnmCNahmm1k6c66oIzjQz7Zf7KBs9n5OOqaj6dzInyjYW/Xe4USnBNcQVtauajroO1AY1iZnXKtHywfSuifAJ4WRvzcZ/oD/Lhy9n9+LdtJ1sY3vOdgYVD+Lch+douNrA7XtuZ8gjQ4idG4uuWodUJrXxIVI6KLGGWels6GTWzlmkF6WTsDGBqgs9eYWEzATyovNIHZDKqEH/zK7jOixGC4lLE3stiujIrcSj6gqqrlasXmO5nqxoq2jj0DOH8Irz+sskOhIp9aGDqA8V2dTrZJ9EKqG5sLn7ZY7tjjjXF2LSiXuWyPM/Y5GrqA9ORu/ggefk/uTkSBgjgL6yEd+Cqxi0bqz1Ej2tnqh5ws6WY5TkGS4KX4vXZ7VP9JZaHmD46eH4V/nT4dUMY3rGqjyrePOFNxl1chRBlUkIAvgk+jD08aE4BzljTReYWQXWv4h3TC6xCIKRIdW/YdV5oZGBvqyO4Ct7MWhdOTaphKuWDtY0g8xJjoCE7N/Fjpmo6VG8ZH4JyV+0ON99RpTu7y0G1ng7UtR/PuGXtuHcUGQXj3lVXOSZQ8/Q6tJKR6ACt7fdum0HRIgHTDzxGhFutvdQtaGU/OD1NHQMpulPA/Ee4mvd+4cS8eEDHD39BLfnxRMVfoUt6xYTG6qDN0Qp7dj5sfgk+tBW3kZXYxdO/rZ7IQGBFtcWLk64yN2TxeuTKqQoHZVIZVJ+fOVHKt6v4MRzJzj8lr3H5rjd49A16ig8X0hgbCBaD21312NDTgOvvvoqOTE5aDu1NLo1suubXdRerqU+q56oGVHEL/77OaA2sxYALdpei7z+J3CjZ593aSo14cMQhJ7uq+sKMqUnSv8jss87wZuELS/x3fcwcPcbXLwoqg39J2Tfm6o3sZgsLNq8yG7M0Gbgp2lizNlW0WY37pp+nMC8NNHP3qJCECRIJBLCJ4UTMDaKymP53a91TxIVw2Jmx/CK8ArrP+gi84szyA0dWKUKu/u7Iag/FrmaoHYd4GY3XrC/gEvfXGLMK2Ns5B+vz0Xt1e08Uf0Ejr6OomferVtJvCORtA1pzGIU+cEXaHNpwypYObf2HGfXnGXhbwvBIjDizAjMyksUhkQilYuefTXpNczdNBepR1/qfE2YlCasXVabtVgqk7L82zk0eNYx9u3vIPIYWUHv01rWKiombC8CXuW+tz/j5XsP8OOtB9BGLuPHf/2z36pgfwHZW7NRu6ntO/wcJPy0eC1J6UlUnF+O600FBqoSC0OzhrJx+UYWD72TyvOVpH+bDsC0j6dREHQKh1YDd/z8PLEjB5P0XT8K9hUQMjqEEg3kRaejMQQz8cxH+AeI+y/nAGfOvHeGwThx9Y4wisOLmXhoIhrDJf4Tsu/gU6Iqx7JDywif0HthBMCOn39mtf7vpXprELOtJ7An+q5jCLDHYmH000+z5Lbb/l/V3ffv9r3/0/gv2fdf/K/BarVSV1eHt7c3amMAzm0F+FQpMXfYymvInBRcGHwB/0p/+mb1xWq22lQ2Ga5WMu3wNM4NPUezezMmi61MQH3sw9S67GHKvnxkUtuuLasgIEh7JDG1eiebxc2QVUBI5hWcIpy47efb0Ejk8A+tgswmM2q9mqy4LG5JEB+1oy8f5fQ7p3ko7yGkZikKo4LsvtkEGe0TysN2XSTO4W5+vf2U3Zg6xonXXn6NoPIgVny3AoNvBTcSMZ55SkaejKPO3wPvigKcI0UCqjGvkavbrtLqFUFpeD713vUcmXCE5UG9d6/0hqbCJtoq2mzMeW/E7lmnaXQrILDrbhxjAnAJglNvnSIrIgvnJ/pycuRJWlxbCCkJwZhs+1tfJ2On7ZlGdF4frPcK3FwqqZJAZ7uGjhoVRl3P8RVny+h75heyByVzdvhLqE32gfzfYefdOyk8UMiLhhftxiwdZvqe2UKnsy+FyQu6PXe6z1sAl45BTDl7lGAXMYDM/j0bR19HgkcEc3BmGs2qdJ57JxZzZic837MgWy0CEqHnGiU3JOwNbeL1OQa7cTJhEenBZl6b9M+rTaUyKdEzo8nblYeuSmdH9slLCgjMOUOTXzwNDRo8b7ILFLBSHlTOmWlTGXxT5+fgpKls9x7NyCslhP18nPA3w2FGz3ie3zHq+lRyqf8ljo89zq+j0u3OT6ISv0ffwtPA6H98XafePUX56XKGPdl7iWCnqz9BWfvo+DkNHp1DxdkKvh/zPVM/mQpaGeVB5XQ4dOBstJcUCT/rzNDL9/LLkl9od2rnh5vGXV1dUco8MVjrkZltNy01GTW41vUE2te9Zm7GjXOfVNrze3fUinKDvUksAnjHeTP6pdF25urXIRGsBGftoyGwHxBJ5flKnAOdcfJ3IvT8K5ildbTX+FE9ytGG7Du68yhDlw5lKEPZP3k/SQuT7N7b0c+RmvQa3nF5h6UHlv7l8/+fosU3lpCsvXhdPgJE4BnjiWesJ8n3JiOVS9k28zUMympGZQ3Aar2pw7m6Gf/iAgRDIgFVAbSWt9mQfdGXXFEFxZI6KJVzw86x1F9MZlit0O4WSNao+0jwdsYlVUzYX5/7vxn6Tfd7zPl+Tq8ynmalmcCKQELzmu06+yRIkUgckJvlqPQyBAHGvdbTFWBq1hFZEEJQeRAafTsSqeQm+VbxRPYv8GCI0zyb91a4qjk98jTf+oD7L3NIP9Efs95CS6kOReth1F1q9Bo9MqsTVoeBWINA7Qrblm4je0s2FqOYtJm61l6CuSOziem7p5M6MJXPVSN5hlOMf0OsKl3fdz3BV+Xo1U6MPKpAKN4P3GlDbPXNiEAvdaUyoBKLo560b9L4854/mfnlTHQubazcsJJzQ85xePxhwsNsJY6a+1i5rNrHzjJHZq9fRdJwJ3gO1seu737N4F8Xcm7aIWKHp6D1tP1e/g7mMgPxmfHondVYzFZkcilvKN4ArhFz8xLIO/89S35eQmloBoMfGMYt224hYHAA1fJq1j+4nsHnB7Ny/SRaH2olbHwYmT9nonZRY3YWqPYX9ej0OvvMVfrgXDA3caTVjcy2MDxv8PbQxviw4cFUulT7mbdtHnFZcVgfsF/z/goGB3diz37PgefrGTd9ms1Yp76d4BIILhEoH5QJ2M7f14szLvW/RJQuEoVGwfh/jcc9yh2rRaA6fATZIaI8uIBAY34j6d+nEzs3FsId2XjHRho8G1AYFciurV8xs2IInxjOG09X4ahzpN2xnTVPrWG08tHuz7VarWy+fTO5v+Zyb+q9dmR6V0k9SQc/pypqDF7D7YtayhM76XC8wqSDkzA4N6HQ+LHs4LJr1wRdF1LIKEri+6U/MsltIhKZhFU5q3AJduHMMz/yzsfPM27lNg4P/Iok9SOEjQ8je2s2feb1IeJTNcHpd7LC/VtCyoYhRJ0BhrPwt4XkH6/i3oI1FAX/gWuzK6UhpQz1ea77vFTOKka9MIrgkcG9dup4to+hKPxjisOLuxP766LXETA4gBnfzOf3hTuxyEQC22K1UHWxlsb8RsLGh2ENU7Fvyj48Gj0IL/VFWCXwaex62iraeLL2SaTS04Thwsjqf5Gbl4DF8BwfBa2j76K+BD3anz6vfsbdLiD/4gCyKyW0j2+j/zUJwQLJazzKo/z+5Dp83MfQovdBEOCXWb+QtysP73hv6q7Usbp0da/XhSAgsZqxGKWAjIL9Bfw6+1dmfjkTp8F92FPShdkhE3+zuF5UXazCqDOidlV3F9YURRQhNfYnckokgedE/zuA7NGraPF5g3u+XkT5YB2n3jmF1Wxl+FPDObF5M+NOzaZy/EUuHklGEDzhRbF7xG+AH9UXq/FqFTtN9+wYgbeLeI96xniiclEx47MZJNzWuzRT7LB5PN32JgPPhxEWMMD+kiVck04S77mQUaKcmanDhEauZVJ6DRXKGUw8PBFzjI6EJ3pkoMyfNrP4t8XsnbqXrQu2sjJkGMOeGMbZNWcpO1lGS1Ixn3VVs/WKlnvffZLg/h0YkxvwLr2Aps2NuJRbKPTfyKFOBfGaLq7b887bNI/N/8qn/tDPuNQXAuL8PuzJYcTfFo8gCBRGFnKAA0QURvC45+NIZdJufxi9Xk9g2gfU+CkpSXwOrYeC8+vPc+DxA9xx5A4klSYGpA4gP+oEf4wbhlTeRtwtcd1+cxW1P6E3HSJtUCquQgRdTV3d8oSJj47jrefewiq1MmXHF0ypkWK824gsNpKMCauxKDRMvFzB0KFwTgfCtTi07FQZWk8t5z89j9Vspa3cPvEZ5BdGePtBaO9RZPFL9mP8W+PxiPags1TJr16/8txbz1GyowQegYgpEcg1YowlzSogujWWgxMPMi5mPJk/i51fMoWMzsYupu6fSH5kPi5tAbgstLVasKgEtMpWBjuDpstEU7stIeJd44qDzpOvWxzolLazzEtBTU1Nd9znHOTc6zWBWFSZMiSFzIRMXggWFQtqL9fSUtpC5NRILgwrpc4tnSsJV7jPdSsuQS44+jpSeb6SyiWVPK58ilMjj1PtV01HUydbn9hHydESnqx9EqtMwKww0/fSEKKL2jB2mriwLoWLX16kpaQFgPLgcjJLHqRZpmD6pBXUnS0m+nxP0cOjY+/FzbeVB/dcJTi010voFVtv3Ur91XoeuCz6zAmCQGtZK/oWPU5uTrzx4hu89KZYnGLqMLFoi20y3CHUgWHvDSPmcAxhr4Vxft154m4V/dRDx4onssZ3De6R7qw41UO+WkwWsk7JkEvl+FX70SKI1zns8WEcf00sLvOu88YsN7Nr1i4+XvDPPYEBzq87j0eUBwPvH2g3FrhqNufaYglP2wYdYoyftTmL7K3ZjHpxFJ4xnmwctxF9q5770u6zOdZitKDWNWFWajGrHLpjzc3zN1Owr4CyuGa2TPkWnbMOR6ejDIsQ56Tr+5DUmWJ85qUBi0Ic6yhtIDDHlvC5WfYUwP1wKa8efxWAwuYKJq21HZ+5YyzOOjGmNtTZ6qrrTDrMCjNHxx/Ft9mNlwUo2FuAvkmPZ4wn/d9PZsL5+2gN8qax46bKL2BB5ceEuKShdfqT/IYYLMYh6AsrkVlMyDpb0LmYMERIyGhRMz9jBIJV6J5vHLwdKDlWgmesZ6/dkgGDA+z+dh0KBxVNAQk0+ceDxL44d/AoFy7XqfCr8qVFMCOR9NgOAAw8F0WLk4LBp2RIvMttjrXqrfjU+KCUKYg+/yMOrdWw6Vn2HVVxaFMN49IjUaqDYOoVjs7cjdk5CrWrmnsv3AvAsVePkfp5KoMeHGRH9unr9UTnRVMSV9K9N3OPdGfRlkW4hbtxVX+VtP5pDPpsELtbdzNj/Qyb4+86fReS3RJm/zmbpfuX4ujnSNbmLOIXx6N2URMyO4Qz9WcIqAzA3GZm3Jvj2PfoPi7/eJnOhs5/S/Zp3DS4hLjgNsgNq9Vqs+/9n8L1305yjahu9Y60+T0jJkcw4e0JBA77z/ayEolE9KqVQGXMOEYFlxE80t6v7e8QPSsa/0H+9F3Y125M6aBk1tez8Iz1JHiE/ft2BUfTihMBuUc4PSWVQddis9wduVSfKSJn2J04NRbjWpuHXNtTpKWr0mGuaKQyZjyJh9fitf8djH3vROzJEmFWavEuvYA1zQFUY+zu/9TPU8ndkcvABwZCXM/fJ7wzgcPPHqYhpwHrNQkeuVpOyJgQAgYFcPGLiwDM3T6XWp9aurT15GzLAeDM+2dIem86x949iZP1btJEdepuotN/oD+vHX2WYq9PSMxIxEOvQeOusdl7pg3Ips2phm9z70fWFcKARDF/lrszt/s1+o5rUvFGSBQESkqufd+dLWi++I4TLgMY/YJ9Huq64sLghwfbjVmtVlxbXBl3bBzHBx4m/iayzzHbytT9UykKL8JqtaKrErvtXULEWPrk+Ct0coln3gvDFFCJVN6fW7aK82HxyfPX3kVC1PlfkDpqgJU23szLNy3n1Vdf5cDkA0Ra7WWg/w7j3hhH2tdpqJx7V4Pqvv62Nv4dLfc1YjnnXxF91zEEGKtU8s3XX/PCi/a51v9b8Vc5v/9T+C/Z91/8r0EQBGpqavDy8iIiV06FVxs/39vKJ9G2gdh1Amjw+cEkZSRh/tZsU2WclvMDQ84PQa1XY1KYkD5j/yBJBCmxObGYVLZJNEO7nuDSYFpdWpFKQnE09LFZ3IxFFfgXZaDp0nBh0AUihN486npHR34DT7//NAcnHsQyVlz8NO4aZCoZxnYjfc8EMfHAC3y58svu7sEbURh6AdcWVyYciME8tAQIveE7AUEqUOVfxaerPmWynys3aiMoxwwjw5rIwoDbsf42huprSRWFVsHsb2fz8rb+FAb0SN7d7NH1dzj/6XlS1qZ0G6zfjIL4EnTyMoKu9kgABA0PwtHfEaOHmsMTDzPszDDm/zGfrgENdOv3AKoKK7GlsQgSAaPKdC0h2fObPf7B43S4N5P41neUeo7Fb1RP95JbpCflfSahc7Myde9UfGsD4c1/fFkEDAkgb1ceB585yKR3bRdNqVpByggjlf770cojkSnj+XPln6RtSOOxisfI3/kpiak69Fon3JsFYBRbFort9S+ZX+J68r4sMBupq44bq2+08V6sf0hMaN/3xX1orK7d3WKDHhxEn3l9yK7XU+96mHo3MLv88+5SiVTCoq2LsBgsvZpQG/oPwVhUSdyJz/nwrcd560PbqtT+F8XAJa3vC1iUe23GZBI5CqszMsEBq1zBzUYxZ5N2ohtU2P1/802MyJ/3/UnXKTGq88s/yX9C9jn6OhI2PozBq+yDr+vQtlRh7hA3oIJVwLOPJy7BLggGDT8u+5F52+YRXHWSG70wAZQGOY7tqm4C9uZgd8LXr3Fk/QVO9R2MxnCFdTdo2d4og1MdMQJTTO8boBvnvhvxaPGjdDV19Rp8VadVk/JJCgPuG4BPgr3xhzGnCP/cAgqTF9HpLFbcfT3ka+QaOS90voDOoZRWl1a+u+s7rjx8xeZYqYsUs8yM3CJnVsosHh7ysM14xg8ZNBWIXWeGNgPtNb2TmL2hIbeBrwd/zfCnh/caSJs0zpiUDig6xMpA50BnVl1dJR7bAILERFBZEEF5JXQ1x6Ly7blHXQdFkzJ9BSf79EWtV/N4v+ds3rtk8a1Ut7aB8BNIBKyCQEtJC5W/5+BS54QglSEoPHGpy0PTVovZOJwbO0xj58ZSe7m21+sytWcxdd9ivBpqsFqho76Dwv2F+A/0xyp4kjX6XmJT/qR/egO8JprYt5a24hLsQnXWeeZvFecIV9fJWC1iIlPppETroWXF1xOIUI+kbnEXreZ+tNe080nEJ4SMDiHsHXFuarLA9GnnqPdJRKaSc/XXq0zYI6fE14vy4HIkgu06mLszt5voi7s1rjvZeSP0JToGXxhMp7aT5mt1FPpWA2Up1TRcFTtIigc8gLq9jtBYeznXoSeTcGlxRCpIMc6sRD9WrKxP/SKV0I/6cXDiQUpCS6gMrMTP21aaximsL6WyLJLbF2AIdsc/KAkQE7OF+wtpddGR3TebF/pVMWN4Na8aVtscf3HDRVI/T+Xe8/fayRxJz3aw8HexSKLuyVb8+tqSxtOcnuW02wYOjz/MvRFTcA50xjlQTBJVN1RT711PrU8tfjVhSOVSxr81nuFPDUeqkOJodUdpUKLt1OLUIhYPvCZ5Dfcodx7Oe5ic+BJaZfl8tHk7iWcL8HvGANd4OZPESKX3LnxlIPcfQalewz9digUBLHIlNeFDiRps3wEnc1Ly5covmbt9LjplATeTfZ7npNyedjs75uxAYVUhV8tpKW3hyAtHeLz2KUCLxCpl0PlBuLpZOfvBWS5+eZHKc5VYl/pTHF4MiGuWo8Ed3hLft+JcBbJPN/EkT/L6S69jlVm7SR3xvAVyfxU34G0VbXZkn1Sjoskvji4ne5UFEIl03xpfBl4ciGWS7TwkCFBz1YW9GiVVAw9hESykfpHK0RePct+l+xA8FFxKusTXliYqPDdS0zWb+zfOZ+7GueK5x+u54neU2hoJd/3ch5bZ6cBw/JL9+GPFnwyXTaAoeCstbi20uLUgVffEoRp3jZ3c0o2YvnMQnhWv8tsKbfffmvKbsBgtmM0gFZT4VLoyJHU+uuRmdFcrOPnmSWJmxyAEKDg37BwLtywkPisIwSpg7jJj6jCRuyMX47RsVj+Qwv4fJ2NyqcJqEWiraOPcR+cIeCSJ9a1wxQgJ2mOU6h0xG3riTbO7HHmTmWrDAH7+cSxBhZvRv/kEebtENYzrPmEn3jzBrK9s/WQNbQYG7haVGJr1g2Dl9O6CAkObgRKnq2TH7CC0YjwJCvE7PvfROdoq2ki+JxnPEhVJrUlk9Mvollz+894/yd6SzbP6l/Hyq+fniZ9x1jKMD9WDcQlxIWxcGEpHJZ4tblw6lkxnZCWXzsfi5CFe03V/lZVvh/FiQDm1Zg1fLP2BYMXI7uswtBr+NsEhCFDv3cbvC39n8cDH7MYvTi2jvuUss3fMpn18HurbosndkUtTXhN+E/oSf/og5r4hfLnySx5O/oaupi6ytmThP8AfaZQzv8//ndKQUtpc2pB4qei7MEL0iPVx5PI3xRR767k44CIpg1Nw9puAqn9f0ieF417yJ07NzTyudyZl9GW2JJ+nf5TYQXvp20s0X6ihNH4aZkXPPZb5cyZV56voM0+UaiqMLKQwshAnPycK9hWw+4HdTP14Kt5jvXFr6KDSN4U2zXhgAO017SLZ4KJGlmNk1q5Z/LD0B4odKxEEQazIvwa3w2mM0blRGD6Q4coHUTop6X93f3yTfBEkYFQZCVfA1M4CGvMdMenNSBRaTJoeAlkqBbeqLDS19QjCGL4b9R1BI4KImRNDxJQIhj/ZI3F/HVargCBYcK1OI+cXI/37j6TiXAVHnj9ybS0W74vPH/icw/eIBEf2lmw8YsTCT8nZM4xqmsKJ2aNwjUjCPbKBpDuTGPLoECq1LXx353c46TyYtWcQMQrbeDPSPJSjGU+zLOoP4qyNdOrFRdNqtvKG4g28VK58/+JbJHUE01AZjlRpton7Hiuzv7euw6fGh/CicPKi87qLv77oJ0qaP9P8DJnJFdQ7XkFqVeIiEQs97zx2J4UHCtm8YDMqo5wJR8SCmtZbm5Cr5XTUddCQ2yDaJVQEEJavIjD/CAbdAMrPlNNe087Edyfy8IUX8JHBeMdj/Ln+VsL0l3BMjqIkcRZODcWY1M5cNkhZ6vIySxNeRBD++UZL46GhLrOObbdvY/5P85FIJIx6cRThE8LpUHQwIGQAbeFteDd7o3KxfUa7mrsw15gJLwrHv8Kf8InhPNf2nN1nDFg5AK2n1uZv6YcyGPu2O53Tkvnqvq9Ymihac9xI1Kxdvbbb7/g/xbjXx4nywjpDd8f1dQhIaPWJ4dK05xh4rb7g9LunqU6r5vZ9t7Pv0X005jZ2k5U3Qt/YTvzxz7BKpGSNXYXJJBaKdNSJpGFwlhu6Rbru11/fZpX3nYxvwWkSD35Acf95SFtcUbdbAU+cE0LIHnkvHhUZJPnUsPDHOb12JEq0Gjoc9DR41NLUGGk3fnRCCp3qOvKi81g3+KDNmLnDTGB5IK0urQiIHp1lJ8uoyxLXk+D6EFozq/AoMaLR7wJs90q+234jIKYAz4B6mg4Oo2t1POrEaAxqZ1T6NpZ/6cVlSQKlV0NZfjwZ/b8MmPVikYjaXc0PE39g0ppJDH/Cfs6oTqvmwBMHGPrYUDs/++6Y49p+9mbluqYBLfzg8DNqUwDzDdsxdZk4/vpxjDoj0z6ZxoDUPrQ6B5M1JJkIX9trEko6eeCLBxiz5ACy8V9zLFu8Bw/vM+FVJpIhofPbef7yAxxK+JxYpThPlJ8pp/ZyLXK1/C87yJTNSqbtm4bRr6dwO3tLNrsf2M2yg8vwHu/NzvKdhG8Ix9xpL4FflFOEt5M3Tbc14RbhRtbmLE6+eZKig0Xcc+4e7txxJ+s3rOeDqg/Ydcsu3C3utBS3ADDqhX/fBdt3YV9i5sWQmZlppwL0P4XrH1Pabzal/ezlRjXumm7Z0v8EBp2Bii8PknCykLzBS1myZzTy/zCbfp3Q6Q1ytdyug/06znxwBmWDgdLocbhXZNA3tKt7L1d6ohSr0YJH5WUqYifS4hODsd0IqGkubuaTcNFn2y96DEq9SKTd/FMY1U6UxU0lOjYQiu0/v+JsBUEjgrqLhLrPWSUndn4sUdOjaC5sRuOuwTnQmajpUWRsymB16WqeX3ScGtkZhp8djjG5hX5LEzjx+gmqUquQerhS1H8BQtNT9D91AV3DUzxe9TgI4hwd+LiMF/eJ5FC7m9buvE6PvoheUUVXnj8zt3nQYd3Cg2sW8bLlZR759hu2V9/DsY5R1J6RY22UM29ST6OAsqsFqa6Noy8e7TVHMeDeAURNj+rV307fLpLsByceZNIw+3useYiEXX7fEpUfhdJBgc8KMXfTWtpKc1EzAla6NF3sWZLA7YOn0VbRRt2VOvwH+tN5tYVXXn2FY+NSqYocjKunmMOrSa8hdFwoJUdLAJBZHCgOL8aJDvsf7G8w+sXRjH7x3+fdXJydqanqXVnvOnYgevT9E9zR1cXan3/+fxXZ91c5v/9T+C/Z91/8PwJxaZ0EOw1n56I8u8Wnq0zHqk9XURlQyfa5h3lRZTtBNA6F38LXMHPXTPpd7oeL2rb62KniVwTtZT596FNcDYmsv2GsvbyJFd+t4Mi4IwxPmYdVrkHoyd3jPGUYhwxBNHi+Tp1PHUqrvQHrX0HmoiJ1QCqTDk3CoumASSJJYTFYaKtoI9hvJqn9nmPFtyto9d7NdWP369g5ZyfhheHc8cMdWMIauJHsC5NHE1waTItrCw1eDbRrWm2ObXFqoiagnF+SjtPyQQbLm8TK+jPvnxHlAX37I2AlsDyQ8UfG0yGruV4I/G/Rd0FfUtamsHHcRp7T2W+0QMClxYWkwz9xUH6BUGEWFSkVTHpvEvmCGGQWhxVzYYSJZF9vmyNjUyOIPrKYNU+s4eiEyzwt/cRmvCCygA6HDhT54KHL4qe2QiIQW6Ndwtwpj47hauCzDDqhxbHdPiD4O0QuGcTlnaW4BLvajbUIzeydJGYzk4pET6a0DSJR1VzYTJumBaN7IxURChIFMSEmU8mwGCxYzVZisgIxCjJ+WvoTiTd5M94Y+LZ6euOs6elmeM+9R0Rc+5R4PebOf96FCWIQ9ledYlYHZ3S+MQgSKa319iTiwNSByCwycmM2UdaYTbdmKtB0pZLICyeoDRtK9uj7mXeTCobyWvTr0O6AT60Ppr62snslR0oAqIid2F1x9xeqK3ZIWp7Ua7fVdfQ5+RVyUxea58T7vupiFQ1XG3D0cUQoFb9viSBBKrPfdJSOhS1jPqJ/Wn/6lryI8JTt+PvvA9prROBNssA+CT7UByfjVZaGX+Fpfv+xH6Om2Urj/B1cQ117lRUDsbosY2MG4ZPCCRoWZDduKSnHt+gsWaPvx+AolsYnLkvslqrZtHxT92ufPv40f9z6R/f/naKd+PDxDxl7bCzTO6ZTsL3ApvMvf1c+TflNvGR+qVe517+DUWfE0Gbgyi9Xeg2kAUxqR7Rt9qSaucvE0k2zCS4PBjJorxuN6w1kn0Quw6JwoN2pnXandqRq205LrSoOrQWiipKYcGAIhmWV1DuqKVy/nzCpHKnVTG3sfbhVZ+JZfgWLYQggY8WZFXTWd3Ju7TnSv0tnyge2vh0A8/6Yh1eDF5uW/cpq68uUHCrij2WiPNiMc09S57IXp2AjVkV/BAF+nf0rpSdKWbh5IdUB1Zycs53KgEpKHJ7C0DaNj8PESvKXLS/TFllFuGsL8ip/2vXF1GRYMHWaKNhXgPeDg7j3q3vZMSgVa/Dz5Mv6MVsmJXxiOMfHV9HkLpKyHep8rM3P43RCTcOjD3Ynxeqz69n36D4K9hbYVZU6TfDjcfNSlm9cjjrDAb6BD6Ydwnw2lf6PjaE2YAA/5r5Bgf8HDBOe4FXe4zWJKIP9ivAKu+cfRdLVTlB5EAPDH2Too0PxiPbAJciFbFUNp0ee7v6smzf8A1SLaSmey/Ojb2Hg+D/Z0FAJwNJ9YkJE+3woY9xKmesIZyqnkhhsuxZnbMyg5lINBp0BjZstEdk2sJPK1KsIEgHLNYOLG6tCW7cfIa4thiMT9vJgqFgxvS5qHR7RHvT/rj/aDi2XEy+TG6dng78TjXmNdDZ2YjFaGHxhCAHfPI9epUdm6Zmbm/KbbM7BKpNj1Lohu4Eg6tTp8K/055bgVj5adpoR5adB8hP/BIIAkambUejb8B9vf39arVaq/av55u5vGFBqXxShapTgVRqC3CzHKhUzWiGjQq5truVoiu8jvqaSQakzqIkTiFsXR8W5CoY9MYy82tLu98mJzcHbJBIJVRerujtjGt0bESQCwaXBON30XI7bPI6IsAj8k+2LhRReLpQlTCc0fTtl2/QMGWLbWRVyyRFVlT8fPvYhrwY+bOPd8aL5Fb59+GumWWTMOzwRRf9OAmYEEDA4gK6mLqwhanbM3UHs1ViefvdphFWtWOZbMLQaUDgoaAgz0tbnPPMELWcW/EFouLh+5O3Kozm3nkjEJHytby0IItFwYzFS6clSSo6VMGDlADtJNINrOOaacibtDaXuSh3e8d64hrqi8dBgscCMA0/hX2bEu0aCvrKd/gv6cOiZQ2T+lIllpPisHJh8gPSBndwulbBk1xLSv0vHI8aDo1e8KDHBlKUHeLVUxoOazwmbEIa5y4zFYEJ+chCFAfV8dMdbfBW3FscgN0pPlpK7M5f6h/35UrKCM4EaWup1bO6IBYmUQQ8N4sKnF5j51UwacxsJGmG/5iCV0OYRinNjCfIQsUgvcGgg1ReriZgSwbeH/qTcfR2zfnscR2E78CC+Sb5UplRSca6CsIvOjMmay8RDE2nzttJ0VxMtxWLnEgK4On2PxSwneOYJ6kv7k7AkAZWTio7aDqKH38Lh6jwuOTiz88knCDL2BLC6Kh2XigJZoylnoasCr8B62s16LCYL5afFbottS0W/uZVpK+2kto2NbUTnBlMYVovlZqNYxOJBtV5N8qVkusKqsZgiWPjrQjQeGvQGC45N5aj1KrHj11FOxg8Z7F+9X6yGH+NMZmJm93sJgkD5mXJOvnkSgGj88PKSs/m2sxCzl+W5U5Eo5JhVjjT6+aHRbkMVUMt9rnq85FB7bR7debco114/01ZX3NRhQt8qEhiLf1mMY7sjX98rSvNJZBJaSlr4444/eLD2Qd5+/m086z3xq3sRs+FPGnNEr0qvOC8MiQq2Lf2B5LRklv24jM5Hujj2ylGaC5pZun8pl4eX02rJYf62uwhUOqN8U8nsr8XkV1W5Ab8qP14Oa+fW+9dxrGAJKhcNMaEGzu3poM79Kq2OuSisJtQdF9HmBgBjmPLRFFI/T+XQ04dYXbbaTlEDoORQIYN2i3NmekEnS94biWuoK33m98HB2wHlHgNTz04lZUgKci8xJk1YmtAtbb13+gEaDPl4mpNQoCV4RHB3N0VViY7S0FLk5mZCDY8y3NY+lOALbsz/XYvnvbE0bBhA9ABxH2a55l+mNih52kXDtCYNv/2whOblnd2FHv8OPmk+zP9jPvsf3t+tZuMS7IJUIUWukSMgoNKrmHbsE/xHBVByrIRd9+1i5PMj6bO0P7uvhHAy6QF8azyY5XcvcbfGkbExg/LT5XikyLh3z738svgXCvvexRNKGXl/5pG4LJERT49glEcyhgvhWO5JxdQsYGztROnhTENwMjlxreQGvMDBzv4EHBlN/eEW8sddxWPS3/v+XMeMz2bQkNOAwlH8LWsv17LnwT3M+2EecbfEsaXPFvY670UTqkGmkLHj7h2kf5vO4p2L2fLDFixbLOx6ZBdBkUG83Pky7TXtaL20qJxUfLLgEypOVnB39t3EeNqSOAovBRcGXui2HThTLvpWiclwEUnpSaQOTGXOjjnkOuYS/sRfS5vdjLqsOi5+eZExL4+xI/u6yhtwbOqk3S0I4VoRYeKyRKrTqtn36D6kculfElNyrQq91h11ZxOatlr27XPnzjth5lczOfT0IdKyspmzfQ6nR5zGua4AY1sA+GioDR+G3sGTqAs/41l2CXlJI1GtXQjCo8gc1HS6+tPp6s8rX4O7fe0gAIUjdJwIMeHWPoOZMfZuS1fjCtArRFWDm4s5Owo7uOebe2jwaKAy2IKwCka9OIqUtSlUnq9EO30c98xZRF2BJ28f3WH33l0hodS5asiI+JySaToGK1YgcQyiwy0QVXU2AFKPFiITC0ltmI1UIWPyB5M5u+YsDdliYdrNv8N1fDXgK/E3uCPRbqy5sJHEg9/T7h5CbdgQBCEIY4eRluIWvOO90Zkb6VQXoLC4orC40lrWwOl3xLh26tqplI54nBZZKWX+h9C7tQBLut9b4qHgyLgjJAZXMC+xjK/6OgD3oSzMxqmpjLK+U0jTDUSlW8NwN2izWDC2G/l2hOgvGTI6hNITpb16l/bt35dND25C/rucbV9uY/5987tVXUydJjakbQBg7b1r+eiVj+yOv/ubu2l1aeX4C8dxj3AnZnYMp946JSo5IFo9SOjZ96Z/ly7+BnIpQcN7iQ1uQntNO3sf2Yumv4aEhN476v+/gaHdiNkEDm72MuO9cYo3/q3wQCFnPzjL2NfH/kdKNQ1XG2jafxEVIDd20lquwNlX0+v69HfYvHAz7dXtNpYwNyL792wcvBwIGd1TzJf1axYO+R0QOI6mwH7EPNoPh2v8wdDVQ7nyyxW8ytJwq85GbtJTd/5WGBZL/p4exaGAPLGT+cqYBxgbaZuDC8o+iEt9PvK7noRi7FRrAgYH0Geh/VxfdLiIk2+epOZSDTl/5PBQ3kN4RHmgq9ZRca6ChtwGTDGTaTYb2e/fyROThhKk0eGd4I1MIaNifzaK0QmU+ZaQHyVDIpfYyOHrg0wkjsig75Bs3rp8BFOXiW23b6OzoRP3KHfGZw1hz7Q/sLqkUefnivsNeTQrAiYBOqxyfK9EM+b4EDru6ikU7HTxRz9mMitf7r1/TeOuoTGvEV2Vzo5oN+j0jDgzgpTBKQiO9s+lwQ3KJGXcsekOmspNeL3vxbwf5pGzPQeFVkFiWhAWq4KToxcQY/qREXtj2bVyF33m90Ey34Oc2BxqvQs5PmY44foFrGUr+mY9lecr6ZoxiLNKOcEN91Ds83F3YdM/gSAImDpMYjzxF/nB65hz22388NZb3P43Up6t8G+7/67DB2htbf23r/sv/v+H/5J9/8X/I7Bl0RacdE74lQdgbuvkug8KiIuzIBGoDKjkalyZnR+KoJLQ7tTOtvnbiCv9kJcFwUar3assFVd5J5bJFixm25VL6aVl79S9VARWoDLsIbT+WZtgQKZVYdCqscrE45TCPzfYVPiIhsn+Vf74qsXFOmRUCAt+XYBfsh+yLCe2h23HocMBB2vvVTwloSW8/ezbvO5sS3CaS7pY8d0KLgxt4fiIr0Fhe11nO9aTEroOp0IBvaSJiQ7iRQ1/cjh7H97LQF5DPlBKZqwKv2o/zG22k3jmL5mYu8z0X2Hb9QQgSZAQ8nwIKl1PIH3h8wvIlDKS707mkQ/vo8qrDEEioLd0IJFJRN14hRRDuY5lm5aRmViExO1NXG/aZPiNHs+BDjf0aj0ywZ4k2TlnJwogx9mFneW1NFXWQX+R7BME0CurKPf6mvIFoDR7su6faq4CT76kQeezjD6x9mMdJrEKavzh8Qy5UEzH+x08WvwojXmN+PTzoWRUB2eGfovEqkDf6Q0M4p6Ue9BV6ZDKpYw9PQeLvpq0AWl2Eai+Wseg84MoDiumLvpuouPtuzTcJwejNCpZ/fFqmqdfhkVJ/+iaTF0m3tK+hV+yH3ccvgO1q231pmAyUR+UTG3YUPw04jOz856duIa5MvqF0fx0+0+ElIaw+uPVtN6ZA8t6yL7iimyc6nM5Mvwd6oNiyGl9lqH03McReUEMKYig1qeWWbtm0RFQDzco8jyc/zBrFp5Dfa6amsgRWK3wn0j5/3nfn9Rl1nX7L9yINs8wZGYjLtcei7DxYUxbNw3XUFeE011MPDiRs8PO0myy95+bKVmP94X1xJ76BqW+xG7D4F5yADwaiVRE490QiXDTfONe2ZPIM5v+swrGC59fIHd7LvN/mm9XmewyKJqKicvZ9+ZR1C5qomdG24wrRg4h1ZSIWelwLUKXMm+TrdShzCzDu84bmedNc6gg0OnQyZHxRxj87mAK9tmSfdPXT0flqmLvI3vt5F/+HVxCXHALd+uWWroZPkVnbYi+mowavkz6Eo27hqUXV+Pe5E5OTA5Ffbx5xM92/jW36zGbDuF4TUrIqDfhoO25NnlLIwq9EqkgRWlUYjVbCRwaSMJHd3Hkowy8ytI43zKTwvmFuLaG8aBKlI+6Tqa6R7pj6rKXegVIHZjK5cTLlISVABICBonJ7oH3D6TD2IG25huuRDRi9MpCEKC5SOxc9OrrhaVeQXr/dIJLgwkunGgTbAtWgdK+97BGeYZBGy0I7pvxXb+KRVsW4ZfsR0p6DQ4dDhgNSvY2B2MUmkQ/lAQfcuPa6HAUq/u6lGXUuKcSqh1tQ9Aa241UXqgkZo5tMgxAopTS4djBsbHHcLOI81BrdiUOQIk6Fnl9MzFX5FS4KxFU9knwav96OmQVOMneJDhuGlK5lJhZ4udkZVUzSAWJjd44f/8swUts17yuo+eIP36KyPEnaTO4o7eIhF1HfQcfh37M+LjBcEspp7KCOPDzBMJX2PrUzvh8Bm3lbb127HQG6Plt8W8AfOr+m914x4VMYgjlyARRQvHKr1doKmhCqpDSVdfF0+8/TcrgFE6OE0muU++eIv3bdB6reAxVoBd5sR1U+VzAvV2Uar33wr0oHMSN//2/vY3Q3sy778+EFXDQpye5qMurZOWGlRTN3c2f3heIzv3DjkD6O3Q5emJSau2qzwGsJjNhRWFM3zMdj8ZSuIlDlM1N4pv4H5l+8Gn8DX50Pt/JmTVnSLgtAaWjEp8aBzq1Pny//HtitMsIGxfG/en3AyB5JI2XP32Z/TNScXTcRd8B4vlWnK3g0teXOHVnPYdCxbKqRVsWYXa3/U2co5zxTfDtNWkFILGYcW4opqPc027Mo1xNSHYcf876E4vE2i0ZBOLSGqiw8J7cma9PjKTJzYj/QH8qz1dy5dcrCCMFSkPBatLzvm8tak1f9q3eR+pnqcz4fAaR1pn4djTyUdxmCMvgzhoxgbRl0Zbuzxh+ZjgNng1MODKB9nU1MEUkilpKW/h+9PcAxM6JtSP7aqL6Y5RYCcs8RHvtALzjvZn34zxkClHmNy5DiUnlweXxtzJngAMOXlYCBgfg4OOAcKGU+zfez4HJpwnq/BiJRIJPgg9TPhRJXu+PHFmYnUR+YiadUgFBgDsO3QFAaloxM/bM4NLAVKb6JlNpvsBIAaovVnN2zVnUj0WiVYWjldYzbNZZnopPRen0LdPXTRd9h/8GCq2SvGHLAZhxTb8n+Z5kku8Rn23tSzpe2PICWfF1+HmOACByaiStpa0onZQUDu/iZL+d9MtIxsVhKNVp1VSlVrFw80KQSFBuULH69Gy23Pob0dcW4s0LNxM+MRzlnbchUcp5MeZrNjoKLC8RyfWiw0X8MPEHZjKR6hUNREzI4aKXilllehDolk40tIqV3b15EV5O/425f4ymSz2I9tCmG0UzABicM508XT1vP/s2X0Q+S/6efH6b+xvzfpyHy7hINt1XQbnHb6j0Kiwmc7c0YvTMaLJ0YnI8Mj+SOTvmYFhVx5Ujrcg1cmJmx/Cx5VemJB4kUAHphmselh161DodTe5u9Ll1K31cobbMh19ORDBmdk/SShvqjcRqQZBIuT6HSGQSUb7LYKZT20lsbiy3/XQbTUubCJ8QTp8FfQgaEdRdfDH4/GAGXxiM/hM9Qx4dQuy8WCQSCWY3KYWRhbi0uiBIBKQKGamfpXZ/dmlMA+XyTEafMUOXeL2mLhMIoKtq476v7uPSuFOMve0Sm7r8GSiR0Ho+l4Sjf3B4qo7LCR/QYh1Gy/Qszo/tw8u8zNDVQ/GK86JgbwG5O3MJGR1ip2rQomnjStwVyoLLMMeI65xEKqGtUvS20lTA0JShVARWUJXbTqKP2ClwXWK9wq+UjrZiZv9wGb3RHR4ZxB/L/sAp0AmXe4OQWCWibitiWFV6spTMnzMZ+uhQFKEBtMXGoXJOp2rsKWq9xCSoXC3n8crHeeqWUiz7+uI7eSUzVvzJpiDbiso1PmvQt+p5rPwxO4/mKfdNYfuZ7UxZN4W6FXX4JvmyunR19/ic3/phUAYRWViNwekqkplRtJS00FbextgPZvPD3RDXcAbk4OUDHqGd3L7vdrzjvGnLtJBlOkpFYAX52leQq59ixhcz8IjyoPxMOY4mX3SqRA5RTuS/NlIVOrw7kedSV8LwfHeKhxfweqUzyU1KXOpyGUpPAri5qBlHX8de1UQAlh9Z3v1v/4H+JN2VhFwjp6upi+9GfkfyfclM/GCi2AW+XewC17hrEPoJJLSd5bvwLp6XmcnYlMHuB3Yz7MlhTH5/MrurdxOgCWDhloVkPpBp89TccowAAQAASURBVJkOYU7snrkbgJCSEPwbxEKTS99e6n6Nk84JhUlBwpUEGi429HrufwV9ZAIN02NoMWq5OUtQu/k4sWeuUDDgFmjyAdwZeP9AnAOdu9eW3og+ALmjmpwRK1C3N6B38iIr69r3NsCfcW+Oo3h4MW61/WlxbSH6/E90la1AiAxC1dFEp4svAhKkFjPCgAHUlop+rxaTBanZjFWqIGfzFU6ezmPaumk4eDnw7chvKT9dzsvWl8mw/kyR7yb6lL+HQmKv5CAgoO3QElwxBJdYV5sxpY+S/ZP3M/TcUPpdEtfx9up2Cg8WMuD+AXQIOr66+ihW780cHPI+sNHm+A9vfRIBKyZ5E3iVcLdGgcVkpTpqNIrOYk4O/521Fg9GFqdBGMjV4ODlQHNRM1Ezo5g6bardHuk6vGcN4nKWHH2sfX4DqRSj1g336iwcWisRhEfZcdcOsrdk82Tdk1DQxvjT42l3cUahKMfBy5NBDw3CPdIdiVSCQetGg+NRrgY9RYtpODeSfYKHghNjTnACIB+cTAUAdGo8qAsZSEFUGonx77Cu7w6+eO5+GnwTMNwprlNecV7M/HImHXUdWC1WZDd53CsdlWS7ZJPUmkRzlbjXULmo0FXpaC5uRlmpZN7+eRwdd7TX7+TCoAvo1XoMFvHzvOO8u4u5dVU6li9YjsHPwL4p++hv7s9FvdiJ6Brqiq5a16tc6o2ovFBJ9pZsgvT/nhj8T/Be343oGzp4omI17jc1Xl3fu7vW5BCZ+huV0WMQhLHd43VX6ig8UIhEJuH2PbbF9n8Hl2AX/FdM4VBeMBFpW/g0vIXFOxd373X+HawWK/tW7+Pq71eJmGLv49Ve285Xg79BV9ZC1PQoG7JP7apG3lFFzJnvyB98O4LQE78EDA4gdHYiJTsvo3fwoNUnGscw8dkLnxjOvB/mceikCt1Xv4rfj1Ru39mncULv4Nlr4XVNRg15u/KovlRN9MxotB43KAj8KM63jr6OTHxvIlpPLe217VhNVgbcN4AfJ/+IS+xAfCJFNQAXR3DyFlVm4m+N59PYT0k48RuZS1r5dcmvfOfwPalfpLL7gd3c+set1PYXmB/wB5HVz5Hk6o9R10HOH6IMaFVqFYO7+mFStHHggcMwciMfus3GoDNQmVKJ+nA7H9VNY9aC4/wm8yDdvwql7wwGDoTUVLDKlej7DyPsL4Q5Wkpb2DhuI0MfG9odc1+H1EnJF/d9gVVqZWDXGLtj+6tnc1j3GxuXb8S1axq1l2u7i4IdfR2JzvbHuTWE4rBiOt3LCJk1pfsztQFB3ftWACviRi9icgTPtjzLm891kGkeRrNrFss2LcOjPQBe7f0a7CDA205vA/Bg1oN/aQ0DcM+99/LGa6+Rwl/LdLog+vb9E9QCLi69WAL8F/9j+C/Z91/8r0EikeDu7o5EIqEiqILBKYOZvnckXQPquVFDWhGo5bNVnwEQVrva7n2kHQKe9Z60uLZgUvVscq/j91tO066+ytgLGwm02rbsy13UpAxNAaAysJIZqfttFj+rrgPHNhMx5S8x6JSUMEUUvP7Prk8QHev56r6veGGQ2KFVeaGSwgOFhE8IFwkCQcpPS39iYsb7dsePPDkSk8Iknt9N3TRaf1fK4qZiVm7lyQ+epHyGbeI1bL+S0Wef48PHPsTD5IvGTUw81WfXd7/GpUVCYWQh7z77Ll8MOGdz/LbbtgH0SvZ9l/4dryhfwTXAlSXXAtk9D+4BIPnuZCwuvjR5VbNj5sfcE/weoWNCibs1juaiZsxuJgIrAikJqwWNfeWVMsyfpgAz/a/chmeDG5Y7RdLiRigkoMsPpu2bBXRGNcFM8e+lx4oZeOgwlaoICiMLAftk9N/BKfUIXu1NZGUtZIKtnRSGli5WfboKbacWnYsciUxi04V1PVEiSE2kad4DHsK3ny++/cQtcu2Q2eQpNjHm2BgcnW2TgF0lLczYM4Mds3dQ47kdq1XsJqi8UNn9Go/5EXSWdpI6IJWJsfaeEH+F6yRUdVo1pSdL7YJRzenDJKenkDHhMeRyMVjP3potehq8AC1uLUgECaeHn2ZEcBxmvZmOug7Ubmo6+1l44+XXcGtyIyZXS210EUzpSeD7lnuSfCGWTcs28efMP7k32l4uw1RQimdlDuVxU7FaNf+Y7Cs8WEjaV2m4hrn2Ol7ZZxIOzRX45xYCEaicVJSdLMM5yBmhxsDI0yOp867D6ioSOfsf30/O9hxWXV2FtbwWj/J6ShJnIUildvdo+JWz6PtkM7ngWbwry2zG2irbkN7gGepQlQ/YVs6B7dx3I64/R8Z2ox3Z99MvUjraoa2wnq4mW38MANRqjBolA3e/QZN/HIJ1ATk7cnAJcsF/oD+Dzg9CaVQy6dAktMscYVXPobpCHdP2TCO7bzYfP/IxGxZvsHlrraeWkiMlNBU0ETA4gMgpkd2V8f8ODl4O+A3w6662vRl6B3GHZlZcSyZc91mQSpBrlax5ag0AwXX3IVPZJo9aLhYyes9VgrMWEloaSsW4Mtxm9GzwvX7/FFOQlMjIj8kblcTYMRo07lJcEoKpC7kmieG7gRa3FlpdMpFIpQhWgQ2DNiBYBdpr25nx2Qw7qUGA9P7pAChNnlitIjF4vVvsdGoD0/ZN40rcFUqS9iEIs3kg8wEsRgsO3g5Yj4lz05jjYwis8EGhVfB049OY9WYkMgk+XSNpk1Sxd+5bIAvkA99nurvwtK161j62lhXO0G/nq9SfSqbz5QTw0eKo7yFoJYKctAFplCZaWB/5ESXHSqi7UseRF47gHulO0l1JdtdkajOySu/JpNH5PNksbiwKBi5GajExvKqC5o3bGYSKPpcfojVMCa/BA1ce6K6gDpeZmeEioUoJCrTUXq6l6mIVMbNiaMtuYc4X9zIovoST7Wa0Btv7RxDAKsjZXD6IC6eUNAW/xsOPreWbYd9g6jQx6EIcJ1VNvOrvw8jSDkxNOpvjveO88erj9RcEUs/fTCaRTFvfZz0SqYQHsx7km5XfoCzu4u6v70YnrUQzNJrwieGMfX0szdpmLgy8gMwiY9ipPnQ2dlK4X5QnVjmpcOwXwel6HaXex4iuFMm+G+WtpSYTVmPPWnS59SSTuLar9FZxaMIhQkIrCL0cT2RaGVaTFfj3E6EgQGXseMLTfqd5/3m43bZ7z9RuYPmm5X9xNAzSLqKicBE+9T+iMjVitVgxdZgwG8SE4Pf37KNdc1W8npu6m4J9BpLh105w+3S6HHt8beJuiSNwWCC7t77a/dqDkw4Squ3ZRUskEpSdSooPFeM/wN9unjPUteJXcJKCgbcweZW9Z1/G9FZ+H/cFri2uWL1NaD203c+d2Qz3t3ohOFpJe+Zj3NxeR6FVMOPzGXj28eTM0RNcPDSPQZMusP+ejSwJn0He06JcpVdfL2J+aMWcOYLc9/fhrGlDuKapOuHtCRx+TpQATEtOw68rlpogD/o59BTPXPeZi18cj3uUvdxPashtVCf+QdCoe3hlQjiCIFBytASvvl44xgSQM+xOdJpC9I6loIpDIpVw6/Zb0Xpq2fdMFkqjEo/Wvrh0+WG1Cpx8+xQIMOr5UcRnuhFyeS4+bgJnah0xLTNx6t0U/JL9UIZocH/kFz4MaGBciRyXGh+MbXr6r+hP7NxYPh20kcdbV1L9kYl+qpfwkYuSa8Z2I/pWPdlbs6m+WM2cb+fYyePeuDZe/3dTQRPn15+n74K+mH3kZCRmcGL0CQyuu/mYEqJmRBE9KxqpTMowj/VcqQZ9GPiGQsiYdu44fAdecV5YrQIpQ1JodWll7td30RmjovBAIVoPLQm3J5DZ2sqK4JXEyYtoLA5AXSPGL8WHe7SnXH+bBRNy+PTZhxml9kf2uozlR5ezcdxGAocGMuGtCb1KoulD5eQmX2RA2gD0RfaS1aEXHQktmcyFwRewyCS4BLuQcHsCzoHO6M16Sr0/IzI/kqU/LcW6sp7R/1rIwPsH4hrqiunpdJ749gnSktNodmvGRy2jLrMOs95MS3ELJVNqeT68DaNewWdvrSYqzAO9bzbxx/8kfeQI9naCQlCzOyWW5L3jMA8Qz++J6ifYsSafAR+8ee33eBmJRIxT/Qf6I5FI2DlnJ40ejYw7Og5DmwGJVNItJ9bU3kRMTgz1XvUcmpzB884qpAopValV+CX7keQ0kb5la0kbsJq0AWn87PAbWi8tXn2uJ4nEG2D3vHJGOY9DV63jQ/8PAViW/RiHJhyiIrCCr3P1aE3fYja+jirIi+qIEbS57AJAhpw2lzakDq3dsVHEpAgufX2Jcx+dY8QzI/B5x5bsc+jjyNZFWwEI1oprv6nLhK5Sh6nTRNLkB3jRYwrLvpvNoTNpTK0aTsHeAkwdJqZ8OAUBAalVitTYRJe+kdITpVz+8TJaLy3J41145fVXuJh8kfKg/vx6fgqVO0Ooy6xD66HFFJGE/4xcnkjeznO633HxFMk8iUSCk78TblfPYNZ3MWrAbK4Eb2Kx+y02cd91Kca2ijY7ss8r1otxr40jd2cuGnd7okWjV9DuYOLMRH+mDk0kZLQ3fsl+HH3pKGe/OU997DZKQzsYWJGBIEDe7jyxa3ZKJJOTX2JjZQEG1QCUejUCErzjvak4W8HBpw6iBOrH3MkAj2dIcqzj1a58gvOjiD+yE3VnMzCIOeP+oG1KFU8O3cXm2J52xeuS42Hjw7jj8B12553xQwaGNkO3BL/KRUX6d+kIVkHs6F0Fe7buIe3LNB4rf4w7T9xJZ30nwSOCqS0XWBSwnxMlnrjqXLGEijHKdW/zA1MOiPfbj8s4UHiAyWt6uv+tN0xW0/dMRyNoYB02XUkjzozg5KiTvP7S61x9+Krduf8d0h7/EaWrP59/Fc5bb9mOuY7pR0mxQOTFzVg9pgBDaa9p7yb6Jr47kZztOTQXNTPscdvKAkEAs8qBdpV4f1yXC0z5JIWs37IgIpxC5zi8rCspSawmztsNQYC+xz9H5xnGxalP0qUuY2S/WOqk1ySKj18led/vABwTtxxMfGcieNHd+SxYBbzzNEzPnY67zojVLO6lbsSy7+cSUOmDBAlON8m6K7wUnB1+ltw+DYzKOsXia1YKdxy6g+CRwTz87EoqGg9QrKyiyy0X2Ej5WfGzg4YFYZTb7hcEQaDjZBpxJ0TC9vyQ8whRQIIEx8xoBCGXg88cAqDmSiPTPv7rFto9wnToC++8Aztuaip0CnIjZ8QK3KqyxMIJQYztNR4arGYrkqJ2Rp8UFUo63H5A4/68TVGMXK9DKbfw6NpH0XmYbaxErucJJIKMwSkD8a0Pgzeh0y2QdqUHyedeJPhkOKwDs1yDWumL1lPLijMrULuqyf49m6MvHmXlxZXdii3XYTQbKfEp4cDTB9i2RMzf+CT4sOzgMjyiPdA+qyX0cihdmi7y9+YTNc02tjo5+iShxaFMXzOdotgigoYFoW/Vo3HT0NnYSb9z/dBGajn3xzmc1jhhvVY431TQxJHnj3RLof8VrttAJCxPsNv3/v8CU3UDSrORAwdg8WLbseuPvEe56OUoSGQ2MUv8knhR6nzIX3s49gZHX0e8Zg1F+LAaVWcLAG5hbn9/0A3oauriwqcXSLg9gfk/zrcbT7sINS1qNAGBjHzONm/iFe9F0aEi1O0NuNbkULC2jOH9xuHg7UBNRg1ufXzY3XIfXU7eIJHiFCoe5xkj2pgczddxacozuLWW4ll+CXN9MtBz7vXBA3GtycV4VPSrvvH7cvAW5yBdpY6qC1XifH0NAUMCaClpIfXzVJ6oeQKNm4b67HrOvH+m28/RMScVmVMKZtfFmNrD+X7R9+hb9EjlUpR9nTFktuLRKMpr5+Ra2f2A+Kxf75QHaHI4TZsuG5VLNPN+mIfKRYWh1cAfy/4gLvdWQIzTBUGgKb+JHyb9gBOQyxAGDchjRWIhJ3wcCAoIRntt6yGxWrDoDJj1yl69S8+tFfOjfgP87MasUgkSQcKDnz+IudMEc2zHHXfX89ifC/h01acIWmN3DtbUIeaJ9sy5iEnI54EvHqB+kBnP9zx5svZJZCoZm6+bFwKLNi9CJfEFkZ+jtawVYc0nTI1O5Jfbsmj0aEShEb+7jroOGnIa8E7wtlO5uRFShRSryUpjXuPfkn2+vr688+67zHrhBf40GHol/JKBr7hZn653bNJomHvbbf/glf/34K9yfv+n8F+y77/4X4NUKiU4WJRLkVglFIcVs2dGCs8EPGHzOmeFKz7Ncxh5LJLYqz7o39bb6Mt7pEh4aOdDbFm4hXrP16hvX4S3U09FuM65izZtK4NOVCOT7gce6B672avuVJ+BwEUsJgutZa00bz1F/xOXkI99FJ/6nUhU/7zaT1/ewqLNi8jol4F5gLhYVaZUkv5tOmNeHkNq5jvMvTK710orQYABFwdgkVkoCy6jzbsY6JEUadG2cnboKeq0PyPV98M5wFbqqjXYTIP+Cs9YRmD6dAjy4X3gdUhcmkj2lmzyPYZwYvy27tdbbvoe7j5397Wkoz2KPy3mweMP8v2d33f/TaaUdftB1U1bTpr2uM37pn2VhsZDQ9+v5/DaHacJKEog9sRaWmYvgNE9m60KQxbNDtkMOj4av7KKawFlT4JpwqEJyF3aCZ2Ug+quLahie1g5s96MwmhEZpERVhSGa+tfaJP8BRybK3BuKMZ6pS9gK29ntYgRT8qQFJpDbkfroaU+WyRdgkcG45mjZHTlaJzbnAmsDIHX4QO/D4iYEsHc7+didPOhWS1n7p4RNAXbdgqpot35fvn3NHg2kHh5M81FTsAKmgubu18jdVNirDGya9YuZvT7h2wzYoXxA5kPkLcrD89Y+y6JVj8Z5gI9LuU/ooxcAagZ8sgQpArxO3dod8CgMnBw8kGGhD1KxbkKNo7byLR107BGiN+JT60PU/dPxdLXVi88fXQ+h4f+SZemi5oACepAV5vx0hOlmPJKAEg6+D6C8PI/vq7iI2JCb+n+pX/5moCcI5gvVsO3z6Bv0ZO1OQvfZF/6DJvIc/fchU9VKEHZYpLg3EdiMGdoM8Dly4RlnOeP2+SY1U4Igm3333d3/kSHQzNDc56mOdC2kqu5qBkJUBU5CpPKEe+I3qsYb5z7bsTUj6dSn13fq5Sn0NmFRa7gyriHefMO+wDO2tGFslNPk19f2t2CsBgtbJ6/GRAlCycenkaTWy2HJhxiRvJCm2NNtSaGnB9CvVc9paGlXNZfZsENvpLtNe3E3xbPiddPsOPOHSz8bSFxt/TeqXcz2mvbyd4iyu4s+GWB3XirTwwZE1YjEQQEQcA3ybc7eV97reFPrK632HU9aUK8udrPjcLAXdT61HKbj9jlqW/RU32pmoK+jVyJOo6zyhdX6wgkiEkMi8lKl7MvpYmzqAycjdwkR2ZWIggCglWgOk3svnCPdLcxw74Rw5rX43fkMImZiZhWWbiRoLEg8O1d39Ivox99zp8EZtt01SozdKzevJrLiZcpihPDsBsTehEFvzEh7GtGzcklNFccLzlWAoDgLFZy3u0M8pAG9tYXg0zC/if2M/sXWPuoHLPCjEQQz+e6F9aPU3/Ecs2rqzqtGsFi33XadqQKr48eIubZjcgcxONMGjGJrtv4KQAHJx6kz9U+IBFJbO+4HjJ71sdLcDXLUbvm0G5MJ6emlWOvHMM7wRuvp6Noa3Fki7qLxskPkzjRtsv4wOAdHEp8B3WXmmdTnkUwi/PJjfNgYu5dJC5+k4FTf+VzYwA3ztP7Vu/j/LrzPF71OE43dYCGHQzm1a9epcqvisz4FvwXuNGQ07OWG1UmZDILLq0umDtNREyKIGKSmNjS1evYPXM34w+PZ/TJgXTUdzLm5TFUpVahdFSyu349pd5foDC746kT16Pio8XI1XKChgXRMnUx5eUQfOIbwvLUTJ9+A0npqeDUqFNINbD796GYM5qvrXn/jOyTWK04NxRhrOylSlItI3VAKgMv9l4cYqyow6W2mYKBt+IbqMDRB0a/NJqCvQV0NnVxPXnvphtBsHo0p949xeFnD9N3UV+cFyyiNDkCpRL8M49BbRe8Ng0Hbwc6GzuZ8U4AE1TP8s5z73C532W08p6uBalUypYRYpKzt3nE1NaFd+lF2px0CII92SeVKemTncC0/ZNQrraXtK7+egE/ubRz/s7fmOfkRdZvWVz9/SqzNsyCBiOXT/fji8B8CvtUIAgCt++9HUObgaBhQQhCPlZBxuLzExn7ezBOY1vgCRj2+DAMqHgw80FKQ0uZJf+Dko7+KG8IMTxjPW2kYW9GZLaCpJZp7J3+NSAWExx96Shxt8bhN74PjZ4NnIkajbZTS0NzJinrijny/BFWXlxJ4JhBrOkqYMyBZEKK1mDWP8+R548A4D/IH/WYqyyZfIiLhwcSnhGFqdPI4WcP4+DjwNL0J3FS3UWnZjvJOWqSjo6lc7kO1VAvVM4qLA0tyIGs+tk8cXYTvuVBGG8x8rb32zbn75vka5eINuoMhFw+gFtVFp0uU+CB/mT/nk3K2hTULmq6+ij5w1OUi3Ywi5mVr5K/wmKy8FDOQ3ZkoaOPI1m/ZXFh/QWmfTePo9P287xKjeLsSPI7RO8iXZUOn0Qfflj7GflfT8DhkUZ++n4q7o4S+EyUskpcmsjbr97KMjdx7ihwm0pAqFjcceb9M7SVtzH89+G9En0AZncZu2fs5tDEQ3w3VUwetde2U5tRi3eCN12jR7N9+DsElUVjUDfi19+P0uOlNOU3MWTjTGJyYpCb5RRGw7CQSLSe2m5S28nRi0pHFQWRBRwdf5SPkm7nsZZnuPzjZRy8HNAc/YVLdY74aZzxL3TApHFGPtCXqqjRGByc8LN4MduxhUsJHmxXFPNwrFhwJ1VIkTuIRRcmlYP4+ErE380lyAWZUoZWH87pkacpi4nljf5+NOQ0kLcrj5jZMVjcLSz5dQmXki5xdHIxco2C+qx6zn10jvBJ4SgOt7HwuzY2Lg+hNLQUs8XCilMruj3Pxm+NQ9IVyvd3raPJuoP1Qo8vsNJVzalRp1BLYN6+qcScHUrX/Z2oQ/2o7ONHi4d4j0glMlR6FWqDAqvFylfJXxExNQKVq3hdvcUc10kcuUmOxCCu1WoXNTFzYvCI8UDeqiSp8iiNIUcxqcQCE99+vriGuwIw7+sRKDtG8OET77DaYS6N+aJ06bxN8yjX6LgSdwWj0sjcHXOplBoY9/o4Uj5Ooe/Cvrx78D0K3d+lqXQKDW398PYQz9NssHDx51z0HgE0OoUywjIN5/x7GTY1lODgHrm41WWraS1txSPKw+66frr8E3deuJPJIyczI1hUU7gumf1k3ZOUjH6FU2GTaXFcibejAriTgQ8OpKupi8a8RqaVTKHdoRO58AkNV27h6m9ZlB4vZeyrY1FKtCjNnkzZP4VBqYPofKiLwgOFnHj9BAAWlQaNvI0jdYGsP+lNULwJVBKs8p4O7Wkxebir9PzQZStnr3RUEjQiiMhp9h5vAKmfpVJxrgKNu4aEJQnI1XJu3X4rriGulOhLeNXrVRJGJDCtaRpyjRzvwJ74oqzYRGwI7Nw2iQnZkSTWJ+KT6INHtPj9LeizgN+v/k50eTTlQrnN5zZlNbFwy0LSktM4Mv4I0W4iMRw8MpgRz4zg9Lo0LiRrMClMWGXWv+w6/ys0BPXHrNTiorcvUNUmRlIW50erdxTB4WIh0JEXxfnbO8GbhtwGDj1zCKlcajfHdtXpiD67jWa/vtSHDESpFM/r6MtHu7uTm2eK1hB6Zz9k10KB2ojhSKxmzG2PcDl8J56Sz4EFCAIofdxoCErCszxd/N5+XYBLiHjgqpxVGNoMSGVS/HK0hKYOBszo/C9yM9lnVBopCy5D7zGbYVG2ifDrxJZJpcKiFn/D/av305jfyJM1T9LnkozhB+5CCKqiqUMFr4gd9EpHJauuruLZAzOZFdaAVCqw91g/uuLbkfmIv3Obq4ZZO2dROTeNqiJ/bjkdg/4RA61WJ6CZTPUgPo39lNEvjiZxqb1Up9zYiVNDMV0uvoDts9ddtOLfFwliIdOEtyYw4S0xtjMMceUbr+cJqAzATRCLW80GM1aTFYWDgvDjb+PoXU1lQDtmJ1sCXyjt4r5vVzB2agqWqkhyroYColdp4JW9uNY645OUR3WXO/v7fsjUqSBTilYcn/X9DK84L/rM79NroeXxA8dZsnoJBycexLpYjN3z9+aT+lkqM7+aScu8Fr4K/4qn33ualLUpdmQfgMKkwKHJAaPOyKVvL7H34b0MfXwok96dxJqn1iA1S9kzYA8xg2Nw8ndC6aikYF9BryohNyN2biwe0R4EDQ9CKv3PrCD+Dp3Ofqjb61HaN+h3/5Y1kcNp9YmiyT/eJuZw8nPisfK/9k39Oxhqmok7+RVmhZpnGp7E0fmfyxL9MvMXpHIp09b1TkgfTnHk6uj7AHjtphrpIQ8P4c+6oZS1uRKQc5jCPcf411unObh/O/WVDUh04Bh8F77Rj6NW+9rIcG4av4mWcxWE+vbBrUYsZjBXBXCd7PMpPIO2rRap1QKCFSTYfV/iJho8+9jmk+Z8N4fke5LpqO/ozs+6R7qzeMdi2mva8Yj14Pirx3GrKyf0whbKI8cQOi6UnD9yOPzsYTzXJPJqwxIiCiOYsm8Kpml65v80H1OnibhFcXjcI2Xt1Tk0XY6mTfEjctVb3c/2H8vE+KFw4F2En9xMaLaFybM6cBnswowvZnD292o6sy8i85DyU+ar7M1/gK/u6VlXnBpL8NjzI//6hl5j+LBxYVj0Fhty8zpMRjOCRCAjMYPYMHuFNkWgD6XhMoxKIxq9tdsrGMSYst1Jj07ZwP7J+wkJENf566SqvF7CnB3LuZxwAsd2R+RSMRegq9ZRelJUmYnJE5+9PTP2ECOI6/PBpw+SsTHjb/M0EqmEpxuepr2mHSf/f69Y99gTT4BEwuinn2asQsEdej0+iN18m9RqjhnFvViK1fqX3X8AKcAxo5GN99zzbz/z/yb8Vc7v/xT+S/b9F/9rsFqtVFRUEBgYyOMfPk6zWzNHZzohd7UNZLxNfkw/9DxOTWXoHdvtgvD2ECsnRp1mSMoQgsuDMb1laz6sbVdhtThS2DcAD2y1wo2lnTzw2YOkDDmHW7MbMqsTggBt5W2si1yHIsiH6siRmJUacoffRbyt3/LfQtOhJS47jrCSBAYPEQP4gCEByNVyai7VoKk2E5GRhGNXHDJtHWAbLP9wxx8EVLhw31f3YZldQn22Exk/ZBB/azzlHUVcDXoSgL3Tq5ji0M/m2LpEE5fjdvKbv5KjU61c8hL1Ezcv3Ix7lDvVHqMJaNRS5/A1wWXBGP1sq5f/Vr9cAO96b27ZfAtcs5RbfnQ5glWMCrKc1lKr/YnBKYNRdJloKmgidn4sMbNi6FL5EFXzPBVOA9ArhtNiaOBGKbay3zYzLqWDE5MshAc+juSmSvIhF8ZT65vPq5NSSItp5RHvnoRx8IQoTs8YR17MMyz5eQlRBf/MQ+I6CgbcSvL+dzBcyedmsk/mqmD9Q6IsWXzprQB8Fid2m95x5A68r6oYcHE8mfGZyCxi0Nde007GxgymrZtGmttz5Ll+whf3uxAmtZVBdPP0uyYFCAmZCagNorTHddNlgJpvs5EPFJP4NxOz/w7e8d54x9t3lwGcTdyDd0s2w8+qaRtlANRc+eUKCgcFY14aw4NfvITOoYQvHvgCKxacg5wZ+vhQfJN8sZRWEVEQQZN7E9+s+IYH+9n6AJhV0KEUE/bjLxUQO9f2s7fdvg2hS5SPrQ9OttOG/zsMe3wYScuTcA1x7XXcN/8kUqsJyUyx4rL2ci2efTyJmh5FV40P8TWHiE7ZhLbtCjCDaeumUbi/ELlazrnkUxSH7qTCvxiF2ROwJfsqgiqxyDrprHYHTZiN3IVvP19yhyzDsblM3MH52VeCge3cd+PGZ8gjfx0mWQuL6HtqK4XJC7jZzB7AdOIsiUdPcmXsKvSOnkjl16SHncRd0OHpRor9tlHvXc+kPotsjp00ZxK3PHELRqURlxYXrC22P8aeVXu4uu0qJ0adQCqRcm+/e//yPG/Gdbms68mE3hCYcwSPystYDC/YVNaZ9GbiM+Ppl9GPqIJA6pfW4zq8515WB3mRk+RCRpgoJ/ZTlJh9T/0ilcPPHcYhVM7lfpeZVG7Fs/QiXeUh5Oyo5+z8zfR19kVu7OB0uBsjT41kQNoADPcYkHqpmfLRFPL35GPuMrP11q280PmC3TnHH2gn4GoihZEtCEDB/gJ+mvoTUTOiCHxtEmUhZQw9N5TwfAcEAbK2ZLFn1R7mfDsHQQEdDh0URhRicnPB0GZg47iNBI0IYuraqZjyy8iqjWfUnFN4yMzk7crjl1m/ADDlxEPMPPgxn8VUcvfI98hJuIDcYRMKrQKTSpS89m1aQEjDShTtF4kpiaKjroNpn0xDrpbjHuVOa2kr7bXtdpKX0ckDOd+/iE9/nMAYmRJeBMfGUiJTf0Me4k+jrJysuCxOjzzNALP4XJz/9DydDZ2MfXUsxsBBVOnT0ZZZaG0uIn7xGI69cozomdFoBoTh9fx+ojRVfJTxGVc7BgM9O1kBK5O0sMjTQMRH7/FRy1xATGyauiwsW+VMkHM2t7jkY8GKxWBL9Jxfdx6gV/+BCL84rlCDf7U/G7/qZMoCN27Zdku3DI1LnRMNzvV8+MSHbBgszu+n3j2F2kWN321+jHJeSlWEM1mS+3EKcmXASk8GrBQLbEzp1cw7PQ+ZRUlAXRYwgk3jRX/MV4RXOOX0KI1hdTg06hh/dDqWmJ611mwVY5Vn3GDQnfuZFJnIcvk/TzK6V2ZSGTOekNvtJcAkShkHJh+g1aUV9465duNtJy4QdSGV3+6ox6ScBUyh6mIVV369wti3JhFU4k27Qwd3fzsJS2wr1qXiWqt0VHJd9LvR8Tg+bXsx1mqBaZgN5u7nXW3oWZuFG7rsrVYrIdNCqDldg2+SvcuDNMiRDx7/gKDyIBryRsJI24KJyQ2vUl1RQGV0J65uLpi6TPw27zdkChkLty1h9+jTmOVd9E85zcDZUbRVnqfyfCWmLhODxt3Jt8UG6hxOMOHoWFB34b2kZz5pGhNLdqAnBtVvdDh4IL+WQOpq7uLUc3uI7e9PXjT8brmLAaTbdX2bukyYOk2oXdR2XXChxZ5E5Pflap+rdDV1oXYTv5/8PflMECAteBz9MvoxY88MGvzKSE4U57LK85VEhA8ksjqc6sD1GJUyJFIp83+az7bbt6Gr1JHpJhAdk4+HXyPrhx7nXpeXkWvkdNR2YLUK/J79NJltiXw17Q4+CKtB4/Ms+lY9+mY9Xi/dx7ndTTzs+wENh6NQnxqMqaPn2QoaEUT56XKaCm09KAFMnSa8ysTqY2NBGdC/OwEdMSUC4ZLoE7No8yJUJkd4A9RuaspOltFU0IRi5zH6512lPLoPkjZHYDJVF6q4+sdVJn81hxXO8Iq3Hj5fw5S8Fxi6eihJdyXh5OeE3l+Bx6y9vK1qonB4EU4KsRjOwdsBB28HsvoVow4o4Xj7bJoTBhI0wAWLycLFL0X5sdyduZxfd55xr4+zK66xCgJWmRVH02j6+Ymxe9mpMrYs3MKCXxbQ7uFCUNGvjPt9Ox2Dz8AzC4idH4vaVU1HbTtLfl3CqRGnaIp+idD+UqouVrFh4AZGPDOCsBETOXh5LC2eO4FyBEHg0reXurv6b2c0p4NieOSdz4l6RsIF/WzkwVAV448MCCvegMr7YT6P3cYZh/noPKMRBIHvx3xPfVY9qTNeBokEybXb7+yas1RfqiZxaSLTjryNQCutAWIHWk16DQefOkjRwSKmbpnKtnnbaHZrBtwxGqFgnxiLho0LI+VsES0+oUQURpCQmUDnA53oqnS0lrXiGuqKRWbBVWXiiZoIzrZ64+Tv1J04q6gW76cHXeD+kYVscTYg0zx7g/ywOD/IkDNz10wSriRgfcuKqcvExS8u4p3gzSNFj/TaPdGS1czMP2eSlJ6ESW2GF8V478L6C2I3fJcWVXsn1ZEjEWSK7t/+epV+ZUgDZpNICpsFC0nLB5GwJAGZSoautIIjU+volJ+nKLyIpOi5xM6N7fax8lgjY1nmUibfdoADWa9S2TAXWMC697poe3kLroDj24+wuM2B3/cuRuWppWyctTvucwlywSWo97io6OMinvrxKb6870u4qQheIpEgtzihNHteu18tNBc1o2/WM3fTXL4Z+g0A2X2vEFKmp9nYTPySeAr2FnBu7TmaThtJOnyZC8ntpPVPw2KycnbNWcLGh7Fk1xLuHv8J6sb3We1SxCe/rKZzuh7nR0LIHn0f1S4/kh52JxvLx/CzPId3S72oqzd0n5vSUcmKU717UIGYFF7fZz3n150nYUkCuTtz2bxgM/N/mo/EX1z7MhMzGdx/MFoPUR0g8+dMEm5LwP28hFd+XkJDRCdzl45C46YhdExo93s7n3VmWPYwPH734O5Bd1N0qIitt25l6idT6VJ1EZ8VT0loCamDUvEM6klWJ92VxJmtVVhlLVilVpYIS3BrcLuZB/p7SKQE5h6hfWQ8N3bKAHSY2yjy34qvdi5BnmKBgdZLjD3qMuuoy6xD467h9r32fRFWkwXnxhKcG0uwKNQk3CnmMGZtmMXWW8SOVoNpK2aHfjgYo7qfqaqYcbhVZTH0TBDtmhiE1L1ElhoR7lmCQ3QAJf0CKOk3h48+gsgbctieMT3fS+7YJg70/41ObSdTNZ/bnVvquARM8haC6wegDHC1GdPn67n/iwfJi2lGJW3CanUj/rZ4So+X0l7TTkNMF7f57aG62I8/ylwRBAFdpVgIJSCgTE3kanspAdGlCB0aBKsVeWTY/4e994yvqtzavf+rZ6303nvvIUASeu9FujQFBVHsvSu6ddvbxoYNQUVQBAQR6b2FElJJSO+9l5XV5/thQmCZuB+f876/55zfeff1CTLX7HPec9zjGuO6uDTjZWKPvsngjMEcHVSFW60b/mWBmPRmrvpORuHWhf3VUlrKW2jMbRzwVoVc3o5DSzmVMVP4803WNnZD83OUhOQQ3HUAUFFyqISmvCaG3DcEkxqa3Ztpdm8mQOdPW2kb60PXA/Bc93NkDSqizbGSC6kX8DRaG9lH2aRg05lBuFlF/CPbWV3lBryOQ0MhrrU5lAe18o9Z20jMewmDx3qu6HwRLPM48apYPN2U10RTXhOLdizqd04KJwXFocW4N7nTW9YL0aBr09FS1IJZb8ZJ44RJYWLrkq2cfvh0v/Xn/zKfTodONr+0mXfmvEP+TpEMasxpRJAKdNuKcd3L5pd5U/Mm8fp4zr57luR7kome91/nWux97GmvaKckp4SYkTH/nxF+RSlLAYHUARSDb8RnPc7+9Dj3L7xtK2ujIbuBgBEB/RQm/h1qL9VSsFZUv+l0DUby3/EfASLnRBI4NpDSw6X0NPSQ8qC1Kse/29yWaVtwaZFROWwtZ2SXuSb9iPFfKHhUp8MLkXz5pvJfHK98n1E282ma9TYkBXP5y8tUn68G6CP6mv2TcAgWCQiFrgv//EMAVEeOx3PiSDhiTfY15Ihe1Z6Jnv1yL2ffO8vxl48z+L7BnH7jNIt2LsI13JXK05WcffcsD5c8zAzTMlSdvQy+PJipfqkkJ4b2SXE2P5mN/QP2BJUHkXIxBYnFQPxScazrbevFJVNO++VBRA8u4BezOEbpOnToO/VEzI7g18JoBKkM+y4bxpwcgiVci8ZNw5B7h1DlARs3zuL0yVdwL79IYPEWGnOWIghijG2wEQtaA0YOTMbELor9S9LMptOG+764j3Np5zAk9y+gbUuUckHnxIOfPkGPoyfuH7oTtziO3G256Np0eLYkYna8xrnh53BUzCR7Sza7lu9i/tb5eBv8GHQlGLksmW/vfgQ/nSjxWXWmit0rd1Mf0Uue39m+fQ0zPy1er+vKTwMpAtwKlYNqQNuLv8Jjjz/OkqVL+ebrr3ni7R8xGjvQqB2574mlfLd6NVu3bPm33X/pwCyVirfeeAMvr7/r8Pd/B/4q5/c/hf+Qff/B/zYIgkBrayu+vr6UhNXT4VCLV9sr/X6nbegiMO8PqiKHUx05uZ/pclewhaOuR4m+Gk1xWDGruc9q+aKtozAok9l890sEdy0HbhJjUkGG0miHRJAQVhyG0qBGEMSKCwCFlyuVXiPJ93uaGpctaHkB+HuVQM7xgVya8TKD971G+946eESUtPNN8UXloKJ8lJ5tae/xxAfPY9RkWB2XIECPnVg1d3jCYcYEzKXqXBVn3jqDjaMNOs9O1n62ljMjzpCdmA38+SMjoBXgWHcwtqMdkclE8ip+WTwaVw2OO66hsfHBsXEzQ4/n4uBjfU0vbbhE1Zkq5nw3p1/bccWsCnrye4jOj0awCEikEtxj3JHKRQk8h4yjJGuSmXpgKu0mCw3ZDZz/4Dw+g32w2PTiVF/BubRaPnnoE34Kf8pq2yaNQJtzGx3OdvQY/fsSFjdQu3QBVwy7uE0KHnJxAL0BiwW0qnIA0ocVoXB95W/dp7715Upyxj3E2Cn9P343qodDi0OJzusWZWgeSuHCxxfQuGkoGtfJ3qGbafRoRCW48Qn/YsqHUyg9XIpUJmXsdguTDA+w/pH1eJqtO/ucFO44dafSbpfOD8t/INQiElS3SsF1HKhCFa9izq9z0DdUwVzrTs6/gkln4uiLR/FN9SV2Yf9gRWVx5cyIM9QFhzLVUSTZJ7w1oU9/v813KM0251n2wzKEaa2Yh5oRLAIypQxjfjd3/HAHO+btICchB8HB+nPi1x2HQVuBQ6cD0SfepW74DEi9SaJOfHsif2xtJ6MtmB5nvwF9p/4Ktu621F2uo/xEOUPute5cEQRwqc1FajEjxInBomAR+l4Rs86IoldLTeQ4FPLrPjUPpvQF3BWeBVzzzOWujXfh1uKO8Oqt2xaQmCVIkXIhfCYO2iRuNcNSOaiw6WnpM8DWzRi4q+bWse9WtJe3017ejm+Kbz+/E7mvJ3WhI7DpaaX5WrPVpBxAEhhAfciwPjlMqVzKM+3P9HmcXkh8q++3xW1FVuvKVXK67btRGBQ89tFjWK5a4Ba7v6i5UTR0NVAYUEi1fzVyv78fOtyQtLtRff1nODSV4FybS6drEEjEsfe7Cd8RtyQO/3lDWbBjAb02vVyLqkFhZ/1uin6u4vvk0T4TuVS8Zg7+4vPb4COO41W6DYzImUNddhqj0+JwGxdH7dlylPpuXFtcqfKvQmqRIrkul5z2aBppj6Zx7oNzVpV4t0LVY6HKr4qDMxt5Tybr836qOluFt9mC1CzllwW/4NO6lDsF2LFkB4JZQNeuQxet5isPcbLo1ZmA2WCmLqOOuow6JrwxgY0rtrM7pIR/PXsfyUoB1VjxvFMfScXcaGDImTZaDOMZHyBWW7wpWBj/2njubTqCWW6m3mUHQU0PElIawqhjCXTVdfWRUzUXa9ixZAcT357IiKdHWJ2TR3ww027LQ3Wom5N1wQCEXNmB3NiLzYgJ5CZco6u7C4Sbldt/PPQHAGNfGYv9xHhmRt/Dh1092Ep24xrh2pfwraqC6Spn7GhgfdsmrhY2cyvZZ18u4bneoSSOzEblrOVfuptFOxfWp+PQGExs6CmELhvuzvDEKdSakB776lgashoGnOAEzE1kkWUMEkHCzKviNyd67s0ExfQfxtBjM4ijs13xU0dQe6mWI88eQe2q5qFFDzHo3dl0eITT4u+JVAHnPjjH5S8vc8fBO1DVSQjJTqTZtRmFTkyOqxxUfRLD8tJM/KXNbBpXyv7Az+kN29i3X/21FlZ/tZrfJ5/m0qACsiJexcRz/N3Q3L3qClKTHoulP9nnrPREInHFoDTgVmrAYhaQ3tKleib2CBX2RzFYuqHaDX3XWNyi3Ljr9F3YetszbV8qnQ4R1Pt74Orjx+gXhjH6BVG+6r3X07Er+51il3RO33OEIMtE3gTS/5XO4WcOc2JFHWd8vgXgoc/ex8nDFa7z5YIgkPRGEvHx8cgGyGro0WOWmVm0fRFV2nNwt3WSRnbqBIF52WRMe544JwX6ju4+WVVBgLbBWeyTuPDzyzvotB1P2vdpZG3KImtzFrJhE4lwy+Ad+zq2n7gdY7yBhpwGTrxyguR7kjkjf5uxI77mI3fYk3SKbZ1i0nzjcPGeJV9JRqvRUhpSiqb8cXpqnwfEcbi3rZct07ZQk17D6vTVohT2LTgytYCygGxWbl5J1aIqIqZHkPJQChKp2D1w3+crkFpEyex437kEjxPfv+pz1QiyINwqL7N37Lv02JlZp3yRqKXxxC+NR9euw+Ppf7K6axduQe+T7Q0SuYzl+5eLUpwtWob8/h5dUUFsTWjmYmQzS9RKLn95lsNPH6bl8TB2T3yP7xwy8Vxkx9jEi6xxfokX9S+ibdaiclShtB2gbB5QuWjErmxg1FSxGnnI2iFEzIzAM8ET1Q86bjt7G+5N7vTYiYRA3OI4Kk9V0lHZQZlHLoLpGj6V7XR3Wcj9yYfsLdkEjQniX25voQ68mw+TC8i1/YDU4FRsnODrtK/xGexDzygNz0pPMUmYyB3RE6kwinru+i49Fz65QPBns/hlzfe8FL6H6uHjOKBbgUl3c25w6vVT4vFel9e8FTaZPTy7+VkyhjbSO78HsMU72ZuZX87EZ6gPJ3Me4lrsVlwr3mB8ahy1l2rRteqIWxxHk62EnXN30uTeRHKteG/PvX9OvF6OKm4IcMcXPohXVTPO7s50VLbiFu1GR2UHBgOEDBFb2ufHvMXZy08BN8e0YKcsnFVVWCwSihsTcTdZsJgEmvKaGAg+KT7ou/SYdCZCrrVj21FHYYoOXbuOkEkiQWrSmZAqpGQnZjPq5CiSd6bScncXLmEuBIwKQG4jpydexqlJCvzzPEnOiMTca+boC0epOltF4p2JHJt7lerok3z54hoUbX+Sqq/R8uRHXxA7+grht2+g2qcIhZ2KtvRCwtMvUK2RUuEBMomCktASzColUrmUhwof4ux7Zzn73tm+rvQ/o6emmyGXh9Dk1kRDuNiVFzo5lEnvTsLOy468z74n/ngdv81vxFsvFuzdefROTNe/L+cn5tNmyiHpShKygDakcilV56qQyqT4hAYw/NoJrvo9TVHEu0S7/cnzvQOqCv0x6JSUFQbSrhD95Y6fURDl4EmnQw63OZdh3+uPh2MVSkkHra3Svrjv6o6r2HrYEjAioF8Rq95Fj8qgYu6uuTTf34xblBuPVT+GXCVH46bBuSqb6G537j36Eu2j2miMaGT/I/uZ890cBl+cwuzfhiNIxW/00pCn8BRET8gDjx0AwA7Ijculxa2FTXYqnEOd8Rvuh0KtwDO3gXbnFl7VqtGt+AmbyHv6kr4ygwnXFlfkznK2ZyTg8FsavkNuysqbdCa667vRuGkG9MN0i3Ljnkv39BXl2HvbEz03Ggc/BxobG7nv8/vISszCkiRe6w2JGwDwGuSFslGGvCCUt9d+hN3aXVSfr+b4uuMMe3IYoZNCcT7mzLir4waUynIf4cmrL98M5m90I2Z9n8X+h/cjtOuwcbVHZpYR+c9IThafHFBi76/Q7exLU0Ay6gFajE5veJm0kl62L/uGocIZQOzQSf9ItBBx8HP4yw4jtbcTV0euxqv0HAa1I67Xw9HYhbEE1gfyvtf7jDiQx55Zb5J69E56UucjXO8e7XITvyGBFYEoZSpUPSKpfWsCv7OynYq6DnyG+KBQK6jPqqezupOwqWEYnI3U2tWK2+rtf01DGh/lbOQoinxeZWzTDsZy83oN9hzMOV0eo094AB9jNryI2kVN3k95xCyIodRXSnHsBWZPhDklsJGNDL5vMM4hzgiCwFtPvIdFasGkMEHcaX51exKhCZBIUPcaKQ0uZbtPLXHRmZwecYQnXV9H66hCrrdHpW3DNcKVuCUDV2fbtomkR4dH/+42bVMPQ867MeT8OKr8HkAQvub4uuNUn6smen40Qq8Juy47etW9WDD2FTkDSGVSTo/Opsem8Ppfbi47/spx8l4/ye0vHSAyPJdVDXDq+nBi0y2OW5fSKolWW8iIepWLh4by1mVv2ivGc22POK7Y+9gz+N6B5/7qADXHxh3jnq/vofNIJ0wXZQe1TVpKj5TiFeiFX5UfE2dOHNCWILgumPreejxtRQIkYmYEq9NXY+dlR09PD+6N7vTY9nC07CjVndW4X3MneHzw3+rqA5GA+G7cd4QsCSF6xH+vEPvfwavkNFKzCaVycr9lN8esXjzLztHtHIAg3GS2C3YVcPCJg0x4cwIjn+1vM/JXsJgtSOQyBJMZl/p8voj6kIU/L+jzev+v0FHRwZWNV3CLdKOzurMf2SfR6/AsvYL/1YP8NDeK23fd3rdM465BXlhF/ZUnqa77hFMWM6k662/jMgykA9N1O9n+ewiT1ryF0l6JX5ofTTZ+6I+LSkYNwWmEam6SnDqNMzbaNryLTyGRjLK6hiASv0A/P2qAiuMVyFQy1M6i3K1gETAbzASMDKCrtovtC7fjH+3FtfAMDk86zPSIV0mckUj0vGharrWwadJmnty1CP97trN25GkWuD1MZ00nRfuKKD9ajv1lW7ye/IZT3tWcrZpDd30373u/37f/EODa/DbSxx/gUvR5Toesv3m9TU3MSHidoW4ZbNh5D0Y7M70WXd+56ezdqbhrHes2MiC0zVq+Hf0t8cvi++ZAN6DS2FIQq6XKv4qYASrWL2i3kRf4DxJCn8HOEo2hy0B1ejWpj6biEubC7B2pCCZv3nvydQRB6MvX/L72dyYdfJKsiY9T6S52LgrXPft8hvowZ/McHsj4jDPOYhy78Pet+NuLy6etn8bwp4bjO/Sv5WnNRjM7l+3ELdqNkc+M/Et/3T/Dy8uLF158kfPpLwIQFAQviP+06v4bLihYbbHu/jthNPLWG2/w2OOP/619/d+Ev8r5/U/hP2Tff/B/BA5NuURIkZLpe87SFR8Is0XPPkGADqkz2aOmcWTwZLocegG91bpe8kg82qeTH7OPfPL70V558eX0quroVZVTxS/Aa33LHELd2XLXZZodLpMfnY9Z7sibwocobZVikjo2FG13CeWe6wkuDUZlqP7b5ySRSEAC+SNWMfc5sYrbOdSZyR9MxjPeE8sZAaPEyMFZeoKbrYWeBQGiytdQ6vUhp0edZpTKBTtPbd9yi0VAZpbh0urC0h9XoUqzriT0ypAzs3Ae86edQGWpYozT58BaJr87mR1LdxB+6Qit3jFUxU6ldFAEQwdbD0A3tLJnfzO7T6anb9+CRfQakMCrUnHy9HHEx7hFubHi6ApGnBpMflQ+m1ZsYkz4SgLHBLLgpwW4RriSm1dD2KWfiHQO4axbvZWPAkD9ODO/D96I1Cyl3HMzFksbVrJ9NhH4Nz3N0vJv2f7VLDqfb4Dr+eqGojosnW/g2O5IrU8DrS0Dm3P/FWzba5CajUjsgvstM3YbGHphKEmZSfjW9qLv0jNt/TSmrRdlGCyOGhrsxASNYBE/uDcIA4Cy0FoESyd2XXa4WKwDwsbThTz63jR2zdGRlZSF7joZeOt1910/DHONmbi8ONT+f78qxKQz9SWabA7YEDrZWobFK0+Pb9UUcoYqxeDVIrB9wXZiF8Wy4KcF1IePp9T+feb8EoPQqqettI30j9LxSvRCGmzDnll7qPGtQWaSYfmTr9PYM5OZfjaYnXN3old28mfmNod46tN/IKL1NFemPfe3zwlEuc3j645Te6m2H9kHcG3YSkAg4XoMHLckDtcIV1T2KkpPnybxyEkujnZH5nYXgJVfxg2vzYKoAhy6WqylxiwCL77xFLmxuZhlZuJzwjC/Z+67V4JFQGfnilGpQWHQIsu4CAz92+e1Pmw9glng/qv33+KFI0Lh5Ua7VyTRZzZS+Ju8H9kniwyjuimMwOzfcI/3AoZayR3LjXI0Wg2Lfl5E0vxhcIu/gaHHgFuTG1qNlrPDzjJ0qPUxJyxP4NhXx7jzuzt544U36DUO4Bn4F7gRRJYeKh1wuaajFqlgoTpmMnKVnLaSNprymjj24jHuXjaCo1M6KArcRp1PHR96bbBatyOjhFF/1OMaOQ7/mggaFzZgP9yThGUJhEwI4YnUl0m+nExBVAHfL/+eVfFj8R3qS/iz88l47DwBVw+gV+kpDi8mKymHLzRKLCYLmZszsRgtGLVGhj/Zn0gB+ORu8ZlV6wMQBNH75cFrD6JyUJFe1MHLr71Mdnw258eK78Xsb2bTVdtFwvIELB9nAjB351zCSqLRvK9h0Y5F1GXUIZVL6dEYSG3TstC+BbNaReDowD7SLPO8jqKhSwiIrWZeaRQ9RQHo5uvAX4Fn+yyKvd9AbnJAIki5MugKDYEmngsTj9WoNdJa1ErMgpgBZbY6S5vpro4idek2rjWK17oubBQqbTtxw5IJ/GUnL514iROjT6BxE8PHkIkhuESIhK5c2kuwSod9D5gt1mN7T30XJw5NYoivlvk7b6P6NuuiB9erMv44OYOwxGJWFTjSIhEJ25wtOZx7/ywRnOWXsAwyWocS9/1YmlZad/aNedlaUvdWaI29WGTifZBc73htK23D0G3AM8GTzOH5mHo7Scx6COkYM/U9otX4xLcmYtQacam7illhQ4dnBEa9DRc+vkB7eTuCINA4xsy3sW9hlpkRpALv8g7Pdjzbt+9hJ5OwMQtE3XaVqLgG3lTIsZgslB0tw7HWFvsue8p7FSyzaMjrVWEymoD/utpSEKB4yO04NhbhV1EPWFdKCu1G7vt8Jc5tKkAvJqVuIfu6/ExkuWSx8tuVeDeCtknLvvv3Mfzp4XgOCeDYhAx61FWolcMZ7WX9rLTWFhGVJyM3qhaz3Iz5+jfPI86DpLuS+MOuEoNKvD9SHFDI/p6/J4DSoMKp3YmDkw5y17gp/Zabo+M47fojF+OiCWk7go2TL9M+ntYnbfWE9lF8fT/HbrAUVZAvEokEmVKGVC7F2KHl8ZD52Dl1c//bn3I6pJAvBoljbuDYQNxKlaSUDsUy7RKz7QTytHMA6DHfvB8GpQGvei9ich3Rt3Vzg+xrLW6lJl302b3RtXErpA4CBVEF6FV6Hox+EMEiMPaVscjVcrp6ISsxjw7HZs4PO88jQY7IbeTcc/Ee1K5qtr93lYgr53GPdafbXtxHW1kbEomELwd/SUJrL2aZE7896Y8TEgSLgHeyN1KFlKz8WnriKnCNPcpjTWIPlYCA71Bfhj05jN1lV/Gw6eatAH+eda0iPLAbiUyKTCkmF1sKW+hp7iVgeH+1B4lUilEtdiZdVw3G1t0Ws96M2WBGUWdkUOYgvl71Nc0+ejYhFpyFTRM9Xy+OzKFQ8QsOHQ54mIfznK8D0XOj+3xNHCsCOGfbhUItgLKa3jZXLEYLKicVAgKvucKLLoeBw3yUGQIEcnzdcc5/eB5vXDGsfxQ+fgW3o+6ouzNQPKvgjkN38P2k7wFIujtpYB8WhdiZOvx0AGW/ZjNoxDAOPHqAa3uucefROxmz1Ze5Nc+ye7kN0xIi6ajMJ/uHbMKmhyEb6kF2YjYeDR74Z39LY0YKaYti0bhrGLp2KL98W4V7SS6usnCCc+uxmSDhzDsiARA9P5oLQYsI8huDxSJhwgkvNMJHhGfOIeLcBapip+CmEeUJf9r/CmVbBOT2RUimhDPqxVEUZ/eSvPefdLsEYjYuRaaQoW3SUnuxFpPeRMGQUXiWHiLiwo80X1uFX6pfnz8McinBDY9ill3ApDDTq5Mw6vlRdNd3Y+g2cKH9N66EPkaLZikSt6dw9FRTdfamVOJo3bvsqr+PiYsPsbLKHV2Hjk+jP8Ul1IUR781BqevhctUk3q/YQJERXrAI6Fu6sWutQnF9KJdKZGQOyqQkvqOvM3b4k8OxcbLh0+hPGfuPsYx5yXqcdx3hyTtPvYNBacDVTpzHNBc0c+ipQ9h62NIW1E3RyFPEZviidXgCeJ999++js7qTx6oeQ0DAtseWObvn0D2mgZ7GHjaN3oRziDMzdtxFQM4JuoxySr3EMXfz+M2UHytnyodTqBltxH7Cu8hc4Jd1/0TQih1tQdl70HQ2IHc10vjxHdSsqaHqjg7U/hZudUHfvkCUM7777N39EsWWqRb+yP2Dafun0dPUg1uUW18RHoBP9i4UbgqKwouw9xiK/wh/3GPd2b1yN4EvxmKjs6FXI8ZpFouAg78DMzbMoKOqg/or9Xy96mta3ESSQWYjZ83lNZgNZl6VvIodtmy493N2uFoYObGOpwQputpWPEuvEdpQz/SW+yl5+BTZwa1ULvmR54Nves7XZdSxccRGNG4anmqyLuaE67FmlFtfAYFfmh+VZ0Tv66B1QSgNSuJz4rFfb0/PsB7GrBtD4d5CImZEUFok8OUD103QTF3k/ZxHycESTDoTp984TUdLB1vv2oriMwUzVs/A0GMQuxwEkEnkyC1OGOVtLN2yFHfBHe4W/Y0kUgltoRCcX4N0ghTbx21JmNxf/vHfwbUmF62DJzYa237LGnUlDGpMYuWXqUiGiF5YziE35+wT3pqA2WjGYrQgV8utyEpBAK2TL6XXpfdvzEeaC5qpvSwScRaJBYPSgFmhQJCIBQbh6VvocfLh0MRDNLs1o4/2JO7ip6QJ0JFTSdCVyxjVDvw2QezyeqDgAdwi3fgi6QsAnu18FrlJtHJQ6VV4uPeXzHavuMTYqii8ahMw39XOLcr/eCV58cesQPS9rxFSkcoCqQSfob7M+nqWOOaWwLL6m78XBIGZn4sFGyaLqS9+uAGzxYI+v5SgzBzOT4jk1KD1+MvsOKyFVrOAIIBK20bc8U+pDRvFg0UP/uW9yp74GCDBrBSLKLobumkracM9xh0bfwc+v+9zJh6eiNKgRBBEP7bqc9Xo2nWoT3fw5K4n6bbtpt3diMubLn+S/hPjy1EnRyG/RefbLcoN12GRPHfqDIbqIs5qJuHQI8YOHTHDqPJNI6htMU9mjIbAkxz7ZRz+Hr0o7ZTELIjh6i9XkUgllB0pI3p+tJV8Pojf9FaXVn6Z/wtPjRHfO7WzmsjbInEJdUHYI7D629U0JoqEjdlopq2kDbWLGlsPW2IPx/Lld18yO2c2baVtOIc49xUsFZ4v5IHPHiAvJo+Q0hCaeptIL0in7GgZZUfLGP3yaMa9Ou4vrzfAxU8vAmAX9Pdjwb8Dz7J0ZCYDUulfk33+Vw/gVp1Fs1+SFdnnlybGNDfitr8Lv1Q/YjY9Rd5ysYC2q7qzT9Hi78DYYxTjGAcVdxy6o99ySa8W/6ui/+iNosEbsPO0o4susmo+4BTCX0ompgL7MDP69/f4R/2jxC+JJ35JPP/60EKmMQZHQzNGld11slqC0caeXnsPtE4+GJUaHEvKgSCr/EfweDE3VnKwhNLDpYRMvGkt1JDTgFlvJuOrDB4sfBC1s5qG7Aa23bYNn6E+1GXUsSRjNl+sqaPOpw6zYGHLtC1UnKhg3o/zEHQWdLUe9OSHQKABmUzGl4O/pKehR1QFkkn54dqznLZ7AC8syJQyJFIJchs5Q+4fwrn3zrHsais284yMDG7mtJ2UhuwGfl7wM61FrYALzs/V8NXaFZzQQr3raAQhsO/4TdaCcFbI3JRJc34zhq7+Y6DSQY3W50kmnEli6rCofssdswVmZc8iI7WH8ObR1F4upr2snfSP0pn64VTa/BNoUV7j3g33Yow2E/tCLK0lrTj4OiAoZBht7DErZERci8DVIH4znAKdUM1UEfumE73hybT6jSCwWIvUcB4YgVOQ04BWMLfCYrL02au4RbkRvyT+3/7+7+Kxxx9nxsylzJ71NU/Uit1/gQGOzL9T7P77/1tH3/8p+A/Z9x/8HwGTrAOjwgmdWopEfpPk2LwZvt5/hTMT0ggvDGfQFT+MWqNVFUJibjKOOyRsXpJOi1tLPwLpwrACutR5LPx5IQqzPdxinq019dDsIHpyaG21qIyijKdngicPFT7E28uPEJCxGdsgWyYdmoRjb/9A/q/Q2lZHheY+aoPzWGz3I4m4cvbds6KvUM3jKDvAt8OX7Ij3sOV24KZ/iCDA9F3ONDsv4as1X2IWTHgPFiuL/Yf7U9pylU8e+oTg8jTGnAjALso6IW1fJcc/J4Gj448y9Y8gJOE3SUrxwwfXYs20eB3FzjEGta+1d0nkbZG4x7r3ebfdCocCB4bmDOXKoCs3r2OTlsqmSiQyCV+v+YkuTS0dTh1M9HRC46rh+LrjSGQSHN5MZOfcnZhlZoZcHIIh5E8B0vVbN3PvTJKvJGNYIEpL3sAJy2w0gc3IbfQIHi1I1TeflZLzBczZPYdtt2+jILqAUo9/0dr6CC4D27L0g1/+IexbKzHf9hQgJuuMvUZMvSbM3Xpm7JvB1Zgi2sLWonG3fg5mWN7E5vJKyhzn4tomJo+Ov3Ic9xh3YhfFcmZMNl2qa7zwxutIvfzhg5vrSpxVXI2+SodjBw4dDmiM4rN9o4oKwFsSwIhr1zg1B5645+9Vj4HYYTJjwwz2P7Ifo9bYb7ltXQ8ROQnkJ+RcNyOXkPZ4Go4BYgLvbOQoWu1P0/igkScHz8bBzwGvJC8kMgkSNyUZgzOIuBbBwx8/jPBgK9xiBacJTSFbV0BhRCHZia8TmzDdat9bt0KwUoPMbCAk4xcEwdpH7t/h+KvHqblQw23f3tZvmSCAWakmIGcv0vQS+McjtJe1s2nMJsa8MoYqxwqupJ0jKzoLX709cAc/zf0JgEGrBjFovzfzMp7no0c/wqS0vs8SiYScuDyq/CuRWqTY6O1u5aIpP15O5PnvqQ8ZhmNjEfI/67z9F1CoFRi6DQNWzslkoLN1pSR5Pg/M6h8w3diVe2UGUododB3xvO30NpG3RXL7rsU89e5TtDu1Y6OzQeixJmYP/3qYBz99kD2z9nBwykGSR/TXnhcQUBqVvPLKKxSHFBN4R2C/3wwEi9FC4orEPkPqP6MxKJVmv0F9E2/3GHfGrBuD3zA/pAoZxRH11LmKHno6k55buxzMvQbsOk14NHoQXGJPZ30HcF2ao8eAa4NIpmhttZSElSA4K/uuVWNIGk1BQ6n2u17tLYhJko5WE7+t/q1vH27RbgP6Xbrqh+BbYGBw7r0Y15iQSOR93YtCSQc5cTn0qnvxqnXAYhFIWpHUt25E8yBmHXgBhamSbkfxmKLnRffJ4QzpVvC2sxPb1mxnU7N1R65ZI1AUXsLDQ1+gZXcwF86lYtQaKPi1gMCCWs5Eg0neiUfKCbo6u6hTN6O0VfJZ7Gd9SXSAuT/M5c84Ok/sfNC82YDeLF7npiCR+PV66FX8UNLk1sSIMyPQeorj8q2TVcvlL7hSMppG+3o8vLScefcMh58+zLAnh2GfFEbj75A/y5dDS7fhHWL97tamGBg2+Ed+NusIf/8RbJLEb8ONTgSA4NOudC8toHPhb5gCrf06C34toGhfEeNeG9fv/dnx2yfM2DeDovAinGx1CIJtn/TSOmEduanXsC+yMOH7q7SOCGb0S3EEjg7E3tceuUbOodunoqx7kTEHM2guWIvPEB/ay9txCnTCIhXE6vPrEASspH0PTzuHSdJIR62UJq2SlFAzug4dP0z5AU2IJ/969FOGaQyU7ZxFzMnBGG7XA/91nCEIYFLZEpi7j27bRHhtjvVyC6hM3oD4vRf+1O1wo7Ph5OiThLctxM7LjplfzESqkGLsNVEbaE+DcyFwN0rjE1SnP8quO3Yx5uUxmOId+fiJ99DZ6HBvdMfFIB5v+PRw/Ef4c3qenG6XdPLi8kifEMKcxPFW+z6+5Di7C3azeM9iImdZV4abqztY89UaDk08BBH9SbP0iN+pkWezaNsYSsJqkK8L7KuM1utBv03OYb+JfDb9K4a5urKgdCHjXhtHyMQQ3nnuN/I+Wc2oe/dzxPQ6oxKcGLNuDB0VHaQ8mELgFzsoypuBdkQ+n//4KsrkTsCFcmUkQkQU382bgt5Gj8KgICsxi43BT/Ydl3OIM/N+nIdXoteAcoN2nWp0wMWUi2j8bNF16HjH9R0GrR7EyLdnc2TSCUwyUcbMZDGTvzOfnB9zmPz+ZK6G5LF3+ff41viSnDEY43wjX0/8Gm3TzQKwqPh8og+NpDAjEu2ibt70/4CUh1JwfSCGQQ9+yxJ7+H3TcEadHE1bdD1Ji4IIGhvEOck5lrOcbP9TfFQ3AlX1IMwmC69KXrM6fo+X72Xtq9bfHbPBjKq7HZW2DVOTK+BM7eVavh35LRPfmUjktDvZrDxKjW8NSkEcG0+/dRqVvYpRz4/Ccj0x2unYib3eSMDIADTuGpoLmrGNdCfoRw/ipXLmvhqLYErg3AeOtJW20V7aTuBnzchsnqP8yR85unMyjbpWeEeUHTXpTGT8WkikuygxeuRSBLYSUVo1ZGIITzU/hVQmtfJRzfo+iwvrLzBn8xx6o2344LEPSMpMwjNQpGdudFco1Aoa/LU0ONRwNeBNuszLiLwtkrTH02i62oTzELFIx67bDq/KKkrLBFa+uKpP+rE47ziB+aXsnX0Brcs3DBvmwOxvZrPvwX19JPGnF77AdtDDTN44gpoYE+YkLZrOBvSyYl40niG7IZyjml9RJwRhZzscqTyS8a+Np/bhC0gtZhyaS7EYLcgUMsa/Pp4RT49AaackN+IL0iO2ElgZzZPBT2IxWVDaK5EpZXS3Gli8yYeS8EmcnOnFU052FP6ey47FO1i4fSHKAiNTz0ylLFKOWakBCQx7YlhfUskvV83WEy8zb8Zc7EIM9DT20F3XTXddN1MDnMme+BgS22ZUZwezODeO7hmdqEckc6UumXo/kSRTYIPK6ImN4I7FbOHKN1dwCXch67ss8ZoOEBehkKC1Fd+DNBeRLHAJc2HCWxPwS/PDoJdxxPYID61/CJlZ7GzyTfXFb5iY6B1xKo1ugytblm5houf7FPwqyoqFTg0Vu2YrLmMQ5Iw88gL1s8x9JJXGXYNQI9BugVIj9Aqgui6xL0ikCEC70kSpXMe+7kouKo4wtzeGmczsO/TEFYmUHi7tk5P+My6kXCA9NZ11o0Qy4fJXl2nMbWTcP8ax97Z0mp2yqfGrYb56BBpXDQq1AsEiUP6PXJ7hGS4nX6bZrRltajc7nzlB1bkqnut8jqkfPk51ZzWJmYmEloSiXdlL1ieXOfrC0b59d9t3s7nVlz29WgQXAW1ZY18CGuCtsMsUKc2Mre9Ccotfla2n+C3obRu4IOzbUd+ibdLyUPFDfd19fml+uEW7ofRQsv6R9Uw4PIFRR0eh79Qz9pWxjH1lLADBcnEM2b9lMunr3ue+HffTWd1J8j3J/DD5B/zxp86njic+e4LfL//Owu3i2O+Z6InC7MDEc19zKmkFWo2WlEDxmzH8ieEMf2I4y8Y/hVrfgUVqYWvkVm5Pu33A4x8IVVVg31xKf4UdEUdnneT0yFMs3L4Q2+udf8dePgaIXpt2XnZ8HP4xHRUdvKB7wUqKvLexC+faSnqcfDFonPri/F8W/0JDVgM1sVq+m/Uv9DZ6fEyLGRQiTnht26sxy1WURdjS4HKeBdJX6EGMG3prWnGrye7bh3Ooc59PVPj0cIr2FSFTyojd7cG0DJE46pjS1u+8vAsPE6j3o8mtqa/gxxpS8uLyqAt0QyKTcvTFo3TVdJG8Kpm4cwpmXrif0kFZFBhE9Zw9d+6i5VoLK86uYHqzN6nOehy1weRdC0U2WIaxrh636kzavW5nenoeG5eFsbfOnZeLXDD2mgjIEaWQ9bbOpH+cjv9w/wG72MxK6/ft2u5r7L13L8sPLkcf6EqDVwNblouKLW+wkRHPjiDytkgcAxwJCZ3M+cjfUXXVYJb1L8qa/ttwml39Sc5IptvJSMXJCrbO3krIxBA6qmpROLyDOayShQen4lkYBl+Dqyuosi4SmBfDaUU4Y+aeZP2SH/CwicfW3ZbhTw8Xu2QlULi3cMC5vLZSy4QjE8iPzkcTJp6fSWciZmEMQeOCcGp14lj7MZIeTmLLH1uY+PZENiSIBX3rhHUsjltMwogEflvwG5Vposz2uffPMe61ccjd5Bwfc5wOxw7UvWrC3cIZ+dxIelt6qTxdSWdl5wD33hruMe6ETQvDc6Tnf/nb/w56HH2xby3vJ6kON+fEelsXzFI59aHDrX7nM8SHxbsX/yUxUnqklD2r9jD5/cnEzLe2djF29PT9e9qXc/6WlOkN3PCgHf7UcCsFpxuQOjmQO2IVRpUd6z6wPrbJ709m/ZVjjCxXkiro+617K1KB0XIl33z9NS+8+CJN+U10f3kAG1UcPll78AGM3gsAcY7p3HCtb13DJTO4Blltz9bDFs9ETxqyGvqUXW5g5YmVpP8rnZrMZizXPUs17hqGPTmMsClh7H90P015TYw7Ng6pRYpxTkOfzO6hpw7h9vUgMn9+n2IbKZMuHcZWaodbpBs9DT1Ezo7kwscXcG0yMvbYWOx0nqhd1LxsfhmAPav3ADBuQgbX2mL5+uSnLHhoGEjakNvI6RqpIL43m4BIsahkazcsEoS+Z0HTXkvo3q/Y1hHJ4t2L+TNu+Pj+WQkHoNPQQa3z1yTXGugs6J8nsK2QEJYxmNMj/4WDcTJSeWzftTHpTDSFpJLnvpUZpWosRvG6jXpO7KrcvO8cV3zuo9WxjHu+eQC55OZ4YzaY8SnQMLtgNhvvr+PsODkRnuIzWp1ezZ5Vexj1wqi/JPHkNnJWnVtF5qbMv1Rd+l+Fh4cX4REvQoTY8vfNN+AxsJPQf/A/hP+Qff/B/zZIJBK8vLyQSCTM+GMpde7n+HnZZcZFPt33mx07AI0ZBIgojGDopaHou/VWZJ+lswdNZx0TD0/EtscW833WiezQumdocNqD0qBEbrZ+5FsbGhlycQgxEdXMcTHwrypXLBbxI7Bl6hZ0B0vwVfkitUg5MuEIYYaZ/F20FJYz/+dkDk1s4VDRHuampOAa6YpMJaOtrA3fy0rGHL+Hb+7+hiLPR4HjVus3BQymSX2a1V+tRjKqFJWDD+HTwlG7qqHl+m88O7g882WmT/3TtZ09hiPJR6iN1fH+p6FUOIpftb1r9xIxO4LX0s5R5fYKAHa9UTT2HgF8OPHaCRqyGpizaY5VIuRWeJ73ZPCJwfSqezEbxK6m4U8NR9usRSKR0OzRgk7ewejcLGYkxmMxW0i4IwGZUkalo4LsxGxS0lOY/sd0DKOtg0SHfAmjSkdR71VPdnwui2TWZOOszbG0+dSifLoO37kxpE68SSA5DnFny9ItRIXW8dAf86i70oDOOl/1b1EfMhz71kq6t/8Bz85H26zlXfd3iVkYQ/Bjs9gxQo5BbY9g44FMAV8O+ZK6y3WszV2Lcc85Rl7OxDtkOMFlYvXTDZ39iJkR3JgEVoX64ONtXf3T7tfFz7f/DMA9X96DU7c7fHwzwFBoFKjs7FAbfcUs8n/DN14ilTDk3iEDdr8B7Jr+PSM0Sdy+ZSS6Va1IZe5Unqqkp6GH1IfTGHHMDZ3NJMxeXxA+MhizoYam/Ca0zVosTuI5dTp0khOXQ5JHiNW29aFeHPc4gU4tOjx1Ga3vddSZjdi1iZXhLrV5WCzz+bsnFzAyALPBTOTs/hIiggCqnlbkei2CRoMgCHTXd+MW7YbPEB8sbbUcmHqAZT8sI7i8DL69OcE1dBvoctFTEViBWWYGBKvJgUQqYee8XxGk4mTr8pAiflRYJzpafOLocA+jOmbyX3p83jr23YqnW59GsAgD+o4ZC4qJSD9GdfTEfl19AKZLVwi7kE/umPsZPMGRGxWm13ZfQxBEyaZWl1bOjDzDtvEXrbftZeTkqJPUe4mltpY/+UIef+U4HSc7xP3ITH3yUH8HTVebyNqcxbjXB678tMiVONddxb6lHEP3dGycbPqSOxVVJmQWNWFFYUQVRNE0sZFAn5sko9OwaN5VvINFJiaW7hgmVvPe8M8rSKnnxNgTfb8XBIGaizWUfJSBpmcIWkdvHIVQAjNtiSocgn65nt+PK+i1dUXd00LYtDByfsjpN9EDCCx2Y+L+BDS9TZgMFhrzGjn33jkS7kjA0dmb45PbGHfAhdk7grD8U0Db0kvpoVI8Ez2xqTIxOF1B0dCXqIsIx6Q3UXq4FKdAJzziPEj9YTIZjj28/88vONhuoD6zni8GfYGthy0JexZyIWI6PxrhzflqLg4/i8LpSa68foWwnOI++VVbqRNjs7NxdRATgBKZBM9ETwSLQMSsiH6eYrfivf3DsNHkAynIDFpcanP7lu2bvg+PztsZbScmaI29RiwmCyp7FdorTpw2phCvVdM5oQ2nNWJ1fNnhMlwmqjl/548c8WrkjKadGc6zrPbZ62phtb6ceJ2UMZOP0+YmVuCOfmk0V3fks19zlPyoTF4PbGJxfCFr2qyNjHbdsQtDt4HhTw7vlxS2yZEz9NJQhl4aSshbNyfIN3w6BCw0eDZweFozUYkO2DjaYNKZMPYYKegu4Ex0GuGycDIGObPGUc38rfOZt0WUrJK3C/hV+aE0KLHR2WAxC7yu+AcgJlFq/RvolldCdgh3fn8n5rsbUcwT4xdtaQO+rcu43WMH2qhiTpt7mSt75C/vy60QBFGeqDpqPLHD+ntNGBxM7F1kZNE3N/5iPdYEHVDz2MXH+OLeL5B61qPQKGgtbuXsu2e5d1gQQ0t2ke/7LBFXslG7yrmqv0prUSvp/0pHWONNt71Ixs7cOxP3Fm8QVWlpL2/H7uglFrKQvLg8zkdOIMK0BxDvt0QioaNAHEsspv6yN3Jne06NzKAyoBK9uX8yo1GShUqvwrnNmRazWGnb29aLqdeEwsWe+u5grjbpKJPtJEloJ/uHbE6/eZoHCx+k1auD85MPsFXrSXR9KiP0RqtOoYKRHZwZup3L2U8w+EwXNuoSIAifInEMmXRoEntn7cWoNGJUGhFuiU80rpp/Wx074o8wvMuSef2l1zGazNgoxWeg4ngFIwSRfA2oCGDUqVF0udTR3GmmYFcBI54egdlFRklYCXG5ccRejcFitBAwIoCCXwtwCnaiza+D2at3cmzvFMydnshsxG1f+PgCfHyB0phFfLfyZ+727+JqQC0S1c1vVmeSDQ6ZOmRyBXU75jI4P5+eB/tXqV/deB5enWP1N21DF/HHPwFAb4iHN+Zx5RuxCExlr8JO5YbaHEfktUjUBicAMr7MQCqXkvZYGiFZTgS3TuJCyoW+IqfzH57n8heXWVP8KM+a7+AfYd9z7ZGf+IfWwPKRdzH8qeGcfVf0KTHoVGxvVXFVK0GiyACWETM/hpj5MTS71rA8+gfy9TF8cN9nBErG9NVYaVw16DtFpQI7bzsUagUHHjtAb0sv5SfKcZJ70enYyckxJ1kU8QwgymJbTBZ6GnvIHd5Msc2vrP5qNd2xJ5HeO5/C3wqR28iJHevLYx88xsWhF3ntxdd4Jvxnelt7ObbumFgZn+zMBvkG2p16sCmrRm8bjouLC5GzI9F36FF89zKjhmfg7XCGN2a3obCZQ+LwwWR2DqZDk0GgUz5PecJCFzlvRF9B5i8eX/r6dFrOFZE75n7McmWf5/OOJTtoL2/nkbJHqHYWfaYbPU9g62FLzo857Fy2kykfTSF22SCMKhccDGkENo0SC2KuF5y5Rrii2GYmLT2NTpdqVNoajHpPJr93s5tCdfksIdpO5DMALDj4OrDktyU4BTmhNeu5EnQP4Q7lPJlrz9VGT4w6E8brOevAxvtx75jO9MFxJP/YgZu2EuMzRvbeuxffFF8CxwSStDKJxBU3rQ5uQNIjIbAyhdirkUxznwZrxO//kWePEDAyAIlEjFU+fvhjQJQLzPgqg8DRYhwRl5dAl1HDocmHGKNS4Zvqy8R3JhK7MJYr7bV8+NjXqLUmxh8dj9TZlyXfLOnbt81TENsZiU1CBY9VBXLo+nhUEzUB19pcPK4FEvrx6yxt9Ud+Lg1VvMUq7puzyfp9uhXSC1ImHJnAmRFn+v62d81eAIY9Nozy4EY6bMWOFAuivP6CnxbwccTHCGbxnIPKgxicMRjtom484j0oOVhC5elKbKulDCoeRETxUKKv+mDoNFB+rBx7H3smvTuJxXtXEaO153PXTj585hm6J/RitzSA/OF3o+5qRGo2ctnsy1ibRzjpHIfKOR6zwczpt0/jm+LL8z3PI5ENHMtHzo7kxKsn+GbYN9ybcS91V+pw8HMgel409YIYf54YcwLv+7xxDnHG2GtE26xF7aImtnEM11ojyOxQ0d7eiWeCJwt/Fv2nn6h7gt/yfkN5REn5vHJWTVqF3EaO2WCmt7WXDQkbSAUaNBH8OvdXdq3b1XdMug4dXR5GDsbsxqQwkVGXQZeh6y/vzZ9RUQEVCbNxqs9H0tUJOFgtT9TM4Lywla/v+Zrn9OKHsqVQnN/F3R7Hryt301Xdie+0hH5zgva8WkIzfsEsU1CUuhxBEL/13oO9achqwDdPg36h+J0UJCYsFjE2yJzyDA6NRaz5bDhXhi0gNDaYyrqrQAzOYxLYWRiLY2Mx6u4mnklP6VMBWfr7zbjKO3I8BT0/0ubcRlBQ/+KwH5Z9i07VTqNnIy9EbbFadvrKaYSON3FvdMestohFUFJJ33OhxoSjREFUVgLe9a6YjRayvxcJSIlEQsrnqwlPLMbDr5H2vTGol9qgHD6YMz2JeJWcI+zST+TEDcaxxJZlB1PRvdxLt4s/FpkcicXC/of3M+zJYQOSfc61eYRm/EJl7FQgFQc/B3xTfJHbyNHqtNj02mCSmzApTAgCuEeJBX9SuRRVdAh59X5cDnsYD/0welt72TprK/pOPavOrSKozBuVzsKxGU6Eqcei79Sj79CTv0P0SYtBwbTka9hO7mavq0ge2Jvb0VSIc7KLazcyt3gSNb6HUMrEXELRviIyN2XiEeeBR6zHgPJ8+hY9Qy4PAT9YECMWzub9nMepf57igfwHWLZ4GY9UPYKsSMaoIaP6JHbdY8XilMu/XsagNTD3l7kEDg0k6/ssSg6W0F7ezuIri3G4z4Hj145zJfkKM6fNxDvcm8rTlcQtiRuw6PbPiF8aT+ziWBobGweU2f1fgSBAU+BgOjzC/i3ZVxc+mrrw0VZ/A1E9aaAcwg3IbeTYutv2m493VHbQnVXKtbQ76HH05fHFN0kYi8mCxWSx8p7/M8qOlOEe4074jHCMvUYUamsJRZlKTo9zfyUFEImxqtrfeeO/IPpuYKW+l49+/JE7x6/g7HtnMRWU4OxxM26VOIpjlVx/k7ysiJtO4vhgyPqT5G91Jw1ZDYRODiV0krVKVNZ3WVz+4jJ1oSOoWZbNM5tisPe2R+OqYceSHdxx+A4mvTELhc7M1P1TMbfqmPL2RH5b/RtdNV30PJ3LIy9kkrFvNGXHPkbb9DgrT6wU91vTydUDZQw634SNbSoNf/I1D50cyo6CrzkRWkj56a+w+/k45Z5yUtePYG32WlZu+JCVDT/x/OWl2HQW4VOtonfQzWIUpU7MTd0o6Pozpn8ynakfTu2LqW5FS0MtwblX+HV2OUueeLnf8uopZjanvMPok6ORSn7HI07Mg2ibtLQUtXA2ZAVadS6Fsz5jRMRs6q7UUX6snNhFsXTlN3LPl/NIn3yRpoh7cHYWzzt/Zz77H92PRSrQ6N5Am/1pKj2+oN4yFZjGvgf20ZTXRE9jT7/juQGJRIJfml9fd+v/Kv7de3cDA6ib/v8Of5Xz+5/Cf8i+/+B/G6RSaV9Lb0S+Aw6tkVRF9B857No6eeXVV0hPSefT+z/leafnrZdPTONwdxqW+pn4Vfth+ZN82IjjTmhtpvPt8rux1Yey6ZZlvTWdzPx9Jk7KQ5jPeTPufBzCewJdtd2UHCzBHGDLW3e/AkCXQxe21+XrBEGgIbsBzwTPv3x5ZV527Ju2j+l/TEffZIFl4BjgiFkvyic0hxopVZxixeYVNHp2wMc31xUEKIsJoV3qT0SRE0q9gmu7r7Fz2U4W7VyEQdAx5OIQmj0Vfb+/FRHKqeibE1HbvseLm19nTMkTAOT/ko97jDvGQWKyzb/Sn0U/z6C46whMv4PjLx8HRF8VxwBHvAd7I/0T4Ra8JpjavFoW7FiArl2HrYctk96Z1HddbLRKNEYXxv+yi+qeaq7qAzj6wlEW/LwAAfHjUxBVQJN7E+sCF1pt2ylXQuT5Cbz99NtcHHqFtzTW/gfpwy7R7tDMgye2YaOLIuXW83aVUhRRRKo9hPk1U9imQf7fGOE6PCNoCE4lOFrszGvKF7tgGnMbCVXZoHdORmoyIDNoMZtsaMgSZTs7KjqQBAXS0CTQFDoE4/UCL/8R/lSdqcKkMzHsVBwmiR9FiWG4+aRg7DXSWd2Jxk1j1Yl6ZdAVvHVJgFidDKL03pm7viHr/mIcWzooLHyaadP+nr68xWyho7IDlYNqwOphQSahIrAChVFBgo2oo528OhlDt5i0CCsLRCmzUCJUo6tzxG2SG2tz1mLvbU/9l8U88+4HlCak0BWSyPBR1uTwVeEXWu1P4tjuSHR+NGZH6wSiRSZHp3GmPHH2dS39v3VKgOi3davnltU5CRB2cSsgQXf//Ugkok9Zc34z9t72CG3ijioDKlEJIhkx9/u5GHrEjrrClCYKU/cw5vgYnNs8EJ6w3r5/y13YO1wi2CWDE53WUYxHrAfNAclEnv8OAF3cOgbCrWPfrZDdQhz2W8diRqHvQtnbgcVk6UfWCK1t2DeXiZNdpRIbR5jy0RQ0bhoEAfbO3N9HUv5avonbR90kgCXeKo5OOIrULGXplqXIq+Uw6ea2a/NqkThIWPe4eD4LJ1q/t/8OGncNqY+k9vOwugG5vge3qkzsWysw6SajtFNiNopVwu2NrSz9xh21Tuzi6mxuB26Sfc36mj6iD8B43fix5oKYBHNtEKuVA8sDufO7OzEvq6RVYU/T/gyUg8O4GPcOnpahDM+bhWdRIWa9ia4eFdeG34XMpCdSebCv0v/PGHEsBrPMzOF5g7jbRkFLbguZmzLJ3JTJnMx1eLctIDPpA9rdnFmChG2zt1F1tooRz45A7h5OwbCV9Dp4IlhA36Fn68ytADzT9gxnkwuZF1DP+T/SmNBwF/WBYhJM16Gjq6CZ5//5PKfGHWNx4iAaemxZqlAy+b3JfC7U9R2fWmZP2NV2/AqOUv/QGtZmrwVEuZUtU7dg52VH6kPWIjA+B8ZxIncMyRvuorn2VyCFoKzfcG4oQDNjLBfHZdNe742tLACLizeCIPCG5marvAp7Kl/+gIZrAfj4Pk3soljRC8ZWwf7yHP4xphB3GQSXA4Ion3RpwyVGvzgamWBCCAewQMhx1laJxzbq+VHELh/Es1+9wSq/GhY7wPPNYJBZD+7OIc60FrfiHNq/q0o7GY6Y9uDTNJZYT0ckEomV9NLin5ZSocrit9mfMNduLobuID4K+AiXcBfS9qYRVBZEs1szB6afRO1hT11GDdXp1cQvjcfloozVe1fT4diDY4ctlh+sBzIBATkQoI6iOK6NwYEjUagVzPtxHsePVhFyrZYoLx+Ch+Wzxiffqlv9v0JE+veABGnwqH7L2g3NXPV/gs/WBjA+8wRmkwX5LWOGSSXQbdeNIBHQS9oBMRFr72OPjZOa6FNf4e6ixL98KA3hJgZ/NBgbJxuibovinSNncWtyo8u+i4tDL+KtFcfR/J35XNl4BYunN1neYpV9YmYibfID3CD7pFIpL/SKBn4DJUQEJxtOjv6DBz59gK7mPJgzlMxNmZQfK2fO5jnEHnXCoWwYHz72ISMLVtBV18UHPiKN80zvOo4uLGO25zc89NMSOoeYiXk0hrIjZdRn1mP2VnBu+Dmi8qNIOvQ+eVFJTBg3lfrMepyCneh0M/BeYDlTFf9krXY1TkFiEqfLJRD71gqGXB7C1Zir1HnXYddth0ln3WlcvL+YzE2ZjFk3pp8Mc1uMK5LeKu7/9H7KBpeRODMOryQv9F16BAFm756Ce5Mrbs1uGNp6GfnCeI69eIzzH56HqeIztWf2HvbM3sPtti8z/bPphM8Mx3+YP4O+X8arZTqS1Avwi1rOvY4w7MlhpH+UjsVkIeRqDK0ZCayYmccm99k4BniQszWHnB9yaJ5hxwdznsUYJmHfUE8OaBciVyuomLGWwN8/B6A+ZBjtnv2TYgo7FXWhw7Frq8YlQaxYDp8RjtlgJmJWBFm/dOPUoiMxazYKk3ivR704ioOPH6TiRAX+hY74FvkRnxNPrz3Urqztk4b8MuwjIue50OLgxuaQfMpbhhM2JYymq01MfGci31Wd53v1S/SqdbCyEE/dTdK26WoT+89L+KJ4AwFDekH2GIJgQdui5T2P9widHErxftFfc8WxFQSNDWLh9oUU7SsiclYkUZsCmHR4CZcHHcB8PVOx/OByDjx2AIlM0tcVa99lj0WrQ9uiZfqn03GLdKOyyoREFopJkY1ZbsYiETjz7hkufnIRk86EMFpJvXc9UrOUy0GjyWvZgb/em7yf8gCwQUZ9mSdL7bWkLrzEuKw5N98NLGivhxxBChNz7aBAEDB0G9j/yH4ADFMcsChuJh9dI11FotJsYfXXq5Gb5H3E1w2JsNNvniZqVSIH54BrfS7BpQoMnUk05jQiU8rwTPCkY6ScTYFvMPrkaOIvfU1t9UoK1+eQ/X02TzU9ReFUZ+7x/ZS4ncvJrHdG8arieqEbXM1uwa06m3vj6ll8ewM7hymx93em91ArjvVNyFxicVIOwVsBHe270NRcRSK7jeUHl5O7LZez757l7jN395uLAMiuSLhro1j012SbA8zAL82P0S+NxtHfEfV3Bu44fAf7pu8joEMkpie9Owl7H3vx3Z53J1sVYuexyWLCK9ELr0QxNjNlVdHhWE2HIxya2cbcqLFW+3bPkqM/tASff26gccMSguJEAudG8hBgkUZKda2EjgNT6XDo/tsyVpIsCSPPjKTZrZne1l7ULmqSViahdFBi520HCKh0KsYdG4dDuI6CXwv4ef7PfevvnpdDcdBBHDsc+Sp0KRFh7px7/xw/zfuJhCZ7EriN3xbo0Qa+wBo7sdho0KpBxC+NZ8Y942n3iCX97lqkLo7EBIxF7qChx0VDj4uoLJLZlYbh7Hnyd3szxduCzkXH8ZePM/i+wYRN6S8RfgNjXxmLvkvfl+BuLW7l4qcXCRwdiNHeyP7yZOJCGvgpOAmJVML2Bdsp2lfEnO/m0Hmmim0XlvLUhncI6brAa/WvUXOxBu9B3th521H7Yi2PZT7GyU9OEj0vmqqzVZx87aRVJ1SVf5XV8TTmNZL3Ux6Df3KmY1wMp0ad4r4N95HVmEXMp/2LvAa8VxJQdzXgWptLj34Ufyb7hpVMRleXT0WYLcL1vP60j6dReaqS7B9EgqvZNwH9sXIqz1T2+bUCOER50+SfhHtVJjbdzX1k3/jXx+Mc4sz+Nw7zyEePsG3xNpwr96BvWAChTgCYrnewhRSrsWm6RlBFHYIQAzIpgkxKu3c0zsHR2DgOfF6ew2bxlecGGp0OsNS2f6djrU8DJrmYS/hzZ19tQS237bmNNqc2Oh3NGG83krgikd/v+52C3QW0jRqJ59g3GS8089Af+7Fc5y4S7khAJpHTEjuMLIdkXD1taRhli9LVHqFehkUu6yu+OWG6h4TkVyhWm1HYPUdtpJhI980/DNBH6P8ZoRm/iNfnur+5UWuk5kINXbVd0OLIs28/S6V/JTnxObAGdt+1m5wtObygewGLRY7keueSIAh01Xb1fbNMOhOfPvIzWlUZI/LP4axKI2icgcjZkVhMFgo7Gpgd/QNBMWW4eLbxk14sDpLlZqHsbubShJkIzeOY5P0VYQoZh3vMdNd3U3VG3L6hx4CuXTfgOdnF2rHxro0s27aM8x+dJ+3RNAJGBWDvY09TfhM+QSLpuWPBDn544Qds5DZWMfDvK36n1qOW1SdW4xjgSOpDqbQWtRI8IRg3jRu7F+9m7KaxnKg4gYCAVC4l9ZHUAWWwB4K+U8/GERuJmheF16t/bwz8ryAIoLNzRWfn+u9JB8GCXK/FIlMgCDe/jZc2XOLUP0+xeM9ivAf1P4/2snaMWuP18fYmGnMbqf3yd9SxUzEpbSk/0k1wmid2XnZsm7ON0sOlvKh78S+Pe9r6adRn1rN11lbKjpbxkuElq+USBCRmM55l58n4RkPyqptKO9Xnq9Ebu/i7V9AT6OjooPx4OQW7ClAOS8bpnKh4kDdqDamB4nkrezv61vG/egD5fS/2I/syN2UCEDLZurAb6PueqrRtyP44Q3uFD7YetmjcNShsFZQcLBHtdZyquJJ8hX9GHsY7wptRL47i1OunsLQaycybhNRXSV3oCBS3+ESXHCihs6yBYVMvMHjKeZILR/LPNh2fx39GV00XSjslTTFSHuow8qKjHm1EKA6RN6+QgIUYJQxzrqY8z4NRR4fSe7e279zavaK4Ou1xNm4auABWMAtc+fYK7jHuVmMzgElrIPZqLO1O7eyr2E5ExCKr5Ra5BZ1aR1xuHFo7GbYetsz9fi41F2qwdbdl3Ek3FtkNRpj8NQWW2VSeruTgEwfJ2ZKD5OEAwmaeYNWQIl4tENBpxOsuV8ux97bntwkFHAv6EnttAmqtGpVRjLGH3DeE0sOl+KX6cfa9swx7Yli/PLXFbKH2Ui22HrYDKpH8XQxE5P35XfxvClz9X4m/yvn9T+E/ZN9/8L8NZrOZ8vJygoKC+GOBhlblERIuz0Ef2wLcbCs2qWTkxeRRFlxGk0cTFqn16LKz/QX2D/oQi6yXExMLeUjxNB+Hf4zXIC8W/rwQu9ZKFEox2Bf4k8G6vz2b79xMok8bb4c2ct7ZiCBAS34zAJY/+ZRYBDGgvfDJBfY/vJ/bvr2NpJVJA56f3FXNhdQLuDe5ExEl/iZoTJBIlvjaIyn356z/B2i0GmywJi4EAU7FJGFQNGMnyWXyoFiQ5ABQ/EcxxlgTM3+fyeEJhylzG8+JqinczzM3r21zK4ZaE8kZT3PF/h1GXv+GPV7zOEV/FHHfnArOp2rJTMqkzbkNJ5vrbfduGrTNWtGTD3i241lUDtZSFXfNuosd2h0oq5V91WG/rvgVjwQPht4/lEc/WM21iGs0+vnhG+SNU6AT8UvjcQxwRJtVw+PvP87xscfJGJyB9E8G7jUTTexK/Bq9Sg/XvQduxZlR6RilXexXz+bQobm0X3sV4kRZiB6jKOXzU6eEHQkn0cfKeJ7vB7w3A0IioSp2KpHX7cocAxxJfTSVyFmRtFgEKtw2EJqdTVKGFx2Vj/Bsx7PoO/XYONtwOuN9LgXuJ6B5DV5SsfJ38e7FGHuMqBxVhF/zR2Lx4ZMJo9HqHyDv59XsXrkbe197bJ8byvTfp5OZlMmloZcIMYn97iETbgZUiqnRNDi+warPnqezNhce+XtkX29LL+tD1qO0V7Lm0pp+7fouLS70qns4MOUAiUZ7dO06Lnx8gdjFsWJF46ufcEebF9+sc6UtFIrdXfjl9l9Y8PMClAo1gkyDwuyJ3GLHnxuFDBeOsSJ/BSFjLiMcmIoxyLqzrzDtTgKzduNTeILCYSv+/n26jp3LdpKzNYeXjC9ZJYEEARoDhwICrtdf9/il8fil+eES7oJ0h5Ylvyzh1KhTtAWKQbxCo0CuliOVS/sSeUHlQXg2eFk9gxaThVl7ZhCdaMeoACPvXQ7G/KLZyl9R2dve929JTRXQX3b11rFPJru57h+P/MGF9Re4P+9+3GOsk8XqpEhKB6mIOreZ1xS7/+QTAdIJ47liOx51Zz2WljbAmbRHRM9IkwmUJlcstBBzNQajQWu1bl7+zXExsiKSqF7r7tOsNVm8E/fOzeMfULJnYDj4OuAR70Hhb4X9qgEBPMovYt9aQf6I1ahd1VSeruTbUd8CcFvGWuq86ygNKeXy4MuMD15mtW5bbRlxOXG0uLaIHjWtWvCDQXcPwjnEmfWflTExfSr5gQ9RFF5ElOsgYubHUP5TGJc3K6l32kWvooKAtIepDlvIKifxXphUtlhkCmZ8NoM5m+cgCEK/YHnv9F1IBAn1vrtAMpvQyaGEzwgnZFIIJr2JEcdcaPZ4jeaABCRSSR8529PYQ9PGTdTOdMepwwWvMi8UtsP7xl6zwczllAJSVMU8885b2NlKiNoehW+KLw7+Duz7o4CKwAraHXoJLn8PebMOqUmKa4Qr5YFX+45PKpGitfegKyQBtctNQkKukuMY6Gjl53gDAgKBcpizYh/3VXkDb6AwiCS9KiWRYflueBwMpCZ6IoKTMKBq1deRnbzunkuxThzbb3jTWEoFLuhgbIsXz721kvqpEo5kHSHz20w84z2J2+fDq6df4dH1H3CyeQWhNmJRzw1ZwFGpQymdV8uOjEEYvx2P20LrCfjyg8uxGC0DJoRNjpAxOIPmxqHInfpLwXl2+tOhEIs7LILo2Wg2mHEKckLXpGPl5pWcHXaW4+MzsFjg/Efnyd2aS9DYIHThao5MOI1CMgzf1gVIpFKrd3PFV3NQSI189N4GGAN/OH8EQPySeA5kVTLuax/etnclUANxSjCb/97MSBCgxU/sdPEaYMJl0RpISU8hNT0V19bNmHTPo7K5eW0qxvRwKuUrFmxfgF9dKE0rm9g8fjOjXxyN0sEGk1KDQQXvP/E+MZbVuEa49pnTyz7u4MEvH2T7gu3kxufSbhK/K/VZ9RT9XkThmgh2++wGYMyJMUhVN2VJzWYzFTUVBAUFDXhe3eY2BImAVqNFYyc+O6WHS8nZksPML2YiM4K6V40gERAw9xVISBVSLBYBecBb3CdxY0NRKLpgAY94D2ou1FCwqwBhgsDnHjBX1caBpgvkuvjy68pfKdhVwKT3JiEgMFYNcqmJr6ZuYGFVPDAS+9aKvuOLzo/Go9GDqQem0uvVADgBYhJmyzSxu2HI2iH9yL7gyBe4YHyRweeUSBGPeflBsYihF4i8FkGNbzWb1pawcfwjSKQSku5KwjfFl/NHT/PSjy/x3Z3fUREkHou9tz3Jq5L5/YHfWbg1mbxoODzxTso9PsVsOcPkdycz8pmR7H7vGIVvX8KmLIDHL2RRWgrJiB2YZcfKIMYeFHDV6MTsKZvZ6xCNwnEaekc1BcNWYtdWRbP/IEy33MMbUDqqqYkWq0JGXA9hb3TWAVSc2MqgY4VkpwTj4iQSMgEjAvrkH89Nr6BWOEp8TjxKeQCN+S1W8uVlOx15Nnst3y5/laTrpoAnXzuJV5IXDkuicMybyt3u2XziU8Kd5WLMnvF1Br/d8xtRQMvwM4wfreVDvReP1UiRKcRK/lsLOBz8xcR88LjgvkSOrryUEacjGXE6kl5lM0wVO/siZ0finexN1AfOhOgm8eHjH/KiSzblx8rZvnA787fNxz41jsIRD1AetAO/Kj8E5+u+YcDoF0Zz8cgZbHptCKwIZMm2JZjubeL4H2JFeeqjqTxkfJNi95NUNDniK+tAEARMDc041TXS7adn9fXEfME1fzbtTWboog7MRjNqFzV2SaHYtVVhUtliNnkiV0iRyCSUHi5F26wlPzqfwZcH88hHj1A7oxb/Yf6Me20cTkFOtOs6yA5azcSiiQRcbUTXFsmUj6YwtnMsAGalWPB4LfIa3XbdLNXczeUvLosHI4GTAR+xNTqH/Zf9sRjE73zT1SZMehM9zQbm/jqXE93HcbWdyY5LbzJkqRTtlXzCLx0mf8QqepR+SCRQljSHnilzUdpC6KRQnIKc8B3qy5Vvr2DoNvTznXaOcKcmYgztHmEsf0wch0w6E+c/PI9TkBOydgu+Nb4k5E3BUzcMgLordeT9lEfUnCgsKjWxZW9wz+enkY40w/OwddZWDD0G3N9IwKPBA4NKTWrxPiakiR0MBx47wKIdi1AnxJHi9xlObh0cm78Ti1qcPLhVisnU7uR8rqbHEDWkgLRnN7JTPY+SkhKCgoKQSqX8Qyp2gN/wS7sV3qu92Vmzkzm751CdXk34tHCrDpr7Pp5Ko0cigZWBdMnscPBz6Ishou8cwqXWdWTb2tLlUA1qOX5JfjxW9Rgf+n8IwPnU83Q7JCHo5Mg1Uu46dRe2Hrbk/pSL3CTjSmwuM5QRLH56B+VRz6HvtSAxW3Ctyca1Ogsi5rK/4WnsHS+gsFNh42TDiGdG0FndyQe+HzDsiWEMe3xYvzEDYPJ7U/qkriNnRTL0gaF013cjaZdwftNstCOzWDx5HsZeI83XxPfaLcoNSYIzMxNfRqky4NVjInNzJkeePUL0/Oi+zqnslGzOVZ+jrquub37mleTFqJ8nc/uJ2TS7N5OQlcC6R9fx6kevkrstl1Ovn6LD04h7kzhmy8yyAbvO/woSCfQ4+XJ15D34u7r3W+5y3MTswrmUxEYgjS4GwrB1t+Xxmsd5VSLKyLd7R6PpbKCpRcat6WQbDwcq42dSHT0Ji0zZNx+x97YnaWUSx148hkrrjF+1HzEZNRjGNWOxOOJUfw2TQk2PrZTSgNO4JA/GcGUOSQLoW7uxbWtHr3HCp+ACHwVls2zfMtxj3HnL6S30HXqebHwSQbBFcv1bZTL3vx5yixT7VmdCS0KxqK1lW9Vhan5Y9gMpF1IIrAjBYhS/1TZONsgUMlJUd5F+OZIq+yJ61QnI1Qqe636uL34rC5gGQJAKqhxBagPGtm407Z2URwVQFHSEbqduzuYVIwgg14iFg4HZv9Hr4Mkdh+/4ywLDquhJmBRqWv3EIiW/ND8W/rIQ3xRfykoFLidfZtCVQXjXeSMI0JAtFvbWX6mn4PwJotJP41FyH96KOTg+7yhKQQY7YeOixl6bgMLkirpbhlTZjdLWjsW7F3Pg8QNIPizi1THFhBi9ad+dRZdZnBtaQsKoblXjiBtzPb9nfsAPvP/AE4x3NKBbpOvzO1eoFYx5aww9jT19sqs34OnoyYQhE5BdkGHjLMb2GjcNXbVdtFxroa22jXs33MvPi37GaDZiI7eO//dM34PORkfZ0TJOrDyBykHV13ncXNDM3vv24u/jz/jq8XQHdPPzSz9TsKsAjZsGpyAnAkf9e1uHkkMlNOY24hDrgNlstpr3/q9CECDkys7rjEN/j8Yb74pLTS4hmbto8k8Cbo6hEpmEzupO9j+yn7tO3tVv/bISgfY2ASTWcwrfFF8CX1xO12cnCcjbz86TMH/rfOIWxxE0LqhPMeSvcGnDJfJ35COzUxO/dAA1iKYmBv8hFlplfutvRfYFjw9G9Z2U+v5rDYgGwNbWlkF3DyJsahhbNptoz6tBaexBkMj6rpHOzo38EatwrssHLNxQXbw1/3GDPD/05CEiZ0fiGn4zn3T4GZFgt8iVFKYsxS3SjdaSVgp2FuAR58Hhpw+zLOhlPl55j/g7i4XG3Eau7b6Gg58DndWdlGyMZ+S9u/h+4X2skZ3hwBOZnP/gvJhnNEtYEnmezjYtSd13iCpMNWL3taHbQOqFQbiaBf7x+L0wGr5wPUFXrQd52/OQN5jYGg4JqpM0etixZWgGNuFjsRSZCczeR5dLIO0BCdj+hdRkQ04Dfzz4B0l3J/Uj+xRetrz+wuv41vgSmHMRJi2ivbyds++fZdLbk5C3C3g1efHFvV/g3DOZp46WsesOsat82vppeNU7UWV2Rm+rxOxRTswbMex/eD8qBxWCl4plS0S55yecl/Fu1nkAwqeFEzIxhG13X8C90R29rYzbf74dryZf+EAs2E9encybDm9i6DIQeZv1vbpxzb5JE2Vf7jh8h1W+8b+D/3T2/T38Vc7vfwp/reP0H/wH/wPo6hIH62sB63FucyblnCu6qmar3/Taqdm+aDulIaVoejQYTdZ65YoGI5HXgoiseIJxuYW4azysTK7PThvOwTkQlR9FZL51QCKzk1MWUsYBm17+UXaaUsU2JFIp3sne3HHoDhodaonKj0JmkrHsh2VM+EYkltyj3QkcE9jXfTUQbhCLv8/8nY4ZYtnapS8u9elLD5M8CsDhSYfJTBGDyfe83uvzD5u5ZyKTD4gyOYIgmgJ7JnriPdibmEGpXBgXR4+jGwt/GoPmUrvVvrXHDhB9diNmQwthRWForkcGMqWMo8+Lvgw6Gx313vVsXLURUpwAWLRjEXFLbmoP3kpi3IC3vTcPrn6QNa+s6ZNTzfoui0NPHkIqk5IxOJv86Hy+uOthzieexS/NDxtnGzK+ygC5hA7HDgxKA0END6OWWldAxnvPx8n8MLF5sczaPRND15+lCgScpNBY60rJYW8KszP6ljTtqeSF11/gm/YQtGGQbCP8typKQi/9TOSZjX3rOAU6MfXDqQSPD6ayNJ/5mxoJKFNSGG1BYadEoVFg52WHXCWnSSigxeE4vcqq6zIqAoJFwMbJBqlMyo93HuDLNV9y26+3EXbCjEuo+NwEjQ3CWNdNysUUXFtcGVL8K8MMN6UAJr4zEYAL3WCSm9g3bR+SkX9f/FqhUeCV5IWhyyAm9v6E8SdnsPqb1SgNN7zMBMwGM2aDWZRdAZw92jg9fxd5IVnY+9rjleSFTCHDZUgIV0ffi9RORlD2btqyKq22remywbPBk7UxDdz18kYkQx367V9qNqHprMeupQLL30xyA2R8k0HOjzkEjQ3qRzgIApwe8TsXYh/Hkr4fs8GMRCJh/8P7yfg6A9pNhJaEYttji0UikvsbR2zkDc0bmI1mgvO8mHxgKicnqzgza47Vti1mC+5VV+gqteXUlVBC05P7utBArBK2a62iLHE2ALKigWUh4ObYdysurL8g/uN6IsRsNFP4eyFmgxmpFLpdxPHrz0TgjfMGiD71Fca9B9B36Tn5+klKDpUgCDBl/0jGnBjD3F/n4pluPW6ZCzq5c/OdBJcF81jnYyz6wbo67QYB6tTmRNq5tD4Zvr+L4n3FoqTcAOhyDaQ2fDR6jdhxdWOi6j/CH7m9iu9WfMfpUacZm99GkLN1xbjuaj0Ldixg1KlRrPhuBc0Z4jjq4OuAV6IX/mczCSm2QesuZ9uSbQgJLsiUMq5VqhGkMiTXy6zNMjOCTPxmCNpehux9leT9b/KB7we0FrcO2L1dFlJGo0cjtp1dmIwWlHZKlu5dStojaWh13XiXF+DYUo+9LgZBED0N1gnrKD0oTtzzgl5G2boDn8ITyBQynmp6inXCOmw9bHGTmZimgfIpAXRMXoCNow3uMe6o7FWo/B3YsnwLq8fmsrBrMQnH1tNT34Wx14jMdPM4Oyx1XBi0lePTy3EMdCT943QOPX2ITyI/ofpcNQl3JPQ7p7opx1E+8gp65w4ueYpebwUjVnFp5jp6j56j85tfcK+6QsKhtzFvfQmJVOyQe9nyMuuEdZQ98S1ltc6c6xW/fyWHSti+aDv1mfV0pTfT9NjzNGRE0+VXg8HBQtpjaczYMAO/YX54+08hYkghcrmZM5/FoT0kju3fjhSJ39T0JNSbFnJJ+z5SYyjhamufWolEgrZFi7G3v5+JoBew6bWhyu1LuoxtCBaBVyWv9iXa7J9Yw8W0Ip5981n0B2vxTPAk5eEUpnw4BYWTgt+n/w7AnF2TaS9vp7lAjFGcg52ZmPAijur9aGzW0eYT2++bU+PXQM31boLqzggEAZqvNfNx+MeYStr5ccmPmCNyuC97KHO+W46+w5qI/ysIAtSHDse94jLaX/b1W27s7GX6H9NxbRUneZJ+sq3igXbbddPloENppyR8WjguYS4IAmy+8wu2LH6NHrsejDZmBEFA36nHqDVi9rLhxOgTNLk3WW1rzEtjeKH3BbLcxRjHTw5ZS37hypxGqz0X7ink4OMHrTwkb8BQ1cwd39/BmRFniVohJmbCpoWR9ngaZqOZK1Oa+XLNl8TlxqHpbEfjpmGdsO56dbSEpfkJtDU6sf0fr9MwQSTZV55cyZiXxyDP7kF4Zw3aLjXaBfuwhKj6EnnB44NJ2+nO+3e8gsUspbpbicRkEqv3XcRuijZ3L86MOEO1XzXHxxxH5nKTOJbbyPEb5seCnxYM2FEgl8pwlL1J8chXCZ8YjbZZy/5H9vcl8n5ZCWcmeRBX8RkuKg/ay9uJXxZPwvIETC5SisKLUPeqCSwPxKA38OuKX/nt3t+49NklnNrUjDg7gt3FKdx93Ivelh5+XfErJQdL8FsWQeS77/DDPQeR11TgW3CEnrpORj03ihe0LxDyXhuvvPIK39aHUGqEYv9nMVnMyLo7kBt7ca3JJuzST3/5DN7AjQl9W2kbP0z5gewt2dQE1XNo4iGy4k/R6y5eQ7lazpC1Q3CLcqNXY6TduZ1To09xNbmGcpsoMic9wbW0O0h5ciTFcQFkxeXyzFvPkLjPgczNmSjUCqLmRhFyzpV7MucT3jqFb47GYboSBIjFATcwODeeFos9mz6bS+BPg1E5qLh91+3M2DCDqf+ayjphXV8sZtKZ0LZoMRvNaAZFcmBaDka5EUu36Me5afQmTr52kq7aLvyL7EnKTBLXEwx4JngyaNUgpHIp15vL8ax3ZPU3q5Fd7mTWF7NYJ6zDKcgJyckmnn37WZzanbg0+BISb1WfV19TXhPOHm1kBsAVrZqqVx4k6rgZQ1YBYZe3o9Zq+Vc7PFznwpq8CGIyBmFu7kXtrObplqexDfEg4sIWYk59SW+7GDsHjwsm5eEUFGoFx8cd58CUA/Sqe/uKT0a/OJqE5QkYeoyMOjmKNuc2tiw7gNrDHm2TlsNPH6bkYAmSHguuza6UB5VzYuwJJGoZMpVInirUCqQWSCqT8ntMKqZJr9CY18hnsZ/xZfKXqL3s+HHJj2yPyuH7M+uQljdh6NIjjwqnNGkODe4l1Dr/TJ5xH2ejRnPYbU7fPXQNdyVzUyZXvr5C+fHyfs+gU7g71xJd2D7RnxX5YiLOxskG/xH+2Hra0jFRwVvPvUXSlSACcv8AwKw3Y+gR1SuOGRZQ6fQ8bR7OKF18uLrjKoV7C2kuaEZb3sX9n9/P6BMp+F09SE9uGaf+eYre1l4qTlUQE7QMnV8aarteSgdl0xQgEh52rWIsbJsVwek/0ljaFsJUm0pqbR1vxn23vDtdtf1jwUXDF5GdkM2+aftwjRbHcLPBjKHbgGARqPFvpjyonKNzkvGduaqvGAgg/7tL+Of+QXB5MnY9oWCWcuqNUxx+5jCr01dTeHs3RyYcocn2Ozp4H71ej6HbwKGnD7Fj8Q5UBiWXhl5CKTWikosdEG1n8xn8xz8Jyv4N+9ZKpjsPY8a0FzkzyQ1LqEik527LJWdLDl21XRx84mC/cwL46vbDPBxzmILrfLvcRk7eT3kU7CpgUOwglm8Jx+I4mK/SvqOttI01l9fwZOOT+A715XzABpLHXqGy0J/Q89F9hIfK8WZB6h/T/2D598vZN2+fKIlvI0dhq0ARpKTLXrzOQy8ORb9ZfD9iF8Uyf+t8dA4m4nPjkQgSPnnoExL+0T8++itIpRB+cSteJWcwW/rHiur5I8gfOpLwnGtICsU5QcmhErK3ZJN8TzI2IwbhWpWJpquB4nTr/IcggCCVYVZqEGTyvjH3tzW/sXHERsqSu/hxyY8URBVwcJYMRbAfFpNA2KWfcK+4xGcP/cDuWV+QF3ieDq9I0Zf6XAHRZ74h6dD76A+doqOiA+G6IpK+4/qcW4DK498zb9sgHtjwLjGX+iupPPLBGh5Z/wgzf5+JpLCXgt0FfBT0EcX7i5E5ySgOL+bHZT/y+cMHkdvZ0NPYQ8pDKYRPD8dSeg3fqpMsD32aFPedCIJYlJDxTQaCAAqpDrW8i21uDhweKqGqpxjj5WxiTn9FZZCMc2lbeDd0I3Oj3us7HvfKyzg3XENqNhI0NqjPX/PPaAgdTkvAoL7/V5+v5tTrp+iu68bWx4mquEVsuG8D39/xPYIgjpGBowNR2inR19cTUuxByjkvvDIKUNmriJwdiWe8qPI0Pv1D5ua+xtDj+1Af/Izma80cffEovqm+WFZ6cqpbxsYuE62lb+Fx9jnxUvv40uUWQviRTRRtEAvz2uKvUhnajGuEK891P8f8bfPxGuTFL7f/MmDcFGQThK/Cl8trL9M4Uoy3PBM8ebz2cVIeTOFk0UlUehUTjkzg8ueX6azu5FXJq+xeJRZl5cXlIUgEht83nOwfsumo6qD0SCm9bb0Ye4005TVhW23L6FOj6b3S2+djrm3Wcuqfpwa8zreiPlNMQgUs6i85/78KQQBVdwvqnhbMBlO/5Z2V7ST//hohmSK5onX0topZEu9MJHRyKBGzIgbc/ubTIZQ6JXEp3bq4VeOmwS4xtK/YDsBnqNg5aedlR/7O/D6f2YFwwy+2xiOJwEfm9Fsut1XR4ptAddQE5v0wz2pZ0PggZDjw9V9u3RpfAW7+/th52eEc6oylR0tZ0hy6IpIJyvkNU3k1IJJ0Pc5+1ESNx7P0PF2bdgL9yT7vZLETsPZSrdV+bswn3aoy6XYJQOWgQt+pp+xYWZ8nomt5DXf98C+mnFyPV5svv674lcacRmJvjyVooag09GkbaG1KMFtMXPxElLb1TfVFapHg0OlA0rXnCa4aj62nHY9WPsrTrU8zY8MMAOZrxJim3Ah12graSts48OgB1EVmEq5/Ijycu0kJr0OqliHo9LhXZhCSuYs0dWa/c7qBG7LxaY+m9VsmIKDSq7hr013Y/yHmE3K25nDxk4tUnKzA+6SM+764D7lJjlFlujm+XofNIzt48M0N6PY50JrxC/be9qwT1rHi2AoEAR68PnXa869Y1L/f9Ag39hgZ9YMvd21+DO/2eeRH51OYdF0BqF3HpS8uMe+HeazNWTtg555cJSd4ghgvdVT+9/I5Vuf/p/lufX097737GhePR3P2kC8Xj0fz0YevUV//d+np/3sxUM7vfwr/6ez7D/6PgH27nib3Jr5b8TvvxDxjtexGonnCkQmkXkil475O7EJuJllcs+TcfvB2zo3W4dKVg9kUi65NR3tFOwAZoYtxtStk3qf3IejUVtu+IdETpNTTk+KP/Jwow6J2UWM2mnHY08Li+sW889Q7dDh2oJGJxxIyMYSQif++EkKb38z9n97PiTEnMEeI++lp7KE+sx6j1ojlQh73ZNzD9oXbMcnF7omehp6+qmO/ai+MCmeqpafQe9njHBLAfZn3AZCXB1Lb+ejsL7J3xl68g8da7ftyzDkMrpf4Z0gmlz9+htok8QNTnV5N8j3J3HnxTooiivp+b0EMZgJHBxIwKoApH0xBppINKLP1450/UvR9EYvLFxMZKEo7BYwKoD6zHplShiHxOcpdHsQs09JiFOX0ai/VinIIy5L4ZvU3DD87lfk7XJCE9sC0m9v2tyQSVOuBrPEMyZmDMOtNwM2J3KKtczF51xF+xymEZz/DMvimYa69hwtlwWXc764l52wcsTnemO6w8HdrGsxyJc71VegOnIBHxyAIArvu2IVvii+dPmbyYvK4FnkNrcdM1C4acrbmUH+lntEvjSbovB3BlfNRGvIJLnoV4ZuXeM/jPXyG+rDq7CrGVF6i0OELQktq0DpLCBgZ0Nf98eXOy7z75LsIUmdWbFfhoqiCt1Opz6wXdbcHjaTDuReLzMKF1AvMCrPWXDF0G2gtae2THroVSjslyw8up/pcNZ4J/Y2xLSEjqOjYy6LtdyKZ1Ira2YWIWRHUZ9QjCPDtlQC+Uugpi88iwEvcXn1mPY25jZSqdayacDtxdRa2HF1OyTUTcDOQzx5fx46JX/KzEfQKuN/R+j64V1zC9boXWNS5TZgMLwJ/r+KlIbsBJDD/x/kDeo8V+fyD2y7chkNmOib9OJCIXXmCRcCYasvbHq8z+NJgIrO7EQSoyxDlDzsqO0grnok08wpqh2cwK2z6afxfmHY/z42ZR7TTJcJS0nlZfZOc7azqxL3qCr0OnlxLu5OgxL/Qx/kLzN82H32Hvq8r5PxH5zn89GHGrBuDWZ2Aa3UlpUlzWft9/3ttbqhH3VxBZWQcYUkR6Dv0HHtJrAp7Tr+OxKw4KgJK+XHJjwyNnmK1rkVrwrvOG43WnurOaqLdrSf2lecrCS8MB2DqganYL7SHgS34+qG1pJWeph6mfzZ9wOVdbsHobF1QaTsw6W3wiPXoezeyC9sB0PRo0HQ2YdR6Ajf9DSTRzvy06Cd61b2UhpQyKUyUF9V36Sn4tYCOiCG0eUUjuc6eWgSB3tZeLvzWgVTtjNZGlHLLcJyJhzEJk/6wmLm5DqcgJzqrO/Ee5I1E2j+JM+7YOAZnDEa/Wo+hR4auTYfaVU23vJO3n36bscfHE39ChUm/hhvP9qBVgzjx6gke+uQhfr3tVy6nNPOy3Frrf9nPKxkecJEZdy1lXsbXGLV3cuHTC2IXU6pI9ocpLITF5VPSEY5ELWfLtC3cnu7N28+K2+g0N1Dk8yG2xgAQHmX/w/ut9vHvtONr61ysOXRBQHdCJGvPjZEj7zmHneLmBPnGtgK/WsgumYVkvQOq0S5U91ZzdftVqs9X4/hyJBUBFbzsVIk09UlmjQnHM94Tz3hxXFIOTeKI5TXyC9Yh79wNNwJjCX2JUa1Gy9zkz0gZuZ2fzK1Wx3xs3TEub7jMmow1/SR57HYIPLv3WfRKPV2vViAI1u+lztJDr003Nb41RDrEEzgqsK9SuaKugospFxl9YjQx5yLobdcx7h/jaC1pRWmnxGIBrbKCq35PoDA7YjF/zaEXDqO0VzLmpTHYJ31FU7PA3b90YZt+lrT7CiHEhdbiVihupfCVQqJs4dAFZyTFEagsA3vlDgwJJqUaQdk/qSV1suHbld9y16a76LEDqVyGrkOHoduAvY89UcVxqPPvpio6FPvYu3D0dyRkcghn3j6Da5I/vUqxg8ytyQ0bQcLxV45z8h8n8Uz0xLLWm2Pjj+GqH8zab7/AxdwGr4ndTx2VHdz+zyjgFR7//hXsg2tJKrae4F54XHyW3GPc+xUuWHQmXFtcUWs1fQSSe7Q7dl52qOxVCAg4tTsxf+d8qlOsJ+eCAE07J7LNq4nSO68RqhI49cYpLn12iVXnVoFZoLBDzfoqGQVKeCxI4N4r9yKYxcIcvEPoMJfwarEX0n+swWOkFuEZAfvryfuakEDandtpd26n2r+ala4341CvJC9WnV31l3dKfvwg4aVNFKUuw2wGQ5ee3K25mHpNeE2KJ6rmDeS6bmw7atHWy8n+I5vj646zNmcthgQV21y2cdvuhQy6EotljYmcH3Osuk8e+egjDm+biMOlOAzderK+yyLruyz874ziijkU7fA86Pon3sW+9DSFAg5U3aJol9kWydwLRqIKotCO7ybw4EbkupvS2/65+wDrMbyntoPw9N9wbCrB1D4cnp9E7rZcSg6W4BbjRm+wjDOOZ/Bp9YPr6mObx27GM8GTFcdWoOm2wVHqSLe9mQDtXEqrFLhWX8StMoPQL+aT7jyRD51k1FxbRr2TPW2lbejadX1jmQxwjujk4q9jcZOI783Mz2cy47MZPPNQAg+6drDeMpyDqadxt9zsVh/Iw/jzhM9pLWplxuczOCdc5VzqDs6lwtvJO61/KEDJzDjO2N1PYmYiFt96XB8eTM2FGipPVTLoK2ekjZ9g8SvhwOQGRoWtwWKy0Frcio2zDRYfGzKTL1MSWkJ6Wjpp0WtZ/sAs8n7OwzXSlcqfPsVF8OOHgHrekpmxSEARH0lJtRNa2zZsJbDeu5V3RmbyafQpHooVK+qbC5qx6E1oHbzocfJBqhBj9/aKdmw9bPsUOnLjc8mNz+WLpC+oOltFxtcZpD6cilFpYMLRCVwccpH8uHxkKjkdVzsoOVhCwh0JOJwy8tBvD/HFmi+o86nDIlj6pMq2L9rOmu1TaHEZwrlpSaQ5i56NIM4pZLYKCiMLsZVAYOZ65Jft6Ki9F6mXF61+HhQGLKbW5SdsdHch1RVg0xGArkPHO67vkLQyibBpYWibtQN2ihW2FXAyNhHXZldczGICuqWwBbPejHOIM7I68Trsn7yLsDrxO9t0tQn/EWJXzZgdAWDx5uKUSG6LG4xJl41TkBPztszjclctx8YewyxXknzlHL1Faub9OI/ai7XELIgheyvsyn+SczZaAuQb0ErFca4magLBmb/S4R5NXWQ4FlM3UnMBXCdULl2C4mIJz2tfwKQz9imk3ApbvS05T+Tg4OOAjUp8tl9XvQ7AA/kPsHvuebo02aReex6JWlw+9P6h/HbPbwB4ll/g9rrbUOh7UERBxYkKKk5UMP3T6TQONWDsNpJyIYG09G461nXQllXPtd0iEdUY5cOi6m9RRq7n/S3DcRu+lQjPUbT4xoOhAImpAbVnPYnyMsKU27nUsJOkkNuY8dkMVA4qGnMbUdrfPKfdq3bjHu3O8CeHU7LvGm7dzXx5u5QPssaj69AxZ/McnEOdUWgUhC5diiKgEs+Rldh62FopEDjKRCLsh/1DSLgSS/z38UTOikRhq8A72ZtH9zwKgFFhRK6REzo5lHsz7kUQBH72+YDneI5tt29j9227iXUWZYdvxCBZ59so1W3uUw4SBpIt+AtIJFATIcoIK3sMgPX9/MPpS45NXk+579OMcxdVWc5/cJ6KUxU83/08FwM+wrlBTLjauFh3bDWcvMagfb9QHzaSpqChCIJYGFB6qJSOig6CK+zZPLsQgLODX+JumxeRSAVKk+YitZiYujeezCQzlREX8ED8Rqoj/KiKnoRrTQ6aznruPns3HnFiXPlC7wtYzBaUtkraSi8TUO+MWdqDvN3Q77yvxl5Dp+qh1t/C8qGPoVArsHW3RaaUIdwycRIk4jU99/45uuq6SFqZhJB7mo4jenYUzUNXVUjvBzp+u+c3tE1aYlbFk5z9KCmxZTyQO4jju0eh+7gdISCQ6qgJ2Hde5e6vVxGx8jA2FbsZdEygs+Ze7FqrMMsUNPsl8Zr8NcasG9Pn/30r5Poe3Csu0u0SBAT1kegWs4UOfTv1Tr/S6NKIzCzei9hFscQuEp+X5vEaPhn6JLEFyxje+TZw3cOtvhuPBC/CLm4lOLqI4CllvNbpRmvxnD4yTAo8wiM4L9uHxKaK2h4xhrZYwL1cJDaiU/LJ7hzMIf+HSEl0QiKVoLRVEnd7nHiMRsuAxZ4N2Q3I18hpn9xOzXAx73J83XEubbjE2py1dM/uZr3zep556xmKdxcTN1Us6s7cmMlt34hFVXqVnhrfGjRuGnK25HDkuSMkrkgk9v1YXn70ZTQWDRmfZuDk6YSjhyMT3pjA1R1XsXXv3/X/Zwx/YjiJKxOpuEUl4f8tBAGk1/3le6taEEUrb0KilNPhEU6vgycdHuH9yD65Ss7yA8v/cvsKXSf++YepOTMB1lp3iQoCNAck0xyQzJYt4HC9ljlrUxaGLrEYK/HOxAG2Cun/SqfbyZfqmMkMlP+Xuzry2+xj1Dnv4DHbQzhdV48A8buqt4UzPZAOpPZf/eZ+gLNATJVoJ9OQ1UDvN9vwCEjG+Xr3uaW5BfADwYJn6Xlcq7PQOvpgH+gNemsyRyKVsGz/MhpzG/s9g4PuGkTGlxlkJVwjO+hx4FO8B3mzfP9yyo7eLDT3qrEQWNyB8P+w95dhUpxd3y/8a+8ed3efYZRhcHcLHiwJJCEJBEhCEuJGXLlixAMxIFiCuzuDDSOMu7v3tHftDwVDJkPu+3qevfdzH+9+r3UcfGCqu/qsqrNOWesv/i30W96Pq19f5eLaiwQ+MYA5n/9C8BUX7H59gK7RWhafWozVbMWkNVFyvIRlWxciUXfSrtsI0lU4+ov7uMbcRgS5mTHzjvFhbgDXdwwkakonHq968OCZB3nt7GaW1MFYIvihxcgFUw3bkswIGg0mhQaFSYdx625+2EovxSSAhMUJ+Pb3xaNPb6C9WW/Cu8abiwMv4tdfHCO8k7yJnh2Ng58DrnGDuWDZjF2nHX62SryS7uRuGvOaOKfV8LzcwtlFv+Fus0D0fLRYkSlkGAs76Px8MVEDLxHuXI+trbiWqb1R2+0vaKM1YmMIJ3VgKgEWcT9y5p0zXFx7kaEvDf1HT0q5Ws6iY4v+ufP8m/HX/vHp2rW8+MILjFQoWKvX4wXUGqr5Zd17BH72Jh989BFPP/PM/+3f/E/8r8d/in3/if+x2DZrG7U5tcTlxPHwhoep8a5hx/yzyGx7JrvUWh1T9k1BapVyM64FhaanrGRjnInL9nsZcHUG/mV/YjFG0+/xft2bW79yd5717WLiY/t54cyOHt9tz6zn9Tdfx2bmEQag4sSZ7zF0rARHJTvv34lGK+ePWX9gVBopTBpCuL2YSdW36am8VIlLmEs3MvjvIQhWJIKESQcnYZDKYIEoS3Pxk4sU7C8AoxFbrS2TDk6i0b0FEJlcKgcVggDfrPgVzxobHlnvSbvdRYqOmriy7goj3hiBoBSTmUa1hJyUq4xU9ZR1bAw10BV5kcmuUnwe2cO7FpEhtnHCRnz6+VAwTCz0ObQ5kHgjEatMTOr8kPIDnomeTPth2j8+t2v213DAgZ8Sf+K51OdwjXBl7h9zuxNProZ+uLcOYsgpN5xiJFReqsQj1oOUFSkcqipAaVDS6tRJq1c0CoeecgdtWw4y8HwaPz0m5Ub/q9zr2LM461HnTq1Cz/QmK4X29bz4F3+j5Ikjubd+FEH20Hf/YIIvxnVvrv+dKE2cgXNtDpZKEYHSmNtI5qZMsrdn479vAtvnitKmoTUTEQT4c6GYBPIf4o9zpQr/rDhOjDqBVWroZmZVX6mmq7EL2zYdTiY1/3r2X4QYe95bQS5Ba6dFZXRE3dmI1NwKiAWo9F/SsQUSy2w5lXzr83/bhG6atInyc+U8WfzkXRE8tu62/zjhd/p7Ut9YwdDzQzE2NwEutJa2dvsVLvpzJl7ubRQ8sZ4awYyjvyPT1k/Du683p88dI/KqK47xhTS/9wEW/w09zi25Nb2UmkBqlWIx90TH+eaKDFOj2p7akMFwl2LKP8Wkzycx6fNJdz0mCDBv28OodGbOT4gFhYL0X1NpzG0kZEwIQobY98MLwgnLFw3t7zt4H6WnSlE7qrGMGkuW8zCUXa3ILMZujwwQixoHBnjQCozL+Imh5Q/2UPgIHhNM5siVBGXsRd3ZCKOm/NvXBBA7L7bH/8MnhaOt1xI5LZKb31YQnL6bgpT53Rvzv0bH1YP0uVTON8t38KRvJrYeFuRqeTcC88bYFziU7IBJaaJ/4Iye9yzelg9f/BAEmPb0NN6c+SYLZy/sPq7eqGZu8VzWPruWh88//F8ymv8e+lZxrPwn9CSAZ8klvIovoa1bhWPAnUKMscPA6OOj8a/wJ7h0PU3ZSwiJ/Iu0spuKnBhRvqk0uBSllziepH6eysnXTiJ1VZE9tIiptV+jPt6M/0Jf8vbk0efMbgpSFnSfZuClgQxMHYj+qS5Q2VOQsgCHxmJCwxvZOmMrr+he6QV8ePT7R/Gt9uXUiFMsUL5M7s4cdj6wE+++3vh9MRGdjQ6z3IhV0gYIZGzMYOcDOxn68lAqlzvTeP4kVb5VGOzo9pLyjPcUixL1LjQrXejq0NCofJ+MTYkce15M6sb9MZNxR8bxSlQRS5ILOOa5lhG2rxE6PpSjDXc20jKpnLCCMOIzU2h9uJUHTz+IjZsNNm42XPvhGmVny3pJ7zh+HM+RM++j/Wkqjze6wtvgUplByI2dKPqEkx+ay8XQf9Hu2E600Q3BKrB11lY6qjrQuGiQttuxd/RxktLnYmfnwoAn4mjMbkTlpCL9tUwGjcll/sAavj0wBk+5iqwtWZSdLWPMu2OQY2aw/5+EB/1K7I+ZPFo5EbiHN6xvoNeDwztujHFsoq9GTm7jQKzOPZ/HtW9FSTk7r94ynYYQK2aZgCARsJgtSKQSnql6pvuZbk+diEzZyG+LLvN2tMhqPbDyABKpBI8XxXft7LCzXBiSxiORr+Pj4wnCLc+W02kknjhLgKM97g1umN4xdyNBR7w2AqXggMIC1+0WMLN6AHmF7dwX9QjRs6MpKBPBRa+7gPOiIzxlc5hFvfM4/xiBmfvocA3EdYI4vzdkN5CxMUNMTCmllAWVseaNNURWv8ObAhxcfoDMzZm8aniV0OwAfM83kmazCs2t98ZqsmLSmRCsAn2vRtPm4MD03dNpDLTi8ob4zkdMjUCmSWbYzTQ8XWyQm7OQGMVEYGtZaw9m1YY2iJRL4G8SYFGPR6E2qImY0ntMkIW4svbZtQy6OIiaC+kwMYGTr5+k5EQJr3S9gkeJGsfGQLbM20Ks8D36Vj3/8v0XVrOVJ2tfYuO9W3GWSggrDEPhK6Dx1+Dg5wASMCRo+Nr9c7yrvZm7dS6yaV3dRQkAp6THcY34gwDbA+xJSsPGbyh64515yWT4EalFipsinNDs71FF2fdoe2dtJ00FTXjEeqBx7rl2qerYR1CDK9aW16nLW0Fwkgfefb2pvFSJIIBdUxnq1gyCcq5Td3UqQ6eFc+qNU2RtzcIaKOAnB6dYHbWWkcg1KpamLWXnop0MfHogi/NXcNrQStb4nbSO3s0Dvi8TODyQsjNlVPyaixuz+d63gZOzN/JhfxtUQatozG3kpUUt6EY/icxiZGPMdH685gk5MZh0RtqC4nDNvQiARabAKutdkDDrzdi2iMhwQSdW824za/st7ccvh8WExMSdtji2rQY+wW+QHwX7C6i8VMn0PTOQlyfQ5hOBzN4OyQN61J2NaLRNXHjxOK9MgykOFm4+9ytbyhcy6o1RDF49mAsfX+DM22dQJ9mS7RzNrhnf4GS8I4clkUg459mEyaEGJ9lUnGQvEBVjh75Nz8nXThI0MojaG7VcXneZRccW4d3XG/9B/jQXNOMY4Igxtwu1To1RaewGBE7/eTq7H9xNQ3YDIdbFXJR+yPTd0zHHlgIw4KkBCBaBqpo6+l5W0q5x5dSoUwz2f4KSkyVsHL+RlBUpWPrZsle9t7utgiBQcrKEAytus3MncSS8giVr1jPttb18kvc5Mk93WnzcMasuUWGGYqOc5z2a6GsHN2QSTDoTv479lY6qDgrGPI1J44D8VvfL3JiJod3A8FeGM/aoOE4cGyfOJc1Fzdz46Qbl58qZcGIm3z32HTqNDonVCatV4MZPN5Cr5cTfH493Rj03S28Qkx3DxEMT6ZzZSd7lPMrPlXej1fV98njUWEFRZgOOAUu6E2c3i8RCxksu8MiUX9kc7oHUZWV3oui2CopMImfI+SEk3UjC8i8LEVMjaMhuIOfPHOZsmXNXebTa02U8tOEhPOs8kVll8DV0NXRRl1GHWWdmnPE+ii80caHvR+hczwPzMevMWIziejQzuQidpJYKtzoarcHE3xdP/H0iSyHtchOnR55GZpZxMzqDob7tLE9ei0+yyOLQHj9C4tnDLBqwnY0frqLTNwzeBqPaAanVjHNtDu+teZCGSg+S3nmduhl6DMMlvP22FIkEgoLkDBly9zTMmbfPcOnTS6wqW4UmoOdYIu5trUisEmzb9Vg7Okk/2M6uz8twCnGno7iBOo8GMhIz6HvzfpTOtox+dzQ/Df2J02+dZuS+WIYWrGHPPXs4PP4wsyyPcPzF493nt9U54X+9kgFJBZw7P51WVRE2qxZQkuRHmfu3ZAa+wvGaWXysz2RueiAVHnpqNbVsnrKZMe+PYeiLPfejWb9nETwqmMGrB/PH/DKmbVOgLE8DRlN4sJA/FvzBvTvuxS3SjQvH5rMvZzsFobPY7r6d3F25HFl9hClfT8HmkAsvZz/JhVm7aE3K53Xl6919ov+K/sj/lDN5/2S2zt/KR6s+oiGngW/jvyVwxJ11jkSQ0OjeSJvXHTZDc2EzI9Z7IxkYSq13LclXk6k9UUv8gn+P3SeRiOw7v9wTNMRGAj3BRl3WFqwyK5lRGxlpeh+AkW+NpKtRZPG3ahrBpZ2qIAeGRvv0+K7cVoXMasY3/xRmhQZB6A/Agr0L+CZOlPqLT48nMy4TmUWGxWxFIpXS7BePfWMxcZfEcVGT5oR/8WcID6xCGeBFXagXdaGDGT4c/P9SQ//rOvfq7Ap+nvYlMRVfcl/kSAAKDhSgb9UTtzCOg1OOYJZ2MSqjiL7BITiHNjP89eF4JXlxad8lXn37VdIT87GRPYf5XjNDXhhC1u9Z1GXWcSLpKB/3u0xjmRflJW50tevpaujCLcoNq2BF164hp02Ng0cd2rASUMiwevtSG+ZLv33HAX8+rPZhkdKM2VFc95YmTENqMeFVeO7Wg7n784o6vx51VwvV4QBBOIc4o2vWUZdRR71XG0NOQGZcBMUhVVitAle+vkptei33fHcPwq3i0s2oTdjVj6AuYxrfJnwLwFNlT1MeO5kHxt9LeGIhc24miuozt8KabItTaTUTPNuInHyZR6ptgU+QXr6EZ+llUgcW8e6QA0ws2YpH+yQCpaKfYH1WPQ5+DuTsyKHsTBlzts7pdU12vnZUjasiuCQY3TUdJIkMNIVGIbKBbw22Xy//mqMPHcUl3IU5W+fgGCjut5Z/tZxar1p+W/Qbv078FYWtCKrsauzCIljQm/Vo1BoC43ruGW5Ldv9XcVtBY9K6SbTKWkUd8f8HQhAgb+Ai7BtLSHTuve5Xu9qhMHRiMtqhdfbr/s7tKDtbRvm5cvou6dtLFhVAZjLQ6JdAWEpPW4ucP3PInr8d58TZvdQ8fPr7UHys+L+U8hz66kjWfSPDrewqBZ9UMXTIPd3ytTvm76Bl6008JqVSb9fKhwc/5JdFv9xqu8C22duwyNpYguiCvZe7F/xSbx1fAhzXajn73lnOvH0GibMj7rcKfSXx04iKEwtU9k2l+OccBaDNLQTbsYNgf8/7dfmry6T/ks7cP+Zi59nzftv72lMxUIqizcLib90pWyjuL8vOlHH6zdP4DfSj8lIl6cmdOHeNJzollEDXeq5+fVV8Fl+mIv+wgbqGYHyrfDEbzPgPEwE5dZl1GAJ12LZYSBl9mR/zn8VislB+qoy28jZs3GxJHzSZaScX0eSwmlk50VgTjagd1QQOCySkcDCbUl8iXZuMX0ExL6V2YZNoA0jIHP0USn0bCdlbSJ4fftfnFTElgogpET0ADLfDWN/F/Zvu5+zQs3RGiuugsIlhhE0UVYgCYidzoLOB2X+4IlGrcP7MmYmfT+T0W6cx6Uy0FvgR7GmlPbGI3FaBy+suc/jpw4x8cyRWjZmAOi+esrchdck+3igXyQ2VqZWceuMUloGDKBJCQbhyqzVi+24rz7jHuHP2/bMkLErAwbe3stb/E3H7lny6di3vv/IKZywWBlh65vruM+jF/viyaM3xn4Lf//n4T7HvP/E/Fg6+DpiMJnFjPvQcWlstCoMMq8mCtkHPla+u4NDgj2CnIuVqCpcHtaDz+QyXvyXDtF4WsuyzaHZpxrMhngWGegRBQOMqbpAm7xtErVMMNm/8geDSc2Eis1eQF5mHv0sL5jp/pNJKrBaB3N254uKvv47MeNErL6z2RYRb89vNrTfZt3QfSUuSmPbj3QtjqkhXvlr5Fas/Xo0mXXzVHPwdSF6WjFu0G8cGf8e5wZ/x1GdP4d4iTjJDnhMHc70eQKDJtYnf5//OSM+vaMprIm9PHm4xbrQFVJC8/yAt4zIo8u5dABIQKDTBe9XTsdPcQ6KbiOAY8/4Y1E5qhq4fSp1nHQaVgdEnR2PxFeFF1VerxX+Xq+lq6mJZ+jJsXHsuWjIHZtJS1kLKlRRRrlIQaMpvwsbNBrWTGq+zu4j2VNP3agoNSjPNhc2krU9DppRR/s1VnrB/ibNjDWQNGsLEQBfSf0tn32P7mLdzHroQBeV17oTWvY+Npc/fpdL57sktmOhivY2Emx02WI13JpXbLIDfOqB8wiHSBp1gibSn8fF/F2kTX2LULcaSvkXffc/ybt3fuIw4ktMVdNZ2Mv3n6aT9mIZrhCtXZ9bx0/RPsUqtSAUlW2SbmbdrHhUXKlDYKPC/+AueslYuJIrPpvpqNT+k/EDy0mTMI9xwr3en3aGJk5MEQjzFwlzUzCj2LBHl0BRGM1KLlGXfLsPSvwT+Yl2WsjIFBz+Hu6KDu5q62PnATqJmRJH8WHKv4xbBSJWviALEIEq0hYwNYeCqgQgCxM05iWObPZVrVuEw3kBVcBVZW7LwiPPAWqzjyKYJOLq1EhfZQJlCQt6ePG78fIN7t9+LS7UtMW0xPOppoG3dA0gfaoP5d367oP9CLDIlegcxmf73Z12ZWonaSd3Ly+R2ZGzKoPJiJeM/Gd9jgyoI4NRsh17dRrurA106KW6RbkTPjkbtpCbK0I/k9HmcHZFDRexcXhV6Ls4EDWgdjARfWY9rgwW4w7qymC141XgR4qrjvqCP6Kr2xWoZ071Qlylk2LZVY99chn1zGQZ6F/vMBjNShRR/f/9e7KrWslby9+YTPDoY9xh3PGI9GP+xKOVrG2vL2QHT0Tl6IFiFXkyzzNhM6j1P0uwkJlhlShmv6F65/WgRFGr8m5+k2Gst1r8VwWXSO/fv/k33c7PsJsy+czx9RDrXY6+j1+jxH9zbg/C/Cp9kH1aVrkLfpr+r951b+XW8ii9RF9QflaOK5qJmvgz7kshpkQQ+O4ThZ4dT5VNF6qBKhji0AXeKfV6KcOJL1pMRLDJqbnuqesR5oLBRcG5YKjcDdhFfsJGQ4kJMpd54zQmmKnIUOrs7hfHcqFza7bu4306U1WnzjKDNM4L+S/MJGh2ERNY7ayC1TeBa38Ok9W9HqlLckbVp6sJqtuBZ60nqgFTSBsASxYuce19MQNSl1+GS18mBUZdpc2zDqT2we8Fcl1GHtkFL9sgn8O3zOB+/+igJcgH3He4Ejw4meWkyGQ3tDLkwhCN2ep4IECUA11gFhr08jKXFl+48U4kMzzpP4jMi6WrqotAUyJZPYeW9dZx89SQj3xrZq9gnj7EnTnmFKWo1G4vEpJNP/mkAVCnxVHulYttsi9ZWC4KA1WLtZgPcjv0PnWVPUTi1jitRO4leFjk7c7j27TWaDw1lwaB85EyhIPtHDv10CG29lpAxIaRf/YaZ9cWkROZTZwXdrXtSfq6c0++eY3bbNaaO+Bmt6lNePWiDS99fWPAXn44Jn03AYrRg792zAAPgN6ovL7nMBGBdyKNIJJJuQ3mAsfsHI7g3sC2yGKtgJW9vHle+uoJMJWPEohEs/2o5aUlpFMTaIggSLnxynmPPi5JouZ3HUVFDp10nUkGBRCYlYXFCtxytffpZ7M3t7Jl3iCvDD7NbJQKO5u6YywfP7eLld1/m03m70CaIfov/rvS0WCAqxaxUdEsHNhc2c+79c7jHuGNRS5FYJQw/M5ykdDO6NXpCJ4Zi424jro8mDOO6JBnH+kIcG9vpqE6gtbSVSV9OwjHImSHnEqn38ObqgHI8HIeT8EACCQ+ISOWil0tIOZGBYdAISpJmEXJL5ODyustc/OQiu1fmkeb2OzTCinXLGS11ADHfg0QiYezbY3F2dkYq7c3KFhBBUuOPjqek8yS8lYCNmw1mnZn2qnbCrjrhmz+eHQ85Y2MehqG9DVOXWGCymq10BpdzWeXBNy8+RMMEDSnvp3BgxQEOP30YYYJAsgrmS+3R5sSgHWwmZ2cO22Zv454f7gH68tIwMZE27KkcVtUMo75eglmhQW7SkXQjDovMQL+EZ+k4UowhMQIQkbrNRc38OuZX2srauO/gfd3zye04Myef1Nxy5m2bR97FHEL6etL3sb4Y2sUEQfjljTS5VLNr+gW8g4bjk5yMVCGlPqMeP6sPS04+xFNzd/KB/HlkKjke0R4svb6UiosVyA66k+bhRKtTKygtIIVZm2ZRfLyYPY/sRTBbGbJ/HauWzOGarY5RShk3fr5C+JXzZI56AoODK76KVF58ECJGHWW+zwuUhI2nU+6Md+E5ClIWoHPszSa3D3LlxkQRAJd8a3kx/JXhDF49GKWdEoe39Dx+5lEi3Nup8hYBbXH3xVFyvISuxi6ModH0j/qT8vQ6qpv7Ys7Ow70iDYCyg0VwcA0vjTnLx8OOE2MWi7Jfhn2Ja4Qr+V86sLnxWQaoNLwY2MWOelHqsPBSI5uWnmVUzsNsum8jbwxp5+XYb8lhNcYOI5e/vNxDUvq2n96MX2Yw45cZANj9upcXt77IjYQbKG7haoJHBXP/4ftFGfODzbiaXdg6bwcTPd+l6GgRV766wriPxlGvsPLdY9/RaddJaM0L+EZEdye0gkYFcVNnwK9xESbzYYaeG4owrY3ixmJ8B/giWASu12RxcdBZJllDGeVQxfauWd1jgkRQ8LKjmhClHp1WzeG0FEIG2NLV2EVHVQcyWxUWhTju3F4P95nfh9w/c9E2aInK7YNbkzNyomh7uI3Y+bEcXX0U9xh3JHIJNT41jD4+muFnh9M+pomwiWHdyeDAuLGk5oxFaDuKa5MrJoOZE6+cpT6rngGrBrCz5Rh/LDvCT289hKyq55zQmtfAS++9hP3kM3jee56DsnpG2yownLpI/LFLFMyVIbiDo9RERnwGDV4dvOr4KjFvzWfHB0X4CWfRt+nRt+l7ec2aOoy4Nrlys89N2j21gAhIaK9sx9RlovV8ASOOyMgOcUKQilJv4/81nvZK0UM6o18RXeZChp9xpDXoBBDGzW030TXrsCaKYMJZThb83erIwohZb6ajRgS3ZMkOk+jYzgVdCDqVC6ZbIEwfTSsABg8PpDIrto5aYgdl4uw7nba2IG4/oIwPD2Pqq2HQM4N6eaNnemfS5N7E2qS1LN6ymNBxoT1YB/N2vkOZ136GnL1GYf1Vqi/ei31WBh2ATT9nvpm6Blt9BM7KmdgEgINbK0NeHML5D84DIJVbcIgr4JSyA5mDitj5sbiEuxA4IpBfJm3FxzEPG/suXF9fR5nj092/69BspE9HH+QRMn7Piib0QF8cRxjQDNGQtCQJO287Cg4U4Jng2Z1gtHG1wdgpAkLqnY+xbsVNRjeJ86B7H3eSliShdlJz/sJ5TkyMYNy0EWjvFROW2+dux2qy0l7VjkEmQ6s04BtUS7m8hZw/c9g+ZzvDXxuOrlnHmBNj0NpoOTDlAFbBillnRuOiwSvJiy61ic9cP6YgogC5SY5UJ849p98+zanXT6FUa3DrGA+cZ8r+KZSYS+AOHuy/DEGAdrcQymMUSJS91x99v3ZlYMvTfLXiKwSDyPzzTRGZQvtX7EeugytJaYw9PhZLWwdw5xxOiUFkjH5KZNw4+SIIIti4s66TZRnL+Db+W2btnIXUKmXG7hnolhcixIogGp29yHQyqAyYNZ3o7D1ud73u0JXVc+nLMvrMjsLex560n9KoOF/B+E/GI0gFrDIr+b5vUmbpi64lgc1TNgOiD7oEKyVBcNzwNVbrJ+TuyuXo6qMsubgEqYOUotAiwgqDcGzbj6kjGgcbBdk7sgmdGEqVk4yDDiU8FVtCUTtIFFLGfjQWj1gPrILAe09+LvZTQYE1MIOf3F/HXCq2uctGTmlgBlf82jirzqYi4hJPenyBWSlg31iCSW1P1Mwokh6+I9X515AIVvQ2LtQHiR6bSjslngme2LjZYGjWEpcVR1yWWCS1PiJw/sPztJW3MfSFoUibjIRUhlDtU02B3Tco7eegtFfiFuWG0l7F7nHL+CKukDM6ca+vtFWyum41B1YeIHt7NmO//JZIFx2ftMD2diU/3uo/Ojt3bsadZrijiT1+sziSNZ6cEw/TPH4kPw4QRRtt3G2YtemOrOPxl49z7v1zzN8zn5q4GnaF72LF0RXoL+vhEVGB6sizR7i57Sa4QtL1JLJjsmlWNSOVSbvZigBaOy26W76LFquFgKEBooeiXEpWaRYxN2Mw+hnpqO5Aaa8k6/cstA3abh/nhpwGnIKcUGjuKK/8PQ6uPEjY9DAkj/37wN7/LiIv/QpAZ54f0BM8ZrWCxGpBbtQRdnkzbR7hCEJK9/HiY8WceesMxg4jY94b0+vcDo3FuFWmI1UM7/F3Oy877JIjcarOJ/T6Dj5zgnk75xE1I4r2cnFe6Wrsuqu3IkD1lSpCblzDLFdRmWlAsEztFjUy68U5avLByUw+OJmukC74CwErZFwImqMyht662uHASMSPeAK1wK/AaeADwAO46uiId19vkpYkcbPSEePhUwAYbZyQyEVpYJPSjibfeFyrMtB01Hd7qv51P2LsNKKt1yJYhe69/K7Fu0j/NR17H3tMERbyPfORCBJsXG3Qt+lx8HPo9lTdN/UQNxKuEl3lznivUGJnxxI4PJB/+f4LgKIXVnPve1/RZ9AJJnk+SmtZK5c+u0ThwUJUZRpOv/4R2zRdFAVKaK8awW/jfutuWyJQkjKTk6v2wNg9vKraIOYnLQJR6uE8Iq3jkRFPsNP8KGXNPgQGhnI+y4xK14pJZYfnu08y6dFejwoQAWzv2bxH/APxzPx1Zo9jMiclByYdoNqnGhdB7H9n3zvLuffP8UjqIwiCOxJBysVBF3FXRtGQ3cChpw6RtCQJz3gvRh9JRqJIYMwb68hpEbrzCKlfpOK6aTirfhTZw0nA67fykNGzovFJ9uGFrVlkNZ5CpyxnxKkR+DVFwFswZ+sczHozH7t9DIhMw78X+3QtOn7o9wNOwU5M/2l6N0vyfzUEQZTufPGFF8RC3z98bgCw12Bg+PPPs2DhQry8eu8n/r8cEonkrjm//1Pxn2Lff+J/LKJmRtFR3cH2OdvRq/WYFCae/tf9tD2bSylw+s3TBNi4cHPkcm6M64dcrsTK3cw+xQGw2reaat9qTIKZM2+fIWRsCHEL4jg9Ko37PCtxVTejlGmBO4gUv4gots7fSpAc7OLzODzwPE/a/ws7TzuiZkSR4XOl+7OuFWko23TAYLqaRESeW/TdCxEAXupgkop+58C9hay5byogslz8B/tTdKSIhM0Szi2Cg3MciWsSN3FpG9JwDHTEZ0gIQcX+6NUt5EXlMQQVMffGcPCJgxg7jNTIq8mNzMGgqGfllytpHtiz0BN+wp6+uUs5stAPx5KHSLnVzJTHU/hl1C+MPTWW60nXOTjpIN8s+4Yl8Z8B4iSxY96Obh8b4S4+alKJlKspV7macpWvI79GEAR+GvoTXklePHjqQezLs5G45PDRc0cZ67aSPnP7oHHRkL9flBqx75Ag117m+GA/5nZlEu5lR/CYYGzcbTgXd4xjof8iruAFosufwmL04K/SjvOrq6ivh4TwKIo/XUDX6npu11PqrpQz/vB4Jo2/yqqIZqYdW88/QvvuEq6V6UisFqxWEXbmP9i/e3OdufU0U/ZNIag0CPdGBUaticTFiSQuThTvk1TAKrndMcV7FjU9iqjpIiIsPbmSerujBJcF4Cpo0DaISYlr313D23U0K75ewZ5pZ7nedxFN5jHAODTOGuZsncPW+/5k7xQR2SW1Snu9ALHzYnsxwm6HWW+m8GAhhQcLsRgt9F/Zv8dxWdpmZlydwclRlxns/QSGdgP7H99PysoUvPoHoAlKpI96L0cvR2JViTK0lZcqMbQZEGI1bHh4PSN8OrCv9kKQmUnfnk7uzlx0TTpCbrgQcXUuM1/9iauDMhHC7+vx27eRdrfj70nujRM24h7jfldptMbcRk68coK2sjbGvDemF+tq86KTdCmzGZC7AJNRIGxSGNoGLe2V7ShvGrlnZzQFKW/T5hmOIMCZd8+ga9YxYe0EzhteJtd3N2VqK3adLjwl3Cn2mfQmln23jOZ+N0hVOGJ78QLGjmGiBBziJlyQSGn27oNLzU1kxw7De3ckMy99donDTx9mWfqyu8qqfpf0HfoWPff8eA/uMe5YLVYkEgkSqQSjs5mtk5J46b2X2TQpupf8SKefjBt+N1j882JMvoewvjAeQ7sBhUaBgByjaQ/yzuu89N5LaEdK4ME73022jiA6O5oq3yp2T9vNyIEje5y7JqKGOm0dEquEgI8COLz4MNGevT08/imOPn+UzE2ZvKJ/Bbmq57NSGDoxKzTUBQ9E7aju9nbI25NH6DtjWLdiHTqNDq2dlpVOPU3Yredqmba5gdiId/GqNKOL0EK4+O4tubSE1kEdoB+AIdaW9DFPM3uUCq8EFTXhXlgk+u7zlAaXUhFQj9xGiaA34Jd9CoW+k+2vwdz140X5zL9F3vBwTvmvxKErAUEAn34+3ePFzgN5PP7t41zre40LI7UIAkz6chKGDgMXPrqATaEJ1WAVEw5PIPl6MrK3ZTx2/TFRis1spUW2i7VNNgT0uY5M6kDAkAAWHRd3fA3nGvn03lRGhvUhJa+dxiuxdA1thz5OCBLxJfJsmU7/wJm8MfBjMvuW81rf11g2XUBu0LLt2VSiJ4SS/Gjv4r/DfhkeFY/gOPNHDpSK/bMiZrxY6AjwIXKLHUPyl3Kt7zXkdo5IpBLG/2s8LqEiI3jnrlxMFjUl0nKazDfQtUSib9Xjk+yD0tcWXZuegvQwzu0Zhplz3Lt2PEefP4rGVYOkWM3VSymMmHmaRcdDEdQiqufK11coPlRAFAWc0k9gb99i4g6FUOvU2aPtA5/q7adwOwY5z0BudsAsb8diFf3n0jakoW/VM/jZwTTOO8oTCguGE2MwG9tpdW1FKpey+ORimpTNyC0O+NUOIbBjJfqHDd0sQo2LhoKBNRyP/1Tsy2Zn1sm/Z8bPM7p/W150FKnCiEQq0N9OYJdFZIBlbc1CaNBRHlBOpVrLRos7vzsuoK39G/iLbM8/hSDAz4+cICSnnr5ZbsAcgkcHszJ/JXZedpz++TpvvCX2xzZH0b+2taSV+qx65Co5gqM99Z51hJftwLFWRldTOKmfp6Jx0eA5OIytCw7i7VDNSqcpXGl+uMdvWzr12LTX0GzN5kbIGzSqQ4HXiJgagZ2XHVtrbwJwjy3YxBZQabzDLJdKpbi69jSL/2uY23TEZMfw58w/mTRY9KhwDBT7mrHDiEvKsxQ6dhBcH47VCey87ViZvxK1kxpBo+AJZ7BXt6MepyIq+QEAYufH4tPfh/7aIB4qTCEiPp/OX9bwsfUndj6wU2RpWgWsmSf59PTzPP30RwQoAMGCVGKly8ETh6ZSAKp8q9A2r2Vw5zAsejGB3VLcwpdhXwLgmeD5D+tCgbLAMn5e/DMT+t+LUWvEo48HjoGOGIADk3bRYd9OfmQ+Em9xPnmx7UWkMinrl12m5mIg2UODKHfagdHSn8LDFcgUMn4d8yvTSAQSufnCF1S3azB2GWlvbyd4VDALrj/H/mUfk9xnD7MblKgMKswGIzH3xrDxkBuuFTeQCFZOWX9gpPRR+qrAeou90BCUgsHGGU1H/V2LfXfz7LOarZSdLsMlzAWJUcC+046FT+7iupMouxu3II64BWIiVX8Y+sddZ/oDfzD/WAPubmaqw4cjM+nxLBULcqkmK6OOTkbj6UZjbiNqJzVBo4MQhOsgEYhSqljh1EVmk5gg37T8PGRkoETB4l8W4zm8BO2hJgyNh7B7ZQXPVD3Dn/f9SempUu7dfu9d5fgNHlIqfStJTE/E84b4LD8L/AyAF1pfwP7kNha2TSNt/OOEhNhjMeZjaDNgMVmwauTU+NTgWxnMrG3BONvL6f9Ef8ImhWHsNNL14B4e0PTl4kAdcVl9sPbTc/1nUW0gYmoEDTPPsNm/gFPm1ez/ZDYq7wL0jfUkHLlI3uAHkVevAqcP2HTuW+x+LcXBWYlmjobZv8/m0vYK+v75Pga1A9qGpWgCbNA16ai5XoNJayJ90jg8c4oZeB5ailpw9HdkdZ34juVWVaDSq2h0a+RmTCEzbFREzY+l4kIFLSUtWK0iOObMiEucGHOEBR6PUp9Vj08/H2qu1RCVJYJDZj7+J7EFrrxY+Sqf+ovj4oRji6nwr2CftJ2nC8CKyFJApcKotkclN1ATAtcNx0gOqaQ6sEMEK70CEIpJKKdm7g6GvDiEse+P7fGsnMd5c9ryCY84wuomJ0D0wDv/4XnsvOxwSIniz5rTTN03FaXUCb6Cy19epvhYMX2X9AUBlEYlQy4ModGipSGngR3zxEJU3PbJPPLjE4wfe45+cSUsN8CGoRuouVbD0JeHUp5iZFP8jww2vYJL0msEixY42J8XZWZV9fX89v4iHJ6NJzV8ADGxMXh52SKRiEno+r2Xqd8LfgP9eoED9AP0HBp3iPFHxmMw9ZRBFgTwKcxAonfn7NCzuAUOplOIApdA7JvLQBCw0dqgtc3nbHQyNV078W6QEzYhjJaiFrK3Z7P47Y1o7Ds41QhSjZwJn03g93t+58zbZ5ACi14Q9xwrIxtZ3AWduZUEpl8joaYVhfleZC+eICO+nc+Dz/JW/Lc4BztTl1FH2vq07nbeXg8FDg/Ezkfcd9u016CWOpHh8hlwD55xnhQdLqI+q57wL8OxRBdQnuePw2uuNKc0M+PnGbQUt5D0UBKFZZsYMuM7NrlDxcCNFP4s7lOvfXcNbb24p/r06U+JT4/n7Hdn6RvcF12zDv/B/miWeVGwRXzPHvjtATybPWGVyH4KGBZAWbGVxNRKdk+ClrdbWH7P8l7jwj+FUWch9PoOWryi6JT3Zhg1BrVjcGrk5fdfxhS1l78i6SxGCwZTW7efoLaqHbjD7hMEMSlfc0smVBCgKrWKjRM23jm/ayPNLs1kxGUwwvExzHoT8Ue/pNUrks0LNtPm2Ibg44jFaQsDBWg9m0H02UsYNY4I+3I5jMgCsfexZ8/DIsB0yAtDsGtQEtoUitwsp8H9AhJJYrdPJ8D9u1ZzSVFI5TV7ZK2XCZnihnsfd0w6EzZRtmyZv4PIwrH0z34fmUaJb6JvtzKI8KPA6kZ4twU6rVAklzLomUEIVoE2o7b72qTIsGLCKggI19OIOn+NC+P9uJj4KrNVwdwUoM0qjicqbTMRlzdRH5jMvD/n/ePzyhyzqucfJBC7IBb/wf5UVjTw1mtvMfDSUPrdfAIEGLBqAEeeOUJLcQtO14wsOrIIk9yEXmPC+UNnXmoXQYJmM0hU5TjIIOvnaQwz2mN61YTVbCV6VjRNMjuePHERh+AabrQsJbTZB8EqYO43kBzbgYy+eY4ZedNh6W5uXupDi20hdl5TGfvhWC58cgGpTMqeR/agclQRPCq4236iubAZIVagxbmFb5Z9w5oZawCw9bBl2CvDCBgagPR7KdP3TKfdoR27NjuMnUYKDxfiFuWGRx8Pjj9xnK6SLsYcG8ON4TdIHpXc7XnYUdTB3O1zSUtM41+f/ouQp0OwXLPQkNNA1u9ZyJQyatNqCRoZxOKTi3vd7zeENzj24jHOf3geGbK7grz+d+KvqRCZvabX8a4GLZ2ugbhWpiM36REksh7Fvn5L+3HmrTPUZ9b3+i6IMp2dLgHE/O3cDn38yY6bh6Z2T/ffbtuJjP1oLEoHJV31XXcFYANo3MUxoiJQy9KfXkCquHM/5u+az6erq2hf+yNmmZnOwXf2ORKJBOdQZ4KPxrKBcxzHzAJgPfAZ0AY4AjMQC35ewDilhhkLFxI1I4qoGVEUrIXL1n74KhsQWqRYtTpAg97Bg5KkmZQkzsD/5kHcz1/nrxTMQ4egrMaVuIVxfB70OdM2TCPpoSTy94l5vY7qDkKqJYQwg7dffZstMVuovlrN7od2d0tmT903kWrvcjKDlpHT6cmhVWoK9hfwmvk13nZ8D7QWjmYGQGQDgiDweZBY8Ldxs8EqFbha50CtdxcuWFHaqwifHE59Vn2359xY/cfd7RUEgdJTpfw6+lc8HprMC2PF8eCRe96ickQEypCl0NxAnzPf0uIZiTDnL0j0v8XR50TG421fxvx9+fx+z+/0W94P3wdGURbmxKytyVBrA4uhvaodY6eRtoo26o5eZNi5Ks6PXUR08pDudzZtfRrVV6txa3Qi/vHtHFvxKn1iJIS9Fcb8PfNR2CiokfqRUTmUeL9zfL2nP0n1IjjX1t0WW3dbru5ZRW7QdkJqn6FP3iRcWkTQo9JWib5VT/8n+1OwvwDvZO9e1yRYBFqKW2gpbiF/Xz4pj6f0+sy/E1Yr/PjDD4xSKHox+v4eA4CRSiXrf/yRV1599X/r9/5/Nf67fe//2/GfYt9/4n8sLn16ieLjxViNVu7hHr5/9Huqg/wIdXdC4yxubuqDUhCkMsbH/oy6uIQbl1PoqJuKq/8dtIzvSSWrT69mw8MbaHZtRhDAf5A/Zp2IkMmNLuUb+yKqX1iJW8fXdEPMAZVUQ2zZOrICV7KmGWQWKYIgbr5GvzeatNG5PNv5LF8+8SXKxg1IC92AwfRb1o+o6VE9JOf+HtYGA4lpoXS4jiHCXUxcpq1P694MuaMk6XoSehcJwi1a020m1/NdbzB9zwSanZvYsGQDVkSE4kPnHsLex56ywstsnb8VpxYnkWEn6VkpkZpBZVBhJzERd+JzhMZIeH2i+Lux7mRcy+LM8DNozH3w1b5IpPdgAKJmRLG6bjUXP72Ib3/fu6KSPM57MPnaZA5PONzj77VptagcVLzz2seYpVqQgFkpIFPKOLXmFNVXRHmvsrgptNsfYNKBSZhd21EkOiBYBExaUzdDMSZdQXTW9xianwbuIFLUpbmEK6px92jBPOQKEp870k03Lp9n8MXBzBidT1eHDueaFP4XbBfwLL6ETXstnXmegIi6rEytRGWvwtJhIuVqCmeHnmV/rITnAp16fNeuXk5AVwAuzS541/hgednCn/f9ScDwAAY8MYBTgzfzrFc5LmuepkQjJfTHUB65/Ai27racPmbiWrgZeyGQgLICfLrEc5t0JqRyKe3OOiwyC1aZlXVPrOONkJ73vaO6A4vR0m2C/Ndw8HVgRc4KDj11CDtvcYF5+NnD3Nxyk+U3l6M22aEyqhh8oT8yZRFq5yRGrBnRXdz47uo6fJPy+PLhH1jo/jH+g/2xcbOhNr0WAmVMiK3ApjaY3I8WI3m4nbj7RuES7oLSTkl5vIGsoO1EeVaRM13NsH49kxkSqwUBCT4Fp3CqzcPU9Qh/9WOTq+R39eMDuPjpRdrK2lievbzXgloQoMMmk0kHJjHg8iU6XhmMxgMOLD9A/yf7Yxvdn9L4ezAr1ag76rFaPTj5quhtN+CJAfgfa2dU+lzee+k9LPKKHglNqVzK8dHHWRlTh5NVw3aTAzLVnUJQ2ZkyQq/voDxmAghW7D16FvR0LSJysrO+k5bcFsLDw5HJ7nzfI9aDivMVhE8Wmb5bpm2h4EAByUuTyUsWx7Oc6GwmDuvpuQcgE9QgAadWJ8yOzbQUt7AuYh0ecR48dPlxBh65hleQPxX+FTh59ExwduRVMW/bPP6Y9QdpfdMYnDC4x3HprfEpuCSYRb8t4rrpOtGv/HvFPn2bHoWNgvgH4u/qe1cTPpya8DvISb+Bfiy/uRyNq4aKdjleXW+R5S4mXixCz0K3zE5Dl4MXdsYA5NZcNLI7Y4VMIaPdUYderSdV8zHyyCMMlKwERHa3VfJ3DxIJggCC3oDXLYacuRo+fnck3//R+7okyBhwaQCxWf3QzRI3TN3HlFJOjDqBWq8mNt0Li9FC8OhgmgubqbggGmUlpCfQ5thGWlI+ErkM7yRvvJO8eVPyJqOA8S/8ydp7SjnSKj4rU5cJq8WKk50bnrpVLA/sT2meEyfLAzFqjZx++zTxl7Xk+kGd825U0rexyC0YMWI1W+m3/10EJEgQKMq6u9xlyddnAD88hrph6hLH1jYvMbHT+tabuKGkwq+CyLxITHZypDIpg56+owF17usjnPhxEhnqJjSBF0j9opXTa04TNSOKvs9P5dJTWznXHMzep7/A3fkB3r7/fuLvF6WycsdpkU/6ml8sAiO2zaY2XNzk1lyr6T6/U8lN3OZW4vD0KSpcH+nR9pvbbpL6RSpTvp7Sq5BedSiTR36+n5sxmZicOrCarOx9RJTSG/zsYM4EV6OrURJxYhauQW4MeHMAA54UMYreZn+KByfgVXgWv4yfaS1+jJBxIZSfLReTVn/1psHaC7Tw84MbMcga2Jnlgl2DHRP7GGktbeXwqsNIfGzY+NhG/OWw9/h4uo5G0XpfI/9OsQ+gznkXjx17jbrwVGAOJSdK2PvoXu754R7cHYK5EFVOVp/LGFxn8qaDmpailu77ubX1Ma5FbCTd1xYXbQxrol5l/p75NOU1oWvqotarkRU+jdzn/AvFN16l8JAoGT305aG0Jfiy/UETRtcfUVWmoywTE5VBI4LwTvJmVGwgoR5T2fPMPnj0CE8WfNHdZovFwjty0Xtq4hcTGfBETyyoqbKZOX/M4cCkA5jcxXFn1JujGPWmSLm3eoK+WkPS4Y9oi+yPTDEK13BxE9XeLsCLL7ApuJqL0XOY6O9N8fFiPOI8SHo4icyXczmwfgo7FznwS/hx5vgJTPpyEsYOI8mPJvPN2tW05dkRn6Mmft8kHMIs6Cd0dRf61j6zlg6HDiCNQl8P1tyScK/PupMsWnR8US81BACPUnsURn/yovKQuyppKWrhp2E/MfKtkUQ9NoJrydfhFmDIarWS+mUqV9ZdYeH+hTiPSaaDDIyFTgReMNN6TxMH5mzvZssADJpynrizI8m8EE/nzDa2jl9P3MI4wpeOImF4JkNi9tJ3U18mHp5Ie1gdPo9H0+TvQ7994np4U1Ejn+mdKKtzwjTeSHTeUWwLbnSf3ypXAj3lrIwdepxqSwm7uhXhjAO89TTV16rZMn0LA58eSPNQBReTP+VDP7jYqGAi8Nv433CPcWfiZxNJtVnDn+YSVgCNgeNQuqbR6mnBrqWS+TcfI2q7D8MUUka88jrERJC2IY2m/CbSfkwjsrqTNazB/vFzvPz2dFwFAV6DZr94OkzeuKg/IcG5nZumoRTV9sHOqkUqk2LvY3/XhORths8Dxx5AF6rgjwd+IznjPl4f2VMNRGmn5GrfS3RZ8tG6uCAIjxAxJYKcP3K49t011E/1Q6VXoTApkJkNCCZx7A8eHSwyLACVrhWJciibH2lk9cRBTFo2nhOvnCBkXAjLDONwkd1glPQAmTdt6eyMQjI4Ar2dK1apnBMliwj2KEPh5URl1BjigzxFT6f5saSfF8dMlb4dk0FMvPRf2Z/+T/TH0d8RL8MUTM6d5AxpxbuvO12NXbSWtuIS5kJXXRcvffASqf1T2TPjOO962lOfVcqW6VuY9OUkqq63E3Q1Cy/vOEw2rqhlNkz+ejIuoS40FzWTl5HDyR2jGDXnJE3N8m5AG4BXWAD1cZ8SZmvC9uIshp4bSktCA/L+fcnRJ6EIfhYAH1n7rW8ImLpM+Gcdo9PFH6FSfP8ChtyRVO/ul4LAGleYbAsb2sU1ksJGwZgPxqByUJFxehfHhq5hZsNMXE3iOOHg58DAVSI4ZNyeFEySCD576jPGqj/p9hca9/E4JPIAPOsDqLiZRPbP0/Ge1EnwmGBqrtXgP8gfIfMW0FRykRr/VXQp44GH0UUmIJSVUxW9lzq7djwN91HQnEKoxUp1dRGCEAJSGZEvzsC2qfyusugWq4WCiAIKIgp4b8x7gOh9V3mhkoUH7yNr1BNk+68hN/g4Y5T98c+XYZGLa+Gua608f+15CsIK6LDvoC2mjqO/ZlOXXsfzTc/z+Jd/MKtwNC6ns1l6I56O8R1kXa7q3qMBlBthd4svB5vb8HC3oq9uxr3iRvfxJ7zTOSJTsV3RAqpbnr0jArF1t6XgQAGuEXcSWykrUrpl7eZum4t9hz1bHzrRfbzvY31FZqM3vDlnCwMuDWD0ldFYjBbiFt7Z4wUqmvjEHX78/D5qFueyPHs5EpmE0HGhHHvhGPlp+ZgVZoacH0LhlUKmX5zO3D/n4pXoxaFpv7KidgVb5m+hILwAo0UcO1MeTyHl8RRen3SFEntxbVbgaLkrGO+fQrAKmOVqrFIFJlPv45njq8gxHMIqtRLjJgJ/v+/3PYJF9Ix9b9J7zNwlMkZasir56zjbUVSHT14OzT590Nu7Iwj08EOsC+nim0XrxGcWWM5otz8AE0aNIxa5Cp1Gh06jQ3WLLSgIIJgsyE06bNvEtUCTTxwO4eL1jnpnFHXpdTj4OhBz3IsJWSJgpmaIAbWTutunE8Av24Zr6lDKo7NpVGkYKp9Fw80GWopamDRsKqOzCqn3eZ/zA7bwlOp9fhv/E7omHU+VPMWQI/ZMO72GphEXqGq2w/K4wLZZ2yg4UMDjzU+yPDeBft5tKKxy0tJDMAd0YdUbUeg7sGu3Izkzjh9GlnK0PIDP0/3Qze0iMHMfAO3uYexbto/Q8aFEz7rLPkWwIjfqsMoUgJLSU6UcXHlQ9AhTg1VmJXXgNVxUc5DKIWFRAqHjQkUP72g5VxT78KzzRGHt/d4u/O5e/owrpq0gAC+5hIIDBWyfs53o2dHU/ZHDEIWG5rkpuGSmEJQRh9VixWqV4Vl0Af+cSG4A05fu5sWlX+EtTeQtj1cY8vwQhjw/hPz9+Vz85GJ3EW7i5xOpvFhJ6PhQTt04xaJfF5E6IBWJq/g+Fh0tQqaSETgsEHOZmS2yLdy/9X5KqkuI+SaG7XNEa5I3hDdYkrSE39J+Y9i5YRSmFiIplrD3kb3M/XMuttG2bJ+znU67TgwqA56hnix8diEWg4Ujzx5BppKhsFEQNTOq1/24HYEjAjn/4XnKLpRhsVh67Hv/74RZrkJuNqDy7Q2wMrTq8CxJxai2p8kjgrK4yfy1pGHvY88TBU/0eJ/+GoJEQviV36n/YxA8ML7772++CfX1QPw9NPvG8+ZHasKHiWCorTO2UnVZVEu6d/u9dz1vwd5cDEoDh8ZtoLO4nh3DvutxXBbgwfsvvo9FZuHoI0d7HJu8bjLba8LZvLtPt2ffK7f+/T1SgTNmI789Iu6Vmoua0b79C76OYbiXX8cdsHjdgyCIRT1Nex22rVV4ll7B4BQFfn279zNffQX99m2jFoiZE4PTrRzYsoxlaOu12LjZ8FnAZ1ikFiSCuBd2CnJi8leTcY105bexIgtv8IXBBJQHIJvWgMw3iObCZn4a9hP262OpP/0s9VYJkbmRmLpMJC9LpqOqg8DhgRx97igTLw/EJLEiafdF9Y6ahfsXcuylY91s9UFTLnCs0Y1nr9szJcWEbbQtsfNjqfaQU6P1w9u2kuv1g/izZC4PGzsRbGyQAM51edQsfZPP3nNkVemqXvfRrDfjGunavVdprxLXKc0FzbhLbXDRDkNmqetWWPHuKxbXBKtAedtFXHQmDPIWfCRJePcVLXKkCil23vbopHX4JNZQFlaE8VZeJvKeSJoLm/kt/H30tnPZdv9Aus6106dTvOcWo0WUIbcKIAMbQwjlyStQ3yK2Vl6qZP2g9cDdPQhBLKA+U/UM+fvyu4ux/zshCLB782ZW6fX//YeBRTodn23e/P93xT6LxUJBQUGvnN//qfhPse8/8T8WDgEOuA9059ETj/JwxFJis2LJ7p/IhHBfQsaKg9T06SA1mRhos5myZg9cauypbEjE1f8OUlxj6067YxUTDk8goDwA03wjZoMZk/b2qlvgIQfwjC5gb0PP4py2oplhJ4NZNWEZiW77eT7DR5SEclXwdczXaIAKv1YEicCO2ZsI0k/kfUDjrOnlyfL30JY2EJh1gLqg/rQVJkOCByFjQ8jclNlN1Z++ZzrnRoOaGv6q8y8IcHWgBIu1gJfee4nOUeV0VHdg0ppusYjEGbjVuZWfluxjuMODPX47e1wLqqk/cc5Xwg/uj5Ehbwcmsn7QeryTvfnl0eu0OrTSt/A5/Oun4SKIQ8HV767iGuHaCz3713C86UjolVDaHdppq2jD0d+R6T9Pp6VIlGkSJKIMl3u9O3JBoKupi4h7IggcEUiHgyvnLitQGSwMuDwA8+Quuhq7KDtbRtIjSXhdVDC7YDZpcduocfdlimZ1j992P7UDvZcX16bryZm5n1HRT97pB+Nd+VT4lJF+WtI3TyHkzG+Yul7i72bp/xQliTPoc+ZbzAePAYvRNetYP3A9HrEeyD6I4+PVH2NSmPBuC0cQJLyjegeL0cLiU4uJOe3O+KyHyY7OJiYnBqvZSvaObLJ3ZBM7PxYQ8JTB8PnHWNMwEKlM2i3jYhMI1ZGj0LpuYfSJ0XjVi39P25DGwZUHqR1oxqZL3Cg7dw4iwW58j3bflkBYVb7qrlR8tyi3HiwwW3dbHAMdkalkXBvZxIn+G5m/7UmcVCrkKjk1V2soOlpEyjPDiL+0jpA6P0bOFCVJpHIpGmcNCo0Cq8XKNy4yOiStHJ56HGvoYkLGhOCd5I1ULiXAfTGbA75jfhNAHkf/lgRPPPwRnS7+dDl4IbVaejF2n6159h+JmfH3xeOd5I1ziHOvApIgQHh+OEqjkuw+VQgyOVWXy3CLciN6ZjRHG9tIT8yl/8nzeFc0Y7W+zvDXh4tyHp1GqkJbmOpVwBdeFlY09pwipUopZ4efZZ47mGWwzy0PheZOItstyo2qyFEYNY5URY8lIr7nZvB20tpisZCZmdnruh4681CP/3v38xYlkeI9sUst4uVfX2brvK0MfHFQr+8Gn5SQnL6K7x/7nofsl3R7UdVn1iMIcHLUeRrdqvG07OLhh3vKndhH+rJj9g4q/UT/JevfimoLdi/AkGNg+73byYjLoE9AH/7daMxt5PoP1xn38bi7MuQA3Mqu4l6eRkvJHJyDnbvNv2WtVsIr7kdTv524jGC0YW0IfQUsRotYCE5w5exEdxQWBS6djzP11hCavz+f36f+jinQTHpiOnKTHAdZLk2tU8jdLSXq/AWK4+4UNAdeHEhCehJdUzvAzo76wH54lF2lxSsap+N/YuhY1MPby2q2Yp/2HQOzRmOWm7GYrBQfK2bb7G2M/WgskmgXzow4w/Rd0xl8MQKL0YJJZ2Xvo3e8mhLTE/npwZ9o8CrDYrZy/qPzeCd7EzQyiNJTpWT/OYYda9YzUq+n6EhRN5J76qVXUHZ1sr/oZVbMnsnV0aexDXqazLdPElJ4p4AnlUqw67DDtcUVs85Ms3cfTCpbHBxg9jJ3zHpzL0bs7fjyl7XMNbjDxyA3aPHJPwUKOZjMnBh9glqvWkJMopxQU0ETXY1d+PTzobPBhvSOAKKqE2gcqsN/nriJKDtTBl6eKCf5UBiUxTX7ZsbYiNJc+hY9btFuWFVmfrFoGap1ZdhDO6iTiEn28f8aj7bNwpJda1HZZbDW/QaDA5p5rLFnVu02G+P23PrXaM2twKvWA6/aMViSW5FIJYSMC8HGTZS0dJdZWBlbw+HFCgIniAj6iosVCFYBz35igrndLZSyPirUHg5M/XZq97ltK2UklYlyUU6t7hieMPKxrejL84bwBlp7LXpZJ9bSGGb8PBfjghbcHxb7t1DdxYTrbXw7fQppKVl8Z1vHcMeeY8A/hSBAYGkghyccRh4iJkRlSpkIfpCAg7sXl0bZUeqZRVj1dAQBgscGixJgtZ3E7HFhwrWXxaSCXSMyhYy69DpOvnYSr8EhKIwySrvkfPf1Q+htLlOuEpMZebvyqFzZQp6vKJF936X7CCoL7m6Xod2AR4UdHhX9aLLsw1UGbpK/ecZKASt3B6j4B7Hh3m00ujUy5JZM9+4lu6m5VsOyG8sQBEgPXoljkQNGh2CsZiulp0ppLWu9VcDVcIbxmAwiCrbwYCEX114k7r44bONCyO+/kPbqOfQvaSFlcQhJS+6wKFKn1LDlnt0kaB8h/IYbVk0YEuUdAMqQ80M5NOmg2D/cN6C3vA/Y4T/Yn4mfT8RvkN9dC30ACWf8caoN48MXP8RssWDnbYdEKuHat9eIfHQEIBBUEsSMXTPQLqtBGhBIU34TjXmNmJQarugGk6ndjKNFglWw4DZjCNUbTxIwPJBapRzXUadJK8gl276FWS5PAZC5OZPMzZm0OYbw0oohfJxYxEHdJRSuD3a3K22AErvmLILVpehP92XGyeFo57Vj01jWo/2uFen8vdjXWdFK2NWtAEjaxQRI9g5RjtZvoB9CVSalZliybQAdrQ7wpJiIqEqtYsgLQ/A5X0nRwbnsGncFTbMbEjdwbCjCN+8klSfH86ErPOdspenjdayvOkafue6493Hn9JrT3W0wqQVaXMJRycW+ovUIpkEIJriPNyNDz/CLdSDfLX4LH+sAPjQ9R0txCxoXDWa9mdobtfj298XO047r34s+NnXpdQhSAYPaQElYBzaB4rsaNjGMwkOFZPyWQX5CIxXqi7z0/iS0advhqXvpauhC26DFUtjCSx+8xIlRJ0idNJm4YR7o2/Rsm7UN9z7ueK2YxsE8T9QKFyJq1fjbgsq+ga7GLo4+fxSLwZa2QbMZufwQWct+okq7kHnJ88jrEMcYR1UG/dx+p5/b79gpn0PjL/qjXfz0IpV/XBZlWW1dePpWrWLjRHHeWJG9AkVHEc3KTGrdnVA5+HHj5xvsfmg3g54dRNSKfhRGClT4V4BEBC3k7ckTx8qxIZQf+hrnFh1hpcn4Nz6Kx/Me2C1wQ9+mZ9fiXWjabDl9tj/bky/hplXjEevBM9XPoLRTUtssRWl2I9HxOG/Y6thd0AS3pLFrnLdRQTsvNoJZNoVRJ2rpkx1L1yNdeJZexqa9FqOLK4u/GUjE1N4en7ryTn45EcvxgnAS693gBdFX7MQrJ3CPcUed3YDkVQl/zPmDWItYuLj23TVi5ojzi1OLHWaJlFbnVoyI7ErnYGd8B/hSUmtD+vjVNLu+iLvqPPLAGMZ9OI5xH44DwG2vhHdqhhI0/DryHD0HqQYeptM9GM/0i0Re7MuDn3+Cg3Umq/Ofp9K3DwEBoWLDJRLC5ySQnJxw1/FCsk/Cgr0L2DVjV/d67MaGGwB0Nem4GfITeX4i88GKFblZT0NAMnKTDrtbPppSq5S+aX0xNuqInBxO6clSbm6/SUPaOZ6vrsCIPVcMKkydBrJ+zej+7T/m7OfHc37I7SJ48PPhtI2w4rSyD7tzImhUf0293TeUa5N50fkKRQ6O7NVbaMhuoLO6k4FPD2Th/jt+z4IgsGHIBmLmxHDv9ns5N/Qc0/ZOY+r2wfC9KKVXerKUUW+NokqowkcG5UNSuXG/C+4x7nQ1dVF9pRr3GHeU+Tb8kj6IYjs18VPCcQ52ZuQbIwF46OxDBDwVgHe1Nztn7uThxIeRqWRkbsyk/Fw5nVmtuOOOb5Uv54ado8W95U4fatFRqt/H+f5iQro0vXdR+b8KiUJOXcggvIou0NkQw21p5+7jyBCkAnum7yGsfTdwiyVya0804rS45jgz7AzPzuy55+0srMen4DQ+BacpTpqFIMThlehFxD0R5O/Nx7P4znwTVLcSqwvIVApyhy5B01bDkg1LOD76OBZnFd75p4ER2A1LIrMmCZW2GZlJR5ejD7Zu4vt4W5YRoDSpiQL3NNoc2wjz6q1pWjzjGc65PUqZ1zYe9kvBb6Afj1x+BKcgJxqyGxlcvInH4vexWN6FILxP8JhgdM06Lv7rIg4ZLgT1y0ZS5YNrYRD6Nj15e0T5X6sg4LFlJnb9b+LhX0/RqcGYZugxJA4mTzMQ/8y1DCybzVmfP3As9WDMkWFoG7to9o2l0yUApa6Na99dw2Kw3LXY51l8Cf+co9SEDgHG4pvii29/X3TNOswOEFAWQItzCzVOfyAIs9G36BGsAko7JXpfOVedRVlmR0Mf2ivb+SLsCywGC8vzn8KkMJMvmJn+/jomF8ewJkz0qcr5Q/QWV5h0LFGvxvveWj7qm49U9jrSynLcKtJocdJyYMn3nCt4EKPqZ8ySnmvZtPVptJa14ttfzBG4Rbp121xYUi24NLvg1OqEVS+OF7f7/vBXh2P1tVIYXohhvoHY4bHdYL/bXuoTrk/gMpf54okv2Dp5K5bz4nx65u0zDDk0hJuxIlClLKiM+UPmd8sCztv5zwzK23HgiQM0FzTzWNpjFBQV/Lef/3dDECB/4CLsm8tIsu2dj7PxdaYqchQGjSPNfgnd37kdJp0JWw/bf2TgWeUqGv0SCAzr6aNZdaUK/8oMGgP60uEWhNtfhJW8+np1F/t0Lbq75gmlMik50Tno1Xoassqwmq3dwOb8ffm0f3qGEeE/0+glZ4j/kO7vdTV18VX0V7j79iUy+kMm5r7MIcH4j559k1DwwMyX8PLyIvP3TK5+cxWhrR0nXS4ARpU9Nt7eCALYtFYRc06Ui60OH07yQ4Pg6N1tBWb8MgOFjbgudvB1wMHXgY7qDgS5jAZ3PbN3P0Hj/Y24Rblh7DTy29jfCBwRSNnpMm72uYlTqxMqlYRxH46jtaSV7O3ZcLGSx76q4czeIXgcGor1ARNTvxH3WWa9mbTd2XAOvIOryeyw6W5XxJQIqi5Xs8W8iVSbSnR5ccz65R5MWi0eKzyY/ftsZn/5Bk9WVxJYvwK7phwics5wdnAw2E6nOHEGrlWZODYU0VbW1vtioZdVU/TMaGxcbfBK8iI9vYV7tjtREzyIB14SS8lxC+KImBqB2klNaYaen4Z/xgObXqUr8w8cl96DW7QbjTmNOPg7cqJqCgXlV9E++jtVrS9QeLiQm9tukrI8BZWbHTQ2Y1et48airWS3JvIvRLWbw08fJuRhCZdHwTbFbnY1rOi+J8dePHbX6/h72PvY39Xe538lBAHa2tvprftx9/AE2trufp//vx76f7Mg+v9G/KfY95/4H4uaazXdCAnnZge6NA60y9pE5JkgoK3TojDL0JqOsefd8UhmHGbNG2tY4LOElpIWOqo78E3xJWHUag7ooEU/AoVJgQDM3z2/O5G5cOME5G5NTFi1jRUFHj3a0FRTg2fpFSz1jdQVxzH8aH8M7xuoulyPRCahfqDA+nEiQqLFpQVno8jM6Wrq4vPgz7H3tscnxYfhrw7v1lq+HeYQBftmlDNpj5Xzr+QyfPbT2PvaY9abSXw4kZ8bTnPTfh9T900F2Xrg1W4UhlYLduoPyfAaSmBZBXb2seTtzuPQU4d44OgDGMu6mHRgEiWhRsI6jjFmDOTuzuXAigPc88M9CAg0WEAqFVj8zneMLRITiKYuEyadCQUS5CY5gv4cSUcKqPMbizB9MIeePIRLmAuTv5pMR3UH0bOjeyS5ASoWV5Ahz2Ds8bE05jbi6O/YLWdpMVkILPVHp+5g6fdLaeovkBeYx6nXTzHhswmk7Uylxv1rikKLyIvM48PoqShsFfRb1g+/gX7YfSAjPCuOvffsJT8yH5lNz0XYiSm5tKnTuNykpFEGI/6yErGqpbQ5tfFsK7wUVYSuKxyL5G8VpP8idA6eFCfOxNlHlNCqyxQlYmw9bdFJRU17pxYn3GpFbzu3aDfq0uvQt+rxS1xKsaSYvZM+5OCkQ7yueZ0hLw7h8peXMevNTNs5iuvqDqqHDSbQayXaBi1FR4rw7uuNweyITtGASdHImeFn8NGKKCuPPh4kL02mY2sqodem0OAfw6izi2hXF8LMOyw5p2AnWktau5msfw1Tl4kdH5fgEurMhPvFpJXZYKa1tBV9ix4rRhrdG/lzfi4P2ovJ2yEvDCHhQXFh7OKYh1QvZ9KZKTTE+iJYBYa/PpyAIQHsffMa7657jXlP/07RqLN4ez/GxU8vcnrNaZZlLMPHIhakwtqdSD4zjIuG3xk79s6GsdknBoOtK7VhQ6mKHov8b2vijRM24pno2e1Z99dw7xfI1VJXHK+3EJri2oMBKAgwZf8UtLZatt13gw/sNFRl1NGY24iNmw3pJbtJC3kMY1cfwj2nIgg9mSOlfRqZF3WEneunsCTHD2EFfzm32N8+axUl6iIUPWULXMNd6XTyIzJVTBoYBt4dVfVfhSAIIIBEKunRrqPPN1IUWIZOo6NZ246XY09JAItSQGurRZAIGIQO7LzsWHR8ERKpyFi7OPgKZnk7XpnSXoXVbMVVsuKyAHjiiyfQpGj4q91gaFAoqZWp1HvW8+fsP3ls0GP/9vU4Bzsz9fup+A3wu+txTXsdbpUZKHWtWE1WTF0mio4UYe9jT5fOjoRjnxGpGoRZbsSiN5P9RzY75u5gzrY5lOrruBIuNnRo9hWssj6Ahrp08d3ttBfHa/cGd5Z+vxTDjFoMbn7YGFuQWu6wYuRmOWq9CqtZQCKTUh43hfK4KfjmHMOmtaqXnPEvo3+h7/U+VHtXkzniPrwu2zLapV2UwV22n4RTYnH9wuALlIRbmKmQ89u4n6k4X4HKQYVhRDT7VTep9m3Bs20GhnYDx14QF+gr81cy6uXpJDjXoF8/EX+9DS2uYmLKI9aD2oxi4k/8TqYlEskt4vU5i8CDpx9k6vxqlJLn+d4DbCxfEJcZx4QjE2h8tJHiZNGLTN1Rz4Hl39BU1snED0b2uK6oIwtJO/scETtLKS8LBSYRdmULdq2VqMcMZu/YnZQYROaDrTYcs8HMuoh1Pc5h89k6ssrdMKne6PYZurn9Jjvm7sAvpprn5//JO+32zGmDPxf+SdmZMh698ijhujZ2Orjj5t2ENKCJ+WXifBoxJQKtFiryf6cwtAsHKQwskRKo6TlgeCZ4otAouhMhf43jY89wIvA7XJtceTRgClK5lAeOPNB9fNhHD5AdWItmjAmZnYbW0lY2DN6AractD19fiXP5Bc4lPUu1bz0vOldRerqUrN+zGPTMIFxy5CQfm06tZy1edV5YzT37ikOrHT4qPRMDGrk59SDBEctROahYlr6MAz9UUXMwldq+wWj9z5OurEam+feW5YIAc7ffT5NLDRX9xGKfwkZBS3ELbWVtCMEQXvMKCdnzcGgxYew00pjbSMH+AgztBlqC9DRaCpBZZCgNcswGMz4pPszeMhuHQGee+HIR1T4V5LbY0O6VxuBn3xaN3v0cWP2v7URnR1PpV8nJUSfx6mzmHUSviqzfs2geFMsZj3f4rQIm7vwQd5uT8Pqo7rZPvzaduLi4uyIc7Z29afALYPmn92LqWwePQmdNJ3XpdRg6DBj2fcKccj8+fPFDhpbdQ1t5Ww/vjnbPUGZPX8f1b6+TLvFi4dfLqLxUSe7OXAocDZxL2URycSAJx8wYE2zpanLi5rab+Pb3xai28I2/lvvt17Fy7g0CE2NQ2CnQOvpg21bNwNQB1HhXUx5Qjk+1D2anTsADGzcb+i3rR+HhQtZFrWPMe2N6JRnTh5fjkWlmzZo1lKuKGPhiJOFTwmnMEb2pln27FEFyawxXQMryFI69eIzLX17m1PiDeIa8xdLR7cwacYZRltfZ1zgQzZAQRj/vwOntDly4NeagAHtXmPr9VI48ewRjhxHHNg3OGXocZ5XS4VWK0tuBTY+eIu74DQ7OlJE3eTNd/rA1fjybdbbMdHNCeHw52m9+xa6lgsqosbR49Ubta7wcKU6ciVLfQYerWFyNmR2Dvbc9IWNDSPl0EsJJO1yLMvDoEJOL036cxo55Oyg/V45zgwZ1tTPnfvYiUCHHZFeI6y320LmVR1CNGUd+SiHf1fyGxisSn35qTr1xitbSVmpnKVgftYYU9RwSB/nRJhHnF5MJNG01VP0xjWd8XyX6vsPEK6HLYKa9op2vor7Cb6AflZfEgsj8PfOJvCeSJReX0FHTgUesB5pnLjL73GzODD/Fufp9DGMqY94fg8ZFg1OwE+SJ73hpYA4+Hn40FzUTOSOSkDEhHLhQxvWk69R61ZIfGU5M6wZsn3ah5EQJ1VerMbRfJmfBZtpdfBiZ9gvGZmfMnmb0LXost9h49YZgXKVatg/TEn9Z/Fub5gZn+yTh9BdLMBv/T8hon0z/wniOPHNEHBeiJd3jA4g+ga3FrRi1RnwubsZRVcOPj2zAaunqlnfK+j2Lga+PpzZ6GS5aPffu0NA1oZ3GnEbkGjluUW7UjZXwXcoHzN61mtAbf9DV+BQZ265z9t2zDH99OPuPqXl24WyOfDUXl1IvZJ/Iuv1Ttdkd+OakMtB/PQnD0tkanY4m8CPMe4vxLDzNTV8VH1oM3OMUhERoBqSoHFSkj30Gh4YigtN3Y+oK6tX/ANovNZKyZQ4tTi04yMQJMXxKODKlTEwiAq+98xpfrvwSq7N4L+8/cn+3asmB+wupUh7GqcUJicqEU6AT9j72CBZR+jjS9RKfjP8QrRWmN/RUp3AvUGA+OZahw3LZuHskboG3/Jp0dwBouWaBPh22BOQb6Ig8itV6q9gnWNE16jDplHf3uaqAyPxIBl4aSGNuI06JoreOnbcdXgledMi3E9bhyKCd0yBCiq2xAL+rf3Z//dul39Lg3oDcLGdD3L0EuTmhclCxY+4O4nAih8EEvP0ZXww6wn0Oq6m5XoNcI2fIC0MwvW8gKqKCOU9t5b4BKtzCByBVyLAoZLS6Kij0K8XWPIA/z/RH//swbN/qpEPZQebmTAKGBfSSxrXztqOrUbS+uJ58HavUiq1RrEbrW/XiPqpNj75ezw/nJhKRUMjrgrh22zRpE9VXqum3vB9+Oz0p7ZjAF899y7OTfGkuaiZ3Zy4hY0NQ2Ch46sBT1NTWsHb1WnyTfbEYLOTszMEr8Zavqlc0wZ0n+XqahZgocZ4tPFzIzS03CT0lpXZwGCUhJcz6pYMfKn7g0UP/YOL0txAEkAgWpFYTFlPvPWfUaW/cjCM4PfJ093t5O5FddKSIm31ukjoglUb3Rp617amm4zoonOLEmQRl7EVm0iEIoNAouHfbvTRkN/B98vesWbOGH5f8yLCz7hhtahASb6HebvkDhRSH4KtJRlZ4DkEY0b3+N9j2ZqaJ1yM2siaijezQMwB4ShairdfyiecngAhk6rQppHTYb+zTwh+ClavfXOXKV1d48MyDtJS2IrthZEfFHAaapeiTOwibGMavo0WPNQVq1s7aRpqvlDVNEqzW5TgGOhI7PxZ9q56tCzajt+/E1bk/bY9c42XfB7Dewp/4lokM5hLlah4Y9wTS+FxsvFfTHJKCxQJ+2SIbauAzd5d2988Rj2ud/aivB5lKRtXlKhpuNtDpJuPhnx6myqeKGu+fMemmsXfZXkpPlfKK/hVGqJ+iKi2BS5GjEBDQ1mu7x22LycqxR7dTFCmyd9RSC14JXoz7eBxXv7lKpxyWPPQujq7tqG31pEpbQSJBnp+NqrORjcsuonFp42nnizxQ48HP7UoachrYtWgXZr2Z1rJWgkfdAVaVnyun4EAB/Zb1wy7Bjg0Pb+DJL5+k07cThsLw14ez8/6dpP+SjnOYM0FOQdg9YkfC0ARkUll3vkkQ4ORrJ5k7dC4bv9uIvdIeabQUY4cRj1gP9ELPJPXt67ZarHf1yP57XFkn2uEs2L+AOur+28//uyEI4FhfgFv5dTpz/GBaT4aSRC7Ds/gSOnsPTGp7TCo7BOFODvDUmlNc+OgCD194GP9BvdlNPnkncKtMxz6hZw5C09mIZ+lljDZO2LTVcPVzLQn3huPRx6MH+Li1pPWuxb6UF0dx4JO9TN03lfDCcIyvGLutQC59dglLaRUJLd7o7D04+tRRJq+b3P1dmVKGoq6SiJhHEZAwLO95hskUPGzSd3v2rUfOeSyMZhRjg8VCVc31GsrPliPrnwSXRXWxsvipRN8q9qm0d8APHqVXkNqM6r7Ht6MuKAXP0ivdaj0g5sny9uRh722P8snH0R8tJTpzH23lbbhFueGV5IVXkhdOgU6sWvUMbU4t5MTksDpAVCca99G47nl6x/loAhLLKa6dyXy/CMwGM23lbZx95yyN56qY8+Q22mPyeaEwmddqOvkodO2dxg1QcsJoYYtnIsVjlISPn9N9yIqVSAViH8QlAAEAAElEQVQ84nuN1BJXYjMSMDcZEQRo9kug2S+ByZPh8cd7Papb90DgwIoDePf1pu8jfbH1sO0GC3VcKcYsbaFTnYVSEH0fCw8VUnW5iuGvDu9WK5MZa+g0iXYEcffFYeoykbkpk8Cya4TixP0vwqBWCfVZ9dzYcKMb2OMQVsHiUX/yrGcbUVfEvJx7jDuJDyVyyjadDAO0CWa6LFtorrIBptHv8X6UnRYHyzclb3L/kfsJHRfa45qMWiM3t97EI9bjrnvmf4ra2lp+/OEHrpzajMnUjlrtgFJhpPbf/H4d4Oj4z4p4/4n/d+I/xb7/xP9YzPxtJtkZ2XzT5xtaXVWcG3qOkWc86IwOo62fA58HfU6Isy9XhvlwZdRJmjwrQAJWq8C+x/ZRfKyYpyvvyDxeHHSF80PP8LTqTT7x/ITomdHM/WMucsEPi1SHnwJClT0LIqYwCZ88+wmRDgY2Kx05FlKI0uEV6q4UIlgEXOPvLBZVehXyW2uetA1pGDuMNHU00ZTfRNTMqF7FvnZVK1cTN4CpH328xPMEDA3gdevr1GfV88PbJykJLuHE6BO4akff9R61ORvY+MBGFgoLCBwhJlPOvnsWy3gVAy4PwKC5iUdjKqZib+TRchz8HFDaKkksGUl1sS/Bw1IpVRwjySq+6svSl5H6RSrLnhrGlX5qzg/5nezY+3HzkYIEBj07iItrL3azSAKHB/Yq9gm2Atf7Xqc0qJQXUl7AarHyffL3hE0MY/Bzg1n060LSEtMoi0okZWAk3n2dGPXOKDQuGvSnG7iXe9k/eT9X+l9BkEupOF/BxbUXSXwwkfxZWn6Y9CFWqbgbEf7GMDrfdxtWiYkiazDHd8zD1zG8+5i53YhTixPpdp3M8ckCnyweVrzJvxsSi5kW7xhUvuK9co1wZer3U/Ef7E9WrQ1xuUsJyq8m+Xog7a+0s+zGsjtfviHQoJNhtdXRYTUgkUgY+/7YboakXYcNGpOAq7qRWoOR6it17Lx/JwA2b6Tgl7aNohGnKQ0tRW8Sk1ZBI4NwCnZi69mDFPveoFOjJfxGKs2qdnj7TtLhqeKn/vGaym52kr/mdwDCvR4gZGwIGheNyIiTSXCpMOGs7Uta0u9g3kR7ZTs/j/iZAasGEDw5hvUPbWBckxe2nyzD38aPggMFotzJycUI7jIy4zIYZd/Oqy7wowDVl0UZILlKjvLUEZ7LWc1jK3ey6WoyXWE9F/llCdN7/P/vCLLiY8W0V7Xftdj39ddQsu40WWVXWV23upfc7NUhMqpdj+OknYTZDIOeGUTy0mQ0LhqUX3bx5PYn2TVjF2n92wEJDdkNWM1WPOM9ERCQFMDUNgm+gqRHu/Qdepb8uARFXB5z3UzIL0ahf0KP2lF950N/IRpKs7OAO7C/vUv3cv3768zZPgd65o0AOLTqEKmfpzLhswm9PMiUCZ5sst3E2KNjyfw8G6/Xh/U4XjLYzIWhP9Anqw9yG3HpFTxa3BhqtSLjVmFUEJyZRsMZE4y5k4xuNt6RStRpdKg0Pd/54NeCWbH5TtXTIvzX2ux/DVsPWwztBg6tOsSCPQt6oShdK29g11JBxphVuEY4UnKihK0zRbbItItPURbhzcWkT8mNzmVHn7kYbooSzxajBWtRO6POjKLCvwJjxwrKCteR1+iArlnHg2ceZPmLL+FX4UebYxtnh54lOeQREh5IYP2VBNoqGrrbcG7YOdL7tfCk16cIZguatno8Sy6jt3Mha9QTqJ3EMaGpoEkc9/OaMMnNbJ23lZja+zl5Ep7Zm4hEKkHjoqHc4MDS754jI+4cOfGNSOVSPOI80DXriJ4dzdl3zjJ3ghOzLrxHTa4Hslfk9H2sL4UHCik8VIjCJGejbzFOf07GUSqj37J+9FvWD4Dft1zh/ODzlARvx6XJBbdGN4ydeuQaBVa5Eimw2AGuWwpwsq6hJKkVB/878qZWmYIm3ziqhd46/hVd+VQ57Wb+UkeyL4q+QwZbF2RmA04TRhBw8hBRmUvITvYnxe6Of6JEJukuiD7vbuEnVR37Wu68CD7JIjK2rF1CTYkP69c8gs94LU5BTpSdKaP+Zj32x7z5JnUFz337EeNau2g33EkiNBc08VBZCErfQo4UhDLp43upmmWFZ+60vcd4/LewCAKtzq3YW2bi75rU+wN+dbh5tHCuqwTXlgJubhdlGePvj+dCxnlCM85R5ehCSfANsXD+yUXy9+UTOz+W2gQD5z3/oNWpFZlFzdO2r/eQTln42wyUdp18+OH3EFnPasMzCFYBt2g3pMo6vIsucLZoJXK3TdxjK2C1/vva06lDmmhySMXLJL7LDn4ODHhqAJ7xnuQUluGdsxXPKhO2nWb0bX2JvCcS3xRfXCNcqejbwZW43SzYvICQ4lAaFzWyaeImhr8+nNILNXQ5x9EvqpiV93/OsJy5qJ3U3QkJeYGWedvmUfPQ73wXmIfJJD5fk86EUWtEOmQyN7znIDGBtbwUg8UKjOKL0C9oKW5hwrHeMsS3Q0ycSrmReIOISHFd5D/Yn8KDhbQUt1DtVYVSKvq/WQULKkcVXkle2PvYM2H9XP71ymmGu+eRbZtIl7QdB18Hqq9Uo3HWUDmlja1JPxAd4UimWwJVkcvZNHET1VerGfnWSDRtMhxUbkgcGvlqeiKfV7dhtSpQ6Nu72xdQHoDcLOeeffdgDmwCQjiw8kB3+0Bk/v49aoJaKFPnojKoiPUU2VgL9ojgl5pbw2+FfwUHphzg06S5AIx+dzT2PvZc3bQP551P4vjW91hl1bR3WrDKlXQ5enPmse8Iqm7Axtee/P7z0JjEjXvyo8nEzIlhz8YL3HzmJD56GxLKxd+ZYrXSplNiVtqiMOlRdNiSZbJhXv8jBPqosXNejSCD4qSZaNrr0br4Y1b2ZiyqnDQ0+8X3+FvQyCCCRgaJ/SS9gbjzdSx+5SC7bJuBj/BJ8WHahmn49PPh+MxsyuNP8OOZKL5riiCsS4/cdCexaDg+hJcbfbk06CXCjGOANyg4IDIDNMUWTPEmwm1O8NWU31lZPAC4hNPlo4TkXgDArrmCOPlVFjZE8q8qV9ROaoa9Moyz757t/o3bhQAHPwcc/MRx0qHRhoCsOOKy4tA2FsNs8XOzNols5iFPBWE2z+H3hb8zVzWGqstV7H1kL7M2z0LiqGLP9D3YddjR91pfZBIDth62eMZ7MvTloWx7Yg+ObY6oDRaiLvxETfRwvr9PTKiPeX8MSxs3UCeUcK0pAYwZWC1gyitGXf0OmmAN9/mIRZCfr/bh/LZBBC1qRNpfSsDQACzuXtRkXENv64q+vQ+Ojko0zhquHrpKR1UHJ4bvJvFGAq+89xIlI0sIHRfK/N3zUdorEQRQm70ILp6Ib/5pDC06FuxbgHBrPLpdACiM0KGRPYTSUdN9HwsPFmKXVo/3ihqi+uWy37UBs97M9R+vI1gFTIFueBedZ3+uHXsj4Ns2mGoVsOQUMOCiJxcGqNkQZCBQo+UDx0M0jwe1E5jUalq8o1HaKTm15hhWs5XEBxN79DfnQd60PvQB98SVcdNNHLcFi8DR5452z3utjq3EZMfgoBILzjc23ODm9pu8ZnqN8fUHOSmdx+IfY6hLEd/zzZM3U3WliuBNY/BnJY3Vbrj5NCIgcObdM5x89SQzfp1BUX8d16O28FunCu1DG3BHfK8FbRdd9h5IY47Cxb7sHF6L3ycfcNqwiNsLRIW+g+MTP+M48ODpBwkcHtjjumQrZPwq+5WFvy+k5nINYYlh3dduscBj384gLL6AgmpfqtzAEuSJ3sYFdVczdmPdqfeoxyqzYpFbQCLBJ9mHF9te5E2JuCeavXI7ZzUa+l//Haf7PHi+6Xmy/8hm32P7UKBi4KRLqDUm/njyIPd39sPU1oVNWxvDs8KYWfUutquXct60F7s+lcyNTCB4VDDj/zWe1C9S2f/4fpKWJDHtx2lIJBLUjuoesq43km5gbxbHiJg5MRQfK6bwUCHSaVKuHB5IZ7MDQa0GdJN13Unlq19fRQWo5qkp71PHvOsfMOL4CM6+e5ZLn17q9uw7PeU0UouUjMoMKssrUTuK77xNVCBPPafEKlNQ99s5WnStTP1mKgUHCrjx8w3qgrV41okFyPyIPKLi/8ZG/y/CajQjN+ooj51M6DCfXsfDMgLwMapxbnFG4XgRuKPMcfXbqyRfS+bI+CN4V3vTVd8J3FFokNmqafaL7x5nBQHKzpax99G9DHn+DuvHucWZ0HwBa0sbZr0bXgWXMGocqPexp9WrHxGTJ3L9hBNRAuhLa3Etr6HLyQff3OM41RdQeHAB8bMjuvvH8pvLGS1/hQLDeUzyZgSsdNb19EkusRtPcVYw0qvRKPuYUIYqRe9cq0BVcC07Hj9EbJECx+wIBCTYedkRf388UrmUnxsPUmeBt9usnDHBw3ZKVpWuQtes4yPXj/AbGs+lITnMV2wjs1aBpy1I62pwraynJH4aRhtHnCyuODUvBkU7RUoZUouJ6BPf0O4ewjPVz2Dr3tuGBKAweS5mpYZO1yAaGiAy0o3l2cux9bDlwNl8Dk48SEJ6Av2u9cOsNyNTiqCk4mPF6E63MOzIZdzypyLIbfFY48HTlU+jclAhUanoEuD7Nsi/8CDxHaI06+DVg2nKb+LaphsMNTXiUj0E12uHcb41FhgS+lOqiiSs4zSfdQaSGJTH5288RYrSivVhK9oGLW1lbcjVcjwTPWnMFVlTx18+LhZwlDKEBwT0aj0lQ0t4YfoLgCjVp2/T01HTQUpBCvqP9axdvhaz1cwbI8U16s6dsH07PLppOh7eQAMILgIyO1l3/0o9ksrLa1/mQt8LeNZ50qno5Pfdv9NR08Hg1YNpyGng2rfXkCllvGroLc83+evJnH7zNL+N+Y3w58IhrufxrsYu5Bp5tzzpvxuCAL75pwDQldUDPQt2Jr2ZyuixODQUEXnpN1q8ohGEud3HffqJ7+qBFQdYen1pr/N3OYh7pJZT6fDAnXe22TuGdrcQfHOP41aZzoV0cAuwwaOPB1HTo7j6tcj8dA5xvmu783Zk0e9aP7Q2WqwP9LQCGffxODbM2IdteTW2bTXc3G/TXeyzcbUhenY0HV9cJqHmU3a98TafuVRy9MiPPHt6MyZTGx72KnwagxjQ34TW+z7cR4oqPGPfH8vod0bzyWttVDbZ4Cw0i56dgngfW7xjuO4ZjlNtHurORvyaRQfAv+Y/GgL7MXKmMydfO0nCogQChwfyy6hf0DXpuj8jC9RzcE4Er4wIpDa9ln2P7UOulpP+azr3uS/g6xVfi88OK/n786m9UcvU76ey77F96DZPI3DJHpr94lHYwe4H/yBriwhCRiLh0Zr3uOgyA0eJtZeCy8DUfkTrNMx4YT2EwbmAl2jIbuDSZ5ew1QjsTYJw5SWs/lLOTztMWcJehF8biLywj4bAfgjC3zrlX6LwUCFXv7lKzJwY+j7SF229lnMfnCNkbAjtbjr+9fTrDEgdRd35KTAijJw/csjcnEnyY8nYVUiJq4lj4/2/4KccxIId4zj56kmkCinhUyNpK2tDZWNgy/r5jFbPo/+fsZQcL0Hfqqemo41xIy/h4tlCU7MbA8pE+4PQ8aGEjg9l9Ws/E6sEi2sG1wv24VcWDEwjdl4snvGefB3zdXe/+XvomnXdtlG3/Rf/u/h07VpefOEFRikUrNXr8QJqDdWskcn4Hrjvvz0D/KoRPST/E/9n4/8Zl9T/xH/ifyPOf3ieU/NO0ZTXRHCRGod2B6JyPDDVNVN7Q0xWW+VKdLZKls4/zc4ILUElQRi0RhpyxEStXCXHkFuCd/5pNDoRQWMVBLwSvLo3p5VJL2MaHs03uwYSdrJ/z6KCTEqnfSfXBBORhkY+8yxBppLTZ24fBqwagPSbSww/PRypRcr8LfOZ9Z04cfok+9BveT+iZ4tJNs+43vr+txEdV1Ou0jhQ9LTZt2wfHzh8QOrnqaRsd0aj05CemE5xpIiSelPyJh+6fIggQNTZr5m1YyqhNS/gTAie8Z7ELoglenY08lg7Pnn2E0qDGwi4eQhzVh6OAY749vdFKpcSdN2eQWcUBJpHMvDiQDzz7iyicv4U5SQq/CtodW5l25x1lISVIJFI6LesHwNWiaIA7n3csfXsvVie6jwVjU5DhX+FmPwTROmj8x+eR2mr5MDEw6QnpFMenYwmKQqvBC/qbtSxa9EuAMr9y+mw78CtwQ1rl4m4++KYtG4SChsFVqmAVWal/+X+LPlxCbrGjl73NEAO+nY11enu6Kpbu4917q9j1eer+FAbzsu31lhmy7/P7Is5+x0xZ7/v1t2297Yn+dFkPPp4YK7pYvYWbxzbHDkw6QAat56Tp0Qwc2BWCDtM4xh3489uabHGXBG1v2nRHvyf+g33/TXY7tuBZ7wnUoWUQasHIXSZ8KzzRGW4XWC500GbC5oJy/bAoDLQbpPO7tk36ZjqwL8bBpkNtSHiIrX8nJjpUzupiZoZhZ2nHTE3fJi2dxrDzg6DkkLkajmR0yPxiPXAahW4YYDPlE2kj5iNw/AkfPv74pXoRWddJ4OHzEeIe47MygVsevE9YnPicAp2QmmvxM7Ljg7bDtrcmvANqeb57z+ga8zd5c3smkpxL7vaa/HmGODYq9B8O/K/O4xH2VWqwhKQ/40JIwhQ52eHWq9m2HE7Omo6sRgtrPVey4GVBxBkYFAZRB9EiQlBgA1DNvBtwrfoWnTEXPThgc0LOTjpID889lOP8UIiSHBp8SKgy4OLNY7Iqryw/gXJW3qqFN+8kxQniUlBaXlpj7ZVpYryHm2lbYSEhPQyKk/9IhWg28PlytdXeFPyJllbsvDXRDIkJ5WUa2PI2HI3KRSxodP2TMPlfAPtVe38sfAPsndkI1gFHl6/kHFHxxGQn03a6d09v3q1k9UfryYiL4IfH/2RvMk9ZRZCXUJJ9ErEtdGVabun0Xqh9a7P5Z+ivbKduvQ6LKbeRcJmvwSKkmZjVojjt2OgI0o7Jf0e70enWsdPC5eSGy3KjvzVH83G1QZJSQcjzowgOieayQcnoy9qYcv0LVxcexGZQkb/C5EElAfQ4dDB8bHHMUfc8S2RW+xw1IoFtPE1JxhxMwtBAGtTC33Ofo9b5Q38ck+gMNxJcKyLWMf3yd+jrddSHFaFWq/Gq7weqUWUlExYlEDE1Ag0ElucWxyx1doiSCwIgojkXpa+jLPviAnSQQG78elMw6UmG8EK93x3D09XPM2hJw8xbfconnWCwvvNFM6d1ON+yb1sOTr+KL8MyePP6jgW/r4Qa62Jksv1GM2HiFWC3goNkmhk6vFoQ+di42ZDQMY+gm7sJv7EF7hWZWKX3DOZJQgC2hlXCXt2DT4yI+t9RYnkkqSZ3By5nI6vfyPyoAP+lf6MPCTB4eA25Co5bwhv8LpZLHC1LP6D9IxgfqrRgMFCxsYM3lG9Q9WVKuweDUZW6U1loR+SkZfoCDAxfu14HjzzIOGTwykNt5Az+hQftnoz4P0XGLZVfDa/jv2VDUnrsP15Dhvef4BDcgfK4jMxufUc25sLm8n8PbM7IXXl6yt84PgBlamVyBvBv9wflcEZCTLMBjNvSt7kTcmbWIwWjA82Yu2XQdA3wRTuOkjo+FCmfjeVIc8PQeah4ufFPyNIBJZ/tZyWvDqMWpEV6j/Eny5XC4XhhdhbHqBvVUE3mv52pPXNJDPhBnl6B36qC8ZF4UPZ2TLeUb5DfUU+2+6v4kbYJvqfG0S/955GW9H6T6/R354XFEd0MX33dPpsEzfbpi4TGhcNNu42NLaUEpKnozQgl7yBi1A5aTj/4Xm2zdkmfv/WeFEUWkRWfCG2HrYMfWkozQXNXPviAjKTAY9RxQAoJUZ0zToyN2dSl1GHMVTD7JU7+CiphpEaceMO/xd7fxld1dm2/aO/ZVlZcXd3DwnBgrtb0VIotLRAW6rUjVIqd92FlrZQWtzdLVhCiLu764ounf8PEwIp9L+fPcYe7zPGfu/zU5KZ6dc1r1OO8zhg3MZxPFf8HEakmGgsOO0OX37wB0efEJMfd4phlX9U8suAXyg5W3LffbVl5bNiUyTl3uV0jxeTngFTA5j+83Ss3K3IHFnPvnn7GHduHA5lzZjZm7E6ZTVLji5BYiLBrXArRemBfPjEJmqH6dH16Hg652nm/jUXs2vdnFnzNvU1DqSEGii3K+1LtEQvi2bwKVeyPliN0SDlh4TZlDdc7+tIAbgxUs/JKScp8SsR9W7cREfj5g83++7ticQnCF94P82xm3EotW617Fm4B4tYO+rS6/g5+mfSt6UjCAI/P/Uzx6cfB0QwReX1ShBEwEavj4zhMxNoa7RlfPIgZHoNoQm/EnL1d/Q1jUgA5xo1b0seYXjyBtSlzfwS9wsZf2VgPdyFYT9+yLfLz7A28xJrN21GqNDiNj+e3JFPMn2Phpc+f4WzLSJwylwhsmtQWopL8TUCk3cSfvEHJIb7mQPufI7N2qqxbBI7fjtqOvjS40subriIanAkuqmuuPrU4mguvsuamzXY+NhwdNVRVn4yir0dED8kDyG0AtXgCNImvULmmLvAkpqgYFb9PJ7Akx1c/uBy39+t04y8fmomVs1KNuwajvqsSL1nMILWVPzOW0qKaBAs2HFwOPYnYlDZqRj3wThWXF7BogOLWC+s76M+b69sp+BYAV0NXYSMX8W2x9Io9SnFaCOu0fse3seRVSINs0W7CXYt4hptxIDPGB+GvjiU/Uv2U7zgAKOyMnGti2DWkVnIc3qZ8J8JrElfQ1tpG7JGHdOPT8cgbeHElBNUedf26ZbW3qplsNMZamIvYoIa91feYdhxE/T5JQy/GolprynrmuD5Whd2NDti2WEJt/WaF+xdQP25TFyLr+Kbcbiv+BE8O5gZm2Zg7mROWmwG58edJz0qva8jI3hWML5jfWmqa8Q5+0sa7QpJnfQqVgGONGY38q3ftyT9kISyTiAkN4Rc/1/5e/oAinoKcQx3JP6VeIatG4Y2VEpTjQORkxI5NO4q9Zn1nHj2BCefP4nc24xvnvuGXwb/RcW1X5l34FskzSAbMYzD85tws+1kiSUMl3/bN66MeiPKzmakBh0OWRdpLW6lKb/pvjEYGzWCdRNSiHVuxsooxjbmzuYMfXEos3+fTd6PMr5+8WtC86KJuSKi211iXQieGQwCKCsLcam14Gr8Vdp8Zdz88SYlZ0uwcreiq7yFtq0zuXlmEL//ZykBaT7k7Rd9kZbCFnqsBG4519IklVHnWke7rViotq/ORNndhjzHn5y0QM5oXHmis5dqqTUeHp5IJCDIFJjYi3Piznryz7lV5VHF1ke34jFeBAB21HZQlVhFr1pD2fB03IZkUfXRf8ib1QVOThhloi/cebaRJTtXEZ4VjmuNK7oOLfuX7mfz0M08V/Icac/W86FXDlduPURYlhOSXrGQcXTV0b7z+4aX0qhx4XrNBGY6vEh7Uj5hCb/gUJmKXNfDM+6evDjoGh2+rxMeMwiJVELhscK+TuU72vQgdlLekRJYum0psw/ORriHdaXyaiVV16tQ+arY+cqXnDNvQflLDG3lbay4uIL1wnoe2v4Q0nGDmT9yG6U3InE/MF6kwgRC598FriUPSmbVL6uwecoGla0Ktzg3zJ3MaclvxKS7DYC6ayVk7xRpCUe8PoI1GWswSsG7XCy47l2wF6+n+1N5dtZ1kvhdYl/x+14zaPR45p7BtiYbX9/7NmP+1GP8/vjvRGZGIq8Sv5NXP73K9S+vM3r9aC7NqcO92p3Vv6ym6XxGv331vTrkmi4kRkPfuND36mnOb+bwysMUxrWyeeVmcsJy+Pj1j5GGBGLs1eGRfx6LlkoqYl/C4LAWW4cANBYOCAL0ZBTim3GY8Ms/Y9MgxhJmjv01nE0sTXA53saLXzzDi1++iMdFHfZB9jxb+CyvNr8KwMxfHmPrtwu4eWYwJhVG3Aa50ZDVQN6BPOq19ci9bmA/NYFDq3/DxM6cvAN52PjaMPmryXiauvPEkcfIv/wedseTMJNbkLwpmf1Lxe7UqIYFrKr7FR/pVTxIxaDVoyjMwTftIJ12XniG1PBVrBvPmTpg1RmA0SDgln8B0+5WDAoV3U3d9xUn71ibayid9j4AKBSQ8lsKP0f/TFNuE16eoUhsP2bLii188uonyC1MGP3eaAauGYi1lzVIpBilUoLzgwnN8kFmIsPK3QqlpRJBAP8b0XyREITsQDgBCbWUXy5n19xdBM8KRjfBEtP0MJp7LPDNOEfcia/QdunQW9lhlCkY+/dojn0uzpMrw6+SMrgU5yhnXih7gWeLnmXk2yO5/P5lmguaAbH7X2mtxH2IO5omDf7F/hRNKCJ0rjgfVLYqnsl5hpFvjMQ+0B59rJ45B+ag+0FHc0EzGyQbuLH8Jzo6YFbaJGb8NIPvvL8j/c90qm9Wk/htIl0NXXg7exMYE0iAc4Co112mI3xxODErY7j5401u/XxLHOdODy6uDnpqEDErYyi/VM6F+Rf6xb2CIPCZ42dsHrz5gfv+v5kgQMGQpRQNXIDt6Kj7tneUNuOTcQS7WrFzrMUtol88H74gnBFvjmDg6gdTGTbfpv6s/v10v3MKMgU6U0uabxfgvScEEDhd9J/u5NdGvTOqDxz3TzOxEv9+YewNwh6bIMqi6I3kH87HJdoFkycnc3FMMr88+QvlP/SnUw+bF4bazkhKTAoDUqNof/NPxgc9yqAxOcwKOsxzQRu58UoiV6ZdodfSGpm5eC6pXIq+V48hM5ceSyckgpGwyz8j5OSKz0QqxShX0uIRhXPpDdq2He273zs/RFz6idyvTpP6W2of89XIN0di7mzO4GcHA+BZbkqly2nkSjlSuRQzBzMGrhmIzESGU6MTlmpLRiSMQJLfSfqWdC68fQGJVILXC/EIUthU+0TfeYtOFQEQtSwKBIEmxSv4lPrgX+iEpacN64X1xDxxt0g12V3MW5wrfRQQ8w4pv6agrDcSeDsFKpUZaZXqESQCgk6HZUsFfqn7af9tLymbUx74vq59JoLHZv0mdknmHsjlxlc3OLrmaN96MOnUCNL3it8u9yEi4K42tRaXZCXz9s9jiswEOfo+LVv3Qe4UHsnHNLKI+c/uIf98CNLiQuRKOY8cf4THEh5DYmfCz9XinNr04XN4nvrnHBH4TQ1n2xxJGpzE1Umib2LQGvgx7EecIp141/Bun4bgvWbmYMa4j8YhVUj7MWP9m331xRd8/NZbXDYY2NLbSynwOrABMDcYuIZIHfv/ZonARa2Wlbc1JP9vMqlU+sCc3/8p+29n33/tf82svazxGOZB+MJwNv18Fa2Jlk1rk/l6xDu4D+5i4f6FvLfJDSRJDDaFjCMDWXF0BF0ru5j9+2wacxsxtTUlO+M3PAoURKeHYN7liLBIwNTWtK+NGaC114GuGxH4t4uB/ZFVR3AMd8QQbI5DowNDHDuJs7Tgp0ZRAFthpqCnSUygjbswjisjrpAfnE+nk1h88B3ni+84XzrrOxn19iisve9vS+65Wct7H77HgTkHMLiJjrqFswUGnaEvEFr35Tq+euErjPdws/e29ooFC/sk9HItoy88hP0QJRKJhHnb5wFw4+QxOi076TW9wu6HTQhyN2FhpR9J3yXhFOEEI4fTZm/F8bBpfPbJW9SHik5vym8pRD4Sya+2XWRE3w0qBAQEQcDWz5YpX01hyldT/vW9Wf1hxTO7nsHukh1Gg8g1HrUsivzD+ciUMm4OuYkg0ROQfpcuUGl9t2izb94+/Er8WPvDWrQ2zTRZW3Ni7QmUW5WYNknwbPFE1aPCut0awz8Q8i9++RzqoGJCXzyExzoz3GfcXehlQWZcjb/KHt8a8lMG0ZYYSO/MHu5FSf6/WZtzMK7FV7E8tgsQeei3jtuKrZ8tVnPHkz2knpywK2R7l2FireLcm+fI2ZvDo2cfpe7Ml/zw+1NobBUMz0hD0zGJrWO3orJT8ULFCwgIRCqhVaGnwZiLlYcV72hF2szM3df49LVPAZi3dx4eNX7wAeQfySf5x2QOzz1Jvr+4iKdG7mGs68J+171z9k6KTxezrm5d/w4zQGKqpDp4HI3ecaxbKxZT0remU3urlvh18aQPrSMn4Dozj86kK6YQM4cAVPYqzr56Fv/5UcQlD8Ro0Y3W3hKptQXmTgbq0urETokZEeybPozKQg9+71gDPUamfT+tD4V2fvhxssf+wS+3uxae/UcQ7JV5lB4LJ8zUdThWpqLrieDeJWnMhjF9qN5/Wr3tMeROek5M2obSck6/bYIAUqMJLnUuBOSr0Kg1aNQyHEIccAxzpNeimk3Omxh5eSRhefbo3zdg5WFFb1svLYUt+LTIcSv1oDcAlCVCv+DAwtaC3LEf8OssTyplVQwZfo6PHf7Tt13XrcOkux2jVE7axHWERfW//pXXV6Lr0qG0Vj5Qv+6xBJFu12u4mGgoOFIAiFQ/QpgNkTfbKBm4lMnv3C9G7lHkzPiqxdwabku83zQ07RqydmSRtSOLF1rfwaLTAqlRyjfPfcMY92X99hWUElrsWtCaaPGtf54Q3ui3/YePf8A0yxQLVwtiU2PpLurmf2o1t2ooOFLAxM8nPpDSpNfGBSNSbGtz6G4Kws7fjjc6xPNfThM7Re2a7XCrcUM7sJeQ2XG4DHDBLsAOfaE9P9m8hF6uJyMqg/cjZrH8wnIacxs59+Y56oLckVq+jq3wAq2SEoyCkab8JhR5tSglfphpfGk3T0ZoTcOhwhqtOhyJmSn1vkMwa69DIhixri9A1xODQqXAZYALdWl1PJH0BPE/xRN/LZ7ojCzSx09AXSV2iLrGuqKwNuPWlCfxzLJl+GeNaGb1IBhN++hKAM7+OprqWW7UzngSlZ3AicNaLK3vjom4Pxfz9tqd/OfWy6ir1Bx+4rAYPK4R6UOytHJmDcyjVB+HrZsrxx/fysDqGk4Og6u90Km6S1dj1BlxqriFxswGAJ3JA4Lye8b5zjwXzHvaAJAYDSi7mjGUibR3pQPmINVrsLO7X1DTes9Uzlt1M77OgYZhnbQp2jBoDRx/+jjKtwO4PvQ6P5uXUTemjjFmomae90gxwWYM8eSMcxpa9esEqqRYWosBioWzBZae1nRUtrPDP4dnfdQseu06Lzb2V6o48+oZ8g7k8ei5R7FwtqAprwmNWoNcKcfpopyVl1YC0GGZjzDnHpoxCXjqdnJRO4XW+Bs4+szCJdoFl+jbKgQNUsp8y7BttUUiSDAYDUz6YhI9LT3IFDLkqJDrrTHrsUXZa0DXq2fT6N9QmCt4/MrjXB9+C62sBZcL13A8lE7QcgvMB4rPv3tPNjnv/co8a1dOGzypN+sgVnr/c/03q7HbTWbkRBxsRA2p+sx6Lq6/iFOEExJXMz595VN0Ch0DyhchU/oSMjcElb0KTYeG0GQlc6vncmXOYc52NmDpZonKXsWVj68wc/tidp2oZaF7InnJITg0m3Lt82tc+fgK9sH2GJ5zROWZhZUJPP3NAtLSgkEEndLT3IPdps94k5eZ8Pd7QC9aU9G/ea74OaQKKV97fS1ef3LNfZRv7fIm6p3r6FH14GYiJq3ay9sRjMJtjUUjcoOcUQmjqAi8C0poKWrhu8DvAH921hvp9bmJgMD5t85z46sbvFDxAnprKbl+xYxp7aZWdZU13Wm8euUuJXFpWBvBoYV8XKNE/3MMPXGJ6KcPxeR2wd+1bTYrGjfyu7uCVrtspNbi98wuwI6WohYGPDbgXylxJuyMZEjzeg4ubsHGxAGhR6A+o57cfbnYjRW7v21abYi/uRRrB0sKbxaS8GEC/pP96Q2S8bHLOeafnozLuWEwV4vEqMesQ+xkMri28NbGn7m0fww9vZ309PZQe6uW2lu12Ix0oaB3FhcePkeM4QYVdb0Y1BoU/2BwS+lxJ+n0YM4ej2dCeD3So4dxamsDQKHtvq3N1x+725JdS/i5Lah6tOhMVMCr3Pr1Fh3Vor50mVcBEyd/g6ZHSX2qOEAOrzyM1wivPrqrhFR/5o8qxtOkF4kEnEuu99GsCb527Jn8HhntQyg3c8TEwgSnCCcassT7Nr0+EJfYJnJuhuPQLfrglSETKQ+aSGScM285dfGJfgW/TTiJkyacj29f951vzr22ZdQW2sramPDJBCQ2w+mw1rJ1xVbWBYwGRJp8fY+e6EejObEol2rVKcadG4e1QyeWr1tSc1Ncqww9WqKv5SNEhLBt6TYmR6zDoDOQfzgfx3BHupa4cKPubyq8Kuiw6mCCn4qZr8zEfag7gkHA7M9R7BiYRnOYkfqILAzWwYSOHsq3AatRW6nxl8A3rnVctu7kzbCLTPMRE5H5h/LRq8WYpSxyBqa30dy3Nt1CqpCK2iyC+C0r8y1jX/h+8g/nk/BhApO/nkydsQ3P0g4qXU6ht5iIROpDU34T6io1pjam2B4QWHxuMd+t/Y5mh2YkUgnTjz2NTAZ/Bn+EslvHZ18s4NTKBCK7F2Lpaon/JH/cBrkhVUhptWvFTAIzOmqoSRfQN2rAyhGj2WDkiu+p1iqpVX6BZVMZ1rUNdNQEE3nxexo9B9DuFcGMMV3/qiX+1Y0txBp309BTCcvDydqRRebfmRSfKiaowABvw41RBqaaPIpBZ+DcG+eIfDgSiVSCXeJJ4iXhXJo5i2DfSKw8dAROC2TCJxM4WZLP8UW7KVdY4nUunvDgeJadiUPbqcXS3RK/j8uQNrbi5laLhaSe/B6RZrEifAo29flU2H3IjSEJ+HePxaXWBYU9mN8ueutNzBh9eB3x8fffT9bOLIzvSXGLc6PUtxSlkxLBKPClm6jPvSJhJbtGpfJ1aBn6Hjk/9YYgkYja33c0NAMKXbBtmYB9sy3dnk2oevRUJ1Zj1BlR2wnEddlgXd1Oc8YRuuq9qU+s6jv/yaV7Sa434Fu1A+nueiJnFaEK8aIqZBytLqEoTXupMc3CU5LGigGvYjR+SndzN/6T/TFzMGPQ04P6jqXr0bFz1k6CZgUxdsNYbDoCCCgGx44AeB/aytsYvX40nsM9yTPmoXG3p8rBhJdeWtyvMyby4UjO52jwMc/n/Zuz8L0VRcC5ANYL6zn31jlkShmu77qCDgoDC7GSWuE10ovhwnBkChlH5v1GBHBiegaXnnQi1FME6Fq6WmLhbEHi7BryFPv6zvdP9ooTz54gZ28OdgF2BE4N7LdNampCadQsTLua0DWrucM2dMdMzBwJLtrCyUUShtiLCeOUzSkoVAqGvTSMkUfdCbOdyo34WhYE9C8yNpzPZsCZQzS7R1LrPwJBcOpj7AAITLbl7xniu7PsjQOpDKmZKdkjVyEz6Ai68hmZUcWcb/JF3vsMgmCB6cBwbla5YNlchmvxNXJGPIlzjFg4vZeVQNAbkBkltFh2oDOxQa6U9wERAS6MvUinRQfF/sXM8XwNg6k5tmOjMfdxRN9dxaRuW9537CJdo8UgGMnZk4OmQ0PewTzsM42AN5HVCnTVx9F3RXNszTEAFh1fxq5p24h23oF9TzO1R8fTPcEFTUg0lXgi7TyJyY48ql2dmZPng92B0fRM7cS0vR6t0oIWtwh+jvqZgWsG9tGl3msybTfemcdodwpELh+AlbtVn9a8QqrETefDS95ajneDEQHPYZ59NI/5g69x2Osk5p2uBLc9i75XT+HxQtrK2ghZFM2EM8OpDfZCNRLa9QNRV6nJO5hH3sE8TID5zGfWk4foGf4xZy7PwyiAYBSwahKBTx7xrRxrWcDeAXtwIbrvmu387ZCbyolaFoXPWB8AIhZHELFY9B3K9pUxb/88elzudlntmCl2371Y8SIDVgxgv/d+rB6ywlhu7OtWvONDKDRyelQ9lI0vo9Ojk6ITRVxcf5HiU8UsObaEFZdWsG/HPjaHbWbzzM0MixVBxKFzQzFzMMPS7d/pPAWjwLgPxiGRSjDqjUjuAcRJJBI8hnr8v+7/r8cVQO0oAjgkD2gKVFibUes/nE47LzrsvDHK7/+n8R+O/5+dyyggkUro7ga76kykBh0t7pEkz1jP2i1gcVtV44424+WNlxn7/tgHHit3601anUNwkazH//ZtZ27P5ODyg8zbOY8WCw0Xx4jFNoPx7neos76TsotlpIzv5WL4YUJyQzDxckaiFO9L2d1Kc1E51tFhaJSpmHeYoO/WAia0lrZSnVSN/uQ5HO28UbaI+VFjj8iUI9d04ZF3FofKNNodA3AcEgH1d4t9kntYtl6ufxkTS/Gcw14axrCXhmE0GEn6Loki/yK0wlk0HRqcwp0Ys2EMaX+kMfuP2Xz0TTZm3ZZMODeBXrMuxn42i+IzxZx49gQTL77FnMilnDgaQcqZcvKyH+aZ3GdAEO87Y1sGj2x/BHv3BmqanO/SlI/xIXWzmE8dMjmJ1gZbzn7pik3pn7z80yO82vwqi39/j5GVMM5sJodK62hW17LBW4fg4kazWwT2NVloU7I58mQ2sU/E3ve+JvxnAhq1BqWVmMd0jnTGLsCOsRvHktjexaCkQZwbfw6rAYF91zTq3VE4hjlSPaSHqQMPsrjTkxfrbPF414NpP07j+DPHcQhz4rT/eSZWg2xmPAN8htNZ30lHdYeY37jcjCPTGGp0JtxuImaOIpgo/3A+OXtzcLMzoHOG7G5bCoOu06EXB+HpV0Sf0DHUsR+t7L2mUCkY+cZIRr4x8oHb77W6ujpef+01Lt8u6o0CxgIvIKrT1iGGf5OA0/CvGpIzlUr+89FHuLj8TxX+/v/HJBIJVlb/80aN/1/bf4t9/7X/Ncs7kEe3uptBzw7i2CvHiTfGc2xOAUikWLpaEjo3FN02sGjrZOeHK3D1reHYtGNMtV6E/0ixjRmgdHg7R0L2MOXsanyLpLgq3bF+bTi9beKH0bLmCKPs9zD41QQiC+35XPiclF9FBIf3l6NZ+8NaXBadIqDHDI6Oon1uK1Ympox6ZxQXPY3sblmLUWrkxrAbeGruTtbrX11HYabAOcoZfY8eubL/dJLYmJATmsOk05PorjbAk6IY8uWNl5HKpRgECbWuMG/fPIwKe6TfSll2dhmadnHx3f7IXhwbLFmzKRijmxO3fmnh2mfXWLhvIUGKaEYn76LQ+zdygk9jJfdBKpcyaO0g/Cb40VWnxUSowEyu4/H3NvNU0xgAjq05RuC0QDrtxKKQTasN48+NRza2A81sDZ/YfkLMEzHM+rW/IO29Fjk/kpacFhpGN5D9dzaRSyKZ++fcvu0CRuQ6Od7JH5Ci8Wa4wxxKz5f2bTdKjdS51JEap2aGuzuWbpaMencUHkM9GP69F8NurkT4/AM2jLnEQseN/c7dYtdKtVkHQ1MeptWQyFvqFIYiorLkkRac6TzDzB54vdYBx2JfDNr/Od1gdegEVB0NGOUKEVV7o5KyC2WUXSgjZFEYH635iQ4jxFaK/3/l4yuA2DEnF2ow6lRMHnOWHzXfolApMLEwoaelh866TgLzfbjYZMGEZ7ZzunRMv/Pe6VaSGU0xM4xEdrs22ZTXRNHJIiaoRjNWNpwvXhb5yY3/oDa9I2reXt6OaVT/Yp9eD4JMjsbcDuXtenTkkkjMHMzobu5GaRVPiscBatxqGFgm6lNYuVvhGO6IYBSYc34CHt51tD4RxKHGN3nY5S1WXl+JhasFFw9Xcnj/LIZMTiR38tMMm2RL4fFCbv5wk3EfjeNO9UCuk+PQ5AA2/TsDHCpSaXFxJX2QDGv/GB67R7dEEARkJjIs3R8cAFwenUibRRIurQ/1/S3vUB6mNqbYx/ow9kgl3WYe/PD8WeZ6vkvqHxdpym3Cf5I/wlXRMbRSW2HTokAwCszbOY/alFps/WzJnFPID+t+ozAtkEcSXkH4hzzdBL/fcTGv4okakND/XQROCyRv+ONEnf+GDjsvGPRYv+0KlQKFSoHBYCAzM5OwsLB+ulV3inx37KG/H6KluAU7fzt2vpONS8l18oLS0dssg39IIgfnBCBLU5M6aS14qnAIFYh9MhZ9jx6pTMo3L/yIUSp+EzUW/eeFIdyU31f+jtSoZOmBaZiHtSBKKYtmdsqMUcWj+Pzlz/nPa/9h2/xt/E9NKpOitFT2BZf3bZeCTUMhHnlnaS15ArN7uma1Td0s2rkI8y5zvCq9cJvuTm97Lzd/uEn4wnAEMxlNjiKSvM22Dam1CT5jfCi7KM5bO0s7bIJm4tz6BTOODkcxsplibTHWZ09iGv8YUkEMWJyLe/HJOEJ3kzdSS3sqw0Wwg2fWCbyzT9DbFopCpWB1qkj1kvRDEo//8TgCAnvm78G/8w0KT2RydNVRTCxNGHbyWVotrmNta0abSwhSuYzE7xI59cIp8VqjZcwPO0F5dCYr1RsoSc8mafbXAMz+YzZ7Vu+h2rKdrioz7NrSyTvkQfGpYgB8J7mx5O8lbIrLxzLkcXLtIomzVhG8YghHj+4iVwuvN8ESdye0nR/jl6inpehJ0ia9glEqQ6bX4pFzBvX1bLinA0kilaD4M4LdN55l1unxrCzwhV/AqSwJz5zTmAwZgGLEEHKqTpIR8Do++ql8otvM586f09sqjispZhyYewiX/EDMvEYz6q1R2PraUniskMyXMnn+0WamjqrjuzbY1yWQ8HEChUcLWXp6KXNMZ/KCTS4udieJ/GIfv5acA8Q5oFbDc7/KueIqjtvD+c8R4SJ+r7oausjdn0veAREQ4RQhFjnvrKN2gXa0RulprSvBRGuCStWKQqXon9Q6f5YVPloOLT5Jk7mo9bB73m56Wnqw/1rk202NTSU1NpUnA77Ext2AVC5F06Fh4sVpjDxrSbeVCxZtn6Npf7mPmeBek0lAatRh1BtxDHVk/MfjOXcwDes2a55wb6Nmch3DwpOY6/xW3z53C1j9k3AgBuGrfnmCeqdqtANFyl+ncCds/WzFbjQrcLfpRi6BJquzGI0zaK9sJ/W3VAavHYxduSm6lGiGLjzCZYn4rbbztyN4djC2gQ5YfLOBa4ZhnDowCgcnA94/e3P9y+vEvxxPtqaAgHKwkMAgi3o8fMSkbNnFMuozRMSvUWLk0xYvDA2dWDaLQbKtny0Gg4Fh3w7D09MT/wn9dSQAJN42bF2xldmH5mNnAiyD5J+Sqb5ZzaCnBhGQaodPx3C+W/sdIY3f0lHb0ZcEv2Ops44zP2kYOi8JHmM98BjmQVd9Fz1hCnY47MCzwpNnvn8G+bweuAdkWhrRRo7DZaIkcnZPPI2722RULtb0WDii6mzEoSKVjshAJLdJUQy30Uyz/5iNmYMZpramJP+cjMdQjz56yDtmtLJGWefItOODkI4XcJ3niv9kf0rPlzJCImHy+ZewbJcRkW6OxTATYteGcfXTq1zeeBlhuIBWgLwhNznrmc/bVsspGLIMz9wzxDwcxHvylfxSq6V1yGm0g0+zwH89YfPDyNmbQ1tCHU7EcnxQHp8ueI39ES9jE+1DyekinIvr2bVCTrXf27R5w82CQbRJ2+nWdaEfOxHFgT3iXFJZo3a4v2VFqpDRZF9Mq20rXXYxGI0gU4jr9qBnBvHjvp+wM8mh5IfHaCuphz9gwOMDSPo2iWk/TiP9ejZmf87n6VP1OGCOcXUbqg6RtcMxxo3IxX/ipmzl7NwTlNXNYugLQ3EMcyThowSqs1rpCnGn0sWLv5auxao3gi8Bo1GkZD+uNiFV3oqXvRM17jUYDC60V7Sz+5FDOE+MoutmLgVHC1hybAmB0wIZuHog5944h9cIL9ITOnCpsaLdVNnnby06sIi/p/xNU35TXyfr0BtD6fEVv33j/zOeXXN20d3UjX1NFma+JhQHFKO3k5K7P5d9i/dhamuKqkdL06wmOqw6bs9jAYPWwIm1J8Q5TBDnW+RMHnia0S/tZ07qc4y2tKDN3oBBbkBthHaDlFFmnXwigYuCQEdNB0dXi8nBssgZNHkPRH57Ga1Nqe3TL1+8YwE9qi4OPHQAg0EEJ1UnVfPnuD+ZnL2IjW9vxKzbDMuOcei680n7NaUvwSXrAcWj+1iXG0vbtRjah3ay6c0s7KsyCIt0Ji+3gKbhibyRGkJBqTNWX1qx9NRSBKPA5QvFONU78VFAO4tnf8Ipv+VYBTgj5OtwbR5PWdFnTCwWWD5iBLquT3DJdEEQgqgNGIFM14NT5gWcn5953/gDyN2aRPOmWg6YBWHblgi/TcHMwQz7YHt623qR6mH60ekURBvR27gABoJnBuMSI87PcxOT6JLWY6P9DzaCH8GzxG5HAGlTCTOG5/COPVTEq7lo/jYqO1DZiUX+6Au2BJ4PY9UXl/jznRdwcu2Ej0FjYY+8spvhVweTPPA6I9W92G9aQ+OUVvIHFyAIgUgkkvu0k+9Yxl8ZCPlaluYv5dNXPu1L+EoVUqQyKVZe1kgwYiWRYJMxgMHS2TSWF+FSdIVuS2fMOupp8I7j7IhPcK02Z5nrNEZMDSV3fy7fB3/PKFzQ8zyDXvmLt5VDmYKGG1/eAMA+2J6hJ8bhVBiA86SLVNRJ6S5VYTt5EHUBztTa7KfK/k+66sezuu48vQnVFAdmIFiZcPZVUXd4+GvDcY0RwTqCUaClqIWWApF6OWGqDxHXEnBQi9vLL5VzcPlBFh9eTMykGG66TcbMToFqvPgOrn91ndMvnSZsQRhte3L4kLf545XP6Rxznrclb6Pp0KCuUGNmb8bNozdZ0rSE7Y9sx05lh65bx7YJ2/roeQFqnZJJrKogsDuQ9ya9h7ZTS2tJK3O+DOTGkDGcnHqSiacnUkUV3LPkDls3DJfYe0BA95pUitSox7X4GrraAP5Z7OttrMW1LhqDyg2poxjnLL+w/O41BanIdUunONyeJb79ta1Vng7UeMpxq8xEb2KGIExBKpPycsPLfOH6BXrBwJgLY7gx9AbOdQ4YLXuQylX0WLui7GrBqq0b084abLbLcSv4GuHRt5E62KF2skPtFEh1qAgU+qeMAkD5DFMSIkww753AWP+RGLQGrn56FXWVmhk/z+Ba/DUesxbwNoBRKfDTR+001zrQlOSJs3MCZhuf5+ToVEZYNNM7oJNZv83i8sbL5B0UfTXrx5wYqtpBQuZDaLtEdgyfsT5UX69EUMgol8SR4bibulE3sFZMwWA7hHZnB+KO7gDkKFUaXLzriBqRjsJSSfmoZfT2gneG2IF9Zwz+0wac/hwJAjpTSwRB9Bmrk6rJO5SHNFxK6LkDeCiH8Z+IYpq1ek48d4bmgmaWnlxKg7GAJqtzmGpXYGkYTENWDbvniWwJLoO9qB4zh4+mTcTR/XeeSvuUyCWvcO2za9Rn1qMdZMYAIREH90Y8A6s4FAIK1UuYXTyKff4tCgc9TIrKB5eWE3jJ9yAxGOhq6KLweCE2vjak/JpCb2tvH+ONpkODRq3B3NGc+FHxpL6WStfeLrZ8toUVr6wgcEYgOXtyaMhuYFv7NjZe3ggvwJOxT2LjY8PTOU+zZq04Hp/5bDnF/kVsWbKFeSHzGB45nIvrL/Zjd5Lcph0V7kEGOkfdz2r1T3tf9j4Ar6lfo7CsEIPB0C/uXXl95f/HYzzIBAEkBh3Knnb0HWZAf/YgE3tL7KozsWitpN05qG+fO5b+ZzolZ0qY+t3UB3bh2VffBcQbDUZkUhm9veCXKnZwaVQ2dDj4YtAaMRokSGVSxmwYw/FnRIaGknMl+I33u++4I7+Zx69b6jGv+pFTS3sJvbwaUxtTbHxtOLTiEKa9emYPmE3ikETkmXK4Xa/uqOng4vqL2MfLIBzyQvNwnrKcri4gS8xhPblrAp/v2odnpSdjDyVR7+MA0wZx46sbJH2XhMTTHcvKcjqnzadI501MjAWCALa1OTjc1ktWdjVjOSIa9t19XoJE2qddLTeV99OZ7W7uJm1LGskzK5BWqVn7w1rKJpURPCuYhEPNVO7NIXxxOFq3BXi1yTkzq4J5E0fiEOxA/CezubBmN0W/X6bNfQZtptmobZrQSbRYOIvJsPzD+ejcughxqGXQhJtsvHwIbaeWy9/doOx8GQHTAnnXfgOZ+jZe0rugVahp0TUgU8hQ2anQKwSud4GDzBu7XBlzT05HE96JIEBp7DyqwiYxRLjBpCX3g6gB3Ae705jTSHtFO9Ze1njGe/Js4bMAXP3pItOPT8dsxnn2u4ogaEt3S2Iej0GulCOX6HloeB6/vvMk4c1inGThbIGptSlN2Q3EZo/BzqqVobOm827Jz6T/GczZV8/iM9YHabgFhlw124JyCVw4DKf0EXxHAvWZ9WRsy2DYk7N4ze4o37basd94V/aoq05kdcjenU1bWRuzfp+FU7jTA+7sf2abf/2VsQoF1wwGPgYuc39B7xHE4t9wYIRUypNGY5+G5BalKQl6Hf/56CNefOkl/m80g8FATk7OfTm//1P232Lff+1/zULnhVJbXYu6Ss2FMedpcGrAvsECfXsXYM6BRw/gnGtDp5eB6ipHtgVlcivuFjJLBYnfJZL6WyoPH34YjYWBWOt6KkZ20unxBhITOam/peI1QkycuxUkUK92w31SF1PcRQ0uuakc5yhnZE4qLo26hLNDE0N05lj6G5CZyMn4K4OTz52kcnUHda53E3eS21Mmd38up1+629q/5PiS+9B+cn9rdi/azfIty7HpcgTEbsap303FM96T7/e5cqXuawYk9KK6zeJyxzFQqyFAYSDWt5HUh/wJ8g1BZlJDS1EL1z6/hnJgBGOP5qIaNZzaMWcQjAKV1yu5+f1NYh6P4YTwMllO29lb7IRBXoWNraTvOvU9emKe/wpiomhwaiAyK5KeQC1SuRRbP1tSN6f2IWWezn4axzDHfvdlMdGCEIsQLH+zxMrTCqPBSOpvqdgF2OEc5czLX/+H7NAM7JotaG9rQ9OhwaAxMOHTCZx99SzrvlzH7kWXMbh8gbsP3PpF1DEb+vxQ7AfFY+P8G4du02/e61QCbF2xFUFi5IK1NbkVNtRVl8HtYp+FzAaAZA28MeYEtSNPMMdm3f94PLaaX+fQvCSCq8UCo0QqwWWACyPeHMGZnnK2d4Dh8iCeuTiM5pnNrE5bTdmFMjzjPblZVMtTXscINYFa4Rpy0/GsuLSChqwGzJ3MmXFiAd2qLia5vYMbEsoulrF17FY8hnlgeMSbsOwwWl3ktPo/j8Vt2vn4l+MZ9uIwFsc/h6RTTIC99cFbCOGV/RKUazLWUJ9R30+X6461l7USdeYPWl1D6W4eg7WLipbiFrJ3ZTP2/bGEqZ/npt1L1LrVom0Qi5M9LT0MfXEoggQMq/8mVmvD/nVv4jgFMm0yufDuBRbuW0hpUTLqi7Fowir4aLovh9Qn0F1oofB4IUNeGIJrgQqP5tG8GV/EhU1P0vlQN9yWF9R168iYuI4ypy2kBTyDS+tcJPK7BWbBKLD/kf2ELwy/rwAGYN4bjH+egcG3BtD5eicWLhYcXH4Qt4FuzDngQ4tdC13mXSC1AIkEn9E+8K5IXSAr1zIsaRgJIxNIiTfnDYUcp3CnPmdoaMclvk38Dccr5/C/3Vl2x3S9OlRlxVQ7uLNZrmdDmR3abh0mtzsQGxrA9Pa7smypQNvRH+VbnlBO/uF8Bq4ZiMFwfyFaXaUm4aMEgmYGETg1EJWdCnc7sVNEGObA14qvUWqUlOTHMn5I/6SDIX4kOZbBdJm2ohGMSCTmzPxFTJCp1RBa9Sn5bm9h3ilF9o+4xnjbozfVueGdnQS9TcBdaqTix4tJyE/AKDOyffl2pgT8e+fvP81lgAuzt8ympbAFfa++L/HYtz3nHC4FVyiNno19kD21KbX8MvAXzBzMiPhtJsH5weSG5lE0cCGzg1zorG0m7Y807ALtEKz1WHRY0KPqwSg1YjQaufb5NS5vvEz4wnCOtQ5DYtAj08txr3bH2KYlcHog249Z0YU5jVZi8e3EsC/w8hrNAkfLe2hLjDR5xqB29O/XkSgIAmUXygC4MPYC2RHZ+Cab9HWLuMa6UttWhqb7HRKjtEQ75yFTQfqWdACR+mZXKlmxnRR0KHCrtKSr567z5xzlzIlFkchUKuq/GoKEfHxeH8HkryYTNDOIg0fS8S31pTSwhmMN06hpN7JIr8fvoRgKS/4G4KonFAib2WYF4ZqxGHSGPs0tRW8H9jWZ9JbbAP3pBgVzKRZetSyckMrfvuLYt78dAJrEhGPU6XGsaUMf2IRG0opBa+gr9N2xy9PyORNrzSWtSEcUuSQSKw8rMrdnknZqKPFxBThuncaoyBmkX0unuaCZlM0paC8XUF0oYPOEksP5z6HmbtJLEMBTLr6Yljo7Lvw5Eq8ZLYANX7h9gWAQcIpwYsInE/qohOJfjSdqWRRyUznqAD3H7XcC8Jbbk/eN0a5LeWTXRuATk0aDYOTWL7fI3Z+LlacVpkVdvPfee1wadYkL4y4gYCTvQB6HVx5myfElYGdLu6M3xb7JmHXJkJspWLBnQV9x+6m/vsXWqob1r8ZTP8KHBDsxSBzx+gjOlpTx4tcv8vLyS0R4TWJqRRB2yrvJkzsUk/dqetz7TLQKHXq5HqMgru9ShZT2ynY66zsxKE34Ve9E+dmhlCTY0P54K4FTAzFzMMPKw4rKuR6kjj+Ax4F1PFYbQvPyZm7+eBO/iX7k7c7AmBhMaY+Uh9ccYk3DEAKmBPB2r6jHYroymZd3vcyVOXnIrX5DGC3O6ZTNKWT+ncnWVxMpNTuBX9tIJmwOZUyPPfwgAmMa8xqxibYhaHjQA4OeO9+i8KxgOoUSYBhOUU6UnC2hPqMe7zwb7GqcODfhHJpOCbrbSUKA2SeWEX8hgkQ7M/Yfn0zFWC3hP4VzeOVhzr91HmG2keWWMN8JskwEagWB619e5/S604x6ZxQ+2qn85rwP0BO64Bo/NI7GaAS9QvxgmrfXoEs5QpgmlMknJqFbVQ148sfIPzB3NsfE3ITWklYmfDrhvmKfdvw0qhXFBCX+RUu+M+DFiNdH0N3UjQQpQy9b0eHgR86IcQwfaYONj7lIR28QsD7Xy4r0lcx9ej9ldR9gY+ZCpcsx0n0OMMjwFD5ZXtTYN1MfWNQ3NhbsWUDeoTx2L9yDoDXikB7MavdB5HW085a+g9Yr2XjmpiGPM0NqsEAi6WTwpJv8J+ImKq/3MQT7UTpkKc4l16kJGkuX7f0di1YBTvz2xG8AuDebodeLtE4j3xTRwk6n9egvvonfpCR0zuI6HvpQKJVXK9H36qmIruKRsDSSEsOp7fFFm1+KQ1UaAI2pNZxPncBf8Zb8MukEwXrx+3VHT9p2+UxuNMcSr/qKYy7dfFMngj6sGgpxKbnBwBNPcGbCGYIWV5HgAf9p6BDPmVhLzZW74LM769GI10cw4nVRcy3n06956FAIpT6Lkc7vhqUim8drra8hV8mJXTgVc4syfnz6R8aYvkPmjkwOrTiE92hvSs6UoJfBmXGbb49nI9m7RMrAYeuGcXrLDbFrtdqN2YdmI5ndwokzJwhfFI6Njw0/nt1CwtBTDBWUqI0AAsZeLeNTblLjcI5PxjyMtcxIe7MVP131JXhoD+rqu5qSd7oX7hSSYp+I5cbXN2jKb8JUo8Sjyo3ZB2dTN6OeiMUR5O7LReUgrm8GuYEBaQMYf348HZObiHsqjpC5IZx74xxOlRKmrsykPkHBTud6tHrwzDmDSa8aWWQQXZY99AxNoWtLNMbSu3q4HbUdXJzwN0/zNM2zz2C+8CqbtTFskOswHDvPkJu3SHv6Iv5eF+nR1nJh9B5SYux52+1tqkPGY9rRiFxqRF2p7tOqutcaTRrRuBcjxCeyQ9DyrvAufhP8yN2fi0YtAj4GJQ8iOe4nCo0TkSnmEbYgjKrEKgxaA9nBN9AaK3li6zUkbXpgMDe+vkFzQTPCAjuCbzeDeFnnIugEWopaKTxeSMCUAKQeHtgOuMnR3smUxWTQaGrfl4Bu8orF3rWZL7vWogy5gPv88+xz9kN/m63Eur6ApFWX0czxZeDqgdh42/TdU+i8UBqKOijutODVz16l1rsWz2c8+9hAtBqB5/eP43JhCRf3jkM24Dq4hGLWXotUMGIb5kKy33SC2qeDBfj5ga1vD3O3zeXAsgPi3PSv5Lh9FWeHr+Vt85mM3TgWcydzDDoDzWubGRBXwNSQDZz4xp9rNlG43faLukwLsHM+xEKbIn5IDiMmyw9V0TWGPrqQ4a8PR2mlpPB4IUpLJXYBdpiYm+AY5tgnI+DZvJz24OVY3Y5zvEd5M/jZwei6dDTmNPLLQhfGLjjHqNCT4DaFq59cFefkbVaEqsgsVrt3s71LXKdPPHui77m51rii9RKDaQ8rD5oLmjG1NSV8UTil6no2SF6j1q0W+yZ7bNVi4vX0y6e5tekWantNn1Z8XHIcLdKWfsU+j6EeeAztX4i7Y4IgoHbwo2DwEuI97i8Gdv72C57yFjKHeCNtehxwwspdjAsOLDuAR043KbGrCK8YhXCPdFJHTQeCuYTNK94lLCcMv9ZFCAI05TdRfLqYNelr+CniJ8ZcGkOjYyML9g5Cv7QE44gwZDoNWlNL9s/dT6lvKZFFA1FoZhAA/YrMyq4WnEoTqbgYQsQsX449fYycvTmsTl3NZcOnFLifYkDpVixxoz6zhgvvXABgxs8zCDAR+MVWTmujDR/1+tB2qxCP4mtkuIXj+oSCvFFXMG1wwfRSDL3LOjFYC2KhTwLYWJPZMYxlo9dx1d2IzvRxHj76MD3NPRxcfpArIzNIHJaDRNVOx7gMvrL/qO+a67xMmBF0BXf/GjYWp9DsYM4XzhbIpAIe2aeR6rWMfHvkA+m0AdQOvnRbu1AdMgGDAUztTBm0dhCewzzJqW+h19jG+X1jOP33ZGaP7SZ3Xy4dNR1UJ1VjVmZgSNkQNFZKTNobsHCxIGhmEAVHCvhz5GbkM2bh6N5EsRZOmP4CvMLq1NX8Pe1vCs8Vo/92L9ZmUqZWw2l9Mh8IoHdwpcU1jIKAq0wPeJP1AfvY880C8nVRNI9v5tBjouyCTCnj0XOP9sVRp9edJuXXFGb/MZsBKwaQ45uDb7EvNSmi7x6xOIKL714k8+9M6kzqmHxqMpdGX0JnFH0mx1BHtLfrY9nR+dQ4id1eBqMBW1/bPoDZ9aTrbHxrIw1BDSRPScZV5cqpdafQqDVM+nwSbaVt/DX5L2KfjGXcB+Me+MxB1Ha9Mz/vmEFroDGnEXMn8wd29xm0Bq59cY3BzwzuA57cMUGAgSfEcdHiNREejb9vu9rRH5leS9zRDdT7DsE4927sWp1UTcZfGWjUGhYfWnzfua0aRYBl6Bcrkcqkfccsj5iKZUsF9tWZBCdu4yc3mP7TdOLWxKHpuA0yDLC7j2npjjVn1xJ86yo6uY4OSwNGvZHgWcEEzwrm6qdXOfvaWWLSYohJi8FoYeQO0Y5DiIOo27olDYua5WI+7B9FeqNgRKNoQG3VTUmIA1G+YjwRMjcEGx8bzicoaO5VIXHzQF9n0afZ12XrQVXIeOyqs9CqrPqpEQgCIJHQ7B6Jm4+C5sJmbHxsMLM3Y9fcXX3Fez9Xe67HZHLJ8hJPBz5NXWEH1y8bMHcOYc/8PZiOXYWzXgRTerpAbpaBb3a5EODkTeGmCxTixRPvn+KXyETmuLzIrf1lnHzmKPq6ZjTOCpY99BchVZ8Q4BJMZ10LF96+0HeNj7KEgs6jxM9PJv7Ln3iB79GoNbSWtiLtNDDXAva4fs/5Id4cN6nHxSMcyjqxrS6j28YN1ZyJxDz+wNcFwM8DfiZwaiCLDy3GaDDSVtYmgphdVDz65lZsHFv5ulzsEj3z6hnSfk/Dd5wvA887ImyTMGHxGf6sCKIurY68g3nM2zGPG9/dpPh4Aerzw1CHVKDuysJ71iSAvhzD9MeOEzhA9Ot1UpFhZMRrIxj20jDefMtASu1eXGQw/fB0/CpEhpX5u+bjNcqLE2tPUJ1UTWtJ633FvtaSVr71/xallZJlZ5fhPujBjCQAh7ZvZ0VvLy/x4ELfHfsasbtvptFIrlkgekMXCoU185Yt4a93n/i/sqPvXntQzu//lP1Xs++/9r9mjuGOdJZ38q3Pt0w8OxGj1MijW2fTkZxP3sE8MrZl4JJ3mXYHC/j4Mz6efQu39mmYylScfO4k9en1GA1GHAxd7LVSsWPEL2gsxEAwZ08OVTdEaou9889Q8tBR1M1WROa9gyDAa62v8diVx1C4mnNh3AUuODZzQfUGJYMWYuZsiVucGyFzQrCokqPqFgPhh468z+J9YpvPHc3AO3ZH/+Jeu4MI3rpiK5nPik5U2YUy1FVqCk8Uovj2C8zVErY/sp0Tc8ox6AyceukUmdszEQR4KWsAb1aFYjF8GQVmpxmwYgAWLha0Frdi7mlHTeBItEoJ699fT/BJK6IeiWLejnmYO5ozaJ8Lb7//Ds06U6q7TcEgJjT8J/pz/cvr+Ja54lHlQb1zPe+/8z7d06wwsTBhzp9z+t2Dwvx+GsX3L73PuMRx/DnrTzxHeGLUGzm6+ig7Zu3AaDAiN5qh1Pvy2aufUTS3i6hHopj05aR+FHY2rSJiRxAgYHIA8a/EozBXIAmORB0UilAcz7DEBRjuSeSBWPyzkIBNrQMNXz2GPlHUyTn8xGEalt3kkb8eIabDlh4BGg1gEP4FOvsAU1W9i1duGkWuH2M0gscQD1anriZ8QTiGmi74dgWOKdHIMKDRaXGJdmHoC0Ox8bGh1WiC0+28ZbrLu4CY9I9+NBqlpZKKyFn4zygkrS6akbnWfcnb2pRaDHmtLNyzEJdqcezeCcQkEglViVWE33SkwksMlPOD89H59XcgnSOdiXok6oEUiYJEgkQw4FyWxNVPRK2w6EejWX5hOdZe1jhd3M9bH7yFaY8pGLrpbe8l6bskKq9WYjQKrLeuYoRJBakDUul2M2JiYYK5kzkSqYSagS18/PrHVLgVc/aqF1VFGX2JV894T9wKVAy7MJZwux7Gzj+PJkgcS4JR4CPzj/BP3k2HudgmqVZl9AtA79B8VFyteOC78qhQMeHsBGybjeg1Ysfg7N9nM+qdUQgCHJmTxKXRl4jLfB5dtxavEV70NPdQdKIIkyIdk09PxrbVFkEiXu+BZQfY9ZBIf2TdnMI42beMnX8em3n9BapbW1qpPWzCR8diOHF8Ni5bF/UrdqRfaUCiyaNggFiAll2664wCHF11lOufXydjW39djjv2ledXJP+UTE2yGKypq9QUHCugs76TZlktbbZtLNq1iIb30+5/19Y2JAx8ifDLn9Kz83u0XVoqrlbQXtmOQW8kLmUUw5PX8eLXL+J58B/UtuUaRl0ahapDze+P/0zhpP4RRI4yhzrXOhRaBUN7h2Jo+f/OaUn9PZU9C/bQ09Jz3zajqTkddl60OQdjamNKb7v4PLubulG6WfD++vfZO/8Aba6hKO3MqU0ReWHdBrohvVTPy1+8zMQzE1n//no6L1dx5hUR/DDg8QH4nNmMvPl7Rjh9StL01fjOXIqdvx1qt1C6LDrQKsQEcbNDM5nh55CpTDA2NBF8bQtxxzYSnrCJLhv3voTwRpONvC99n9x9uaQNyODy6LsaUg4hDqwX1rPi4goMHVrm7Z9HeJYvtTb7EARYuG8hz+Q9Q+n5UuSd4GHRy6AbQ3l022KMAqRNXEf2qNWcfvk003dWYKMex8lxezk7+gROEU4MfWEodv52qCId+PDtDxk78SZBDQsZcPZ7yrOyMRhAZjTHWw5aAZwk9Tw3NZVvX/gR11hXVOp6zFurCb7xJ522HjgsGNPvPRi0Bgzba5lcFELrgGy2Dk4SkY8xc8kdvhJBp0f9y2YGJ7SzcPdCRhzxQm4qZ/mF5Tyd8zTPFj3L+Zd+osUA+XVTsJS40JTXRMHRAlxiXJCFWKCtd6Ak05/CtECsmsxZsGcBEYsjcIpworXZkeLiUL6omcrnV7M5e0ukNr76TTLfBP2I/IV3ObN9Igltr2BTlo2++q5mhMcwDxYfWkzgtLuAG32PnpJzJbQWt/bTehQEMbjev3Q/f035C4PWgPy1Hxk55zLaTYsxSepCKpfiHO3M0pNL0UikZIdlIzVKibsZR1ejmgvvivPaLsAOeWwUuUOnc3TKt+ye9xUopYTND+vrDjFvboVG0dV2tijjUutfNOY0cmTVEYROHdeGXaPSWsNj0h+ZX30V2u5+CK09rVkvrO8rsv3Tfl+5Cb1cj1miqJli62vLupp1jHhtBMbqTi6uf5qmQi/K/TTITBUkfpPIlY+vYOZgRrzqVc5FJTO8VYN9WSW6Lh3VidVkbc8i6TNxXC96cRchcXk0e/bXzvWwGEiYRzO7A9KZ77cTE4NI7TX81eE8cuIR9DLx+7DIsp6QcTfJHSrO2T9G/sHRJ4/SWdJJS2ELHbX9jwtgqFMz9vxY/n7kLxxenC4+t0hnnCKdQAIJs0r56amfsFRbYtIjwy7AjvXCetYL63GMdeVTpTluCoGH12+maYiItB31ziiil0ejLNHy8NlJDHPo5rGPf6RnoJKED8V10SHUgbhznmz/TBSPH2oKMvTounWonQL6rq/Acgcz3TagM/PB3PSuNnRXfRetJa0Mfm7wA5OMu5Tj2TUxmKOLHXGbGUJnXSfNBc19xYuCocspiRxGi6MEwUqKYBRYeWMlc/6cg/0+LVVFnui1CiIkrphITZE3fUZIRgslX51l3Mlwlv69lI3tnjySPYDe1m5S/0jl1AunxELfE3t4eeItdjYdparlHO0tTdjOHkXu8CcYd9qTpz7dyFXEIpqrXExeCwJ02PugdvDFqqn4geOv33uTdmEwiPpjCR8lUHSqCK0NWAaWM3hiEvYLxfnsM9qHVcmryDuQR9iVAEYObmTEQ5YYh7yCaYgP5eFT0cvvJvUSnWpYsHsBodfsKDl3V+Ox82Iypr3NqIQuvDqscWgVs5aOFal9lGjjzo/DX+VN/hcrGfThUuyD7Emb/JrYMWauZE36mn6UeHes1U/LjSE38C3zRZIhJu+KThZRc6sGuVJO1GVTph9fQJttG1oLQaSsnOxPyRnxvHvn78S11pW3N76Nxbkuxn80nscSHkPfo0dR1MPSv5biXO+MzCBD0BhJ/zMdhZmCCf+ZQPPcBN4MbSJTMpQPv11LSHIAmpMXiDv1I5YdlhzuhDOdFjx2cxCBu+Yi5KtxCndiwd4FSORS/FL3E3d0A61Foq6TVC7tQ+H/uWInZyaeISYthpYysUC4YM8CZvw0A12vHs8KT1ptW7kw5hJKBwuCZwbT29qLulL8XzOdK73DUti6YitYmWDSq6bZPRKVsyV2Hab87iSh9qFuMpctojG3kQ2SDXzl8RUOo7xot2rnZ+s6lDnmXLR7nuauBvDypM47EMH+Kgme8Ir5zzQ7NFPmU95Hc95r6QgGI5c2XOLWr7fue1fy0Ra8+t5mvpmWSXdoARKJBF23juLTxXiN8MLzzaVcnuLOzCMzCd0kMroUny4m6dskDDoDAgJSoxTbxiZ6WvIpTyjn1IunSP4pme6sVgrXvcr1E0M5kRjM/ryX2bNwDyefP0nKbyloIyI44fUGldIvyHP5CoX/xwgCBCZtJ/TKZkyT85BcryBR6cvUyMuku1r1+ZPmbdV05FZz5eMrfbHqHQuYEkDcZwupC5hIi0sotn79OXeNRgFFiTsttfYMWX4DefwgtAFhdDjcBovWt5BjH0ua54zb/y9q8ul6dMS/Eo/U3ozBT2TSrBR9LZlKgeeUME48f7Kvw3T8wvMATHUoprjrGu2JeYRc+Q27RgM53rDEOpunJ17ig7c/QDrMDLsAOyZ8PAFrT2suvH2hryMdIHJpZJ8uVm/3mzRJnqZdJiYwbXxsRC2vdy9g4WzBwKfsSSv04B2/y9Sm1LJg9wIW7l8oFiLlRqwf28cLJioi2sXOvX/a1uVbGXVpFO/mvUtzfjO9rb14jfTCebU/tW7iOjTl5BQmfi92tHmP8mbg6oHoVBCbIlK4fffsd/h+0b8jp+hUEVc/u/pAfUVtSxeRF7/Hpr4QVPfHYaWD2rk18BYjT3WiShCfb3tlOx01HbjEulAa0EiV5StEXtqN+nJa335fun9J2orNGGVGsiKz6LCVIwhikeLkcyf5KeInjFbWVPjZUu9cz7URRUidndA0dxJz6hM8cs+SEZ1Bh1UHmXHZVIZPQRCg89glYo9/SMT574i88B3OZUnsm/0nIHbRdzd2093UjUuBNUNuDCEwox2huAS5lTnN7pGURs/CaBSYufkxdnzxMD++upbgJB9aXcPptnTCoqUSmbcJu8ed5YvxSaSMnYGFsyOmNqY8mfwkb/e+zZEnr3PYeSJOxRpe1l1FI9HiFOHUB9ZyrjOlU5XX10VvNAqorpwl5sRHpA6x5cMJx/m71YNSv9GcCRxNt64b0+ZqXEpvYF+TxbCXhv2rZlrh0GViR6NEgsEADdkNSOVSnCKd0PjDFy9uZOtDe7kw5gKYSJn4mThW1NVqbPIMTD05lfHHLfC5sgkrDyumfntXV7u9yYaRh8+xYeMyxvw9DnW1mtwDuXgM9aB3jjWPVigIKo6k7FYI486NRtutozt0ICWx8xiVfA2v3WIRuL7JA2mHI46hjhQOephGzxjMvBz4Y8Qf5B8R2XzuyFzcAXsYHY28/877mDwjIhQsXCx4aPtDhC8KR54uZ9iNYYRnhyPLktFZ38nJF05iXSce69TMi5T4lbB4x2LazrchGAWMeqNYyC5TM+TsELyzvTk66yg5f+ZQda2K8kvlHHniCJtiNtHV0NXnS/3T1gvrGfLCEPYt2kfqe6n9tnXUdrApZhObYjc9cN/Ka5Wcf+s8WbuyHri9/TaNp8LJ9r5tHWXNmPS0Y10v3uMdbfU7NuXrKThFOtFS1PLAY5dFzyJt4josQz366BAFARp9BlMSOx+d8m7X4x16W/fB7oz7aByPnnsUtzi3Bx7XzEnMF6bEpvDHa39h6WZJR20H+x/Zj7W3Nc3PBtBu1U5WeBatK1r79lOoFDiEiv6ib5kvbtVutO0/j6FRvH6Vup7yC2nopLW02LeQOdgOqwgRZOU71pehLwxFEhVJUdxCkMmxq85EaGxCEKDb2pW6gBHkjF6DIJXRsuNU3/3e+UGm16B0tOLXuF9J3yqCV+/E4wByrYxx58dxYdwFHEMdaS1rxyPvLFKDluClAyl02UGJ49egV2PQ6Dm6Yh+RF74lZ/AKJLfzWj/mO1BvEEF/R+ZtRV/XjGBujrJdhneZN13KArpMSjC1M++bl33vs/AuQNkoGCm/XM6mAZuILAhiz+0m33FB5bw6NwO/wDBoaMA/dR++aQcf+J7u2JbRWzDqjIQvFv36mps1fBfwHX9P/RuJqZTdrqX89M4qpu3zuf0gxHl5R9fw/RXvcsG+joqABtTVajK2ZdBZ18lDex7G8bUtPPbMfi6/9SyRp1R4DPVgbcFa1qSvQf6UF60u4jE++GMCk46KSJA7Pp1eMMXbOosQ62KMUiMG6d3YcfAzg5m7bS5jN469TyoBRJCbhYsFGrXmgdrp91q7Wk0yInXnvxX67tg0YARg6/Eo8ROrGTQmh8cef/v/+kLf/7b9t7Pvv/a/ZmUXy6g5LSa1263aabZv5vTE0zzp9ySmtlr8JvhxWjMKhaGWFeZmdFyK5NGDUzFbqiJ2VSxV16uw9bXF/qQjn115jbqXfyTdYzQ1XX8xcPVACo+JwtP1bk0kWzew45vV6Oq0GH8WaMysv63/Iq5knaj5yvw92kwlvG7IxHOYJyp7Ff6/qXjt2Gu89957eFW4Y9qeC8xm1FujGPnmSDL+yqDiSgX2wfb33Z+uVM20Y9NIG5CGwVFMfuUfyidrZ5ZYMOnswjs3i0x/UwTEBOSNr0QqlRfaIqk/NoqbHo20z6tAphUDjGdyn0FmIuPUrTqS4qtoMLuAXY09eldXOus62ffwPiZ+PpE2l170uhqeM5+M9E0PBL8A+FC8rujl0RxQH+DMxDMIUkHU67uN4vEa7sW62nWUXy7HdaBrP7TpHVP9reLJK0/SYtdCsjaZQY8NwsLFgs66TpRWSvInPEO2o4hAvYPST/o2qU/TZO8jauybowm88RvqsRMwG2RG9q5sXAa40G5s4/WBL2P48UWarlrD0/cmaQXGXByLl2c9voMraZxzHNfQ59Fr9H0aiN4KbzbYmFBpdODZQlv0vfcHg/9mUZmDsFIruam/W+DJ2pmFTCnDoDFgqHXEbnoC+6JusM77E9or2jFoDdgF2OFSaEFx5UCOtakYkepH16outo7eSui8UMZsGIva0Z+wqHwSPh+Jh0GJxz4P3ux6E6lcyiebzrJn/h5q3dvwuPU7hhYZsJy28jaa8ppo9benyUEsSuxdsJenXeb0u+7yy+UYtIYHLujmbjZkjVmLa+FlPOLFzqOjq4/SmN3IK42v0GvnQIu+jtc/eZ0a36tYey5i2o/TaC5oxqAzINdZ0GHZwaE5h5hlHYX3KG/2LtpL+tZ0hGHwsquG/EIn3P+Yh2JJDwOfGYjveF/kSjlZo1pJGrmNUmMrqdHJjHQXO32QgFwlR9nditZ4lWnHphGXHEfP/C7wFp1niVSC/2T/+7rA7phvgQRVr4pNa9P48PYYPbDsAIHTA5n0qw8gEJMaw5DziXSsjEYY4MjNH26iadfgM2o8W/U/otTaEpwxGL3WQMZfYvGt6kYViqQr7Mt6iHe2beBKTSsP3VP3Ulgq2Ll4B0cj2ilrGsLFrrUo70HvVaRmEJiewoE5x7Do+gTHoP40MmM/GMuVj6/gP9mfFu4PMAY8NgCj3kj8OhGhePyZ4+QfzidySST6MfbYtNqQHJfM7KD7eeWvGDfQanaaOhcLbGzdaStt448RfyBTyniq7DV80w4i9YNrw65hHxTab19puY5xF8bR4NRAXmgemcYz3KvPpNAqMNGY4FzvzC+xvzDpi0kMe2nYA9/NP01dpaa3pZfhrw9/YPDdHDiUGq+hfb/7jvXtQ5SevSkWgwWpjhTfJZSo1xMW4cSAxwZg7W2NnW8YaaGNVLtXkxWexQSHUQx6ZhCtJa0IRoEqjwbSQrYyXhqHmTYWk9vfuc5OECT9C5Z9FDW9Gszaqh94L0ad6ExHLomksHAQoy41MSRxKMVD2wCbu8eyULBr4S5ca11xz9mPrmsGtgNsqUuro620DYCbh0ZjGp/C8amnmWn2JnqlAtPOZsoulyEHPrd5mj2hWfzeLCLymvKbaMpronjlccaGj+XRFZfptddy0KYTQSLhyvN7GVEoo+hJsJCCBTWssILn6oxoO7SEX/6ZXjM7THo7MOntQPIPoWh9rx5hXyMuI4IIHJlHo1F8Hj1WopPe9dsG5EgoCCzArNsME0GJVCbFZ4xP3zHGbV3M0VOtVHQ5Yj0wlYyOVhI+TMAxzBHT2S60F+WyrdeMnoVLiBoZgnMUzNsh6tAev/YVt8bsxkkWwOIb8XTaie/n/AfXEZrEuXL11kTCZh5h+JcZ7O6VAMO4vFEsTF3/6jqZf2WyYM8C/Cb4UZdex9X/XMV7lDdhF30ZcvA9MiIzEGaq0Wv0ZP6dCYhFzu/kXYwSBGozA5H66ol5PIaYx8XAqrdSyZ6Fexh2bRgzjs2ga1E7A1cNpDGnEftAe5L0X3Bq4csc64IZNaAz6OmoFYvalq6WlE5fS0cHhNy6gEVBHcOje2j3aifl1xSkwOn3ThOtiOT8pcVUnjUncE0n8ABNxX/YnTUqIisCtYOo7Zm9J5uDjx5k8aHFCDYKro69QLVvKV2m0/iPowXmLubY+opJEaGhjssVM7CYas/l8DdxGSDl4SMPc+OrG6Lm5dizrDcRgRSDpaGk/pHK4ccPEzI3BNsZi3hsyB/itsu/k58YDEzGOcoZ+2B7Vk6Nx8EtkGc++wGmwc/5Iro4aGYQHTUdJDyWQAIJDHpmUJ/Ga999NXYw+vJo2mzauNi1jRU8S+QjkUQuiURmIqPHXEeXpIu3P3ybFo+7YKSelh7y/kqneeNaTsfk8/DLO7CtkZK+LZ369Hpin4hFceQEN07E86VbEac7m3nCVWDBngUYDUb8J/ojfTePJrUD55qD2P3xGKxDVKgjOnDPv0CPhT2qzmZqXWuZFjOck6UOhPvTDwGNBKZ8NeWBOhn+aXa4awdzPf4Z0tvDsG734+jqo0z5Zgpucx1pcjEnMXAsvfJChtf9ieQte67+5yrP5D6DfqwTwW57SL8ShcnRBHpGxzDh7AQU+rv3P3bBOdpSYzBcjKVrQRuXHj8MgH20Ow7acXg43iIkWcu0ozPoDmtA5uFLl60ttrcTYY2Gh0gscsM0ywPNgG5M/t7CwKq7uteCRIZI0nPXOspbGHl5JKG5oZjorNFvhKor5Zx/6zzRj0bTFAeSUX9jZQNn6iQsBr5w/QKvkV5UJIhAnt9TP2Fd/AqyWmehcJqJUa5Artf0ncMlPp3wI7NoFNr7NGwBdOW1RJR/jzbsAB/tn49tmwN8AdVBY2jwGYy/wwqmO3RRIX+TJH08Dl539R2rQ8az6JfxOEfdvZcjq46IXRJbZtPjLiV20WlKxtxEYfE5AAeWHkCv0fNswbO0RY/kltX3WLVbITPR4zPGh+tfXO87lqpHRa9pLzUeRryd/foKup11IjLbKJVQ41bD989+z2s+u3njhxVI5eL3OMRo5HkrKUdVsdzKFOhyckA2yZPqch1GuZyTHWassjIyOMjI3tlZPBw1GoWZAt9xvgi3u8Y0ZjZI5GKxzNbflinfTsEh2AGl0Y6MqAya3AJ4arg/zQXNVN2owm+CH721Xaz8fSVXhl/h3PiLmDpYYtTrOf/WeSRSCYJR4MArj2HzymagAT0S8oatwKAwBZJRqs3ZumkWN+fu4pjkCN903n1XzlP92Rj2Bg3OTYxLGsrIExNp965DMmAoN1Ul+JiLfnqN0R0oQsBId3M3QTf20uoSikm3+O7CF9xfRBcEgUNd8IQ1+CjEj6K6Ws2IN0YgkUjoSi6hzdGS/OB8XNolGLQGBIPAhE8noFApWPrHVLpN1Xzw9gZG6t9n3i0RMPLIiUc4X1lEmWM9SWoFbX89jM2EduLWxHHh3QuEzQvj7TNrSYr9DYwfYqF5EzP93W4SjZktEgs5LdZDcDF1JLRwLgMj/RAEcCu4iEyvIXrriwRbVN+n83lszTHyjuVz5LVNeIY8xpeTFmI0GPkl9hfqM+pZcf1Jrs5czvJpcRzq6eRI2+fE1n+L1lRMIhuatTy6dTbl3g3Yt2+mfcJkyvck0lrcyrradRT5TmTLtUn01rzE25fi6YxpY+NbnXj33qbaN5GR0TiWLXoViUXpBHjIMGq0mPS2IzXcdYglSPBXgPE2Uk/XrcNlgAtucW59TCO6Hh1yUzmusaIvHHtdwKzHmfIpmYAIohj93miMeiP55PNs8BeYNZkR2yFDopLgHSvGLaFzQ1n2wRy2OQt88u1EwhMHEFYaJtJWN3eT/FMyeV1ih8mUuilUlVYx4/0ZrM1fi6WbJbssd/Ee72F4ZROLp6fwaZUI2I1cIq4t1+b6c7NVLFB2WnbSbuzvH+17eB+9rb24DXS7DyAgUchpdo+iy9b9gdSsxfFtpBlvAjBYEAEsW0ZtQWWnYtWtVXz9xx8MvTEU0x5Dn4YWwLQfpnHu4+ss2L2Ay6MuY5Ro+gCyfWYwkBHvQ5NjEzeHljDe0RGpsodGzxi6bO/vRBQEkNjbonbwo8UtDNeiq6g6G3EdIrZaPnLyEXrbenEKdyIoyQWPgjCgBbXiPDqzJyiNEaUT9AYj5u1WyKwb0cU6YhbihaHWFLOOBlEOwyjwkT28FljAk002yJUm7Fm4B4PWgP9kf2Z+GwaE0TkkhY5yV/Tz9Zz74BJZO7PwmOQNlzQcqolAKYekC2MwPNGBwcqGTjsvrFpb8To1nqkzU3BpMWPvFU/UM9pwSRfZljrsvNk+fTvBs4MZ8dqI+56BxGjAvLUSndISg0FkE0n8OpHwBeEIgoCpBNyiCzGzWomphTnBD0cQ8XAEEomE+usSTvr+jmOjI+Y9ol8jV8mRmcjwGO4NF/5E15GF1kSGFDnFp4o5vFJci1XAW7xF8uPn8S+zxjZxIPoePYKgwLnkBpaFHjRaiAClU2FfMmgQqOyg3TmIducg4gYU45mfgp2/WFgavX40kUsisfWzZedfO3F70o3IKZEIk8U5euHdC+h79UQ+HEn3w9386PsjT/30FMYSI50zOkn8JpFAEkmesR4QUPWoCCwMRFOt4eKGi1x+/zJTv5uK9WRrfnj6BxxsHXjj4TdwjXHt009N/T0VuUqc326DHlzcAlFzM/HrRJqSm/r9/U5M2FXf9cD9FOYKrNyt+tbHe81ohMLBjxB37H2qvjsAn/WPa/XdWszbqum096bD3od636HE3BPPS+VSnsp46l+v2ShXYpQqMGp0CIIciURkBrNsLMGgMKU6ZDz1fkP5/mc57reLVX+OE4vmclM5w168P0bW9+q5+f5JalxruDT6EujEdbo+o/420F+ge4iU3176CoDl0SLdb2EhVJUbGLIyhjWV+8ly+JLYW7GojyZg5e0N2OFUmkj2q6mYvmxKl0UX1l0D+527u6kb3fuf4+kdh9w6Er/U/QhOkxEE0T8xb63CtiYb27o89K5B4E5f559Ur8W94CINBeK4u9PlvOLyCr7x+QYbHxsoa+t3PttgJ4riFiEx6Al/0ZObR2Zi32TPwpPtVJoMQBnkRfetXGKPf4B8RzBOpY9TVe5M4IWRaAf1UhU2CQEJRktrfBJ3s+z0VLRWHWjrfsLE6jPiX47HwsWir1t97ILz9HYrWXIoGu8YHfbL7Bm7cSyF+JDVMJIwx2v8nfclN8tG8eLoMM47iv6YRWslrc9vYMPz98slALgNcsPG14bIhyMBUToCwMrTig5B4PVmmONfgM5U/B5cbwxArsjCfOF0jFXtOBRsZqmNkV+6NQRMCWDF5RUUnyqmpbCZfLs2Em3aSBxxFaWNGHPaB4r5ZP1PFVxnJR+s/Y3QIg88WkSWtTtAkTpjJo/efJ0mo46kaetw1NXzAyKwR9OhIWppFP9mlm6WPFv4LHXpdQ/M895r1lZWXK2p4f4n82BbBayr2Q5BIkj1QRTR/7X/s/bfYt9/7X/NdJ06QheGMu/vebxu8zpjL4zlwEMHmOpUheeIwdg850PbL+DY4kDDtVE0VtoiCOJiO3OTSE9XdLIImyJTMuOSmX9pDC43w+hYocbU1hRLd0sEQUBukPCqpRQGl5Oc+ggGnZHNgzfjGO6I+4sTWfvT54yac4UBVtm8mOBH96JO8LAienk0l08mU+wgJgVzh87D2+UuJZ9BYyBsfhjRy6IfdHvYt9kx+OZgnFpiiLIXk/NxT8eReyC3b7GIyI5AaxmNjXYOcqWciIcjaMwRg5Btj+zERW3B+I9eonpkG9VJ1VRcrSBySSS5bbdI8xWdgL0LIE72OGYOZgx+djDug9zJ72gm1HIX77vCoRGzyVLGAWJnSuhDoRyeLTqfCq0C11pXJCot3c3dnH/7PP4T/f+V/gJA0aTAocYB9xp3TjxxgsErB7Nw30JaS1uRyCWU2P1Ehf2vhGeFY+kso7mguZ9mgm/rp9SYTEbRpaGsyYmRylgsXCyQq+RUHNnDV4VraJtaRYlmFdPt70l6CjDm0ih6vTz4LXYDVwYnMd9TibZDLIR6zh3IyuiZuNmC/5HJzD85DPnq//knbvcjhSz8y4vAbBlGoygcv+/hfciUMpQ74nD89DPKjFDdLNJcfO39NQDzds4jNtWKk/nTiR17C9daV4wGI405jTTmNBK1LJpc909pUqXz0DMVTC8ZgEQi6UPl2bn6oHYeCmYpWNafw7lURKqlb03n4vqL6CNccK92p+g2Rdc/Nfu2jN4CwKqUVfdpFEilYDBRURU+Gd/b7BV+E/2w9bPFqDdyaPx31MmPMu34bMxkE5HKpVQnVpO+NZ1Br47jnU9fxi+kiLMr/qIFI3KVnKilUWJyoqOSx9SuZLl18PqinUwd8DzW3tb0tvXS3dxNt6WePGUDypJP8ap8hcm3te8kEglvdr3F+9INjLwUQn5QCvnB+X3JqTu29OTSf31XZybs5dowc+rsKvv+5j/JH5cBLggCDE4Mx7JdSmFUCJMcLMn4OwOHEAeGvzYcocAJF833eCXvxLWsDH2PngV7FnDt82vITGQkDajkrWHnqDcI7LXfyJdsuPs8FVLeGZOPvyWUS8xocY9Efg+Q1zLKhZ2LdmLZYUuHvTcOQf0LsGHzwgibF4YgCDj2OiL9R8Fl9u+z+/0eviic/MP5ovbCySxe+PMF/ljxB0yyue+ZBJ7QMD7jLT564yPGKIZi4SJ27xk0BiRyGakjoinw+ovcoNOMt/Hpt+/QMY/zrBBFk30lcp28nxg3wIjvRzChbgK/rPqFgrkFjAi+P4D+N2spbiHjrwymfje1b8zfaxIJOJTfwjP7JOWXl+I9yrtvm75Hh1+xH56Vngy94UWVdx6Tnp6G0lqJ0lKJa+RQLmUPRWcnIX14J0ujQ5n2kgspv6Wwfdp2rCzNqfYvZbb5L6Tr89HVzCLx2xJCDl0kbeSYvvOMvjiakQkjaR/aiMzLndRpom6aW/4FIs9/Q2vpU/1obZoLmrkZ8zuehnjqnRsRJFIyd2Syf8l+4p6Kg7ne5Ibl4lfiR2xKEIZePV0NejbFiOjVXkcw7zSn2KOS5KA2dN06fFMOY19zF72aeHEgH4xMp1PfQ8ZfGX0BDcDoy6N5P3o0Xwx7DyEsHVPft+ht6ULZI5CsgeGZo1g/6DG2Jr6Cf74Xve29VAeNQWNmi1FugkzXi19DG/cWKBXmCkI3P8KPZ1w58kcHK4r9Ed4Thdv9k3chsbGiS9LC1eFXKfcpx6tXLNKUXy6nIasBC1cLjB0OZLmYYF7WjNo5jdC3ppDwYQLtle0oUkyweukQds4tJBQdpFQ3mqobnrSVtxE2PwyZpJEIiwbi5GYMfO0PdrWHAdA+dRHlFeDmUsaUoF+JM0tBYdbF3/XN/cbRze/FZNod4fj8Q2IRw3OYJ96bI2nhElGZUfQO6kSulDPpi0mYOZihMFOgbbPA3KeNMT9v5E/Dt4Cod9Db3ouZty2O7ZNpdfEmf+ijzApwJWbZ3Q5Cs4J8TiVMwsJMw4JcJzrndLB5wI+AGDimObxOj00P5uoeZh6dSaesG9/1vviO8+3Tsr3kmUPWWDmblcsZ7XT3m9BZ18nP0T8TtjCMad/1L4oBDE4czKnJp+gIEJPF1l7W2PrZouvRIbExYfRDl4hTwqp0I4IADsEO5B/Mpy6tDuHMURJSB2C14RL5DubAi30UMw8dWYHJxc0019qz88vFeCj9aH1ERBjf6c594WQy742fxpnrg7DKvgFMBsTObYCmGjEwbTWA/jYd98OHH8agM7BtyjYqLlX06S7fa4K/Hd8/8z0WnRbIasUiwF+T/6I+vZ5Xm19leu3vVFVoqPNNp9fGE22XlmufXSNnT06f73QzLJuAyvEMUb5F7a0kMrdnMvGziUSPe4KDrQ0UGRMZnOiAfJAe31V3E7eNUyfQ2mFNWnsFVj0qDFo4cNqM1sjpqNprUHU202nRiVRyp9tA3G9t/lo66zpxGeDywEIfQEiqB1bN/hQFFKFp6cVxpCMWrhacfP4kS2ZFcWaAE95l3jy25U0MK5pxniSO//LL5TjbBHMx40M67S9j42OgR9bLsdlZzNkXA4MsyYw6iCL8OpmV9pR55jLN6ylMLEzQdmoxaPSk7Q0g0ew1jo4awB8WRzDxXIy2V49UbyR/6KNYdlfzqORdKnMCsNo/jp6FnRjNzLl3RbZqLOKfxT51aTPjz49HL5fQaW2HwUCftmjMEzGQmIFOgLM7x6MsdIKXwNTWlLyDecQ8EUPq5lTadrawuXwdTWotjgNAphf9Ob9lYTiOXMh6c+DPjSwumc6wR4cR+UgkJ5872dcNVWKuIzuiAqsuERzQa+VEL1A0Mol6mx70EitqgsdiHgi97b3Y1JbSa+lId7lAwkd5hMwJwTHMkdpbIkJd065BEAQ+dDaCczPft6wAIHpFNEnfJvG199c4ARZz3+a577yR+7jBf0SmkvbKdurT65l9eDZnJ1ylIfJDpowUiyBfeX2Fa6wrtjNHcLU1FJ35JaAeATCxEL9ZJ184ids3T7AyNJ95G2W4rZFzsXoGrlFWHLP4kXKnH/CXyPCW2vG63wXGKWfQ7DMGXbeO3rZebIeFkNrpT5N3HOa360dHVx1Faa1kdcpqZt58l2Z9F4LVAjQ6KQ0Xyzi6+iih80JxfSOUU5NOUe1ejSARvxfHnz2OwlzBkzefZNu7hbRcvEV6hj9jG8LRDNCgVVlh3lZNwfZkANIaLbGrdSGu1Qr3Qe6sF9bTlN/EDyE/ED4ynFH+F/kkppr9ncko7OYgNEO72S2atTCzBmJsH2fasSsMvjkY9UNqzNpqMErl9Cr1jPozFln4/bS/XSmtHNo/llu1rowq8cXwooHSc6UkfJiAmaMZ3Y3dKOeHcGXEFVwNYNCJ4K7o5dFIpBJabTvoNhW/bUYMxDweQ8jcECycLbC8oUaz6ixzbJr51lxNk/MQBq4ayMBVYhLV7ws5S8vnMnDxDpp7EskpmAaspi5gBG75F3HLvcSupfk83/IcU09VYKl1wXtMMO6FIovElv1TOHjwfvr93vZeJAZ4/qtlfPP8F9R1PIeLucNdPVSdEZU2kBaNLQ9bqdnR1oNU3Uq3ud3tTlxxXalxLcau2Yr69jJGPjuYfYv38Y3vN/TYuhHcdYOBg7v5zr+YIT16XIvErhyFlSlZoeZknJ5CbVwmM7Ysp2WIHpuXo8goiaLBdSOvNMIwZShmHd2caAtgV7OO7D3Z7F24F4Anbz7Z19WiUWs4uuoocU/H4TXciz0LdvHwjodxvpQMzCVrZxYnnz/JQ9sfol5Tj4+mDM1A+Cs8h+99v6ezrpPM7Zl4xnsiu+LKx8dmU+QmY/yzgSKjzkCxcKbv1VO6qJSQvBCCtwUzPXA6JpYm7Bi8ox+dXlWdLVnDcyjyv9tVru3UIsvPxXhbEzgu+2MiRouAu5pbNeycvZOQ2SGkbUl7IKW23MIUtYMvPumH0GabAYG0tEBWFsTHg/G2xufNwTcJbRI7bWJXxfZ13AZnu1AU0Mmmpw/zecibfccd9PQgCoulaL5sJTwnnDrPGwgLwmnoMif2+xWkrN1CR4w/WmMaAALivFVYqiiPntXvGgclxOCXtQfmL8BkYBRFzWIiuMVDzF9s3Sr+372FxLRx5VwcdJwu8y5CzVZgNIr+oFSvRRCsuPlYM1jVUJ4zngGhPuiu60if8BJ6EzPCMv6k+4elFM2+Tpj1JYx6XyIejaAxq5Gkb5MAcJ98ldYuJWiVdDd1k7VT9H+tg2ypOq2kPTMIR59ajMXeGLp09EbEUWYRR2TCB1i1j+REZDueNWa43Iqhp6WLlrDhmOenYFNfQNV1sA+6H4QN4JFzCueymzR4D8RgmEH4gnCyd2VTm1qLxtbIhJSB/D68HHXETgTJa5ScLUaj1hA2LwyNLVSYV1DhXYFK60lTXhM/hP4AQEe1mtyIXGJDKnhjbhILyt3wjP/0vvN/YNKO//JbvDf+GgrLdzDNT8e27Cb5wZWcmnSAtLIdVNvuxEqwQuyVEa1+5wUUPeq+wrnKVtUHFJCYSWhwasChyQF9lVi0r0io6GPbMVoYUVurOTLzCIuHLMY+yJ7A6YEk5Ip+2tSD46hyLWPjuxvZNX8XTjkiyDDjrwyi5kbR6NRII41s8dzCIudFuCOe915g3L/ZtknbKDlTwtM5T6N0VvaLe02tTVmduvqB8wrALc6NFype6GP7udfuFBEavAcSMcbxvu0Wga40+A7FIFdQ7z+83z6CINBa3EpXYxfug9zvKyYKAph0t+Kbdojk4+WMbXoFM3szjEYISN5Fp50nhUOWoldaYHIPsVfsk7Gk/JpCzu4c/Mb73a9pKAGPiSGclJ7ESm2FW7GoQ99RI36P9L16PF8oIHpONOkD0vvyPS+9BJZNFVy+8ScBY6zJGgM5YTlYBoUw380TysG+Suy20yg1ODY4En3lFs3RAowP5tLGS2T8mQFyOTZ1eegthlE0cCG+Ac4IAthXpOKbIeYFGz1jGPLiLNgtPgejEYR73tnod0f3+bl3OnHbytrotNWgkXSyfMtyykaXYRnpg0LTiXfmMZozFwHQa9pLSkwKvq72RA+ZTW6iGpeS6xjyupnt08mRq3GMPz8G3RO91PuJeQZTiQbNwAaUt1wYFFTDPp34HowGI+5D3Bn9/ji2HLTibKucWZZfEH10Gt1osX/fnlFvj+KzmXCQy8yaBSlX87G8dZG6GbZg7k2j5wAcb0tV2Prd3x0KMOnzSf1+9xnrw9M5T2PpasnRXRW899575Ew+y7VQca5l64Jg8quor17GzqyMlW//zl//Wcqk2lCkb0tpzG4k4cMEEj5MwI4X+P3jjfg/dI7rjWGkb0sn4YMEpv88HTMfR7rLGnm+PY6Kl7ewsSqCL/iEW5tukfBhArIXL3Bs9CU2Fs6gp/wnXK3FOXBpwyUqrlQQPCuY8R+Px9rLus/HvNdMLEweKJPzT5u9ZAlfvfsu/9PePGdAp7sLrvs3beL/m0wqlRIcHHxfzu//lP232Pdf+1+zymuVyE3lSGVSGh1baLNpA8BS5sDp07D943LMZHJkeh2Xfh+KfKQz2WPWYOUNxWeK+WuSSDtkiQLHp88T0eBHoVkHyBYQMDmAASvEosra7x+mzrOB6a9v53mLoUik4qLfmN2Ij0aBTYuenrZQKuvUDLoxjN7WbjK3l5P6eyq5izUctzyIRU8IyYFzqZHEAp/TmNvIj2FiQs9lgAuTvpyE79j+aD+nwVEcnBhEyLUt6LZVw8eiMKutry2RSyN5rWYbeYq9PLb1RZS6C0ikEczbLnY5tLVBsUs9tZYtmA29jo3HNErOlXD+zfOYOZhhqOnixW3LaRiXwN/uJQgYSfkthaTvkhj87GBAIEMDad1BOE6ToleLnTOB0wJxDHfEuySUNpsabNVhrPhjMqqH7NHM13Dr51vkH8qns66T1tJWhjw3BGtP63731bS2ib0+e1n5+11RZc94TzzjPelWd2NS+weuehUL9i6gbkAvxaHFZO/OxinSiYbMBnzSD3P9oTo+fe1TPg2eRdGpIlR2KkLnhmL6tRG5ToG1ayttLiHITMVEyeGVh3lo+0OUTVmDWqukU/spfgrRYTO1MWV12mq23boKlfBeC2yIyEfWOx8zq/sD6QeZIIhaJ9uXbMe3/iVRFyFPRKBFLY0iD4GZ5lBQ7EZcihtdczuJWRlD6m+pKC2VVE0oZe2c4/hFlLJ+wjE+cvqIGZtmiB2HUglT9vVw7sxC3l35Jx3O/rQUtXD65dNi4UwZRnT5Zno9T3Bi6iM4dUfzASKVj6mtKTs3HCCuLI5LYy4x6dQkVF7tsObutUcuiSRze2afRtS9pmntxqUolU47T4xGcVEXDAJpf6Qx9IWhgIBWqaU4dhzejfEYtAailkUx4LEBSJVyPAfUYmfbTMjuOWSHQnN+Mw6hDviO80W6/gJ//b4antrG/OF5qK1klJ4v5eCjB1l0cBFyDZjpzYg21OCb9CWN3mNgZiyCUaAht4kG70Hkhd4gJeYWt+Ju8ZHlzrvX3aHha6+viVoW1Y8i5Y4127cgNUoJzQ1Fo9agtFLSVt6GrkdH+JqRDL4RTpNDEzseeYUXbRfRldRFV2MXclM5Rr2RHnkeRye8hUOzJ08r3yZsfhhh88XkaqJPEnsdU4n+YSHrk0MQ7uFxFxCwvB1zjPPZwY+dLRiMx5Hd5vNvtWpCo9SweNdkYBvacf+OhTIxud/5MmgNNBc0o7JXYelq2Yc6BjiUW8bV+KuordQk1B/hVfoXBls8dLRIcjFKjegFDWYOZrxY+aJIxSKTcmj8PJCIXpdE6D9WpGYqGp3a0So0vPHRG7R5GPo6gQEaIhtod2xHba1me/R2ZnrP/Nf7+qe5DHDh8auPY+Nr88DtlrUF2Fem0mXrgdJaSWd9JxfeuYBztDNyN2ce3fYotS611LrW4qOS0lLUwq9xvzJmwxjwHw1AQN2rANiZQGNOY1+HdFZ0ISbAUONxbp30p5ZzDF8RQbe1K4Z7grsWuxZK/SqRmpr0Q6FpVdZ02nnf12F6/cvrmHZ3U+/Uzp/Ld/KT5+a+YDH5p2T4KZn4ifGcnXCWpKGlzLB6h20TfwHAKdKJHId6nhmzhxOODfxWJ8fQo+1X6Ds45xR6v1xqP1mEsUfVpx3jM8aHutwWeuvVeOZOx9PuU/SSbg4bjTi9spwLpz+gR4CPPYtxM5ymKNOPOSem0f5SO7VB4rNSdrUQeeE72s4Oh2cn9J1TKpPi6BdAoKqYmViQ2maHwSAQcvU3TLtbUYwdzoXx2ynXi90+AkY0HZo+sAGIvPCjV+0mRagkvf0DXGNcWS+sJ/HbRE4+fxLP6iEsXfcns6w2s7aihJPbB1OdWI3ymJIBDb2sIZzOiAoGunZwolIMFgz2TvS2Qqz3b0wN2YxOo8D/pjNRtmJgNurdUVx+X+zuG/TMIDyGiIhTn7E+SOVSTCxNsJ07hve8X0YnKeBJr51I5dJ+nalPffs43/qVcmOsN0+OmEB1UjV7F+1FIpMwYt+LLNj/LO1OQbQ7+6KwFJGTl96/xNiNY5GVWnDjUjzNPuWEl3n3dbPLTGT8GP4js4slJK04wLPhnZxcVYbc63VkChmLDy1m46s7WfL3EnbOS6PEOY8rA1+gjSmAGLR11nfS1dDFze9v3lfsEwSYeGYipb6lJISIa5V9kD2tJa1UXqvEbeQQIk3NUOV5MOSyJd2NXXR0SenSyjEYIDkygUjvIs61FdLR1UBP6xpUdiqWX1yOKsCdrrnLOB1ZjIVTJ+V6V8ZtHMe4jaIey8a3r1GdfZm3nMZhtWg7h0YH867wLnsX7aXwWCH71yQiWFyntH49ih8dmS+VcwczIZVLWXZ6GVKp9MEJHKWUJscmHv/9cTptuuF9UWu1p0XUJrM5fQybxkbSJ72CVArqyiYubbjUt/+NITdItK/E+vWHUI5PYdLJSWJw/XECqiHTkCrsGNhrSviJabTYaulq7OLShkv4TfTjnMVrXIk7jJscZr8FPb2vokw0Be84ApJEfzMiK4Jz7p9gUqOkvfpRdN3e7J6/G/ch7jTmNHLsqWOM2TCG0e+O7ndfZ+ekYVXewTM/PkOHtB7VLBXRy6OpvlGN0WDkvffeo9allrToNMJdpxH5cCRnXz1L6u+pNCdW4wn8+eaXlIxRE2+5mEovNR+//jGvhP9O4+VX2JFZRZb302Bdj9xSyaIDi2gtaaWmQkfbh6fQ38rhXHQL2YOa8LZX0bjhELE3sji01Bet6+/4Si9xKMqaLeYVjHR/nN6ZC2g6dA7XIlFPtcUt4r535RDrRfbIVehU1uhNxARM7JOx+Iz1wW2gG1ZbDUTWTaSqxB3bWnG+TvpiEtunbcdzmCfHkhKx7U0jN9ESnXCNrhuBeGWfBKBkWw4l59dQNTWBk42JePuZYu0l4xvfbxCMAkVPwl/u7zHddBmzfWdS3BrXNzFU6nokv81j3eDrvDitgZDIR6mTxNJWtpiAW7vpsXQk+eL/w95bBWZxtW3bx+1J7ri7C/EQQnB3d3dKoe7ep5RShQqlXmiB0uIUd3dJQgJxd3fP7fNvDAQofb//2/renefaSjIZWWtm1lxyXucpFoZt/WxxCnPiqVtPIRgFZEoZdvPPs+HaczhOvofOUwOY0feVvli6WJK+K53M2gLOR3vhkvc9If5hVCZVYtQZGfjOQP6e9zcAcv9yfDrSEIRI/p7/N10NXeg79DSdvYZFlB3+5TPpXdUf53HOVCZVYuFogbZF7OqRSl+ml2wzEwf8ztVDokBzta0I9lhoY8RKKV57s+Z+QnZHKsdWHiN47QLqEwK71weAiPkRVNyqoL26naCrrfhKlBT2hPZ6DQFjxGJ7+a1yPK2iuNn/JpGpkSzfMpLWnnWY9CacQp3YO30v7S0GJr2/k8aLViRcGYTsaQV2VVl4ZZ0ldOUAfir+DucRx5m8Yz6e2V6wRTz/g2Tg4KuDiQ+xwW/YQVIdign2+hbTX4kMyJHx11wZx0xG/Ow9qHKrIiMsE1sfW+6OfQer+mKCbm9j7Z5fiLSOZMuULY89g10pzcRfGsK9qHu0+RQjIBC7IhaVjYrKhEoy92cyZn822S/ZYrIyoDBXsOruKoxaIyajiePTrtApLSEoNwi12ojKWkVHXQd1WXWE2sYi71iEj88XfD3xHiOqHkfK21QqabwdjWxGMmkJoXTYVHQntjps3UiNvYvW7Tze+NOu12CmdUAmiydp4mqRVru6AJPRr1sP6oG5RLtQcrkEiy4Lxp0cR1Z0Nu5zBrMqZRVmtmaYudgQqTmAqlbBL++9RkSQFKVtCfbZ5wGQWMn44tnP6LLo4vTY03wfPhv3aLH41lzcDMXNaLEm76l0dgXeYqH1GhAEOmzcmPDlEDTP7iAsOo9Z4XuZMzUWc68Rj/hFAola8JTZUpfmze2/+iF5U4MxWiyUucW6obBQYDKakMqkmNma4RrjSvkNsThf5F/En4v+xFLjyOd8ispahUOIA0q1ko6CDpb/uYjYYcnMc8vAKBjZ2mcrLaWiL+CHKzpcyXu2g9+/mUp5UjW/9xM1Q609rYn5JQZfa19mhM/gs+Gf8UavN+is60QwCUjMZZwYcozh0bkEKqBNIw4o6Zckzv15DnmWFiuVWEwddUTH7dKjxN5aQupfqbRVtOHWy40pWx/3ubtnRACduTXNziFYqcXn/cUXRa3sp56C+D1ueKjHcXL8SUyCOE+D3hV1TdN2pXE35i5nR51FY655zP9sq2zDfkAP1ilnMGPfIKSqAAQBXn7BiCDx4vPU1fwdtZbpSGhZ6sPY030wjitA6P0kiMa1wgXb6uzu5P2/jQHE7p/2mnbs/O1odGun0FUEmfqZTDQXNBBz9gcAjNr3Ueb+yTNTranwCOOb2jW45/TBtfAmqcNfpqeqLxcLSjm7w4rW8jLq+hSzqeNz+vzxkAjuy5ib5ES183UTSFSvEzQ+iLwTeWT8cJc7sXf4csAFFFYONL/3Gev8piKItweLTiNm6i5mx9xjmLOJurBbTA56kU4ve4z1TdjWiF3Fo9Y/TvP3wFyKRWBYk1uESH+vlFGRUIF7b3e07nLiDk9iX1ItUjMY2LuTK6+cpi6zjjdq3kAwmZAapZikJgRM6Nof0rr6jQpgq+P77A4Wn9cOE+htHCmOmYLvXVF378MdawCRav+4uoOPpVKU5UWoupo5NT6BCJdGPnN5iYJaO34tcKH4chg9T+xAkMrQGLT0/s+g7vNl7M0geXMy434Yh0W4Bftm7eONr9/A4GyAOTDn4Bw2997M5bWXIQjsGu2YsXIGH4wUmZfmH5vPrvuhXERqD+RauN33NibBRPiscBTHFFi6WlKiLcGiwwKdUsePiT8S5hSGb4UvJqMJn0E+ItVneSsKcwUWjhZPzPcDeuuuxi7sg+2f2P5PfeNHLetAFvtm7mP6zundnVWPmk/qEZzK7iJVTX5im8kkFsAMSnMEqRythT0QRNaBLPbO2It9kD2NeY0MeGcAIz4d8RhQSzAJ+N47glVjCY4jorpppQUBCqIHcC/wA6z0ZfS/4sGWfjpm/DyCwFF+2AWIfk75rXLqsuqeKPbJVXICZvak1/u9CM8Ix6vci/ZP2wkYHcD8E/O5+ZXIEDDt0DQi0yKJVkbDNHFfvZkV6kA3QlN0ZLoFkxuSi0Rtg0Ql5hKqggYRG9aAQWHATGOGbW0emmoRcGzQGMR1MCYSZVIKRpmUZrdQhPv1LZlBi0GuQm7QYtlU1s1Y0V0clSlodAvDvioTHnHbHwAWASTzxnCmcRVTDk+hq6kLC4OJTms3dGZW3Hr9KNaLrWm1aeXIlCOsdJks5tm8euJaeBPThnI+8ZrN5Dnn2eubTZTbJKQGHdb1hQQm7cHg6EjAxxsptGnmUGUAq4ub+NFJBGYOXz+afPej1DY1syC6jdi3tnLG5SUAcqrKyPTciMxkSWPH65ikVciMrXR2dYhahbYeqFuqcHpmBi+te7Jg/MB2TtiJY5gjo78cjcpKhVOoE4IgoNKr8AsvZHZoPWOk4rPvUnADq4ZirFoqkGg70T2lwMGtAUEXRFdTFzY+Nkz6bRJHVxwFYMhPz/HS59/Tp05A26qlIbehu0M0aEgJ4wedxFwNn1eIC/eD3OCluotc7QKVwY6o9OG4qETQ0vBPh7N3xl5yjuSQcySH2X/PJnT6412v7TXtXF9/Hb/hfqL+sOx/LkKtePpp1q1eTfX/+B+PWw2gUDzMG/+32Cfav+X8/l/ZfzX7/mv/a7b02lKCXw1mrXQtfsXe5AbnsvLXlWgTism5VEWPm9sIvr2DLisnWgfFkxW/nnO9lOQ0ZHFs1TFAFJ0tWtfBzwGdRPRLZ/vokwiCwB/D/uDa59cA0EXm4R9RiE6iQ+r5JRKplImbJjLtr2k49PLl7pi3yfJdgjD8OrUffoOFlw1tlW1UJlUi0d+n+TTPRtaVgaQmA4Db393uHkf13epu9NSjJlHIMagsKY6aRPAbokdlbm/OorOL8Bnsg2em6DwlDFZSHD0FwSTQWNBIR534EXrVGE9ThAarmadp8pd1dxCef+c8QocBxypXFultmXx4Jt5p9rjHuROzPAYzGzN8UqzxOzqG2RX+TK8ychExIJp7aC76TgPLts1h4LWBeLb8h9KwMbjF9cHGx4a5R+bSXtXOiedPcPOrm920Q4+aVCKlwqOCS0MuMW7POPSdej5WfMzhZYfpbOhkwokJ9Mjuwd5Zeykb0EHQuCBm7J7B8E/FRKFDRSq+hbYAmBBoyG6g7KbYoZUwRsu6F77nY9k5zkQ706Zr5dQrp0jfnU7R+SK0Vg7oLGxZ2aWh/4bnMF1tQiqX0lLSQudT6czYP4PWTiWvmLVy23IFcov/+8XVqD+PQW6g3CO7W7Nv+fXlDP90OKZWPa+dCef6CZHKrb26jcm/TeZD4UOCxgdx107GFlfR6QlVikXIXit7seL2Cmx97ahxqUXiWsdL1hICJeY05DaQcziHfbP20XQ9k+gzX2Jf2kKTfRP1zmKg4BHvQcjkEDpsTCLdA9A7qS+uWY+jj6bvmM6HwodPCPACaOrb8cw+R48bW0nZJAY4brFuxD4di5mdGUGpdow8O5IHPK7NJc38OfJPCk4XIJVJOe35Bfu8bdDejsGiQtTQu/D+BVpKWzD4yTk//DxxXo28aitStmbsFt8PpzAnhp/sw1tfvsV6v11Y6SoxtosdIa0VrfwS8RP2FankhVR2X+ujQa5EIkHTrKHs+sPOvUfNq24xkWmRzN43m9ZyUa/APtBepJIAti7bysFpB9EpxKTYwHcHsuzqMuRmcooOn2DowUNIBAmZ4dnIFDJyjuSQdTBLvA5M2EjBM6gcbVTWY9fVUdvBrZfeYcPW0Vz6ewhhP0TSXNrcvb1FX0+7ZTtGqRjQy64/1HQD2DF+Bx9JPuLWxlukpaV10x89sMPLD/Nz5M/dxarHLMias6PPMvjKYJTfNz2xuaSnhkPTDtHvZj+cU8SkobWnNba+tvfHICAzyHjp5y1MSVz02L76LgM+1dORGWSkRaZR7fc4pYr3s96cHnu6+/d/dpf+n8zMxoyii0X8GvOrmGj6h1lX5WDZXEFBr9m4RrtSnVJN8uZkTr5wElsnF0oiJ3B+VDrbl2xH2cOK6hTR5bTxsaEuIQGvuz+iqj+Od+oxWjIr+L3f79Rl1LHw9ELsGq2JKPLHyrYdx2V/Ux+tJ2JOBLl9F9Np/VC3MC0qjb1zjmLhZoPQ3oFNTQ4xp9fjUnSL/Lg53ZqsKVtTOPnSSXKP5VLh0cjmpzdjlHWgVosUYyvvrGT6DpHmaPTZ0fS/0Z92q06QSomYH8Gg9wcx4ecJOF6UsPmuF51nBvLszytBLiev91zqPWPo+9Yg3CtdSLduJksHzSYJUQui+FD4kCUXl9D3txUURU/G2l9C71oXBlwbgKamgz/+AJ+6ZxiVeY8eShOB+l0cGZnLiQU7cQp/GMTIdR3Ue0aj7hn82H0wGU1oG9pwNQmETq9COn8AEomERo8Ian3iUE4YSfgRe9797F3CMsLwzLdBKpfiEOKA90Bv7APt6XD2pbd1LdHmJkyShy9O7+d6A1Ca50tVsRt/rVuES7KasFligb0+px7hmjf7v5/Fmmp/bO5ZcL1OvGaTRotlfSGpx8spz/cgp9SLp35+Fqc74vEfdOSM+XbMY5SQcc/EMe67cSIFs8mIQWGgzboN0/3mUk2zhtaKVgSTgHTwHZ6OL2REURhOjS4UnBW7k7wHenN16jc4lyRh0fJwrbq+/jqlV0vRtevIHaBhwysb+GnhdtZ8uAYzJ7Wos6f9D3WZdZhpzeh3ZiRTnZv4ZUghekcZmhYNjQWNuLX7EFgQSm3TNoapTLxtrsRgMHSfx7GHI1O3T2XxhcX/+m79seQPUqNSsakTg0wzWzMmbppIxNwI5FUmbq5byOFt4wlPs0LbouG3rTLyzSLIbnYlK7ALl9E36HV+OAv+nEpjXiMHFx3kj6F/8Ivnp0gNcmKG3GXJm39QP2DQY+etL0ghIlnLq7IkPvftRGnfgkQiwTXGFf9R/tTbaUizbMZJ5ohzlwyz+yDPtbK1rJWu5fzm80+sfQ/MxuSMZZsl1wZeIz9ELLJ7DxKBKk2FTUhDPfEYmoR1nxAqbP7G0tWSge8NZMyGMawofpteA3qzXjmJBo9wZB6uSOVSWitaaa9sR+hsYdNgD34fcJYF7/1J+IBZ/NjjRxJ/TKT8ZjlepWa0pYrvxWF36C97JIF0/0MgN8jJyz9FVLKMzqoyEbySVkvKbykcf/Y45g7m3bSNj1qbXSf5gflcGnIJaZB4v0Z+PpIlF5cgCAIZYRnci77HoWmHMEaJmnVTt0/t9psAfjFT4SYTv7UGigjPCEe+KI+hv99jwl4DdihQCwoEBPxH+tNc0kzOvgQ0Pg2Ye29mUbWS7W3i/qpgH+o9o1F25NNZV8zTzQv4nBaK/YqRmcsxmaAqcCDZ/ZbQYeeJIHsSGypXq+iyccOgFMdjMonf/gfUvpYFEm4cH8DkFUcI+kKkpfLo48XSa09RdqMM2+ouPn1/B7LlezkxJQmpudljOjhUuLI/3QtlTS2SujoMWkN35+h9GWp6KBt4Z+BsolxEjbHAxF2EX/2VkNxA5u5ejKNwh9lV2Thdu4qNtw2FPacjMYnf59Dpod3dpTKFDLmZXEz4GQUaauwo/HkUnScuAaIe5qD3BvFs2rMYZSaWbVlGdk8XJHG9qMsSASYPOlOOTjzKofBKJjW+SVNaJr7DfImYF0HMshgkKgWqziZ6FIyi3xUzLFNhc+/NXPn0ClO2TqF+1avIIh34tU3F6EQXMlT/QZeQwtQDIzHrMmPofSaBv29+xrWPAqg7dxfncGd6P9+bztI6Qm5sxSvjFB01ou+uslZReK6QltIW6odOos7VnLBrm6m+XYytjy0v5LzA0stLcTZ3I6zsa1RaFXZNIgX+lK1TWHRuEfXZ9QhVzWhJR9mvjPShz+PpGoZX1lkAWnJqGXamFwuM0ZT1u82NcefobOhk99TdHF56uPt23spdyOhSC/EZNJmgtAy/fDENcdodVio3khIr6vs+oHdrc/Rl99xdTDw+kcKNT2rLqKe4Er/2B/a+epQ9C3cgVUhBAufeOkdTURMGtUCxTzFxSXEMOBWCRCoh6ZckfuvzG5omDQICCr2CBTsX4JUgQdOiYfeU3WyK3UR7aSOGhFoqC90p14PeJOHoqqN8JPmIa19c487INj55/xNmdNmz+uWv2DZXjDndci9jXV9EVKstP6QOoMW5E9lLlVT1syEvLx9BANf8GwTf/ovPrT4nbdd99piDWXwf/D0hk0Mo2KJj09ObiLkXQ2NqQ/f6autri0FnRHUujbxzUbTaNqOzMKJz96XdVgS6KILVaFUP6SAFQcDW15YlF5d0/+2Zz39isWc7PVUgs7UgddTr1HvGcGnlbmRGGUOmX0YmN7F/VhL64C405fXYl99D1WXikie8bH8TM696Dk8+jBCk7PZR4p6L46fwn9g6SKR5lqvkqJ3Vj+k1l3mXURhYDIgAVLWTmjOvn8HYaaQ434OMW+EMOjuS5pJmAscFYu7wkD4j4/1vuDzgSyg/RNrOtO6/t5a3IjVKOTbxGA71DrRktZCxN4Ouxi7GbBhDwOkhJMTfQaU0kLdnDEM2iGt7xe0KtDe0ZPXIwq5JjK3yw63wnSwWdsduGMuHwofELItB3/XvshC6lk7sK9Jo9IhA4iEWVVvFsIS7d8GqVolDgwOLti/C+bwYE1RUiP9TeK6QXsm9UGlVRN+NRltSTX4+vP++qNl3e8ZXhFR+yr1Bg6kKHoUggFveZeKOf0z21brua1DoFajblQg6PbqmDvxSDmBX8RBAdmTuSZInfCAWJtNy8Mw8jUwnUkZa1xVQnSxS13/p9CU/R/xMbXotCAISkwSVRoVEb0LTLGo8mqQyBCTYlaVy7NwgTq5bhPqGji4bVzqtnJHrOvEeGcrqbWsYPvsCKT2T0Zkb6bLScG7EOTzWeHB1QQGVCg2zq+DPVpBbK5l/fL7YEQ5YdFrQoe56qNknCCgKsvDIOsv1sVaEvruZ3Z0e3DUayZXqQCpBJgPvTDFOeU+3uhto8E8rmfAsqSNeoc3RF5NJZGx5IecFRn85GnmADX8t2k6KSyXlFa5odVr8R4kFk6KLRXifFlj98Wrm7p7LrN1jcI9z58W8FwG48+NthGI3Xq6D6YeiabwVzr590Gb3sHtm0i6Bt/OrsUkzo7LeGoPeRHP/cdwd9QYji53YWBiDr7KOnP0jCTwdi5mtGe0Ovt3U1vVZ9ZRcEQF3yZuTKTxXyK0NtxAQ0Kq0nBx7EskQMZ6XKURNY3MHcyx2WfDydy8jaRK3mQwiVbLkvo6d/o33aXm+hYD8ALTl4rmCJwTj3sudhpsNvPXlWwy8NpAXv3uRju0dHH/uOEeWHyHx50S+8fiGb72/5UunL/91vl8uepngicFsHbiV85se9/1ay1v5NfZXbnx941/3ffAd+LdclMlEd1eWtq7lie26li6qAgeiM7PGO+MUTiWJYoHnPrPWAxaZ619cJ+nXpO79Wkpb+Fi+FuuGYsp7DCf47WmorFXd50yMO0tG2BWsai5i3VCEpqCCy/trAdGnmL5jOisSVjymIf6oVV7Jx6neCYtOC3bO24mVhxWWrpaY2Zgxct1IpGNEYJVvSQiWZg/jVI2lI7a9A1C1NDB/13zkejlenWEIejFmqAoeiverYjdcmVcZV6ctwW2qWFwf8ekIXip4CenoUaQOfxmJzUMgvCBArX9f7o59h6wBT9HoHkFXVlH3tgc5kIqQ4QS/O4M/hv3RLX3y91wR3BS1KAr1lTIG3JzIXytKCZ0WSnVSBaHXf0OpaUPb0IlS9zAfZ8JE24VEXAtukD74WWg2YJ8WhqzUHamrHrm5iqCEHQQmiV3w5koXtt7ex6uNAgYEJI8Upy68dYZRx4ysKs5DoTQwKboEg6OEkqsl7B+5H7OOQ7zX8yNk0oUsnj2F139+i0zvTKSZ6fimHceiteax+fin3fj6Bnkn8mgpEZ8xg8bAH8P+4PDSwyS/uA0zKy3VJW6MPyfSV1nXF4mFVhfxvf/z56mMmn8GzaSZlF4rZef4ncgUMl6qex95pA0uEWW88cl8elwxJ/75eIauHUrYzDDcJ8cxeNhZzNUaTme641IsxsTeA73p/3p/8mVyPOQwyf0qjqUpWF0X1z6fwT68VvEaVh5i3uLfOpw76zu59c0tdk3cJYIB/g/m6urKoDFj2PR//K+H9oeZObbu87t//2+xT/R3/y3n9//K/tvZ91/7X7OEjQlc++xa9+9SkxTzLnNMeh3GShHC1eTagyqnG0ztP4ORjXbIkmPQjNV06x71WtmL8j992H+wiAnLjzHATINEISFoQhCtZaI2z08jb5JsVc27e0YzrcIW4S3o9bRIw3L6SiPl9sfRY+CLJi1ItLyqkNL/jf4YNAa0H1zENnABOxbuYNTZUfiW+MGfYjeVU6gTFbcrSNuZht8IvyfGV9tcRLn6B+o9C+nn+zrgy7FVx8g6kEXEvAgiLtmT6ubJzej3qavWo217k+8DRc2AF+pXY//NYHYF+OL41J80Y8Ta05pJv03CZDBxzrWU0x+u415NLLEpEcjMRPrOq59e5diqYwTddsC2xoPksXvpfcMPue3DIPlBwr3IrwijbTLqwJcwD7JGKhM/IvEvxnN3610m/DzhSQoCQJWtIiQnhEvDLuEz3geJTILJYOLutrsM+mQQ25Zso9W6lUaHRlzMRmPnb8eJF06Qf1JECF4cepEWm1pCM0MxOnYx4osROEeJ6Ga5xEi5L2zcPZia0wPoiGmk16pelF4t5c9Rf5K6OJOeAeWopUaaDRLkJgGT0cTuKWJXWGR6JCvmXCbNrp6dDtvp0M/g/0b/yGSCBTsW0GbVxrFp9ZhMoLY1w6u/qGNgrOoiaucseo1IZOOKP5jiseKx/WcaDvOWlQMpeWakZdmibdeS/mc6zhHOeA7w5fC003zk0oRs3bOM7PDH+21vohZG4TPUh9RiCV2WNujN9KI2WtfDZVnTpMGjUE22rznuDfNJHD2LeROfRG3+T2bhYUdu/AKCE3bQmCMGhpV3KpFIJdh42eCbZ4NX4UAKa0rB4gbm9jH0ebkPTuFOdGk1XPCdz+WIUwT2l/N8yw/0mNKD1O2pVN+txuAtJ3fYVT6vcGPI2ncwm9GBe+9guhq7sPW1RenRH70pCUf3evzW7uC2ldhR+6AjV27Q0maRSmhmKL7Fvmgniwh6EOkFnCOdH0NNPmrjj/bErcyPExNTeNvDCqPeiJWHFR69PUS0mFM94enhzN47gKagKoh15aewnwibGUaRazHVUffotOjEJNEjCHQ/Py8VvMScc76YFcbSf8N3TAuFlSaBB1A2iVxCuUcFgY7NXDRApW0jwiOQGcOVZp76/Sn2zt7LzANLkbY+HoCY24lJC+ER3ZNHLXNfJkB3cHl57WUufXiJYR8PQ/AR97FrskNmfLKLU7hPFdT3Vl90jmKH4Lah24h9OpbQZf159/N3yAjPwLauAl354+jKe2d2s2CrGzvm+3Ns0jFCDXMf2/5M3DN8dfMrHOscmXJ4Ci3yFngSZPmvJpgEVNYq7APt/1V3oT58CJWe8RgVYjDlEuVCzPIYPPt4orA2o84njhr3ZkAM+NvvJzEdezhSdeQEzuV1FHifIbI0ltYyN7StYqCq79ITmulPVUAmP2jKuBiWhodS9Ujx9h9O132KaFNlNUGJ4vMg13chNep44Codua+DBVARX4FnuSeDEz/D8EIHoMYt1g2XaBdud7Zyc+0WTFITdg2WmAxGBr49kM6GThpyG9BNjyHMeBdbfQeFRnF9anEJodPajVvrNxBPDNPnnuSnJXvI7HRi+yOXWXY2hRarNpb3eZ5hF904c24Uuqnt2FZlIUhl+ETV42whFsBcXJspCG5GKpcScmMrejMr7CvFgrxFj6mPDb+zrpPbs74heGgGFoNKSKiYjMkElSFiscH57U/wNYrBw5IrI6jV2qAwV/BC9gvdx/hq9FJKbvTgCxqwtzByff11zr19jiFrhhDxnymkf3KYwioP2mYfocoxnj4v9aHn8p4oLBS8l/UbdQEXaDPYsnLTSvQWUvgabP76Cfv2VsCZXZsn4fzyLarHXkDj06/7+fIa4IVzuDMX/nNB/DaHOZF7NJf9c/Yzc+9MDif8Rli+OYX+zghqPdo2Levs1gHwVuNbJI29yHCtNY5/FdBsdY9JG2LxHepL0YUiSi6X0OgaSoHbtww+donCqdNBAJWNiuAJweiTTQy2a6HDBBe66NYBBohZGsPN3bc4P+gKtg0OZBkakJkLlFwpYffk3ThM7EvyuPcZ2cOI4nIc5ntG0fVLk6iEjpgo/Z9owgUBzGRLGH2mC72luK5U3K6g6FwR7r3cEXR66irdkfcycctyHsu97HAsP4VlYwnZ2QMBgRmWcH5wCt+61vK2/9uM/3E8J54/AYBOqcctVgRA5DtPpS5TDAqVlkpsjjXSf+I1vILKuFmjpr3WBkEQGPTeIARB4Pw4JQFO0bwz7kVYD6vTO7rvFUDmd5ncfPYmQ9cOZcgHj3fAKbM7eWPjGxycehAhUFznwmaE4RrjikuUCzUDvuKjoL00rp+LRlmIma0ZIz4dAUBdHTgf1hAQVUbPD5fyQ/NrNBV6MPm3ybjHufPRkp0U7l5J7PNXyJMsxTPaB+eJwXQ2dDLo/UH0jjzGkSYfXv/ha77c9h1KP0dUHY0EJeykxjealPDfSY1Kpctcw4/P/chzPjOeoMd55t4zWHs8mThQdMloUeq4NOwSM0NFXeTtI7Yz5tsxWI0NY9/sfd3/azQZyTqQRf7p/G79VpQ6XKrceDmjN3rPLlb+Ogt1p7h2SE1GbKlix42RJF2IRxPWxqe9fsSgMSB3VDH68++JUko5cDyS4ecG0/lWFeohfSlujGPcwXvAMmIH/I0TC8lpdMBcZoniXhKRVy+g0ItJ+vQhz/Gg4/SBGbQG7npNx6aunNCiZzCZVqJp1rDefj09V/TEb8xkVk0ZibV9C5cbYIkgsHbcTZpl9rjdTMECGz6ph8UhdWRY2mDlF0z6wHn0PP9b9znSR1xmxed/Yar14cwbYhH1hdwX+HzBN3xc9BLyp/P4ZNt3tDVI4Ftot/fCqDDDvi0Pe6c2tDJ/dt+owaLYB3M7cxo9Immz92blYi2jFzh2I5pLrpSQ8nsKfV/rS328lNMhXxF3bjhOniJd2Y9hP2IfYM+8o/Notu+kU9FIit8CAkwm3p4+jebiZi6tvgSAXqGnvUXNrZP9aDA7wqoD7wBw8uWTCFo9bvnXODQjC6PsM3oMtCE8Wtbta/d0PMDywcvYaJxP3E/LqXcBk1cVodn+HJ8gZ1IlvKEaSj4XkPh4IlNksbj/PDz7ebJWuhYrwKqxlI6aGMCSyHmRePX3wjHUkXy/M9zzeY/g3GDG+z0HPEwENSY1M2GfOanRg9m/WMPCHq4IJoF1tuu670XL0eHIxuSjsXTEJDPS6BZOu50n8ePMqS5tILShg9M98rjooqI+u76bSnlO5mrefEuCszOUZHzIzJQetHs1wtSp7InLIF5tZLQaIPk+8EyCrkOHbVUBGrUDI2+LHeiC4km/SWIjI9S7HrkCphONIAgozBWM+GIEtj62rM3czT7ZNhb/sRj3andMRhN2/nbELI9Bbian780+tJoVcmjKIVysx5L8WzJ1GXWETg+lo6KZ3LPeOKrCuLNhNp59TbQYRJ9OppJh0gsYFAakUhlGuRGjzIgggE1tHkpNK13Wau41m3G7tZxD0n301dkTZgomKGEntnX5qPrE4K5uwdxeXL+LLxbTmNfI/jn7cfaRUT2hmi/f+JKzi64hCALbP6+kJqWCpZ+GcGrSOf52KiN7VAlR0mX0q7VFZhR9H92dVl7Lfo3rA67Toe7A4NzJ1kFbKb9VTsDoALb1usCH3rXk3Q1kXEIAnaGdOJSm4JNxsnte7Zyb2N4YRBl56AQJHakF+N89RYHnQ7+pj3c5H6hqmOzy0B+1DxR9y/Kb5d1/W3j6ISX/W+veosyrjL/nPjyX0lKJylqFOlzNune/YMjNfsSfGkNbVRsTf5nIxF8mknM0hy8O/sxnARqOHRlIxjfpzPn2aRK+T3jsebCMtmT4huGo9qhwPOpI/Evx2PraYrG1juUnLtPRqxWDcAZ/mUiFOHnLZE5MOYF+tR7rVnHdvtE/hRlDHtfb+iXqF5oKm8QkqrvVY9uMnTqcyu5iVJgjCI9LYJibw/7n88lgP0/99hRIdDQ0wBd9D2FSmfN10hje7nwbtyo3ph2ahsaQzTtZrmi1EPfgIB3HaXHS4WN4GkGALitnGt3DkViYceypTPItz9Bs18zGV37mY/P1GDubcKhIQ6+yQmGwRS9v5jWn0yQj+g36ghJcC29R69sHo9KCgKQ9JHzhQ/ioBbjGuFJ9txordyuG7wjHM28wAKVxOhyf9riv7waCRMD/7kG66vzQNKmRt0KTSyB+KQdxLE3GZHJn9cXTnPMfz80pOSxy/AHnT33QS1vQekjpcVTN0IQF3O53E1mTJcZxRq58coX6TJGlwLPVluVSMzwN3pTkB6F0UaIsLcC24A41Pi9wJ8uDD0Yt4Hq9JccrrNG263C5tLd73tN2puMUbItXPy/+aRprZ3SPdC6l/J7C2TfP8tStp5BYy8kPKCQ/oBAmH2KSzXPEvxCPfaA9brFu+HhModzrNmaaYqza7j8Hj3Q69bw0gGuNFvS91RdPk5JO6zQiLx7o3u6Q/Qn5PRwYfnk48YnxdD7fgVFuhXlbJSFnB3ME6DnkLt+OOoG5yYvPol3J6yPqp08IyCZr4x56TO0BwNiNY7m+/jrhc8JJKExg0NVBZIVmQYx4rsaCRkatH4VXfy8ONRwiuT2ZXq/0Ys+uPQx6bxCbe2+mF4j3VKHgi9AvOPTiIXyjfMnqyiL/VD79XutH5d+V3I2+S51THXqFHmQilaPJYKLgbAHtVWJMFjIl5Im5BrD1tcVniA+5x3JJeC2BEStHPHx39EaqU6qpTql+6Os8Yg+AUw8KLf+01BGvEHX+WzqSc4Chj21rL6rDN+0Ywv02taqgIYQKIhAw7hnx7TJ3MOfG+hu4RD7MdT3qw1cHPg50EwSwaxcLagVBOqKzvKj3ikHpJYIDNvXahK5Nx7yj8/Do/bgWK4CuQ0dbcSOd5p0cnXSUYr9iVFYqDFoDWwZsIXJ+JM6jxnC1IYJOG1c+PPU4s5bLmJ6kJ+vJdDlKeEY4AYca0K0qAELuX9/99VkisgEIwivd++q79JiOn8La5I20zp2Y09uR6IYiCGJBUGrQYlBa4FJ0m8Y9eRC6orvYJ9NriLz0A7mXRKaNjjrRp+/3Rj8ur7ksdg2W1ODR6UBskahzbOZgQb1nNDKDjn5zHKi3XYNFhwVjT43FPK4DTUYhjuXZGOVKJB96U33wBF/rnFFeScB/spIbWhHAVdtjEM7ZV3F1siNaHo1Dow+Wr9rwofAheSfy2DlhJwA2Di0IAmxLe4eJQ1eha6+gragddZSSpdZwoq2Ci20OJBtaEQQBQfWQ3rl97TecqIx+QkMcIHV7KvZB9szaOwuA3GO5FF8qFsfoZ8+5jkiczhqxlonSErW+8djU5WMwF9eH8qRwttRdpkG1FaewqQz7eBg23jZIJBLiZ6cyyv8Q6U+/heV9eYAHMdHH/l/zdclw5jxzkPM7pjKh5vG8jYDARw3gXDeM9F4WuJr17t4mU8p4rfy1J8bywByCHFh+fTlXPr7y2LP/P9mWbdvw9vDgtslEn//D/90GLut1DPZ+mCP9r2bf/779t9j3X/tfM7mZHMfejoz9Yhz/eXEz1q3WbHxlI2s9F6JuCiWt3Au9mRUmyWmW2HVx6UIf/A8NRbOsk36v9+Pm1zdJ/SuV9p1pZBCBo3sdwQWuGPqbsHK3InlzMvpOPUENLzHZ6i9ySp3QVTqibdPy5+Dt9JjWg3JfGVrN54QrG5hnac5rJeb30Xsq5Obi6xGUL6JzkuKSKA8QEx8+g3xE+gKTwNTtU/9Vp6XydjJT91izb2Yef5Z8zEucxnuwN3kn80jfJSLu5u+az1evf0Wj1RVkyncf7iyA1teTKtsiun6Zh2NIFyajiehF0UjlUs4eLWGIOcT4lvDH6DeJ7y/HoG3rdpo0g1yonn2F2qAsPkubSrWHBsEkcOrVU/iPCmCh/wtoldXABxilXdTpVmHQuNNW0cbgDwb/K3XiA7M5ZMOshFms/XAtRsGIXCVnxBcjyD2Si8xMRrFfMQBjkluYOEoMnixdHqKTrg28RnhGONMPTkfwbKW5uJnz75xHsVGBfb6CQ9m9aLBso8yrDIlCStrONJwjnem5sie8Bu09DTi/nI5u8liGThvd3dkFYJKYCHdsw+pKf1pPJ1MzqzeEPd6+/m8mCHB10FUmnJjA7J32CF+InS6/RP0CEoj4aCaJY8s4FHWba+p6JGZSdk3aRe6xXFbeWYn5/pP8UPIsQfEZTL/ak67Puzj5ghjMvlLxBiDwrj2cjc6nsHQSKmsV0/6cRkNeA0dX/MC9gVc5H3meubvmEpwbDN/ArW9vcWfTHU7NySbHJR2jWRtKSQSyf9Quv7D5Am2rlldKX3mCclWqUtDqHEjSxA959jPxb0XnimjIa2DiLxO5MqGYJtNVXvjxBSr80rBwEIvcBxceZGHWCwy/IaGtsTeK0QkMsZ6OmY0Z1Xerqb5XjdBfoMQXDpv0HHOvwNHGlaFrhjJ0zVAAND1iyDaLISj1EkVWO1imEp+LhacXcvWLG9z5OYGhF4di3mVOn4Q+97tjHzpfk3+b3J0c/qc1OVnTYH2NnNByzGzM0HXouP3tbaIXR+M2OpI+mX9C+w+oO2ww6YzUptfiGOpIwNgAjunLOeh4kFFnRhFztyddi7uIXhzNve33aMhtoF1uwtxSfM8bjKJD9SCaU9oqMXvhT5a5wO42WBuRwDfOPz2cb2sZFR4VdKg7+PS9d1mqfzwRMX3HdKbvmI7RaCQtLY1/2svFLyOVSbsRqS33qQzrs+ox1RuZf2I+xyccp8Pait/+sa9Pshm+lVPZumwr4fLlmIwm2qvaubL2CiHL+lPmVUatcy03J69i0fTHk7aNjs2U9kqixUY8n8Dj857+ZTrjLowjOTYZmxYbjO3Gf70v/2bFl4o59dIpxv80/jHtzgdmsrRG0tSOZ9Y5ajN74hzmxJTfRbqkhGQNTeq7qJsb6H1jOHrvdvq9OoE+L/ZBqpCiGeXE+qhVyAwyrg48yzexY3kh5wWqkqs4++ZZSnwqudrzDuXNlrSjx10pUHC+CI/sQgp6PEzY+Bf4E5odR+fwNiQuzhTFTAVBQN1cgVveVdprBmDpYkmfl/tQm1bLrP2z6PXsUPrf6I9/Xi7Gtk6q77ZRcLaA0GmhuHoFcn6CgtAUc1Zumol2SgeCYMWWAVtoyGnA3M+T/KIY6seNobh/X2RqA+qmCrzTjnVfk3LLbO68+yfzKjRU363mt76/YdSK8x4u1/P9jGbe7ddAumcpSv/ZeGWdwaAwpz5wKEdzXmRI+BUuFHgRlP06Rr0Js/YG9Mr7z5XTk4ABubkcwxgbbrhkkHYnGPuKNEzGCCQmE7bV2WAUz2078yRyazc0FS89cYy2yx5cylMwptiTql4mlMvFQsjlNZexeD2Q5PEX2WWbQY5nK32U0cgUMsztzMWuds8uMjwyiJPF0xp3jxbEAEQbFE5HXQcN6r3ci77HW7Yyfl50hWVlPRAEoRtE8ueoPwFR48MpzInSa6WAmHx02qAi+qYoWmq91P2x77VcJedMp5T37dsYtPgcyf5jsfGywcrNCoWFgis/3+Zmz40029aRGaok1mo2M3bPwKgT58NT18ZmSzuMJjnxRWq0zRo+8voagA/0HzDDawFGWQdtmf2Y8+so2ia24PiVmMBoOHYL6dhhjLJ6mj9tWsiKSSHAamz3tRm0BjpqOlDZqB7THYL7AbhRxvUB17GTiECK5uJmMvZmELU4Cq2XDelL5QRatGCR5wVSKXU+vdCZWeLS2kr/c3ZsLV5J4oubSLStw8LRAolMnJdJ+xZx8/oZbGVw43g/gjrsSelKIWOPWCiWAbdrrBlloeXYLxNYdCcK07embippz9NpaLGF+25EnuQMMJUPhQ/pbO7kSzsRAd5c2PzEMyRzsCUzspV6x3rCrMTEReqOVNqr2vEZ5IPc/AoSCejTg7D1EVHpgkkg50gOe6btAdSk2fkwQ3WRVqGUmxtukvhDIq9XvU65WxVlA66xu82bwIaFLBwoUBg9ldxcmKmCjMFlWKgy+bDyLeQX6pFEZYDDUExSGbY1Bbjr3ckIz0BvbqDOvA7hgY94n35sws8THiv0FV8qpr2mnYg5EUza0QeFJp6Nr2zEYDRibm+OyWAi9c9U+o0Rk3f2DfYMvjIYRndQTjnJm5IZ+M5ActcqON/4LUJmLJIr8Zhm68gLyiPmXgwGazD30rPi6V9IuD4QiYcahaUKg0ZEe5sFWnN49Sq+eekAq2x1VKr0mBC69fjabY1YNssIVVRhl2mNancgmn6tqO7cQKZ/2I3jm3oUeITTGqg8m8mKn3sBvTBKKxAEulHCtr62uFi7YWfbRFmeJ5X5agQEVInXUNv7YB3gSGtBPaXXYvGdcBc3k4k2YwMVZlPpyUQAJMNLSQ3WU7TsENu18Xj2eRZ9ux5tixarRCVG7DHT1JLVkYV9h9jB/iA51yfiIxZEfspWye/8PPVvHHShfCx8AiDSjraV8LH8Y8ZsGEPfV/py7Jlj1GfVY+tni1rtSEWYFsJOMiZHpILTd+jJPZbLR5KPCMGNL9/4i+kHJmNhl47ywzlU3BI7Y2RO5kw4PoGvzE7Q8p9vMNh9gCAIXPnkCm6xbtQ958SF+h+od7XBXpBhcjFj4HNi8jDrQBZln5bw1YIR5A1tIXvwFWTEEThqHB8PXY5W0Y63DD70ukSHh5QX/E0YXaaKc3U/oalXqsmLn89cP7GIt2/WPsztzZl/fD5X7ZejkdVT61KLubctyb8lc/Tpo8z+ezZyT1dkBj2eDTNRlPZDKoG842KxTqFW0C7Tknwhjlte7php2jGaLCjsNRNlZxMJ74h0Wlu2PcNfL78GEgGHYAfGfDsGwSjwd591CBEtJAeZeK3WnOqMCPQNXQhSKYJMTpsApzpAavMri44OwKapmKbpTQTe2UudV0/yAvNJjUolZUjKE+uFpEvCqhwXJlcE4ZP4HLwrIvFvrL+BpaslQZ1yeA12zduFgz6aNR2rOffWOWKWx9BW1cagC6HUOwXzw/MfE6cbgUdvD/q/2Z/ez/fmcFo2p545Qr5FBdYpfhiVShYef1i4sn39F2ZqPXHwacWv3o3rrSJgKKffEmyrc+jUf8SFPpeJ7ZpGZGoklq4yBH+wrRO/WRbzp7D4kc/oA6rzroYurBpkDLIexLWB15CYSTBojRS/L3p9aaPsuROVxS8eDfQ1g8Xlor9WEjmRHjfEjjrLDktCckLwLfHFENiCa6wr5bfKKThTgLmbGT82BxNc6oPycj/kixVYNRR3X8f25zazs9KApT6G/r+MwSXaAvWUYPLTbXDUebDo0HKeG7kZS9NprjpYU20TReWdSq6vu05HbQcLTi7AOUJkGmktb2Wj/0b6vNyH0V+OpsiviPDMcF762hM+gqqUKuwC7Bi8ejBJJNEhwIXoFG4GZ7MsYhmCScCoMxI0LoiY0+Mwq9jH9UprJBkCduF2TNo8iaNPHxUpqC8uQVOl4VTcKUa7j8azjydVd6oovVpK6tfn8AKcguUsWLoBhp/rfm823duEZ7wnOqUILqxw/JPStrFA324K8gdWm177RLFP6WRDTt/F2FemYyivhkfUjVQq8NcOJ0PYz+8rfmd23WuUloJlUzl6lSWYqZi3ex75AfnsnLeTF4KfR3tH3Ddp4oeM9zzL1F+gwK8QmVUyghBLk3sETe4RDLeDRrdOmuXNABhl7ZgEE0pXW+6Mex8kEsLKNmCUdmGLFVb1hQiCP4rhg7kj64NOJY6jNGI8k1eKMfqqlFXd16727Em+6RptVg1IPEIeS9pq9DpantrFcLd2FptVEGX9DtIzSsp7jKTTxpXraee4WL0HqZMFPuo2jCYj6lYJYEvFa4U4tbmitOwk7E4v4jJC6HpZw9UPLnYf36rSjf5nR+PmW8uZXcOx72FPQXww+S4DcCpPRXohm4tOS1hYnYPb7lG0Dm9AkMvRqB1odg7i6NIDhM8Of6LYJwgC8o5WIs9todEjAhiFWy83POI90LZoMZkEghRwygO+bRI7MuwDnYh/IV6817EOZBplXAl/BqXemS8aPifpF7ErzKGHE2SD0baJwNnl5DUNxmB6PDfkl29kWufPyPsbOGrsh1SlQNrWgm21CKyqmXCG9fWzyPPdh73wOJCoK6eUXs/0ImqhWFhyCnNi6rapAFz99SqDrw7Gys+KdweKuaSjK44ikUl4Lv05Plv9GbeW3+KXkb+QV5fHGKcxjx1bX5GDUqpkwq8T8OrrRdrONJI3JZO8KVl8tlYZyXDLICMig+/GfkdoHzGvEjYrjMmbxS7ZxJ8SObj4IFP/mPoYRXvlnUquqa5xePJhRjqPfOy8dn52DHx34P+odWzhYMG8o/P+VQZCpM+1ocPGHbX7k91LKg8HKoKH0mnjRotLcPc+j9qodaMYte5xutd7f4jad/lxc7CrzCD/q3z6x4xE7aTGZBIYejgFx7DxXB1WSvYA0Se6T0KBySD6rO3V7ejadU+AwbStWqqvF1AWXEaxXzEvBIqMB6l/ip1yRr0RfVk19S7QaHuFxApX+vmIRRyLlipKtl4nq0cZZ3v+hEe5B/UDpdja20KFqDFf9nsrBIDMIMOpqouOolrAmeJLxaJW971U7B1aQe5Np7UbFuZqBAEsmivwyLmITV0BdV498ZkdDmkPO/skxoeMI2/Wvdn9c9+X+3J5zWXSdqRRtdiHvPIsLJt30VwyDhs/B5pdexCYtIeaK+4wBZQ6JVFpUbQ467F9ahq5a3bjUpyAEBjMZ2/s5/C+0RSfXkfrqlWkDxM7ZtWmVtxKbxJ8L5eRmlgKCt27zx80PoiZB+fz5Udv8+KANFrqbShdZ4Yh/yoT9o/h+Fx70n2yUOfDSOV4sosTUFfZstBGh9EvkMyBTxNy8w9kWh2JPyb+a7Fv4emFj+Wi7ALs6P18b2KWxbA5bjMewNHJ1/DUiGDgThtXCmJnYGklJaffEbZH7Kc4y5eEsgPYf7Kc7AorLg4TRVJTBph4YVAYnhPmEekZTk1aDdV3qwmeEIypqB1b/Hnz9Ej8nZ9D4STeg5vf3CRtRxrRIzTMtIR9jVWcj5+MvT4CGMcfw/+g6k4VC04uwL23ezcF7aMmU8rw6u/FgpMLntj2b+bq6sq69euZ9N57HNXp/rXgdxuYpFLx+qufkZb+8Bv4386+/337b7Hvv/a/Zim/p9BS0kJDTgM9Ms0wEkZGRAYGQYdcrUJrKXZ62NW08dmqt+g3KgH1S9tQuk2i7/vxaCwdSflI5DzuGhNLR2sCurQQjFoDg94bRNwzcaisVCy4aUafFlv6v7sDebaa9w1fUJlYSWViJY5r45mzdw4Bi4/i2W7JSweG0TK2gVpjJ3b+dph/9Dbv6UUnOKdHDvW6h3zk5945R3tVO96DvPEd5ot9wD+4yD0suDz4MhOOT8BwGXhZpEU4/cpp5OZySj1aKHK+y5I/lqAwqJF/Jefl4pfRtmoRgMTApbTIBUbcqKHJyZlrn1/j4gcXWXV3FUKdnoCMUHr1qGaL0gKJUkRSuvVyQ9OswWgzCLXBAzjEa99/w9TiYZiMJhK+SyBsTgRhqm9I8Z/PN1Ib2o82kdu0n/rw+fwa8yvDPh5Gn5f60NXYhaWr5RN6VREvRLD+8HpW/bKK61XXmfXLLAa+PZCBbw+k635ySCVI6H3qZ9rrQkmUuXF3293u/V2rXZkZXYrOej8yvz4YdUaGfDiE8DnhBGwx4969SXR9vZa/QlJY6vwtib+noG3R0lTcxL2oexR7FVN5rZDWagfGS0CpNtBzRU+S9KWsd3uDfSX92KbsxMyyCxP/95CSxPhErFutcWoZjiBA/ql86jLrUFgoCLV35tUZ31CiN+dqylos5JbkncwDoDKpErW3DG+7XAyRApkSZyydLfEb7kfRhSLaKtvodz2OpMomRs0/y54LZ0SHrkMvzm1fR2pcxE7WgoAC9PepRwWTAAIMPupHH+ViNr6ykTz7eaSWzgMe0os96GRqLm5+othnMpiQGgwIUhkmk/jB7/9mf1L/SqWxoBGd2kC9qp6P//MxcfliR4fPELFbU6KQEnYnluz2Uhx9GuiyNKGwUPBq2auY2ZlR+9k9Nm5/B4vRHWj7zaXv6EgKzhRwbNUxxnw7hkaZQINlLQZjDZGpkcj0YmLWxtuGIZ+P5uJf57DotOD8iPMk965hjsMHj40p50hOd2flPy0zWkO6z37MdCJyTmGuYOB7A5GrxGd1xKESyj1C2PzsMeaErydt5znqs+rxHeKLcPYSAO2W7dQ51SKRShj99WjiXxLRm0eHFjPa6wZJF3syJ80N4wITDxivlTIlr1pZAW3oBXC8T6n2wGSxao7VHePl715Go9Jw9q3tQG/+b63ZYEVRHgxyAokEJm+e3B1IVSy/hH9RECrtFfQGsVhw9bOrlF4tZf7x+TiUKvDOiOHopKPY0QunUCcm/DyBlrIW5GoFfy0SOyrt2qfg1DybRbzSfV6Nn4Jjk8RC07gT47BzenwtqzpfRXhOOCfHn+Sb179h25Rtj21P25VGdUo1Iz4f8QT3u9pFTeT8SByCngzGACR6LVYNxbgW3qQxzxfnsIeFyOKcTHqe20GYahIelR7oRnXRVtnGvln7iF4a3V2UNMqNdMm7kChkOAQ7cOOrGzTmNeKgtkWQClyzkrB1zWvU96ujtLEUt/xrVHvNxqVpMg22Zxic/zK+iU1omzqQWLvS4Cl2U6k6m3HPu0xnfU8sXSwZ+61YiLm27hpz9oynw6KD26OHE+BsR+n1ZM69dY5zb4lJpNH+/bjU9wJGuZxR5irOvHmOhhwR+WcR48OYSd+Qb1HPb+l96axpI/T6w/KtVqnjjH8uYTeiiKkYSrGiuLvQB6AwKDh2LZq6GFcK9DLetHSgOGoyzeq75Fv9iU27Gn+b1Rze5cOozNPUrfLj3ug3ACjTtBGUsIPGEzdh7kP0upmNGfolDuzLz+Sdo9NRpxzAqA/DI/sCroUixU7KMi0To7Oob/Cji0D0nXo+U3/WfQwpSk7MOIBvXhAtdmPp/VxvvPp7cfXTq2R+nckby9oY1LOVfB2srNZycNFBUv9KpeeKnkw86M3Gt21ot65k4vJyXikVA/DOAaMpKtETO2o6u+4/QnOqoMMgfpfHfjeWUy+JyTiHYIfuLvsHgvduPd2oGNTBXduL+FaMY4inB0q1kg+Fh3qaS/Y8wxafRJpGXkNhdgGTsTfbhm6jLrMOq19jKb70FR3qDvKC81jh8w0GjYHSq6V49vXE6YoD3195GavgKpbmusHTD5MWWwZuIdg3gLzQVJzNgij2b8LdJQiHYAfmHpnL/g03aeJdvAz76R9r4Av3dOa5fdy9f3XKQ02iR68XxAB8zhYpOSG+pEwQkdURcyO48vEVis4XURajom/Yf1gls+DV9EUYtaEoNK04ld3FVB+NSgfSTnNcFHRrebrFujHwvYFYednStb+MM3mjOHclCpU1RPwngs66zu5EyGnrWr7MtiHEPxdnpVSkx/s1iZaSFgSlAt/gfKoM7qy5YUZFw05gKgBKtZJRJ0YR2SsSK+fHk6YARjc1p8deYuLBoVhZ6WA15B7JpTajltFfjUZ/KJaPDIHs++RjKPiKhrwGfgj+4bFjrB50jvPfTkYeYk/oylBq02opv11Op5+Uczbn8C8MJe74WnLtwkhsnI6qo5GrJ83J9NRwSpVNX10zViNDCQq2RN1uT6b6WTwzjhOSG0JRD1fS7LdAlwGTpR5dh468E6Iv0GNqD068eAL/Ef70mNqD6+uvU3K5hIg5EdSESJHVlrN061K6uhpw/sKZ8NnhZOzNwKg1MuHYBCSChJh7MXSF6Rj621BS/0pl36x9eFfpmaVdyOcrN6GIus0Gl7mcHZ1EvWM9M/qtZq3+Xd5vbqK38hlcIsczJ0ikrs47nkeH1IRdlRv12b68ODqRQ16RmMX2Ivv3W/jcq+HEXA0Zg8Tia7H1cHQ+biitzWmfOBebHT8D0G7rSa1PHP80Cw87knsmUxBQgCAP4FmTiMQPGB1Ar6d7cXcbLDh+F8usX4gvdUL4A8oixuGTdpyA5yZw4vc/sT03iLeux6E2WJC+ah+xybHdxx8aUYOizZw9UcncaXRk48Io6rLqOLj4IOUTjFzy2YTg6ETBop9Ra4L5BnEtkuk1HEjy5e/C3+gzw4kuiy46lF1UJlcTdXYHtX7xpL4nauMY9eK6OvyT4aT8nkLEvAh6/96DOzm9CO2d1f19n75zOvtn7+9eV6QmKb7Fvmg0osbwwHcHEjojlCNPHUGFigstntyW32O+mUDSz0lcWn0JS1dLnKrbsRpnRVbYLS7Y+xLdspN5zAPgxlfiOns+X8HKEen8uqyEngl+SCQSDLIOkIDHfXdcLTXxog0cRCD/VD7Jm5MJ/GQJe6+5Y5Irkd7PMaqd1SitlAgmgec2LqXOsYY/F/9JeXsx/q7iGrp3xl7ml7zK+Wl+WLXU4VmeiqYhgJQNtzB3MCdqYRQHE69h3m8Ty7LDyTr3NZlThmFfbotP6lHsoz1JF0yoQjPYenEiSTmeqNeo6ftyXxryGjjz+hm8Cpt5YfIexsQ2cypGQNHvRYp31WNfK+GGJ4yrhANui7BsvIRjcRJy874U9pyOTNdF/5v9OTXmFDLpk0kr9QEJ/X97liJ7b6waczHqjXj29aTXM71oKmii/Ww7z//wPHvmX6WvsBGZUsag9wfhFuvWvW5cHSEm0UwY8Rns003vps3REhWczK8uQK/NzLuvDSwIon8WctMKl5srmLnhB87+vpBIcyNsEPWNVF3NBGRHcLPfDQZpauHADJqH1UJ/yO85A5eSJKx0RuDhmB7Q3z6wYZeGkR+YT2ddFwZnCS1OAZhkSuxC3bCvNOKjgPTjA3GVmaNqSiUw8SRdlo6Yt9dzZnQy6eGXcKtyY0XACNyiLOA+Li3uDxfqmc+R+XrUI16lr9qIY4WYbA6eE8Oin1bg1SuLfk8Vss3QHxd1AApnOwoDM0j3mo9NZ0/iun7HN7ONooMyQt9Rktua281MMfDdgd3ALplKhoWDBa2lIihz3+x91F+ox7VG9N0rkyq59OElPPt6orPXcaq+ByqPGt5wDMXMwozTr5/m9re3sfO3o7mwid9Zybo1a2DkCV7TvMbtjaKkRvDEYA4vO4yqWcXdxXcZFC8W/c+8fqYb+AAwq/9G8QcncXt9dj0+xT4s2LGAUu9S/lr0F7P2zkKbUgNj4c4msfIWvTiaHtN7/Lu+mESCQtuOU2kyxroAHi32AcTlDCdTHk2B/z1MGFEqIX2YyIjQWtlObmgXKZF3KPFvQer0OAW0WYQTx8cfZ8KJCbQ43kEQHq6PJhP0PepDrGEmZ0afwbfYF51vFRKJRzftslfDUgAMafsIScpEeOpDBJUZOvOH4KEGrxi8hz05LPuhyzhemUe+2xZ6G19G09xFYMJBVJ1NdLUsZINXDl028KoSzhsFbCszMW+rpc67J6U3zzDymDcxrcPJ7JKi8Wzh4LSDzNw/E1WbyKV+dWEaP3rf43SmHZr6x2NXvYsDqcoZ6Pws6egvQelqj6nKAp3aAo/cSwD08j7IEYUtbeOvYOayjMah0xESEvFJF2PZnk/1fGJMgkkg4Mi34m27X8hwi3WjIqGCjL0ZWAX5Mf8/a8gdc4uVgeU09tWyY9wOmgqbeCHnBbFYJDzw8QTqs+u7ddZs/OzYMv09bgV0ojTT81vqBE4XRODmkYdDRSq6QUqmBh7AI6ACB9dGThl9MLMxwzbxGOriDC5OG4ZW2R8LQwXjLPZRqNFTn1OPZ2YyWrU9rWk3KQx4KOPRWNBIU0ETnv08UUSr2bxiM3P3LuRa4DWGfDCE/m/25/bG2xScKSBgdAAqmYo/5v2Bq6Urtj62rDauZvIUiQhs/GM3HyZm8OmVT3FxcME+0J6yG2XiuyPAknlLsKizYE/GnsfAoI8W9XKP5ZJ/Mp8pW6cgkYn+YOHZQrL+FguZjUsb+dLzSz7n88fuyYjPRvA/WfXdavJO5nV34j12LwVQdjaT33su4Quf9CUVNmrsK9Oxr8qgxfmBlq2UCx9c4OonVwGxs2/ekXnYBdh1g9IDxwZSnlBF55WLWLTVUgtoWwehdlIjmMCgVKBX6DFJ9EgNOlQdDQjt1oCacd+P4+iKoxx9+iiCSZRz+ed4ALzKvJlwbDJO6R1Uh1dj7WlN0IQgqpKraNqXiat/AwX9TrJlzTD6bRVzCApNGw2JGchGijmfCs8KdJEhyGUukAZ21Vm0XDZAAKi0Koad1FEtSYBVE0nZkiIWFKMisU5Nw6A2J7ffYiLDxHl0qEjDpk6UL7CrysQyajKkPZxng0pNu50n1l01j43nAc0qQKHVHcpdzjB/lwXFl0JxHRPTva0ppRL5eDktNi188v4nLHD5FolSSWXwYKxvFmF/UMYuv2k4OJpo8wtCYf2Qutkl8xJa+xYGDrlH3MgkhuS58nTDe+xcfoiG3AYm/zWbP6YeJEsHf5n1QhpgjUWAuAaXOIs+bKcAepS4VNow4mg/9D07xSJnSxUyow75yCEsX/fvnamWrpZkH85GZa3Cb5gfbj3dcPtB7AxXjXJDe7aK11WdfOm4GXgJubYdo1yF/Z2TWOXa4jCqgSObJ+NYJgJgLqQ64mvlhHlbHT2v+zHNspIli3vyn5R6co7kcPE/F5HKpQguCrTScvb0uo37rAhWHBcBZfpOPV1NXdhKDYy2gKuKLiQmCZhEMINjD0eKLxazZYCob7zk4pInfAuD1kBbZRvmdubd97A2o5b6rHr8R/k/AS4FePX110EiYfBbb9FfULDCpMEFqAa2m5lxWa/ni88+Y+Kk13jtkabC/xb7/vftv5p9/7X/NRuyWmxVPvncCXIibbnR/wY9k3sir9IjaHWEX/qRwISdGBSQ61jDs6oq3rQvRmIu44s12u5CX0vPTlZPm8OEZSe4+9nXSOVydk7YSc7RHKRyKfqcZvLvBVKkBxBQ2ZgRODYQ/5H+KPys2D9jP+vt6/EOKUU96hZKO3Pu/XmPvdP3oj1/jZCchx8AE2Ji4Pjzx7m+7jr3tt/j6NNHqbhd8eQAPc25OPwi96LvUeHUik4HVh5WLL6wmJVJK0mbBOdHnCcvKI8y72YEAWx9bO+3VEvIi7dk3PDnaFgaj3rAsm6k5NEVRzHmdFD06xySMjzJtZvI9cZPqUiooOpOFU0FTUjKy0jMHsm0op8xr9BSgDlSuZRXy18leEooU7bnMiOhH8usJdia5AgmAUsXS8JmhnHxg4t8YfMFG/02UpHw5LgWL1zMopcX4W7ujhw5Bo2BU6+cIm1XGo0FjaxZs4YDiSPx9c9HYmuDpYsl/iP9mX9c5HB++renmd3qwsej08FOScJ3CVz97CqWLpYUD2mG57ZzUys6k4IgEL1YTLxXJVZh12RHSmwKn/QYyDPmM2nLr8HC0YKx347F5KvATGOGVqlisE8qHyzfhNL1/5/CUzwPKPWOnB95nqqQKZhMYOVuReT8SOafmC9SPWas5txfrzB/s4munHbebnqbZ1KfIWxmGGf6FXJnUgFzBxxhVO8uVGqxc+/ppKexC3Sgd2JPUq+J40hyWEb+yXzW2a3jW+9vERIb6H+jPw71DiTGJ3J+nBhc9nutH6tSVlESUEtusCg4PnWvD7Lt6Y9d+xs1b7AyeSVuPd2eGFdrXg2xpz4n+Naf1KWJTmZDbgOZ+zLprO/EokOGXZMdRrkRk0xHS2kLKb+l4NHbA12blsIPvmHQ3LP8uXYRLRfTSPghgQ1eG6hNr0WpU9JS70SwWTV/jB6FudSKvJN5NBeLtIGtGd8TkvEDyR4pzDgwA3WiGHBX36tmo8vn3O5zmxMTTqA10+LW9ioS6cPPkaZFw9VPr3ZrZ/3Tsj3eIf52PC9snEf1vWokUgk5h3NI25mGIEClnwc5ITmYJCJSN3JeJDN2z0DtosY8Tc/kw5PJDMtk27JtyC2VWDha4N7L/b6DI7ChCYrSAgm+HY/pEUSXSqti+3c/kpsczMAyP5beGExHQ2f3dplEhmW7GDCYac2ILZnw2HXf/eMu3/p+S2VCJZGRkUilj3+C35hfyclx37P31ZtPjNlt/FDOzu6PQq/At1AsxnXrN0ggZ5wFG185SEzet4S1ieeNeyaOEZ+O6B6DxCTBoaqOhsJMtG1azr51Fm2btnu7fdtAYu8OJKL4cerA86+c59sXv+3+/Z+df7lHc7nx5Y1/7cR0Dnem16pelN0o69ZKeNRc75zEK+ss6UOew3uoWCD/SPIRH0k+AsAoM5IVmsX3L3yPIsqaltIWKhIqRA2Heg1+hX5YtVrhUu2CoU3HlU+ukLw5mdinY8ke0JMZ+d9hKTdh5tCC0VxKz6d7kzb0BUxmQdi3D8Yg0ZDQJ4nUEa9g6evU7ZiatdXR4hxE6rCXHitU6rv0VCaI+m3nR5znUq/FmGQSQqeH4jPYp1sfz6OwGP+ad2n2XYXCUsX17Q+plNsOXudqTh/Kq+Q4FScgSKRoLMR7Gjgtgu1L9rEjJoFdO56i7aw/YTPDWHJpCW81voXT532odqnGIDdi3bATy4xt2EgdaXP0o8A/lTz3T1gV+gvhDTMY2OMgGkc3lPf1JgAMkjYkxi502ifvhSAImEth3qRbWDy1F6QSrOsfvoOxxz3xPteHwWYiqONBd9ujtja0kc9n3KbRz4TJBM1mrvR6QcQBXrnQi+Y6G06+8yLeV2y6gQr5J/NRNphz+a1Xae0cyA8Jv2LX+Ov9awIEI8WdckwmCQ3V9thseB77O85IJBLqMkV6YpW1ivnH56OyEsc6afMkXqsUI45OZwMJfRJI6QsKb3f+ad55tkRXBZKoEZHcVz+7Stn1MrQtWjrWZ/H8T8/T644YtBtNRrIPZvP3vL8pvVZKubeEa/2vc9LLi5KIcahdrXm3/V2UlkoqblcwZ890vjs2kyu9t/PH2u+JGCp2hgRPCKbDooMRp6wZnziMHa0WxKvA+Mg7ZHk/Ge/Z1/OJawZIj8iixKcEB614bRKZBKWVErm5HKFVR0x6BIe2TsDj1AFaSpppdQwgr/c8lN6unBtXz87XN9KwfzQrfl1A5Z1K9kzdw7XPrrGz7/d0lCvZmTuGLUu2cXFiAe5x7kzdNpUXcl5A56Jg5PmRrG9xIWFqOk3jbiGVS8n6O4vEHxO5/nQE38T15aWmF9GfHEL/k/fZEX5N4tCiQ/i6+GLpZPmvYyrT36NFnYx5lzmWGjFh3GNaDzRNGgrPFyKttqG9yIN2E5gkxscQ4ZOvL+PbtWtwlLYRmxKLdbkcv2F+NOQ0cGP9DaSCnvWOcCa0Ht/4dMptivHMOkvE5Z8o3nEd7q9rY23/ZMvCl6l038/o0eKxTXIR6OXZ8i6hOaG8+dWbkF6JpknTff5vfL8n8YdEym6IWrOh00Pp97pYUHeP+pqykGhcaryw7BTHNXL9SJ5JfQajYKR3Um9UWhVbninAcUoMCgsFwZOCsQ+wx6wSHBsceVFuQYdlB0gl6JUaKt0r0SfXEnvbnz63+pAUMJXEwMm06VoZ8dkIxv84HvOe4rpiU+TOuoIG/kraikKqRJNVhENFGjpBwi8tUGa056LDWC73jUHhrkZv50z6kGep9utLacQ4Gj2jnrhXdlFeHJlyhIyIDKo9WjCZIG5VHAtPL0TtrKb65D6CTxzi2dE3CZvRiAQpbfaiVmBnTRty3xqef2M3quhs8kMqMBU34V718P28+F1vXtn8Nl82QZVeTP5c++wa9Vn1SHxdaTQPYoi5A7V+sMC2GZMJnAtvEZiwk6hzxSjSzuBsvEyJrTWzNCCRy9Ba2OGZfQF9q7iGtVW0dd+r+cfn4xjiiD4rj2O/T2L1OytxSRbXKO8B3rxW8RofCh9ydXg2ITkhfP3G19xZ0EnWgSy2DtqKySh+POq8evLqsFx2yuyRdJgoPCuu/SM+H4EmQEmtcy2eZZ6s/mg1yiP1bI7/jTOrrzL+h/EcnnGB8yPOk2Oy569W0AlSDDUNeJY5IDVKud8UTG6FM+u3jYS8dhoLGsk6mIW+XYNC245c24FBJ16L3wg/0nak3V+rajHvMmfF5hXU3CoieGIwQz8aytC1QylrKyExaDId+s/xv3uQlqI6Rn01itn7Z3N7423cbyiwdmjFNaiUrL63MSrAK/M0MpOBxnvluKdW0hC2iVZBhkQQn++Ouo5u8I9rjSsJCT2407SAny4nYK2wh7PnGHlUnN9nJX0IMOymosdwUse8gZ2/HY0ekTS5hXF69Glcq13xyH+SFs0+2hN1jIKYWXmiFpFCgc9gHwrPFFKfJVICOtU7EVQ0Go8KP+Rmcqw9rUn8KbH7GJt7needr9Yy6IxYSDn3zjm2DNiCSWtkrNESvU4snIy1fJsDXxfyjvNWLmwpRhPmRdfYTH7S9uH4oCvc6Fkm0iu31dLiFEj17MN8plbh6tDA0Bd309WzAX//QGQmPVaNJbS9+QkX/nOBikQxznIKc2Le0XmM+2Ec7WNUnBl1hqd/e5qSPwrQG2Xk9VlIQdxszG2VrPxuGYYDo8k4NAz/VFc6zNvRKBsxb6/H0tcBe+Vh9GZ25PTIQWKjZFXyKhLjHo45ol8ar8ccwdxGh9Takr2zDvDzMz/jON0bJBAaUk6YTRJLP9qPzzwxVs/weplWdQrTg7Ywy3YgmW0TqNSFgkJN1KIoFpxawNzDc7n97W3yT4ndi2onNc6Rzt2/RxZv4uLwi5ybIXaNhs0IY/AHg6lKqcJQbODWD3ORJEWxQb0Ce7k9uUfF2KepUNSp7gpw56AbjLKA1C2porYccG/7PaqSq8hxEY9rNBkpPFeI0lLJiM9H4L66NwUvXcTJs46iDF8+ekcsOFz5+ArLti3j/IjzJPYW58e51hlTuQhOnH98PnMPz2X0N6PpMaXHv+rAGbUGdOY2ZA58GoKDn9guHDjE2DMLGX/hZ5zytDxSH+HGF5cJzjKn1LsUhcHhsc6jqHMbqFhzlqS4JL578TuSBtmLeYKqLHxSj6Jv7iD4nhMRGRG4Vrsy/eB0TMWlGDR61I1lKLtaaDG/y13fpaTFZVISOQFBEDC1d6LsaEIwGUnyn0mS/0yM96UFNvfezEeSj6jNqL3ve4nvr4CJppw6bGvzMG+vR6cRfb/RMgVDssNQlYuJc4eKVKRGA5IQJWNXHUSuNKBLjMbQ3IVEkGDVZkWrayeS4CBsijfRah4ILkFILO1JHf4ybfaizlVW2ALuWCyk3nkqWfZTkFpbQmcnyq4WSiOcGDbzAvYuTVxtKCRHeh4Xd28EAdzzxM7ygZ+MeSLBDWJxqqKHG9eGtlAc0R9BEMGiE36eQNTCKCztHWhxCSYlzZe/f5iJprmL9up2GnIbyDmSQ2HK77hk/8qA288w/uJP2AXaM2SN+H4UnszFoS6QBpmJv7OWcqtsGhoNlIWJToTyqo7tUUn8qXBhxZFCLhYvwmSCDp9QKoMG424KpbfUgXjzNGZuXs60XybQXNSMa+FNfNKOg0TCpM2TugEnl9dc5q8xf3Fn0x30XW6gjMYkUWLSi2t/wOgAalJryDuRx+3vb3Nr3i1UmoexwKO+07lRl7gXfY/Xz7zOzbKbKCwULLuyjKcTnmbirxNJ/CERqztW9LnVB32ynh3jd/DXmL9ozG/kzuY7fCT5iPyT+SJQ+P5xC88UknUgi/A5IrVtZFokU32nPhb3alo0pO5IpSb18SLSA6vPrifppyRaSp+k8RQECL26Cf/kvx8QjzyxvcU5CPO2OuKOf0xg4h4E4SFAGsQu6i0DtnDpw0vdf/OI98BnRAAWbbV0WTrR/9i72PmJRVYBCcdmtnN29Fk6zHMwb60h/OomTHdFsISlqyW2vrZIZBKOP3f8sXOBuB4CWHSZE5UxDNsoXxRqBYFjA5l/bD4jPhcLn36FDizYsQDPbQ+/eS3OQXgtHkKfcyre/exdAqreppfZ7O71JGvACvr9tgqf2ufQqrTcGmTCcZjot/V/s7/YxRXag2r/fkiUiu45AqjzjiUvbi71ntHU+fbuPmb3eiSRUO8ZjdPYXmTsy+hed8+8eQaAcT+Mo9m6gSq3KvbN3IfnQE+aCxqwrc7pvv4B6QeQCZYYFAaQgq6mCZlBT61PLxoPZFH7dTS9nNOpWf4RVRZVOJTdJe7YR9gXp9BoITC2zxkcW5vRG+fSWddB3vE8GvMa2dbnF1787kWG3exPkPMdPlj7GmW9KmkpbcEvV4l9sw29VeAobWJkTCZ2z/+FtIcZ0rIS1C2V5PeahXxgX9xin8yhAWw5tYU9U/dw/P3j3X8rOFNA5v5MjA1ahs8+z4he+QSrd97XVL1KcMJOjBaWWFSZk3c3iOnPHWDnspMUXCjGO+Mk5SEjiP1sOgCabH9yU4LwbjlC+CzxXTEZTEhq9CycdRl3fzHfYDCJCK7B/xnMy4Uvc0Gq4lQnrA+6zFMHZvLS5zMBmPDTBBaff6jz3vlIjuqB1WXU8Z3/d6yzW0fm3yJI5/Srp9k3ax9l18v+dR4AXn3tNRKTk0m368/zEiXTkfKqUompf38Sk5N59bXXnuie/W+xD6RS6b/m/P5f2X87+/5r/2umeiQBGZLWTGKUOVOOTMHGWoHWlIp5ez0KbQctDkHEvfoH521hYBmYBBO67btRApXTnqMp4GXqKpzwDCzH+W4KPuO8yNJcQNMsUlcmTZqB4P4HQwu9eLFKbFl/0Lr8+8kk0iPTsTV5srnuOBU29sxwdcZ9XiT6Dj2JP15nNrNZE76GcSfG4VcSDJ+CVCbFPsgec3tzKm5XdCNBH7UHCfHTY09j3zYErRZSNt6kKrkKj3gPxmxQYzb+d86MWYZD6zC0rVp2T9qJ3wg/wp8fwvNnxhHv1Uie7wUyDePpMbUHgWMD0bZqCY7tR+e8NTTZyJj/dW8Ke9cz4twIXHu6YtQZObomgfDUH0kZKSbBLVEikUiw9rAmc69YLDJrtCPbspnbb36PyuJHLF0tGbl+JJn7xYXfs5/nv9LuAbw98G24n7vWtmm70ZXLMpaTGZrJQM8a4qbdZE1DOG9Pf4XW8tZuXm2Ayx1SblYHYe3uTPxLvgSOC8SoM2LtNpcQWQ5NhWCW7oc2tgOPXg8/wA0ODQQooL3JRNK+SCwsLzJpxVx+7/c7irRaXuVVOv5zkkQ1XNLyfy2GKggwZ9ckGu1rafeVYDKJHSHTd4gf45NfX6N+Uzn+/qXk+fRFYWmGykrVzXWdy20ySSGwcCJ5DSLdh5W7FVbuVuh08MeSXezwaeXKoUH0q+3AZt5DFKPEKOBe5Y5rvT8e7V9gb/6wq6q5pJnQNC+uDCoC4Eb/G4S7Tnns2tXO6v9RiFxmaU6DewQOlenc/fEGkaOm4zfcj+XXl+MR78GIOT2xqgrlr4V/YdHagb5LT1VKFfbB9txZdpiiYXHcmZSAw/QzXHbtYnLYU7hEu6Dv1OM4PJrU7GictT+T92sQdhOLUd7n7g8cG4jDW2b4ZkZgaXOM2a/s5iurpQD8GiMm8UedG8X1gdexaXHHvTr4fuFARDlbulqislFx55c7jN0w9olx9b85mn7XA2hyMCBXyTEZTIz6cpRICSjA0Ql/IujzCMn0pbNvKy5TXDj71lmq71ajqjASmxJLYu9EWmxb0BsN/NpzK7oOHc9lPMfgAmuG6vtROS6ftOAhzH/k46xt06JKK+Rv2XLMnM9heWYgXY1dcP/1F/K19Mjuwe/Lf+epLU9he7v0setO/CGRlpIWsv7OwjHGETOzx9FTYdc2A1BTIM5j+a1yEn9KJP7FeATBA4mgZPiF4di2iMUn11hXatNr6azrJN64Grv81USd/Rq54x46/zOXrANZeMR7oLGyYuTZkVS6VzJ732yqYtq4s+kON768gYWTBaoyLTMvzCQzWkna8DdYuuxxShWT3IRBYcCsy4zg3GA0oZpuXQgQqRDVLup/1eQDkc7u0oeX6DHtyWRJp6svGqMcrdoemZmsu5PY3N4chYclPz8nIvPGpDQR56mm+KRYCHUKdcJh3hWWsIQLwy4w/OJwtMpqLn4nwhGdwp2I25xMR79oZGM6qXjxd9pMw1DZWaC1FBP3UkEMeLRmWnTmNkgVYMgpIOrcEZQaEYmePPadbnrCBwVIgMuDr5EWmYZSp0Rv0GHlZsXSy0vRd+rZPOMwdacycK2UoDCfKOqCxIvi3YN7tlFw5hvcg6/iX+BPffpJNHpf0oc9DwIESg7x9OYFHF/7CaXxoVhLQrH2tO5ei5XhdmxauYn97mbozr2J1SkbGicuJjbWj5xqgWglOCvFQt5TUz9lsvP7WPvbYleRjkmuJChxFyaJiQ0xr/MK97rH017djnJDNf+JjCKpbyrvGat5Xiolp99SLBtLCErajVBbx81jA5CmV6LufBfV5s94o+YN5GZyVNYqpq+2ZqrGjA9bNQiY2LammFt7SvCd0Qtpb2sMiVCW54VRqadTKmfCzxNw7y0m9y/ep3Aq7fqL0iNnsbHJAfxQXzpOXFYK0mP/YXdMLhELwaneD5VmAIJJwH+EPwoLBXGr4lCoFei79CjMFTQXNXP5o8v0fqF3d3dOnvtHFOsmYdBE833w97RXtfNm3ZsM/AqihEOUfPwSbYMMOC92JmJeBEorJcmbktEqtUgECeOPj6cztJn8H0R6Xp/BPvQr2citZnEO63xBZgZKNQz+YDDn3ha7PL1txWCrvDUEwRKKLhZxbNUxVHmNGGQGmhy1bK534OyVYWjdHtJS2/raPtHR96gdmXKSGfvH4ZIv+lJdDV2M+24czhHOXPn6FFd/mYlNZC534uuYY2uOd8ZBLJvKkK18F6FF4IAbnFPKSDHIkSll2PjYdCeSAE4ufgN/ObS1tnf/rfRaKcoaPW5+lUzzaEJrgkajuF5M3zEdo9bIhh/HU+t0nZWqaHzmNLOhXAQPHH9GDJYtvSypTqnGKczpSU2dwiZmH5/NlaFZzAsU10OXaBci5kZg6WLJnUVJJPa4jvMtPw7W67EPsO+eo6ziRr4qC2O0exvvf/o5VQbRz5v02yT0nXrKZu/Hwv15rF7aR8jK/ZzX9sd7s7he2PQMYOqWALy1z9Dnwy2UGcAogNDeiXNRGnqVmpKI8ehVljTbt5EYl0iMw3AsHC2w9rSmtbwVQasju98S3npf9FnSd6dTdaeKYWuHiehpw3tkjIQZi0TQTcKPCYTPCkfl4kDOywn8FXGe3bmr8bAfTltlG/HPx3P3j7vdU/OGuQpFmxMmnYGxJwcTfTecTlM+gwgHwnnRr4WULA1NRbWc2XqLm1/fRGopJ37tjyyx6+KPHBM2eal0lgdjt3IWf1/by9SdydT8uoZtm+6yTvksxlAt9ZoS0LogNRpQN1cgNeoptX2ySP7ApEYpMgMYjQIGrZEjTx3Be5A3xd65yAfdoSWgnYaKKEwmqLa/Sqr/ZbxOBOBUHkrlvEN4T7nI6eoQHG16sVn9I/aN9sw4MEO8p97NrPplFW0uCu4G30VpqaTfG/3INPhjeSOQUI973EgvwlDpLibCa3KwahKTFX6l3hgkKg79NgmXHF/sP3YiZ8ByHEpTiLfOxi/WloDRT1IaK3pGcFK7h2FnnWitEbOI1764hqWrJTFLY4i+4027hTVJvZMwCUacI5zpuaInZ984i8voaJKUk3lf+wybPngJm0l6pu2chlFn5MybZzAr0LG0YCl7Zu8hJyQHL3tXSs+1cadRS9RzbvjEZfKFg56DJltSN8wj3MwJg9kNliUsY91b6/iqqQvfRj8SMsIYdbU3hoAu4j+MJ/75eL6J3UFkilhUacxYAeEeOIQ4EDEvAgsnC05MuYpHgZLxJ8Z3dzs9AF2eup7ExHuxDPds5bc5uxns9SzuvZz5a8xf3fOysMGTxOA89tjmMcDyAxS6TnRm1ig1rWjtWvnd3cS38SEUu62j/FZ5d1ey54oozl3dyyWXCtoyz2DZUIq22QlTdAzFKltmlv3BT8OXQOdtEkKOAgJG9gJSDGaWhKSJdJQmsyf9efsYb2Yt2oKPawGH/L4jPaeeHp4SVDYqwmaGsV51nJPNX/HU1ndwqP0DeIeWshYqkypZdXcVf/41mR7mWagtxyKRe5H5dybX110XDz6zghrdG7jMP01JUzitYa2k/34By+YKDq3LIWjp29yoAQ/VYc7FTMWxS9QE9k85iNSgxUw7EVV0HleC4BmHm8QxmHk6PYLkoZ909dOrWHtZ49Hbo5tNRaaUEdN/Flsat6LSXua5uHg0DzEFCAK0W3RyAy3FY3oTEjaQ7fJXiE8XsCwMoL24gYV9nmGlhZ7lBgskSPA/54+qUUXgt4HsqT3GHL9D+JiUJLq8R4fhSzrVEJUaRUbKTSSChKgBYvLa1zaVbAzUnrrKc7vGsH1xDd8GVQDXGdHvO94wfszY3rbYB9hjH2BPwo8J5J/KJ/9Ufve63P+N/t0aTzG3BVR1Q6iNEP0Zc3tzSq+WUpNWw/NZz3Oln4p7hXqa/i7FN7qKxecWU59dz73t99iruU7soC/pU+/KhXxbVDEPcwgP7OiAo0w8OhHrE9a0rGpB06zBtacrd41dvFglFoI2H++N6h4I6wWiFkexrmkd5X0TWeMkcLgDfnzhRz6KFZ9dWx9bbH1subj6IroOHWO+HvPEObW1zfS4sZXKwEEgf3KdNEweTV2Zkqik80jNkpBI4rCqK8QkV+CzIJAD1+uwbrVm2bZpdIy8DFLxnVRqWjFqRLruRodGOptEmlh1cwVOpckYOgaQE1NDvnsSle6V7Jq7i+nuS9HWNBN6YwuVQYO5G3SCcsc/uKiIIMZHvB/6s5eIupXI7bGLqLb/m/c/eZ9N363lvc73qEwSE8v6Dj0Nt3cSX2LBEM3XWIWGoR5tT3HUJNptPZFZqfjP5+9y17Oa1AIf7IfrqAobgWVtIc4lieAiJyfmHsU90vi47ym2Bs7FMcGRo5OOYhkTyJyildjfy+Xji4fpNNny7HBQaNtpdu2BVWMpvmlHye27mA0tAyiPziO+6wjWt6pxzb3H34tsCO9zhcFAnPM+TtSKWloW+ako7oPYeizti+wRBsXy2+Vk7s9k1PpRbJ4rUpUGVCmAdWQfyibrQBbjvh+HpcSR/Lh57HZ/A7tmGwZZLWXMhjEcf/Y4giBgLC0hPN0R9woD1q15mDvMJGh8EJfXiM+W79UNrCz2JihpFw7G3zH2X4o6W8yP6L3l7Cy0x9HJG6/a7/AuUtFW+y7t3uG0KP3oefhL1DbtKIcYyHCqodHKBp8hPqQOfxnLpjLCZLlsH76dSb9NIvapWNzj3Un9KxVti5Zwi0F41c3g7/kX6TdHFFRXu6h55t4zNBc3s3vxbnRdOvrRD6Obkcb8RnaM24EHYejMrMiLrqfFqgXfF3y5/extIp6LoLO+k4rECo6uEIH1ZnPMGHdqHDobHQoLBYJRIPHnRG59c6t7nuOejevu9vMb6YeZvRmTNk3iRvMN4k7H0fRBEzysQ9BW0cbBhQeBJ9krQMwlABxbeYzXKh7XIBMEqPXrg1VDMR1JmbA87LHtzWlluBY+BM522Ip+2cgvRhI5LxKVtYqfwsV2ZxvvhzmZ5uJmEjaI34CMoc8xxgIeLNmCACaprvt/deY2VIQMw89H9GN17TpGrhtJ4o+JlFwpofhyMSGTHjYMuMe5M/X22/w89wwZPY1MnG/RDSb9JeYX/Eb4UfWKHR1HE8nukY1fdNDDAUkkWHiLYFKNmQbnKjVNv+zFbHg/wAuTXIlcDVJBiVFuJC9Mh2W4WDx3iXTBOdwZ/q6gzrsXLkY97tnXkVr5Iwi+aKyc0Vg50+IaQo/rv1O69g/wX4LJ9JDG06EinZbGdvYfusXgDwbjHOGMQWPALsCO+Ofj0X9qok3VRkZEBta+1pSkVuNYfrf78m20o+mZvxVlezb+A6Jo23eKoLRc0oY+T7hvJ3WXs/i01JFzYbeZaGjHO/0sAO3O/jgU5hORFoHMfAV+NSuw8lKy/MZyrn5ylbwTeTg0OhBR9zBXmN9xl/Jb3kw/OIWzs/dxrXcLh1qvYmdTSc+gRn61lSGtKMOpNJlmlx4kn2vgfIwNI6Y8DkLUd+opGyf6kw9kKUqvlT7mF+3SuuF+NRIvnTlshC5rF6jOQpCJ+YWdXy5A+/XHtNk3o+kwItd3IUHAb1IkTWb9mWerZf0zb9HlU4TDJgdWG1eDBNY89xslEj3hwHvfzEJd9jnwdfd5e7a/TyTLgUqCQkq5a3p47X7D/Xjq5lNIFVLcez35TVI7qwmbGUbm/kwqEioImxHWTZUr/LNa94ht+Ppr3nn7bQYICp4SdLgC1Tod22/coHdsLF+sX8/4CU++p/810Ol0T+T8/l/Zf4t9/7X/NWutfJjUyg7JptG+kb8W/MU7PV+gq7qDGt94an17A1dYaw8JRwexdO8I2oOaaXP0o8vKCcHRCcvLbmxLXM6y1b8zUrUBmeo7QmeGcmP9Dfq82IeTwcNpsWrAfusSLEtd0H2qI29/Jk5hTt1JwEClAXN5G02acAQBXGNcMbMTX8rcIBFZKBEkYqs0dGva5Z/KpyG3ASuPJykETOmNLNmxhIvDLtJub0IiEVv4C88VknNERLvEXq/iRpQKEDDojJReK6X0WinBywbQkhhJTm01CQ09MPMsBny7i5Q3bsAOyWqMxu/IHnYBpXsIXY1d3U5ZZUgzbeH3GGuTTcynS2hw1ou6SPmNhM6J4rfmM1zve54d9zXUl5qLQbSdnx1vNbxFW1Ubtr62KNVK/mk75uyg8nolT+c/ja2Z+D9hs8LI3JeJpaUa46ChbPVK4XR1OwFSsbD4QKMQYM2aNcQlT2HMtoGEvKlCNUnF8WdFBFQP2ThmuE6l+fIA7K4NQreyC7dYN2JXxpK8KZmed3vSI7CSoDGpJK74g8joD2gtb6U2rbb7+B+4ZpBT7cQnSV5o+z+JaPk3EwRQd6gIKOxNYcctBEF0Gm9uuClyrFtU0OpaQVN8KgWmmag8LSg8V0h7dTuhM0LpkeSErHUUJzTuBGdeovadEH7u8RNRC6MY++sUoqquogyaRFZiGFbN1rjGuIr6RQ2drBv3O1csDtDoCeNORGDfKaJ8S6+VUnKlhKy+WgoCxO6aq4Ov4mU7+7Frv7VRdLR7P9f7CW5uC3dbinpOp83Rj4GTxCLiiRdO0FLSwmsVr2ERPJG7bht5astTVPhU4RjiyJgNY7j22TUAXGpcGCk1w3bGDS5Vh+M9yJuaezWk/plK4dg6nh+5jNv59nBjJM2e13nui/EMeGsAEqmEa5MqODL7W07UCxTYZzPmfnPUg3sJ4FnmSVhmGPE3T9JZEwyRtgDIFDLinolD06zh38y/yBOFXsGtUYE49nCkvbqdneN30vv53sSt9qTB+gIxKTFMOTKZ1kHVgDW16bWY25nTNFzOprC1+JT40PfIcvTzdSKlaZuO3GO59EweRV1+CyqXELRNC3hUSsDSxZKO6f15Y+Q0fmmS8LN/Gk+7vdy93VDYyYAbAyjyK2LfzH0MVH3/2HVP3T6V3KO5BE0KIicnh8jISGSyh/esIngoOjNreiwU6Wdub7xN+u50dG06ageFI205yI3+N/C6nwwwdBnQtmkx6oxImxsxb26j1iccJy8LWkpbOLZKpOZclPYqA68PJDk2jcOTD+PmHkfI5BBkChmB4wKRr75FREYExYFFmORKUDw+37IiGa5VrggSgekHp+Mf6f/YdqlCisJc8RilywOry6yjMrGScT+M+1cqz47AGFpswlBoWtG2qfHq79Ud9P19WVwnzbrsseiQIdFJUIX647hAzvUvRcqzUq9SCv0LkQgSpvn2Z/AHgwFR66HDzoMiZX96lowg13CWJdYCmhYtCo2OLjMTVXb7xHkXjiB1nIlBuxuTRIbOzKq72KfQtGHQyropYgH6vtaPrY1/0e9mP4ZfHI5ubQ0mozeCUUBuJke+ypvvA58hOrUfMTclaOpngER0xGV21uisBMp+mUP+sj38vSSXPtavidGkBLxGhpB/II2vUvvTMGItBysWIggrKThXTNqfd6n4M5UVbisI+nQr6pBDNPWaiMzWHMXa7xkhh4JXwEouotJ/aYHTzp/S2fgqASl/0+oo3jepIKXZWP3YfdC165Dd6ULm5ISlFAyI66JRYUaLS8iDpidih90BicCeapG68dHibfQXL5ESWIFtjT/9IsNJrs7HI/c62nVXUbzhSVVFNu8YtQjjwggO6o2VuxWD/zOY7EPZoFaTFzaZKLkM67oC5C3is2RWkInUJCbczxr0xPp688ZP69lTd4Guxi72zdpHr2d6UXqtlCNPHWHGrhlEzI2g+l41OUdyCJkSQvB+a4bdXkOFewXN4+6g7wqjtUy8vwatgYv1q2mW78dDrSNTaiJ0eiih00MpuVpC8qZkkuKS0JhpxIJybQf93+xPe1U7Nt42mEsbODrPEZMAfjdW0d75Ffe2F2DpZslLhS+x8eOdTBh5gj15v3F5mzvRM/LQ2wo05jUCcGrsKcIcjVy6No7GW454LvoHHfj/YIIASAQsOi2QGUX0cO6xXI4+fZQFpxYguJqxd9Zeql2rUUmGo7JX0+IUiNbCDm9BILYomIt5/WkY3kma23xcIl0Y8uEQdk7YiUOYC8fCfuJnZxgrmLOspYPLH1/m0upL3eeXKww4edaxfcdTjEvui+EdQzdq+bkvRgGjeGHHGoiBdyzEYHP016NpKW3h9lf3qdcmBTPvyLzHB9aiJSwrjILA5u4g0cbLhkHvD8I5wpmo4+J96/hlPhGuD+nZqpKruP7eeSpOz+ZWnwy2v7SP2WUdXP38Khl7MrrpXesrnRhRoCJdAwucBLIGrgAk9I+zRbHfCmdpESdbfZC8NRu3UBP6ua14Z5yiPGQoxWHBILRQ51FHvmsGkTafdXcKtZa30ugaSruDL9L78VzcM3G014iFUovrZ/BqMlIWMQ6TSdSySvguAQtHC7wXevFxYBVSqYmRlj9xr+FFbn5zk5tfP0xUlXuXcichDLNDQ7D43pyItB7ITA+/HZOePkxVnj9Wl+Lpmt/Gnfv7mkfZcetOELWj02iuWY93ppq6fCl4xlPsto1qb1uCs3swRJFNQpc7J2+G0RXRit2uAzi2i2h6q6YyZEY9MO2xW1VzPZ/pf0/HudYZ1xpXjC8JZB/KJm1HmqjBO6CZ7WPO8HtFT04XrPz/2Huv+CiOruv3PzOakTTKOeechYSEABFFjgaTccIRjAPOOAKOGOdsjE0GgzE2YHIWIBAIgQTKOec80mhyn4vGwnrw+zvn5nzvxffsG2PN9HRVdVVX7b3XXosnTTDuYA6lIS3014nzfH0nHA6C1vYAPDzmcOnWDwy9PhSJXEppRC5Dx+8nKP9xyhUG6q/Wo+vVETYrjPSEn7ADHIe1sH//GLzrfRC2QPnQhUhMRhK8h/Gku5q/pCvZEnwYd8f+AfR/u+8Qhr49hOTku33ZPX03pUdLGbtuLFesCnhv4XdcKojxhgABAABJREFUngWlzWIF0JnXzwBQeqQUuUHJ4Rk7CSwPxFYOnome9Lf3o+3R0lVUg8n3I0q6LbCdfJHqgHEorBX0d/QTPDmYqyfzqLEopMGzgcLIQp71mkFR6woAVq3U4+HpTLRSy2krA7IOR3ptpUiSoqnoVxLUbM1Fm5fZG1DFH+YaXrS/zCNRPw70QfV3os8jCoW9CGjpKOsgeEowLhEugEBxeDHF4cXsTl5M7ZVaig4UMfSpoWia+hn65yyGzD2HbXI6ZvZiIqX85N3K7l8/W0LDex8D/QgWUnImvYJL9XW8is9h3mnLJ5/Pp2Z0MILAAGhHaibFqDOSMTKDdtdOrG7/SPiVPhpvzkIIG0JnfzixiGNbJX+AXrMDWPZb0lrSSvyZX6n3j6HPWoLGXEPXrC5A9KNubr5J/LJ4Do/6CXiQV378mHC1HHNzAyWHy3EKcaIxuxH/GjnGGUauDDvJkNbH0aq01F+tZ8w7Y3CPc6d11FjedoilOG0sEa5j6anNRK6UM+rNUZx98ywAh4xmWB7zxspURn3YeOybi+gNH8bfoUVziQ5/MxCkegQBmgOGoejvApOWQvPFSGS/ASIAtKqqhsCcAwB8+fyXyIwyXp4vUmznbsvlzOtnMLMwQzCTkzD+JIKjlNQZbuj6DCQe/gAJ0DhtEd8s+5Updj3MaB9Gu80MDL19NHh2EVQhJq+L9noQkDeBlcUhuLxqi8cNDxxqHChbVUbHGxZMaTby8KVhSI970RRajnuTNyMvJ9MfrKZ7jpE/WseQ2TaXmhMVRCUepV/RQadPLVpzLb+qYLENVEgKsE9K5Gz914znWXqbenEMchykwdXf0U/psVICxonv34CiZoIKx9FTlwrrRaaEsevGIpVLsXSyotRpKo3tkYydW4S1h1glY+9vT/CUYI5++D2vufXx6tfjcbqaSFBFEEW7iqi/Vo97vDtXrK9gMDNg3WuNhcmCIY8OIf6ReARBoN1sF0dZi9s7v/Dsgou8N7wYeIfgycFcy7xGnByesIOF1mBXIVYGgkgH3tvUy4X3LojtHx9A6PTB1XsyGyWNwanoLO0wqrXA4CTkb5HvUBJ5mHH23xFhNh2JBAJz/kBj5YT3mAdxWnmIKWdfos/eHnt7O7gTGskf9RRDhWMs/3E5x6YeQ67RIAjQGDqGpqCRRLiY0zEphfYea3ptsigOL8YkUWJmZ01t5CR67b2pcP8UgE6j6FcKAhAUSGOrHINczGDIDeKBX9Ol4YW6FwCw9bJFW3SV0EpHQI/BRo+5kzVtvmL1q9HYi1VgLUkhjayLaiI8agqSIikygxapQYcgCDzcDGACMzABk09MxigzYnHRDm3NTpwAu8i/kBTZ0blgBV5FZ7FtryTkiVhKN93iJbsRyGv7Sf/tAfqfakXrHUStohDLrgxsfllE6+LTpKrepf1aGZ3Fj2KbJ+551dFT2R71MZFzw5m9WQTH7p2zl97G3gFaxcTricgkDQiC+Izrr9aj79PT0t9Al/dPzA1rIKO/ASyk+I/0YWXhSgAqso3sSv2U2OIVBNeLLA12vnaYWZgRNDMC9p1FpvGkJOYUMp09fp3jcKrNEce5xsBz3zyH+vELOJrloa4djqZPewekIsaZ+uVmVOjHcDYkkUBvG+SWcnRKezqU9shdpYSGSwfkLZKeTiLhsQRkChnHvylhyr5a/phzi9xmP+ZHzWfXlF0obBQUHxT9qA/XfsjLn7yMylWFuk1NR1kHHoi+vtoUw9mxp9HL9QgygcufXibzi8yBdXx+6XnaktrY9PAmPP098Y0Qk0i1l2ux87UjYHwAgknAoDFgMpqQyqR4DPHAztcOiUTCzqidLM1cikavwWQyDfi9/xOg/G9T2CgG1uF/2t9rwelsLqq/LsD3g5N9mN09H7V5xdIYPJo4k1jJWXSgiLLjZbzS9graHi02HndjeBc/uoiqtptb455DptegKmpHHeuA0kmJQWMgItcMkzqQiqAK9Ja2NIaMJkgcDs6vOY/UTMpTN57CoDUgV/6HMw3IrS0oSgzhUnQkuUWOvEY7NZdqaM5txsbTBs30u7IbSns73n/vPbLO78ag76b0mpLOBCdaxl1jQs4aNDmFKBKiAbDoaaGnCgxS8cwpSHSDEi2CSYCff8HTMxpJeCqeZRfRucsQBH8ArDtq8Ci9gKK/G/k/dN1NJhF4YNNRjQaYt3ceLnfkNqZ9M42dU3Zy9q2zcJd5E5NgwjXRm/zRT+HQWMj9L/px/bAC75aJxJ3Jw0bbB8mx6G6XEHP+O9yPLmPcgjc5dVtK929z0Yb0UTFkLkikSK0tCTpXwbyD96EMKKev/HNkFm/jMdxH1CL+7CpnXz5OYtp1NH0WPPfLBCyGmHB51YX9c0/S4l3Lq20g6EP4o92GBouzrHMW0EUnUCYLxbnmJu6VmVy6BGl3Yg9NOU0obBQDhQzXkq4xK16UVOmpE1/QCmsFul4dbvmRnBp3FqnEA5MJut1CsW2rpD88CdVII2ldvzPJ0oLqDnAfEURl3H14lF2i+oCa43oLauS9/D7zII4KMXbx95qT/thAFktZ/thm4ujHzVxko6jJqBHlr7Sx/JX/HvOi36Ii9SpHw7uQSCRc/eYqgkkg5fmUe+be32brbcv9e+5npmomMnNxnUxYP4GRr4z813g2iIm+j958kz+NRm5g5EugG7ADZms0PA888sYbNDcD3E34/TfZJxae/FvM7/+U/TfZ91/7X7OiP4sAGLVmLKwDvVzP/nn7kTs7YrR2ojbaFwtVKzN33mbDr28zckYGkcn5WNvPpD5cPHAO3bQOsEUVXMHN9CEUprsR9bQGryQv4h+NR24lJ6bVkf0OAjXTs3k/YwOajn4OPnIQaw9rFCvCefmTlxmy9DQx8idQXX0RVfVsCHQk8YlE3qs+yCVf0VErGBqMU8TnA+1vLWjFxtNmAHn0n2bq0+PW7EbqpVTaXWQIAkz5cgpf+Hwx8B3LXj2pl1KRG70xt1dSGTcbq846WtslfPDip9h32bPyez8qhm+m6NBSsnYUM/nDsQiCLXMiPmVC4FasHSAQLzTdGtxi3Wi+1UzZkFaGpJzjaQfInHuB7zQ+6Pv1fBv2LZGL4zg7VkQtJprMCbkdjSJIT2tBK4eXHyZpZRLBs6OR3XtGASBTm0m3eTdvvfwWyycuJ3p2NJM/n8yw54dx5eNMJjS1sHxMKW9sH0uBvSXVF6sH0b2FysHVSo3O2h6ppQKli5L4R+JFmtI3f2H9rdVYr93GHi8dwzyW82Psj0jld5GwTeVevNDRRaZvKVGO0kEJocr42RxWx+FZ+CVDjo9D88D/92Tfjyt+5ImfnsC5XiMi0HOaOPmiSFFg+jWSOW9spUIPu5tn85SqmHMTfwVETYhhLcHIzX1pnZBOc6t+wNe7tfMW8cuH4afTkWReQcqbpQQWxLPhzn11vTqkWR0ohiswWqiw7qzFurEAQZhL/r58rn19DR+/MNS6WKr9q8W2MhjdfGLVCUCkAAqaOBilLgaEJbT5JqC8cw6OXhRN080mNN0adFEJXJVW02lXgr3+CQCac5tpL2ln2rElMHU3J3ZK8V+xF4MgYGZuRtpHabjFunEi5y/G1ZoR7HOb558p5P7gd5BIJFScqUAik4BEoBOBa3dAcH9Xus7cOJOkNyez0e8jErMTyY/KR2H0Z5byroNs1BsZ9tywQdW//7RTUyrosD5FSIuYTJMr5SQuT8Qj0QNBgLn7F6A17yN35ChGh3uR/m46Jr2JtPVp7D9cjtZCS1z+dKJuyRHUAg+eepDsjdnY+drROmwJnR4dqEpcsdC1YjI5A+Lhq7m/hTGTJ2NlLdBw4wCxbSOwsbqLCBw1Zz7vFWQz9HoRVf5VFIVeB+6KYbhEuOAS4YLRaKThdsM9/WoMFZN4fxcTDnt+GNoeLcnPJfPL9+eYdNSR3Q94MtRXrHYLnBiIe7w7tt62mF/aR1RpAeveWUe06WHWBP+IwkaBTqXD0tGa2+OepdZjCzdD92OujMQpxGkg+dYz3JwfQl7FQheIrOQDLhf4MWfOAwPt8v3Rl4VdC/lx+Y/c9/t9eEUPptNK+yiNUW+OQjAJ9wit99T1UHK4hNBZoffof4KofWPXUkrQjd8pPTqPxIeiBj7Td/YzLHMYTu3uxGV9TsuUhfx0OhCHpnZGTplAy8xLbDe8j0FuoM6njvv91zPuhRSOPXeMPx/8Eyug3O8M2+2VZO1/nr7IGK59nkHc6YtkTE2lw0Z0dIdeH8qEMzF0T2gBf3+KUh8HwL3sEjHnv6Ml7wk8Ez1ZI6yh/GQ5xYeKWbb1AXRyHVlDs5hm9QRZ32Vx/PnjmNtbUGnvjO1oWwT6sFDVYtIbUXY1EHlpE6rDIJurwT28jlLPFsrpob9DRcyZXzDv7+acmJ8lN28Iz875imx5Fgdei+XWJycHxsWz0ZPZ1yL5OL6Qoge/wdrlKRQBXnSq2zmlBv+Mp/ll8Xre3xbB6MIIeqeqKU+Yj1Zpz4mhzxJeFI5XzWC0n2OwI93bvPmg4gwPbp7G6zdj0C7XYdndQdRFsRrXLKaDxPHZeAY28PjtkZgMJi6tv0TZsTKcwpxAL+c3vxJia+OwNNlREu6Lvu8cnnUKTH+0MOGFXdhZKbh+2wMrWcwAHYx3ijfmIyIYEngZN+UP3PfVN3zcHAc8TetDL1Jdp4axtuxwByudM6U9ERhM5sgUMpKeScLKxYpDj4ni6I4hYhKh+EAxUrmUIY8Oofev7agt1bg3udOh0mNuY85TOU9h6WiJlasV+6qicfVtI/LtHzmvfgsQQRTNOc1YvBNBdufXqGxU5EflsyNyIQnjw+8O3PUr7NzzAI7u7cy4KaNjdisnHz4AQNDkIKw8xfdDdt/vuFWlUJRXwFOfvsTI10aS8XEG9iobfvM8x/VxEbyr+5zJQ+6ukZa8Fn6I+QGvZC8ev/r4oOdlNJiYdngcuXG52LmKICDv4d44hjjSXtKOYG1G2ogC3nKERYUdIiLZTIFr9XW0BWHEZvlSXZSG2/NOOKhFmtX+9n7MbcxJfmMCsqMbSajxZ8N3c/BQgvkrg9/Fp7xG0tk0jrpCS2zaSjAZTPTU92DUGVEHNDMtQWQI+KsXuo2iczP8xeGou9S0d7ej6FeQtOJePVNDnBNr164l9tZIDMW3gRgOPnqQzvJOXmp8CWWVIze7A6i47yi1PfPpbenj4CMHKDtWNvAb38Zk8cc1D3rMbVAJRjqa9UxZMpRftdnsU36AhcaC6YenYzFCT7dSpGdSVbQg6xvGvqD30GjewSWgEsHVGzM3R4qGP0KftYTTceKa+dAni2NHpTiPFFHX0UuisYv1ZVtdGha9bXTWWuAebE35yXLy9+aT+GQi/XUnse6TIelqpbN8Aon3+xCQFsD5d86zeO5w1h/azRyvd7mxNQzPR7JJnhkE32aDVtxAHdqceBcl8SGjibV35bOXPuOZb5+hb4iUnOHHKHTOJ9O5CFXUZQ7GPon/WH+qzldh6NAivTyJdREt3B73PVt9nNCH3oeurA3nFkt+nb8PfbD4Hr5SPxzDxSS0S9VogqOwyrk8MKZmunvPUqrKNmJvx3I7+jZdjiJt798UpsNWDePAvizeUUDbb76EFb+KRv8+ZcFlhBeH0zlCi8Nlc+LXv8f6aAmmPm/sVjsTW/IMfjU9BH84gkmu75BiCVXv/MzrdSlMeWUj4z8Yj6ZTg8RCjtxWT7NDFBdHfYJ9jx9r74ATAC6FFWJmZSLIOhkLZQwWPla0lXfjUZpLj3MQvWUyutwssPezB8TADYCdnx29jS2MsoRRlnCsSTy3zNg4g8NPHabg9wIsAd/WJxl1so9+F/GgHDYrDIdAB65uuUHyJR0v2zlxffgZ5juNo7Oyk68Dv8Y1xhWDg4zciBtkxXVToofj/wiEeKXvxrZ9EY8nX2Xm82bYvnWAbwqfItg7gI7mACyMKrSCuJ4WOjcSaQO/yqV013TTUdaB8/2jONR+iQ4vBU94ivM6Y30GNl42xD0Ux4jz8XTYO3M96TqtbSYamxq5vOEy5cfL8fsxkcxlO2jx6KREJwYnto3bhqWjJS81vcSrj+xBe+s21gVhPJQbitG1HzNNL65V1wban+fQxujeS9TVy3CPf5Q1whpqLtWwZdQWPKd4siSmhWdGfMyfikB63CYg3InfFrWN4Im/ypj/oIGZV1sIuzQC9SI1Gjs5zdZfEVweTt5Qe1bNeBMQQZbXvrlG443GgXs/+tVMIgojEeZqabrZRP5v+cit5Dj3mTFduY3jox+lzvQNgvEhGrMbcYlyYeeUnViprMhIyaDV6hyOUltSVqWQsiqF3uZezq45R1nSFbZMucLN0Co+MrWgz/0OlUsg9rZgdvEgU/LOMHFqIcP/WER5fyTCG6IGmmtFJr4F6ahcI/Du7OKd9atpSOlFeBiMMjkyo57Igkguj7wMdwD57kNEnSGx6tJA6qGtZE99EzOJGVqDnr9PVgp7K4RGE0tt4D77r/isZi12jSLw4+TEk0w6JdIGRg/Po6A9Fam1Jb9N+Y2nfhIrmiZ+6I9e/gG/37cLj9SLzO6fysSTd7Sl5GDKHkJhQzS9M8+SdNmDNnUBqie82OklBqCXNMHPna/wkOY4q0tHcEqqZv/TZ8j7QTxPLdi/gIi5EYBImXf1y6tIzaSEzQrjz8V9pJ41w/VWBpBG7rZcjqw4woL9C2gp6WRawE6ujFjPakHPONtLGAoNnFl9huBpwVj/PIS39MFcmXGUbq8qnnN+jrHrxrJr6i4Ek0Cvcy+J2Yk8dvQxJgROAOBjh48HUelp+81RBNWSa6kZAKc5CEHc1olJbVsZHO8K5mJuP8yGTcmbUNWrCJwQiFFnxNLhHxHtO2Zmo6TPzpPg7N8whSqAmEGf/+37FISewEM9H0GAmqgpmGTmGA0CmBppcC6gKu5b/JOCuZP7pt/OHUWAG0KRwIM7H6TLqQZhBphkcpDJEYCY8lHYN42n0v0zBEwYBQNyW0uaA4cPasPY46lE5X6LMP1pJOHh1HeHo1eIWrNbH/6VnyN2YWFnMSg5cXbeLZpM14lo2EKa9zRMJpHe3lzdga7flc0P7CTfAmJzrjAsYhhHbnRTkTAPtZ0HHhk/8uK3LzD6/nSahFSch3iQk2TCtb6Z2NsiI4753ON4ySAff3ramrFtr8QoU+A+25+Srdnk1ThhG9mAzqYXpFAdruPA6Dd4+723aTDKWJfSz6P6WiT6drp7O2geMRt5UR5+ecfQAUoX5UBfll1cRm9jL7Zetsz/bT5RBVFUBXQiCJDwxFAufXmdnJ151PkJBN+6yhuenvT7NFAlmLj+43U0XRpSV6dilAv0y/rJivuefvso6q5asS1VjKcU77tNe1QSYxN3sGjyWV5usqf72la4KmpKGz2kyBpNPOFYQ/C4MtbMPI2l5+s43DiDTelN0sdkkxubwWOmWXi3zyDUQ6wsUqj7MCiUNO+7gMLDeAe4ITJMSZWis2jp6UBxpBb/Kn+Em8B4US9Tr9bzVM5TrD2zFlSw/aHt2NnY8cOwH5i7ay5/LP0DgIiCMNocW/hxxY98O/VbgruDyfwic0CaoU3Thl6mJ2Xq4ASCzwifgeTjwUcPkrMlh9U9qzG3MSf93XSq06uRyCSsVK3kh+U/0OzezHd8N3C9ua05b6jf+FegKIBRa2TyF5Ox9riX9v3vJEJV3GxGjv4XPddgT1p9E5AYDVQNuQtUKjlSMlC9fXv3bYY9O2zQdRFzIrBws6fsaBN2LaXcPHGTIKd5RC2IQtenJzFTCroo2t3vBa/O3jKb9pJ29s3fR/yyeMLvCx/0eUN2Aze+zKXS7k18anzEs9CSThpvinuZql5FwDMtjB81nnMWZ7l85iyWigw+02jEKqoW2NRWRftNOD1xHdr1T7HIOhKOQUjWbnKbzKhd8jNynZz7dltR23oKpk3kj6V/kL8vH2xtcWrIw2g3noKxKwmMViII4Hv7CK7V18U2OPritXI2bL87xga5JbURE0l7yIuoBYOZzMpPlFN+ohzliwr8OvyYv28+t31v4z4hCYvedjxLL3BlBdQ86IHMJMExYiLJCR7ovQOojejCp/A0OaeO4hVdhr4xldjbsRh7dHS7ixWRjn21mII6kNfYMmpYFkcsFmAymuiq7kHVoCJsbhRvXdbR2WHLe+5r8Lo6lG5rEzeNN7kdK55hP+mERfJpxN1qYumxUdivtaHJUomlqgr3ShEw3+l+91ltHCL6vGPXjUWv0JOZkkl8dzwgxtCiF0VTfaGarWO2AhBqp2FfSAWCAGo7D0pSHiTmyk/YJXgx99EjbP/wIQIqPNGtBEV/N3atZVx/u4wwnqH66S0sT8tjY91QMjZkcPq10wx7fhiCixxJq54xOltGP3aYT5rEjHLu9lxu/HSDntfP8Xx0Oh/krGab9XoUd/aZ7I3ZtOa3cuG9C4xdO5aw2WHY+dyNU/1tBgwYlUYsFOKZ2dLB8l/3N4CmpiZWv/YaK4xG5iBGtVYhqtM2AduB94AVWi1ffPYqo8ctwcJCPM/8N9n3v2//1ez7r/2v2d8JoIvrznMt6dpA9VK5NguJBBzrcnFoEoV9TUYZmzV+bHf9jaioBGR6DQlH3x/4rWWLzjFkTA75U/JpUIvaUPGPxGPtZs3MrfM4unUa8Ql5aJ2nYuEkHgB7G3tBIaPdqZ3LJgmqTltsVaV0dndyevVpfk75mYAblgy9LgoD+7Y9huSOu1V0sIjvo77nx7gfed/8fXK25tzbwSQnNry2AXOtObE5ouaOXCkfCCpcfkDL+2+vIbAikMh8d0wm0UmsiZ1BT50Ka40FFg49HJ91kMYwA9cONlHx+00+WXqT4pMXydrgTmlOML95AAhc/fLqAO+5W5UNf/bCrq77yXFdiKfv98jkMlLfSMUzxYchN4YQ3+LOKQdLwv+cifKmFr1aT82lGv5Y+gevBv/JOwE7aci+NyHRvLSFHQ/twO4nOy59dQmT0YRerUfprOT691k0H4KSG6GYnxiLe54ldZl1NN5oJGmlGNh7+N032DE0mw1fP4FlhA/p69KpSq9i9/TdqHPqCBlSwqrAap5TLsHeyhXv4d4DHPRXk6+yfeZBghRanrQDkyDgEuXC2/q3GXnibdq94ynum83h6ALyntmEwvNu1Und1To+8/iMm1tu/ut8DKt/n20Pb+PPhe0IgkihCaKujiAIyABtYaBI5Vbfy9Rvp2Ltbj2gQzbk/a/5aFEG+ue3YO1pw5KjS4hfFo9MIcP/zF72fb0AC6UWnYWe+qx61knWcfbNsxjmelIcVoySLrpnNXDl0bEAJK9M5sHTDyLv7cWzQQwyLti7AJc/BmttTftuGj4jfLDxvBeN01vTgX/OAWxbSgdQ7RKphJytOTTnNov0DMjIHJ5Js7clmm4NPiN9WJ67nMaLNcj963AJq2LzW09ifcGe8pPltOS14BzujHm+noM/ziW/yZ5NMe2YrCQ05TZx+tXT1F+tx7ZNQUCVH0uMNqxZvxqng2IVhkFroPJECefHnOda8jXKQso4PO2nQYLMHWUdfO71OZc+vvSvz6rfsgPHDkd8C3PorunG3NacphtNnH/nPIIAYcXBOHQ60OnuhdzeCqWLEudwZ2QKGdG6UaRmf0mbXxp5Y1Zg7qjEe5g3szfPxjPRE729B0qfPuaqniE6/Xu0mruUHQa9gR21NvzWYs6jYetJczyJyXj3JKO0tSPX/yHCi8OZcmIKLZKcQe3WdGnorunGqP8XgQEAwYRDQx79JWLA1DvFmyVHlhAwLgBDtA0HZh+gx65/4PB04b0L7J2zF4DcmFxOTTiFIBEwSfSY25rzes/rrBHWIJFK0Vo5orUQHSIBE6oGFfvm7xN1DiVgkplQW5Yx6mQXup9zBzWrcUojF0ddRGuhJWBGAM5hzoM+P/XKKb7y/wq9Wn9Pl/zH+fNS40uETg/9137bFV3Fs+Q8TQEp2Ac50V7azpd+X/LH0j/QtaiZenwqNioL8mNUtNq2YabvJyDnAE2XSvHwSCK4+f1//JpA9YVqrn0jBiB/v/8ItcFLiBDO0n7OAUlBI+5J3jT7J6M3v3v8qfOuI2PEVcydrLH5xzLqs/eiKXD4oOq1C+9dIOu7rIHrjsw4gsRBKSa7AG2XBs+qOh7e/jAZIzP4+LX1WLjZ4Z97aOA3bIvMiBh5CxtHESFoMhox77+rSXFz8mucilvC6tcfR792JSezndBaDj6suzW48HCjHV91ibQb9o/O5cZwcWLsjT5Aan0I4S3upGak0tusotMzErW9J4JEYOrxqYTm3UtfZxJM2MvgMb9urH0bQSIh8OYfA5+3SQSs7Xop1YoBLE23hnNvn6P2ci05W3IAmDnjCi998QnC+DSuBr7GT49/yPkx5xHK+mn/ZR6j5GrGqT7DrPIWlzdc5sK7F9gzaw/aU9eJrPwNG9kh+iVG1BJxbIzIkEiteaBnFY4yEPpUPHrGhrO1f2Jua87kzyYP6G1M/2H6AGVJ5IJIUl9PFRs+JogNr23gvXfeo2eYEqmZFPc49wEHaPHO+RQfHs2cRjAJRor/KubEqhPivi6XMOfPOYQXhYOZF2YKcyrPVbLBaQO523Mpa71FSZE/2zvMkAgSpJYygiaLY1t+ohz1lnI+2raRdVGnMX//M9rTxP3fY+l4AFIvjOGD3GRye5aKtC//CHqo28QEy79p5wpGgYSbCQRUBpDr/AYg6mN213RTcbICAYExSmi4HEfYbz701nWhVTrQ7hmNmb01jBmLYpoX+TlxOFZm013TTebxLvxfX4RHih/NO2bz4+Ekrod10BjoSMqqFNYIa3hT8yYOKxfjWmyLa6stsx7/k+4H7kNhreCPpX/wc/LPbF50g7Ex6YxpeoRdXyxk4bcjxDbfCRyFLw9n7q65+I/x/9c5CDDu7FA0F34X53qsG71NvTTfakY4NJRtG2eRb/Ea1vLX6W3sHUj0uab6cGD2AXJsOgj/6im8LjlzqGcs3Wo5W+YdoS9YwMVOzQLBWtTIqzdhoWolMv0Hbi//AWVxDrYtZcRY1uL7zG5aR7cjMVfQ6+SHVnn3xeBW68yQi+VIKlppK2rj+HPHyf/pMh7ll4g+/x1XPharQBwCHfAa5gUC7HzqODse3E7ipUou/bUXM3MzYh+IJfGpRAwaA36n9nLgnAcnJ54kxzuLzelB9CvurnkrtRX+PjY0hI3DzM4KrYWWn578ic7xEsr0Cv7oMqPavJ8Ohy6kFjJm/DSDZZeWEfeYGPSee+Z7PtWbOOZRg9RGgeq3Y9z3ezRG7k65jVG5bHhlA8pwJ3qGTaRk2AOYJFKqYmZQOmzpPc/KId6XjU9u5I+5f3Bi+m0kUim+S0dj9/az/PnIIcK/s6H6qwcJsDAid2nDYBDIGCkG2azyzdAnReER24W6VIa0qgXthVLir4rrvquwk7/ef4w3DiUQUAVX1E5szN3IBscNfB30NXlPm1j7xPf8KfmVhxOrsB1SImov93Vi11xM2IaVHDiajKOZhBf8LpCsMKezsguv4nME3NzPxaUb+cr/KzI+Edszb+881ghriH84Hs+zsG7pWlbvS8G6VUy+R8yNYOGfC5mwYQKdIUnozKUcm3qMkhE9lJ0oI+/XPKIWRmHqEBMLXrGlLDeNw1cRx4kXRECWd4o3ynwdQ1oTcOm2Q/r7VOS3+3Cqy8Wqsw6VkxhEKwsuI83amqUu9YR3LBukebLJFZzNDBh0ZhwpD8VKb0PB7wVsT9tOfWgbJ8eu4XrI7IEzQuobqcgUMlryWxiSHcGIyyOYkL6a0z+6EvtALAobBep2NRJLM477lXMlz48nP1hN+41avEd4EzozFJlcRuCIpcRM9GKW3paAwgiMvTqc63IGaPOKIirpmHwOp0Id5perBtr7tw7O1ONTSckLx9mzjYNDryFzt0R69DBBmd9S6riNfEklyb1LeHdkOVnjzmMf6sDmxzaRlZSF3qwfl+7xyKtEPaPI+ZEorBUETQ5CMcsZQSLQG1xJxogMJAopaR+lMfGTiUTOExFuSRcqsey3REDAwt6CZZeW4T/Wn/ITos/566JMhtedobl2MwDVF6opP1mO86ERyOcfx0xhICm8FqNdHQgCUoMOGQZKNEdRayQUagXqOkLQ9t3VXFM5+dE1JJFA33wm25sTFNRAkKMPIKF06EIAUjJTsOmxoa9HHEO/0X6EzwnHd5QY1NOYa/G98RG3fr4GcjOyp73F9WlvYxPizltnVxNcEslHT79Mw553ia6fSuKNRNqc+7E9FsuaXWtJGHeDkuQHsY70pcf2LpMOgFyvJzq6njMTzqCzk1LtV03msEysVnviUHOLRPaxzX8jZSs20zClB3vZXYo0awkEylvJqgzk1O5JGMolnLrpTL+1C20BiZgMJnS94rnZ1tuWoElB3Nou0oIWBfzMjqVfUbxQZMdxiXQh/tF4ig4UsTHia1q+6eDFKldS5K3ojDq2jd1G8aFijiw/gmOlDfI6T8wD6uhOuE3ViSp2Td0FwMIDC/G74kdcbtyAn95T14O5nTnu8e4IYTbsXraFkPgyTqihWC+ehU+8eIKl383EJMDYIyfY2RLA1W0zke0UM9EJj4vA4lFvjeKR9EcGEhv/aRprZ5HG08Vl0N8Fg4H7N8Qz9ehUmh0O0Wkmstx0ekbT7RZC4d5cOhw6uZR6CYlgNmit27aWI/Ox4btnvuPc2HPU+ouBeblGhWVPMya9Eev924nI+IWg8iBe//B1ZDlFCIJ4Prvtu3Lgt0wSAUEqQzDd1VAySsV5F1kYyaXVh1G3q6k6X0Xml5n0tfahUeqY49lLXORaqszSacupIzr9e0Ky9tDX2kepHqZbwbdThgMCLmVXiMj4BTOdGkEOjo4qtHnBNG11RFfZRbuLkdjbd7VfN7lV8fh9mUQ9swdsLWgKHI7MqCNj3hHWvfkBq8aeZbVPP+se3YTJ3waDRKxY6nHoxNW7mVFRhaT53OTDVZ9hEWaP1t4dg/mds7oExq4dO3AvxyBHfFN9kUglRBVE0W3bTU6SgCDA7/sltHbIyLgswdjUh//lFPZ/O4/fnlmNvFXGhfcvcOb1M7QWtqLoEXBrcgNBR4HPC5gkZnBH/2z4m+OomvIWiybvASBPY4ZRaUPemKcBkDWaWLNrLcFxZexWicwbgiBgpu5BkEjJHHGbDqcOws0zWWU9FY+bmyg/VU7s2a+IO/UpSlULY9fd7dPFjy6yTrKO6gvVZCkz2D/7MxJuJiA9Kc7/2VtmU3GqguyfshEkAmFFYbQ5t9Ht0o1EIiFmSQzN/ncBV3K92A+TYCJoYhDPVz3PysKVxGyJwSgzYqmyxGQwYTKayNmWQ+4O0U/Uq/Vkb8oe8AH+lj0wtzFHr9ajU915F/TYDjCE/NPklvJ/BYKCmIw78cIJ9i/af89nggCB13/DrrUMM/97da1NJrDobceytxXf20dFfXSBgYptgOPPHeenoT8NigsFTwkmdH4swdm/4Vh/m4An0nCNcQXATKng4pRALqVeQmYyR97fTXjGZgyZYqLMZ7gPLpEuFB8qZu+cvffE0FoLWinfeQ3LPjUejR6MzBhJd0038Y/Es6p61UAcR37RDPtTUi6ZBI5rNCwF0oClwHmTkQwBrE/D7Z8P8fm7EVw+5cV241dkOF0DFQgSAaS2SBTiuLrFuRE8JRhNqDt6Mz0/SoaitXYGS/GdorVypNslCKPUDEEiHaTZJwhgVFjSHDQC66jBib6yE2W4xbnhEumCRCJBL9fT7tSO3FaOvlcrAhPuWIXrS9wMeZKSGAu0oeJ9VXfYZmRfNHP8sbXMH1lI3vr3kAfbIDVocS+9SOC5zbQrJdz48H3Wx2Xy033v0Vnaxlf+X7F5xGZOv3CUwDIrpFXxWNv18uIPH9M114St+eCq0RDpFGKsUjB5uhHqF4+g1WKSmdHuGU3hyEcpv7MvgyjDMOXrKYx5ZwwfvPEBC35bQM+q/9hD/wGMWDOkngSlCpMJ/HIPEXP2KxSdLfSfvUlniz3hQwvJHlJAT20XUpOBkmEPIL3DFhS97z6W20GisnGgqu7qV1eRtOqx2vA5P07I5wk7cJGJMbRhzw6j4a0GTlpcJKvXhjfj15NRN4QJR4diMppYcngJQ58eSn97P8eePUbV+ap71kZrYSvPRDzDzIkzaaoQWX4OPXGIdZJ1AzHNf9rPmzYRCOwGLgDH78zFv+fkiTt/3w34I6G25ud7fuO/9r9n/63s+6/9r9mcnXP4JlCszEnOSmbn0p289d5bGOYWo/VwGKA6afUZgo2nhglhMs42NCIIHrhWXkVqMtI7eip109cR5S0Gx/+Q1rCk/xUurzxK8JIk/Eb5UTr+CvcHtdFvsCTaJR255RgevfwoRq2R20YbtoztJMRtCjmeCylNvMzk4JcwlCmx97PHL7MXozqUzOGZODd0YaPPApK4/v31gX4YdcZ/LXsOUiYQW/UzGeMzcetKRBBEkfGAtACKDxaTuN+M7MecuDbOH9/Wp9D3G7BrKgcJnJ29h+d5iZWffMMXaTc52ZxGtdlINFa3UdzOpmayLW4ezYCE5s+WEeJjQfQr0dh42dBW1EbvaS3SzkUcfnQSvSXLCAwEmQLSPkjjzyeOMPvQbJqG3MI66RApL25nr9XTeA715NmSZ/km5Buc6kXHrL99+D39amwUT2u7l+xm6wNb0al0fBv2LaEzQ5m8cRonnjpKXbkXKz7+jvtbZ5LweAJuMW4DKFy9VkFduReO7h3c7DnBKGcllo6WdFV2AWD91B50/eYoNN0IJhMBaQHUXam7M09SkLgdZExcOEffX4uwqAvJUgkXP75IxlvniLJ2RhL7BLf6fqC1W4+r/d0DoEQqwcZTpMX4TzOZwL8U9Bpv9HYGBAFCpoXwWtdryC3lHP1xN5sOjSWmw5Xkm5FoO/pJXjmS5JXJnHjpBLtOHGKvvpfLAniagckkEDI1hMC0QN43F5MRoUkFdDQ5kqqSDlDI3d51GzNgGcvwePooT45ehXdhABJJBU6hTpjbmSNIZeTGiYdql1YX5JLBCZOkp5NIevreCgkAXZca57pcnOtyKdyQQGrqTHxH+TL5y8k4BDpQd/UpxqkcOXSfK1aaULqru9k3bx8pL6Rw84tMul3G8qLgydj+bvp10JLfwu1dt0lZlYI6xoxji37lbe92RmhsOCYxkfWtmAQJvy+cuM9d8S9exss/bGB/UANuXs8A8EPMD3SUdjCWsZwfd36grf9E//yNLio5VML498bf0y+z/iaG3BzC0Gxnuqq7sPO1I+L+CFGnU4BPXvkOo6yHMflvIwiQtCIJWy9b2kvasc3QMuGvLq6nedLh1IcgdeLyp5eRyCQMf2E4eYqX8fNLx9JaTatNCwZeouhgJVauVqis+oh4/0UOj7yC0s4CzdEyOld0YDdUTH7penVITTIKwwuJKIog9c8g+Ecuau+cvVSdr2L4S8PxePheMejQzB3YtlfRaRUDbw4OKpi8LMkhh6W7liNv/Ro+fA7/cf5oe7So29VUhNVRaJ7BzEMzUSoc4d271+p1BjTCNbSSfJ7/8nm6I6HAvICC3wvoqu5i9OIH0J2cSpn/CS6M3oa7fcSge3ekdnCjTKRenbN3Dh+lfUSy110ONKmZFGsP63/V7JPJZeRuz+X0a6d5/NrjeCUNrgq0aKnFsreN/NErcImR0pheTHdNN7d338Yv1ZWdS3fS6tJKt303wz3H4dCQL97ToMew/DcWSKRcHf0egcVaFE8a2fruVkDkpB91MZkO2y6s/MqxXeSCLGoEvhNsqY0ORWtxcaANVQFV1Pq2sN7VFumlFtzLSvAuEuHV2dPews5HdF4PPX6ImkuiDmOLTzy/Lv5AfDaCieDJwazuWc36d/XwqcirPzJjJOljr4h0M8GpSAQTcdO84MNvOeXpRkSVO2uPp2LcYiJvzNNEp4v6Eb45m8iYoKTPrwIrF1us3UKpGx+CTK9hwdIKvvlzDxOjffFsbMSUmYrGtQuRYVrASQrD7BrACEdmdjElrILlSc/DRpCYjCRmJ1IUVsStpMJBz0HboyW6Kp6nGp5Bl/YtZ4dksNJiG82BKfjkH0dm1GN/y4kvnnuRvvuPEtfhi9xSTvJzyQNO/b5L9SRrY2mWdVLHJfplpSi0Ciw0InKwqSiAhRfGkLBzBLoxBTy2Po0dE3YQfl84Bb8XcO3kMC6OMVJYpcFRJzqV1t31uFXUUnzEkZtPxOMc1Mz0w8mUj+kCQCKTkPpGKu5x7kQtuFsVGjo9lHqXerqquhhn9Qxnqk/QYn8Yo2BEMAnUX6unt6mXoMlByBb+xUoPNdVXU7AKgMYuca+KmBtB4dsF+OBDi6c1Ye1v4jrdjvS1u+nv6MfSyZLMtFJupm0duO8Tbu/ywPEH+Dr4azrLOwHwabyIpdzA6gAVD7RDZ2UnXywq4u8V/le/B5vdXsIh8Ata2rcA4r51qc6fm5Ne5Y0370U9S8ykfLT6I0JKQwgoueuAP3zuYSwdLMnbdo7Sb16l3KofW5Ulmj4N1p21mGRyLPzdaZa4s2LIZA78OIGGnADq8x+mYcdZGoCi18Tf2j7sEpGtWbja312zZuZmGNu7cWzIJ1W6lQD/Knb0KIDNxCyJEWkr1WKSLlzRibmbmk7BAcEk8K7s3YFnFq+Nv6dPAEK3loCKAE5POI2flUjB5TXMi+s/XKerugtpwlyeDHmcZ4LTeeHEQrocutj20DbUSjXHV+VQfeIx4gUFp8PHExLlRUY9KKxdMHZ3E7Ma5scuYPqyI0h+3MCTfe8Q8cpPyEwGXOeOpLLDQLjhOKnW1xmrhHkqNSbTnWiHcDcKa+rsw7GxAFNnOPJ/VHZ7FZ+jxW8oXqNFtofOik5MBtOd4JdAl30X++btY/rQFej6dPiM8CH8vnByqjqpSDuPs18TuX5FRPrMpeYPUAxdTOy5rwHwj6jk2SF/8gJyVPoXSLz9NPFXtfjUKYlgFgbZNC68+Rk9febo1TqktlacfOkknbW9WMfLCI8+y6PtVmgELTMMRixGJHDI/zukgE+RB5uSS9it9UBv1QsyURuqxyWI0uSlCNJ/p53Jfm0fizQvcmyuHsfeURiNsPpdJVadHURUiXPfr8qf5S8e4E+ZCqPpe/TWInhO0S2DrHwe+f4rNuZ9yeXGB7ByOUejpxYk1nQeb8Wh1YdTwUUMzUpGae5IQ0kD8jsc02rLXnTyFtxk7qx1gv5+kT0jMEfUpQQnph2bhvVzdUgLrlCdb43tEy+SP/opvAtPY6EW29dV3XNPv/qdBCr9qwg4MIX4GBFdLpgE/Mf6E35fOMfWfMbYNhfyRp8nPsyKvpZiqi9Wc/XLq9h8MJRj8hn8JnfgyGYvekf5EbNEimeSJxc/uIjEJOBcO5vm2iyunRyGGSJ4pcVvKDUx00lfvIAI61bK7L/k449ewWDqR9d8jMC8S5x86lPkRnugi13ZW9F+W4bvU/YErPBj2vfT2H3sPG8feZsq/yq649ogxhmpmZSaizV0lndyY6Y3FnU9pJ7TUj6kDws7C1Z3rQYJHMu5yafOEOXfx6+eDUgtzUj7II2K0xXsmLSDtlMVpDxym+Zx2XwZe4G3XS7gXnE3aBpeGECKUsHhCWc4PeQyq6peZGPCRqRmUiRmUgSDiU+6zXip3JMKUwNzBAH0ehA6KfV/h2JvK+wNfZhHw2WrcsztzemQ54Er3Ei4wdTjbpT/UAuz5uIY5MjrqtcByPqmjqTwT1kbV0VAJUjvaDifeuUUPiN9UCZGUNjTzbhz43Br9YEPRGT85Q2XRd/CfhOeUimBRychjeqlo6yDnVN2Yug3YLMsiCEXH6Nq4RkqPetp11qQlLUH+5YSeiKHcWphDz+P+YlF0i/pjX4e5Z1CosBskR7cvu0Wscp85O5PUjnSiLnnUCSSXqy6xf2l1qeWlz5/idset0l7OQ25pZyFf4gBx8lfrOJK3WZe+vwlik7fJnBR8sA61PdrkWQYqDQLozrxJt0Oftj7jaCiahf3/zEb40w9w2tBbYLGoPsY2/8ZFhoLdi/ezUTNZIJah/LQfS/i5NENjaBXmrFn0R4s+y0Zd3A6IJC26AwSCexIrWFeay/Skh7SrqVxLfkaPYkqYCvf6GLZ8Fg2ixNeob4mjvqxcbhVXOH3hb+jdFbySusrSM2kuES5oOsVqR2tVSZ0CoFTAcuBbPxG+3Hpo0uUHb9bmV1V6E/spQg6kzsZ8eoIsjbl0lXcQp+Vnqznv+acN1QErKR0/135hur0ajS2GvZP2E/apTQKbhTQ39FPT20PaR+m8WtjBb84VnNbI+HgxlmsyIvBuEo8C8hMCtz1/gQoW3nAtZJ90zsxTxIr6ceuHcvYtWNpym2iKbcJ9zj3e98Xde345x6k1S8Rwe0/PjcJdLv002vdy5Mbn8Recx3hmbtsH6r6Hjwa3bBR2WBb9Tk6h2cBsWI89OpOOq6C9/KZpI/9i9jKhxAEcKu4jHtFJtq5dyUE1BZ91PrU4m6lQF3bRvilHTSOyqRazFGQPvkSMoe/CJCC6XImITfKyB4lBtodOpRIzKQorBRsG7cNEKtMrbrN+cLGiU5DNw91/4ZE9rb4znQNQWlrzdxDL5Ll3ErZpUCs087R4xmOsrUGq656SDDnzdCf2ZsbTW3fKpLc7LE0DePk9OsMjxjJaf1eamxaia6GTiO8GaigLnISGisn/G8fZuahmRydfpa/WVVMgoDX9TaWdCzhp6e/5y0ngQi5F44yCRUGAaNgwqQ34lMgMmBMv7YG+T+KVPL25FGfVU/ah2lsf7aZKvtfCG1ciyDA+QNdaMLGobL3Z3RsJp+9+Bkpt0YTl3M/VpZ2jF07lr+e+IvajFp8L5ox8uoKKv0rcW5z4XqMO0WJSwm/spUrH5yjab6eiW6gPD4GSb8jkjATOqX9QDuGHz5L8hBLdtZOx6HTAW2vjsaUOahU8GjmWZKtAnCYlUf+5RA6VQ04fuYoAg4767DprCX93XQEk0DMkhg6ykQq+Fu7bqGbqme0nZ6rj20nZpgo9SFXykl4MgG5pRy3tW4s7lnMZy9+xsjAkWi6NTTfaqYpKJW6yMmcSLBFrpGQciUFnbcOhol6lbVXaqn5voaFWQs52XmS9954D8+HPNFe1SKTy9D2aDn2zLGB/tkH2A9IHvwzgee5yJOlu5dybta5Qcuj+XYzP8b+SPSiaO7/9f571tbf1c7/k1l31SEVjIOS5H9bf0Mnco0KC3UHVt2N9DgHIAjJ3Lf1Pn5O+Rmj1kjyc8kUHyzG0G8YdO3l98S9rSJhHmMWhuFyxyWu/Os2YbdauBZpQI4EiSCgUHdCf//AtV5JXlg4WKDp1NCY3ThIMy3y/kj63L15P/19ZEYZpSGlvJH8BnJLOQX7Chj2/DBOKIs5+8cpLiIwuObwH+0DdCZIKG3kSRoHKqy2ZDciywQhzYyS1JcZt0T8/shXRzLy1ZFsW/sVJ2Z8gNQoRa5RIWgUCII5zYHDaQ4cjplOjXPNDTqOXwNEH/+fsRltcxefuG4i+Zlkxrwzhqxvs2grauMtzVv88uFlGmwa2PLoFr6Y/gVFh5oIyfp14Fqj7E6BR2QinqpPiD8Yi3dBI9env0Pi6feQaAUuFHrTGt4OEohK/2EABGveYsnVEk8avBqQC6CwscA9yZ2mrCZKDxbiTyFDXbOQSAVsbPtBLdB0uJnX1r/GuXl/siPZhmrJTjxGnSRk+lmK41dg/kM6IVnnyR/9FP22g+fZuHfFd3TNpRrWrl0r/jtc9P1VjSo+9/ocxyBHnBcMwSLwfW5lxJJQ74bwOsj0GhQaFarQBGxKbvDNy8+x4JtN4PA6rbmN+N/6i4ohc5l97U32rx3POPd2vntlJTZOUmJvxSJTyCg7VkaXlSeTbbfQrndhzDEFVu2in+ca7cpvR36jS2PCz0IEpVTmBRJ7MxTBKGDvb8+UL6bQcrsFQ78B35G+98wfbY8Wr1IvvEq9OPrBUR795VGqzlYBUJdZR9issEHf/337dsqNRi7C/zgnhwF/AaOMBuzqthMS+tY98+f/ZvvfoO/82/6b7Puv/a9Z9bnqQf/fb9lPRWAFPo5RGNu7AOjwiMSl9iY+kSpGWR6gr6oPVe1r2LWUorWwRRefjPx4AF+WrGDFul/4wEmH1FZOU2AKfUdqGHGjk98Sc8hTdvH2j/ejLDuK8NUYfIaLYbaaC+DVsZggbxMrW8V2rBQERrw8ApPBRF1mHRdGX8CzYyE2TQdxKbcFkpj46URG94wmb08elacr76FPBFC0y4i7NZxexwXoLWxERNFbZyk/Ker50NhLfE48zcFWCAon1B0qQq7vGfQbJ3ZOoeqhXQiIWlBVsTORmgwEx15jg8UWJjua09johLmjFvd4d3ZO3olEJqHfTY9XvReNEgnB13ZjUWUDzATAxvcOvU9EHmNqXDHre4wpUSJi1s7XjqCVk8n6o46aqCmsmTSYOuHUa6eYsCEK0hq5NOoS1pHWA3zPJX+VUPJXCVlDs2hNyeBrqY5+O5H2JGdrDvl7xSB9SfJSHmi9guMmR1Inqhn2/ATUrWp8Rvjwaf8frLGCs78Pp+5PHb6p7Yx6aQT1mfWiRp6zPzNcvsbHrpvmqAKs3UahVWk595Z4gLTsbWOFhyuj+x7gi5IvUEruvly9krxovNlI/m/5RC+KHtQvQYChGQbCrWZydkovJpOYpOjr7aO/vR9adAScGUvKolN8MvMIw0PuHkonfzaZF/Rf0dzoSHKeEx6NHqhmqkj/9CyeiZ5EPxjP+o71FMZeY8x3jxPV7kD4V+EsPrwYfZ+e4xtu0VzWi5NNKO2N5fjWOQ/QIYoc+DWYa0UKte+e+Y55tp8MartWpUUqk/4rN7xthBc3pr5BeMZmJOZ3Kko3XKavtY+U51Owb7XAut2Lp394Aa1dK3a+UYz/YDx1mWJyVW3rRpz3CZTrdlCsXcawlcPI35NP8aFiDL7gNaSY7+rcKPtmBVazNITPDcfOzw4rVyvKYjroC82gUupG36Rk7IaKQb6/tar+toTsBKLzolGPVEGMmDS3drfGd5QvbUVt9/QJYMrx0QRW+PL9iu95PeF1VA0qKk5XEL1YfK4GeTcRBRGMPLidppjFMCqKA48cwCHQAcWoUdSFj+NS9Hj0cj1d/ZWcekUUgq48XcnoozbIzSex44MP2e9WwBNmEvbeJ1bPLSp7gCspV6jyrqaqZwrekWoMFneTr9d/P8wTPz/Ktoe2IdfLkdgGDGp30JQgqs5X4RTqREzMYKofAGVPMyaJFJeZ4nHq8IrDZP+YTfyj8ZiGi05vq3MFZneoXPVqPdXp1fTU9gyIKgdWBGKwVtBwvYFNSZsISAsg/oNJpB45hdmwOjQWfhjlchwCHXAKdWL8++M5sr2EyMvHmBB5i8S0ayzvuFu511XVhVQQk3hObU4kf5TMhWUXSP7hbrKvp7YHbY/2X9GZ2h4tgiAQOT8SpbPyns9bx9xPYdRcQCKi4kb7sejgIgD2zN5DcuQD7F6wFgDhTsBdq7SHzGwApIIJj44ZOHaew0Znx99KdAlPJlD/Qy33KSVs70vjmFsBCTblmExD7nxjsHcoSEwiwrGmaSDRd+emA/+8+ctd9GefgzeBFYHE3opF/2AH4Ii5jTkSG3POzx7HlK71vOCmo7kwGP0oHZ1e4tyUuQrkLdUR1SjF2bmbU1H5hMulaGxc6LdxwVLVilNzBx/6bmBNTB0X+qyZVFZLTtArzDg4jtKHm4iKmMLrc6dT2BzIX7fi0LeqUWelE9wlQfoPPXdzpZYqOzEZHH/iJ7rcQnGuE6lijyw5P6j/7SXtNKw5ysg5VdhJzLjU7YogCLT5JtDmm8DQw+vE8ZbrCciOx9ThgFwpRzd+KjZOMD4ViqLfxOqKHau0cswdDpFamEhi1rxB91E1T+CXR7/Bx34sH6Q9xhphDYV/FlLwewEXRl2gzWTPzL9motSIqEybAzuw1YrVMsf238eID/ZjemwP/crx9NT1cOzZY0QtisIh0IG9c/aS/FwyAeMCaMlvYefknUz6fBLqPCvSLgdyKzoKWbCO/o5+fhkuUi+tqlnFwfhsYgV7hn65EGOaP6mHU0lZlcLtX29T+EchbV6xOKu98bv2PV15D6GwVuAQ5EDo9FAk1428YA8KCXzcKQI9AJ66+RTXvr3G71t/4pmHj7Gnw4PvK3X4OxmpzagdCEpdS7pGt7uB7GwHKnan4uSWC9PFZN/evYDCks27IXk0/2EStBZaRl8YjcwkRrXy9+WTvi6dmZtm4mETTp3zZUpGn+LPoCIW+72NTXsVyu5GJJJZ7BMW8bC8nmnzjjE60pbHwlZTmrSI4Ot7kQgC5YEVJIc08nHMdH4r3UjFaRf23LcH72HedN9sZtKSk/hHVrGn0I3s8iKMOuOAHo7L21p2eMIUq4PwJDxx+2kEQRgIgAhGgffN3idqYRTz9gyeH7KSTh7e/jB7Fu6hw0d8t0YvjCZsZhgKawVO54xEDi3kp7eexF9/ALl1LJWBlUiNUoxaI9ZvvYLNsHy+em0GezqvYp1fg8rJH/vmYgStgcKsSEoT2/jT7ywhVhKagkdikimIe3AE1StewFYwka1p5ucdE5E4uKLVNTD0yM9URQ0HsciDKyFnyXhczpJIZ+x87PBI8BgAM9VGTcFj7J2gfJ9+gELOrd6BHjNH8qPzmeQppe5KHTsm7mDGxhlIRniTPPc8K+1hihr2IuBaeRWHxgJKhj1A6NWd1Eriqcu6SdSZapoDi5h02Blz7V1whadPK6uvjuHaiRQ6fBv4+nkxAOeWGkzKtD/xdy8l/GgIs36fTP+Ldcjjp3DV9yqeMqgNaSSvexxvW8i43eaKXq3DNuMYnvl3KRqvjfEBHh30rLynxVJ9Ss3wTBeU3Y0YNAbMdFoiLosVUpWPtvP4iG94vw++aYcKk8Cko/60Orfi0iZWwDzb5MhHw54gv2ENDU7v8+vC7/Ct8WX2ocW0O7bTMuoWSz94GZO7OyatCOrr3tbNkOetSOpai+2b2bz+9fPIW7oQnoU2n3g63cNx0f5MYGgNLSRyOn80tg16BDMF/bbulA67S1G95E68/NauW/z1+F/c/+v99Pib0IT9hiHjYbycxEqUr4O/xueORkxO3A06bCsxWvkjNU0h7sE4Ois6SV+bjny0O4oZoNPIqS31wSaqh6gFIyk9WjoQULQrK2BZ+XI6Hy5jydBxFHe6o7tTuf1c9zpWhj5NtYWCS7kXMCgsMcS4YJR2I0gEFrV2cazvWZQOOpoCUwgP8sA9zh33OHeOPn0UkBFUEYS2ow9wJmJOBCFTQ/jjgT8Ymd7K7bErKU1qR23rgValRdOpQemspPtqC73PrMV64WnyntzOkpCPAdgxccfAWOX+HoRntC2SnhgszG0oDh1Lv40rj7/jyR/rlmOTH0ZnzHWKlGKiW9MpUuwHPTKMzV2e+LgtpLf4UVacmo1a1ox+xhxOtpYxxhJ8LUQ0eYMBOk2gauljdPpoanxrBthVil3zgbloujX0NvVi621L33NlZPAIE3at5ak7bB+qehVp69MwMzfj14p89lh/yOSj8ViqLTBoDZiZmw3ojMMEvlPLKXh2ExX6yRT8XoCh34BzhDNtW8oBH6qLfbnwwSMEjG5HZynuSwY3zwF6xnZjFTXuH2IjcUMQHsNM348AtAZWkmHXxghpONcb/En0AH8/V263XKA+MJTzY79DZaNiRtQMALI3ZXNj0w20PVqitNacfEQM5KkautDrTLhUZWPbWo6qejQFqY9z3O1XbkW9QhALGdceiXmLH+baRtTPlzGldS0jZlxit8qIyrmG+/ffj2ejJ73OWvJetWVFw3yeK80g8Hwc2oe7iSyI5L6D92FI1EJkGYJcx1tNE6kwnaYPOdJyFaMujaIgsgCtSYW5FC4ZXajzuYXJ9i4YReUoBhb/rko36o0kPZ00QPP+/NfPUx5YzqFZfyEIAhKJhIC0AAwaAx1eaj63+BCPRg8mX56MplPDxJcmUuc9nDPbjJjFTeaPcBXbtiygPd+HRX/FkPmZSMFW+EchrqWuaGZoOPL2EW713OKZy88wY+MM/nzwTyyAgtBlLH5uH3OCW8nRVYNElNZ4X61isvQmW8aJ7wWTjxcewwfrf22MF2ndlucuxy3WbeDvRp0RVX4N1l31dHhGDzCogAiu6ln9Ea3DVVwcfRHrXmukvfEIAkRe2Ei/tTNxX81mbcXLuDW7EX/DCaNtNViLh7g+O0+suhuIyXWn09oK7pxRe1yCMZpZ4G9pwebXLlKnOIeTWQhX0wKY4xGJYDQg05uQGf8ZXBTnqiCA0N6OdUcNIPonIWUhCJiQyqUkPJFAy+0WXCJcGPtQDN82jgC9nLDYXhyWe3F9hqhpZWFpJPaGLYExLZx3akQqu4DaTaSxdq65wUy/B1n80WgqHH+hM/U4U+wCSTpzE5NUj9JGwuRLgSzofoHbk0/TWeeG8TEdPvnHcarNwdzDisQbiaRMzMbCaKTh1kiEZDXm7d24VAZi3m3LsF4nUiLqWG905MdbXmiS+/A78dNAbwt/OI95mhMxS0Q/a/9isTJs6FNDia//nsjK71H0iTSeVtUFeN84S/6oJxHkUlS2Ki4nlmJusQwLZwiMCcchyAHnMGfa0g2UKi+gVCsxmAl0dYFBoaTX3pvIBHNGH5tMZqcdgYVhKPutEYxX8b12l4J/wu/XUPouZnjOEJIujqJvuQpBsMemrRKXzCQqgeopXXx6/2ZsjF6siVpDXaRIy6vsbmR02x8DDAlpH6ThHO6M32g/bpw9yYP7F2CbdoW/bMRJmL83n4BxAexfvB8ZMo5OPcqLP7+Eb7QPzUHNbB29lbg77Yo6/xJbH9zElBNT0PnruNJzhcwvMvEZ7kN3Vjc3429S7VfN7ejbyL3lLFu1DID6q/VI5VJMehNj1o5h1OujBvo68+eZeCZ7YmFvQUVLBQ17Ghh1dNSgoPff4Ou8PXn/muwzD/S6529/m8kEtya8SPilX2hf+w08/eygzzX17Vio78YbKhLm4SeAe7w78/fNp7+jn/iH45n61dRB15158wwl+25RHzqWHtfBMj29NZ24NPRirjWnzeUiOqU9tya+ROodEpFvQr7Bwt6CRQcWUXuldqC6fKC/SjkKFysMcgMGuQGthRa5pRxNl4ZDjx0icEIguWQxXCYw7H8gAPoC+Aixkuo/Ey9LdVquApPOaagI/RxBeHHQ50M26RH8JnEj4QZxpz9H2zsKEAHVZtperDrrcS/PoL1bCYnJmEziO8Oyuwn/W4fodI3BJdIFKzexgnbsu2OxD7Sn5lIN080/4oZerLQ2CSasve2pC5+AxGTA6GuGt91a3rO05MZnD6EaosXU04dtexWhmTvQf+iJOucr/rLupKLYB2OUng4nPWbaXtQBM/G7do4xF8ZS71ONR70fVstteXnOy9jH2rP65hu03WjB2aMNk1HKnJvOOHpIENyh0aMRqZkSO4sWzHTdXGheyMWaZBKi1RhcvWj2T8K+uYSoCxvv7F3LBo3Xzc2izy8gUPCgKE1QfrIcBJGBirIO6gKW0Se/iEubEyYTNAcNx7GpEJOFJX0OOqw6FeSrfJhg7oLzEB8q42bjUZLOgcQDJL0cgl/4FT4/Ho/eXIy7RM2PImp+FOuk6/hB+gR1TycTfn0bftX+ANTUQJemC4CX22C7O+hGhFPj/ypSuZTe5l6kMinLLgzuyz/Ne5g3H77+ISmZKbwwVtRpnbNzDvl783EKvZeetq6piTH/Mt/+04YBo4HL2sb/l2/+32UymexfY37/p+y/yb7/2v+adZR3EDI7hLEfTuHF+1fiXefN7qW7ecH1AWzMkzjdnYTZHYqYya6fUVvqTdt1C3JKr1GavASpUU/46YM4FrjQDVw8OArd1Qj0W3QYFNZYdTfQVKFGkAi87ggNFmqqbJroKG3jm1E/kfp6Kn1+kXiU5JMWs4dUM0dezHNB1y06qK7RrlRH9dFt102n416OTXLFscefNazBLUY87P8bYuJv671dRdCNgzQOGU+rQwxgT8ITCVScqhApRIFRl0Zx0tmEeV8HcmsbTFIZUtPdHT7TvwyX95/DGCtHMlKF0UxBv6ULpsp+FgtWoOzju5e+wVmYN6BrJxgFCoY3kDJ6N7uc4Sflk5TqndD1TWbP7D14jg5m/Rub0CjqGZV3A7v+IfgqoK+1j8qzlTimhFBR/e/Crpc3iPzXoy6O4tKoSxgFI3JLOQv2L+DGzzfobu4hNqSep911TC62wloiQdujHaDhlCkVtHk4otCaGHd+HPooNZouDbnbcxmzdgxuubB1y2QqQsq4kZRPrLyHX2cdQVUvOr02rZV0ZlpTP9zEiSW/McN7Aq0FrQPt67fsp0BmhTwnh8RjH9L5wOMQf/ewGPdQ3P+IFNu7YCdLdi9hyl9KTJ+JlIvfR32PTqXD+FMgTa9+xxfKXg7SzwopfB30NZ0VnSz8cyFTjgTi0BhHTUwe3teGom5TD1R/+k8IIjC0kRhz2BWbjUVvGBKphNDpoWIVZnYJvf7JHBCeYM/WfibnxWDaaOLoM0e58fMNrs2qp8qxEoDAplcY775qULvX264H4MnsJwcokwZMIsEkk1Mw+ikmPCn+yagzDlDonlyUT638DC99/gESkwELewt0fTpK/ioh5P4YhANXGFLQxuw4mNegRSKToG5To+nWIMXIGW/YIajZPSyTAP/xJD6ZiFFrxKg3ovSczW11OO8VTEJdnsCiO5Igb2repOB0I3/O2Mysvx6k26YGj0YPjNrByLqFfywECf9qrWGulAeeosWtBYWVgt7GXuqv1uOb6oubAH5Vflj1WVHlX0WCnQXlJ8ux87GjMbsR8v8kb04KydeSGXFlBH0zVUz9ZirHnj1G6VFRw0I7JI+CO+ydfwfv7f3tUTgoGLnkBK9Zwvj6IhgGHzi/NtCuDvcuModlYjAzsG/+PqJ5blC7U19LJfW1VARBoKenBxsbm0FaBTmTXwVg6B3q+L/Fyy0dLDE7V8uqI6vYtXQXGc632cpGohdF45noiUOgA3HnHEite5yvVn1FsGEW6+zEBHHlmUoSPpNxJeUK1X7V4PYbc2ZYETpDSuiMUAAUV6ypiZrCC7GfY2sBLjIRpVh7uZbNIzeTTDK2YbYcnXaUeq96XFwHUxWNWTtGTIr/i9Vk1HBm9Rlm/TILhwCHe78gkaDsbsSr+BxVZ1KIvz+IsFlhtBa0Ire1xMowjMiiKcTk2KFT9tIcNJPmoBG49ZZhVZRBSYw3Gltb2vxeJiYecif4E3p1Bxffv4ithZI5Q6o4pm9CY3+R2r54CnZLCMoqpmfE3YqciIIIhl5PQhXTiiwshILUJzAoLLFrLSco+zda8ifgGuXK/H3z2TdfRO032G9kZMZIfGt9MfaqqThdwcUPLyLYxWGu7sc49Tyd6SNYcGwiunm9gCO+tw+jUStw0pqQZIzkvdc+50rwNaYrvsCpNofKuPt45zs39iWso/NEKpkv7eGVFh0FHad48tcIuJPK9Ck8ze/Vm1k6aj7VsXm0+16l7ZPThFt4URt0mB9bMxk7OY6xZ5ei6bPEYDCgtnVDY+1MRUAFdd51mITB683O1w6/ldPJqBuCafcjPFzjgH6lAUwyPEvTB743eckpdBElfFoRx7PVsPV7NTK9hhEn7ekqNCdDG8uoMmfqYnXk+w7W+ZLMDsHb6yrnXGtxlOnRdGkw6oy4RruSOauK7MBsRihGYD3lIpUqb1HbJmQ8RqlAp/t2lJ5dxFgoWDf+No9UJ6Dt0VJ8qBincCeOrDiCplND+Bxx8fztpAVOCOTiviuEFTkSVjSfphn9yMxluEa7YukoahRc14LERcXSt3ZyxjoLMwszVI0q7P3tcRoRRoU6EK2VMw0ho7H0sGfJkSV3506HlqdUoZjJDZy95URfag/rIr4FRJ0YtfQ3nDx0nL6VTNrn02hP0+C32Y9+axcse1txkK/knPer/Nyt4+TEkyz1HE1HB2zaJOqkOTTkIxecgMBBY2kymvBo8CBjZAYGS3E9CiYBQ78Bo86IS1AM5WPnEJv4Jdd7QG8wUh86DpeaG+iqG5m0z4Ffq16l/L0NVAS2IrUwR2bQIREERn27AFsiecEBft8UiqDKoxhP9H16Ks9WIgH2VdszQgJXTyczNzMR3UYdlo5i0jHFqocpd5lvcdBNQSqT8lrHa5SdLGPXZDEg0Ki41yG0CQjjr+lf0OzWjN2dhPyx547R19zHooOL8LMWE1ByhR6VvBiDPhZrlTX3/zmPw+/9DFhxTRODZ7sePZY41t/CtTqb6uhpuCi6qbSOobZqNk6VLSRPC+JsqAjU6rpcgM5oxo1ZB5FqlxB80RlNqAeSEZZ0eEbRb20/0MZDHZ/Q5l1Ll3Ei4MH4D8dz41QH+64HIEhlAyjzqIVR1F6pRdenY9KhZAzSOL5f+T0GoxF7f3uULkourb+ERbgzB61imZlSR8ZX81CO06PVqbHpqKHOTIHLi0s5me9FQ81BHOrccNIZyBxRyZhzQfTZCThFOZE8KZ2LDQ3UjzGS7LVgoK29qm7+eGssGa9F8kHoMa4k3EbqNHYAw9B357/Rkuv0FSZj2j6NPt92LCoHV/6G3qq451mVbr6Iq0lAbeOKUtWCQTcFz5LzABiUNgRsho7mNMzdGkiqdEH3pJGA8sHUSobz4RTKtEhandA69ONV78W8/fOoiPBnyNyvKAxS0fD6zxxqXM8F+xLqdHWU5payskukqDOTmeiU+WClNEMQoM1XzMgmDnufRb4ZHDQ+yjcPrMXa4M/S/jeQ6XSYZHIEmej+/j0OmZ9nYtAYuPL5FZIuypi0NJ7hy37k5RuiPq5bjBvlJ8t5T/4eSYSiX34Vze+5yJ1U8NH8ARCT0Ksn+dW1NDxxEJ8XBHTh4sEn84tM5Eo5FmOSyeoKIUJuRKoVCHYK4tqd3IF72SXaDrfw0ahvcXj5NLnzD9PdM5L46NV8N2EuABYSSPMSGVEki19DGTV5YCw1AWY0CqVsf2g712LfAeCH2B9wCHDAd7Qv1enVdFhn0O2jx0nlSM6WQo4/f5yoBVGU/CaC8XocOhEQg3VnPxQTKYFTw8jNyYdGJd9dXca02kfxSfbmeIgcqcmIwkmGdUEkuxrDOBuQjY0AnkmePHDyAXZO2kn51qv4xczAJ8CKl9zaOSGRIRjFRSJIBKr08FNDJI95VZF7NoJHLsTQmtTK+HPjqQy1o8b3LFeHXSU7MJvveZv83/I5/ORhUl6466e8uj8ZdX4I2pH9XN2cTda3WSisFZjb9tP/ZBUH5lRhpwtnveo9Lrx3gbDZd5Hrufn+/DbsHOFaPSHTQ7B2t8Y53Jlfhv9Cv50rt4JtaRt6ky5PF7psp1MTMx1vb/CvlTKjdwjKYecIkOk43y0m+0pSHsKu4TY29VfY61GJb089dlXHaLPyQzUkFWVXA33WPrS6tnJ86nE2jhATSdpuLQ1ZIu2bHXIe2fIIOXE5pA2bhb5Pj1/eUQBqDtlS5ZbLitHv8YgDDK/QUscx+uJrCCtwwLFV9C/7uq3xvxiPKaaPgsgCPBs9sW4zp/7YGRpkt3nL4MiISh8M7b1E54lgJFODjp+npfN1sTmhLiFMfmcEbTEaemaHssV3CT22PViUwySLN5lpdYCPrc3YrO/CvrEQr5JzmGnVLDq0iLCZ4vj21PbwTcg3jHxtJGkfpnFp5CVSM1J54ctVCJ8IlJ8ppyGrganfTuVw9xXaTtfR4ltHVnIWWclZGLQGtNWtmOmsUNbYcbkjhgKDJ6H+zjgGOfLg6QfZMWEHHaUd/LX2L+iGM2lnGO8+HpcIl4FzAEBjiR9N1R48OOcyp1NzkMm3A9BukUWz5amB72mNyoH3wu7puwf8AhBp+P6Z7FO3qan++hC99t7Yt5RAmTMgJgikRj0IAi61Imj22LRjpDU/jSCAQWGJ0cycfpWG2YdmczP+Jl899xWv2r0Cd5pcOOoJ5rhkwJbTBFasRmulwDhCrLjucQlCagkmMxMmmYmJijX0NCzG0gOUfpA5dTJZ0S8PtNOjxBnH6vMIptGYpkwnx306giwHgLyoPOJrEpCaSZn508yBa0pj6ykKbEOn0KF0jBtUoSFIpbg/rsTKN4/Nphwm2qzC7gqUDV2AQaGkr6wL69xm4mfvpsSxjk7dPDqcujHXmJDtKUGJPU6BdfhV+RN+PR797D7cKq8CEP/LFK5OO0jE1URcgho4c3giGv9+qtKGcMjjce7//X4u5cVg//Yu/GocGP3HNNRpXajcQ9CY26Ho76FiczpmLaEDyb5Jn08if08+5rbmGLoPM/RMNnq5BfAaap8w5BXFWPa2YVTrmdHrxO/hVexTLqHf8DNHDgmkp7vw3hfWtIcY0UZdYJ4tbOqwZHT313gVn8Fc3Unk0qnkZd3EU6tk3Gtb+bPLHueM5fzTzPQaZvdP5aGJXWz26kTu8DSWzVUo62+jMddyZPph4o1P02PXg2AaTN3vVXQWxxRHYh8QASjW7taMfGWk+Dx+N1B1M4y5w/M4ny7AIji/9jx2PnaEzQ6j+GAxN4fcJLxpFmkzQnAMchz02xZqGascR1K08hciJjyD+pqantoe8mvFfaHBs4F673r2z9tPUloSHkPEWIPHEA+GLhfBGBmfZPDL8F944OQDKJ2U/PnAn7Tkt5D0dBKqYhW7luzCzsqOd4V3B/xex2BH7t9zP+rWezWBAT7eIMXkHo6ltnvQ301GE50l7cj7zel19EGu1N5zrWWoDzVRUzDKFLT7iiBPQRABqOffOS/SZz4cf891mk4NVh42NIaMxryvg5yn/8DhlUQSHk/AJckfCReIy43jzIQz91wbMCEABND364mYGzFwJv7b1O1qeqo7kRlkSE1SlGol+n49N34WmXMUNgp6Ttzi1f8h0dcErObfE31/2zDgpBFSC1+h+nwKzBzBjZ9v0FbchkVDH1G9UdwaUUazfzI27qKf5Vh/G/eySyhVLbR7xRC5ejSXRfZxBAGkJiNm+n5kVpY8cv6RgXt5DPHgp4SfuPb1New+fJXA2lmE57vSPK4FpU8Y/TYuhGT9SrFNKk4ymKmEfI05vXoB5dI5lDdJcK7LRWvy5MNJeew6OILI3yehT1Sz8XExHhLW3EdUiy9hxaHMtBIoLgrGZBLo1fXS69OL7UQX9v28G5+QEqJ7LUn44mm6xkDQrjDOXpNS55yDewPMkNlT33gY17wemty3ofdfQWuPPdHnRQ1Jm46agX6tk4jg1vhHxPnx3crvCL7zXvdO8Wba99OQSCQcWXEE78oWckc+wu2YIEwm6Ld1p2jEMty11eQsqGZc9C6cz4+hsvgKFstTMcnMsOxrRwDSNybzxowAbj/8OuGah6k4XUHFmQpSVqWAAFKjAZuGDkpmnCCny4cPeJ+d8w/y5o03yVq7gY+c9fxY/CAZ1c8gs7JEIoHNIzfTWdHJjI0ziJgT8a8gawCduY4LYy7w8TgR2OUz3GegEOY/TaLT8fdTbwJ+Bg4C3YAdMBt4HFHDbxlw3XRX/ua/lX0iXbNKpbon5vd/yv6b7Puv/a/Z9R+uo+nUUHqwlAjC0SnEzdoo3NnlBBPOtTdwbCzgxI6pTH34KJVrPyEq+BgmmQKfgpOY14n0hnGLypELGhokBiKA5sAUmgOGsTBKxsuv3Y9DeB0LHz+GpETKOoWoLXXu7XMEvGOBV8l5Ops6ce+PYPGeifTO7KToQC8teS2UJKlE7mugxa0FleMdTnOjiaMrj1J2vIzwOeFELYi65yWpDZUz/ekDXD3QSmPeBeBN/Mf6D6Cy9AojR6b+xbDMYdipvkFmsYYbU99AYjKxbFwVu446UGzTz0jzLoxmlhivXSfq4gUq42Zh9lotZhbPYff+T7zf+yo1Vq9y8cOTWDpa0t/Rz5j9oRTdYeCcse4nHqoajslgouF6A9b+zghuJsyAV0Le5eb6WNr0I2j1CWP/ov14LxqBhSoOc3UXvU2eWLvfre6LXxZP+pEr7Jy3mVc2vMLVk1dJzkgmYm4E4XPCeVf6LuYNI3AYXcLCj1+iOayPq8qrFP5RiKWTJWpLI5/5jCE4towzQ3K55LqRlrwWhj49FEtHSwIvOlLd7sW7y04Q5l2OxO5T6q/WDySnWiPaGDLmBK82RIHZJQQE7HztmPLVFHZ2XOZbyVqOti7kW5d6HBKa0CnvBrSbbzfjGOxIyNR/lL7cMUGAspAyDtx3AIfuMAQBKs9Wou3W4hLpQpU5fBjdSma7Jellzhj69QNVTHvn7MUOF1LnnKdl2A2+HpbFsyEvMHL1SDLWZ1B1upzI08sYMSedQut3kdmmom5T01bcxpbULQB0uQUyweln9GNy2eOu5m2ZBPch7oTfF476pJ4Im6kcmdfPkFsz0VvU8jciExh45j31Pfck+/R9Wqw629Aq7TAaxecYvTia9HXp1F6uHUAmH50vx69VrPgJmxmGjacNigRvyo9kY+ywZPGvs7EKiEHdqmbe3nm4xblx7MmbrNn/Ov3LdyDcfxyN3URqLtawbdw2pn47FX/NHBRN0B30HVYtP5F9aShLlz6OmbkZ9qGudLq60+J8nMzhmaSPTWeR1/qBdneUdXDqlVPEPRx3j8A1gKXvg5x0+QTbHk+MOiOOwY6kvJBCV3UXggAP7niQisAKdj2wiyXRGyk7dmlAzxKNnsywNCKMEWTps5hm+SiJK4aS8HgCH1h+QIezij3TD/HMlVQ68oMxzDSwRhDRrGWtNfjIIdUSltpAjvZuMhDAFGZBZkomq75aJc6N1+8NkAKYTCYqKiqIiYn519J+6Z2CjdFvjmb0m2JJz+mHf6HPqg+T1IRwJwievjadxpuNPJ33NGZaKVZ9YoRdQMApxImHzjzE0WeO8mfSj5x7+Rw6cx0B2XKkEilGnZGajBpsvWwpdr7B8fE7cOy3J3DDdPwUtvCCSMPiFOZEe3E75lpzeux62PHQDoZNHexmNGY3UnqkFN9UX8xtzQd95hTqxPgPxuOReC9tKYBZRwv2zcVYt1fR2xSNqtvI6uXdxCfISfh8KfrHf2ZMdwQuLTYI7TrMtH3EnP2aVt8E0idDbsAEbNSxJJUdok/rhtSgw1IlAgDavBrZ0AUvKo3krltN5/AuOhJasW8pRmJMGGiDjcoG7zovDH06sLREbX/HQRNM2LZVoukSASCR8yIJnBBIxekKRl7wosuui3P3z+ehwAiab2dSda4KqCIFyO+cwLHh1+i8v5mxDqsJytqDQ3MxmmpwfyiEB0d/gNytl/z6IRg6NATkHhTH8spSjk06yxmnVowHRmBqt8ZJdS896o1rtrxtcqGdZr7wB7tXniDnogx9uw1H2qeTsECP161oZv01i46RnZQMfxiA82FTeeGLF/A0RMGau79n5WqFeqw5p06U80xbNhVlIeg1wwnI+QunhtsAdI6qw2f8NV5vA4OtM+11/cSfFCuN37tTWJzx8GbiGt2pEaZj27GB/Q/9wf3bRbDGRIdveXvEFdZrZTzT1MW+BfuoOFWBU5gTKcX+bPnGyFVFIbMnVvFsQzf9/dDqn4xe1s34Cd/z6p34REw1OOoccYl04S3dW2xJ3YKmU0P8snjiHhKxyvpePTZeNrjFuHF00lGK4g7h1uxGTOAyzG3MWXF7BSBqiD779bPsjCjGMeQ1Ekfao2pUsXXMVnpqexj+5ytkvV9OftiTVI+sZaLdb3RVS7j5y01CpoVgleXKnitL0McWMP1WJLrH7gYdtozagnrUWL6YcxZ7Gw1Zoy/gFjIFO187yhPni/o3l7MwLQZL71YuK0t50MWc776Da9fAtqteDPDmwaCHBRj79Tz101PcjL9J5jhRKzN6YTRHVx4lfW065k/5E+aUzf0aZ24XuqAercZc3YlzXQ6GtlDU7r0EKjtY7QZ7ewV0GiP9Ni5Uxs1iZrg7PVPfZGtyPjn5nhjMCxjx0jwC0wKpvVzLH8W3OOKQx+miILwSbqJ26+JNqzc5/sJx+pr70Du/zx7TLkL9w/jt+5FIjVcBcX8JSAsYoMV5IeWFe+a1m0ciHT7jWLh3KBZSC3gP2oraqMusQ9Ol4faOSl6VPsLOZVuwaFjF8F9bePmzu0HNPjsPLngoaflUjzZ+Jyr/JzFXdyM1GehYHcb528cIqQll2PEqzBytkRj9sG8uofrwPuyw5nyrNUlRyZRNGktQlB1mrrZUJMxDa3aXMs6yzxL/en8E3366qrsoPVqKRWA4ugJ7Qq9s59YPQfitH0n5qXKyvs0iZVUKV1PzsejW8sqGVxCm9+H4gCPh94VzY9MNqOxiJHN5piKboB4lEp1AQ9g4HJoKicjYzJnxRg4Of0+8edJ5vvBZyvlRO2lwC8XLYRQOLWu5dP1ZJDI9RhsJfhEy3DbNpPJsJVWqdmS5UtRlxcTPqKXYsxqZ29t0frKPpR1L2fXAroF+FTk7UpGkYai3Jw3zn8OQW0Bgzp8AnJiScc+zcozzpeNmNb8u2IRPz7vMs1DQ6qGkw0ZCeJ645m8eGUVLRBHjC8MxGgQOzqtn9u93AVgu+eHsKfPHttcO3SN6Jp8Qk1c9jla4N7lRroRfG3+g3yKBqx5vkPZrGpNOTSL3STWnrX9grOPj2MTH02YYKQalpCDRarj+6ztcsPuBqWtr8JCJuqjNV6sZcnIHLX5JuFZn0W/tQuvNWYSGejNr8yyacppwi3Xjq4k/knsxjpiRt8gz2wo8woyNMzjx4gkqTol7+grPfn51KaRCIeo7Ry2IYvLnk/nU7VMAzit62GKznoBOGT6vjaHidAUOQQ50HsvANGYDG6edo6vNjsOdNYAYjLFtE0Fd1Q1hjNTtZ/nkfIbfDMD4DwDg7H8k0aODPmNXZwpum3y48tkVZA/Fc6j3GD7tj/M3Uip4SjB6tZ7U1alc+DQTr8LfOffgToYXpeM+RKSFyr+T6GvziuWt9ib8siZi8NeQ8W0Bgrk1hfGLyBr5DO5mP/Jx9gjOZSror38Sh8Z2gm7sJ1+eQNuIxSidbvPFXwsozA5DvkZO0MQgHj7/MNvGbsP/9mHm3neSmPAScoNL0HsvRbKtBL92PSe84b3GZCaG3oesNwO3Vhcs3ZU0zHwKfUsfiVe7OTf23MB51TVa5CasuViDxEeBUKujuNKdmPJABJ1A5LxI7P3saS1sJWdzDmvXrmX9a+sRZAIKGwVzd8/FMdiR4oPFANi/sAX/TjkSg5g4n3p5Kn7VfixZ/RanLsP6mKchBuY1TEZ7B/cik0HADTssrs1m3PBN5ByejafuLruGBKPI7iBIkLR+RkheHH22OTS2hHJj6hv0WZQMfFdvFN/fKatSBlgmAPyr/Xn3oXe5f+pyBJkZ9WHj0CusYN9hhgSX456m49vvZ+Fn6YpeepHh6eYcmpvDvKjxzNN8hF94NY8OSeeF6N9p7bnrM4SdcyaMiWx76yPSx6bzqnCC4HIxkBn94Gge36AjZsQt5q78nnnR92MZ4EtH7ng6Ev/JyiGhp8iLbT/NR3hcwKajGktVKwa55aBgloW9BS6RLjRkNSCRSjg98TQtri24N4mgy46yDvJ/y0emkNF4sZj9odMYMuYmI2jEaDJy6LFDVOy6facCKY7rxLHxlcOsmeNHZ1MnOyaIlafxj8Yz75t5tEna+OzRz+gOF5MDf2sFAui9nHD3a6LQbCFvLBWDuaVHSwm6LeeIEzzTAt+6gur3Gk6d+JLknFWUHhMTfSNeGUHik4kDFS1/298gW7lWhaxPC3139Y6Mcgts3llFzvkX8KwXKehMiD5pScpDAKjq6rkx5AY3Em7Q6diJoBgcmNUOs+KvripGn/XBuk81qHLQZGJgTZi36/EqPI3gGo4wzBtBMhjMFZcbR+DtdEz6VEwmsc12/fFITHKKw4qZ4D8bfZ9+kMxFUUIt33pdw1kGyytD6KvvIvbUZhRaFZ3PreTP1icYG/An6Z6woUuHS/ElrKvzKUl5kI68IryLzhLvlkhgxXQ0z7RzOi2DZ769C5T6OO0Y+Unt7J17lnbpfRhlcmRGPdfu+4uzcw5wzFGNh0809Y+cZVJgGoZScWxj8sQEXoflTNL9tlGzbDeJgYtpjknGorKA4Ds0ulO/uVuxNfyF4Qx/YThalZaEMyIrSG2wK4IAensXrLvq0TUVY8gxMfSLZzkx9gYudk3UJpWS9cI1bHV9HJmwinGStchabHnW923Se00YO1txaBLfI/p+PTlvfs6ZO6GgPFM/jT4h1IWn4V10hj5Pe1Y+9jnOnm1Y2arJtmxgsY0Cp4LjWLTX8+mrO9Ao6pgsa+dlmRWZajk1l2oIuXoRta0rdq1lSM3u+sM1GTXUX6sn/pF4TMOsCPjyA/Y//yYjg8VnP2fHHI6uPEr8o/G86n0Gk0xBzlCbgeq7NcIaXrf+Fou+dgBCLsJrK2rpmPw4jvc5IpVLUbepqWmqocX27vlHQBBjMhKRAelvKz9eTuONRoQ7OvZalZbW/FaOrhRBCsYHjVz3uo7JZBrk90YvHMy09E9rS88nqKmIZv/kQX/Xq/X8OeF7Is2tKBu6mDqHeysAJRbm2DcXo+xupM/BSwT5CA7suW8PTTkicHKdZB2zNs/CNcoVr2TxN6Z/P52Et6fTHvINgkSGTqlD1ycmL4p+EQHvHY533oUmA46NhZhq7AEfZvwwg9aCVr6P+h7v4d4sOrgIK5e774ybm29y8dXTuD/ujmuLK7MPzaZqUhVeyV6MeHUEOZtz0KhV/E/kpT8D4/j/VmE1CinH9n/DM5+NoPhgMeWnyumP9sEuD/os+6iNnkpIgBj/sm8qRqkSn7FVZz0WXmKV2QAozMGL2+OfJ2ncvfey87Oju7obmVZHcsHXhGdupSO7C3dv0FvY0OERiUZphcoEMqt+TO98TbPpQ4IFaPUbinNdLrIj3bwSFsQ4/x5KEyOJDhmBpFmCVZ8VU/fYI7hYkvrgGbzHnuGHZnPm1K7klQ2vYKW2ouV9Gempl6iRwlRZAPaj+/EetZBg+wjiq7ZQ57wVgBahCJsOC4ZnRmEarkcQxGpGgNrISbT8xxwDSFqZxPre9YQXhbPYYTEAzmHOOIeJ43Mtq43WzVdZE/Mqn7YdRxDArrkUm/ZK7KuvE2puxSspZmwoCKa/QovJBGpbD/qtnbHsbcO6s45J5/RMSISb7SpqLtWQsT6DjPUZ4GpOcXw2E2P28WVIGzOKRGCimacrlZ2V6CUmmg2wPGwHi50PsiL9NwTTJOIeiiP93XQOP3mYw08eZu7uucQsHlxVpm5Tk1ydTLFdMUq5uOfkbMuht6mX1NdS7xkHM3Nz3HU6vkBMNo8DVsEAfex24D1gPRADIJETf+Jj8sc8jSDcK3P1f5v9v8X8/v+2/yb7/mv/azb/9/nsSBMP6tUhrlxKvcSkE5OQDu3C5NfN0CNfDny30r+S+80aqKOP6TIJ5n0dONflYrSywWtpKNOGfYjCQsfYiAzGGx4m8ej7NAcMQyKdgnmnK2pVB70msJGasPW1Z9Rbo+gs66QuTM0Py39AYutErlMrhZYqLDwfJv+nbPJ+zWMi7niHT2bvIpHGT7ijl7Y9bTvV6SIN6dUvr+Ic7nxPsq/epgqn2BycVTL+KgsSS+EdLAmbFUbxoWKqIrspCi9CZpTh0BMqbqoSKRLBQOH6g4RIlTwx50+2xJdQ3v48Sd2BGE5fICD3kHgDjTmt9S4M1e2gumMJBgvDAJUfQPW5pxk7Skq65lu8dVYD+hwXvs7h9eefon70ZSYsPYE6wZwqj9G4RLow+cvJnFh1gmjEA03tlQVEzLmr3zV782ze+mArrYZWSkNKiRoahbZHy2/3/0bI9BB8JvtRewKun04iaUIWuVZv4JXswbDnh5H4VCLfR37P/m/nMe/ZfUxPyeeiVkrutlzKT5QDIEfJC9+IeldP2gFyKaPeHMX5NecBcKuy4rHUK8xrt+X4/sVox4/ExsOG8PvCUbx4Hi8HLyQ+Eqbal6OedYm9jssBMaDRdLOJc2+fwy3O7Z4y9b+H7VbcLez6FBiN4BLlwvgPxhM2O4yzxQKfHP8IQ90xnjk2nr6hXazIW4FgEnjPTAzC7VFsYKNPMtVWPUjkUsa9Ow6nUCcOPSo+rwP5XkROuUlLvx+VZ3v4feHvA/cPuLEd18k7mDGimXcdNEhlUpJWJDFk2RCOx16mxr2UfoVA2I0sehuK4YvHB659ueVlBKPwr3ppqpImIjK20m/lRFviOJgShcloQt+vx2Qw4VZji4JACoNeRWZS0nx7Hnvu28OoN0dxZNJWWnxqKBp3hvAPXsBi/BByt+dy9s2zPHXzKQx2Eiq8a0mz07LBCx5VGbn9q5gUcA5zxvaHY7g1FPDatDV8+tUrdMhF1FTWD1kcffooal85mcMz73kGAP2d/RQdKMLC3oKw2WH3IGFkEjPicuOY9dcsqiZXETRJpMdsyWth6LtQETuc6+G/AWAU9Ix6eQRDlg3h1q5bnN6ZweObHmff/H0URhbygYM5vU29SKQSHr/6OGmbluIsg0SNLdeaXQYQ6MD/w95bB8d1dGu/vwFpZiSNmJlZtsiWWWZmiOM45MRJHE4cdsjhOEwOgwN2wBAzs0ySLJYFlsXM0oxGw7PvH9uRozfvOd+5dc+951Z936pyuaZ3z6h7d/fq7vWs9SwG23Uc+ehm9iRe5Y1APb+eTUAzvR/8Pa73QRhGneNXOtyl6ewbZzn1wimW/b4MooePlSAIuLZWEJn3B7W6ETBvybDnpnFufBP+DUnFSfiUiLpG4arA0Vu8SOTMbqdKuY3g+mA8BBGsCpsahneCN13lXai1aroV3fhW56Iv8KYpRslPU38ial4ULdPbaPH4lff0sLZrJtLrUYF+qX48VCHmWpS8cmMMbH/LXwWiMb72ZC0WowUFw8E+9wh3/NL8OPfmOTJfzsQrfnhUoKz0K7zrLOxYo+ahFSPZ+VYF3r/9TstvYJ2TiFUm40zmWcri83g7+U8cTrchs5oQZHL86wZwaB1LVVQVHZL51Da9i50pnpaoiay8Q8UzF9cT2qNC7qpHMS6fweCpJD85g+/qZ2BwukGtk5ORQ176FZbEv4lwyorUYsG1/Sp2xgHy52zglfFi3y1GCy2XbyRc3z9/P0GGBwEJox4YRevlVkq2iWsg6nIaj4yuISOpFInSDsl1o4z9yFhMP1WwJ+0O/CL03Jk7CscX3WkPHY1PXQ71u4sYUGupDK0m+scJuPW60bk8nIMZT5KeN5mp0Yu4WLiLKs+j6AeccOmVY4kyInF1xSymxsPXqZrw9jf5IEbJnw1KvEK9h9o8raQDjdceYn3/6fRQrD1DefB60uO9kSQFY+ekwM40MPTcLSuQJ9pXUZGewWTjWiT/QoVqlptZ6GLk3vAaplQZcG8PJWTgcWqTiwkr3M2+w2kUR5TR9uLj+I5vw3e8LzXHauiuFI0NXz38JAMfHefd3jo6B51497pKk5oGya7yQ5/Yi0GnYt6r99KYgYh/CaK3fdItScz/cv5QW5b/foMe0qAy0u7bjr1sPomBdw5rs4CAm8rMRBcrP0gq8TcEkPt581Be1YurPyB80Mp9Mb3Mdiph0KKjZGsJZ187i8pDRXWUiWyfP2nzbUOVksPMsPt4WXh5yDM0OmscPlpPHn9qG9wHKzvmAQzR9gHsaD5Gi/dI7nQWaejar/sl6NXeDLgGgP8/gXKJXMaVVFfKYsuw/c35I35FPM6BzhQ1VxJbfYA/T05nRGEsgwv6MTh5cnX0akJHhnHMrRO56xHaP1nC47lJNIXUknBWBH4uPRMMZjuOaGfx26P342Ucx2fBLrgEuxCzMIZdS4q464e7CH7iN9aMr2ZRgw65Qk5naafocDH6YbbW3sLiyBY0tVvx7CzDZpnIqZdOce6tc2xkIxs3bmRg0PKPfgkCSAQpzQHNeCBGAo96YBQNWQ3UnKihzaEdMzparRCKlV6rjS6PLowKIw/9/gbfftVMnPorVIMq+vWdGNReKHQ9eOu66D7TyesRPzE7QUOBKoVrkbcS/s1O3NoqkLs6UeNeT4t/C3Mc97H49tv5rKYIQRgxNFPE/yT4Nngz75fpDKzsRNuiJeeTHELvcwEhAAdNG7oW0RDgl+pH5sZMHDwcqEisA107YbVhuDqJqPW8L+Yx74t57NyURdnzpzEpjHz0+EfcGfg27AatewgqbSeB2Sa6boUv++GF7utUbFKB4IZgYgr80Cp/w3lwgPYRy5FaXQFIuTsVm9VG7QHRKBXa60R0rQ9aWvnEW8Co70GlF/eJV7rhFtcAGu020+MTjk8wCFIxgmT/vGM0BFfR4XPD2HfqpVNcePcCznEB5KXm0RhUwMDAmwjCfPZNvR2ztJUXS18cqv/iqiyOmk5hEx6n0e80sHro2cZXtvDUhTAu9QYwusMf135Xfr35V6Lbp1L53TJ+HFdLYdrXBMiTERAIbrxOE+htQyvX4iXJ47ul3mysT0IQigkq3IdguIh3hwKDowc+VjnZ1kA2N/qgGq2mMzgV1/YKAFQDnVRuucj4lSuGqDBLtpWg6pbS3u3L+w88hXeGCO55J3pz29HbALhl5J08fiqGI6t+xNe0nJpjM9h3zz5WH16N4v4Y9vS9wunkJuaWx/CCQkDXJhqp/8rhafVro63el+82rsVx5ilSThTRGZxGdepyKn0fR+P8DVFWLw51Qb/VDnlNM6M0o8hLy+NZd1Hn/liRwLHfRxI4px1JUjBSuZTAnjAWnv0Mk9IZ01IDoCR0Sii7b99N0i1JXBqTg1u3ig1vbKA8vY2QiZNY/sdy9t69F5PWhAd1rOqJYTCvH1mHjStJN4NgQ5rdR5hEifcIXyRRfRjGxCFzVBJ2Pa968958/MnHZa2CnpAGOswWLCYLLZda+HHyj0NjfaAklq3ulWzq7+ZTfxvy7PMk9fZxZDLcEfEbYQMGwpbDAdUnhCaGYXCTMGA28evNvzL24lj6XPsA0bP+9hO34+TrxDd/HsShZDO3jW3ix+N7uMcXggJt/Dj5Rxy8bgAnYbVhqA1ByBVyNE0ajj5x9LoeFXjPQ4rL489TH6flwM3ZBH0VRNyVOLq/0zBL/zC6fkdy7HTobHJ8GgsIK9rLwMwlXE7vwyH+OLmGERTP2oejMYL1Ari1lGGVynBJruAbOxeqna0kvvopf+rHc2PXleLe7c6MYzOoC60j4o4IJDLJkJNg1dwBDkZ+h01qo7vHhsRFRmvUJBBspKnKIQdsf86i7XICboEm8sddRT96kHr/HKSjF3KhqYouAXrdTaCQMeAssGPZDpbvFPfFyctOsSxIxsuFb6KI8ufThz7FoLCwJ3QVgkQgYoSYQ+/Tp3byQPct2B2xMiXvBFnJM7jH3cZkuwy2W6NwiKxmVOQoKton0xY+DufOarYt/I3Up6ex8O3xqNxVeMZ5Ur7zRqRw8chiikcWI5VLxXzjAhx6WKQdzq0fjcLBwDJfR4y9RtqL2vm7aOOiyYm/zCfZl8lvufE2E1Yk0HC2AftQMV2BVRDvIzarjeV/LCenxgtavmPQzoOqvMUosuuIeDOC/e/uJyVLyoHJ8OcAjFfIqVV3I3MRnRVetg13svlX+Ss/mZ1BS9mkdUQmDD/f2rp6SD4bQrNTAEGNQQR3tQx7XnegjNSCVC6OvYhLnwuCWg+Iejl9/yt07Ie6h/ZRfr8rU0uvEWQF75pLuLWWYZ6+cgjsG+xpIKS6FnO0I6ZeN/xqa/Hx8KHdV3x/ZyedJS8tjydlLyG0tOLSrqHfOwoJUgKbAmnLqcY8aOY1u9ewWWzceeZOxlo3sMRpMQACNrT13dgbxfeiMfdzMTyVmTbQ7s/E1UOO1GpBZjEiATqT2rnn9a+ouhpId0kIKqsNvaqZ2rBaBJkzWrWJfscBfjAbyFEYSHB2omDOBtL3v4JgstFpZ0eDt437gney6zKEu0BV+zEitBEcXbaLR9UKLA6387ZRhzWkm1UuCmw2hoC+uHfvxCXkxhlL16nDYrDg5OPEgbkH6PLswuQ8gbsFkequYuydWOwd8Pa/xNmJZ7F0+CI/nUrMSgMmpRrH/hYGTuYgb7ZgKZbzdvMoLO0utK1o5Ov7vsSj2wOX992xHx/LbrcKDufcw9Xu8YR7QFvkBAIrTiBoNdzq2EDgwHT0WZsZ0KtwWK2mI3YStoFBUvq2EO3kSbz9VaxbVqHqcmFgtA6n7jpcOsX1GDY1jLrTdYRODiXn0xyu/H4FbYsW2yiBuzoFbolox85OBFR9R/rSVd5Fy+UW5h9dikt7FB8/8haxn31OxUPiHlg24R68GvLwc9Mx5mEBS0gIrvauAEx5ZQoWo4UDWw+w7PllHJlwREwTYbGw+c7N2DvZs3jLYs69fW4oTYuTr9PQndQ11HXYXB9/fjz+6f7DyjTNGnI/zyVydiQhE0P+sb7sTOLe6VOXA9wAb+UKOSMenkTxp2fxqzrDtdG3/OO7NrMVjWcYzl21JJ75Ao1nOCy4jah5UdcdM0XZe9dexj4xdgjsAyj8/AJKXQ9dQclM/XYRY0QmVbzHhnOtpJ+r0aKzhsSmJ7xgF1Z5ClzPwq32F8GNpotNVO6pJHXtDcfSgNEBhNyejsZZtHNpZyhwDXXFK86LkEkhRM6K5PNp79CG9h/9ATGa6rF/++SfcjcWXjWKwPaiHxZhMVh4asMZjkR8yoC0Y1jdhsTZNMdOxb2lFIudEptO1EOCMNw2Y9IaOb3xIgEZAUTNiWLr3K301/cz470ZbLEdoTC2hB6XCTywNInawhZCig/g2N+CxNJD3iw4MgjVZrDJbJiranHs66A+YQ4hpw/hdfo2Yh/eTv1YPUovNZPOTmLKaRFdtEQG8Xb/Zo43xiCzSmFAwHFQnGcDLxTzgvwF4heexm/ZOR69bxMHnV6kNb8V/4pyQsI8mRzahdnczKSxFURn7OVE0CbstpTgW1NKaeYDGB3dh+Wm/svB26w3465wZ/SJ0ehb9SKiBRRuKcTQb6B3lo0HJ2/EVO9PQu1uLMZ78KrPxbm7jv7UKXhdOMXGtts4es93FOldmXZ0OQlndyK13bj7hLg2kn55DL7aRYS+lcGhz6px6G7CEiTn3dU78JZJ+Cn/NbzrxDmunDKWbcnjwAzm679xavtUYrIuYTFMJfOlTMKmhvHDRDGYwN7J/h9zo724nbk/zGUuc6lxqWHExhHsvXsvglUgNDOUwDGBw+r7+viwRavlCP8BfSyQjZgsaibgKHNDbjZgZ9AC/wfs+5+Wf1qH/z+Ut956i1GjRqFWq/H29mbx4sVUVlYOq2MwGHjwwQfx8PDAycmJZcuW0d4+/PDX0NDAvHnzcHBwwNvbm6eeegqLZbgR4fTp06SmpqJQKIiMjGTLli3/aM/mzZsJDQ1FqVSSkZFBTk7OP+r8H/nvk78nxA2p6iC0aSLjLo5D1jCAPn84hVBZfBndHt14dXhhNVgIK/wTm0RK9033Y8SZg4eWc0mjYKYD2OykaN2DkVrNmLQGCia+xb7Qx6nLTuCJ/HEIAkx9barID+4go923HZPKjO9gN/eGFCN1tmf629OZ+b64u7f6iVRTy3Ys49nX12Gz2gibGkbK2hTCpopGthGrR/CvYhMEVrTBzRF5nJ9UiiDA7jt2U7m3EoWzgqhCd/xa/chLz+Py6Fo0TRrS979C6uG3MPcMoOzqgMNhTBUS8bOOxT4qhPawGypWuuQwARHN7Nq8HC5fZuZ7M0lfl86cT+dg9Q9k2pEgwu3HEVseS+C1G4fezmKxP+rOhfw56Mdny38nPyUPRy/HoZB1gPKUwH9EijVlNzEubwoOOgd2L9lNyOMh2Cw2ao7XcOTxI3QXd9EcUU1IbD2yCX4IoyYQMSMChYuCP2/9E8s6XzpD6vnDNsiiqypsJgujX5hJ+H3Thv5Gs1nClex46javwtiqIWjcDRC1I6iZY4Og7VdTcSYGU7VoyPp2zLc47zRwz7f38IiqjHlOYnSF7W9Zm6PnRxM0Pojm7OZ/jJUgwNObnmb59uX49M3HbBYBq4kbJuKd4A3lAyi+MTJuwB39hBa8AgOQSCRIZVJeFl5Gs/5luoxRbLh8P4WFryI1y+i+2j00x3cu/ZOWe39hZNkRvE/+gW+y6LdlrxY3YXuzPTmd4oYYrjAPtcukMxFa5Y7DoAPdzqc4MaOAttnDwRSpTIrMXoZE+s/QcIWvK41x01Hpuuk4mDdUf/LGyYRMCmH0yTAW715MckEyPk0GZPYy3CPdab7UjGXATIfcxB6JjvqxC1GNSyZsWhi+yb70VPeQPuUOatMfIL/+KV57aSOpV8aha9chlUsJnx6OSdKHTDqAyknPbZs+pX22+C7EvDIQ0CD206vDi4TSFMwDNyJiAkYFkHRLEoVbCocoYP8unhfOMPPoTC6nXcY50BlDn4GMRzO4++LdCAL0B8yh162XmUdm0lfejNpPza5bd9FR0kFfKvjr1LzkKm5/NkHgy5Ff8oH/B0hkEmaW+PPE0XnsmHOQT595F5lKziuSV3hF8gpSo4yoqmhkTcl8XOaNvCARY9+NdttyB5hxbAabH9iMXqknomQ4qNxwVgQ8a0/U/qNPVotAZJ4IUCq8Raqz8l3lvKF6g7IdZUQqxhHX+A4JVxIYc06kBnMLd6PxQiPNuc1D0X5L/lzChMMJ9NX1cejRQ6TcncLsU2tYcHQWm64lE1x+gtqsHXSVi/kQPeM8cc41cf/n93Nrnx+69d9y4Y6K4W27HlmgGlQx7fg0TBdNw573VPXg6O04zINx2PPr3tu6Dt0/nlUlaTk8+xAmeTeCADIPUVcJEgnth0qpCasgPSOPpU4iyOh8/dLr3lxKxpkuZh2dRVhtGMt3Lke40ktYwS5cOqqRq1XMPzCbN4w+KFQmDi3aS1+KaiiPuoMxgpjm15lp/Jr1GoE5BQMIAphyi0g9/DbhBbvwr8oa1tY3lG8MRfntm3+YDu8OXDrbsQ0akCvkRM0fDqB1tnriKgWbYOXaqFXkLNiA+vYlyKZOQukLQfrLuNeWYLPYqE+ajkmhpulwKTdtX8ptdgqqZjtxYWk0eodWuj27qQ0tQdCYyZqYxfHJ2Wzp8eeeb+9BVmHC0t2PVn4Mr7i1fLMgEs++78lMuYDL6JGovV3xrzyFW0sZKUffwa29EumotGFtrT5WjedzveztjqZZ3cEjTqexYCF7Wiyn59/wJo65GsNtF6oJPvglSndHLs9/mcvzX+b+jhfQ37uNEX0eTGmCQasM18YS4rO+HjIKK9o8+aDMh+KkYup9rWS+mMnE52/k+NAr9VwyJBD4411M/V6kk4o99x0Zh75mxCf38fSLd/KqbSVd/q0YnS0YtUbqztSxrmgdsz+ZTf3ZejRNIkjXVdnFp1Gfkv1pNvYaKfcZPIgVPFFJXDFqjENrWtumRf7EMaKm55DxQweNv20hfHo4k1+ZLDZq0Iq9fztW4Kbv78BQ2UPZDjF3Q93JOib+5E979DWC7BcQoT2Dh5No7Fv4/cKhfoUnic4s23u8cJH6k/dDMamH3wKgKdhKvvMRFtd6MeLjezGW9Q5F9ppVLlRMWEvv2Hn/WDdSezlViR5MOjuJlT+LFFsdpR34jPAh+c5kmppzubBnIm0ufexYtgM7bwcCy48RVrCL7m1HmP5THJ/1wzbXViqjK7H3cKIzSKQ66swT9ZQ5xcYdzuBnp8ditNBZ1klvTS9SvcDIiYUsC22m0wKVRvGSfNux23j46sOip7CynVvNkbz17OscX3cQq9nKubfODbV//r759Pzyz0uo5nI+8/+QUDSyiOLZIrWTd5I3U16bgkQqwSq3Yh1/mafKUwmvlmL1kvDZw5/xzb3fYHI1k9b1Ns8oT+H5xod0zRT3+qsZt9LvFYn/djsq3l5Lv0ZF77hBmgPacOwV62gXrMJbY8eGvctxl5XxR/Es6voqMbf3EFK8D9eODoI67yKoaw29fv0cmHsAc5AL/mn+rG9ej+fMNGxyewpnPU3ai3MBKPyhkMufX0Yql+JmTqTftZ8ta7YwMFFGy+UWts7eSvWRanznReC9eSOHHzrKk64iAO3Q14LeyYvmmCkMxFgoMsJ9Bg82N8VjGzBy13d3M/7CeDwrZYQVVRJS1cwz6gVMLn+J2qxGng3ayoF1BzBe6Wf2dxt5Z/Uh7u7aytqfvsWlzonOW8P49p5v8ZRBnD2csIzCaFOjkmsQBLDrbEFu0lGYUjIM6ANxX7IYLBg6Ndib7HHpc8G7zRmr2YbBvon7vroPAHlEIHZTPTinHuSwSgZIce6zonW6Ycja2Sdl/MRipGNK0WU68MpLr1AZU0l8jnheGfCaxG3fxZC+Q0vS4ZFonbR8de9XeDWbeejSJFQDRj48GU/ziQDRU1vfjNws7jN2rt0YpE78tGsCip/moA73on7EAmqTl1CfOJfa5MVE3SZSQZr1ZjrLOzHrzUh8vDgzY4BOz05s15k9dq3eRdZb4n4QdM0Px5oQNDYQsBKSGULyXclsnb0VdbOSiT630N/twm8frMItX8riHxcPe39Jvy1jfL2UsxPPUhNcx4B7CEZHN6z2KsYnbiU7/Q80VjOtH97BtJ+jUZXUMO/gPKQ2KWvbYU5NAl+2RhNWHY6ky0zq3ak8UPoA3X+cIKxoDzHZP9NXJoIKvsm+THl9Cr8t+o3x59IoGllEfUg9xuvMFAkrEnim9xlGvDgZP8cK5id9RcvcxbiODMZi70DSyU8YcfJjwk6oybg0hjL/Uj6c+jg1kitY5cobnQqVcGvMe2SNu8gXS3ZSd7puCOiTuSo4MPcA74Yf5VrNWyw4ugnXLjdMk2eyY/kO0hXwure4r24ve5Z2bTw2q8AR9zQOpjky59AcQutDCW4QgV6JRELY1DC84r0IKPHGdjKT5V6t3JTwuugsIJWQtDqJzJcyyXqvh40bN5KWN4HZB0TKO6WrEovegqO3I5bIINobfFBmFNIePMjhZ+pILUhFYVLQfttZSj9ewa/n4vji4ztwzk/Aq0HkWLTraKbD28zB0Gt0yhxpDG6kObATQYDAimMEVp5FfnI8zUXRXJFGstzWTZHcBavRiktHFZ5drkzufZe4qji4fiVpzW+lYtd1ILpHSq97LzOOzSBnYwtWq0gLqdK0050j7ic7PZv44pl3yLqljvYQDYfmHqLbsxu7VhnOh6cQ0BhIos4Zm8HClGPBLN+5nL4VXpx5s4Ca6WfwG1Ayvc2KVG+PWqtm8Z4FVP5wAYkgISi6kRL9VLr6RzJT/RQpxvNk7stictGPfO4NI1S/EeZ+Oz2xr5A2aT4SpRKzyhmpzYJEECjYdHxoakx7axprs9diMVq49edbGXd+3NAziUQyzPDeGFHNXr0V9Ve3Yq4yc3/J/RRPfZTG+Jm0zVpA5k3nsZVHoDg6Edco16HvZb2ZxdX9V3Ga6cSqbatIeDgBqUyKc4Azp18+TcOzXzDBextejvV0XW7j0geXsJqshD8Vzi+3/sIEJVwNhVUuFhR37aDv3uHe91vnbOWHiT8MRfj8JfZO9oRtWIlUsOHSUcXfrpzIzAZ0n4tUoS0hriSWJuJXLuaJ8665hFvLFQJnJZA1zYREkPD4R49jPXmWfxXXPlcGldXo7Zqx2UBuHkQ52IPNKvCXE8ou31f55OFP6EpQYmzrJf5yMRHVEUO/0eXVRX1oPUgkyHIuEpX7GxLRa4SMHNGu4BzojM0idsDOwY64w/ZsfX8V7z57H4mn3XBLCKB8/F0UzngCuZsD6z9cj+G3+VzePQXHKyq6kiYjEWyE5e+kXlLD9y6tbB6Ry7uPfIIsxo3kwmQqYis4PK8CiY+a1YWz+TP7eVqOn8LJ6odL+w3739IDq7il/jhOQi0ednXYLDZCc7pY9esqLiQV4zRSzxyvh3nSVYrM6ihSEV7ZP/R9hZ/HUM5Qm9XGe97v8VHwRxj6DeSOzqU2vJYBlQhQqcuzCc/fCQhERo9nMHADH87dw2cPfobMV0V5ihcNYRLkPi4IOh12Wj0+RYmMzBqLzMGeNr82BpwGkMgkOBUk8uBlf5x2pTD2SA+KpmoST35Cv1ckDeH2tJZGUGn0wOdKL6OO/ImmphedZwg61wBmfb6K6I/vRS6N4GhcCcUpFcQuiaNg7vOUZj5Ay8SbOPTwIa5sF9udcrd4XnPwcMDWa+bIYChz7jxGZ/ikobn5vOF5SraWXAfLYNrxdML3hNOc28wrkleIyv0Vx/4WsuzfIOX0F7wxxpmsN7Jozmmm5NcSjj19jMK7C/HQeDA1YCpp+WkIJQIxi2KImhtF3Zm6IaAPGOZ0/Fe+urilcTSPbyaiJoLpT08fNre1LVrOvXmOLZO2/GPeA3T8zeYl/I1Fx2q2YjWY6fGNwyaVE1B+/B/f7c+7RmDFyaHP3f6JCAKMXT+Wl2wv8bLwMhsGN3DLgVuGAXI1x2so+zEXk8KJptjpw8Aum8mKU38nLv3iPdUqk3AtfSXysaMA0RGq4PuCIarVv6LQ/5LQzFAiHhqP1llLc2Az4x58HK84LwQBNnm8w/l3zhMxfgnfSf599E8//IdRf/8qPoD+Oljq4OmAc6Azg/1VDDgN4DDoQETu78ivFGGzgUXhhNHRndaoSXg0FVP5+JfiOxfECGL7wV48Ggow1Ldz5pUzXDss3sP/AkiPPXmMYmEfFSGv0xhQikztgMVoRX797xtVDmht8HGXjPasVByqbZhOXiD4ymHMSjXqRaL386aae7nUJDo6j8kW9+vmaA+MuSVozLfh1eFFSG0QbjHefPDkr2RNEM9kcoscZ404JuWdYxGQ01LYgv+1LML63NniC1Mda1ArTQR59yMoJMh7u3Bpv4pK24Fv9XmcO64NC5gAKP6lmNFbxYi/pC/F6Lgr26+wZ80ejjx2BOsP9UwtcOOTg6kYj7Rj1JqGqORlBtFR1q4mBCvgJbNh7+qAxjMMjUco9m6OhG/czMPP/cTh7XMZyDOz44CKsrF3c3n+y1gD7PlwdwbPtMZT8HMEQX9u+wclZuf1SO+ehHKyJuUNrb+gcUE81fkUz2qeHaLU/rt4xHow44MZAEOg6bQ3RTvsQNvAP+rPXLKE34B9/Of0sfuA3wFn7wU0xs/EpHL+PzSe/z+Q/9HIvjNnzvDggw8yatQoLBYLGzZsYObMmZSVleHoKE6+xx9/nAMHDrB9+3ZcXFx46KGHWLp0KefPi5QyVquVefPm4evry4ULF2htbeX222/Hzs6ON998E4Da2lrmzZvHunXr2Lp1KydOnGDt2rX4+fkxa5ZIGfP777+zfv16vvzySzIyMvjoo4+YNWsWlZWVeHt7//sO/B/5fyRyBzl2Tna4RXpSrAnDpKrl8we2c3PYmygNYdQlzccmt8e5/SB3hPeQ2pzM0a2z0I/U0BWcitRqxk6QUHP4GL0N0cSEBbKi3gtGSujxTySk9CA9VxKBMLQmd/acSMFaHYS2qZ+cN3KImhuFzWRFNahijqsemdyJXzVyJEhwCXZBphA324Zg0fDVHNCMnUX0FM18SaSkaitqQ9+j/7eeE5KL3Tz101PsWL6DDj+rqPDkcuxcVBj7RYvz7T/dzpsb3kQ079wAa7oDEvFoLqWuU83ln94mPjoeWSA0JsymKyiZ6KDtZAWVkSHX0X5TLk3OUizG2UPGNMusBSgspTzj9hq/H51Nr0qCxWih6mAVgZkR7CiLQ+/syx/Vl2nyrGGFStTGShclLrumsr4wE0Ei8FvI3cP69N2Y7/AHHrd7mj/WWAh2CUblpmLyq5M5/dJpBlt19Nx5iW+866k+/T4jxfMPzdnNIr3DgkA2r/mBuLI4Vm5+BsN9/by6W0bU4RO4TkzkYMAhJusUVDa643o1ArPGwKDcxtgnxnLx/Yt4Xo0kZ/ckpt50juZFy0lZEEVTdtNQDkSA2z3LSasIxufURAxhN4xKKncVnVc6h+g3/y6CAE2BzSReSSSgMwDTdXaxky+epC2/DcWCGQxERtHk70SfdD4h4dFc/PAiTRebmPn+THTn3ye4t5/+spHEXG2j/54+fhgjJguf/PYsNg+8gaMFzjZ7Y9V04RHtwcvCyxj6DLyR/gknw/dgC2zA/OMUUgsTMa83U/pbKc05zRxZWk6Ndy4A2Wm/4OkgAgrdVd20F7dTsrUEwSaw8NuF/+DmVni50B4xngH3UBavFOdo1ptZ6Hv0eCd549PsQubSUzjsn0Onlx7PGE9G3DaCA+sOAGBQGtig82PDQ6n83FeFf3o4bYVtlP1RhsfK+fw2eQpnCpdytcYBabuFFdtXDFF47J73K82qfXxYo2CAbua6iOtp4fcLh6IdoyujCWgOIPNsJoMrNcANr9jYpbG4Rw/PK/CX9JsuIFU7sn/BfjzjvGjNb+GPpX8w9c2phN46EQkS3Hs8GHdxHIZxXZgGTCicFVw7dA2nWIGXPv6Espw4Hjg7ncGpA2J+zjP1fJP+DR7E0K4y0DL3AD224TSdQq2AafFYUr0HecF7OZIxWSyPvu/G8w4zUVVRXBh3ge/u/o7Zur3D2r1y90r66vpQeihp7hkOOttsEmqSl2BycGXc3aJRq+LPCiwGC4VbClGkTiQtJ5ITmR+SNTWX13gV11BXEUx3VhBS44+rfhpVIyJJcp3GQNsAOZ/kkPNJDtMPriaoJpy02HYkG77jJf14XlnwCLeF34bPSB+2P5qHzCpjkyf4+8KHOjHfT31W/dAlLO6mONp825h4biLmMPOwtnsnef+DvvMvaTjXQPHPxazav4qQzH96buYk7sBqbiOwqY/++mdQhflxef7L+F09Q8DV0+SOyuWkwpHKWh8uuJvpCk5D4xmOe3MpiuY+ShJLuBp9lQGnARbEvE9HVAz2+n4aruqxC9Dh6d/Fl/3hnNXXsFwh0FHWhXNnPxL3EKKMzxPnCoKmF5e2Dox9QUg8PegKSqbHPxGpxYRz5zUGuwJw8HTAyddp6CDc4d1BYmkiyWeP0zPGAV1HNC5BLnDTCko6vFkx7j1cayw8+NZGBiP6aPDYR2nQ/QTb/sRd20vHSYGG25ZTN2cRI70dOTEiiD5zBvd5PEjt1ossOzOVgAW57L26mB38wpSTU8g8m4mBRtY4ruHqyM9Jj65AOy8Sx8hELt/zOTFeBqrGfjf0bl8/u5v81tnoe3rwrzpLR/ANgE9v0QGOmAZMHH7sMFaTFUEOV602tH0KLLZSzGYLFf7PYpDlMHn/UwCMX5CFo/MgvxYuwvy3aWCxCjh/cTtXR1QxqTiS5kQjJtspHDRiNIttmgslnWfp8uxi17JdJEpXYO9kT+KqRLLeEC9sB+fnMMb+QZQacLRpRfDXYsCgUqPUaymPquI+L3tWvHaI++rvoudaDz9P/5nMjZlIJBJOv3ya+V/NJ+3eNJouNtFzrQeZvYzkQy54FDyMH9DHK1hW3aB9lEllaFuv0eSWRNK4YioCwwnNDCVkYshQRPmZ4Bp+HxxkVrsPFp2ZWZumY9KaqDkhRv1UBspw9viWBTXfYNOb2HPXfhBgg24D6760Z4wv/FT/E0UfCCTMjuSyz408HldGHGSWk4W9fW6gUeFntg6BfX/Jv7ssCQJUBGzA3y8Jr74wBAHqztRx6KFDeMV5YQl34OuHPkPiPECHvQGJo5zugBE4OrVgdyIXP1x54fxU9k45ye+mSzwfupU+3zjcW6+Q/GgmxfWtbEzfgFtvAC922XP0yaPkfibuQ16AabwEZ3ctH/wyhXnn0tA9oBsC+0OCX+TNpNcBUHjo6e8eQGYv467zd9FwoYHjTx0nPS8dfYfrP/rVYivFLBcBW2+raMzSNGrwTvRGrpITVxzItLgAso9kEOdgh71M3NdS8lM48Oo3YItDNmBgQ3olRTo9niUH8GzIp9/nRih1WquVLq9clsrOYJqxQSzsBa+EZqJd2vheYyTw/XQGwi5jWZqBV0M+eicvRtaLa+vyuEiujS5htEqFzF6G2l+N5G+4x18G38yXMhloG8BmsbFy21K6ZekcXFREQHQMfW0Gao7X0GN0QF9VS2fiVAwrzzM6L4Pi6BC6m4rwqcuhaPp6jCFHmNYMW0pi6dw5A1XQwLAIclQGbn/8d6pLwtE1CJw43I5Ls2iMsQt3ZOdny5Dfcp5UfRU1Na0IbYPIfER97SiBm9QAu6m0lnOgaD6dVx7C+/gO1A4eCJNsrPl+DQ6DDkNMsvcXixS42Tnw9JFHmbd/HqMuj8K01sDc46/g3SlQH6kndJ6UxyKeZ1fLTRjzP0F5h5Lbf76dpoAmBmMFahT1ZLerOBjWT4lOTy9ixCJAR3AkAfJ2Hsz4DUW/kvNyJUKfGovcwqDDIJEfuQFT8UjLIutCCPHl8dhsUDB5PFlxD/C2yYd7fQ0cZDPfpF7EI6KFm66vI61nGFpP0VHPXeQH5OcZP9N4vpG4pXHIPF35NPYQPSuv8HjVswBDEdvZH2fjMKjEuugocwrSMOOMR5QHhh7RsCzRg+cZBdpZTvjcuYtSr3n06nuROEkQBgQsThIupJyh06uThpAGAl1vobFNjHRUajtRHh5P2aQCbDFyuj26sSjcMGbEsCPtNawyK3lG8DE5Myqijf2v/8k0/5+GpoGlXXR+Oz7tDJMCHxPn6peXhyINQMy9tPXWrYypvIuK3RXkf5uPYBWuG+/86Un1wCeoCokiCfvBPmTWG0q+9GIS5+bvxmjfgkVlo2jmk6TvFyOYTa0G3vt6HlduyidEMwbnAGfsHOwwD5qx9hlp92ln0MnAvAYzDRf0yDPN2PwCqHOsI+76VK52/5ztWUtxGaikJceZ+z9ayKnJakoTS3EYdODQHDH6q+pgFRc/uMj0t6dT93sO4ERpRTqGagtWo4Wzr5+nrbCNhqwG4p2cOHETVI3wRhm4isGuQU6/dJrAMYF0XOlAVtHEu588x+l5ckJcY7EUamj2b6YmopWJWWIuqo8t7iwvDUPhOYbLyaOwM2gJzAjAn3wG+nVEunvj6QCNVqOoh0csxE7fQbP6ecrjyplJEmqNGrlUgswsEJ37G+2ho5Al3cXLf/NLLtteRvdVMco9IEeFf5I/I4tGYnToFekHD92guj888wihacV8pFSxtcePCpNm6Jn23hKKycTQ6Mvy/BiE+7pQDPYB3rhu70QTo+JpnT2VZ29HlWVFGmTBv8WfqGuRaGP7OPP0RzRYNUT0bqH5s2JiJpqpDiyjY8xFqr2LeP7kCWKTfQh3Ok1C3CWklnUoLHbIO9uR2izUJ85B7mAv5qlp0XLwwYMkrkrEO9Gb0PpoIqsjCa+LxvKMhcHuQVLvSWX2x7PZXneK146+R0T9NNbfu5DRY0VqUZODK+3hYxnlv58F0ZtZv3kuLhdGE/x5MJkbMzmz8QwNWQ0EbQhiT/se9J56PJ09SbgpAZ8RPmyOE3MybXthObdvGODmm57n8G1zsXe057DtMHVhdcRKwfH6vrtfB6PcRCtq9dFqmnObh4zbZdvLhjnF9tX10fzdETECHwGhpxeuR6RL/7Z2/DQPUTS+mVAmIAjgf/U0OtcAlO5LGHdKhWPPGC6MvcDYkKVwfSjzZz1LuvlHbvvlNvYu2IvFuhVBeJqWmKm0xEwl2AmEHnGzMdub6fHowWaWowjwJG9SCmXxH92YXAJIBAk2m4AlOZ1G+wh6HQuwyW7k2pZIJUMRLRajBVl+LkVOsZjlPcgsAjJHJTo30fnWJtXiPe0ioyLbmDfpOCkeK3GtggG3YEwqNUqzlneaFZjtzCCzIQgCGdkZuPW5wYG//qILk2JdsVWcwTZ+JMGlhwEIXTySut1FLNfeh3vuIGF/TkPvvxJNfAaX0t4hPTedopyxpD/ZzsprgThsWYjE14ZjRxbgwK4lx+BWGJgTyeqDq9H33OijUSM6Z046M4mAlhiEeSDYK0WjtESKWu7OWDtn7gyx8YW2C4kcslLuRj+mHp+gDK4p+2keU8xom5aGhnEk2wSUeiVNgU0s+PM2+pK6sSptyJOUDA6GYd/bjnKwF+VgLy6dEF90G0kPHMEj6QXONk9FQIIggINGDGIwu7nSJUlje/rLONr8h84SBrUXkvZG0u9PZ+zjYo6WiBkRQ+N19vF6zn90Oyse/Z1fr/f1g4AP8Ir3wqwT52FB5mQSzrejc9MNAWfqHpGhakrLFO6YUYB5fC0dXvFovtWIFOPXZdEPiyhnDSV+JciSnx+ynWqaxPQhCrWCH6f8iHuUO0atEYVagaZJw5xP5zD6odF89OVH1NvqmWw3mb+LT5IPIZkhdJQOdyj6d2Kz2pBdj74yD5q58s1FLCHpKHU9ONdfRhCm83cSIDtPF3Qu/sjMepriptPnGyc6g0gkfJEk5uK9v+R+ouYOd9I8/855jL2DFM94DqnFSMfRQprdvQgYFUD17+IZ2K3XTaTllQn0+cYivR4UWPB9Ac6Bzqy9tJYlPw9n5/lLnOSuzL8sICAgSxbLvn2sFEOPnm69A4rp3hy5YCWbfwIrLvyVuf1/Le2Ag0p0HNY0a7AarYQd1kHQGC4vrMW1vRJDt8d1Oksd6u46gsqOYrVT4D4tCYw37h6ejQX4V2VhmryM+0vuR+Uh3mEmb5zMmVeuc1ubBaRWKXUeL3KtaRleKaFczbgNn9pLdAdEsKrnJd6Lf4d39iykK8OGfNok9PXtROb9QcLzTzJ1mRt/HB/PxUONFF1ewPabL6JxKCG14wsCrnYz99BcfNKv0H45AeE1EGRyTkw/wZQxd6L7Np/YtAqMent+/WAs0nFbCbjLgU8f+hSLs5Z5zSC3hFDZo6DD1M+6UVZ0o6dQ4TmByJxtOHfXiXPM8gIyOxmb3MU8dn9FouZMzSHYSbTL2GwSJHIZDr5qdIc7WHb4UX6/6XeUPgM8qHoerXswHSFpSIOiuJz6G76DZRyw+fJuqwseK4NpDx9DzKWfMQEH8iM46NSJbvxcwvxCkfcasNMbsSicUOzvRcEcDjmXkOgRguU6xbMhu4i515ZTtWAnV00CRdJwdoZXctFDh8xexp679+Do5cj0t4cD63+JIAhkf57NPc33UL+xHs2z4oYz/unxjH96/H84n8bxX6OPHQt0SOWYlM5wXb/9HwGlUvm/rvT/kvyPgn2HDx8e9nnLli14e3uTl5fHpEmT6O/v57vvvmPbtm1MnToVgB9++IG4uDguXbrEmDFjOHr0KGVlZRw/fhwfHx+Sk5N57bXXeOaZZ9i4cSP29vZ8+eWXhIWF8f77Yth0XFwc586d48MPPxzasD744APuuece1qwRQ/q//PJLDhw4wPfff8+zzz77/+Fb+d9Hfl/4O+YBM2EzIul4N4tUSSJt8V8wIlWFnR10hfiQePITlIMGuj67DevK4yjnnEbqlUFnSDouHVUEbdkEyClJLEF7fBnqAgdmr41hwG2QxrgZ9JS0EF+8H9eMDr6eeRCd0Y5b2h7gwrsXuPDuBVRPJPDM+8+Quu5P/M0+hGxbgWyEgDHSSPzyeN65MkC3k0gLcWnsJfJHVyKViTeC2pO1dFd1Ez4tHEO/AaXL8IUsKKXEBXXxauF4jl2yg6fhjMNs3PpvHKD63R2ZfXg26gF/HJ5WUznmDlTadjqCkvj0nuWoNWqe+CARg0yGya6H2POFNMVOx/GSwLy2m5kW9y3PpBxg0CCj4VwDzoHOaJo02F04TeLjNcTYl7P6PiMvdYaKdJtL/yDu9nQGPMRIgXlubbiemENnhIW6M3X8sewPFLfGIbj959q5JzCDKSftaf+hnfD14Ux8biITnp3Aj0t+wa4plA0zqni37jFqBiIpDVtM9VHRE1VSouPOEY4Eh/TRltJMgF8Gg9fsaImahCUsnHF5JZzdvo4Rn7/G48nn+D7gdnYu2gZAxvMZZH19gEyFlB+6F3E24AvCzTeRYg3DI9qDerc+9kX8RF7HfDa0NeOZm4L5b4d8i8HCuuJ1qNxU/7ZPv676nWU7FxPW4IbZDDUnash6XTRCe61JYuOLayjsSeTIiScQBDi6XqQBKttehh8y1G5yesbV0GmYgMrTEZWHCn23ntPPHuFBnuee179ixKofMOvjeA8xqmOT2ybkgIOvAwP4MdCTgdLghGATuHboGmU7ykj3CibAeyI7Voi0n39RKJ5/5zwF3xYMtb/u1rohT7qhOfg3vnW76+wZY9ePpfpI9ZA3nrtvD9fu+I2a7tuvfwm84r1IeHsyLIRYlZzvIzrYp9vErdKvuXnvzaj91Zw61EB2VipEScmf9yJ3rIC+um6y3sgi6ZYkkdcf0NlkOAyokdmJjUlZk4LjyCh+TXufiOoI8lPz8RncyEzPGwapwe5BlC5KUtemIrP/p4fb0WXFdNnl46GZitUq0oWMe3ocKjeRdiLiwstIQ6L56NGP+DTjdk6+cJKGLBG0V5dJ6OhzpKPJG++SBEw6I4u3LKZsZxnHnhTzpbz62AeMK7yVpa3pyBfa81DlQ7RcbmHXit8YOauRJWN/5fkaEKQ2/r5SLLPUvBX88NBniWQ4FYKdyg6vOBHQjPUenovQJkjoCRS9Af8y9k97exph08MIGhfE1mdKiCwp5ugEKz1eolHPorcQPjMczxhPRl1Kw77Jnfy5D2GJAr9UK6n3pJL/TT7H525lx8ubeDVavPStbzCg9lOj9hPfee8oORfjP+MRH2fKc+NwF8R1Y+y/EbU49eRUvrj/Cz55+BM2zNkwrO3Ja5Lpq+3DYrD8A0w3681oGjWYB83/NjGxUdqDf4c3q7etpi62Ekm8eJzsCkohaUE/CceK+LM5mu6z6cS+HEWOyhWPxgL6fWI4O6OH4tCNAPS79jPHTUmPfyKph9+iugrAEYPNkbsUjZgPLqUoUEdtfg7R2bkUTV+PWSb2367mKlGXDzNQcyey8BDqRoqgpGd9HqEl+2ktuJWIGRE80foEx589zvlN57n7+9uxSWwcmHuABe4Lyf66iKwXxfnjFTqS30IyCGkOwV5lj0xmh23gQ156fQOQS/PqdhZ5XePCiNPs7X6HqKtv89wb96NRa2BjKn1H93LBbOUpyomSHCey1JXks5lD78xJ58T9l1L4OGkeDaHBpPi7Yz8xgwbjSU7pIezcI+x/8GPObvmKiPJNdDUvpTTzASz2Dny7ehlTT06lu2Q+r3IKq8lKwXcFpN6TSttGJzY1X+Ox3XN54vxoDHcOElbjx/w9T9EXqSQ2M4LU+E94xazkT9/NrNZMIir7d1w6q/k5yQEECe/F5TPCZMXg60duTA5lYRVIbVJmrniETYYDWJQmftXCJe0A5985T93puqF+vRQzCn/XJsa98xi/VGxGEODK5AcxybsIz/TiWSfw99zGwZwQonka50Bn5n4+l/bidvK+zEMikxA8UbyQFf9SjNJVSdKqJDT7DIRbK+kvFj0cHbwceNH8IkhEb2H55UvkBsQTfOsftGlF2tqTL57EI9qDnvu9OdP2NgaVgaLkIr4esYjQyaHoe/REzIygrKyP7zbeTWh8HennXqHntrso/KEQEKleFOvCqfWoQW9xw6H/CpaeQPRp6VSnriAifzsZ2WN4aeXnfJnhxgMR5/kocTHS63dn585qorN/ET98N5xOzKQxcOtPsyiLL8MccA82G8QuiqX4p2Iq91Vii5EwN7yLb3xgZrMYnWu1U+BdfxnPdctoPPEFLlkZnLrtJNF1YDaDVW6HXu2DQ4gHvgXvQKU9n/42ExepHYEfBA6BfTY5PBlzmUdr4gizGkhUDyCRSKg/W4/VZKVK/fNQOz9oUjHQ54BUJiVoXBBeCV58s9UJAQjIEA2HpgETHaUduAS7YEhy5auQz5ly+mZGXhONvadeOoWuQ8eMd0VP1MrSMNrW/Mb51nlM0IXwxHtPoB64sX+8lJbFT3+OADdvHNXe6FwDaI6ZQt39V9hrfR2Pbg/u/OFOmGil6/p34s98gYM2jm9mNtCmdyds7EUcXONICw2gYOZT2GRiLiOFTMdKpwGydOKl2aQzoWnSYB5wAkGBa1s5nQXOxMQEUn20mgvvXiBofBByiYFYdCiO3kmMdxTFEeFiHpos0ePeuW0SNysDSNkfQeBNPlQGx+PacZXgkoNcnS3uMk/6VuKzvI/bvWfz7drvuWXbSkKCVrJt3Hp2ONZyLb0WQ9oJ7tPdhzEkHe/6y1gaB3GsTOKXcQ1UTHqAU/4rGRiZgGLnTqIkUVRFV1FmhHgFDA4o6euU0tpbR+/IyWjMSkCg1a8VpeHG+fYvFoG/1HllTCUmlQsLFHJceiVYZGbKUzSEfNzAs9PTkDW6k1L5DhbD8+SMymF07mhoBh+S6L7qy73nmxE0nkjuthBSF0K7Tzt18bFsmPo0ka7lHLwF6jtS6Sp7g5KEexlZOQX7uBDMkgGUoQ9yZNYazk8o4WHrSwhYESQCfzh2UmwSmCFzpT60nk6Lhd7yNsLyz9MVlMKg7BwtvkWYrG8DdoROCaXxfCN+6X6U7zpJlXsUC9IrhiLm77pwF9+P+x5du+ip/rjeh+NnM6l3EDfrMevHEL0gmr13iw4+2lHObI0txNl+DuteX0f8QDy6QB1ygxN29mZK4vQ0WuD7gRsniJCSA6h7xvKJRs7y2C7m3n+KRypmEydxpNbx72wAAu/4XMFOAl8g0JzTTFN2Ey13OnLS/BXXoq5h7yUa+cq2l+EV58XLwsusjrmTtLw0Tk85jYCN/oZ+qg5UDf1qY0Q1L0cUctV4Fwl9CYTnX8CkdKZ4+uNIPB6FzhzmlCSTcmIs1se1qPrbMCmdcRkTy0XTV+DRwnOVSVwrT8L3E1/WN69nx8odVB+txqPbg03pHSzwe4pshxkYU57Alg2OhmgOcRWP/IkcXrISJ+PzhOT70qddRl5qHkaFkSmnp5A1IQuLnYiM6Tp11J6o5ccpPxK0JI2cYgXPHi5lVK6EtgebaMhqoPNKJ27hbniU2TP70GxqEmFAEYVEqsctwo3wGeE4+TpRuKWQ7DFZuPdvItAxjOakS/yZ+Q0OxjAyOzK54HqCyPha0p8Ppd0zEuMJd4yO7siVMPOgP5kF6Sz69CzZH6+lUeOI8KgIJrs1dzLx3EQagxoJG+zgiQ+eoHNUB0EvRnB+5EIMTp64/y3/GsDIO0ZyfpPoyCyxSbj3m3vZ9PQmRtb9zOp/YTyecXw6bgsvMqXbnfauexjofIMxhSs5OfUkPpNjacjvYMeIWqyKHkb6pnBw/iXu+0KMNl/weiywgR0ry4kYN5IUs5yZx0QmG7O7jaifb8IpsYqRS/7A2DCIvjSK/iUKjkQeAeCXTk9mGxYzV38Ecupp8UrB64oZl6tiNG7diAX0BicjkYj3rvqz9fil+iFXyclavBSX6pcYUZyAzWKjfGc5hx89zMwPZhKtjmG3aTWt7tHYJ6eg9oNrh6+Rvn8r7WEZCPvzeOXrjex58gMGk/JZ57YOlbt4n1O4KDh97DQmqYljc44hhIvr6ueZPw97b/ZKM5V2g7zQvo+lUgmW63Rqhwfhkh7GqODt8nQOVdrgNtg6dyuCVSBsWhjeSd7/oKG3mqyY2vuQuDgQVH4cg789IHq5mhWOqF98lK4uC751zggyO5SuotK8OvYOrDI7JDIZ/V6BNAfsJi89D1fpFcgRI0hsdgqIdeFK23Eyz2RibzZgm/E3LSCIugBAZpHhqHNEkBmROykZ9Eilz61vqO7i3YtJLkpGeNeCNSCYbk0wFvlpAL5f8z1v+H857Pz+64JfkQgCtiWd+IX00Tn4JDaLDXVXA069DZh6g3gm8wgZSpgshV6bEYfuRrSeoXQHjMDzZDHPffUcmbcfoMpmRec7yK+rfmXND2tQGVQIUimqGZ6kBx6iyGUMFq0Bhb6PHt845j8+g8YLZVQPZFDmeAJDehkNjlfR+y2kMEzMwWkC3uyawJ1OzTiHGLHaKfli3UdEVUWxYscKAKIXiA4+cqWcqHlRBGQEoHJT8eBnD+LV5YXRXszbpYtKxuP8XgLLj9NfNAb3kyeZ6hPAhIA2SkwWZhydjI1uLOvMNHABe6/dfBgEX7EIW62aZzc9y/55+7HdIaNv0hgeHZVDZupdfFt1Ozn13+FYeAE7k466FBjd1U6Gbz0B41uoTOpFFvAwIRd+xaG7kar0m9G7+DJKcg2VBOTY6G/U4NLexqCLH55XztBsdcQ9UnSCtVlsCDYBqZ2UxJFzsK68g5zTacgcvwDuJ2ZxDJoGDQ9VPsSEd7+h3usmjqc34apy5cuML7mv4D6+SvlqaI54Hs9kwQM/8op/GPdm3DsM7NN16FjunsvyIHj9byYU50BnnAPFvSbtvjTObzqPrkOHQq3g8heXabrUxKGHDzGLWfxw5w+k35o+LGeVXCnnztN38h+JUttBY9wMhIDAIdsbiClxHANcECQSalKWIhFsmM1g/ze/e/tgXwbcAnFvKUXjHTV0YDnz6pkhcHHv2r0s/HbhsL855bUp1JXrqX/zNDaZHTWHj+MjGUvAqAD0bSIw4jAoAi82yXDH1/vy70PXoeOTyE9IXpPMpOcnDXt+8YOL5P9xDdyWYrUco3BjAyPdFpJ9rB9/mR3d58oZfdaOVq+RzOwp4qh1OMCyCPiZvxOh/8fyHXLCJaJNYc+aPTTnNGNxtCOyOpIz7hXkzX+J8HCxbmjRHlw7xLOAW7AnAXdMh69FHWMygUvH9VQ7QT54Jw7XgTELY1AHqjkov4pvvS/3fnMv1yxXSLs3FDujFu/6y+jVXsi7F2OTv4P97SUoA+9A6hJES1QmoSX76TpXQa73AtotbcitJgaMGoK1n2Pr1xDcZMIWcYYAq4mp0/LZo86gqbOekGsSXPsm4HlLFC88dhu77OCoyQ1Jj5Turjr8VIl0e4rOMwcHIVHiiG+9ilXbH8X0kAFBAIWuZwjoa46ZMnS2/SsqOHxGONcarnEp4RKRVyMhBeod48mdHY/UYiS1710YsDJXbsfnETVI7eRY7B3pDkwh9uj3REyX8f5tefzx/iocr0RgewQQxL3Bzqgjbu9sjhok1Cc/xwDLmXAmkpHnszDbO2AJV2Bq7SVeWc3dj0/jhaYIBGERlitXGV2UiOOKHax3gy6JibutcEEjnlVrjtWgadRwftN5fJN9WfjdwmEsbWadmQtvXGB64HS+W/vdEICu79Uj2AQcPIYHDQCc3Lfvv0wfex/wTPs+Iho9qBxzO/wtzcT/riKTyYiNjf1fV/x/Sf5HaTz/Vfr7xYTK7u7iRpaXl4fZbGb69BvodGxsLMHBwVy8eBGAixcvkpSUhI+Pz1CdWbNmodFouHLlylCdv//GX3X++g2TyUReXt6wOlKplOnTpw/V+VcxGo1oNJph/0CMNPzr318Ugjab7d+W/73sPyv/K6z435ULgvBfLgf+w/J/beN/VP7f2aeoeaI3TeWfVzg55STnMr5GZmjEoh1EEGz4XDuHcrB36J0/btbwbMZppF5ysFmJyhX9l4xuBibNvkRvspHysXdisQl4NeRjVLlS/P5xlHU9hFy7htqnh3bfdtzjvPBJFueL4GnH5bTL/CCXYa804u7VhSCVsPOWnbzv+z6+e4+SctmDWZ5ijjQBGzabjXObzvHTtJ84sO4An0Z9StHPRf8YJ/MIR9a89AN29mb8Gn2wWKxU10owKa7zec9ZSsC6KgIGnPHsVGM2W9F4hNARloFnUyFjc0eR4TFI0+wRaEcmgdGAU28T/pUnMVX3oj/hT11ZGAXB4CU1kP1JNvpePWnr0rgaWcnd8s2sbpvCzS41ZPsakTvIWfzTYnzHhOJU9yoe6od43y8LVVYGDlet2Dva4xrmSt/H+Tz35nM8/sF6KnZXDOvTvQX3Yn5oBVeTE/BszKdsexnGQSMVeytoL2mn8UAd5uPjKT6fhLE0CPvGbroqu0AQo5rku3oJ3fgU97sKfPHkN8TEzya0aDf+VWeR/7QFrkgYN/88i5xgmRPYZAJxy8WcgdlvZOPmomPystMsVB/mKc/fqW3NI2BMAA9WPMjA3d40BUnQWiN5JbKRgxvfxC7Odajt+T/k81HwRzRebPzHnDSbrQhSCztW7ODSrEno9VaMWiNqfzU3772ZioFs/hw0caJAQcSlL2kpqWNN1hoSbk4Ymp+DT37F67e8Q/raH1H4KFh9ePhx7JcfVzAoiHOocn/lUE4nEJM9n/Rq59PH36Z2xgbkDnIWfLuARxsfxaJUMegg0pmt/WYtsd+IbVd5qHAOcmb1kdVMe2saPiN8/rHO+suaiMv6GremEoyDJrHcYqXopyKUrkqMUzJ5RT+GA0G1DIR1o2nWoOvQMe+redT9UYbErwNLyhVqHnsS531qLn99mVMvncLO0Y7a0suUHo2lW7jI5sXuXNUepfZ0LUU/FtFZ1kngNRcm5afxobOUJz54gqC94vrQ9ei4siWbA3MPcHbSWdp927mSWIlEaTfU7ubLzfw842eqDlT9W91hJzgSXB/M0p3zaLjYiMJVQXtRO8eeOobFYsUk78JsZ0brbEGmUhA0Loik224kKJ5zKoy58ef4/v423MN8UAepyXgsg8RbEukbl8nKuks80qxhxKk+MAq4RboRPj2cCRunUto1mW1Zz9DkqubZnmDMWtM/9CaIlJeSQeOwZ+0l7RRuKaSvpY/Ozk7MZvNQnywWUW/7VGXRdjgfq9WK2l/NyNtH4hruin6MO4cXCQw6DBLcl4EgCOR+kUvW61lYrVa0qZO4lr4Cm2DFbDWDDOZ+MXcoQjm+6i4ALCY5mAVMehOb4zdz9Kmj2Gw2/vSHKAcNB3+eR8uf4nuPmBPBC5YXuPzsZfYt2IdNZmPj6o2sHr962HjkfZXHtnnb0HZo/7EPhU4N5e6cu1F5qhjoHPjH/jTnwHSW/LmE3Yt24zsuDN3VBtL3v0Lc+W8RHKSMLB7JkUEldYnzcI4KAqsFv+oLOHdeQ7BZkNgkQ+m0zBYrNrk911JXUJW2kuLMB/mgeCtWq4yenBGoGkx4T42nLnEeens9fcoC2qX5HI56nxMza7H3d8VqtQECWmUZp0c9wckZNbjHuA/16er+q0NjXBZfRu7oXAS1gqy6IFoiJ2KTSPGtKyLlj2ymznqN8nF3Ye+tJvPMDbDO/bgbLVIzTYN2OGq0WATR4OSsdebEj/UcXhjCnrhU3npnPWXfq+gJ+BerIODS68Spvkc53rAQo9mEbMZkquLEehIkSE0teLt9iWeblu7ObgxqTywKR+QWOWOyx+BbK+pGk85E4NhAfFN9sdms2EtgTVod6snZYCdl1EWRGsX1moFufRDvnXuEvZYWtPZXGejpx6VTvAQaukQdde+oGn554XcCRs8HmYS89DxyR+cie7qePW/diY9RSfquBUwuy6RsZxnXDl0j47EM3Kd5IDlZjULSy+m6VbTqIrBYxHduE6wEySFdAR3G0ez5eTk9V8w4eDqQfl86eV+KRsZbDt6Ce7Q7giAwYcME5n09j4/DPybsgB/PzNtJY2AjjTOk2Gw2BInoWWsaNKE4dQRZqQePdoq0vrlf5nLuzXN0X+1G+LqB5z9cz+XGNFIVYLGaaS9q533f9zn3zjkGTN602WL4wRpNXbgZmzvM/GDm0Dj5fHk7+7Y9xn0RC5j9ag6OCyYjk9no8RP3NddOLx4pmUFD70uMq91CkDIOmUw8q/2dwPhf141Rb8JZ4zyUd81stiJxcaSrQUfNiToEk5WlNieO7x3P6Jc3Yqs2DdFCSn09ubzISsILP9F0cQS3/P4ePaUdeNXn0esTDQ52tJ9M4bNz0RycdIaLY66RcHMCL1he4AXLCzQvd2T+/vms6PXnx0W9OCwYg72rgiNPHGHv2r0c1ToRVgsHjanUblnMHZ/NQhAEjINGCrYUINd00OWfhM3VDUEQaL/Szndjv6Pwp0Ks142vCaUB2JeIejBqfhT99f1DeRTrKkJ4z6WZLg8T1v7BIaDP78EYvrzvSzrsjIzfsRS/Qhc6QtKRmw0knvmCgMMCc5wEnnWSE97jid2ABee2StL3v4KDVjT+yKQJLHSUs+aWk5hHNSNIJVjtHbBKpZgk/Tgoa1nfpWDRxw9iV9JG3Zk6NsdupvtMCYJgIzJvOxVbLmK1WgkcF8joR0Yjd5Az8f5S7npsM14V2fRersZotNHrG0tDwmzc4sWLeGDhMq6OWoXLpET0zr5IrFbc2ivwu2zhFXcY5y/F3mEDni4RCFIL2RnZ5AUeRWh3o6veF8N1PSiTWWmMnU7R1EdxvE0Ev0MrA3m/ey1faT3otrah3l/NxCyRQveJLqiz+fJcWB2vPfkuigQ3eiNS6fFLAAQOzznM7iW7h+bej9N+5BXJK9TvzCXiWgTXoq6Rm1EL9jIKxtjz1nNv4dsoWt18jmcyKbEY12QrAhKKRxQPzen29HLSZmZjb3TEbsCLhLJE1mxZg0+7D5LBXH58YhH3/DaGeS1wQecKgpS7v7+bqYeSsb9zNTlhDyLYy/kmtp+xES2ABPfWDkYWjWTBiy8ysG0BTkIPu/xggXqQgdY+PFpKicjfTsqFWubtcmbfIx9jtVrJ3JjJC5YXiJglUt8VZSXz2Dcz8KwWz1u+yb48WPkg60rW0TKtHU//Ljrv3sb5+dUU/VzEzlt24hruiiLGiy5vOV8HNzHdQeTskG8XDegnk09is5fgr7Di1utGw0e345SlJaDsGC5tlbSHpNPh1UXRyCIcBBNBcpBLbTibosksKSeyZQOlwZAVUcIPrZF8eCESoc3I1YNXOfzIYQa9LFyLEiOQjNfPFgu/X4hJJ0YhBzb5Elcex/Rj03Hu1pDxSAZJq2+ci9RdHnSXRnH3e0+gyW2mK3Ak7aGjEKxW+gKMaMfmMUVtQyG3YjNZ8K2+gL1Bg/50DqE1gfTGVyKpt2K90ovNZqPpUtOQo9+ivYtQ5SSiUJv42bqAQQcjql+/577PVwKQbo1mVL0Hby3eRd4tf+AQ58i+hfsoHlFMWVwZMquMMG0CVquVpFuTSFiZwITnJjB4t5Le1S+gzLjI7kW7kTjJaDwvnvE948TcmWOyx9Ch3kmTw17sXeyZ8sYUTDoTckc51ug6zq3YwaSKrSiKsrFJBkkoTSDz5GjsX4vGb+VhPgyuZ17816jknUjNRpw6q6GvB6unNy7hRqokYykLaKQxqGNoz9K4RWOYEMQDyjuIUHQyetYl7CPb6R/U0RWYTLuvnu9cFcSuiaW2tBar1YpnrCd2jnZD4zHg4sbo6jxcDTOROdiTO+8liqY8Iq5xm4yJZYl888Y6BvYXoO5VEFMZg8OgA2nvrKQ8cy0j7DYQEPYu8XHz0CtMmOxuUFDKFCbS03awZ+Ya+ly0FCU38tvK37C71xu/Vn+mSxXMc3kSl2fP0bXahFW4gTY2eH1Dg+UyP7b2UngmhfOl2RgDIugITqMxdhoy0yDC4CBWqxW3cDfilsVxftN5THoTTvp4rP47qFv1KnaOdgRNCCLjsQyOP3Ock/f8Tt4mP9J0vxBq3ozVamX//SI1o09t9tDfnx+oISKsjdJvSzn8iOg0vub8GjzqPfBtE0nu/Jz8aMlvQSqXEjk3EsN4N048thnH0Bb+aHOgt90Ns8lMwBsBPPuW6NA9uRk+63Yk58gY1MfF/XbB1wuImB3BpJcmMeO9Gfim+Q7bi51DRaDDKrenJmUppsAwrFYrPlVZpB94DePFPBRfbKZelU6v3R/INN1YLFZ0zr4YnTy5tjsfh/6WIQYhmSBGXAs2G75Xz2Dt0bN34V4Ozj1I4Wg5VqsNVW8zbk0lmPUmZjGOHx1HMqUjkPUfrkdVWIfFYkVlDkBmvUGr3xTYRFV0N4LkxllCuO44Gl+WQvUbOWjbtJx/9zw/TvuR0CmhFIy9ylMTv+bp2K14WkbQdr6KmEs/ElB5Cn37AFobLHCAH5wcCLR44FxXRGjxPuwHe5Fv1+ESU4uhw52WXxZiadTQ79KPyiCeV2xIKdDMZdrUfSTevI1+tY2aEQtpjRxPu7yDD287zKvJBXziYeTt+X8w6C+hy+6yuP/HlxOWUMOokGt8b7nIidg3kQX6YlKY8OgWUycoQ7xJX5eOIAjIHeRUH6mmJbdlyFn2dOZpPnts3/X1aqPHLwHH/laajpyDbi0nfp/OjseeRN/QT2y5P+l56UjLNDg3SlHmJ/JN+XiuNSXi4OFBZfwAzvoJlG3Jx9W7l8zUg+QY4AfZHgSZlMLpj5M790XKR9kz8ZUvCIhoYVkrrNPnoTcPYnJwZcA9mPLYIqoD9hAlLWbv5ek8ve9Wak7WEJX7K0knP8ZuUMO8L+dhtYj78IGHDvC64nVK/yjFzsedj73CqC+KxLWtAavVyvR3p1N9tJqst7Jwa7lGWk4QNqkNm0x8B94jvCkfe+fQHNH4dNBuAYvNgvdIb+Z9PY+Rd44kdXcqr59+ndk1mWR5HGal50qOPX2M06+eRhAEemp72LZgG/pePTfvuxlHH3HeeY8czorm1+rHW2ffGrr3Wq1WjING+pv6MWqH35X/uku7t1whqPwYKO2xCbYbd2wpmHUmfOpyRZuYwgmj8V9tkzbMCkckgsCIY+8TWHoYy/W0M39JwXcFvOHwBnvW7hmyWQZmBKLwcMC/6gzOHVeJeHElyXclA5Dy+mLyx/mQnyoCoTq7QpIPv41hxz6sVitKDyVKTyW91b2ceuEUJb+XDOtTX10ffaXNWNFR774BS0knPU09tEeMJ3/2c/T6iQ7bizqXEOc0molImKlUshU4DrgCJxDzo/1nkg1kYWXyuFux2WwkrEwg49EMNNHONPs3I0EGCAgCWCxWen1i6YwZh6OPI0EZ/tefCdhsNvbts9EVOJJen2ikrs5o2jQM9on6/dLHl/Ae4c3cz+ZiFazoHHXkpueiinRmsF2LYqCbHr94BAEGjC44KgxEB4cTGTwPq9WGzsUPvaMn1z7Yz7FnR7Ayth7ZS58gDVfR7vAV8Re2EphzglovK8/f/TVre9aR5fQgfTVdLN+5nOknptO+7hgzjs5gdKcfwS69vPLVe/Qs1WHUmFBr1Mgsog3EJlgI82ulf+o5LAESJL1dKAc6aYmaRPnYO2mJnCg6qVutrCtax7359zJm/Rism63c/NvNOD8p6vr6ehsSsxFsVixvRrPsoe28MLqBZS4DWCxWorJ/Ie78t5i8Awg8GUpO1iiKR+dyePZJukpbcOmowmzvNDRWU/NGUR1/hlDH37H6iw6QdqZBMNmY/PE7fDS+iFCFAamiGYvFSt+EOZydN5H+kp2cql+Gp9CE98kJ3PzjAvQaPY/VPzZ0B20rbKPxUuMwDEBiJ8Er2YugpiCWb1/O4d2HsdlsfJ32Ne96vktfU98/MIB+jeb/Fn2sySqyqlmlMgThfx9c4z8qt1qtw2x+/5U+/XfK/2hk39/FZrPx2GOPMX78eBITEwFoa2vD3t4eV1fXYXV9fHxoa2sbqvN3oO+v5389+8/qaDQa9Ho9vb294uHs39SpqBieu+gveeutt3jllVf+UX7lyhWcnMRF7O7uTnBwME1NTfT09AzV8fX1xdfXl7q6OrTaGzSDQUFBeHh4UFVVhcFgGCoPDw/H2dmZsrKyYcbkmJgY7O3tKSkpGdaGpKQkTCbTsPyHMpmMpKQktFotNTU3NjqlUklsbCy9vb00NjYOlavVaiIiIujo6Bh6j//dfZryxhT6hX40V3RMPTWV7cu3k37sV3QOU+iQmgiqOD9Uf+fSnZSMKEHdn05nUw/+BbsB6Jq8DM2s+3kzuJmNJfWcqnaloa4R7/pcMA0S9PgY9he8S/3ICj6VJnG6azHd/b2kvJ+CzWxje2WNSAUoxHHZdA8tEwNpE/pRJioJtYTCkTrcPd1B78Gy30/gPtBF1wNdVPw5fF7o1Dq0Wu2wcero6EBiAOb/jmpGKLOu3IKyVo/kOrVHwIk9rEg7ytv31/N9Ywqzs4twv9KDo1TAt+YCoaXzuG1EF/vmL+Xz6jU8Me57mk9V4tIjXgqi0q/i6Kxj073PEBvTS9ATQbhPc8dngg/2p2Tc8tn9nH+4hWIreJgslFeV4zvDl/KnLhNbKqDVD6CaO4jXm++Tb3oM/3R/xmwew58Zf6IwKeh278VkEy+Hf597vzivp97zFG1Oz/DpbWkUXirk8LLD2KntGPPsRC5uOsegxhHXj17n16b5rF3qwZrpa2gqaMLiLEGuEehpd8c3tA1BgF73MASTEVeNmGPELl5Bb4cboeVhNMmaCA0UPW3snO0Y0DjSY4XedneaXr0HyfReOm7pIOuRLHy3t/IUtyB7aiZfKH6lRW7iARlDc09jp8EtyY2+tr5/9Km/X8aMozNo92knL/JmKir2EhFhpuuOFQhR/gg5Rzi/ZwVpKgHzYC0XS88w8YU7cElywXWaKy/v+ZQKg5Epeih13klG57sY5f0kv5RM4auFdHjHooht4JwujG+b3aj1GZ6zbeG+hbQl9RKWUIuX7DQdHXH4+vpS31SPZ4uCJi/xsNLn2oedyp+qqiq8bvZiys1TGGCAEQ+MwNnZmZKSkmE6wqhXIzX3EFG4i2vPe1KSNB6Dn4FbDt+Cc5gzkle3EVnoTdZ6GDT3omnScPrl0wTnB9Owp4FeVzvudGxgSVgtBjc3GmsbGegeoKKigstxxWQ9/A2P+Ospv+JPkeQobpdEQ9moB0cR+6knQQ2zWbekiGOzLvGHm2jcfs/jPQDmMY+8NNFIr3EopL6+EaVSHBs9etxj3Dn14ilIAJnjDU/AmJgY/GvdCC/yJaBeS+nFUjTqbtLWpRExP4KKikq+uP9L7MwyJp68iEuKB0GznajMqyTxqUROHMpjye4lnJ5cz0PlT9Jc3Ub+M3uRO8kZ9+o4Cl7VEmw6gSTWEZPNj5qGWmp/qsTSYaF8Zzn2wGFlAcqm+SgPRyOdK6FOJeo9v9oYZh7Yxdmpt/Dg5gcxOP9MSckNKo+ip4qoO15H8KJggh8Oxt3dHYlEQkxMDLo+CSOOfYbCpKMBH0pWS4kOiaa/tZ8WTQu5+vNcSnmF1T8/QFS1N9rXtLhPcUdv1HPp0CW6FPFcjc0i8dzraCqcKVy5Bn27nszPM+kusXLtmIpFRk9G7pYwKryfM/Zn6CrvwooV84R+pJZ0BkaVY5nbRKtb4jBd3hzQTIOyAalViluLG42mRtrUbUO6XN+jxzHYkermauQ98qFx+mt/qv+znsJXCxnz2Rimr5tOye8lHH74MCmvpiCxSRl0GKQwpRCpj4qWcjEfmr1BS+9reTQGNrJp7X7sjJWcM+3Eu1TcF6Q2K8t+amMZL7NryS7mHJrDwIw6oi79jEGuos/Ji6T837nl2Z9ROhi4tiYKV8+RdDuaqHGPoku5j7LE2yizxdDjVwl+0ND9HF2XKgjJbyI/o5264DPUBUNd9+NIeuDcLefoLhc9Bbcv2015vGi4tgk2clr80fk7o+iux6O3AUGvwlYQSZt6B5XXEti9WNyz+j31rPv2QSqMElYXjCZ892R6nzTy8VPv8PS7TxNevJ+ACkfOZyzl2Oi7cdQ5oQlK5ZfVvzAqdxQBS6ayR/EAB4MlvJM9iZDLaZzdNwaj/UxMJiMRdnBtwsdQ/jG/LIKbRxeS4JmJRNeH1F7FHT/eQUNQA6emZlFSUoKX1Iv24nYKdxYiVfbxdLoLVQll/OBbxhy5jSOzjjAqdxTOen+UZ/bh3iJnpnYK3e2B5IVks3359iFP6rMPfMO9KjOHqpZibg9BZZIQ3hPOtBPTkHZa0OHP3FNTmXM6DdnYeDK/iuH3jN/J/ki8uvaQRPmt0/nj8hnctFsYX+qH01Ut/sZGzHs2snLsBZYu1uJffQ6tagRarS9qtZpRb41C4aNgwHuAkpISYmJi8En24fCbh7H3tqdJ0cWvoSYKnv6ObM3LFOUXUb2tGn2bnsh7I3F9/Cdu8tBT9udC9IECtcF/08/lOmxSOUa9At+SeDpULex9XswpnPVaFlKgOO02Ds+MAomN+balRExR4RDgwGDz9Zxz3eJaGR+8i1crXmN6/xwis2+cz86bVGyTbGGy2R9d80ykUtBqtWgVnrRMegxXVwsGg2HYea+hQ84HT3zA6OzRRFWdorg4lh07PMkLX0V0jBmnimPkbH4SdXQjOrsAOjp68KwswNGso+Pt84yVRfL1uHNElp3Co7kBXasWj5ZSPFpKyRZ9m/hz7Dk6fDpwGZQOO2d6mWMIkNWzzL0MP9dWZBoXiotLSHsiDQd7B17M/oo6C2T1B+A14ipXnAWMGiObXEVqHC+FmT/nbMB38GtsthE09TcRvTYao6cRXXYbo5tHc3xmCemD91NSUoLRw4jCU4HaX83ReQV0uF7BzirFajTTrxJ4+5m3scgtvJP4CxsbNIxzELhv9a8onMfQ27MdSXQj6flKvC/K8b64kfs+fxenL99jVuXDjNq3C4C6yEy8pC0s6zYzQnGRqc6ws7UfbXcfKo2ZNlkrWZkp+Evsed3qi9LmgJsklC5LF9Fro2lRiWN2beRiJt7sQklJCedeP4e+VY/fGj+e1Z8j0mKPaUYUE5J96ausxKTRY/KNYcyr0Rx96UcWJvzAdzUv032dE7QsZQXRhTuRDHbwkgfg0cEHrVasAzJS8lNYtHcRZjsrY83zsElsdD79OV0dbvQampE39TDyyl605UpcVh7grvBuHmkXkOv1JLTMp/umKI4JX6CUQEDl22SpUzirvw2k0NjYRF9fEAMDMoLqnqQu4lVcdOlDc6DuZJ24NN49yJjIMVRHVmOz2KiqukZF0LOMuTiG5EsilV2LXwu3Tz/FiUmX6Lc8QVNQE42BjQQ1BeFzOY4ZK87S7voIZ2qWEOh2ldw0e9R96wgr7UVuUHLJfoAZR2cgkXtjsnWRNSELjZecmX29DPTLGOiq4Q5XewpNdlitAtH5V4m13QwYkVlleEuUdG9bSHzeCPq/FTg18RHC6i4S2piLVWoFlWTY3P4rH22zfzMB58agi9JTVlaGU5cTglJAo9LgcyqA1648wDfrniNYocWECUEucPCpg3RNkvJZwAvswZ2ql9fjlm6hdnYtKr2KlJ4UusfJkedH8lJwKy7VfticdPjVXKDdZqEibBLVS3/gNmcdOYNz+OR3P3z6YlF1/MmUuiq2rfDnyCA4Wzw51xDJrG8S6JrbgXB3Eo3TF2M5sI+NZRvpd+6ndPAi7q4utNW0oW3RsvvO3TQFtFKSVMCyXcvITRHvb9M/n47nEk9OLT9FcEAXP8doeS7jErPU46lxjyK0PpuUo+8gP+RH8tLJnJx6jHcDC7jLMg6fluqh9+bX5sX9LvZ8mxhHifJmlp0sY+ecnfxdnraaWXs1ii6/tXjUOWBSu9PleA1vGexOFfP7+QR3UGDt4GqDCFoOOg6icdYw7uI44vQ39N+SrUswmUz88sOnvBqWQ0wMTAlqRqqU4zXSi5aLLVQdqMKUmUDRQDczj4wgvCabkoeDqfy9koovKohbG0fzkqNIpAJCmRVj8GVszk6sOCDuZ7oXyohqfYq2e3azuMsdD/efSMktR93TgKY/gbrMCZQPTOAmmZZs/zpMOoGysjISTpzAIleiHGgnVFbAqfRAdk/cgap/AV41tdhsSVikWlz7XFm1ZRUHJQcZ/8h4kpKSSLk3hZwPcxi8cxk5NUE41PfiqDYzMKBDq7Wi1N+4XxdW+dOZGI29RyBFyd9zeKwYxXah5SCRyQ/zfHgD+6rupKr+ebw6jBybcQyViy+TNNN5bNaLuHhY8e0CrUbOubG5uPZb6f+uEQWQPk0EVtYnHmVujRSnC+O5uf5mTizcR0eyjn29f/CERwIHnnuTRR6P4CSLosMhFs+SI4Q1ZBNUcYKi/AWMTB2J/xh/+rX9lBSV0Kf/A63aiqP0JbRaCR3SDpqrmofl5i6rCqZyZzVX9d8T/WA02U/nIRVstPhpuXD71+S7yPnAKKezoXPoO7vW78IqtbJ78W62sIXQklAKCgvoq+0j4fkE9jVr+Da4kwaTHaW7prIuL52ChQUMhAxQY6ghSA7rXeEhNx1P3/0r1bZVGAwGRtwxAkmqhCtHrlBRUIFvpu8wG4tgE5BF+uJ8rY4e/wT0DmpKSkrwqBfBAENRGQZHI90e3Yw5UIh3ZzVVtyzDoPXGwVGNTjDQ4VWNvcmeVdtW4RymZcAe9AY9gVdPw1WIWRJD8chiJE4Surt7cKnJIbC1mJZJqxjnGMvt/u/SoAnk3JiLeKgSuHYyl9QDx+iwppCTIab2uDzqMtXhziQLAvZH9pF49RoHZ4uR8/4tIgehRCXh+NNi7jPfxb7IYmvRXA2huU+NrqaIDm8jPa5B9HiEM8rHibveXkVVWBPvHppG8IhwygITkPd0EXH5NyQ6gYixzdSMKiO7/z7uiEsn7MAjfH3P16h0i9A51TPg9AmSq4DkCndor9HoFEfmuS/IHQ83eyXxzb2/4W0Wjd+dnV0En6pj3cA6vr3/SwLc4Am1gl1uE7kkc2Lp4Ld4dXgx9ZTIAqZ8fBY2m21onGwWG1f3XeVqTRWbHxJpXR0G4qmtrcV0TU+/whUPQymDZzR4bn6DqbVRbMiKxNE4gj2Ls1m1bRKDFV34NckIL1mOIqEGpytnMU5PpNnzBdw6Gyh4Yxe18VNZpjyFrfx9Qvs96I9uQto8gODshdaiIKkB4iyL0bTlktEYROf4ThoT59DXp2Fm8TpS1Hos6SGUlqZi7XZD6zJIWfQMnAa7cTT08u3ob0m8NZGIJyIYlIrny7zf8/BYk8QMzwqKX7pIdu1KSkpKsJPaMfvj2Vw9dpVZB4OAIGrCa1AqlWiaNFzeepl+UygX028nZ1waEnszkz5eQPjMQUrcS5CPkiNpllBwcwFphjQu213m5IuXCFnSSmd2J45ejrgEugxFswM4jHXAz+aHAgVGlcgMI3GUUBhRyOwjs3Hpd6F0cekQy0t3YTfn1pzDK9GLsVvGIpGJ5X/ZLDu9Y+iXKDAP6CkuKkbloCI2Npbd9+5GkAroVG5YjXqEriZMpiB0uhs2y9pzZgIrTw21TabtpLW1lRkfJuP1mxe9Rb2E3xFOyZclWFwtlJSUDNksC3+5AMC1wFGEetmhDBTPZO1t7cQUdlEQ609jcCN5ETcz2vU9JBI7SkpKGGgYQGonxT3KnZ6qHmqv1mItsQ71ac4nc7AuTOL9vavp9Sjn9fjXWTJiCfwKvmXH6VG60T3LSMRxGU5JambIFxI3MorHv96LxarFTSUnrN+eOdRxCOu/pVXMBmbKwG5sGLLRifT29pJ6dyoVFRUU9xWRNecb5AY5Y9qqkLh4UFnZQa1bBEplGGt/86JrZxcVn+5Fa5xMS4uOjg6Bao8Y1KGjiNf186Hfx4QuC2XkCyO5/NNlDE0G0p9KZ3BwkH7Xfg7MP8DKoCfor+4irER01pANdHA0Qc70I5/gob2ddGctA9/vJqKhiwtj1jKp9GPkPQJ9na68E97MS81NLPlVjb1ZtP14XXXGIzSGBrkOk64VvcLCxTEXGXtpLILZyvgL45kd2Q+0inu3pp+2HQ088cMTnLnzZ/ZNq+bnnlYi3QaYnlHChoFbkORfIKKygMLMB+kR7ECroaSkVpx7I5LQaDSc+eUMdnfa4Y03LYtFG6X2Wj1pR0T69PY0D14KvsKaE9MIO5FOQVQ+GqUrztTT7xeGd3sjF76Zx2cbNyL3bePqxSv41l4SdVS0D44zPmCxl553HnsUNw8LtlVRXE67FXVfE07jL+DS78wpm5k39ibi3xJK8cwSskvC6PJspV99jBjv09f764i6x4XS4lIi4lM53ROPx4J6fCPlCLHiueTvGEDAPQFc2XiFxCuJNG1rondqL1JnMf4r66cs4lfGD8MAFArF/y36WDupI4N2LhikotPf/y64xn/UJ5lMRnFx8ZDN77/Sp/9O+f8N2Pfggw9SWlrKuXPn/qeb8l+S5557jvXr1w991mg0BAUFkZCQgLOziP7/NaCBgYEEBAQM1f2rPDQ0dNhv/lUeFTWcP1p6ndMtPj7+35YnJSX9o1ypVP6jHMTJ/u/K3dzc/gGqAnh7e+PldSNk+7+zT9WXqmn6o4mgqeGcVB6j26ObS2OKiQ4MI8acRJPKhZqRi9EpT+GU2MdHNTPRHJ/KuMwJtJiOoHP2QxeWiHxbGi+XTuWx795gUkA0XWGnqR2xkLDivTR+CDlrq5F5NTPtt0yEUiluT7vhHWKHykNFVr8OesVIiI3CSXQ+OmYG/cSUJYso/rGYuiN1XEm4whSPCdhr+vBsLsfVeTG3HrkVi8lC7me59FT3kHlr5j/GyXtXCZlZmRSPKMbgaCM+Pp6Qjp9AKuNa6goi87dzbt8ElLc3IpNJCfYKJqJu37D3lHMkgz1LG1HI7JFKoTZtGYGVp/C11PB5yAoC1GcxBLTS5WLP2EVjedtJzOkQjg+9nl0scBvLnBemIVXYk/RWEhKJhOBZ8Zw4tY+T489yrGkypaZTLFCL3qQJaQn8sq6GM/VvkZORw5dLPh/Wp0MPH2LNF5mUxXnxx8pN9AVkMC5kHAXJBbQXtnPp7SyqJ6ziUaU9uZ2XSFA4EB8fz+5bd1P6WynOH2Wgb30VRzsdOz5dzuCcBnqDkvFqL0dQO2N1VxHkVUFuVgayXWOI+TKaee/5sfS9pXR0d7LwrgW4FARzT1Q3uZPPEDRiNvaD9pRtLxt6Z8/GT0LZ4sWHdf6YB4w35l4SFL9STP5n+aTdljZsPXV3C4y/MB6tkxb1YCPhCyI4f17CtbOtvHHsKl7jwbMkgcC7dnP8ph0szjiHod+AvWDPlDVTeOrMB3hUh/FQsSMR1RHYRlspeLYAr3gvxr00lS+yEvAI2M7Z3Zl4NPkw48cZZCzOwDRg4uO7tlE/cAlHn24aKoNxaC/C3Xk8/Q39OFucMfoE0+0hJjnYsWIH05XPERUVxeFHDlO5p5LFPy/G1GoC53/qiOqWXjY9/RyzD81mlu9TJCUl8e3obzEPmJn7xVzstRrsseOBTU/R6inB5y0fFm1ZxL614jzcsXwHc731zF2/jW39K1j05CLe/eJd2n9vx3GiPatdesht9oAPb8N7ppRJ6yehWaFBZicjb1ITdfI83tLI2Z55mAB7Ue/4JPvQXijmJ1DpVSSVJDHhnAOuYxU3dFMSDCwaoPD7QqLCo3D0vuGhKpVKSc8KRd1rzw/rOnnk3mn0l7Sw68FdTH1jKt4xMZAjEFoXyuTz++kaJUW9OJmmnU30Vvfig4K4UWWYTXIShZNEBN1K7h6RIrX7XDd29f2YnQY5sTaC88FeLIkK5cBK8bnf3EAOtuwjP6YCefVxlPFNqP3dCQkVLwJFX1xkXG4xZSM8KUwuRKL0GKZvlQ8paTrfROpNqdjcbSQkJCCTyZBKpfQLZkyObmi9Ipj9UiZJSS7svWsvxT8VEzE7AvdZLth12tEU2k+Aci5qtRofNx+Kcovwd/KnLPB1StQfEJN9MxIJeJg8+HHxj6gD1WR+uYrIj75mYtQFtBOc2e7kTHhqOD1LepiwYQKlG3/gwIH5nHXv4O2o3/CVt/Bh6K1oW7X8OvdX4iPiKUguQK1VU720Gs9HPZnx/oyh8eiu7EYmyEgZnTJsnACCXYPpNHaS8WgG6bPTkUql1ByqwdhjxEHrwJH5WRjk4sHLZnMifN4Efu6aQFDZUXzqc6kPqSdebaDNvgaTRy9W9wD6LIM4XafeABhwGqDNtw0Xl5FIrXrs7JTI1O4Q3ISzeoAuEhnsWUJ8lIKwMAG1WsqAgzhmooejKGERYTSZ81C3lCIz3+CDiYlJQqGAPeV7ALBzsieu4xB21mWMP5cCa7rx85PS3u5MzZjbKbY7yhjFi/gq5cS2bsbd6UdMTiPodD3ETfbf0ZRagU+DH12zsimfepr08DcZ7BikzacN33ZfFCYdH8Tdy8tRhZzUSRipm0NJ1DUmXliK06ZmXh+3gMgHd/Gki4bfTSrs1K6Y9mWR5uCAPPHG+m+1QK4R7pN5k3bqE1oiJ+LT4UObTxsmlZGkpCQkSLjzzJ1c+f0KLe+2MC/Og0FZP6UmsFfYUxNRQ01EDck9N7P4k1ikSgNjmkIob/DFM9SDK4lXyDyTSYQ6lmk7l+BcfY19VbfgGqNlRJsHCQW3D9NJWqmS/JlPk54hJ3KEjDXn1vDdaDEX2rEZebgaTKQUBuPfGkB8fAKxVa8PZbMdcWUe8nU/EP3ifo4alqGrGcNPN//E2CfH4h3nze6bdjPh+QkoRyrpbe2l9P1Sxj4xlqrSbjrfug91tzNhCRB+Rzj7PhJ1XOazmdzqUYOzvQNxeak0y3Us27kM6ztW3lKJdMvFCaUcGNAzd+9NuESrCBjthEuIC43nGtH36Hlz6WQOhdoY3QBeXt4kJfmRVJ9E9dEafn71Iivuf40P28ewJaePWD8ZbgaBwS7xcrJ70W6kPpEUlsyncpuc+Lh+pFI/1OobtJRubgJK5Y0LAoBCZYF8GFk0ErU2j9jY5fQfvcyk4sNUCHfgMF1O5ZhsmiJtOPfezeqJNi6/14DUZgGDCTkmXCRSGhKWUh8rYWKKEyWT1hF//nvUQWrOeB1iin8/34dKefpaELYCG/vuEt+ZEgibXotfWCuf/vkOmhIPooKjcUxWojFpCK24SkkowD5YCKOuiXtCwioxEr7ktxIe2TyfXr+dSD9OJjI0kqPfHsXTzRPPLhkxh+ay5Y4tFMS+TVLSlyQlJTF93XQG2weZuSqF0TNNlB9cSKckgoHJzRi6Dbh3u+PaqqLhmaepm3qZC3fvZ3VjFLrGrVjsTMCYofe5slLNSXcdUx09aEyci1WuoM83htCst1EZu6ggkW9eT0Dh7oNdei8JZ3/BfqRoVhkUJLjGOhJ4l4omt3Qy5llJnp7B7t0S1BUS+lxG4J4MSUlBdEzpQNehIykpCbuX1VxRelA+fjVyr2eZ07CGEWX7aGAWrfsF4sIqmDlmD+qRe/jM8gqBZWPwbCygJPNB3CZvZXzJRD5u12P8oxL356Yw+/BsURddN4bEp1XiXprChQPj6ZpnxPOKSCnsnhhFpKc/EUE2gs85sHDbNOT3aNCHq2k0NxIhh2+nPku/LA11N5y/moBXpDsJl/ehU7rTFxpKem46Xj3JJL6ZhEQC6mNqOq900mZw5evqaWRcyiC5aCxBc/3w6vBi9pHZmOVmGh5t4MORP7OpF97tNZJrtvDwJw/T79JPwZQOfBqX8X7eFN5flIm34ybsRz7JheoUQsygHvwUrZMWa1op49+ehcbFTIeqjMirEzl4jwK7t75gCiBMOceqLfMZUTICy51WmhJnYZMYiVuRyseeZrrdN7G12Q1Xh3qio6NxcpbSOWIOu1Y9j8ahkM8yDpKUlMTF9y5y4tkTjH9mPDUzBnFJ2k/+oB3dppuIj4/ndfnr+Kf7M/3d6Wj9omkL7UFuliOx2EhbnUbD4QZKt5XirPAgycmNXTI57ioJXu7hyCf3YDpgIqIwgn5HJ5TNQWxWdaN5dhMrFZ/T6bASm53y/2Lvr+Orutp3b/S7fK0kK+7ubiQhCe7ubi2ltJS21F1oS/WpOzWoUNoiRYq7eyCQkIQQ4u4rsrKS5fP8MWl4+PX5ve95P2efvc/57D3+gcw1ZYw5ZN7jvq/7unBQaEhztPGeJ1wNG4Ek14JK34o5MxSdUYZao+fZNgg2JhLi6s2hiVcYPXghDb2ZNKkkDLzxl/jttUvx8fYmKSkaD5MHaUfS+DbxW6IJ5+TIo+xY1EOm+yg6KjtoLW7F3e6OKlZL941I9APKSVtyEOeoD9BqnQmvvsNuk7djCLLBeZiEVjwDvSlLG0DkVdE26oyupOhSAjcGv8mpkDeR9BX2X6fz8mT9shdA40x8eSKrfp+P/CErtSOmk8MlJgqJaGSFWKXOxJb3UGe14+/gz5xtcyhKKCL2pkh/FDl8NklJkXRUdFDyVwnBw4LxOaLgE3s2Pyy4yH3OYiblmHfG8NfSv9DX63GxKumISKGnZRNKi5aYsBgqzPUEDA6ieH0xwTzEa9lXufL2e5R3zWDa2QeAKpq9m/G55YMZR96s1TDrz0mUDWql0ycGra4GRVIqp30ep8RnLwn2r3DveRBBEIiLs7PDoxSbXI0k0I0cSTzp4aEc35JEjFc4DhYZKSfXUhvlQ26Ynj1T9/D+3PdJSkri2EvHyPlMpPWV7NlG3gN/8fDaudgVvqgV95FQfQSfapFKuT7Ul4Mxfii8n8Bqm8JI49tsQmQP6Hz3Bk6Xl3J4/CUqK33wmG9j7NGx+LT40BLUgdOqx3jqzAzGKhYi3zeYniWF+DXB7J1LkCTL0Q600GSJ4s3m+8mv2oBG442bQYF3WSTHrXK26aHJHoSToxKVwoyblwsSpQwJAj2haRg6q5HarSQkJGAxWHALcmPKe1Nw9HFk3HFHfKNquBRcgIMqm6SkJNQr1LQOaaWwu5bvKnZiFUoZqvIhI8CbCTMmsPVXH9ybirEu/o6CZD0f/jAJ46ksEs8kUvZdGTazDUO+AblNjkSQoD+m58L1Czxc8DAJYxI4+sJR4i61sIk1PPHZF/xraDkbXMwkpb3AXw/9xc5zO1llmshTbmKW4EWXdvw9HPo1dqq/ribvxzwAFu5eiDTljo+lp7kH15gmGup6sai0aOVKkpKS+DROhk/lJZxa63FARUNAA7nph4lumsesyEgGn30fvWcYQU+Npe5cHT7NPkSUR2DwsCD1BrVaQ2XydEKLdpOZk0lFeAWCxIarqztlEVkY/WIYGBJKQ52Ycenl28mhiYe41/gWAX7B1PiE0ul6Rwdthe+3FNyMRRamQFBrsGmccXAQs+yCa8UMLBdvFyZ/N5m+9j4SpyVy7qlx/K6wIDGpcIw5T/zoh9lxXvwOCrJqQsoiiPfs40RwOuHJAcisIShyupFZzfQ9rOF++TWUutGkuc/EyVXCtN0OOBoeoizGiB+BOJZkUjX9ALbCaNwXadE03qGN9G71Zl2PJ3KNherjs2gcpMKs6sNk68OnLgBJfhCqCdeZbanF5WosvSHNLNq0CIBm72bcdl7jtK6akWtGEqC546vy0/jh1BePY1c7fi1xhI4Pw7vnNK6Vou9RcFHwWCtInEqJTlzGg4kDKas/xUfPXSJQm0pAXTpeGVto6XPgkmMbM13uoybhLdoNnjy8fCY8/RfnXXNxMziiNTjg0FqNS/5xADLOAKzh3OPleNeEE3V4DJKH7AgC+Fl0qK/HUAyErEzkWMJswsIgY7ydT3+RYgBUMgtDlX8RlB1EUlISMe/HkDopFbcINz79bh0Bf8ay/J4yLmqlJCUlcfiZw7iGuFKxV7Q1d03fxYItC+h17aVtehtnXjpDNqJUSWDTbPQLdxF/NR1LnIWGHxv6adsB9kzdw+qBrbg5+OIzfQHBPwYjkUpovtaMf4Y/jdcaEWwCGp2m33ad/MZkYgfH8lT+U9RU1pByPYXMnMz+fS9Ah2MHZzlLa2Er8bHx/5SBcPPDr/Ya3iePEOH/dL8udGdJJxadkZ6QRFSmHjIv/YTJ9Aa+vnd8lhdkxfy7EmBV5mKG+UpITvUlZUAKUqkUm81G9sI7tqFEImHDqA00nKyiNH0+gmc4wcFytFpxP+vh7E59nw2F5XYGtkxFafa9xMcLJCUJfDbhM9yj3Zn35zw6qzsJHx+OTHG3FIlMJaXD42j/32HhYah62giqPE+fiw+FK9W8M0ik7Y/SprD90R3cLPsQh64G5o7r4pmmZ+m5Wc2QkzBckPMAVnwQtfzWSVWcE0zYxkCK7xqCg4Nxc7vjhx3685+EeS5l56ydxF/ZRJ8li6iocWi1UlylXXTv7ybn0xwEuRztsOn4+mpxdATprxtw1tVwqWoCmVPTSJ0ZTGJSIuH7wpHYJThpndBo1HA7edzP3x8XNx9K0+ej6u2g3dNIQMYYPtYq2b5Kh6k5FodAf6xlVQy7+AOGJ1xIU/8LV6WcnONp+I7yZM/A37EoLFi9BjF2m4aZBybTnHoW56I2nHxWcWjiIS5lXeIrn1N0bN1MQnYRNquUmQficQ70xDnJk9OZf9Hu3MW5HiewDue6bi5nC5uZsnQCm+MaqFb64V+VQ2r1ZSwqR2IjHkPhoKDmbA2CINBzrgeAXk0vuZNEcLq8voW/BU58/tVOVeYk3nXXEe+pIzYulo7IEAIb8lELdoS0NMxVN7kYBF/rLCRPHMaRC1E4ddUz84VI8vWrqHKwcz6hAIUmjHgBBN9wun3Difn1JEd4hp+e3smginBiSmJISkrCTdGLXlFKo9cGnmj2Z1sonJx4iK/S8/jXoPd4YnoNbXoVBucF7Pro3/lhxBiAh9GDo+8c5VLWJQyOBlbPXY2bmxuLty+m4mgFoSNDcQ8VGRb/jgHMu+8+Nr7/Pkv+Lbj135VfVBq0IQ9SFPUkIFLB/u8S1/jv2mSz2XB3d79r7fu/a9P/yPL/E8G+xx57jL1793L69GkCA+9oHPn6+mI2m+ns7Lyrw5qbm/H19e0/Jycn5677NTc39//2979/H/v3c5ydndFoNMhkMmQy2X885+97/NeiUqlQqVT/OP73vf69/D0A/tO5/7OPSySS/3j8v6vj/9Pj/0/qUripEM+BngxYlcW1h3IwOBo4OHEHHs7jcfUZwIGGAYRc30NcTQ2RHcMJiq7hXFc3dpONksH3o+ppI2rPetRtbtikNjb8PpauY4MI3NiKTXFnIk/eP551D/3AIfu9aAOsNOc28NesXxn4RDaeAV488MsHPLRkG2ZFHq/khKEfqkMmC8R/oD/XRrTQ7tGOTCanNnEyranjUWgUSBzEyT7qzVH/bVv9G/0ZdXIUD7qq2NIzGalURnXiFOLO/Yimp4W2xEbIj8Ke8wATqx7GaaUrOr94XFpKyR/9EO1+g7gkkTBi9WrK0gzYnJpwbm+hLm4sjhG/kKZZxQiPBtY88Ct1hikoVIq76hG+ZB9PuFWyK2UGRV0T6Wvt4/u07wmbk8bPy3/HJjOQKETi3uaBTLDTfqud679d50pNLzlZ4pwSxYzvtCn3W/FjF1MSg5vODVO9CVW8iunrp3Pl2ys03WjFL+AtNo76nezjIcgVSkoudJF7rBMV4N/gz5KUC9y8EkvRxUQMCYeQShYiN/diz0ynSfor+34ajH6whKrkaWS7OvKJ9yeEjwtnyEtDmbpvKkGGG1QlHuXUqBPM95tGc744b73mjSC3zZPrVj2jKk5g+HE6TrEOd429Ya8MQ+mk/MeYlEoFjs2IISLvEOMOxcPLEhpudJFwbj0AnYPccP74PXZKzGzRw0yJhE98P8FqtJK+Mp1JR2Jw7kugIrGIgMIkLD0Wak7XUHNazMJMG/or6Ym7WGXSoDWE8bZMhluIG9sXb0eV0400yo8RHaOI2G9l4JUurL1W9qzYQ+2FWvqihqFzv4M2EQQBmUyGqdOEvl7PxtEimnfxvsX/EJpWSUXD+OCkg8wI3oJMJiNkeCjmXmv/exs19zgnKvwwybJROCjobevFbrUTc08GK9bDgDGXmb58H7v0ZmQyGUFDgvCK90JtsvKO3JlNXn2sm7qHpPhZRE2MovVGK/pGPe0+vVzXVKFp/YHA/HuZMU383Dx87WEKz3ayfdgXDDk3hBbvFnTuOqRKBTKZDLPBTHtJO4OfG8y4D8bxn8r5UfkYbQ1U+5ZyvfBTohyUuAS7oNQqkUikDLycgdQupS4yhfgQb/J+yROdzfMTqCrqYPDUdRRdTMC0q4GelZ08mPMgv43/ja5qkUq6aUgDWz1+o8kV+oRmEhcl0l3XTcu1ViakdfPxsFJe6gpErw1E4wl/DyW3jCjWVX9Mh1sHR8cdJdA48a5xFjsjlld7X8Vms1FQUHDX90KmVlIy5AEAHPzEe4aPDReDfeMiuHYwh1cPvcqqj77mvDkWiWQKSYuTiJoUhWuYKwNXCExofI6Pn/+YQOsIvghxBUBfp6fgu7PUZsURm/E9/mGNfFc/gohxEUSMEzMx47Ie4qKuiaGKQjoiapjeoEcmk1F/oZ6WghaiCqJ46NJD/Lr0V65kFJOckHJXuxbsXIDNYvuP63zztWbOf3CeWb/NwiNCpNiJmR5De0k7keMj0fzph3O7ncV/LKaw8zKqoZkgg163AFQ+1ZRGlXL5XCLHt4/EcF8ebSEr6PaJRmYx4tDZQI97EAq1jJOT65kbnEpJp5hB6nvrBJLqQDHwKRQxSL4Bc/s8Kn68TtLJQtqmiBRfUkFOcn4yY46NodurHtmgLC4LGei8PiOgLoC52+aS51XA4EdTebbxWexWO31mGR9nbMZNk3J7ToK2qgDPy+dx0DdTHh/E1WW5+J4eQeTRUVhn9+LRM4z71qehcu6g08+OW0UoP3pv4fjwcrKQM+7wOA6PP0xUVy6Ddn9BQW4Ce0fk8olO4PxlG4/mPIp3q8ilX3w+mc8mSXk0ZTsJ731BhescTF+cwyXAyN5emFcwk38t+YGMH+PxanXFnizQEjIQg1sg77/4Ps7dzjh3OiOTybBZbKzLWEfoyFBc3hjHW5Xu6Mt+YkFxDMZZonbj3D/nInXxxBDfwsIhF2gaeI0nK6P5RrYAz1ZPbsbeZMrUpVx8o4PzOjk+FYXoVS3UJfSRcM0VnZsOj69HIVSvYoTFk7KK3TTLQ2ktDMVqEJ1VFqWVpswjxMq0DFh0jLPtfshk71GauQSbQo2vZzWebg04S/YyL7aCI1W3RHtGIaO7ppu9K0QEqVwpRyKRcGDVAQASFyaSd181TVXi+uN/ERzcHMh+OhuNhwaXEBdqjDIkrkZe/OF9pjXfI479i/W4RbjR2d2LLlqPxDmR8rQ53D8ghFFviIb4pS8vseOTreivhVFQ78rE60HoQ3W8kyYGL0f/awxeI/5AJrdxtdOH2Zsn0jzUjsejKRwuDsa94me8lTb2Bu1gh3kpF0cPIyZShaxQtNXUPW0E3DyG4O+PRCJSLv49x+xmE8n5yZwefppuN0deR4bg5o7RyQulyYDFX0nNzAM86Qp7Cz7CYH2YltBMAouP4DQum54jF0k5vxaH54vZUD0Os/1R1H1dyGxmghZmslT7GlM9ZGz410q8eqOpG99417zeW/0Uxr4HOV9yjdiS45i7k9F6OjB903QGau5sMJ9phRqLHIWDgrl/zKX1ZiuHTh/CtdOVhhixnY4ejgx4cAABWQH0qVv52fVfGBwNeHRLkclk/DLiF4xdxn7NPoXKQviAKiobxqDu0xNzM4ZFmxdRg6hhu8m7DnvJk2Spn6Wn/jHSrmWzd8peZlhWc8HuTLHhGhH1ZaiCQRckzmGfiovQbebGgLn0WC4TUNtOt1WCzMuD+uiR6N3Fb2m3TcoppxtsKoShQ0EikfHgg6DX39GwEwSxn/zT/SncVEhPQw9jTqfhoNWzObIa+gScIvwwuPgRfOMQeTfArPCgLsSVjrdW4TFWwKB2RmK34dJeySDHlbQVruT73i9x1hpwVxo5M07F2H2g97DSPvEYXsnXOdroQvOcVvxc52CVq5BbTZTl9dJ6ZAD7Vm7hz9RA9uglGEIGYb9Ny9d2G4DqYssluHoA+b/OwBKtR2YxIZNb8OgZQcLlVrRdVqRSGVIpRIyNIGJsBOfPg3/hWvzaLuHa5YxEkDDytEjLWDjMk+gtVvbXZdFrtTGnJATLFCsaoxseOg+oBL17NbKLUrapVlDT4Is1+izFQbtx6U1DP2Iwq8c+w8cu0PfxWl5sjCTvZgaXMs7hysNw293SprJSFVaHVa7ALkjQe0dhlRrolFuotYJWrqQ+dgy9bmDt7kXb0YbR0QOJIO5h7IK4JyrbL2aS6cp0hB9xYIg6nrELj/HImY+QySB1WSp5v+Tx66hfcQVeHHaEId88T7ciCOn7UjxjPdG4azA2GJnz2ZPMenQ7P4y6j/QxSXxRtIR0ixpjopwAl4FcTfVijONJTA7nEJBidhQdK5GXN6Pd9xKzp+/j9dUK/GeYuFo7F2mclPMJ+yjzew8XQwYjen7khQHP0ech0JqwhOYmAQlQnKRCarrOpsWb2Jv8BjKZjE2TN+GT4sP0H6fz46t7CW9YiUbxCFmpXhRvP8+R54/cNa+L/eopMkO4RCCk8AA2qZzOgePwMdbQU9ZARXkQ02vDkIy10+0ViV0qRyaXoKkK4CAWNPEuZPSoCRsfxsPXH+a75O9wb20jqDaI1LQWPosv5K80d3ReCVgapLR7HOeUtJB3rj/GjBlP4bduObMvZqHzaSWpMIl2j3YqwyrR2h/AKzEImUxG5ZFK9q/aT8z0GKQne7C6xrHMLMf7YgZBW/RsnLC5vz19F/ORzJ3AqZGncDBX8UGTlYIfr9AXkUDW0hSu/5pPYZ+UgzYz4XIBc2IyPyQtx6w089jax6gLaOJa9E0cJx7G4B+CvGUwzRGDiQ+DlGI5D+pGELzwMK6O27lWNRqp9HluZKVjNXzCyMNunAzpYqJ8FJOOFCCPUSOMAJtCg00mpc+hT9RpS3VFJpP1a3EBaNolPPPhLA5O2E9I24vY+qz9gT4AhRDIMp82VoY3M6mqEp+WYLJvPIHOSwpXRV1JjaMRfaUUY+sNDk04xNKNS/GudcPy+zacdAo+H6AgRduFVG9lzLEx4tplFzgrDeXQiadQj69kxDcr6Ig0U7nQgR/GiXSX85pgvGogD2n3Mt3kz3fdOowFF/C5eByp3UZ1wiRawzJROUDbzTa2zNrCsNXDGL56OOH378BwbDAhx45h7kmjdF8p5z48x6yNs8jV1aG4vA2DHX5IuMHitMUIFgH/sjOo+rrovhTB17fkFLs6kz7Hm6DYIJadXsaP2T9i1ps5P/s8Wr2WlE9TSHBLwDXIlWMvHKP+Un3/e+tocSUxs5iq4BI0zhoMfSJdVm73Hd/TK30zMbg9jUQiYV3mOhouN/T/pm/Q9zv7ZDIZXRVdtO/LoScgmZCCvZidRiCTZdARkES3VwTphb9S4i4Gsa6mX8XeshCpVIYuMBmjkyfmHjODLwzm7JCzvPPaO9zHUSR5IJFKaQ8eQFaMDsmfZ5m/dT7OPTYkL0vpc/Wnz9UfqRKqetV8lvsee3xExhSJBBzDfLkwxoNbUXco5wcUpKE43wJpNszDx1HiA0hE3fmPnvuIrYMuIpFIiF08kPJy+HnwJ8jsUhKzi9gbdYNCYSCLpauQCjYkdisqlZaW1cdQqHqpOTeVhEEgOw8lg5aBREK01zqG7xrO4LE1qCPXIQgPUR/mSXRhO5ElavThMnzSi7D3OGOrD6C3qQff21StoT+No2r5ERwKI3GLbqAsJwVZnIVbQ4M56b+Kld+tRN7kxxMhUTxqKKDs9BAsg/ScH3wed507gy8MpqOlkDplOBKJpN/GVLuqUWnVjDl1P1EFzTgZnJA8IqUnNgNZTRU9bkF4P9hJbF4sQ0Jl+Esy8HYIxK/Rj5iSGHzMoajNGvbMLuEbPztXMyToKstJO9tAfsoxXNMeoyq8HG+fRt556ApftjtgOvUy/7V8pbyE39Qm1iVW4RB8Px5VuTh11lEeXk1O5nmWSzbSpj2OUq5AIhneb1fEHv6a7hRnMldlivPbWUPMNFGH2owZicKKZ4c7DlbRbrr+63X80v0Y99E4vln3J8VxxUgECZEBkfil+RE2JozKYyKLhVerF08ZfYn84X3e8H8C12LXu+qsMqmY6VsDvus5mbKEWDcRgBE6MpQVl1dg6bNwcs1Jjjx7BL8BfoSNCmPjqI101XQR7xdPTEsM6x9YT69zL2/I3ui3Yz2jPFl5bSX6Rj0KteIuXT6xSNB7hKBUgcpB1X+dd6I3tWdrURp76PCPx+Tkickk3OWbdMxI4FamGot5C7nppfj2XEIiGUxrQRM2sw1Hb0ckUgkuwS53PdEnxQecteT1BaGw9JE/fy3KlWlM+nISBe/tQwL9GsYhLY8AEsxmCTIZZD+djVKrpPJYJV4JXijVd0Cjdqudk2+epL6hF/zBweCAf4M/PfU9eNSL9ofV0YXYtc3UDUghPzUfm90GSJFIIOHseorPgtdcN+pHWFHFq7h1IYA36uR0dvXgZLLg6ToI2/27QQtxfxbRIjmPdN4wDj17CH29HnmfQFBtEEa1kcqkibiG+yGVyggsOYZnYwEn9nQRPy8eh6Vzufi9RNyHqKHHIwxnXQ1tgQPYi4qV94htcvZ17m+fALh0ujDuyDhaNbX4TYmh1y2QqNytSCLDaQDClHZ6Y0to9bSSOPxtbrRoCb5xCGm3gnsGNJNzOJPDGybjHG3kyHjRNgnUu2NP9sChwI8lzgKn5XbsNnFSdLp1Uh5RypXA9+gtTOWptHvJ2DyX9mwrkjfUHJh8AK+uCXy17yADB0JvTQPmgzfRxVVh8YmkU+9KSMH+2x1kRyoR587Pw34GYMADIoD5iye/IBERSatMiSe3zheNvoWQwv1k5WRxaNFmfr93Jx96/YBJ68T10U8S3nIRe6qaQXPO07ZpMlF5A1AcUSO3mQkqPkrtplbOdU3lw8En6Z50kAjjPBIL8om6coOahIkoPLRY2vXc1/Eq5fctYXOXG6ulbxJauJfkVjtxH8Hvvg3sUj/DniJHYmUuoh1xZCtuSNAFJFJzOo6gwUHIVeL+VyKRoNAoMDQayL6YzdrH1qKKVyGVSvGI8Oj30fxd/p5PD61cybtvv80l+I/ZpH+XS8Bpq5npquHEHPmYytRZCELk/zZxjf+rOv5dz//6+393/v/I8r9Us08QBB577DF27tzJ8ePHCQu7W8QxPT0dhULBsWPH+o+VlJRQU1PDoEGDABg0aBAFBQW0tNzBbxw5cgRnZ+f+6OqgQYPuusff5/x9D6VSSXp6+l3n2O12jh071n/O/yn/48v1X6/TdrmNs6uPYpeKGjoANqxIJODcWo5XjWgkl+VHkd87icppy/BMDQJBIOHUt6jbRCM+b9oBtrg2UJBYgCCTYnTyJH+smHkZ0ODPtiOzaBn7ByfH3qKyU6RZvfzlRZTNEoKq+mhqGY68KoAxx8dgaurh1FunuPDJBQxaC2qjGoOtk6uRq7gY8RQSiYT20nZ+Hv4zb0reZMOoDf3adv9e/IYO5pH3v0Eo8yH9WBeCAEatFzKbGe+qKxxN1bF+xQ/YT4QSee0YUqWcivR5XJv0CnapnKu+9Rz3qyE/OR+dr5m+K0WE5+0k5vwvaL5rx+Ozh2ip8+JlN4hWdbFx3Ea0/nfQADXB1XTa3XAZ74hp+DAkMgluYW7I1Apcdc74W1Tscyvmsa8fJ/GIL+2l7Zx++zSRRa3E3IxhyNkhNFy728GX9WQW+aOreWf1Oyz9dSnVT1UD4J/uz/T102kraqFxSxTtjR5M/20Jmcc92PJyPqrWOjp8Yqhcu5+d38wiLKGCrO/exJjpiEddHma1MxJDD775Hhhr/Hh5xG9I49xRujqidFJStKWIHwZ8j0uSjZFTjlFr0TJMI+orBQ8J5t4j95KfUcSfAxfzus6NdvcsfEab8QuPoPZ8Ld8kfsOuB3ZRc6aGwKxA/lkkGJ0cOTLuCH/NvowggKKxrv9X156QbYcAAQAASURBVK86mNHtzdhed+KL4rHoTDhni0GF3O9zUTW7kjLlLEuWHeTyB2/imebDnE1z+q+3nQ3EtC2bmpAaKiLraC9t58TrJyjcdBv97LyS4qhd/Dz/En8s3oXKWUXSkiSGvDgE96tHSb4uiiwPzBmI70Vx2Q4dHfp31cVnWP6p64XOwoCrGXi0edBtEfUvc1uDuboul7IzTfT5dJIfWEPRwKHIkp7CZrERPiacJQeWEDIhFoWDBQdtH5PWjqcvz4/Gq43EzY4jZFgIwc+o+OyJZ7jR5MrwsblYgm20FrWyPnM913+7/rd6PD6SbtLyfsF4TKQusNvsWE02ctNyqQmuIW9AHj898BMqf9Hgbilo4Yf0H/jY+2PKDpb9h76ChpB26gPqCakK4frFLvzT/fHL8OP0W6ex2wWm7J9CcE0wGxc8TFdwH41XG2m72UbR1iKkfoGsLrzIm97VHJq8jy6nLgIGBvDABTHQljPkBhuG/MZrp8fz2sfPY+4yMeePOSz8ayFqHw1dSiNtVgnjMwfjnjiXDuMdXdFqj2rODdqFwUl0tkiE/x5P81+RO39n/cvNvZg6RHqWlHtTeEN4g0HPDMIUoKAsNY8jdiPb5CINzaZpm9g4fiNylZwOr75+3Q8BO64hrjxRLuq7VO+9QV36s/iHifO5xiIGjXLW5lB6oBS1pzcrJn/G40nb+PaVlUR+KYqK+2f4M+KNEegCu6gNqsWoMbJ36hZuxt693uVvzGfDyA10Vnf+o52B2YEs2LmAkOEh/ccS5iWwImcFfml+jL78AkkFSRg1AjKVApu+F6+qywgSKd6z0rn/l/v54VQ8jdiwIyCx20g+9jm+5edoDx6AycmTqMbVJNV8i480rv8ZBjc/8rIvonHsw2pWYNjeTPfxv5DIpNilCuwSMSAhRS7qO2r1IJMgkUkRZAoEqRSr3Eqfpg+bIBpjTr5OFGwq4I9R63DqqMOlS8vax9YihLohMxtRGsVgccSNWnxzBnA+7QrfPvwtigBnZm7S4tSjQNHQgxCo555nNpPlJHa6Rd/LkPNDWLpxKX4Vt9iwdCPvZh/k4x8n0PDNIpR9At6td+tfdOWvIbA8mjmNYjaD8vWX2HDvJ7TbYZv6L2wKL8Irwrjn93sw1OqpSZpMl080Ro2Rpb8uZfrOyQCYuk0EDgokZmYMISmDETrv53mjN3ElMVh6LSz/cTmJRYnEn/ekal4xjoOv4SsDhcQG7VYeW/sYw88M5/KLB5EZBUxD2wh+/2Nypp2iOdjEm6+/yTePfoP/Xw4MOpfIV6GF/Br9FHbTixx86iAbRm1g5oaZGB4+yTGJD+MdSliV0ExodDV2O3R7R2JwC2T8gN9ZPW4+F+wDcM7Vcr3DB98UXx4peITTb58GYOKXE0laJKLk3CLciJ4azbqB6+i70cDxVVU0BtZSvMCK0lHJhE8nMPzV4VhNVl575zV2/zSFzNzniNLMoymviZ+H/kxHeQeJa5bxumBhW+hx5JHOqDy1dNV2sXXOVg4+eRCHGmdObBtNRVE4srOZ2M13tI2Ov3yMa/ucefumB+M9b3J94Xba0yRIHdT0ufjhV29h8J9zONkczmnH4xwc/gBV8lv9zh2Hzgbcmm7ifvX4P+ZVn87A7J2zCa0KpdvZgNUKet8oBImEiKvb8P60nekV95LdGIRD6X5qa24itVtQ9XYgDw2kdkYpmeNzWOZ5Aje3NzF1dyNIpNzKXIJLWjDFrz7F97+O42Z7Oj0Wd2riJ3J10iuUZS3GuiiTJvcSclpKcJ75OftXbsTRx4mN4zeS9WEWew2wuk2sp/vux1m4aWV/vQWlkh2zd/Dx8x+TP0Bc57qqu2i62oTdasfqLKc6tJrZO2Yz6ndxzGvcNWImWZ6YAXxuz1B+d95AVeRkhDoDizYv6r9/blouuwKryX3Ph849RRwef5iCxAImHpxIcVciUvUoYnrdmLdtHm5FdiR2G363ThJ84xAARbb3KTPu49zqT8hdVInU3ZXG6BH0uImb4EQlZFd6McS+nK7GfE59ewO/wxtQ39b8Szr2Obmv7gSg/VY7pQdKMXWbGPjQTubNPcX8917AZ48MtZcWnX8CJo343VNalBzZPYTSsErMXlAfFYfJQUv4tR1YG5twVTeza9DHfPrwK7R762j1l/HFE19w9t5WriqX8VXpata7NrAnKQ+bqxMV6fOojxlFr4uoB6gsK6LGzYBu3GkIdiL03QKW/foCXXb4plN8d+s829k6byuKUC2V4x+iMm023Q75nB7Xy6XxWX9/0incUsiZ984gCODbOROr778oGv0yai8tt2Lb2T1tNx1eahzrNNz4axSq8hCSCpOwWW2cGdt7ZyzIKrErL3D0lA39xRu0nr7AhB296GQfIbPouLFrJJuuRvDo8evYGs+g8+qjNLoUm0zG0bfaePP1N6l1z2SSzwd4RH0AMhmdDlc4mObE550wuA4MdjMBkS9jcv+G1txqYs//THDhfp7+aDpr1qxBd0DMeliwcwFPVT/F5LWT6XW3celwFsWNAQg28TsxYs0Isp7Mwj/DH5tjH/7h9URl3MQWGkfj1UaamqVM3fMQQqX47d+jtiOTWhEESNzoQnBtMB26Xnq2HSK2ax+bQq7wcUMcPk3+KHs7kZn7qI8bi84nHLNlP2Hmmzw95gFcffTopdWU+b0HwJNBV5gW+DlKiR6NQtSw7Np5nIx9b1EVrWLTYlHL3HJb+zL94XTULmpCR4Xi2SQwMCcCbbcCW5+pX5/w79Lp0slbbRIU6xZiLG5H3dOGWeNCT0AMliXDyJ08hie71KTtnYyiXYJjZz1SuxXHUQPZOecgVxZu4/6TY5n61SPY7HYqjtyhVlq0eRHvdHvgF9pE9/1bIdEZh7zzZFwKpdoKh+rH4dq7n2eC9fT5tOKS4slbr73F2aFnGZA3gD77Bs71iFSf4ePCATGQZP7Ul80LN1MsMdPpYMD+bz4SzzhPsNkZt+UACrMCAYHqNgeKhzxAecBwrCbx+zt5+V4ymv3R6hTYHJzRueswK80krV2CMeYFXlJMYdu95/GNr7kzdg29hJzzR39sEC62aopzwjEUdWO3g0nRSGXAX5TElFAZsAM3w14cG6tx67qGo58XN4atoDY6uP9ef+uIZazM4L+Wi4MuUJKgR+6o5FbWPdTGiYAH7+orxAn1vP3GMgK2x+JY0sjEPe4MuPkMwhdB7Hvya0bNPYHss7fpyRQz9P8utoJiPOuvYxl4gR8f/BGzswyXbhf61H3YJjmRfKmSRyy/81fAS1waeYqGAZ3crSAr2phllaGse+0hLBc0yHs6kNptdPpEY3K8E7R08nMidFQot/bcQrALvBpxnS2T9rF3+i6UjkpRt8YmsHn6ZromlfLKntkcMwYzRC3aNX9M+QNVn2hXBZ0Np33PKArcxnL/zyvoqejhx2wRWDP8teEM3zScsUfHMnzfcB7Pfxy7zY6uTEfY6DCa34/i8BPfEBhZz9vtsENM2ECyTcLYI2O56LaeB3NWY7KqubUulLKnxIxVlVbFgAcH8IbwBm8Ib/yjj7ziRWCxRt+CycENm0IMAjh21OFXegbl4AykNgm+jSJw3CaxIAhQnTyN5vBB9DXpyU3L7dcytWNFKgUEAYnNis/MIWxa1kazTzNWuYD937rh0CE45jKfHzxewdiYxAM/v4pbifH2On33ftB4oxbvmlzs/7ZP/LtPkwqSKHjpKPoGPU8+auXNFwxwe38ycto5fh5VhsHXQFdJE+n73yHt4PtYbvRQfeU8zpZAfpkn42LfKrwu7yfpxJc4tVejeVtPyvUUonJ90a2to/biRf6a/H7/s7UVPnyYcIWX7j9C9lfvY3ZSY3DxpzpxMlIfR9Y9uI7H4kp4NFDDu6+8S1+iCx3KfABcDCLwZnjQTT4IquLE81+giPfmysArd+1N7j18LwB+aX68IbzBix0vIpXLGHDRgEwSQmXSFPGdO2hx6GrEqaMOoayHhVsWMvlsDEGXdmFo6GTmXzMZdnYY0TlBBOdpcEDCABV4y8Da0UtyQTKunQ40C630PPwbG+deIUQBEomAxTeIW5lLqE6cTOWsZbzw/QdEJpfzh0zPbp9apA5yvGquom2rZPPiXZTEloD1IvcF34dC/yjF2wpJOvY5odd2ojTqCR11x09a9GcRu1fspretF1OKmhvPfMvenycz6JQrAKturqLhSgNHnj/CxaGN2KV2zJ4PsuXbLWjcNSw9upTqhEk0hg9CbVJTURiG2tGITWFj8HOD7xo/1SHVvNAGj7fA8crjtBS20H6rvf93iVRC8JBgoqdGo3IWEyFcQlzo0/XhUeSBd6s3Dr0O9Ln38V+Lb6ovUZOi/kOgD/xunSK46AAd8UP67wsQPTUaALfmm9hkIqDHYvkvDnKJBKeOWhLyrARWVOKgK0QQYO/De1mftZ4vwr7g85DPOfzcYfI25PVfNvHziQxcM4XUI58QWHyETtdQnMNua0F6i8GtZh8RLG2WNRNw8xhB3aIvZ+hLQ0lZmsLhZw/z+8Tfab5+J4nE3GPm1JpT1P8ggjb8G/y55/d7qDpaRYdfHJWpM3FsqURbDbP+msXwU8O558cl9OnEd2aTiYHDoRfnM+HcNkxeJoSZTrzwdglDRlUyy+sjlBmtcNu1EFEip/KKuHdoLWqlMbcRU7AXgkTAorTQEpqM2Vf8Djl0NSLrExdFU7cJmVx8l4IAsbGg6hUB54L07sBEV00XnVWdAMxwfB+VSUViUSKG8k5RD1ChoTk0ky5PD3JN4KS2ErxqE41DRT9vn7O4LiovG7n/x1G0BNdQ8sBG5GEOSOwSkvOTefCTZHolTrg98yu1k49x6Mlv6a3Vs2bNGtasWYP1WCvVnt9z0FaDVS4QsXILumFGbPa/dc/ujCu7SY+Trobe9q7bRwRMamfqYkaTP+GF/mQAtwg3tP5ahq8ezg8rfmDEqREk/iEG+yTOWno8QmgNHUjUGwvxDWnk9zAjKWqzaBOXnyX86nacCi/Rc2UnQzSnyNdHYLQFYdKbsagcaQ1Oo7ughqxLaTy+7inWeUO0ugOJTodr8y2Sj3+JYBdQTA9ibMJWfg+2kOLWK77TVC/ODz5PpQVO9sIM46ccDvqRkYUDsZqs1MZPwKj1wqfqMr+O/pXc73Pv6jPfFPGde7V54aZzw3rbVjz60lH2PbqP/1R8fX15/4MPmKZS/bd6kZeAaUoVjz72IW5miag7yP9Y7bn/fy7/o7P1/p+U/6XBvlWrVvHbb7/xxx9/oNVqaWpqoqmpib4+cWFzcXHhgQce4JlnnuHEiRPk5uZy//33M2jQILKzxbTr8ePHEx8fz7333kt+fj6HDh1i9erVrFq1qj/z7uGHH6aiooIXXniBmzdv8s0337B161aefvrp/ro888wzrFu3jg0bNlBcXMwjjzyCwWDg/vvv/5//Yv43KU9VP8XAxwbi6KOlxbuFyrBK7tl4D9rcXkyVDURf+g2AypQZVCdO5ivPK2wMH8WFphNo2yqRCnZ6AmNoeecKu+dfIXlIAdvnbsdmspF46ht8Ki4Q8+BQAHSe7fgFbsQa/iw55S6Up82lMWIInqOTuTLlNa75f4Q97QYeL3+HKtKViqMV5G/IZ+jeAFLzUrEKJjpVf9Jr3YtgF9g2fxs1Z8SNX9XJKvSN/+TXlTlp+K3uY/oSoqhNmQSAxG6jJTgdhdnAkGoN1+N7CZxwi+qkqVgtAlKrCVVPGxmHvuDJtU9THmci9cGdVCX14jxUDPhoDHcMrJ5OJ+T7RxFd5EJAVgDhY8MJHCQGs6ZeG0iV1Y2lDRr2Sf/C0cuR5eeWI1MreHztShYcnIyfuokh08/ilTKYkOEhrLy2koDqThZtXsS4o+P6EY3Xfr7Gm5I38UrwomBkDYJUICczB8U0BT3NPXzi/wnHXztO6D1iZtkrJ2JwmXOI2sxWrJExVCVNpSp1JjaDievnUqgsCkcrFxAA59YKtB21SK9cRtahYdSa7wAIz8ykWWhg4KqB/e3tKpTi6d9OnKGXGT/dh8M5E47ejhi7jLh834pjVyvdQgOH7d+wz+Fd1H5utJe201rUSt5PeRRuKuw3nP69WG02yt0fQueuozS6FKlcijUwhLKMBbQFisj/d/YMJP9MCvP/nI+xSs9hp9lcH/UEEg83+hxMPBhQxAhHgVlaUfQ4cWEir9teZ8Gph7DKrGwyWlntDulqM3UX6vod1ACRuVupL/fHK6ANc3QlMqWMlKUpDH91ONvuy+XIOBHhNOzcdKIv+YvvZ2w4C3cv5Lmm53hDeIPYGbH/aFdPRTMzdk9l2S/L6NspfqILqpzo9I6hLSSdi+k1bCjzoKpHDMTU59Tz/YDvabvZxuV3DtMr8+AbzwFknx+MZ7mGGztvsmfFHjaM2tD/jDiPHj71AjkWzn0g6qmFDA9h1uZRvP7ea/wQ8SLK+kZsNeJ82b18N7vGfk1qfjZtIXfoOW23HR7Ogc79gaHiHcWYDeZ/tCvatISQ6hDu/+V+VPVV/ccFu4CAwC/3/cKpEacwKZox2nqZ9OUknq57mgmfTcBQU0PYrvUM8GvjYuZlZM4qKo9X0qfr46Xul7g8uBilBJK99Ch9WxEkAn26PqrPVCMIYNo5gfe3DyOzvI/wDd40FFT3P7/ckNf//8n7JpN+yv+ueu9/bD9vSt7k4qcXiYiIuAvRY+qx4Fd6mtTDH5H77B//aHNvqgMH5/xFdU4i7r+NwG6zEzI8BP90fyy9FnKH17N1wVaS85OJzBeNObdwN6Z8NwWZo4KGPhnv6eDN9z7l3tyXqD1fy4HHDnDk+SMIAowN/wWAvsAGmn1Fh6VbmBsj14xk71QZByYfuNNX9rsdP6YuE1aTFYn0n2gkrZ8WmVLG75N+p+KY6HBqKWzh1FunaC1uJaiohfTcDLYtLsR30gB6CioJKdxP+LUd9Gy/hSCREDK6EqdXvsWe5oBbo0jZa1E6EnT9T/wLf0VquI5/yXGslbW4113HpbmETu9Q9k08yDl1NzK5jeuTD1KdpCNs2QhujHgYq0qsqwQZxfHFrF+xHsdYP2yd3Th21CGzQLNvMz+s/IGgKWIAqepkFUdfOEp3TRcFqe38ulR8ZwJ2+pIyyZvwIj2uIg1ExLYZPFQbTbNvM4JCQmOgjFZvAzfukeC9y58vDw7AJzeFVRuWobM2smmh6KT1qs0nsC6AOqduLje6YGn2xJY+gt3TRF0Mr3HJWDLCeSxrJHPMMODqAKyNBux2cDGkIQEeNM8iMs+DoyPKqZu/C4co17v65GL2RQqSxc1p641W6i7UIdgF7FY7Xqo2Iid1EPSsE76hgbR5tvVf57w5hU0vrcKvyZcJLS5I5QqMqju0Hh0+sSS41XJ/UA8yBxMSpCQWJrL63dXU/3mBC/sG88r5FXz/6sNIjkchH5gKwF/3/YXLV2P45Z37OW2Kxa0MvteJa0Ozy350yp/ZeczGc8fnY9arefbTZ4k7dQdRGjQ4iGGvDiPr8TuYw0lfTWL+9vn9f/vKjBQ//iNNiaJmrt1mx2a2gQxUGQW8ktLIgLI5RHZnce6XO3rBBau+59SOkTRUBDAz9hMEAXK+yqF4R3H/OfsXbeb9EYf45vmPcIrz4A3hDWRKcW6HFiYQfWAs93qUsH1aAYZAsPaaURj1dLuL8/RKzw4ilWY+8fxbbFy8b7dXBLVx42jJnvaPeaV0VbNp4SZKo0rR6jX917SEie/AXtpC3w8R/LltBL6HR2Fq7UVmNdMQPRK73kCeWxevZp7k81/HMOSTx+nOryU653eic37n6gpRj2KTawO3smbRMGIRuXky7DIFnV5RtHTnk3qpjSz9L3wf201YRD1ShQwnHyfMLmZabfBuB6xVHaKrJAO3ym5MehO7H9zNtxGfs2rtKuBORrTNYsPYacRmsuFFJBK7hPyUfMpTxGBG2kNp2K12HDwc0EenUZwWypmQJZT6vYvdVcHljMtczrzFuGMriR4VxcvCfHrcgsDNDaPGiEe7B3KbnOhLv/GUdiLbh29h3hNbCUsZTdzZ9QTcOkWPawC52aV0RFbwmKeWC0Ew3jqm/30Lt8EBZkGCvkmLaX8IhsrrNJUZcOhuQnpbi9nk6IHKS9zUqVxUDF89HO8Eb3507aIvoorm7Mvow0VHR3PEEIpGPELMFyPp9GxD7t/CpiV/0DUYciKncDX5thZOaR4bZ/nysV8HblKw2wWMiiY63DswOJkYftSHibtDkNlFEIdEIo4dZW8nsr5KTJ69yN3WMb8xkLd04hiThIXRpxUDgWt08L51FfvUFm4k3EDipup3Kje7/sX1qDdp9L7WH+zbvnA7x189ztGhbxJ29Y4umyBAY6CeW9G3GL1VdHKpTWrefGQfSd+8haCUo5CN7j9f4dzAK6u20jX9MHumn0QQBKR2KWZFM07tN+g8NIxv86LoaXTF1m6nTynqb9z0+BSzXUCQCvhIO3ln5ChGh/2K3Q4ZR09y3y/3ARDcshKj4SpPCLuIv7UDbYQPNfETUPV29tfBflQcYxp3DS7BLtz48wYOOhlWk5Ktz63A9Yjo9HcNcWXi5xNZfm45nSozD52K5FP35+nNGsm1PXXcXHucDx+pRPJyDD+s+IFUv1Zm9j1L381KepyldLp0cn7wFZwWTkHhAW2NHmz9fAG+5wSSj3+Bf+kpjE6e1GTNx1Gt4IDJkYU3fTij+RduBXVM2i/uIZZoIctrF9uK1vHHq3No3pODItAHnV8CHi02puydQtbFLMxN4p4kMDuQm3/dpOZsDbXpM9B52kg+/gW6i+fxSfJh5bWVDH9NBPe4drmyQtASfSsaS3sftwYtxeToTsiOz+n6+jPiNQuoDQfL5CRCvTKJuSSySlhbO1i4aToPd4zlVsINSsYep6+9j8PPHr5rzTpQ+CJz66N4qV2097QV10koEDW5nopbT1DzE8wad42ih3/BOcSFyQUGBt26yCfPfIJXqxf1u7cA4BHlwXPNz/HAxQcQtFKORvfxwewc1j20DpWXA14JYgDGrBdtR5vURlJBEiOOZ6BxlKFtryLx9LdEzU7C9PL3POhuY+r3D5J+wpdS2Wam7p3KM589Q8lbe1ji8TTLXE/wWjvUWzX43TpFxt43cVz7EZoGZw5O38UXpmRenrOBtfdvQxAg+MZVMZNRJmebzZdOjYrwNcW0T/TH19cPiUQiArI6XHn8y8cp//5OIC7zCTFr59zKBt595V3xoMSOVC6j2yuC5ojBBKwSgxTmyx60GhVYLXKqwmvIGSqh2bMMwU/BDfc2Gq3gJOE2SErOm6+/ydVpZfQFd7Hg6U2cDrMywQGMngrWrFnDBy99gNAnAm99UkUdmSP3n0WW3oBPcwAZeUvR2mR87QUjHQo5p/Lm6JijdEVqMI6ZwpWpb9Dr7Et0zu+E5O/GarKidlHjGuZKc34zVpONOiuc9qvlSto15Go5PaN6uJh5ke66brAIFJxPpuRqDOMuDqKvtu8uBz/Aqan7uDJuBYqCp7j++/X+486BznT6dHJ54GU0vRqqSqvI+SqHhisNjHxrJDZHKY/EtXBLkHDhaAaLf11Mn64PpytqBlwTszcu+rzDdV0KXf4xeAwXKaeXHlvK9HXT+e+Kxl2kwnTobqIufgLGyEQEAQKLj+BbeRFrUQkRxf44GhwZdXwUg/ZZ+tdRgK6iRtKvpqPp0zDl5Cb8W8KRSkHV20H6gXcpWPIBMaXZlA0YRO6YUQgCBBUdJPXQB8gsRsa6dFAcCvcoBxNQq8axU0lfXRtxuTUE1t4BtlbN7eHo7ExQKpHdLMKx/idyosU1RWaTYe+zINgFQn57l9QjH2PMHM5Xr/3JK9J2NnSLgVdjk+ggt0ukyJw0OLeU0VzvxOavZqM4LWBx8aRP641rszh2IpNLsabeoiKijE5NB2ZFFxeyL1Adk8SRCTdo82zj6Tb4rgssTo4UD1uBuqeNhpfOEFYZRpdLDxKZBIvSgiCFwBKBjMsZ7F28BZenf+IGXuy2STjl2IFdIcOpL56a4BoOLhhG0p+v/cf+UrtpyB2ZxdXhcbSEJgISsFq5MWwlWl01ss11eN2/HaXKTH2BN/ouHcVxxXfdY+a/nmP91uE0fb8Ae6wLb69+mzbPNvJGbsF6Kot3dTD2SASHzyaK32LvSFpDB6KzBHH/wQZ+7OnmsSYF5/QyLBYb5WlzuZV1L9MrQni7KZJgaQWaPWOJ+WMqUoUcq9IRrU7cPx/9y8CV70U9zeJtxVxbf40Djx9AEOxs6IYzw85QGSnu39SuakxdYgb8vT/F8dL7LyHh7j1aa1gm9XHjaF/6FB3LlDx11RehTcBis5BxNAPPeE+U/kpWrF/BwVNJ9H66HH6H3yf9zs6lO6m/XM8Hbh/wrvpdNs/YzNwtc/FPF/e93kl3AIo6Nx2zd8zGRedy1763paiFzTM3U7q/9D/2l0XliNHJ6x9BJqlMiv9IMeDn3iDuZYT/ElewdhnovR1MGn1iNFGFdQgCDHpmEBmPikH7kOEhXPjkArf23Lrr2uKfRfpqk6M7pZlLKHMX1+XRe59mzz3JtHuKfjirTIdf2VkkN+/obikc7rBslR+5o2sr18hJvicZj+liNmaLdwu7pu/CJ9uHPmdf9O7B1A1fhC1EBIAlF89B1SzvB6Vcm/Qy0c9OxbMun0FHCkEAu90m7t3kSs6Od+PC4Du02x8/9zH594kJEfccvIfHSx/HPmYavyz7RXxfElv/eyvNuoey+a8wd8tc4ufFoy+s6f9No4GahEnkj3kaEAjP/ZOctSL715/z/uSn21lwHtJwJMosvnr6JOEr02nJqST18If4VOUQWFaOwQ7/0sGRXrBjx5JzDZfmEkqyl6K6ZiL0+AhCdJ6szK5AqlUybc80Zu+cDYCbfBiHrh9ieTPkGO8Gt8t/aMYl9mEy45/HUWHhnuHFCCF9OOc7cd+vz5BRn8z9qc8TqjlO1ohJvPzLq1yOP4Xj1TNEX9xI8dAVNEUNu6v/nyh7gmfqn0HlrGLMsTEMvjAY70JxPNttAr5l5wi9thPXzBjSHqvEZpMx9+AjGDuNuDXexKmzDt2EhbgU+fDVM09yJjWYmskraT5zi/iz6/GquYrdbEXipSZ1SB7+OycTe9oXy9BR1MeIjHEeY9N4YtazxHheYUGhK2XlfgiCQFVyJ8fGHuOCERS3p/PVE2m4njmBucdMe1AqdbFj+9viGuXFfy2j14p2+JNfPknDa6Kf99wH57jy7RUKtxT+43yAp599lpffe4/hMhmjpGp+B44CvwGj5WqGSmTEZ7/H4iXPILOI647RyfMf8/J/xyKTyf7h8/ufWf6X0nh+++23AIwcOfKu4z///DPLli0D4LPPPkMqlTJnzhxMJhMTJkzgm2++6T9XJpOxd+9eHnnkEQYNGoSjoyP33Xcfb731Vv85YWFh7Nu3j6effpovvviCwMBA1q9fz4QJE/rPWbBgAa2trbz++us0NTWRmprKwYMH8fHx+f/eC/jfvOgqdFz++jJqdw2JukRKYkoIrgmmI8ZOb1Fl/3nVQTfJjX8PF3Md8WV+9KZ3E3zjIH1OnjSOWEhKrjuf78hgxoMf4CDtQ9DIaQtMwbfiArVGF95/4SNctAZ25g4iuswHw0SBDv8EOvwT6BbaaHG5Qp3UmdEdMkyqJnaqZdx34j5K95WyecZmOl078ZTKGX18NAPyBmB+30zGIxnYzDaKthbRdrONgY8M/Ef7mvRl/NFmQK+14RkjGvgxF37BqbOeLq8Igs5ATUIIXWm/cVkdRXfVw6Qd/Lr/erVBycXv5mBdtB0BAU2wFzXxE/oR6C0fvkOIj52N/7oP57hYxv4+lvVZ6/uRKWe3jOPR8R8TUFaEo/VOMPLvhbdVauGC9BW2OtxDRmoUKi34JN8Z778tPcKT054DoPKo2B8HnzzIyOB4JOk1XBh8gVxZLg/LHqansYcz74jc7ydHnKQw9BY/eLURTyixVn80p64TfOQTDM97Ycvdgcapl4P5IQhBZsoyFpF2SET8SQQJHQY1V09E071zKPo1TURNjsLRxxGJq5x//foRxzsCibRaqKvxQRYrNqb+Uj2OFRJCzRo+8NxHjt2bM86ZdJkzyRodhk+yD14TBnBpazUHXrvAiiPz7uorc5+FJ756mOtJ1zkw9QqCAILWmU5fZ4yOHjjLigi5lsLwWSc5tPRP5EEi6s3s6IbmxSeordWRWP4oQ4RkTF1aho3ppehEI56xnrjFefHOa+8wQgOzNixEXhZP+I1wwsaEYbfYqSvQYem+g3aPV1nvqlurr54OTQeBbcu4NXAOIzJEg6Mxt5H2W+39NB7/qVhDlfy++HfmbpuLcE3MVNS2V9Hr6ofUzZlR+5OBZIbO+Jwd5c50atzIy8in+eMWfOq9uTaoncLAdViejEeuHU+1SwoGlzI0KhtGhya+n/UZIV1Q8cTjOAyVo3JT4Z3kTWBWIDL3bBTqK8gVNrK/+JFd9jUA5P8qojNlNgsGbhBUE0RYZRh9SV2AN9oALfWXRQP16rqrDHt1GEpH5V3tGrnJF0nzLPZP2s+rA5fSUdmB1l/LrI2z6BagKqwK30Zflvy2hL65jUjmJfPbhN/Qleqwm22AhHd8bNT0iNmhv0/+A5vJxrR103hk7WRCI5N5bcnvnIy+wDH1O3zo8eHdfYLAyy0KYsyyuwJfiqMGntjzBJsWbSKqNArLf9Ef+Huf1VIgCgR7e3v3UweYDWYCbouJV2rEDLXL31xm/6r9jPtoHILSjosUohoCMBXEgQDuke5c+OQCtRdqEW4jqIadGYYMDc0FzRx84iDpK9MZ+Nt02u5v4VzLJTJL9CCU01IoBlPSHkxj2/71vLn/dR56ex3fTN6DXbiT0d5V00VK6WvYJSaanNcx86+Z2Lpt3JaHAaB0XylKJyUuQXfTofxdBEEMJAm3kXaXv73MlW+u4OTrREfSPEw9ZrJvhYgbizBfOr2jcG6roONKOT3uwTybdh2TBL4XBJzbxLXItbkEda+YVVkQf5C00his0UqCblzE6OSBzi+IJ1xhkiOAjR2ZF5kkmd/vxP4b1fzvmn12ux371Xzizh2n1tPh346L//4d5M54cypr7Jl4tXqw+K9d+MwLRvd334Zl0dt8Aq/OerDLyLRqsFmt5A6GwpCPiJfMxHfsTNQ2HQOtDbS3+GC1SymPEDeFjl0NjDk+mslDC/h2yVZqrPBYz1v8mX4Vm8zGgksu3LPwezz923iy3pOtu2dgC+5C6Col2JxMYuZV1vnuBBu4+0LbQB02m0DEla3o/EUH1tlhZwm3ig5DjygPZvwyg6oTVRQ+8zYvvLYRvd2VnTefZ5IA+6Ye59iYY9zTdIKg30S6kyN/jCe8PBD5QjXvv/w+a9asQentQHz2TmI7G/G/6IGvxJHx5xfgc+VOBgNATm0SttFbkPoFU1OUhG+snsCbx5AIElq8Wqiy+jHp90VElEdge8rOxE0XkdllgD8dBRocsozUDrmOwS+U7vpuircXM/mbyTj5OHHxi4sEDw3GP90ffYOezwI/I/X+VH60HWAN+Xi7y7mnziYCVHw/AWBJzhJuLd7OILma8J93YepJYdPsTdROqmXyATH70eAXhERq59bnShTyPTicvFuqvMWlE43aiEUOwu2A98JdCzn++gkaLzeQPFR0Sj7fKo689iO5pBw9TEn2fZS5+jHNT4n/VT9y1z8IT7VjvT2WrCpHmiMG4/IfppVa44SL9DMST27DQ+eN9VXQNFXi0NVEVfI0ImpPYuvQUxtexx/DD/Nx6BR8y8/j1FFLdwFMI4N3XzlER8gt/JR64oIfpC52DIE3j2HtEgExkoRSRoc4s6ctioi8S/iW3aDX2YfebjvpY3PwDmhhUrWM4m6V6Cj8MIQvf/gSGTBQKWO2/VP8PjzE27kFWPusXPtRpNmU2qXc/9P9ECquQT7JPgRkBWDpteBxxswbu99g/QPr6fAU53e9zY+AZ+YTOsqPS9cPExP4Ni8cGcSF3nLsg9PZN3UfLoY0nLSuxBw1kZGZQ/Mbf7G1614cOxzZNncbnm2eLN60mLObBkGwkWrlVJx9MumUV2Jw8aM8fR4pl28xvXUM/uM7+Hr7B5hMWZgt9cScP0xVbBgkQa1Vil9YI/5P/0Kd4mtcRgwkr+yO/Xcr+17mPyr+v3BTIfoGPSNeH8GJXiVpZhNM3Mck9wj0+RUkH9mJ0tRD+QUlaf/6ms9c4GgvrLMIeNe3oTJFsnnBZgaEdSB0yJjR7sKnV9Kp0fSyeL0z1SGLaU6w4VmbB8Cp0lTOHhhBx3ADXsc+R9XXRauvM/M++BBXqYo9F9KYdmAYttnNyBbNp6R0I6Fy2OsPBZI6ZsgnU9puw03pjUtVPgqJA33RNTj2OGK2X+y3HQc8OIBr66/hnBBIjvNpSt2+J+XWSiw9U2lx3cvrb74OgG10AL4Rp/jOZOJPg8AHVgF12y5qg50IqtEiq/fm02YF9yfXYWoz0qWVsTlCzFoP3Sk6LEuTSxjx3S8ITs6M8g8lrmgUH77wIfF7FEzWLUf1qIEvtn9Eyy0tg5ZZMDgr0LnrGO8A81wdQOHOhpOJOOQn4PC2Gy3h2XR6R5AbOhCj2sjyhLcB6KjsoGhrETKljLo0EzsH/8DIkyNRuomB27Vxa4mYEMGYf43BLhMDIkVBTyNI5jA25lHaghoIu74bIciHhoEN7G73wH/bCGrHrKM8OQ63titM3pZJVdoRPM+ZmRq0kOQBs4lPd6G2VU2Pu7hOTov+kuUDXmCzdTWq3yYwqSGYvsB2AisGcmDyAQbVQpJhPDLhbaK1EkotamKGraSiLpG0g28ht4pj0Z4l2pT+Gf7M3zGfrbO3EgzsnLmHRlcPYr1FR6lvqi++qb4Ifv7kv7WeSe4WfnxrH16ZSyEPXFtEJ6xrjQan3RPpe+kS+dLZxDv00e0RitKox8HFGWWME1Odink6uJLdgcUsP1nQPydqY8fQ4Z9ISLCW4tqNTLoUiWDvpmr4PRwPGcxkB5gTIOqBrmqWc6LPillvwamlBYUmir7oPhx6HdD0afrv6ejtyOHnD6P+uI7C5Uk8NKaATR4uSCUyJn01iV9H/0p3XTeXHm/hgMc3zN8yn7jiOCQImDUu2KUydi3YRG3UePKmXkA74yhlRBFyqZmIIhHBb27u5tKfA7hpd6blvAanJDva9jvgLgBMKgS5EoOTAZtCBN2ElbTj0pJMn86Di97tNHU28C/7n0QKdiKrB+NTVoktsAPBT8CisPC3CVK8o5icL0UnqvdNByyjLMzZNgdnYwvCoyCzGHHS1SAb4sa5CYc55lPPpRXHCLZNRi10Uhi4kfCm5xCuR/LdjnvoGnsFH5uVpgATq759DaOylpKnBE5mldDmUMKaygCGXwkjP9JCVHMUk/dPxi1YTY/cSphvB593zsZBsoMeu5zEHDVhueE0xqhY5drLXn0N25y9uTjsLFEhE1Dm3X4dDm4AeNVew2ocj1wlZ8raKUz6chKG9l7e+uJpho+9wqwkcb/Y1ttGZUMloepQ5CY5VzIvcl3RQfJfU+md1cuCnQtYml6EZ+01KqfvYGVaIUdPZFBSrmH4k4H97yvn6xwCSgNIfzSd5LXJhNWG4bHTg8RFiVQeqyTylBMJMQOIy85nQusQuuvdsZltTF37PA+v/5x5TrDVD+ASu6dpqYlc1d/Flccr+XWMCIB5svJJXENd7xoCIa8uofrd39Hom7HbAxAEsKjFdcNaLLKTVIRXkHY1DfdmcRENyd9Fn9YH96XRvK97FJPKxMCTJQg9WmQRUUiM3XfGui4PucUbj54RCAKYNS70OvshSKSUm8Rg6Kj4g7ww+SpxcWBuKSf8ZhuFUd7UBYn7vi9bl+ESMZCZkkmQe4AAwx3WnnNDz/H4G5/hHOiMRemIwmxA6ePKjN+GY7NkU9boR2ZMNx7fR3J1wktoddUMc1QRnfM7vhIlTVciUCQb6E7JxPHmVTwaCnGO8aXG04dD7qc5tWgT/wp7lEkfj0VtUtDuF0iw3cboExPYP+w0PS1u+AZY8ajNw6cqByMwSj4S1xHFREicSKpPRxWkJLzARoxhOJ8/9Tlv9wzkcfOvKMzxqPpcsBstTNuZgcExkvxB2+ksDqL+igMBGQH0tveye/lu6i7Vcd/pBzgwdAFWeTcjC24hCFG4XD9DcJ4IvpUYbOyMKyA2uYBvhu/lW8/RNIWlclJ2Ep+YbMJvDCe4ZTeqRl9iSqKx2W3Y5DZ6nHrQDPTGJzeFX1zbGHZmGAPaPbEOzCM2/yqVKTPwrNxNr7qDm1lpZF/MYtyRMXTGtGB2DELR10XKHzOwAZ2DlKwdsA9NtCdPTHqf4mGxSK1mJHYbAw5/SPlhIxkrMxj38Ti667oJGRmCtKGJw6VZ1I/P49nGUADaS9pZcXkF6wauA8RstCGH1ey8dyfJ94p74jSpjA6/eDROMjKeepOqZdtIeSaFt0vfpvn7ZvwbxcDdjbgbOLnp8ep0g04Y8uIQlE5KdGU6jJ13AH/Xf79O+op0AJLvSebCxxc4mnUUh14HMi9n8tDnD2H/2N6/7+1t66VkVwklu0p4vu15HDzu7LsA2kIysCnUeOceRN84Ea2fOK8OP3eY1qJWmkMHIrXbiL74KzbzEv5eTHvbemnfdJjI3DuAgAuDbxIkQML8BBLmJzBlrbgH6qzuvMvHcP7j89Qfv0W3ZzhtQSIQoOE2k68ggF/nLBQWDyyKdprc9lEwspSZC8R5uHnGZtyj3Rn26jAufn6RhHkJ/feVKWUYWg3IbzNwdbt0cy3tGi5RooEfc2EDdgcn3FcsIn9/PZ0+Mcx80RGH2zHT4IK9lB7KQ+8eTIuv5DbgeR69C3KArLv2sgAmtQm5Sn3XMUXxLYIdg+lw6yDl2E/YYpMRhJHi2JdA5MRINs/YTM3Fehj7iugLE8Cm1GBDg9Rqxr3xBvU5ClgF9uQUivUmLl0SzxtYLgLUAhyhVdtIj3sIyr5OWv28CFZAvUmKcuc4HIOlWK8V4lnXQF3CBDrvdUJ9uALfkCaSHAQ0HmOouiVqqF4e2sDAs+AToyDcNxzPNk/UIzxYs2YN2ReyGW6fw0oXuMcZDnS/QXN9K2Pjn8e1oZW2yiI8I7KYHTeRY23uXDU7c7wPBEcQJFIEqQzXllu4NxTR6R2F1Zhxl25k7YVaIipEBoQhZ4cAYDx1icCboubi9acaORezmPjaZhzy+zB3G6lMnUHiqW9R1d8B8UT7b6dH54xT6HQaIwZjUTszcak3QTxNduBunl/5LCYXCbI0aIwaTmPUcNIa1vLq2pl0zWtFutObGTdDED4Hi/TOtyHPBEM0UB4Sy83h2f3+X4N7EKdm30uX4xWm+97Nmqgr19Fb1kvlg5Uo9yuJ9xdZEJefX86+R/Yhlf/3eWBPP/MMCxctZtTI9Tzb8AcWSxdeni4o3RYz0vdB1Gpf7HZoDR5At2cYVpXT/wn2IfqWWlpa7vL5/c8s/0uDfcL/GyNArVazdu1a1q5d+9+eExISwv79+/8v7zNy5EiuXbv2f3nOY489xmOPPfZ/W6f/U/4/L4IgYOww4j3Ym9QVw/hk7Vn0Tr289+p7zHX6DBeHJM5UBSGzGAmq/4sXB7Th0RrB/q1TMQ/spTZuHHJzL+qGSnpPiEgW5xvhJF4Pw/yQnU6fGDzr8ult6CKw7V7s5hZ25ytRFkchTaglMieHpogh3KoroFn5GOEKeMrByuftKuxWG1KZFKlcitlRhYtlOVke8XwV8yfdzt28oniF9IdEYyZpcdJ/pk8Eag4fZtrmVn5Z9guXo75EEHR0eUeiNOppDU7H0XCDv76bRf2HBTS4bUHu9GT/tY0RQwgyXKJJoyb/5+mE+GsgBdqC0wi+cQibs5QrVisXDB4wPwGzlxi4rs+5o00wfMZp3GVGpl9Po8OqwthlJOerHByD/Pjo2U8xqXuJMK+m2N5FuLUOq9EXY5cR3UeD+dIwHm1vEhpP0Rk3bd00Bj07iB/SfyCwxIMlJUv49d016GQOOHg6MGvjLHbeK1JXRQ4q4MHgLnYcaSY+xRWZDBTGHqR2K4JVIH3+MYyl0ci/WoxqWS/S2ynkgrMz56bk4l7vgavNRpfChHA7oOqf4Y8kQknszjDK9fGMXnaYy1OXsGhOEAV/FHD+o/PUT7fRE1XJAJUcJ90ftJ+royjZyPhhs3g4/2GmTYNowy1u5v2HdUcqcH7QeaJvRfP0x/diSGxHIvEg8MYh3OsLCXp3Ojd3HeKPmAIOOrXzjpuCkPzduDcUYg+/j8B9+wnRt3C9KYuEC9l0TG3n4Py/AMh8dSRpKjmuUitX+iYg00rR+mtZenQp3fXdfBb4GddT8pkpl5Px3QQyzmbRfW83Z98/S09jD0SI9fXumgzqOKS3QToFvxdQvKOYC59eQCKRsOzUMjxjPe9qltXJTml0KZ88+wlPuYqbGff6QiR2G9Ya8UZBUTXkHsvA5raHduX91HvXkXpFzGZU2MpZVRfOC6Ou8mvbTxy65E6kvpluVQQdz5XT6dbN2mothXILgkzCjJ9n9D+7I3k4bYZYHq/L5eu+PxjjIGZkLT+3nM3zttPb0EV6rjiPRp8YTd8CPeCNRCIhYnwEcpWcuDlxOHjebfwD2Fw9KXfdS05WDjI/LbpSHcdfOY6DpwPOI30YUVhIjcM0QqpDsOt6aS9tx8nXCfcId27uvcmYuce4kRNP1N5sesI6yXw8kwsfX2DPij1I0dDe6YTe/o/HEv5EPJe63mBYSi2fdAocTL3M3Kg7GeJ2mYBZaUaQCHzx5Bd4mbL59y/X5K8mM/mryf2aff8uEqzUqikathKrUoPlNrVbQ664w6g4WoHGz8DQ/Jmw+CC5s3Yilb+Bb6ovQ14cgmuIKzF5nkQYR7Jj9g48bEk8pzdTdbKKqpNVxLw2BKPaSIKDlZe+e5dd+kMk35NM9LRoNO4a1uYepi6mhMc7layOgJ9uO/uL/ixi2/xtpADujYs4NaibqNJghKq751Dy0mTslv/wwoDincXsWraLmRtm9msE/k3X4pPsw+Xr39ImPczw84vpyJuOKjiUsszFRF3ciJu+hqsjh6Lu1HD0UipCuona+PE0xIzEtekmwUUHOTjhIFdTrlIa5sx0l3T62udilykQsFNohs16OGYZgUpyCptgp/1yOd4VrdS7iuu2FBk+TT5ElUbRF9mBJDyMmrhx9DlcwsHgQGpeKs1xtYSFBRI/L56Oig5+PhHKs+neZNXGcSPvGrZhCSg6rLjXNaLq7aQpdACaUadRlaqZ/M6L9H3eSmHIo/g0+dCoPIW0cD6aZnc2LfmOxrERDA6sRHtNy5b5W0iu3kjcpb1EHBzLyUf/5C0dKGp0JFQlMHXvVMzWLrZ8voA3fnuToKg6Bjz5GwXRX2Bb9QdpEV6UDoAjrQmkpq1g2o9BWOw2HGOccW0qpt6vkjHXq9GYg4m5jRFw8nUi7+c8uuu66UqHLdJ2zJXhaFsrsPQG4dKbim+9BvltDEL66CuEJpdzTvN4f4A5Z2AOM9OW0/B9L/ZMKytzkqiJ7Ub1b3oVQoiaK+EXuBF/ghanbqKFqWhzpbSEZjJwuh+nD37MuQEHSZPMxhxeQ7VKgt0OZZFlmFQmAuoDODfkEs85+LDo0YMsr5qMrkzHwScPMvTloRhaDVxbf41xH43DP92foq1FgKgpEvGRgj2FC1A4GBnuG07nfZ135p2Lkp+7YaCbhdHzj3FSO5Ay5zLys/LJvpiNe4c7JX5H+FneiYtnO84KA/f8NpuDTx2k7IDoyCtO78BqceSlKgdMrb18Ff0Vgk1g4Yml/LzbjTh3mFc2k5CPIvAfKKAe6UdzaCYmBxfcnVvIVn3ClzjQG1ZBjDYRR7OZyMvbaYgaQZ/Wm75eGfwXRLZEImGkz0U8BxVwuiEUmw3UnU14V1+meMhytB8vZN/pM5jj14AZBKWMxsihuDUW491Tjr1Tz7t757B98WaOG4t5zM+NXhfRaRP59BS6hRx2xShpy0ugptUDZdFZnEuv9j9/p1TLrd5w1Ptcuf/cUDru7eD1PDHI84AzfO9jA8shTjRlUUohDp4JvNjxIhXn6/lzym+E1ITQY7LcHhxiYEymkCH1iqMmTE6vQy/q2wH5jS8VIRHsBA0KZGjPDl51hPW5iQQpJf0UeIs3ZLHzow8ALy46JvPAiGts0F9j7NGxDMgbwHsvv8eloSac2yrYVj+Q0M7nGOkLJYOXASAz99IQV4dz+C1K5PfQtNuMwq0Ae7gbakM7MouYsWu2S3mgN4A8r3Ky7QoMhn+ueX9nWU78YiKtN1oxdhl57IslGNx1/Lj8J+yCDYlSjtwsBmSsXWYu/TGReUtP0r59OE5hAlGlUWTkZnB89HFMbVI2yW10FgRi3TMKEozcir5Fq1crCnUi6jATS+7bSNHNAQgaFRK5op/6zqBV88fqh6l/4Az3qnvo6XJD6LPSYrtFpc9XuAMJKkhgF4mNCez+aRZSfyN+1w5gcA1AN/EsS35fSGBdIIJIusD0ddOZvm46x06ZeOakmjFHx5B4qRCTbhSjj41GKki5nHGZCV6pTAs4yXfNbjQ1OGEfamXwGT8KEguombmWFslcfOtdeTuphfM9Zm7+G1VgZaw3zpkn+DHCgPvc7RzQx9Pe1U1lWCWCRCA01xVwxaH3HOebKomr9hbpwYf5czViD2sd4cGAzzglm833ow7jlJnPFEEcn0YnN6pDxaCNX7DIiLBr2S6qT1cTPzcez2o5O8ITGP7yduKvigC4jooOLn1xiUtfXMILN5LHdxJ1xY0a21Ukwxz7MzvpsfLa5nsYOfkCTz7yPSGei5DUyjGqjbT5GuhJUXLGcyedrqm0BkVgD1ZQF38HEHFSeQJNcxQtrnZuxt6k00MN6YM4phW1n3R2KJZdYm6QgO75S8g8vuu/1uToh7yrgdKBi0iLFcfr9oXb8Yz3RKqQiuCywCryU3MJd32UK99f4dwH5+is7Lx9Byd6upyY6hWFzCECbdsd0KVZY6Nb78DRumzkgoDEQc2tQfeRdOxzzOW1WOp6+XHjM5x+6HEAatV9HJh4AE2fhsxcFyqjjxPs2cny5l56ckZjT+jFqnKiW9uJBSjpjEUx4GtavvHmgbY6rLFmYs9vpC52LFaFlU+f/bS/Lld/vMqeB/f0/9340xy2aGU0Xl1AQqYb2+aLoEnfVF9sCjHDdN+UfZwadZ2JLc8Qfm1H/7WRpWGs2xLDtw+9QqjRG3dByrXUa3S6djLqpIiwf9fnGEv65iE391GatQS5yUDKsc8ACJSDl7STFyRaCk1K7HY4OTWYJsdfiL4VzfXI69zDFMLLw3FRKemwNhF08yhmWRhdQ7v47pHvWLB0CQDlh+9kgUSdcmWofChuHW4o7VZMHQYGHBJ12epuqMlZUcS0oC4+cYXXG7u5dZs+usL3Y2Rr1lBOJI4uBm6dTUG6tJMuNwldji0IeCK0OnLEEEBP5SDijiXido+WhKIE3DrdkEUr+eWNd9kvgQBTKg5rHfD2dYPkZKos/jTL38GnAlY5rmeB0x+8poil0SmM1r4GvCtq0OhbqEqaQq+LP0pHJe232tkwegPZT2cTMzceNzc9lw5ls+JYGj0P9XD8yHFUJhVyoxz5bBf2Jh/EsceRnAeaWJ+8nkuXQNteiUtrORk9s5ksHOXVEk88zjih+kbFzA0z+eu+v6g5U8O9R+6lzKeMV469gj3MTtQkkWnmj8kiU8bu4zNw89Dx5GNf0OjzAU6+TshqwaSRofu3bfwVzVFkXdHATHK+zrkrQ7V4RzGDnrkDhivdX0r1u3/QEpKBuqcNWUMNdnswlSkzaQnNJPa8mPUiEST8Of9PYtqeZ44AHnXX6fKOwqz3YtHmRZwdcpaKlMmEJfsi7YNerTd1saPxNZ0jphiafK/j03wBW2Y2zeGDaA4X63C9T0O+CeptPvQpa7DijVNCCIdnR1AYI2ZoSO1KHHtUeOgcsHvaqZjkz0mHD/j3Yr+NaMsfLwJ8h904RXClD6jM1EWW0+HthEQmQ9PTSETun1Scz8Lv3l0kBLbzyITfyFTdT/RNaA9KQWe3k7xaz87th2mzNuOnFEGVUWXhOOudsRfvx5zgjKwwhIFOenzPDKJuaTeh+SLY0W1lCh3f5/PA2YGEJPSxe900nJYGsntECaUu3zPi1AhGnB6B+lk7+/S+HP9hNqYnm/DSxaA2luFTeYXab/7gs9/6+LjkfdputlGyW8y8svRZscq6GXtkLEmFv2Jb9jpmDz963AIxaVzRjuvkaK+YsQIii0mfo5Sq0CoCerKRRqdy0CcC5aAYiubArN7BBNUEoXPXUTsyFr+PWoipDCNktImGDhmOrUaUvR0iXXi1mC1VW/oj78fION9gwssrEJm5F42+FYD66FIEyQwKo0pR2dv7gYZ2uZLAG4dpjBjCMz+IWUguQS4sP7ccgN/uO8PR3yax+Pl24mvFtfKnoT8RNCgIiUyCYBOoT1iJz60f2F9YQbZX9u2xYcOjvgCrozO+OW8xavkAgqYuZdNzm4hvjO8fHx3JHZwfXQWjP+FVn1fIHCNmulmNVp5ve57uum4OPnmQvJ/zCMgMwDfFl6vrr5L1VBYfmD9AkAg0+jUywDTgLt9v6IhQJn09ibIDZf9tkEGjb0FbcwNj58j+YJ9Pkg+tRa0oTAZsCjUOnQ3Y7XfuW3Gsgt4L1zE6evDD8jfQeejw6hr/H+/velvf/u9ydd1Vuio7kGiD0LZX4VV9BamLH5BE9Y4ruDY6kOb8B5eiJ6C2BGFy8kB2G5DXfL0ZiVTCnM1zGPH6iH6GDxBt9trztWiyAuC2qsUSz49xVDriW3YWVV8XXb6h4OhIc1A43U6F5NQqycwUA47e1bkIwOFJxdwI/xY3nZpbiVVkebsjLTUTe7WRbluMSAUL+DT5MFKzEICqU1XYzDYUVy8zwGkAeal52GVSBKkUQQBHXS2uTfW873II/4H+hKwYT04l/cE+hVGP1GrG6OjC5SmrefVHsV17mgZCBBw4ADVO26n0LGHIST9KNyUSMmcgpVnit63TIZfBavjcXcK75wfR0muBhcPZXfoVrtZ8DMPUfDbxG3KPp/HtS6vI+LKSz57+jID6ALLzZgEQXJLLR97JFJxJxfvFYDy6R3Fx0AkMVcs40glbe2CE8CydH3yL18gCEt+dyk/XM5BLTTywu5KoBFe2dBbR12BkYjb0pgyhySOTAYc+QGq34dxWgdmQjFwt503Jm+L4HBUKwNZ5W1ngKepRS7080PnG4djVAKWtZJS2smNuPtGu36Px1mJWW6mNG4vW05EDywpxUF/k+x4Nv1b/gtPiFeg9DURf+g39rbE8LzMi1aUhGXcvoZ5+OLS349zSQY9HCNJzbbgTytVkE6FOL1AX2ypmdp/pYIFuAbXLthCmgHzlcDZEf0xhwI9sUumIP/0dPa4BfLNczEg/WGdnwICF/eOwvaSdi59dxGesD3+9/BdjR4lZgEGDglh5dSXvat6l8oFKpnwz5T/OFx8fX6KiV0P0agDuu0/s/39TU0MCOHbW3Qa9/J+kKUEQaGpqusvn9z+z/C8N9v2f8r9vEWwCW2aKlCyVfiUkXWlGappGVfjnCAgoPJzpcXcm6djnqPr6uHRtJZPu24912Z/IQt+iyzucwOs/4pcnotOawgfxeW4+SRdScF9ipcctiFuZi3FpKeWeny4TMlvFK3O+psUGq8tm49pSimtLKUJ0CPdvuJ8hT25F3emNZsMU9L4tNLq64ejjSOvil5A1QohTOzfjbnIz7iZSpWiQFO8spnRfKWGjwwjICsA9wv2uNlr9laSPuUzE1WFc3+mNbaWdxuiR+JWdxbMujyuD6/DX6bH/MpdlzYEoH3SiYNTjKPs6MbgFUS8bS1zwi6R940C9VUP9ln0kHbvCxRFOZJ/qYfSXj5Dx9k+Uh5ZSLpvI+U/OEzQ4CI9YDy7cdKM7sAY4zSMv/s4TDcEYO17nxGsniF42iLTeE3h62VEaXmZyvj8VhkBKfe9h65ytKO6JRBIuwSi/SW9vHyqVBoWDAr80v/62TVm+h9A9C3FNFLVyku9JJvmeZL4Z+QPyzbNY8u56Wi5/TF9NChbPcNwbRcerIq+XS/ZAfH1b0M/ajyn4QQyNR8jJLCYocB5ZFyrRN87h/LvvsTkkn/X+D3P6WdHsjrkvnrb4m8wIa+GI7nMUvUk4SiVoA7qInRnLzbACCs0wpXEVr/Udx0mvxW6yoivTUX6kHG2bFw3RI/ppHe4ajzIphyccptmnmYG5U5Eo5ChLCnCrEHXm2t48wEdfvcKJzhiOlizCS+2PZ+1pJAgYv1wPgF9ULcbwEIrsGpbFBBA5KZKyA2XkvHuS6axmztObeC5qJlqFq/hMQeCzQNFhILNJqbDIsLm1EBNdIWqwlXfQVd2FECkasC0ue9ErT+DWNRRYTNTUKIp3FNPbKjoMa87V/CPY93fWmVllxnJ7ta9JnIRXdS7m24jT4bNOc8ClmeONY5nQbUPnrqMovoi5zzwKD0KMRsF33I812A2JBApHrEKQShl8TslnF5/COEHg5ui5zJ0YRltJGzvv3UnGIxn0yDx4IutBhvsf4fThcDTuorP65+E/I9jENiUVJPHr0l8pjSplXZCIpK08XknF0QrCRocRmB34j6w+gJaRGWz3FKntLDYbgdmBjHl/DPoGPY5mO6O2baMofijvvfoeH0Zd5OJnF6k8VolcI6dhah9+k87SeSYdpVRAEASGrx6Ovl5P4aZCihOq2DFnA7NuRRFWHYB1pJXV5tUgwDuqd0hMT2XuqHI298jp5G7QSt8wNd+FfPeP+v6/UyRyGX0ud4/N6eunM+mLSUgVUiqn/4X9Wiqj7j3FccQ+v7X3FoIg4B7pTvgNd1zbvDk56iS+fYsJGhzE7N9ns2PJDkrePod+UgHfzbzCyY4A9llPMt9hJHKNHARoTwKn1C286San+/ggYjt94UXu2qi4113H18XEx2+8zbOD37yrnoHZgdRdqKO3vfcfyEyNu4aArIB+uiMQdQ2S703GO9Ebw5YqLPZbJOZ2oIsqwy04FICygYsYMfAAYT+/z/rmYfSeS0fiasSm1BB2+S963ALZutyVG8G356hXG1aZDL2neL1AN4d64VAv3AgsIGn9wxSEmmh2LCT4Rh57xx7FTzefJMeJpNcmoji2H+t4AwRG0RwRiERej1bvzvgj42mILUcyP4h5W+ehq+ri4rQStJ88zC2VidrRg/AK8Ud5+QrheaI+gnNbMFVO61nlOYSQ9CsEaWcxcudIRp4aCYBhvAszw3ZyVetAVefzWOu6ePJLEexRlNXFrfgbFHvXklkWwuDqNG5UX2be6X/LRhYkZBwex+gkJ8qdOhjlFUjbdD9Ou86h0ghTehq44fckcSU38bt+CGGUjU9f2IHe4TpZtwYQl3MVVbUMEOeQvl5P0OAgzmZ38FlrJ+8WCcRc2khz3f1MOvICUdevYvY8T8vL5xgbfhq1gwlVhxNWvZmX/vUSapOahsuF2KQ21g24RLyTjnbnUCQeI+nAjbC8vzDHyPlheA7WwG7OG2FnRzte1/fg0nwL7bxHMbtd4Ev3Khokt1h57znea6pFKpf2a1B95Akva+F3yUM83gI1Zi2+qb4sO72MPSv20F7STvI9ySQuErMybmy9gTZAS/WZalwLPVEu3kVH0QScfePocuxizZo1ACzXLmf+X8s5HXuZ0sFnyDVEkOKbQuCPgTj0OmD6PoaF3R9SKjHzZkAZ78bMxDnIGXPPHXrhD1a8TPzIa3idHIA8QoKuVNS3MHb0ccsCew2glAfT7WxG7qjFIT6U2sRQYk99jJPeQOzkrwmN8+NV33O8Gb0Q5xsGbK0VROtqkVvEwLtgf/0umtyu8jZcvrWSPV3NxpHXsdmgPSwdVVMNATdPUKwTcIx9gH3ech6oVWC32LDL5HjW5eG4aAqtuetxbfThYhAMqRXXMbPamdbgNOI8XBA2llKtC+WPw2nIzQ4Y0uKRtLWi7ailKkLOrrGH8bYnIg2o51rqdRSOSrxPeJPWnEbp0DtBwe1NlVxrWguSBahd1fgM8Of7h77H4GggxUt0BPTp+pjy3RRChodQctCBk4pOMq5k4NPtDe+DZ801pHYL5UcdUf84mY3ybrY9/i3VraNYUjOovy//Lu+kNrHj27FIA0Koi7qAc7czAfUBlCRaqPDdQmTleJKOvEWzPQYk85Fb+kg88RVySwzHJDpiLQoqslxQO3gQFBVG/rjnMCoa8dctIFRhY4T6NNV9IoVSX20bro0t6D3CsCk1+JSfo36/EwxOoepkFcdeOoZPkg++ibXEOXfhvGMetbFKnBYF0+kbi/ttWmKvs9lkezYy69BgHEbbOTnyJF6tXjz83cP8tUT8xrwXXoDTI5U8GjSa7XNF+syV3dt4a+FAXqaOCY5/Ik/PJiEK6qNH4lV9BQeDCadGX0oqA/l10jEuhDhR7r+a1q82EuptJj8VLhthoBrMSg/MLm7INCqqs+djRESr34i/QYN/Qz9C19RtEqmjbjvXridfp9NDywR3J+xSaPRt5Pzg8wz8ciBrw96h2uUAy/PisC6ycmTsEcYdHYf51o/0ajLpdejh/vNZeLT4oHzUSkZhBjdjb9LhE8U9A74lRN3J11mXuKjr5XJvH6dGlIEAeYM76Qk6gtolgX2zvkU2w4kljq8jSMSg27864NTN11mZoKXLtYs+rYyW8+XEnzpMdVwaY46OocW7hcAkMSCQen8qXTVdpC5P5ca2G5TmRTF4ynkERFr2mvHL8dv3I9LbweXwbjf6bsQjlUgw99lojBwqOvdP/YyMSOyjCmjxL8dXbceztpDYkliafXXErbXjlRrBV/M2UG7ewDP1a4FH+8fu77KTbOzuY7fxGkcW5TDzphR1z8x+HeBIBbgJRla4VeMlh/dsAobcYoKvl1OVMh27VI7J0R25i7hWGLuMWAwWVptW86zrc2RfzGbf1H0Igh25Wv5vgT6oignh7Y4PcNvjx73xPQQWH8GscqJs3CO0Bj5JteE4n7R5k7/vLW4mpqPpzkBu7kWRkMINfw8EbTc/XhtOfn4EO2YYuZR9ifiieBx6unjUkMMk7xMog80UftjGRb/9lP7RiZNezhEVPJTzJ99nu5Ji2oOtpAW7NIwzIyopjr5Dxfx3+XdqR68RcVxpC6Fh3xD8y85hbDfQ2ybaR/P+nEdtYgNR2lVsuP8YwYa3kWkUtAan9evBA/QO6kUjARBoi1ZwNnUXAENUUzlr3kNGaAPzP/yOHzvH0Hda1q/LBpC4byIDZ26nZfMcHGq8sT8B7R61KPQKsi9l0+jXSJilkqUbl9KZ0ohkoZYbg5dj0TiTWvEbVlkXoc6iBuHod0bjleDFwSfEjIaxx8ayZs0akqp+YJFcht49GK2uBrvByNOfP43tyzUYLiXj2euBvUvL5POP8ee8P2GJF7VVB3lq3mm6w6soinyVqzI3arzLmdyeSdwXViCBffO8EYbNxFtZRmp+KgCaYA+WvLGaywMvk37fQjqbd+Lg4oU9KIjW1iBSqjcDUjTpccTptDT9ORS/2W60nj9P8A1xn2dy9KDX1R+pHKRyKYJdoL2kna7aLi6kXGOA0ZnefHHe2S7bGH5GpJEV+uwcMAZR4tnOvzQOOHs4893Uq4RWi5pD1i3lfLrlWda99jaWEUe4T3sflzeJ+lchw0M4/NxhbP42crNy6fUUx8CR50QJBGVkELa2BrSuPVyun4I6ejl+gLGxA20XHOsTtUsfdYWHT4/hoN4RHoFjrxxD465h/CfjMelNBA0OumssajxEu1Zh1OPWXEKXuwYIRiLYsaicKBq2EpnVRGS9kVvB72OXiLpOeRNfQkDCGI2O8mgZOm8nGqMv4Rw0HmkZSAQ7gTePY/NVcHLESYKqwwiqOIxgz4R/y+ARgNQaUBlPEKQZg2/XV0gUE+lzkmJWiTZKRNPLxF65SvrVdIQ1JvSedtoc79CzB9QF0LGnGlNqKtxe8/sOngRg9oO78cgsZNKtCZi7jWLgw27FXK/nhaCbZLr2sUYDF3oNOFUVIvR10x6YQssr65hS48IghwHUt7hSJ9fz6bOfsmbNGqSCBHt1AHn3nOGlkEaK/bxpNy1GgkBD1AgGTcri/PF6SqWzcAxyhqweZAF+BFQ9jXPD02ScFvcfkW4HaVF1wujzSHzGcXKKL23KtTy07iEAzqaK2cUe0R7M+m0W/un+CHIFa95cA0CLdweCAMbQWNxO78IuVeDrGsNL77/DiEXb0bt20qS3MHq/msCaZbff1g94LhjBej94RweG0k4e+OkB9k3eh+f+0+gD63l3wTlSPHvZefMZDrVkU+ouBtauTzXzkO0K8aEtRPoYqHFS4hLiQ8yFb5FZzRybEYVRE0GK0psMFbSabbRcb8K3rJwe9yB8Ky6gdwtCqhb7qLuuG0OrAe8Eb4xJSiKe/4mTO8cRJuQCY/F71I+bX91E7xFCXeJcDK7t/PjAdzj/v9j7r/Aorq1tG707B7VarZxzllAkCETOyZhgjDEYbJxxzjlge3kt54izcQCMbYIBk3NOQkgCISGUcw7darVaHes/KCwWr9e39z763v+69hpnUklVNatmzTnGeMZ4HpWe77K+43nj87wZ8g24XagsvZzdHMzsFWtg4veUzislpeA62PfmqDeBA9fm3PWYV66WI1fL0fpqGf7QcDbftpmu8i6CMoIo/7OcrvIunuZpAH5d/CuTnrhO4/2XjXh4BCMeHvG33wPo2ysACZULn8c/+fq6GzoylMu/XcbQdpXC6S9Qlz6H26+xZ7qdbrYsFv2j5vhxdPt2D963IMCmRZvoruymtbCVsJFh3PLrLchUskEgcfmh5RQc6OLY3Wuxa7zwbbqE4JsOpFHx9WEC5IFUJkReO6cDhbUXZ68c0PJ4zeMIgsCb0jdJmp/ErZtuHSxuNbeYsZvt2A/WwBgIr48n+Qs5FdoKXHIVfYZQvGouYnr1IsleEjYs/oo/96TwwIw9wPU8iF1SgkvWR6d/H8WLexnnF4+0rI/YK0106WIHwb67froLV2wJPJfLgWcOYOmwYFM7CWwPZEAzQNGkRUQHBIjU0yV78DCJ+VSn1UnUnGHwmXi9v6iDfVpK+eC535GgwOksQi6/Ho+0tkJ+xHpaPI8TUvsYF96oJ/KWf2c8EzjUDw4/F/I3P6LCcS/56lVcSvwJ+JBZyvP8WDMWmTsQS1AECr0GVWM0SWUxpBR6I6idZI+8SO7sMxxLL8Te9xRjjsai6w2hNsXGKRsU2mCCTkG/PhC5rxdOpzhTnW4V7ZYowl1gaFMz77fb6dM4EGQgc9qQul04lB7UZM5D5XkjbXTqbansNu3G5GWi63wXzARFSjzVLdf8xpYPkV/oY5HhCr96/YxU8QxKWw9umQLD0e1k3SThq9sbWf3RrcgvpMILIjVtd1AyXpfqmHJ+FJeHXsA/rROlPBJFUT4J50UZic4ABR0+l8jw6eSdh0fyRd63SKQSNJ0OvJvCUStglgcUSyQ84AWrrtG99qtaqA3cgU+XDxOOTqDceA7mXAf7wkeLe1hYbRhlD5UNxpXN+c04+h247C46yzr5P9n/7NNyu69LMvx1XN3XSWj5MWxaH/4L9v3v23/Bvv/a/45JYMxLYzj5z5MYK7poSpxIXewWUkqG4xGtQBLmxqO7CanzekJtqbObyvhKVhvklIe8xfCd4gJlGt/F/PEP8LmyiXfG7eE96XhiCrfQHTIED6PYGaNrbkI/1EWLA2TpiTSfG0lAbR7sauFC9gX2ynRsCG0nYEwBMu/p/HnPn7RdaiNYpkQSORKpJH3wPgQEti7byqX1Ij1A4ZpCpn047YZqPwB7oobxmbvI2zyJs53XtcnsGi8MbeV0LTQzL3UbFzfMoNdqEfXAPHywefgQUraHZsM3vLt4O/YsFc+e76LLuhm1A0YeE0V01a2BtDcEckvkP3mv7RLFv0zAZrJx96m7OfKQHZnHWi44p5JrOoCnTMqPIZ48UPgADV1avD7RM9VrO/d4HufjhidR+wp4x3gz/OHhnP/iPK/zOgDndKeZ8eJkjHVGWotaeaz6MdZthD01WdjKa4lVdGPrtXHu83OEjgjF2mzBWRvG69tHEq7qwSm14ZTLsOiDqBy+mPRDn6A6ci8Jq74nbV4er/SvJOzKn0TWJ0PeZfqDlITetofHAkErBadcQWvMSIKqz3L151IClbGkP7cJq/11FIF7aaj7iaiFUUSNj2LdJ0VgAqlEysP+Ltqf/JiXMmbQeLaR3Q/t5i+yy5JxDwA3gip/Ja0uZl4kyH0clb8eQeuByS8GbW8rmPv5cdvLDAvYwYr1AWhHqiic/jwhFccIqhY7Sy+MN/LlxC0cqbkDn1A/pn46h/AxxdScqqd2dznrD2RRM+smZPZ4in56nO0rtg9e36azkGuXUTiikOKMSjb4rmPpbjERuup5Mdlt9Mhnzsbx6NxX4A1uAF/v2HfHoFD8v1t/fgMvv/8yp0afQpItUrnKnHZ8my8jy57MjtvO8ImjHZPDTIjWhqO+n8yiTAqzChm9uQKXbzeVo8/R988s5CMv4WupxnD5BAOeAVh6WwEDQ+jgxZvGsknopHzHVZrPN+NY7sBc9wqXW9yMWyphwdrlmDLEZJUuUIe5WbyXY+OPYdFZCOl9EpVaJ34ffXYcFgflO8oZ+cRIvCL+zmG302smEXUR3LTzJjqdFaimjKG1sJWS30u47ZYxdAen0BosJgFcgpOUW1Oxu+W0nqrGmG/jpf5xbFxxnN2ZF9BEPYHaS82YF8cQMCSAHxofRycXWN4QT/7hETiecSBTyOip6WH40+M4sjWe1VsXcG7kw6wpSsE6+joFq6zcjofTA4vOQmhjKOPbP7rhviv2VHB+9XnGvzH+b7uv2yUgsw8QVnYQt1QOzEQikQzSMgTOm89OdQa3NXxBrykSwS1QvqMch9XB1HencuKmdjpkpxla8Suj9CIwlLYkjaOvH6W7spuEPdM4tuA031a6KCcfa4+V93zeI2ZKDMIU4RqFkZN3D44hyC3OpaS5SbwuvE74Q+Px0BVTmG7kjW5uqKAEsSL9+JvHiZ8V/zewL2p8FF4RXlTurUQXpMM3wReNrwaZSoZEKmH6Rh/0pnv47t7v+Hbyk3QWVTJs5y8AOH3jiGiI4EhaCcb7fmB02ivIq/owtJXjlinQ9DsJbA2k26cblU0FmusUuDJBTVrtNyBxkxf4JUarEolDIGLJGNZo9tFhOIUgFbjD8juSxH5KTaHkJvsgXCMACOm5lbDOeVwZ3cZNC67rw62d+BOR1wTJ08cVcWDoOSJVcdjjUmhx+hOb/zue3fWk7f+Up19ScL9qFj7BkWRcFLtlf7zrR+5cK+Fo4kSWrniPjss63LI0iocUk3Y5DY/uK/y24E9sKhvBX3+LZ3cHxfdeQOosYvTp0QgJGixNHcicUnwcf3DlIkSNhI5MqJGKnRgOeQ9yWwPDw3bQXK4EmRSpIEbCbokdtbkdiSAfnJMuu4uUhSkIDSLF7vIxZewJ7EGqvxf/+vOADGWnkYDiN9l39RPm3vI7wboKfJxDUduuB+Ayt4xFET08P+4IkyumU9u1mYYhq3EMjCL+APxQcCuZr3yL5YulRCeYsLicKG19NH60kQTVCFqMI/B9VcO6S29hN0+4IajodEG1Q0KHM5qur5bhGROI2ktN5NhIuq52ofBQMPHtiehDxXc1f918XA4XX6aItO+f6psJyIpicfwYBK5rErgGXKQURWDQuHkxuZghUhm5B0bSmn+NqvOBq5TzHLNX7GRUbj7nBTdd5V2Dmr0AJSklnJKNJzo+E7+UMB4oeoBvMr9h44wNRLSuon/mKX654zMqX8xmt/uTwf/TmcU18fby23DoLzJU1YhbEHB7eNITnAISCX6NF3EotaLO2r919znlTjyim9H79ELHbTid4JaJa4W+qwb9szB02AK2+RuJ3TMa28dGrJ7JVA5dREZ0EOviLjLOcJZjO0cx6nw85mW1xBQcx6b1xtrUTcdVA2tdd7BtzjvobFHkSj+kSS125ira/smjnz+K/ekDrJpzhbtbK9AG6Mg8kYmmW8OXQwtIKRnCJ5kTsO5tZWHxEFxrXHRc6SB/zUVu3v4oe+aXk6yaAUBbcRu7HtzFnO/ncNFeTHXQB4zdeycBPQYAWmNziSnaiqOlgwOzznNZ04IJG34qBwrX9YVU+0Y0rzjuRuZQMObUC1Rn9VA7bBEqWyN3rr2T7gAn6kc8GRV/nsaoYC549ON/JYbIy9eZQWqjasnp+4k3VuTzcs0E3O6bEBBo8P2RkK4ljFUamNF4ntZ/rKJzVitmfRlxFw5ROuY++pUagitP0bw7EP6RQcCQAMa+PBaX3YUhLJmcUZ+S99TjBHmqEQRRV+8vsA/Av8OftXdfYk76XPqMfXj3eKM360lp7OT9TNjS18+v5n7cyusV8BahnYhqT6TuFM7HPo5WPYRoyb9oSRhPr38MLvsBAppNeHXr+NVYwsGzHsxeaMDvdDeJyYlczLzIC53wr5ChHPB6jROjYhid7E+vnxrnteW0YKgIjPz1PX4Y/CGOfgf+s9IYohnC5bTLuDRdyDRKuqOWsjbnEW7eJXYcaGpqeebRSq6GeqPXBXBm1BkC2gPoCQgkw9LCshFX+OFiOPVqN77lem7adRNtgW0YOrv5/eW5NI1dwneTnyXApkP+17uWwLZp4nd0vyqH4iiBb3psCIKEmNJOJD3ZFAwtYKPvmzzhGs3lQCU/dwoILjcDimqiLzaQbBO7IyouboUFK8m8K5PMuzKp3CsWQTVWhvPIB7cQG6TG4YAWQmiZ/SqffAJT1oaSFNLMuZHHONibRPTmC6Qe38PVkXcSdMsoLl5wUhR8mK89VXzZLxBWYwL8+POmLcw8eR8ZiQ10tfrw+5srCBnSh7piA8agJDojskEiPuQ+mRdnBsDilqLrteLn8qPTv5P9odDrrOCDulwk+Ra8M+3YqxoIqL9Aa9wY9uZk45CZmG+9AEQx/aPp/L7gdy7+fJE+XR/6Xj1LflmCYryJzO8yMTeZabvURsnvJRi6LtAmWU/0mbF0F95CU+IkEvJ+IWXn+xTPUlEU14NGKkOefRmzbAihZYeRuRw4rlThbyrh2JJf6S8dgbQ9kIFrXbthjaJu2OVDI4iLb8SRNZGPz77CbXf6EXnyCxblz+bLh1czI2wvSWXP8uTdHnwa+xU+Q4M49G/a0OkX0zF5id2qqbemYv7EjEwhwzpkOPLtnzJgVHA68F4Whlz3Fzcv3ozCJsXf5k9c4x1Edk5BrlNjDEzCq62c2LGhmHrK+e6md4h7/WkafGIpm25l9MnRTD04FR42MDvxBAs91BgEA9OUT7C5t53Iy7sGr8GoMooYzYWkPFyBgSx3Q5v2Q2QqGR23/sk9MR14yYykrzjG7zJ/UMixeIchkUiIaFuIX0MhjkIHTAGtn5awkWHEz46nbF85F9NEZyQu3oVCp+bK6DuQ2Y0M3SfuaWP2TWXf0SwMvjLashpxycRk34T7VvLDF/Pw8gvlzumFPNOnoqb/Y/rV1cS2pgMF+Ie1syylgJPOJUgNoey4aQctwS08FpeOdIuUAV0fQ1XnUL9USK3hfda2vsvhYS8S1bWcmV3P4yU1caHnHuxNG1Hb/CEnkBpZLCBB19OAwmpCcOvxjvEmaV4S+V/lg0xG/NY5vLfyS9pzjvBZ4DeYc83skezBLXVz94H7ObfnHuav/IOk0GnkhOSwvVqUe+kOTsGnpRSXTMlnQW5O2wQuf3158Hud++NcPh37KZ1SMUnpElxUHajC0e9g9tez2ex0YO/8CaePgw1nApB2fMs/k59m79OfMrbNzPlUeL0bVI4Q6opiMVjFpO/yg8tR6pT/Mb4CCMsJA6kEbW8rV0bfgzbEgCCAf10+4VcOYNMYUFmNdPpMxL/Th0CzSKnmlon+mOlMHbHlLi6lyyiJfJQE9y/opAm4ZUoEQNLn4kryFYrTioltuotZghRDyxW0phZa4scxXmfi8RB4/6IvI3+5g7bx7bgm2NGarShtykHA72riVQY0aqYo5LhsvSjl14+llKbQcfoK/Q9NJbpwF75NxajnTcck0dATv4U0OWhlTrryq/FrvEivTyD8eonneR7XZ6u4sz2Sqn4BTUs1+ppCJIIbeb1YFKF1Kqg/MArJkH7wAqfMiSCRIEgjUFXfj9fwRYSoFFxojuLK6HuRuuzYui1cjLoTlZeOyFDI94epQUCFA4lLQnumB1M9TyHzKeWUuY4SLSyLBcnVckxeJqpiqjAajJxLEfd1D38P0peKOZyepn5KUkooTyinKtHGzUpxH2mLGUVw+TGceRp8/fvpuZhEeUEiuo/6kTragOv66x41v/P5+TFUV83EN0fJ3ul7sXiIvpxnYygJPv1suvIUBypXIv238LU+VMmxlP2Y3dPYfXofnZ2Q6YaekFQQBIIk2WhcvUTIjvHy7nkUVoXQqmwgrOwgNrXo1zYnTqCjogePTG+OrjpK4ZpCZn4+Ew/vIB5p9+P+llBUepH+/B7lPbzW/xo+/c34HP6MlqhYjqRdL1BVe6kpnvQ4gsuErXchPjGNdMh8KKzaT2RsJEfHH2XCsQkkzU+iq7yLZ+RPs//wfm4aD79++iueIZ5MfW/qIHtL8i3JPFH3xCAbT8qtKZx4+8Tg+KNqo5BKbuze6+/qp6WgBf9kf/Rhev6nebVXElibR9WQYcD1WGPk4yMp3FhJ++kqIot3UZs5d7AL8t+L4iyGUEaeGcmMfTOoi+pGuBdcdtdgkUvj2Ua+SPmC6EnRLNm5BBD1Pwe6RP++MzyDpsSJTJoh+tZZHyzhg61nOXtNV80tcZBx6GPazCnwgBh3SyTi9cu2lnH6/dOMeWHM38Y180I/mt5WZB4HkKvkdERl0RE1nJjKA/iUnUZvEhh/bDzJZclYXukHlJSPWEKONJ/b14PRK4ETc8awJOsWAJwKLSdnjeFC/NrBaxwfd5xlcWKxzIQ3J+C0Onn/8z20WQ6RqZmLyhkw2LnXHD8eb2kvQzUlJM1Pou/aI3S7xePGoCT6PBX0aVYRVRtF7cVaknKiibi0E6nbhT1kLoIgDH4HHsP86K3pIqy0gJ7gZOyyaIIqv8cRcT8+vr04zC40fWp0Vh19nn24X8mjvnEyS59fh/XpCzQFZBB80JfcM6JkTklOK29M2AEWCJXMZHqziawC0beIqjNyKlu83zjd/cx7Zze/SFezteALrnh/jEybxVjjB6ikPgyLqUa5/Dyu5CUoj9bi29hMVfYtWHUBDOgDuBZC8bog5kDdLjfdrTpmfn8Th7YeYXTnaAQBVH2dKGwWrPcE8Nyt76HR9eMwvonL8RSRF//Es6eBvszRBO2EL2xGDk88QkHaBXIKlhN25TReHZWYroWYSc1hrJy/jM01H9AQPx7V+eua8R+/+DsKVFR2jsFs90EQoGyhjoPKF5GbIVMFT3gfo3X3FFYcGoplpYUtt5fS4rObsIYw0ovTaUnuu2Huqb3U+Gb70lXQxb3J9/L4a4+TvjSdDbM3YGm3sHTvUnFf+z/Y/wT7/tPPXtd0W/sNIf+l8fx/gf0X7Puv/a+YVCYl654sWq600N/qIPTSEdp9U8k67YN2RgqdjadIPi12SZyZ4M2+CdcpLgVBYMzeBkCkvukZVUyk1sxU20z8aj8kOtWHtt48InoaaEyaQtPQYdR757JBcoSdRi3SYIGmpEkIUhnBVacIUAeQr29kh2MtFVF67krNQPmsBw2nGzi/+jwW5WVaB/SMOj2KhPIErI9bMdYZ8U/xp/1KJxJBQJEc9x9GKaCvArIOI0s/y0vSrwkuP4baIlYZzf4ik9yHK+m6bS//akzh8VYz0YUHUVpNeHbXEUIWl07WkDH2Ei6ZkoacbiSnK4mrEq+Ve/tRvAN6+PDhZzCmtvDcoaXUn6yn/XI7Ptt+pcyuZ/qnh1CYPVDbVMiUMoIyg+i4JqUhl9qxKOXkvfku/e4XCMoMYtbqWZz/QgRJijKKmBwntnEfePYApZtKiZ8dj2PEXM43zEbIlbFqK/S39nLkFZF+wyc3gDZTBackJmw3mYn2MjJOPYL6Tjn+NWdwJChQlDuwWtT83BaLtyGYnSPyiKwXq6S0rZ5scuYwoqaVkIIojOOt9HuFYPEKIWWyB9VXdtPggP6uUI6tGYl71i54+u5r88KNQQo56lr2DQh0ugBBIGpCFABOuQqLIRRddwP/E+yzW20s+WUJVbFVSJQirYErMoaKkTF49DSSfGoNLdukNE8KJTCkBZlKgVuhojFlGnErxvBl81SafM8xvX4Gde0TSLPBPU/pUSpH8/F3Qxn9WjxJXgP8bgxjU30z+tnXndq+SZ4MOzySFdNKiIiGhy7eWMn8VxWdRJBSklpCBDnis47zwSPAg6QFScROi/3PH5pOSU10DROOTUBy8TJ8NAWb1puy3BWMGOXHnJdHgSQH48ufs6tXhmCyk3Y5DZ9uH4zNzTRGmHmHLqbkXEIdbSaofjlWfeBgBVh5zh3ohaOs++VhnNntVJ6pRG1QM+KREUS/60e7MQjuOMTsu3fwpUwUxP4L6APo1/Yjd8iJb3gIuVv0shJvTmTYymHkf5XP+hnrefjKw3/rmh2SH0VAfQJylw6FW4PL4SLnsRyy7s7CKnWzZ04JjYbDJFxNwOHZxyVdFPsOVRETFIrucjupzZNgxXEMMnAJbr4f+T36MD2LNi9i2HI1Q2tTqMppozHYwhQvP75K/4r2YpGjQAd0aXL4SRaP44+pWGaI/OnWbit+75h4hmd4/9n3mbdtHnpHIXC9CKDoxyIqdlcgU8sY9fGowaAAoK+5l6z9n4jv/Fqrf2dZJ2Xby0iamwQyP3QeFu7KG0vxqXScq51MfGsiZz48Q9P5Jkb0r6OrC1KOfYVK+iWWN+6m/mQ9N317E7vePsa52kN8mxfC8rXL8Qzr53L8NYofuRTDVYE/D81k4tyTaG6KRB5yxw3Pu1dvws/Qg9GmoK8kAk+VGFU5rA7yv8rn5L9OEjI85G9aJn9Z28U2dj+0m/nr5+Ob4EvJ7yVsu3Mbt/x2C40RHcjsZprCmkSQ3XVdD21gfSXNwc28fUseAc5wzocvwlYiapbidjPntz5gJb8u/pXbf7udzmnthB37kAHPAFE7SxXIqPA/WONO4NdHPmOy+x1UIb5cTTiOIL3ugUo8tPQbtMh1IJw7T/KpQqqGLsKuNWDxDkPtD06bk7fVbwNg9I+l5fllDNNDfRuYXW0IPjEYg3wx+8Wg7xS5+rvLw8kQbsFD7sV3932HIBGwaq1Ux/agjT5PWX4yA3vb6c6+ytb5W9FYNeh6W3nh3RfY8thqWpIy6OrX0R/wA2emHeDglIMs0Wwg0/N5/qnUc+50Odm7N9GVMAe3Nh2kkKOGB9XB6Hq28ND85/hXwj1I4m7D51A3Em8D5+PncP5J8HAH8w/uRqaU4RHoQcWeCoKLBdbP03JY18iL0kYO6OQcnmVD33aaGOM4oo+eoqgvGUn3zfSWH0a/aBYlYx8g9cQ3SNVyrrz5Ggs9BZ7ugG6XDF33UUadiiK6VgzSXZ0+3F/qze2tXtT7S7Fl3Iyh7Sr9JbUoCKUFMGhf5/DufPTWNmwv2Ji+dzr+Hf5oquL4OraS+798jEMVH+DQDEVwi525d+Xdxeztszl48CAH7zyIVCZFF6Rj7xN7yb4/m7VVO3k/qp9Uv8f4pDmCmX1B3Pbbbfh0+2C/xc6EtxsZ4fc7pW/dgzlBj2qCCrvCjtIhrkn9vt243RIUe8bjHNfPphc23TC/XX5vItdG05yoQuUD/vFqYqbGUH1AnAdqqzh+T2U3WAWMZ0qJPV9M3hg5JvUe5OpEXrX2Yzo3jaKxFhwOJ1KXA5/WKxTMeBG3XIn0f+h6ywM1fLj0J84dmMyEilSEp0E60E9T0iSkbideHZU056fSN/oC+UPzuUk/kbj839H1NGC5ALfpx7P64QKMVQbSG4Nx2Z3IHVa0be3UfiUGajumfYAipBmz1QNJ8/VrD2gkxKdVMyyghn390OIUA7zb/7idnwt/hkq4orxMkzOH8ZML2BzZg6XdwrfZ3wIQDOQWvkqSTnwuIcNCGPvyWAyRBjQn+5l9eTbHJxuZhEi7aPX0p9cvmqDYMC5rqlFoTIzu9MdklpOWM59lLz+IXdHO+xkb2XgyjDHBZlY8/ykm90TCK5+hLeYYYU2V+LTL4bWnmbz6Azpe/JmDzU+Qtk30XTrCM1ELB1hZm0yYohi9BGQSF25jL6quA7TFvs/VMCPhBVa0yRb8hpbQ4hOPNi2R6qteg1pVZbkruPsBMbQ6tuoY/Z39KD2VNGyCV0x76ZybyILFXtiaugZpEpWBHsju+JWtsVf58vL3xIQu5puWlRyccpAhl4cg1WmZ7wE3K+Wk2pUMOJ1M3zudTr9ODoRNZuXvKwHwePIHWusUDATUEVRZTljZIboCdGQ/8xPTgvo5bwqBimpsTWqKHwtgn/J9fKSQqISjjOcD930Y02u4xXwGQRj5H9dyENdCgI7dxcRmxnI57Tp4LkHGzD0zSS7xx6FyU51UzLRYE1KdP1yrBN82bxszDj2M95ksnJPz0c49xJGeAEZJk9jn+TMd/h1E7xJpNit0HizctBCp3AtUt2CUfcKBqQfwtuhI7hrD5NgFWBq3oWjwxjHgIrbUhKE1iyvDClBKQCqRcebHOXjkpeF3Oo7PHvuQ7AvZjMgbQZdvF+Oyb+wo+KuiuTwZEkqScQ1YcToE/GvOY9P50meOodUpZXErgAODwo7Lx58+QxjRhX/gMXEGDbYUpthe59ITLxI5cYBNc7ezaOMiJh6ZiMkTGnaPY7F/IxH6HmTyAPSdNVg9RWGemzyc3KqHZnk432wYT2xrKv7WOtKbHuDtV97mUyMoHIFUd/szbfdo+uQ2DPMncMw1huqgDURVali4eRnmJ6tgQRQqLxVBmUFsX7Edb403F4bmM3PPTPoyxET/2JdE0LPW3o3hSjWPJVWzcq4RfVAOvQEzBp9L2m5/7pk5nPxZ+3nFUMFz2mUY2i/TGZpOxPBwPE5fYG+YwMfCHEo87iO87jyP//YaYheSgFPu5Hmlk0kd72HvMuHotdIZlUal3zGmG+/nxeHPAuDycLAjZC9TXKNveC/zt86nLKls8OeRj4+kpaCFU7vLmBXzOWHeVVxJfJNW5zmGPzKc86vP03Khhbz72zmr+o15O58gouZTeOc11JZOlLY+Gg5epeTBPJ5UOtEOgFd/CH5Xr5B+cKp4kS+asPI61Y9sYf2Jn0nM9SOiZA+e3dcLPUrlHzHZr5M/Mr5G5QjkQzc8/MXDdPl24d8WRNz0s5zPyWJJ5A9E2+byjNoDicsBUjly5wARpfso3zGS+CmR9Hf24xHgwbhXx1GfOJU850riKuLwVlthMhxNTcaqqmU0vzOw7zKFdoH2oQsJiQ2hJuJ+DmStE9fL7j4mqz/F0qvFQy8Wow09n4Rd4Y9rsjdlT5xlQsxRxng58HetIN/5A72+UUTVqjCWlaEWpPwyroYOaT6Zkb/RZHqV6AMOHit7jK0Pf8WXI9dytG8h1cHfkT/zVkbM1MM5L7raIggr2Yd/fQH+9QVYOp5GF6gjaX4ShmgDal8dhevyeRkvnhd6EAQBp7+TmhhxHRSaRIA4r3gSTV/piMmuoiZzPhLBTb8+iLwph/EKPstTDi/czZ7IDNc3xb1P7qX7VDff3/89446NI7YnlrY72jDWGgnKCKLt7GHeijpBk0NLTWkTWaci6VvZR1NuJxUthWSrYLU/jNI0s/KJ76mU3wVA6AiRDvfQy4fwivBi2APD/rYe6oYlQd4VFLY+TC4xSerXWASATWvAGJiA1jWe1PweDG2lIm1eTxNOhRp1hD8dEdk4FcU8/snj2FLbscaB1tSMBJD0uVm+djnvP/c+ni4HSCQY2srxayyiNW40BrmTSVq4GmHk+zl/kuLzOn3FNUzd2kjfvGQuZl7ErCmmNaSclnAlU+QKhv7QRqRhGWvuXQNAQXYB8+Y+hC5Qh2/TtQSBXk9PUxmf/3Yva/oGSAmKpO3WNi6lXWLI5SGACNpYdkzkh4Nj8Ujuxzh8Kk6LHUN7OQA+KVX8OfQc34Rc4qnULUScimDNPWtweqQxufAB1E4TLxw6gcstJzwcXHIlyae+p+AspEnlVCx6ma+aV5CfmkeY40MiD9aC3cHG5e18GbSN8T1PM+rfQAm79T2CWoNYt3zd395RR2kHrUWthE2IY9Mi0X/TWZNEnfDqEuQ9TUgFN71nShm+qhud6zQfBVdwm89w9sw+zz3fX5eo8MkLQuu0EFngIOimyXww8haCWsXuFYfKg0W/GIm4tBYf42b6Zq3Ap6kdU0A8dMv5olZPalAIut63CKxw0nh1ObXJSRxKC+bVf71EeFQLqkXDOC7Y6VPYiZyZwicHBpDarhLYOZT4vA0cfiqKuw/fQdTEKArXFNKU18SIWQuYYFhD/nMnOHAtR+SWujk96vQgYBJcW8WQ4iHINDI6Sjs4/Mph/OrjGdCquBpby8u6Ni4+swjDnP2kTE3h26xvGbZyGDHdMex+aDfSl6Xc8sctOHud9Hb1IpVJqT5Uzfa7xMLlK1uu3FCgqwsSC3jrk+qxOW2MOjuKqueqkJy9Hve2XGhh/fT1KD2VPFH7xA0MMACtcWMwBcShMXZgMysHO6++y/me9rwmzD4RuOQKDC1XcDsSABnFG8Q5rEyNI7ZgM2m94j5m8hpAEGDxtsU4+h10VXThGezJuc/O4R3jPXhNU4OJttPXKZXtWgNScSh4pYTRdfI6d7sgtdMWnUNapphTKttehtZPS1BmEK1FrfglX2db8gz2ZMnuJVTUq8nfqaE9sBvr0nTiF8XDrxBVtA2pSkbFzXZ0eV0UZRZRldbMQz4vXr/eNdkgD4sHmj4H1k07kQ1PBmksaiYzuqqKDs+DnEucypncM0wI6wIYpDRuO3SYk6qTxDunEFB9FpU7AIjBFJSIzAfu+nk4B184SOHHX8KQBwExH9YdmoZV6YXULeWun+8i33yOpC3RaMztSN1OapuhQyNg87axatUqfp2UR3+JkaDq0wx4+GDxHop/5wqm7vVngGOMjMvB77cmcpoe4utH94mCgYDgluBW19Bta2fR73MB0U8ZciyMBk0ONv+HCOteRteEy2y8dSOLNi1CIkj5IRAW6mCzuQaLsouSvsNE5yewfN1yyu4oYs3N0exr/xCrqo55SdU86boNTU0JAVfyKZ74KC656hpqdaNcQt3xOsasEnPN5xdcK6YtqyLt6HoALKODGWpo5/mdk5HsHYt5Qg89wSl49jQgKMW52nlgNFtHH0DqYcJm7EfXXYdDqSVocjJBY5Yxw+Dkw5cexqqtwH37EgqnP4/c3o9ZspaDRdH0xAywa4sv0TU7cX4wEUEi+t9O4CZFINDGPkUvXYHtSOVSWnw2A9AU2sRbr7zFopjHbxhTxZ4Kugq6KE0uJeVKClc2XyF9aTopi1I4v/o8jWcab2gm+J/2nzr7/udxiyEMhb0fh9Ljv2AfYgGAj4/PDTm//5v2X7Dvv/a/ZjajjaqtVUTelEjFnD3Mya7imPVPhmZEocGExSsYD1MLAZ1WlnrCnNJ0yn6fzsCrvZi8rguUan8Zyi9NY7n93XXkRt9Nl89hqrMXknhuHRGl+1h16yp0Uhi5ZRHugmTkLxoJqKsjuEqsnlhz7xpyFKl0W6KpNYYhCJB2exoaHw3nV5/nSsyP+BgH0PfqCWwLxO1ws+L4CgBWpF9A5rRxqcWf/xkCyIv6uPXorRyafAijdz+CAIa2cmwaL6qzF5J8ag2HN06m+oXLgIDdbMO3SewWdEtlSN0url5I4uvaj8DHhEtwsGP+Vp794Fm8JmWzznE/BvcULGPOUertjS5Qx6aF15OQqjAb3/eP5YFfkmBAh/tjN7ZeGzaLlMqgj/lxQMLGzuUc7PuaqZrrq7Hhufv4oGskjeENvDtGVBxLvS2V0k2lVOyqgF0fMFTj4tiLP/LArpl8O/MrRjw6grzP8+g+3U7b4jBC4tXsl/2OIFhJYx4u8zsEV6difM2XMaGPomyMQPHBMqKXRFIZVzl4bVfWUBTtE7ivvZKpB8cgpPTR65eEMSCB219vZuN964nKj+eO4ZVsWriJ7LjbaDjdwOXfLqPXeZKp9eMfPlsJ64nBqzgDZ/AAnqGerDi5gqde1RFduJXgyhPAv9MLgMvmIromGoPRgM1wgIH2SUgkXug7qvAwNtH+gBb+qMWW3IzP2FA8knzRXusalelCCK33xa92KGt6hhJ/pZuCjCoy9v9BZ3gmkoGxRPZtQa7ax54T5UirI4jZE8PrwusIboHnR31I6Lh8AsI6KDg7HMOBMVie7qe9uBWX3YXemobLLSB1qzg+/gg5MpFaY/OizfR39RMyNIT6k/VEjIngf5o83ocNSzeQfSGXeXwCQOyFjYAEYbJYxYYgJfTze5jq6cTzbl9+WPEDd/8oAqibF25mqVbKY/f/wVsdM7Eo0vE7uwurzg+vl6KYK1vB2YZwencnINjPct+/pmHrtQGwZ0EJKwJP0eSUMyf6AgmKvycP3VI3I/JGMOLAt5hn3wNTwvh1zq+U7xSDxfBR4cgUsr/9X2phFFKni3MzpjBz7FCqD1awYdYGbl5zM5JQJxUhbxBVE8WSX5cgURj56QTEd1RjqW5HCsRlVNBSE8zl41NJTVLQWmGm6VwTb0jeIJ3pSNQ2WpZOoLVnGlq5YxDoG/bMOLaeMeDyNvB+WB1ey9byZewCACztYgAgQYKuT8fxccfJbXvghvue+OZEFBoF6cvTiYi48X3JNAo6wrPo9Y8le5ko6n36g9MUrimk5mANrePSsQ7swZJVjNG/F4lcwoBpgJaCFgZ6BpBazCisbkwB8Rh0Ltovt7NxwUYAsj6dTdfjdXjFtzFpwVH+lHkTPDSYsS+PJevuLI7c/wmFh3IoSXbQRzJB3mIg33yhme+GfcedvpP5eXkNcy6+y5zf2ohekggPiHoN/65notAq/vauWgpbKN9VzszPZxIzWaSrOv+lWFCg0qs4M66SHk0eCrsCl82JflgC+Te9TtKpNeh6GskffZpMlUCNxQutVI8pMAG7Ro/2WlmcU+ak26ebcyPOERocj8TPB5tWDDajDReZHL2W301ioO4WBNxONz7GMZjVJYR2L8WsaWSP8CzmUAmWgW8QHA7kDituiZXTCfOQOd30WXYjuK/Te3RGZNLohCNHMrjz1/n039eO5FpDY3nOHXRpdxDq9yizbf6knPiJ/sZ76fe43gFq0dShP5/Bbys2ce7WbjJ9HsLd52b9svWklKSwaNMivmxPJG/Mj+S1jR+MPwSpgP6zehzuR4h/91v8wx/kdNIQZJ4aZJ98wqzEufjetp27/FtoV/gxsdZAmWYNoWVLuHPtnRyadIgT48RK2wGJWHQSMzmGgecGOPbmMXyLpYyYr6Rc1k+XW+zgvJTwHiRAhzmUmAtKdAFmJBIBEzakChk93uJe7DM2msTnXsOeexFrbQ65EUGYbCVE14rAdd9CLa79LUglQey9NYLU0CiEQhkVOcvI6jzAwNU6GpOmkCoFr44qDG1XGRgYw6iz18FymUtGkHwXK1ZvZHvrVKoPVrN++nqS/5VMV2cXt759KyeaTzD+lfFY2i0Urikk+/5sAoxSut6/h/bhV7A7DmKftYjksmQArF1WtrV+ymVHDTF9XtQMOJn55ExilsdwJEoEgrpCmlFrbez5eTbEDZCxPIMjrx4ZvK9F2T8wKeMr/lE5je6+r2kpsDHhjQmMfXsqP3zwHTfN2clh2y/89lEY8eOqkfd1Y2i/SvcoGUVZp5jvGUzZxXl07vUiMDWY7q5mfFqvACC3W7DL/05l7MaN2Q2ezUHou+txu8G7tpCQkoNcGX0vlpX9FFz4luqYakL73iYsJolCH3Ft1vU0oO/VsPDKWK7kRNCRMIqRI1LZkxNM8qk1mPzCuTr8MxaFtfNBJKxs7EW95ziRBdfHXJfqplYXQcHP35DQ2EbvLb0EpfjzTMozHPznu+yN6gRhDa5h8J1fP2qDmklvT8LS4+DcBydIOreOvmofeP9RtH5aTrx9gu6KbtTdAwzPH0514mWEa5+cQ+1JddYCsofqmFxvZkskfPrZEhplwHRQ2FVkFozAVt1B0doVKKef5c/le7mjvht1WR36DjcXZr7M0D0iWJ9boeKydx8TZBoqRiwhvGQv/g1FuAI8CJdLKCOefS9PwlerxTmhhfQzl6kKjMTobcQmyDjv46J22SYsvXeiCPGnO/R618eApz+qa6w1GXdmYLfY6boqJlxcNRasiV4ii0NTJ77NIkhmb7MwxJSDVlfEithHOWS+makHpjK0IIt3nn+HoN77kFXCvsphONfPwPsFHcPPD6cqtgq7UkyGjJh2DnVtLMd3jsc9sxP/sjwAer01VDX7Yg6y0dD0IzFF3bSlj6Q30opFZiFaDl8GQJ/wDXMUenacmoDd10za7vcw+0RANow7No6Q5hDcIh7DI2WPYKwzUtgk492SdxlzYgzjTozHPLyDgKYB0ovT6fG20vVEKRvidvBB1UyOFb/J87PNvPbWa1xKu8TV9D5a/AvYKvRQFACxdl+UAQ/wu/sfuKU2TF4m3LThGr+KnK/vpU0uw9LdSkpDLgenHOSxD0VqMO+MfFZtzSEnLwfbYisnpvtSFv4xn/rDfV5wRq7nq8AawrIF/oKuCoYWDHYrBkeJv9339D7OfnSWYQ8No2WqjJGJP5G/pJnzbcuw29xEluwBYFsWTBm5gH1T1uHdpcDDrUOREI1bJkdpM1P+Zx7Rses4a7FTkVWAMiyeXq9eTF4mPCwemNQCnYYeTnr0MHDPD8zkM1o8nxucP6kqgTv08DU2whrDiKj3wThqFPW6WiI67uNTviOuPxedwcb393zPnORnkCiVOJVKHIoe+uRmriZeZZSXqMPSUdJBzmM51B2rQ2vVIPM18t4rb7Ncv5HGs43Un6wHCTgbrVjKYuhsKWXlrHxqAm6kswLI25NL1RRxv3d7KLg8/iGGHPuSgYsmHDYF6/cMoyNWTHoq7L5obP70BCVRFtXFpRl3M91Pg7bqR4Yca6U1bjxncs9SEryJpSJ7Gj0+KxhyYjMm2XYaC+7jwa8e5HTuaS5lXOK3xb/h9BR9wcZzjVQfqB5cgy/obuMfX7zDBHU3dqeNkKHXu3AcGhfd3t1cSSrCqZ6O3diPR0/j9ef99Qg+PmUg/clKyqo/wLNE/CZL0k2kXhL9iOfaI5l78U9cqmzao4aDRIL+GlCvNTYjQYGHBCQSF3Y7nB15lgH1ALXxFRQLc0mViOu3gICuz8Wwvf+iIi2BP2Y9QWlAKguXLQdg/zP7ufjzRZBKMPnGkDzuXYYe2IZOI8Nxu40524aRXLYM08xO3nntDRJVAve3LsZqiLih67v4mfX0V2s53j6OwnMjiFkSgb0oAafMhjBLQaG3nJt6Bnjt2Ej6t2TQt+w8oc16JhxKw54Gwpxm8oRISm238uK6IOLiC3DJBWwqG2aJwJtd4HbruCx7gr6cn/mz6XnChXeQuJyYAhOQul1InTbkajmWdgvN55uJHBfJnsf3IVgEtPvH87jhLXrut92Q/Jq29QE+/FRBma2PiLgCPEM86TcEEX55D7EXNtH+3Cm+Syzmme+m4Xk0l5GnQggfHU7DqQbqT9RTk1KDTWUjpDmE0LpQRjw8gqy7s/gk4hOi+uysYRWPfvgZX864zDfRbah9nqFtjJHTzadZoYJR13CGQzY3YQoxqe4ccLL/2f2cXy36rIJLYPhD1+PH5gvNuMz99PrF4FRqcVodOJ0KmhInEZf/O/quWvIzt3NyxHxm2DcSFD4PQYDkU9/THZRMyTwZTtdVtL2dWDW+CAoBqRScKh2dYZki7Xa/B569nkjcLgS3WKzcGjcat0zJudp76Q9/gyiDlYKhBUT1aVEE+dASN4Zun98AaPUWNSoFiRtBgLpUM426q4NjsOjVDJsyA4VWQcnYBxCkMoZHaghau4kAqRuJIMVmgVJdESfHnCSkOYTIzBHYzpQw1cvB21MKCAwZgqdEg9xhRUCCM1jCScHGxn4HeDkQ5FIWbVyEzqKjIcJEjPwXnLUW+nLtSAs0dM5fSkDttXtSypDZnSzzHImsvZfi38ZSNbGRuhgTnepDqLuamJM3h4EZEoZbVqG7rKX/ymJm7Z5Fj3cPVXFV3PXjXWKX0evQcKaBH3J/AGD5yfsAiK6OJrwxAfcoB9beE4S3t2FTSdH663m5Jpqi6K9heB2LPOV0+RvZtHATt26+lbr0PiQzfsLSMIcOx2wi5Dpi2p7CLmvnwqQpDD18kICag1RE7cCn24/QmjL8C4+yd76GmzY5gac4/0gTie596K9OpbuljQGlCb9OPyROOY2V4fgps/kj0YLPcBUqHw/2jJ6FS9bHzNMnUFozmDhfLI4esngIEaMjUHoqWffCBcq+z2HBc2v53ms3sIzla5fTGnS9ePKXJb8we9dsrJ5WjLcbKdtaRhRiAYVfxQy8H1uL0OuFrU9KxM8RPLld1KO/+tZVzj96hp9ij9P+ZCy2m9eSNUHUkess62T4I8PpuNxB7dFa6k/WE5gRiMpTRWhOKEv3LGVK3hQcjQ5CmkNQFClu+Ob9U/xRG9QMGAcYMA78DexzqD3xbSgiLO8w3RX3D4IQrc0iwmDV+SN1OYi7sBGH9QVAxp/3/gmAvaQSLdDr2U9jWC3VGbPIvJbqUmgVBGWIe9Wkf9xILfrd8O9QBXpTPOFhbB6+KPt72L9VycqVHrgdLvg3Zhu3xEFD6gxyxZoZ/ljyB5HjI1m6dykOi+MGENHcYmbDrA14JgRBwgOcTZzIyWYzy3or8OyU4dd4EZtXAE0j7RzN/gYAPX3IteL+EVR9FmNnNV88dpAOn5NMObse59kLOIO9wR2NwtbH/qw4LLqOwWs22kqAqbT1tfHGsTeI/V1BpGwphY90EVG6D4szE0EQ43K51czWZQepP1mPIMiRCMINQI2AgCAR2D1zN58tnifOjdF3X/+Da6wE939zP62FxSS8uIxLkx7DqRCD45sTP+WerKd4845VKDMl9KQFUhC0C5dkAK83bmak8WbUvQPItk7HOa+fS8PAr7mD6BrRv55xcCrKtEIchTVY75hJaWop/4r7hvGXL+K0v8opK/xuymXdbwqCo72wp8g5cNMONLpw/rjyDEaPofzUm83j3dXMjlDgSh1Fk2EIIVePDPrivU1Pog/Vc+T1I7id7kGmos8f+ZygIBE0dzVd/6Y8Pmhhvs+j/DDmJMHDzrNMr6Y7LB3/+gtIEKgfq0VqrsYULeG9LoGg3BRK9wcRWHOWxDHJHNEaaQN2jD2IpySVoRJwKdS4FGpyd5qoPnwnG57cT6zTjEdfOG6XQFrDGDT1m5CE+rOvp4zw3NdoG5PHb0NK+dHnB3LO5tDt001FQgVuCSR75dwwv3c/LHY893j38Pkjn/PuHFG/dfLbk0ldlMpP437iwrcXeLr5af6T/f8C9nWFZ2IMTsYt+y/MBCCVSv+W8/u/af99C/+1/xVzDjjZ99Q+vJO9CZ8WQ/xjMwmxnUOS6IU8PBivkGCqz/mRdvQLYi8PwNOriFq2h86QLiQ6T85MhPCGHryN3gQ0eNKv6WfrtptwnPIjY40Jp1KLRR9Er38MwysTGIhqYLvXRHSZGtKv1BNeej1BvfLLldx3316szgfpLlhBb+VYGBZAcFYwBxZUUhVRxXSFF66gnygMkqD3E7uyBowD9PrH4lB73sBX/JdJW+2klqayMrGD98pcuFe4uDrqToYc+ZzA6jPYI4Ix1kPX+TSGt8bisdSXmoy5eLdeoSptLKeHDEEhSHnw0xQq03aj9jQwtmgsJ0efZHJtCOGuEwz3LWXtLaVUdE3CVG8avLZFZ2XPmCMsFiK5ZeIlfuv0pae6h9UJq+meY6Ns6L8AmC8sYlpFEh4+AvWn6jn78VmEqFyaQ0VRe6dLXMVTbknhVcervKV4CwBfQw/fWVQ8f7IH2VwZo58fTWB6IDWnWwgpP8SKGZe4Y0c2vXIdpZ5fklojUpfQ52azGQIHtPQNeOHldhPcvZzy+HKITiS9tZFRhf/k4PJO1gbvYKVhNpkHPmRA60372WHk5OUQ6u+kdLiJkiElpPiKFGB5n+cx9ull3Br9LXZBz7hONW1b5uOKErvIIkZHYPOAxpSpSNzXqf7+MrmnkvTVbxNwNJdjvwVg6xyOrE0U0QWwSmRM/+eXTNTCuYECzrXPI+nUebESsFDDZMtQdAYzpaOOYpdnYcWCwt5PcNVpfog9zeI0CzPu38ZQVRseA8G8xdu4nW7yVufhkWehPVtNnsSTDbv/SXDVKfq7Bzj4/EGsXVbSM7/C6YTy4Lfo0h9BQHwn/kP8qdhdwY77dgAw68tZDF95I4jpvqYvczm9nLlK8VhXWAZKqwnDWTHZNWXxAdxuKYd7Q5BIBXy6xS46j9tCefqjpxk6+Tyed+xCcu267dEjCK48iXl1K9nLmmkNz+eDu/0YFXYXPnE+XN1+ldaiVuxqB3OCLIQpQCYRgRaX3UX6HenUnG3GXNnJjL0zODjlIFWJSrKu6bD8BfQ9eOlBAoYE/MdKmJ23nqZXVkFCx824XKCP0JN1TxY+8T60W2zM3jkbo8FIVeYMZgyNpenrY6gsXbRHDkMTcJCx049TVRxL6P4YtPerKcy6m/RDnwyev234RJpbhyG3W5DINSz8fSF9bX0cfOUQGcEVLLj3F7pPXsRkFYgIErsq/ZL8CPtiGf/Im0yPdw/tge1oWs8BaYPn9UvyY97P83C73dTX1xMWFoZUKgJrKoOWugxRUPkv5yn9jnRqDtWQfV8257/cSeYxOzd/2o5Zswq5Qk5QRhB3n7ob7xhvghd+Q6DDzHvPvYWfK41XUg7jHeNNT3UPhY/vwqnTsXRMB6kRRaxpyyAsJ2yQqqFprMDvmR/wuCGaFxPn8nHbc8CrYiIQ8O3Ss2jjIuJvv4JtXDrZS0WnV3AJpCxMIf6meFIXpSK4hRsoVADaL7dT+H0hi7cvHqz0HP7QcLzj/YgcG4nkiAwPiwfPfvAsNXP2EbBC7CbuiByG93gpSiD/4HB2/zSb2KeK6A7NQq/UYgpIIG9UM5fi38asN7Nn1h6WsJLWLFGHzi2x82XDCL5r3Mn7N+8ja+/9VOgmUH5yGyuOBPPWKzKafH/hoLMPj8u1rNg2n+7gCsjNpViVS6thGy53EY999iSlnUfJ+eUmXhdeZ2AAls5sZeoLq/APaycvtQQ/z+Goz58i88wJ5E4bbaFe7F5Zzx95YagqrFh6uvHp8sEtdWP0NqLVDhDr3Y9XVAenDB14DfiwdP1SLmZc5GqimPC4WDaSp+Z8yHdhX3Fk6/OMPfkcP9z9A4JMgr3Fj6kXc3gn8SKSR45Qpp6CRKNkQGXm9z4w7cjn47fSKG/4BzFVKdjn2tk5eydNoU2AmOiQu64nc3c/sht9qJ7if9lJ7DUx83AOT5wbTl+uCY8+D+766S76/OU4JrYzM/0I8cOvcN9Vf+ZY36RBezPl8UPx8IpgwMPCJkMryZ1SBL0nRblWNO09DKgH8Jw9jHFTPiakN5HOK1l4SRIpab2KwtZHwPN30VzZSGhHMYJ9O7e8uJJtHWFI1E/z0ZMf4VA4mOJvJV0JUmEqUokLBAFdkI7U21Kxnbfx4B8PYlfYUcWK49q4cCNqbzVDFg/h+P7DePoZOXMqjQBvBUo/JRtu34DFw8KciDlsH8hlsaSaez4+xCPNi3j7xNsc+PkAixIW4fLxInnUORKSO1j5zpd8EfQx4+4ch6nBRMG34hqqPF9JrTEZ3Qk9jUFXOfToIQDuylvJmCF1SKUC6ztLiLxaQZ2XnDFPvswe8xi8u9/iH0dm8vR9mzmV+QwfTX6Cu3K1dBy3IAEcSi3JJ77D7BeNy77gBg1NS3Mvjx+fworbTvJ+y2243WDxCcfsHS5WjvrJUY7N55Sfjm0XfPGSB2AMSsK/voCW2NEEV50i81A2dz/Xzob2DNyCm17VfmROO72+QdwZaWKyVM+bLyzHy+yJI/I6pTpAgTaXTM3D+HfYCK4vxGay8e2JbznXfI547XW9h0WtUDigRKVXMeq5sVw40ETprq0gdZOWLgISEomE6R9PxzfBl035PfyU9QrxdTej7KgCYkk8uxYEgb7kiQxZfys1D27FMPU0B4yhZJksTD0wmsyiJJyISY4PA6vYXqanyaEnylmAprcbkyaftfcbqfdfjXpAzYQjEzDESrB4h2ExhKAzNiFr17Iz4EWCdMX0aw/jVCuRhoVQMXQ+tVGivu7RbC3zvX4n75iUUXEj/hb4yuxW7L0SQI1Kr+Lcp+cYMA0AMFK6joS41ylv2IJnSiS9vlHou2oBuPx9IOe69+Hxxyl8ZhyjI0iN0cvIqDOjqEkRL/KmcQoZkcMI8Q7n9Sc+wSVz4Za6OfH053T6mzndo8b6wFUmqBcgOd+JvrMadb8d04Y5fGk4xqnUlziwMozL4dlIm81oNVps3TfT5JNHqGc1zuYYrHsmIEvro9cvGqunP8mNHxDYdoqo2pjBsfrE+eAT50PJgR6sNVZ69b20BJuQquRoOvcAWfy6+Ece+MeDvDNC4KrOROSFp2ntvJviIcU0hDdg9Engjm234Dx9G59nGOlqiyDxGX+G1H+BXd5O8cgwNk58g3gl9L/1PS80R1N54mE23PIkbun17EK/VyKlKT/S7WNirupFfB0PkdiRwonIiUiA8R7huL2fot1fiaWxh9wzM7mSeBaXDAbUFhSIyUVTreg/q73UBB9wEelI5dFRVcT3uHELUupTphNRKnaVDzvrw45He/jk6UeodMuQvggdkUOxq/V41FzBt0TNoiccDJv3JzmaR/FujiW+Ip4eHzt6E5SlXWZJ4AB+MrjUfeMEeq9bwocmWDdSzdCHD7CpYTLJxpH0eI7EY0AEqSRI+SryGCFx7fzTJcXWaUbT2w8BLhoiGvg94ndGxb8iru0P7yY8N5xHrj7Cp0vv5ye9mmUqkSGl5nANh18+PHjtP+b/wUBIE5VmGOnvJuHsOuxqPWWj7iJYWEtfYxuGqigeLhgCMwdwqHQ45SpK1Tvxl/tR4nAxxXGabksvlQGLaZ74CKFlh0gqKCNztobnA7vokbzAoTlZ5AdH4+wSC6T2WuCZ/We496kE4k/XM/HoWMyrjCgcCqRu0T9yGOYwIk5MStafrL+h2MIqtzD3p7EMPzKRIb8Y2b5CBGbvPnU38p/boHQcB3KXciH7JA8MLLqBOhcgPaaVyRGH+bb+IqbRvmyY8BhOuZOkljdokzfSnbKHjuVGur0bMVd9hzE4hajCrfg1XSKzewOjBS+25U+msCILx61wamw5oQ1Kpq5fRumwIHSuOl5Y/SgtMQOYbrLSGZqO0acPq6adssR2KlQVpBJN3Iw4bCYbZdvK8OqoYvyWKspz7iA7G2z9/YMFKqo9rTyc9zA8/zVPjryX72qO4Vunw6chh8tDLmMx9iABQmOayS9QI5XLWXfHTwB4deexePtDOKvG8vXN3zIkq4DggaFMOHRNg8pTRlvJAra0Po3izsOkn+6hr6aTmqUGjk0WE9Cvd8M4pRe3yQoYXZvBRpsJNm5k6BWxQKUpYQItabNQe0HbpTYOv3yYCW9OIPvR4ez9548YfIxozpXRUT8Nn60+vL7+db69/1sMtwYyOm0tJ/Q/8a3Qwc3e39Cv6CWwVixcUPw2mte1SeQNy8Oh6uWO8DsY/9p41k9fj0KroCm4idCmUF498Sop/mJR5C8zf7lBX9dqURMa28x5VTMKnQLntXhwSx985g86KfzoCOTdOrFw7R2vd3DZXaj0KpwDzhv2YBD9W+uVOkzJU0g6/SPN8eNwuSZiDErCofRA4nZSHVuPW2qjOuYKdovYYdOQPIUu305+L3uAe9rvQWF38O0D3zLL9Q3BzTCg86M2c+5gh+CcHXNIqPDH9cwADrUnDkSEPL7lFV47NJnC1BXAVZAIqMP8OTu6kYaI652wo06PIrU0G+GeAS5PtHBZfXLwmNYWg1Iqdtn/pRtufUvUxcuZnE9FQgzV/ZPRyv6kPbCd1Y+u5vuMctZOW04JGqLO15Cd5k1JvoW6tNnoO6qIvLyblJYUvjEnoXVnYBNS+SM7nxEnrITXe9Ed0UTI+CqU/u10N6TjNJUTXncem1qP35sZmJ87QWOND15eNmx9GnC4uTC6g3LfA7z8j5dROBVUTW8jRrKXkqZ5dHZVs33udnR9ukEN30tpYvG0XCVH46shbkYcSr2a2Ttnk1SWhGefJ877HFwc3cmAqQKnOpiwBZHoT6zhxQhven3E7tPUSzHM3iWiOb4NKrbILJyZvp4/k8z0l7/Pwg1xtMYsR23pxu7tQ0rqZdbccoBPu2VcPf4Z7QRQmJhFjt+9eHf6M01ex0PzzvLj2Hy0qbcT//FJ/LSjOTPyDJVxlcTJ7iS27WmCpeAYcKGyabCqLWQfOU6/VxgRs8UYRyqTDrKpKAJ98M+qp7c2hOBrvn2YNYy+vj72z3HiFqoxRtexc8FWUCpYPXk1Q1+bxYU3xcS/X5cfs2sTGPbVRzzs9RAj20eya7tIV1z1ahWysS78RlrxG3YZrgF9IMa0sz6fBcCVrVfYuGAj+nA96UvT2XTrJsxNZlY4xeL4b+7/hpaQFt50v4nsWtJMH6bnserHsJvt6IJ1/E+TOm30ewXRlj1jkOZTEATcjSK7kKG9nKqhizD7xSBVyhEEgfhZ8ZRtFed+feoMts14nm7PY2RX3YIgQMnGEgS3gHPAiSHaQNT4qBuuOezBYfTYtNgu+4EgkH74M7qDU4GFHJ35LrneMi5eI1P6SyP4L99o7k9z8fD34Phbx/GO8b5B3ucv6R9zeStF0+5E3S+QfimXjtwOvNrEokm3XMmYVR6MGjOF3yYdpNPpHjz3X4wxepOaDh84PHw55RkZPB/8Eso/e0k//CmW/iz2Txfzm0t+WYJvgAUegi2ztmCttxJo0aMe8CbP8Q0hY/PwC4hCECD+7Do8+ju41G8mbWka3vcs4MRHDNJ8hlw9jKa3nsPpAnk5eQRP+DtwISCgM+sIaQnBtL0B+ZsK7NcYL+yyLkrcndjdChTZ3ihSk+jPbOOQ+RBKhx/nLL9zi28BJy9l0XJwFKpcG5cz6omTVAyCfTrtABOyTnCkfw5OQO6Qozd7onRqOHh+GwcB35TzjD8+nk6zDdd0KReGXSCkO44fi94nOxuCmgrI+T0Rzzv8qdP70N+nGwT6jAHxyNUiLHL8zeMAxM8W1+suvy78JaIfLx89gjxHFrrueuLyf8e325cQ7z52Ze/iWx8tTqWE6qwFxNUcJGHMMMYNPcCer+ejOJOO+4Qb5YCZgLp82jd3I2l4mI9m7qF62AXCrcHknjlJ9okjNCVNZkAtQT0gEG+TMmrZHrb2HkKu/gD9sRLSWxpxv7yah+OOURn6IZeqnkYpcSMIAjP3zqQkpYTmUAs3nTpLbsaNUjijnh7Fnkf2kHs6lwPTDtCjE2l/VXoVAUNEhom+lhupP/8/2X8C+3TddSSeXUdd2mwE4e/d8P//Zm63m8bGxhtyfv837b9g33/tf8XcLjdtl9oY6Bmg7KcijBmXKQlo4WjYUiQDzzKXJdh0fvT+GyXa6t5EzIlf89ZYBd4FNXgbxc1RkAgY5xzgtNaGl2siAfY61BY1NZnziby8i9m/LCF99EW2RWXTrR2BIq6DfqUWhV3stAhsD2S32Yfpfa34Nl+msTGQ3+YdxtJuwR8t3p7eyCQy1M7r1I9Xd1zl0CtHSb8kVnjUe2fAnfNuGKNlqp4PFjzOtq/nM/JkBi6HG7dcicLWh09LKZvvtqAIX032+oUkN4QjIKErPJOu8EyUvRVoe/VUj2vmzKKDnHQnkFiuZ+S5keycvRPbqWaU1c20zAhgVWg7HdZu1s/+FW24D0/VPErwqmw65FdYqHyYi2Ed4DUTlV7FhewLVGgrAEhWwjrDdj745UV6sq30ju7lypYruGM7GBkzjID2ABojaklZcY27WQJT35/Kp8Vv8Pv0/ax68hGiFRqEdwWUHkrSlqZR9Espzae8yB+WQdbeqXQF9dMZokRrVXFs3DHGfzQeH9mrZH7+EUM+fp+dymfJfduLhIpQHA0CQkAXIcGNPOeXzI7OZwkNSKHAcBGdsZGzDxxANvwiY+ecok2q4llvqMZNxrIMEm9O5LdtGqxOHd3u2fzicxj7fRtQJ7/LhW8usGvlLuL9YlDYLFSMuP1v81Ej82C6So9yUgEP+7eQG/My0j+Lrx8vcxHSHsDPFhm/nE5gzp1WmpImE37lAILFCoDPnCM8N6mA1dNO4Zn6CmWj7iTpjKg50lXswZFvJ9J7y7fYtFoaTjfw5z1/0lnWiS1SwZDb9tHh8uRqmieVydN4McrA5H9OxmlzcvDbvx6/lMkHJxNu18MrkDA7gdPvnQbETfovIEUQBNoutuGf6o+tpodp+6ZxKf0SVf4ngTFYDCGElR3C5ZUMCaGc0ifzafj3OPvGobDKMHua+erBr3jW833kWite0U2senspQrwnWls5EpdD/C6bq/nl3WW0Pfslrw1r4JDDirnJzPYV2xn/+ng8jSp21fowLKmbB1c/TFOUGtejLi6tv4RMIwbS5QnlVMdWUx37Ejf73QvAlPemcPC5g3yd/jX35d93Q5X2X9ZrsOC0OvGrO0NXZSyBs6NxO938NO4nAIYznKKMIjoj01CGaJEIAnKnjaCas/RnpLKoczH7cr+lUZ2NLsoXu0ZFfeoMoptP0TD+DjoFTxabl1N6eAimp1YyfpHYaVd0roy6nr3s7FWxKu15TtbMQSkVkwrOASfdXxwnPCic2uhaAGr064B7B+/bZXfhdrmRKCR0d3cTGho6eOwvh15rbMZe5QQiiJoQxeM1IgWD+ch+GjU7+N4YSYdZBBC3LNmC4BLobexF2Wfn8MTDuKUDuFw2dIE6Hip5iLc1YjeLvK+PUdFFrGiD8wOi87f3ib34xPsgVTowZ/cBxXzy4gOY+3rhDRhy2xDCRoZx35NPY/Op5busPJrj4lCOuAeA/s5+SjeXcuUPUbPh/oL7Cc66kf4haW4Sj5Q/Mih+DnDWmsFOYwYZrTDsXAyCU0dRRhELE+fhbO8m5sIhHCodQrQnMz5eyOsjzxAU58TDU1xv487/Rq9fDLVROfj3349Z/yEA7n8TjnfIejidMBOAeHcOu/+ciS65Ec9JEZxpOo5wrQpRgoxun24KsgqI812MW2zixKpoxK60czH9IsmJ0wfPu2PFH6ScFAHpvn4VZ1duYqb2UWSlnlgMoXh1VhPYZGLJ/hn0z93PO5ln+DpqEXe+cQ99OiPf3f8dcqWT0SNLyfECAWho80ZdF0VtVC0OhYP3n3mfyVeLaf6wFqEqAsckgbbANoKNy2kN9yeosQ6/Vi+e8I6n3n2Ol6Lhyt0hHFYeBgEstiAkUgXBrX5MOzAVx2gr+cPzB8cwbf801ANicqelsAX/FH+y781md1ctgk3gZT+BE2oXgiCw4scV+HX54d8J5285zBvx5cgkEKxw0dpQye2/ieuppaIeNR4EDy/hrplWNjRswyw9xIYlG/AyebF8dRIO2zIeeOo3ajSPUNT9CJ21GvSd1Zj22LBpP2CespZKewKxhh5U7gEUMiW917r4F+pguR4+6BXYcCgVP68jPJaey8LfFvKGRExM/XznzzwwS+ymTVmYgt1iZ+2ktegw8NGt+6lcUEm65A7uUUkpTxTfoaXLwp3fzady1FkSpu8lU6olKU/G4t9FQXPbfVoiTo2kWX2B17xbideqsHRYBoE+S5iUS6cyWCc3Edfij1QlJzA9kLZLbdQfrWHrH8uoTwnjiclr+OJpB42yOxl9bZrmHHfTRw79i4+hVfdi13ghVYFTZwCVBwqbmAz3aSnFabv5hkSjtc2M5OAYmgIs7I9byUPCHCy+EbjlKsKvHIDHIPyhGGRdMhz7G7CM6cQlU9HvFYzFEEr3+HFkhv9IivQAaZYMehrn0KX8hC8eqiejYRblq5dQOiaPi4FOtN4C+hFTaNbGDe5pgR3RZAyoSLt1E8frFuEd70d9Yj0KiYJvHoQqB/wUFkzgjiHcUzwE58tOXnhFTnW+g9KMA1TFVtHkIfAKoynbVsbBFw4yf+183Aro9+gn65waQ/tPwFuYfSLxr7uAraaZDoWLCS1uQoR/4SudSFnVdjKLksSHEqlkX+IOioIaGfvW81gzutk2bx2T90Qw6Q84MVlOa6iTMRYtI49NoF5qo0PjIrD2/OBzVfd1EkAr7Q/9QkHHREZ4ivqJJoMIBk3WCtzSeC/tRa/gkKbTc6WSjAOnuDryTgY8/Uk9/hUlDV7MmHcPUoUUpU7Jgl8WsH5vJgu87Xzy2FO0DHuP6S+9j8UQOgj2AdRWHcE06gIh4S707V+htX7N+OPjcQSryAhMwx7uQT2zGR7oos96PRg/pLZxaADQ2EHTy1i7ivrU6aj7ulBZWoHjzCp5mS2pt3MxoQyDWkHuS6BLm8GZiW6eO7aJdxZvYI5+J/0PfcHL6ZuoHjoeAN2AlU2LnsDLks3LgriXHHntCH1tffSPTEFr0XIp4xKNMQYeCPbmxLjjXMq4iMKhQOaQYjs1jMjpx3BK7NjcNrYs3AJAaFcikkADId7NNF8MRN7ThnHPGeb/2kjZ6Hvol1/g148W45x5lLe0rfja/DCE/kK/TgYSBhO6KxzPMjujjqKEZqRqJXKbBS93NOvMsK5XQqtSy0q/ThotWfRcamDavhzirwYQXSsWoNmmtMOdsGiLSJ9uajBx/J0TFB7NpmPoRTx6pLjcErrD0nHLFMwYbaa4UUyOREw7zcZuP6LLqzC0VdASP5a4cyKQ9J1XMwvcwxEUqcRukaN0KOn18yf2iokRigTu8yjm5P7h1CrtyNsrsGu9GdD54QZcAozRaQgJaKf54iN0/dXZLXGzMQjstnNctYZRZbaj1Xti2nWS1ON5VIZe76x3CWJn0rSPpnFl8xV6m83I8tPZ2q8hPH4yYRnxZNwfy+XfLtNe3I4sQkNIcwgbdX3M/moJ9vs7sas90XdWE120Fe/lt3NWF8DdvYc5cnYYfTk2+oVfkDttqAaUXEq/REP6Ze47p6H3hJq+ZRc5FrkC7+Bg7q4cxcBbj1L1/Do2hgq0TT6CXP8g4zeEEOs7i92zd9NpDcG/7W2+G1HLl3YVHsMm8vljnw+OJ7HpLcali3tWxvIMTr1zCo2vhisvu/itdhXRV5IoyigiXTdn8H/qT9bj/PYgo4EDuQAC6iAvSsY+QMK5dSjs/QhSF6NmnuXSqTS6jf8gW7eJrks/Mn3fFM6MU2Ac9yN3e7t4Lf4kT7T2Yq4SfTQPUwuCxE1fl4o6axWVldn0N3hhs4EgcdLjbaE9LYRgvwDG6hop0jrwUPrSr5RSkzmPVu9tIIDULcV5jZptyOIheIZ4UrbtOkhTllDA8IRbcXua+PCpD4msi2ThloX4dfkxrHQkr/0zB7N+N7kJ89EfLKMhvIGu9734svYxXk9ykz7mEutcL2KsNgIQU61FU9gAgH+8hT/j/mRe20zUhh6aQ5oJGz2GoPfPIkikeEiNXL11GwOqLARu1NMScFPcEkXH17EMzLMi99DR7xVCn084Nq3hWnGgFJ94H9LvSOfCNxcY/soYHFVefBF6Ef8xS1D7exI1JJa85DOMiB3P4U2/Ifu9gPfvrudfAdWY7WaST/wK+GP20hBYHgQEYVq0id6oOlrPtnJw0UEAZq2eRfnIcnpCmgmo/Ax8v8DSOYCxThx351wDRTlP8lqEwGPtsKdfjJVi1gewwLSAshFjeODyzdw5ah6Xfh5JancgfCjqXvsPCSD36VH8J0tZmMKOh/bi23iJjohs+r2CEQTw7KxFYbdg1fnR7SPuHWVhL2A2XkAQNtIWO5pGn/WoB9RcTbxKdYyY53Dh5N9zgRdGG6gM/Qb1gBoZYURKpMgcA0hdDhwqHQG6Ony1TbhbfXjsx8foz25GyIGS8MduuM9x8oVIOq1i/PE/Un+jTsby56dvEXn1EZT9EjRmsWjGLXMxcuYZZgXu5vFLGfiUeXHvd/dyNreW9t/2IwuYSNi8M7w3P4j1HWfxvtBKeEXRDeeOKQ/j1I5kQp/upi5eQ9aZXhROBZ71Pnybu48jM69SOe08nx+7i86wDHqCkvENNfHPF/+Jyh1CpF8Mlx/9lDs0vyA0iFllU0gLfvURTAq5xF2mdjpe+wcPBi2gsqkS7+7rnVR/3CJ2NAZnB/Ncp9g93VTczfD84ZSklFCc2cNQ7SsYVeUYjAY0AzLkBb1MOjKJVGsAdTVh6F/2ZuzxjMFzqvtkqExeXHW0UqzfjKH7Xpy0MMAhYipEX/WpBaswuaDYJkVq8MMY7IdNbeOPJ77gTDjs7c/i5lYVte5+/iUDv9ZOFLoAvnlQBNO93DU8EfgopXVTKF07gufee5jy+HIUDgenh/3MXNMkYgjj9AenOfPhGe48cif54ScpX/wDbaueJ8tPXE86v+0k/eZ00ouhNk3H5hF+1I6uY3qbB3KVnEL5cKozVbikA8SXbKOn3fvaty3QMaZjULPPc7ondYYmXjBn8M7iDQCUbi5F5aUidmosLruLtuI2nANOcp/LHdS3DM8N5/Kv1ym+DUYDLSEtuNyuQbAPQOOtQeN9Y0ffXxZeuh//+gIqbnkOjwDxb5wD1wu2lQNm1H0ddIVnIZGD4HYPAn0A7dE5pNd+Q0hFETJJEMTB0VVH6a7oxu0U51PqolQM0QamvCMWn01YNYFLl4CXxXO0xoyiXy/m/wImpHKprWbw/EHGBcTm/06PJgSWjSX1VjFX8PM1vdm/NEuBG1iKGv3WEtIUwrQD02gd2UpbzCS6Q9NIPvU9EkGC/PAY7pcJ9JwcRs/sHsAbm8YLldXE3K3ZnBhj4vyI87RrqpEoFbjkKmoSg6mLrBu8hqfZE6lGnAsKfwV93X3InXI8zZ64ZW66A3QYvHwRBHDLVdgCwpl2XxCpi1K5KG4Rg2Cf2tKNztjJtKIOEKR4KsQcgmdHNUgkmP2iAfdgXA3gtDpQ9ltwKrX06a7widc/+VgGo2/bz2nlHZzoXwWAXdGJU5ASVAOKkMtonqzi5SHraLj8E4sKHhWf16h6HDetp1xj51jsBR6ozeGVt8WCppbow5Aqsvcka/7A+MJumtW3ECIEXBuEuKCmpYGmfQa9ysMk+A2ltlEAiQSTfyy9fjG0xeai9RX/Zfgjw3H0O5i1ehYe//Bg7ra5BHeFw0uAXIFTqcAYlETvkwFMvLSLJ4a2scIGLrebyIu78GkpQSKV4D7bhX9EDZtDvKkbImFchxmZw0pXaBrKtib8O/25fctCZn/2Hp82d4BcjlRwE37lAF3+crSvvsN9BhVzDVBtE4FfIXcMBc3baVEcY1SvmpvqX2RzTzyv5EVh67Wx5u41WDVW/LoCST79A817J8H8sYPvJX5mPHvYgwQJQS1BuHrEOfL7/N+xtFsY+/LYQfrq/2T/3zT7gGtySWDTGP5L44noa/zPnN//Tfsv2Pdf+18xpYeSx+seZ8ujW+hvcmO/5M+viaXc8uV0+kbXYOouJ/LSVWoybkaQylH3dTDQ749LpRApMC7pANGxanpnB9+FFXJnK6yNKmWs9WZiC87QGZaBMTABz+56SmMrmRK2mYp2I7KgaVTk3EFMwRbUli7MeilNAc8QGn8vfFiEO3U2rkMuTPUmMptC6FZHIJNK0ZhaUFt6cNoS2Pv43kGBX4Cj9h95lXk3jDFJM46ZJ/+JO+0AA34y7lLIUJs7RN04YxMjzpt4I8KXoiVX+PbUs9gtDjx62lD19xBT+Afpx+/BNux9xs49SXtxMN5ZIL3Yxcw9MzFO1RB0qhspEg6vvpUwXxfVXlmYXC7+/BPGHk1EJQRguz+IY6UPYzCALhB23Lxj8P4makCjsJN47xa2Kqcw5LYhDLltCG9I3mBa1TQAjGViUuPIa0c4/tZxJr41kXPxJiQVED9lHyn+QzA1mPg06lNyHs8hbHY0DUer+Lokg5B7N1DiyCDZ8Dxr4pfQ7Sdl/PHxuF0yWmqCKY+t5djAj/j0ZgKVKAbAVu/k/YcPUBTRRp/FH5vqE7pDUtEZxYpm1/kMZA/soqUzFL+37sE8WkDxiEKkD5QLfHrqR0ZP8qPEOZ5KVwevhmViQHQMvTqrGdDeqP32l7nsbtbseohRGWuw+fYgVcpxJ6VQYg3Cu7WMkIrjrHn7TgqzChlzagyOaVbaYnPpDhnCCOdpits385l3DV1KGKMRneUrCce4kOZi7cp7WTN3LWV+Ahs8NWzvlNJ+uX1QG0YSZuDy809z96rv2bRgJFMbFCC1X9fhuwb2IUiIrYrFcE3/6d/FrG29NlqLWkmen0xTXhNrRq5hyntT8OkykHsml4yLGbgCzsJbY7BrDLTEjSFuUhIXj71Ba+VVenwjSeifhq5BzbL1yzg27hiOpi6sTm9WOm9ibp2bBp8+1NZ6DDXnRK2+zmr6PPpI8jezwkPG8S7HYJdh2pI0pkxIxt6fDF9+TIDagUY1YpDm0WV1DCbr/jL3NWqM5PnJ1BysoWp/FedXn2fyO5PRBd5Y8ae2KPBuCmf4aTfNCReAaEKGh9DX2kfVvioaQxvZM3MPk0pEvvmOIRPo9onFw9TMsNwUyk/ezlek0UQ0aRoF/rXncSg9iPz0KXYVvoxMaEJvK0Vit2J030b9yQEOv3KYtlP1qJyT2TXgSUZ4B91rm2gZXktsbBS9jb30l9YRZ53AibEnWLRxER4DAfDu9fv+854/ubT+Eln3ZhHx8I2Vcf1tZqILDuDbXIytWA9fPHnDcUucjEMhp5hz6mZs+dtwfJjEkMVDaLnQgmeoJ+cvFXN8/HHGHh+LfkB0KORqOS8PvMyvd+1luzGI+afseO9rYISvkiupVzj36Tn0YXrG3vsgnc3f4x3QgzD8EpeNIsCvC9KhC9JxctwlsrzFAOq4KRCPppPM8RuDV6QXdx69kzMfnKGzrPM/Bmzl1nLuevcu5vwwh5u/vZnse7M5sq6JmOozbHhnBAmlYUjcPnz+2Ofcl7OZrsMlg5X3zk0eAMyYdZaAgH2cUn9F8K/HkLqd2DV6Jm/Ow+aRxImJq0i62IfXMBleVw5i13jREB9GmBxCZBIu+uygNreN4CR/Am4KZLPXfNwyMdCTIqchooGGiAaWx/yAUNCIb30HDQawaq1sXbCVlZO/ETtxv8ij9Ldi5IDhJg/umvYRO/rFxJczdQj7Qg4z7xcFSpsD1fGRjAjv5tP4PASgLM1Nu7cIEAXlZfNVQwvresHrwHikT7n49OlNWNTlJJYlklqSSnd0Gee0RoL9NDRn+XI642eiW5/E4pNM/cKDTE3wZWvHAEn50xCm9uESRKoXbynEeVahMRZzfEYBr0bVEpz2E1yvXeDQ5EPIXeLfN51rovZILRl3ZpBjnEloN5jH/IvSjDzGh77MpfRLTDoyCbdEIOLnYfyjcxSLH9pKbrU/kkQ5FXEVxFeK1Y9Hpu1joWssavV52qTnkCAhpTSFm3bdBDipJpY5edOY+PVw3EPtNKTcTOrxbzBtOoCMDHaQgWmEnftK9ej6/XlEkBHVMJl2nzO80tXP+VNFKCbvZ+6WNKrSuwbHM/SBobzT/A5NYU2Dlfqjnx9N26U2THUmin8pZlbvRJyBpRwxise1Fi36Xj2CVMBw8yFeT2qh7FI6XgYlAy0Dg+fWfNdGKalEJtfxbWQrZ/RzOfyK2DkiU8lwWi2sufsXGsIbYOJhdqXdz5yjd/Kez3scfm4vQYCHpJz02Y18kw3T65zYWrrRd/RQlQDRlS5OWyMJVRXwXNZIKo2/4XZHMaAPxqujkorhS7BpvP5GkatL8KXtxc+40GdAfjUct1ussGxOGIdXh0iNLf1yOduGVOCuMmPs7MCroweH0gN9ZxUdihQ2Wf9JwcYgpBd7IaqeOTuvdcVKxfHt8GsizPknglZMUrjlKnp9o5A5bQTWnifWlEdq6m7ypI0g2YR5lJkyYxkOxGTqOuVT2KR7kMudmJvNSH74gzRjMxnC7Xz41IcIHuKar/XXEjs1Fl2QDlmRkwBjAGdGnSGwb+SgXkhAXT5yg47GmY8QZb4PrT0aCVL6Aq0cXLAFq8bKkhkvMab7DPOdWqqjc4iIDsHo8Rl/6TuPPaRjaeMKbn9mA+b3VrPSuJiM9z8AoNcvBltECD4+XQxxX+CJQFhkNuN2Q5/meidQjQP6rB44is0Q3Io0XMuAhy9uqRhOdYWmEztS3KPrT9YTlBlET00Pne+v4OqDf5C2ZDcXPDNwO1z0BKeg7W0jLNtOeecOdBmX2RRRwVzfLJT7vOn1TxK7h8wT+DzhRc4ZtvI1WiyuO8gt+primGexaqw89tljOBQOdj32JW63FMHpwqHyJPbCZmxaA+oEBZmxRbzaE0qjs4Pn1ALVoxyUh5TT4nOZHeoyHvS+iFn1J90BHQia60kpT2sa6bXfoXT6Dwbtx98Sq575toBbE/7JwennCTSKyRaJIh25s4m7fr5r8BxPLj7G6VuO4PB5lvjm16gIeZOkEm80rU0sWv4tX5d/Q0nb7bi89tMWJdDmc4aES10IdUmszzpHvDMeT6cf3V4VeJjdGA0MUhtL3U18EQDvdrlwuyH16BdYvAzsGwEgQeY2ktn1C9YLHejfeIq1y9aSc+46rZH80nX9HRCr/qWCBAGBxg9XMDUkFsc/BOoCt1EQd4yVd69m/SoFs36zMqBsJDJMjru9Dd+mS9jVnpybGMnZIffzs8YX2Y5wGhJH8mfcGbR9Low+AdSmz8dLtZOjdVHkbZmEf6od/5INtMbk0pgylVCFm3AF9PrN4aP3Z9HerkfX9yvBPRXsHPY6yXINTqcfe2q2Yf/wCxIXhdOfHElTkxa78gSBrYFkF2Rjny2uj36JflTtr6Kn1ohxcisJXhay/ohHK7ehD9Vz17G7kEgkfJD5GaNbw7k1t4zzCgeCxM3FUVFM3nQRz54GnJ9+xU3jrZwd08T2V94lV/kNow800e3djcnLxPT90wkd0sTBoXmUhF1FcOtJKFVQE32eAc041FYHu7v1rNF5UeWq5im9gGePmUBnIDJgefor+HaswzcDqvyrGOf/+g3vJfPAZ/TUh8PK2/Hw9+DZDpFT9qG1n1AQAYaoMlJTy7gn4F+E54bTcLqBY28eQ+rrzYDUyrjTuURUJyDMFbA51qKwWwmZlETZzGVoPR388f0cYsIsyLx7mbNT7FQZc8hGvGs6Q2eeYUFzFx02A+nFOwmouwDAoWmVnBz5Ky/J13ExdinEwlQHLNw4HaOhg9Cmq4xNXouP308MzD2BVTKGa24HgsQ1yGjQXFYLIuM6oSNCuTj5CfLjH6Ah6HdU9hCq7CFkyLMw682DnY4ANOmpiqnEpvYjOW0iO/SHUEmmIcgk+Cqvl9tX2/Lx6PNAkAj0eVnwHe3FzZM/5vW4Vh5qhxq7nM8e+wy5U87DdSMAgXETfsBDZebD6eXc0hSM5kIyI40jqRl9ll+D4YK1mO26DA7fvoERKQvRDpnFlSCIKN5FTNE2TE3F9HcuQOunxSfeh7oTdah81DQmVCCLbuSI4SFs7KLG14ttC44SWH8TWV+UAz7UlUWSdCwNS6yF8oRyjAYjDUnj8G65SnHKUaoiodZjJBf3XO8g723sRQjp4LlFJ3Ee6aeq+WZaKvWDcbq0z8UngQKn+7y4vHsIT54bQc+SHtTtSvx7/WkU5GT5n2OUdzNnkmIICBfXiJu+n8uzz8LRZ7p57Fnl32IRpYcSd78NLe1UDb0Vm84PQYDwa7S/CAKJKht3+sGl9bPxLY7HvVDUhvI1T0AQDKRdTqM0pZTommj0vlakelD3thNUdYqOyOlYfSu5EriazvBslitVRFzeim/TJS7MfJmUoF08PeIR6i+nYVPZcEvBXFLL7RvmcmLsicHCw3EPP8VXUsjVwtQNY/D2v8zx8eI63hhcyKScXBRaBemHPxwc27HJBRw2Wxjd403ewCZultxJYFsgI86psNd3Mr5sPLXhBt4/kgJR1QyEZ+DuFp+3h7aGJ5/6jM0NCbTEjiYq0BdtazTHJ+WjcSRSF3KUurA6FrZAvwCROjV1abOJz/sV431NzIucx64F5wbpaQUEMk4rCfWcwJYlv7JAK6XdfgvVjlrMEvEvZC4Penx6/hZb/rvpIw3sXRhPQfw/0dlHIZXLMA0c5/AkLbdtvA12djH8lW/wrYugsHQssapIPpu2n4VbFlI4/ip+7aEs/vJBzk85BQVJND7fwo9PvYl/91hSiiYDcEezlJbTGdDtS1yieF1DXw4NunNEVKl4PuBrTnfORWPWY8+0c2xONucSnmflqQkMDTahyaykPs+fnrIeUm8zUJRRhMaqwawzk3g1jD+e/ZzxB9/FVG+ir7WPA88doH+mkV43HJ/3J70DIovPR9M/4gNE/yaqOISfy+/ipR/+SaJfIvY+O7LuXkyBCQyo7JxY8C+Geil4c28asiwpXeFdHJ14lObpzTxlf4r5q27jN9lvaJ7dRNCsIHoP9eKf6o/KU8WaUWsGn++zHc+i9ROpD6Wy62tVW1gjt228jV9v34D731qBao/WsunWTeQ+l0vOYznIVTempc2+UQgSKYL0+u9lShk9gYloe1tRWU149DTRFZ6FIHADs0x75FD86i4wrHgnAEbfJgRhHDevuRlLu4U/lv5B9n3ZXNly5YbuX/i3biWJhMaUaYO/T35xLl+u+RGAAONs0uu+wav9bWw1NzLaqL3VDPQM0JzfPAj2/UXpqRktFsR2+nXy3b3fsW3hNhyFXqgt3TSOX4q27xt88z3olTnpDOxCIhefY/Gkx7g14RI1H29n9u7ZFGYV4tvljVtjwaX04GKOhqsx1+l5v3nwG27xeROAuA/iuH/t/cgdciSCRKwyFRwILjeCIKVq2CICAmDsS6Lf2ry9CqkrE0EQ447q7IUAKJ2Qcuwrjj0fxbxvZhJxeTeCVEbp+JUIEgGLzsJ7z77HZ7k76MivI/3wL9Smz6HDID7QO1qhTtiFy/oSQw6EEtMzj23ztw36ljKVA4fKgaCQcMe6RXj0izkATa03+yuDqI8U2YZcXlJ6DCK7W3BNIXGzzEQZipFrrzIvrZPbO60oDw/w6NZHaVrYzNd3zsZnxIMUdexg6NjvOOLbhc+jO4isKKRw+vO4FOob3t9f3aoADqXjhmNCn4XY/J1oetsJWziJcu9tRFeGkX0iFfNYI8oBMzKnnZ65y5FuX8sv793BkfulCKMNGC/WE3dhq3ieaD8ck04xPbaDK58vIRwt7pkjqTLqiCzehSlsFh+Ev0WjM43Rxxtwt/vjXOGiKbab0+Fr6bWDt2IAqQCHzg0j5XgidrOdhggRaLP2WTk24QrhqdNuuP+/NEsBHvzmQWxFNpgtFv6a6kwkzEkg8eZE/k/2n8C+f/+d2w1KqxGAPu/w/4J9/y+w/4J9/7X/NTPVmaj8uRJdlIFgiYx1QbBab8al9MVa3Yp/fQG67nraYkbRGZFNnPsYqvpmzLXTiC0VgT6bWoNtZwZPdySx+KkNTNRCq1JJY9IUwsrEir9Vq1ahksCOvCBaCl04EsfhkqtQW8RguCl1GZnyKB7reB6ztJ4fJbB0z1Jqj9YOVue4cCDt/pTYEg2mzkeZ/eVsjGYpq597E5VNRVV23d/GpzplIuL4aNqin0YlVyKVQXThH2h7W2lKmEBE8VGKAjK46e5dfFOYhqWpg+RTa244x+61M1k+czOhbjlOHZwYe4J52+fh8JJS+ta7BAZB6YUkOuO76IrLYdipN7i04ADpiM5FwoMXaCz1RCOxAnNvOPcOC2R4vcx3ygGGxUwZ/L0tVEWN7iJbbtnCyeUlwHU9soLvClhRP5OToz05OPUgfqGBKHVK1AY15z49B8C6O9bRHNKMVWsl1p3GEIOOBmUDKnswKV8/SveuT9F4DJB3OBNJgoUj04zc+tP1+zI7pLzy01OoDnkSuaqe7tAh9PlEsOptJZ+8VENBw3nCfc9R59OF00OLtceKpd3C4a41/DrtBQAcnr+wvv1RgjUQN07khe8o7cAtk5N88nvgRi5qe+8A5s1q3qsYjXHsBZwDDlCpsHoFY9P6oPc4ha7IA1VgF3se+5LbE7/BcqGSLr9jjMu6k4P1z2NR9BN5JJSA1iBe9unCu/YnTFG1tDrm8P3drzJUBe61S0kqjSKtJw3BLaDyUnF0r52Bxh1odVY0/RpSjZ44bU6USnF5vjhkJu3SIoYTSfGjedhsdwFw+ffLZN+fTc6jOXyV9tVgldpfzU0SqQTP7Hh+uucso054E94jAp3BFccRJFIkfr6kFoaSSiiPW4s5Z0xBv8SX2uQsxh8HG6b/h72/jJLi2vv40U97T4+7u7vD4IO7BELQECBKhLicKHEPUZKQQEhIIJAEdx90gGFcGXd3ae/6vygyHJ6cu9Zd/xfPs9a9Z7+CrunuXV27du39+xpVaasItfmBtg/fp047F3nRWGSN9dh2VJG9sZ9q7aeYO2zpfuFFbMeZCJgYgGOwI85hzpTENqOR1vFV57N0T5uAc1T6f7TkDCkPISE3gSHXTiwmF74Kvc3mzt2WS9rTaf/YYN+9fSIKg4SfV/3MQ0lbaMltoeFKA+NeHkfl8Up67Xtx7nQmMuMNilySkEiW4FN6CvVAJ2q7cCYFbGeH6wtkmruZ0puPX+FRJAh0/NbAjONKbB2defvFYi65F7Nf8Rk/jRMX93InJdneWdQ69fFWz3w8/XUMqUVlZ+VJMdDbv1qGa7vIApT8j0WOz2gf8n/NR271z0evvk+Pc1MBWhsXbKaNBW6D7CnrUhCCBOSAh2qARkU7WqMWhwAHzr52liX7lvD8ZTGTIrIkEmutHbUXatk2YRsj149ENsqW2CcPkzvrDH7aRDp0RobaRWXz3B/m8senlXxz6nFmv3mATUlXGDCJykGzwUzt+VoCqjwoiStldVkYLl8nI8s8xtw9Y1FYKcQ5UgDvkd7DljL/3kyDJvq1/dSF12HvL7IBvUtPiYqW3nDqRj7DqShRZWG0GLCLD6Q2ZhbepaeRNwzS5KfjlaBeBixw0SygHujAJFdh116FwqhF0dNIUPMqAqp+QQiwx1J7kSEHb+pCvXjEHl5xEvip/xhZ4X0ku49AENxxHhhHk9MuAKTIcJKCXhBz0KRFhQTmX6Uo8rb1hdFswmwwc/wp0catLnoGxYoI/sz9A6cCV6Tj+ymV/06x39MMLQ0goex1PLSHmaA2sL7DH/OAAZnjJyhszwCTyZiQgcKooFs2RIpLO64yLeE1b9Nhdwr31jLiCuIISX+S10aUc9oEoyUiS1uCBO96BdO85MwLPgDN0UiuTEEfNoRrXR/R3iE8P6aC+4In0Kn/kkMWKPZqJKB1iMe/epxLYy6Rk5RDRWgFEkEcg5ELI/FK8eLyx5cx7C7i7W82c12Qsr1XysOCwPkJ5zk/4Tx+2pms/VAsgGWdSsW5OADpw0qOzThG6NehXB51mWU6Z2Kb25lS4YitcjfxxSEEZ80BwH5cOL0XyjA3L+T3JR+jsQvHq/lxaqNnDudhVQZVoiCB6cenE10cjf5xHau3jCMr2YrL6f3U9MQzWrmTjsWn6LeKoDW/leubrhO6PJS8A3nMPTCXmuAaglYGYdKZ2Jy0mYTVCaiXzWSy8AyhWneutUlx7HXkhY9FpvfAjAFOjbuAj7WUOR+9zmCsDadW/Ea3W/cwYNHrH46V9V98/MgLRG0op/Kc+Fw0683Y69WoFWYW2UKJQbQrVtmpePzm42RuzebiF6cYO+8ifWYly6ukWAQJfdfzCbuawaG7BbYvf5tJlnXcm59N7Xcz0T10AmnflGHAzixXorNzQ/o/rcKVUr5VdbFy9ywW1PtheRkcGwpwrs2mImUJIS45CFn5XEm7wr6FO/gqYCpOTUVY94pZs261N9j6wH7OpbbhFOzE6ohZHJ51mNlHZmNUyum1bcQmsIENXnM4WfsETXXR+BSfo9ctFNVACRPTMrG4DxCeZ4ulK5NnBgy0r2wnozADNxk8Zg9rTV/h+EAdj+aPRyKToBzqRW9li9VQL+u+XceQtwz+JdpsH285TvGfxTjlD/JoxqN8+cSXDPh2YjbDkL0nBemPsWCcNQFXO5gY8T29R3oo70ljaL4tBbEiku1i8UL/4aOMmZjDihcf52DHr3jWeVIUXcSpKad4+vOnqSvz5/vyURxzOYdBraTbIxLX+hzkhkFsCs8SElZHqbaZXw6k0G8IxuhSxtj9B2mXRFISVcJNA7gEdzHh3f3sG1yBb3AIZa0Jw5elMXIKHmJdhLb8NnFeD3fGZsCa5+usKI+9RqBVKH03yoi6KG66+49D+quXWRrZyqdmWNtjxqXmCgp9LwXp65ArpGy6vonpqk8Zff0iVc5BTNnfgl/YbPLi87Drt0Ou1rMlfywZeybSM7WCxJMietDrFsyyp95BoZDwVZYHK7e9hGRRK1bpq9F2mUlQFZITms+V1gdZpRmketAak86ET/FJdDbO5MdXYt3diUeHP4IgFqnv+vUucn7MYUhhT0V3BOOy5mPfVoH27j6kZoGVv60EoHxVOfOS/+CNHgu7B+DrABNpGQpswhPRK8102ht5uzmcJ6fMZnOHKyWKx9nn8SYOAyMIKRAzhS1esPzb1ZhlGmqcqwkpf4o3X3+T9T88hGOTJ/afVvP2r7+gL+xkaFcf1WEKqnx+4iE7GG9lYVBQ8GfmHKzLBMy2AlXBVTT4NCA3ydGpdazzE9lUdRfrOP70cUJnh1I6uZebyVuIOP84NtIE9AMm5m8vB7w4uuk9GhacIT/uGmOLr2NjiqF7TC8lleUk5EKfxgXjCC27m72JuRSHxqqF7ORSwouSCM9vRW9fS3+dkddCHkC3LJsEhwUMyIPR2oos9HeGnuO+mI/olEuRXCsjoKWajkAVmn4xYzW2QcvdrZtJdlLT4R1HUKAHirAQmutCMCkv4NjtyMhrIzEliGoirxTRGaHrZicONz1IXXudnWn3YxcWRFlLGY+/9Dhjfx6LIsYWSYMtqQYbBl7/jkG3uVxSjGSk9XpsBsX1V0uGFTZzlJSZvBhjp+Za6jWSspPQq/Ro44tIM9ixy6+Qcw5NzLlqZOLRufy56E+uz7iXIVUVwc4zcGp5h7GH70M2s5esabM4G/kiq+1gkvt2Bm0n45F1jQFpP0tbtSRnJVPvW0+bexvV/jl43SoiG7VGTFoT29K34V7Qy88z03l95Tm+dhWt66d9No0taVswDhoxrNfxbsxLbPhyPqZab1p66ml2+hOYTdOZUvrK7iMlMYfexbuxkcYzJ/e2yheg/FwKxc7BRO23oTqhE6lZJIt0u4cz+QQYpCMYiuij1uUHpIICo3E1gTXB9DiH06up4rTJxD3qQE5XB+PlZcGqvYiAvAMYw3sp8jZREFPA5AjxGXnxw4vUna8jJGsAulypnw9Ld0xFbTyFIm0Ui880E31eVP+Upy7nlHUAF8esxt88iXiFG57lm/FsAs2Rz3hs9wZyHtrHn61WDEWVsXbrWowKI38+dorcGcGcaH6Qjfpt+J+JpynRQHhvOEt3LaXRV4u7fxOTgs/yk2k5Z9oP02ZSEpsvkFAygy1jM4lVQY1hALMabobfJNFdQLi1Be5zCcKtNgv79koMAwbUjmpGPjmSMS+O4fDLp4hDyY8Tq/GuqGVoyIR3+xqm5Yk2fyz6lDNdBzgs6WX2lQnoW/RcH92LV20jsRXpGOdfZZtPFTv/WERXSSqzv48n9/tcAAp3FmJV6YvKSse5PemU5l7l/vMPEDIzhIqjFTie7aeobzFLH9nHOnc9Nxz6kUglZK0v43LHIR61tuE5L1EtLfGNwWeKmONVVgZtF8uxv76DTz+FNRfX4DfmNlFP263FdnIS/aezyfddQmDfHwiCE4OOPmj627Aa7GR8hRdPhtXzpv0QLXYGLGaByAs/0OXuzuUFN/l9ye90O3azZPcSWuJ0SMaAUt+PS2M+FrmCULUHDU5yuJW51+sahEmhBomEn2w38izwUkQF4xeOZpwqGVP/IN6N3lhpRfKd1HI781cQwLqijsCBwGGwLy8hD1mqNzYeNvS4heHQJroeTDyRSkueL4oOF6IC+uBDO3584EfWfbdu+POmN8q5YBrEZCjCErUMRYFYtwl/bAYb/gqgMKaetvFlhLn5EZdxCNd2b2pjZxJcXoRvxn2cnHEcWZUvEj897tWZ2HWKqqmokii8JZnYGbqYVJJIj7ce90olLuoIspOy+atbQ6yvN/5aFcqmUCzqIZ746mHaXFu4OskXm9ZjWBnEgnV3dTdbR29loGWAWVsX4tEzjYmFrcgEGZZ0gcQbMYy+fDsC4w15J24BXkjHr2e8v+hA88fdfyBxd8ZY68gYzyJ65WaMCuMwuazd6QKlsfOIKBik6HIscfkxeDd6o9deICH3Mg02Exh1JgnbAVf8344ioiSMmYcnMRjagSAxYjNgg/vJdBoAzXwZL4w/iGaEHfPSnmNf/b7hvq3btA7rHnE+nvrxVGQqGd4jvDl/6Qiv5c+lc8Jl/tUtjuNP3vmEkvUdRH7pAkBPQB2vPfY668J9qfGqwfj5ThKBfkcvQq8+y7P3/sDy7YtQ9w2w8/xONuzbwJW0K6ieVnFuwjl8PdsZ8qiiVqVjzS9rUdmqMA7dCYQ0XG0gbHYYAKNfGI1EKsHa/RkoDuFygw/Ldi7HtNkEt1IEJDIJQx1DnHrhFCHTQ3CPc7/j87q8Y5FYTPif+IHWgkW4x7ojlUmx66hCZjZSHzkVhX6AuJOfon/mUSSS24TTv0kZf7dLk6XMEsB3lC8ALw+8DMCMjTPu+LtdC3eBuwcwgf/ZBEEEnv+9Zc96hcWLxX9v9N1I4ORAJr8/mda8Vsa9cltR5TvGl+WHl3O6rQ1qwaAy0OjTiNJdicRsIiDvABK5lKsPGSmL/5Fmz2Ymp83Fxkvci/rnH6T6cC43UpuQK+fh0uHCg9+vRjv3BjAei+TOawFgFsTx2asX1wPTj09HbpJzePZh0vf8xUDQTQRh4R3vKdpdRMVX15BNiUAQFMNgjUViQKdoQpCYEG6RsxuipvI38yqq/mPO2x4juigarUsvmlAHWoJGobV1B/qZbQ1j1KD6bgUGP2tcGmyRtirwb3sMwcHMPTbwi72Ki23WXNOaUBpuz1vOzbaMu5hOXl82/rX+mNbJ+eKpL4gpiMG/dTVPBX3BFI/r3N/yBF/VFWJlZYUgBaPCyFrPMlwNhaCfS+XQNGqa7LEfq0Lv4oVhUIdrzTW8yi+IyuzOCcM5fSBmUno1erF/wX485OF8Cxgzb+DYIpKfDV/nccr7d0713cXInGi0PUNUJS0i8fiHqG/eZtiu971BU8vD2MV6c2j2IQSJwKyU17jLWkakWwlbTySAUoOXFLq9Yuj2iiHq/Pe88+Cb2L07Eder75GYm4j5axM3JYfo0+QBcHwQxlmBOXw0hdKlWLtbIzVLh4HXs+m7iHIewVpEZfrxZ8QaRsSrEdxruZf0c+mMGyWO0dXnVnPmlTOUHy6nObuZxbsX/2M8/admNv/ztYbIaTSFpWP5D3nz/23/++2/YN9/2/9JMxvM1F+qJ3RRKO4zIyj9RQTmzj36I8HS93D2GcXp1hhiz35FQP5BHh/9MDKLif1HFpCZa+HYvDpaHH7Hq/cRFuwQswZ6LybRdzkBy+MG+p1FVZReLaqfDAL83i3HUWfC/moWsWdPD/fFaqCDaZKXUKj0fF7pizFKD4C1mzX1AT20ubUhYOH8mF8ojHTicesnCZkRQnMz/LxaBAPnOz32j3Ns++EA3oB7WCcl5slYLLZi3ln5RUxKDXrvLm6cTsVmWhMygxaVkzUGlQ1K/QC1UeNxtPuBTP9KVm5+gPZAFfZzXei3FTPoVA0mfmrR8EufFl57G2/TeBJz/0cHFhxHMPsSLe2ktdOVvsaJLPpzESWRJRRHF1NvAo31S3jUWRMRBgMtA7QXt9Ow1pvfZf8CwMlaXCTO+W4OUz6cwocOokxp7KWxfHz/KR42mtE4a1h2aBk/jf0JhZOS1cFdzPLRMaoREARkggSZSQZYUHo44RynR6dVId09B+VyA0aNuDDt8jHQkFLJmIYADnvvIzJyEhpzA/4FV+l1C8Ei+OF16QidDnLil/ayafVPzHd8lfzt+Rx78hjaRzr/JvAT1bqRESfKaIlfBOPjSHooieNPHcessEJn4/KPa6VXGBlYeJnkJisSvr6fPq86JEZfFNohFAYtXTNktKdtYVd8G2XKIc5bSQm89j4T6v3It3zBqpMTsDWmUxRZTNSFsbSl1ZCekQ4ZUFGUyfLRXjj6N7PRtxRXqwGU1kqSH05GsAjsWfE2LpEDOHt2cteP6SScTad7eTe7nzqMxWhBm96OXtnCFj87HM3ubGwWmXpVJ6uoOVtD9uZsQmaEDOf1eY/w5lX9q0gVUvbtkzBgZ2bn8p1MHxSZSlb97VikMnL21RIKxIwqoKXWA7WkBY2HHWZ1EJCD0tuVwMYThPuWsGoUrGzoB5Ua9UAntl11KM4J7L8HNsqNXIzNx9YnksnviczG/qZ+FDZpPDY+m9nOnzKv6SMmRkgxG83cf+V+bhxpZcMGuJFaQJtzAzFFMZh6tJgNt1cNo54dRcKaBJyC/6nGbAu3o0l9huqgaixOMvoa+yj4rYCgqUGk/rSYncd3ohnS0G/TjVw+hE1HDYJESkvwKHyNuTwx8kG6d49mxv5pDHh1kjGtn/QTNvQdFwFrrUsnzX93RXEboHSe7UViSy/TJ+TyZFsZusR25vssAsB/vD9D4VJa9ZUolNC5ejenO0fx7/B96rpUUtelYrFYaGtruwP8tA105sasVxGQMCpBZPJJZOJxfa8epwwta3MfRPrMbxwYdYZnNB/gO9qXBT8vwDPRkzFvByOYHPn5vp9xMIXzrPopEBgG4QECvLvZsOJb/ujcQvJDySQ/lAyAdK+UplA41iewIyqb19pFcP/Sx5c4++pZ7iadnMoQToyuZ4FSj7VC3GxazBai74mmt7aXsS+P/cd1Amg+0MyKHSvYu2YvwVPFedlwa16mspLQumuciFWx8K95tHVk4zFtDO0BqTi03kRm0pO6uIfuRleOHBmJbJSeqiTx97btqMGv4ABVMQn0uttxY84bJCZCkTQeQSIFujk2CL0mGYLNX0hD91Mx8Ca1B4dIvjpE01xAApOsm3lR58sXh1IYsmvFnJhMBcHo1AdRGBTMOjKLlsFiZKMnMmfzHNor+si/5ke6egmhTU5cvZ5KX9QQtoY+JtdPJjMtkwazE1kRfzGveCROW9ZgfKubXqscbgSLfY8ricKl25FfX/yUbbGFLFIF0m/+lQFpJ9dTtUw6OwnrvCgqHirn/W5oyh2LsiKf6ylbcGiOoqQ0FOm0n1iafC+9HxXwl+JbfH6+SUpSNPfNrMBskaFznMXa+k/RSxuZqjZhkd5psfI3K8DazZpfZ/xKS04LTovS+akwkN5Ga9ZUeOMwV7z3YgpiiBqYiVndSvLIQtwnX+HdBBdeFO5Br9JzbsI5Gr0badmxgsGBNu7Od6Q5oJ/u4NsMXofF8bSqVLi6lHPRt5QAizdeTQLtAakk3RfL3rP3UupzhbskMchSirjp0Y1Z8Ry5ST1UhFSgtyul3WcDQbJa/rXgBg/W2NBT08ON72/Q0dfB/OvzCa0IpSOnA1bCmVfFLKqIhRGUvXCKjMp0Mo6PIFmjxOl1J5RuSmTWMnyCfTiUByPkAkue/p0Txi3obLbQ7NlMaXgpEWURODp14eTRSfLk61g8HmLbo9uY9N4k3Nvc6U1o4oynFKXJjU3lLgz59vD2aDHb9p6ra+jwn4avHSxseJjUDZ40JRvRzAsjp8IWjx4l79XpeXHihzzRPIqbYy6S5LUCadetDbprMFKLCdVgFxazwx1M6Zbj5Ty9YyUJqTfZG1+NIIDCMIimrxW9xgHvRcs4b5OOX9gPxJoEBAFq4ufiXpWJxGLGuamQx/ZM4OiLn3NO24jgoKDJSwQCqyLdGHH/K3xiD9cO6ZDXlKLpMaLprMG6pxGL1MhOhSul3Uo8ricy6ewkuld1DmfErrSF150Bcx37y55E0f0wDv4O5E99Bk17NlFXD6LRatCZbrOpjVojZoMZeWgIZyVfoLXSohYEzGbwKT6JIJFg6koirOZXJsb+yN7BhZjMN7GQiH+tP2u2raGLTMCNk9VprBx1BosgIy0zjfj8eN5+9W1KR63GuqeRzMrZ9FVWkxQeRk18PFKzCeemArq8e8iOy8Pd+h48Tgi4WKkRFqrpc9SgtdLykB18727m6W4ffjLnMF1RQKwQMnwOfVYFSC1KLBaxyDj729lUnqzk0oeXAEjJSeR6dCECAm32jXfMk3+Uu+Hn3kPW93fjHCmjV7KVgA4fLkXOJKUui6MV6yiT7cEpKxphjIGSeDWlQUV0OXVhGJ/J3LHtXOzQM5iai9HpdpFKMDey5fF7qL5f4BnPHVQEeKO1jcbBEoJr33RazB/wZz8M2Mm4t96Rg18/hGF9O241ZfS5BFM/eRsTsu2IKdJhNk0GJMStiCNuRRynT8ORz8Gj4iKelZfQdUQx6aRINCtK8sC5rIEwTQTXlN14VblhjjEQXGZGYplH0ch4CkM3o2jqQ95ng02vHMFFnJN6bK6RNSKVsNnH2BnQgusjWzlV/wRnulppdRVzwRybRHa8taClcigIa3MfFjNcHdtCpWcmD1nBCjvI0XfxyZINyCxW7JU/Ij5/VAYMKnHsWf5WK75znqasJpzDnYk4bc9i7RQWPvEeyw6+i0Uw0eEyhEuHWARasG8SGQvP8/2fv9GhTSY7LAelVCQgurQM8uqGDcx+bgdvzXyFmbOl2Jb8Sa99L/2CDovTCvpcggnXeyNY7sPX3plC39vjoKDpLn40uOMdUkDWhO9RtEUxGD6Gy27P3/4jiYVEj+0ErGrCZvRdw7Z/AhYqQyr45NlPeD1CzOT6zOez22PBWsmgkx19CXUMuqWycf1Gxv4hPreNhf10uNnRIjXiLoMKwULK9VGcSz/HzWgT9+6egm2rlNP5DzGnbhUhi+CbmUsYcX0Ejt2OWBVF80V/BBfvK0QOdHlL+OXeX2hza8O7whbngQl4C5Bu3UhTexT9/Sb61KLFW9OAL2eHJhAw8zn8t68lqigYk3mIuYfmUpTszx9z1/D7st+B3/mUx7n65VVOv3R7DyccTedNv0Z6b0ST9q8BtiwSQfSkB5P4U/EjAPrl+/l0ykV2da9g9pHZw+/1afRBa6Xlt5W/4a0N5qakhAbrE+Qk5rD+67eQm+G4IoCwwBP0usjp81hATcICVAMdyA3ZjO5zxtmYy+Nuh7nR5ojRuJrcGa9SZbeClCtyril6mGroxKbxNAazM5pwH1wbcsmOOo5eLdraLpi1CoD2onbKj5ZjLUB8jgvjjSto9+yie8gPQ/fgMNAHUB5wmlEJX3HMD+5v6KJcf5hq11IcByZgvVvcJ/5W6Yzt6XHo1YNkx+djlpnQSBwpK/+Vgd6bLDc4M+t6HGpvA1MuiITPfk0xX8w4zid9WuY7hqJ5+3GinOV0TPbjUuxx6s3gWgVjlKnMtivmazvYbm5Hkn0D7/IebLrqqIueQYd/EvZ+clryWtictJlJ705ioKGPwSY3MvZO4JG9E+iMa6e7tB5V3U4KkrQk3ZXG+ZzzOApSvo4q4aexfyEtUzIhYwLubY00JjtgY/AjpyMKT4MGx0BHVp1ZxS+TfqH+Sj0nX9hClF0nJ5OuUxPcxobRb+IS4ULF0QokFqi+Hk3v4rMsnplF5cgsbP2+wEYmqm1yzbdtkX3syhAEUeV+9eX9BOTdHD5WcbTiDrAv96dc+k9ns2PZDuJyYpFqLiAI86mNm4trnejkoFOLc0399LMciJazApAbhpAZTbh0uLB011JOTD3BpenhhAdFggz6XAK5MfNlko++h1stKPUv4tbiBnOH6PKJp8tHtJasNSr4tgfCbZYRV/cdzkFglablg399MNxHi9TA9Zw/kXeZsRgW0rL6Eba73rmv+vvZXTFiGRKzifDMn7HpbsB70Ja8cZfp0Hgy3TqBbsduLo2+xBy3p3B1Oonas5FN6X8QL1vKiDpoDUxDNdRNxVvHcQI0o//AybmIWt2fWKQG1HoV/sXnkURIsRdkBFUG43t2HDfn9OFTKmYzZo9PJun8DcZlh+Mc3cjZXfegv0vLwWUd1DhtYdFfi4goi+DaKwZ+0qk4/OsSLGs6aXHvYMCmjxn7XXBtuYseJ1Gt01XRxUCLeH2PrN1DrExGu7cXHnX1GBY+RXloJZ5NLjR6NxIXuYiSAQXdim7i7M8zYEzEIrUgESS4N4i528+kneYJDwM1k87g0vMNSQVJ1PrXUhuiILTIjHuPIyn37yOrX4nTlSUM2Xtikfbg3iYCWdbN5/lz1E3+kgnIPGZh39mEj1l0U8mNz8VVMYUO1w5klkEM5tsAzr2/3EtlcCXpL4r7bLlKzrSPp2ExWbB9cx/ZRcncN7oAtVQcb+YPzEgDO+lz8MeuZ5BfJp5k5ukXGDvaF6fQ29fftrsJG7kDZx2sufTo72RETcb5J9HPcFTmKOruq8MjRcnOpE5I+oXnnNYTOiN0+P1vCG/QUdbB5Y8vc/L5k8iUMoKnBnP8qePYetvirbfC5N/C9Cd38litmmdlt8nW/uP8uffUvVSeqBxWBP6jCfyNson/FQRkt34Xx+YSuj0jMWgcb/2ZwNh/jeXi++Lzz6C2ZeP6V9FqtPi3rfv/SmVUc7YGhyQJ3OpOyLWdDDp4ARMo/eQQwa06SnydkFmsMUt0SAUVgiDu1Z1CnLDxtMFvrB/+4/1RWN125Kg4VsGFdy8gPCCuDyUWCTKzDKPeiG/xCVTaHvr8YhjdJeVLPxWnjA/TJTw73GfX+lwALqX9SXTrw/Tb9nMuPYP5IatR5PeRdqYVUuIwpubjLIWOnAis7cVCRtnWMhLLEgmuDEalV3Fw7kFafb2QuYrjzqX2BtYDSiCW1MdSMQRFcOOE1a3fG2y66tBLSjgydQZn4uDm/aJYotf9tgLMVheNX+sSZh+JoPNIPrY5dw2rIgXJTVJVsM4BXqn3ocNKoOgBIyd1n2E3lMDe3m/QhUDu+Qgufn8X8pf7+fHBHxl1eRTTT4gxGqHlQSzz7yQrK3XYyrUwthCjk5YVQ9exqYZIlTPj332ejjgd9q+Gct3lMN+GFULQagh9hC+aHkSaV0qM8CXN0fM5OGUVr78t1jvda66h701D46wZjobwHe3L6uuree+V9xC45QYU6E9jWDqOLaW4eDghMdvS5f4qx5bJuS/CC7NMzOxzVfZy8JGLeHleYF59JP1Zl1HcnU6reyv3b70fk76FL3o+oyLoAlfWPIy3dgZr6mrwLq0EQUDTJ8aW6FplVI66Sk34TdarXiV2N0T1PUTHs5tZZQe1vs9yvWcBelsXLCYjr7/9OnlxeexdKCoI2/S31JAGM5kbMwmZGYK1gzXxZ+M5Oe0k4yeOB8AhwIHpG6fzibtYYxQswh1K2b/b/7yH/lNmn1LXi3fZWdp9k4D/zyrB/39pEokEDw+P/yh4+N9o/wX7/tv+T5phwMDhR8Tw38E+Cx3nU6i3NiD1KEZAQKpSoNc4YpYpkJmNHPpuDtGr99Hw/DcERmykp1VPm1sbvh0yQKApZBwZdTkktTjiaLIwZOdJ/uSn8Cg/y4YNGwhccpQXZh6lddpR3ry59HZHpBL8Cw4j6axkYrcX/Vvno/PvomBHAXK1nCNLimi2KkUuldHs1UyzVzPSW7k5FfsKWbfpKRp8KklZkP6Pc5SFu2PdfhOv5l3Is/LR9T5Pp2/iLQ/yG5yZ0IIyOouq7xOJqfsI+XtvUDjxcaQmI2alFe3SiaicXyB8wMSAzkjYKXcCTt4LgF2mnhcyX+D4EzeYblyOxT6JzvyDDNp7YUhJ44LLs1R5lPCCKYTFr2zm1RYPtN1PEFsYS6dzJ8XRoi1VqOF7xmZ0YOhKpNIxjH337UN5rztyPznhVfOQdsjgVmSfYBaw97OnRVnKE6sOsu+D1Yy02MFDIjP/DeENvlu+jdY3n8Tv46/Y8Mlaulz19Psd5bWM1/h8/Vdc2P0hVhzHZ2QT9ut/psbtKdQ5Zzk5xUhtwiBr96dh1RhIyxsfsCuokjj71Ti2lODYUsLl94PpjuomMqaQY0OROEvzEQQBrxFejHtlHJdMe2gwwhDe9CsdcA9pxmBjpvpMNYU7C+n2iKDPOZAOv8R/XCuzwsIz809grvHmcYmeBLfH0J4/QvxpkT3DBXjjtxpMBjmb6jVEu1iGWZP+vwwAzoQk3MSSns3Z0ErWxk7l/LjzjL8wnsGsLsKyHmLC8mNUSI+j8FCj69HRWd7JjyPEQoTZx5kzQ6+S7/sb/WMv8rjjszgEONxS64lPVrlEQkedG1atQ4ATK4+v5PTLp7nyyRXqL9dTk1FD3Io4emp6yN6STfi8cJp6BzFZrqMwKDAJ4uK4ImUJgbn7hpWvEROzyDfeQ3XLMgo6cjiTsoI2q0SmjZ6N8bODuPQYWHV0DIP2MVjphygbdR8xGZtw2W/Hm/s30PjyF1Qv34Of+nk6yzv5PuF7jENGxgLFnhsQ1Icw9D1LSU4Y2q57RcuPW887j0YnDs3cw/XU62yNeACZSsaUj6ZQd74Ofb+YO/d3YPK/N+uIDeS6iNk/RrORkBkhxL42n6xfStDpdCy7vIzsxGx+eOgHnvKahv2ZfOw6a7DrrOHSuR6OpsGHwZ3sSSxCYjuFG/GH8axJIPxmOL8vPUZ5yHWeKYpHVxiGKdnAG4JoKfWB38doCGX2inMcGpRTYWH4+rhFu9EzVs4Z4QwHY1tIUsOI/qH/OP9JpVI8PDz+x6sShFvynb8XT+lvpJP+RjoAJ6e9j2rQmkiFBGujuDk/9eIpJFIJ++7bRzx+dDhr0Kv16E16vFO9WXtpLVvHiIq//tBKNqaXsbvoZZodRdCpvaQduVpOdVA2I0LeZaVnAed2TCOk3RVeBPdYd6zdrRlsHSS4bjzvPvwciuTjfGb3HAANVxoo2lVEyMwQSveU4hrpilPInUWEdrd2Mkdm0qi5XdyuSVhAY8Rk4s99ia3ZzNzLJUQXfIfBowPLFECwUJGyBLuOKmy//p3M0WnUZCQjDRAX+FHnv0Nr686+5f2UeU/Au+Ne4uq+wywoMapvZQNKOrmog8tDCvKdr2L/xqs0Bgu02xWRnJ08bGksk0BrmwO2N+LRT+1DcI6hz80Vs9yCzCwjMTeRfs8WpHIpyQ8mk7enArc/sug6N4oslYGSRWNJjU3DvPcQkZfGURrViFNbP2YrX551qMVuxlGWu80k8upF7KrHcHjOYTonXuQpRynvBsK3ndZcqpWxYscKTk0+RWZaJhnjM/jTqxnNhRgkDc64ae1wy0inMKaQblcnHNobKCwI4tuwSGrM11joJGXv2kwM9lfZ0Q81ubtZOT6YsLIA5u9Zg+mFIb5d9wOCVJwD7tl1Dx7t/rABzEYzzqHOeKV4kT/Sgb3nOlivO4SsOIChtkBmH5pHalYS0EXThweZ7Z0vWrF4WWPq0vLcp+JYKIwupN+mn+up5QTGNFFFMDHuW9nh9Sp6WTmaq7+xM8QKt5RG3pTC1+1t2GRux66zGlPMw4RMzOEHtyYyhALWTC/m085ydHI9++Z9DsBOd1ga8ibvyN5gUi0MaJ0ImRHC8x3P87HLx4QSyq8rfiXrDTGbsPpUNc5hzlx49wLa0g7alx2kflCFURKPjbsN/2oVN3gZWzN4auNTXB2bicnrDdS+cSi0Cubvn09EWQThX64jrXUkdk59VM0qZNA5D0m9BIVR3MSrah3ZsuEB7KddRHJiLLqQ3uFxPtgywOFBcYZwUOu5kXQDK/8olAFedPh7EZ2xCcMpNUMpGiYHV7PJppk4/zUwFIbB0Qf79krs20XF8FDHs3conGu+ysK+OQSFjZn+hcexWKAjJA11t6j+0MUvw1NSz6saA2+W+WP2MyKxWHBpyKMhYjK9bvuIkBs56wNPt4PJYqTXvpejM47ipF+H7vWnODouh6NZ4dj3tWKYuYLBIQkK/QC5SZWUBJwgwjQSXVgVGUoja92ewe0DN9IH07kw4xwWAaQSKMuLRGiuwWxwBWT0uPvxzivvYJFaGKF6CICmG00ETw8mcU0ixUfcOG/sYcYxR3yaQrG8Cbad1ciMgwzl29F52sAHATFsW/kTQ/1xPHAznDXbRGWITqVDZ2Uhx3UKNZ9KMYWcIislC6VByeTTk2nyKKUjsoTAjhimHSiGCS7U3I4SxanRlZziQEJXjiI7OQqlWoZrgCvXJkdQE1jDvFs//3uaKvIrXJHKjehKqvG8WUdjUAjno+OYfmw6dVuegTnTaMhs4MQzJ0hYnUDeoTOsTC9E+s2jtEfKMT4gJ3NkJmlX0wCI27WIx7X7mVnmj9JZwoF5+1HqpvLIt/P4fc3zVHn9jEHeAQmneMr1MH84vjTc7/c8jnGpsBAfZ2dafQQS/F2pjnfAubEAk7QTqUVCVUcB08fV4PpsBce0d2O98VcCHKVkhsPiFnjOPpkCqcBQQhcxXn78MVdDr3UR7fYnyEoJoMvdhydvbVTrr9SjtFYyaLKi2Ocd8r1riAh6gkmBblQG3sAk7eDo1G948cPH2FEUQE3MReZmptG7tJufH6wGqYzoxg1E3rhCl0s77xSH41Pvh81agZnZM7k8+jImuZHRVuAtEdgz9CrNdmkUef9Gr7WovPptxV76bVqZ6fAUppgx1HiPYZErmKWiBe+DrbD5yu9sGmmNVCoAZpoPFvD6569zevJpkrKTKA8tRz5RJPeNfWks7nHupD6WSt7OfGpKAgAot/8OmeoRTsyqZdZ+Txx6HdCOy8JOBu6SJmpNGqT9egpiCzg78ezwfHjJrONwqjX9g48w9nwCYeUddLj0Y1e3A6V/Ha/N/4mBflveudkB3C4IlnWmUdaZxqu9D3F2zmXmlvfQ25I2fHyuNdi7voub1RVGRDTwTmc4keeDCb9cSsNMe1TmQDz7l+JrI4IB/uP8sfG0Yexr6Xzh8ymdV+1pmfcYWqMNQ446OmIKiC0Ulfwdwat5trMP9x1jmH23FaMzJ2GSttMS2Mfle+opNx/mYFsCGRk6jMnzsZJZYZKZ0Kl1/LzqZ0Y0nGbPpSyyLsZx+D4zVT5iDplTQxkJ3ScYnVIDPrWkfvURP5q34/BnC3FyP1ZGy7h8cyF+cgcWa22hOBrlIxpujlyJ2cYFWMO/N4/422unrgeV/MnXBFUEMSUrAfpuV57S30wn2yubN0niww0bkEsEpLZqKoIrCKkMQaqSY9GbeG7tcbb5wahSLS3+nVwJuIJKr6IhNAKF4EWyxx+8/9oO3m6PpvwsKId6sOuoRiE3ob6SSsy6iwydHI9XQTCCKEhlSJGNZ/N8rAetqav/hIhcK7RRTdgv/I6Tsx7mWtLbw/003Vrw3fXLXYx6ZhTfJ4qgXm9hKH6bNpDRGczdagl1YdH43RRV5ROPVuAU2sHu3RPQmNwY0B9l8RlXji48Svdj9lyu3oHryHzco8uw9viIzvK7KfJ/grFtK5jySygQxcevv8WN5BvMbvgMl06RBNno3cgjXzzJ2fSzyAPktLq3IndwJsRhLdr6teTf8hsVEJC3qDn60suoZw0ivXEd9/Z2TEoNUpMB5AokUtA4a/BI8KAlp4WRH0zg3YQldJdH0pqQxyilAhqbic1341rqbqRN8VyuSyIyroo6ux5qZQKx2Qrc29wxy8D7x6kcAHY+XsVL70UwoBvgl0m/AJC4NpGedx6gNryVZ9dvYnWvCO40XGlA7WZLofc4IoST2DgM8F3JA8jjVqKSqXi4/w3Ml1I4PeZJ1rXBt25gl3mF3AMfs3jxc9Ttz2XINZi6mFl88kY/joEOd4xFrxQvJEo5kSWRxBbGUhvahcUCqkFRidvhk8Bpr80saYZMnaiOkcikFEx+ihrXTejUOrITs2n0aia2cym2DjA4CEikCDIpOislaq0Bp04n/Cu70ev1gIY8//vptc7GKOvj0XZ4w7Ic96oryNQ+mCx35isCXD2wlZHXRiKM1VNndWrYxh4grCwMbXYdQyOHAA2avhZsusXojAWrj/JyWiFzK1Ox6lATURrBmMtj6OY6F9JfwcnpNf70u8yO7m5sCy7jVJKL+ta5Azxab0/t4ZWUTe/np7U/sWHDBtRDPZAdwvdLdvF7WgON44ooyRXnuIaISfQ6C+ROP8lVdx3uzmG0Lj+Ch99TmAYtGJVGIspEQsk6V3vOKfvoXboP64AJ/OV9EM2AhZTPU8SxHC2C6D5pPjyc8zB2PnZ87PoxMrMZVV8+7T4jkCgV1PnX4dg9A6VBicTPzPM/Pk/S0mMYpc+TW/Miz3z+GprB2xVujex33opeyNc9kF2sZ97BeexdsJdRZ/uQOfRx9IFzSCXwTbeUE9E68qKv0mZ/mCPrFWyTKmj2qSFb3UX1pAu4uWtIOl+Nn/OYYfvRuZJ1rLKSUaKV0HyukgV7F1AeWk5wVTB6lX4YdGi81khneSfGISPWRSZIzuevrbOZpJHDZ7B70e5bmdqDHL//Mp8ltJJtW8v0j54BwOHTV2l4+yd0Ni64NORReDmWxStOkDjtT6aa5tAl6cKuz45Et0S6nDppNUGGFiyO/yxYu4S7ELU4iqJdRcNOUP3N/VSfqSYfMQ/37if+IDshD5n8TpuKoMlBBE0O+sdnArhVX8Wmq47q6Y/gHie+r6++b/i4TU8DVUmLaA0ejcIWjIOGYaCvdNRqDGo7tJpn0QxqMAglCAJ8Gfwlul4d2k4tdj52zP9pPnK1HL+xIoj/YveL3LgBh8TLgW1XDRaZ+IxuPpyNn1s4EwsqOJHoRLPTbpYeK0NfYwt4ct/Z+wD41PNThjqHeKr2qTvy6oHh6+fR4sHDmx+mwlzBgGMkMqMO57oC+An+kKwk+YEqusrOMrhoBGA9/H6VXoXMKGfQZpCM9AwWu/ojzenAvdGAU5ATjzvDXdbw7PszkNtL6WvoQ/exjtEuo4ddZSwyCz+v+IhFTWUIAniVn0dosQVicQl3wanNBeG0WIsQBPC6mYFVbz17p4p96NP3A/8kYidW/QB8+o/XwcIH3bDeAaI//5AvOtdiK4jZjn2aXKQC3N0MEn0ATiGjmB8eD9VgMyAuul2nexDqtJ/o9NP8Me4sobp9rP5pNR4tHlyabKbMG9oEKQ86rqUi5BShST6McJlKZfloSv0fIsItHQCrXjnjTk+hy8VMqdcmLFJx/dfnEkh1wl3Y+1nf0evEBxLZbLeZsLIw5gvimlsS4E9zmD/NYRMIvbeRH/u86dK589H5o8ikMuzbb4q1i/pcUnyUfDdOz4f7U9BlyzGajfTZ9XEj6QbxbZMIztuNa4OMoBFQYRpCWl6GZ0Xm8PffTGskyv49ChJaeKLRgFQuw6CxoLUMYCMBKwko9HWsT1nEF9c3gDCFylAtDT4NOHU6sfiPxQjTVLAWpAqRFdZd2U3NuRrmaOeQnZQ9TPQo/L2Q3rpe1A5qUh9P/Y9AH/wT3IM7AUBBYJiYJNp4/hfs+881v/+99l+w77/t/6QprBXM/n42J545gdrOiuIEe370LCPg9FhsAs1oHbsocbmb5vmhRBUFYtNdx2ZFG03W7bynkdBhvRuVTkVe2Ns8uO43Hghew4sGE5+m7+dtYQpxpzfSGjQKmelWPk+LB+pYkALSpHAqitQE5h0YpvXeI9Xwl/9Nwu85RqfPWM68egYrRyuiU5/DrWUZCVNuZ3yYLRY2xWyivagddxxwb0tGCO3/xzka6ttw8e8kLKmMq+gxScS+6DWOWPc20+OgpdYrG4PUTLfDEAtuheQKUjmBOXtR6PpIHXeZJZsukVI4j3EDMbSfLLrjO351zUaiK+Bc78sM9YkT82BgLBpjNN7tjjTIRhFaux0bg4btkU689dpbw+qOJx0gleco8lxDq40J75HezPpmFp9X5+HY7cjinXEUe2UTlJBO7YVabnx3g3v+uocJx2fzh6SJVKteXKxtGWwb5NhTxwibG4ZFJ7KI3jyTwNj4euTKiZR5dpEfm4/z0Gxsd+iABXgG/cSDqdU80i1lRGYIPo0+dOR00BjTi8fkk5zygY3dMKRR0RA+CZ+yMzQdr8QRRzyeaMZZYs2O7KkUBM7D90lffEf58vqHu1jeAmPs5vCrKp+Ge37greBZBOVoabzaiCPg2FJKt1c0/15cARE4+WMA1oc0Uj2nBStfFy57/oRLmh6bARtiC2N57atXKPU6TsyeORjf7GfHqiye/Gzm8GdcsH2CbyN2sMeniyFbORfGXSAnMYd3bL+l/d1S/sz3I2HkVToHQynY0c6Rx46I362QEGy5SNLgJaJie8kIqkXjZsOCbQsAeOiVdwD4pPNTrL9uwca0C75/FplCRt35umEwpuJIBXEr4mgvaefiexexmCz09g3y1HdPcXXEVeycxQWyRLAMW6W0uXaT3jtIh+vHOHtlMD3vLebtTydjQgbdxy+DSs/+CZcJfvUpJIkJ9Jou4VNwmZIxa4m8JIJIESqBzX6wqn2AnC05GIeM2Pvb01vbS3Dun8wZvYe8zJfpbS1HaaNk6idT6bY4sG2fieNpYxGkAgICFkFAKpMy5vkxIMCpF0+R/FAyKnvVHcHWIC4mPJtceebXNzDfU0vjRCmnf27Are42A/dmqHiOZsFMX8Ro+tTuWPc00utygaOD8ExUGQP+ZSh9XkaaBefSzzHodjf1ftswy81M7nYnMz8as1YESIr/KsZhijvHsy/zx4kR7App4PeMUQwF9MGt6cFri4FRkaP4KKqemJJIlE13bmCyf8zm4IMHWfzXYlTxKgICAoaDyo2DBmy6WvC8mYE5Rw+vPnDHe5unKIhd8DkNdQE4FkdjfMSItlt7h9rn6ye+xqnTCXuDuED2He3L4j8Xc7Woju+MH7K5S03FxZE4hbbTWW5iU9QmPBI9aFlcxP2+uQBcu+mN4pbla/i8cMLnhTN3LsS5nyHapo12E7fs3MDK2YrY5bEMdQyRuy2XpAeT/gH2GcINXB15lbTMNGoyagiYEIDMqEU11I1i3f1ov/0ej6JPOLpkPK8um0nrmXxSDouMtHafGAS9wG5PA/1vfoi7+wdYb29E09eKpq8V12Ypiq4YKkL+oNThInbd76IanI5FKmfoVmaNRJBy0hTNUEQ5fc4RBN0Xz+sTb9sW+ygGSRpdyJNeJTwa/QDCLRcvQWJGp9bx3r/e46dRt+04jqz5E7d+cUxMvCsDg30yCoU1upRYvkx6lAEbM6M3VwL3k/bCT8xIu8oKRzVSfTGO3SKLPGjPHHbZy1imAVPGEnzGxbNv/ms0ezaTnneBspB7aHVv5ac9CwktD0X2r0Q2hT1At2M39b4VmAzllChfYFBrS0+PI4KDmRavDjw03Sy3he12Jcj1tbzg40BulIEI/wRmH72lZJEYcNIeR60W54Nrm25QtLuIpcfv45viL7kRspEx8R44B07BxseR4MrbDN76I4k8ZPbgqbsbCW79HHPwbdax1kqLXqUnyesqb4X0MaVCg2bQmbScdahbt2LXl8gepFj5b8Xy1kvYJzbQ4+yPXWc1vZ/8gLP7HHYNKLH+Us9dTVCllzKN27uK00NiHbff3p6YD5+gNRRkb8uGLVeaPZoxxBuwtxMtbx7OfRgE2BS9CYCyBhfqEgKYbP8cnUOdHK04ipXJisL7C3HAgSVyFdt63In37ibmcAxhJaIVUdn6byljNfMe2se80bl8rW9Cm6/FqdsJo52RLkcDdWF55AQV4HJ3M28nL+TphqfZ6LORY0v2EK3bgO2oArY+vg3d0zC/zn94Y9QamEZE6z6srLUIQ/asthPvLUEQi4SCRIZcP4DW1u0fpIfxP67i1MKNjJ53ntVa+fAGzKZbzGvo/GonLnaD/NUzCtuTozG59WORK6lMWoxRbcO+1BL2uN7A8OsUbC/FY1hSy9Nbnmb//P3ItHYYuxw4ULaGXSsfQaP3Z5nxQ/pcgwjO/oukK5B+bB02b57g2fENPBTegMrZCturtoSoQ/hRf46UzMfZvmQutUXFuFUcZ7AtCqeGGqw7i4g5u55tq7chuIk/RHtRO5mfZRKQHoAguAESdGoderUJiwUOLLjEkl/DaW8q4df7DtPgUoDODLYIWNSi/Wt2UjZFMUXEm4MJaNPgfciKGvsm6sfUY3vclsjSSLTWFwl46FcmxxgoHUymxHMqvheP4twk3t/dDv3kJObwpOE4Dz4wkw8yD2I2T0OQiOuaS1qYbQ3dTa6M//xhWiZ1oVNV433zAtXB4v01KnMU7VzBMJiO0lbJxLcnEjw9mN8bvuTBgGa8ZRLaBAFvdTg5iTnDYB+AWqfmvVfeY7bjc0japPTai8Cxp20De0I6+KoHvu3lf5hIwehLo3HtzEDltBa1UY1EgC6vGCQWMybzFQDsBYHU+kRqjNd52s6CtLEGi2338GdIkGDv/D03fWBmJGT2eaJXiszimsAaagI/R2AjAFtHi89+ZYAjDrGnyU7Opse+HJkqhxspN2j0bmTpTtGa0HrAxMd3XefGqJu0Wk+k2vtnJBY5qcXvII+9yRvTs3m+0IVrXp0E1o9g5LWR5MXnEVnkyqkd9/L7mgIGNRdxVNQiEW4/64acYmlxzkMj1PP1DAnfNYciCDcZfUaKr/c4Loy/gEGQoDR30mLtzK+NtkhdlJSFl5GSlYJjjyPOV52hoh/eh4D0AALSA8jdlovUIqG/247Vr68k0F7MwLkZtIubT8NjoR+zvfQSBY3QsfIXigZ9WZS3gpEHVrNl7RayR3vSZbWdBX6NXOn05M0uA07tWiwSC38tvsHI66NZHl5AT7sDXz3zJMqIA8TWNtDun0xLyFgK/B4HoEXuzsZuaDIqcGkYIqw3jJvhN/nBDartD/Ovqlk4HdPhNKIRQ7cH6oFOAtofxa9zHUpdH8YUHaBm7Mtj2ZK2BWtPO0oTc6g0Ggh590n0k/twnjmFL73u5a+7/+LTXzcRXryLUNtUZOfLcBvhxK6JDlR4foVCMh7X647YBLjR5SLHaSzoVHIW7VqE3CzHr86PlBspDE3PptGmF613M4L0NjEh7MYxtICmNYcV8lRKTcd53NtCyoU+DC5TuXfSFsyui2k0nOb1pRl8EzGJ8Akin5qPAAEAAElEQVSj+e3E7XX62Atj0alFIDdkRgirzqxC36vnk+4DvKZrQh/gwhnFv7grUUG9xyUGWgb4PuG2Eq5B4c7kUi+kk6T8eu+v2PU6stP/KvXHfyLKvZfPHn+WZBc9JcsF3NrcWPfdOgZscxn7xHOE+NaRrYcesxy7tgrCrv0GQEVcP9LJp9FLZrA3fjca/5uMML6FTrcRiaDHavoFXgtvZ0CiIO3FrezW+mPX1gFSB8xyM3KjnJlHZ9Jn6hxeQ1q7W9McPAa97CApozK4pAWjoKVT1s7W5YuRmqU89NNGPBog7MJoLuYHYmUNjeN7KIgpoMz3NGFRL3Ld9TpnvCAusIHX++TD2WfKQXG9GZZYxjv23pypvQeFQwBb13xKr30vru2uCBIBB+dexiovEffcH7zSNhah04zELGAvWDHFVkuMNJbizrEo3QvxcI+laUEIB2RvMyLvAaIKitA6e2PU+mLvZ0/w9GAufXiJ8GdTKDFAiX8J+Jdwr91P3BiRz9XEjWittAiv9nGicx7qh/bh7tdKRZ+RqGIR2ClI6CXhhj1Dtm78Fv8Wl/N+4Url7eyhlHUpVB3NIcjrFFEq8JaZyPslj966XqJfmsvFM6FEJf5Op9mPyut3E9koQz9RT+Wm48R393N6jLjG2NsTwuUhsOjbAZhw7CU2fiHFIlPgM8YJ+f+omvmP90cwmEjMTeT9l94nqVq0SbNvFfciEsGCuXcMOeo/cC0OJ6LOB/McCyDFKOvBtd2VpJwkKoOqkJiNCCYpssFB/PPOobd25vgCK3LCX8a3dSl9/j8RaafCveoKoysG+WNxPsscLPzqATfr1rOz+B4GrMegaw8hvDScRu9GBmwHQJCQk5hDk/cAi9SvYG7IxVXvSrubeI6+9b4YL7aj69URlfEzmv42AEpjrfjcu5vXTaCUmBkobmLh3oVkJ2aTlJNEzLlvaJmzmaCsRLyM1sh7O7Hqbx/+bWavPUiv1ERThS+MMYAdVAdUYzPoTptbBx12vZxV9tPn2k+/mz05017ArqMS5+YycsKaaPeQ8mtEFj9ehChnKKjOpt/iwpVFe3nQ5IrG6m7e6ZbTG5HLyy4aEi/tJDtwDiemnmBIo0XnagW8gMpWhUeCWGh1S/CkLbeZA/MOgLWE6XbLQZBwacwlZh+ZjbZKR8D4G0jaXCg4k4IQrKPLRY5m0MiQgxGlJojk7CJe9Agit9YV+wQJ21dup82tjbv2WQBHLvXJ+TR7PR1DDhjcM6i1F9egRWr4lwYchGgGC67R0m5iQXIQhyLHMaAJIGboD3ytjDhYmpi5azFBuaH0PtpOQl4Cbm2izXNZeBnh17tgPmR9n0Xu1lxilsbQtkjFHpfLTD21ACuLqNQpi7id3zZ9y2jMV9355F9fkvT9BbIfzkahllE69gGkJgN+8QPEpImW1iGOoQiCwOE5IiF/qstU/D9LItGjiciSSJJGK/h83ee4x7kz94e5HH/qOIW/F2LtZs1LfS8Nq1cCJgbQUXI7V3NbZgiL10fdsYDpquzi5sGbhMwIwSXin65L1j2NODUX0ybcNfyaUXunXaV71RXqY2YiCCCV314j6Gyc6bFr5J5d9xBVEkWrey/CgxA4JRCFlYL6S/WkPZ3GvtX7sPe15/4r9w+/99/Bi5wZ/xr+9+h9L/D1JglS4XYfwq79QY82EN5YPvza30rSq19cZcoH4n7zb0eETQeuQg4MaYboTAa/RD+6CuPo8orB5O2H+/XDIEjoKFMgOX+eofZowJqcyY/QopjPw5sfBvZyLl4EDi0WAb2NC3+usaHE9xxWvXB+UElT9ArGj7Pm2jeiYvbk1JPcDL9dIxlSV9CiPocgTKI8dSkubv+WB8vfSkrxWjaHjEUnFcnvyVnJNHvUQbA/EZe2YJarKB+5kmaHPejlDfh4epM60ZfuoiZCrp2jLXAk7bYCOgFy9dByq4Ygq+0lQBtATWANFuCvAfC2tSUxYho2ARC3L44xl8dQG9iLfPQrfHZ9Ocea7QADmwf8MdUGADD5CJSMAjUCMzVfEv/mJ+wxV9NX0oCiuIG9gd/wrzkFMFjHpNBCwl7ZzzmnB/A9aSS0fQ5fP/Y1Y2/WYFJZ83c05N8EbwBJ21VWb1+BobxpeGxYdzei6W0iu/UG423b8Dbrman5CEP/R3jdzEDT10zPpLvw/AvePBXBkWe+In+qjJ8uz2Ht1rXY99ljsRXHiJOmgbU10Rxtmkf/yJHo84pR6fowyeCXx3+gxmjHgbL1WNWJoHnhHC1XZTugH5LVsL7tD66fm0rQ0WsMfDSK5tjHuB73Ic4dzqj0KvQm8Rq2t0voiR4NRZeHz231X6tR1CpgAhx5/AjaTi0p61KY9PYk/t82QYCmkHFIBAstIWP/m9kHmM1mampq7qj5/W+2/4J9/23/J02ukuMzMRi1l4bu4naiSnqZHBJF8+mRNI8VKGvbw70HJnBwzkHKoyPIjPsWuVGO9YA1Jr2ZB39YhdZKy+aHN2PQqyksSSU2+hzT5CDIJOg1TviUnqbNP4WsOW9wHRNnh0I5UedES6yFHvdwujyjcWouIichB4PGxF1mDZ0xmbznruKu7XchlUu5tMkHdRf42xqYdHoSyTeS6V82gFu0G66pAWT9kY9mUI/n1NH/OMfuubYUKa7zyIQStkTmIlHLCMzZi3VvM3ore5ZtT2bcqmZ67rrKvTVtPFPXRfTZn7EavM1acs5S8YguCZV2BLIEP2pCVARU6GlcqyHV9luOyJzo/GARBEooHfsATg352FSWMq3QC41Ege882N/gje2QOxbBMszms9FG4adaRpb5Fz68bytuEmv+Fb4Ml3AX9O8WMCAd4PTki0wbtQyA619fp2h3EUMdQyT4zaHAEEdz2LN8820cht5uCncWUrhTtOTJjc+l0LmfGtkpYmOh1/Vj9oTtwadjFb3zlGhO9vJIt4n8Gphob+HYjGM8sOUBkVma4cLOgIukZAfQeSkO5ex+Bh396fBJYMQCCTmFX1MjmNFow7mybwx2UzuHfyslRv7yApV0G1e7I2kwiuzT1HWpnHzupHhNPCLxrLgA3OnRPtQ2QN0rT3LfyGy04d4IAnT7GLjhfxzrAWtiC2ORZyqQJMylJKaPad5+WAZM5MbnkpCXQON9/6K5U8n7NxfQoHVmUqg1CqOCAZsBlBOded/8PpFqC18WN3M1bzKeT95eVJocpGSdGkHC+FwOBveyvNlyB3PmbxbYXXYvM7TIjoNtojK1taCVCRsm4JnkiVlvRmkjemPb+4rFbn2fHsHLmkujLxFZEolDVg8A6oEOqhLuwqG1DLfmYh784UGOLdmNYHbAHGRBrVNzzx/3YLbrozSwiv2DcpbFzyc42Ql9nZ6B+loUugGk9yZzobWPsJan+GjfADGJ46krq8PGw4anap7iOffNdDaLaq7Qp7ezxzAWpbWS0c+O5tK1QS42rqLfrh+bfhucupww+usQBIGmrCYCJgYQvSSaH1J+4IFrD+Cd6n3H9bK7sodUcxAlkSWEuAVTU6KlxyOC9oBUos9/R59zAEMO3oy5aIckeRCTgws2/VdQDXbRE2TAqIUJInGV3Qis+3o+OrWOjJklrLoeRZy9D79POcHlMSd413rxsKUDQAIJXHeo4Qu9DLuMkegXiCzGrO9ERVFUSRQftWqwPjGKUV1ud/S7NkMMNsnZkkP0u9F3HOur6SLispgNKPiIctrqs9WceuEUE9+eSIRiBo86PM+BP2KJP5eMYchA2lNpHF53mKUHljLm4loAFu5ZiGOfO90vdHPwvQIi5oaiCu7B85NpfBKfz6LrWXTU5nIzSqTo+aT5YH21hs9+WcWDj++l8tGfONwfwcZbwe5/N53JmiudfvxwwhtNrBQWgWukKxo3DQU7CgibG4b3yDuvE4j2Bbb9toy8NpLWvFYCJgTgXn0Vr/LzlGkX0Diuln6rYvTqGCRyOVL72yxI14ZCBux6+W7mccwCPGsQ8Kw4P3x8yiELcDdb1m7hvl/uo3l8N26XvmPAyZ/ccaN4wh5W2er4zuLNlkW/kKZ/nQSVHJ2VbvgzSnuW8xvP4Jk3CockT2THj5CUm03dcilIwL1vJd52YRgGDbxv8z4AA84+ZDz3AGqZis4MZ5wDdZg10KUSGc3tfkm41mWja3ZnbHQtFsHCd+u+G/7OG4mluPet5eO9C/AtOYkwykTuLR/mcdeyuf/Q/RQ+/gPVIzyojFUwVtNNm0IcsJ0usG7iQTa4G3nrwqMEbx0HqyX0e1WCXsa5ITPLYjfQ3+3C0pH70avScPHyx636KhkpL1Dtf5U069OEKEWbqqJWFzp8Evht/XWiDdZ89hqc07XwirqPvXRTnP4s1h3XUWtVjMwACKR8oAJl/mUUd89j84ObmXtwLrX+tSRMyiJVLTCjEeqNKkbeLCYody/giEFhQGpU8kGrkXmejXRqJEjDJuBadwOlrh9VkxMGoEUSRt/BWaRW+zM0aZB7f7mXksgSfkzN4sc+eMFJhlFhxCyToevV0Vvby5T8KUz8fSLRbdH0N/Vj62ULAvw0/idC54RyoCqT5aHlrEk4w2vFP1BWmUnO2Bzs+kUGvPWULCYvzKD4od203tTjMNOBprImvJrFvKvWgBoQJGR9ezfSGTrmHBKBDEWfAq8+e06PLsdKshhH6X34uIWitIaRT44kZ1suZp0ZBzcRVDk7YI0VrvQcvkTM2WyuTopg4X2HuSbMR11fxsSDk8ifOoCkpwPnxnxsu+ooT11Gr3sY6tsRkgDY+DlxZu49XD+Qx8M1QQgLwGqgnT7XYNS14lw01GdNrUrLkZlHWOI+hsDsv7C+ZQ0Tmjmbbx6r5jvTEBH23fjZySgPLafPrg/bKpG8NBBoxyRnHdWD/Ui7oN8pgJIxDyAd2swIWRlBTrW83yVu4AVBoPTzUvaX7Qeg16RCO5TDmvs/YkOVO81VjxCUuxdBIkEi2BFbEIujo2izFbkokp6aHkw6E/pLlxh7rZ5t957HFjOfClDncYCrI8YjDXKn2bMNVysd0yvDqBp0xjhZyfZV2wFQa9W80uDE6PAf+OTTRup7p4AAR2YdYdqJaQTUBtC6cTWaLz/Hfk49LX1aXHaK9123RyT2nQW8kj0W+awqLjWl0Ghswqu5ldC8emrsXXmfdn4tX8Pl8L34zz1HpdNqrKJGcHUwmn77O7O+dN06Lrx7AbPejGeSJxGnxvOFaxeWl76hVL+EiRVDrN62mn6bfmxfmkiU8xxet5Hwbg/cMBtwbnOmOKqYYzOOMVa7ELMAHypsGN/vTKbGwKqfV1HnV0dVcDPTTk4D2klJWUFVXRhN6vsJvnENh7Zy6oLtWfDFRyxUyfm17U+a/qjEe7Yf258/Tp7yO/zk8IIjqIRimm713WwGzYAehcJGLBbfan9v2u187Ohr6MNQ0429nz1KvRJNvwSLwYTMrGHJriU49jjS7xmElbeFy9ZaMn20hAMezR5orbQo+otQ5SZyakQBMek5HOy1o02ays7IT9BaaWlxvkY0Y3DuX8uYk3uQWzrxCArGtSWE31b+RkCnwPj2OFQuA/xS4U1hhT3RsVoCqmyQmP0IUUCI+xGQPcf2v8bRl5mA/V57di3dRWBVIH51fgxqhnjQf/OtcxPQdmoZ6hxCkEi4Ok5JaKEjgkyGUWtk9qHZCBIBSUgveKo5cmu/bg0MBFlo9Grk/q33U5YcTEPw45g6Ozj6zlK8xvRzbH4nlZ4foNEHYts3huLfZrPebSMpyVnYOyjxanbGIlMgIDAj9BsetoMCyfuc+nkWo6siERT9JPQu4KMXP+LBNlAYfBgY0jAqO5JeVzMOq6Zw1CgWFO1bywm9voOB0HnwYCI2HiLodn7DWSJIZPq9R9lZp8Ush0CVuAZRGbxwnR6G/MBB5o3YxE+qj3EYEUpPzwk6XDvw6Bxk9Nk4fEPTyX58K18peliuCiKoKograVe4GXaTJfnJPD3hW5bIWjkfe5k5WRt4fdPr/Hn3n2SN90PRe4BtKoGxxsV4Xk7Ba0oAZ0Y10eedAYBMagapFc+fzaO524s43QC5AU/R4CIqt0ZeHUm3421wOnBiIAU7CxhbMIk5UZ64BOTjObgBJO8x/vXxHHn0CINtg5S9foMd4QfZ++1dLL0Yj3G1SLp86vMnuC7soGviVF7K1NHiVcegvR3e1wXGnRSJVjb9ZnLfW0bvsz+x82ICgo8TUdW3WfeyATWXZENMk7hQGlCD1NyKxWxh7Kl+gjynQrMXkrnnaZoUwUp1PV5COGt7BrDt1ol7WbmJ5Oxk1AHi2r3+ahOV2b3oNY5Uxjbx07Q8dIeS0XYqEdLE4uvrb78OdNMefpNz4cXsGHkKe8sI1PbOFEaIbhfKVgnfl46gQVbO+9J+gq0tJF/SE1CwnP4F7uS+fxQrm6s8KrUhUN/DX0o3DEoD6efSkZllSAQJn0f0UKcKw3HAmwnKJ+jfs4uE6mqUfh/zZ9jjXBgo57LrRi7GwfzZ8PS1ebTJmhAMN9D0pxJ6eTu9dY/hEu5CwuoE/Mb50VTWyYamMCaNuMn4BtAbLHSrGhlUiWtoDOIEs6HeiqTNDxPhbGTb6m2o9Crs9XNxDnmCcJdM/Js8yLvpgXXibRXGmVfPUHOxm0lv1HBk2ywickPRvaqjv6mfip8uYGclYYzrTjr0gahbm6k/XsvAs2EEPjSNk2eOMVMDh7xAKqlgz4oKSizzkUgkyK1VyAx9RF/+hoy3Ekh/bdw/yIfKOH8M+bWodWqQiMoH284a8ZSs7FhYt45PRv7Bq3uiUdwIRTCacK4vos+ipDC4haMzjmK2iiT5r/cYqPNGHj0V+/ocABpdNRAOfTYVDGq6sAheaHqa8W4W1RK9t/aKge7lWD/zI9l94XiVKFn2+zJ2Lt1JWUQZwS0vUOn5IT1OCpDLSftDhrv/NH5bKYLWF8deZNKaB3DwdxgG+gAiCrTc3PEiLxouY3a2QvsgnJxykqmnpg7/jePBqey6moDF00z/jLkMDElxbsyn5/0BspR5nDOa2PWvPB5y2U7S8SSujrxKp5cfidUeTGyOZ6dyDea2EfhbhSHtaSA4WxzDcZn3Ub/yKWQjhlDLLYAN0/Z7YlB/QOnY+9kGTDeARCLalVoEgdCCRowDsVwecxmJRYJ3ZxQguldUnayi+mw1M7YtJGifqCjS6A4hCBu568S7RF4XawlWVwfgRQnlNhn85FfACu/PyB5ljYlcumNVxJTFY64y4JUfg+ZSErkjzFSGiO4LOjWodbAk15Wl+5OI66rl7KIqJmVNIjMtE69Gb/IURvxjnImvtCIg8wJDk9zo8xqPxGzi7o8qcPLoxHarmt+9azALeuatfo2nFC+hHmzHvteemUdnYs5uhncg5ZEUcrfmIpFJGIyDXw0CuaszuVH8Lf+pVRUG8/2u8dgaoD62no7nd+DvHEm/ox2H3LbxhV7C4+veJzk3AweJA9IWKdWB1SRaJ9LR18Ha8LUoNiswYMAr2QvHEEfai9sp/F2s/Qy2DVJ3sQ7/cWK0ztiXxmLoN/Bb0ErGfXc/XE8k6TkFkvkSuBXl1VbQxvGnj3P86eM8WvQorlGud/S5OnEhLcFjcK7Kob85AltPW2rO1Qwf7/KKxiJT4Fl+HuPASORq1fCxwKzvsO8eBMRxcDO8hVHA3O/n3vEdUoUUpbXYIcEiUPRHEb0mF+CfShyZtRqLHP49EaEqbiKjJol7/Yy3MnCNcsUz2ZPmG80ETf2nYvFvJZVR7UL0g28w6h5gO8Sc+wb1UBeGKYmY6nP4Ou1PGmLsWBPwCgA23XXEN9+y7/WrxaHbgSV7V6IbewNIQZCIhDN/BUyX+POjSwx2kWpCAqyQqCVsNW+/3QkB5hyag72mCYvJgllhhcVaNXwO5944h3rCOgTBDUGAfpdA2uxEoHD68enU9JbCw+MwqO2GVY81bl/TaXcW5ZTfWfXEdLquVmDbUS0S7CUWPGXwRDtYCkPxtlUQfsyeuI67+eT527WGt10HsbZ9GXiPGcdEMr1KmojpjXeRp9zEpt8GzZAGxQgXtq8tZsJJG6QSX35wgznWAuf18ZyvXYLJW0PvtRL8is+x9t5NkNEGIzbjb9vDTO9mMnQmPOqsCSlOIScxB5vuerQ2rpj09shVt6ERQRA47AXGlac41C1mG5qzcoi8dACAvgxbEuLgheuxSA454X9XE40Rkwi7+isOZ0TSMr12NMuMWFkbQSWl37afDpcO3JLTmDb1MaZZw5vr3sWsUDLwxAAHVpRw7xZvpBb46GAKhjFSWrfPxKX+PNrXk4ZzGAHib91HZzzrKBshR2mrRCqIL3a6dPLlk19yj+/TAGzbaqbeLRmbm9koMTDp3UlcfP8ing2iJf60T6exf/V+sr7NInJhJL6jfVFo7hRGiL/JP///7+6UggBNEf/vwcL/X239/f8UBf1vtf+Cff9t/2ftyfVSAushep43nqFXGZuYw3znEnwcHsLeouLShHPMPTSXVr8B5oyVM/lmJMe/W4Tw0iAFMYUYlQYkFgmVx67T0h7JpFcvkNocw4V4ObVxc4g+/x1utVnUxc4GJOw9mUz/9Vh4tJmA/OLhgNX98/fzgCaCWoORSoP4IP7bk1/4RuyrrfYyK7wGuVTXgUQGd++6m/p6+Lz9BO32R0g2wQO8ccf5uezqw4UlhLp+QYNDNw8LFuT6AYZs3ahJWEDUhc1c/HUWr0/PYiUCZqP5DqAPoLrBAW35J8R7OVMddp3zo34goGIV9hcNPLS8GtuBcEZM6EDjLAUdtwqrYAGUXjpcJLWsOj0Noc0VQSewRraGv7r+wmPgHjplr/KxvZ6bxndwVdyevc2CBa1Gy4Vxp+gPFW0IRz8/msH2Qfqb+ok5ISeGIFyevILFEodToAMrjq3gtxnixmHgrqNM1dhSf15AECSYEGXyWmU9nTMT2Zj0MoFVgTz4zRu0LdHRENmATqVj0HqQ/AlttNn38F5JCLMzkxBGiJvA5pCxOE9ppPOwJyU3wpk+tpKi8X8yfbaGot1F3Nh8A02cFEdfAD2r9EOEnp+CRT6IYrGCGV/O4Nj6Y9i33URqMfM/wT5BENBptEgbPJmdM5de31oUt2wTBm0G8X7lPgq2Xue+MYfReJmwS5iA/2EjZpmZ2vvskRa+j4fQT3XFMrxKS9HadvLCxy/Q6dSJzqcPi1pPiwxOFUQhVFrwSfPh+Y7nac1rZfN9v+EwMRMP/xZ2no3B8WwS3VM6KbpYiVQmHQ6DDlRU4TNhgFcqvYHHOPzIYRqvNzLi8REETg4cDsW297dn+sbpeKV6kVfRxMlpJymOKialXbSX8is8glmuoj0gFafmYkauOIrrgXk0q42oH7Pn6MyjLN21FEEGN5JvsM6i5uNnR7Cz9RT1kvE4nN1LyI3d+M5fzMakh8koXUx7DljbS7nr9yV0tpmpPV9Lduxp3Ofs4ZrOliVW9STb3A7q7WvrJKC0CYm/Jz4NPmKGiu8Q2k7tsLXpzK9nUrSrCH2v/p+TR+slrNUyNj+8m+d8l9CTX0fYtd+pSliA/8JELpa4YNtnxdRTafTZDSK1BgQBiWDBT9aJPgTeyQwgOzMaXfgA1YHVRJZGMv93EzCCVqAk7SI3TSAIt5HXtLdmcupCJ0kBU/jKaQWmZz9hS7yYDfZ3vh5AkFlF511HGSz5+Y5uz/1hLtFLo7Hzs6PF3HLHMbWrLRnpZdwMOc9czxMAFO0qoimriUsfXsI7MZXc0gRGTLvGyah8ZJrHkFvJsfW2RWGlwLfeHXt5BJ2+Y7CVRpJzqpPqLWep3nIW53tCSMxNxDWwifvX7+CbQRdCZz2KjacNwVOD2X3PZdprPXmnbiRR8gAcNGImRNEfRfx5z5+kANvu281v5dGsOzSSgR5xU9F4vZGrn4uZgLZetneoDP9u+st6xl0Yx/aV23lx7YuAaBkCEJS7Dx+FL+//62e8ml6hviYBh6h4sua8QfS5b7Aa6CD+4b9QGJQMtDkisTLTGpiGQ+ttdmK3QzddTl0cmn2IULcnEJqj0GvsUZqdCOyfhYdtJrLhlagFQ0sP7i3utHq0IjNriBBmECt5k0qDFFOvK4KLK30uQQS0L8K3+0Wse5sYSOtGMsIGx2BHuiu7afWLIVQBqtJg3A6fo81RR6lnBrZ6WwatB6mLmUVHTBrzNXuZ8/kzaJ6+c6kVVZyARltN4biR9Lqtw88zF26tA9s8egB4qDqKwRQDWQ2juDD0HdZ6a7RWWkac11G0az2232zitbBMCqd50eO1nBfee5KylCzSnz4EmDBpwlh3upvWDjWr/dvwKzqGt7sDlYF6BEzDhIIrrUEI9h24FB1FbSul2wJ9JgntdocYNA3Q5HSA4vhn8OpcRmRWOBZPe66ZVyLxGCRVFUhM7SdYDxWJG7CP1qGbmEV4bhCunkYGVGeGjV7OTjxLcGUwWist21dtx8+cTlwOlKcuI0FVSs/Vk1wclcl8q1cw1a7DsbUUo96IT4MPLR6375VUWTZ3f/Yzj9SMo+pkFX8s/oMrC6/ga+3Lgu0LOGU6xV0b76I5u5mGKw14JHoQV+RAy7GV7I6tRKrdi3GmyzDQN/nIZH7e8wY7q+cSHtJIr6uSR59+mhh1DOu2rMO93p0+pR6l2kDR1RhINXNh3AUW7hU3fu2RbbyV0MRyz6/5rcAFiTaWnK3Z2HjYMP/sCn4+HsYYLz3n5Kc4+nYPqal+WFQGhlTtFPluJc0ll2ZLG9+1JOBf5gcjTEjra9B01TFk54Fe4/iPewqguboGNP/CV2NGqrTGYgH7phLcbgF9vckq9sR+R5tbG1qNlqVOKjp94jF22GLfXoHMIuX+K+P5fcE+Lo+5zOfBL/G7rXhO/t3OBHrZsjDxL2YqHHhPq0J59gAJhTlYpDKMCgl5PlZckoZy/qQLaZlpdIxuY8vdW+j83hZf889sD/0UOqDQHbotZsy2DlSkLEHTVYdX1RUmnpuISSPOF0prJVc+u4JzmDOWrCZcBZBapDgYEgDwaA2lILaA8bGTuXfoMJt8JWzaNBU3k4z+ybc3o+Fl4RTvm41h2hU23tfIau0AATveJbDASMaE6/jV+SMVJMTWmei2yWWkcBiPye8SkvU7dm3lDMRW4xRSx4khN/o+TEEuL0QYn0xYYRs5US74e+p501VOiUZOxsRzDHavQ2Jrg9bOBqRioeT0pNPMaV6DtZs1k96ZhEln4sK7F8R7+/paDnjuY7xvDOa2Iax0ohX4jeN9qO8Bi0XKPTXBdKi8kWUmEFsQxrsvv0uVzp7YOvihPJzG3+YifWwAjxYP+m37sR4UJ47JS06h75LRla/G4FSBQ1s5ADorOX/9NR7V+AaSBD2aolYU0VosIeIE4C6DxxzgCA1MtETgUTeZrryneeKrJygLL2PX0l1MPTGV6KJojGsN4KTk0eJHMQ4Z+etKKxvyNjAycyQzj82kN6aJ+Se+xbGnmkavTqwmubJh9Av8VP08NSUbGB1UziPfP0JOQg6XUn8kO6wUmayHAiewEvScMwoM2ohrvgvjT6IY08EsQy/+rR3kazU4ddrjXyvOKFN3hAKheKQUsu1KGBPOT0C7cIDPn/uNIWU1251gZcDPFMpe5ZuQAlzsW1krEXOLq4OqqQ6qBkHKsxHiGNu/ej95v+QRtzIOeVI4z/puonLped4tW87zeh2pWWImMtdhtcvdnH7mKxSF4QgGF4ZGCcgcevBu8sajrpk062qwD2L/tOO4+KdQ6SnaWg2pqmn1MtNjk0+bTT+H5h5inHYCvVYPDT+fVBKwlYJUIsEitWCWWShNtKLZTizUHRwEN4M9Cc79/Pbae4z1fBvdbe4KV2OeoV0hJ81NVIxe/+ZOENrGfoDOZzdjNm1BuNHB/HPzScxNpIqLgCM97Q4sm/Ip9Q6LqXZ5A5VORavjCWAU9eV+XNKJtluCg4Z3XvsEs2wQmUmG+pf7+Foq0DGjDPfBIAw2CsrC8+m37adZbUN28D6chUimDbpSdbIBB3c5DYFQ6lPBS5Wj+SD4MlJzL82t4Qgd3VQebWLtNwEcnp3MjZQbbLl/C2aZmS1soXRfKUW7iqg4XoGgsmJNShPfpAdj3d2IachAW8FtwMLKLI7zunAZ1f3zCO8Y4pFvH0F6SyXqdPY8bTcjyZ7pgrsqELuyErRWWi6O0jH+lC0yC7zdEc/Kq4mgi6coPIlmP1sksijCr+2g0TsFJ4UDqSroMJoRBNg3fx8muQn3Vnd+dqzmMYkEqVkKFgFLYwfRl45SYxfN9RHXyJy9ik3viySpbWvPYykuIwDok3mxKAn8LyYS1GuLsVvL6m2rh8/ru8W7SLQ28y+NDSd6ZTT+m+rV/0OB5sFZuEzOYczpRHoXdaHoqEc5YE2IfDkHdLkc0IP+yhosx52xudtMSLe4RiyJKKH0iR/4zq6DFOWLbN84geBoFRUBB9DaFFEjC+DDS7/jFRGFi6KAqUHXUJjn4jkURwtnafOAuugZyDCjcdHQXd3N+XfOE70kmrJv87AqmY+73T4eORdFv18PykEZLkMu9Dj0YPjRmQ8L1/PhYCr2Bi1TJk1Bu1PLsp3LcOzxx3v1eZ5OW826H6bhcW40bo+6Dedz1Z6rJeRBa+yc+pArTAwoTYxcPxKlrZIDaw8Qxm98e+1hnt30MS88sJ7THRPQ+GnY7vQWJ8d+w6MK0XYaRAW3o1JAsAj0lTXjXXoVlbaXi29nYB4yMO2TacO/9Y0fbmCsauVs+llcOlzQ3LLubwkZh2NrGZ4VF7GPEZk6vpN62eHpwQQBAvMOIBmUUe1hzdhLU8lL0ZEXl4fcy0yAldXw54+8MMTICxv4+rGvaVG9iGDYSnXSQr5KEa34D93CSRUqI/KgBvSVCqQ+Tuy5aw/Nns23PuXW/l5iQRCgeLyJYpcbw9+hV+vBWYZULiV75svIDFoCc/di11lDWEEXgzY+tApaypR55MflM/HsRGQSBW5RZUSEN7EmqAQf1TjiAbNCjc7aBdV2A635r7LrjQ0gER1Wpp6cipXOikafXgLlLgjNPoTHBWO5nkPDrBRsq8W9xICjLzbd9SwzL8Uzz0T84TQMzKMtfBwGiRKb+h9wbmmkfMw4dndP5vo2b6yW2CArbUI6GENOUg6PffMYFqkFvoaqU1XsmL1DnB+/uMqEMRNodW8lsjQZQ/gAcpOKHoceqkL7mDA4mX3Zz3Eufgo9AZlY1FJyw14gO8JAqvRBjFEWHCU3kfRI6XCwQykDbuW7Z0w3M32/jPCbYUgC7elzjMKjrp2YzPHkxeexYscKAHIe62C002bKjA6YbqnUZEZxEu9qd8akimZrcibSJAVPyxVIJc40e+Xj1OVEXnweC6aLmXfeqd68IbyBSW/iyl2fc/HoOla9vpU8iRaw4vmP/i3nFShNnUDXhS4irfToenQIQzpch3JwrQd3mwTWz7pMfXAnLZoWRh4Zidd1Lza9uYn24nZmbxxJlHQCHR8786H7Mu65V1QZDbYPsuj3RbTmtZKzNYfq09XY+9nj4O9A9ZlqYpbF8Gf3EENvbuHkzilUtY25o09+4/yIWRZD4c5CdL06/lOz7ajGo/g43ZXu2Hra4pnkOXzMLFchtZjwLLuEcSABUKG0VWLoN9wC+iBjfAb9tv1URMpJ+w8qo+jFt4m3BTsK2HvvXhR2ahgv7lft2soxKzQIghf95a30yS5TGPLl8HvaAsJQR7sgCALn3jhH1N1RPJT10D++p+lGEw2ZDQS6RzIttx0ECZZb8Z/OdTmoh0SyaIXvIXat+QEAxU0dUqW4f/SouoFtjxcbn9pIr0MvE7P249l1E2FQj9Skx6MBOpV+TLKzYZZbMdukBkBNU1YTFzdcJCF5DHVzK4f7k3IjhV5/0PdoiTvzBYNRKcBsXCJdcEuPwiJVilGJt34zo0wk2/y24je+nbcLgKrkxcOfJ0jEZ2128FL25bqxaPREdt1vRm4+i2f3Ij6QTWaV/2lee/0eOoMM9E9wJ6N/GyDaUdYHQl9pLif22dNsd4w9iw6j1kJU6+d4VFaRkhXGZNux9J5NwxxlYtDGzNb7t2I3lIBpECoNEjokIbT9EI1b3EG6xrXR/ch2HG+RLXEbzxft17i/vpx7fDVkTezirznfseqXVXi2iOfTXfUoLhEubJuwDZPOhGOwIycvP8fB1VuZ4zPA/YAweDueRVFlQBsLm70q8ZmlZ6rLavqdb2e6FqbU0uWVT4GdHT82WWEz2pN9C/Yx59AcylpkVPTC3kHoilmMo4OaU+bVNLpcYv+8SOYfmI9h1xxOBVQSpnFkwNEXpBK88+RgSWFgdBbZeohziyc/PI8zvjeRqeT45R0iVhJLQZzoWPL39dO29BF79iuag0ez9ICCRbsXEfxJMJfWipnisctjkcqk7L13L9unbuexksf+o9r2P4F9/9PGU2o2oulpxGBljyD85z3sf9v/Xvsv2Pff9n/WBmV2XBmxhviJQxSvy8bBEsN45zfx8RhNjfVuzmnOkZ6RjnudBdMzr6J6cD+e6ddpdp/PmclnQCLg0O2AY5Ocbodujh1bjc0VDYGfRpGrFK1QLFIpMYU/YD1CxRcpF1GE53Jvddow0Aew4c0NLHt5Gz0DoaT+8hy+b92eqNtUV+iy78K+9UvWzrvKpoSrqJxFFkp/fQ9SQUq7azuC/j+YGN9qa8riuN7oiH6mlvK0e4k5sxH//IMMBQVh013O/O2jmdUQhGKOLZvWbWL68en0eKRwcNqTyEwyXnsnjsG+WNps8lnxp7hYtG6Rcff+t8lOr+L0hK/xMKUx8dSKW6p7CXrfEK67LMJPcobVC/fybaMzPRU9+L/mz+erD7KnYzwSNwiQtvNo5ng6nCQU/1nM2dfOEpg8Dm45t5lvVYS9UryYu3kuX4V+BUBIXDnh/Qcp2JLIpJdGEDAhgCfKn2DP5hPMPDiX++87wQfdD9B3wweZaxdzS+dyasop0M5GqVdiZTtEc3gzahfRuyYzLZM+uz7m501h+omH+OPFT3g/Lp+X7I4RnikyezvTRuDV4oG3zppmiZz88B8JUKYR02pDc3Yz+vDb01mazkztubFYQrUMdQxh5WRFa+BIetwj0GscEAThjqBUhbMVczdsxrUogv1fxWHojCeuZiSJOQoKk9T0TGnhjffux0nZT27LZEy3WFFKoxKyRasttUZO++g6enrCkbtao1PpcO5ypvPRAl6Vvcbyt78ndMn3yE12vMWbGAeN/DbrN9R6M/X2Dvyl82ZPyRxC61UYBo1c+/IaMpUM5opPUTNS6v8NfBrxxAjKDpSRuTGTqsoqPJM8sfW0Ra6SE31PNGoHNZZyUQ3U7WKFm+QpABoip2DfXklAvphZluzfgXTpUU53e5EgWDDJTQxqBtGO82DV9lWkzbzMyYAejhm/w4bxNEZMJqDgEE1fH6cl3Qpbzy5OTH+JOfNlvPaGEfO5q3hUXWYCUaTOOkySql8sdCDQ19DH7/N/ZxApM7JnUBZWxplJZ6iKHUO4t/+wpzhA41VRFZj5eSZBU+5kx/346F8MKmpx65mFV3AktsGuNAePFjOpJgdje/YvnD06RdVR4Fu4bt2PXX0u3R4RCPXWHLOFpA4PZFmpGDu07L1rL5HvRwJQEVLH2fTjOJc8wKT2EBwnuPNwzsMMdQxx+MkTeLRVsmLxXj6rNzNoNYBwyzYk+cFkPuw9ysGOdxmM0SOVCjzVc6eyT66WEzY7DLPZTEvBnWCfysmas+k7Abih/xn4kFHPjkJlp6LuYh1dG/9gPwt47tuPuC5tRaqQouvWMfZfYyn4rYAFO+OR2iWy8M0t/Fy5kyFnV7q8onFqKqJzdwVDLh08MyubDomZ8g4FLhEuwwu5ukk6Dk78jBfVk1g+dIxj9aJKoCmrabh/6edG4PFQJo7PFlLvLRYa/gZmF/+5GAd/B/R9elR2Kv69WVosBFUFcWnMJSRq8Z6rjp+PIJURlLMHpVGGzCzjwR8fpGvcdWwfFpmLdTGzMBnP05MTQkiLJ8e3zyTuwUSOuwRSGzsb/4LD9DoIfP341xz0M1MTmMXxRjX94eniPW2GM9cOcwZ47J44Ui5P47gMlK1dPJzzKG+9vgGlyQWNsgNFTT6SAxH0RUZhTEilQpnK+vXw1y4Tnoe3UquOQ7HyLtZXrKejAx5ZWELscxuwOPZxbeI5AtxiCMyUMeXCs3z92NcY5UO0Kzv4s+IhJENa/CQOhJWFYVQYqQ6qpi/+MhNUcpYuep6NTaE0Na/iqR+e4uLYi1yLfZsx59ZzfEDCOz5bsDV2U3/Bj0nnn+erx79CrxbAJOf5ohQ2RU9kKNGAJsKfiphO2vwaqBr04EzZC8xKn0hr9yGcCmtoSI5h3903qfI9B0Cf6TEaWr2B42g0YNWQh97KnmsbKnmvAdZkjuH1Y1PoC+3ApUXAW+KNoLGgnObN8hHv8JbKhoNd3txlmotz3yiuTYI2+9/ot+nnfadaYhwdGbJRUDO+GonjdTRDGnLSrvLd0itM1km4qoebAwO4V11BoRtA8chU3EY4cb/BmRhrO6I33M3xygdReVjz/suimvIpB0hVQ6L3Gi5lDBDY8yKuUa7oV+mJ+zOOUUOjyEnIISk1CYC9q/biFuOGZ6InkoNyPEfnklMZgFFWg9rHh0+f+RSD0oDP9z64ZpbTPMuDsPU/Ut0+G6lESkJOAlig/14nPP2PERXRjVVoMT0O35FfKRZe0jPScWvwYuDre7mUVkTlfgkODuVkvSSC9anB6YTZ6zBKoVeajHPTF1jqpOSubueXKDGzyVUOrjRwPSiGX15+jyd8tiGRhnJzxAo0fS3EZGxiyM6dgdaVd2T2lf9wmQkHU3lx8wd8lPUYggB93hHo5DbYdtaijavGMayWGj94uUNkvve7BOJXfJzmkLG4dhzFt8GXXz3g3S6wljoyLUdkt0vNQ0Sp72cUN/noi4XIO5yxJIuM617XEKqCyqj0uYazMR65qR+VXoXZZKby5BCcnoLtpNskh3c6JNwYkqOyU9LjEYFO48igIP9/2PvL6Liu7c0b/RWTmJkZbIEtmZkxZkhsh+xwHE4cdPCcMMdxHLCT2AEzMzPIIosZS6xSSSqG+2E78sn/dL99+46+b4/RfeYYHsOq2rX32mvtvdZc85nzeRB5BhCfLlCwmnQmpnw8hdb8VpqvN9Pu086gkmWkNt+F0wmz90/FpOhBoZfhuf0xSl7eTOu8w5zXBfGg4Q2mH15OQepZQhtCAfgouIw/TsTQIw8kVZoK3GTs2aFcHKvj+PjPcOl1YfEfi7EkSLHJVfT4xeLaVY9bfiw/KDvxIJiOpGLkjiB8YuM4NSeCqugq3vO0MMNzE8dNPuztg3FOJw6TBYnFhsgurAPNoQ78J05BIpdg1pvZvWL3QF9M8ryO6vTLJI0Nw5olYduybSz/bTmZ58sxnV/P+HuLmbg5ifDxCRxJ3U1teBl2iZ2/grWvuDUQMf0QWb7P8umLHwyc99d3czgrhSv9YmwvX2KZ6Tr9jaMIrLyAq86M5NYw9gflc3zKGsLXpVLk/ywBF2W0+viR69+GXzU8HDMRbUs12gI/bFnXuDzyIh0+AgWXRW6hX9OPwwF9rX3sWr6LpMVJOAOFTXxLQAtXs/IY4eOKd6sAPm5fep4nvvDk3Ztz0CscZOa+hXX+HE5OOEmrfysmdQrLtsygObSBhwuaoCkYyTLwaRcC/zaZjc3RufiLC8mPs3GiLRTd5WLeWyJUfncEavFESXDYE+Sl3UVtRC1jfV/EIbKCCH7thRwzPCiWURVTRXV0HUuL25l+aDq56bmoDWq6vLow2YTAqMZfSO6KmBBBwa/7aDJG8tCMc7yHE4lKzuZVm7l3y70AeJiU3AqHDRsmUNqvoGK4iAujLqAN1DLpJHS3w+T0ZtaOyGOkchBBVUHIrDLqIuoIrZVg84GNgQ7UYthQ86+REydf6OALHewP7eCR+4/wUVM9nY1rKA0r/ZfDRKwPuESywsJaM/SVNeHZ1E3uoBNUhh+gMhzS/dcCgv8UNzuOWZsX8tjEJVjMclLkkGt1YKptIT3vjob1xjUbuRbQg6HXwai27TzwwwOInCI2PfQH+2ftR2VUMbUqnjEnR9A114RGGisAfyInBakFNAQ0sK52ONrKWHIypnBpYg0a82PI7O6kyCEUIyN8PydujZHyhMdwlJxEIYJbff6s2V/BC+tDcPa8SfwVJa0pqZTHlQ9Q2eo8dQPtbCtqG6gmSfrsHi6egAf3jmXk+XBiBl+jcoOQ8PBSz0s8PvMZXsgZS018OKbQdBwUILVJOTrlKNP9VuH4tQVn0jle9w/gYutD3Bp1i19HfUJk61rai1RYxGV4BnQR/aiVfm8fvu9+jUbvn0ls+AfxbR5MKwxmskpL1vFZ5N2Mw/4z5KXnEdwYzMKdCzGqjHg42nntvVfojOnAMcGN6uRMGkM3ggisCglSlZAIV+KWhSo5kqDyM7jbV7DR/2t2P7yDD1omkmK1E9Byp9Ik61oWycNLuE8FGt0j7Gg5SHjZLM6NOYc8IQhTTg3f+jiJHZRPiP9Mvnv4D8xyLWsNq1n8YRZK40j2LjQQlxaHv0nFpJOCz6fz0OF7aDz7M24yNEaEX81VrAZ3qlYYuSbbhczmxfftSuZZEhlvO4i0+hR6N0+yt0gZ2fs4SrOS1shutIOnofYGXY2Owq2F+KX6kf3iBPZ+K6e49jIBN9KxdBuIuO7C9DOP880j37Ag+j4yCqtYmPkpLfFteMf9kyBtGmKHGIvaE/2Xtby69TlOr/wRW0gVS0KXII4V5l33cHd25ZzghDYJtynHODghjy/5ipvf3QTAIQKxUyClK1b18pHyAONkNmr6hYDoNz0wXQOzNLC3K47XGrywW+xcf3ATrmoP2sIzyQhpJyBdGAOtFl59FbL03Tj7TFRHVfPAjw/QEF2PwwH9nsFoY0bhqS3B7BrPrbbR1Ha+ihvjEMkcVAxdTmXo79gldkwqBzaZiN3zdzPMmkBEjwvNsaPp8won7uqvAKTnpjPyUgyGVV2AP/9qExvhjwAo7JeCzY7TV0HB4IKB7w2KWiacnMDQG1k4H+ulOt1Kqcud+UTdr8bWaMJuseOQyJBiwO12ZeKouedozBbR3HYvDqeNXrdepHYp4GTDxCN4+HcRLQKzRYe8UIte6YampxluX3670g1JUyCHxFa2Ld/GAz8+QHCjOwZPNV5ZIkIjb1Jv8gVDG+7tVdREdaMPW0XmzS3UdaVRL7qBKLaJRtcSOiJXYrPBkAN7ABFdbCdEfZMiVw+QyPj2oS+RWcUD2neXhgt0dW4h7jgTE3FJi6H/t/2MPzOeHrce3PXu2PtNdMdMw7+2i6iaCDzuHU7or0d4b0g3Dm8RDU4HMw7OYEhOGgAWeRF/PnSSijEdHK7w52rlgwz+5TX23LUHi8oLZ7Yba2LSWTZ+Nntq5vO61pvTQ35B56GjKiOX6JvpTPMsZsmIUxwemoI55UFi3/mSxoBLHJzZTIDhHTKlnrdXAyf9LXr8tR7olTLm75pPRWwF/lkJfxv/s2+dxeWwAGxV5MTjNHwNPEdzUDNWmZUbWc14SSfgG/gM18dX0CWS8e2Mb/F7fiVtH/6MWeWCpg/8r2Ww4LmPeEi9EqtDwn7//awbto5LUy+Rfk8wSdPBJ6gTu8edeJfGV0PKkhRSlqQQNjaMbdO24RrkSuaaTI48eQSz3gwI4zHtuV/QT1+IWHlnn6/2VrNg2wIWbFvAf8uUvW3Y5Grqxq3EL0XYTwcPDcYmVSC1mZEb9dQOmk1HSBqT3DWYe81YeoVk1E4fK1eyjnE9S0h48ejLxumEEy+dwC3EjazHs/7teo7bySFBM9LgNqlBdM52+rzCcdqXkffwRmIilZxLP3vnN6I7lJ4P5z+Mwl3BnlV7sPRbWLxj8cB31cerObnuJGkbQpDbgrDZ89Fu2UeeX/oA/T5A5Fk3mAQWkwzXPiV2ix2Q4KprBRhYEw2uauQvP4fKFRTbWxh3zkJs/C+4+v9KkdxOwIUPKc93YfgXAgVqRk4U+/4qahTB+y+8z5y6BsQKGS1Rw1EEC/5z8qJkeoKTOfQPcNrsFDz9M0MK64nxlJG7Fuoi6lDE/LsuKH+xUO2aR2tfEdot0ZQHC0UQkW1PUln9Aj2B1+CuUozuE4lLXsZmvaAfObLoOvVuazlcL6O3KgBltxYf+2ck5pcTXnkBR0AXUV49jBxayVHbEJqtVUgMV0huTqYhtJrNegAn89xOk1xvoUtTjmmOhpNBVciVQMxD4BaPh1bOuo3r6JjvIEO1gai6rwhseReAc6PP8Zy3kBRff74eEFjo1H3+aMVdHNN/DqxENGokN6QjUPW24TX1T15wgTKNlr1BWmReSpxiCTdmvUFgIPSFJ/NBYDF7fppL34V0LCONSG1S/Nr8cDH14PP8eo5POo520CYCpQmMOD6Y6Cuj/9argbQy/9HpbOn0Q+ZyN5E3lEQYRyKalMfTnjbqAhbhV5GPl8SJ1WAjpLoRrVcYFbEVJBUnobYLCZE2iQIp4NFSxs27LSyvXs7xfx4fuI5EJhkA0jV+GjR+dyrnrVZ47jmIjYVVq/4+6j09LeTlbqKxfBtWq56qAjc825MYboykN3YGTud/qvz+d9v/T2CfzWbjzJkzVFVVsXz5clxdXWlubsbNzQ0XF5f/8Qn+Y/+x26ZUquiX9VA88TSdIQ10a4MJstiQaOQggty03IHN6BpjPzfGHeT18IWgEwFOzAozNRl5RKZWUmG+G5FlLtFSBb71N9D7xuBXcwFlazPJwflYU1upt7dij4iiomYXCaVBA+341ubLanEzXtJGnHYnn4Z9ike4B7kjNtOmPsd+2SfsbDpGiUUImp3/x3nyfi1mUrE7k3iK2uk98KRwLpsNJBIwhUjITLqAwaTBmp+GxWimJHgdyQ4nmh4VN0a9jk5zg9C8SqKqIzDajLT5t/HLyl+YfnYUGTkZzJmYy6zHD7OteT7K1gIkDoE+RKQ3k5ILd09qZUgEvN3Wgeetw5hcfWmc8yiHI1LRScrxN7zON/7lVGsCWe/vSkvaVG5V+VAT+yWDXMv4wGUjbx1ZjyK1F+aBvk+EtsYDX88E7to1Gq2hCtKE4KlLoAvLDy3n1HktgZ7n2fHxKBDtYfTadLQ3tbiFuGG+2EdvTgp/xHXgfcsLkWcXJqOCsJuZnB17lqhn+niZl1n72Wdosn/iV9tGHnj9EqGNoexcdgSf+GD8wq/wnJeSsy792MVSOv068G/zofqf13BJamLw6HxsYgcvxZWwvfsxPnj5Q7KfyGb/e4+zr1fKHFcbB/wcFL30OQmxG7ix8QanXz2Nq5s/frXXuDlt3d+APhBAzVApJGWV8vOKMXiOSsIt/zxBFXFoI0XU/3mEM7FBlFiCaTnwALO86/ljyR+s+HXFwDmsiw/wythaTjZOwxn7Lpvv3cwDPzyAzCbDaZew95dFsORrEDkp3VPKH/P+GPjtyuUn6XbIMXn+g8Ip4JsKy/YvExyy705jt0n4rdeFvl+9mKCNgPWQsjSFnI1CdmbbvjZ+t/zO6sOrqT1Ty9bpW5n9/Wwcxe3cve9uLo+swn6beabXO5KYHEHrwgl8UPcwmyNfReqiZqHWgHuPOz/d9xMLm19EpLBgSCrn3EsPQoIFtewGkuYimmNHE1Rxngt/jkT6WCVvzknioOhTOkvSSKi+wwce1ByMNKCc1z96np5AG457HRi7jbhPieeHqM+oiq7CpDJRk+iF1MsdpfsdrvTqE9XUnKwh7d40ANqL27GZbQSmBwJOXPWuzD6wCHelFJcFXogddiIL9pHzGHggiFi3+xsRuSuxegdCZR6eLaXU9rgwVwvGyTkwqB3JoOVYOi2cHnea6Josmge9Rlz3Ol4yPcblQyHIVksIGOeL0+kk/u4hXPvNwr7CZ7me8Dy7m31wGO44+rHv2pkZMpPaxJ1EiaGj/84cU/hbIXVn60i7N42AzABCQ0P/9hz+lRkVWx5LQIuQKuod683kDyazMUPQnln23FacCjONvWE4nU72rT2JxSnDqRU2Ad6jnQS5lSPCicxFQXX6AmwyFdGejUwffJBEtZ15zZBrFoLsv835DY8ID3yCeukKAzjFZy8/T09nPXwLE96ewNjXxjLsnkm4ezbzU0QVRMDR+If/9v5c++IadefqWHV6FRHjIv72nc8SH57hGaa6TMWus4MPdIYKYyqy24gs2Mf0w9M5NvkYMwe/QH95E4OO/45F7UFNjBSPE2N4fewtIuLGERcfC1oILxS0JPKHOLFL7Uy/7ZMebvxvk8O7YaLlVBZ+4WKcIwfRbAhhau5T2CV9hI34Fk9NB9XjmgkLVeAUmFbRaOBq0H34jZIxN3ssAA6bg82DPye+8/aYawx4zzqDxf4chhA1l4edRWnLIPbKBvxNbYif3cclVTlD4mDW27Poce/hh6gfUHR5onexkyKWMkvVxLYuPf2afqwyKyaViX+88AH+ulko/vE6smor3ct7uDa0EJPSRKcvRFWAi15JjvElThTBykw4O6sCjWseFnsYfRYPRA4jKD4ktHoCVY1F5KVsG+iP1MJUYiuEjA61tgqbVE5jwkTgEwAejW/namchEtc5pF+sJ0E1kx3LS6mXZnC0dRGeyR/h4izH2NJD/JUtOAa5IHePwrXPlUlxzbw9t4TnSz6iSV5HU2QNnt2eLP3lXg60+jLt468IPT2MPLcEtC1luHbVYfrDgI9vJRHyw6jHhVPZlUl7fxghjjvjOVoFd2nglnoMP/6iwDXCA98kX/wf8Uf3sw6Ao1OP8vQIgbJk9CujEYlF7H9wPy4mT34PdnD0vg1E2qeyRDqGXrdeFCYFjXsbUQCiUg1zwyHR4YJup4679t4FgLnQQMipZeQtOsmf/mWM8hCqzsadHQeAzSeM+jYlR5ojCHS3YvMXUZVdRfTVaGp3l1N66hPsowu5965w7C8/SKVmMkGSOxqQf9kI+aNIqmYRE5tNnas7ej93VLcptdT6Vix9lr/F+DzHhxOg+AapzEZVdzoOB1jc/dAr/OjxjSHhyOcs6R7BlYYepNvn4vGiKza5mtaILHq9wikcU0KyWxXTy2O4K3cUjpE9RBdU0Bk8iD5NL5UHR3CxE64MysWtL4CQzMU0SisYfxJ81eDZMgRNhol7J4nYHuOHf0oIPyV9whCnN5+nuZJj6mVr3Bhcdriz7lomjsUmOlwvYfRqxl+5hCBtAfarxZj1geT/ks+RJ48w8kUh6/ta1jVSi8bh13ASGM7NoS1MOB5J81UjTSkVfNXTR4dvBy4uCixtOrKvxtHmW8aVYVcoSSyh0V/LXf9cR01KJ8as2RgqG1H3tuHeL2KoAoZa5PiVJNHg04de1EpI6cmBfpU4A0mVGVAtO8GO1nTsTgVujhVY5PfhfXsNzeyxs/Kne9Gn6OgznCX95CUuzpiAj34S87cPp9bwPY4XXsMj0oPU5alkPZnF4V//ZFHmb1TcH0UfifSmuVIdVY3ezYCbXpiPY248R81gGaNGe9NgbcOgqAbAS2pgrjsU4EKHeRk+vilwRxqJtg43WkVOjG69IHfgtIhpiRpOR0gaduN2Ahv7UPUr2G+7m6tWK8G2RkZu90U2dAiHZh6i3Q42kZJXA2y0rF/PS8FHORlyp0/OjjvL2XFneUf1HXaTkZpTNYilYjriAvDGm7qIOhpDO3g5yJOiobHsnf4WseWZSGxiJMXxxC/Jp7M/FYdCyvkxQpWjxFZFV+ZUMsPbKciPx6sujNCbw0jfK2Hjmo1og7S82QV9Dht/9IG72Y0R/Is+7UMb8e2ZSoY6hLfjuzlj7EaklBFeG0S3u4Wj/m0cNcBDThsb/eCS0Y6mSkH2tWzC68IJaBWC9qWxn8A9rzD5g8lM/mAyLXm3dQqLI3n5RCL+PUoQO2kKbuLH+34kvPVevGIEv8tt8SGOtfqQlTOdOafmsOeuPWTcWoVXaxNbNCKe84AbJhtTj07Fu9Obj57/CFW/mMHuLiwTqTm0eTpBLkYaRSNpDGkk2HiHZsvV0cMwNfwktaDoXoS7IYMa/085k7abBkMbx3qiONYCYl8Hhgs5ROfmcn3wnYzzv6jCVp5cycmXTnJrWz7ReYP52qSktk5FSLSZ8gldbE98D3/dUtbkpDHtyDTOjTnHil9XYH2gk+qYXkzyBiyyDnKGCMDv+qZBXGv3RtRvI6zbn2Z/LzT9AiVlfVg9bmUBVF0LxjsljsFtQnVC6on3UGxbz4xHd3AlZQ2lig4i/KRMf8eOIWgR363bTkFANwrDe3w8bTOf+4XiGJzB7yG/AxAng0EmF66YBFBs9LrRWHotNF1t4pbvTaQpa+jT53NcEUGUzx093tNvnCbkgh/gx/YV6xneNovwiLE8ft9Nul0uM616FaKgFt6fu5ecU5m4d1zDGeoY0Cq6MimApEEv8oAojHsGvcuBthAaJT8z9sxYxp8xY3eTIPNyYpSFc0V8DpObfqDfuz27KZ6/j2GeqXjbtXgNr+CqmwPXkFBOjv8HWh8tIoeIepfV1JVvIDQ0il4f4V97WAYSmZND1Sso7w0g2/NeZL4S/rnun4TXhnPf5vuYemwqQ9wkfHJkNiJNKx5JSpJzhnAt6xquT0zjwg4/Ng2ZRtyUm2x3/gbVrwJQfb6XTKMQ/LNnfsBvKbCs/hjdUX3cSjqNyjmH7KtOer3jsVkPIr+/gC73MThuB3Kt0i4qA9+jwupHa0cjUQdG0Wm7TGeCnHZHDi7moXj3uyBC0KULzAhkzGtjOPP6GRadeogb8hVcM8xHPN1JZoyKznALZ8ecpV/Tz09LT+Nb6Y5o5YNs7fmaXWtg/s5xuOpV5IyLY8SxK8i6XBgc0kVrYBfVf1Zz/TkhkL/w94WUepXiY1Oy1vcyt7pVtN1qw+l04johk5/C6yiPeJzrTaM5JrqEyC7BbrMz7otMMkSRbLl3C/Ob4XPPLHQHE0iuD0EkERH90AQO3fSk1z+F93+Dv0Jc330HbW1wgElkBRUxb/c8Dk87jI/5TjVVU8JEmhImcjo4no86OxjV7YWb1YRTpKDHP5YOXwMyswwH3fSpWgfeW5FUSkdoBjaFhmNzFNT7fo3YIUbqjCRZpUGlb2XG2R0cGruQTAVMVMPLpTGE/eMeREPbUGW5E619iapAgeJS6/UnIa5DafXrJBzxwFj+ZaMvTMP5YTU9U3vwaGkj7LZ/7RA7EI2/zCpfIz3dT9J0qY5139/RL0s+PBnNmj/4wR+ebLajyr+CZ9kdkDH643+SUBfJzq+WoFxloSGygZ/u/YnImkjGnR1HTv8MFoy+i7SRcPxcJsWTj1EXXkewYTqXBxfgIQ7GzU3FtcU/sUA2gpKG9dR6f0tAwyjiayOJcK9ip+0mV9NgRip0trUhNwtzhEPk4NhUIemqT+1HTvRi6IUx/qcxtPaRk5lDQVoLPwa8RqfrAUyKfgKbrTR/sBVvwPdUKkVXlyF6qZ+0vJSBe7JJHXg3hPBHUDz1raNRugRQHaJAKkojvkiNqLCBzFGH+EoHR+W7sKgfpFMjJFGdmb+Xex7eS651PDGV/nSKbvGBqRmrpAeLzMj1oeeIatmPJ8u5UD2MSyWDKGk6x+LfBlMZrUHikJCbnstdDmEdPLz2MNe+uEbq/UMAaBks4/SFFNzUwhr2F02rqzGZoTXXeFVlRZXZR1Kj0EfqxHCKR6/BRisxtV+jUAogmRQ72fPGospWMThmMG2b2/jz5G6O9M9iXtK93Ocfx5k3z+Ae5k76fen0NPRQeaSST/Z+gnmxmUfHPgrA4l2L2Xf/PnrqBXCqvt2NledXMiN7Bt4yb0CgzXTYHYglYkTiv8dlAILKzuDVUkLx4jdQetz5vCL7bhIv/oh7eyWe2mLaooYhkoLNeKc68Mh8CRVB15HYJMzdOxe1fSTOuZCzMYfAjMABsG/z2M1IVVLuOXIPyYuSiZoYRWGlCoSCfOpSZ2FTaEAEYfeO53Jd3t/amHZqB52tYYjun4v/IMFJL95ZjLXfyrWvr5H1mHCd1OWphAwLoaxXuPfSgIcZdnga+RfF1Kfeg8EtgPCiw9h7FZRY4MbBbO7flc37Leu4fmY/VyVtiGVmxr85DskgKX3J4oG4gUXpRu2g2Vwc/BLDfM4SKFNRH1KIrzKG7upuEMEvK8/+rd1GtRGRxA2pWkZj0hRuq4cAd2gZnU4wNghrv8Hljv9lswtUkr6113BIZHSGpg9Iznh1eaG2mNG3dxBbHkurfysOkZXclik4HUqWjGznsPURbM47TGZOayBPX/yeGzFz6V/3T95I+hOTtgVFfyVdXiK2rv6GAJWNw40fIvG9l8jmC6z+frXwLMRUDDzr2YpiPL78lTzjMjzscfRYxcypXs6+hULSqL+7jvahjdiDByNr0KPRm7ieVYXWv4ibmTfR+GlwOp0s278Mm8lG/Jx4Xn/5dT7YvYS+wnh45U4HGV39sHmJeP827vlKBzicduIv/YTUaqRr6aPEO7OJlFTybXwJZfIu0lruIaNgMXoPGy4GAYIZcz2bl1Z+yItNSZi1jwDCfOEQwZBv1zNO7MNIVTcHLd04cWKYNZ2KpkYCcmdSP2cP4ZWvsq5qEB7nErGutPHRc19hlvfgpndjzv456BE2LmaJGourP+reVnrzwB13eq/18rnyc9bet5avk76mo6SDkOEhzP5uNiqvOxXmublQXQ3FxS20tmzi+hkB2LM6rRw52MFosYKP7SYCgJb6Zn6gki+wk2JVM57/gH0ikejfYn7/b9r/NNhXV1fHtGnTqK+vx2w2M3nyZFxdXXn//fcxm818++23/+OT/Mf+YwgPv1wux5DfSOKpcRSrjpB8YDOmrixiE8J4bscPVKfPZ8eqH6j2ex2zwozaFIVK4sKiXU/R6ZHHqYmn6F++hzcC4aU8LUV9iTitbQSXn8XgWoo2OgOT10HqU3P5TjWaoq5Eitzy2T9rKzbJdFKKUuj26Ga/q5KJquUo71bhmRFJ1OSogUkfoE8egnLfeRZ3tqN8UMnljy5jst5Z+Op8haCMVguPPQbTpoHICrt7JGx55Cj/nHEcmfejSPS7kFuWABB39RcUQ7UU3v8Hn3SHkN55nwDw7Z8DaIE5ZKU3Yxh8lTOu6wj2GUJXawmJpYnUZM5guuhjEn3K+fO1h3AJ6KcxYRKIhMV5wa6huMgSkT3n4JgBVHYxHTYRJ7OOYJG2Uuf3LfNUtxf0F76lxDGHpIVJvLhF4DYXOcVo+jVY+4Sg9t779pK3OY+MNRm8EfISrYZivBflEeISzvONz/LTqJ8IGhqER5gPHTRxpnMWxqdfo8WYhcwthq2jvsIit+BQg9gAF2qi6G+dDrHQ6ScmtBEW/DaNNmkvFzLf4U3HIfy6gznnKaFwUDn+J3zAVYK9IoIuk5R6WwB1zy8nYogFXhbGYJHyI3pqAijMUPNT/06KlTW8ohIRNiqMMa+N4Y9tduxSBXKjHqfD62+OpbnHxKc7RzAyoZGDcWMY4+gjMPNhciXgaTHgfexDCo8u48LIC4xqLienwkRVTBXtPu34dvhyfuxNbro1M9fZT6f3TtT2V1GJl/Lbgxa2rXyL7++7iNi/lQuiHexud6dXe0cHxyGF3a88xpp3NrJ/mYjXTx/F6ZyCd5zgEI5u+ROdDtwiH6Xf2YzIcafdg1YO4kfRj8isMhaOXAiA0kPQQTr2zDHQm4klFr+2IFwN72H/9kUcUjlt4UPxq7tOY9IUCs099DrAAyemmj5mH5hNUVIR7n0mHGYZi1QN3K/px6IUI+3pQdWjpS51Fr5NZ6kPqCEs/E/o8KLMdoTQYmHD6rHuYbr/uYGTf0wkLqOcjGHFHJf74hHuwdrqtRw610rR6aKB+7BK9FjsVuAOJVvUpCieaXpm4O/9q/fTXdPNs83PEtTgi6bHi9DKOoyVEYhEsei9I/G/TQNzK/kWJyeeZtb1frIXSjiXJKHL4Ymyv4POmM1gg0WFa/BtfoG1M8KYcXAGOg8dR+Y0MqPWiJ+mFkeALx4T5UiUcm5svMHBhw8OtGWfbw0RPWOw/pyNy+caumO7+f2u35HrwFPjQ1ylhLu33k140wbgdQB2Ld8FCBuNmOkx3H3o7r/Nh7qyVh7e8DABrQE4/3K0b4MdcbPjqGptIWZwJYc2z2D1yaEo1yop8xmJR0spSYujySuQcNUFrmzIQ9LzDs7p3+DaWY9LVz1RK7MIlr2NoVdN+D/vx8fdyTXva5TvL8c93B1mKyluSSR2cCXqsfUU9wnzgEQuQSKXUJxcxQ+hzTid8PXlDUwcFAMIFb9WgxVjl1F4/yM9+K9mc9jwa/Nj6LtDuSm6ybBnR+HSWUf4rYM0xU+g3bcPz25P9s/Zz/RoN0yn2pCb+5Cb+4i9JTwPD88+R7Ymn+sB5UR8tmfg3GNOiBlzYj1PvPA14h+X4JvWhVvBNgzuATQkDiM4cCtRrg00BW7l2OxGYgOjUcYn09IF1QGvURH0DkMdk5F6dnBm0neM9l+H5OgtQoqbOF7fTp7kV5gEM5IexWa28fOyI/Q367Fr3Dn88EfsTeil1QKdNif6eCnH448S0rES904TNrsr4ZVudGuuYLSncGDWAawyYT51tnlzvsvGoZ2ZpB6Yjudjrmxa8yEAKYUpjLowioaUuzmSvJYA7wCa4r3IjzsEQHWcg+DxnzPTw5edFXcTXiLh0rkRWDVmJCJIcKvHGrkFhT6cHTMu8GbmFXrkXyPqFOEUCantR6Yd4dSEU7zFW0h17ag7a+n1jsCWJ+PpTKgIKGNPcBnPBL/HqQmnsEltOERuyK7vpfGyk6X2TPwqAjCsNXNwxkG0gVr6Nf0QW8Mkbz35LdPwNM1EbvuG8afHE10dTZe3HatBwv1F05m3LwVRaiyVQ+eTfvR9HDl5tAFtjCbz4SE8f1OOm+40ifoZxJfG0+XVxSLaEQM3U5yEFR/FaE4GYvFSebFt+jZ63HswqUzYnQJQPnjFYAq2FjDyxZEc/fA48f49LPGHf2qt2C12YstjiS8TdG/kfl08ctdvXD0wAZm3G9uub8MHoepVkWei3kPOsH41O/rD+Uxv4/WTrw88g+LaGqqHLOPg6JFYJrQzPmo+RZOLiLgeQftvzXgDmgoLcpGe+cmfML9GRWLZSLIrssnJzCGj3sY41xU8Lt/OMx455OmWU+sETVcDbh01lA1bRa9PBF7Rf3+v3IcF8a39IucvxiK5VYzdHoRQtyCYQuvAvGcKl4ZWYjEE4i7xJaj8DL71N/GvvYZH66NoE8fyY80OFGV1eIQ04V9zlTZ/A76NQnXued9mbqXeQmkJ5qJsDEFhvmiG6olqnU5wp4m5mjOEu5VxzemLCDF9C0XkVN/CJunlmhny3ZbjGfkdNQYNvbUdjNt1gnNjznFt8Fesuj4ZumWYeiYTmBHIiBdGMOThIXyru4Gh08CN4d2Yo54C4Grm74w78SJGlYUDc85gknfcvksnythQNj9whhafIkwqE/9M7sAVER/N2YvMPQ61ejcSjy7UvZByw52UG+t5+ad3kG1dz5zK+0h7U6hObg9Nx0XVwyQ3NUMUuYzyhOvdJpy9VuQmE1KbhPe7bOQZFbxileLbFoS8PxRJRAhtYZmobemEtj+AxlAGgLHbyJk3zuB0CCwCzV/18uf80cx46hg33B9F33uJQG0g2+65yIjSA2Q7tjA+bg8nuh9DGeOP5/XhA2BfoKKfL/2gRGXkSKcnYlEoicWJ6N30dHl18cynz2BQGSh89hukOndE1j5U3UUkXD9ES4gG9YwzLEjq4s3WJrSOPFZYhlM9s41bAQUktz5DlmE6E0fGsln0GQB2x78nTYgdSux2cA9zZ/w74zn96mk4VkXKuBTOjjuLXWzE4XByI24FUw9nk3kzE5PChlHVy+oZuykfd4Yc5diB86XcSiHwchZTRmyjRvkgxdFjmRJTx+XUavpcBP/oh9uxoKyrWagsgbhY24gtj6UiruKvViFzGnjMQ2BSs9udLP1tJi0hrmx8QPAdPBQaYo+Pp/tyBuq3hrH1vq1EF2uFdRYn/yXuTflBgSK6MrqS4N/mkObjwG6149ntSbdnN0mdz2GvfpMx2jzOJ6ejUlmYXh2Ke4uUaSdWc3OYiAvp69moUuP87CE6EhVcy76K1GrCq3cs1bEW1CYzTzfJCL6Sine0k6SqKZwbfY7uiGYyFZCmgE6v8WT/UI5vSywpJWeQG3toWx5PmQV6LG4cb0lg8keTcJtkRTlyCMWmGKyynwhsDmTy8cmIpgrviNJDScOlBhouNdA/8hrTkxoo2rSAlqlmelIcWBQWITtRISJVDoui+vkhpZA432Cujqyk2+UyAK+/9Ro+g8v5c8Uf/PZcAfPN37Jw43AujnDQ7ttORm4Ga0aXcDpoOdoJI/HVdxF6q4CO0HT0ni74arvZ6VBxWPcbBd57eUAvQ6qWYrld7T/I/wz1EjUH3XXcyGphmKvkr5gX67xAv/5+EiSu8AGIxCIm/XMSDpuDlT+8w4bEQwRIYVRYE2IvBfFz4ynbW8bVz64iygpnmO82jm2bTPbFvdjvH4HIKWXM2TE463W0PPkjVWoLpw+Ow+pVg0+ihrSWNIqTviGm9CXGXL6PxBUHGHdxDCpNIcv2LiO+XFg3GiImcssvlsGuUo4FukGAmOx+M099+hQdPh3EVMUwev5ZyuRj+HHmBxh6R/CgSkPTbR1CsUPM3T+P4FzTbkZNfHbgGXRKZHgoG5gbsxlb7tOYe0NwOAWWhbS8tIHjLFIbnRHpuPu7Uz78HAeG/ohZYeZS5y6kGT+glxkobhuLxEeBv1aOXRJMTsLrjNaMZd7Q33gjDL7UwVm9gnMzW2jyziHzBoids3ky8yL9lhLeGLWNRY1FRB2diE//NGrnHOFsCGzT/8p+yyT2P/k5Y0PupVLXwi35cebvdCekVI53UwGd5UvwjvPGO96bqElRWKRWPOIfwU/eT3nhz4il0BFuIS/yNAC+lUJF5h+VKvxqXmDX1rFcG3oNp8iKxCWbY1NzqQu9QYU/nDJCf3H/QF8U7yxGGqZl4cKzlOfG8o7LK5TsKqHpahN+D2cj6auhPRJu9pWh3zGCiacm0jG7gz5PAzp0hEnhZS94yP0aq6Y30Ng/BolMwq0Jpez0epopFzfRXjwJlyyhwt12RzYJR7MOL7xoiUjApW0oDgdE5WzHva2Cwglr8ZN1sjGsi0v7foECF6xPvAQocN6uCJLYJRjkpczdMxdpgAipspJBp4SEzDPTZDSGCgwtJvdmkiQuBJYfwqulhENjIUsptLtP3MJP2QWUek9FdK2NJZt92bEgi/K4awBcz7pOQXo9L0m/Y/y3KgIjp3J02lEAOv1VRKwYhdJDScwNIQlV5xfHydFfsKfSjZkn38IsDkMUX4/OuxMPnQdqo5rE0kSKr2bz1v6Z+Pv70pmRSX+vA5OrL+UTdvFGgInP9UbyhkcyMnY46WW/khe5gobQBi6OvIhd8haHtQ6cQKRHGeUBBTyy4RE0hit0eAfx+72ncENwfJxOJ6MP9hMdkMnv83eSoYRo07NUOF6gJlJLuuVVXPWu2KS2gcq+v8xmA7HNgsRmZvTn07lv9wJKE0uR2l1BLMapv0i/ygQI+5rUSVdxJlVwqw+iXGL4bs3PPLrhfhqjE3A6rzLrwCyaW6owXKlE+vpctINeIdgMcYWCtvtP2x/nSvQO/FoCkE4W4WoX6P0bcDC1GSa4JmJs1xLW5I81yMDJ2QpKg/fx8vZlZCdfROEYT05uFoZCL7xWaDg2+RhqoxdiRzXjT4+n9NqfUPUCHhEeANw82U3RnHUMSnqcy+Y91OgFCuensp/is6uf4dOVRuKfseyW2Hni4wYy+iT0NvdizGvEKg/CoPDkq6Xb2KKwMmLjRAJHuLFyzkpmxQna1IUlhcT+kkhxWDGXHy4ifq6N6hPVhA4PxS/Zj++zBR8qjjg2r9qMPVjwwaMmRrHixApe/fxHAr42kL9rAuOz1diftHNbJYXincXsWLyDQSsGMeWjKX+rJgLoCMug1zvib59VHKog8eI2TGpPRDiRWk1ILEYcNgVqH/XAcUMveRHlMpXhV4YD0ONpxemEx0of+5vkhNpXjUwl7DXNvWaarjdhcgQipA1DV8ggAMQSCFk+huZtJX9rj10qQSSXCrq/XUakCimT3p/E4ccPY+w0DhznHuaOe5g7p/eXUhj2OfU+V3lv3U0OrjqI8zkJYqcdfVgyl5Ye5aM68KktRsdFxu9Q8LH5NpBihx+4wNkCOw6fU2CZwNyoDdjlMXSEZVARfZx3dRAqtXB53l4yVSsw683YvG20ed8Ap2hAcNC/xR+d7QhOp1Du9xcOUXu2lpIvilD0D8eJBzEvzGP7UTeaAgXf9oX3X+DWgXNMuZ5KQNUlbHINnaHpcJvG84cHf2BI5W7SSru4e9vd7L5rN2ZZM/3KSp66/BnyvqHEZoDl5/3c33U/Pz7wI/W+G6kIeptEuSDn6HQ6mba9EqXRjcbgRkYcnE5tRC0VmgiCLD44POUUpBYwqHAQoQ2hvOoFoxxJ9Msama+BUyYzXnvgtUOvk7L+AOz0hZF/sChCy5ynTvO2ZTK6fceJrCvi02eL6HG9OTBOIpGIuFmCLI7D4eTdjPe4pB3BdpGwL3JqW4i/dBjXrnpaI1wZHgobqtNQ7RuJLrgDg5s//rXX8d2zCd3sr1l242OOpXtBaDmPVevIuixQ0et83PFY8QvTQ3VsXPcI7k45dY/4scnzn6z+fjUNURKeESm44JSTciCe0IZQjHONeHqNxK7L56p6K1KOAHC82ge/4njMfWZMKgs2iZWIzk8oGZbB8gUCAG23Q8Xo1WQeegcQJFDm75pP55VOuA+ip0bTUdKBwk0xUEX7l4nFUFP1MWWlLyK+IONjk4kLwHbgOJBt/zsF793YuApMq/+AQwc9WLr0Gf5vNrFYjLe39/+26/9Pg31r165lyJAh5Ofn/63h8+bNY/Xq1f9LG/cf+z/bnE4n/f392F1UqAeX8G5iJ2/WtKIJrsW9zw25oRuRw45EIsdPI2V+/SQ8z44mIyYbXVU57p5RnJp4Cs23d7PWoGHxG5tYHrONY6LTVGUsJPrmDoIqrvD+80J26Kx96ZgvBiB5zUlASwApRUK21s8rf8aTVN7p0VLnrOWdvmGs+m4uGzaAvVZw3lKNx4nStJBzIROLfjzp3zzAt99JELU9iE1qozFJSLM+d04odz6w184gswOvmgjebZVyHjNTcDL96EJ0HjoKsv0Zf7iGkIoG3rvHgUrvoK+14zbQd8c6TwxnxpTdhNqlBAMnJp3g4siLxHWvYrPyYQzNN4hzlKGzyzAFxTPkwJtwHUAQSE5ViHn4x8249uoxPtBPRdCbA+f+XCemW/kU36k/IUN6J0ioU+fQ5lvMZ08Xs3O6UKXlHS+862ofNU6nkygZGP076dF4ovZR43uXH817muF6MzkzXqXR+w9yMRImFxMlVwiaAIBufTzH8v/gaN1C0vIGk7qkjvPjqvDotBLWFIk1JBijysiGbb/gfuUmAc/6k5tRxtWs07yd8hNb9rzL2H45UQEd3Mi8gVt0Eh2lHbQXtyM2urFi0Gtgg9GyFJra3HEabUSMjaCyWoJafxaFQUfqma+wGtcNCDIDuBhcCTk4hV3aHEiUY/LvRi/q5Hr0E0htLii0cSQWgl9CLTlzTjIs7g9CTq7i+tDrDNHOpia6EYVZwYITscRUxrD8Hiuz94ygKzgV7hXTEjsad6+zlOwdh0tuPEPsQ7Db4dpnl6hX9uIuKsLFo4+uFi9kLTcxdI5CZhMhEotwOgXwbqrqNEFPlDK88i4Adi7fiaHDwK2UWxjUBh6860EAfBJ98Ir1QuOrob6klR0Tt+Kh8yC9fC44IeHij9hlSnKnvohDLCHRfQPztzxAi0SM++cBHE7fQkZuBl1DxGxY8ClDXewkvryJIz3j6aufiKiujpDSE9x8qwU3fuFIjxLZ2w+jzrbQEZpOd2ASHn6+3Jh7nQ8jqul3SFg7cT8+ziX/8u7fibAlFieSfuZPmqMVGCYN5tQrp4ieEk3i/MS/vQs+ST70t/XjsDmYvXs8fS56Di0O5b7JcXReryH2xu8Dx4bVhxGgDSTjyD/ocZlIm4+YgLbjBNeDu3oo99lyyG8MoblVidVgYciNoYidIlpvifHX/olUZmX3g1O45RtNisPIydtAn8s4H/aK/6At1MFBw3ZMMSVMigikq6prQCcmpCkQTb+GuvA6DBrJgHjxmNfHcO4tQasudGQopaWlxMbGIpEIWVcOmwOZVcal4Zfwc38PgAMPH+DmJsEJlXo4GVEj4b3QTlwTHEjkEmxyNR5tFUgCUzji/x2H0zdw86vF5PTJ0edWE39FqObKf/sA+azGOsZGRHQxlXIZKk8VoSNDSb07lZ5He9jOEuLe+ZkvEv6g2Xyd77gXXZ2O/C35zL6YxpL7WrmkHUnn161cqtxF4pFFrL6+mm9SvqHtVhsx02PwCPfgv5pdaye4M5i2KW2EjgjFbBYSHcQOO4EV58gd3MKJCQJVkdVpRpmZTGFNGAmXfkJp7Ieh+cwPbONytwulfZexy1X/do3nfBz8KLPTKwJNTzMOiRyDvIaU1DWs9YR8zUN4WIfiLocq60Xywzej9RSCKRd6o7jKORyYceJAUl1FQHUe1T13Nml2h4PudjsNu4WKDr1vPFGtp1ipv5fgU6koJ+qQRDnABiDi7KxIJPrv+cjVzJKuJ6ht+ojaqG4sMmGd+H61sDmOqInANOQ6Ue4P4KOfSIfbSeQWOQGtASya/AYPJNzkpg1SncsAiG1+lZQCPwbnTWPeU7vwaPKhs3E8PVoT8/PvpiPgHuru/iep/mfpsXfxXocPp+1tJFUZeOOLNzg0/RA3s69h9dLRczue78jMIp9E4q/8gv92N16bCof74YQBnnI4Bnj/3fuH4t0uBlzZdyMGUWE87c9qByhy1P1qZh8eR7KliO9y1uEaaCBOpyS0Ohq72M7WZd+gNqhBPIbCcY8TnajA3qCkIXEyoXUXwGCkJKkFpaGUxEIlgwsGIXpIzLLfl3FxxEWOTzlOfMOHlDV9hd/jh7hq7aLqeDo1z9XQmtFKu2879/9wPw22BhAYMtmzcg/xc+NpH6FjTo8v8RXxhDd5YGgwDOineN/rzQ6vL1FrOhl9YRQdCT3cerKBayk/8sLml6DdQZN3B0q1iS3v3IfsERvRrtFUIWhfiJxOjHH/4J3Ads4awWw3c1N6k5uv32RrwG5ufaJj+JLfuWhJ4emLetxUOiQ3W5h+cToFgwrY7G/DS3KQnJvPkfd5NIFrqpCXWEi8JNCrdIQO/rfnHYQKgP39MP9qKoNKLuGwjMen/BrepZcoG3EvdQ8pUVy+weUkETL/xxmXAMreYwO/96u7Qa93BKUh85H6G4jPKOJzz48xKU1MvvQ9hqQi3ENbqAtT8JXWhyM5PizYNY+8wXkojSqCU1oRB07lwW/3omhvZOaDfRxLOs71wKOkyuEbPxjR9igL5jiYmV6KyvtrmoKb6HXtRedSiGe3oJ9mM9kIGxnGzqU7abzcSGC3ldRbi/jg+Q9oDWrG6TyIWWnmrTfeIqztIdKUZp71g56PHqBJH0Xe28eovU1P69fqR+P6Rxm38BTXlpzjgXopBTe+o9lfAs4kghuFQNKSWgU3ZDaCbBq0cWMJKT2Je3slclMvQe7FdEqjmbdxIk5zEPgUMfjEXop8o6hPKOe0wc4H0RKmvJHH7u53kflDfVsiTuzkRi9j7m3KKplaJgSunFBxWACnfuiA04MuM0z1HBOKJ/LAjw9wcHYuFrUH0ybsYZD/GVbwHvd3PsasI8twaxnM1SljafHdx4yKMta3a5DszsWxfCRL/lxCUVIRNzOE9SHUs58ZFYM588dkGkcXEHVdyO43u2YwPfsXFF5ifi13ZfmXz9I3q4uaQToaFY2M7NXx46jJVJk/4SVlJ/UNETgUZu755R7qwus4P+Y8UVVRhDTGY1khJOSEjw4n+f4s9l43UZqwgejKaLKuZdEd0oi6z0LmzUwAStZc5Z8Zx3mmHf7o6+ZtTS/3/3A/5XHl1EbUcjPzBrUVb7FhwQz2lz9JZ8g/6C17HIfsK0AILAVJYGpeJmaLkk7XzSw7v4y33nhrIKDbHPkl0/eNJvvkBPTftlCfPB2dew//9IZpagkqhQ+7ip7BtbeaS7tkdCZXUzH9LEemC0GS5V6fAULQcfvC7Qx9PIu80a24ZJ7kRqcrXcbhmPRGHt3w6O035xNqBs9FH3fHZ06ZuIYrN84SWX4Gk0sfiYMU7O8Vk6xzQ2lzRxumoMv1KhlVfxLYXIqyLoVNs3chfu0tsnpfpaHvB3pde4nqvJd5LvCKF2i9Y5ly5VHUdSW0h7vjFEuAWuZqIaV3PN4qJ6fHXyF18CQkoUE0hzswK9qR98kJ1AbivE1lJlPfSaDSXMwiKFzHjuVNBEfcR0/XcYKagsi8IUKntWBvCSe0y5unn9rJXsVdOPPtA78VOyR05SbitSSSXkcNfUo9B2YeoNW/lVb/VibXxSDTu3E6ZRm50VIeuL6DkNo8ChJ3Uz1xOjkxa/FzphKvVfPcj89hmdFHddqrSMUWLjXYGBG6C4mtm8dbXDFLTWQ3G5h2eBq3Um7hE9RI9Nhc8kyPAdBT30NPfQ+7V+wmzORk3Qvwkz+84Q1tOFm6Zym/zfmN8v3lhM2OZkrUcfZvH4tRZcLaZyYlP5gJp+MBE8p/PMpT6bnkPfIpntYsHih8nsHn7NRG1BJWZaWlPpwdWjnjv5lAVWoLjQEtxJfHcy2rjKxrcGHUTfK7HyOoZx4OB9gdQpV+j3sPlhgfLvavYKh3Fr3Vb+MjDaWlLp8HzzzI+dHnKY8r59T4U0wftRSAiLw9qHu0FI99hDyfz3mjMRb7r55I+zbiGDuNhJKEAaabE3clsMu1C+ecF1CK0xgtf5l8qUADL/u2ibSKWdQuuEzBtXtIWFDF/F3z6df0s2nNJt50b+dQ4SFebFmCbv9wDGMv0RQr+GAdPh0oh+UR6N/ALscMvqncQ5fVjaxKKeEdqdx33xHMTtDZ5SAV0e3VjU3twNQj6CLlZF4nun4yMosBh82BudeMZ6QnszfNZvOcrWTrzdz3zE5eyHkMW38aLk6B/SKs7SHyRsSiV1xEGnySqKIZ6PQ9XMsW9qCBXb8yMy2X/SFNfPHbOAxHRjNpVwAVmyqw9lu5te0WthYf7DYJBecHU3S1lkeLZqDN0VL27S7mAZ9ufQOv9/7k9ZRWLvTnIvdcycVlhZQbL/GRZTEPuf8JQJG/FrNIqEYq0F1kUEEiQ86V83N2ObO3zCZjZcbAe6HWNSML9aenBwbVv4pTLPj8JhcfvLTFpB3/CJcIPyYndFKVdoLjJJNttpF+6CM84tz5dWkdXz3xFWK7mMXbF9OUZIXBd3zc0Go7E/oncHbsWZwiAazoCM9EGyKgjX/0Cmuti0cfEffvYk/xAkRSGRa1O3bpnff3sagvOH9KjTMMJFaQ2u6E/xrC+4leOBG1DzTFjUVm7kMbM4b5O8to921HpOvH6Xce63wF2xdtZ+0Xawd+u6DHnXPeg/ALc8UeHIZTkktw2WkuDHdDWgFh7dNI9l6Jmy/MePN3ksPu5tcVv/LMH8vxaAjh0vy9KAsSaUo3MDx3OBqDsEb7dPqwziJGaWxk3rkpFEWYUBkkqIwqPOtC6dZ50JntgX//VoIrvehTVPPEl09QF17H1nu2MubsGCR2CbwBnbe0ZBz5DoC64HQUzpFo+hpw7dXgMFsJqxITUxUzcE9/dMo54VOJ++jZDItIpcPcyDePfIOUWCK1kWQNP0uZRURdnwV3zlMRmIOrMZXa5BFEFF2i2C8ff20gUdVR4Gph0bVn+fzJzwf67eorHWS0+DBk9yTM8d04xBYUZgXyonhyi+Lxe9KfgyEf4Z7oICqhhEtdl1BYAjHLtczeNxu1XACchz89HJWnig82uiPu/YLwnXaCl8GrEgEE9/3Nl9S+VNRWoU+9swr5+R+rmCM305jdSOsXfzIYMKlcSDv1BF8+8SWDz41Gr9Lz+0O/c3X3VXa8uIPXI17n2IyDrAi2ELEkEr/xkUz+cDJSpRRzjxmxTIxEIaEgpICG0AZsjjtIuHesN18MeZlzc8Zy4+QQxp6YjKnPBLdlvNzDBD3Lgl8KSL07lZipd8YBQO8Xg2/tdZL+fJOah1cSOT6SzgohC0Rp6ObGrDcIKT5O+rEPMD7wJCLxHX2wuFs6YDhXRvXgYs6iJ3gimfA3Onzgb1SbRX8UcWTtEWQeahh1p0pXevt1cTgYAMsAArrnUTBmLkkzgrFbbHzo8yGpy1OZv3X+QEXfX/ZX8UC7uZE6P6EwxqKwgAwU/V2EFgv+uUuPiLBboM/v4gKQbf5vAylTzzhxd4ixTiwBVQzlgUJ873F3eNoTQmrAgY0xr45hU/Imegt6/3aeNd+toSu0AlNHPzHX9iLqSwAyaLvVRsOuG8iHJYNVQ8nLW/EPTcPkOpeJ+fXofc8zdJgAIFVk3Y1TJKLD9Qw69Y0794oDn/hwvpz7PA2hDaA/x5SMe5mr1HDguXcwlARQL8rBJhWelbrgt3nTE54SuZJfEcRxDwvVkbWYFF1cHn6Zx755jLjqKIpGfUBA4SH6R01l14Jd5KVVY5Hp+VwBIxX1PNy9nDcu1xLi7oFHiIO+9GKm+JSC0w4SJXrL+3x6pZj0CQv5Ln4T+sjTTDgezuCCOTSGGui6pwvPKE/6WvoQiUVUnq3mx2NJTBl3if3JNXwKOAtv4dol0Hy6nTByZRnc1woj7E7sDjsNKTPwr72OvL2Ztqo+NL4G9gfBoX7wio6mLHsFCqOObh830kY8iE4OZwPrUToCsYnaaQpp4t1XPufF99fy2Zp17HnzJlHVVrKvZWPpteA0yzBKSqgO+JDvW5W87gEXJ51i9/BTPB/6Cv4tPuhdnbhZRtLvE406+Pag6HTEXNuPzi+WYZ9KWFqxlNSCVGakCnuxaZ9OY9z6cZTvL+er+K+Y9d0sIsZGAPD7to9pKnuFC0472SY7LcD9wDkgm/+2ZQNHHGZGb3uBDz5cTkBAwH/nyP/zzW63U1FR8beY3/+b9j8N9p0/f55Lly4hl8v/9nlERARNTU3/yxr2H/u/wxwOO3Y3DfY+DfouN5pXbkBie40x4Y9wpFegdJp01ocsr2nEpFWwv6ofY5ue87NGUxD+CwqTgshKgY7s6pF0DEdHY3qxmu7A2TTFjaM9OAIQHKwzhjVIfTvwPKdjxak79Ivz9j/Fa4/9wAl7AxtqPGlvqOXSpcEcPQrOBAGUGK3fgnG2nFOyZ9H4u9ItkmBRw4GFOwGIcQqZQxZhf41ncxEynRMf/CgpHk5/z2gkU2VcHWVk8DULAY0W+mNbaa335+vrU5iS+zrez4VzNEVN/C0DZqWG8llbyfXuZt1766iP7cMyE2IqY9AGarEZf6bM6zva3QycW92Nn3UIWfl/79v28RdQi0cxPXoXRRXZ9JR18PCGh7ky7Ap56XkYnBJ0eCIxKhHLoLWwFd+aOrQpFwbOYXMIm4VRL40iZloMPwz/gVkTIvnh7hJ6+hQMbZNhVzt5MfoFnkPgul4VtpBJ6ceILvTGxalghuoZSrW7aPNrY0jAUqrPLaVD9Dz+tde5VVVCmuMnajP0uM4PwlDwKLGGnVQGz8A7YQK+Uin3b5pD/uB85N4ylvy5BKn0GNLQYg7MPsAkTcyA1qDqqfv5vf9Vpg06yBqdg8DPnsb9IVe4H3761kR0dwN670i6gpIRS8V/6yuXQE8i7zUQZG8k85d76HGpwq7Q4NJVRJu/gdKsKOKXfsLPIWYqHE52ieDuH8JQGSOpnAxL9o3BU2kiJ7aMxGvZmMb34qUtxktbzNHRYkbOrMEv/hbvObrwDKxnVs6bvHt0CIvfGELn6rdxhrng6tnLgq3DGHTITPtjrfz+8F7UPmoccYLQc4CkFYuDO3Q6/VaabzTz5PEnOTX+FKGPCnzrSnclT5Q/QXNOM3++VoBYVsXFUT/QkzhB0BFSuWOXKUk/+j4Aj334Fad8JtBr9WFwxAieGqnFq6sW14xJPPPT8ySMv8yShYco+BdRYs+WUmIPqfh4pYSPRDYOTThJQEQ2xpZM5CY9xpOXGXogC+sjjWjEdqJy6hmWqcakM1Gyq4Re6206WqcEOWokdgkOuwOz3kzOxhwUbgp8k3058cIJBt87mMR5ieCErsouLH0Wzo2/Rr+ynZKk70iyfkUU0/42njeG3MCt142OkMH4BvshK92BS083N7NgCbO5JziHTScn4XV2A7qiVfyxqollm0Pw1wp9a4nrYVfQKjr8oU2ST/SUaHR1OixqCXOrI0lZvIsDtwJpTgjEJRzC4z2Y8skUzv1Uww1pNV+Emlk15CzDy+6siePfHM/4N8cDwsJfWPh3Kj/3hEC+fFLQxFxQLdB5BQ0JGgD7FDoR019/lVtvHOR0x5s87Qa93hEUj1pNbJiYae/LuXLqHla+/DNvVwYz0vfVgXMrvDWYO/uJSCvjwexdPNkSQuryVGKmx7AhdQMA9XEJuNrncy38I+7XCpvFyx9f5tqX10ghi1+6PHG7u4GeOAdesUJG8p8L/qS9qJ2UZSmMff1O1ca/WkJZAjN3zWTfY/uQDpYKeiYewVgVrnhpixh1LoZzI+Ws2rIKR2Yt4jlyzC7e9PjFYJQ34jf7OBXlwXz/4zjk08/QkbGOfvcgcDqQm3oR+SgJD17Pmx9tYGn+OfIVAt+9U1TIRj2c6nXno+B8RnscR9s7H+feK4xvtPDzKiGYE6nQ8aY+kB+3TMV4TyOmsVMoCRiHSv4OOGH1ptUYh5RiHDaEW2MeQWozYZWrmVj+PiHuQVxo96XPZMMr38TSgqUcnLmL60mnsCgaKKjMpOv3WcjC+/AyjcYq6UJpDUbuv5UFLnA+sZbTUbVEOh8k/VwoxQnBlMWXwX5orvenZjrs6oPvj7qQWLKYkkwRflpo1ybwesNjPD5oMx4bL/NP7bcE7GxEKUok3K0CbW8UUs9xfNUYgqRXSpzSwa3kW3R6d/IPH3hSJSOlSshEFUvFRObvQ9nfSe6odO67+goN/duYnZeCMflfN4cO+qJbGBtbw+QFZ/iyU4LDOWvgW4ldQmeLNxfqBxNQeQpTrzsdIy3UK29wceRFrBoDC9076erS0uJXh5tGg7zfjx7/OEIXZLGrbiKtXoWkOoaTMfkalYPLmOf/AtVp8/AUr+aRch0auQ5MP/LI8Gvk1zkx6x+ir7qPUYZRuOnd8OrywtIlLMJbZ2xFLBMz/Jnh1KxspqTVlRslYSTZ1chellGQWoBJaWLBiAUUFjoZaoOXPvqKJ9uzuFVhQ2lVsmfSWabXz6RB1c4jvemkpWczLmMo877xZtPQTTTfaMYQ38Ir6jy8W8MgJ4oOe+0AEGH9ycaYh74iKqCU9b2vMfN7CU0pRqpmd3EhTqjCcAB1dj+ueEymfFglPnFq7I0qdH5xdIYMQtnfiUdLKZa+KOQud/zuph/yef74alY9tJd3xl1nnOg17AoVJhdvHGIJXZEyzgTvQBcNIGKHVUd15iLibmvgqvo6SLi1Dd0TNi62pxMud6PXTRjv2qh23luwh1luDrb/eDfu9SmMUAvz/6CCwRg87Nyon46rfB6W/s+IL+ino3IFNbUOcIe7XGCUSnhm1rbDNZMIlzAvfr3nd5xiYe7ITcslvjoFtxCBAydqchS9Tb34nBfmYJvUBmZhHlq5ZSUWuYXewCmMueDBhFWbOB3UQ4PYnS5DA656V0xKEyMujQBgc48Ce90iEkxPoy/9lJSi5NtzXBytQWLKO+ci6b2EwkVCS8woVL3teDcV0J0wmJuxowiXtRBcrEfiVGGP86E5NhCdh44hSjgbYqPK/VXe2voYKSn/Kkwv+BTvrXuPD/zLkWvkjHhuBNc3XOfseoE+aVSXP1dFYHb0YQ9VcnjaYarDK/Es2chrB2UkTFSh+fAhVGlSnCIlIlxwN2Tg1pgFje/wi3ET7sZW3BxG8kbEcjP+R3QeOjoX7ScmuY5t3QqsM47R43kvXioDaqMaVZ+OnW/MRzd3JR+MHkXeeAedoYtwIPh2Fz1/5O12ObN9sxnRqqH766UY7tYRVh8tVOsCaQXjGZQfirnPgrnXgaHDQMI9Q/ixQ0Nkz+/IO34kpjIaW4+RqccE7Z1LE1yJPOtOXtVE1P4NTC+KxufBQPo6w2nqb6IxtJHR50cjqzVyMWcajbUBNPieozLgCv0qAeg7HgxhUvB94Tc+aPMhpyKNG2l/VfUJ1ivqo86rA7eEEjLUctojhqJXFaAQgYtYqAKrjZsFceBwNCIy30KqlGKTCUGmv2gXC7cK63FZTj8pOd7EdU9ixbpfmLv9MmKFiLzBeaTlpwlrRv5eNr+wni9eWUtzTwR8BK2Rw3BI5bh1X2DJ2+uY8MyffDDqA6ZMAer+AtlFmH0WUe3tZIqtEKPrboxmB00hgp/g6LawWQ/njLBFLMZ3vIyymrsoDTDTramkyXsr7v2ZRBoX8FLMESTTZLQlPktJtZ6TgwX/Dx94/6X3WeslgJlfxX010Fd9Hlb8Q1tZrhpEuWY4fScOsubomtvfGrmZfhN/TTciCzjlTrIueuAUTeb4lOPURNTgrUvkRl0Ik7s0EGvnxlAhuCezyNDkpbDBacMZdotEi4yChCv8nvQVJqWJ5MbhA/f/jG8BzUFKilRBYASbQ86fRS9T0TmUcWlRpBTEM+zSYhjTxbCrw2jza+OdrkbmR2Ug9ZgszB0/5Q68U/oxTjbrQXQzGc8zI0h8VcfeX/diNViZ/NFkyuwy6vTR7Bx9ll9TrzO28zEmnIin3acdb49QPCo9CGrzp8y7B0ufEcmQdH73f5N+TT8Kswi9q57zvs24jLqIwc+VojAVpyesZ2jJJvR1RYzpUeNtzeMp/8+5VZaFU/Ycm9YI9KW3WmJQ2Py4xyOBKSdSwccHS7gB115XZFYZTrGTc2PPMWPYfQDYpXLsMgHoaVIUM9ungrwRp7nRMo+kXjlL/1g6MJbehtmM87zK28l53N9owK/zS5IbVtDjaSKwXvCvL9i6iW7Io6XOyIlJJ27rf0J8kQshVWd5LSaAEf0KVDoHs/bPoiSxhDa/Nn6KKGFfwZuMioSEz9dg04i5ereeOI/vWOwK+WY43B/PcnUOv+HOh2Yt4Uenk+iwkVScRE9gGG3p0/BNgsarjfw48kcmfzgZdZgrLU06iq6k4HfsCG2jlQTnSVlVeD85IzRUJLTSG/UNJ4NBPqmMS+nb4bbkqcoSBlf1HK12p1gShmu4K1EpUSRcT+CbpG+w9FvYcs82rHYDD741lWyXqfgk+vx9r+cU8W1jMtszd0BMOS6hmwZ06W+0j4B4Aex7RzSNG26fYem3kPAPP+K77wAIx/cdvwP2OZ0EVF3Eqm2lZPorJJ7fhN43GqdzMs3x4/FuKkBh0KHSCMH6I6l5HAvW8ZBCTq9PJHJxFPA6vhJ42svBubfeo7P3USaZfbHJVLSHZRBWm0N0+Ri6PbvJup6MaGILPb6RHBwigCJdDrjRkc4Qn1wOGcApciBLDyVnwlgaQ+4wEIxrmo71Yj1OXxMdC5/hYNCde2r13EOZ/iajyEAbNw5ASLZGRKRDzuH5O9ARyUjnTHrce7CLHUgcYlh1hvrYTuqkcwnIBnEnaCOzsSjdUBtOYwdm+VSRqPgah+Mx2kJ8aQnKR2lUIpba8YtpIKLdD7/CQRgD7GRfEmI+Fye5MfKEnsg6P7zcmzlzagSqaVb+WFlIm/shnvz8Sby6vcjPEvGC1cnhw1Poc9NzLesaPe49A/5Xp5cAChlahBJxh1hC6ZZc5kgiKUyZTFp+GsaYDq5mnUXZm4HeTU9M0HiKZVWM0o8jyTyXcMUgvDu9sUvseHWK8W5047UxlXwRp6dveA7NlUlEFlynOaKWhvjXOTp5MVJPPT8HOTjYD6ojR7iS/RlmhXmgv+fYIlk8+mf2+7ajjx1P1AUr0oBotAFayuLLGCN/hB6XEuyaO1X2IsS88P4LlMWX0br6Doibdm8akq9KGHSlnwbCWGXXsOl22antVxtjA56lPnk8Zya8iFv6RV6MbOOIXUnw0GDk/p5YWrtRGvsAFW+a/Tjy3BeEJ83BesmKXWLHaDLS9kIbg9NDedh9F8zK50LWGLyibwNZweB7yZcnDj9BQF0AT37xJCUBJYQ8FsJ7Lu8RNCQIf/VEHFYJi9f+yVmnGanmTmFISHYIj5c/TuPlxn+rJvrLzGpP9MGJA1V7w9YO49OL2aDVoulupM8zmPawDMQqOXaLnfgFKZTtFDRdDW4BdIQmU+5+BV+9N07nZDrKOpC7yHEL/nfduY4yoSJeqrmjPZ909lscPr5YDXO49fTPxKv6KQwXgL4hVbv+mgoQiUVkrMkgOCv4384LcPr105x/5zyyT4QEPqlV0G4zao0EVpy70yfaLs4dgXOO/2cg5agDRoqOMSjuZVRFLcw8b8ZlTAb2qTe5ZoLoC9kESjzg+Tt+zr/amXFn8GElNpMV144azHqh/9PuTcMclUTOVyqcEnDPjMZSVIVPXQ4d4Zk0DlpO+lPCOUyuAvB8JX78wHmjK6Pxae4hKCyG2lgFYmcS4CRQAunqfjb75mJSuVM0q4Ncm5CgohTB696QdymMU18vwvmUkT8Xb2b2vtnM2TcHh9pIlMbKTJmGc50+dNqEd6A6Og+ABVoYUXKUkOGXmPvL3ehiLHQ8Y+do2E4+9AT8xkHwTFraddTvasTLpZFbyXnUuJ8dmC98WxVY+604bA4+CRIkLRTuCkTiScyOLMbzdjWoJH0QlTpPXLvq8R0nZUjzO9zKeJW89DwWxwlx0JKR9yMz9SH3bmNI6CE8To0k/PIQDF8ZKAv7mjnb3VCFRvD9jlGUJpTSMesg/sYxJOXqeOjiQ/S69nIbB2WV5HWOThvD9pFXud/rRcRbv2CktZMRw2Cdu4nalI0UX8olwiGjv7WP1ZtWcmn4Jc6NG4/GHMOj9qOADJHVgltHNTq/OPQ39Mw5M4d9c/cxedjkgbHrru4e0Bn/i3KypaWFL794kXMO+8Dz+D0wnv/+8/mvz+kYiZwfvv+eV1599X9w9P/ZZjKZ/scH/f/J/qfBPofDgd1u/7fPGxsbcXV1/V/SqP/Y//l2J0ACdoud/hZvanRqLE5Q4iS3/wAHhgil5UMOrKekKom+qFhM944lYEIyipxKEprfpyRwOQCFWdeoEfWR4q8lWCxCajXSEj0Mt+o3WL9+PckTr/HdsBLa9T6MxRtoHrh+WI2Trq4UsnUNNO2fgyOxn9JTJ/CtcYMEoaE5bosxdF5mX/AhXiADm06ET10Zrx16DYPaQM6cyr/fn1hYFERiB1PLk6i/LEHylJi2IAmuva6462HfvAZsWSeZuOMR0ppO4f72SDpi7qUt1oRTZONU8vNMUIEsqgGTtxz/C14kX5r+1xWYwGquvLyBNB+4qu8n/tJPGNz80Y5bxqmw0XRqKhglXUXEtJsUJqbhcAqVQ+LbfO8+YhHPuVwk7sMXsCQqqAuoI7zoMFXhSmQWGclFyfQGdIPACIFMIyNyYiRNnocB+O6de5lt0ND+oJE+1z6+W/0dj1x/mlvfppH2YiHrts6l09WKNSqfh888zEfPfkTXtv1Ma7vA6Pu2UDfGyM/2V/D68xCdLkc5m+jPglpPVO1hGEe20OVchMRbjneXNxNOT0DfU4s92p+opBr6HCrGqHQ4cZAwLwHPaE9eKNrMDeNlVltfY7L6LJGjtajj07j5w00CqgppjRyG2GahMWkK0tt+XG0tyGQgUUhZOO4b7H0iFvXtZXHSMtoOf8+KIyv46d6f0DgtrPN1UKxT8kaxDxkjrNSHaYkvC8G30Iqiw4OgIcW4zr7IkZHXiRm2lJ0VB1iwaxZOqwPznnA8+luoGnkdsb2Z7z/owrOpmT2bQxi1ZBZupp2ctX3Guch3qJi/k1URaxl0zyBkGhmvd0+mV1JDjVVD6U0fomsFCoCle5dy8NGD3Nhwg+yr2ZS9V8bIb0fSdK2JC/+4gMpbRc/hXFIT/agIUw4E92rS55N2G+gDOKGXsCdlCfbe+TgckKz9jZqhFqa5yLD1l+ItNzBzy0gsXmGEcpNrQy6SdSURr1Pw7qnXqH3lM8JmnqfHmYKsrpdBpz7nr22Np5+Q7Tuo6CpKgw+985PZ98A+/O9JhxgQIaI8uYr8pBw+DHsC9zB3nu94HqlCSntJOxWHKwgbEwbAyBdGMnjVYGQaGbcGlWOlHZ92H5Bb8ZseTW3qLLybChn6j1R4CC4Pu8xvi79kedRaVCcb8NQFMOQqdE4K5N7LDbw7chiHZLHUqkZh0DhpCmrCoyeYL5/8FpO8hffzs+m7kIk1xsDKo/cA8PGkb3CYZUiAxNDFWNQaWg3vkGjzFWhTga4JOlb5Cz3gFDkHKvv+v5kTh5eeo095iz4acTp9yVyTScqyFP7p9s+B445L8zmfmElr32ESLm7HLlNScaERP9y56tLFIS3UWSW4RvtTMGEtqt5WktoPoFK18GD2dsY1QqnRgy+A4y8cp7epF8W4dF6at55Yn1x++XIBofUB8DzEz4nHYXNwedMV9ih1vOhewidvnGOxbQ1PsJySXQKdiVQl5evEr7nr57sYvOLvlUhRE6M4Ou8o5cpyTDYTLhIoGyEEt0TXbXi2ljH41hw0/RocRgsOgwmXzhY6wjJwWsSEvv4cT4y8QEq3JwaDA6cToWL2v5jB6kKwbTTlA584KLFAtUXFN+3vYNs+HmnALzg9LXjoPAaOilZ0YOwVYzEocNjsOJUqLCoVCrEdRAjt6rcik4sxuQmbIa/GAtryXGiXpHHynbdJFe1CddRJUEUsMquMyFoPBkt82BBVTsfKLawOH8+qFydiVnlyaG45MXL4UNgjcagfDtzSM/JCBH3qMFr9W9m+cDtdXl307R5Gc0UYbi5yEkuSqBjkwOSZDtpjRGufYqW4hgLrHh70hbwZ6/BUNXG2dhkXGhbx0AIfQup9mb1jDtp7rOxYtAOA6yb4fMts7r4+CPvrdsT9RswqN1ojhmL3mIG+5i7Wak5x62Ymlk4jqzavwqgycmRWBXUrz5MWXoRZDCVyGKTt49GvH+XGkBuojCokvRq04U5iRn3J+b54LN5pXEkUKtRcxfCtHzRqinDxHccHPUF45b+Ia1c9Tp+FzIhv5jkvEcedJSzNaGJTTxNGFAMUOs+lrWFwwAk+lH1MQKEYD4Mf3y5IYkP5Bp57Wdhgvf/C++x4ULhPu8VO7PRYfpv9G2adibaJrlwK+wOpLZVXY15g1wJhk561JosneZLzE08yNyUJf7dsXNs2M/nwQoK0QYS/PJ27mn6jwz2ADcHTmB4C6w9+Q21SFVH57shrvdlfNpeu6SfxOjwOceodh17f1MM2XQ2/96lxCTJxbcY5RJ5Z6NzLqFcImaFpCrhgEnFQ/i6F0/bg5fIt6piHqBYnE1J8jIBqgUavu/rhAT0QAHufBd9+F+KCu2iXOnGKxOij0tH6pyOy21DtCGNsyDraFd9TWxpOWeZFxPZhKPs7aYkeSb/fL6R7tPB88im2dUm43PsNXp1e9Ln0oe61kPfEKzRNvUB56XRUfR1UrnZwK3A/8RUj0AdGYZG20NJzDtH4dzkR78OCqEe464lkWvwV/D7/AKNVMFkN8lOPMrsiFMsiM07RnYzvvXftJdEh432VjIKtBfRp+wgaGkTVsSqKE4uZt3seCaUJON8SgnShDaH0WPQYa1R8cvoHyr0mYnRzJaTic5795ln2zd5HdVQ1SpOSU3GNBLwzC0mCjhOTTuKhcyekKQSDz0hELmGEG64zasu9VA3vp9sbxHYBIPYszefdhZ/ziP5Bjj++EXNvJovFH3J9lI02/zZ+E5ZeQpqexdesp69jKh61YqJuVpIzXKjAn3lwJr3mo7B2DbpaHbnf55L+QDqlDV+wdFgFondeoS3KgONFGVeHCbTTmRdaUZhHstO9hWGqfmwKT3TJE2mIm/jX6oAIJ79kbKRndA6PeqTTovWnPlx4hr50z2GLIY1ej0KcWU1M6XuEHQt3EFceh8aoxa8hEHFvD2IPI16Lj2Btv5tFH2ZQFjuYnXNe5nWdhUCLiu/FHihmHSQj4kWuzl3M2RQhU/xGVjV2z0dYoVHSWd3Fnwv+JGjpGCSO8fj0jgPFOHJnwMQhUHaxkaagMi6OvMgbb71BWQEw7jzZ17IR3QdHF0aSFyXo8kbUh9MTfZ4tR9SEV3XSpj5O9pUG2mZ70eXdhVoENSYXvij8lX5nOE1hz9DuJmS1ffjchzjEDh5we4e5AU/RgBfqAF/yIu6l0WcL5zrg6Q47HQ4r7sHf0Wt1xeWmkycOPcHlYZcZfmU4teG1yIeb4UmY/OFkRr44ktc2BBNxuYj2NiEYLnMqUWlkXB/Rh9pQTkRdAtZF+wGIia2gvno8+vpK2kVPc2VCE/d/Pw+Aow4XFJJ+nE41Uw7Go+6P4cqUekJKWvBzreeJ2H00V4fxcEco0kApDrEDnCLCcqqx4sR95hmen/wAb549wHnnTcqDhcD9U/5N+HYdwltair/PFdqA3j/38ULuC3z69KdY5QKY7ri9yRr6+FBM3SY8Jg/h3MofOXRsKT0ZGchj7Ojj7RTZzxBdFU2wNobmoGZe7Tcz4vvZJEzpJ7YiFofYwfEpx9ly7xYm5jewzrCHq38spPoBCdzGF6U2KVtWbqHDp4OfL0wi/9RQjj5poM9doGMdfF3JvC3rmfLqKSoDtEQ+/xNXzF/h/80l1B5G1kx6g2NVDyDCxkOeRqokTmzxGr58Rqg0tprgqvJzHrTPBcYTMy1mAOw7GS0EMXN7ZYzr1eCw2inbW4ZLgAtxM+M4nvg1m1nBL+vXA07MXnZ+XvEzOg8dj9S+jrjSxJcPnEAqc2NBlx1poA81ci1WuZW65FG49ceyotuNTY+s5/W2aHTXvyS5dBLRFSqUdhHismg87HU05XvQdQ18H5YytOIgfcoiSkJfQGb1RqXPxL/pOlKxhvoRi/jmmU/5V/sLcGpImfEvnzpos4Nj0mn2aPUs9nmIqrR5eDffQtXbRpLPBbK88/nHF3chc7jhUF9n0dlobo19lDPTfqHV7WvGDapFMjKfOtEmbvULwW//Fv/bDDItHFt8gRsZx5ha8S7DDw6hy6sLlVHFwp0L+WPxm4hE68lLy8NFHoJV5s5Wq5kdpUqUtkDWer9FR8eLbHj3aZjchWelASlRaAO1SFTqAZ/XPcyduFlxVBysIO6tLF4+ew/X2wIxTz7BXd4Lie0bD02X6Gu+nz73jTzVPA35hOu8Je1kgtPJ3b89R7+ykpoUP0L2jqAcKFs+At/kyYh0zXwz5BsAstdmc9+y+9AMLiMmaTV3d89m64h9WA1WnAolR2ZKaXP5GgvDSBamTR7FSfyFMNzFFg4n/MpDN15g45AP6PzdG0fpbpx3P41cJ+Fmch4nJp1AbpHz4rgXhRlZp2PIwc8BEHloUDY+h1WsxkHAQHyjeNQaRA47F1w/4XId1Ntuz+VyGZVZywdGepMfzHWBdd4WMoocOCUu5E19AYDWqGH0iYXgq2uvHKfVhlN0JwanFMGKvlwWWSZh3ZJIaIiayuRrnE69E3xXWALoKaomqOIyNmMGCqfH356/tNw06nYeQz84BnDDq6mQqNxd2H26mbXiCM8NKWNlrQ1xjZXxp8cLQB9QVfglft7b+Gieiq865uB6ZhXJeQLTg102hOFeP7JS9TlnNtfQNW8/vy59H7OshZiKMNTF8Xx9125+nFmIetEFjh97FZ17D+fGnMPusYrty7fxm6s7AR5xFD39KaPV7+EwCnObyilcf47rRbZYTXQ+sQl/7584kXgCqVXKzEMzAdi24ixfAAFj4siZ8So4nWQefheJ3UZo2zzqEv2Jc3ehy0uPZ7cnXl1eEOJk4c6FDFt0Cl3Ld5gak1m+bTlGlZGK+DYybvhRmunCElc9vQ440glDrg5jR8gOTMZ3efrSeB665yRypZ48M7QGj6LVrxOpSMueV9/nuLuaOs/H+b5fRH5MGdkeCjKvgGfkYDY+LGizv2hpZHrSJPJ0Snr2f8nq3avJS8tDbVRjVBnxFAkJAcU7i9HmaNHohAfO7N/Gto1zGCGRwqdg32zn1PECAvsmIU+9xT8i2vlDspRX3/4CuRwSPlnNyZeOYVJLCS27QWZDJM8tPM3jqn4+TPuQkpASxFYxLitcuGAu52c9HDeAa8leBqlGobPpKLeXc77+PDaHDbvETq9rL24eAogWMzWGkl0l1CEke4TENJKXUMOU/1LY4h3rjXfsf5viLqzwIBpdEzWT1+D/r9s+kYiomztRGrooHrWaukGzkblDr1Y3APRdnL2MMylpWBVGQhpCEPe54nROZkPKBmJnxrJ0j5A8kftjLg6bg8w1maTdm4bCTYEjNZ3zAg6FTa7GKVMKCcmtOhQBf4+B+9bdwJijRrI6idkbhbil1WCldE8p3vHeBGUK1ct+yX4kLUyiTCkGPXh2e7Jm0xpaFC10BY1D0d+Fa3cDZVeaGSeB7H/H5/5m2cAICeTUHsSbBCxyGzapjWFKIUkq+WYGGqcHTdeacDnlgsxdNuAfAJwfc57hjY+hCvQgd8YrREQIn8s1cpQ+ciF+IRahL6hFbrXjW3uFpugwJA41DodQLSqxmnD8l/jG1KNT8aQFEDGq9AoAWo/dPNMB81xgzSs/80DTQuQoB37T54DEWrArG1HN28WsqGehS9D/89B5kH/3n0x3ddAbX0fOiDNEdm1h3cfrUFgU7Jy/k8JBhbR67CVFrKRg9mGkHhnInD7U2WBrdyZ3jxAAZqfBiHdTAcY6bxwBwn5E566j07uTY7N7eW+QPw67A6WnElO3iUnfTGPpsXlk56YT3ByD82Un4gA/OsL96AjPJGukleuSOdicMK4RbE47ftVXEDvstMSMRJMRydORtaz/7GVsLV7oDG1UhnzPzfRxhGo1TDo5iTFXJ+Pyjzc43anHpzGJwJYAAlsC6QgdjHdMH8leZ3jK3cAz7QaQiKiMr6WbUjxtcMEIQ7pOcir8Ip8f/xaJVMbp8edoCKlFaejl0S9HcbliI9OnPo7d2w+9dwQu3Q107LSTUZ/Bvrn7BDp74MybZyjdXYpLoAtTPp5C+BiBIe77TZsYK5WR/S/Yz17gqf/nx3PA7rMa+Wzbtv/rwb7/nSb+Hx/yd5syZQqfffbZwN8ikYi+vj7eeOMNZsyY8d//4X/sP/bfMafFys2kIl5UN+O3bR5+OWCobSS1IBWFScGP9/3I9axKvhCb+cb3GU617KBd+goNrmswqA2Uvfsevz96lPDhBfy86mccZgdpxz5E0/w9jYHCQtMns2AMeYaeuJV0JTr58b4fB65vl8g567mPnqhWBq39BWechtbDN5B376NbIwRj8t0WMrWlHq3bV/T066l88gsiCg8gcUhw7XMloj3ub/fU4yf8LYvXMGrsMVxGq5G7KakO+Ijjk49zceQlXHoVdHt20xDjSWPSFJxOuB4zC2PfUsbs+pmX33uZKRp487Vf0Y6uoTtDjEUmBKVuM13wsZudBfmZZFV6Y1W4YnT1x6p2J7DJh4jaCGocbtxTncRW+01c4zz58skvuZkpVAqt9bSTxhGSRpXiGZdJ4rxESoevQuclQmFWcNfeu3CcFrL5C7YWsHPpTgIzA9ke3YaoAvbHFlOaUkNfQw8Pb3iY4KZgXBeH0+vSyyOSNrwnXaZtSD2y1ChOTjiJRW5Bd+QmHTfVuLeqGePfh0PuJLimi+Rboci1XVT7Gjh338+8lvoWL0yLpFldx+VhQrCTm0YkVa0oVGZkIjuP715A8BkFfsl+pC5LpUlxiw63k3TZmrgo+Yzt7t/ikhhKn7YPtb4Vz5YS/GqvDYyR0QhPPAEPPwwdve28X5WOxq0f47BcJCGutEX1sm/2Pnpde5myRc2Lb67gqZMJDN54H84SI78t+55jk4/htFto9e/k7Zh8lvlYeDSsD625nsJBN1i/fj2mtwIB+F0v4hF3GK6yo64vIzp3J4NOfY69W4+0sJHUjrd4KK4DfUYhCm8VE96ZwOh1o+mX1mFQVvGH/Ruu7XiW7KN3aC6sBisWlQWb1EZvqVCVoW/SU7qnFH2DHrGLirgSBWPOjSH1ogir0QoiEZLbNBsHZ55jmaSVpn4BUNOeq2DMzq1EVJhxnC3AbpWy1Xs4WccmE3LLB13VF0w4HsbG1W8NtCFWY+HXAPAQ9RNSegoAUaTgJJzdJVR7BbVeRVqWh3uoO8sPLSdi+jAm57XymLkR0W2dKbtDEOlWe6uRu8gJHhrMa5bXGPGsUK1x/r3zXHz/IhKZhNDeBbjp3Xj868dxO2VEJAKv5iJcu+poP1DOmbFnyB+cT6fbKbpsjeya9RN/LvqT45OO06ex02C3stOji4KJZxD7K/Fsh2sjdRSOniUE3UQwVAkuItHAxv3066dxiqDMCq8dG8HU7hyi9zqovp6P3WonY7WQ7RtWpWOpNoIHdw7B70A2t+M4vCl6c+DflU+u/Ns8aGzrxVW7j1GHChn+836cduHCUoWUxPmJ2JROvnrsK1KrYll5IRB9Zx/9HsGYXHwGzrF5wXau50cRVyL0PyIRUosR76dSWPnobqwWKdk7v+T+ax9Rf7Ge3O9zAVBGuRHrI/y/XmKmSyFsCKImRTHzm5mUv3KEvU8dZIRbO1dNd4JTf5nlNmVY7g+5/3Zffil+NMc2s2DnAsq3luNwgMzYg2/tdZoSJmJVaEjPCeOLtd9iXOaHIaeYhMtbSLj0E8E1AlickCpH9u7H9IwQ495YxJADbzLkwJsElRwgtXEjxj4V3//8EuKmBjxaStF0Nw6IhYOY702R1Iy5SEVyIzVL5Hyx9ouB9kUpukgcXM1vz3yBfEgIDn0fiv4unLermj97+jNEq+JwOpxITX1IrCZ6vNzoDOtnwaO7WOJ6u9p2ajZvv/42Xd5dzN+1gOjtc3mwz5+aqBqcLlIMLhrqA89RHryeiybIroefrsRw8etF1Ntz+W71dxSmFuKh80BhVtDn0sfJkhAk+UmYJgxh/fr13Ir9gk5fJ52x6ag87CiMUuLK4hB12ajz2YEl+HvGRvxGoEsVYlMDH6TdQj/rOIYg4R2TWb3R2qA1qoa8zBuIRCLExbfwbcgbAFAVEgPh6RrSX2oiOWskdokdu8ROSOdKJL+P4ukPFxKa68aBCj8cDiei2wKTHT4dlCSWEOXdyPPJbbj46v6W1GNwwMRG+LZ8BZ8/+SzmncPpCE0THtMdO3B+P4MN76yiRxxEfC282gntOgNdmku0eOzlp05vPqqYh0Os4KE3X2X4HmEetNgttPi3cDP9Jka1cSCDdeWJlSzZvYSkRYL+pLRKijSulraINnzUPpxaeYqf1T8PtO+tqF78br1PQu8SBt+MIkgrbNC7jt7k8JaZNFaF46uux+mEHZu/4brxCAcePUGHr4Hjk47zfUoOXz/6NeJsVzauEYI2XesbGfHRk5h+m80CxVkO3n0VS4IOsV2E2C4GJ9x3/ivKRF8yQ9PEJz4gQXj3pBYDNrmGjpDBNCRNwSXo78l0vg+l4/3Op+RbHJzvkeGwOXAoOsmNvIdu9SmSbhSRsUfBzTMZHPppFo4mA3Kjju6ARDTdjWxPy2VN1lG++Hkitc+9hPJKD09++SSxFbF4dAnrw59mB1VDl1Iy6XGcaiVih5iwOg3+dafJuFCOrP11tqfoGJpeidRNgdwiQW6RU2GFmU1wPSUfU8FUwkvMdFV1MPHkBMLqhLUGJzgdDpxOJz31PdRfqCd8bDhNyyI5OvU0NZE1lCQLSVk7Fx5CY9Agsdq5OfUlLsmSuBL8NKXBL2P3knAl+wptfm0UpVTRe99uFsunYpcqcEpl9Hj00OfSh11sx9dWxkOKRfye9RMz7jtAYMJgQnPewLOlFIBdC09il3YzWdlPWayNReKROBwMgJR7bkvt6rtdCT9uwVJyHltjC17NRYgcAsg7uGAw9jItxi4jvdpeJn84mbHrx3FL60FTqweJQ0roDO/7W6Z1cZoQ9JC49bFpzSbqJxu4GDWFA0NEnE1OIcYrh33LJKz1a0WEQAf1rwHfOXvnsGDPI6id/rdnPBmt/oJ+rkXajEXTj913F2s60ljRCg4RiFReqGx3EjMaDGU0udk5N+Q6Fl8FVwaARqgPPk1LUCdIJFzId6XbP57m388x5MCb+NTdoW4CqIrVonfT8/yHz2ORWbBJbLyx9AKjPvoIfNS4mJKIanmG8Np4JEkVrFt0jNYxFzkw6wjSPvBv9UdmFagnRzbC0pp4GhvHY+mK/FvSTL9LP0a1EbXIzD+ys1ie9BEOm5O7fnNnytEpQj/YVfQa23jH930mmz6j10vDudHnCGkUUKqIugj89gmJOV7RXoRkh+BecB6Azm4XnnvkTUIv70EqkiEjkW13b+PPe818HJ2PWxVs9PonbSPmUpb7J1OPDEFmauPyBC/+XPQnUYFtvOw7BnFNIVKnFzapjbKg12kLH0J0Vg3tXUH89Nb9DDqh4NV3X2Xs2bE4RXbUlkg0ligK7W6sbQjmgOuXRJc6yL4i5FCv84S74o6xp+Fn3n32PVp3XwIPF1r9W3GKnKj71SQWJyJvEzrLlPr/Ye+/w6uqvrVv/LN7dk12eu89JBBCCL33KoKIooCCInbsvXztBTsqKipFkKKAdJBeQmgJIb333nb2zt7Z/ffHQtDzfc77nvO73vf543nPuC4uYK8211pzzTnmGOO+76Hkb8pn++tFmDOS6VZrca6/Tt/xw3SEOjk36hwbl26kT6tl5skp3OXUknE1A1GDjV+W5rBupTCWBbg90YwIo8HXH+1MT8J1o3nkq0cYcW4EMZUxLN24lAebR1AUXkPXyBxELjc+HT7IrXIiSgXKY0ujL5/3pDOjSUAaBFYcRdS2nWS/czw57H7Eji4emHIFXvoafUISw6oqSavbylIt7Dw/Ddceodo+NCuU1fWryVi/CpfEzZlQeHxaHp+t/gxZoifPtj+LUq/kuyHf3ewvheYYhmdn4hZBVUwVXT5d+E0fgu/T69F59/LFyw+SuDOTc44vMXsIWpk2uYTH0u7jxZgf+KIHiqwqUq77MfZQL6GVFZzNqmTXivXkiVJ5bPgBvl35LW67m4FXdGjbK/g1EMrjOjGItKS9UEjP1CCc/E3sDbh//f3YN/TwHy1IZsFfAvlWcONEqpLTHZpG9aB5FIxewOC095kdepyGugBELV40R3dRmK6hW1eHwUdMVUwV8z1drA6y4paAyHWLztWl6WPs7Sf4PNDMbDUYAtx88NwHXBx6kYbQBooX7mJofBMjRIc5+sBJlJOuE9weTWz1DEa1vs1a91KS5DVcFcVwfvh5GqMk9NzzIF8+9iUdvh1EFJ1AX32VvvY+NIFamqWh1JysIf+ls0jqg9jj08yhkWcR+6mwZA3mx4fyOJ/2CMP2OMg5NIx3LkZw+fRArA1WdAYZ+m497ht6UNfSrrF7wmJeHDWfaxtu0dmEDA3BEtHApCHluJxieutPcfb9szRcaKBw+HKMejFPZ1WzLKiM2PJY5v0+j56KHlLORnPX9cFUp16mNPBDrltmcd09C0VqHAqdgg+efJ+jU47iFruxelhxyW6M2/3CuOGUynH39JFyXc+nq9/j6ghhzeBTl0tA9QUcHhqsQKwMxuSlM2vPaJwW2z/eteLGuLbvygQGtt2BqK6GlJNribj2B2KnA5tuIMaIURxcGIQ9IJSI6wd5eK1AK/ygJxRHwmrpZcIKfPBr0uCq6WH06dF4dXsJ71tswzBdxNkZw0CrRVV8heFXbmkoeRo8EVfYcFgcDNn3JtG5QiHUpll/sjKwlIN90O2UEN2ZwOizo6mJqAEg5upvOMwmcn8fAafFuBVKrEpP6lKmIQsN4cvR91OFk3pNDy2iciyKGlziftr92jk66SjNQc18b3LxtclJt7+Lbx/6Fo1JQ/x1M+puLzo0VuwyCR2eBpweEF/kTVRVFKfv+J3g5Ttpcvuxw6lim08jTrXQvx1SB2sfXssfd4oI7l0l3KBIJNCrSqTE3yNQTFemDqAtZgoyvRYRYipiK/Dr8EOMk0mLjuK2Syk5E09fRzvnR5znUuYl8gdW0BLWy72757Fifyp/rJ9FT7iLNU+toTShlNgyDeaLA9nYrCL+XCBfXYhB4lIS0nU3wV13UySzMK91LBXyx3inQ88WozCXH5nVzZ+T/mSEBzzhBSpXG7GHJ5L0+UqcfXbUfWpiK274u3IbvocFhofyfeWcfe8sHqZO/rhTwqfLfuRYUgGlcQI6bV3rOopiPsEu7SRt/XhOvPIIbw78kq1bhDFIqlVSO3AubdFTsN+zAOdtGj6t1mI3yhCLhPCwS+JCGiYlfd90XroSgnX9HSh3aPgy7ks23rWR29feTuSiSN544w16db388MAPZC0W5qqFvy1kyAdDOMVJfuYrJv1Uwvq34NP3P6GlRaAZP3HgBHMz5hL5SOS/jYGCuf+JDgA6yzpRtdXQHSz49l6tgv/mdgsatQA2hQYXJrQmJRKHhLl75jL0rA23G0a9NIqUhSk3z3fhswtkrxHiTCGZIUx6bxIeobeSj2XDl9A5bCYKrYJBm54mf6gXYpcCsVt4DyElx+k7eekfbbR0Wfh98e/kb86/+duARQO4Y8cdoBf6ap+6j6OTjqIdrqXXP47yrHtoHn4bNc46ltj5L9lKB1Re+R2LLoBt9+SRPzCfJ9rhjmbYcvcWcpbXkb85n+C1wSisin873v0fRIs3XtvImNfHcHDEWwSVn8Lthtjn51M9cA4npnVzdJA/tu45nP9akDZJPvUNCRc2/+Mcp8eVYLk9kubsUgYcX4O67QI1/gLDwK9G2GMS1syB11UMviLETVxAiR3KtQbyB+aDn4yJf04kqiaKLu8ucsQWPukaz9xm2NsHbosDhU24n/m/zydDAVExHzJVeZ4/FlxFNqCFsPxAFvz2AGdaZoC5AQzF6KLu4N7v3qZgxB7Sz+lZ9tMyPlv9GZuWbKJXlQeAWCLm+a7ned39OhEzY7g4sYZn26JJu5SC2+VGLAZlbythBQexVDRxtOo+OuoCWJWzEG2vFt+Ga4SW/IncYiDXrGRRM1xLS6Bg2tMYs+t56tOnGJw7GEW/8B2GDaxlzLmRpF8Ip22qN8fHH6ctwEFL0hjeWHEvCeq9rKzx4FyTDqfdSf64Fo5NOsZOE1y2grZpO0XnklAdvUK/oZ8zo89RFVOFXWanIraCdh9hPPirUFpqt6DwUnBg+gHu3XgvyteE4oGCLQW0XmvFN8GX1LtuZdb3bNnC0v9AJWsA/quknAGAwWD4L+79P/b/hv23kX1r1qxh6tSpJCcn09/fz9133015eTm+vr5s3br1/402/o/9H2h/oV1UKhWuTiNZF7NYFtlDcf5AmjycKE0W5h+ZzxePfUFdRB11EZtR9anwb/Onu7WVafti6dZrWb9iPdZLaXx3KZoJ444T5tWB2yKhOSqJbh8Dp8ac4tQ4oQL0ct0ALpaGUDthAClNu+j1zkbX1UFVSgyR4iru7I2gSX+Jt3USej5SsanyLZIULp7xgmTTPl6+OBb+HIXptl70k4eQW6NHX7cds9JE4MwlN+8LwLtRoAQqCNKzKqEVf3kFLg8RD3z3AG6Rm8NTz7P8x4VMWNhO2v0rGH7qfaYV+fLop1OQOoXPUm6Xk7J/OmljDmKyK5iSeAd/LCpAZY3EpDrCjJj3+cGeinbTDCSBflRlLABA43Ixc/doNHojXredQ969F70hCDfL/vEOzlqkVCqeZZfv0wwY4os2GEw+WmweLixKC0dmtfHpPYKzXLKrhOarzTRfbUb5jBw0QlWQxh6Jy23Hq8eLmQdm0nOgiDVvrAHgaEo2oY4xGPTHOeNzBolTTd0DcvpzchnfYcXY7cvYQBc7Fh2lV3mVlMqhRF0YTXaUJ1+f9cR1dDT982rJTc/FqDUyImku/RUVdMrkWF1yCi+moEu65RH5ifswR8FZ9x4OMJUelQGDLYqpL4/hkzND8GotQ9dRTVzOL1i6bsfouKWLULL5MuLPJzJzoojs0adZ6nLS5+/mqv9VJA4JLpEb78YQegflcmD6AWaGPwptEFMZg6ZXRfU9ywgwPMgd5Vn09ymZMcBKZHUkXd5dWNJkvP/GG0RJYe2haYRdyqR+QiyeOQISzNHUTkNNBA6HhLcDoc35H5MpQqd60OM9pKta+P5GQq1oZxHxs+JZnb4am8NG9grBWY2dJiwKjM1GXCYLXfouvLu8SSyV4bQ68au5QnPsKFwSGS1Bv+MphmHNZ7DZeqkUmymPr6Ek5iMiVCspdFnJUe8n6gGQegym0pKNZ5uI+LJ4Lj/cSbZ4C0E2O90vLUeR5IXcYsCs9Ue3fBn2H7/C01fQONOuPkOeaz5yjZy46XG0ngWFQ9Do9u7Qk1wVg1NhwtHvoLO8E02A5t+Euvt7+jG3m3G73czbnE6HRsWpMaeIipuCubEbqU2gHOvcW0ZccBy56bnctus2lEP6sYZZSSpOIrUglUPjXqQ5YzOv90CPC35wu5n9eyAyRxjXRnTy3Pd3EhPbwPOzd1EYl8Na2RLeFN3SuvTUeiK1tvJKTQhRdUGojHbKD5TfpNsMbgpmW5cfQwtiGVgZ8x/XKQCU7Stj/sr5iMW36m2MVR2MOKWhw6efulgHiODiD9c4+MBuJnw8ne8GX6fD0cHI8kwuZ2diNdlojNAy8NxZ/B6aymteK/gozIz0jzHUNvjTN7mZlBM/InE56FijZl39Kpa+/S2qIgMSr0pqT90SMjf8eIo3f3yDZ77+iPLb97GrN4z1N7Y5bU6Sq1/FGT0PidTF19/OwjteDctg6cmlHH/pOEU7igCImhD1b/d6qOIQJrOJgNYATK0mXC4ILf4Tn6YCmmNH0RQ3ln6NDzOurmFkGrSENmHSh6LpbkDXWQNiEZ+NPkCfCx4wuvGquXLz3MGVwr9PNMTRfdyBNLOM6CvnMfjH0RgSy6e+sNKziedE3vwwcgvpliGE/G1xcyAYpqtzGVSjpdFhxOVyoTh1hNTy65TecyuY7nS5sBhsDPpzDe1hgyjNyGL98o8wuDR4/fYjsVlZeHjfom45OvkIk49OIdApZ6lWWNBfHzGck6nLhOftgotWEJcFM+1aCrKhJppChMRGQmkCs/fNxrxqI98u3MlO906ecrWgtEZgUdRi9jAxf9LXpMWVcu1iOhFb76Z7noUe3U+clJVzunkM96c/i8kQRnJoG0cnN+JqmiPQdCXXsmBYJ4/PySX5uhSJXIIoMoL6pMlE5u/FKerng6+f4WzTFL4v+I5374bN92wBkYtJeV+RWLwOCELkFpFSlAI7PFj76FoA5FY5q3QiErUWYos8cPXruKtwNinFCjYt/hy73M5xC9g6R6DJ+g5zgAq7yoRXXT/6bg+Urd44gR5pBDF7ppN6PZW6d/IZduA3Lgy7wDexgnbPB4ykOq2AHl8tHeXdJOQm88viXzCrzKTmpwp98EbtzdrktWiDtFyc18j0oJN8Hg6DKqwY6gycjhZocyQqCdaHv0ae2Eba+i+gLpqyjGbEzl7kNjmpDXp6faMY4L2JwUc7uBg2kgnHJ+DT5YPJ04zGoOLkyE6UWjP9WjMOmQODp4E/Zv/Bgva7sF3ooyayhkJXLJcNF7E4xQz6LYiJea/x/gvf0NHwCPIRMLDJTcsPq1HO6kPcVUZ89jFUxjZqUmfSETEEj1usW9hs8POnMZiU1xC3PcFLp8bRF9tJi/U1Yq83cCVjPlmBX6NtqeK8l5k9yzYwP3It4YWHUBoF3chVX9/O5sVmfvZpIGrwFYLDJnNm1BkhWHuDztgYWc2IVB05Rn8KjCLUoU5+WvYTY3LvYlxEMX6aVu5tgStWWO5y8eu9l+nSCv6WrwQiu39hzauf8tnFbVj7bYw+OxqL0kJdRB3jTo5j3Kk02u9sJ3DBaKxHXBQfbyGk0pdHfnucj575F2UpwvjYr2hj76y9+PfeS5/ka/piD7OiqpsKkx7rgDAOTT+Er2Ey+p5K7jybxIihe7n4xn6O9c5GZpVxYMYB7DI7z38o4crFAfgPbaEueAQapmEzXwV0lCf0MSmrkGvBcEWuZNP+F7E1zcRDWcXwS0Y6pwXyJS181x5MaYSJQQ9t57rjSaKip3LBcwr9qj0AtPu249fhh9Pu5OKXF5HIJCTMTyLmeiobPU10z/+dEttsoq738tDWhzgy5QjlA6Q898AbvK6EXSb41WwkqE5MlDGLi0Mvkuv9NTsNKh6jn9FVmWwL62DUHwX4Js4kb1Aeg3MHA8380ZjG9SMLKYn3IOPUk8gcMk6OPckDaz/CYNdxrnIUi3YtQp7VR9f8lTS31JMmh8/9oNV9iYmK4ZS1h+HpFUJklSe9OhHNwc2InWJ65dk4HZNpN3rQ6xuNvrUUl0hMi/dx6ry3EFczBVPHUPo8Srl914sAnJ+9n0G+IjbbrVxwWxlubUbWdRBfu4yE2t+QXt3J+rQSFmUVsqe3kSYmsmHIh/+YPyRAYs5GJA4r4ytj8G9K5MvHvyRKCnM04CPqYcufr1GeE8Cg8G7cN/R0Jiohq28KiOX8dCwNcXYGse+p+Nx6HN/2fPzb/DGrzIzSCjSWdoudmpM14HRSG2liz+wfmfTnJGQiDyxdFqbv9sSjbzVmrZi2OzT0enWR5/sVfqIEIsIsHNVtpdOnE71TTFFEEWcsaup/WEbLsM2UD55CTsJktOZUpL2HyTmYwIfBY0lIHoIqREKpfjctgS1IRHbuTXuJeYkfc871E02bhjOrcACdvm4k9uHkDMshqx7uTU/nov1rYp2dVHQacY6fyIahGwCIqE3jzu1zUC/wAmDDTjXpQGD1BaiGEbPP8PMgHWJ/Nfa/5Ry6g9OJat3DA0F2vn5tG6roj4g7+jqX4mbj2ZfJEA/YHXyJj3rVnBY9SZrmqlD5L3LT5t+GY1AhD0eWc5e2gYuRhcy5msaUPx5j66KtXJ0wG68eJe6AF1CYljP23FgUqTb2zW+i3X8nn//VCJEUTXEAfdJW3pO70fWIUVlmstITCirC8DPdojHWhepoO36Z5DIQDYP7dOCNAoNIikgkuqmPdOKdHH4MOcj59XOYfGYwzhVCgE1m9+EH08to63Vs6nOhzLiESBZIxCEVL59/ma8e/YqAhlaOHJpEwENlXPxThCUC/DsOA9Crc5NyOZrL7mYcYjU9+h4AHP02QkuO0abqodouINrrDU08JN6Nn7ad5bZRDMrNoDaiim7vbiGRdaO4Qt9YgNRupj1yKGKRCx8JeB2cwLi2cOzjHVilnSRcPYpvUyUvjjnBF/WQvXIdPs5MfCUtFEZsJ7RjCR59p9h0egZ1WVcZ1SMi3d/OI9++RavvFXYs3MHWFz+iVAFfGX2w5kfwo8qNJUjQKO7R93BXcg/T1fArAeztg26XmOF7dQzuHcf8tw4yKPAYZ40KHGoRR6YeYao+FkfNNVwS1026wtCre+mu8qe6Vc1pSya+yVI4e4TVHZO564WNhNUICV8XTpp8hNhRfeSbeLcbyXE4mbd7HpZhFjYv2Y9RlU9M8wuceG0Xk3zKKckfyNWLQYx66FZA/vRbp1HXhOPld5Jd38xjSHYqqh8E6YC4nM0oeocxPTiKTi8tyuaxOPJ1SI1Swp54msWeCeglcDIUYB+dUwLoCL5FbRZdGc2STUuojK4kZIjg57n9A7g8S0AoZ5R9jbW2XvgdJ243eDcX4tleiVdLCY0JKnYl9vJBawj9RbHYevsJLTpJacxJAK5aYZoarnwyhpC4JtyJOpSmDpSmDuojzXiHHyTRmUJT2+OgAZdIBDcoWa/foFHx8jWw6KM/WFP0Mn5VvzDx+ETqw+rp0fdgl3bxdN0UpIM9yXJ2oT+7j5h4F9lC3otT404x+IFFeMd637znsqGLGXeinu68WP5oDsJT7ybyX2P4pvM9Fm29RdkfXdzH5SvpuPVgXzgec2s/4YWHKAtbwXdXPme39icOLl/PfNXnTPttGl3eXVzMuoivh4OskjR+V1/E2eWJp9RNSmEKE05MAKyk5M3gVMpOogG5IRy5Us7EowlUxIjZevdWsoEPJUcQuycjdUhxOVws3LaQ5qBmzow5w5jiRxHdKKix9pgJKruIrr2KgO9GcnfsbOANZlyxAnIm/DmMzJxUbDIbnrvh6CQPrk04xeH4bB4IicdZ9wy5MXfibRyL3COJQHkOotoIwq4M4dRE603681MTarmcXkKhooU7T95JQmkCFeM2419dwZWwEww/OQMvYyTVcc3ElIQzbc9dWB9roTGskzafNp5Uw4vecFYs4cuY6yg0LcyeNYXPIu9CbVLj0e/B7b/fjvKaEHif+P5Eyg+U06ryRI+FM71xnJmVzdvtwos1N5tRSpX0uX5C0+hDr4eVPd/Npd34JWW+M7k6ZyvJugB6faNxyUyE/utDzkVvY/jiCKp0bahr1FzMuohIJ6Ldrx2Fwk58eRxOrZOhjw7lsugyaoMauV0Yl/8q/PvLPl2zhhdeep5RUidvOyDQCi1W2PDpV0R89Bnvf/gh5h4Lg68OZvDVwZTPLCduRtw/zlGXOgt1Vz1BuQdoL87EL8mPy+suE3nyAgVjH8boHYHK0EzU1d+wtE7Gw1NH6JgoarKbicz/g9F7H2ftw2s5NfYUHrYIxrm5Kafxl83fMv/mvyuPVrJ5ymYCJqeBYh7/K4voeJCIjgc5mZLCviEirPZtzM0ah6Pfwe5lu4kcF8nApQNZ+PtCvGO8/+34vwq9zGoz50ad47mM52ArDD70HgButeu/lUix3tBLdYuEeFiWBzzsCXOdBvokVjIWZ7BNuY1+2T8TNvduvBeRqg37MhuerbVI9N4s3b0UrUlLStJArFIf1BYLdduOotCkYtAL81R6bjotXQKdelfIAByyW4UkKXVfEGl5jNiB0GA+SKdXKVdj19GpK2eEB+RaofbPLMI0OmJyvIg3jOVqxlVEgJ8EvtZGEu0Kp9V7Aa5LQqK0NKGUh9Y9RHGKAYNHAL4dvjiGqXnjjTe4bddtGDwNvOUDo5VwwP4A35WEkxm8Ak1TL9rrPTy54lPIM4AqHLNITIfEgU3sRGmS4NPpQ0pBCr4dvnT7OLB0WVB634oNOl1uOp0w8/59lM0bhUgiwn7kOCmnhaKwjmtONnisZFJTMqIzfUQttFKeeRcDj31K2rHPyInRsVMO9/ucxC5WIA7yJndQLmaVGbHHQj6evYkY7xxeenY5OFU4Rro4PfY0FQNCiL8+n+e+modmcQh1f+iYkZOBab4R198Khq7dGP+3WZ+mf5AMlZ+aqNYnqQz6gD5NH1vv3spdwTfonC0WZFahYnHYvcPoHtxN5apK3HLBV1qevZwLn13A7XaT82UO8bPi0UfpMfT2/lt/9ARa/ot9tBXw9PT8L+79f6aJxWKio6P/EfP732n/7WRfaGgo165d49dffyU/Px+TycTy5ctZvHgxSqXy//4E/2P/Y9wQuUWEVCpDFRWH77w1jBxQyzspb+PLs8TbVBzQf4dJYyKxw4/XvUUMtPuwfcOd9Hva+HPcSSxKA2KnmLTds2iWwLCYa4SVpXFquI11S+/8x/UCuuew+7II6cV0XL5lDDhzFrVZhfF2X/4MeoTn0DNZLWKDWQjM2sV+mNVm0uRwvyfQ8wNeqYl8ZbqERLOYwCVTaN8IR0Ysx6JoZIXovn9cT20QArfxx5uIGCii1/M7jpqfpyWwBbfIjVUlVOkf3z6J0XPPIo94nzZRFpUxlWiNWvrUfSTYVTQ4/fA/MBFvnR+B0UkEdycB0KE7zrtOE/HmBNIzl+IfKocbkmpiqxmRW4xYbcVHKuHugsHYKiLo7zCTcTmD+rB62gLa+NMsI1f+Ii10E+UGt9tHKKLCgVPqpD7CSp9aoK2b8dUMoiZGUfxbMZE9Sv5MgXoH3FntjUeYmrWPrOXpT54GYLkOBoi92Hs6j/hoD8qc7wIQ1fYEPUkifg/ZQXhtOI/+9BANt7loHCQEB68mnwG5BqvCym9dWqZ0eiLptRLZFEl9WD19WjPqU55UBQWROc7A0flTmDYmkLMfnCXnsxzEixQc94dehT8SwzLGFURy9cQYFs5eidLUQdS1PbhEYtxiCY5+B7K/DVcNnwuIvxCRmIc/e4LeRcWIgkTgBqfUydXx4/Hrusyno6vQ+nexx1PGxC0zUFgVdMwdRerhU8g8+slpH8nIUyNxrOpi2YZlALQtMxIXB14iCecbF+BymXDqfbk6/SXEdiuJJZ8jSy7BP7Sdbw4PQr9rIs3R9eQdLEPtr74JsdeLm9FEtmE3RgJw8vWTtBe3Mzp6NJczLmO9V5hEpAop9xy5B7fTzfr1Z9jm+w1GrZGSlmBe1coJrMrG5qHFEBDPkq3TmP1mDRfLqmkXV+GIms6Wu38CoMjyOwM7BjLXbOerccVsrHuP93sk+LV7M3vfbGofqqNsWCeftMiwdHlCn5TSEcI9D+nsoccWhCvUi2ZnAA+RQ7I47Obz/isBJhZDSEMQ0/dPxhnZQ1dlF9+mfcuwF8ayv3s4Gf4NzF6ixzvWG320ntI9pfTW9+IWS+hXujgx4QR+3lMxFDejMrbdPH9n2Hi0xrMMujaI9mAHWk8tNrmNoqQiomRmzoTCSycSaD46HKtfL2dHX2L8iREMPJ8N+FDb5oNh5i7MbhBJby1gfF6IpblqDQmDO/i9T0n38x/wfeZpAgID0ARqMLWYcEqcJEltBCzbxa72SF51CyjI11yvIRKJMHeaEUvFeOhuUVgAeMR6s/HejXT6dOLruhuxRMyhb2oA+POzQqJHm7m9YzIFE0/ya8Yp7vJbyuXEp3H1ReKjkjA0P5wRXTL0C4/zRr0Pts7lNxGcuNw0hDTwtEnJ+rc/4lj3BwxeMZigjCB+mfbLzTY80DyYzIAzlMoFD/LMu2c4/vJxRMChpum4pjcQeXkwvQ5h0eC0Oqk/X3/rHvT/vCeA4p+KWfzLYjYs3cDChxficoFPk0CzElQh6IOuezSHUTnLMV0fjCwiipKRy0k7uganTMH9r6yjrVvHkjPh+Ce4ODolG5PoMCHtC5h0UE1ZXBlbJEX4rjDgIwpD77gNm4cWt6iVMjuc6lPi1AiujgsXmnIHSaYkipOLsd3ohz8oJGy6lo5D24spOJ4elxakQnVz2rU0nMZOGCahNXIoffpQcDt4HB0ZzWEUnqhD6tOGO1pJTFMMdeF1nBt5nnMjz3OlaRBRz7yO88l2TqaOuvlMJEB05zL8Zd9xeZaMkLgDyHo+xylx0hzUDMDosiQyBlZxwgKiih4CTCIaQqWoWr7h5OHhXI+t4oG4y9gfqGKrZAlLv51N7ugzjHnomPBuNANYfHUzbcYO7lQPwnnFgo/tZeSBx2iJW8fIGkEXUhwaDKJalMZ2qqP72Gh009DhgbK3BZtJj9IWhhsX3Zpz9Kv7kAe2c2emhWK1H/xtca81aun4cQENM05z+8nb6fG1Ig+wEdKgR2tOoEsuvPNiv4/pnH6eQMcwOphFib+w4ErPTacitoLF4gzcXr10BrQSKnZj8DT8Q+8k1lHIc8/9zsO1g7i+v4L5v9/GsQnHMGlMzP1jLtX+1aSMTiF/cz6GWgPRk6MZuFOFyzmdrw5NYIYplvKx7TfPN2bnGMZefBWTUcL0aWfJk4fQFGqgKFEI6sZ/nEKvVyUnPSrxdSTQbW9h9227ybyUSdr1NNqmFLF2bAWjPWRs7pDR3GZh5v6ZyG1ykr/MpKV6HNtU8C9mI3/7ISKj1VgiZFx35OLXOw0PqQkxMrZYYghXGRHJQ1A31aIyttEUNwajT6TQd13wl653ZSXoOhuJogJ9Uiu1xlwGK+ToG80MPjuewpRC8jPH4xDHMSD9I0bJ+nCrpTTGjyf2yvab977k5CQ2P/wte1MLuMd7Csc8hb7TZAxBm2LmvcgYErpdiBs1hF0cQ1ShEFy1S92cD/DG6M6i+IQn8w9Opy2iAW4k0l/xhrd8gMYPyTYMo1rezdQEH7547AvMN7T/2vzbqEnuQqFT8OzjkHwxj/N5chQ9bcjhRtJGoEEelDcIq8JKaFUTIQ0qHp+cw7Zti3CaFVTPFD7iHs0Fhp0fSNWJTIx2Px5e9TPHbMeZu30OKUXJ5A3Moy2gH6nNzOO1s/Gre47kZPhs9Ses/mQ1ml4XlTYpvxrBKziTgl+7Ubmu4h4xkuA6B+o+NdEy2BTUS6VczZdRRch7ZLjcIgE5cCPJtPbRtYzPr+T1AA23/XwbjRcb+SpaQBJ39Hmw0wQD5G5CpSmU2/OEAJnLzs4OOUMDnfieGIaPTkpyQRgJxcO5lHmJCqedO9rMvFkej/uXmUjvs9Ll3UWfug+X2IUkpIXb5p2lrjsVi1GNxAYyhxCAcUqcfP3OEiyT6pmi70JWnoEh2kaN53GavK/gKYYBCjC6TbxgvMSBzRNhhpS7fr2L6wOu89uC35j05yRGZINxSS9isSftUUNpjxqKQXWVM8kZDMwbSPppTxrHqxl7UkDiFyYX4hnUwSOBDt7r9CO7w0WqsoOxf3qSm15Ad3ASW+/8FZnSwHUdmBxO9vb9c+64RwtD1BaGTN/MqbYErAYxfWohQ5XljuUzvwp+dtXza40vGUVizF0GvntoM3ZpNzu9YF7IHhqk6/l6cDbauAJmKYaCFTr8OujwE6qdZ+qFCubf7vqN0j2lyMMSqL2rgM6BXVwdtJ151/t5wvoA3u0gc3ji2QuLDkxH9fAvtF3No8lYg23WMCrlAnrNr8nCx4Xz0E4+z4f3bCLCZwKitr/GSTfFA3roTzuHQzWDtuhh+Iea2D1yNwDJdTYuSa/h0xOOSySlPqweqVNH6aBA6n2E6vkSO+xruk6ivoOS135Hp3uDAIPwTMQuOcltf1KV3sDgpGDcboi+uvPm87Qr+vEPayNi0jHyjWMZejmLuEsrCG0MBU5iwBNjj4aVsXDddyZnO3cSqAmkJegSFyWwuAXMHQKtsstLydePCBSKin4F0rwUfrQNovyOt0AERn8Hp0efptOnE28M1AdfwakoYbylHdnJ8bSr7HRGW2ny7uLV4vk8Hp6DCAezC/YirmlBnW4n9eRXVA+cy04T3PfK97zRCO/zFRe/usiFTy/QXdVNhLeYUQ3wh30g3dcnEhuRSN25OqasmYJfsh8l3x4kWgb6209zb2oeM1rH8dqbr3Fq7Cmyh2fz0vaXqI1W8dOS5/C1jMC/NJzqqDocUgcBtZWAiDekdSyvHQOqDg7OzKNLewRfw1Tu/CWeMKMnHli5z0NKbb8Uh4ebEzMDuBb7GX8IkmGsSxTh3+qPxumBxuLktj2z+X3e73R7d7PmmTW8mi7oJvvXXsbD1E575FAa7B680AHxFRFEtPlTU5tH6rGt6Iw6bHIR1f0SAjROdgfD9x29VNxI3Db4buSNr96gEpAoTdy2fwKdM004JCJBBxVo7tJwyCnhmcpRRO1NJ2CWHwGiAIxaI2a1mXe64PteSA8eQu73M/GS+WNLzyTSsZNBgcdwuUUUWO5nnOoH5ssSaVKFc6j7W4JNwbjELqpSMhF7xeGb4Evp/hYSsvdg8BeKEM0l0VgK43hh21j6l7dQ0fEHyYZkKmIrODsZykI+5juFFl3oFnpTX8V4I+Br8ihifudqHo55hIcK/Ai94IfrQxcLti9g58Kd1J+v5/Zf5mFKHsfa68/iP1DKq0tfpelSE1fWXSHu6hE2XV1K2e2r2Lg0iJoli4kaEYUkX8SndXfzqO9XBNyIiJX6rKfVXIWjfz4D8waSVCysu2OqYtBs0cDMW2NU9OXt0NnFpqWbSL+ajp/JgNsNNQPnEpezGZWxjUniIKAF04IDfDIyj22K5wisyqZLJQx4+2+Me92pGcgjYxDHCEVzNoWG6ogi9qZupLkhhm9OTgPfSGpSx3J48BwATgg5WkRiNxc7Z+EQgz1Dz4+6F+jy7hK2uSXIbBKUFiVuhZsTCzoo8d3xj7H2rwTEXwlMib2f+IthRNaGURtTQb9ShlipoNPPTKd3Jz5dPvglVpM89BQvZbYTKZ7ANCf06UPpcNioVxWwxmig1jMPFQKt8OCrg6mNqOVi1kVSSxPRNgcwQmYl/shEcqb5k3BBQGQ3ROkJre7msQp/kn37+fXT+xFPSmDP7MM0+wlFXwpbIHppEN8b4Opnr2CY30JCmYBUj6yOJPXEl/SpJcArtF2sJaRMiDOcueNXJo6aSEtgCxGX3sAw+gE6fQzUhdVxbOIx5hc/xCF9M0U2B6gcuCTwFwmazCaiMWI+XdpJGIbEUzX+T3wcT96kP+/TmKkJKUTiFhGSrqAxvAdXZwmhtTLMiuuMP7EagOujvuf9UC25oQ4CvOOIynsSH8N4vpWuYF8fvB0Xxum4UogrZdoNzT6zykxWThaFKYWIJ8QAoAnQ8EzrM9w+qZf0QxfYb5jPg2+v42yzINPx9CdPU5hcyJXMc1jU9+A9sY6W/ECsojLkGiGRpeptRdUrsABEqoeR9fhQQkeEcf798wRdC+Ji1kUs+ywEDqihckQbjHiXJzX3MnnOZK5nX6djfwdfPfIVfeo+ZHYZo86MovRsKfsv7OXdF17itNP5b7pei61WcoDZL71E1v2TqLqjmtFnRiNR/Ad+zxumNLbhU3kJQ108fkl+JM9P5rdjemxKT/q1fmi66/FpKsBpHgNAyIhIcqu9CKvNvfnsClIL0JrFjP13Fap/aAV2VQjfTGufGm4A4fxqLuLRo8RhTaLjeBGqHh/MXiHYpIIPcXT4ncwLcOK02yncVohMKWPIQ0NImpf0j+sU7Syi+ng1rvRbVTbjZM8Ro48hpOTYzd9Ucjkt/8EX+s+sFVDIVcgsvQy8GkRRQiBizxYkiBmdt4/kwQn4D/CnJ70HR+k/UeX6bj0WkR1zs4G4S1uwiMZADBh1RnYt3kPm2S/Ruy1IlArETjsSu3D8N6u+Ye0Ygcq8MWkyTtGt9Vmd73cEdt+OyxWCYoDupn4twGovWKCF50+NxRjg5MqcRqptJwDwEkNrNJTnydj13XAiVley9pG1OCVObHIbU45OIanQk3kRqZQcGIVm0GBCO5aye55Q5CTrga29Iib5LaTiix78BrQR+vQMer3b0Xr3giYagiazp+scn+RVMypSy9mpJTTodt7U7APofLaToMFBvKt+F5fdhdzLgyr1Cn544AfWhT4ssOF4aunxDsfkHU7n8Muc63+BDo/7GWx/lQXx/tiVHvSrvPEwd2GXOVCK4HXNYX5vv4wj5gvOjjrL/T/eT3ViDy/W6ZC2ZWLMWoxOqUbes5X4pnisivOMOynElU9ObSJC/CEtUbVodToGHQsiXDoB96zjpMqhalQ2P5QLVLlvdjYxb3s8J0YM40qmwCD119gu7u4kpFR43t6x3ihlSg7ef5AHBgsUp0pvJen3p7Nh/AZ6anrQR+nRR+nx1OloabolfQUwF9gELP4v9NEfZUpuu/vu//sd/w82kUiETvfvGqH/u+y/newDkEql3HPPPf9Pt+V/7P9j5na7MRqNSE0uKncNJIRuEkY20d7vxh4kp0QmQPMXvfcIpcDAJ7fhe+d+aiLv4arXVVzifhT9wmzc5eti45UUkk+mYU3qgRugleTCZBb8Pp+g2VJeHvcxjpEneDh/HR5mFW7cyAtMrPr9URa++T3D2vVEfPMqzse7uFbpgS5Gx2VLPDXKfiK9CkgcUsLPXiU87P0tIhEoTJ2MufAYZrULT2+fG/ckXLczJA2/OgHp81bFIAquJGDO6OX0xGYM6stklu+jaMQWFjjsHN48hQcLo+ELJVvvvoWOVVsSyGj8iPGbr9AU48L4xwkSz5znxEx/pu+wMEH+MKeX9OEO0OH28cCv6JKQBIkbQG/qCCrkoaSq8li57HE+btRhrJnL7H2zOTT1EJGGn0lLlVDZ+SxjGtqo6gvigvJehuw/wrX7BUyPUVnIB5/1Mn68Dk2gBs8wT6qPVTPurjji5Zep3z2BR5uexr7KgVFnpCWgBX1kCHPVMFvTg7t4PebqGEzBcpZfW86FiRKULVZG9gXhF2igedIpzGFj0Rl0iNwiDF4GZlwOQtUVy8/Pr+GT1AJW2yczY/0Mzo48S0xyOoU+e7joduIvegBvy0j8pKD07kEfo6cbCbOa4InosUyU5HOtPApHhp3m3BaUva00xY6mTx+KISAebTB0dd3qi/IoPY76dt6eUMy7+ZlI5R6EXVUy/PTj7LhjBzp9Ij8uHwfAjvZI/PzDGHgtCZ1Ri/VMAdLmLvSBnQT6JVGQLmJ4bCDlAdkEtgbi/3M3i3mDyY9u473I5RAJIWJwSWS4JDIcchdGcT/rjUn8YreS4NWNWCGhdE8pzkAnfbcJQSSJ20CJWU++5CfgPmZ9N4sr667AJnCKneT80kbc6lD62gUvsSGnAY+ddajvVdMa2IrR04ZYIqYs6x4CK8/dmPRVeLf6I37wV851BTPXPfXmM9GYNEw/NJ3Rc0+zLW8unYpI/J2D2X3bZyzespiIDVH8dvk+5KOUHJkznqnDIunYayH28jbYV4seaKr3p8nlj1+bC7VYSuv1Vr7P/J6AxZlcD/+DNomCupgGNt67kaWRs1H7qRn35jhKLBE0FxuoWbeZfPFYxr0xjqDBQQxYNACpUkr1xEWc91sOCDqyvhnhNMeMRNNVx4inh2P/1z685cP44pV3mOz5EZM3TibtehpvvfIWU9zzeK8LRruUXOzww2VxcHr0UcafEIKUh6ecomDAZQY2BhNbH4J7gJvnOp/DbrGzbv5GQnNncMfUz9nbp0QKON1uAtKEZF9NoIH1U77AnCCswKc4jDfHhL/EjlU+KpxOJ9evXyc5ORnJjQi+WCujKkagj/LqEo6/4j8dTdYA4nM2E7JVRzsjcU45RYnChsvtZkBRFi2B10he6ybcmsUOfTdHn/ucZv807s2Ko2zoYgKrsvHsrsXX6cthdQ2fGybSpKhkvr8afbQAF3JJ3Bx7501meyQxtSyZylph8WPtveXEX84ZwZjZm5j4+Qd8LlmE2+1m81QhALnoj0VYuixEjf93ZB9SgXrGLXILldwuKBm+FJdETvJZYSHQL75AyqUsuqRu1MuEc5SMXI6z/3c+25hCV4ya4Xun0zLfRVHyJpySPjqCDtAUMZ2qwC0UhoM7vJbVRdKbdJRumvjGABtagzmeepgRe+/koERBQIediKY5FCcXs9YAmTJ/PFs98d4+F7WvmO6QAfRqB6CVCcGQuX/cjm6gFVZJqR8gaKZ6tufg8fpTlKss1A2ZRXxMFJac89x78l4+f/xzDJ4GRG4Rq81KIhJLiPHMZMS5EZhVZvLS87hblMbGYT9jGryLXcXPklMbx8trX+bQ1EPkp+XTElHLRmUHn/f4o2iM4dyVXJZdWMZnT3x2M/G1sTqJngQlLZF5eHiEcWzCd3RFVHC6cTK5DYtYOiOFpGotg88cQbvIm5NTniUvcSlhwftpvpSIX/Zp+nsjEYsVeBjbcImlnJrczoZWeKq6gZTT66i4NIX5B7fQ79HDzmkzSVvlz8fBdrwDutkXGUGpcQCZFzNpDGnEq8cLo8bIGp9aEtOl9HnoCR0zk9yAmVg9fsZLDB3RcKLvEsUOOGEwINu6HLGtme0LtxM9sIJHNSB3N3Jw6Tl+MVyk2+9J1q8Q5qNv/CFNDgWSEP7VCeVWDerRXmxbuI07twvFPXn35bFy5UoAjr14DP9Uf1wOF+5+Ge6x56kpn4ZCFc/5rtabCzzNsxreKHyDyw/8zNXpRzjc7careQjxhS6ycrLofEpDpt/rZNtdfLVqLSvDvqC+vp5eXS8ql57wSh+yn3ge623nqdsxFu/HFQLqEVC3qeh2QpMDVAodYrsSmdOD1hFi9qX/hrc1mh0jteQ7ZvBJsIZ9K35gqd8XiBhNuX4ogZXnSD25FodMSdPVJURkCfWVEgn4117Cu6mQV1eVcn9KMVJfDQWZ1VxIPUSPVw/11s2UhL7Il74gAna43fR5CaiE1qgsfLv+JMTDwYlQeKcLut23UPIKmxPvskL8FJ18dHAMuvpQOqb3oehqoiUihB4vG1UhBxjkuo1eXS+VMZWM85CTWBhAq28MdemVN8/1cVch2bbXeFS8mD6tHKtcmHiLUoroT4jEM9wTWU87zTGjkFsMBPe00e8hZVDeIKJrUnE+72bsqbH0qftoiOojpEHFCQvsvXsrFcYQRtdFsWjvInKycm7SP24IlLFj43DcnmH0RdWQUpTMoGuD+HNaC2eHfcvAlgdJOP0+FlMMjIND0w5x5/Y7CXr5Ye564w3WJ+qpShYjccoJHjCYdQNX0aOp5AMdjNCYaHfZuWSFEbixt3ah6TSC7lYg5S+KTUu3hUNPHLr5+8ORvfQfnYxE70fQ0iyely3Coqjh4W/78fzpJWLu+5H7f52MO8PCxZE5nBt25GbSE+CjwAp8l/7M2MjX2HC/UJSjNafy4cpv+UoEsz1+o2VUPNroYq6o3mfy0cn4dPqgLo8mZ4CTXSOO0/bVJn4zHab8wAeYI49R3A9+VfCvlCk4+6Mx2x2oRGL2zj5Bu281AE3BTeQNzGOuQoajqZW0o5tpiR1Fzw3kZX1YPXvm7OGxyNvQmPT0anvZuWAnT322mu88fGhJuMBjZzKwvmPn1zt/pcerhzZ/Ga+99RoFA66z+JKNwIoYFPdBUmMS1VHV9Cv7+cxHjtPZwYGxRVzuruR8VzJtGoHZILtPych6GBEUz6Hh73I8Q86G6NW4rwqBjbc6YVfdJN6bKrqZ3JNd7WDJuSWcGnuKwJZAGkN6SA2cBQiU1aV7SumNG0xs/WAYmMBgD1CKHdi0Tj5d/QnPfSToaPnNP8LbfvB9wUAqurQUz7pV9DDmcC8mBhI7tJrK2Dy8ZaPQ518ivS+dotQmUvJV6N0ZzJv/Jh4p/2Jl4ds3ECbgEln5OHw1H3dBTl8xv9yZw+qGRnobFt9MTvpKQI2DpV5XGaGCJSY3omt1jK8fz6mxp2j3ycUtn4DIF6xWBH2WkDRqBowiNP8VOjt1hMvgGi709kAcRi3X0q6hEt1Pa0QMlQ2TUF5XMm1pH4POlRObexvfrvqWdidsMcKX/dvoOd5Dmb8v3GDkdUqcHJtwDI17KZ9WpdCUH8ueyU5yJgrvKqIzlxTPD/FTqJgXpMP/3XV8KdqAtKoEX4cMi8uDN04e4rkJkST37MGVV45t0GiuDTaRl7qK6h74pOfmI8blcNFdJaAZagfdz6S8VeysKic6/zgBU9rZ+soOgjKCuOfwPQR+MIoPPON5/q0fCWhYBGoppQmldPp0MuqsUIATl1LPb0HwTr2FhkwL5wZtAaAyrhtZfzNeQWfJ/PI8zYYUaizXUZvUWGS5EOTDkEuZxDuvk7B/FtfODKJvnYFTmav4u3k7Gnjkp+UYQrppHyejfcidtAceAMCzbzBJnkOEe0mdidj51zjsZrkOst74iejyRD4Q3UGPVw86ow65zY3EJSZA6iSqMprgLh+czV5kXbuf3+b/xrEJx4gvi+d0XAEJk10ERN9Dvlc0xaEbUPWpeGaNoHF74s447EOHonAXsOrbVRyecpjsEdlcsgJWuEszjP7SMuQaOf13DaSi00JF1wG2FbyKKsGfDLuWjs2Dic8Mw7S7l37pPPw6/GgJ76AzeQYeXiDCjay/9yaioHrYRc71SsGgxW1zoCluZuG5hXz56Jd4OvvYKfFmTmAvV3wrKQ9QEVcm6EZavKBrbQuvb/6IPQ9+SP+gbOaFzaNqs+Azh40II3vNBbxGeXFiyCV0I0sRS8UYG404JG6agxoIawjDaPPllVPHGDMlgBixCKfJwtHq0byt/Ip3fQRk1aflGXxUHE3/8n7m7Z5HSUIJu27bxQuqtxh8V4Lwevr7STy7GU1PI65kfxpDG1n28zK6AjpxucDuoaV49IPILQYi26awp/pTdol34JQ6EHkoyJ/wBFdi7wcgTarkJW8LRaue4O2qbYQjozJ9PjaVHovqG/KtYCgOIzpvF4bQu3FLblH3A4yqh3/ptWwzf4/FFYS3JoyWoFu4i4i2R4jNL2LUuVHYVvXSHGmhVdN6c7tvuy/i7n5sc22AHHV3A7434heZU3JInX+Qz5sHUN9djFVSjE+X4AevS8ql36uG+1RQ1GdAerkEr5YSNF31qDq6KEt7hz1Sb5R5yawPsfHp6k9xiV2E1odi7vSkbsoJ5kUp6fMopqJzKl49bTTFjaUrJJkI54/UOGYR6O+HKqOJ/rBQgtp3ENQMM4yVzEv8GLf4OC5dH5qsPDp9Usmd/hInU5J55oNlABQOEtbCmpgA6pKn0qcPIencj4w+O5qGkAaUfVYs/SZUIc8iu/o7t+2eh3xyKlP+XMwLEevw0/Sz3+Ik82QLoQVz6NWrSD3+BZWj7uCPSDs/9drZe0zE4zsfZ8eCHbSG23nEE6apxMyK38ixqqU83G3l19kCFfLVuftYWJ1AqF8TQ4NLUfmrMKSmMPHlN+kOTOCY9HN6cKOYpiVcKuiY+dQEMir7bgoSDzDmzBgKUgro8BbmAkuXBbvFjm99LhqDMAddPJpJTGsdMIILWRdoCWyhObiHjlmziYr8ifzxazjUJ+blcd9QMnwpidkb6PYPQd/WSM4jdzFtaStMOk1xUTGH04WCN2eukyGjhgMn+cEADo2wZlXL1DhkDmIrYomuiubsqLNMOjaJ/cn7efHrF/6Xib6/LAvYa7Uyet1B7KtdFD1UxLcTv/23/XSt17FKayme9RSR45Q3v/X2qDAiLv2O3NJL6fAlNMeNYWSomJ7aHnLeP4HCJ5LLw6EmeCd9mr8yZ25wOtkwYTMxU2IY9YIw/tvNdtwuN3KNHJFYhFwrJ5/Em20ILj+Nw9MHa2801Wt24RsxhDqvEP7yy8ROMbX9l5BrhvKq49X/5I6h7lwdl7+5jOsLgX40rHEcEw7FYfG2IHbYcEgV1E1bTpz5MpvO1LL4v0DluU4qImrkHSgMXYzJTsCoquC+oS0onV74H6nHdrwMXniC72Z/x57SPf849osnvmBY7U7kPhqqB87BJ+YWfsuF4MO6FEpcNjuBDdmMs4gpWSkgEt3qvyGU/kYpb1QVkHx+B91X1Ph9l/CP633cA+kK8H5yA8f7RmPxc9DkFJJIVjd81QPN9bOxieKIUshuImYBbOnXGRtkZODofGqCmnCL7ibpuo5Uy3Ry03M5EtSCxKlmsj+4RRIQixHJZZxpvYdVrufReg4AhQ+qLikr1q+gZboTd9I/KUx/u/03no15FrfTjcsubOsQ62kMaUTbq8XR2IHbFYR8RCal9kwAVLILbPSE89pfyGY0ZvkcZP12Skcsw+6hZUzQj3wbDr+uT6frVCba8VYcUgftfu34tGoJ+nQIBWnXCE0IR+Lhhe60jZGXhMRYY3AjbpGbCLmBd5bN4feiZ5B7Kgkv9cJPmoB+wXGe94b2rlO85QM/9YLd5kRu7kbZr8TD4sHYU2NRDxT6qUOnxyUSI3a7iJ0ey0uKl3hp9Es377+zvJPaU7X01PQw4K4BhA4PBWDu3Xez8Z13Wfw3Ks8VwFtADvyn3zg3tp9x2vhlxYr/i73+zzen00lRUdE/Yn7/O+2/nezbuHHj/+X2JUuW/P/dmP+x/+/YLZZCN6iUdCQPZZOihqKyUMLUItyet7jv6sLqCK8PZ2qvifqkYlYHLYVuwbFxSB1UTD/KhNhWvvds5NiAU8yVvsEd2++gYEABiSWJiJ0SPOtKYJCVXmw0jpWQr93FvN3zUJQJg9eT1lje9qohLLWcBs94Ru8vJTRqNnvm5/LYwUK++7qPcRv09LntuHFT/8kOoooNaHqEiarU4xvg45tttqqEIHryqm1IO/U0mtW4HM6b2i9it4yjI6qJDstm9L4ZGPpl6P7G95dxOQP/Nn/GLrqPB9LdfJB7lN6eSozqegoj3ma24QXAnw/0PzLtdj0f1c2ip2oSTqmCuqTB9GROpqcNehUu0m0d9HnaWJGWwKcTyqiLdCOT7GOhLpXJ8u94O/cVfGNM+MT7UJhciFllRtGv4JHPJtIUtRl4mL72PrxjvVmZu5Kdu7v4ImciLactaGVl9HVHkHE5gz1z9zBr1CPcVwMPGEKItLRht+uQmgVNALukg7gPA4hjJY9+/CU+953g9pbRLP5gMRKnhO8e/I6OuGaSPVt52wcq7VBrkvHTsp/p1RmYWGQkpk/H2JgWPETtvJYwhgv128l4J4PEpYksf09I/ohFYj7T6Wl98x2mBG/g2oY8Igpy6Fd74197+abw+d+ZMn0+mo53RzyBWvCaM4nQ8aNxb9qNVWHFLXITcP4oPxqfolWejOGUleFvBPLsym+ZeGwikbaJKJqhaWA/3839kYK2sVyMvoNtd25j7KmxhLQMwa/Vwf6TA8kfuxKRW0Jc/n0MOXsAt0hE2WNynndV0G03cy6hlnMJxbyS+jlPNT6F5M1bk8JH5gl0fyVhWHENvA/hI8PJ+ykPgLD6MKq/PgarB1N7upadC3cybPUw3FIRSzYt4YflP6B0uHE5XfRr/fBtELQuypMqmCJtpEFiQan1YUaVmekHppOflk90VTTI7BQlV9L53hiciYVEeZkZmn8bB2ZXMWNvNE2XIwgfKubk5LvZ3X+Z2o4qtF21N9t8xe84BrOE1b8uplspRv6UnOhJ0Zj8ZNQqv6bJrWZ80EOUWIyodQGo/dWMfW0sxd+C/aqFmtRZPDBb0DxU6BQkzU/i07BPGWB1cuahYyz9eSnyIXaUkzTIrH1ou+vZ/dtQQj01PBSfz/JIO5/2QnFSMV3eXbhFbtqdSl7qhKaxeWhH5FEYMhN3jpsT404gc8dzOTMfu8zIc1UZXN07Dvssgdahr6SPpGlZZHdcY/vpPyjLuI2dBUlY24UxxOVw4VsgYXH7YkqSdtNdF4M659Wbyb4jzxwhe002w58ZTvqKdJzOf5YY/kXdOih3EImVepw2J06phKbANmLFUsQuBw2JkzjQ/DXh/WC3Oki+7kufOh2FVTj2t6mH0OcfZ5JiPCIx9PrH0usfy5QRe4mqeYFfgtzMLMpBL+rAbrHzVbzAo982vY2zYfBLt4Lvf5+Pd5vQ6EnvT2LS+5MY+8AQVoY2M95XGO8+NTux991aiRTvLObaxmuEZIWw4sI/nTqPWR6sd6wntiKWnqIeQgeA6QZaqXjkCpLO/cDM/TP55e5fmBX9EpHXyxmyTwi2tQVp8G9O4kpiMdWLthEXv4jItrtRt+aQVDmfolQjg/RqIqR9/Norwv03HSylLYyExreROjxpSd1GfXUQWr2YhokS8voE1N6REIA2+qJN1I+YTXxaLLY/hePfHf0lQ7vXkJNRx6SZXjj+URQpXMcruJ2h8x+lRPQt1YkXqNcewKwys/qzJ9EadYQ8m8NHo5IZFjuAkR+OpN2vnbz0PBztCzgsi2GE7k9CTJdwS+LIG5hHu187ZrWZb+8TAvnhazehra/k6pKzGHxysCgt9Hj1YJfaETsk+Ep/JScbFi2CL0cuI1NtZnhQEyUtsxA5+0jSHabN1oXb6cIhVyHyMvBhr4WmM3cTXtuEzeyAayWoelspGvUAdpkQBLx3QCOFtuPYfCcRUnaSDm8h6F/YpGd1fSIa6b/oMEcwXd/CzAMzuZTVSq+6AK1JywT/Pj6bdoQ5VUmUOvdxNWorfR7laNywpw/67RL8fprPgOBemnrlhNUmsnLdSvTRDXiJZShegr2lj9HalYly7C3aNjngIYIuaSg5m2Yg8vRDfq+K4uTim/v8EfoHH4cL8/D8rfORyCX8NOYnPKwacnPjKByVxkifOfRbBURpeG04pkIhCDmuchLPhPxAnFPFxOPTCCsVAmqmK/0073+NzOU7+DmsENeNBb7By8CJORcZu28k8uBGTncvJjYikPCBnmyf+DmTjk2iclMh1j/f4OkpR/j5CQXGl4o4YtxMo1sgrjMqqvi5F/pFUvRiKzEygQbMJffArvTAqhaKiJxS+T/mK4kEahPimXHb+4jELi7Vz+eOJDdeqkFUeZ5B25fGoh+sSAIeYeuoS0h/nw5Pd+OUeVA7YDpmXRC7Fj6Ch0cdQy+kEXMpjvb0EO49di+nx5xGabJTeymYXMuD7Bv1PlqTP+LENPYPWk9CwzukXbQx9MwinCnNfDOlnHczy5GEfc7Eo4NoDo7nh9h3KTIG8/vYRSi+KmJl9jCsd5iJbphPZcj3SBzhKNw+BMmGAeDTcI2gynPUpUyjK9LFzslfMzx7OMmFoTgdbv647RxLNsyk19vOey9+hdXWAx6gFOmRdrmJK4+jOKmY8yPOcyXjCiaNidfeeo3qxA5qZo+iuPIcSSVJGCO7uU/qQWroTuyaRRRLGxlwfQBKixBEagoSEJ997dtZsXw9nzfFEWSYQ49GQP5H32ArknYoefj9J2jL6MFtySbx8mVqQgX6qYCWAIJqe7Fb7GiDtaQsTKGvw0KztYfUEdcZ+fpyOoKFQInILczvkhtji9KiZOvi82RETaDH1k+nqg6AQFUdL/vBb0oTx7QmRqnEcCN+JUKMb3k0dpmdi4Fr6AvxZhDzuDzkMmXxZYjcIkwaE42RSnJEf3DwvJT4WB8yT4ZjGR1DbaQwX4tEIgr9nuDcKAdTB3pQ0FqOVS6M9wWpBRSkFvC2pwqJTEBm+tbn4bZ7UKEPpCWohS6fLkS+Cgwxr7Nj0CzGnRyHtlcHvXYeWeCkUdpGr/8sSpKEIj5VnwpT1lWWpFewqSgYg8aEb4GKIUfv5JuHvqFf2c8HZ36nT93C194rUNtUqP/GElbrcZ3afhgh0XEkys7+PjtOl5shlwbSGlDPtehqrsn+5AOcnA2Fo2Yw9doIag5i9t7Z+HYK33fr5vdg5YsMfXQomY8MZdXkCiQXKrhT+RralmOEWv1x3CVQ27///PuEdq1EpulA0gmiZb9yzOBJ+okVvHLwFdYvX8/xCceJL4vH5JHMTwF5bOzpJ6y8ggGG8eQO/oRubzuTvVpJsav47u2lJPja0HQcpCcwgY7gW1TQiGT0usAByGxOPCwe9Cv7qYuEK+5ytjaO49cTnniEuZEXtDD22lhOjznNlZgFKOwBhNm+xGSaRMG4R0nI/pmQ8hwUFiWfVOiRlM7FK8mGctJIPssYh1vs4KHv0gkvOceftxcy82gGXeE2jk7cdhOx+pf564y4w1uQE8Hos3OpjriC1CElviyeE+PfRdqYgi0/Afn4W4GhUUeM6LseZOpjOyjz+40fC00MGjuAe18+S0f8Aj5+9Rcawi4j6t/OE7evYlvQU3Rm+LArUhjLU+RwW3sIp3qFD3DYk8PQBms5vakWtz2AOeGb6VZ5Uaa4i0nxgeif1LO2ai1hNWGoOz3o7wznRHMahpIPYFQZ2xZtAyCpKAlt5nVWTD7KsU3zGG0ZwKV5QjFCdMuzVA6MZtHgL7jDFcpsdTNS03J29r7Kg989iNakpTW1BV1oKWb5BI4EnkY8xIUXS5Bb5TglTu7ROwmSgsppYOK8U+xDgUnpgSEgCrNGeK5jzz+BX7wvjIV+rd/NZ+YpdnHVCltNgq6S2MeDH5f/SEBLAPG1y1gb7GCFHp54bxYyiRx5ah/6bi1il5gzY85wZswZDvn6My7jLMtb7idPtwy7rJOgJsGfFkucxAXX0G6+A5W+h+PjP6U+7BZLA24xseIqtKubqVc+yvre1RSG/sLRvBcY0D2KrAQL5X3zac834BumJ39kC63yy4S0jCasLfPmabxTgmiJHUVY0RHKM+ZQFT6KDUGTYHUxXyQ8QGOrjfygAxi1RjL2SSmoeZzWVcc4WH+aDbfDpUMnscvsnJwVRldgLBbPQFYF91LvNlP6dSlF3wkU8rO/n82ayA+Q1ldyXyRk251U/VlFb2Mv5akBbLvtTdLkEFbjx/EeBaarxWRNDKD5rc8Y69fAgGHwZQ9Ey8ZQlBNCfHEIHp4ebF20lR6vHloDW0m+awwR0QJ7haijHU1PIwDXUwUEz5a7txDeJQT7pdY+RC4nVrU3pZ3DuWaIQqqMZYhJjXuoG5vKC7NaKOpbIosGConTVXFdto7wnjW4pAqsKk8QuZjTDJ5+uYTPdzBW9zT6omIyL2ZyaeglZqhghSdc7/Ql84ul1A4y4Uy3o+hXYJfZcUlcgJv6sHouZl5hoFzxbzpdaflpeJ4xYnzYSFDZ9ZsouNZxUzCMucpdGjioMFN6/ABP/fwUFTEVeNgjUFgVDPCAN13evGZyIC0qwLu5CJNXCBplA9fCwZTvw9Edk1AucGAZINyvri8ZhSUKd/UDJE94BmVUMfsPDaR8iB67UoddqScn9XnUKkgOFnE6CLKigb8IGXx3MiPuW/ZbVFxRXSU7yMm4GB3VTaC0RfLD8h9wiV03qPHfwyPYm7Zowdfo03mj7u3iwrALFKQW8HX0alR1TkwaE0EtQbTlS4mWGvArSCbn0DAkj5mxOSrQmDQ0RMsxa6RozBv4pGAYZQ2ZWPWKm/TnHbpCRihhlsbJJ9mbaOiNx+3z483nfGnIZcZPvky76DbeyGukshIeTIHuoGT6vEKIaV+AWORCSR0bro7j/NVYDEklTDocT3Cd4CTkp+XziFjQWzz05CHyN+UjixiCQwJnJ5zHeDkLqaMLtxsOTRcKjZTWCCJrD5EiOs5xP6i+sXQzeUdwZfrLOF0lxHa9QmBEC/SWcb7mOO4IN3XiOsQiMfceuZfKI5W8LVexPmc90wcp2b5gOz2+PYj9xUw7PA2AvbP3su7BdYzuG8V4mYws5/8CRvc3ywJGucRcOqXEPsnxb9vdbtA2bsOz286pwTakikU3tzmdgm6aythGYOV5WmJHIRIJshMAus4aygaIKQsRGEUWbV2EwuaFa6aLlrwWfBJuUQD/PO5nTC0mVtetZsjKIWQ8OIQ5c261o2zoPWi8pCh0CqJfupPTefkMGZ5KGe2U2eGpT55C5H8S0ZNZiCSCk9LT3MO65HUMfmAwkz8UKIEnvD2B0S+N5tA5N1/Uv8nipM/4YecorMYk6gdMp37AdCqin6ZEtg/Jyf9aIuWCU8wrw5/jynk92+4+T0NgGSVN8/FyBNAVkkpAjI1DTx7i6g9XUT6ixKKy3DzeyzSUYMN8ZFroDEtHHwAD/AdQ1FzEY+efo7I+D3fkIDzC/elv6KQhQug43p3e9NcL66bw6wewekhh8K121YQcx8M1Ea/6PkacG0F5XDnt/u3k9IPNqWFOfCs/1VmIc4+h2PkHLokLsxsea4dkVRjGxdmExCXifcobu8xOYusvvDt3Du8CWMFPN4jn2i2MPOUD+BBRG8G3q75FhAi98y0Wvf0z+9yvsOX6K+QF/sadNQ9y4O5nwOUg1qcJ29wcHDEjCGxQ4elOZvsd23FKnJQmlqLyVeF2u3nZ8jK93U4Wr7QzfsQ3LN8/kYKz32N/+kVEolvrUtwO7tXBvTorO1T30Wi6SlxODmKXg4Lxj2J0iSi0wZFBVygLrGRZZRITjk8gsjYSk1YYh6Nagnh5RQrbit9le2I3JksdcpucvQt3cC3WQaszHJvVC6dbSNnseTCfWskRNAb41QSXEq1MvZKJcV8WjglOLsycz9m0Z9AZdAy/MJwOpfBtuZVquoIH4NuYz5ZtW3i582Vm1c5iZfpKBi0dxObpm+mp7CF0eCjTv5yOykdAzax44AHeevPNf/THQOB9YDaw9z/ppznAdJGcO+58j8DA/yox7f+59h9jfv877b+d7HviiSf+8X+73Y7ZbEYul6NSqf4n2fc/9t+2/sZGfIsu4u8bQsSvK5AMHUxnSTnPn3iedSvX8ePyH/+xv9vtZsnGu+jx6mL3vN2IR1zh7kA3pZX34N/2NOG6TijqpCWwhSNTjtCRUE6X7Vs2a/ZSaNRwUlTD9dTrFKYU8sKXr1EeXEiBSs4F+efUDnUwcvhcvsx8iMbAAsTqInxC12HsG4F/fjLpRfH0TTdhvFSKWxd6s03t8ZZ/tFFqN1N5v5Qv/UvpGOWibdx5JIEPkHExGrOHArXExqrf5qMZNonzt7/Fh0N9+LB1HlMOT6EsvozEkkTiKuJIHlHORf0E+uRBXBvzG4eH3KgQmxzACsVuZB4K3n7oWXoDDNRmLgKRGLEbwn/7iHhNHyrXWJqtYmQOMVq9J/36LJqC7kXqKEYnFQS+T7z5LkbbItbMjGfHZQHNIrPJqImswaYTUDZ7lu2h/IDA0S2+fykX25bDCBf33wc6QxOz982mObAZghx0iuDrvjYybrOhUILBq4PssT8D0Bf2Mup6GWsbVNTb1Kg9AslNP4LILSKhNAHvI+lsmt7CZz2liCvCqA20UxtZIzxUmwTUZkxuN1JHE8e/HoU57Fd450k+OPcB3mJ4wRv0ogL2O9WUuWAyblIWD2TrmVCkNjNqQxOerWXYTJG43bcmbFuDgYOb5lKcVUyLKVoQsh2kYFfaJ0S2PIZncyX1zToqU3UEu9vA7aZP00dEbQQyVwublu6n1beKsQ0zMPSm4Yp00+3dze55u0mpGYuj/1sU3gb+VGnYVeVLi1xAn4jcbvxP2tmdv5CkN/bSHNHBJzUjkYikWJwW3OJbgZYIqQHPWAtFzlsBgfTl6bzc8zIAM+KWARCcEQxA/uZ8RA7h+MnH5hJR40f/x/1IbCJao7IIqM6hMLmXGoUFRb8CmVWCs6WfrItZNIY0ojFp6Bc7edbVyuyUepzhRsT9Hih0vbQGdqHInEt1txda6vjzyGz6gg0EVdzQUxuWBRdyGJk/iLTp+7DMPM9WcwT6KD1377ubzQcq4RKIELFI/zG/14JX+t+/cXDKlXREZBD8l67Em6ewdFtwWoUJ8/PyDApaQ3CZQhGJwCURprOY3N9ojpZwYcA1LuaHIPFyYowLpDj5BJGtj+IO6QEgtFpI2Xwd4uK+H+8T0AtZbUxpDiBOraIyyUG/phIf/WJ23rmTwu2FgJB06AwcxA5JAo0/zkTn00PF4QraCgQaUalDyur2JcR/O4xBzZdxOmcAopsC4NkfZ3Pt0DUmb7qlCQLQcrSYV//1KnaZHQ+rBy6Hi1qvl7G6drP93sksS1pLy1lYV5TOlUPD6Vtq5tSkBmbtjgbAKZaSZT1KTO4OnB1Xcc5bhaarg361D47GPhqPDUMRXc99P92HQwHndedvXlt91YuXGufTM09Pe+xDxI63/aNt0QnXuTtY+G3uxUDEvnJEEhHJdyTjtDq5tlFIHvsl+fEfzeV2oe5Tc+f2O2nzbcOVDNqOKuIvbKI2dSbVUa20+7VTHl9OvW8NYXUDbx7r3ywsKF6eWMYMnZ117vtQ/bSWkPrbATdJ+RrgWWyffkbD6ieJGNJNQMEnGPzjqE2bzTJpPAkBORwRjeDrxz8k1fw4d/i8QFfOvZQTQbZZwnCVkzLpg/T4JSLzUeFXfAaPtloU4sWopApM/inIo8BqtBFzeRe9vtHUxQWz5s03OBwCu/qgxVRIb6iTq4FCNbRFZUFr1KGuDSHL9QC+siBev++nmzRa27zfYavNysgDo5l8IJ3W+6+xe95uAAbmDWTOH3P4/a5faQrOwsMzgc6w78iLviz0nxHZzJuTzaPicHrqmkj9cxs9AWNxhdjocYLFoWJyzA9ITZk8MuNBtiTcQb5iLNWmZTRbz/BpODz08nOElXnxovdr4HAgtVkIqM4mxa3jrbsh29XC50NbeD1YyeUxkRRGCEnhMXtmIxJ5UThgLaH1WrQLX6V02FK6Qo5yJX49fiNzGe1r5v0uaLLL8Wo9iqb3MmIvMSaJi/nNoLN58URuHDXWZk7P7OW+r5UEtAVAWwDXgZn+K/jsQAVqQxPJiUZGnRlFbUQtyxECkcUj78VypYH+4ChsN1AQG5ZsoN+jH5fExZmqy8R4xxA+KpycL3MIHhLMAfsZIkKL+W70Hj6peQd/UxLjToxj3KlxAKTeUcek8dvI+eoOHH563CIpBp2Bk+NOMtJjETafDmQOEd+VDeGq1MGjXz6KVWHl2rBa4q8HUpg1CrFqKHWpWuYn9jBy/kgc1Q5aTwrJmrD6MDycVYT5nuXPXis+523MbJrJ/ln7ua8VFuizWOHaRmj+KLZFmHCZzXg11xBYeY6a1Fl0RGTgn3brm5JIoNsXXgq7wg/HBjLh6D1Ykw2ord6obCp8jELFsrPVD41ZRW5iMQm6VOIu/oK2S2hTpXckNRG9vJgXxNDLqeiidUjrkhha8sAN/dNOsqOP0R5VS4fNhL9YKGBq9t5OumYSyaFdpKqyqXeA0QUul5vKIYto0wvfwGWaaFJlMCxjD+fVZsZU3s2dG/04OC2d5oDRjCxeSVR/CT0re+gOTsElkWH0iaAm0EVCaQJ5w4Kxh74IYhFO9RBa/VtJKgigPiiViyNPEGpR4+6XM3Tok7yonE2b1zVkNhk79Qr89HaeeugHLOJUBtheoylQSolcxLxvAZJ45Ke3sH3wNYsq7+T2N2+nNKGUT5/dxwivGr7RQ5C7keEesFHqwGFsIqoziuagZla09eNvnEi68iKBoW00qoOxRyRSJC6jR1sGwPCc2STk7sPSFceOBTuQqWUsOb2Cjzzf4afuJ5n28mGuyDdRV3OFpAIvKmO8yBnvz0L/7/k8sp+zpc8QF5nFkdK9iFwCPahUf4ZVXoJG3S+9cK3PyZhTY2jzb6MhtJIlm57BIbXT9dR3yA0htLsnMuPsAgZfTeTopKO8suxPjPYgOq1pdJbX445w8PvSXGq8L+MngbFK0Ltb+dg2iZrB5wi2/cZfVet/N6cTZEG+tMSMJLzoMBG9EBIfchNF4nQ5wS1m9t7Z+Lf7UxldiU1uY1JCPeHB3hTpPECIjxNWH4YmZzDqtBp6Zv3JMbOMET1PcclzBwZPAwDX28fhcOYz6coktH3+eKteoFD7IBeGX0AEhHXfwfSAxcRWvExQhwa7p5lphyeQOyiX6mihQEEsgsZfZiI6mYHpX3rWDf6AjMsZzDgwg7rwBhJ8ZwBw+DBs3gyqunoCqnPIHxjIsNPDiNWJcFhsJBcl45Q4wcNFsQ3+1QVgRK6SkeIDlTGVWBXWm4mWH13jaHjsFRIyDeyatR2pQ/BPJA4R55p1fNwAGRYJMrMI34Y8bEpP3MHBzFHDOCUYtCk8vyedyLpkptYpcfEAXz7+Jd8ZwKr2pKA3lAlbB9E92oVjZiZfjZqBU+LEv9mDxb/Mwjq5GtMCaNeeYGBfF95NNlSTrnK7r4UTP8+iwctJm6sETX8iKlsURi8FiaJmfhxwgRfHWzEEBXAt7NrNdz/cA3YGwYvaSjYuqmSZaRwTf0znzCgTdeF1hBg9+d1Hxtei52maOZKU1gMk7HRSkKHg0rgYQixfcS3IyXRnOI3t/aRZ7ZQm99IZIaCAQ3WlNIgVfGcaT3ZwDTGyW7Tsj3iC4qepiFr08IPwW8rCFEq6/JFuN3Nf+jNo5AYerYmliu2sZCVEw/Qt0xl110JW+WXTdyCR1OOf4lw46eZ5i5OLycsqZoc1hYqrY1Bo9PgWG5lcPZnGeBUJ+WYk259D8+zvLN1cRESKhttP5qI1CTIMVudScglnZICWjSmPQkoeH/W+y0vvvUT2sGzuXX6YwQr4QzmABYmvojVnMt3SyMkBE+hTlgrz/YUaKsQyJq2IROS0AyLcEilTmr7ihfQU3swLIqjNB4YK32NrYCsqkZXLdjeeBg96Y5bjF+RBSfrL/D5aoGzVmZUsFavptnjwiRtcuBmUl4RZ2U1TqJjqh37m7qhuXg01UByzhQ/ys6lLTKMhQNCx3REIY5QuOl3XyUr4gk1Ncxmx2QO/0HS8h5fzywQ/ThnXcE36BBdnPsviu8WU5X9NrSKfYZfvJrj+EoE1udSeuQexZwR9nsE4xVLirvxBWpA/kzxVvGUw43K7MAY4KQwVfGvvZh3Qz6Uzb5Fe/B5lu+dyfMol+qXtTArbxpEF3yBT1nHcw82FTm/6Jbey/5898Rnm3k4WJP1OenE6nvmRVHZW0pLbgiQtDLlVztVYG0c0H/Dm/hB8P5xK5bhKigdV8Gx4LSMCoMMJvpLTjLldSZd1BP/ykNIUrkBs62fUmVFMbU7k9fvf4qGhK3B73RLR7esR/JDq6Go8EQpzonN/Q2VoJm/q82jlnWyYkc4v3yyn64oD2/w+5GYnJpmQVDM7haDq86/cT2qADrFvNXGXdgPQOR3mJ8OsUANTPS8ysk1NYE0JsVemcmnoJRLlME8DVkkjB+f/ySXHAvxOt/PizhdZf/966sPrsUu7qEkspTyujjSzjLvWhHNt4FwOzbyAVdZKUXIRyfFT0QRqbiX6Iodi6Onmt2MLOGQfiEMeiGKYjdxBucRWxKI1yQitm0mN0sVHf45BqhVhnTqRDpceuaWXntB+FCLYEFDH1gfXMdT/SUZ0+ROtsXIyopXS4csQuZx8U3KImhpAYaAm7Ctm7rqFWqyY+ThbG78mO/5PVPaHST9YTndQMn9E5+FXBdO1/qSK9cgdAjV+o3wBYstVGsJuoRZvzlv9RpTGNsoHT+Rc8jg6fAV0nMPlIKDSSliFUKyja65g5MKzXNJZONF/gmTfe/hupUBXPCW3g37fNwkvUWOXDCHokg8eqxezLSOfVr1wzTXnfmGHdTp+V/bj3XIa7QIx0R3R1IXX0dUv4UWTiPFBUSh6f0Rd105j8TRyxukpC36ON0NqeSyggWLneQ50eWDu1uI/Ko3toqMYlEX06nqZcHwChr2nYMVwYqbGkL8pH6WxndLZT/HQwHpOp1dzrjmS9/52/8o+Oc4LbRy7MJkKjyYiXSLai9oJKblGr18sJpWNP5Kv8bVHAwv+tQj9iNN4T/RG5BKxbu466o/Xc+rNU0i+lXD3lrtxFjipKKnAmeHER3IraWbUGTHqjORuOMfr/f/Uh/vPbDkOqi57MaFtNq13txKQGvCP7efHNGMVncXP6IvVaEWhVbBryS4SznRRMWQR4YWHcIvEaLrqsPcF4hPnQ+SMJPIuO4gtqWDo8XvYfdtugVlG7AKZjOe7nv/HNeJnx2M13GKxcfyHvKPFMxC5DqQK0A1NRG7YwgeRBTzjAP9qoZg4xmcSLoeL+vP1rClbw6aaTXww8AN0Ybeo++RqOXK1nB1tr/BVyhq8Zf2s/uIj+u94Dk5DWOEhPEtaKbkHRs2G2ftgr/M/T6RME8uYELEcZX0vLlkgMb2HiemFi/pcwoO3kj9hC/boIYySJhI+Kly4/7/fd2kkmu4q3G4hhiASQZx3HEaLEdcfPXgGVeCyJ6NJCiPbnc6+qYIY+r2b7qXrYAksmYVXSwlWlU6QdLhRGLR3zl5UHvMIru1jW/6zewABAABJREFUytEpGLVG+jw1ePZl8lDZCpp0PxPvk87QjZ7E9j7BZ6s/ZUjFLlTWaAKT70emu0KDZRQPf/MkXf4qagaN58HvnqAkIRdL8DMEGGbTNfw0J6ccZuqRqZg0JjYEQJrCSLHjKkO0VeS3bSPwYDorTi9iyRffwvY3YPRvZPpWc+/CSzzSncrYrVPQtCaw7rFLtOr/uPlcRCIRUg8pco2EsNILLB9bT3tmP0eVggSLpfgatH1DS8wIglwSlBXwbfFQan6dQfTaDjpD0wgrPsqQfW/yRayOsS5A3wL6FhytZgbmCzGO5hARd97+HaP8zKx99wEs1na8n5rIjph7AQhp96OqXE59mDfP700iqjSPCfe04pA6cUgdyGQh3DHkHpwR9/BG9x9ESpw4HC6Opwnv06Qx8emTnzI57CFAiKt1RAzBtzGfrtNdNIQ1oN2l5ezBswxaOogrw64QUxlDi7QFDy8P3G43IpGIwMBAnn7mA6Z99DKHXNab/XH1jb/H3PizDEFDsgX4XuzBebed+MT3mDrtqf9FD/4f+99p/+1kX3d397/9Vl5ezqpVq3j22Wf/H2nU/9j/+fb3KvU+vQ3vGaeYO6SStzs2EZkSgstSQ1NwEw6pwHc8SgkTGyKxbp+J8W4rEqcEsUuojtavWcnjSgvLn75AiN90fpQd4LXX3xBOLoLTA65z1jWezD+m0nc6A4/HnQRaAmnzb6P6X35s79nGEMl4mvqnkNekZDRwZVgbrV7XmK2En0c/BFfhTft4zhSmYOu1kbz1FTZsAFqX0uVVgSn8nw5KQHUOvqcd9L8oZWGzjUo7vI6LlOvBOKT+HJjzMWOs4/G4buLFB2FbjwNHdz8jskdg0pjYN2sfqz9bjeFKMr/LP0LrJcHlvlUV0KeZzXf2h5A0HsMd/wVWrQKZVqDeUzsdYDDTbxDR13eS+3bcSXBZLO7HBEQhgEPay6PZaaSP/ZjjtmdIQ4TJZrp5frvczi/3/MKQijsASLgtAXOnmYC0API81EQEbaUr7B0+rEtkT9YWds87wW27xuP+RMLoe1+kX3OJE14/EeQYTiS3oPxXHu7jVN+XDKh4kImbshgwP5PNw4WkY2BzINWJIQT3PcQTZ+qYsm8o4vuMyIP8scnaaI3pJDe6jG9rfBjulULVxN2Eeo2g6s8qun/oxj/Og2f1/ZygHK9+EYPqYhGpbATeFYTZU07ChY1I7BZ8G67R2/AIbu9bi4rOJ/cTTjrFRh0qSRt96Uq0okC05jSU9jCyx/SQdtlOZNZmoqYNwyszGu+d3lxPvY5MF4W+S4fUGsLXF2YSVe4gzFbNIxse4eyos/iIb8OojMVLUsm+E2ewXE6lf2Uqbb0eaLvqkMovUJhcSIMyjPBiH3Qnb8My1cwHv31OQEsArYHCImKorJaUO8qJKhKG7a8Sv8JhdWIfZqc1oBVXhvBu9dF6Em9LRB+j59T+6xyK24JEFI5SMhuph5SUU19iU+m5POt1kt0dzOn9Eq+tnXS6JLjXqvn46Y+xKqzkp+VzeOphxqncrHhiPd92ZFJoTOH2MyHc930Q1ldkZMq+5kp7EsYt8dgzc2lMmEBQ+Wk0Tc1cmnaJ2ogi9vYOZVfMfuLdQX/7Qv4KJIow51wnY98uzOELaR/hw74H99EbmQkM+Mc3ZQmJob2iiAevPcH6jI9pb/SlY3E6mcMX0HCkCP/ayzf37XdV8Ga5nke+foD2KXaUmcKCRNOfRITsIO/6w0/VemqvzUSZpEJj0jD8wnDS8xx49AtZx4aH5lDak85kqYEzNxJ9yavG8MelIDSh/ryjdqOZv5MH4j+gv/vWwiawNZAGZy+G4AT6Nb43kX1Lji3hXw/9C7lJzuWoy0zmn8k+qV5JeVw5F4dexK3M4jWllNjr5Qy6tpj60Hp6Q3KI9TmJyasCq8uFTLOQTt0ZIJrilD7i00axfko0J2zjuVI2GUNBHYnnt1E1aB7yEBktzcH83BLP/ROuctamIigjCJWvCnOHGW2TAppSaZhmwy/AhCRY+DZqz9RS8GsBkiOLqLnzNN0BGgZ+dQfqwX7IXpGxYNsCfhx5qxgjenI0/9FsBTbS8tNovq+ZlQ+uxO6ChAuCXlzE9f1kj+ziyOQbdDFuJ6oR6ZxsTcav/ioOx2VmJe5ils7KgYqH8UkMoyHSiK7bgUOhQ98pPNwRrXYGDSgAv7HYlCocciF4MjjoMFNi1nOSZ270OhcKtChtWqZdNfFmrp2MRDOBYcuxx99Hp+0T5KZONF31iMSw2/giuTGlZFheYoB9AF4tpTjkKtwif1IU8F1uOH4/3Ylueg8t6bcmtU2LNxPaEMqWQLBefov+utfp8O0CkbCPSywsLMsjSrHN6Eflc6ss0qqwInFJ+MYeyG8DL9FkDsUpvoWiHHdiHBSnkfnaZnpUb3IpQImj28iKg/dxfchldHfmopNfx+yycX9dIpekOxhQksgdW6PYMyeJ+uBc6pxuRIpuQIRsWAZlPRGknlyLwuDJfQ/BdiOU2gWdlWPDblG2a/o0gAO9qxGvniScUhHd/r5067vx6vEiY8N9xM85za+5s0nV6ZDY6xmddzfvvPTOjQpz8HDGsXWFi1BdBrWaVRyclkVkTSRJJUmUJdqYJZbg1VaGX91V7K2jmXRsEscmHKM+XEj2qfuymf2vUxxvyaRmRynPv/88vy76lU6fTp5//3muHWoGIa/OoceFymb3I2WsGXGRC4ey6G85hEgfdTPRF/9WPH+W3U9Vi5G0qhg6XEbK4yZQMNiFjKl4HbhIi7qfYI2N2rULkN3noDqyBrvMTpuXAVfWAFLCOhk7cgz5xhD6bd9z0nySwnmFfJS2jtrfChiVrKTW631ef/kOgpMtJFam4ChWsH/W/htPVkR7+2xKt8kIWBRNb+Fp/K/nYFNobxYx/N1nEouFvnzNCt5NvgxouIatK50xZ0cz8YKIa+NXcmTuOrS9J6hOzkMf9RBj4wZS4iEE7/vV3szaN4aKIV5czvSnIS6IB0aO5LxZqNDGks2F8SfwS7hOXyy83mbm3F4ZI+oWs+XuLQSVzKZVI6NPGseRfe+iL+nGFeqg1y8GBdFMVcH+YJBULCZrOJyJqEQapiZnaK6gW4wTVW8L8pIr9LWlY/YMISp3F95NBUS6Jaj7xrIhuRuXVI7bDRKHlHUr1/LUZ68yqiiBNXeeIP/txTT1quGjWwi5iNoIzv9yD6MWHOP4vDM8VK9Ht78SbYmWxoTx/JUpyKzyIl/SSapNR+XC7Zg0JiCQTLE/D3l28L08kSXvKvDsDUIUmc/S40v5eenPtEXXMCZY0FZRP/kzLbXf0yo9zJFYYTHrZcqiLlaNlvko9UqSFiRR+GshH3m+A0CedhGns19m6lRoyNvH7H2zOTCnDH/XKmYn/EScdy3jvWexz36KqXvTiCuZzJuvvcmVfje+lbC+ZgiWH2cRskRP/IkJ1A/K57BG8NviEuoJaIjhwq/TsGU0E3dVoKHq8u7CBtjdYipbjhGfc5EuXSrtGQZ6PXqZIIcdQXDYVcBHsgbqcodSFmjmkZ8WU5xUyMEZB0kqSiKhNAHzCCMikRaDfxw1UjllYVsoTiomriyO6QenY36sEX2HB/7tgi9qWP4reyLtvHr9QfIK1jEx4CovvPcCuem5XBx6kYPTDvKLRxNlIfBDhxc12gc5Jd9Pv1KYTw8N1hApEfHgqfsxdXtiU15nQscELgy/wFw17IrbwXXbTG7fm8rMAzPRvtbOhiVbMWo72RkEU1XQJ5XzkaqVmIRSEmSLCG67iytDtnJliFCYNF0ZzossYMdT2cQWHaEzeADnl1qZNOhRzsZ6Ud01jjk9Fu7YKfjCfWonLauGYlIW4WF2o3Co6UkT80eUQMGv79IzxanhjIcYe0wl/b7eN304AIkTDFYp2VIz2Y99RUrPasrnr8UpeYvUup+4Vwmr9XDGbSW6KprUgjhqklKwS3pI98viyfYc3hxwP86adrbctYWsmIUM0GbisiaB6AwOqYNOn048PMTYbG7aFc8BdyGzmXDum4h97mmy//UuYudbtFSvJ7pBQr8+mT6thp6iAFQmDZ8sOcInrrmoy+Pp8xCS2FY31NiFxD6AXSdh37xmKsKvYFFa6PcycKnOl34PNQDefeGIW/0wN8+nPK6YHWnnCbENZXxzAelHdtElyeLshGaavC/yc+14lkWcQOI08Ib5Cn3aBlaVZrFk9xLOjD6DKqCaYXPOsqNekG1oyGmgJa+Foqf/JFQp5r6SiexI+53FARXk2g235lKXlZD0CuaHVJDXr+FKSwah9jjm7XmQ/JRjNAU3EfTBY2wflIdo0v14q3V4VxUSmT2SXZEQ2NBPf5+KV5uTmX1pK05DEgUDCtAatZwdlcNte4SkiJgxN6/pkkHewDyagpu4rxViuu9gWawUixuUgKKyjieOzGfXvF0UJxdzcGYBL868HYCE7A3I+43kT1pNhzmMt85txLKrjtt7RDin2hh9ejT1YfW4VU7WGeCA05Plg+pxew4SXOobua+7frkHn8YQrHNa6DiTSur0FIJPD6Pbq4vLmhFsiFzBBgt8eHgg5i2zsU4/CKHCweHtD9IiLyJf3U6XfhKvbFuDn38r6j4JIx1qXssQqPbaHQM5a32d0mHvsLllmYAWc0P+4HZ6PZV4G1R4eHnQ3mnCo6+TxsSJhBcdxrKvjacny3Cc+BE//wBEf0OOFg6fidqiJD7gEpJeCQq9iurYeszSRiaLREwL38unIW08tWkMnocmMGa3D5HjI6k5UUPx7mLUehfbrXY6ciPxujyQoTuG0lHUAfvKeCn/Jd4G2l8/zq7R5exSH0ETvpCcKdexBwnoH98bRCq1cgsSmTAGDaj7AovlSSYdm8SkY5PIr6+G/YBGc7No0q/pFmXmX6wP3YFJN2mz3ZnJeCvbCBpUzzXLQ6TYnKQd/4J+43gOzDzASeklgs1aMGiR6UQ4A28hfPs9XLzsDVoxiMU2XDhoTJyKVRQGvM0nPbDGDxRKG/8/9v46SKqz6/vFP+3d0z0zPe7uPsAggw4wuAZISIAECCFGXO84cVdChBgBEiDB3d1hYBh3d5d2PX9sMiTP/dT5vc+v3npP1Tn3qpqq7tndu6+99yXrWmt9v9/q8efoLL0HjSSckvgqdBod97vBfUG/cV8rNDgcOBHREKWnxb9lQHNZp/UgNe0+FG4K9s0pwqiow8s2iQk7jtKv6Udh9cHmpqL7HgdHJh8hviSePjcbbn1SZkoUfDH0Jm7KSBK0HjgR4dWYR6felfhaADMENmOXS5j8wYPEplRybcFmBlW8j6LHSYfGj5jzVnKyvHBrLwbGDFz7VO+7kXe0MfhIElejq2kO0uOM+RW1/y7qLGBFQWTv91gq9NgC5jFnRwpFCRL+WPgHM/bNQKPTwBvQeqGStGObbl2rDzdDNdgldoIag7B49OHsEJgebo5KJ+18Lt+fuZfNC97CnHWDWO+lZPTBHwGwwfwy+3yKmKmppb16CZa48fhKQwlqXkyLh8AWYpMYsDsk1PhtQKyPJDrPhcQb9/HpM5/y7GfPApD7Sj9etp8IyZtMe2YpVf6/0O16notOCO4IIUERwC9xaUiTRKwI0HN+2BfghJrwGgwuBhK0QoGK70RfDA/r+W3Dezj2Pcu13TAqy0RjQi5223Lu3nw3daF1NAUJNADFIy+x7NAMusR22ma0EVB5noDK89jFEgY7lvL+S+8TUx5Df3A/w9cNZ/D5wcQsj6Gccowr87hLdY2+RzKwDrmXSffNZfWp1Rj3GakPrqc2rBaRQ4TcIsfQb+B/FcfjBxjEBsLqwmi80viPZJ/DAd3eDrzrXZi020DtvbXEzoxFJBHjFIuxqD2oGHYPAeVniL/wC6amhwB/dA09qPoNDDvrBKKR2CVsvXsrqdU/MfG/acO418YNvO6q6KLhRjsSSyh2ucD4ILZbEUi5hBhLq1nLnKbbCMn9M/ezSDMXc5+Z9ePWUz2kmp5ZPSi+UDA8XUiNOJ2w4es+/Lx6+bPhfSRaB7/5w0udZgaX7cW1IxO/6sv4EYNWDGFjQS6CUbthvFzJMrNpIJHyvRQu2iEy/iEyi7yxFZeDNhqRw0FuxArSonfxrF8fm+sg1zSb4S/tJjczF/P+2wlNgAlHY7FqztJXribl+BYchlHs+EzoxxeTevnwcwWhZjMN64/gEXF7j3p5+GXmxwpxkvyJAgBoZs4DnEyOG/AXHA4H0mR3fnjgB7o9uhlSeZY30z4jPfVePln1PHI/D8Sx4ZT1Cdp/Xg5PXh76IkniGm5cHMPhwRaq4wOp979AcfRshh0Yh1/nKBxDLyO5UkBvWhwXMy9yM+0mTpGTTAf0OeCwOZtPd9fj5x5Ed7KZANcLBKlviedKXNhrSOXL3qtEqF1RxWXRFqRH7Mwn/UY6E49PpGFKA4FDAyn6s4iyw9UElV3n6+cep/LT1ezRtvCdQkp90U4yrgTwyZhV+DlfxOSEdyRdDEosJMp9DK1RIwkpPgqAf9c0XJzRnB60kz068B86ltfMrxNTJMapnkx+QCPtEtiTfg6NKZxhotsU4CvXruI8cOLDOmS6GkJrozDpTYytfYza7nt4/pl0Zg7zQC/24MCgHBiUwyPur5CSl0KLfwvtvu30anuxKYU1SdpUR8IFgTVIGikFK2yctpGnhj8FwKaYTYhfFbOqYRVvS9/m7j13EzdLiN8uu/9Zdu8RMbrkBbLksoH+6ANkSBWcsFm4LvcBkRSZzB1t4CKyQh9AqfDjb6R1/7H/h+z/L82+/2oxMTF88MEHLFmyhJKSkv8dp/yP/X/ARCJQqzVgs9N5aigNWhMp2WXo/MeS7v84O3QQ1fELo/84j099CLOe3MpxMZjECo7PUlLjJyxKUpsUqU3KyQNZmE54wgvdA5p9SqMSb4Mak7aXnaqZKJNNBFy38eC5B/l61dd0ntOzom4pH6zax83+F2konI6uLA25VQhWVP6NM3v8/FNs9nqQqNR4Ws4ATgeHJ23GLrETYL+NRlF31dMVkIhuRB4v+NqorPPHUeWPeZCRjfduxCly4pSF4Qhwx9pvoKogkh+bPbk6ScNnT3+GWWHGIrew/aF1WNQG7vriExoi+nHzUDCifgTXMq7hV1uCzNFBb4yI3XP24GFNYsw1M4hEOOUirDIr9el5xDGCFRnVHHQ1UXIyn9E7t9NvEug/8oKeZIToaVIag9HKJBQfKGbGvhmcG32OXq2wYXbe4u0esnIIQ1YKEKs/PvFmY2AnLTX+TCitROoipyyukbOjzzIl+yk+V68lPeAEYw7HILZrcfWQk9KcQlFiEVaFDbvUjsQmxqWvFUefHjdtGn0uN2kJaMHV5QApjRVsTu9ky8LtTFGm8vJ7j3JmzBksg/sYf2o8etfjiBGRk5GDXJFOxeEKvLd60/6kkvBqExtHbOSlvCxublqCfqEBhwOcYgkWpSsGv1j02iA0/hp6/pvCs1kpVZRtP0rdjQRGeC7A79RQOgNTqE15hYxFX3GXK9RZ97CuJYRl65fRo+3h6CxY/vMYAkJaOR69AVXfGMxGb3w6fLhj1x3ArwSM6icpoYi5w6/iknKNmeLX6fWPo9c/jtTTFzBoXamQw0/HXyc4Zw89VZ3wCkz1msqu+3/lmE8Ag8Tl5Jtvi+2GjQuj9nw992y5hwuZF5D42nDYHIilYhbuXEjR9iKcB/KpC63DrlQRJJ6CXANdQak4xWLEVjOzUr7k7uR3OFE1nv39alSicehcheChb6svj377KPFzTjJi/mm2iK2AcyDJzjvXmfHOBZoj9vH7wggifCYirVpBWN5e6NExtG4oZ8YYsSYIlW1Op4Pe+l4ufXEJnafgRIsQYXUX0xUQgZebCqvBSmd5J3qNEZm1l7iLv3Lug8GM/tdoCkulBPV1cXiPmWPLthLk1oep6QkynKD01mBSe6HUd1KRkkqH1ykmdofQmTqM2NQk5CfjcZSPoyhzHkrRHkYooaE8i/oD0fhM8WbNE2tYvXo1SpOwJFqTQ+gxClVsCm9Xpn41lc6yToryi4m5cZI7H/oaXcUhKuQG4qPSSB7jgaHTwP4/KrnphD8CH0D1zGc8dO0ZYDwAERMiWL94/UBfi4uLQyy+zXvvmh7Ilnu2ADCy5FdEIhHFCcWk30zHr9UP+XuHkAMNbwaSLz/CSyHQ7dHNV49/RXzlQhI23+D7Cw/h9vwvbB3SyAcRK6hLnIJBG4iypw6pXUuQ8QGSZj7CoU5/YmfE8mT1k7zvKtSAbpu/jVnaKN6eHs1XNU8DH3N93XXyNuURQiS/vhvJvC/34zLcg5gpgr7hj8N/pOlqE0NXDcUzxpPAoYH/NqYsly1MODmBqo+q8E/zp6YGOgOTETkdeDYXMfJ8Armpvjzw4wPoUkU4HwCHTEFrZCZdihoag2r56lwGnT+HELDoCgcnfsPBiYATxlx5joqQ33grVI/LY9v4oXQaOomgg2KV9PJVyRJ+KbuXZ5dcIL5yCZf0czDv+o3kxm5+W3aTPr8fGSm/kxmNpXR+tpjOpddoyJiLOWUuvxZ9y0HdBzz707NYw88i/T2DnJmvA6DqzyVr9b+IT63koG87ImUkIWckPH7tcX5d+it97n0UJxZzoDwT6wU/fEc0oxa7Y5cYMalMZCjgUS1cdm/jfEgbgR0zmbtzLvkp+TQHNANQ1JTAF3MWc8wAX25Zxtibq9hw3wbce90xtHmypGoMb8SfI+PVMtZVheN+XI2LTUAr32wdT5TnMM40BaGpV+DwdHAy6wzNAc30OcC9W8uQencsBhtisXRAX/bMhBY0FRBTGcUDp8ZhcO9AbBcLa5bYSc/4c8zzMfHGnHN817ufPlZxaLCA/g5qCKJfYmOT2U5kjQy7u4qcGQauhR/CLrGjFsF73tDQLaa/fjFuKl/qKj+m26ObK8OuEGZRkWqaDs4esh58ktJuLdaw8Xz/4Pf0u/YzUw2eYpA4+okNvMHlvn7sKg+aA5rJPpaNV6cXrX6tOGUC4v6zoM8AUHoo8b+lKbHreDrBRg3iu+Xsm7EPvVrPA+aV5Lc/gTXhAJ/9YOGh+kxutCyhzX0fyXVr6Z2upNl+iXfdGrDf9yvZiasolifRqt1NZMvzBAdWEmY8RmCjnvwTLlTpLzPtw2lMYxqiT2DOqEIyAg/yduskQgtK6TCLiVz2Eq93+6ISCVqEBtF1CP2NotEdTBvtxo3uRvoMEsxqLzybi5CZ9fQ3JOGa4A4IQZCkS9cYt+clVq/7iO9iZqGKC0Ef0EhXnJpmr1N0eYi5MOg0hYGeaB3buSj7gCPJQah7m+jzjkKp72JwdRgTx/txsScLi9NIlzofmd0DsVTDi0kVjAyw8PFnC7BVhuAV2kFYbQgAvZ5htMldSHNdTHCzBr+SvVhaDFT5folJ3siDKvgLdDGzCY7o4WFvBwen/6UZ8jkXRO5MGfs8AYMEWrT2sCFouurwbO7AJrGh6Zeg7m7Ebgtk7mY5ve7L2Tevk8Xh2xihgpKMAhq73Ans0xNS64ZB4kpMuVCdv8Zo5XizlDK9hpTmS3i29+LeXsmp7GZuJG/BYVYxLH8YLl5yTiWU4NbrxjMfP0BXSBKLho5n6lOXUZo2oDTJMAR5cnHGPjq9Ollxq0hbJ/Zh7aVPSfcZwSXZbY0Wi7QdkWIcbaHByFxkJN2ZxPkPzgvrn9jB3CWZ/LnrIrYOT8xxCjbcuwG7KoW4a4V8/eFLiIf64bHrKJrhVfT5upMvzQcEKsdOB3zfNo/EwBA8POMY8sYjPOVp4nC7kZJHfwL/Hn7olyBf2omv/VukTTI8m0uxyqxMboQw8yAOhyym9L5I9rj7IraARCrBWv0abza5MX76BGRdJyj8Yzry2Xr0GitGlcBY4d/iT/rNdLqaTbg6HXg2FdAZ4M2FoZ8CgmabWWHGIXKibv4NCGTto2sJEVvZ2g9lVVfwv/ks7cMyaAxqpNe9F51Gx8gLI2lvmsr6Fj0tdZF4LYwksvUZbkYIc3hLBIhx4vPOT7zeqqXtxMscDRQKNxpssKsjjGCvMJoCmzg99jT3+i4j/sZNaqXfkmN6GpFTTLZUhcF/Efl+IjJ6PBh2fQGHRu/EIrfgEDtw3IpEuHbUCGNLIkMSdJbVvg7w7SIltxeRRsGOO3Ywb+c81HoJn4nmsTz+Mutef4DaDjdqVnujMcZjFxtZuHUa/q3+jH/vK8bd08UQ6RIUeh+s0k6UtjA2Lv4IgPEqiJXD+X4LFoWAmm/22MFbXYI+3e40B9GLT3G4rQGvstdBJCLO4zI32i8jEUl4zv8UQyLKeM7kxNFsQq1T0aWBdt92frn/F5YopuGwO7lnyz10eHWQPywAl85z3B3VyApfCz+1QUC+lIzTd7DzbhNdPgoOTz6MXt6FvA8caifz/hyLUZnKtju3cd0MoxpgTlcoDxdG0Z7SzLU0gWlEaVRi63Jna6uKMeHXCbOXYYmbR54jGbO0nasxtznRYlwXYpsQzR/+UqQ2LQqLP9taJlFX8zTLR8SQnpPApIMP0Ddej3tTIC4GF97qglCvOqwagSK8ZFfJwLjS+djYptrBtB1pZOyZiefrnTzwwwO0+rWyd/ZeEjqeoVfxAOnjctmdnAvWR0i7EUinNojw5jtxMSpxE4kICv6NFsNoqsdb2Dl8Hf2u/WScewGLTEy9z1Xq54gxaFvJicgR9h5mV2ZfWEi88zgeFi2X6ody9fogjPfLB1D62MCtewwyp5FHf1tCtxbkia7UhtVicBHocK8N/pOz2kzmkUyvbzRSq7AxMdpcmRi6E+3CYtYX/I6938TEExM5M+YMlSkC9GSYi403xs1kU+UGtI1KkjuSKYstQ2YW/IATJh+CjWYkJth89+/CONUUct+W75GZ+vlx9AaGRFYi1cUxbacbZ0YnUZi8jo8bVxLdvJHx/gUEnq9GLG7i4MMdLAnYCcDu0iep105krPRr3m2L5gdDL0FbY+hWzyM135WCpAJ2LjSwNuV5SjfWEp63l8bYrIE+0FQdgOJ0HaZBzcTkqphw9Tl+fOBHzF4upHuW8Nio+3HOs3E8ZBCW3B4AzKIeKkwyvuqBq4H1KEafZVrENMa/PZ5fRv+CSWkiJ6mQ7g4pVw+vQyPRoHBTYDX8U/hqb0cU45LPUTa2mZnBapw4yWqEECnsDoBBSjihduXOahV9DX1M3Hae2tiH2DPrB6YcnoK4/7bPbtQITBbBVV2Etz7OnRvcsLj04XgN2sOHYhPrMCjyCFAIKNVzLilUJfggcXWhJXIkVZGCv/99LwxX9tP32pdcab+XWVYv+j1C6PeOpNNvH3YnbD6dxGO/zkI3vgyTaxJObrNoLCrO4snoEl7pbCEVB2GDJnGlYhI6t3Uky2GcCwzLTyHzZhKMN3JqbhNVmssoLEJKRqsfjqtIeF0aV4JZUoJfnsCZGeVhoGK4D33mieDcjdHFSHlMOQp7HE33v0OVJQCPCw2kpYto7ivl3PAPGayai8Qm0EO/oggl1RxPruMO/Ce8R3JMPpI+DX2SdkKTuzApT+AiHoW6s4fh58bQ69bLzWHujD0G9j4bMiv0lYch87Xzx9376U27Sa0VZjSBq9hIqHk9BSV3oI/L4/isPfRohf7i1ueGe6/gL9lv6W8ZXH3R9LSxdMNSrg25RkZOBraoTk6PP0eqSIS21YUTM1SYHL8wX9GDmxtYnHa88hMoabNQrz9GzJnh/DS+kiMzvuPKoDpKygMYe6SPdlkodWF1yEI/JLXdjydmHWWbDn498xCVQWcwKU1YlCbkJiVD7PU8OCaHy3El3PDcyrjvwqmISGfzoFw2U0+J0p+Exg9QKKDbJuieavsTeOqzheSm5dK9Qs7nn37Kv158kfEyGZ+YTPgjJILWnxHRc6qAz8M+Jb4yHovcQnlCM38uPIcp+Coz0qspMjuJmvwZXYFJeDYVInY4ECFilz6c9R+9jTJ0HnfW3klDYAODAwZz6tVT+McoGZxVB5nNMOcbGvoaqOiq4JkZz/Ch6jvM0jYCGwN58IcH2eDxDX9Pf//fWStg9DCybt46npr51D+O7f6yhrhCdwpjWygcIsIzxhOAGevm8FMHaFtKkFoM9HlHYZfIGeLhirHLSMyCNM4f8MW78jfaPQsH4lkKmy/YbBRtL8MzyhP/9H9PSRbvLObYC8dQjnoAvVxIuied/Bqn2pW+xrvIW/QNsdEitgb/83sOpx2pSsrUL6dyqvIUAKHuoQPHL16Eq5+cxqfuOooXFFw2GTEb5XhcS8DgZ8Cn5ioAnz7zKSOV8Ks/cCcsTAP39id58YfN2K3dmNxtGAYbeSEbftTvYMv4SGZ7TcftUBWxV36jcVYBYcl9HNGD3/5sfPWh8BJsKdzyb9e6444/CO9+mWliETa5il5FNSer+/mz6E/cLYnYZY9hk8vQjk6Cc9cZcXEElzIvcSnzEncPfRUQYmu3TfClBucMxsPWhzhNSmOwQOVQ5/0ztRYpqSI7jkhfREHhSCeN5IhuLgCNfl8yOvAQ+y/FcX3bRMReFvIzVOSFC8XETsUIYuPqSUos4GTDdLqlIhAJ+oEAq9ohoOsuJseJyD6WjS7QTvUzRj7zP8OrylvNc4tF0ZXG8B+fRTY+mAavEPq0bVik7VjkFro9upEoJNiMNrbfs33gqvSuKjb2g/JWsW55Vhc7Bm8W1nCnjeFKaEuo4M+YCuaHCwn961P+hcRuIctSyyiX3yn9YxK9JwcTvkFFWvVXpFxaQ5efiZtNd/FTxjWqRlzG02gn8+xNVhSuYNOSTdgkNqR2KR4iC+HzT3Bk+gnuCX2WiN0V+IqlOO9+AnafQTrxAv4S6HZAb20P83fM59jEY4T0rcGvPZp584V9m+PWszK7aJEmSuEmtAS0UOYpJGjnJcwjf1c+npuFceYa6Hr7yTohMuoZAoMWERf7I89u+B2rtReZzJ2MsYtYHvAAra23x5NX/Q0ijn1Pl38C3HnXv/W9/6+ZWCz+t5jf/0n735LsA5BKpTQ1Nf3vOt1/7P/l9leVulgsBqmYdv9WCuR6TrttQGSXMc25FACRU45PvTBRfdGbjHnUy6yYFcj5r/Op8VsDwJ9PrCVW6eBmwWSCUlR4iasJqwmjR9tD1qksBuUOIjq1nNNDFtIsc0cVsJ6jyh3o1XrCKkwENEZgNcsIMBzDr9qLigoj03YHUhaVxenxpxhbDxuTJ7Fp3+O0tM5CJnaiz6/Cr7qVV/YLFIoXpucBYO3sI+HCz3QEp1G9AH7xdfDNsXgi92dhnm8guO9tCkOfIKArgR0TO3gv4Tdy9oyk6HISLlOd9Ln3AeDW60aLxMqogB68J14iX+aOX14sKVemcjPtJrHXhUo01fQ25vvBru5+Ys98i02upmbySj7914+YJK3McnmEyqgK+lxHYVAZKIstHXCGUxU2npF8R8CG++gK09Pq08rQa0O5MegGfW59ZB/LRinu+rdnZ76VAPzxh5lM79LS9HIjBmUFx7MrmBz/BmfPL8TdtZi7D0+jTeqkJVjOiCvzqYiuIHWjB5MaX+XxtZ9jn+LODtvjZH+bjkUWwr5Z+/B2SPGUtvGI60qKepMJ9E3ketxh2nzbiBE76Blyk9EJtXjJqvB0h1yjjcxnMvlX3SF6XPsRd07DIfUkTyNHs3g3XbEPcvLloyScu0m/dzgSm5mu0HSUWnAKMXU0ncKmZOTCC4ybfJW5fjWMiduM/WAlIUUnuJr8HUHNLowN9Gdzh5n1l6PIyLZQnqikyecmfq0TEJsUON103D33AmemXCAiqJQDVXMJLTiIwtAD52uR9IfQMe8gODuRFzQQUF5Ne8hgKpZKmOexn36XYPaFa2hzn8OSwUGIHhJxuf4yKXLIcBMau/N4OqOKY3C+42TW97PYdv8eOgvbGXlxJFzsR79cT9O1Jo48ewTvOG8UxTrGi8azZ/YeKry+BJ6kIXESGfveJKDiHAVemayX38fzQzZgNETzcr8Srw4vdBod6X3zkcituPt1cd9nczD7ueHvrqQ0rpTEYkFget2rD1HxxhfcN7qEi8YMxGUW5ObbCNG60DrSJW48vnMBRnzRzdRx6bNLaO9Lg0gQicS85TqF7ocaecU7h8Ahg3mu5Tleew241I9DqkCiEByU1qiRtIUPw/36VcY3pvDSsiO8WHCSnio1PkviaQsfSkD5GcwqJSNOZTFu3imyXpzOt71OJEfOouzrpMPtCJ72u0m9FMWBQccId7MgiXqWsAOPsm7lOtwsdxDdKVAWLmxbQeXBULrmL2XW40J1XuHKH5F0l3Ncp2BV4M/ctCShkWQBcOr1U4i7jIREDUElFRzQ6+6/4XQKAdGe2h48ujzo9hTQ8XL533jfuZ3Ede1zJbApGofdQWl8KZ89/RnPfH6bBuFMd+TAHHr/z/fj1n+bJuScysCWDgNKRwcuwbd1KqTualSeViJdq0msBU+nhh+B028JlD3tmZ38NqOAfKOZ9z++C2uxCl6HzGczSViQwCuvvYYurI4Vig7+teoJLgZ3YzVYaboqrPlZq7PI/TWXnpoevGJu07r0NfbhaHSwful6MgMzB9pdPXg+SafWDnwuqTCJmvAa1P7hVDQdQdp6HodEilh9hJ7t93Fw6kEGe1uRKhtv3zAR9PukoVE2MF61BaMTnH+jnuvSnONqzEzc9UP5zhbGyd0RBLnU0xNpocveTLPHH+z1h6nqP2m2+9MYXUmEdjTOHuH7F5pODfTh8ICof1SnKawBOExK9AorXavWIzNnoehyYpFbcIqcjDk/ktEWV85OPs65Z/N5IjqTVc88SKtfC78u+5UIGSx3E/4uGeHn2giCbjppDmimJryGbx75hoSmjTT9XI2tIBZxvGFAO6nbQ+g/MrOU97uHccFYxAI/M/mzxhHsEs3JmnBO19zH81IVXm2uTN81hrIFFk5nCc/6ggk+OTqUwSdG0fd4P84WC2K7hfqEbIpj7kDvhM/d5Fzq9cBmsPH050/T7dHNzyt+JmfKMbI84IAeDuohtfMaq1ev5sT4E9ikNpT9rjgC25n71lH21r1Pl/dVboYJaDKVGJ7QQoGsEJVuJpf6H6LrxASCG4M5Ne4UIxPqWe6WS23vEaaGVSP1gXoXP9ycj9Ls+hrPaCFNpuSY1Y23js5EKysl03sOG5ZuYPXq1QD8uvRXsg0TAHAZ6UKZpYyAPQEYT4xA7XMZxcLNuNjDeMp1FteGXkNlUNH/Th/ZpHPJ3oHbqHNE2j0JaJASVpqKyceKI8TBIzmh1Ed28EhkNXatmITGD0ho/ACA8tI3aazM5FNHPyOvJ+Ey3oBFZkFulWNts/D7hafYHfAMLnP3U/zAdloYQ5RCjk0uQyWCpW5w3FRHpUiOQRuIxA3M4XE0yuKIuLETbWsZ2tYyOsv8Cfxbsk/vrmGoZy0SqZ1tIU+x0rGMlgQTW9Kex9WYxPTD80lsHcrZThXG8xPwXtwGIjccYilWpQbT8HiCAvKYE/gBPv0F7KofSxef0RjaSHr5W1x6dyWXZx/likSOr9bG2bml/DFPQORatKNx6WtlkCKGkAm/kBs6Cm16OHOeb6cyqpaPAgV8w3M+WlzOxHJvbjKmYfp/zHfFYavps57ic+lJ/MvP4lt7jW5/AY12ftR5ogrcSbj5IzbTyxQnFJGSn4J7VyufhJlYUwWxsndQuSXiVrSTmbuisM6NpCixCIvcwtXYItKef5mw2C423/kLE3YmkFCSQKufkV5tL8N0bkw/OJ3qYb24BEfi2i9UPHvWF1KVtgAXZx/GFVvJ7YknQ/8w16IE1PiLwh6Ycu0Bhl3sxz1ZRGK7F+PLHuHn+3/GoKwiJ3oeY4pygMHYzDZcA13Rxnlin/wEd3r2U3RpHfVVduz/SqAqqgq/niS862/g3l7JscGu+KaWog42o1G+Q1XwcZzi7biIBL1Ag58HpYOXMDqsjyM2O2JVLwY5bPGtBweIXMQ4I3oY21PN0fn/IrwmnHYfIWg7uv9Lrkgm0Tw0B3v1ClZ+lsWJ8Q50wUlcaZzNeKWMR6WBmJeuZ3zgu7Q7vqA0Utiknxl7hvOjLvKn++vYi+sJKjuFUTFoQBeqKqqK76O+5+vUFZxuOERBohajykiuQczdZgdPGlxw65DR79TR8vBG3tDCS00yHG79xLiqyD+fjltTBWafC8w42kbjA950+HRwzAA1VhHvdTuRWVwJDf+GZm/BF7pmhjvMtbzb30lmfCO1UY2IfVyROORojPG83w1uhlTaJDKWK/1o14diu1FM+vl8FJ1zBA1HTT/9qcCLUJlxFzgdiO1WGpp/YY2fhEEOJWK9DLFSTH5KPtoeT0L7v8XDLCSakkbmkdviyojaVbjtjiR3cAuHp7zJ1JJkfpJ1M96UhIs4jKHrF6E0iPji6S8A0EWB3Cbl5tk0KnvFdHkG0u/aT6vHLnBArwOSFU6GhHbyQd23nBf9RVfo5Hd/8Os/R7NDQ5NNjEQkxbJ9NysKRrJ69ZGB8eVwOAAR+2bsI7IqEk1/FEOvDGedTER3+0jC/QbRMOQQ5wJ3YHIfSkSVP3abN+vb5cz6+X7UU4xY7OIB+lGA4I5lLNWXkHd8HK3+7YjcBJpZuUXOsexj1IfU81CxlMKjI2iOPE1e2FaaPAR9vPPBECrNZbN+KK4L9tGjy2TOl5NRBWTz4dN3sq3oBST2ZN6Mr+V4k5b+YRP4YZyQ2JVYIbDucyYELAdg0P2DubGtGkNFEy464d6Uq3R4hdXg5ZqByClC5BThp/bDvLqQL3iaN35bzRNauOTpzcYVJdT7XmLEJYAJrJlxE6O+lGdNvzNDe5Zc6wUckqscm1KEu9sZRrvp+PGuXbzZFgy9MPTKUBKLErFKKzF2uNHUf5H6wmF03vBB/bfc0oLCSiROOUmy03Tr3emWy6iNUPNbxG9IbW7IrQJryV96zc1/S4gVB/2LvfYKRvrGYE4chNj7Bt899B16tZ6gvkm8ZnqQCZ7Hee3Vu9A788kMnIzrhWi+fOJLflzxEzKrjFe9YhifHcB52dM09Qq+mm+rk8gSYQxJl9jYHL2ZrNIohlw1oTQJUdE6nx8IbV+JEgOl2aew28NwOj15pRM25X5CVOuzjB8PdbpAfv84HcuYDqRWoa/kDM6hxb8FmT0aAI84X1rDh+LZmE95ahrHx6xki1jBiGF3EJcQhrnMTqdXD0GdD+DVfpio0ovkhqt4R9rFHeEtLP9xERa5hbPLP6TJqOSYA/wiqrFHVNNb2MvBRQcBODvmLMvXLycvJQ+5Wo5SocTcZ0btq6bTS88PD6xBYpfgabybu5sUOCRGnnY6GbdzCG1aL/pC7+fjkihez36Ui5sSmJOTiOQVCTVhxRRHFnN9yHWuD7nOPF+BAlDU2EB4/j4AWiNHk1z/JEaPXViUbgO+Yo/6GpfixnOmD37sA6fX80jd3+FJVQ8NiZPwYj0wgh98YaZa+Itvd4BIROmo+2/1hL3MbAKVw0BAUCMahRyro4d+lYBGDJaCb9NDvOe+g8Xr3OkPN+AcMjALcM4k0K2NNI2jp9Qb0ygLTrXQ38xyISWTckNDxW+f0ZWxDJO0mOkHpjMoN5e+jDzumnydR5K+5cP8N6krVHPPwXuIK4uj01fLmn64Ty5izz0StjT8Se32o9xTPBJoQ+9tY4gbzG/QsOezkWjm9yDNtOAZWYD8VATq6jB+Tcznkyk5JMzKYdqBpymLqedaxjVELnfzxZNfoSCOZI9kjr/0HpMcn+A0OTllgFAZFITBet11Xvcw0/rGe6SIt3HdeX1g/GxeJCCtv+Vb3DNiuDbzDUR2G0MOCih7h9jBloVbWBo8HUNzP4FN8fh0GDC4Ski4MZ0xbjLKr8dTe4cPvvumcdazl/z0HKaWx3Bj6FWQNdPi9Rut+ZPx760Z0P39V2AFC+Jmc9oAzXZoDzBTrRVQ5Ptf/4ArofC7YTQv9HhTJ2tknFRMcp4/diLIHZQrjEuRmcdDl1JWO5jWrZ68cvoVLmcK2tRtvm1s2/4OrQeucsZu/zddvMV2J5eBWa+9yqxnF7FDtQOtbjjjwhp5K8jJEnMphwzwrVZJXdJU9NogTPJuYnKv4trmw9bhZTyNidhHYxn18Sj4AD588kPO1eVjc47kubuFte/19+7nePtxHJMchHXfR5nPJxhcDNxIv0FCQAYbT55g8f8CledPSBk+dyzLH7sfdz/3fxzLe3YDY/Ag7doi1j32G95xQrHlX/HD4KIjKA3d7L2jBlXQKqTuanpqmrn4+iHigEP3jKHJs57gjmUMuT4RtckNMsz8ueBPMh7JYMY3M4S5472z9DX2MWPtDGJnxILGldxdt2l6uwMTkaqViKViXKKDMGkaWeQKVVa4ZIJJRybh4daH7DkZw58YTuOaRmQtMuo+q6N0Uilxs+NoboYev1h0GgNWmRUfCZxsVWPbdAf90n6aY2bQETqIQFsPxZYv+LoH1DcTCdk5nrEfLKS16i0kNjNnUqbxbPh53vOGwJ4mHpc3MdnFhlXpSnlsP12eXYxVCUh83/pgfHpdKdpWRPjWcKSDpdhktzlKy+LKUNYb0ET4sueuNkqDHmLdBuHYZPE8NIYlINFirBTWi5D6EC5lXiKjYheDpghoSJeeRuwSaPNtR68UpH6G5AzBW6/D6yl/ZBYZdomdWt+1LAdy7O48/fQrfNNSwQ/mSdgk/QAUuO9AUgEKTQ0eD33HwtTVNFXeTlAefuJbejR2SrT9nAkvJbbhY57e9DTHso+RnyoUxQV33kew+DJND/xOi2QETqc33Q74uH0Oz8+cDS6hZKoW0qj7nUDJUE66fUle0FMAdCaB2XMWAYMCsFvspCxKoadJzx5SuTFsNHddycS9MBHTkya83VLpUKwBpxg5Zi4JIWpWtgpU9qH5+8HppHTIIGJCfmBR8lc8VDwMY0ATSf0mFAYD3X5xiJx9JBYnElsRw9Cv32VzRzcWQwdufW6InCK+ffIrlktVTFWH87AWXusU2HZMQ8dQpS+noOkMVWoxq9pOkkMgr+aEwCgxWxZuod2nnWHVkxh+eA2dxlRYeQc23yBypr+K2G4l1G89AGK7GIdNGFAhX4aQciEFcYCY5TuXD0gC/d2USn/uX/EqtXWvDvxv0iSorobWv7EnR9wUaFH1HkH/Qfbdsv8a8/s/af/jZN+ePXv+8d7pdNLc3MzXX3/NqFGj/rc17D/2/25zOoW//v4+HBYrOqeTtb0w4nACxvgSjGnlBJXU0T5YwprH1hBVk0VA08YBWH1gVTOx0ljK4sqY4G3iFz+YrtzHH4n7WN7zNcvXL+fYxGNURVYxKHcQrdFVDPLdh484gJpwF84HCQ6LVzsExdfhH97CCbWMiPc/o9BnLZ7fi/Dw9MAJnDXBjeB/cbNqMJr+Giz9AdS8tRGvW7SZAGLbrWy9WIxV7oJF5U68dDLzL/exbvwvbPOT4x18Lyp9HmqdGrkJAmoq+VIhZ8GIblq9ghjqDCGj6H3agg8y4pA/icWJ+H75No8sO8qu0gxMhjAOJG/ErDDTFNFJYLUX2V5b6Ng3iXiRiI7AJGxSEU3q/Yw7k4ZV3YtznB9bc4TKmFmRxQPIIYCHPY1EiLtxn3uUm5I4Yh+MZaFhISalCZFTxKgLo2gIFxbi468cp7e2l3mb5nFAL0dUDkkZF3G3+zDqmqA1dirrFL0RXZxpTeaUaB4j5++lxBjCuMDf+SZiIha5BZ9aCQ6gu8mTtuA2Thm+Jrx+ME6Rg9DaPKr0cnaOPcH+iMNkB3rwhzGX3+8RqodnFj6IMieNe2YU0ia/TsXny7CrvLh+z3VuJH6ODAiXWZA6eyiWPEWOupSHYobiUmPGpPFCoe9C5HDwV9zkL4fRKZaARMyl/BnEjz2PM7wRp1aGKD2GnwIX0+rXyrz3X+b7HQ+zd+ZeZu2biTXCRFG6ncqAHFLKsihIKiU/5ianlOArgQKnk1bP09RO7Ma3LZa0c42ccg9hjZeIswYRksYKgkrPUBFVR2K5C3UX72XcqwepGhvLcQMY5R10ju+k5GoJ0Xb4uXEIvtFZ1OTXkJIXitPhRCQW4Z3gT3NgPwFNQgWO3WxHLBHTVd6Fd5xAxZhUlESHdwfhdQ0Yu40IZEKC7bZ/SyEHcdHH4KUfQ3BJBI9//Tjb521H0lCD3eLH++ZpZN1UUh/bRZdrK4nFSXzy7Cc896kQiIlWW3jJE5Y0G/G/9s/qsRdvjGTE1DKumhJotoXhl+rHU7VP8ce5WigXkH2iW9xDzv+iEWRVulI09iE+vEUMHnvxV8R2K739bchI48zgMnyO6CnXbYIl7yA39iKzGPBuaqRp8kkysq+R15oFSqie7sH2eatx1w9mkTQHfd0yDok3YlXKGeupIa7Ih/JQBUaNg0rZBzhEZlyll/AY3E+jrYaOUg3b79mOWdeHvi6cQydHMDHmDF0/eNPmU0x3gD+BGYFUHqnEpa2eb2372L3xQ1JLE7E8YsMicvBl+Jc8iUB5YZ9oJz8/n5SUFCQSIZnZcaaSWXtmoe3RElX1Pea+FwlpX0GTx3r2zKskSO3HsYC1fFIxlf7ct9A1P01eah6jz48euGcHpm0n/UY6nt0BdGY1cTlmBU6RlZWx6wkWfUxEeBEXfpmJTOxKYVIhFz4WdPv8YrNQdR8C2TT2B10hUCJsIP3T/fFP98e3cgfb/YUBs756Bnr1ddLNQvIu/f50Gq82cvS5owx5eAhRk6IG2lOyswTLfgtjnxpLwCMBbLi4gZFP3YdC14nc0DPwOf8Wf35f/DvTtE8TdCaH9KtihHSBoK8zanAFw+e+xmbza0z7YyVicxNHp1xm2MUgQsTDCEzazWs/3EWItwpV9Rk843ywetvwkUCgzEpL0FfUpxSDeyhFmZs4bn8PgKkC4xgB4S38cs9m3g57DtX1RlR6Pb23aJP/vOtPHtIuxGG149pRi1XhisnVj6Ipj/L6Ej/EOthjdlA70sy54cJcNfHoZGHOiq/hUEApTpmIksQO+t2MRLY8xw7/T9BUwMqqRFSHsmC2k3dffhe7xI5fqx9pN9MwebdzwSQi1uGkILOBPeMFWpDSuFICXZVk+keR22dj1OVRKJJs+PfMJU52hvHhj9Guj0RsSmfXmOP8qi2iWztn4F4nysEvvZQ/FX2scn8Re0khvrU5VMeH4dYrJd4TWpOLMbxdjH/8nZTEl6BXC4masyY42yyg+LQ9WlKGiShMLKTdpx29Ws/psafJkMeS4H+GnZ1X+bvuV6cdAqog3TSaER+kQ5yU4xOPs3TDUoFS8zRskltInqzGu1CB1CZlTboX8XXPEdvwL3Z73+SIrB+/4Fpmbw+kLbAdXaQQWc1PzqfTqxOn2InkFl3oI8mPILaLWeZchq/zGebXD0WcsIRTfd2IJSC1Shl3+jZ9zxeDm5hWFYGnTEVEmZakgql8+9hGxG2zuHZsGEo/J6+rHyY9bSYHy07hkMjRa4Mw9vRzdNZxctNzOTLsEi8kfcv2hVu5d9O9dH5Uixe1tERomDj3Kq9lVTGlIpmS5u1o9WLa/EBdAXdo72CR8h1eGnWQAtO3OJ2pSC0GrAo1DXET0HmG4Jv+T3qj6sQY4sf8zg+NLpSXe2D1NVAk/h2AflUhCbl3Qu4MerPyMF4HZXYTno21yMx6nCIxBT4LKbAtpGOLFfNlNYrpPdx94G62LNyCxFQO+HJUr0Lr+xvtvpBtucJG5XBU5jB8WnNxry8l1HiE7OT36FTOxSTKpDa8jRb/Froc8HInTJp4AtG6Z/Dq9qC3uJ27N9/NtYxrVMRUoDKo0HYrsOgt2KUKLEpX2sMyuDboNGVh1/Ft86VPK2aZSMzB6QcZfmU4Y08lopDci0UlxlU3AolTRat/O6dm7aEhuIFO707mpteSYvGiPTgeDz8/uuU/0+8ajE6tY+rNwTxzRcY9/9pI+6u/8HzvSJ7+7H7aA1zJG5ONq1pOuDaPAHMeuwNhhVWPUX87mPZOFzgdKh53niewvAeT6wh85elI7HkDyfisk1lkXj5MX0M0RX8WkX5/OsnPDiPwy+UUtsOds87zu8MHp1Ogx2nV7uXArCRiRYewxuSyYwiMdnmAiHwJPt2Tkdk8GKTp5lwIbLdeYpfDB5tjPHlnj7I2aTgWuYVVa1fR6tfK4Tt3IRI7cOJEapOSWJSITqPjrmAjQ205fNmfRI6uiiUqEcUpDbT6tVIaJdBTPmrtxuZipyaiBqtC2BqKEPxaqcOXiPYncSJCEeZPS+QIovMv8VL+S+y4YwdN4X5o9cPwGBJAn4+ahBJf5u2cxzePfEObXxtPLrhA1x0XOCq/yMeIkYod/Njmj4dDxoKMnXzjOZeiyPGEqM6j9+wXtPGAJa0CRWtAky9qoz/tvseRWWRY5bczKv3WRv4MgA19YOkxkrF/DbXRXlyJA5FThkgE08Tvcal0Ak0ZK7ls1OLdJAQVXXWuqC8Lz02oUJfgX36GQUfSefOxs9z/y+NMkvthXyQgm8+Nucj06/EcrYrnUPUiel1uoAmWEl3TiH/1JVqz9lIdVs23kdWssSaw6GowpT4TORT+E9i7bt1TuNQXQF+fF3k/zyIu0MLopgc5Mf4EZ8adwVsC7mJw+o7jw7MnKC0PwKf+ZyzSLqr6K7lTA+10c63zNKVvfsqoyWlcjs6jU3UFAE2/hsHXB6OI1iMSi2gOcWfm/kTqomy03V1IhNyO24YwRGNElM90UOZWBpQxqTWSkIZ0hmZfo7fbA6vRwfpl6wbuc4QUnvRt5ZpdR8lzX2PULeGNt97gzJgz1IfUM2/nPOSLd3Ekroh+dQf11hbUPedQKh3oXOGMEUIlWvYZEjmjP89MlROvxnxMYgEisSDxI5qcDxGaUoU+rAwT2dAj/PYUF3iw4hDFB7vgnZcpbvXkYvjdJNV+Q2NYD31RUBhcydTYSoZFL2Nd4zp6e8HdHd58/k3EMjs3z6ayMSeC2CdLqAwR/NQLIy/w2Kwr+HpbeOPJJwhWOnCGGPDo1tKvFKPqq+Tj9nS0Ued5qBUqDG7M3HcvGdcEH+fzpz5Hr9azVnqY48FPYpzmZJS9grk751IWW8aLWROJUHVi8y5CtaiJ2u7xtHRXgAeoLOG4GwaxYKMHLiEGuJt/WIDvIV4PzefjRgvF8o2MUGYNaGPKujwYpykmy6uSbzStSB1mYtOmcMT9VXQaHVa5FZPKRIjnDeJdCtjUehGxXRjLJqUJS3ohi8YWMiyogVwbbPa38elznw789v1uMCzpPUKcITw3u4B7GkxIz2Vj8gwn2q2VF0bdRat8BWfMQdTN3kZQ2DAc6TM54DWVJRuXMOT6EC50X6CjtAOlhzdGV1/8aq4SXtyHPNtKl0cHx4euYajmFeoyDOwaup64preIOdNGFVHszA1B2jQYq4eJLs8eLHITetowKAV9x4JgJW2OEAoO3ZaVscqs2CPqeGt8CY7vQmmOWUPBKX8KthSw7/58TCoT54LhWkcOP1yLY9T5YbSNaiOyOJjJkXD/A8tYuquBkxznqPwEsRPsqH3VbLp30z+ei+RvOoF/mUdLMR1hQ6hJnyvMJU4IKjlOgK2IS3ECHd/zWmg5mons6iDMkTrsIikSh0BB9IcO5mhg7M4UfGVKJPpCMvZtQ+8egHP4Q/jY0zDFL+XPkGrubvyGmD2/4NrVzIVEWOwKH4y7B0O/C5/qH6bU7KAvJ4/Q3HzcwlVcU/awvBUee/ppVtsjSXC1MvPAAprcFrF9qoC4bfO4iNUjCpFUzCvvvTJwXeszL3JI08zyFjmHxVtI7l5OTHkMhSndxJTJWf3makqW5vLt1gj6lX20hhmQ6erIT8nHNyCV7/1gbZ0fXaEp+Ht7sb74SR4QbcTFu5XfFv1Gm28bb3SBmxi6vKzsn7Gf7GPZqKzQ7TqK3OE9t/eHVgdjTyWxqtqGPq2IpW6ANIwCezUNIgtJIv47mVngtgabUyKlJnUWno35XBu2hTafEuwaMZi6aPFvwafDB7nYxswV+6g2udJUH0WAPQbLkudY4GnlF0UtHTfTWLJ7Hl/0n8F6I5HCu67x88trBn7r5z44Z5By4FoE3h3exPu+SLV2PQA3LaCogNWh73Ow+xFMpn7GuDn46ZHzdLif4U4NTHIBqb0LXb6NzrMm7Auk1IXXoNGpcIgcuDa5cqXwCOdw/rdabiBovO21mBn7yQbuWnmeztYw3E5/xo8uBtpWfYfS7o3NbMMpktAWPox+ZSVbZ8zhgNLJoNNR2GPdMNtuUz7a++ykbJnPrrCjWBPfxyvZi9A/QxkXP449rnt4fu3zwGpWr17N7rm7+XD4h7x25DCX+e/15v6yy8A5HLx/fTqZ7ZnYLXYk8ttIsZq02UTc3E1ZbBlWcc/A//N/z8OzAVqiRhKevx+FrgyDfw1OZwYekR64hXtSa/QhpDGR6PKnqYysI6z0AgpzA9aRI5m/eT6e0Z4D56s8UklbQRsz1s7AJ9GHfqUPtgO329mQOAVXV9D4QcQb93L8xCvsugViEpVDckEyMu3t+1XeVY6L1YWar2vQWrTEzY7DaoVevzicRGOT3c23vpDo1BHyzO+cG/kMxvUBGAFv40jO277g8XZI7IYsnDidDhxSuUBrLxbxfa+wnhaZRGj61TgwYXTzY/eYX7HIOhhVLzCndSxfT6JsGqMODif8YDiSNAk2qWNATgIExiWAKr/P/vFs0j6OwiHdgT5mOREv383vu7rZNzIBgCFnuynrPsbYfXOJuboZg8aV7Y8/PPDdbQu2MZHVDLsOr7z3Ctvmb6MgRShO+KS/FxdRLycd00k9E0q4ZSLHs48PfNesNNMS0IJZZWHSPh9a/MdzKusU+soQdrr3UhfWDxiJljrod+3HKhN8wTs1MCR0H7HyJt7JrOeupig0R0J48Ob95D/gDhFLQSxhWNC3ZPywmn2OPNLeqmJC21MDRVh/+bsSuYR5v83j6mkDJ566wAVXVwpFGs50euCwOlCZXNHoNCgcgwhxmcAHFde5x1rKiHPJmHz6Ufc0ou5tpjDyK75XH6P4yhPUB0rQ+bow/HoJsZcFX602EtRBBSSnlyPaNp2oLi17Z+8jb6JQqKBQm/g4qo9y+ygm3fBE1+bJvLFmKgf1k2M8QZ0BDhscrPQZyR8Xcgg7mIp9hYOSBIFh8YpsLsrmoUwfHHvrWYOmq464SxvQp2nABVb8tAI/ix/MB1GDMNfaNXaCh/8Tuvr3hN3fJSX+Oib6L8ti2bBF2BQaDO4B/0n2IRTf/deY3/9J+x8n++bOnfuP9yKRCB8fHyZMmMCnn37633/pP/Yf+y/2j4nDbCO6zZ/7lWp2GlVYLCJq8w4TUNHJlpmf06vtpdN7O/dfeRe35gp0NcOIyr+OpnUGZXFlXDTCI22CszxXAzfFQZwZf5XKqEqaA5sHqj5OlUuxX0umPGAQSMGrL4v65LkYtM1s63+IN3pfpcJ5lOflIrYs76LGbyejlfCDH0haf2Sq2Er9xWTqy+4m8MEZnLrpQWjujzQGNZI3WqhokbhruDn5ecLy9uL+Xj6Oke/SHHEFF++xBPtEknWglyHqJZyY0s2oc/GkOCzMW7WD2ZWHGdfQyMw/zJSMm0x51E4SixNx2zEdyfi9eJvUJHjZqNcIukW/LF3DdDWcMoYz9FQ6PV56mlMnUe7/HqXBr7B6/WokUhs+TxfiU2PHpbcJuyNm4J6HSqHaKuKo9Em+8DaQ4D0ckUI8AInHKQiexzULTmzT1SbaCtpuPTsniKAwuRB3iT8StZTymHK6PLuoNuRzLnEuALsBP5sHEwP9KZNW4BDbuZE1G+82E3NLu7AWHyU+rJ9Lk0dREPoYgXUuhFSHcy39Br8cnIN9jwrxkiIIE5pkyXRF3zuI9Q2TsQX+ic3Ui1MJRwqOoDKoiHAzcSXtOFc6niZD/gu6JohSQcYT8NmZIfjWXEXV10ry0S9ounYXTn+hasXg5o/MW4u1qIv7t0zm1Ni9ZHpaOKf5hnqlcL97PKxou2WI3XTsXf4r2Qmv0lcmoDn7tAZ233EIq7SbrBwv3Nt8uWdIF4rWddTEnMMY/AhHn12NjwTmbbmDxPNpnHswh4K07+j06mRq2XnaJHm0q6fxtmYyJr0Mu8VGp1HgGW+wwSZpDptMjWQ/1kNsfhuvS17n1OpTOBRKdt1ZjlF8iscD3xNQBOFapCop+nYhQN/lJUFlVBFU74LNZMOv+MeBflAXdB0fqZ2nDi6iTxeD/GFvauKTMLie5mbaTewuI3ARh3DznWdoNg2jor8Slz4pIy6N4LeHKjH4bkGhU2FZ8S/UyTYsSndMLh4oZk+hre5VYkLrqRWPoXdyImLtJKQKKe6h7uDy19IjwqfJm+E5KZDVg65VR83JGkTdgcBtZxzA7OKBUt858L5i1wSqF2+j13saXYXNyI0CKtavoRGJYRC26bn8+GEGktE/UJZwGYBelxsoRJ0MDzrHJrfPuSkp4bMOL4afEeEZM5yCDFjwmxn/8GaeW1lEhV8R34geZW28gEITeUppDGrE6GLlk7Yn0Kg7CBT3UH3cSOURIRAhNbVzUy9G0uuKd4cWh81J0faif1yLokPxb3Ni5WuHGMIQitLUVCT7IJJJmHrsPoJKQ2iJGMHNZQcxOo1E0MWNXj3VHdXUhncwWqhbYM9rP7IlpI/OY8mUl4Wgv97M/N/DOTbxOPkF+7AdGc6nYacZ3BSCUSSmLb9t4Lcdv+bzO4vRv3UHlzK/QukoGzhm6jXRWRFCv6ILV49+zv/iCSG7Uc4az3Ntz7Fl9hZ+n/47vsm+jHrhnwU/Sq0St1At5W2tSLVSnFonDgdEXt+GxGGjzzuS9pBBFMULzq3daacuop6+hM4B9KjTxcC7ETqULvC72UFkVTTqHg8ujzUTWH4GO3CofBWu1zSIkj3RlJ7E6kyCSDGfeMN9brkcMDRzPrKTMDe3AQQlwJh68LKN4rSjFjsNOHHiW3oW15Zyfmm/rQFpdziwGS3EXdpIe+hgalNnYZR5MeNSNvIiK64xJpwht8+7+Z7fmXZ2HEMVTt61etOnt+II+BE1oDJ/jR3QO+GIAcbbRdiwIHdEoJeXMarXh+TLI/Ba9hGvDi3l2KQjhNon3O47ZgULVXpG+Jzly7IYlMcm0Wq3I+3KpV4fTn7rOO5Keh+zfjIVMimN8WXISpws37ycM2PPMG5kJQ8OqeMNbR1yNyXyQYlUVYmIKDnGQyUP8cZvq/mjR83bXXp+cjr/pit324ZdGUZaXhqmSWL+vOtPANx73FnuBrHyRuLOeqJxbGVE0WAmljzF5ie+pl5so8UO9s6H2TvzFexunnT4a2jxa8H/Fv2HUWLDJvMl+1g2Q68NxfB8LYMO76M+IZtKidC3ohzfUjLpNO1Sf2YrJcze/jjHsjfQ79rPhOMT8PIRClS8VF5En4wm9FAotnscrIj7kSFKDbNbZfi2+/Dqu7doaIKgZcVn5AX2ce/nr6EPlrFr3FUuDdtLu0cjgafup91LTLqkitx1N+iWh+DReB6jykaX5wg0/UoscgtDlUKsyeF00BjUyE/3/0RUZRRZp7O4MvRXIkXZbGw+RaddQfzum6yoXcH7L7+PwQl25DRWHuTqR5Ppm/snLq1GEnIvojD20hQzBp1XOPLbAN5b/pODVzohLTeGlbvuwODZiHt5BwvKFnBw2kHqkv0ILWrmUuJVjg45wqOR6fhV1SGz6AkoO41v1RGuDK/i28SzBHgEEBI/mTPKXTQFNuHVZ0Pvoge/Tp5KWMalhrnUNLsSa4mlLsKAzUVPcmY+DhQML3Ohp/cM7/W2sXfuRfpdCtCKIVMJ0Q3vsHXFKV4Y5QNSETHlMZTHCL5Sal4q0w5l0nxvM+0Rw1Dp2gmoOIu7TMrU3U/w9utv0xIiRyyTILFrBmgU04t0aMcpiGp8mrr2eAomiLk+RKjo92vxI27LVLKmXkH03Fccb38Hrw5P8lLzODDjAKtXr6aREP7szOSEx2nqnGMIdQvFqQjF4p5I2LhA3gxsZptpJQ+diqa5PYK4fhMvXP6UDYs/5jtauB5qId7+DLNeT+Kbljvp1IjZNfXBgWejV+sxuKoRScSU7Cqhp7oHo82Gm8GD3yXdnEs7j8ORxZhLel7b8BpbF27F4NnBe0NySVZAqQU+6m5DVZ6Da1MT1iHd1Nngy5qxzHQeQ1TfziGVjjF7akhonM6uIZfx6fDBp8OHZUNqubFlMpUpvUzMEbQ/CpMKedPHgt7lSzY0+/P42sexDzdQNvJOVKYxjFMt5Q9/KOxbw1hVN5F2cJgd+FecJQQNTZ7g2RNKYtls7CYLTpkcnWcYVF0CoNe9l1F5+/Furkc9TFjX/ipACXc38KYvfNgNF4yw2Ae++uVfhA7aznWTgTCLhLUtAdw5cwx7erVctSzhTNI/95EB1kSW7V6CuN9Ok18Tfk0jef/l9xmsgIfcwSpq5vO9X9Fx0kbkCiNV0ToKEvaw0g1ivK5jd9jZenohkjwHtdlFHJzwFO69wcjNZkxKEyMVq4S1SteBb7XgJ9wY2ohKY6QpRo3SkoasQ8KLH7zK5eHnqI76ikr/j8EpYlxRPh4id1yyzHzl8x4NfrkD7T7dGUDnwUwUwzu5OL6Anlt6mU7gqWuv4OlhpnfBaXwcI8jt209daB0Ar3kKCGgrIrpP9BBedoZ+VxsmRTcXm08hA36d8wjkitF5hhIa6ElrGBxJF6KTGp2GCScn0CYWfHnrreCpf72RUfd6MT5wL89ffxq/mOiBBDVAQ5iVRJOeRVoHez/6lEvSz+FvZD0jzGN5Mv4gz1Sns0dpYaxIREFSAW2+bbT5tkF6IXN9zCzwaaTIu46ZV0cyaf9CNi3eRIVrBS91wnLxZhS2P1mwfQHKVDs5014BkQiz7XEUUiNiez+Dq4LRSWu4z2klviSepsAmng3qw6Sw4pALa2xDg6CjJLWaSL2uZJcO7nWDY0FwzuHgjz9g40ZYtQr63vfg09An2f/LdFyvDqKrpxGvDi8MLgYcYgff7B7By8ENKCYeReEM5Y6jp1l2YRSfPHuR9NwULpbGI4pKRvmzGJ8wHZ6tcYCRvJQ8Hvn2EW6m3USUBY3uezEpnciN6aTfTEciTmTTqPcItrqzRBXMzpLnUSicyAzXyL6WTXUMWDRitL1aAcoJBJaeROSw05iQjd4u5rgRurbHE9F0AZ9Ji5l5qQeHWIJIrOK7nsV8VDaHI/Pn42VLJkbsi2f9Onx0fcSV1TLz8ki6VvzB6EpPUkJyeOanZ+j06uSX+39h3fw/OWVJ4pMmdyIvZ+Lq54S/gWru0kCm9y52iL7k3S6otcrI3u9HbFwma179GIDjfeMQycXcGHwDjWsa8SVTcDOkURlVSWRVJCMvjqSnpgdRlDcdYRnIzDoCys/wvaeUqREQwI/02R9kJt8RmPMdTpzkjHgbdXcxZU4bd54fjW2sheIJERR5fgROmOYCD7jDiWOzaM5JY/pngyjeICCeRlwagaQ6FKnsHPu3Z1Nxtop5v6cz7IlhbNLXIQIiZFAtM3G/WkG3WYXdaufHF7dRGCGsSb/ODWYfxVi9FuJ1W4ZxwFLyUnDVCQFhZ1Aw3f7xeLSU0BAZQmHIU4iccqJansfp9MGlpwmFXvBFwqTwjjd8J5FSbJFh7zUQdPMLbqQIBVwzbhWdjd01m7YwM85UYX+k7m1Go1cxN+AKcwIguFrQ6u30U1PvL+wpjhrgA8DF1YDoX4dprvmAmprT+DZ0IbeAhxgkulRsUqFYyGm14lVUgC0sdeC6cjJyCJkwB22YlsbARoKagjg828HKH5ZSH1JHc4s/sa42dK/LWRv2Dq+//TogUA9PaXLjWrMEkaYaadYsyqxrmHFgBgXZ2Tx1IZUqm53GcTaGhjQy5uh6XjRO5KNRx0k4H01ocTpbRp6hvy4IpRcMKhxESkHKrVYNJWDIIbwdrUTXx9LstDPoShxTZXq6xxTxehfc6xWBi+ksIe2BoDXxxI9PUBZbRsFQP2R9Z1DrBapTfV0nUVePom0to3rQHZQPW0RycxaiZhHBUf6MOJtESkE0JXElxOfEs0EaSHmWE0n2S8xI1LGprYIkL0jpge6oarLCuugxuODs9kDk+KcO2kEDHHHIueNmGsmFybQm1fHsnrf59qH3mXpoKiENscR+PpTgGj9mb16AfVkbRi8LZqWZTCWsdIcKh5FViRdwCy5g6qj3+S35NwB2zYHUr1IYi4Th2Pi/s+HAKJGIa1deJtX9Z5RdbnQbZSz87jnsIj+KE4oZdGQHTqDLP5jU00/w3YNfM33dvfSM6eHZG88yvWw6B2YcQBQj4mrGVQyOAjoMcXT1tNOxspurpqvIrLJ/++2slCw++PBDZr38Cnst5v824XcZmKVQ8NT8x9D9rmNj9kamfjmV4U/c/rRS38n6pesRO8Qs2jSbuql1hI4O5dJH5whocFI05iF2TswmrjSO0Qfy0I8Yi3KaL/NOPc5jj0Hq0U+Rm3XszPqYmvluBHXewyiJlOS7kwEoai/iuSPP8fpPr7Mk6LY2eWfnf23tbXM6QWcXENCbWiNJE2XyzaPfMFP9LoZOA79m/cpov9FcGHmBNY+tYWLCTOYwB+ut+iQRwnp82QTp7lauxJQhDr9N7RfSfydR9plUSvZRFl+PLErCvRmRSLfq0cnOYZCVITKGIc7fiTjwXp778k4Mw8rBdzwOkTAmF7nCV74wuh667U6yP8jmy5gvsRgt+HfPo8VToKhc9fUq7IpeDLN6GXo5garIqoGCkquZOYQ3L8Slv5eSx7/BLzoFxxhhnVLp+tDXC68bY8djVBr+cY+6PbsxisQo/dzpCE5DY7MBBSxyhSgZXP1tKqnKAILLPZEYtBzPPi7oEyrhCw8x3g4Zq602ghqDsEltiJwi5u+YT2VkJWfHnCWiOgLzeDG/r7wdy3rEHdK81/O5+SU2NuylzeZCol5MUI8HLwZ8DW0zwXcsnY50jlUtwxympc/dhE0kzNH+zf4k3LDQVdmFZ5QQe9rycT0xuef5MVfYFz858hTveH+M5+FWnrv0HD88fI6qs5OoYhJX7NNJOzUUvzlGisc8SMa+Nxl/ahSXRpxhr8dXrLE8RnfnMHrCdByfcJxOr06aA5o5kNpNuMiDl98cjItRi0BsK9jEbXfwUmMcQz9bjv3Kw0w9Mw793f1cVLxOk+bswOfE3iM4YxmEIrOH+ZGucONWP3Y/y87ZZwkNGgYkIe7pIjxPAGw5+h3gAhXRFag1wgL0+f2fE1kZSUBzAPH3x5P1ZBb+af4D/f4v+6/JPvj3ZF+f7+1483+Sff/P2/842ef4757yf+w/9j+0f0wc7kpUY68SHt1IwUtfYjQ9hVFk5FjiGvpd+xlVFc2bGjmhSTP5/cASLl9sYOe843S7FQJQaoXSXtjaFo/5chpMtlGQ1kGbtvkfv/lHrRseNe5oLjaxsnglR2Y0YHT1xWrqZJDjdebK2/m5yROHvw2HWFg0lWKIlwMdm3EZGcQHqjrM7vPwmJRBXxN8+OKHAAwVC5Qbth4dQcUXUfc0IvJ0QSPv4p0zu9FbtaQ74fLQI9ikNlp9unEoosm/kMq8VTvwUDUj8XDlxpA2bkTuozmgmcltwRQE1jP/jzvRe7igm2ZHYpNgl9ixO8Ts6xWhFblwedXXaByRDC8Bq/S2lyIKa8Rq92eo9Fva6sPQVfky9eBUCpMKWZRWzzveRq64LyW+dhCDvMDfqmfSpX2cHDIXm8zGmLNjSCmox+l0cu+RewfOmyi3cDVceK2tdCALVLHlni2ozGFMlSbxvpegRTK/WUBrCc9acGb6PT2wa8KRW/5g3o6F1M6w0+Suo099g74EOHrndkLsMnaozjIoYCgacykz9s2gPKYce7aUavloNAaQOs+xadnnZMjuY/nW4by46UV+ff5TPurqJ8A3gNjuz7Cd76Qj6A4cEzMQ26wElxwXUJcuWkQS0UAf1LaUYG0Vqq5TgqoZ/e6rOObXYEztQGKT4BA7ODWjkSjnFbaOrEOpMfKlTELWYQ39rlO5nlnPC588hDiyjmuB9Qw/NpFWUS4Tjqdgktej87Mx2+yOVNvPGo8GfJPFdLqH0qIWnKr+g+cRSdWYvIYxd0cSGX/Mo8m9hpDPQ5jpPpPCO/ZxIhhwttAsCiBdtw6RSETOuhx0zTru1SRzYFoFtkzpQFXcE5VP0Jbfxp6fO9jvvMbVxANcH6ng7QBXPJsrASW5mbEsDOjmU3/Y2NSOwyBGGeJDR/QC+j3WYxLp8NZdY5DJj+fCrTzS2E2u2EBcWRxxZXF4+Jbzx1grb7ZKqQxsxO7pTZP/PKKubUW5ZQsyz0Q2ivyZq1jB8ep4kpPBZrLR19iHry2YCXlVpKeLuNC1jOFXhmNJ1NFW0CZwpU+bjhhXfGqvUXnMj6jsSAzaQLyaCpB4a7F39HAu/TqH4woYosymr7oDz+bCgT56efIxihvk+Jps6E1t9DWfJLEvkaLEIjSSqzw+di6G/aPI3vUY5hc7OTK9Hp2rheTSO4ALtLVp6f9rqVPfrsJRLPIjKv8csYNKWKNvpO2RXD6JzyAiZTgx02MoP1BOQ0gFIfYdvPXYTe6su4RU9QESmXCO2T/PZtDyQdjtArLv76bwdaO/R8f2Wf9CaxiDVLkUmUmgRHXtqmP4QVjccT/7H/mF71NPc9xnMV0ePQA0REcz5kQG6sAIRq3cw6P1bsQ55yGzyhA7RdgmuLFV+xXBTk/e+eRb/mx5maynshj5/Eg+cPtgoA3BymL2a2FtlzD/7Viyg/zf8kliOT8HtzL5tR0oet1waCWIxCLUPmpCx4bScKmB7I+y8Yjw+Mc1pSxOYe1bnUz5vYe1j67lrmV34bSDuq8Fk4sHbh1VuHVUkR8vIftoNt6hUurTrRxZ+MfAORpCNdxsC+DlDZH4p8Pl7GByQ1/B3TiGbr9o9O5+yCx2uM9An3gCtR7phMcrcXCE/XroMQbTKttCe9zHGHpXMviKPyPNI7kw6gLnTDBRLKZSbWDN3ik4MjtpjxrByWFfUN8v0PuOOzUOlWcH4tEyalNmYNR406W+wCOZd5HWLmHHyftpVJmJc6Rgrx/B9cHXKY0rozSujDktyVjffQzRg7cr0EWI8BQLtHwVKUWsTSpiqOVetG160Kr5NK2FX7ZCVHMge0cVcMUMZy5Nxr9TR256LnGlcdy8MApRSjjTot4k+M3LvFH1INEnd1M+1I0UP4Gu06GJZlHtYBrkx5ljduLV6YXWFMgRQyWDWwJ56q0HKQ8vRuabgNRym2LxxQ5QXfuGxUWxhKVG/+N5jlTCo+6QMvcKmwaXIhKnDBxz63Ojat849OJCFp4MQe9qwx5nIcbLyFMe8EIvRMvAQ97D0YwcvKxpYHXy4wM/IrVJGX55OK1+rQwRLUQRU0OtzIm7djZNEWG0+DVS47sGhdUXf1s13y26wJN1cVBuYPBVL8ySERhVRsaeHUtnsrDxvOv8XXRXd6OcrsTH5xrjAk/x3ksPMqhPg/daf0olYpxiiPkwli+vihFrPHlw2V7OGl6kT3uedneh0MNpt+OQKLjaOw9VVBkyH3fWPiLoXCINJCpLyR/uzYSoXMmp9+JglZWHvn8IlVHFhvs28NziUzyvglfFKwh66SXkYRaa0600hp9BCgxSwDj1EP4QlaBOv4GH3zC8yruRmfqpSr8Ds9oLnA4cdhHcqnJ3OCCgup75rROYMS6f85aj+PkvwvusO8mFkRzLPkZb+MO0hUN29A9kK3TUSEVUZtyFf+V5jK5+BJccY1xOMM3T6rgcVoenehL5bsKcVBXTw+zp63hPIUOba8OtKAJtawCLyhbx/svfI26r4XSnP7t1Kiw341m0azZ9Hk04b1UMv+4JT3sA7Ts4XXMP5tqXcJ2s5u3X3749roMbuDy2jCdDhSizUteBUt+Fl+l28MiJA7sdJh4bh0NspjY+mTHpV1ie9R4/nVqBs7MFB7d1IpILkmkriWCbm5ZHk/5EJtYz4fg44kpiaAxspCTJgptpBIW1synRXSXWM46jM8uo83qM1KK5XGmQ87oCUoLHItkXTHoXmEfJQXQbDT9IIaDO7lLkE+J+Cj/riH+MkavDriJz+xy1n4b7z93P1nlbufbBRZKWe2AP7OaSCaIlYqxaEWWxZeg0OsQiK291wRK1iI6Nc9AEeNLbtYmIOi2IoN4GH3UGUtuqxP3YMKQ+VuLHlBCbUMM5NyOyjDymZFZy0izGGV2NQTsEq9SKzCZDr9YztwkG997BPR6baFf60CKBAMNkFN1O2hxLKbZAq9XAQ10GDr23mvbJHfiUlNKh6IQMiCqVkXrmD4yzHkQe7Eu/ZxiNMSOxWn6nMaiRzMJ2QoqOoCtTM22/EKy7kHmBIC8dD7vD0jwvbNW+2DNMmC7IOdLyOs7sw1xJy6VH1cCHUjsFkm6u/ZeowPte4OdsIXPxz5wqv4fL+k5avasAIXD/oDv8Thc57ffg0nkIEWI2LvoCp9jG2xrIUoqxOKx8NP1dZJNkzFJ9DRboc23B6S4ES82yWxRGhYdxb6+g3zMMx303qY83YEp8meW7WlApZPR5KNBpdHRpzjPSvYGFrqA7/hmdneGcGqqjNujQQLtzQ6HbUMhXE3eQOVaOb9eZgWQfgNipYLBjFRWmZ1B6t3Es9eWBY0cNEND0L1LGNXIu7We0XgEUpoqp9buNsrM57US7HcRveT2eoxZhyz02cKzDu4NvH/6WSW5rMBlNLF6fJIzpeDV1BW8wPeY7Ziz6jtPWdUQcUpJ59mXWrlqL1OFFo1FKvknEeBVcsDgYcWU+vZoiihOLOUIB6bXgU/o4M5vvwn/UTb67U0jqqnVqyE3iHamIquk1IDbSFmZk2/xttPrdDl5FSyaQYv+QvsKRNPk6uBj7Ct2aC7yV/xjz/PMIHKohqN6X6MIkgnw03H3wbq6NCWJTyEqmzD/AR20HeJlX6dhyjJSTQpVTl6eN1a1QWx2A+Voy7nfoObguB3elK2vXxuI9VqAenrL4CKLZx7jR9yePf/04xyccpyK6gqzTWVwbco19s/bhYVRQqiyklfNY5BbUeiEIdqRjKvHqnZiVVuqHvEAdDo6kKnlqzeNEWVXIHT1MD15BfZcrTm0O16e9TF7o42zvAehlvs2AsuMiYoUKhdJE2vnR9Hjn0aoWc2jKIcYap+N0OtG2lCB22GhMyKbaImevDuLkDkpEDoyNXWQcWktTzBhqEtNp1Zwh1n8H1eHwdmsPNfpTNHjWY5a2Mu7QVPqBhoog5u2aRku2idL4Kvpchf2hujKMZoeBp3UeTDuSicckE8mGZJoDmun07mRZKyhF8Eh8HIVr7iHBoeX6PDvzblFy5zRP4aboEQYrVvKQG2wztqOqyCNKG0RQowvnRp8jZ0gOL499mSs7axm8fwP1SVM5tWAuIrc3iCsM55FP78G2pA6xxQtJ6zoK0tuxhcqp9d3CXi8Z8cO/ZGfsJ3D8dhQxSgZz1PBi1Qhcayy4Bblx77F72Zi9Eb9WP759+Fvel/UQFgyB2Z28Ov9VUu5JwfnwGZxAUDWEto+ldvI6to+8ic/wxxFflZLZ6OQjbyExpnY5w9So0+gdKzB0WMg+mk11RDWV0ZXM3TUXY9rt9khsgp+ccOUUf077knGnxuGq+xWn8znKR9xLs1YIqk8RAHycH3uWzak5jNS8jV+TFM9gwV/+sAuqrVA/WYHNbRjuwXFYlK5IrCYsWk/mxK2ltjaAkTvvRh7WRWGKJ9eiBf3b62ZIroUN6eu5vHkpQR7Ql/0rX6Y9jUPs4Dk3eMYjj00lhxD1u+AUT6B2wdO09/8zlmd3OnA64YcHfwAgpvENVCYlseWxXB95AaPUnemah1HYQslLyUPpHM3ioVew+bWydtQGwhzjSTMOpS60jksjKlA5n6aydhQdSVmIw09TZnmDcZ2u2HpcsQH+3Z549LsTVxVFysHJnM7uJePqUABOjz3NuDPjmN6pQKuv58xPi7CMtPPDQ/spTS9E7Qavdwm+9M/WHo798CAt07sxK8zYpDaim18h/mIo2k5h39Vf0ohHaykAkTd20BSZiE7rR2TeBcxR86mLbEfdA7npuZjck8mPKcJqSiNF1ojZLqfFLrCA5LVrWOffyF7/HJYE6amYfAJR/WPEF8fTENwwoHcvdopIvOMkFeMvoT49WkBTAan5QoJV2pzP14nFXJ8opcU/Fc9mFVanKy+L+nmzC66MDKQloIWWgBYmi4Xvyiwy5u2Yx3Hzbl74/5Ho+8tW2Gw8V3gdS5aN8phGKtIv8lx9PO1m8QCyTQR4tTQAbvya+hgfz9pDyqAUvHd7E1MQw4EZB7B8Z2GaVsmq9F5I/5knlfNoMRqoqhDW4r9o9N173Hk8/3Hc0t14dOVjXFybw+jqrUyQObjPCn7c0hVUKDhrs/HSIy+y9OFlTAicQObpTOZGzR1ou7nPTEDFOZZVLOOPO//Ap80LU69ACzr1+7m88oqA1OzV9mKT2pDYwGkX7tVfbsRfkiJmhZnmwGacqup/3J9pv02jrreOQxWHcLwhfPfSl5c4+c4FlEn3YnK9pclZdBSpSoKhYwQtv11EalAi84fYnnTm+byDZ9lbDBvrBU6wW+yIHWIcEged3p380f4rW1mP1QoB5Wdwa69CO+1d9ulf4SF3QcZhsNNBZM6fuLdV8NNTF4nR7KMoUIRc3M/68pG4yTzwqzpKeuUlcqKtJKjjWBF+mLUSO5eHXSY26j5UDS1M3zeR60Ou4iOKp0BRjqPWio/ZFbWPGp2XDmeTk5DOZRiUlfS55NLm24aMIPSN3Uw+Opn90/cPJPsuj7mM5vIQgqUGNGmRKNr7BmRIvnj8STaMEIrdO8KGYJTV/+O+antjUUhVqON8KBsyluYggV78bg1ku8ArxQlYNCKOLM+l1i5IEnmK4UQwlN2I4odPFqNabuGT5z8RTnjreaZ2+7BCF8apM+OoyLDB3wogV7WDWmQiXRtA1FvPEujjpOtFLaUTfkHhYoYTE+GOFr5tzaXxUiUx3T9ycWwtbW6HAQhsCmTIFVc6KzpR+6r5wO2Dv58eUTn8FQnqj5BS6rhMh/Ysxz3CcDUmE9H8KM139vH+iOHsPw6tEcMR6fYiltrxEsNk8xEuldmpy5zC2bFCok5hUvDApWDCdK9x4P6H0JrSyagfRWpnKoVJhaTmpQGgE0VRHl9Cn3sv07xXMG1zGu3uHrgt2cNdGhAZmrC5RGJVudNdXMK/3v8Xx7KPcW2oULBcoj8PLETa3S7ICgFu0W7QDycnnCQgJWDgOq0yK5OOTaKUUlJmpQwk+/5u/yvIPoCAslM4RRJgzL8f/I/9H7X/bZp9/7H/2P/E/rGn7zWiPziOejcraZl1XDQ5sWocdDqEjcmkDUs4Bwx98WdEj6+nM2YVjY0d9KpuJ/PCWx/jh+JWRl6PQzTaSnDXfbgZ0wioKmfi0SS85xzjhQlnMWTv59mLvyPtUZFQ/whurdfxKbmEZlYuD7Rr0ax5AsMDOhIrkrFHB3EsuJFLRhihgi80nXyd1sgiTwU3io7Qqsrjzj8FCiSj//04XwZ7v4GAygv0eUVge2IUG/0C+PX3+6g8FEPb4nsHJnm1KYYLY4uZpOhkxzfziLy4DutdT3Jk8gGMCqHC9/3pu5na8C1DqmtotvTjtyedCVde44dHr7HymwwADr5UQUbXm9gV4YgKDiJvcqFqNuycu5Ne917iRGO5c9UmPusUY2oexYjLI2j3aeeIoR5Dk5ZH3HPIzt+MoSWNa6JgRh26RrvbKnITv0QmGoRZEyBQRv6NumSYSihR2vntHSxo9MI556+HKcLucJAgF1CWH61/gDZkFAa+zQs5T/Htqi2IrGsIVl4kcXAuVtcodN4zsVdtJrw6nJqIGqZfGI27TsM3z3zBz9HnWNqdxtDfh6JX62k0lVEW+igeRBMr+ov60YEqzYWcohyaJBZe7ITXfdOZqvgDD3cvLGIz5XtL8K1tojF2HGYXD3SRaQQMglohlo9K1wHAhFVbSYlp5omSm4QHJRB2TkH20df4/sHvkft3sW1QEWYHfNANDi8nfq1+yKwywpsmIDM14G9wpTq1kpMe3QxJe54t8vV0eXYx7dhkEvKfZshdR/lCtpmOGBEa+28ojUrMCjOdPnH4autxE1Vww60T0eAcxgUuQOKQIEFCyt9AYDfLJ+Pd7oXT4eTBnAc5uPo0xetyuOvPu+DPFhw2B21FnZzfWEnf1TL6TlVT9tQacIITO04nfPPYNoyKKkaUnSVN5sBFBLoFAVQ2LsHQu4/TiS/R71KAf7M/sdeGkB5+ggcvRVPvjEbm1sL2edsZfnk4EcfCeOvQvyh76TtO37+RCPEyPM9Y8GgRKAQ8u8QEdy7E6jTRb3udukp/mnJm8svoXwh9cDIulkw8xVAT38Anz37CI/6H8Uvx484/72TzSX/E5RZCio5Qsj2DqOxIdB4hNMaNZ9CyNOSdP/NgQh5ZLYIwdsD4ePLHP47UYsQsKiBjC1RNPcaxF78gSvoBw9cPI7EontVvrOaz+rUcd8By3zZyg9uxKcVcHvILiByE9nzExiVP0hxQy5zaKLSFcThDbbzhfAOA96Z8BadHMG3+ZU7alNQCdocTj0gP2ovacYqc5GSf5FfPBgBe8tHhdILaTwgaFTYU8tXurxjsP5gxin86QEmvz+PLfWexS+04xBYcDigYFowkxca4fTfgkgcdeKBzgA2Bu338yRHkpeThIhpE8Pl09rv3MiflLAallIyEbtY8ISBzbaFyaszFWM0JvHLiONrgGERiEXaLHaW3mooAGYF3nMBHU4D7+QwSC8NwvuDEJ9EHj0gPuqu6adPFU6cP5L3P1rK0fyYOm4OOkg5GPDWC7A+ysZvt/0bDcunzSwSWn6Fb241JaRpA1d3MfoaYK7/fvvayhURdl6ExxtCQdm7g/yl5KXzxxWyq0nSMPetBg7sNfWAtNpkNjWsnu+af46GE73jJE+wOCQ+ftmF098fpLiRp/tDBib44DgYcJuynB8kRa9GYRPjqhnFh1AXGKOFYyFlqisLhVCaOoF50PqHkpl0eaMPQq0OR+PYhkUtpD8vAiZ0LCRFMFENskIwb0zJJTk9EebqQ0PNThYSyToPaJmORqI34sacZ6p9M+IHdWNWu1PiImaqG32750H/0ifnhqpIZP6/kwLQDvD+yAFnmVV72qOHrsjA86r2JLQljcO50SuJLBjSUqvTTyFPvp1zxJ9pIC78t+hOzbwcnaxeQ2zyTR+YFE1jtwX3bXqZstolPnv8UN2cIQU4oOCig5OrO1iBOTcCkFnQWD005xKVu+MF8A1FONb03VUw6Ogm9Ws+FUReIl8NiN8CtkSDfRhp7TEzfP53SuFLUejX9mn5OawyETT1Jg92PoWnvMip0CL/3g48ESsLhmvvHPKSAzV1d2L6dTEDTLH5Y+QORs0/zoTv02Ov5bEohbfZCtjoeYt1SQafpVJAQ4D8u/5aFzVCld2fMuDS2NdjIFPZrrHlsDUsVwhrrddwLkYsIu9TOWdEpSiKhODkPo8GT2UGe5E57DQCXJ97nqa6nuDrnFCdClqP3HIvIuEHQMhA78HpsDFl1j3FR7MqnoXKWhsWjrxGSoy4mJ7Ltw9lRMQfmHsGxbTrixWY8eoQgnnuvO0cNTTRYxeAlEopWvH1pT3VywX4BXwlcCYVi+5/85uXg2NzdzNcOQ2KbSmnEVKKubR2YT6sP343XYoH60eGAwFoDnl2jWf7gCYo825H4unIlu4JdkzYKOlq3NnUjxCbkvd5Uu9owu3jg1ZhPR4gEW0AbCZHNFITBZ91wxHB7B+fZIUW2+0Fs0y7w6bFhhFTbuXDPKfSJRZjlzZycoqPZ8w+yWYHer52Lo88RETCXrHWpNATJOTU6V0j2AXsvrqCvGaz6f1bANwY3YvEtQxumxbWjmj6fKKxKN7wa82gP8GTMmTEkFqVjWWwmJT+eXvderkzsQOqto69+MG/d+xN6kz/3lDzD49se51qmEaO6BbPCQo77CIo/V4J/CTfT8okriSGoKYj1k9ZTE/Ee99XfZP6vzZjSQjgy5WvcO3uYeFAKB5ezevVqTnll0BWZRW+oGfe4KOpd0nBGvMcODzhu0fClaSIt3ccIs9pxNuaTUZ7B9cHXcdyikHVix+GAvoY+Gi834hwUz8rUBu7zhXvfvQuHqxumpVK2agU9s8BOK3/qYHuviNcvpGNO03N6WiF/zD05cL+avLawxk2K4rnzTHF7H1XWFgZpQFEPr8zcwbaq3xgaPIiSJCtuPv1sClnMtIPTcIqcXDFDOxd4xq8Kx7u5vFW8Bv9tu7B7J7HfDFmN8KZ2CEXOFvQJrUj90/hxxVfoNILeX014DQVDtMR5qNEX1zLoyEZ02kCK492QOH0oijuCQT2J5PQAOi61UhFl48iUI8gNEJs7lpDrDhYez6Y3polLizuI96hmQs2ThB7fy4XMCyy/mkFyYTIey3WMqx7H1aFXMagNrNJCtVnHRemz1PiOIdfnCO0qIcmzRwcelbAqdgiW+FTKlKkkJoGzVHgG35+8xFZbMF8vEGFWmjErzXic6uHZC89yYsIJ4krjuDHoBqpbbPztoYOwS+U0R4+m1foTAEqJlR7/tSi8XuP0NA1FodeAazzhIiQaf9XV0t9pQ6cDjy4P+tz6sEvtFFugU9TKweHJtEhmM/rMWELURRzPPo4YWJKxksfdVyHPsHLviYp/jou8zWzovpvPOz7l3Nx9LG3Q4qi7C6f4Fk2sCiqqdzNd2cy8tKs82elJ0FkbC5oXsO3ObdhkNnzMrxOjvhOzvYd+135upt3E6fU9cquKq/3eFLrt4krHIPw9HNSGNWCT2rgw+D1a/ZZzuOZBNDk3GTlcy7jTw+l286IypgOHOZqbkotsduyl9Fg93T5pIExHmBVmtizcQmD/m2wqbKbyfBpb7zQMUHcBPKeFMbI3McmrmbjxKHPqXkZ9sxalqogwL3e2FLzOU/O9ma1Xor42FPPdaqoGzUMuC+GXvpX80nf7HqlCfQZeu+iFQsI/y32Yd2IUzkwLgSU7qA2r5WxWKU+/MJM3mckbv61muhdcF0sHqEejK4RilulDy/kkDB6p1VESX8K1GAFxlTMkhzZ/O5Guvjz/9k/80BnK6dzddInfJaY8BqlXL6GVUfjZKvG8kIn16HDs7xtwSHzo0J4eaGNZ3UYGF1TS59NH/tRMvnv4Z+SiJFLVHTSMu0R38ExEIhHFox9AdGtz7BQ5GKmCpY/uxbNMy2yXD2kNTaFf64lOWUJL/B0McoEbJwej7PWirfMPplx148D8DDbfvYWgxkAMCYVEaXuwa1fR2fwsFxJGoenXsHy9sKZ+8PK7FD/9GUNqHmbB9gUcmHaATu9OWm4xF4tEIhxiBw4cRAa8zbXmexDl7uNi/Tzih4pxNRkpeetJfAb1I768n0Fe/gQ0B1AZVUmvtheZSobcXYXR1RdlfwtSsy8fdIPUZqI3rB6t6wi4UsGgqypOj9xBcs9kfu2MJzOkkm2ybpxKESF57ag0Q8jJyOHrXvi6FxITIXPWEOyudn4fJviRJfElrPxhJYbxBj4b/RnVCdV/80WFOcG7byIxTa+SfTyTtMFipmj8eKrlIMdr8pg/fBYAuZE7SGkycGBvD4Y5Kxh9fjQOsYPK6Eq2LtzKXSmCZp+zqxu3DiHZ0a8VikHSbqYhEpkG4hsOkbBXPm2Ex9vghEEogJVoXfnkhR8xy4UYRrsdXvGEPRkdHOmcjdMJpSOWAtDmeZQDbcmU5U4hKceVDlcddo3xH3NGoQVapNPxbLiJzOqFQ+TELhUeYvUtRFHO1rUMOT0OXcQQynyO0OJ1e2yG1YTh2+JAv1golEoqSGLEVUBpJnPiNWbPO8rqpgj8+gNJK5hGUqE3Xd79fNu+GVfVB3wTcpL9Pb34nmtn6amlSBwSKoY00qNy40VpK7a98/gtxoH/+58x1x1eORmCvSKcXVOO8GBmOb6JrZwomIXaoGbfjH1URFcQ7pRR7OokQJVI+4K99LsvpFfbz3WZgTEI/mCvTEWeqwPJ7KMY/B9iU8YvOHEyOdePnrAV9AcJPodragQb7t2AwcXAw98/TGBVEcVxOzFopyFWKqmJbSXlSgCz98ymMsHJ4t8Wk/L47wQHTGVd8yoWfvI+qwIKKEzN495t87k5OpFvH7mLzf3wW52UKbsF+vOShBK+9IGxSiPpSj2b++FLQweX5wv7xzOr1vGBPgidu4E+ST/MO4zd9hRLPhhBzmAZe2fvxeQEmVTMNBdB8092o4dJhZMoiy0joSSBY9Ld/Hv4/b83P8AgNlAS8QMxo46yyKuGk8XLEJnv55GhHhSPWkHkje30+kThW5tD2L6LnL23jvK0F3jK7zsu1wr7IKebE4PSRK31FpV5mAy7w/5vv6c0KZEel9KW1Ubo6FDSO1KRho9ld+QjdJZAnxGcCph63wN8v+RpNqVuYsMXG6hZXUPNjBo+H/f5wLlkLreLvvxb/PlzRT8fzhDoCH3SAjFoBWmP1ftW/1/s/XV0HFe39ov+mlsNYmZGS7ZkWWZmxjh2wA44aIccZmZmO7HjxGE7dszMDJIli5mZWtBq7q77R/mVd/b+7tj73HHu941xzjvH0LDk6lpdtWqtVXPN55nP5N3n3sUQMpHJkYE0Xm7kt2m/4Bc5lRNzlNT5b8ShcCBxSehX5SNp7+GjgE1kPZpFvUOMtXl3etN0pYmQrBBUehVyb48bst6iebaWglKBuXsonX+fIy52HFOjtvPi+Ft4vGMn/dLXULSvRePrwbqydTzy+iNIXBJ0Rt2gzKTTCQpLPypjC7+1vkyMGl6v8MD/nbX03daH2T0JictJn+oHZIBSKtDZ7IM1ux7jKAP9PlH0aguwqC0s9mpjdcjzjLKoSFSXEur2GKqKHlLyhlITXckDo4pps6mJOzSRkNYwTO+bUHYqkbgkXI2bP3hf25dvJ7lhFA/EB3FxejolSR/9l2fqVGmwNnXh29nL2PNj2TdfrFNaYywGRLKnILk5FiLaH2bZ78PR2Q1cm3CaI+nTB4891AG/BcK8975kTfNYUIBVEOdon0tUaDP0xuMVkUBGbALYnxNPlEDvpPOMjepn6Ih8jscXI9hfYtzZcVTEVdAW2EaJyLvgbv0ssoPsBEV7MtnvNn7c/yLmCB/cpN2g0CEYTczbP5VmkxUh/eaepySphObgZp4a9dQ/7r8rIIRzI7/Bt8OXZYa3sfRYcB86js0JoiKEu7Seg7H1VHuE8XfBW/jqfNF11WEITOKPmbNZqRfv+eMvp2K8FAC32AbbjquIY9mOZfQE9uLIgBanlbBrboy7toTy+HL2zd1HcHsAKsvnNExs4c3ONuTuKpRWOSqriliFKPsrazvOs2Me46+yR5Cp4+nyk2DUiUD3rX/cij7EDR4Da2AExePuw6bx5O/1GiyXLDx++PHBuExceRxJJaJU6575e1g3dd3gtf53mX3/MJeLjEPvIHU5Mev9+DfY93/e/kdg3/r16//HDX7yySf//Yf+bf+vN0GAfs11uv0u4ha0iINLFWwJLcf/ZDq+3jKIcaDv02PUGckenk1mTiZ3mPsx+Bh4wEOCpe2mvswyHbwb8guPBATxxqTtLO55lVGHOsjOKiW5ch2Qj64iBtlQkbVuH5vBt+kv45B9xLNHHwYkePj00uJSkDb3LOeCljFqUye+TSv45faPmVjjx/WHc/h8QxrILNgtDqzPnCZZ30RUrSg1V+9o4Nw5UASKQVOJ4OKidTNf9kBqRA4FcQ5qHDel8VS2YPITmyn0vMbLJZkYk1SUD2RjVtUCMOH0BKKroxl6xwUe3/QzM8uHkVB6L41NXrjZbmaJfOp7jlivQxxqeJrSVgUyk+ig1IfXo7GmYlMn4ddkB0HCzlHefLz+YyxqCxNl8HlwD/Texxnp/Zic/YSNi6ApfiJmjSjLVjBcgi3oLqQyaLjYgNPmJHJiJPuMcq7ZILJXit4px9JiZtbBWdRF9eEIE1jUDstl0dwSbMBlHs4Vz2r6Q6xEtT3L+L1twEzWjSrHZ0Yhy5pnMnqLHrtiFj/f+TN1sZVEhHayJUBUtvkJOR88/QF2hR17xynOZEC8QopBsZQn84ayw7QM7VNO9g7s/Q+DS8pDUheFd33HPcFDsW4IIrgiH7tKi03tjjF66OAYBGiNHk3gajl/qt5kvJ+T1DV/Y1A/gLHLSX5qPmY3M+O2jObNIzFcH3KG1N/mI33QyLcPb8EpG2D2xWeARmo04/h82HeUJjezDTm1UeKz7PZxAnKOloQRn9qP0eaF/lIQUy88x5Z79lMTU8CtngeQ932JMbyJI2FNvJr0Pe9UvYPFAg9/6Y5d6Kco4A2O/BFPRP032M3Pow/SI/P+Z8HX7opudn/dSOvGwzhS0pADT3z2CJvu3URgpxaHxYFN2c2EMxMYe+EkX/v683pPA3i+jE71G/cXP87i7RmcmWAkvD4cQWHncFQLQR+vJiAqhiB9ChF5Kt5/+iOe/fApBMRsnV/D4c5mA8r6a/+4nqEBx4mylLC4wIc2mwn3D9wZ88wYDL7B0Aw6eRsf+LZxdMCIIJeg9deSvCwZST44a52UjLmHtet1gKg1LkhkWEpr6T5gomhJCot/GUdHhhTJbCnnU+9Hb0khsfo+6uKqeHloC3ne8HOfwJVR2RQli/O/06Hi135YllRBSnIFBX63QJ44GC6m3EaPTpQFWlodROHFLJy32hAEgeLtxUjCVBQPy+NI/nB+j8/j6LkJdLiJgf+YWTHkbMhh2l9LqBz7I5oGLSfLgkif7yRsdBieUZ5UvVKF3lvPtTXXePidh5FKb8p31Ow4j6ZvD2PPjSW5ZDjmpyycSRmKrl9HSMkiQtui2Dn/T4Ia/ZjV5oe1z0ZIU8ANYEHUbfhjyQ6sRjd0Ns0/mFYOl4PsMJAINbxxKpYYq4ue2h4+j/ocgJa7c/g+YS9fG4azMzuKIUVDQIDxL4xn3PPjeOme47w4fhEa9QANdrALYO42823qt2Q+nMmEN2fwic87DF09lEU/Lhr83n+RBA7NOkTGtQy6j3cTNQmcCjUKc+/g5+SGzVyZdTu3zk0k6HQOr21/DYB+XT86owJ8B7j01C+4ed1PZ9OnBDcFUxeST0bbBDpk4TgzG3kgL4ga50b09tVI7FJcUpH9KhFklMjiadf3YUZP9QIbOc6fADhzo7B2aGwjnzzxCetjDtB3tvQfY/j7+75nofazwfXCeUMe7e1ueCPOzvrxP3DW9QfSMVp+D/uKAe0AT36yHo1Ji98rX7F2ykkyAp/Cu/McMR5NGHwg1wpvNaSw3CpBkptJpO8Ejkz/loawBg4GSPlr1kW+M1n48PAs0vPS6L8jlu3DnmNAO0CHXwdNwU0ccjyC2ubL1V4lk/QOKuIrmOwGk0P/orR9MjjNPBigpDLUQZTHLGbkvkT6MAmP3VnBB38W40EdQdNTqfz1HOFFp8meNodLo8V+H525m06POFzDHmPoR8Po9u7lwtgL/NIPfxshqWwPXr1zGJF5gKyrWfS592F2MyNzyoj3y2NDVhPLa6PR2aJIUoqy0UYXvNkFKpsZ1dvrCIpqI9enjcjaMNZ9vQ57VD15ShfRG1y80AkmASL0NzeRFyxQaQer3Af5xtsI17thSO6hx1vcTJvVZmTykfj6iJuV3c/vpquui3v/vpcWfxM/9kBE5HCS5XPptXXT4Z5NWGUf9htZrLMdMnY3ziQ+RsnkY9EkFb3CO8+/Q+mVv2n5aSzK+3+nObiMRks6Xt1euKQulA41zQ4lfclFXPCtI3zOfsaOXskB8y8s3bmUsefH4rXtVraNP8MTD58g7end3F07B5cgSpwYXfB0B0zzXESUYgPeWpALVnGsCQJ9PlGoBgxY9L5oAvSDfeFyQV3aIlaNnUuLAz7uga99XEgk0kFQIO2EOLd3ug9Ds2si0of7kbicNCZOZcAzhO+m3o9N0Y5630jMZ9IJzJCw/tR6di/cjX9nFH11Cn449wx75z2Cm8UTlW88dfLrBBgWkXXRjaCmB1B9V81vExp4M6UBRagHmdmJaAZcbEvPY2zhOA7c9hh9bzUQWXKOgZrpxFTG0O7fTr+7KJ0j3AjC+hnK8S67REPyDBxKN/KHq3Fr2YPGpMLlEvhq3Vc8/97zOE+XsHn1RlwWMUCgkggIMpGNOqCuJT/1GBXj9hHT+BpZJxQ02uupGFdN7rBc0vPSIbyZD3whU3s3TZlDqPbNYsRFX6wqsW9zhouZK97GS3x110tsuvYpRVIFZcGb8FT6EixvRKLUoO6Q8/RHT9OQbkJi7WZe8TzyhuXhkrmIroomuqiaga4RWPutTHpjEp+cDueS3cZwK4zp86HWJRCsnT34PP+lyOCSunj7hbcZobkde78Ts5sY1B2qFGWZPu9xsNPowCkXuLsNvDvFrL/RF0ajczThE7oCd7MUmesadZF1bHhoAyAykqUCPNg9kWzjXhYYnbhXX6fR86aUs4CAt/8WcmNgbBK0DqzGIRcRlsawRv4Ke5yl7g8h99Qz4BGMrqeZEZe8yU2VkB/7HPmxMEp6jLMTz5OSH83yP5dzYlohCTWneWVSEO0xPez2fpjbQ77ifg/4bGAYvaNz+HhMDU9XeHBNacOrOpmM05MpTi7GpDWRUAsBxmn09xlQyU8huTG2h1fuoN5vEx0eB5FIJLw+wo1DUdH09Z5k7r451ERXY7JH4HIEIpVYqfPRcqRTTbYKDF4Gph6fim5AR2JZonjzn0NPUDI9Qcn41Ocy5PfhhDnzGH9qEmpXHeY7THS5nxjsq097RLk23V0/UWTWMPXieh479BgbHthAa1ArK1vhAb2EAzoTnxqb8alvQ+8cwvFpx5FL4GkvMBtlvPXQ84QEHGONdA3ZmdnkpeeRG7OSro6TOFQT+KUP6mwyfNvk2O1BtAS3cDIUDvYe56v2lWw44oEytgb3RhURFeK96AYSiWm6G0eAHalCxi93/MJz7z9HS8QVfr/3dlZU6Zny2xqCUwUC58zj41Sxbuns7bMZUuROz6yJeB6SEugVz4ERLnJiP8em6EBQdOAhBZO7iogxbXQppMzdN5f6CCMOpZpFuzJoHdFMuwpsUufg3P6XrdBDuGwjt3UtZmZTBdOlEpZsS6JlmJ37n96JNfwEBkcBb8+9wO9RMXSGPE6BUcz6jVHAMw0JXGoUGQRek4Zy+HoAKWc2YvBx8KkvGBLDyZ/yGI9N1rKq6Vksagu92jZgHgC9nR4cadPijJdxYqr4LB1yB8ELj7NiSDM/vXsXQwQFF1fdvO689DzempuHYDuCVQAnIO+/wKrdcwG4tGQXcRI54Yrn+CTsG9xnGxjnWobJ+SMSexPvhIKPDKSCi/H37WKn2Z0M32+Q5K4CJHyzWE8rsPFGsE2QKQbLjoUpLPS5YGYTmASBRlUN396zFIC45pd5z0ck3qw9NBKF3Q3T6F5aglqoDtrBgG6AssRS/gyE5UPKWFwrRdYsSs+pLeK/vsEd3KNLpNKUxYB3GH8u/2YwC9NfBgoJRLlySH/yTz7pjEHXBA19yRgsgUR6FKCVasm3L0UiuYJEGcSZxafocisgqiYK725vvLq9sA3YcI/148Koo0w/kopFFcXpNBkva1rhtl94PeJBcuVXuDj0G/r1/WT9FMSpnmT2rf+O710DPBPgJCnXnQFtBjmZOeJaWPMDT4xYS7Ng5sQHP+O0ij7ChTEXSK+PYW6COyG+8KfRxfWt1yn9uxS1Xx9SYIl3E+7dDdRUzyfCVUf30G6s325hHDB2NrzeDbuto2ipu4ZnSx/u4e58/ujnWNRiRlFzuIxxWbcBIO28uXZKXeJT23zvZuKb3+J+AbSGRnxsfRADBTaQSWB8ewiVNSE4YqyDQB9AvwvubwP3upWgA2lHG74NBfT5xdCkOcLc3kJG+C0jfNI6PMN0RBUfx798JodnHeY+d1H2V9/yANF5Q+mNzaCvy0pYQxitga1csdjJMctpSyrjnLKfCer1GGy7afG+WZQsujqasDNSBtoHuGXbLaQUi9nAmx7eTEWEhc1KCFTY6LtWyay9AVwZcQWtJRxp83Lso/YwrTaWsh53upouInOFcS39Gt7hAexamsLx3BguXkhD7e3k2x44OAAGpYXaiFrqdX3Ue/Sg8u3B0BnG73eewuXsRWVVsX9IHoI6lj9T/+KTozAk2sXV3t95otaNib5mnvWCCqmURyU+1GWcZ4XwDHNyrPSrizk6zB+X1MqIiv1APApvPdUxIjDbHqTAv8XO9WG5XPUKwj9iBJIaKRa1BY1Zg8LNTEx6Ff6tgRzeNg3VvTZCw3uIC+vkYEgT/V520psbWHc1hKoWL4wBdnYs2UFzUAsZVdtR+N2CHAnzclbTZ1HSqjs42M/5gc18p21GyxDaqw9Q1WDkrsnpnB/SgtFXSoh8L75SkAoOHjk2m6unh2Ea0sfYnLEobeJeX3Bz0drP/8jaAKfWSYPvZu774zHiwi5xcNQP5Fr38oVwggGvUPKm3YfU2oFfcBMRWWo07tEMDRqOoBEwa26Aygvg6p8a3s7zILYylswkT5I3ByKPlXNkxhFu//V2+vX97Fq0i4rfKlg7by0avYbVJc9xx2NlmFJhmLgMU2y5j2cffg5/mf/gdabmp1IdXY1TuOnvS+VSjJ4h6HqaGFI4hL7YrMFjTocLBMkgIDcsbxgdN6QwlTolvsNCKTHraA+WUhcqsrrXbFqDzuiBa6kUn3gfND6aQcXEWYdm8dOWn3jB+ALp96RT5Z7OmZ9u9mPxxAfR6iR4RkqI/nQdFccsvDherGmWqASOGRi4tBPevn+w1qHKquLJT54kJyNH1NkF6lPncmmCGZfMxVNecBsOfoprwycxlTOdom6wUbmSE2bQFEaw+MgvxOccp3t6M33+KdT6uWHWmPnBUc4tFje+7lYAVlyCkx7/GN597l0ccge7Kl7CbPegPyQFxWgph584zPRfpnP5xcs4pP/MChVwIdOoUKgWYNXdRHTm/TUXv5592IPm4T5xGG17LtOQKvqp0VXROK19sFQEXAd0ck7cUAVu8v6VruAlaNzDsZf3sXDXQrIzs2kKbaLJAfV2SFQ5cMpceHVq8XX50unXiUmADb0QrgwhLXUFk1IMxP8ZLxL9Q5v4dNJRtneupqUlH6eqg1c6M4g7bmBAO0BbYBtKCUhdCkbL/ubOF15jn+k4A6UNeDf1sLG9g8fXi+NjmG8VAeuu06CfSlKOPwnOcZwbfw6zxoxZY0ahU6CUKXlVeJUHH4SNQRJ+CoDIE+M5ub2Fnqd7kHITiJYCnjKYG7yVGEcBDtNJIgr2gURCYNYSKlU72dALf07ZSnWmmjVFr3H/zvvZN2/fICCn0dXzXV8Eu2rmsWtEHxfi92JT2qgZmc3VGLhoLeFC80oU1Ysh1klAkyc+7RpyrOBWBY5xWlzXu/H84yL9MwNpGfYO9VGiOlJIUwj2G8pSgkqNyTN48NrlUjnjL44nIFus2bfy6EqkHVKagpvITc/FJJjwQYxp/3c1+/5DKAuAPt9oLDpfWuImEPdvGU+kUimpqan/iPn977T/EdiXm5v7P2pM8r/K4/y3/dv+FyYIcCYlHYA0NNwSHMydCQN89fFCmtKsyCudPHn2ST554hP2zd83yCYRzxV47JO7aQlq5udVPyMATkFCvMqCQQKCy4lEkGCXGdi++FNGlv1CruDPMflhylsDuerjxKZoxyHrpznURVhIHjtdQWyRRFI35jgzA1ZgmbuEEtU3hMlhpX0iEUIHtxYm4X0+nf6UHkwyA00hTUTVRiFIoCtsJO3tIJHLqBy+HH13HQHnK3k0E4SxRXSmFWEMfogFnyzCqjIj9fyUkYe2oZnsRfxth3g0KY+Xmm/hjp/vIDszG99OXyLrIomrucRz7pH0GtNQjsrgoryRmqDN/PqgntedPhQJvhx7/W5c7moKpqzlQIaYBvbw5jX4+UPsXS48DZ64md1wyqHfvR+pS80Uw10QIsp1PLz6O2Jdi5mTNIk/Fz1BnyYPALOyFrusF/Dg8OOH6W/p54n6J9A4plFp6Ecb+xnjp4Vi6zrEqMujMOqOsb9iH3jAsZ4wTN5/ERkJ5UHzKczaR1rtROoS7ESUKbivVWQcBkoVHJ2+H5fURWhjKBHHJ7Fz8U5iawPozx6CMtaKyfemHvjhASiUKRjtP5FDv8eh9Gijbs0NsEUBX/iB0XWRXUhx3hhn51RTkU0Yg1tfGyqTAf+K83RVJOKSiS8xl0KFy8uNgz2io/m0F7xt9aYjycqFuJ0AuHdZcXV5YPKYT1VsN4kBQ+DGBq825CBtKwMJ6V3JD7Vb6XVJ0YbfrDdXHW8lJ3UjerWcXb2lXMiZTm7IGHIycjCrzQTWDXDq+GQCpx6lJwae6wTXjbaN1gG2mPrZUglXPCqYOOMEO3LnI5VLMVQb8B0VwrcPfktSSRLz229D5a4iuy+OUMBe1zy4wA+9PpQR2Qn0tw0Q0KJh0ulJgAO7Ugx0B7QG4GZVg8ZJUEsQaosas5uZxoBW9g1omZc4Fb84TyR2B/VdpwhvzqIoMw6LqpNowzq+ONRObNhYzHXZOOQqGpOmE1mwD5++MjwkBhJWXCW3Ow7PSE+mvz+dn3c1U1z9NN6Ck+/kNhq7fenSidfS0wO5uYBUxoB3GB6R4j34Nubh1t9B3w0spulMAkJAF1adN8XN17AJJ6nz2cucXTEEWFsJi23k/aPDcAtw0hTWgl1yQ6rwRp2axTf22u/7CTz/7nOUx5ezY9kO1uRnkuxhZXvGFXqSc5mn28MHPh9gMYgb/mSG0RlWwxZMsGcK8ogB8n7Mo2RHCXa1C1wSVndPI3CXjbTsTGwv2ai93khPTQ8APt0+yDbKsL1mQ61WD44Tw8kKhpNCfqoTT4MGp91FaEMo6bnpnJ54mknhq5E16Hm86hDX903AeruZ85OdzPtbPP+7R37F27+Re35fgas8BmeKmeHZw6mOrqa41MGuAJC7pAQWb8FeLiUnbMzgdw/7LJ3X7cPJ/6CUPYv+4sD8E7wieeXGUQmxftfQqMX+u+dwMm5eXii0Cia+OpGGC4184vMOAEFZ/yzqrPUTMxrVFjWTT03G4GNAmCjg3VSAwm6mMXEaHW5fURtRgk61HKlei0t504/QG8Ug/IOj/2B6kJHF7Q7m7p+PX4cPW1/awYIdvsA9zH3pI0Z/ugaG9BBW+D6O+lSY7eRZL5jul8tB2aN8vORl4o33EKD3wyCIhImf+iBVFkAjTzO8fiFhw0KJv/wbKUcfGcyKjOj5mNTQO7H3W0g5uYnu4CTIELni33dLMddMRAiyY/dU00QzTrmT1uAWoitjcWsLZ0xUJy7BhemdD7nTz0lB3lPk1bzB35YEKvPjiLq+B+mdLi6MvYC3cTQXLwXR8kMas5btojExmtaIPuL9VNR5iJvW7BHZvLYwmxWSQH4qfYRJH79A62wVjIQ6B7QYo1ie8hYS4zjuHvEbR7zvJdeSRUBuA1cit3Jm3ze8cZeEdZpsijt8kfv0YFdqyDx2gLirj7Pik884YGxjg2oS7+o6KB33AhZVC/AdNgFsAsRfaSG09mOYncGHT32ITWnDJXUxeUQZmR5WVrVCsUVFcnkLj51fzpmJf2EUXLzSDZ52F7cKdlocCqqHjkFnvIyb2Y2ImghaAX9lDKWnM4mtiEE6v58lh5Zwfeh1XkCsiflNigTPHk/6JHC94xCFwY9S8rgMu8KOf89MZDYz4EaBo4BQVyhF4UXUxrTxgAd8GrWZLwtWUlZ/hchrP5JUKgKDQ17bwLToVgqeeZXeKhl9Pl4UDCnAJXVhbOnH6mcgBgm/n53D8XQHd/14F/36fi6PKieq0p+/F51HI1mBxHMm4WFSuny6KBhSQHCzuLFS2pQ4pTpaHTKkgo7EfW5EGm7jt9t/46Me8NaEMc3UhP7AYk4mWbB1teBX10hE0UHq42KpGpqE39BgnHYnx184js3DH4d6GB8X7KC0djRrjk7Gcns97mYtEcYImkKacChcyO1SlPpOTk84TYZ/PPFbP0FrdNDnE8Ek6UhqIkt5v1PGcLMCp0Kg16NXZCL/KzNB60d8QC9tNjsKWwYAfn0zsPuXkeTTTpisgC09Mq5ZnUS5BL59eCf9WjFTqcKiwaTwYvGKtfxS5U18xwzu/OVOdi/YTW5GLgmlCUw4N4amRU0MpI3GKPPApvFE6rCxomo7ix8/RnxJLQ9rP8SqtlIbUUtkXSSp+clUpOcyoz2E7gFvjENk/HQD1NIatfxpiiA0egNvvm6g0jQMkHAl6wrl8eX4qhw87QX1mmr87hrgau1yJnw0gcKUQj55ejdL/Es44wUSax2tFm8M0i7sHYV49x2h0L2Cx91HMcZnDG8M/My1KWYKNJPp8nNxPPNnnDLRf4ipiiGuoA6LwcKee/ag1CtxhMdxvFfLQ64B7n7xW/J7h5NYr2Tx7vs4N3oXHf6HORgMcQr4otdOjd1IV6OSgN4A2gLb0EghQQFf6VTcZnTnN6ODyXuWUhdRhy00gJlHRgAmhqQ+TVingou2ccy4PoP8tHxag1ppjoIayxkOCL/i8ftUwocP44un1jHGt5i93lBrB4VQy7/yRVwuAaVNAYIch+JmcMgluFCG+NERMRxtvki48zJ40RLUIkpXuZwoXJ4s2r0IgOuTK4nwKKDI00iLZysumUCsSY9FYaOiIRn/i73kRLQROOkK2Rbw643hYPIPGLzE9dnRPZsg43NEXf4Q365A6uIjUPR70ZoowUNmI1gJKomVqy0qSsvAK7SVETmZOBR2vCJb0QomJJJQDu6cQOvJkfQ8r+LP4T8QXRXNsLxhGN1hvGqdWIf6hkSuytxDQFMAIdUf4FdZil2poLmtiqTcfrTh0VTHVNPuhPYbnSV3SukPd3F23FkGtDflkAONKZx5dRnhwzu4Oi+ES4FiRpBNgKgacJoVjIutQa6SEVqZRFr17eSlT2atBywI/g679k42/jWOhLIUvA3e9OsWsuGhDaxtB41XFO02LyaeCaXbYiNnhYXzfABAePNI0o98gHNgAsLK4YPjUtfTxUe+cKunhNeNemxWF7GKCWgs0ZjU1XR7d+Me0MVD05fzuf0b9FlJ9FraaAptAmCmFvYGwwOKC+xUJ5JlzyYrewQmzTnqoxyooxp4ZsROlni2UTL0LDOvPciTW55kx9Id1EbVMqMJIiXhZOnG0HtdQkRMKucmHKQ5TCRhqeRmQMo3ZYcoaoggLLSX4tA3GVBV8qwHePw9jIhrYiqh08lgBpzNcwIPuX+MymsfO2TvIFV9QF1k3eBz2PHyad4KOsPZXTMoPjISvrg5npuDm9kdYqCpehVlEgOCXErEZR0JpbezY+kOLG4WRqjglKOTB/4chaDzJuVaJxAujs82X04EtZAh9+dKeC2E15LWW8uEg3U4zRlMTD5NmBwuu8Uxw6MCd+UQ1pnNKM127EotP5SvxlhiJNroi3WxFbe+HgSJBIvenwCFjUc8ofFaAr2tfriG3YyY9bsV8ZdRLGGxfdl29M5oFP7BHBixSxwDHX48bvGjU1/F6gErAgLx18pxGsZwadQlLr/yAfO97LyqdqfXdYm1PbdwPrlksP3fA2GYCqpUniQoNIwTVmPd+C4S/wRCFtp5duwKjvR+j0y3ht1ZS1m2DP4o20GnqomZh2cS3hBORm4G1VOqMUT6UBp/isnHk4gor+E1rYogDxNr2sX1pFafTbtOBM76tAO493hxpiSUJw7OQZBb2bpq62BtyTA5jPKqYKJJQ05dBK7Am7InE85MwL02lCDFnyjPjqPzcgJN85oo31eOsMqEXAIbo0o5KfmGvSUrMfyYTccYEYGIHlLFnT6w2h02ODzZMGcvQXPreVP9Ogbvm+TapMYPkCP67K6QMJEoWn0Ro7sIxAzoBrCqxWBoaPERoixtHBsjBtpyw+Ho+SQu7BuHbWIv8WXxdHt30+nXyTbPKGb71vD8thMIgXnIYkcQVHWeoKrzFERLeCYZXoh6l/W9M3HIsghsMqE3DOPwrMNIJSCXQCh7iHy4mB+6lKRfVnLvoXv5+uGvuaLsYHHuXTT4bQLvZsY1OVi+eQiXRho5NFuUHr464iruQ0fjGeU1CPQdnnGY5NxhVNRYWGhwA5k7iQskHJxzluLEK9zx6xoC2lLo8u3g26tpuNRyLq4up6A0mzEXJ1OdmMvuAYE/AqrY+dKbjJY+Q3B+ChmBfZSFNJA/7gL+Ljl7W17B0BiNf99UgutmEnPtr8H+Lp2+Am4kIgkWKw9sWAhJd9OTNZx7ukay4lYpIL77BaeLgOqLdEb/iksqAi5GdTEwB0EA7y5v4svjyR3lwcWh/8pYKaXDMZWlZ14kqExchyJy7cgmOPg7oZTDy7sYGf4ICn04PpEnCCx5ApUa3Lpa0OWkMersSPY/0kJBWgE6czI1/p/xcAeo7H48uH0BXg0FHF3QSWD5SHKG5yBr9eGQXcHwdH8S22IJOH4SaVAfvZF3o5IO8J7iIHeEXqTJ1cMhTTdCRAORt6zml9g/adfn0RrYSsTZZH6gkNv/B1KeG+UwkApuNgvWYrhSPAqZuoMkwUj9uXriL56mMraaqrAddEi1HJbJuG3dHcTNq8QtxA3/Nn/aA9ppKWwhpTiFEc+NoHNjJ9SAxc2CXWHHzexGdE00AH8v+Ztfy39lWMEwnhrzFBKJmPU1QgU9Lmg/mcXYs5nobtehDFfydMfTbHxtI0u/Xkqvey81GTUErxR9ZkEAk3sgFdGXuZS1l6zSFPoa+3APdWdr1tckdyioHHErfdblOOQO4oqVWLP68Z/vz+Qf7mDvehh+/i/mNK7ns8c/oyqmCu2AH1F6d+4+K2Y2z/51NgcrD9I4vpFHhj0y2G//GcxwyRS4pCBTgjLIB7mm8WYf94JizAXGaTOxGW0c3XCUsPowWoJauDTyEg5t1j/aEm48t3Ib6PwGGPmwlMyHV/DruX98DLOygb0TF+Cd6s7QuCVwFASJeG6JDSIbzOiNgdy2bxH6hE6aZS6sanHclzaPZkrYXkrDZuMzMpFY3wLs7nai2p6lIvTtwe8Yd24y3gPgsLnocjuKQzD9yw3Ct90Hta0Lh2kAa0cftWkLWDlyJYWtcUw/Oh2J0A7viFLG0v/QXw55L9XJfgTFpuFq/530vHQqYytpCm3CSwrrO0HWGIi/RMOEXSlIrPF88dgXg+evdneQNvRZbMIzLN+2nKqYKn6/7XcAejU5SJ121GYtrjAf9i/rpyxKlOc9FgJxCgc1kgCqDek4VG50H84mOjefeQ+9CI7fQa4lStvBXaMrWduZRUKxP57dwZwbfw6ZQ4bCrsButaPUiOu5qr6C1za+hj2ugaT7dlOmW4tHhAfKy7ksL13O7oW76VFbiamFNwpH4fprIqmBBTQlTAFgWM0qjumP8ZC1D5VqAMFvAKFFwM3shswpoy6ijpnfvE2y2sUXz7+B3Sin5+k2yoLEe7IK8HMfdBJH5bYncC+6SG+yqETmrrGxJxi0lUD4razhTdJSChmrv1XsyBvvzE+e/IQp7g+K/+VyInU4cEkVgJS1WWvRWrR05IgKHq7HXPR+1IvBy4Bfhx9dHV2EeYQNzsd/mfM/JfX+FxlPqZTsSXEYdJeIbJ/275p9N8xm+2fM73+n/Y/AvpMnT/73H/q3/dv+L9h/nPy/me/ioZg0vNxM3Pb0LzzUP5cgs4oa2yVsShveUljlDiNqo6n4bBWG1TaqYuro8hG1pXcYYYfRwB4VrOmO5mlPF1+v+3qw/ergn4lpfYbLxzPpPp1J/20XGTIQQ3FyMSdnm+kI3ssS01isgpwKO8yQCtiSEzD2hDJZSOD95L/g9F887j2cXUiQKKV8+/C3AMi8+gjy7eOSRy6rpbtw2RyYPIIIrLqAzzE5ZIp6+r0ueC3GRXh9GGY3E40+4oS3nR/OtDWH0EgEnGYHYQ1hlCWUcWjWIdIK0uhsUlNffZwEnYrGwOtkxy0EoCLQyG20ENM7j8Q0C3JtP4JUNijdoRjQ0lMDGmcrq85MwTs3jaDbw5hydT+F0e9g8y+meSCardrVOHkVAYFuk2EQ6APQdF3Eo3YTDstjjH9xPLYBEYy5xfknKvNeIhM20+t8FlWsjo/Xf4xVZcXXGM5YNQTTPRg4Em5EDp3SAc5MclC18m0CWwK549unaJ8m41C6iN549Hiwa+EuGsIaeK8khGWHxqNcasXf6c+AdoAB3QBvG0DmkvJXyFqup9STPkKObe9Wlh5cSt2yvUx0s3GeTmKtFobkjEE5xIqgd8di0ZByZgP/Cum0F3qjSBXBPrnNhLN9gG6kvFcWiPWP5wiZ7w7eNwdpyYgJhFbVsTK5gtCZ3ViHLyVm6wtY1BZKIp5n7qFZ4FZK0V/PE1x2Dc3Sfl777TX2z9lPZ4STbp9uLIKSg8cXYLumxTU2kethgbjb03BKoSi5iGpnNK7rtQwcGUWXTwvZZxr5sfFn3BRQEgERfT/jmAT3+nYhV63n+xHf45JJiEuLo3BIIUtmJKIP1oMWmhImY1dq8fQ6zXFfUU60z9uDpz3duO03kZna6eciytPIEi/w+2ExrSYVHe+q+eQ5sQ6lwqagPryece563n9iGftav6LC6EfqgXDCfw2nZXYyqya+weW22zGcLkQx1I3CzFlE5e0iuPw0Z2df5FRoCf7GGWzTHyHcLXKwPw22dqoDP8Ll8OfNK3eg+C4c7Z1WKg9V8uvSnXglzsEQlIJEcOGwgUIhoz0yi4iC/YNttCktnFn9J3IeRHrZyqNfPsrvK37H6OFOqFTOQL+G0G2LaJnoQjLpJptmuK6TDZHwRkEAFdlJOHwGqA+vp8Ovg6TiJEJ3zqMPuPLmW9TIBpjrcA0CfSlPzuRstpkJkcvZol2D5cFvWZfxLWpPNX5JfnTn9aPv15Pq5smKyWWcDbYgd1MMAn1FyUU0hjbS4dfB6rLVpKam4rK6cFgduM+Kpe9QJfvm7UPpiuVVr3fJupJFWkEaga2BxC7wY0Q3NKdW0uTVygifWSgcN6VG5kngbbU3daPLOOeegrWyn/n75rN92Xbqeu3skUOWysSPd/3Jb71a4ufdRdOVJi7WBBJYfVGcC9Ig7rc+iEEeiUQiIXdLLqV7yqjfZWbn7BmMv+UCGbvmYQlxofxaychnJ3ElYwsAzXETiFya+Y91PnJyJBWJUlKKUth651beu/c9BBdE5ouZuKGlx6icZKQgtQC/rjUcb7iPrlQnHzz9AUEtQcgdcr73UpDlYefRCh2CSkLpkH6KhALuDH4HENeO94JtPD7pJO66hah7k3APC8HbAVmOVMZ4lXGIf8n+Csj6XWjRMqAb4K42WGl/jeciCpmsuIbUFIpdJceiF6mzabqpRDXOxxliAhQik1Qiuk236qC5Jgi+b6dvxk7Ojz6FUXUdBPhz2XZ0Rh2H5CGs/OF+eufpeEWr4KM+B1Nq1hLp0uKS2DH4K7CNWo0s9OqN65OgUIvr7FqzH/lRw+gcCGW3+sNBAOZfmzEJEp4OuU7leCsK23288PYLHJt1iKAVYnatTQJvXS+jrNqDZHUTMbk76I4ycyZyAM/Lk4g+8wc+s5djzxpC0zUX0Xl/49HrSYAMFDIT1b5b6LTehcGzi/NJowGxrsLIgXnE+5kY6NMQ6JZATOfnVAd8hH+DDd3PqwhetRfhfBJpShUWxUG885MZO11NnvlfpA0pGx/ciKc9hZCWUfyx8g8AnvzoSSpjKxkmWcLoojfwq76CY5yGtII0GkMbqYoVwb5QaxFLP/qGV5viUZ2M56ldT/Hj6h9pCWrhru+HYkz9DV69F59OHyJrI0m7lob3kKNs8Id9W6dibijHLcB7EOjzfzmC3/JbuDTwKMPcjZS5CZh8v2bnZDmCxInXdSkDggSz3E7Z8SyUgS6uZF3BqrJSG1FG3opuZsU3cnvAQSwDlVwxP4/UJeXIDLE+7zq5hkX+vYTG2Nm0fDXpEZ709/+NqtdjcJ4Um4/T2B2N//l4nJ5O9KWX0VVdpyXMi6up71MfpUQQsulr7OPiR+Jctc9RUaH9nlApeBl9kNikDM1OIOHSaDbff4oDi+1YhN3ooquQpxTg8FxHcdpFsq4Moy1YzZgLw0kyqtm6bDsXxl7gHt02NqctB0BJDQMTlpMal8dmXxUftSko3R/C3NrHOLFIimetljKXH7XOYWy+7GD0+UxM97WR1Lader/vmR78EfMDTyK33kl6ZDtbte34xkZxJHUeEf33YCk4ic76G97dXtgH7Di17vi0lqAe6EJhHaALP2SAzpKAywUxNdO4NOoSga2hTHVo2B8Jh36eRWOHJwVDxMxRuwCB5fGc2bOQqIUn2br8NOsae0j+42WC8vvYN28fFX1qImtMCHZP6iVlJMj+pHPdAWxKGxJJDN4ygTgFHLFK2f9NElE19egztMy+MJv+WDO7V+zGQ+XBb78cpW7RAdrrZ1CvuEad6mYtmIujL6JWf4wu2J3xL40n+9tssg5+j76vlJqIIh4LfZpE9zRsTZ0MzQ2hMMmTDv8OHECMEp7pC+FLqw7vE0FEVKTzxqtvcNECgTXwfWMsTZtuQb/ETlRhKnd4O3jJKQaAEjNL0Ha30X5pEtqUZsYUjaE5uJnWoFb+HoABiw+Z2gCkVy+iVJuwj7fxbaCL0Bsk5V2uGhLtM/DpjqC28Ame+f0RrqVfY8/CPQwpGMLoi6MxP9aKPjISQ2ASuvbL+La20xrYSlxFHLf9fhvOB9tZkf8bcAybwkZa9DW+CkrjzYubuFx7F2OnXebsU0/yU1Y+B0c9h4+PGWNgGw0BsLkXfnC6aHe7mTGzqOULvEPPEKd3Ud1vx1Xji29rOIVZxxgVeJw/guAPoYJnL0WzeNdCnI8Y+eDpD3DIHVz0e4wErzykkk4+CawiZIIFb7kou18dU011TDVaSzyT3CcjkUB4wX7867IxeobQfes8Ps26g81j89nfNIlRfSuYdHoSZ8edFc+TiDXNNPUhqMwe9A51cTz4+OB1v6DzQehM5dy4syhCIrkQfh82mWXweK0DUPTx+21bSe56nu6IF+mPE6VDoxQwSg0NggudUYeXwYO8zD5afM8xLmQq1W4Kvp37Lfd/9SWbHv+crMC7cFluRlrqA3eQP3QZusBIHLahjLok1pTs93Bx3AT3BfXR8cqn1PW9THp+OROvDefkpGakLil9bT4opA7Wr3iUk57NnB7ie7NdO3zfC/mSBjo9GsA1hx8eOE2Lz2mCO2Zjrw1lR2ELwnBfvBwuXEo5Bi/DoIxZtws8JComOZMI/LsSxSRP/h7eRKdHBRsqF/Ng7N9IBTMV7WMwtPTjdfkci/fAngXFaBNg6q3HeTPjPK8Ir9NzqZSginzaI4Zj9kjgnt0NfDZ1KHGKMxjMN2WyAIbrulng4WRntIvWlPEE9vaz6qdVXMu4RnNwMyO+fJSapCguT8lF5wohPb+F0LoIZE7RtwqogYi+Zaw6mowrPIo/J7xKeH04xcnFPLjxQfJT83GTuxEuk2BDwK6TcWT6ERrCGhh7Iya8I+2m/2G6cpW0M6cpz7qdRf4lnD46FsO1dhxmBzE523FJZRRPfIjrFhUTGyH97Aim1IXhXGhmyY4llCSV0B3kZNcA5FjhrlgDV43dtNhvMvZXbrmLPpMWn+XnCdmViXqCCc/mPOLt8VwYe4GDmDjYC71n78a6U4fvFDtE3uyz7Ua4YJIxIeIhHnl1HOEhTjpC/yQp5RseGyuO8xb7aHSqBrJCclGThYC497yeaaEo5SAefV48FPsQJaW1ZGZncmLKCcK7nucpVTSx9gYaz47AObwbqVxAY9dgdjPzyx2/oLAreMQyAZ+RdtKzFvNS8U0psaU6+DTqXVb9OIOYo2MYs8uTkY+N5PLnl4mpiqFiRDYNwxbxY042IV0exD0Vx+4Zu9HsmYJD2MriZnBvHMknt91JzuhReAy9neaRPdw24eLg3NO6xPVUEAQcVgdDmu+iwvt3rGorU3Zep6jGyNSpK0CjoTF5Bo3JM2j2EtVwPHo80PSL4701dhwWmVhH/LYbSfkZk3J5RxhKlETCbb/fxvkx5+mLfJqmoEIOOn6jOvI6Dk9vomLuRH1UJDirdQO86g1Kh5OLwTNIay3g+hQ9ZSFi/duNveLPquBZHA7aT4AEBgIVXBYOYdQZ+dAX7o78hdh6EXBxyRXkjmqiOqp6sF+NeiP9SgcylZzPHvscu8LGgG6A1157jQHPHuwCWNxcXNOd5HKW+PxPTs1n7fkY0uNaedqnGS9JEiqJH4KkHq3Jk24LLG0RRz1yJ4LgIuvPZSRmlpCwvIGsI8txs9qwjUjHdLiAipE2tAZxwpjVZtwsbtzp8xIRlQomXwiiq2MOrdGjeWT6E3RFanmo8E6a7VP52jKegi3T6B8RgrL4AFaHlkuZsGz7MsLrW2ATdJ24zqNfPnrjbjvpdIuj16OX1IJUmGhE16vGorLw5SNfct/393GubA1/JzhoS95Nin6ADYZsrvtd5aWeDL6YmskQ32pKunyxquSgFpVnjG7Fg/0pQYIr0JseEomozCa+eDYFqQU8tOEhAEpetpKq/JXKDiP2LlHlQe7SUl7xKj93ZTNtkjtfDLsMwy7zjf9XyOTDaA1sRWVV4ZHhwdlTLi4LMJL/73YZuOiQMKF/PPlWsb5ac+xQZl/toE8Q6Kntwb2rlowuKRmXxffie2nvsVC2gBZLC3G/xJFRnME7L75DjimHJfe6cY+qlIonVfzhNY5Fi+/ni61foLApODH5BO3+7cjtcgJbA3G0OrCb7VQfqELTa+CUyI1gs0pBg6sDp11cKzS+GlTTVZwqPMWk05PoLOwcvP6BLjP+9Tn414dSHenP8FMtlB9vIHN1CpHT4yg9JsOq8eTYhGOMvDSS0WdUWGZ2A/rB+GJciQtwxyV1cWLqCdys4dx3k4/Dmow1CAhMnT6VkWPE3qw9VUvHzgZktkycSlGe162vDZUdnHZf7F0mKqVXGJM3mwr3g3Q6gayreKkqMHebyXk6h6TRSTSEN1A1dCQJza8Nfp+uu56gfgO5MaIaR+6ZWsYkR5AJ+NTnoh7ohEyIksNinQuZt4FDvgakHnL8aq8QltNLzR0aLFoTMiCrYidxpbtR6D2o8OkjqDkIg5eBCPdK5sV/zZWmBQhCIqm3pZJ6Wyrvr7p5728M+xnz+504lFq6rlYxc/slLAuGkZshJveM334Hv703hmBrN8YzuWhjNARpphEpTOXYtGMsD35SHMvj78OoKvvH2DuXNBKdYwNTRwXy9gtvDxKOfgmECWoJL36yHJdKTscoFVW2MwAEyOBkKHi37uPMnvEUOw+ye9EujDojWeUHEVJnM8mrEP3R8bj2TUV4wobBV8CkFfeVhwegUKrG4TWcS1vmERR+kfqRRYyPP0G0Wx5s08FyEztNI3m47QzjtX6Uzy6lT30FEOt9L961mKqRVSTPTubos0dxPy8SFxsrwpjo6GR4rJI1nmp6m/JIKkli37ybSSjfepeSMFHAxz2dnqAFAMhdIHe546CP1kAVJ3qVnB2l5vPAzwfP22q1o7XKaAm/BZXcgUDR4DGjAKvaYK5XKhFSGU65Cq+kQI4uzMHTqwShG+ZqQeq0UJFQQXliKfc6XiOwfB++/nLqb8y7f8l0qmtKyDi+g8rht7Ah+wyfXfqM5U8v54VxYs1o6Rgpm+/dzKRTk3j424cpSClgWOww/rP9r2Q8/3Ou17nkLPR9enzbPYBV//WE/5eZy+WirKyM1NRUZDLZf3/C/83275p9/7b/I/afkf7nLfmclULcsErGVHxAjvYtDkWJrLOn3ngRuUPBmBd/hBFFXPAbxt5FR7HdkD76l108eg/XTuiJX5vK4RslHby7vEmuOUysPplPYyrRe9eTWTGPRVcXURtZS3KejBG/PsXaV7eQ27+EQ1u+JeWBsbylWkp1yD7cO+6FaPFFJozOY2dMDlOiH4JckNvlfDi8kQgfI9n1IUilYGtoI+3EJjrC0ilfWcaOILhQGEF+Tjz2SDNfPSJmi0zLewu7vzu+8iqun03j/sJQXKvdePeFd290ELz9wttIBS1Pf7gNi08IVe5XWFCzgANzDjD64mjUFjV9UxPZP/NddPYoxl3NGGRi1UTWoOFW7EofHp5cxPbQdvrzbmXC/qv0znfSEX0JhxCLh6SXFcVD6ZVpaPyjnrt/uJudS3bS69lLfHk8ydeN2E12EhYkDPZzpud3LEheS2l2AgcuTSN53tBBWa5m1W/85gMTNQU8dm4t5rJAgrs8CWyYyqlJ6wfBSKnCQb/3ADI3r8F2ez17mZ2biefFiWy8fzNfPPIFs03v8PC3D3Ni8gnOTBSdApfERoOliKL48+iDoplYGIWj0MqBOQfQVcErcUtZIbxK8YFF9MmtqGRWBIedzrBhWLQ+2EKjiJzkTeMNvzKw8hyqIxfZ/4E7cjMcLO/GUteKb7uCBYULqE6Jw5no4OF7XyVaX0+3OZBLLli4eyHt/u38dNdPjLg0itDYBq4HN2LVeqPxVFOUXES3dzeR5XLW7nuQzHt3MX3WT8imyxiFOw2qHwk23IpfawwmNx/qkLOjeBnj87xwtNrY99xBumLaCVkFETcCci92geMGY2fk4yPJ31vGtOPT6HPvo6T3CjAemcRFc8xYIq/vQdNipmBdAd79E9GpfkCpU3J86nFMGhM2rwWs81DyuM8Au2dcINegRyakDT4P/3Z/7tt0H2nLD1MdVcgl6RZ6pbOoC68joj4Cj/pi3GqqCfIr4/ykdYyZpMRyRYu2V0yZG39wNJezpFgz1TeGtEBbfhtHnzmKNc0btDDezcFjMXvZOPwVPILT0Pi66HMLwCVTInXayTj0LgceHMrSnxfhkimwKzV4372AI95ZDPO00NW6gDHBEwlKiGHT2Dfo9u6mMcYHaYkb58pCmfHEr3wsWcXS3bcQUuHNX6tsWD3zaXDA8K5AQk5PwjnazK93/ApAaEMovZ495Iw/i0/bQ3i36/HLiGLJlSS6K7rJ/qsW79PXWD7nGN+bnLQFtoFORuKiRBIXJTLx2bUM2P/goNaJ37BeqoTFSJVywseJXpdvpy8XxlwYZMsD7Ll3D4V/FBK1ewXrR90BgMJiRxDg8sjLeBm8CGoJwrxhF2Zg4ONevlfVs1gjA0HCpns34Wm/m2Vf3M5PIWYkSxIojZ9HfMo5tt65lXb/dqK6xABGtwuKEkuo6PMlbEwYq46tYvt8ODdyG01+O3lUsYzHQrazq+RNBJdAyY4SKvZXAFBwcAxxC4rRr95JpXoYAF99BRdDb0EaZMem9fovTmB7YTtxpS4gnkOzDuEe4g5WaI4dj3dzIWqTgUmnMmn3q2bx34swxDUy/P6VnNB+QlVsFYEtgRxqjeDvC8n4bFlIwKI0Pkm8RLf+KpMlEv68s5mf0n9jmIeNDatPc9e1WxBYQHIy+Bthy8l8tgBrVs8nqWwEfxvUZOSGM9L4Ot8+/CsafR4zvY/j2XgZ1x9+GKXhlGfEczZBrJeSbzzO3PO/YS3VIl96H8UTHxq8t+V6mBg2wOrMq7gH6kk8rGf2pZf5eP3H9Lv3Y1VbOXzlbex1HfhZ/QnrDsGiFtdJj+FjeCq4gst9fuT3hdPQNIn79t1HXkYnrsw8IJPTNifPZqyjvjeF+l9GcGvOa4Ntz2uGocrRfB86Eu/ZV5AaEqir+x2jez9GuzfXDcsZ45dGTfs1ZIWXaM/QkzOzi/zwnxHs4KcXJVQN12qQTEviX0hir3svkbUwtS6aF36/BfPaVrS9PoOF2Zfq4LuYfZC2jz87IjFKVxBfswCHqxK7cwcD+n7etA+Q4JQhuKSUTsnl9SUn2a4042uDPwLhQn8XeS5oMFoIKrGhax7J5VGXOfbCx0yxTiZBF0b42qdxOGXs1E3k3ZfeRJAIrPcEvRQygqbTVnyYiPb1NIS2U5J1mTt/vhOZS8alkZcIDZ8MwAMbH6EppIbTbzcRKaml1g5nihOR9rWiCQlmwwMb6Nf3827vp6TtfJ2eW90Y9syXHGsbjjQXEKQgcVJ/ixGNsJc9gU0UPfUhi0Jf4HzP+cFxkFLoz5D9QzAOb+TC7hT0C02s2bxGrH84+hJTg82MVkOlBPzqsnE6wrh2bw9XhG8JkMG+YChznGBTgC/fvfgWcz1fItCVQYc+Cpctl4mnJ9JU1UPblCZiRngRPj4cXXocjm92IEXD6z9V8qu/G8KQLFoNP1HldQB363x6vYxUBRVzMRRC5fChReDspDyOztjPuKKrRPj/yLOT84n3gl0DECwbxvS8Vk4NSabNay/joysYEtfA218sQlaYgHlUE0YdVAa9g1tmIHKHnBhhOm7mIsIawrD3mPHp9sUmjCJxCEzT2mn1yCLyBg72gJ+OzojhCIKAX8dC9Po76VghIXISSDY76AwdRlT+HnFchjby9KGHmF18H7bVdm75cxytAbX8fF8d98WJtQ6bJ58j3+DJxP69PFm1nbihFSztF6OqW50mDlwNpNkRRJY1GrmzkEW7F7Hx/o3UaUz4md2Ye2IutmAtZRmiH6k3ufjcIOeTPitfREbQFnAcuVOPKXY8RxJeolXfysMbAnhh0jtscw7hUHcB4xBw/SeGvbt9LoImFYkcwsaEse9+MSCwYNhXjEo6xKtbt5EyVElfponPH/uefp2oJTW/GTJ6b2HBpyl4pcdyPnkfNSHX/9H2i4oOkieewid0Ng9seR1/OXzTBIdfOU6pm4NTvb7YH8tjTM8Odo/fQKe36Ls+2A4Rlniyk+8l7GkZR10PENDkxSaNlteiBrizFTIjJzLQdYHqC2NpVR+lZVg+9eFiTRuFXYF2QIvL7sBS10Z44Tl6fNzZseBNejx7UNgVXE+7zlD/JDzrRVD+/WffJ94Gr3RBW14XQ69+hmtIGpdGXqIlxIh7r4YVfy7kxNRjPNZgxVodinSOCze7G1aVFY0tni9np9Jl88M2pIGSLj2263XkdzlwN++gwAbvNiYwe+SD1Ef8yvZl27kr/FZMA2Lw52T9Ekrap3CLVEJ5QjnlCeXcXruACXUTuDzyMla1FYkgHXxnSR0iSNTvE4mvuo3RvvmMBsZ0DiDzUfPV2q8GZVXf8IH1XvDj79Opqffj0tB/vvce8e2h2e0yb0yrJlV6Cz4tvtilMjr9OpEggtPztRCvhB+6bbgkds543A/AU53iT8WYDpatPsTW7kvUtC6mwbeQ50Nu553ZYh2bp/wOMyPawLIeO+E1KXQMDKMyNg+j3sjOxT8yzTQGl93J1BNTKUksIXdsCOVG2OSClXp4t0+gtySXkZdTOD/mMNcyrlEbWcs1cz/eVilS55c8+vmjDGgH2LxmM/k2uL8d1nSGMePsaCqGK1AqkrErd9HheYy6YA9KlL2sb15Mf72CSynD2bp68aAUbJQcwiUW0ty/IXJVAWe0y0HiQimBo62jKKp4jecnxNLV9A1DTvfSO96bXi8DNqWN9R3wgqST7mCRVW4qrcertQSnTEm/TzQGayQrd97G+GN6PJ47wIsfvUhNVA2/3f4btvPxXAuQsmTSYUqEB3EgIaglCJ1RN1izD98zPOOWQl77UrJnv8bWmVv+8Tx95DYCVqox6YKoDKqkMq4SBDBH1jPJEEKm0sGWkpFcOzuU2rscXBh74R/nq1x9vPjNOnr0VhrSK6nMrKI++itC3MtInh+K5K3v0fgpaI0ejSARiWlZRTmsGXknY+4+ytxKD+40zSCtII1u7266gsU1J9UVzpv6VnZ0Psb31VcIaZzC+bHnqQ9tJKE8gT/tgUSGNSDz8ObLdRtwyszIHDLWf7Ieo87IqSlRhCT442705vl3X+XQzD00xvizgcPIHTpOSiGw6gKyGiu7VpegjBSBnu9yPsMYkkKG8yfC+7+jqeBh7t44nXa/IYQ2xtAUouGv24/hn+KP7XQpoy+NZt/cfXREpvHaiQLeTo+A4+PAu5fkRl8WXnyGT574BLz6uN/dwitex3FlnaI6+S08L4ViUXRicbNw3CSuZVcTcsnTtzAjZgbKdjELoy2gjQG5jfX79lA0tQjJ1GOcOnKG7OZsCP0OEN9zkZTzpWCmLPQgjw50EHzZk5ygYH4Jq+SLHjgYeYC7pMHcVRRAT1kPy76L5NKY5Rya8TN14YWYPW6s9y6XKO8glQ4CnSt/X4nWbMb1EvQGxAPxg2OgxAYbrx9BJ8lA7mPjwJwLNIYUEdmv4UTNKj5IfIeUh7LZ0ryASJuWxsSpyBxW1LoiNFJ450IMt36/kobx1WiDsxDkPsDNDKN2PGlzQgAC5ig5l7wuASI3zEdhYU7OCLzPjYS5/VwZX0+LthyVLQirsgW5XY7E6cLpcDG06Txu/b10qTfhNu8Ed6Q0sUmSQbkhCy9uBm6awlr5+cHjqKWgDIF+qze6Fj+afLrYd6sbZqWYLfKeRsVQgx/vWQWC7tvG6CAjz7d6EC/04x7rhX9QCU0xAsr+egJrLlERW0HesDyW715E+0AkVlslLr2ZVl0Z7X4PkzxSBNVmSK/SY32XSOlFSiyjweli5/JSmvzFepsGLwMqq5j5KffWUx9WT597H0OKhuDT5YPcIWf8ufEY480Yxy+mTbqHu368iwNzDjDl0lTu1zYS5AeHzGb88pv5qiODPlMUSSdL2TfrAsXzyrnQsIw/zj/E1OM+nJ3Sw/kRH7BcGs9oy8M8/vhkTjfO4eGiQM6N3IZFbaEttoqAyhjGKEuZOeYMqUM9OGa/g+i3P6U8oYFN0Y0Eti5ihvKmZLuzz4S2X8wGWvXzKq6MuELKbSuZ/9fL7LVa/5eA32VgvlJJlnI4cjc5fe59bHr6I2KtM4l3P02pFZ5e+TlfvWcmrPgI3V7deBu8Odcbx7aHn6fMbyYWvMj1FYGfCWcm4Onfi8+Us9RHgd3djEahAcCutA/GZvza/VizeQ12hx1zlpnj9/1JUFwMHxngKS+4d+l5ckY70YU/RX9zP9+mfYtkqoTTE09zfux5Ch+7WUfSJdxED1xSF8dmNRLVF0omMPbNWWxuAr+6q8xun82FMRcI6/+S5DB/uiu7yfvoGlpDEmUpYJbnDRIkBQQYMHL6zRwiJ0WyZPwSliQt+UffVR2povPPc8gnJQ+CfdHXdiCVCnSWLKNq7QYCM5Tkpx3kSx8x8+qASZTD1PprGfXHKL44L2aqCTdqdoIY8wyoukB8ezsHJoN/z1zUZg2O47u4nhGFV2sp7p2i7zRcDR/fiGH6SSS4XC6U5j68upxIXVIe9YRP/eCo6hVOJu9B5w3evxfywKEHKB++iPErn+Fow1zqz57HsS8bHhPBHIlwE2SQy2SUj1qFIJEy3ttCeYKRbu/uweNBbmJGl0PvhTI9haDc8/z+59dUZR2EWHDE38yqVji9UdkCsSrFBIzxZ0YSaqhHMjIQu1LsA7UtlFMdnsT5GFBPrKNKksLwEW+zq//nwXZUlih+rw6h92Iq/XHNyLXP4imAX99MRigCeNW3jaqUZo519FGm7qJR/ZVYW13m4m0DgJmHVecIrGvE2ttEy7RWHtXl8sgg11tA3iPnybdfomWcA4nXLzglVgbiH6DBJ4+cjBzuDLsT24CNK19cQQ5cGH2BnOE5dNmghCVsEeyUzbfwzZw3AHCTiP5UkbqFU8Et3BL4HGZFA72aXIrCH2GEeyNrPWHfjklUHRmN8MnN/UJYfRgBP9zL4RmH6U09gFbiw8T9mcy5HsLHT36MTSX6pJ4SE+NuW8LRHj+cwcsYfziV5iAvPEd3s827G/qKmaoRKLBBS1U9oeU5+CboqA+HmMoY3PUi1OPUe9IZNgybmycdAx2UdZXRFtGG2kOM0b1x5g0yyjPIzMmkLryO8NDwf4zfwfn434B9EpeTGYdnMObiGARqcM2/qZzxb/s/Y/8/gX3Z2dls27aN+vp6bLZ/suh27tz5f8uF/dv+n22CAO6mofRpxGBGsQ1eb9diLdiLKvyfdcjkDhHpmGvqo3judu4NHw7VNxeOIUqY6Ab13ikoUlPxUHQx4soIGsIaGJY3jKGXRxEWt4+2Fe3UhHfjGzSNc7F/MaAdIKwGhH4dMrmTKI7j0+eLq8PMLVvTuD7EwaHZmwmvgSPTX2XVoS8oEQw4LA5GXhqJIBHYeXAOAKPDDMgmgtRdlK3zbimiOMTOQi30tAajvzAWx/KbzN5jw4IoD7mfUYZ7CC5oRn9+KJY7b66gwc3BeBm8SB9ew8TbznG8dS1etSrCczM4NOsQU09MBSBhcQcJ3vBdVx/u5y4TXDKPH+79gZ/u+gmp6w/mOo7xe0gZRX5ljA/zoiV2HM1BG/mjH8Y4grhf9xn9B56i08+C1d+Kb6cvMqeMM6FwdtFp9odN52X3my91gA5rPHnto9m2czh0HkL1+kii6qfR6nses8bMWwZoG4givlhCq7MTW7AnUblxnJlwhic+X4fO4MVzm95BNfpbNhnakbxfhVVl5dc7fkURU0+MS8c9Pp6UeTXiaPHg3LjD1EWIsjzPe4Gv1Ina8iEPjPiJrYYlfPjlX3hHyXHd0BIQBAmnNJ6oHv8eY8B6Qr7Zg76hGIvGC4nTQb/PONy8wHWDQN7vE4l3RCtHXRbc9HBt1nMsny/HZ+sJInIz6IzQIxnYReV1Pb9LF9K8734mPNrIvnn7btZymHIW79gm7h5yldppQ8kPeYDt/iLbc9KlVIwNtVz79Rbst3+JHTte140srpxP4XAFZ+dZ2Jz+B5VuUezyyiF7jsAdCyV8UfUp+cIpFoj+Js2KaOqO+LOsIAPrc1YmvjwRS4onex/4Hp1Rh/5bM4ZxBrRlVQw/sx+TewDKPgeL/l7ExUlOZIIKl4vB2hfRrXOoyTnOG74H+SDsRRz+wdxrsJNUnERjaCNeBi+kSjtO/y42vHULvYGg9M/HorZwdKE703dXsKdoMTxt4ZGFMzknPIfcMeefYyW6hvFSL17++S5MA95YpluoP1dPrvtJSIFyuwJzxAQcGTHIQgIJzoTy0SLQInE56QpOHZSG7POJonLESob3tDGxeSh3j6pg4OgKIkYNw2N6ECemnsUlteCoqiKkzsQ39VqKxhaQJgeXu4tO304sin6OqU5xrBEuDi8jK2YTp2J/gBvEtMawRj59/DMyqrbzatMOcv/2QTtZzY6aEKKjQ/AdKaWjrICc7tvZGvECRf1udFtuahqkHNKQZV+AX+aP/FX6ErtKn2SOC/T+WjwjPaEWJp6eyG+3/zZ4Tu3pWgAsNd149HjgZnYjuDUU24CdptAm9s3bN8gIdSkl/NnvQubUIkHK+ONKNOY1g21VKSbQUDMHnQ4ETynVMdV4GkfhYUrnS20sFomdW1rr0Auic3f23bNEXWuHVU3kDjGw31TOtx+/iL2sC8dnDhZsXkBrnYEHX7mVmKhWknQtpI53slH2JqYuE23vbcM7KImDUz5hwvEACrfcQdALN2VLcvaJzv/Pd/yMr3E5kwImUd0ooTlxCv51Vwc/F1kbyaVRl3APimFUoyf3bX+LPm0tffomlFdGcnJOOcMis0iNjuNfwJREIsHo7uAei5G/bVBm44YcGzeO3xyHga52qo+OJUamRpqWTne9k/El6/l6dioRnn/RI/OgLN3C0FDQ3ah99C/LH1pNrPeIQWfXrGgA4NEO+Duoh7g79lPeOwOV0U5+aj52hZ2F++cyoi6K1gd/pFZ6kcUZcPfaZcTENdI6czNXVDUEyOAR/w7KPTtY05CCdsAblbOLfHcD+R++xn5DAIqPXoTiAfqntlGYUohDfnOjIEFCsWs9f12AsVN7+PWOX0lXgTp4GmOmzUeCQKvjHbJKU7ke18nZ0V8BcNUKb/hVIR9zkcyJt1NVVIV7Zw2XJ/tzMkssLvFysJWS6Bb6dRKGnjlCcMlCfrz7R7Kt8EvRs5QEfMBxSy231zUx5PQuzKNctHt7ou3XE+tu4d2Xd/Fe5d2UKmxERjcTbQT/tkzSVdngcDHjYhYXJVGUVVcSYc/i8qjLLMkZyRz3fpSOFvpcahwuGU6BQYLICj34yQCvMXzw82aUeiXC0pMc9D3IyCtiyOPQ7EMs080AoDRNRnXodXTKdP7u1bGlFrj3Y9ROX15X/kZrUCvrP15Pd78oRygvVTE6CcIcnsyqy+OuvI8oHBGMp8c1Rp6eQtqc4zyub8D1T/eEhgY/JLWJ7NBPJNzSRbKflrMTy6mLqMOv3Y8vD2TyZcRkPok+R8T9UCVMZBLTKTafRqYrxlcGdS47Q3VzaaoMIC5oIh3+4XTJwgko+B3fLl98u3zprelGMTWEsjF3w9WruDmcSOXiIL/WOpNhKZAR9DGFPTZ8upX41bzOAtVijowqwL5/AX7LvRFu9KUEOQfH59Og6ubdrmC8CjOoCbARUVKOPgw8e/y49vNcDi7by5WISgJUJq5Mq6VTJkpdaQdi0A5o8ZAM8OlYPccizQRkZZD63mcEhcTzRBo80Ql10wOZcHoCoy+OpiO2nmavQgZUFbgxGh2TcLnE6/fKOYZ7yeXBPi1YcICsS8tQX9+DvT+OkqEq4gv8GJJr4jUfCa8ZBAgpQ+Hvw7imDo78MotrSiOFQwrp8eyhJqqGR794FE1EL6YZS3F02fFqK8OhMRMhh1FKG0nZI6hP70IfqxezjyROdJZkXFITvkIPYxaf5qShBs+OW6n3E0Gvr/xA2fQLcnMGy/9cDqFWoppiiS5+lTdffhOXzIXOGIHMZsLpUKPQKIieHs2xjqGk+2wm2us6KWW/ILT6cX2oE4NXk+gHK66jtPsTXLGNhqTzJI/wpcb7R5rUYhDNXwaT3KDSFkJj4hiiAlNZ0iZwqw7qnVBjd3HFJaBSOTCpe3BYFAxvvMZZ/Qh6ldmAGIy6JpvHdXk7PYZa7vhpNgdmC7zuEJnNU9SJvKMJpvbpD5il/JK9sgO4pKJ/k5uRS25GLj9EvY6jsAOf5kJ8mkHZO4fNazaL2YNL/iYt7l7aRo9ln3wK4fXhlEXU8abNxecBFVj8QrBpFfQvPsTZYAmvFcXROSKX2XFtFJ7NIq4oDk9dPNPO+tD4gj+X26S8H7yWRpuR7wdAZdewUuqFTAJSl4oCG3zWE8tkVRKPRhsoDuvCpVHiV+OHSWPiSM0dKJ0+3Cpx8po3VNghqNyDuAtTyBuWh1VtJf2qH23VL+F69g1q0xdTm74YdV87lfXfM69Ky4QebyQdPggKCZ1+N7MOLlgg3QTu089R167D99oc7rh6B/vm7aPHq4fxTU4mmxJZ7V1NhdnK2J0zUdqkfPHYF2glUBsFdpucY79PJ9JhocZ95CBR71+mtLZxrwdcMNoR2tfh3zOfucvjYF8yhMzlujmcms5GJAoJicf1xFcv47tHImnz2gWAU3AhUcrZeP9GzG5mvGyiXzWlWoFXfjJ6L4EzE86Sm/YbRp2R8WfHE1cRxwHNAVZtXUXvHAN1EW2YNDcDgACTbHoq85NQRTvxkAto9VqkLinNwc10+nbi15JL86khhESkM74ll5NpMQDsCga9pIBS2atU+8VxrsyHWy+nYxzq5Me7n+VsXQ4y+0d8NP59tkoTqU+ewY6pPwDgBiTKpDQ7pTidAp4LJ5LdFkZo6XEM2kvY4z6mw3aA86ZUJvolUppYSru/2Jfxe1LYSwoZk3KZlvARteHHeHLNDpp8L5GRk4HUo5+3px+krT6XtkZ/hPh/MlEXaCFZ2cjaoU+xr+FNsMAt224hujoae3w10j4dgkTB2X45DpMKp8spFskEMlXgOzACnaOdYVGtFCidFASbODPkZ/x6Z1HcNRytvpF2mRGJxJvOiOGD36uW2anrGkuHp4sw/TJkYVreePkNBImAb/9MXvWGpZo23n7mXSROO4GJehIvTyA7M3tQ9mwndzN6ziG+bf8Iq/VlXFIbXt1eaE1atCYth4as5XAizMn/HZrApDEhd3oQ3boeqaDGYr6OsKCGLlUCgsTJl71w+vomwjvvZXII7DZU4fnbDDqzLtMeZaTTsxGrLgKrOoJRbWKmasqYMXw68wXm7Z9HbVIunTHhPJV3HtV0I0unB3H12Fl67FdFGW4pfOgHx6pX8HP+WzyVAXdvupV+fS+b7ttEgU2sfzcutANbSAetR1rJf1LMWjo7/izrvl5HbkYO3UOL6HcJNBQ2EG2Ipj68/j/IEUt4p1ODXW7h+Whxv557IpO/R5yhxwWtRHPlgCcTLw1F+baSnsRRNISK6lZ/rPyDFUGivL204DqZB/ZQnnUbjd7ie/VaxjUCuxf8FzLz1n5IVcHK1Jd468wBUCopGFqKWdWMpf95WmRODIIT843zBIeT9qiRuKQyGmw/41cNWoWBpJGX8PGcR2B7FHPaLnFw9FhS3OvIVIPCZiLj3edpiHPgl5RIWOfdNPhuYYcRnvSCUKVAv8yJBNcgOCkVxNjK1JNLGX1BR/fSLvqkWxh3QoXCHsb5Z3cyMtbAu5oqNlybRu6VLh7d8yh/L/4bi1ZCbZ8/o8O7OB3h5PHmXown9cysuodvHz9KjCuY9X4woj6II+/fRcAsG1WzSyjVgOtKCNoWf655jeeJUc+xcGIRM/Z40+xxnpKkEhrDGnl1SBFpLR8Roexj793fMsb6Bk29N7M8P/GDLaYizvrv58oECEt2ka/8Y/D48WkiMP0Lv6BNC+eHe7YAAi0p67mc+DoKu4IvHvmC2yO/o7x5K8H9Vvw6w0iojSKwJocp1WoufPEMulucpF0eRW9wOz+P2cjSgZV4d3tzqi2L2q4s3ORhONTVqJxi6Y6HfLvICn+Zb3vgmuIA3d4LaPUU/Yez9/zMz4FwxjGPJY3+tEhKmC8pQj3QgZ18WryPY1LVoJAv50xHOjlFiXT072VyTguSiaJqSktQC7fNWU9GhozxTz3NOEHKvTgIAFqBzRIZ5wQXr7z8Jq84nx3sj7t6g3gg+DJzlZ3ky2GzFLpC0riS/Cp2WQN3HJuJTJDyvi88hRmPFWNxtjuhGA7MOUC0SU+7cjUJybO43T2cps1NxJXHURFfQUhrBgOqaow6I4dnHGbOiDmovdSM+2Qxn+zv4odOMVt9qArutV9iU3sBcc44zF1m2AbCawJ2pX2w3jSAUq+mbNQqEi5tZfrR6fyxugGbm6iG8a/5FVxxlkjLSNqD3en3SUCmdcNQ00zJ9+dxS/XiwhgoD9kFwNhzYwlvXYR0iJFTr5xi8puTGUgcwGgz0vJBC+0X2rnv6n1krcuiVpNMzoWbyhut0aPRuAlofDV4zhpJD9cojxSPeUq1aH6ajb9Sgvw5OR5jPegu7UZulzPmmJEB743AAwC0xYyhLEkkd63waebZGYl8/8Sj7P2jkNqhTyB1OYCXOWOGBc2QUJyEbsutGF5voClpFqcmX8GoN9JwYykL0jYg9Ir9YdYH0Jg4BZs+hEBdNX0mT9o9rqI1JXL+g/Oc2n2KyjE3JXJlUhlWrThmtdFwekYLTT51jAwZyfZbtlO904Z7RxWCXwyOJnEf32/rGDz/X9liPo3XcSjcmO5o4Wy6N0laAwl1kfi2uHD1OwhuCsbgZSC18TtK+mbTM2Eeby45xFsV4nOR3YhFRJQc4iHTcPIiVtH10pusCf8c/dEUqgI/otH3J65b2jjeABnmNZiiliJvucALP77AkVmFZA8/jU0hXluMrBLji99w2jQapysQgEfbn+eL1StArsFdZaIrrh2HfyBuvW4gqHHKNDSGNdIY1sgvqb+gkqpY8MMCvv7QzLlxH5DgbeL+/HSE3dOpm1SHRHITNPWQwuYA8fdvesAlCITmvkKg3E52XD2RCrhVD3cG1tE5UsIM05NMPHc3ZfGF+HSJQOi4q1ncv/JxvmgPwrvhLypDNyBIBNyl8KkvqBTXWampo05ah7G7F7VFSYgzjFlB8yF5PPSWsLEngi9PpiC5X8GX674clJRftGsRTh9x0+oIDKVxaOg/nt/2Y9sJ7Qzl5btf5uFvH8a/zZ8ejx52LtnJhIgJg/f539Xs+89g35iLY7ApbJSluhH0vwAH/23/e+3/Mtj3xx9/sGrVKmbOnMmRI0eYMWMG5eXltLW1sXjx4v9/XOO/7f+BJgggd3gO/q2QwKF+mPHjeRjSw9TURcQdSyF/VCQfPjUVjUlDt7YXmVOHXKpk0b4HqQk+zNWsq4x3g6/84YnwTiotmbhb9zP3wFwOzzhMUUoRQyvjKB51hU/cptI2oOa0rx/l/uUASJ1G9MH9DHjZqFeuJW59G9LUNHr3/jq4WDY4oDtgDgnZ40lr6sJ7nDezD82mJPFmjYP60GakUpD7eGIITMKu0tDTdRK5AwzTrnE8vYT2mO1E/hWJQ+6g3b+dEae7sHhc5OEJBxFGnuIuy2biy+Jp929n1KVRpBWkEZD4NeGTDpFb3YNBGsOvM77CprRxfewFptfHkOp+lcvfLsdT7uR6hoSGADFwEy6HKIUVQQK/3ijmbA60cX58KS3BLSzXwG1BZwFoWr2NUvtQxt27kPu8xIBAhBzKfPrp9ZYglUv5dc6vuOwu7jx6J4/ab8fU207QzFp0dm+WlT7I6h/GcXSamfPjznO9ZQa/9q4iaPWjlJoi6XULYe+433HIHegMYibf2aZYXJLROLxUmN2cmN0GiKyJpO9aMr8MGcWrqmPMM7vzvlLN9RFmOm7Uq5qnFbPcqqjk+huPME7qhrBWgksq4CmDGSQyTpPM8wRwzfMUqz1lWEKiMUvdUA8YsKt0SB02nDYZwg2WU29APObJ3bzTb+G0rwXZHD2Xzb+SNPVlzssdeNuVpBx9k4tHb2HP/D0sqLtKdnEVpYmVCFLxZf3dBHFjka8Dd7d8+pxLCem6HaXDl86YW1E6T9EaruOA20+c6nSjuc1CbEEapUOryDylZO/u+xnx5T4eHHoHJyuXgmQJbVE9tJnaOGOGF8tnM2/SWPz6z6Jt98DlcGF1WKn2qubrdV8z9txYNBedmDpNuDxFB86hFJl3w64Pw6JtIjXvc/oa7hocs3KnB9WGdHItEgY8XkSFgLzCwq3bbmXbLdvw6fLBZVOw1tnDLWY3zFYbmiY1MeVBFGfJMKsduFnkWAO+IkXqYIvhZ5LO3cxYA3imOhWv2flYoyI43QUREyIYXTCaF7e+CECXU0a4sJ3MEG8qXTMRyx3fWCOkMmoyljD0HvHv+EtbcRvoQuTtT+WoUY9vYTk2Nzd6rQGDgUltv5nOKWeZMK0E2Ukb06bKqJpaxLWSHhzyNwbb/7ziCUJ65zNjTDrTv5xOa2ArBWkFhDkVpKNFHupJ2KQeymqVFB6+jiFvF1IfJa4uO7/HV/KyLYXuT6cR/JgXTfFNVByswL/FD6PcQq5kObv2eZFa/Camtsdx83TSU9sj3kdFPI988QiyJeL4Cx8XTu2pWlqeOMLMxJn0ufcx6vIoTO+IWQTt/u28+9y7BBnupDZoEz/mLKLx+BBCp4Rzbtw5ZhwVwY2m+Ik0x/gg6V+MssIP54pZgMjm8zRlslgnwSwoyd14G2q7joseFznxwgl8gNpaL37tj6cpKZSLgWEE+piRyqXoAnR4qqWMmXuRN0TVW14/vY+RC2fgsltRGzvQ9uiJL2gjqTiZytNFTP8PYN/BKz5EAj5dPsw5GMh+234SX1iCur8dmd06+DmH3MHxacfJcPNg9JUuQuodhBAKiI7p4oQybp33FKfUVxn3pRcevXfy/qrlhEnu5nAIlPeq2f3RSqKDTfjV70RmC+HrMaPICIXJzR/SGbWJnWMu4ucWgWfiWFpUIs8swlOUzOj2NPH7wveIDJyN//FGRplGcWm0yIzeP+tL5ku+wWmx419zjeZAcS1qckCGChodUNoj0DjMxNU0keyUfnUEDiC5w0GRqgqLK4DoW44yws9Ijn85200OttfC0uI0UrctwWuFiR13duBnT2W19TzC1jux+c3gQMR3RBBIzvB6mjU3wZA79BAsb+RQzVuoa0vJvhYCMeLGQ95xDOoCwHcku6bu5uf4E+yzzBs89z53mBnUxwzVYdYPj8B28Dy+jdfp02rw6Nez3N9CmVsTV9ZsZUj07ZwbdxaTRpx1uVZ41liLvjqe4OZgBuaZODP+IhVxxQxoBzi5/C+GBJjpNSbi278Il/uPXOhRUumyUUs3PtUwxrKEGb8E4woP5M9bn0Nyg8Er7JjNfmDlk6HcXeSOus9FVkYjoQ2hGLwMzG4aQCaBa0B44QEcOi9al4i7iONTjt+sl3WD9FEyTEVJWB6jhVQEXGSq4BYd/NzjwOVw4dHjgXu/OwBhsy4zcUIOp85n4qb2wTSwlbhKDwoz/RDMGirr/cCg5WmFL80DDlb/8jwd3sUUJ13Hvz6M4qEpHBn1EsZJZXyUcpocaxU9uhbu3HonMdUxGAzhdLd9zb3j/mBFWT1JlU8yp2ID22ZNIKoWVnnPZa1bIWu895JruoVjAkiddsxuTvbN3UdRShH3zn6Rc+egqAjUjgjUKeF4T3+ZF4qj6MpTYkvuQhC0tHscQJA4GLlXDQxFE6iktykAX3swU06lY1VZMYUaCMluxgB817AJ6fUmJNNqCC3LwWe4D2qLuAG/3qsiZ+wFpK4ivFxxg2MosSaacKMH0zRNZKrO0NMxGpVUT0makg7/a4OfU9g7SPLrpyekhcbsIsbtO8/hmYfJG/ESioLPiayeQl9TOFb/ECgBp1zJkTnN3KmWc8vyP3hedRipVs3ZcQdIvzyV0ee1XM3wpNvnBgAhEZDHRrJ/cS9FPtWYtCbeTeqm2wl7J59Eog3EU3kIdVgXBk01cR5mzkdBj70G5YYPuK16Kk9+8CT5qfmcmtHFitaviPAoJEFazkpfeNhiplpwIHFJECQCd7WBtq+Dl9xdHCpJplsWSEVgOTZXB4JEIMCwkJGXYokp/pCemkc4/ewB9EF6WjVhrOkvZ0V1GuvWbOei+VVKzC34dPpg0ojkgZXe7SSH/8F5yTJUaVK8aobQpD4GiIS2P4PgAAqud2fQ7xrC8Ypw3tAZ6fHs4ZV3X6AxpJGie/ciN3kiNVlRmgykeiRzzpHNx75gNFfzVrs7F2TbWKyPpWPKZfojG1jU/hau/kzCJ4SBHExaEy4kKG2+WJQ3M0dADBBo06KpGbqAqOt7/lGjDkRA8Wzko0zePpqo2ijee/Y9LG4Wps/5CeksNz5zxPKDGvQyAauHnejVu1nvBfd0HiQnPIIR3lW0RugZ4TeOTUovCgwgFUwklibi0ReIQtWBptcBPqKfIEhcSCTwmq+Lv/qhutPM2m/W0hiXQUuC6GtKJVJmZI/E42IGPTMnc03dy4BWlMDz6PVA3yZDcN2MZPg25jGk2sGnKXKity4iXabEeasTrVGLXWHHprKxwwjXy1+kMvhtpJ5q7s1eim9DGAq7GDyvtMNK78PE/3gHqlAZ14dX4kDMILUDHxqgqldB0JGRhPlZGNaxmGNTj9Ee0M5QpejjWn3HMuRUPNq2UMafy8eu1hH7wU2FjUN9w5jw6kiCMq24MrNocEsErqA1apl9cDaSMBPCCmgJFpUWPNucrPOA8VY1JTsWUzfSQWWsC4NUrEmmtqiJMet5yd/BxZAmVO4xHJpzBqtSPD9TBU94wQF1C1feep24xleZ/60K3aRMyuPLGX9uPCv8rGT730/XZA/cjP3EFjbQpX6a/PgP2dgLQdIQLgonOKD/mPiALmJscswWkc02PmIbbdIvuOZjpHz6KSzWGXDDRRjrBg/+vZjCi6k4n3KBSkVPUBI9QUlcj1hD1fAfaHHA4tBWJBFr2bFsx2A/GaIlhCkMXDo4it2nk0l+xYjOKvZjRZId71kfY9XCro/WYZd24J6iJM4QR3V0NU65k7+D4KSxmCW5U7DI6lixawWJZSIh6Jtph5hS2MokrYaDYasxzReYanXjvi33cSXrChvmX8dPKKdIFcoPS/+isT8a775JN5YuKTZNIb2mTnaf+Yi5E9/AJXGBIKU24Avig//i8cSLfHv1CwJMy4AcXLIbhEacBMkhQO6gzS8dD62LonHnOTriT4w6IzIgQQnIqukzh+KGN74d7jjkGkwaE/ZZp7h7WD3zAhXUO+182uTGiblOKoLLgDK+UWcSKwTjMkr4YPb73NkQzKi/5lEdMYQRYRW8PGIIucI7ZEt9+Ovej4j2mcR1+TV6lcWs/SoJv04/WtpPUns6GqlfJC61mEEZWZKHbWoOatS0176AUguNKX0UJovy/ObeTO64NIe+Vgs+2R9xfP8o8tMKMLsZ8ZAG0+sSiTm/BkKLE0rO9Q4+Z+9ubxTR9XwwpRStEEK7+jHeuHKBYZeG8ckTn+D07KMxCvZqT/Px+Xjm751Gc6DYnqbPnfJIWNEC16Wz2ZTyOOEhJl4Kf4nm9MnURDw8+D3OG5FOwcOT7uAU7Gp3fPunEN7+ADWJNchaxiMIEJO9DZmtB24o2z/lBadOmMnY/yaWCQ/iupH1060/ywoPOG2Hi39PJE7tg9J2jYwTB3BJZRStkRKvgNlJ7WwMPkRm1dfEnN+D20AHIYs24iOdxas+YLfu4+sYLSUecWSc07LywEi+u3sfBd4dZNVDfVo2bSnZrG14jbnbUshN66Y4VQQeLPpoOsLSUbqrWbhNBEAtKguHpANkt8KjpR/Q3DsUIeAEVpUVl9TFjENjiKsIJnfRJd77chS+Sl8ujThEUXQnne6XGDGwnEc94a1mDQWZXgSEjCD3wiK2DsnCGNjKZ499htntXc63K5FKTRiDs3H6FrJw90K6vbvRDmg5P/kYkhv6sjKziwmHLTzQmskT87P5tR+silhqLT+THViFQpiPzCFDkAiD8+RfJkVOaOcqjKoCuvSnEaQCNpWNblU3LrWE9p4dpHWKa4Jc4WDxQzs5KTNS5teGQxVG2LObWOHp5P2ePsyB7SzJHsWJATNcsWFd5UnN2HuxBIpAyqvNoYxv/4gDrc8Q1hCGbCxi9pHURaEN0uthqX8adb2dKFpiEOKcfLtuP/2afJ70hCWh2ShcRhquD6X3XCTGZQIFqvM4Fe60BrSSUJZAxfpPeKXxSYIqgvl+w3e8IClmQCVF6+ZCiB0gNDyatWufIrMzlTm/zUHqlCL/7nY2B7fSfc8PKJ1gbDMSqe/lfFA3Hd6lvJxQxldygXG/jUKfFMpHK19GEARCj4SSezKXrNOTaJrRhHZtIwPRAzReaiRtSBr14fXct0GULXzttde4OOYis4bMQqlVErUwjYHDpWgPxzCrogqpSYpSquC75o288eKbzP12Loc+OMRLb77E0elHMS40wg3laKcT+n2jyBvpQZPPfgRJENYb74OLbx0nsFJFfcosYnO2M6TmObrC3BEEaM0TgSmr1ouIShfe9eJeLrE0Eb8uG8ZxPjx4/UG0AVqWH1nOwcqDfNT8EQqD+O7WB+tRhOsRbm636ApPx6YHfTD43jEL14Wzg8fOGSeh79ejUIj7GLtTnNdSl5TIak8KdKf4F9hn9A6n0e9G8oJXK4HuPUx573POJj+F4zvtYJvtTtg7ANdkfYxOKSTKVwRdotvW0+5xgJ2cx6MKvDpDCOheQLrlLiz6JbTqxXTAJw5n4wrYwtm5T1Aq7WdawXisV6z0TbupFPFb1Wd0hm0kuOs2BOHewfVoXPg4Pr/8OZpX9IT1+NEedz/aVUvZW17K+YzXALj9l9vRWNvgNggrOoTJPYjegHiGqR2cCoOzL/zEs5VvsiCvh/u/v59tt2zDom2i1y2PzSXr8Ct/DALBsf1v5pvH8ceKCsqDX6FHdwU/GYRJQJDA6P2H8I4fYPeiY+AQ99r2jhH4GuMIDLjCtfRr9Ho4UdtCedK/gxBnCMGyahaHWbi1aQD3PwKYXPgsuo+6wXMIALP861nx8t+80P0Ifq/8jtLcw4G5N0lWcqkcmUJG+t3p9G1pITM7k7/nF9AeaGJzUBtyjRxNh4QoQxR1EXUYJC5mN8GHbXFI9o3F/pDx/8PeWwfHdW3rvr/uVqO6xcwsi23JtmzLTLFjjDFx0A5zsoMO7DjMzODYAUMSJzEzM0myLMliZmqp1Yzr/bEceefc8949t+rdOlX37lHlKle3eq255pow5vjG+D6sassQdelPRrhQ8SqV8S9CQiXzmp9kysFYIkz/ZNuEDLQLDzBuWDOHPr4RXU8gknvGsXHMVDxSJ4ESWOULu20W0o7FENwUSdpVUFPZLCfG+AMEPQMyFZ+Xh+N3diTOmx30BYm+5v1Bv1Az3MXEvHTg74CcWxDbN2fXHJzfOGElyGJk0AVqqxqDrwEfpQ//mf3PNPs8MjkfPvYhTrkTOWnMlL7wn17n/zb776Dv/Mv+l8G+N954gw8//JAHH3wQnU7Hxx9/THx8PPfeey/h4eH/O9r4b/s/0AQBgozTULgD6fT/g29DYK7GwqbsCo4FaUhsV+PbXY3ME4dFa8ahNbOk+TbSz44i7Y45XC7bjMKYyoXRF9hqgnIHPBb3Gbclf8Tr7c+x/fb19Pv3Y/Az8PZV6szdZ+MxHEtEu0oLV5N2fl32K2qpwJ+d04jw+HNe38CdjtNsv+MC7V4nWaSF38NBX/kU98j9OV80AsEwhu9Xfo9FYyGyLwh7eg2FYxuQXsUp6kYuI/X0OmZ+paX4GfjBaadKbSfTC27ccje9fg1suHkDWWVZqMJqaZhjo0zuRmi1sGLTCnbP3k1ZZhnZpdloTuST6rUTf4eDOJ0b+1Xh6e0z9lMl8ae08yvcbYewadyUz2qnOVikQbhFB68HwTn6uHnLTwS399Ka10hZrJgZp5SAQ4Adstv5JPgHEt0ZOF3XHOS9FmgwyVFYXXjcHhRaBR6n+L0dAyoJ2KI6MHgsGL1NHJ9wnNYoMTAkc+vosoayXaknQBpJqNqFXiUG5z596FP8BvzYVX4fwy+NInVWP38s2Ydd0UlUSxRmbzNumcDeAz9h/7OYpCXJHIi5Vs0yow2kAtwVfQMtcYdRyuIpuniBmOZoIpNb+CW2ksuu7eCR4OX0QnB5sKbn0X3VgfTpriH19ze5/PN8VGNHDF03lBwmlZUTFp6Hd3sAbh8zeHlxOeFJXLJBguL8iGv0xS+6kzMvvEti5PukFr5MZdRzfxvX15+KIqE+gZm53izYfhP6yCxMASpaMmYRENyKcfdkNMdyOfvkKTZNPYxH6iGl/Gb6dS46bLm8o/udUlMMxq7pCIj9Xe0EldRJbte7jL3XQOj4Ot4KeINHcx+lTFsG00SdoJSls4nIi8AdK8GqDUZAgscLauOr6fepw6SJAImE+75+hrbwK9SNEJ3LIFUPb+xYTI9Ji/sOX7Ys/o62yDb0wd5UpX6FX6genxc/p3Iwh3ZbCg5bJ5P2ZrL17jIio7bSaYpC/vIqVMMs6MMz8O+soCd2JM6EF/GLbaLUPZJ90zZjto7hHaDbeO2AbvC4eaUoG/1WK9Kpf9JwJJfIynq64vJxXdVC+CuLyBgUj9xhRjVuOK5jZ7E3R6C+XcAVO5ILh/Zwy0+3cHTyUcKbjQR3juXBhRew73kbu2UMvpNHYlKfQq89QayXqKHR3hJLXXs2MqeMgtMFlKeX0xnWyZ1fPIhceZJ9D97B5cAAJltsxF/aKrYlXMaZ5DN0BinY5/idnphzZMYm0HqulWNrjiGRyeiVhdGkmIlP3GsY+zMRpFIqt13jYQcRwMvKygLELHGAD2/ZQpnjVwZ9BIKd76P0UzN/23xyi3OpSqni1Aw3ckDna8InzIJULmPAb+BaXy76iHeT9nP5q/lUNAQTcNGbf77+EucnB4E3PLK3BKX/PrJ8fqbbKsPj8iCRSqjPmk/e5m00ksrJ9wY4OOVGZIKSb+V3YOoyYeg2csJ6re32DXVUnN/MrOJbKZ31FCkn1jLl6BSKRhRx18uz//acDo0vfUECSruS6mEW5k2Jx+30MOzU90gQaMq8nk1zFyDxiM/hwU1PtouGeiXRdT20RDcz2q7j3vgaemUa6m3nUDgUQ/Q8vuY8DuuP0GTrIbAvkB5fD4HtpRgCROf9uQC4IfopGgKexM+ThY8KqtlFZcQZAk2TuPXPThIT4UjAOEJk9XgEgcCmIqZdnjcE9oFYJeGyOokp34vHKQaRg93+jD2dQvzRGPwz7fRmXAsYf79qLcOvZPCgroeVnnmca/sn38RfRiZzMr3tEaT+5/EAFzUGpOnl+OhmkdP4NhI8fOPZg6cikZujt3HfyFLO558g2Hkt4x/g5UBwCFXs6orAVJZDp5eNm4s+oi3WG+G+95E0boD01fxsjOKwdyNBV9Q8ve1pti3YRtiEKlLl155LPWUMV5q1JJTuIjE5lh8Kelg/COsG4T1B4My4M3+7d3vAL8w7OY+8ojyaF1RweNo+ACLaIlhRm0F6/HneXP8tKm8powcCOF/2HLtefQWLStSIMbnVVIxZiU+Iiv6Ae/92bb2/nk6nkZgKPWPOjcKilnDXhrvYMXcHhSMLCTZcR33nbnxvOkCFPRVNUTzLzy5n76y9GPwMLPltCYHRNngcKqKfBuCybRcjvGJ4LQgS26IoadUhyM08/tHjQ/f9c/xB+nyczDl6BwPBVv5YWMXv8w4zrOMrfI96MKis5PjYaXvrIXxvFNB1RhJgSKdyWDv9gX30hG1hblA1MgkYnf0MaMVIwbn8cwQ6QtFl+3CUHJ4u2o/Z3sOww0dJ0A+AiMcjRYqhS8uetxYQML+egMomYupL8XJpMasiuTjqIlKFF4PNA4zcKeo97L1pAmf92kiqSeKWku10Dh+NrFHG2Csn+OH2HyiadD/JrT1cjNLCdU+TlAOp36di9jZzOrmUIKNId1oesAhF/gDCqD0cTfxqSJy9Ia4BIayPi5FKNur9OXc+GWFQx+mC0+y9Zy3HI6XY1K/yyJZeBiq7mJ9gYuvcT7ErOkhXQLQXKE1lfLWgmPvGFiOxLhoKFALYPWsJOd5PR+GNWBOz0NdUoTF0MOKCDuuxZYR88R67Cx7mMfkJ9LoTbLypixWbVjCpPpnoxHOM3ngzTa0JlD9xmMCxH5LrgbYr8fS/v4Q5t+7l/ltPs7ojhTPF7yGJ0FM8t5hoLxFoqbaEU+xuw4Ea56gi2iKbyG78lYLkjazIepnT8q+YvzUbRXMKEUIrL51/iS/v+5J9YV0o7AZ+jKgi7utTvHypkQrfrTSqtgLQ5b+NC1lVyJwPINcqcRgdlOwrIS6mlbMJp/nCPcgmRRLRvtvILx7Bwxsf5uebf6Y2Wc/LgRAbdhNwE093L2bmwfvI13tYe+cnlDpgRXMQj/Y6CT26G2teNKs2rMI5ppAPJoqAYFRbFHMaYyj5cQ5N6WeJvXKK3cGNkAZ3+ECr1Jvjei9efO1Fukfb2DmjhJ1xHUzTiAfxSvtPLFW0YTCE0Gp3cveWj7mScpJDkz4ksDeQ6JZoHP6DeIf4YdMG05wUxpEponZ1SFcIY86OQZhjxOQuIr7xDgAm+Lh4MRSe6HNywurkBlU36z6/kcDMZjZFyvC4GjhV9hFfzpzKyeZlXFG+S7MhlZF+HrjqJvhKYEHRaLxaIrAFf0JCv4PI5a2EWOClqD20d33M7F2Z5Gybhc/SbtpSJmH2j2Hl8GcJ865HKt3C7xefQNNYh1wdiMVXOVQtvHPeTjrmRvG6QoZvVw3RV/Zh9gni6PxT5AZaKJ52BIctlZx2E0+99xTHJxzn8LTDAEi4dnAfPucFvgr5mp4QMRATLINDJjlUxyBVurkyvRCDSvTN7QI83QtKjxz5E+8R3b8It/UsvUFi5eADfnCPL+AXw/wz76EsK8Ls6wBBQCKTw9yr2lA7X+D0mDOEp2bjG51Mh2UQk7oSL5sXmeWZNKmcqGUa1BY/nHITgkRghBIW+VlZMLefoMgJyCyH8B30xagzsm/WPl6+ex/5Ksh7/ifeta6B9mt7Woxc9JuOdSVQIxTi593IkcmNNMY1YtQZkYwoY3j4AB8HPUxNjJFbz75LVHUNxjFiEPwLAyRJY8hExv1f3E9rghc1E94Q9xRjEhG6OqQeC7e3R9Ivq2KR3s3E0omUZ5RzX1YfManNlPZMxuMBR78ZpcmKQ+MPEg+v6kVq1Z/D4A8EMVJ4le7+/D0v8XKolu9+z8U+6I1r0EpMvZweVQBuWTPHPl/K55ll6Fd8jNoZz6oLz+M5H8HbT7+N1cvKrV3Qa1eS90k+nVH99PmKiS8nC07yxAdPcCVtJbWDL5JivIv+fpD5H8Xb7I3SrmVX8dcoJVJmTB1GRcMjqFyhRA7KWLF7BZfzbWzXpjFwzEniCSWmu01ElHxEr38JV0ZuxeoF7+hh4HcboR1v4p44hejmaAy+BrztKXx+fgkfyzuxzXkXnVckwxWPUKoTQc6btywhuSyT7vuPsW//G0QWNLHy15V0hHfw4+0/8kFuDaWlTdzmyKFr3wjcw45Rk/Xh0LtOCrxItiSY/V43s6JdQplZzeKLESS7UnlvkajpfalfQCqT0hrdSpjagWAQ1/XT404zrHIYCQ0pWHosOL2dxAx+QHtyCU4aeHTEneR5SXmhayQu08QhTffk9hfwseRyRnuSINUOhrcsp6+9g0PT94uN8sAqH3jaH37aMxLLnomM+zSYrj+7sA3YGHlxJM76GOpMCty18yncakH9hJodQTuwqWzIEFkNWuy+PJfQTlVaHYrAKZxacIR1eaI20+Zw+ELqpjW6FVu0Fr1Dz6+pIUP9MuH4BHzDgPvAExtPfW48AConZDd/NfR3giAmrti8RKB8+FVynICwPlpSKog0mHngs5s4P/ocp8afYrY3TFdDy7l8BoJduEeJmXVSjxsFLlb5wJ2+sMssJlR0xCWikPiSIE+g4tx+nlD3MqXgDSrv/pma2peZ0OVG6rIzov4nQsNWMMU+jaLkZA4f80LigpBOHb6xvtgUYmJOQ1IfOvl8VKEuzo0WpQMOTTuEQ+mg3Q1921zIvc7Q/aqMzdFfE9V7OzKv4UA3y6V2zpCFj1ZGU0ofnXLRb662+BFeD2H6+USFPYJPkEDBhld5qHM+m+dtJ2bzPIKdKn6bdhj/8nE0J8uwoyasK4ywLrEiJ9LdSKCrljHFBVzwCMQ0hTM8oY2DFnhNDytDgjAZNpDaY8Fui+fF9S9SPLyYbQu3kXYljfCOcNyr3ZirOlj6cyi+hngKc0/T6qfFprKhtqoRdC4yCiPxM/ixb+Y+rtt/HZ/2VrDjpl/wmjuOaXHJvGTpJ1YlJos1jL5InlzJFbsbd5QFvXI/TUFGolM6wALHvUrolB0mvSmW4SXD0Vp7San8J2898xYr160ktDuUinecRPV6M/PnWZgX90Pi1Sop2VXJDsHDb+rPkE2xE1iwnwOGA2it6ZwYv53pB6YTZRhAEARu/PJGmvqHEVPhgaBtzBqsomrhTt4w9+BxCGi3+pNYm0hnmAiA9dq9eGjtXVg9EupT6vH7YwcRiwIYfWYOoy6O4p2n3iFz1yz6e/rZ/+R+NmzdwI+3/khsfCxeSgeztT6kTIlDmxzJByM+oEvWhXB1nTV5m4bGv9sj7q/fffM+p/Y/w1QvN7c5IQwPndhZ/82PxH65jrfeeYes1VkU/bOI2Xtn05/TD8PFa1gH7ERUneLIqC9wyp3c9u1k7EtK4MEcan6/jL9ZS8WEu9l2ayqBLd8zblcgxtxbafxWHH/qwW4KzkqBWbSkJFJaMAN/4xiSveSEZoulWHtq9wDwZOaTCL+Jz+G0OnGZPSAo/oZg/Gtlk8WtYKcZTnflc1KSyolVHzBF8Qz6Oj2Hkw8zrWAah6Yf4pWXXkHqVvOv9heo9lR9PqM0Zg6qDnC3r4DcZkTicTGydR0Xo1YC0BbVxpalW1iW9QLqzR0oLQOoYoIRdDDoAeS1rPz5JgaG12OP66Is5mG83D48JpnEP8Y+zvx2KLUJ3PDTDTyW9xiC4V9oeC0N3PTLUqS0MjixnckHwjheEEmftY/1l9YzZmYBiaXfozb2Y/z5G6IzYiEXpIIXKmE6/rEifFCXtwynHEzKatoQKwv8ZODGRWByLD9M/5Ku0C76gu7mhQDIc6dw9O1X8PX/jVb7KTROsY9duvM85ANPqpTQEcKrAxYGfU1DCWVSxIJ5o/Ag8TWj0SdnsH3B9qHnmesNSRIzH7kn8salA8gJIs7fgV+snqf8f4S6PEi8Cx/lar4vHkd6ygp+T/4Jg2oflwaOEtUSxZyT96OfpUeTLybKew+0M7V0Kp+mtbJiVBU/BVTx8vAviHpASu7F28VkNpmNvRZoa1AxqyWYUIubX5f/+rd3LkgkfBYC9U5Qh0dTPXoFvUFmLN4WRs45hU4Kn+4346fSESy04bka5+1yg64WrvOfRlDdBaYdmoKwSMA0Zhq5qX9VaUpAG89nGZeRDSvms7hd6Bp02JUC+dplbPEDXxHnRN7eSHxxMZ2J44bm6KXhl1COFjeoqkeqOH3pNMk1yTy+8XFyp+RC2v84/v9nlX1IJBj8/orzFfJvE4G+v2J+/x32vwz21dXVMWfOHAAUCgVmsxmJRMLjjz/O1KlTefnll/9/b+S/7f88EwRwyQbwCOLGd9gK87QCK57ayFe1CnKC/sEJ71k4VKXccHYcq1z+ZOQf54crCVgaOiiddg/7c0RO6Q43dFhhcbk/9fsm4F6kpzG+8X+452n9Pbi9mok8PMhzV57j04c/RWvSckfbOJ7M/IN/dFhR9rfQ0pqIG5GetvdqBkNA33HGTvTlsud6/IeFEnv+IU6on+b9jz4H4Fx1LDIZOFu7SDp/CIkg0B0dwe3mVWwxVqHvymFyWignxkZRHPQjVrUVZ0gv/khZ1Def1PrlBE9O48DkGlrie+j1a6V1zn5qIjq55+t7GPQTaLzeSpgpjK7QLnz7/elzytkZ+jEt955G44zGcpVeDmCfBQzdsDJJwZLwPylvysJeOsBtP9zGyfEnyRhdj0ICDVJxJRcQMF4xkVGWQU1yDUu1DrZsncKC/YMMvjHI0l+XYnVa2Vu7lynedg5EivcZ1uChU901FAABWJt6hPi0i+Q1qFE55axUbebtvlz6AuroC+pDJZmFvFkgsvooNfFKRvpt5VTaGFqjW9FnVRApraDG91GU8fn4e5zM2xJNYV42l3MuYxFgWsdeHDH1/D57NxnSRZx9/yir9q7i6xdf56XL91KQN5Mn7eepfv0FpPN1tIReGwMOlS+GhOH4J/pjvrphhdafoetcPQ/cUcqumjQG31uAavYAQp4Rl2M/HWE17FuuYLHgzS8p/cgVLlY7YPZvTrIil3Jmqjf5MT9zg87Fz2djGHNkKh5fPSFNFXSEtWITUpim+w11hJEXbZWETRnALncPBZwiW0TkeUAayujDiSz5RUVFynkWvTSOYWl+bFn2Gy8mHQS3yA9vv7rrymvkBEWLKKZH5sHjJSCRSpDJoHzyAwS2FBM4KZYSn0DKYzdSmeXknYg1yFxSpB4pHomTCZNjeDq8hV8qluNyakiOu4PtzY8UMkgAAQAASURBVEUY/D5F3dHCqp2riJh8hQev38pjDhOFziKWV94KuMnulfLrGPhnl4Vz+edQhyXgck5D19dI9JX9eCoKOJirZvKM21HsfZJhkTrMPWZKvjfgK/HF4GdgktpOgcHBkfZwLGYnzSebCa89iT4iE5fSm9jLOyj+NpyJ/xiF2T+KkKaLSOKiOCk9jCm0h4vxFeTKm7mhfzxRrVGobCqqhkdQpXifJUYNiRm1dAZfj6v8Av7dm/DE2kmSwxtB8H6bBdOB99CPX8a+RRFcTniXiPYIAFw6B7/pFtOR1Uaq+xA6v0jUph68cjVMvxBCYOw5WjwhNGXPwycTsnKDiJ8Sz3dPlRFo2ku+8w0WLjnPG5OLUQWspWyTGOgrymtk78yN3Bb3AYODg+h0Ogq/KaTpWBPei5K4Un4FpTOU/oh0ZCpAmQBAcG8Ud3ytRupaQ/lLxZwfeJ95kQJtkW1UplbSExnPqC0BNMRPYexDv7ChS8lo3T2cjj9Aa8gVQh0LwBmNv5eZ1f/4nTU9Ogoe/ZGCpwqYNw/6gmx46GORrotvg+DVqxp/f9z8B20X2zn6OOwwwTwtmEJtaMPE9dHLC3qjhuOQlHMu/xy3Sa9pFIjzzQcvSQLTDkv4+v4dLB83iXh7Nkgk9EZm0xOTh0fmxqGysuyXZUjD/aiYeZydC65qDHgkvB4j4WCPjl+/mIR05D5OT1XQHvgdAHE9D/LLoQfJHbcIn5e2cK72HfTRi/DJPAnApwOwzQSvW8vJC2ygxTkJ5e6TJEtKqBkhENw2Ay9JM79LDXy14RbM47uoyh3OoPLawXXujrlESYx4zVJSOfYOaqNEiqwNoRpSFC2svzKb5jA3EZfVzGuYx/6Z+2mOaaE5poVF3ak0fr0c+S0m1OGZTPbRs230J3zcL1bJHUhs4re4JpYOziHw/CVM/lFMi+rgAOCttHAufpBaF7y+J5zRLXM4OP0gCk8qD9aqSQ/KZV78EQrW7eDmg4+RsWEAuXwCEmOl2HBNJG+0ZdNj72eMxENXaBc2lY2NRlju0fKP3+bS7riMJGIUxoBYHAoJ7cHdzG2H/o4Apl4YjlMz+Lf3udAbng+A1FVHWTvnDO2qd7h6vkNr0tJenEpnoo2wmr0ICgW9BQaGB9Qh0QnUGuA6DUiETnpDwe3nQm0RD8JWjZXNyzdj0ViY7XyQ6LwKeqK6CI37mqb0BBIG7yOnwooEAcvgBp6esY8nWytxHH2I1KpUPFIPZm8ziXWJONziwr5iwwq6Q7o5MuMoD0juYZLmAZ5bP5OIvgCEN2WUp5cjd8rxnxZGfUkyPln1vPPMBl7Tx+BW+GBT22gL3IwpbgwtxhYqTLnk5UQzImMSB4VEzGboCn+N0MUHeMZLhZ87iPOlcexsuMy9W+4VNUiyyvhh3FYydF/yjvR9xn1+NwOhLhqnK6jua0YnhUXeECtr4op2FbVZxWSmxOKq68PuHci+yRcwqqoI7QzFqrfidlxLqUy51EBS2WIevnMvWzQb8EucS1fjPnRXKyVPjjzH9ikn2OIXT3L6KE65D/PJI6KWSFrbh1yf9AVLM97gyfNvUWOMI8jHRqdSDAq1Rrdyw+ofSPaS07pnBqGXx1DgjEHRXMvpgtM0uGHq2Rf4avlz9Pb9SOqFBvoKZoO32PeP+omAhV6byKJ2uGSHjGwfvr7v66H2N8U2sW3xEf4x8h9ItoN6sAuVWU+YWYkZJW/pYUDRgCCImcR2pZ0ti7dwe3oHTwbDFrkTqZeCDkc5OyNkSKUertuehM3ozSeFadyb5EPswAMYC/eTVDOB2Xtm89lDn/Fh90xCNSG0G7YQKlHSnXoTMokTlSGcE83LaRrIYlSGD8orKWSWZ3LlOiVlGWVDlN0ewUND8gVWr5MQFQUFhk+HwD6AyrRKNIowVIHezN8yn88jPieiuY9JMgUhPrB2sBZvIQRbZC7HJxyn31+srFraARPVEPzRXdhipagsJgL0YmKBpuMBjM2fs8WxHu/GZiQ5MGXFEaJimzmkceCac4gRiZ0cRIpqyim6lZNo9TtFk7/IT53YCMs7t/Bg+myaR1loiByFgGfoANg2mIwiOYXpRguH3nkA21gjgW0dBPuK9D9xjXHM2zkPT6oRt8aKxOOmJitS1KwFkvXp5Bbn4p2vY84hkWJ+73V7iZC7SZGDd3MkueWJROTHoig3UWlbAJFfMeCBdmsg7W0xGJrVtCScoSWwkmanqB28KwJMbieTVu3j204d/d0xTFWYOCz1oJVAvByanO3UypyEB/ah89PQkSJS+kb7VBDrK+67VQlLIQHSFW1YpcXoLBkYNWICzl9JVaH1p1GZ+7Aq+1CPP8DuSCChjFv3fInEW875UeeHEtoe8crgvoy9HFz3AD2toQhrIKrvdnTWLE6nFXAlFmpMvrwx43XGjQMGd/1tHc1uWEtUwgXqAr7C7LbRHFM99N33g3DcCj973PiN9qbOexbl0U4smmZq+isJCRGryp4OiEUxtwlz5sv8s/B+inOvgg1KePmfLzPW8DESGzzzzmMUDy+mdIyaFytuZNHIP3hjSilb+j/C6+A5Jp9+fEgLdkwLfBsCSVeTQcadTMeqjuPs2LP8aYK0E0+ilFSTa9UjBAocm3wMAL9+P4TiTJ5TGxicXoyfso+TmX/Sn3JiaHyLJmGBqpA+aRoDXs6/tg6+K/qQrNCjjJvgS0xdOAtOjUUV4ib7wlS6QrtYP9jH9GQ1l/V3IZHJMOw8QdbRc5RMexwBgVf10FYdTeDBiehuNHL/j7uxaoNYv3gUOil4JC7OTz7OryOP88bgHAoOWTFLhzHgN0DGlQxMWhNV6RVobf5IRuRwQVEjUvsCG43g7VSjyL6MzVfB2ZE1Q0HGERXppHgUSOyN3OT/AW19oZT5LuDjxz8CwNy8HKUzlHv9E5l+ZBluuQJXxAliGuJpGFHNJ2m7+MIYR2lwHipfFQq7G7VV3BMbXCKrzKS8g5yqnkJEr5w7v7+Tw1MO05UUTmzvPURHf8UXeXt4tjMQWdunJPevpDZ6Pd1BvSQDx4RuUutOYYho48yEU5i04n6eeSkCdc9+3pMnM7k2ioAAB1MPTaUqtYq2qDZmtkFOw7vcuzQC7Yd3MdPkx4+rLrIm9SgAF9tnUyddQJ76bl6N8OKDwU7S99xHRfgGskqzMPgaeP35l3ltyRo2f3KSrCOHODq1CUfInbylh688gah+O0OzfYBYqR9pNUsoLKijJuJPnN7lfB4Ok2e+xsuaL5Ff8sUpMxAiS8Yt1GAXoGJwFKE+/iSkJ5DxZwY/TPkBmVvGpw99ilVtpe7eJ0m6Y4DLsZdZvm05Vyc6M9sgvnMkh/M/xpNSh8z7Xgq2TaFR48Q7yszh+rsYndvJJq95HHPfT39tPzdtvIlLwy9RkV5B/rl8CPt/1x6KLt+L3GZEEJbSOHwhVRGi5tkMMXbM8YxSfgktpUD3GSatGYdC9KPv6ITH/KH80W+oNU7mekUCg4HxaAztjLbfyp2+P/FTeTiJv09CiG2jKPc0lVHPMc60kgD99xiBVuEEvxrKyBAElDPyuOTIQiUouM9zHZOSN3HUWY3zlB7JGG8uL7sNp30hf5UdNgd/Q7J1BU73aPZcv2foeZ70g4VaeHrabzjdQSQgru2CxEVpnozRyz6jUxpFg2MRqakQ7jpBp1xMDnMJUjrdMF3bQV78OjqF29GE2fANNFBvlZEikRCksxLXHUr62bFIZL0Edo3lwPQDdIV2ccuGW8jDjq9zkOPbluCb6+GTp35gMFOMc/yjB5DAs64+jvy+gsYpdi5nXR5ao5NrksktzsVpdWJr7sbXIL6EvKI8HP4DxPtLCdsyBcvYdk7mtGOjB4kgYfu87TgUDtJ6biFn4DGCvcI4YNAxIdhNo1nBD0Y1/0is47mMbsomXKSn3Au/5hqa/BlKIm/x/5l/3tnJJcseXFt+ojWuFY/UQ2i3GIyY6I5kSk4xlbf3cyzgaVIqI2mN6OE5uniuDwYV/pi8u0Ai4CsRfT5vs4o7vnuIS8Mvce7JdmRyMdnEd/Jw7I31pB6WUkkaj96yl/MSKy67m8NP72Ns6o38Of97jk08RltkGx9YIml0SjEnmDlxXSnNURfx04v77W0OfzbeuZacYaMYLBzE/yoT04JtC0jV2XlA8TXM38ulUb/St62PPrNYQfTR0zsY0BQy3urDqJ9uQugVeOPga3z99hpO4ib/78dCbnY4OAfMfeZZHnrkOVZXrKbouyIiR0cO/Y1t0EFEzQlurrmZX5f+Sm9QLxEakXFi0YH7+cc/QG4dBO8LNETtJiZ2BUlaDcv/WM6p9TWU1KRi8DOiNfoQ1z2bPt1RBC8teKKx6u3INfL/dA4ffv4wLR+eRT7tMZxXaUPji/9AgZO+6um0f3AAbYCC5REwy+ZHmK8RXCAIHuQaOf7T/enRXqO7/IsqFyChcAtZAx7CZjcg9ag4GLaGAZUvLoOLuJLtaPVNeAIfJTltJRtFvJtPB0R5iuCmQkKaCzmRJ2eqGt4JgtdaLOycs5PM0JXoKq9w/UUfDk87TWh4NOd7szGX2EnvFeOkf4E7Uo8Cj9SBTOJFSU4J/ubJjGofILUigMsZfqy/tB6AyqQKgrpTkCsteKUmElXXTHh7OO6IQLrS7uDGR8X2GYPiGdBc5GT6KCROsXKV1lCC+92kzR1PfXo4fQGixnmBCsaqqvlDswujQ8fB2y/RLIj6toEy+DQECi9HsPPt25HdaOSre98Y6rvnAuDVQDhdHciBqkTKpriG5huIhQA5TW8yukDH+HW34NbKOPVQEx3SLTwFcP4eSLqbHms2F37Xk3adgWMTt9OvFe/va/AlvMqJodmAf7w/74W+Rxywf8Z+Lvq2skYk9MEjeNBnKzgfunXIR/A3jaU0+wyl2aV8lLOL8H3L6AgQAb9QGeT51lNwOQfO5GJ5xERp+kWaQr5EY9bw+I5U2iPa6Zq/A609hfHlrzCtYxpHJx/F7eUmpONJrk98grV5GWxNqGN85s2oLv3CiZZ4xt20mtij18PM04RZbsAlOHBeMvPEB0+wdeFOpP8BqPMa0OPTdpm+yKwhGs+yrDJikkVtPoPNQEJ9Agu2LwDAYbom0/a/otmH4MFf709uUS4xzTG4H7cDyv/xR/8XmSAIGI1GdDodEsn/uw/xv8v+l8E+f39/jEbRRY+MjKSsrIysrCwGBgawWCz/vzfw3/Z/pgkCtAVsHBJ0/XFQFJw9ZwO7R8Yp61p2jhS1qF547XkuuuTYRg1DszKO4Jm5WH7uQ5AFAaLjJwG+M3kowIW/JIGYnntoDv6GsafHct3+64gaXcrWUTq6YxJJGLjAoLkJt8xNSnUKwtlkrLM1TDcYCd98E3qPg9yueGoj+zmeUsMWIyzRwVMdd3MqcJBp0n7ibHPo6qrlvXUC0kEd0RkNSCeAx2zBr7sGiy6UwbTb2DL8Pn7a9BD1O4PwHa+jJymYqjAxAPP7lIP4SSQ8cz4VyYkaIuYvQa7NJsS3hw7FF3w36jRpCljobceqEkg7FMSsS/fx+nOv88injwBw4LWdzNTAFbOd5O9vY8BvgO0LtlNoh0I73KoIYdjcdnqyJ2Bxuoloj0BtVfO1QQworE5s5e2PnmDAT4VhXDdLtyzls0c/4qleB30hLXineaHQigGfDlMHszfM5lE/8R1++dptLGwOQ5j2d4GCFomeAm89b264nx6HEkfMIR4uvJXXnn+N/Ko0ru/0EDd7I7asdi5zI/E7BjDZcygZXkJ2STaBUglVq5w0ZsxEFXiFwL5ANBbRUV/Y/ToZXhG4JZdIU4DU6UY2VcV2fqcLN5+6itFa3Yz1ScCT34A9cR6G0ydRtfbQMOIGbD4hdI9dQNwkKLuqBa0096HVN5EXsYtUSScPTj9AduoKqk99wJ1HlvP5A5/jH9HLxzE2zhm9+K46CEWkm0FfC0adEY9ETpbSw80+UDj+EheG1TE6fjs/Bm6nOew15u9+EHtxKNFTm6meWE11ajUpfVOI7oqmI7yDi2POMUUNK4JeY0HIl+yZtYfc9Pm0ZvbSGt7Kzde0ujleE0ZCQzQum4v3n3x/6HOtUYu8xo3NYEPe3Ebs5Sv4d1TgKrESMSuL8lgQJG4EAbYt76BPV0xa6wrapG2cs8HBjLsYMIwhSYC47odpDv4GlU2FukdCsnSQ+duyMSki0PjXsn/GfvwG/Bi9aTQvb8qm9o2vEBbtQ+9ais8pAe2ASOUpFaR0hXbhJ/clrcpOQKuZzmIZyo8bSZyXSFFeEV4SgcSset5c8zYp0oOMWjmKTw9nitz8EgnBLcW0HLPBP0ZhDIyjNm8Z6VGh3DD6OPf7QmyZBqkbZBN9ecv7LQKM48lqUJNzeiFfe/9G/C0XKBfq8F1XwaTqSRyfdJzzdlhQdDsvjPyMekcaVZphWHVKzFozDfENvPfEe9hUNh5ryER1agb6uTYqx4tr0UjnpzgaIwkVXJi0zxOdYqXD+giawDg0gRqME/t5LvZd8MAvRtg4KOVWAbyDxSrK3MI4FPalyO8Op76+nqysLE6/c5r++n5yl13H5NJK9LqTdPvsw+O5DmX0h1TIWkk7tfbaGPDeRknybKr732PFxhV0hXYRZLoBVe1u9gxomFsAUo+MZ8fHsummH3DIe3G2DkcV/086PGpC60FwaVjzL3M2NCiEBakb0SkCOLN9PMEX0rDfbSd9aTq6bJEaZH4H/Bo0h09efYbFvQU4Lbfh11bNoF8Y6xd/SFhHGOaGQbjG4klAYxG+PQ3UJveh9y2j19xHgkrGpeueIfX0enIOvk9q6C+syLuLP5pj6ZbYGXDZUNqUOOVOxpwdw6ZXZrJ1wVbmV6XSGuMgznU/7YEbibhKDQVwo6+JGI2eqpKlGKTg8RK90WNXKxLfanmdwL1B2F2VOGReqImjZriEafHreCxrFe0N4Tjqo5Fk2ukKreVi4oND145si0QrdSGRyTAFxmLwFzMNj/SlEBBSyi9vrCHK8S4RuxXEF+VxeOphIhvDyLT48kZcA4rFW5gQv5pHPr+dlGGX4cHvGaeGR8VzNLUOWHPYQnzJNhpzpvOp04l8wW6+DW7j1ZPZ9NZEEmMIIe1yDE1p2Zi1Fmo8ahZEvskXlsdZbPiIrEQX7qmP4K0z4Ym5CWn0AlD4Edym5ZafH6doTic/3CECpCo3nDR6Ia9IwjyiD7evFY/Mi+0rtDTGNtNohm/swbSfmIg708LNB25GH6Bnz/V78JWKAXadyojeY0TW6eCOH+/gwqgLyNwyzBoLzZ44Eu/6lAvGCIxRCTwWu41Ks6g79kcEnO9vIWpuHF/0BRH67j1IPG4+e/gzbhzexIP+AhWuFh7Ja0QurWez0YeuBPE9vz0hB4XMyjHl62Q2gdUYwM1zp/Fd8tvc80kGIAa552pfAyCsMxyn3MlE73soa72e1ae/5kDB+7gsvjwZ8xDPL1tI1uUsFn+VwjKWca77GHeP8cajzkXbX0yYIYyu0H3ET3uSVZ07qVFMZUP0KvJSoKN+H02BWzGrq7D+cSubWiJpmbufqC1zkC8UkNiVyNxiMOZ9ey0RDjUBvm7OjDuNTJVER+xBSuN/IkkG68Ngh+Uin9gHOLj4M2Zrn8cn8zXM8VPJO1dHWOeNADQmVqBKz+Xi3JeIKd1JQkUhkMXo4N/5PbMG1CpOTzhH7RQxEO3XWk9G0ygGcquoblZRGrH1b/RWVwK+p9FjYv3kW9llhrc6HkJlVeFQOPDIPLiBRocX+07dQUDHFQy3FrBdvXro9y7ZIAZHM32Zz3FcHcic0JmsXn0/hbmF/LhsB+dt8J4mgZ6jz5HbLoOYv/sJBj8DVd716CJ0eNeWoDZd00Q7m38W1+bpPHFuDOaJg/ga/ACBw1MPs0Gjp7V8Ca1B67D7awgofYPXVr/I0kd+pSG+gZjmGM7GmfB5/W5k4QEcn3wG/z4fAvWBuLxc6EzZhMtULPzyAWpy+jFELxy678/RSxBw843jTU7M3c3ZyVeYYDlJu286Br8t1MfBFYeL2v5y3JZKrIZkAtqszCqaxd7Ze//l6Tx4PJD6ZiqP8Ainxp3inqReluhA98HdOORK9P+Qctj7WpLUBbv47xGDBqNFimXiQkqtSkboZlHiXofMo+acupyecXtZLIshwTSJiZmvENoOr40SaaX8hDgGJh1geN9kLgUeGLr2gAesbi1h3oPE3fc7py6M4v6vHmXtSDvyuyPpD1nAnFHBPL4tEa+pR1FEL6HN71HKo8Xq17ZoF1WjlxIXG4u1ppVhZzYyDLA6RjJt5kVev/44q2VrGD5jPH82f0ZXUItYGW2BPxpgfn00U49MxTlKTtXCR5iZ+B1d2h7OSMES9S5v/TaStCsyuhf8SnJtK6UJYhXwKCVctHjzQdvztJniuJL8PDL/Uvqcc/jTDH82wKvDh3N9+DIaxsgZnpLEZec9gIQ1J35DJqjYcQ94+R/E6ZZh3l/IjAtmrjxiA41YjRzWr8Tj8tA2bDo9ln62zBhNsAManRAnh0CJhuDgaEryA2kL2g1Asn85aX5QFxiMRR+Kq6sXi+NDauJFOvfvDGA1xAz1/8ydIxGEYWxZugWtBNZmbiEvbC99nYEsNIXQ7Hdt5AiXd9Pg8QH9RZ6eNptPtd+yUzhCe+BGDje9Q4HkEPgMI0Jxhqz4n9nFF+Ts0DKy+26+vedbkIAgEcTiNpmMqjQTzTHNKFxjSG99l1MBD9JtiUVQgD7RTqtQMgQ6ADxW703+uXyicgSiLw1n0MfA2bFnSex4jqS211kRNo7q9bdzblEHiDFiTFoTP9z2AwZfA18V5nJl13h+v+cKXRFdQ9ddGwIpMhnNUgtz3vyMbaX3knigkJDYQWYkfM+e2vsokHqxSmWhpzeYunzYmitWGleZYSfHmSk14XYrUQxLpKZJgcdLAVfB2lO9GpbUxuEZcBLcdBFDsLhvjHt6DZ8Ar65bw2gVdLj07Fu8hc6wTuIbxMqshydWsi3Wi5sarchiwvli5Ftsd7h4uAe8raksdz3A2scf5a2eYDpL1uCl/xaNRYNSayWgNxhvdxem2j7adsfinaYlp3c9fbqjlMc8gq85D5kxlbTuP7Arfdk1RcNPY19nV6AYzXWlNNKTbkflp2LHkg5agjcN9ZlKAiMW7OeZlmNMDXuEsxlmglxL8O5MIC3oFMmBxXz001RMg34E2A9zc2Es39xfwIHJRzk48RjXq6RYX/yYGvuznFGdQ5A68TZ5M2f3HOA8p2ZV8Fb2VsaVPMTMPycSGWzgpyjRfy+JvwOp9CR1iXX4WEMx+Fl4xG7i12MfIe14kEmTweFU8uMDL6AZ2YF/uQ1lz1QcXp3/op8u4AmRUjGsgtB2OX0qI3+awctjwWfCaQJikxh1YQGS8oucmLSNYN9yauOg1SlWYHtUAiu2PEVL0GEOTzvMD0b4wQiTvB8iYWU6/vJOvp4iJpBUpFXw8GcPc2HkBZ4Y8wQAHyV/REpAKtX6qqE+lSBhWG0wdnkXZ0eruDDpPOd8W+m/PB9rfwY7Rpyj75SdsBOlGEeGkFiXSGNcoziG71zLlKC7AZDW1RB7uZL25Ino/dvo9NtKZLEElVEzFBgVrlbxbDaCv1RM8LIIgL+O31aU4JaKDqobeCEAvpEN8FqzA8EGNWNuBUBn1LOx9CV2df9OSk0yzUEmBInYv2ZBz7kROiSClCfVrzFv511IdSEcSvyYvbnPENV7O0cab2VS3CbMRzcRUy7QnxmLlmQk/xKuS7uSRlTJHgZaUgCYdHQS2ZezCXxqLQXBFnYtLubJjhCcLVMYe2ksbbFdFCfdy2NuHXd2PsRzcwPZqJ/E1F9vIbd3gLV3rSWl9z5GO0Zzm//31O9Qoc/2ZvSTXzHZvx/F2UVIqxP4fuohnplRRO70MvYduJHc87GUZZbRFdrF0Ru2skcTyDDlMEru/wYfVvGvmJEjGb6yFLLfR4Hpro10yx9gd6hIo5/c/gJHpnzMqYJTPO/9PMqxcXyo+HCIzUGV1Mzd7kB2AN6X7XRPGSDuchAzD8ykMLeQvKI8cu7fh9bwLGfa3+f2zx/g6YgODo87xfL917Ft/jYeLOhmhxlaOhSMPTyNQ5GF4AuHIiFD3kaglweNEkqiJmMJC2ZeZD/HnvmQdQpf9JGPcWBASXduEe5uKfO2j+VkgXsoeQaJQNboeFpMSpQHVnNz1c0U5hUS1BeE2qrGcnXNq9pehfXoAN4GsWdcaivrv1pAdkso7pVuDsyXYVfKGVV/lMSFqazWwWcn3mfehLm4oo9waKxYCdwSLcbR7hwM58vpF3mGKD4L+oxjw44Nza1QHxfv9cMfJoiXbcJH6kOPuwc/rR/6q9U7O8Il/OBU4Xb7sfrtpzmOm3z+c8sHdrqcTPjgFe596j7GPTlu6DtTl4mOi230h6bi31XFqAujWLdqHV+EiBqucp0KtxwSL/xMjt6H9//RzeHZ/VwXG4a5u4rKr49jm1DEnplPDl3TX++Pl2URXj3DeSfwO6a/PX3ou9jGWIrWFjFi5Qgi8yPRFAzHI7smzi1z2pAKDqz9ViwlNWhHSlmihR+S9rFtMIeSkhQCZBJ04Tqu23gdj37+KBKPhPiGeGzqa9IkDrUvMpcdjSMOP1UnXoKJMR89jtAkoI9Ix+wbjlNqRO+CnWbI6QvBuP16wu5w0hszgpaYDvp8f+QmSTh5qg4+i+0hihZi5HcjvWLD1+CL3KliSdpbXNBnEHwxjdTqFDqKOoisiKQtqo3w/qV4uXWEZFRwfNIxktsn4j92GN88eJbOgGvSRH/RPHpUGtw9fShtTlKqU6iKECu2/hX8Ea4C0gJiYt9tu68jql0G90Ne/W9cZAmdAb8zux1+CoW1r/xEbM0I5MK1qsdON0xsgX63Hu11e0lMmPa38VJuh9+MED71NN2jTuOueYUb/riBwrxCmmObsQjQripFJcmjsOAMSnkCbo/AUTs8b3qJ1+eMEtvocBDWcBZbpRth5L+ch9KvsPdRJS8sGobT6ETiLUEwCygUd+BQnia8PZzUqlT6lvRhiJdwMeYifqYxTOtew8bpCzhp9uFFwyAeQSCjNIIQXzGmeb03fJ/wPWs+eB1K3TgMVZTGiYw2qZWpLNi+gK54F6pHX6No0EpYtTdxJRM4PvE4UtyMlMuJltRzKkfP+/16pGov6uPraA9poMeRTqzCD3pOciT6EpvOPYNuWDBb88/SFzhApe0YUYUHqavSwd1PY0sbTo0mC0EqG3q/cE2/T/arjIKSAlwyFzWv1JAyJ+U/nbv/Gdj3ryZxmnn0ExENNmusCC73//cP/i8wj8czFPP776Dz/C+DfWVlZWRmZjJx4kQOHDhAVlYWS5cu5dFHH+Xw4cMcOHCAadOm/c8v9G/7tyEuFoLkGj2j3gOSGpi5byZhId70J5aR0JNAU2wTHz72EWqrmpCu5+nx28hKk4KyoFeYgoEDwP2+8EkITFSV8lV2Katax7Lquwj2zMofohgQHHIuBy/FEtaD1b6S0tEbAEipFoOE0gBfgunmunv/YJ3vLeT/moYsz0xNSg1LO6FofhEbKuZijW6npWkciR8fICB0CuYu8RCRczIHW2sv6rR4bJoAkIDLI+fzC18yPv179N1zUARrORR2PSBWrFg0FkxaM83uqQjDI0Am40r0kzQov2DmvpnklOQw+PynvPHyDyysS8JaOYGawIu4ZW5Oz93F464ApoXWElyZwmaTglaHAoVDdFD+yt5xyv1ZfHkZ4OG+yTreWv0WAPlesCsSsH1G07DbEbymM/7mGTzb9R3ozHwXCt+OrWKtOx9NkIbi74vpsYvZSl8MiNnH88KbcSosDKs3ceuPt1KYV8iVjCvc1w0/DSh4aHQZdUY/dCErOS37EY/Uw4wti3ECc5fvxCfOyJ42B1E1l1H3jKYxrpGq5BqUw+p5LmEdivi7+Ef9z/z+1HtD4+SVzG8IV3zNFVkCs4sn8J0+EFbLOWcRadP6dacIT2vksul19gTDTSMhYP9GbB0NtE91cin+dkItk4Fr2YvNWXOpzMvi90ozw1PewHtmL1b1bbQbu6mWlGL2NnPHm0/wvJebU/N2Me3nFfTfZuerVW9fa1c/vN4P1XEW+sItfGIppD5aHGPtkSayi/3Za1CyRAtNTogoimbEqUl88vAnVIy+yNlEaHX/SFp4P02hHahifTh9SxkV1jPECKID5czcgPtlKwuqW7H2/wuvIpBWkUbUbuiY2YG8r5Pg5iJ6o4YT1HqJgr2lVIXHkNiQiFVvpT1gIx6pA3/TWD41SPhkEKYPpqNyaTGer2Dczt0MeuagNnbhdsjZ6concyt0Rw0SoPMlqySN1597ndEXRFQnQeFiYzjc0TaA5sr+v7VrpSOSfOcvYDbRUR9PSOYj1A2/nsY4scIk1LYXkzCFyRposwpoAjXYtZqh3xdf9wz3fS5ujOHVxwhoL8dR6KCHNfw29zh37pxIW5YJz6i/vA8JVm8lFwtOURnYQa1FzwxFKQdnbcQ1RUxSMXqgxKrhZIQdzZJdmH2XYS89gX9/ElrXZFqCRVBknCCnqiuYAacLqctOWN1phEwPpTGN/NGQw09JPyA/MZZqxRGc02/BYXZw3nFNK+aDc/GE1CVim2/FJ/oa93lmWTLyXQFwVftYHajG3G2m5vlfiSMEIeQn8s/lM/hEPhKJHw6VD20pkzHJtnM6/3cmNcSSXGujp60Ds7dZpKNxOjF5W9gxdychnaHoTP4YnQM45GIAvSrqeeojweB089p3Wwj1ldByuoWuy11IXdkEqtuZl/opx1xr+L5LRZZBi+ARGHnvSII6DNzzYzA3+huptQfyYT8YPF5Yei3EnNtCe2IuAKu+X0XHpWpYfo3K06kUkeq6ZD3zdsxjwN4Pq0DmsCJz2bDqQvGWOUjWdlD/yrtcsk9nzKnxrN61mrWr1g6B/CO0I+j54BU6XA8TvK2PTGcmZVll+IS9wxsRTbQMjuCbDU+i8G5Baxbw7/ODf2EV3+UIoiGrnCaLm+bpUi5KvyS5/UUeGyMKQkbEd/Dqc2/yRNBBJCe7UdgVOJRiAPTr+75mmut9lgkCguDBJRHBvjeFI7yohXdl8FGbh4GZSbw59VEcCgc3/3QrMrcXU178jMezyhjr50WAug0/ZQ9FwgQWd5wgWwnzeyMwnhlOc+xJzq+4gMb3GGdj+/my25fdChtbTyeSW5SF1yMtzLl9LZbumbyjXovc5YtE8iZ2uwp1cyT4O2iOOMa4oEtImzdB+HQQBFanXOTslF4GQsXAog+R+Mva+NVrgF9ffI19UwYpfvMs2YePM3D9CK7Ko5AcEUba6hPoRz9Ax28XhjIY/wq2KW1KvFxe3CZxE9wTjNqqZtBnkL7APiJ1Tt4Y1cLtzQLFllSS5FCgCeVVfRc3dUCu5Xpsr03Gk9BORVoDbql4eHS9cy/rfcyM/kXFqg4NbpmJmWoTJmUNNkUbmzszUQs6gkZomPjR/TjUUqKfH0FixzMM+P6IPkCPIBWGDi7r7j1Nv/YMDyq30meLpbF/DL3Z1Qw4dYSqYsmv3sfsP65RlN6baGRt6QHi4yHoxCMkl+fx2vOvcWWfHvO5uQQu0qCUmRE8GjrsD4PgwK/fj36HjLLcQs7G1BB4+3rG56zk5+GfDF13iwkkgp39zgO8dOcJ5tRKEdyiFmW7C+a0wRy/B8hWrGekP9Q5zaImgiDQGeWhK6QEfYCeBRn30Hr1zGT1Ed9n7KyTXLDBO/3wisr9N2rBu78Tg5Kt+NG6pQD5KhuR0khcXi5AwlfOaj7rMPLphcn07ZlIUoGN6SeevUorWcviDtDYwpmau4R6iYRQ3346gkRaMh8pjE36iJZuPfty2vggpg3BI2YwKxwKTtnglA0+lGtIr55ORM1xhEoLYwrHUJNcM6Qp8VdVlbJHDDB3JI2nPuYio+eeYEZpKhu7WhFkMr584Cse+vQBbtx8I188+AWttgF8w15FKijJ81LSlHOZe5wGapPbSE6Sk9l/g1hhYdPRFd7LupXrCO8IJ8bPwvvJ/yTRmU3HIn8MnnnoLn5CZcpJGoYFYVJVgsRDmtBOZ7aFlyuH0WDyx88iVkAU2sSKm87yF8g/PIKWjEo05mRGN40eAvsyyjLIvdiKqcNIdEs0e6/bS0lOCd16MAswPlrPWY8aPdfAIBApijzAJ49+Qq78RvSGJTRqfsfPGspErZUtSR/yz+ZMvrg6Bn65spo71Nc0aDOaP6E7Wgx8/zUHhr5TgMfnF14wTuOw/keWugXUVjWS7jmkTMoi6CrNeYO3ktKJ+7jefRPnAmfQ53MUgM7QEvRhUUh13igigxkMjMOnr5HcolFYZ1xE5uVBopAhlcu4PKIGk7zhb/d/YeZlPPkN/Om7gbt1jzEt4QccrTnM1MLDfpfJb7bSHBVEYu84ksozueJwgRxCGiDQnsLYhodAEOgNf4mjVojVXAsSSZDwftZYLgbO5lTnZ0zb4eLiyItiAOpqQOqjtJs50RzInvAVNOcWYtaI423UhVGMuBSIa50Ls38kZv9IJpyYQGZZJsMe/Jw7f3wThasS31W5hA8soi3oJwAe7oEneuHGgC9xjvbH/+znTDmg4EpiPd62FFb3VbNGBm8k5nCy6yNC+pOxSusAUZ88K+QovZYEvnjqVsb7+jLJ8hyHpx7m7NizdPr/SUr7y3RJG1nfncA+7x9IvpCBT+AwFOM8cPEhiL+N/QPfsu7FqQybeQkBhgAWmUsmJqh4ZDhlLn5d8gVumYXUtsXE+JZxsm4K8k1VePK20bPIxqlE0WeL6r2dpaHlNHuayToxkU6th63L2ujy3wHAQOQb9Me9TV/rKiKvtxKtvI6V3zsoyi1iXkYrXj/eTseUU1RF63GNKEOQgtqiFql7ZR7SFBArM/K+cQo3608ywqomrnQnelUjY6O3MjZ6Kz2eTh6Zdp7P8s7TathJatFBfEPPc062im8aczjXcSdm40YUmcm0NyVfXUMEDkfCiYAqXhr2OmvStnC5fDmCVIbSETE0TtrrI7A2BOPJFyjLEjP93DI3qfdvZmJiF++tvpd4txcXHvmI1Zpu8qViEmlVWg0tg5vZZIRKm5K8onHEl4hnoT/u/o4IvIhTLODe+CJ8bilnsWQhwy/lUJxWyOdZIJcU4pAoGXFPJac6rgdE/yHg6p66XAtHbeK7awm+ltQ1QglTNfBQDzS5Jch9NfTGT7ra7l5mZz7LlPCT3Lv2HrwtOnqn2TGbAunzvSD2i1Rgmb+b22L7mVfrIcAwgT6fw0g9YvA5fXQ5ywOrqHNCebidB9/9FK2vmU2tXozUuFBJIM55ip/vOszjHWEEV8yl19WH3KNjRuI3KGUzOOZOw5h9CKKD0EY/wFn/J6iJ+QFvkzcpVSn0N/ejiPSmNKuUZb8tQ6g8x8HR8JvUDNP281jog0gDr2dzejldwcUIHvhhEPZ0ayisjuCG8S7Cmtx4GeP/tp68X7AKpSKM3e/PH/qsJrkGa1wzS1M7CenbxVcG+PXZcVy3cSZtq1px68y8EAAtzlp2dwaRdiUHW4GNI5PP45T0opP+iMn/R+p1/exqPU1gQCeBw4N47cXXhu7RH9CP3fdqhUxXB8HNRXTHjqJPd4SqqOepulsEmpYKENB6mQSjQG24WKG5OgDqLicyeCwJ1/JBxleKvsfOkRKCZXBXF3RfeolseQ4yfT0Zx/YSfv0IDlrG8nvFM/Rn1XPhwTqyBu4h7+RFRnhu5ORtFawJMfGcPxg9v/DRxRkMxITR09RPbkMu/eEyGgeyONRwG5ppvhQ1DCdarURbU4nbK3DouUK7QomvUmMdsHLnd3cS3SpWA73SL/Ab8E0I2D2Q3JpKwn4H629fz9jTYxl1YRTCvHYspxLxMlmRONxDer9pKisfT1nF5uIRNA4q6RTO8EB/Py/oteh0Hk6NO0VrVCv73QLV8kGa4o288fw3gAjM9MicmGQyorw0XAptJ38QEuoieUqw8W6WOP8sgppfhSBqoo4yxQ4aCXhLILXrSZwyg7g7SCU4lS5M2mtsHZejm4j3FqtclcNE9pSe4B6MWiOD6dWMjzbjMks5tX08+PURNvEiN4QNsinEwEBIN/PLc7jliJHBmhi60y18de9XpIbPJ7duM6cCbqTdruHhWhlqi4bMq0H1ePUI9nsbuNcdzePa5bxd+wvt0g7u13mxbeEJ2sPKGK2EJAVIBSfTjk7m4v4xGFOdBDTG0RYp7llmbzPRJ7S4H3ZTvLaYwe1V+AbEcOB6I1XxP5BVmoW/3I7dY+dU7ovinuN4g8kv/wNL/hW8IlwIAhjs1+Q0NJJbaJ7voDoxlpK+YgZ8lUO+GcCgzyBthXlsTC1m5MWR+IaHcfOWHEqySzhz2xnWrFmDwcfAE2s+5LZNek6cDmSKXE7+fxT3+g+WD4xHxpxJ16O8Q8Xau9aSFpzGyTdPcu7jc7hicmkPb/+X5IGr87CqB5nVhsXbgY8eYppjMIWICVY+0T7ELMpjm2zn3+61YNsCQrpDsS7RkHdvHj9ZfhIXeCCvMI8d63eQc2sOmcszOW3IxC1ue0gkUDt6BSoVROVD/Pcvsv/s82y9WnkXIJnA/F3xeCtF3zs1KJV3pr/Dc3ue47afbuNy1jUArTV9xtD/nxp3E6k+pzncPJmzw8fQ5xyBBxdHEuOxWsVk/IWHfmL4mTMoZ8qw+EWyc+Q9AGxpXUGeIZo98qPAOVH+IiaULbM+QOWI4sOzP9LgVcuhaR9QO2GA9NfSmbV1FudfOk9i5zP4WLPwL3iMCz3HxMQBqZTE7jfpDDrEX8kzi9ctIqj3O7rm3YV63gxqjxzkcvZl+unHPfg6xT8mMGPGTeTsewdDkDen0iBLAVsj4NOJJVR2JjDyXDXDTh6lz19KZ4D43CV20ErFlxlVHYSKFKpTq7ELcMIGaA0w9izhwZMZ/8d4+gL7qEiv4E8znGtZTo/PAZxeepYZPKRfzqEusY7m2GaGycEr6gvyZbfx1s3nuLHNQcqFaSR2LEJYGQ2RIiOh0etBpr+zi83ORYzfmgKeIHbO24kgFTjrv5puy+0E+wTz0lMvIXfIya/cxrlICfL2ULYfm4y+Ro8X4ho3oD3LOdUqDpq9GGuT8m7jKDqTk8gpvIBJM4aS4SWctcHD3dCdPgpL+AyyS77h+Y3P89OtP1GTXEPDrINkpUUyoyobn45Ads7oZtuk9bi8XER5wS9j3qSObv7ZpeCMxUNSVR9RTaGsGlHBSOMXsLAVTPVUn0/E9VstrnFx7J29F7nbhyZHEfaASlwK8awhkUkRZOL+768Ss42nHZxG2qY0uBnGnR6H1izK5+zs38kXF77ggVEP/G3uwVXNPo8H/84rGEJSEIRrwDiARybSnLdFttEXkcAabw3/tv9e+y+DfdnZ2YwaNYqFCxeydKmoMfT8888jl8s5ffo0ixcv5oUX/i3C+G/7r9tfgWgQy7NfCICX14yjLc2C0An5527jnafewaw1Y9aaMdpexmdQTlfveO5et4iUYU1MuWcDhy2wrS+CiTIFyT6NuNQyuiMj0HgWcCl7B2fHrgFBwhlLAg21w9gauwipfiLtAb/SmOJNmF1GsXA/T9svURi+gduCJBxcOcBlv+MMV8KNWtANFPJoaQqqbfdg/miAvlQ/Wn3b8O8Ct9SNzCPDVNmKkBRETf7NhNccJ7hyO7+o8rll9momhx9gMOJp7nz0TgZ9Btl2w05WrVtF34jLfDO1BbTDuK+qmkU/qTk4PQdBIuBt8Wbs2RlM89pBnVXN1Ixl/BmxBo/Mw/nRF6gMgH7jVPhmCi6Viu8efGqoP+/3c3G/H5ht9QzyAYF9gVis1wJDfxURO6VBbA5az7BhoEsCa9B0DF4v8HIfONpXEmwUs59OvHECQS3AEnACKV6w4c6j3N+lxG29lci2SKpSr2VPnrUqOJVzAj9nGuO897A7ZSsAx6ccJqsqjawOJ4ZWNSPV3hyYo6cp5E8C+wJJOVHATp9+fimzIzkwAWF4PSReGzMfmfSk2eeji/Wn87hAiEeL6yo9wRgVbA6DXschjgjjsSpMmN1+BD96E9u3CVgVH+I7oGb84TCqJlUjxF7LWrEqmrgS/gaHg6DKCd9ZXOjj7FyMFjPmva9ugJ0egWMTjzHsX6oBQHSPUlrf4EnZO9jcdoJUboK7gzHqjJwduZazIyHAHcL2wnGcPT2OQ2MF9l63F4vGgtMDBQ3evJ92C8djN/K6HjzXUkP5xSTwmwnaYw7wzI3n+XD/elR+Khb2LuSC7QJtUW30hsgIWjWWgMQAPKOicZ4+hbehbah9yTXJTDiZjaXXQk5xNiadeBASEFBIILX/DC5LOJUJlZiCmunzLUSukVM0oogKjQvNsga8FPFY7V2ozF6kVaTx/crvkTvlhFq0PPPOFCTBAfh1VuJQ+dCUdT0pJZtJUppxSTXoll+gxBCBLkJHX9QI/B33ENAhxa3KQO6jRl4SjyTIgn3Qjtxqx+ht4lRGAVF9tyBTvSq+g4E2ZG4HeHuD2UzHhUzCJ17kUkAM8T0OUitTMQTaia5pJrKxgB9uOs1NG2bilwjGJDNGRB03L6A/7EueMYLTCI9721j+YxSlmTnYoj/k9j1ukjPreSe/hI5nLzC3byK5e0WQXGiAZE8yrcOr+agplYgLmZDp4OJXF9n/xH687vJmUxj0yyejq5Ex4eR4rHorhV//nbfcduQYIFIlTHl1CqZOE7sfP4TC3w+T1kRXaBfIpMg79pF4oYim9AKOF3TRFtjE/JKJlB2YjH2FjR9v/xGA64o3cnT+LMbpjIz/eTn6y6mczPiNWXtmUTyimK6wLt7Ug8YWybDqVrw8Hq5s8ebsh2eRTUvmhKuXySdfYvzwLA7OWMOh6Uf4wPcDALRyXwr0jzE69B12GDI5v7+AYF0Mmns0NI9Zgl/1KdasWYPTy4nmuuC/P6dG9PAFdx2Z5bNwhNkQ3B7C6k6hGeyiJW06sZpKdHILCVW/U5B7PSfD3qUw9wJmbzN2pchdf2/CafJDYFGHk+jKKhJLZlKWVUZl1DPMSoQL3dn4ndYgjRkktrkct2k4LBKrCcarYYM8kQ/Hf0qsMZgg/gpySNhc9iLZWQKPt/4C1OARPOQfbmay6VHeffrda/NbcGPvtzBq13swspqGueLnT7fJ0V14iezI+fgHD2Ob130AVGVcIf1yNn42NavCxfncP3saMyd8RpMkFY1E1F/pqExh/vHRyJfVUpNew59BAwx0+zP4zU3cPPsQP047zcFpe3hU8h1R2sMEtCWBGmqTBlFJ8ylsGsPd391NdUEPu0bdQY/Wzkd+OUjO3wNh05mdUIM5pJojl25myuEp1Gd18XBWGyt0sL9E5I/3io+hM2EcPs5x6KTQHg8HGkP5vvQXHp7m4POHPuc/2owDMxhZOJKWt2RD/eRt8iZ34VEiwwXymqHXpGHxxaU8tymH2KfTMQuz2WwC1+AknOnvMhggcGrs6SE/wKvfD3O/H05FOMqjo8m5lEbXzP2knqrn6OSjPOcl7i8buJ6BsC6cXlqc3XoCO/v5/MHPcSqcRLbGkO1VII45/go0SNnvt4Sx8YdojIab2pxYWvWMPdg39Dxh779JmE5GzsMfQGAgF8f00RxaikfqQdrnxKoNws//APNOl1N6YgILti2gL7CPo5OPEtcUR1lmGUYfo0iFpxHg7+ynCBIP3bIovu+FAbcXk9ZFMs51D9/c+w27LXC9/ySyBt/D8c09WPPA0FKB32AfUZXelKd70Zw2nICMcBpP20g9vRmXXMXZ+f/gsHIRjSfn8tzPt+BZWkewlxbfvpGU5JRgCOrFtzcIfYiJw4u3MD7mKW5871YG/Hq4NDlqKJjztaqXjJQqnBERFOYWMuhzrfEKRQ+R42OotHrTZJwyBIJHesGGMDjgruOZbi1nHSbmhSl5/4mvMGuuid3L+s/y6kM3s6nwOdwWF7P2zRJB4aA+UitTWbj1BipHVjI47joqY2ciSGWkFfaR0LGcSS98z92xzdzjvw5B4samshHZHklEWwQDUQe5va+djgEfBscuYHPIH0T3riShqYMH++0kjfiFIy8Ec8E0AQkSTDoTNboa8mUwSQ2N7gCcI+JRti9A2/4znYFOBrwvsjrAw5P+YFPp2HX5Lux1i1D3VhJbX0NUdixLO5tICkhim/dZ2m5t50PrQhoCzZzXfjD0zKFdoUQ1mrAbHUw/OB2JIKEjvIMyRStvCW7mLP+dc+ZsUsrSufXArRycfpCOiA5a4yHMC742wEFTP0KrCm9HBuWZ5RhlUOLwsFrXwE216bwtNJFw7gmmmyZycNJxFnrDtPjdxHXn0HxwImd9wsnqu5njE4/TEtPCqSgoDXiNV82zmLd9Hso4D1dmPMrWZRLYD0QvgmFPkKbIwNFnQaeIRG2oxc/tx4D/AABmVTVOTw6qID96YvLw6WvkSo7Ahe5wjl14lth2b0YM9DKEsAFxXjBJGslOSSddPhYEoQ1HdyW9qkBOXFqHKTcXhwCzRtVwwVZDuSmdDdM+YEHACzDw1zItkHx+A1p9CzFdGYyxZ+L/lJLWrlGkBV1A5+nnQOmDlB2PxJ3cQWBfIGqrmqyIU6gkADP5+cBwXEfGoX8M9ubuAEEMeFwceRFHQiSyq4G6kIZzxJSPo9+vH4/Uw6CmErXDjq17kLxjTdismUNAkUOAYu/dyGRSoiP7OKndh1FnxKYSA9FXZGXw8TMMDN9J19j32ZIpet39HgioDGO+7D18knxwqkw4bBeH5t0jad9y08ifsMjOc/jXJEYXj0DhUNEWOQ73XQqYdgRUoRQf3Up8bw0N1W0Uzu2lzGs9ACqbilXrVmFMi8PkGsQtE0t4IvTLWZL7T8YG/8k9ZxdjCzD+66uiNegHXolRoYp28sxTm3AEP0D68Q00RIlhgtkKX36MNvBsdwTl0jUE++waYt14NsrKiVlnSMxqZJa3ifqEQmadWsmSA8+w7o51NMU1Ma4V3tOeRsaLjLwwEq1Gwh9LzhMZc+FaIyReZNSk0CKtZrmgwm8wgXRPNJP9V8GhGBTncnFYXX8LPCERyFDAFA3kKeECAgdzR2NSi9qG37y6hrFquLx1BtknC5CkX0sytWgsfNlr44tGLxRJlcjdAYTvd/LRlceQP/sVEmw0O6U0mpPY8GMAdp2CrOYjAPy+6HdST4+hJ7gHj0zLQOAgPcGD0G4mvuQAjYFtuMTmUWzqZ4ZqN5rwSgraH2ZY10g+GAgjN9DCpSIDSUVa+mf0k1GWgV1ppza5Fj8pTFDDnn2jkDaHIuQLV/XtJPR7n+c510kiOmDbnWvxdkcT4lVNbbYYoQ6UilrzFyxSJlzWokPCbT/eQWvESH5b9A4731xDnwpWeXRYWyO4yw7lfn1M0oBKIuf9IFE77JJETb0TrHY5N/4QhTF2Movu/51x0X+ya2ATEkHKn4v+ZIRqEUFVZXQEi/TpEe0RrNi0goYZDXgSZTTEN2BT2jAEaPkjXAz2zmwTkz0kUgntgb+ABLrdcEcXJNaGc+uGW7B6O/jtzg7qA0UQNE8J41Rg7vBQXubPV9KvmIMYxJ1wYgLqxhhGzL5I59aZjDicS9m0MrjKoqeWiBTkOw0afLvmojmjRtWp4nFPCxLrGl4dISZvbnd2czy3m5Q4HyTqv2fga41aIhWpAAhjxlKsGoXbSzlU2QLiXi8IENpwlmCPnP2iNC56N7TWRjH27FgcE/oIq63EFCCCap8FQ74K3v1Fi8dvAPewHtSmHgZ+3U/Xzb3MTviJl4f9xj+P7KXaEYXGLOBl96Pf3cIVB/xphoXehcxY7cMnpU8QcKmU+cfns/m2DmwuLWuL32STKxKmwN2V1USf+gXDiGsVR8cnHqcpPZm7UgKGgL59M/dh1VgpssM77y1DKmiIun8mPxia6QjvIKY5hoD+ALylHWzdkY9EqsBy4yy+CVwGwB9Jk8iqW8RuhYE/H/mEHMe9TD0yhelJXWxI3IoiaCRTBoP5pXsQp8kbb9e1SZ1Qn8C8nfM4vvwivlhJ9ShQmDws+3UW7onnmBu4h11meC/hPiSIFSQSu5v3T8xnbGYjlenvsrHoXWRuBxJBgtNoJqVaPO+fHnuaLjfotYPErbBiSZvCmB27SagI4NOHP+WxTx7lfEIL55f9yckHq5gYomOEz3ISh7/AvG45+A0S6VIzWJdA2vFx/JlRQ2dYJ8M0CtSOGP6pB409hOnHMsktzuVS/i/oBsy47PFknXsAwaiiLUMgsMuHJb8+gnR6P1XDGrGounjVF+72BavHylchjXgXeHBdN53fPS/iY/ChPqGe/HP5pF0JwWV3Me2tadQcbUPiceMf1M92bx2bFxzjy35f3nS7kblkCBIBlSMUDD78XhvMsN4KnB2DtJtM3P/F/ey+fjc55YtQ9bawXlhBwa4ssqYH0plzgPTGdK5kXLm6zEqQADmXs5F1eFGbW0tLeAuKqxVwvoO+fDgIY/wmsG3jBzxms/FfsTtx8Xx1Cyufe4giQxFpb6UNJce6FBp+uvFXZG4X83f/A0dBMxDDnpt+IA4Z6295mbjGOADyT3hjj+umvKSc5j8KkS6L/Nt9ikcUozO2kePjz9zP5jLv5XlD350ed5q333kbiUzco/8Vo5RIRKBjKCQjQLJ+JSBSTFbbn6It3UV2Fpg6TWx6ZhP7vffjCfKwY+4ODH72v7Wj22cPvT6H2d0zjAj/c9TP3Y9n2Gi4IFYCWxWtQ3/r8BlPx63jmTdH8Tfps9qAjfxwUaAobhkPbRqFe9ggPVFicqpUkGOwhXD3iFeptvZSZ9cz+sbRfCb7DIDjGdn4mfJZIB3HrD2z8DNJcUywEN4RzLCE2VxRi8wgRl8jPmY/ZAY99pIz9IYHIgvVgrOf0KZaTEXiPjoYnDjEduMlgQQ5fDizhAdPXaKydwMuoYIej0hnPFwJtT0T2VnbTZw7gpHHk3F7oqlOFWnMJcDLmnhmyqI553cr+kNKKtIqqEi/Bpj6GHzwMfhiiBF47fnXh4DgH8JEysxS5Rw213eTLL8FZbkUqlW8cP9dwJ1XX+ggESF67Pp+olqiUdj9ADExKag3CHO3GVWECObln8tn+qHz+A8PI2pkJZvVZpImJzHq435iSx/i27u/pVXVzpwuWHP8czjcQ26yhOPTXfT5iOfVCof47xbvOtTIcQdIaYlvwKay4ZF5eGLZSUI8gbzx2nzUPSHYc5ro14rU5wMeeLYXEr2TObR3DDP3T8X0kp6+oAG0PldLs011EDKRjfo4lCm1ZIUHE1V3G0qZCmmYlC1LtzBSeZP46GYT3v0DWLXBrJ6wmli/WH7Z9gvKXpFic+/De9EV6Yhsi2RY5TB6Gnpg1LVx/5d5PBDUeom4yzvoTBiHMPkaiA3glks5OEPUEg82xPFv+++3/zLYd+zYMdatW8ebb77J66+/zuLFi7nrrrt49tln/3e279/2f6j9xzLgnKuJAY9+9CGrOq/D0R9KcfAhHAoHUV7wUgCktuo4/MWdDCx3UlFQiDqyk2OdY9jrdZbLrRP4TFdMcFM+z4ac4NTdX+NjHsGo2h0UJi5B5lFzoMgbz9GRhEy3Y+8/S92US1wY6eGg5jiZ3fORXuUU9khhuPeHDAwkkRn1EM8EAMV3szIphZ9yriCPmMSvi3+hR3aOGwpiGW7X8uPBh9EmhmAYNKM09+HbU4tCaeBExhL8mgFO8W7vfuxKO065E6dMdIQCL2Xht/xlTum2U2dbj8m3B4fCwbn8c0yuSkfv8Gfw+EhCpKGkFsxCY/+JAe05BjzweC8USIYTm3Yjaq0Mb+t3mNVVSAQZgsSNBFBL3aysGI7s9EhckywMqxhGe0Q7y4PEwEJJ1EXcEitOQYrLpUSQOHECd/lCcVEjpsLfGGy7jqW/LaVuoA6OixvqvVe5sn2lAspUsWJQ7hID+7frINQZxumiTYQFqbms/MfQez486TiHJx0npCuEx798gIbpdg6NfxMAo87I1oWH6A5u5+c2X5ZcSUQT4SbDmkFXaBe9wb18bzECG1jv08C+CRfJTgpF980p7j9+Pxce+ZoutweHIKGu93byWmUUHRtJaO5CBjVu6kPfJ6BXRUZZHL991s38d0Tn37u/DclgL8Nz4P16H1zfL0M3xgzZ13a3TTduIqU6m6cz+xkzvYYP7Uso+H08gz4GSrNLifaCAqWChmOjmXhwHH0zLTy4/0G2zd9GcW4xoTLQChKOVt6Io9tMfOgS9gy7RsN52mVmU/8l/ryUiHvLDHofbeK+I09R1PURPy0YizvVBl3rITIWU0wqcrWc4V8MR5mi5Jcbf6Etsp74cTPxjQGpAppy5iNz2pDF7+O493f0BPfQFG/mkYjnmLVvBi3RLbTnuJmp8XCrD9ha9tLe50v1rVFsuFusStEN6lBb1WT7WtiSV8fmhn/wib2G5A0akmuT2f/YZ6zN6eXz3nBkFUko7A7KJj/EsNPfE3t5Jxdm/pNCiYcJwk6ekr9JqK+bT3gFve4YJtUV/CyjCVW08mNlKoq1s1AsGuDk2yfJOXSSgwsjsSobGZR+y0Dd4wQFBdCeOoW4km3Ir78Ox/Z96KP7KJm3kxbbLeRdMnHT5pv4+eaf6Y7UEhVaj9QlZ9yZcbTjRhmtxOVWY9VYmaaBvZHwwNloeg+NQbLYxumxp+kM60TleRav+lga6mNpz7tCs4DowV41971BNBk/IzbOjwMSFU0vvsKTiT8TGhVK3n15HFA08kA33BoezQ/L/+C7CaVowp4n+9ZsSn4ooS49jJ3TVxMmZLJCJWqJBCQFoNAqiPxuMV8dfo2W4EIKRxbyatC3OPq+R2lLR9e+kTFNUNAxhsKRF7kYV8Et0ZvgKo98bdgbLK9K5aGEPjzXXWRDah3u9pWMOTeGxrhGusK6+HYQxrt82PL0G5zseoCR940k+fpkCj/wpjFgIx0BvxHveISn/OGoRTzt7H9qP32NZn6zPce22udQj9okZvPqZMg1cgajMtDVi8G6/TP3c9ekCX9b182+EVzJcjGieATvPPUOr8/9CLfDTXjdKTwSKQOhwzjp/QaRPfEM1AuEBfQymCxhd7QYtCrKLeL7aVXIdAKLjiTgClTyx/wv8HJfeyfRDaATyohcqkLl9CdWex1haSLo2O8RRa4lEjGwISDg3ywj0iuS3DwJkRkvk5Lbx3Pff05mVSq4LXRGqTDJS4euH9MUg4/diXSaF9Wp3bRHtAPwvD8k9PrR8rsLxcQOhNHhhBnD6A7p5tdFf8CiPzhqjqdzzZO0zTfwYeQrvOJlJYVLLNDCd80TiPbsoWiGHb/EXcDPdDiUePuKGbe3RVhQhxvpwULD5UTer/wGU9Ag+MAus4BWamdx6Cl0K5TUN+bw+EePs2Xxn0jSSq62XMLC0o1U91dyXU8mQcfLGRM1g8mTyug48TqPXH4WIduG17BEWlvEjAq3ACesUG0Fuc2I2yZF6lHhkYr7VbIcRqlgcmIg7XZf/FQRcPVMH98QT9/vS4i5/xfG/vYggtSLoOAunLVGxijPDvVnlfJnShduQ+tI/FvCT3l6OSatiQnchpSrVHRuD0q7cqgaASDEXsnGZ/7g6bYo+g+dZPiJYvTqB6iJ+5W5u1egMlyAL6cQ3eiFzieaFm0JC0MO8UHYAN98cAvxbckYnurEr1s8YMoTNdxntHOLLZobJxyj2j2D5lQ9rUox0UNlrMYjODmjKSKs+SZ6+xs5PmcXdqWdtsg2Kh7/kgciBrk+RKToeqHDQ25hLi3RLfSE9PB1CCTK4aR8OFc+XUaENoDBYDtG9zVQzC24OGIdRqhBjdGuxq/5Mr5dVZTm+dIUJcHblorHA7YuAzq9OOkDOkcSqPJmeIycwYRGAnwLiLvgR8rFuVSkVfDt7evJaHiZpLBe8pNqcMm8ODeunn6fcpSS6xirdjHBG9YOL6Msq4wxXvdxNk2cd77mUTzluIkQXSULpX+ytl9OzW4585ue4vUXXqfVBQvbIScknx2FVmZvm4zj1lasAVY8Mg9vB8IyrQKJVEqrJ5Y+Xwgc4cfaVWvpCxRBVpvKRldYLyo/FVK5DKnHhpfVgG9PLS2dMRyzQJNTgscjVst1hHdgV9q5P8TBC0mwfvNofFqCOT8GGuOgKmQbr/yezsVD06k31PLwXT9T097F4m/mEtUSxtvPvs05G2jrYI5jJZLLtxEaCmv/KeqMq+zRdLrFKvpEn0z2/awjsO08ZA4Q3FxNmCeSVpqYFDuJXYYGyocfprf1BgZlZ/9WNXBkyhH64m/hvvgA/lj0Bwu3LmTl+pW8/4/3qfQyUjkAkSiZ5cjB2D6I0i76nVvNMEcDYy7nUIES3bloIpoyKc8sp9AO09rg1dZI3N8tw2eOhziJlMV+MsplsEQLN0fsZXfdnejrAvGOcxPQFDtUGf2aHjyWBIarutAUjaZL4qY29RrNJy1/QMwyXvGc4/iOJJqj/Eg7fBfnR51n95zdZJZmMnH7n/SGOYjKH0d/RAZHEqA4aTH+6g6GqT8h7Ug/JbGhjKnPxr93FPI7f2NDGEAby6vj2WHtZIG8lfOfzubzUeU4IiM5YIY9ZpGSbcMgvGcWECSi9jDAAm+IVAjMnfwd52vTcTllWM1KwrEwTjvI52Hwm9DIN9U6hp8140k1895VJoj1/i8SrGkDWvk8uZzAgHaCVUuuLo3iAaQtqo2U0ARkchkpZ3/Ep1esSGybt4/+NCc/PP4N7zTnMdN4E+EtAwTHBw+tg7FykNo/wTXgg3lSAZf9Lw915w+hUGe38ccNf+Drn0nofwD/LcoGzHRjGrYQu6qZA5kLhr7Tu6HJbSNcIsPga6AzrIvKLBnN4Xt4SJIDoZMBkMq+4uDrnxDgfQ9C5zVf1duRR3PGLJQxobhtTm744wYa4xoR5B5e9drEHjXEPrKBfd2PMf7COGJLTfyx6A9sahuPXnqIDzLX80p6B9uVj/C7WY/vgD8Gv37OuQ081gNtg+JeYQ/S8N5T7xFMOi8PxuI7Zx+n2+7G4N4FMtAHWbiYd/Fvc6PZfZ5UoYOsXXOpH2GieHo3OzOu6RuBh1Hta0kp6yRC7SLq0DvYgxYgDYO8m/dw+9j9rNI+heXYeVLOVlKXuwQBD6ENsFuWi6wxkfwZM1EbH0Dq8GHQd5DRKtgSDneMvcTG6Cbm6p/lia+e4Ojko3SFdjF17R1cGGtm13Xv4m1LZu6FmTSrzeweFPDIYFybi2FGbxaURdATNsjZyTLqwr6gO7SbObvmoHTJkUpgssaNFTAGuSibnERJ1kcsFguhOSiR4G3yRulQkFGVScG5uSx6ZjtZCUW8WTQWo1GCx+lh1t5Z9Pv3U5tcyxErFLTA7IokcqsTKa86RPSF3zk/+jxFyUVgE8H07yLd7Bgwc/lfaBkVEkhTwLEWX6a//Sit41worQZ8B0MAuDyo4JLJg61uOpGbc4ibImWun5YAVToWWQlv9INOAtNDR/Lm5gICjBF0JoxhZv4mxkX/iUeQ0OKYQrr3n7yiTqZdEcE21SokzjZQQ1doF38u/JN7x95L6bEKlmxZwpYlW2iK62eyAGPsOj78fR5tI/oo8v0QP2MfJqVsSMf85uQe4m/bRknK7dSbvhp6rhkaeDMIbtrrz7B9cXTceZA/F/7JDVtvIKwzjMyXktDO/oYXX36E1Ig2BhcMsm3UNgCsHphctJTopg/5ZG4WHZNH45N0J7qXvmD+/O1D9zDairCmrOSwIw5T736SapLoCe7B4Gfg7m/vIzgQWA0ShRy3QkQS/wL7opujiWwTEARozFlAXagICKz2hwAZ5F53lscSi1mlvp2oyoO0J4v0Hj8Y4aQNemNHoArSIWTmMdBuJVXTghD+Jc+l7EFv8GGwTo6Xt4Gjs1zURnyFjmA2GWGTEa4ss/LmZlAooGcM7I/5CY96Go+kryI/chuba64CGd4a9s0tpj7qmk/mkXlwSd0ggTVr1gx9PloJiQqQ25WAFzJfLf3BGuwqOycmnOCHBcU0aSy8pfUjwJOBQ3fj0G9bPQ7u5g+07qvCUR6B1MOT8LWWEjvsCnEXR+LvkdPjUZC3fTonp3oI7Lqe1oiqoQD+fZJB8uW+rHvlOeypGRyY+ymbR9ZzQCsmy/nJw3jeoKfp4+doHiWn62QuzQoPx+e8g91wkbFnx2JsfwzD6QqW/yqet8adGUd7RDtJQhnJWgXLA1qwq50YfA0IEoH2CDMX/Ls44rZDsB1BJaW45RYCx97DumgJ/ss3Mbf9G9pCttKTU4hLNh2VVYVEyVUwXqw2D071pVHbhaanh8SqYNqdfczcJc6/Y1O/5p/+odQoVTjlCQxre4se3918oVzHLjNslmvYmVQJSZUsUE0FC9iVdsI7wqlNqmVwgjdytZzgtGAC3niCwu0Whu2qZ3/f7Tz19dv0VL6HfdDGi6+9SNGIIrYv2M43d39DoPUB8o+2YG7vxHtG6hDVvEHyBslN6XQkTSDrpmzCc8OoXltNep0I9mksGpJyq+gcNYAn4wue8ZrEIf1lOkwdJMuS+X7l91jVViZIFNh+KqavV08Y/zULBfQaPUapEadVrHYe98Q4/Mem8WvBJzz68QNsWfIrued9cIQ0AjFk3lvAiWNnMPgZKBlewvDi4aSXeuPoHiBsZBheudk4FMV/u0/J8BIUzmCy+/+HJtAZ3knSLJH2uXhdMYZ1DUgD5uCRK5FKQddRjULixNqfgKWsncHeYOZtEhgxAvLywBgEsmSw6nto/7Ed90Q3nqkeCkcW4uUSg2WCAIEtxUhVpzg/+z2+6XqUX458izPqEK9OyaW78gAKSyvkicD/LTrIHL+C0sHFeGQ3EV2+lyeOPsEnj3yCn9qJLXM6CrsEtywIQeJBZTSQXZJNZ4QEjcJAsraNsTVvkBtyPXFT4ii7UDZ0VjOrahCEfOIa41A5pPRfqCH95Fa6o7VcEXMZ2L/oAJPO/EawpwfPgAE/7wy+HtbMa2Ur2bpwK0uDnwagIXcxfdrjYt/ZRQpzW/XHCAJIR/vykeqjoX7+zCearPAinv32VhRuDWfmVNAriBrzCXKoi4P2BhvbvsmA+T18fe/XQwm/c73hJt/z2E9H0vzbDAbvmUQdLmojXgfg8wHQSry4LnEZpz+TERLhhfyGKdzl//Df3vVR2wheunKc4T4hFN16EoP3RQBim2K58ZcbaR3TiuQ6CavWriKmJYbm6GZG9psZ9Nh4Vv0OOj9AKsEt8wwxxwFsC+khM3cJk5LDudK9BbO6Cl/zyKHrv5zwPEUdfuxOuJ+NN28c+t0zvRAxMJ76zPkoPUrklp/xcfkw6DOI0QNv98MNEgs5fE1vZDnX54/gPvfd7FI2sc/nPrI7D0PASDaPzcQ5fpBxHSdZ/NtsjFnjcKduBa4xfajqykk7tZfygmVc6RGo09exfcF2TPGij/bciud4su9JUqtSmbdz3l+SrkPj9y/zeMDkH01XfD794enEASZTJzXV3zLQvhGny4D3aSBeQ4HjITxRDuDv1X//N5pKpfpvu7f0f/4nok2YMIHvv/+ejo4OPv30UxobG5k0aRIpKSm8/fbbdHZ2/u9s57/t/3BbUyTSDfgFGzBqQB9ro3BkIS65i0CpCD4lBg+SMO8otjgPW8ddZHVSCRX6EYDoZP1RlMmhn2ej1v8VCBa4L+hNnvt5KqsrF/BZThVf3fsViu5BRhSGg2Akq8jDM+88g3pATla7g2c/+AfaMw6wWvByerH12jmVDZFdrF/6K8pYb/RepXhkHm4d2cBgvw5tdTuObgPOhlZSzm+kO240e1e0URELr3QlcfPPN+OuNfPzrT+z9YatCFKBlowr2EaU8lTxWF7YeBuOUAXf3f0dFekVDPoO8vzDH/NdwG1MOTKd3MJ0nAeOcttWP0KEa9mGEgkMhiRhCY0np2Q4ydXJCBI3Xxjg9sNb8fXN4oG5F/C5dxPuBjM3/nIjcY2p6Nvu4mhPAeVd35Fln0Nn6UoOvvEz47ftQGvU8nIfnK7OJKDjCg6jg/AR4WjTxOq223zgYT/4adN9yDduG6pCc3rpAXjAD96OrmVh3X4yTxXx/7D31tFRnm+/72d8MpNM3N2dKEGDuxaKF1poqbtQp0JdqFAX2tKW0mLF3d0SICHE3X2SyUzGn/PHwy/8+p599n7PWfusvdbe77UWi6xk5pH7fp77vuR7fb/xZwNZtGkRUocUOeArgxBdP/VDrtAfZEVpUaKwKrCoLcw+k8FLB2diSCnl7dVv0xsgZf7W+cSXxv/jeQlQR+BtnkeINhenXUBul5NnFRhSD/aQO8iVtKC5lIGmwca50nVUu83EomymJaCFd15+h4uy2/zwARWnyThzg80BcIdGilTvjYvVlbBLWu778T7c9e4oB5Wy69ktrAzvoN+mw0seTO7pkaRdTwPgAXf4cfhz+OgSKU0yQ5Q3J0afoCWghcGXBvPw6jf4qsWfK+GPkz/+RQT1P9vaXSTQYutke68Kq1WB0+qg+9AV3DoqSXO5jZBbVzeSm25bxO+sduHCUDFgkwjSgQK61NRHzJW/iLy+k6D9ZipiK+jx6KEppBW5WsGmxZs4Ov4oxwZFMMMVlurAPHc/hfP3Ivzbju7f6s+d2+/kgW53DlXeR1t/HPHCHRQlFQEQfSGHa1vHEWa+g32LY5HcMxNBJkfb04zS0odnUzFOQYZMJkNj1OBiUFJ1tIrRu87j3VRBafCrqD3X8GLSDUoWb8UcrcMnXXQgtNaRANy3/j5OvSRSrtoVLph0gQhaV/we9CR78VkOm8CJE2eiiu1zttPq34rSqsFUFMWB+nju++Qzasc5mLp9HM+uFXU8GuyiQ5rdryP5ZjJ02zk0+RAFaQWURWzk/NDz/HLPL7h3ujOsOAnBRcb1Cc9QlPsgkjYn0b8uYrlTj1QqxSlzIkghanwUM76ZgZdfN93RsE7zG8FeRhp925Eq5Zj14hxaOEu3VzfF3qdJSEhAJpNx9oOz/DzyZzp76v5B42R12jgwZT/nh57Hu9Ob1F2x9G6dQoeLicPu7ShdVMw88DrJN5JRG43o/pzBT79PYKJbJWsjDThzXFj7zFoqYioGjtkkmNjmGMRZr27avdq5HHCZbrdruEjtxCkgStHOE/pgxhWmYeu30ZzXTPOlOvG5kkKO5hLDnv+F6hW3YYY3h0/njTfe4HLOZSK1yf94rj2bikgqlA8I0zsFAWQyqjLm0ucVhlfTDY4qT7HVXo33qWIMO47/g4LO6GrkJVUbj9cKpP14Nz5X1LQGGaiJrBn4TKcTvgp0cnDWJToCbLRFDcUYLSJDn+uAZfkPMca8n80XJ5J72ov0ne7M2DMDjYuU3KxG5NefJM6gQrd+MX7XtZQOcvmH/tadOx5l0D4tClcVh6dVcDVTDCKHu8CMwD5qU6bijInDeeIUD3330ACNEcAHbdE0eHbiVEmIL9JQdlWkHnvZXUVD2kVWjbiHxdkfo60z88rbr2C/FIubp4HFP7zFmtibrMSVFfo4/C4V47rrLMujxQ6iR9rh695BFLtkcCyrnNZ0EbHc6d0FfmMg+RXQhpLcOo7pO+LwEuK4MfoRiiLzuO/4B2jNeiQHymnYeXse7dI+5MY0pjVBQ5WatCOfUL7vGEv2XGbGCZESZ7JG7Oi6f/YGxiy5iMQGSUVJeHV64ZA56HXr5TmDlYboKhrDWlHMyWXZZ1sRpAoi5NAcCY8G7mK1F+RqTNyz6Wum7J8izvXyLYxeth+ls5fPFp/kmQ+/piNDwron11GcVMx6P/jBDywKb77RQ7FZRWl8GXum72HcIU8e/P5BCtMNKGdNBmDWjrHkns5lV99qmuxS9hmh1JCNg0iqA6tZs3oN77z8Ds54KW+88Qa9BR60zt3LkfE/4eG43fV9enYRupfWURNaz2urPqRtmITipGKqoquwqqzEn8jFvG45Jw5N4NflbxB+0pdZu2cNII5dpeAuBZlEhn+rP+5dLlya3siOOTtIVIIxGpJkq2n31vLRk59Tn+vkwvBONi47hl9TL+MPD2bCLic1e4tQ+LhTmzKV9tAM4i5tJPbUb3yeWUnKk3/gTPWlJLuDDXdvwKw20+fWx8VBz3JfYDkf+Ig+UnFKG0UpRQgSO6M1Nt71gR/94CF3/kHbpDVHsyz+B2aE7eetTaOwfHEPzWG9XE8TC8kGJ+w0Qgv+2OVOenW9OFr6uO/HJaQWpNIvQL9MhlOXyrC2fD5TvIyLqyeeztcxacWsdG1ELX8s/ZuIMRHIDN2kHvuc1BNfYnb1pXRiFk+fjGXs0WFY9CZm7p6JZ7cnv97zK+2+HVzoh/3T9rF/yTa09Q6u/zKT1C4VJq2IMN6POxN2JXPxaiwtQW1URlf+Y10y21qIuvg58sK/B34nSBz83CsW1fqVfrQGa6mLS6Iox8YHz39AnvI8X4X6MUPezkUhkg0GcUxdeiV4d9ymRBOktwpWEgkdPh2497ojILA+wowhGnIu5pB4OYywsdP4fWXjwFr2cBvE1sLWH2ZiORFD3rAatt257R/X/ZFbE1vmbaEjzknOO18ydt5x5gqZvNQJqbVwNvxzLk56i/KRCbz7yruUJojo4o/1sFMfziN+9az88gssmaOwGl7l4xJvnFIXmHQRwhZQZ5lMa0Mggs3GwUkHKU4UEdVGrZHa8FoEjQLTzRpSj60jovAwyv52luvgm+ROOqcdwRapIqDZl+jKaFJuxfY/X/sA47lkXvjwBZRtDnbP2E1NYjVyiZjEsQPLfhrDxbcewLVdRnhNOFIrePeOY3sgPOZpxTzxJOVLNlC+4AyjX/ueJkkcpw3uzGmCVlk0R+O/4euHv8YYe3vt3XhjNd/nicCllsBWbqTewK9Q4M6td6Kwikl61/4kcuyiX2B0FykfL2dfpie2Cq1U9GUT1CY6fDp54403OD5O7Op6zhMOB8OyG5mMOjZ2IMn7L5uphXFKXwrSCmgP60PefpK4UnFNUUpgmgbu9FvFszNd0Pt+x5T8XpZ3ibpJ73bD8AZQ9VXy6byLGB7ZSF1kL81BzWLAfstHW+h+lvOh4CKxEFjsQmaeSKdtU9ppixyCJTACp8VBWkEaIQ0hgMDxflBJRSYVJ078ugKIqooeSPr+5P4xLxzZz+t7d2E3Wog/9zN3bpsDQJkNPtfDIMmPjLj0NIbyg4CY2N7cFcurnbCjZTCvtvry3pEZdHub2DNzzwBl7yQN9MoeZYJ3K8te2oAl3R9w8sGt5OvLR4+B0ou4ViPp54qwmXsoTxA45b6OpzogpNGOXmXB4RRwdPWi6WlG4nSS2PA+o4oK+GXPt1xcn0x3fhUPf/MwU/eLdOLuPy/g2fcWkh3sYGT4INCo6Pbsxqw2E14bDkCkQsK3fpCqNlM8qhvTsz8w2cdvYD4d2iNEvP8J5nv3cj7tSdr8RaCGIqCDob3eeFpKeOxSFhNfWU1Xzw2U05dSn61nrhgyobJ38fxP93PftnEoEwdRPWgWXUqRmtKeJUX4PAXvOG92zN7L4Ym3i/AztPDm09s5/OLnCE4nCpviH8AXH6mEqXo/kts8icnzZuGfC1GZVTQ7wL8aPrGYuTLsPD1hEm6MfYym1MeR2+S8/N7LrPpoFXrt/VSlz0FrDuG5tc8RWyDSS+81wp99EKKNJvP6dFKL0mhImky193j0Zl8+OLMFk9OfIUpPBu1LYlheKsvXz2X2TrFo3evey/X067hHu2PvsxJaH4qH3gOHtJuFLbCrW05PcRSSdguaC5Ws+GUFarOaABn85A8vRPUyY8JVHIEyQutCCWwSOdl/N8CSq/dyPqGObx/8lja/tgHdxVb/VtpO9NJ4xcqlIZf47e7fcLj+my4RcNJtC3pzAF9d/ZEbqhcQBAFLi4XLdZmMbhClRFLtf7NHH8voU6m0X2ln6calA/FnXYwnmpwU8bk39uHS24LEYR9gERh9cjQjjykRBOjX+dPuJ+5HpTbY3e3DoDYL1zS9WH0U/HrPfnZOepqhpUe4VD+bNd4wc8XH1A5SgUxGU/wY4t5ehlJmp8cJGwun43n8NNqm0gGwglRyG6//yfUXORsVRKXuTfp9BSpjKnHKpFR2ZSGTOnnoygheX/M6sh49JckNt/XhELtlfFsUWI1id1B0RTRDzw/lfq2MPwIg+qnfqLhnD22WSvSa00icEgSpQGavgQfaHYyP6MTDv5PgSk/SrqXx7/apVx3n6zNx74DEN75kzOJD/F3rgcGioCK1kNlxngSMKyegI4c7tucw6dzPdPh04AxtotE6kT6PaXgMsmMJisJDupfvmz/Ho+VB9kqXEyJtxNfViFdsLSZvJVcnv8CqiDS+NtioDa+lKsETqVqOPVRBeUz5wDXl2LS0REJi7FvUW6+jS3mV9SvXY1PaKIm7RGh9KO9L/Dipc0WwOgm5spcT28ZQWxrG02+txnZNyf6kdt5L6iS8WMeLH7yIS5mZ4tBVPO0Bq/26+HraZh6eWc+pcS28/8L7SFxktA9rQeveR67bNUbHXeGu1/bgFprE1D+rGHtiEcbq1ZSWvodEpsQFca9IbhnB5NPfoDVqmXpgKj4dPnRE9iGVS3HYHDhtDvxqr+DaKYKGGs+mknjZhCCDgtQCGkLETjHvyCbGBTST98T3HHj8K1wnuPLl419SG1FLZZQIblwdNZbZD/3NsHt1VM2q4u85on+UUJKAtCABpQS2OcwYXSSk+qeSEZCBTqWjLryOdr92dnu40PW5P06Lg/9sRrgVUMgUrH1uLWFP3aY2l2nE+1FblITXhnN2SjaqYVni9awYRmFaKbmncln+83JKEkrYtLwZdVIUKQtTuBA0hw6v0v/G2QRk3R38ueBPYspva767q9wHfm682IjlciHSW/SrEgkElp8isPAQrQWttH38Ox4tpSyZ08SaIam4Gh+iLHANxbb9eMd58/GzH3N2xNmB4w2q/W7gZ5/66yRdF99ZiSAnvPVO7tg0gZDjIcj7K1D1irmVZCU85gHD/fahkz9HYe05bEotPR79CBKBZW5wOPUo1dlHWP/wbzRNccOrrZ25f8/Ft8OLx3Lu50L9XIxbY9Gt3oHgFDh29zGS6j4FxFxdYddlfnjgN/YssOMSFcjNDD8KfP5dd1r0Myw6X6TeXoTdPEj732eQIkPvqccacNvn+ffO5vvboGiHP17bfvhnBz6wtXIpxfo4ou7ay5mZeTRFdVIVLeozG53wlwE+q5xAXUcCTrtAS2AL3V6ig5CpgiV+1UzLrMB/8R6cagU+rXI0RjGX9qsBfm4PQiIBlUmP0N2D1F3H8hMd7HK9fSGKfnhy3ZNEHXYlVtVPWyQMU4v7x4HJB/BK88JqtBJWLz6L54afo9e9F7lNjqO3D7vZjmz8WL5/8FesKiteUljrAwkJOyhJrMXoZkLmEOnU45peZ54rFIbB3qMZFH16N7Kef3a67DfB71zCXToVjWoi6Qf9eebTZ5Dbb6/tbvTzzOz7SFtUh1ukD0ictDhA4+iAG29CbzGPeVpJVYK1sx+PtjIUZgNSiZT0q+lEnPcQ5z0wjPrEidQEHyH562T+KvoL22obh5eJPofKqCLlRgoRtRGcGH0CkvhvmtMJZjdf6pOnYPQM5syptXz5eQhBFe+ytq+EzeZmvuuAnMtW9l6fwZEDH/y3D/R/kMlksoGc3/8K+0939v3LtFotK1asYMWKFVRUVPDzzz/z1VdfsXr1aqZMmcKuXbv+xwf5L/s/3v5jZ19B4Fs8UJZLsakQi8MNqc/tzaPQCu6VYBF6sGSdYEngGKgXD5Cu7eAtf+jz3cwPinBOBVwhUjGZaXun0RRtxdtyk6KW0fheL8OYZcLo2cLNsf1cGPsZJo2JwGYxiHjW7zJKhZo6Py3XzF74/PYRafE+NESk8YBtCN/PepRvPplBu7wHW4+FqXsnUR5dzCBlMVOPZaAB3BODaL9FGRJcehzHJCc2ASL7XbHWhmPvtf3jntfP34yHFLJOT0VeHwiO24MSWxZLcGMwyaMf5/EXevm56HNoKMC3T83xYBjbCt/5QbftIHct/Iz1DRMIezOblgA15XGiU+uU2EDpyTRJK+2hrawJzmLncAVmXSL7+5SsSTkLlrN8cvkZJK52+qPNtPq3EqR08rEPfLrwczYmv87rCT7YzXasFjEQONUP22pn0FemQFl3Gbt+MAnFCbT6t9Lt1c2yFlgoTSSovR5znxMvtISVixtnqgrywwAMMGgPCxv8eeztx+hx7+Gne3/CmH2dWJ2FZzxEENKxbgmb52+m3Vfc8H/xF5OnYZaveCVtN6e7vsR0n5Iv0r/Aw5LKpwueYXT4aJZoQuhc8y6+2tWEfCmQ2byED178AEEqYFPaQCoWxwYdXovS0kfB0Dje6waFU0vT6PeZOxvqNz2Je487UqeUyCOj+U6loUOzmK4TThLvTeSVex8Y0LLabwSzMY6PJh6kc0QIm1TTOBFwAuA2fcipdG6OXo3TKZBweTSPF7zEtnknUHn2UpVSxEHrUXZGNPPFE1+wc8LDxAYMZ8u5tUy8JU9T6jGDovMNTLhYRs9rPbwgeWFAYyu8Wkfexo9J1c1HevW2Xo7cAks2LuHPRX8is4HgFP5RKHm/Syx83dQ2IlF5sqTBzk+lo7CkF/GNSYNUaaVEbaH1nUQIqMcjxImqy4u3Xn2L1W+vphSIyI5g37C3OGL6k736poFja7o/pWbBjwi997J6xwL6GgOQz5djd5cMjNtP1gvkaIPxSFCjUoURMSmO+l/jsCvEAOXk6NNMvfMWBYHgpMcvBk1VGWbTAaIEC7OPz8Ma7EHMqkwK2v6FrHcCAh/WD2ex20XGaQRqYhrocBfDjyKr6JCeGlLE1Nwi3uJzuDVkek89B6eISawtpcMo+msy3Q+bsCm1CBIpnR0T8Is9Qnnvx/wdt4yq2jDOdN1+r/sctzfyF669SlH1IBzTnPil+lG6sxSrwoDKrEJhU9DZ2Ymnpyf5P+QDYLnYSqAQiNqsJqImgt7pPRh0BkrjS0kuSkZpU9IaYmRHYzxaSQMOm4NBeRLc2zNoChcXntNRZUTkV5JBFELGLxh0hoHr2R0EXbYa7m0uR+0soOuMng3XN0ASzLWNYlsE/Nnfw0fbl+F1XY1lrYV7jt3Dpp3ttOUlEqu0E6YcxLRwPZvbDehr9CTs+p6mqGRIh7t+v4ubZ64x8XDIwDkV/WJLw08rfsK33ZfWKxakE6V0BacSVHocj9ZSguRONgXCu/OOUqgZg3uJwNz8uZwYc4Iu7y629oFa3kfLjN0ER4xCY4miR96FTu5Fr72LZKWIKCy13kbWFkh/Fp9/u464pjXYJQ9Qdy0Kf6srxWN6qVKcYoRkIexNAlsvHj4uVKXfweC0AJwNJQPXH+maRGvMVOT+MgQB2t1va1JOb4JTIf1MXvYUh/o+pdGyH5PuBjaFjeV/LCGiLA7Vs4V8O13DgtgMZv46mSsN1cRllLOjZBVJPheI150k0HaVPZIXqY6sxnArQf5Zn4N9fUqStn2MuriChuknyIqp5qa7DG41w0mQEKD7gY27fyBlSA1r/SKZqFKKlZg4kWc/RHaOzs5KnOZYzG6+mFRdnGqr4i7bMoIyJpA1LpzO09eIvVjE4ck36NVcJ6z9AeYm7aNjYhulAfFEFO6iPlQck30m6C/Pwa/hTVp6E0mIbGXBlgXsn7Ifi8qCzqBjmFLKhif2s7I+kCu91TzgUYkgbMcGlN96VYbumYpa009Xcy/aLlEzZZwGnvWEi04jtY3PgL0fH81tkEeSUnyzG1Qh/LZzGApnAMopEq54XWHGXpFX9XLGnwx1E6lU/57zN0atETkqdvep+KsfctNGE6pIJ569OGVOXnvrNRyIiJ7s5knc11ZIgM2Vu4/PY2izg59WfE9EvRrVpocZc+9m9nnW/iOxL3VIKbYJNErNdOlXEhooxSNFzm8+n9HhI07UXbeyHtUJflQ9o+dY+xsIPAeAyQnH+6HBaUchkaOTAk4H7b71VAaeJFc9GIW5E6VZiVlvRhqkpj0ih+CSowAMWyb+v6F+GNPD3ej1NlMdIC5mATIxYbD3UiyHN81GNd9AZ/gJAG6EPUqTwcnefvitM4bMqzFYvZMIvbmQU7nHafL/kyXtIaTV/MDBlMfwCqyneZCOBpcj/Lu5SfrZknuDPzKuQftduB+MYD4moJDsegntcheGnh/GuBNjUYZLCW+9B//O2bRrTlEc/SgelnQAtFfPIreJgIj8zBsYxj3ArKMTkFzIxNpj5uiUOpKvKVn5w0rW3f8j6/SAWxsyTR+xZQJXT2RRHFtOcWIxzYHN6D30PPXZU7QFGii/I42tHqLWiZdUpPtOV65HbZ5FjVBMVGUUBjcDlgAHllsuWFPTQWauWMmfXd7Y69fSr+lnqN8EHna9iKAysckm5dEvH0UfYUfT5cH0uod451URYezR7UFETSRWow2r0sqFIReoiahhrmBjsiBj/tUhdKOEhQx0zP7LLAL8PfcEYT7ZtLr30qoW37tYhYj03qHq5ap7Ed7cwbIWmKpWUtDnTb0G6oEIyUaq/PvwJo7/aGZpGy26L9l/sg9dczCz9kxjE7tY8OzfhLmL/lmlz/NcHAdJKXbOq2/rWVZHVVMdVc37gW8h1Aio+vVElsFQj6HEJe7HU6enZuwZor0fRDrsR96/S4KPHnbpPZF2ePBkiC89OR1YgmbSHfcau4Lg67YcooFpWvjKrMCrX03EFRkTLq2g9DWBYWVH+cTyBz1aJ29rlqGwK3G169haC0sCJBSqL1FohKFSL76J6KHIasPcYSO5OJnG4EautkxCektL5LcAB2U2aG4UCL6Ryp4ZInhh9PFBqG98hPnFx2lMnEBj/Djq1OnoTg0mWX6WsUWpOHsicKT9M2u1yQAtdlDPPkC+XknwqWU8feZpNtyzgS7vLryq4B79PXwQ/REHe0x439zBxKKJlMWX4SUV9bIddiO/fHAXyTIbKtMFNPI48Lp9DolgJVQOblKBfyXeXCQW2CSF6Ps4Y0zm5FU9Unc1cZfd0XaOJz8rn27Xc5yLH02K+UEk2rG8/crbCBKB1Dqxs8VmUXBi+2gC1Ao0cyazNvNjjK7GgfOekX7F/K0RdNudnB1+lD5t27/fOpGuzfQqLajNntxx6E+CMr1ZEfsx1SpYIDxIUN4kis9k45JQ/I/vfekLDqGUau1G1rc14N6XwhMbDLTkesMDh7k77WUk9r08Om4he1R3czRUzUY3setWKYF77Trq9VocVjva2RM4rRIlBlxsWiYEXKBT8KHaPJus5DBOjj450EEceVPMWD3m+xenTVJ2+bjz030/ARBTHoNHYiX3TzxKyaGx5LYmcmBKF295QRcWvuqG5z3B0+LFQ74VBBsXs8UsAm5sShvtoQ24Cwoc8mnsURvRxJdjV/jT5RTjlG2BsLgZVPYeRo25yim7lDatGx2h6aSGiwnhZ4ILeezoAlgK3QH+dLmdBMBPBn0CPGqwUKJwIAl25YsnvhgYz1/9YYpG4NVXZuHW64FHnIm40rCBwi3AtwH+LHrsIDPKhnPEHoCAk6im5QCorCr8/DqoDZmKTdJMzqQt3Jt5g7tqfud05HxkDleCJN1k3WWnwTGdPMl8TrjcZOvhLXj3jWZsLLQwjpKTJWgG67g09Bqdnk0sDn+WTbUia4lDcKDJ8mHbndtY/Odioqqi2LxwM0sd3fDq2zziu5n6y3oqfC5jVVrxkMIKHXxZO4wD587w7DMSFH/V0undyc/3/kyDHa70hjIpqhOToh3j/jtIv56OTW7j5OiTZH6TjluMkQnxUGkFtyI3hl4ZSl5WHg6ljQAZWDwOcqBLQ319G2MWJrD92Qrae+z0diURbhhDQ2o0V84bGXQmC7fXPDk+vpbqSHFPLcqUMGXIMACk+VdIPnWSG6MeHkh2nxl5hsamIKY7QeKw46sfRZPXJrb3mTjZlIpHTzQpZjl4Q1WkqDOfVZdLoHEEDHZhetgBHuuxEt6+E42+B8HiS6VZi381uLnsIXZmKcHkElAvIOuJpz3ZyCyt2PHotBURsu5BamMctAwkaWWcrF3MsrRXCQ1uJz/5Bm6qJf8A9gBkXM1g7IlAeh7vIb4knsV/ijHXd8Pz6Hf48JpfKwarmfIDf3Df3zH8tCKUuvA6up0i08MvOjWvmMwYLvvhVz+d6+nXSVDAl35gr/Lk4PpZ+I7v4lBSB3VSUNh8cdH701M3j1ELPiJ80FYObviK6rRgzmQ9Sn1wPWvu+565hXaG+sk4EgYhISBpADe5hTlxYgHliD2DPO8jHEowkRQSRG+FmsmBxQx1gw2JJZQkPsla1QrM4VJ2zdrFs5+IwI6anDxcTRoyX3iOnuGd9Kck4NqfRJ/LTYINOmJbghlWGcXxX6fgMr8br+5OikqjOR/3A1bVH7hZallfk011RxhmdyV5mXn06FR0u55jviuEyQ1sLF5HfWcoNpf1mF3MWDBydexFht19lXrFUr6uKST/qoRRqXr6df4gDyC+SZRakUjgREMGZy8m0Gu6ybDKVvqdYpG5ObCZGdo3EQSBv+74i6595Wj9xP0+f8gl9Mcn4tZsQKqWsf3O7QNzfL87PB7+Ff5VIl3uPf82/3oPPdKUUty8epDV/cZNXRb6WD2larFgdmDKAVqNGSgCv+S1c28wIcKDR0sWEzoilI/dPyavWQQNanRGlqz6nSOlCfx2KI+7/hNUnuuRk24Yzkn7KRy3JFkKfi/g72V/0xo5BP/qi8SXxnNyRjUSV5Gy1+kUKS+HnxuOi9mFkIYQTB52JIrbKe02933/OM+U/VPw6fRHkmykdGcpXlNvb7b3nbyPt994m+dan2Pal9P4vm4KglQ2MBcNSZOQ4cA71hvPu2dwtucKi9VDoAf8HEZGHByDp9WK9OWp9LmJ8YTUIeWJL57H7OkxcJ7q9Duo9BPn+LWoE8xP92bt48/RUexBwYhsziSJcdsmA+wzwqC6IIb/sAKvx5tpiV2IWRGNXfE1Z/pv39f7jpNY1JkcDNrFpkWb6PFxo7E3nm5zIAavMHz8pexeuZv68/V4pg8b+N5zGe+y6b0xAKgmw/5JX2Fw6Rr4+/eOvzhcdYH+tKFIbmmuKf09kUqkKM1KJD3iXAUXH8bFq5UL8WLMMV0LFq+dGM0JaFrE4n9NRA09Hj2cbxnDWN+rPJedz9flrgRJxtAgiOtgqwMWtUCcPAbHsmZmZPmjqlThlDqxKW180A3r9KAQupGF+bOszsKUnRKMc6MpHPQvJh4JavNvTF39Hacc97O7+xFKwy+S2vMqsxBBSwHqTnrGFWMNiSKxTYtZHkKaewPnvbq5MOwCujgdToeTz5/4HKVVSZdXF0vcYF5JAgU/rOXmmDlccV7GIRPn2UMGz3gCntc4GzKd0/q/WfHjFPpd+tm54AdiHC7IJP3siymhXaEnu2Uek/ImcSnn0gBFfoJGz1+z3dhy8xV+x0CT5iZOqZMgGWwPglrnJXxk1bgpOzE29zLsZDrpGaAUDDDxDJjbeaA+Apcdw5E+qebSzOfw0MnQSE4y+PJgPEwiGNHpH0irKZCy8KcAKGov4rufvyNXlUvqklSKpxeT5Ejil3t+oSmoidG+oweeh//Y2acwG3DrqOK6YSf1Ne9yRnAwRPinRudd2LkITDv2Np9+ouXpZ57h/1RzOp10d3fj6emJVPqf7rP7n2b/r4t9/24xMTG8/PLLhIeH89JLL7F3797/Wdf1X/Z/mGWrId3iSdDHT2OPiaImJI+c64/zw/0/YHYx0/tvvqlTEHjv7+mkxzQxfsGWgd8XKJdys28+odrLxFxu5LjuBo8l5ZEzRorBcw6fev1Oj13Fd9aj6JV6QKSOBPB37aXW5RX6ZteSGb+SPZWPURlZwZvDX2RCsg8WlzB8awIZXJqKybOHzLxB9KsMHE4qJiS8iiGtvjQfKUKeNZz6xAlouxtRtTczqA7qx1wjeuQ1jvtsJvuHbPpd+ilKKeL+7++n18uP+fPegRH7eaTvF0aeHklFTAUpN1JIK0gjIGUj9aE59GjCOTXlZxLu3MsvHaIQ9XQtnOyT8tkrK+mSOTk10UptgIjQGKKGO8L2InOM5eotynD3gFAcXun0Bf2MVndiYNz2P/o9XebBLB03lg3+GwiQwQUz1DuAW0HMuuh1yAJlMBMummFL8xR+fnYVRxqfR9EVwqK/FnFg8gEuDLtAmQ1+NsPIeWCUqCh2L2PDHDFJ2OEQ2+31DtDb3LA6fSlKvo5J00dYXRiuB8byxZQDfGxzRdboy3G7Y4AzHsSuQJ0UbNZi8n6LwtC3mbF3zedQRRTpXhKWK8tAX0id05NzTivzFXJuDLmK3CjuVHKbHP9Wf5QmPeCB0iJu2E0RvmzvhavBnagnDuOccJ6K3B42j9gAQOaxsbQAN4dEENt7HnuvibbA20mQc2Y4RxmTJLOx2dwRlLd3xgtDL2BwM9DrZ+bvgCMcqvHmvN1AkENCn0sRBpdeNvSCXOXOtTApf7Zm4apw50frmxRFfkyXQRzzD4PTSfM+QIu3Hqn8n5uF1CnFYjTgsDqQRkXCyZOYtd6ojZ3Elccx9dC9DL4YQuf8jn98r8kh/gOQ40TV6KBu5zgWxbTws1mN06rkk/4+RobX4PBuw8sRPEA7lp+RT3TNEByCkuqiSPplvcRcOYXJzY/6pMmEzRrMqVA4RCXuw/rZVt9L2Mgwri7XUq4UC9LVTgMjlQ0o1dM4LckccCgkt5rOz404R9QsMciJuL4LVb8eiqGfFI70aYgviULvVBHjNgjX/gT6XEqwS7txD6jmmcw/eLzZA5lG4EZOA23CP5NPs5tBJoE7A+0s2biE6shqzg87z/j2IALd+2mK0KFedA1X7RSirm7Dq/kmNqUGrCYaMqZyTRFB1drZyO4yULKzhMKNhZj8XeEWYNB+ZDSTrp5Ff88oLnwqdmBG1EQwe+dskoqTqF9aj0LhgT40BZ20D+t3N0nPSceisjDq9Cj6Ogyo+9V0e3az7ol12BV24hvf4svqAIq3L8Yc38fRGTLKg/fj1zOfksFT8XJZRFLxUSQNO7HN0OHd4Y3BzTCAQHNKBVb8vAKdwZ3WrBZeO/EanzzzCfVSF97uAqk6mSb3p7GO06P2ELsk1m+wMmdcCS96waqWJ1hWcYgYjQdShRSzzg9NbyePfPUIfu1+VBn7/zHGvX4xUHqMsLowxh0bR2dUP0wEubkPhcVIr28MKSo73Q44nX6CImsvyy+8gaxQwsUhFweO80mQCw8PyuP9rg/xf9+Oun8YXzzxBTLgRjhs7FBQ+MIzZIWZ8W35lmaDEcbCHI9echM+4JJmEq/d9zhBfbOZqf6M3srlzAjyhs4PAajTLqcrJA1ZICTu6SP05hQOTD1AoEsYvf6p+EZDf28fM3bPoCaiZkC3SSuBft15KvsO0xPbz7XY00iR4d8o6kUEtEWR3fs6YYlaTI/MY5y/GPzGRX1MbrOZsaeWMHRXHPo7C9h71x98pI0AIFwOdnkPLV45KOKCUaef5O6kk3xnGg/Au97gKruJrV1P4unNSAmFbDALUuhvhMqfIHAq949eSkFyOl+15lLZ8jRl1i/4Pghmu/5GmOMc02Mu4Nihx7WrFpnDLurahF6lUGVn+4QDjI94hBF3XyIo4CobgCobhBR9T237rwTXHUOR/gB5OaOoiPkCo9bIlciviQ3Q81MP1FjlGPSb+KLQi8CEMLqN3YxqqCbaEcOik+m0BbdTMCOTi3EiFfy7XbCpfgSbBmfz49lQnA1NKMZVkVmTSXVkNcMQAR/100eTf3YeToWKisni8/bXgr9uUWR3km/bCLyHyuVVbkYsIU4xlnpHMekqeDN3IXsbFuM0ZhBWextFnPtIPgkJFzizaRIylQ9SsxF3vRsARoucdqkdmUXOl41JnLfZeOT7R2j3bedSziWSb6RyYaQBV69JVGV5MmVQA21SDQZNzz/eBbWliHGRv3FJ/xhxF7zxkOdwacglZjTBg97juJOveL0+gx9ceqlyylFalPiq5BTGFrF/2n7uX/geNwrEIKsrMIkbab1cTl7Dhvw00v5+A4+5agSV6CgFds3DN2Arr3jBarWVRq9OlKoo7t5wN726XnbM2UGrQwywX84LIev4UPyGexN2IxGvYeP4ngc5KzTQojxOZXgllTGVBFvGD9xLshLOhMAl6QncpAIuEnCG6rg59nl23ykyK7zWZUJibmBCYiM11gt45w8ma/92yrMWkp7gZFnda1ReiadlaQvWmCQ6nF60RwymOOZuXtHBivlnecvjWZR+7jjd5qLrPSVqBPVpMboaUTikSByQlPE0v2hep8K/HIfcwfbYFurs8NmCbTikYaT0v0+HoRf1oD249p9gbzA02WoI+vZj7igcz93v383V9KscnXuGiRrIVYPO1sg4HeQZbeT1WfFv8eey/TjRXQ6e88/hIdk28pTjuYZAdUIfZWFFSJ0qnFILY848QHreKQz1KTz61aN0+HRwYOoB3u0Ga8X7rHzkVy4YX6exMR/vpmp6leL9rPUBqTGNStWXxMdGkt82Y2C8YxTwmrdYqDxrhpOtTnyvZHHQ24DZww1XrVgQ8bSsJRdoaH+ftO6JVEaep8+tj0PBUGUso0uWy82SVgL8nWxc8get/i0orN3QWgXuyfzS8wyXM7ZhdohI3De8wCbAO7e6r5yCA01yDDWDZhJRsBu73M7XerALUGiBSMGJm7KN+a6ittZTnt2MVj5IqDKbQelyKmWLef8WqUG+Sc0gDxjlAvdNOcyhiYcZXruSMu+zRLhNA+Bk7RLUIaUMvjEYv7ZgorzvZ2rA9xgi6/gD8OgbwvigucR1OnEzaThV08X8rfPZMXsHCFKG3yJw8D6ci+rIYCrvsbN+5vsDNHFGbT8OL98BEevQ4kNkV98p3vvIs4w5lEuiQ4Ow2E5EdQR6Dz16Tz0n+uFEPyDvAG8JixsFenW96ExD6PIWmQiSXGuQv/YMKfE9HJmQh1neyDDP2VzT72RFK9QZPRlcFYhOq8Cr8xzGQFeChSFkuF4kVQl2z0zGHYoltC6cey8kYQi7hydjReosvLK5au5gyPcRBKRYKB6no0rYDIDSomThb8noI6pg6VjsClFbp9lzC5/4QLFRzrk9I/HNMNM1vHaAfh/AEA1HXLfwS+lM/AM1nEu5OMDYMUMrdgSukndyZOU3zK7/hIxdJUg048gedpBsoEYt8HHdBuonuZHYspPYC3ZOjj5Jh28HT7VDsDSBSW5xFLTG4e1ipDqqhRYv0SdN8LlAOwr2O8dyIaKfDs3t3pCxLjDryAguHRpC3zIjcLsTQ4LAiyMXYHVKebo6kwrXTzk55uTA3z9c9SESQULvZ2/hmg+8eTsBXRFbQWtOBScJpODYXGQOgbiYy5QVj2DIlAsoJPCuD+zt6mL5+hLcvLXce1qPyqJi3ZPruOaZQr1xJjP9RvFr9DIscXksrM4h98tX+GHRfu6fms/LXqB3CWd55kks/SHEdmykKOFXniubyGN+rTTXBSBcvYapM47olucHin2D1fC5L9x30x11pwbi/1lwrrZBockdk+9jqKNsXB79Bn/O+Bq4HR8aHS783At2QUJMqQ9GrRGriwP9PZtZENlFaKgUSfjrPHywiKmLxA57j0pftgeCn8SCxnaTewa/xo83Ipn4lw+lcW6MT9nHWxlj2NW9DaNyDvlTXmTSZCmnOx/FKu/imeCfOH9iD3P+nk5ReBGSCCU9t6jJ40vjWeImUkr9bhDpxdoi+iiNuASIHXCuFTD96OcMuvIFrWnjOD7uDCaXnltzDZVBb7LTU0GfTc2+oHQAFHYFSTeTUHl3kK3bTE5rHL9fD0Vv8SH+dDyFqYVoXWw0RsHv6pf58IKGoA8ncC5YTrF0C8mBBq54h3C92U69fBhrB7+JT0o9T6Wtxs3lJ1x4CPiOkpCX+b27kZV8iSQqksYWKXaVFotc7JKriaxBorEiCJB49keS7TaEeT9wNeoukl1MzL0cgf60HSGkF6VUiUPmAEHKikyxCLXj2zsYbXNHqSgk6cwhTLPvBZwMU8OqQAMf++XTV6Qha583SstCzmRVMUL9O495gODcy+8TbZxzZBG5Q8bqC6vZsqyHNmME8zb3sSfDDcv8Uu4tfoOVX03i9EglF4eKPnZldCVqeworg/0GCn29OhN5DjttTVFcPT6PXrMvmki4Pug6xlsd9IlK6HXCmtX3InWqyJ/eQI9dBC9qpZChgtfdOti/9De83MZxvkVLncaOKqid6+OfRiKRsrV9F+fOSbBr+6gIvURd8DmSbyQzfe90mjPLOdB0mbyo3dj6p5F5tIfOOH/eDA/nIrUM9vEiWB6OmxnsjhYywu7gOf98dN4iNSqAxebEiROLysKvy36lz7WPNrsdQSZDN6iUtoBhKKtLGW5M58jYEhRyJ5MXHuGUh5KLORfBO56Qjz/jTlf4rfs7ej06SesQ6Nw3CbfzSkKWTqMsqYkh0VLOdm1lRhNMLLuE5812tNVVeE6Xg8WfErdzXFJeZWkLLAmMQWk6jLK5Gn3EEM5NCqFXc4QXEu/iDp9KsLdxps6b3uJoJPPjKPEIoyXgDwpTCsm4moV62yH6F6URPiac8n3leLSV0TD2DmYNlnJSb+Ks3of5/6GtakMvXDTJUTf6EOiU4Kxxkn05m/LYcnp1vXyZe5R3FJ1M/nQ6nkl5CKNvfz+6MppBZwchLJOw9PelqFxVXG28iuAUkM+6nUZW1tpBW8H9C8eyYe8FLgJD+H+2i8AZHDxBJqpDaroTuiEaqo6I3V4IAldHplMY+Qw9umMIlgcAJSfu/4Mhend23LGD3NO5OKVOMktexTHIQuGmMiKuV6FKkGNyvX0uXa8Oj253bIFhzG6YzZvfi0CSKPkIFkxbQJW8CqlMilQuRZCJ+YdGr01UBX7AINnPeJgzcAsC7ags+m+sGzjuDvtY2vxakdn8sJvt+LSL66xZbcYudyD8W2LfqvFA7yWC5ge5taBxM7H6lzWY5/Xy+BNvD3yuXwCLTcVltRnLmOMsTsiBamj1EJtpLllgSB0sbv+A7usGVCHlaJzTiDCNQVIn4cOWPqIDt1M3Zid9QfEMbvPDxcsFyb+xgsmkMlHCRnDgdIaJzQH/Zt07mwiovEltwiDkKfEcCRNo9LuX8v5LLNm6hLAaJzwDfjWXUfb54NqfSLxXMT/6Q9UTm5h6cznLysOYs2MOfy78kyGKdSh7J/FdoZJXu014KBOY9nMG6c7H+P7BL3E3ZpPQ8B6uEe+i9jpOs+V1Xnr/JYoTy/hr4R9YBBEMN7LsKB69IzFEH+HMpIM0B4pc2WIjQC0my2lm+F/mhyYTg7aMZ2rDZF4aPpd/AaaStTXcf98RHm+ZhtuaBfwiNdD61i5AZOxxCk6kMindXt2kXUtj5JmR/PDYfvThnZwacQlJ0FyEPTXktOeQl5VHLQ7CqmFNbTJNm6bhfKqX4sRirEorrZ4n2NQPm2oB30bwbWRo0RyGnx9OSUKJ6EsGQ4i8ny/efhxDkxPlqlx2pYmleLkEgmRQ6ZQz6+BcgguNzFbmM+JUFpNCOwgzHQTrE+A7nDfq3Qlv96Kuu4z9mbms0rnhJXuQtXP+Jl424R9z++/PQecfnfx97G9SFqVwPuU8nS6d/2gI+Jf9x2KfX/UFXCsPUsSnnMH5//ieDwH2OayMev55Fi9ZQkDAf5bg938vEwSB+vp6PDw8/pec//9zse/UqVP89NNPbNu2DalUyoIFC7jvvvv+Z17bf9n/xuZ0gp9+Om0eYoH4LW+YoNnF0ZlGLkmG4+wTsClsCBIxkRSvhKwuX4L/nk7baDuGen+63cSgrcsuJb7OyS7/k6THfsK95Zv46LmncLGl0uFZzc0x1cBR/jo0iab9w/FcUYKHzoMe9x52zdrFztk7OaBfxFLtMo6Xwl3ZsHfGfuxSI5dLFvPIrUTHz7Ic9l/OQbgbPnv2R4yqVnJUkD35AnW9gQwK8cICtEaPIPnEV4w55E1BArzaKSI0072cjDkxhk7vTopSighuCia800qZSuxMkbZbmXB0Av0u/VzNuEpaQRqu1eG8X7UZqRSMkXJ+v92og8eNaB7y+INqj9dwyuRYPCbRFCzqlczSwouhv1BmfIgZhyaReiUL609GwBuLvJ2Rt9iOKmVpHJFfJ0QlYL/VWdjiEJ2N0SYPSpoFrEYrCXMTcLrdrrjetCiILpqGTlvOn7GRHJ1cP0AX6CoBl/4k/vT8Fp0tBr9/tZ8B9XZ4rB0yql4l90gaQ8Yn8cqkT3FKLYQ1jOBa2jXcnHfx1uWjDN+Uju90NfjevufptxrHnnfJ4VjqF/jbTYy7KiH1ehjJk9vh2gug8kFmlxDUGIRM4qQ8sZJ+QQ+Am8GN+3+8n5aoywjCREqG3YMglSNIxULQtW4d0nNDsaY2oVZ4oLIGYFG2cC3tGunX0/HN/oWwEYPQZKThUuiCQ+bAqrIOXN+X+5/Ar6IJ7ah6lp9dzpmRZ0QazaCJ+BPH+RPbMJzIoeTpIraOu1Wodoqi9A8GpTJM34X3iTmoEuXUV13CTeJGs87AKk8Y2vQRI++wkJxSidb/G55Z+wwlCSXsm76PksQSbuQaiJoQhewCtIakIXE66A/uptYln17PYDqCB6F0U/HSuy9RE1HDpiWb0EggyzySuVtj6Wr3oPx5GVEvfcM2Tz1XNBXkZeWRoHYyZciXbO+O5aglkVF1/qx+ezXrV79DttxMsTmM6m9G4YwqoSl2NIHlp3Bvr+SEQ8ITbeAZNJy10e/hGmrjQ8DO7fESEDhQs4mLb1WgmnmNQrVA9OVSzo4RE0yCxDHQBdwenol/9UUc44ZS2P4+y0cXkbHMyBm20HiwkAe/msW2O83I+xuw16mY4JDw1OdP0ZRjo3C62PkW1/gmLpGv85wn/FbvS2NlJq6ubvhUR2LUGvFt9yX3a7ELpP2lh7jW5894SROKZpHeQu8fj9EjmCA/Xz6WBOEyZT8j4lbSWdrJzS03kd8nxasSHoleQ2bKScotQSjcXYieGE3JjhIOTDmAS78LBjcDs4RZ5OVBRdqdSBw2hszdxVXTtxjcDBQMKmBS4Apm75xNYkkirX6t/PXgD8S7VGAIbsVl1CVsrhlcSBPpj1WSHjJzKnk8ZyWn/hrJ1a6h9FyR8/jXj7N3VhlS7Qpm9b5DpFLKo5nlFOnd6ApwYI+3Y5fbqTO7sLoTHgqKRPDUYvHwQK6WU3emDvf2Fv40wA2znAT5aB76/gU07kp0a3SUDltO7Jmv0XX7cW7YOQT/h/6xzttUrjQH2/Hq8iJ/qJ1YrzuwmawMOvY5Uqcdi9aTKKUTT5m47jgFB55jp7Pepxj/uDQaTY2UhUOsspuzdXfi657OucBfkNpviyC90QlFJgFtQAtmbQxSuw3BLgYu9+kg1/cbPpesxSF3IMgEdITi3h+KtxpYaASHhRM/PY089mFa7U8TVuuK1JrJgakHONd+gLCwh3DKptBnyiY7Lxun1MmN1BtkqGBrnTt8u5SQNCc9w0VveEnQm/w99zxu3S18FmLknsIKymu/5AXPWrw9RJnwYVozcuB64EXMo5vo8RGTl8kaEXzwghc86gEPVyjpCYykyunC1EYIdY0FjrJcB22OOvp6vqdF1oWhzMmS0iWcHXkFwsug+QDYelnTmshV4SyO4p9YuCuajUuiaPArx+AEjbYSu9OOavIYzirG0OPxJGoJPBB6mV86/DjZD6OcVi4mXBG1r9pBYfdC1z8IpbGciOpMLEoLLeHeA1RtHyWYsQnw2jer8UZLvLEZx837af2+A399ONX+n6NwBvPXsi5cpH7cCJg2MI9dTlCYfZHJlGgby/AsOkNnSACzDs5i651bB+hctNZaxj+Sx4WmufgcL+Gxi4+x4Z4NTAwykLtrIv43lNhX2QcCGicORriYedcPNNcU2BsPgCaBe3++l/NDzzNyyR1sv/AWwc3fkHtTS4+7Bcud0znZ0c+g2g3ElJQgsdoI0djo+GQZurlmetx76HPtw+BmoDj7CqE+WQxPeBK7w4DZ+TV96tvdodO1ENM7gy6/p3jx5WnofFxIu5KE2ZnIzUF99LncRIIUi2kUu39Q4DlGx8jaUoIbnkPvoUelEsEVNocDS10XaUe+AaD6rgz2mp3Et/izuOo81mot8yruRn09g213Oynxhrga6PPoRz1dy9yEVGx7ryBzyJA6lShkVlQSqIjMoD5AzvTE8eR5LCYzsRZ6xcSqEPADxmgRUXtyezCZPTPZPWs3/QJctkCTOohvzv6CIr+TYdPFKs4r1/ahCp3GC15ypP31rB5ZwfMJFTh6n6cx3JtG//P8Pu1eyvLj2FSyhNYCb2yhabQ6om6t+U4e8YAWs4oyWy7TZTIkyChKLiL3TC6TJCpG+xjx/GYpNdWBKNZoWRlZyCWNgy19YEMEO52fUs6qJgu2HSXktqbwfehz+MtgeQs0mZM5bb9KoFVHw4zddHp34hDsTFa486x3D2e1MeT+Ooyo8kSiND2Mv/Ewnz/xOdXybrolGqQKgaFrvuaPG2/SI++lyvUkamswUqeLSP9pnozKW0t1ZDVpBWk889m7VAx9lhK5kueqnyM7Gxrz32Le1hlsuLuTatdqVujA0/M6hKTzd/duJu5Ixrcpm0+f+ZRT/ZBYA++3pOH6yyTicoMJ3jcTde4VdgY/yZOJ21noBluP6ij6aQayiEYG14xg06I6ShNKiZJDn8JCk6GU5NN/0RMfTcWiMpxxwOF08UEduZW7FYUsbYziqNnEg8ceZMLYQoZMP8cPJX6kXU/DProTISkGg3cE14a4kZ+Zj9YB0zuDaFk/Df3UPiaoXye5MwqvRDFhdzIEHj2/gLqaVSQPPcbvbzyOJbuUramdrLfV8HQ7OGJhc4+Kvfa7aZMVMck/gi+8N3Az9BkiiWLp6TjspZHIfIu5eTGGkROKuMsNvoq8QrvpKLn705m7eSZN6Sauz99MU1ATV6ZNQCqVAMd5vd9GskcX3kwhpuV1yoJfZ5kbuM7ayfZ2d95yX45f9UX8q8Xkd0XGNR52h8vz/0ZvSCCidRrLNyzn+Jjj/ygkyewypE4pTTlWTqatJ6xd9B1Gu8C1djWerj10KqXURbTQp6oiWJJFvwC/9IIbCo69+AHBbfNQWA0odHnIcWG+q0iV3yg4iKmIYcilIbQHejE79TY1GLEPIeEdds3aRXTIcGIc76NueJluv+EYFXnIHDIkgoDT7iSkPoQ+1z4EiR2bACkeZh547nNwLKGh9j20nfnIdKJe2l4jLAw0MvnJP1nYGTnQRT6o/V3k2pfRSkB6C4hl1kmpzJxPXJIvtX2JhLsW0+YAQaHCodQQ0B6J541UjKNdOWcsYB9XSZIFMLqnl6w9n9ITl8n6uef4LfrGv3kMEp7oqqZds4NpVR7cmX8np3NPszytjbjMUn6WGLhD9TD2hhY8mrvp8Yul0Wc7TxXN57PkLbwQdoVvzfX/8EHG+Zn40Q8+Kmqn2yeFeGsqS7dv5PhQsXDa++UdvDv6JFEjHgTAr6iCI/kTyR5/BT+Zg4w68O4dytAzu3B4+XIhvnSAYmvcQQ/MbtVIJKMGAGo2tUB1ZDWbnQa+qwONJZx3RyuptYPaKcG3pJFl1zOZ8tghMgOLeW3LF0Re34m++n7G7iggPmApvy/7nesWWHvxW3S72nmwzoEz00JmXiZNQU20BLbwehcEGRNYmGpF7hb0j3v2lcHmQNh4dRTnNz5OamYCWYfM1IbXUjp0Fp/F/MxnwId5MVj/mIWQsZF7d9YwakgHVrM3TlMsVrmDZmUWb+9eh0SQE1brS2yQD69miACpPmcwZ3t/4vSQp2jpn4pwi1qysOcsIV0PIFXK0Ol0RNhC8W9PZNvcbXT4dHDIE+JRkHr2FdQZQzkubOQhDzhghBo7OMyhhGn1qDQWkMm4ll6IQyYWl172hLd94MnNWXjtnMKg9YPAKXYCDb4yGINUYL9BSktpOJrTQ+l8vYwdsTswq81InPB5N1Tow/kh/TxXDWoUntnMuhjFOxl1BGobCIz5nl+lS+h170XQdSK5lfTvdhW7nKfvmU6A0gVWgSQinOb2cHpdrlMd8OnA2As4EQTo8YtF4rRjkZcwVQNbAy/SRiu/9L6Evc/Iy1+9zLGxx6iJWofMZzulJneO3whD5SrDNiyMhoTxpPu749pi4UkPmOMqdvy046Qtdh5alY2hriW81fY7n+jhzeip7J+0j+qqNGJbQmkIr0BrTWNB8jsEuZWz0yEDwY5TIqHHXSxGDLx16kmYAlei9dFxKVdNu9s2ShJKEKQCddrz+B5fgp8goWRKFzsDb1NfnwuBKxZ4O7EIjTOAjsA+6lQiZXaeBbyrAMwQU4mLcTQPfLiK1BEFsMKP4rwPcFP20BE0hIgrbVzNsVMYITIeuPS7oOnXMMR/DW7dzbxyyZMzslJMblIWpvzFddda9rdBlNNKmn43OQ11tDiTkO0bw5lJLoxdtp/UglSSbiahn9UL1zp56fOXkCBhz/Q9tPm3sUBvoWXhXyQqx6H7cQ/Di+M4PkpG5IXBfF9mZseCIjqm7We4IpIH2uA+d3jcR86dk0fir2umu/4hLBHJuEmSCegeTKhCXHf0diWWvgwqXCaR6khh8Hlvoisf5uiHN1n14Sq0Ji01b0swmD8k9coYurwvcjXrbSzKZgrkEGxRMEoq47PYcIhrZEaYwDnPHdT5llERU0HG1RwSPWYgU8oYsWoEJ35vxF5QTGD+ERzVDWSv+p7tbSqsxonM3DWT2vBaCtIKyLNAnsXOp3vmYLLKSMxKZMbeGfx+1++E14aTeyaXdY+vY9DVDHodvYwrH0d2STbrnlxHXVgdZZ4HyLFqCV3oR3/oKGY9vhyZQsZszWz2nKugS3tuAHCr8dLw7rvvMeWlVzjgtP03CwEXgZkqFRMnj6OytpIhl4bQuqUV7oKRL43k5vYS/Gsu0aeNRZ+h54nPn6B//ClYNYG++m60EhmlCaWUJpQy9NI8Mo5uwJS0gAZJAz7111Bb3TGhHzjf5oWbSa5bxyiJBLlCjiAV47V6ez5pD6cx5HHxKrsqutB2GzF6BCNIbDglVup8v8ejXvS7BQE6HTJWtsKwlPupagxh74yfyHSG0FHSwWNfPcaxscc4NfoU3Z4GQhovAhMBkFn7UVpE8M2i/AdwyPRUBXzKcyfW4NXmQNIXQ0VsBR5SuLPiNKcjF5A/7iR3x72Nx0/FmHp3IwkVKXQvWUBXdTcjj36HIzkRaWQsUqcoXxHtdYrXhz7G8hY42DeIzZ9d56OzH1H11ycD4yGTSgm7sRe5tR/9+Uks/WMSu2f0MWvWLB7Kfgj5uFDeWO3A1dCNefsOvLPcONEvgjLK4srwCRUlKW6MfRy71EpMs4YC7TJAJJcRsCNNcuXXJRtpDmwmVOLgqaH3EOxWxvo3VqKR9EOEH91WMR+U1b2CZ+L/Its7n4ZrmXzjr6A5IpLKKJGFIUAG/jLIasvD9eIV6gJ1XB5+Hu/eMbhYqvGV1aKTwn6msGrfCXyl0TgT25kd/0+Q93V7HNMaweHwIiJ+BO6qNnokxwmrDWPavmnURtUSNjmMwKZAsq9kE9oQygONh4j3beaHSc08m/odfgWQfm0a19KvYZU7qLfD2/1mRvq0EeMi5dDkQ/xHOxAkgmXbxg9mg+5rmgKabj1/4r/tUXl4eQaT7nyTXVKx2Fdnh7AamOc1BnvvERJujsXhLnBmcQB6ryRStAJuSk9QuLMjsQJLykWe0G4gvTCK14e2InH5ipd8+wmUi2u9qvgqiZcvU+SrRX+rEP5d9HfcN+Y+JFIJN+fepFpfTUJxArN2zQIZMOb//u46neBQuJBPHrkSBUMEy3/jDb9tQ4AxSiXrf/yRV1599b/72f+y/3/s/1Wxr6mpiV9++YVffvmFiooKhg8fzrp161iwYAFarfb/r2v8L/vf0AQB1NbbdG+f6WGCBsYvPEp55RqyXD/miLaf4eXPoc+OIT8Mmh1yfm/1pcfoZMNdNpxSL37Y0s6ZhCGY1FXkHRvMoV0TUC/twxhjRNXn5EighDN9MtbonWyVzsQlzkDauQimlz7Fh6s+JKYihriyONrmy3BYewgoL6C/ImoA5VL0b2tYX5IH5gcEUkYOw9Jqxya3cTEMrp8OZUudL1IXFf3XSkk9dgCjexCliUU87g5l7W5U1QSQ6mnjjyWbsMvFY9vce/DytDC3O4CQJg8sSQq+e+A7etx7sMvtnL5nI/u8usj94RWMbg66/V0IMQxCmR3L3v5tIHEikUj4e+7fqOw+pNQsHrjWH3vgsAm+znJldWw3x7vraTibT/LRXylxt5Asspey2+VxPLtWoXUqsF3qYezJsZwZeQat1EbrhTQm7lHQXdXNtC/EpOznFz8n7+yTPDzyQR4q1dFk0KLy1FIZ30fHLYTuZd9IdEHnGHY5CK3FG61GQ7QhWtTNuYWgVloleDUV4Wj2RogUg8QeTyvByAjtKWZbkpFdM3eRKDzEC++/xLFxR7icc3ng/qQSKVcGXyFa8KLizzzCb5RSM+JJmHwJtBHc7/yeqh8ewDDBBLm359CkMXFo4iEWD7oHQYA+7whiL/6Of1kbr4yGjjYr1Ts86OjZwNTEF4g5dy/XUg+TN/85xt+/g+d0oJDCQ3oXnv70aWrDa9m4dCOPucO9OnjF5XWSGybQ3e6Of1sIKouKgOYAVtV24pdylYcT8vD0r8Kp+qcGIYBecGfdqb/xPPw7LakV3PHpOK6me7Lnjp087QkIFlZ3Qq1dAgIY3E2YNKaB7ztvtdDLZGBTuxFYcYbmoVJ2TNlBaPu9uMvnoA2EipgqWv1FNNTHPvCwxxkOF6o556nH4RLIi9ZWMEFcRRyLd8zBf+lulkbe5JrRTJFFILhJ7FpK6HdjdcYhXm0pZs90b3zc0lDrHyGicA9+1Re4ZpFz0QnPhHphFECNQGdZJ6HnzPgm+dLu185D7v2k9T/DBd0zoFLRVdqAe1s5UqdY/FiycQlnKvaw6LcZ9LsFoLAYsUkllMzZy1oZ+Jy7ybAhEqwKK3qPLuxyO3nD+9kydy2Ram+GTz/LFa93uOtsIj1leo5NqSVEIeoUtlfMoHdzOP6hCTz8qpik82+5pS0XDE7BUxzPAF8a48ag6Wmm06uDjJrTJGnO0eX5Hpd8W0lIzGH4vDCGrxpO9xs7OBP9C/AaTIGNCVNQ+z6EQisWG3Mu5fDrPb8C8K7kXVxcIKDiDF6NN2B29ID2bUTn+7hpPCmPLSexJBGVRcUTb4lOUs3LDq7pfuP1pD6UBUqcUicSpx3Td+XsujYTx6JDHMupI0P5BKdyT1EfVEOf+zW0lmhSOh5hxvypVHapqZSFURZfBoC7WXwHQ6R1fLPAlZ+rHkAQvuXAkweILTGwfQIUmuWs93PHFh9EUHz4wDpenzSKDtkTXB58mfjWc3CLLgNAZuvHq11KYGMmfy03E6WLA2wY3QPp8YulNWoYv3VJ+KIXlnz9IEZfO6yQEtnxPEMDRpJftY9aOxwqT6fh8zF4TyrlwJTd2OR6ABzAm13wvKed9Bc38M7V9zAGP05ThkiH8ng7ZDc/z7NLVEibF3GpbTqKK78QZdcikcwHpx0M5dzZs5m2n+9BP+oU5UMfpl/ewPDg05xrPM2wY3r8e68izMjhg+c/wC4Xg7WPfCDLCBtlUCsIxJzRkVVyH8ILciTqYeRlv8r5+jGY9qShcKmFZDFh8C97zQv2yTuoC6skoOFB5ux6jJKcWqbm7gbgeucg3hwfTa/g4Ok/pjOxdjnDnr3MgfZ7SZH+RI7LBJ70PsTdn5zg0T0rSNgRSX7mrS5ohQ5U3uxujcLcrCdUbufS4EvoPfSs08OzCg1PfL6SlimXETJminMpEbVhxl18nN6+QhaeG4tjlp5iv0bCboE+7/Pu4v35LrjLLWwx7KbAOZ28qGUD9/R8Byht/sSXBCBBoG1WCxPSLtMk98JDHsirXqA3QGfXO8hVarqcGxAkAk3BTXhIIVhlAMFByLR3kI1wcM11Ovt8f6PVv5WJGrGDSGbrZGL6H9QJ2dTqBRwyB/O3zGeSvxHf4Hb2KgUQYOK2CsJj5nB9QRuDNTYGq+GJvdmoW/wRxik5MGU/dWF1uLqHcsp/B6Ndr/H7x3qebQnnqPkuakPOkly3jvoYGQ2K0/yh7MBv/haiYmdx0zuVm8FiR2JCYi1TJW14Wpu5ejqVKp+DvLLtBfZN20d+Vj4vekKi1wV2d5zBr/AIFp0C67ynqK4Ed7dnec0DnNLLuAW/Q1lODSlZntRI76fDu4G9M/ai7leTUphCd2YrUqXoZzulMlx7TCgdUq5NOMrxgCDqw7Kg5watoY20eWzBKoi0qaf93MiI+pX1po9Ye9/tzfAFT3jTG9Y0jKPSnoFUpsbgUkW/RCw4l4SDq6SH3zaOp+9qAqEqBb7NIeyetZsqG0xqhC8zHiGmwx/X/I0Q2orQs5PjXU08Fg1L3OyYdUkkVStoddq4N6KJH1Y8DsD3PTA7ponKiSuIn+0Ht7d2QODlDhBq7kAw2LBb7Iz9O592n1C+eOwL1oT08Ign/BxbTZ+bHhdjP0vlBTzmrmJLn4XRDWIn3CPuUGR2Ib7+HL7NYqG41SEi/sMdOqKLk1GrNORl5/3rtGypeI7ySyu4b7gJjw5vQuoDuTZOTrXvMfpd+gmWw1dn3iQ4dx8fbSggUj4cvZtY2A3sWkCAfjYSjZzauBBUHi68tPUl9sXtQ6e3Ej9kOJFaMwePXEYwChjjpGybu4V233YAomvgvppD6DadQxXTikXnQqe1n7fjD/Lh1ecp0VxnU91yol2UuLkns+jFFXh7GxDqSjlsgm4n7POyop2/A2Pvy/S5D4dQkcoqphaCzMn8pZqG79wQvnMw4IsBEDABPNNJNJk59dNifNNtqCxqOs0Kuhzg3enNiHMjEOL1WH30uHVUURIrp9OnkzQF2I0ynHo3sDq5fkbKjdqFLP/uPTwQfejW7iN4lrbTGRKMXGbjplWF0i5SC/nop7HzRBCNNfGEZg1nw9hRANwfCQ2RUGDuYezTBXzRrkZe8DGzIr7iPEvocBwg36QlWO6O3tPA5ezLoBtLoP0Aq93fRdbxLYTf6lKJmMDpMDsz+wcjKxpCeeAbPOohECSDI+1id7BX0+2ik2p4NV/7AX7NTCp3BQ8le6bvoSlITNC85Q2PuLmw6eO5tJWFc36NCISQO8QMyt4gOOn5G9PvgzimI1jExKJUchvln6q0E6CEy7291PgdQunUEUQ2r3WK8dB+u5GAKWfZP/Q6vhX17NUNZwI5SCaLBcn57qdIn3WNt0zJOK45QZAg4MSqsvLl41+S0fU2doOFletXcjn7Mntn7OWFTnjeS+BCRjdZN+2EHnFl5MWlfLjqQ0xyE4ta4IoZklUiEGvutqn06Drxu0/Kr02wwwhjOoNZWhtFY0ATV9NOYPXxYUrlSEo0YlLtLeE6rlITHfETqPTO4d5REhr3/0ih9gFAgov5DhIH+7LLuxEJMv41ItdbxhEkUxNVGsTyPZMwBivwK0/lasZVPuluI8yninMJKahcNfSfPUVM3mWuTXyOZs8tfK7cSsnJcNI3z0G2tJd79txDbXgtJ8aeoM8JFWYNBWPXcWwIrLVcJ+ZGOeUhIThkDnw7fAnu9CEw7ALtpjAKcrtpG76VbdIZuNXHcS3wPSKFVoJGlWBxaeZE1omBOZS52QnWliC31PG3PommwkAOpvvSuvIP7tOIGoeyjtFIBQeL9k3HihqHFhyafrLdxffea8rrbPFZxAsxXvR6+dLjKVLPNthBoill+bRj7Mx/Fme7mVm7Z3Fk/BFaAkU/MVpl4ZPJOeysXsORBjVRxihqImpolziZ3wwuLbFEtnYidNo5uGArPrp+QjteI2Xjx/R42tiQsIehrj1oeyzoii/QpBtMi+4wL5bMJ7jzLlYmtSE5dxWfTj8+f3YfvWkiXdrZujtpUQ4h2eUXbijcWGdqIvavmTQHNPIRj0MWGEI+5OO5iez7vIw520dzeHIJhYP+5J5WOKLW0r8d5MZSRtq9SS19hLnP/cTcLimTLVNYPOMhMpdW8ZY0CaFYjGvStFMosh7gTwNc9ujEJzMPXw9ffFb64Nfox8awjUgtUjrUHVz/7TBWg5WPqj6iXiIWYfoEeKoDYtuTmZhxEOvsIyB7ipQ/5mBw2c36rjG0dg9nzMT9HPBR8FSDjPoztaQc20mLJIgrWQWE1YXhIlf++2ZFje83Az/P2/YAMeV+CEugMWE8VlkXVvlpctSgkcJWw/Ncj53MkAAr19Ku0erfSmnos8zuFGU5TKvXsbd1HPNMwbTEBKPyhWBlPwvd4PVrgQT+OB9pQgMGn+FIdBAh76dPAFt/AJfliew07iMGgaiRD3HcBoEGSPUfR5r/cYYdicO9IA5nnIQ/lx3DoL7NmBOgvwP3/nScTihObqXWNw+JIOFLH1imgyeyhmCzeoHwxz/u/d1uMBgS0boeITJSwg2m8x9tpTSVEXJfDnWuxD9nO1EplXzc0U+E1o5vYBeNxmsEdQ/Du1HPUyef4sCUAwOMS77u1bgLVqovjME12cKGB78hObCbK2aR5tbh7CGw7wcKr03Bqi7m1PijnA9uYnkNpHT4EF8aj1FvIcQcQwsiJaVNYSNLJXbO/torapceGrcL06gyXC2D2Tl7J7peHYme14jUQEW/k4iqSJrlZtTafhRFXnyV0MLPE36hLn0Ypy9HE3ntBpKMOJBBmMKOv7aKl2ceJs9ymOOHJ1MbVoxUHoTiFsVujLWRWcnXMLxcyU+da8g9lUVJ/DXW0sBavQ2b3IW02h8BaAv+ljq/7wivH8GK9RM5lXuK3XG+vKQTQWB1/b4EUYysp4/mniCedui43q9GJ/EgKz8LjSWWgrSCgfkInHSWGquUyMmr2XD3BloCWvDuFPfiV5yebHvlYwJCh/FgyYucEMS1bvTJ0SRatExd/BmMBqa+SNyvg3AIDs6sODOgs94c1sBnT71K33QbU++azmHnEUZIDjFWDstt4A+0AL+olJy2O1gx/B7CZkXwRtXr3Ei5wdsLxO42n3gfohdmU/LjWaKLygmJCyE/o4iR4SIV9ZQ9j/Lht/MIagzCs9uTsN7PaY4pJijAm4nPxFIZNo7QgjC6vO79x7Po1p+MhH46L3bianClz60PG/34f+zP4WWHyQnO4dQ7p0k8e438KS/Rr6zHoC7mzj/HENj6GTWn5tDyyi58MjT87gPz7G0kSsX1ySkIaP21nBx1krowUee+MagEmUMsZgkCxF7aSNxZXz5aBVJBgdMhJbo8BkmglIzLCjw77uSDFz/g7a4veXRqDgB/90EXAp7NN4lpyuHE6AMscxPYEADt4THc6TaJCI9wfI9fJqTkCBVDFpGTvRunIGHotXTMNwbheNHBi0dfxOl9G7C/p2YTJbndePZmEmmxoTGqkDlkLEpZxPbi7fQYBMxua1Ao+pElxJKaV05+ciTVUdVcGXyFqABRG92mduNmyHNUBawFJyjLIbo0juhOV4QcxYCsECYnTkGGq7Ib/LQ4VT7IZs/iWM9nt56dF5ge3cfuwlDy189CNVegICeeK0lXAHjYXWS3KIz/kYOF06kTdBwIhqzIM0QXDGd6Uy1acxxPpUiYdGgScqmavY8W8IT8MA/a3+SpW/dtt8Xi//VDCEl+xOeU8cywe3isJ4ZKQYLCpkCJEnWPmge/F8E/2+dsp0BqgFsMp07BSU2ujYPpv2FT2FAAoQroiqtkQ0wlb8XPgn/DLyUr4Q57LtLjNvRHB+P1sj/JrTto8YvETg3L/iWdOuIcLtZQMs6fY3H9Yv5a+BdOmThfChzEjbrGwZwLpGfPw+/KWCqqHwTX++HwcJjXhQyxY54bRu749W7qvP4kcXAJJxQTOeYQabUldjtyWz8SbtNtNgU3cSToCGukawh0C6RaX41ZbaYloIVUr9SBz/3Hzr6WmJHcbGhhbd9/v9D3L7u7v5/P/vjjv4p9/4vsP13smzp1KkeOHMHHx4e7776be++9l/j4/3vC+r/sv+w/Y4IAOlMaPh3T6fDZy0GjSFV4syuF/ZYixrgOR+50w+4w0mSHlzvA1D8M4Y7FjL0jhcqfxeT56jETyA6oIq0O9qjtBEVL6FRW49vmi5dciurYCPhrAp8Ht3Fz1mPUS0Aif4Na77PYFDYiaiJIKUqhb95FwnofoLs0iarzZcypm0lZXBmXBxVyVwt8MvY1Nh59k7Y2uFtux7fFly73Pj7o6kN3LAuFyUWssshkqEx6JE4n9amdHPCDz65G0fPHHKThNlKtuzkYnAnAj4v/wEXuZN2VITiOZdPwrkBzkFiEUVqUXPBtIsHThE9WIS0oSbsShHfBXFbPXINrNYzSNROiusgnWjhhsJC09QaJwSIqtNoO1XZA4U5rohdWP39skl7afaqxKWw81y5Saq6UvgV/3EOryokj1cDoU6MpG3aRy2YblyPLMY8czqogt4F5W5m5kvLzTxKjhGm7ZyGtCqXpnnqavP4a+Mzx1iGM8inglcMjadS7UhdkIaIokzdfe5MopcB73hDq/Re+w1040L+cqVsn0edqoDDLjn9rAEE+vawMWMWVdk9cCKLNb81AUWuGVgSbBNDAPFco6bPisiyYc1IDbvJy8B4MQINKS8CcwzQFj2La9knourX8fO/PWNQWusJnokmKQBBA292Ae3sl3X7hhJozwLWB9xf/QXrIEnzP3yDq2jl2j/qc5B5PFsgV/Gk0srsgHM8oI3UxHlSEi5pJKgm4SSF30iWqx5/FU36BIlUO7SH5pF8MpuV8CPLGZlrvbKU1oJWs2ngyOzK5kXIDFxcrn/mCQV5Esb83xUMXMH6ML79dXUepX6nIB37Lbt6IIKckHrPezK/3bcIiEeltPLo98C5V0DW1C+eFSgIrzgAQeMFJQngCJYk/oXB44HSu5cawbIpDVwFwxgwKvYKnBx+h3+7KQscY7pK50Cq10uiQoXDI8ZMLTPh+DE6NB35+LuRl5mF2Deeq0khuPYQJLowde40CYwCcB0EiweAdMaBtpcHC4tOjUbZE05bZRuwJgQKvAFLD21nqbqXRs4etTz7NYHayYsUsfqydiNH1SwDiyuMoLc/D/uMUDN7hFI16kKAwK694SFjhIbBEfxNLky8FY66zfuWPt0ZJdGw2GryYHLUYRUg6uotluDR00ua+l2ojqCrgeHwJ9rtqaIl9iOSLn1MU9iStAa288cYbDC09yuO1L6E5rqE/6h6a40TuclvfOqR9XTRiZaHHZTJ85fgpw5HcSpxGSGIH5urVDvjeeYDjTicRYyMo3FhIn2vfwN/d3MT3yr/yHApbP1ZLBABe+jRi65fjIriQn5WPVWll3rZ5A98rNoiFNqdTYNVHq6iOrKY2Wg12gWNVoXzX2Q1CD3ckZ7Glrgqpwp8+l03c6d5HKHX4VoHKqSbK7zaVQ6q2m9+DoVlawqe/jaP7mA+dd3Uy+vXRHH1HTHBKBCkxqm0sf/pZDkvOYeo0EVR6EYN3BNeH2gmtD8WvTf+Pdd6rqQiVVcrRcUdxyEeKS6RawYnpnvRqjhLXNBIvmZOiCPjK3UShGuq6TmFwFvBlxVqQwsRGCOiWsFhhoFP4J2r/XzZKrWSi1srbtzxSu0RcL8ptILFAnO0Gl/d7kNTdhskHepV13NSfYUTRMSh8HZnckw63XlBJEZwy1M5wrnaJyUuz2ozNKmDHTr/mNk3plsoljE/9g0c/+oo/S14n7KgM9x53egQJYU0WplRMYFf2FcoevslE3xBee/tVNg2qZPGzmwB43gse8RA7FFdeTyUxvwPXQB2wmzuaoLbweeb1fY/tehQ+Wjv9LT4gyNBaYqh3Qp+gZr1pJFs7jpKabKLMNZw47QIIT4KIJSBT4dGhZsLGpVyaVMW+6WLiXyPAlz0CfSoL3gopjtZ23Fu7kQXZcQJN/X68795OQdkoLG1WMn98ivCoJpixmTYHXLZaUNvgiAmstSdZtX4WgXcc5cGYfHYbYQhy7n/0Euea7iIv/CT3h5/DRzIMm88Z3vKGg/JrpA9O4kTTvbj/MQe73M63D3/Lm97wRPRRivrLGR13Bn+5jbMN9xAnyeHyoE14yqDArKIUH+459DhKh4nQUTL2DP6GN954g5J62Lv4Lyoy0pCr5bT4lWFwM1Bnv8KGLn82GY1Yp+5DYlVzr8cILg69SO6pXPwvaem64yC77ODeBJ5Wd0IaTcT06nBo+5AFGHjGLuWiWsmLyUUEe8wi5sYLCE4ZxaGr6D6dzf4OXy5NOs2QYzlYp/fRFF1Jr+6WXuXVj/G1xBHqf4PSOdvR25MZrPPAqhHRqqs84VfDNaQqBb1+seAN+YPLMbgUAeCh92Detnk0BpehHjeBKzNeJ6xwL2P2XOHN8S6olUaOBm9CKr0Xe2ouP+fMwqJsRgH4yKCgKoby6zOwDWkD1e135qoFdtfO4LnMBRTU5vJh12Qq3T+ju2sQyETNEKdDxcEeV2IdMg7PLaZRc+z2+mVKJc0jC8/UXylwDMYrwIfBO83IA84zVA3jmtXsV7ghuZ7IxMJEnCtudxI82AYP0odL9ig26GrQXDlF4s1iSoctR8DJJ3oYf7WWnBOfY7j7QepDylFalYQ0hPCut56vjeCjfg8hVMegG1v4fstDLHpmE+jEhF6FDepeeQo/Hwt/3vkzRknlP9aLAKeDqVvnU5XSTn60AkEqVrMv+n6MxZDOA0IUnfP38MeMC3i1vkF+wCnxuOFwpN9GnN94hlzQoQryQq33ZmjPYjYt+XSgwyOp/hPgaVJ8UtgvP0B1lI4FwXlM1ULT9Zcw56sxv+pGoaJw4Jq6nXDY0UhKSBNyHwva8E/o6IBsT3C3foGbuo4W7yCMOQuIjWvkKTR86d1KtFLGS92wvhf8XJ20JReQ0HaNisErCdcmwa1YPLnvKVqlq5FPPo3j4HhefvsVhs3azd4vPsfLNxMUOl5xZNAzfzORuvuQhHxGaPa9XO05yeSsCg5FfMZE/5NY61qIKNxLRCEYpMlk5RYRFlLP+uc+ZYb/jxw05qNIMLOvWdRo/qUX5pu6iCrXUjyug32PfsvXXm5c6IrltBxy5REcPzwBz5oCJObTfLB+NTNebwR+osYG1y1SHuq1YrIpSfWr4bKQjp9VyUETHDT18rtZQkxEC/XhZXjmf4CbORQ/pRE0YTBCTE7Pd4yjt98Nbd41Em4WoByv4qDRjFdpDKnXfbD2WanIXojE6cQhl3NDu4Ev9TDDRYLdIUHiquDK4CsDcyUF1FIbqalVHHfrZXLjOsIPT6E5MpaqgE94qh1MdhcSHD54yH0YsyECh3Q0VS+aCJLB2VAIl3VQkpfII60u9DrdcaiUSNQSauxQawd/oYef4nv5Pu8tdkulVHZn0TOpB4+q58FQgUpix10KEpz47P4Z3x49h9Nu748CTnSu3hwdf4HGoLLbv3dK6OnUEWEYSmP6Ba75H8OqvM228F2dG9P3TscrS4p7qzdyGyTLqvnSF16tHsxKq5yKXZNpmVnFjawP6XAOYpxnH1lK2GiA+JZ3ubl7OJ0LHPSrB9HlkFIY8QCXQsHsPMsNyTj8Vm6mrnAJd215n0OxUWzUivQdXyyx8XJIDVeC/Ckc6sm3i95HkAhUWaC9fRlZlb+ikYNqSDrFfaE45KqBjrIih4MwjREfOfi3BqD30AMi3eryqicpzX2PHUZotPtQNOpBClM/JLZc9NdWpzcRG3gnLzV6kexZwCOBcag0VbwhJOAuhRCXRn5dcZi1HR5s7r69jkhVrSisctoNN+iuTKL+hD+aWFeGaeEpTwhrv5+fG9YQKCliRH0keqmCHXOluM74Hp9bbFYBgV3UR/aj9lCzc84Bmr22Dhz/uPenDPWegFnIRBnQxq/LfqXLS8w63uUGU20a1qxdQZ9ezyjdeDyvpfL+C+9jkpnZ2gcvRB1n1BpvThif4rvYMtJV8N3F32nZIvp+5VOkrF+5nqF5D5NwzRep6VcagrrpdD2PuzEbKUGUZOXj0hcIEhHkZHUoeP/sVsaOhU67Dz+8+iC2jGYCmt2wKv5No0sQy7i62ACao4cz9HIYDYHnyY+oZWS/iYk5OfhlJ2M6K6PTJqGgZhZTrZlsHvEix/qULG0BrWcZy39eQpdXFzvm7OA6YsFZEV6HEFmO7aiNoV8PpSqyCn2inkc3PMrJUScJDhcBiUtdl3KhLo+rHZf+DdwgYXGdPyZVNSeGurJj3t/85FZHQtkmlJZYpgv7ads5gUWHh1AZVk637iZGl1oAvnnkG4aqVoi3dyWf+LPXqPQUwE88spd5ITZNLU6nAEjIi76TTt0JPuqGb3qg0+sRHF6Q7F7Ajjk7BsZIJoF73cV/se22gcSqVAoNNiXzm6HR7CRNacUpEbDI2+mT9+Eu92Vm5UE85P3I006yaNMiTB5y7BG3p+Hjc5v4bU4AK81pVOcl0xgK/AfNPv/qUpJuNNFUMoPqgE+5e8PdRFVHUfvpm4DAL/fn8O2N5zhd6sqbBbNwTjjDm/IuPuqGmU4XNs31Z1f9Wvp/i8e7aRAfPf8RXlKxw3lxh4Jza4cTNsRK9vwion0aaClJI7YuhL/Cq3hm/EUmzTrJkL8X49/swC630xDSwA8rf0CmSGeUMom/Xn6HeOPjGAWBPw1w9BZO/DtzId+7Suh68TPaTI9xwes0r3vBVCk8Ne44x8cd587Qx/AcF8N7sl/oVv1Gr66XR9TwqquCpj8nIIk0YQg30a5swcsQT0JlDJmXckiP7sVyPp3dg8IYt2kx38TUUT+xisS8TBo8OwlzO0uTogh9fzoJTR20dpnBF7YGOfELHEwbt2hnYxsxaG4wVr6UC6+uZY+fnL/kE1ivD6DbuwRFu8DIM9kYXNtpCBW16iUSWBHxKE1N4VRscvJ4w+NcGirqRtoUNioC3wNGs+vlC2h6WujxjcG9vYL8Baf58vsZBJdE4fqkO1cnv0Ct7y+ASP1/jw5SpQV0OeHxIFf6wjzpl1cPrJXjHa48E2/mVUcXkz6exOQ3xYLOxSEXcZMp+VEzhZXT/sSJBlORiT6XPnoNAgqHOwq7FzKFieiUSvJDgvGI9CA67CGu5V7nSFMr3SUSes0C/QqYvHQpr8Qs5dQjp7Adt/KC7GUcv8WxcJZYzDM0G2g+VYEgkSARBMYeH8vnT33OjCCRqtrpBKXDm5l/z8G3w5ftiz6E6EEMDRqPXA0StRzJv6W3g7oWklg6Bp9uFXLfJk7MOsHw0Q9zaOxH4t+vBbHp3CbSNqaRND+ZPZf8EKQyZE6RraIhuBwXRw5SmRSJUoFT6uBpD5jSvZNwWTRbLw0mzKzB9U1Xjo87fmv9g+wr2dSGNwxcR3dgEs260wDkak14qErQ/rAUW5eFgkw9fQrxb5Vdg/nzxquo5b9yeeN4Aha20Jg4iW3TV+GUOnG/hZDxVRoRkv+mxZaFm84XWWAPjYG/MjVW1LQMqAkn8WokhX8Ukn0hm0tDLg1cS72hmovZ+whvc8dtxLN85f4bVkUbdqedNafWoLQomWJ9EadWg7NDZG4JrQ/FFuNN6M21TEgSZRBUxi56FDsHjmsDMq9kE1njh9ejgQO/P6xZie1aKc+kfMFHry1mzY1mNggT6dSJ41Xq7MOnEpy047X0N7Kjl9HZcDveONoPlg7wSbzB1afaMBY/QMU7DxC+8ChdbqJ/Ht7+EO6SbkpmH0Bvi0HASYkNmiQxA8cZrVtOcf0nqILSaHETY/EARwW1EVC4ZCTrlo3ArDdTN6gOVYuKNr821FJIbg4m/tQImrKbMPrYqfUW94QoBZREQIEFvtCLdI2TDk7CrDZzavQpRrvA236nWXZ5OK4SG/FGK0pjNxLH7YJbgEwEkB3qMeFoqyW6MhoQc4p3aCFGXsdrYT182wMOpxP6DMToboLKFyKWQudlGrycvJufQp8XOBfuJiBcBARV/x6MrG4zrH4Ra9pgil0H0+67nn+3fxXrk9cno1AqODnmJCvvXclzo54b+My/F/v+dek2Wy//WVJOf6Cnp+d/+Ln/ne1fOb//FfafLvYpFAq2bt3KjBkzkMlk/+Mv/Jf9l/13zGK30udyE4VU3EztwNB6WLQpBbVfIbakKvyq2miIDqfVAe91w9SCjcgENaMEGOX8mkU57xEaKG6kheEwjB84mGRmcuXTPPrTo+SNMnApXkxwOqJriXI7h8Zbyil/J5fTRW27/Mx8Fnk4GBZRxOVeT8a+fopve1eSui+VXl0vhYMK+cMAb4QtRWLpR23oQ19q494f7+RU7iledDvGfHkfyYClrQdVcio9PlFoeltw7Z7D4mtKXkw8xvW7TmGKfw1JnUj945A7CGgJwOrSz4lIL/rHawhQJxFX8RwVwZ8w9vhYhl0YRtPqT3jxgf0sqQ2isi6HR8Yf44IlhGB5HfvDzBwyvor76SGYTDKuxlwfCAT/ZRK5Cz9dOIDdDhkL9/CTRhSmR4BXvABnLcpRl2i0+RF7Rw5fRb2Np9bMS17wTlwTP9ksaLw17H9iP4ogBfZ5dr7sgYMm2Dy6kN1R9dw8l8mzHz/LsXHHuJp5la/rcjnRksvY8V9R2+NNuOs3HAx5EUEi4CmF+W7QaW2mR7CRZ/qRtMLhGN1UNIddpDigkRPpN1jvm8edY7pZXVDJz/d+N3A/73pJ0Tn8qeUsMQemsbc1lDOvHOHwiDdJNT0A9kSQqqhS38u3gaeZmziSrlOVSAQx0otufhHPvmEDG1fiWXHTq0+dw+rDy0nPGcmcoWe5aQdJeho7fD6lz7WPYZ89xTcmLZuX/caCjQtonW/jzBBPCiMu4tszlbXsZ60eSsLsWBH4tQf6VKW0ux/kZlIgw88P55RyCK95fccNK0hv+pJyMYPK6Eq0GivLdXDS3khJUhi1sQaadNe5MqGIRlsJ0Vb4uz2S2AlfYvn+Y0afGoapw4TNaQUZhHTcjWv7JaL+cqNhegOCSSx0OORKZHYri/5axKZFm0gsrsPQ2ItEuL3s/2GAPww2tP3xuNt9yanPRf/LC7y+aiOL+l2wWxX81TCbzMsOun1MmOpNxN2IZ83qNThlTs6Y4QU3K+/7wgqrEXX+GiSCQEUiLPcw8Zg3WNTNlLaHUFkVSsTYCCrvGkxF8Acs1UC2iwPo5X0f2NLuxC7mE1A4PP7xHAtOgbgLv+Gqb4BTsPM5L1ISO4k7u4PmFhNeoxP/8flMFQgWKUerVzAyGKR3L+Qdq3KAusO34y4uMR6PoDZ8AnyJrAyiQ+lPa4AIs6r1/QZXexneIZkUcJkSr5WMOjmMskQTq5b9RJxzNrs6NnP9zxF02q7TkxFBV1U3Tfrbxb4zl+MZkz+InqwOyn4twMVbw4HpBeRczGFEzTwiXoqgrFIGEilWtRuKl64wOmc0Sps/6WfXoq98iPieR6gL+50/F/6J1Cmlza+NZ9tnICtbg6H2Hq6nXafDpwOnajDVETWcGHOc2LJYvDu9iR81GKmwgyZv0aF9L6jh/2LvL6Okurs3b/xTXtVVXe3u7k03jTfu7pBAEiIQEuLuQtwT4iSEBELQJLi7O421N+3uUq7neXG4mzu/+c9/5sU8M2vNc++1arWcU0e/sr977+u66ND/xPGtw9CZvBHSHUTfjObImCOkdC1iQOJVLqn6sUKxlZTIamQqGUnTk2j9Xc7bvvCIl4WT5lA+vPEM6sROstvMhJadpMnlwLNHw7Rd07BrWv7xHow+EQB42gaTvsuCKnUFLHqGK/Gzxf9b0vGTu5EC3cvWcdqQxrjTBxl1QEFZmB3EmA2DY68S98FVdhseYNzOhzGoz3Nq+Ck8pXDCK4vTPUHMWONLnMITfc9RTAEXIQo0EtBIHZjDn6Iq8grdwRGcGvcdBYrf8emWws23AfAJ7OSbxb+wWDsF18HjmJVVtFtEpPKOmTvIMsczyi5qd5k9zBj0Bq5XL6YtaRsFTgsgkD++nQ3j1/Gg9iNiixrwqRjK1L7FvBJUh1MtwTerjIi4elxI+aXbzYpOmFSXjNehgZAlcGP000gCxYVPqhKqETjZ7EOfxgDsT/3Jc2lV/O28B7U9goH2HNK8/emySsjOy8Yn0BePtocZ5LEdql+DkAkgCCxPO0Xdopts1YT1vpPpWrAqLXz68E9sGPAKXZ9eI+HSWa4mC0iBMKWBrug2Yr5/l5u6P5BeacbqJ84rW43ix6/ND32PnmG+4BfWyjD/O0nQeFccEzLWUigkYROkRCigk1bK6CKzGpJdWWR+GI/bT8eJEXt69bOOmkV9sBFuKdOrAzHLGrhPlsmDQX/iIwOL5wjShr3Mns56xuwWEKRtVKX7A1CcVIxRZ2S3AQbeDmStfWBt7zWNr62m3VSKs38S5fY7/uuwU8NQOpSceAJiFJB1IQC9W0vqxVBiC7P4+YndhBYpOHFkJC5vFY/xFFnRA7lw6yz2HjctGj9sViVHB5zjXMolTr5wg6EBj3Cg/8becxR0piNzq3nPcYEnJhUzs9yTW01/UONdQ7EdoithiLYvD+k38HTuFq5JlvcGAQG6vLvYPH8z3425F9PtgGV3YCKlEatIktn5tUPG6moND4VauK5ciU0pFgxlquByJHxU7EHrUR8UYXX0MfTBqrZSklzCLhO0ljzDrfoUek5qCexvZPGlxZxafAsi4Nk20NiCGOO3nWo/GN0Dv3qLF5BmHcTL6lR8LfsYmvwIao8JVMp/5Y97/uBS/1a8FXDMaEWCi6hOH5JKk7BcMzK9eDrnB52nJagFmVOGzNZN/lUHEqcDqdOGBAEkbubqYPnAGr5yn0eme5pNCzYx5685zNo+i8/ib+HoHkmcZRBul4y6mG3kjznCSofYX5fqwWRKxO6ZilbrTY9uO2aZiBQYpIa//P0wuEtxP9XBB13xPP/VcloTPVg77Vk8peDheRWt2cHZCHixycrh5juL8m+7odQOb3fsJKrgGkZzOh7uQcjbqnv36XepH0PPGui+v53i/deIfD6dtzWz2dUEd3nCXcOusqvHG+j/j7EyTQnd8YtZH+MmTTaTfiIrOlIpPGx5mDf77+YnWwWHzFdRCQnI8/L5ugOMIXt7j/EvKkG3xIFT3kO5TdSKnaeDFHMPa+257Gq6zkiVi8qYSlq0RgTvDBEJDDg1UJhWSKBViq4ZCowpuNTwe7iD92xZqNQBqBP0tETmEFhzBU+DJ5rbbXIxg5CpEqmMq8OsqGOMRAwuFtth2eQrBE/N48OejZyUZpHlc42+qlv0dcBDUT+wJP95WgMzCJBdxmLTcNC1hGDZr4yrB7tLia5bi8biTYHfdgL8Ckmjkw/84IduaLHW83eYgcs2+LmmhsSre6jTh8DwO8/3Lq+nyK/J4WLsUpqUEO6RynBNHh0lCWSeDsJmsOFUeeJbf5PYq1tJODObJ1/4gvJv3qW/TYP71zuRDi9TP75vj+d09Wpa9YcgVsKc4nLCi0/TFioW4vzSA29I+/BoiZybqqkc9d+BQ9ZDkucILCV+mPz2chAd51fMpp+nlZGGbA6PP0H9YHHOUkgAfSpriv/m8q1EAkrXYPSyY6zsxrvsRxAE6kwHWPXySkYMDOdk7HHsjnJsygYkbgk5V3LwVrrRar1ojO9LRYhIQz7ZAza0eHHr+WdISffh0AI7V0Ou9t7bV/5QbFMQWJZAU6SblY/9jkPexdvKJBLUYPF3cFWhxP7Eb1jaB/PSJy+RN6KU9fdtByBUDsdCW5BMOEqZx0UMkny2104CRI05iaDmnKQf65r+pI/Bg6DKCxi8FHzxyFTsLg2CcJZhWaXUxhXR2vxY79yglMDG9mLOXnsVQ8OryCJC6QgTqSvlEhctMfBXQB2PRazipfjhFNz6EJuiGRCDkBJBxi8X4qgqiIWJheSlHELt7o/Vpz8D3/+W/iEGPnz7QfxMHmif7MJX3oDCVU2sxwBK42BfTwNvtME1o4bxB0chYOTQ+EN8vWAdKiSsUSzleMBXtI6xoecwRzeOY/7EAhYkrSI7+CghfucIWXyAlqYxqNQ3enXUAebo4MXblfZuiZh0VdsiGOlTy/cB8FFxIEXuMuZ5JlARV9H7vbs9YZz6CvfaQvGxq8jK8aV/9q8MG3yGmWcfwOBxk2S/C4zxb2Nz4d3sN0GWCjyS/VGGNTI09zQh/jVUC/BbTBvL14kItd/y+zLJE+KithNmz+bs3Ossramj9Uomi1xt9FfO5ZGcJ2iXP0oZUdwcuwZNcDTN/ZLID1kBgL5bj+7WNkrOTKVRI3Ap8xemb0vlwTUP8u7y5RRJ7dizT+EZFsiVES1sHLmLgO7x+Oh28VIbXGnUYMnLIWuwC5cgFvb6yyNpc4qImcJwHRKJnhe3agCIqYxBIkgQYmt4M7UV66Y07MFLaFsfx4xfJlPyYj5qTzOrg+CScI1NVf4MOZ1Ld1w3N9OK8ZTZ6OO4xqAhOhp8P2O19U9ismx4D/Tnp0fvrD9BRIIBCBYLKnMnEsHZu60haXTv796Vv9GnW8rRMWAWwOqCrMJU0i/n4LqrE1RiuxQkboaLt0FVURQpTWpUdTdIuXgBy/QZhFd/Tk9LOd5p97PziZXElMwn/ehX4O7gWL8QHoruZmbyV9S7/Fld/igVUQK2vBuEFzRQlzwai8OT3659wohHMtnaPpxglZ1Ze9+mJVDH/qHTmKKFUN/9CO5xGM0GnvnqGby7vQH4rEvgLxMs94N9nqvxL3ga4Xxfho+6Cre1PO22IHb9nkWj2YwlwItKlVgkl6KEraHwXUMwhsBoHDp/3jmxF29vsCcvYvuM7TQHNbO2x5PdZiv1ET1czRD9psDmQBJLE2mKViKRSHAoHci6BQae64tHXDnusGpqndAiBHDF3cxNdRd9XS7iFOK1gojiBHDe1oGQCFJ6vMQCrI0G+FwrI/zsQDpcdmShUlJlKkZ6t3ErsZlx8V1UGyIovZRKZFQ2UQteZXiQkYm6APwCW5haks6zew0obqRwbfRh/szdRX/tQrCJRTdeTjl/lETh2+5FiuodJBIdOZOa+arjD0KaHbwR/yQ7aj6k1lXG3cEqflr6B13e9czTiX1UIjjwqKygfU8gkhQFZg8zGotYqaU1aRlxOBpLp4XqI7fwbhELmXbPbaYw7Qi1jky8/BpYJpHgUqgRZOL32l3iGPwvb+bArQO0ya4Bombjr899xUlvCznXg2j19/tHmw9uCkZ/bgiHRxVhnrkWJUoe3vcwV/peYZ5rFVKpmszq73jMsYfiu9bTN/0xQnNCqcyaQHNCMyRAakgeF0MWYHI0c+8j9zEkdAgnHxOTNHKXhMJfJsNtcqoDzx6gu7QZCVCaUEqbv/gydbepB2sPFBJRHUi73y0C2gJo8/wTt6YOt/shuqq7cNaYkbhFx8S/ZywJDW+ScHMH/i1H6bjrKSZ+PZFLu6TAZ73jh/dFbxwmB5Gj4ylLM3Ey7U6c5PC4w3SVPUnYoAgC3nwUW9k0PhKXHuRLksm6Foq3UYdLuOMrAuhMOtIKk3v/bo4bgk0Rycibr/HO+IkEqGq5+XADfyWNoKa6mwY/ERJW1jGAso4BFPkdZVazHy6DA7vGi6aQepCIfugCTxisgU6nBLXbRXWsja3jP8HfkMvje2/Soz3PiZGP4c6tIGZ9DGNOjuHiwItIXRrcMgsyqXh/Am7cbhBus5mNXSfquy3YuICI+u8omfkSqkV3ceTaai72v0iqZB5JhTpMVRexTQgh7cSPeFZNYc39XzPOAw6Gwadzj/B3tYTnL2l4/eO32DJ/I2WJZbilNk53hVHQpmSdcjIJ5+IJkAzkwqALuIF2N6C20hlfToxnCykXm/BsELXNT1rgZO8ys4VpLjstRjVNVhkKDUzQQrT/DZLx4MXR5Sysl+O/ZxBvOGfSf9lXwEIA4jSbmP3ru+zo2snJbzX8vG8B5YvEcUd6W/pB7a3m9MOnSVibQExlDNsHN9PRpWNzYSrGOiOC2y3ONxKBDjf8WJ/BbFcLsy+kcHGElcybg+nyqufkiJPsMEF7/iSKosLoinQRUVBD5rG95EV6UhUDj3qJTCTJJ3NprA5i/ay/KJmzFYAQGfweDFddGhaWBFLfLuPB7kYSd25l0v178Ky5BBMugcKT7Zf6oduQi32Zim/SrjBLr8QLOeoYOUp/sQ1KJCJjg8HjTpHh5D2Tyb6Zjf1uOwHXArAFic6JQqbgv2duN/jVXkNvs9P0393rn9YMeHl5/Q/3+7/VZDIZcXFx/8fO/z+d7Nu5c+f/m9fxH/v/mNlddqqCvuv9e1OwGAj5uCGCSomL+it/ElloZf30j3r30TpNeLfdwFoTheVAM2eahnL385sA+KwTxnu4uVsPp1r60JAQiYcrkefDvoDlywE42WikoiiKE1pvEHOM1EXUsT7Gjq9sEz9Yt3Jc8h1TI+2889YHAIz3gCe8QNd+hiG2w9SeSKBu4mjqU3LpDChFKYErY4+SVpiGsbIVoqBs0H3EX9rIgKNqqrLW4RwxifgMDUL8COoe+Yjg4PtY8+Aapu+ZiiS0mdKJszFqJpFSdYuFv+rYMb1Pb6XtqCv9CVEewW72wD/YSQoljO55jBbXDzzcDBZXAv77M3Aj4e/H7zzPT/1FxECFrQ59Uzua1hrcpjuLklCZyB8t1w3nvQh/wlUpxHlLaQ1spcst0rEZ3fRWzV7//TqqNBXP2Z4DoMUFq5KvkxctY6HySTp9OrGpxEkiP+pxCt1Ktkjt6IJieEWTRolvN0hEfn/fq0OIMI6k27qeWJ8e9LOn8czAxXAjHN9rg6gKbObAlbG0bpDjHnIW/s3v3HDmCGqlG78RF6muKUDVrcN025kf5VULW3QweB2DtffSVHMPGTlwYf4u/nKuBCCt+h2Szq2l2wxCnxFYdAE4lR44lGKDmBNYhq8aHm8TOBawkqv++wBQW0WRw1bvTrbN+5PMxAexttYDIjf5v+zeokBUXUqG+BqQdu9E362jLqKO5cuXo7NHsu3QWORHctg9t4yrCccwaU30uERR+j05m/leHsvXnWJ10L+qbfaYoEJZyeH6X9g0/xJz+pTzcugr5J7pR3NQM6VJv6NVazkwtYPnRsUgr82k+3IZCquB+sRR2BWdBDcdI+t6OuZ2M5kXe9A3jOP8IDHhrZHAS2cex9Tlj3yKP31HX0bv201deDelfVIwKuDyUw/T7Yijy+TEoGxj7OGxHBlzBEHuYr/BD/uX45EHKHFKauny0nEl+SNmIAatLJoUHBOMuEZORuOjwcdvMhYPC2m3USbHGsK5ciwWZaSJjtI29K3dSL1Fp3nLwhs8qP0IhUaB6zZ9jjkhniOCjfclkLPwACucKQyosDP01FCKk2tpC6hmRQBEOSt598YrVBNF18CA3kSfjxRydM38ZjlImfwsTzZH0+9EMfKuwRyYcICXP32ZuP6FPDnFTM2CNUxvCuDe30cB0BQo0qFIJBLWdz6B3VSNy97M9XXdHHvjGDfGD+J7r4Vk5KSiad1LWmEa1g4z0aOicfv4onIGEFYfjr7YTENdA4IQRsmgRSjsJjJ6jhNieJEuryZqEtzYNS4mnliAT14gp3NPc3jcYQDiWkspLI3mVm0+u6eJfPbhbclYBq7hQR3E/TGX8ovpdI4qZ9j+HizjwqgPr+feZjd6RxeDKrKQtfnSoGoh9mIsJ0acoLJ2CffULuG+Jd3k5bxBXk4eqwNXIwjiYqDdBZV2KZdtNgqOleB1spun7hlP/ojHiLm2lQdXDaE+tJ6KjET+3WxaX5xyFVJnJwI2nIIRl83J8BPDaQpuolt/hXfNz+It+5JdJrHdd8W52D/hSC8y6g1PL5b6dROhgCNGO7GFUTixc2r4KRRAtGc1hs6xdJzT0R3gjVfrKbrksZRHFbEnFHIifqBSeIITEb54yu4gAVwoYMhG0ASTufYV4AKCIBB/9ShSZw8n/i0W7xKc2HpMLFu5jIv9L7J3yl6kbjVKByjP9EcvcdMTLbYvNy7+mrOJqbVxTIpt4HODhksOgaKEdxjV/1FkuHnES9QaPNihZ3JDAJJ0NzYPPff1E5OPwzTg1f8+3o2I4OC4HQySBXHEAq0yNUZNEUcSC7FJdXxTlUbKzhk0ZJgwqcswed+mUz9/P4RPw+XpQJ97Fc3JaOZvns+JESeYn93MdC0s7wCHy4ksM43SzgDMHl/iK4MjQz7ll9ZgHmyGF7QCg57aSLISqLzzPPpf6s+gC4Mofltg/5I1nJeCpFvBiUgHMms1wZe8UVg3Mez6MD787WEcb0ViE97kph1S7Q+wd9hy7BoFpcltvcmpHSbxc1HhReaxfgw5nY1xVAn7TvVD+2QVyZ8fBqkcRcdeenJu0OPwIaF6JOMKlvL3nN94OdhBaEEMXZVKHBaHmDCTuMlxPY5a2kVmwHmWB8p4r1VKQEMgj/74KEfGHEHXLxjBDEkKKUv+WIxNJ+fYxOtczDhKg18x2d1T6PaKYIA7kbqtx3C7NQSXnUUwtuAR60FIUwg3Mm8QpnGi8nDilv1bCSRwMXEiKnsIDYrHeKEVau0qRv9RQKCunRvxIoonQ1BQW7uG+rcG0j7kVwa7slBYI9kzdQ9WjZWi1CIUYV7YbxrI3v8dEreLLa/s5XCHg+DGYJ756VEsU+pBfpbJDZM5NO4QLXIHX3fC8cS9XFt2hpFBPoz/dDytAa2UJIsIuIsJU2mRRxM9LBRpdBzXfW8i97tTBauTWXgw60UKW4dS05JLqCyUhrAG/FTt3Jf4K6W2YGZXBXPLdpxJXpXcSrjFShl4dYlod0lPIXsfPMEXo+04Sh+g79W+FKQV0Dy0hfKbUfzx5SLKNdexDBlDRfCY2/Opmx8DwTO4nHOedcy83aaroquIqI3ASxLF+IYfGWl/gfr6SI4PgmsBp7ALgACf+IPVp5qqJ3dzvvVhdD0eqAQfOn07cQpgsAZz0x3OhdgD3KqJJBxfvF3JyFHxtq+N5+O+IF/4lReuB1HSEErf/ERG1H2F55dSVnV9x08zf8Lj+pvMe7mDD0pnIffMpNx2AXhdnBsECVI32Bx2Dn51EA+LBzlTc7jS7wp/GaFywEnqbcn0u5DD87ueZ8v8LdRG1nIhApQSNxes8HNnG7pb11C29dBhT+SUfh2/N2QyzOtbkuz7+enWTHL2dKPoG82ZmYd51hsGqEF2Q0rN5me4FqRgcv0z7Jixg8rYSn4Pgnzlaq67U1n8y2KMYRLyh+fweVoF3mfngrkWBvxMvLwblRqk3W78a/J4ps8aYgLgl1Y5lpsv4TfEF6mvipqMKTRGh3FxwLsUdsGLMh3xef2ReKawwMvEL2FQ2BNOqrqOz/3hmy6BBpdAolNN0ZkB+OZ0kJpUw2N1cLkrjrFTBnPZqGOveQA+EzfxsHYKDf6iH7q6NYT7ty5DX2egOaKFVPsYlnzxLahhuAbK3Y38eWk5tfsUeEkctAX00C/4ONRehIq1EHs/O07eheOCk4JHTnKg3ysEEs1zbaAfcJqeaD1v+HugsPQQe1UMsvzL/+4MCsfDGkx8s45nVyznVooWS/BCLsVP43zCeHJLRAYF1VADV62pONTeaK0JmNRlFBqDkG/JRpFuYP+0s5hUFXzr9Qp1lc/xSuW3yAd/Td20vXjYYlEbKmgKacFLGMQHQWIgD6UX5fs9CLv0t9gXU8oJL1snFnJkf4rspByXwgO5RsXl7DpqdKIvIHVLmbpnKjVJAoJAr3YpiMihaUojT8cOwisqTOww/2ZPeoPMp4OfvnuPy9YXETrE7WWqF9B2fMwLEde4xzSKDf7VZLqyaQ5qxq67gwr81B8igrqoC7nGuJMLyT36EDef6wA93NUE9ypXINNVMGnvJGw+Sq6PeZY+ieWE6UWqrxapB6m3+tAkzWNQD0R2R9Ic1MyPUTYcBSrM3Rpc9n+ikgTcFNhhmTdM18EKQUCCFMntgok0JSyNOsnhQ1GkHc+lfkAZ9d7foLVlo7LJeP9sLI/GVeKTcwGVzZ/puw7y0dXXeXX1B9h89vBbp5qO9vupWteGWgvxtYm4JK0cGn+IWX/Ow6gzIl0op0V7k5YAK+FdemLO5PKI3p8FARtpM7t5RB3AmmufIggCHt0XUF4fwqpcE9F6F18dU3P/3mhq7qoh50IQzf45FKV3YROg0QX2S43E3NpB1NTfGH+1g6uxC2n12s87JdPZ1LGYv+bNQO0KZKKuD/3DL5On6CDTns22tJv82NZF2r54wj0uov/6eeZFVWN6tJTiZd9zzh3HCwYXUdeGsk2p4N12mKL1pFVZxLPe4OX7O6XyyfzcDdUGDXP+GkHdIA+WPf4zcqmLbZ2TkMiknBl6hlS5N4mXn8Uq6aIs9F28ur0Yd3gcNeNqMGR5ci39L1LyH6AurJzCKFBLIEv6EbX2eO6VriBOJvBHyG72mcW1fEyDL/cfnEB7gIvz4wZzNeoBcIrMLlO1cO38JBouRVGetIvk4mSOjzyOp8ETSUUkesc1dq8Lp2BvDwNfV1Lbrx6n3IlaIurnSiNaodGBvC4CW6uNIJmLe7zg07j5AJyVGcHrYbS5IPPM/0dbC68Nx0d6m8YzN5cb6lwcHm/2bm/22nO7mNQXfctlvJpTODrmKAEyeFinYLpax576cJwNBuZeWURpUh03Mo7yjq/4/VXfzyRaJUOSbEduN4Hbjd7Sh2yvJt4NN7KkGYpw0eXjwCFtRUo415rGYXXq6J/9F1krfuTPvOUorm4ivFLFiZEnGe+cQ1FrLkOCc3EpQWruIPZGCYqUcfhrAvjYrxX5nCMMTjZxX/T9mLQmvLu92T1lN0hEhqBT7y8hwuZBzfNuSvt+whqJAwQ4EAotTVou3AxE6W7G6653+TlQXNiVOGBBIzhM06kfEUeQTzORR75lck4hN3wdVDWHEWj2ZauuEVeDL2q9FLzF5xDYEsiIkyMo97tGpNDKXe1htDdJGHpkOD5SGQ8kVbPOAEtCI5C5CvHq8kJjcHP3X4u5MDSfgRMv4N3pjW+HL9ZuK+U1p1F3HMJD54FZa6bTDcc7BhH1XDXKwHcY+Uc5yfmTeX3Ne7z+xZPs1ZsonOfEMfVtkvveYq30GksiYHmPkbyoQFI8oK7dH7+SJG4Ou4lL7kImE8OpP3aDzurDiGuJ9L+cQ0OsDX1nB35TonEBLlMqad7j8Gn5hgd+fJPW0R20DWnHIbcwTSvSpgq4eDKkGJ/pzWgzRnNcsRq5Q875AXsYdWwU6TeDsRvtNA2fj+T6NyhsJpJdBi7FwwteN/iiCzpNHXQpfsPhFhPkX3SJnw83vYFG4U29q54333+DbXMvILM3k3Uti+0ztzPoh8XoU4zcVXQXQ2uGcnrYaZqDmilILaCg+ThB+TqUWi8ujL5EU3ATBqmCZp8dTFPrWJW5nr3KLLoicwE4nygmjj73hwciR/HihqFEHb6bnjE9KBOVxH33DFff3oG+vZLMo1/z111pzN08l4ghERRsFpktTg89TWtAK3P/nEvXkKuwIJvLb2wnRx/FheFxnBjxGh5mD0btj8aqquTU+Xy6VuVR9PLHALTpD3MiPZ1Kz0hC2oaTpfPicNZh9nTeoRM8PvI489+dj0eAB2YLOGV3GHj+ZQqXD4JwW6/sdszHIVFwUdKfLfO/IsE6n+aiZu5fcz9Xcq6Qn55PaUIpnj1euBwuRE4q0DhEWOqaa58RGvMxl1OvoErKRVp1J7lyLfoBsqrWIGgG8+ejtXw69QEkn1hQ2qRYNRDcOYtlldPp0J3mgR9D6UwUONtfnBfdiFSZ7w16kec6Hew12Bm3Yhyvfv0qAKGd8xFw45+kYu6fc/Hv8MUa0UL6jURKE/Mx6UR91NLEUjTS+ajb67Hu/B11XxdWjRWZRIGmp4nuG0XYTZNoihtCY6iYtG27net8Ia2Z79zdyL11mH0ScWnEvjEy+BgqZQfb92UTbwknOT8Km8yfC4MuoJJAgkKMW47QwGNtdjJvZiJ1S/+BSAxsDiS6KhpDkpO3nvim9/8f+IHWZyv7+ZBHW6DMrmZorSftHf6kSbb37meVhHKjeQxmmS89Ahy1utncA549ngQVtNNS0IJHggcR+gjib8Xj0+VD+bxzDMopYc9Hy7lnXhlpo9WMv/42Pz55jviWt9ljGshR+UzSt+cgxFs5MMuL4ogPAah3wmbVPt4x/IDbHk1rWD35uafp8hYZk571BsHpwycXYwmqD4dxd/Te653w4sUPGZozi8orBUzcMxL7M2Zu9rvG/RG3C6tNlRA5j9PWLdhzGgmPaKHLDt5SN23E81Z4MNlRIqJO1tyAR/M51BY1Vo2Iwu/y7qIrpgsksP6D9VR3VRPQEoDrFxfN+maCMkRpm/9K4+nVXEoM6ayWnOUe4Y4P+N+z3zUaZi5c+D/c7/9Wc7vdtLS0EBgYiFQq/R9/4X+x/e8/43/sPwY43c5//N1+e5J4ZeVnWBZdJ29YEasfWo1D4SBZAWfC4b0BAwi/tI/SU2vZP3s7gyeLotkzGkSaz/jaMGK3TkXicNOQNArfzDBSlSLMGmBrqZ6qi7FEXzSzcP1ClDbRaT9gtdHCKCRuCSqrCsHpRpAKCFKBMDmM8wC1rZmJfY4hnXgCQ5SZxrixBDlXciwMjnl70NGvmoDRGdhrmwkqP4tHdyNyhwWpxMU7J/bw0emtWJ1WrmWcEbXrACLr6R/XRKBaHLSl3p5UJqho92vnWtY1pGmlFAU2kbFvHNmnM/jXwlwl1dDjFquH8+0B/L5gAxvv+mcy/qYN/jIAUhWDlR8SUn4Ge3Erw04Ow6fDh/f9xefSFbOHfuV/M8j8PsO105l2tp0Flgs8GejPjKu5zN4QSEtBK8/WPkvMdzEAZCpFdMh6g0i9Kov14NfFv1KYJmoAfOQHHwfeHvwlYsDBJb1D7yLgRmOU8uCaBwm/oibCt5QY/3wW5Vxmx/wtmMIa2WytpVluxWQpZNTRUYTXig5SfstIbraOplUWw+cL/2DbI3n4fCbh1Q9fpdnhBbEPgC6OYMPfDM97E8uhIwyWP4F/z1h8DcM5lBmJkxLcFhuCAAUjH6MrKBFN+wkSMxfyW73Az0tfI2yPQLtQ0nvNmxet48q0PWxI7ebazAIs/m7Sz99gyJkhtHkdYoAKluhBl5/EuFUPYTx9luGHFUTWRCJxSxgukzNA5eRbuqkNbqQrwEJ5fDlOhRMBMAlglYZyz/EMOp56i7YNpUxdPYJB5wYBUBgFoW3bKHVFozW9h0QiYczRMaQUiYg2k85EfZQFz1BPBAFq0idh9I3E6hlAY2w6p4ee5uOXv8Az3o/4IgcxFWJZ2hK9WIkV0XoVfWUxitgIftdvJM0sVi5ZPXaTor/A0YweRoc20xbcQUJZAkPODWGETIErAaZ4dtIlc2CTwV8LbqKyqZiybxo7aqcw/MYEjN5T+bvoZeqMGbgcLgSrDalLygETWFwqqlr8ke4djbrGSOGaiyRe+AO5XRwfdAY1166J76Apfqj4UxOOkiweaob3Ui/RkNQDt2yMPTIWfY/oUH/RCRaXlJgmGbRVcq3+SyKrRdqJ4RrYkX6YUV1tPLRqGsarVZwZZeJq9jW8zOIixSBz4P7X9Oh5R4/2cr/LvOIDM/U1/O27mU+efZNbObXET4inOm0S1TE/sE27gQJTNXvvO4vvd8vxSg8h5/mRbHNNJ7f4LI7ILVye/BbtPe2AgMxlR2XqIPipezCGTOHEwHf59Z6l1KvqULWKyOWUohTuOzOcVcemsnvgWVa+txxT3J3y8FbddqYdGse8W6lMnHccy9OrMXZ2ENjs2ZuoPmqBeqs/77+5jr6vNpDxegafP/85Fs0dVJZW0shvQTBDK1I7fNN/HZmnjrGqNZjx5bGY8CWqOorASh/kKjm6mAA6/UVqgjO5Z9ANUP9jHLJpfekIyyDzqherF6+m7T4ZLpuT0cdGk1SShMYezo2GGXxftJSQs/2JvxFJT5iL84PPY/QUF1xvBliobktg7PqB2Mu9KRw0iV8fEhHKHW6Yvec810yLsM3uT03mVNbdu46LA8TFQWHVI5yovoej1bu4ljyK/MBXib7iSWpBKhKZAqLvBt++7NUV8dGVoSgqDBT2sXJq2Knee0i/mU7sVQVuJRwdeYzSRJES7dGRubjNSg6unUJgsYT42kyGnr+fMHk87VoTa5Nv4FkVifGVl/G+aKDekERm0PHe43pJoT6hns9e+oxzSds4E+fHCat4z5O04qLnw6AuPvUHd/F0jh6YR7BUw62QD/m420Kp8wFyA66x8MU/cOpMJFzaQKrqtzsPX6rk2fqhzG4EXZeKlKIUtCYtL7VBhVnOy1cHY7xWhzQ8lPaILDztQ/Bpu4v1FbOpy3+LWYdPkG4bwqxGyBbjBeSqRbqRZSMLsczdjUMlUGKHamsffKw5NDqh2a5j1sa7mbN5LIEuB0kODaMUl5AiJtrVUjfnhpyjIP0W/zXYDCBBgiy4g5qMfKx+ciyhvlyVPw63K1H19lrW3n+StClnCLypIPdkKH3z+rKwIplpt1JJPajFYXIw+tg00vLT6KSC6JRBvJ97P59+dA/ON5/DXxaJpzkaf+PdJKT0550O2NaaxMxZR4nrH0BbqIHilGKQiJXUUpeVQsNonHo9gocXq5Z8x7aZ66iNqKX5ow/5asYVjvmp+JsAtDVSJuyf0DtnnQiH/IRGmuWx9Hz6MFkb+lLap5n89HxUEoiSQ5wilv3uvhTHlFPrpSKlZABpBWn/fDAuFa4uAzKnHanbhey2/stzESYMI8/gjNYSXO3JgEsDkLlk1DrhzZpMshXxvJbWgksjYcv8LRwcL4rHz9bB2hArtsQyjo45SkucifODz2PzEd/Ju77wTqCBbPffhNScwffcHhb9vgiAC9IyEqtgrVFPaXEMc959FclZsZG82i7Sh1rjAbU/u0qepLvibZTDovnolY+ojBGzxl5+3ZQMukBAWuA/blNnTcHkkrKmB67bRS2UkcdG4pQ7WfHsCj6MaOG3GZHI6tpwlbYjINAZJ2oMAiy5uYBfyueQE7IPP3U1k3eO4pGfRP2NyzYYUCPjYMtH7M7/AJVpMRcmjGDzhHdwYuO0RaTi0fhl0bjrIfr9MQMPg4DO4GRO9IOUPlnKqJhRfGGJYKlPETcjt9xOqNxZSl0acIk98+Soo3X89IiICpm2exqhMtBKYJ8ZGp1y3Cro9Ons1QB9rwO2G6Fjx2j8LgRhq/8D/8oDnOn4m0Pav1jSXcPvRZmc2DoKdauTmD41TMr+hiyVjSFqEelTKShw6EzYlQImrQmXTHSs72qCtQ1jGCErJbzHG6UFEtoeZVbQeWQtx8Eo+qSLHRVMe/8lYo8Yib6xi6a8cPQyGFEfjerPi1jzbus8SSQ49NnorH0wuGHp7i+4tdUPw40K0raOZuWrj5KqF+euvmpo7NRRUhSJtM1K6Z5Qrl8QqezPWOBHezl3+7YwyKcCN07ORMADmj183wUTPWCxt5nFM38lZIQFn6hGIhJqsd5G+wzVgJe7herGu5DdsmPSe/HT0g/41P+2DmOZiFb6bPB3/Pjoj7R4iW20xV7F6kD4JcFAm68ZmUJG3JU7VPT2sacREuDZx77EPiARb3kQSlcAfsaRuCU2+gfk8VDYJQZ2/kR68RpOqdZwpP891AVsIrTjLq5HwsyACgqHPox99EyyehbwoG02kf9WXjtc8zTBwgEk+qc4OOEgtVENKNFxzAxbGvvQYzdyJmYDN7KaWX//DYz3/yl+Mf1NUHgSojhPxH0CUYuH91KmA7ilbn6/73dK+8ro6Whn7HYDQ0+JvtMbbaBUOnn5hWewJEeQeNCbp1c8jdqiRmdJJfPGYE6YpSzWixXg2TenklE4kzbFBEbXKphU44Gh+iUmXTYTq1vC2gfWYs6u4veOEKbUQ/8aMLrFcakusoN9E/fh8L1zbYmy8WQKZQy8OJCoRjlXkl4iOlmkKG81RYBEgk+rJyOPjSTnRAIP/fYQI5uWUuEAx7jTvP3cOxi9rZh3Hibz0BfI7GacgoRR9fD+LS/+3JaLpNKKT0Mh2k6xGC9DCU9Fn0QYdY4VT69A2u3klU9eIf1GBE77fsbtnUJgeQzXsq9xvW8xJeGNaPpV0C4N5CJ1PNRq52Tz3WgsGlQ2BRUDX2PzvTfIUEKEXUOQ1QO5YCMoeBqZMTMwhmmpnrSA7lg59zTB08ZK3IILVXchqu5ifCtdjDw4nhnqBsYF5jMy2EBTqAWVl4r0676k56djVpdzzAJPtkCcSU9Quz+GkjoG7fyO/hdEnz9Pe4AOxS0uRsASbzMXTX58eONp6vHCgYQCOzR0a5j/x72EXRVoDTRj8lQikyhwVIZzprWTNwt9Of/nWCJqdKy9FMt9Rf3pcUu5vxnmNkGPMpFtaycStn4Slyfm8+b4G8ilLrqsAdTYxhIpreH3IMhWtaNvKMa/WRwHm4Oa+fnhn4mdE4vlcivPrHiGq1nnOTHyBHtM4GvU8ca3T+N1sAt9zS0SyptY53cnKf1MZgPT3/wFaxb/QLnnqOARLzhVMIHOmwoMngY+eu0jzg05R0NoA7/f9zujvMp4Q1/CuUHncM1ysXrqKuwqOw0uUN6CLcJEHh9yk4nff4QyQMnDnz7LorLbc606CA9nCYPCt6GQGDFW9TD47GACWkSa2Sl7ppD5dwD/bokNy4lsfRiAbvdzqBrfw2V38cuSX1j18CoAnvaGDwIdOIZf5Mg7H6HxjyA9P5YBBcsBcc58ow0qBmioy0rGktaPm2OewTMhmEh9Ae+OmkhpawiW3ffh2yTj7MgOts7ZilQi40rjJNbffBfloEJW7W8juuUJCseZ+PLZL3HKZSzNeYpPxw3Fkf82Svt5bDodJSMfoT08k0v3lmMPmcHqbrApmpDIJKxauorly5dzuf9l5ungfT+oD6+jLryJVI+RnEqw8U6gmFiIV4K/Ry3fPf4Jvz725+3xR1z7t7lgkxHKNKdpShtFsfpnAkqDcdT6AVLCihOJqoglpjKUBZsWkFDiR+7pXPxb/WkKbsIruI17Mq4zWrhKyrcPE39Dwc6H13H/1Ivk/Bsd+bvWKp5d8SyxeTK6OrywmtX8WfgKv1QNYNG6RdhrbRjySrh7890EtIrvzkMCP974kG1lq9DrIqiPbKM05wqLz31GSZKMQ2nXyTeIbBgCotwFwAJ1G1cHXOS+nKPUTjzK0Y/exa5wElETgc7mTVrN1wBIBDmRI65w64GNSNwulOZOpBIpOglkqs0ohTZeiyjEb+ANrH4KPIwq5A45206dYumuUpAqqQht4lLfqzjU4rPWWDSMPTyWmsgaVi07jWeoJzeKlXSEpgPg6vLgco+c9/3EmFVTcz1jtxmJKfonhXm4qgqNsw5ZiIySpBIMuk6i68cTXR3NhNrXOT7yOE25TQSdCWLwucEA9O/szyiJmoKkVr7+eDXWN0RmlbLEMiSCFL0UxgcUYrcqkP0aRFZJFrVnaxl1IITQ+lCe9wE/VRdvDsjHc/xp5L5y7EY7SiUUDZrKb4+1UBHXjilCTDQNfGog7WGiXthDvz1EWH0Y6QXp2KpEHFHOh3NojBuJoAiiMbQRuVOOf4sGl8FCyuwU5FNze/2qf1lNVA0lSSJdwhvH/qkbZvGw0O3VjVQm5cSbhxl04DxS17+NO5dziL9RQXtZB6Yj5zG36JCUwdqkHxgXtYT0xv2kKF/BarQS3BSM1qQFCZg9zIQ0B+Gyi35YeOFBIvJFJoaztXN5r3wc33WDUyJj7L4wHv3xUQDGBF7m/VFjOJB2jE+0WcR5phFQ8j4vffoSADM7n2WJt50IpZWCtAJ6IsGv2cCYw2Pw7FGjklnwVXXxtdyLgAYvvOK8aAgTk0tpNd+QXfU7kZ4xuGQuXDIB0/VbzNg5CZ/OO5ox54acozptFG65EmlQIAq7C5VVRb37CidHlSH7fSSeIZ40JI2iIklso1dt8HgLzK9XIhOU+PSNoWzgPdSGi8wXc+L+4NGUX8i4mUFaXhx77j3HhoUi1XqsQmRoG9sWytrFbxJ0Qsd7b7zH1tliAdbDejgf4slaUxST903Go1XC897wxW2E5VOtsLS9h3ZpFPavHmTAjyM580ADDa//yp+SV3rva2NHNQ+ft/D3ra0ol+zm2Rc3k6YE/zZ/ko43cnb3WW6V3mLAwwPw6fJh19RdDKkH6S24ZBPjIj1hAvlp+diULVyJm8fJ1GwqNNPZP8uL2IFDQKbFoXT8o43N8fyB7LbfcQYrRKSoTxcA0xvgmRsP8PvCDXzz9O94NamIuxUHghgX/NrzM3pIpjG0hcNjDuMxIIjtkw6jTLjNJlK3C1xW3HpvukJSkFR08Ob3T3L1Zhz7nKMZYNqG57o8Ois7UZQXkXYpX2yft+1s7lnOPn0WpVaJXCoHCQS0BmBfY2fjrjsMNf+VxrMxYRgBya9wChcX+P9vF4DjdjuLlyz5H+z5f68JgkBTUxOC8N/GPP532P80su8/9h/7X2lO9z/h7o+3wmPe4u8NDg1GXzu1XqL+hEoCkXIo9W7HZ/GfXAuZyHlpEbp4cTDNUMKJlr5sKY6j39E0FAFOrECYbyO7DDGs++h+5BOP89mQIoy5e3lo56cEVfYgdYuTuc6ahM1tYmBzGyNWvErtJBt9feZQ5Xee36jntx5omXk/ufbVNA0+xvdBb9PkvQOroo44BVw4mY3v5SisLT3Yy7uIKDpEY9wQWvsFsWO6nHV7llG0LYG23NHsmbqn955/fmANRxXw1mkrF0+UYB32JKfGtFMTLAYk3pq3gSHVq8ktvIlJ46SfMQLv4gwWrdxJbumTVAV9S4AsnCTjz9jdPhiufUS3VzeX+19mnQHWGeCWwp9FC/9kyzgVV4tfZczRMTSGNLLD2IlbE84I61Um1OyhsyaFsgpvcg5ew572PP7Z33P+BzNylwm3W0DlqUJQi4PUbB287Qd3/zwGeX0IwpB/Dl5TtGI1rfDbXJo7faj3/5BFtwbx7VOX0EshK+A8gR4VdM6KxuDfl/ON1VwoyWCbUsHMPWOJimzm7YXbOfZEF+OK32TEphGYPczURdRR7f8THhI/0m+LpAm4MUe7aLKXUeUMh0GiZo6P8Dxym4Db0oHfzRCmFHxOaXI355JH8NWzX7Ay+PPe6w0tPU57oIbnkjZyq8ODVZHVyH1jSTnix+SLr/DDYz/gk1zFzogqqh3wR4844acVpKG2qjmbe5a5nmJwfmxMC8cm7yMwegonQ/6kLryO3DO5jD4yluQZJ1jt8znF/cBPtQ0QUYNKCfSVe6AWWjgjdeNKLCEksB/6Dh2+Hb5M9rjzbPffeBOhdirIJKx8ZGVvokZrSSKuUXRo3PUNpJ4W9etktmrO5JThkrtwy8wgkbLimRW9QcABajFgdnlkJq1tA6k0b+RY7CraZW1ktGSQeS6OYX238lypFzdNfqDr5NeHfiW4KRiJysomA9Q6PNi8+DfihOn4NA9GIkhwy5T0rRBRtwpZHR2yL5G2ayn6Oxe/n/8meV4yu0IyGGPy457kVUx4qpEwzQoi+6Rz8HoQNpWYrJm8KxHYiNv5Jna1nuboAdi10fyoGk6GdzV9mmuJFdw4+itZI1tJp48oqLLDBAkKK/N//JB3Gpcy6LdMgmtH8tFrH5Fvh+daYaBPOTWKREowcmjEZ8hcOsZev877r/vhlDuZ1ZbDmOKRdGVJeeett8X7krp52QcKrXWcs/kTLBODYqH9QmmNCSXUr5PD4YCwilIHnLTCIEGgXaS8xyZvpiLoS6RuNTALQQD/mjwCaq/i6IkHlEhuVwA6XW7KJ+UgzTORnOfC79Bo6gHjtN00ucAluJm+YzotgS3UhdehPJbLr7fqWPXwL/T4wNNxeXz9ysf/6JudDm++urIOlzaFDKLpUzccuUuLI2A9KQGncFvTGFoVR9XlGEwLTZjdapCpGXdDRF8NjH6B3I9X8lJHMm7nZ7h6LNRkZPLLffcAEP9fxvXgslMEVl+mw6cLQSKiVWUaJauWbMTg2Uh27ScU2+LJCtnHqKPTICSYY9kl/zjGU0f2USvsYdhePZXDjFQFWTHoDb3bj2ck8WFgDgNn5zFzs5vyeHFxm2i/i/03RETvW32nsP7AbI42h+HZ6EWQ/0Ck8yRgaYa9aajNCmy7xqIeZeBWionqwLze4/e73I/gFg/i3+lDbUo2lUEnAJEOR+dl4p7X1vFVx59Ena9Bcs1MztJpvd9dYjXQv99lvIMmEHVkN5eP5NBvjBiILo6GAz3duGXw7lFvHtz0DG33nIHJlRwywyPNsMHpSVdTJDXn++FV3sjYSXeQ7Ama+ymXXOK3oB+o6BtOXuBZYgPaRLKSlJdArsW7UcXTfz3NzSGNvLP8HQA8nODlVKLZMQGLrR5BBK0S1footD7KpnIYo/6GPqePYUsdwwcNIxgU1kXfkOsMUsMbvoBvDc7YGqqq5/B+/kii+lQzUSJwVxPE9kSSM+gKZrMfyYMeZ2LI03RoZpOihPwoONixkkkKONBjwrlyNjaViS13bWGZF0zWglww8NaEi0RNszGzMoju7O8Z4B8CG56E0MnYfJbyRAtcNXowa9ggLkj1TNoPW/bDzE++ZWf4Y6i8VOSe7kNpoprtyUc44G5D2gmnIyuQ6ToYlBBK0YiXkQDhanExWWrx4GbAfNp8RyGwtfcdnp27ja+8FNxodPJToMD09FpamlvgNkLUb+UiTvbosU4/hHvtHAIHWQk9P5jWAFGD5aIV/GQiGrjbqxuJQkre8GrqFOcYohL1u3ZZ8lgj1bNl4UYG8CRtimGcSe4HgMas4ZmvX6a8+Cz+80eTP3wZATVXuHvT3Xg++TsvhhjYteAQh6zPcUlTw46xa3tR9hIkjNa2kSPVcMGlwakbToP/OgD6KOFePSQr3Zy2wK9t/5rHJQwo3c/CcRMxu5y8u2ck0de15I8voU0q0szaBFEP0yLRYfa0kZedR2pnBK+//zoFM3cTFH6dDQaY7xHGmtPvojJ1EJddgk0tXpfuFqgk7XRN3Md7A8ORbC3FYP2AikQXbV6nGHviayQ9P7HgehbmuE5yz+RSGVPJ9azrHHK0oWzI4fUZWzA4pIwteZnVaxczbsFBgrv7c1h5i6qqV8jbl4hcZ6cmrZzKqLLe92nwuIHOGsjctXJM0Rr2jNpMk+Y4IOpBbTfBQz6puHyWUdmnHWN0OlUpk7gvEDg5C/wGUCcJ4IwFMgSBFsM3mCSH/zFmIXHjcrsxacXgVY9nD2uCob8K5q8ch6cjkK6FsD36VyJbHiG5LppPECuvXz0wEEuElcOTL2OQ5jNXEP/vkHexMqgE1bI8EnT30id8M339TfTt0DC/CRLzXmNE+HSuDzHg1rRyKfJOFe1OEyQ7a/lDXUry94cYf/B1Ms+c4kPNVl574k9Ifhb8+nPeNQhrgAGrfzB/3PMDqYldlNvhNboIS4lEF+yGJQbZAAEAAElEQVSD8WIhObv/oqLvHPTRfRjkVUNqykmKOx8nOiuSM9clBEjtbKqcxrDw3cyoU6PJ68tDe0dT+0AD16fJeXrMcbptPjjpROpWMmvFONKvZhM8r4ZLdf3IzL3BPJ2dQDk4XAZKvBbSkjSEwoiN9AuooU46gEjXRcbUw4I+qQgJiVye8hYWhZhgfNLnNvproOgHGT3NdHn3MGa/B4MqlrHy0ZUcNAtoHQokXTLsNjf5GbcIqG/kUv9LBGSL1dV9fG+iivgSdfh3FIx4DIA6v3d4Ww8LPe1stJygrDKBNlsbRuVZzMpcBARcAhiU1ewfPJUKj8FMPdiPiQmH8BCucCvhU+aGXGSydxAPzKrn4WObevuqBCmreuBo40TG95Swbeo6Pu2EU1UTWG61EhT9M48qvMHSSKDqY+4ZuINX2lvJuOBPAqM5OuYoglQg0PkzUdIxdFtLUDjcyJ1y+lT+it6cDdHZRHsVc1bzBPFKXywaK4JEwKguJDC/iBfbNiFY9hMf483gg3oioyu4Z0YSw+PtxFe7WMVf1Oy+QK0zGgbB+tByEpU23qgEr/Lv+cq9ls4zw/lhTA0XBl1gqvdAaBMTHn1kvyN41DDm+03MKxmDw1bBqvYGGmpf53LZ66yc4WaMUYr/iRG0DtZQkzqIUL9m3u/4lvdv9yu3ICDRemDz8AGJtBep/2u5L/dvH4dUbyUu708KUkVkyD4zZFTDy4EWAvUWrK1SLudcpsNfwLNH5IUbqoEvw+HTFguX+l4k2f8vzkuGYhbqAQEfRTv3v/MLW7t8aTr7Fm6ZnB8D4ZpUQpBNi8bZwNjriRT8PZq6+40sHfU+/cP2ElEJdU4oqz9G/9IdGCRwaIw/0UPX4BPQBUBXWgn7VDX8khHE6sUbsKnuiAKmqeC1J3cwpR6yPV6iOzABSZRYiOQlt9EZ+zy2G4no2/TU1exFKNJxPuE4F73XMqoe4rDSZ/YObJ5TuTfUj7vTf+S7nrn0W70El9TFihd/pPSRlSRWTGLRukVcnH2D0jTI/1etJhL0PXp0Rg+0/k8QmdQHgPdP7iA0U4U3ndg+X0J0rBH5pc0Q28W5bMTkWlgD6gA1glKKVW0loDUArVHL85holQmo1RZccui52s3evKnEZuSToHDxRzAEymw0eNfhdEJIeQ1maxIlySV80gnfdEFqTBCamFTafL7g4Z8f5kbmDaqiq5ixYwaG+QZ+Sf6FkuQSNLc0/xiW7828l1GSe1l2QIt/hIUnBkRyw7ecky2zsUUfImdICF67nmOa5Agbd5bQoU1lwsEJmLQmWgNbOTHiBP2k4rgqNDbhW99KV1AiUrd4npSiFLKveeOyO7Gr7qAdjpnBJcA3HTLacTEh1I+8Sa8hSMQYxGkLHA8HFn3M/QfurKclEmjQnuZSRwJXKxaTeNRKXYqCa8miny2V/LNeX99ajlPpQaPaSY9HD0EmKXvLHic14CyG7QfJPCilfLpAcWwRdoyYnXeRYThNQ1sIJ66HYWkX15MDzw8kqTub8YtWco8e/nhgP393eRFhWsn2iqncCj8AOIirgklNc1ieeoGTPWY0BZcZXSCOQwC+hqG84HWTwDMT+MxLwZhV75CokPJd1QNkCQLFkRVMzyoj3L+VU2VZjDs8jm6vbvIz8jn3wnectiTQR5pJ29SDlMsHcD24kb5GMyv8oSwKTsiktGgk+I87y0mfTNbN+JK9Ui9etQ5hSP+bSORB+IWF40jVsXv23710kM/7wDuTcnl2yylcXZkUZlVzSnmVAaXLiLA1MezUMKKe/4HRvut5o2shL65cweY+Fxg15Qijv38E/cBW1j7+PZds8MkRb4YdWkzBIxZ01hR+CQSVq4FpXp1csMJz0nakgpaFcidT7GPYlHqEy5a1OPwMpC3ZypWTE3l2xaPsnLaTLmUIDlscEilkKwUMEgi4LmVQ9yCagpvod6UfVrWVxig7UpkUlamjtw0lHQvlzVw1Lxb0I6cgAkZKODL6COMTW/EXBtGiPU+2Cs4NnExOSj+kg06zuUUscEkvfxSo4fWQtfjP76EqdhIvRR6gsKGQdbPWUXqsFLuxmSs2GX8bXLT7/Vu2FSkD1DAv5Cpmg4bz+wbjiKwgangUmTczuRl9k5BfIbwIui1VKFRVDPg7DMVlBUXPnmUgoMiu4/f7djJqhlisKLgF2sKz8KsX6Qb75vXll8cr+GDAWwAEj0ii+Uw+fre+5fUtr/PVs1+x+rErfJj6HvETQVsSi7Prn8k+AIPmBsqSapZtW8bR0Ud7GS40Zg0/7viRJ3KewGF2IHc4ESR3+mFqYSqRtbW05Ldg/OsA/fq+wHNjPmFJcRyd+vkEdPyKt8aNfx83n7zySe/3Do89zKlh53lL8xaCAPq2Slw0Uxx6lqi2pSjsUrHYZqyadoUOo06M/fmoOoj1aUDusFJcmY+hopUbqQdp9hNRVsPC97Iw7WOeAOL8IcqZQcB2M6lXh2GMucC7E0WtxZPbhzP/WA7G143IHXKcCifHMuJRuLzJkCxm2+xtRLQ9yIDIZMrK/fEOvEIddzQGBYkLq2cAgtVK5hUweo7m4Mh9NEQ9w/nup4AR4n7cWfevaY5j6W/vkd5azsl+W9mTMx9BIm7/7tp7TE38lpnPbeTNhigMOhXdEhF11uqCt9ohvzWI0KAwIsNCcclX9B7XVwZRSjupA4qp9mvBZpzF3JvZJKVX8oakixO3a5Yf0mZRqSpB7aFiGiu4eAImPHmnDZhs7czaOpzaLAPh8XYGqmGgGjYGN/H7fb+zbsw6XA4XKrvYxv+VNNaYNYwqfxd1l5rUgR/xWUYKRk0xUXJYGVhPns3I+fKnCPeVojKuI6QxgcaQMp70ghUB8M1PE+k+pUPIsPXCrJQOfxTVy+l2BpMZ+hVdUjtpZ4KIKriPd956B0Ei4JB3Um09xLVhRfyRJaAI8UAochFWCR25s/GpWgepL3N/ytccLZuDTKZHo4A+HlZi5T/QvOMlLEYP7EY7jtQsCtQ99Oh7kCBh/737mfDHhF4WsZGGkRibjJTHlfPd49+xLHPZnbbwb2FeQQCLVwh4hZAk+YTJpa+z12Vj4H/T48RE32Spgo8//Zjg4P9Zhb//2P9q+0+y7z/2f8Scrn9OxIFdU8kqu4SqwhOF1A9JtB2pS0q8SuSzH1EPFY5uiOhmhu90gjrdrOgSUQbv+8MUVQWvqjR8nL6L7Ob5JJ0/j3doCU1GcXDRXMyGfnW45C5ahg/kUO6WXhjzqdSL2NwJhAcFYR9polaZwfQNGVSkzuH3+aJjr1N50eWWYxXAVmcm6/ghLuecIa0aJnXKiQckMinKPsnY/zpOSPlZjk6/yiYDqAIPIPg7uWqUw7/RIDe7oMsNTRoTdv9mimxbqbidhMq4kUFycTLhwwt47/MfeLzBj5hTKwiOv0mItIhIdT8ORkGB7SzjB6xiT9kTXN0wgNqIWkrTnPR4XANAJlPh1+jCIenk/mwZW3RiQuSWCYTwvoxzVKLs6UTmasN7WjbXC+QkaCvQXLmLxNy32OD1Dv4p0Hi1EUuzOJtuMECTCyZbfChvDcTRZGPYyWE0hSkoiztK3xqYIAvjQaWASwgiT1aOn1JBZOtSsj3r2R63B6O7BV1SC/OqI4hZ78nKyzk0ZnxAUr+H0UXV8YG/BZ0U3u9w893jP/YGzVYPEyufVPKXmG1K44/6+VRMqOLvPn8xVPp877P9SOjm0jOrGeDhQdaKVkLbjOycsap3u9sNLoebnN3vIQHqo0O4pwk8pWauPbCedNXrKE84qA+rxyVzEXRgJO9VppA39iLxG0fhM8XJB69/0Hu8X7qhrGoZP/dfhbWfk+ebJ1MQKAYbHAoxKX2xMoyoPpH0OBTEH8hl7NU3WfHMCvx8DZyLNnO8cyntQV1svnszU4a+iF7zPq4LZvqpxGqrtrQfaXjfi6yiz2grfgS9bBFN3p8CEFajY9j2DeT9Mhl3XnPvdR2c0kF14Ep0Bh36Hj02k70XMQXwaAssawG5/0fIvX2YUXwfK6/7ETYpiFGJpWjf/Ja9ajead99kgI8P8QFVBBTA+2+8j2CHBU3wss6OLQIW17dQU5zOrmk7sPiNY7rlMgm+l3C29nBP1XWaC+Lwec2Hq1nX6fLu4nVVEjNi3qXGqSYztpmmbim+WZGUJ7fT7PvP4KnT6kTfVoHa1IF/TR4yuwWd3MoDq5ZgDlLhfgGaQv7JHv5TQwafe9xkiFRDc/+b3EgRncpyB3zVBX8lVzL3wx94yfwKtIrojFshH+BUiGPTwm4lBYdHcjPchSpIRXBTMO1+7QyqNRDm6st3XnnUXOnHXuudRX2TXdn7+/O3PMmv88Paz0zBljzCiq0UZ4XQpViNb2cYbscMXIKLgFpRM+fS7o8oitxLZIWKzBvzsSYb2BM0ncgEH9QdA9EZdRSlFNHQrGdgmx+uJCeZNzIpjyvvFVg/m1KAwSlBKkhw/xf9gOpoqLBc5bvNYwgLdHLpMykhzlisWl+0affxVeJVNtnv4/FzUQw5ORRLuwXtA/OoOATZwQdI8LuEFCcGN7gFaCtpI27rj0hTUiELHv75Ybzd9fDOnXO6b1Ov7p28l4ybGWh6XEiypbQEN+OQ93BRPQmnqprtCbDzoUWUeT5K8OUveOTcI/w952/aAtr4LXwB/Vof4MjDv6LST6DQNhuvLi+6vbvxuE3NtMdSypb6MGr1X/eeO93xEP96M23KaMpN7WCXcXxeHZWafTwiSYBt4vzg7S/np6U/Md7vM9w9/6Sk2D11N/HG+Uzjn4uaB5vhgQQnsakV6POzYUAwt+RJDFQriJKLVYpf+bbztt9uZikmE1t9kVvXE+g35gqXrXDW4MswYxbuhh4iXGM5l7uDFQni4nOzAeotgXxweAj9LqTDJCkDh96kSrsIeA2Av3pewl8bylYjBPsZaQq4Ror6dmWmTzYAM/WeGDQKopjKyJvvkd3fyoMLu/j6VynluVHcs0DHjfUnyTiVR/GQh3BoRO2uwWn7cHqYqEx7gPI1uZxLqoY51/m1B3aZILxoJ3JjXxLCDlO8cySxvns4NuwSu0xwSL2PLTnwQqMv0uZ9+CmW4scZOl3wUzd0O7oIX72QBJ2RIoTeRXWKEsZqoAwXazr1qJWtuHHhL4MZOjHZjCYMh9KPio2T8Hd6Y7zbSVfgHZ7nORUPMy9kMjIF/Lp4AyaPNmzyLo5Z4JgF4mKHoI+Kw+DooltTicytxe4QeMUHgp3NbDv2LGFhMPxIMpElg/n4lY/pcyWHgr0TUT+1nqqgMkqtStQWNW6Zk5E3i6mVfIIptIaTHs2kjjiOLqsPmwb/gFUt+hcv3taO2R12npEvbOH1uhwEu4iGrXeKiVutPB5fSQWD1KC02pC7ffEzTqbZZyeB0hx6vNUo/PTINQqs+kAkbhcBbQE85gO1DljUDAv1LuwqJyaFOFcmKODnIA92XMrg3B9D8ZxvQhp4x+V/twM+6IRd9elITmWTFKIjt+BRyuc6kBmH8NSh07glVi4PvRtdlg5TUAitHmINpwIIkIGeHoqH5LE2PY/CsodorjvM5/EG4jQQ3xzGXRLwbD5PTMEFbAPjCbYF0+HbgUllx3R78ehyu7GWbGXUzVjO9f8Qu8qONvFJ7r+YS1N1FE6TnR+X/dg7j643wHquiKsXmQSpDSqavfirMxODRwFWTQkGjzzii+x0+rWRf38ZJrkYwIiSw3xPGOh6ktboMOr1EfjW7CfGGkNl7B2eWmlPIR8ufIgdJc+ws91Fq+oc55tTSWrcD1IFUkkIy99ZTl2cC6m9msz2ZK722QVASEMIcYUuLG0m9D16jo08RnVUNZ0GEdk3qz6WBqMWqWYK5deD0VlT6NLeoSn64bEfiFaOwii10K5uRyaRMUAFL/nCtx2NnNBZiXTZebPLxA6zjme9u3jMCw40QoRkIA0G6FAc4L8xQcJKxxiON/9KdJeb4IrzFEak4hq8DplULDAJCV7DtjQIjunklt9zzBcgsxGuZ7XxZsRDNLsuEtcWTFdwEu0+lbR67WO+bzuLY9aT4rcepeIb/pp4Bou8kTmle2lomktW61Cm9x2LxHsfF4KHkd7nc7wD4Gz3fL7y38JsnZ0F/q2Ux5YTVuLH3ryp7E3sz8TAH/nbCGua0zA2hiIRqqkxatnhhBW+7/LjzZeoVt/gbqkHd8fG0c/fk9V7PmHksZFcH3WNBX3vAr2IgjoebOOWRcNhB8idYh/oowLn3lE8tC+L+mU9XOtbhqPPSRZsXMAFgycBgy8x7tQwLMYWzINNdGvKkAlqOnUX+KBDLPrynf8nZdOlJB95ijEnX2X7XQ5uRW+lby1MVttYF17I6k4N8pIg8upyCJ94Aqf/dt4LA7Org28ff5YY9XUWXriX/L7l6JJCAOjxuA7ye7lghVoneHUqsaHHw22CvemQ+CRbDYms3z8Am28FMaU+KKyZHB1zFJU9GD/DKAQ9yLzUfPP0N0hdGnJLvuBsSn9umGTk/3AXUf7eqMaO4OfcOb1NJEJXgaLxbiL3SfCcOICYORvoE3ESGRailRAiB6kOonNqqZFGM+roKD5Pt/Bz7nlqYmBAlRVrjwZzix9Sp1is+a+29YI32N0r+cJ5L0s7DhLW7sUj6+dwdNRRokdUk9rnVQT3p3w2/Cx7AzQcd/1ES3k43t67wAWf24Koqg7C2mVBNWoIJe4ht4/v4G1f6OqYQP7Q+1g43o83zUPp0Yv+Xrcbuu0wTaJGbVGx3UfC7mm78TEOwbdVIGLMeR7JqeHYmpkkteupnm/gVV+A00TK4UM/AYPuM+IU4CtzgfUKcnMBu00wcWQeh8yQJrufX/QmfEacAlU6B8rT6eN3iNoYBy+1gUSwM3buMfaalEQYDhIr8UahegaAD/1hbbkYbOvxNuKQG3vnw1g5jKmDs1YYEpbP18sW9L6rDcGQq4Gnf8jFvz2AmCEVBGk0mN13kDzPBTl4LOkqU4rGkB0gUq15Os20A55aK1P0mZTaAukM9GXKQ7t4PKWKVCP0SEREk4+jjJlPb2Fthw/Ks1+S1zgeAKPdF3DTIB2LueUMPf4BnJ/1dy/1usQtQeFQ4LA6UKR5s3fyXh5Y+wDJxcn89OhPfOwyweKfWaj/gbqyBm6kr6W6006yBoJk8Fm7hJ86ZNwdIiH9cg/+dQMpSS7BKoB3x2wm9V+AS9GOZs00whrCCGsI4/vHvic5PpwByXI85HDZCj4nfZh2fhr7J+5HpnIwXa8mxO5CWvEYUT21xCwMon1WNhfLYeDt/EWxfBj7CwvwLbChTdHzx5Jf+CC5g8NS2B2UQXbUbdrO/Hxir57h6uiHaPbeLo4zI49TkexPrkJBQEsAgkSgLaCNIxY4YgHfdj+SWn1wp1pw/5sm0r8T04Zrb2BoC8KvtpHz1TVsD3uU7e0wzyMK6ZBc7BovUm7oCNcMQjpNxpSE7xgcsRXPhhkkXuqiMyCJmn7g6/RF4pZyrm6W2Lfj6zg56igBmhlUen+IwaOKBuMQApPeZucvfzHl4GiMS3vIPZ3LuMPjAHh+slgosjkEii1OLKeKub6tH5PfuMZ7t+dUW9SbLOkJwdziSUftegY0DuDomKMMUsPWmNOcL07k/L7B6Ic38mknBAieIJVg8DBTI7Mh+BuIjKil1jmSdVH7UUaInKZbjOBt1vN80s+8K3UQoJMjd/xKpEbKNJ34xIpx8qY8hmu5O8ntnMWEvG723+UFfITJDXXdcegNHriCFNzMvKNTdcUKdpsCn12HaW4ugrGinu+QwAu01xqIDraR2BrMXz/Mx2uGmyDPJhQKF+N6spioNZLa4c/bRRHUtCpoCzJycuJ+IoOywSCQrASl28ay8sm0WSTU+P+MwSMfg2sTNVY9r7WBWl9DZfN6rpU3Mtozg/Lkw+jc9+ArlaHV1IMQyueF2Zw/OJAOt4YU4yiOjRILrdr82/BrDcJpc5Jdvxtnhei3VMQbOYQFfUkC8dWRyD0UVIw+xRsxsKcjjDqZiIpN8lzDaeNWljpG3Hkeye8y3LyMNmU8yfLrhIT2xeK7nQ5bBztKdlAYWEi0OZpPagLxLPGkn08Ii3cupiiliKpkKVUOkXL7fW87j62/jGT04/gF+pH1Wx8kuwVyZbDUCcFAE7Bm5Sam2v9gFGMYzBD6Xu3Lvkn7cN0uFj2/4jxJFw72Xp/KpiK++VUEQSzydruhzesQIT16FE4FgkRAwMltRReaXEX8V5v992xiKmOxjXQjc8mQ3D4WwNDTQ8k9m0vP0h6GfTiJD02uXtkPgG2ztpFbeILnh0Ti9eyDCCVylspF5KdMMBN/eROas904v7//H+c0ehoxa+1IpOK5Coc/wpH0GCzqKnb3+ZMYdSUrjyxB4htKfvzz7Okn+iYrLI3c2irg0V1L6qlfSfQs4nL/y73H/VZymH+VckXJAadAZaInh4e+hbc0i71lj5IccIqOSE+6x8Szbd42nrv5HJ++/Cl2RSt2RSv/unsBF24vX7qDfHF5uHtFHYeeGkps6T6a+s9DlpmG8/AJ7hv+NYKyH5ebvkPbJsVpdZJwcTO+/qmYxyzEJ2QDi/3LaQ6TYvZIwaPcxNhDo7mWdY3WwFYqutNpsQYxN9yEYDah6dKjlWgx6Uy0uUR2iyBpOP37PkJUdiURFyIwe5hp92/nk05YXT0Mi/oWJq8qllUP5cBWbzyf2oJ3QBcWF8hcWtIlFxjx3M8c7llFdWkN2k4pEkl477OLUdWjWnQCmyaNvJNZ5Ec0E52Yj8XDQkVcBapQFVqlluXLl6M1ahEkAu/7wV2WUNb/baJtYhsQiVMqrrO0UnEeHpv1Ip1xH3HNVs/wAw66vYZzaPpoyj1+YpsJ1g1aRWuCnrGV93PPuXs4MOEAbQFtSHRXCZQrWSXzZ1vjRNb1beNa3EkEiUCyAi5GwiHrJzQYhqOqHouflz+PbJvOyPEXsXumwqDPwdqKf+05pBt8sS8ayeW+X6NWP8XDQiY+gZ1YjB54R3nj9lHRGqbHoXTgqfSkorOCpOIkUs+lYr/PTvxn8Yz3G8/+B9fh42mDOwDAf5jbDUpzJwHVV9AFzSNirIShB19iqETBEreVoNv9fJVUyhnBzdiJr/Lsc8/9/z7Yf+x/i/0n2fcf+z9izv8SiH5GGM30oOts/mgeLaEO7NedzLj6FmUffsg3gXZqGkSNORDhsE989TTpmeXwiCjAPtizizCNlQiNFUmDk8CWYCIdBfQbk8dvLT/iUOtZq92AtSuCDXYnFk1177mv20Aj82GPOp3ySX/g7UjFo3EaXR5OIuUwQKdF4+xmeLUfQWfuxzanG59OHRqLhnY3GJPi4Dq0nSpCOjmGqqyZBN86jXd5JQu84HpmBVl9KvhZtZJJKybR7dXN2dyzKCUiquvBtHO4086xqLMPs47MIq9vHkklSaQVphGbkc9XBi23bOFEDJnL9mtTUcpWIwhLMAnQ7dbyzdsvYHN48Msjv2FS16JyZTNHBzOFvnhKXUgtHmgdKux+bppCmtAIfvww+SuivaPZW2HljYd+Rm3rx8v9B7CjZyOD5bEsi32SghN9cEksuN0afhn8C2WxZXAXlDrAaIRXHriJvDmLpkYrY46OYc/kPRAHTuCswR+PoF/xTFJzOXo8VZygb/lmahU7eKwFWp1SegQ3RocH1rHHiPe2IGuqwPfEALaN6eHZDj3uG/F4qATaYtt6qVz+JWw/Sp3IgT/taNuMOHNFD6+PRyOcux+SnsIsqEX0ExL2Tj5As+MqnRIx+NDnWh9sERW40yJ7HZ5uPw0nLHAoDPqppDzSEkxhv3Ly+4lV15E30nG3+9PQMRV/n06UGu9/OIOlDpB3JLK/dSB2twmtxL93W17fPDFpqPbjhMdgzl4eT4nuHsoSynDKnXS64eU2CNd70Z5Yz8/d4HYLtLcokQGfdYmCzIWZWobmfsMZ6QA0Phoyqt6hwudLnAonZg8zPQFx6CP0SPuFUWqPpjswHrtG1ADLvprNmKNj6JjRgl+bH3alHYPe0LvI1JqtKBydKNsECs6nkzXiKjZNMy9iQ2kO5P6ofugCPdBpfCizb/lH35Vb7+a3rfkEq3IJONdOU3A/9szOp5/3Xu7JfJtLzmfoM+YSm+MbCe3/LjtmbgMgRP8XAOESO5MEDWscLlwuaPDdQpOPiKypjK4kpioGmUpGSNlJlDYxEPKg+xdOOr3IVEi5IID7NsWEQ+HALXOzORhCbQYWn09E5yMlL7mBNor/cd1zb+cPZnm6eHrF0xSkFXB43KfM1kGWEg5I2nC++D0O01Je+URETR4ec5jTw04jdXtyoC4B0+/jUM20cuG7S6Qdv0Sxvye5tfBS9F30XKvkgT2T6R7cQs3+IvzqDFyb9QpDjw2i/+X+aJ/T4VTKaY4ZiENmIfzkDULGeCFIBFKKUrB32fFt9kDfo+fg+IM0B4tJ3M3FAynaNAl1qpxNC8LIj/4ImUvGd49/h0tv4KVds9FczaDjUxfJRcnURtRi0pk4Y4U2q5rgop/RXXARGBxFaGUlp6fNodKqZWETZASlcGrQX1zKusrT0U/juj1M9w/bzbTE73izcz2vXAonVBqGZqGG9th+6DtqWb58OVaVFWeSuAC69OMlLn5zEaterDCMK4+jb15fnPEC7vsEgtqHgr6DOs1ZnvKGLhfsNz5EQlwAkkJQ2pW9izEPVQvzMz/laR+YV9fF3C/Ho7BL+ObpbwiWw5YQ+LTNguK95xkc0sW0jW9yqf8l3MNdzEt/Gx9lJ+Xagbw0634CDRPQK7xpVjeDVAYBw8BUxT7308R2DyE6LpFRf17HJr+LzXeLVa9JbXuI907A1mVm0jYvrvYZxPnB5wGRfrqzKwvBLUBICF3BIcgVAjZBpKjzdYbSV9WAy+WiY/mXzAwWH2iGUsYrtx6kuOBhgo5tQjkFYuYeIvk2o6FcApFyNy1xERzyqWdUgh8TB/7NBYUY4NwVCib3QY53LOb199+gaGATW8ddZpDH7UTl2QUQMZt7004jvG3gi8tvYs1vIzgjgYzy+aTEHWLWZSuqYEClwqnU4pLa0agaeWHgEi5Z1Gz0imd+UBcTXriEn/8F6IFOt/gZeM1OaNV2ep7yR/Lq98hDXGSqIFsNDuD5VjhvVjKsqoKf895i9OPBNLhm8WgLRApupjR40eUjoyf1V25GPUqoKpGnWkt5qhWuDfDj/OVUwit0BMTJmds+gVMjyhj2ehlIpEiLzxBdFY1UUHLa+i0nUz+gaFkAPkoX3vZFuGxOBEFGfditf/T3CDmczlnPofInuFF7ALPlM2ojakkwv8JiHYTI6jhbvoCuGi1ub08aQhtAAl12Gd36buSCjfV5IzgQZefZn56mNqKWWxm/M6QqhENj8wnxeICu6DTCIm30GHrwMvXFrrqDEJXhxEcGconAiJ1JNHlLOTrmKC+1wcO+gQxlL3NPjGe7n4PLstV4tzv49uEtXPxrKFeHOJk6PQS33YVnWy1G30jqUt/n0yMPcNP3Iabsj0c1qA2Nrxx/lz/tfu34SAWGBp1jd2ggl3PU+PlPZOHqcBpC7mPdonWifowA71T6MqwyBLmPCYVDBoITudsTenIR5B1oU9rocbchNd+h2+yrhvMRsJ9zHDFpybebcIRr2LhwIxsSxH3sghuJpY4JM5dwMiWYgIplPLr7UX598FeeyqnBu8WfI3+Opia4BFN2Fn9kPolT7kQCPOcDd809w4LcswyIeYOOlo5/vEsJMNDog9mqpDVdYHPql4zwWIar6zrdcVASvIq3fe00W2Nu7y1aglLUFSu2XEf/4kU+vziCSb9M4nLOZSpjK3lILxYPYGtBpegERRNOUx3YN7KnfBD3PygWPD23MZr8AQO4rkykJrQOM3cCiQllCQw4A8amHpb+vJSG0AZOjDzBbz1wwwZvv/ozh7rCGVf3HIPOJdMensm55OEcCxNRQHt03TTaemhrkaN1apFJpYTKYZYWIgQXFUYPvq0XGHZmEnVRLUyfeYXpehhm+5DUnhD8zY0c6EkltyGXwtRCOn07aY6B8z2HKfH+gdo/4tB5xPL1M48S4WXB3rAVjbkawqYgkdzmQr1NC/l3iLhIXN8D+TaIlblRJ0RQOGg0hRHPYVM083knLPYSC+92C258pG5e9oWxoWvJ1W7m16uf0exvxjP8AtbOXJ71lOJ2Cwzx2kJRNzQ7pTiHX+TC0It41c/nXPoe5uum0mYJ4FCHNwGd76Mt+4CYylgCk+J52duf0QkzeDrdxmMtIJe46DS101LnJqSlnPD8kQwdXgrVm6HvFyDVUrpnKDV7h1G2tIkL00S0934zRIbVUpYsReMlFqVM2jcJ/3Z/+lzvw6UBl/C8mUKwWU9lyyVsxpdpCwTwo9AOhXbgtnZPSJidvOw8nMo0jBox0BjvTODqM3NITG2kZ1IsAyMvkU8qtU4YUAMqIZMUvQGZW0V8eTIREU/RmijwhBcMCTiFRBPOS3v7kFyczL3FKdhSPVnU71nx9YROosFZTNq+MZhineycX0Kz5uDtviEjZ88HmMKTcC8dcLvDuLHLOtga5iROLmNLfgy6VCuZitno27Pp0V5FAvw+bgo36wayruNlAvoms6loBQerG7gv4wOGCOupjYEFlmZODtGT0H2VUXtHcEWdDyL7ORdHPU9wUQLNGZ+Te3EWY3Yto3OpHOQwrh7CJF4MDk3Dp/QudJZ0zg45S21ELfPiziCXumiRfMnOzt+42B6NKcjIreBPMLt3cpcOcm4kY9w6CvM0A27BhVviRiLIyKj9lOU5B8BvJX95eqHy+ZjC1GIEqfhuVBLQSeDBP6bS53Q6mpdlcBvw1Rpo4XjGORoN0yho0aA1eZB1VWD70ZlMW7KLPiqXiGxyljBuVxpSqTfDLh0j5eJcPn7ie/abLLilbsYofDkb1oEh6grTrg/Atc2Dd6yJfDC+gLkeCpweEfQLKEDpHcCcMgW7i5ZxvWksmfFfYu68xZjTObSVtKHv9sSqsmPSmchSwZt+UHw2grhmPc6YfxbI7jSJCLyD4w/iYfPlwWwbyx4oZZ17Pn5SUX+x2w0vtoLZLeeHlW+g8nXTem8e7he+Z3KAlbu9A9B632B8y/f0G/MFAB63PDgcIT43u0JgnCdUlM/AdPRbVpa+xrh5+1k5NZk1rQXEeE/j1xHTGD7SySrjR73XFtIYwtJVS8lX5iMM8O5l3FDZVLznBxY3fNgpal+VBu5FqqzEyyX2qegqiKqK5I01D9Iy38XGBRt7NTSDZNBHIWO61E1XUwSHbh/3atZVtCYtTdd6cPTZw4OKLPzOJdOl9CYqL4pD4w4RLHMwr/kXqjQ++JkXIGzYSdnQyQhCf+J9L5NT0h/qYjB47OHbEYdQDb/EzsQpdBfWMTcYrE2plDS8hRB5e4TMyOSWLYyS6JVYVCIip9u7mw6XFYlMygPrXsKoqeDHx35kUMkRkqV6+tqfpOPPiXQtvkGJzxLUoUpybv3NV0MfArpZ/d5i3JajqIJziLl+lNY6sfiqjxK+inqev8zvcbjiIZKv6UHoT9d0Gf4edcR4X0eoyGPw/XG81zKRtEM6JuY/xeaHpNhdGqZtFDg+2AvjiJPcd8nIM1/ex95Je5HfK8OzdgsPjS7g3sB6xgQt6E30OcMUNLtgT3skIw6HorB4MSxbhWtkO7/LxET6XB30uG18+s10ZFYthxbl0SEVfR6LWyysXO9VzqFnvyTEPQF1QQxJgQKJ8SH8MnktbpWdA8bX+DpPQohhMp7Wr5kxFB6uF+fsPvXfoZKrkLtVqAwdvPrxc7iyfXh/qILrLUO5a14yEkQUocwuMIdPaKoKJji6CV05DK2ejNfXLhKm/5N9aK8ZynxcJE89yV6fsSy7tIhsUwwzl73LU+em4BUOu7wsnJ+ym+DI6URnWEgJasFukCHX2IiwGSm4kEHW0QEULbnM2rvPE644j4/5KkPrQGsN55E9zxBfdJmuSRqsCgnSBKix6fioE5Z4RhBnGoT26DE8h2gojX8RPfBk9gJGRG0CwcF+uwAqC+bRIyhus1ATvpajo44SVx5HRn4GbS/3oB6cjbFUTPbF3tKxOszFqSkCN4reYCJldLhgco0Hhs4MqvT1HCh5Bm3LPiTVCiz+A5i0dxIXB1zEoTCyPnwnPxrjePmrd4jqJ0UeKUdr1CKVSEksTSTrehaTP51M8bfFeLo9cQkuwgmnGjlTtaJfddOiYXzNbiZrXyb9Viq6YwIHBRj4X0B291htXACmyI9jlcMo6xBmb51Nj6QH+kLN6ZrefVcvXo9R24a+5yTd9ceAGA6N/5yMoGK2zN+C2qpG5pIRVaHG6d/F8XeuYfplF7IHZLjkd+KMXd5dNAe14BEaw89PfveP66mKrsJT7YnaS43JDW6J7R/bUxv/RCHri9oXFHGRaBvvrC1OezxAaWIpgYIf7m43c0pepEjWQWH8atQWNX3LVuN2ufkXnEuQioFMg8MbpbeDeW8cI+S+U+y5y8R/tWPZc8gPNrEkM1PMnty284rLhFXAYNNk9McEdH4SCvwldPh1IOtW8Gfha4yN/ZWbTMB7+CCSZSeoVUnJqviDa7H3ivdsuEVKYQqhLZ40BwrU+f5BkesO81h0VTR+DWW0mzoRnE6Kch+ib78IpAUy+l3uR+pZb4zPG/HoasApjyWwZwKRkRt41gfsz9zPo8esRNT+Qu7ZXKqjqmkNbEUKrCl9gOeMf+Lt8Gbu+qFYFFmsXLay97wTNXJGpnxCu2sai39dzM30m/w9V9QttstbMcmrkbgl2ML9KZ3ZxZteNXS44WIESN1uBGQ43CoEQYJs518MU1YQbQ4HxBhOmLKFpRNu8HSDLwGbcvhNZ2DrsjvarG7B3Uu1+OBvD+KSuXj9qx/pFlrxneBFxJAIhNeuMq42hz1T6im0u9GXw3ulo3FtGEL4K5c4m3sRs6aLiPaf2Bv4E3vNgLYbtN3IbkJUdRQqm1hdsq3vamTI+e3dNzHXSmh+9TJlMrEA3HI7jtDoDuT03uXIzpzEY3YJQfkpxA8uJaj8fQhMhKgFLO70IDWokWCVDJAwL2Q+bomM7fGFDKgIoyW/BYkkArdE7IwGu4Fle5axVb+VG5tu4DA7GP3JaNYcX0NpqBy5Vc2n1jvj5n/V7Iu+vhN9exWCVEbKhOeIjVvIkcO/8HzDBuzOduy6FkwZbugLA0OW/jdt+/9rJpFI8PX1RSKR/I93/n/B/pPs+4/9HzHXf6F7G57+AmkebuYt2cWnxmQMHX70OGoJ7bgLotbxsBdkdwQg/fYBWsa46PHtQOctBv4PmkSE0UqPGr6wBzE6yMSnL3/KmGBfJEqB60E1+BiH0HQyi6Z9g2HyGfTehbTqpbhlbu5thlH6XGQuLScsMMULDKkDMXXXscjXn/e82qDpEO/pu9neEoTgo+CHJ7diUpcxVwdP9P+J9RVTSeq/kA6nC4NvJDE9zWRfCeBSX1jcLE7zQ6JcZN7IpDGkkbO5Z3neW6zqzK6BazaQdTvpc6MPbw6qZNbAC6QXpKNyt3BlezEBSGgeXkaTupsyu8Ap73Vk18BITSa+USpkgoBBb8CqMOPRo2aYGu71yaPDbWb+2UFEnx6I+4N0BhZuQJ5wg+SmLWQ1tVOlep92NwQAdeZy6v3Wc02YTFNtAdPD/uRS8V0Ym97HuNDIFeOV3velq1nOExfexscHsmfsYOUjK3urOrNUoHGbsDj1aNz06ok4ZT1UqK6S1w3eXT4s2DCf5mw5MZMu83kA/H5hJ6udN6iMqWR5YTRzNk/Ad4wDbaAWt0yLRdPCI7c1aU9lLOZscAUxfe34Hv+NsQ1jCZvfA5W7IWI23k4bE272QRfm5FjIWcy09V77tF3TsKRcQLowhutjnkXidmHSiQuGvXVJFB+Yx+A+YykIPt0bJ/ztwd+Iqh/NssCpJGbVUuz3IJGXP8WisdAa2ApArf9v3Ny9Dv3VM0T1l/Pi+RfZNW0XxSnF1EXUobN5sO9MLp0nAzE/HM2mUWIi0eKGTzvhEV0CuypDqDnYh+gRDciL82nTH8YuwJZgCLuxiLtHwc9x+5FrXiRj7ydYWyawZ+oe6iKKODPWi68mxCPNg66Q26K6t9N5lTGVHBp7iBn+y3j8+8cpSSph892biVeINLiTN8+iviaIK29KMA58h5+VAnSJ956ltvPyU89wrOklrsqy2JQrXneSAl7RhmDrGUP5Lh3KUF/Wzv8Ou9JOu2cT73GdtTVqJvkn8Zx+BZ5JEj5z/RtqSivq/7xTtAjpR9EEjXdxs/EI0/+S8cMyFTa1jbUPrGX8tVZkChm1aRMJLTnI7kmrMeqMnHAZcLzwM+dbJvPI+WG8uvJV1ty/hqqYKlxAurqFhRtformvibx/W+yN1oiC4cvLvGm8kYAyDSwaS29AYYYWFukhplNDlbaMWZ13tETMHmb8ZbDA+TMb1Yvpuu93xiU9h8spILZ0F2etUKAcyat9mrnkbEMfGcT43+/lhefddGufIz89iuagZl6Me41b5W7aIrJp1+ezbeK3WDQWzFozZ4ae4Zf4Y4z6YjgJZXF067v566WvmG0fT11ELfa5eyC4L+4CcUEjSAQW+7pYHgnXM2o5b0/CWOjg7s13s+7edZTHl7OwCVJcOn6euYmjbTpuWJO5FV7NuZQf8HCkUWCAuOBApjgX0SnxRKPRYD19BV1TO6/YzrMifyRZYV5M3T0VD4cOz589qeszhZjjokh2Xk4hGYPFgE/l4UraitvQJtZTF2nFv82fDQs3MCrifhw9Fpb82I9L/S5RNxU8JdDjhite31BlOIFxgJ1Nfb/tfeZtsWB0wTstUrocSprTq3FLRF7UZifMa4RKm4zAARdQSAZi8uiiNaCVK9rJvO0fT6TaSA39bx9NAKeAxC3BKfOAcSfB7SBm84dM9GhA6eyPwi5BcN9BaHobMlBpZFT33ERrsOJhFnl15+jgzzpPml+bgS1nDfumHie/7yaiGj+nySlhUoPAUbcvz+4bxrmUAL6JNfH17Uo5q8Ob8JYnsPn7UN53Lt1h59jeA5/4ydHLnKwMBJt/G++13ItRJrBR9Ra/V8K0UGNv36t3GXjEr50bqbVQncHTK54m4JFd4FfRe69bG36jrNSFqraYpPP7cGcugrDJdFcVEae/iMuRg3TwQIqEgVxImIDM9yC/Byu40hbMGWE7g6xDcPQsojLgkNhlJRBiSyJCY0AqsxOoSWJj9XMU6sL4w1fUc5uphbd/nEKYxRe5808cZWa8l+xnXZCIqG23SvnxsR9ROgIZf11Cn5qf+GhQP9Kk4FcOUqmEEeWv4nvlPE0yLbUX/FEkdzPsNl2R1lrNYx//yMrWAOL+XMawy28T+tknPBxq4cAfBZzf9wE9zzyDxqzBJXNhV9lZ4Akv+UDhgWT+H/beMrqqs333/S33rGTF3T0hSCC4uzstlNIWWlrqBnVXWuqlRlsopVBKC8XdJUCAECHuriu6XM6HyRve/vfe5+wPZ5z/GGe/zxgZhKw15zPl0fu6r+tqKgLRVCuz9s/i4LSD2Ae42XCwC/+wrxhhLaPOJcI+6HN+zBB8xcKqI9D2aBHLnZTsHYd6ZhfZ/bMxehmp9W8hb2YHKbHVLAzRofEtZpt9JYY2AxKRFDTwlKfgERMZ/yXvLLiXCA8p6oo/kfmEInN44hD3ECMfTlHnKSoPBePK6CS9yoy6N5GGxxr4gjhsaT0s6S7Fq0ZBfOYWAC5Om8pp1Tl07hYiqybhiBMzMCeUuJx03nvxPa6IbchKIcyjGd2C/UhEE+mKuUWHlwCchUkhWAo5w7K4OOIME9zvk5nWwJiYMfylTSNcXc+Ihg84GAlfGeHU7iR0xkS23LeFRgd8ZgSldwwf5zSSdjaU8IxuhvaeZJrlHIMCX+cVVQMiZy+LE+so8a9D0ZpAaUkcyfVLeHZiItXNKmoKE+iubMfup6dUKwRQZMASnQAw7WkP4h63lOSypygP/A6z2sxqPcxQQ9fG6ZQVhHP+FSfbAqDJmc8ZUy27esBL3My+VHixoZHk357DZL7MziU7uWIRfMWktlgy3UcI1v2M+IHcPuWAVLnQfkvcStb8Gk3azWYSPfUEVy+iYEp935hQIgrBef8fFBWso13RQaPmTiZ5XkoROp5mcrCWS8MuMe70ONZ+uJb169aT2+PDoh4JvpJwxpTXElx0iv5zwvkLiJEJstozrXre6xYRs2cgPk2jsPwoZk8vSErhp9Zoqr+6i7Ax3SRcyWC+7o5bxskQOHb2EE0nhxAU3ED/ukk0+TdhNBi5YYUSu4VhXoMwnb5IZ5gb19BObkYDZ2feno8jGGD9DE9nA6dyHuaV86+gn3mRCYtPUlPpz82tE3EPb8Q9Co71vwP8FtrhvaxQLBvvQTvPwlfSmYTU3GTwRCFZ4oEBz/PMmT/JbR5OUkIm2170YOCMC8xaeoz3jVBmd2GLgQO98JzJSvmICuBLnC4xzVXPcU2sI9GlpanDC8MtJYX2KKbME6QMN/rBPnsBj19JYv5vU7EOlfL1mq+5ruplgt0EZ+fC+GNs0BqJHJSFXeHfd92BErClFbAt5DW+8XmVEaeDiCkTPJ260/L5xg82P7SFio5Ynmgdz/RD08lYeJIV/+ZBZ2gzoLQoaUw2cTl+L7H1/QB4VA/W+uFc738De6Ce0+Evs15ezVxTKFa34P0SKOnl/INfEda8mn7V3xIbC24uMVoFi3S9dIlE+DX7EV8UT35KPYszbgekkl+BoGlAEVtWbCHVZxI2uxOzWgCiLfI6yqNbcXjrcFrsDLg+gBbfFi7HT+KcGTpdTt595V0SbctJKixlQFkQ54fn4JI6eacdXgm5zPon5rPJ5GSfOxKHpAetawPv1G/jIT1USYuo9y4hSPYcZ2YmMWOIDthFqQ0umKWonRGoHW4kTjkSpwTJ7SBnoR3cYj1DFelIt16nIzyKb5Z9xluB9UjF8FnmzyydKCa/YxFl9VY8m7Yx9kYlf8+pYNEgSBmWx7PKBob4LePK4Xfo6TlJzoB2RhXf5NEDeawbsYDxIT9x0fEuIsR9LoaLtLA1AF7w7UdjVAaq7gYWHFhAVaoNd68X0R/N5cIYD25OyULmlDD5vJ6bOdHMeGA/WRaQl0BCz3xmH4jCqdZwdKTgw/mwHiIzR1EjsiMRi1GJwCmCboOdC8OzqHAHcfHEs3h2j+D16f9a97oJLLpK/M0LLHn3CAkRl/jgx6+Jzm7BWG7kwe/vpT6oml9W/MKRXhhbC/Fnh7IoNxmGmZl6aCqFCYVURlbyTacAYi5IrKPI3E2hNZabVuhSGPCWCFYLnxYOwPbVKIKSJXgUX6Uisg4zPvylbeEXM3xT/ChNPyoJz7CxvDKMrRHVPOqcz77eX1G45UwzLOSeF0rQO9qRaHdx98A/GJd0AYAeVwi+dBDvXYjy35gTLw/6hK+Nb3Gj/w3GJ42nvKaHfjn92LxiM40BjeTrIMQpw+f0YHL8rESG9XIgAvZ0S5jXKMQFZsjHoEl20z96DhsVL/ed+zFPeCXqD5b+NpX4A0MxrdjLz/f9TKe+E7lNjn6EE/vA2Xz95xnCbsXSsr6CbyZ+g1Vhpc0FDzXB1LAUXpyziIr4GESGGXgXLGfluNssot4KxFo3brEbl9uJRqJF3DyZe8w24lqep/+FAkxXL8OzK3H7+tER6EeLZ2bf9clsMuRmcDrcVMdFYpGpySg6xj1+Jawa+BTdzVq+5ho4Uli2dRynJ19AK59Pd+MxDnbd4hg9SNSFGKLvpcHlj6ffXgKaBdA3SFbPjeCHaGr1onhWDS0eB5gteoMtNz9gy80PGLVyNdvjvseoGoZOLqLS8xpuxvHiyPkopb2MvY1f2FRKbvQvoNmvGTFiXLokSvTnKYsx4tZI+P7B7zGrzBgNRrzE4CW1k3CzH2qThrwZ21iT/CW7eoBO+NUfjpjg5eEX0NqDMGusdMmFOMBNG4yqBXCCvgs/o4t5m1fQf8x1ovrLcGy7n7DoSjqHjiDx10wq02S0+zbzRZPAjsoOgw7VewS1pDEt10i1bCS9EVGsGPUB130S2N1hoNcdwMvuoZRuS6RGH0l9VjP5JENII08W9kd28hcK09zIzrXx7O5n2XHXDupCBC/PqU1OnBknSZdPIuKyB7nF/Zj78B4MdUFcsus4kBxE9eDvGSoaxtoGJWWJpykwwFPzEmmXm2joDaRktBmXQkeeDcQiPR1aoS24xGYcWk96vEJJvdlAWHUKlvk2TEphjSMSiYjgIM6aQtz1OtAnAHCtfhodFn/mIOLjxHycKZd5krVIu3xo9luBU+Kk3dCOQjQVo1mJakgqZWfr8K+4TOOIGl73gLCQC2ytqsZqtLD0h5UUJpRTmexJnQO0vYE8kTmazi4NzggnGVcyKEwoJLgumIknJrJp1VZaq5MQ+1oYcmAIoxpGUTu0lr2z9zL+/lqmmpSE31vPYUUSX70vKBlErnmCa7c9DVMUvfws9uKXShMvrFvHWSf/U2k/EP5+wGFjhPMYxomNzD8+H8tp4UTTv5rO9ePtqLubmXBiHLvn/s79m++natgf8NJadLEBdIizcIvdmNVm4orimPX3IKz6SixeFmRd/IO5B3BywknELjkrG7b8D9dSEleCfqwelUHFrT/KEBu//MfnCpMLuakDh90Dp83NFd0OnjYu4OGZD3H2eiF7p71EjPkuzOVTSd2uQZc8g1sxPzLu1DgyrpTQ29wL6NAYawmoN1AZUcezF37Aw9wPPz/48UHwr85hiH0IVzKuECeDYSF/ofJtpyigDHGIkkm/LkfT3c2eeXsAqHeCKX8XQ659gigqkkK9FY9OD8ROCd6qWpalvoGnopnrDMY1fAw3M8fg23VH9enRtHXsfuw8iq4W7JKzTD9bSP0qP5r9hSDb8j+f4POX0jB01eM8fwF1sgciUShikZhbCYX4ByWjMqjIm/Y8Tifoews4aoJ6B9zs9iTT6ylCU4fy2ZOf9a2vPxk7nwBtGWvfXIhPmx+VQ0zUS4REuUgpbA6AlN4TXDlkIqdXwaUph2nzFvb6aXJI0t3E7/IQvHZOp2WJhdowqFX1YLUJRACZQ4VOMoIdm58m2Ksa4/h8pmhzKbZ5czsHkWuOfoRX/E6gM4Cgu45hULdQ54DQ6lDu23wf5ZSTeG8ig7IGoe3Rkpuay4BqUIi6MIQP5v4YFS3lhxiU25+D0/f1Pc+N2lwGDJZi1YaSOVRg0Qe2bxCICiIX+aEiGmwiPtc72JR6x15lRw/YHFLKDYuQq7twc7XvsyoHTKuHuw3pBDrtyC1deA2I4M3XJrNN7iQraCkebheIxORHNnIl9gprTStIaNhNVUMabZGp1GhvEBTbhjZQi/LsUabcuEzhEx506YVxWnyfmMeeewyVl4rBDw2mqLGI984novl5CaKXHTBNuJZ/B/uOHYMgr1A0nfW0hg0kGNBoAoiNewXiXkHSeYKTg+chVZlRWBU47f8Ez/9PLGKxmLCwsP+2+v8D9v2n/LeU/yrjubLZRWEEpA7PY2DJO2yO/o2cQX/ynp9g6DlDA4N9HRwMbaFFq2f98s18GCcce18TtLug6MJybu32IHmRjoLkv9nf7ctkXTu/TZzNS+e38bmzB/2Uw8SWprGseBnvv/A+Vsm/BiERk62rifkphZBZkbwZpsftEvH9iDOgyQLvIeyPbeL86+uZlXQM93kBRPkjEM793Y/grDhkK7SYT19h0KGj1MWNJWfARr7xgwtVBiquJcIkG5899RkusXDsTSvsao5mdT3klATTOlzC2G/eIU3joqdUxNdPf4pFaeHRAztwSRXkGs6w1tbMFOkZ/mUvbJCG8tfM55E4NfQrexyT3EiP1sTLbbC1eBVHp/izdnA5JxQuurP6MW13MVeWuygOPs1gTxVyuYnxVVFYrF6IqztZnLWYnAVm6uVhXM52MvysF8aKDnT36bh1SgBnXjPA85HvsW7rFBxVepIDh9JpkGNWmADYHgBKynk760Uc9V6E2nwI6MigMep6X9azxKVE4pQhcUk4d1trWxd6hlG/ZTC3W8M3E2+xaeUmUhqX8/zHz3Nk8hkuDW/uay9ttgaKY25CqBeJf4cSWKqlfe5UWLwDxApmuj+m6s959IzqwT3hn8Dyjrt28EjgaUCEXakj/cBb+ITp+XMqFDR2cylTilJRTmCXhoCaCZwfeZ5IQy9fxh1guPIIcomNAuNnrNiygpLYEnbcvYP1PvCkZw4vXWtC7HLikslp923BqrASVhXGC/mDMYy/wcohu1D2U2LQZgOgscRilTbjkHZiFSn5umgBY875YHG24HHxBu7pBwiQwBzBWoS59XDK4gapmIaIGGpC73hL1Rl+Bab/l552G2jVDkbh8SkqHw/OjTrX513wkgHu94DdQ3IpjazALZrMLpMbhFfJz/6gE3cglpZQof6ZTusdqdRgKdzn28Ae2y1ODb6bxDQ5Lbr19KgqiK1/nTL/TyiyWpgsVvVdS/nZcub9NY/MoZmsFDfQZVfxlN8ePoj5AkPAMJDWYFa290n7rdy0Es+encAabCpPlL3deHR5ME7jJtcGBysXEq8fSGxCBr8ln6NHK4Ahr2Q/R0PkX8x5aA+figYxbu9gPJpH8uOqH5GJQCeGIUY/HAdnUO5l5vvV3/fd1wut8H47RNvmIWvIIDR8LpsemEtIXRDFccXUR0BD71382a6hPLocu0FM6j1DeP/EEAKCtlIaC5gfgWRwy+PR+huQaMB5Gz+qiqiiKqKK6oZGuiplJJ8VgkfZj7sxaUx91+Fyu7iWfpPYkmjkNjkjrDo+SzjK/ZURbE4p4FNPGYZmB95aQcZQ9dXjbBtVhmngcG71G48i6TBH5v9Js9+dfuNwiXENzqWlS0phvbtvc2pG+I63pJutiZvZk/sMUpcUy6nT6BXl3FJmUWYLYqqkjpbpZyi1CsEdlwtyhw/mVuAijAYRSbdjWhlPZtBR1UFTuxn/aiUQy+hntnFWKkUsl3IrxUh1mJC9+a4RNnXB8p8j6PZsRvPow5xzfNp3zZu6oLg6Ho8P7mbgiHA2jvyQHpXA0ux1w64e+MzHxaSHDzEvczCG7vlcixc2aw9lvUCocSmrHn2ZrxriOVjvwaAjaXR4jmDyofngtELDUYKbPsf585PYxnhweE4ddd7b+uqPu7oLXUct3fOH8/Hzd7w+n/GEAS47+4fk87uvmbT9XkzPewbzV2YipCKaXW4u5K/CebUDf99gfLW+dBqs6L27eLXdwqKJkZxomkpnvJ2O6ijmbl3GF5NreWWE4An4WQesTluByG2n7vvxpJes5anP3uVrIK4KRqkn8UVkEGn3nePkXxNRlBVT7by9wZUbQCynst0L441c5EG95I604+l1DKIfRH36Gbx+O0R2Ux5MeEB4lyI7LU4YfuwX1KJfePrgHKRzOpGLXIzt7c/3ZPOqZBTrks9B8v0cb4vgimMR3wRcQ+MhMJKdwNcdMLTVB7XJg/qFZ3lxdjGxrlLiPQSA9OWmHrIc0IID75ps3FIZyWIhkeQBlTceUg2z5u1DPauaD0yz+G3428z378fyyw+B70gSvFPptkYR3bySK0EmelNu8aJBWE8ERTZQldyDVCVl7fp1FCTeYueSnURIIU4Gr14chbrRgWOVnF+W/0KrTyvxThdmhw6LU8nM137kw6YwNI30eWcWDqqBQTlc0zbT8chGBvrdw2GR4LcaZJ5Ev0oNGUWp6EU9XN8XhW+4hSeOP8G14V7sm3yFZTrBs88pFuNffhmRUsHPT5ylUX6Bu5o28EzUUYzyPH7SBnH4qU+Jly5D7tmFWZbN+K7xiJEw9tRYuppL8c8YQo9nCNqOWpIvneDyMy/TFCmmOPoc55Vvc+HEQRo8Cwky3kdlgDCmfeEL0zSwoMbG3wv+7Gu/q/XC+P9oo5NcO4SbxqGqySAjFbaol7M/AmzBz/Luxpk4LqTjEdWE0iyMy1UOeLoVnvdJQNWbRb/cfpiVrQxo0yKXpTMxWvDBwiMeVYE3dlkbH0Z40RibhNjt5tVjO8kIP031opUseyCMnd/fAa2cCAC+Nud1FhQsxZnmYME2A8Wxc9hx9w4ylDBLC+8NyKYmpIIRxp9IadzLrKQbPH17R+9fDgfqdRT1+jCkxR+3MYmR/lM533SYLCvEO5SMPTcWm4cnFwfV9NX9dKvwUzglEKvCSo+2l6JUPRcydtJPMRLqj4AqgEPSDDY1XSARV5+0r9ipwiUx42t+ApFmLmJ1I2fGnmHc6XGozWo+8dKSao/lh7//JDrCiWlgJ+eNaYwN7gIjhFZCWu845n0+Dn1YAFfjjlIV0kGq6M4G8VW7i/7pV7GER/H8px+TpBWyxH/rhotmuOTdimP193g3fsaByW/T6C/c29R68LOGkWe8H88ny/g4byfKNi/OGXoZ5WWD8KXgk0Fxx9sUHJqKOfEkrdFlJBuEQMs5owJlWQSWJAvWygamHZrGjQE3aAxsRAR8brEyOaQGjYeeoiPRFLX4kTC1GZ2jmi87oKvMyMAru7HrvMlNyWV0aDM5Fg1ldiEI9PqJJLrKQggddWf+23XrRe7vv45BxjHEpJ/h246D+Ba+zIr4z2gXBWNw17HZqCYjbibtvls4OO0gBt3jtPi1MEF/e2vrI1DOSuIryE+6zthbqxlYOZDrg64zWyvIUb/YJgQyPIx36tanlPKwHh7WW1lW1QY+Kma++DPR/h0oqjPYGQCLdLBl7yQqs+M5+Fbd7SMFYOsrPzhKJlPm5BDpmkiSScIWfzWXpbn8DahEME7VS7QcdrcLAU6RSEjOWtoIq6uiqZ5rZfzdx7g87xhnG0dywTMHZ/gGlqQJckjLPfezc3Q99xjbCbqaiLurgoYgQTr+13u+IKHtOeb3WJmzdw6ZGZnUhtbycYfQ7/8KhHeqbfRmZzHu6mAyhxzHKnXyapsgcTeAIPIln/DQptlIApoZk3AImxzmNcDSdi1jsmaSH+GFm0U8Gy3MOXMaoMis5Mv2ZfRWdbLfz4+9S05zT/wQkk5voCH8WfzEWsLdP6Gae4bDnYMBFyY3FHckcKbqbpYC9WXb6X+snLZwMdgFqcMnW0AnaqMwrg1UUqT59YzPGs+VIR/gRkx1VzIPH1lGxl4XkmW/8ezuJylILGD/rP0U2+CbDlBFWWhTRCOlgdS8VFRROeRLhfbt1l3gKdcCituHcH7mOuKWfMc20UwanXtwAwqRGP1kPd2Ekz0gG4C/vED5+C66HFrU8pfY1KKj8dxyto5wsnjZMTSOAP44dhQHIHVn8dSOpVgcMsrCbhGcWoe3l+AjGjmwmpyEdUSM0dIQlUx+tDC3tLtgTN0GHlzwKa8PyCO0I5mhl4fSqe+kMrJSmH/wYHNAF/sqH+D9nDpWG6UMW5ZApQOSqkDXnMoAhRWnVMv+J75msqcdu3MRz2x4hltJt8hKG4U8zIyhw4vg9WvIu387Vlk0r7aB2hrMdLEYv9JMtMYazrz2C19EC8Hiw6UPYtN6YHAdYrnqRW5VPsC6zc/i07+QuTMT+VDnZu+cA2yYuIGsDTeZcHICf87/k8rISsYUx7LHy0TX7sloR1iRXk3mzRsTeWjjZ3iLO5mpgaf6XSd2zC/sZC2iQqnA0hTBCZPgF5sfVE/FlMNMnjCZ7Se3I3FKaPZv5in/d6ANGAuMPcHisMVYuoX+1e2CH7rA2V1LsaQTY78jDGy9H/exGMpibtA4YiuD+ocTc3YbRVpP1hT64Thu5ZEvhzPmgUuEZTzIc5dHYu/9F/tZKG7RnVjGxOMTybgSi31iL0fGfUyHNpP00j14qRqwuxS8ev0c9ZZY4uKucXTSJ7REdhFfB99kfUOj5x6y7pqHj3kYi7oEWb2/W94lRiqsm1666U/k+kew9mvk1oBx+Ep0hCn6cdsOnCZpANlWCMdNTZKJPNkJEmon4XaLGRh4lMzLSXy7N4OW/g4OzziNVdaExO1E3nyU+tu34KHwIrz7N3J8VgFGvvWDxboGFs6xUmrR4YeNVIUgKwtwV6Og9NDcP48WmwlVRwieYk86vDr6nsljGgnjrHrecTqIWLqffhGt5Lk+w6XcTq/SD191EV0GMx4thcRduc5OdTUkQ7MTLHY9RaZaenoMdAZW0R77MJdkLVyWfYR36l+0W/bR33GVwspESLVyfFozPwWXENwA33i6OahwAG7cSjFGL2Of/9b9HjBFLdhYuN0u7BNHsPa5EYjELtqmnKaiYDhTdA1Ee8Of7Y34VlZTpIkjMLwR2WEne1IL+HNeMzUJA9l4+AGO/taPzNVDQDmByWpIaH2Odx6aRrkxmbuuqbnoOMYAy1xivS/xqS/cEOeTEbOFcV82syFrBiEnvqY6NoTjVUs4VXkvc8SCXyaAy25HYnfi2+LL6u9Xc2rcBbrCV6HxhXzbQVo0f+JPGA26z3n19FweDn2YWOf72Owr0XfqmSeR4h19gzVGaA16F+sCD9qtEgKmP8G77ncxq8wY2gXp1A1KCYeffoE8z3TssjBcwR0E4YlY4uJx3yzoAGV/OCDy/bceIMIO9LhAZpFz+eUnqY08whixlAznP2Mv/7VkACNFEq53lPPJ05/w1uS3ANAF6WgPTkVdeIKIyiASChM4OO0gGdGCjUv65/fwwu47NioNgQ0Upo9lREw4U1/tz/3ez+Fw/Y+efW7cSLqMpNenUx5VTrv3HcWIaK9oAHJ/vMrUY6O5OvhM32cpVwrxbbxB0e45GJ/5E9nsLD5r3c/jpQ6GOpTCe8KFd5w39cPm06kUnmdlRCWhrQ8jU8sQiSDi5t94F49l46O5iNwycDnQ5lym+KAvmsadjC4ZzZWMK8zVwkujBHnt4ybIw01S8Th07dXsnXOQ13xsvO4Njb5q5te9QkPVW/TubMXglKOPDmHCUsHb0KvkBqazH1Ds8TSg7tvTAEjEYurTpuG02In1aKYqvKov4RlgQNBgxGLo9QpGnJJIeN4hvt3+CdcTcmgKq6HesxuFh8BO61EWYpZXk1i2jakV3chbfyCxvhPXYHHfWDDK+iG5Db3YAw6hSmumuktO4qhvOWwRfGA97aEki7v4vTKO5sPD6JzWSFfIIkpChZjTEh28aIDK2GaOprVSo3bxRuoXhMshuILbRIB2Pui9gU/NZaQlHpx4Mpetoqu8w2vMuH1fNqeCBeufpjXBzV1zbEyPyOFKYzgNql6K44oZFDIIU5OJWftnAdDmYyL7X2HioJGUtt/g8swy8qdtxylx4iGGMSooCG/gYFADqRFL4Tbxs8Gwi3EqmK+FrL2jqfh7HMq1ZlDfaZOvtAFYGJB8A7FLwYSDI5leHskXT3zRRzaQ4CJp8iqyhqiwJs/HfcZFiQPELjNk3geh80hVOKgAmkovEHnNhW7CeWLo5cmSCdSWeCNVSHF5edPrE8WgsmOcGijA8C2KFrxjhRjSuC3jANgqNzJgyGUGhQz6H/rQv0p9wnjqE8YL/erfgECJ3cKAc+cZcO5ZstOy6X+zP7ZVtUDU//Jc/ycUl8tFbW0tISEhiMXi/+cD/l8u/wH7/lP+W4qfKoSQlvuo9d0MQJEdPjGCouRb6qRzQP8bALmOOHzLYYQSCm1GypdsZpL+OeiEXCukKqA+CpY0QIPHeOzxOsTSCpLyk5DFKCEIYuTw7ZhlxBQnUBBRSLefitzEy30Lvxe9wFdewPCu9Ziqw5FcV7OqbBmZQzNxrEiGaGEy2tDqT7usmWnddhJvhVMd0sV9uhZiem5PoDIZYn9BvjG4+DTXJ3fzsB56jH7EHJ9Ed4wda8CdDIeDJqiuu5f1VZk0Hc2gNd3F2Cbhmjw7PdF1eBIcWUfGtCvkto0jrljPjfx4Zi89QYAElqrVZGg9Wa2D3d02fH/X0WNI4dYL2RzphmuGTVhdX/FjaCVFQUWMbH6FlrCBtHvs4uFm2OMxhmftN5h3eDoNLgnEWEkqSKJNeh4fZytXk+opcafyeKwPHjUefddd7YACk4Hgup+xFgegF71KaNNcyoK+xil18kE7jJX4MbykheomB5LAQEJKkvnzGSMDFbDVH5QhdkLTv2FjzVm03xlZ5d9K3igxs707SfLpZbqvkipDIUa7J8VJNuSSmcApnvUUZOJ6jD8yc8ir/NU+iNnPnWXnb05GS3V9I1qBwg/t6t9o1s3g3u/uQmmW8/lTgp9XaWwpYs8AXE43htsG0EbPKs6YoVUsIWfcE4ydoiDgrxPEXAnjypAraMRWUpQuPrk2n5oj9xExvZEjU470eaUV2uBQL0yf8gvGEXr2KSezZZjQtmftnUXz9VQsHYGY7t5Ir7aXxFwJ0bVTqY+bhMz/OBsTD1EovkKO95/cnGjm8TUOnt02m8rgUp7UC/fUqx+M5VA3C2+k0LHQyHf33dP3TgxtBoae86H6fDWOg4X0z7xO3rjHcYsEYFltjUZtC8fttnBq/Km+437rhrKORNZHXscRIWF25wTe6w3jlk8zvzotLNTCOTM8+/lknEol5sCtTGmawpGpRzhrhhkHjzIxqZqlM1eT434AZWsQPaoCtJa4Plahh7uHp49NQ1IYT2tEK2k5adxKukWIrIEZOgtKuR+WYSlooqOJmRbN0sBxfdcXWhsKtGDtsmLW+XJt4ixqQn9gg1bMYk8XD2xZR1CsFs/HIjgzsYZWr1Z8OydTHvAxn9phRySo7BkEXLYhdonpV/kDRyIeJLYK9se2EPXiZt7W3Qd31rg0OEHUfDfrO5q4vjUY6X066sIaqL0NTn3fCbHmWTzr9TlmN2Tixn5bYlgsuuO+8Wv5Vn7Puoc3ZoKxrB1Fzz+lZFpajCi8IrCoDcisHSgtSmQ2GSqzCu82b+z9bJTHVnNi/AkmnJxAXpuae0sjybT3gBR67J2MOdRIQPxEChIFED2rYgi1emFDFOZ3lNx+ufh1TCe27D7u7v80bVYVY2pB4pbg8W/ydpPUsMUfsijg2y2v0nneQevKVkx3j2W3ez1qEXiIoZ/CyJSJBXzWOYOOyg4irhylJkJD3eA6Bl8ZjNt4EJhO+OhwHsp6iEdiBe/FZWt/xS3xJyjwVSQqOD+umXrvO9J3FRGweUg+e91qxjZ4M+vqBmyzCjhi3cTDzRDdMozpQSkEhMT0MYX/vbgR4QZEuGjT3WnfdpcSm1OFn62S+hMj6dfoizkiBbPIiyCVDq4/AyUbUajUNCS48Ah0E97yMF49w8kLfxSA4pgcArp98Pkvda5phrUGC6IH/qC2+hmCrHZ0ke08INnJZ1Euau3wsvkE7TP2kD7UxprX1nAkvYDFT+1ELu1lvAqigk6RqLQyrERBfHV/ZpMDQFoV5NhAfGwVpmtdyBLqCQhtQix24ycRAiEAVbLVbDq5GmPwZS5MXI7aGsm6u52ACEQirnW/xZi8MC763eLoxJ3ESiaA6EHWtflj659DSHQMlJYTUFqPLEJ4riKkPB9cTp0qnDKRDeOWFryTkzjz9SkO/JTFPtteirw3ccZWSVp5EWe+SWHGA/twhDzCdvU3SFwannvxEDvqh7HToSQ2Sdj53F8wmZ8TjzJcZebBkkRO9QRRVvE7ZpWVdZbXWSZ6k4+C2qjDgpe2ER9ZOY4G6FV38XPoeSg7D10FKIafY82vZxC5XdRP3Uj02F00S+CLVsiPyKVc3x+Nr4a66DDKo/YD8L5R+JEtfxeRW8Sjqs2URwsMSKfbRbCuiA67juhKMNg9WVmQw4za+zk+0Q8vcTnzSlJJiK3ldf9m7BoX3FbckYgktN2M5pzFmwZxKn7VpaiStNRFj0AlTkLqeJWZ9V2Eti/hz0FF+M2XU2MbxaiuYdS2dhKiayfV7xxZrgXEageTaVtKgHoUZ1M/o0N7jWsXBRB05PmRmDurUc6ZQuHIlaTvfxN9h52CcFCIXZyuvAdJPMSGbKACMzrLSTzF35OuhKuF8eQeH03I0EAQVHYZoJ/Awd4TtDjheYcXhRURHLOIcVVdw9UVD8AnHdDr6iTbq4GopHzOTKmkWX8n6xRA7HbxTn8Nmc9XoGicgP/WE8yb+if91FA75hQhIuh3M5HhFwfQ9UgFbdpWpA4PzpZP5kbLQkQiES4XqI/t55W8V/hw3YeoHMmcFGdzb5kTz6vbsdStoj4qieK4PazzAjkgLgF3RD5EwJs3etj99wKWrf0VnVc3IPQP3YaHiPO00jppGcWmTMq6NiFBkKeLk/cw6MxYahKbyOwvFjxf/k2WW2IzkjD9AtfH3qC7ei3FQbcYIBkFp6dC2GJEhDLt4DQkGgkZ1UkoemPY/MA+cHoidQprpX9JATUENNBtiOMp6xgGBx/getk3WHM0XBrQy/GMt3HbH0Pq8MAh7cKr/Fvqw+sITfIgJ/E72pXX6C9aiK8EUuTQ5ZlEboYv8Z5xzOrp4ROlHHv7Gt7q2kiJHUKVImoC6xncqyCi+3eqwpP+MUpWyUZSZtBic9TyyOdP8uj8P7m6PROFTAUiEV+LkyhY8zXhLCPLbztVStjXCOOiqrn1/rsEus9iK2kj40oGVeFVNAY2siMAFsc2I/b/laW+H1FYez/WLgvd8ivsaq/miRZ4T1eOWyqnWePPwUV/sisGci0aZmkEb7L8vfEMvJ5Gf6OON98bzurPf2JJihDM29bbw/pWELskzK9by9Wmh5m9dDc7yx/lSGswl8YlMzOsiebQAlpvSdD0aKjzcID/aEgUgLH79VbaAP/r4XiXB3B90HU2d0FMTgqLLiXSc3cPPz/wMxKHhJDaECxWB421UhZYPWhv8sEVLCFdUYWmMQ6tU8llC2jF4DH0JvaQGgJyBjKvcB4lacKcOroGojoSmWbIodlsZerPczinsRD09mUS5HArHKCSs3tGkddloVp1Nw02BS26ChyAUVGGyGXlBYPgc1rd/AT2tntIWdoPjo+BsCU02P043CnFZpcwcW8U2Ffz29ufUd1dfvttuxHrFPy67Fc69Z19rSBVKiKpNBqDUcrJkUeoSDvW54cJcKJBR+jmBaiGtBLk10FqYDfTRIeY6SuwbSe5XFRdSkerdmIW91DUMYN4zwM87wUPmkUENx7hxuF4IiYvQme8i1GzYDcbKAiHevsZulSP0ZxopeNmBEu2z+CP9EucYjseLgW43TyT8iL7G9I4EuXHpYjNRInGsWfVEeZvSiNI4sbpcpE3tp7j6X9jk9uo9f4RtaqWNtcBFGnBRPoa6PBpp1snjAVXrHClBUz9NpAbmMlZxU6e+cUPkcjN4EoVPlFl/JlaR11eDn+XPcZZiYu3Au34Sc8jAQYrId7dxLrFD3Kw/Kk+T6NiG0yJbEDaG4oVJ9nN3rjKghGnWxiggKGejdx7t4hH9hcgdxgZqnVSYoWTsdVMX/gDvnqo1j5HfFAm11tEyNQyKpJTuJYijPkaux/dXYM47p+I2ScDIrVseGYDVoWwh3zNAGkyM2+u3YC73c64pA7k16bhcVctNjcU2GCjwZexr+7nq8rN/Jz4Hv0VcMZq4nR3GCPqw/hxxgoqH2hlwsVvSOg00umK4WLNIoJ9y5DbfbHb6nGMbKPeFEqtXcyfPVBT/hwn8j9i3Dg401OB9dt5dCSV4OtrYpCPCUvtpyTVbsC7ZzTRhmgGjArm1Sm7WPDXAnTdOmriknjx1lqU46sZPiaCs5lvYUgtYG3JVCJ9f2dzABwoHs1Xl39hyBQl69Y/R31QPb+s+IXTZjhthpTgRjoimkkqW8P9m++nLKqMrfdu7WvDUgQfPGmLlICGAJr8m/rGd7FIxHtN/nSrWjkwR8fXa77mfY8ujqnTwcOAQ6zn4olBjDo0CtFXIrrDYogJ/4G/Gweze/5OEsVCEFh0+hT9L18hJ0hCu2DBS3VYNTpTLOlSKW6R0FAmjJWyfc8bbM97o+/6HDo5F0dcRC8ORGfaSZvuDGm6Ol72gr0Oyz/m2EKbANyaXWYGpuZg9ZpOVP0SPM1LSM7Q0OV1HX9tOT3YWbPxEToMOhrmxeHXMRO1NZoPLqxjaNVuAnvfR93qjcjp6NubSURORHID8j1Tee1UINZ0E3ZRLRaJIA15sFdIVNo1agvbct7kQJad766uJGHFXqCFPb1CbKYpCl5o7KTtl+kYmoeyft16IqRwjweMaPPh8puPEDOqFtnSfZSIFBidvrg77NxqzGD1gB+5a+hhRvz+Do0e2X176sl1kFSfzKhoC989/iYxxodx2F7BIqnn7zBB03CT/SzFoV9QtcBMo0c354M2sslPkJUm9CaHk2/Sc+ZJLIO0/BT8U98zHaiABUoJ57YswxzroC51L56aNmpd/lhuJjA0K4Khg7Mpee8xQkf2knp0Mjvqyghf9iaquq14hXjRaJXRZgpBaw3Dw5nHUPlNcIv5yteFWPMdf5m7qJCepTzCF5ushXSxBJUYwmVQgI0X27Q0k0uY5Qqxta1Ejv2Be0Y9yep9FYCIPeZoyopDacz8m7ROC80ThcXbv8ZqiQTMO4+j6xKx924DQV1qyjtS+PLqMFTZATjWuvj7hU8ojbjd5qSwSt+FvqyLLhd0DFvJs2cFIMWkNiFRWOknE7HYH96yd+G+283r5pPUOnzYCIyrhWXpT2HWRDFZEULOrzl4x3uTVLOBNv/dXLfWMFzmIHzqeSqvFPD+/yab50GXg5euKaie2YZTL7TLjqoOQgpP9H1nyJUhfPnEl4z2WQ8ISaYu0Z05S8EQegLGIL5t3z1N8R4/m+f8o56U3BQSy+5G5tPAzOMz2bVgVx/Yl5adRsjuEDozOhn49Bje8RXS6f+VwJWZ/itJ5Q+iC9YjG5hCl8dJNvlBVNPfeIh8GX9iPMHtgSjfU1ITrsdmcxLZ+DSV0X8h0dSg1AuAYEPsaLIjfgZgcfSvSNTnaHx9Aps/7uH0xBbatYcB2NsDPsW/0aN6D9fG6agX15M9ajjnklbhEruodcCnB+CFHTBKvJ53HLY+P8RNFVuY+66VD+6CNJMSR2cnbXtO4NGaz/Vhd56HSARWT3+sVrAle7ItTUhw/WjSR6QHpWMsbkHbbsbiF4a7U5hLTbXNdMQJXp0u9+34Tn0h9aGbuBq3gX7NTxNrS0VVGkN0URguiwt9hx6T2gS4+C3vTZa4pXx912t8lJdND6AzJ9OtzsFa8yz3XXuS/OBHaHv6E8b7PkPo5ZXcug32/doNly0wnAdpGTAWS+cNTjzyEgsf30Ws1kSJRIjPeLtraFi5k2umqL5IgVh0B1xRiWzYPLpxKRUcvfUi08J2s96znWm+3fx+1+88PulxFA4Fh6ccRt+ppza0E28xjG8NInj7PKq9q3HIXZhFAkshWgZ7gwTv9N29UOuGpduW0q3rZt/sfQxQCIz0OdpmZCm5hNtmM6BkJgVxx7GohLE+QAJfj17G751qNEV7aFOUg0hQn3jSE5AUsdI7E7kKWprqmJ39IoagNghIB/8h0FPOHqmTjafGUxjuonDNT/yil/IhDXQ3CJELuVaOI20QZYpBKP61gQUe++MxykvKeWfBO6S9nobWR8ux2fs5FniIQXF3wD73fwm7iJwOpLZenDIlbrei7+8uiQyjrw9eLa14dHlwZfAVUvVT+T+9uN1u2tvbCQ4O/m+p/z9g33/Kf0uRIMPDnNb3f7UIXm+DNT+5UQbtYn7yUwy/toglr9Xzm7ew8Cvq8+wT8c75JUiSciFDYJv9HggvdpjIa5uBxl7K4l2LCV12ClJgS5ewMXtGNZweczLbAq3URt8JCD/uCW2uErrDFjDu7Qtcty1EUnwRkVuE+N/0dcfXr8da2oyf3IdZe0dzYrydLX4taJKyWHG1H103K5AnD6Y2YQKG+jzarHK8y+BoaiW89j1bAp4n5HoCJlVz3yLDjpgX+5/HEXOdRJ4npCYEvddAgq9YGH5pOE1Pfc+Y2Ts4V5vNNWMQex4Q/NJCpbAhwMQN2wlqf59MUI+cuuhhWFReSEQ5BErASwK4HHxhVGKXWEmKtnBpjJnrvr8AcLj0MA8MnU3LrM/I6Q0mekoy701ax2z5WKIsxYT5OTnTaULuocS4yshE/USOTzrO5i442tvIxQVH2FmvpfXacpb+ZGDfzP5cS79G9YUy9mmKiZ97PzmdPlTJgjg+ooBYv/l0OAWuWa3Fm+a2oZgcGuQWBc1mGRqRJ5XdUi70DONV7RdESiQs0PrSFXU/wl7qWR7SCzrWXa5bSH9ZwtSqYKQb1diVoJE0QWsV6BO4SRhHAv9iino+osgSJLZ/zlQOlxOHyUFUtuAfl5N0iD+aYK1XDSsXh/Bd96ukjXyLXI2V+MYULnssxLccwjuvcX9hAqWx2VwfVIhDKlDhf+oSfnJCfyfE187PdVEobEFY5fUCWNirQe79Jr/ofyCnW051uReJN1LZknoFPA/hL4F6US9zIp4j05GBMuBuipLKMEmMHDdBhHc8U1I24NvyEjH5KZjaTf+4H48uD/ple9OY3QieekyewbhFYv5l/R5VJCPhxia6SmegM6XSrRaAluMmuNkZgV1fALjxKnBj2/0Ab774C9sV5ejLhPOvqQvGrHXi7JYQURDLuXHFmBQVHIydTD9FKk/45vN3fQsPfDuJuqAACoZJec3HyivecMZ6nXAPFxcUdsIWhPFO1Ts4JU7WKMFH6gZXFQ8NfII/Os7xv0wEFEHqyc+R2Uyks4r1we/jKbOSUPYHjs5guu0Bfb4ZrbpT3K0DrTWciiuXyBigoWrOZa6XdeMSNfSdcn/Nc4RZ4xkQGU/VH0dp8m+iLKYMvRiSzCPRBOQSObyOarmCUedGMvxiBnvm7uG5xEIm2O2st/iw/7W7iZnrT214Lb6VDbSpXuEd0U6e6neJ69lNxJ/7gMasRZR+fpzYKiuMhRHnRzDgxgDMKRa0URryxj+Oy7SXh354iIPTDiJxSphydApeQw2IEHN18FXyk/Pp1HeS527i25K76Dy5hNA3Evhy0ucYvYy0+rRyechlNPINdNiWENCgx24TgvZOsZmygA940LeRio5Ibv46Dd9WP+pC7ATXxLL97u30Ku2U2aFb5cNp8RB8EpqQG+RYUstobmrmW19Yra9nj302zx0dTb+RCZiNFrwaCrAoE0jKT2LUuVHIuQZMp+ZiDXVX6zAGJBJQeoHSnBiufT0AVfQvBPx4L/Xe2/vewxCFIONZNOsIWZ2xjLpYwqCL3QTNf4oj1k0AzA4r4b7nvuKC82cm/jYIN/HsnbOXRDmcCYYNRjFvfbSYVKUYmU2GJCWBwsRCYjT1BLqysUW/zm8pfyOPCSYgYDatrSCjDko2AlCrFPPdXW+xSLKV6N0DcYnG8PrTf7Pt+p/smfIxAZaxpFrSSclNoc27jYagBm7aYNltH4V0ERQNb+OveEG6rrTXmxhNG4M96zgg6cUJBM44Q1JEK70iLzwlRqIrYXRZOrF/DEc7oYVXf35P6JNdAeTYGkmt/Ja/dX8TE+CHbfohViQL7Mtj/h680dlFqKyF07U/Q+Mp7MH/mhzF0FsN9i7w6scnAw7ieFXCz12hdzpS/nuM9epiwdy/eDN2FXxdTEjhZaRj3ChEMNLnJjU+Vtqf/5Ye1deEVFzBGdiGU6TgD/EFdjRpsNX6EtgQgDnOjCG9G9/gFp6I/4aAbtjY4I8PbkK7J+NWHe97zz8nHuXNNsgyLmLI92LsWg/+nP8DNrmRae6nebQF0twRvDBSxeOFqfR2+OIRUMgTGgP53ZA8+mUImQ1AeM5+xC47tVNcmF0CQH+4F3JtEHd7E1o0sD9ZyXdM7XViWO1nJ8vKPwDj35rXkTVxHT4iBdlH41G7/Gkv30pMiZrfFh5EnDeSW1eSaRt+kfu9rDTbrNz725t0eUJVWjFOu5SbiWqupH1OU/olVmr/osk5DwCFI4AmaRdRDY/S1rCZp6d9wMOF9xBw4y1Enb4U6gNZ9Ecvc+bAHI9fWeB7kxPd93HJ7fWP4e+bR77hqdBDfRuurBmvsT9djLobWk1y6i6NZrC+Fa07CIf4Bh2ayyTLBQ/ad7MzcOQGE5gQycRbE+nUd9I+rpUbFjAWf4p/sQLr+WZUqRVE5BbQ26QEPXzRcbvy9GtcS7+Gp+XOei1BBu/7gErZzhTfw3j0LiPTM42Ade+SEFcEQIh3MjjMJKociOV2Oi9eJ6mkipOTKhhs/gS1TIO1UUxvnYFWv3bM0UbcIjdyhzdDlPDirO/ZrHoVsY+Oa8Ps3Ii+zvdaGKSEFU19MXfc0ZGUpi/hjP/DPF0fwKN6eEQP1wZdwy3Xo1FcoFr9E43mWqao4XAw3DJn4bv+Jk+U9uO1t1/jyuArHJxxkDgZDOydhtxcwyZ/eKXFwc7bc6gIMQz5AbRRrDLt5dC1wbQG+9DolYNZ1k1c/et49WYQVlhEdO6buFbez1cXv6J+eDcX9WqmN1qY1j6fdQv/5FLDQ2DrQWlWglo4911aMHhnUR2/lLDBoOkIx2rpJlQTyziFkt+DLWy16ylsvYsmmzfXO715tEPNucQS8r1AUSwjQGJH7ZIjdjiR2aT4i6NpcpXxlgHqTC180tbCdvdvDAjR0TKijBbfFkQSqRD5AZCIaPFrwb9XeLp+Elg//h2Cb72C0QWVJheK/tF8/OzHWJRCoCLTAot1gmdfvtvFhdg3eDX8GEGmUu73gDMmGDflB2LndrOy8V3KAoWqUpW97A0Sfr8n/BduejmJFZdhbCnliu40F1qjOWGCfHM1obWheHYYcPU4GR/7A4mOLO7WQavJhEgEr/p0UmmHTdkOxl9/nrnvnAHcIBOA1/vK48m5kkx2wgRKfHzwkn3HUGUj2mZ/km8lY+0WApPTDk0j/Vo6N/rf4u+5O/H7YT5JXV640l3CwtUlQmtJYkPHXnaWPECN30/gB8uPDyM6J42qRCGodM4CUb47eWjPXE7rxbjSb5Hi1ctRdwydzhu39yQhyHaNJ15hJ80i59i0s7i9ncTLhLW7SO5J/2sGRO2eTNjfgT0giPjqT6H5LPiN4WTvMOLeDiUixI0xJh2bxYX79tg3+chkpAopzBNTGnvHX+gxPTjdInZ8dA+hiV1kL22iTi+wEqUIfeO42E2HzI5L4ubivTtZG2LDjgwZdvYGwRdOJ82vfoio9CFGb9/A99JZbLjnAPd5gIhucmoepndMJ2KbhSm8TUCjEOjc0gVydyjl0iK+kr1Pqq6c6fVhaHu0oLkdjBOJ6A7swrzwIK2NiwGYoW0h1cOHrUVDOLZzAtagbux6EU1agWVWGrWK7hg41ivi0fCrBAVO41ftX9ilxr77ljkMvHUyCMflAPRLGulU++ClbCat8xXCYl+i1Q2//bISR5eFmAQH3rXB+EfW87hrCp+EHuHP9musLphCk62bu/6+F5HMRPvabh7a9xUNPTF8NRlORC6icXk7MV06vvpgGcHLzhIaV8Mzw5bjkH3Dobt/p9Csw13zEN90Qq92GqNMlwnVZXKi8l5svbuQuTxQ2AKwyhtZ5N3MJ0ljefPMfpwdMxAPOE63R3ffPY1QwiClnSPq/ngoOjFM3smyGSe5IBuLBDBIQKsuwyC1onMG8kOVjid8HViCnsR/3dusSjxCvNkDhwg+DFTw7IoPhWd1fgwvB9SitqowGQ/zxcKneKLCi5CTQ3k2TM7LISbenzCak64j1Lo9OHjXF2g8okhPOMSzvjCpECq9Mkk/+yoFu2cg9U3EphLkcycfm8yOsW/glqhxl3+O3hcKBzdyKOMoywLeo7fQzQM9vXSWhaK4uY2bXgWUJBTRbmhjlv9j7GsSvLZ+DTGiksBfO4S2G1wnBNB+9Ye7dWBsMlAvWcav21U8fO5hPn3mFzwN5RwLhuviC5ypN7Dq76dplbfS4teCnwRC2z6F+uHUaRbwWuDnhEy1smLCOvrX/E1ITB0TuiTQdCfIjU6HSR+IXXonU7AuQodYm4REo2TG3/0xqYJRPickA8slZmLaDiC/Vk5vcH8IBDESrkcv4TE9POsFxov9qW4ORN14ngFXz5IXYaDSp4NGB9wT3MWVZX9hz1/MpN1bkEjciBY8yuToH5ge+y3NPd786TGX0zIXQ88OoH+RgZlP7cFf8zkvnDjH+eGr6Ii+ypLs+czeM5Gs9EuIZFrc0atRahrp9I1DKpEzc3spIWmz2DNvD1u64YAJ3q97n5LWdCT2M9Dhgc+/MYRsxvF8vjUdTW8Al5LOI4oUuIZRMmFOWNcQSmWaFLHHbLYf+B5/Pze1qY+St2ofJrWJwpZg7EBh7CEuDrzAv5cmjwOIRYKHoKJXRPLVOoxJhVyJhxYnlIp1WOxZnNMW4CU3ME51G+gD7m8S2O6BuEjXzCOu7k2Kg18H4PEWGCQREdwQSI23myrL79QUhxISW8NTGeX0JDRzWtJKk8uJyS0i+fmfGeJl4TXVOEI8upnRGM7ufcE4L3jQM86PZz77GA9FG6IiKU+3uuhX/gJ7jBsIrvFFPjQfmwyCRUZS1RBbCfdFpJPfa6SjOQCDv4w33niDPYEQpIVXR88G917st+IwHkyiI9VKVfgVpK5gurXd+DbrCa/7lO6a+5B3iwmsDiW8uh2H4mN67hrOwfTLuKMLmaJeydfKicBxPmgazC33VVyA/fYa0mK04NPiQ6e+k4KkAl5NKuBLCUw8lITYP4ytT6+CM7sRqwJx08ppMwxxyol7Sk1VQxU5VTmUjCnBpf+GaJkLjQhWVfZn29DjeF5REcD/XvEHbG4bYVVh2JuFfcyeFXv6Pm8OTyc7UfB0c7mEvlfy3UniOoLIHiAEKIYWC/sMtxuqzlURez2cDNERLidM6TtP0q0kEgvsNN4Tzo4Vf1DrU9X3mb5Tj7pIjd1kxzslkLIY4bzePaMQK0zcGHgesYcfAYNDUS8PRd2+tq+dHVEuIKTWRECLlsaeRg4lhCNySRlVcI2KgE8pDnoLeBSRCBrDA8lJE5I5Hkz5CAkO6t8sZ6PfBDoLrDQZhDlNXbiTs8ZF5AR+xzi/JhL0ifQ6SnBJhPv/8TDoT8BZF2S4/i1TGVjmEvwQp+6QYZp4CuUzpTy1OZLAtlYKF23o+96J6kNcDjmFR9dQEniw7++PD3mcr69+zalnzhKXJSZn+gtIF83ll7a1lISdQS3ypv+lJAbfCKRjbgdRV3cibVVwtR+M8argi6RP2ZKk4MWqFFZmBvH0d0+zbek23OHCmjKzdi6NPVEYFaHIjuxjqMuPv+bCrbCnuBX6NHKxGxngkrhIPf0r9vbxnJxwss8n2dUyFXfrQNT6m/TElVEoliKzRPJscDVup5ogilmVZOTeulb6fzEb38AwwpLy+Jfq1RBlJfd9uIkX66cjfuUsn51di/Xln4BuZE49nkpPArQBZA7LZPbfs+mXE8+BsdepF7v5RuLGjRtlpxg/qx/Nfs1U2eHeRnjcGILfkWHUTzWh6dX0kUm+7YTDdeOpirhJb2wBb+ctZc7udJpX3aAupI7PfYVE6trfp+JZHIpp8SS+00wGhPX3OgPs7K1lZFYC4mo587zrGPi3nHlrLqLN/g5G7QZVEN/lRyI+OhrxPU52BFfznd4PBxpcIRJohea8ZkCIA4jcd6CfsafHovhAQXf1nbWFJUb49xvRncST/wr2BZSdJ7j4DLUJEyBjZN/nbrGE8vSHUHY3c2LgCNoNVSSoPuA/5b+3/Afs+0/5bysqaxie7WPoMJzhZAhkKGFr+EWy5In4VEgIqMrH73ZwXi2CsfUrGXZlBKGThtB8fBd5nW6SboN9L7ZCos8XzJryMi8WLCJv4d9s6ycAao80CwDRsZI2Cs+G4zfRRW04aMzx+HVNY7j0M4brJhEoS2ZfdSdjUnP44baBr751C+S9A0O+4yHXj1w8l4J8dBq7552gLqgAbzEMkInR9aqwGwUJwcaYkfiXX2LYGR3bl8IRLBgC6pHrtdy35V6qQgvYeu9W5mhgwcBXebkNatTdDCx3suqXVcx9+C++tK4GWhhRFY3Bsx6VtRt/Ffh0t7NE8xr5trcYVS1lidcacm/+itymoHJUP9xuETKRiFcN8IgnNDs7mHf2c7xL22l5uIjr0Y8BMFwJQVLokEfxVlAxBpuKyH8tpOyenDbMRt22G0Qu3G7orO1EJVf1vbtWF9ynq6I2Gt7W9XCjf36fNKTYLaPFEsBfmkZUChE+Ek9qRDUkiCVcN0FKNYxsXEPshaH4DpQT+vJ3fGuA+3MWoekNRuyAmze/5MjXJfQblkCN9502M7FOiLm8kDyNS7LtKDy9KK2+QJekDj9ZARx9A8Yf75PAcOPm7KSLmEV35Bqe//hdLP7bEW9aRlNkBq2GRsqihcXdALGKQRYZ1u5W3GoNVo0GxHc2NuKwSk4+uRFv/ZP0q/iB69FL/tGm51xNwrdWylCfKIIvbMNnSiDvBSSw4+4dzMv5Fd8To9EcGEH+ykscHPc7Pdoe3A7BlP6xiFF84vkORokJW+fcPnmw8xYY59aiL13PL/dcYPq4C9wX9jALdi2gJLaEnLQcakNq2fpgNi/e9yLHOuUU31bK/1dQu0N9DJFjOhazmRVbHiA3cQ9nxgoyFSqxk9fPTaSj0ZvaiRJCl+zHw78NSSfY3BAgkmN++Stu9YTT3pXAX1N/IbjtCRqC3yFSBmdM/rzw6GRUXm4qIstp9W6iRX+YIhvs6QW7fyJbx/9EeUYe97iewiETFkGr9dDsELGrLp3qnUE0xf9As8c4hl4aypUhV3BJXHz87MeMzruCQqeg0z8On5psrgy+QrbLwecdsPruZgqYy9Gj3zDxcBRnR9fS6tvKC16gsrWy+fB25I2JRE+eRrHrMG26M0RKYbwajLWJ1DbOY06/diYfm0zWoCzKYsr4xg/ujn6UtyqNXPSFKPtpxp8aAwim1YWJhTSIrnJJdJB2/yMkhiRQfqiQ8LwLdPk+zmXzbMoW2qhT9SdYMQ6zo5fIhYPI+Ut4ny6xC6fEyb/bllpVCjIzMmkMaETmiqYqZToTIhKZtmkC/a8n/yNz2V/ZgUVqRoSIS8Mv3TnJ/GMsV83EsbM/ubnBiK76sXbbq1wcL+N8+mukVsGMuncZ4vwCs0lDV5eFwAZfRG4RFy2Cv8abqWn8PV5YaH5oaMVorUDikHDO7ETqVKHyCMN+JJe6i0X4F95H1szXSTrzLYv/WEx2Wjadg0YDkPVtFjlbc1D3n0WXh50/ut2oQ8WkJIVh67awYNcCyqLLyB6QzXIP8JfC9m6hvTb3g2sSKxbrcwBsC4AxiqtoULC35Rr+TV6IXAL8b3VDvg16XF4kFCbQ5ivCt9lJuy6cwsRCXk/5Dj/ZV0j9qlA70pAooFKxj2a/corNo4mYngNyAw9/txyd+BQut4vgohO4xFJmxT1CZ7ucPZW/YBN34OpxsPDPhWRmZNIQ1MCIwpNUeH/IgMtSxMFSytLcnDQJbStG08bHRghS5vFQfCo/Gx/ho8Gn+cAbnnTDKwbY3QO5bhkqnxa00n78XfgUaRG5/G3sIj+skTr9tywMMnFh6CFSJRo+bBcW/v00XWxVQb7tOifb/PDMiqaloZPg+g8IHmOGvZFCe1jq5orTF7N/LiE3BzDr18e4uqgOWi8yWSMwPZw4kA4bTb64H72aA/hI4PP0d/i2KZzX2+HBIDcRi48SLIU6Wx3FwW8AMOnoJEZcHMH11KvsmruBTHk/NjtaWKwzEy1p4JkfbyFx25nQVcRrG15g+A+5TBUfoMEJPW4oTLkbl0ROfchb2KVGwkRV1IrhW1MlT7td6ArzmHI6hZr+bvTZT9C+9gisFlg6jbW5KKeepa43gIDrCnprZ/LsvEP8Huxky4V46q740v14N1diZ/Z1jTgZjFPDi04DP7bLaWrt5bWPX+Pg9INkDc5iYydESK0sPDYFswz2Lr6OcfJNABQderp0XYTI3JhfX4tquhHvUgndPpE09BeCjW0eZxmoqUImA7Ork071dSROLcnVn+GtqcRLpCVXFMGrpQqqTOXMP7IDqWcoJRl32NnOXgsH3pyMfNAtJnfH4cKL7UsFULzNpw2nhxxTTSvp+78G4GqoP2tFTXi36nj86naq9X4oelOJq/2DS/duwWYXPIpavQOxz3iN9EFG0vemUxdcx9Yhwlgicksp0i/ENaAR46D93ErbSrDHHVlLnRh+95dypEfKmXw/nM5Ucvvl4i0RpNWrFVrWXymj4KKZ8BHt5AYeRq68fbCpFuSebJ2byfrRmZSceRN9lh6JU8rvS/tTWxrMj68/SPH+ERQNbqfothesxmrjQDDY3M38lfEjT2ju4Ub03YCQ2TtIKfhwzT50N5Wl4RQ8cA7LhF+pVoxhfyvsChDYYvb7TvN5SxDH89ahdDSDL9Q5YHMXFPTGk8MVatRu7Cm51AcJfnxPesKaiENUKZ5j8akorFVhjL8F03ufpH2jE2IEWfnU7M+J+/wMj51q5HT0JFrV5wEB6E8UJSLrmYdKp8OSa0HeYMUwdge54Wf4QwTZokFIE84w4FIKL/z9Ajcf60Sil7A9EMx+92F138v6lunMPfw07o5O0meN56euSB6VlLLcXIQy71suGe7j8UOPYx5dSOJA4bldCXdQdlVE4bcv0RCURWB9DiK/wRwLKeNlA5zFgx9EItZ+uJaGMBf7lh3nVAjIf7+dlTv+GIOldaSKNeS2+fDwrsOsmfciwSUfsa3ZnzPZHkT694JBQo+up6+NfNoBq516PI8ORpnoYLDfdkb8Gyix0Q/m1HVQ1mYnvKecLftmMXNwCSmDBSnmhVmP8Pa4GRQ2D+Zsz8eUKQZyt3Y3H8phaSMUW+zMvTYa75updBs+pSu3neSR+0jWwjxtHeda9/PKpRQido9A6WGjOXwwUl0pNP0KhZ9BwlMcuvws4mulSJZE0o0vSrGG99vBkOym3uDPE1HehFeGk34tHQCtQ0qiHHLTryHvCWJCxWjeeOsNJtx1jDVewvrF/W8s/tipb3JVpyCoR8J1XkMMZPYqibiQijOui+8W7GSLsoEJPE69U7AeCJP6YV39LV5daSh7m2kLsgDevO8DszQgkWmZc3kH4gsXgXoiYq4grf8DQuZBv7fgwFfkJ+ejDwjDoU6n1+TCz9VMUTR8eC2dhgARXnJfdKYUutWC3990DYxSupg9vhkPn1hwNCJ1SJkf/CJ7Wt5mgAImRPdw6d1vebVtDad6JIhK4JP49aT1PM14NXziDOOUNI9YQx5XBiuwBdxhWKzwgLiAz2nyK2Puiadpu+ZCM3YfBaHnWNcGMWJPpqpc3PvLMpqCFORMfoMHBybj6DrFgZIUoT3UplDDJdKbZDzWNYj3xhXC7iA8vJIQ+evx9ghGUalEZVVhVpmxi2BjB6zxdHNNBS/Y3Cid/n1g3xgVPKv0ZNsVb2JKI3F19LLm2FUSRxhoMVYyu+ElAEJnfI3K4s2y42vYlDeIV7a8TbNNzdMt0NjtjeG7aHwVLuxSEZFiF9Ok5zifEcvFLjFtlgZSLI+gawCd+1vCq8LZVh3BC3E1HCh5lMW6II5V3o3EqSGuVIL28hz8HzISpS9k0dF4Zv0xiNLRZQTmfsPw1kROjW+k1AY/d4HpVD1xVetxTeiHoc2ASW3CorIwpR6iGp/DPnkbaoWKu9QzCYt6i1ptGPFyyA+Hz0vaePmzbWjjTPjsfZrX44tYuD6Zk0U7uFDWy91J4+g+Pg6J/2kOd2mY6tHLiqT3SdA0oLAXUyFfwpMtcL1dw6STGVwc5mTlxG8Qi9wUSsRUWUQUJRQRJjLwfRd8L+Q+4mtVEFbeS2N2E5JxcQT1vkNu//e5FfcXR4PBM/QL3mmfj8s+kghZD9/6Q4O8jqKWZVzquYVccoT5BWOpSyhlzzwhIZOmszzkIbDF9l5IxXpkCJemH8OQbKAwQRhLGpyCIkJb9lLOb/Um+eVIflPtoEddhR4wucGFmCcCmzEaunGL3Qyzq7nb00pE2ztwGkQhldSE1VAf2kKeJZvfPd/nHmBv/UiSCrOJNAlMCeeAdIol6fRqPutr/8OKTiFxK3G7QdOrQOzUIhZJiPC8SXrQQYbYfuGP2hH0Nut4aO9D5Iws5uFxtazQQ6sTvtw9AoNbiTNNT48hnC5JB1IEVvcXfvB2G+zGTXNoDCqVC5vLwrHyleQ1jyF9wjc4nv6RzPzVxJxqQ9tRiwgRyX7nWZj4AfvC23miaAKdPRKi8uMpjypCKpYgrv6Nu+fcYrmrGYKM5KaW9knsg3BdBb/rUJhv0PV6LzdGfcqxplkkto/mzOjnOdVRwoW21ajFFm4NqKRVfl2YjyzQrwoMnQvRha7DW2sl5uBnTJiwj3F+p/j204cxR1WxnU6UeaPQhyv7GJL/mostYisJ7nJeKknjVr2U/jfTiAvtYEtXARs74cFgA9VtW9Bbc5G1TmTsphc4P+c8I2ef52CJH5EVkcjFPTjKWhhxLgP1gLfIjhckDn81O/B8/0PKel8nbXsKP+VG8trWN7l4YAR1XUp2PF5Bx9Jy4jwGsE5WS3kobHI/xPODxxBmsHLZmo2HZhA12j2oZcJY4xY7ONALxa5iYirCyLg8iOC6cAIafRj3/VmmGAS/VhEitJ1K7vruYWqHd8PkPsU+BgYeBpGIHa4fYEQXTaN+5LTyIPredDIz/mbYxWHElTTjsDi5cU8DB0p+Z9WPq3Cqmvl08kjWNMM3nTVIHTICy5qoNwWxOrSW4CrY2g3v/noftnZPCvWFPPb1Y2xesRmfVh/GnxzPtmXbCN01C1OEieNiMDX8jO0ZG9xYDYCfsx2f0E48wnz5dti3dHh20L/BRajMzSAlbI69xrFyQCWjETP/O6UJkInlPPDzAwRUBcAc6Le8HzWdHlwNnMXFxKl0qM+w5us1yFPLcD82loot54hNCiB7AMjtPrSJX2DUcSUWw3yyzpdg+z0P8ZLh/6gnc1QH5uBF+Cs1NEe10mO7s4ZRTB9M9CfR+MT7UFMDEqcGp6SX4LblNAYJCadukbMP0Ohyu7hpBb+IeZT29OOXFWuIMM9j8eVGnvvoOU5MOIVILiE1J5XU3AF0ru5EJNLTpr0jDfrcieOY/TdRathKYpAEcY4TuVVOXOOnBBkXCfXorWy5bwvrB8yj7sBThNSEUOtZi/gYHHb/3/shHnbZGXkcXKkOrKvG8aXpqT7/OICa7grG7VMTUtuFfW4pk2oncWHEBaRiKa+cfIWwqDDSrZ+haa3Cvv9XQsYpKIp08alvFEW9M9FK1LjdbioGzKMwTGDNVpgEL+cFvlaebO3FIyiIY8P/xOhlpETxCpuSXHjITJzYMR5v0Qm6jdcwKAWWmacYJqvdPKyW0M+s5/EuIyKXCqnjnzBFgf1BUmqTaQowsHOxsD9SWyN5SA8um4ZT4sGMK1LRZvFkktxOitqFzn0n4SlC/wh/XtcTrFjO5cj93NS3caCxBY9OD+46ugHVaBXeE72ZEDmB6LJoOvWd/NwFC8Ia2PX4RmbNW0HyVANT89bw1qtv0S5ysbUbbpXomHU1GdId/PDQD331mdzQ4pDyZKCFLhGIUgIorNRi9BJYqnM00OSEd9pkhPUqEbu7/u1eIboSJniMxV5xlOl7RuN6wEXP4DQalSWkkofI0QtKP74M6IRHv2Kg90OInWJCpTZskiBuilYQ6XUaAGlxPr7F17gw6mZfHVXhVXgqPJFr5Jxcd5Lc5lzyioO5dXAE09+e/z9tX+E5+/CtFsZ5s06Q9P1X35BZuvFquEW3IQy7Wo3c7gOq/+lp/lP+Pyz/Afv+U/5bSmNvPe26C+jN/engDPt7BbBv+YtbOVU1DH/bs6y6/0kC5ELgaU8QlFtPsfX7UAjVUDjmAZ6c1w+AZ1vgyw74rKmDvOOjcA9toCHcQK3OSRoNjFMJkplXmpZj6Shn4Jl2RrU/ScHjBtq6U2hsasXDZzyFxqOYnTsoaZgI/2IlS5Qg9wKRmISUbKrnxeE5IBJ55fO0+s4mPwSSFN0se+NLRiR8SMmhUiJu5mPSuGkOqCNVDr9XTKPdouax0aEUJURzI06QKE1VwHIPuFxzD4WVY9EHBzJp6TsERTYgMUrp0V/gREgFC/9YiMit5Oa0VgZLPUiS1PC7G85bHdC1hfOrf0TsUiBy/4hTYmKkazkHTIKU1RqJhkXKE9xqjKI925sFRQs4PfY0z6a2MVsDW/qYi25EtXYiyyMRR4mJN2XjvpyBa1ssNdOq+Pa5bzE7hMXbNDWMVsF6o6DXX+FZxt9z/+h7t2+MWIHFKWGSSaAuL7T9Qe4tK/0nV3Cs5xNU1nC0Jh3huQeo0UoxBK8Ew4+sirvMx4MjkPu3UKP6nq5gJTgchN34lvqoKPyNs6nx2ktK7edY+nmxbcoh/J3pLP3Ki8EtNk6MsTBn9HrQxbLQbWPE+y9jHablxuigPrAvQz+PLm9/PEMCEcmk1MWPR1a7nNTcVOZPusH1xniK3p1LeIYNRpqRme0MnyKiqkMIGjzlaWeEqpn5NU7G7K/AN34KF0fXMdE3j1Eq2Jflx5j9I+gd2Iyh5CbO9Gmo1Crus3vhH/QOT7fWEp6RSbeH6R9ZugBusY6JZwcyflMw8os/8kT+AxyaepCctBzeEl2DJtjUDQUOsHZbSM1LpUfbg8vzIfLCH6Pbowe5Vn7nfLgJ6JhHkHERFwYPJ2twFnOj1uBh7MXLKDBH3vGGFRFHOX/6Lkrrg4iIeIWPe1Ss7BCAllWt6wn2usob/n/wIr386KylS9RFl/4dJqvgSDC80thCdkIhck0Au8cJgUeRS8b2bhG/9cKxoVNpuZFCkFKNt9mf+SfyOTh8ED93WVishWhrNy3ZScgNJm7+fYipZ6eSlZ6FS+JC5RqF3kcw52yKHIpPTTa9ml6cUicHTXAweDPhzhrmVYyjX24/rg26RqtvK/c3wesqLRGBhXSpAugqLMJmeZGGuGyWKGGTP3xwqgPr/jex+c7k+ripXEgVZF6P9ILJHMpPjqlUDbjM8M6HiIm0EFURRXFcMdsCoMl8C6nKk7L0JaQPgYBADWvbX8DLs5LlsmFozCn8Pq2An0cUoE1+A2+lL0033Mgcnlwafgmr39M8khiBsaSN9P3fYFXI+PKx8/ToetD3+tISMRiZAfROgZ0X0BjAApWYHwNdvN7Vj7JBH/JgfBO03Gk/GWoHs73LyV9ew+HpBxnee5krHkWUBwpZkTVOsDhUvPHMn3xpFLOpcjh75t6RtoloeozBI6K4EipIZrhxE/GalRX6Ffy08id2OpR8qbNhU7Zhkbm5nRtAa2h/wm4d4caAG/jrRgEQPSWanK05KExGZE4fwm6kU/XS79wMCCPJMojUvFTMKjPZA7L5rhOOmmDQwSmgUHN1wi4uDvkS7MIGeKkOfmsSs33bGNxRJyiaaaA0SJB4a+3sz4eXz9F/9OP0fPgdZ2+to9Lwep/Uze9VMwiwJrF0RgMxhjpqupLwvXIYs/81brRrmDxwFbjs7Fdf5tvf55Ef0k3ekDhs0naqOiuRiqVkZGYQVp8IL0v4ffHvtBvaUbt92Tp6Lf7ifD76eh3VTjNWqQfrT00g84EhyDq/Zpe5iwc7fCn56H68JzmQZfjxpJfAzlvTDOFSuB7Wzub7TzOh8XNq8gIZ7fsLk/VqkhQQ4lXEzwopCimsOxbL4WZP/Ff241CWApnfn0R6TGWwdyGjP9vBM9uXknbZwvAxdwzYATa2DiTfWUCGy41T4hQIPGMP0PyDhg+PTKSrtokzIRs4M/FtAKxOePvmM1yxXiOjKADJIBvPqAQwdornCRZoYZkOUuZlcWBIMXbdvQxUgNjtxnnbu+ZYewSBRQdQWMyYhreTpG5mqvgAAMESqJdVYPS/zTR0u8ENryu/hmDoXwUutxOf2Fqa5C3YlY/QYxVR7zsbbEYQSenoyOfNudt5u8mDms2PEZyTzkuPHkEmdzK1PZAzOZ6Y283M+ns4tSG1ZA7LZLmHALCufXsa8rJwnC+JKIov6pOM+qkLRisk/PTgfjZ1etKrU9EmEzLUTWoTnfpOCo2jGJisJjRsIhenDsWNCJPPKfxW7eI3DzfdSjO95Um813aV8GsnyBnQTmDPdj4f+QRKiZX90s+I/OJBokQKSsYYaBY1oddl0U/XhEbUjyr1Qmpjz+IRHIIs7yY470ijaHo0OFUmHOI7jO7k/GSaAprYENVD2z17qPN5HNGpTLxbBbStySl4FG0MyGJE1Cj+sO7iq8e+wikRsgye9ITVAR/w8fWBVCq9cQRaKJOU4a8aDWbYFQgONwTc7EfslckoyyJxW8rI7ZdLXttAxmR/xvpHR1HRdJyIMxewipPYnOqFzdLDlwMGIzo2HOY381i9B1nOLvyHu9iU8gle3cPJbxpKhG8uVfP3kzFkMZ43Y/vuq013jlMmQULssiwbl9vFkh1LILyOAxPOs8YTtgfCHxYndMpoE53hQL9dwC7miyJ5tb2CvNZ0PgvsJMK4nOFn6vBu8eCjtR+RZ4OPsj7BJ7CLa5Z8vBwyigZPIC9c8DLc3g3V7RmsnhxAT048GVcyKI910qltQiHy67vG9oQtrNtkwylTMqx2J122Ds6kCJKZBUkFmL0n8GKYFytvPcvnXm8y7vQ4XFPPoBbDEfc19C4j8f5JXBuYDbpoLLI2FjRApNRB5Lb59GpUqFpqkBmbEIkgNPsW1cBx+W84L5egGK1i4PSbJKbckUKWidxkqcRoh2RTK4mg0S+fbPlFABKrYGjj68xM/5T6yBCKdAZkIuh/p4mB0p9JjmoyX3uelAg3fpWZNJV6kzroBmO7tZRuWYZtug17iAnvVm+6dd3YFEJW+aR8D1YeGYm/hx8eB+bxV6MP07dcx7/nBAYxiJp8MJQEEeGbgPWCL9UGJymDC2lxQF5PEEr7GWS9HdikR9l39xSwwfp2eMkLYiRWVi/IZOfgWzibghiottKLGg0mnG4wWyopsCkIdIowBnnTo5tOvaeYZH6Fuv2Q8BRl8UvpDHDj42Wk11FGoCKa3b519HrU82zew7jdMOPgHdkvn3553AqHRv9K3rzyBG61lJSMPHwCW+nf8igrVQFE+hRSePxRuku9ED+hwC2RsmoVHMoGRyz83erLj1NeI7WfCAc7uUsH/dRwrulxtJYEklKaOR70Jk26NoyabDSuQLww8HMXnDHDZ7jRphgo7x1LQYSCeUmbMflORT3gIwCWaoMQTw9F1G8tT51fSo3ndiJscN0KrS9/wfnalSxth6c2LCQzI5Bjk4+xuAGS5SImRs6jVz0Kw95q5tx4hdovPbC5wbtc8PicrwURboZdGkKXRzvOpEBWVMTj6fADky8BTj+sShsHZ5zgTLDAnhxVI9StlRRhVuZzcvDXxA9t47xnPLctgXHjZoDoBg0t/si8jDS65Yzy/4mg8BscKHkcAO96D4aeX4SHzR+f0lg2hkpZu2QArU0KLvdby5Rw6P9BCHOurRN815VWHm2B7JogAk8OQTbUwvIT31AccIHjo14hUgazfMr5bnwt7w4/w5MNySRcPESNtzci/zF977wmrAaZ3YTZMgZfrz00q5dwodOHSiWEWXzpH1KLSyrj2OTzfBnchKcbvvaDBLkLh72NsaofMGva+c2QivHd9zgUDE094ZyouI+ndDDx3AvgdmFz/0FCzgAG2zdhoIUxERJyh9axOsZAUJ0WpyiAoLa7OO+9A7Mbnoo9SpElFUWNmye+fILDUw6TOSyTwSX7iBB58POUMXxtlNNVVc/aIxNZ9GAUJuu3fN3RSlavjZiCY7TIAriccZlmv2Y6HU00yDuQOQysLx/IuItxBA+uIfN4CulDS9GoHQyugZSa53m9fxD1PyxkRHk0lx/cycsxdYhFbixuH2bPU1C/o5RrofBdSyftf76COfYA1xNv0OjTyjsvv0Phmns5/Pkx+h+7zB8L91CQVMCGDnhPqUb962lqywoJDfbH0TmdZcu+5SWDkXrv3/jBW8zycRe53/g8yvZQLIoaPKTeeEvaiJXB1/UTCe5RUuo8SPWiO8DU80K+KbWPv4DnqEaSFydz6mAOzuJDVDugfzU8GDWMdQlH6HmtiButI5jy7lomPb2DznBf9AYfQlyVfCWay2XbaIzXW/H9cxLFs69yV9hx8n+dSnBDKP9e7NK2vt8N9bl4N1Rha5vKjntO0a3K55aoisRp5wD4ol3BsYcLiagago9JjcwuY8VthtCQajDP/xMPWxLj5B9Tp0ql3eseBingbCgcqPejYMdQAnVd7J/6Cy36Q9D9M9Xt91Hank6HppT3W84SJnJjmjKP0mgIbPuacZG/sqL/i7Q3Gsi4lcRRjySOLOlCKhpCkEIFhnQ+Op5GneE3mmwD+HPBr333s84Lhijhg2FbUJs9iJWE80MgfGpr4adWNTlWEXVmL8rTFxMQAAb7mT6wr8ctKC0sVHSRGrSf6u5RqGTdiG77sbtdIuJlInwbPRlycghZgytIuz6XyxmXaQhq4A0D5MircLildO6YTXC4md9efY+/Q51cub0MilINZI79W7K23kNxSicNIXWUqrv4uQn2msM4fHga7fd2Yy6uIbwwE5P3AxRH/Ey4roKBYh/E8lbyTSauJFczLbYIRG5GzLzAYzVyDI2rGWl8AKWuk0zLR7icbnBIsPi286imlSXJ9bSPukpD/iRW7elPzKBY/pXwU+XzHSvn2GmedB7Lvlfosbip0z7IyspLeEsgXe3PkqhMehe2UOhYQGR5JFtczdSZXuZg4Yvsuwdsai02qQaHTNho+TR7sODkA2QOzeSbx/byWNT7VJy/TEtoLW+88QahUojvEpJrHG6wdpo5sHEBGVMzmbr8MP4oqMRKanwttUYjRT5FnBl9hg7PDgIaA1Cb1QxFwV/z/yImNIbq8410VnUy8O2BKEVgjgHaN8FcyEv7lqJdgoJDvXU7rU1DyNHkkayt4Jntd7PHp4AtLXkss/0/S3n+IFaijnmKevdIpi4Q0N6BKwdypHkgqoP1TN8TzOWMaGQOHXI0Ql/55VE2ZAseYP0rfqXK/wGSDDPwlcsZ9sQwChUJOG13ktYylPDogKNsNJXiTxLfDD7Ppz9Vkxk/AdxiFnp8zIr+wneP3vcb6y68wPrnPyY7ajkAo8+MJqm4ivortZh2XMeZ5EV/aSXZI5eiLxSAJDcuRHIRbd5tWJQ29C4JMruMiMpQWgtaAT1Dj1cSdX0Rfyz6k7LWUZRKzlEmlRJtlzBzTyTqnsc4tkDKnPhPWJC4Hoe0jUNmaMTN5KNj8G/yZ33GekZJION/tCT8R8kAxomh4pAcx112bAobMocXbqmRICn4SCXkRVRiV3oRXlXLiMsjyErPwvcjX8wOM0UJRQTZY/Fu1yMekUGbjxDbWaW/DA9fZrvIjVckXE7fSFHwZwAU9Qay5sJX5Otewqtdw4Bxs/i66hCtvscAGB68D291NXtFjWib/dj29CGaRcIaNkImKLRllvnw1WuPIJ/UzEfPvth3P28YhD1LVaODPdnRnI9t4oeBAoN9QUMFE+sgvHkx88d6E/PjMgZ1G/j9mYP8Ic3jeeluxt8+T5ckjcO7uglIkTH+sXdYrDfhUw6aXg1B+bXUZ9WjD9czaoUQyzg99jQ/NcEDt493uV3UJfZQ6nMV9+2xLKT1PrLj/uDaG2/wTOQW4s+/Q1GwIAVrEEOMqot51RHkZSVSMcNBQXIDdoUCMBFdKZzXOXsfUoee4cVtDO0cSuawTGxuCMy5TMrkMLKiNrN36XbuHbmRzB4dmbVPM35pCNLiLyBiKV7idNoVdQTdUPHavtfgud9pHPsA6aMu0zjtcUKHg6RyH+HFFeyY+w0AH/lAzPhCnjFbURlUfeSARYU67rqWRGKbtu/5/wvMU3U29gF9BSMeoNcz+B+sP7mpg7D8w4LXsesAhrKraKZ5sXMnyOUwd+7/fbv9/2sRiUQEBAQg+je1wP8vy3/Avv+U/5bSZm2kIuCTvv+/0y5IPgAUWD1ok/xETWscE3T3ktP1Lss9IKt3Kr4rtKj6DaXztC+evh2AICH0SQf83GxgTIMXPr3RrGxbyYyBIwE4EAy7umFvajKVhkgMNT+ht9mI0DbTIo6lX+8EJnnMpKTqIIN/WknNcCM9XulUh9Zg8V8K6UsBWNPwGTUerbyk6cK/cxb+HbOwhe0D4IZFzGIXOOqa8KnJBkApd5ATDltzkinfoSVU7cfZ6BjyUoUs2/fbYYMR/rregu/hWtSvLOJ7+05Guaool5g5PuYYHmJ4yKzF5JYy5VAEPoVDeG3rW7xz+7mpRE5ma6DS6iRlyyIaAusYfu/9vFu7lQO98Jhcy9B51/AYIefANQOpealUZphwxyfSHZBAbF0ln29fSYPZC/x6WJG1gsNvX+eWx3C+kl3GM6YLhZf6H7Jn49XwnBfk/jyHgJJoXL/cyXAGsOpPESmFz967n7YGH0SBpxheVoZ76lSCJLDIu5MQ1R/w+C1aRSPh0GP8VQdlQ8rof3wsSUPyuZUupmLAfDT2YyRcbeRm/z8YXHYCg6oeNxJknMdHAlKni4b0HmqM1/F2zISk5wFI1UYgjc3H7D+HGWeXUcFhzow9w0L/FzmTOpiYMcLkJbWZSM1LApw8qr9BtrOeD4ZdxCMqDdeZ9aRlQc3EAFLk8EcgHOgU81KTEqfDjcjlQOwSI5E7GKcW5GAvpZRz0b+UKP03VOjmkhQjYvip4fifH0XEpCsUDS+lIKGACONIDG0GjF5GPKRuFst9udd/Ags1v3B21FmmxT1AU2s2ZpWZ1fo7z3ZvrQfUe+FWiXn7lbdxi9y4JEeQ2qVoO6RYOiy4CioJLqzj/Iir5Ee+SFzdm4hcgsm9zWnjyOIkboV+D4AMgR2VmfwMxqgg0lxuBlhGYbImom16hK/HPk9BRzyLTkfS6vRF5tmDwW6g3budUju82gZ1Di31Cw7gto3nX81EhKjPF8JTYSCmVIHe3kbp/hL6ndtHTlgImzSlzNK4mRxbyOo33yLNsYkQbzU/RX/cF5COLo0gqOUSdnMGNqUHNzMCKI4rZmcATNOArkwIIHWPF/FlxNu4xEKd163wg7SJ+x47yw1TKtpN+5ncNJnsAdmcMcPC0kRWJn9Bc0McNzxC6PGagNEgZGdu6YYt1PB4Uxjqi4vIzHDzy4pf+t7BPA1cpoVC9y8ER9bRbLuLCtsRSuKuMUJ5jcdCN1Pa9RqrmwTp4bUuFx0dcCD9jm68hyUVb28/2vIF5rLCakdlVtGj6yG68QU61Fl0WKPRjHibctu3RBU00WvUccXQyS39H9yKaKCoaykPffcQ5VHlHJ90nA0dgp9gs9OOAzuTh8Tyo+h7bDIBEQyTQo7vC6RUQbtD1Md+BYH59JRfOxHiVppvJOJ5cgDGMUY6+4solQibGZfIipdzA9+8+y73NwZh7lyJrrWe9sBoih/PpLWslZBWYQcSPzuelZkr+WjhFVS9nSSNLGC0fymfWgqQGdS89epbfQvlPBu8pIqkuDCJRrWNmxPujDVuwPdGOm57LY9fS6dM2kGC+Z0+sC++7i2kLi3L9TaS1E3MO/8ouRHP9B1/uH4cAR1zubvqZQZcq0VbEYfb5o28OwYpbugug30x2K16ui+noRRbyRr8A42GPzlSvhEvcTyRFZHEFYchUkgoSBK8EUM9QokcNgl33jWe2/gR0yruJfqwlojcUYTfX4yvuossB7zq7iBs2kFCQxfz2Pevc3rkH4xdcJpAibApP+lRSbwSns3upW2PjZqAELaFN/OLGf7sMbOtPYmOcj/SikKIK4lh0opNfO+4h/aq+xk68HlOuN5nbe9JpCk9mKQv4y0vZ5JfIcQ8JPS9RgWvfP8KF8bm8M2ab0iQCNr5rTYV5ksDQdpDe0ARcqccu8yOSezmYM0M5nhfxHp4Gu1aO6vLFzEipoGL8XZiZQLrRKo1otEYkRe6WJy5CP2oWyS3jKVWc4yUlnGMWvwp5V3etMZ58lnc0b738ao37Jd0Ej5PxR+dShJfe5Quj07OvXeDUdJrZIVBtdvCg+l59B/Vw9IrL9HumsyLQemw6xr4j8cW/jrja6HepGbA9HRuTfqCRpEdlx2M487zW6g/ryb4kJKfgsgtInNYJlcr72VD6RBO9tuIPKycWVFz+OyuO9I1AMUmb16We9PiGY/CXIPWoqVH10P74Bu8NayCHNNjHIp8lkmJbkTdpxkTdIItYieWTQvYIXVTNuE0wVsX4zugG78aPSUJPfRo8/nKVorWYSBMJCI/OR+lM5irQ87QKs/iiY43eX3w65y1/sgPXQ38uuwVUmwraYvopsGws+/a1mxcgygoC+0XQ8ma8Rr9j3xIcF0w3/rBCr2V02Oyudkp5+SMQ9TOO/mP+5LLr+Js9uKa67t/MLIUIpAp6nht2Gjy7U6erJ3ZN3aD4EVR74AfD72Fb0UWrSPHcCpSkHqTuHSorZGY7b3k+z5P2VQV8d0ePPnFk0x87HdEoRchcS3IdFy+8jA+DTZ0EzwEZqqhlBeONrF/kBj6ZbEizovg36RMaZnCsUnH6JG4GF8HA64P4KkzY+hdV09CWQzJHi5KVPB10TIk3UO5EPEwHV5S9Ndf4oP3X2DJ07+TllxBGhBUFsGXm+9B/H+x95ZBdhxbu+azmZuZmUEstRhazAy2LNsyM7NlySwzs2WQbFmymJmxBd3CZmbu3tgba36UTus4vnvn3h8z80XMPStiR0hde2dlZWVVrlxrve+r03Bu2CpwdgEinV9Q92S8Qy/w/EfPU5bcSlfiw1yPfgQQUfQXHDU8KNNwZugZrmYWkdSxB7yu8p3fafhTAinP0GJ4nnZzGTp1ByF1VgJ6ejiecXvMBYkbs8PMgM9SeYb7qA+r570ASFPCUx/NQdMdSM3jHvaG7WS84UkwwRaz+MZ7qiQSW6BA27i7MBphrMfEjUgRjXBT20bFG+sYbf8IH+NCpmdv5Y/udO5suQFAtI+C6inbiGi7m7qA7bfnthMSnAHESpuZ/8oxNm59lNnfr2LBMBsbVoaISQ+fTDa6MpDl5NHkM4jG+KcYNNAJwR4q7I/SPURNcMoAHFfKePznx/lr4V8UpRbxqDf0j4ln95jHmTley19tmzD4tGLwfoiHr4q6OyOqIpi+ayqNd2oomPg8r8zxps6pI7nago/vdl78fRTpV4Jpm/o7e65OZuy8o7zgJ1IIFSrlrDEvpkYRRdXA1Sh86pEyBIf1HGPq4c/ofoyKGsGVsQ68lQEUa1+jx3lc3N0O+h4Au6Ieu8qN+sAahtQ5+X1gFTc0abReD0bfXonDHE5Rfx8ue/ZRHl9OVGQr+T3QV91JgkKOMiyZ7aMVzMsq5kXrMmoc15ns04H05FSu9/jg6TSj6zThcYYid/mwwdRFkTEKQSYHGUzdN4wJQUZS7itmTPuP3JG5gqSAAwQY1Izr0nL21nMXKvRnp+UMOlsynzlNPD9yAAfC72VjdznHlccJ9vuNRR2XwGUhSnWBwcnfcVDyLMMP+NIjn82uOVsZpoGaYDP72txI5DKaw1S9rBtmAZ4QxrFk7HhevVhPRZSVLncVfvIRvXPluTolf17uR1C0imGnhxAX18hdzkXMDu3LU/tPsCB6OBXfPsKZSdeoGQIjtcbe94lVgF/KfanY+SI/zz/JI2MrqEWAFvgxCLQeLUqlmRnffs89F8bhc7aIjVc/pMN6ew+xUDBjv5lD/ohGjuZsQ6NO4YVB37Lj1uvQ44HOGBetXOz1EwEONgVwd14f5GFOfG8cp8emhhEikvd08Ut81f99ch1wyl5NzqK9PJRdyePGAiZpIVcL93jBS809uOOTyO53A5nERKvXJZTOAIY45vPd0/fwS6eOwpuP806niTuGrUGLjcd9oMBjRFF1hRub0vGfF8zXgzWAjUuNU/BWtYDNzcC2nzFa/Fi/2M2c5e8S5evmmmwoqqgzXMotIygzmB8eOUqX/lzvNflIYebiTTw2bhO52gZOJLqpD/8ZgATfAhIlvvy9dwCtTX74tGxDcb0O6b3RuNun85jkFwZ5X0D3RD3FjplMz9jPM76wy3g3xZmniXQkYZVE8NZrG+hfMBfJ5sk8q8xnfcIxnDLIi3gaibSQTt9OGsKa6YluJDq4m/rEFwjPeBqUIMfD/sdfJCCunbDrcrJUMQzPyefldgGX1IXJY8bmbxf3PFUxtAW08TnNVMldpAy4Ql24k6ii+Vy4XkLs0kiyDcWUxEORw8NauwOXVMrCLa9QGPcLeYPzerV3J6pfwTLSG7fsZ1auXMnpoac5OOFg77h90vEJ7kg3q1SrON64+x/roUwipV9RKkbtFdalKCgbfIGVdPJczAkG9Akk4NDXDKyop3OPG9PMNqIv9oNRRVQZtOydvJcY5zDeBSSlpYQV1SJNtYBKZCqSWYvwa6jAZR3Ty8wShbjf/zLvB143vIRRbic0TMN3j/5NmC6SYTUSLIJAvRt04U10WcKg63Z/W93wS3MMp5sjSL7Uj5LMHv5FYi2V3GadCdGFMuLkHJSSCPYMXELu2L95OFj0x09UL0RTUkHZJn8UY1rx1+UCYPDcRFL8MSpTAkElxbTW3i5oAeivEhP/4XMv8GuXirbqbD67kYU8Skp10DfkNghMb1nAwxNi2NfZj+mbHqalx5u1d3yKHJEZYLH6BtZDNdQGNjPn/Y9QSqUcsqdQ8dy3PN0NfxpCycm+zLEjwxl9uA9FKUU0hjUyph703ZmM9u/LhaXr6eoZR4nMwdhmKIvS8GmgjRMqgZM6L9R3bKNQ2of8+LVUaOFFeQJpA4qoDGzhktcU2nVWtrla8Si3gSBlhR8s1LVTdCmVRmE8fyVv55WkEiyCiu15iYzJyybq+VOk2faxpeU1Xvv+NVZklDJs2XlC/5xD9qDzvDC8gVIn1FT7EXt4GDdD7SyMgfcDoMjeRV+1i2KnwBuJU7BEpjBYe51GF+wJh0K1jaMyHfpxJ3Hvn82y35dRtGgzskgPcqkdUKEZ4I1N6CH4wP1M6plEfXg9UbUjKUsooyauBY8HvIohwJFKYVohtS5Y3QnSTZMJyUvHvMtEyVQ1dw8RkylTDG6GaeF7r7foPzAZ34TfODr2KKEd87BqSwB4x0tB1KhiPnQJrE3+mec8F8goHMDDAx5hU+03mOTebOvsxmb9oXeOdGvzCQvcRpYPdFkNOCoiyR0WyLuuAs7zP0efAZwHTnsc3N+koXbkaLLuEf/ucXlwnb5AUGU9AfXxqHvU3Bz1DDNzg5BIQBsVwMBtV2n1OkBekrin+eWedXzi/wNh/eFAwqfkuT/qPc8vwRDrVLMp8G00+bEUTtxK1MCxnEsGiSDD1dxO8c42okdG45cRxtUiJUpXAC65yIAS2BqIf7ONrvIOnOfzMUSoCZdD9oX5+Mvi+KAhlJBuHf7Z/vxy7y9IPWqCb8i43P8yV/uU8s6Etzn8Awi3ttx+5mGoZTbUgpvX3nkNyygL12O78HhqkJCG2eFHgykRndSNZc9QtKPkXBh4GY1Nie4c3Pe/SPT9y5a54NWbFiQVTQRpgujygyHWiZzI3s8x50XeGnmSkM4A5pLF90PuxuhlxNNzey0WcGPXByAbMommFvGev9r2Ii1n72eMKF2KzKPu/b5EkHK1fQDhRycxtjANFkBa7cdUB4nJpVdPbGZ88mref/1rFtcFIzhEVNjc8Kc41PgZ8xqh3GrBf/hJFNEx/7iWOpeIitVk30Tzfhn2ojuJ2DmBviML8JO2UOsCt9CJBLiZfgMfWwR2j4RWF7ikhn+0FVF0BDdpFOfI0EnhKR/40NnCxUWjefmxQbQ2tOLWupFZZbjkLmbGLOX05YvElRroGthFfZqZa5LjBHVNZbJlHh8OfJ11xmB+dlQgCAKJRUG4rQmUJZZxnzesjj/Hm59/jpDXSfOwm1yJfZigrmm4ZCY6DMeJNU5maPheymwu4q8EEFmSwLmccyglkCSEECBTc7l/ERtMUOvt4VTaIPE5cSwHWwNYa9kZXsX+a4sp8tGhG7ADtZeVuO7neXwwPFDWBryEc1QuB5Nrein4n7ulFvHULf8r9kQsDomDotQivE55kZmTeXsu3ArLqC23WdIsvpH/OAZgM4hriATwL1yDb4ccU3MGa9eCwfB/brJPKpUSEvK/S278/7z9J9n3H/tvMaf7v65WklIYdH4QBmUIDXHHeML7OknSQtrlEiY1CKibE+n2ushEWSAdfR7iQIeSCX4OBquhOw5Gy8v4OPMEs659jfavY1zTZpI5THR0slTwqH4+7WnXUSdGsz6lmjx7ID/EractVopLGIk60ITvnbV805nNtN3T2Dtp3z/6l6d9gxavS5Re2Uif/RX41j5KX5+dyJ1y+l2ZgEXXiGrUEOovNODXeBNtawCfVOeQHVpJR1YKkhA1xxS3dW/ciPSibQExKNJ1CHIF7bYIGuzB9DXPpO83j3Lqjr/48K1febPFi/LjuUyNFoWzh2vgSARsMV9gcHMYGzoVNNWH4lD24C0PuD2mSFiSvwqTzUbi8Fa+ynqTBMkk5o4SubvjunbS6GfF0pVJ6NQJHHU0c692FBNaxnAhJZM/pZH4JgXS99O+NGgbqI6p5r0O+NsE70aaOG/qwF1pY+6muVzqf4mq2Cpy68WK8VVplTT6daPUz6LTpiFAKiVJCZ+FdtFoP01ofBeXKjMJu3YYq8vO48PO8dhwFd2JNcw33MG9s9uZc/E9NqxchV4IY/QleH/cCKxOb5ocWRyqz+LTiigqJnVz2niaWZIZvdddYXiOTTEwaQRErfoWH+cwrHOUPF8yCFVWGP09V3BaDGQdEdFcpYmlZNZAoqIF9ewDtLuS6bS3UitvJEwxg5amCSyUHaC6LJTJ392Pz2Qnnzz9Ru/5Xm8XHe19ibVokh3cU3OUsyPnkyfPIEolbsgO9SgYoRFo90DqiUTS83NZ/cJqInxs/BDZSmXPVoIDurkx4Qqzs9/gz7DNOGRdzJGKQVfDpKuoPz7NPZeb6Z5vxC2/HegIbQzlrjVDyE/PRyhpJ7TsEl3jxar+Dv0pAluDCG0MpHNwG5cSRGh+TPNjvMhXvNgO0zoGAuDKz2f4iV/54H4jU6U9vFMxki5zOhEbIUDrIsLbSlxxBG++8SYVTjFJ/5Khh19C4P66DqI3PkZd6FnKUqVM0/cwyquNQNNRZrCLovwUgvo8SG3qeKz6HaQrRWo9gKf9PRxqhhOK3dRE367Uja7QYbhxGKe1H+HFRwmqbiKwZjoFqT8yXivw5qfP0OHtwriKf4yHFNhlgV0U049rmCZcxu4Uk15NbjhsNjA6pJuUBzeyzZxGs/EXQpBidgZhVjQhEWSM7HFz40o6yrSGf7wHtOWQ7hzHmoDX8S9M56BDzrAlIv3XDYeoL7rYO4rNFf7EVcRhDe+mcc1ZZrXPYtvsbQy/9AGx5V0U5RfjlZDA5YkvoTAeZ9oeI4VZArjW0+d8GBf75yCVLqEtSE2L1zmOCXb21cOELgvjaotpra9CY9OgsovwDJMHTIB3lzcamwZbuqU30QdwMxrOm6/y4d9ryZZ7aNQYaVK1UZxcTLrOw8Mxf1IuTOWlUm/GlcRiN9qpmunihEmkTxqjcWAhgHVGaHDLaStqJ/nc75Rkx3OqeR/LNi0jrMkGv4BCqyCsfxjG0GT86q6RkFbBta3jCFAp8PT39GoP/MtGerUR+vZXLGiIZsKx2eQefZ3vH/yeluAWhhRdQIh5nsI3V9PonEH4wSoSS/tQmlhAhFTL/Iw3qHOM4o8t81CaWvBv88emsWHVWdHYYwA459RSElpPnUTJsVEtNHrtY7QnE3aK5PSHNN68/erbTJF+JwoI3np3yqQyNs3bhLc9hdeE0b39rTXWItn+HjoJKCUWYhQy6nJ9+ejuz/FTdCJRwQQdvKm08IY2D18Wobc00t3uTaNmOGd6TvFgM6R1hFB/pR8N6uO89EAzEYm1eBxKNt3Kyay7mMGgM2m47zrPI/fvI0DRTk7S+3zbDU7PE5g9elqaffGSm6iNrOKSxwC5t/VoZ4bcwDKolbYgcdOKRAId+XxmiWXLS+/xaNyPZP8Uyty8V/jo2Y/Qu8ajdAYxIDCAsJc3cCh0I53bjVwTpHgkKt7vhPc7AQEkgoS5RoGSvHSmZ5VTM+YgH3XC7m4rXw8t56PWZr5tHdvbl3KHhEqXQKF1MKXvhCPTWylNrMCqNZKn/YSjjaOYqgNfiZSPWkNBVUqPuwd/TSVKbpXsh00FuZ6gXxYSbfMiYulsMlW7iVQeI7oSalwuopVOkEp4+9WPcMsspNV+wkXdeUotNqrTb+ICHlKmM7jkAOeTRMraX4LBxyXh5/15xARB5KlHiSobyVsr3iLuch8OHx2N71LxOXMLLsaFLGNxoA26vqJYfoSC8DrOBDQTPu9vfGMH8fPMX/AxDyHQuJ8vukDudnLAepw99x/l7upkBKtYoVxhjOWzc2sIyx5OhPRrlntBaaeFmIoNXJJbafTaBcDlfvmMDh0NSEACRcOWczg7mU/0cMwKY+rhDt0/BU8HqeCPIBXfHB9O3e+D8Jsp4Bflh0vuwuht5INOEZ3/Y/FAmnYPIyfGzuLrr3PpoTYIgr41IPUomZWWS33CaJQ+WqpjROSo4H0caU4kl5s/5kRWARtjYfvVZURq95AQfgvpEHsXANlFc4i8to/uYB3p3llMGFjKTxYLEkGGIHHhdnvwr/CQdSOHQ7mHevu+KtTJMYMFQSrh8rvvMNcf5qugSnGCR3b/TFPkF9jVkGgTqA6vZ707m7iK/tTLqsly9EXfXo2nx0BNbBNmRQUAadoO7h3dl3jG0Z4LXa5svK/uJEOV0VuEZVc24mOvo6W/iS9qUzjenEKoXU5S2y3KbpmaLdfeZsARH5rDBfy6o5H23KasiiuPY0CBnuYptQSXatg7aS/l8eVUd4iJ8hk6CQVmD/9a5SQSCRFty6gL+A2pR8VnT39GHLlorTN4KvwUxuYHSIn9jI8DYXXNWCoEMWh0umY+r+kmkBgzk/1h8GjBoziD82+1+s/3a6hEjkRZyLee2fxeWcgoq5rgRjmKlqVIUtJ6v1el9OPIxI2kmuLodC6nsEfJ5GEbGNEciHnSg1zXTkGt97B7+Fna/cX7PEMHE4KOUVx5Hal2JIeHXeJe/zaWtF5AFiQmW17OKSEssZ1P5G9xV+zrKJQurpnCWRtiYabuIhnVVjr1YQTWJ3ChIJFtuSMY7HqFC3bY2h7NoJsimu2M9+dckYAycDSHOs8RIgOpRMq7MbNo8/Nn5cZ36Ndeys3IEqZPewl0UQC8lZVJZbMXP9VPRTC0kKsooa0NwktHk3L2d7qrH6YpWoG82cqj3zzKnsl7GDQ4j/vWrkFtljJ4WCy6mjcgaz3p2hMMawS/pkimhr6DZYIvqjPfk3q6h7Oz4glru4dF8k950BHEOxkDOGd9DXV1LDWdXQQYBqFXdjAu7jdaPQP48KEHGKtQMEjTjzNj8tFlitpm/uaRuCVy/uhM5qy7kLhSP7SBEXg5m+H0C5D2IhctL7Hxo34kD6rHYFShlvrgRNSIya/1Q2uS4DbAztn5NPqJ8yJOAaXWIN565E08kWepXtjNlb5buU8xipSG1+kf/Su1Fg9j90+idqgV78f2MC20DoB4v3yiR/lgqbuf6NwmyhjHgg0y7hvSxk+5RzgUATm1UKb0QHgjToWTO5thuP8QZO5WZuvNdDrb+IY5zK/4g5zyEAZc3MCpyEYEYRUqQKeDZ0aeYe+A0xwu/ppxPRmMDukEt50Bskx8bbEUFb+Dp18CuzK/BESJh1+D4aSqhXdeeYd7oj6lxXYXRV63iyVKQ9/Hr8MPQ60/rkCBhdPPA/CJcw0RCvE7n7+zlMBGf24+8iEfxP4OdhjmD6sDoKztEJfsUOuUMvTCUqJ9r6Ed9jUOAeY2wht9/XkloBbNnGLiZFl8s/ELJuWsY0rit4TqyxBYx4DF17nUOB6bp5MKqYMb0nikEhUv+sFvLf+ihr1NiTpaA3d5wZBaUatoUrQXjclTqY0SESsPZ75HhlLGk2/NIqwhHJ+JFYQYinHZLvQW2i308fBMbB0TblrJkOkAC+2CH8/6wIt+JSxt6CTU46YksoN7VqzBP6Sdky0ytErxDeJjv8En9x7moxYt+S3xJFvkFA9Jh9aTEDWfakkqdeE7sforuffRakYkHuEvO8idcoKbgzHXmpHG6zk/+Dx3/3Y3qYWpfPT8R2x3Odg+bSvj1PFERc6mINRFsKqBSl0yx23wc4uSszX+5ITJiCxvxe1KIm9wXu/YPDPwUbQyO5+vywFg8PnBHJxwkDsMIoVzUMUnHLLC9kP9GLNxKLV3leGlcrHUCyIk9eiNWrIuDceT7GHdZDEZ+LxM3Ix0Ghbw941QtK46DLmB7I56iEZZABfyV9Eatxg9YnJbUllOWNl5BhVv4FLy8xh1+aydv4zA7on8GLKChH2hmLUuipL60F/YxMWGqXh5f0x2RToRfZLILalj5ky4v05GoFzgRV9Y4SXj0aujUNaVE15Yji5FR43ewkc3HiMwooUDD58nsv0ZRhy5gUUzEelyKcv7PsOomD/QqB7CeCqDTkMM5rAS5lvjILiEzYXPs/7aSl546CYlN3uQajToO2ro0fkjaGMQRmyn9LfPyDmXg63Jwqyts6iOria/Xz4LmiCuHY6Gg0EqoC3JonufF3evXMNrt3SF1eo6osu0GNoacXe24GcW595Yrcj8su5aE6UlabT4nGJRqwm9O4QTscWYNPBRF1yXK3D713A9u5bTg7f3JvAL7OAlNNNP6sv+mDLiWkYTVh9GkJ8Jrdz0r8WTXdIoziX+RZRJZIDZHw5QxnVzADtUccSqA2hQnaAg7WsCuycjQcovRmirepD2z0KQ9+3GPh2MbnDL/JjSp5I6nQt5j5V9v09BmGQlaNA1UuIbeNGexQRVD9k9Bh69EE5PZQQNER14PbaWn7Pq0YhuHceMXuRUynE6bSTc0mvwdxdznze836phZOBYvuraSqG7lImBUnZP2c09yQ08kPQaHeYkYD5LL/fl/KZxuOQebGotLUEiPNqhdJBcGIG13cbIEyOJromm06eTCwMvcGbYGX7yWEjw6aDJ1Ujx8JeIjoJzpuGM0J5ikQH+dHkj5bYmtESQ49aOom16JfuFfsTZimmTK8hw1jJV74TOs7h8BzC/CRL9gvA9kkyyJ5mXjk3g4oCLlGZLMXtgdeVIqm58T8vEaDZvVeH1STgTXnyOA+7/ccLvPDBZomCMZALBRjO7Ixax6eZclvafz/G3juPZfYJ/RbJaA1tFf1EQ2aKa6qsoDn6ChsB/i9H9G82mG/E9qpHqGRDRl2G1J5nx9zT6XcugZbKegGHJ2GWiPy9InXRcOM1fK/J5MP9Bsp4YzZoKkPBqb9Ob520mu3Iqj89M50xbImbHV9TdUkuQGQYzd/sgfDutCN8Jt8ZUikQQ4z7/SvpLJHB2fBj5cX8TYBzHxvl6BLeE7SUzuZTsz0VdPa3e+8muXMzhymUcrrwblbmdzGNfEReioSMskSbfLXgd0RGC5X8wov/VggGzphv1nmsssS/hu0f/JrJ8LVuVqxGGO4AjCHhQKIOJbf6GfN8lvb+duG8imYV7KBohPleeW2tTnt2Nw3yA4F16rBPnMn29lfiU6eycsZMcnyoeSXuS96RmTgfYSbtQTtL5M1wJCaMhvAGb05cmaxhVTnAIAqHV/ijl4eRkTWNz/WdsNgNqM+QeZoB8If3O96PDrwNHQhWlTphZEY1CpsAoL2NSp4zze4cSk1KDb0ALKgm4gv4kxRPP6WVnWdYQzoBjw7DrlWj63F5X67reIuuldRxxDaN84zj2xzYzJfcs73e62ZU8huPN+/E3+PPWC28RUxlDj7qHn1X5hAV089MXC2nv0y5WyUvE9fpmyHvU0MUDOg/T2uPYY09DfWE/gdUjKEss44wN3myHkqgxmH0SiCr/gacPPM2+6XV0BMgYqIIslZZX2uLYVO3HH2NqODh8CwBZSlgzM5o811t81+HNOUc3iTVtvPrzo8xaug+JKhDGHADBhbHIQ+dPLXgWRfH8tM3MVmUwGC9sjQps+afprulGovWmMO5bBKk4T33LIflKJmOLB2BaYCJ7czZesQaKp5VA5TOsk7q5c7DI8vWvZ6szLA1EYB+J59fRGtUPoU9a73GPQsWpqWPQ1X9CRUI70T0niY0FKv+3puz/b83tdlNVVUVMTAwymex//YP/h+0/yb7/2H+LuYV/BqZW+oFeCk0nRmLxFegpd3Hq+sOM/u1NHvQR+NUEV4I+QeJup9boz+S3XsQ4+Do8sUlsDwmDlIFEqxrwyAVcXl6otGL1QkwlVDslHJWYcFT34zXJQlyJq4iQG4BWfurUMS5oFr/KqjmW/CnJ7kSu6S/S6aVDZbkIhcchaj6LavToNj+PZ7aRLj8FLX4i77He5s2UXdl0eApgwmTqU8bh13gTt0fCl7X92ZZ4lXueOsbh0GeZ//BImoOT2Td5HxFyiJDDD6FBWCRRzGgoY2DedmLCFHh1KyloCeThqlEsCt9CqUVNUPJMfnEfQ6lYiaflOn94baLek8TVrzNxW+V8+PwbIBEYIl3Jsz4wTQdSj4VS2Uvo3RK6eqbjkXn+ASO2+k5nnf90glIhvR90XErFLrRC5krkZzKJa5mDxwMTN07kRtoNqmOq6fCIXNIdk49QMQoiO4aTeT2TythKqmKrANFRn9n3GGpXEMMZSXlSFfOFqRTYZIyuc9PTE0idrIuAHj1RszsYk3kEc7eeyFM5HJc6iI6o5tKRMaiCS0SPBbAp6tjU2B+325uYyBiOHxWIqI6gbKIYNhujLYB9A2HIr7iFZNwSJy6PHNvCZVSVwIoME9NPhNDvYjRu6QU86aN7x6E5uJlmF7zhB/d7w5QaB+VpXZRk7OcezWxaXUo2msFHaSVgwAVUIbeTtgDdHlB2TuZzZQEueSOC04HS7sajdnBmaD4XBl5A6Y7gWFUm504NYV+IlpPDT+JUOKl2wtzqIF7sP4cLcatYZ4ROQejVhFndKX4ctipemrCOrcoHUfnriKyJxOhlpNunG6OXkcqUeMIGhCNpzuSqLgelpw04QKf+NINuZDPxwERsk02k3kzF6GWkKkLUpZQAQaZrCHYPVbqzpLlEp7o+8mXeFNz4e1wETWxCQSBap5KGwKp/XHuh3Y/X1+TilASTeSUQfXcKtQndzHKkMy9oNdUS8MmtoicpHHOEjYNjf6PJr5CT0YHgacXRoyCqOhSF3YHF343ULRURehLYn/sThpQTqL3V+DWI6AWNTcO73QJSy0AG9i1jPzqkXYFE1Ebg0iTRFHCEwmiwuiU89tdQDAEqKpI76JLcTiJ26fN4risQuxnG08z8X4IZkDuZAzHLUWTMIUft4XuVja53V6Iuu5uHv3mY08MvczXr/K0WJGxqSUN3aDAqmYvWz0tZuWUlnzz9CQe9jcxShhDWEMHUPVOxDG6lp7EDP7MfAKquI0TftNJaPQZDUAIehQq1M5qoam/cPrMoCXsTj9SDBzfuypOE1FRzoW89PRrxfXZPeyTF26dgSXGQ99LnjNDAdROYPRCpgEVnx9Bzvg/HDN8y4vIIrmVeo8u3i8+6oMsUQeYNB5qOJgLCEhlVvZC3X32bwzYP2dXwUUQ253LOcS7nHI/3exz7eTGIPVgNi7xcvNuTyp+H0wlwRKKb5011xhTCKo+xcuVKaiJrqMgQ0YsHXzjIuU/O0TP2YQCO5sfRVRSHX6ASp8NJVHUUJoOJTr9OvgwEH7mLj0/kcU+WP6V+n1GaWIpdZUcCpAWcYkjMMWaH2lhYYyLyxi78yoZTmlhATVoui6NgU/dS5PsSiTE08/ipxzmTc4Zz40t4s8+HqGRWSrV38s7w1/AzDUcl8cKhcuCSaiDhQQjJ5YMtv+NS1CAIsOi3ZIyGe5FNliL1SHEpXDgFB8muvrz83gryBp7mcK6IWLG6JQy8/Ccx4bGEBwzGP0QsALhihwAZ6KRwpwGajR48dw5g5si3wQZ7w2FUHRSUpTNlZzZMuUifWQUAmKtDGKgCbxnkDyzlQp+j3C1/isDwvQB8HCh+Ch33U9mRzZNfPElB3xvsnjyGUlkyXBoHxmIYs5exUSUEPHqa308MZtD5QTizlJD/PN+FXmFrdxBypYbOcCf5ffJxKpxMrPmCHxeN4GjN/bx4YSfzsroZ/N07hMpgnXR67/Mz7vA4RpwaQd4zdk5/sJLh3uLcfs4XNOGnmFQPdVYNuRfv5OUDmSx8yU4fPmKLBW7Y+mH2u4RN4+LApCNM9DIxwe3LNCO80QEVMg0dF1NIuTycpuhtTOn8mwtZWsa8Ju4mpNX5uGVu3HI3ns4udlXO5xP1z2QoIbIzCb+qRbhsLjxSN4JUwKwuosFvA+7ADVjiRMpvR307/c64cDjuIT/jF8Ll4C3tIfXkD8iUMpqHVZMQXY6fFLrcEkx6E0GaGu4oXsiV/MEo1y1je2Y5npG+hDeEUZpYgsnLzI2MG/ST9wGXmCwuDVsFgEtmwirz45AN2l1y7vh+NF3xobx7/wvsuvkKRiGRVGcl6b/eiSZCTlfzTdJd2TQOEZN9bfF3EDR0Ci5zD8EV+ajM7Qh9BEIqxOT+U7/cg3NoJwHBerwsaRSmFiIgYBMkOAKs7Jm8h6jIRTz83eNURZfxx51/9N7L72xKsr076faXUJrYhlPjTf+yTVxKmIefNQvVoCHUO1Q0dg/F29qf8ZnZNDd/yYFw2M4VvuwI5qyjGWuwjMKEQ/gFAYN/BkM82Jp54Y4HOX51HuZ6BxWb51AQ+RsfT5mIwuHLL58upMF2k/zcVjZMWIdb5kYngVX+MCnqOr8kXicubCU7S2CnBQ6Hw1hDLe7Qb+kjfEG3xYuG5PFsT1pLrv5JfqhwUj50E+O8jOx7IZlCU9Y/EMwRCljq28MVoxzFBBeSi9MJPLafyJBIrmdeZ5WfyBThq/LmXP00PJULCKi5jNTZTf1LZwlPCAVdNHOLUjDOCuKt7kHc8DNjFW7T54Y2hpJ2TYejs4cFGxdgMpgoSinigBWUmgGMWLKZs6YQAgvvZe75uTinS/F2hnM1Dq72uDhph7MmI+56E7kqA9EBq5kVLgYjH/Ep4J6GaN5sLSHg2iPkedp5OuMy/nJYFHydLFsGnYdSOehOxuWZxelhp2kLaqUhwcU+vw/4i/mMPjoamQJuTnuKcam7oeggmEog5RkCFTF4WwaidHjj4E+2hdpgi6iL0uHQi1RQkSEcGXsEQSoGUJ68sZiCPpuYG7WSG9adBMo8jNaAl7uZ+70hSAZnTHI0kRV0tNeTKtmKqVNPm3M5gnoFv3ep6TuwiOZ+NznXImVH7lre8Z3HcLlIJ+1GStS1Xfg13EBZM4S5PmksiXufF8NFTTOVx8L1+mkUnvTCr7qWgNo0psuPwc33IX45GBLYdXgAlj3DKbv3BpvHiZStU+shNbqUM5Oc3Bv0PD3yRubvnSzOk7oI8gbnYXeXoHYYsDd14n0hjw36lRyIb6Db8wPdslrOUgFKAd+kSq7orpIgnwK3EOttqhtsf2s+1oQTSGZMxzvkEBKvXLb7rGJb/kiWx79PS2Q7Lkkz2Gtwyzwo0PBpAIwM3Qyyb/lmlw+D8gYwrcMfZV8tUypeECdZ9BIKndfQ112jWlCzdVEBrfqjJChEPyDkk6eY6BVF8x11/0AIfxcEg5WbeU53F53aJoR/K8nObHmedaPeorg7hE8eWU+LbjrNhVc5UgjvPb6PjLbJfBXWzXP1QyhWz0Wt/oP0wlQq4m5TyZ6NhACzhfa43xl3ZDYDf5rM9LfW87ogJagCnlX+gjT8HKk3UxEksG3mbhpiSvh93lMEa4PR6WBiXR+K3fkkuYN5KukT0vzPwI5viWuOoPjwUHommBgkX05RVRcl4W+glMB8A8w3NDHZAtskcCL1Oap9RF3SEBkkKeGVA+kMOzwO5eMe1BLIkA7jbOxpzB6Y0wj4NOEtcRJyqpMf19/P8pU/00flQS8Bu0fFyg3DkKBleNFhVLfQjPMbxaKyN6RaWgJstIVcJf2EGceRer7q+oxvHhyK3a2h3GYkR3cEeewFkkvv5YurYbQPG814jYd7TkUwdE82FSMqia4Mos1XDIwmK8Vk3+7TqQTXhCCkepAgwSAJxiQ086HRip8UNs/djNJlYEFwMI9GXuW4UUAasoWNAXDVDlOr5Jh6vNm76ke2+ochPHeZSttOHO3wkMpOcFsEowWIShb94wlWDc/6iyhNmyCgloCqw4enPplLY8Iwokc8D50FsERAIpHw5x1/EiWMwLf2Kfa3tbJdsQuDycD9P91PhawcYWIIRi9xvGqiatgaKmqnPtYqFs6otVb6x53EV2qgyAlTGyCiNogHf76P2ulwaHYcBXHvAmIRaYoC9J01dJT7URfRTMaNDPL7isnse7xgnBbO7R1C475BXI643CtZEClT8VmgnRtaKGybQNRhCb6jfVncUI/BcZ3+1/RwDayJNo4nqwkdlUBKjIo9RXa8bb68OiOAqtNDCNKmimt6zgiOhTdwJmtCb/GaIBUQpOIee/yBsTi0AhXzZvPNXx/icKv5wmng6pFxmH1a8a/Nx9kYztVoD5m36JQ/eOYlgm2+WBNq8K84i9aqxaK3oPY+xbGkbWwXnuSn2lwiag/gr4llmvJrrvTMot6YTErrARa+0MErJ06RcSafk1ULGD3obaYmfs2vBauZdnUarVOaWXR8HylnznFhhC9mz70Yyr7h07lniMu4yEdh6+lzpQ+CRKBhcDfNzgoqnHD/9xPwMvrSb/I0vNQ/0qmMJdTewXN+0O7o4LPfhuFl8qF7/h38HCnqENe54Jd2fw7KIln/8nuEd00k4dQg5ieYOBrux3X7TYao4Y/uUKotzWgRcCpvB+V1ElBJ3MgFNwYpGLok3PXnA5gmHuPZ9qepaJnE3fcqkSAm/TVmKR9dG0eVs5yYtCrSdW0I1//Gd6g/Lc6jLJOr+SjzMPNaHRywwkHl9yzte4W0oT7MOJbO+t/G8OC733MwfwRNrT4cHn2Ia3d1EOM7myh1AlZ/PWBGJneT4lHRXpRK+KHhlNxznKdzygH4ougu1mp/p7w9icGnAxhytj/X+m1H6TzEMOMneOvh3Y5EOvFDa1SyfP1yLMl29o+4wFth4nW/NHwBILBWa0TX9wqXBiu45rcVnUXHXwv/Iro6mpxzOZju7uTwuMOk3UxjyPkhTDg4gc8WneHdKSdYYznBSOVCLoq1LgwxnOKZWljaBE/UnkXTeg2HysFdv91F/hAZqbUv4mguYZ/Dl4HrU4hM0FAw7G++cMHQu15AqBMpOyUSCf0v9cfX7EtNRM0tGnwZ03TwYsAJLBEDWLa1GSRSnnrmGZ458BzDDsJwGdzvEkM5TcAvCgWnPB4Skt7HR9EMVyF3t46i1muwaT7NBaI/1RqZhe3pOXzlB5/8MAXrteG4HYkU3fE7gzOVbJkrFkl4d3mTXdAXV98Gti3LY8xf3px+GWweMydrRArdK3FltGgshAa/zuWFf7Cr/F/EjPBV+AOMum8U3lHemNqhzXAMi7qEfzdB4kaqkIFGw+iyvyF1DILSH3PCH3RGlUGihe6KbkYdG0VZQg0SZKh6VCKiubgNCMAjEWnPJYKCo5V3EeB/BGHpVqyBC9AeisPL2gely7/3nA6tDyXjH2H0Ai0cELW65QoDTf+byb5mwN/bhXloDKfs65EIcgK0dUxO/JbzrqkMOj+IyPpwFCMhtm4ykvQpXLbtAUDqkYJgR9vVgGvtcSJSfejyVTBP/yfXnYko1o/B9LKJNr8aLDqxP/4KK4OCzrN1Gty55zrXqn9E31GNxiaKtoXpKykzJrPmbAQ+lgim7xtIlF8S2snVvX1erg5kLv05LluIZudVCrILeGx4FeO0ENweAA4namsQTdEW1jz7CX/obVQ7oShajMOZNP3ZW92fwJ6ZxB+WEJugIkdZ0du+RKhjfHoZxxoCyLw2ik3NPmxMO4vMJcOny4ee9h4EjegXLV6/mLqIOrJH/oUQJsGzrJS+d8xn9sNh5FQE8ePyT2mVCgxogJUX34FdTmKeUbF5rIOqELFw8VSP+FkoB713E24l2AwmPDKIaL+LHVlH6fa5xJdfDcGrMJHul/OoU9SRWfU9zQkP8nEneCsN/HSiHxO2D8E1vB0JAr4yUBS9D9oQSHmSta1/44opQO4Pgd2TeSq0BDVmhp7KZPa2ATQvb6bZeg25pAGJSsL89MW8OOJZnlv0HBGFETjMDupfryfIXYjFqKW5JhijcPP/Fprr3VqO2S/qn38UBGxaJYdmbgcJNBjHoHd/AWT+D9v4P8lMJtP/+kv/L9l/kn3/sf8Wc7n/meybowokU9/K9RfX8WlHGrWNkQyOv4zkVgXCoXBYa+uhedULdI7toG3IRWLTRcTNGRvcX5HOCrWdtOYM7gjcR9mjZ3g7QYQbt5rj8XOGc65Qi33PEFL7B3Kmawyp08RSg5e6TEyRViETNLS6IUvnTzi/Y2kEVdeHUPEC+GTxZHgNG0LjcMT15Yf7XsahaOU5H3g3upv3x0fhO6YPzV0mVFaRDjDrWhZb5q5kjBmkJpgqfIWXyYDRy0iKLofFyjxWBLhJ87xFoQ7iK99kOG0M0u+ntb+A4lw6Uo2L/EtJKJ3+pEXdRUnJXbgnQkXP/dzZDLO8+mMInw0eJ0hEqqd6xw0i5ZChAokgsKg4AZ/DOVxdbiJSiEQRIofTi6H1NJJ0cYMnCPRqcNkJhNZTzDRspOaaivaSwTS81MDp9tOASC1jkMKiJrH6c0WmmrdffxNfsyiIvFAPLpee2qsb0WlUFIa/Rn3wWRyMQtu+kOP+fxLU4ebhL96gcEgrgXd+RLgv7LJuZcvsN2gJaqGrLomZp/sRmGMiVh+LLUDO4exIDgPIYZd/DTsydhI51IfkTRZyeubwxHProQMQ3JxsepRYYyWX8tLRhbyISefisUvjAci+ko3UVYTCMJlLk1+lWf0iPeoeYuVwqFNJ2Vfz8I/TU95PdDiNniYEiRtvSz+eiWoiN2s3bzYnMuDMADr8OqiIryBQBpmeSApPwrhd96PMcJF77QUOLrpAXYoHQW4Hu5PjV5dgvGEhMuMxvkkR69bMAmxxtJDdVcSJwniaNwwjYEAlD9z8i6pAb3YPH8aNKFCcnkFMQgi0hSB1y1m+Zjknh5/kcO5hun26aU+4k/AckBwHu16PWy46lm6ZlaKUItr923krbCoLNy7kevp1Ns3fxFgN9FVBn84PqCiO4sQ0Kaezv8IqDOYd33rWGEGiOMuJPm7+Ll/G124LR+Sivtdgtcj5vb7ThuXCQPQGCQdnhXI95gN0rlFsvPEq24uf4q7Xa5mkPIc6uoZxlvnUBP6Axh5DnF4NHgMrCmPQfDgS/1EWwk6GMSp/BSvfWAlAl28XXQ45UrmUomH3EnvhG/IGiRW+35ksfD12B96O4UzPS+G+9fex5p41EAC7LTBeoWH8ofG0Z3ZzNvl2UGuRHn4Lgbsv6zCeGIQiW0bE0ALC4+soUh9muRpm6wWebdZTLcAIQUDukiN1i8vlKA34Srs4qfDn+rMfMSNoJTHNGm6m3uzViouJdLN2ZBn7In5Fm3YHYSv78ULhcgAu9TtHacJN3vebQE+7hcCqYhwaH/5YbqbJfyFmTRHnBsHKuNH0fP8H0ZWhhNRNRzbwGvP1cE5SQufDv+AX+zxbNBAiB4cAgTJ4xQ+KhxWyM6gdVXUu446Mozayli7fLl5rhwxrMHsffoPz1XPYVbOULr8UtPbNGBXXuOqAcksJywwSCh0Cbo+H6C0yhhqGsjp2FDsaV9Fvdh6Dzw/GvzMATYCO1piB+DSIm6mrWVcxhcXicoFcJY6Vf3kezcE2SoxK9j30LTl+y5jQaeLeX+7l3OBz7Ju8jzInXDIG4d3sjVezisbMHs6li/oASgmsHj+Ca12h3HUpmDapF4UTN+KQi1WuNxyQXg1K4TraCZ3oHUNwO09SE1XDdPt6dPq3UcvNvQUOAgJ+LWpsTh965N4wSNQzfEv7NAeqIqj0OGgMM9PlVc9AJMilMnw7fAkw+iIbqMDsHdgbuHrFF2JcShp2FeNMVyFMsPH23o3cP3srse71vNEBI7oDUP84n9N9LXyT/CGv3ZqDm0zQ1uPLMO9vuTyuDaK96F+zi+maLF7UBTMqoAmbR8L97jY8ylby8gaw6Pdm1i8NRiKBbjdcsdcwSmvBa5qd7ktJvPruq4S8sg6KP++d6x82vENBax5DivsQf6oNV8IkSG7BXtPE6yUvkBCdy4GsvRRkb781PlIUOn8cbg14PHjcAs+1gloKuQEyYuUiOmS8X18sCRr81JGss0KlPZtT6qMEy6G/TxmnPlxEXGcAkd5NeKqgT49IqbPLAoJwlJPzNiN3eaFw+TJG5SDTvYnqWEipEpE6GqsK/3Z/LD5maspC6DI3MubWNWl7qvj7lU2816rH/P044gtLeW22jiSthZMXgziy+SYtLwwjojYIq85Ck882XvQV9Wd+/XkeraVxBC7pILgqD32iuEmdUA+5rgQezd5LlXkgYyadYYKhk52N4O7ywWA2UCg7in/eDNp9KmmYnMeSpG6E2Cbeu7eTKVHVvBrmJFYBdxZJSa5OxuqtplMP3waBDGjU9GHjr+PRukJxKtxEqN2Ea5vQKozYgSuuFHRFMsxKK+HFR4koDSPpsUfYdfUUCpcfHg9YKpqIvCnSooY0hdDl28UgrQe5wYJdKSHjQjgJN/rx1mtvccHu5rH95/BJfoaYcXlYrIsozJJQHnm99/2doYSdmRVc6nOCWPckKmX7GK69D/+yMbzYuRo5Ul70X8W6Jg0Xd/TDq7sfwg/hnHDAA82QHpbCd8VVjNo0C/VAI7/0h7XdKlzJV6DqDxi2HqWXE2mkk+5IPQmpXyPzNdM38DSt9QFoFG48UgGjrhOjUnyuDFKYpIPPO2GjGcYIHqbXxBPua+ZsTzNjtbBtxNNs/2wRxfmJWFe2UBABF+0XeV7r5I12iLR58VLiDlY0nWHGDzMwdI3nu4e/Y68FdGUwx3U/XflTkSqsHHhxTC+awA04BUAXzR8/PUxA2XmC9A3IXTaCCj8Az8PQ90MK5GlUjdhKZWkWbdp1dGv+haiDCwMv0BqZwtAEX3bM2MHcLXOZ//d8royezbyhgdx9ZhAGj5LHOtJRXVfimNKXMs6KemcSCUsa4ugyqTDvy2JDt4YXvhWfnZF1cEd9JNavZxMytJ2EVi13qYvxl4tz+KXYM5w9n8jNi5GEBYBfWx+uZF+hLaiVzzuhtTuGCO9Wok8Npy7SiXHUUR7OXNBbpUvMElaprjPnWCynbBn4X3+ChtnHSJ53jLYGf0q/c9ORs5bgWcvxtvanS3+OrLr3iGp6iedMTgLWZFA/8wOelWTgnaeBO/cDMFMPX9bGcFjezuj2MjZ8uQz5hIu8umwFNU5IKvWmLaWZkzZ42tTDk4EWpqiOo/DAEDUoHHKm9v+Cq7IMUtp8MHbdpmS6xwuOuqr4vjSFxH1ttA6W89XylZyX+7AH4OoKGPYn3wa3EjhpL1ZdKPFVoJbARwHQYqjnU0M7Kl8tY3co0fSIQamAiBYuRcLXT7zPpo4wkroWoC2rIHLQAarcSiLlIqLMrngabZcPTVkpXI8+S7x2Gl2Gg3wbBJfNzezOPYjCKxyfljGkB53AVvc8NQFigWKn0EZN5nS6tJc4lXYfBk8kMYSjlYBe5kQikSBIBFxyF8dHFfDasEviRff7FHyzsPAJB59fTZB2EkKb6N/EKcSCi9X9tNTbs3B19zBx30QqYyspSS7hVyMclNv58YEfie2ZxfjLuWQVxhP/ah/OudV8eOZPFqa/xffDCvm2/QL7r9gQpDKuEscjteJz2WkXaZqMYTpWrVzF014JvWtaRfMkJDQB0OHXhTuzmAJVAi5XKQDtQg3xtLJw40KqU5r5bf51zsXY6Nuci3y2iCCMNn0AeQ2kNgn8fWMCD7xVSWhsI3E5nXzuXcYHCTOQrCtk4vl0Ku6R06VwISmFQ8wj2lHEyJgFbOErlHYlDpWDaTr4MRhmZZWwTW8irXsO92x8iKLR7eSH6DnQY+Zqw3SaJ4iaOYv2L8JqEqnJvu0W0c8DTNHkXNbhkguU9hvB0owPANgeBsNqQSqVkKC0E6WAlggH1kGZGH0kzPtbFBf77L5ypG4pEreEMWeHEFDty3N9P0SnsLDHloS+2hdzk5lp2/tTHe3HH3f+wU/dsMEEd99MJe1yFtdH78b35h7Cdb4UpTaz0wJeUngjyspZq5tCWzx/mkASEIe3VEyAXu1SM+jVl6jMcuLVXIYgU2CXSDlgFXWSO+pz8fkxi7RR7XyceoRn/AS+DrGxqkNM1izWZvHAzkxiqhLwiY4nLqEYvNJAIWoLhMo8PKqOpNKpZn3gHbg9bpwCCDoLh8Yd4pVxOTQUGxl9bDQ/3vcj9RH1vKqAqQoFukNT2BeioCvufV6O/4Uem0/vcz0pvBv/WYcoixrBOb87e/++xADP+8L0owYG7O5L+5I/+OD5D3r97nGLysFp5uW/7idVbqduaQM7TaLUhl/3SpqGpGCt688rQ2diS9UjxM9h6Mavmbvotna1xdVMWfJUCtwwt2MPoQ2h5Ib0sFz4EM+2KdhqpfA2oNNxPfH7f7BUeHV7EVqvx2lxUJ+xGG9vO/Ot6cyfAXaXhm6zhJvB9Xjq1hB7ZQeO9Alkxou/fakNugxjUPq4cWQOodQ3gQ6/t0hQwPup23A5ZQxSbeA325usu+c87f1OAPBG4Rm2FL7An8ssvPKjQI9BT93AZrRDrgKgllvZNF+LTNbDa+1w08vAvgkHKEssobtnFqFh0/jt+nFMXiacWoF3XnkHj9SD2+kmWwn+MghuDsa304/g0Hru7SvqaI1vEWnZNyg38syUMLSOYIyBw3vH4qYD7u1oJ8LuBAXInRLG7JlC9LgL5A7eTa5K/P3UXcEsWf8yp0a3UBZjRxoho0J+maY4OGuuIcOrk+APXqBTG0H3tA081a+Gq7KtlDSJTD4LTXVM/+pJygL8MBenUOtjJyatioITfUjZ9jUuvyXoO4z4bHsJ+5s/Mtm3nuM2CFMK2EaspSF0MpYLKrpsot69Rm/jSpeCHYIN4ooIdcP+/NX0jS/i2+yRrH9nC8dqnuCm5CIno9fQbQhlixm85Vn8XjmLSwm/44sE7yhfKhyNeLcIRNSFcFE5Fe+2DXztusr8lnU8qg+j3qinwRJDSt1qLPqNoLvEl+d/5PElsDmqHnvcFeJd03DKnWABuUtOSVIJzaE9zAl/AVOch2NBxzAZTPjF1vO0Al7xMmCoeRZVp4q373uJnAl5SKZUUeuqwMs0kpTO07Q1BOLoryGkKQS5swNdVwNhpScw+d9FREYqUVle1G6opapDTmVuOT9e/IqvAmGgrpaQF9fztj2WdQ7R5xnRlcZNMdSAVm4h2bKDmjP9UHp500/Vj+IHihnRauHd0zLau5WoXWpGLZzG4Qc/YPXqEG76PMa5AeuZ//d8JLfQPwu3LmTlGwK1eUYSbySjSG9Bf66Oju79SGXJGKYlUS7f2jvXfLp8GHtsNC7fOvwH+NMaZeLfary40wDzphTwcEsBoZVS1teuxi27rX/dHNJMXWQdGj8NRZ/m4VX9BSTf/n1SZS7Rld7Y2q246410tQxit7aLqdNVUCGhOzgRRQAYK04y5tgYLPoDSBQy/Dr8WPzXAgozCyFwBJFlXThsqXSEKvns/K80+mymMfhXns/KYcKhPsid06hNn0Rm0FHSAk9hcxnockSg9J3HmP0RKOx3sSuxgh/zW7jD/U/2hv+R/a6AgNSZmPsEc9FzEW1PAjannusto7AE+ZNQpiCuIgCP7DhZ+Xk0DJeLtEjA3il7qU+bS8ZNC0JpGZmqZ3n3/p+Y2Kmma7KVhTGRvBzry8/Lf+4tcMore5bfXNlEe19HEMAxUsvq0Hd6+/PCkHvw1tTzwnvjiaiPRLXgGKP8O6j2BJCthMMRIO90sOPHQLyzXPx851pMBhND7WIRxxuhtZgL4rH99ggVM+IJixhJvdfXNFs8rDFewenWMCNpJtu+NRCkNaFaEoxEZsai7Nvbh2uebAKKVMQJkQx+/mdS9SZMFghs9+eRbx+hSdKE/33+TN01FZVDRV1EHe8YXqR40zPY3L6sSgdMZXh3KXqlSAC2GSz0zZhFSnw4p/Wr8UgdpNZ+RGGkiIx7LSeHtk49byvncjjxJ4I7ZxBal8lXVbn0OH3IjxmKOkSCxH0BmUSGBCm1LniuDeb7OBgm+wSrzyXicvtw/8jH+EYKrUlPIHWK+6a/k+bRnnKTB+t+5fEjk5D4xnLUX0Jrwu/sXHKG5wY/h+G1fTx55Unee+k9FFIFyZeW8cu918mZUcSbSW+y5vU1cP0ditasZcOni+EFK9whXl8vjafpNltUe3gmTfHDCP6344b2Kgbs3o9/3xkcyj2EV1s1blXL/3Ku/sf+37X/JPv+Y/8t5vL8k8Zzxe5aopOeZW7K13RLYqn36UYSdJJWAgkUWtFJYaCvleIxFzgRHcoP0bv4MlH87dOtUOdUcKg0haK/M9DPu0JDSAffd0TzoF81lsxyVh77gp9i34E71zLpxBOcuzSQM2NfBEYCYhV+YLuVJ396kLYcOc5bT4Yt+A68E0aAVwrv+sjJX/4L06NnI9wQF9q3/UEhdSFPMKCKCcXxy0mSzouaOSen7uFEBGwsDaRu3zDU4z189biIptqV7mR1vg9dgoUV5ZncyE/i0kSBple/INhXQbLMydVP3uXDvF8Zv/syUo8Cfdoeltp/I7dlI7C8t9+dYWkIuEkuSsasN2MKbeOZNninYjSVs7x5aOQ1Loa3UF06nuUnlnPg4SLQRYOjkxuNm4lQfUBzUygFX00mLr+WC8PjEKIWcqyglqDqi1ga0zANMNFwXUyurvKHJ3zg/m/24OkQkL9lxSPz0O59ChBpbzxCDxtqyjEZ/ZDXRZHd4Yvsfgn/opfSqpzUphdiDtKwsl3cUH8WPo45W68QPTyfd8aX8N5L7zHoxgMs+30ZB6adpGlAae98CdFEEmh/hDADeFoL0FtvaQ30/Rh8sxggraT++DD0QzrZ5D2bzshiOi1iEvaPO/7gYcU5BAEUdhNT9noRk/U0Pz2/kuPtEs6VxKE3yEm/6MPTDYu4svg4aSFH+TtYjkRQYUeOVvBl2u5p3Ei7QUV8BSv94JG4H5jd/ATVsRZc0d5cUJ3H4uNA4lFQFe+k0t7C9wnP0eEHof+kEAegxVHP4ZZgZtaG4lR3E3itms5+Y+irgrRblZ+vXLuPPHs7y3wM1KZNoDRxzT/acLtBsPWg6HFiVhX2/r3Tr1PUpDPI2TR3E93eIq3ffd6w2ABfjTlNY1Y+HmZy1QF9HIH8FgMPeMNes5NLjVPp6Emir2QwZ24l+/QSSFRAqnwgP84KJjE4mtqeHVj0FuSWZmwuAzaXAamsGYlHgtQtoXlPAw98/wC7pu3i66DnmGk+zIqI/cycYsEneClao4sm57VeR13mkoG9B8Ej4JYpcCoFPFIP93lBqnczz3aDFuhJlLBv4l46fcV7/EwbrJYM5743f+A1Yw7zfp2IX6sXprfGUOiawl4L9O/RYL44kMLoTu6bsO3WSF3hgBirIt2qIL45BmOgnS+f+LJ3LA+EwynTZV4zjcZsMONSCbQPtbHRayMjNWJhQln7fpxeVjyJVUh9VJzf74RbWsddvl10+XbhUUgxVbQSfV2spDvcby1mTdO/3UsPx3MvMmPDCPRmPdkoeMXPyUyLgx3B1az0CeOPNe8zOOtXdvoWEa8AaQf86VdJXb86JunGscX/294xAbDg5KwqicIoJ60eC+1BbRjs2dj019BLQSsY+Uzmxc+1gdg6bAQXKHGEJtAeo8UtKAiTlhJ41y4OGoPxeET9rnO5WVyKn4pT4SSn5FP+/humPT4IY52RE/tN+LRoiHeF8HiyhRUWNzKdimNjb1AZJc7Pz7vgnKqGiRf/xlXig/DQLZHokJfZ0vQeT7VCeYeRAZ88DwO7OJt7FptKrATsEcSgwp/BJSy+K5/UM5EUBYqou7HWcFad2I4EGS/1eYqNpQPZVRlF9LlEShOVyGbf1lDMcXZz9vNnSRoYyJoJVbT47GeOdD5pPn1YvOUjgupqkd+hoXLQYwzqMxVb0Azmq6+TIXVxas4xDnumIN2/B/fNm1QsfY/3SnZxxmmitCmBdLsMwe0mrKaTOn0EEYl1FLUPYH9EMWdlm7B6G9lVqiJ9w1MsWraX5hixX880+vOs3o3O7c/n52oIqj5L8cMfM+jqs5g8MEydyZJQNc0TfyRfbcen1J8R6ltFNFHzAUg2LsB3axDSqDCKcoIYHB4CESp0wp04N1XS1HYUyajbu+IOWxhMusjJbe8xYM9b1OtTmS+8w9CEfVySWLnTC970BxZ+RH7jeD7rXsZ5Wwy1flW81ibBLAhsNsMEgwmpoCJlWgp3Rb3e2/7xCFjRsot0oNjmJOnnnzEH1sHjoljH6gCQ4uHJ2Yfpf0cnfa+PZuOcY7wfmgObAyD1eTzKfmwyQWGPjPa0CzhC8nlAISY8YlKrac0aizZIz72/3tNb1GDxQJkTrlZOQ9XeTVH4ddY+83GvdgFArTof+R35XO64zHc1I6lTbccmQHTuUWInnOW8bzMXVuQzUHkv0eo8kkMBiknMW0x8cRydfTrJ+3UIAxK1BBTEs2/KNWpDxWIKOXBFIiGxNBGFW8tvD+7ngRAREbM0+zUO2tM5o/Xjk1deIqVnGT5hdfRoS/kk7VH6db7PhR8O0tWwh5C7h1OfNIrwkuNMPvIqU9VxfDB+GGVv/ciK1u8p9q4jP/VEr3bpifQ+rA+BRQaYViZwo5+M4nCRkWCBHp7xhdUyE8VuONpxO3igdPvxQuantFojeWPDcLxPDsGUUUGXdzMx0iga3fCjEV4Ji0QiSFDZVahL7Nx5807OjjkNxiKwVIE6iPRaCz3Kt7nD62e2WacQVqHlsz13o8rN4NJj3/LlmOcZ9ssrtGr20enbSZsb0qpBWhvCsNJ4nD5G+q9ZStqgG3wR+DcqichasHfoSTrS8tHXjqPt4jhmj77M8r6d9Hig7/lMdlS00O0IJlDXg+AR77OHW4gVeojKX4M5wJ+e7Ntz4M0O8eNWGDAF+OCxp3JucB6d4b+wXPAmruZvCBnPFWU6n3VsJR5Q9chQo+5FXjtUDqw6JxK5hGtZ15i9dTZ+Cg95U6bS1plGSmEKPrYwoicvYrddYF6alKKKheTWg17i4tEv52D3c5E36Cbn3M0MUC1lrH0tail8KVUQOnkPkvA0Ji79ktFe0OOR8VSbmyKHk8nh33N5sptGn20UxN6BS+5CAJ5qg7TuNLYkXSXgg03MP7aCNuczrKow8EacCbLeBr+BdHjyKLxgRB5j4dSwU8RFigmYgw4fOjUW3HIJHVcvM2NXEldy6ygY8zJ/XuthXZue0iFnCYgcRMPuRJztvky8lez75sI3SMsPsWLXVK7N6OTw2MN8O0Bsd2UHIEh5YvNgtDcTiR1fz1NeAfhTQ4lLz3sBZo5LDGinHsQ06iASly8LAjsxS0PQe5p4oBlmR0dxLPgZztxlIcj5JADjNLcQIv4iPXljeCsV0Q1MOHU37vxxHM49jEEq0uj/yye16ZyUJJZwavgpJkV00U8NP6vhIZeNan0lIz7+gIG+TnYoPJQ4RLTY3p0DuXEug01ZNwHQ0MOGxBuM18JGl5Ifh5wnwjOCwUeukuQ+QJyzCp1EpBqM4g1GTlrGp4X3MO2iQEqqgFxt450bD7Kmaw5nl9l5dVYeRybmcb4pgXHKWjZmbmF+ymwAphmO8EuEgwVtJmzlfnjr4jmeWM7fJug/5SrnLz+L23KRnHM5eKQeSpJL+NMEz/mIVfDLyq1EVIfhKW5jgXQdvwWf4Zq7k/K8j4mQ9+CVpqTP0Y8Ii69nSdu3SH1gcRO8LvmWpBsbOCrzgB98Eixq+r7bCda64Tym3oyydAxfxbfy5vyjnHWnQwXkqKGBV5mvGErEY0f4vGsASDzssUBK6hT+5RInNvoTfug0xsgQ6iMVvGn6im+XzCFuZTgmvwY8cgmemnr8G64T2D2GxoCDZFZ9x/flM0g/+QNBc6/y2OZFFGQXsG32Ns70wOMtkBaiQqMJRtKmxccjZ5ymnUFaMyUuSHIl8agfHDY5OT2hDqvvbsLkSzF5RHRgp+oQgU8XcN2qZ4d0PUbvdmbe6u/pSLjmbOXJch/qdjzDhqlH+OrhPyk0h7Ht1EnabeFI3VZe/eU+nBYt7an+jBtyCKVajJCrY3TsexreWJzJQ3+9SpufOJfciDqIq+45yJqpJ2h0fER4fTg30272vq/0EngIA3KLmr2XYtjQ5MeAl7O45hBlMRQON9OyruAKTcYnewqPDXqI49L03t/j/QK1aR78O/MwP7mCX+87xD1jTvGiL+jL4VF1AIMvziO0TkB5v4EXRi6FsN8hdikAOfoQMs4mUsZoBm0fRtjwy5TP38HqTgenRpzCa6AvnrNtZF3LoiiliPqIerJr4GOdFNOpvngP6MFY3Mk31x7jqV8+JEYOD3vDFJ2JjORTnGgYQmBlIA6lg26fbn43QmHbIAqj2ym8cy2tga2k3UyjJahFpP/Xx7F9O5zol8+x/nnkBuaCCYI7Z3HU9zVC18KRXBNHy14mLEwgzeKk/ZqSq4NHYJ39I/Gp3oQe/ILr0lBeKwinq6mJB394kKUv/46f4wAjckwcMDwrjl1PD7oeDd1aSS8tWvaVbMYdycI+uB1jYDw5Ket7h7qyK5tRrZW0aBuZE1lN5ZxLBEZXkdTUn8PuOrZULCGqTyg6lZoIlQabQY1b7sZPCuO10NVq4IdXHyI47hxCHzc7zDBFdzuE92vhH+xP+JDg7lk4/Y3k+1zvPXaufg7V5UUo/hyPcqiV88PPI0jcaG2lcOlxzEZ//Nq1uOxOnEoncreeN2P20Mc+kul6WPboWs70KIlyze1ts9wJqVXiv5MTGmiwufCvSMTgiWfAsDvZ0CyyGiwPOsi4+mRe7oEJL/+Ct7eVXa45uHu2cNwG02L86BlYQ0z9CIYfCyLoyUU84pPKOpOAvXE+KVlD0UReoF4SS//s/kQln6ajqYZl2S+i9jxBrKqLRi8b3YFyroU9xE5ZKImNp3gvqJOY/jU06LU41HIqBl7ii65I1krq+TYIHvIGYj9kQW0X5r6PcDOxm+mSPE76NtPdquQxRTQT/Gv4tsFE9JUdyKweIlN7MLz4Ms1x3mx6sYQal5uX1w+k4IsJVDzupifsU171hTZNKe+nVdGYE82CMjd3hpZTI3uHNfUF3Ew4y3bJRUZE1BP8xWle3nCZmX+1siX9T9YFR+H0qHgc+NcmuF/DQqKbltGqfpO5W+Yy5vG/mBReidyg4Q6fb/jKPYPTw8VC7CF1sK09lIEFAsoJKkrD6jnvdmFvGYNGWsFQaTRX79hAu1NK9MhH+cDwARFtd2GsFn3RxwbNJn7SOIIHLuAjUyFBphbGOiJRSuBRH7DJZJw0dGH0shFpEvW6TiTe1rPvdipRHirhB/Mppt2xlBk7Z/Drsl+pGWahX9ooXgmwEaitxPfeX9i9Kp/MIxuwD1Si8PHns6c+4+EBot66RCrBgwS1qQXDz4vJG3eBM/f+TrzhXaRyKbp7srly7UrveRtDG/n2oW95NuQ5RryiY5lrhSj4jsietfaWVNaPRlC21JF7Ywzn+p8jvvMvrMoqrsYu741J1u69QXZhEgcm3H5NTjx6J/41NynfHU/PZzvRDVyMR5YEZ+aiMntRHTgYqyqc2OF9OflIK0bbQALbAojt+JYrQ4w8ND2JonOQftlIUE0Oe2eLCZrQzjkMPyVneEI4Dc35yB0WatMnkR18mIUZ7+DxSDjdouNQhROpB2RuGd3DCzhz2fO/pYd41AVPnk5GkRHGoPY9SAU1Joc/giBBI1jYNG8TwZ0TmVs/k9oefy4794HqdhtOWSfGoESkb7xOex4IeCh2xLNbXk5rvBGFTtlLxw/gYx3IppsDCS05RkTXrzDpnymGfeX3kRSyh6hJp7nQGkxNYjv7lSUslQ/GJoiMOLvKBqAvSkUSJaN8kIiY/ev836j6P8ATQS20pMK+GUcp0Wdzd+iHRISUUXzyJ97VLsHPNJCZg8DQUYOsqZ2OSTM5dXMIsep/64REwkvvv0RtipGJTxhZ4O1gTRXU6SycGHGC5f2XY2+zM/Ci6Ds6FU7cci8sTl8kCAge8EybwYdy0QMIl4vJ5OuR1yl1z2J0sEeEVAL+ppEsMcByLzi0qz/d28eiebIdkUVXglldyDuyLfjYhjBc8jNoYMyWAiLKBrJ96e34BALcN+JNLkUORp0RCZehwwMSSxWUfgMpTzPdp5oL7i6kTUY8hZ0E2M+Tar9EfV9I1jWgC9RhjvehUL8Tt8yNXCpHZ7yOTgEFDxXcPpcuGlO8nNMzdjEw7fZD8K9kXmipWNxtMwRS2XfOP44BOJUil3G//H5IBAl9C/pSNbEDm0Kksv2P/ffYf5J9/7H/Fvt3Gk+528DhlIFoHFF8U6EgyuGPUu/hYg9IVTPZVlZNteEgDXboGLWb4eqHEHpgWRMsiHwAg/QHno3MZ6dLx6mF11HLEpl5YhQzRxXTpg0kQN3KY4OX8011CK2B5VwQ2tiSvIYw71V83A4/BAOys2jVMmo8/ZEUyvEufJPqrGkI6n4QIPIr7DH50ygvZUpLD6OOD6Yo+SrTtHX4S2Gn98v8xjxkCTG91xVZEUuiIo8Um4GAK31oyfDArQTlZeN+TjotnOyCvY2hyC9nIhkj8F4n7OmaRGrHRXoqItAkfM4Dd8OpmgW0Xy2krFKsog2L/JnXvaBbuMDS2b7saBpA7esLKE0sRTFYrFD7F8f2kwobban59Pd5iqu2LjJ87oI+Il2R38G36XOxL3VmLcasdhyKdmSyfEz5F7k4upOD0ZNYPuA1fK/f5ng/agVveyx9ug/TVmjAY0ohsiaSDr8OLHoLS5tggMyL7PaLOKtiibQGEFIdj1Vip1/QaerjoMTeQFKfjcyvHM5jb7+IJrGa+c/48eHQqQQnVzHPpwuZj51tLQ52Td1FW5QdgJ+CRKi+t2UXz/Zfx8n219j5YCVu1ybeAPCIG+grshi6X/4cs2Ih078eilPaj28e/QZArGCUqXDbXYSUnwXAqq7hk05o7gmgdtJbjBkDhk330HotkYX3nuUFCxTbVZw49RvNhwQyxsXw210LeukL9lvB4vRm5dBqXP3tfGC+n92SFcRIhzGwaDubg6dg7PGh2OtbTHIHMVcyWXp4KX8t+otUvZMzkXBQcpSNoUZWv7SaDxP2c+OvvhQZNrHkVhTEnfQURVt2kHOig+b5/WiOy+nVt/Nv8yfq0mpubJyGsL+c7Ev57O1/4b88dx5B6NUmAni3A37uhiO+lQi+MLtpCms6B9AYYSakAk5FQIVTSuXqWcg1VrTRl1jSuoQ/7/iTwzZIvTaIb8MnsbH/Gxzvfp+j9n3oTXqMuisU9AvATybgsXzDm3tm476YhWOlE41Ng9Qj5YODj/FoLGAAn8FmcETCECubMz/r7d/E/RPJuLAD8zuJKOwWypPBprExWw9TdO3k7RiDTRZM9P0D+FG64h/X+nv9EFaFHyBFpycouok23yaS5fO52JrILFspW1OrGfPNB0zvXAxO/ou90d7Dzd/u5tSCergta8QTrdC3+0tWBD+LyhHADyYP/yqy63CLqMIodRivVXxPXa2bD9J8aD5VQnxCPOUJ5b16Z37+Aeh1QTTHDERuvUlwczBGLyN6s56Yqhhc/j10BBspjy+nT0Efdlgg5+oo6jw9SDR5eFwC5gM9/H39LYz3vcUB51UOWGHqRSvRyJBP/YjmkNv0cltDodRUzQJzOUiOEBXQSo1BRBnM18PGUDjsvsnKXYvwPRRKz+weNK8+S+m1MxiCXkbldxEfySDmpbWQ3+yh+UoT6cc2U5ruiyPNwfiD80krb+CQFhYvNjDrt9kcjBc1FZa9tJaqNj+k3RJ0g3zoiphOdczfvX37MVigfvYu1ponEFCiZGLZRNRzlbgRk4EKlwPjmCMoQ5Po3cUhOjC+MihyqNhosuARwGBLx6S5wffaKIbpzuFrGYxSsNN5OZXIygguDK2hJvQqE9w5cGoRRM3lWHALtamXCUkMBX4FxEIKnVJLd+gwJIFtVFkKKYj5kE5pOFqFltF1MFXnZmhWALUWG10cZG54NcktX7AjykSPB553ebF2cQuDvYYw/e0FnBtyndFDg/C7kIHN8DOphl3E+p9gf+kbqA0WJAo3OxoXMbpzP1q7gdBNbyLkldMx+AxD45tJ7N6Hn1TUZwSBuKAv+PrvL7B67efY7EmcdfXlmSW3x8fLUYKnroCuIF/MYdFIbm16NncfpjurjpicVCILjhFdNYu9k/eyu58So72NnIjfcOdkcMzPF/f3Tm5kJyIfLmG3BVqdciLK1tFqTiRE3s7eX+5mxgPbmNlf4JtuCJXD5id380WHlpM19/JEoFhYUO0UkQh6qZtJ54dy3ummre4KbpOit78z9VCHgNR0FzXmGwSaR9KqO8ZD2stgt0PzMVyJk1l1MANNjz/OPj08H3mcUAXEVUK0eQiqPr7oQw3snbS3V1fsq27xMzFrFgq3NxHyPzB53abUeNB2F1p7HNNdK/Fyqnjw8DNUmZT8veBv+hamEn+1L5r715EnNSLg6Z2Bn3f0o9qkosIlIO94Hi9VM544F1tiv6TbXxSfHyJKLFE7OBivB6WcbZqLQbqLmw74+PqjzAo7Q2iyGmmtuLnzCAKlCXm0ex2h2TKPVbXDmGEeiaUkBoX/FBoTR6GwW2gPcPLj2HFIJfBaXSw6lR8tEd00KEW640g5zNbDiZsRXNk4Hd2IbqoSf+i95u+7xXXrPZMXsdez6XT1Z0B1Jj6zUtkTt4j7myGoaTaHotYQMaqHK/0b6fQq5nnJqN42JBIJX2cXcDz5OHVHfyBhVzxv+J+FpgMwXgxA9bmcwoT992B9KIjlkU2MilnPlOKXOdaWgltqQ4oM/+OXeLLkSVatWIVLItDphiWtEXgdGo8woAfV9Er6Z1ziZCTssUBqNeBfD/4wN286p/aMIDK9Cr/gTk72QFHAGiZ9+hQ9GjfX5vTndNDdgMhKkKiAMOkP+DcMwaM7T3BTMFatlb7hczlhEp/95rarZN+ZwyGLlM6qx9EGdxAnABag+i+kkkjuWHcHDoOWqPphxMiGcPzZr9hkBq3Fh7iaCbhs4qLy9mtv40oWFwiVYOL+y7l0NgQhmSpFkIJECh6pmCg0C7BnwlH81Ek0xtQRoL+JRTGHdc1KZusc7PEO5MTgvSQ74nmgBSrtevzl8TS5rnDaBomqw9QENuGROP9BiQbQrD6BJ/BD1p8uI6QpiVE7l/D3/M288drV3u9UeT3NxWkgCyzjUPR9FEu1LJ1WROqWAZS8+COl3VNwV1oJbQylySm+TJZkriLXoSU4wsp0n/7Y+r5HctQXAHxw5XkOtsUwzycOZ1oXPsEDKBz0JekxcKPHG29pN19HNPL3kT70q4ok9IiebwpH0e+z00wPPAjAxebHeE5xHACN3Zt3TJ183+9NPr32gEj7pwjkifBGusLaOXbGQ2xFLMVqE/RfCNGLAFgVaKQLaC3xJbgtiOS5hzlihayL/Xhkbw7t41o5PK2EJp14zh+AA5WwqCGWhvZgvMdIeMVuZapKlGs51wObzRA6+jTt6VdJPDuWYVcfJPOFMsYHw0kbbKycx3PR35BnspJ641dO16VzcEA2acpvyYsCu1DAnu9mkWK2I9jPIAuIRxEZTFaNiDaXYGWiFlrcgETcLxkcTbA5EFKfo9CeyLvXgnHgxYijCfSogvguoZxkJWhDN1PhNwRZ4Ag+efoTHEpH7z12Azf2DyHI5Yt89nSefWwRChwURYjH99UXc8+P/bAMV2EYmkdWlOhDLDKI/mKcupC2jlwMRgcTTv3MXuVeJkdv4uMAmGP+iuSSNIr3jcRr6QF+6gaZNga4wb4wuGavRPBdz27PUOSl4Ty13Z9f52zmvmVv9Cb77s6awvFZk/heE8Sh5FfJUEwH5jBB6cLc4Y+zq4cjM/I4PuV9/MxjmHHJRlrgOboCWqlLySWuTzj7K3+iKkqEidx0iJ/OqG5qFHY+8A7lvc/EavuvW6KZ7VfNnTkfc/Fwf4SqRH7NMfG8DwwS1hEjh1wtyCw6Xo8zsbNiJlvte7hhh2PSXNq6D6GTQqRUyRmXGi/vblxSF41umObXwLQZ8fyc/xEy9whG9S/jYreaYu83+VP2LQF9AvBv+4IXbGtwNT8hUufH+VIXIAY4g2WQrIRnnSYuaU2MCVLzycvv997HdcEQq4BnV88kqDqSYWmNSAoGEOkp7/3Oar9IHnp+KzOKljA5YjQJXmZa3AIPeIl0xcfabZTFjcbmqmC65hQjIqo42NKXv4R8FC5fAqROMnMDqGjN5pBhJTfKMvh4bDK3GAcxK7I5uXMURPvgO2wfA9Nr6BM7ltWdYsGr2+NBMtiPn+/9meVrllOeb+TiqDDe9P0T28vvkqN+AOFqB9bAEj52P0Sk/GNe8IP3y6bwQsF3jBvrRdIPKipjK/nzjj+57oDWrmCGhhTiji3HvXkqA88NpDGkke8f+p79hX/zy1+xZGUKtHhAX6In82YmDt8+tPhuQw7UmgvZ05xBuKKb4Q/4seFmB7/Zwtkn8wKgRxnP+nOJRO8ai26FF5LpZ7H4OKjxWUWRvQe3E4x1RuQnTnH/5TF89OwlzAZRZLk8vhyZEEmarxfd2svs7oxmgXwsQa4jfH7uF9Q+HxApdOEM8pCftZNIRTbfHC0AYFLsB4zITefrxiQiKq+gsLUhdUvJs3uQlMJ7Gjmdk49R5golpMHA6+ZwioasYEj4NmJ8r2Aym3jsmzl0+EjYe5eHn27JXzQ6BhC58A+ef3swacFNuOSyXk1HQekD0Ytp/r2NJ/YPwxFvJrwuHIvOwoqqkUzQwg27hN9CBFa3u8nfvZdXjr7MhWVuDoWL6Nb+KjgSAa+12Gg+mo1PRw6NwxX0UcELviBv1nPk88WED63gTN99KD1ePC7fwtG2JRSf/Y1HF9xDfMY6Fv+4knLVZQ6rVoJE4OEWGF36FBMWZ3Mgfh4SCSSZ6wHoE3KEPiFHOClM43rYRjYPbsFPEofE5Mf7A78nNxgIreabwDWYLfdiMyj5PWgnM13DuBhmwFdpQhBg1SPPEpgsR5s2Alk3SHmPwY2h6K5nMrS9kjOv3kPgxAYCa/Px9mviL64iOH5D6ZBzoLMPDSY5PXol3amlfBZxiIAwUVv6jK2dIw2rqW/1JjHoA8bEtHPCdY4ATRMlTrDKDOzt/oOzZ5z4CCZAwsT4H1k69SPu2V4DRHLI6sO180NpvFyLVCqha6kYa6jSd5Ncvxq3G7y/Kuextsf47qHv8Eg9e4YhVwABAABJREFUVDk9/LBvAYYbEiJWyslfvpar0fBDhx21HO6KWYumDHpk8ILwr+IuCdcSVjOwdSRKrw6COz+hrsaAf79S/vCqBvNV7vfAFFt/Hhz0JEvWPsRw3yjWGJ8nekQ020tmYdDfpNgBlu5MbEu20p2QTMqMFL7f10Nz8Gp+N4G/FVRnfwe3le/vlSBRyHEpNMQXCwxrmsXNtJu9jF9NV5qQv/1977Y7ryqQq6OrCFeImSij+Z9gAYfKQXNIMx6teNwl2HuP1d7S1vtq22ACrseg9Gmk//n+FCYUorMn4Jb20P9if8Z8PobGfo3kfLaAFRv7/KP91sRUjL7ZzMkKgQlDKQz8klNNNqbZDuKSxhNzpZ2Y2hQivp3O2PCvOHsW9MoOUrRhNIaNIDjTD+EsXM8ZSodBSXhHEk8NvhuDoo29uwdSddhE0dB7es+3q/QxTtUuoNnvYdK/mADTWzk4pZwWnz0onP6kTTEzbR/scv3P9RCny+GJAWEEt/ti2XmCnB4N5QMmI1O1kBV8BLfXg8yu7cTYLaM9sYy1/R//Rxubo3ezdZuSrnABQRCzNBKplHe6XmJt9/30UQsIHoERJ0fQEtRIUWoRgdpqIgOOUVV1AanJF6E9jsSSROrD67HqrByoWI5DUc/bw3YyvaYTt204xVYZMlkgHW3jGec8SKQnFsMyFdkpHWABhcuHZb7dRKrMpFaB1iPD2KeY0XlO/np9Doue/ZMIbSN5iFqJUutxhj+3iQuOueyy30FLQgOVli8ZRgYA3hhpG5SPNcCXVdciGJ3Ywd+hXWQ5LRwZd4QnhzyJRq7hx/t+RGfR0RjayJ3CILJil1H4ZRwH3hjCecntQvpYObwfAF2+O7iSuIMNnT/yxOeP0BrYyoHpKxggg2wVvBPUjHPsYQI7+zLsxjDqYxRY1eJa7atu4K+53uwrf5Cfeox0amqREEeyQmTPKnRdJ9VrF91+MkqqZzA/fyBBmaVIkh6D+PvA0c4Kq4Rfty6kLltL14IJXPIdSxJjkeEiSSXKsZhS/DmUJdKLyiUKGLaBzptFmE4Z8R7qzYcBHxI/IZbQuxUEjboI4bn/ZW61R2Ti33ANjakVr5YyUftVuB0f7vEKojopkuiSWtwyN7un7CYmYKDIuvZ/sEkkEiIjI/8ho/X/pf0n2fcf+2+xqBgXnAcvax9UzlBCAvcS1TaAzE/fBm89FckFDGm4l3tW/Mx9GQLZ1dAhEZ0dwePhvZ3zGJNew+CQH5h6a5Pa3D2PS03p+FnO43fAxce+BvZaGgj1aSW76W6eiPwNuRxWub4iPF5Eu7iBCVqocjfTEP4a+gfPoLKOpsSznYLUJ5EIR8DlApkKn1Y9ScWjIMbI8FODMOs6OBRZh8aq4fEv59A47gCyMRMo7zef6GvbOZtdQGgl7E6t4o6f3uER54tkHszErDdzKUSseg7qmsqUAXuQ9D/EJPPrDLjcF3mMmjuLsrl4YAidy/9ElhGEyxHIhuRfOJUoBtCWGERtuZ1WG1u/zaWhU8/x3FasOgMjZXoGqiBN3oxEcHHCqsImhzFRWcjalCyM3AnGePBKokOdxvGFn1FhjiIxYwnrWr5jMdNxyTTYFS1YtT0glRKyMISZiTPZPms72yxww5hK3n0/cbR0MR0141m+ZjnbZm6joG8BJ3ug0OPGPc6bZruOM/4HaOESi1zvUS2rZaMJyuw6pGYTzS4NmrhK7k1sxtylR7iYwUcyE6/0rSO6M5TNMjcXB17ESxB5oUdobgU/bPnUb5XjLtzCmC8f5OD5Mazzl3Gn0gOOLuolgWxVdTJSqaSrfzkWWWPv3NOb9AjuLpymYIKqxYRYTfhFdnXAKz5NrJgexw7rSbbmtlM28VeGuxdzwV7NiAYbi5szSW5cj7vVm8qc24qzOyywg27G+8lQymyiFooEQKRierwVvC1hbA97nStN3pyzTSS6PpxgaSJueycnbI10aX3ZEX6Zo10JeHkFcjrhDUp9vqXHKAaPNk+6h76qo0iQ4HQ6sctvJ3KUDiXe7UaMdUZscd7UWjW45Lcd4vFHH2fwqSCUof8Uhr3ugOv/9n/vSgk1u6dx9+treCVODKw+1ubkbp9CJCoF/h0xhNeH934/s/p7lJFVSLtMCG3tPPrNo5THl3NlxADu6fssz/nCRXspEzNq2emWY5g0ks8RaQaf0N8+7/v+Cn6qm0Gr8PU/+lcXUUdc91zkajmJ59egcEKaejJ39b3KU74w5Go2bWoVA30m42Xtg1FbAMBPsqk0xe5glUmBIEg5MyYfo6SGl6Wv9Qpmv9Tug0zZgL/Tw9t7ZzK9fwV3RogVjv4yaA8LJGDOeeIUjzP4fAnlcTdoC2zj+254unsmvrI3KXx9KX7jnLRdcTGlegqHcg8x2+HgkO8Q+t0MY/CWjZiiqoi9dgLfqlzKE8oZfmo4uYdz8ST2oEoyUJsxBWlHI7O3zWb9ovXozXqm7J2Co48diUzKrmm72DlDpCIqt2h5r8VN3R9v4H7eSMmgJQhaA1pEzbTkhldIrtmGor6VhtEe1DY1DqUDj8zDSA3InGbmr5lPTFUMVUndLCh9jk+e+YQKp4cfuiHAL4p65wK6suuISI1CekHPjZiX+SqigKVeBXzsehLfEikqj55n3AIggVtVzUEtvhjaK2mywI2NNyg/WEFDYi6BVec49Nd4SstDiVdqYAZ4pLeDjwv1kKmCzEkn+bWhkLTdn6E754vvwlzWIlYHP+JrYMVd+XzWuJjI76S4ZT38vux3BqhFnaA32iQcWfkgI1T+GKxPczzna/L75TMq+BRaRwMVfpN5etpifEwDSDZ9gl5oIydQCdeeBK9k5Lr5NMcPJTgGBhwJpc13ELKpEHlazcLJD7C+9As6jPtJvFqDMaIeRZSUbg/8aYI/2USm3B95nzqWJt4WjFdLIcudRFXFlyRkGdHfO5TBUd14dUFYynfc2SEl5foLxP05hK4J5bz07o8AdFf+Bi6wqipp9hoE0b7oRu7mjj7iu6oqFvZb4LStHJelh8Tzm9FEO8lLAglSqNsJUjmETWZ6xlPov67g3vzRlAtvE+hIhjM76O9zgicmhLEs/jH8D2qJvtKHAxMOiBTN5d9TZZByad5mVNpc+s4rIjCkBZP/CpzF33HRFUxJ8w2C668RN3oO2tHeRCTU0VcPGqkYEN5vhRK7jBvOT6ht8iEypItoBXzVBVu6h9K514BV00P1qEeI8S/uHbMhtbBNpuLQ1dfouNlEQNQuvheSUYZVgw4YtYPEHhcTjtyH0u5gT58yTthE6sVKF1Qa/iYYE0j24qXeyfnEoN62dRJ4LGceBW2DqbXF49/mj1lvZrj3k7wcuBtBdpyzW4eg8gQQ2HUO76pE6tXQbVdQa1LTYVPyXk80pxQOsjc8yL70KvyHdBBeHc35nCISwodQPNSXxAE3sFZX4mWPBpF4GgBNTynzUr+i2jSWURfjWZHlJrLvWkoCd5M5bjgj1vzMU91JfNrTzXaNGIhZvHkxAF8/do1HvETNRongoT55DKXhn/NCRw+eply8t68keWwYB3zEcy2NfYnG5vf5PBAerQCzU4bb42Lu32NpCmniyLgjlDihxAm+F9NJ3TuEuMQAvErbCJw2id8NL7HF2Uao5DKNiWWUJZahtYtRE5lUyjA1/BIMlz3n6as20+iBwoxgRvY/QUJsOcTdA+pgcNkYEdVMc/Y1/E4HcbIkkaiViYyK2016zWd0XNOgSpdhi4uiIHAdILIIDVbDm2ML+VW1EO+EdM6VfMyMsPlABVN0YtKu61asKiztfs67TPhE3E9yaTgnI+BNP9g0/iAyIZBM8xc0uQtJGFyKqn0L28Mg33qFvusOMP9SNg9/8jBnh5xl/6RfGaeBNCXIHG087QsWj0C5TcBu1HFRGsOA5MmQ8CATDtxPQVc2xUhxZVWQbjDToYRNwJQjb5FxqQzzjESe+PwJqqOr6U7YjUGpoT78L+bceSfH656gru0GHms1LcZElBJ43Q8qjHG4pDtJTvJGJs3gcDRYLZ9y0TyeOyN2E+quIlOAkw1u4spS+dVPymd9zGz7v9h7q+goz/b/9zOucXf3kISQhBCCuzsUaAul1KlRp0bdqFGhBtShuLu7JMRDhLi7TSbjzz4Y3vC+ex/8T/bev4Pfe62VxVrMzDPPc8+t1/UVX3CsgC7j60Q7NdJU9SEBkmBaPZuwKI3s9IHsvhZ0mvGcKUrFpi1n79y9NPo2gaEdjK2gCWF3x6ecHrIFP8MUAFY7WSH7WZLkOhqUUNptQ5wSwieBn+AihodrvZmlX8rc6C854Qcb+21IJB6olPZ9xsuJn9Hh9xkJltHEjysgu3EaV+2EAI7pXJig7mGiCh6dfIbjU48yvHgu1R6lCNJkOnIncKHmPsReamJaYvBp8sFReJ0ZYTtYEfks43zhlXaIdI5jiEMffQJUlRiYkreCFZ9thP4aUPkAkJgTTeGpVI6MrebiiJNone3FOoVIhEiwI5P/Vej7V1RboPnwGGJavbBlWOhr8ETvqkPtoOfVDui1AcomCGhiQdV4FEYFeZZowssdaRP38pCyk5jvVoGHAaZfINNVzzHRJOos8HI71NmSCb8ahzdipNaTmAIVEOBFss8xXJTNIF1J+PkAAmr9efHkJJrSbjE++mWw6kDhQYHBHf/vlhPsIub66F661YWoRTCvCdL+nkaUuAJh0Wh6nXoHn+lrDwiTwfUdk3AP7aV/aDf5PcPx8PPHWbyN20Hwm/o4edG+KLzzyB96mBeC7wEz84NgklFD9gsbGFbwNpk766hwWAxBuxinhouhbfzWvo3mif2E1HvQ+Vcw45c2EtH4Ji87vYeTNY1JweFcqg3HyVaG2akbi9TyHwmQfOUIKuIFmnsb2OUDC7QH4c7PfGWWs/nz1fQ/WIMpoh+bxEa702kUiiY+mjCO272BfBUfRUXgK5ycehW9vObff04+/WMdspsC8lVGnpHDw46OPOVpf8++Tk9uHHoYSXcvIwIvomhLQjQ6ly887ICJP9qaeHLHLQYsjjx63QWpWs/YjzazoWEZOyqXsn9mNDtC+mmJ2MzYGw+w+4Xn+GHuHf6cfIiJIVuxyWfzavoZ2k0KXGt3cFVzlJCWhTygdKe+OQZRZS29dTKSqn8jvvY7SvzW8bh7O6+EbmNsuYIynZQx7v9pfWEFjALoHR6mM0aHesFrvP/EP1xmLo6CgmS1EZtVzZkBMFhhS7YnOe4a2qLn4CfNZqgCvN3eZvW0Dp46ms+yZ+xymZtzVzIjPIcJMhsyUyXPTHiKny5+Ss1JZ3Kj6+mX3UvsWSWuFIx5Gt+YHqzRT9GpghETrKivRLD0z3EUFBchWuRn938CfBodWS4LplQcyx+KW1hlArWxbVyQnOMR8UxuGCGkCtIuv0XcjeMIDilcGH3hrjeYPVqcD/KGjwoXGWzy+E8fIPWVxeybBfXl/mwr8Ke+NJCkwiT+WSUmUQE5gXCw802alTbGfT2SvYIjt1034u0CASU3oD+Dfs0rvB+Wj8uDdRxdWMSLtxZwowDSh5xib+tGRt1WMNCViTUgmApj1qCEKECjXyMGxzqkzhoST/9Br2Mn16bOxXx7GDqzM2vuDEV/spPG+5wgGkRImBW5kUeHPUu3yZHcg2NIanNH2XeZtNxLeE1Yy1b9BhQieNW/He4/R+LJ+5m+Ox2ZJp7A0VI8/PcxIfQ3anrDyE2P5md9KKMPhHKsaw1RG77FR3ELQqE2pp6bQTdYcH4pT//4GJp5J5GZe8B5CKNif2Vzv41FDmk88ssjZCdnc3D2QU7o4ZpOQ032dNp7nREcbNQFNODkZZ9XvST2dfj111dh63Dl2oxr2KydpIpX4Cuxq8asbjdwc94ezA4B7G4VMUKmZbr3as5XT0SMlJt8x1t7vqDXwZHqsAqK1dsH21PAxtWW0xQE7sGvaSgdf/dwoXs8/qkdXK+fg+PwIfgpnHHTRWPR3CY+YjqjnQq5OgC5RtjUDctlNkBAK4J9MZcHry0qh1VOXVjUnsTqthOtPkC0cAVzcjw+IU2ctPpQGFdIs4uYFRt/J0opcFa1iS6RlclOWVQcXIrljA736al8vW7G4HWvDcBHWR8SeWcStoJimmZ5sFZo5sWIU3wdWM+TraBWxyLqr6O//jYi70hyp6cy8a6U/heTU4Fm8qt8aLs8lL6h/gzzv8H9oyr5Im491qupzNzZRPukegQfLdpqLW98YDcIqP5iPbWJXrQHuxEpM/Dd3e3vo67Xebkd1rbIUPYoUVmkCPUCMcUxmBwklEWV8WpUGUoRTPw1E41nHl6j7P36hiIegUI6BSlWqYYH/ngAr1YvLtsuYzPbeEE3m2nRxXzjCR91S/kkMps5UcE4Bzoj8R7LgNpeaDDKmulr1ODY1kxPbS9Bc5JYV3WFtFO9gDOxxbGIVcA8OPvG2cH21KUm4Jh8FI0IBMGGxWCh/4urpPmmcWO43dJDbBUzLu8WpNnI+yOP5FMh1E28ihg7+PaWAW6V+ZBQEkXDfXH8PnIKOkUfQcVixIIUg9zEgNMAYpkYmbMG3b8BAQF0Pp70qvxxjZViGGtBL0zmEw0Q/RYH2qKxSv7ELLNiNVmx6oyIrHISPM/y2qiFfHp5G2AHHYnUE3CzTkDUbWNC6HwEARJ/OYlkZjZrPlpEn6qAhJqfoG883QZvchwc6E+9QWr0HIaV7sQmMlHm8y55KV/iaoFRx0VkimQ8bDMN+iH+LoPzVnh6ejAbllRzW7+Ffz55C6nJnj9VSnUcLl/DxJFzUfaZsPUY/4N88a8wHtQTnnuTPPfhCOUtVLh+g0/zH2x0G6D+cCZjTg2jYUwD48+OoTCukJKYEuL9drI25SXKUyGmSslLBRtY/vdy/rj/D9yDFiIWZFxqmMEBzQ7aLL48/NdoQr2UJIwuIbz5VQI6ViL2+gOJ/y806R7izffWUx0u4b4X38VTY+aBVpjXcZ7uyggGXI/QPaKZS5YoGiX1diKA20V0/ZeYG3SYfY03WfDBo7hFgfO4s3C32BciruHRZw/wQsMwwl9fxc9+A9Q/e09qtLLKhvcQMQ3+Daz5Zg0d3q48kvYVugioTJ5Mu0ckhpx84nRxFMUXkWOE0WXhrOnW0LwzE8U8PfX+9XQ7d9PmdIqvuuGrbpB5tWH2K+X+a6lMOjWJnUtLuR2xjjN+0GOr5c+Nq2nOV9O7NpYCQwGBbWEoJXZWd5uum1kX1qPJ6iXR6zJxx2awNPxvODvZrmbmPYEPKkLwyo+iMfw8R2I3s1XthQQLz56IJfmPhRRHFuMnpA8+55jACRC0kPzfz3Purb94LPcxGoY2kMsVdhlzGesNW433gEP/Yu/1ekZglciQWM1E3viLhsixwJj/YPeVDA3nj0WPYRPbsElseHcqofN/N7NPLBbj5ub2f37j/0fx32Lff+N/JMaEZDCqMBeD3sLNtBROekNi0Fa+jl9PjzwMuU5OQ6sSoyBHiZEJanDWa0g/OJ2mCAnG7Hjq5QLDp9gX+0db4H7Xm7w7/hPuz3qL7pff5pFAC9+0GCjSwynHj/nn8lhK967G+4FdzFHpSHZOxyBcxKMSZrkuYKn7KvbeWkXK5Aq2LZ2PxKpBW/cZnHsdJl/lV1kzR88txLwaNj2+lV7HVnwkMMfZhs67FYmLFjPQ5RtLS4AXpdHrANjWL+KyzIxVaWP2gdlUhVRhyBTzhBM85Z3FzDYR1VYbmgYbMw/MYcGanVS62lGGw81aXj1tp013J/6BSyXsiZjHM857eaQFEjQ/c874HBrBioP8exzNImz8yRpneNDxDv2CieSb8Qy7sRhhdR+PDVtPotMZOPUlRD2HUTqSbeoOHKVuhN1F+1VYXGlyHMZ8UQvXO2SYdCa0Q7V0yu9BM3oEMxOq5mEQm/kwNIxdmTtp8GsYfN1oCGKj+0bkVhfciMQoMSIWSygwwZJmGN/5LlEHIkiOdyXkyRGMdQFdt5aQ2CokkjR+LnIjZlMc4ZlWrKK3CPJy4pTkBaLunt9/ChjJL36H8FEKTK24n0n1BhaoH4abreA5FpFNhFO3E1IN5KSV0iUtGby3hbsW4th6AsmUBCqT5lHre4P8RDu6fcAiojQnCqO2E5FKgkiQIBHdLZCJbKji/kYbEUqPV+y/1J/+I3679CWGGwU4x9SzqGYRVaP60RqiiK7/GKXFi4aSP+g6OJprjx5l24LrPOD0MyerjzBdtJdnIpKZaz1KTc5sVM4+NEn+QmqWUoyFDCVIbj7ClzPyWDYsj07JMkKvfcMwyzCyU7Jp8m3i85d+5pUnP2Dr1yf4K+6V/7gvm8yPfld/ZHIFL2x4gZLoEg7PPAyAd99oXrnkRuedAK6uFNAE/IZngF1je7rGLoORsPYzTvd6cmIgggaJ/cAUhCMTfU9TYRrBjU8fRKytJ3t4ES2elRQHbOOUHgZsMCJ6KI+Ev4kuoJLvxI8P3tN79voUXa0u/PnuUiQpN1BX6ZlRMWPw3vIT8wl0T0TuoOT86OOMPTeWm6k36bDBXh08s8yFPtEiWk/l89CWeWxf1ESLdwuTAw5jASyvrac5po+KxfadiI4WYp0qWeYAexs8qK73INjJEcv1IBqcTHgHF/CkE8zQiPiq8QAXTDBEcodpR8exf3Yv7R525KYgwA+2JESjLuAfPgHNUV/SatM4O85+SJGIxAwJOMLtEBfkHo5UJowiO9wu4dvm0UZBfAE+XaNxsUQDEro85ByceZAWrxYafRvZunIrS0LeZt5H4wisWopZYaD1o4+5Ya7F6iLFdVgRlVo3Hl2ZyRinVta1xNAOPO2xnca6bLKrUtFdVPLq/lfZszgPk+NwgvV/E2RJ4fHgKupFUgY0Sjpd7WM62wiPtcLmmGgMDh4YHDzQeIGpuhGpopY/++zvifcMY+lPb+PWA+7P+FA09kk6XOwFrr+W/8SEvHo0A1B+pJy83/LwiHahz9HEHtsA4kgvVAMTMbT1MWFfK9rUDK6MvMJMu+oDh/uh1SImYfRCsiRGVoVqUObCtQDQCkYau4fhYR1NqebKoN9CiwW+74ZisxIXsQ2xDTTdLTj12D1lHon5HrGg5JroY/QaPTIGcGoehsYCLh4CLLUfctqLv0QW1EWLsJiEvBD6tSEM90tDVGYk2n8bxR2+JA0kMObCGHLTy2idoCdCBmKLhBFbHkAXpCF/jIwNXXbPJLAnG7zU53nVdRT7mzbzYlARLwK1vpuZaXqYzX02chwK6EixUe3VzaMt4C9ORCavQgiBfCN8HRhJqzKSeslHvN1hl1AGmKIBhagKpeEQrj2lqG7LmGOZg++cO3DB7mXCMoHfdHJ6xc3oKz7lvrNpnH5yL2TKEdOP0eEa7eY5BE98n7MBOejV6wkSPBDlvoxGHMmf3bBc0U/H8DxMUjNRWn+C2h8DwNT/G1HFSXRN6aHI8Uk0sgeZMxDDBHUramS8se0p1AMqJrbp2Fz7JPG/NTHKsI1sowENAeyeUwLI6XZZwMtRdnZ1iQmq9M6IJWIU1eWE3TqBslNMU/VSOt/9Cb+wRhAE1DYdwxdXUtqaTuzZFowNK8l5/leECPjmVDzFx+JoXdaKSLi3vR2ngilqiGu6Q3lbK/Sv5OlNTzNkxQG+cbTx062vUbueZnpeD/1WGQ9+tBkvVRmqOzO52RyFstcBf4MG49cPETChkbYBKVaLlDnuJTwZE4qToy9D3L8hbmgNdXxIdIP98Fjq9yYZjq249GVgcJ/LY6/dQhA7En48lRudTkQk/U2C+xEQZSIVxbP7y3hcEmwsrxtArw7nl0d+GXwGq81K3+16hh3ZDED5oh380ANOxps8X3IKs3c6KzuX4Vw9nMyfqphmgMw6aHdwoW9ZOyluQwk+bsQkNzHVayXHWn4FoC5oBOWLekh2n09VYBDTAjSYWuzzXI/bLqqCYJcO9h2IQmoKRJwoxgr0C2BDwqPFm9FfaiQsyovv2n/k4sDXvDUuCzqzwcuJj8fmsikpl5IzMwi11uPh2MQrmUtoqfXkh+1P0nJ5LDcSznBTbUecRuDMPt9uyvr9uGm6j0lKBX0mBRcbVhPrap9fNyszMe13p/xWFKYXtUyPOUeINgnaIFwGcg28Oa+Id1pcadmVR5IphOqES/SY4KU2aNFncM62F8Eqp2X86cG9yzIHWOUEzUo/Ru+Px6c0guk1jogtj9O1+Sq4poJDOE5CH6s+/5qR2Y/Qoqria1UhkjYNUkFCt5sjjRFxJHmo6XPoQ6/Wk3rlJfY/8yEGA7x0s5LQUOi98jppZ+TIU3/CXxLNG64ldGo6cQ32ZU/Tr1T8Ooat5rE89NZWRvsf5qOLu0h0+IbYbePQhbgScjEW1aRrzBlln3/rg+HoDRsl3z6Ct2sT4zpXsnXlVhqCa5ilAYtFT7epnfizP9Dl6ULBmjxag4A9dgYqU7MYLr7KSL2YPQ0G0g/cz7wpeeB+nIF+JdcOJaN060Nwt298LICbqpmbd2YQo9lJ+45ROCWIGOuygRTlrsF++4k7PHjqJbraZuIZsZcvNq/CklpN8rhivCTgWgnVoSZuGeGrnlm0aQyMda1hrMurbHP4ju7+JcyvjkR2YyhWdRYDlXJkkweIkMMeX6gzlrDsYhoTt0xEFWZHEkdIbQS3nIaiDyFuHZ+2OhM3IEEjiue3vlpmarK5PAClcdkcCb7FqxHfkvx5MoJIICc5h4yBSdwfMITvJ+zGpvNiZl0qK757krz7j/FBrCe9tgP8exSPbmX36D086PwRVf29xEqhzKhA0qugQyPi7+Bqtqk7WMUUmq3wWRfEq1zYse5DPDpH4t0WjMrvVyYxjNmRXxPrcRFYSUR5OGPOj8GogNneZuRWHXiNg7BVINrCxVEX8XWKwWvgN9x0MMRBwT9eJtZ1eNAmlyKYbbh0umBQGhhQD+AnhWApPLZqCy6i4Whan2GrvoKVjpMoDrL7/r7iZ6Rn3Q8saniQnIF72ZkrA5CmBJsgossG/c5QEz+d6Oh7e2uFCEwKT7qVEFAWy0BZGx5zSwCBH3tghCScySJIPLmBbs8Ivnt4K++523As+RhS7WzQx9q6qZDuZVjjfKryp9K74DKONx7F0dWR2vEXiAobjrpdSmBvIHUBdQzY5GzK+pYliWvZFFXLw7rp/Dv7f5EW3nKFn5EhlygJ7B/C7T938OcoPzpjx3DDaOHVjlbmJdyPyGbF50YNB4pTSRqdy06d3bdP0RdP3OkrOCKiNLyJca52JtcYhx7KbZHAXaANYJZZ6Nf0k2904YMLe+k1ufHkMjlZRhDbRAzJKySt2YX0pE/xVuTw5bl9aE4dpHLWLOLOXkHnGoDUtpEm3S02DyQRc1DPuFIR/Z+Kib4dTYtXC12uXaxosQMjVkYEI6gcyLZpUGkHUDFApEzEWX/YVJTK3h8fJyLIm6CzkWTHFfJzwo84u4G4HL7Rt2LaNgZr2NdcrFnMqKAdrE96n1s6F3pxpF8Rytqz39BZ405Snp5u5/b/KMze7DrGseFP4SYNpK7TbmV3oOYEEU1v4trbg67OlXhhBJ5dEzg04xB1AXWUDO/HUdpP7MmX0fsvpkC7mpt+cFJ0jZ8FqDFJmYoEtbkDwaDn4uiLg9/3vptd5nP9gXRc9mQS/l4of/f9Sl2AnT6/XwcFRmi+FoHk2BhaHy7hRNwJBPFyuqx2cJbeyZv3I8/RmqHG4DCaB5qDmRPVjEp3BW5fQRrz6qDUvkhlVx1QSPsZqf8GL/9WDgypwzPeE0ukF02yxcTXx5EdvnDwHgVs2GzQ4t2OyLGdqabXqPGMwWRRkancwtGuBbR1DvD6+69TPjePR+fZpYcb9D5svxqFWu+IcUQg/aEjGB+pZXf+BjL/XYYOgbbw+Tw38iHi2Myj+fnsLXmBifd/yqXZf5JbcT/jLgVjdLhrCSDYIPc13nfqYTdgs1lxtMpYoRXhVvIyOIUxb2g1n3a8RXhEMpcS+rkd8+fgt/WKdWgOTcett4WCt+8wa8Kf+MrjKWlwojC0hz974Vf/GhycdVSHNaCTVTFDHsRvepCWg9QqwpiYT0CrB2+++zapk24QFvoF68d+QV1vFLvKtuB4o4TqoXUUB6wf/N6fPaFd/Cqydi+er2wgq03AZHRFYevnTPUS9pe8xup00PRlk9xaQKUNtP8MJ2tZN1cyrrKsy5fsw8lIUnpQmgf4MG8BXUvP4OLZxY92sQm2PLyFDMkzhP1ZQF55BHPmQO75oZSU+7PviTaKF/1CkHUikzsNaKw+HIzcSd2iUwx4tmFsmUS3bzAexAze85wqHw5YmsjoGkWh4S3S2hNI2ZeMQ9cEtL/brSCKTJCGiJK+d0m5lkh5cimXZr+D7O55uE0fiAvwuX8QxidvMUQZyZqoPQAEy8HLVUehoxcSlZwL025wJ+ogj//4OJ1xt9nqBY8lv8hunZHZ7dvxPDCB8qgaIpLucHUALhnMfHl0Kb3VPmSsn4hmh4bbGSupCLDbgRgEiLo6HJOXCcmdEJ7pNuPxgztiCulru46y+Tijpw3QK1Kw+LXHUToreezdOmLVYsDGa8HXWVdu90aHewWCgkDI7m9ku/xFYksiOXyumoTk+6nw/hRpsgs9Tj1MOzoNoUgMb8Kwx4dRMBBGgSqNyaG/8PTw65gefhVTQDXWp6yQ34y32Huw3WPKZzPqwD4sln5KsutIuhzBycxQ9MpKZjba1zXXxfsoH55AmllNr7YXG1ZEiBEJUgoSCrBNsPH1kK+pOtGDQ48aozsEtzxDtddGDoWHEFP3KYLwElbBQpsV3uyQ8p73BMTtjRyZcQQv/QBPHy3F9tFOnIZOosolki9uLKPn91Iu+F8At9GDDSKIRCzcoaPZ6xsq/d5m8uWP0fR0IzL0YxUbUEj6kUmMmJVWjsw4Tkr8UtzOVCO2WWl1TAOgMx0mWhqxnlvJ6+J8+iQivLycSAlzYEtyFMuT7RLUL3XWE7p0JOdrz6A0HyVeoWZ+zAb6e4LwuFWPpr6UFlEsyxqXsXPRTt6Y/AZhLmEMnzKKgw3JaLrq4dQOHGZUkCCzkyS+H9LO5y0dqFzV7FhWQqOHfe9+0tTLWiBCbp8HrdFK9s3ZR6tnK27YWJP6CGpZLzu++xhnXT0anxakMgsqSxlRlslMDT5HhFchmpooPraq6fWIQiYLwepTCL21RMrBo6UU32tnKR0/wF9TfsC7ax79mtN3iQBWyhSjiDrhjXIgCOeEO4wNrwMherCvNAiBrGiG2n5X4qNSEDvI+Ln3Yxx7NMzdN5fzx9oZvs0LjU6D3CRHLMgZiFjP/tb32TR/F99PeZjIH1zwv72Qovgi+gVwyi7nY8VyRpgFZMCuRff2v/+K77z0GMWQnR7FCXknWmEBsBelCBpt8KdrCQFhepIH/sK1GiIiBb40gKoC5mlH0tF7ivSckRiXWCme34NVHMu7Hl6g9gOpml1BPbS//S6pupV4tDrS5CSn0S2Ro9JO0sOb0HprcbhRxv3V97P9vu1Ud1fRc/s7irVnMT0fhaO/IyeXnSTZXEFVUQiHt87AuNoKd3EM/17ME/+bDVePZwQe//acqt4WYm+XUqf1pv6uDYBNsPG/uM4HgNVqpby8nIiICCQSyf/5A/8vx3+Lff+N/5FwUDjgOJCASGdHnX7eBQ86wrPPreeX0t3oRO8w3WkeSomdjv+FB+SaZOzPi8PDuYW/ns2mJOUQAGlVam5a9Iyq8ePS9lVoJ5h4fkQrgTJ42gJSEXzcBYdtU1EF9DH1+GS8f11CwVcCcnMpJlkrIpEYm96Ae00RQo0FHOyHN4s2HoLvB4UHsnAFjo9Xokm7n46q57FIBqgOBE+pkTXP/Iqj10/07ssh+lI2d5Ino7nrLVPSJ+dsgyujI2DfvHMYVEaSRd4oReAgNTLV4ERZmwO9vjYWv7QN/6Bm/C1SjCo937nXEHX6OeRGOTWeGiZJQ0mPuA7YizBWVOxZYN8Izsj6FRAhYOPbbjjUD1vFct726SbfwUj91Xz2l43A95UCPHw74fZniPwzURgUyE1S1LkWUhpSEGeKcTI14VIQy/Lf02iZVIfrN65c3G8/dD3uBNN9TvJQrRy9WYLY40uKknpoc7cbtzb7BNM8IGdslTMqowuOEhU2qxdS33vTjdQswqv6JjrnBI7p7ejAMeJgxEoj6S41nHeGhgmn8NTNZdkOC6FzxJwKudd/RGIxN4bfwMXSR9rW47gYmlCtbIW4daAOYDz5JHz1PPrULm5N+bdVCigYUoBH3yzEchn9zn7E36igV5LM+pm3uNUr5twP6bQm72HFsLeZ71RKZ4iZlf32ROBMv09IVRtYcOcN3nrnLYriiti9cDfPOsN0NXyS8zijq4fTorIRdzsOU3I3R2JS+NS7B3X9K7wmqcVj5kH6NXZmU7O1mGYXu9G0CDFf5eYj2vYjmhv7ebFiLbsW7KJwSCE/ewEdN3iuDU7rYY7ZitwkR2K9t2jYRCasVuil/v8x3poCFaiUy5kT3odeq0QqiiG2bgJvpq7lvogLnCiZRE5XDwOObnznUMXhHPsm93EnWO4Az7tAl9lIrh2kh4csiAR5DW9FvMjbbaM4M8aAgziUK0OyMCjsB+/jevvfCYUn1RaQCyL6y/sYemso5RHlPNCi40dPu0S80axEZBNwKBYxtHjoYLEPwIYVQYDcpFwmnp6IwqjAVwJdVthV9RahoeBrvIbSoCWy4U1cLL0kiNexTh1A0ugcSp3msOTS6+haDIgeF4iT25/HcHUBpr9smBfGs+ftZVyQ2TijtydQcuvnobXX27F4+rJn2QEave3eNAWBYsqdJyFXvsqR8FIyEtLJq4nm98X3E+hg4AkXiBV1MzbtRU75j8YSu5oBtyAie/KpREtpdCml0aWMc3qGjpuVpBz6G4Bvn6qh26UbAG/dswS7J5EXeITAKm9Co2t53wvwKuKprFe5E/ACi4Z0Ieu0S2/E2Z6F7mIiI36ieKqBsvQWrG2zqE3OxSLzpM3xOFKbluiex3hq+XA298BvteNpUp75j36iFPrYuUjFkZqFmCxbsG75mSmhU9i2bBsn9HAgwBkHryi0QRJsNjBJ27keMnfw89WeG4lp+ISx68eS91sejm2VOPYocOpxxfeh8xy4/go6UwNmSeMg2nttO9w2gfvWJQTbpLBEgVmlwElcwFFfuwTGxip/aj9fgTq5iwPzzmCQ25PzVRZ4qg2echKIff1nvi16Gu+WlZwb8joAX+S8h9rsz4PjdbzctZD8pjTCz2+m1z0EkXj8INRsbs/rNB14kNthRynNWIMgEhPjpeVkfB4fPPsM6aV1iNKH8u1T36JRBhMqcWCPD0SL4dMODyrdzERf8SC7/j5eemssNQVHOaraz/q2QHJ+T0WSWQlB9jYqVY9keb2dvdTs3cvF2efxrZuKx4k5PDbuCgMedkk/wRDIo+kOeMt13LctnZL9i/ls/Rg23voJB6c8kuRTWKk+wLotf/DId0sYenUoqYty7F+S/IV9DHZFUtdcRrDCRGFcIRaVFEb8inepivd/XU1VjAkhSYJVZmciD6iUMPk6uz/+kJnnIpGn6vk26DZJYoixdjE8ZSXL/a7hJ+7l1MIT7OkfwWzNWjTyGFoMXvipWvmlJYihuYkodW30zypmdFwf421nQA5pCnA0W1C329Hb19V+7G6AjGiIlsNIjf3ZHdJ+RhnYwS2TKzeG3iLCyYdljk6gqwCbjMWjN7KrWEVlhRjXTlc+v3vayBArqO2TY+wzkXh2J7b2iZyadIpZGvuYf+T7dHxKIzC9JkY79QKtPs2cM23gXMAGxpsz2P1qEV+1O/FswxCi1Hqu1E+iLXILA4og8sV9RM08gs03gcpwR74ebZ+jhgwvYq5bDt1WPd2F4RT07SLlag/iTF8umoLY5NmK1KmB272VKHJ2oTDJqR27mKdm2P2C6SuD3lLcvd/gZFIhmgBXWoX36NPUDY4r/zp/JMoehKB7a1lYRRjFccVsCdcR8NJZLiifQ3u6CXefdkKEYnpscNkAO53cmRx2gg9qP+bTF+1ev2Nsi/jcHdY4iXm3ayklJl/UKgl1WiNmiX1tKgmyS69mnUpBdzWeYZ0uyMxiJGIJ1wwwtBa+H7qMBH0w1kvfY+quRud5izy9IxT9At35ML+ZKZX+VIjr8Ui+ypqZW9DaVcZx8exCNd8HvxmJdFz5evC5zAZ/vuvupq9jCV19nphMBjKvrifIkgd3iRzz/S6xxX00tsBGVJYmHgx/EbWgR4iAjzvtCfoP3eC2QY5/xXlCdM3sbrODVDZ0wzCbN5KaYMQiNRdHnx/87o+7YF9TKj/NckRZ70NsXiJFCXp6nHPYYLkCl69AwHwawj7h/h9P4W4YTrPKft1hFTtxHEjC4thDRbwDPgEuFGUuoUt7lWBdBs6Nb2EVeQBPIwjQHWbBeeFpHgos4rO6YKKrYWLzywTt6cfqqGdArKLQbKZWMZpA4wV6jB7c7P0AW98V1LJg0h97l8Qg+56r2woH++Sc0ijQzjxBfdcK1P0qrO5SLIDsDriYAvnbeT7xk5RstnhiEqDDCl7/2pYpfUiwNXHt65Uk+upwbJdxu6WGIYChX4X0wGQ0mX1YQ3rtjIDgOj4N02F77ABZRxyoyA9D5Weio6CGA91TeWKLlaaeEgYE6DbvRFt9Db1Sjr7NhdmSEkY7NpNlAJvJiztF4+moDmGMVxrrJw4BwQ7mOR5Wx1ftW3n4QSN/T72MrOhhMgJPYpB4oLS2kW+Q4IyENrWV4phi+r1modT8iJPLfuAd6LZL2rZEJVMW0smI1sWcPf4cX0wbx1WvfG6b4IhewGYTyLiSgVViJSc5h1XWKTzs8iJPTIBldd1YLTLGzD+Lf3g9sdKZjHc8QKzIjdt/j6TnZjw7XyoBGXiIu8gNsjPVtzr/zqonBHytGSAI7PSGAMvvvH63uX2lRlw0NkosjeR5XMTJFswk4Rv+yH8frbyL9x+04ZiRy/nICqTN53D2i2Ja+kokQXaflEz1NVbNvcm3eg3Sc/Zr3jHBKT10v7KJ45UrSG/t59mNz3Ix8yKnJ55mYROkKGBBfAPn2nqIPRjPtPxHmTjiCD5S+L3XDuqZoQaLALMPzGShVzu7HrzEHTOMrIdRfU5Ma4ynXdNI3tAa1kfaPZxH1ME1g5X1olIERQ+66EDWPrMBtYMe15pMwC6N3dn2MoFB7fR45YDIxjwNKJuOD45Bj1pHpu57CjEq+tu1vJNm4fPxk9hwYgd7hxp5N9qf0A8VjMxbxfuvv8+hBH/IEjhrOEroQV8cRhiYXTCWO2HFZKVmIQYkIrg+ZS3Xx8DL1aeIzS5GERLAuFQLVRaQ2RRonPsxWjRcDW4lacQpjoiXcKnSgTqPX0gwuKFNqsRqVXJowkkW+NmrBal+h/HSViGy3uCzAS9k5TP4xN0Vt1e+Y1hnJtey59597hrGX8xEblDjbjMTOeDAcMHeN9WjH+CUYjyPjlyDRZ6LTarAJG3nslBLUWsE7476gmzH0Ziardz3z30cnn6Ym2l2lr+HBDbPDuF49RrWNroQ3uLF6klzqRb28WQr6DpDCKtsp0fiycGZB/koqgtnwb53Cm96md9sRYzQKXBrNXPxr+GELb2Ib0gTydouRu87wwdj9DRcvkjs7VjK3v2cff56BlrPgqvdTsLTmk21dzuftUHlrgW4x1by1cFp5IWAYuZRVo2eSu+tAmbsD2T7kiu0eLfwbsE6ngvahfF3NerUKtwcozjalMgDr/7BG92z8ZCamTHhRWauOM/zzVvgLsnm4dD1NHSsJ8cIWTIIiihH7ubI+bH35u/Pu+3/ThF9TuwqPaJxpZTcKSG6XkKNBZY3w3MuQxjnfpCQlfspKs8k7IeVDHv9V3oynsPJSYRj503+9oYvu6DmcBFpN4+x/MnNhBrLObb/VcL0ysFastYQjdZwL5E8JD+BCWcnofdo5OCcs8x3a0AORLhlEeGWxYYrf1EbHYtOU0JXSBUOLna2zf6S5/j4zkyyH5qGgyGW+b2f0SyNQOmWS5wcDvuBvk/NnxuWEubSR6ffEBLTbyFGoGPAj44Bf8q0Q/m8+0/8EJAPfY+ivkY6HAZw6/sLqn5nZrsD9RfSuKZRcGztN7wfCAOqOSBz5LMr+/DvnIaLBipiPbgTYT/r/OwJqUr4NDwTidmAp+QDVjgCFDJ5YBnfdP9NRftwRB5/IAuXgMguKuilDMajZxptTkdZLEkmSSLhQu/jOMdm4xXcTJ4lmERpNRJ1KbfqPie8LoEuhxoeyXqE41OOUxtUy3IHuMBNpMIw6s8Nx8VbT/JnHzBcCwW5s7D/CCKcOt5FfCMZRSRkpV9lr7KJYSbobHEh4EYK7aF6Qs1D6LzRw8DsK/QYtFwa0HH1riLUp63d/J55HVLbeQtIm3yD7wNKsbnmM1UJVX1mPJq8Qe5Aim0lpT0j+Nyq4IvRh+mMDOTgmTheefcPVj6+lxmOp7nQDSpZL48sP0Hr0gOcPpbM2xYPGiQhLK7NZZoalNQxNTALz+eKealsMWnX03grvpyRHbspaB3HwachveN3enuhPv45Ng00MlHiykl9J64Rt7k6KZQHwjzRlbfT7NPMX+uv0y05iqIumG88GgixqPCweHHpwCiahsyk3KmGS5avAAgYXkBTeA2Bk1axc783Ro07AM862xUyTjy6BS+PKB7Pe5GBzgFcfF1wFkNREND0E0yE/sx9jDs+FZFIxPV0O+C+2QLuEhGffPsMXfEDXG6+jP+vJwlYGUCffx0ifQQvTTmN96hz5ER+SXtOHanXY8kadp7cobnEFMeQGTwJgMgZkZjOQnXnJgSPvZgFqI8pRes2AYWDAsM3kzjQvn5w3EW0fUdbYDa+3l7M+CCBksBJpN5ZxPn4eJzEUB4EThJ4usEDfd8Arm32AqNIsAO6gUHPvutrd3B//Vy+ev5Ltj/7HD+W9NH4cSMJBXoqhhai/PE8ysl+fE4v71n6CTD/y0NVoMuxi6xhWRTF7+I++fssCD3IJpf7EKvFdhbfiY/pdXGgeORcMPvTbVFDgxtWVxFTjiYhNyZydbqEZUPWMz9mAyabiCIT7BEEfO5cQmoeoN3/SV4NXkiqspXlleeQzw5ipHYubqc8CJbW4z+iBZ3JjjitKgrB83IUrUOvkR/yMC59Iwm4c5xNN78jOVREduJ5XLx88e/qI6QqBA9pIG+NeYuzVWf5u+N3qkPS8GlNxDZzFo2+h1ncZLf7iE4p4ZpXMK7hrrR76gaZ1/V6L+bsL6TSeQqxnW7YZkrJHZprbx6zgKemBkdlGyKlBJugRbE8AWdxNe3qpbipG7gv/j2OlCRw88v5KMbqqUi1MyKvqyYT2fYPtwKhpP0l9hvnITV6ctkfNL5n+bU5iida4cwA/OKvYMTRKfg0+/L9y3/ztbiGzcovBvuKXuKP7rclOHl4svT+H0n338fGOwISqwTfRn9sGhnaBi0vbXgJgDqFgCX4CZ7e/hc9mk4sVhul6R1cGWJXWpEAKmkvbQF9fP/U3zzq8yPcFRJb5Wg/h3zZ6klYgQ+FZ4bhuzicdtt40AHCA2T8K1036hIim4wXzjxDYG0bS96YyZd3hYIEAQJiqzj5QjaTov7GfOAxCu8AsxZC9rMQtIR/eXwG5GiZcfpxxr75B7LwYJ7xzuDS9NUEjgxE1JlPQJ3dZ10rd8Ap5wme8IIgsxMfuH3ACJmeP9yhukugQrDhIHfn/x5B+QcxK7RUDFuCQeuGVapAEO4VA+UDPQRUNLK6YjWHZl9k+JVouoZ30Pu/vdoHGAyG//Ob/j+K/xb7/hv/8yGI+aPPxsF+yOx6FLGklaEaN368sZmfZtlRk7/1QlZ/AtpH4/CPeYSo076Uhw4nyv0G9YIeTwkcs/ri7KynXdHIO7VqPvG18bG7AYkI5mthh+4VSkxg1j/KsJAaVjtdpbhsMb5B3xIqbkHdto7gAif0NgnTNNMojK/A7Dkb4uxMib/rsyiphxVjLWj6VOjVBk4PmFmghf39kGazIRiMaLsbSDqzFZl5EtsfP8kH9R5YfllFxxIZIQOXYAAk4pf5shu+7O7mWN5Erh3K5PIzJpK0d5jSO4m3tI2ciM+jQAdjQspp06uZdDkA18qh+Iz5gH06aLGCv6ie11zsLJKEU1/R4xmCZKGEm0a4aYStYilCohbXsEbyrvkgWHtZ0TiOI499CpYBvHOO8eU/D9LY5YDBx8LkO5MpGFtBjzaG9ZqjqEaUM8/flccP32NkxchhqgYWH5yNV348LcNraXTZMfh6QetYHNQVfHFqFjUVftT79uHRkE5PioQYOaxxAleHIwS9JeF4XwbWfZO449vO86Mr+PLaQjKmXsM/9EOOV5uRmrR4ar8j1cVelBirggEBHIQOxqigqt+CcW4wPdV3pYqMHSDTMiBREDj5CjfdA5myLx29PHJQClGl/gFVcgqCADKjDvd2Je7t7qQrQW+28sns/Xh6ZBCYdYMdOUt45eePiJLbWTVfNUn4KjsIowJa/Txo9G0E7Lrdw5UQk1ZMRdJFrIbd1IVNZmySwKGen1nmINDkfYYmeSu1AXcIrxmOa2M04hQxAVJ7m6i5Q4ezE5VD5zNlko2/zq6jw62DT/6N+Z1d6UlgRSC2IfDNM98M/r/CoCCkMpCOsg5E125xX+V97Fq4C4vMvoGt9N6AWdqFQXiaupT3AZBYfyLbAI5WD2amnkRIhVE8zTQ1WNWdCK1PsCliEwd6FUzYEYfV5oSHuxKNIZLR49dwqPkZFjSBOw5ETrhIeZ8IA/cS1P8KKTYm5CfhWhVGX0cHcw7MYctDWzjgoCO2G1a4yfjxufdJs37M0tBn+PH4PTmthLwEQgryGOgawoBqgC0PbaHPoY9/vO2Fg4dPdyLuhrOjjvHXS+8Q2PoYTa7/YLbByS5f3IdnIKgn4/3LcaztOsRSD37pgFNV8FfYFYT5Bm4GLUVTeoIKt+3g8QPH9VDTlMa77Z+Seb2XEnUipbG37k1X2OgXNTNN3UOws4BEmozHNDjY1U+QAp51ASztjK9T0Wi9wBvGRoyxF7jON0gAjdiOoLTYrIiV8sHryk1yZmngWzcZn5UvJEyTQNaoMo6P28XOuyDG83po04Uht7gjNvdw5PHX+SyuhOi5y9jV+gnr29Zyx+dDoJEn4l7kHDW46IbTpf2DqVoT3sY7iMvtx2Tff8NaZShhjTO4mSvZdSGCsr1ueCXd4ua4Mmpccgff52a7xCcv3cex7i301vfSp1+Ce5sTrV6teDd541srAgk4BTrxTNOrvDn8BHRUMXtCLmEBpRy4DlJ3Fd8+/e3gNdus8IE7/GZUorSKadJdo1dcTU9fPzPV8Go7bG5Vs3qgGX2/YhA1+u8xWmVhsQN8K+vkUvQ9F4OylunIrM6EGb9myFklqmIDRlkLRlEndV2eeNIJmhB+0ifQ1a9AbLZh/leVFxCLxCiMClR6JTYRtHu0IxZ8+HP+fjrzP0RX9TMPffcZS8vvI3yfB/53vJksbCNjyHlO6uEpy0ksDxYTrNWz7rOXMCbcRjLLxAwNvOIK51VFjNHC0hvDkR8TURDkzDZzA9fEp6DycR6yHMSaF0BMh4ag2iDWGp7mqGkZZ015BEk9OGDxYkrVHzhnlDEj1ZFRTn4wZhP4TQdA0ynj0Z8f5eLom+xatIsk+QIATuoj6Ol0RKSzQXcXDp12fxixUgbuaSyWNlKXPYMWHyuz96whLakCU0YpjU6H6ZPATauYLKmZpqqjDBwXiF24iW/Ej7DH5Tlc+p35ZGkhZa0pHIrW8WX0vYLyE87wp+0m701z4GLNfUR/txyjZxu8/ysAe/wNtIkgLXI/Y5M72XjjAaxCHJPdTkJvO+S+SkfSj8y7+DJmixb9jCrmem+hyGhPcouTs9nq5MDbSe9hlNYMFpS/7IZtfdA68iKKpFsM83iK5qlnmKoBZaN9PF6UXSFBBxaRMy41Jm5YZKCpJMyzi3VOcvZ66/nY6Qap4kRGFjxJQ+8ujKouevZM4R+3XvIyKxiydzSSof34elcxfuhmDtV9yQ/CKRz0icQNuUPx+INY9UE4iV+nwjYCdw5C3W4Iug+xLIoO/0QkHnA6qogu7ZXBdpt1cBYuQi3azQHkTH6ZoSc+RRAJXAuA4UorhyRZZPXsITp9BBPCP6VKPRsRBUiApjYPzlyfCp6dcNfH53zbTiIc4XxXCGvjxtDu78UjtXMpDNyIqX0xYPe5qjbDwVpHUpo8OLDkJJ3OlawVf24fGzYF0Q6pNPscIHdiMtaePkacacUY8zXo+iH2VRDLqCkNIrZ4KP2ZYiKC7d+/6dq7vCk9jyn8O351GMuQy55kNNzPn/f/SYdNwpo2SM46zOxjElqixyJW9SC5K0v9R9773O5I46Y39KlFDLuVxZcfvcCKN7YSHFPDSy7wWgec/GQFMqmUf+ZvwfSv0/fd0IotzPjjAWrCWqgMvpcsLjdDZ78DIkGgesoZciecRd2+Gvz+DRSR9RRjQn9kRLYfKJQMvxPDTGc/bkydzht2i0i8uuailOzFs3cKOSHL2ZP+Dr52sjyxRSXITrmie9fKRq/rNLIUnbWUOjNYTX1kyvuRKuoRor6nrB/w2EZneRkSsZkWp1HUjRqJU0QhazQ6yoNauNq/jIzGv5FZtARqFFSknMWvPYwu7RWSPdOp76kCkY1g3SKsnCJq8Un0+xey6NPnmTTrOPkbNoI2BNS+bDZl0j7zMFbZeNTW14lN2Q7RqXzecZz8NT+jlX2ER2kDS3Ys4dbKv4ByxOUbGRYu4tSX71Nnfp5scRZyo8CImP1k/hNl77+WAqILQ7kWU8f2F39hQwRc6w9lZVslU0zjOHjwAZxKbmAbVsJvZ1aw+Ll/mKe1H4yNWJnQp6NNriPZz0qRegjLVVqO125lXYeV68PdcPHtIPv+q7gXPoJH31B8ZGfsZohRdjDMLNNDdHaKcD57GUlfFxWz3djYDYGNEcy9tJyB2Tp2LN6BILKvLf1OM2kLzEdV/TtqsQ2bs4LnUs6TGwSf8gYNFjBa+9EGdHC9u4ZZ9esZnvsyLz33Al53l/RDOikONgUKQcmjf8znolsP4a/qSJAf5RcvcJfcQlrtz4ouDRfdpIhEYhBBRdcw+wVsRvbFd3GmehJfNAVxsTaEtZ4tUPQBeI3HTdJFhgp+GbAQduYHTHI1h1PMaEXwmgscxIabix970yqoCbonaTlZDesdpTzUGE1dfDerYrMYIbuOqNz+utwop+r4FJxCnEgq9ifcVgNc4kFHeKpZympbL1X/LOTapFscG7mIwLvPm6KAbANENT1D6e4M9LPdcXCxM+BavTdywwfKjLvR8xDRL2/l5PXRPLT9OH9MCODDL/0H7+9xlxqqpH6cSawlN+4oEY6TIfZlHI+8zMhSCBgFJxLN5Pqcxia+J69RqPPAr0+DyCAQXRLKgLKHrNQs/tHBmYbJXBpyigqLjdMWV5a+/Dc+gZ9zszMDB9kVNnkbWeIdzhctKhw6i3gseDFpwj8cGOLOK73QYWxl2xPb2dwlZ2+zkkdaIdVrDlED+wl0KqbK1I6iNpjiA654zVHzkTsMuFyl2GEqv+Z+ikZQsLQ6muYOF3YsrSfl4W38rZnPULUCX/dt1A2twDXCja2rfkevqCCw7RHK/N5GKwIHx0zUzi5Y/VTsnr978MyxUAuxAxl8vjmIjjopC2SpSIrTSZjnSbvVDnhY6XaTuJciOd01hMWJ2Uy6yx767sYPPCRyZ19QGRuf3ciIG8uIOBJJWWMAviH2ubLT4QIi0WzKI8rp0+oxy0xcsVhJlLsOtrlRpGXTS08j9u5kSJ0XKd4GRC45nBmwgxdFItCE+1AXO5lWTztz83JzJobGpbilZKNKiKP/1gmqa1y4wGym2VL4fugbHOjy5Pk2KDPn88DvD9Do24hu7nped4W0OlD493I16iyZFxtY/9d6chNz2Tdv3+B9ySRDcQuFsSkdOBJHQcnoe/O3CFZWRdGlvcqGBBHnZx5mj6mXs2FPg5sY9aXNBJ7PYPr2SeTOvoTc1IpCZFeSGP/Kr7x8JZ1XdSakuQUE3L7G5bFm1MYQ9IoqIhrfQGFqQbAJCAhcM4Be5IZa6ODjSzv40OkNdO4VZPAil5f/TaQ8nWW72zFZVbg4HyXN0Uy9yAC9YBH3caZxH1Vm7IXbLj+8W5yQq60MiPv5+faPjE8oRiXtQyY2IQYmH5+MFGeIgbUj7set7yxEPAVD1vP1mfdQHndFmwHHjXYZy47M51FdGotHUzAR19ppqRhFg9vfg01lsarxRsovTyawr3w1O0q8WX1iAtFTHbnRHcqXChij8+aPSeM5Vf8s8h3j0PTJEH8lxkViIEoJS/rFVHydQHSQnPue/RuRREa/qJ917fBRF+wdcY1Z4/czes940s7FDQJYXSvBp2MqS8cn8/MT76PRTUBlsOdyVie9weqkN7hgvcYehRnzsz9zXh/PHb/jrHCAODnsDinhznufILfsxy1oHOcHxvJW0DuIxAKfutu9pQ/uHoWzQkVbSBe98go+sjyI4twAU2/F4TO8Gd9jCfzt7U7U3+NoTijjnTA/1MdGETUkn8iZe2jTgMHsj7pUh0PPDR6N6OFRN7juvgqV3IKjFeSZt5jiDzfF41GK4FVX2CY0ccbiizz0OuIiG9NPTqfCcztjw/6kvjca8GGG9/u040DhHjUNovt47uFqNrteI1QGn4XPAToJvuqKRj+SpkwTbVb4lWp8f5uB8UIq2sNOdMyZxcqhb1DtNJkJ2JV60sjBHAwtkc70BXliuVtMf9EZ/GWAZxcfGBpYsHUUHAjFlCNmQIB3OiAxfAFzR37EgFVB6e1SjAojfZEuOCi68JZCvcEBrbOOLo0VdYCasohSBlQDZNTDiJLPGZv0IC7u1fhrxNRdKmL8meHcjs5Cr9EjV73Ec6sXD/Y9mw2KA9ZS72igwQJnF+xlhGgRANf121CYvdAOxKI2hlEQ/g1FYTLuD5qI1luFyBVEDXblrcZQ+/XaGtxRVjsi6shnzbY1/Pbgb4AYpdmPqe1Pk97gRk9tD97TY9iZ8xsAscHuSEolGJQGDGow9ZsQdemQWqRIRFK48RgZZgVRJVH4t/ni9IITh2YdQm72YKDBmQp1H7lP/8zrj39O4VaoCdFT53ucWu87zO1fjsaxlpivHqMvvYeLKaVYxM3IhVmUtqdz7M6j9MhPUL49DUb0cHmMDqO0lvUpi5gp2w9W+CmjnPd0TZT1PUmCegjuilssjvt9sA1rSwMJOp9Oo9BBlG8U7b4yBMQ8kfoUZS2ZnE25hFvvOMZYPmDP/FV4ye2NdajsEF9c+4Iwp1dw043BlpRMl6WbnTpY07mZ7JtqZqYHYLMJuPRE0qWWYZabESGh3+xM5NXhJBQkIJ5zj/N1Q7aBt84JjA3+k43vzmFT0T4+MX1Ki6mSLwxmqrsTeOJwMbe0mzDM24Ov3zj0LfmIBAk2iYptOrvvozqoj743/6I9axplX9/PpEVn+TLsOnt1UFK/FLmtn6ZR16jr8Rv0JhWLxIP3McnzAa4UVGEOieJQ70EyRAK7fGAhXeRMfRmZ1YnAwB76U/VQbOOdMYdwOvYmLgPpjDk7g07nDtr99ZRI7L59UyT+7FjkxM52f3aZQYSVsWfH0uvYi9fEW3hJ4WPfVhaciyek2QWZwYLYYsImkSK6m4PRiOB1V7g+YEVaV457fTVWYRohA6lEud0kUCLik9AKTg0I3JSLQBDsDHvHGBAsMNBCnnclv90O5ZyrEfnMMxSppKT172dNLFgK7UXTzmlD+G7OLADudJUx4DwMVXc2krvWNt7/jCA7vAavFUP48t2vQCwbbLd/FfNENhtyQx8xl38he/obIBL9R7Gv1yNs8DMxxRF3Pwz/66l9/8Px32Lff+N/PESIELCzu8xXa1BILcjCR/KkywOD73m9A4JKP8BVl8m4SIEh1v2cy5tP4cI1HPV7kEQFpIl+5mZoESPylhH41stcefgsv0Rd4DkXgU4ruMkrCHA0cjqindUp+wBwN0Zx0FdMhS2XEsGJmS82sKFoOMOPDqfZp+s/NIZFVgtSk4m+/Eae/+pJDk8/zDL5TaRmKZmXMukbWoo0M51bveF4F37E9YRcFjTBSm8JvfMuMhDzIGTZr/Xvi0+2vzu9Yw14qofj2fYgeKVy/XAR/scXEfXYL7zzxBE+blORlT2K+8fYWX1ztfa/v3VPsrwiFs8OBQUeRdQEHiFGuO/ePYvgn4KztLaCkP41D8xdz1dqQPkHSORYxRdQJ5bQ0eVK34hULrT8yQxpAnH1/5AQBr+pitH4OzPm+Bjqveopii/i2TZ4pyGQk2mVnHUyUHzTlUe2PsCZ8Wcoiyrj6+tbcXQ9xbS0F+gIrEMjfpsGTQ+ZTvFUdsKTzlDWX4m3exW/VQWQfm0kMUmlOMzNYcUv77KqFTYrDKyYeo15uzuImFGDr08nnjkz+TPkEC39/piNF/ggZyTbb4ew+76vqfE8zviBOcz2nwtAk2oBn8fsZJR3Gm5n81Eq7v2OcouHnZXU0Uf01V8B6HbuxrMKJgtRjBufQ35/HLqB0WicT7NduYnz+lXE14CtxoclP95P5XgbeZmJXI1+GplIyXPtBta2Krge2IaXVM9LZXcNjUUirJJ+wmtg6J0XeChxJU02UGZFElcQRlayHl8pvOwKB0V15LqnUr6wkujwvRxRHwHs3jFXLVpGLKnG9OcMZh6dxsCUuzt0QUTR49lMWD+d+3YsoDy4CMdmGYHlEYOJbgD3FhFBJVdoqZoF2NE6KnU5v/XBBmsbSpM/CCIm503FdtSFh97aSkByHbX93vxatIrYS73YJCL0jmb8qpP5IOoZAPboYJ2TgR894XFLHxE/L6fev57zY8/zpBMs0EKEUMfT7UHk5gwh7b2JfJ5XR6vnx4CdSbHEWWBnUBsfVvaDXI5Vfq8AFlE5Do87jZj1FlZtfRq/Bme6XCxsf/N9nMSQVrSejtsShEzHwbnELO0mRg7tRht7br9MQgJY5i+lrMRGkuQ2egFs/QFUS+bjkXALlXMosTlGxjGXi95NXLTup8V5PxKTAbUsg3LJ3v+YrxJqYYhhFHskW6g65kl3axY1kU14tHpw3qMdv0qBhsXLKKv5mKSbMejlrST8Wo+XQzxNj+byc08wfx1einKZEscR/mTNfBtJ+0ZW/roS/+d+o6PcA/e/9tEZqSJRdD9neJkHW2B1K/TZYJ2+Cqe6DxmomkxVSBVtHu2Eivq442uXDvSr98O105Uh0ydTV70cgKLAZzjgC9ekv3DxQiLu7e4YPNwI7x7NhdEXCJLZ/TVK1S6s7REzUjGAIBGRN7yaJultPnazF/gbRP78WLQGnbsjbnc6mHJiNIen99Hq1cqoi6OIK1aRNUOgr7GPrlo9JTG1lCYcZt2IQr7+ZjEuug8RhMf+oz3dJXZU6Io3fmNrjTv+x1wYf8GRkvQaZgZBkwVCfXKQf5rD9e7pjD59H50O17BNsxGf9Tqro37ihkXLpD9b8TV5I7h70OEmJtPpJ8xWZwBsPrOocHGiI8KLv6d/TKfDBeY2CQzLfgUSP+CgLJjzz2xkvPFrJPmXMEl76WjzYEJBGqqHYVx1Nu/KpqLt0yKTSfF18MV35Lc83eDHt7fXEWtRIyy6zS/hpwafa5IaFrqb+EhShafRhtang8m+fSgsTZSYYU4jxDZHcu3MOPARseSNnfS6jeFKcwAlKohGxJmKIIbnRKNb+z1vD7NXC+a7N2A1B+Ip1iKyQGRpJDJRLFP8gxjv8jOofQfv4eGAqxjmq6jW9g2OD2p3UmH14t3nv2Ce2zq8tvzI8Bwj55IV2GwWsOix+OgY//XHfGN6Cd8cCw02G236Djb12JOIDr1uqAZEpJmhzSrCSWxha8pzBHXAfp0jSzI+Z3fpE/xivndYADjaDwd17jRueBqzWM3VESewafvYL3+NutaPcBTDNGBdcwQ9sut4d89gvMQHZ9Fd2bygJXQYu4k91IFzj40ja2xM14CLBJ6zK0/iKhIQxGI2PvPZ4Pe2DHgwpLAcId2ZdisIgkCQHIYq7Jvg6wF2JsvkEg0as5rRF8LwqEnl+ONtuOXEcfTacGwvubOodQ2h4VFE6L/BUGPFJ1jPgHKA636VZPuVcuHJArwdMnk9di9eUvArS+FWTQBiQcmUgT85NaeAldV9iBo+wNflKh2hr+MWOgWc4tAWXGJV8naudjwyeEj9V5wfc56xsrUAWOUqTi2axuWYd3hfBD91iVjTaGKaqoNzwjX+zlnK/b6hTFHDUT9497grnbu9YWobUV1R9Gv6qQ+o55deON0yk9cbFNQfUBMXZmV8xTNkPdsOLjD/bn1soedpbo+HmE4dtk4r01eIuXXAyhjPm6hMRSzwX0LQ9GR+KnmD2Li9LI5tAL0OXIeCIODe4ciwW8OokvWyKG8CK2eWUNsdjs5/EyZZM3pTPy4dDvjUuYIIetV5JCtgd2Inm1tzkTpN5LcnfuDl4Lvzr/dpdhS/jq8rdPZBpddJ6kbk0SiaSv8tCA78kRnm0WjMwxAEOd2uZ9DJWpmpgYN3h8aF3uuoV8jZ1O3Gmu/fJS/5MhfTjyMCRCIT0oEGysKtdpZgpw2Pf1ddcRuOtOcG4QXH6XMMxNkaxx11G3MXnOKNDoHYolhGXg2ld1Qr3nfKiFIEEx9rrwLmOb3I9OSbnK71QCCQKWr40Gk7N9wFSkzQ4vcRv66AACGTJzqOMSv9I8yKj/g97yM+GD+R74r3U1M/CWdLFNG5VbxshLFhi6gPgYSazsEDtUXSi15ZwaXeCt53t8srFvc6c0Q8ke9rzzFUbLNLjEoE8J4w+Gh6mZKzKTeJah9HRLWGWv0MiF/A27c3cj7ai21t4QiOSnYt2IXGu5lci5okqR6JSKCvdwi+jvGcSdiCWdKDytTAWed4dnV6MWN4IakjN3Ct+S0uOw2jSCfwedtIvvb8hgz/3bxQtIEOpS9OxgLK7kSwXZpLdms0LVY40eOK0TyAzCzjsOZPIjTNzLMsxiBAvBz6zD385lODRYD3dtYTdOc7FFMKYMJn4G73KFnu+iB1Hb4c8F/DsugtfOB5mUsWJ8TXPWi6DL21PUjV6TQ6HEBiVdNhi0KV9h2vrxiGW8sApk8E9P+GMXnt9Bmae+PpcNmNLVXMzOJagvKyUOnywdXOcoszJHOhxcD1gaW0qC7ir7LS6jAZv14rvl7XKLHFcvHD8YwxyRnDfE7OugJJ/9bPRFKOtP/MxdJQ3Ct30OOiZKC2HofCd+1F9IGtrHt/OsMjVOR5X6dP04KAwIZucC8Nw6dViZt7AA3Rk7kTapf0mqACnQAfPPIWgT5B5K7eT2a4nZmqEdnZzzkdEobdGkaNpI/uDz9m6T3bU3Z6yzlt1qJa9Q+6jgCe+vYpnphUyaaZR/nGE8QiuOrSh3bUdc641PPPHTdmB+qoshiRieyJsCpxIivrYXSVD/ElV7EkhyOWOQx+x7whhdx4L5dvyjJY4NPHKJXdVXqOeDTKkhTaTz5Ie7iR66H2s5CbGL6bHse2HhmvhuXxgOdn1EtfxWqChLpN5Ac8hSCycq3Un8ZSf1qjbiOE9eKg7SOVK6QCPVb4+sP5mKv98HuinzRvOwAgyaGd4w7wqzyfjd1wUy8l81IazhodUeH7+abbzgA7OlLDDfcPqRnVi836B0v+GMmo4XrWRB4nwPE2btoigpdb6GmKpt3SzlNt8LRXEmnk8YgTfN1sZ1MbZU0Y5PVYxXrud7CzbT66vpLqnhgmxRspSCgYbKf3ZMNxdapiTbMfQY1SouaqCI49RqA0GrFNjk1sYqhzGQt9D3IoKwrxv51lZ8V8TKBDNdb2oQy3wV9R5ZyesxeZwn4fS5tgpFsRDoYQbizO4fXG2/xV6svi9jYKF6UMXqdDHMjFtEMYnZWse/4Uk90rGHHXzcHU/y6NORKsk904OuEbOt3skvGCyEq+0E9XYieZkfVc8i1hu+Qib8l3cUv1PRu64Eizmc7CRAICFXi0edOv6ectV4hVQJQMfgjswUXhyvptdv/wqFI7qOBdVwiWQfL4kTRL5qD7OIqAXRLqR6cTJLUnVAWhFMduNWNPzMO8zMLZlJsorWoUpjIwuGL0XMb7LZ0Eh1iIWuDMluFfcMqUwAOBv7Lp9zeZejyJ1sJWxK0teDS0Uev+O3pFNQCNIe6Y3BeRHAYZ3yfRp/UlJ3Icw/Tf0a4PwFvnypDceHTDreAPAlIERAz1OU6Y3zYmStx5sVeBqjSXiPLrbI34HJ2Dfc8VLgnizgtfMKRmE6mnX+b0nTbaop/gqdTHGRO8jazucZiLk6lVS5FI89EZTTAdsPaD3IknnvmEFwp1iK0GFhz9kV5/OTLnYTDxPA0/n0fdU82dGiem7BUoiR7FxdEX+ePyCU7jyKrxCZx1/Qb15ZcJODuK7DQjXSYXxDYFCDIKTkfT1ajDqlRhsNiQiqQkuhayIwA25XvRJQmkX+TG00fzyYy+yjNDpuJ5d009b/Ig19TI7Zhargy9B/Q0CGAWpAgiKW1OPaj0NjquJNKS1IRXoH0vbBK5kS/24azLUdylEQRI7SC6GDmIyq3AACslgEiMIAbRXX9xHylMK48mbe84FNEmQqKMhGnBQ9zHyNEltIW2UNcVR/aZFLzHhBIx72cm+nfwqmUY2pc2Mt3VyMoz4TgWRZOftIOombfxDtLfmydNTWwvjkbVZSHDbTz4/8QQaw6/esELbRATupijVX3kmK+TGGhm68o/eSiynclBW/DyVQLf4dVwifJ/RuChlBHgp2Kb3zWebgV5rR+jbw6hL7WLgGwnkhomwSnISdKyf+5+trjXEjRMxFDVXGLDrpEecZQQcxOWAVD/2xyw5/YeLvmcYUGAK0IEWAWY0gDjzFruKJyg6SRY+pGKpAwIsL4TXpKG0vfqNbryu1h7eS2XRl5iWcdC4gL+xNviz77GEZy//1mmhU/j5enBbC+8578INn5q9qdP1chmkw2fWYm8IVpNt3M3Tv0pBHQ8xF2rXw4/dRjNrkZsTxr4qgt+6gHn9sU4eQZiNVupLTiCg6MVjRBNfO03HE6xp7LNPEt/mw1rtwkEMZq7KZABG6zelkFyThW6SRkwI4Lw9l+RSZxRWDzIbLofw+6jdC3vwn/hMHral6OyleH7hS86kw7GQlPEONYuTuaAUYFY/CqnfaTgkslNYxJDc8qJKA9HuFvxECHGUd6FVgz9NrAKdnWiK+P1VHqfJ7o5hS+m2OU4L903koP+ARRLqtCpbpNeKuFK/QKu1C+gwGcOD+XFgq+JktijdDpcwCSegOCaRo3n93jlj8FZbz/PXBhfjybmdx6829q/FL7A8snfsyW8FN8f1zCxeCKHVhQjFlnZVvAWzhOMwCXuCm4C2AuY2POTc/fOJSFfSc5UM4IgwyYy4yeFULkTnWXJuNUW0zG6g6VbHckfMos9C/Yw2/s67098gidv+LEzqpRX8+fxwg+vs3PhH9QG1QJQYVCytdOBncp1xGf584yHBG/TWUzWFOp7Y+hwUFGRmI9V8Mc/62WavXtI7ZYxxxWkd0ApDGAQdTJuwEB1WQB9fWoA5mhgs0Mr/tZsjk8t5eHGdnz+GY88wAt1etdgL3Q1n+Gxb7ZyuOkTLv4t58fIiQxfYj+3i232RKFTgBNtL3VxZ0sBjynta2J2aCnffPss/fE6CLT/zqscYbNXPfW9kQwzdxFUkMqvwWYyro6j3q8C/QQ7SPyzNi1lYf5cDq/goZIWko99RNaEaQCscLDnaacXJSIp8+b7hb/RLLuIrDmcmT2PszHhJkd7jbzaEEhlbxcJlbWkHH6X+1/9HXTpdjCHWEZWcSym76ehmN3FupQLPCJP5T5lImU5KvpPVNIwL4xO7gy2Q3bzDRoX/sOSFUuYfTWT1umtqC/G81mLjDfXjsQ/+1mIeBKc7V6H/yrmVSfNocczAu+Kyww78j61cVMh/h7AWhBLyJvwPK6Nhayd+yLewc28efNbepv/d3v2/U/Hf4t9/43/sVixArZsUYMgYocPyERwsSQGk1JCR8dB9t+eR1JmHgDuYpCbNDi0V2NtUaM4movHyHxCjY4kKqDCBCNUeiZLIUs+DI+UFpID71BqEjGhwT5LXTQ8RXubilMqG1MaYKTLIxiMfnxW+AyjE6fwD9fZpV1PZFogh2IPYJWqkbcdgDv7IOFdwrs+Q3PClXZNEi3BqdgcyoCbSC1Sxp4fi547CD4JGLTu/P7QWXo1bdTq4DXvVEbNPIvW7UGkH35Gn1sInQvEpCggQQGNutep0cYzrKKYyL0hDF9diMHQTDvePNnrS0JNPd0GFeYAPSu0ueTfbb9/+iDbFMy1/SqUnS5secGOuJ+u/oAfPGG+BkSCBUVbM14VtbTFWhkRfPfDBW9DzItYnUayNuxPHAQVKT4CtcpaKszjKAt6ElPr9wgiMzaLwIiLI8gfkk9RvF16xigycDDhFnlRtwhqm4TMLPuP4lJjfxAPDMlH6u/Is/qxdInAQdbGGT14VMJQ3UPk6L7AT2zg8Ye1zE7ej0gEwSobZ02QVz2BXT9mogo8SqH2MJOk8IFkHH/fWMaAWUtSeAfX8ttwKQ1BoAWdAAqRBS7Oh5mlZDrfR1nFfQwPgL+WvUGOg51JtMj/eVz/vIO46DpCwgwMald6HGqpCLMb0U5w6OZlD1hsGOBs0E7ueGSzSrQMnWDX2nfWDHBk2hE8/KYg9Ns3SDbhX/rVIt6sCULQt+Kh0+PQXo2gt2dK+mxwNfBx3s2L59qxdPamlVASkUuwdCY5Ooi848jv47awyebC4X7IsxkH23JTD2zv09HZcJCT027zanArfs4jSD6ZTINfA7FnkqkcpiR93h7mTnyUm+p6fptzTzNcZnHGvz6czMuZGGb04FFxmH5HN64v3WC/63JYe/ULjJ1iiAsgOHEXKq1dr9NRYqWnfRa1y+ZgMLsh0XmQlXjPuxGgwOTDc58uxIIz/vVuGJR2RL6f1C7BiDoAS3o4lqDhuEb4Y3ANwKAyUB8C5wegrd+R8ktBaKVmTC6dOHQNDF67MMUBReQ6NN5SVAP2ImC9fx27e0Fvg1WTbrCj1w1Z/TCSc5Lpd7Mvaef8oNh0k8173qTV5Et3cij1XgZiBSccxOCvMPJjXyXF4izmmf8k+EwNw6dexU89mmWq/aQHXmVF4yiqH/+OhMKlvPXOW+yfs5+8JPt8JAhw1vAwdZUFSIPqkBws56msp/hg3Qd4+sTRZ7Xh3OtAalYqxvRuWt2a6NJ0MVQBbQ2eWPMUuD3gglhsHzNDr3UBcq6bBKZ5dOMypAeTo42RhzNxa32DnfPfZ+DuZmuI5Dy38ybROTYf42N/872LiM1tbzLEZj/kTiwZRuOlZAY8Kwi5dYHGyDHYxAYeagFDn47MshjcSiKojuwktCSWC6MvsL0P2i9XsemTQEpinqYoPo/nkl/Betnex+UiUIngjsnCtuwsPLuamLN5BltXbh30/buZepNur2AUwMUPL5K1KYuaB47SEnGTd3VgaNagFHVj1hkZljWMFq8W6gPq+d4DvO/uRCLkZuqDjDRnXiRFPYQt1RMYrb3FXJWBeU4DLNf1E3rLF0/HFBZseIaNWQ2MDjiIxPg1PSf1WGRyRg3cx4VRF4jImMNT0+PQmx3pV5zkrEcqYncF/9JgGpC5w7CN4J6OWFR493cVSD59jGbveq42pDIbyFDZE1RCh4kXP3+R3BHlg/1TjIi4wjhcVDI6Q2TcMUG4HNa1230L3CXwmTvsa3SgfvgKzmnquM98h61edkmpCy1BTM6JwuYkweLmhlnlQ7/id55xBrPTa+z0HMbRCduZqAaTYP8d1gSfZQ1QarzBX4ahLNu2jMbgHmqTA6kLG0aA9Z5UhKdLDz4zbyE/kczMgzPpnyyB25/zsFMRr3WCRTBTFd1CvvsNzDIzwRIz7NAgiGMZ3WlgrrOY6e/+gJMYLkleGrxuStZQxlzI4OJT7ex45jtCnAOJBta7wRBrA6ElcsyGPaRenskD+xfz5RsXcBeaqTIDtqkcTtiMVQxXMq5xIsDMRFMu3/bH8U7lVKYvBOcboUw4O4Zez1o8e2/QE6fEzacf3Ecg7u+nJ7KC3l5nYiqm8lq+haLhf/KsM5TXu9FV7ElvU99/zFHh7Q/jKuvjcCBs7oGiCj/aDj7Im2OuoPO7Q6HRLvX22G+PIbIoSL//CKEGMY7eGj5KLSY7vI439bMJOn2YgS4D9YfVZLemMGnZSZxaPRGHl6NSmTFr2jBZLfzL7v6V0bNJ787GQR9Pn+IpPumEin4VE7+HyyMzue/JD0CmB89R3Gn6jvbvfDBHbiRdFE6Xo8D1dHtCuziumGSlFmNrD4EFFwnru0mjMoQkcSV+zaN5/YdxtGQY6fPex0pjNy79BTRZ7B5FNwIOU/yglADXbhZvXEx5RDnbl9qTMFVeX7OxdzxByRp0PirKfDqQONyTJZMAcyI+paIjjvzKUSCARuZAotiNtbGbKTCm82JrEIWWKxjcr9DvmkedRcPbS232k525l3NLTvBPZhX1++YTf34UYxZfZ8bUZbzQ6M53L71Nx/LbXJ/VyW2l3TfGXwo/eUJwYDWFztUkO62npBGmNsAxP0j0OkuE603i9YdpbnJhf6KVG6HHme70EnXNjvzpDtfkpfQ+lkd2yxykRgkKq4IpHvfW1FZTJBXJuygvG8GoHjXDOp4jS3yVHz16eSDiEh0SFR9WOHKr1YGJJzIQhBhu/eZIctQQcEtFcnopS57M5f2crVQ5LsJTVc3r9ZsAUOvVuLWr6e7txLf0FHGEszwjm9fStlBrDWdXxgaKhroQeeMRHC8/zrknbyLIs2kKhWsD0GSFf7q6kFVVYzb2Ee85kufS4UrdfNKcnybNzYvP66eQdMoBtT6UjCEHcZbY2dGKqi70Ox/lqlyJpu9RDs4+wOsRdu+mE/67OSCayIJdCzBqPMifq+TlEC+4tgp0VZC+BVdRPzFykBqsuLfm85jTeNjVgVQQcfLanwQEpSLxrqdwiH2+zKgG/To9xl9deLhngEuScRQoLHgrQGLVMdajkARtO681ebFTk4/G4oat0JEOzwB8XWoRi+D9Fg+ix02jwShhm86ZJuU59oqv8q2nXSpWLmhZsudj/Mp6aPZqw8u5l8y3/iDTDRZrocvaxY3qZ6g9KMG92YBR1k6gaA/k/AYemeCexqnL8+g+YyJ76SG+n2kvbtW5ZJLjmEZTxn04h7sTuXs1bYqjmOV63hMkLOrOQ66NxtjfiWujE0v+ep6t46voi91CV6cLCsC37a7iRXI7bSpHKt09WVixhgqLDY/+IA79Go3Iz8CBh/ZidOxjqXwYZ+78yuOFPnjP/IHCcf/g0O+Bps9Mt6t9zlo/ZhphrtkgbiXnUCTKk2cJAfzS83DI3QsyZwh/DMUJLQqDgLjfxsE5l+jRZJGhtCegzu+ZznCRBmGtXd79X7HeDSIlGt71SUTm446AwNBaeMH3AXTh9oJgtdbA3598TFX/Is71CXzpc6/KOdVRz/diVw4GHiXS5oJELPCY6z01hw3uEOhho3nCUSaemknJbwuo/6gW2MLQWpgoeYpMNxGjLoxCJIKiUY8yKrQReorBKRaAtJoxlFlP462XcP+AG2Nc78DfIlxxpbt+HOKOPgSv/wRDODkJvOKUxwoLvNJvtc89Inh49DQOFs0i3UnPtstBpJ+YQNej+UQ3NRMpmsp+3xyMkhZmN4JHeDFO7i2MP36MD79fx7otHwLQZhHRqhvL+f3BSC0qUspj0Tj1wbJrFJvs41UsltKn7aLNU09QpTshOZM44x5BhsNv9EhT8ZRp+en2VgYGBHwlWXi2JRIaXYtZ5cikCwFM2T6bopACUq5H0eIpxuIsQSUCNwnom3/EvzwE/2lfk3H7MldiRgLw/c0fsCnq2Ll4NmJBxlv6UlYkrcPSUsgY41i+jTrBr101JF/yQWS+hONbT/PikDI2PHqcQIdqHqn15T5ZJ063kzlkU9EoHyDsLjbne08odf4Jq3Q5B/uhodGNRzav4My4M//h2ScSiTg16RTe5hFsv3qGLa77ueGzBoCJJzOw+ubTOjqCRrdtg5/ZP24leX3eTOspwsH8CA/YPiDEupsoRy9+MKp5qR38m9xYvW8eVVPM3Jg2l0uxqeyptVslLNBCU+E4CgtUlEVeIro0mqwUO4p2ssbO6rp4Qs2NvQOEz9fhGO2HSCTCUwKPOMExURtTpHp880dhGm7BNQCSFAKas3bvNVWGgFT9EroM6JDuocAEDX2OrDWVER9bwxVbDs8HOWGcNIMN82YC4NEzDY+eyahMAVhFRgRBwbCseDpd2ymSLefrvW8iExv506ufowUJlLkbmVI4E1IkbFtgl27pNDrz5fqFeHW7YA7pRNvaitRi3xhPzmmnP3UFRux+aFaxlZmONh4Uj+eY5RGOlj9GWGoJSz7axLTsxxh3/DfO5GQwcvplqPwVoteikcqRu7kiq+ggJLeJ+zL/RlVSAOZm3nvkbxTj4CnXL9D0272qHnCA4aH/sK94Hb98Mx/vZm/qHxrgauJPpHqtot9wkR1+Roqb+jh0OQXHXh3Kxd/R3g5icR5ZtjaeaQWzZSa3xyYgljThXvwmEwJOA/CcC9wxw9UufxprOpFo5fBv3oSpClAoe3AWehg74ISlRUHAvnmct5yivPZLrtUtY/VqENlESM1S3FrVPHJyKZJJ2bzp6Mcnmiv8VOUMjiZKG06Q6PkDJoMcudJEjgFq6lZjmhZEQkwAs3Y3oCoLwLi1ne+OK/Aoi6DyWTM9k19CE1PA/oCLeOCCSCQm0q+Tpz3gxjkfwrNTaE46xIfh9rn7pN4O4hsQYHJeJBkXk2kK9OPXSwIrX/gZpQTarRDtMgll7++8+tWrVMSJ2Dm7hple9nGVqN0KfMfbDu04TTzD7WgRfyXZwQcLtaDu9sHxUjr65d2cWHqV/q46Vm9ejadnN/1hsEpbwD8JBSwSXmBZyEsY9Aq81Llsa4UHWmD94Q+Q98lo6W1hyY9B9C2x5z6uG+CEHuK+eARHVyuPl+/Ay5DBwjdf5mHNQTbnbCar9jy6Q524tHmRnZ5NbWAtgqieFOdLPOcGj4TtJ+zWHEb4D8Mm3Jujn3MGm/9epHl+TDi5Dr1/P6Ihchr87ciAEYVniLqylVMrZTx64X5sZhtY7bvoxQ7wpze8u2csIh8xxl4jD/7xIFnDsjg76RTn4xJw7nJmwukJ2Ibc4cDDldiOFHPuzffBzCBjPTghnw4PgQiv1/FID0dxlEE5XsmQOB55zR+3SDfaShQEtz1BbcCn9JraBp9BENkLdjYsyETgL7WB5xia2kM5NXEThSm9jL81joU7F3Ir+Q4KrRFXi4zQ82lUxVQBQwfXYptNztbcTxC7HKcm4wydXrMQcux5JPG/eY3LxHF8s/YCr41+CeGi3a4nwz0X1GMwaoYx5crLTNjfzHCtMycm2fimBzbeBeesjrfnA/19zNTOc6VZ8gvVydWQbAfWnLc9wbgz44i4k0j3qC4i2iMYiLNXWyViCU0+Tbj0xuHUVoH0uxOETPamaE41sJCfHV+m8YQaXUsc9aF+VAfbwZgDVvsA/j6tAUVZE2KrDCQ+WCVWxIKMZJ9jGOWtbLgQgEenH0k3EtH6+BM55TRS8bNo5V08FnCNcT7wUo2REddGkZ9UhJPpKhIRhMmgt82BsHofWv36eOe1jzgtmcVFyhGL4K2gmzRKn+DNDigwSpmeHUMMYjyttwG7RKxNpKDX6olFpMa105kTjT2Uub9NyqkI3BqrMGpc6TOqUVqVTDw2lReac/glVcDVo5q29z4lfPzvDH/cj1kVL/Hpz3ZAqcmq4vWbcUT/k4jsQTNXp6RzM/IzZljsc8Iv/Trm9D2EGDFt3tm0J+VQGliKe+9kvorYR4Heia8uRxBcEI8l7TxGpf2s0t0+jY3Xf8EpbASX8y4yc1sGupRWTIlFtKn0hNX8DQ7h4DOJ88ZjdA25g8w3295PVO2oDFW8n72e+Owa2m+3I2vuJXIgkvKIcr4Y9yNikRirxIpZZkYkFrH9ne30DNTwdpEL1w/dIvShEjzG/mexD6DLNxafO3ZbJ5tY+h+vScwGjKJC5q18kCT/CowW5X+8/r81xGIxoaGhgzm//7/jv8W+/8b/WAQEiJBKZcisLjiJ25isgZlffsvT9bHcrIrkidFFg+/NDYKjpoe5sWcOvTIzN6efYvmQdsL1FXzWBd90w7t6BYFF48lRW3jy+U8BcKsV4y1AsxUOlslQn0wiLjEfIXsO9z+7m5jMn1le78pvfIkIu/a4VWZB56BDZlYg7cuFyq0Q/TwZoXu4PiaRoqgg6swrcVd587XH36xxNPLmi3/h5XeBrqpm3OpbkIcCGvhl1i8cO7aC61elLHqwk7qAKlo9c/ESh7BAa5d0WNdmR50Iag0RQ8qI8bxEp1pNS50n1Z4tqC6m4WJwomyEngITnJfMYIz1MGUmKDYHkDPtt8FDAYBKoqHBAsUmMcPFIqKNn9J1243+UCu5bcNIzShEUfwxVP1GVEojGaUXcHCAUbE9dJ99kGlTNXjp7+PF5jD4czZtyY18+9RPDKjsPhVxcvBXt/J+J1iAJ6NlfH9XUlJtCGP5kLdotYg4Z3+qwYleIhZjxr7ZtQ1IWPPdGkqTu/Gc3oxFoeaqOYgek50av6dHhHe3GKn7bVQX02hPqcRV1cT+Uju7IVR+hY8XrsLBGsYDm2bxgMqHKW/9bjdBlqhwMl5n9O1jiMQpDPN8H3OnH5Lk39lZ/yXL6cWjPhyZs4bC8U9zLWIyHU4dzNTANXMLr6x5Fm2MmObxl1B4nEApjMWnbzRm5wusD+nggaQGFlZE4XL0UxIHEslLyiNJAcFiGx1XQ5mwbwlNEdfxKc/lRGQ1hMIQOZhEZn7qleEpNlPv30KDdwGh0tmYBKiyiLBJnHmxIBHN9xORR1WwuH8xl0depsG/gbJg4NpDlIlisCpmoxhQMfvgbE5OPAmjWlBpDOQn5qOJcEW48p+JkID2hymK30ZpVB5vB2UQdDuLvIQ8jrdHMsW9jI0e4NGzh7LCKKzzFnDUsIGi4ofobjhClgHC3U5xOr2V9a16vumEzrvyBTM08LsXfN7ZRnOnP2Kpir9WN1Pvvhuws3Ff74C2GcP4tdiO/Ln/bl/wMo6k0HgZrUsc35YNI/nvUFzTTZRf/5GU6l5O3QUK2SRibBIZIhH8+tBO1n6+ArPUPl6262B7xHU8zMksKoLZB2dzaH43AJ92wXg1RJSL6ZaUcDj6S4wuHXTZvmKZA/zg2cpjl4uIOhyPMbUMz2k1RCaVkWVZi6sAVpsEvSCj1wYDGhOlUaX0ONn7/1NOYJLUsF18jDOv/cRMzTt4FzqRoz6GUmqlvyOX0uabZE8s5MDQcnqDr/OH+w8AtBtg8pBuDMtG4D40Al11ByE5FwZ/K3dtOsGxezjr/CV+LuG4lWsIaxFzwAdmaeGcHr5VXKfm9RyifV7nEw2EyQUC1Ud4RAJPO0PTtBv8E1uGsWM4bo2FtIQMBeCPPggYkFD81CFONo4gq3Uxx8e+ASL7mScvcA1i0W4+8TBx3QQ2m8CCrSMoD3Vg7aRTrG2HjTEywu+EE1AXiMxBSU3wPXmw6pBqUA8QXycifFo4WZuyyLyUya9h1znSEYTlqZ9oa17ElB49sw7N4tLISxDoy6H+G4Tgwc/X3MgfcMYaayA77DSh2gw6yx/ixYzTnGhMYtp5C0aNL+MWGpgTvQ1ZZyBFbS8wa5vA0rmN9I0ro8HoQKn7Q7R5tJEmElHSPgKjRU15/WXOx0/FSZdITHE4Dd4BGKXOELUKgO+UJzlSPpQLmMhLrqfNtZBp8ukUh//C27+vJ6E2AFGAiOtp1+m6i+rj8nJeMJ3DddejNMT0UakYwvPlZrbPvIlU1MOqFvhL5Eb5+jWEZgjUusRT3xfPLPVlnm6FAiMMBPWzfv16nAxRHGrqIl4Tyrtezax0tqNRXaytqBSw5dZopmV78s0CIxfaDhMks2G1pZOgvEn4Y2X8dXAslj2VOI8vgxMjYJn9Ht9sncw16ybGN3mQkp3C4ws3QeJftB15gDfvhFLmItAQZuFWlF0yMtY3HXwdENcMZfLVaFImuzG76R3EwAfhYkYo7R4uYUMraFAbMWpiabfCxq4IRsi7iVD1IhdZGfXnffg0+SIO6cS1MZDL0hm8UniUUnEjy2VqLow5j9gmBwTkd3OHa/yKaCn70r5WaAx0e7dgVHhjMLlQ5fAGbuPiQOmNrOsaOx8+xxcdcto3LCSuIZoLy+zXuJETxtGjcbSubiPlZgodbh24W7/hyaEvsXL0x3zw1XwM+VF4PxqGrd6XTJuK74GHWyFVCODLCdfJb51EW3gzC91b+L5wAZ1ulzBq3KlvTEc8kIXFIuP3B35nqFs/btpnaV2/gWdcTbjKRURb1UzLV7DxZiYvjClhuF82eY7QZC6lSxHGzR8XENXjTVFsBdMj75o1OEWDzUqOaChGVTnNYgXJWbH0uYYNFvtkFhecRYEM1LXhWZMNiAmtDKUyrJJpbu1Y0nLR+acSU+iJqXgok5f9whMmWFXnxYOycNJSznK8w8Se+XvQae3SevO1kKmEL61N5Efcxsc8kibZZcZpoxD1ynjFzUyXDSYZPkdeOR/99Vqk5gFEorX84fwZp+tgdoCYc7VBzPl2EdXD4c/JcExw5O2zU0BXCTMKudG6kO62sfRMFBEzbR31UhHRgEJlRJxciDRoKILtX+VRcBbDMCW81QH7+iFJa+W5nHQeiWgHPzsy9YspaezYuAhzdhwBsxq5HgE5ph38oX2Qt6pS/i/2/iq+yqtr/8W/yy3JWnF39wQSgru7FiulBYq1VKi3tIW6QamXUqSltLhbobgGJwZxd0+W6/9gUXjevQ9+J/+934P9jKN8sm6dc95zjjmuMa4LoT6B1YmbaDWGsHjbcHy1Si68s457JrikFzLA+C7/tAahMIWRM+JFgoNhdegFrrWlIXPIGKMK5Nbhp0m54YFF2oRdJKVHxUKoVcOMDrYb3TkT+ICblg0kVmyhRpdA+8Nl92bmTSpjvOgdPY+Nz/6CUW6kvRte1QzG0dbM790gt4uItUOgQ0KypJrKphF853IKN4eSuFsJhBncKb9dztkrQ0gf7NTh3HRnLcEezyPYmokq005gYBWZ4TvRPNS1nq6Us90iw6hVIFMKURgUCOxCxMWwyFWIrCWLWP9yPMsiqfeC0I4nWeyfAmXnnA9uMzFXfI2s758jRwoBFfspCEtgwKSLtLd7o/zjPPYsI57zR+PfNp0Wjz3oo4DsheQ0zOHYF6F4TC1iTuNw/AxKTBMi+Cv3PWYnf0BReyy5jZ6M0pk4vWUQ5v6DmTBvHsOVMLK2jubwOnLN8Lcple/9YbLkKPdMzuDmbbuJF0Zu4qrHcARd3SSqH1dQJMrgnqOT6qYVWG5vpiVdxY5Jq6k1h7KLbijdCF69+DLhL6RBWgwe/ogeBv/qw4/inZLNB23jEblI6XWggDjlQn5e9jMr3O0k/ZPMqLkvsz9nAZ52DYZ2ESnCKvLFpST7dCMX6zAUmrE1mbmZ0kpx+llesxfwqyaAGNca/qjs4G7afPzD5bTU57M6bAOShneIcE/nVr0/ozyX0iBZSlvQZU7F9cfdHgUOKG7LRGt2ZxBww2s/6kgJt3pe4sueD/dDmT+C3BuNqBjdE15EjxwE15yaNN4iGKSA3ZP+pqBlACMaGuh1phBBT6ef+k4rRHYOpzVtLN7eAhJP+6EuXYT7R49pbnNMct4OMnC+0kRs/kBe6mpmfUYBZ7uVbOtwxbt7JcOqPkMf8DdR7y8iWeM87+kG5z7rX2qqptAaOuV6wl3d8NaG0iypJFowAi+uM/TcYJqiGjgy4j3WRu7EfjkC4di7AMg75WQUZJJSMJxLFcMI/nQfMSE5lKrtfLTyF96MGk7qliDSC5fxy+JfaBXbUM8pYMPanrjn2ZF621A2VyAQueInDyfR5wwf+usYYXZhY0gZfdrn8d7G98idXogitJm/uqDKClUZtxDa5YScGY6rwco29VmeyR6HTaRnfPskAup2I7ZKKc16lS9GOjUIf/JxVkQKHA7a3cYT3uMBlZJNONxmM2eUPx7955DsLD5Dqm3DobfSs1FOwIMpPD/5Q8Q2G1mqQFoVZkQyEQPPp1EcrSK3nx8bu+C3bniryR8exGPMr2Hwgav0Ee7ktoeQ8o40hAopRwLguNZGSYeFH/NeJHP0DLizBgfQaZQx6acllMV3YRVbGKZ6vDf401xHU/kAemwaRnS/+6zz9GVtbAdypQl3EbzWAl/Jw/hybx8SChIoGXmRsYlVSPSVoA5xLl2OJtZ6weXWLky1dposfuDvvP6e6XuY6r0WlzsdLNm2hL9H/U1FeAWlXVGM8LzJL98vIjtUTlBEA/2Up4hs30qTxuljz4lsJOuFP7kvGY2g6nFZ9SgVvKCB5bfn4nvjHLpZTv/pX+tb7SxujhW3EpecQ8+XemLqDsK0sZU2wxDcS8/ydvoklqStQP3LDbYVvscrPy1nwfP7AGeSocTeSoLXfTocsVzZqaeHKIWUGC1jGp5jTAbEuebjFuiGwwFiqxqruJPEqvUYZJXk+PcmvnAA5qYt/P70PsZ4NDHFcJXxo80EuJbyT6cK69oPcb2+i5idMqITA4CDzjXt9qdkR1agNLvhFTuUmzEedLi/R5bQi42jBhCgKOZudipv1ruxZ9YZtiY7JzKhwM53N37h4LswaRLEAGfGjCRWlUOuZCjJljNwaSZuLcWMDXyZve2vYxudQFBUDcL6QxC7nA+uncfmqMWssPLDy+sBcPgBft9hMwZyos6A1kXLYPdE1oS0A2t5RZfJFBXY3Sr49KkLyKweTGpd/fCZhJRbYUOLD0mCWxQmz8GjoR+DckdQFRBMYm/nXPa9D0y858aYrQu50i+HJi8RpZGldLt1czkYTqj/QWm14PhuKU1qM97L/2R4ZAuGpscJd0/pHjDw41XUB7XRVa9mb0wT77yUTOW+Ogw/zqZjqo1uYy7ex1Jpi7yKPKiZXtU2pknE9Iow4PCM4YZ3B4GYCFLPo9DnIFWKuzSbxuElVeKQiNilBRebK9cjjmJxgG8ZxPbLpijrBq2dEXxf7EGAbyKHqz0wSA+i1vXEM62IYt9qvM49Q32Bk01nQ6eTbn6JXceT7vexRArIcfVHYBdwVi+iZ1AmpDuTcO/4NaEPqsbbkkG73Znc+n0nhEbnsW1ZA6MS38LYaqXGpYY1760hS+nglgk+8HRWWFeWFvL1ilfoN+EixrFh3DVtAiDCVkh9fSACFzmtnq3c0vszy7GE/Lxu+ncN5k76n7h7uxN0NJQOXRjaD31Q2I2cDYRAWR7Rn13nmU5f/m5qBEAb8fkjRgGhQ8jLtwYzITODSm0Vo4+P5mqfq3wZ1clx6Wa8tQnItUZEXhHUdJYhtoiRiG30jHub7gdiGsTOyWPCLxM4LgZYhqfIqSVoK+xA2P4AqSqTI+OO0OzdjE7u3H/6dfiRnJeM2bcFx3AH2a1Ov3mK/wt4+RuIrNzIG9EVJBcZcVSIsf9HiMSOFa1SjzhOjEwlI/eTw4Tes2Pr8ThpMrIkkvhiB20PGpHcraA2QkWawIOmp16hR1E94R2bcZN6oK/tJCk/idKoRkrNGWxvlRBxfAQlASXQM51el5R4B2fSESRl3/3XqfBWkRd6hul+MsYf7IvA3pOaNAFDw3+jp/9xWvTB6C2DCfMczZzf/8FBKoVv3cTXZkTVdRKAZu9mlEoXQoslxNRncETThUrbi6yg/fzcYuHnGjf6BwrRu1Q8ep/9919BOMwbla4adacC15sdJJTORbcp/tH3m907m6bw3vS5LcEhk2EVPU4aenrW17w5+BBBfYJ5kJHK7YcyJ6cbe7HmSgaO2EXYcODeN567BSpqg55H5JAxP/VNpJJ2qtb2I7I0Ct2KfYzzbeameCXRHjf4YkR/yut92f3KSkIyWqh6by1/BFlolvfDTVvDX76QV+VJ1Z6JnB9fysR4+MTLKc+zpBHqbF3MkoaTs2UiPRuCOfrmdipV9UyWLHz07Ke0nWzP9sZLcZmhH3zHk3IrgbFrOSR7i6hbJZTGe3D1eiymN7SAJ+3u7ZyO/pJlv31Nu+tlessE6DRmGn3b2VY1hHkhZ7HZJZQ5plE9vpBR6b0wZQvQumq5b3Z+89VWeDL4TVqrNfwSPI6DIQfxb5uO3OLH5HpwaRnB2dHHEA87RUB7X/zwQ5Am4LzPGs7ZzSwyTaHbXc/V3lcJGjCUTard/GTxojvhTbA410zUGppDM3CvvcL7fy/A8dRNboatpLj3R1yJjuXt6W/j8eVF5uTN4ZuXDxPjnojXcQF3ngDfXg/4IeEHnr7UxFsuJoqyz3Di9zl4RtTw/GD+h2nqCxDabdRFD0SubaEltCe+PAYD3ZrLiLx9iq7WXnQ+3cKbmw4gimwGvP8/XdknEAhwc3P7Px/4/5D9F+z7r/2vmdVqo6tLR0LVOkZJ5pEug8VqK5V2KS1BOm6532f+fxyvV5cQOOso5zQR/O2XTdRDOuB17c5N5vHSdBIOxiIeKeKD84fpnVzJJ65fMNi9iueaYEdCOx3R65h2YAWh5RI8RJdoswMPKbMi2pt578N3KetVjzXBToynGZGxCqZ3gNiF5wigdMRRpng9SVPVcYwut5jlAkKhA6X2c5QefrQcOUv43QsoM0RE+cJTJS9zSt9A10UfOkJi2fXUT0iAP0KqefXKx5wzwMuVa1Df7EHr9BXcSl9AZkg7aYoT7Asr5MDd9fS/V4Bcr6JHqxsx7SkM+syZQd5mB5VQQ7zhG8wOExEXfqXZu5mghCt83AWfNCvQCkQMH7WNmp4ulPwzhdNHJ7A/wJu1c4eDqRWxoYRR7dtoKg+n666QPiX1uE14DXXS6/z2YzFeLTosWhMOWSw6l7MAvKyBhWoY9GcvfCpCsL34OFAndMgYHf0z3RYXOnePoLXKn263Lwho1NFqnYqLAEIkIFPvp364gk6PAL7JmcTPRT3ZM2EBbiqwRUFKzSHyX7tO75vP0LhzLLXu+/mlaRktmn1Ird4IhWIcQgfY7eg0RnTyh5p9vsNA7oXGthNzlQWVdw1SQxHJ3aN5+eXZjDmSwPYnNzHbfJZ/VRwcAgsAB/xhfyeckhswiZWMPe9LVNELLPv0dUrUMfweAVf1Nq4awWIT0O96FjnJOdxLu8cKNSxQm+nh28n5geeResdzO/oGFm+nk38uCO4aDaxt/oCSoVaEyjaURh0qsRqZAALFNoT2bk4a1fTVtBNk9CP+fjw5KTlMVDkrhAC23dhJXVsyqhH32fbkNlo9W5E83PdkFh8Buwh1o4wIbQTl4eU4hA5M4ia6XZ06Hw6liJ+W/oRRbiS34Dm+6PEJz6lhQ3oWtrBg8hx/cS3oIBPjx/Fn9e90ZtRxuVvDxw0Srj+kS/jXOm1wxwRau5zNCzfjb+lHSPn71Hj99uiYhKqvMdmMtMr2ITMKOP+dkvTjuZR6S/g9Yw5PJDyBuWk3v8/7nVjlcgQ6Iw9CHoNfTcpvsFWNxmzIRC+vJi8xj9rAWj73hHhHMBPbqnFgp6uHg78V2xBJxwGwtgNa7fDOT5/ybsNQJvwyHJcuBRnfnGe2K/zYAamyStpa47hlbWF51p8ozaEstgbzbjNIrGpGGCGpKJPiQAs7Z+3FvbsfAir43geOSx+w3hyATAB2h4OuJAlnw88yTgVHAqC89QIHjGKyxXoSJY8zebKNMM7aTKTiDr4lHgQ3K/GsdVIzPejzNBENJZCwjww51NhtdE4bwgHdUZY/pKEdrIQ1Ais3JVZiHHY+2TySkLAm9sdYEYsbuGaEfdYGjAENDI8+wZm47whsrXl0f6tdyh8ln9JhjyVMPphjns8CziBhtN9RDPoHzDNoEGdH05nVhUInRWZy3vzNfm/i21rA4pd3saHdFZvto/8bF7tD4JwLYsbH0HvVMJp+OcJz3z/H7Pc2EeQGvRttiDRyfp/3Ox2aDlIFyznYYOX7xAJSTj6BjyWS4wnOQIC7QMurfZ+kqDWTn4tG0XunmJK0No6HL+fS7ZksGK3lSIbz5quEE5jz9GGmHswnNyHXGXA0wnfXfwXgoxErOXF3CH/nxKAu8ycn2YFwyeN+sZvFaLdPwjfVwt7h5bS53kIkFNKtGYwhO5UJNzOJn9eX0wGZDIp/+DHqygkW1DF9xW42tr9G3wsKREVKVOP/4QNPp5P/dpeZwfH30bnH4V90DpPaF7sygggJfOcDXzeX4i2DX075MGzfbK7NquJiTzmh0m7ebYUPHTbcWsKo+3sKvnWdeMyu56MWMbUOM2s0M5GpOvknbjc3TN141/lzwuLKjOjH9NfKdhGLdi6iILGEsd9+jrfGBO7p+Ii0iP5Ygl+6leKxjze1a8b8AkovxLt30ffvbKSKDJ4SZDE3oo1KcynjVE4qLLxLOZ9Uyo38eI6UZlIdUcOrVak0Ki9SU5dMWnwRWp8OIoYN5eWEl6mWzCBE7CA/Av5sP0SWEG4YbFh/mcmn7m1I3p9Nn/b5vNBrIQJBFS8OvcToSc245UWjkxeT7XiOg3e2gN8ICJjNB61wTS8iKn0AomgoMDkpvqLTivmw2pM5YR6MPzqeeyn3uN8rh9OC+wi64LxrCz7BCtLjg7E+lcm9iN+QGpxVky12qI/sT4dPP75uOsLy9kai2mIJC4b3AhsotpWyb6qEhIAuygRlfOkPQxVruPzVq7RF1nKvTy6nfp5GWmQHitIkmiLb8A5sIUUG90wWpAge6dEcmXSBXUkPgfLrS0Duj0Wq4MdFvxLSPQOH4nNag7c96peFv63GR1qE8v1UKpLHE5Z7BIFDwEI3+MknH1NcPjOqetEdfIf35h3kjiABqMGBjZGaaqZ7wtVmbzrUC6gMfgVwUvot14AXDdQ5YH2D0w8SICCp6gdei1rMfZ0X7+0eT8AVP8pTKjEoTBS036ZCcZ0KIwzFHZsUSiNL8amI4uV1L3PxyZNgbgOXCBDJ+f7KFqxtndgGHOAVYRvflk0k/s6vNPRNo3bKHj7LfA3NX2JizDEUxRaRZ3ZmY4dVhDH5Thq20W1oDo7mQd8cVGkp3HwQiY+qnI8HH8fS5zTJhZPYcWgWw2aeZl3Gh+htQgadWknepQRASULyJZIEcFcAv3YB2JlIJDO2x9LtHkhO72ykYgGhSPixEzaZPTAqPNG4raQ6vpaW4HSkSivwknNjXfEndZIQdmshGAEi2RrSPPMpkzp1DgEcOLPA6wOcIMpUF0jPiSAFCZnXM/FuDaBhqpn4Md/RLvHHVh3KC82g1sXzzL7+mFUCjo85x4XQ2kdjIMYzm53SINrnb8VN1Q/fsLMsDayj1ebGkJouhE2T6OO/mOyxjeil5RQGvfPo3A3ddpL1Yp4gl34bs+l3/CmSbrWyVnWMV6Z/ApELQR3HfVsPumxCul1tCJ7cR0yisyrwZsAtmoOycff3w3g9h2f+6oV1WRzwEbRco1HzDRUpgfgmB1BU4EKTVkWEQMypsgXcaxyGvfUnVhwaQuG4Jq4N8cIjupkr7R/yacfTKI0RLNkaScStJOIm3CWuKx5rehGpMhupMthCO02MwZQVT2fwa/hqutCLvMHSzMha+D41FkFgAHdHvEKT5jKxEnhb81Ag0cOpf9el0dHtVc7Ug4PA6se+afs4a4AiawsXE3pitOipj5BTrXkAPExIE0oZHvQ1Obo6JIE7CHrRSs+428TaJmLvb8Zql3CuoA+591OpmfQAT9+9hAKh7s5bu6pK+KfvMgJksQw59ix37PfoNeAqFd4/EhHyLhHmZL4bc5O3br/FFZzBAAGwPfcDAAbZbawd+Q1H+zv4s8GVsYZuTmYcY4RfT7Dq8ZV+w4KBP/Bx+1tkZEfQprEzZ/XHCLt28nrqfV7950fqOq7j3ShC06EBIFF0gpSwn1iaKGFOeSrSvP4E610ZKNrB99c3kN80kDzXnXwi2oxS4c7snYlEJxVCRgEikZ4tRj1bzAeJ+ttKUYYfK/o63/XPlnR+675DZP0bvCk4gyB7Jof63WDdQwCvxORBgwDiHVfwponpH29mdbMPzZISPmwx823fpY+KetK6bYQfG0ddiJL6qHQuqV4lZo6KKR+lYpLmYLYbCbCn02bpAEBod565sy6dQVuDcRlgJ+jibxQkttBsXcnBbgk5NhjhpyXSS0vVAyu2pHJejbhHqMSOq9CZwPiBJxzptnJqwClqZKf4whBMH4WDHCtIBEbGvvcr53UyLGd+4K/yz3g3yOlrfuUNZXY9Yx5oqNz+Fp1ju/ltcSxNgqH4iF8CWxb12m6SCj9B3+LO3Qk1rBx3lGaxJ760UhlYy+EFx1g3+Rv+mHeMLtcKPh33Am+fGo6bQ8rqFYfYMv8Q+uYZmJXu+ClTCDfE0SKwIRFZiK0N4n69ilsFH1FVJ8Yl6wLnZRdIqgJPu46+I/7BqOpBhscUJg0bCkCxGUwOyPM0k//kNsKqeuDz1TLmP1nAzjFO/fVLRhAKBATUBRBYG4jv1HSWZzwPD/dLAG6CdlS/zyBZIEKQvYUZw2/zaRD0rYHyiHJsniocZd3ITLJHzC+fnblKesp0FG2uCH0cNFy0sLvwCd754zPSZfChJ6gFOvz8izA3DMfvYgfBrsE8l17NCxoYUQtSvwgqRwfQ6LueyfsnUxRTREFiAXbAo3sARbH3KEi8Q+X9Sm7V3yLB42t621cxZtJCPPX9+PLvV5B71DDON5V8k4NTVROZ/eoPiKVipHf2sDTgNX4/MQ+r3oOQB1N59sNfHr2zzeCkDO1ou0SI1ZOKsG6EDhkCh5Dg6mCGn0rDmNaEl3EZcfJ/cHNkU2cN4nLVNGZ25dImK2JemIacoS+S2LOSP3I+4Er1NKq7EpCrQKh+uIY89KUFCAEBpV2DOfpzf2JD1FxNs7Gr21n1tDN/lfM4ASi6m3AIhLR76Ij1byTZ0ggiJcQ8z7cHfkO7SYWj3zlOBBswly3iw4HRCG+vwNqURFSVCzb/x30bWSHmomcYz/d4C0O4g8tGSGkrILdsP9Eh+VQY1IhKIKW9FwvDCrmj78Ll+j2EHWbEQtWjdXCZ5hqxOf35SCxlzndf4K+wcssSzt/d5VwywMiIYryeKOdORTCTD05m25Pb6Hbr5o0WqNO7MtQ/js4B2Txw+JOYVISHGnqJjzM4ZDcP7MexKewE973LP952NqQfIkWYyluxs/AM2EbMqBqKfJdSJnvAffsJSqxZ7Kk4yIUg6BP9EgadnN8aLnK0dwFaaSmbhUvQ9S7nhuo0i6XXmOW7mk/bxvLSd+sxR/pS+uz7NFpa6NaH8VtMNR12Myu3RdF67gky39/PlsSLvGiC4y0dzPKuoM1u43W/4azv46wAauxMojYij3zDPoLd2+j/5jaObX2F9z94n73z9vDqnO/AzcmsECCyoRVBcJ4375XGU514nyMB4CMy8rWhFZFcikqrQmVVoXPRcc0I0+rhVG0qKTeiMS+2c6f3Nao0nVi1QuKkME4CeT2nEDDOD9PQU2wRbiGkaQlfD/0ZF6sfAsdhyof4UhecwDuhV8i0NhBUZUSIjSQZVCHh63ao/g8WJ5Xo8Zgx26VYs1u4K7+Dul80vbN7k5ucS/+aTlStI1mVepoefbPZFzaL1i/vs2rPKna9/C0fZXzP1yFwUvFYs89md6IHURIn61fF8KlEJAYilou5mXmT/7RG30Y+ePcDVgor2RvxIqdGOulga/XX+bDlGokPZUYjXBpQVOZSfmk/zakSPC2voJUV8WF3KhvX+VH9VjUdxfVIdd3ohA2Prp9QkECP22KqTxXivu8uwwceZcJKX/i7D4k2Jb7tp3BR2vEde481761BZglgTFMl7wYZefHlX0hbdI5T5yCmQIHYGM3NwH8TGAQs/HUh8h5ydEYhYpuFixP7P7pvu86bovx42sqXURPcilncyI7yiQzyXUUQx5jgM52/5+5BYnyWPhv8iDANYvzMteBd5Oyby/14Yf8IchfquR78uL36h+ykrSuWIxNOc3lwO3MKF9PiksQUzzQARALRo++3yzsS85BIysVvIyiGn/zHc1V7hGt8xueS0dyKfNxnIORG5dOEHPXirbpCzv/0J6dT5j58UwGb73xFuN9hRsz9g30t17nuBr9Ymvhc3Js2gz9Hip7jbGkGNlEbboo47ip/Z79Fho/7XPZVn+Y5DfjGl5G3fBdHOhOZXxWDTlKLSq1jiBLeyF3Mir6BuBl6oQQyO0qozoXwCY+fsN3SwLSd6dSFd5OcaWeKC5S0H0en6mTv1L14mFdg6bQQWhUKwCqljD6FzxHcvJqIqsEEjwohrN/X5BfYOHuvhnkhsUR73mSYSwC5+rlE+YJesAaVVsX8MB0rH/qD3/46gfazVqTPWcHb2SI+nWOpK/XBzRCHe+wedHbouzUKv+p0BGMFVPlsAKDGOpfdPW9xI8nAP/Yh0AJWhxhab0DbLUj9hDEBn+Chy6CyRIWyzZ0xigb8OtZxNwrS6UCilNCW7sXZqF9x/IcWH4C/yLnWxLcF0GmyExRdzY4Fm5kY/1gS6l8wz7/kEqrOero8wyjpNed//AZgcHXqNBbdiaVWocMv5zwV7i3gOp3/L5vNZqOgoICEhAREItH/+YT/P9t/wb7/2v+aOScIB0pTBPNT53O9Mpc19+rwsvrhGmwg7rF0F4oSwKTGGHeDVGEkUgcMqYG3k79gtOp1JskVfGPVc/r57whpGoDyQh6X7UpG+TzBYPeveMVNzoE2G13KLuoHRPDzqqVIhE4O6pzIDioNGWSpXDGl+tHdHsQzm0YwbvkxZDWboGsRePWmwqKiVgTCaiPhtw9wM+MmA2rAVxiAZ+lIAlJNCFOTedAVgNF7Ow4H6F1i6OQ4EmEmd1uLae8DciFkK4IoMYkptlh5xqrHYjeSzddcCPmac/XTSW/IwnTTi759O/j0zR1saJVhO/QBUvnjwP0MF8i13mXx8Hc4Wfok134fRmFiPhMmLmFCOAiKdQgEMKnblU5lNWFZzST0OkovXznEOwN+kty/UTTWImkQIx0+kA6tmvHS/nD2Kp4pP3DK+hpP9nOgypagFyswKA1s7YLq+qnMaYGG/CQaG230vNkTS7gP3jUbebFMQID6Nj21v/GgOYhCcSlqo4Q2Sy2T7Jn8EXqD64Z79IozMKs8hYx3I4nvUYib0JlZuU8Lz7o14KeBN/x1PL1qC54BLfzTFMmVYCizQIrtZ2444vi9YjrHZ56n1nqR1+1uuJqcwk0bdXVcXv0JHo6n6f+pFoVExeBDr6K36gCwO2xYuo34F18nUO5KaxLMb4R6q40rS38hUfAUKacFyOQm7hNHjVHFr63uHClKIHBnKppMCV+98hVWsZN6YWMX3G3qz88Zd/Dro2NMcRAFkhvEiIeBFT5sg2aLiEDTMIRd0Pu6DZfqLHx+Hk+17HUuBGs51/gGWg8jmxdu5q3wA1w/0BMXcQ9GKyc5O7zvX5jXdpNxew3mIeMwaQbQ4b4VAI9WD0bsLeR+sDsplyMJyInlo3c+wiq0Mih8G72Q8n61ApvZSqOfMyNvbGc6m7vglVpvOjx24HB3MKIyne9quxAFfk+jex21Vthr7EC2+hvShBAU1IasoZXvV3zPJSMMr4U3NTY6I+CVumaaqtsJUAVQF1hHmgwG+dylq+0ac5v20HKxJ20vBlASWYRDKmP71O0AlF39jNAeZdxvhPZYAVe6rjwa41nZWSRePYm+MYHE/GRctC5ILBKSZDBWVc1L2ybRYdagX+agNKqUviIHRQ+x511NUWyVlxBvc0eT/gCBycQ4oZMy54t2WBdVy/INX5JRPtVZooqAAvsBACziNuZ367h/aBxtUx6ALwgQ4QAG1YCbLp13A87wRXk8Hwoc2AOdNy0xwydtMFkVzdJqDyQdVj5wNzDy/EyaXYto73WH1S4Gjuq+pq0znqT04ZRkzCQ05wgBBVuJGLuV2tJA3P9Jp26ImR0uk2h0u8ukehijhL+6oaNbQZjWFYHMTsjFvuR05tMcWUqdDW49ZqrDIXBgF9kfgXEVYXCms5RDRxegVlpQ+9lRd4bRqa5gjBJ+84PrrSd56VoE/faPpGt8B5uXn6RDfpd/Bn7MsGAv9rebCXcxozCZqDlXwuo1qzk48SB3etxhxMkRpOR6UtzPjEAgJX1Ff9aeXU+vYj8ylHaq70cSWKlEIBdTFlkGDgG9Rc9T3eng05JwBkxoo924lKDb5STkzsP3+XaQwyZTFQXCNmpnnUWiSuWuRyJmSQthdZ+jEUKaDP7R13CmNYAijw0P3/5/ppBZpJ4U1XiiqHfn8KTb1Pvc5BnzADiaDPGvstI2grK5fxApe5Z/kz80xiqCu87R1vcGf0VV0Fu1CplV9Fj3YuQVirLzOHipCKH7SFx7XWNQv2OP7tkYAc82dvPrE7vI0L9EypHzJPbLQeZYyw0T/NYFvaxxBDbrOG7tg6PXXeL9TBTV/8pQ3SRUhli+z04j5Vw8ZFkYOfxvfLuvoZE4qDXDReOPJAnC2dAJhJXxdUYZM9yBxLcfPcMgmSvWbhc6umLoMgoRis6AxI292jrKMssZPMmThlPryShazNant2K1O+ezBL8TRE4v5W7EaGRfjOFyRj5+45Vs7HTq7oVW7KKry5cgx02O/eXHlOV7OdjvItPqodJ7P0ey4Ls2Kfl5BxEJXiJMv4tMNdRaodjSQdaxsVitDspaPXEIHDTLh3OtZiI9/Y/TBZzVelJka8bBvwEvAdgMYDfjkKr5+3BfXLs90KZp+DLjFQptYgTFVvrdP4a7sjcSTzO/PfUbWhctNkUrf3TDH90QEzmcqkghKXYHSQEHmBhYz6YymOcKDks3v53/DLUahpz/C/fGnpyZKMJHBM94dPGB8UcKVcUIHFN53d0J4NisApQaLbvFXeQLICHjBtpoV1Y/sxONj7MCOdcETzXCaWMBz7y8i+/qw4iwm3ilGZb1XkWUvRpUwcgFhURKQCS0YJG7EuFYjnvsSa5X5OMQSEEsRuympD0gkXKfLygLfcBRH3hgcGFWs5Yg7OQoYJawiRkuq5hX9yTTQw/zT46AvC1z8Rohoi5A/WhsvNMKn7QJ2N7ui+XyUDIkCkIaolj47An2KTNZdmErDrMPF1IW4BnoQWVYOzpVPYO6eyHg4ebBYedgwiVuROrZceQNMnO7ORbWCG0tMMiZcetSV0Dg9SOUe7nhFuqGh7eQjz+tZehBOZ2A3WEn4aqGoOJZfPC+E+jwE8EKqzvd99Kw9jVyd9kmno/U4ylopUIxm88v7yAnQwwKiLnvwv17QQSPeYA3zXzXaafF5Sp98sAmFrD22UI+lFQRaIDn1aASQqxlDR2uIBTb6bK8Q6m5mTCzk+rbJK0HQyMvTXiW8+XT2dEg4vDoEY8nk5arCAVBPPf9c+hdVQR3dhLg5s7ZD8X4llrxaPUgotwdc7sRmVGGRmxn78MgqknkxeKqWGqKQ2idNZjhyvVAPeHKBqoBkV3JX7P/wlOQhJtPI0FRTrDvZOkChoT9wRWdkjuaCkJNiXyqr6O8xoeJmnTGq/5mn11EjGQoNW1Q4/EHPOyjb73hhglutQs5KxrIG5XZyBvlBNw/T2HEYBhx8dGrBflu5rdM6FLe5XD0eq4IPDkxuZakvVFY3jPwY9VGhrcMwKT0ZJCHs7oHXTmpbguYOKWdatfnOD/jIFaRlmn25wlRl1GvjeTJ+AI09iaOBXoTHL0TnSWBGssvTPeAdzX1vHM7GA/XblxuRrO7cBDu3zUyTPUTYRLY0haIvmI4NhsUuMhYp4Udww4y73RfbIBE5kGW10iSRnWx7p+nmHa7Fwlj7zopPAOdNHubApowCuBcm4oQi5xFAVBugaE3emHfNpbG6EqOj9xDi+oS4KS/fbbHDra8+wHa5jaGPGHkn7ohTIpbj4uont2Fb/KgJYvmUXOpGXCWhDMzSbi1Cu0363HRaNncCdu74IuYk+xpqyKyzJNcQwK9BlxlUPARnnGDRkc7+9aOIVRbxaSASZSkt8B/BOQAuuwCdDYHcr0Ms9SEm6EM9o2FtM+4aAji8PU4mgSF9LgZQ4uXCyKhELFQiAABQocUob+KT9/+9NH1DgoWkNdWh3H7WKRSFdrxMbwX+SIAZyvmIRPp8e9cQeIxEGakEzzyLD1inbRQCTIYrwIXVQs+IRaK7d5c+7sX4fEVhLm3URsOk1t0KJpdqL8bS0dKLr91wXw3+MDPSfV9U7eDc8IXGO5oIiCnN2/+M50tT2/B/uJTj57xy7Q73HlBx7riH/FWeeGmbgNUPC3vpCs/HhzdCEdP5WaQBpv4M0LEAVD0A08HTeRwH0+Gj/Xjd8US6v3KiDFPoELQQYUBtnoKqTaJeN3Pyguv7sRHDFqbiFfdbSx1E5G9byARFT7IxjYTKIZ5li94LdS5J3lLuJ0JKuiw2ik31nGnMp29c++xKzuVXnKYKpJxQi7DM/M2XfL+HCqaRmZ0NVyYCGHzEMS9w+gRtzjXKqe95ggvdSpYu0hCd/lCtvEHqd3OLIGGgBYMslYkQjGzmr4ixuMGg2qOUWqBST532Lx0NTEuGWSU7uOrEVm4yJt45asBRJZGEtejAtd2d+bq3mW+0LlurvCy8v7TlxibF8RIl18ftbGPCGrCYUlDFDcNXtQ5BAyZfobnUgtY1yrnd60zAUhiaWDgsr0c6z6A27V28jvG8sMTfo+u0y1JpTC3g3bfbno9cYIeMfUc/w8fVCC0Y++p5lv1t/8xqh3sacrkwYtvkSF6hpDCdozBZra6HsdDOIQhCljRKGGz1kRfkYhhN88i6uzHG8Me6n/ZZMT0HYxUWYbbj7OIK4wjtDKUgsQCXii9jVRkoCFhOEU28Lnuz9CioTREQYggmrFeAmob5ehL3sZL3kS/z9347cJEbtbC7If4hdEljp/z/VHeFtE2REDUwL/YI1AwJ3YnX78hZMbtFqomVuFys4Cncp/i0zc/faj1JKQkqoRdM8/xRsRrxN5eRGH3EspS/iLe8Sxb732OSPkWwY3BONRWzEoNWoWGvfmpAKT5nWJB+iv8nvMldc098GyuRxwt5p5dwXPH8unfx0ZJRjEmhYbknAjWlMqpnrqFvsF7ifK4Cc3PkXTrHFqxL5fThLyikAJmyPgWwufzp9tmfHtlY3ZL5Eb4fG4A7/tXILF3UrPlDrP/SaYp2kpaThotXi00hHbz4aW/8FPn8n7fBXzTDq2XG9l3YTThXx5kv9hZsmrz3caHAjXf6JQ0lG3FXSumxtiXoQr4OqSVvCodZ3ZNRpNVxDeZer5XQE/KyRHDcT3MC2hjeGgZn13yoDB8P+ZgZyD66w5QmpS8EfM+O1Vt2AVm9mj/YoEa4r2clVvltLJDmsypkb8QWf8mQ3LX8vHwxXCyNyOEfQmSbCTDFwxqIVfdr1EWpmTbw6jnWT2cX/YG5qiLMM35P19rPldSnDSjO/PlHP5kIe4D2/GxNBMke8BX3QPI1RYxVDeJH6sO09ypo9mnnYohF4n3LHf2owy6Xdv4vH4gbaZuNCHOqmut0JsQeQdHdVAlruBe9/e8cf0+UeJQKiPymB8YDScyYMQV8O7DpmYPbh8fS2d5EAavSIIH3OecAToNYlQ6BTaTlZG/Z2E3pPDti87vq8UGn2f3JvaOP97qQOonneBACPzaocZFBLNcIbzkGGqrhOEmBQDRinZ8LgfiELVzo3YsvUJOkRyQzN7IPfykboLKX/i2GL4FBoamUnKvhChTFLOyB1GQUMBbtnB6Bh7krgmkZe9iHx/M6I/SaZSLmd39BHqlnlojROlDWZAzmzbZLbameNAa3sW9lHs0ik081wSFZhDKnPNW0ZEiJA9awd8pY3JCB/9kxDHasRFYxDOqPWzRPQYPHEKHk94WO7b/YInwoZ3Eh4myL/0+iPCbyVhDmpDU6qkZ8gceTS8iREza3TQmHp5IWa8yvNb25pVrqfynlWYo6PYLYPyYRC7V2bjv8w7z9Zkg7sYmjyPizl486gtwrB/nTEJHiM6s5qtOB/941fFGkAKHA3Y8XUe1915i2t5i7cheyOS17Dg0E7tFxpXET3GTNQM/c75yFuuubsPN3kDU0U34T7jGuWF5vBVykzc8jgPQ5fkkXyX8gaoVDhkdXB1sYFXK9kfP3GwIQe3jTlGWgrg/u/DxWgifOas7tWYPLA8BeQR2jL5htBLGvwyDIqGI5Jxk4ouVGIKsOBxiRt5t4YUXbRwu3s/uxtMkGBXo2wyMPdyLylAvbmbeJEsqoV/Cp9yqSMHo5oO9opO+1/qSn5hPp6aTe43DQfmAj3q3cU5pRNIZQKBQiJJOGnWD2HDL6YPTB2L65BLd4UGQxEp022YEMtDkRaCWelLvc4PZhc+zY3sic1//g6jUEsK7R5Bc+TOh9iu8tfQ455rfISevAalejkymedQuHoIWdBP/QS/1Z/WpJCwRHTwhXIlVMoTclFxSyyW4JLnwwbsfEGWT8U60BbHZyo64jex6eyLtHi2I3WWI7CBCxI68d5mV9CFrBo9Eb3HjqK2G0fuMJAdOZdpvqVD1Kt0mDdsSdqH1thBXP5ApF6eQ21OC0hSBXlaOq8TIOQ8VB8r7syG5gQeRtxhPGn3lsM0Prrv9gqo1Hq/WWALr43nxzjhiR9xx7vWl7mA3Ed3xB9n7u7H16c3dYW8RoJrHk6RhMUlI6pZj7DSiDVZwT3oPjTYLhwMaBv3Dq1/Nw6swnO6nurn31hOYp8WTs/A8orYfkakfL+YOB0Rn/4FVqkKn9settYLoa9to90/AEd/7EeBndPUmPyOMxJsVZEU0Ejx0C8uqvkLc9l/NPpvN9n8+6P8h+y/Y91/7Xzd3XR+29qjjgNCLMx+YcYhlWPy16Mp6wM/OjCi1ELocD9Fwi4NVm5bRN7MQZWILW3wBDGzqNGGTtiKqNRL3IIyrSWVc7fiBJeXPIsTOmvRRaEwe/GQXIZA4J55zevB3BbNDTLYygdI5P6KvWkyQbQyf5y9j03sW3AT1YDMS1aRg4KUZSIO7iHsQR3F0Mbct0GGVsvzUV7Qa0hCMnUSnrxd2FwPVFihJ24B79Sw+/n4941sW8+efwwgMasYjGhA4Z8dZCVch4SpDa15mzj8jMfaE0Xm+5GX3ILFfOTuFRu7IZah7zqfZ3BNwLtg9ZPDAauPrVW+gbVLw85Kf6KF5XKXxnWg0AM1mKTqBGHd3A0uiy/AQAbVHIHA896yufDdnPx3mICYFZnDU7zzfCa8C0GSMcTa3wcqCjX25l6Ji/9T9tBd9w+3mZWx91p/LY2Wcb+jvpOWbVsRgnxu0G3150DSC1vA4TGEenIweQZP4Nh8LxlJqVLK6FaqN3nwrquKBSYF40GWmxj3mRp/RAMfUNnwKU1AaRCxzq+QVsYpkGXTaQWsHN6mG7AP98cwR4/jIjsEBe02LeFooAKseLUpyLZApEnFo3F50ssZHQF9gTSCKzmYssQYCC8/Syz6fNj8PtvMbb7jD6SBYWG3h9IAWtgw5xVTbi5yVfsPVhiB6do4gxN6BxK5A66p99MzXjFDdGka6xkgZjcjtXg9/ESCwS1jfYUFmcSfXP5Hc5nTOOeZgVqpRSlypscJX7aDycuF8SA6XjXDPBApLEGZLE++3wZ8WXy66p9E77UPud8aicPcg7ewWary2sssPHBI7NxRuiFQyWtLl3ArZj13oBC22+8GtM8m8vGkCZpcOFGYFVpGNP4fPAMCtQIxRchuRTYSyrhe5p7JY2H8jk6Kg0QrftktY7JuITA4asYFGt93/4/v1EA7g9KVu3Lt6E3TxPrLWfux+YjfPusHykN8os4xmUK8H7HbtxjFgCX+5/0WAeRDkrAa3OCaJH2DTwMIGxyOe/8Gui2mWZlMUU4SXZTZStZwJh8chtolJEooxOK4xv07KoA5Pikxyyh6lFgkQ2ZQcCdZT0+7G+yUBuEjE3OtRgF7YSLVoA0ubDtNqgxW1iUxrvkAvg5jPdjzLoIE5pI98h3MGJ+B8VWhD9vxWjK09GXNsDPdS74ErXDBAnElDTmcozd8OwqufA0GBjvnF8/n9qd95p9XBSNdY/Gv9mPnbJPSWatKzI2n0VjBq7B1ibsVTsXES5qVdiDJkdPjFEXVzJxIz7GswU9TpifJCBsIkK64iBTa7F3e8W7jz0O/a3hJH8aZJdC/15e6wiWg8jMgE08DhhLiW/zMazc007j5nILg9mNAgITVAow26bGJ8HmwgoKoLXWQgL5bN54P313DdCC82w6TIKK6HVVGwaCOjek7FUeTsj5iuY3D9Ck3yfFJuu+Nm8mbRGBcag/zo0HQAYJaasUlcUavh8peXyd1dyIVhR0iYYUKmgCO/jaGvRYVwhQgXQxxymRCdo4lZSR/zskaAR7wDS31/8g+LCKwNRGytp90G92lkSNyP/OQDT1V6MPirmQS7GfDok0cPmfN7fbetA/N7TzJGKGG04BWu9bkGPWFm4ofozBpaNL15YfwqXIcmIbKr6HCpQyQAHBZw2Jka8hK/XpuOr1sCYw9WUOcvRT28hoC6H7iSBcOOnMSmMxJ58whIYoE0AEyKJK7WJBESAvawcHrFX/8f34avLYAISR1moYHON7Yj9DCh1t/lQ7Urn1enUFT2CdJtfyEYIGb1iwfY1x1OWZEcmQCUSGkM9uPc0BIGx0TSZ4BzXs5zJhyyru0cDeYoXl63krzEPPJnnOS6QU2v/7j/vKhb+H57hQW769lw5ClKnvBkyd9ZzFa3ccS3HFk4iMw4Kf8cAhz6Wsh7ixKZjAOBGSSE2xn98hncNF3kSkOptEKlFWLuBhBbVETDVBXa574jItE5f69yh8+Bz9rgmk5MUEMNR07PZejYfD72ustf3bBTa6IzJxKr0EZVn28ZGHGIWMct3r6yA5HAys/j4ExBLOpSf7JcRCxXBDAm4FfQWiHmOQRWAbGFsXi2erFv4GLe7mqky+50ZoUO527VjoDyiPKHreB8NpkADicf4mbdOE42HySv/AanrGq6RZ0sU4PO0cHbTZMx1mhIUloIdhUQG3+P1zpBUwrTaGNLUQaHXRwoP3yXvVkFDJ99Cl2NL5agMkJ9ZlCWqsAjsB1H8A4UD73rvx+yDsrtXfRSwA4HxBwdwFMZhUT4fO6k5XNPI8W2lkk5/flTZOeS+hWKFdXsHLyGihuJbEi1EtLXqY3gEAho8dbS5dbF6FqQ144m+rwVZbSVvFApjQ4FUrEb9sCvmRh2jyPtgdyPk2LRJDNmVz3hMZM4OPkgHXbodAj5uCCWftnJeEY34l6jJtCupa/PNe7d/AlwoI1oosO/AbvFmR4tEoiYoIKDAbDLfpFGq4xmm56qJBNzxv+KtyvQ8ztQJ4LdSsaQOdz39MPj9jBW7lvJG798hvzBAZ4VhVB6eDxaSz0ucc9xIHEyAK5CmOYq4MUR9/goNQ+74jyHW6r5TSZgJdDD/wTeyipCTArsFjHFPTs42Oc7bmve5M5tNTmp5wmPPsmPczvpNns/+hZe1Th1ggBudV8i4ZN7vH/yS6Ztn8alfpdoS6thgZuTptZh1RPpfo38pnR09uuUlWjomdANQ0+BVz9GHR5JcUgQf9vF5KbkMDqgmY6Q5exM78e6V9cx5kQCxpGdvPXZW5QnFFG0tBifrHncNkQRv3gqN9tUxDc+z77L8xg9fC/+vkfY6wo3XO9yw9pNh8mfgTIDdruAMrfxJHhfJsitkAXtMYwWwIZiO6HXBpIXbGLugDI+8YIrxj8JMUxmjO0BRyu9oDONhthilml0LAO2WC9g83gP+zawiuVsevYXpJ4/Q8sB0FeB/xgEAhcAHNjROuCKUQ76KgJEBrabnf+vi+vgpyWfkK2MZMLo29D1gKunfuQ5zSXKbI/1dqWWJlYPHsvugjdp9dASEZNDe+V4zvvCRX0du5scqEXgLrJT0+8WxQPO41k2mJvJl3lFns6NjnCGtFroqX8HffOnxJZEoapLIsGznZnRfZkZDS81g1wkQ2y/g8IgIqi0DmnxWIyD7yO1dIMyCADLmR6U7R3MpXnnKRzl1D/a1AllmjZKEgoYrlbg+FfA56EJZZ7YbyURWx3M/SHHudL1Lu/kJfFxUh7VnfFk107iqL8eZHpGeunpiilFJHEmSyxsgtliIS1vvkqKXwcuE1zoF51NjtssNlbvYE83+LsOQqaV49ZlJL0pnYSgMRAMY6N/wENeD8KPmHsmgficGF7NTcHU6w5ZSU49NNzTqbG04HFoBKEyGTvnXKDTJZ8nrCOx+E9j67Q+xFk3Y3827X+801a3OvxFsP1aCmJ/PT0GLuZkaQ7BKem4hq9ma+YXHCmZwamqp/CJj+S0oRdPZ6wHnOwShwNgWoeaO0MM+DYX8PfvY6mcnEty/xb6ulZyodfPRBb1ou79D+h5sy9nvniKKS8fwM2zi2caocFkZqxvCC21A3GV+nA7/TZaF+0j6k+Ae4KPuGV2x6CCFwcNJkNcCBen8aaijXVbn8EytAnHYBDbXElVjWZQhBJuPs8kr3CEA59BEvoWtzLzMEkaKTM9rgBZuWM6yf8k4L3AwSApzFUlsMq7AICfqgZw+76vszIrwsA00wj8wk89OtfP5QaRp0MQW1VMzl5Hl7qDZsM0dmlhlxamieRcC7BRGXyIiUdcuLMvmQMfSdkcfh26i0DiRt+Ae9j8ofcZCSKHEolIgEndi83XbhJ3bhgNI+oRWQSIRCJEIiG9Ao4wL3UVO294IW1zxaTWYpRV0213IsJ3GkYilTVwtc9Z8pLyCO0h4pfgu1yUz8FX9CfTXZx7pS/aoMMs4+i+ibgPLCQ14xZqEeQa4GufUiKCrpFy6hcGjl3sbCdggcaZIKARmp36wYUjaTi3k47oTBCHP2qXVI/RnBw2GlVQGddjI+mQwCsPt3Irvl2BXZgH66P+xxicnfgRd8UFPMCpP3cjfBudMdn0NERz2gCqUgioC+CtrbOoGGJh77S9dLt2s1H+Kfeq3qJLG8NM3xpc9R6s8G0krjCOen9nBfULWQvwUtTQ9CCQA5cTqK0Nx73SnV3RAi4qw/nknI0W7/eQ6ZbhdWIT+clDgIGoZU2Id/mCezqC9KusT7iFNCqXJN0XvO1XiFfbMKb4PEFVRwEhltPYrXZaE9VcCv0Fq9iK0CFF4BChddVSFdaAyFVJwoUvsHr40LFwIOuvbaGyI4mVFhmmrQvJm3CBfN9vETamMD/VzvSEz2kz+HHytyHYq+7h4gIB+bc411OFRe6MewjEIjr8nFVf/c+l4+UTyiuTsyAdbHYhlTdrGD3NxPtVg5i0M4Ntxkxe//5ryF4EgZNoC27n9thbTD0+g/c+eJeA57cjsrSBVceKQdksURfRQzWdyQcnczPjNt+O9KdU8APf3PmAIYcTELVpSAtwpTMqjkCVCZHOykAFdNhh9TdTUDT5cHjeISyCNpbSG5HA6XMdErdyYfEGxERgKvIhVithRUQtR3QgBC4L32HV1Qf4dU/CQ9ePWYnwdKEbVlE3kQ2v4yZTozSrUXRUE//2Gv4aX4p7upTL1U/Qa3zSo7lDoZczT7KFRMstNnRCt+EKQoETqHBgx08EfmKnM7axExY3wdgeNxB7BbM6dwLRlhJIb+eaAXor4AJe5A8+iyiwB0+PLcDTtYPNpkiecYOvIr9h4baJhJ/oQe7Cg/w44Q6Kh0DJymbY2ODOc9c+xf16Ntkj3SlPKCY8sYKnfWBFE3h4hJPoGAbnpBi9g2lOmM7wWOd+nLOj4Ikuzra7YWt3paS/ijHx95ingfN6sN1ORrJ5Ei1B5fhljudYl7OCP1kK89zgdq9obvq/zBz/Gxx+uGdcpDnN2y1wpHIcFtNxjOUqzB4jGXh+IKlpnYjMdZgQMr8smVcuLiDsphRBBHRZBVzynQbFexAgYFH6Is6/e57gmmBsIhtdgV3sa+iBi89BPvKCrYY9LFBf5/KtwXw3bAvdbk4tQ5nAqTeecfc11KU3iJ+cRGmfPPar9gPwnTds/rsHZ1DDIsj+JhvN1TIY4qRJnusGpiYPxGYzumYd/p8XMjB5IBcGOVmAJGYJ/vX+oO7Ef68/T+Q/wa6Zu7ihLWOxwzmXuVrsJIlsdMUm88Ok4QCkNAoROMS0eraS1zMP1wBXzNUd/F+tt3UnTUJwCYHDcV8jlV1miuMyhKyhULGQmuB9mBQeKLVKBpQsokmsItbrGkvdHZTlr8YVp06eUWHBJDchRIRG3oiPqo5lX/6Gy6SzTJrvBOFeO3UZjbwRu0PMxeBVFI8pZ0i6DxjslP1bRBk8nWav9bxzzk71vXLCBEruhVpJtzfwbz1iZehhZilS0URlcqIzAn+F7tH7SKVQ49eboOrbBDb4ovWrosktnzK9P5CGUCAkvDychFwl990aEWZXIRfEMrLJk5FqKHmQQe9DvakZWkNyXgJ2oYWbmTfJdGvgqdR3mJ0kZf7hNoIqNzHy1Ejq/evpfEjRfLVuKMp8CWKbkhe+n0ZUcCMpH+5gNxMBiHS/RZgmh0ZjBKGfvUDd4FsMfPYw/koIEgtpxRl30wV40d03iAb3ZqIoISvgFK/3nY2YBfi4lCBr68L/4Ab6Z5Than4OHu6MfWhk0fRrvF6TRMybU7mQUEjWjBEIsg8ATh8XnNrbc9a+ylO9s/lzxXEifPKIG5VM8JC53Nieg39RG9+vGo1UZOJ48RLKH2ho3qFA/cxVrve67pR9cSQD8NnlPdz3WYLOu5TUi0OJz0mlKKmUdpdrXB78Ijk6Tzb/8CLteVLKXj9LnfIK43kJqwPabSBVitl87Ci6Q2exCmvwashk7PBcODMM4l+H1E94viGScIEFq8BEt9mTVpsnh0P+4fsti+m7eR750fnYHQ+TZx3OvaSX/yBmemzg9vHbdFZ2Muj9QXhFSlAZ7vFBsICtpsd6zA6HA2VnPV3ekRRnzSXqxl9oGotwba9GN643AoMedWMNBlcfKmN8sMx8mvcDnOea2sSI2/5vn9V/7f9F+y/Y91/7X7dSv6+ouvAek4VGGsb24Lo+kM4WbwKCHutI7POHjRUOIn5eTlW8GYNFhNkqpF/3RsossKQJnlEU8qerCz2iaon4cTUjxV6sOVdIva0dpSmSzOtR5O8ZiGhIIaP/3MKyp0pQCFeTUQ1T3Ecjl0rYo4UsHzvC2KEkeF/E7fRDssdx91mtqOJM/nTa0yWsWb0GcFIMvOlezd2sAiTR/R9tWaVCBUqJEqlIyu6uZDZpSzCL7VSe6MfJ2CJGPXGVj71srNRAbKUzgKpqFRBzqS9PZOygLVhDHiBz3OH4wVYEDgedYR0Ua72pcVtCUNcGZjWAUJyOVOOCTGagwb+B9aGP2/X5iBOYBTDsehwp/8wjZ4GapZ1x7Mr6HM5PgOjlmKXPcE/YhlKkosR0mXK/r5nZvICNyfWMbZzOH0WJGJqOcD/FSkmUUy/HuVBImH/AWUUXMXQrv8/7HZVvPO+kTwFg/oEa6rVRSP+jOlMghBxBLdfaIFDvyfBdQ/CNtDD0iT9JVUO1eB5anZO67OOSHoz4uSch/WtZOUDAUKWOpVYY+ZDV6oHnTE5Le+DVp52YK/eZ6JLO09Hr4AGQ9C5ih5X0kmjUbiLyYioxiOsePceg84MIKS9G8uR08gcsxipVoZetB6CyLYHfjw8lye8J/gl0VhmIH1Ea2Jnq05PgpZfJts/Dq3kjRrmReUFa+shhqeUEhVevY/nnIplhrfSuWs69KQ24GP7kduQMVMZoLtwPp+ZYJI0jotDHj6WvEvQtE3jNepi3fGOo7HCj5nQv8G7DtaWMxshsmm3wJI1wsg/zh3WwIhlaO4bT4+h2JNp3mfHsh+DawTdTLMycGIPXPgEPRC7YRc6M7FtGuOPRyNU+VxjoksQbb77BnbQ7kOTUioiQOFh+aCr1d2I5/EY9dZlf8Wywc2lY1QphEjtLln7OtcanyDVN5XcPJydCohSeVKgJlowl+08XpBIB+6ZseKRtt6kLLuvkvDMgnWTXYoQZ1awROwMKQoEI8j6AoMlsaZuD6GUv4jLESCQ25jTPoXKJgU5TOzXhVQTgi9RVxs6ZO5l4aCJBycU4HHDWZKZ78WYKtbFMOfcy7+5+n6qlwWR7ComSQIimimc3L6Ulqpv8mQ7chKAUSXnGDZap4b1CAaV5iahcpUi1KsxGKTKhg1EqnNpkeneKvG7Sr74XWdezqAmqoS7QOY56aT/giNcnVE/bQ4z/FAKu2fBt9MXxEMC3O+x8FOlD3sAGPOKmseHJH2lS3GCLK9SH1aMdfRaRRzw2kxWx0YBNCCI7XJR1czK0CelrXzAzfDVDPuuBpqU/7R99SS85HGuJpkbjQDn2LO3+81mXNJpw9xy+a/dD74AFblAa1sH1ehtBue5knlxI9tJmwnmVLL4irnMkX434jTtlvuTIAhC7OyMzDyzwoAPGyINItU1Cq7Di6enFlDPf0ilXIHjCAZouhNcEjDk+hoiyCNQv+XFzUOAjUKVtnIQSl9dJCgddk47GG9WMNY3lymznhq5x0gnudfdiXIOQV76Yja1PfzrH2kmTgYfIwU+VA8izP6BwiJYdQ9azwOtTPqiAhgjIN8NPHdBgkRGR/oBYtYFgYzOFFniuCcptbkjiHqA0hePSaUagUCEQwJjIn2nShVIt+OjhJOTAIXA68x3yKBjvpG0b9OBXGlW+NFujUNRrsIi9adJkYtaNo6igGzdDKlXtN1E33qe6sQhIg5rDGJrbSDpdgyHUl23TLlJaq+HbQDNTS704bq9it8SFvufHsNUljK9TmvnaAA0yT2ymMMIaV2Lw8KcyaQz1/pdIqoRAYSITPI9xIAw6rAV8YThKi8XEdteFaJtEfOnzODMsQdrMMLWCyx5mXG/2xVITSvK72+DsaBjnpIa92v0+JQVteJRcx7fwIumja8A1Cm1zPaHqXByOZPKHdHNvlFNTcvmBOex3KUIrTGGPew5PmPwJEU1EZTEhFUkfEkqBl6MFua4JtSSTC8JRXBCVMJbTpMthiRre29Ef72Z/PDq3cacxHPcnn+KnO6OoeLhZ/Xb5j+AQMv7Wz8xJeAO/xmqGhX9JTuNQIJZBNctRX7hKh5eZ/BYpyQMqiY3tASIlIkMjs1/9kyNdEgZvXYyi5UUGfvoNf/nDyYo3uLhvODUXnkDgED2iwnrKFcaooP2WhI48M7I4EwG/zyLk6f2Ywu6x/OabhApdGNpUSYdewbRVJSR7/kmRahXvXrtPp7wYX5OCyj/H4zOwkvsxRVxX1+MqCMdn7Hk8/auZ5RGLWtbIOu14rrXLmORuQSS2U2qB+a4QHPMaC98YhkKnR5N3iRJXA6kDcqDlCkQtotERRNnOKBxJTYwtkdHgp6Tl1RY+K5vOtZ4Hkds2EHhzCD1O/EkPenFqeCdnNJfJ7Ixi3D05zTIZI3JSiCgdxoCdW1jVqca9FOQundjn7CLMGIinh/mRZp+3CNyENu71LOd05mFiTc8wI2g7vp6NWBovk5+SSGjTYrTRdv7shu2bBiA1CxD1FVFjdVY/dSq8WVPei+AzEOEXxBe1O9kfrmBHRjVU7oS4lSxOKWJ7WBGXlUsZGLXnESDylouW725mYAvvxhIYTo5vDgBxEvje28EXbQ6+1MFrKhV9G1ZSamsGj224ytp5u/8MRv81lgdXkzm/6h7PekGboIJTbue5YwKDopBzveHTJj32A0/SQ1aOfIYzI/q8Hr5qTOOI4ypqv/X4zxTR5tFGomQBWQKY7QqdIjeG74qlx+0OxrQFsFs1haItWmb7OYNEOpEbIct2cCt3NlWqcvbLKjjt9ysxbjLEIX24NsiFmEAXbqffpsm3k1dOXueX6WCsKmJYqxmZXcWH+SXkHo7E2OMq2fZUtvtBrETHB0hY1ajD+M0MPjZIePf79eAGc/c2MyZsEk3r5pCWVk/4XX9iJlxkvNJZ5bDbD27n7CTveDLxbi306prMloW/ctmgo58C2sWljPAeROWJG3RrXNAPrONGCHCyt/OjnFBClOVnFkjucrB4NAP/XknGhMtwsg9GvYwrG8fgEarDOLgbvbyUSrMSSn8FZTCORinR372AtZ+dzS7ziHa5i497Eu0GX2YkfMb8v0/zl8GbNPcrfPJFEg1DHzB2wtNMdAHXUhNlYWaKLDDN0Io4Ip9nlPmghMPn9XgHGxnRvZmuwlhiyiNw826HiXcBWO8NNfpiXrzXk0kb+mNN0GNdsQWlqx467kHex5D0DmsFIqJjCzHKTIythWFK5/g/H17CH0+U8GrkRvq/H02XSsrZoWcZqICI6t+5MuUQ3XoNsxqTmLx7BmPmH8WWKEZr9kCAAIVegdQspSi9gZOZp+gSrWVv3nfM9KzAXD+Am8Hl6NUyjkWtJjBAy4yuPCwOOKaHflIdl5dvIaBlBhklf5CcJEIggH7BewjT5AIf4dPgS0puCrUBdTybVuHsp8CJ4D8SATs4Mv4IocpU2t266VZ24qnPQ1n4MQ7/JbRqQwjTW4h9EEurZyuCUAGF5mZaRLBu5Tq8zH1YUtbAn7ef552kI2zNdOqcp4ccYvxbu/mmopL9XYMx20EiFPBck4MnXaFalE257xlEwlmUToEBPUO43DKHj2XniROokOCC2OqGa7cr/vX+vGV6ix+L3wIgWuBLuiaLxT+NoMs7kPXLFnAkAKQ3l0OfLQAUmBZxpdqBS9fn7P82i5gV9bixF7lKzOUn9pIWtYrbJS+jNV4kRJXF5+M+gY4V6P9+mydT3mM3r8JD4FYsEPOixjlOVmaPoCXIC013DWGHRnOxh4Ir8haOmprYXTWI9nFXENq7mH48lCvF3oyY4wT7eleDrasPM/b3QmyxcLXXMV4J1DKxeRPe/vB9h5OC9V82gXr/Fu6m6jGJfWFcAUhcEdiFGB5iyclFh/FqKSfk3u/IzCXsvf0nCReKqL1Wzcq18ymNqkM5XMmJqlmUtqcTeudnRtzsSZ2/mQE3BmCIc/o223I+wSqpIqHPFqotAm7rBxBQDl9kTCZC8iff+8A7lWEU/zAc/wA5Lrdy2WpN4usM575gUh38pB3OH2tHEt3DxOpUFat9dOhtUjrtZgoMCpLUvVj0Wy6SuhbcrCdp9zhDu3Yd7i7O6j4hRvxcarGJOp0MA8Bnwz7jzdNvUhFWgbs+HlW9kR63elAaWUqnppMZiZ8wR2jC72QqlySQFN3B1mC44tjFhw99m8HWsSj95URrBvFXvFPDrFUSQYIUvk/IZdbuccQfyKR1yj52zdhFq2crANvKpuLmUFB8/xh+V3pwf/kVbglv4ad/jwqbjXdaYLBvKG+OnkKbvysdknT6MJdRWc45lPY7iIQCzDIzZpkZd5EaN30aIXZ/fO71Z9vKy7xwupywwWG0Hc/mlvLWQ7dS/JBuExwCO3Y71EaGEhxYj4dxL+4KWNRjJeGiMs5Mu8j99p7M35SJ7slbTB/jrIC/1JTJ/hYbHsYaHMlzqVJ1YVAYCBbZGRv9A9Ga9cSPr+ada9+SO+MI16KczAF36kew+vxx5r3+Lr8lfUqOJgi7TYa7uPWRr8jd1/nDvZ7fgSqVCHNCMZN89UiuTIHIhYj82nmQWkOy1zS2z9mO2VVAuPUQ4RFwqGwObRfTCasI4/77X7Fm5A40Yk+GaOFkoBM4+zHlLm7dnrR5dWAQNyESijilh7hKcLe60h5QQEBDKMt/Wo7HiOsQUcveh9qO20oHo/rTQm2sFxUR1/i58Uesom6uB0Oh8CyuzXKm1V2msCkLs7c/RcZkbt+cgN0hJlMAC8RpDNr3OtXacKrKyrkjT8GceYnztgC+zV1CtXIAGruFFdeW0vrSLnJkCha3OAMJx8YdI47J9P69LznVrsS+GMu4GlAao4gxjefu4A+JMHmyqLgPPmo176c1YW9L4pfSKIwh/tzs3YJEJUUhhFxLBJNqyii3AvIyHCIxdrGM1HOp7DiRwVubnBXWx/UwF/Cy3cC/9DLN9kxavWLRCJ3Ui0QvBeCriHq0L33N+Iar9Ip3JtieM0CGbzNlCSL6+alRDI3mQpuzMj9WCq+5w+aAq1xqn46lxciJb2aR1juflH65XDNCaVcga265UZsXSadCz9CzQznvXcdLoXDFYKfc5xuaj7yLpUGObMdMVhkh5HAICsEeFvv5k9F1CfuEy2SbQ/nx1xM4HLD58/6EPORgHueTg6RYiN1gR6V3YeS1C5zJGIIx3sbpoJ2cuDkIl6b76BrSMIkeVw191Q6dJ/sQ9DDZaPgXw/ntY6dcTZAY5rhC/uZ5WGTtsAocYsGjZGYAdaeaBVsWoB9YjBfBqMudDBaNNgsbu5xAqNeYi2wddJFJZY8r3QUIEThEVIZV0hzdjG+yL7qDNwgrD3Nq0D80ocOK0GrHbpMgQ02FFeY2qtmeJkNc8iZ/j/sDib4Hz90ezLA/gijpOYM70RK+vbIR/59rOaM9A72nEF0xCs+2UFT2wSw8tOrR9ffPA039fQTYmdx3Hf1C9rLm/BHa1HU09r+AIXQWWT9FU++igKcug8OCQ+JJaf6vpF/8GZGLgtzA+7QaxXzltYRXMxYiKvTjl1vr0St7094rCw/5EcBZPBGoysHH6y1mnv8C15Ii6mJ3ktVczR8ZpbzIUZ5KfYrMjX1Z/543fnWtyO6eRNFT8+h5N2XW8by2lKWxXnz89lfYRE4Q/VzFk/QK2k+051XuBS3EJ2wIf0o30uLV8nBtGIBQ3MWaH6YSWxiLtE8R4QF1OIRxBLsV8GTKu6jFD2i77snezmFUDLqId3QzBq9pKFr2MtunhOJ7/Qj/8w3K+xuxR4XSP+SHR89ltLrQJO/PK1s34ic24Tq0CPfoQtrt7TzEnagmiowqsGh9CJ2zG4mmm26vdQjsh3j+u+cxujRj6Q4hqCaIypBKCvzquBLxNie2x3LVtQcb+vugffc7/MvMSB+O4TS/U7xW2I+EBCMalYozw84A8EppAB5/F9FmCGRPZj1SEbw6qIm9vTfg0zUNN0MLx/VQqpNSKJ2NMrIBh+DMw7Ep4LoJMqrhk+gkhEYDbi2lFKVGsfOp1zjk8ORW6GynXIJQxG1/I0eW/cQT5f2JLDqJiykMu8SDm6oOBH3vsChtEQEbLvNyycv8vvC2U3P29ssMGpJE6qVn8E70Jii+G44mILsWzYdfvo/k5VYeYrCAgJLMOdhFztigTSR71O4OB4hbGoi+8RftfnEE5dawMiIUh38lzzcKsP8vVrT915z2X7Dvv/a/ZgKBEJXKBb28mCdrjbwamMzSMbexFM1inf0U0wMfZ1b2VYDDs5W7QgHNUjOfPvcrn0QDduhTDU02mFAwk3Vb5cSN1iOOzEUsdDAgbSK93ItYc+0HfrQp0PS/SERhBIF1JfjP3sV0T6ejAQL6S6Yh+8sHaS8RfyRFMcH3YUq+azTIvckONWD4eg1a4RZw+tncDQGV0Mbqubswid9EvOcc6eevsGDyNabMtULhd6iMcPtWGkHREr5b8R1WkQPP1iT0FjitdeHpdg3V5b48CLaQ8tk6onwMmE0SrvhXsVzayqwbO5Aau7jkdo6l8gaCum4A0E8BuTYvtk1ZCg4IbBrLN9YWvn9YXfL8sRzWTYcVCWWU6aRoLw1HVaxig38DS1JGQd1xRKHziG/2Bb0HvvcEDG8fTsc0VywqJXn3a5h4YCTtgxrojnuO3ARnwPzV6H2MzFrLyl0XwGAl0qsvLqK36aEJ5FSZDyMitvBq3yf48PQM6JQSWObFM9IEohMuo5c7AUOr3YBvoy+dPs381AkndSLW+18mXubM8vpdYOKvWX8RWjOAy0vex2f5XraF5D4aDzqLlvLIJro0UnruTMbD1Qcm3oE+f4DElQHCu4T9MRdjWjOXxv+HGjNwtc9VpDEj6CMVY1D707s3HLBeojoMDnUVUX7iCQTJjWTZZOxw9MY/bBtVcnjZr44etg+I0tzgbs4knv/hee6m3qX/sgPMcYWRMS18l9+FpKseo0GMwPFvAMDByUAo05bxlqUWxTPnMbmdxiDrIl376aNMIoFAwEflMxh4xBuVpoHYjm3kB59HI4R13oClgxGVLly2aPnGR0iHbzh+0mQa0rzZ8M+r3ApdTpN+BBrCETqyH71vRjVEdM8kIHgmAdFu7O25lZqgWr62bGS4/VmuRTZwMrGUVoUOq8SXUyItS7SvcqLqK772hgiJjUi385y1mDA2DHl03USxlDd9Ojmpv09V4hi8vCAn9blHv982wW2MrBL9q8biwHpby8hTI6nKEMO4fBC7MMbswS7fI2g0sZjqD6DuVHNZ+9Oj6zgcdhwOKI4uxlXryoNKH2ZIIE4KF+oH4ykOJzp0AHe8c4hXjSCzJIzpVdVMiNjKmJmn2UYAk473Z6pIyPQVz/CvOExKt4gee2Zwb0wNq19cj7sllP1GN9yEXTzVCOGivnQ2K/DyXcL6F+ehdWlCDFii4Zr6BdaIAshLziNAPomq6TIOjf+CvnJY7QkubdeZEH6e4PEiWhOSaGtpp0PWwZha6O/WwKa+DbwR9jFt2aWk/ePcyOUk56BX6hnqZqPVZsUutnM/vQK7vp3Lvs5nflZdzMSinhzudZ7nfReR+6A3vnEllBqDkUoaqDZ4ccvraaoHJNOuOcVd4XEUmh44Op0BdqFDQkT/exSkQ2H9GCokx/Ho7k+bq5O+TIiVs5kbOFs6Bw2ehF0/R2tgCmL3KeAHSq7h2vseDyLqsdkd3Ih2Ar9DFJDg0kElYLfDyC9Hcv9gCbGFsbRd7Yk1+BYr+5awuzQGoVxKh08srl5enLK9w3cPczqe/SUOsd1O2tKnsVb0YUBKFGvFIBbAL/Vq1GtWEpPoyS8T30Hp1UmTywvUWgv4sRM+8uqm//JDrMhZQGj5Jsb3hc5OeOVkNnaHiOcHHeCT7kCulPqRenQwNeHzSFvY89EYC258A8Gm51HFwoZlZ+hwyWaueC7dPY/w/fJTpFR8TOvIVD54fw1+pAHvwq0VZOkqyQt/imselSSfUCGreBL5t99zLLIKYTHkls1AeFJCSM84lJFKrGIrFk0/9t1/nazAQxwTVNA9+g7aOivROyfT2VeNR7yTCvSI3sbTsU/hl5bLU99PwlyVzK8bwliR9xpGB2RJB7IpWMzY186xfst0ZCUqHHYBdBU8eq+q7mSKrxfgUJaRMrKQJI+DEHsSSflU/Pf9ya1CJYx8XFVx3+yAGd3seX05z+7LgL5mimT9GBhxkACxgw3SISwKPQvRk7ndGcxfTaf5Wt6ETNiDLZ1n6aOw83oLxFWEEVwdjP7JU8yXt/Cgw8ANIzyvgQd6V4Rm6LLb0dTf59fjn7Bk8UWWZy7nYOGLCATrGdgvB5fkAl6vDadBvYEC6Sx2+sdCdzF+qnA0rnLqWydz2N2EQdTK3Ie61+HeNewOqkUkkvLemve4m3abg5MP0lfhpDB68e8n8ahqRjdEgGz+Xq4F1hDcvAB55acYfPax5MNVrGtxZVfpLfbnLWPY7IH460tQuRXzl9mG4KnfUKvTueu5j40+MFIN602jeF2iwtZWQMkxD6LNZnLvvUX8SzuJz7zPTz5w0wgykYyQogKk+g7WrdxCc+LDBBSbCTryqBOHsGPxh6htAwnWNNHqXsbwbU5wKSE/AY1Zi7i/k0JF1VmPReJETtfGlpK16QBfN+goyp5Cj5AqlLYGGt0KwQ4bvc0s0kD/wlZ+XfT7o77+yBMWq2FSpZ5K7AR1TSfAXAVU81OXmbtxhdwJXsM3OwfQdro38UIHRpkBoVDIbRPMbYDXQ+ORmq/S51om3X4WPDQBBPqmQ/lQMNRBwhtEFCTSJM1nSpInO3M3onP8yvQBt6k//DXSZ2LImDOd0n2P1+dKKyxvAu+G3xh/aRh+k90Y9rMbKf3LsceKEQqsRHveYk+0BZ1ER7+OWUyw5BAYeYAPopz0ia80Q6FexM1uBWPKXHmgjyPxqVIExc6AWh+7Cz1v98QkV5CX5Jz7+rjN4P2Lx/nJ2JMjU8VILBIUBgV3MhrxC73CbG0e/LkZJtdwzWUo7945RqBDxL/AggABaX5p9FKlUewKIvf7HJp0CInVg7lF1XjdmEQfWS+mXmrD192MMb2FO519mOu1kjdq5xJ4Q8fQptXEHzuMi8KVexEXyZA/rpTLDDzCVn0y/gkFVIaJeHryATK8ndna+7Rw1QA5yg6Yvx1bwytUea6l0auF/g+Z3/318cyvf4lpCy6w/sFbtOml7G4XMsPdCFIPkPtS132UnP2DUfvkIHXvJMLVef9fulJIu5dGvdiMoLqDgXcH0p1hhZJfwCuLC+aeyMUmtCIFdf940mAcSNRqTzbeXk+foP34dLnjdfEO3b1EVATUUikTEmySYRc6W++X69F0V/jQM62Z4xGPx8LeJ5ScrHuekX2O8efsYwgLZjA04DwOhAiwc7gpgRSRBr2rifMDz9PtGc9fnpWgDedD93JnVRVQGlvL7cSzJNwfRFFOJMejSjFEgrvQmZRkd9gJL/FEr3SCfYMVEFS1idJ0eKvJSEeXjKkv7CIwtAE7Et4bNIHStnTO7RxBzaUe/PH6FRCDt6CdtqQKALbZc3hq+i28LT0R2gRMUwoBO7qHoE+suJNkNVzsMmIXS+Ehccnnl3chFpr57RnIGnWNOwOvcNhi457KzHdpW+mV4FQyH6r6h0+GlvNSuwceBRGozGaUAjvFyp64P7uB30omkdrUh9k7ZnN14G0EfcysKDlBpgy+Czazs6kT6/ErhBcU42K745wT2uCm0cQYUSgX7JuZt20gz4ZGsGzdF+y4s4DfRV1MsQSQlTOE+5oQXux7jZ4BJ3Ct72Z2K7gY/PnUvJLu0hpuRZfy2WJnYLVPfRIfmPIItGWgMZ0la/gVci2DASfoKjC3P+rzmuLjZBy9jlFux4Y/K5pH89u0Bcz8ayWnEnJJ8pcTeEhMVt4EPkj/gPzWBfQMGMzXJbORfzoC+v/GovyZ5CXmIJojosQCe7VgC1bSruyLSLiL3tm9aYjIoa9rE8fN0OV2hqUd79GiC+LEyJdoG/0XUtmvfJC/CKMDkmxKFJn+6IxqLg1YxSQfE12SKKa5lDDNBZqFYtZ0t6PInsoniRVMHXubJuEQUMc75wZtPU+fGIuoXYMj7D5RPlXIzM69kEdwOzeGLmLZZDU7v65HJh1DpDqeBi206IP4etRzHIuqp7F9NsPODCNfnYPAwwaIcBPI2RkIx6tmsqpKh7shETKhrDuEwTVV6NojGNDtSqfFwqYFm3g3RP+onS3AffE02r0e4Nfkiu2TTzi9sIphyWsJFcHqqlRmiiQo84vwrM1Ft/YL9vjZaOt4ClycrDEullzeDJvFP1UzkO1ZijIrjzemGvlMm8XhiYfJKF5EcFEzEw9PZPuc7XRqOnnh+D3G9ZiF9vfJhCR3YLnTh7Vl43nlh7V4iyBDBgtjyxg67AfW1+TDwylQYSnD5IAfOqBQY6Z5wAViM1PYbf7j0TutcXsPAHmaN460K8QExtDa2opflYAaK3zSDgKdDndZAy6Dr1F6tSeqIzF4Z53FMSYHgU2PvO0yNzzlfF7lwsAKDX0OTmLi638j67oGwBnf2cBVpA7XR/eV2N3AISQpN4lpe6ehXXGf7dOX8aVfO7FaMPkNJsX3HC/8U0wbblijDnDZuIc+AQY6jN58dOEgV40Orszsh9IYyczGb+kUD+T9gT9SfryaZVnPYzZKWfv8SlLDWjg/rDd+YU5wfM/9N7E7RLTKQzmiA18c3OhTTJPmFq+KQvCwVYGxGc8KLwL296XM38Ha6TuY4gc+7lMQ1B/n26Yguu2FeLuE4Cr6GLVQxlWFij6G4Xw7fCQjRLOp1NcQKzKQJANoJUwMy5rgvhnupd9FbHNBblIjtUkJc9znTohTX3GsTMdgmYx1bUb8JpwlNLqBCskywiw/kds4CBd7ESpa8am8QczdIsIDLnFJDR5CUIr11HY9oKXSFbOqjlGLKxkZuZklRwoRC80IHVHEm+/xIMcPcZiRqwPN/KWyEaeHiWYJHu3utBqtCIQOxBYJDruAXeXTeTPkG151d1abza9poSD9OXThVoKl+Q971EGKSy3DNHC8tQRRezENNgUpIgfdmnyGd+SRlzGEzggNXx56kWV7Spn0Yi3nAn9lRYsVa90ilj3xPN3jZMy9ImSxoh07Ql5osrPWC2qteSR47mTU2iss37sL/4Jt/Fq4ljdXJYHC75E/Ac6KmuMly9jCdQLEUBNUS1vEKlRhQDMPk9qsHNNBfAVccTuNymMlTa1P0nw3mu0KE5Kw6Zw1vA9ev2IZHIIkNQ/90H78If8DF+t0znbARh/QunZx+uXXMLtEodqdjN2gJ8nmgacI1rvUYarfSWS/Tm4bzI/G/sKQy4/+VogsrP76FQyRHdyvyqHvibPkhHrzR2AD1RI7M6ZsIGHhBXSDVmH+5AKTLk6iYOxJlqgNrH9mNzdETr06/3R/uv19ABirAqtDwOVe2YQKp6PyVvHp0sdSBQA6lY7CtHgiIlNxWVLDOyfewatrGC1up0mTwZ2HjOMT60DS1Up8QTzl4eXgcFb2AY8kDLp+LGC0fTQ/L/v50fU12X8TVHCT/KjBTNwYyW9PBqHPmgx33ySFf6M8dgJ6BqAbOBqDyA9/+X2ez1rCuvMfEjIwhFoTDDxSSLtfAmUZj3X5PGpyKDvtgnvFFoQOG/1C9gLw/qDxLDApqeuCl/cdpfbKDSwCPZm5MGnYLZ4aFI3IUUzjiPn4+guZ/Od2vKXv8GrCGjjxA1K//Ry87UZM9WXECRHcl/R93GAZ3yPBgSFpII3qNMTWw9iUeqRCJ7VrmCaMgOQwfhRCp7eW+3PDOOM5l7ld49judhSv8DruCopxj3AHoRS7yJlA2GKV8tr5/VTJBuDX0IgtQUBtkBNYT7bNp7ZbhEpRhSSynQpFJT1GvEal4QSJnrNxk1XTO2g/vxX0o2rXMNr7NqEY6MHy5NN0qudzvGIvqz3hQUA3F4I7KZO4sm5U1qNX+jb7V06VLeTpwXmoGnfgXRbB528f5StRJ5flYY+OMwtVpP2wkE5fMa882UWKRwWtck/kQgft7u0gDUVXomPWzlmcHnkSzz73CNE8z6+y36lPms/xsm3cG3+dPQFXaDG54SXrwuaQIPDvYnfsQeZ49IGHVWyZ2sO8MeEwhwtX0HIhlYoDA9HMq8Pqb0WtFlDp9gOzGwDqyYpzBkKGnRyEvDkTn82B/2OMe6e/Q2GonlqtL81KPTpjgJORI/d9iHoWP7EZvQDMpedxvxpA3JgXaGMbFu82rgzNJrBXIC6ySMwSG4lVT2K3OaDoe9wAtxGXwDWEPb8PZ7oYXDTdFCblERIQ+ej+DgdE3fwLiUmHQyCgLH063Z4hGFUeBAEWdx/KUychNXYRWPiArR89w9inj+KzdRztA1rpVvP/aRMKhcTGxiL8lyv3/2X7L9j3X/tfM4fD+QHUeeygygiXSnOZcMuKr58Qqf/ffNPhzLI+rYcd3XDTpKNp2Q942yeBAz5tgxcC0nEX3iFRCt3uT9IYVoWZEt66FoqnbybvhB1mhBKC+j/HyPL+3Aq7hK6ohqfsXvQTF/BVOyxXg4ekmEzDLpoLHTi65EyX92PM/DPgA7jFg8yTn9ojKROWskBsIbwsnGbvZr5q1xIhcQKGyz21uGo0aD1CEcnEYKyH/I+INwzF8+Bkykd30DbPual9WSzkxy44XfoEv9SUIzg4iKIlLbxq62KKDCQyC0e89BjNkNbvFlXN8WTlBFBRkwTjbjxqv+GqGEZK4ZJOiP+aXmiDI+GzeQBUdiYhEMCpsGa0Efcx/z0TuQHcFWUw9i44HLhf28Br/4ylos4LracQz+Y0nl+8Gc/SNn72VNM49SCjE19Gdv0xYGY3+aO3tBHR/BnSXC/clr5ASMtC4uLg20ujadKGIVNUMbnmBsX54bgHROPZlsr0EZ/QTw4fe4LW5EbWt3/wY/GXuPxqwRpeg98z5wBnQPiOXUR5eiGtllEkZuWh8erk0kM9kyfqoa3tb1KzpnO605fhy28jrnuo51C9B8LnUi/wwe3Jg5QpElnwywzaNS38NecvwCkcn+SrxmaxI7KYuWr6CYEIaqzQZof7/RaSmKkg5NwJTt7N5J3hHxIkdoqhf52bTtXm1ThiBVwdeJ4Gvwa+64B+cmi2QZ9+p9AldPGrOYa76leJlwzC9WYCEQ5PLLYgtOoKmj2aSMr1w73dDX1cOwmKTjYHQIstF1fv9dwe2EJGz0Z2Fy2iMrCY8ykZoLsJvsNoO13HkNxwqmeW8O3SechtXvyY0Mz6rZ/R61Ykre41OP7OZkjRfQqeE2IX2XETQhIBiIzJyKStHJlwBIFDTLDr02zO+Yt4oYYlkfsgEgbr5vKa2AuRo5bTAmdf/N4FU37tj8TqQrfvMvrc78PVvle52TCIZ/LfZGhvK+NGfUKubiICuwSH8LFgN4DIYeCZG31Q5SRiT9XS92pfWhLvPQqEKNQ66nqPxT/EleIR+ziufwz0ZV3Lovf1fLpXDQAB/PLsL3wVYqSXHLb6wqv7NuGpsaNZGkVpQQKB3tDZdof7wa/RLoLbadCqG096tg8tlsdlpr5l8HNIMzEv7OKcIJUuO9isYnQ2V+6Zu7jdGczuDjO3fupBxzBvjvbRYhVbEQG7usHc3ZNxvieZ5gO7Oh0Mlr2K9fKLCAdKyZRBo6WD72pquZYt4ZkkkOodyO1yzgmMdNrBTQgOAcj93THL3ZAauwiuDuZNLxurVDJO16jZrbdRmFpFszCXF5rgWx/Y16mi3CwHGbTr6rizPoDdic+TO7Oae/YbHK16h8jmJU66v8STZGuy6SdLR97qzioXT9qtXSRUOtsg7GGFm8AhZpIKPvAEizGfrfveoHq/HNeQIir6zkYncn3E5R8gauXn0fn80BxMS0Ed446M427aXb7pW0vng05+rTiBPWIoIGXSyeV8OPhl+p8YiXjmLVwEUqZb9yPzklKaOZOUJNDS9KhP6gLqkNiVpDXHYcnzIXKiFo0I1rXD7qY05rmG4aKKpMUm4I1WWJuU/OhcH5GFeClIhP8zg6zV4KRxczWUE3G+P+ab8XR5hSIWRRIgdkDxz+Ddn7+VH9AZ3IrO00p40wuY2utJ9AjHqCuhPuAKDrOA4P+rt5T5Ezcvvk/oy7/zR9V4wv/xwUf5+CB7NEzpaILh6/AJa+H1L19H3OsuwuVWYj2vMSziN0xdkYwNL2VEeR/S7o3ibmIFLzYpmftws/DDrcl0XBmMp0sLHmI7i4yv8aOUh5SuAlqUM3nz+ExqErYz6MknuSzwZsT0x216tu19+t5Vc7H/ZWJmnEKpBLz6sFs7gM6QGpo9PRhV+gYhNbEcHbqSaM9YkLiwwK2Auq7BlGq1KHecw9inCp8JRtprn+U8vuSq/ibHVo3i/k1O7Uxl6nN7eKavndByMBuC+eKlM5xpC2e3UcJXSbn/P/b+K7rJ61v/Rz/qkiVL7r13GxdsMDa9995baCEkIQkkQBqkkISQCkkgoSShhZbQe++92IALxbj33mRbltXPhdhk73H+4/zPxTm/ffH/zjE8fCG9S+9a73rXWnPOZz4Pcfps/jZ2ZpFTBlssAj7W+XGl2Y2c7L2U17WT5/g2rg0bGBu5jiZ+IdDtEXGhh2ltf5cqryqEEiDtTQicgbznHmbtLgSTmZZhMzgXdYlNzXChHTx88jkyVc67Metp9oqk2sueOF1Ua6dXEkz+EKlJSnfHJei7ZPOFBk6WbkNTsxqtwQl1AShMjix4aqa1yhPLTDWdlI0cDYQPa3T8KK0i1BJNDzmkyMFmFaC91J391XWYuoxFmXYbSYKSxH4P0bg1v3wO7xR25qSxBlVfJVXaeDrVRLJdvZvXEu4hKtkLIXPxVYWCvCcSQ1dchq/jO69yFtfBrQ5IvZuKV1MbDnN8eNb7daodd5MeeR9zGIgEBzib/zoCmRDH8Ndw8FpPicsiNMJ3CBJDfkkMa/+OxDswDMJBZJMzOfBtzjSspcEC38sNNNdEsbnCgWNP5yCeGMZj8Sb+aIFiUxN3bB3Eu9VxYvQFGtwq2MC/+nVWm433/DU8XvQU4/XR+GVeos+kv6HlOST+AAIB6jxf+t9JwTCgjGphAHUWNWKffuwuG8DjOgG95WC88TWLKxaz/t311FpgixaW55mJvrUFQ+xkHGPsovFCgZlpB5uQiAzsjvWCkCzev7qY3VdnMf+LLfiFlzPIARqs0PLtm0QgImf4ezSrfybd1hmwJ388JK3EnxtKVWDNy2SfSCAmvMieDBVY63AffIdL/e5QVzOWnC6P/33vn61FKPCj++3uKMyOdCsdSLJnM3HJH0LE1Zdfs72glnYTWVkzOBVJWyXObY9IKfZFfxUudq3h7IINKIQQFgHDmmNZVJBApWstrl5ibvZbz2S/wpftOcurkStjuDD4Cu5Sf/pTw3GRkmDngcwruESbFSJkEnKC8+iMAqFkLh2KT15eH9n0Hs3SUhqCSrBktzF71Qq+mnaayduOgdgBJCq2GLrzYO52RIJwigMvck/uxamULWiuriXnu9UUNq0lNr+FAVcGoI2+ADYz1N1iYUg5ca+X0NdhIbbqudhMFhCKGf7mNGprpxFVtIcKXRHNsgB2zd6Fr7U7NdpgQo1PGaOE2w8iSb3bhYg8M99VGXlv7XrkSjslfaUuFmU+WIDJNV9zq1HPqkHpbPjTlbsV4/hpJCR51qEdlUNeSQBD5QK+cnpB4RswFYChKh06EYRe7InUAruXfcOiOuhWEcDEG7Fokxr4feFOjC+4hjZp4bWBG3h24X0qGhTIncUslD3l5r9yZVS1heId+4xGWQd+uS5Max7Kp6/88PLz0w3BdHV6QIvVxIKf32CLdwOvr9zO00DQWaHeeoemW51pqNOTp3idVoMbA6Tf0GL4l3r2e18dB1shO2Mp2honfNX+cGMyhMyhw6agwgwmm4BZ21NxCYihU7I9ubbUGX4RWBG6yDk8/jBtHgLiX9AvhklhTJMXaVoxt/scYsPwx3gbc9EUQIvVTonmumsgwvAG/GwSEhQiUgvj+MaynrSyYfT2W0DJxf549WojyescNhv4yYtYIYFtBiuBVf/w8HgQgYP6Afb7ecX7Ma8AV9sOYxDPgB6BNNz2Y8KhCQzvd5XauYde9nmwx2IykxI5FNJAduxl+qjngM9QCoxK1EKwWq08793E5cTD2AQ2LC+oqdJsj/EN0qFyEiGx2QPlYqGYUzo4pYPKrh/R1uHJ5prDLPg5BXcne7XBAAXEht2gd90cruRM4ZjSSLO8GZtAxBglGG3QZGpn1dypXCl6hXVGG8vqISHlEtazX9An4G+EAhtl1R5Y7sSjDqxilhpqLWnwjxRSdyBw7sEwsyMFbRr2jXwK/kdpc1jMjIAYupadJatlDA4eKoo72/W5/+u8ZbWJuOvkTnm4LxaNE84f/8iOoBY+yWxkSqffUYhb+Wb1GoylZob65+OQl8CsSR/zhUXCNT2skIYy4qsr/Jq/mG87/cZMNVSKu7KxOp13neBw22Iev3GPQTdW0zetBZvRHsS/1+qEsy4Vi0WHoZORUp8oigw21jTBPPm/L0GmrpTiX6aid+zAEwmDHWyYnq4hqHYTLoX/IDf5EjdQy491i6nwtQd9K1oj+eP+XzgOTMMnoh9FWVtQuTRwUfwKgxx2s9cL9j2L4vMr7yEOcuf9H98nLzyPd8OXgzP0KwedVwtVofd4r3095uM6zg09R7vy30SmRWTFJDEg0otQtzjyRtA5rijgiM4OqvyuMox6dTFv9engoGAfPzcpaXKKAwEImy5SfTee2D0jMA/W4RtWgZeLHTGxvdAJx0wljfmNeF0rYVH+Iv6e9aK6DxEt6hZyI0rx1jgCNn5qhr5ebxOgPsTX149S2x6KziDA4ibhwpAL1CiG8/CI/bzm5XaY19Vw12bAZgOZ2ZPJofP5rKyZ5W39yAz8FTfvCnSOMSQUbuF72TyG98xDZ9QQ7XYLgUDAlH1TEOHJ7SEa7vs/RGIphdQdEDKHhTsH0jcrCrnGiAlILRFjHPkxwtvjCL45g4+uzSTi+zieNHXFTQQNEth1bxWhzo+40P9vTuctZPN9F2b9NZ0Br2r5w+8079ZBtdl+1l1db6Dsu7Eo2qV0+iOTzjLY4gH6BgvPly4nsVsOyW/8Q5IctBYF0w42oTM5sWbmbMZv2sWIvR8yuKScYLU9uxtWAqFVscxLCuLTRW/iph1Mg86DzOKh/D4qEoDLtqfU+C1FN7OSdFs1dyO+YYHa7qt/JSqh7JO1OLbuw+pcQ0bvX3FwSELT0Mq3L1Q2NmwYi8ZRRVHAZVoddEj1DTwJgKyOUqxVORg2zaM6vJbwk8m0dcnhxuDveb80CtAjElgQCcy4NgfhXldMZ8sRvKRmjvnAHeld0i1ltMp0xKSaeedFFWOwxM4w8Y+tmQ0tFlptjTTIDhNd6EF+/Y9w3QG6/gquyWwUOFL/vB81xy5SrhBxehl2yQ5snMQe/8n6ZwUjTEMpGGcix3yeirYg1l+Ig6vxtG0o4OrqVRzzFrG/zchMNUxSWZHmF4MTFI06TvmRH7GZBTT4b2S6oz0g/J4zfK9v5fmcSraJ70HZOd6z2OMus3ouI01vIlLkRf65fNQBzjwvG0Nv/+M0WuCJNghNaiZ1ng54dQugKqwXbao1zKqBGHdnnre3oBIKSSo6iuV5K4kZiTwefJl7HVDmUUuLtPXlO+zZNhjcoNAiJ93sxN3ud5GZZvJ/ZcFenWn1m4LAl5eafQKbhEQZPHyR6Ct5FojX7TFI654yNXsqv7/+u72yDzH+pf4kpydTNaIK8exALpXZ9U2/7v81B54e4ELkp8RahpPspaHGtx6j1Ihc6PDvnpWZhDA/HvVXalrj42kvEdKo9+ZQxTDyB7Xzxuxwbv8BlRF96VC5IxYa6Ru4lybVTTjjzbZP2jkz/Doa2b9yNlse/sSBrLVcvqCnj/Ac39jMeNmguhJ2/lPOqj3wxSQFMbEfcL/jK1rdO9E/8veX11tMFdQrdpJUPASr6RBaDxtZDYlEu+Qgyf4SNNGYPabTbPalJug8aW5b6SOc9vJ6XW0b8rYOSr3SuR8+F4C9NY/QCHayqXo2gWIrVouVkMJ46lyzaHRtpEVzBs/2VKKeR9I5I5H6yRYkRglmsRlntZRfr/1Botd5ds0Yyl/P/uBJexIXnyURNhae1AUz9h8reZ6fo31jM5GqSSRXv8dmQ19GRzlTbILXa6Cv4xs0D3JBn5/Lz4uWMOGtwwRGlxDldpsLhfNRWOsRj7nINa37S2p2oeDfBIsIKwqDjHaTldNPVpPQeyL+V+QUdIUuHmDN68UrvjJODz+NPKSMS37AwwX0FKfiuWMeutomBJ463HzqKWn34H7eDEaEb+bH4Dbu2yAPE1P2TaHWo5aWCVcBGB35K8Nz+uDuVYmvMYbUjJVEdY2hosMey3UQwPK+Q7jQDoq7BxA1VRPv3J0gMUxUgbuxmonuB8hQmyl53odBeW+h8ZJCQDAYGsGi57JTHjtvprDfown9q9e5KNMwWn+NllB4pdy+ptp6jSTfdSRqA4ANQ5ffqH/0FQdu/8k7I1O4s2oEgk55FE2q4+9JB/kg4YN/51OHiXa1F5q6AgQ2G6EPD7B26Vqc9VOYbNuCUN+GqqmMev9ElPX3cGpo52yRK4qkJ7Q72l/C/6dr9kn/O9Xd/2H7T7LvP/a/ZsHBVlpb214ia2xAUNoh5AoBMxNepbfAAv5XGegA4yqh7QUi1sUiZWnGGDwDKyjtto6cDjtK5heJlLLY4Yja1zFvxzzuTilmskzAJyY7aHaarDPteg1pEQ/pFf4cAFcRvO8Mtw0ZiBx6M+zzw5zImobvuSYM+hdlyv4TAOjRvpig7LH4u3oxZ+8cjo49yheOGQgtQiYdnAQx+QgS55Fn7MxQd8AxDEY+w792EsnL9vCNtBfaBjUSqRkPP/vmExhs5RdFLsZXSzDKh0K9M8dUrox1y2exE/zYBDNnbGZTVShXCj2ZpCqhVeBCraGRD13guvEK0qup1FU4kR8Qi8nZ++X4qqUNCARubGx0oVlcz/AEA6snf4KLrBl0xaAMxCTzorHXPQoa3GhITKWzZAOBYvvGEOSh5ZL0MWK1A0E3dpHSlMK91Hvsy/iVwxYnfhkRxqUYV7RpQ0g6dRGLegJLu39LmTaGfc/fJLn/RHJjsygQu/G+eznlwjgchdlES+Gh0UadLhCjRYF/mT8tarujlmOEWVlTWRt7m17OEOcUSHNQIO1+NqAUgDc14GzIYuz5IagfRsKHjrwVs8ze6YDJYLORLwhhR9gfJAnjiXTWolVrX46L2OKIyKbAUN1E4rnfSNNlYx0E3cthoMLM3KlJnG2bhFv8Kp47NrPHfQQHi7shyIPIuhKmP0gjX27jaVcNLepn7HDxIVBSSWIpnPZeRbJHC6ueLuSAu4pEh7sEBs0nqtREt/y1fJE6lCqLkKqn4UQ/C6d8WjU43WSAAs7RTIDresRxXhA+mHSNXW8kPmEZNNyDuM/x/W00XdK6YerfAQ4v0H/NT4gwOTDw8kCavSrRi1owv6j6EAG1wVDevoFtJxSY+g/Av34eYqGIJNNievsU8M6ZTOhsL60IyVCivPAOC1b9wbdhkG2AxXUw43kkYrOU9mozblUxFA4op5ALFKov4G+ayFeRh3i75ApLvl1ETlQOp0eeZo0bvOIIhvZ8ZoocuKnVoB+kZHPYl7gpeoC+GkQKIsoG8duIWlY/LkEldEdm9MYgtaOcJCYJEqMFs9HK8m+WIzPKeBaVz4klu9FaoVflWhpy/DAuWIZZaMKCBJvAwmQVdDR3R5u5mYRgDYUTrnGrTk9n4XyU+q3orFDVvoYaUS6dHGNwffKUAUE60vQRbGqrwK9xGhrXRkKSOkiXynFp0KDVGDDIDcwviqJbRxBr3ARcWv4+iUlhmCIKcao2sVcHzoXwbPgkSiqz8L6VSVVEL+ZtGkONZz8mf7aWsKJg/tnfj5pBeXSZmMr5iaGM2vMI52ZnRI0uFBR6kb5+CsFvB2NztS96v2rtf6Djp6o4bFd7oBsXy/WeP1Pr1cIoyRGc079EZnannvm418ho7bCXCqTIy1jbz45SvEwdGWd7ElgSiDZASmLdeNJ7OyMWXEUuALNAwMnGaXj5P0Hh68j9+N100Ij0QTa03kbgVMrXp4+iDoiltbSR5PRkSgNK2dJSgeOdMIIf36OtVy/K79Vy+5dHODU7UzVvBwDfrvgYmW0Xjf1Hca3TQNLEUiRISJZBVzlMfOMUO7WOcCSTsPTbyOq9QGlP4vRwamD8+ye507KC6Zd64eeow6lPM73k8LsH/NGm5sDvgwnWu+PSfh2rbyAC10DcHUowW6VIg2eyN+gPrI5CUNrBEKL2q5C2ELr+xraWR5yau5Xk1q+JuT4Zi1xJSNtluDObz+ZCchl8KTtJcGEwUo2dagaf4RyS5fFdThqhRgdcRz3ms6D/qdmX4lTDLbkKi6wVdZ80+keXIzGWUasL5MMLN9Gpf+Hu/qEouhazYttqzurjKCzrTb04k/DKTzlifkCkwAnBlH2s7Gx3BL9xVvDQpKfQICOt7izG5gOY5fmkymGwqs4euH9hy4LPoVli5jdjM2ub4L6hOx8/+QZ3UQtvTz9ED+UrBO7IoMuTZs71EeMgABrSaXADp2/WUtb2DuHVuTQHlNOq6s6Ojl/ZUZBMm+g56gZfUuQ2vCOtaFzt6+sYJZxvCSTFMZPm2hS22h69vJfNwU84pYM7uhHU/RaD0Szj8IS/6JB3MEX0EWvv7MZdWcrEEfBZySBKqoMQaO6yUCqjn/ieXQMu2P7sAjNPoNDWcLGXmL2tkF7bg7SGMMrdduKIDZtITEHXqdzrandeTYDZBrN9dBSZddisUG+BfJM9ibBqlDctBicy7vuhMHpgy9tOQLWEsrqbPFCe55N6qDWLGOEAta0Cpp+ajrXTUzoSc3HxauCSwoGc4PU8/+Ikw01bed2nEB8n+xr2cxMYyt6nseoYKye/wcdPxjOxcCjTXU8jCFwEQZNBE8sgh2v09mjjaH4KQ1wbSZLDbLU92Xdq5Cn6WD6gy4szkGfrdFxaf+dCUykVRe9w4d4MEnq20N4yjp+ej2NJ/zIGOrzDIW/44vlA2m464i9Ook9lH+rc6xAGiTiqg6s1qWiez6DuRCM+PoUIKkvwGHCIZBl8YI+FI0zM4mGXNEQW+7wXCkQkyuxUyGJRE5NdzpMU3IlftL0YN2QTvULvgyIMHOwOXrDNiluTM803HhMmvYiqXzb1F7sisa5D0dKBvtaBZocKbKp/0eJ+Yngn5WMONS/A5OvCxYShLB1kp9WWiAy0GZ3taAmBDX1AAPVxHlxyX8br+Y585gLjlPA88jkiqwqhsJ7Lmr/4oKkGW7i9/aut1wn+uI0vijz56LuPuNP9DoJYcHcoQSnVIjTJOeYD65rgm1oLTi+qrlBHg+8IBhvvY77Tl0aFIzZlMVazEIWhDIPZgO3pbeKvpiEdMZrX/voFia8B14UfYZG4UxW4neFDl3M7ZyICm/Wlzg+Ah7IEb1Ue5TGDcYyGBRoTE1QCbnv/Qu4ZPXM7f4yocDeBZauo1NRRINjCvBJfsmN80YZC73pv2tosaIR2lLnM7EmwxIHCIPvZanW9lD+1er43HCcyuBE/syv7O5WBvgpcEgEQiASUBJXg1OaFFbjToYDQ+Qy/s4gINxOftdhoT9Cw130Fvf18IHQ+1N2kusYBISWAjXvxG7DYTLRq5xNXOBbCFnC36zHm97tHz2crqAyGm7p81tW/wgqXU4RI4NOArWRL2vFpLaZeUsgl12aun7yP1SaiRZmAQvAJGq2GJq0LEodW5E3nmNM5E71ZhdU6iI9cytCIYOXWWEKer6R9448oI7uD7wgAFtW5UfaoKycCUvliwPv8GAKX20Hd5E7l/W60V7VgFQmwiO2BwjoLmL2H89eGWwQXBVPzHVSaYUuTF2rrP+w63xeAk11E4PWEKScXIE6PwjzlJjKFkaAiiHU5wvd3+nMGCcqEHCI8/g1kXtPDM6Jp2tWPTnoFnYAL4+5BIrgqyhELTUAwg5+50VYnZ+SxKPSecfhW/20HtLn34K6+D4++MxAscKQ+oAsCzypM/vO4bHRg26bbxNW7YeslJishC40tgLe6/oh31VE0Qvhl9WyEbnpMrz8lMjiXapsPKkElv3vBiUZwbnLG0G4ie9lvfPLCpXiz62JmGdW8VzSCtg9+o/7peL6atZKQmQ0IU26w2g2CJLkUNP0Gvcqobglhzs+P+XNhIlK5iWvtkNPhi7Gtig+NR+lhrKdvdjz3u93/H8E4J99G/F4/QGFRJFs9IER2FfQ17NOUcmz5+7Sl1NPcs4NCT3si/JVdvckNMLLYP5rXZj9jgNMXHI/dg1HcQIxw8st2/8hwwJTuhz65iHvCGYz3+AmA/g7wzCDgr22LMJSaSFpiYLbECbnlDite9H2rsILPckdS2i5h/JmxtDpqEYwSsP7eNtbf28Zfo+BW8BjyXs/BtyiUST94EveqhZVuJ+DBewiGZpL+yj4eG6GqaDwbtfCpJIQmbSYpvsd4tzUdi6mY/yrvFAjsesOTY77hvXMPKW+JJmjEQW64VaO2+mC2yenpfxC5WMdF42bU8lJEA44yadhNVNYm2uV2UKmXopYuzrl4mgLJaBcyXWXDx5zO1y+SHa7CGhRORn4NgRWv2Uk0LVYRRc2deUvlQLv2Ab/Pmc6XxUoe3I/jB59K5jn4vxzTOpuCw8NPYVDYWNcvm55KCMyH0pBZvHJoH3KTDwrXXjR4u79MxjnLqzCa/WlVxOPqD2muH3JCfJ+xTGSvF/xRE8D5kkAE2ddQmcqwudfRom6hRDmFQN1+tFbYFfyMADmsP1JPQlYClwfYqc8OeYNKAPHtYnJaZ3F8lxtjHo9h0e6vGGy0+/paUSl/NymY8PfrSMaJKAopwtkQhKB0H6hCsboP4E1FBRF9rjFr/CucOT8MreknYuMK+PHYDKae7k3FzAqsItsLvT57/EJqdsOg7sztQTKGhPoxds0wGlxrqI1NRtl+D6NFjrf2Poq7aRTJBRBr999CnB8xLmotbYIWomtCWV/mjvDpNbyfZ9BRPx+dyYnHtf2oCd7Gmdm7iC77kQm71tKe4YisRxNfDxiEWGikoCYFgSWYNIOIkWdncT6ngZGzT0LOz+A9lPKEcr784ksG3dnCxLPDaOr5AIEmBnrswWXHPbQyV5yMbiSeXk17dGecx7txqGgOx3KWUNcWxJPKFBwa0wjKD2WUwO5PyATgWDeTLWdVWCsDyIu4ibvVTJ10MgfqzjPZuYnlFfFURhnQSgbxybE/WdL7Uzp7/cX+BGdsgGfDELQdcLPzLvR9qngoBl4slfXqSwgErwGgaDNS+aSWKxG5vO2rQi7WYRS4kWUo4JwkH50snz4KWOrkQIgDLG2Qk93RyDiblT4Ob3Ds4gTihmlYFBFOntHOCPVGWifEgW04d15MH2kzHfopxMggRmZiZUktpooeNPlqSXnnH5xddBzR5dBNqedPNRy6tICmcx20d/YmdlwrXoE1L9+L/WUDOdN4G5dyMQ4pWRxz1BHv3pdlXOPtWlD5JJFtMPC0To17cCu3u+7gI28xNLTAk2+gzxG8ivwp+LsrRqcOZJFZPA2ErxrAlBdCYMYGmh+PQVlsItAcSKtYSJnlPL4SI4f96lEMO8Nk9/dZph2Mj+QCbzmd5YYeTv8r24bQKETWIaR70EFOhtpBDQFFMKY6gBaFF/+8Pg/S7tGmikJvy+FAG6TalHTeGkTu6Vz2NO2hdEwZ9bbDbOwjY6bGwsbnEzjT71t6BfRiYZKaq/3u0eZor/rq6d+Ts4/LsFY4EhHdive8vizt0h2TxESdBcIQ8/hF2ftf/f8i/l4NJz6CzU0dlOGCRliNFSuGVgMpd1Oo8q6iNNAeE/os+m/22JcAjE+NhBSEoHOVYHlxXr7QDj9diCb1nitFg324FrGcZqdmBJV2Gk/nJmfis+JpLmnG2kVFniIPH1tXPunzCSariS9rvqTZuwcfj+3MFvMGOknhT9lakLrQalOgKQgiKCuIuyV32ezWBwdVKOtMbzPS/zTfC05T3jIDiKEgNhizsBUXWxXvpc4DIHdJOH86hlNuqKZWXv7y+fxy2kr58ypuYiPlRdXhf9lME9wDRh02kNiwF4cEfyyRC/BbBIQNB++htN3L58/J77B5yH2UKz8l1lpLvOsLf8x7IsVuQ7ipWYjF5ofKZvcZhS/2n8zqTE4vOkvs1Q7yZooQmUVYhVYmq9RdWlEAAQAASURBVG0Mkl7kYm44A2+nUNKlhOl7+/Gos4Zj444xI/gY70TM4JtCRzb2vM20Wx/zyd+fsGPODkLH2ujvCX4uUWy58hOlxlRUdw8y3+cQXq19gIUAWEUCqr2rCTFa8b5+msSeF3HPz2WpM6gLwMkyhWelbiD5kVpVKxU2BT5WOUNCt/GoaihOpib+SqplYbmepA1TUcXmIptUB9gP/2HkMfeXjawojcP2eRC/p3/Aex+sRCSAOEsEz0weRIZHcj/lPt4iuN8B3eTwV3Am6xp6Y2kw06pU8GmhkmmuCvoH7WJX1iqCHHbQ5cZIKmKM+BQHIbQK7ZS+wJ6sL0kL201DzHU+uLWDsAvFqCL6cZe+rPQqoMIMzld64JsZQm7vkdQ3KJgssFPzrnGH85ZqXs2LpbWsjdgsCc4PPXhj9WYoj4GAKSCQsCcvgsYDw5ENz+H7+BwSLVOYqUnmn6PjCDyazLPwZ0D0yzl05qyAUZOnM/G9XQw/G0zp5VIkVa7sdKylE61c9YXHpn/no7lVj6augJrgFBp843Eo+4JlP9njrq3LQNjSjEvpQ1pdAino8hry1nz+nPgTGtcWel+fhFMF/482q9VKdnY2cXFxiESi//sL/n9s/0n2/cf+18zbGz74oIK5T8Tc8LSXwm+XPcJidcT2xIcbRX3oP+kqAHEyaKoYxaB703EOD0Z04jyufQU4N9mrtW7rwV22jrf6PWRVZhLV49KID1NwAVj1okrick0NJdc9uNq5FxrhAT4KXc3JvCy2e+wjUTWIIHkK28qs+CRVM2zQ94Q5v7jRtIXgNZCp7Vd4cM4RxfAwDow8SblfORIgTCwg5lkM7Q7N2BLsl6x4NoR19WL2TNhDidgZVdRNJO1j+GXxUpq8ovDaX8IYJfSV72C1o5RGlZGRD6WMOP4uzQuvQC87PYyjEER5ILRUo1G7ckFegc7pKKvTx9FLAaOdhnPzVgmeVd4UDBnMTwNGvBzfHeN8ACNDnn6MQ3Y9zwfd4+e2ZlbJgDuzofOPdMiDeN//KTIvL3rJuzHQxZ7os0qc8RM3IRaasJgsKKuK8PCw0ypYBB1IgOXuBZQ7FzBC20atrxoHdSX9g/YAsDNvFh9pShE6SvGzCfhQnc8QhwOcbZ+MZxEM1E/D++Iw8Oug+88TmKyCNHMcydJsmswSamo+Z9UX5SRHxBPzyiailIf5ttFOx9Nug1869eZA6z1UFiENTecxqGXIHAxweyYETEX4X/oJNht3p58g0qEJd4M9iLPgr624NT9HuKEXdf6JVPtcfjlm0VKY6tjExZbn2Jy8adV4I31J7QEdboWcmpeLUjaJ5IIjeHlBsd8wvm2sRGeDRc9iEFdqiTP0JP1+COIR9VhEJzGLW7AKjMTf6oL+wEAuTjzFtT6XGCP6iKt6cCiATzr1Y4VyKUqvblzueKEVaRMgCJpmD8blbuTQ2Hus6JKBULSOySeHsnbwUzgdy60EMZ0X/s6SPvu4+OwwOYPtSGmlAGRCaG1yRFRdg76mkXGHR2LwDiQkJRlUIBLYWJKRgqnYj2cJZqKHX0PlZPe2Ksyg0EdS+d6fFOs1tNVH02bL453wH1n/dDZSATw1+vH28lcRGTRoNVr0CvscqrFArgn85V7s7neJ7NQr9HP7A22zFk+LAI54Q8BUzjSoyd3lS6XzT/Qwj0GU1pN/RtopFm/2vkmHzxI+8ddQHFRMeF44g3s+wSyEtc2wYmgN2vqBHL+6ElvtI67Ux1HouYNML7gsL+DixgfIg71R9ppBrO0BVk0vPk8385rESrUliUd5Q+gcXoT7gfFEjL7JoM4XWeIGTb6/skur54IPaMx7eWvTfA5MOsCT2Ce0KXKoMz+gSHmQOvEhnB0jMZy9QlCNFuFIIVasOMmdeG6ZQqJuGHUNRbSGpNIub2OIci05HVKsda7Y9BYMFgPp4WPpFDSH4OJgfmgzsUWnZkhEPwZERtNn52kkbZ24NnMfthdjGiapIqfBEbFFQtKUi3zvBgfrj9EYdpdoZQN+VyE7LZkqDxmx9z4i5PV68IB2K2wtnk6vpmPIikKotLXgVeRHfpIfJ5tOcaithotdkznVazDG/rXMCMqi2Hkb7bJCaqVv4OTjhMXgQvlFJV7Ga3TdN4kfPvgBo9TIVkeQj7nDLC9PBkgVPNh8ibx/MlAD58xmupVCV5kTbjJn9HXN9LoSQEtkEfGdWxivhsEO0GiBsyILDZ1klFvDiFV7s6US5qlBoy6kl+Njbj8fSGBOOP7ObYyu/5XtL95LiUBOQmYCEpMcmfEKd33ziXXdxS/DulDcHEewxxWsHRtACTWa05hFzVSJEgkfcAnUEQgFmS9ashGavo8Wz3BQRwH2RCSArdnAnJ1zyOiR93I9ECCk9/XeSB3UZCdq+F1r16xbUpBCtcM9ujqdYX6yB982jObooFP8LIQMYxmvJn7I5vRf2ad3p5NZgFjfhQulAUg8exMgPct6XyhUnGeFm4SLKbfoIbdXgyiFMMxRzzDgku4+j3Q+9LgZQKuLmmu6GEZO3olj7kaIfAeAKpEKY/R9Ao91xffESIpebYa8TfSU28EAVqwYklK5612AWWzGT2iEc8mISeS1WhioFNFr4SGkAmgzaXnqbz/c9782ir5XunJpwSX2zdjGx2H28fjVAy7LnjJnez5iUwfDtN+ydEck/lu6kVz1OlkGaLS28yxgCjYsPI9aQV4gOFVPY32ZfR2eJADz06tMOhWCzqkryuY+jP79e9DqQe5Ba3sdou73qW3SoHzgy8b7PQmM6c3+0MtsKPGm8W4n6pOrSA/9l1I4WAxeYtiodOBgo4RTeVZ8Di/j9ITT1AQ+Y3sLCGlm+rnJ0OZI0juH6SU2UGnweEn/tdzoSPA379LQs5KGWz480GkIi8+nvdWBDnEOgfJavIRgMukQCDte/nZWwbt4dsRSLMzj62rIaX/OyH8S2F/6Kq+5fANtudD7ABJTPUc/TUUe9oh9d6ZyM6iEA33tY1LjVUOLQIC+qIaA7HQa/DrT4/kNNjyHMqe1jL+1hXKDBJWwL6NN+wmpF/DEAJ/WQ4nKl7ZBC3CPqWLA+gE8iXnC3hh7BZLc5EOhZgpNMUVURJ1krvNx1K4tLBbA0ReBoW1eJu4b4MwjX8CIqK+IEAks0MB1gZhDNRe4c9lCk1cuK11v41fnz99Tr/Bf9JZHxl3kwMCLHN/4FhFPwpnneABRwxOW+hxl3fUlZDgHcX9AMWXys/b3TQbHfaDN2srlnlcIdhrFmcieL8dz13gvNqVtYHzmDCqfefFg1BNa+9+mTDoImQA+dYHzOlDNu8jeJicu5k5jcruQVa4vmyBN15VtTjvJdBOT4FNJq2MrAgHMT1xGz4BD6CS1vPrAi+YSb8bdieT78xpGrsoldpQ9mZ9QcIjgzw+w7NI99if1pUNawc2EB+hKrnHW6V16uQ9C6aDBr6gZldw+DqLeeymtDOVLJxe0KY+IuBLOyoufMfrLwxR1jGZyzHe802MySmU+O+r74nRkOedU95k27116dLYHDCKVB3Gp3cvZ0kl8dOMjzEMf4RG/D4Bv1Y2UldVTuP4T6pXPUbZKqFwyC7CjvVO993FR0In5W+bTqrbw7Sc77INxNsn+f1wFwYIaJikhtyiRWRfeJbKfEG5Oxc1mYGi6L94CM+ZQITVeNWSJQiFiMUS+S92O97iZ152trh24BG0lQmaizTQDdEWQuYLfmvuwtAm0rRmc3jeA5tAWhgc1IRXApwW9eLXbh1Q3u3O0+Ad0ASm8Y/uIx4ru3CydireDjlEPuxF8JwWLaAMWNxPyiQfwU8PXAwaTa6jml+cxuByMQa0D3841iCVmqL4INVfBsx+3Hi6h40ox1p6JWPztE+GYDu4HPEU/azCjuoXidc8TvQxqPWtxEIDI2Ehe5HMaXJsJKYhi7s4PCZ12GafUw4yJfMTx5++9nE/q1CU8dNZwx3sSbzyNpcQMYpuA6xeSMXq0snHuduo0HSgdxzAi8zj1FhjoFknBlH2oWzxwbJVQ721FIIAPe07DW5UPVNPz0W4Ep+4ClSRGnYDcE6CJhcj3EHCIaq9qnMTQ5DGYcsDspaHHo/lcvD8B/2Y3XCW+RFasIthbw+DQwfQJ64SpvZFx3WoQyz2pNsuR5sFvYVOpCPkZgJ5yEw/XrmV97Wgut8E//742KKUtVKLmtLIef5/bOHWK5pLERlsTjHG2o9CHup6juFsmI09DcE4Zd6V3mVHUhQozhApsvGk1MPxUKjpHEVmTxrG3cxGCkr8haAYAr9Z057HxFIH1jvjrouiRWABHvAiWSRGolbi7RGOzXf4viTyazEbwn8So0sM0hMLsRsOLlQeEQhEDFDBF7sTJuwF0vZ+IKKSNPU+W06Ycx1TfgfzTZmJBuTuBPXajinNk3JHx5JYMYNJxZ77MSCDFLZM0nQctf4txaBfhq/PGzUNOSP0a4j3HUNkaAfiTJHgTcSVoylYSniMk01AP4W+DSxekEiW7KwdiNgmIeu5EfPZAusSnobYVknLNn1FbXiVDlYZD0VYqfaoob/+FGl0wWTUDkWXm0ankMm29XGjukPMUNUaLkhWXrmIRtvFo4G+IxeAtm8+hthK+9Z9K79I4rvvDF0XVvPHzbSzOBsJ//5x3u2Tw69KjAHx88Rq9I99EdHUoYsfrrK/xZbFnBSKhhWn+V0lzLMYinsHqRsgsd2H80fGcHXr2hT6h3QQCAVkJWYisDgz7VxYdAK+SchzlddhsvQiuXUSx53rEFkd2jPPlaV1vNqRtRkgwneWtzHSFnma7z3K7dDo5lgdMzgglx+kJ/8z9C4Aqv3aO7IviV/c0zj9qxHolgv2puyn9pBSLyJ6klwpAJIAbF4fy9GQI/pPVFHk0MvfqTr7s/ik9FKVUOUUx3eUKEmMIDXoriVLoLNfDLTswSDjSSoVfBRV+FYRLz3Iq6mdKjAPYrOvC7IRSjkmOsbT/UiofFXJ+wEqcdd3t74YhlOQCuw66rSsEFfujMIhoECdy7O4O1LJ6ZsfP4sKD4djKbczMmIloSAvrhtnX32dN0WzY0htllRftnSSYZBLu1l2mSemOsy6VKFkjHkCxwEKTVwyTu64lsuMaOtRsfbSWpJFX8X9/G9/mDGfK1kjymsJhNnbt0prLLPUL42C1P871Qjo9SGXuoGcIH9srvxdOXc9saSXWAKj3liHwyqVn2yh6joXFZx9wbXsijpXPqfi0keZR23js1pn+1PODG/ysy+Nx7lIcdLXcn55Ga3Q9DR1lpFb68LuuCd+O2TSGTUXc0Uanmwe4oJ1J/qwcrOHn2NkCu2s9aUhPQuWt572AKkYo7X76QAdIN3jiYa7graoQygrc6XI7ljaVjqkHiwFYsABu1X1Js+QInhWxjDi8GN20S0jfWci6mm/5KD0WqXsHMpuR102/Md66jXnNHRSY7NX/3338Hf705LWDfRAW+tK4V4JvoV02pljmTO7CSziJfPjCKQeN1QfH9n10ldrjUZva/yDA1A2LRxFTOtkrmL+vVfOxtgVvYyWxRT70vJJIdWYCp0QWhq/ZC9j7pgKkehFLf17K07gm9k9sZ6bziwCw2X7wOmTZRWtyDeldv2ZP15OA3Rfxtyg5VFFHe62Ws29lkmu5QJx1DGNUsNerkvGOlRzVwTypC8naQhqrXXDxauReB3zSAJ+fG43tWRiPbY+JO51G3ahk6Am7W+z+pWLvOByUAq7JfbC2b8X1NSE9ns3jdgeozK2IzNWofGUc6HWAeu96hrfk00tlZZQSunf9DbcSOy12na6O536fIMIOyIxSmcgpcmbg2QUYQzqISUjCKLMDvHoq4Pja1xldGwAfgn8vfx4W28FlKXLY7fKUD3aNRadvQ/+anuFnh3O7+22cY8OJD/bhcs5fhN4GQ0d3Am67MPPmPC5N6M9tpxMIXrhrLskPkMk/ReLpyeOwx2Czx4hEVgcE8YmUTC8hbFQYpkP2QgCRwF6F8y8QxS4nAnZf+oFRRt9ev3Kx2Zs9I4ei6VdI/PV4Fm5cyOWBWdQ5+XO9VUbsvhHk2nKxKWN46r+EaucjJJb+xpfXTtLmfIwyrz/BMwSeQeyLwp+SejnPcz7ips3Kv0SV/9NSgJNmG70uFTIvI5uMYSfYU30FfegAUhWeICxFLgCJAM5NamBU4H/byYt30ySNJPBaCQmZLjzrIUQk74ZwhL2vl4su85d8O1Fdh+Jf5M74PZ+xZ8Ye9o3MA3Yilw7gbp0nFqOFW33KKQywM1+km1sAWBHSyjpakSidyUi4QZuqje1ZW9g2LYHaWhu7zvbEVZ+HquohLe1SRK1HUEqmEeryiOmhpxjhJOPNwga0ZV7klUaRYrbL0oxVws3iD3Gug6oIA6df3Uu4Www9Xvhb05M380z2JRMrBOS0SZnUrCHFrERm+jcRH+s+k4t3KlHVvUKF233yJQ5UdX/KgV9u43rdi1ibD8HOwSzsvJRNGT+RUgbmPiMwmC/w/IsfiU3YSuKbYVCcQNoX6TgLCjny7H3apDn0vBiOwFnADx/Z/askI+wvnMjdirFMiNuMTQKGMGcay8PppFbRs/JnFml2cLemJz/kaYgr8Mfaxc64JDWV8L6LmO8bzaR2mUTF338xckcyLdEWhF08yGM8XjWbwSkeRFJ+VqgxztuGxNFOvSlAgNi5L0fTQ4mo3E1bdRu6/PNIqzIoi4zh6JERuCpzsXjUkxWXhcJDwbaV2zjqUkdYhQf3L01k9NsTXo6bSKWgMHECHSo32jXeuD2XvPzMZgOziycWkRRNbT4IBGx4bQAqpzb2tdrjDPCfyr7/TftPsu8/9r9qfn5GpM/F3NZDHwV8/813LC4Po73oAd0lH7383m1/uGKq5fr9PFSCNjK6T+OtERtwb6hjWAU8MMD3LfdpuRKCMPI5747OQCZUcq7CirsULrZDRsUEWnLz6N3UguXqayT+kMaKLkcZ8CQVoyqFTO05KlU/0Nzqh4vDf7tJix6sZqKCbyCe7EVOxOvoq2ZT776Y3Z4ww9HC979+Qb5pC0Hp2bgU3ONSzE2ytHroqOambgyfFwsYExRBdXAgerUnPoJS+ilgiTPcq3+PnBI/5C4KEkYcpGeQXXPpSjuc0CsYcKEn6hZnbgwrQWQWobY2kBkIm5rhVNtlTk/dh8AmoEPxPUUKiHlx27fKJtNbAGPNj8nLU2PTuJP1aBibprSxkOtwdy6C0P86ANiQ1cO0Mld6RMznQGI50+4+oH3zGKod8tnw8RGaJPYNvavfEcLk7SxotYPr/bzTaV28ghGKcOp0frgry5nX6QcuYaecOOjhSbJLNg+VDi9pHhVGDWEPDlBsEGCpH0Mpz7CoRNzrALn3blocVqLV3EXWYSbjsD9Bo32wyZQ8MOQRWLsQk8SDrYPOIe+XwdJf5OzxmcmrK7dB9PsgEJIqayd0/VK0ETJiZ+mY5QSLa+3VURaJA1YHR8ROKkoSxpAXsxARdnHrFkEg38ybhyRai8PgOoYH7cffaMFdZE8EvuZWw6z4FsY9bib0wE5soaFM6tdGiAp0FinXs90ZeHAYLX7VtJXr8Eo9iFloZqoKHL1O83WVgYC4bGq8aqhzr0Ys+ncJFiBk/pMeRP2QjNFlB+8I3uH4GLt4O9fGQGsuh/Vy7kh0dK830el2d84HaZkfUYZYaqbJpwqL0//kg9bbILYEWnyfUvZZJmv8B+Fa+ZgmmYQC0SJcdPtY1v0VCg8GkJPRiYghr3KxSYOvq4W/T2Ui0dxmlKyZP2N/5Ot6Hb9YW2mQ1PCkaDa1L7R1VjWUkeZmQmESsH/i1pe//WOT/a/ROYaMjJ2AkJSQQQy7m0ZYghOEB4NLFw6k3SPmhjeqpDbaM68SXNsCI+1tBNQtwFU7CBBwv78/f8/4imcSH6xauK6H2x5X0biVMeHxILrf7U5aFx1G93qmVsF4mZVA2SP0lg6EFY+YFfwK4U35HHkR21x3bQoJ558hMfXAaZwSdScBJUYNgVItJ1pU/Nk2hidJZ4ms7EdVbwt17nWIgXXuUCl5isDRhafdF9C/L+S7P+NxbTbeYivJMrDqyjjQ/TF3OmfzWH2PWl03KqwP2dsKM7o+p8a0ABefZLSPS/h01aeILWIuDLpAq7oVi7Seqoi+yP3BSxuKTVvL00BwFoHBCu83jae8/zS6JeQz/4V/ONhpJa3SOuap4a5KSXaPu3g//4cO/TPEtpl8X3OUU8YGRHov9iw9wv7WI+wsG0Cp8CqdyjYRXbYGi7AdD5kPx/ybuWYAQ5OBJT/O4EGXdEomv0pETDcsF03YLNVYWxqwiYUv0dq/a8FB3Eyttw4LIpLfSSZjRwaNLh088SujvxBEc3dyrWMy4W1hpNxPQe1qYInXZWZV26li88+nIjM70tRrC1k+u/Dq+JZoqZ2SZ31DBz9sHoDF7QbNr+hJ73IMg7QT4vRzvGOV0n3Ir5S8v52HVX1pEtxGrzIRC5zIXUSz3os4iw53hwZa2zW4FuyiODCDJy3rCY+zUxL+KfmH07eGclRs4VlyFwwqAc3KcHS9spn9+QQG5wVhnWLh9MiTmL1eCMSdjOHz9iK+v7GMugAdRgcXDj5I4Y2Z90h0fsqcBugjdiBz5duEd2vk/nABDRawqBNZe2cX7SY17d4ZbHhnK4nFX7HzwqekjHRhuuuP9HWAZNkj2uUe+Mnh/ethTLjpzOJ5PTj+oIhA96uYBD3p5FBIj1Xn+WXbBLzOeCEZb4UHi14m+7Zre3LNeJ/eegUeja4s0xyBTqtpuL2O3+7FcUmtIj1oP1dDl9mrgHU1kLSCtKNPiM02oQgQ0KcdAkTwU0smE1QwxAG8+z0kL7QAvVPiy2Tof9kjnRO+z86hri/FktJClECEzJzHsEozfmLoImynySfmpVZpugFGWBs4Nk3I5vQNwEIcvRuo6lFPe1s/gjwcyfS6SuogP5C709rwhO+nbWRjkxDht0tRalNZM+87ABQ5CRy7G0NjbjWJdzoo94sjOz6bb5ShTPMoYNmPw1FnxGGdb8XLswleVJOtaoQwCZyYfIPDjQ7c1xgY7FnOTSYCDwEoaB9Mp3ArcuehZA4ajDL0DHek91F/8CfrXTsot8gIqQtids5Vdp6LZ/44HT7BVWzvuo5nwdcpkK5D+OscBlf4kZPgwSu9d9kHTB0JphYaVJOp9z9Bu4MvLgWVZAt1ZL0odhNYBdhsZgy1zXiUpFPp04iwi4lefn0R684ja28j27E7ITcvU1uqJqjha874QkwJfOmaR69Bc/i74XM2vbkJg8wuKr/ECcZ43OBYgZXy8Eiqgg8yLcmui/uoCXZ72vV4+lYEwJ3haO6EYhPC5Cmbada+Qp+nfXlz4GQKtGV4ntyHrZMzp9yl5Eo0cHUEmLQwpoBva3x4YK0kfWwuXq9eZIc+lmnuNuTKfNSjLmNIeMtO+whMDnmDoorfabTA6qYODojusMioZdW50ewuKsdv8Ff0E4xmYfLbHEyfgjFfidzhEAei9mO1rSNJHcHrNbnU1Q3j97D7RGvH435WglOHjLLO65haDVS+ia9LXw7adqKUiegYGIqTk91Bvl46jeLmeMZPVlL4MJL+V/vT7KQn09FClDyB2Gc/QfRSTJGfsXLjOzRLAknOO4lVaCDCJZKnza3kROdQFmTj1Yj1VM7/hP6uX9gfoqEBUYeR89JriK1KfF2DeR6Ry1xlM+4uJ9iQtokO5SO8H7qjN8gw5ojIMnZmmh0ATlZtf+pt0TRfeIpDnBtRvS+SGPsMkdX+PFU2A8+EDjiEF1GFCpGxjXP6c3Q/c4w+Yhc6BJ50i/yWUgcrj8VmltTBz/8yRoJQwgBxJvc/+IJgtRllSy5Jseeg9A5len/GbZ5PQ4IUXYAZRbsCoVQINyeBSEGTtj+5e5xxGgqRZ2ai6lBgHe0J0R/Asx9pblLTXBhNpKUL5adF1PUPYkj0dpLl8LpOzWrvGziYAnBUn+anfnNxN9TxdvJfRLndodzSi9GDH3MmuI62ck+6uOnIitlH0+mj9A36G0wtZOicGVLlSVmMmgeRi+ijUhBh/QkqToJnP0pDp1MlMmCQCVh6+RxO479gj9tP3BDFsCb9W1QKmLJvPE1OlWxZsIXTCT0JuNSVdcO78XduN4w2AYEBNfhoWol0WE9iElitIhLvvIEpX4NupCdWmQsWhQ8zHGGVK/xSGcj11E/w8hVRZdluH+vW44ToeiDThRAR1JVLwTsRm5swi7W42+y0+1eKZuEoa2A2oA4NJzdSQnGoitQ+aZj9JiCOWgICAf3krpiGhSGNmcpHaeOQOxRQV9UHjUDCk2mHyKvrQfcqKRN3KRH3SQZA1v8U9YUNRPuJMIoD8Tr0OROfL0ezrerlNNjeAh85wxahkaT7SSQXatm0+By/fi/HSV5Lk+9uVCYVzc5aot84yAFXeLcqmPBie2BRIEynXn2NB/1y6NLdCZnhBALswQYT7XjYSkjKiabZv4H5k94ixuU23P7rZbJP2Shj5NWR+JeFcLvGlapvzjA7ooB801RudP6CScOh25pDTHjyOas+W0U9Nuh9gO9/HIzsvAC5r4iZD0+id3Qn6N1K5DJ4w62Zv7s94ZvE60zL/Zm4q+lEx5xE6m/i9gtcRl74E7AJ6NmwCRP1aF0G8Xv1Xb4wZpLYHkSgIg8Qsu3VHdSGmXFrgNUD1pNWMQIBJ4kW7iHaNYv1rjaKZqwjWvUKJG+zL/HAmId/0N6ox1m7GfeS7ozTfw166OTmQlbcMyZEziTmiR82mxYrZq4Wv8LV4plM9HmN8lpfrM8d6bX9Y8YsOMZBRSEWqZAGi4CL3b9nXyuUPLOgqhQSroZ6q51uvqhDRUjGUWoCNWTFZZEU+i9Sv7RDye/l3ehyNpjApBz+tDpzLyaOPVF20MXSp5P4Y5Q7V/YNJOlhEtWvHOCD6EpE+kpQuADgbKvirA/8XaWi8vKbtAY9pjL8PKVm2DvjGNM9V/PszFF6HXlEqSKUdq9wTuWGk+K/n8AjO2l/6kxEjDcBJjcYd4OMdhVnXL9nia8L87raeKtxOF5N4wEQCUREqaro7n2OtVc/J7rIirRzGmbJv1Uvo18kHKc2rab31CJm/9SZWw9hWMuLM5tzIu2erzI8/H3Cf0rnm73LGPvXSpb9thZUIaDwRmSo4HPreNKbg5E06RnzKJiQwRkkFv5JYhD8YynG0duREONYhuSM/R9nH4W2CreyRxiSEvll6WZ+9W5lunYPMV0GkOB1mW+rfSlZsZ6a27Pp8SAQT5NdB/BxbR++f/Im1/p9hKPOj2T5tzyNbGbNo5GMjgnneFgeVouQKwcH8p1Ox8ZXx7DhRZX6jowfOFvwBnIXCa/d2oeHVcTzvgtYPVH2742V/MOc+quoy9/ggGYQBXO34xtWDuVHIOx11jz5gkJpBrlWAZsWfIC/zYMNLy5dnDKPmfVyPD09iVAY2OoLkPGyaSsiCpKnIpdDgG0nUI+rNYe8ThBcBMEiG+Eu96mqCsBRX4GD1Yq30F55OlsNxzJl9DwxhsyEZ2QV+PGs5wPORtoR2icsT1GaFDT/MwGF0oBg5S+sc9Pz3UU9RosCgQDmGtPJ+vN1an3qqBMb+KvNhy4tuYQ2iBh5cBJlg9vQy55Td9KDth6OlDV1o8nxOm+JEvF3fcTxZh3Xwktw8S4hRjSBSgscbINBLW8zqGgKVv90jspPobE4cNPnDO4CcC+E2X0yKO9zg8yCCJZdikIZO4BVLRsBqHI5wMzegTTGP6D91Gzc2uS0qAeys/g4zVYIljrS1e0ewiHl3Db74VHjwXYHC1+PWAoRiwCwODqh9XSiw8FErQVyTQp2t+pxDs/m95XZnO+zHNt1e/Yp23gckcDub2/xhGQtmGs72P39LPz6NaOePpg1TXanurNrO6WuWp45PeNZp1amxa/hudGuXRsrhUsDL+Gl8SLyaCQWk4V3uizilr+dCYOqDbiNqePxB1+y6ORDpCIpm0OiXs4Fk0XG15f6UO3uQHlNIbN2zuLsmBPMD28mt+0p0+I78FCXUOI/j46aZuJyZlAT9Q+3tX1pcOlKQLxdnH7AqgF8UXYBsIO/TDYwV4agsklRearYMW8XWsdGfgrOZO54f0I+jmQUQzFam+j/YX82i3So28Ne3tdJHxgZXsdMwQWsjSuYL7jB0xy7/yG2qhhl28HXL4rAZZ+VscC4isb5QYA92RebHUvCYzWVkc8ZeHEgaclp/Ow5nb5BMxBmXsUkNWGwGrBZbIgsIgQIKGhMIsIiJCQjgfrAemx9rYw5EEJOZB+G97pBScEAbhu7kaX4k+EeEuZtHUeswgir/2DX9Q4GiCHlfyqi/L9ZCtBTKOC2+j7hOWIGarNIFf8Mz+MRqv4kOUuNe507EqWZb20tNGXPYIDf60we8hS90QGtRkuNVzOBT2VEafuTPqL6ZZ+fRz2n1T2RmOdJZMUdp9WxlbQOSJbDiMGXyU+8RMgQMO18n4IwO+3p45IFvFP+EXXhQzC3SOjafQbbDceod35BE5K/CWWHmQ5zFqpCP2pW7WCWWwNZsg2Eujxi9YCB3C6NYN0by1GkFnLmq8/5MuDf/u7ygiNl98m6N5krzk+xDAOwx0s7Ir7EP2wSWTkiJNumMOl5JH988T0npQa2Kte8bKNdEsPuQ+8hcVEwYOUgoiRg0ifg45hHfU4DEkUN1RneuC5WEt8lnqyELGx9jxH30QQqVCf4zmqlIryJviHPCPVrxVVQwZyE5bzTYOPWBz8w2v1LOj/5i4zgOfiJwWSU02Jw57UmZ+qy47jSSUx+Jw3+Do0se8eP11aWM2KSgZOjgzk5+iSdcx5hkpVTUjWEUQozAxUCGv0H0Oa6kROjjxPm8zYZdQNoaJ9A70gZmO3AfAd5ADXutfS7E8Lc9BT8ll9AKASdsx+FUz4meSHsXP8D8Tl6bvX6g6D2voy1dGfsUIiNKsAlci+vX2unhwLyGtQ8uBCH77RQ/mv4BVIJXgW3cWipps4/kdPDzzBn5xyu9LtCkm0lNpEYvaM7Dq3VuFZms+eHmUxefID7GzYSFdSN2v8eU/+P/R+3/yT7/mP/6yYWinmtFoLEMFoJj/RKVM4H0QlrGfffvpdviCBgppFGt2SaCyLZUb6JqVE7ONduRz0drIgi+UEwKqWSP69/TdeuNj5x7clktf0QVBI7kMeinngUrCKsXEkXjnOzJgXX2tfpHjGZqrpFBP+6mLKoUm7Hm9D1K0Wp0UHyJlAF8WPbJjIkNYz2sBCcuQitx2l6Ks4iEMBlPWiEbRhLnhDyrAL5YDnhTnqcz8QwT/8V4T8PxXVcFP4zT+CiqCRNkMhXjfBTM+zIesStowpaX3mLvdIpjPOzK5n+roUrZinTqn2Q1bsScMgV1+r++P6dQ4kJmq1gxUSSk4EGo5DYAxPYEFTGyGl2HuiNaZvoDSQPukdiLxNrdw8jKT2Vo8MbWDj+bcCKS2slv56fRFmFF2aLnMiW2Qi+rUbgHMfHbbdxdq/DIgda/tXC6hfyJ0M9H7Fn8wCinkdS/4GFn90hSprHzMO1bBsTSJLnTdbtHk/d0xAue3hyVxfNwp9n4S8WkOIAfuKzmObkUCeI5N6JH3jqbmXTKzEghseBsEOsIi/lFUTNf1N305nGbs4M8A9jZ/scDpd/gFDw/MXd2HgcV4LBzU7nQKsdVZLqGMAd9V1U8giaz4wg06+ImrBMEl16URk9EmkQL1FaFkyIBfCtGxxt17IzPhOLrzueV76nLtuRHilfvkxs7Wqy07LpzFIcWqroaHPjiWAyIdzia3cj/QJruTHlIM6soB+ldPimoq/fw1ZPuKZey16VN7ei7qNod0GulyMSinATQbIMRrr2Zr5wN7qkB4RYu2GtsNqhMDm/QEctBM/mzUPXEHfoSAwXcvPjtewJMkG3P3DfuY5WUz4dOj3eeY44NaZyL+UeVqGNgjY/hDYJyIowywXcGf0uSgcBWdIxuBdVEeV2hzNdd1Ef5oyj1QelwIhE5IBcl8T+Ef25VjGEVx95UKxXIXEQIrPIKJQZyDDYq40aLWoeTdiBoL3z/+X7rZQ6EFE4EHlbPY26TFLP3sEcuBCSf7OPv9Nd1ixbQ7zpA2Jtg3h8XwfYK4gCyqLwKi5A3xyDwGZHEg11rmS+Bp4aoFNpAwaTM7V9jPwT9htdnF4Hm70qxWBt5KPlZznc6INm8wVOXxvJu7+sA+Dv7M/xdv8ZSVAQmWoZafr3uVPwPh8FJeMhTmdefQvzdO0MujOYC2EmLg+0V9jIBfCWExyzFqA33sbdJ59GS18Kgwq557+DKXLY5w2trU9Y0+hEnrmJREcrDdJH5Dt9z8xqmFkNI4uW4u1diqGuGrHFvg2HFIbgNewWB8PzueEyjGLT3yjGfEFaaRblbWdw1hQjE0KhwylyfLJw08bww+aZvJaaj8eAYlY22GnnnnQYMYt1fNEjksf6Qfj7wcd1DSgF4KP5iwHlUGsBq8iKVWgFbHSzRtHJ9TZO1g5UdS48PZ2C1qGR0sAqGl0aX6Ibrfpz/PjVLL5sENOhnYNHjQdajZbPO3nxpK6MIyYbVqENny4+9N06i2vzd7H82+UM2biGPvIMgp5rEHtPZ82yNazwtFBnduCkrh2DPog5GZ2RmuRc7WV3DIRYWeIMl9vk/FQawbwbfSjoVI+jaTcTimHHDlDZfQzmqG1EODYwNe8D9kd0R4KCGPlRAm2PuVr8CqaSfUxoOEz61W4ILFFIjDpE/00o+aE5kNIL3XGPhgOjP6XFIZO0ivNEq1MRFATQ83ZPTBMtpCen01saAyX7Qe6JvOUZb339BwuruhJ5KgyfwgiYeY/ZLq382ArLtAZS+1xD5tKNZVv/wubjhm2hmk7u1xkW9idejUlsdoDv7rQTfWsLlqip7Alt4qENdraa2KhXISvtRp/rsfiX+xH/ykG2Nc3iTuVIpsVPIle8i/WSHXT0KaFv0AhyzNPo3H3Qy34pGoWsWLeCO93vMHrdGpzlgFsq8CsN53viGdnO81C75kywBHbJMoGRBOgNBByaROWAVl5pG8XChAqkLWfoLrdXLraoqlD4VHIuLYl3H42gst8jfIKr8CgEx5q+9BixmtIGF1q6GMD5IUGi9bxVs4Ufer7G8SYhjmPEXGsXEL7yPQ77VhC8+V1SynrxVvJbtAgW8krSI8b2ryT54hia6t5goFIHx9QQMg9bxHKmVkGBQYD/MAVjPE6Q0RBHsGMJsoQcDi48QvdOC0lZkYLEJCE7PpubZRPQV7pxNehv3BStuIfMZVfEV/RWgEuzXau1xgI/uxsoVLmQ0eDOupZK3usUSoIUNnhAhrkXZyPnEhZhobWohiDfG/SyPuXWFx9zo88DMjrV4LZpBJ38mrCWO9PS+wk+wfYg+iFjLUkCAWX+ZbSprBwfvREn1ybWqhNweLIavIdwuiaf3+bPx7N5FPIxWSgUpUjNdnqlWbtm4V+lR7UtlIzBy7gTNYl6wTm+9xtMaMNlJBMtHC9KoDj8KP8EFmJCSrDEiEYILg5HCLeZeWL9gRqvfxGuIRJIVNVhCI+hX0Qz80pj+aIBuigSWFafSWsoOOlh3ck/cMq8TV1UCmikOGnfZ3p4PzJLZjNipJj1Py/FuZeR6CfdWPHtCi4tO2FP9LnZq/EOP56BvKIe1wgLa0TQU9WZLinLGJ75GtVdr/NN7EcE7nYmQNeDpOReHCj8ndhSCM0PZcHlAYiHN2K50wVHiZBk3uanO3/h7lDG1YhPqHUCp0fvsPb3RbyyZD8jAnLpbIZeeZF89t2HYLNh7LGCMQojr9RARdkC4ks24eB2luVfL6fKp5n62JUEy+1b7e2ySdwGJqAnOy6bCt8K3Ew/srX/P8Q0l8AjQCSn3WMuhc0GhLZaUitKGBC1C6fyPsg09qSKTWik1aQlT3mGR5Wd6TdyA163ppECDL3cH5+yQIpfb2Nyr30EOwFUoJRo2Vk0jpR7Z7CJxVT3no3JYKM1woZj7hKi3O7ws7mKwve3EtXxHo6myUxN2ku50IXg542YgRiNE08n78W5tSdNjrcAcDbd42nFansw1lrCzJWXGXh6EKKNKxjaU8y5dy3gkgRyd66ZO2OMf0KRwp9oj/4EpvwNjhE0uu2iPq4RZVgYkns3+ejoRyiW3ITWPFCFkOAXza89R9Cnr5obza+gNIrpARC3EmI+wnx6CVP3T6VgqIrsfnOJSnTgYEsDbzRco1FxkyXb3iAqTYlhxGae1sTQc9QtVNJmxkau42yZnGOGnhR4+9CSuBGlsoNE5c+sv7+VTekbWLFSTZ+glVyZ0ITIquCJ/Dwu7amsjl8MEfbqXotEiEEhw//B5xgkNSinRSMMfYPGE14odRUY29ypjRzAwyB7oLXDKQFUcrpziVKHwaS5xpA/YwNzQp6/fH9GR64nPSuGzFJ3zG02klUH6NXwPkNfVJA26f0xKdRY5DD8fH92PdUya/xt7sXd5lhON/zFu/ktDHoX27j931i6zha8AdiLct7tFkma70gGNj3mXEcBd/3OkGIzQXsFQbIM+nX6mUvCifQ778ygKCcCYu3B5tOdwfNBK4iEGOWOSOQvAv/KQCSGe/w6fCrrH+7nppsOPxqZKthDYJFdt9pmFXDwaiwaJw9Gn+pGVGIenk1badR/Q6Pel7meH1G19n0eDLjD5Ch78qjUKKPDZj9XbK000nL6HS6OPcGiQXaAYJldLpDVtY4o0fLJlu/44GkEl++8TtiQdKQJ/1ZhDzA1I0vvw+PYfG71vIqnKJXZE+9y58Veb7NBk3c7OnMlNoHt5XXXmiPoddEDh0QbztnnqA1Mxl+ZxOaiKVwtUvBW+H7GYuRIUxaJw/Lw9y+z73tiGKip5Wd3+LvVRnFYZxpesLoYJDVITW54WmL5/r0fOdUmJLd4NI9a7xPe6X2CKpaR7HuaJmwoS2+QtdOHmKFKdgwx8cz2BJ58B74jwTGSpNKfqS115da8HGYFpfNE2gV/hRSz+x0ujzvH9oH/8MOHv2GUNPOq8OuX/Zo7dxsHWqGoKgNF4nOcvJpZHWXP9Lx1/jLZ96PRlbjhmbcd/6oinJtCqK8bzlTjGRKEhVjmlpNviSI68RhvvqDv3NsKt4On4qNM4aclPxH3uCeTDk2izjkTXsTub7qvQSD4EKvQSpuqDY+EXIZ5mbBYDC/vTYqBwtVvEGQREVyupqujmJFDQJAHlV7XqVWUI1K1Uxqei7MqipTKk/zRAofarAQHl1Li2ojHvalcLa6gz7gbdHZooyYEjmkN3BXpMLQqGHP6NfSOHkT1H4Vj2AXePJmDg1Mk2UOgynsdfmV+VHlXIbX4opfZ/cDHkUcgqZVJ6gisViV1On9sEmc8UrcjaII5+Uk0qi8xObADc6cyPqp3YcNbBQAICy8wU/cE6eFkdGoT7k97sWjcupd9zjfayxMkJfl4FNZTG9TtpciiXNeEZ3EaxuoAbJ427uphvGsycnEb5/JfY217Fg2KCkIjtHy7fC8Tnfqgu3GQh9VDqZY+oyymDIVBQNcKsBOMQ71Jxs3SyeQ5PKf0eG98w6zQG1bUw3KPCO6U25OhKqmKuCfdcWz3I7tTf0qtEAzgPQxiPiL/8Sjyd1fh1LmRBofPeff8RH77Kh3uzaWhORVh9SWeltolUZa4/sv36KMq4rPxrZzTQVHBDNZWhLIoqZRqQzg/3PqHOpOcaX0TuaMNw//Y1xy8+xeTXrUzLnWTw2DrdcQFB7ggHsDbm1ZTavBgtWEKC8rv4yCEbyLTiH7/Co/uRGO91JOIznkUmxqRC2BzeSKjI7vwaNQuCk1eTHBvxlkJf43zQSVt5oK1nCcyJ1zHXeCY0syj8HsMkKSC1wCK3A7humAfWaJlVLm3MytqHxo3LQWVGq5GQpT0EeV5fpgbZ/Bt3DraJWWsEU5gnBIeG2Gi0wV6+a9hl/Z1Pv7zY3LiSrixoIJ6g/1M97FLOzbgnROhqK+lULJcwiRXOOANG5thgKISCybWGQL4KOkb3LSlLHW266k+kBl5KJESPeUcgt3zeWvTW9x47QRM8gdjE0gcqQ2MoTWonOCLfViV3Z07Pe7wLNCe+NqotWvTuZc40mH1pyygjAwDZBgg/k43uNKF1g+bKOrlhHNoFQoUjFNBnBT2albiNkqOU+xh9k1eyzBlPFf0cNoHDDbo8HnCVmMdB5KszLLaEykXhaFI9QU0iNScrqnjXsWPwAsZkf9mcqEJyaNO+ASa0fu341vhi80kRl0Ay3r0w9f7byKjsij1XkDJDxeYeD4C5RenmOE2nH0De+A1dN7LtkwCO5C0s8wOIsnv/BPR0SCWQ2lgCVbMPNCeZuOfWylS5PLr4lzmGGYRNjSMjhMgKO3AXwylL/Yem1WAzncVfjeH4H70Ltquc+hwfPGZrp3yu424hLvg4OaOtCKQRNEYwA4y8K3wJfy5A7V3C+l9szc5UTnMDl4O9feJrd5MZ52GljoNLq+58Nui31DqI1FWahnnpCf9x6+JnXiDe0dsODXJcW1V83HUPm4rjdzOGckHP3yAIdpIsU8VLdI23gKO3YH3/m8Sff9lC6xWFrc+Y6B2IPfKohk6/RI0Z+EivkVEbgSjTo3ixpA6ylUqrrTEEOXVF57NJ84Kt3sWkJ2kJa50BkWOO4kVDLP3WWhHEdsEVpo9NFzsYZd5SS0Tcj7qJpfOd+D1gsXMwRCK1OSORdrIrhXjKKlpYdPSYSx+lIR1TwchNUuocT5GqHMo9D1B0YOzvL9sET83SDne7smmGiNbguKpaPFhY9pGzhmeQOJDmn2FlJnh8xpHpvbbyt4bU5jsCLLgKhK+/I2jOSncPN6L2O6PcXJvRm6rBacYhIJcSgNKsYoE1Fqg2gA2kfLleAkEEJB9Cr2HP1orJMrB/GQmk2LqyBhxjKKmXugMTWABf5GAzHDgHwmB4sE4Pe6EybmDppQyXgm3x4QMZjljo37hbF5PDovaQWIjND+MWqkP45Mqmemzh5lhe/jmz62YrpbR4P2Ynr1HEimNJLouiq0799Ju62BIiZ0qPuluCO7VCZTPiQDf0Qh8x+Aqd+Jq/C1yo028XXqTLO/5PLN14ZcGKRgawGphr3cTd/Jnc15mxj34GV7yVszNN3iz74ccr+0PfENxPyWH+vwCthTMVgm1xkg8pM/pJAWLxYLgQQTlgbV49Pbggw+WIHX8nxk6eatdX9a97BGTDk5izbI1tDm2kaQFm1hCSfwo4jrZsG76g+oSb3YdGI+6MAe9UyA4BP9/N6H/Y/9/sf8k+/5j/2smFAqJi4vD4aZ9QSk2w4W7UWiMQVgitzHTJfPf7+ZBSJ0fVkc9EZqb5HXrRb0Iwt2ustr1JJNUMF/WzFcRX9H/zg9IDj+gXD6US8EiJqutDHeAUeKplCRcYVB32Otrb/egrJYD/e9RYLlItrIe1eg8juQGMnX/VGpC/yJEU2TXFgOuClaSG3GSuCe/EXvFhtI2kwj5WQRAUG43Un1EtPYM5c8ub6HVaFFYwBz2Jg6ljaiDNZidDUyItiNM0gVJNFvtCbsWdSgOoQK0EikSkd2RqsKBKks7YpOWdV/tZF2jmLwTI4kJqMAm6MTyBjjmDXt1D5mOM0e1ckqz4ykQQovUxq5b89Gb1QgEsKLoB0prWjH1P8ars3eTqAqEoPUABObdoFBgocXgibVvLwJ0O5nttBMyISAskr+VNUyL9yTwd08kTg3UetZyIH0bFyR6Fms+46lARGuZjU3HhxHWs5Egp2yWnkujTdTITM8vaGquReeQQqjbfcTWRlLlUvZ5w+22B/QILmJenpqgHf9AmB1iXmSCXS3QW/MK68ZmMOLuQva+8hNSuQGx5AmpfseI9biGqHUABYYQvnvmy+GBGWyPyLE/UJU9K1cpm8OpuFkkdxNhWNXMTpce5Kwwk9V4k4KYzrRYFtCvYRbBDy9S5hBEVkwdXUrBRSKmcfoRhKYxKO400dfB8vI4u0UL23JCGPrbOJx61LF6xSeIbQomMx2A8aVqvol6SmRCK90fn6a/73HyJT4ATK2GBguESSy0CCHl+DCicsLR7hbRVQanfeF5xyWkqg6ujL0A8p6c0G9EaJVBcxZY7EnnAZ9sJ/5KPZZ3HdF5ttIglhAiVqLRanhn/XLaxhYQnOdMwNNh3Eu5hwB4zzKCIS6HWPssjHrnQvJSe5EqleBt2Ud69WB+ubsDqYOS1lbo2X6Gsfn/0L38Hh59NFR2OHG8Ngbp7mDiDQpC1SZcagdz6NMf2N0Cp9thktJAWTAsKmmi9OjH5IQeJjcylzFKSJCBQFfIVOnHZGd0QtZ3LFWhPfBU/nuImCx+SN+QNnYVwXW3g1zq/8fLz3yLTQRnHae1OoC4NB0OQfGc7ZnFfA2ohAK+2DcDbbMz+StstGha7F7Bi2qYM3orZ/R5hJru4DKoiO5yAzqhF3JzLXsff8mYQYn0X36ZfaULaC7chlrqxInmj3jNPBmZyYvBHa3k3O6Nk8sjeJHs7bDZUZ3ebbH8InudOY2ubMh6TJ89DjjHTkD9lhN0H4CjR2+2VHrhXOBDfEorTg9+I9Srkr7D4XeZI3uEXyGWvIOmbzhLTEMILAmk1tPIdZU/pc8F5O/z4drgtbjJvkatj+eB7SarSuO5ZW7E33ifOO1TGiqFRBcHssNVhz6hkTJzDWWAokOBwizHqrQHC1xFj/nNHd52goMNt1l3cSMBRguO/nJ8a/rS5KGF+GQWe4KWGHpnBjHxTiLGsQYOTD1Dh7QS/4Yj8OgANtEw7nbY6V3rbhfx1qa32DdlH3eDnlFxdBCvnY+lcXYz4EzN3WIAZEYZ/RQm1pxKIrrBB7oLaHNsY6XOgXMli2m2fkeQIYKGBXuoMDoSeW0igx4sJfarx+ABD7K2EWqo4e77X9Nk7kbIw1KEYisC/PFQltHV5xQNwu58c7QrpmYBcr0cxGLGB8/HQdxIZs1AcowysiTtNHvVcHT4QwQuj0ktqITMB5CyhT/pzaUlPxGpX/Ky4sylJQP3/G+QD7rHL10esNDlT2iFQUot3JoKQ+7w7e0b7DB/CHIJ/YdX8ZrLkZdzNzsQ3q7tYOOAKyQ1JZJ6tJDOoYcQMJh7FWOp0QVj1pyg8l4yetUtPMcLafJO42aHgNMv5vDWu8mkXg6jY/wF5s65hL+lhA7/byk2t6C19kAs0HBPK0Xt3MTviV8QrsqD4O0v76GnOhdLmJpmp2ZyjHZKHM52YUfbSra/vZ5ox6F0PenD+Cef8Mfy7zkjDGS4ayp9vYrwfO8f9ku/gh8cuYeUuFmxfN4A3zSC1RCCVlLA8EYp6Re7ERRdgk9wFf0UkCMysmvwc35vErGyrBeXgB9Fngz0vEBFSzg15niat05BrheT715Hs1MzHbJorpVMI1DzGBfgYIs/pzoqsa++NiS02GlVHSMRih3QHhxKdJ0HAYNGMS92Fdse/cA7Zb24Fd0DB00bArUDa5atwSw2M+xhGyVWJSXAo6RPsQkzWCHaSh/DKF73O8n+VvjODZpNUq5fKMRZAWPvv4Nr7WDk29uRCyFYLKBQbJ8Xels9k+LjGeZqQCS20HP0DRYrKjEq1LiMOEWHrzMPe15CKLJ//1EHrG4UcLUtgx3zL/N5lQcDMNBVBk2Bc3EQmUAZjIY0xiihSKdnkyqJnv6lfNcIyxugwq8NZydnhBIxZpkKR08DTRaIarzALXM0fYqfkSqUkGlQ4VVs49eE9xBW/sBpH9hwsz8V2yPR9K9HkazALDZjkppYVAeL6mBvZRgVp7syQmVCXTOEHp/8RSexFMcC+yScHdaDcqcEDA4uOKvsycKHlgyeRMt4VH2Y4yGXyfc38MmxIKY01nM2/Bm0tUPsZwB0y5+Dx/VDFLWK0YaE4phkIS7M7SWAwGazEpHtiU9JONJlUtYOWYtFX0No3Qky9EpaBDY2LP+O8jALUoGJK8WzAWgN+hutixlLg4BikZGbxhgimlUcaVHgYwzHqToHgc3KurBSdkiqSLbCjvB7SPym0Kj0pS25HEOHH7KcS0jEwbxc5AGRqYnHSe3sr3fj3I0hxKhLXn6GScs/WZvodKsCvVyCqkVJdaMjkoHv4hP5Kd6V3kQ/60RZxDPydB8yyFuEV7n9+iaH7rxp05BZ70aVoIGFTv82OzbqJzZV/MWmhZtws8STKhpNL6d88qWbycr8hv5BuzheGUu2pBWTxcLd4nGsyS0gpceHbPc8xA2XkdzMf9HYiySImwiudP+GrOJGtmVt4IpsJGNyLpNUFUxorpTIhN4Q0w0kdsqsYoEP+yZswU07mEcGPQ9aZ3Ju9GdE7w/jvTcc2VT0CB+DiKKAx7zp9+LHLHoi5dvROb+B0Bn2D72AVWhgie1byNsEzkm8GXeP0Lcf8YllPtOd1iNwjqFJlMB+N/AU6pgScIN2swfqfA+u5nbh5OBxDCp9glpWz4WK5ZRVa9Dr4ZxiO9t1HdzHiq/jc5o6vBGKRLznupolPUp55583cNVf4EKogNWvnng5ttMjPcBPyO+5o+jp1MqbtuVgm0dQ7TGirwiouj+FhuBEsuPsla0lqgRI/IUv+5+hJb8Vv/cTqS36BkL6A3CzdDK7s1bhFlxJq48THY+2osrQI+hXD47wdi1oG7rxZZe+3Kx7B7/sBCzN1TD+9otn/Qs10ol8/+pgBrdJiPEp51G/YoQx/A+7qIvinr4Rz0oxLWpXVLp8uLsIuv7G844ZnNniR3C4kbA8P5oldt+hRtmJy0X5SDuEWN1k3B+cTFzMi2RfRx0WiRc/fPYFLVI97bNyeSvoJgCuTWMIcD9OgzaYiYcnUhzXhPvCw/TzacCz9iB/jdvOmbyFIOxLUK8Cco3debpLTG1KIcN99BxxhT5lUGeVgsBCgcnKB3Xw43+rHh2g6CBNnIi4wEbfO2Gor7YyTr2U069++/I7C+PvU/bLXfZkv8kgx7GkeNvfS3FHIn2sHmTfnUldD1duCn5/OcdJe5sU90J+eXcvg1wXUyubg1GuxlXmSpHmGFahgUSzksZ6H/ZozLw28zKqF1ifj1xgqgmubR8FT0Koe+VnPL3SaWyeTEPSdQDWFD2kw2YPtI/PGUeU6iGSLt6suHQZT1URc4YL+M2xBtnwLES2iWw8/T3DRpdC5nIoOwSDb5E67j6Zpb3ZKFHTTd/M146vMU+Vy26vOyS3m7FawSg1YhabEQkFdPU5RbznZbzypdTYjPzg44zXCCuBkaXka3sSprnFx6lv8P6vnen0tBMufYtwURTjX7EasAdtx2vaWDm4jiHPXBmtkOIqsq/pMxxhRqcCJlS5onNvoSC8hN9jihEHNzC30oF0UzvuIpAaq1kx6wp/t17hqxb4qgC06vCXz6pBFMIjwT30GhGvj3pE5/C7rG2yM6w6atUI9QbMnRz4x9HOIKOXfEZwyyrSmyP4a8q7xAqmMjhgCtm+Rh4HTCa2NA6ArY1Crtdr6CRVkZx3nUbvGJTtdvCCl6qA4X47UYnKMZ5KocedHtxe/hOLA+rY324/oyfFL+dBh44Tn3tQf1vI0HcqqTH7QvNjJB0O6HUKAnMTaU/tYJfPftTtsS8r2WzOSey6/zmCmjJ0oxToJ/zIGmsIU2I+543177H0r1dI90tHmlNKwPNs6gK7YkOIWdjGP8MSkfWX8m3QewTfDeC8solZYYsY6Dabi0XzcDQUEPg0iVDPaURXH2XMFNh/GUQCEz1dn3IlTMDnebE0VhYS3FBFRqCEHKMj32fsx9j9S3Jeu4hb2ySGnh3KFY2W7otO8Wrie3ipCohx+pCiy4PRGt3J8b+HotIdS1ADIs9+oInFpW8nSuLy6VC6ImtroMwSicDXA3odoGz7AQZejqEk3sjg+4N5ENJE9bhQ1px4lzqDhs0jO1FiAv2DrrTdiaDlx18J8HlKZ68LXK/tSq+WRoqrH9NWUUJeWTi8ah/Lfd6wreAJZbcG0Nj1BrNqOxCaxMhdBDRY7VSaLUoRrZEFpAt1iPrc5JqyjSvF9uvlkrv0F49gb/gTnFokrM7X8Lncij6u2b7NYeaKJJrjnX9BYbDXoAxV1cP1cVxUz2WzYzkDZP7kaG7SWfSMQdp3uNBpN1FSWJc/hsbVKQj9XHGYaUEhhM5KJcvsLjuHsps59NskLF0gKKyC5JBy/tQnUdJ+nUkq+PJpEIZyBWX+laRNOcSBoHwUL+S8lRZXZpc6oqWcOY6FeChLX74zu1rA1yOGna1J3NQ/JM5Tx5V+V4jxarTLqyT9BFFLeLtYwb1d72AxyGj3ruabyXc4qYPcBiWhhR50dGql27EwOuPNxrc3vmz/i3ohXQ0iLNYCikevZIcfXO1o5RVH6KuAiy2pWBwkCAVHEQHTvR4je7EWrsubQKr/YdqtAlIF6cxzsVcNVQocmVMOYyNjyb0rJaQyhP5PQnmQ8pAZzXEkB29Coouj8MkWDOPcWP+7O5fzipnrPMnesBWeNBaxuTKGam0ZawOgKKaCR5aL6K1Z+FZ/y1XBXczFq+nb9xPu/HSH0AcV3A2ChXV2/ehYwGq1S7NgsSIQCGg0V5JWmfay71ablQeVD3gse4hFFcaHodPBak88z/1jEIk3etIQa8Dm4opVKCYuDrKybZiLc9ja/QQT/5lI+GfTuLgJfF+MiVAg5OKgizxP8OWfaR+yxHUOCpcm4nVrILeZ0IZ9DDv1KfKnYizvW17MSyEaWR0b6rzY1lFNt44qEInY/uYD6jRned7qTw//I6T7Hmdv1mSe+ko5F3sTLwd7aXKjToQX/wLs/z+ZJ2AxmDg2s4JKl72EeS5khqsTrW2pdIv9hizZcXoeGU1izjT6vbeffj2kYEqlwSwECrBhRSkdgotZRT+36Jd97navG/HZAeR2dcbNnEC9OBORQEKJLZxbDVfpmrafxv4Dib1+AN/c/fya1Q9ZzSH8Lfv5NrSI+2IzAU/KiL2XQ66bKw5uYpB7Y5D6o5ZYEUss+NZpmODsjLvlGQ36bpzJX0iu9ypyx27A35zC5xWRDPFrxqN6M3PUEFkCAURTyi36Niq4dC4Z39AKnNyboekRWIy4dKRz8pU7fFSVSee70bS41SEUWF+OV0nDToJeP8p9fSQH/h5AW2QV7/d/Zu93r1nkZv1AnmsgXy9YRff/xlpzOfQ63323HGtLO6IeNqrMYJSEUVaZSJhLOl96PeVLrTu/mcPh7hVktf04GbmXuWoo1cZQ6jWU+n5yNJV/EHjpdYYu3g/lzxF7bkXo2pO/TF4cKtOwqdczjNZy3hT9CKEL4PoYEEo51+ZGvqWKoDQHxufNpf8bZ6DTNlAGgECAojyTip86Q38pb07ZSbJlJn+15zJQk8GZJ2005C5D7ySgSl6FV6MIk1XOjpornMzrgfujOGr71yLeNpb3YrNJSfVEdHkI/qHTmNz1PQAMtVqENvs4troE0CF8wPtr3yc3PBdGgbS8gE7XD9J19ETWDQzCL/c6M/tcx2duJfMvrADD/7NpPP8r3yH8byDz/5P2n2Tff+x/1YxGI++mvEvFnbfQ2YQYrvRHalYifOLD5eK+dN/+DWBXfylz24ZRUousfSSzf3yfpH6PCOp+i3F2RhH8JSZ6CkCM+UXZlo2c6mmIOnYhsMEBaQnutZF8UhNHhvIhnZ0KOaoDjRBCFN25Jq3nfPcvsQZ14aMemXj41WGJ+hSRwhtMbaTWWBlz+C2MgTpMYiGN0ixMQKS0B9P3DkGa5MXZ0SdpcrBTVFRaQND1N5RZS3n/qy/Z0HqXz76ZjlN4GRO3TeC7S++jEcE/PsMp7epISmMZ0Q/OcCH8YwbHfcc1P7irTGVR0V2y9UKawrpzklLWOs9Eor1BmsstzASy4ddoTOVyVn+6igVONqSWADp7XiTG7RYCwS7ushiLqhKBJBhX11Z6SB9D9iqI+wyLS2+2OfdGnAr9e8Bst/4vn02EbgupeT1wMBsZ93c/HiaqqY/8kJq2ztQAQ8dchqGXEKd9gcuNVE76FnBu9EDyGrrw1pXjDE26jCBRxHRjFKnuz0l338ytojeZXAXtHT5UNhahb3fAb5iVmKCbWG0CgiU2fmqGwI5qqq92Ru7wmHfb2/nT8d85c68xmi4SHy5fTMQ3LQ517K5/8WVtdnSmQCjEahNitUJe71cxWUWcmryI988vI/P0ZbyrajAHdeBamY1bjTO2GHhogPUe0SyS32BBeQOnk0Tsk1zHxfEzSgv+ZE17NWKhiKfRT2l2c8QmsCC0mamyiLlggIxWDw5LrcjFrRiEWrqqWnAR2jfHUy+Akg9afci+M4RDKikZnZ8RIwoj2wjzauBNzx5c8H+LZyb4ru7fQyOp2+x/rQUs7XqW622DsDq707PVhXh3Hdyeyd1IVz4NH0e3Tp044nOdC6kHsQlsuArh2/g/yLyRQPKO8eiUeoYJAvgquA1X4wRSu0Fa5Uho0aNqaqfCIRtBsTdiYzN9Ap7z3Ag7bA/wT21BanJEo/ek3EtHpxfCwVXVUGt14fujyega/Ui6L0etHUNu5BrGq2CuGvTWDrxSC8lz9qS6cxuX9dfpoq6HGwfBox/xtgpocWOfzkqLSxU6+b+aaNe778CgXsccNxkJj5wQGyLZPTGLm9W9OF02kh7et7guEiDSeeLS4ILQ3T4TGkLgSBusvxCPo1xFengZacJmBjue4sYh+2T6ra4brzdnEN9QyeR/Cug18hb9fK4xxwQHFc0cVQdgXPkj+qe9mf3XbAr7G7gZsI96Czhaxdxtj8J0MA7XRCN10RaqvKuwEQzBMwHQNKmZfHA4FnEtEc+DkRht7PGG5w98qD0BQqd0/GIHo7IMRx9Sz8fCblzPc6efaCMNVa4I9VYsVVkoa40ExHTm+AU5Bil816ah8O8J5M74f7H3n1FSVd0bL/qrnLpC55xzbqCbnHPOSRRQREQUAREUFfBVUYyomEXFhGTJOacm00A33U3nHKtDdXflcD4UL77/ce4d93y553/HPe8co0aP0bX3rr3XWnutueYz5/N0sXXd+1REglgAe4vdf5dmD0B0MYtrT39AI7eob/Vk2SMuhvyWPnS75Y93zX2c4RHEV0s5OTOLw0YYXg37J8ZTElPF+2ve5/DoK7h2usevb9tZqL1Pl+4lVmaH4dXqg8cUP25ntdPs00yTAzzD67iRqMdLKOXg8wcp+NFNf9iUWMCvBqg5l0WKUYvDasO72Zsn/OwsCd5Om0NBYsQFdBIzBxqm84vcQbvWiFkkp6C5F06niPHB2ayI0/NMeQPRd7YS4FeFvdCXpswPeCEIjtjepWufklRrGenHXufc0Avs9VjGU2GTSPE7j1GzjDd7n0XZrRyhU84A7058O90VbajCEQjvYNAasDldLPp2DAXxYShGNyPTn+NQLIiLTMRYu7H6k8/wGKSEtTdArCbFbxzq+2oC1eGE+8QREOWO1tbb3RpxcgGMV0FNpw3lkhic3kbEhpusHzSen+98xN+VPej3hw+24edZ/MxZdnbEQq2L5EfaDaXJjeSHXeQZz9FEJLqDXYfD3boIpzq+4py9N2988Ab3Uu/x/YK9PGnxZUBnOXhEADA48AG93yzhu4sBfHIqhcyRzSR71LNC8yObdJ3YVE4MPlbKIuvxEbsYSQ1cmIhJ9D1vlv5M1vAWRn85H5HYQbVyHSaXmxp44I1Eel+cxuVZ9Xz86scEhIaSDOwMhLWWPObWQ4lFxOCbT2PpnEDSqO4Mjniaew1DaBB25671eyRO+Oupv3jPG/wNO1l1xb0p3zYXyvKiCL4ZTobqKONlx0moPgmSDvDtj0AgQN2hRtempaIhgSf/rMYhU5GkvYardhkWQwg2kw1vz07SZKCQtmA0uzM7DbE29nfCg4ctNF7rxTN9r1OhbmS4EupsDqpv7kRm66SrWzX9Ip2kOAt4xwxhJSo+t1Uxr2sq2RUp1O9dyrF+d5m+5G+Gzz5NURH4CtIp6XmXQEcvzlgdjHyUTPptO5il1TiFMmodYG3zoOeOiaQNySFItxpingNVKOGOYuL3T+G8RsAtSR8irNdp17o3/06/D9EMjsHeZUHeYUBocmKVwNGULew5/4Bnf+iPKV5OY7SGVGko6rR2WlzuZAuxtovzA88jDejLax+9Rm5yLrtn7H48x37dHEx3owSrlxGtTztCkYN5ziG8xnHCmp7DkPgETRYXNZ0RqEQ2Cv3epLZoA3fC4FbbTY52hVBgK6E0oQV7xH6uC9UMHnYGNO7AwYLhr3LXtzvtVwTMzX6S9WPegQN/8aVHEOe+egb74HouDS+g05XLa8LNLEubDsU/Qt98ChLyabDup6nDjPTRxvv1ftPZkfcWpeKVyEVKClOTOJp1kFrPdchOv8y2aT70d53j9HPelHdEAE50QvgiLJI+3APucd2wDO3MSu5fmItmzyX03nogirlpb5Dgk41U/DddxgzUDVPxqyzi1oVMegy8CZMqQBVG/4NDSB4s5NuWSKp1WqZ2cwMmyHwIrglm4MVMbKM6WfjTQjyz8iCjAuKXU+CaQ+ucntyfIEb2YDFvHx3NyhmXaSKKcG0uLydu4brazv3OTpI69TxNFyXmOgxmX7TyRp7xvMsioy8ftOWhLl7MTQ8jMzMeMEADdmcl/RxDMJ6dx6muIDokHrQOvEqIxEBI7HcgEHFNmETUje7YJE5W/7QRmfRfcK4v+A+HpFV4iD1RWMJxYaZe8zNWSwyURSJxNHK+fDEAkthIKtIiyQ8ZT8r0n6GznJIDJ1mcvhi943XkAidCAQgdnXBnFfgN5qFCQWfKXZrL6xgX/C259kyyFDuQGHpTZQ7Gr8dpGrOucr0mhVPDDrJE/iK/3Xsfg8WX0FDwKbiEtjQHReloGv1qUU4p4csxM8nteILkhK3cOhtJ1TU/Igs78GnM4pXnPnTvAR5FF+5e7EbrnoFcm7uPbaNz3H0lUnBU18itwefo4Tcd6yNquX+b0WGjpfYh2gYjjqYWOq6YeV+Yg0+qH1fv2GgyhvFAoMfh7EISk0eDuoYcUSrX6xr4prOc2fJKTn7Wiy7vq6hmdyc69CE2bS/OlLTyQWUGLwxaT632IWJJMR6dHkhsYgQCeK77MjSyZuBP3jilos/ZNOY3+6Lqf5Pk++7KQ/wGUmZrxVKaR3WFmE2r/sQoK2Ngyh+kucq595yaeQYL5f3vcSG5L/ddUXxFCZweitLYTolpHCZxA412Ga80QbrvUjaorYwJhQNCL/6atx29rDvNFfd5oY8IqyIZL3J5Mm097+bVc13nj8P+HQXHssjRaknROcG7nBPBIhK6RFQs+pZB5wbBjpkce/8u2w3lHHMVM1GwmJQQISGVoXR4tHJs1DEKtEXAP2DfW20ZPLBeQu5I4MO011CLWmHbAp6Qy9h4ZDLC3k0M8dtAjnUfnYoHJGmDoegbXvWWk6Iws89i5lKPDTiERmbaPsVb5MRbDC+eSGbUkbHonqpHo4CFHn35IdANvr7SBJ02C+EyI7H3suljECLMPEynQ4CHyIVO3M5Th1KRm1SMOVvHztz5zJjeg/uNcdxvHMLTQqj2dvHQ9ypP/zGFpmMmSiYtJ8rxNSiDMdjNZMkKccUUElQ/EI1Ig0AgwOQ1kOVHtxJzeChl8cUE1PnRpRIhFApJ9r3AlITP+OR0DOF1XjhG2/jt7gc0BX3A3buQpvwOp7CDkyPe5+KAi4QH+DPb9y4HPV7Dx+tL3vaCQrOJ+fXQZAjn4h/LEfY6wuSJWwE3XetiZSWZAi8SZVb2RhbztEbFVom7oua0EYQuKz1kcCE3jLVfPU9T95Go1/2zGRMIBPz69K9ozEkkGF7jZkscr0vcVT+zdszCz1hN16f/JFEYJWWs6T+VSyYXrwG4XMg9bUREFaMzXeFXA7zQCLpaX1Z8P4+iAUZyBy3BIZaSn1DNoR9PIhFaCRFew1nXRV1gF8dHHmeml4VxGgvjNFBolmG7EwlHevHA8ABFu4L9db2ZGnwZsq+iiPqOLH0/Mg5Y8A8Io6J1KR4SB2x7tIuc5OJcYAKuqbHEpmeztaGVVFM4M5QRuCQaKqIa0IZpKehr4WHGLWzqDcTVr+f70ZlIVFaCyqw4JDBr5wz0QU6s0wbwyp7rNHaF85bfFqoPTqSmfxtSy0PsbYEs7bmCkdE/0dAZwQ8bliOt8EERkk/gwzoOD5MicLmrbEbJ1yOtX49d2EF6Th1RkXVMaPeGSCho7gX1p5j1/GE+uLSbiF2l7JRbePmjbyDndQgaz+jzT5A37T4zD/xA6uVaJq3ZA8ZfQX+D76YeZnDcBRK8nqbflX6UOO/haTvP5AQH39z5kMHbehJcE0xYykQqjC3ItJsBmJf+BvmXfuWdbYPwqw6mesoCJLZKrvoOY23OIF7QwkVXBL+/9S6azjiCridj9/RipI+eBRq4ZoY91iRebL6NRGJjXHAbM9WQbYZcC6ypdT6uHguo1zJ72wqOjTrJs2V/YHPImDEv8LGGpLbVg836PkzqWQSJq5nXlML9PbOJSJRwnwtuJgNpNZftbVxog1ccx5mfsJaQeBWvXk0jom0ArgEKfm0KYrRnLbskdRROOYtUm0qfWAMKRQAChEz3gHe8Ye6ZBBKPZNGycDubhhWgkMLu6v585rpEad0A0nJh3Omnyctw8szlr3hrcxTRBWOpbE/ERxyNyCJk9l+zaQxScGbweTaHPXL6aw5Cwgr2OMUow6s4kV7AK90LGax057KO1YcT/NtM2kIruTuknAZXHgBDFbDSE94efJWv+lzlWd+3OfcorjxYvo2tBggrg0n6S3g2diEIFjBt93SMz55C5teGwyXhwL1PqdqRjE4npXjoz1xuk5LaPYnydiFCHoFA13sSWR5JXUAdVqGDnLZIegAvhdznuHUzv+T+jkgJNoc7hiEEtvR/kakZixi54wMmXlmEzcOEKUnMJZ3bb+rlvRdZTg/2HtHifNLJg10PCCl2r8dhYvjJH46JfqK1ZAAtxd6se3cd13uVsG3MO26fxCoh7V4aUl0LB+dfIyG3npTl3/GS8zaHLB48W9OJv289xuRcxAmLKEy8T131DjLVi3EKTfwUMpK4kXE8k/QMzcV1tKqqaMMXiEQoEOIQO7BL7Ig85HR5OYiQSonp/Ar8nwXgmTE7+cS5ha6KVjJvZFIT5iLJ9xKz/evZcTGYknuliF3gegTgLbi2lj0j/6autYjIJdu5LB6APC+QqXIj0IZO6aK+nf9L1gCobHLsEgc2qY16ZQxkvILr8h2+Tmvmg5Bmsuv6MiLkPuP870HRagBa074lvuA6kRUp6HyTkNvS6a51syeJBCI0Bg0+TXJM1RrCS4dyrH8tjf2awOnLUXkamj+mUjs0AauwjCqvsxTo/Uhvu4dHw07OT4GnDv3Ow4oLBJc14dHpwR/qStjtiTXuOr1ve+HRqWP+tvHExlaR3GMbMB+A8aIQuokmccY4gMbvOikYfp3oyCOopTBGCYUdYlQmFRVRjexe/D02dSzvUQbNV+D8eJTKNyg0RKJpHcjEvyPJGHSbMFcrj+qcsVuLWdD/BgWlrQzc+hS3DXdpWP01DWenY7cYETzSHQS4ZpQx/PANTk3IQiy1kfXccRp7raXfLxlsqFjI219uo1fwPqbutNJfvAXzvhq8n/DieD8hheFXSXOB3Snmr9x1mCU+ODUuXHZoMChwOYUw+BicG408aBz/2t8Tj+vdaXplP42aIl5sewJueEHYDPCIYtO9bgzYOYMEhxqRU0mq1ATnx0H0s9BrC7/or2EIOo9d04K2qwc+ykg6PYcQf8uTBVunkx+fDzY7UoEUAe41xS7QYa7zJe12DzrrO7nx3A2U6hqyqm+T2NjFAcdFeAT28R8gVX10X7xK7gAQVxRHmwscKh0OsRSLvoumICUjpvxMzCOGhAbNZVRNg/+vDej/Pzar1YpcLv//fOD/F+y/YN9/7X/NnE4nhYWFRGgjmOGjwAcTZxccZkdLCCVV4fQMLkYvCMDbVY8rFtY8VCJ/+230GTXokwvJ9q4nWBbNnwb4uU3M06Ja+rVHMS1+Hy/OvoPB5cOhy1uo8TiA1tiD0ioJ93b1ITjVi03Zb7FkQxHPdszgnRZItVjxUKt5YIUUz0SMor546Jbgqj8MBXcgaDTPeRVzQxhJTqSd38a6ha2HKGBPwHW2Dp2MK6EbElMeWquWDnUHiN3l8J+0yMit9UIr+4ZBBdFkC+ws1QSzzhte1kFIyQxq4q143X4e/0YvIp3/BOK8XV38nReM3OxBD+trGI2Q4NHJEG0jn7RCmDITq6Ybwth6HGIH//IFuaOcfmHlALj4nZEP/Qk7PIGjU24xzOLNjR42Au+vg9Y7CFL2PuoL9+fJPU08NbudMWEfMqt2CoXFv9JRnsHJCVeo8ilggedJvB15nC6bz6hqES6hnUlJ0Of9T/nYy+0oxHrfIllbTHPZr+ASsHzA82QpTZyxVGBs68tujyvEGNVM/vpVipLqSX32HWaqIU/6Cvr2z3C44I/GVPofSSAss5InUv0wq9uRKy1usWXpl9zybmJbWCWeQTLGHc3ifmEQAydfhCZ3QPxs/YfonIe5VxBPRtxw3uo/m6e3wY4OmHlvJon5ImQrfMmeuJiz6e8gANpXVrNl/5es/UqHxcsTV3oZAMXCWDbVJGKWqlkQYGf48p18WDGWWSWJbIpvwyk4TIkABNYQLt/0oN/BKYz2dnG04m3mv/ULy306+bzN3Z8382ZSd8WKst+zdAWE0M+3jQ8KYasBskyt3K+PpGJHKtEeCp5sP8CiqZvhzAgYehKcFtKSsil0DaLFICbgs5fJf/5vMgbexUenR9JrJBHD0nGc/ZMqsZuiyOJyaztm66rInbifwd5L8HtnAdlDbzL+WTf9bYJPNpF3dlB4NYUdax5iGbKNwQFxVJkbWNQIWs+LnBwXwOHKMWw2Sritfpcpj8bnZl94r8VG2cXu6EwaHgxYhF0uAD5hrR52PVzIz+N96Ce9Bam3eNoaRVHQe+gEM6BqL4hUbOyYTNQaf2LTXejalMRZF/HD8+7qvrrAAsrN9QhlUjYt34RdbGdPJ7TpM7jR2JO9w9ZgMIcx/uIcXj78MrdeNiHwctOZBAilTD0yjpbQLm7EusgOgbiu1Ywf7t4czc2ZidelwQh8PPCJrUDn4/auwyQwx9PAL106cgVdZNmFBNYFUmduRAAkScHD2sUtSTKXF39LqvopTGoX2c5spgsHPH53P0ksIv/ZDszxn/Jx9PdYxU38CQRF1WB4egcN6kBS9R0MvfI2qqypvNDjU6QKJxOrnuPmqk/o7bka6Qc/EmyQ4tPwGkcjaznQBdftLcif3kl70EjOhrgBPnBrNcxSwwN9KWclDjxL4+h1YQK2pFsQBj9W9SW34Hc+mB/H7aKh5DhmUqvuROAyU2N3V+vVWEyM0JipFlixdtoZcqYXpZH5FM/+kKywGISXHfS83pOUvBQ8XwnHFrCBJr9NhIohok8er6kimGZRoQvVPW6HbSOO83UDeE3bhZ+9G0k1M1j61VJCJ54jMfoc19qjqBMI+P7ceoTqWIp7bWZf721IdZ9yqbGdP/vP4lJXCC8XeVDSpcM1/BgTAzvRVRRzxQSDqyFEVUFdbzPqLn+cjoc0BDRT2DWRVpM/MpHpH5F1J3h0yCmXygALhEyGgOFMEa4lxeHJ0S4nHR4mzHIzjf5jCFdF8/OxZWha5Yh8xSCQ4yFxQclPoL9BlsfzjN3dht1fgX7GSN45e4D1I2fyarOBPzvge6WStK0zOBBs47OkvYg6RNyT+XK+YjbVhgS6+0/j9qAy6gK7EBSdJVPak0/UZp4LqqDdIWKeswlXWAWH7vVHfmkQLw3/gor2ayTJoNaZSwYgGVgPF7rhWJ1Ar+8+gUNxMNudwb+j/WVee3iEQVf6kHgfJP0PQ0QUNGrYUDYX7+6T2NRrMQUDtpEkAZHTvRkVCazYnHIEAiHvmk0ILQKeFQgIErkrKYYpB2D1sqGTe9Ll0cXzPgWP+7u/bw43fxpNamU4YaJmhK2txAkNvHoyG7Pdg/BMwf8Am17zBHP7tyzqkce16knAcDzaZSTlR2BWGrlvVCJ/poGxGYNAGYTQauDr1bv5vQOUa1MQmY08/8lPhOvyuHGhJ0e2j0UfXcpkhw/veZp5R5NPt5gtZAYd4eDecZQ9iETcrRXHFQc9E/zY6tFIVDlECeS8F3GA6sY4fMdlM9GvkauCQUARDlEnZZYL6M8OpKtPCXcH1LDLvwFv+R1ya7NY5WknRlxDLwVMzBWz7V4IA9KaUajM9FeAnwgMHims3ZNFVFkSRr2WNyoVHFJFILO5s6erhVEYLwVgTGikpcDIV/c+Y/eKt/CSdiJ0SXG5oPNeKSnnd9Lkp+NhMjjlAXgIc5AbVTRYXQw+m0xk9QDG9nsfNPDu3VcoljTAuAN0tPSiIiaQ0qhSJqncc/crDQquJtdxOW0/WlsCO5Pc/djPbuf5yvexmGP4LvJJ9rVIObxnGmK7iOpJ3pw1QpUN+sV5s7kuncRtfUiIkPKyCoJJoLpqLzjt0PNbwoPyaRf6sU+cwBLF/sf9PkrVTn6rNw6DjfagThpldQgFImjPg7z3eKFezXcdHSxUORitD2Cq2cHraR/QL2whmUGH2bdlFAUXu+F64wLfBUOO4wHXHHIOFi6l3gEbemzm42Yl/j9PJFMgpOfq3ei7gjhWsghh8iq27VBiFpZz56UFmBV2hgpm4q2oJUhdBFIdm3/+BN2dc0Q697G7YjKqpRtJkPmAy0WtPJqKkT9xr3gC5coDHBA2cXvQfbycDZSl5PF9cDPvJYzj+MjjvJPo1h8icRWOEjsLGkDoErKsJhnBeR+KRnUS6lNOlSGRQEUDb5j8+bJNSsfvI9loFzB+VBQ366K5WDmTYbFTqf94FnFJNYQWBDIk9TRhRndW8mhnKUXG3pw5H0WsQoDc1Jef0+7zapOBt9S+FOkzUbvqGXF+MAaVgI7pXiiEtW6fqekyxC9ltbqJsdmZnKkagG/JMEbPPQrZc3GYJVzd9hBz1LuEhrorNgVCAVyZCy47BeVOyjek0zHiEzZ5dSdJKMIh8QKfPtB4jpOd4zhlu0tGUx6bPlxJwagC1s7/jCT1VRYdLGLL6HjqHTC108Aor0bWStZztedtfsn5CKXYk36xH5PfmEhSQQw9jCr3tYEU9V9gfp9fK4bi8+d9GtKMyF/+jOtCHcP+I434Bw8nfgPP06HtIKAU3o55jsVNl+gMauDg4FqWhqoJurmRSUWTEK5SMFnWheDiNLKfPUaJVcjKqln4FlxjVL+D9PC5xaJJMHt3Kzc0M9EYlBhjfKlJuUKIdDM7DWt5xwcKGhs5klmLS6ZF2/ApSwe8gKS9g3kGaFQ/ZKZjNeUZk2nSnOBa3Cr8XekES7MZE/Id9Z1ugERmlqHuUJOXfpePBrkpMYl+DnSptDj3cOiZTfhL0nHhRAhojIXIS99FHv0j9Y0JqNvM9L/Yn5bwRzyhEU9SWlnAD89/h86cwdjbMymtTmb8ayKGxroD+N6OcP4atYfvSn/mcJ2AnXlvMXhiPH/8ISDV7yx2lwKA5gA5H636iEjVIFoMg3ipcTvh5jR4FJw3Ko20erVyXjSYvzt/xiAHvcCKHCPz/ppDU2Aj1Yt/YZOHCLoqQBUOgNa5Bum1UjLuK9m08xVWfPQRas8OGsQivl/8LeMC3kNx8w6TSxbwx6xX8fTuBlNvU71tCaPNpzEI57DHcyB2sYFW26ss0NnY6APDams5OfwkWkM6c3+dx/Ozr0IgTKmFM00ZGEa6NaIWbX+evCoNQeJQRpe6K2R7GaPoeV2Etl1L7Ph6ekcdITjva74Z68++gpXAQjxFViJFUBFtwSHvTqJYBzNc4HLhshiwidz+5fT9I9C092DOgkxsXv7sqVexMCeVptwGFv70FLnJuYgWCNn94HWOFC0hufIlgq9mUhp6kFH12fStzgHvj6lviuSWPoOFg1aTa4Xb7WGsaIbPY3oRKDWxRAcvNHYS/NoaRCF2dOUPuCCY/Bjse78F5rQNJPvjJDIGFLFOW0yhrTcrdedIkznY3QmvKiPpcSqS1PupJPtoCQsoROBIBrF7DHgIHUyTeVEoqOcd+Xz+03IycshypiAvNjLq7ChyetxiX48/SZNBT6cQn23DuKCUok56i2Vxv6NsMjFf46427+XdgXToFYxBgfwdHYAAAW8JjWQFH2Jo5O8M3jWdIfsG4Jy+i+y+2QQKIKnkWa7VTKTd7MtD/RskNGk5u/AMRbocyn00j+/L4jWSNzOeRx7SQJnjM57I+Z2x8w4//l4ohCsJfXEKrfi3v46mXYOXl42M2z25PsjCCsvPxI6JpenMYfLCDzLQSwX1oJa1IrJK8WnSYNd0Up00mZ6Jl+lm/BdEyUjwvYLBVovX8t+pyllF7KW/CO5jZGQvt875oaKXuGfLQK7R05bYj3tRzZgUJlKdZl7IXEJ37xtM6m9k5YXjfLv8Cxqj3WNzR+6b/HH/PQ4uauazr1fQoA2jvvcDWj3qMYhD0dir4MFGzmrz+cUFR/w7iRpxgcjACrg4BdI3ctSpoiqwnggvB58v+5xkpRqZs57+YbspbkvhyMFwkh8ko5xQyufjRj5uK6nIQv+g47zS/yIKkwp1zEl+jMgjsDWOSSqYWQepNhUOiQOFScKM3VO51beQIX1ymeKWZ2P6VQVrv17LpYGFFA84x1UMjNCaGaEEL9oYoWhB/dtzVLaHUTXgIpkxtVRUhOHC7dP37yql59aF1Fp9qK6Vc00bybgBLahvrqbHyVJMnaMIFgnpc3YtL2z8hj5WKLaBUmShvvc1HAHh+O3xxVrrSzdJECsr43i6rRYfp5Pm9PPEdUazNXsfUVHwZGofUkTwcv4EiuPauKnejsqnBQfwi2kKnxUMIzf8EgEIkfvpKEmqwb/UA7HdRJ4jmSfyEymR5rOm7RqzVH40Fgcgd2mIqn+VoKgfACvELwNgZ1AHrXP+QGL3pJtfB8VWtxzNSJ86Lkw4RFLMfOplZvIfUW0GiN2Ve8tlMZzSjyCoJZQPNyxlyJhr+A2tZl9nLbV2Ab1tP1J2I56OGAtJuSncbixgQLeJfHvza5RWD4LVSQQHCWn6cQLbXHa+/vwb1gExAaBy5hK0oJ0vmpX86VWO1KngnF8jnY8KxxN9rhHQdYPGOz50FreTkJ+AIdLIMw1fQ1EXWSIx3mYPFA4VTqd7PhQDX7aBMD+U2JwoHFYHsw8+wYgX9gEQJHbvuXfcvoagvRqpxyuQkoy3dRrwO6OUkCiQozs0gdYedbgCXZjlZuyPQPJacyc9ZDBgyH0+zLjPwJKNnBAtQxTigUi0GFxC9D56sn2y0cXpuLPhOwy2PzjmSORDdhDpGUmCfhxelYlYWo08X11GZaOYvGG1JMd1QslPxHV/SF3hADru/sn4w+PZPeMiR5uf547sMiO/G4gh14DnZAcxD70IDgli34DVtIiX83z2EooC3+WJ/pnMODiV6d1Xw7hspvR18vshMU9a/oPf+/+NbUFEhDAFix6+8PbFy3AZeAWhPIpFt17BKPchp+cD7vheQ6eez9xANdQcRoiT2KJYMm8lY/b4GaHNimDZK4A7bnlqxCnuZyoZeXYU0fe0aNI0QBMAP/apYKzrCosHrODbJb9jkdbxjmMNpKzD5NBRcv5vBDZ/2obDT+EbsIvtpIld4ITU8jkMOTuE1NxUnOMvkRHWQKN4DAk+V3i557OYjHJy90YR7qPi9PRfWZjW8vhZN3jDnyUO1N+v4t5AFYP6NPNy2If/NEbAMGSiwWz6fhc+5moYYcAZU4dA5vP4kCphIopiUHd5MXb1V3gpLbi0X7GwcBh9fwlksNCE5XkLWdezaPaXoLHGw5gcXl+jpKAujE8HQMmGP1E2lGCW96Ctpo5+obu4Xjgd77gcdBEhHI1/A4Abi8q4dlPJ4Lm+DLnlhdrmYHrrYO4OOIhW1oOXtUmQ/CZoErjmV4fHQCcOkdt/kmECUw00XYL+Oxks+hdC1yVqBwn5o8cytrb0paRnHHi49cYP+b9O0bxLjMldy8pTC0icl4RYLKNJ28beKft4YcILJEw9T7+i19nyUhEaaTNzfSZgH3abz9Pus2zQMg5lvg6nBlB8L5rvPlwCL7eBW0oZiZeaku7T0TYVY5NrqAtqwbvFm6qQajxc4FCpEdmt3P/sJBOtDobIu2Hrk8u7Dz2x2IpRMfj/0ZV9/8Y7UlNTEYlE/7f//n/Bvv/a/7qNjB6JKKSIk+e/IjR+I/Z6CQV+Egb5n8f7H0kGBmg7ac3Koy7Ezvfpf4NLwGhlD55qAIETUqoTKf05EeXE8xxJsCFxaVkVfobx0e28cnwjv/u+i2vKXgZffAKd/gAqp4JZarc2ns1lQd7pYsHvc2mJ9eC8Yg7x3tcYMqADQdwLIPNll6eWhpVfY7C+/5gqcHsAeIrthPZpptYvmvC/YMDVFWxavolAfxNcmEqMVUTT8R4I4wVsWvceIqccl8vFSSMYnfBaYzj196I5mmbi5nsfsCbQfe1RNTCq/9MMP3Uf/wZ/bInH0FpLkdn68mZIEdo2uOcU0hKShkXkR2hlKNM6ujjXw4LI1cWLh3P5eg482S2fLoWBqhvpjPm7B899aOBQ1gMQCKlovoOH5/MYWv2o2TYaaUsT1ycNYozfQK4+lOJdcIOuihByuxfSLqjkSf8/idEUszjzRaZvOYnSYEA4p4pFoe6A5a9332d++hs8nfoO2kPP0NUkJ8eaicXlovuUPwE3f6pE4KLNR49JaWJlM2xoUnIt+k+SFbA/CNa2W/ly6ZekPhjFxfVLCHh5J8tDHjweCz5KH/wtb6HVgibvDcrNUjfYl/YuAPGCXBTHhmBObCZ1wbOYjTLCHtEcnRxxkrrMUDIEAkQuT5Z53OGZOZ2oD4TQX9Cfg9eGoIg1MVHUxljDRCbGzsMSAbM8BJgs4aQq4Xu7nMTfZ3Fj8C1GLzxBsAJKu51jdONyOrRO2n1huH8eKrWRUQohazydnDMKuRS5isqRNhxit3csFUseP1OTuYb9jXFMup2Kl9CAt/M2xv6B0HERao9D4AimXzqIwWjk6YxwnFlKVIEOGLCX37eraeiMwuEAsUOBUCDEKXLS6YIlTYC8Bbq30M9XRMvko/jENXJDsg1d/ae8lLWIbUIfHIHV2ARBlNrgo64ZnNBvYH8Q3LVAc1ckNks4mR2Lua1+102dBHiJIF4SyNfDutAI/HA4ChGJ4G3fXI5fbMFiiUAo/GdhU1wy89SVp3g4WQhz3C+37cFLXBhwAbl3P8Q1dpqtjUz3f4NOjxyOlRzBhQOH00W7zr05OG2ELclf0TvLQve7oHJCR6Sd4kEX6eur5mJXHMNqHrJSkMni5Tv5uCOS5/6cSi4ier+z5fG9JNg6iDk/mJxhJbww/Xc01gR2dPox2qORpAoIswvxbfalMqKRD193O5QqgVtTcpf+PjuEKTQENBAjcuB0up+lj7gI/g6CXr8Qpe3EmPWAQpUS2l1cCYXrZljZ2UlOZD6jbSIMOWXE3NwPN2exo3s+qZ950FmhoN7hLky4MvQuZkc9nzqKQAlDlDCoTc6FiAfMVS/lxKF1aDO/wz+skZNGMDjhB78ymobmE9zixzt9DxESXcOEv9z356F2UeoZQkOiiwaLmFKhEruo6XGbCAVCfvPu5HB5EF0VHfS+2g2H0Agyb1AEonTeJmPMFe52z2UY63AJ3ODQv3/bv20SLjkMWDOAouwWqg7moGvT8XFCK09EtxD/sA2pRsXlvjksjG7AIZDQu7GURG0m4/82oXBdgxXuexUg4M9AdzLC+/paer29jojEJn6f8ROtajOvBr+NvvBtzpvgR90lJi8oYMiDnuTKryNDDXaYt89NwfzRuB/4vTGe83eSCclO40ZWIFemzKBvvDtwMUrSQsm7y+gdFcIv8z6hU1HAWtbiiniO67/KWZZbhuQLKQ8GvUBE33Lo+QNcX0RA24v0GzGSW+b5GPfvQFhZhni4gT8CYHsH/N0UTla5P2KPTmTtt+hSmXAqo6luT2R2yrvcdHxMcHoZJ4rV9Dz9HBXjXIji3M//TX0or3rryVAKee1MC6baNgoXXqXHQbfHnCWJ4XkfX1RzvifPoxF1URRCoRO8ez3uz2jrc9j2JuKQKAiffoAITQ74vYKi6QSO7fm05IBrvPv3CqzQ5D0Q39DxnN/qSbeTb9Npj0InPsBzmZtQGQ6zyhOWe4I99C3Kh6XyZtVsssW+WIwGZEoLP7TDm3ro5RAhtovxHKdhaOCvCFzjkImELOz2Cg8kS5iigjoHJG89xRv+dha/a2JC3BRivW4gEAxnyvBLDJ2ylW6lQswuAc+J1zDW1g7NVxFos8i3uasn89Mu4yGqIlznBvBDQqswxCYh1CrwXP0S63rdQjkoFqX0CBpZEzfurEFdmc+DyXcJfvVbClRWJqvcdNoPrV1EP7uL260+7NOH8H1HO/NSnsRfdJ5hSjjvqqTg1Y/xJpNKj/MIXEKumbrwEdl53wv25GZwcmsSo2VK/MpHULVyG3HdHzJPA5dMYBIISMxPJLAuiE9WfYFDZMQQ8QG+tXvAUIRB5MvGN5/Du2MEScFybH6nOTX3FBX3wvn1z2sIKnKRzcmiNnYgLX6nEQBaaz2z1Ofo83UuE8qeIrCwnJFi8+P+36X+jsUaKUt8YEy7kfu9Qrkde5svPCFUArtCTLxQ04JeCMdbnTzfABPU6ey/9T3fjE3iTv1I3tnTH+mRQXj4NdDq1cKwxlcYFgiBpZAmD0boFBBaFUpIjQhV0xTujmuE5mx4pO2UUhxGo/Qv0iNfZEnbIMZ1vMKzPeP4+/wsst/6hD6hexh/4gh1DTYGTtOCdycMO8uVD58lozQacZSR3psXkzDwNuXdnsVZfpAwbR7nUu9j9q/CuzEAj8revND3KKtmelDQ3Juvsj/nlOUvas06hjtE2Jxiar2mYc7chn8E+AbYCHz7AF0eDvTd9QhdUgQC+PzaVgAOLgSHWofBJ5JrmUdIjz9Dwt234C4w+iYlHj1Y1vQTAU4JMoGTMAl4NR4lPH81lxIgqawGgVJMdt9sIsMerX/7gklR9SOkKgRduxdhw+aTrywmPfgdHAIJq09doVJ+i+7HTuFwSbmReQ8nHUy3DeTXyYNoD13LkyfT6Bp0jpZgJWOXbmHAI9qwufXQ6uxijN8X5A3vpMJzJ4XBb9Lp0cnuh0sorPwagQDeF8zD89UzLLk/mNE3BQwLDebjyBrQpYNEi0Xgx/1zUhReZuh9G98Q99rwhW0Iqi4VVquTknt/4VF8mrN3TUx3HQNVGFeFY2hLv4vNLwj5uQwarTKmfCiD1Leh+SrSM0Ws3LGSgswGrva6SqufnJi+Q7hwUMQT83V88EcPJPkh9Bqcy+YQJS6nid4h++kdsp9DlW+S1OMADcMPILeJeVID96VL3QlQzVdBquO04huE0+uxqMLZZTdyqWM0w/jHGkJbyYu8S+/swdjKdZTExkHHX8Q9KqZwuVzIzBKkCilLQ5Lwu+cOel0Pg09anNxz5vDihq9I8u18fM2PRvTlxu4Mcs9ksXXFaXRKSBHc56v4NgB+6rKwcNBldNZkhl9qxNwiRunvrnQIF4OtfTOTh55jW/l4RuQ0kJEuYl6gOyi15Eg+B1+AZ8dc5+7QC+xqh6MCONf/Aj3C3AlFg1Qn+KG7gTmNTfTK0/GvaBP+pe/iUkUyeOJpvrz2MyEt5xh+ejhXhrqzsEl+nebGT7kYAp/WG4jNj0RQbKOb4DwNZi9+quxPUfUT3K4bTYc2ih5H3qGmWIB1dgrnK9I5XzGHvonv0beslrMYiYwzEirqxGwV83U7BOuTmSErwausF5dTC9nypJti7QkF7O+E/e1fEMIiuj9xij02OT4yeFrrcOtUPQL7ElrT8d53A7PCiN5DyneC46yansiTX8yiLuAENp0QQXER0QWdpGkHkOiXAnI/Lrf+RP2rn6PsUcBbt17haq+rNCSVc9kE7+gh0t+XOk8XggIFoXVqVGY3WGB2QWBnf+YH5pBnhb8n5dGkuMRG4T8JKbWS8/R58h6VFjG3A+t4KgSatLMIdexgaa/nMAgW8nyjAeuh59k17HeWzVzHvbYRYNsDQhkCXKzaMwtRZRCOeBn9Ay+gMfpjkc3CpInk+5eusPDZN3n38ps0+dYjEgjpsnnSZfNky/QrnOx3nzL9c+hvJmHsVsySwZMAmLy3ig1qCb+3iei8G0RC10DoDRdNbrr5TidMD62i3dcLV+oL6LQWrMiRYqbVARqf9yiKriCgQcZrG1+jYV4Z3Qa5fbrv2mGtRE6fG7OJy5eRsrqIaelfgLUKxCEA9NSEkloYzeWSQbReUmIefY4BUy4wsw5u9LxBlPRFwvK76HO1DyUxxSgFzewvWM596nCdiMM7ro2a+xY2lb3Cm1s3sKMDtviByNXGtOQTTCgZh8rgpp7WdhzGJW/kh1tfUBd4hD1T91ATXAPAWSMEFC6noTMSi0PFqbQiDmaeI1gdjMYIPrJH1RojruA0R3Kh9ikCdbU489pouK6mY7YPzGgHlwthw3kaPJW8fzcax10xr9x5hbGf/ITA6V7P2s1uXlqny4bQIUStEPPxx3A6t4G3dify0o7ZWOYVow8dRHqPzwkw/opKOguJ0MrI+iQadRfJSrpFkcRGSEA7dxuGsOHCfrrsCmwprQhcQnxUOlq1UpwiJ0FCO2Njv8Xo8OX7118kPOgchp5mZrXBj5ph/JW7HoBj1TfZ5v82GkUvGrUlBHgXo/k3VqBN5m5+FF1bRiJLtrNx0Bn8ULDCMwNuLOZSWxDtXe24PF20ebbxfpDp8dh/Ou1tdmlF/G37m5mW7Y//v79wGSUt3ck1hOGR8At6uxDvhgjsEh2+Mfd4SQc/tsNwxTk+agvhXYGTQS/uJM3LTL5gClrjdfKsMDxQiz6+ifSiLGTnklCtawDttwDU6Qcg84jHLq3Grgti9ngYHlLGu/ZD+KvKELmeI0bYQHlHJJ2hDpojFiD282JcVA6SS38TEF5PqVKFSeMgNqMUqdxKswm2BcBMDxDFvMQTVXFc6bsBo81IkkiJXnMOgDEKFYO9YIexneCC00icQcR3ayBYBIeb4znQ8xvE6Uae3TaEH79+itYVfjRrzrBQA3rRTb5O1eIYaGH0LSUeomYGMJYYuYk/gyHHmUO6TxVxv/3Ks5tuMmXnPbb7FfHiv/z+Y9Vy+/mpFS+xpdFMeczHrPKEdk0rZ3rc5OUgDfMbDvC62X3ctg7356HLhbLMQFmAHw8FNt5o8Cak7SlumF+mj3EEBaNPYhx4lsaeQ4nM/Iuhvi4ENbkM8zpE0rBggoLmkDxkEN8+f4/R0n/Wuzlq0CtcVAY30q6SIDfJEVkU9PEseXxMmKaI0Es7+HamGb9+vZm9Yza/LtkF6e+DLo25PbfSa9JnFGVaKV35GQvvLOTmc7/yua+NdU8c5MNhN1mnWIdAKsGscM/PS7Ru3eqdg0+TYnwadaAa07jptLpdft70gj5iI+nzfiNT8g3KJ4r59dKv3KlZzH7hDVKlbpkUgFyzBFddLQMrB3KnWyFi3SP2pEfmcrlQHzLR3bc797u558HpSdO5dVWH/MZlCjiN54E71A1YhF0RA55QPkzP9FfPItQeIXp0DF/eOY7dIxWlxcCHsWf4bGQMCU9MobDezujDieR3N+E78AAy+x7829fT7bqK2UFajlZd5px+NH3HZbNwKLz7t51rwD+7uf+zXQMuChxEPq9j0i+hNFyaRmwPd1KQWKLFfO5ZoopPYeg5k6rgEYRP7wbxqWAoJP3ei8RN8mBL91pGXVmL09ZCnbkUiPqHYl/goCW0BzvHDscmsXHMr5rkhlE0qPMoSalEHaR5HAMQCUQgFGGNWsGn8wfi1XwR62AXNqlbfPCeBdJk0O7Rl8De57gfWk9hVilhvnXECmfhcglwukQcLVhM25V6ujKlFE+6jypcgCloHudu5zMm4Ab28HIqh13npnwqz8e+idMhdEsm9PgS4pcieAia5jI86vK4Fb+Om1VzGaT4j0YTinjpo1dpCuhi4ZrrpMtAVLaJ6z12MT9nCJ2GoZgrzYw7Mo6L447Qd2B3qJqN1bUEeFQYMX0C6zO0ONo8EHl18nr/mcxrv0WJLpLYUBm4pe1Q56ykX/1eCJ3OpgtpGHYOxXt+HhdGAtyC/WEw+iZirx7oBODSwPi9lXg2D4RvPndfRKIDYFrybkoWadgtdIN7DiSgv+H2gZPXkKy8i1P+EE11M86cdhKf+hKB60PSvUyUdcvBL9mPlnAhVaocBGgQiyyEyq+yLgAWDH4TtUINBe54S0F4OGeGnKFX/D+jz+WCgOJLqAz1+FTlcL9bJNtnH8XDMZO+LnDIVTzo/xyz+1Zy76PjHPxxEg67CNEv4wnu2YnpP6e2/9r/7fZfsO+/9v8bpgymXNmfkRXgbfEkSGqh/pGjukPvzxfGBhpaQykdt4tw59BHJwmw2E0MU0Ci1E6eWUbb+IOYZXJ+OpSFNDGO5Vp34Ovl3vPYVx9Kmd890jy68VLAFWINRSxrcjsxZsFNAuxWimtTMNaIEQq+Jj7jFMKaGtDGg1jJyc5g8rnBJIOdXg96URxTzCq1HrFAxB7rWQaYNiINF1AtuIxFZsFbIoKa/YTbhyC8MoCH0no6hwOYud14j0NdcKgLDtcF0HKhN5Loco78o4tNrgVMd7fw/phg7lVGo7+Xg6y1FZFJw8w6Nz2cj+sWr4xdz+WGGKr/9Sw56TncmLqdTz5yYHW4y4V/8ABLr0tYTAMJC61nmlcEDPkUAEnOX4y9k0BJcTAtQXVobI2sFw+DbNgSFM+15228mzqGpLzpVDU382nlFKZG7keIkwGtF+i856Jsgh+fFXjzVKiJo8VLaOyKQC7qIqnjIAV5CTxoS6OjS0b/iZcY5W1iczBkd+Uw4f3jPFEcwYyPVlCXWIL85QOA2yHopWyke1QLxw16Bk87i29wEyceOWwrmkDaeYvFPX7mUv1Cvnr1TzqEVUwRBqMu/hFiF1PlCqft5Z8opReiN14mJ6Kab5/6HYBWr1ZaZGbsJisafT2BHRGkBkggagEe+v7cHTmHtHQhpkMvcCcvlInPHeAdbwAX71/9gV+O2whNlHN6xjLyfLvocCxilGAzl0zwRg8p0uRCVtaM4WLkJoTKcL68sZ5VPV+isjOSMuURagI6Sbvlh7q9E5VkMpMFCXwWXsBt1x2266y8s/YdnrBswXFxKL7is9D1G5wbDbNtVJlW0TM7htqQGdz2X0WLaRUbQ+F86ym0d3+i6HA6A45GM/bhWv719r8ACBC5hdJvWcApga8yrvGN04Od4ifYd9cfD2kLb/o9gcu3iNFNk1hmTaRdV8ploVvjK8cC33yyGpnDhEfQBcbdHscv4w+zqxPMdjm/Bk3mx7R1nKoeyLuq0ThEDgZxmfLIRSgdXogcf/HGlSEIL/Sis5sdXVUoZY6Gx+M8SSOgZlgxbe0Tacss5op9H9NciTicDgadG8TAi5W0zm1G16pDJLUxOcj9knQz/siguxlIOgLwmpFC76g9zPUv49u8ONqAQ/pw9vhdQS6JoY+uHcGjErhOq44n9rQwd2g6gz+8ydjmR0I5LgHXTF7UOxtpdcLnrU2UffMiJ8fepKGn+xCrC9brQVS3hKkxR3hZp2BDrYPgs0raI5KwhcpBFQFiJW81fE5uqZ4Vw8IJqwgiWi3EL7SRJClIBeC0OnBEKmjyM+PbKOdkeAnzam7ypSKclXdTkCU4qY5ooFVUxBfC5Yw0f87uNjW1XbGgKkZoAv0eIRebhzJ90XY+8QHPUpicp8fXLMDU619ER5YCEKIu4Mm0deS1jmBE10UQXiRUmUtVynXCG19gsdZd6dPYcoMPLkxF/mc8iesdXB09CZdkMjEyKXQUI0fPm91qWdtcR9O1YhIvnyXPJ5TXBlZRkJOJ5pQfjkEGQEPwhG5UHcxhaGkcz44qo6hdhtQoIjgoEZnX37T5XuNm1AF4+D5GeweFCXp0MiG+JUoG1A4gJrAEPGCdHo6ZXNDzGmZfL8DFgS54Wtf78RhqdSnJtYDdJURq80ElVbMsKh6fxId8fvVnpLZWZDluoC+n20OqI8sR4YAHH0DkPCrCL6GPu4rVNxIXbvBLbtUjbj6NTJtKfVQgOkk9D0I+Y4r8Idy9DAP2UlBSSXu/s1TpVbgkB+kW0Pb4nuyxMKVQwzeLbxBh7ckzXz2DTz83talSYkAnbyDeuYcevTZzo+Q1vEwqzDSzp2kCz3Ztw9sQhfjkM5w5WYI1tpD6FD23Gvb+x6IpIDn0XTb89i4Vft+zftJirtt96Tvi0uMjpNYGAkuuUB/Vh1HD9yCV2+DBh1x3XMISX4hHRjzhuRcIbx7JqREn8Wq+CB7BJOr24IyJ55rcRPg+JdUtvmQO7OSIEZodENH4E1WNvgQ0N3D87xfRLNtBUs98NrW635FvXzrM9g4hB+9c5/fiL9goisFHeZ4k34uUGuawviyVG51iKh9exWyIpEs9naauEAI93IGDMNcYBKZj+HUmUqY9gExgh4ebwWkFn6HMvxiBqktF5KA8tsff5qP8uRQUr8FDc5uCjET6xQZQOfQ8CdF1pPZx8PVPn7Pl9ueoekFXCghU36DSNvDXPwmfPHl1Pb0c/0JqN/PC8c1Y9B0M/2sQFwtW8GdAF6+6urgl6EJts/GEGkYoXDwj6A9a6FEJYzs3YOu4gTnMzMh+ex4DJUYnDKuBarUnJRP9uNfQkwCxA38piGxtYGkCXAiFQuwSO06Rk8Hjf2CdXyeltd34prgEkbUeCmOQrR5LbfwQGv1fROWC/vcW0iTy5pU6DVanD3ejr3HYqwhfeV+8uq7SU2XkdkUob2yehDTJQVG8OxlmaRM8qYZ8KzxrNdNR1IOWplTiXQnUDQ+gvjOa729tptkYwh7d60RnXedS/0v4+vxDcZMsBW+lNy+H5FD2yR4e/vgW6ffSef3p49ByC7K+AyC8yIunD6+kZkQiba759BvkBME9trXHU2FvpZ9AhN/JQwTXVSH7dA3I3BVhfZq8Cdw/DsNsAS0DG+nWM584u4AJF93v59HuShx+pYw/t4AT53oSkVCBh7YLm0OG2aHE6/fpBFjl7J3pjUNoICNiMekhdcQGmjiQfYKgunKafJtRG9TYpfwPs1s70E5J484YeNDcgysZdf98WbkHoSCMsYfHInEGkJ6/kNjoGsIj3PRIp5t7EFvdC4fVHQSJlv2jvaLrusyyOzPoup2Mor+B72emASDCwmcjs3i9+Q9Ot13BwxVMUfoDxLI67CIVBI5CK65ArlNyeMg5fM29mdbg4l2LLyMjIimxXeeuBVK0BVT4NWBQGjArVSRpk0greAuLuBOz3QPPoAWcuRSHtjKY7rti2fvkaT5+s/rx/VUoXiZnpIkm7XkuJ7/OIWkCN0ZepdvRp1G9f4Lfq96mR14LCQWh9LbvBKEYTHUsDd5J8pRy+sqfpav1JXC6eF7hBzov8BvMLN1HNHq3U+WVwrlur+Pn7I80fCQDh+eBYwsflQXR+3Y6jmY/Pq4MZOSPp+ijdM9jxe0DmGD594xn59lWuDFIDgHD3R9ggk8Nzv4F7Hjgj1+DH0L1/8ycfVqnxykG+/UMpHIzYU9VQ++tKDe9zgv7+9EWUMdfc89jUN5lZ/UuAFozt/Dj1je526yCCAFvSZo5qfrnmhcrZmPq8ROugBricxIY3fAUz7yyFRRQ1JrM+fJ5zI+7TF6HhcSbn/JX40ReXPsrdY9YDQsdx6i5kERcrRGLtZLgKP5P9qTajJ8INrdDpwtUXUVw6HlI+xd19lB+qPSgrcuD+C3jOZ/1gJnLdyLoKmNQeBmvVkXTPWAE3y/6HpH6n4lO4jIjyEnE1+CJbkoC8wOW4GltJbwOKgUH8NboCdk8HlfSFbSpRfRKLMOn9DCTE57gob4ngaJzVDzoj69HJV/ZI7BpDBxxVfKcDt41ldCtVkrZniE45ushrQ1w+/RpMkiR1BLkNYFrWhvG6x7YTqQSPfMIJbNTH9/fBP+J3BwTyyeSOK50f5cxzhdZJf2KIJGFvg4FWOwcnHCYkjFHmOvzJmv7z4e2PBQyLfVRfUlO8+FW1wEa/BuQiMRcMsMlM1QGdOByWljojGL9lheRPMoyPxoM1zz2oS2K50RJBPcSOx69kw4+9gEvIfzZKGJTUhNna4bykt09F9ikofydv5JkvwsEAA+6vFB3ycAhQCeEHqJi2KWB+OUIktczMkLPXYGAU7rnEHnFkOgVj1/TQW7EtPFZ9lcIJWJyMhtxCmyIhRLUUj1KSTsfCDso8m9B4u/Cvukdevv8kwG7tPtKln4xiaB7KQyNrUFelcgy/VRWOlQ008UypT8r1v7BM0V9WByUxVidnr2K1ymq2chXfqAXi6hJHI7Zkks3YR3xnjUc18dwU1CMwhKGSugiKT2VxlBPtgg+Irs8kw9Eav6domiXhHB0+wRsEhHatPvMiNYTrxIjEdgxucCJE0tPNd/7rMGoNNK/Ggbef4d8zz+oefFNIiQj0OXZMXhVka18mllsBWDtwxH8dXkFwSFerPrkIPdT7hOdOhUC4dDDpUQEtWCJuI/H6T6k3k9FueRPNo1NZemF3yhoGE6g2EmbEDyrPTHrpRT42LFKJpDm2wdBNRyrHoW0rYslqxvYq3rAwdxnuSl5VP0n82H/7Si0v0+kpS/Ej7yGTSrB4ZHI1iNzkV/Po/SUjdTDZkYWrOPOlgoSEiAhAV481875gecZGjiRrs5SvilejUfKVAYyie9ubsZLkI/N7qBV18qNfrsxSaay78wZADQ+B+iTNYkDrV647jUixAYuyO704sm9jbRnfoYg8wF2pQC/Bj/OS61cTz1EVvAhFOIODB1mZm9LQWCRUviSkf2PtOcYfh78BvKe+ThJOHA90q9a06BlRcpasHfhuKRhzbFZFK00orPryJWLGaiy4PLKwh40gcOsY6sB9u46xarvXuaNtb8xKf4LPri4G6tDwf1w+LjFSdnXY9jvENL9C3eV8IlgOF0n4coHC4nOzEP/3C5qLb6k+n7IBz+tBODnue8T+tZ3LPkpm5jYLeAVxVMlyeRL8qB4IXO9x3M0ZjwAXfczudf0CmsHujUpT7qmUR36OfsmFEJXOmpTKBkBP8GldfzilcDPT/5BqDETvOxs676dfNFklkrzeEJdRIcTPn/7aTQaFyEx0zEawc+5D1cstDkEnCiSUL56NdFpdfgXX0Lp1cYV4Wnmnd+D1hZMniWRTlMrdrGcdm07SPw5mr6XDBkc6Kwk1/AuTY12RsR/xaYgPVcs9/EUWx9BeBJu2d5h88kVYLEg72zG13EZds6HzM0QNZ/fXGZqs4dTd1yExTMM5ZvgIXQzsMTV/AuJQI71p9+Zwxy2Pbnt8Zzwy7H5yK450QwQUPTKd5wKgV86PZksh7ERJ1EUyXCqzcwWuthvcvGGU0K53UaKrAWzuo1k+xpaK15FN+g2r/m6/b0/DLDF4s37Yz6i/5/TSJclc0b5Db9cDuV90QtMCT9EpxMa9AMwjzpPjdqTQaNXcKpQTqppOCT3A+BAWSh/3ZzHC90FCIxO1B1q9A7Y1ArXHA6cIrtbi6+0DWWnO3a3u9MduyiNLsXX6J77DP+4odhcIJY4qI8pxdKpwPlvXTG7gjAhfPkIZDiSHY/0agqqtjLii4dSkFiJSAS4hISXhzPq+ChKUkqofkXMwbaDxAgnP/4NZ1QsVZ0KMnrreFgmwSWXIHNUg01LWVsJ8vpNJD1IJ+Prz4gW/0ZHE7jUhQBIE3sw8qnRFHzu4OpgHXqvBIy2U2gk95mfvoYHJ+QU7wuisM/TBHiUAOvxUOrY8MYsxnzwA0cdrv+XgN81YIIYFo9OQSTdyu3uazitvo7LbyFp/75voQinVEbKlVLCg2JI0Ya5GTfq3VXsn0Z3cu/OQS4N/IZC/w201C/lab5kRNQIdnbfyfaNcsqifscidztgFoGY31zf8lbVQHwFTmxmO+l3EmjwlyMSiqCzHJe+EpvzEF51KmqbVQTVBNHs00xFz19JixuP/vYZNg/5kVVNJVzQpzFC2YDStJ2P9W/y0tFcOmS5dDx5G9/INsao4Fu9mpfT+iPolLP65CYeJqyiYdgJwg9NYOPCN5j/5lYiksofS/hgeki3uSfJ7+zHTdEsELgwWH/EC517eseMIbqcLrWIpUUerA+0MVn4CQCeE89yLW8Y8jg5W+dvJT24hVdDDHBvLYOCOvDaqiJ7XSo1STcBqBGnc/rGRJ7JeI2F/SbQJK5le+drvLT5JSrDKnFOSYPqvVC1m73K/nj0uoqiS8edc91IzMpHrjKD/jpok/l+XApXqyfzS1MHJqmFEklvon29QO4PzddI8diGb0gAh28tZootCf/UVkh755FkFWySlLF933jO+8mQPBFDsXIQ3Y25ZIfC4ka3nmVVdxGXFAcIaZ5PiymY04oC+qi/RtTWG7PezDej/Ega/wNX5hRwYdBnZEZn/vOeGYy4hGJcCOj0CiP1TgUCzUGsSne7SuvKCcs7j+ezQ6kN8yCoshOrWcqgqWe5ZJ+B/B/ZxP/a/4L9F+z7r/2v2uNy1q4KeurCWbb5M+QOJx09rjHEGEzFsgI2duWRYwGBzJ2xIrTC2itj6JFQRaSgmVPuJEPWip/icH0TvnU3GXxSy3VdNa86WzlYD376TCZ75aCRwgXFr8Q+OkchgIVaOGMqw+77EtFr93Dydl9i75Qjkdpw+Q1BEODOEVZ0Ssh6kIW30Um3c2PYNX0Xv/noAQdLtmZhitFTOlzL+cQTAFx4xQECIcqv+7Ds88+Z29aL4rsxeOg6kN9xZw4rJUqWxBdhXplDYOdkIgri2Cz1ZmlYNjVRMLYmn4g0J+UaFz8mbyNd85Bs84+0O2GeBn7uaOHevhjqS4I5N+Q+Fk8xAlwEqotQiDuBfpzp0tIkgpEhvVk3Z6L7wRt6gf9grPJQro88xcOeOjqiB3El4p/KJ4GyhfrAJmwOO6O+DKPOewzrXl1EcUsmH13Zwbezw8jr0xtHaSYdfyzlw/lhfDy8N9nVU9mRv5xuWfepilGhD9vFukC39lCjsIlcC9ywWKhvh2q7GK2mA4Hyn2qE6HLYFVBFb7uGvZoufk67zCbff6aqkUrA+ADbmVrkFw+Q9cpKqjsMlPbWkB7rTuFpFATwla6GRKcIS8ZdWj31ND+KuYltYgRWG5baFhKu/EKpSgXiV6H3T9iOHeajMQO50PkZ74x5SPP4Pxkh6k9fxyVm1IGqKY3Iiu9o8U7CNTiXKJmUINdNfMTwejPs9X6Ij+IOzurRrPSEDqeBdpuKCbXg2x7L5sAFNMrE5JybjmetN/fbfbC6XOgd4HDJeNe3jpIuP26aICf1R2aGH3ffdLdPwGFkoMCJV4M/TpMNF0585YXw4CBdwr0klI2h7kEusqBM8sXuQKu3ECojofR2PH/9NBvJwgAmBMCBoE4wCZgyFBYdLELgJ8AFhOXr0J+ZxeIPvuXNKDfgvLIZFohv4nKIUNcYSexIxD7zMMeC4b37zyIX6IlXXeS6eQRrNq/hTsYd7M+Z+SisgKc0YLC3MjW0nWPhNRSPi2X7qA/oKZoH9WdAEcQKeS4vh3Wy6uoy7nu/CMCeRreGS5pnGo2BMpCIWP7FcqpScxmfsZtm7wm0imKY/6CN8hp/kp+bz+VLaXj2rSKyysmbsT9zz8PG13aIcAn5Y9xJfhF20FezjIrbJkDAXx0pfCu/j9EpZOm5ETyXVUabqpUv2yFeAlZvHWFjrxFgn03igxKqQqvoVHfyTgtMaBxLevh+zi5/jcRoC54lMuqEYeS64mCke+PVSx9Hwq+/4bLW8uQfUzjUI58Fr+zkjZZwfvtsNuVDurBlCPh6yUYAKiLcXS2vldBvx3Tq5nTgekTV0yIKIrkCpFYVGw0dGL9dTc1EC2Xpk+jwnMh0tqMTgU/7CEIqbiGtrOBuCgiKYJPvML4d79bPsuHBiIMjSHqQRHN4F7KWFBpeF1PeBQe7IEvqSVXr1ygS8pjUOwbxrQDELtDcHwqGuxi9bxNTDi12Fx+aahCbipFapWxshZiKSHyqc2i09CZ3RyVXF+0BQBOgB+C7D+Yytd0b1xj3c9pdKm5WnOFFLWiEhby85A1+M0jR/fElXtdC6TXTHYTe3gGTNRI+XHiL7yqfI2P7TIK8DUyJHY0r1n2tt9tlfPPlDAYYohjhXEXwqGQa/dbgI3tIsOYhbX7TWZS1EW1CCb6Wt3khzkiv/IXuk7174ZRmURY3AbXKSa+riVQHufBuu4a8+CO+WKhh3r56vFwn8S2/SJ7EBYmp4JnOXlMrb9r3Eyq1029wNc+H3/0f61tPRzr2h9+jCajAZ9YMBkU2IzTXoJB08N3Nr7Eqqzj0+dvYB5ay8avf2NGRwLuV7rlOrzlDg+JbbIEyNBP38Fn3++y1pfB3oDtw+mFzPbhcRNzZhyi0C2dvcP2jXgrA8OBV+G46zdRzw3nlwWz+6uXWwPVxbeDHYRKyQisJ+llFxIO+3B11im9tQbykCKLJo5qq536j1riKgeYK1MEiTKFrqPxLQaVLSILBgvThbQJjBlIbG47VLw/IZ4ba3V81dmh1wF3dCpoDrnKnZTsv9XyO3KZBFDCDa/uv4zIpKBn0FDGBjYgcej658hcSkZk1s+Bq+QZ+z15BWuR6lgpDSRBcBm0yhM9GK9cy/tRydA0GDsYc5c1mqGyPpdJh5ULQU8h8/VmpqGNA5hCeTl/HBf4Rv5ie8jzFLVFcssm42yrnO4GNbH13fo27weuxB7l7IRmV0QtV9QP8DeVEtN+gTj+IyZIjaGxSVlt8uGyE7nvmEtetBMa7NZ8eWmF6eBJXBvdEFncRR/RneOrcmiMbHwGgCksNi3t+wteXwvEtTmBJUh0ih9GdkaqJI9bxA7fsIXxl7yRB5s6GvXBhCTsaQDpdSqbtZZIfxXmd2LACVaHzaWyeQMvvUfTP9GKT/2neboHg1LWcLx/D7wEwt0yEo8YfV1g7w4+GUxUq5Hqv6+Q8AlBc9zII35tJhq+cjiYzTw75ntAs+OPeu7Rb/CjtNofCuPsABPzH2NoZpkGrktCgrKPWDgf7+LP2mZ+J9bsDodPAw62R0cNbjzmkjpALWmJFG+nzRA7BhSdYKTxO9mE5KX7eXAyuwCB6JPDgcoG9k49753PI1ZO62BWc7ZzDHc1mVMKzpPqdpaIt9bGWjyx6GtdE4OX3FJd2+7Cs1zOMD9/DX72uIbVpiWo4i90OMXXPwy03PXSCcgzDPzvPC/cCWfnZSs4Pugi9IcX3PBp5EwLnYDb4OPmhHbIbXTidAoRCl5tiK3IuCTc+xlAWS4NQSnNcKUOiHqXy9vmN+heTmHP0EKaQRp7/4XnO9H7I0BlnQaSkNvE0sybO4ExMFHWGJuosgfj6NyKWONBpzCyLk7P6ygWiosQM8vLjXX+4bm+FgBFw51UGKYYhcsFNgxNFQxg/tfmyfOAwrnRd53LyZjYeXo065CbC0vcYV5rL0kV5DA5yR303XtqJWTuDnfcHYtMeIXnyFs5n5bnptH0HgMyHk/qfOZv+LSKnG9Wqc8hAE0+asAZ/BfyGk7a+Gr7NeJnPA6MYFbUAEFBw+zop0nKk2JCIMnAJQerUw04fiFlEjwQjye//wdR7W2mIggOGh+72Kv0V7F1cGmzi1JDDdLs3AD+/atqF8/jmbzfopvL1IbQxlKDaIDTOF3BKdIQ8Y4YT/dx6l1FPM82jnkgJVF6OIilvIDGbd/2PeXBIqRdVZ4fy9ZDLnOlTio9DD/lXibJLudvgj63DAo+SPJY3wfvRaTi8e3Fr3yCS8pN4+C8H5R0q1jhDSfLYy85j7vX0aPeNOOIqmVaUjqk0CKfDvZ957cRNQsI3M2j3NILETjSjLzLAvwuzSIvc0c6vBigUD8B+KpyATjVwmGH9d7O7+TAlBZ1kBJwERtD9RgxexV6sOjeIyv5XCenxJRjctK2F1gyEX84h2aTj9Ih75ASW0ivqdcqVcXy48hci62tx9RFSF1SHN49KQG+8RKCpnu9/G41QZYcXQKx1f1URCfct8HXrFeoDe2P2slI15g9WhYDd4smz3V4FYGbJEu699y8Crq7gr3VPEzPfh8huZ5iuhsioPM5ab9E0tAFhpYinvy/g27l72NYegV5WgNUyklBJMIeKlqLpOoSPVYrF6a4I+LfVaeIx97VQ2VBFXhhoxX9ASXe+0Nzk+5WvYchso31sIwZhM1+XrGBsrouxD18hWTWIY9PCqYyYRY9uu1iitHH1P677+5Gl2K+aUI22sdwXvv6PTPM6p4kj+9YgLGtifMBJnrbG0GK/yZRHU1KTqZ7Xjp3FYPFi6PlveTu4mSe+68/POe4Ku21Pw+FwKfkvfcnQ08P46MRgwt42kuZRBg2nEaS8w2cDTlHXF9rzEnmgqKV3+3zGaxxgsyPRN9NZ18HgvAcIBBCgglnJK5iU8DnhRQqqnHbGezg5bHYiaElAWfUrzydOwE9Rj0HyCoLoVgRT/+Y932pqxQkEWjRodddRoKLBARanlT0lEpx+OqaGboRHOve37J8yZPhd1p09wcLXgqgRTGbulQWs7P4s73pKEVibeHPCHP688jIPLonYnljH++J/yiQcIh15mc9i8HjInElv0iSSkTLHRsLnE9HuMhNsaML+chAGrQEfEdRHQbVPJkvzZpHr1Yq/y0ZVZhlnxTmMk3zNuZat3LGA5f5zRF1qQpIl53rWdeoC66gO+gph4ec0doWzNrCYBAUsk8aiMCnoI3OvBZsHzuOaGaJbvNn7II27J/vjWe9J6LBPUIm/gW3foEz4gwrNAWb+GcduQwP3uv+IV1dvuHwcdKkIYt/gNW0dgdN2M7/vdv48u55hDvDLamTj7lk8VWal5rqKlmAXzeL7iIXe7sbYJqChL8j96hjsqSXx58+oCq2kbuJKmkvnU21IYJ4hGMsefwqHqyjuuxsQkBl0mPWDxnOiaiy+dzOoK/XHVHaE+JIyxu/4nEMHwzBYfGl1tnNl/E6i6lex8KfnsYe10KvfGMYNOIfeGERjUzrKWBF7mwUk/jGKPd4Gpr20B04NgtG3aIlp5dsl3zLq3AaW/jEX27QLbp8qbinpUe9yuocNX+FQln+xnDsZ3nDnJfcKazOy+rdTVDfrcAoFWGQW91qI29/ssmlY/+k0rOUhXB1wjR4yt2NxvDGdUX53uWtN5+LI47R6udhaB772YLZGi5Db3BRGFR5vsHH3YmpVcVSGVNPD7zryDheeQmjFSUHrfQqD9hBYF4L4hJKS9EwueU/HU16PLcqLMI8Q1M29aJffITN9IlGKh6AKp6BJTr6zk0CXBSEuDnaBQxLDB7prVNggqSCGOSYxRgX0FX1HmPcNxC4304dO5CLPEkq1bwMFujq++uAwNpmLUukmJqat4hUd/HZoB+37yggZ6UnIgiOEhg8itzUAmaCezWUj6VU+m7bLeZRM8WGDpJWRoWa+Dq1lvR6sSm+8rUKqKxqx+/lQHjuDGTFvgb0DKrZD1HyaqvyoO96fNl81/vGwK9BNd6ksimDmDg86QypBIsH+CNwKFbv3BYWBqVgHi8jyMvHaIx/vGY9tXDbBh60uxDYxQqccUaMIYUkYEv86kuVu0GxgjZORuWkIpbdI6/sPheRDG+TbRNhl3jz1x1NEl0ZzghPI0wbze3Vf9LpDzFLDpoL5nElcSJp/GitTorH6ROP3H1uSpoZgDPc66axpp2mqjD8HbgIgQQoJXRqKavywm+3snb2Tfg/13I1LQpiYgNZUS3rHVZxGJ40VjdRdfZnyGLdUycZWONwFqQXH8Q5IQ3j2GJl3M0lLK+clL+hwwi+lE9l/t5OBN1PJHxXBrhGrMKhdiMXuyj6hU4jCpMBmsdEVaKdF1IJY8E8GmDM0nIaucBT9W/ijYykJQjMJBdkge59Ag5UMuxK5QYXD7sT5CNQYFeP2NQeG/4XLtQiEIsQey/C3wrqzk/lkZF+mJ22k7SMdnT12sWbPbCI93M/kIW1j+frv2HA2h/6Xr9NfKGKh044/UA/8JhVw3iFk7pA+xGnnUV/ThcvzVRY/08zExAgAxNYyti5LYl/9VG6v60cCp/A64wbN0aWDzIsfTr6IukONy++RvqLAvV6GakN5cO4BaVeu0uTtSWBtIHpvPa6qAcyR6NheGM+4/VMo8M9l3JGh3OxxE++GQ3DuNXTA8Bfh1WaYff89Fu1axPmFpUwQ1sKdVdgUKxhf7k2pTc4Lv75EfvIuRj11/HFbG3130RLyDva2UQz61yp6DbmJym8Ro2Ng94PX6Ce8woVLYFMfpCO1iCJpEhEeQij8AlQRdHYG8XT6D6wo3cbCDS/h1e82LmM5kAGAv7OC1a/tZk1RLClvv8qtdCcTvm9EVPIdPeVwFCcijYjyyHIsIney8TveMCn8EwyRcymSSCmqu89TjeF82z+XZzIus/7cUdK8Xqdj/0h0vVyYFCYsMgsuzaNE8l4/k5f3Ga0Rl5h+ZgEHdk8iNK4SeWRPePg1LlM9F/d252G2gpIntegVDzFK40FihdJfIGAEK/LfoON8LaElFWgbZ7L8uw/hzmoIHAVh0/iiPBnJxUzahp3nvYhrDOqazXcec3g1zxvdV/O5bLtMnG0G1qo05Db3nqBDEE9+2XwOLTrEEwejqfGp51ZbDn3aa2iKgr+tZY/7xd5uxKPNHZNpjMjCqA0g5fw3WBRaumKXI7RaURgasLabuN3fn5rwd1kTY0Qqt7Hy7JME1fP/ePvfoO/8t/0X7Puv/a+ZSCQiNfVRRufx3iS55ASGvkCX2R9nhS+5tWpChDaGKvJIkUK2wcXo42Op03kgupCEUShC4WgG4FLlNMYH3GFBzz9ZdH0a8pd/ob9/AOeMNfzWAUi28deDPtT8/Qqlk/YgMFXwdsJHZJtXE10OWYrxPB28gN+uLECRfBLTwLfw9wZXlxIuTIIJRayhhLwjE6kY5OLnZ36h2acZCTDBA+RCF0aBCwnK/9NzFjiC0Uuu0iGBPz96ipQ+90lLreVFLSzwUzGxGBqkHWQVSMk8PIfRIX9AmPvcF7Tw8rECbDYoS95Djt1CX6EfD0wK0itMPO29ksM17xDapEWc/AHfDppCfLkHqcMVSEVmwEZ0fhCTsjNpyDTx5pnTbBg6DE4Pgf67cMnT+EpTh0RporvLycYWeN0L8BvM7JZbXNOrcFrtIJfhFEmo74rCYPEBXCxqmIJRCmN0cdi6lzEs0kCotoBQ7ftsebCctf5fgj8sVgbh6ddGnTSDM6YcztRAP8tLJPzdn6AAB6PWjmOCCs46hzBEeJao9rH8bdWR/V4c3Xs18XZ6Ll6P9C58S6HLCSf6dOcrzVf4JSoY3zSdwWGfkN6aDXd9IeY5BAKBG9Rzwa2xJ5mrtSEywxUzTN43mZS8IMRbVBwbdYyq0Cqad11j14xdiDDiaLTgNLVilzswS80cls2m3113lvm8kCM45nTHoA5gtAre9LJy0S5lRROU2uCvuz/ScDqP8ICb/HRnLBMn5NJNIqGjfiUe5gQkdR20bB/BmflbqQqtYqO9B5e7VGRVwcaUXiyS/EJARToNoiTksc8y2c8F3T+D+OXQUcx3Y7N5v3c2D4s2EF24l2npFyGnnG2JMPi9m/QK/APngz7sVPcFQCV0i3p3OZXYhN4oLVK6ffYCF/rnMnDyRW7VjcLiUPJO9kDMubEcH92K8YndaLzd7X2gC3RC8F+xgetGOXfao2kXFzAM6HJISZbZqbFH8+7ad1GYurjf7TZVoVVo7Hk0OtzZ6E/I/FgSc5v6yNt091gPbaASOuHMMIh+lgbF82xfX4oqKJuwKjlOVy+u9b4GwL30e7h0L/BygJrLfS9jDKrn9WZ4OnkuGkki6hFbQD+ctgv3iL36J7Fpd2nQRLMo+iD72hWoVq+kNcDG1jkXmeUBZkkQdZ2wsNsKvrV7cKvBh5h2D7zP9aDZ086QjGIGKKHaBge6jnBUCMGGu8w6MYu/Zv9FYYI7W88lcPCXcyhdPW4iC4lHM3k14rxlzAn95/2PUJ0lz0uMUKXk+OizqANqGawdjFVaRH1AHRKfmH8ztdBd5tZcOKPoxTlNKJIFO2kPHcmU34eiNQ7jX5+upncQzKhwoJSbkYc0cF8lZdy8OTwVkM/prlgOGIv4ObQWsecH3LiZRcQZMWtvrKXx1RLQQZE+k5/vf8tQ7Wg6PEzoHSC2ixELJBw2wjEjlGsSsas9qY/pjzrKgdTYhkMixxH6FCLHMOTtXvT8cw1RxVIU24P5ePXHAEzxgIwJB3jStokwnR8Fey8+bocj0nZ+KIaJPlmEBIRgqmkhLvsQvUM20z3iJjZf9zuda4VmuwBNrydo8byHr88ZFlS6xdmtAiVyixfBliEYLNsJd/yzEdvVAU1OD3RtOrz1UuSW+zgb/dheu5d1HhJCNfkIBULaPNswq1Sk3puMqbkVYvaBVyaETqO+6BiSoGLajP0ZeD6DitgMVEEDoPgjVFIDVX5fENkewphjYyg3FsDwm1DyE3JnIBP3TwSFP5eHGplTD78GSHmiUs4Jm4EvFPm8GTyArS3v81LyDYROKc22VkbH/IDeFMQv1ha8kuooDijgpUawucKJUt0iN8Jd9X26LY5SZxytyn/xahOkaIN5+lHcdriqFKX5Fj419/Cpgd2SEWxbeBpOD4NhpwHYbzYikbQi0q8n7EQPXvOfxocRe/AQVNDg+YBSm4XIvjvY1vNFImVOXpJWQ8GnyIV92aSHgTIz2tGXQShEJfEgtu4tADrN0xmQk0pJSh2t8TOQBf0GuDdF/SUqlh6aiqZdwYwSKZqWXigHtHO2bB4NXRGIgsXsHVqBwOWgZ/RsNsZdhUIwWPIxWN2BM0FZKfFX9+G8GYHB/iwxv/8CxjYQCJHiosfwNirro4g4kUSOyJOBI0v5vXcGL1wNxXBsCO26SirbuuF0CdG4HpDi14G/qow+ov2Y7GrUBZNJ2PU6zc/s4rfwG6Rl/45FUsPUWw9xNHvRc3kZo0I/Q9jmR7O9O3ldsNohQ/nBM3TPKqalPIifta14O6KQOEqJbptOkOd+hiY9IN+8jLLKF+ivW0u5VUyB1U4PZxROXSav7L1AV42UoKt6tg5qZHXkFmi9BiET8ZAFcfC9hfjEdnB9ax/Opj7g7LgjAFhlVqwiB1155URdO4g6eB5TX05EGTGExlMaIu++j5M05liWEWu6xzPx01gY4KY3u6Pw4ugLNwmjB5M+C0SAhN2Lk/Fo+43hlzZQHmwid1I1YaIn8Ozwxdv3PUZLf+Bk6QLaLX6cD+vkigX+PNENgUvA3iGfMtXzLl7314HTyvrGLyk5XUiQLJaNbX+TlenkhRE1/Fug4cP+V7mSeZk9X39CT2MB4ewAB8z1HUzDndU0BmVyvu9V9NY6xhmiSKjOh4vTqJZM5prwGSK07oqL83VLiUnU8P6woRwpWszI632puZFI9QwfgmNqiVTDdZGZXsH7cYraWZR2h29aFNTuvsCM+PdQNFx7PGcUe4xjTsFRGuUGGntdpTa4HoEApidtJMn3IgJJCyPPRqDJj2Npdm/eC+rBiB876NPjcwB8rNWs+fwkPe8O567sJqcVbZxK/pSQoLHg00ljWA98veTILDJslkd1MP13YbP1ZOWdQlRqCc6zr/PD2ecZ+v5BBvTUImg4Q4j+XxyadYq/Kz7lzB9j+EBoZ/BPXhCcCapw4v78Ds3v81GpVITf9CNp2jk88t2JMf0qN/OEQU3RR8uxSfVIrF/SPn4sPNJGmjnsKhbnBFLPfo1J6eS1L91ANRenuf/O6CDEcYFloioOV0Uys3AqXr2r4fQQZE478/ZnEKB04kx34RQ5WaothvyPof9O2tvzee/4RE5FOEkJ/pRQTT4u4RePGvsHLrRX886RrxB4/s0HH87G0L2JiSN/BZcN+vzK4YYhtLnsrGrLRBbmYIXyJ/YEijld9gwqHEwti8fzbH+gHIevBqUwGpqvuD9BY/lXYRZp36Sj01oJnnCWieoi/tN+rAwjsTwQUo3swsAL6MGnL0sCf+TG6o/Y32ch8RtDkHqa6NeviLvSaBI7i7mbeYuS6BIyShbz1G+rmPDcfmS9PyFhYD0bL7uD9QA3Rt1j9/JfAZi03YZG1kxVVwg3H1bRrrXw4dC/MXnDYlkK6rvt2IGpQcEcX/wdXm2xzDNFYouw0mJPZGriDDQyPVCGb1EEQ4/2B+AFjQAPw11QhkDYDOAw99Lu4e+MQCvbg70Fgr1+IfzmAvZVPoWyLAin3YXMLEMofhRkaDiNn9nG/kl3kLpCuND+Jm+3OGnLcn+dKoNZGheh73zKW3W9uGSCFgcIvUbQVvqACF0uHU45hU6Qa5zUxA1GFxjPhUYfQtV78RTZ6VLEUKGMITqvA/8yB9krGlh/bwk13gWMFqQzRgDxV36hUyNiz8oviRBpwGEGkZuJZEmjlVuOvcTr07l5YCCzJ1yFa8/iKxXR2fsmxoghKAwivO3e6H30mERKSFmP152v2JZxnrmNapKkLsLEcFMo5ik1LNDA/vOhKDsqCNOnUvr3b6wcGInIexxqkYF1LXrmxM3CEWQm5GwdFysHET9QzPBqNyvG7U5fUg6X4d15E1VUF2leRnTtRxkfW0lRSxbQ+/F8J7VKUBqVlLm8Ietb0KYgEAg5anS38fjCMqRmA2XTH0LGM7y+exRJl36kIFlM4sVCDH7RCARDudswFLtLwqD9MqLviyleqSbRFoo6UERZV0/m/d0AuJCFGNGHi7ko+oHvOvV86fkTvZRvsDsc/lWSxuefLyVAISHsejO/d89h/Mp9AMSVwzpnNpYdyVh9P+eJPW1MmAByx2GGyiXctzpxSr3ZcPUL6oo96H/Zi1bP1sdJFgAF7bc40ftFbOIWjjWA0u7L3uLjDPZ+ElNJBzprJ+GisYRXDcEQeAmby0aI5iHvxB4hPu81bLp+NHm9zApfiO1Yh4cXCItgutkLbfN5LK0qjjxaA1u8X6Csppg1/afx/uVMvPelErLYi48y3yJD/Q8V3wUTXLyaRNeeYVQNvcXV7tkYcxfwWYabGtIl9WFJWAEki6gUKXiy05dA70aouOoGWmLfoM2zA71PHS6te0wKBKBoOswnvS/yTmAOry9dQ1nbSW6q9jBLtOR/zDVSAdidNixSCwqZDQldXK6aToBHKYOCf+ZGcjq19ZEs37Qcj6cesn6QOwCvEJn46mwygeURmDJCcFgcvJw8lry/rPhp8uihKqYPcKROSmvsIBYPWoGP7T4Pmvrxa85GxiXuo3Dqp1yoGMCQ7ZnkGhxM+/dNNZxjkayB4yoQG5341ASz2NMEB+MgeiHP9bvJYfGrxCcM40F0LSr/UKg7CdcWIumxCceeZcSX3+fSGw+wjdvJVtUclrGNNwZMY/fD5/mh2oC/p56W7nmote7K2GA0zPu7lvrE78nu+zsBDYms/nA193qVE9FtEwefeIXK9kSu1xzDfq2QxuTt+KZ8QILLwfeRIBLA0s5faa3ryYL2wzysTMZDH0mBvYYPL7uTORZFg9RYQYL+Lg2GQtjdg8YXq0ns3pcNxVtoOjsUUYAFcYOA36tHM2XaNzxZNJ0rut8QusRseW4LAc4edPtTw8N6X+TPa9CWQLhAjh8xnJ6/HpU9lNH2FnSWOL4IyCFDBgU2EHgoMWoDiRIF8Hb8fu6JBzL+4XiqfLcQrx+IVP8velbF4L15Fm1KI4ZzSXzVBjfMkIGAc63vknUxiLyUag4P2EL6v7c0Pn0A+MQ7mq6F5/GyTeXoADdLT7sDJnq4+EXkwIWA47OvkW3fx0D1M8QKfmGLP0zt/iqHnCVo9KsZeas3NXFVBEfXcMcC61rg41Oj6LqRSuzSeGJ/iuGW9wX6pJ9hTTOYXBB8rj9SgZT6h5EsFJj58tUTvOPdzJHKRqz6W/TvK6A13MacDc+z9kMVs2VFdFOAvwhEjygxXa5/qo8Vkg44PgJCJlNhrqfvQ0/OnH+Aw++fUp82J7SczWL+sQEY3jKQMq8bN3+wkVU0iwWzj9LnwUs0/jULUw0YZxvJuJNIm6YBkt0+br0pnal/XcWaakXsEjOkZAiDRhygm9y9Pzzy4C1s0Wc5HKTDV+pFo38jUpsvYjGAgLKoMr5Y/gVvTnoTx7dWJFYJYpm7Q36+8zNfqlbgGzkBo/0DamVnEQkltPgtx8s7i4j8ucyfW8/Avg+YuX0GCTuO8LD3PO43DGZKwmcc29SdnAd/QOJTAKileroFnuC68Gc+Ot9JeeAGYit+xGkv4IRf8+M2Edqa0WSk0db7Gi0PB7L6ZDkuoYFgXTOTJw9g47M7WL8+gLpD3yIV7sNzwHMMkBUR6NQBgdgFEg4anRwU7ubh5CXMjMrF6JGKsvM+tN0FuR/y7HOkX8hFWS0mzHs4wnn/zKNpT6Xx17kgfOuOM+zg82ydv5XJKW6/6mB0OBv82vAIVPPnnF206RqR2hIen/u0Fj5oBVM0nBx+ErsuDAo/B2MV0SH+BG/dSWpzEXJXDV3t7sSycYMrENQcICLqEEnNoazt1GEQR6JQ/cNU0ztkPy31s4m+dZ687qV8Nf0z4r0zQPQo7lJ3AqPuM7rnaLA1+zI9rJbeQe0IHf/Q0RqE/rzaBDkWNUmRGaANQ5iVwfg9TcizDXRvcz4ev3UOeLcF3unzEjuunGD7M7+SHPkxyauj8K5N5u7gXILbOyhu6cGxjt4MqtRgTZLy20K3DqtTuwl6fg++/Vjh8wYqKZzMCEAfWMItWS9G9dsCh5JwGqv50llLglZDkn4DnwweRnjXCegCkt+CsOnkts5gXHY3auIttPWpZXPbbN4Kvwdi97Pv1HlQvfpDNOYYxBY9XfZmBMpIfupyME3TjkQjwfdwPT1LLJRnpKEQGwiwnSDbcQXBszpUMSp+mr+FTmsXSQ/H8fvWZ3A9IYQJ7naT+OkoT5uAsr2WLs8Q1LWHAChIKCbEBdaAMEQOK3lfnSVWYee7Pk6kchv/0rsr/IH/R2v2/Q+843/B/gv2/df+18zlctHR0YFarUYQ9yKW6ru8tuo1fn3wNVXmZUSqPqSX7QC93BT5HMabm9k9kfeoYMviC7ikcFY3ilm727A6FCxK+ITsrcPRdnOxZEQ59y1mzjc3Ey+Hb9qcnDWNQCU3MuXvaUitYhJ/KWJ9CEyqBQdqXDY7uvpiXD6d/K0DT4cvS/uNgc5SEErxC/ZE+tIZKmWvUql2O18ng2G4Eras/5YfWp4h/koAc4rm0LjQidDWAS03KTEH8EWtjiiVgoPjj/Kn1sKa4JeJqsgmgf+Dvb/8suLquv/hTx2Xdnd3F+hurHF3t0CAkEBC3CBO3ENISIhCEgIkuAZ3a6CBpt3dXY/L8+Jwket+/oH7N8b3Xm9gjK5Tsqtq71VrzjVnF0lmTzx7XWn3HiB81QE8A2wSh991w4F+8Cp+Dt82JbneKuQSD2RoqQvT8ksPtFlVHJhzCAQL4+6v527TZCLdbqKS9nOuajljgbV2XTRrIpHcq6RP18qpMb8zkUeh6EtEUbYirRULbiUyfrWEkZ+5gj8j20ipbWDN50vRPdOK6cml1GUL1PVEMSNyC87KJkZoTmIRaRnt9AIbnnsVuURLZVcSIc45PJO8nstaORKTBB+RhMYOO25GPw+sAEBskuJfnEWrPogTA9BslFHtMJ1vGy8wyL6dBpkfVRk3cNIMZfc7r7J01Y9c862l2wwmQBArODf4FpLUAiI/iuZuVBz+aYG4RtpArlihkLc/3UB/WBcBT5tY5wx/DKi402Kh3r8TtaAjzkFN1pAsAE4U7YPrj1DX58G1T2bQFXyLFSM+Z5jdFuKkORyWQaYSRrq8z8KMKhbcXYXxtfWcGXmX8UvOMMIdMi0xfFP0HKNKI5D36BC3pOE5No81qS8SpR+gv2kFr7UZ8Bp9ni7nLswSMxbM9Kht0n4Dpl52Nt2l45stOHISR96h+YMf8VZ8DFEvgNKLlXXe3LE0kdpvxqXOF+2AjWXbbpbRITJgxEK/8C8bp9Zk6+5aKPmagWGZTIyuwmBSYjGLuC47hsj8ExtHTSLvmDdFJimdblr+VOWjN73NkfL3mKaGMSr4yA0+6zBxp9+CWWImXe2IWtzD7Igf+bDTyI3BRdhpvDgzzibFqjRkcX0AfuuFpVI7bj1gGKY1mhjbFMHekfvBPgKCltJXMwRt82bEjj0EVtjhJvwL9tmeTTNmK5yZYJOeaOuHF0zdmNQu/F79NQ4O4NFxhe5aFWaTgLN0gIQaWGpNYFBsLeXSwTya9QqPex8n2Xs9yUm2/VZcOsuQn3xpGxZA9oYX0akcUGoiyVCV8EdDGs4PWN1a9wD2zd1Ho08jagFOeinJE39Jj8tKtqdIyfBPwzDgBBLwktyH/GMQuIiJoc8S9boPt5xX022YwiS/SzgFz2VM8V/cX/4H80Sj6M+q4entT9M17xiygGoyYlexq9KNM4rTzPUdTrXzfgS1rctnshr6Y9pYeeMDquJGkBzXSLz9BwAMVzUh6h7HaN+z/DHRin5EO825qYw1dTHJ+xSfXdtNU38YJoucF5Zc5JTmIr81J9Novccjkg8ejrUgCHw+OprijiQ6K74i4fxP5KZY6PB+Hm8nZ+yyjLiLhyP2bsHCvx90m7rBTtBzO3IJ/j2XmLhpIuUX69C19lIeUskMFUyfeYU73T/S2leCvL8EXa+tCD6qHjRW8Dw8EbXWgZjpDmycPxdnfSvbHhSqv+tXcvS9jQgBCr57dDt7fc0YxA481tDLn33woocVjw0/s7diASFluxg5SkDbC9/c3EaH1pd1ElipnUZtUwiht/8iry4W0+I1SDyHgERJROtjTL0+hn/UTRQOX4fMUYm3nzP6ORZSlyWSlJOPxtuOvSu3EeriDRI1FHzE84jQlL1Ep6sB/U1fhN4piJ89wWBRMPstvfT0e3Hqx1jsYqshFkDArI7iw8sHcVI2Y3BrY++CvXg1R3Hn9FTckhJZ7PnA60xsjyk8mOcT65i3L5nCtskEvuuBXbE9g+36CGYSzr3HeX37B2z4aj4xV4ZRv/wmgf5zHt6XM30J3Gq9RqLUSItnCx1WH1hiRfxHNNuOz+MfeweEOEd0ZhVFRrirHkSKVxJXSkyMuaBAEWpkXnApE1UitmjymBl5lqH++zGLCqjOvERx3yYmeX6K1vIv2/+fjjCGZU3DqbkIy6RshljM9Flc2Fe0gSjXG7iJOojsOolZpCFR4YFOI8fsHMHWadHsynsHQdiIKPQgwtQarho6EcRtdLOSTRMW2rwjLQZWTNjImcqVtH7jAlYF62O+A+BZsZrdlYFo6zowHqniePs8vJc7MiVsKyMC9/Do53MJyYlHt6qHwMw7VLt2MUoJjX3hNJh92PzEV/zRoeS4JJb2fk+cgrfQ4mz72jllUBE35hL1ng5UprUjl/kgGEcx3PAyOYP2carpHtEtLuSW21NdKqb2yQCCImvZ4w23FWKa9H0M3DyDT72Y8kEL0QuuCHG+MFALJi2+3i9yLPo6epUzCs1mKiRaSmyvPk5dTtgNWNHad6Hor+duz2FedPoTF6ULAabvsH/yKrdEUxFfbEEl1iK22CrLW3rgCXkQs4Oy+bj0bd57awxSqyN7tHoEiZk7ozawIcdIgVmC2hGaDSCR2Rjdb2bO4GrNQnS5EZiuxjE5P5xeh17mVI2GKmDIn+A9nlSxFr+Tn9HrWUteUg8qQyDcWg2CFKbmsa4+nEpRMcXT/qDdKZeKnqm8M2Q48vtvEjq/B2VmCtVNr9LtWEibbgkofSBgPlnXn6OiKxUfTR9j8zYSKqumR/IhGGBK+A/8dmcUyI14yivYkLQMkcnCtNlWvr/9PTnEMs/xPHVGEQFlp7lfFsWQqVWIptaCSEbnzSxq6t2wCGJOTj6JxKpCEODvgjdxkLfzxnIpmmpfhl0bSoNvO20h5cwxS+H6ckj/ic6INwj82h8HbRJWn1uUGEHjPgL2u/H+JJjQ18r4aNi1qIFrfZlEZJwkWPMDCu159OZxyC3QHaTj8ogcpttroL0Qxl2mpUTPn28n0GsUM6B1ptxiYrQggZxXwX8uneZ3qK66hCzKnfSl20mIsjHB8RwLSi+uSZtwGHON6s5xOPUmY3KKgsVmetu7CXGWc+z4k4QNaeEfs5lFTfCXN/+GSEq8uJybnzzFeKUWhVbJlMQ66MqhxhTArMOzaIgxow0yEtAcgN63BblKD/depXpgKrWX3DGK2uktrqPCqiThZQtO0a9C1W+UtJ3HXFMH/X1o88PJ85KCrhm0TSBzRtI0CWuzA/MME/lwzDiwwIqkShbEfsTJxneYFp7NmdRSLIWTSAzaj9V+HcS9BRW/gNVCs1WOp2sHzYHBVDg/jUX2Fk/816XVxEVwI7SY6JoNnDybzNAPjSSW2PO0s4xHW8FisTD6fAIaTw/mTyqD9oPQfpBdo+CnXhioFDN4/C3cfdpxkJ7H0beax1OeI/lGBoZb8fyxPIv3OiDeeS5vjpjJYN9/+L1sCSue2YWjIQqLAG+5Apr8h+dkJxjwdx6gxi6HD8lhe38q1wMeI1B+hw+vHOANQBZbwUXnBtSarYRFyGHIatu9BqLl98mclcWO/h7Cc/L5bko8ZINW7kPL4zs50RbB89UZvPbJa1wfl2M76LQirt7YS3rSAvJ7BNJ2T8W7Kpmbf2SRbj1HmQE2d8M7LmC2wvgTkxjr1cyOz37lje8ViAQLxL5JalMwWmkL+Sng7tVOQ/tdMuoANLwn6sBFoYEET2YuPkm45nemufhwVgqCSURD658EOV+n1rGPJx3hDZde2xzoEAGAuk3Gir0r8Gzxokqn4IuhzryeOsB3pUa2jiphgeci0n4Ixr80lffeeY8nz77F3Fda2XKyGMVvEtSB9hT3TaI1ugLFDAkeYhuIuX7Uc9wZ0cJjd/9k0OVKVKGD2GN4hmzVh4jNakwOjuhkTpzzbcASd4Pl0ne51zqSc/aXyByYQXVoA4oBC2nrdjDVCfq0I1kz6Ef6DU4gdPGUUYlL5Vi+HtRI78KTeMvWQritk14wakjMScSp24moTjk+XYHEaq8iEj3K+eQxNMn8GJQwBYlhALHJtv7fbpzO7cbprEuYQ6klEsd6d1R/P0bS48dZPcSPjhhP1p+9yr4FdlyoXsatNilqgxoByJEWsrEDSgf8iMjrpcvTh3NjztHq0fnw+Qvons3vJjsGlfnipjEhFZ3HlJbIB9Hfojc68fa1nexZAjdunGZw9mBkL/3MrtB2RKZ+eNCFLTEUcTfiDltrHMg7MQljdD2bD07ivBZGpfcwMsMB/Y1sVv46kt+XV2OOrEIqQI8mGIffxYiDqjAHx1EzkETy2kP80uKP74A/4zK+5omlB3mq7H38JNg8wq1mzFYJJouM2yZ7Ehy7ET8oHq5usfBJswdN4lZ6LTBe+x5pa7voj7lEd989vgp64FkZ+TxGt4n4e6wm5cXdlHy7Ds+D61j32wcQ+jgovRFpqvjaQ89+DRjOF5OefwCvUanIDE1McDDziXM3UrUMC7Z1Uix6UEZbqGfU2lUsPOWBeEo721ZvZ6e3mbHdJ3GOmECs+2V+KdpN51AR7e1l0D7ACLWtI/zDKwe40uPNtRnjUeo8GdP+AQOeyUjVEO12gfdG22Qr93y9gFqgOXQYFb6PIDgG8c7fMzBaFCQ71vNyO7iaJTQnfoWdSzYDjm2oe06AppblAxost9LZ6W+ldNYnLPMUg9kJDN18dvc81qpM/EaLWbxhIq6qJshJBk0tXF9Gf3AOPUovQhVX2eoBP+v07C3cwPyYT6juTEfv+xT1fgIxsgN89aBeE+dxBa3JniSZJ4nm0eR2zcDFo4NpToPI7rtGjBQCHIt4I/sDwip8MVrraKtYxPYFdawMuwxAukcWbiZv6s8Nx1WuJ+a7jbyphnl/v/LvPNryDnY3gjF56qhILCXP6s9IqT2ifhkplzKpGmogVJ1BxVkV+sm3aJJWk66AQY7FpMjh+45+fkm/iMSg5yl9Cr0WyENHnPwfUuVQZzah7bJHJFYSbHqGIDUoKuBcxi28PUWcuTaIt77azuCnhiJYbR3rVsxMmnwa68S9HDoyFJVGxVj5Arb22JSJXGhnrctVfFYruFYbxJDCWDarennukV/BezwAY01/UtUMt8OncdrYjsxsR6O5n8+d9RSMm0B6ejD1N29jEekIUiRxuRtWNYn5JbCadQY41RrEqT9V9A810boI/ui0fedFxFTRLDcSOGEyQvUFEkNsNZBeC0xRwcm5B3BTe7H25rOIrL2ojDalmuwAaK/axLjJdXQmfcPynCepCFPwW4pNwcW1AqYpOtm0fwmNKhFZuceIOX+XsiVuoG0AYzerhvxDSrSVv8KeJTQ3gjE5z1E96htUgpV/QkupGgevurxK7KODaTkHDW4byb3xLmFqKLXrwcXZimOEI5+++ikmiYkAuxC+sKtkprSIVz0ikLl6YlykZVf2Lu5WT2VIsDspMVMZbF5FkFM+b5w/S47OgFKjxCoWIxbbvOGxCiBYsVgtxLwhRRGwgKbVNrDPaDYSfyec9Fu+NGnusejqIk5PuERH4CZcvKAs/GPWnFiJFQulsnKKo4rIjlxNX+M5wi+/ziRNL111hbj6a0g89Q0DEW4snbuRMvPjVOlHUN6txamln6V/zmS3k4Y1H/5oS8cOurMlIoQYB3gzdBThJyT4RbTwxBubwd+Nvv6v6FLPJntIFZ4yf8Ju76b0vSPEbAqE1K8xSz14IteN8LJw2n2/4UXpIXT2m3ht/EIo/gqKviAy4SonO2bhWyPCpTWa5gdefdXd1VwULlIS64m0W8K5Mefocu5iSzc87QRB0TUcWHmKTWO2UpFdiUWsxawOBo+R9Aa9yvity7B0OqBNgWvDr5Eg8eE/ygnyvmuIzEMRGfV4PB/DnOh3qPb4gTUjixAuPcvFrjS2vfsYypROyoY/ypAp/5IpVqe8yN3ckxxtHoJdj5SaIPCU5EAPML8XBDEUtRN8chwJeQl8/MYHfCIzUWAX8XAfA2I37u0fg1nuzhsbnkdjdkOo38BLfvkcK5kMBjH2hfZs3LiRY1OPkT04GwZ9y/v7V1CsKCXaaqVwcCNPKcoZbCpCrtbx61PPMTJLyedRnzNB+Ro8sDyV1OyEtkvgEEVanTNNl0binz6Ix9JvkOh6AY7FwKRsLKoArg2XcW34NR47/ig3jw0i8JEHppQFH0Di+7h5dXB6zS+4Cu9hrf+IID0gGQ09NqUHxKBVaRl/PoaU7LnEf3YIlUxBSOfP3B8lYdDjgzj22T5cGsqoTpqJu7qWDON8MhxgehSs8VvDl856nnCAdaWO9DW5Itb924kmSCQ4NZdg11lDfcxEjkz7E42ykU7XTp5u34JVLKHdLwkHSz2ReV1seuYlnvjwB0y/TsbPXwP2/D8d/wPv+F9APf8P7Pu/+F8Li8VCZWUl8fHxiOPeJN+Yy89nvOgyWwgUIihvXAiR29CZlDRqvMipfB3xE51I7deSVJKGWg0IoDXaZpEs0yD6DPdoETUTUg3x0qGscDjEckcYo4R71ne40wfqrkcZ3m9ggfkvdrUGIxGq8BBroP17wrK70NYaqavJ5KdYDc/E5kGArch4XXeWa9UQPageVwSGq6wPX6CfesCCGZcORxyqQjna8jVfnHycl/v3Mkw7lbDNz1MzUsPzM7uJcsthp1nLS+3wUruJ40Uh3N47lqOPlrE0OBfrA4mVr7pBI3Ek1bkCo9aDRWcScekYjs/0XHZ2y6g0GbCX9bLKwUq9VkrC+f1cLsgkfnI2t4+e40rtIsYC7ikqApNPkfPTGIJa7Pi1q5GJC2+ASIaypYwfDy6hoSSAAXsxgUwgbWwWaNWs0CpxHnSbMT6D+VIfiDalk+E9nzADGOp/kDmbZhB7J5nud6vIF6JIdbuHKGge3S0NhAXr2PrtDGpvx9Lo3cHnGjHJRwJJkMEcO5Aq7pHwjBOnO8PIPT6ca24axk9Vs8wBpnvf5k/1Ln6934pjlQlZ3xEMOhlL7CFNDpMaQWYZIFoGzQYLrZnx3NcPYqFhPZRk20x6sRI0JJdsBzXdv8/iamgrYz94g/umybx8E6QZ8F++zMgFoHonzsJIzo01oHAIYdK169yrCGNaxvvkB9q2e7/BwtpcNzp7ZVhU3vRIAimTrifc+CmzPQv5M0pBwQvn6GrfRILOD5+Q9/GXmMkAzluu862sh9KAStxb3fGr80M6SEqEFGaowcVah6C0pzZmIm4OZdT3H+G81cJSfRscDoKZ1Zyu98SpxZmBKCvb3/yEW+JAji/UM+KD6WhqixBEetpzv2Vq51SOTzv+8PrK5fvRuediUc2kaswzCJ51xIkGI9H8g4CVNRmn0Q05jbdhIqlycBIGCO0fzb6wC/ze7sGkU05otSrcnWRscfMh0n8cP9b/QYsJ+iyO2I+7QfWA/8PjSQTpw/8LQHRFKH5VQXjoBOKylyCkboL4MeA5ml03vmTrC38SY5AyQ/oC5XcAvgEgtDyU+Hv9GFp1yI2e6KUtrJD74lliK+WJjVoEnYWjmXuJzPiWGHe4HgCfdMLFelfCxweCcQoB205wKy+NtAm3APjs2m4yfP+ib1wjV12G0djwMzqVA+M81wOwvTmGTwd+IzO3iqzIMPIfmF47CBCh0FKlLCdZrUYmD0FnzqS1rg2JXoWrcAdy3wSXFNa1qqk13ydddoOw5vWsH/EestouRCKbblZPby8WiwmLyMI1HUjKwBRZyFKHGFpqfyBtBGybsp4ecQXPidNJMNsAUKs+AntdNPL+Tn5/6wUmjslh3OJ7HDbEcNq6mC+7XsAo7mVE7BgWz7TJuV47P48wlzu4KasJqAa9FVQP2FYeskDGqSVMV5mQamq5V6vj7k573OZe5vqQLCpCy+g3PAY4ozSVsXnDVE41vk1NSwKx+bFUB1XzW8gAd9vU5DSrsYoM2PvYE/36bHKe/53JV0byxvLzpPr8Se6dr7H6K/j4tY+4KLfn1Lgswq5/j68lg6amW9j3O9BuzOePggU8m/wdgsjKiQH4qFvJ483F9Ko0mASY1QRnR/3Dn8W2rofBsjYWOcLR5kquRY+hUu/EJtehjI/awAunbuOjq2bmDTE5l12AErqcO2kPew0vmQaMvVyxxtNQ5o99kEBHuA+iB91zgkjAsceJwJpAOoxmagNrGaZSgN8icEmmteYCG7Z+ztLqQYT8loF7vT8tomK8ZVqekcG3ulzsZ9eilEXx0vdP0ZdQjPCshGEBexkVtAv77lTOOMCqa5OJOOOIQ+BijksFftJfpK5hMs9aamnNjyU9PwjnLmcQBBx6JnJRvA8naQTZVgcW1pmwTDzOqlEDHBpYzHMR6/6d1/oEXvj6BW5k3OD793+nwBgIfEOtOZT6whCc/bQQPIDaCLVukDRwB/T+pKGh50omTZZ2Hr2wioVpZagH/4mDXIKXXQV5ZjMd7t30lx1Ane3IfUp42bSFi85P49ftx1uTymhsi+BE8iHyHHOYIllEhOtNXhsxj396dvLL1NfIbRvK9Tdf5FBiGVHfv05SRQJL4t9Fw0aiQn9nQUo1voUBGLThTFW4wPFoiH4FfexGlme/RadGheXJXbwXkMeR+lFIje70RO1l29uf81X4s1h+PMiROldmicdxusif89XLuZL4AnkBlSh8RrE36iiPOMBGe8Avg/EH85hCK5329tDUzuFmLa9EaRmthAPe8FZLF1sycwjSzWJE6Se4OutIEr9KiBR+61JwvjKG0N9TcfI24ihqx2L5N6E/Xr6KMQMN3B98hKoQf+QO7yASAQN1UP4jJHyAIChpCbWRVfrm7MRV3g8PCKmTTk4iotSRvm0Cn63/DIDRvwymb9gYvHV5SAf1c7d5B2lTRrMg6iAWkQqRRcOhfpipFlN5NwDBoMHiZAELCKlfw60nQBXA2vCRyEJbmZQ3E4vfDzze/hR9Fz/kk3EjsCDi3QI/Rt+L5fisgySF/Zd3ndWGRLoqCigfmoK4IJ9hpxrJG/4FGLofdB/B7aoAfIp9cInQcVAKE2WBFDOJ52p6KPS/zXOueaQeiEDZG4JojBjcMmD4Hg59/ypJJ0/TIA3CrRsa+vvxFUfxxvlzKCV9XAjW0+VsYsitFj7Y9BZrPtqKV2ALjw/ewPWkG3y9ZBYKnT2Ns/YTI9dwK2w3w2S2uVeMhVXbV9Lm3s72ldvZ5a0nSv86P7U/KM5jpSQtm/L4fHQif87G5BPXg62ooA4gM/YDhuaEYhUERpSe5LHwViIupj0cmjNJHqy21BHU+wg9PaDqPgbFT+EJBDSsxemcK7df7+GnzPOEygAzUPodudZ3uNsjRiF00JH4AxYLRDmVQ/5xcIxD77eM7HEj0Htd5rXAAi4Fam2kmbFnbfP85TnczzyIW68Fk+ggnYaxzBU+wMHdllQq0TDp0RPsOZFA4JdPEDu1iIJ3XrKRF8RyTphG0DTmIq2ycELdZ2MeUgeB4Vyp6OT+ip10M4+o3AFWHV/FVLtCzr75DFxfTIxXKY9u+IUU5RIKxaVIrCImGgcg+VNI/pTmE48QfzecKxPLeP/t/QRbx0LMHxC4BASBg4efxe7uVSRx1fyRs5KRr4gJVPyCk6INR1kTK0zdtLp2kxRnokEewzLDAPYJ79k8SwCRywBHHz+GX9kf+HXOw/P/T2Bjiuk1ahrM+J/6C5MsF6toJcS/h2n/OabeTEKaKOHYjCz67cr49j+PtyAlRWEkSgvXPWRs8f+H3zxlXNPsJIkNhLveRuyoItdey6jqdym44cjzTy/DwbccgPyuOHhAGHp85yNs8mvniafuc8Pf9g7pJefZpHRmUa+FG6oefMUGAuUXAMiqn22bNxJbuOc9iHcvZnCodCRL3auh/iB4jsVT3MhSBzjVb8L92HFOd01gwtLTKPWN7PKB8B4DTk6BFMRcxdsr9eFY+Fka2OUFL2o8qYpuYXJ4IenWcyy/sIMLinOodMG8XHIJq5snI29GEpNWgG/104iFH7BYxSwQ7lH726PcH32FxxePosscxGVpCEPlcE8PobWPYtk3mP7pEpJibYXkHyJsBLRP2rZitL7K2Dd/4bs7MbDnOw5Nq2KW/F9PwdmyWro7hnIjOY/7idmkKRbDmE24NsHofBgcAHtjjlHtaSuG9ehtKhTFOj+SyuXIxSKU+YOIcTBQKxbzVTd83aHkmE8nIrGBP5rtyVyZhTSqj0utw3jMAVRGB76aqmBHh5q/c64ygJ4U5xCche/QC2AWdPz93FscGYA5TfBcG+Qu3EHU7TDsZN30A35NXtzfHUbS7HJ+GprLZfMtGzkgcBFiz7G8WBlHdXEQZ18+zDMB+fRLXkfoyWdBYA0HPUpQxDvx3TO/ISDicX06/9HaDM+ohgh37mvDMcw+SXRoDc7yVpzlDSR4neDnfbOpL3fmkV5/7NqGMmPENp4xwrudsNQqwXe1lNKOMMaNvkKqXERZ4EXO7isgtSee7Pjf+OS1Txh+fRbjTlupbTqGKniAfmsbtW6/Ighf0ejTSH5sPlEe7fTK9Qiif8tGVkHCXy89j1InJ8MoZWjAWcb7gm8liKxSrFZQBHlRFutOj2MPX7f5M1fiwY7cDwmIOYXZL47ewiLu1bkw+qmbrPasY7VnHTsbBvNRJ+Sab/L04fmMjKslIUBGQjS8eOoWTQFvkP3YAfzq/EhuSyY/Lp9qJOgfLLly0ShCBoN/Qivnr4XRMPAPPdIpxCR9gtAO6yrSaHauZ1ZqC+2OBSytTODUozbZPZpvMKY8jLZtU+mX30LdbcFD/Dz9AefwPvw2Tt1O1JcOEHRPim/nLGQvyW2/E8sYbXgXU/M+1Bp7rFjZ3QdD3BfjpCjkcMkLbLL8Tb3rJVzdImiMv0Sjw3TOHC+kvjcasyobH5d+enRyLK2gkzZyu7WCCiPszNtIsddNuB+Gg4+IFk8F5xpfJSXmZd4d9T2vn7+AgEDazTQc+v3w9i3nmbTVqHvuQ9KnELycb473Yt4biNcQJUfiYUa1F3kjIqFkE3b1oQTe6aJzXDKubg/W+K574D4cTAN8sGQC1+tm8GOpP8tvDmHYJBFysYZHD9Vjtkj5bvg8rtXN59qOmTwrsvDOO2W4Gu6yJG4jdlIVlb9GIHHy5OmN71CvTecNXTKPVtsOsy/6BuO/qeXxY8lYLw3DxVDB1a4grlur+bt0AWsT0tm+6DV6DS6I9KBGytHFtht9ytLKWcGA8fFdHJFoqfSsY45iFIik3HFxhDe+pb7ndQYFTSFt6FZ63ccirVNx1McGJmZfTiBQ68yF4GL00hZU1n7ed7X51CdpDAy9OZrfJRJC9g2jclAZu0NM1D0o5LtJPycoUsfBUzIk94w092SwJOg4H/vCUdetBKl6UYn0XJlxgccdQKVfzmA5bHKHP6wNFFh8aEg5i7gsjAl75jMv7ico3gQiGXiOJM3xV4LdzbgcD+bCrYlUza3hT9diaozNDItNYrWhlrC73jiI41FPcKJCp6TMqCX16kjaD4/E/m1nOidGMjHhdxpVaYQBqQqYYS7GHHyXosFfU3fdA5W9hiKDzbO9wQwhvs182a9l5QfT4YANuf2lxybzGRU1jtHxq+nQw8miF5Ar7KntiSTAsQQfCThINaj1ChQSM72yHjpcarmpOgizbKSkojPHEYXmY3UUcLxiIPOWM11jbETWCrc0Gpo/QemipN9md82AspgQKfRZYc/EkyT3J4PIBmoAPOaTyHhrAzL03J1ZRZhqKC/MXE7W7aEY68MxhmkIzUkHJ+jvUaPqbSYgt571O9fz3bqdDzr7IK59MUPNdvSX99M2WE+FXQUOgk1eSySIkBqlKHRitDVdhJeFc3lkNg8diBySaan0IbYlFNVcNX8v+hvHgUHQAYJVxJZntvBkytO45gg0e9VT4n/YNq8LP+MuT2DCD0+iDdRxL6WASmUPj1sFRIIVq+8Mmhuqqf99Co6eFuonN7Aq4UFNp+4A9sCbaee5LLnD3raZTLsxhLKBUCocUgh9cN7eTd5MPD2R3MQmtGJfxCIRKL2hyOZRtyU4h4MObfi1LqDG4wfmCDZ7odsNt1l5eCWunpl4SedRkGpT6nmmDTw88/j13GEm+Nlk+lWGIMwiLTJTH/QUgNQOtysZTLmdRn2MPWpdBI72PjZyliCmW5fMt++ms6/0A96y7CJxwJ7grh8RGofA4K3sP91I9sSTKN1icbeU8v39j3nksUEc+vUWgY75NATUE7ppG0evZnB062xmzb2C2/gnQGqrA4usRhrjCulz7cAosmCxgui/1qzRXnMYyCpjwNGT06N7SHUthavzGO0Gp6b8SEF7JtFhreyLP02kdxe3w4FdAjGy6XifH42k24gyvpYlIQ8IJCo/FM27WeKQhloDYowMuT6ELucuxEOvQNtVODuSb8unkJITgTq+i0RPW56Hsdv2/Rf3Hs87QbEBnKuLKS0IhEcenHDqZjB08UPwHToCtKwvEiPnQZec1MGWuwP7PbO4UufJTrcufMdm0S2VodAUsCftIw4Wv4RYkFA5I4Si9n7UxizUmihKheeJsH5NqBRMBhNlhzL5NbiRFza+hM8bUUj+a9wsJgt6tTNWQcCr/CqrT8xjy7otD/8uaDX0uoeQ8U4Gp2fZ6k7F2VHIWt2Qupsx2v+/3dn3P/CO/wU5z/8D+/4v/j8TpZpmCgpv46hpITA8kJmezwLw893NnK54/OF2UREWYqUn8XNpQGSax5YpQwlwLGRs4zLOP7+DwdnjefqtjeQt6+WjCDGLHCyYAYWoCxelFuvgHJ4OywWgZSCQ/cFVXNVdoc2qZvaT1Wy9FMeYC2N4fOyP0GsCvS3z+M9EJc9t5JkD77Dg+b/5wgWebfJGfrUXd2c5qjGPsT9uE30OfRQ0XIHRX+PSdBbLuGxaAxNJ8LwIQFD3tYfXU+bhj3lYG/bSVNx7OlnRNIIXU4sY37qDO2JnPlt3nB+6BW5fGE2KRINRNJ9dAwaO+8K3PR/zRpc/F2tduC8qo8r1FFrFUQ4Wpzzc/8W6sxQVQe2YNex+/RPOG5PAzVbUovUkzsH1VAgmCuJdecn9NOPkFVALbn4+HJmWy/igV0ndnkiDVzW+AXV8deN3supn8UzcGHItCgpKdGy7GUfxHAlLE9/EOORH7OyG4nx+OWXyAYymRwmXZbGkcSbVCifedu3mnKSKsf73OXVfz7id46iPlCOd3MLxAdjfb+XTgLdZMWMXM/d0M23Zc4S72AooYTL43tELp/7r/FUfz/acQH4e+TgD8naqFC8QF2gr5rXKJ7Ap7SvCpPGM/wK2div52DkZHiSRFgtoa9tYu3UtWRlZ5CTnwEIt1/Z9TdjE16gckNI18CTuHv/KEf7dB/trA5i9ZSy9qVoKh6ymx08gTrDpzr98+gbPDJtCalgXU69KeTpjHWKJTQt9cC0o6yYyLuA3ekTgcWY8wVXBiOYIJMjhc3fYZa3iHIsoXHgTiXUZBvk+lj6Qc8Vok0bxaXBn2sFhVC6zUBMKGqMdiGUo9DIWb3sM7cguPBpleFakcHzacZQCPO8ELxp28u79SBprAhCII937b4KNeyiwSvg++wc6o2IxCfDM9Wk43IlnxZvbcUvqoFHrTLH2OfzOV+LY44jOUUdz/wgiwr9Bik0yZ5hCxylfeL5aT9xfc6gKrkI8UsKzTjBUAXJjN9/2BHH16ggkq0ZRq6xEZa+FB4XXiZodTI/I56nCJqyOThj+q1CXmreWyIIBjJ1GHtu2gabEvTy39AYAPaoURlQ9RXedF22ZVpr+tRhgiAK2WjrYnvM5wcEgHa9Cwr9yEFdqFxHi34/32JOI25MJyYHwoDp+7UlnRnMOEuU9TPpi6BpBs2z3w9/1WsCzChI6JnMkYBsB2Z2UoMY5K4/qQZ1ctAxj+eR7YBfCjTYP/LOCEBK7ib34HZetw5iw5AwXxI4curaEG47RiOe48r3D9wjABV+gZDMhtYEMOV/LgM94gp1n0qut557dWJbm3qTWBBsdKjHcv0t3XSR99n1s7pczSiTlCZ9vmNERikuHM+o+N5IDl7P+zDQWjzrJp+OGE+V2k7PVazhRFEpYiyc4O+CpFxCCBd5OmMGI9gNorBpeqPdiapcd7koxZ8afwyoy4lH6HpT2YhRvZn/ZauqsYViLG5i/bz7bV2znz94BWi8lsvhIOs3TWtF2umLstX2QxaiNpCpgw0/LaO/+htDMhQDk6KXYNx3l95h0TrUM4+pLT5Gvk6HaZ6X7lhv9W9TYO/ezvAUcRZW0btrI3a50Jt+Zg52dhqTuf9hmfhwnWTfVggvjThqQ9vigdrtHGwJD/G3gaLTbdXCZSbmijv4QV7bOfZO3A6pxvPIp6Gph+F6ukMCO1z4mrPVFAhpyMUvM6CvuI7+5kpFTJHw+7ApL3DcjtAgMU/RC/nswo4IdvWN49cZyfLQuTHj0Mq95l1ApSmW5x3WWA/3mKrYrDAR1ueEutmOSnQUseu41TyCvdRTdzof5Zc8UFDIT4566jDFUwbF2Ly5pwRs4WRpBxsUQBlbtZGlCA/WmjxiiHqBK7IBUUGIRZChrfZGbE3gxrQiZ6t+uXoCprnewjIca1xr29sOjDjWwS6DU8gZvv/Ih4dJRjPvjGxaW+PLu2wJbrDE8G74Wxb33yfzoW97vHkvCVn8KO+yI8FjMjtwkduR+yN2YdHrNhQzpU1BW646/rprzQ59mbD2UD3Tw+Pg3OVO5km/7FGQPwFiRI+OCf6W2J5pm41D+/nIefd123B50i9PubXwtC+FU+WosVjGjBNjeFcXm3mqCWp8mYWAao53aQB8LjnFoLQbMZwpJqgqm8BUjwVI42ZHAjbJXOJe4F8FiwqwSEH/+AT+7wkXL41R2JVPZlQxp82kQtAQbRxIlg8X/xTZ8NfUNJmlESMxiHj02GNfuTDynNtBrgdt6kOrTGVnxDZ5qb+arXiY9+DTe0iaQwpQGHf4ufVx59DfMDq7cGHQH3wfZ9ZX6ydyoXs2cgcNkzbvPy03VeHc9xkSfqxh8v0bmOxWkDiiMZcyL2cethulMdtUxzR529cHSZshNyMXk5U2S1fzwfKUC0HmXu+YExtddJoFO6gf0/HL1fX5dG8GdO2PpCIE/K4O5+EswpA0QIA7AqBLbvM7CbDnVzaWvUvyXirGuJpzaXmHox0fZK2/nlTM30Jns8Ans4sZMAVfRcn5LDvl3wHpLQdPIOPuFpCyVse7oWtY6HuWtwDugs4CdbVvHTjWZVxKpaG7AMcKA9+Re7Pqz6BGyaXc4R6dxGQHVnng0OSH+j8eWScOHYX9zJHI4Da7efPPCV8gtzhwQQ27LGADCfKCxE0o9nqUtvoE/hIm8mpKC1NRHrKMW77ZhSA1Wtrr/w2FJByu6TsKuYWAXSrxkOAEzy/irzY5Vv71JzIwDhKd/DDzIi0z9tMXq2d6r55lqLwYp/uulcoiC/koi8/ahxw6lZjCl/WUMGpcNyV8S8ckvzLsTS69TI04VTYwK+QfZACB3pcVpAQtDC7jcF4JcZLQBff+J9ht0qZv4ddXnOJuiWN8Zz4iAv0D4ElK+hLx3cRHmAUl4aDOR3+zjc72V91f9CSeSIeP3hxJ7OmkTbvaFxFitsOtDkDnD1ALuqCYyLmcPiToxMVolLljBd8bD5LYXR/ZmXsS/LQT7msncM08mYxjMrA1iemYLKwvCGLA3cG3KCcI95eAUC4Zu4uXNqEypuMhDOT7sNxCsCGYdZK0C92EsiLvNsI/PcqFlJqcdBtEoPGBaX5gIgoAh4AR1WiXqziKqGgNRK57kxKVE5BINt/pf5qT7egSrwB3lNhBg+sBr+A6cAMc4cE5gs0cuMTJYs7+TkPofUC0fBfwrK7XIaT79biZ2uL/CoowvSCpaA4GLyehqo+6kM5aRWkz2o1AFXfz3fsicWfvF69gXDMBzFvqk0GGWYbVKWHu8FEGw0hP4LrrJFkZc6sWuKIuOdaNQW6pYfrAZP8d8CqUxnG96nK7ecvw0enqchzPIYJOgO0sw3747lWE9DmTKDcxZe5BWvyR23Xn24Snc0r3H1WIfXGrPo/dwxNhwD+mdpyHjN+oM77L6/UFEOlhol5QgiGzknaqQZ/ntyG7s6tzxjI3nuTc+pdfkA7wJbdcIdAjh3Wc+Ril2xPLCduYG3gZgrvcd0vvSuaSoJ/ZKJlXRbVjf/4JpHmZEfb4cWKikpD2NHd3BOC46Rm+PnMovVuI2sYBHQ7WsdoenWuG+WoNjSj6H3W/RV+HA1uA+mi1WvCXgKe2hSxZBTA1MuBvD0OttFKa8yiy5y8NrnhaRh+nba3xQqeY51wHcFBUADLY8ga5BTPfFlVTGt1IjshXMgiTA2VGMcVfw8hunmKB8hcEFsdQ6SfFRPkigBTN9LW4MVIfRpSwjMPU+NdixRFzLEk8o0HbyzZbZ9GXHEbnCTLz/FdwEZ8qTbSD6p3W3+asP7ugh40YG/Xb9sFTNsyfuo5T28f5MuO/yDqXpLXRov2LZoSRmTXWB6h3QegH5rDoi5w5gqFOQ1+/KgmbYEhMLLRf5zltHfj1oDFp6VTbPXysWpoR/R5rPMZ6/uo1+k5qgIXdpcCpH42nmVutS0jx28kj8u7z6YxgJeS4ETIZQyVUC+s8yKr+IdvvzjPKvZ/Xo17me9QGRMphqZ0HfsIwnB9uK7y91TcHDDi4k3uDg+CtkKZx5o+gxPIOuEm/XgsTQyvG59/ipLY9N9Wo+7bGgl9g9vFcDEm+uRhWiVer5eEoZSW7NrGqxyfK1SV+jsSgTxvpzdOrPqO072eDZCdTRK2liX7wzcYFwIuEaneJ8wpx2MbFuPc6KZg7Vp5DT0YWLygXdvRBOW1VkTCsEUx9PrWxgTG4H/goXXvsnnvRb6ZiiyxjsMsAYB9tavWD4HKTidBq3RpF+Lhjfjxrwsf4CeW5I7Reg0EiYcGsCxmQjhzIv4jkwFgxdgAiRayqvV8zGXy2haY4OQ/BnVJmn8Y5IhEunC+u+X8fxtltEVMaiqBQxJ/4dsFqg4hdGDGrn+57nyBykJuOfNEqdu6iLXcQI55mcr3oUaX8+o+4n0BxupTEUOiwqRL2RBDndx9X1EodcVLzfIKGj6h7q7lMsOfIWbo4z6c3fR4/Dc1x5/SMCW9cx8sRKKjqbCRx6Ejxh+0x/KrpmMjc/mbZeRxyklzFW6jCnuyHuKwezhmfXfs+aky3IRO1MvLSZxSPOQstRAFpaTLg25lFR3s/ObUuJTiskZdQ9GHcJi8VI7e8K8ty/QXl6LaFZE1kwbjPOkV1cqllCu8af/kILptI2RDoHei0W7rt8RkbdLGZHf8mh6m9p6lDRL3PguZP38A+UQ8jmh89QicidZnE+F4Zdp3bMJVrt5/BP7gjKvD8n2KjCIlaS69aCU4+CgzfD2e5moDW9BgCLICVfEsTfvrsRWeS4iWGEqhXuv8EV0Vg+kpYzXAUipZz3L55maRT8GDQMgKeyPkL9ZydWFx2hq/NwU4CDtYc3H0xFv+fHceVQKM7xjsSMP0VqSAu/GCfRb4S3XWD97UTEhWpyk3/mapqOyTJ7nneyeVpdsTTzbNFg9L31POblxxD3PLpEOh5zhC3dYPYazFmdmbN9ZwkKbcY3IB8Xzy5oOgH2YeA5koCmfVRszyAENwJF/WzzPMVPPZDT5MjIO5HoXDtIvBzCgNwV9RQHFEYfNOIKvpO1khBZQrRzAmHRpUxK+oNycws6Ebj/V435cMlBjuhqWAREy6DaCOtaYZ5SRKFFDhU/P9zWUQQLmuHbQSnot/dStKeIt+68xaUx13kp8EkGhX/E5/F+fFKQxpYlbxPlFsWrMY/yq8uvOBtjH+7n985QqjX5vOkpxmPSMO5Zo/CQ97OhpJjgvjGo9WEAnH/+CB6FlTQM+psJDbbfuvWOw8ESil6jx7PZkyT3ft62HgSgziyjzP91nLtSiHYai29XNHJRC6qOHQ+PvePuJkx7yxHiMmiL1xDV9O1DwG509Xu4Hv+T7uRuHB4dSUe2nDS5jUgqEkRkDcmiMtqVTycs5qXhK4jGBc+SmSBZjkgUQVJOEmm30zC+aCtCCFYR3z6zhaD2D3iyFSxWCyiV7Fl8mT5VHgdqpjIn8DjHhj3P5eZMTtg5kRV4l0BVO1W94ZwseYl1X6/hwz1PsfyyG+pYK+2P/EyQn+Hh9bTHn2akeRo6LezFys5HLtHmeAL3ge2EAhJrD6vSK8j2/oX47x4jYWAciyZtgax7ELKCJquKv8u+R4QVs9SCUWZE9KCzTywSM+LyCDKyhnFx6r+FlUh1FI4yXxyKHie2tICahCpe/Gwx7QGpOL3gAc0HUd9bQV+ihRPubSwp+wrnPS/jvno6eGlB3053g5V7vXacUv6CY7YX2RpXwtO7oex7cEmlV7KQrCFZeGr7+LSklCQaURf8yMzIfh452EZp+iRKzXVkdifRkJXKI3FenJzhC8fjIfMgDrpCbkwr5uW6StQnRmEKrkOw6B9eg8qQx7w3j/JP+UKO73AkL8GZT2eJEKwWZsVsouDKSDzjPal7pouBroqHv9sXcZT33tuIUa9HP0XgrAbi1QF42oeBUxKZ7TmMq4/lcyUMPjOJDn9n2JwA50aB+3Cqoly4HnGex+4s4cMVb7D2k624enVC0ecIvjNY1xTKyRJvPp/9C952tUxWLSM10BvuPAdKH7Z3BlFnLCPobDNBpW+xcuoe8HkRpLY6Wme9L20fzUMx5j6PZx5klGktC61mRAYdpjttVJ6rolx+mJywjwhqeRbnus3cFTYx85+TDLs4jIagBuxOZXIsMYfJ+lZURYchcBE42vyp9Q3teFbZaiq97qEURhcy6uIoNCoNpIO8uQave4fRNC6mIHMNLg15WEPz2TD7a+buH7ARGv8v/tfi/8C+/4v/b8TVhaR2W4nPi8ehz4W2ujvsbl7MqAMBnK1chUgwYbGIUfS3IbRZUB68SfC42yi0ofg6FgIQKesgVAzVwlB0/r6M85zBQeMW5OUPDmFawxBaeKnBmY9dYJL7SG5WPsmPugj8gqZyXlbLHM9ncJjSwNpp58hSW5iT9iO0Z4FpAJeW7xl0TIMxOghDoA83dM/y67LFPL7jN7xPdNAarsHiZ6HI/2VGKGC7YzP0FhLkNIgFKzfyR8sHfPGskfi0HBJW3yRBBoFSaDdvJcfZgZTKbMbd7iHsiUACOn/gBw84qOtgbD00GgUaQtvYpszjV/kHVJng6y6oNduxef9wHEtD+OatD/jMVcC3YyMvZlTgoa4GriB0deBSX0+9m4i/+2ChfQ5cngMj9oNDLKsiWxGHDxBipyIz9N/FzU5kRS6AWWtk3OmR3B6kYsXE77ha28qF6uXcTsmlNukOwo1Xcajz5XCbhaXhaUjFJqwKH2YH3YOge6xv/YRhgX9iVMZwrKeRv3TdOGljmFNdilP/AG8slzLCrxCDqJ5JnhBaDY3dgzj0pTcitz1Ed9zjW8GPp53rud88hjvlT+I9wpmDd4NxykohdPQNhjpCvXIKcfHjABjhPoubJbMIDYVfn5qNTDChEoNOALeabFyL6rBEpCMzyBCbH2R8YgX+VPGiF6yp7+Vc9GVa9FUE+pxg2+3J5BvALBFxKfMSgnsS7w5ayfCAvWSJT3Kk5Fk6tL780OqPYNEh6zfQXOtNv3wECfXf2/bv8QkX62O5f3IKh3yqGIgs5zl9JwfNkF4LL8Qs4x3XsXQ4aXi8tpv4B4RR3IbAqOOgaeTvEdnscGxEbzeHmOJIPk9ph10CtyMkLJrSgX/kMrIGdVIqsSW5/hKbBGdBlhduxyagdenFu/4aEZFHCDDXERoJZypX8cL9o2gaNFidxMhcqh6ClG6OzswZsYRl5SMQDErc+gLwl3dzXWPkOWcbA7LB5MlLv45D2+RDQlEIRqkRrSBhqAIW2gMyB0xx/qgdfFElDeduRxqnDU5MK14MYhVSi5m+u9E4dQtYJP1I9AKb427iE13Lzqxp5Lrpme2rxKW1D12bOx1mK6qMH6lshSFp2zno1YaiOQT/1kg+tsZzxLyPG/6wMaCYf85+RI/WlZ6oRLQWG2C6t3ADKmkPv/QauavTMLznAInnGhk19wIjfS+ht8CR/nx+6p1A9YbPcbw1hpe+eIkDcw5QFWIDU7TiVorECyjIuoo5pJ2CIR3khR0neiAQnBcAoBpQMOrSKHTKHswiA4WdCUzgDH1d9lSficBuiBWzxTbO+rAHxXvnZKpEw/AKaMaqNjLu5mKcDLd5NH0NUhdb4T/AsJv8W8PpDCyk+JlfWGuvJk8aw9H+Nty1Fawrnkzv+XT6plfRlyeiO9GNUR62JO1W0xyGFjUReieFNu8u1N0JWOcKjBjzBxi3oJK50uTXwaYXN7Fj9Bys523nJ9NUgr6ebjc5bxVew6exiulzNrJn/h7a3NuYETCY7qQaXu0uJkjpwJHVRyg+WAxAqaKfL/v96KmW4NKux9yvZ3hJNLuSW6HgQwBGij2Y6Aq/9ZjZ7aehz1TNAekcoiTXmO3URrrSjceUlazW9hN0PgZf/1pcyz9i5YN67i35Plr3x6LQdAIpXJxwjW3O3/FscAQx7lcwKZ/igtMiDAkCZukG5tuDUldr0/p3iEYkygZAYrCSefggd5PvUt+XQSjwkYeJn/tB0mbmnffe4UZmKes+U0HnPUSCgH+tP85mFflBSvaZ9Dxiuk6JASyAo8TAemc4OeBG95SpFDglM0hXxwsZK9hbuIF/2iPJuOSAPl2Df1wTDTKo7DrGbDUY1HsoVk7hcuI+5rjr8HSy0ICZ/XEnALjRd53rDOfxXx6n2a+Tr3peJXWUiiX/tayGOzYy4tF8vj0ewe+7RyOer+cRl+tMk/3FOsBiNVMWUUm5aw0jlPCsUAAXJqIXjybV0EGKg5RPv/kSrRUEl38e7jc6J5ARp6fwz/J8Pnrlc/Z6hwJwzg82i9pJqYFOfif1xiJEBieEYQIRbjdp7I2gw+rFaf9SZJ4CVTNPUhgIvbUzWHrb5jM4WgBTvjfJZ9bhLHQTYL8J/7k1oCsAuQuCFbSunbTpFLTnzGByUSq+0imM9TtKu15Nf50XhoB+qo02KTqlpRZIBiA/WMtFLfx5wxeHQ6v5YIKSm8Wb+XDMGMwWOY//thbHDkcM064xU2Yguq+K5XqY0AC/GgaRmH2AJrcR3L8STFf6LFY+a/MBUorApLBSHVyNs0lJgR58JdCkdeV94QT3QhZjkj7Nth6oqfLEbpsH9bNciPZcCUpP8J1GafOvGPZ1YufxKXvaRhAR30BOgE0WqjC2ECE8jfj2AWIKYqjzr6PLoQ/md3Fg+ybe+GAEDbEa7of/zHBxB8rWaIYrwEkM16THKZwtILMvZ9X2VeQlFEH1LugphIT3+UpcSnCEgl4XHRlhFoY6FjN+ejgfXj5IVsMsRnkdp1bkRV5DJi8eu8bbTxzFyVUBeRvBI5Mt3e6UGm5yM2EHyb53qO4J5fnV1+FBUfjP0VfJj82j7YeZzLs8mvVLPofaP8nyUvDhc+sxDW9n/8KzaMQ17BRmQvstKP8Rv/B6+p/YRf/AGDAACHiajvH9lFf48c63+GtKCOgUsStDy9nEA4gcPoCAeXDID6eACkat8KaofRhgJV0BQc0P1uH+CoyukykZcZ3KvEQGnZNxpuBZgoZ8waGFYloHgkBUwE/NcrI7Vcw6toorVUdJXdKJaspNkNphvfMqcx85yc/3P6ZMvZQpI+ZjVQUghK7CpesQcQVxdNc34J6Vi7vsHg5Nl2FBJ63FVRwZHsKdwZdxzlrHs9uX8cLT5wlOyYC6/YywfMmPHnCqpx/nkiJcVXkI5l6Qu4NzIurmlbwRZGBz9STiLofgpXCAsq3QlQNl37NKV4/kwFJu6hWMVyQzddUDJrihC0q+QRAiGHNuDAJ2jHr3KF+5VdoknTpuwtDdyAUjbmJQGKyou+qw9DrD/Y/xEBvZlP0bMaKFdPn+RVJQEZ959ULWSljQR+eOPRyx/EalaSzlsjdxEIGABSq3Q81urlvSyFaVorQEENJ6Fjs7LfSW2EBIzzE4pz+JKEnDr9pW+uVFvDwQQFHZ07Ycxh+WHvuMsLsDdDl1URlSiWxxG2QvBbtQmJpPQ+9Sso9pCajQ4EATGdphQMfD+erm7YnUH9bTsmQbI6Y+IHApPLkpfYXm9HCc4/2JuPYOTvrhdEc9iVPCqwhhq3H78jxGSnFvdMb/+jqOjO4mNMGdFO9T5LWOwr35PXQ6UMTfYNbID7AqJzPrd1txbMDgypGfZ2FQGbn44ne0y2FoTB4rf/qSLq0XUYt3cXXwz3i2eDLWqkRtr+Ge+CcyA99nQuivwFVuXhyDfucRwoGAcVlI75y0nbv3RBSCK06tIOro54e12/hG2k5p0jacGvfRdGgs00sisE6xbe4gaQRdG1xfip3InXb1mwgyKZUGJRMb4JQvzIz62jbeDelcer2cWkM8v5v7adXAhkGbCSieSqTbLbK7x3Iq6m/CyiKpvjyE421RTHZfDu7j+MjJkcHuSspn7GPyP5Px3pbOnu8r2Ni1g2IjJDOVpyMFEnMSMcgNlKY/QqyTI/8d6zpGcWfgCGKjiI1KBXbKY7BLINQqZve9t5CL6yHqXwlxhUiAngJW2WmYoIAPO/uYGbeDwb7/kGNNYqRmMpGKAX7IVjDy2FD6F2YT2tREslTMVlcb0Lm4xYDCowbfRBODs1QY+qtRTssnvzOOOJd8VKZB/LCvGYdeB8bnJSP260AmktLQ9688mMleoN3LhcCcQHxrPGif7wvyKoi0gbfbGg7S0GTFS52FU08QIkTgN5O5P24mZs9oGjtLSSpOotOlE7FIhLddBTHuV+i0rMa5xgufpI9ozr/NzbEKssq8OF74CAZMHJo1k0OzDvFUezXVulEExWwl9XQbKxL+Zn+PHyPyHGjXn0L04Sr+iq7grxXZmKzweasL42RVBDdEkq3X86yoGrUplRrHP7nuDZecjoLwLZVG6CsJ5oWfl3Jo5qH/IUElCPDP1H+w00axq+AQez1z2W63AoDI3HZcavKoHd5DryqHTHkmYOuomhT1EX8ZT9Nvmcoy09t4GG8xRf87dX3hPHakhma7d3nkr0coHtKP/8sqErzaYF4HNJ0mrL+KipPplOXFUBtwj7LwMt7w1bHC1fbeBWlBklvH6XdjcAipRS1rfnCuVij8GGl8LIMlfcTeGEqZXQVSD5AC7LOhPMJiK2Y+pHoQSEM/5mgfZMg9kOkbSXfTcHvEFRbELKHXfhhFYdOZKwAWI9xawxhX2K1ciNUayvgz4ykLL6PrsUHM29OPySLjo4jt5GbNpUBXjV3TMNzS4NPxQ4h0u8W9tsF88d1M5EWhqPxrcanWIzaLEbB9n0qxoBQAzLh2OzLWq/fhfbjdMI2QkDZWvHia+SVDyNgh4kh1Jq+k/mwDbXyno3ByxuTpiktBBXFlXUydYiNN4pzMO4+9QMZYET6mdYgLA+ny6iBl9Gko+gLBcxwv/zGKyJJIyqa3cSv2GOGqoSzhOF9OyOD9q7vZdmw8no2gnbcVzYBAlOkDTlc8jt6kokGdyfWxXhitRThV/ECr4ExcWAP7veFIP1zXhnCj9D4apZ7FbkaedTrAWxkq+gSY165DZtUSYZViV+/BjL1TOTH1AtMrqwF44gkQPSC4+NZ583FFAgnj7sPo94m7tRfHPjvERjNFHf8Q6fs3IZousurncLthBmX1y7AMP4qXlyPLD7Xj0OaMeZczT7TARDXcsgymdeQi+vxuIx38Aph9ABivgvUusKjenehLgxBFVnJoRDO7DVd57fovFPm+ikfZX4R1/sDwk1Np9XHiVUUMH54PZJgCLmkhSJ2M2HiTZzc/S0VYO70LL6P8D4YeZlOt2SwF9ZAcagfd4mJGIwB2InjJ4Maf/0xBG9jCufk5tAi5LBcms8ipl7e9YImiiL9jinjRaRKPhH+CxSIQJj4FYhAXK3n58sfImvW01jWRetSOTj8XXLw6uaa1+bYHffkEjhYlL1dfIojBLFuYRlfFBexbC7nfdI+bv2QRVhZObnwuHV4dtFvbGO14m0m9t/mep3HuH0pyXCBmq418ohSAyj/AIYrAEieWHn0D6/M6dGEyiGrhK/d6YryPcfGElrrLYjrL59NV1oFdr02KPloGOQGw7e/hlDZF0FHcwZM/PMm9CWepSBzgtwEDH5cVM/7MJJydW/ljzB+E3e0i8/VC7Ou2QdzblPss4tMzkwkeGYGP08s4+01kueIrwiRtwCMYXb1ZemIp7rHuxF9KpbBxEjEP+PL/IaBZBQtGiw1scxJJUHWfhIERxJW+ztvTelgd2MaG5jjFAAEAAElEQVSLhSFM/mcyhQk63BUltrkuP5Gz3RKwA6tgRgzsKF9KYkgwN2sr6J1wgnpJBtTAmy4Q6lCGyHukbW0z+vLjUweZHrKC+wPwVx8ssgfi3sbgOp6x578j6uYZxmr9ODDb1un+H8BOYtHwvJ+Gr9Ua9kwu4rXwewToy6DS5rsnBD/JY0VJ9NxLpDpSRL/gjshf/HAfGpWGHmcNvtUGEm48xrFpxygeUQx9LpQ7fUz7IT2dk9X0eISjdfBAKPsWOm4iBi6PhsSaGqi0IjEMgNkCOa9B7d8QfZ4nsx1wa3dg+vExtEaXc2XSfKYlOoHSl0EN77PIBz4qs1J9aCyuE7NITDyJowImhf5ItVZCcH0wFeH1XE8/QYriUWi9BD35cOd5jI5v8W03FPTJybiUSZoiC5mhCRt9FkSY8PBqQN5oILIkkmz7Plhk4uT7aygvjUbdVc+AwRmdWcM5LSjLQRsmIAhWYjZ+xw3XNfj/nMTfHXHEf3HGJkE+o4xt+xfhtiUa2TQzJUMfQ+qgQCwqBacEiHyemYdj6e/R0eB0DGV4FVKZ8eH8LSr/iY8vRRGQNZiB53axOKSZVM0OKAKiXwbHOE6URzP5+4UYZH2YA+opNmnwztsI9qEQtIjLhms0RNzD6GwFDiIIInBJYfmV1xmXVUHRP8Wo+i34Wn35Lu4EeBrptnyP3CDHsccRi9XCz0//TJ+0D2PVSPL3HcBnQQwuI2xgn0j9L+OxxyMcx3I1gbWBaBVa+tPA4O6LWSyj/W4tHY5yVs9ehwFX3jh/ASzSh/nC/8X/Tvwf2Pd/8b8aCsWDCaQ9iwhtG1++JuajZk/OlsUwVlxHemcehxc9CcD2/JXUfhyIIVZD6cg8qrzbiVWE8cf9D8ltGUN6xPNEFqfzsZ2Wjz95nHLDPERVVkTYfN5OluqQ7B1DSEwBZ4vG4PvcEN4cvpDXO2CU8SMEuY2VNaDu5zNBj3NjGnOaTkP5DxD9CjFuPyIblMqJEDUDmpfRKoHGXfysfJ5vnwnjqnkcg9uaCdOF0RZRRZZeTIZ7JncMS3j/0DukDq+jT2JCbLSn32M+61o+4QlH+GBgAHBAJLPg6dGAt6KTt42RvGOupUzTR9OdaBx7HWlItGl6iaxG3nSBq1qoMntRmLEHVYLN922unRXfrnfxDf53jD07fkCVY6JDIWKjNorQqTCo/iCcG4X/kEuMKrBpPkcMrmHckWd4YokPjwRs4qva31HvfhzJfBO/rfqTHrsOTnY+T6rrPpK8zrChTYFJbGREgoF3J32NTHCAVn8o+w5ViuvD42d47Sbd7yhVwcUU3ZhFvwAJRhkvfraekqh2klddJ8nrLN9Zl3Gr4wLdZjjdF4S+tBc7aw2x+bH0JFipFuL49tYvtAwEM1Fdx8ejTiEbeoWnty0mM7EC5+hiwAb2Kc2VDK/dh64/mtVhT/H+6AlYC19jv24OWvMx1BU+iH0n8c1z36ASYLEdUP4z3Xjw0tuPoneVUD3uIG0Ot2k1TeFyRyT9yhLW+zTweGI5T5SYuH5VgmdMGBlJkyh1G4yrsoGsglDG756Dh1sup9uHMu+dP/nUDb7pAa0Fdra64d7iTMWIs9SMqwXtLUb7waBa0AnO/NoUi/GbVIaJnAkJnUr/wkvYccNWFLOY6FO6IwkMx6XcgwV/LSbEezf4tiCVmWDMDbR2qzBVW9ApdLZnReHJGecMvnUs5e7qnxnr+jwhfwxQoI5DFRXIiZxJJHudwb4qm8I70bSsfo0S+8GM8fXGOUGOLPQxVKU5FI+u54tOMV+3mjmoqmTCA2/tbzzg/Q4DVeX+OHe7sWelAol0KEHiPhY3w5JmMEsd+Kl4OwYDTBRsBtX+sgtg0YMgYodlMr7fqwmNMTDwzxd4y3to0KfybMwj7JCBSaJApLRwZmE6NyPeZls1WB99gm5dAWPC1iAPcmPprWgcTi8m4/0j7G58iS12m3HVhZJY2kO3RxMHUtZzI9DG7pof8wnzYz5h2Z1ljDplh8E3H6/MXvzCbO+/XAThUikdFhXlRoiXGely7sIoNSITYKYaeix3+Kvbgz/efpcUHqdT20OVuhIZVlthVWLHrvi7VG4s4rzsZ75NexaZWMfM0feocvAlbyR4h+kx1BeSdjONAacSnNx7IP4dPsm/xalFW5jgqGbEbn/szLbzWmJvk0Q9IKpj4KXvaFNP4RdPiJYNcK+vhBQ5vOECVcPyuOnbQn5FJJ7VJZj7FRQKw8htGUNu2wS+XLSMe5md/KGZSX1TG4sV0TYpiAdyEGtcuqkFdDUDLN25mHvJ9+h8bje+Tp6IsmwyJin3UmC1jMJYG9Hir5rbJPlBdno5/m0OhA8Pp6NXxjceX9DgU0aM8yuceGwjvYZg3muayrjdC6mxnsHf19bh3KieyapjJ9HrVXTFdfP+hKPMtwPcxpFuquCeeBCzz1lptfriPiKF8UN/AcBsEbP2eAlzFjnRkVxFr7GJGuePaAvqpcsYSkl7Go19EZS0l3A8Jg6VxpPAqgDSerW0DekA7wngFMtizjOmNYKt7SLuJ5ZQG1BLj+c7mD2H8uKW+XjWOGEJtpAfm0+7hxEqdkHFL4xSTqRj33z67a1cm+7ApkYdnlNGs676a8qMsEmtwPT+0yRFGsj3nExVPyyXdbEj932K24eiCTjK5y9/jlWk5lCNMyNc+3jLq4lVztBngQ2WRpwUGr6vdOJ6YSoblw/h3RJ3XvFoo1jvQrC5AK8l1zm9ZwzCgUtUD336f6yvX3TPZFxDPlPLwhmcPZj9E/t4ZPbrdJz7iPfaLRwz2VMd10O5/AzpYnvMag/EAxU4mqczLPdJYuLVzG/aiWCVkOMJj8+5hbuinH+uVjOg6UejtqWQneZ/PwTClH2E/D6X6QWxENCOfY83bsZKVh2uw2IVMWKMibPjbebeSQ+S/wFlGh+MGcPV2gXAWkSAVbBiMkpo6/ah2WEqzimPgnMyglXP8ScusKMXvF7+BqsoiI+3TkYkWJmdHcnffy6m166Svv5o2qK7UEVU8PLQxaT5HOXL3yaivxdF0LBgyB0gLbObmwi8cf4CLsoGHktfQl71UDRDC8hw7yBLuRyw+dp2DcSgq+lDozJzdM5+epw0ONkXUlgTw9du4CgqYpodxFxVsbs4BV1qP1uaJ/BG3BbK5HfRqwLZuXsMcfmJVHo3sN/Qzy2fqSjFtvauGkkkjT33aJWLCM0ayeZGO76f9zIAbj3jcZHF4lbtwIK9C/h7wd8UxdjWbT+hmb7wekweaoZkhxPYnkrcjM9ADq+3g7ODC5mZp7jQ4siJSWX0uhgg+xkwdIKpn6bgLm6FXkZp9OWEtIG3H9TeU31OYLAomO3xEnc6RmO9mIencw1O0zZDIzbWp3MyJ7vCyfh8GMOjWvhk6h0iRGE8f22RDewbeYQ+83CwBpEzUc44/9+pMLkRKmlHJLbgFthEg7MnZokRk8SESBBDXylUbuPpFi92DjQzTzDzVGk8So8BBBH4OxbzwZjx7P1pLs3X44kfUce5YLhhOgvG2ZD6LVaJBwtiF3K4+Hk2bF+AFyL48m+b2b3nSLrb/HjreCcKqzeiqW8xOeUiIqueLp03NT3xeIsVHDy+mIzzQYCWIw3LSdo8DnqLwDmJUzo5J8LzOKXaRHTxVN67eZqjr3bDycE8OqGRdclXeS50Kz2P72BtSB9a19nYAUjt+bLbNr6LesWo6n2o1fsSXLcfAhaia5eQWRZJfacThed0FDCH1U/KiVe6wuCt5F7cQO+WJJzDtXgpyxnidBna79rAwPbrSPRqOkoC8RJbKDWL6O+OZs2qHbYc1ms8bi03yCiMo18spbJ8FjfFd0i/Z3vGGLqbmZJzDNn1KHd77HBp3EZ65FHwv4PUIjDwQzl2YQcJemIRg/WHgL+h/jAcDsHS/SoXvkrHZVwJz5kn4yMSYLnKJuVau5fb/d7c7lcypknCn3uXw8QJDJl1ANqvw6BvecTrEwasJo7XOTPWwYobLailXcR7XsSgiGNV2n6yTekINWJiRQpMcg+bHFV/BRh7ae59h67z31KTpKdj+afsM6Sz8r/mwS99zyOZ04rev4KzGpA6TmVk1R94eS+hzGccYnclCWe+ROQqx2limc0jsehz4ma8xbXQoXjoFVB8h4mZ14jzXMG8UeWc0h7k1B43LM09FI4qpCJYj9KqIsb9CquTX+Rg8UvkhS1Fbi8lr/kCeVh5Q+1Pu8b2gk3wXE6+Yhm1iUd5OXwWPwgpZKfVIMs+Tbd6GgD3pf9g72WmKvksc0eWYFB4Iwt/EpReKIVWmB5JyOgo7Co24isDu7583Fv/oTcjnAL/TvwaG9j0xytMm3iI8Bl9kPgRA71yqhNnIpdD6Pl9yNu84M09D8eqQefBl7E3WVnbiXdNJHs6DTw/YzRvXziFv2MhifZz0BTMp9n5EF+88h4Jshk094/lg8uHEAQLePwKQEtAHfWI8VMEYewaB3ZnCbBmorL0MOPcOLqdu/FavYY5vqWgywaFh20tGFDjlRvLvOvT+KpRySM/HCPUPpsCix9fvXyAGR4BDDmSzPBmH3Y+spNykxjmtnHg+1GElDWiQEyb1Z1+OzW+9n8xyu8EG11hRH8IuxZXE9I5gpe+eIn8uV2MCzTSLwKwQtptbnMbt/2DccaRUnzZeioPAJ+wDkIrivBs8WTYmnKSfC9gtW7gyGIHjpWuQxC2UCCbh3T4SUr5DI3HEjLdwmH06ofjKtJrkeqMTL4XirMmAq/llaBeRHa/A3P61eh7tMw6PIv7CfcRBIFf733Fr/e+YmjPNOyyB2MMayTl+s9E+NXh6JdGVbMXp2vm8muKhSwdtLUZKeoOYYHYDaW0Cj+HIgxdwYz+7DmafLpx6JHR0OeI3DoAAhw2dFLRMhz/TSmEDb3NUWU78S4ezHG6T4MJvu6GXQp31pyMJe1WGrq0O0wMaQOLGR4UwOXWfjY4w32zjoa6AOwJgAegxfWh1yEpAZf7GpadWkbFFC197m7Yy9oZ7pbHjl/nccdOTlJ0E6Nd/8J1oA6LvR8Amf6FhC4vI0+eSKBjG/Ee5wHB5tHZcJST9wtwOvkX4tm51EWUs/iBXO6b7fBDD/j0qEnyrmXKh3GEdhf/OxGIpBg9prMyaj2Dvr7Oiwdn8cVfj7Jgw1//bmPW4WtfQ6/eDXOjmbDmMMTRIpTtZzkaOcAY9TnsIgOwNMkw/6czWxBDyAp2XD7JPfM8HFs28/fK7WwL0jG+1Re/EROIdM3in24HIj/5irKT7zP+bBCTkvOJdLNZCxwqW8tVj7s4Sw0YgzLZPeMz9HI9qUIXH48diZdLDn0lwTzW6sDOFz/j5QCwIOPdi4e52zSJo+/A8t/AoQsOzE4m0qGNNbJwHAxl0HYZ6aUZPJW6hK0dX5IcdRl7Jxv5kbg32bzvJeqpxsXFzPZ3PqIuGCiVQcMRBJ+pD2QMFUzwUPNFTD37LQtp6S/A066aCLfrfDVnB1KjlDcUYTyVtBJpu5FZUTB9txVd6mecTVqPT2M4TxxeSr7qPvGSBubIbZYei3JkPPvts2SlZ1ERX8H5RAljXIpwBZ6M2EnIgJV1W5+kVSOnc9Eh5oe2cfH+v7drzEAuMR+/Tr/YRLlGRUOUlvjeYib11vHCly9TP1jGgPISbpdDCI/6mu9rvqdD64eXopmQgEDErklUFf+Nq10v0Y5x7GkaxM/qbKYZQsDeHaNSxhfd4GZ2ZVvQGUbJIL4G3JJrKA29gVWtZV+bglY7N0TNjxHTvAoBAWnIO5RMbCXg8mTE1jws6vGoi9wwS9v5JcTEeGUhRrtQqmQG7v+n8Sj8KXCKA+CqewutE+/h8l92I192wVinBi6t3sbq1Dl01PfQTDMIVrolbbSaYY0jDFJAfXEOf36yEtGULN5aaiOJWMRakgy3KS0JQ5whQWIV80NdGjVB7pytriC2diFl4UdwEjvh90cUxVI9ljm9PEEhkwOgRn+VweurebJJxAGzDn+JM3d9XuamCd42D8LZEMlzeWuZHe3HxbJ2RlweQffgHsh6FMLXMdizAXWUEY2DExf6t5I0+C1iHlShj7aEIVibsJgtTPhrJd989CUASXIwIKYtV419wxUUngs5P/o8Vb51bJWtRF+3kc2uUtpvZtAe3YNiqgv13WJye/2YAKBrxbnmd86NrGFccBuehWosFlid8tIDG7lHMMqUhEwMQUCg5YeDeHR5IwgZtilDEOHZ7ElEpSsanzZCy0Mp8BSoGdJCUBBYuqupkBYjVa1CXzVA+q10akJvoon+lsUHrSTscEQda8EyS8fQKzFYIwwcHL8EU+R7LD0zlJv+JxjipWLakbG4+/TA8pM86R8Nh/xZLfsUT9VsZD5P47OmEkVCFaw9BGYdUk0hIGARWbCIrKRX2LE+LAxlxwlgOWapN77lAlqTGFFUE7Mcy9gc8BVPe9lD7d94VW1lUu947pYFEl4uJs3paQLH2kBGsSDmzqA7VMaIGXVpNY49jv+S4oHV095hmv4QTy1IoOI/8vU+08DQjtZzDtsurKbDAqpRseQ1j2C0P1BnUzTwbvqSpJwkUu6lYHjkOGvDmiiyTgKTBI5FMNkawJkP1xAe1MIPa36gJFbz8Lgrk19mICcR7R+Pcm7cbVbNuMd7rr9CLRC/ERxjMWkC2HNgOJElkWQ9u5U+934SZP/WI7N1Jj7MSkdkKeG5LT8TYwShv4L9A2ak5ysJ6N3G4dNhBO1oZ3x8AHlerrDkLolvriFX+hOrnWRYRFY8pBY6BS885UD+B/S6TCFvwl1iYkYwUOj7wGZKBOm/gusgFviMRZD18Kr3LO4O3oWTMIQXJ3xlI106RLE/YwqqhLuILPbMMNkDD+bnsh8g+XMMCgv3E+5TH6HjdtwJ4jsXkbv4JZDYpLBEjg40RYzEqf0c7+xfhGVZLYIAN2OeoNzTkc9TFhG98TrDa1Yw5fcPwb0M2IpsrIhXEu7z7KhnGV/dzGCZmcP3eujaMg8vx2bWjLCdhtTFgaqkWai6GzEoHSmILaDBt4ELoy/wePcnmFX2iM0Gin+5RiaA2A/N0Eqacx1ROLSitfPg//V4iHf8L8T/gX3/F/9rIRaLiQp0hOo/YPQJTuVkYeh7jCqVC7XhbWjccvhvZVuFuoKQ6VWcdhCzK/wcYouCtxV+7C18HYCemmiqdwQhZMLuvLdxCY7hd7ebLHGtY2WThENBItrX/MDMo0uJKI0l8PVgJH0gBqxWC87aTl776kVKIis4NzwXR6sLOMbCsL9B5sw3knCq5u7BTpOCtOIiVqUG7jwPgIvyESY4vkPFvvd55M4jfP7y57zdp+J0wfsEa3WE31RglLtw9rmvkFvs+C3yPVZd+4QrWljc8AzKoiCKk5/idtorSPwcKdQvxa0KXsx4kZTNLQTWBJLaYMTTMJygMQZG2YPFCoJhCG2iaAbUNaTcOcyk6m5OLtuIuvxtvrn1Cx8ugbjBP+AWqKZ2/ygGZS1ky0QRv0XUgtIPwdDOGO122po8kd/RMdggx3VFCvhO42KtG77FGlx6Xanzr8Ms6JGKjLir63l/9AQO70/Fr94DzUQr0TKAXghcDLlvomrYzfqcDPrKA7gkk3HH8gKTf7UgFQ/gLgZ3qZ669Gy6nJW8nLUBR6e38M68wjCpjcW1VnuSf97+ifRbU5m/bz4W53OsL/4Sg+oOAyYzIkGBXqHHIDejMYo50Ctitnzww2fFznAXS2k3ct8SEsOKaa13xz05kc6aVvbN+RW/iVN4T2xDrDzFsMsLuPUECaLx3O+JwKyysOJqIDFtsawIn01qqII0NRzo6qPXAphAe3wkpfpbhCeVE+F6mw/HjCHjxmiyU7PR2zsg0MgLrq286gKvusDePvgz+Hluz+6j0+Mf/js8xGAyazkxEEiGXoprp5zuhsHoZ2Rh59QPxZsh6jm+uV1CayuEJB2hc+lP3HMbIGpuB29sPU1tQQ/DR2dyt/sWnnjS4tVC7nN1SMVSlmel0uxXhN5OoGXt77R7arkuK2Ww73jcVPWcGHgKtbsTZar9lHtfZ492KnEBiXAkGA/HSfzSA9k6GK7S8FcA5D74ODFYJDiKfNm+6iPUukhG5xejVkOA8h+OdIHDQCoWrHTIT2M1m6g52IzfrRpCR2+DIV9A9EuQv4F9c/fhJJ2ExqOfOtlt4o1+XKu9Rq3TNZy6o+nuG8LNiIkAVE5+FXYJJNrbshArFjoS9Lh6nGKIVwl3wmv5+M4d7GN+YfUHP/JmWzjrfn2ES2KI+/w7wCZ/Gi9qQ1uYQa5bEWvH/I2LIYnDA1MZrj7OjIpgIiVWhpfFUOnWybbHthEtnorWWs8eb/hdWs9FQYRVZMVqtWJ5oFGQaroN+16GMWcoR0a9extmswR0VtpDQMh7nkOG9ygPqsTTLx23GmemnJjCbI9OEpJ62SxIcRU/MOrCiv3cAJaFfPzwOVlsD1s7JFyxbyPdamHz3mG8mtiIa0wT7/TCWQ38ygD60LtE+/7NjSXnUNkPMH//AEaLAjs7ON/+DG3GQJIClqFtgDA7QN9pkytWBfC8TMele4m0t3cTWBNIRWgFUoltJlZYW1i+5BzHJ13FU7EEn47FNLruxlMMF7U2Y3qLBVIeS6FOEkL8Fwex0k5w19dUh1qJLTci9lAyZ90+vAJaIOED8JtJR1UYSfu3YqfVse+lOyT9p6u12SajlS2dRMIv/VSHdZAfOYNPK8J5f/UAzj8Phqgw1ume5aenvmHZ2cNkBWZhJ3igFlrZfHM7Tf1hbOraycmaFM6fT0VV48v1Ide5/eqHDPazeRm5Cz3c3LqImCATOxdfRC9rQiSIEDxGkp0dw/zToxGtT+Rc9GRGpRlhxAq4MptU7SnEi1rY0z6VyceTEJpjcFmzHof9I3D1+JYDrUEkuXTQL3XAvfo2uLvj4BZEgGMJ82M+5t3uicxyHeDg+TAyj0zj/vIupj79DN35z5Dj/DpTWn8hWeNBz18v4NJmJvGFIaxr8WJjXxuPSceTLi4kJ+M6d4z9mOq96NCV8jo7Hz4vsgGBJTuX0ODbgOtbWxjnkgS+X6AS3kC6bQHJXhrOLrR5tIwO/Rix9EcIewKhLJXxBy5AazhWzzi2xXTB9WXMiFVB+U+EpUtoGWTi3v0x/NUZwy25lMebnqDP8SfON8YQ69tImVlC8FR30r2Pg8M3pHif5OnBT3BJ/x3rnaHWBIN2zudaZAOOLz5GoucnBDnmAWtZOeIyi6ZV4FEJ3XonRB0vES1cAKsVwWsyW7vhpg6U8dEEOlTQ3B+Cj30FXoHNVI1vw9PPiaQ3F7Ih/TZzXlxKT48GlaSPCyYjYXIt0ogwfJ6wokzt4+gQG+K4aF8nuuQwdF4JfNDqxqvdHfwwYhER0j941AHajd0cX+CLvYOFPJc8xBYlDopQQqU2tu3Hd4O58+VTTHDpwbnTkU6v07yfvIV0BdSZOhEQcOl0QWaQsGO5rfNaZAiHljMw+HuMIju+eOZbXHsyGev2DkL07Yf3cc6xDXj0V6NaH0xB+jAafL/CXW4PdQcYL7nA4Ddvs6h2LKLoFhY69D383ZYe+N6rnkdcYFKrNwbVD0S6iSDwc5tsTslmVjiFIUjg81bbevhl30wSBhKZFPYeTqpuPj4xE8fTvnT5tZGWWPJfCZkXKD0RRAI9jj0Muh2OT/0T3Hyq2sZ2fSDj+WPuHlprtAxEfsRsWQPT+5/nyFObWPV5JrnL/2CE5zBcbjngaPGxsZID5oP3RO6+PJnx9+JRhenw2DWXqDG3sSydxok7a3CQt/PToCxMUbfwrInjUtZ0Js+7CMdjQSSnd3ITL+z4EoNeRUTwcQLlFuq91xCS+D4A4SaYv2cGBqWakpQqlofOxiT48tTx62hNDhx9FcLsNtIUWoo+0J9PZ4/F/ryNcc/wPXSow/m+B9zNEgTVR4S41mG81YO0v5wnXeCpDjCgJyqhApO9GK37fOx2CYQrI4gqisKr2Yvase0opu5ipOeD8QxZQY5ezq3ddsj1dpyYdBKTpIdl2tGQNQmiXuKCMpLKeXsxOfgTH1eIxEEPoY9B9CuAQEFZORcfKadXlUOd23YynRexxj4Ukj8HIOzWjyz+5DCLr42i59JSeuTzSU8dALONeFNHOC31zuhcetDNPIV/uI1oUhF0hn7XZsTObhhu5XDxXCIe6/uJSpdARzb9doOpj9KgjvCj4dANGq0i4sUiiFkPAQtQXTjJ+l/WU5zRTW56BJ4hGvCbCeogUAfx5sFMvLLDGDLxOht8HbjhVsm0kdOJcb/GgapPQemFo5MjWt8PGWFnxir5BEadgP5KkDoicRGTn/kkJQHfktcLLfJB/wPs63bto8njPlOOT+GjG1omvjSRkfJ2rkt+51ziFlb23aPXzRccROi9FyFv+gsMXSwKWIZSGMs93SmmfJZNkm8OhL4N+e8x0fN1dNYIcnKTOTH8DnN7SnjRw4svx2Xabqf/Nb4RH8de8GLSpc8xyVQ4ia04RD+Kg6wb5/5RfDjmKJ+VzSDbCt1WKbJsm1GLU/oLAKwc9j75g/V83wYndFA4vIBodxvT2kO2jTUTXuOLrueZkxPHa3H1eFd8hcU+kjeT7flI/wE17beR5aq5HO5HOCIIWkL5je/4Y4GIDfXBmH6bgne9O514UFqfwdHS57guus8e3UW0On/W/DAYZaQU1Qtwr3kC95on8EjSdtwulpIbZk9SBDiK2qAfbjbMBOBRxVc43Z3GtdTb/Dzb5nX8WDI0mODz+mrsLGZWv7CX7zVG9E5N2InFNhnEB+HfY2L8vvn0OopoC0giz/k1QhfIeGbTCnrtfqeXVhI7h6JtsvkBJngmAHC4K53Ur1U4hZm4Xe6OdOpQ5OtfYV//DooMkOLTRa9fF+W3BiN37yLcroPrCMgEW2fXl+6QpYO9U/6mR5mLj2bIw3MSEBj0/E6K9bDvXi5X9C/ytNhGjpoW8R1aYQuTKvW0bHuL7qm1fPxMPDUDnwG2rj6TxYRv9ho8SwJpWnSXVU5dtJrGAtDlquXrF77mu4zj7Ox9nz77vofdGQCvLD/DnTnnuVpwnAGNPRJLHzMC34BAOFc7h+kDrpiaVRzP24C/xkhx40RyW1az7GAr7UGbcRlyHK0ikFT/RUxw6aReWoW9Lo8yA8js1dyefYCA2hBe/vJlbj3ewOJkCwESOKOxMfEDagMIrA0kfckAk+IbwGqGB1/mSmsvo05ORF3jiblwCx4Txfw6J4DFDdHs4RQRSHEvM+Hb4Eudvhp7WTsAO2smIilyQ+yjobJcQUPrcl7/9UOcFC38Pc+Bc53BZMbncrA8mPz7w7HXNpB2aRpom2H4XvTn/MjPfIyy8E9RCjaCXodRwYddOuw1cVSEN5AXf4n9BfsZLNFw0e/BYM6oBJ0Df1Uu44hDDnHKZHrqOrjdNJfpG2x5k9B5h/eSJrPj2CpabsuJqXuEhG/2YnZby5pGB6oGFFgFC5X6rdQ5nuNQ9TTi4lZAxnY+/34Ic/dOYmBOKxLP2YwPsTmAGs1ySjvTeKldSp38BOOj1BhNc5kYsZ/9RcH8lmPz4VWpVqFXwZ3AObQ5XQdALbLg51BEryWdnV8MIcnJnwtx4FsFZ0f0UtyuQywYASli3QBSnYWGwAYmu7XhYMAmO+05hr/2n6P0wwjMg/fzm/0wrt3+gx1P3Ycrc6npSEHSZEaIxAb0ATQcsf3bcZOEBe2UGM+Q3PgPV6sG8IqexhN7PkUt7cYg7mRC+reUGaHzgoTjrYuZNdlGkMrwPYST/TXeL0rhQ7OBlZ9soVAi4YBxNrl9V2k1wyD3ShSTSgmodcPll6WY3jz6H7tKbnTE4iIPojfhLGUmEUtH5zBJDYFsIMrtOrWcxl6qRYioYZ9/ORcScohTTOPVqJeQ5x0gbkwhPfJ3qfT6kaXO2YiURjq0fvw4LRwfe5vk0x+Ve/hw/FUMknbGi17FUWxEL0Cg62cs8lnB5toRrP3layzRnpgCv6HcAIUGuBNYiYt4gBW/jqHoSiaN79njrGhi7aB13GqYzqik+zilaXgqOIl4vNBoG3hWEcqnIe1ctuTgqmph6idnmbJtOgs/epWzr+5l3KzXHr73csGKQgCnshCG1RtpCazjhh8UGHR8rq9ltbMCodoGRpkFHQf64UA/nBsIRnw3imNxArnxuVQqrIh6HkFq/pNpaqiIn40iRUL7xIvEpG3hSScwjG9DovWi2F5Hw9D/H3t/FV3V1f1/wJ/jJ0fi7u4kISFAggV3d9cWChQqQJ0aFaq01FtooVC0FPfiDkmIkhB3lxM7fs57cXhon6v35n3H7+L/zDEykOy99lprr71kzu/8frtp8XLig0N/86VrN06VZeTiRKykjUaMHDYaKJYAZrAgwFXWwXgZbK58wBj7PHwyrvH3m/4Y+7oz/NJw9vY6A4OPgTKQfrGTmDHoIu/Ikwh8zwyFr8KOj20NHv8jX6bI+CziHVoKQSe17T3edJUhknrwMGQFrolG1F5Wsvq24yhUkV3wDud9wGo14rPhc/p07cI05Dy7VD8QWbOZ5QMKcKr7BJfirbhIIU1hQlB+FTLv8or4JSYPtoHDd3nbs+O9LkrWldB1Kxu5dwtaIgAnhAIh4Y/DGX4plO7WUhY+WMiOFSefZilJEr9h34fgA6hWHmaj5SVklniEQnjc5U3+rJ0MjxuFU5ue1Fux5Ag6YGQx4oqdCIUfMe3Paai8HbGvs+dGt4qEbh+8lDXQU8WQwL14Gx9yqXENzc6tNl8QwKNPcBF9S5LjWrIWHKatexyzv4wn45E9KRNtlwhFYhxzo3j20CzOTMjmbh+wiOQQugJyNgOQN+4CHydmMu/4WaxCIZOdbAwo/1l/rJgxuj3DwVWTqFXW4lXhzg6XhRSbPud672WI1FVci+6NwCpEp12MovY0wsDnuf/tceZVlZC5+ThXo7+gRz+Ol1z6QfNtOpxGM3jsFu4l5nHTuxMn10bG6a7CoxJAQKboXUpbaxF4BVDvdZSrEgUTY96gtqQGb35jsH8ZhsUnOaiT82yrL3pFIzI7A/R6GwBVaSfejUNwbRVTV/yIvHIJ6nXKp99Vs6GOqfsDaHLTMG+47RzK3eX8HH+NsZUxmOsnEl1vxPtWX56NasDORwrXZ5IkTkVXsZKQkF60jhrHB/F9bNSULbY5coRbHgeliwjx0nOr7nskMgHWGx8j6KkAhxhu3H6GlpPtuMwzsCcW3MS34XwKzDEAQrROGrodNSzeMZldGiXvfG8LduNjA369G36P9rBaXqi2gQGEiMClz9N2Jdr/iNDXlaLrMoSPwkmVnEVobGaJs5l7qnoaDVIq4vTURV7j3zZDZeGhDsxmMyOtTkyXdfCiXQlXZj1gbMjYp9dZrWDfWIxLbS5dzv7I9f5cHHERo9SI1QoWiYyMsa/z/LgSbj5/gNtHh3DCrGFixp887uWLVrWc/5dNJBIRGRn5//3C/z/Z/4J9/7P/M7MYe+gpOoIqdy0kfUWnLJaVhWocdO4EeGgZ9US769eHWzlbvJLHTrt5mLQOd9N/xOZtq+3I4B2EOj9gm0jE+aU7cdKEc/CsL6KhQvo4DAOXXWxwsGN6l4V6RT2KyTrWBv9Bl3Q7fargjDcUGZcQKFDS7m6PJCOeuPu9iVndCunPQ9gaCJhFkcGZTDMMr65lbf3n9O11H4IWcrt6GvtzUklJtVIXIyDd5xgKOz3DZCLQ1tGsP4WqPZryphZ2e4IdPbipvLirszksp3TXoq0LJX3AG2QHHOdxx3Jc7SXYC8FHrmDTM+c42CJDcXgdruYOQqRixj6OZo1PPmGSXD4Z9An3KwdwdeckMno/pHXNQtafWPi0n983u6JxfwjDZMQKLfQTRkE/G5JXWJ+FZ2smHYURWKN64WotZWybAC5Cu186D8b3JmUcCK8JsAqF3O9eTFmpjbJtUfMluu/14U7vdtZURbI1uRpV9puQ+AUmeQLJDd+SmxtNg2cLYiM41H7It9LhzA3+jQttVYxcd5MFj73o/24MWQn5pEx9l5ryjRQZYYSkiInusDWgntTVB3ELbWaEOoUUhZmtreCmO88VaQT7ylL4ftVvGMSdTJRsf9rmEx1N3HjlF9p7hqD7LJacwFd4a8NaRFV7MIvNmCRm0BhJTE+k3b+azxwb2RA5lILWZL5d+yH+5mFMOelLR4s9AH2Vtky5HaV9cDviiijYzI7nvueIQoK33V56a+dzvnQ5b8UfIqF/A33zU1jifRv7J4CiY11wqhvCzVOwtELqVWfe35XLW5+/CsBpH9in3YtZZWX7uu2M0H6CuWAQDm0iXvB6CzJegMj12FUXEX/lGBa3SHa4+XPHOIS5Mmc8zVMZ9udvSDrr6J3pxOiKFXT/4YBEJAFtHc87NnK6W4zZauZnzzJO48wPtLE/dzPtOg+OeW6lJ7CHtDYhH2g0tLU2QMdzYNYhdehNzjv9CerqQhtSyx5BM92r9vB23hLUFR/SO+0kxQGwtbaDjrYqJFYVYqGYKCn4CdoxGTqYovkYzek0qpK6kTu2IRRaocXmzI4RlGMckMvd6uFYouqY4nybJsFYzpWcw77qKmmXtXTMDCe8MByJk4aALhus07nzOktuD8JU70XdYiNG32zkCsCsZ/PYV/hEuwSviu34oGJwbD6BdjaH0sXSJXx191fmjuzNvL3vEFf0j6bA+R53jvVAk/wRn2jsKdw/i0tjblLrkYdK6EaJCWbWgabDmxccz7PJGMDbFVb6XAvgbmI8rb29IGQF2PnwYn0YTZ25LHEx0OdeIg3GeoKFV4nvWcCwkDZy299mYO9n2TvuEvc9akjq7AdXx/OqJQSH60NpizXzs8dLrPdsxYKInRozx7qhTCtGqVMis4jwOj2SPV2ZvBlTxkuOMLQGZAixAFpZDznyToJwZFDAAWZGf8i1muXsz3gNrBaS3a2AwHZoKd0JmRth1B1WP4wm8dexqJ8z8OEbNg0rWcst6JAitnoy0aubR/JuBBcaWLo7gV+WX2JOSAN/3Yhi3rFBmBJrAW8M7T3E5obg6V/GCtVjDhd6oKh2g14SZvvk4mN14kLgPBCrkYtkdIcNRqqGoIclvPrXNPauO4VcoYcBB2h+KOTU+I/BzpesCEdEAisrjBdxENroEu9r28loCyDXafeTlUHAhhAvCIGf0reBwIWaYh8UFT5cGnYbWUQ+HnVH4PEmiNvC94KZ3JtyFIV5OE/gnii6HiNov0pXaDlH7Y4yx30K6rxoJjmsg+vbYWotVaUabt67itZ1LP5JV4kVX8WhM5ITAzfipS5lRdZSti/5leDGKSz67jRhQ7IQiY4wYFwUwubBjNX6EalxI1ufiCWsjAAHEx69V0DvFaQB8yYU8uBUGMIoA8pge8TGejoVtmyDfMsJIkRDeL8ViMmmanw2bRbX/1pj40R2CMtCMCp8WB1xDbgMOe9RIDlOZWwRIQm+1N/cRv/a+bAVEMpBKCXU8RzuozPIcopk+v4ZpDdkkjTNB/xng9doPv1BR0WjAc+GfA6fUGJ+Jp2DHpfo6AI7h8tcXgS7Om5z4qKV7AL4dBz4qD9FJu7BoFMz6tYAitoU1GVGcc4kZJrcnzadOzUdkYQJIE/nzB/CEvRWMIrbEVq6oS0TdPUglPD9371wbXZlcEI7rw5axRe3d3GragZSkZZuuQPSwEYaFhxmckArZqOeg3lvAtCY+Aql0TomSDxwUl1mkPTFf/rK4zLb7+1AKIThVaeQ9fRGNFhIqARed4Y3en4kR3IdR0MMbzrDOIWelCYZqEFdDBFWF4Kj86j372Jn/GO8g+o4ZIAMHaxugkxdFRPXHuZQu5oUAYRIwKIKAUwgECDEgr0QxCIzq8e8zJDAfcS0y6k3SXCkHKGmDbGLPY1+PnQ4dJAqtYfr01EIYhleDZ0GJXlOPTSoK3jPfiY++pvcS+3P79fEvPHFWOwTBWjtg7G3AL3etR0iJY4M+WoxbffGMLRHwjijBNmsHpKWvMOp48PpO8WVPx/PwGOcivA+j5gc0fXP4Hqi+fee51Xa3qpkzw8LSKp2ZpnXA+hqtQWhAHltKfEX92G0Csh2FiJAQEtLCw91LuRZYIDVwsizqbg2OiJcLgCRDIQuLDSC4dYAqtxMXJ2/nyuOJi4lwZ5HP2AfDNfOO2FQtzM+L42CGyEMHXsXe+dO8J9FXU8T8uwCXLvUfLzuJkZxG+n2a6DiAIhVyNtl+Fizadc7UK5cj7EVnNzzGBH8KycerwezgQVD3yA9eAC3Oltwd679p91t2QgE4cz9Yy5SkxuB5XqiEtuQDDwAMleWZzkQUGXC5KJjkzOAGU2P7ZsVG1vZUDWQ6lsJtIxS8bXHH/+Ue3sRVqcjnBx7BrXRD8LKCJC3g0gOgfOhPRsHVQR5sXnY6y2s7qjEU5NA3YyXoPE6+Iynl3wcFY1Q6vEloRIIF7dB8c9gHwXuAymxS+KL9oMYKp0Iyj6Oqc8s6P/r0yoEef3CiSFQ5/gn6aE/c9SSwqOZRXQcTmHOR494tvBFZpfOQ6tT81C6g8jBHrBPgrdoOb9PayRZOZ0bz/4IwFLr2zYPgXMio72+Rj24gXP+3ch6fUyXKRUcbtooRJtuUq+V46GVo7iQym/V3oh+daCPKB2AnJYJfNfdm84uC/fkZl5qgUdCMTjF236AAPkqlkzI5617ocTkxiBK+zdMEF5zq0QphryCSJTOHfQzXoKoxYw//yWWH1ehMVbz64J3UKgK2W7DPmAIXcP6XafQVHYTGNfJ2bol9PU5adNU6v8biOy43/95WqMfEHVzCOMfvwhfWunu9REllY6YE1xYp5rFRY0PYXlKPIIaac+ysDfBtkZV6nXcPuhBQEUHL0aOYNmwR/9UWGnTmvMSm6n/l9aJuukiXIyB1L3k6py5/diXwtYcEk8MJ6cjk+DYMoRSR3yVpagsTpgDNLzx6xaEIostk5dA9AIZx46nYO1WIVrUw5agbwA4VbSW7IZhKASDSP65A0NIGN6DM0iJO4vEsoDn+75LbuMQXKRFtMg0KHVy1pXG0OnfgMbpZ5aP+ZatNw/i1S6i+lof2vyLgYandfcRwyiHi3SI32GJshKHq8Po9cUKBq05Stliz6fXbQnNoWZVBR+VbUVl1weJ3LYe95fWEdPsT5PUjHHSdA51XAJgUuBgqDzENLdRnE7upFeiO81Bb3HFv4K+Ahl5BsgzQKZfF2ohjAtp5YOJZzjVFUhlQwTHQirJ0Uox3O5FZ54fxwaVIBUAWNgybBghTplsLz7MMy4mzvfATxo/eswyrDIJe3PeQSVpZxRwT+yEc2QR9XJX0huGIHf5p00Ao/rn8cCrgavdX7LZBF9EBUHpb7TFPmJiLRitRorCbRRrQoGQQMcs3BSVLGiS0UgnfiEPCNqwlyT/yqdlvjd4Kqt2DSY2K56EwDqUbWrmtjzHGdlk2nReTPe8wYbVF5iWP4Q5Tu+R5HWO29JFGDtyuOIL73f7khNSCoomFjvfJcinmcMPT7DF6W1SKt5AaO4hccZF7o+7QnnBTe5rX+atuf+4jfRiH+7f6IVJaCZx/DUGJBUhlKgZKGsjXQIWg4XugWq2hX1Mf9kY7tXPwleZwfWm3lx8eT3BljEkPYyDVjseeB+kT+0sJCIj55p7Makjm1CtibjDHRSlqukbY9PnwmrGHDoYt9gMBh8YTfL9ZNq+/pKiruE8U/AZRoucpl4pVIvAmhOKfY8U44xMGhiJLwIEFi1tRVvo1nXw4hcyXn5TQUEOTPxPo+Re/JTnh/m6HU39m+mVdot6VKjdk2g6W8W0rGPoFj8i8HoN/UqjKZ/wDwCm1V3D4emHGREykF45X7DLJY5Ryen08/2BnZmfIBZcxbnLGQvQ4+hDuWodex/a7vVSFbE6+Tkuli7jpuUK3jXe1HnVcVPjy8K/LjJyJBRHPURrZyEmN4YuVRfDw+QMnwHpdaOhYzvB14+h1whQj9TytTtoJd7Y9fsFnJP4SeqIa0wujW6B3A1fy3i1O+Q1AtD5IJTl5yaRs6qVW3VhhPu04erdDMMugtXCZ5dHcbATbv2t4+/sXsT8nIDFKqLT4IJS2cguBxk/tIooSX/AJdzwWnGCfjUTeWPwVH6vG8m5HZNw7Z3HvqRcWgyBtFrV/Nxma/eRwFqmLsxh1M0YouMbKHKN4+e/X6bQ/hheZd8SvRw+d0mjR1rG0obzOFvdKIreCkA1Bk7KU9kz/X28WmfQp+xL1g64CmcTaYz5i93KCfj6iWnx0/OC9RJDzM+zLGED3upiblSP5+8PEjHal8KSJ9OusZqKXrbz5fdFLfz95lp8ejXgXNeGT1A9+3RjOd1ym7jWRfylvEdPTzWNHg2kJ6bjqRjK5iETCHXOwGIVs7c1iS5jFYlR37DFFTIafUnsGUFhsxWlRwInLO/z0rWbBJsckbjXE6402Ohgn9jv+m4en51LbVYE3mGVLHt7J/d0cL4bBBYBZrOZ2dvT0CjjkXz0j/jvz/dGEnnVG5e+ar6dvg0HbTxzlEWEiGGNBfoUFiK06knQt/JxG1woW8ihvqcwI6NGE89Il1/BZyaXe59khKvN7zGkpI1WC6xKHMX+ggN4NHgwsygCn6RSXuicwqDgnbRboFITjWNCMlMXRXGqo4UvPd/h+1AjmHXgFMcXNaPZUHue5X1c6HHvpLmnEo9SmKmC+zobcBeg+moJHtUWyjwgy30adzotHE8NILL7OdyVHzO8KIu+Hgd5Jdam/y4QgNKpE6NJaNPHAyyIkWhu2c62wFtn4lBcS0ZizMapTsudKcdYaTUCY/Cr9GX+7zMoUBRwcksOmdr3maszAs/hrnSnJUbKRXknLw5bxEVhKwPqVmHXfg5cwhFIg/Ap+Bu38vuYVgfSpe5C0gnS7oeMd89mp7wEbbAOHqvY8cxR2uyf6Ap1l7M15muuH06hxwJZ/T8nzPk+Xsod3O54lZQlL/Hzsm68D+5BGXOK7xfvYprPP+uJwWkEn8d/y0MTLO+2cnZMAS0OD/jU9VOSAQEmYplGY6SAsSfjiK2x4jOkFNqyQawG6vi1w0ZP2hyQxL9NJBQRUB5ARHEoYrtu9MJWABaEvs2t+kF83LgN+x45HfUaUq96UulfidltMIStRth2H5llEEKrFGNVM8GVQrqjamCUDbygz77GyzGNbPZs5HizK2l2EN29A7oBgRCp8xIKhoB91G0+VsjQ6KSga8Tb3x7i2njhyzkUhJxjyJl32XHAinpdFC99kGyj/3fpg4OpkfXz87lRPR/ZIxFmswSh8B/+SIW1E8Pgu3QKxfS/GMwwbw3fymwBsLQBefyUNxq7JDssWz4lzNOIvZ0BqmCDl4mA9+NRP+5ENsT2nbaq5+IcGAk5bzNMvZK0cc38ZTjKlN8beBT1CMv6ZxBlvwmaPH71uoRkbCe+na5kHhjFwEk3UKh7oPkOAtdUuv3FnKmP5dOIKgR02ypr52MDtDVcJVrijabLk765I4nUuuI75l/ASyBS+x6FF8bR7t2XhrkpaJxWINJV8qOHkder5VgMemoj9JQr7vJnfRzTPbPJNG9hd8nXlDzyoK1PG0UvreXs4khcFzygudcjBJ7Dn5ZvbGpHbOihInYc8u4WQgpjyY/OR93pDjKwK8om7vZZpLNnURylJPRRN6vcjKSuO8jy6iW4Gvh/2iwWC21tbTg5OSEUCv+/3/D/Y/tfsO9/9n9nOZtRFTxBL/RU42YQMfPQTJxb3bCE1VPbmcyDrS1cLFlGj9EBk9U2XKVaER+dXMSAvgVIevJZ189GlfJ5wzIqAypxzfAh4WEgD+LqyG38jVcqlmM0y1kat4wQsYyTLQpqiEcpdkMM+Eug2tRBqV0C5Ws+pOLOXObXiEkKvA99f7Qh2K0WlBoR46+Ow1NrR0ZuH8J7P8YjeBmaxhh6n/0QU1EA7TOsZAZnMlkJr6o7wH0S1pYO3vvtAyaVJ7HvWAorIpqwe/jy025YHlaLZtNWEsqmMupOItIkLR/63uZQCBzrKeCGYzP5UuhIWIa7qpxh6jAcxEZGK6C5q429Xy+lPs+R3xf8jr9zD3LjYwb6PUQsMgALKDMqqDGDX0A1L0a0Arm21PCwVTSg4o/RGdQNekywQwIHIzc+rZdJoAaBAENrF2988CqZ/e7y6rq+nNctY/u9Hfw86xMqRjVS+mAy7kfn8J70IZ/M6gSPoYjFUdwOl9Pm78WhkNFMdXlMTAfkGV7n1OPVZMhFnDbdJMcoRJyUgV1gMzGWi/jLYXkD/OjWQHBFCF+YBSzzzydQ35c4QSMpCnARgUpooeVcL7yuBiLfKMUgxqaB8cTakXFa0ISPqIPTY07T7qThef0S5FYdHm1OuLUoMNlpmHRiEldGn2NDaiMog1C06egIgZdrWvljrIZmaTqxqiX06/iN+fXQ3JFEbIOOds9W5kQ2ECaWI7M2YLKIuVy2kN5uN6jRaVGYHfnMDc73BDCmqgar0ARWAVmOw6l2cOFiz2SMJgnNTs9h1/wTd/RmXN1H8anLEtos8GOlAldBDMF2Nn0u+v4Eph4SA76l3NUFhcyPvsUnmZ92Ek5GkyBfTL1ZikVvpDy2g0eBufQVTAFNAZyKYmplFMZtK6iYbyImBh4GtII+gDFpsOH8bartzmBSduGYOZeiq0msGbAd/j4Edj4YA1/AoLoA0k7kHb7oZRdJlcPKxN/YKYjGJIig9nE0joXJ+NzcSV2EPyvGT2DsgVQGuN7GbDEyPqaGYz13KOzrR7bqFxZIgunfWQiN1xlqSWeVG/SusjJMVWHLgrRkUCLpR5VfFdmJGvraSZi3bx4NiQ8RDjgKSV+Rp7FjeN01igv8qbW2sr8TvnR7MgDqzhJnF4BDuwNyi4QDA68itjewCrhVNQ2AH5tSebHjIfZN9rxxeR5zhmURG/MrLzfBMDvIFUmwX3aIrjYnBl8dTE6vYwhc4HAXhIntaUHB/Q+GE+CuR9IhQtWlokYaBf1sGSAOrSqWfLWR7sF1jLs2mkudNmdcTLOB65s34JbSjbi/kg5fIz3xWsCGVNe2KvD4ewgyx3oK5FK6OxQo7Xv4st2GKN3VIaXs643UTBSTmTYOR5+5XLTcAmsTADPvJeDxIJrL0yuZ1OLCtshoXuy/BABZfTfuhb/iV1KDMd2RXh16dKvWgEt/iHkD7Ly5a6+nYO4frE7+xEaPASjz3wN9Awa/PBxy5EgNUj5TytGp3TFKjKxuhDCtBHWnM20mE9c/vM7jT2ztkYjNVBjhyp4xDK/1QjhBRGmPEwapKxy3Zf94um7g57WfcadhETV/2BOe40ej8QH+VELALCIebODB3HSeqehk3NblhPo1ER/Zl3Yb7oAPutso+2IE4xo86e+8iowBeRBqczR6qkrROY/mmbTFSAbexCoQ0xHViKz4ScaksYNh3lO45xCItCOCiccbKA7JRzU+E8Hjj8kaAOMufoi9yAv/7JPkecWTuOJZQIDRLpKzJZE4O0NwrC/+bi1YRXZIRLbUV3uxDpUAtHZtKNb9hqNbN+iaEDeeBr+ZFN6cyOW3nRH11fHhmx+zt6PXfy2R9e5utAzIZ1iCI6+kvk2xIYnHARAmhRfq7iEQpLHi5xVU+ldycuF5pHjx7xIW+hUQvut3Zu5tY9Efz/Px/BfxzXmbFMElfnG/QmQMONyU4VvhjKu1AeQe0JZFlWQk12N6IXZ0YebKi0idzZBw1Faoc28secV43c2gItWNQzN24eSaiMAqYoIS2sywQwO3dSDVNCLUmxEIvJgYvp0qTRQ11qFcv30ARZsTD6Yuw9PBEYlAwsvn7mOxCvlpNfxdHsn5IgETpFYmOBtwCM+1BT1kbggEAqLzo4kojGDPoNfIqgVzjxWrWYHe/AQhJBbh0jeXyc6Q4al52h87/coobQpnZ88eCnXb0dX041HG73w7LpZR4du4ULAb+057PLplxOsljGx7j/VaCC+HZHM939REc9aoRv7ty3QNzIZ5FwCbNl6w92gyBtVgcSinOrSEUAXsqobLWtuzpeZ2FtlDfo0d7veTmdm/CIFIDk42cZAgUzZXa+LZbRCSoypigEHMG+52CA1tXF58mSP1v9NoLCPXfw0AP7vYDoQ3GYLomBfO7i7cjhZT0g1zHBcQqQomMn8rIvF8qp1b0Ivdib76PTREgiAOnHqDnSfbH8XR+0IvQl1baW52Ztns73G+5sv44V+B/2CaPFup9HmE9AkDSY16ND4u7nA6Dob9jUJgwiSAC6PvsjWiCHuhHGI3g9sAANxiF6HpdKPP331IfrCBIdvzcDnnylZnf744OBRpjBZT0GLqHHuIcIqzZRlra9jQP5v9/iVk6t/jsqgAO4sbgq5iFvY9C95jEJ0FiUVEZkoZFwce5LHqbfZOWQx/D8VN9CMZY89j1qn5T/A+tPwN6H5oexe+G9j45We8e+pDFv2+iIOdl1k58yqpfkeQinTAevq576WnW8DPXeW01juj8tYjHbQf3AYQefMFdHaQ2W2mOO0GRp8mhvXdjZv3MIp/X8XSY31okOrYsvNNEgY/JPm7UThMX0ZZeR39ZgygashNAlqv8/19Lcsnvo9UbgAsBOiv4xhWSY9eyBIHPW+4wUNThy07o3wvQ2Vmwpzhj1IxYTlJaH1VUH/JBkobfAw3vT1pghyM1Vq+dgwnwOMy3Dtne3H9fsHBeQBFmcuQOLWgfXYP/SN/gPYLoG8BjyFPvxOrwNZnJoEYOgrxsDbyyGrBLNDxKKmcjl7f85Hb85D7PvTZTs71c0xxzqLF8E9QVGA1w7lkCJiDs1cbw5ZdZ1tZJLd84XBHue2ilnsgUpLbv57bA08TWBiHY5ucYZLVvHjOFuzz8YHqjumEFYWS0pxCk1sTUm0lHLJpoRD3DiqOEqXqJPphMGHFM0kbu++/5lGfh34U/zmXqrl/sn9gJUK9BSo1aCRmutybENiJoMumUf0fM/nPouJBKf3u9SPXbw+F+s8pyB/IH9NGP6Wc/ISVGJ0amaruxs2lA3eyUCZ/R1wyNFz8ns5tK+gnNeI9yURCQBY6h0Uc7rRpHSvdR1FX1IxLixVZ3UCEcTUwcoyNGrViP8S+yZQ7vYi6E8A7WQm0DbmFZ/S7tsop/Kk169AfGUlUsye7Fx7klEsDyQnb8Xbvy9vrGnGuO491cBhiqU3LDIcYuDCIYIOcA5cTkQpFhPRdxb2aRvr6nOT9oaOwWAX89vAT0vP6oAz05L5bX8p0k3lBLGNUyE5GhezkudIrXB5zE8eGQk5/N5PGeXmM7m0k2CmLLUNHMLgikYpNW4nOj2bdp9N5eW06z7SW4yyELkM3iySu3GsLI8yxE3lYESaZ5b/eVYv6eR4rdejULpyZqsbcJYFLabxnf4mPv3kLcZwJ61iwM/gRrk4kXC6FG7MYrphE94CxSEOHkpB4mYkqK/ltHz8t97OTUwk+GY7vlAaG2YOLIJqN7jaKx7+bY7l7P4DI7DiW+rcxS9aP+oRM4j0uAxCsTCfqvgsCnYzZf39IrV8NXYZv2J9ryyYYLYAHnnKyZv7BzIMzOX4ska4dtfwHfihAwKygDHoCIfTyDxjsAlHIBCDx50xtCPY3IhCr/xl8QoRMDN/OqJAd+N/3Qa+3w9mziYPCStob+uNVeZpxbmvoNinJ7HWZas86PAbW8KVfFUXyIaikbcS4XUdnlvFrB7RoZRy9sYSC8P7MT3sX7KDAAO+7NLNTVci47rV4Dd1CjqE3PcX92Nu7nDzFbqS8wxYfIz/d6Uf+pVzqkgbZqMmeWJzbYPYP6EeL6iZuQ77khNWJNYMu8vzJCNrPjEeb1U3jVhtgMlTaRV/Ps9R3BRGotEXUrVi4GPc1beJ8Vne/zvZ7P3GvZhJF8h9Zf3I9pfFdZE86xSP3VuaPuA51Z8GxF3MdqokSyJns2E5JSAnpRjMjfI/T3/c42Q1pqDvg3M3hFN7phYtBhmTWA3w5C0d9kfU+gEQ3il6Xvib7uxRg1H9rCSm8+dStGbtV3+PrOpDfxBkM0j7HcxJQys0o2mswtfpRGaXnkf9dIgUDwdgBhxyoToR4FzAKFSSd+QC75ny8og6yM/MTchuHME11BvV360gfcp+syEu4twxgXCg8l7yG65UzaLklxPSwmDHSBUQ8cuGD1z9AYP0HuNDil4BB1MK0v6ahjyyCkRUAxHv8TWXWR4wc8YifSvrg8/4qPvFrYM4XFvwvDIQhJ9B693B45mUmnt7EW1+8zPPv7XgyOMXMT87gO3ElFnEfLmybz4epeZy8vg8sOhBIGHPNHxpd8HdWY/KNwiKFMOd7RLre4XpdGq9/OQ1lYQjHF/6BVWBijHQNX93dwbKEDWQZQjm64Hf0ciGHKpQ4ibwYF+6MQgA6KzxQvMi6a9fw0kzD4trKNI9DuHnm0KYdwEmzCy524NTtgneVPSOPfEzDwFC2t7biYleDfeo/GTvqTk82OP9GtLASAhdgFnpgsYqwWsGKlVwDRFpd8FJnUNcZzMaHL+EWug3kSrY+mkCIqAbxVDEmK4gFkGXyo7LvXTo9vNm3ej9d1kDOWwbyngu8FLqb+QdnEX5sDo7Lf2Pk8hNo5UFczl9IQXMKe7LfJ1JcBxdukTVyB0uDilkTJWF27AccebQBN2kv4u1iaLvujk7pimZwM/5e2+HeKhhmW6/vNbrRkxdCc2Q78yZeRySAOjMMrwtAtnUp3Usfo4wYQkebjFSXISTWwnAFFPoNJmvEJOLCbvBlDwQKygkR2/bA6yoiaZF9h1wrwJg9APeMPhS4+WBOf548gwMbS1JYnjmfqGwzHn62bNNHwS/RWvQFAGPDx6HZ0EHE4wgARoS18K2dIzUmyPKH2XnFvBX4DvvrvFnk/w5an3qmOAPFP0HALPpXvIPm2gT6hffm1oifuaL7g2gxvOAI+moPFmX60tPcw8OPztC3Q8WdJKiXBxOvOc8EmZ6yJi1N+U0EXzlBe6wDDIQiAzzfCNp6F0TdBpQHlIwqHcXUiX+jyr0K/jOp85nD+T1bGdziRMHwEE7Hv4ApwoLAuB03xQZ0MgMlISUM8x6GoVqPWWxGLLDpjI0KGcUsl1E8KAOHBGhPD8JLcg/3vDEg+QSz0UxsZBmFXVHYG50IK5sMkhDkZZ/xecwBlmoCqIlZQlaJiGbXVrSyLso7AwlUlzPI7TaDvrlNTfh5Vr0LAoGFc8UrqFXNIkXuxk31TjwHZBAaZJsDPmqFEQrAfTAd4T+y/cqbVFTdIqTdlYyIClrVVU+z8kSmFg5M2cL5hNWc/CqEoWoJ02s/g9rPAChKu0XrxxMIavFDJyxGL++g1eADeCAUCAkuDSb1RhQliRVEZQfRHZvJa6Z3cHZp4mp6KCN2LSBX/pD+d/sjtAixy1wHTdeRACtXjOKHB98grfqTqUenct2jEnLeh5oTGIMO4lMKHRZY99kydoSV4PVWMz76qzDwEIoHZSR751HbbUH36Wu4jb0F7j/bPnL/mWiRoLGAxtWNqqh4ohxjIWcdlO2C8OcRq5bQz+cYjyuTif87l6jhxQj1n4DKptnnYGnky0VXeaPQi9j3VlKQegeWTYb7z5EotwU/rWIB75m62fs4hS3urzDHawrR6qP0HaGnI2oQmVl2vHjhFJ++NA8qMmByBdkf/UjWVxYsI/PIis+izqsOq4sNEEmfb6jM+41mnwf4XFjI7Zsx3EkSsTnyDFwcjCDyZX77aS01tx0pXXMSt6AHdArdUTvFQ3s2mHUcLr1E1e5LJDU+RKJPY/7Mz/+9beK9ung8GxzBrZNKTSw9AGoHpuZ50PvjVehHZWJJNKOxwC9VqbjqozH49GGw8w84HsxGMElAn9V98EvxQ6v7g8Zg2Gksf1q+uUuLQ3MpnS6B1IUOotXuMLMOzcIihKaVYJHZIbBa0Lf14P22mjuVG5mlhiwBaLrBteH/bc0+q9VKVVUVjo6O/yfP/1+w73/2f2cG2wbEKnND8OhT+kl8yEz1pqjZk54WV6SYGGBt59WBM9Ca1GyvimTaniVUuOrRVXnxKKCeQCx0G+w5+fh5Fgec5oyLkIHmfFKGZOMoiuZmejv3O7wRm+1JyHEhb9cMatI62Vi+gXf7uTBDDWNqwF88nAi1F99pICBQy3HjDs5nWvh10CYo/RVmanjXLoPr92eSP0jD1k1f84lUQbdjDPK2JehiBdS5dmO12hBW1U/O15i1PDSEcafxb+p1UpKOjeZM/yymzPTkU1eYo4L+FaCxQnCjjLgLI/CPucaRTguvqEHZeJ4fz/eg1YJFbKW1PQojMqokRcRWQKTMEUezF0rnBkpCS8gIBmn1RF4Z+PhJBRYQnePB4lOvcWlMG/2a1dwe8AXC+8+BxYTBaTIn5YUIpVK01v3MqoMtnqGE+/ViQGsKB1x8ya/9iPJYDd0OnhToJqA1qZGJelh/rZaeHgHSsC+QLN6PV4AShE5wbQqC4Y8obLHR4Lzg2sJGN7ji8RkHD65Cb1Ziif0NxZ4ZqL07mLPyOAPkAsrMwwiW2QSd9zQMJmSrJyEpeZz1foCdwI7vNPDdE7/tFNfRHOkORB1ew/LCC6yNqaNZdw2wcZAIBAI86zyxt0i53+8e37mBIvMl1rXspu3OZLgbi+QbR85OaqAo6AmiunQn0V5XubyjnmjBTI742yi/chTjGZ75GwBv2I0kcPU96rrHMMa6l4FyE43mTD68foSi1r445J7h/vGbDHMo4IUjHgxalslK/RB+sNuNb8si8ltMFO4Lozo1ioNrFkAbIAJ5y2R6AkYiNQvovJ6ES7eOhf5rGe23F+YYQSgGfQvL0r7i+wQovBFC8uXj9I/fCR3lDHJ6lTvrnsMcPgVR1mgau71IcEkFLKAM4KZUQGF8FlJhElPe2My5MXcYveA8Jx+vQWdS8sXlYbTejGf/igfIe33PG65POjrmNYTGVmYt38XDhrFkts0nr20OY2JGQfEcIl1vUyycy7EfN6DQ1HFu1EkaPBr4xn0hQdNfR9BdilXqRG9VEcahRfQSLEFrgd2dA+kv3w1VR7gkHseBZ2UkBynokPThc1c1LAxCIVFQFlyG3rGWVe4q/pryFxbXNsyD/kLklIDOWMWNxSu53aUg7epmxl9/ida3fsXZow0QEGTKYu3Pz6Jx0fPd0iskPaGGfDllIS1aH17TDKSyIBDHHnvkRUG0x9kOzp+7QbFOxtwmbx7436Bf3TCGXR5MvV81hf3aONQJHYmLuf6omuwxR3FwSiLft5Jq5U0mCP4hD1vv7Uplch2y0FR+C/oMk7qTGb0+4mL9QbL63UXv0ZdKbT5GSRVW65ON0ODj7Ko5TdfGb+iyn8LQb8fzk0XIi19+zd2ZO+mR+7Dn+3dwTbtHifcQ3gp9HbncyipTBHOMTdzyBYOzmQyZlPh7kUx62Jvs16vA6TpFLX04XPguowcmUu0SSKZAAno9Ee0aKCqHXptB6YefKI1Wtxoc1T5MuXgGrYsSwYousOgRNQsZcXEEfdL7oPg1gEdl8bS6LGWyCiaPymZKaxYODlBz7xYmTQ950Xlkqpv4shwik+/i1OlNcs063t62hpEzz8KTYJ2q4zgHqwdSbIT00TWUTfkeD4c1rBr3DBzxIsYUyOFOqDVK8AqpwMe3/Wk/79BAjdCZZtdmpAYVEqMMqUWOJuxrHIrW0TumnR4EWEQWzFYzQquQV1ueBIY9R4L3GJIrrjNL5sqDzkRic8OJcNDg/fhjUPjS1CTArSuNts5K3CvTuVukYmH931D8E7rIGwTez0BoZ8f3c2/yZo2BrZFiTh2zUdDM7j+Ok5lD2GYNYlPcVcQGFUarBTqLQd+ERe1BbdgQytwvM7Qa3EWRzK85BVcngH0kqe73KdEYOKCazK0KF2Z7dDDyia8tQd5MP0d3BAYnfG/7cq3FhdKlUpb9a4l9bFrGucxBeOZexbXyIQ9nfYyvSy76Lg+8VMVYraFcn1hJmfwvfmMN1Nq09BosCr6zP8cAjKS59UMks/uvpVvWVYeq/hHNoijaPXsT796fl81dTPEq5Ga7D2vOu+Jd602f8l+QarX0dC/gfO5mdGYl4jABP83fg8AqYMij15FaHRBa9YQ536ehOwjwZVDDAhRnbgNQJjPQZ9RDG+WfRIXAamHi4tM86LzI4D0z6VYksmjJ74wfvoS9F1aSeyiMztB+HHeH910gyf0YW5Y9wN+piMoTZVBoj79QhO+JZahWXaeuK4zX/r6KQVrFmNpTdFd70O91K+ODPkUn6EWPFYqMMMEip/nn6YQmFFFq38bfOhM9TEPdc4QYKSxxELHYs4ut7X0533OQNAW0W2ChGhrM4B0wlWkvnMW5qgG7x2W8K+vmh/5noaca4t6hW+TKsR0TEXpqaKl2YX6yJ9/vnYbz/WVUOx3gTLeWeXfmsWxPH/6aWoVnSDkALYIgBtxyozqqE9+M4QS1TWJaqI1SD//ZFLa6c2jZl4S2ziUkXUazsAmubYamGzCtiZyIdi699DlKS28sVimbPRpAC/RUgdVKWWgTd3Twze4RfKq0MvHlEBvvndwTJPZsahqF9UIxcfJYpjWso39kIL8o6mztApbFZFIWYeCnnpHMdrtDlMQAFhhpV8+Da8voNurRhKfSpgKpCKj6E+49y/sdHvxKA6lSZ6Ibn8FZqYTWDFtgq3Igv94aQMGlZM69fJolvnoE4ieBJpEMsUnDg2EVfNsmIffAKuZ5ZWIfdsP2e4Uft+WDmVL8GWKfLURMiKJXYAsrgVj3awS7FYJwE4nH+xL7QMSUkhEcDQsn8Rd7Ep5Q6giAgev38UtRKg8k97AKTax2HQQWI67+kdwa3Eh0QBQhcbdw821Eqs0HWTwWuYheHR0gg/fvVNB12cxfo+qYPbkEzvUlSHiRrAB4t7GD5s8W8YbYyNRzruC/Cvymo9y+GrtPNzLEuwXvAn9EU26B7yRbn6Svx6F1MF0nguknsLLv3DxWf/INbn4aW8aAvpW+ngOYeCwDg6SHvluPMUEJnH4CD5hnxVO/jyX2f3OyIIoxl1YSOukhnEvGHgVbfxuBn4OUzgF1DHW1abWS/RbEvIlBI2PE9yvIj9Wz0G0JvsoqHO2eZBlX7KewJ5Pvr7fiJbjOW9+20DqsihnDfwBdI6Sd4kZtCmYgwViOSdzBMNGa/5pv5rV7YU23URbpQ8qximRg1EDuuxD5Aq8VD2TUJ72Q+VUTvuog/uLW/7r/8y4vIhxa6ZYYbFluACIF3zl1cnzxb/ySuJqB78eh8lDTFdWMync0otYHpKfcIS8mj0FNLzL98BKmrDoCtxdDRyEMv2jTsTFLqB6cyZEVlVxgAhi7oOUueqEjt6Sd9EiFfBT5JQC3HLYxs972+KUO3fz6/Md4NY9iZfMzNPjPJlrcDvpmyN4MsW/iXudKQlYCRpmeJRGNiI2tNq1C94HARW4MvIGXMZTKgGpKxR2IDG1wvh9Cx7UYGl0Q9QTxdakvSwPbUZu6QCjFKpCwe9Fu5AZflpdv5OvCz9nzvG3eFwqshLvcY9n7G/m+cB9vdn+LQCBgjfgi50uW09vzPKVcodD3HXzEg7gxPIKwkGncbZuDs+myzbku1qGVaPGs98TlYS9+Eo6kKVfBba8PmS/aT5RTBOu+Wo1WAXc3vshkqbuNRlZk26AVC1ZzrBCknRv59MvprH/uOKL6C4gEAoonncHkMo6a+hXIOktZFPoB05P7gV8cXN/LmuTn+L19OKESW7Dav+c8rzjBi47w2cPFdDm24apRUX92MI293ThsUPNrTykXqhLpGZXB+WEVLD6WxPl6J0JnuTDncCueqlKUnt6MPyjHoaUD39Q7jPRuRWjp4rk+m8hvGgTMQyAQgBUaPBoQmUW4C7yfjj+BQEDFkzPh3JxzqARmhOaF4DWMw1duEXX6exRqEa8ffIO8mCLchmwmImQHOzM/pded6/S9k0jmsh7e0ydhDtcybqWOLR99w6BRCpykdjRaSnjUE05wOfw4eBM+6se8MnA2bxct5uzXs3CWKxBnFnJG08r8NFs9VjTAFmkIR999Ds8oCQPdhaQq44mVaBALDehMKiR2Hqz/8zbmompcmnJ5IL4D1m+eeu2EAgsyuR6RzMQrLeBo9GSN1InLxmBaZFqEDiBqMpHyOAJRpIQcwXv0Um1mZchRvAuCyTUKCY1oYasnoPkQl4AhnC95hgRxKg7SWwQI4hk2fgcTldjGfN6HUPAF398cQ/TeePSTj3J8/j7edfjnW79SvoCKqr3Ynx3EvTHn8QmpJLMtnN5OtjOxWRnGsv7rQdZKhT6INNV8vGL1wOGnZejlRjrUDfhJ7ZB1eaNQOELlYb6esYlPPXbgOy+N6rN/kau6TpRg8H/R0OqtNlkQa7iCgMgKNLLBmCxSBvgfxktZReO4mxRWBrHsx3B8XzjM8mRbQLnVJGFPgQMRFQbqR7hS6XMes8hMhELDIP/9KESHmD36Ep8+WsL1OWfZHWbLsr9fM45td3bx7Ju/szvxVy75mOjTGUqbUyPj7UbaKqXJ4z1pMaccoUBuxtGtDaHwSZ2TviLS8BI1qmLUlniOTj5Kj4sdtKbD+X6Q8AnS+wkkZSRy+5XNTJh5m0LTC2xIPYG3ugRV/ga2aYpxU2rwjqhgptqMvamUi6XLuFi6DFPfzykJLcGrNpiNn23k/pAi1vc18+MT8NB5wwS0+8U0BYTgNeo+HpazzIo5BYDFKkLSEsnsrv0UlsZilqrwl7hwvmQ6ACsHwER5FIFnNtBR6UdWXRseLzdBvBHvvGEElMykQR+NT50dS7MXYl5h4INbx2wPVl/mxKQTOJujGfTbTB5pPAh83x9JMQisIgbrx3F13DpcelKZcOdVgjxcWNSriYCOAI5UByP09iYroRYXeyvPO8IeQx0HCr/655u3ViLVddLn0BjaHdrRLQ7gu/vfUdqWwGRAbS3GL/88rT7RVPuOwGhVIQmc+/T+L3xN1L+5hem13xGWsA2wMQStte+mM1KDh48XHX7J1GaARAQD5PCJK7zkWkKboR1LqwHZ0Yk4JRdDHw2ZOvirzYWPs8Opu55I/YhGYi5O4NycbsZVdNJh6STPdyOa3R+QnaGnWb+cJbIeNh5SURoIP3eKidSX0X9gNo/C2/n8/aloMxpZ0H4Vsxiua8GIAL20jsZuMZGyRBJyj3BKsY/xHIKHr6PqHo9DdQZd5T5Y5BbGKOCvJ9PkJ/fCmXRiOJpKDZHrR/LjfVuurUJgYZj5PrcOzyYu2wnjM0Zk3a3UtcXgW/YfCRQBz2x/nqbwGuRWOTENMYTOfgJyknuBzA3pkHtsT84kpXICFpHlaeChuceXlog29s3bx+aRm3F8W4azxBmJq+Rf87ftT8uTT6axO5DWsJ9x9uqL9fZyZs58gE9vFZMyzjJ/V28eDViBKbAEbelZjm5cil9qFZY0C6FVi3B1LKNeVUdg2sss2bKC1nYpn34qwL6phPWJs/APrcRo2QvFXyGRVZM98i9kyiBePDeEqaGN6Af0QdZnG9Z2Z+7nf0bsjd0EGvRUxhSyzBFctSUAWEVKDuS+QVVPX8r6TsI/YC+m2ATEuTbZo9D7c9ldOoAHp5Jod/yNssBczjX0ZzqvEucRx4r3VrDncwXKmpMMOxNKsa8jzlbbHvtITDsrkrIZ028mn7/0OUaJEZQzwajBInUnkfO0+L2NVB7Bebu9OLv1s7E4GNtxl31GwvFxJGYkIvJtZExwiy3QB1B3gYDuX3E21/JHx1DSEx8zMqjun8m98CsiGgWM+H0NReEGWsL6IHYSgv8sW7DPqEGnjGfajts4dNYT3u8IXlE5GMx6nsBA0Yh9GFsDNUYInXKSRvdmCDvDa8VlFHxayJh6JaaFZj4yuBHlVk6RLgWSv2PfMUdOucQyZ3QAuiPfoGzSUSUbTJBaALoGtvV04uXaQZf0FkenHrU9zKk3jLwBqmA+8XoXHxmsHFjC38nHiZPPZnP8h5D1OsjduWMei9yvFHe7Q/zpBVgaofa0jVnOezQCfRNuVQ+pjBWQPuwbchpH/2vFgnvOMvI3fMbArK+IbMpAbA5BKHPgskGMJTGdlJC+xJ9X0Kd5KTdHe7I3+2VW++6l79CRRP01G59+PiROdQeDhks/e/Heu5uQrOiASbbypQGePBj3lu3wIxAQVB4PVFMTMRKJFXReQYhMBm69eBinNBmHZik5JehmTj1EPgEP/s/+7+x/wb7/2f+ZWaM28tgygJDkuYhqjtBTkc3GSR/zw+PVvG26xzCnB0QZgCeaJjPMLTxuGIizSxXvbtqK2KLisTKWOX/aAhPTDUI++nwywaNLSZy/m7aeLgLCFjDC9RTf5K3jZ50Pzr0zSLgfjsT4N1XL7PjBHdY1QonRSqIqjVlHv6HTv4Wz4eNx0SeC12iQOIJQxmNXCW4ff0pBy2y0Ci12VgUcDyXNWkL7athYN5qU8/1Izl/PjpU/UjrpC4INVZiNnVzICEPpaM8vy3/BKBfjYTeHNrMtKDheK6WzNpJifw2pm36kxKU3H9R3crAN2i0dTL19FGV9CcdmXEQoaCSz+Rtu+sHBTihmDpsnJWAWdSM3+PBJnYKXonojNz7meOE6JgEr/UvpilCivxqD1CCnaOx2IjpWQ+1JhE6TUXeokertCCt0pFSexNczF/ONuxltdQ7Dd01Fv6QRt7Ff0PwIWs0fMDniTVJ9/+SFU4+Qd+lwcxjI632eZCoa+kN3BaKclzAxErHWQlexF9calJj9A9GbbUg8AQJi82IRWov5VgMnO1QEKdN40LiFySootIo5PvE43nXhvLp1Eznz8qjwdMMgsWUvWaxmagIEiAReDDo8ktPV5QwYF/R0bEVbM3nh98V0eGvY8BIscwBL3Rk6hS40xeVRbOeAnxLS4/YRq2x/ep/K8ID7B+PAq4Nhw5WslMUS0fYuN1SQZgeR8p0M9/uVR1d2c/2FV9CPusvYxb+zecjv5DYO4kj6DlxqcxHphUi1bgwQ1TM9LosFWrgiyec1oQX57HRa3f76r2/hXPFqBvaGDxtn0W+vA5F0cZ9get4exgyrGcwWkDgwqCSCEkEhM6RC1C5diKVGCFvD8czT/OXwPUFd4QTpX0BcN4UEV8ABmFzOzfSbFMe2kxohRBm/C3e/Rs5LC4h0nUey9ynuXvPA0lFEt1pHgbqBJfot7Jm3Bgq/xtJ1gKGBe6kQF5MiNzOm3+e4NKgASPX7Cy3rOB+SCuZmbqe+8fT9CnzH2/5utdD9ZK13KZYyr2kgX83aBwFzIHwNE1RB/CDZhaMgGHnxRUpa/XhBeIqMJ2ELs7AHgUhIVoKNXkWg8IXMjTgoxrG3MQqJRU24UxpG9WHqGIrz4EXgNYbHJ19jxOh7HLOoWXBxBFEOZph/HqVUg1KqIbC+N+G7lvBgQD6/vvMBZ01RbNO9TB/55xzI2YKHbwdOXSlY7Rfx/apX0DvbeGdmqqGk8xj3BH242/8ucZYorDob11awKR9uzoPYt5jm/YCWxSpOWX6gvKWcXzysCKxmflf24tzY3SS3J9F68QEzf/dmQMtwRqlf5p15Y6kSpXNC2cwwhYTwpAJ6y2yUEap7y1ABF5TjOZt2ismSkXT3qLAKLfQOCUdefgNnqYxbQXPQOrhR23GT+QH36e97lzcvXaCuKxQEMCjtIZkpmXxW549GUMkqgT1U/AERzwNwMWYfNR2hFFW7kHDjFHW9xyLysekjCpuaUUeVUiAzYjX053TSGAD2dcINwm3zuhXmHJ3D0aPwTpbtlDRPDfOHF/BibQsCiQgP71pkChtFDMHLuCfcxKF33sS51Y6opZN41fEh8YJdcP0M6Oppt1/NjdWj8PBS8P7MVzE7QV/PCQy7d5JsA3wc0MKwVSfZWjwez4KTzEgDhz4WCE7BT+ZMWXslG0xOFBQGE316KsVRdnT9JkDlYMss9GjaguqvKHylAj5+5TsWubRglHkgGXycjcOF+BQepNnHnR82bcVL2tumN1XwBWGFw4h1nU0BMmLOC0nUxNA9RMO778K5cxAZtIBTbxYRGOYLcb/ZNsp2HpCwFToKqJVdpnL0LVpaHuF+bCTtqa62bBcAhR+z/JcSF3eYCV/Nxb4lkO6+MjzLwFkEAdZIBmJg2Ue7+fyLeQQVBVHGvw5GQJ1hIHdu+yBW3kPYS0itpQAmFNC9dwTJ9z4h65EXPKHlLxNFQMSLIHXi+p8aZh9wxBQpYafpE8ZEH4KHr4KhDSoOsm6BhJ7ZQr4oPMZPajUie396apOBvziT8wmhxUfoldsL0ezzjFF0U9LsSUHLEFJ8/8IsKkRpZ8AELHB4hzDvYkTW3bw2aAY3KmcgEByiT2wTKBr5RtNGtfIuHaJNjBq1EgC5Sc98/05cawexTwhmkZnxvn8DMMj/DOn2YOqMY9IPH7EzpZZlu+OJd/oJ6i/w3qldOBYW0LnUjOvMM9R6qFjh/gKtWm9OVQ/ls1f28UOriPsVVylujGTC6pn43piCxekSF/RuSGceokul4nHAXcQWFb0Fn9HHfISzPpBef4/6m80kloUhK3+Lz56pYUTUTj5xhbPdIJco8c9tw6mhjK+f/5oeRQ9fpeYiFtsCqS3iQH6f8y12xkiCCysp963E7JTIJel0/si9h1+NAmugiCa3JvQyPdcCNzLFJ4YZeU24fLmNw00HeHx3O7HGf1EYhixjRuG7HAiDwYVlfPniPhQWD3brnnhHj7jhIh+ASdmJV9kq3BrmIuj7xLOR8RK0pnPqcm+qzvRnYIMbCt8GIpu/g2Yg7Dlw7o3QLGTo3yPoUUBD2EBieiVArr8tKBE4j5nliVSJ7+Cb7MLxzi8QypR4BLViufszwc8akPV9iVNPZCTsOq9C/rMQ8gxOZSvpfcWF2Dg74n9+RGxaIXgMtV3Y9pAar0gEfbIZ0DmGtV2VuNn/AEe/gdBnafFdxe6HO7jdIyLxkSPZJfEMmnwDJpWAKhh95m0CHkVglBhJ75POFGe5LUvMfxYKmSsIBEg6RfhV+lAeXYhj0iN61eTBwZ9hTAZFrqOZf203rno7kFoRANLOR3B9HH/2h4llVvzDbrN5+vNMk7qwSeIOF4fi6TKPMffqkIvMmOO6KOyUE2aPzSkwvZnunGY+nvoH9Cio8M3ALNIx3WqyaXsEL+asqTcK5yaKfZtZOeky7c56G+VjzBsgVvBL9m2Es7LoqJ+Li2Y0twNfYdLMf0Thlfe/Z/y0QxyoGsv0OggQ9qc85M7T37d1HiT/QC98xK0o1TL6Smz6j4WCEFJvpVIbYqAryEhndhLdaY9Q2vdA0bcUGp5D0qylrUtC/U0n2iRO9N0itWmy1l/ENc+L5KslaAIcaRbVU2xS2vbV6lAQyjj1KBBdjYpp4aXsC4UD1nsEhbfxbNIL7Hn8E2P73ODO6Dt0PxpEqnc+do7fQe/Pbc4QgRijzMr9PvcpD6xlp2c+M7SvcOhf82BFRDO3oo/iU+2D53UP3pzwGmtrX2CsSs1xHZitFhIzwunwswOuQ/leZKZu8pPg9442rhQKGbf0JD4hNSAMhO4yuL2ET+8l03KxPzvWHEElrGRt6DhGXkiF9hz8HZ7h19kHsDP4YRb1ACD6F32Ph0DDNBU86unhQdcMpLqpDC18ciRPtGVShPfNJyM8g+vyDrIcjHwz8BwB/qMA6CW7yzPDi9jSamKYVsbZYBDkb8Zg54d43jf81qZiad1vtH2+gsMzr7B0uRWG/01pxp9sDD7DubZWxKcuEVxdTbPdVFy1f3G7agp7KtNotHvALetFJhwJpjyoDIHQyvZ7vwDgk/gSCx72pVQahdzudeK8oboatt609fhzCfdoyp9PceoxBi75ihbHl1ApotC2bMPoGoGdLpOU/pco1IezzB5WODSCWfs02FdbkU7chXNI9Sp66MWrnW58kezJGzfS2ZN4gtGysYTttMOrLBXtgi6QOoH/TA606tG82Qtz6BlauhfTMzAH66I3aK6dTqERBD5hlPYPRGL8gmGXh3HPr5a/5WJyTWCvvMuq9j00tTvy17CF6EVFvCb+mW6jEyVtSYS5gjQ0jE5vPSVT38IqgSAMjAv7gXFhP6BjHsu0FXimj+a3mGySp19DIviHc02AgAk3B+Jc443CLZ9edk00VRhx9XoBk9KBouS5DBznQOOtTDZFV+Fca5NZyGoYzruDvyfPXcOj1v7UHZ1IzLPnCMjw5qexMtpHd7LoDFysGc7ztWYcrXoYDI9bk/n4xkGy7AqJLOuizrud/bP302VvelqnbiuUSxfTLDiHd7ma9T/+QeRzQ0mKXY6nqoxfMr9krBAMmRo8y/II+/BL3g3oAL55WobEUMn2fv34K3MSgbsWoe1fAYXbGCoppXh6KfvqRtArR8fow3NZ8tZOfAdtpPNBGt2oUO6KJsqji+4MJ3ZoFCx/ZwdtOk/6+hwnxuMu0yZ+yNvZJ7nUA8ESiM/7GGQu4D+TIrsetInpyHzVBElsTAeH819hV5Ytk/N2yFdYn/+aHocebol1pDbOtQX7+v4ITr157JBJ72k5pH/pQeDjYAaN+uKfyaKrjJPebexpscPveBpTC4JJ/CQes0BItaWRTL919DPdwPKEclCIAKSOMM9K7IpUJu4fgHF0BT8u3MQKP2iWbqC/71ECHbPZlneTJmcDbYpTPBTdY7RrNW1aD9aeyaGWNq7MjEBsVhNa9zoFvjcBmCSz5/mEuRQb3Ln8/kz6SIRcmOjG6zIJz7uf4/190QB0KELZ2QGOZiHHppzBJNagly4HBCBSkNxq5tGRCdxzMPDxyJ2YTJ5sUrlC9THWVvXijvkBc1xCeCY1F7PIGeRPaFKyXqM2YQpNAY+IVepY6wg/GZrYn7uZhXFv8rBhMPf72rKFnpUJeNEJMg3F9PY0U98VjL28jOEK+L2rC+ch90iNaKVHPM/2rXeGIKIMe00FiqIWKkrMpL++nmQf2/vwVJfQpDHRWiAHUzPTX79PrMcmFv1Va8vAt7oTb8rm0V0Vcjc9OUlS6kSLSPGow5xzEvfCIDrlOjAKCa/3YZDsFB/zAbNituDgeJtYL3inoYP0yOex6E2MldjeqRUrPtJmFqrhujmHLvMW8ju9iRP6EuJYwcDmCm4lR9Dl38anp19l5clKwtcPBqz4qgvpMdkzbfgnWAbX8czfFgQWLVNFS6juiGRUyA7sjM6ECD4l8Z1zvH/qK/xK/+J8y9eM95/9X3MGQEW3D4fzN3Hb7ROS5bBP1ExR6BcMToaaJ7LOIhEc6LKxaXzv/Sfu7r+wvfEdGm8k8adYztGwNXzTuBaT+iaS3mG4eDTysLcPfWLPcKVXNYoq2OYGSXINZ5e9jEnqgv+BAYQqtcQ83gUS+NDZREf1HtYNz2NTRy2KhNMcvjOWRZ5+AASVwUhVJTuPzaGiIJC6cRkM/TObTI/NjI+rBWUAowbsJnboAfKjVuD2E4R3D4WFtmB3QWwu1+3r2BS8ifzqexRpbJTGk9r+pF7gypmAYgIJQB9sYNuLPyE2H0Bjgl89oI/UyroYL5Se0dQt/5U/cvZQWLqWQQNnI3Pyx+vKYK77wbQaC9r2Lvwr/Lni3kKE/zNYEWK1CEEEJqOJYT/0wrWXFWHkv/bRd+6QeO5vMlo98Mwv4fyEHrTeX4ET1Pf+heRfEgAL7kl+1IYNwiBXI3AfyPwjzQQ5/EmfeEdq29qYvkeMvtco+qcshgY1JutaVM1lND2U4J23k5PNo1n93s9IhFqoOMAnYZWMcIYJjUIczg2ltKWKlPdWw4O1OOiFSMx7qIufgY+nmVnbdUSFh5I8uMNWZ7GKnRkDCCqtxM69mKvMZ10kEPMyPFiNoGQHvvGBHK4cj0P1X8Sb7XhX/BrU98HTcwTzhszj6LdQFvIROeH7aHVu5a0Of963r8TNp5m/x13m8z5edGbZ9ohWnwlQdxZzzIcE/fEYj4ZiPAf7URReRJKkry1LGEAoR+zeSXVIBQmT52BxOPBPPxf/QJF2FA92TKItrpKHg9z5zl5NakhfaL4LVYd5y3kc16RQafTh1+n2VJpiwCcd5tmcTJ1lLVg7thCSkcSHmw7ylkJLs+wfSm2TUIVq1wz8lSL2rjlKdlcUPPqMDZqfeYE0BFboSe9G/+Ea/J85Rkd4HniN4Sftl1yJmIex6lWKxmTyqnM+QdoqqAE6HmH2SGPXkl0MkKwFI8gEILowACQqCFmOKMef7GNzcBv9mK4wDa+4XQWL99N6V4RnYQgIJPJKGkfvhjFl1VFbhcv3QPI3SHy+oXJ9DVdFGkrsm5CZnfm3iZ78+JXewzerDZfZFxEKr6OXmzkx6QQDg97D9Uokdl3V3Oz3DSUt6SSITkDOOzgkfgFeL3Lh9xBGipook6ym3b0ed0f3f+YjgQCEAqxYMVjvUBk9mFc/92fzBzJ8rGAViqiKGkms6SGaK018fmUjD1f+yqbdsymOMWLw/H87s+//2v4X7Puf/Z+ZoOUe3pp9iA4thYA5mIJ3sfCb9YhkasSBg0nXQ5U4jsM3XsLPPp/DnSIKXvuIdt0TkUurTW/KTVGBnaQTgfsaWrxv0GKSEH3HjXCPfix3OslSB+if9DVzq6Zyw+84k7MS6dPjCAxlQi3ESiFUUkO85T4lD5ux5oBHmS+1Ix/C/X0QtAhEMnZ2xXPf+JB4k4XvDB4sC24Dix2PrbGsb8pFghWTxIrWTotRYMVZ5QM3VhNpnIrTH/PJ61/Ba0vOIUWCUCznwza4pOzPm+UiHuwfSf78h2zwqmOIYCoAGXrbghETl0GtvRcT7kTh3JmE5/JMCoSbcO7oZu0iFaKDBso0dnh8tJK24ESq/0hk+8mFlLb1ZhKQE9CJ+6o9NH73FX7NYlugDyBlD7KGbH6+No6C9Ai6VWbc7HuzbPZv8DCH5y3xCMedYlLUa/9hCEQRNBQaQSYX4FH1IQ4ZMvTJSzmY9xqzYj4CLOA3DaPcj2fa9pJzM46Wh8M5b7WwJXE9k+bO4GbldJrFiUR/fYhv8+dT+ftUHvi34eTXyAJ7GGonZJ2ui6KUDOrvhTChq5kp3tXMFY5nfthvrL75NZ1dj3BN6cWDHgEx+iJyNWoG2/+zMHXihNOEcxRYXZC+t5rf44qZ+c0s1PdX0BjRwkMnP0bTjFHcjuFfoBOhVUtp79n4hMiIuHmde5VuDB/0EYdsDAB8VBnAhTNHaFbJqO1zEDvvBgLtPiO48y1c7GqI7p1Jl6eEwxoJpb6b2Rpgu2+AHbRLJXQ6tlPuXEDY4zC+OBfPS6NtAawZ0R/RZfRgusenXOpXjlp5iwLLDs4bGplx4Imzbp6V+lo3wgs8eRxzm51Lf+BXYR+y+2zno4vtGG47IQ80oLp5jvDaBgSCRU++Ewtholg6uyTYqYp4Z+oBNoaCzjCYQ1XrkIm7WRDyLcawuyToUhkmssONZjC0Q87bCHxW0v+PPig77YgN/IZBPRaqggKxOi2nojkA/2hnBqUdIKP1H+074b+ofgQWCxMLY3B72AupwQHPkhEYp2QgqdgPA/YhlkHhgGU4OoJ29vscDrRpsXT25BKTG0PfB0F0923/17xh2/SFcpgxtxuQWbSoVgTQ4/SAGI9P4doRiNpIoySUIwO/JE/ThxHf9adEZqb8xY0ERg8HVTC9Wxbhv/gxlyQiSo3QYVBgwZ6GrgAKqhewRvkHqd8PoywsgntTbJSQziXwpxe4SZzoRSEfusDZagP9771LWkwQE03XoOIBhK3iVM95Ll4TEZ9qRQosswdz6U94CvsRKAahSIDZVUJuTC6NMgsFzSlwti/T5A5crlUhkMKRvrkM967/Z4BKnemw2JxiXe0VXP40ltIkEeO35PG3Du75LkZXvZLWVmhKuMzwKFtf1nRGEO9xkRZTNOGlciwiLfZPnBV3Gc+Y3gmQuQH6/cql9Hnk/OSAz/RSyuIngZfv08craOOPsfn82iiisqSLtLtp3O13l32hWk5WdtNUcwuBKBawx2wGv6ZlOPrs5G1nCJfCuYb9KPq50z0lGGuwCeZmg0CANb0GnVxHj6KHqJahnM7div2zeqLE28FvBkL5W0jNP6E22GCgd3RglbvTaAaDFQJELUxTw3ciGzG8QABcGQd15yDtLNKeOtLupKK6mIxBJsexJxWlvwe0ZYC2gXz1YlokhfTYdePfuhYP9W0k+jPQU02TYzUOih4Moi6Wemn51v0WtIig96c8enyFlJd/52x1LN77U3DtcsDZXEuicjuJSfvIkO0lb/BwrMoK3nnnHQwD0uFdoOoI1J4mRHSNd1NuMelwIkm3JpER1QzRmyB8LUidyNj9HRevhBHSYUBgNSEQCLAAzWZwQUi3ahgvHq2nOPUjVqx+nVn04992oekTYu4ZuZ98j10vnOa+MQx0/ck296JO1EaruJoxxZ/QWvUG4yfFQJE7uKYwRmKl9nEKzeoWVNnvIugohfjdEPs2OCdxqK6AVkENouJr5F1SEbfqHuc63mPn3dcwi1tYtOB1MttLOYiJ3eocXpG8SZBjNgviNnNCE02GnSs5HQoydrtgSmqh31wb7jLU2UbdF+R2jz7B37Hl4UhKlaUIBf8cvEUiKUuOdtHRZka6PI6f/Sv4rmAp+vYhtIc8w4HVP7LJZT2W44U0WXvZAkSu/cCsZd/NNMSDjLi4jsUQfZdP3BwJldgChX+UFmNfbcBqlrH0kRMlmlQmrlcz1KGZ3cGwvrqJr2PyUBvCSJSBh8DMYutPmCTwTmU0DprVdFw8i8xTSXj8PYb6X6JBKQP0vF/vylBzN7IEexraB+PT6YlYZ0Wq9AeRDUHsJHWlzcMHk9Yf+4Xf86wcBBY9t+QTUV6H/rkhGL5z4Dfn3wBoFdhB5gbczUHk6YdisPNC0bcffl77MQtViDCCQyxtXRF8/5c99vaeEPEksNP3J/g7DTxH8V1LMeouLzY2Kekr3sMZ0wPGzu8FtxeCS3++b6uir8VKyaodnOz3r2CyayoAs1zUFK7IQnw4hcCc44TJim1ZV3429H1XpTspt2ajCWmlVOxBp1BKqNrIse7tXK4VssQRBOmb8daYkdQ+ybL3HoOpBJIu7gLRCMSeEtrlETYH5MQisJp5I28AurAHrD0xjZ9+WmXLYvNptlFGBW3i4eeL8OhwoGrQNIKcM23lZmyAwUeQWTqZeno8bQ4afg4p5U0XHRRuA5d+4JYCViuyQekcGnCbUp2CdYE9iKyAyQhF3yGQ9iM2Jxb7blfSiheQFl+G2/VxtvaabI4FK1biZDBQoUXadRcar+DQeIXB5Z9iyO/m/uZsPlp9hFzDFLhmAG09+KdTKUlB7abn5KSPMYrbeUfwCmABfSsitzD2zdiDFCWpyjI8db0pB4jfAkDJybncj84iWrUUGhMQ/8NyBoBZ4ooyQYyhzMobW97g/NIMeLHEllkF7DMO4ur0X+hSddEVWEWJOp5f4j+iNvcyrW9+zf3W2fQtcsJ8fiLrAgLY8dbncDqW4R43GfPiVRJE8zC3rQVgOUDsGxD7Bo5nz+FSm4smxJ4fnvuBMMZA4Bzb+zRquHA3lpRr8fiEtPNZN4R+FcOE2E0AaM329Km3okXLRMMGsqsEvCG1g6iXbD9AgKOGxpl/86jbxqzx7/kCIE7WRYgE+v2xkE4nA46zQiH6dwS73mD69ST0Tu3sWH4Co7KUbQYLWLRYXFOoqrpDp0lAjz185P+AI97YQEqGNlD6Ex6SQ1H3Q3wqPAgoGIglWGLLzAMa5X0IEP+Mzmxi/RcvYQ2qwnXqFaxhtjrdsVzCrSmEzXXdVLusRSDU/1NhlQ24ttW/jmwPCzcKBvO4Mx57kRDur4aQFYgFZhyEIO+REP/2Gk4Nv8+EZacQGTV86gZXukyYnaWMmn+OgKhy0OSBfAgKQx3rBSoaDRKyBh3gOXkVrtpstt3Zyd9lS6ly2UXbj8l0u+roW+/L9IA2xJZOQpxy6TS4MFBQQOXxcUj6NvLpM960SwZyVv46niH3uF09lZDaXbQfcSduVBopiceocwhl+oBZiIQWLje+iln0LC6jJbTfsufhnjFcG1PE7n8NVB/BM1j849kflUVWfBaz5K9Dr7fJvmIbVxarmeyUCu7GlzKX3k/vu6ErwF3ViEBgQlHtQUVrGa6qVGqvdPMRVr4bOxRlfAOvp+9n5YdTGabsz9TmIj5zhQ5LDi94pXH98VT+UnTRrq4DgYBoV1s2sEXQl/efmUFO40D+sxUtlTij0bnSY7TH0Qo9Tfbkn++Ln2sdLznV8Sf/7NsEAgHz2714/Dic42/sROZSi9I0kKi2LBYl/MIZu8WoQsKJXNhMasRtTDJ/fr+7hvquYMo9VHQ49dDdoaTG/TdejrfprAksemTVu/jymy10PJYwTVaNc08Sz44cT4nyOjerZhIfuZPk939iR+VQNscU0Fsq5Kz3QU48nEU/OfzFp5xffYQxV1eRcMuES4+VHN0YGjs8ACtYLWh9lVSogmlCj0kjYtG/PHYlPbXc/3gRZo2KIDsdI+V5kPchZbJ55DwcT2+3MYQktSM2f0GJ42QCmq+j7rmCyfkYj/vnIHWJpfnRRQQ6ASV2IxkccIDBAQf4PfcTPr/9O00WL9x+W0NzvwKQvWZ7aNppOj2+4cyk0yxr3kT9STsMy68wMnjH02CfXm6iy64VkUWE0CykXufKDQ4wMHQWgi74vDqeGm0OSWk5ZMTe5kDBSHKfTpI6HAqCCNo+G6tjEy6ODfiLzyLw/YGkEgMhJSa63OuJu6oguWkxgk/+OeuYpGZaXFpQyoTorVBlAl9zM3aKKg7nv0qjLpKKdmgJO8P92KO0KhZjf/Q3AJTKaqYq4bHOTLtdHgBj3aIYY87mfMkydtgpUVk60AulxBdcwzkYZNZrfDqyP3/mv4pAIGbEhRE4twXSObuFOS5mkLrCPAtYrSz8/RTJ1/vgnNLOHTPc6raHrsfQVUr8ww34X5mA29r+DPH5hm6LyTYH+U4BdSj3LF9wp3oy79xMYs5RP1wWxVJZvojL5YuQ25XRPBh+7YCy91eyVmRh4X4f3hs6mvL2WG7qfSh74U1S4vNZvP4IN3ucaJKMY+I+28H725UbGP7b56z7YyX++RbszJep7Qzl2/s/kt0wjNmTjvHy8pex0wdiaetHHiM5MMMRgNNoaXObj2huHtclRdwOfovFyrHQ6MQ1+zDKPnuPnJo3sXPppO/Mn3gsHUeU600Wxr1Fl1HJiSNDUJkU5HrkYTSbMOnUXPaBYqMFfWM+bvtnIXNswvfCEBpSCskauJGFVfvQW8FJVIKzQw1edSvxKm7AT5TMit4vMTlyG1fL51LJddSqJtJG6VnnAPWGavwdyhgVsoMs80L+6Oqi27GaSrvvSbk/kL66b8E4Bp6Av16XmOmpTqT1yEMKAkRUrYCPXeFSDxRhA1VW/P0aJkELJT2raDEoaDT3cCDTGdWxaZSsfsCZV35FaYrlOUktX4bBYwPEmBowORdxYdptcndc4Gp9HlYOsNYR0nWQ5K7n6852Tq64yYdPtIpjKmCoHSyMWch+cy/UanfqLp4k1biN2m5fvJXVeIrBbJUh9atDZxXhGO/Pg3sWPo9PArspELaSI6ZpXBdKMWnO4JIjxFDen73TKpnqoKHHpZliWRtSeym13VVPvykv13gyah5xKzGdxpA+6Ix6uuVFjJUpOe3/z5LpkJCCQdELs+wXprsa+CP2C/iXrFlzrSuB1+ZiyCqlf9UyvvD8g98C2vBVF+CgsScxPYnG6EYyxpXwyDGHkULb/vVc8TmOqt8jwb8vQUo3tLJ22tQP/ilY6ohzYSiJ5SHYz3GjNsKfZQkbkB39GoEik3tpauY85wo75TQGJKOR2QCmVB9jYMQzVB1z5sRbVs6NvkyovYbVQI/f8yhSPub6zSmk/5SMv5OUvMGrECXLEHddhIp9yIEtQ+vQmh04VH+ELscQhDEBqCKGPa1ahdNGUotmomg+QmlMIPXdg/B38oISm47hA8VbNAf04ULaV1T332+76dJImGfFqDUiNhjpdoFCf1tHbmmoZNywQlL3RSCxWjEazDi3OKOT6xBIHMAhCoFETVR+NCl3+5OZ3G0DuCKAwcdApMCiceHYvM/5s3AIV9vGMdz7H6pt7Lw53T2acwu+w1mewuzu51BaDNAvDaqPQ3sOrYYAFPNq6bifx9dvLka9oJTof0pAaO5GMPABlyOKMUht/gDhv9YsgUCAd603Gkcje7I/JtzzMWRuxAVomfcX18u9SPQNRzf6Cmq/RhZ6D4Pj0E85Gf9d01AFCjD1q2FWYBVdAkdUTmHQep/59o5EyaCgW8/ko5Np8qmFpB5bhnRrOt+2jScRI6H6QM6pnLCXFUHOOxD5MkhUzEpNIEMPiss/kqX3ZgpHbRWO2gA91Yx33AYJHZzKGk9i7VyiA2L5t512vcTdzFi+8i4jKbqALKsXozWP0ETV8UGbbd9kGr4MTVsuBtPv9PM9wWPTMnxkf3K3/CpRLnM4/VkaN6NK0c7VsXvxbtb3W/+0fHOPHlVLPeU+fyPU7GDArVHM232IljAfpvWcQ9pci9WoZfAPczm1+BssNRb8mt1x9G1Ab6fkf3E+UKvV/2fP/l+w73/2f2bC9oeo2s7b/qEKBqEUZWYOCoOGdfrZjHCA7n7PcqlsMQAVrj+R4wweOiVrqwcT6tOGwKRh5+RAAA6KrZQlTkfS8QFrvltD+pIGNppEOMsg1wBpsjB6dQ5DNPASb3rCMZLI1MFJbzjV9RiRdCiTXjvP/r9GE5gTy7p5N8EpARQ25FI80xHkhBPVJKfxWiBVm/YQuvItdt4REnnqd8QOHhSnGrk/9Cc+CJqK443JELICVUsV/suO8lDmgMIsZqzKTG6zTevHYrXwh28H5jl/0mHvjVQvRSCDN50hQgIvNcPShZ+xvzGII7nOpCna0Yk2E5kaSOSNGaAdyLCCCNILfMnz9MZg745Z4sWD2nE8WWX5VeNLpaCU4UOa2T3g/X/6W6JGIHFAE5dPk0MTD6Nc+D3oAkHCdrCA3KmHm33zGe8kRvr7Mbzs3PDr/hJr/13YBy9iRkYkpd7ePCxIIueCE8EvTKNPryMw8BAGvYjmpB00eVRxAynJinbUphoABvj/yfWmAJRSPVKTgvjseKyiR7ha67nUA99VxfFiyD1+CYDeVf7Iw10wB8YS0Wrrs3l+17DrcmdDVj+O3opEM9gTnVX5X6iROlEI70ffxU07gFmXBdzuEDDDtR/XHFfysHgvIqsCXWENb33yFmfHnOWKRyZpiRu4XxNAyqwx3O4ehKrnOcS+lU/L/LINjrdqGXM1G1O8jvIJNXTKVEwUiJCJtWy7s4sxKXOYFF/DJznD6A613Xe5JYyFDUYc62azLPZjugXQdTuF5noveBLsi/O4Qrq5miBZCXFJRu4pvLiqusqb6idRRodosFrxbVeQdiWVLOc2ut0MdFosoKtHbbUw4MQk2pMNmLWNiLWNaAwtoDVC/laWSfajKHuG9sDxBEhGAheQ00isxxV+zfwUQfB3AKTe9Mf9+iie//xrOL4N7Lxo9X2OqNxtuDW50VGk5aJRzNK0UzioiglPeYYLFaWsTHibjaUOrPx2JbmxuTYU0IPnoekmgpHX2Wh25lJhGDdm13Ju+F7e+k9Wl9WKW+Fcjsz6i6Xn9PSR23Soqgmk1nUE9h0leNXYo2vu5oUvX6AnPg9BmC0QonObzri2N6jM98W6/O0nFIBPrPhHYn3O8/P9C/g5O3Fp3kkGKnNxMNSDagXYh+Hu/jYlXrfx6IggsbiFiX499Fi9WX36EQazHU7yIjx99ZSJhNj12KGX6WnDQkvqISL8Ukmtf5H7b61kAH7IKk7j45aDyNIFs/UgFFPZWoLdzSt0GUN5/a+3OTnsAROXn2S7oZUDf83ivI8Y8yQ1Jx0Ps0wSzLcBTtAKI5p82fD5BpgKn8aKGFkDB0c8YOYAm4j28lc+Z/rJIEpjw8lMOE2tfzc/CrRM81ZRMWATy/e8jl2bDq2uBo9S+NmzH79Otp2KHlnWc+JwNIEVvmgdLBitYfQOvwqZ39v6zazlTP0mlI4X8A9y4Y7qNMmuNfDHd+A9DpHrAbbf/ZkucS8ozCftahrZcdm836pFk+1GaOYFqlSu1GV205mjJ6BpFYPDdxIuhU3vfotdcylVI3x5zXSRsEZnbv89FOTuqITPMv+Zk9zUChDtmoXwYT71z24kavQCABQ3sxjyQhY36mOZf3Uos/06sffdyWoHSLWDSwTx8V4TXo1hOOvTMdf6gesTGpe2DKQ+M9nvcRbdoGbM6pf5dOxguFgP2joYep7fm4v59pn3CKxbSb/cNxAYh0PEXbg2iYWzZLw0XE9/2RaSGjzoknaj4jqkneR2YwirHp7CTevAtMWXede3mBOWTZBu07tQezxAaz8TFM24JOURH1Fjc6x7DIXA+TzafYz3PgtHHfGY9du2cUYYA2Il//HS/9VWQUSTFNHSw3zQv4LzhlBW2gtwFFkp7oyiQJNJl+5nTNpLrLCHQmP1f62xM1zO4bC0k32iJt5qgZecauGIG2rBYt6evws/YTLzf03HrfwxdkRA4mcgcaDr0e8kfP8eX1ePYIimE61LiS3IogoGgZCvjg/H3C4l1aBGKrOnRDT46TNlFjVz/CrwMA/kR80jNEawCKSMCf2cSk0U1cYRFG5fRU+zmFPjTtHm1MYxoYQ/H22kor0X6wSwo2UYz6e7o/faxnQVOND5T6MEAtzuniaoMp87g4Q80ENuayxVFYs55/ACRnE7FqWQ3s9WMyN6K+iGgdrGF5vk00KLoJPuDisiAUgFFuo6g/FSl/JeykomVDgjNSiQ5P1GYI+JriovMiXZbGuDFgsky6C1R8G6W+NIjXxMmF8GyGFZqzdKp+/I2vAn0V2bWR5djKdLAxvzJ3JKdgK3qrV0NN9h66LxfF4wjol6d1JDjiERPvu0WalqJw47+3GhdBpp6u8ZpQTLtbFsMnTx4zhXDoRLGMw/umpBDUdBV0+T/St8fn4J0XFiypvf5O2cN/lmWxcBPV/CUR98LSe5f+oBvnFhJHXm0OVssemzPUGU5u74koo/OohTlWDu6sBv6N9QNw1i3wLHWB7FfcyD3tdYrAax9MnD7Xyg8Qq4DWCO6ir6Ph08V/w2byUsI0K9DYQSm/4j4K4TEvE4gs7aCop7z+RkQzwzJn+BUGhFouvE1ClGay1FalFw3+Mdxo07AsDiKE/ODJ5GeYAvF+xWEhggYpHVahuD/zo6tnn7oOvy45rrc0yfYoSSX1Bo0ml0b6RbqUMjcKdMM4gNANV/wR8CAt1XsvSZy3xd7MSar58nY9pNEodmQOsD8BwOUieu+RvY3wnzao1c1UJvOaAKBd8pRLeUMO3GKLQ6FWKjDo13GwaXFKRRL/HFG1FEX/0WZe8+PPtgNL5hVVgGRULoKjpkqYwf8hm3lKOxCi1IBSC0WmzZSR2FiEzNVMWMwdsb1jvrWO4ERosOAhdC7rtMd7iDvnwalep7XFNdwIAQSnbA/ecg7bRt3bUtrJyYK4B24I8n/zUul0NdFp7rvEykTx1DQhKJdGh/0p82EwqEPIp+EnC1wC2dK8S8SkT2DgaHtdM/x0hLLzF3vHazxNsVtDWgDMTaY3uu1WrhXpKNtUBnCkJ1bhh4DCEk/BGLd59kXP4C8gPg704bap87S0HuSUDQLnJN7bjVNNHaVYWT81I+veCPnaSDcvMMWtTPoeiWUtEaSr0lyAZEevwpuA0E77GscchngByWvf8iwZVy+uz6h0IRYKPOgi59KKdCvFg79Edi6r4AwTiGa6XUX0rDEK2hy74bhbIb8ROqLov3JDa8r2DQjUHIVpuoVsHJTi8mKANgpC0VddIZN/Sjs5l9ZD5R2WEI57fD1Cd0spd2cKikD0d7xDgE1JHo34rS2PS0TrmEU/aLJ+EtzoQDU17cD31H2Jw/bgMBWFPtRVujkenHFoDvIJwafoai78FjGFm6QZz/cTp+dQG0esdwubsvEwY/S6PJwqtbX0RZ5QmjhKSMs9EhowyA4p+QmVrZ/N4ihGYJ4pcvMyYgG4AlCa8Q4pzBMY0az6Iwqswt1H64lTA7EAlNbBtjo1FdV74K1v5KY/4Qfly/lIhFPYRFnGVU8GsEOORS2zkOu/6F5LalsPzHXF78wpe/7ljYkDqfLqMbhfoeFrQWk9qoJeVOf/akNNvmiycW4lFPr9cyebMSdnqARZoOph42219i4qeryPGXcmJYGzWiPNuev+U+XBnHaIdkVj97hoHStdSIv+FlL6joPIbIYssIP1JswvQwGq1XNaXCAZjthvOtWwWzneCuVsiu7evRZMGQTfcY5yBBZa7i+ZGrADhQ/g3fl42k2mDHpDOTaHJrQviMiKXHqhALjfy6AO76jCd36R3c6+2Y+tMw4p5LYfq/xmDJokOUz4X8Jn/SW2GvPAC6yxkV+A37xHu503iQixnb2JO/jR9/BOfsF/hmbC/WnH2M1iBH1n8H+Q6lXJf4MLbXJsjZjLRmN4WNm3HSPsJuzENGqG1ZHan+h8lvHkiYTM9k77tcK11JowlaRALG1M5izBNc5HudeSR7VXOudxNvr/6KQtMzfHTpJwa6fkk/n+NYjKn8tmoQ31eq+CLXj73uHSz6V5varUKOD7iKXqbnwKRs3IQiCF+LT/EBlMUzEBfmYb98AP8f9v4yzMqqf/+AP7t7uruZ7oGBgaG7u1sEFVEMbEW9DUxETBBFQAHp7q6hme7ujj2zZ/fzYnOLPs/75/fif6/jmBdzXPuKta51rfie5/c89xg3s0JzCYqnQ+yHKNtH0SCfRIQf/Bn6Ee3iAhLtTtJQqEEjbeFsfQBdOcXg2YbcauV2Lwz/702tVrb63SFJCesuGUm8MYCyOdfo49zMwdkSDuW/zGcuFmoNEzn8YwCujQre+fkz4Fu48geiwDeQ94hYcGIBtXH1XIi9jb/2H4FT+whmdsuISbpLWOpqaltmoDJpiBQIUegUzNkzh9beXOTdAuzbHbBazdBTC1nv8UtiFc86WElTPs3Yg5MQxdUin1CLRWBCb1Ih66whPOMsArMJ+tkCz+6qMkYFb6W8145FBg9+rnbgRqaRcRUr8HnjAKHOLQiFZtq7x7F3ybcENKxhwdbPMAeHoRjbTphLBm+lTyW78Rmmt3hTXuuE05FVRMbtJjg+A67+BOEv0eHXzYZ1G0jIWcq4q+m8MNEGKKIOwdkUiFGvx6nVmY2frcUYn8LnK4F021h+60IT2Y2JONSWEHE/kbo5T75XTHac3zcTQ1EIlT45WIQmeqX+lLXHEOiQxaGaGdT5ddIsSWTqoS8IDZKhVPzGsWSb5UF001Q6O2Bnv59wHQCFviAUQEW77Z0IEGCSmBB3tnHpURl/BHUz3cUOlbQTKyKKrFKOCJXU2uczSA6DlNVQcZISwVh+63xIPBaGyl9m19X5LJzoz+wo2ze96PIv+F++BUozyqenECTvoU27jWFKGAK8UVdH090RdCUYGLr0CF1uWu4ZuukjtXnMHbn7ISV7y9GGB1Kf5E6KpxBfnY1UdLN6Kjvq6xFXO9E/rpRKlRaVuYsp4V+xP+9VXL3TyDGd4HizCGXybXoSH6GXhPGx1PHvZk2oc+Pcz31RYCXOK4u3PeHnDmhqiCek4Gda76fjnm1Fam+hw9SI1CpCKICDDm34pV/Gz2MUszuWYSdSoxacA6DSZGtRAEeZM+fz3+E8kD99D4UGmF4HM3ucqbDYc2b2bMj9BIN9HAWGR+QaoL/AmQm35nPjixv8bLjHyHk9PPJZTo7n+8zRCDjTG8SCxMu4prnTr+8HZFaVIlMYbPLrwJkOObU6E4uctHTPiOdixVVM0s8Y5ZPFoOb17NHaYmLO600sNC1kx6IdMOBPzh59BQ9RNlarBW1DN9FZ0QwKbwVs8YJiSTy9JhVYQFQpglo38Gh80kc9RnI37x3sj12gOEHF3ahdDFDrcGrfjpNyEWqtmgE3B9A2vI38tGqKjEWMfkxYNpgNPIi8QZmfmXVJE/ixYCUDLCG4Z4RA7OvIVNEkVQUQcb0/xzMPcDJhFT8EmhFYjRyZHM0nrfBlxixS1HsoSkjin2vXVcFbyV4SzTarO/mOxZiV7bS7jcMhZAKIlextTGBUXQ9SoQCdnztCJ2xrRfehtPu+j+7YmzSLC8gW/YI0cTlz18B/9SqFlk42pRdwIv4T7N5cx5C2I8guvgkCHTj3Qxv6PL/nX6De8zpWLDiUQHswIFbT1N3EX2/8RZ/DTZRNFcNjUPWN4Hl4Ga/j2+TKgPMjKVHnsebbNdzqdwvB0hxovIK4cTCSdCf+isyn7905vHnmTTKeagCxBroKsYoD2JvzBgVt6SjunOd8yQQiFj5CZOkEr3FY2mSUhJQgb0sm6tpvhCYUgWUQuKWDx0hKbis4mAdCy3oatHKudkXwfM6n8OgNSD+C3ODI4X51vFuqJfLPmYgSchBadICNJO1iKmP5pk18mh9K6cZIWvr2Z/DLV6Ejm7cdZCwxORMeF8XByZGE2F/6+1196nGYjwveRGs28zBBysYWAStdTDZFgsjX8D13lIAbQ9jmaiHkYQI5BinWlDUIzg0Cr/GUhzZxO+o4L57+hk17XmD5l3vw8ciDvzQw5AT9smPovhvKmX6jcJSpqfH1xFuZZZPKd0zg8/p4muqKmLR/AE41Rlb/8jXw/N/Pd746hPItkxAOucIr/W4wzvIqr0nUXK9KhR1D8JncQ7UVKjuiefvCObZNDkCPI8NPhDN0TwQ9G+7hkBnFPXEvaZYm/vKAKtMTcqehtpnwm7/RODiX3PBSciJzmPnTNEQWEU1TahA11+JUfI2e2iDmn5xP3uU8zIYdTJq2ndjL6/F/Yuf9/2QRiUQEBwf/n93/f2Df/8r/WbHq6p5Me3lfonLIRmlIQt4mRH/LntPt45m5PA5PdTEioRFz+zhizu3FTuOAy8kbhI2+hb30CXtSov+RBamn+CXfnoaRlbh5uNJrkTHl8Xh1uaeJ5msOfBQ6giDdOV6IT0DcOpop8tNESEbQLU5jQ7WM7jGXGPP09/yhDyPBazw0XIKw55mqK8Bnv5GuYBeuDzvMPgNkBS1CeWsLSfcTafHWYY2NQWJ05lrDQfAGqg/TIU4nceBp5JVTuPfc2xjTH5H66g0mqCC65za/qcOoD89l8o0IFp95E+GaLOYP9CHIWI2LTIlHaQ86axkmVyF/yUvYJQsEJx8bO985he8uxRCY24e8Gf35echCPPQT+GnCH7bFDw8ZULuI1IwR5PU9w3st4OzwFGvSX4DeBqxKb1b6ZYGvgDjLBMrs2mm3JpGQvJjhTZ9RYQGDrhtpdgHt/W/h3H0K2m0z/zbvGtp9CnC99wxiOwu3mUNyRDCU70QQ8CIrnUrBqZRRUkdGPZZAzG0awGvnrhMVaaHi6hra7SqY/s1wRstEdCLmBVcY2SLAaHyW/6ytI94niX3+U5kYDEcvryXF6yhtvZ6snatkZ7UGdZ077po9jAi8iKj3bcBmHP3fzDKrwMx3q78jWgrC3ldIkmhZcuJjugplWFaZKQotot2hnQiZCHqqEOgtrPYsIbdGhM7xV2rNfSlM7GHUfiVtFkh0g4Mz/0ImGcQvLmpCnB+SJ3yBbQ8+o1YbwkdVIZgbRITWTGVvZSi+Q8N5tjCXGpctKMUtDM/34N7ORZzpf51p067+3XdPFz+FNXk0fcXJhMd2k1H8HS85wFR1Bww6AG6DwdzDqeGX2R7ykFtVM5l0bRA/js+Gg16c8hPw9Iogeh2Xc0NwmDLFfqTVStJ846BgIx0tfejIFGENamFawRzyzVWEJ+cT6nQHpaSTOcXBSCvcKHHRYd//LjLFY5a5zA2L1ImyBfu4oxeibw5DqGyi0mEmMV27wdhJpzSOp7+Yil1ZADKBGU+LBA75gsILrCYQSjk24CpZyVep0c6gRlRLljmAOFE5PHyNOzoTxXuH0Gb6GNe8p6jtqaKr35cglXBzwE3yYs38GfoCRpmCIPlj6YKkb9ALoglJe41G53hOZDzPhOZH6LrlKFS9YOwkuPsv+l4ZgMzZgl3ie6Qo38ax/SNomQr6ZpqFg/kjewwu0odMOjCf4bPPMTB6BUNmrSCvqT/XVTc4HAryhm289tlrHFr6Gw9GlEPJWxBZQKXbd1S17UTs401dWjjhyd9hFQWAyBYRP9O6isE16TTXaXELleH1WHveaJBQfT8cudiE0WTAQQjf+Zfa6hX+Mtm1sYQMuow5eD5DzmRiseoQjnyyQQgwFXA11xtliAyHhYdxM3vQHvk7dbnfYM3bxOLOfCqvD8Aef1RVa+h8rhmUGQD4D3+agd+X4PowgXaHToRmMGMEhxhI3Q6OsRzok0BLzEPW+u+iQPYW2/6b2KfwRiBRc/9iAu4dhSie8+B7v9fpsO8gRAKTBpQy1HyWCMPLXHjzMJUXKxBMHsNeLTwyQKBRjdLSi7ahgcnZFt7sfxsabeBUjPgayRowWa3cDnekxRpFkqgLdJ1QshX79ibGh31PTk8wioy+tDXWwei7vPa4O9ywmkm+m4x7oxtwjJv+LayNffzcHXm4RoVgse6m1x667E8gk9Uj0NWBQAgOMQgEtownea+AiOvbkPZ6wAwPMLQyx17PD10gabFg2vAMtydfYdisC3BvLQLBRGIyY1Ab3bgU30tyJSyLiYUhp6DhAvkF+5g1aBVf1iewKe48DlYVt809cH8tBK8gT+dDWK0VXJzJ0Xni5joVepvg3hpw7kdeXClHEg8xWQXndWAUqvnR3QbSHBRl09ydRfolJ/TyiUzXDiZ0cBx9/zHHWiRC3Ac9wHVfCoXXB7B9tZEXHQ4SI7z+31mYnpi+1NmHo+45D3m2wKjJOonJdRAmkrHgzd9pNQttknuPi3+RK4NOpnNk3jF2rT5OqmIh85jLj+njuFw+l3k7mzDrjExoeguZMRzRYDNmi4S6rhD0UnuynGaAXRN3+q4HQCAQ89tDG0C6Bqh6dIoxez2A1agdugje1vakTlYLluhMmu2ltN/qw4tqBclOKlYmraa7yw5dTjjtgjLuk8lUqwBRw0UbeGrRs82th+xeMZsfCIjfvoqfp9fyqPQwL6YuplzrwbyTXthVeyNdfpj5ql786+rINsDaZnhW0MusL1+hIbiF8gf+tNSm81L86wAY1Bl4SUbg79SNWW9kb97LrEldS3PjXFwEiThpB9OEmE+boLCqAu1vfemdMZGpTz/JCpCY6rjziTcq+9vsuDuNrPhi1ixwRtp4kZCQWspFzaTXaBl8fTBZMVkM7X0cQNE7E31xI83VVhzUY4gRXcct/yR0XQOxCoF7CDnpyZhdi5m4fyKZCXlw7yWw6CHlO0pUc2kMyaXO5xT+mlucw5/oK5Mg/CVI/JK3XTopNcPFYnfW1FuZN/VLUp3abH00cBG3rZu5eNVAqd9lvva9j2tzGl+vOW/7toCtI0/RNPwQv72+mtFFEcTIzlJfX4+7aClx51LJ6nLgyLSz9IobmSp4Hir3Q+G31DopKRiRSaH1Kue8FqKyeLGlejNcnQbJm/mqeAId+RouJdbT5V5PtFQBZkfIWo/Gdz6rlp3itFbKpfLrjNFo+WfpUoTxp+9P3DO6EX8rgV+7JCSKVZD/JVTuhcnlrMt3pabKkRcvDMI5tJasT4cRk/wKAHFN13B5bjdf3tjJB6k2JnXDwAbcVG6cNk8kURpAXYcd9Sf7k6iX4CV1h+gf0NZp+crpD2qHlGJ/1YMtX7xM19tezJ6QCuW7UFWv5d3xZznamozjkfcod7vJgDHBYO6F0OfwvbeN5ZYC9udMZOhv71I8PhOe9rV58uVv5DldA3O+fZF6Qz0bD7/I4rd/w9Gt3Vbp6kMIBaFM2z8NgAkv/8EIhfBf7eJEK+kKaCqeyMDMOcRNdIDLk3CinYm5GuRdAgw+QpzcSnnJtRSuzYaRV6nY8R7X6mLZJeuiLPRtnEWgMz0DLbeg5RZbuqbwZaeF6t57FJxOxexntoE0UicIWsLUyN10epvYkfkRAOOk8KDeJlfp4QFDH8aTeH4w8Dv13kr0vZNQ5Hxse+jZvRxqDOPU3jQ8a1R4+9Yx1LwbeAKk52S/SMuhSlzGuBLV77HfoCqQz2XuZM+vZuWgGdhfUuGCmi6lF5qYdVgCFlLvsZOs6CzU1X2J2fccx8eVM2G6CPK+hMCFf1/fOSWNp2d9SIfDE8dUg9iFk0fSMMiM/L76e86K4beQbQiObQBgeVA010buwq3RjbflCpzcWyHlGFweDwIJjM8iPOdnBLvuALX0i3gJ8k/aLu49EQE3MUgNdNvrqIucgVQKWPbhnPMh4jtxDL2XgGC4L2U6ewIVHbYx8P7L+CoiyYzwR2K2o6hXQ2Il3PcDB3kTE8M2I2seQOWmG2S0RHKuA37qgFbZE1O0Qqsfp10q8PO/hEvpUC6brIzqXMMotzdI8jrB8vpgMlPuMuakkuDiBkzd67hcMY/LFfMID4e+lrskZThhFgm4Nn4Us73+CYnBKx0juNqxA68WZ4ZaVPiEnoK9KvpL4FrvZHwFScAp4LHUnUgB9lGM776LKQSW1DbzrBMohBBes4o492AiXK/zW74XA8+nIBij57MLOxk+Too3RykxwMyCATj0OY+zowPDdk2hvcNA6jE3jmY8z5CAndT1BnHmjB7POicS6oKRhJVhp7bipSmix2iPTKahn+JZjE3P4pq3Fs8aJebV6n/Va2NDHHqDlehsV4wSJ0QTheAxnNTrAxm+JZ2W8TnQsJMqz04cJJ8TGSVGYvXAufwRqvJcqiJN2LnALm0gY2PegYC59GiFXB1yyDZ/aIZwQlDCT65TaX6gZtc0F3bk/8DizTl0C2uQbnud/yRmc+jVCmS6LL6/8z3xQVtwvDaQ49J7jMgeSqSXHwIBLI1/hYyayQhEQ/m5AzJz/VmwayYHph/9V52EAgF3+tq0BFOrwMkUQemQdPTltxCWNiMuz8S6ZAAqSTtqYaXN2y7zHQI11dhJ/4NYoGaQspt5TjCnYSwHu1/ik2v7aXL4lLG3DWT2zSTu/a0stnt8Q48RUH+WG0Uh3L48iEuBp6h9sZYBMh19ALHQhJemiNKMaO7ujsE1VoirT/6TB64+jNB7KSM1tXhUJ9Pl20WEFLzlXf+qV5NHG8f8HpIkcqFUUUBX91wWdIzkaU8d12fuZ+6AqWSEnsXV9Qcum4BDtnErTQlj63ejCIrB/X4ilVIp8aMu8MyFe4iFBmYGj+WOdjyORa1MqZ1CwMQO3hoxCBdlDbfqhvD5joHYPYrGIVqHqltPoyyMorYu9ma/i2vIRZJk0CkwYXFWMzziO5zk6Wx/+Ak9Jg1pAYWkvLqdrWVRjPtkHrurnuetp7uh5AC4DmSKhxvmyki8q+yJyhxG4qRHEPEqmLSsnvUc691OYPJVo1NY6JHVQk+NzU/KZwoHf5uDMvcu5U83kTfuJ4Y6vsc76RPp632Mr25u58Zf03FoLOD6e6dwkhiwCr/m02v7CHG6hyJgOJWRbkh72oi5cRa5KRrnyEsYQmxjyxWtJ1m3whG7NTI3tBXh463OzmnuPHc8G5W5lVkd7mjzwuh7bjgnJ11lzv6HAKxcCRcb/qTQ9WtCikNYcHI5vvNuw7CviLv9C0ElQaiFVpxE9gQXW3C2erEv7zXOli6jXufOmRVbsLMGsWbHNCStDhjHuBBfAYMUkCvw5sYbv4DAgtGxFqExFKv2BCPlMEsD86o3EdoYgmHwNb6ftJnbhq9488Klv/uQF3sZeGwwHec72OhTzcv7PNGblGj1TrgKxAisAt75zztkR2Wzb+Y+zOYxIHoiuX2ardTFV1KVtpb3BtgIGyYrLBY28UtRHXVFj/jz2XO0CLP4TDCWdU6w3h2G0MzFwEe84LSCudpzeLhX8aXhY8ZUzwDgpXuJyB/1IavkDknH/iRn6HNc1UGbBSpM0PHbNPx67LihTEIq/IGARQEsLxzLqR6QWHrRVlRiMViwTDiHX59K9lVHkRYEczVWrkptTBmL1UJO+x3y/dbSZRah0RZD7Uk8ylTM3PsCXSv1hAVNxWxM5jXZTBzLM4i6Oph39w6nfkw9KndXOrQ2e54H1dd4rXsjsoepPMp0pD2tgxn7ZxA3/yRE1vKRNZ4yy0ikV/9C4BFK0K0oGiqCmDOtL3/MXIswaBFErsNf2sKuk87UhJaQEZ7BZJknTaE7yd49GLNvHHtXP2LuirlYdtjq8F/J7X/GkiyPvTktCLGKlSCS4ZT9Jt/OySAiJpOX765nwf7p1K44h0+AjdwecHYIxY72WPvAxeg+KKUdXK4ZxWDvM3xU8Dz1wd9SKpVhrbNy2BMcGk9A7OuP5zchm5/fzFDVs3iUrcRJWoVRswXJ8AsYm+DZR8+ScPEc8a2lXJi2h6MVDXiETsDPMQghFlLUJnIsJr5bcI6NIUVIrHLorQVdLXqP8UzLP0fZ+VSKwxwoCA6hI7gRe6cE8pvz2diykej4fgQWhjL0+Bv8Of8iH4f+AY/+4D3jOKrzw9C36Lgx4D413q0ICzb93XePxrcyuLoVs0VClV8NErkdFHwDlXuQxBzl6OkFOHRW4VJ0A0FcKeWahQR3bYWgZQwt+IrXA+C1zjYq7kVQSyRpXYVwPBKi3+FCrT36rmIq/co4New0A6XPgunx+F2wEUvgFlY1Qm6blOH5oQwKa0Bk7gIcAIh2HUlW3hikNVPQStooMMthfBbm/J+4/IqItKpE4t0SedYvHjpsJHjso8lsaEf0xcdUCJ8h+tNB1LQOQ791BwqpE8R/yq7fWgg46AMj2/n4jY+xCC0I5P8B/7kQsID0Gy8jlkG7nyO6xgiOCdaxyjEN2h5AzTHeue9N4u1Y3KZW8N7w5/GuyrLdO+FzcEomr8GHkZuG0asyo4qXUKSfifM/5qxNglB0s/bQ4txu67cIQOXLicpzaPK+wBDRgdZ8BEFbHdY+g1h0sJaxY8xYVeep8a5B7idn/fr1YIWvlF9TeUpLwsJRf19f5GxPZdQYanwzqPesx6PeA5HFpqBhd+c8jekzsL9znpyfruH1bj/Geq3hnFaKUwm4Wf+9z/h/sVgsFhobG3Fzc0Mo/P9/e/wP7Ptf+T8rVpc0ehoykad8hCj3Y8w95Xz16RHea3CiOSOXxmoTE1Te/DzRtvo7kPsSWR/ZIQvTUN53BOcV4xnr6U2O8yEs6jDazj+D62UNdsG3+X5JPge0sRQXj6ZNtYXyrgDyqkdSf6OQtIIu6jwccIqtpyzpNB/Ve1JtjudB23lyvVZjFpn5ptOKe7cvn3dkQ/1ZwEqwyzmEE4UcFn9IuWs1vho1ZK7nXd7n5CdKXmhIY0rhakR1T3FuxBcc9/JifPx67hwz8VpFNsnicFy9Y7luSqPbo5CZ1fdZZAcFHSvJru5E7tBB3OD79DjDpaC3uX5nFY0KD/xuOuNZ58m5MbbBX4AArs0C1zQIWcHRkZcRDj1Ll+t6hCoxylYjSk0xFoltKhjX3UxppgVpbz/Ol1uJWxwBpxLBcyyCuJ9tL0NgxdQlY3yxnGdjnyZBqmBVtwnDpuewTu7gk9c/wSwyU+D+K33yl4LEjp0dCvRiLfF9TnBn/G8UGJ2h2gm6ipCIn/hVLLCMYKbmL3Ij7/HaO4kA1AnuEX7zBBVBFhyMMajlLVw0B7C/ASrkDxDaL6TL/CfqNiti/Wl6G+IAD+7U2kybrfI2Ng8+h6T/Zd785E1E6V3oBufzX7AvSKTl4z+XUauS0jH/Fr97gPXmImS9dTiKfek2eGD2VLB77m5CJeAuAMp3oVd8yCurX8DkZEIxtJ0+zvkoeoUcmfiQxpoOWuo/ZHZkDnPuhXDyp5GMGtJJwqAZ4DGStl5PyivcGLFjKD2qDvK6PUmP2cBnoQq+04NAXcamFjW+nnVUB5YzOdEGdFisAv7Mfo/ZKWI+qU1E9bovcZabqEKWErbuD/C1ybpiNrC7147rjo0o7wuJPTccfWoluLUhFFqxiyqhzWrFWv/YS0IgBPtoGH6BpcWvce+Nj1ls/hXvQ1VUBvnSnbqOstx8Un0O0XvVm4LrSdQvGcRtsQ9X3BoYO8LFZmxdXcqlSC2b2mG3fTVmVT2eXZegpxoevIyDx8/UC0yY7Dr5ddkvTFYKoMcKcR//HRA7V/Qlvb2Q5jSEh+FHkEjKbZkfDnEcrVBjd8yKW2gnVZdEbHF+iXVh6/AK/oBhlRcwdjmgVrtQ0fdjNjyWb0FXT41qMOPsqxD2/Zk5d4ZSdnE4hf0biZt1EBovY62sI0B/g94OT/StvjjbFdrOvWnjJovb7xN16QbGkAjcx1kIjS/CYLVDKuikVefFn8bnuR57AP8Kb3ot7nRrHi8ouwqh9jRm8Wgepb9IVBRkaD/nc10kC9UDiW26AQ6x/Bx5C/22C3xcv5viG7Nos44gaZIzB6+2cGR+Cf4KT1wKi+l7ZQIL+zdj17qF7+ZFU/sQDtovYmoE9Nt1DAeM+JqK4fJ6aLlNsccFymdpkMcX85YT6C1t3O81Iq86TpJCRMdgR0zRebRe+xSXhiISRRUQsAAcYhE6RvL2slNkzTvDvEpPdNZmZgmGQnsWyG107w/da6gXQFeWlhd/eZGyOafpP96EoN/PWPOtSHtqkFcXY5Ym0ehuY0+e6YEWWSdloeWEFKnpu7Ivnb6RtBtvU2wEsxVc5+3lelsY85r88TszjCY3A8Fhtm+gyuU5pn2/G2W3ipbkz/hj5BEUbYlQOAZyPqZaNptFf6bQIz2L4ql2joXbxsJ6tzFU+q/AmplN2ZKNXGr2Q2/swugihVlauLvaBpRbLUhFeiRGE8amz+ifIaJgRj+U6gBQeLDYtI+xef35qlFGSUwSw+PPQ0cuDDvH5I9XIcl3RxhpxTTuAoFRj7N9S7cxSpFL9dXBdAskXFYb0JkiMcRpwGs0eI1G+mgEpa+tITmmgl+mg8QsBpEKUr4HhRcGlxt8vu5zwipf4quzWxk1VcXw4g1QsRtqjrPQMZAQJbx1M4B1OgWvrPRnSVEyUz3uk9UTQ4y5kqEv72bHD9NIO+KD55hn/zXH7ukezqHOS0yq8yQiL4IMowUmfErFkdfYXmTPDqM790NOUOn7iA21jyUe3YZQVedmC5TYSYgRgItARnl3FTTfhPZMpsRn021XQrezgigpOAh7WTj8ODRCUJgC1x+v4lSVibCPAC+rAjtTHesvn0Ah1pI6yECrTxx6cSNjlKARglD4BMwWCEDm1E11TDaatgiECm8CPGb9fdxsMfPl/P+wrwtkr7xKm6MXv3xqY2uHZwaz6/QYeu1q6GrtpswURIjoWbi9AkQKPvt5EnYZcWgmVmFnluAq7OIRsPHWdqSqR6wftZAzdSU0BdUS4tEKFlALQGuFJt0AvBxEGOVJ5KRPx85dyTqxA06Nq9jh00W57hGDLK5MaTjKrSoHTotHkCxO4OTAedyovodF9gVFv08k+X4S5X08udsdw2Sr5e9gQo9mOHWaFowiKSGPjJxVdLMy9Gl6mx6ytbMNq9WKoLKLoZeGUutVy9qmVr6O6I+i9T7KEdl0aZzwvSVC1NOJosMm30vEOuyLilky7Bh/1Ezh94W/Y1LLoOCxT0bbA3C/QW24BzmBf3HU+SrzRINsRKLin6H0d+Z0unLvwXg8Dkfh5V1DavxCKMPmW+mUTJVBj3LXJpKCBfwWBAkSXzgcYMumHHmNfe3e1AtL2LHsTz506yLLPBq7+nr6qW5iHi7gmvfAv9+tb/s+yP8AgHea+3DMcp1RvWP54N5wFEGNtoxCgLurca+fQ+P9QLyH/c4vCSfZ1jsDrP+BpG8RaaIYV7EbXWs6nnvDcZZYYMP34D0JYj+gucrAV+0gcGilbrqQid7+4OoAEnuwCweBkKv3+jDmxAgA9C5aND1FULrdJm0WuJAfqgZSJU8mpegYEokVe2Mz/OHO1pEKIvvoeCfoMKrFIUTF7kGAzdPVhJDtklMggYmKZbQ5GhCLvG3qETEfUFdjoOO6P5YWI423VdSLx5Imc4WGQvCdRuXlVu4c8ELp60powkOGB4jBcxQofUDfTI1VjUjTjbbXQJnYyMe6QXw+Yzs0XAS7SNybDhCrs6PeBN+3w01dMB/9Y7wYILyN46cv0a5VITIWMShpP9RkYY+MtI2v0O0ipWyVhWGKxyf01sPxKCz65zn3UzCqgW2MPrsQR6kZFgG+06DqIOVaVyor/AhrGMSjEx40jfe1ebC0PwL7KGKdfsRip+dy/SBGBvyBSLcKV6U3QwN2UqYfzQupJdxRm2nJC2Rqn2raBLNxcOlvG48sRu52eTLyYThFkVIUaZPIV04g+R/1qvGfTUl6FyKxkI23fmX2fA2eRTOY57yOLx5swNcVlm2ZhiGgFU3Mt5DxFNKMp3ht4CTuBLpTXy3DS2Klr7oe6s9B5ttg6mZV7ixEhXK6ElI43fEC8epBkP8N3H+ReL8f2BL7HAKZkRzD9+QYQCFVIzW6IbRKcZK6UxBeQEF4AX7+0GZV4iaSgVAOLv0BcPaOJd/fiiamAaeB17DE/gehKgBEMqKk9ngNj0DoO5AH1UNJtW9CV52MrKeCivQ6LkZl8lLTU+z//DnGTD9CNG42b0e7Kfi4edFjtGfAgZdRN0fDlz/93VadLhE8xQ2ut+kJzw6n064TkVjK8ycf4ihvAL97CCwCqnyr2L5kO4ny6XR0B/L8yUdYLCIqw96j1nk/jwbdQhTux91WV8A2L1mtoDA1Mywrni6FnlGrfmaw+19gbv2bJCXVihhycQgJDxL4tdOeoB8zWKg5SZl4FVf6fcKQITD0t8M41SQh+FUADtEw4hJ7v5uJ5kIdEpkD9aK5NEafx5i+kQTPM0yP+JxvBXFs9L/OqEevk3DpM2oDUjjhOJdT7adwNoupirwEkRDcuBqVrAccxvPz/an8fH8TAWFaZPqfUPYoGbFpA75iKwrpZjaPi0Wv6Y9YfAMfwXkmedxmq2cud+0zmM3n/5qLF5aepr64C7/S73CWGHDRFYB4HA2IqfCvwM/eheRDgTil3Ud4NIRQgJHXif7xEK29MrSPHIk+9w5RT1+AqzPBKQmLz6v8OuBlLvTAowfNaBudUT4DxS2XqWiPpsVixe/OAbrsxdR419Bp302L2wt4VTxFvc6duy0p+O/1wjPuIY8EmVjkFSz33USrzotdmR+SIlGx5Xgy6VfS6Rl7gdVxlf+qk9TSye/ucK5BRWHGLEQ+juA5GrPxeQJm7+R601qs5w/S99JNwhN3w9TzcP9lrJ1lDHn4Js1FHiSkeBIudoHBD5GLbJk6YwPuMemdWzzUhtLb60mHqhH7hPVQvAVa7pJR+RIJdyzIPcsIcOtgpBrae1359vZWbtdMQtVWTfzgbJ76MZk/zriwOzuIOdH/Ac+xWL0nMybwRSZvO8uCDfNZ8uWLTP9s25NKGbWsEKaR3zkUYZ0Qlw4rvZ5F0ObK595GpguzkAQ5MyD7e9SNS8GnL9hHQdRb7P1dQOdNPXJpHXvf+YRkpQU39zxe7r8AD3UJWzpcUH3xKSVHRzLyVBoTx+7GRVlDWVssu/JXcyz+FZz8Kmn17ku5x2aohThFMi/0W8ZdkzeaW4msr+/mr1VvkebZjUvAMtyGzsXTE2pUJxl3dhP2ehnZQ5/HKhRhUjw2w5XYsab9JHHdk/jGdSrWFd+yp20UT/d5AWpPsrX0Ex7q5Pgo7/Bw3ctITUGg3w9lv0Prfa75ORGk9yPONZMt4Xp+MhQiEtj8H+3lTZQk2+SUl6i/5CtPAw8N2dR2TaO2K4wRAZ24q8rQaQU41Och77TiK2xHIoDVDpCRL2bOnjkURuRhLK9GO/YhantbPxgZ/AuuOinpe2ZQp5UjeXkL6zx6+fXyk9c1tvcG/hvXYhaZqFL3UNQewqim66T0lrJox2uUxhjROz4g5G4Gfp1qdjS+jL2skSCnE/T36SZD28nd4DIcvMVEiSbzyGAjJ07rXsmIgj20OB/ngGQCHmZ7trudYpQcnEtg3MBc7g34g6oGe96/44sk4h92C0BITCkNrlkIjk/AvTkaF4dI3rpwAWdlDcGWbqYoMrCkOnFT04hKq0KskPzrfKO9J/U+nhSobe1caxJztNtEpqqOE29s4NmQ8xgrjHjJDDzfO4NCeQiHa2OYq6nkRRVcL9Vy6rNBSAd54rDQphCk7A2mvwTKTVLKNeWIIhuY0W85cxohXAJeIriTcgdHHPHZ6oPKXUXYtCJ+sok0UFLzK8HjrhP27POMvHgNqZ0dX4XMop4kqgZ+iN31Ht4oSKC5zhtzRSvpV6czYXI9lyOvQ8MlhgYU4jm6nALvQZg7tIzx/AZHi42Ieqr+RQxuOhQOClgTyJGcIwDoi36iS+hETX4kAUVKVCFq9szaQ7lXN8P7/sVbree5nv8rh9uX0qUWk7Z2OFu/7yVC3oKwMwd6G7DcXEJE5XaEiTtQdY1gcO1e3hg8i+7WI1is84mr3IGfHwQ4G5jx3UjyvdPwev4Jcdyvwo/YrHBMnQ2klqdSHG6iOeUOnp6g17Xwed5FWl30GCvNeNR78GJpPNdfvEPS8wcYca4CR18rVvcOph8ZjznpIYPHnYHkzZy/7slF4bfEO8lZ9stc7nm3EbJ2j20dCiz0vEuoC2xu66HP9wG4DWjGsqQWWu8g1crAKqA0qJQeFzGx5fuYeraEbs8cGPgTVokD3tlOOLTLMDh0MtHSzK9Bv7PE3Q3Kfscx/1MGqCMolRhJueWOb0MaP47z5LUBq3HP+Q6iijgXAP1vhiLWVaM2xtGtlqHSXmZZ+glmNe5h+Ro1v+WNIqgXBMlHoXIPFvexrDmyiGKDhYkDnuZyiw9jA4BK2x5P1HqWHlMvIUVe6NdsZ15IMy4dj2MrZwcQYoln+1PvYB9dxIdvf4jKGMKb0jm249kfMrSnH9WXx6IfVsrz/cFkyYTYK+CcCgpPrD3OlP41kgH3Ezn0+pfccuhhn/QJLGaRh/LNvs10yxtZ98UAbveKoSEEzFpa7rrg23yKmluuNL36DsWTgrnvsYlZc8fy0oY3Oa//lPFOMjoDmnH07qBb4IBD6z24NodKV3eOP/sdsXaLia38C2dND9SehLDnwDWNFcZXkOT0Z6uTkpqYOAar7kG/LSAQg8yZ4yOCOD7iLCPqjiCo6/wvNglZ70PEK+hVFs4NP4fSfhIi42pMXWJS/zFeSBR25IS2MuZiEn2LZqB4x0ZqE6iU3Bv3NqNfFnB7/uvE1IsInHYYB3ksMxQbCOgr5TX/hzj2e4EPqmGgArotQ7l6qAPNyFj+m4smtlfTGNgPszIROMn54ecJKg3Ct8oXUewCzBI5JqmS+uul7BhdSr8pafhN92Dars/ROgbS7fL/tmef1Wqlvr4eV1fX/5P7/w/s+1/5PytWr/HUNksJbrkNPpO41xNNxq3J3OqRYfE6T62qidqeKdAwhFj3S7TovAidWUGpOI2arjQc1IAEokZPBmDDn5GEXnDHbIpjgySGkIRQ3nDcymh/+CNvEmVR07g93oxH7isEVfhjFki5WjsCY80owoNHgelzXt/wOo0+1RwbeQaBI6DwgQG7QCTjAF9yRdOIm6MXo9pnMt7xDGS/D8BpYQ8GmQlxYQ7JOQrODhGzs8PA+IzlLBF+i9emVchSYpFOakWpaAepHe+2wMZ2+KL0GncOhFPQ72V+0fgw2QsQbGVRA6T7eRFb5o13fh+CO9xw0UmxH1wBpu6/fVZcnLvQG/QknRrNRO8msn7eCplvIwyzpXjH98tgTOI13vlyKYMvDyZqyW0bE9xjOFJTC1/eGUFdvj+iOg8aXTQ4xraBzJe3mvzQWEFhtaCX2+4ltj6WX8xaT+LdoYTl+5E7Q0CdCaKkLeA6EbqKELfe5Mubw2l5EE6OIIq3hUNYfaSaF1O/4VbVVBqVObRPOUmj2J19Bw/j6KNGOOFXpqrhC1e472ZHbvpKRM3fk3CrCUnTesKcN6KRtnKvbuzfclVWgZXC8Hyuqet4Wx7/d9/qZ+fLpd6HYPWn/53BlAZWETAqAnNLNpV9RlPuO4QwuS24HSH9+zSCzA/Y5y3ArFGRdO19VE0CfEd9je/j4wcEw9jZCR06BT35FuoCPIhNzybW/QKhbpn89TCBS+NOoupegnNXGg6unzJRVc9EYLtmB6+qYzgR+Lutv7bCa06gM2rw0hQhsmi4YxbgEVZAaFEYDXXORBXEU5H3JXRXQPImXq8JpNPcTWyfbuR9/mKvvZp186wM+fJpLmu3MFljwaNChdIciShZCFJ7cB9KrVWBQWbAIJUgWdiH7gABWvFSkrxikQj17I//CKOnDGfrEFr0QkzSMBAWwwFXHL2f4qUSNXndYka49/CBO9SKHEAoBZ+pGMwOnJ58FKvZtlnJNwgeS8DF/t2uMdVLMda34tF6j0+K3uSNXz5GOuBzCJyPMOtDflj1PW6Gqfg1TGRexFIUphIUXbn4NI7B3NCEvqMXkLM1axsvJE6Hh+sIE3+KVtGK0CKlJK0v2vg9RLk12RhgI65SKbFn1AdvsKtuEa6b91L80JmhAxxschHOqRjqvsJF40Shg5bjwvd5WP4+qzzfIF7yKdsffYpX6KvMvx3GJZ86jk20sZdH1cAZb6D+DAZjJCr7SnqsAdSqD3JEcpMFWjs4+xGMvsOxHg8aTaVYrGZ6ZYV8NjwFy8M5RDr8iVdbEvaiRoQ1WSTeTeZA4EMCe/tA5X6chL6YBTEYLSJcFvcnwXEXXlIplNpY1bnmO9yyqyO0S8CGv6bSHtpGv7As9reAQmDmdreUHnUVCweG8M3z820vwH0rBC9HoIdnW8W00Uu3ogedsJsiUX/6BXraAsI+kxhmMpNxdiSZ0h46Vd2832lm/kQbUCow1fHq25PZ0QVdll22TapGwbcBan6oLUeHFasVnAY7sTvvE7K6fgXg7YBolgmOk1Z+EaHPDFZ89AP2Tl0wow2kDjQ8qifxXhHuDe4cSM7FYIVenKFkKwglZMrH0/90PrW+7TQbvuOl1hp+2tyBhyYUD6EEn/JcXo9rY8mlb9jutwhH4WPdkT5rQeGJpfEyT5ne5ty5UWAeyuV0IfX9thLkaJOwaxc4cO/QQALcLfy2eDZNmh7myzwQm7TUl/gw7uwQCvys5A27QrODhsCU7yHvC5x6y1n5fBXv1/mQfKwvdq0RiJZabAx2q5UzktGUJF+hV+XDs7/vxuTrYQtqVh+BxisMlg/nBQ1sv2Kkz4VvMMQuhFnpELQUwlYT9d0C+pRFMeZkOvYd9ghWQcvdO2wFUlKgTbKT11xL6Jhwj7iyl9CIKvlbcwUQ98ALG18gNzaLEVvfJ1w4FOw+QISJhmNDCbfv5ei801TaHcXd+GQcVPeYWLRjEVXJNTgohjEloh4OP7numFAFhOm4diuNl+8OITeuFocBX2PVr0Qj86D6wEjqat2xDivAzrkChXAE6f67WZu6hMPtR5g2egGVRj3Kz54jOL4IJvxracCUmHssHJTL8Buvo6xYy5h/rFQFAgGrG6HECPJpRaQ5V3GjPp145/u4BNZQ8NwveNp/QsLHw1irzeBobIntRO9JXHWvxSf2ES1B8TTGXGBxoANHU2xzyfj9pewKqibbU8mlFnvE4lZe8QplmLKIpXaQVZfOrdhlaOysdHeaUahsgjxqITiJ4I+KELI/H8BAdTcqrYqI1BMkDpwMJih2LSdZIECr1lLpW8lvc9aDAMwWA0KRDey72XSe75c9j1oXiUB9HaHAisVjL1ucv0f/1U4WZ8VjXu/A9898T4dDO3u64GvP0cS0nCN5yTVW1SRiTLjKKmeblBuaMIj7kPq7oxjjcY2vdPmUKkpRWbxsYPOdZ6H5Jn7uQURPKGdBjU061oDU5kt2+2lovMjW898juXSLDp9wvGL+0UmU/iBWsqftZVQJtSQ+SGTdhnVcf7ccsILcFmDaXT4ZQ00hcvdy1ihamKuJIw1Y2TSeO4P/YKzzIIKv+CEzuxERtuHxhLGeyqtXmH8uGWu4GcvZQfiMygCXvhD7IVhM3Ah7jmq5FVl2BR/vX45lpRyyP7RlyM3SsfyzW4gNPWj6vcVIO9PjcXAYOMYhqL7Lyh9X0qsw0xP3Cd+m2UMdMD4X7CMAqAis4uCUg5jVY3g9uoqA6j1Q/TPEf4Y14hWym70xGi0MrTYRFpiN7LLNUyxCpgNAa2nnnNmLI7e3cniRzd9QJBKQeC+R4JJgqqfX4jX8FL52L9kkUcVKyjVjOHuxHrtOB8r7LQABCMzdtkzg0Gf4QxTEpaf24GyaTIBjKg0+jzeOk8oBOPJoCbeWbEFqdMUgaWKk/VKb7JOvLZvP1VDGe+//xrRLwSj+fIG80f/kBkOONYo21yaqvQ1EB0VgF24PSh8aPTbR4a9B5uKA96WLWM6/y+73zjJn2ljI+5xo92H8lBxAn0Q1BedeQWQxM1wggKRNEPsf1G9+y7Jfl1GQIiB/wBKiohzBX2+TZlIH8dbO6fhesyIa8wXe1g6KQv7ii1G7cVLUszPfnlvqSKpjNeiGbadTBgKh1KZ4YOwEsZIB3k9zcJGt3+XqW/E2J/wL7EOhRGenxPPBS1ysKUMzcSLLAxbSVBGKzNiOqVdFU2A6w2N3PzlHrGKh6xFCeqewUTSV68t346JMYkrQEhvYV3uC4T1e3M+OoyPAj4ONLxI4Xgr302zvWijBpAoFSQ9DLg6hzbENWW89263PIRFq8VRd5uVAmFcPyZUQL47h2qHHK83HHoyLYoZS7BZM385LbOjVU+nxFb5iARg7CZDk8Uz8ZxxpTKHxioblfRtRlG/HLPdid0ItE2qbsLZZ6O5WYTI+9jB0TkHY28X2Kd78cHczV6uMoDDzjfwXXkiPBnUAjw6eYP2ZC4jwZ86edOQpmYgERsrb4ygHxnltof8H75Ez6DoxM8+ipxK6obzdtt5b2dKE+NRSModf4se5l9kniGDrxADc1RX8XHQKmbWTlz76mS8rXCmrWEmI9yAC/uGxGG1sw/HyEGq8argx4AZhjGPhvBNkHrYdt1igR61H6KDD6x/Bo1u6KGIORGPvayCzygP1xGaCps/kUP5QLpYtZEryGESSdva3XSEhTYjR8TZaYRoxUrA43OGQF5zrgVuh/ZCY7f+ptoZQIOCZZw+Ta4AZpckoRQryRTZZMFmXLevGruIMxVuVjJ/sxbtDOrgkeeI7BRCR+yn2Dx2oXn2Bxe4NNFvXgKkHT9cOTiy6xOKANLyCPmCpm+HJSRJ7piz5jKoFFnbfOos6+BKRLiao2gdV+xC4LYdiP6QVLgTd/AuJTofA/BTZTYN588IlLCFfI5uyjxqRipC+NxggUJJheZ23KqDR9XkGOE7n5Mofic6JYtGORWQ9U0dwSAauqmpqtSEIhQKUPUqEFiFJgx4xza/jX3USW3uRbJlDcIk3QR0aQqbr4C97HIBmnwYKxekoaxzxdelAJDZB4WZovMQt2Ys0S1uolVahOD2HI23txA9+yNjQnxgb+hM7aweic6zGVJpI1Z7PqRv8EfaxvRD6DLgPIex2P+6Ns1AY9jkxrY5YfNvRGpy4XTMJgOLA21hjWuj1isZqhetVMwmMi6DfwAkILPBRRV/e6C4n2KGTVr2AbY0RT0gPulpek19k9/6naC0TIDV6EzPlOHi8S1qlhDyjkSlWC7LaMloblBS/Wk9I4wJwS6e00YJ7/jYs/lLyPPXk6eEpgW2DmdkwjK1aA8WybFSJudxOvITQK43TV//Djerp9EjLKY8tp9K/Hu8Wm4SlTABe0i66DQ5c643E+Jcf/ioBx1LaGV8HVs/RjFXfAnUIjTUifKr9ceh0Jzt5MJ5SHU3Wn3BN+R5cBtCs/ojrGzvx6NPLPcXvFHTB03W/wu0V3G4aTFXHX3Q0xPNXCFQY6p8Qa5z7smz8drKNV8gp78v7Dz2xhrqz/rJtbS8WdzKu/2hyuryR//EtPzgUMf5zD36aEMLZkqfwkCsJ7zzBWdFo3t75Pre6YtljXc6GStsH/ZxHFt4rWmi5E4ri6EgepnQx0D6T7MZBXKmYyzBlNjmDNlFogNVhNSTJxIQMG0qs+yWOWS10SNR4jbrB707VXAvPZaRsAYQto7I0gz4Lj3JJPxx7Tz3Lo/cjlrkCL7FzmjvdRg2GNkd+7RjDa6m7MYk6mM27pMmh1gQD7c4zNnI1v5TNInnT8xTHNVHko6Wp8wEGK7zgUI6vGBYcSoHrA6n91IlI16tsGJHObw8/JTK4GL/wTj6wm02KTIyLRMqQgF94KvFlHhhvIBFaSV55gMPbhvPqF6+y6POfwNgFEpuvU45qAk3J2bjf8iC6JIrc6Bx6guFsj4WftDqsEgGOjRqkcic6AqVcK13JwfxXmOiZSvv1YRjS2yiKCaFX7cNwQRanveBmVzuHXT/GOtECg8+yOGEb/aQyJGWQF2DzA7Sf9JADvUrOpSkxCoxMdf2cH83BTLOWUGYS8EsznKr8li4LvPV4renBPZApcZN3EFvQh4JHYViFOmKyY4hcehdCVkHkOtx90pkYlUs+aXS9tYf8Fjv+K0XiG9fAWfuVuPSByxefUJD6dV3FqAohP2AriggY4/iQvMg81rvJMdxbgczcTpoY5r35Of51vxMxNx6njOu8N3gCVqEEQcMFhG0PbeN4xEL8fjqLfX0z7xhP8cJzbagkbfT0apBqO9G1iFAaNbi39SNIGfd47BfiUe9B8t0QrK31jCkdw1b/s38DF3q/uWyqex2NXoZxlIYN/huYLUpGULARO7ErP638iSFBI+lb2EtEfhh3vavZ/PBdVo8diIBSlv2yDLGjjG5VN3dFPUxwTEVlscXbktTNBBb2wbPLhdzku2R61DDRaoGrU7ETiBnnuYqq+EtcaIMlGwZytlXGyJUBf7edT40X83fN4PyI29xIzbcRC71Gw6UxCIEvou6yz/cBiZkLeCnsPDOoh/YhhBV9whAF/NltpTxEwJnBO/BreprqPpcIsb/ONx+VYDTLsFr/ORtYoHwXeIyh6K+RPH2zP5aP/p0xjdIXrcsEli+dzKEWCb9ixGTV8dk/fnLcOoG8kCLqXFsxi81YzVZQeEDoc1B9AA+vJqLe+JU9pS4k3U9AH91oQ5F8bIkAot4uOu06qfeop8Bk5WEPCMTyfz1GwMND6NQKXETQR2KFCyMRWU10xT6gtmswnW1dNFS509stx1FSApcnESz1oKjQD1m3gM5RmXwalAlmQAdU7iFIOY+Lbk2YhWKCKpLx8mxBcH+u7YYCMXdv78F4Igtm3WbFmBnMVGMTKJhrBoGQWKWZDqsFv5MpbNIq+eTHX5BZOyHC5l39XeAFZMGNpJRlYRSto9P4PK/zJJvyG1cXCkue55yxBSeFDndJE+hbGOK7CrM1hBbDizwY3ICf9BofeZSSL26jxRJLf2km09VgNBhxLQvA06MLxykePJf3HCr3/y/Tb2zJJjGZMQy6OoiDUw9ybOIxVtR8j6DHQH7aMl6c18jFZTvJOBRMjps9vgX3aAiW0u0S9f9zrf+V//+V/4F9/yv/Z0VQuZvQClu6OppQDL6/8/MjH+y1/nhFrufNkBJ6Wi28deEiAJ2KTBrtj2Nnd5eG4OXYCTRANjx8A1pu0xzch/2vfcqA+6/RuKcXlZ07Ov1kRvv/QJLrLTZ0LCIzZS+LhsEGDzhsfpNlPbe5n3SO+71ltAt6EA1p4+71aJb9ugzf9X9AzjmbX4rfdI7pvuFG2BYmPPiAtJxa4sKPgMKbL4zv8Ufhe3iKNOhSBvJ7/+cIVhmZJu8GpyTEbTpUdgYMskamJa7ES6pDXxrAZSVs7YReiQcSB7CKnnyOmYV/0k8OuQ0POPdqBrvahRTuH0s4QuTmVkj4Au7ZWHmXXY1k1Dny8FZ/7iY/BKcEGHL872tta/0PmQWd6Ca+x9uejxgk6AaxGnwm42LSE9ylorvGh7boYOKcO3iB1+ESVLpO487Sa6wOH4bnLU/M6m6MshDo/zsoPFm+8zPqmtypre7graxBfDMmG0d+g9l6jB1VxMmf5qbISqtVhVjcjUfhZDwCYXjg7xxpGc2kmRk8k5WAx75taCOSMU6ScLwb8g3QV7Ga9WNO8dSjYbwz8SAGByGpvW+T4HGOc6VLkHfMJkfkzuZSZ7ZM249JYuKdf0hgdCmHcz0lEUQqDLv15AT58dEzDmhb1vHIax1FnSkE109izMkxFETn0M9URcacP8jPbOT2nBcR65NxuRRAmupJEALgSnsykrcTkUQ18OnrPzFAYc8Yl1RETddRxa3lA91XDE9sJPaRjDm+7yJX2cDRLR2wswvshRZMQpj46yJaml3hxy9RSTv5ePgwbptOg1TAnjl7cDf2p0Fyk7WqAHhgkw4jeRMDihYSu7+FgokC/kiq55ZuMOsAkUXIOx+8Q3uCGbdmN5zrJyFc8jhV3NDBl3alnK70okJ1nx2BX6G2+LLbuoTdWe9R0NIPqYMPdV2QatrNuLaDRNYLoNALxGp6HdNo+dVAXJ0nbc6dbLaImPNDhi37wyEWaWYm+hB4s0rBgxNvoPLphgVR0HzLJtUVtJipPMvDjHAMccmEppYiEpttxwMXkmR6hFd0A98XG2gK+pawwMd+PsHL0dTk4frgHJ1Fi3AtKSVLFwqvXYOzAxFYjbx+cTSG/GAyXtahcKpH/F9y5KM3uCV4ldWNFbiZz9B3hI5EpYXs0BPEJdpY88aqdJZs2Mn0gonoyo8hkjpSZFrMoeuDaNV5Mk1fT9G5MQSMvkBpgM2gOjZmDfRfCVInTAcXsk7Rynf3Ell6cwjTZprwCc2CxK9B6cvWliC6SyykKbtY2PIyObeiiGI30db7POU+kGMt6+gepiEn/FX2eIqwF8bBtTyCmsJIqYjn9jU7EG3hRPZMvhrfYmtvpyRyN27A2XKH9voUkh7E0SDPY3DVN7zoA9JiUFmECM1CrBYLhwteYHKfbyDjKbj7POLQtVTc/wBZr4EJ3g2s8zqKQdILJVtsf+NzmJsdx+RTaVimWNny9BZ+cxPAHwIY+BdCwUC6LNBrBeW2WtbmrGXn+mMoBC2I7kfx8s4p1CYXsOLoG1zpsvl9vOUIywTZfHQ1FGm1C9YFIvqYGnBp8qVM6gCAEKibdpyCXgnRVyewY+tKHD9NYtCwN0Aow3SnmqvLdtBpCiC4rI3RfTbC8S9g+AXobaJXkcR3V76lvd0BvG3X49wQmwdX0ibq1VHc1YrRqzo5Nfo8KX3Kccl+G5rOw6BDHJeMZveSb3HSTsaKFYMVxPp6uLmI3ih7fvYuJMnnKSYoIUXRZQNK5hg5dm0PP5QtoN7RlSnjs1ihqabYrIE/bdm4EaIVfDXhGKHVC+h/OZ/4wL3AYPCdDo4JOF2+jXNeJGL7DFTDPOiwqwS3X2xBcGDb7TRSjvnQO/QKwYFdODPg77HIaoVeoSMHtKDxbeDU2ClUmscAT6S2omRlmDViBFIDwRKIE92Cc0O4bZ3PxlnrcZSGkHYihknVz7N7i4U5ssvgPox05yaclx3lL8NUfHd602K+D4Mfy98N2s/In7+mSnyNMbXOXD6aSJtrLii9ESi9QdfC/gmP2N0FK8oi2WeFbRInAuzP0Kl3otPsj/3RNBxqXKiyWinshuh/BFYFAril8yHflEuXWfZfC4onxxFw/2R//Cv8iR85iFeSn+PDK4f5pGYSF6JD6XEt5llnIarXfuArjx4IXg6dhZC8iay4aO4kNjGv/QOSOgOZq3ziQDTW9yynL7UiAGZlraJVJ+TisUVIywYzXmWgQWwbyzvF+UwfkkiM0kq743E2VcKbLRAjc0I6+BKNnl28HFtEsTgClGFsLS3klp0zaT3lfL74Ij90XGShFcKlTySCAISmNhI1zbRY2tjr2Jd4t5tYb81nVK+SOmcx7e5WYh3T6NAK8FIYKA0EstajF4zEoxQcTc70WJs53t7N7pAXiJEUwZUp3K4NJuvdNJT9S2D0Y2WA0Gcg+CmwWnB9rS+NJxcy32wmSafB/cWrIH4XpjWBSEarj5X21D4kRV9i1dAPsIjsEJo7IWgxmA187XIIxZp6Vm7TM7PHjtlul2xStCp/ANIrn0NzcAd5sZGcH1SP4HGEvseqpMNik3zqfz0J1w41oqnZtozBqLdYKz9AeWUqRX2sfPf8JiRSR+YARL8NQIn8MrUOvbhWi2lp1tBh8gDfEaAJRSAQ4lSdiaynjQ2zHvCVqJu7qSUkaeoh9zOCetQk+3RS1OVCe/FD6so88Qysg9JfIcEWejiT0MqlYCOHrq6mr/L8P3qghQN5B/C4eRaN1hGlVo6T6ZGNkOM3i1EHakgu1FNNJlODX6QVNUJh5eN3rOWdbi8e5Uahn3aVH93gmDEbSu5DRxZCz8HsnrMbldGb/qocotT1NOp2og59DpqvI5MoqPHJQ9o8hBNFn5DigG2OLf/D1s8ff0ciLExTwzuqU3AsEmI/AL8ZlKoHsuTeVoIL+5L6wBG7YUv/9W1VCQLZNO8rlL0h3Gx1olj4DD9NmYfD8Ql889Y5nrnUS5BOQZN7NjK/tTamVNa7+Iv/pMPjN6xuBn6cZQPM3hT8bJPAV3gw2/8ugQuEZAkGMcLlCGr3aJAPgUtjQBVAnlMX+mBnNA8DOVHjAyP9cK3rIMXrGLfrZ9NtXk1jIxxLtlWwHDNYBTZ1AIGQxZojrE49zaKDw8CYTUbbWJ4m4u96JXv6M9Gjmd/vLmSCXMDktg/AYzlJ2o/JOV1DbehoavsMY/CoJzKc+M3k2++nUXW9g4DR/cituklUP2yeOCOugFhNYWg5Znc5ohvbSSzqoWDlWAanHwGBEF1rKG/2m8jFytmYjqURHFWGpO4Uc6LeA0Dr8BK/Pv8Cg8odCA8vYs2sa7b72keCgy3QmGuwJ7dXi3OzHT3KHpQN5+HRWkg/TLVxCO9t24mdQIr9rSR+lxp4b2oYQpkjWTm/orMA/va898NbT+oksceKJ99/8xp17VJMSzP4wS+H02YpOMyG7iqkAgNTds6j1q8Vj6UHGefXikho5vORqTyqH4FAFo9dUhHl+kj6XzChj29E6LiRD+au5Y3zFzH1SulpcKZEK2VhPSQFhlDfHYS7uoJYl5PUy+YT1NxDv4uRJGUY+I/vNLYKn6yzF/ncRvHxDcZV9CVVns4wZxuI2Na2mGHKKu7njCV7eA/V5vt8IloA2jIo3U6wopafVxwnSjqNlJxBlDjJiey+S3tvKu29HsyX9OCkt2enRxuvj77Jvq4+lLVXsMsXbui6aTuRjuRuH4zDf0StaaFRu5SdU9OxlzezqeAkARKb1N2gW3+gcXBEKlFCwpegsgG026W1yIfUISKOP26+TszMKf/6tlJHXCU/KJbPpWa2mhs4oEmBlgxuhD3guUYwCyxsk7bjiID5I6+BvhlqjzGg1JUucQNz/JzpHZZAePiTQLjy4UL+czqOpPtJOIZV466qJOS+Hen+f3ClYi7pTg95LeURk8rdecoOYqQ6jltFvOMEs/xqmdPZBT711EgsjA9rJyLQwpace3x+ZSsgQGDWs2r2FS5PusLCTlhZZE/bP+qkl7hzoVGFSNHLvBkPiIx8CHYRXGqDH5qK8NdZ0CZKeCrgF6x+2MZXr/FUtUXy5pyn8TYNYnLOx1TVGyiIfZk+mQ4AXGyOYWn7LUI7HUm+VcxN6xDCp9bb/En772CA613SU/KxHhtK8r1kKr/bQLfBlVTvQ7T1uhMcuYYKYQ3HP/gRycMuvl32KkazBpr3IVAn0tStRloXRk58HseciglpWfQE7FN4suvOerrz6qhP7yYs5U+yDClMkbuQ06rhqR+W0jzqDprMHpzrq1B3uEHLXWi9y2Hnr6h/qoF0pxg8az3RKXTILbVEul7nft1oJIYcwkvDcZcsRd29hPCYHv6s9QDAU95Kvb+c3+rD+LnzFnE1cTimPmK8WwH36kZz35rKg2k7sNf1Z9j5YTS4N9gIetdmgMyFUUNOsubcU3TViul+5UvWu/VSbW61zfWAZoAvdeH36LH3QmTQIUQCGbbxrrPAm0mnw3jgVM8n99Po9TWQuswJJhaDWMnC4m2c7YGmS8PgoRzBt/F/9wGxSMsMyUPM+kKqC+IpF3ljkgXipylhcfwb/Nw0nbqT/WmOucHKRis9ehPjIqayuWINal0kIi8Tfv0zuetSRtLQmxjdHGnVebApYxt12hD6S5v4IeIhQrOQKxUSVIIAzjk82evfkibwy4B3/v4/TtYI54eRqZnH057ZhPe6keF2mB+MWcw1fMiL/Wxz3p7sd6n4yQJyJayyoRYuYivXHvMtdhS2suez2QiDRASrjEQ7NnHa0J+szmMkyeGXygi0tQ1U+VZxavQpktQLWBpvC8y7qSp4vzmIDh6yyvU+CwKP0aWfTrDjAx7UjUTv4csBfRoLmi7Tx7uamIG3aZTKCRQ8iQE931pO5s4paJscCYwqZdHUHK7p4PdWMe71zpicdUzeMYhe1zDsU35FJrZlQ+6oDCat1JPi9AJOTH8NJ11fnhc5MVgGo1QtTLxoU8lwFV1lQi1ENazlzyGn6NE/4lbNeOYEHEMusNJXcZKn7SwYzR+yx+LLM+UlvDkwnZ2PKggrDGNQRQDZkzMQBDug8o4mAOjSDiMj2IWlG9z4q+U63w5cR5u72eaV3PcHvmlKYEVrLk8HqKgMLKI3OIN7VS+wc0wsmY+uUODWgt70Ms7nxCQYE3DqOwHB4Ol0tohBIMAKmB9Lac6y77XJmz8uXmKwYOFO7W2cvWyEaoHFCI+BvucOJuFxfAQmVRUCixl7eSMuBYuIdjtEZl4qDid/5GHIENRvrqHmAvxXaU8kFPEo7hHVfgKWB77Kb4VrUds7o6r6GqTpaDQRTDn5PhGPejF8aSNqzHCsg7zP+CHBjwXtnRjsDAg83Png3Y+xCAzosnaw2tGPAU578HHspVhhZtvkg0y170LVhm1Oc07htxv3CDn8NTE+Gt5Z+IWtTjqbGo7eazGLOo6CHM63Wdi27Ayd6lySHW39W2A1kuzTTO7ocww/PYLUWnsCRmZAudDmGd9wkbt62zLKrBnDTUkn/cXNBDReRquJpO6UmQl3U6kId8fBvh9KfSBCIRidkzlSY0HTdYbGR4mE3TpHu1eEzQokeDmCpss0uDeRE5lDfFkjPrnVaPo5gwugCsAi8yTNScdtoQ51mTMJDmDBhFDuDM4pNJR5s3vuf1AZ3Nmt88XR/jHBJGUzpGxm/eczuSvbx6DmUE7vGUjmc00Mbr4NHTngPwtJbznbZ2fwfeMDPLN9aHNsQ/iPlLLqtku4zjxDbrcdi/b0wz2wmbEDbYTQ8VOG8MqjNdzzCeW1Vz5DoQ7gafs3oVbLG86u+O5ZRpc3VCw1YLJClTSJQN84qNzLK5p7bJSreLvRE4drW9AGRMCG321KUiI5Na6jaR2QjF3rZgzfz0e76pAtg7noBwhaylmlhpO1Gjb0vwMYmOS0jiGyG5C3AVwHUG7UYKCRvrcSmNrshmrpXf5Z3NqPc2HzBNpixDyz7EemGN9gqrGLsU4HyKrwJr8glWbPbvKUpawueYXNwV9QaprOmjw9+us+BDs30/D1El5NusvC4XuprT5HYO8EJjuuAKC3vJ7IyweRDjdTLBAgsArwrvHGvcEdHAxoCu/jf+c0ly/bPpxeWS+RrqcYsqGENRdtkvr/L2f2/V+X/4F9/yv/Z0WgLX3yj8sAvDovMuzCMLxqfeBmOw1dE4n9/YmhfYfyHvk+b+LRFsXqTbPoP/wudFdB7qcAaMQB+Nr1IjOZUXQ2YNUbyG4ZzcKD9ZgtYp7vN5SIDl8+brDnV6M9AX0HYLB+T0YvdJvdKZTYc3LsWloCH/GfXk9kmh4Y8AeYdWC1EtDZS/zuxQhMJupb3DEaxKCroq/PCp7e0obI051D89ZTI69hhBJmyrUgsUeslvDWV+v4vHgbX/40hGcTK4kf/JARStijhXN+L3Gvby9hrVUkFh9A2q8/y4Iy2eQLu2WxfFp7k4cWeBDnym5JMz85DAdhtc2DQyBm+w9P05vpxP3XN7I3qBcKNoGu1hYICV/LCeOL1Htm4G6nIynIyGUmMGHW44Cw2cxel63oRysJDejkhb5Pgp8TpdtQ5NoTElTGyp9XUjDwBuFJI8FhPQQuxG/KeOwmn+X20WcJPj2cn0M1vDYwGIydCBReTE26gDHxArHVEQxzfvD3dbfc/5oc+x42956m1ihm2kAZPoH38aCQwU4wtQ78m+6iuu4G4rskG5uJ6hzNtJK1JHico6YrjDiJmhuXEnA/n4rXy1+hUHYisur/vodVKKet1xOF3EpB6iLEDirwduKGSyltjd/gXlUDpm5SM1JpdmmmMUYEAXNxePQOlYHwfG0rOwfp+E2ag7vHZsYq74L/XNr33kPoXES3SseP/nUstquDx11U4DGM6z3bKTM3olBUs8C5ie3aOI47hvFX419Mi5jG13mllN+cyAGpGWNoKXccniel/VsALKpQNjsdRe4EacUmoqQQJNXb5AkSNoDVwpqwDG7HRSKxd0Ouk7PAvQ7+ELDJNY2tfsOx8+3DjfDDdFnus1jwmk1y6sJIYutCyd44H+1kPY6+GgRSAa7mC7wxaCnvXz5Gvc4RmbaXtq5i8u5FMnLORihoB7GKHrcJZEd/S6VfJSqdE7Hyx9kdt5ZA1NsYRUn8eCWcjvwQUq8IUSb7wvXZtt+IVRC0mND4LBqsGs74xFOrsGOe2AxF34PnaCLMRUyWKNhiBDdFJgAnJPNJQklu2D1UKmeCVDr8867xSHkPHHbCXAuV+cdIl3/PLQRIjRasRhlWiwGB0ApN14hT9yGwNBC11YFjURkcA8bLg/7uI/v0A5lXfZXA+hqWnjrHuCnXSFLcQ+rxIknh5Zww28O67+go8mf6vul0jDTxsZMJOgvAdyplwiBubO+Hi8pCu5MWhaoXZ2O5LStG4Y5CK2fm9iW0xXRQlpWCq9RCVGoOtNfyaJsfiphrxEydQVBFDeFhbwJ5AFRqPRFmReDsW02XoBpVG1jEPhBiW3Qt1lVSvG0xWaNL2fDWf7Cz9KFI5EFRh80TcOXNOOwuPsXZGV/yrrUCnes45jidALMOqzqYfre9cC/LQKD24Jp8Cst+fCz55zcLFN7UuvbwzZpvmOr7EXRDjOwxfa9iN3hOZeQDZxzaHZgb6UqOsJQAyRAaQnzxdPyWYv9aTFIl0s+lPF37ND+v/JlyEzTbpVJ6MYiBxUGYZ5tR6hSEqS2Q/zXcf4ko+0mcTmrmQlM63z0SIDVIbaCE6yAwdRFoOsqV9CpWVYrw3rmRnqZ6SATO2/yyCDhHxZ8SEjrvE7v/Ha6Nuw3Bp23Hugoxe0zhxaQrCBJu4i8S8lOIHqr+fNxHlQgEAuo967EvN7Niy2IKEx/Q9WYYmoodPIrvZFot+HX0peKDj7k/YR+JQ+9D6z18HBNpKvkQZ4EXHo59KFf/QrDrJFDdgrqTSAVG+smhRanD51kZYmdf2wb14Trwm8nppr7UbpahT7/GKyuP8Htnku2ZuqtAKKU+oIe9M/eyxC+ClxMvUKrx4Ohc24r5SP0WagRevPrZq1T7VPO+dw4SgYF3/zHHDnDMYtSndwi56Ura+UA2DfZkQuNl5glLeMG7FgTuGGRmulU6DEJn0JZA/hdY5X/ynfZVNCGdzP9kMJ1SA4y4Ck3XwHUgPvl/MuHiam4NLeW7Z78jwmMCmHog8x0kdrG81ASFRki9twyrVYvfpGBCvU7S3utOkzGWozVqXJs0fP/c9wisQl79xzMLBFBQGoD89ngGWTMZ6jOMitqRpPDG4+MCPOs8CSsMIyNyFCuP5qM1OuGkqCGobQZa2rBzc0fr2UCwCzZQy8UmeHLFv40KE5wsa6azxML54G38dn0cO6Z6EOd+npwj/ig7G9CFtOBksiC29HCwS4K02MD7Zi2LrFO50RhM1SuvYx16h2GvXOaoF8RWgEFm5sbQS0hNziw3dBJnCKM48wDdRqDtMgLewFEE6iov/C+kM35cxr/APldTKf+5OYxzVgnXy0cQmJqLPceIAVaOT+YTr0NIBUYSCr+mNmohPJaGvC0cxpBdTujd1GTE1IHUiU5ZCOjvQ9NVpMrl3Em+g14dxLvvv09XbBCsOmuT7XWM45uqKPqVeqJwbkNnlKC3Sm1ZnN6TIWUzlT4v0eDUiUHewYH6eGKHfkAI1+BUEoy8Qb7RnS5jPZeHXOMdv26azAGQ9q1NDhMYHfMNVQYZHFeR0DYd9+T7ODk9hY+oi+V7ZyMOM3Fm5FV8hT20qPrj7BAHVQdYkpLJmc2ZXKx5lyZBKxqLyiY3XPg9BC2mTrIYgaOeWz7uHB/2iCWuX9rueXUqRkMXpqkONOieZK6JzZ1w1pZ15RG2kQkrbrH11DsoD17k9J2FLAn8zAaamboh5TvcBe4EdowisrGQu3cHEvTCaEQRz4PCnT6XFmKXXM++Ok8aHfvQ4R7CuMm1yBQumL6Zw4TjsVTY99DzxwsEDb5vM6IEBEIJotnHaJ10jJ6H01l6ZxBeE/oxISwWMlYQoD3KRC8txT3tpAlv8KyzkfLeJpA6QNsjhkmdGKVW8m11FtKq57jtooGOSCjYCDJnlgh9mJsxjxtdMlb7FBI+7B501sG1mZD0LWaJN+SE0q4ykPhSDUkRM4BbT9pIKEFkVtIjL6bI6wOU3ROgzIrI1MqVChtLWRgTRUlXFKPcAaEZhl+i8Vod82PeoVk0E0xPvmNuLgCnZFpUMrzSr6MtCWRF5Ffc0E4Hc38QiECsRpD4gEcpZ2iu8kDVrWCh8D9syrBJ6zk7g6Yskz5Zd1GUzqQkuASRoQXOxNkyFgfuw2q2Ul/oSvxde5y7BjF9+S/Aa3/Xq+lOENk7l9GydBcrRhTZnrHpKhclUJF6B6lLNOYKPTXaENzVFbZsyOh3KK64iFNpNdq6SlzLurEGB4PFHpySQSjlSqc9nVojbn220tfdhF13NfhsAaCl8hi3NwdiEjzCbaE73u4GpN5jQWABTSh68VCKDGdxFGTjUhyIXnsfBu61kaCKf4S4j3j2upz+x2J5rnEk4pFXccy1KYlg14c6MzTllmNts/Dlq19gEht4JewMzj3FHP6qP9Pyain48BxbgUaTN28uOgcnorHapfKoKh6psYBqo4xPWsHbMwgq9kDGclKFs9g8uYgusT+drde5qdOzVaIg3CWDcJcMPiqzcssTDD1fcv2QI0WqEsbFBQLw1qAppFaOIuelLxhycQiaM8Po/MYFff/TfHM6nwXLnGmoacLY5EijYytXB16lwqQFVvz9rnYZYyiRnMFVkMSffT/AYpDAH0aWSOHD/e9gH9rORJc9FBRAvymA9jxkv880VQQ+8bWc6WzjD8t1tkYewTN3PXJxE1JhL++eH8jAg8mopjzEXQPRkiQ+ULcCORypHMO9BhPRHSr6VVxloNhCT/tJ7OXNALjIyhh3IQhNl4Yp1//E4usOrISIl/5+7koXKXeGnOeZn9PJOyEjeEEi/yxDFDp6w65iNWsRmUQ26WpVAF+VhaA9E4hTUi9lFgeebjUz3zXNZtVQ+RfUReHd6oqgj4HbtRM5pr/OBFkaCESYPSZzK/U9cqJy8AjRstS1Ea3IDW9NAfOi36PAouDVJmhqjObYrs1cDNLQ51UzVUY43QOLpXfY4SEj1tzKfbdGGixDEQgE7J+l5FrlDIT8ylwNtD7sw9odT9E+dOi/6iQUCtny1C/I9b7E975KjW4Oc4K7GPLgZcYcnYr3AyklX/wr/QPaHuBrtc0NVixIPGREuN7DreYYr5y5SUFLKs1On/P272+Tn9COdLIvEVG3obHYlt2h9CZI/BcqUy71Hg5cS7vGWIGJvm7XiHK7xu2aCchb5Fw6NYeqnCo0Sj0sAwldcHE0woHHCeiIZ8huP0xj5Zx1SMLLzuPJ80k0XFGNo3uqnq7YL9gqqGacYQFYzKhFKjocuvDRyMjsX4Wd9QHGugKwGuHKFG72gdFqMHUKWPnzSkJSc5BOMvD+5WPUdIaxwGs5xj1zqIix4OF/i0ndM5k52wYQ3KqexPnf5mJ4EEg/jyL6FAaTNyATsNLc7csIl5cwNr+EXlhD2tWtiBKzwPAYehVroOEik2f8zi83N6D+9Bk+jypixmCfv6u16PIijk+/xqwD60jI6Kb/O/fBaxz0NvLl6FOs881ALx2K8/aR3B6YB0KRTYobGHgsFvdKT5ztR1IdLmGyZ1/kfnuYFPYNm+99zRu/D8U3J5LcUbMQWTRYZGo2XN/NxLBvyTGG8tvarxCaJTg+DERq78lTMVJerT5EXUccJ8O/ZFnlVZB1EenaxVBNLQDP932KNy9c+jtQH1oUytzdczk1+QbrzScQCw0sXyH4+7hroyu/Gj1R25dB6DOkm1MZnLsLFxfIdV+GCbAgBqUnWQ2DOZj/Cu6+N3F0EfHFvWRCTTKYLeFuj4xkpZ7zZh1lI7Lptrdjc/pdWghBKlTwkiOssofZ+wOJPDCalmW/88rwW+RJkth9/V3U0jYuV8yjVzqL9GNjuRBh4ab7RN4fImFo4E5OFK1CLfZBKBAw8chE6jzr+GjECW6aVnFerPj7fZ3ROSKT6Sge+YDnRtmIdkLga4sLv/y4Csu0au4OzGO9XzkAMlEPL6YuZl1XJRv6/0Gqejan7cCBXE7r32J/xylu9kJEZyHyzmYkV6UMvzmWmggZaT3tfHTUja2n89moV6EUyfEaYCZisIiBAgnWxylcQoGQ1FupROVGIbDvxNAr5VTBWhZ5H4LzQ0DQQ7FuABJv6G0+j6vMbFsHuA8FQwdOtQrmX56PbIqZgmg9DR6XeMGgRl24k7USNRVZ9dQXz8f9nIIkdRJVqWJwjENzeywzPFM4n7+MjpvtvPbpG5yde46RI+5QbIDJdUB2GAE6HUdW/EFHezvPGJ7lhzlSEMlpC1zBhSPj6etRh7bPLOp884mkhQ7/D/DuE8xm8Tg6LK6khKXQXlVDj9SAEXdAiVAgRC/X0+WgxeIpo6q3ihFWNZqil0D9JcrWezzV9xYneuYjb2siojCCMSNssc0Iu0p+0trzfosIq9Xm+4cAsIoet6eVwWv/4HZPBMJaKd1mMR2e47F/rC6BQEh57ERiB+qQCkAE4DYExuei6/Xi/b9+wlB2iH4twZwZkoFWof17TyG06vgjtp5t/vUcr/Plmagy4hovQbUt3tQ44g5+70/Bq9QHF/NEytrm4B4bArmfoAYUTWmE5wWitkYSm/kfusIi8CyZTbchlVLHr1l+cDm5/iZkXQU0yC/AqcNgaEMAHBwdzFfuE2moeYhHaTuXGo0sDQKarmINUeGVpUTeK+fFb5/nZkwOdptfY3y6be87pOkAIUzjapsf+T844D4qA9Y2wr0XbV7FVgECi4D88ELqPMrxcx4H5Tuh8Fso34Uk5DusRifsK1NYtKM/A6deRWhsBbltL9CjK+LFsWdYn+OK98fPcSf9FqzfRsulGSBuQietwGq1+dNPcmrAYJQjk0GApInhc89xWjIZt+Of83STll/emw8GX5jVRd67v7L7w0rcRvjxIFGAwa0d7B/v56PexHJRhdZJhaRVTm6eP1fqp/KT/U6b1UhXMV8f7ofk4gC0L/xIiHc9LZI+EJYIVjNIHfmmrD+Ru4YyssYHLdD3mX+Dfcd7jtKqPIdBXoFC74dS5AhKH1Lz4xj/9WTMw6uxpljoskChPoidmR/gGhpDdd3XDLs8CN0EHecmnqPZrRltmYmnGq/xZ68Z4h+vzaxWhBYzyZpRdI+6j/62ngnHbZI8evedtMUPwyySgtFATnQWfk/tp1kqYnt7MlWuN3Bs+7c38/9rRSAQ4OTk9Lcq3f+/y//Avv+V/7MiMD5mbHhPhLLthMkCmDRFyKV6N0qr/BDqtWjkq/lw6AjiPc7zwv01TP/iVZq8m9A6t5AhNDJMJLOZdvvOYMLp3/hW4M6z/XfwwuJaLunPwfUztPfaBMd78jw599t0HGIknFXO4a2R9qxza+eHDjCaxSS4OHJZB64O4WwtOgUXYUzKTJtkStBSlqvucadpAleSqvlu+Tam+krBuS8RrXPIGxRElXwUmG8jNoo51f044iGUcEAr4IOWAJp1exl/N5U/hGLC5vgwzHSMzmDwKZpKTXQ2PjdnIiyOQt/+HlelEcQbrhIl6OSZEgcUOgXxdZsQWeUEOQA9GRD/CXiOpFzpiik4l7l+HShUVlvGn62FIXwticUawve+xMEpB/HX5/NszGgmnE0H92EIYtfTaXABoFMv5ZnjuSx52pF+YZksO+iDvus/GOoHc274OTp9amhyWoNrbxP01DK5ygmduJ6EiA5WJR2gztEbWtvhgCvi8RX4VNg8K2bGvs4rLt3c9viYrVtmUKcNRRD7J8HfrUIcUMOwp94gWirkoepjipptWUP7OwcS+qcGv8QcitwLEYqEPKgfzcQ/bYvOiY56fnSMxGloN4uuDWF2SgkmQxH/lbDLaNiPRb6RVkswLp4bWTtoHlzQMabxEll54+m5moDdZ7H8OasGYZQLj9JHQcZTdIgS+eTPdDqxxxpbgglolUdA6nMAhKv/wvX1X8muDkdV6kebVxeOY9+3ZTlI7DiTG0nC4SEki8S81iEn/Y0z7BiQxKyoWYwJGcORo19TdNKCZsAIvn5uILTb3tTJ9t8YNTuQfT97UrU3kdHtCUyJ8WLM8DMw2rb5wWphcPRORHYzqTg/gdd3vs7y9VvBHaKk1+k/bzCtvnNoKNtBtbIQH3M51GSC+3C+EHaTO+IsbqqBPL/5eXzH3CT0pXQO/bSWPn2sBB5dTNmlGH577gbV48/yrqsVZM7QfweOQgt/jtFysTaVT7q13KGFMeok6L4DTVcRyPty/VwygWWBFPadT1L0fdvzBq+AgHkglBBrLcGcmIOvoZzVnmexWAUIBVZoucMRxSRMs4WkBuno6RrHxyEhDHmhlc7SXWRGvUx3sA/zPSbz84qf+cC/B/6yh1k9mNXhfDz0FGUDBUw6+xzul96gY+NGHML7gKkbO2MDCw9PpVNjJOsx8CM1NcEfto29i34NKbdTkFgdcdToEEtsIObk8I1YHWL5JCeOm8omUnv6EJMdQ9XgKqTF30Px9zCzi3xBFAfnfY27JJ0Clyx+jHlsqGxoBYUHz3rl0TKrgQL1IraMeR+T2ETboKN8d/p3jo0/hqPjUJJ7JDQU9aUj0gP7tNchdBUHvv+Qex+9j5t4IUlffIaX0EJFyzT6tNhBZwF5aiXymcepdfEiN9REm6mEB6HfEXRvOZUBUNFQQU6PDJ9CByIyx/MwupE5nq6g8IKgZSyYMJyK4iByOqcjlmYgsmptmQRpu0EgIELdTZNDG4J2A2kZaXQNLYCAZkj4AmGDkKR7SQy4OQDXrwfxULuYGWqQ615kbmI53zgcwO/BNlywwyS2jYG7umBlxGaODx2Pst9tVhSk8uynrzF28Qm4bwuWSY01/JSxkS5CKBi2iT6zzhCieNnmKXa6L/6ykbzfJCBLJ8QxPYPIgMYnE0mfF7nR0kZW9CNcG2UYRfV0OGhhVAac6WcbCf+7wLJaaTYKGF0l4LSv1ZZF4RhHgnkD7yilHDAKkOlljJCI0VTsgOTvOHz2FRp0JkJNKhx1rfRoH2/Iz6QSFbGPKbudENJD6cI0LuWk8dEkoN8JADx3jeGVgzP4yx4+CClH1almPEJwTABVACkeU7iZ9ogy7xLcSiFJnsii8j/gxnxQeOHqHkeXTy4HSheRd/gr1r3rTk7jQKLcrmEna8Js7MIuqhzVnSiaPniO3NcE/wL7Lhvm8FWlkf5nBxNS4Ejl6PHQZxdtpQLW1yai8O/Pd8PXUCk/wS89j08y6xALemnu8UVOMT/IWzCY5Qx0GwgOUWDSkS4cCtYWlGIJTW4nCJNbIfNdyP8KuWMiGccHEFIcQnBHL3KrnmC5is23bf6w7tGwb+YB28b3vzPV/9f6V9kmJuluCgBZxV34LP23ofU7aw9wuecAkWsV6Ow8efvDT/G1z+fEpbnc2dIH6TwD3/sr2GBnQlj8M1QdhO5yyq+lU53rjaemCsG1csz9LLT3ujFnXxtSkY7hzu9Q0+GL/fzTPOfcRVGrkhB5NwVGaBCcxXp8INaYQu4lFnLYoYUE1TiuVH7ICCX4Sor43AnmF4vp7XBCZC9EKBWywRnyjGBUBzPlZDhp19Nor/HmudA8zv6j4l0Sb+6d6odFaaC5rZuvKvbx/q5GuDGf0yUrsFqh+0YmY/4qZIcsgMaQatwEekQCK0EV/jQJupl8KB3vbhlpyTYJb8LWUNclpWLaLwirn6Yjsw8ypyC4+Fg7acgpqiPauBT5J0KLFIvQwBz5ehbKG22EoZ4atkd8wyO9hS1fzmGXNAjPwUJC6g6DOhgUnvzSNQjVr8Ek+dQzVHSDvrJ0RjfdsGW6OcYR7XeeEHchL3YP5r3Q23SKhPj5+fGByzl2562mXQ61MQ00iltw7i6HbJss3oTqaI7rskm3WEg1apBKAG25DQip2seLj5IpOJPK1ZcOsMwDKkXFNgks/zl0SZzZOPxNtrQLcfp1MiPtjMSFJoBQBr7T6PVZwYr3X0ArvkPJsveY5fu4L+pqbAQZ4Mv9BxBeOIdz71VMHS0Y7PqiUNjWklqZD+0TdnK7KoqHkh0AvGd+BlnFblzCK9i+6AEJwVO4M+gqrS4WZj7uvkKJkkk2TgYrixLwvG5CNDYZFI4gkmFnKOUnmYZNnSLqNy3lHWUPS8e6QdC7ELMey6eTyH9rHUm+dThVueEUdgX83oFbi6H6ELHGQZw5GUYEcB5/PFLKCBn4kk0dQOGOuM3AmPMjEBlUSFJbEEr+HYhfZW9hyMOxnM+OxbHKSMrTp+HmAgQCBd8c9sJRtw5CPmOw/y4Suw/APU/oLiOn1p/arx3o6vcdvwfE4yuzPBlzW+9ySLuSV3qvE1Vdyn82vEvZpCIGjdoOLbch9kN+8tqNSgiJxgacxFZkQgMigRG5WIsIJf1dvqTU6IcmN4pYixyB+LG8UNUBsBjY3zwHNjrSGtJAwpq/aCPoX/XaJnHAPekuTep/yBDq6riqtvDbmON8HPECTrueYbfeTOJiwDER8r7k1/SfEQ4QszR7JP45Z0npXw0FfeDBqzDqJjfsFyCV6uj2NDEiqJFH4oWgq4fKvXSZnDkamosAJfLGzznZDEPs+VsmlhYoS5zONUcj+4fPRWvWgN9MePg6SOwg7iM0HSrcG90xeNexMLkMk9IXqbgPaMJotV7nr2kbcTMG06PsBoEVWfNVhP8f9v4zuqqqffuGf7u3ZKf33jtJSCdA6L13ERARUFAQRbCBiF0UKyAqgh2k9957DYQQ0nvvZWcn2f39sLlA7ue9x/PlfZ//GM99nYM9hrCca8255lyznMd5HseD1di7bqe1NQI6zWy6Fc+rcZ1wJAIUXrTaJvDTi98hM7gxLPM5TrVHM2NFf8ix6iuHKDw5MGknGx4s5US2CoFRhUAIf2R9RIDDvccZD/WeEja8vIFglyEUaMezP+81tHp7BMoc61g3CxGZRAgQkJwqITk1BgBRdQ3z/pxNj1RP54rNTFJ4PNVXUtliSq6lk3LTg0/2rGXpV79iIyyl2WzLwdl/4qgcR3xuNm6lHUC6NZBj1AMkh7/hRXkR+o6JHHabwvc9EO79CbNiVjM+/BsGXMzgatpVlBoH+v0zmfbBNly1fY99d6rJa0rn9FA5p4ee5tVtC7nZrCbkdTfGbTchEhrxC9GTcuMSQcVBJE28i6NrO/Di0/XGjEoAZUF6DDapmDHxyD0LQJuwELPcwNvrP0YqN5H8YiDYBPBTdRDPXExDK9Sz7Pwy7vXOhL0eVj3MPn+RunkHkZcSyJl8lJldH5JYdxv8gLh1GLxfYmTQYiqNjVzXuHO1ASTRf1F/sYmV6c+wqmAxms9eJFqiRF2ZQwcp+Pdc4VlHmFQD/Y29ufP2VML63ObFYYcZYBNCX1UejVpvHjb2pa9YhdclN1KuRhIptOCtaHyqzSJM9JGKqRFX8pJsKe49Kczw38+lO1sQhZRRYY5CWmng5/J+dDjdQ91/GTz4kN6ik/xQkMBto5rY3ot43mcnNEOoUwT5zakE29diSipD62UhfdRGUlUGMLhD0Y9QuZuVmcNI+2sWljGHyBt5mnj1kzrty32D8tZPCHkQwv6Je7GPKia1PokMNysVP3bRvBP6D64vZXGv+QNCm/cSnKgB/nh8j/N+g9HICvAyJiJChBAhVB+kKqaSL8f3J3jEaDZdmEc/z1346LHqk+d9RUm7hoYOIW4mKfI0O3ziekDqQZLn3wz0b0Unq8D+pb1svjgD2eG71Pd2xzfMmvVd1hbLheoB2AmKuNZnF3cSblDfbeHnyj74ICBOtp5Bg3ey5sqPXFz+PfsD9GBIg5mP5vCuGvbRjxKXaFrSN9Lk3MhEudfjNn1oOMMQzw7+9s8mzakVf7dLMKAV6s9TXDGFsuBiLMY0flrwE2KFD/82tzI/km8koH+hjOnj/6Bb+gEzo9/Hxy6PNK/DrDVn4uhbRmT6n8xxaqa7/SiXK6ZzuWI6kvQNdNh14FHtxXO/P8fNQUW4D9rDuoFWHc/nSpfx+vrXuZ56B2F6LRaXQgRCCzFuF5gS+Snp0hD+PDqT/PwAanrfI9SzkZZKAUazlUY3TFfK2l0z6az04XaHgqS3KqC7Boei+SRfrqTFO4WQMjsG31mO8b0mEuZ/ydmzsGhZK59va8fNWYV8XxCdGhvcJbZkFPvQJS8iyiQmp+9u3NpH8/GpLCIjoX/IXIaIYV1lNPV+XRSPOYSrWxOxMqiwdHG8duSTb8PGnhrPFgKyPGh2fEB5TxrrT1vlaGYNhMEqe+ozE6gI7qAuIIogl9Sn3vlONyEV838iQgo59vBXkz1ZXW28IdVQMvQsAaFx3JcWcdUlh35AP99/cLMpZ0T2C/Tq8cLYFUbmphdwG/AA34yVbL8txrEznVHit8m/14tmJaTcTGGf5SR+f1bSXyjhK0M97kCdQcsvZ0XMPGvgXc37nI+/wFkvaOq6R/qEu+xMykYTncfbHtBbMJlmywAIX47gkhAbYz2aUgvaymYEJT58IXdjBTvAJogwM3hU+KHXiLAIzUy0gU/cD2MxCPgtJ4S062l0VGnIXtTIVe1eUgWvW4OvWi7Qcs+WoFu/Efz+PE6rqtGa6jjUeevxfurt3ZPRuLUjshch0qi41RYB+UvAbybijlwyUvP4NTKP2LJR5Pk8y9UgoByed3uPDe0+nBySyereq7j696sUOt/kfudGpjMWtUyNnyEBaX0wsTFDmHivB6FFS8vMfBy9/aBkGwmp19jR8RWKnP1M3zedDf5HeGt6f/48kU3+Z9GEuggwz9DhV+VDL89Wvhk6FOq+43brYL4s+IcU3wjmHvyJV/otxs77AIQvBvsYPG3ykUXXIQ4IYdWvUxjRGmeIBAABAABJREFUqxyhvgXsIrCYIL89iOiHoXjUe9CYIuQ5RxWOmvvAQCxCBSsfzKJCr+bUiMs0Od1nlff7DK37BboqUVX8xdrGQM4dHwx8SatbGBVLPiIssIMHJhVHCmZzNL6KZ3f3x6P4CqZgN2xadmLTspNTvWxYMPooaWOnMCdggrUD/JeAxQgqP0QXfkOp96c4tYrfg74hwmnC43FtV7WR/hf7k3ojFXNcDs8nFxJflQh/Ay79SMeDrOtaWs1uHB9+gHYXCS8ZtVC+HRouktwUz+gv1pCZ3I1dfCCxTkEQ7GoF+1zSkdiG8eGW69jXFxDY7yEWH0er5Mwj65D44FcKWkE7k1/eikhlBOc0/vRazpW1d5iXG4B+sIG/ayOxta/mk6x9fPqZgJ2HvfijPojJk8Gwfyv27Y102I7BzmyA0r+oUU6mwe8Sag8XdvWdCBbATgdTWkCkYEK0NzGJDSSUxVI37DaDVEsg4zDcegmc07hkr0KdZiBEbeGkF+ga5kGdBrwngmNv+ktfwdJ6ljvpNRwauoGzja/xL+lSjsh/5cZzJ+l7bzavn/mY5KVpIBRTK5BxfPhxxqa8zJA/AlA3vU75Mh1FdYmkibehjipjw5Lv2D/mIJfmfAK6Rr745yLrV6TC2HY0/TXYetgi9fPgwcBXGJmu50TxCWbLZj9+tsHBlW53f0Qma+BK1IMYXhefYJe5ixUtD/AR3cWB/7PBPqHQeu79n7L/gn3/tf8xMydvodpzFV6uaoQBsymoKGNg/EruNLZwPVBLiySbsYJlxLlbKZRSnXPQefvQ7F3PZ32PITM58bbcFQadAiB/4yXKvh2BedwRHgaUU2XsYojvBsZmfMD754+xS+2Ocfhx+l8aicSwG6F5OkvsQW+Bo5p2BBZvJu2ZRJuTDdgAWKyRw84pIBBwSe2B9uN1ZNUMByBIaoDmG7hwgxGTo9hc+Qm9D9sw6cEqPn7nY0a2unEs2A+FoYSOS16Y3Ax8+tanSCyOfNjjiYsYtnUIeb5DiD63P8cDq1F9vo96yRS0InvkRZd4u+94hp4qJOphFA3x57ATtiE0j4G7S6xZL54j0QTFUeMppbjEgbFBPaiCEqDxEkS9A8Ck4HzUI7VoTw5jxPER2G9tsl5XeNDe1YjBbRKGVls4P5xmRQ9FunGkyF3JqRlL94U2ZD61XO53GSxC9IoYKFwAhRsZkXscc30HrelZjAqrA+rAaKXckVT8wajyJAzNZopbE/jOpp2wl2LQmVRIhD0YEVgdAWYByxrBxeLC18MmsGPnW7xuL+Wo0MKWF7YQ9bA3K75cQdWim/xb/lomluHRsx6RRIP4xlfsEZsZNcft8XUXwx0GnU6h1U7LkHediVeZoEmKVuxKfupNbrvpWebwLm5dS1jho0SdOx+AMLmWrPNJKOwNPCvVMl3oTP+QwSCcB53FJIoNDFHA4WIxOevnoR51lWH9y+HcSOipx57RSAwSOpQ9NLg00semAtn9t5gCYHiHXI/3yB2gQagS8+nlXbw2fB3yoPGMjJoDAvhFk0b6eXeCgOx78egDhzKjowDMOrCLZt6dAooru0hxFuCWkIVcrYNh17l6sohLFdFEelujZQHCDLch/xiMK+HMlQUU9z1Dhi4Z+5E3CYusws4rkPFJR0F4km2FYCNrpVumwSi0ML/nC36Z+Axkvo5cXYaNyITa5MznNq5MCfsLUf0j52jDBZwDXudesoy82A5UQh0+DiYrUOicAm4DHvWI1fEXdcOdMOM4hAsPQtIPEDCbnuoNZCVcwqgORJEnwVVioo/xGDw4xlonWF5jwCwQUONVw3WFiBfsomCnEoeAFVzusX6mzV46fPpkctacyKSMv0DuSs2x1Ux7bi8/dzjy+t+zCFbrUA+9/HiM+PS04Hl0NHfT7rNk7k+4GVKpi/0Est5B0OsjhNm7UGqV3I3P4mayVWNho8QRmaEFcj5GKPCkLKAMqSEJTGbebYKPnYFjcTA6l1S7RhhZzPny5xEJu7jgBeK2e+TJgridtIsoTTIdl7OwPXSZ9ngpdlOGgkhGJzbc10MfkRjVyCtMtNXTpQuDE9b0kFvSaZyMOssg42ouXnFB4aJg7ry55DTux04m4YPqy7T4PkBals67fW8R6pMPfQ5bgyJMYPK1IHNrR9toj9bijE7sicpS8Rhx+dmjmIZWO/4s0jH09FC2uHeQMWcL2Pgjru+mV8pDCjybcFG+AIBIYECq9GStYDCNdcH4WGDCzxMY/ffox+/as+UUN1LrmVXbBGI5vQfdxsWrAfruBu/xVJeIyF/wJ7bt2ajfNfKJM1w1Z8FZa5ZEsawfPW8mEOrRymdzf6TQBv6O/Qxp2FIQyVHvncv6+b/zbJUd98ztOAsDwSkJpnWBQISk8TabtF5kXemFx9UUzg04R/2213CzsQK/g8W1VL//OgPVbny/ZAXP2xsxyd0ROSXx1z8/MuxSCeIXbHljk5VKhcC5UPIrDvmzGdi3H5m1o+DgH0S1tgOvPG53viCS5ht2KMIr6A7eS6dQDqIvwTkNWu8Rqp6CIVqBvtKTmN+epW2MwkrfBeA1jiV15xhlB4s2gX3TfizvL6U68hKbDsCaNXDh7mY8Fu7iukxDV4UHAuvC9diCxUup35WK1NSBNKOEWJUZEr5GV74B21130asbsTxj+ZdLEoj7nMwj7kRf+oiedm+yHM4zNfqwlcrVNgQ0hSweGod4cCuFhZM4qVZyRVgGzmPAKZmeXp9j174Oj1oPvCdo6B+wG4F5PiKhgZHBmykTq0iRQbdZwNp743DxeloHCKBP4j1Sk+/xYrWYWrOFNeLfH18TIEAEiASQF5WJzqEFH7s8ABKDT3M/PBaLRMT8T99kbcZ1Plq47XHZ27cOo8y5Td7zhYQv/ov7bn78EenGjarxbM5+m/6L/2BPqyPfN6vpFmp4gz8ZpgRfA5zvrKZp0SZMEhXNjqXITc7kGuXYCuELZzhQ4UPhb72Z2maPQ70awSs38AlS8bxUw/5Oq+5TVE4UPlU+fLD6A8zCp4GWLrErXy9+BXVnKLE5gzGFnMDo/DKNIfvI2eiCveYM7YMlZPZ+QIe6g5WCPvw6aBV+F/5k6ZZtLKruhcstNYMk5ic39X+WIdlv87MfjOtooThpOiEhFnBKgeYbcH4EcfJUgkRwsvNRfQQiGHgS9jiBzJkfjvTBtHMI4UBBSAGJRTPB1GE9hNr4IxAI6JXdi17ZvbDplKMdJYTyv8AuBiJXMrgkgRJOoY4vYJwxivmSiQRWVHBQG0XjV2upEa5h+J1baLv0EGzN+EAdQV27HSFlIdiKzST9vpzQ4ddhzTDr2iKx53ZAFZZRZ/DqkBN9P4buqDYo/Q20FWjD3uGt/R9T2i1lQL0LDe2Pgq+SNkHQPGgE15Lr2MhryOxdSZzzIwDfoTeMvGP9b7EEg9yWnKQT3IvKZUxVD4qbM6H/QeocUphWB2qDBKx/kJX/Ddmr2BkBAjFEOhk5MvgMNt0Rj8FsoUCAY7Mj6g41bn1mkSOSMcvXBey7YfRDCu7ksOUVKaIWJzLj76KXdvOCRQ//yCBoPkcFqWgSb1Ll08aPr/1FmdDGmj2fsgUkdhSWDaB6cBUPbQ5Q6voTN8WjORKxHCKWAxBf/i59XjjOmhJXZjYV4GlJ5sFTo9DC/RMRqC0W7COLiXa27rruSxJxrbfDtltEUe4WgktPoQg/Bbbh0J7DXfMaeryzabQRIzvSnyq5jsh1Ausa31GA8o6eRX8uosK/kfsx92mxV4A6EoIXgtKbX2/GIM53ZHL/a2wN1vC35SEJiYsZHryFrQ82kx72B04jLGia7Rhka6Rbut5KJ998GwQiTho2YBxZSL1bPX8ImlhsXk36v1rV5NPJLb/TxGfGE9psx8fzXmZq/TKiZG6gBbPZjFeVK0pV56MCV6H+DJWhcEtn5KOusyxc/iuFjvaMsLPqH3J1Nl8XB1B8pC8/LdqOv7CBdyOHMu5gIJi6cXFexpkhZ5DpPRl1U4NAIgYUVr1ss57i8v1k9P2S43VJ7M9ZxcCQe9Y5FqC/Vc9q+IBMqvue5I8O+FsEV/sU4eFgzbaJU1zg89QmFlVLSW1y4duALmwerAbbUNKGXmGPx6vIO24x/uB4Ih2PQBgQsYI2kz97Pb7j95Yeom9HYGlwJvbTPuD6PnRV0Vg3kYtXXajrSCT+9DpMgR6IhLPQ+L3LNS04SraRXJtJj6CRxrAmVLIe0MIvd78CIL3XCEZWRVPd7w4rXzjLVdlyeLjO6rSNXImduYEpo25yUmdimA2MUP17Zw+xun7I/7wH1NBmq+CQ6ijPZHTx/p5fue3/PalCI5Ij9/CorkYgSLeOf/sois3rOPSCNwLvJj5qX0HEiKu0THiJh01nkBT24OZZQ6VLKapbrsQ8iCAvtYImWRj5jWFYMDDDBiqNsGX6CfTiGjaKj2JBiNEsBcwIR14gV3OLqshS5MhJNBmQiCSP6z2no5QXj87mWto/jJv9MRW1n5LGW4+vLzgzGHVeCMogA+kBF1DrUoFABCoPfl54lCkR81AK9/BxdLUV6AOQOvLW8FtURxeRVTmUh6eTiI4uwd2vHvQtCAVmtrrBAS20XQ/EYnbHqV8RQ3ut58/7H1CqckYqu4nWVkh32DLs3aV0SqrZ0AalRpjt+x1nfC7iXgWvfO9P2WwLI3zX4aKq4nTJPBYKBCTdmk7sfUf6r7xEevQaoOtxm3rZ+bOlIYTTNzMov6+ifkw2ZL1NP2EurYPhg6DBpD40Un1gMB3xJajL/wHXDEraA+g65oiHrY6cm0Y+bF3Ogq0XcbcpYVHiYmrlhbyw7DTjiqJY02LgbYuQYYP+gq5K0LdQf+E6x4cfp9KnkigJSARwpGAxm+9sBOBixGt0rP7wcT2vNcWR4l6JfOh+hLa+ZLcMJszNhs7MVtruO5EwtvDJANRWku1dzS/5fmj+6Ytj21BsvskG1SQeaGdTrw0gzGI9Z+3qhFD/N3gzbClEvc3klwYz4cf+NA6u47LTMiqFy5gtAQ/bIuxkjYyrSabYYxfBSf7U2QUSajeYb04sorDFGtgkiICa8EGIZWUU+H8KFnCVaBnh8zO3e1Zz9KN+BFjOs//VZqJboTxo/uNq39c0sUn8MQJPP8rjrD4T0b+YA/SNtrRvnIwo0MTLQw7hXj6U2tI/IPsDvmoI4aqpmWS5mV0pNVTqup/6JluHXmZf/4tMMa3F3/4+BZZ2vr2xlX6+O7lUOZmO8I9osEC63o4QnYU2Yy3jw76mpDUOpfwhs0yOfKs0kDT7MO6eOlpFy+nU25NVN5gUtYp6Bw0Dbw5AckpAzroCTlYMZ0zoBroNtqgVbnRqHBCo7HhxbhFB6io2+8GdO5CRAcYdFeQUJKP37KE2dDpqWz9GBRxHUHwUR5s6mgVCjAojwe4dBCvuY2+rZ5LLs5Q3NnNh1DneqHPiwKD3MKNjisyZLnkRAFFyDbMd4XiHFreSa4jUngRHlBEshX+a/Pgs8jreveqYtyOVVVvjkL4Zg0hgIMXrIPVaf17LKMI+I5v0cx4IhV08J3wJF1UFc2LfptOwlSHyWrx/3sjKzQdZePQBwyNKrVrfj/vMuhZ0VI/n5fw+5AS8yVVvuNCt5ff0i6z2fZ/ncu+RfbEWJviSWTuCX+6u57ngZQQZ7/OnbhCNjZ0cq/ZhvkBIdPOLCARmilNuIAzP4l6AP7HOl6g+queiCVJMpqf6/FlM3ADGfHcU8WRYNQaqxZ0QVs1OZ9jdAbseLVdOxvPg/yUq1VX6N2/m2OhIFGkmnr32Asu2fWTdU7qk83zCSdKGfMIF+1LK3vuBiMwJsHQ/Aiw0JGeyO+AhC+NfQvOgh5b2FsQiMRzwp07sxcawTAJVTrye4EVB2hy0hhRONW0jXQ5SARydcADPnsVYJl3jeNkBelclW8+X+lZsL41hsysYLFBb3U5kYwJ/2lUxw0eNuLsQJ5EFzKDv0NP3YAzSpG5EA62NS/RMZFH+Znp2H6H49C/EVjWRNeR1TA5poARj5JuM+6wfRmE5fn28OKK/gkjfj7fClrD9y3oUIWcZONiFpqZG5v42l6pRp7AJrQWLBW/9UKYd3sW4SRLu5J/gbOcIoheMReDSF4AhUXMZ5Xad0WWLSc6M5oLESKJIDrvssbWJQcCr7J6xAxdDKs99PQmb3vkoB1RbR49IQtnldfhlH6F/r3hKQtrwfn4MpFppxA0VhzgetZUjduUMPLiK8PArOHYWg81yos8O4W1n+LTOTH1AKs3esehFdpT3MeLY/A5tPeu4l3wLVYgjPMDKBeo1Bs4Nh+SfeHfrXtSNxeAHbQ5tWCRPghoNtrG4RB4iV26gJOMyMwOeMHLReIkm1RZuHK+iM0TJ9Wevo9aFg8rfKn1RuYdxTja0p8i5okphd/orVJpGgv2Bx0EPglawayzGtfw2V4Jf43zJCjKeYH0glDJ80wI0thp+WL2DQ51SuL2EV8t/5KhPH8p0vqgf6ij4cRpRi/fgqS4AozcmoR0AJhMYxw9jss8c7DTnQAPUHEHmUEtp7EAiQ5rADAkygXXfrvQF3ylk3Qzm+u8LCJp6ibBE+NFlO5RUwwSrtq+XfBoaJ/DZWcs7BjOffPjro8nXGjw/zPce5S8LOdCtfDQ7PB0hqxTewsXxCn4Vg5AUigi05ID5Q/zlevL6ZmKOWEGTeydK+2Z2hX/FA8f+BEkucsgDqmNHYeMfDdeegYYLdLi+zp3kG6QcSuGrQ1+xxrLm8XPEAikeTR7kRuRyYPwBomoPE6HsByYoSJ5Jov1ZOk7W8dUrb3BiwVZW/DyP3Bgjbk27kNa7Ahn8n2hms5mqqiq8vb0RCoX/9wX+f2z/Bfv+a/9jZqk9ic/1R2i/3J3G0IMklIBCb4u32EC4HORmDfMPFuNvn80ZcwVXZixFZvxPtNqjya7+AuiaeOjgROvAszRLu1h8MhKP8F68Im3GXt7IstS5TGqN4a7jdWLsAxgu1mPoSSKq3ErJ0FteRLypFVG2FazC8XM8X06AqzOt9HbAme5QLnWfIbxFwvemaCyupWCn5bJiOZNv5eKnWoGbt4F2YSFCoZkgkQGKfsLd/CxDTw/lQUIRWwfrUNKA1nYonxXd4f0WMwfrVWQeG4T5mSv83G1kqMid8uqj6CxwsOAQn/QJJs/FhCT3NipLM9IuX4hbZ6X70RSxMOFFChtkVH22lOlJ9zh8bbs1uljmCMBJeyk2w09Q2ONMtFbNZPEdMABR72LpqmLBQzceXItBL21C511BabsjOCxgqVBExzOnGBXrh2/zGHQ9ArTqYdDiAU7JpFXm0XWzjevRSuaVyPnZV4+o7b5Vs8w2gn6Gjdy72xtNYQpFSi3TJ33C8AnWqP2/G6Yw5ZvdLCiyw+e7FymMrEa0wEyGEpLlBnKNDVgiqsgyODBfaWSAdy5Gz7kMDvyNz6/sQNidyjPR27hdM4Qvl3+JUWxkuPS3x2OrTehB17PfkN3hgmThaq6k3uXdbRmoMl8n1wlKRTWYTWZknc20dgeBdz9ou4+r40iKk0fj7SvC4eJKrjc40H/iRSixUkqJFVd4eY0GmZOW0yPX0ejfxTD7mZD/LbgNYnlsIBL/GyyrCEbovxEH138NeKGcGtl9qjzaCXsgoKzACd2EXsjvr4LS3yH9b5AKWf/6etLKXse1fBqKni44HGYtP9PCzc5XiHiopq7XGHI91tMjWM8yZ7hldKHz1jnqTfWMvzYARXMy7FeDkxyw0FvejtAIeqGJZSnnsDX50QEIot4BgZBXbiyn2z2H/s0JvNrqi9G9wioWXrETYUgoa7/8CUVnE+7BjVwt6EP8kjRs4l4EiR3ORe2sjzjA+bpefOoazWWznM8m/+uwaLGwOC8KxZk0LDobHuh9Gb/woFU3UqzCUWSkeOwljM2+NPe7y8ZQKyCndUzhzj4Di08OpMuhEalOym8GET/HrIHL03Aq/YLg4kAcmp3QZQRQEXmJt93yYK8beI2jTjqT9zw+os0uhkkIUCHAKPW08sSKbQjbOoqkdy9zslP6aDYRWnWmPEeBfTSzWtcz9IuVXBp6mjPpl9EB6bUqbqfNAOdU4vL3c8ADNubpCb8dTGJ6MxoPM7YyJWDhM83nXMzNpb9TCIsNroT3dCHNXk2M8BnGqKCwTUezTyed/eow2QusFHI5H/OscAD5pQGo3OB7/3I2i5v5SxZkne/cBtFZao1oVbbLKf3Zm66Evkyd9ysxdu6QtAmX7fWYG02Yo14jJvyKtQ8efAC1JxC6DWVw+z0M4lZmeR/ga089Z1jKYKUZtotgbBHbs0dg/i4Wx0FOXB/qToaXxErf6D4EoaWHL5Kq2BxahebIPfzv3UE8ahD2999kYc9Cmo5PptW3Hvm/BLFHKsG78geqAT1mwoMGUxhZjdJzEAG+1vlfb9FR4deKfY8UYbEjE7N602fqMPr4BYFATFVbErkRm9FY99v0koKo7jRErgSBAIFAiAUQCCxPVoaaY3D3dYhahUhoR1BOFA1XU6jwrqTBtcGq61d7ApTe6EN+pdX7It2iICwCM6UGEPXUQfFW7GTLafK2xUH9rzFdfwGGXKC+LB/RgD00VblivnURH9cOKurt6XXzMhT9iFixjq9W/IZzaxAvb3oZ+YBH0eWt96DlDn7CvUzs/xYv/fYKLuV+lOpbIWyp9QeUfv0XH+wvRm7TRItXKznNV5gyLJ1hw6y3CfWax8rNUyiOfZetc3+i2JjGv01g7MYn9xRt/pG8vfBlGvUjoOFt2sTpdPsYEbkE4593B5+uNEoi2gk0PoSqA7gIL6G28ybPVE3GwXLkzTqIAjSF4DaQizdfoL5Wj1d+HVe/XIn7slLwnQy+kzHrNHz04jFOdR3jt4tZ/Fb0LBvErnjYnKO/3z80NM9lQ7svJU223Nsdj1u8lsEv/6vOAohTJhIlzYN20NCCUCD613UBQ265YtNpw5Bn9/Gjp5EV19dTX/sMIa6nKewdSXgvGXUpt3ELqH5y4xGZdF6IIc9nCPUO68GzhN0eVrqd4cFb+C3vRQIr9EAd8w9eotjUTs0fNljODuCkFyyqkrNZ2IDU6MwUG3AX9JDbfosjLVZN1tHGjdSV7kDn1UX62Eto3BsZ6p8CNQfwHnIUb4WK3n1CKYpU4iQxI/tf0hkFCOiw60Avryd5xueMtwFBSyqXtV10dG3HPTuS5jkJHBy3Gz8xbLRchLODCRUk8VGTgBKdLdXx+bjbNZIkVCGxCwNDO2WtXrz/5xTMXhIKPdaikdpD6lYrfYxzHxZV/4FtVTgdFV6MFAuwG9QGIgXErAXH3uwQ5xAZmYN26EWmhdcj+Y9P6JF23ljlfRrXX6b666X0v9mHgGc0VkpXtXXdsqnpYeHBKZREhiCwX0KfMSZaWrI53BXFxe6LTHCwEH7sPApdPUx8dO+034k8tJKgv5+lLEOCLrWYPDdHEMlgUgNYLGy+7YvW4wGjj09j3/VIaj5S8UJPNXSWIBBIMGydSnyjCzvnt+Ajb2FE1G94uqSBUUtx7RU8647SIevBziLkWdtH2i+Bzz3uj67+gyhO6eGowUqVtlXw6NtqOI9A2Z/0y+k4tgaQWDCElL7ZyEOs7+NmdyBurb2wc3cnSgoCk+HxPYUYWZ6VguFiCg6ri9i+YBhF5rVw+Qo0XMTkf5nzUdnYal252P8yZlEPCCTglAzGToS2Ro6MOYLM4EZqRyc+xDDdoAGX/qDwoE7YTJ6TmRqnVmpdGpmpaIKWu1YdZ8DLaywXxJ0oylx4dYeJU4uznxqDdbaLuZfWSIXLLm5Gv88RmxGc7L8R3Y1NpL+9lbWlExl6S4n+ciTPZTiwf/6HsNeVCQ5HGDjrIpFMRmuchgUBSyRKCHkJgCkHv6XBVE+DWsmeUeuJEIwD175WR0brfR4W+JF2PgZzTihrm5xw3K5miMc/AFRrIggpUWEQdmIvbEfWCncAAmZbf0CibS3GYTfZVidH1alC9K/5AmCoTSNCCTgcH0GVdxWtC31h+E18N0xn8Z6pGIe38O1rf/Cr7yMWBVM3uPTlj9v3qG5TYZSYeNa2jAjjBJZ4jgT/Z0EdgSXsN9SSbsJyA0i5EokgFPAcCi23aVD1ZaTyG0qE7fQ6tw5pRCmG0tFIrs0EwE49lAH5YspL2sgzLmBhwNdPKiyxBWCeuo1SA/zQDp1GkLVchzvzIG4dnWYHjreL6Kh3JObrKZwcdZlJz57GYupmeMAGdnU2oVeMY9KKP/HxepSR5ZaBpCKb+GofrjaqUI8KIEF5EDvt0cfvsqcFcjY7YXR5gIdnCcmRR4BZLB75Fyg9+evXfdTdScBH0MgeRxX54joemC8S33svh/KXktiko/r3Kahm7qE6MBsjMsh61xr1b9Tiaf8Wt3yG0n5VxMF14Xw/J49T/+qrRMlihP1FfGtjx9XkH3ih3cIzjm9iEvxNqAQwmTg+6izNPUV4t6SRbEoFQzuIVDR5x+ISpITq67QptKjbj9Opd+WH2z/wzZgMnCR1zOweyepnP+Cfzgiyysx8NHAolR1hxGp8OFnsyasedRikBgQCAaNCNuKuKuWe4Q3WRmvI19iR0mICtHz/v8zf5R3OyIq8kPVSka6AC2ifuj7Gzkiu3MA1/yTE7uEk2fWCjgL2B1ZwMGcZrn79uDX6VSLddRDwHES+BTIntit60ITmUuGQhil2G+8EP9qH5HyMxGzDko2jcLmRwBj7ThyFZvrN/AQ720oCYkOZoLxOwHu/8naVHx+57iJQoeW8/Cx3dfC8GmSONpTHjkPcfYPkPBPu8iqu1y/jflUiBrMMLGYifEdSb6NlJ9fJrEtj+b8bJbZhz7YZmNotuAVV8JxXGZSc5KS5N281Z6LATEeSlNaAb3D1bgaNATSFVKgm8uXUz7EXROL3wA6pQy3hzuuYGmalCf3i3kaeK52CxaBmwNYCegbcAb93rAEqqdtwcmmn0f06ARdT8Ds1DP3bO0n13v8Y7BNjzam0a3ZEZBIhdzRQIZlPqHMKAgucLp/OqYqJjFt8k6P7O7mRt4aDjydpCfcL/BB9Nw2pUycxaQ8wijrAMYGjjVtpv3aVNp8CMva7YtP0KrJfjHB5BoQuRqcycDvhNm7uwRg0HRiEAmS6KiKcr7Dz4TtIBS0Iu4Mp9c7iQcBBDOKXkD0C+oSKfHr1C+dmtxDHrCPg8SkuIhjm8pDzZTPZJg+kzf88IosQWbstDRIjaArgRDKEvkKz4ziSjotxr5bQ+Ra8aAfynlLAGgS7WhdHRJ0TMk9rMJkAIWStgq4KnK4nsPrYUG4sqEZvK8FofNoZeSKslSvdsG7HRcp2TkTxZhllzWPIb05DKmmhPhC2dcC9d57jQ3kPLx71Z35va1DE321jyH5tKTExOXgs20WF1ptOSRzP7LGuqz++8h6en37Hx3/8gl/rJTT2GZy7OIdzZVZJkVnOcDioHwDm+3EE+7azcpXV6S4WQ5fXW1xdcAlNe3/s2sPxsr8ODz7gut1ADr24j9pGR2Su7dwf/gctsufpX7ETKnejkkSyfsN4dEYZ/u5LsVjASdSEJQRqjLC3ogc+eYV413bsHp5E5mymQPwN7144hKA7GG1oHRXdOsRGCRKDFfRf0ecZ0n33cLJ4HlW6fhQ2+7E45W8WO+q5b2hBJWnDzaYUrUVPmeRFfr0+ElObAXOJhlDDN8ATp/rHohp67qdRvTsejasbTsshTQHZevCvfwX7AE8su/8hTNUCE2BkyI9sf7CG6zeGkHnZG+MiMV8t/wrn7jRiBB8ye6qV80P60AmDazVxJmc2H9Zz2Qwp/H+3FOCw3kj6DjgaBD/OW0mvG+fxMnhxPWA7TTWTcH40njBo6DLZ0BCdS6NNK1ED3qTTdh8ikdkKYnqO5L7GjRO3Y0mb4ohjnZjmLl/+EI9jhouEoqo9dNjpsYgFCDotyHpkiG3E4DeD0tpyioMP0e3U+7GY3uYx4U/VdSAPaS6TYGc2s8QO1ge9AqWjoNoqWVNd7IXlfBjh95pQt4xhR++DTPczclzzFzm165h1cgAN0gaOzr5Oifo2SYInhxKRlzt1gX2IjhdRRysWoRCBRQdmMVq9llbdYmKyY4ga/g03BdMYr94LB/yZ6PU1BzRv8cyb8P3nHVzJaKbBNoL7QccZ7GGDm6IQSdlRync4UJA6mzyhmKWxzo9dm/ubQ7m6PpYAuYK171kpvN/oWAyGdsQdmfya9AVuNrYsf/ATt5Le5ZhnId+4LSIE6zlKaDYh7dGQeLED1+aRhDs/eWc23iNp5gzqDgF/P/sOP4WfxKUHOHsMgFTX+Qw+1ZcSt+8p9Pken6Z5CEVr6Qp7ndTD6wALBq2R8NxwWhxbQeljZWkTytAbduJabqGmpRs7oR2h7i3Wh4a9itEhld+HFLC+tYCdjcKnQQjnNO7rRnB2wh3wKGSuGhSGdmtjIt8E/1ns3LuRpim7sdmZyCfPv0vUmwFP8zgY2ogccZuiJn/uOr4JQikCwbdP+hIjZttOjKouxlQJ8BFJGV/0IwBew69yItSP+PAw7jyzi0aFLUOVD+H8QvxFG0m83UVzvSfiMZfpHXCOGuUwPB2VULUfG8YxdvgtdjdOYv7P82kJKYb3baH6IOR9xXZGExSdjbNByLqGUPxcS6zMFI9sdVoGuQ3J7Cg10WM2UKwaTZD+EvhOB00xEbJf6RVawaazs5ig98chpOmp8f+x6ib3jg9ir6uYkHhb6sRDidY1cSkwk+/boNli5HZGBeV2xznR2p9kdR75+jkE+vRgKxsATQI2vjSQ6LFjYVQNx0YdI1Ydy/B0a3KLsVWDS3k+d0ty8CzzZMyRMYSNzOCvwn6gBEV5HvblWchCFGRHZ6PoVtDHrwnvgTe5KY0kJLcYk8z4v5lp/t9vFouFlpYWvLy8/u//5/8/2H/Bvv/a/5gJtP/S7PN/lgAbL174/Wscm3QY+lxitMQbQUwW9dpJ1GsDaXP6HQNg0yPlvYf9CQ2pA0OnlSscSAg8x46KQFzLMxlwJpB7C1o4YFhDp1ZIU5cPKX6/008M3SnbGeUCp9vj0FtgiT380V6Jg90kUtZ9x/49yUSWC4hw+hVCX7FGewNinYDInEiiCzxpzkqkyXMbKjstl4V9GXi8CrNNCw+GQG7qPl6ygw3qeoheg21HFuM++InXNPacz/dgrmcXdhW/Y6t0ZYxzMp/rK+h6+Qc69SF4V3ojChCwWF7AqyEQUZZNeno1Fh9n1iWcI0HVRFbnYLzs7KHsD/CZSOetbjruepOdfhVXP6yOClMX6EUgUXOhy43CLpgSGc5nfVbTLJkMCb+B0ht62slKuUmRfwGFQUpuRdzgqrkJWECLtJLisCuMFS5i1tbxGPwc8es3EuI+g8A5jByeQkOkM40FM3HZ/BbX3zxLelolhC/DouuiMuIwXU5K9qj3kmDXgJv2SQRvu9GeY1qoN0CYTobYIEbUdp/L3fByg4XvvR/wgbMIf78q9sZm4yhKZHir1QGS7HkYYbcHngXnsT8OIc+8QKvU8hTA0C7y4D2XEtQKEfKIh5Q51PGu73QutojJLl4KmGnNzSbm/GFueoUzfvFFAPTX7/D2iLncaHuVz0bfpltQRm/P7xhSt8IaFZ3bG9e8zzEGeaObcINFThKrdh1AxBt4l/xEuP9ugppfZLcvXNP5kjbwU2vUk0RN3CkvekVryDv7HLaddlSJfLAD66HNbGCaqogMHwF/CGqpsj9Fol2B9d4eIwCINxoJzommwlsPElCJGqDsNA81Z/HP96FJXIZU2Q9jTxtB8gYo2wftD/iqp4sdW14jv28018IABFB1ELLeguSf+M/uMiYzEP2VsXitL7ACRGIbNEHvYDZ8ibzdTFu2K5mFCfQbvw4q1kH0akSCSQzx/5XSakdWfbgKTf+bsMfFSmcpdYRJDbzgqOeY3MCfY+/xRryVJo1rs8FjGIskBSwP7mDWwy/w8LFynOeLk/AWKXBVaGmx6cTQqeOdT9/hdmImfDARMg7R0pDHZxuu8+BaDI7DVtBcFAf+o6z3rj6IvfNAbunA0STk85lWip576k2gsHK33xBGsNvhErWdQcy82YeXetdBya+Q8guIVUjVCgIGZHLfMALf8goa3OrJb6uEhG9AKMGFLdx97U1SjBJEejHuftXYxlyHqVbAaYDAA48/fkUc0oHk7mKujr7CsJmneKVnD5KvXkHuLaFydiUlYzYT4AbkfAxAQHs9k357jp7hFVxMFqAzQbfMD2ZaM3bGnxzCuN9fprCXnsqIYaj9vKD+VytvfeJ3OJdqcMgt4IqHE07FcMwngmSBGPLWIxCrSLwZR1ROEBLbLvY5dZOxZB/UWqPLEEpp6vgNfeA1omKDERcGMtbnB8g9At314L2VPhUC6s0WpubdR9HSxtet3zJ/2BYad7fgXnKVLgdPuAAJtxO4k3iHDAXIeir48ItpjHgYjnkXnCubg9EHBmsr4OHnSE1pvPfyq5zqAtmPX+CaHY3z7P6QbFVzizr7D3EL7rC1qi9xu2bQN7gBkdNpuLkQirfQqHyH734ZQkJ5L1IlQ4mY4A8XHtGltWVhDF7G/KA7KGYVE2s3gXupBYgzX4SeevAYgUmezMPQ2YgMPcRnxlHnXYsh2g1J0WZ+fn4zP9zayH2HTh5ci8bFqwE331Jw7c/5sh5mdp/AXlXH3MmtvO5ZwSF9hJVuChjqFMGR3Ad0i+8ROPY1QkIeOZMd4sH/Wa7sNHLw/dVYkgr58NcN/KZJemqNrBekoFUZsXvmC75OzeFwTxrc2g4tdyBlCzJxFP4595G2OBKWDO3mpx2MKcq3GLL2ACvufcfKzJW8knQLTmfgKhzHj/1cCHZrJfRXZ9wq/Ti+xIHFAXqQ2NNUfBXTis3k1UxnhkCAY5DOCjg034T4Lzi6o5qu++U4KJPQ++nwcHkZLBarXqtJh70IVEK4HTSRLnkJWU1nGBm8mRpNMGW64dzZ1xdJuTc5/Ubi5hbAkP+FxjO3bR2/HHuXkIjxzPDIxcH8NH3Y5LNv4pav4dz73/NtazM1Wh86ejxYZ/sZna4PeV12imEjnZkYchgm1sAjqrg072+pkcnZJTBRY4R2swyJSYpSomFu9BdcK/DHptMG59IHCPQ6TPZLudoYx0LLPfRGR54XKLmrVTDo5GSiY4vo8L3CQg8YVwNyb3eyhq2g2WUvJ+xvE2eZzcQBbwCQCFjaHvJOxqdsu/kaw5qDGBRc/1SbnA3FXFbYs0VvYqASYmVguTyVcQIlqlkCliY1MlaWBFpwF4PqPzSozu+Q+000vcMUfOebxiE9+KadZ4zsHJwbRqfuJcR3g5BSSlDDDRp8tWD3Kgy2anXu+fI+rn8mMFAhwNxtYcCgNitVpk0geI2hMuhVckKvslANz6mh2ukZvIz34f5qSPiWDFkhTnITQwbJOR37CxZVoFXTzMaagRSi0CCVOpJ0uQVT2GZszP0BmGb7DR77XmfwaBUPnLKxE7RgFtkizNgHTol8EV3MzWfs6VZ/R3a1NyFOWPd72jIrJfGjNavbL4NbMhWjfVIgwgZMXchbbpITlYONRo1viXW9UXjp4LB1j+Dk/RavvfsHa+5488Ka99g67TQvTMsFhzhrEJVzKl+613JMC0cf0US1Bi7BwWsIuKTjUrCHkQWRaBud6HGvp1ahRR++AqnXSM694cJLf+7BMtefIbdnIYkue0x9KBZYeLb/Qy7YedGIPZeKxqKIDgDHbmi5RaRMhlq2B3tXEetdfXnVEQpMHeCaAblfECZ5noEKyOkBU7sTUpUYqo/A1Weg724uduymMXwvQbWvk+cnQCLdD8f3WR8+7Dpmh1R+zkyl224z4gF7GOz2dFbtjeZjXOj9PR2KLGubzUrwHovTtbX0s4P3MVGVLuSfhHX0dZsOQjH4Tqe22IKbCKRmMxKJlcpMLAT2eYFjAkn+IcR99xVz772NJQT+bn8EMmavBYdY7ie2ci3uFL3u98KzxpNI4URWnT2DTKyl2RSLk9kJ53p/VKYpIPFHLQQuTgCP4RCyiDHKYoYqQLNhCoFVPoRuuQE8cdaNbjDTdXkyOwafZ1bvMqZU/wFdvUk1m8jNjsIS2w2+FqL+EwHu/yw4JrHnEzEDzg9APs8CNv+Kpu7zJwAvKjfRHXeFWXtmEJwdjoOlEjKsWXnCy9t57cJQLvWA04Bb9PFvAvFUa3lVACV2Qzh9uALnRhfgKNVpnoTGF0Lc5yCzUutPKwhD8lDKK5fTyO17DVW/v6DhIhg6KDfFcfPbmcSUBHEu4wK/O1XQf0IjCnMPE14chrKoA5fROmLiip50cPb7qCTRfLJlBDadtihfNZMadJpa01hovgX33qKrczwaZT2dKgFlc36kXQr9BQKr/iJwTbGKh6s/R3ptFl+/vIKoRU64h95jpM+3eNvmc6HlM7SDiilp6M+Kf4KZvjQdMq5baZ6d07Donfkr+wMc6jfiVdXDTWPlU2Owx9YG3/F5NNZKueUD7aJDUB3Di7KDeH/wDjX+RvZMzaROfBP0ZbBvEuhbcDC/RtkEBT0eiWwx/MpKB5iXP5OPB8HY7RbOZ86i8VQD4kQdH/mZaTd7McNnAbGOZzFaJPz1xxvoslqZs/AQ4xU6NIYypiRas/R7ynrx7bW/aTdbWHD0EOV+5QhWP71onfVx4vpbnzH05FCe/3wyGevSn7r+z8ALNGScJidPSLbRkbG6VKJEPXjJ72Krr0Hd40T5uR5m2sLffz8a39VH2NTihVbSRpzETI5Uw3mjF0MjRkLxFoSabJqMM8CjDeW4A7zo0YSdsQaCF5IeGs3hi+dwEoFIoGFfgwF3hQzvUCEjlTDFFrL123in71H+yH6Pt+f15oFpCaeaBhCs/RMzQjA/z6dTRrL/9mS+uNvBgbDGp8A+i0jFw/BZNKuv8umM92kx2UHUOwzL+YSwW71xKpDTPRf+tq/jr/8czxK/pzvfhlrPWsxGHyoHFtMpLuNZ4Uq23l2HUqKhuGo86su3EftIKYjKodaxkXH2MSBWgsqPd1zvk66E6W1eeFT4UNwjJMK+hkPPCDhZPI8N6hJulwVx4a9R+JqlLNu4HvTAru8g4yjFzn8xeo+c3a1ZXB18lgjtS4A1mwaFO4sNSrxGHCPCZxW/1H7BVGt8HTu7ZjM2P5zayyF02OvpEdeTahFAzWGoOcyxXjKmuugQaSYT+cdylMYCLOavuF8/kAatP5P1I9H/4UpO724ax33AKttTFIV/yYjgH9lT9Dwjq4JRFbhSdqeYwbvfpuGzT1GIDAQ5ZNKmjebg2N/wahzH8q+WUxVXAa89yrIs2ICPv4XnnfWcbdFh9+NsmqLLcEl9kq3V49nDlyu+pN/Vmby0bzLto2qsGaSqAGI8a8kNE+CqDePCy+9S3seOZ5c+6eevS2KpbpOi7FKi7lBjMIMAMwKBGaNZwudbh9GeE0xm70wMEgMmsSPXKieQ5rOf7J5Ibqdfpt6tno2V4GMK4qeQJ/duVj/P9zsHUShLoi4kgXjHU6R67UejdySnsT81nVWUuRzEpV5J23EHcvqnIhBYgT4Af/tBCCsGYVbdoE/8DOzECjB0UmoJ4YSpmhBRAxLg2zaYo4iwOt+BUy5byKvYilAgJSPkJ7xVDxEL3gXAUwwlemfMZhP5trV8v/o8+XIBlcIFjI1+j0k2cPHWHSp+v4rzwC6Miy8S5zmA9jwXAE4UL8S50ZfyoznoJ5xmi6iKFEy8mLiUk8XzkAX3oplOHpToMAcq6EgM5oLTeMb/K4BAUm9H8e7hmEUQn3idL71hlwYcWwcxa08ISqOWNk0HYc5P2Gp6uZ3lWmcU4mRH7DwrmK8CO0U92kfyJpcrpiLgHAAtZ+rpI/jfA33/sRSgjwguXQH9IlfGHhxLQmYCxznLiDmBXNXMJCLRh5DcL9GaNrDELxuxfx4n+53ELPodoegJ00R9Wwj3bwno1d/Imemt5Kt/YIl4DTOUnXygkDGpSoVOqyftcx8CHGeiXSaGxO9pObGYdDmUYObytfMYm3+ittQDjwDr5kwr8UBd8ja23UNQ3L1Ia0cYV3sryIiJhabr6EKWMGX7AUac7c+DZBV3wrcQJ7BH3JWLo7gAqVmCT6UPukYdlUH1NNGE6F/ZN9JAb6oivfEZe4Pd3j/TT3wM59O/QO+vETv1IUPriFNRCAatlWpogp/13Do//jV6ux/DYjmB0EaNzPY7XhkCg/s9GrvqW4x8axPlzl+xLmcJZqGOus6f8LC1Ms7caA8hxOEedkob+uRfoVesBVROEPoKnY4zuPfzaZTdjTg12VMQuIjpz3YS5mqlyxZYetg4L57zIwaRvTqSIbLv4Gw91J8GiT3iqa14WhKo7TRjwczEWrD857uMeR/V6WEkHDxJ+0gDUucOZKIGZB2XECnUuDS4MG3nNEorMpnxzwyupl217uWqD0H1IVynq/hntIWEWy/y2qnXaHwtB3yA8h3g/Cqza0Tk9ghY8dtWdnhoWPnqWiT+48FnInfPHOZO/Es4NkUx6evFBPd9CCYdmPXg0Is0hwPs2mNEIT1FjWcBGjyYkrXK6j/p9RFm+0m80nc1n+a4MPOnGdgOvgqmHnjkH7Qz1bLlve18nOWP8a01NPeXw8d3oeIf3nSAE2VmcBJxKCyHRn8pzhKrpnWa6GUKHJ4jx1LOtea7vN7jzFfRJ0E9DFJ/Q7JzE877puLtL8em04bqLgXEfmSdb0IWUZ/zgHuBR5hyfBrb96cz5MsdpHvkWdkc+vxF1VUFxRdE3BnVQbNjOdPcthLkeNkalOk5gs11Syk+UUj/W92oO+cw8KftT32ne+siMRzuT1ufm6zxbWRGhx1DxL3YVOdP1ZcjkKXUEeuzgqraZzjonU6oxyQkNhpKtJ+yY8gOBn2ZRYW2icyyPNK62snxhXOv5DBg/KcA6Gub8cs+Qk5gPQWhBVRG5JJ4qwb/3E7MPaGINC3YN+TT+JcFOx87/p6/lQ/tYdTco2y+Gcg9yVv8D7JY/h9v/wX7/mv/YyZovPLkL3nr8Xe8h59qAjqtFH12AJe0EmKWBBHjeg5bWTMn9J3MupxBq0WO8Ewa6pFXrcAWgMKTEcYaxkYd4nNCMM/dRZBvCM3F8E/OagCmC7agPfAi3/c/jTKomB/6xKEQQP9KUNOLNc6zWH9xFuWRm2mcuohbPX3pa9bDw88g6HnmWO5TvmsaRWEN/DPtH7xsdcwPegG3mjK8W51oMYHlEcVJgf5Ruyp2UsMArrrW0NBjS9snL/JXv0wWv3SQ18IG8ppAjFehgFqXemZcTSb85BBUy7OwtSblscoRXj7XTEsLXI/x55yojXSpG7inwtDLoA5n64NdROX70+91d17tvQ5aMuGs1SnF0Mu4VtiSePkZOkJs+dS8i5HTwnES5YCmCGxC+dC+EuwrCTek8H0buLqGQmcZ7zkV8asGLCYLQq0GgcEWKxG1Fcz4Xj+cdrsePF0iEEU2Uu+wGkLuwpVnIO0vlrhvAHeYqZTwkqM1uv1A/qtsyfyG4CgNl/6eCA71TP1iHsESEa6C0ax1gviohVzIMnD99QkkJxRxLr2b9MAEfr7zJgfzl9FjtCHhFSlfGbV4OFXyXFc0ab7HsTU38R/Nvv+IdhskenZO38lwhQjqzxJb9z0rzw9DdyGF+iWlZPa9RLP6DB939QV9K2JTI0HSazzomUCnbRcd0laa5bEwwxr5ryytp31MBrXybj5ygiiZwfqeQ5eAUyIHyhL54uAbSOwPsOrhANyHmEmzj7VqsABBhW7c2ryQynGH2TgkD7/6R7RR/s+CcyrjlZ10NHhwvrkv05Ln0E+mgjF5VmAW+HnAOY4nnePCmblEFp5hqvkEXG3jRwdYu1ZNnmEBHuXDaG2FbsdykORC4SYauofQ2eKCbbOCV/+Ygz6+3Ep/pAoAkZKlD0IR5gzjXEwdDqOPESwbC2ozGNrRmTTw6ntc1kNubRBj1S1PvluDBp1tECtW/oy6rpLi0HwcXFqsVIQ2geDSFwQC1gQUUvnKfepa0/hAY6C3QwS9RLlQ+CN65zH8tMwPteAOdIVxPtBCzaAxhDZ/TEQfPfP8i/kxcDl34+4SF1YJt1+G5B9RSKNRJt9H6BSIIPMKvYq30JWgRBk7A0q24qu7xYJfXkAvF/Hbs1bnpdCih+Jt0F1Do9mBnZ0Q3KYm9OgwxI7Hoe4UHA4FuyhEXrfZZQNuxVeZd97IhZe2cDamCnaqYIae8+JRFASdRqhyR2C/mPmJW6zvRNcCMkccBZnIxN0YZFJyBp/DMaCKYYBGoKLWYsIsFGGxWLjcDX9q/JiVvgAkavbU5mM/4zDlbkE8u30MQpEO0Zgn2SFuki40IgsaRT1Dn1uOWeQKSX9ZqS5yPmWoNJs7lbHEnbNncMVyvlvpyJ/Ov4NIDmHLGCC6jEhrS1eXiprubtSmOnDuA6nbQOkFaqiKHEZ8iAlhrh4f5U3rg90HgUCM3eGXSXugpnm5PVvM7xIqGIBF30G0yyq2T9lHuzaEM+v/ZlDJIO4k3uHzVlg0NRfz5w9o8mynq7ye9MKtTI7eAqf00FWFj2Q/pY8EyFW9Z1Bm28pwl1DoKITs94g0eCBVdBPcOQh9ZQGNCguG9J1Irlizrk1COf5l/nhXq4EbGGvtrBFxFdbsEKFQQqVTMzg1MzB/AVl1+SR5Hra2yz6axup8TI4FmGvdGH9oHILBpYg794HYFowaFiW9zKzbG9izYQr9J57HzbcBjiWgsFtB30t9sen2YvvAfDZ3VfB1ajxMqIbuaqR3r/JW0FC2lL3E/MTzKEzOVpnOnI/BZxIX9BHYulZRri5mbTO0W7ygIx8Oh0PIIgjeRIEmGLPru3zeAiEejlBojV6ncjcS7PCuvIhnoZxxqmTsBqcz9l9r7CVdO7YujWjrfqL7Ygi/RYaxWn4OhbmYSpeD6M3R+CfvJce3k9W66ZBlpbPSWeaxuhmCRd30nnWYdpM9DL//+L61DV/R72Yo98eUoI9ZSJIn1oPc1WdQOKcz+uxIbNvkLLjlicwEwsF6chr7IRN1YVEK2JdYgSgujxb3AlwkfrzOev5txsJSoi/+AxdHYO+Sgt3Wp6MYY5Mk1Ln6oT44mL/dypgZe4NZacuYk+WC+OQ4RH07ONb4LuODf0OY/y14jQeTlgTVWtwMcrIyhxD29Xs0LP+bkLgidjxYRUFHANNviZEWBBC/NI8BPr/TrZ9EqV5GZgcsE5qwX7MS55BSGgoD2CY2MnGkDTVGUHUmYSe4TrJ/HlkNE3FqnERSn0eV1RSD2AaBypf3Tx2i5aEBp5t2NM3M5ZUnjK/YSJRcXPsiPjITf8k0bEq7w9+vOCCrO0WHDbQ6aRDmtzP13FSKhzSCz3kAuqS9CLrzD5YGH2bK38RbUMGwqEmgrwSpI3rXAL5d8SfOHUE892MKheTAgUDr+IpcQaFzLPdG5uOoG0MozlTYh9ErJwHsoiFgFj97NFBmgj8vRNFLYmTDu2/jxSFrtk6vD9hhXM31U9dw7I7ky/t7SUyzZUaiAURWXbVViVcQJR/jxxVrSbK5gHfXPVp4iYGKJTTd8KDNEMHZIbl8EbwdoUlj3bvInCmXvUC2XTIiR2+ofsQy3HgZzo+EkMWsy/dHc3sEd1MD8Q2wxV/tCLpaKPoJWZCYv6feYrdGQPHOTAYG/Iki/4njs0OdzFLTZ1S6CAiPzsZk3wG6ZjidYdXxHJ3DrNuBkOfB+8eHEpnyAPn0PAi30qbZdZfx+qq/mFjsywmFNSBnavRxpFIVQocK2twjUNtAeaEPGocuxj96rkCkYG3BBXQSFZZTa+m8Hk/T+lZS0gdAzic4t2xj/7gtHCtdzo2/RvG9bQ/D0hWPKFMD6H3sCBF7p3KqR4FXViANE29b6wvw8HOmdwso+cBK3/7pz+/w6jdfY+/yCNBrzUTgkkLY1V8xWVqYsO400bJ/8xuBuPsSUx0vcLXInd7Fw1H31cI+LxzNIsLP+uChUWGMAndlJ3GyGmjPBf9n0T7cz583h3DC0UhAwAbs5E3A+1bdx+oa7ukb2HRyEa2ivXx7dSxNcd1W0EruAkEL+NL/ZRTiDhZZHKgMc2aYTQi3O4MBUKlgRJk//ketWT89HjrkIhFUHbD+gl/kz4Z4Tn07GletgtiBd4gVFD/Vrl1loQRfj0E2J58pkY/QW5sApktM3FjzPquCDxF80JfLRgExsTVWNg/bYMr8v+PswLO4V/Tn/a3v0z07E+rOWgNrIlY+Bv/Kphxn2/JdnBZOswY9WMzopB5cvBVGl8DMhpc3IRdAmceIx3RTLXf28/u0L3Bq9uBjsRtuvvXgPxuy10DFLhhxC0GBNxn7rBmlayQqJG33rHV37Y9QmElJYAldDiZsbc/SxwLOld9jufcmXhX9CLiZTNkgC7IiiBXHczNeAFUHsHdo52L/WmR6e3br7/J6vpDPnNx47fYSqyyBEGZ/dJZ1DV7s74S9FgEf/utdNuPAOVE3MU5GGvyScHGKp63DlpGur2IjbUFnm0KuMoWIeyqcq+6R+v4QcJTCgCPWb6AY/O/tp0tmZNuqzwiXJz/VV8ubhJzUXCGuMZDmxgSGDb0CF0YTK4IDkQW0OXgj0YmRG+TYmeusQWWA2nySQ2k5PFs5DAyg/BcW92b6NE7eXAnlJ3B2j+f80U3E9w+jWlJEV1k339/8GUf/Trqk7QQcbSBLoyA03chX134jwfMYVd2JNB3Oxb4+D2cnOeEi6RNNyv9824/GgnudO44tjgj/RaEIsEMjxigwMDynhC5VNjWdI4jySue9SyXYbfsWQ/lpgu9pMPj6Q2UdmPTgO4XhGwIJvSPi7qxuRrg7kSFstK61My3ou6DbDSrd4JL9j/xEDcec3yG96BMo+gkPz+949bNPkJm7UdwQcTo1i3fSSukWwbRaeE72Nz0HHWlS7mXCDj1TpgpYON+M6tA2+qR0g2ABP9/7kqL7Zoad6WK307Gn2lSlLeVs2jKabc+TUAlepmiqRqVRoxxPQPZg7IpaUIsnYirwoy3gFezFtXD7FUbazOT5pjcwyiJQuq9kkh3ozM38kWd1krvIWvCovEejTTAL39hFihwovm4NkirYwLEHMVzYOx7FCPh8xKc8r3jiymrrcedQQTTNfw6lPuIhIdHllHd64mdTYw1GEoqY4p6PnVMgOcBklDjZPg04d9r1cCP1AYF6JdQ+WnvaslnqeYAdS84zNGQ/dyqOMtprBzN0QNhrkP81fhId3mIwmc3YaWxJMSuwiGw4VPAqYGG072dke9nTXOpF2PcLCX9lP72jPsZG2kas8zU+2RmNX2Ycbb20dMoqKDKIuFE1ijOlcwkOP8ccW7jVaIsx0pMlaXuhPQH67QWpPV01t2mZvJGz1f6M+OIZtsn1fGob+bhNw4S1yFUgr7PB/X4oK+adhbGFUHOMZdqXWOY2DgfxEJq8m3ByfDprSrv/CzzvXiX/xbtUTPmTMLtP+WZEbwIdsvjt/gdsvyAg2CigacAVQuUWhOjZeOtH/n6wltZe5zk9dB2u9a68vOFl8lKbcU08wKFnJpDflExz5xmarlRSG7YOj17fE6VrJar/o+/29EWqassYJ/qOhsp4nKvCKetsBeY++e4M7Xi15eJUeAQOBKJbcRp6jyCt4Agp11OQKiSomhX0q+qHYIIF4r+AqFVwv5stC7ZgY/JjzTYltT0Cej6U41ckJ0Deg8TkxulFVkf3JCnYmnqzkFp6y0EhAIFQgFkoJk7ow5qII2TJprHq9g/8cPsHAEo0S4krdoL187nhVUXSSEculU+jsCWJaOBg41fEnjeRH5bPn+n/0Nj0KuP55nG71qkiaJ9zhDRVb9b2/ZEucSCD40ZjufIXDzsHYNR0cXRBES+FHHpc5o0+sxjl1o9j5ksk6SaypiIGd/8WHlqe47PtVtaC1T0jMJ1N5jf5Tyx8mrnzf2svGiH3AWg1ZWSEquls7WHqSxY8pIX8dP0fkuPeh/ozCIXWOdCCGYsFzpTMZXDAIwr+sh1k1xYSm2PmxkVHLI5mfMXwXc9aKIbOm0ks/XM0dYPqkKSG0NniSaxiMACjWn6l9eIg7l8PocGnntRrYVwLLWZSQC0MOE6LaTgRL2zCaHcbodmLLr09b8oTue79JngMQxD1Jn1nmTikAKHchrbgUp6xdweBiGTFWtzdg/j0nU85O+cslt+t4OR/5u/M2kw+bJ+GJdSXYMMiqh12UKxPROfxLDJ1OOLqQ2ycdYle/S7hdWMAsad2oRtggkdStHd3epN57w8sIdbM+SivLCi8Dt4TuNSlYr/YFjl7MSlP8YVHF6KyvyHGqmUvQMieKXvoI36B8ZnZ9DPdBPmXkPg95nbY8SCdkOt/YNuxFb/RLzLSrgjHx/IAQvLN7dxV7+TkM3+SFnWVHod4FPWnwdAGFbtIytkHR8NwzY7mdkIjFWE9+Mo6IGYN/sZampJG4Fd8hEHHXidnfjGuWf3Ru/TnBTvoFFgQ2cnYM2kPTc6tUPGIRUUdzvd+eUyuhVa/Hq70eYBBFmANWK89jkpzHfHvNxhQm4OyqQQ/l1LaHObgYu8D+naGCo7yjo0jK6s9aW4LoLF0IKN1zXCqL4QuRdT2IUFZF8mOzGT/C58wUL4MVI8Ce+vPgPsSBhRJ0dXaMFZiYpjaBOYnGWV6oR2ft8Bdkxwvzyjk6ijo+yZ7L3iyY+NZ+uS7Yh5jIlACVzVO6OqeY+qIEvbmN3Fm/m80GGcQ/V0yQk0faracwNM5BXynsqPpHlE37dApGvnmtW8QWMQg3wa9PgLX/ky3n4+TAo6Ey2myaaTa/C7pQeeh+BdovcsHTT307ulmvHYyyxPXYtv6OTRlWqnwnVI53/wKA8/60uSkQzcmn6va8fT71zd6QOpDycsb6JH30KXoos00FiQ2fN0SxkidBKnBTEimDROlmcwb9hyLjjxkWP9Gug0XUc5QIu4lZt2CdQD0Lp/BlW3T6HR1odqzGq8kL6S+buSnzqYmYhUauQafomBK29pxrr1Du0JGc++h2GVdxGyv5cK4w3QHWxlvPEsh0KTAncdKMf+1/wH7L9j3X/sfM4v/TLQib5TxbyHMfg+TXsbqt5aw8+FqtjdMpaFVw3DnVKYMtka5upeMoXJrBtqwcn59/hjblErGSh1hTAFI7dF9v417PwfiHm1k7ZQctmsk5NbfI0r1kBvV47itHYCiw8C4f6bRx7kJSVo39/2sE9LBjlAAVK2V2EgM7O2ECIur1WEldQAgxElO4ItHONUxgtywXN7tjmS+roXnu1+jdh30Lkom/mYiYfXjCXyzD8QYQSCipAxW10hxNIoxDT5Ni6uexdGrETywHqGdzbGIDHJavRuJmnaCUicnFjRAjh7y9KDIX45vmQxloBCR3gYQwNFeYB8Ng05xdPx59kzezpyKXfyS/SMvDHy0g9NZD8FTBU10lKTT0tRGeY4rrdPd4EoMBMxBEP/d4/6QlTrzcbMbrwxPgZ56hraaKH33fbontFEz/TUaGqDJU4TX9alQ+gd/SvLpUlQxTzGBr8a9RqfFF7JqwGJE4D35P5ggsRLoIxFyLugAW7aPsT4LW0Lu3aHJ04lzXVAkVDLXZ5JVyFbuwp6Hp+mKuY9tewTTd/YiaskADiKkrtOaLSCUGDjU+za2wfks/8qX2iFKXCboH7fFzVTK55tfplGhx/a1n3nPyQS3FiMVKJG5NlPgX4+Lh4IzQ87wgp0/7LeCaXVOX/HDqpcw2FYwa9jXOAiLiHcIhvaH0J6DQHuIr575gznXx7P97aVoh94k7fV00FZAwSYu1NaRku1Mj1yApGcAMwZvsEboBswCl368bbbHM+U6jd5VFCs68QOwj7UCLcAVdpO16if6UkLd7hX0fPPjYzo0gA8aA7lvLkbdKMIlK4Ie7QWwBhPSYteBvsuIVliNVqrDrmQZdJyEiXWcr1JzZ6CCgICbuF32JDG2EtwHQ97XUPQjvXsUFFZ6UT7sIded77BAOR0iFkLLHaTafH52gx/b4SR1xNtoKXedi5+5APK/ocY/nesxv+Di7sKBCQdo/w+nQtB8iHobgGNaKXphN0H1Ci6o3fES5YLbIHDLQE9/WnPyUbhWo8yO50iUki+SrXQml7rBIjCAQsKBCQdY5iWE4iwImAuSIDYVWw8SAyp3k32lF95TI0ju/QYEPofDw2sMdMmixBTEnJtLcVG0YTex0EqDBUwU3SfwfQWVYXbUvPolNe4icB9qBfyc+8CjxBuNcwDHh5+k1a7N+g9mA5TvJFgdzfZ+DwhSxeFYGsXxqmW84PIGZL4Gid/Tx24pKeua+LTkLjc7osmSRLJ4xve8v+szNr60itDOuQzJ1LLy0vMEPn/BSqOl8kVz4QA/udXS17sPvWS3CFWYse0pgL+tVGwldtc41U+HU2gBozw2U6VXgsTGSknZU4c+wwPXhBpuXwskqEfBSodt4DgaolYhkDmwaPp5qiefJb1Sgllg4BmxP466RlBb58DngqfT5WHhytWl9D5+hhMByUydZQ+BcxFWWfBun4FUkI1MqASz9UBSbxPNmnY9h2Mm0PuWnpq5NZzOPU24BF5y6otMYGbA8BKuVU6kpiEHXb6elnpHgtT2YBdNleo55r73LS6NLvQfVc+6YYlUdn4IWfegcg8Gj/Ws/vAvelQSDr2xEFshTLJfaz24uw9DcGk/sje2sKdiEMH39zE4SQ7JP4JTIjgmIsfEWMsgGmvsEV/OZ2P5V/y6qRf0NIBYiTJ/Oi/XeLOlNZCC5KXMTNyEwGKEMXmMfX0sdjcD6EyoZNqygzh7PAKeWjMZonmNnIJxaDpt6VHrMRhVCPqIQelp/XGd09/2xcm/Cn08CC2Pdrup20AgxKQu5Nfnf8W+1Z76y0kER0c+pp2hYheBLveZNuEO8w6Hc6jTn8WxrlYHTd0ZcOlLffZVlq77mvWfzWTQ2UHcHfIEFAY419OL3Q3/MMQswCQyUY4vzLRwd8e77L3pwQmDErN/CMKuHuS6R7o1gXOpuuJByvUUjM4GYsWVxIjbuV9z3Jr90ZrJi2lZCEKvcLdjBX1FO1ETB4+ydUROSQy7NAjH0kxkiblE2nRjNHewN3cFHrZFeIS3EtJxnm5JOWMD/dGZRQgE/wvY53YJQ982SnqK0Dm3MkK88qnrzw9bSVbdYPTvxqHukPDqLGv5jfYSfrs7kR63NpT3M7mjTiFJ8Dk8/ByEMtbtHoX/tViEE/PxjcunRSXmRPEC9uctp0fUwoZpa9hfd5sr6gCGOHbiedOXUR7wlwZumCAu9RaFbp3kjGjBRmpPuKw/PcaNHA/P5HjrEiYJDRRVlBJe7oUqpA80dsOpdPCZTEfyVsovZ+FfZKQiagSlmiVPtcnX6zl2ep3HbGlD2XyOsm4xgrBl1LQ18XXBQ5QaCcKGbqIeRlGVet2qR1J7HFXHXZJnZFEiCkN2rg4XYTtS/SPnZcQbTC/wYbanPV+0TuPL5QsJljlbs+PurYTs9+nroOWWCFBCAZAuA9L3WLPFcr8ioNqJlisZTD6fRLVnNSHPfQjNu6x0x44JxDskIP6nng5HDVlpeuwNNnAqBpxTYdBJNjWF0ii6zYEla5irhsrOl5jt7o5eWEny9AbKvWZRpFxAmVTz5GXomsjqWsS1Kj+iJO2MLFmDl30n2D7SwCrchI1uIO0tdoTZ5LA24w0yTb+BMQQSvkfrkIYmZxM6IOjhXjRVXchH/AiJG8BnCj1VjVxsVGKRdVE4dQ++tg5Pnh1tpbuqKPFm5PFBWARmGhxakXZVwN2VELIIXcBcUo5WotX6wCOwT9x0Dc4PZUUqjC21MCKlgk3dVUiMYXzwyO8vEApo7PIDQO+t5U7vcnxlw8G+F/TdTVezK+c/aqWqVoaxLpD7Ki3DJUoo2QDOfbAI3uThjbNI/ZSkjj9GT3gtOPSyajDahpItbMIm5S7VjXEojFHc8shi6EQn6K4GuStZtz7HP+YSWR1SljZCL1kif/xrDIbxkO6PX2VCjzWVZUbK39Bdg1pgQ/9jz6DudOaBv5kFMidel52Eu23QdI0W3QqqD/nSnVJFZ1EZ2GgxvK9H7joAOnLJq8inseweiOppu5JAlmMldJZaAxvENoR2J2Lu0LKwcw6TIzeBzEye81WmRX7M0dIVTO/9gId+DbRl9SUq/BgC2ULwm2EFxYBykz22JgFlUR2UBAwC2zeeonYqjPTl6OsnCK39hCk7B7Px4yzcsvowVu3D9VYwWcxM3j2c8qh8iP0HLo630psnQnvvEu7cTCMqJQ+DezV05EHJNjB28UpeAOIbo9g58QQTlG0M8U9ixPXnofQ3vFxX8tnCHxBYpBgfJTcI/gUACc0GzG5NlLg04e+TzS2TC0PCX7XqaXtasyMtvg2cH3eICPUstBEpCPussZ5HhBI8RMVMmnCFAx2lJOTdZGjYP1CVBVJ7Kode4FDqVSY1fsnKX1eRPvsU+N8AoDJ4Dd7xAynrVjPql7nYtiRgTjOBbTB0VZJnTKW/+CwKE/S/0J86t0cboclNYNIh/PFv3Otd6VA3kJPkT4CPGWOdDVN2arEgICnFiFhoRhDnglsfMQpKoNsBsIDCnca2GwRKjtNoNvCnOyTJM5+aByUaIVN3TiXqYRTXgYeDknhNvJn9xhF8NPo0CZLZDN/WG6e6frgklz8auK9x7KaWY7+GILNxZZkkA21IGWXLTlC3ZyV9fXfxZk8XmRH3mHTxIzKON2AfNpykFYN4772FzH8Jdu5U0W5y44R7CZiaWSQZzbmygZwrm0NYuJl615P0iMzMWvM9GvP/VTNngkXA/Ko+/D7lEG0ubcRZns7sCyoKwKFJTfLDaDzdWrHX5gDpnLdfgWeyhlgfX+zOVJES9ydcegSq9dvDdK8zVNeq8Sl2Rv7LEnKX7ST25VfBYkEAbJ/syL26oVxqA61BiEf3bWtZpQ9uThn0un2cHqUD11NOUuPTjrvIRKAS9tbF85c+gbCrtjh6mTDYZ2Joi0RV9hUovFClrwehiN/vHGTI2UGoZu/jm96lT7VJ11PF0ZhD7Cl24vLl/ujDpSB3x7PrALMXdrKt7Dj2V64SsTubrggB9tM2QMUulAZ7eu0w0iOtQ5/UC5NFgDj0yfmtd/hXLNjyBcvyXuS4FhQI6ZXxk9Ux2lnGDW0Ucd0SDPpORighRGbkdPkibHov5nhtNOVtRwkfX0N2r3/YKcrFq2WEFeyLWAnOqfRyqWX6R2fJX/UcIV8kMvrrX580yqBhuUsjl4zgcqGYkIZCbAVDoKuaOfbdXAzoxuIkx1JhJksPV6T9SA+eDxGvM2ztuzgflCGPhh+Xf80BMWwS+TAvfgpioYGTDZ/B+FLKM1sJzlTySvM0fHJnEed2mpNVwzjSdziq+Nt0qwNptruIVCfFIJHxTr/J7O8egPrQIGiwpb7XQJpDCq3gp4s1GKCtR8ScelAZZHgP/QQAhydkN8zrKcM3P5r3e1XR9cwWqjt681NPPega+T7/CjUPYsgYYotl8k8o7AGeAH4Gb39qOwTEOR3nGw/40VjOg4YMAh2yaO32pj1gBXcCYLrdRt53goemarQGXzr1DkSJs5lt7kNtayru2BAgCUVosQbPhjnfZN7N9/HLt0XWUY9LXQaaOWewdbDuBeLdT2KrU+J7agDCDgUx337CWLmEf4N9ytoP8bhsi0ksoC6giiJ9NOP07QQYKhh5/DlKwrX42Mcjue7J0GmVoPCCzlI8dPf5xRUOtXexO/kscrOFQT1NVFh6qOiGSOEdfMTQZISHXSI8kBNoWk8/FQSVwbbkHOTKdq7eSeGTn38g5eXUp76NuOQzyGNqubVnEEaxEUQKdj58F3/7+4jNrUyXnsZhmpm7LQKiywJwiH5aJ3Gc8ndut8ApzyiiymFZ9DoWODVz2GYdN0cOZPIoT0oKX2BAbTcF/o/6yX0is9oy+VYJH97y5/bPsdQPFJL2/L/q5dNAba9C9MUG3B/9Wx2wBTgAtAN2wHhgPuCOlQjWwQhexd/Qf+xlGt94i3cqq5le684ngwbQrPgS9xGfYXOrmW/v9aXqoT937v5Jz4VK1vrvYE3IHOgsJiPkGLELujjgO4yQzASm6NIhwBr0v8GmAWP6ZcZ5TsVn+hwyD4C3ArixkFapB1e7BdhgQRFvy4aXNxCpcmHS8JvglISwGnQKe8xyG8onlJLVeAsPQbJ1X6UKQHpmIF+YrvK27TlOWiqwEcJUpzp4tC6HSA1cN0BXdRcLPhjPtTQXxJHWtVpn1GFbZKbv5SDk92oZ2zaWW3260Mb+icwRzDJXJp/5iAoj5JNJh2sR40uHc3v5jyTP3U1S6TnErZk4KgcRdX4vrYFtkPIeOMTTYBJzUl9KSKs98759CQJrkSU8oVZf5H2Gvq7wWpmR7r21NLY0wnIdZH+IBE+SAjvxtDnHrdq+RNz+leYNe+HTFyFgFgKRlNE53viW29Lg+jYjjJXsUy5iwrBrYOjA2FmOi3cu7WG++OTbo7IL55rYD99xH6C/9Sq1Jhl5vYegLzfR4HiTTgJoC/4Om+bf+DS6Edelmxg9ZAXZJ7IRmCUw+iHcXwWeoxmxdQWlHUKc/Hq45n2KfrK54DMJao8jNGmQdbehbKvG7zVXZiW+D01Yf4BcMIzvV7yCMrSJuxlvoFYD0keMNQXfEaPtorzcGycXuOQN5YYrELQe3AaD2AahXon0RAYDr6bz82vf8JNDN8US5eN3qhM7sudYEgIc+evTOdxvmwLFA4nW3kaQF0lUthOOmWpe+nwFoS8c4qAuFvp+yt77X/CX5izJUildceUkiOtxNJbBgw+h+hAdHsN4//33ibJMB7CyRNx/z8rKIXcjo1WA5ehY7vsMpls+h2nO+6z+pqTNgIBrfWy53PcCy8//TM6lcIapH2XuPfgAQl5Cqe7i2MztCPVzcer5iiH8LyYR0OTSxPM7p+GXG4Hvz9b9XpJ4CefHVbN88gDyF1yhpUsC8+H5uJUk2x4mRyvnTHIP4b4H2VMK0VL4ttSR3DsPKeUs3xz7He9wL3wjZtJhswCtfRf0wPkFvxKrMnLY0k1G9ZtYLFZfmbHPDWpdG/ijA9rMyUz7x592FxFqXTFipZLHKPj/YSYQCHB3d/+/BIn9P2X/Bfv+a/9jJnROxubKNKjYAJ6jqAvYxE+HJVRonXAV9MfYA/Yy2J/3GhPCv6a0Zip2L2RSbxxDdOOvODsDQkBtzT2/q8ugttJEjk8l0eUQJotjnvsORgd8zoXyGdxVbOdMPThUzcFJCxKLnm/qvCkUVuEl9MDQcYyIqzcIlpkw6BPpjhdaBcF9rEIyD6WnOFgNUtezDFR8iI9IY9UQAN5ttnLy+lR44pnryobC9TwbMJ+E4k+IMT3PO5++Q1FcCwOHOSFXCMHQYuW4HnmPjxeOJvP3t9g1/SJTI88yXL6cBTFefHXnR2LdetFPlIeqJ4wXd47C3ixCPbEY3AZay1vMDFHp6OxSYnekgUzfZF54zwsi3rBSHAFBvaXEpG3kw7VTcWzQ0lZ3wtoBTsmIdM38fHEkFbciEWpsKAyxQzhSCA7xzG6PwSUyB5VzFL95yRG7O2GR/GUtW3+GtMtjiL81g5pZ9ZytGsEg7+NgHw+GDoTGdracG0PVxUSaFV0sUXcwY18XW8b6U9URQYPzSpqf7aCx04l9p+PpcpQw92VHKPkFWrMYGXqZb3oVE5JjxqE4kyjtKVIXj6LixmnkyR8iQomzCAwyPVUxYeRohxMuf+LekdGDX1A1TQYRnvtGkR1ZQcx4BxSNl6gQn0ATE4uj8gR0Q7ik50m57tvcTKpCLHZi+FkRwg4N6tfL4PpyaL5OhWE6b9coqdHI8dYG0dzVYHW+lGwFsxGhtx+355fQUf8Grh1TsHf5EDqbIPt9CHmZJo9O7nuew67NjskP7Dk/6jlizZeh9A/wGoNe7E5tcF9sm27Q4FDP9LJk8s+PsQJMg07wT50fpkYjql71NA7dRr2sN5tm1jJl0yrOlmyit9JCc90bmIxV5Jo8SfEaDQo3isyXqXGoIcgpiY7poyjzfuSIMPWAQMTHaXfp6nsSU2c4PmKQo4fOEjidgSLoDSZd86Jdo2REQCtzFWLyHSfi520P9RcwdtnQPegqN4yACdT/kcvxnvj4vbrWueJdHkHatUR2CONZvnE9xKwB1/4cvraHTUvP4dZlYUjjc6zq8yTS26fGgz73AhF4mvFomcqAkF1WEPlUHxTAfxDlfYPO4DzoH150EMOtEvAYgdBhBK6THlLaNIzQrWdxde5GQQ/4TgWlL4FZ+9EmFNFiM5yzHe9SLfZgsnuhFezzn4XDhSP0LbjHdQ938mKMtCk6yYz/jd53n4P2h6Sqk5ijH0ePKIkafQsi86NxVPo7xKxlXZuEan0VzcI9JFR/gZODEZFA+Fh/TCQWIOqyIG5wQmmyQHctlPzKSJtBFBbvYUAIHJ2lRmSjwU8w5fE7cRJG4tSpxrG1jQ/fXkhJXD0nZlZD3CegDuf5Oxl0iEsYnCZm1XP/yp52SQd9G6PK1WhFbcy0NbLRFQrl/XH084cHH0HoEpo6b5D7dzoVNmeoib3HH5ZCpibdAosFoVnDz2/35Vz58+xtTcen2QenMBNul0cRKZCws8UWoVGPxkdDVWcV190hRX4ZU8kmno/bSE1HKF2h9kz5+QOi5Rbo1wESG7SFrQSWHCKgNIDG4T0cK5uEp0sIPh3/gE0gtQ4v4VT1HZ32djSbodkMAtsg6zwHBJg+Y74rDNDkoE8Zj14mAOMfoPAGqSNqXQ3L7sGlA72Au2icNBjCVyLBCGYjJZIw7l/zx9UeauKDyRdOZ7RwG9QcprbeicR7cTyI0GLfPxcXMRC9BjryaG+u4vVVf7Cs2oOYLaNRav2RmLXQngedxezstuPM4FOYJAoW//YcjVFl1r4o/gU0+QyTJ/K9P6y+NB3/Mx64+D0HXjqrdqTbILrXjCUvN5WMswkITSJrZJxtsPUHFJvLmKfpgCm7GdqdRH/l05p9Ip2AVR+vIrd3Jh9/+QPXGQC8jVGgoPBGNLYKI21eBmSWjieF2nIIsgjwOD6SupgKgm9PZ1RiiTWj6pEp7d2wca5HevlPAnN9qHD8BZ47BjEfgEBIUJ/NNPqFcqnfb5Q4lDJE9BIp3gd4OWkROxpO8eHAz6jvtOX6svcJTsvCbcFT1cbDezNzF2USXgY17TGMl7o+df2d/HepalGif/tjZjpq+S1/AWqzGnuPn8j58FMcun7C83guv99PIynDSsNC8EJuBl6nUNVAvb8H95J2s9YligyOkOq1jxmHs5mvrKLaF2oajFwtkLI7dRBxrSdY4wgrW7L5YUQzTp196VewG3d3sLXZR5YOJAg4VxfJw+/C8JFZUAmq6SX7EfKaH82FE2jraSMrcg/Nal9wX4Wt7dO6YgIB1IQNoEdSS3PyRBQCIXiN4qyHCNdPNjDqTiLNnzjwZeQLxMp6WwtoK6B9K+nDTnG5xkCifwxTwqwUsvhOhai3EWVvgNoO5N1ddHp30tdW9uShpi4mus1i7qi7DL4/HI3zdrybVjDJbjZYTKBr4NX74Qw/n8TdvlfojsrDo/kRkOg6AAAFNbTGJeBw7w69z5Zxpn8rqwSCx9nsF2r8UOaLsPcrZ7OljsXOwbi7uzO/bSbHXI4wLuAWww/HE2kDTLTSaRP7CRd/OED4+SaaWm1Q5qsR++Zbx33qbyCU8vtdLbVOnQy8q2ftb6sJWG9D75JtUPwzluEPeOPXQTi0OmI7dh8LHQ3oFIHInJJB4YaAJl745QUMEgPnlm7mKzerfhC2IeBnPawXhZXx04KfaHZqRifXMbdVAvXHwdRD78TviH2wA622iRHSInqF1CA7P/Txaz2cICKwazuB9a8/3cdYcGh7DdccGzIXljNm4FkEKn8o/xuab1Fpt54LRe7Yt2nJH/ghFix4qgRQvAXMBrrd5nJ7dF9q7feSH3sOuSnWmjU43DrGr197gUsjt2LbVYJSdxeLroWhkhdBYg0YkhubeWnBEV7clcKAL5Zy+4Wap+p3X5hCcdJ5aoQORPqlUh8+kjCPAK5qnKkcep7O1iRCrofSfXEJGz+8yqt9JkLrXVQ2XXy2dAse0nTChJWYpQZmmAww5BxYLBQvf5m4625c7ZvHZ2+eJUQ22hrc4j0ehDL27ndAdvkM8uhKSkvsCe6l4dPBwxELjdyqn8yHwm5aA3KJcYxBLwkgxqRHnL4d0q0OEZPSwsZXNhJe9RnBlW/iaPf0fDJS8AXuhWZ6Hf0SjXMnJsVAiF6DzbFbDMkZgL2jIzvGnmWaf9WTQkI5k2y6aDbBMU8BK8J348tongmab2UXMHbSV2wmV6siIncNlgeBhK+OhQpPALqkfpjEJoRmE3N2TqPduwaBJt9a1qQjShhEjYeMmbWQWKmjJbANjidan622ZuQciqulJsCB1089z84iO0aoiq0UzgpvnMVVzFJBQY0c59OncJOdA/+7oPDiTLCOybU6lAZ3JCFZBLs2Px7fKmMLp71hVbU9ecF1iPQa1Jb/D3t/GV3V9fb7w5+1NXsn2XF3dwUSCE5wd3daSkvdqJcKLdSgXmpIC6W4FHcnQAhEiLu7Z2f782JRWu4z/ud+njH+57lfnN81Bi8YK2utOfeaa645r+srEQ+lSUv272Xjnr5IJQ6MOz8Qt5QM8VylKH8+zniV0O+foiTlBtOWRFFpjMckuYCTup7mHm/8qucinA1GOlrLqilfUWqKhmMrROZsyk4MRmcWv7eZL7PcuXVsDcpJrY8g1AdaKtDfT6YoqIhrKdcYw2qYb0F9CsbsgaQkuBwym0r3Dtb3PtjL9PmCwuvrCbnpib1LL3aNw+ljfQ6z0pN1lw8iYGHFGB/eVtawrbSbmKlFWHuexMvrSX55509QOtO3YQynOmzZeGMbEoucMPvYh21ycDTz+HMvUGyAoQ+GiOW/JG0im1xI/3k0Y2fJeS3uIidMBY8cf6U4kqob0eS89wlDnQxYpEPB0Ek/91tcmXoPe/tEosO2Ms7rX/YWtiEkDvuEqBQ5l+9Pw052CpVbO6Q/DzefQDL8NkcuDKA035mn8iejlIzD87tt4rmTSzDnlWM/TUNGrwvJI0/gJrHG7DyPpRuHYCXrxjT8Zz546wOGXxnHoAtSCisvAAIo7EQlCKDLppuSgBKGhpXjrul5pE+CIHBl7SoUle6MAJz9zsLJfpgktmQ1jcAikaHwdiVq8H3Utj2iJBnQ6fMCBt88DMoAqm9X095hxYA13hyZJ/6m35ZP4De9nkz5BVp+n8D0fuUQsOeh/HnzzU18/ezXuNe6E58dyu/qIsrq5vHWmGhaTPD99+PxTobF/XKxz5XSVB/EPds/iYsTfeReLx7O3I5cksPykPjU8EXNYLb83SljF0s7ZVg2P4m5pgZrTRdS/Rlw/wOvInfae9p4V6MnOEtDr3EQ556fwMDuSrDxZyyf0Jn3A1pXD2pNUGuCmPAuGu8KFHWP5kftHe5LD2FMaebY0GrGOK2mIDeZguZk9LJacG6lxakVx8aRwCVmh8wlQrhPXU8c1V59uHFNh6PZliKnYI5Wf8JwZ4toPWDlgYCa8NxwXJpCCPTMwM8uG8E4BRC/w9v1k+j8OYLQfnUcCTbj0+0tnlv4HbqyZJzvXaUzJoynE16kTBsDTH34nJ8Y+yRFiWF8WeLJytJY3Eb48ted1/npzpcoZHo+7z+P7IYh3NgylResdaz4VsWBOVZcLp+NXp1K7dZAiuVhvPXlE9xrm0OtchfTfp2GgJl5U1IZtK6EL470w3KjD39OMnHs1G787LNp0XqwyPYOe8b+QINewEeux5p/VTCBy4Z2VAvO8qdVC3d8KpihfhxcB5BdcRevl37lQPMY4rzGMyp1Egq7YeI+6+xw4uXB+FYGUNpqw+GoXMxSLYLFwOMayDdArLGRhflJ/NJtwmvfJHIGlnDSW89xHbSaQGVaw9SYWk4dM2PI0dO8xJYRAdt4of9S9ue+jLtTHRHKVvKX7yNVBVLTu/T3PsyCmLXcMmVRY3HFNGIf8i3jmLl1Cau/3ghsftivYNUxlB4dWP3VjyovDaY4W7g5ExOOHI1pIVx7hOD7Pkg1BvAvBYd45IO3UrVjOhV7Z+E02ZWivpOQ27nizlmOzBvJlYqZjOuuRO+fQeCWAOo6W9gIvAYMB55HLO7VAduBD4D1gCuiXXKNbRxBsS9T1tXDz/fW89MD2Ue3llnQegwrqTt+rU60lvtQ6XkfhC5SXX4BnxkQ/SYdZ49jTrqHnjHEXjBTYRqPaUgBUmMj/a0XccpnDAERQdy6K15XIgHKfkenCObXoWdxi1MzyH4ETS5NHA1sEf0qAS+gefY6mtpmMmvKMtwOeDLR5azI0L+5Unyt9TLchXxSjrVi0/oacc/d4F7cBQDK65JJbU9CFimjIqSWJuemhyxFiSDBpssGn0o3pO2d9Gnuw52k6w9ZSoJdJOcq7XFudqKlr4FtgZsJ0c+Ak/15PsyFBQF/keQ+kNQ7XXSq8vi05gqjhh5DrglDIjTx1HdPgUpKZvR9rtm3MCTiV/5evkgEKDqTjHObgnv95ZQF9meyvgWy3kENhLhOZl7yYb7NK2Xy1elk1vYlRh2IHeL2wKFVw/QD08iOus/Z1C6RregsFqV79V0k2a+EeXfxqRnE3ogMkvTd8OcnKIDsDrjsugVvllPc/ySBdS/S7fUMVmGjWfDRRlzbPMAiQWqyQWpRgL4ZKveBy0BMF5OYfzWZay82Ilfbs9AlH25uhRGnMZhCWbemP2dKV/C26T7xXQ7E27Q+HPeXGMulgX9hsvPEQdqJRCIVVahi3ofGS1R0NTJow1kOpYVSv2c06uEtIEjA9gERQG+kyreCm0YZDTI9vQYJwr8kWRNcBjDy/Bz0VlbcHtVCC72QtpxQwDqsgbvuKsYH1HHL9QadFidSYgrgUCARsjHEZ8TjplKjHnSFjb4lDwkNtN6ljyKEagXItQbisuKI8GgX7TyKNoPCkZ8KB5N0IQGXqY2sGR+Fr/I+XOWh0sNsWxN1ZiBDy/mbIxk8uwFVdxqEvwgmHes8zuPlU8/wqwJONf8rS+5tu2u0dtqw36OewRojUgEwdPG+11ec7xqHtyqUg+MqedH/BwBsFK1UG4YQJb9Eqhr03XoKz/TjQEMhu69sZpiVjA97deJ8kJfHr8Ufccn4Hi6KEOIUcQw7OI2d83ZSHFbPYKUVsrZWGv36Ihl9EV2VlMX1Jt51jSQy159CaS9BOTsxNIQAc/m/MSQSCe7u7v/9H/4fiv8U+/4T/2NharzBw3RXy20qPNo4VH4a52Zfpgd480TcJqy7PuCXjC/4JeOLB3+4GHuNmVDrK3hoOsAyFk72h65iTjsu5shrm4jPiGf8h69QvMzEDWEhEwI2oJJ1IWBCIlhwnneOL7yr+csUya/1cVyIq+JIxyka2+wZsyCdowcGM/HoRMJTd0B1mZj04p/J1TdLS+iu11jy2nYIXc3Gsjiu/vUiXoIj0kFr+Gn4dPoqG3Ap3wqBS9G0QVdSNgoPR0YH7MRbkw/5QMofoHSk2sEfed8q5NJI7Lo7cHTwIVWdi1kDrQ7BfL16P392HuPmX0MJVViQkCxS4m8sAZdBrDfLyKkO4W5LNZkBX4Fqu8h4eRDZHSfYfBW0CwazLvQaWnOQ6L/knIKACU/7bqpdWrmTUsy0kBwcjC4gnU2VRuDs7D3MsAkh6moYctcelL05MOw42EXxwomZlFa7kFl9hO3XrYlZ4IILd0VvMUMnvoHfU9uZQ31Xf5w1aoaUzgEbcLMpJ8/On3HjfiD71jSmbJpCWaJU1OX2mw8+MxhauJ2JqT+yWpfDtMUbUThHEG71EzGhx8GhG337E6T1BPBzhi+bxi1Hq9bypqkLEKXD2tT9eXHQ79h2ubDq25G81a7g0JpF6KRedOsdsFhAUtrBooOL2D+0gZd/KoXeBgquXEA9bgnV3WG0dy/GxlAOMhtovgEqD+61RuP0ZgSuEe1kDXuW0ARvyF4oyskm/8KTtW8zIayGmdcb+CBkMjL5A5Zl7AfgP5+Ys5PxESBmzyxsO21xHnEPutIhbQWMusrN9s2kzd5HuUlGqyqdF6xjoeYf+TynelsmbF/G9VHFnHQyITU6QtkufKTNPP3N07SEGjF0uaFp9abjFTNUfgHZ63jS6msCBHduNT6JtvEJtDaIBQm34eA9leKzo+k2w+tXZ+FT2kTkWiuoPw9SFTrX8ahO1hBZGkCFQxu/y2G18xRoHwuhq5F39XDHF94tV5N2dDAHmsuY1r9A9H2S2UDch/ze7caFkyOpCPZgasoDWY87L8HYW3g3/s6OyHM8k+uAi83Eh4bfhqFH6dqSyagzOqT9DMzcP5dMQz6xg8TfQ++SytCGuXTfjOBwYgtyTSvuMqD8D2i5jSE8n003tuHsDE6DJaAsxaDpB7GjAbCUbyVu4Rmu3u5PYJ0f4Zoe8I4Sk+QqD3rbxtFbMBSdz07abG4y2uZFEiMWQ8RisFhwOLuBIfVnuJ4/i5CCOiKXHKPdfiR2Ca+ClRuXOnywXIUQaQ99s95n6MhT8McdNpkFphQM5cfm0bSN7eDrqM84rJpExr3Xof48vo0ZDL5xDKlVPFn+o7jZ0MmwIXGibK/KE5sv8+nXmkmtUoayR4VZLxfRZUWbYUYT6i4Vaq07LtI3+VN/lTnR68DYCYcDIe4j2ss9cGzxA42ORk89DkkVkP3gmQQu5aXySKbkBdCbauLg2IPsd1fCHjtI3AQ2T3C+bB65rX3R/NnAjI4Z3Hg9E51bKiVnynn+17mUxOdR0l6PYBbY0GJhvydIS77lpZ2P01x+C8++Q4mtNuMqRFAvt3kwtwrYPLabdIOAw94mrmdGIf1NR9xUMZmXe/UYhW+/x/0eFwbdT0Vio0WwAI3XoPYktbJgJt7woLXVjvGBmTyp7IaDPoAFot6E0KcpkY7A6G3P6RnvMcuvFfPVUVB3HMZlcEc5gI+ffwX31pmE19bQZukVpU5uPcUvE5R81f8y7fqVePy9YsrfBLPaSLt9iscyxyA1KFi+/CRrnBq4YnKHozMAcFM+xeXBR3BtjGbilXGMkjwolAQuA1MP2mPnOf/XMOwkOhLn5qH0Og+aZQ/ZvEfz4+l32I3uqccYmViMzNhPlF8WZGDjj1liRV2jHR5mb74edpx8k88j39gkVRa9SV20e1ZxRwdRqgLYKSCXvs07K7/DmTCm7/gM11odJ1/axxj7dLAJwqH0KBPf/JW1jT5E/JxKm0fXPxcdfopnvvucMskpRjc4cP9mFFV9y8UN14Nv5NPjnuZG1RQ2tSo52wEjpCpCHG/TqXOgxRjG7d/DqC90oCQ6h+uO9TxmAz/+CHK5eP7pnlAO9tzBvuZl+tSuIXbuo9n7zAuXCc8Nw7SmkynOBl7MS6GsaimX9GfoUN9jrmMXcV+/xzv2wOAD0FUE/gtoCOpHSWg1bt1TiZDDdHXOw2vOCPmexx/kMp/cOR6ZUSBz5jLcBUf0TXtZHvQh5fs1qA3OvOA/DX+vQu7I5lAOPNNkJNTahksz99Du0saCsBoOGccwLPwdsXjkMwN57WWuzcpgQ2sGfs0r8LerBk4/vL/M1MDowMNktvvzgQv0V5nhgCeDpOHUxTVwBXu85bGghyRVF1wWx1idbA4RpQa8TU1cq+nHJyV/su85Az7Gs3AklPru37n+STOmMAlODk7UGjTQbxhEvw0OcZSvfons36YyA7DueZzYL46CdKXIIJVrCPUp4upELeMimngh/jQGqwDkvaXgEAeGTvroFzDo2XvM3vIUr7imMcWpDAwd8ADUYNNqxZgT46jwqeDCsAtYAi0UFxdTYbhDjdMuyvV+BGVGkmFQMmTaJRh5CVwHs9AplLtOw8h3seWXt7/A3hzESoDAxQBE6UHVAPl2T9DhpaVO58TiwCWg8sROpiC0NBW7ZiOfTD/EVomB7L7lRNXtgOwPcFSlMnloJier7On/x3OUTj5MQFSpKJn9IApjKrmktTD1gepjt+8crFuui6AJXQsBt3YgNLUjMQ/Hy3QLy9AGhLDnSN6cwbjbRoQh3TzlMoc2uR+CIMrjCJhY6X2by0Xx2Eg6ecYeDpsqobEaKvfTGvYGP61Yh9Lowvst4cS4XkAqbIDINVC2E7VQB4IH7u3TsFzW0S9JgIYrIvo36o2HbCMr6/u84ZfFs9LDsHOVyLLv8xXVDiPof+FT1h4SC/dS/aO+ka2CC7+knsKtbRLSouepUgPDwe/gCI6PLiT2diptzVBuuYSNMoznHBLA1Iu/vAW92hmZwpkDY7aKfRWArA/AyoWpfrcZ8Hoz6R1BbHGMp1EZKM4XB33AOQWz52ZqgwZiU13Mvew47NamcC3jcwIc7lJsXsLezpfQS3o4Lt8NwEzd16hbMkTvRvso3rC7wjh3mHukF8/GXcgs4/k7yQ0w3XYaC2Ly2HzjddSeBjSVH4GtnBn6u7TuWYnGSqDBN4hJMf+stXBK4qMfRqO6YkQzUcZtNdhIVCBVwPR6kMj59cTbdCxpJWqfDKeKi9i8OwTGZgBmXKqMZJliOV05ns5CKWHWILTniPKlgME9js/eXU5sjRs+9q20rz6EQ2SZmBj1EAvHxabV3MpT41hTAFInLFWHEDJeghGn6RIWs+IrG1zq9XS0NfNtSTTfv7AQqVTJdwfW0FCvItG9D2++PfWfPsV/gqfEi4/f/Rja1Mhf+oZvfQs5aQwRk+CN17CzNJF6NpVGtzbc3/yeIY4PQEw7BbCLokHRH6dJ58nsktD782w8UktQue1kzcSn+O7Wd+Sae7DxruWg5ioHijxZ4qBjpfc0sWDdlolOuQTnEph6ehDxd1VUTFz7yBhM9cwm8rfjuJXAbFvwE8Tqmp92E3O7sim7uYR7yaU0yG7/c9LpQYQoYln3xs/EmucTnT0ThYuSiZc0fDk2jhdP3sKhW4a8OIQem3JSRx8i3aiBsj/g2nxwSmb/byOpudSHPlOsCHFrwFEqPCw+tcQd4ewVqDJCwp0E2uzb+K9R7PA8lYnF1OuusfJSAMOmhjxyPG6cHtvIRtZ3u7DHUs5FmzDQ1rLZ+x7rWiDD1M0XuhIG90gYqH5AOcn/irfTTtPabsISs4O0gGu85iWA0gV6KpCWfM/2HCn9brvhEq8jwj0bK2OjmCCUyDC3nuLlKU/xwp0XGKMGX5kWrSDwRJ9nSPI6wsv6pQyzNdGcdJFlQzK54uQG0TfAawIY2kFhz7apdzg+IoMnGiWoSjz5FywIk8yOs841yOwb+HDmTWyde8CpPy0d7eyo6cDBsJvqQT581ZXATBfx3SVwGTXmMPaFq/B2teVCyim0khoGMPvhdS81xnK57A4anTujb/bjuMyamMWOUPgdOCTwplMdAXJ71h9KIjEjkcc8P2ZZdxykP4+7YQjL4q9jb+tC9+9x9ORUM+eFb6HzWyjYCL4zkRolDLo8iDr3On7un0Zs9wv/dErpzFtFs3HRqsidlInHgL9o6H2MSRIZOoMVr2x4jd6UeyQUJ+Gs1fBKx2twAXAdRpDfS+ydsRLHIOiTGUunYwvWslas7bPwGzAR3W/VxN+zo8q7mxPDQS7ZxYuSL9AomzDLa2j0k7C5RcLeS9aE6VJpUG/HyQtwi6PDyoNNK97EtTWVweeXc8tUB8IxuCHStuyjv2RpdiI9uUH4Tt/L+LAtSPX9H86DTy/ezuodRciEMgameyLzDIUqEYzSXd+IW2khFcXlvHh0FC3hXexd9a/nbHWZLrfLWB+Zhded6bQO/Gc9JggWXNvuYl3ojrrSm3Z1DxalCM4a7LebYx0zKc4LoctdzSunr+Lu60RUIBjNooR0ncKRK07XODH+GK4TThDr/Bgmi4yiFhH4YJEXcdajAlWPivcyPJE5eFH+rzGYLw/lmxDRQkEAvOXtcHMlVarJPG57GH+zDpOdG29cuslj/SDsjuibmm2/ml2/FCA1KfF8rgAbKUgsOn50E6+7o2AxJ7a3Y+crJSolg+DwavYYhtLZC3Nt4fPcPvTebyA74RduJsAkozdLA0QwdFuvKxsq42nouc3r7goWaJqoliuJcb3AjaopGNx9uWZK4LfaffhG5ODn1cRFIZQZ/+pXYO0PFP3Uh2gCcDYYmN+xBBz7ca6qk6RsT9B3kXo0Cf9ALxhZCq13wdDBMYM7gfbNSO390Hl7onYQcBGuATDIdy/6+67oLeDYP5C15WW0Wyxc4n/17lsApAGTAFsBSiKgwm4AGVd8ufz5ed669ybpi48TMbiAVufVeGlr6DD2Ze6IQxiHH2Kix7MMm/k1gzRAlwgUOdgZzpttV5nvZoM+big9HWYq+ryCj+w4jjfrcewahFwCZb9uxlpnosU8AWY0k5Z5GM2Nj1Hp/NH3GLDptEFp6oV/+QE2+r5JncGaIR7PIZOnM8n3S6ioE8eQwoFTleto+6UOlZ0dTd5yEjpmI7644JLzAzHnvkZwEaiY40drkxxXpfeD8S2QFZtFUVgHy9w/5Neax4kVElBnPQahc5G4DCEqJ4pRZ0ZR4WNCaQWjbWuhOY35fnCiDnItZkyeLny3+juxse4jQSpnkONlEgbkctZsw4mk05glvQiSf8B4P1SNIeVkJW4yFZuefBMpcrbohwKgC3yJ708O4EnzYazMcjas+R6dvJ6BlqXEAVjMLAmtpnTJLqK3zSVBacZn9jnQpYNtMBLvf0Zbu30nr+m7+U46gHDTdQA+T/NnxdGJZPeRUOwhvl2CAFZ2YcgKviCpMBtTUCdrPnmP1qA+sFonqn00pVEWUIFRpmVw+ctEXk7B+VkFRP0GEgVmi5wScy851vsx5+q4IJUTH/WgIf4L6Cxz5FzqOVRaG/bV38fauwUohRjRjunzu4u4qi1lSKsv2eeHkx7exbTa0+L6IvodpHpr7k/K57OyavRXYunwq0PAwt+bU5WxgVFPXuVqeSzrf3dDG1XKJJEXwcej+/PY4WI8UoKxPPY0LrZTcXA4CN2lrFLvQnllKS1qKUWxAsV6UFtH4uHVBzoLGNl2n8e6/Hm7S0XIwWnUhdrAhyPgRCLINZQGaLjx/B6W3FzK7jcnMPN9e/zDJHBqIPTfwgdaJ64UerBu4m8YFWZGOr5BqssByHhFlPHUBtBjqSfhip7k9ucZNFUANj58hpoOGemvvUx3ylWWj93PAv3HLMaMt9V1ZPkxtLiUcM39bQ5ZbjK/aDcfOH2IjaSTCdkDcdjvw7ClrVzY40Ca0M4li4Vk06MavwsMvaQBY08UUJloR0d4AV/7afmoywFB24NNaRZ+985DmhsL/ReS/1Y+j3so8dqxlgVHMqmQGHDw/y8ovP+LwmQyUVZWhr+/P1Kp9L8/4f/l+E+x7z/xPxaS0m3//Md9DF6Nu4i/G09QcTBt9+9xXp/EqDENj5wj03WjaO1Ac/ws0RPSwDgIWm4B4ClpZao1GPSpCOoQxjqlcr0ykEl/iMiJNwc9ySDXm7xT5cJyeTWpLn74lCzlkNV63OPHcFHaxGNh36JYVsS3CltqJGqxIKcVtaitWvbQ9+x9tHIHeu3cUSgNUPAdEuWPTDg6Aa2tCSbLqHY9wgIr8DXWQPt9Qu1WsOi5J9icdZRv34xnzMi9DJtyDpqug40/HZ5/cN0dBhZdZOYZN4KfSSC+/SdmucE+Sy5za6HSCDfj7/KFrJ3DVl+ATY9YGFN78fP+YdilR9K1YR0bXc3Q/gbcekKkiEe/BV1daBrq6O5SkGMy0yRLxX+MqCsvGHqYFpmDJSKHRJkri0MKuWGuBpOOAHkPzRaQdJiYfmA6tUNv4lL+NLj9CJ5jKUvKRZ98E+nhFQRkxvD1CHh/XCzUXwDXoYz2Twf/dBYWvEKi79F/HmLqeS7cKSG15AfkQgOvzFAS49clooJuPg6D99OlM3F68yy06q3EdVygj9GB25M/g1vd4DEGQeXO3vRAlKcGEx+VhYeNFqlFy9/Fvv6uIxmek0evrJqfVi1CaiUHn+kYWtvxbr2GVXEuUvtevKq9qOppE1Gs1r7YmWo57gUv1nRyPL6ETlMzI4xy3JN/BbsodPuvcKvvObrtvfl1XCwB9llgQvSmc0jgcJc3p3prkBtMqAzN3LJ8T78F/+yaXu3UUX5iDuedWvFOzKVZlYxX1zmQyEETzlLlXF4Lr8SzwJEp1hCv6oDQZyFoBQAbY25yc1YtHfjiU+HDywl5cG0eG21h5bARtDjEku6SRYf8LoOlDzawmW9h3RBAz/ZpaEZqUbbmIrVzhvYcUXbBJpBVpUdpKerE3FNNW/d9zPI+0Pg1mLSY7ftwfcAKsmKysOmyp6/9A6nA2hNgF4lBPoW3DvSj424EA4oCuaWGaf3zIf9LUHlA3IdYhbvgtQAymhaztXcMyYrRKFtuQ+kOHEwt2Fb64NgsJ9j/HGaThOqQw/hcnc2c4SP4tPcHgj2UOFVl8dul9Xz6nZio7K6+zpCI5zmms+DQYWa01hujcy0ytwRouY2i8i2kTZ70dqvJ9RqMVuvGIuk/CcC9PQ5sLfYnou0sA4/XMdb5OPyVBv4LQe3FASdnyj9ei/5GPE9+9yTC8g44FAAhT0HkKzSrR3DttB6TtJ2cxBwq1A084fIsSQ8SdXK9lFHHJtASoKWzXkVH+4N7WwSu7ByOl4+JxrHi3CQgiGwuoEkYhK1VMQqhjQlX36GnV4LnsyHgIiaP3LSvU3k1AuMIPVde/RK1IRK8PhVBCb31PJXri/nkLCoGtnKuaSSKkMeY1jVVvLdTMgl510m81gejzMBxl3aejvlG9KYLfwGUrrS5aFn31jpGWr0OvTBA9eB5t2cj2FuxtOgmXtVlRAwYw23DOSwWb3Q+01kQ8AY/DL6G1vIYfTf0Jbk+mc9f/pzJNbBnyLPUltcQlu1FV2cPfap8GeKtgwfG2sGqcL7zMnKoG3507aIwpJ1wabCI6GvLwkd/iz+Denm2yojL/oEERRchbcsQJRKB/sGXuPu7N/1aa9ECr8+6xMap58R2qzxA5U6a45vUxBtx836al9wQC30AShcEQcAoN6Jpk9D/8PfkjbyAccRQZI0XibPW8Ys1jE+T8f6GNYyYdJVBk69AyVYUOEGTA2qthhs+rXwotNFH5iUm5+9vQImO5Rq41uUOKzypsxU3vdxeDW7DSevsR+QRAW1iM0PnHSFP8oDZUXMSlE7UhfXw7VPfstyvndGeei5ZdHDkQRIx8QsklihW/biSTgctH+leJ3ZQEP+I/kKouoyxz11m/4kQ3tifxDPzfFko3ctA01b0Sj16DBQG5SNzbeK96nVQLXobXDF+zOPy/bi6ObFxy/uU62UwpQxaM8EhnsBsZ5YcfJeDMy+wdu1ahlk/IcrynRmKIfgJZlRqqDdeJOF2EhacYYyAr102OpMaiY0PB6w6sPfqZOfMPcgsIkfX41/qHu3FrrgeW0ZCu5z4+BVY974K/COLZpKaRVZW5hBCCiUkaUOIdz9NrgnUte4I7iauSiysFB4sch8wQNMC6mg3w1tnfYi6PJ/7U26jtlViq2hGo2xj1u75BJT60Dv8Fv5SIzJzFwsn7MRi2YHl+GHW5O2iyJDC+T+S8BocTWSfEtY7wzUtmOSQE52DYJGxoc1CP6n3Q4ktAItMzekeqM0IQ3lUQ/TSy/w7SuuO4J52ikaThF96EpHFVtE3sBZ/apkzBD50lbCoZzJ+ZX6UBWpEuDWQKRvOqu8d6HKGW3Fr8VK00dU5Cxr3grGTNP0xzoxMx2CVwzPfPENGUi58uOvhfb/QthFoJ6VX00a0TEqZ4CVuUKUqGH+Pvg7lVPRYUVG/hF8zfJn49HxcO36F82Ng3F32dimp07ZyInUd89ygS59CyowjD+XP34nPQO92Cz6dzyvpg/AcfZ7OzpHEKypJ/OFJOgJk/DZnPy86aumVe2DVXQ6ZZ/Hyr6Pt9Z/YVfnkg5YKIqsp/QWIeIkgrRHH7i62R7VzZtCfrLTrB4Icst9DLVWSNF9FaYvIkPKSgVVPAdwT/ewsYZPQjr1I1e1wwn9K4FLIMgL6/QxHI0VW1ZD9nO+04m63iVWnFxDj04rKPgZmtYlNyf+GqZO3sv/uc5RpUmh1mcfEmcEgSLBtmUPyzUhavFu4uT8Sj4F1/A0sFiQSrg28Sna/q3RfnsrCv3xxf2IQk4PCoe4M/m1/8IET3OjS4Vt9gzjpETC9BwpHkGswNDzJK5HZbM4eh3taPyT2/UXPsrrToPZiorGcOSenkttuha9TL9qFJ1HZaMVEiMtABMGDvrfEOUbj08CBvo96cP0tMSPTCSi7W8BgCxlv4S2Xsu3eRyQ0vkxVyAaWDThPrDofHL+E6Y04n7jHYUzkGGeyQb0V6d/XynoHJHJyJGNo8LqFrCuVYcYjlEgHQG+j6DPr2Ach7lNkwfX8pLuJmVZkTdHMW/0sBw/CmpVQOX8NwVd60cu7SO+TjuS5Xjg/WlS3mFpBr2k82/dXEXYHVOq7BDd+C/zzfmXd60/hb31omvsHWyadhmrAfST2zu34jW9DFhNI/KWtmALLsDjkIUwuAokMv0130bZdxbvCniV3VqBPcYPeJpFRaxuCff03GJvANvwwwweew6PLCexE9pK8togzP42iS68mb/07SBSA27cw+jqovSnJv83p0E2Em83YyyzkmdT4A9SdgTsvwoBt3Lg7h6ZvdhLGMTw9zmO5f19MWan9sJJ5YlfigFVbOe+9Kcr2fa2Zj6z+NBeODmDk5UEYX9Xyc9PnOAeGM/Wp8SLQw3MGNbpFyIVOcnqtWFQHw4OjIf8ruL+eGNl87q28RJ7BgV/l9QgdUh6mVNtzyJCu5Pc+vxCdFU3e+SHcDXdgkkMiADMj1zO2bigZ87Yz9cBUYrfH0r7fAkk/inJVgoCQWUhgcSDtmk5K4qcRppQ/MgZ/6B3MzdadOEphixv0WH6AnRsJA/ZdfwOzbxHEi+umRkkALuZSaLzKGGUXgwIMfNfQTDMTiXITgRSBDvcY4vcHX97py/DdsdRNTsO+vJMhahMj/vaNj3yVnxXrCfUtJ/buFdzVJSiWRolM385CbNQmPjwZi3udO1Oup9AZVcF/DcHekXpPR/xuJGPV0xflgqhHju/tOUR2vQVX8xW6bbrE90PlwQsF/dDuCsEhxoK9IoQFNb2UrTsCaY9D+R/cM+fi3qrAt+MterIb0PrMA5tOiHoDkyqSsyPGc2nIRWa2nAbbMMZyAvK+gLwvyLJ/idXlUgrNu6j/dC5lYbX8ntxCp96JMyXLGOF1jxe13sywdPG5YwUSSV/mmw0iw9R/AQzYjpUAwo0E3to1ib3zT/6X+QIOTzmMwuBKXPUvuEtsWTnoHC7N7+OWH0zQLRW5sWXkeb/OBy0DedvxKjSnoTdtJi10E8WGBKb1voTGVEqgjdfDvXqry5c8seUJcmNbCHrOyKyYg1Aqhag3QGFHYE8YumwFVd5VlPuVY5QZWRG1HPL34aGoxdxRxMXNw+lu7UXqmsKf2cEi6O3OC+CQQJCsgwFnR5Le5zZFIUWPyOsikSNIvyBvIJTHPMZRvZ7ZZi8wdBKv1pPTJ53YsGgaPUayJuWBmozSGRouMMX+AoXuv1OpH8yk/dNJGHYHJEqYXIqgsGOVzTjqzo2jMLiQgyZbfPuqeLH/Igb77eZi5WQ+/3EC2qt9CPHS4labQtm0s+L1/WYjNAnorHQYpXpCCn0Y6NYEtadh+ClQeaLO/5XlS06xOt8LyS/T+LRoNC8udcDxQbcUNgo6fSLxvFZE3FUzsz79RswPlO/i9dnLeDJZSXHjJIbfiafaNv+R57z0UBJhOf4UJVaSG3qbMarnmRGxgaXxr7H24kk+/nMlvqW9lE54CYlgg6MNrL1wjP7eB6l36s/t0bPo5SZW+ccpl3rS1yRjRcK3ZDUM45Y5jD2lZzDKjPR1MrBR8z3M/Z6azmBWH8tBgglbCQSWBjJtzyxOzbz16Bh8kMQPLw3g65ZAZMOyIex55OWFSEwSMFko1p6j1raEip7+ED0OlI4gm0Fu/A40ShUvHp6Em1EBC1QcKVhNrNt5ynXjyRugpNrlCkFj36ayNwgEgaUaWKSBWdecSDjaB9WyHWwaVsgx4Q7fnfsBa3kbhS1J2FndY/zeF8l2VfK46xA2TQzEy+0C50sXorTSIBEElm5ZSo1nDRvHnKLA9PIjxb7t2CKPK6Ym8QZ/DClFbjaCMo535UV8vfc5eke3cGLqVQ6F/kta96APsrCRbAk6wzSbcDZGxyEIcq5bbvD9X3No7vFiWe92VNXdlLncIs1i4Sr/a6Hv70gGjgADLWAdDs3aZj74+APiMuOwCqrBxr6LtRdOsGbql5C2AQmtSLUhuDoJmC1msvVwp8eXxIEvQVcJjpVKXtj1At3TzLT7+NDV1YV92bPI6n9grCKE0jQVdRmpCPeqUHpUUKTXgGwhs2o/wbYtkr3XFtLhmc/Ln79M7lP7iB2YBb5zmF4upfVcMT7aXo5O+ZHqGhdenFPFFys2gHMyhog1rHgmmISkBKxtNjI5aT+LYl6m13MpZyuewKCwwWH5VKLnuTDw4GisK8DP5tHxBRYsCoFeVS+uJgPKql/ALRZpey6/DCviDVstznWTGVU+lM/m3QCnJBqqi3mx0ZZXe+3B3vLwd/3bD7DF4ELnlNOUakOwarUQawWy3loQDV6QWWzYM+ckAVaJBMjAVWoRJdvnW9B3gs54nKQTo/CtDeW3+SKYQiqRPmi3mXf9m9jp2MT3VWf4KbYRTUc9lIvrIun4PjxWG0BnejTZQV2kWdQciVhD+KQp5DXl0XZmDFKzhND7MrxK5sGoOlxuxUD829gY++GZc5yGG4no1XaglMK5YQ/7VzgOFtcB557DtqmUqu4VoDoDZ4YiGXCLCfdkuDTCoj2LaOyTydnxr5Ga0hcEgYCyN9nqBj8XqLm5dTKyUbcYom+DM8NEoIZBgmu9K9lROdyNzyDJ+XGoPwcVu6GzEJJPsasT8uvsGXR8LKPmnxSVsSQiuEBi6aV/7Cny2z3om96Xy/Y3RYnyzHe5e0CHVWcjvXoNFhqRSXR0yAZiZ8qgy2JNzBN7+bO7H5Y9g3hfn8i2N/aAchSk3ODkm8tQfeSPanAnhf3moXazBYVWzD+ErGLqxeXU1WipVG6mU93K6pYUjnIdmq5BwTd8eiEaj3MDsX7+a4b4NyMz1IHHEFFRQe3FkdoY+n86nBC9GZOPjhLjXRL+9a5mSdKp979Ij309cFocY3INL5dcweWvA9i0XkadKGWUxY4JHsf4PesDouPUGLo+IbAwhJbGFs4Ip7hq+d/PByfMZgamtyJYPKjaP4rhBb60R1yiwz8Op3vnAei16eJx/Q2aDX5su7wPnTaIDl81Nr7/Dxf+vyQ6Ozv/+z/6PxT/Kfb9J/7HwqJwRADMQSuRFP+ImzqKdx+r5fe6G5zOD8dobsbVMoUtU7xxVlez7tZnmL9uR+dhoLFvOWfsTYyVKCHsOXBKJvngj/Qti2arfzkbFn7OXeOb3Kj68OH9zhbWYfxlMk6hBWSFyBgYZ89PS+dRph1J8KRxXDixmyYT4NzENJqIbpvLU8U/ix4aYc/iIX8PvW8S+90MCJoP2SOdwVPRR5hz50XuLTPzWpM/A9ta8DR68oWlhqcdlfg4xFPatIiv90/F1r0eG/119lVHMiwiEXI/g4KvEARxEaCR1WFLI47c5oA0glfJoaunnmv3fXBscUSILAMe4FPqzojsPMdEjkeUYeNeQj+piR6zBHrrRfRw0w2Ifgt19R5Cb1bQiSeLdJ48PSYe9jpBwGLUfTYyKbcJnQ56nW4yoLCFD1bHg7aa3db3+PrwNIrDVPwx9w8k9haaAr7FuXwXqL15rd6RLlknMf2q2DZ3C5cMkVBdDlnviqwAiwCChcnBvzIr7Afq+ubjHugDMhW90haWvfcmlb5tjHzuAC62tWC7D4KfAGt/rujdqEprRRPQSoBPALIgCdhFwsiLAEhM8EFMGq5upSw7MJVxw++hGNMKiP6OMlMTSY37aNF7E2X7GW/0mQV/WqO2mPExjUVaEk/L4hg+e309w9XRcG40tGZgsF7LK19NpgU1hanf0aUopqZ3AkSL6MlY5WaWrjrK20XB/HXJjelRXnjNe0dEagky8vICGfbLWGxkZrYaFjJhw2ZIS4e+X4Mg41CNNw53w9DO3s+7k7Lh72TDoD2gdOSiLpKGz4fzZKMDfYdmMHZaGvT98uEYdnZQ4pXci88+f8Zcjybl028eHhsw6xybm4Pp6uilRd2Cm75ALAjbBvNy7nVy5u3En1GknGknyvMquO+GiFfA1EtK/Sdk3Qgjd+zzdPvOJcEF6BMC8R8j6e0kb3QpWzpgbYMV3yh76RfwBZHGa1BzHLnbWIqzgggrCmD/wk6eCtOJDQpaAXFiEmpL1X6q2wG5qBWu1D+QONK3cMZuJp0baol2M5BX2cXbMct48r1PAYGLEilnkxYy0fkv0setwd3zH0SMxTacsUE30QbeZOmpx6m5+hgdX23CccSbUL6Llo46+hV206Y08tlQ0adPry2CnaIEYmvzK8TdaMYiz8A3qRdnr0bxwmW/g0M8tea+XNdBolmC1CTFTuj4x+vKbzb7m7L59OlP8DSmUmGdxgSHemLLXgEfd3BP5W2PNHQv32JTx0LOe34IFgHL5GIK8gXana/Q5e6LrP4ek8uiSYyvhT6/Q/8t/PLDJrav+J4E5TymnGnAWtWOzPAilF+CljscU/Wif2oLNxS+nPeG/N4K8BovJrIvTmFUgplydRdNOe44F1xE25sg+vEpHcF9BOvGPk5NfCMbSvvwfdSpBwNrAMS+B8Bom2baZECOhal3ptI49xrucV7Q7wckTRBcFMiQywPJedVEpjqTOMEXiSDHyquBggmnsL/iTrNDCy0OzSyyhYEqMES+w4mJ3hwbe5yVpd8z6eflTFxxWPSMBPRWwcy43kBXtxX6gQUsmHYLucwLahRwbR52NvNZlOFCkdaNKcPdRalLi1ksUoaswqTsQ32gM/W915G57mBcSAWMuSV6NkgVtPW28bO3A3JXOR7F7oQ0d3I/NRl5wzlQeZKsu8xWnQebOyUUBxUzzKcVWWMOzO5i7yZ/tpYpMZsErHxqsbF/sGC7sYxIh2ksOjADZZsDe5bd4l5rB9GxnhC/AOLXY793Ov02rULmpuea2/M4OcEqEAvGtsGYNPV8+9S3aFVa9ubEM9ZHwY7CzXBrFSidiXNMpI9bI9/Xq/i5OpKfgiPFObJoMyhdcDSWM27cTS7sG07P1jKaB/0j9wqwu3c8c6svM+Z2AhG5EVybp2Fhv1SyMzJ5o3cv17VWZCVXUWd19p/FoMyGENVYYksccXTVs7T+GYxGNees/UDfBm2ZRHjU0Rmno8v2ARVOQPSx6qlEWrYT7V99GXI3HhuZBRulHqWpg86Ew5y4amb5U/B0zQFM0m58ZPBfAIQASA0SnJqdoFtJQUEMobJH5S/2P3aG671ncH3iPdpdQ9n0xSAABt4LJW3zKurG9XDM3ZdOSQcONx8Xk59mPXuODKI3PYQ4n0CU152oHBrLd9fWYDaDUtrNqtCp5Bh8UI67wSrHHuqr3wWfDxBcBmIQnMi7GY4+oIdzY07R5mrmYPgnfJ+/k9m2YCdLZ48zDLljh6XeGXmgRByjF8aD6xAsPkt470gfRp8aTbVtJ3PrlBT9q08tUjcy8r0QtCq8b0ewpzOevjOKIeMllpWGYqsLx6fUjWV/LkP5aipEHYecj3CiDmvbHnpVEsaeS8bNJCVi1MfiRQMW053bgefYv0ivC+dSezOtXkaR1X1rFXiMpdavmfOPHXrYjoWK9eDpDLXHoCWDWZ6PUWoM5simsdy3dmba8jyxiGMTBGpvzhricVrvy2i/cpaMOcVwdR+euv0MOPaBxM9xU8RjcWvim6nFvBFxgXZBlMdbYJPGVatxZAkWum27iXLtxsrQKTIVLEaebRjM+d7LJBjMLG8IwmIvBWOPuB5rvIxX1Sxaz0eRFF3LPl+4TA6op4sseqcklsWN5GzJEiw/LyHcXktQwCjwngqe4zApRrM8bxRypQbTuGcZN+Ql6HmQwLeLAeC7U9MYfCgUd8AxUYutrhq6nUHpyk0dHI+5y2G3z/C7l4lFjphoyP6A4X0zWe91jL52s2ixpKO1cWTxg99WkEhY0yQBwcyMGjtCsgJoMSrFwqijmOxY0uyNrM6WjD22XLNZxOurpag8RoPfHOq/XUPmJ4vwcO3BpTeXWOlF8P5FnAd7qnE06Ki6Ho8zcA+QTKti8qjnRba7JgJVexlDMxPo0nQTvtCBBvN8nP41BoeaT5B8dio3MoNQtX9NRN8jIE/HCgkZ2x/D07CVoJC3WKh5G6iGGyug4QKSrg+4vD4STXIe6xxG4mStE5NkMlsA0nSBnOuG4aUubP56FaqZo0iZcRbasyFmLTPcFuMu6+F4NSQoQW1pJ9CrhRcX5YJtCE8F3SCrLIiWYg1TO10fSgzSIzKlu4RPKN23gbzoBvqv+o6bxnlM+le/vteUYhlVTofHbdY0wbLgmYS3XsDarpulCzbxp/wzYt7ZyA+ZL7OpaCLynDeg5TZuwzZxz340Xk3tKDOq6JvUDtWHRSWGpB+xbR+OvKaR4j5ZEFNDldlMUEs65H6OTPMEd90WYpL0cKzuEMcFeEPpCFaijNbwAGd+sh/A5Qk7uDL0JXJ1TjD3QaHFIPKncvXnUFuDNPoOY/rnYvCciFIQQO2FQtJN19C+yLyHoBbeRCMBWdMNJHlfoA0O4qJgYmDnZG6dCmbGrNOwcwIAJp+5lMVNAcDrwlEadZ50vu0FDaKQoUFQ8XNyFY9XabnTrMIifZDsTP4ZtLUk3pxCwfUwqj128N7a9+ivXkhNbzKbb39Fm84Vwf0YAA0etRTrlSQIEWJV6G/5cnMvs0+NRW8WiH3xWYJNEuDcw2fVoLOjO9+fqcensbZZw+w/Mok0HKAFNzYvzSDcTk3ypTgkHcHYvPqgWK1wIE8bSFJVE7YdMoyWaq6WTGfUKAuqiq95ccASdt9K5eCUgzi1BbHq+1VUzpJBwnqI+wDkdtzqN5NbSbdw3v4unQ3++JvVBE4S16kWnYWw/B1E3Y9iwrPHsaj/AY39HZd0j9GYshWd6TXc6+cTYvdosU+wmBHMZhb9OQ6ZVy2qpxtB3o8z7Z5MKQ6gQwMLbi2gNPkOdBWD3BYG7mLwj5vxvNSXVkUlqXVbEYrvgVc0hDwBOnjHu4dCo5m6OmfqyuawYtJ5NI0ikNYsSIhc9wJhUjM2Xda0Oemw6r7N6KBf+PrmjzRZG7izdgJu/a/zzajTpDp4iSBUtTd4TgRBwtjzIQy/GocmpJgUt5ZH+iS1GFmugbJOA3fKUom0V0DUAC7dP4l/vxwybIJxLZCyrWA6k+ecg+Rf4d7r+JlvszNtJLf11gyOL2GI1y+0S558eN1opxKcpt7BZOXOwNDzKFV6cE4V1z1yB84UnMb02zZk444hSbnDD3bQ3eUEzikUGtfzeu3TxEu76PtDFMUVC3C3WvxPox3iWeh3lxEfnub+viR+OzuZoAX7gM/E4xYzNvI2OqVKbBqluMpckThLoCWdM341rF5Yg8X+NYzn7DhS9wqLPd5GSPgcao+z+04Gx7o3YmfVSfeyP3B1F9mRnB8DMhts7LuY8N4PbNy7Au2OELpj5lLR/icAGXUjOaa+hUdEI3eS2miwP0pnJ6wKHU5K822GdBspUznzmF5G2ob32esB9M4W/eCBwD4f8cqut+lqMLJv+gja7Nt4XvKPVxUHfXh3YCAflG0hKfosXRYXCHocnAeyI28qWapSej16+OiNdThb4h95zpp2DT6VPjhMqOHtxFJOK1sZ4CcChqJcLvLduJ+w7rbmLXc9U4O3YtJdJb12HOm147Ad+jN/9X0c70pvHvvrMW7bXMbBaMXQ8I1MDd/IY1Wv8NqG17iTcIe2gekYHeqRKYx42hYxPeIT7Dtt2PT7cioLfGkfep1hwdWPtC1Wn81HXz+HvtmBq4AmJp1hLbcZ3XOVdz54l/JgA3r7w0y67UD3gB8h6CsIWo5zTg8xbqOQ2XpSXZxGh0FOrCDjx3RxTx0WJhLSuu1u8WQD+Jg9edU7m4lWsLwe2iPa2LPiV1y9Gsk3gF6hoqYz9GG7lG5t5A28TvjVAfRo74FkAAckjeSblbySDDfz8+nstkfb2ybOD//FA/SMfRNF004RKoefesDb+Ssm9x3PH/dGcmfBnwT4v0mJpIoXNJXs+9d5A9TNREsgO+cqW3+fi3LSbbwmOlLTKZZ9R1kuk50dRr1XNYMQSMbC/y6SgUECjGiE+PYrzHzsNJ/VXiLLrgWzlROlxXE02j+JU/Q4rDOseTL7W0YMVLAz+wh3yhP5fmAcP12dA3EfEaZqRONgh1Gh5KbtaorHboMH4gLbcsagbiumsSGWPa9fpch0geck46ApDSx6ym774Jd9HOHJENxHpOHs0QwBi2HANqQVi0i5loLRSo8lWYZGXsiU6HlQlAauQ5GU7eCHxDpWeB0n6e4OXNVlAFjVbGWCbCvXvM4giU/Fzqcb4eQJ7AhCIhFBlRJBgm2HLb4V3kgNPbh3upNu5UD76DbsHKwQqg7S5mCD3mcoqhwpfc8N560kCZ9N+4TXjl7B5Y8SopVyLLPaScpPYkZCGZI/rWByMQXdUXxSC0HW1kw/OoZVwc3YDN8B7uIaYYosGgeep8FuCPLvHBmUkiOqSPW2IeiscFS04dVti32zNTMqPVgeKMGm9Sa4RiNIpCQXuVMnraNi8BWGS2G7/3t4Jn0OVq5Ia06yotuNk7f7EHEbSoIH4DNWlMJU9NaiD6xg8+ptTDv4KqHZIVQNbUYw67FkvMLX02p5XL+DIYtT2G9wwtpKD/HrwdgNDgnsOzqHCqMB7+ERZAlTGe+DeAyQd6QTdy+OQVcHwcgrPJZUTEB5EpQDCkf6E07u5vlE2zbz28LfMFm78iZA2z1ou8fwrrGM+P4pLg69gMfkC7hIqyFsk7j+9pqMVOXMR6diiL8bT9Wi3VwOqWOA8E/eqMQkMOrqYvTKMp5evwV/sy0Y2rkiD+HXozlEZ37H7oO22F05zDeB7owM/AafietYv3UrPyheJt46hYR2Nfa9SsqU/fFvvgl3X6NVM4Lbwy7jFJDIPO8fmBq+EZr/hIn5ILdlpuc0XB1vM6NxHnuH/sxom5cg6Wco/B58Z7E/Yh4azxLiHbr5wkGKUPcc1BlFuwSbAJAKVPpUkheex+1+txnb/vYj4ACprZrK6HE4dx3i1VMTsFoqKiJcFNahXZjOhNCnGbM5kdCeJOZ98wUnPWX0tb7OxVgt298/TVLtTIbIINnwv50OSAZSpHDT5RgpMc6cdq/AXjqXXkcPCvvNx9vpNy71O8wuOzjVlkf23QQEqx6wUf/vL/yf+D8a/yn2/Sf+x8LSfxtVOh88hQ5I2Ul+pz3a5vG0CHpyVAra5fkIgoBGKcr6aTTpOA9v47xtOz/FXUNt9OIzqRL6bALgXvUFKr4dRvOQVo65rULp348J/uuYnvoWX1zfzkFbJ2oW/sbEv6YwoioE77f9sWrJJNw3FGSgpIdnfn6Mcu8qjow+i1QiE5NDPqLO/yFpDJWPbSGv7Dn8asupMynhzou4m3rRhUUy0HSS7jMvszJvJe+98x6fG4PY1H4fB2MaDmltmCLM7H7tQ6yM7vB3+if0acJuP4eyXEaa6zKM473x8A1gTFIqMQfyeGfIO/T98g/iMuPoX1aDo6IXxykWKP5JREWHP4eDw1KqbK5w7nYnf2mkVMzxB6laZOoAbsGfMuhxK7p3jiQkdy6uo6tFNLhZB0YtSfodtFTaILnQSJtzEC4vu4Oyi9vNT2O4qiTQJYyDifnECo4iSrH+HNSfw6esP65V/rQnSnFQmJmhzIbuB6mjpjRWFUWhL/Ljus6BIrtnmNZHwL3pGqg8kFt6qY7Ip8VB4JXr3+PjIeXbZXFQ8C2kP8dl3WB2vfY5STf7s2T7ErTPXxf95cwGkNkgCAJdNt3IjC1oa13YUmHL0zLnh2PLSncfTUkuEksjI8duRK1vAFMPeoUn+4ecID0im1maz6EDFtk2Ql02AP7yq5wu9kGpFHj+WiQRpmg0oyvg/hXI/QxfIREfGah7BVr+HEXe5Mt4jbggJpMFOXp1JPlh+fRaWTDKunnZthCK74rI+rAXqPFfyZlltZT6ZhNZDsdjp+LnHiB61wDnDAmEloOjzoqCYwM5NcDMb9XHwKwHn6lsKy4lKwssPl+h8PyDTLmaESMvs/VMEduyC4jyGkJVTx7WZmtCuy+A2QYG7qDgyBQKwgpQdw/AetlFdJ7d4ia0fBdYuWMOn4CbEk7anaJZnUtB91jG6jVwZSZW3ks43g3ZOnjD2cAzDnDTJgGkemi8jAQT+6fvR6F3YnRRMU8EPvioOySClUg/aZfdp1OpIzT9CpVVMngFGHIYvCchrf2W06NOo7DE0q1pp9c/C2/tTXGOkN2nU1nGpdbfONV/FfOtreD+e3B3DTbes2h9ADkvDevAziWLDHk0qbbBkPIbLec3suKlL/i42Z53Nr2Ah1MHyqn3H44Rf2MdLrf7kZ10h+XjD+NtHEZlnxcg/RlI3IT87u8ElfpTEljCnT53GK95GYJXifKkZTuQCB5023Sj0xmxCCa0ZrAy1MK5kTC9AbMM1GFldBY9kAFGAjaBVBjLuTrCDo2gZURuKPpDM3ks8kdRRstsQIIJkwBmiwX/GXoGeB1Cpp8Jt0St83zJCs66luPX7cmnxxLQugki5ynoMTDrWHTnO1oTLhMX0MNXDjVkye7DoN0P+31fNx+ttTWJbsPJV0vwU6lQOsSDrgUU9jxrW0VbYQi/35fgfS+el/uVcfKpP0AiRbAYmDP6Nnfj81A6r8G5LpVwuzAUzdd5pxmu9sJ4wUzp1FKuVF6m3UX0cDQUbqIlppeFdaDXypmx6Dg+IVUw/Di4DqGl0oDPyq9wq+4i/YPDLNLAWXM1ZOwEoFw1DI/vvFA6dXM17kkydTDfGZgtJkXtL3/Fby8+x6prG+kOPMdsd8Tf00lksUi7SjnSFsy1g0NRlvlwatQptK//glwhjlVvYzn3v1tAnLWZH1Z+gdEGVqk8kTTf5OPrcUzeP5iaWd70X/kr7s7dIpr6/Gi8u66zZIwnh8qTmb9zHpZuLcmPDX/4W+fJ+1HVW02P1gqHmhwUghPgLibzGq8SpIghJKyRGxfjiT/bl/Il5aAUN3pEvs646h8ZJ7dGteUVbNqsiVqdAKE/QJKou6+r2EPJiIuU95iob9KQ2drOEtY9vL/RoGDY/iloHdqQPreVSGEMhGym5+5buG2ZR7LUTN7i+7j/W1XCZwZhWVZM316LNsqdPYmBPOvXK8q4ISZtlyTa4tW/ldycIPbq/DlpUwSey0Fuh8F7Jt1fz6PZqRnXqQ0M87yPTr2Ufg5n6RfzEWb9FuZpjPQaZcw+NQZLwL98sh7EuMQ05g69TkoVVHSE4KudTsS/jh/plpBvMNMaG4Cjk4X6Lj/cbMpJ8a4jfXQOUsdRLN+ynG2jrvH80lNigUjhwNnmwUS32tI0KIjo4BYU4Q5MNm7CTlnLzqz30Ay0Qh3kzkv1tpzT9bCb81B5AJpv0STtz5EFMWiVZdz3vYa1yQtnh0h85TBcBZtLnfjpk1eYbpSi6LXC7qWrcHMV1J4EhQOCr4B3lTdKvZIvHv8Cs/xR7xmjTMPHz2/CsTWasdcPEBntDEFDaVROIuCTZuIay1HN8yMvahyDPcMhIhrcUnG5spPP3tvOMzUhuBcrmG79YLfmkAAxa4kveJr1HjC51Z0xg6OYG7IbTiWLBYWGi4yz7cc4KWxs48E8JUDEi5C/EUq38f35CQh7XTArO6jx1GGfIbLMUXuB0glBEFDqlKRcT8G205bWJQL0VD9kKu+u/J2cLDP3AxcwXFnJs8JLLAN2dA0kd/WP6IWXSKhTY90qozT8OQKq3gf7WLruaRh4ZyB21gp8Dy0iZPw1cB0MbiNA6cSfYVlIXLKQNzpQuC8VyXhEdlLRj+jH3ePlQ5vo7LAm1P40NvYPACgx74FDLD7tMOvAC1gEgRdeHEmo4x3xuEQOg/cAMMD+Y5q8M+hx96HMbQFz67+DS1MgcSPt1lGsbTVjYzDgbfcDMocuLHfOI9Qc400PWNsFepWJzLhMND3x/MsyBJ9Kb9zr3MgZnk/OxN+Yav+Z+C2OWUtNlYa926uwr3XnxNi/0Ct6eV6/Cs4mgN98rikSyB+3n3ZHBZ8nl1EvPEguDj0Ccg2ZuWWcXJ5Oi80l2mxuMEG1iskBCyBgAQAueYd566XdvP/HWG78Mpn2t2Y88l41C+6U3LZBagG7MZdROIsAmA7vVzDKVRglJizpd/hp5xPMeXonmkAZ6JroVA+hLsCdbg9nWi9WUaLpYoiAWCCymFDfusU7n79DSZiWopgovDxk4JgIfb4Ehzi+PjsAh3QPJo28ypv+nZyyVMKVWeJac8B2nGKMhPs00u7yBUlWIJH9BMNPQrco6Ca1kpPffzEZYS+xuwledxn9SLGvwaWOApfTjDkxhjNFjYxKHEa4IR10TVzogRvsRWZli0mhEuVHa45CWybjnSczZoieT+6kMfTFQ2i9BoLjA2Z14Q+MNh0j/U48F2KuMbSliPe93Rj6wHev1zeOSwPqkZmtmXDpT7C2RmIxQ0cB6Jqw7tHxdtKP/FgXwvw6iJT5kLLPRZSPfrB2HxD7Bk0bWni+CdbroT1xE0qlWGiys+zkx2kL2JC/gqeKo3k1qh5FzntgF83aeHt+s5pDrfYOPsVVODc98MKU2VKLho3TlXxT70nvN5Ox7hA9Bkn4HLRV5KV38k3JFmoaQ3jt87HIo+zgIx6qS0RcOcqcK1fJ9PGlZgq4ytpwdoa/bj4DwGjbrczPHE5tVB6fLUrjgBAINy9Aw0UYfgKluZXli0+xr0sgwKUQFx5NAtl2m1i6bSkAre4RNLquhaHwzm9fUeHxHLYWDVOLx0FDPSqLWChh2Aku7rtE+XtxODvqsG35mX7TL9MRdpwPtkzDzy6bMPef0XpnUn/ZBQezB0Zpp6gc8EAN5F0nC4UG+Hb+RoyyDkK1oxn4L568/4Kj1GtPsbv+J+wET/rxaKTWV6L99TmkI3OZP20s7d1/AIMeHpefX0rfW0HoJlxhhn8b6MV1Qq/GxLq31jFR9Rp2wUf4IKwJLh8WTwpawcpJVzCNuMH2a19SUeRHTXcy3k6RcOspiPyUlVIrzrdb8UPlk1gkBk7YvsfsUVsBqDj3B/cjf8ekcCYgcBZJLqMwqP1FudXGgXSEFlM85gSBJYG888E7ZL7QBlUHoadKnGsB3wpvfCt9GfrqWfpE/helHXSsuJ3CxWtR6Ms+QeMQD6fXMcR8C2Uf2GPTgU+OHaXXY9FOvIZ92nLQhFFp7ktNugmVFO5meZGpfZH54535YPhIvGwLONISxfKEK1zPG8S7+w/w7LCn6Rs7ELwng9IV4ZYbBckLyQ/byBg5PGYH31eOYujozfSegly/Nq5FHUNWtRlH+XIculYRon4Mzyk/giBwuWEcOfanCeiKpaLQiOPUf8mL6Vv5KM6fvSeXULrFDVu7+Th+nAY28/mg2Zm0nlYm2pkoln/FjeYC5P4nmR84AgIX88nuCUzamkRlShfH+8ZzGYEnZdaAAHI7NjQHkqX4meQYKwSrOUR5+bDtyjvsynkHACdb6AqCLs0C2hzaAHA21EPdLfrGfcc3K8JJ6nbho5dOYFekwLhw4cNmCzIlBoMMzB1kxWYBIP3Xh+d0tTPXXkvBFLqXX2yW45rvwJ6s9yDnQ3bXplLc2Y63I7RGGLje86jfpMfEq9yZcIy4im+obzUQHDqct/euwMO2iLruUKL7fkSDqZG2dGtyeuKIGGHgmaTHuN84CItVOk9Vh/KNxMjSt7YQoZbRLP+Yi2XzKG+PJswnE3OKlshib6y+SeT4+nPEjlxP/dWfGDY1jmpFO63e1VDgi93FASxf9tEjbXO2NGPSKLkWWsBffdPoa/04L8RPwXxuAtEDsqjVvkyL/88k2pZgY9UqMsrPDMVTGsumsX9yoOQDZs7ch1miZ5b0g4fXtbL5iqfHfMDPRfHE7vgUaawHjn676bGISg6/+l0nNqSemd+O4czNJLSbfLGWtzEjcgM5DUNIjU8jvn8Tc0OCCNIFYDBpmRb1PUg/g550gqUVTPxyO9N/Hcra71eT9MbZR/olCAJYoKvMk7etevksKAyOx+Gk0nA4pIynXBRQZ+HQvxT2GZ1G4JerCMwbQqVMRrFHNQVGDc/TQZLXBTp0zpTJx6BZpqPkaCaf/zeFvr/jcQt8dhkmLW8n26qLKqcuznRBercLazyOEVT3DIRsRyVrJ67te26+HodLlJJJOZOwHXETPMaA53jiQ/ew7KMMXmh/ilEfubPm2AI2vC1Kn5YM/ZU/knqYljgPU44Ji8SCnbQLTvXH4jqSm04r0PeXERiTj0ZyDk+NTgSUZX/IPO7y9uJ6NI0vc33INhZxiWh7IGU3dBYhu/UEU2zgVA+0l56mvM0AQf/0r8LnVTY1l+N15yektzKxCjVixh+QIwgCvhW+zNo7GrOqllXaVfz05C0EhR1IQeI3hy+vzIFO6Bz1OSUxnzLG2RVsAinoaOR66mZ87QOIb2ph/PHxVNocQRiTAjJrJBKB1DOp2Cld8Mny53hnNcEuwx569vV330V/+61sLD1Oh17K6S4ZQ8x6OOiD2iaC6X6JPD5qP4Nzoxj54SzqUu4RNsf09wCitsKdhb8t49KQS1wddBWjTANO4pfL4jaC9ZGryXgtk7G7d7Jo4E+MrPsIjkYT2J7NL24wtcpCt+8bZPgCtdDYH5x6tvH7qaUcGTSbyXaFpIXMR4YK3N8VFUHi1rFv227iiyupWHGOG6GrGWb2gY5cCHkKs0M/kge8R5F3A1dD8xjrryfg7wehb6FWvZrsvEL0vlqKU09ir1OL62OfGVC5n/7OpahmaNkrr2ev4ECG4h6o3GDwXrHbvRBUMZLAEg1poSsp6fZmzb9kE5u0dUzco6BX5c2rn2k50CHAxSkMas/miYhwulWDSakOoumv4QQuPYfaVAAlxwlSanBrmYK3Wx8008JZGbgGf10e6ICm6wxzqaXGZhG+Ht0MC3wATr06R9wHOfbjdvpIanfF4ze5g74BIJPWgV34QzB/q1MbjS5djP1uFa+b5Hzy5dfiNWTi+miRx23kL9/h94fK+4+CA3xkJ+nvpaXhtA22FQnEPJ4BZhPD1bVcjSpG7y4hP7qS+Roxn+6gqkPAxCZ1CUNdYM0v13j3vyn0/R1PGOGl1gpWO7dSF1PH4pIwMIFObY88U8BLGoet7DZ+go5ZN7ZTHuxEcI0FS3AQMP7/u5v8J/5fjf8U+/4T/2MhCAIebb8hqauGou/Q98tiRKkMK50tYbY6Em3B2ljP4gO1aJRNZCnTuDFoETKTiBZ+iIQq/gV0zVS62nJyzi5UvS40ngjHY0w1KUZxIzs5bBN7OuMpDi5GNaeehc4XUEvHw9jbotl5+W4CjJkoLAqcbvQn8UYyYRvi4M5CcB0KwY+Tb/TgVDeEt95jnm4ZwxX3IOY9ilr68MKGVPz9oSSyhULvNDyk8GboAKjeQ33PEZyrNJQ41XDaC5wkDaJfSR8RYWb5ZiStWWPJnvkqBzzuM6ntSf50X0PWk1noTXrCZzzHhZT7KA7MwU7WRIBSAX2+hrLfoC2L3yO2Ulptx5Uf5nGz/z3x4zKn++Hv/IfMg56Qc5gm6hkiKIkzPxD28BwPhnbCOv7i7r0E9NZ2OKurURrKQO5Pjv2H3B2lpd/Y+wzphAOeLTQLZghcCjbBzNlyDS4kcdzrPjOu+rA7sQElzWISxiaQifV+3E7rh06ho85Jj1vuXOgRk2qJ6vmsfm0vK/PscPsojIzkAuBPUYrQYiJWWkKgp47toQWM9OlG59MMV+dD1QFI+BTBewaHbII53O3J+pc/xSw187TU+mGfb3a1kr38BNW1MRg/HsyVvuN56cc6rHI/JU8G7fYdeGqDGX3pc1pHd0J/LRh7qG0J4NunX8TeEMFTOwdR2anGy9ILd9cAcEOfzPZNtrS7dbNt8TZK3BSkagaLyPnAZSwPu0JSfCFTy50YrGnmsLwf83ggO+LYh1jJPHrrYejZECRGHa6bfxCTqkWbYdQlBInA+tc/JqhiDr4Nc5kkC4WLIiKb+RYkDXVEXdxPZbKaT+LNJBvmMcJ1EC52g4jfuwvnSDkTChKwbe/HtXEuTFBkQOM1RqpriTFDlsnIK37ZOJjCWGA2QN9vQeHA+ouHaPBvpl/3WR7v0dLa0gS62dBVgkym5siGH3BqaKM0pIAjDg30f/1nGLBORIZm3+FCkJ7tTZ00Z3Si7VKJ0mHuo6CzGGyDiG54nPB9o9Ep9OiUDzTp27PBayKuhho8h93mbrUX+fGVtCvusNflB2b6m7n94mHe+Gs+Tc8YCNV7sTGyS/RLA+R1Jxl/PwZ1mQ+5Y3qwirjBeg1wLAas/Wlw+ZlJhnoUViqWhxchd+jBaBX40BDZb8tIxn+/jxHV/2jk45ggSv6pvVmqy6Fo21JujTjP0SEXxXc29n2wjwHnZOLzvyXDypENBTJU90bx5IQ0egN6sXLtB1IlH7QP4HpnBUm9Rl6ojWJxWANcXYCktpNxF31pN9pg+/SLFDULZLsk4lX4HeR/yWjTJLrTk9B6SnnCbQt6UwcXTUrc3UaAUzK6slYEs4B1hxXqXWPIGZIPBd+JY3DwPppMP9EmQLN9DS0etbQhgfZc0SMx+HF23f8AgwHGBf/ApLCvqXPfhnvuUsj9BKbXs6EomrBNqRgH97LujXWsdNCIzNchBxCIZGVgK787t1L6jQLnxhWkbEhBWexPSoMn1geHYXIqxtUhlCuVl/m0DT5wAnnuh+ypsqWhzgqHIAmvRafhLvHnD9chACikcrrdxtPg1Er3rVwm/uXCgJfGkRqZAsZu2ppDuDJoCzqllLyoEKQSASgQ0YLGbgp1Om42+ZOl2cu8v2sovXXQlgP2UQgWE8YSH5RlPuTGZmIMrBDZIQ+K0YdtF3No0HYU5nAAigwg0dbA2RHo3RI5NeoUCe4vMcRd9L7AbQTMaKKruor76YfpNE7Cx3IVf+UNlD1D4cxz0HAJa/UbfPLUt3jXpPDYj3vxcsgFdombF7OeMWpnknusedngj0LTjZsC8J35cIwefy+LnD/9kXkakEfYIJg6AduHw1WncOXZRnBLuUKOH5zTtT3yjfWRKlHcTaAqpoK3k05RJbsPhd9jYzuDssAsrF3ciLvRgap7AumvmejTewK01XhKzyIMyCJHrmHxb4uRDL8FETWABTzHcfjyBGoaa3DLq+LgKV9Mz+WAc5L4z6jj5KLrnOq5zk/HWrlUYs83c4Hu09CRh4CZJQXh1JS5c/98P7Tdin+59YjRZLLhlg56zdBtVYjBonvk+HvXQnBucGDJqPusSljPiyfTKGuLxdW6nA4bZwIiq2mbupKU8HoRdNCeA0OP0HjnT/6MrSO5J5IuoZjh8jEMf6CFcql8Hh9cOgLAkPrLaOmleJ0PQfXnofUObcp3SA95H8Es53UHcBXasZj1PNsIFgtEGMNxdWqk1bOW52JqKXW1Er/FrsOg/zYkbYUMWn6YCz1HiDObsRcela/7W9pHp9Lz5LQ3GOS7Bwo+QCZNxsNQhqS2FpnbeBoD/AhwPAj7ZoDFjKN8AnNroaTHBrVPGT22rbykisbOLhKqDlHUGsDrG55EHiyQNLkEJ0U9GBABJgFLGb31NaT3hlDQLGWURIl85j1QPC2yzW2D2en4JF5DrQjun8ULoc00uC7CzUYKfUUE/HzlJSQfXWPfJwuJr/fCRp0Nkwsf9kuoqabPsV/QD3KgeqTYTx8fH3KMvhzrhpHWZobumEGBWcFYn/fF67qPYsSplajOjCJ3YDv3px8m093CfIBUMSl28lQAHfZlTDg2g/s3Y2gY5gBqCTj2od2gpe12Ol7VXrz52j40UhN3kq8SKJFD000kOjt8O6/TrVdQ0FCDt1yN2rZHLD49iFGJP5Dl4Mvu6kAkghaTRC56TOuaEBQC446Nw7vKH6+aepLG3ECoOQHA57XhODQ2IUgszLaBDv5ZiwkCvFkVRt2pgVRHlPBj0A3OCfmQnQ5lOxAi7nE65Qo2Xc40Jd7DUWoRk37uo6G3HoXKyM3km0jMVkxs6SVACGKptlYsENkE0leTQEnDNIoUr2FRXWaqMgPKd4vMbWsf2q3DeNXcjldaBFCIuaUNcHvYviDfr9iVKlDs9im5PmcZqVjA6UnvUXHpdRa/s4ll+eOZnf4cFSUBXNSUMykwC0q20GP6kB2zbuOvSKQg/A8sgoWXha9FeXNBQordH6ji1NQGF1OX+CnVTALNYbi/ASwmalrs8SrzwbBzCt+2a3Dc7yMC0gAc+/JhQTxVDZ1ci/gNgDYE8Bj9sN0ObGTGpMvcLazHt9wXSeSjiZAFduU4K6DuZhKFIYX0KL0gZQdFW6ZxbuMM5KmtfLn6HeKsmykv/p0gQyc4JPLhhTrk1U60KlvZXvApg1wBByD6HbCLJDP6Nczu9wm7E86Ik0/ANxaxoNxTQaUsiRnhI7jZpSH0RykeMS1QOxouimVIachr6K634J9ppiQ0BWOSBNzaxAbbiJnIJGU3Lf+aJhSlWyHnfRj6FxUmBR+XOJKelUPMgZnsmnyWpxcZERQOJDlf5qrsMe76yXj6yy+wtXnguzepEENNBgU3Q6Beg3FZEc96XaTE7AQuosRtbFUP9XMbCLO1wzc5i+jYo8DzkLMe5LbY04W53YCjrZR32wLIcS4idfQxxik30hP6MVd36qk5PpQOTSu/BNWi0DhD2Teib3DxL5jls3jLrQjFlbE0/j6JS28U/L1CBuAN90y6ltzl/epXkVnNeOiV7i3UMqfHkUo9dE5bRGmJGRDX5Zh6GGKXyqn4DJzdVVh17uCuWzmDit7GVtGPvwqf4ZWItSSoYERCIR/NPs3Wjj7QUwMHvcBzIrNKw7iY7cvBhDsYZQ+S7TdXQc1xJEPP85x3D+UGWJs9FG8HV/5r5JpdsbdrxyIoadZ6IZM+CuYYEplLkdDDn/YLqdTa8aV3KmjruRZ0mh874KrBzJ2Eu/h4PTghfgNowljXY02vvJrG+HScEq7xZsQNyBa5PFK5G0/9NpzIG/0YrO7B2b6bETWroXoDeI0nWHeTvc8eY2VJFC84PIuTdT3tynKcVNWsSHiJX+xGkJnQgNSlhREOOhwcuyD0GxHkqhSBmxHDb3OnbyaNDd9zX2HHU//qk0Wq4vCpRFRNzgQPTadPxK/QUkmatC/jq2/jgpm64TLOJb3LO74PkHmCnEZFBG+seAZ7UySjbsSh65bTbjLQx12c26+XzmJV7wlcWvWsPPs7tYIEYn4Wi/v9vqdMMxvJjLMkHxiM169RuDzzLbOaJ4GuGSlWhCm1NMkhtDCYzx2kBCf+RIdp5AM5NxmtZR9T1/Uej38iZfPPUmpLhX9SkFIr/igKpPaIC50e9cSPuE2T4A/Wvthb6kn59gBuoxTEna7Co9uRUaMXQrM92MeiszVyfOxx/F2eILFkF0FBQE8pNKdBxCsItGGltaLH2sLU0T8TrnACXhXfbamWF5KWc692GPe7pNhJ7bB1bidUex+8p2D2W0SW6x1MNBBYHEiXRvvoAOxtwuX0XhwKGxg6Fp60A62hDkRxYL4V/LB3raXaLYerUWsI7FoAZaKyjCbDk2ePD+bSokJ29HpRr7Ih9V+X/sK1k1wDbPvJjt2l4xiwK5Z2nZp2nStymZFz3rCrE26d6uAP6yheecGK0UG/MDroF35tX8i5r+fgElbI5Wd2ka8NY6DEi2+vi0C+ZXGDmb76Csuu+pJcE0ypTQyT+gzAr48ogS5vb+FZD3ti/dLwrIIlNaH/4gHDRdUINi15EU1PPKHVW5gW3gtnh6L1eYWDXs/QixulsRv5yuoIA6SvMbxoM3TkondK4eO3XkfbK8XyuLjelph6H/p0fl++jBvPPE2IVzPK6m7cfSu5ZZnPgtLDOHUM46pdGze7Gml0aaQwpBAvuR9vDJ5GrNsFrOVtnO4O50xnFlOTvuNxu+8o6IjD3tAmyvpJrbivWM3aNHdctQqsJRL6KB4FvX1FMU1np1B8OYGC8DwiUw+DxUy5UUz+my1mHt80E8fAcnj1AZhTE8q9oplU7DUgn9/Ob4s34tY7lBDhB+YOEefKhNNbMAoGtO09PKqX8f8cbkBhK1ywGcobxZdwq3djZGEI1aG99Mg06JRhyOU2GLGhzieZfmu82SbcwjjoL9a6FkNXKDjEcaSrH5uai4h38kSqbMPLtpdSaSAB0SuQV4tFXF2PjsXvjCAr3pG1rzwuNqDxIluXhuFzfSEf+/7GD1tPMCV1OKQ/B8B04A9/yNWD0mymB2gzaLCvPy8yp4ALRa6Ydg8nJCeXKo0j12c5M0DTRKXyCc415PLE74upm13HT8/vocnqY/p2nmMYKahkKnp95Zwam8NMpxfJTu8kqWYF0o4MUHiDlQuakgz8si7TGG0mKaie1Y710NuAwSAhLyIPiYMVUVXObFm6hVbHVr4eIvrNBlpVYlcQSY9MTtaIdxEEAVd/xcPf/XTFc1z8JBBBks6Gp79BjpoP/lagMHWT3pxAWdMOmoDDk07S7FTDaZcNgLjWDO6ZToeLgVFnRuFuUmD1tww9ILEJ4JDeCFZGymIGIQl5wA9tF4HvB2psSLgdg6qrHq3G7eE1LX5zWdGwFCzQ2dRC+D0t1d7lIhvcZwYgwa7LGZv2Qmpre3AwVODi3SFK8Kf8gaXXng8TavkuoJbfG2FtMxz/+5tnE4jKdj5ZI6HX6zQbnDZgNDwAmPT5CuLW8eOWd8hO3E344Wf5epcjjm/4k6pvF9eLKg8Eo44nJnVzr18/5OUDkOn+hbQD5BYdvfE5tJkh4bob9ho10zRinxcMymOTpz/9+sfw/ZNfYGUewknjech4mcmql9FtT0RQW+Gxoo1w+zyaXNfgrGmHoh+IkX3Cx6m32d3wDt+sepb4gZlMXPGX+M3qLudXWxdsBzfiZOhhXtpASgeUi0o5hg5QeVEZAPc6A/ksuBSJ2UKm0wvEdm4FBOjIZ7BKiZNZQ7+M0QxVlGI1qOSRfvnrnqHxVl+yXJKxHhCBym4FWIx85XmZ3zvgssVMZp8SnrO9x6vNKwmTlGEjqeS7lkBuZQpo21r+f5oPhF4J03sdOWUxY7GAXX4arrdPABKS9cnc7nebGTaFDFt8lM+7E3BuSEQi/b+35CQIAj4+Pg+tEv7/Hf/3/vL/if/xkDScg94Hkgyhz6DSVzJj3wzC8sPQ9cmnj7QfDtGldBsc6DY4oJffBcC6W8XbF8cQ0adQXLynPQaA2fYl8iLySErTkHTDi8zEYq72fk5Tq4oWrSej499lmT2Um3pQuoJFkIOhE2qPg3N/WqQ+NKx5kbK/pjGuRUGoTQUkfCIuzgBZr8DQC0MJKomkusIVVf98uL+entAWIi7/iDzDisqFHZSqrnIuYiwuZb9A/CeYrt3njV+/Zn6FI7+eieWZsCaQrReRtDJrngsQqHz7fXyr44jKcUSvboFri6HsNyRDj9LqVUubQy0lnZPxUzdgsgoAw22oOgTBT3DjQCJlaY4UTf2LaP820LeLfmq2IeCYSLHBjmwTDErI4V0fuCxMFyXubIMxIOVov2qqQzqwtxrBjgEvUND8CvAJglSCUWkNrVoi177O9jE3WDxnJaTsBP95zB4bSW9KJVmXxhGzI5Lrrx9k2Fg/MQljNpLhv4LWWR786juLJLsGHHr04rNWedIgD+ajFsgxGzGHFCNzaYXmWyKDLX49kSdeIa7Dg53WPTzvdxt/yWhGuKSIxb6OPNC3YnPVG5+DQ3FcnUOTSxMSyT+TaAsyflblY+3WgmFQL02eWl6K/Yt0m2k03EvFrssa63ozKec6UYVNh3hRust0+juK/eGThna+nHeabkUZxxSnRQSZz0xar6mIzokmSyhg0JRS1jkDWWvFm3pPxTojHTVywmQKtrvDTz3Kh4l7gHGqJQzw6uTc+WEgk9DjuhhV21kw9YBMwwKrCyzwgBfrXbFmKkHK/eKJD1DVgbY7qJBpsdeNZFBeGlMeAI/ldGDTXoW5QUJheC0YGhlpGQEVf0LFn7zQ6svvn7xB58hmcvoBCFBzXGQqDNhOgWI3nc4VTLk9g4obMXgmVYoFSJMWi/8K5FzCLK9CVuJIvfNNXBp+hkM7IPQZbBRzULS445HRB/dz37PF8hxPPbYe/goF6wCYUsK0wDrOJmXxaWwJT8ZliI2+9wZ4TyW+5wpLPSGmTs8yp1ped4WTghHK/mBycAu745V4ksqMzY/z5qDbfHP4A1C60NtWxBPff0HGzUTyRqRTYoAeixVqG3foLsNNcw+9XobcIOGLsWISv0jt9/BZXDT35b3Ws3Q2u/Dq7SF0xHWCy0BRes/QRrm1Nc5zjtLRY6bP7T6sTj0Lh7+HKaWgdMIsyDnx8TLC28XiS0liAc6ekwhIeRcARY+MtWvX0uJpxK5mFu0zzoP/TkYCQnIqPzcFEe7hSrH9KJpsgXrRgNpW20P4kfHoh5Vy1KUdk6QLk9TmYZJ74Z4RjPzoXaoGdJIxOAn3kHGgOwFdJWDWMyjHB5/0gRwde5vR3a3M9uvDqtLt4jznOR6HwgN45d6nRiZj28klLHlryT+TslTFbZk1eZMP4eo5BIPCwCKHLuhphvwvEUK24pRngw4trwoCFokEreCFftgJ7O+tJ6g0iDLrLhZmLqLyTBkfjjzLOKcxDJBeZP+OUQzMjKVuo5ELWvCQaMRi3OmB2Pi+yc/PvsO9hrF8vc2D4OwIQAZhIlvA9czXHFxwndcbVExavxKnsDIRoXt6EHQWUG73Ift+SSA1O5rq4EBOTUhnNCsh/EVI/BzswpkbeQ1F8G1me3ST5mmmp+kaWIus7T7uKezw6MKhxZ0xJxopCc/HEtyBoG8ha9AdrobFcqImlHN7hhMQWUqARCqOAY0Tf+bEIZNB3IA7DPBOQyuYoFnczFkjwm7bNFUErNyDk1u7+DtbzODYh6bc5Xz2lBJlYisb1n/Jr53/+NIBVGucaUzMYWR/E88OXEeZYSiceVJkSYy8jABM3zcdo20XmctO0Wp+VF5soWMpQb98zTPH77LqyCu8MX873HqKWHlfNjjdIiEOvLcfx6neg0u28fSZLsqiFW+7QPGgcqp1Aczx+RqlhxHi1olM4Mg1WLb5ozybQXmoQPb4ozjZJYl9arqBILfjdA9k9IJc24HcJAE0UHNMlCt0SuLQpRgciwI4Oz0Bf+c5/Ne4VB/BrvwahqHkLZ8W7M01jxyPyo4kISOOn5LWk1ljS50RDGYrqjvFCdGsbCRqdBqj7IDUu6AQsbKfOf1FExo+1/9JpeIH3tbJkStFOONg/23cvrMOx2ZrgirdkZvBq3UvKJQw6gr6i5f5uNWPC50ynH4dRfjgTHpSxO/F6GrwdUzm6KSztKvrOKTOJNnwBMuj34ToN8VXy9jFy04WjLUu+JQE0j8p75E+OeoKyO0JYrvWTIe7iMwn823skfD0896sOl5OrDqXAo+9lOiqGCY1g8qTAlIoOdeAUuHNjehqLhpg2PAdjDD8BndewCC8QKdRj9ClYPenMzjdZxVv7O8DCieQKvitYBeh+yKIV+hp0yuIm3YFcj8V3y/fmTS7dVDoeYMIO3CRQoPLPNyUBXAsHoYfx1vSjLcVTF/0O+YQuKgNfaRfZvdn6I1TM+BKKEntjxG+VoeTkxMSiYS+t/ri4CylwXM0rpqbGBXuyBz7gsKexyLuUv12IUfax5LueAdHcxj0NkD9ebGA+wDsdS0lg+vJ55lj9z64DoKKXSiarnJrwHWyu62xYKHdDEaVDxwVTSMkfQ/x2lcfsP7IGtI/WEXu7HO8Mq8MTDqxOOY3mySHb7E1JLNe+ioOKgPN3tdw9Z4KtsG4ZP/AeJ091Z3WNPW/zV2vSsYNPgC2wZxYvY7VO8PJHd9I0o0X0KTkPCxYCAKMH32bE1GFGLt/ZssJX8KnjgTHDKg6jEafT09YMW2GZtY5WnjCHipMPaIMc8E3REh9edoOjtaosCoPQOqpguabcGkqDNiOxhhHsjwbr44G3vLwwkN6Ha5eEW8+6goq+1hOFEzgATcTmc1YIOPhs/p7I2p5IC1vQA5mA1aGOuzkOozyWjKHVpOXeAGZxMIk9VyRkXivmb5OmbRrrdHJxMK8RBDgZDKovPBxgdkvp3GyUkW2L2xte+AJVfY7uI8mO7mWMwP2EZ0VjXOTM2OEZ2F0GmABqZJb3dEE1LjRr7Mf1V7VSDDCkVDwmgSJn2PR/8wM/3xO/LgMv5bB9BtzHPjnuxZZYkXDwSfJnXWIl1IK6V+4C+pCsZcYkSJgsZixCAa+dTPjk/MyxLwDUiW5n//O8LODyZm5jVzPvTR1jmANPz2UvP5Y8jpdngXMzQzB06jE21IgngvIb/9F0h+zsNXKCJlST1TgTVCvAE0E2IZw32EsR65JcWyWM7p8NL6hUuAKeE8Ti7fAtOy+BJ53ZO2dPpRPOoFsyAnQNYO+jVYhkOxdY4gpCOPP2X/S6NLIExNyUZh6eG5sI7K8XKTLDbT6d6AVlHj3+RIujMfOfjB7jvfDpdGF0KDJaPUteNsmQPVRSFuBh933tLlHYdHYcT8ijmb1GJHFdu918X1TpnNw1iWk1TfZ/tESOlbkMnFKE/5WZ6Ctmied+1HxzDe4FQZRtXUcI9/xE/1ey3aAUz+EBlv2t9mRYtWD0q4dQfJoMk7qtJgW/2wczPb8OtKKlsZRcN2JxcJp+GYlru46LBMkWCQCneoh2PZcgrPDSVBspq6/E70uCbwvnc8mZ9CUbuD1wfDk0fvsvDGGk/s9UQ/PY45NGWpJEEvzvhBvqrBj3Y1Awq4n86J9O5Nd3ak1lEHlZnEst6UzOs8aehVMP/I1BREVwLZH2p3l5siFpV+x4PcFbEiLIHWbgeB/HX8zLI+O0Nu4Z3xIW3s4NtaApYWKbke6MjzQOKv5Uw3eRTZUPfGXmMi+v55LrR7ozO14KkopVt3gSq8ng8KGiN7oCmfu+ZfRJNfiNjiDaV5anHt7RYBg/CcgyDinhZZeK/afWYmLj5Hhw8DdpoQAh7u86jCcN/xkvGxKYnDq71zQDQaVF1xbAJ4TIOYdvgnqZPeNRC6fM1I0NOGRPoc6xbIxbi21Dgd4Y9RaLun9CI9cQ/L99Qy+Mgi/u+7UPivhvsXEyPolnH3yQ2i4jJDfg16px2gwcmDIO/TIaljUM4A/Tl1DJjFQKL/DkqOPUenXzv3U81gCm5jkPBB854BjH+bYrWOEtZbhVr102nbSbrEwwvsb2PcyUcrZ/GJTS2ZGf7IODWePTQ/PxG7BUXcS/vz/sPfX0VFd798//BqfyUwyE3d3IyFCCO7uTqEUCrRIKdS9UBdqtLSlpUahuBR3d0mAJBAh7u4yPs8fJ4Xy/T7Pc9///O7vWr/7c601a5WezJ6zz9lnn72v6y0KGH4ZqSWB6GOfUWLtCfIJPJZ/k6p5X2SPfN4mWh2NfONYwVzTDJ4BxGIRzpVZdOZYKQxvZpo2H1dLFbRWQWs2mVEwzRE6S5eTtP19Avo0guhFAUDoNpAU029MWfs6t5Me0H/ZXrqMvowKcWB58lJ+TvsG++L7iG/4MK68J/aNftR+3q3SYBYKe/V+CZituczbPI+61FtCsdpOADTX5v7M0F4H2UsS1rdeYnvvDCYPe8SatTha2DR/EyOPT+fNn5YgWnIXtDHQUcysmLsc1VciNnlS+dlibgx+nNm3LCMAY50MrUJGu6MvIqkYV7sSfBxyyW9K5vWvJqNKi+PklD3olXpWil/j75wXmBTxNXnGUI5M3E2bfRvfV4kIsPnx71VunmYOUReVKLum4+gQz+uuqyHtRdAEQfhzOKmccG8ZR5ezPebqi4Q/WP3Yuf1z6zxbI3jb7RgeMjfwHI3ZIZFmvQcKBZgQcaoTAp09of0uADfsF3Bduxalyp61+ZMIVtZhr3R82G6XWxhZ0Wk0usk5vHojOZ0TeCAKZasHzA49x7QDC4ndMQndvK2ELt+Og30s1x5MQi1rYdf9N4lys1G58zJnRnxMdkQhzw5UCApAfjPAzoeeztPIOh+BWaHmfGIgvrp1/HslnF/nRsPFnnQ6tTGjfyUpHdvBtR+9yot5+7PX6Opfi9SlB7380h99KXcdXiFLuTXAhn3Eab50AqmpiAZbMg2dnhwreJYipzVITG1gslLN/17UIFinBLvFMPL4SHreEeaBmMBj7GrujUERhebqPAzc4zXfrbi62ghSJPFUUIHQQO/fABho+BznrU/Sb14Qqye+wvLoPcjxgbtv8a1Ci3deOPoGAZgS9CD04e83K3zQdBbRJa+k+kYp8tNZlPn64hsmyHgfE0Vyp7kQO6OFoFN+RNml0DSkCp1jPHiOxBbzLotPPcET98O5m2ylZsQGpE3BJPsG0mkOx0oObfZtiLVi2tWdGMSGh6zYSNdI3oy+zb48cBgMtS0Q7XQB9YWBAujeJYV4n80U53mjMmvx6/DCpqtCVL6XbxJOEe2iZF7Jc9hUMkoCSpCIHqXdo+WOPPHJBjI0v5O5WYlIzGNz0XVRFtKoOzjLnbvHukjwKU7+EYNdL+7sbqDqjh+ezX7cir+IRdL+EDQoEsGZce9yMWoahz+NZaFMSWLdXmA2IIDuVJ0qHFodMIlL+TjtI+wGf0Zv96NUm6ycOPUBTx8ZTq3fTZo9IrDzt2JXcQaxahDO9c6sWL+C2r53GHp5KLf7X4FTAwX2XtkeJs1cyrnCmWjKzjHn1Bx2u9Uyb9xCSFuJKPJ74kug2SBlxXdLKY3MIfO9McSGeIE2CllaPWHOBZS31GJY9Q4hY67CShNUHgL7MJpsah6YwMvJnargXrg49IbcrwT7IP8nEMV8waiQn2kstCPxSB4xk4rA9vvDC6u0NrFh8Sk+ueVL5NdLuTriBsyaD5mreVYL32DFphZT416LZyNYRMIc6m49QUKKF4XmSWTfjubbjLU8P+MV0K2AUbdo+v0Yt7/siTSqhPbAEnQeDYJP8c0lEPkylfdvU+Z9mslHpnD6aC/Koqp58socYV8c8y5nd8zhwUkPcp/eQ5V3FgXaD+nh7wd3XgdTK2fazpL+zUXGFlxGpgijz4iDjz2jG+uj0d0OhZhmMltGkCIGxFberklF/NpgHGM7sI2ykWmA89axmFsy0TnJ8NE+Rd3G+2hcfqGapv/t+UApFzHHpMPjj1nUuF7H+i/fW9d6VxIVIJZZMPW/SU1WNLYBzxIU9L/V/P8rQywW4+zs/L/+w/+H4j/Fvv/E/1jY7n2CCLApPRDlfUeg7g7hMQbqNWYsjU5cs1jpK49kZPDPRLpc4duyJIYemEINNtrTY6hxaxUg9QAuqYzqusv3gbCcTAKTihHpelF6Hy6WCvJ3U3PFlP30DlmD6nnW/kO+GOgPbUdh2AVw6U1T6R7eawTn2BrUVT9T26alZ+mTgla07xSWyM+Tfm4896KLWL98PUaFjo9CF1BXvweV/02qRU7YumURPGzdycnSHZRbx/JRl4nSTgUxf03hzwHpvBF8QEi+l+5EJJZgxcrQcjd8T6Sijc4AbSIAkqLN/HC9nPp6UJo8qW8Rs1AKaPvC6LugjeRiYysqST59B2SwxN0o+JhcniUULoZfwq3Anuf3P8+1wZkMrXNn2fA5gu+Ic29sHsPYqLkFGvA3FbO0FgbpvAkDEqVBtEfJyW6ch49fIzEeTZhU4ciwgamNX4ouUlUFnQFfIplyjASvAii/A4ejYVQaafWTMZthqVcTs3RG7gSsJz5lMUjklJ3az/mtIzA6NbL0lW04i2WC9FfpTvAez4mOPlx/ezShSTc5NO4wYplW2DBFvgyAyGbjt4of0Pnc4cXaQCZ5i5BZWvjHs08sFmHfao/YKuLUsEPMtNNAxjvEPfiFbzP6Un6sH6JVGgrjJ+MX6iugjtoKiNDFUnloCBGNM7D5Cb5HFomDwAAFZpTkcPO5yxjbw1msFZCJhC6H2NUg11FV0Z+v/7qGt+oiK2X5dM3WsPhfY76myMStn+KQDZby3qLl8M+eKmg+OITiLW7AmBWBd4mUD/oMIMC+/LFi4fieb5ITY2TD71p6XztF2BAp8DoDWoNw+8yPza27aawso4ViHLXRoHsB8n9kj9mbav8SpHoH3vnyJWyDMwTWSfBi0ITwVlo4hovT2TbmMoawdMaoXxB8m5TuiDAzas5R8hp7caFkNlbnWMz2aUjbboO+Bhe3OD7/ah2O1dlc7HcRj8BaAf2rDoCw54RnT91G4/g9tHb6sqcd5roG42spgMoj5Gj6smNVKD2VGto1OnbGZOP1RDNYzYhDK9mvy2aBZgQnhp+g3qMNHOOFi2GFm6OPc73faQbcnE909rOceLmNSSOWwcl+uOqzeGHDUvRSG98tFVgoYpsBLi0EfQ1G60Bqy13xqnLF7mZPYkPOCe1emg72oZyWRnMm8if6X+jP+DNDSUjdIKDT97jAEzbuK/pyOPlnlNIgzif+jb+ynXPWodCcBboYZjjaUxVdTZNTGGf7/sJOtxbuBczlZmUFL/U+TUO9BN+u6+R5nuBGexRzxmyCrmou/32RrhW/U6KK54XvFyKy60Qy3SDIbFq60CvEuCfdI88riNWpX9MmiYLYbwTPqaz3WSFXcKfFhREnhuFTPIXSTxBMoBWuEP4cUXEpdFrCKOyUkugsyGDiNQZ6/QQye5zkcRSEpBEj7c2IqwPoOUOYPwlZgkgsoseFVAad74v4jR4UZPYiQQRyUwvPxJ3j08XLaCpLIv/ARmILYskbdZpOh9dhzBHu7o2hRtfEmIrxfPrnO0ycswvOjgBTC+rSzzlT15MHnXbcHnWP0vFbmCz5FOpvwI3F+FhiyTZCswXCdW0EaDuhvQjahAeoS6LFJrIhsUowl3pzoKaCEXFzBXYpwibJoDRgUBrINMOnjfC85pGOS4KdhknWVPLr3Yi5VsoozzaQiWBUGtd/foPy+mA62vRc/Hsgh7tm8GX5QbgwAWnwh3hkB6HSd/Fp6HVeyQllQ4yEwG5mtfv+BVwtSeQLg4anws5jZ/YWeD5Vx0Abg1n5FrX+SRS7nmVGFWilITxdf0OQWfQYQT+/41yvgr+1/blX6sfUoGYCas8LJ11/hWTtFEorw+hqUPK5RYzqiV488+/5Rjyam7lO2F+6ibq2lOpnnsGv7RM6pdGoZc2IbXYcmZ5BreIia0VbHn7vTGcWf6jXEiyZzlx3d/QSL4h+U/gAoqZM3ErTqAj3oDjciQjXCHiwAW4tR6aL57ULIXhVejHw1p/IpWaMppUQMFco1gBbR56FEUfoUXkFB0k4/zV6NYxFtltgoBt8a1CMa3/seJ9xl6jum0bCb3PJDsnng3Fv0c/rFO/+vR39qSZEDn4sCYMpGhFOl2eD/yyw6vFoOQcVTqRUhOByeiHbXrpIVtPHOMiqqNQ7Mbbie0wZ/iS+V8j44C0YFT3AXAWO8SgsxzGsW0B8YAmlXTJO1ep4Xm/mXgfIRTDNwZX+Gj++bHEj3e6usDG3WaHiENj5oHWKZfb+37G/XYbhgZVL9m38+K8+mSUa9vw4GUWbhmtiC99OPMWxF7wQle1hY3srF4OG4Ht1NsNOVPOLspynF70LWe9jUNsz6NwgWlwMhN6eiLvIyqDQBMACKm/qHJz58bm38a8YTVyuHSnyv0CyR2Aq26zc87VwesmPWCVKFFY3pqqWMKnqVwE8kLyeI35F1Fjhq20Dmanq4ru+4dB0WFhniMR8bZhOzpFDRCm0JNakMCy6NwP/1a9pYZfwiann03dXMzEkn1C5lpycHF6z/5vzp6ZQ6y+issdkknruQWqsFpDYRZv4ujOZG3Y3sZl1eDbOwMveS2AoX54F7oP5NiOQ8hMz2bdoFwmBjbhJ6wVPv7YCxIZ6ssfks6UVTuyZzwyfPIKupT48p0JpABPz1Rg8fiJ4SCzTAqqF+TP9BXBOAf8ZTLyYits1VxZlvkRk8n2ki8rAfwIAMmsHM1/cxhPlDpzq9ld7w3ssWI04eAZyM6UMlbcT1ZpG7poVzP9XsS+gtQOrtolXzzdQmi7FZUwiqJvA3IZT+xUqg+Crhi4erF3Aaqc2Fg5TCr7UftPx3rYW1RfLEMm7cC7yxzDjRnfhE7j/Kc7Nw7Dsc8ITf35lMS+u/wJ7x+5np6uSnp7TmbljMlBO6subiHZ9XJbPoesMc1wPcCJDzeSLc3GeXgGHI3GQhzP8QCzerQHU9c3mXsIx4Bg0RYBEiVxvZPGO2dz162Kl7yw0inZkEplQmJKqeWDdRu/juTgYT7PmZyvNg5ug+rSQuAp+mq0P5uMjgzAEhPVo0UqQPEK4T2lyxP6iwAJvis4BRIJHS85XkPAlGxv7cvPV2Tgruxj15HHUtscLSL81hhLaKcGm6cBD282IEctw8Wqgzwff87fpT1J3JfFDeRkfrlqItzYSDI08iMynxqWCng0jGLB7HraFt6DqBGS8A/GfIbIJN7Zm4nG2+Zk5ztMCUru9GIvEnnNNcowWMZ9GHUUsEtPuGAfjBCnx4ntHWb/sEzzqUzjt1Iadl53gaXb9aTDUg/c4nKu0JKYLe4GV3p0g6paqV3kiElm4G3eXMr8qsqOyAZDk/wyZ72KVvIaiXUVXhxu9bjsyxCWE3b3M0JSOTBXC4bFnkRnVTCx+hXdv/M1rfd3hdhToa3BQnOez937n53uf8rnpFHKJjJeZCA4R0FlORvM1Lvu8gLssCLv+/gT7joGgeYIygGMCnXIJxc71JJ4dSGxWLGJEgl9wN3DHtwue3/AtJlE5d156i56qxwEu5aqlbEgDifEp/tgykAnzb0NRBV4iKB94mVJ5EHUdz9CsLUfTefHh9+xsxbzR/1l+KDoAZsgwPmrzx7FRvHcsmw7bBZyr1BScq8Y5zg28Rgks+7DnOJmawpmehTy9fxCnOu2InKYS9ljNGdhc+tNn2wf4FLfiGZNPcFgx/z2EsdBp14nMJEMienwMZhhk6EUw8uxx0N0H6wRQOnGwIA/lb18SleDLqoJVZMblwqvR4BgHnsNJWF9I3/MzuDL9AS84RRDvXC7IbE6pwmoEfdRLZEXf45xJzo46IxZd9w/eeRWzy1us/30E6k5nDHcrON3LyMj2WwQ53ub3O2sZo67m5EdjkPjLCFfDKF0iz2CDrkqBdQB8cuEIjddLcS/NJc1UDLz+sE9isQizyhmDWs7EKgi2RfCM5yhu5p7G3iJFZhQja7IwsdOX8Y5Zgh/glSeIspvIc52u5HUoGBjaylOOcM/cQG7DMACCXWV4tLRi6Qjl9cWvESYHSu4Jvu11F9lxN4VLGxagH3mcvc/+QrQMwtRnAWiR9+OvzDrkOwdyJ/kGueG5HD65i2tjRwonrXBhXNgXkFpIGe4M95iHyiMIWPOwXyaVhYqgIhRWR0RWGWIk0JzJVPtFbF6yAo++U7l39y9+8TxLgtgZXeJHQlK3exTYxKD16yTc75rgvaj0BH0NE10LsAwyU53jxfpnVlH+WhvPee8HwM8xjXfPB5KQrqEjMoF7YR9QrYd7qlii/Wdw4/b7jBj8K5uL+lI5NY83o+qg7hL4C7oItUp/9idf47BfNn3+nMYZaydTFY+sLZ6w5RGmg2wrOFpkTHbLgUGNUHOWMdZhHPW0Ulc9kRPDT9Dh80g9AqDlVhR9zydxZdkHBIXncKejkrcHbCXI8S677r/Olx1V+EfKcEu8Ty+NBYmljd9uf8Hvdz7HYdCfZPbIxK/Ej9Xvr+bGyCI8kvZwcLbgL33TvpXKrW40e4QTNvAsbuZjkCuw5LEPQaQIYCbfUZAusGkDeRxM1M8+GNvZF1BkunCn0cSoT6rB2Ijq3jxcqvZSq/Ei4JaW0MKpiJcBvX6GXj9juXmP/ZP2o7S4krRxMSU2L4Z+44JtegdWYwct+0+xe/ob2OmDGXxhO9EBLgxxbSPW6M3NBi0KNxdyIsrxdGrjfWfYYivHb+IPvPz1Sl5/HS5suI9DfQG6LdOo8qiChUqIXQPdXp5yWz2BGX/T7uxPm0vg41UeYL2zlqI3PmZh53NM7HWBBvs5OAePIPrBC5zz6ETl6E7j6BnY+z+yaCBzDQ72szDpAhF1mPC7PhTXuBJKnQczf7+Qk/rQeomGI/1YK/qQPzAzh/91/A7YpOAn6iQprpByuY3XVwWibKxk12kbVok9iJwQi6FNmUFbOyzw/IZPTvemLMbIs1eeFPYO9f3wyTlNR44YXGz0KgNLqMBoLLwXiOKXSbQkNbFj2RnKVOnEiT9mztABOJ/sx+FNY1hxqhcNa2oQl9bT1qwRTm7sfbac+JIx3w9HJDYhE3lzTubAmeBpfJO5BtSBiDSBSEJK+GD1+wRXvcGHvh3MtM+AFvAXV+Kt8+SXxb8wYuIItL9oMGiUj0ng/nNrrN3k5NqOAAyRH6Nw6Q2Nt1gy5TuG9oIRJ3dSdOoZOr7/Ao3cmcK6Qdz4MYmoTinGhQo8m2bwWo8CuLkcEteR2zqGhXdLmTNfS5DpIk8lvwWFi4T3KdAlqeTasL/wN/Xiw+wkkgPrBMBk6BKsHaDtaCLx/Ku4VDXQ0vMsKWqQGWqBYEQiOJa/mIKmBPKGzich9kMcghIe9alsD6dqEji5aTgm6V5ODz3F7bYP6d3naewMrbw8p4uTzeCWnk7YjTRak0txyP0NW8ck1voYOB19j+TkWL6OWY1OYxXWJm6DwDGOkazDor7Mt3aDcHC5wLP+Zsg9Da25SD2uojnTjxFXUlFbpUxyseDduhNq4qAlC6dWI8+67uXXwuHkhjRyUixnttUIN54Fj+E4NXvx7B8LKPO1UhE5jJ46HuWEjE2gdOeJLfeQV+cRF32E8JBrjz1bZqmOp6qhUN6C09AzNPt3Qey77Da58MtbW5h4LwoSjAyx2KGXdNKq6AOx75OZ15cT7g4E9gjF8vn3VOkN1C4YjpvKA0RSvmsx4mo2UyK5TOP8v6iSyunnf1RQD7IP5TmXfUTawaLed7ked414VbdqXOURkCi41LEcuUsWI5zb+CUUmivmQWEO9N4EzsnYjO14Flym2stAxfi/KGubyKB/PaO37dy4sPp9ErLXEFxVg0jkDiIxR9pDCA3Jx8s/ngEnejBSGszSFRNZcfQude5vsWhmK94/B6HP7GLzxnXM0ev/l/PBT1IIMiZSke+CpVKEVNlIXeJoOk1yAu/8TVHUfW75wbomGF4B/iY1/10L4f+usFgsPHjwgNDQUCT/kpX9PxX/Kfb9J/7HwhqzhjLpSHx7LURS/DvWTnhv8iv8WeTKiq6ztMrySZY+zXO9BNmbPLOBxqxIxEElvPfue9hZfFkgkT8shth/8DZ/fJKM79C7zJt9hM3tddg5nmGE+2kO5j3P36IQHEIfkHA6glb7X8lcNY6emdPBdxr030WELp4xp9fSoMtlT9CbJJuGscJzuLDJBRrVKuLX/MAP9W7Uu9Sz1RDKR8V/MbTjI4a+BoMKg0g4PJJhZYtQfNm9CfUYTluxA1szvBHJVGybtQ29g5I34j55iJTtaUwmut6P4oAKUlZsocQ5BSJfBccERCovfC7dISwvgz3TDtOhrCG7cQ++6RMFmZVRt/hr9Ju0SQsJ7ITDBXEctA8Fz9EQIpSZ5jjlYXSIw/7gQDp1IYROi4es8ULy3mM4MqMMqVmKb3okp9wl9IgRFtNOzXr0nz+F0yg5YdPC8As6RIfPm+iuzAGZjk59LZ3tJhz9e/D28E8e3diW+1B1Er3NE5FRj7zKCZtBSWdC1MMEjlgkIulWEiX+JfzccgsnmxOf+EyAfrvB3E6bQsPNoafwqPZgxfrl5M1rfnzwiEQ0eHvQ4hCN666xpLXfou/0R3pHrqYHvL/1KepFFmJX/8hM+3bI/gKr3J3agGKupCoY5P40eZENrHJeA8f2AaD2+5HDv05CrLAxepqWMIU/cmurkJiqvYiXScMrqa9Qe/Rjfl/wLoMmXWTwylJBIlHmgK39NdyLb2AVyxFbo8mc0gLF24SkVNTrvCqvQzUuD01QE39UhzLbtw2FYxj0/h2A74xPEbTewHCbmINnBzB43Z1/uX7A8sp48rlBlMWGUtRKqOQg8DrnTNGctl3gjPU9whs2MSX4bVLyBNYST9goLD/DpdRSfFTtuJWmE+rSJCQTqo5CWy4ROiPlbk00uDRR51KESCYWpApNzdCczsSIb9hXmYK9oore3kdoTT6DU9XHkL0WiecnNHjH0Ogi4u7INZz6B72jjRa8n4Ayk4wOCcTm+fKntz2+fveFBLz3BEZ7hZLetBFHlRb7nGxO2zr5o+5tAK53eQItmOVws+8VdOiEJHT+z0jcR/B5qwMgYpB5AB0d6TgrU8G5F0ytp/nKLwxPPcUpvZRFF4bg49CFzFAqFJQB+84Ulv+wnPSE25x972OuWBOYW3UC9DUQvAitVIydPpg2t8E8+cafOLo3ProRBb8jFsu43O8y3no1VomZSKUNn9LPoXojTK5iutN95C/eZ2naDTLdXkRqVUOfzZw8+jt3jWdxt1lpO3OBJ063syDwOVDcBMc4amT5/OJcQpw0maQeeSQ5GFCaqmFPDAB7ZUu5MO5HehvexsWuDCzdaKFmIVHaHjcbrZeYusI0vLweEKgKhISvHqKDJw5Np2PoDfqVgZMWJgPIdIIMB7DV7xpyaxW/7/Uh+ORpTsWuYswCV3BNRdxkQ+NTS0FcBg2iFq4ljEDVOJp5disw9fmbzD1C4m7G4Xl4faPjtAf0aR0MD9bx6qBsNrVlY6sFtaQJq1UE3hPAqSeVlkQ+X/c6gUVq+j0TzYbQC6RJ8yH9L2jOwOr5FJtW+iCzM/Pak18isomxOuyEQUfAIRzthT8Z/szfbKooZUjte6wf/CnIpz/07BOZW5kvUVGTE0DI8elcSJDw0tKoh7fTvuxTom+Z6Woczo03v2JvoAFRhwEUTmw6/SmayydoGXmHzcvX46GNB7PAjdEVvE28ejblra6EnW/HapHTOvEROi7EbQK//elHoLcWws4/Gj/xa8FqoLriPhlDy2k03MfxWF/0fVwFpg6AsYGRzs/xxswNTN0wjqouR9oGSWFGJ3QUgcINZWEdL33zOZ+8/RzR96Ip+i8JxhqGcPhKJFr1cQwBdlSa9PCEjYL965lWuZjMIh+so6y4Kc2Mb/4CiswQOAexEcYdHIfeQ8sXoo0MjLrA8HufdIMi8pg+XInjkDLW3N3KOkUKWm0KiNIAsPrPIvTBeVJupKAedolU1zbyGlIIDUuB+iuI9NW0OXRiFjfzovRNbMoI+JfPIEDPYGiabGJfxw3ynUv4RLXgseNLQjq53+TC7hYtmnYN/bwOAbAsfhU/nXiCpkozLxz+muxxu+k75qgwzwHbjh5EdiuNjokNxI25QKHWgbqKABptLtQYNHy2+DC7Gk5yvex3fNTV9PQ4JfygoZEKeQ+ujVtLmWMD94IL0Jj9eFo3HwdgqzsUdpxAkpfGwNtDGNvwKoZFVijbA5dmgO9U5P1343rDGefiErbO3kqNruuxYl+X0pdtI9/HsckR3zJfSjWdEPIMJV0Sttw4jWuNESzW7gKDCDyFRHW/Chf6vPwTF5peovzaEeJkVsR0+3bEvEXK1TN8EATTxXfQrD2Kn1oG9z6E7C8AkKgm0aGsIaD4ewLqljF4JJA6EI72hIw1lNwLovBwMoMfBFDmUyYwtioPCXLGmiBEIjHjDo3DIhFT0nMe4Un/MkMBPqpJoFJyAvNwe5SW93Cw90Si78BeNoakRTncd/yGtsJ6ukz2dHk8iapIYMz48Ac51yBKqyLo2HrCBlYJyV6Alns02wchC6hggH4Aa2S13LCzFzb7wU9j9JnElkuvcUUPkTd9qC6SIh1+BabUgtIVS2k+xjIdIqB+2EXE9jqQqMDSBfGfAtDcYKBPZghd2hZyQh4wuqMITg+DHu9R4zqS6KOfITNLQQoimxhKd8GVOexJgKn5nbjHnGWt91i0HUn89q9i34B7GcI/wqDE4E2SSgoeQ2FsNpIqMetelNBarqNO+4BCswWRVAVXp4D7EG6Z4xGbmij1rmHh6JuUeHWC2hf8Z4Muhr/u3UUy7haNVYNRdY3ksv8rjBr3iN0hLjnJ8GH7OFUexkxNMb1qp3D8X/eqqXkPpbs0+LfLkSp9mYKwjlVZW0i6NRrXeg8yQmxkt0QSHJuPvHgzlO2lwfoWbXddaZbUUHXWGXundsSfiGFKNYhk+Ky30vtiNa3qYNrq68hvlwiFcJEExDLulbtTUO/BYq9SPvDVc8KSJxR9by6DQUdZPDSdqsQcqrPiifErRyFTC0Ci7rmyUyTnfuR9igOK+dz7Dmtkf/Luv/pVHNrF0dD1uNS50CPDnuNTV5NaLADHhtvBniYrQ872Iys6izaviZD5HDRc5/2oLrylJn64kUrfyXfJ99SDuV343Yx3eCsrAP3RWfy24A8CLZUsjx7AyDMjoeEazj4f8sMTfz06CdvjyVylpZW+aisV5jbatP40SxPwc04GOz/wHgeAZ3gZN57Notq5nmKXTn5M3Ie0/x4Qy/ASb+Wr4ff5timfCWJYogVx0Saw86V5whfsHW6hf84nrPxrJT0XHIPbLwmPTsAzzA7byfXOFnRf70fX1oZI9AIEzIHqk9wzJyLv+Jss0Wn6nhZRHFAknPA4oaCo/fYzRmfHUCeOQar9hKR/xBLGCuuPWRczSc6aTPPwLwia/RMq8Wqovy54brsPRGKsonfEEQpbXFniBipJxmPXpbGhjMjzu1G3BVEh8ePPBf153fM6f1W6833Kr4RaHRn0ux/axmhEY4R5nag3uJHbny0vi9DrsnlfOR1Ncga18zI499V7pPrsBX9fsvs/g7h5NeMPjeeObzN4DBM+wHzDecqr5ewePA6zuJqPpFPBsQc49kBkAalrMrXKEgKXvket9b9LMY005vNsRn9ODrmOa2AlauuTjx1PuBuBW5kLjpYKeniep6XWEVe/wYhUCoriJtCztx2U1fOe9w04ECBIlCf/wLNxR2mTdJFT50nNrlmkvfA3Aye+Ac33EKnCuecv5m5jAtNzOzFI6zgwchsTpAfA0ECTwZuYrC70SgMHxh+gzUUMTMFqE5JNNfZLqWr8Fc8OFUvKd9FzVR9BqaSzHMIFoF7VNTVuGVXEvvw7cyOq+HexD0sHf47wZd+1oRzZPZXmfkboKCHZcpPX5sCkBG/GZgUSdnQh4z75UZgj3Qai73IkYssMnCwSOoP8OGQVEbBOgoemAJW0jZ6emSz4aTVvXPuetU3whEbG4GG/CvswkYQ8iY320Ac065oZr4Y/PODH+8/S47kDVJwR8Yd2P5Knf6XBuYFOdScueSJyXPcREdoFDmFUO55g5OKLXFgtI+SmK6mf/MnDYp+5iw0+DzhuALuf38DDYiTh/f4C+FCUQ43PF3QaQrFhZVsbxLl8ymuhiyD0WZJeG0+vP3xpji3m5Sc/Q46cTpkOcr4AdSB57X/QElpFgfwcrUVFmBXefH39TxzkdZTrnTg6ypdjI44yNPM1TiedBiM4S2yQ/hIydX/8DsSTVKPhwrRWvtDCd869Ht4Kg30EnzWBWGoge9GvwL+sToABnTVYTg3mhF81v07YxdmKSRwu2QHVp+hXMJxbHKe/QwjPDK+kiUdFQoCGUCXH7O6Q5NrCGhcLP9lqOFGwiCVJK7hf148bKWu53vsaa7XwsiMUWZoIdGym3eiIq6SKVxzhQnMnqoQs4nybMIiF1K/RosDaUo1zZSaOddl01bRSHRyFR4SH4Nfamk2LwoQh55GM+6KBU4FHMuGJ1hwyz2uxys2URas4Zp1JiqMcRd0lAm8HIwlsRtIuxysrlt6iA2CZD0Vb8O7o4KoPbGnp4lbwEkRWK08DIpkdEpkddqIORtnBPWsJxeolNHfE8aQ1nBiHCvoWGFmf4kSzewXfnHuOZy9W0WPlaAb0t9IvuQmxTIWozxakMWm8eMiMWWoQGMxOyUKB1mMo3p3fMGfVLn46/T5RtbtwNHkDfR5ddAkYFUauVMYyfpuV91d34dzyHlnqjaRHT2HScGhIgw8vHmD/E3LENhOELCGh6RPmjdnJmzfe5t7h/lwyuzEv4lGzTmGl2Nkuorws43yrmetAyn+bzR7FdeACEKUDt6q/eXtEJmuSsgmYbeKZZ77BxS4TpBpI3Izssok/rw6n8FosioASBmfVI187HuS/gURJUtgOAj/4meuOIwnfq8XeMYbaiHrcLNW8q2mhdPJekv2HoTmnROmsZKr1IzgpAAw9/KspDqvGN7me0Ru+IkrTBjM6QGrHnMgFbPa5iczqTfa078ltvc2Ari0QtBCkKjiWSE4ATK6EojYrpjpHsBf2aUprBT/4tzGmBvQFep7+fiqnh5xGMvTRXspwv4DY0wcpu2PB19zA5eFemMPeQKEGNEFE7XqOErOF0aHuNBfAL2VnWRUUw/Z7SgK79qBS2bDLa+DZnVEMfDYX7H6ApO+woERfJ6atuA3tmZOkVwQSPruRfwxpVvrv5aoHjMhwwryjNx3DMwT1q4vTkTmPYIi3G30CDnMmexmzvniFvgOv4jbtoakbX19fiXdBJnLHArbd+5AZvv+6sQpnFH6umGL8kWWVMLzdmbiuvdARgsOd11mhdSXT5VvuJLzA7R4XUJoG0NTzJLqmTSzwbef92bvo1+MlqtOrqbfYw8BDgkS5WEZduSvKEgs1jpUs63GHsRrAZT7o67DZ+SNW6ml2asRtYiwyHx1OXWcg/yoAReKnuPntdFr8Kjk1rgNHUQiIlcL6vPok7zjM4VSTJ3pHd75bZEeOZRT47H2YBxabobrjeQbe7s+nS/fzmtLCv3A/WCQaqnePQC21sn/JKU4394U7bzCwdA+/NAfhUqdGeqGDARtfxXXBObqk4RD7Djtv/sH6gMVEdo1BOqyVnqpy3Dry4e5JyP4Cs9Nyflj+AwHmkazWwnwHI5weAoHzQKLEu0xD9cGncUm+y/2wAuIlIoh6VfgA9SE1NLoMJOByb9bfjuS5ed2Ah3sfQdA8TKp91MzXs9/uElVupYy3uPN4CHN97/R8fMrWoWgtBbYgFonZOXMn872TCFzTA6utRLgOVqH845A3g8SEKDTDn+Wr7z/735oPrlgg9sls1ONWcW/bchRqDTYrtDoHolvpw3njj7xeD+mtMpavf4YqXzscDVeRiV2A0P8/rf+/O/T/G4XU/6fiP8W+/8T/XLj0waPuSST7XgOVJ9KBuaz4ZgQtelds4f0f/lm6/A9klTvo8k/k/AevkG2QYbPaEP2z4TI0gM2KwnUlDeqj5HSIUN85R3+33izVXmRi0Mf0cD/Doq4e7J+1nVmXBhJkVGFn6yOwWToroL2ICHEVqddbsBh9yEipoHbsTSjPFWR3gJ2GVI5wCZ1cy1I0uKtrQOVFoSWQBflnKTRK8dRLsG9TckExhiDfMXD/U3x4gcW/LOZebC6jl23DbNGAJhC8xkHSOp46NYc7vz/NzhnHeCYqiwnSGSCWCqhTIMx/PzXtKmacSsZJYUK1rAT8Zwr9NrWySNdKW4sWl89epyuyL7yjgMFHHl6/an8D0W/+RMYn72Pf0Yl9Z7cRtH0oIlMzf10Zxf1zAno4K9ru4Qbhyfa+2A08R2ToJO5UD6W/9yZcDd268lYDiuxv6Hmlkwqnqfxw8weWJS8TCkRSDSjdWNLxLndOpWA+/RQb3GuYO1YDW4W2h/tvpebtS+wsjSdv50jS/Mx8IhIL/nVtBSiUT/Fg0CUM1wYzqNmNSboCgTmS8a6Q4NdGYkhxI79DjLr1KPc6ltBfpn3YZ0RKXPreprDBjtKvnuLvXtlMWuiCvPIIX3vX0OhajY/sOtfCp6OzqR5+TWJtpzxiDFqdmLijd7HvUqKcUw9nnwSbmVaHU7y/fQtleguEH8FV28lgp0GChIAuDr/AKVQvcuMnaxH3/dYy0XklXHlCaLzxJp2OBu4npRFQFMAHd8B72DGGu9XD7VchfBUzPd9kV3wmsqaDlHhf4lyDA6e3igQp2cmV3K93R5WTRHrEUa6PuMtt6wi2APusEzhzsxJfhQK3uydR1WZADA+fj3D5EEoaQBV1hrVPrcDFGspIEJhPSg/mB2+lJfQafhYvRtmBAy3QUQw3lyGK/oR+hyNQ1kkZEfYzukYD9dUHcQp8UmBBWO3omXqZ+3oZ1kckROi14eF/xle64n0nkYj0npx2byIq5T5EvQEOodAC2f0Xg83G9rCeSDT3Kbfri0/nZUx5vky+1hvJGAPjnGGvV7MwBor/Qt6ax5jrhUhNXZgGOnNVNYNARyBnLeiraZWncm7461xodWXM2nn4eDQBIkj+EVSepBzdgNsUPTfVtZy3GfG3aKCrSlhUug1gqbWO3r/OothzIEHj3xc6MvqOcD9FIjzMpbygg2vlRgbd2sPouD3g9auQcLZZuCP5jmMXurCqrITJIEBigfYiNHQx+swQZF1+mIJB71VDLlr8mu9B+suEijwRWUXYrDb2jThDplLPC9JHxRBbd0HHUl/Cz5/PorKfE7/OzYLw58FrDBeP6cjPB3H8CN6LPMlOw3iw84eGa9BRQlyBO52yClZo4UtXuOz2EX0dpXB5NqT8SmFlIrfWqbB6VlAeMQyFzY0xrgJrRiRScnDMXQ6k1LH9Uigf2/vybPh2OLkdg8dbiMqikDvIWHJhGXqVnst66KMC0lYyXwsNxa+jiYgl4rkGVJ7+gkeWRI6lohOFQYG6Q42iegEbygIZMH8wGDJAE4zK50VcGzbQ1fWPv6JYQOR7jRZeJZYypuvgeK2ZMMf7WG0SxN1G5AASfR3z7yZzbvcQAExNYwXmSXfUOI2nuOEOZrrwlyaisBNQ4uzS0ijZi725DqNNxqigegy2eiFBGPse+XVlTHnpF96pdiF43SzszI4oLS1QugcqDtCp/oyc5Hj00mzeWvsKLX26E5u568DYhJMxmotDDzFna2/CTo0iK7IDAmaD+2DQBFD4ym/cOrWCuHsOdKg7hLlZqnpY9DCrXZi0w0LJmKeZGrSFnvJZ/DsuNP5B1KVSiuPv8vMr+zhRfxI4SpVNzYNGDc22doY++A6lIZegoNnQJsjrJYlvU5uWRGtYNaLqDwmU3oS7B7ovvYyLlgDUsmrsa47TdKwN52evwZMrIHQJWC0MH/MTJb0r2KEtZ5+6kVfEaqg5Dekvgi6WdDcrDc2unPu6Bxi/in0AAQAASURBVF598/mv4et8i6nTPuLXAjfyrLX/DQn3cVo+xSVSXNd48aWziY9vfIXI6ItT2Apuvv0V7nXb8UyrIL+5H30RfGmIWc3W5HeRhXbQ4e5PmtsZdnm5EhYsbJrG7mwiqrmdThEMvR/FxsIPWftuFGpJLez3w1v1AceShOJCuAzsJSa0Si072qFY6o5d+3wK99qjVsrx92kgVGsvSOMBBC0QkoUhjlTq4pEoxYQZ1Y/1SSWxoyTYSoGkGK/Rlxgg1oBTIg+8NfhfbyH1WioV79ixMfp7JjskQ9luKN2FVjaBOkdHrLjjPl7MlGDBqJ7YNRC8CPP5TI6fTcVOJmeYTxWhctPDQh9OyXzXmkWEyI5pGRqi5SdQmkIFU3jnJFB58GWFDwMrXHkw6SBOcblIzCsEMFK3cXw/lZyMWddx35pIcOZu1MwCHskll9e64nlpPB0ucE8+gAkyC3Zkkilay44SMWOiwePIC5grKxHH5QsAAJfeKP/QMWD/RgwxiZjFSpqtAaB0hWEXQapizZ1ZNIdcY9mO7/l6oz/JmyZC9XrI/Qax90x2/zQRr0ovmgf1QaUtpn1YKRqlqzCErQamHZiI1Srimc+/J0bRDBYEYJdjHACGuAf8GnOJcizYxDZeq/UWxnCeO2LvZwgoCsCl3puRFZ4kJOYJ0nfdIcKG9V8vw38rFQ4qOAd3qrmz8grPDT1BedtyyHGFsj1Ioy6T2zoEO3Edfz4lyDp/y48Ci1lbg9Exld+e3oFF3EmaXQWetn5MAOgrqAzkn1zGqaSdhFYOJ7yyH4pHdb7uE5EQOrCMq88ls+rqai68UvzY4bO2npwY9gP1yk6ssbk06MbxVcRUHtQ0IJp1kJP1scRl9GPnlZmYPs7hwyETofIYzupGXn7nR4LMYxnYuQirVC4wjGWCrLCq6zbuxTdoi5fy1UtfES+dCZ7DIXUzyB05dDWB1JOROAVXsUtmxu9Lf7gjMGWwGpnXoqBdXM9wx5kUtTryhlQGSd89PG9HRSeZT+4k14igrSF+fL7wlnUiE8GU75/jQWgVlvkJ0Ocv2k8sZMOZWCS+en5ZtJkOu0Y+5GOBMW5qxUksw0UCeVoZw33O01MyhyVe4wSgnTaaeI9rFIeV4FrhhbXKHVOMWlg3AR2qSFQiMNtg1foVtIfnwwv3BeULwM/zaXZaPHi9vZkFVXNJDg+ld1elwDCVC3JyH/nn0eFrIKlQR0v1BCR2vtD93lKJ2uijggNZAbj//AQusw4jGn4TNMFs9rSwrBYqHa20DrlEUlC3z0/Kb0gt7Xypk/CeUURunyNESOsorHYkYYwgOXz30CGO/xWOCBOpWQlMce5OlXVVg81Kii0dpx3TaEgo5/VnQmmQTIWOZdCYDi69iazcQvsOO+JGD+DJuRco1gUIspKGBkj+AZFsOpFTK8m7qubbnf3RjTXx6b/uVVvtfBw1EZyLyedq6lWWiL+EQQfJ2P4r8Cs2rGT0KkBkfJTUJP5jLqZ/h6qrAwtG1FXRhIVWIjFUs/v+62zO+IiPx8xiRcB5Xj37KRN7ryXGZ7Dw3Yx3QRPM29GruGM/mLHWOjrsioTKfEcp2MyI7IJ4b/EcGg2ORNbZkNjkvPxfHi1Zo4zcvUPpMd3IyuRKTlsfZ80+1+hKXloCN977BFcXA/XWFbhajAx0/wxx7zjkMRMIceikb+wVMAMBT4LbYAj6EaeQexRm+9DpvoelAWUPfbzFA46xZdtL1N9T83RDM3a6Vpyn3oVeb4CdF+E39mF4eRv7Gz15OTIdR1QYNYlsuPkTDop6Dnds4uflfzD2wgTCr1fi3OEEuh7gNx26PdANTl6UhHmjcmjGbFTybyhHXVcdhz+eR1e1I7GAZ7+TcO0pOuUhHL47mfiWRXjH1NJD8yEaXRuUHwCrAYn3QnKj76AXuWPK66ReL8FH4sTG8YLw6c+Zm/gl/StKa32QH5iPtH82uLwG9mEQvpJ2t71sn72d0WWz6DjhxntDCrB2RoJIjEgMejszje5lxMhhtRNcaEij2f71f+zr+KWyNy/pLxKZeoUCs4ILZVM48E+nbGZ61SvJWL0abJ24RRbjhgKcXqZPeSyGohKmGwvpdd4btW08Lr1roeB3wIoIERKLBJvVhgUw2sQCS1eiArf+VOvjuV8VT1HIKSoGbaWf42KqDK606F2xSJrobW+m1gJtdvcAIX3rZGkBx3gyVYO4WXUbeauOwDt5qKMBzaN7IRKJSLqZhHeFD/InjpGktCC26QFhf/uOYQD+hxIJSLrHoWiw6l0g5xtouEZCzkf0vJSIcvIAZiZ/S16b52Nj92hqBlUt3rx4ri+zz9UT/FQQdx8s5/ADoSCc1dvCyU7I+2QxS7RtvHrYmXWjAqhuD+SIZRAd762gj28Zr760m70tXjTL+jN+m/B+3LjqE97e9AGf/DkP0V0tp409mTPkT0FSWywlr+wWr07awYINz2Lfbs+1zpRH/opAh3YkDjMvsU95m4uBv/Kk8kUQiclQD4SP1nKuYggerlJGLfyBWuUMAXhwazle6sFkXemJrklNplczAGaLAS7PBzsfAqxWlh8bx2mLEd35PpQMyicvYRBfVX5LoVGGKxeJ9jvJrvy9uBebCFQOgKwPEWeuhtCltHGWCOcCBk8xMUENYkuXUMDM+RJCnuFgWz0dAXmUS7YSdSOByI7lwKM5Y468GeqCaN1TQoP3dsyWAZD3Od7iZ4Ap2GzQeOcttLoqodDnmABJ33FgwxSO/v4axZNusnnFBewlifhwgIOzJ1LRGkpIeTv6lNMM086gbOdOxlvhIP/fE/zXgfGAnwicE6HSsT8fVzdhcLKntbwVq15JgC4D1+o10BqHWOSLg0aP2LEVeZArhgaY6jwWjBIInMsF21Jy1JArukTPay7EuydT/EQBbpZqEnwaOOZQhE1kY+z2PhTEDEfZ55WH55KRlM5+nYooSV+umNroBYINSfUpRgN3++3kctl0OnRraRO1ImkXCe/p9BcA0Hcoic6YQOQeJaUOT8I332L0nou8YgsJ8k5mX+tHp62Ta31vU+Zb9pDZV9BYwG/Ni0nU9sTR4Ihdl5EG+zOAMPZRONFc6kmPYh/Ew90Im5TJ0tgZUH8WpSKY1jFmGhRBBOntafSMRuf6e/cDK6ZFehH/e3+TlqPg3JBztGqbGeP7ycNiX0GnD9//OYzA2hByez+JOsFBsLGpPIS88hB2sp+JirnLeZMjnTfd6fCYgdKtT/d8ANne8+lzZByd6nxy1XV0GJNRy7tbdx/MOe1g7gbAubGRFEbl4NoJ7P9BOO4qAHdrPU2UuKUTWjkWk/MwCO2P6tQWDDYwmcwou5TYxFJhtrJZQeHGB6d64nyyD6rlOeR4IKxRC/+AyZVYOhVcmH2Lo8nx7Cp5npw7E+jndwARZgBuiwezZ/wXyKQhTK//ALW9XFjPJX4LLfcxSBIIXXWL01dy+G3tVFSz63i00wesZmzRD7ika6HNvg2RTfPvo4hEIiJyIjDJrezNjgeHcLi/GFfgwfgH3OpjZLnnSIx9rnFVoiG6e+0uxszQi31wxBXPMTdZ51VFs9QPnbkUjE30VuQi08CDRhPXdgyjV49KolLSoSkd1IGsr0qib5UjPfR+3PbN5Kjl8X1vVFwAZVY9ujVfU26VUfblN/g2vy+cn7GFZMXbLBhaysYcB9yaR+Lq8Lgm5scOxzHXBrHOt5zYmCIKTULe+pDfbg50wGWblZYRr9Gs2Mfp5lDKWqMItkFHUxa1Hc3Uyicz0C+SMaU5HLHa/n/OByMl4BzjyqTNS3i94y8y419nclEZugobblmXaT5dTmifHnzmfpJhUhuuMhvNYhO+909gNMfwf3Ox738y/lPs+0/8j4ZZ4giUgDoAkVhOx209uuZrvC2bzHjPQ4jEVURMewp4iv179nKyFBQGFYsbw3FxQpDx3COg0FR+NvJ6z0Pe+Bavrn2VrOfaOWdZxkCv77lX159oJ3tkran4jT/PRy5w1DweHJMFerjUDrGlBxNXHGLvb6PocT2FiHk7QewoFN6AcHl/ioqsRGRacE/X8ezHP4LSjUuKVeh/6SDI6sbVkaWUqA7Q0qWDxmjovwf748dwnHadLHUHPRUQr2gXpKhSfgVNEKdcbNgmHqXepVm4KCKgZAfUXcEW9zEL5q7hbGMAf99UE6SyYJNMEvTs738GftOZXqOjIDOAO0oZKGT/7RrvbA/hbtNt5s/I54e4d6npXCTIRrqkIhJJsQSVYNTD3ZBq3o4oRq/3B0ZhUts4N/gcgfKpaLdmYXVvR+20FsZkgC6W0ZdjqJcHUlEaQtkRKwfd/2b8pESBIWQ1oY/Mxyjt5ILJQrumnSfFzz48JwdLLV5+dzhb3hvna6ncEgk+fPhOBTsfxl/6mvVB0K/Ri/DeneiCZgrSd4Y6gSUX/Sbrq0PYeT6GGx6uSKTuWCyP+twsD2RR7GUcWhxYcn0Jp9wbmBS2nBp5BO35XwFWlPdaeXH3ixx48horl42D6DfIvnaRsCnzyeqMwcBolNZGJrqkQoEZnJI536THdjAfWVgrt2ZeZ6F3p1DoA4h5B235CmYMvs+JQm+uBcBf+nuQ/AMo3cFzNMNPJ2ESg/LYKGQmGbwsglvLoTUXAubgaCklOLmOHRYl931P8bxikNC2XkhsaFoUjD48jouDMzkdUIWrWei0SCRi6p6ptHnaULYWcquiD/0+GYBT4StweghjjSoCze7sK1tOq3cKKkUQNN0R0NaeI7GxHRsw6WQKTtdj8V5fJMhVSZRYvMbhffs2kdmRlKc1ckhj4rmoeVDoDjHvUtF5jxdT5/HxAx1jNs3l9wHZLBiWJnhpKj0g9Q8+s0k4lZbI9X43+GqgsJHlaDxMb0X54Et+n7iRl07coK9LA/20Jm65vIiP1zPY7dlMXEYclbFm/Dc/w7re91m5YYXAZLAPYWrXKgou+2JLfhn+WSKXbIOm2/jE53AqbSsapYITs46hkZgYqQ4Fr3gA/DxE1MXvxVoyFK+KH1HYSyHoKeEDOPESapkBmcjA+G02Vs0+xdD2IhgrnH903stodkxFnR2JtPkGUUFpGAJWooh/BSRKaowBdJ4/jZtDOfMKXmfYlLNwIIjnAVnRUgo6NBRPl/NrxCbOK2Yy/NpToK9hcNcgVr+/Gmu/Zj4aIuaGCVZIXWFaI4gVzF27hbEHXyTbUUthUC5lTl1Q1iSMwzEZPGjdhLGjgYK2VmJLYIhLCjOaMwTUXfIP+Bf54F0SSavCwqdurQx+2gB3BF8xev3E2cbVmGzbkfg7csMjm4WhL8HJ6xD5MuLgz/nz7keUtUYScDKTKnUfmgc9QBs8m2f/2EHYTRn58QOpfdCIS50Lb4vqmeGSgL8lnec/PYK8sga7t+/yXo2ZKHE8p8RS6ChBbBIzZP5h8kyg2NCbmtxOOhb3hH4Cs0qaVUKPZTVcrevJzGt9Ebt2z5Wle6BkOxWKvqRsi8ClJA47/xLafDRoTw8TpGOiXkFs58t2rYLO1GICQ4azNOFNaForIPeB3Y31vLP4A7zqJpF6cSe7xHuYHroEZA70HLKEyqG1lDTM5YhSwyFTPShdIPZdirNvMvziL6gMal55eSfvuddy2BoDl5YCYBc4iTaXyXSKKvH0rqO3ZzfrL+QZEIkpO3yDz36Yjs69hqmrN+LqGCtsUBUCW/NAZRthWTqM83fx/KB75Jj7Q0W14MvqO5ny9lqa+BGl6TSrvUwc1D9u2p0iO4ZkagU31HV82wxDdJWwVUSofA7T5/yF1hbA0g19kTTVkf+VidDudbhZKWfIuk/5tMSf5O2DaRQ1P2q09x/8sGEXda0tDGl2RtPmhprhDw9LxBKWB1Vx1y6KjY3tZOsbQSwTfAylGtBGc/qHZTTnKDg76Cz17m2PSR0DnOzqzUc31tLo+wr95aAQGR7/g9O3iLyXRnWiki4nE7lN8TQ2Duay+muaNNU8qZMx9LXbDA7cAhNLBDawNgZ3719p9y2npckHjRjCZHUPm5wc9j1nW1XITDK872/CBiiqN4O5CfxnY6xX4y+FzjYdn+ZH4RdYg7Ujl4EqOGhU02H7lVvL/sKnfT6TlJOQ6gIYEOckIDftw9G3F/LNgv5sKhrKXJsj7rpK4MWHvx9r78oW+Ugu1EcyL+RZ4hTtsMeF3qoQXh1i4C2dlSi7EDDAGFWN8B4EOj2SePPE1wQGQUXNGE7e+pOfvy7Btf4r2C4nUJXOxa0H8AzyZD21JHsbWDRzDoQuA5UHFUvWcvKnTgaISxBZi/AefwGMT0Dcx+DSm8KIH7kRfZz3nOBdZ7hvrgO3OIHdZGximuIai0eeYF7Rq7yT/Awi6S1g18N+KTulJKYnYqONfVN6cKj8OWaE9RKUdWxWrBZorWunvMETRecBGLQTdNFM9EjlWu9Q0lxncT5gDNEhCJJTbt1c926sV7WLB7bAIMwqb6GoKtMisnZikpkwKAwU2waS1TSUhSoZHE2A5kzUMZtYNOMCmzO9ufHJ08jGXyY8MVeQdO8GDZ0JrOKuwcqQbltpfejzKORaCHsOt/ZGZl8Yg6JIYCiovFoEoJP3BNb/Nhn/S7vQOPlwom0wtc7/YtaJYWLqj1yzDEKmMNBTAc2iNjAphXnQ1klZzBjEYnjVcT5j1SC2doHPJMjfwGinjynOmk+50zZuB81FKVZA9Rm4+iQkfPUQqBVsV8tvE/xwrS6HrQjJ5gmFXDEpGFZTwDvd5+Ml73jssbKIlFyI7/bwM4Gv2QsSvkR+ZCa/xtWTnFNHlczCLc99RGiGCiAkmQaFWSgImTFwK1nwRLLRAxAyJd6uPzNx/RaWFQ/gnDdc6Oz28jmRAr5TCAp4lXu9qvEq9qe6pZ5k55kQZy+AD7zHkdNuh9Vo4k5NAvbGbimq3G9BEwzeY5lrl8Y4Z5j/4UI82zxJ+fVxb4yXbDU45SeyJVhBj6AYUsreBZ0/+lYNHTvGohjVSVVyDbcD2wk9HSswLe18+GqznL7neqGb1gr23UwZiRym1gMw+0AIrdP+YvqOGURkRyFfoIdJpWCzYck4R1pTBHtaJMiVJgY6WISCWXe0qwLY9uMEAsu9CETPhQV7wf0Pga0gFhQwvmrypr6+nukHPscpcORjAJVSUSpTfxxOYnoSLepAtlX8SvKITrCaWPXlDArKdcj9TXy5sJud3K1QIrZ28uIHc9BUeKJYvZ33/Uo4Ze4n+OeW7cXZ6kqPjB40ObaiWvc+drLuB22fUAAokr2K5ekdNOYlsmP1BCLm24jtewauLYDwF2jUDMYh4Q5pjeOZ/esS3vrGHSJfgzuvgs1KncXK0DI5A++3EJs1lJPD7j12r2IcS0j94gzfFMI7juBoE2RPB3OMqIMzuSS3Z9/QG7TJMrivHE2UugkOReElnsyaVevwNw+jRHKK+Q4w9pw/f062Y9GBIi5Vl6HIjEAva0bq7YSzQy/I3whZH4Aulp1bl1J5ScGol7NIdAa5pRn2d0uM9t/P35UpNFm7GHHSnVr3Bv5rpLuNIH3ucfxrZJzb05OnXnhcbrB2wjksY05wpMvAkTrIVHqCzcQgp3epjHHiVNcP1Ge9S425Fy9M/EOQqU1bybGsDKqrQRH1JRneGym198BN5Q1dFdCYxu38QbhW3UQxOIPhvrUkVf4Id7Mg9Q9C5BZSQi5SdusD7EQ7UXV7cc6I/ogYt/N8aZjGfO/bVPYJ5tWnD5OnAvzfFJK27QWgcuebhSEcrVazKk+LpcOFGf/qU7vVyIGYW4iiRRyZe4UHJgdwH4Ki4S75N+fgZWxHOyuFbcW/4qLZTlK/EJA7oqyNp0jxJJ6esL2XB3ppDRNEyyhoisfHPpebzf5UZ7VhUOQRX+fK8epq+nv3FAAuhgbWuF6hvx28cFBHQFoE66LaeFnaCfu88Ja/xQ53K1bFQE5+E0qDTYL3zBp0Xcfg+h6IfgupWcTkvZMp8S/hVGI6ffRTH3VKqmFEZSSx4TlIgwfT7jocV4k/PQCJSMqC3xdgjqnCvV5NkDGIyc3fwa0WsHSxJCCRjc9lYteykmlnhmHxrxLY4l3VQq5Crycw/QiGTik7e0OJ9QQ/OWQTpEsnsyGebXYO7KzUcKhwI9OzpmN9ehfyrjIQSzFq1PzyxF8Elk9n5rbvkLTGA49A0dqGyyysDqTmTjS+zgEMTDqIyGbkn2Jfk07CweXr8S2PIjUtEX24RZAYBeYqz3G+rjeichfe2bwOWZ94Plj26JKUdfSirMkV5yI9EZmRtM9RMm0a7N4N48dD+YUh2LLDaVO006LoRCRTY7FK8NAU0dq0klJFIbXmIEYeOUV4kBMvjHrU9pb6PDqa4bfBvyMZDAt1C4UCqdwJRBLEYgmNmnbSE9JJupXEKwXVjxX7KiUe7CGRAuc9xMjBX1YHuZvpcpjIakMX7jIbsZLX+fDqXJZOiocSAThZ5LSIncfOoe6wx/R6HJ4yGw1dZTiWbAfAJHqTtDNJGIILGTLjNCXebeTZnsBVAm95VnKi4Brpv96h1SeU9sho+tpLBOULAOcUXiuqoaXUzMLIGuzt2wRFhKrj3Z593jwgjB8bwDn1Msped9ln68vz/+rXiAZ3jn05Eg3Q4fQXyQVPgDaGyvIYQou20n4njpCbYqJiutfGTelgs3BR6YxD4i0M7l70l49HJ/PHTiR42zmqqgABgDNm3CJe3bGH5VgYAAwC5gHuQDXwJ3AeWAqsBwoioVTmyVONH3L8peN83fU1nhMrud35IQ3e7+PcfBdEE5gUexFiL7ImaCaLR67BQVoG7oIyx4l2CVda6xnl1EzraCf+SPwFmcgHxHLec2+lRgRinYz2YRORWyWAUOxrVgXwhLWYl0RWzPVGfroeRFJcF/049fB6GSzCOkvRJCfTSYPKexW0DBIOBs6jqPY5ZJuP0OYg4nTiDQ6lT+GyWADGtRpkuBwZRqeik4tDb9AhrnpY7DNYDNz0OcudRXeZov2QHR3vMNwwGNWFVIh+GbzHElPmR9/jI7gR/CdhwYLPKxIVu6YIHtFjin2wdZVSmDgJ+/hfQC2sG0MdzjNk0jm2t2u5EX4Dm+hfSSzgaEMfIrPScWx3oLJ/EBJ3QN4J2hhM/ovYv3sMG9o6cVEX4NB3ObOeA5TCd0Ui+DQ6n9uv/ojm86WM7FxHZZofoanrHrZ/1fQjRW5mTJIWUspgY9z7DG19F6smhIuNE/Ha+B49UlSUjAYQIRKBSKpA1KFizJkhdOYV8PrW17k94KpgV6Cvhqpj7A2qRDu9nKjsfjRvXcndVw8Q1yce6i4htuvP8fxF3K/uj33WNQwKMV3q3tjZSsF3CuYiFVmxWQTUejIk9wOctXrgEIQsAqCsCDalz0Xe+jo1D3zJbvblmbK/4eIU6PMXIrcRnO1XyvqcTvyODcMQX/DYNVVZ6lj52Q9szgrmyvevYj80jimz3UFfQ5YfBNSp8ErowTetaYgr+jxUAE2QpBH2IIYqo4q7w2RsawN18DtMiHCD8r8Jv5nLoMxkvkWJ57lenDPdJmrhdMh6D6QqqoJa+PLlL3l+/0d8sjCO1O/CBbBF+gsw8CATK725kR7C36l9kYl6YFIpgBi4+yZ4j+Pv9jh+qW9m2c+r8GgzMeXHo/zjvwhQ2upK3hdj0fe6ztIB51hiE451WPxo29aHiNBgHgDXyiej6Ebs2Gww605PAr8PxHvaFfoWz6TUfyv9yx7QX2LjadOj+eA3iYJLFiOeUf4YUrtoNuaxxd/AtyXhFHf8jU9lJI41Ali475W+nBxxkmke7jz7xQ8MT5uBWfEiXiGPiBX/if+z8Z9i33/ifyzEF8ah6bqDTaZDZDEgyn0Xuw43nCva0Z9w5bB4EtPnBQveAeZOYhz6MPLSDhz0crwv3SVl/M3H2hPpDzEx/ggH85oo792Gg9aHjlpXntwnoKVeS3mB9nQHvvVJJCU4jRd9ggRfsV4/g3MvKvQ+vFKxjtZZ3/Cs70kyDL2Y6db7YftjbO1oN9nTYS8ju/clKhTgUXMGkfuLpFxPwSoV0xqtod3UisZ2AeouQ91llJJ5DBt/hTP3U9jzzEfkD81i2rxtUHsBRBKOOCgo7HmBGZeHE/XzdPQvV0P531CyHbqqSCi10mYtpCRIzI8iKyfl7uDkDD0+BPsQfjmXhN/NCEa/f5hBga9BN0Lmn4hrHYHPhSDuJ//O+mZwV/syPW4xyB0RieXM9ssAvwySxa6oveowdKOiB6iqsBdDdX0W/lmeXPAPITi2EDpKQBfLcf86zEH3MByajNjWSZfFCLWXwNgAwYt4wrEUa+8H9FZCohzAJiQyzB00FzuyfPVcGu2vMfPDJ5ijVAr+XOkvQq+NKBQz+OyFPkQ4x7LO5ymeCIaE1KehaDPo4kDlyaYcD+zuRJPwxFHC/Lfi5fE+/0xpou43dKu2lc9f+xxPmwc4hOOouMqnVz+kKM0P05AuWrQtDNXWCIi76DcwdGTyTsBNXq8uotDpN6q7QKmywIjroHShZO9Rro8+glUeyQVfM67qWiExE/MuaKPYWO/D9w9q0JU+xZWKB/ToPwtCpzy8F1NqbeRsWkGOTynxvXJw0PeFtnzBP88hglDDFJL6nmH7hQ94zRFSFBWCya/7IAD+SjzJA10ahx+EMeh2Ag69hYX+OPEJ8p8o55YxlTTvE+wOuYNTISC1h+YMahuHknYsAW2vdp68+y66OB9BqqdkG3gMZ2CFM+oyH0rUnShicimXRgnJBECk8iJv9CnuD7yEqdmeHk7djAV9DbTmYLLrx5LfB+F8KQU/i4TKum6/yqrjAvMPuBRdROVHn1JsUHLTs5lAkQqFrQsKfyOvvYaCy360tH7HtJIFTIhZR7tzOOTMYdnoMga4uzHZYxrig02cr4tnpUs35qijlKiETBptvlyueZE+6iwqGwZA7zVwYSK+bafpdTkMuUJEWdjPmM2gkANdNWBqpUM1jp9vTEDRcpZnLlqImp0ptHusF+hiKHT7ngspRiQF20g4vAanPieBK5D6JwQ+iaPvyxzI+QuRVUJFTBKbqwbxXoAPCoUriMRsLl9Jn+IAGrxEmLUuVNMDARsF4pc2crQ8lXBBRFNIIka/CVXHMZrmE5h4FKPfAAaf6Yde0Yl4tvgh2j/CnMvZNAcc+6qpenYr7qZo8PtIWGi3FzKy7giNZ/ugagtF1RGFeKn3Qwkv3AaTXHWYoMsCFq7dq5qU8e8LSduk9SC152/H2eStOEq0ZAoZoj/o/Y99h8INRCKOn5uBY9U92sZIOak+yJna2VwYOJENPfbw2oz9tDbMInZDNBEdYXy78ltuKufjH/MctnYLqtZaWmuKGVhWyBzX63CtEor+xFvmzFItnO6CI0Eu1NniMNr00NEIlUdQtliZE7uG+ozXcf1tKCGJudCcCZcE3w+pS28irvUiqCiInCtwWzyPT+dee8gskSkdkShPUO9sJsb9PUKcLmEztTwUORJ3zxduNXbEnv2eeo0LzP4ewpYxb5OMYjNkZKv48oOXqRiRy6yVwvckYhH+xf44tPnyt6KSG+Za5nn5CoyVkm3kNl5h5NAl/FLnyba+d1FbXBkHgoyiyps84wRCbpswx3mADmIc5wlyq5mrwbkXD2KK2Rq1kaVaqLIAIimcH9f9/H1JQ0c8Ay6aEOmfYIwli8Q+4xn/r7lfLe2i94TLbNnVi6N/jsJ+VQAx1gx8zMI4tGGjNSKJrobOxyxDMm3xPN+kx97OyB+fbCDbKIfpbWCoBYUbngUnmL57GfsnH2TXytuMcqjmqfppQvI+eDEvn75PU6OUkaWvY5FGIRoLWDoFNqZcxx3FNCQhOZwftAaF1ZH/GrczjxGxSURC1xpC4/PQDap77LjRrxh9px03b4RxqKSGibJW+oato9AgxregBybfcjarzCSYVWhLd0OoUHz91asWsQheva0maf88Ls9Jp0uSgrOylPouL2aeG0dAVjTmp/aQqLLSbBeHi1oBid9id/h7Xt2wjHKHLu7e96esn4oh4yqZ5wB/VxWjkJqwedbRXg43K8ZhHwHYAXgB0GmVsK4Rcm+2Uby/P/1XiB9zn5GY66j9qwPHxnK+yhhJRL883uhZhaYrH9cQuION2FojybnJjBzVLWOlCSLL4kT4hfcxZIhxdB2H2QxWdTLc3CyMbQcXcnvPo015h6RNMzENuwXvrBBYckC+ch51fjdodr6Iv2M2J4w9iElbCQ03YXozT2obaBDBvlItaTVW3k/2gorNcP9T8BpHiepF/jw/m7T4HfwVlI2yPephMQngs8TjBCa08POy13inIhwX236CgmZxL/0Vkk4rKc3T8uOCP3nP1YhR5o68bA9krqHIwR7zlGyu1u/hqt1bXLfG8FnDb3BuNPT4gDcrB2HMHcwRTwMl/iJ6yaqFOTDrPWRSByYvOMINPZyp/QubyEZ9Vxn2TUIhy6zy5nB4FveMbdjSY9hT6svhtffgby+wD4fhF/ixwp3KKgmvHBpGTGKu8C7tI/haRrb8zjNPHOGXtFf5uNdyREixDBUKXkfqnya2043G6lauHB2ILuWRZYhIBFudg6kaXU/HDS/e2rQEzXvJTHWxh5wvsVVtZsXwjZxriCTw1EqUPtdwnOkITongNwPfkh94x+Uwf90fw/C9L5E3tlgo5HVVQvFWplmamfjHM9TU2bPPbQxPvr4ZlaZLSDjXX8Fqc2XgOeH96d/3NvNDzz72XNnRQaQc9KU9SMp5hUGzQ+D0MLQmPRNKpNR3ypG7WzmamoWPLAtcPoPx+TRt+4AzrSFst7aQHfEMMhFYbQv4p9i3t7ODvyztFIqvILkVi9ZTAHmgCQHnFIaFZtDpVM2Oe8LImSACvMdAd1o3PiOMIYcHAgcpDbmBxboYSVr3JPyEjZtdgaRt8icw25fA6EKS9JuAkQ/7VfVgOdc3VeA23J1X5z0NzYCtCQenFtyWHqfL8y3k7TJcLFJMdn4oBgmM3I6NvanyrMKr0oEXL8+nZpReYHvVnAO3flhEgmCuU6IPc8ZvwahcKgAbAJtUw77dg+jUyzj1yneki8WMdTsCM7tAJKMu7zT7e+0i2DmYBAX0cm0SgIV/+wrsqiHHcX3wDW4/pAOVpIYth5a1oO0u2olA1aVCrzCQnzoPX1+g9Q8o2UnRjWj6nOtH3lJvnsgeiJNTDOunfwq77HHwnkGmbwc6ezO3OzQklMKK+GRBLqpkG2HKOVSs3s21Nnt+bbIisaoe21EUiYP41S+b8E4r2tyhnGuxMdpN8FOk9hwnHHpzKaWBEYdvEFHagEj0LkS9InwAw4NS/B400a6RcnlUDyZ7THhsDH5lGMPeou/o2eXAQjspYusm2PoNo4AvSl7FzzEMuI7RBhftXySqYyF0ltJL7EdVIDxf0UCJEZK6k55KaSerRy1kyQUfBuyLor2fhDdOX2DSJBhk/Ez4o6T1/ObxLS5JGnpvmoKDrJ3g/S4CqKxkGyKlO5suVxGSH0SfoiD0PbP4r9HfcSmtzUvxv70SsMArjzMKPm7zp8VUT49sV9rs24RkskTFjNzR+H0bhjQqE6P4NHtKRbzQb6cg/9xRgqo4m+CsTHK9jLg7QZKiGrTDoe82bMogbqZewtrHm3adliNKM1cAijZB0SasPU7z7KbbNBrzad60gsyEQv7oX4PerOZg3vPEBV/CKyuB1205LCoZTGSgM3HmTrj2FAQvBNc+HGuXknE5lmf+HsXu+Wce65NIJOJyP4E5n1gJnrZEzvceiLUpH03ZA3T597BODKWxy5ts20sk2aXDmWHo3NYAz2OzwXC1nhg7kNj0rDomzM+tnj8x5FI9d+Py6P/1VsZrgEIFxH8CViOlNe7svBDGXZ9rHE+sodm+mpHOn0BXFU62U5Tm6Lj6WyJ2dlYuBzdRUjif16rihZMOeZZ4u0qCM+IQS6zUpqSjk3T9u1M0ORnYMWsHLoZCJJ2eyNufYaxBxWRNHVmjT+Dns5D99vcZ73WU2+onGeqmgvyfWahNw5L/Ddc6huN/oYXk4TcEr+yJJSCWkeoxgruVQ4k0dLH29xG4jLNjZvQHDPTfxtc31vPJrr6oLqbg7d6Ops2BAZFPQet2SPoOZVk+flKQikyoFB30cf8Uqo0CMAGBrT7ymb95OScNvp7Hj6Uf8s7rj7z34hwcKHBxIOxMMJHZ/jwx7luI3gP2YfTne/oPP8nqa/PA2E67qeWx+7zn7xWIrlymZNwVbg3dzgzVZ8zzm8JTs/dhHXCIFz5agVPpXf5+6xfEChNiiZjXTl0kzPkmDqFzedDDCUV7AyGXrqITORLomMWepyZg8p7DC/meHLvrRatDKzP8WnhL8SvsFWRImVyF1NLJQKMGR5Mcdaca5yqHx87tVOVOroe8SP/MOBafm436iUIIX4W4ugi3Gjc0eim+olCaC7zwlPkIsnqOPWlrDWfvlOeRm3Ws+uEZNDYxojli6PMX1F+nOj+MtS+vwirrRO5upMUQR5DPXUZ3wWQ1PHV1PSGldmiSj/LplB+5bd4OfbeDzQJSNS4lZxi/dS6tcgPLE9P4YawdGBsFRQSxDBDxwlcvUOVZxZ+zt9NlXfBYse+G/HOaYnOoH/gqW1JzhP/p2JPkkje5cO9lSh0r2fHsbrIjKh59aYcSte9EDrkfYqLoS75U/IJVmUOh6zXGr18OwBx1Ko7pQzF5lTBYPJo/bSfYZzVyG/gGaAG0wCRgFTBXLCdqjBGDE4htFmrSCrF0WVD3vsOsnrdYdfsrlB03oeo4YvEj8ezD9d/RICqlv7Uf07XR0HgbXZmMN394k9KJZpSKF/m9KIJ3QheBwoXsCw74/DYV1XdK2kPiaKzUU+y3j4DoAHRHe3L/ZhRDL4Rjamvnyc3zcHXdBgOngUjCdcdFFH5wATfrDhJvjOagLIX4l9oJksghfBUkfo1DbjvFsSKyQ04wc+g1XnAEQ+h7pLcu5YmL7+Gw6DLvz12O7YxgzCcWC4s28UNrBBuWbtM+hQhEXWVgboOSnewZk8HiwDKkJVNxvTEEw5IDKJx7YbZKyLwYi6zSjca4fO71DsPZAAQJ78Jqgy+HeqRz1xiIV5eNUWqQtuWDvbBnENnE/LL4FwIZxLC2lcilelD/BGMzseihU3GesLRzhOdFcnLabk5ViAmLGoNSKrz8prs0o5KL+XTOVv4Mb0Vp/FexpeU+PYu+xed4f9yDosmKsVDWeyiMewfLoWjiqj/mfNwIgvLCeD7veVQLs3A5KYZePzG1bSYhtwKwKAyk9UxH6VcNaYInLypPGlzvU+FZTVB6Kl12XfzYNoANEgNcmoG471n+OPY22rp8PPNPoO2VT5Hzi0THxYJUQ2jBN9z0ha9qO8k43ReNQccAqwl22IH/TC7XjKHdkEZx9GUOjLnOKPUb0JoN2CDrfcQjJ/FunZS8KgcibyYyKqbysfkiWBeDwRCNQ8MYDF01NOjLBLn5wj849HU+I688QZ95YZRaRpNR+2jt3og7mlWfkdOWiNtP4zhnMtN3ewj4DAKfCZw5tgqnTY5YU0v4ZuU32KRyloX/Ksj2+k4m6vZfRCqg2lWL1deHCukoqJkuNF62hw9uBdHjTCph0zIYFbsRWefzwhpY4Qp2ftxuDiT1A0GBzalnF3XWFP7tdv+rOZmWsYeodRPy3f/kQPc33aflyido2/R0+h7G2tGFXjQUcMRmA5FEhElmQuQp4vNXPscqs3Ij5XM2vLuOtwtNtBolBAVp8fScyMyiRC71exO9oglddhglnq34nklCEmrlbq9cXMWZxGbFAnDNFwqkDoQXV2A1OKB08sXmxP+1IRaLCQoKQvxvmZf/g/GfYt9/4n8sRNooaLyFKPFbuPsWptZ7fLPmS35okHPv2D3aW61MtAuGvXKwmojRrWfgpQqsWg2NCT25adeXUSKRUKzTBFH51xcEZRoJ8M1n3cRS/mzvy82GdpxUrbToXSmpTSH/mI5IbTtZUc5Y+qsFXxr3IRCymNtpB7gaMRGA542QYgnlo7K9IBJD3Ef4KQ6jH9rM5s6FXO9ZxoeWoeyLMjIiZwVj3ypjXoU7fbIP0141nbpJl3EvExZX1bYwvB84EtIVgKOjH2ldvZiW1BduPQdFm+gh+xpLVQw6+yrCYgto1sig5xfQmovItQ+WO1mEVnhTOfA8JrkgFULed0LiVxPE6eQH2GKu0KIup7HZg+n/5TqPbJfy4KqKsppp/FRxgbefUsI+L+j5JaIIQeIAG2ToxfQogd+CRjAQeJkcdv86n2tRUiTvrmOWt562qI3Ynx8P0W/ye7OOZmktCX1/YtPT3T5TV7p/1Hcq/0Dvl9KPee6XKJSrhYWu1I7SjnzCL/9FvbuFuNEaNFIJaCMFqr5LKiqXYFrrfkDbJcJmuo6lJVpI3Ictf9ivL5Nv4Bx0jxV/zSNswiWUCuvDY450sObkJMpNIlpn7OMrl1o4EIQc8JPMoL5RRIWvD78t+pUWhyR2WY2wVYTM/StWrHkSo0WJLKkLkcwI2AuyeZYuksW3+HD2DZ67bceWDc8wbuhmQufNA6ckMNTTUObOoA3Couca0UQlDhXYcy33QeXJH21e+IhMSGLyeLpfLnfNDTD4uJBIlCjYY+hB5cooZjZ34N9vAomLzkHoI0bkDauWMv8ivM6m0jMtlVlDBNencHE+KQML+KUgFpvBRrkZmuUB6IbsgbYHrCnayY3n19G/9k0StneQFPYHeG2HyJfB3MbMKicenOjDicne1LnEsNInEHq5Q68NiPWd3E5sZHMrfNUMZ+WQGn+C6I59UPwX0si5lDerUctMbHr9c57XdW8wHSJg5DUADta9SnVTC6PsbDwZ/hnYujfXdn5cVjpQuFODv9JGwXEJn4d/xRqvMWAo5UrNH8SVPUV0PNwc04Nx4emPBrbajxFyJbbUU8y8FILx+ghC+2SA7hmIfQ9bl5TwtuO0NzvT6huMSGxGZHKGfR7CkLerI/zKITod3AkaWQvuyYKkb+NNUHmwt3Edx0K34ysxM1EeQ5C6G4F+dR7o4rDKe3C770s4u1i5rHiHZIcy7M9tFryEEr/lY7fjeK5vYVLW27iXv0StPUx84mdO337Ad6eO4670RV2Qx5KCwfQdmQ1BP0H489Tvh12e0xkQD/2PZuGia0NtbReSP+1F5Gv30Dy9gmLPYra6w5XOSuHZqToB9VeJ7lVPu18p1ceW41HQQKx6LPiWCx6K2ghemHQBw8grTCyWs9q9O43nNhh8haL0dN09rNJW7hxT0OvOYppf3YUucbjAkOsAu9YaXMvuUKeTEOJbRR9FBdj5cFjaTkbCbVTX7SkdXEZDUxG2UKDjefAooW7y19zVq4itayBi33h83fcJC2Gg02MOA/7Yj12jI8rUZaydlkd51Sqw00LWe9Q4r+Kpo+HU8zdhS2RUa0zMkdqD11gImENHgYH8ccfJrHHG0Srhix61EPDGY8+OSAQiUzWbC/fzmV5F9dzUhwug1PaDHK2OZn29jJaoEBb0fANulUHYMsbficOuxIJMI+F6r+t0ej3aMCnMjcy4NAhphSf7NBe5JzFh6W0PgWMhcC78/SRtbyxkgH85382+i9rc/d2Y1SDVYC0oYs2aNfjVLKD6+Ekmz5AyoGgjZK8FuSOD1YnMVcK3mZ4cMkr5LsYTEr+D3G/AMR5Vcx1znjrO7m9n0HPbEGJHPfPY3H/MNIpppQeZkxOKf4k/uWIvGHqJu5fP8FPNFs6067gde5UuSx6NlXcFeVJNIGKrCPdqd8QyJclloLToyBFLBTawvob+QfmIxxTT4FaNk7wdka0DarpRti33kFzPIyTnGmIHL9ydNMhtbTBg38PzavBLoFWhZLgddJofR7MCSJUmmjyaUJUEUGaKZaBrv8eOvzbree53SHBc9jZ3e9xl/WuTAIjPDeDod/Mp69fIXW0+7WontNKX4P7HIHfij23DUV3ogV9KIYpyT661hXC16BcMBrBJWvmwz7ecd2tFn3CfSS4WaEX4NN5Ca9+Pe4YT6EWelKTMxslXh6fdSU40wVx7K2aus0Vhx8jCPbR2pHOsLpEX9C/B3z4Ciy7qXQ7v68/QM0Op93BlU+OLDPlXn8x2keR39UbeUUnQ5VT2enXwxuzZWO9/zoRKM2BD+aCDsUfGUp/4J77OzYAYa9dtVMk3qZWq8U2XIhMb6dQHgM0E3hOw66pmwuCD7K4IQ/bUbiaF1QtAo0vTwG8mYs/tlPQYT3rgNiqdD7FQMlSYCxrToeYsz6Gh8N5wLv/Wl0ZXB5yeC4SiC2DnC3Y+NMlDafn5c/p7OfCxG4xziHjsXt0yuHFPVs22F9bxjoeJWvFcHBwcCBMfRp0Szh5FKDZsOIlBam56yJR/t2ESVwwniTQEsOZBb4y+XSCxE+QAby4jtHoOdy+GErt8I9+lnmVz67OgeRkSv0Pq2p9FWvDU98Dnx2G4uLYgWtEFvTaCJgCL1Z/XGwCvEtIXlOLnGCMwFfW1AvIeOJgWyejdQhFD6taG2Povdqn7II6Y1RQ6JNKj5Fe0WhHUnIerc1ncW8TsgDKmSH+htn0aZoWalf+6Hl+JjmOx62CUdYrA7geh6BX2HC0WHZ4FUrRldZQc70eu5ll6rtOCVA2hy2h80MTZP0DtqCPEu5QR7nbgmiqsBUUiOq0KDO0qJB12PGg2sME4gheGvQgt2eCUiF3pdVKrfel0aOX6tP00dqT8i1sKsebLvPzH05Tl+QEFDB/0DWjS8ANcNixlVq0zF1414fMPwez6IijegpNlBRe+mosusZTZ5hC0jm2I/tVwocGT66XOhD8YzOnjMdRN9xY8ZtvzQelGvO4PnJ3vkVbbnwT3U4iNi6DNIsimegznk7hC8vVKSq7EMTx0DybrfCRSteBrBtwyBNLrojNFoRY8RiRR7xDGv7l9VV6zyO1di1wBa84d5pVlJajvL0OuhKX9rvJb+0hee3clL0fUsvbam3hdmAStOYxNHYQhvB5Lhg7PBheCJenQcF0ohoSvYmrxcOR5w+j0G0SmXES4XR/I+wHSX8A/ej/XQ+ZgktRzvus7RDYx28USQADTiUQiMntkktkjk32hcKLDC0ytAttGLTACfNwTyXLvwCuqCv/e9dhEkocAFV+5huDBkZQN9KROOgFnbQfUh0L1CaqifdjtXM3wlh9g9yImzjsPu4R1mcpvEg7207FoxKTsX4jYGIvhWwfoHt8dYhfe9DZQWi6jNMePTnX3GmHQUbDoER0Wkkm5EbnkRuTSX/O0oDQy6CgonHlw+xeK3H8ms3c0poBw7tQcJjpm3MN7IbF0Mf5OEsZ2O+au/gKdxA149G7AKCHxRjIjj4/kS6mZnptqmWb5jWblMM73/oyYGBi2/zKK9p6I+4qEhKv7EIrLA2i7EIddVxDL2pZR4qLH9kkUojNDCbU7RIvfKDYu2khq+iLiLr9PdcRwWPgaRL0GwM3okVhi9Phv/ZMWE4ywj4ekrYJErg1c6p3xLfNl1qffUSb/794rTtxlqPdN/pp0inp5PcNtTz92fEnLJbLTOom6tAH72FzkhhoQhVFg1uGo6sJikZN6xRvjgFtw921BbjD6TYKtx2iubiXwjpVhO1+iaNkeAnuOArf+iCzwep+55JrNnD3/ADt9CM0TP0dXLPSpw2bD6eIJnGxGmrUmrBIbUlMZ/fx288edT0nr6onqF2eco+5xesAFWh0jBeCjnQ/4CjvJNWdjGXEmFUVKOjMDyh/rkxgLX7rAnToN6RmDkQREQ+w71JSkM2jaN/yZuZQe1/cx8PZtJH3kIJkvgDX1OTzVMI3ConB6KDwZrXal2PZofk12uUfK88fJNKtwEysBPYSvgLz1INNyvmoS0adtKEccJzWsipd0UFqbROKIFHKqF7Exr4Xw6Oss/jmZr3b50pQPDfZP4Cy9D47xDHHPZe7v21m8dgrvb5tHxIrdPPQNtlkZL4sgp92ZtqIG9Mps6k2F0OnNp873eWsE1EmW0+fqNoryDfhNtgenMtDFcfykjqxzrsiCWkh/bR3tGhtjxDI4GApOieTLIX79B3y/YyB9zw7gVn8D17Ne52bFOE6XTSAv4Cu85G0URjhQ6XoEV7vvGac1QNaHxODJmbZgXmtuJ+PF7xnv0QjiNx5es1a34cRXvYJKrkfS/1nMcjvBJ6473jVfYZbKhRe8x+PQ+zduNA0mVBME3uPJSi/maM4w7rleIfftV1GbkoFH9LsrmrWEObmRGniTb+Kb+IkCRCHPQuMtxC7JVMY5UBwxkPfc3+MFZ6gw1pDbkEpuQypjQ8wope0ojI24laQhj5JCWxdyYxHyog+RNr3Jwt8WUhJUiHvvDIxD7yNXdksHl+1FbVEzb/eM7ncSzI55vNCeYLzO2x8JHu+5EjOiiUomVJ8gueU+z/36FmU+reB4moi0bFw6YsB9KrgPRpJ2izGBDTzQG8n1LUWDiFFiMQQ8AQFPkNAKg3OeotzpL9Yp5hJoc2W24TojNDCxEhx63uNc0GlMRhnrHzhhDX5AT+mjvYFXQBVl0zPx3TWRsPxe2Cs0giqMuR2sFlJEabQGmSm1b0BkFSH+L56uBl0oBf6hPHBezPJaGBO9i7ERQaxPy+DIqvVEOL5Ps7kFbUMbDR6xOBkFsGo/u0om6+DGzTa2/D4T7Rh3VjwHVVXQowfkfyIjv9ybBs9yhviGIPe3MvnScQbJZKzS6x8xeZDyHlZSey7j/OxvAMisP07skF+JmDuH6df+5q4tEBtiWnxeR+3xKuI8G8vKg2kv86RT4UJ2/TCUU/2ZfvtlSFxHrEsp2tQiVF4BtFcbSS8fgcRYBcCW3F8wiiuwV2spk56m1l5PtSqVgEYBeFOS60fMnQCs06UEzDmIu18NxLwNSjd+278ChzqQ2JwJnjqE85dlNFWnMqo6UgBj334V7+y1WHqeQqYfToxxN3AbScN5khu+4g279ylWrKePWx1P/bmIzIhGHEcKlQkRInRNOmIzE9A5dBIrjiXdywXDmDMolUD9DTZj4J53Gb1vSWi8FcPPNRWsEIkYevg0Mee2E2RQY3JroEfGWConpeMVIeSqqk3+fNkIPgolMzcuYFhkBfLh18FLWHf2d7zLLFf4pbAB7690+I+6BisejREfdQXu7k20N7QysOkbRp6roitKgtJ/IiIRuN2NAnM5ep9SIqx6LgQs5aFtX2seE6Qy9lR5MqDCG3ejHLvJgvKMRROCzvU+V+fsRLrnRbxLDLQatBjdJqGoOMSmPgdZWXuDSkMPbI6LsacJBlwSrELcBlESuo2d7ZDb6zleGTMdhZ0dhH4PXdVYtYl0mN8mIt8JyawDPJlcjHPpFuhWGg+QTODSO8twdarn61XfohZF8hLPgc0MxX8R2FSN7mR/rvf//7D3l9FVXW3fN/xb2y3ZyY67e0gCSXB3d3dpKbSlRerU3WlLqVAvtHhxdw+eBOIh7u6y9f2wUuS8njGeL+97X2O893mMsT/AzJK5ZK45j+MvOp7sATYhC8L/ApWrmHOSqTl2oid9riZwe+UvFPi0PiZJKVf78trOY7TaXeKZb2ZwudUdCtpB6cqNMy4Epe+i/OQIwk/8im9EMIIgetSWEMIXLR0ES9TEGpqxWVrFgtr1p6C1kFJdX/Yv/hUn9UB+94pkktcVEfA+cC/I7XhKuonAgm686q6nQDGJRA0ieLD2OuiCOBXfl1Ox11hocyZe1UxxfU98ir8XFc+CnwCFwNXeVzHpEtCoNiCY4dFVsaCUcjPhNjOPj+KpawPQvSSy7AQB7g15ljkLpNx/+VXizQJD5n7HNUTgTlxgIVfXHSSo32e8f6cNiSDgZJiKR0oriwGTQsubN57jz20KNlz9igK3HJQdSmrGnqI2pJwzDsU4dX5AleECVybtJyckB8G9hqeVKgyGMSRs+BCjxoNOvRmsEv4FAP7fFoIgYG9v///+h/8/iv8W+/4b/2thif2U2tpmXO6+gxC0jDLNCC7mHuaKxUplYCqN7a00GB1EqZ6mTKwSDcFjK0lrjSFbPhGPf9+bYFEE7GbtIUr2O1DTvxs/yIPwjuvJMO+vmDzgDbbfe50GjzdJGdaOd8pbRKVHIthsYB8qLmwAnbWe9V+8QIu6ja0zdyC4C2KSTiYiNK8r1nPSq5yGxlj6VQxgSJQAKc44CSr+0XZQrtcRlXYNl+IcHKr+etDPUO1cVm90wRYQhHKiDQdHOVhNYmO3t3k2KY27+/Tc9PuU/f5OTA7QgEYJY8TCRuT9M3S7FU/3In/s7dqRTWoSWS1dCGKzUxOdxmpuHu7HMU8TM559/DqHhN5iwNv72PTZfIacHYJ1cVemRmlAsHbyTlYPGlLD0KeFkRKTgmSoOBh/U9cdeakLMv929qnribeqGWTqkiFK+5DgjOH4Z8aQO7SGXCMEKxDlKsOeB6UL76XH0Xg7nIzmbrxlmMvTZ/wh+TVQ6Klu8KBm2GkqJHo+33UX3zA1Xz6pFtmMGZ+i8CsmdfhaFBWfEnMzn4qit6DlBxG5b4gHiRxBkGCWmSl1rOX99g4OSR5KHMXbe1Fd6oq00Z34nHxkpjLwrcIms2Of5gtqx3ijd94GHZBhtgelOLFz6bxLpbwDmVRK3N2XcROaaGh4Go/S3yHne+TCfK62Q1urmqbbVvJ9QwipOAU534E2gA57X84PPI+hdQz2nYPoplPBP+5gaoDoN7nhYWHHCtHL7rscOBszEAxeoiyf1USSUYvgkEtAg460lGB2VGg4mrkBkED487xdHUM2Rfj2yCI84RrnJT6EARttz7Ipfw3h2PAo0zNE6YuDfYGI6PebRZ30APWGeholEryWSDB59xCL2Hm/gb4brbEzUTi049gwF8GkwaBERK+fHoo0bgMfVMq42y5hhYORpxwgVeEM0i5/I6mcw+MPc3j8YcLk8KZLk3gT/OY+uB+9Gp+nOL0Rl+Rf2bx3Ocs/2AzT60HhgKTsD3bO3Im+tSf90lchdbIi6xRnfkJbG9r6EkzNznw5MhE7Zb2YjHAT/VZq7S4AkBSvQhtxkUbFCqbbrOA9mbqiCqa//SZ/FswlZNMvePmUgKmLcyJIaKzbiGOnkTZ9KTvlm4hSwoyKP8X2oCeQXN/LsEwFKS51zFqzQ/z/Hl+K7NOq81gIQ5A1Y0JKvseHOP5rJ5f9LcR/Q6rVk2RJOUa5BbOkBbMgATQE6EOIuG+PhEa0inTcTw1ixLDvRMaAsR4l4YDIuvFe5E6863Vc1E5QJErjZVgrOKPJwM6Uw9qfRtIcbGHY8q5zE2TMvvoq1S6pPD2ug9fj3yVNrRcXRV3xTYeSDnkNfYOgpwGKdSPwcRsisnJd+jJKVkPj5b5U5GjRtlrpfT+SzFXieCYIsGDVCK602shtn8x5DwGd7CRcr+FQm4TLHRaGYaV4QCmBzY8wr6+vYHv8UV6ohuKGz1n36p+4eFXBwKvgEE1zVSdB10vpdq8bV7r/jYMUKjBBiZgELNUNIfqAQKvOSFmPb0WSow4YfAiAsWWbWNa7kpU3F5Hs9ikxXo+PgbQUsMw6jf0nJ5BoncaRMUceQWyK08+7u0YQILdR9+J8fN27GvY4Y6tIYNjePtwaV8PBsUcJUwx5sJ2mo4CnZ59jU4mB4YcHgbU7LH542HuqQWSEHqNFbseinbuwBXSZamd9DVYj0bIBbHGDE+dkBJ37CGPiMhgXJY5x0W8Rm/cWfUr9KdwxCcxSJCsFCHtW/AGm4lNsDEinYdoZnPLeQmZr4VEjF8EKT/z0BE2+JcSu/AcjC8HlGSxCEuYDwwk2yTi79A+89ZfpWQI4fQhRr+JpKWPlDyspjcgj1c+e56NqYce/iQyBEcHOhEdXk3wjkJdyE7kUkQf+L4tMXkM8Gd0m4aTxxG7yUYJcG7EJs0WfuOSXYcRV+g5YQIu1DP9fFiPr9j89+4aF3ebJ9y8wNXkepoytzFc+3v5NvZIKk5GGufsJM9i4XDqG3h7HifGu4MzybTS2zGT83+P53XSF9UGlYoFIquWKyoy/TxFp4XKcBx9htm8Uyx1W4KQs5N1Lf3MiOo2U4Ay210rZ224h6d+VcdZXOOu+JqnXJw/OQe8AUqmSYDl0V8KvpX5krB//QNDUq/d5SH5F/NbbLAgISKwSag21/LrgUyQqJ/74d4UL5DTl8N3Cz7FQjdxoRK5xhNgPSLKbiedvTzPxVk/aZiv5fdHv3Fd78OfAr6C9DMe027zz5HHWlvvg0ecKs/QmclvGEjC9EQQp5TtWMDv6N7YpI4jzzMBDywNGLEU7UGmKmDE6mUM17V13VwCXASIKveoCu65+SceeG7TZudLsloBGrgapUkzwy9RsK/2A1pAUotKjWPrLUqrXPp602lE1nNJyBTVOGcxvb+UNaSIRd+/yS+s4zo78EmfJcubk+9Ctw0phzzcJKH1XZDGnqJl8fDIWBwOcjyZywiVwiBIlyDW+3AuaS8W4Nky5mXx7aTa2pxwgdzNkfIZy4n2e3XwTU20bHvHv4eDQjiBIHkgBCWWFzN42G5lZwZZ3u8b83EckdoEa9wZOjDhBhbcOmSaOKc49H3ZKF8DNqgBamk0Mb/QkQFsN58aCpY0pOtAI0CZtpN69CplNy7/SWYIAwTlBBOf6UDAoA83Ef0jUfSqyXvVRtJj78vWRc/gV+nG/52ys0q4l2tUF4DWeA9YZHJ11AQWJrA92pc2ty5h3cjFIlRxPfotjzz6UaZqteQlcB4o/QNu+g/ff/oMXP5+K+5q3ufXR46Z+ZdJgis0F1Hk00C1Rg9mzJ0ju0eK9GqlmAq2ejQSev8K7X61j2nu7ieoWAlI1AW4LOR4jwS5QRv6VTZjNVmY+QhVWNgqs+nYV90ON5CbMIi7cA5xMMPQU2Efw5cEq7M8OQj38dRL0dSTXOJBw9gdRSrDvNvJc/GgcXo1+6sfo/p3mjbwmFseABKeJ7Jt7G5tETmZFBAHymMdQzxJ7Hc1OGlxTV3OhOZ/JtpnEeU2A0oPcLh+JzU6g1jeGebF/4Vn0M3RUQ1sJSxyK8XI6w8SGjRS88jGCYjg4xIk7bUxjeps9N653I13Rg187JvDmROBmHwAUdGDVjcYoK6ZXUi/qDY1gahbVKUyNuLd0kuoLr9ZCUAH4SeMYWXdblG8ME8vDY4KWEvdSJ4nN53i5FSx2QQ+KfR5CAV/Gf8J3Vz+kPL2Fp/plQ+4ZcOzOH/EpfNZgxXbdgl19Mbr2DHEj14EIcjv2z5bz9923OJmpQN4pQYpFBNp1e5fcK1m8fPZvOpudWPrbbCx9uwBWXR7iPUwreP+zl8iOTcVx5jGkkrLH2ic3vcnCE7Oo6XOTNyb9wwFjDzizESpOwOhbyGwSnlu9my1VKhoaE9C5P+K1DXiZGok+Mg6TzMTlPldBupjps34hvQtIaLWCxCKgtMgY0viByNoOWUnazoMYfnfFXW1F3VpK7+k3EdyfFRPs7RUMq1qDzq2UE81HiDIHUiPPBvqIChcSFVvcTaSb4Er0ADRG/8dyUoIAY544SKttP2Nq7NC0BfGf3D67oqM0/tjJ01N8WTI9nWu2MqDHg3afG58iuSxHmHeQ2WEVmK2LAdBpO9m2chvRtifx7/83L/q0QNohUDqDTEPfOWvwWNjO2/u2YCnLoVURBlXnoOQf6PsPkXUGOkoMxF09jkJ5gWav53Ho+xIAFy5tI3nYburkRrz7pRBgccOkDuPVU+eobA1AGvUzZ+dvITYllpU/rCTtxTaovyNek66Cs2OdI3bNdvQffotw/8d9CAWbhfD9Y2k+nUCQVYL7E9GwU4e3uZWSYLhpccUzJxRHkxWb5RJUeEHVOQoMvShoaKGmsxrJjnn8ZrIwc0k4u2ZoUcna+KZgJsGh2Uiz+7Lp+5sII9fR0+WaODY7dqdv+lAuDGsjJfZTwjoV+MqMXGvpC+F9oAZKvFIxD5QgCakjUJdLjSqCQq+/cOq6Hbtq4vndegM3wUqxCWqaounzb6esZj512sHO40soS4rDw78M+2EXQLuQVdUJHGu9yQA7M3ZVzchbrahlrqIyS+Ai8prNON3+igZrOAei65GbdXwmkUHQUtD4suNcPjdlF1H1vcKNHrcZ4bISo8We84VzESRt5AXlkReUh0vjyK7nThAZ0MY6qu37cnKXI8FVznwSdYoDxVqMrg/TvYIgFiXUbZDaaw4aQY7Acf4F3ba7PMWOqa0E+ik4ozqIhwzm3VsO93/Cq90BT2MuGc2BpMXBkebH5eAHDz1F3ZAGMmqd+TbfnnY/PXiOEiWLgYS4pZS0OdC6awNbXXOZOMKFz0b0IbliBAZ5FDGarzmnH83qHz7hcns/8NgnKqQoXYk7+hOamaWYrkUh+3syW3oEs2zF5+J6x6UvpsZKsmPuou4q9o3wKH3s3KwSKT59kznoXMK+HjeZrnsHEhZSdHUtvSaf42q7Gy7OrUyIOIbKWgyNkXA4kljHmfykV/BXSyLLRh4F4HkEaMoBmQZHYzYf91jHlqzBxP+8jOLuHcTM6seJ4n3I7EN4UpZCH7cOpnw1jNqbiXT+4CsCpC7NgOg3mB9wjwGR5TzlM5julggcFDpIXSsCtSeXoheambj2ALc3TOGtd99izI87gIfMuKvmVRREXUOd7MFhRxUjY+zhWDyBykCuO9QQpBPQFmmwKdVUaAZg8IiCfttwfqUfjueno/BopSS4B0XyMKQdRSwOfwvs+/CL9zTMT7STH3qGE4aT/FPVh59/vc4vL8fzyRU9jW2N2JTgFBXFhH5BSNNfYlvDD8xx6EAiVbKhHo6X/kVyJxR2OPKy1wE8780Gh11IJTbG3I/g1pkE6r1bCalyI2TObXAdDL4z8PXYwRPPXuH5mkWEb/6MJic/6t/7BcesZdgHW7kkfQKtVwOhW47SkJgE1cMg611wjOOW1xOkjR5EWNgegtxu4aAGyo5CxWlebD/F7DWVNBW8ysxZIQQHPU2FZLRYTMnfIoInAUOP4ei+3EJFcQ949g6m0NdQXxnOkz2eZ9t9R1qrBuBSpiNEOxEPrc+D98pQZ2DYmQFAG95M4+dnMh4qoTj3ZE2FBgst2E2q4pdRX9K9YzmrAItNwa6pe9CZfJhQM4ruNxOp71OAZ5eqhiAITN0zFZ3NAXObikN1CuLdHkIAY3QFDKr34Fi5K+WRWfxj18S/sBlBgHinZF4eep5p94qI+WgRF1quM2e144N2XaMdT2xey+W+lzk58uQDthcA7sOZ7TWHstc+ZFBmNBuja/CpHg9/i0qgJSY41ga+CS0cGrOZ0NJ3aY57A6XtKr8eHEQ9vhgR5cZbTY7gGAcXJoKxnlOH+lBzpC+SJY1UqALpYXcCLs+CCTkIbTB96u8kDdDyp6aaPzvMZCFBQCyA3pGO4Jb6GrWaZhod6pB1doiefV4ToOo8gYZKfJ/byfFKmFYQSr5/AUhVIhO9a7SzCTZMchNXJO20tKj47dFvlgBBt3dhVrQxaA7UmIxwZS4IEjoii6lQ+dGjoZK8mz64+poe3GeJIMFQa0CHEuOMs4x3LMHeNBVKfgabFQdVLIX+hdh1qqjPH0mb/R00V+aJ+WWtH/l33yV5Wxr2Y6/Rf9wC8lpmgXy7CLAFDLpOWiRN8M0C3sPGmCNxYDcCTC0APON8lqDlqUQVHUdq+QpN6wcsfwRG+LZzdxZWvcONhjIcmgU8BBGYE65+lbpuamrli7ky8C4DDBW02URAps0Gz0syGWWAS0YLfetdUNm14RrjRU7ibEJubEdubKW1shVQkO4rkkOca5wxZgQRFV7BkMEpXL5tQLBAWFYYU/dOZc/UPfSokvCt92RCk8/Q4N6BQ8UxjPkx0KVm9X9bWCwW0tPTiYyMRCqV/r9v8P/l+G+x77/xvxq65osIHVmQ+jpCv/GszzCgazEQHPAUCw2V1NXtgvHiQvXuqSLet7dgdM6lUZ+AXuoD7IU7L4GpmXpPB3Y9u5G49KVkbHdH7h4E1joAurmeY01nO2f7bmbt6HbeMwicM0VB9y/EyZldMC6WIuy75XH3SgyrvnsG2xdXIG0f2AVB0BJ2N2xlf+CHDLm0juHVjUwdvBNi3uNYbSiLkxdjZ9LRFDuAKo9IEHYCgliUqBHQWmtos0p5ss9a/NRNUD0NBh0G92FYrWuwmKRYbVJaTY7iJ6+jWkR06AL5cckZzk65RNaOUYSorMit7aLkX8ZnUHuTs34FVFXqOXVmGEn9Uv/HNb7EGq5lL6Bt9jNMd6xEaVsJY5LBoRuYW4iv9uBmWhhN7moS3H2IU4mr3Ot2ISQ9v5Xe9mOpr7FniHMn0rpzoqyk1pd5f31NY1o05YGprCqK56UhHQyZ+j74TAarmd5mNVeqXWgzmWkSHECigvSPAPDTr+DHBRd54XY0dl9spKOlPzAMXAdBZw3myk9ZPWQPb2T68eXo62S5S0XPj6JdYgE24gWu2EvZXm/Ht3P/wqSwPuTbA0Z1FHf7P0NLlQrLjxJu9wpg/bc3sBbu5oLHi3TaBEZVjGBA6gAkCUqI3wDeEykpkXJw1jxURn88DvbBZoVqYwMROSKD7q41hob1T9PhX8kPTx9mn50PI6NWw9mR4D+fFwN3MzEmn0FZ5QzSb6awYza4+kJDQ5fcykUAFv2+CMEmIMyQQOYG0TdkXDog8MfiPx704wnZeLG4BBD+PN0rRjH3hwSu9SvlkxE/MdC4lOWARCJj+fcrMDlqUDS4sd3cnVWbtqC+8TQEP8VU5X2mGvX8JrnEKo8buNuimWxcId5L+3Dyz0RztwkSO7cQbT2MvnEedLSBVIVN60/Kz1OISo8iw7eYPxw6mPv6COi9GeK/RlbaSGMgfF2h5OzZ5RTJduPrXS76oVWcAvfhDGxeQ/JFD8xqLVW6SGp8PsM59Q2I/QDP9kzmdi9kV3EQ17x+QymvIKnbdXp7WMne2kDE5V/oCJtGkrEvUWF38Owr+hdibmNFSiLSlEiOTs3njut9omR2cGMl1F4nI/g4E6sqkSt/5YnB0Sgd2jDJHGCuKENq3TKe9ZsPMy69D425w6kV9KIpde/fwDGO8baPyT4wDmnfq4QFFjPU8Xm+D1wiGqHbBVF25jk2hJ3my6TuPH90JU/NP40pUoHcbyIIAtvaepCcVYRffS5DTBOY45IL26vwsOtP75ye1Dc5kb5GRkv3L/lZG8b6u+9A8W4srZOJrLVy75aeVvOf7M19gZ+XWCDmPdBHc/fPrTTpdiCpDcP9wjCKpbmQvQlS1sPQk/y7uK8xOnEi/0nsu8VA5Vm4uhi6f0ZyznOY2huY4n+WEH0GhYZJ+Fzvck2b1cEbBQkM2JFAc69qhqzbxAY7YLsC+m5D4j4bX00nOYKVEe8Hsdl5KWvf+0X0SCzx5flfp1AZlEWzzcbfzXCqDSoDgfKjvH3bl+xyHfJuVobJ8vBqjuWSg5isFgQoHnSZ0u6p6M/E8XZxL7p9NZix3SdCZx3GCifOTzqIyWogoLYTwSwB5NCSD60FoIvhr9S3qGjyZFUYSAVExH/1FXDpi1WiILlaCVaBlCHn8InJQeisB5XI/7imn8ymCe+iMgbR0g6TSyXs87JCZy0y72qOL/gLlSGRpiDY15L94P3s0ASzVl9IprSdWWOvMtGuhTJjPPwzAzqq8LZ/kdVT9xBUOI4Fv6UR7bMfGADek8Bmxr0si/4l/lzTpyKN76Ba9h24fgijbwKwP/k4Odtc6OiWhiG6EJX1EW8ZwCKz48dGiI7O4+7UYWTaXgY+ftDuLSnF2GqHTRCY6tZOuvUmXH+KDmlfNgz/FqXVhUFHY7FrDeDcVzoGe4gJ20Q7GdrpJznUEsWYY2NoslyDHl1FuaClrD1eTr7kCH2zfTl6fDAta/PE5GBXgnD3xAtcbIcJxRp+rIAD/zKyEEDhSPgNV0pT+lFY5kyNa82j9VEAck3ufFUPNUY1ekSvs0fj6IVYvAtcmTbci0WxP/DUoR18fCGUq6FDqPU8xxCzGdWqrUzzaRS95yrPQPSbpN9ZxuX44wzKu0BsoZxlYX0gQNxnpFMKF0+IiPJ56Usxyzqo+qoJ185UKD+OwqccCKNee50VA/tjL9ESZLefZfWQa4R+cn8sCTdod69iZWwJ9x2jwGOUyEaL/RBJew2vzD3P/knnmWoFh3/NNbrCaGrGzT6D5k4HkvydCbDPg8tzCLQ6MlStpl4qxdcwDK/aaGI89SCvhAsrkGmWElMI7R32KORF7LI084FUB+VHoewYyfXu3Fn1FtL4e8if6kr6y+xExopDLEGfjKPt5FiW1SvoIZOhW3UZ7FbAuAxQOlHrrKSihx+j+vzF9J7raGtPhyEnRF8nm41Fsv0EvXSDtRtMDJUo0WquPNavEbUvIvxmz71uiRwe8zsSQYLFYqEePdkmcFLY8N03hrtyGwNil8LoW2AfxrQLY2hOGUZKn3L2PvkLUp2b6NnXW1y65yszyNHacM7WUX/Xg2aLF7i4i0x6mT3qnCScKvJ5Z/pJ8e+lKpH93FGFHT70Vcsor3Dk7uVueAWXYBizWiwc6AIB+Cq6lLzgTnZdOYVHSzzqR+yQb5TeoC3pK+IzRCHW8EkXxIKdQyxP3jDglV5BqfdFJia+RmGrH5D1YNv32pxIv5mIun8+uzxgH/egsBiyNyKLucax0cdQdmrp6ZiNt7KJts6d6L2nQEsuHZSTHnEGzzoX/kjZzoABMKElX5Tj9hjz4BgOEhihgXXyvZDuJLJp3IfRYhdD/2IYcUf0Kh0qWfLYvaqQhfL2wreQWNRElnxGlXo0EbO/w3ZrE1+/Mpp3Lp5GXzKeJmkahZ7JRBkug6Ude9tdanyXoPCqZeOTmwH4lIeAt6GOKbiPN5KrDCDaLQVXFzMoIsS5u+do7sozCdUbsD+XyLZmHc5D/cGpt1jsc+nLjfzZZBU2cLy7mMjqQAD7h+W8KZoMpiccYfVZB2qlf3Cn7nVGMv1Bu5tdb14Zn8rB089R4yTBYjVCwCKqb1/j4At90Y06Q0ncGBKHPQedXiJwprOac1/IKTo7Gc+AGRQWPMHgwYDOJMq1af0pDcxCu9CE49lthF5tpWjdXBL774KOKkzqvqzutZjL5X2wfTcG357pIpvxpqhOERG7gd3fLyThlj+ymBRqB2lFYGPcp+Aslh6qzU3U0ISmVYNRYXzgyQjQLI3k010/UpcJ6sxBfKNr5dOBsaALwlaRAoAtRMcbv76NXGoUwV3Dz0NrCVv/Wkl+gQHbsl/Z7FvMaaMXKAeDVIVVyGfwbwto03QQMvso4X6PF3iC1X4U+tynut2LuXd6UhJeCcV7xX71+hV1m0BVcjhFAbmsq4buzp6iBxdA+XGsyrkM6qyjx8GZNKRHIts6mPWP7H+qfRK+688yvE6Fj2wUUVrRYKay5kf6uGznbkUPzk7OQCK/y+bOFiitA7UXaqGTbXO24WXqy4iyPlQ5xMCRGFHGLWgpI9RV9JXLOR6ez9sTb7C5qZ94P450A5cB+N2MofJaBOa47ZgMdtR3TIeTM6D6Ioy9x1CNjUYLnL16FW8Xf/4z9gkVyBOKaJO7cDRzGeExMY+1xyeex9nRi1ne93nL2sh9nWiQe97/H3Y3w6fVVo64lPKGS9cGoc+CRMmKUh9a5dmE+bsjUfUmOuRd6Kp3CAW/s+5QIn2uJuKoq8TXrxDPay+C8QMIWYFnewpH59xgXgW87Ahq6kBmx5iQ7wk23OQd+VwKgu/jpzExKKwGb1cFROyF0FUPZOOnTLxMzrgzLGwB18IIUh7pkyBRciTNGycbjJp3HLewIyDIyZVHMyT3Hs42K3n9lHwb9wnDtCuZZxcKvjOpNcfw4qxPsDMFMufqcVqbrehVLqhkbQDcaorgeRMEVLmx6PZusu0C6NnnpAie6bGBMG09nr1S4dAo/FNjCVj/PqvaEqDqIiqTJ/4Rz1CvSebcBy+RUHsBk/vTKKwjoL4C7MIo6XCiLtubu0PPsNOpjkH1n3W5gwESGQcy36QsqYV6/wY8pxynyjYRFHoudXgy6Ptl2PvXoEg7gkEowDGnFjKzoDGNf2zzKJ91F3eNHPtGe5CLyXB6fCFeL8safMp8sDeNYWt8MjKXNv6F4Umxkuat5WSjgZ8zddjVBOLSvwJqroJLP6rtuvHrgPdxqg+lV1Ivqv71f+6KKLWW18+9QktKBwGffsZcl07abWZA/HDJ456jKfwUzRo/8VwExPUZsCupJ+U7+uI2I4O3SuOodVc95ou31KGZSgt89stUau95ovoj9rFjL3fdxfFaFXevrSNZ14NJSkfCnZMId07iu/o11OwZTLVfDu93tlNjamCETAsBCwCw125g2LgkjvrlEtTkSIlulFg8jXgBAEvHHT6JvsG0TF/CM8OZVJLIoy6fd5W9+HDkQ8UfF2kTnB5Cs9M0JgZfxtDiQbHTPn6XJDHXNJPYnO/E22wXz8evJWJtscJLZx5ek0PiN13h8De7352E0tlMiFkgTFqBe8RKlC79GCIVeP3nteypMlHiXUKHqoNeyiBIflX0h66+xJamUL5vLGeh52UmeGzAaFsgjoOO3UGu44wwjCl5B0j0KCFOIvuX8P3wmradIm/HQKoKPLkbfZfYSRtBrudmhxu65iqQmlmxcS7yqBzaRvpBv00AHC6NpEeSN2lTbnB+/md4d4yCssmQ9zvk/U59pY3mZrCGJeObD5ENY5lqTmL9FOgz6T168hx/N0OjNYXRmhReSf6TxWV2zK3u4HivSXyYsZPwzHDmFfpxN76GCmUQrW6L0ao9QJBw03UZvTY4sKFlK3NlO1ns2Az1elB7sLMtgScrrzBa645LVCkrux1GmXMXBu6j+M5NFP7XqekQvz1alYnede+KFyPuU278MAKrDCyCjScqIcsfSFoMQDDQWwvHpVYKm64xx2cfnZaTcG84lOwH4OdMd6TfT8et5j6FpmB23FvPhMnDaJcFIW8rJOetPNTzdVQueYXq0odrDokgodSrlF+WbWWQZTk3238lSBOItGgL+IwClSvjrqwj9oyV1CeasHnbELpYmj2cb/Fh92o+qpLS7ujO/vWfkuhlJco+EoAgVQaOFg1lHXJM6zfxrhOYjA8LOEn5O0jesY0ebbW8+9JraAVnfu1qEwS4Xt2TyWWQKjVx7YmfaNW2Mtuh28MxwaGRvITb9LvSjzDkaGcV8IAPJtdx3ySjU9aBU5QGn5DHWdvr8rWMOTqQNkdnauwDkJsdxffDuQ/b7ruibilC09GOe94t2gPCQRUoKnopHNnQKSPauRrH0gaupvUhfNYlNJQhQgmNLPRtotO+ic3lUtpMDwt9yHQ0SDzYumArUrOU3VIDzfougsGgAwBs3LiCE4YfGZo8hO07BlHwSiOj20pE1RmnniDT8tXk6xwceBtZniOq9sfXWdXN2ehGXqKo2UzvfdFYvLQs7HUNbFamj4hhUfIcXHr48eVbb+NVO5+dXVMub6GQF/bNoLnVHvP66yzXw0lLHUS+CmkfMFeTyid2ctan21H4PZweMpwJTxwUgTPGRkr1o6mIj0LfsgmPbVPQzqwRx4qmLHDuywWXDu7VG3g74QYWqYUxkqdA7QWFG6AxE7NNSrsN+iX1YpJJj2raVXik2OfZvoNbf4RR4Gdhx7zNzLd9zRRgsO5z2pzs2JNnJT+4lLmKbJ5rEn0lbTb4qi6e/BM2IhLNnHzrGa4MuML6UafQCOcoivbBqByO1lVLR3YhC3cv5OIAMYfpWOlGQbOaGgvUqpPodlVC5J0ZFHsX0+DQQElTCcqK91j0ejZf3dlC7R0FGk9v/m8Oi+V/qhj9n4r/Fvv+G/87YbMh3HmBasNcvGOnIL2/GXVLJoPPDSbmbgxmryp0KiOanx969eS2pJDq/ySOdY68vm02UUO6EKddqB258kUsrrVo7tgwlKdja+3N+BWj+XpzCkNH6RmXtIgXVHJ2dpo41OSPhzZEXADLdCCRUysL5ODUd5D65vBUpxOlikaRgi0VGQ06UycLds3AvUBDUZsOY7sSUt8guEcHq777BJlUYNvyFyiJOUk33UbWBt0DhQHnuuu89cN6Pk55l0/+6snq+EJC2SMWAqVKbgZ9yNl+nXhWFxN1/zKS+oGQ8h7c/xniv+ZvWsnUtnJ4bA56iYyftH5guiIi/qydHPx1Kc1XPDCu2MIIr/b/can/bn6T5IAdjLazMt/bymlbjcjIApHtEPIJxfYOeOuzeG34y1SZwoFBLHD8Hrvz3xPb/QTyz334Y/wlnppzCuLWg9sQYidOx27KCa78NZve5ybwfXAzQ3wmi/sVpKzskURj/Fmc879BbVTTZmkVCxaNaVS0xzCvAu7bbIyIc8Q5rBBqb0LpQYh6lZrszXjkdGCVXCVKXkJv0xxG+kwSi31KZ0Cg6GoM9v8MxWfFjzS61zzWZ0GuoaAhBomkE0uPaTj6u0DCQqq8n8T+8lDaK5xQtPZj2JlhTI3dDtdXwuCDyIt+oT0YXq9s5IsJ5+iQl3NKtk7U+XfuTXuSgGATsEg6+bzbDZrM+WBdKnqeufQjx3SKg63goMtllYeZn9qTYOzD5fE2VROtyf3Z167B5FGFzNoiFvoAlE4sk+/lPX/wK4AAGbjLW0GuhygxHbLIrZ57oQImtYg6+5ed1Fdr4LrGnk6NJ1fCj6C3K6NBGYy64TrcfIZJ9YHsfG8NwUMKKE6spVNSJXoyXZoBPX9EsIQjNZlpr7jPvbQIZIvLSKi8A5Z2BJf+5Ae8j1FhRNWhQq/vRG6qhYvTRDSV/ilOpPnQlBTJoPNOnGUhi+Z/ApdmibIKY5OJCL1E25ju/CibTJODiafrf4LiM+AxCr+Ou4x3hlPlRmZ572GdSxPnOpZD8gZm9c1lmdkfP6mMlq2JbOyr4sBrXUK1nbVMt9pxudINhfU+PjLQCB2i5GrFSdyaL+Na6YrcouP7RNFXZL78IevptG0QM3MO411RzIqkX2jVFIDdCRGBXp9Cisod6bO/U3PfjaD9Y1gx/wicuQIjL4NETqPMkx2/jcW1yokG5xqOtQrMD3sXQ5iIXJJ3Snjmu2docOykvT6AUqcOfEJKUDeeYfikWn5MXUI3/TTe8XwJqAGTyAmSGI1orvbEhwLydDVgliBIDA/YefMsG8jb9DTZfe+yYfUGDPLuIqPWkABSDVNu++B6egxHRv/NmuZOxntV0C/vBLQVgVRNv+uDcE49C0ziU8NiXtrdlYhWuYIgo0wt46cnfsJZ1Yfljy5ANaKPQI8UX1RtnUwItFKoaePDluW8NnUJfmkzyNG2Y5MKxPwahXebO0WxC6joeRt36xFSzvgTf6cbqW9bqBbAR4qYfD87CgevJZztWc/tBme+TbHHq9RFTGx6iAhnp6KfuTU6l5fKHdG+8DF+PUqh4wM4ICbmrXHp3Po1mLjq+6TvWsvOeeeZyRsQI4BLXwS1B0uirkLUVdY4wJcuQHMWqPqK44UgUOhfSFC+L0N+m8/1XjfhmdOAjZ3GQKojjHx2+2m2fmaHtvdDKQZHfRh38p5DbtHj6h6FyWkrEa6DwBwAHVXIsBIoB5N9K92eqULj6CBuWHsDJAoutvah6IM4OhJv8fraLfzUPFScgVvaQZBR7S5weOxhFkfoebrHdYp07rDHWSycDT4CGFj6y1Ks6g42PbcNk7KCR0UU45V3GLfpKAm3HfC84sIngwcQk/sVocpqcgMK0FlNdL8Rj7RNTYZ2JIMN3cXrqVvJP+qxVDg2sH79Eiq0HTAySUTJBixAkb+JsWeWkBqbxZ1FvxPlMhksRtEryD6ML+qhwAzxqcsQkOIyyw0q7olyg/Yh7En3JSDXm/ffeA+1zYXveTwyy/25cXkIA9pq6R+bQHXd13h49HvQ7l3sSfztWI4HJpJWNZ+6dk8kghlDywDkFgNagxfa8Hy87AH/eRAlyl394F6BXAq/pjRArZS2Zg0aOzHB6GWXTfVVBbq6IswOZtRKC671h8F9JIxNpenydearpnO1xpu8Va8QMDYJ18FpJPnA/EoZjTKBw+NFyaEzwCDTTF72mwV+s8STbi2nnxqSk4NwvRtJzwlXH+uz1lTMtpxEjjWpuF7bE8e4v3Ao3I478N4smGA9RaAjOLYG4qO0AZXgO5OSpkQ8DpYglTtwqWc1eVITCAoRAFB5BhRvkBGegUniwLnn3udaXzmvH1rywLPvu4IEeiX54SEz0SC3oLfeF5Oq1Rdh9E2ynH6gILEcL+c7+NQHEWi14Zj8CmR+ARPzaUVDsRmOT9vH135ww9jnsX5199xBef82uORGgmYUvj1TgDi0kg4mnxyByl7L3qHHmKGzYVJ4IG8tBEsnI4Iz0P6YxG/3R1CoKcbVahBllnN/BPeRXJEup9OvhGtBUvZObmC1/c8gUULBVqzOfTAO7aSq3v/BeUgEAc6KPm7OA6+wbPU+ftz7Lv98Nw3ppGG8viYObq0Vk6vhq+mrssO9KY771c14ef0FxnGi5ySgrzrJbL8aLrfpMPn6cdOlJ8Mm/gJyO4p3rWH6zkGkDS7Beu5Veoy4+dj1cBpxFfs+1ynLjGLF3niCxkWCgxPI7dF0ZBPnU0upqYEpjntYYA/V5hZQ6KE+mUhpAt+5wB81KQhVa7lV4g3N0SLQw2JkgtKFganTKe9oYol3Fd2c74psWoAhJ7Aq9eTleTw4l/7OL8GD9NFDnw+rtJ17fs9ytXEVFFUg6Sgio6YfHWYdisRIcAdf7VlwTIT+u+jIcGF8yEbKlN3/x74AbFoImnIGWeEgXg77hfMtzwFfiOxRUzMNsSUc7fEP0vu+2DXbsUzSW0Tld3kkyovzCL12Gd39adyLvifue4dKfDeHHEFlK0fZWkbPM30wqALoM383PFLss6WoOf3zi1TP28P2UWmosw5ASQAaVStCbDo5TnaYmkaw9EAxB598eK/u5+5FkVyGXJuLY7sSW5MfSOwevNNXWyIobTTjFvIkzgYF1vq/wPdtAFqy71DwhwxJw30CFlvxd+8Qk9hBT4LGG3ngE9wsOYrBmk735O7cTsgCrY94TWqvg6EHT97RkrA9lpcqZ9A58QSCsf6BCkW9TcP92814FrXw7TPf0qbt5NO4y9B8n9oNyQTtH8jd+Zc46m2kzebN7HHfwS5HOnxncTpdimdpNQVGOb80gsHRHe7/AjefIUj3KvsHXqbT5kyrbQvX6jsf4adBvf2r7A8GS8s7HP3Dh9InC5kq1UB7OdxYwU6PsVx+6x0GnRuE39cLafnZDYaceyBXJamWk9OkxMGjjHpVOy21eaxn0IP9XxLCqXDeS3TnfPZ1/5qGjgzYfp6BNgP3/5mHl0MnU8deobjYxt2pZro1LYCbzxAqWcHsXrmcr3Nniz6EzyMPiQoo15aB0oXvrvXj8m/dYNwZopUXCFdFsrxkn3hQx1jeys6mV2YAI53PM9C1hZp6b3EMBGjNZ1CSN4Y6O2Ye340pNASYy6NR6Kji+Pi9rP3yDc4d8yRkoe9j7UsdHKgML8CobgRAIhXn7rtqw7h6wglnRwWnfOTYpTnS/HqqOFYZf6WmxgNpUwBoTDQbDVQwBPcAX7B2Irj0Iy3yJ8o8inCJy+JpVyNSMyLgLX8LZtkIPq2D8opYtu58DvsQDfMH26hv96CyJZBJdmf5MQRGa4qo1xeJRS2bSZxXBS6FhK9Z7tjJvrMxKPfPo3PisMf6JJFK2LR8M1KLlnhhGQ2KaIL87xCc8z0jT44g9kosqa/YQNE1HugjwS4EVZ3IVrBhw+TlianFBDYbE7ZZAYF2v595dcOr5Ac1oxjlQWzPE+KaNex5cErEX/INrk7H2egawZ24O1glVib4vAqnviBAM5U1tgbuHZxI7kUNma4J5EyQsLJoLGTfgTHJOHaGMPWXJyiLb+ZGVA8C3eMedkqQkMRKikc2kxK+nK/1hay0RQEglchE6WWTlbuJTTzjc57WZtBai6GzhpN+X/OKFi6l92fthrUED78jzue6xsLh8jyUvy6jOMiEfUQTEkvjA4yq1Kak5MRQOi/FMbTTiLo1BL/ZXeAcx+4sjlvM3trFWEwp9Dq+D/PQq2IOQCsW76R1Nxg39De2W5aQ/9KLfDb0KqumPZzAv3r5VX6deoyp2+aRcGcXPT6sFwtumV8yLfEWpU7lnKkLR/L3ZPKGP662sOByHLp8NXrZECoCHJnmOvSx9lf/GIHHzVjSBw1BKgSBIOXn218wOvhH8kx+/LVYBKBZM11x0XmKvtQdlaB04oI0kTn5x0FbwxDnGs7Y5cCRC+DcG+I3PviO5ITkEJYVhnOO82PH/hcAEVbrzEdSPa3Od8F/AYH6oYxI3Q3AfcMUyizQiUZcq9uF0uq6kBJDLipDO1/f6U+gvGtfggxsZpKNSq73LqXVuYXfJ54itXOAeCvSXofy4+SlLiN6a38C529n1ujT1CoHQfhqcBsEYasp3fw0EXtjORwg44rfk7w1VQIl+8S8j9weiSAw6Nwgmu2aWT1hF3NNHz4Cj4GrrW4I9TpquqWzZupl/NsqwD6CVfIMtF+8QF2vFpJ73uXt6CJcWxvhRgE036ch1MA7b76DqxDFcU9oNN4F16/E99lzLIpttThUVKO+aU9A4UBaIrRQtAOUzhQZF/L5xru4N8mJfup75DawIcHWVYyRClJ6XetF4k3R833J4Ay+qkhAaqqEMyORSFq51TKZXmFgzPqdWPtmikyu+E5MAqsFVY2MKYenoEgQKJ9wjz6RV8EKqDxYqF9JcX4QBVW92fLEXuwVZTTKg9EHTQS3ISiMLdDehvSikSXbVlG4fD9+4SKz9Af1JM5kX8exGm5/vAsn55GMnHdCBAwpDNDrZ95b4884iYXCnsG4JlbSTXMfSUs29ZrxbMq4QVZcFq6BrnRWtGCWCCCoAQkCAp2qTso8y6lXSsk0ZzJFpkN2YyHYn4Xm+zwVnsOlgn6oO2180+hHcJBYPFZLOwi0ytGbJdhkAnfkbcyvd6XOW3RW18jamf/KVt6sc2dTF8ij9pFnwGi1o9Z/IC5euV3P+uOox8o2LxquRRFSEcTJoQexSWwPckKCAHu7ZXHS5z6HGu1YGliHa9UJCJ8vbmzpILxJj7wwAIX8OQpdGyHqVfzi+2C6PI/v7bPZeL0XrXp3gov/winBiCZjFcQ8x42APqzbsIo6FwcM1Q3UhxyCHV38OUMiBpevyepjpFv5dWrOC6zq+Tm/fCz6pgtyFdp7WpTtMt74ag1FCbe5/c1T9OgWAWoPuh2/zKvmqdwqlXDvt2g8J3SBB2+tBYfoB+97ekQGNU61+OsmQsE2kQxgSIRhZwiRyfHITmTFL8MYsPDSY9esqa2Q9dN38sWVIDw3LeD8hNuwbjfNN9cQaymmQZeEDb8uzLTwYHz2klnwH3mTw7XdKN73NYvb21j160SIcoTo17Fs3MW3a3rj0cOPvPBOqrVjmTDuEzgcKUoD63Q0eEQgFGtJTYklc3Qbi24+B2WHIXAJfxzpjeXQIJpX/ojCpxyZrV1kLCocxfG0dgj638IZkR2GVdtO9Jwzj/XruvVbqtoPYyWbGpMKq6AF4JnKabh/4IeXD9im23jwcCB+mjJrvIg+FI3FYOHcoHOU+5dTkbcVr+vhQD23xoSi0MmwtXbgUe6BqkNFhXsFVqc6CpoVqN5aQzefNjICb2Ex2XN0zFEkujYW6CCnJZsqOzey3XOQ9lhKSAj/jf+l+G+x77/xvxOWNiTZX2GnHw+qlZC9EYNrPmMHZnHPr4ryIl9yZS24qmPEQT73B3zkb/Pe78uoFiw0GaWktqkYBeIA79KXIXdS+M6g47UxPzNrkZUbQgnuHj/x/FsiwvLy3xqubnwOW2wH2/Wr+XC8G5R9BJEvgf9c2gsOs7sF7H2kNGceZ1CbAM4P4dwLlOdISptAavdU/pn0DwWyBH7r8Rp2recJSSyioKU3RkkjJnktJluniKRy7M7VjgX8WBBFRsdRRp8dxU8WHZ+NcIY7L0Lyq/ze1pPsuNM8fXE86owE6qvPiQy3+z9D9RVeLtKgMMpJLNiPyuyFjz0gKYbev4NLP0oUGtp8rrEgqgRfO/P/uNT++TLG//I6h8cfRJpwi+/6jnggNYZESkVnFPVGMDZoeeXUeZavC8YVGEMv3BwCSaqbA4OvMjwqH5P7FORNmaD2ZlqpH/XSDBKiC/GPvYbGO+LhQQUBScU6dBYjS3s8x7MOcLf+EgFdBYvGC8ewbp6N3lDHnOeeR4IKuCguNG0WjliG4/yDBfe4O1yfWIrJJhcTLP8mToGP5dG49cxndWYcE9zuPtbn9NrrVOnXI+3wxOr5BwPD9sDZX3Crvc57+dHc3jeE5uURHJt8naHB3qIeeOpbIATy3fFYKlr02CJEBo9EIoM+ItvO68YbGN/axKVqLeH1zjSoAK9xMEWUMjqcv5v0vd3wb9Oyxiqh/gUFj+SOqCwdz+2/BfyHS1iz5HuqpashYJFYwFS50oKWCwf7MbEglPVDk/HxKYIZDQ+2T/TfRtRqgXtfL+bt994g7svTAIyw/Mn4t67wXmYTufzEu4FpuLaJbCxyv+fbTD3lfa9glbjz8qcvo55wVZSwDXkGNL4E3Z2P4ngEWxecpmJ0MsslXzIpYCE49USQKtk4wsKtOgdeb8ukTtPEQK+lGJovQ0seNic9244kEnM3hsLYcbj45YnMEZsJ4sRFXi9zBw29tiLYtmETLFyXrWU0Z6CthHSHUWydHUo/iZwKXTSf9brDuGevgcyOmwYTV/pdwSAZjWzFXyzyaBElHQCUznzX/xw5vc4weP9cvG69TuhXl8F7JRRuR2cu4Yl/ptNhlfDlM9+Kj6XVDDsNYG5FYfmIqHtRODTZo23R4iOISX9urwO3IaQQxDHnvQxNCaTX9V54zbgBdTlwMBQm5VOuiuaPYa+htviREnkSHxksr9gDWgl4T2KcLo3msVncJoAM/0Ns1Oio6fUzp/KKGeP+Dhq7X3mjczktDTo0dm1I+v4F1k4ubd/OrTffopMB9P36E+QOTVQ2vYSrtB7MzeQpHLEfd5Z8pw5yezRyoTUdfKdD+qeQ/DKTvMopDnQkOsWbwHx/qhLqwCRK+uE9gbF9FlKt13AjfzQJEaexL/9WLNr2+AokUpxUHbT6l6LIE3j3SBx9x6WjjF0BrgORGCHqXijDT/Wl6akBlJUOJdgBMB5lZVQJy9Ztpuryi3inO2DXLmXPmCm0tg2AyTe5syuCnKBMemTP5sVtbzL16X/gyjzxVlYd5O/09VS3+ZE1ZSNa/U1C+RpKj8CFibg6LuG7BrhtbmR0rzTkARVg7Hov5Hoy2+vICUwhxCilyb4BjSCH2I/EIj2PJ56PtkK1GbbYBT/4P6/OHFbq4WqnBJ9iH9Kjsh4UN/+od6VKKEHa5kFlahP2AYMfbOdp58XEff3RNpRxfdYsjqTN4otJPPCq1B1YxvdXRrBNsPGKpxOGFkfRCUWqBImCOOeJnO95kcIAE9GFEKuN5snyE3BuNNhHoHbuTanzDXZVrCLn+Ees6uklglM6a6E5F43Znz7ObZTeCefSxlnIVkc9Pg7ahrO14C5xe8YyqswT9bRp0G0GDZkePFs6G4VzAH9Ofo4aVRLfP4LlNgpO3K93Ao9b7HMrotbkyAjnXmJBGRs9LQNRVpeSZbNQEHCFCLUAWRvEIpEukD3nYgjLCmXsfWfU9kZ8NG4QsU5c9AC7Jp+hc8bf4sFs/I9Q1UvocUFMAOc1OuK+5vHv2oJ5JyibcIbA916izqs7f38ahtxUxj93N5LyhYrWwS6s6gfD1VLCz40VJWAECYrMaBoznYhryEJ2vZXLQz5m6/kFKG3VNHU6M1z9BqXVLviu38USpw4qXObgXrENinZR3lJP5u5uqP2LSQ+t4IgUhurC2dMCbjKIU2Sy0B3WlEup6ZSBXABLp8jcdogBp35EnfVhyt5xdNYb+CeqmEOP9MksN3Bqf3+o15MJFC87wwvLv4Lczbx9Tiwidpy+RMKR0/TqtQOsYiJQqu9HdHokHVorwfdmY5aZkCwToOo8OPWkyWrPubk7cS6YTc9iGaN8dkOZm1i40gVSHtzBu2+8i02wYZPYWK78hnnmFFFO0NzKlyGraQnu5L0vp3HL3oMvpyjw7qgUF6IyLfusY8j/XUaYvhEv602mOo5m0iP9ivQ4zvDFF3nOOJ3V8feptYrv4lOaA1y4O5YSg5SsRdm4uFqRG4GLonfomtpp3Oncg1ZiIQgpOokJ2stEcEze77yeHkru/llcfPY3xns1YKQY5BHg2IN2iZqvx8xlVzPo/hiHo0MLwtNdqEr7CGxOfZi1u5km2VGqZq3F5H6E162XIWeTyO4OX80Pl07QtP0IhpbzhPRLwdoSgcQglj0sMh0O006QVAuXOkBu1fGy9HlozsHm3cTuabuxN4Shaq7iig6efuR6jKpUYpY1Mi+lDx7JjgjjeoCsGswtqIwlHHWV8X2dhKwX1/GSRyUvTVSKY3Psxzj88gw5z77NEPsmVE32KEOPg7MoTUrVeXrbBnDwn2icgANAeHwWMkcPEUEs0yA1CYw6IQIoJrxzDv1/yONOt5MRmTGT81f8cWg24/f8Hbi0EZsqiBcPxuJcvwk77018PrIP1ANlP0LVJUpqXOn8y4Qt/BZ/RUSiVnc+tt+Tnd15rvIY3fJKefPTVyifWsqgiedEAIpDNJs8/iRKBZ6WIloEkeXyaHSTf0tBhT1OFd0Ik4kJOGwWkb0KXDA/RfkbStqdGuizchcS+j22/RbBH7ewLNq0zZxph0EuoejopEwtI3PhTmzm/ii3zKXIr5CSph1424sI5J8j12Dxa2Fuqg9et4sJ7KOE8ni4OAUSvuOI7gs6Agrp7drCch8L52yroKUAapJoNvmwz7kEmV0d12r+QdsO/TRAL5H5KFggL34Gl102cKX/K5zo7CrC5nwnyiGHrEDbqMa90h3kJpZFVkBn9YNiX52phEOjvkTXoqPGpQaJTQE530PO91iUY5E5ypG0a3jpbBijwhyZnf8HmBqw2mxsnbcZiVVC4p15fGcLZ92aRLgv+ttGKYwcn3uNDXfe40xmCDY5/49R5iXlt8U/E+IzWpSi8hwHrgMgXUzAOtY74lHuIRbZ5XZdyhYgVJWyZOsCNE12mN7ZgLNy0GP7tdot5OB1b0Zd9GPDYX+eeO80WDtR0crp8YdoJojE/EwMFUYESYyYqAa6aTx4Ugpv1g3lPa83+NoE4516iz5CFyZSpZtOcmwyqkYdPvtH0znMDnymi17GbkM4N9SbU8NO8NJPT3H2npK45zQws0VUWFE40etaIt2TY4gdkoLGP53/LPb9O8coDDLh1BL2qNAIAPe5TbO2jDfefwtjkAL3l0X50i+rezP6UAABQe2sv7+ea72T4Q0p6PzBMJ1u394h8fQYskYfZp7mM5RVdyA0ChJ+BxtEBZfRILRxpgPuVcNYRzfU5kqouYLVYxy3N80irMEDbUUxTZaByNvuMin8K/5KfQej/X3efXowdvG3mDrmKBOdwsV5lcIBNKIWeq9bvnS/FkZgkxGF5PGUkQAEyqFF2sqMjm10a5zA2JA13CrNwcOzhpKARuyrNMyt74EysRnaiiHtA3zsZ/NFRSh3mgxM7zGFsR4nyDCWsGKFF6dOgXtcDQ2nimn3ruXJud/jLLNAQ4joY6jx5qO78QR9+xzWESfInrmf2UqQS0Sga61mOt9nduB7sQdnhpwhNSaViMoZKDv/BbW586JnBo5zL3CjaCWjOk+idikAIh/066DjRIr7nn7wb4kggaYsrnoeZ/uSF2gPfJG/ckYwTnuFTQ5/87SPBe68SG5TG9nGJhQyN1x7FxIS1viY4kxPQwFu86vYfmI4W16bTeQPDz03LDY5BzPWYVd3m5R+d6nT3yRMM4fBeinoAhFK9vLmwF/YlLqKBa//iqNTkziudxX7cB3ANefPyAzT0abYQIFjDc89Iokwtm4vkd75/BZ4gj4OHoS7nIKYavCZgtPR7vyuqqWi2I2/5v6FwhD02H2W5XvS81Q8LTOKmNh9L0qpL/BwvX8xIA9XbT0Rw7YwXG+ioy2J/Vlr2Z+1Fuch26jwqCAgL4BFfy7i+rhCEYR0XhQklLa8wYrvV5Acl4ylXwFQAQ2p4s99OIFqP7ZfmUbmCZGx5GHf/Ni5uVtKePXIJOQ3Y0m1SujzZQ1UH0ZVtJvE5FpqFCFoG90YkbMCzbvNoipS1Gu03e8ku5setcoB55OxpAs2YiQCzBEtVor2HOLI6J/Qtyaw6kAV8fEwsea6CCDwm02rp5EzQ84Q6FnDLDvYKjSA5xjxB8hlGowKMx63DZRUXqOmfQLefbu8xYAYlRr7pP5UuFdTHOyGm0/gY/3aY3Dg3povGaEBR2e4aXiPhJ5jqd41jsZ+12jxG8KpoEvcU9bwk9MRvHPE+XtvzRuoKydharYjZ380DgNyQB8OvX8BYJx0Cuk346ivsiOqaCg3w9qg52awdKA7rSc0byiODflUv/gyH3qV4tvvPsbIWq53QGvtdeYOvssl/wr8+qTQorKDo2A0DEPl6I2QD4LFjLnVgqJOwrB8R54LmsDbBwKh52Zc2434pMZS7tPOLYOKyWWwzxNIfZ0dGUHY7xmOZ/cWbFipsECK83oGln9Ga/FBFhLI7fN9iOgxikxTG60mA0y+DCo3Mo+9ypDdXihMcuRSd7LqlRTUvMvyol0ieKbuNv39q9n89Pe41TbyW7e/GakFS4YZj6bdvBkkoJx8hWH9B3Ljp4GUBuTjneEFVX7Iu3+PQeKFtcmeRPkTdKbMoURdiXl+ETJ9NFSdZ0TfraQwFv21MupuLcHrW/Ebf6ViEPW7F+PZoqdpWQeu9a5IXB5OurLae+OWo8Nb487wA0OJDUzCv99DgoNOVg2hVkxeBn44P4bIbvkP2oSuYlRERgTRadEYhx9ntN6MrL0UVHoEAT7PmUyx2cr2BX/RrIHXXJ/mgcNF5Tl+b3Vm/54pQCYf3PmOZ3cOBB0IlnYcXBqofOdzZD/+gk/6CVwGO6Iu+hY0Ck6FtLBx+FnqzU+yqf864mX24rfUdSC4D2N50i6yq6I441HPxH5/0t3TCVj54MQTr/ZmyLkBCB5VPB1bglnqJL7vd14glkTcCsqQlwzlYv+L1OjkYg4tawPIHQhoHMf6T18itVs9nd3i6GOIAdeu75JzL5DreP73PBwLk4lKvIjaQ/HYe2WWOZJQBB32FcQt2IHSVQO+07jQoeLLJ77kyVu9kDi3sarajyJZ/QOGp87hZT6sWUt8ogLXLzZjNVqwV7lA5QnoqKZcNZU6jyOYnO2wjhqDt1wFdi0wPgvkenpci+fZyDS653jQPL6EsdpXRCBq2WFwHcRpjROOPZoY7trCN35wrT0d1A0i4Nk+hF6amZhzj5MRUcDFiTsY1vIGQx7p1ynbRQ4uPkdC+nCeO7uf8b0SAMgzu3K/dxIh7gOZun0IfvTHf90F8uiPzQYdPh38svQX3pj1Bsdi3sImyPjt7k5srjUIVc44laQA8Shjw/jkFdHGwrnaGV2zHf0kCv6yb8aodOJG/EluxIvnEiSDP93hw9pCnm8twlsVRSyPexj/N/7Pxn+Lff+N/52QqrFNLEBtsiCRyqHv3zS12VjWdx7fR+TxY3kB7fISZkneEAd5wFdyl3aZkmanOt4Z8zt6c6govTH6OgClB5by5bsvYBpzCtvMSzR0FovyMpkbIP5rDqpD6ex/kQGXBtCp2Y4gPA/33hcLa/5zkQgCQ84MwSo3IFErkP2HdFi63Af9N+9wtFpk+iUbneD2GryAJ56AT2+eYNi+YNyLX6THX6dEFGLECxT/vpM7t+S02NXx9XNfI5d58lngc6Kxq8LA6NY2xuTEcy00jaEDLpBrmCt+BFwHgUzL0DWtJN5MoDwmCYVaD7YhcOt5Ua5qQjYdEd2569rKuDJwEPwp+I9LPdYjC4d+TXBoAvE3EhH6PtIxm40c5WRsSgUuKb1Id7ejTogkEKhtCubW1lAYWkvxxOPka9R4+P+APEksFQ6v2E99YR3EHGFB3BX2dDg9dtxhhdMxVbdR3HSaHa4NhHdXPPQqFMC1Spz0rq0BvdWTHU4JELhMTDIqFWybvY2Y1Bjeevct8l7M5j8jwLoJo+YGteeOsMuhiRcefbxaU1lyMZiKNjnTP1YSrTZCOQhSLVdDMzkyuYHp7qtxLhtKufMrYOwF997Fzekpdh/ri9as4gl5J5FKd2SWRrFwW3aEQKRMMcC9PAcOvfsk1wanM2wNcDga9JFozK54lNthkVpoV7fRjhLuvgN334bev5FheIXMvlW42pfx/Q0npvcPhojfH5z3HssY3A8008Ok4ES2PwO/dMSjvVws1Khc+bniINeTO7BX1OMTuYXeOlF2rsJ+JWcuTsYiB3u5iXFawGYU0bDDz3Pn9iKujvyT0KrRLLTUYR9YIi4S8/+E5mykfjW4D2ijxpBHi10LSASRUVh3G8EulG76+zQ1dec3dV8G+m5DGv0V5DwHeb+jCpSTGS2h1C+HRKdqnh4wC8yA0umBj8u/0pLLrvVlhqMZSaJVlC1Vu9PRfJSMoJNoOrzxLrLjaq2aF0vFImFSeyBQjVluY/2AHHFXjZni4kWmZm+zFJvESohzC94RuZRKfETk5NQKGq/+wvSJm9jeLOXV3bPxdKtHsrwVzK0A6E1NzNg9g7s9bvPB6q/wNA9nYclBkVXlOhgBUdbiVv/L7Jt1Ha19m8hUbMqE3J+RCAayw7Kx7xQ/ow4SUJX+BaV/wVwbiZoiIucU8fUdO6rtq9FZlRC0jIqq45gBiUTAeO4qX5x6gac//RYXBNB40yJ15LikjXBBjvPwSwwydCLY2uG4iKa8JH2BU4mfE9m8hJQsNTZN16Jb4w1KJ/4MzKQ1ZAttGSH0TsjAQZ0IvX998O45hVfhFNrBuZvPER7XxepwEifJAJ+63SJQkLNqv4GoG5N5L8rC+xNE+UhBgPCwQvKl7ejtR/JW8FjCzZlg+ph1wkBOlCoIs6p45cKnnP/hcwC0bRfhzsvs7pvLz81m6nPEJLS9Y7Mo1eO/gKZ2R5LWncRQepfQtzr5xQ1OmdPhzg9gs1Cn6cHdd1fgrW7nxcV/ILPqmKfzh0kFINdjOreBF574nJdrIbUT2oKBptAHBTuMjXxoNVB8NhG3i33YP3E/rHN98N71tt2n8vOnUZmcaPjkPS46A9WXwaUf9658SNjePGRDHbk1/k0mjntknGm4xsjYP0nN7IvlxFYcTQKCMO9Be408gMzTbqg8yymN+IIawQtY80CSJLi6kqKIOJrrMtHuUNExQQutXQs6uxAWOaYyUw9P/6ZHUp+O8HaC2GcAmw3JrcP0WrWdHV/MQd2sxiI8/tEKcVhD2ldD0dnuI41tJlJtBy7R6O8dJ+7cCapbAmGRjfPeMLBsijgeBCygqb2I0Gu7afJz5lDLAeZGnIejPUT2htXIUwMS8Rx2mbv3prPLHW5ZikA/FGRaiPkA94rfiE6LxndMBX3CzyLhJbAaoeYKuPTHX27BgowfS3rTpv+f0h4RIbmEvX2OTaUaMpStHNe89ni7fQdqiZQ73lkUBVxBZhal5oa4vUmhzwraOsy8+c6bZE+8SPjMs9AoCj/dvnkK6ZVL5E+sJG7hOc7reuCiKcJeWsTNCi9mr/6BI/VufNhsolSA19nWNeakUaUYzh8Ln6NV20qVWxUO5lAaFK44SeAVvZnrTXLatsxgfo436no9bS9miSy0lFfBexKCc3+i0qIw1BvY9PQmLE4Oj/XJInfm+1l/41LtQmTuTBQOZyBgESb7BO7/EIprbRJNkQ3U+EvJVwbgH9AT/OeivZXExxs28WqlGz4XQ4nTmJFa2kQpQrUXCTW7+SIQ5hjLCFoURmREvuhtVyMiZv2U41EqrVzreORkgpaLTJ/s79iXFEvL1kF0a9WSG5QrFlqaskDpCioXJIJA4o1EVJ0qbIINYcTjz+DzFd24115I7fBjXDKG87J+KXPCwthyKwzF699ys3k2C8r/Ili1hebuPbCTlIOphdb7CtRFXng12jN61xsETroE+i6POqUzGe6NKAYnEdCpYkJxEOfDakRJX7kdVvsI1ldIyewU6J4WgZdHLYJgfSDhLHSAY9k95Ip66idmYrCFQNY34jMaIfpdYbUgsXRSFpZNYXQyo5Ru/Otw12zozZDHrYMg7SPI/objEeCuhE6bJ1e8TmBo6fvYn2lbNSg6FSgjZpGtdGWxhz/4S8F/LvU52fzwZCry7CDuxqTQqG9EkMjgYIiYzBV6URd5nULvEn4fdJ9bSjXI7SH6DbCPpLS6P01D0kiVHafA9SC5kjVsmfjBg2N7ZPzEq9Ou8mNqEdPyywnXjeXoI+emsDaSddQP+yYNLkHF9NAUis8mEryLtXiXOlAQtZl/7o1g3ITzKFvyoWALxda3KJRVUEI7ir/HonSrRfLlwyS3rBkW/rGQZrtmCjxLqVKqxOfHaxJo/TmaEcrZXCXPxafySXAT2y33xflx7mYY8A99Rm+k+ygzRXmehOg7xMJKt3ehXbwJh+s20jzwNpVulXwtK+VLzRweveoNPp2cnynKjy2/4cmZJ5YQlvsqIQpY4wCfV1rpdi8MP1U78vo70FXs+yagkvHaDqaYzrAqMpVsdy8QEsU5RObnvJPnTMU/U/l52c9ojCV8Hj+MMUdiwNyMJmQzh0YfR2rWMPa2pUva6+G7UVR2gbjE9zA1erCnbCISpwSoTxXHUE/xI9MzJovi905x0mLkqNJKkuYhW8ydVK71K+ONKimj2nWs9OgUr5dTT/ao9lAQp0Lf+j1z9syh25PHoHgPAGZDPJu8NnOkFSLO9QJjJ9HvJoLfS1B9kVbzXPZfsVBe2p9uF7/FFBQIjwBBOttOEld7kg5ZLulxhQRpzSIAq8s/N3T3XMLLA5BPPMNM/30UmLyh+B/x2xH6DHJrM/P6Z5BUp+IJJ8izPj6376MeheWHFKCFCp0Dyc5/MMA3iR3JdRwLWY2nxcCgU5dxbS/AP28d+M8HQwLthQt4a64Mm1rJR5qn8Rp9CV68ChlfQs1ldMUa7k44h++VbvS+1JuMXo3iPFIvcuEHqSzUAd/N/guJVcJvsvnivZCJCHlTr2RSQ+5D7wxarB6M/I/Xf2xrGkuOTyW/1x68Yj6lrO48QUEPPdXGXYvC5d4YNG6djIg9jco8G3BEobDn98V7CFYMQm/I4tmYPKi7JcrPOcTxXN87SL2KyCoM5+axQVT6wtJ5s6FwJ4LnWP7ybCfZCM+cDsMsM3N2/HbGulZDexmNlWp0LXmY5BYy+y7BJcgeq6yJi4UzKWqKZGTUy6Q47cUtH5b+4kHzEgkU/C1Kj3WpKviljCEkzYMRa08QGv078JD15Wvvw9lOHw6fHkxqrpaSKfWQ/yfxbafwGw8xvu7MSonG4eREAmLOiP5DroNps3igPTqEoBY1F5OaSBGW0ntXFeMiTjHOt4gDWXlMfHMrc4odmFNpwWJy5sxr2ZD6JsgdKLLLoHTABUq9SinoqnX9kDOV/qs3UZcEh9220PDKR5hlZiwyC1QK3A+5Roh/C6jdKOuIQd8tj/ZLjRRn6Rky8M5j9/Kwz02OljjT9NmT6IKK0byYBdLRVNkGUN0SiN0jbCdBECBgPgTMZ9HrsxnxQTjFcS0c8f6E+yp4xOmV7+oHkBm+CYdOJeHZA5DYbgCjH7TXdRtEgd8gWhz2YlKFE+w1CnK2i6o3oc+S9bcjvrm3CPyxy+vXEP9g23KLhC/a91LhW0p1txPIzPrH+uTR0Ub2T7MwuRfyzOTtxGat4TqAPpre98dxzXoYf5WJP4fmcLPV9Ni29XG57Am+yWLtTAb47eSO9dnH2nPCc8mIamWqVsF4nY16SzuD/P6ips0HjSyPX/UK9jm3EDTpLKog4wOpbFz6EW5SUW7SMvr4aKRnLFz7MZheI/RwdREgwV6upbXU8OBYn/bIfezY/tZcaq93x2jXQZn/dDyVMRDsCemfYrAU0G52wB4zwWgIkZ6F1lmQ/jG2dj8OTXyZH+vl/M4aAJbI1A/262CrYp0D3GgxYihJRfBxg4ZkcY4R8BOLQ54nPDyd5Tt7sXD3GPzWdxdzMrU3QOPN5306ce91iEGH3TDJu7y/OirEXELfLQxTVmH/0Xd88vt3rD5ZxuCe/5Fw6oqLDV5MyJvLHwsnQ+qrHLf6sWHEUaZoRjAp6x41dRY8pjmC4yegcmfC/jSeVt3mm/zlVKZ2ssdLzrxH9lcSmYyjQx7nDBK2u5ZxRhoFiuVw81n8beNIn+SIV2RfDv6+gTpDHZESsTjVUwVNkgZUcUXk+BbxSh04Vgaz1CkJRfN1iHke4Rw8YZrE7Rk9iXZzpkfjcvikVlxf2oWwonsjsb++xoH20zR810y+byPFy5rxqThJXmAI15b/yIehP5H4YwwFvgpkmKExHUHpSXtYLvdadAwb+zLxQSvx1aeLijYdlTjY2jg44DKG+rHkDjtApCKXIfI/REWP1kI4nshfrqC1QcptFWq1P8QWYHUeiLR0NwqJjUQlGLONzNgxiUPjDqHveRPK0vAd7MuXtssUfPk7sIHeMiXJo15G4hMnfs59pzHnUhaVNSZ8uhdw1+EKFyqe4jSgNHlS4+lGh1WNPrecpw8+jf2Kf0TQgnMvfIU+zD52iwF9LaTs3EVlpAsrVns+uFfxPquZ1ftvJt4chfPZPpjkNgZ0tQkCDPc6yRN9jvPS3SwGv/0qcWOuIp1dBUQiCHAraSsBKfsZEJLI7dgm9B6PAGu0/pz3KiNt0mFCTk5mSeLLhGbXgOZlOnp8heHHIZhoYXL4QKr8+3C32IE5g2fgaDyHs9pI6YjzNN9dg0VmIcUsg54/iKozQU+QaDtBS9YEbHH29A4vxUlaClkbRWlqmxVH3wpy429TM/Ic7e5NTJTpoexXKP6HdtdFHNttodVRyumnTuPcKeY9sI+ApgxGa9Np8JeRr3blwJBnyDI/Ac4/PZjbA9jV5OFWcJ2LXs9ws9LroR8rgFRO0J8zkUtsbH1tF382iExhiSChVdtKvaEF72QB9bYlBCx9yAq0CTKMFhlWG7SNGoG1vQrB0gY3noGWXJT2HeTFz0AflMkMHUzSdcB2GcR+AAonKjI1bP1tDWGjT9KUWIBe2g7+c8QfEGwPFZ5g3FPBW8oOhn7nBcWbRUBc3Ef0c6wlb409vxjbKVO3YbKpH+0V9abrSJz3EZUehktLAZ7GDOB5VIKFpCEXcHToh/JuO1HqTsJUp/gHcU39t0s27RN1tCQOQ3lxBMgdqNFM5nDiAcYfHo//3UNUJHsB7g+O1a5u5/io4ziGllMQd5Xi/A8fO5dyM0wog9xOeOvdtyj0b0NjOYDC5AvE/T+Mcv//HxKJhLCwMCT/6Unyfyj+W+z7b/zvhCABrS8KqxVuPQO5P2IceBtpDghWKa4SC95ykFo7Rf+iutvcvafh/UXzabQCtofyDdRcB6zk6Dyp7pVEpbyVMef9iA2PY1VjmijFcmUunS4jOT38NBGObiRIdNjac0WZzrIjUJ+Mh/Eeo+8k0N6sxeK1Ef3w8Tww9AEum2PYU7cPp3p7Ppa4ownKg8Cl5LZ5M+1gNoJwGF+nNgRqGNG2DVr1IIgo5Mn7J5MekcHqdTvExL3CUfSSCFnJ4BOLSN09gaYZ+/jQLZ9pgvuD6wOwONqZAqET34wUDIZqJJbuELAYaq9BeyVjAt4kVlFJxUevktT/f3r23XXS4Dx7L9lmE85mT3wUjyBdrJ28et/E9RM9gFZq7S5S2NyNBJx5xxRJ8fhbeIS0c648ge9ibiHN7UocuY8kMaeOtnOFHHcORKqDCU4DmfHIcSfaXuLq9b5wfQLnfIoIfdZR9P4CApxfY9RX3/BWrh7Zj/MpjGoSNyraBR2VOEtn4Nk9izwEZtrZiHfOelg067cDPEczKnALyVoXvl/xPW1ay2PFPrNMT+foc5QUufDP6uc5Puw269bqoPQwH2laIa6SqRIZUlMHNrMcol6HK/NwdBxIaXQMdhoJYReMWBGQr+uEpCUAuLok89z6aZgUVZwfuJt6X4m4MG5MA60/a6L60/nSVd6s66TO/UcG2a+DtC/FkzI10aAqp8yrCXlWJ9dzezPtVXdx8l+0C3r9jEQi8PMTPxOfMhuXtmV4m5wZuNdTfB7mWNhd+ypuea2Uuo9gr/JHtFqYBRQrBnHl3AVcPc8wKWM6W7TFLHh1C5QeAHMLztI2QuTQrG9n7uDzuFniqQCIeBG0Pqxz+4Ra94MktLoypNMZg7VSXNTkbAJDAqu+OIW2voTQ8DQKK90J7/mX6IUV8TKu7Q68HRjM9WYDu13HsvHfmzDtobTquBInnE8PxLHQl7u+5cQulAECSLXIbGbOTz6IsmEITapMZMpyilwX4Wu+R8MxR5afGk7NBAsb7WGVA6L8S1eCxq3SFUOdnrRICSeiU3laM01kF9Uk0W54k/dD7pDVqWTJmT44KUWPEcamgsKRbjtXEfVcNtesDdRYQG9ViGwVQwIY4hljPEWvt98mq+cN7iw/jNwSTZ8+v0HGp2AXhLsxiZ9d4e8sAWVaAnMS8rBRj6AUi967bS9z8NYZ9O3BJGrPEKlrh9JDuBgLePLgJMyNbtT1b0Xeo5FU9QiGFe+G22sJs63AtdIVlVrGhrgb/CG1sEVmJ15IyUOYvUuFC5f+8sM8ZDjTll8XJZICl6Dbt4bSvHZcIp5gZeg1tpsCRA/Qkn3gNYHh1Xk0K3JY3zuKXs5wJeBP+pbsEBMpU8q4WNabE2/H4+4rp3ycQILDRFELH3GB8XXvbM7GZnH+j/u0+7ag88yHy7N4y/srWj5IoMFRgs0Gl8sX4aW4SG/vA5DxKW4yCVoB7ALH02nfQYFvN/wiuyTmjDZqnGuwWbSU5bswNy+MIdO7g4cKVK7UaXtS5XqKTqXIGBEQxGvRhWpW2VqJUIBOgFcdwWIDaegqUVpSqgBzC4Nywzh5sQ+dik5atY/LHesi3qNCd5ZWsz0VZjjXJmGs3AEAD+lIWpxOg53AlB7jsFnjAdFLwixR4jGqnlvBznDqBq7yZkrq/Am5d1ZM/hs+4+unfsahQc9Ln72Eafj1x47rYd3P5yOf4vlfnsCQEkfusDYIfgpCVgDQtms7H23PxNBkpcrzBoVNfQigSwtDEPBzG8bST9KomjiP9/yTqZbNe2z/2CAgZT8mf3feXb2C4pbPgXV0KMKpUwdiVnvil+uCoSUK+qSJEkYAbR/hadIib85h7DZ3AkfnQ1BXQkztwb2WOVyr7Evw9RbSPn8b31eLwGu8yIQAls+fS/ask2y7cpn9Jc/ztaCCupuQ/S0ELuEXQUZzSSQXfx6OZ98O/jN6OQYw3HUv36hqqO3okmB8JIam+iBt72Dt6q387mylcfBZ9PaB3PrzMBWDomlxkGCsP0CDc4PIhDS3QI8NWHr1JU3ZgzT3j7nnf5Nkz5vgKaJw5+8rwC7PApQx8/B+8qTQ8skpdKZ0yPoaF30A+YFiIXa8FuwsrVhsFj6qFwv9EzW/k3t9A9ib6T74NvWODdB8Xzzh8LVoJDLiusWT6V1Bq6Eee6n9Y30SJAKV7pVUulcyddA9XKUOYDtAiSaKJvN6eqRHkjXCjW3Rb3BHMo5b/nPh3Bjc1TP4pcNElk2GcdhlvLSd+Enk4D0NOqup6nDlky1j6DAIDBv/NVpZA9R0jc0hK1lZdZzQOl+m5SsZLHHE06deRJ+HPguGHvzVeYpY52rMc/cQE5uH1PIGdP8MrOK7GCukUfLaPlo+e5KxV0bgOsnxsX411tQyYm9vygOCkNq/Rvx8GwqFlYvmRLa2/UOi0kbYkXxqHZywTekPYaLUpv++OYz++UlyoxuQxBVQ4OAAMjVMrQRBxo9JA6nxOMbkg5PYeqs7bZtcRSBG6UGEyDfI2zyDqJwQ9q89h0EiZZxU8+Ccquoz8Kj4HkuHjI1eUGYqFcdP/wUPEpEFEYsoeC2PkxbRM2ST4mHCUUAgJiWGgPwAEqs86Tv8JmR/A8A9ow/qZk90Olc8pKCUPe7TsDIjAtWxgQhPpLHqySmUmT6H5DTI+Ax57DnO+eTjomrh4MSDQNezr3IDSydmtYKtM3cCEGSCMMsYZljNEL4OpCoabrST4hBJltff1Lnk4i2vhs66B2wwP/denDfEY38yjMXAjVcflz+vNyzlVnQx9z02kdztF6bZPcv2Xt0pK0ol4tlN7KjxIe7UbO5e78fegT3Y4tQLpGocJbV8ufxHPI2DaJaNwyKTPDYfG6UKoqJcRoZrJVumbiZBvkC83l7jQarmdkYIvY+FIrmYwHsS8P7DAcpFr0XkegYUutEkKcVTU4bKjOgb1u2NB/sPkFdTNvEs15oFcW3wH+NFrLKGSLmU0J2zSIlJofGlwTCpiNc3zqf2gA5tkJHNr39EUYiZ9hJ78BNltjKbNSib5GQrTUw03CVRSGCdU6KIYPeagL3fr0hGXiE4N5iIjAiEBEFEl1ecokMbQYISqoQO4o+8j7V7LjS9DofEuYvJfxXzGkpozZawr/E3+k10AlsaSBQPCkxLHUuQOLbzSyEUd0jF578rrCgoMQM5/jj9tIDcRQeJGHkLtP6kqbZR5dpCgLYF1cJ/6BZWAdoAGLQfa2sV09oNVFYpMQ7yprvTPtRtNyBmNvjPpiMFzv6Ui0UoxE+VS7jPAR4t9sna/kR2NZjIBCsnpglcMOZDY7oIrPSZSq/6Wip+XIR0+hH0gddFlmbSUlFdwNKOs+Fp8sK7UXcJ1m2wQ3hCys+P3Cv/tg8o7l7BVqc2Tvffhq7OkwGDFtB0bw+OEpCazZwbeYu//Y6gaykE77UQ9hzmYoFGx0CUmg60HfkoEaDiNPhOg4i1fJs7nWCfImY2DWLt1M/5yyYi37kwGZx68ZNEx/VifxbY38ekMInvXdlRaMyAkBVsim+jw9ZMRLkRubWez3g86pscqLsahLtHBfP1OSRZ2x5rnyaB+00OFE4KwMHJiwFKMcn4nWcLt5pnk6QZjzEylhB9F6tv1HVQe3HJ4Se0rjdJcnTkbPgpotzGsfTWavF7Ovgo72wbgt2VOOY02WPnV4b6+SrwFO+XvvQLpr35C1/X2LHB+W9kEhkWZQXXyybgps1HrlNyP3E2do2rCU9xxEwZuD4B0W+JYw4Q47KMsp45nJVcJ6M+lmWP9EkiyNj2wxI6qiT4utbS3y4PMnZQqIxmRd49kaHTw4YlcAt12ngWttyHqnO0eE7lh2FfgNGVwNQAsJnpJVU/8OByc9vCM4c8qK1R0v2fVlLis+CZTMj5ASLWoXRo5+7IM3glJfDx1z146dn9NBj9AXGOapXa6FR1Mtaq5X1H+Ms1CZPmRehSgLxRPZcDqXPpNusEp1IdyCx57jEpcWOVnvb3lyIHegRU0iDYQOvDGdtRUi+l0MOWzdhj0ThLvfHqlyEyaquvYpVZyQzPxOac+OBcHo12qxPlHXpKos4T1es8ZtlSXn99NO+8AytWGjmS7kydUYLztQI8GqYQ44pI13SI4Zwylt3STBzc6khqlSAAveQP5xCNnY04Xr5Pr+xQXNfI6Klre+z470uGEnonFPfu7RhtiGo5ABIZhtturD29lsvTcwgNhCKp8bHz3hWVS7PNxDPb73LrwEJC3yyj+yPtjaGtnG6HC6ufZ4VLNR9NduWFvqJc4C+NK6h6YQ1RXuUMXv8nx5qDQB/xIDk/48KXuG78gm//eA+78kKkimDR0/oRRZtDoZvxqduNoTyNtPa3eVQg1+y8mOxV/1DW0BddfTwaZTVUnqXEYQjJz5wgqaoBnXMVfjNPkC9/XrQKyfkOvWEOO3YOoqZGT4CTyFixUwJ7vUGmwZ2ZaH5chMoG8sy9KFz04LsYXIeAxguNRI3SpsCx3hGvUi+wCXBrDeT+AEFP0mr1orQpnBdHXGCWHdTYzNCQJuabrCbqNJPYmRuAqdaKqaSZENMX8EgGZaUsH1lGPOV/TaDaJw7vhaVQdogY1XA86mbiogvB4eRZ3PQ6VPQTFaqA7Jx/OP29ho7JSr5883V8OsdA0W7RsgN4Q+JLa0AqoUo//tBVUVs7iTmdNVC0E0eZB28O+pq7bdPYNXc/4TJouuXPkv1FhDtdZemIYPSHPkXbouVir2N8/LsLTr53UVXuhLapSCRQ71OKpddt8B6CpaKMVw1/Q4MV3AZTJdPz850vUYUEEZHmis3qT7KdNz6O3tTn5lPuWY7FZGHo6b7cHGTCteM6JP7AdVkws+4NR9dPyRqNiupWX4Ickx/kk94GOgbBrozhOEnNHPaCu8ZfodgLCkUFkbZmDdLz3Rh7wJdb94bjGLSf0KCVkPIcNhuM2DOFzuhO9k0+TIl3Pi3KEHT+4jgqd9RR4x2Hk7Setlbp/3i3b7dPJ/K+P97DXqRB/QqJXa+Wlwwie0ZTbe1Fh6mG6kHZPBNUKuaLnHuhk1USmr+XukIZOYlzydVpWOH6sNh3q9WHzI0T8S/04fN1n6NSGHi0zG5siaaUYByVc0mLu8x5XSmDHOMetEusFhwqs+lZCe6WxfjqH3lz9OHk6P6iUGuh9o2JfOBrAiNw/Ul0wHuDP+b0n3Df81uKPI4SUPU8qBeC5xzC7q/ELGukp9mET5EP7TqZKKHvPhzsI/jsjB/OlyxY/cv5slLO++5mhFvPQdgqBGsHe0el8VufNJZWwrF6GC91gNYaQKBYPojrA+2pd7zODB0IclH2mu6fQWc1l08coOGJPzDvfI0PV75J7PpgHro3A1YzIYm5lPkHkuqyGQcHA49622EDjVGJTbCyuALqTc4sBPRtebgNu8aemA6Wukylsu0wFdIA+nfVZuS2RmKy90C+nJC5R3jafzupDYdFVa6WXCpan6TXiG0cKe/F/i/m4tEnh54jb4gS+1o/tnf0IsyjFG+znA0mL07Lkx89a5ZELKLMzYOjqVryZVKMCi/RuqXmCnTWEiz5jWE9drPhxFz8pM4YDI2Pbb9OfZLnb/biZy8LwX4amuS9APjJbRv1Vni92cqRSZfI0eUzo1VcQ9hsoHPuhQY1iiYDv3+4nJgpDtC7Ct+ih89KztEcLIKaiPQISrxLcK5xZuSOmfRecghbwk02YyHqXhSBeYGcHHGSNnUHh1phlasv+vg0zEonXK7cwWhv4//WYh+A4tHc+//h+G+x77/xvxZWq5W7d+8S49QPwWbBQe3G9H2f4lFsRN3rHJOdfTD3zwTDMjDE055xhDILqNvUvFsYhcpbZOhwQhzUogJO8HMseOWkMunAJDLWdIoo7bZicOpN8P1TuNqB18i/WWSAM/UDCHbxgayvQReEmy4M+5f/4MxvA3Goa8JZuRrY/+B8BauAX4EfvZP6UJ8ZzrwvvoHKs2S5/477tSSUJgu3hjVQrT7JaP1rzA5bDECEJJmYNXu4b26jsNqeyU5GuDQTBuwF+1C2uinpWPYnNQoTjnWOSB2AnB8h/RMYdIDxQ3eRFjWAjYXFRGqayW9ZiZtEDkU7IfQZlHnXkCRFUByVjsbrfyZOr3b6kNQIzw3V83X0Zoqkj/guCDKyIjOpUddTEpHNzuAW0pqjgb5USnO5nrCXGR0fsuS3teTM/ICo3udgXAbowxl6aiQN7hIK701gxPa3aF8f/NhxawMDkY9T8JdmGy26RlYIj7IzpBSaoK1Thn+xD7U+edBRJU7ynXrS+/yHvOUNUZZMbvTJRCYfxvj2cnHT4j3glEiP+p+pOjwO5+GjafkPOap2uRurPTNxUFcwOSOIbKucdZGv0uY1C7JF+Z32O1fpfvwyzt3yILE3zGzCmlXIs0PeIaV2GhvdT2GTNbJF/qRYbPOaiMUWg+HGPkw+jjSsOounJURE4QL4zcIn/zSRYT9zJGMsPwTATx33RK8rrGCIR6mMYHz/bEpOPQcSLVVtXrhee1KUEbW0Ey9k4BZWyybtNQKqetPH1pUwUIviCz5GM3E3epIWZ6Ld++GE82zlLtzvZ9JZLUFGTzKNvjTGL0J/azgAH1gSSNk1jx3eYRxKOC8WS2pvQPFuiHrtQeF8wLl+aG/E4vZThchskqrBbQiS1l04l7ZSUaPlZn4i3QethHvPQNgaBPfVzIj8mPYr87F+spbfx11j8YTLcCAI7MJgyBGe1rRwodGBHZP2kRucyxXjEDgcAWPvMUvIZl4QrDi8EXmPPoy3hxKPpfhKbtJPsZusdjXqKhuVz69n7oAU/p7bhU7rqGZzURS3j/alfNkMSisj6dcXqNsMVeeROrZxpA0kVnj9abEEWSPfCl0G1vflQVwI2Mf9MnvG3ovF6CWAxwhwGwqCgIPqY+wT0smRB/Dh7gu8M/eYaALdX5woOVr20/jtIgami4CA7r67ME1ej6LbWvFa6p/E8qcGk6IVQ8WLjF18GM5PYBRQz2yudUpJC77Ht4M2UGbzgyzRI7G36RJPf/80pp4FfDgGii0gSOQPFscDP5jK8K1LyHaxUh7UFydPL6g/BmkfgNsQXIuDkd0u56QuGKmQzNN+A5ldflz0btL4EJrpS9jdROqQsNa/lFnvWEQpCQBBoNO4kxrvY5i9ApiY+CNjPb4H0/MgVSEIML/YQJmkliF3MtlVEo7z6GIiAuJpqNLjmXYKS+RI8o+30Jkj8EHRfv5+eiZ29bt4YvM4YpPDsb1oZte99xnlBYM66yD/T6SqROasWEaWEUwbX8crW4b90p4QL0qLud8+w+Inb7Czxpm1B6cg+FWC1Qy310DRTqqdP2b8jkSi0hPwDoHa4btxPZ4I3T8X5SNVbjzrnItqWgtDQhw4H1YqsigcRYlnmy6UOyGhGG33KLwVxgKPRmodREnM2b1foyFCzpamWAa3ZnCZ+/xb7Ms2K0hsvIxcn8m7Kyy84tLA0fYRcF9MRA92cKPf/QJapGcJHfoFDuGFj41TmS2uHHrxNeQRObz26x/81jn4sVVkhakXjZ11OC3+nvcH3+Nc5wRI2yMm+uK/QSP3JCCrHfeWGKZFJ3PQWPXY/oNNnxL4/P+Hvb+Krupq+//hz9qeLXF3dw8uIbhTKMVpgUILlAot1JV6S12gtKVGoUihuLtDQoAYJBB39+xs/x+sFMj9HLxH7/Mc/O45RsYIzKw557K55ryur/zDHxXL+DRrNanjkvEDqm1KNgxsx8Opkj6bQvi1LZyomR+xJFAMcjfJPXD8cA2nClOZqLRD4aEA10HixiP2TU5ciKD8khSHTmfMXp0EufVGevdX25Ab4PXIwSDYWN6cQUB7LiCAYxwH9qSizIwit38Srf59+c9Sa/2ImX+8hSYlgOWO7chtvZOzU889hc+1LrLf+pwtynYmqv1A48+zjfu55b+CiaznjcfLSXS9DdNbxECzXEeE+wl0kV1kAXb/AbBL8jzBsVp3dO06gnMLsclUaCu+BV04xL+PudyOCWoobNQy6/wQfGJLiRda+T3Ih3yv2dRW6siY9DYV7t9xxKOSRMUMFqS8BH2+BkDW3cBHwz9gW/Z0npK54uZV1at/lbGWAy5ydrTDozoToYoWOJ5KEPD7LEcGhmcRbS8i0AUk4ibTZwp2kkUc/fo9fD1krA9P4Nt2A1ckSrj1OeR/SYtlDZxxRhlVwOqOeLxDu/jkpVXgNw2kSk5snMblDYn3pDfHj5OI7FLHePAYTo3/a1xfdJTv3GCFIxTYDCJCveY4uPSlr5DFU6FtDHu4mh3Rx2mxe6TXeXlK9HjXBuJfbqK97x4kxv5kZ9cwznkZrTsm07ePhlbhLHf1Dpjt7zMV3gksp3xGPeXGz7hqSSTeB5GdbLOBTHnvm1XjE8U1pQNT3IaBnxFsVgRslASW0K3qxq3gMFKrGns7YIt4jKzfHp576h++vRDGj0tfoXP6JfouHieyoyU9AST3q5SZzBwTCd69Ekh2nYU8fDsOboVhdm+g3iwRgTNOSZxf3595X++idYSaEW2dNEVU97oejyVWcd12hyL5EKo7QrF42d+TRHVROSFz+I4mRyMb3PcxXQsSzCJDv/gPHIV+xCqgVC/FbFAgUQkimO3ECOjzPbvr8slI+oaxndPZFgrwE+z8Sex4Yi44xfP55S2ksAaAWF3vZzCvJYMLyd9S4foHAEZUEPI4toqXedXDzPkuPUUDDRxO/oWRuoUie9+lP90NGgRAYpMjVT+GlN7BsH6edSRtfI8Xc+dQHwxb2nJFFuWVxRD0GEUxtdz0P0XS9SRcG1xxlHrCkL2iR6N7KpI2Ld4NwcgkQ5BJE0VZvYznQBsEkStJU+Yx1R6e/2wpWrMc7z692WKT2+tRXRrL9iEX6d8vj8QbD0HtCM60BTHqUhD1EhNNIWam1ln4ecR8/nX2PX2qLxwciOvMQkqde54BuT2MOgPA0+rfaRp0lMe2ziawIAyttRFGiElK4dZFvs0dwMlGFY79somKKAfbfXnSVock/trZhUOZHw78hDl5ITgnw4jj95J9r1RFY8rtYsGlPmQOvCoe38PebpLG8Ne3U0m8mcjVvlf5TdrK5BntIJEzdEcCrZWe5EbF8fLYLHZ1+BI2IQvyPkKqjWTlhrH4F4Xg+mwLD6f8TKl5PNR2wp0faDPPx2LJo9vORNULX9MtKJn0wLXMl8eR/+JauDyBdx99m6TVLqIfX9Yb0JKN2WU1lrQsznZ4s2PfEF5cmSiqV2Q8DbowkKpZn7Eez4I1+FS4cdVW2+teCQo9I548za/VCg54Q6clHZhDnOk4H21cSoHKyi8zThGvKOWo/dP0r9gLt7+gWZvOzUltqBzsOGz/PZ+5CHBSXP8y/ga5JWM4tCccU2g7f0Z0UGZzEdkGFXugJZt9+1+l+Vw7S5f+TWpQGQpLM5zuSXI6xPB31pe0W2qZfzSD0qDe7w3AVU9/Dr/xHmOOjWbHT8N486ugXvVnRmbQNuIYRwrmcLZ1AI/apGiAEMVOuhwHUti5mvQT3XzsCRsXKuFwP7DoWd/kQYO8Ck+1gRqnGiJVNpHl0JQBEhmVTbPwUTbjMGsnQwMb6LJNAlMbIKC0tNPfDjYr2jneLUFpc2CJP0yK+g1/3TVq2yJY0fcAV2pSeWvZfNJZA05JULlfZJ/4TeXNCWO4nBfLqoJ2WgVdr2SfIBHI8Z9Dfcw+Dix+h9OdPqIPd/URam6EMzA/gPZJCrZ6FTJWNkFkMbin0pzfyA3/UtRmK3lBdzFKm3la+EBUw2kvxE49Hss+T1ztW6hz30+nVi8muh2iQOnGKufLjFHDjO3+GLLjmVW1l0nqbNgSiq/TW+wJuEZbkydnv5jPafsubI8IODVugMM/w5Ad3FRvJvFaBVdaCzg58iRS0/c8KLY8rykO79QzKLXTyVFu5jEn8X08074Rr9ul1HQ6YJFaSNBYSGrbA1kVYGjkbV9/1i0uo6LYlaGn5mG2KwXus0PGmZaS+oMLBWEl/LXyS86b0klJgX/+AavNRmKmlv133cm9vAdltwHl3SaREatwpqq9kj2Tf8C3MpmgJ9/i5pA8di223LM10NWfYrXCjmtV3thvmo7P0Jxec7BNa2PNO2tIupbM/GNjMYy4D/RY4HGbCrcofGpD+W7x25SPsGfeivvH7moMoVnfgXutM8FF/pj/VejpKRv/GUxdVij5Efk0ujQiCAJV7aF46+5SYIzkVuwRmhxb+KgEImwxLHng2C6nSXy81Y9s7XDaAlzx8UAE0BoawSGWVmMHlY5HUXg0o2uw41bFeMY/cHyY+3jaiscjdzhJbOQqpLZYaLxCp+MYNpoLcFDa427x4o1GWOkQC+15AFS5zufizb9xaXRl6MO/4aKuAl67xxZvUXqRX6mnwaeS1c9vJd1eCoqVIoivJYuG0u2c/vY00qGFFKy6ygTXtPtgTO8JXDk2jMwt43CZPA37kDr6CVIx0ec7BZQu6BXOXMz3QRJmIniwL1l2k3jQEdOnReDGZnE9HhRUQGDxS+DSD9/aVJbsdkbb152uiuv09crCpeIQBK4Bczt1TgMojnWmwzub+TpQKOrERAz0gDbF+bazW8E7zXakmOIg4xmQ23NV+hqfXN5BXdt1lvWDiRp4XCrQ0OrH+S4/FqnrSD2TyohTIzjBCcZNkHOk/FW6Ij5FXXcGiWQWS3xKMAfcZK3/WhZZ0lBKuiBK3A81dUdx8LQdo1w0XHjxGFsirnDbMhvGHGVZjisrbEranGX8uvgv3FyrCW0+Bd0+SOz/lYy1cvzSJi4ddmDViGRGBWcCYJLqaCufh2fLw/hZz1JhNBEXegbkT4qghZjXGbPhK0bvGU+deye7hpziSP4E9lj0VLq8R2nxNygyEjDrzGQPzcEi0ZMZlkFqfCBkv8sU1z84Mfg4TeF3uNiwj2ddjyDs2AOjzoLGn0EdOoKvJ9Od3EVf730EqzXACMKcM3h58Gx+zN5EkW0+dxp+pqRvHa7+IvjKU3mTIcvWk21ahcV+LnG6RtoNN9ApRXDvjY5YVMY8fCUyImv/pE9i70BXdtU08krHo9V0kR/vxPQZNuRy8VhBgPcmj6CgTyRZH4cyQfE23B4MkfcTX1GK0ZS2gQ0lYCLfYQUR+r/AaxxP2feh+5fz3IrVkxd4E5WyCElHPtgHoOq245Ed05BKyplQvJjbIy+Jnnk1x8FqZFufWohvYtj1KSh+eZ2da47yyPwnxXFJFayqVZLbZWXZlvW0ayNxneMOLgvAdTA38k9zaNBUnBudWfLrY7gOKBIHax8B0gRi3Sby219rcTVfpk5RSaYlioklW+HiHHENHreGF0cv5MdrASh2jUQ6Zi8PJvuUtmZ+fXcTX14Mof61dxAmiQiQwNo97PEGr0Yz3b4K/ky6il/DfVuM2tZviFHfJr/Ojb0dV6gqUdPf10x84ofQmIHr8dOE7htLkQa8i4I46tFOv6TP4PpqUPvT6K1n02PbmbV/ChvfHofh0zK4/rIIHk/5FumdYupPOXBidA61HuUskrwBDZch90MInM8R/aPc3JvA1GNdOCkWkfLFEeCte+O72hVKy58DaUnK5A0fCy90DaU/cFofQdFPvvjoZMTGvE5rTQskijL3NhtUyT/i5/4/E73oGEW3OskOqyMhuRVrdvy9tk++dhKHxY7M2j6LbTO30eDaQFl4AT9L9FTsHYQ3FpxqxxJ9w57TaaeZM3AOLnYuDG/LZczzO5iSNZE23ZeERtz3kP1/rfyb74iLi0Mq/d+/Dv9N9v23/J8Xm8cISH8CB+wIMyUidDdhPp/AYW0Xg5d5iJJRnWWo6GTszSRsrTqsJ0cQPPVsr3bGqGFAQBG/G3W0PXwQT7c4kdXRbwMAg+o3Iex5lF+SrvBBaAF/hISAxgvSDoJTIs4yL1Zefojycc8yJPQLCjvn9PKeGS1cI/a3RTS41bF/4n5c8WGFUoHC2kl0STAYVGSMbMYEVCqiwF5kYBgENZ3Rd2i660j+qy+wbexlFjx2GEo2QdM1btmpuOVXxMKzIwk8ORTpW2XQUi8G3K6vZtWlVsrLBQ4lOmOWtXJeohR94tR+YB/Nj5lxxN0I5dmPtuPu+j89+7R1CqacmkJVgC/r9OsYFZF8v1IiY5VDJdakYhIVArvNNnwVYmLpIXUBCVKob7Hi2HCH+lo3bIIMwSIGXverRtLs3YSsNhp8cxjoFtur3wVOWzG5tvCwFoaoQGJuv5ewcLzVzDvvJ9Nsn8uz695golULnavFgEHix+Saorj4xECiYnP5Oa2F8SGjIeV5CF4IKk9QefBFgwZ3axvLpM5EuzT26vvfwFyLUwu/LfqNWCEEDA0oGi/wR34/rh8YTPWw61hTMlnmchkqG8FvOjJzHcP9/qChM5QaTRNGeb1Iu36kFSQypOf11A4bR4m2kM0ecMdYLS6gh/4DzilcaBrDtzsX0q7Zwkf5felK9QLn+xjIlEo519a+hXpQOg9NvIKpbThoAsXgjjaYh7Vd2Bvk7GoYxaqBc/GQhcOoc2JQC3gv7jR8dZSjh4YR15BOdr0zM/mBydatmFbnsLImlcDqtbw4YDr215aDNgQ6CmnSj6QoR4WPxYXl5dNpimsRA0cWPVjNzChyRpMzneuetXgOP4NV9hAoTeAQjSBT0/jEa2SbbZRWevGdZw9TwWYFczs2hQcvvr0Z7d076N1rqZT1yCl0lohIL2BjQBM1r33BjZ7c5b04e81RBJc+bPzpeVR1RXhKwwkJr6QJC9Qc5bG0K2j9LjGu6zWcQ0uwc2154NmV4xSrxFGiQVpVzMD273DrHA3JK6DmGC5d15n+93RkZjnbZm8VnwuAqsNgaKBa8Ob7Voip8qDf39PwfVQM4LFNAT4PYed5lN89wefWccIunMR/zHa4eldkjyR/RpF2KIcUf6ILqKMhcApXKwaxz0MLnWWg8UdlLkHTWU2zQkfpgKv8aWegL9Alc+fSo1vZVxdEOEsIUUAIpZB8COrOkJ5pxGPyKa4765iz7RH0ug6EhfdPO1DSRE1dKGbvegY+/intsjgIeEvUvG+/Qz/9Zm7eDSDV4seYrmUIbwniJhPAYwSDun/HqTgYiVlGRZcKvTJQZLZ6pImXVa2iJHEqSiUcursUVeBoRvQgiCUSqDs7naSbWi48UsRV711UdKxkV9gYPArG4zJvLTdq4/jnxQ+RWvWoJg2kxfsVdH1X4fFVMQZNDe41GgaVfc701s/hch+o3IdWYke8UlR/LYyaQZEapjnGiwm5wo2EWCLo41JOecUc6q+pCZLmiJvygh4vRgTC7oQRWOJBWQnsDd7CkqGb7sv+SWTcdGrG6lzPlOav6Owy3gvqA7Qb2+hQVeFQU8qcA3PIfOi+b4KH5iIauYT24hA2/b6AO2MLefTfR1AiEJcVh0ujN98OzWddRwtfBPrBuExouQkNZlbHTGVz2XhW9TmA1hLQK1BxvKMQO1qpkTbyk95EFW7QfhdOjITI55GFrSQ37SkcQl/hl1YIlGhEWUgAz1FI1HMJKTuIJTuYaPsoBo6ayOQH2s8xVOEWdZeKK3upynbEMKicAUCrsZ5Cr4+ptrrjn7CL6s5OFnjcF5wqt4XyZR04ydv5beVGirpDYcy1e/X51UsYfNGPq0OrafJ7kYFeQOUB0YfFYwRDrvZF1aLkqbOR2Oy6YapFnMNcB4IgY29ELUqfO1QHZBNmV0wv5CVguFtO4rHf4dizBEYXI5ve2as+LsqXOnM79UcGsya4iEkL3QBQdEgYeWEEjoE2tgrf4jv0cXwvPSb6tHTXEaGZT59QG9knUonZsoqjz56lzjITnayCkpY4ZuUMweV6PJFLbjPI/xBdQa+h7r4OHmmoSi8x4NU3SXJpprDOlfNWBcOtJjzlCjz9BnKgsoRY9yIUjQsw1j1Nn3EANjHwKlGAXMv6679TfKqT29fiCHo1ncgHzkmJjfw1TxHY4MQHIXdpnXCLXYOywWrksKqFsgATfYoNzC6YwKuzMuDCbOgoRgh6lYirh5CqlczOfgOz1IrLEle4Ksqf27QufPPs5zi0ObHwtylEKy6K3ndSMXCQZx9Dx4gs3LuHEKnwoNauHzF33xYR0UELeNW1gnYBfr4cym923fyZ6gw1F0VP4bh3uKJayjun/sZgjuWn4hkk+gYR/sB5PR11npSvd/P50y8ywmkDLq251LlOII5fKbvaiPluIAfGtmKzwRMSNf/y56rViykPrEfSkihKcAtASw4cTobQpawq1yJcm82Z4GS8AhIIdvCG5p2Q+wFSh0TWzrjMOT1k/V2MTKoGW+C9MZk1wbxhV01xgJQg7wBuCT1zwZF+4J4Gww/ydn4Yplwdb+8dR/LEC0ja74KTOJ+ou4p5bul+nq6Us1nTgNLqxMwk0adToqygW+OC1dZN1oUEOuS9N3Yba7fQ7qyh9fKXGDeMwPZTB/EBcjC1YWeqYevw5Zwue4T87WM56N7KlOlS8J0G2hBSrt5m0MkJHKrV4JYdQ9ns7HsMHMp3MtVsJvn91zEb5Xwgj2TVus9QqXv88zrLwCGagBsigC1l1lHCvHtLonW1X2VAwB/cuR1AdIUfPqPrYacbTuq+JKW70N3iic7XygcRtaiU6aB9FhI/RnFsKxfuDGSvwoJ38M8g2OABt+Ji4VG2HBtGqXkne2+MoDlGLoLwtCHgnMLygDN4KapZ5KLEsbMvY5We4Bgr/gCDSj2J3zUcgDY/TzHZ18OkJHIlZ7tjufhBDI41nvQbexl3Y2+N1WOlUXgdj0A7dycvp1SICPa220jsI1m7ei1jtW/jWu7DBU07Fof7AZ4GjyYu979McIUvEw+8ROP8cpGte/dHEZjWk3Bum72XF5xlXLLcT4VYZQ4cPBuHqU3Dxhe/QmdTceMBRktXyRm2jX2DkHJ/lsh8kHp8BnwrSsJ7joa0fbQUejN8i7heXza4QLyHWvE5FgRocG3gbkgxZq9NjHcEst6E8l2k1fhz9shwrN4t+BZDgDyCh42NkPshioBHSU+8wd2AUoqk53knN5B13l4EZL0M9efQqSWMev8vfmyRcrAFFBY73njgWjYIHmzVNJDo0Y27VwwNjoPB7d/52YbFfjTXtaOJS/fFoa6Avh+mgh/gN1UcdyN455+iyw6+fP1Dhgu9vVzebJKzpbaQcVWB+OtDqO7XBECzLJBmx+u0ycV3tc0Kt7Uz6F/VswZo/5ZTI/5gblkQGOjt+nikH4WNt2i/uQm1LYJfhQ8ZPrQPqJrBYyT0+ZbiLY40J9QSvKOVaoWZiKFGSPwE7v4E2mCKjpTiXFpMqDGUcM82/rMIUgGLzELE7Ug6NZ2i1/cDZVObjnZLKyOzG6j2+ol240w0Cg2/1ldR99pnCKF/E9KtRhIeCrZo8dm3WRh43ZPgS2O5PrmECREqZEIjDNgBKV+C3B6TxyiK3eF4yLe8r6jkkMIHdogSjm4RW5j88auoOpppuujJ5dQcnnyilvBIKbi/RMnNX+BSN8XCDRY0VDL1ER19LV1iADLsKfCbytGyVWRdbmPaWQ07n7zQ65ya9I2c6b+WcrdfkN2FBOlQbqT2p63LRNKNRwjIq8MycTL1d4YwakwA1J6Gcw8T7fs545uewyR4Eub/LoPUILUZIeUrAMxXTHgVXsAW4MOad38nU68SVRzUftBVwZWyMNK3D6IlvpXIJ9/nLScLmc27AFBYqsjJDaXqp7G0uTRwLjIbS+08vMp7fDytJgY75uFkdKPBrGCsGoK6eydvu7VmTo04RXLpQmT1bmI+zdDESPm37HmsFVSLOK7MptX9PI6O7+KnKYfCnxinLuN6N/xttuJyJ5i+wb3BV8Pd3idDKceW48Cqd+eheVLLEEAqBSxS3joWg9+ZIdg82uhWCbRKnVF5jIR+P6AtOMB4NVTInHGPb+XdxFxoSgfXAeJpyZ2QTz9Ceuw1+n3+NK9pO9n7QLIvxlrHaDU43wkjrCCCmQs33KvzT+7ENfYoOTnr6XBuxUeT1GvcbYd/RXn8CC1zr3J27vuMUf3cq/5gpTNxlR7kr9qGs7YbwWbmndMHcdeU0hpdw74Jh/Cu9GbRL4soGWSGpuvid9xrHHrnPeSfb6c4+Bvyog4SW/c8oy7/JiYOxmVSpdeTonsBbel05CYnahqv8iDTWMCKrq0cv7u78WoWkL66A6InI+3sJrgwGKdWD+wctNg3pyAE2SBsOfhMxFDvxe8LfkditePDH5/CoOzE8oUFxt+E9nxaM+R88sLTyIBSNcjMQ3jU3AUNF6CtAGu3EWVXM8OsPrwYc5ws5Uro8434A1wueY/QW1a49RR/xuTQf7ZU9AftUQ7YU7mBsItF1LvV8+WQ7TQ3f8xI7oOdPpbH0TJrFxNdInl78Ha6VImoPUcTVfk+0vJXsXhWc2hlE76ed6mzG4fDLnENfFo3jyOPbMa7M4o91SF0uteC7+R7351nfk5FeWw6ZhdX5EVGih8fB94GsI9EqJLRfjCV+Dth7MxLZ6dbM1r3FsbE/EB+wwCUthhS3bywqk2MmnEX95A8jlx9DXnjMWi9giCsE4FogNVmZX3Gekb1z2dA121ouMyVwpv0uVFDgaeOjmQr121mRjf/CeeNkBdA3oYpOL3RiK/dS9BcT5GfC8F+A3GoOMt7t1OoutSXtoRSBpwK4R03DaMWvg2dZQghTyN/fgf9rGXouoZz1KuRQa/dJrLfhnvxv5RBZRwsbqHFsYUVfQt51rEQS0kC7i3fU4GN919/n8/GfQbHbIxSQ2pOENQMhIZLaIGskEfQWKdT4vEthQwXvZ3lOmi8ysbJl1mafA2PIz/hVJjOkDczgBHcborn7W8+pqurG8l4MFrsMMgCoCcnnGdUcUbhQIvzKeZq0pmggVaLAR09Sj5I2DdlH8HG0ayq8iNaMPKv7O+/iXz/nEO4VGZhmbKCOT4dSG0WQIoggFmdR5PvNW7Of4PJ0RWYZNp/u4aWXGJufUrD/mCqbs5mysSjPD13FhETv4Py3SiPTcBx2FqCrlh48/03Ub+RgcO5NTB4G/N1Fhwa3Kj3sHJ47GHCQprhzhERgOg9EbX99zTL6qnzbaWrTyMZEj8eCV7Qc0pyqrZeIrIkC6e6EgYNuoTcHCKCsGVaoiyHaPZW8npVGOUFYVS5jBXjsAcTwH0YWsMuwrJyyAm/yo5J3zBF9Q5Ye7Scqw9DwkfMLNfR0ahgSLMTY1S9/aOR2PFrG+QjRaMNIlAtfheqPCbxzs4iEnKjsIX0rB5swj3Pvno8KV/4Fje74oj8cgotNjPdI9zBYwB4DOfc7g68jkvo6FfIhrc/QGd14o3wHeKazbU/aZlfM1UFp0L0NNuVEaJMg64eEf32O6wuUjG0WMvjyZN4xP0gjgozOI8R5wtNIAeb/iBpnwpQYhiZRYFlGCMeOK2j1lhyFv1Ch7aDJscmGowi6HJTx1AiSvXYuQokGZKwdHfh3k/cZdlskG/Kx2GiA9YBVt4LeA+dQseP1gcj3yDXyFEE+5E+1I1Kn0ocWh3wLwhnTL8iDh0fSbW/jdzUFPaNWUK3qpsLub+SE6TklDwN+V3w6XImSa7i/sP33/K/Xf6b7Ptv+b8vEoW4ubXz5f3XFnO0cDGfZI+i26LnIedBsF+UxYtXLqLz+ChaHVrZMX8nf9s7MR1E5pRch2X7CWr+sRIWauDV6Vf5pbMn8N6WDy59udU+APkdgbScENyj8hDGy+BQoijbN/I4ggCKrmYsrSq+b4URyt5eQnEaC3GPHWSVwZ2MqAwk5odYYa8jpeY1Rn92gxnlzoSfHkdk13Akg2ziTCoIlNqiWFEJdjYD5gGX2OtgZMGIE3ByJLALCWORAvqgMhImnueOzgOSvhE9tFz6U1P4Lh63bagiLOjtpKK8TMYzYGiAsVe4kHaLg2k7kTQvJdgQ2UufHSDNVI35eh+6861kBEYy+PHeiFMBAYlFQlW5K0806NnSV2RajDWVcuDNFzH1sZLwWgtpkacxDjmJ4nAyeIzgO3MDDfZZLIgNY82SZ8mTvsWDFO1/kfdpdqL84g3bfWkQR5UTcVfu0OZox91JILEqGWwfDv1/Bk0QZXIrjb55uNREMfvvIQx5NVVM3PZsbAD+jr6Lh30Fi35bhHJ+eq9z0pjr+GzXbGrb7VA+/ytP2JfBmUnIVN4Ea905p2unJdCbuRP34qEDsMEuD9oD1/Pxi6+gMBiY9vAP2Ctq8df6iMFRfRX6xlOsf/Jdll0ezmfvLqazTz2DlniJPoNWA0crbhN7CTxxx8hEivqZQV8rMqdcB/C5yR/vuGw8YvPReTVgkkpElHgPWrtQ9wmHZycwy2SmLWM+0c+fBvf7vh//dAbSyHW8KnV45UQxa/EG4AekNiv2fnW0mfRQ14CdvBEkKjGRbTVSsiuU9HEqVIp9xO6KY2joRVFuShcGbXkMbrHj7vU4cpZe5YRXOS+pHoXQ6aJ3jETBPwEmtrXDZrtybiggNrWQyJJX4e6P1Hq9xeWAnwiy+iF58i9e9+tJONus0GcdAHs6HGkUqunX6cBYnZVaVSwxXv4iW8ipP4VXStCZ8zDkTebFoef5J1hcxiysjAOy0esszHtxCz4PImIUjnxXtpdOAQZW/0zJDQ9inygD5xkw5G8c69uZIrlDabcnM28sQK3RI8EKp8V0y1jHGnQfGqn1NWJ9cjOZru4sbisQx6287z9p8XJiaOxZHNQ9AcTbn0PAbDy0AZxPlaExOBBZORZ/bTGS4z1eFHNtxNpWkfbzIUbvySbf+xwe2qEw9ybHrl/gm1tDsLMIDLphYdPumUxdeAid2xDwHkf97UN8GX6VKPtIJhzvIMhOgsTWdY+VUmSfyaGpt+l2vcHvgb9xoC0LFF+Ivjgt2cj6K0mIvEXWyQDc77ojox1CFotJaaUrix8+jW7mEUKLxPZmyp6B+PfuzxdOb5I2+Ba//zYbx7t5HIvrxwjpfURhaPUynJtPYSeDx9wN9FHcAO0yfsCev6JfRWkcxtWHM+hjK2THkK8gF/A6x8Bh3dyNHkhz+x10WR00DXPGy3UgNKbTHfIqkz7YgG+FLyOH5LFy+kJqm3+A9mNQugVrxBaWfHARvaKBm2seRm1T85jcQUTyuaciKWzAsmAneysSCcndzcrkBNAngLcoCYjVQj+hL90VGu6eDebTwl1seOq+XEpL+mzek7WxQR+BMXUMzyR23kvajr/pQ8B1CSavW+yfmI0i6P5xKmMd02/HYbkTRJfOQrfcHttImZjgd05C0vAHl7+LxN2pnuoIsDf/x2pXKeH7p79H16bj8nUf+oUHiXJoXWVw+0u0ynRWTzzMa6f8+cYo45MBDjDypMhGcBtCU00+T7zyHT+8voxRx0ehH91bN+qCOZ7vajqZ1WKPc5PzPRCEsymPnXf6capFQ7dbHL/N8kSZoQf3HHCMQYJAbHYsHbou3FRt6CxNtAHkrYXuWh6Ovo526SXOtM1lgN0pVLZIcQ4EkGmZfu5VHLIvoPCtJdy1WQzkRa4U/ZWsRiIMp6i3y2R2aAo1xpP8j2SfLhd9gg1TXS0nA+/QT9HbD3b+iFcoi/RB9sZYJFZRKhtzJ7/qsth7/nHqOzsxNtei9+wEzTYo2wZqPzbuHo7PyQQch9/Ew6OZbMGPjKLVdHSADQvfjM/kRGwJBf4q0nybUP/rReCcTKsskbbY7dx1ryE9RIK92gc8R4hsnNwPaW/WsNSrnPfvTMSptD+yAcOhqUb02gx8FMuAX7lyrImgm63U+SdiM63udU5+3mMosBew787Eq9yPnPZKiH4ZQ+lOFhbcwmoWUFca8b7SD/cpmeL19pmC3FxNwpg8KkyDUZ5vQGWnx1GhFdc1ulCmdUYz2iuZH/Vp7FzxPpNDzNDoJM4n9lGM8MzjlBr0argODFAgyitW7oPaEwzssFF0OY2pe9MoDixGeEsmyhApnMDOi3j3ZdT+1IZeo+N0aiweVvde53W4PZjj8jJ+fWcNgk5Bu34IEYCT9RrDphRxxvAVeb6iOoPevAB6nPEKzIvYdwt8tK08VPsmLvVaUPWwLwp/wrt1GCW5YfRNuMY742Zyw7hHZMClfIvVOQl/GbhJIS5jK97+FcBGURpPagdWVza1geBeyfnHNuFFkhiItujv+SDlFPsxbo/o/NaMVVx/9BSLx2imtG7lrrwDaBADXEW/w+WFPBQxhOfTduMcV8Vm3y3obA/i9aG4NQmjEbrd2ikIbyZMGgle40HhjEUZwM0TsdTkKzDeSeS4VzUPSaQikEPth4tmFKePH0PtLpA8Ip3IgHLxG65wBm0wpTQhCyumtTIYqzKcm56X6T9EDsZmcEqgoHgvAQG/UCVxZV3KRRI6p/HRA2MLNGcx4INVxLaIwaYxqTvB0ICdvIaY/eMIKA3kxPMdPCzYUUwZFP0CWW+isC3j6OaxdCeUYqy8hYNXMw8m+4o62skvLAaDmfKrqdzQ1oty3B2izG2aNBBvaycvNb/NSP9DOKntRVRzyV8QtoyXUm5Tp+mi7HJfhvU7B8wRA/82ca1RhichVVZyEm5RGhfIArfe7NI7wT7senonoe1PsHjvZH546xDyG09xxv8Gx13h+yobSzY+TmZS5n17AGBQXBEhyelcOfcwwc0mPO1rxHX3rU/B3MnMcnu06XM5MvoYK+3rSZVFwoW5UPoX6ujtfDFzC4JNoMMEMmvvpK8gCFT5VNHpV8XMINjfGQrdDaByuydT3u3ayZnhJxno1xeSwkVJ4p6itdUyZMp5TrffYMjdJxnkli4y7EytvBFUQfvT6/CrnM/MT58nZ2ot1IngSKt9FPLETRRZYMr6FQjWCAyjTOI8Vn+OFnkg4zVwtFlKSkYizS69WdWCDbTtWmo86rndP414fxVo3EUvWakG4V/7s3gPdNI6ZOZasLqAzQJSJR3dtYRZttNWo6EpwkKG+Xiv9iUmmLh/In0z+rLRoQX3IYMYDVTJ43l12iu4m5OZ+dtIwg1jGRP7vJgwVrpS2+jLZ3+MQGZyYJmDK3lB5fDkbTgYC10VHPRez43nTjPqzONM/NuIV3Q0BPhAwEzxunhAY6cXRW6vYKWDl+QLIXriPXm8Ns05ZPZmXnj/I/KEXqlEAPrZjIxoTOTYk3/QYt+GlN7zu0eVK8E1Lgy+OBj/gTeRGhpA68lx4/fIo6twdHTA62Irjn53odUGRb9C1Gqma9TUyo1Ys4Jw+eMVvF84LzKIetaoX4xJoao9jGMdYjBfIggiC6ItH1/XFAacb0Bi1pAVl0mjWzuYWhBqjoJLH/YaEnA4LEHjYaDDuxGD3hmq94oM54DZAHx87Szjzw5CM/4Mq8L/Axxg7mRDwi9cLHLnUI4fqmgJ+E1DWXyQD5d/yPOHchlw5SiTjmfRNyUf5PHgnoajTMLMY61U1zuhG5lAhNpM96AHrmnbRpZ//i0fl0zmvB6KDAoxvVN1EFwaudgeRGKJA/KQLr7pNHOsNZxww9ukPLKQ8rIBbKk5QsDQIg7HrKfWs5bBtwTq/T7DTZYF9mHE2Fey+JstPL9qGcFfPEP8W0eA9+91P9uxkhtW8Lheh4fdbhS2IWDuZI7mJhXxcLrLAvU29nXCQ6oYiFsMoU8w5+fvMezows1Tx+733idd4tAL3FViHo33ksucP9JB2M0Ayoz319WCRMLemJs4eZfR6O5Mo9NlIrt+5an2Arg4D19FJB/eTWTdbTeWvywCeVC63Tve4NyH8VUgszMgnfCGKGP7QJlrvEFafRDzBl7CftZB/qpP6+F3w6mGG1y6BAnxFuwf2YHOs7dEHO7u1PslMsRrN+/5wk+23mCKtvjHOBTVzPPe3Sx1gEZzB9UdYVR3hBETu49RliRojcC/wZ9Qi4f4PQKoPszvJV8RmK3HpbydPi3OSEPyINBLrG/NQYE9A06MoLjZCYAhccuAsnt9q2o/J+JsF+BClXc1FcZ4ZtRvJ6yzmBcvLqa0zgUh0B1ldgTD5jWD2gdsJhTWJj50t3JZ383pPueRS6w8bGpD4xQPTvFYrx1EJYDJBse7IAx70W6j/gKk7iXUUk73cjW3b8fz9dbP6Lcistc18Qk9gfX5Quo3T8YitYhr78SPxFiC1cxAyxkko2rZ1iLFpVGD1a83W3Ke+yaOXIe/InVkVMA7g74gzbkVhcrI23+u4YtaCzeaF/JEVwvHNUP+FfdnrKaIt9zgy1PRHNgcR/MUI8NevN9uf8cOqt1aaYuKwN39BkEeZyBsBTReRVdVRnrfdArCC5h+cjq+Q3zJVrTzaHyPZHblt7z30Dbqn13ED22OnN83E1dFA6aQF5B7hCP5GdaUBtGcO5wm+SFaquVUpTiKc5nnKPydDhE6s54THuFwawxPVflyZ/QOKNvOT9ahaJMyGeURSUTxDH4co0FabgUeRif3Z6hZw7F6R5rjBVyCfmGemxv4iuK7ggUkVjOCxcy5cRmcV1VSrv+et/896bNT+dq4hxcdL7BLtZO7Rmgy2WHvP4e6ajMHCrciVTRgumHiuS9XUDTpMPjkQsMlWuI/5eedH+F7wB9PkwmpywgOJIbw3XBR/h+JgtWNOi7SQoolAwwSNjTH8BFQThuqyjs4tjQiuVtA1KVz6LPjoF80yLU0WuX8ZM3CtbON8JdeZ3NEPuun3b9Xo53P8ao7fJxtpf63DvRTskAUDkIQwMf5LCH9/6E8O5yUO+sw/rYN3tkMPVKeI/IScK6rp9XhB1KM9VzXrrsfoTM24+twHuz98C73xb0oFCfjLWAoVocYqmNWUDLIg2zhFg6Nddi1B9Ad+gYqfSXrQ+uZuWYt+Tk/kxV4mTyLB0ytEK1AVB7cDqhmX5OGlyLqODP1LGf09+F/ggDatk6c6krwny8wasAx9LY1ULIZst5EqVzButVPo3bo5trYN3D/dzth6YLqQwS2foT3HQlGncBLXlBlTYeAV8A+UiQCSKRUn+jHqGNDOfDkRg67w4O0EKtUy9pTUTjZdPzz0ZvcNiwA3qXdPo6yrFAGXo5FFybh++NPYRh/H6zRJvHk8xbwwQ5bTDEmoYU0EFXaqo/Rbv8Km158F0di2ewBYzTNkPex6Bspt2dU9we4XRrBbcUEBIeVDHYBBsyA5M9B7kBm/xDODzrGJzdfR19oxslqFMFbrbkQsghBbmP/lD3IhOG4WvYz/T8UIQWZQFlAGSv+WECAQY37p9eBhcQox3Lo4Us8lDIA9SsX8bTLZ4jyF3bzCzYb3C17gZLRNykPeIFTEqiz6SmwiyA79gZxOaIK1rIby9h+3JEL/X+nTduGSW6ifvEWKoPqufbQZLAOxCZtQq8W14p6G9SoQ2m1ODPhwARManvU1mqENjXgwH/L/375b7Lvv+X/vAjVB0Guo9t3Fj9uLaO4PRAn62wMRnBUAv4zoWw7St1otDMPUtyaTFDLBvz+Bb+4ioGifBPcudZMlrKS4RUQqoni8ZItIhMi6DHCE39n12jwvLUYj241MpsBgheBnTdYLRg68khK30F8uwbT7AiEhN6L5Sr7g/zWBO1u6xmq2k+wogVK9iBVRvBOI9wxShlwKxC3qmCer38MbmRB0lokgsArH71CWVAj/Sf5otJqwSiiVem7jnfP/k7OT2+xbfo/bOt7nJnq90SmV+LHADh0/YhXVR9WffMcGs8GJPMMYOclSj8BkboOujvllP8VQmNE6v9I9kVHSxnyw1o+ffkJlI3bud4oIYEeKUSble+z+1B1fACSNh3pfdKRPComntZ2pKJ1q6dW08jTtsMYGz2YXrpRPK72JNHXpxBzdTC1afDNtbVE9O9P9AP9fnMjkeqTfWnQq1gcWsiKQ85wYhSYWlH4b8JngoHGbgXvn4kAJ0/myu2hYB0Ymxjqt4f3hniSeN2Gd9Y1AtoyoXG8KD0ZOEcMNCLQrmunItiPnyt+YfADfStteoJ1BtpbdMSeGIYprhiiysBQx7t1OQh9JChctjH4XwsUOzGAb+3KIysqBzuDE6n7rehUTWjf6oZLj0HjFcokL/BFM1R2KXErC0MZ1IPkzlkDbkPpctRyfuZthMa5OHfOZE6oH+yxF/2NktZS42Lh9PSdKLuVfH9HwflUf1FirqMYPIZjkAdR75uCR/EVzhus/N04nRMX5oqBraRP2NUcRmV9BcLQS6imHuJv81DeAA4oHuOTrIsEmAUM5llI7rQgeOoBGzjGUmnOp0FXha86FvsVDRg8e5CbzddBpuW3AcXoB5ykqVuGSgCJzSZ65px/BAb9xaJsV+ralEwPaWS+1kqJzQpe43o2m1ZKhp8mZxiMePCLkvL1vV/tmzV4V8QxYdd0fEIqEAZIoa+IyE4vv8q2eenYd4YSfWc69naToQfhGNIUy7DMPnglhLDWnICHSsf3D84dPa/o38MyqR54BaPtdVI6isDSjZ3zAKIf3U5t6UPE/3wTv5BSBEkPulLpRqD+OBUB5XQ7hbPVNpdoSTi0ZIsNeoxAdyeTgaVnKHMzM2LmSfH/k9bC9Reho4hEx9HMbH6Vmo4gbPJWglxu9Xrv/my30NRpptzlN1KKthHVo0Qik0gJKwhD1+GDqsVK0bUo2mafQteaC83XGeowkSv55+jXDwpeEHCzA0G4H5L1kifh1pSExaznyXfmUZOsZ+KSLhi4Cew8WXp+CVWKq0yacotPAlrZYX0CfFeK9wsbK6s86VaU8pyzjacdoNZUDlVVonRO6BIM5m3UHYhBaL9AcVA9eZ2X+OgBNOHap6dQ2JTMxaJgXrc5EqA+AnuDKNA7k9lpYoDNSE1kHZndRfcvxs3XWJ50jr35z7JDOpBFX32GYCcRF8Ixr2LthNC7J0i+nkxLHwWZtWk4B3mAUWTs6l2m4Fa6EZPMyhabGYVFCgpHSPpUvF15L7I1yMAUxRWsDssR1J2ir9K/xdjE95Xd7Pt5LJBJhYcCuJ+0a1b5c+G0DFezHaGLShkVvha6Z4DGH027isEX+5IxspK7wzPwkQ27d5zCVM/KpQf4uErL4J9mYlHokdgsYGwBYzOnLT6cTDqGFQML/p5BVeR9xiBAkiybTD9Yv2k6PqcD8X3vCfDpCbT4TEb2zXJsd0IZ/s8oOjSdIkvTY7j4A1SXX+YJoQ7Z4t/xNYQTK+/NrhYQePXDV6mPvs0za7/nttkKPIpEsFFxPhGHdjWSac0oZT0B3car4BiDv7WQR3Y+QmNoMcXZYVQGt4nndEMMfAa6+hPvV8bhs38xuPQihaVamPGrGCBUOOEVu5dmuwhyxm6gw7WaNOEluLMOst+GCVksS/4Wg76WvDdeInLIDXix17BRunzLxy8dIa0CLrd68qZC06t+Y+VzlNZ20/bSF0xy7kZi7gCpI0qPVoQ3v6Gi5HmSNl3lwM3hrEzsCVAqnMn0aqN04EWyotrYPuIfvvYazFuuqbiqSnjucAZvu5RQ7lDCuQ4Jf1RAZkIKOmMxFP+Oyi6Uz6btQKOPYHjubYKD7w0WzF1ktnqR/8FI/AAoIsxaCMU933lNACariRtBv9FtiKTT/0ncNL3PCaAiejQN/SW0x41CRSTEv0u132IGfr6QtDNp3FqioHDwWnQOIbyQ9htUHaC98TqzZ2xgbUUxqQNdGBqwlWbehnEiAEZx4DdCJWdxJZ5j/WrxkCEmPwHabpGqe4kpY08wNnc4DfbHiW16jQkqD9HnsLOUVdkpjNjbh9KwOxQMuoyAVUwMKJxAkCC16WkKTcD91mXCL33JyTGBLHnghp5pDKErvxGrex1rPGv5XBZOBPC76XE22u+hj28uz99IYZ6jBXXLBfARUaV5VbsJupxFu6eFyhwXpD4XxSB03LugDeTPvFaKXGsYn6HlgwOvEfedhsSKPZD9DsKYKzyyvS9e1V6ETDiEg0M3Ld0teLr0eCk11/LET0+g6dQweu1XNJrLRVAYgIf4ftf5NfHnvD+p9ail3b6dedrge+cU5tWHqMIraGuuM1H7LF5Bjff9poTzZMRHEsXHRNR8hErV+x7bWj8iIM9A1vRbhE3KxEuZBM5zwTmJ9uoO/klPxLvMRuakx7AJAhqFBm59Bq4D0Dsc4cboBCpcN1PdZz9YBzJLIodHxHny2rUX2T3jM5RGb9xbx+NidKS/45R7fUv0Naxevpc3Vi+n/0vvcO3dll5jK1ckUBZ5kHqDkrhEb5r9l8Ck8eSVlVHW9xmqguqIPJnCNxkvU/51Fz+rPEGqpt3mzM+Lf0am8GbkuQC6HFp6SbVl1G8h5ZyM60klfPPMn0Q5PQQu/WFaFcjt2f38AIRDh9DFlyM0VyEd1gLH08R1k2t/Nikt6JPSiQp1p0roYTJOzBUTSIiJs09f/pSg2ueIuf0VLgt6X/OJurVoL39A3KlvaPK+jdWpH0S/wt3MP6kp0uFs9SZ93Do+iqtDl/859PkMgHFqPdN0EOqvpyJpFx7CHJa79BMZX97jGW1eRt7dIDy8n6O0OBH/Af2gWPRbtcl1dGo7kVgkTNs3mXqfenEOvbwQTG24O6RiCYUldWBXCCnKvjzSVSp+h3vADf/ElWEJbWfRob/oKPZmwAPPkpZqPnKFl0pc8bx4hCSfP6ChBHwfIkuRSYddHcENTtg0zQSp/UWPxNBl2NQ+3AyAzxpUZHg1oBX0KG3tkPwlJH9J8ekzLPtzF3Zdaqbun4w6NaPXtexnO4/r56tpSL5J/BI3ajpGAQdB7gBSNS41TzI6Q4V2cAuPP7yJMkMfODgc2m7ByJOYzUGsWrOBDflu/H1xCj6pvRx3iLBWYMnoS5t9G4fGH2KRIK7sB7mPZcI1I8FBAmV+c5mk6sbLkAnx6yFsGeV//YbsJPgJAgpjAklzD4LCASbkgKWbcRtH87RrNtvLxhPg2ISbUzrgA1VHwM6DRXHDyQ9V8e6Nl1FYnPHRBvQa19zlS0EwE1QFSqsz/2mY0K/di8vfTuWx6ceZ/fB5Mi1Nveqfu+NH/akkDM9tJCGoCQERYOWqvMSV+buxdT7OsolXeMLnNlhHiiomzn0YOHwDnuNvMu/QCsyKa5TbeTyQRoeCfGfKCtS8cHEBeLRg/4gEJt0Wn8HKOlTDXbhj1RAxbhdxFncxOTqlECRKzPm/8sXza0k7P4Lk9F3klweBqQMsBpCJoAuzpIs69zoeTsnHw6l3EkgQBOrXzUF2I4LJQEdgBuwJRAmU6ydisrNHoXXAO7oWqdIdOu5C3WksoYto1fhjsnmQf7wvtyQWZj2vvgdoq1Avo1zbSLbDPq7/lUa9dzcvLW8XZT6dU+j03svHr36Mc6Mz6ju+XPQt4BVTCHiGYKsErW0aDknw2LDzyAtvUaxqpdVrFW49ThN/1KbyStcZUoKKkMokVLWkMuyB81otL+f3dQvoyO8kLHEH1o6LYPcBixoXsLfld/oqLUTketChjEcyVgC1N6i9Ga1dR/m1tTREBXGkL+Tbesvube1o5ZI+k4ZJx2ESPOp6H2QnIFDvXk+9ez0OneL+RyIIIhBUpqVOF8+ZM1bciwJhac9B2qBex/uW++La4E2oUzpqVQeCMOpe/RHFUOrWJpESn81PATvp05x4L9lnq6rG885dzH4+vDRkDukt04G/7x07edBaGsIF1lU68cLZELzS7icZAV7zv0BpizeZf0zkFZdWVo9Vs2maO9m1w2lVzWXoJW/y9Qm88NOrZLQsA8/lMLUSBAnxu5YSsTqX/fsGwZnB+E6/K67rA+eAUyLmlkoODLxAdK74/VXLeifFLrZX4zL1Kns1jRwLu80C5ReQ8iV3C7bhPf04+ysCSLP/iHFjlyFXzhOtGI4NxstjOQs73FE1KljZ/xIAP4AoYatwws+WQ0ZjNL+Ua1EcHEHesDp4OhoiVoI2CFnXbJ7qm8vCAx/RcsdAx1JPKNsp7pkTPmKkazGj/StY5LeeoQoQ2AC3vxATXzPa6BY0+E45hd2XM3lq3Qpm/bGee5kcwF2WQZJbPaajw6jwKUEYZIZz08gw+jC/qpJg4QMCC/1otVeKicQe5h6rx5GxbTmeMVpK4iZhrwsUfUeP9IWgBbzg00D3go285jeR2fzB6RoXUSXlRBohypWcHFXIgU6Yu/EkKqWKi6sFNl7/jHEhG/DRhUC/H8lpaeKtq6/gpB7MKvcY1FcehkGbEYRg4uvdyMiMp8m9HIXZjNVWAC4DwO8RpPZbiBh3kcN1foz9VqDRexadzwxEc/sF+rpO5XhMEKGeETRt/oRz+lTSpp8GjxE0y8NYkvQVjeGlTA1ZwjTKkFjaRZnvc48gacrAbt4bXCqdTVDfKt5z+Yhy6X5glKg4UrEHmw0inS/w0C8akkJmcXvRZfqrPXFq2867gZlor0VjZ7Oj1aGFPExkR58hLtgDvWDPl5Vf8eTNBMCGuzCEkqQHpKc1/myudkHWriJjWC6bRx9niGnVvbfy9wW/o7S4Mz+nP26G24SUvw+VH0HgbARBYOGvC7HvcqAouIhaj9peYCMneTu6vHA8cuwpS1Sh9x5+DzwgCODpfJHnxm7i0RA/dGsXczMqjWiFG0JPvdIoZ/7m+dwJvcPO6TtFRYR7D9gQ+qi66XzmE8Ib3LkU04F97UnYshQc4wlMzwIluCdOpM7xANHlX2CIeB6Vooant6dT0qgmKsCVLMBk67EVKPoFrEbW7x1K255UnBdfYYsvzNUViHP7XBuC1cDLy58hvWIib3Q3siZvAv8Mt4cWMX5SJY3lTMw/WOVO9Fo6By+CjmIajEWMfSedNdnuOJ4YAoOaRZWSB4gAjS6t5MTkkK/owmJwfrAVAh1CmHT4CSSCicZhv9NsE7/kw4OGczi5D7mKLub55nHX1Iqxoc89Zp9EkBBYHIhXtxshEy7yqHMDnaZyEbBmakEpNaO170LoMHMgI5TEkFbceQdy3gOfSfxdFEXcnjh8R7ewYnwarSwD6WwxlgsMVRtpBhqPObJdsoinX0wUwbH6KrBZed5xH6mz8gi7MAjXsv/pB/uw6iqL3CUccmvAR6JEgQiOmKe7xGifKhrVozgw0kK8z1nKLSKY3maDMbYS0hxgXYeF2vQYDF7NPD77cV6XngZEAG9tVi02mzMt2qsA6NV6vksQpfLV5gp+yR6OybQGjxoP6t3qKcfKcf9VNFYU0ueahAavbtwrf8TYkgy9tIf+W/63yn+Tff8t/2dFIpEQFxeHcOM3KP2LlqBVfF2zD5cGV2a5+DIs/nfkXZ/DkG3ANkiHg7VzAAh0vImbnRyIhpOjAQlH7Qbyx9trSMpMYuBnT9G02BHch4qdGRruTY4BT+/gDfd2TpnHg/9E8Xi5jvKGTtImnOLEtlE8umca9kOu9Rrvv8fHZaoZcOgZHnvtd4h/jwxDGj8dmYXK5Iax36tYOIvefjh2jokAuAm1tIeVoXC1Mjt8I11WD/C/em9x1qw9hSamAqT+2BlaUUscoPW2KBfoOZIvnjxM+oLD3Ng6hli1BQk2cEyErNeh4Spr1Y00VfhzqbSRm77fA9/1Gne9ZDeP7wPNUyG87FlLgyEC7m1tBKJlAg2CjTv90xmVXIBjdx6QQoVKx6kFPxJlGo1dsZrRLo1IyzbD8KOgC2HJwUepk/hzq+EAh0q7Oe/cwFiG3us3zrWNNp9aSlr96ZYn4mznDrUnAOjUHmH5jDdZc3ESc76fQ+G/h4U+AZZuBtWeZf+A7/jEsJlFCz5C7zlFXCiXbhXlPfp8Q7Haxu8FIbw/7TmMajm/cn9zrVeFM3f4aZzqlSzdsIzNbS788/AshPY7KK1OojFtkYGtBx5m/sPn8Rj/NIQto6aojtbx31Lc7U7KrgnIVGqQ2UPjFQBKJRHYnn0T56BWbox9maQkoP4imDvBMY7HvXYzPewOj18vZZn7ShBWQPgKKP0LfKfiLTsAEhjz9VIMSgPCYkEMgJftgIdKSW+8wrGHN1KoPYZNsLJcPkc8FiDpE7QtSlasW8G5IRfZPOook8xhPc+mhOXrl2NRKRAsRvZKLKgf6UPYsSHgmIBnVzsvuDXxdeujnCx9i1gdohdE3DvgOoisE3Oow8rrF54gvLadoHcQfZmkKtD403ZgGP2y47ij7uI370aeVIVBzOvgPR65qZnyIPi6RMffhxLY0aeGGfF3xWtS8icEL+C7Li3puybR4iZQ5ePPaJVE9Bnq/zO28p2sSdnCJ6V9cJFOw11TTtt4PfZCIZ678xl+PBuJNoWqrE+xT74fcMXcRWLrfDrPRpLnf5Maj0w6bW1QsA3KtmMZaWHNmYPI5RA7xEiTgwuCRAITcwCQHf2LCc+fpODKY4RX9yfWA/BogjFXQBdCd/ujmLL705V8HeEOPOf9Kl8FPgpBC0CuQ5W1m8eUa9l3bAa2mm4mr9wmygWGPwvAJWMk+edKCauV8lDrU4yMPwxbihkvqCi5spCaGg8uvyLgNv0dflGN4s2br0PtCfylTzPspgNSQxT/JCXxD1Yy5VpI3QNKN9Tbi4lpu0FlZwduJQFUhFSI78WVxTDsADKjDJ1Rh9D8KRvqDUSPGCk+XxfnwdBdFNa4oWyVY9XaKPPqwD66GU73CFOGLuGVqv5MPB9KfWIxo57YzDp7mXgve3yE7jb1obItgsG/KfnOfwpr3xH9nSzlXrz+6bMUxBfSkGrjeBcMPniICxPGQ/05Xtq7itZiOzrG1BHb0kFYZxoPOitJHjrOhdFnsT8yge3F0QTvsBA+8DswtVJQmkXmincoN9uILInHYtfDkKu/AM03aFaGMjfXgZJmNR9GHEZjNYnB+5oT4DkSZGoKjaOxuUjJGvsXUXGfgf4u2ImB40L7ND6c9wROHYOouLuFdudHeNZRlDMOC6ng8ssXaLYOoTwI/movvDdmgyaCfh2tNCutrF60j8ecO7hpGQ0H50JXGRqX1zk66k/c6twYsm4Fo4PO82CRWKxUnemLu6yb0PE1uDhdA4fJojcNcPBOXxI2aegcfIXoQdkobLPA2ApYQeGETabiSpucRHcDm6KPcVsxpFf74ZJ8mkIqETwbMNjgXwHdVlU8bz68FJXFmTk73Hlv98ss3NKAn7coj+QkbWb0ysN8Vaeiz5apuGmuQ1vP3Yp+lZf3lpBlK2NSmRMZx/vRkFwjBgd7AoTLxy6jsCmB/g1gaIbhUoX4C4DChcbTZoouxFLv3MAVqYUn6V1yTIE8WQtd1QsYffc3PP4jL7b3wmHCcv3os9zAcr9OuqwGkNkxs3YoudL9JPtVsPSzzfTVmcT3uXIPeE+i/OqTXAvLwb4rkUgZzLW7AD2AjyCnm3zYLv7+3LfPYFHrkX17AlTuYGrF286T6LKvUBkdeSdxGGpHG3RtFZMWEc9TfPNPSiccwOLRwLzYBk5LH2ZE4DLxXgbMQTC1s2/qTf5qv4lngxpPFx1wHxAhWDrp73OcXFsdv/tDvaUQDsbjIXdhZngt5xpzMWhTOSHtpMMQygvd1VDwLXXOHyO9AzpzEf1LX0BWMJufRpvh9vdQtoMGtrP95RW0BbtyIMKOR11MyH0Hw6jTALS/9iyXNg1hdrMSZ8VIAtduA8+dMFdkWkT69uXC+Aae7H+GX8IKqZfJIbxHWcBqIqx5Kd+/sYkF3y5jtnsBRrt8Hsze2nXIGL9rOpXelfwz7R+kUnHd9/21bynx+AZ7y0xCTwzmuMbA3NX3GZxJsk+osg2gwLWOrS/vwk2WwCKAOBFl3scKiiq4q1yCUSGjtbOcKe7DIO4dZHbeJOaPx6tI4N2H9gIwQyIVmamWbpQOgxmfXER6vg+t/4zHMaVeDCYEzgWp+Axvi8ijLrSLiT0Kbw8GYbAYcD+7Fe+SUmASw2eegH514DWOyYekeF+qRR3cyGN+8yiz9QVW3jt0hecerlxJpUHSwnsu8A8l9+q6zB38Nv89BJvAey2RBDjkImEN+D0M9eeQKs2YlRrcOudTfX4iqYOU4jyX/S4EPXaPueuurmK860baO1RQoxX9vcKW0+mYjFMhrKwW5z1Zd2/jyjaZN9+M249jxwDab/9NgjNgD+4dOzg76Raziow0XYun0ZaDiyJBlLGydGEvN1Dr1YGjFTY+sgNsAlseaHec01VSVlRRZJPytlcEnRp/MLXD3mAIeQLcnqfeL5HuuyUczR3P1BcTIeF9KN0GnqP4s01Hq6QBOAA2mMtm6CwV1ycKRx5RXGBdEKw6DKqm0whCWq/zGqZYzkP9T7Dx4jMotUYkbdnglMjxljep/WgtIQMduTLMTF1AI0Ga+0meYxkvcuKUEV3/Zq5qYYBcK24Eethee8JDaXvxKhG/Tsel9jrOaybCuGtgaMDeGk2GtA8naxLouuZHmPKmyGasEGVUrc5j+fzdhQQUBLLEp4LC6WbQhsKwAyIzC2iSzOL2nU5ca4wITm3izegpNmUqS396B4d0aG+Gz/NH8e0sf3CMwd16A0ezHS4+A/jys9HcNi0R9wyxb6Kxmfnm2zW0F2rRrf6Un3xrOWkcAN3J0H4XhbWDsUfGYpaZ6bfqD9xc2ntdS71Eh9fwK9R2SfHaORFdapPIRr8wC/p8T7GlAbnEkwvSLA7eDWdBgB5/+0gx2dd8E8E+HmmhjblbRxNcHI3jpNd6tT/YLod+6/8hqcpMmM6Ig00EyDh2n2WubTslBWPZMuIGyR53xLBRwXfQmodACuueWoenfhDDqybQ6RsmBhgTP4aIlYRbO0mo8WaXdwkLH9nEaaMbNHjC6XHgnELe0WiKTvSj/7AAfF00uCg0sNNNnOfGZWKygdUiITw/lG7df0TagBqn2TQlZHLQeoBr2W7MSnDsVd8/1UZtQB4Paav5qMVMg50nAB+4HCXfCAtLGvmSXA5Uu7LfPlL05esqZ93t4xQVQWXIh+yP2ccIhyd6tftVporBFwOw9zISH1JCsJ3uXp215SJvz53HmtzHmWMPJmsrSGRw+XFozcVeeIJAtzasQ88zc3A+FxwCIHS/yOzu8X56Y8w1ikdeYHQD+JYOofyBvgUE9lgMeHjWsGrhUWq95WDnjR4tz1yfgNa6g6JYf44K/ZipiWOIswzi1tAu6cPOoDh0WhkZdh9hlnYwW7LyXrvl0ghev+WAY4sni86kkTmgSGRdZ78DYU+xwL6cl7T2/PH3cGJy4ljz1hrUsma4sw1Ndx8mhl3G3c2M8p9hSBu60QR2odLfgOJc8J2CwarALz2J/PACjgSVMFE/vdc1/bB0FupSN0qSr2OYehJ388uMlcipsznz2OcrEZwlKBpT8dH68EjXbmAaWC04qku5MGkOrY4thN4Jpdup9/zaYivE/04Tnq6BnBlewlXrIegRyRUEAVMo7OqAz/d5YaUPAUNvivOc0pV2Oz/WTVyDZ200nV/Owxhbx/dz77ev6CxkaV4ylkvJDFjyOfFRlxCEinv1S6ftYvXQTGw2T/ZZCqhxue8l61q7C3N+A0UhDjydN5rm8E42PzBurd0WFL4GFLsm4ZA1jM4hvZ/taNteLHV9qbiWQKVPJRK5HY6qeoYGbGdn61PkXeiLwV7L+2d34xYYSB8QE6SAQWlHe0QhVzVV6ExKTNpnxAB4TxCczja2+xYyrv9l4rLjWF3dnxsP9F0si+atxC/v/dtOYoSzU1G4TeYhdSk6Py2xDOLV3CyeGwsRhY/39BvA+z+Ox6HWA/tXPkVGT3L1jBiMjnA7xa51nWjtLYSEleHvUw4ufcHcBR2F/FEey9e3BHIT/iQzTgJtUYwuEe0iMNTzXXMyq6qqeMPVykxHK20ylTj/uwwAqYpsWX/mVGwkOjqHmMBa8i1hvWTdQ2o/4davsQykL9cTZcTVvwkyLTtbEhEKFKgcrMzdMh2nPjl4tuzh3zXXsc4wQvRQo9bQHZCC1gloPiru4e6so8ESRLsAp9uz2JD5KXH2qYwv+h2AOsV4lJavaLaCNmcO0vpz2IR2dt9exe7bq3g1HnzMpVx/aycvZawmY+xtiuUJmIOfRibIEAR4rk8GHf1OIFNEcMwnlygF0OYAEjnnzbGsqj7AUKkH6oRkEsKuYNUMhkkFBOZkMV1nwd3Vl7pwd/5om41X/K9EhAcirblBZZc3SqMnFrOFhDIJUfrp3LiyBBovIwB2ge9Q3F3LbNlr+KhewYevga9EpR47H4qMC6j4ogsHpNzCi2v5zzBAAItUfJb1X8/Cb7wzx2Z0YbIkIOhiwd4ZReUpRoZX893rHxDd+Dq5su+Z4thf9FuLehFUrgTfCWDCnlROzskiP4J7UqZJLunsjDDzUk07bTEeXBj0Gd7OTizv2d+Fqm4SGF5DRrOerIe34dh7umBX3RROHrASUhjC+2++grMkgI+5749+uXYI4SVQY1dF92sfYJVYmasR4dCCAFM9aqgcd4KwwyN56dJw1I/ngMd9LzYJEsxyMyGhVdi7wF3n5wiVZ2ONeB75hYeZu2Uu1b4Sjo4CbD3fPDtPYlw20bbnFiEBJl48sQ59XDSsuiMCbDWBbNSUEND3KoNK5lF8cTgdT7+L1u8+uFaqKabVeS9dJfV0qbswWt8FxxhoDMGEmqNjj6LSqziqj6LNxQTchQG/APDdupXsI59RNX6cvjCEvDCBh9ru9MTYHgKHGC5MuMb2mDuU3PCnTdr7oqoFMwNn3iKr3J6nt3nSGJMlyoTefJ3X+v3O8pqbuI4bzvGW4ZSUwPie0/a2FvDCpTTqKz3o7neRYDnkYYGRJ8DQyNAtL/Kxwo3V7c44/ziFiyMzmfrkKTC3g1TNHS8p55f/xpz0BZz/OoaEN+qh/pKY/A9/mq+kArdLI3hrzN/YZE5M7RiHn+Eu5H0CgXO5bfKjwpjH4HN2DGYF0ZIQHgQHhFgbOP3UWzQm3GDetF28ItnEI4CP9Bjy1g7KckvYF7mEbfIK3jC8dO+4z5pTaf4VfJIU3P55BpdHXOaxl2xE2LfyrwDqhU8u0B1vZfVfq/ln2j8UhhbycysscYCcysFgMhKcVcvkm8v59MVP6dJ0IZVImamT4Lruc1489wsVGd64hHny/2r5N98hkUj+f//x/x/Kf5N9/y3/p8VoNKKKfhVsFpRtuSTeSCT5ejIG9S0yXYNwm1wLvfhiYNdWg9uBXSRNKQD+EpM/gANxDFaBU8cAtPpIkh2ngasPzBalfiLz3uO9oN2sr3biZUk7I0JcRL+TgLmgC6NJqef5hPP46Sp4QaHmDj69+pV3XyP5wjFoMdHq7EG34A1ZbyILP8Gk/ZNQd2lpnKLlw3ETMZpiIEjk2A3R+tPv1RV8f/4nvlqThjLOn7eXIiJ4rWYMwds5HQAjik7icfEa0S+PgDvrRU+SsKd4tUFGtcXMnqFHAbgsVYPSU5Tlk+vYsTsN9blYgtZ8Q5Szhf9M9tkMRtQtDdiwIdd1igmPf4sgMCIqE3P0eR7WwkovyDKJGwSdxISzBDSNSiZtns/uzjMsmnFKRNdrgzEOLCAw9SKyzdMZcGUA+cN7Y12H+5RgnHuNKQXn8LB6Yi9Xi5KVHUXkVGmZVQZdyhKWj0tlTLgdtORC+lMQ+xZWixtX/kmiSdhFWFML45QCe0aJXmx4TwCpmr2Z4XT/PZI+S2uwubT06jvRox+jbjTRqjzPr4tfQGPvI6IFW7LpL/+T8ls+WNoNdKTH0zHyBh75X8GAX5Hf3c41f3i3voudfU2YsWOqxYBL8heg9sNytZP8qFu0Odnzw/h42iUxIH9N9P1wHchl/VVuG++gkhhJ9DzGbetDIk0/+XMAXrKV03VsMuc0euxjC5DZOsVEDIDCmbTuj3kxORPlXRhpB0HyJpEt0yOps9T7Om0TKmhQtOHc6IzgIt7LUGsuWcm5FJv9yQrJ4gm3Sjxb1WKgovYEYS0xnPxmPv59bNjkpdDmJHqDXV0KyV+xoH07BRmtUHmd8spqIhRx0LEVLN2g9iEn9hY1ntXo2nU4ufUY7+V+AAFzwP9dPj0dSf3J/owqDGKH7Boznv4Mrj4pSisEL8AzxImoudVsrVuLSaXDTT8TGk5B9TEcDGVM7Hbi9w4ZE8LWMy70R9qtjZD7OguGnuXp0itEOsvxLNqH0UMG9MBzLd2M9inliJcjjkYDD3U542RrBJ0oGWGr2QptgE3CVecJYBJ40B3mlNmftVmTiKlMp//Z/QQ6qkHxMhRtBJd+HLV6UvPWWppkZvBBAAEAAElEQVQuJ/DY74/xzqrv4eQ+GLwFVG5YdPEc2fcQ1Boojb7NrzYDn/rPvJdAklokzPh7Bno7K5V6D8pTIgiKKUZi62bgwzfZkf0KNq7zkSvAcXBaBbUnsApKaDWg6KphysktWGQKXF51gR6Ghpt+OXZnPdH2beX3t7/E3TxcZCsGPQZKV+bkOmG3bxW3kuCyeRB+aeFQ9rp40ko3wvJ9ST0xCYA//cpYMqVnAeYmJopMaisfvvohakMUoWbIMzowQLi/VJhzpx6n+mqCEsMp8b7J0o6X2PDwdOZWTeaflJt0qpMZ88kI+smTyB8opyXkKxxtWeQVl9D3ahxl/Stxsbkgd5ZBwxU4OgA7TQg/B3dx09DF2vN6Wh1aRXCF0g1sFnTdFzmS0MwblfbIv3iY0H7FouTZMXHMcX1u4fLr50yvruAaUPXEMZ6U/Qlug3qSfRrS3T8hI+0ugwe8yXsuiMnsnnslEQQ6dB34VjThd+IDNirO86xURLGt8b0Dki4WXVTw7gtP05FazpweZLUgt6O0XYnSaMcl1zasujpiJTpRarNkC831nkzSwC0HFzxfq0Ovuu+LB3DdlEj9n0F0xNUw75UfSCdOfOdassHOmzZ/K78t+I1nIxp4MqCDs1YD/O0oHjxkBxJCWbphKRKJwPp5kcQM69srYBAoKeTpN3Yy9mQwM49GsGj+YB5CDCy1ObRhsMoo9r1FiakNf+1SFvRcjzxpX55w+gqb1o7y9R9z3SwH7U/QfyO49MW98Bte2vESx0eeYO3qtYzyflpE2V5ZDAFzeL7akXpTEbFZ3likHgiPIM5FAEpn/urQ4m3fwi+P/YZS7sF/lvoqD2y7ZzK51JuRUwchmPYB9xNBKr0K5yZnviyx58tGP/JmiFovEquAfas9gtRGprKDILkcV30VJHwAwHafEnzlsOB4ALHXAtgxwYq5eRAzIt4EJIw+OpqoW1EQWUiEox4BMwQ/CZHP43Y1g/dKLnC7fiD7/xxB8Ng2oubdEs+57w+gEEjvJ7LpMtthjNpDRJv2IE6F7iaqLNB2LgX1NQnRy7byYLKvtimTfhXfQ62O960ReIbpeUqWjR2wPA0+9LtDUmsq0Y1uDHWsFVlN/rPo7PbhkW0zkQhqLgx4CpO8i3r9ZbwLvoeOu2RKjnFlQBFGRR6+q1/mvdGZvPvHM/f6/bpZS0BXN1K7Nqz2epoEF7i6XPTkm15PnL0ShYsXhbXL2GVOYeQ4HeS9Ja5RHq7ltN5Ge4uUE5N+YL4H1Fh7SyjO989FMzsLto7lk4IUvA0XMRrH4yWpZcXOGXQoPPhl1lZsgo2Z9h/eO87Xs46hn3/FF/n96VZ2Y7HZRGndzBfAfybuem/iTXX8nlTNobEHeUk9SExqZb+D3GssSVPDKalzuteeIAj3mKlC6lm8p52g6XwEBb/OQVBPY7JUAafGiNLSiR9SZ5ZTY5Cx6OoIYnylCMZmULmKjVUd5KHh33Pk0lyqPPtzwXk0qTNFyU/bpsUMPx1Ntb6Tq1fC8R3RW/rudv8s2vpdoexMfx5NDyfmqb736iQ2I0+5mckxQGLnXvq5HwThXTGJbu6gqeMNnknZxubr43C4Mho7334i8rd0CwhS+lo7GHZ+DDV1jgz3baFselWPXDzgEIsg88UzV5wh7L0beDv0BPD5/f57koVSs4DEZACbAhDw9BrN7oMNuBYu5krsBqT99xNt5w/e42FiHqEZ7fxj6yDTMIqPlTP+x/tcJXNBFnsUu9JxLNRdIEuYILL2vMaDUwLdkbvodi9ki3EnMrMNz4Y+JI9dDVGiFGJa3iJCjnYjWA2cGXZGbPRgnBgQnlaFTpXMxVMQctkJV++/sa86BjEf3Ou/7JYf+9fNwzJjN19OPQnXAPsoJMqHqQpLJSTKm6Ccq3x3G0Yuuz/ukPaJtBcdIswlFN+SmSgHe/NgsW/4naYScAnfRJ/ETOzb94Kz+J1WVNeS/ms/DHWudHz4MTUKKag3waDNoHSl2ajkiPsxhhYE4tapxUiVyETL/0oMmsW+QXrJckrX/kQkv+PidwSMB0VGO2Cn8EaZE4umOYd333wXhcSRbx1/go5ivrnrS8a2RZTP7WJH3is4hfQhUl8BewIh8gXu1qSh7Sjiul7JC/Uw1CMYbn8Ftz7Fz+F19kw/QKnVwjpdCUp9bxZbniSZr4d9SfK1ZDz2TSE/tIHZQo9Eadl2Tjglc+rJL5n6z1QCjwTSPBwYvFVUM5HrkDQYca/xoNajFrlqBYP+I292hP6833YTfy0c8IHjZlEXVGEsov6SEpR3MPeBd5ogJOEc8+unQsG3BAqvsz6qhU01jZRXrSDI9ZTY4I1XoK2Ab7OHMuTnIPLHH0HiBhMcOklr6gF0eo3nm64zxKs6icm+TLh/HhJzLP96MmJq46VzUfiWuTH3XCq1yYX8j+LgSZHfBKKOXKD1ih7FYtde1Ydke7hUb0TrcBSD0nCPZbGxuT9Vm+wJd3bhRoA319Q6kNmJLAlBzonO42jMbXg0v8D4oi7mz+stB3utTw7ZcRmM7foei7ovyQoPOBgPLdlkBXzBHzVww/w3ed9Npj68g5MLjCIjzM6HgWVXeUHpzCKNnl3ut+iS9RMTEnsCRNBan6+JVZlo3DOU1w4MZe+SzF59SwSBXdN3IbNKiWn7DnchDDx+xa5kM/YVuxh8bijpIRlkBX6KX9sL4Pw5OCdTcOMm52KSUZk9mdj+LCqhEVc713sA2Pa9f/L8V89zJ7ye4CeMBDgMFqWKJ2SD0pX+115DyPdFFqBhdPwRXghSUKxfDunb8NRNJ1FWyfWNSdQVeoDbKAo0Q/HKiwBLM0y+i5NEz4C9D5EXm0NJUMm94Py/RS7bwM1RcDl8FA3Wdl4iCKwWPKQdVAbXYW8fgyloGmPjn0VQ9xEPsuh5WBmNff+FbMh5gvnb5pM4obfH4RRdJp37ptDgXcVxzWkMcY696n/aNozaI4NJk9mwSCLwfqpHLSRsBUK3QLt9O/btrThmhOHkUX1Pah5A3lXGkklX+cihhcs/j+BUmoZ37+cFkMgk6GP6EPzPLW5emcKIT+8n+/oO+QBrnI0Vt/sw4eBgilXFvcY1/3QKgTcdKfcrpSSwiIfkL/eqX7vtGdxzW7kxdjY2mTtKqZKvr2ykr/d+GjVxpKcNplvIRHr7PF5KWGrwFSWanZPJlsXyeOU2EPSg0LNc6izOReV/Q+reewCWBtcGNF0aXCq0vfr+95s1vCSE1wzuNA29BIGPoreK334bViqFK9Q61FFnSITQ0WBsweI6ifzQVpR+Nj46NBVPOxPCc4LoxVZ/gS5FPAUJHjR5HSJx8gcUdoqSduS8C02ZtBY/SvRfUajn7+CFEdlkSPNFgErI4+AxEvO655i+8Q0uu8Hp0BF8P0cnKhM5RIFEjiAITP1nKh3aDl4YfZwnLd/yoFPWHpMTltAa2iOz2DzxBhp9FzjG8Z7+MKrf36Q+3sbVKUf5LKoWO5Mn7AuH9juYAhbz3TMbGWB5kYOBblR1x4LnDkjdDfaRPHLgLNKCFloccomsyaFupA2aroFUTatyAOs3jCf0bih/vnINX1UgUokVraIFk0XJmk+NKOqfps/FPjhqBd7yL+azu8FI2nOhYgsSyUxqzM6gtuAoCBSZoMHah6HjfgBTK6p6CU9ueJKWfgqUKY4sHvkBXV1fg/14+ggruJPpTvuAvqyf8ww2wcKSHrnYRNcInsg9THuFGe2dPby25y1OzG4S/cx0YXyincFfe/4hqFlB2/4/2ekznWkrdotvte9k8J3MqDeCSYlNRtA+TrfrBCgWE2JmuRcnihbQNmAMSYsd6b9lDpEORwmqeAdcn8LlzAj+8IZ4k4kug41GbSP9tdVw66AITu8s5puU2/xpNGHX6s3xoj4YBx4CPkVv1nC5xJXuekesdlZyjPBxmzvLnRIAMGGH26zDlLe48JEORquh6wHKlkSQcGLkCS4PysBHBo6CpdezrzfrcDrfj+TiIHZO34lV1lte+v2AIjIcS9naYceLyeVo2rKAuWKlsZVxrRoM6SNpi85nq7qKUN/BkPoVQu1ZLvnB7joP/MtVqDqnEDvuKprLI2Hwz2DyJiRzJ9T5IDWb8Vfnw8GeTbFUTVlQDbnh53nu6EzMt7rY3FrC0qd69m9SJSnZOtxrjSzbuoyWQelIF3ZB/GsQ8xqKvRtY6wqHbzlz8vvZ2E26wkSAc9NBG4xgE9B0aLiReIPcmFwGOiwTvfqy3xbVY6ZVcaNbTsVdXybtnM6Yp//udU0EjEwb/hMtB2eTujmVMx4i+BuFE40dAUgM3VjNVmw28Zvw7+2QClZSpp1mR50vhVtHs1jSxtJvQsT1mMKRyjZv1r+8Ak1iA8UJU2jQPs7UmX7iHK1wYspBJXlVFip5B1urgvPNnQyqOgiFP4FTAuvPJeC8LxXnpRvxDr+C1VAHTkkiu0+m40hnX3zfiyCoWUu3bydNppM8mOwr12bQ4L2PZldRoUTSs3bZpv+Tmh//QuO6H9lMKXKb/N6carNBW6cj4ddC6ArqYvuM7Rh9jNCYzivLnuLToF/R/1FK5dVKHPoqaXRpxKgQ7ZieqBN/3vi1GF/X37g0sJ1a51xMchMeUkip3Y5WO4Jfb31BmyGNljAPHB/A6v+/WIxGI6r/lHr5Xyr/Tfb9t/yfFavVSn5+PnFBWqQF32EXsJgXp13mVP88zuZEoFe1ECosJ+T0ZKjaj9z1JwadKUSPFGNUDZc0ZhHZGfokqP1JuHiVla3B7E7J4YXF67km2EB4Fno2opcK82n7dgrWkEL29kkndYgz5H8NulDwHImt5CR5RsjzLeEIMLZzGiseGK+yawXudn24GCylzetrYu1nMS15P+41Zxg15Tx7m+yRdnVQVB6Fe1Ic/0oqN8mHseLvHNqMzQTVHKfQrxH0ybBbFPwSBHGD42HLR1rXjIPhGvhOEZN9lm7+KfRA12KHLLAEs9wsTtTmLvCeBA5RHPfrQDv8JG86diFX/k/UgKz8DNHnL3LDmMQQfRPrQvv0qn+ysY3SEoFKp+2k3Wpi/atTAXhXfYKyM2l8JzigmbeH8TFV2JK/RlCLSdDXav2okzbQb1AWU6Ye5Yrd4F7t/ouEXxj1PdOCt9JsqgWvVHBPxVB3koFfLqfd3sycF9dhkjiD/CHwmwYOUeS0p5BztAG1XzuOwRIMzh2id930hnvtv+VVR+C4Q8y5kMpwx0u9+7Z0Edd2gM5OeyLDNjM+7JgY6BCkpCm8OH5rNJkDZWgnv0ewhxXaveDkWKSaBbyxOZWaNgdupb2KSdZCg348IZHPA+B3+U36Pfs331Y4knEzBL1nEfGOcTD0b5AouF62n6QNffHo1PIRr2JcryP+gXGdrvVHdToF92lHeOWRS+SY2yH5CzFwK9dyVwjj9Hpvlt4K5JFHzuAyIBsG3/dHiHaxoZ12nQufzWDa7smM+OUv4HOSzWd5dPEVAvNGU29tROVTjcTmB0O2Q81JvirspnjKHhy7Ixh29DfcPdPBcROEPQVqbwYXfoLmTBC5qUvRR3oy1BGIcBcldFWeHB+bx+EueL4ergowfUQ5IflPQftdBLmOCxnhJBcGIVl8kXVJV6FIBt11kLoPgD36XWS2gVzdjb2sgW636agbdoDcnjuej/DDDFcGNjtzUWbjQupcXvRZCZ1FZAtP0eYehtXXwqjX03FzPQb/2hLLHVgQcIe6BZnM2D0Lt2uTidqQDZ4LoeECzZ3l9C0qpkuv5JNn5oBNgiBY7kkF3eEb/G9loNQ34htuQ+2oBqsR7v4IhiYqCWKPpJPRBiXBNV4IFgFac8RAyow2jjeX89nMr5Ga7TB5VfKwnRqOD4PgxyHlC2baXURY1sJrHV4UhJ7ge0sUdaPO0dTlxltbwtEbW1E1ZFKsD8I3rBx54seQ/Blb/vqLn59/BlfJUB76pBN7904E2/tQ+AdI7Tgm6JEs3M4lpYGKINjTdluUMj41DroqSIuopsNgoT5rFG41+7EaY0DtL8oNOiWwZvBdjP41fHQ3gA57GXJt+H0JGCDGrglPOyO1F925cXk0J8eryHjAs8+33J7RR/qze/5ZisKzSGEyCAKRno10zvuH+qurcOpqRSNv5NKkUVAIzLWRPiCC9JRzDKtcxDO7nqH00QzRY8s9FVPIyyz74xE69UoaxlxmtGs17qY4KK2A9OVofN/jmSI1N9t1zB5hReLjBbYeOR/nvijUgbR6WTDJDlPldobhTnqIeklkjfSUDXaBdEVWUlau43ijhEszw+7V+Xdk8LXMnp3dCrrs23FWmcBiBKmCT6pD6egsw9ytoMZmodx8HzkvsVmYs3safgVhfPnin+zVS9gpcwRdCMS9ic/hlTy6fSYXpBL2u36Pjw+95O0EmcBvC36jQ9vBtqvTeCTKib531osJDZf++DlGYu9Ywrtd8NZtLXtjPUWwQ9VBQEBpqmfq4Fuc3zuU4l9l+I7oy4Plki2N50t2knY0jbhmJ6SPid8HlaWRJwUNOQYFF0dmI2huMMv6qIiCFSQEqhMJrHgfk6qKVU7r6DQ5MlblBm5DwWYi0L4Zi7eJdvs2OrWdIEVEVlYfgdZbZJ/xIzkznsQGN3Q+tUitRjEQErUKJAqOj9hHl6IEOwEU/9P+CHmnQMitKKwI3Lg+gIQHnj+ADx49Qv34s2hfeZ760AGoPlODzcpsYz2mL1/gTv92vtK7MiLUBOemid9KpQvXLsZxNSOIYbJwFDfAaaQPvxQuYevN1QjYeMbpe24rtXjMPsZjbgZyok4Re304dFVikzxG7plIzG5tXB50nYOuUh51ShTBQp3FjFFm8Y4/zLvlQGOTDkmwAKY2SF8BnqMQAuby2PFw5m6fTK3UzOLKIG4+cE6dEg1nLkehaHRGmhFP1TODYOAPUL4LpzxvDFY5Uen+pJxdwfKP18EV8UnSufTF02CHQZAxc9cUTJpOeNkG1m5w6U9rcz3d0/6gqN4Jh64uPoi9A7e/FD3mvMZS7VfNnmd/uzeOp6Xrwa5WZE111zPNYx5dChu/rFnCaccBjHpRKXruCBKQabgs9Kd8XTcx9m2MGXuUx9wHiwy8npKsi8FryAH2daYzLTGXRlsg+fn5jJWeJLcjkZtSGXUeojfGg+y5d5tHc7ZrAx5GmKH3xKwxiWyw8p1QvhP/0nm0HAkj7YlWFvhAAYXglCqyDu28WZI4jIyqMch/XIizVyPCC4jydlYzgsyHMfmJyO1lDEt7isRIezA0Qt2ZHolw+OzcOIb9GUUAED4iA4mp5V6yr9gskJlyi+P+P6C99jwyBdB0HaoPExBWwU9P7cdeEoujg55OV3cWP3A9Vtc60C3XM63In9AbMXQvU9+rk9oMvG1x5q8GO86tD+WQ5zLWPmNCHvsmxK3BuOllLq1ZRIDShNSQQ2T3FXDqYUYJAr62ZoqOp6ECLmVB20PtkPQZFHwL9uFI2htJu56CSW6i3zPVqFW9/QTDu8+y99oIrh0YgmD9GIdxxcBvWBz7cmRzE9rmy6Q+8QmfRX5Hg7pSlMi69Rk6yzwyPvBCHXuLP0KHIXfszQbLNEXweiOMvBHGlx8PwGnhCIas8IHUXQCMdXVlgF8jR0shUAYKusHQJN4TjR/L3MrIVrjQ1aBjep3//Yb1YsDa5vAl139eQ31ALRHP/kaB6YUeOV2x/K400D2wAJNnHp82wdLI8Tg0X2CIdiWmiXF0uQwi6u8NtPlGAffln53TgsjjMQJLb6K52EFKb6wGkrYWHGrrqUu6S3DSNZpM0wipvwDVRxBcl3BVvRBrQDXb291RoOMFmUZkjwKJxk6sPl+xb+UnZPT9k+3tEWL0pfHKPVZLYdslZIDUv5IZST3sJhx73hUL9fGDKEr0BMk74ma+6DeoPkKmcxSmQRlYu5I4unMCyfOiGCWXgZ0PyDRUJk/GaATtuRNclvqQlOgLrZsA0Flr2D68kOfq4WqL2NODJVIzmMSiTZQF/8inL37KYPeF4D4Mol8Wvw/XRPWODm0HzU7N4n5FqoAeJptAN7OPjkdd48aAd15E2ZbGg35p1VY3skvdmLdtNm9p5Az8WVQhSZelsHHcKRztFPS9GIPO5ovT0BIQ5KDywNnQyFiDA8cNNmqBbbfWMGB8ouh7U/QLXS5zOTbqGI4tDjz6+2O0z5GI98JzNKh9uZmyjit9z/HK+jfoyE0k0ApMv+/ZE3E7lJRrCaTNOgU+vRN5AGfbPuJuypcIrY/j1LEIF7v/+TcSs56VX69EP+AaglVMoR7VR5Ga4U6Qdwf9jj7JlUE3xe+G2h88hpNQfZWwwwNp6V/EIGEB3uUJwOv32pzn30CD0EHx5TAam+N4XMo9OXqbIEX12ZNMrndDbpZzSZsjggOKfoOo1RSatGSsGooqOZM3puxlupsfdFeLgE5HURJ89KUoxlyNxNGphVhdS6/zEQSBiRqoN9s4nLmMkQFAcjyZbQJpcZe4ZPDBs8yFV2rHYpxwX8JcY6pkU0FfrjU5MrnfbQb5bqdRfv8ZcLOVUzfiIhadHTOG/MrdtmEgefLemC7WfUj11xuIm3yTQRPFvd/dhnHEux6mwXkF75R9T3ydHd7vO5BxayVtrWBR+iDtagaVO9PcrjLipRI+O5jMPzdTsY3YDg9AAf8N8Gra5HTK7MTErKGBX5x+4tsl8E9bKE7p/nxxdzcbg0ELIFWxvWMYh+qyKFf9Q9zcEzT7Nfe6Xt6qZuJe3MQP2yZz67u5BGyc36t+s9WJMO8WrvW5S71LFvFOa0hI/R7UPvhfXE2eqx3PdXTy0oZPkMnNIqu7J9nnGjSDT20x5NkX0zH2W+o8TLz7YOOnJ/FKYg1vXd5Ih70ffxWlMbCn6pPuaEoUJ6gKbOSr577CQzmo17gcG3UkZCXg27+U5f1vk6Fo61V/YOAGtFFSVoXWMcz3AhLreY4XPc7xosfxTjvIkeQF+Jc4Mv/gfK467IMWR7gmAo4klvd48903yYvO48TIEwiBiOuTrjKo3IdC5s2OY9PIvSAmSOI8e/sF+psL+OD3xZiK/bgAuA8ohcqdBNkkvL19Pp3VnjT5/MGEPBcqxuVC4CsQOAdtlYVAp2V0WORUX15Dk9Ik6hKNFtU3LBegxcuZCtcLPFwNEQSypu68CCgKeYL2oE62ztpKQmAVcqFnlnSIFn8AiYOB2wk3ibyZgMRyFxgqtt2zlvWzFSMrC6TaoUX8+//Q5TugbSFz/g6G2cFBAQIC9tIvpT+Svz2ZuGg/HwvjOeqbTbSkg7NeKwm8KYK8w5RNuCug8OZJ1v8wB+WUHGKechVZT/x/7P1lcF1V/8YPf/bxnOTE3b3xtkmqqbs7pUKpQaFQCgVaKFqguBRtcadAjQp1t6SWtI007u7JiR0//xc7RLifmefNM//7mfnda6bDDCtrrb33WXvtr1zf64IFhm2k3orAyd0T2zobSqbVw5i/wGpBesGEWWrGKDeS/8J4TgUV4z+jjp0Lxe/R5vOHWTbyKnsT02n3r+SSxQ8KwBqyDox1CLWwPHM/YxJN/Fr6KU9md/FUUjSjTyTC6AN40oSd3gaLSc8vzi/wV5GU7aHO3AccyJ+M8nonlqkFIJjZ6wlD7sVC0E1ovstcPuPyrbHUBVmpiMvEy8kWhogVicZL2xl9eTSaDg16DymVBl/+MiXTU69btp9NfnVsuW8/wzJfxr6PDK3CUIinnYTO2OG4Rjbgeu8yYUkXsK38DCo/E+fXy9GXuOPcZUZj0fB9WwKPrNsrvnvGVly9zUgTNfju9+LKsUiChpwUl9XHsf+7WfjVe1C1vJ3oomjM0fKetauMkYytAEdBhebwFM771vJSnyLjUbIYRvE5JSoFA3/9i4CkjJ4+QQCFxIB3rScROZGsavBjZnBDDwBAEGBlUSSVqgxyJ5/ilhz2en2N1z8TtOXxkknC3kuj4dJofhvWyAtfirEVCaBRGcl5/T1UXz9F/O14Xt70OoLWASoOssbpRT6Y8Tp63wlkVgUT7ZsFMa+CfTi4DOPr7MHcNUHTWBfeXv0IxfI6oLfiMDpzAJPOjkYWm8Pjo3JpN/fSQztaa3E/Mo3qdgn7F+zH4OzBFoBy0VaM7VxC/AebuT3sOp0Lj+OtagSvpSKwPOxxECQ8eimWxOsDCJp9mlSP1p5zDqDBaGbC5efpcj/JwhfO4K0SLcVct9m8sruYqORvORamI7D+HB0BfgiCmCyslkaz0qYED39H7jvihVnSX0e+SZ1E9ogTCB6hrI7eQ6TnXdD91HM+z3Zbw5bJ+xibu5C9s37kPps3IOZlUTbAfTwHfXbhOP8Ac/2beN0TUk21IjhA3wA2HgiCgF6p59LoS5ybeI7VXV8ztc/6CoUNJfGLceg8yCNpw3EeIH5vz3d8R+7sP4i2m87qH+bhITcS/sZxbpOA1Qoyfy3fvvAOsxNXM6LkHgapHakGCX9cXI30cBUm5wDu2xbJIWs7P6z5gX+3tMEpSBQxlAe1Um0vqiO6SyG64Sgt6mFcuPMAFqkC1P9JPfp/qfXkO2JjkUql/98H/P+4/S/Z97/2/wfNAiN306EMwzX/O+ROdVxyacUgrxcdRbckqPqbcl094QlZXFO088HI8zibYnkLYOhXAJQc2sgvb8fTNLqSq87umANDoPAHuL4Gkv7ggqM3JfMPMOfwHKY1+uP0sgsUHxI/EryGVDCz9K/5tCt1HJlxvD9tE3CdOCqe/YIbJcsIK+6iA0dI20SY3BOvoUYKs56laN9rfPf3PFbsuU1EpUiHYpFoMF+vxuRZz45XXsPBFAGN3dpK9gPwy3mPaXUN3Ha8n6BVnuh8E8HTrycAP3jTLIZfG0JVbD5yt05UD8rEhERbIUQ8icZ3Gpluv7HmrhuCnQel/368TtuYeL8E/pyEW4ULkoX9E4KhHX9jW25AuNRCg18CznYiernT8gRpBzuIG+pJ8n2/Eaq2Z65bEshFKiHXGgf8qxKRRrYhc2zDau7/vOZXeyIUx3Ct1YcKn0dYNE4icscLEiRYaHNupkMt8MLF/QQHK3jR1k/U/ag6ztm6cr574lNGJ4/ksV2PoX2lvx4aQLt9Fw2uJlpOTOeniKI++HSQ6ssJrjxFTY0PS5c8g79rLnSYQO7Al+ENJK8rxsdpFXMdzAgSoCkNzF24u/tTd2cA9u32PK/SEaU2IFvSCunboHwvrtYkpqphv1bNva9nc2NiDg+CSPfjMxOLzIVGl0aanZrR2mtRmoCry0Xk/ZBdlHk+yO1VuRgDb3K8Aj4d7gLBm3r3mDASRVoFLp22HDs4GiG6gbdrzooBSOd4LrVlcvYsdPk+z7igyxgE0XAscdzEOxf2EWKehK1iO0/ZS5Doy8XqT//7aLmyntvxt/FplPOoSzEt/u3inAVfQVcVstBYgqY3cVp9Da19DeWdE0BbIgoMx22nzCTQYLbylTsst4ciLN2woGwEQeDsxLNcGXWFzYad1EtkuCocwSEG7AIB6BIa0MkNJGZ/wfCgW2gtq3Du3t9C7mFuJt7CtTEcpyaBUvss3qgUw+Dfdyk4Gv8uDcITBHifpkFiS4/Us0RKnkGGQQr3QioZ5NBAkXQieIwDj3Hobu9l1ZqdfNkk4Y3PN+IRVAXP9DxqPMy1jLs4jvyBd1m96C8GWOezoPp0d+c4BMpwq3Ojdlgqr67rRonbBUN7EdRdRCpR0OTShGCRc9oLJqo7wQi0iNfuJO8gasQ9arK6jXtBCu6j6KjposX2JnpJJ8NSo/j5yko27vgYJ4seJDK6sFAmNCK1tBG1sJJE/1Ngfb0nuJ8rvMTfgT/h2O7O+5fCqXN2FBGhQQ+CrT9vOv5Gw9gTDIgoZqFUoLrTBImfiP+ANtVMCGghnuWEOd/E36aXGgzgYU0m/nXebLsbRHCxNyem3OnXPzkxmwafIjodQrjrKydQ+SYUt7JK68txbSkjsHB1xXVmca930JXFFA/J5+c2K39hYvDsS4wNKxETCpMuYtWBze+vEJOrJ++lX3jaCc6ayyBbpOBrs42Dj9cQKxE4Puw13GSw1IaeM1KVfoDPN6zjlcw1HLE/wnsBtlDRBoPfFdc3dfCNyUT6b6uQl/pyeO7JHspJAHddHvIvl5Ogk2P3yqd86QK05YJjLHcLA5n7wwJko2VcnxrO5BHuPeOcDZU8PjKfk04qnvj8SUxSJTErE3r661SR3C0xY1JbsRNKkNo78k+wFsBdVs/EgSUUXx5EyHVXGlfngLK7ctVrKpOLTrDUHrY89QKyTg9CHoqAuKO9z7XtCvUzzqKttyFfr8DQqGA2q3q6u9AQcHwinW4N2E+9jKtZDLjamStJ/GkZns12sPEYV4OBskXg9ycELCZY4cmSn6DZL4I/J3sg/aeiKeVB6CxneZzAyOEV3M1V85PUjbMUgsdikQox5GEkXz6H1CzFZeUFJoUUo7d5BmrPQ/EvMPx7xtnqEExKNt0ZQbVvb0D1nzY4KpPZP+xndaWUnIYRzDOb6cvkWWKU0SI10BDhhp2PGSxmEASe9M7gx3He5Flj2PDFBo5OT2HEQ3eg6m8AzpTdR3xOEKVTvEiIMmEOTELIBItVNIVDp9RhHBzCYy0C29rh4BBHUey+LZdmjQvnZo+lUXODjJBDuJhixQR693lg6RzMqXceY36VuD+kL96D7A9FKmNdLQQsw69cdCx3PrYTpWNgv3u2yux479GduNcFMOnmAYb7B8NAf4h4htmfuWFflYNioD+RUw6gtu+AgW9D0ArMVw/xyRs/sbXWmaA7/oy0N2IRgBE/g0RF+NlvecUXlhvcmTVPQkRQLrQLoD0GJb+SoJpGjA183x2/ExBERKmpHRpS2HNzPNpvnFDRTJWfGb1JLybdJAqQ2SIIAt5V3sRkxWCSmZAs629fnGr5kitX4ErEMHZLK/hCPZ2RwHXLUPI2fk6KdjYTBAVzNFbkulLo1pqQ6SUMuj0IB50P0SenE7Tgskj1Y+MN7mP52z8H9X3Z2HYYkZ4fjmqqTqzi9ZyI1Wrl/asfUFWrws94iyCrTLyvbnpERxPcd+QLZIYOgqaew8/ZCSprxAsOFoO+s9w+oMLtAiY3R655TmG2ujd91K4O4ZGGDqSWCsY4HqdVIYMbW6EplS984JQRWvUdZI+8jnvH+H7Pw6nREecGF66PSuHWpBM8rPiydw9I7Xjph0l4ZEZxYdJptPZaTJZXkB+NAcc40mRLuTP2d1rdOvh2fAUFliEi7WjSH2AbSG5+KYcfPEed7Sl0yibuUz4tJtgjxQ+fujqZ19ae4uNjQ/nt6w2M2TaBsD7XZpDYkXVlIIJFQuCoNFqlYsBAIgGpUY/M0In8bjYfHn6dqW9H4W6oAW0Onc6DafWrpd7Bgc5THeBbzZw+88o74JkPnqHZRaAiJBKFmxN92583EziWZsvK0TfZGtHIUXM1XJoL9Vdg3AkC4yrwDM+iU3kLL3k3OiDhs54zXBCgKH4hNyKeYGdbF9+qx/abv9q1nLQJ+xiZPJJvGzWsHhoJdceJku9EOVDgycImBplVDHQ+BXVCD/1/ovIhVs85xMN/pTJ64xHMATP6zTtF/yi3bkayJ+QqX9aW8VmQK0O6q80NQ8aRMvw6WGHW1euonbv5gvWNYGzD1tTBM5FfcaBdzdpaEKyBLG7NAouxZ5+6BW5G91kxLzfBR3o7mlS9VdAq4xV+XzGO9+/OZlRTEBsi66DyCLglccdmOxWBXfi23SUuR8/RnB9Yr/geJl9CZzKyZaoDB5rc6fhwJiAGBol6EbxnUFrmyBcF35FTEc6rO5bBIE/6Zg3ChAYeuJ5Ppkcit6ddxkXRISbBuzXG4w2/MD5jGIqkLJ6KPcM5wwjIToOKv2DsESRWI4/Muc7lehvm+KRxsauZvsk+iUng8Z0izLJRGYunq7iTGvX1ZIb9irMxlg2HH8bZWsjMmhUidbrTYDJSOrmz2Qlvsww/82sMXXYJYs6CbRC0F+JbnUzTlKuYzo7E7/ZAak1dog3cndxf7aCj1grvrBevZUjXImL76Pq6Tr5GxajrXLZuxEE5mP47DAY3pzL0xwU4Jd0kYcQpBONZ/jnLADourCXhgg/KhGwWDS1EahaTfYJMwltb3yK4cyorS5r4LKYC2ktEFhO7EB4ccwOfQbfZdX4j91KiUU/26rfuZns9+ToV2xofp9bGTF7z13gtagFzJ63ZadR4nqXBrQubiFXE+EwWKxG66ftL8m6RPuosIYUhPPfOc5Q/J4jU6J0VPXvQp8ITnyofpmw6jN+Ajn5rSwSB14tiOXVgLLq615GFL4BrzxPfdJT4JHC1dWHVrRGoLo4gdHJy7ziLnvbrcTg0OHI5xY0UxQusmi4TmUYab+LNUNauPcWjBdFs2J+Kq7ct7/RZV+bqQNGg+dSG3uJcJQgFOxjmvooF962iMx3uen7GyWc+AhPIg19leO556gal4OttApkdee0jaHHLx77Wn+JGLwaM668DtzXUjYtVU7n97XA8EjTYDEwD2SwO6YZwrvMmFiwUOH1Mh7SS/OY1uLtHgkTGrvpgxn0WgCm6g0+nOuEqD+1jjcHXTVO4aPsaCUPMxDjNJcHJr9+6YW5/0RADVf4TabSvwSi16/Ghhvgl8f16PSPveWHzyzZxgEN07zNROGAwjqBLkHBtxDVs9IH9gqs3G2s590oSJrfjFMU8SAjtdKcpye10JsegxFdm5E5CC0e1mf2uy3ZkFhdGHmNi60bsDJ1EOwzr1+/oX4M2sAVL8Ri6lBIEAZbGbKOuI4AueTVHJQoOaCzMfOJPvFyVYL9d9KV85uKbXUVLVDMxWTHEZsbi9UUGTP1OrO5ziMKqN1Cn6K0tXhmY0W9tV0sNTe1OlIQWsX/MeSbyIuuHTEJWfogRAXWkdA2lxTeFCGUTGiyir5/2NDbOK/l+7hqOFaxlZux3AKyX9NKBNrYeZfnEDRws8yLir+3YJviLbCDmLnAdzibfrYwPLmDmF1P45pMYHHYOEEFQpX+CcwIbIi8zNaGY2Yd88GofhcVqQVrwlUjjPPMeAZYCHvl8Dw9+M5Htux8k+IXL0AdeLiAgWATuVjmwqsHAmYF2cHIYEomVO0Nu0dw0FQy9f8uUayB3YOAX60goSOT9Jg3lEiNZbb48YzGLNMoyOxoDR+GzspmTA27zp99pfqudKgIALs4m2LqGkfMKKRayuXgygFbHVrxRYzTLkUuNeNuVgGszOmUzO1rAuS2Ade4XkGRvh4FvIZHAXPPz3Nw2lMG2vkRK5yAf1SjGIewjGeD7NY98doKNZXOZ88kQcgbIMEwX9+DPIX9T+2Iy2wN/YNGn84idkI08IRfai0ETSrPfYNpHx6Mdlcr2mfvRGtxFO7TxOtFdmZycdwVdzVzKB/3Ag46VuFhCgaGifX5lERs1UKWHsvyrPD7kMep9wxCEHykLPkpB8RisId78dug5PLJauKSey8Jlzaibj2EWpCy61cGyr0Xg6Gjpk+zb2CoCOgGcE/j4Tg0lJaAb+zTnEn8isHUJDwIjRwj82XUTpy43ootjuO/MfQwK/kNktHEdBggMuTEER50bnpeGkhWV1S/uGON4kJnRW3np8i5qyzypCauirwUyzD2Z7XMOs+pOCkHvP47t/AvwiMi8JAhwo9mVJb9sxDUym9NTTmNR9ElQ2QbyjVczGWt+JvzkZhzMS3BUiv2CxxgGdAcSp4Y/w9uPjUGQWGif2ojGeIsrRgXnxm5G2aUkXXmDAqkL78R9DLsl4DefmAJ/ms8lUTgykzqJjCj5TrhhgaG7ELAwKLqYQlUn2bEZ3HBp5xV5r66eVTOPS1dN6O06yJh8DA99N7jWZTh0ljHUNh/ZZC23HCs54qzgtPkG2IfBrN4YYXDlCMLzXbkSNAHPliF9iE+hxdDEyENNqCzDefej2/zQIibfm3XNpPudQz94MINKvej4awbha673nKEauRNO7SPwUIdjfCAGN3kewba9sYCRHkvJd1mKs2s7QW5jiXFOgwPuPTGK3OwEjn8dQNikWkKngbekUQRBeYq6qo1urZR5VjDh0/VsVuuYf9AHtGXQJspnjLe9i+KVw3xf1+ed79McyGCQZw2mQ174yp3xXSqe4X6yBmoH5dIkHY8ksoI5Lh3YSHvBGs9br/BauI7tMjPvusI9o46TVZmcrktjfksAUIraNYHq6t5zN8QphLeUhSzWwLqlxzhbFEy7opfBJNcIxyI+RWGNJeLKLtqc1MhNzsjaIoH+QJL/tf932v+Sff9r//UmFH4DOR8gGX+DyO4PTJTCRJga5JZWiH4eop6j8tR51o/r1XfoOeyKfwFBTpWDC3dnHUGwKsg5kUjcvC7GGBrFv8n7ArNmBOkD05njEMICTScB8odhwhmxuq/uCi66PEY3elJX4cHQwuGEbO8fpMmzBPGrFgK0V1lnNx60LTDuDVqsiax+YRq2tlAbsZSB0XVEtJ+DphHgM4uCtmTcSi9hoI6sAKgzlYrBtOE/gtso9EcfoOHGNNLmPss+TQcPtD7OTFb2rLthfDqFIVXI/pqJW3sd3moPCFolOvadlXzoe4P6Mgeufz+TlLH9jXSAFBtvzg/dh1lrwUdtRGOp7dcfXbcT7V3xAHYRSpCYIwF7iu2eJnNsAw5DU/mrA55zMCJcmgPzRWTdrCpb1EfHsd/2PON1DYwPG8z6PvOurvck5ZwY+LjVVM84UxdeR0XUm6/HNn55+Q+eSXfDusuTiyOqeJE/xA+bxYyfxYFlwU3c6cxkXGAbLc5auLVRRKjHbYcBG/nK1Y6Tto289sprKM1u/ZJ9hV0tFM26Q01uF79uvg/PGbY88sQe0NWwmzos3loSm2IhdzDV4bV4Lfwb8j6nUTKWH1etQWly49nvF1FnYyAcK2S+BkCO4km++biMTnUnaQv3YfXwgK4asJpAkDLTP5OBL99gfR2EyCHcxg8aup1clSfD1PPQNoDX5QTMMjlea0Ig/VUo/gmm3kRA4MNnPiSoNBG/hoeZLvODc6IxwDIr1lYt4Sl/kRNpx5NDC1lkeISJgLf7/djtcSLRSYqpdCFfqow89u5O8d1QeRAhr8fBFs7rO7jfJQtfy0hRJ2vEL6D2Ydel2+T464nmFZZYTDQ2N0KXLzSkgLmTnZ99gXu5lpbgNOrcWlk4SQGj94LViqSumt1hHRxv6yLl9Bya7Rbx7nqpGNSSdCe6Sucz4vgo0KlIwwP7tkICy/aDxzhsDE0I46+R1mygRnkPiURHddQOvCwVtB8u4oX900if38mmFjVKhWcv1YnVwuAKHxxL4kgbVMX5qHyeka8UqWBLfkMffZIxijJa3OE590aUTlrRcFtiAkEg6PfljH33LGu03UEXAFs/UZPPaRDzZXsZtPNxqgZm8KLPfvw0y3l06lNQdRQ04bgXHeayi5JPb/ty+UI4MdNT8Zi5AaK3AvCtcTp/VNzFvVPK+o5AArxbIestdFoYfTcbc7U76iUbMdNBqft2nO6+BLkfM9i6nsnZMRjsVKzxOYPEIuGusQMH2wCwj0CoFc8953pnOn9aQva0Eqi7JCJhA5dRYjxAhQBePrl4uUOJ0Aq1F0VaqIhn+Kv4HaqrYeXA51gU9R6Nhmmw26lnj+1piMb91RGYwssoeuFjHvJwFxO73Vbv4+HlZAaaUW1/nLO3Qln76KeQ9zmR2nAUB+disa9FsPHkq7IWajI+YPeEZVC2lwvNduRqLbS5m9gbeg57IY5x3T+lIIDVbi41wRVU3Ulj1g1P5jyTxMSgMGi8SZcyiLsD72IVrGRFJGEncQEOi9cFVOk7aOtSk2d/hANeECLvFIPB+iZQOoPVjG+lN9mlvtR51KJzbxITRBIRYZXlvogDsTvRGVWodPBJsy1PKt0A0NkZuTryKq4ec9gV+zMm9Sz+0Tx1cgznuu9I9JJZOOrT8XHMRG6sB8SgbLvcm7cefxSfiige/tYNV00Z8F3POTVerWaiEp4yu6DqskVi7YSgFeI/4OpndeR/OwOV0owlSINgNQK9yFCz3IGlNRA5+2/2e8FdiRr6hJccpTKSribRGp3Lh+PzuaMQHUGNwziKvCowezrillzE5Wx3Rs+93INCdDFfJjgsjxICWf/leq4nZYg6so3XwC6Ea9r1HC3OIOh6J6fORSN7sQCc43t0FXYtvkr+/MvsOlXEqWtBfPcwUPOLmPAz63ii0Z62rIFcOTkC49j+1GAAZmR0WMEsM9PoeAUr/cv/tt4OR91g5o37TzM3/CCYW0WqE8lxMr3ckDrVoFe+jm1wPUw4C2fHQ9Tz1I/J46tB14momEerNpHgf6ENX7p4BYMBYpsz0Cs7sSx3FKub775Avek8FwaJiaBnHEFu7rYthv8AFgMXvt+Pi6ERS2ghS4YWkOtkJ9KE24VAwscIFiOJs69we+wNfFQdOEn6014LgoBJbqLJo47NK5/A6jgIaqNA7YOfsQRFWT76iUP4pf5H5J4/Mr87zteuCGBjHaR1qCiKyaRKZWKhIAGZHbTlU2X044XXV2L2gQVr9oiDumrE+/KczMy/duCRk0BDmZqRNmATmAeOU2HITnAdxo/y3/EdIiFuZAb3R5fTYViG29CvxcpBYIQ1Bf1bv5H60hOMKQzHTprb774sjU0MOvkNhuYQTk2+0UMhU2AdwFetMAAY9fFjqJ06kM7oTfzGGBpxPjSP7Jgy6qafpcjDKgJruu2PY6fjqXG8zdyDc7l8ZzC6iUE9Y40WI3cuXSb6XjQvb/saqQCVws+9z1pXQ6j2DF31VpaEf0i5XA7OF8UEqr1Iczko4m9kSyUcKZuEk1LoQeT/81uNvDqSoTeG4qi9zoSlp0QqLODn1mg6tcUoJBYmOoLe1NnveWyq8aDz4HiqHjnHVyMucUrWC4qQSOWci7mLi1cF90YnoxC66Ucd48BqRJALXBh/AYCEeoiWCSy26EHtB2pfRroO427TYlo166h1/EbM0lnMon0ikWOy8eFTVT3Ki0Pw4DqmxgTArWd9v4Ct7BxRR7bv85R4H+YB9TZmAA3l+5m19SE+rwlm4KEgOnPMNCmmgXw/mDsp70jhl/vOoZG60ZJwErMUXu1zz9GKEqQeEvLDs7k28gNi5CvZyo89/WVVHgxLDcFyJ4Z3ZSaCDrqKyWYApTOfFv9AemkZKZHDkFglGAEG9IZwlLr9jJ9xlhvNpTg3Ov9HlcR4dSmzpXLM5ydwK+EWtk3XAHimag5dO51xCdVx9fkPOR/cREetDbbdyb4vSzLwawqm0NrA10XfMjmk37TkB9/BcUUuIVl+DExbijBcgKjnofwArQpPhkW+QF67nOjvDLjHd0HH43CouzJxxC+4Ft3A71oSB6MGoonxEt9Zj/FiggoYr2rAXoDnGgGrRKym7W4NZvi60obsrAYCjq7kj4f+4qFplWAXgp95HAYt6INyiX3xU1QO3VTddsFYWvORlTihKrOlZtUlHnUvR28aDG5rwW0Evro2HN74iJEmKwFxOQTE3AAe6VnX0XyH9jwBP7OVdyXuXJRkiAH07A8gbD2hHVpq909HN/s0p0KqsUps4d5LIkK9ZDeCw1J2BmRjSpnFQ1sH0vWSY7/E2eO2qbTN72Kndh46mwU9/19tbWGsVUWtEXSzprA2sRtUY2gGx1hinOQcimhGJRgJEW7S6FIOuZ+C1zQIWs7Ka4HMsIWpE1LY/mAyX7ZNFoPFB9whaCWPNrtwN2cQJ4Lu0GnbKfqX2R+IesjjTvBUWB0qiZlHjs4l0q0vWbbYKk3u2HTIkeqV2CpasP5rD470yaJygJH9MU5USh5gh1r8TbY7/02lxsDTpXryx17lntqGELUvzMgEuT2/nz+H3DWPc8My0A8yU+cdINKadbetuycTdmEoEwGvsDLk+gpQRAMOuOhy+fTJw7xR5c8bzlcxSRxAkSbaTeX7UCgjuDj+BmqPRiLvxeFoYwavFTDKS9QMBLwH5ZLmn4eh83Ei6wbQvxYN9v49HFWtK76Dc3B3XgtVN6i1iSUpOwMzFnLHWukc/hEBmqU9Fc6dNqFsmbsHldGNmZdfxmCGLrNRBLECtQ5L8SkCWa2ONYf/JGNEPX3lKe5YnqFp0SEsx6MILlrH9cmO9FWncJEZsEhhbo0/r7hZ+MzjAsji+Ydux9z6OpdTXydpfRcXr0Frk02/IPrpaleyfhFhEAMG51GHD8jt6NBcwPn9yySGq2i8/DcONh0Yl++DAS8DYJXDxTEXUdlOZETeWWJi/r1LxD2h825jevg+PKRJwPSe3qXhT5FrH8MNE8iMMmwsbWKyQ+2PJGAxKbJQJEH3+CnbA0Gj50F1H1phsw7l6b8ZfyOHDe+oCfSu7mfbfEE0aow0OJxDNmIr7e1PAJ8CEHjHkfmHt3Lm/mtcVztSJellrwB417sSmWDirV8HsLMsgOlH+kurnA9sJU0PZ95w5rr7FD7aomJZrOgzf970PKmfLMPRpYmmrd9T3zFQrPKdI1LhBhTP4amtR3jpbARBFQEYVFFiALw7CO6i1/Jz+H0E1lwjMtubHc1x/Nxn7UzlKN58Yh1dJjkRZbuIUQ+G27MRPCZxxPkD6gYGkhf3Fx0O50mUvMnMikNQdxGJahiffPgszdUOfY+3HuYXq/o9arbPI9IigWYTHn4d4L8C/EWa8nzTDlqMBXTZiLIDTlIJXF8n0o8GPsBtvR+FhmIenvg5M20/p93QgoOxTaSvFgRqNPcx/lINrs32qOpdGSy73O+ZbpFkYrg4k8KLiaTGp+I65RJ0lLCzawKP151jgmBhw2dr6YrJQTWnHFzFU7S4cio1P5jQT8/jy/VfEmyYBxUH4Yp43X8ajlNr0WKV/M3qOiuqtnhWaHOhNQu5k4yXvIvJaJvKt7OO4y+HyyfsWVFWi4tNFVV6B56xfxLXBldGFIRS561GkFhFZiKrBUGAGucEBixz4FdrOnHqFB62y4VGK9gFc9sUxviKE6i63IjuVOBiUCKzdiuDSUT9OFO7keg7g0gPasJzvB8OLsMobKtlvfoaQkIO3wYd5fBZH+wc1MQWfAW3NzMfsBkM6+7qUAoW3nOFBssHwAbx/QE629Rod09gwJ167piCYGkZAFL0zPXIIPPVVbT4t/DlI1/S6vAuW0IqCHN/G8vVZQS7XOPMxEsklD5MuyAhqXOgWEFn4wMSKYqKIiKunCdzDhQHFxPU/Z47SapYPCaLc10a2gqexvLgHh4LqBC/WYCrvIYZ9+IwNDlxc8oylBopDiqHnj2Q1TqX218rUBTWsm3bu7hJQ3mtzx6p7vTj42bIVXZQMOk0PzlXUeQgfpsEATxaZ6IwtpKUnESXayNSq6l3sMqN34xq6v2LCJipYJn3x6i6HgbixfFWgYjsCNTaRlS2BnIbh+MrEcBlCOvqoigypjChvRaPkns0enbPG/MK2EfwXbI/jndPowtu5uNMB96P0yMr+LI72Wfms2FF7I4q4pcaoANekdr0XJavfRzfj4mm2vEQrzp/hFbXHa8c+hWYdRzf/zXpE7/D68CDvPPQArze9hFtXHOXCECUKlg7womi4BBsqpfiZulfRSURLBgCy9HplIzKtEOiDGQ14NCSxryROfwQ2oGb80pSHX/ELJnM7O4zdJBdAM/+NQ9zo0DcplLmhr5PhdBbDe5hPc3WUV+yt2Qhv24bzY0kZx57or2n/1dZA+rBldhLinkqdzB34/P7XdcFXwt1Bl++8azGoDAwV6YBv4WiTahvZIyNmVgJfJkcy2CFHn+P8n7jXToeY2iRI4f8RmIf5YOHqxjTXaM5yTtOMLPayt9jb3JYXssrRlGixGKBFFMApVlygv2C+W7bdhIe94ewcuy1vSBwmUqGtKKDMXfGkBmTCU7g1v1Yh6jgDGYG3/AjNH8Evz7wK6cfOU2EawQpZ54hfl4pB1sD8b0yCPRd/K/9d9r/kn3/a//VJpVKsbpPAYseQSpl+rHpJKQmIEQXMsq3EZvR3QLTgtCT3LNvtee9u0lYIrs551MeFP+reZ3UxFRGXxrNyIsuFAzPhNCnQaIEp0EMzP+C4S7gZlNGhFMdRszQVSkGwd2S0MldsGzdhXH3AsI7lYQoAJHYQbwECwy5MYShN4Zyt8GNTSM/hPSXMQ0tIChtPypjOxfWVPB43BWsCAjdaGon8xke/OBTnq8X+O16KGZHgURbPwgWE3o7fJwp3vwBFq0dHhX2tHpXiU7m7c0w8nfcQipp8C+nULYZlTJETKDo66HyMMS+SsVVFfnnB1I7JhnPqMb/eMaFJjcudMDaycl84AGXjf1r/1KizJQ7puHq4cl7Cduord8N3kuRyCToNO44thnY+O1GLPOvwsRUMLSAwpFFo2pRRO3l7tlxLNwznoKX+qMjC/yWYF7owlfuj6FTdbBZ2C46kNpsuuRufNMKeRYLvvZtyJUGUYPLaTB4TiI8+UtWSu2Z6lXL+xEl+CsXM1fZXV3SmgXmLsLu2FHw02L8V/1ArW//wGkrEt52uoXjgALG1cymxUnOI6P3g9WI9FYcMqMV33sOXDwzl8AXfxQrL4Z/h/H2CS6HGPmlWct7a37FJG0nWfG0GGxziMZYqybxViLFQcUkzLmMzqgXk1oAjnF4ywtJUMIIFfzsCd8aCmFGukiPp3BgqPRF4sLucf5YIl0aN+QSqagBACBIGMlVhvhYeFHrg13TQwQrutFKziL1qotwidaOMrzrRzIx4yXuG9HzeuBYm4esVUpGSAXNNlpyA7YyIF10QhdqY8n45EmaBxVxISkLEER0WcoKGPw+aZafqPa6y/wj82lMHUjA6BbQtov6YbaBOHZYsOnKRnZnEJUaD1ztPGGfLbiPRRn5BWOtKkrSE/E8+R4600RgFBwOFhF9E04z36OG28GVvDf0EqWBpeTolsKVl2HMIXybLpDsB+M7u9jsZmCRBjo8ZkDdL6wPyOX7IBukXR489clTXBuT3vsjmzp4p9WFi+fHkx22Dxy6A6NGLRi12BgrKe22QV9Y/BuCRcYC6Enw5Mhi2ckf1LW6syFzOE0xRvF3jngalM40YYfvrAvcaRdISY7k+8np0HANYsUwpgwjGZ88QHSBiM69N7gYD4cIkIifValRwub3NmOVSLBptyFp/QG4+yJhwMTwSfytMiLx8+SlsWtolwwA19cgF9S0k/TnIkyDSrk4T6zisgpWmFsCwOSb8xj+wRZKgtu4OzwY/+Axol5a5REIWsmAfBfGpy3h+MQzRHY18GxIgkh9V/wz+M5DmddOzM3rZJyw507g1zx1tr8Gxi2TF/JJp2l2htf8W1ikaemXERlX6E+zvJhVzZ1kmYdTrJRg42Ci+Ye93L1zlXbZJVyz7IlSzKY+ejx656kotVf5YvdUhl6LRbmxmV8FiFT0VikIAmxfcx9VbWHs+s6LwMKBWDarIWw9hK1Hffsoux64zKcNUiZ/thZpVAmY9fB3JHQUUxK6hw//DGXUlVHkxGcxdM51fIRpMGof+C8EuT0z3UtRPfIlj4Y0keVjEIEF3ULpga5xFAYOx66lE06a2BHZxJPdtGM/R9cgjWhmV1okxvQ/sIb1GumCrS9fp4kBk1UzLzI95A8aeLmn/5+gc5NzMUMfOIrBt6nfs7YoHuC1BUbUMXW89fnHfNUxs19/udqJ+qh7TExq5pFxb1BpmgnJT0PJbzBTrKqdcHYCboIE86oz/JtybbG6Co+PPmXHzZ957vQG1m4SqwZlSg8ynZ7Bzw+8D2SwXx9I2aaXWN6tL9cg9Uc725WiqlDmhewh2l8p7muneAh7FOvlhWgP3aXK7QT5Yy/gajMCLCbRsVY4UW0SqDeDYDIgMXcnKLuqACuoPPj76kDcrkdzboYNgQFb+He7pY3iy4xgpljVLAmpQmrt/22JyAxhSHIsP4d/QnqNOy8JCgRBoNo0moo2sPNtYv795/GQysRqsG46tm2OX9Gq82S77XGyvB/ErXoN8GzPvJXWNTi0yhmUHkOXvRbBiuhw2UdizWtns96F2w0awvaMxjcpC+qT4ewEGPw+Xs4D+O2B3dTbn2KXxMh06ws8HP4YhD8m7hV9K+/46vkqw5mQ8kBcBpX1uyelsZY7gieHTVa8HNPQClVwU9xb2zbIeOroTTpdaim1HiK9LZT5iBSMBrkrp69FoDK7szeqHmsnLBQEKPwOCr6iw/oR5koPlBY9L770Nb6jHHjqy/Ceyo1DRRfw+T6MeEAHhC3IhObbIkVYwFJa3TtJn3mJ+zxgrA00IEDJLyJCffwJ/K3FDLWHj5/aSYofnLRM6HdfDcrXUGvkjLw6gCjNHHzHdCCVShEEgfDccLwljlR6DaBaaWaBy6iecQu80nB75jaz29wo8k7G1zpCpC9vuNZd7SDu9cujL5MWn8YDyrf6rZsRm0G1VzVGq1hw3TfwKegqeerVt/n88Bp+fOoZSpamMGp9lEh/Z2gEwoi3+RD/cDkvOK5HJSBSPHd/O2y091hY50dbqyP6uHvc0XQwesSvYOPFvjf+YN3O+8gZXsC4mnjMw/o79bOG3SPbo4Hmjs38dS2agJm977yAlYKIXPKB/R4w0xbMACo3KNuLqzCDxXZwo0GNpdYVpadMtIdOJ8HgD1CaFhOlzCXEWslL9o4ctN6Fo1HiWTc5GbldMDsrEniye702yePAnp71FVIlUokfFpkY7P1HV0zQ1zBO08KPbdncmdpC5sgKVIYzjNMMgLDHqS2z4Ol6ljZdEM22zQiWXkACgL+mmgdfP8kTdXDIHXY23+vXnx9fy+lBv5OQmoBTsxPe0okw9hyYOkDlxYmOYXi0yIlOj6XCt5p/t7a2T1gdeZncjU9gpzLiNesOMKmnf3BtJ10H7+f6zNM8NCYDdZOIbM42+BBbo8HoDimWJoKL4daMJT1VxNm3I3A/MpSuWX+QEfQM5raFbGRbz7xvSW2pD0ljTVo4vlW+OJkrYNDbMOhtJCWZLD0zlbRqB8Jn5BMQUgTS7rNG4US1ZhC/n52CU6WM+XnzMQSOFpO2IQ/3gC7WlwzF6aScl28OIWvRMfGM7bYvOmX+nPh1GgPTB3Ji6gnKbTp5aK5o24/74WmCimy47O7IpCHV6Dt7g2mofXj30FCi70UT9lgwgfbbMNiFQMluyHobF88fMNgGY5VIqBvpi87eSt83OluaRMa6c+iL77Hr+ccwPJHPfS0ZIo2zYxxn7EKofugHZKW+nN6bxGOv+cHEC2IixzkRiSDn81aYp5PhYVBgFvpTNDu5zMMw7CThZY48McgXbctbwIP4GlJZsGsd1TIL9dMHsPZ0JR9svYlf5WooP0CASwZ+CfHUKmx4yWszp7wlkNq900ft40zeMJL/nIZh+HU2KO4gkfmL92zWQeMNPkyJJPjsSJ5dZGJKSC06c5PogwG0ZrKuXINMZ2bYyW8pjm4Fvum/h1292b/hZVb9sIqDpSrenWzupUwDvoqqpC36Oq1XTuDQPgWZTDyQpAgYin2xb7HlewXsbfZAq3QW6eLVfpxsc6FGKuCovEOz4x0KDHb0BfSkuzahjU/Fb+Rd4n1bQfqAqGPfWYWAlVoz6PUyDiWvw9HHhiUAtWehLZ+NHkk86BLH97FyVs79mT362WKlY9HPYGiF0If4LLyetLRQ9p8IQDJrcr97dlO7cy/8SRpG7OLLab9zs2sQ+C3Co3wfdjcHMub2MKqWyDlno2WOpDeILRFE/TmDScWvczZgknTwRNdYQkftA10tzQW2jP1lKY0uzWQPu0FDQH9a4Jk255jqXMVi80B8DTYkhf+Jt1017H4BH5cNfO9xm9b8OJK/WcCvftU0THPHOWcFXPoV5pYiCP7EH9uOsSwSs//C/6AXe7UzGsnCfTS4NrDNq4aXJS8BoJKq8SjNwdJuQ114JdPd6vHo6KUnfdvtDFcXl3PgxkRmHHoDP4kZ+pwXQbJ6xm57mYqoQu5//k1SO5fSN9kX1HqU6rsGVqZOAOtoJo4/B4cfhQW1oHKnOSgeg7aLku3ruTghmQf7mE7tFceYEPo+RzPHUvDWOsyzkklc3dsvyAR2PbaLGWcn8sAfS7Fd1St9sSHiGg2tgVzUqrn5wlPcmFbC831k+T4qDcTQ2I7UYsEsU/4HH9uOXydD8kBuDrlGtVc1VusbpFZPJcHrJDWyCE6O/hydSsfbVRBNWB8SWqiyn8qIy8Xo5AvwVbzG00FA8x2RBch3DvZKe0LaVoPCEb0yA6f6x/utbZHIabXAwI5RbHErxVURIFLgakIpbY0FoNks5/c2CPDyA1kVADX24zhv+QoPPHi3dAZuNu0oZYqeeeulEaS53MHkYOazJT+SY1kudlycC+0FXLw1mYCfR6OcfwT57O9wUAwT6YwrD0HoIzidjyD9q8vcmb6Z9OgingCRfl/tBxI5gzwXcfJCIlY7GanjbYhzkvXT225v1VB+UfT7p0frCavbBnbBjNAX8NIXG6gJkqKReDHTMZPgrqvQqIbOcnwDN5I8vB5T8HHecdlNu64EbAPESYPXcD3jKTptSnGv9OVCuz9hLv6i/rIgo9ZhI19+EIpa2sq8+46ToILLQIfRiQ6jE0aJlklnJpGULEq2OD14ljON4xHcRkPB10gk83jZOR9H33pajLX8GZCDzqpCPfJbEKRMcdhO80eLiRrvT9rTUzkUlsMtbTBwPz+65mDf5MsJjZU3X3gTi8RCWngDcXI7ukwl1Nsko5C50ZJaQO5fnjB6NKsr5wOQbpvIV1W3sAoWpqQOIKvBiehh3TaG02DQhPFYnomg64kUBdfw8cSjeJZs4XHAIncnPGUtE9vkuEst1HjV4CUFpakOrO7Iq4+zIwieyn2feo94tFrYFLwBDn0B80U2iAGa36gwSFEa5Wx30GBFTMR4yNrYoAE37QjOOinZrb7HBd1A8rynieMU9kx/Zg8p7a8zSv8R4a6pSIQbPXsgw9hOnU8OHkopWP+zmqtIG8HhbFc0bRpKR6QglfcP6Z8f9RGFgdEcfXMEr5hd8Kw/De6xPf0ykwxNmwYvVSEzwr6kUj8RiIfOKh6xB88999PmeIU3DvyBV2A1T1YdgOCpyE0ytry7BZOjFPvqOZRNvwpnxolAfPdRFLhaYdZIbCtewPGvdby7LYsXn3xFXFSQMrPUgZpOA2t/XMa9qHsIi3vvS2Ltwl1Tg6mtBbtXnyZ8uqiDTlcVyNTU4USyDsbbe1PvNwg/u8kiICflAfCcAhNOsiBqOweLV6E/W0CMdzvwQc/8UksXP2w4zmdnYgh++1kuLMkSt0lzCm+7wl+NZvSOEm4HlBBQJ++ldDbnkhhxhqyicVy7PYWMOxGsGObTM6+8rYicH+2xd6qixqaLvToZj03tPWPLXDPImbuf+w7P4fCpUbR82Nrvt6q9NJH0IwGkLvqDcv8CXhA+FWUxin6AhE/Il//Bru3XmZN5BXuXVuJGJfcbf7TLH86FYwzTktyylaTuo+yMbij73x/NIMGZG0vBYAUEsdNqBRe31yh5PxWP+z0gFlxCYgnuOsP307P4zMMd+YkZnN16FkWslQnnJ6AK9qZYSGVCpUj3Ov3oDGKlNhitapyanRCsAmN8h4OuBjlGfGdepiZPQZPqZyIj+T/d/hv0nf+0/yX7/tf+a00qlRIbGwutUsj7DBurnAi/BrRthagLfTnXYst0eS/NhcraxqOXJqCrd6YmIwZP+yt9JlMx2HyPO/7w2ch07GPLCfIaL1JODhC54UMq3qbly6e5OLSFA6oP+GqKvWj4TTwHriPRNV9nQz24j8lgbe1XmA1B9JXomKM4T8ixmdT6VPL96u+xmBN4Lciftq5SbJ3Oo9NqsHRXmnRInbGTiSGDNsGd3+StFOpc8Nr1ADeSuhM4t54EQxPtMjlVtu08mjoUz3NjsI0uAEs3uVLxT3ySdZvyCgt2ukiKrEpWSRBFWz0ngUMUlysWItem8sjsGwjO/ZNeAHY1Mh764yFyRlxn/gBnngrrr633tk0q+rAWYhSgaIQFgiMegKx9HktHpPNd4XSSbATUChM4xoq0UcD5zj0UV+jpCvqUTudLeDmE9ps3pXUtLXpYEraQmXYWpKY2mCUaYM03LvL9oeHolXp2bv+WepMdmNog73OQ2nBbmsi1jcMJiMngz0X7mWHjBjEvQcQmMdkpUfBn7edY3VN4Wu9GtFNhv7UFBKQmKe127Ryc9wsJkljRIGgv4HBhHNf/mErLVFCOcsLJowm0OSBVEmjrgrYygujSBRi8P8MkNYgJpBl3AZh8qZgjy400CPbsct7NbV2dqPk4KxeUrmjPPMtzH95CrjjIs/b5SFcEgLwXJWmoLeXijkH4jG9i7apXaTPMAfcxIupZ6UIgJXjX+eBar+TzaVGgCBd1Crv3UoTPw8z+uoZ1O95kZNYNvE1hwCK8K1bw+he/s+FmDvesWawMT2VA6WkxON+cxiVDGHpFF46NGl76aj1NY4tFOh+fWWAXzNpsF+SXH+Vw4jWEoHxmyJ8FZzUErgCZhuEL06jQhpJSKhrXggBowkCqwl4TxJa39uJelkpTdA6BEYeg8nUxUNeNLnrZQUrNA79Q2k07YpV0B3i6qmm0H8iLz89mQI0vzaE+pA/OJ2ZyO3RVEhlVxJ/qeyS2PkLo+EziwvskSyQKMhMzKQnOZsqdkQy5Ohj7Z++ISO6S37DRV/DoD6uRd6j5bMMX4q4QgNSnQN9EO8O5WW9LfKU3rheSiAm6JAaTjw+E4FVctsRyIPEdph2fxsI/7yd81IeizkT5Pph0gTq7ofwWeBR75ypKxl/mnp2a8R69GkgjbaVUumsptBWoDzrHdZWSY3ah1NsO4d3Rv5PZqWYBYjWInSUXAu4H/8Xk/fEJkjXfkKf04JnPnqDLvR7h0d7bVkiNOPvWUuBuw5bJ+2hWTIDAJ6HmFLQX8JRFwe38MBzrfXBqsqXrKys9SSCnwTj5Poi0KBJTlTvZNqco6hqB68jdIt8+4KD043r8bfxqZvDj+fuonOLYExwGsEsLZ86pJZTNGkCLYR5vH6/ipx+3MF5iYTvgaYKaxma+43su1nzNB4PX8OKb1VTtmcXNxJtEVz7IoN0xeK0c3u+dLdc5UGOCG1NukU0lj1oHiTz9eTvRqGdh7d53ap0Kb6sEjK3QIaI2LRIlmjYNUosUw604Xok28t2o2B79DIAutZ4W21ZuC/Bji4pVCseevjh7L6a3PkRDSS5RKd4ccT3Te2HGOLo62tA1OnDm4GRM84P5JyVhxYJr8QXsmqvZEtDGky1z2T3YzD+qUgFdt7jaEs4nXQZmhN7EyRRJ33SfQeZLk1cUNS51bKgDudIfCr6BG+sg8QsmBr/FmWA44zmK6vJwZg1swafkN3Fw43WCbacyJXcoujoVD9u0M3zZODGY1920sgRym1diOFsNnZWYt4hX/o8DY7XC4ftSaJfnMFToTTpcbitku+ojXNzGsCXeRIM5Xjznpov71dpciHf+JSrdHbk9VMtCVx8x2JvyIDgnsO5mIB7VTsw4/SdSBxlm87PgNkbchxI5B+PzkURfw7/tN1z/pfUKENc6kqhfxG+p/6i7SDb2r+yLTLqL1qsM9d6Z7B14l5efFkWvrdpWQm8epdOkYIY9OJt9yQWRWkmmwa/jEL66WibnDsU5+X7kL6ST3cd/nlTVhPz6YOJfSGFK0AVabd6C7OfBNQmVVYfDh48yQt1JqU7BGd82ZiqcwG0kKF2Y7TCAMGE63zZ7kebyrfjdMBugOQ1UnkhsfHkkeTuyY43UFDvQ9vI+Bve5J0FQcHLnIiy1rryl0XJrZToXRxaD1czrDd6cCFnDkMzFPFWUw4JBz0HG0xD7CgICE85NQGpVEJcaRLtdG5I1gkjRDBht1Lz33HsElY1i5fdOFFZ6g/3snnWz3Ww4u+JnrDITTnJbTLKnWVB3SWQ8iHyWjz1ysErgvQPD+UDTxtEpMtDmiVXfVisHpct58O9ajDIjk5uTmBF3f7/Kk0m+JxjxSi3vv72epR7VBMobcY+NZfbd5wk+O41KiZL8RPGAk/fJEe03xlHldYAOqS8u2vF4aaKhvRDOTQTvmbxSaEvHqSf47b69BESWYyfpDQgLCJyfkcPlLnA5OgOlrYBU6HW02lTezKhypdV/P37DBlNjbxWTecnLRNvKdTibMoYgPSmwOf1+hky+AV0VPUEzuaGBTZv/ZHUt/NQGGosfTwSJwUBbl/PcHViM3hkq8pXUtDr127ujOhS0e2fx8K8m0vM9cJzVq6EsMXdiCoMftHD9wxXk+Nay6X5BBKD4Lyb47N/4fLeSAK0S20pvWh68LYLZAGrP4aptR70f1AzlC4ai/q4CEj8T6bidBxMkVbH612eAPGZu2o3Ssb+9qOq6ywKvg7jckeKhW4DTqhIA2t2nE/VxEI4t7lijL3Mg8Vtq2nNA8R04x+NUdpftJ+dww1lHcFgcZkl/17JG8TLDL0xA03WBt9MdaBxr7tf/htc9BtmYiJRfB2CBsLkfxeLkZhu8T4rJu4rBd/l3O20cTsrGIai19kxb/TdWU38q2UP1MQSWuiBPSGWStxa9w0CUbsMQauDbre8Ra9xCVOpgGlwbEJS9lFLVQfUcmXWEsMYopv29CN3j/enk/wm+ta/ew2tecNL8olgtZurAIlVzssAdSZMj6yanoBbsKVe59tA6lVVc55tlHxHbEMHHNrYYguRgGSFqjAaugJE/o6h1IPGaqPr8mpte/OYpe6m4CkIL6LDt4MbQG6glHnBhJrSko2wbSsu94bTFtON+T0mM/UCW9rnmK6OucHfgXQbknyctbx5vjtFA7nBozcRetY+tL+3j99z1fKrPx96gYT3396x5tzOD/V5P4GX1QhjqRYD7BPGdQYCA+2lVvkyybynLL4wioDRABLw4RvfQiNtprTz2/UEw3ODGk28xQnis3zOts1/H21fW4WSezdUOD5Se+YQDTYoIqgb+TInZngLJ47Q61JPJG/h1ioBQR9053p20hPfSxQqwv9tlTLbtNjivLCJYVkFVw15cCpw4K9ERMchDBHeFb4SwRzl5ag6ywFzW/jWf68kGBi2SwrgTYmLMPpIBe18iOKsdtUsLgR4F/Lv9sxdsumxQGFr/o7r0uk5Ne5eKtbfyMHo6A6K+bp68gIJ33mC6pwMjJK5kDywXKxWcE0ATSnRqJY+cmMq1+SmMD/SjQdM/2V0eUUJ+7DV0VtipgwtydzgnlpRZQnbx5J7heFf5UpuhI3mMlSW6BsAKUc9hLW/i+jcRNLgqcOYC93sk8IhFJ1KIdgOhDubtpuxyIf55OeTojNCntk8mlWKxiabRwYu4MhihTGSNz2iK29rwa+/Es9qRxhYdrp3OSPocg3JLB/dbbSntUjI3oJmptmDEKAKzgLjGdFoKS1Bj4vOXt3Ohs5dyHeBMyUCufjqToZOTefYJcV+ltoqxAotEw/HcCPhuDMWBxdwYegPX1nWo61Z0j7aS4LKb9ohaahR2jPdfj516LPSxnMwKK/die1lyBEEAUwcjDfdTs2IynT7LOdGxkxsOd/nA5Q/+iWH6S1uolIEFKxpNM0H2/Zl2pjvcwTLQD0t6CFsefwrzNg0JffqfvxZD3EUPOn38sSjccPCPA2d/kNmRlfcn8cPe5kSpL+csbVQF9AeOdcjduDL0BilOpUT+sphjdS4s6bMFk4yFODhCbYML96o9aWi276FaLvI3YQk8QnnWKK6OvEq7Z/9vVlHaAMYeGcz1Bz9FE1tNRqsDY/o8r7MSE+Eu9UjGXWekczuC1YwuYQ8/3OsgMPoat2pvEVIQwpZ3t5A+o1GsBD6RACp3pkVXkPVoCO2OvjT/Ezu/+ZhoX0w4A06DmWF5h4K7og/saUnsd20xGj9WXnsc/0su5HdKCPvhNugb0DakYTYtRd7iTOhdN6LqpyGEI+p6Ba1EX2Xg0LxDYJXw3meb0Wm0qD5RwbxKMDRjvFLHL8t2ozQ50Xn3V6IjPEnS1Yn0/nIHFHZ21HrWEe7YxWfusNtSCQO29VScm1uLcS9Ph69XUBBYjPCIINLcd/sdEquOkDv7MSjtyXZ6GHP/TxZfq4PJ2/gJG+VTWR9/imb3J3DyDWZQ6iauKc20yJ2omr6CLGc34lwT4aSoaShXt9PmGoxgtDIsZTi6AU3gktjz3Xn+o3F07l1Ks3MLTk2OpDyJeF0e45C2giG7E7vqPP7MX8XnfpUonTt5ZsRyCpoSOVz0AMEhNTTVF7J6QRlSp3bOX0MESOnrkEigUXOJqq4qZtu9xtIbM3g8HqbmfQZ2wQjtYUTlnQWPUVS4yfm4GWKDAgHIygum5cspWB5rx+huRCMBuaEWLLao27L5OnMEpcdG0zwsE7eyuzTr4yHhY9DVcaroDi4bBrJY64md0ZPU4CpaE4aJhIFKZ5iWRmpeBHu2vIvKGE6LQwtend1AJomAwaLgy/W72DpsK8qLSn4IMOKfHNnD8gDQZbLDIj5CcponMGGiSGOPNod1017huUQYsOsTrHufZP6XYrJPL53Mlm2/4CLRoZzthIt2PFG+vfQAdaYEXki9SuxQLzxqXsZg038TdEpqOD3hG1RdKnZWRWLy1PfrVxl8mX3yZQIKmtn30jsEq81iDEOmRhAgrWYqzUYvGu6/D2l0NfZufSKa2jwOaZ05tmMZVsFKQlMYnz89DB+Atjx2eVp5Z1kBnQct2J04w/O/vga3AM0pNjsVk+4VTKunL4cm/sYobwu0tolAa8A96G20ViO1NmXE+5TQqfbu0VsVJBKqrscw4dww7NvsWRhfjNxYB4h2l7LjNG/EPsqXqTMotoMLnc4sAxGYbheCvCuJ+/9cTIsblA6cy1B7emNs3es/d+o27Tn3iPS6Qoj7Bfom+6wSJc/WQ4mmEcOw6+idRVu13ncZD31xh/j0wSjXGhggB6VM31Mx3iGPJdl7E+qEKMyf/EaTBJqMBrrT6PzSCrZl7uQ4plP/5C/YW/pr+s23LyXMFl4dfo3bkfdIkK3u13+u8TkkkmuscdbxZDAU6stg2Hcw6D2QazBjwSPzCl02Born/k274QEm9hmfaQ3m6JZ3Ca14mJD6DuiGrN0yxGBRp+Iid2XS2eE4qdqwX97LoDI+YTz22+0JGBNAwCgxcpF+VkUBBgytaiRyG6wSPe3REg647iLAZziJ3onkN+VjAaKyomlzlPLj6hscn35QnLQ1A04kYqOew8Ay8Ox0JJH/25p9PfmO/1L7X7Lvf+2/1qxWK21tbWjkzghRzyF1GsZHCz7iaJKR+U3HMcpamKp4uufvHSwVhN0eRItdOx888wH20giRfWFRCwhSPL56l7Pbn8RvZCqbFl3hu846Ealdcxb8FnDZEomtppUBJ/2p8nqegtYnCExbIiKtpqbgq/Fn9I2XadJc51fX3czRz+MBAnvWNwsykjb/xIauDspc60i1JkFLBu71z/H2C7dYV61i2L7FPH3ak4VvOJCkawCVK41CAB/mOyAzQeOcQ3S52YoVAXkiat/XOA11qwtVYYWMCmyg2H4kDNgoatM4ROL7spaA26kcmbGJZudGqtqP4XJnlYi+X9TE7gnbqZ6cwi0r2NfEsP9fz3mSogBjVwS++xZSGzQM75UJ//oLAYVeARnhfO1Wz6wZojHibcjm9IsrCR5sYujDzUSFZMPEKpFnGmgxBVHRBu4hrry7YBd/m337zWqwVGE26fBv1xCnNKIVegO2giAwMnkkrQ6t/Dz5NphdiFf7idSmcge0pcWkDT+JV7UXq79fTctDUpHCqE/irNlrCG0hXTj83ky78my/tW1Mtez4YyUNtc5EvP8BQ5X5cGcL2A/A4GmhOOE2QsBoclpHMNbtGUZUHIK0TdhG/8xHOzaj7Gpm8iOO+NsokFoNUHMOtDm4G915Z8oaXj75NG8/9iwlEwoY+7CyJ1FiaavHJ+8Coi7XEEqWSsWx2e9B7DY2G/Soxp8jOrqcE83BxJr0OE662HPd+6TLsbxXyqx2DfsuejFmez0BfYI7P2rjqDbX4NRqQdbYRoBhF7CIDKOcFr2ZU46rScr+i0mDRTFjJp4BuQOG785yaNod7NurmFhtIlTVBppwsbpP5UG0TEeNSU65dxV1HnXMkMnB/z7xH7A05iWu1A/GUV1ElMttBH4Tk65NqQgCtLkG0WFTScjqt9jkZRAR01Zzz/hsgy1VwKKyUGa66tCqwkSaOM9JDFP7czDfDs+maoQSDz5Fx4kL4h59tHoEZlLQ2ZlYteI0sn94eACkSjZ3ydDZ1vFi/RgaChsZpB4MPpNgZjbG4jTmRlaT0qzkkeujUTh185TnisEBQTqCzR9sJn9ALrlbPyZLMYRH2rop6BTOKCU2yEz2WMLVLEjch41td2C07iI03UYikXB11FUASgIhQN4uird7TQFgoW0BHm/+zsjjB0kJ/QwvawLMyaeqrJgzV35HYrbSdvk4H+/cwNoXf0RjNoBUQYvUm9f97+FrceIp33qivZr7URQdlw7h0JLXiG19iK+8jpPR3l3xWvoHeExCMXwsMaE17L5WhW2tBReZHQSvErVPVW7MHJeK35TLBBSDVIAtkjchcGnP/B+61+LmouXN34fhWn4Hjxl9Mo2AStNFrX85aQ7XiGpawoGfDFwyWxnWP37Lcoxct8LsHd+hlnexaswdUk2tCClrce8sZojlFiDqVQkCPH3WTHieLRPvc+KLmFRuWPMg809ovoMsaj3Pf/oYZomWnzZ8itrkyTK5vXhe2EegLsgmfvVhDs++jlN6Mu+Nq4Lqr0G6S7wYi5lpCiud+aE4XZzJb0PtWPVYL+5eVvQpM+puc75yIhVPfcUbAZ1itaTah30ZOzD8dQxDzHW+X/09Hh4DeQMxcFRXf5OByq9orYoiLK2cTptGtMZmQKRhi3VP4rvv9YSp1fDoK/y78q7RXMm18RqaZLnYXRjEgBEevQ6mQyQJ6vdYMW8HD++O5qLZm4mzBFhQB62ZYB+JWgvPvPsB72zaxKA7gxCW95+/WTGBn68m4CXbh8VRoN3QAjhiarvE45Y3uXMzDO9ZzdwM10LhFAi/C05xCAKMPzeeLjsXnsvdS0gIjAFRN9LQytg4Adc3blOU/jXvmDQEOk8GazeK0TmBsPwSxlwciTo6jyEDKilsmY6XV7zoHFuttDp20CGv4FPT+xgVd0DEPfe0WF8nCsbLyWy9wd/BWRyX2vbrfyKyjXp7M4cPTKfKqzf4eb8qmF0Nj6MvVrA07RMMQ/zF4PvVJSDTcOz87+jP3MI8roqAMalU2jn0e7ceu+8yl8alka59FlNpDOMkcrFCzmKiWfEgWWP+oNyhhSux6biZB4kBUWM73NmKq24dCU2HKE6ewOT2J9Cvk4vIzEtzIGApQtJu1JdH4Fh8mTMTz1BhcejjgoJZ4cbvw64SnRWNUq+kUyqFgW+ib85kx+2/sGvXo2wx01DugdkkgxaxynmwjRy/B9K52TCHppQW3FQSJBJBrBCSqggryuZeAGy0ucIfryejtsZCUyKcEOl6zOqHKAopYkDldryrXmS0O+D8G8g00JKBtNqB8qNjmZYaTUFIgTi3qTuxJrdHkMqZ9dc8bLpsyU16jKARHvRte7WxfCPkUbPIgJ35OZY6xKHVavFXD8RucTIn217Drq0JB2U9giUAEBO33qr3SE3ZxLhOO8ZlfoPvWItI0wOgq8UiUSJRGplqGsSb7qc5o+p1ZwRB4KZO1I5IPBWPf1gpakVvdZNFasO95nbkyMmafgJ/YQTUXe557wDya7RMThcD8rke1cyQ9doeHa7jkPSL9wtwZyvce4fvInzZmHeR6qAUtg99AI+OSX3/kMSq/bR1GtB5qqm2BiJIezegRuXMt9+upSrbky5dO/mKTjHQnP8l2HhRSAS60mZafapYuOI4xSEdoPYBt9HgNZ0/c1OxTqikqSwWizCKRYFPgpdjzzeJlizGDD3C9WInlrnmMbHrcXb1ubbWpqO0/tVCUIM74M7wB44BYoXf0BtDGZA7gLNedRgK/FFGmET66BsPo1Cs4+bFeNqiKqk97oZrYDO83TtvoHQocRdN6NHRVWCiOKY/qrmtxZbSCl/W2zWw3lPHPXMlZL4Jhd/C5MusGZqJybeSkrtRTImo4t+tTdDQ4VxIVdxddgbf4mO717tTKWIrDZXwx+b3UOlUfJktJ3/zYQIc/XkuM4wxnjDxroW5h+eSPfQmgrmLf7TW7g8tZUhMHT9ecWSIXSOlLv1piNYVaZCdeJI9Cw4wzFjOo4MHwYlEaMtDlXiKHct/6fnbdrNzv7FSiw5/x05MtjW4uXhQq7YBYwuoPHuSyo7eDaQs+xOFXw1GLx3bpb1JD42lki2TM9jTnsFCAabbtIqaUlIbVoSW0/TKSSamrWXrO1u58WBJzzjBrGPcgGpyDdVEvbkPqyBHENaL1NGmNmrkoQh0Uqk4RVSyhCr/5n7XLcdAQkkQXdYBWN0/ISkUEeIdLDKsTFWtwf7uePRzNuCsuIjGMlTUoDM0gdNgJJgZ6f0X5ZUOfOEP5039Eeo6fSfhKbuxb0zklFcVcctEwFibIoB3k87haIxk5Xe+BFu9mDo5Sqy8thio0Puw740tNJmaedVzBub4HHgqHy7OhuY7KIJ8yJj4FJK651jy5xKy/Y3gNkL8B6yQ5pBXZ+XAhCRMUi2fyteD92jwngqAwnYi1YGpLH76HbKFf0XmgYHGImYVxpO14iBOPtVIhK/69fsXeOFzbwBu1U0MG/gyZv0fSJWOCBKBiohJ2DkJ+GS2kySxE59V5WEIXMHC4EIkQ8z4l7vQ9ddaAjdf6zdvTpCeGl0wScW1WCQWMckY8jDUnEIncyI+LR5Nm4azE87S4icH62oRrKFvoMV9G0W5H+JZrGR17R6Snh0m+ityDXiLkI385ADsb6czYPnfJMWU0zfZJwjwxZQIzt0O4s/j07BOEEDpioO+lCOrc/BJbGZ5yjjiL27E++3rPeNszHVM/WMxZZWuWBOyKLA1EDiyGxxjMRAhKWbiz9t44/aL7GiBSr0d0/rcc67VAzeHFmSKOOAax/Mf4Y7kAxJWvUF9kTM/CStQLPmdWo9aWpxaGJ0l0Bx3GCchA2y8aVYeZPFz+/jwmQ141EHQ81/TN9n3tGsWGXopwucv4unvQNKwaLCacTGfQjOgiCsGF4ydVvKNYOyjN7Wydim+O/W4+pbz3to/cZH40IeLhNbOj2kbeY90m+t4FBtB0oeGEzgx7CznB14koTQNlTkQszdgK1LZmuqvMP6OHQ13nTCvvcggob9NZNaE8GwD4FTLmY2fYdcV0S+4OsaUR+CtoTw2+DZ/3L+HpM5eFcSdLbHcNWUT5+DNrDmFNAtB/eZu91VwbsJNkgKrecu7he/MFf360xKzSB5+jp884EF76MBI0tB2kobL+f16C6vsocJOj61vLRFOzX2+43Wg68Cj+DoeXMe2dR/K1jngHdctIaFD11UH2VVAGCpbHfPiHgZO9aydKJSSdtoZi1lCS6gNaeZpDFH9hU1TCkvKp1JQZI822IxTVhixXALTJKi/glLnzmEvON1pIT14FVaJhIcB1N6g9kYuqSBBCZVWLedcl1DcPoU365eLjAcjfuXJdguVmlZ+TJ3Do980MGxjf/DiwIijGF7N5LODJhqdm0QwgOtwscreasWl5WvuX/Mjf17YRFjrCVSWGKA3hmKRWWl2buZwxWAKDnzCG9v0OBm/RxBg/bu7eLvUyu3bcKZoDYs1iNrSxnZiOneyfOGbvHD8MS7umUba/MJ+706wXw1dY1LZ79jMNI0BN88gcN8JZdUodVc5Mm8LUoOJ2XsX0eSoRepqYoTfQUb4HaRO58ozwzL4dHgGjnM7eOJRNT6OpSLw3H08wj74rDCQ6quzMRvUdBp9sB1mC127waInUHOImM2/cckUjTI1hI+8WvholJgE+kbqiv2Uk4zzW8qAEm9uxmrRpEeA+2XkxhYGO3dS5NlI0dgG5sz4hk7pBXA6L+5tiwNpLgasSkcOTfgSo6KWJyx/96qD7dWQ4QVzrXCz3MJWGwWtLqKvL2BmsF0dcjlYLljZ+u5W8tf9wdSxOSIIHEjvfAOHYxdw1e+l0kfPkcARPJbwkTi3/QBmlwZyx1LCPC97JJ22XG17hGjALNjS1WmLSWjHL9XKY1enMP2TQJHFSuWKWbClsi6IgHodNZ94cih8Ajs3dWEjF23VqS4n+NID1twIpe6bRcQt6A8eiHTMZ9LEG1yzGcnSb55g+tzTIkuVXRCCAB/e+AjP4lvo1QLfWr5mgnufwYIUBzc5Mk8ZphoTqlo3hH/oNJ0GwoTTmLMGcHn8KyyRicauPmYHSrOOtS4VrH/iJ3Kbn6VQUkitMZAfF/ZWOL8a8h43K514L9COLYMyOa+LFnVg1eL+lpmlmGQmlKsbcI01ou5zhqZ3NpO/YzFtyi52L0vHSxCT2OgbQN/A85qBHMqJRCtR8dQDPpRYl4LPe3Cftkc2JqX+RSbfHsLOFSd5X2ZLX8JLQaok+dgIXC0yDm88zuFW8e0w23ihq/RgSHYI1uOtLD27DcmaWz3j7rbn8o7bWtwJw228Bb2smUmdMxiMGJ/T2+nZueU97A3hnHEDd2lL/9+q2Yz89yW4RKRyJyyfRFl/X10fNoBieTgRWX/wd4UPUQO7NSZUYnyn03SV8nkOnPD8ikrPfNZZAvqNFyQCXeouZpwQ8Nc8g6HRB3gZiUTC3kX7GWmzmrHvDMRfU0e84jSnmIzVCrbV7zFmfDEVMW/R9buScoUPlR7reDXTlVXHQiiNCSVz5BCMdl9RK9TibyMww1rEAk9oRMnHW3bQVbeC4IYN3PMSZba+zz7KirAnqK4TWP7rcgw2TthI05E4uQL9v3n/V1pPvkOj6ZGy+H+z/S/Z97/2X2sWi4WioiLi7O4gVB1D6b+MF8+doUXnAUH/lj+HNpkPt5/aTLrJQrtUh42pO3kk1wACDm5Pc6vzDzIawaPoClOcB7G24FsxqdZ4nWbvYH5c8xGrjs3CWQJScwdEPisanPomguQWZl0z0dU0khNTT1ARFAh9MOonzOP50fY4EhuBDXYS3BVVYNSiUyXyUP4t7nRKGFNjj02zK0mtH0FxFEQ+g0QQePyLxynzL2PI07+iM7qL+i8uQyFyM6uSPyTriyfYv2A/D8Sls0KxHGTqHufZ3+EzGtR6Fu0fg2tQFZbndGJwR+EEFiNT7JqxNrri8smzSAdPhhf6PzeDn56k93ZwaPNLqNrrkQh9Kn2AH9LiyftbpCu9knQFaXf0892uGZgTb6LwjeOr1M/5IeNz9q/onVd/6wcSTpeRP2E6z2snMWFOcL91F7StIOfEGNpPbGJLcCEvzHWGfS5gaGJQzCVk63I5WefIiYOj6PR34EWJDDK3i6gkyWpOTTvJ1CujmHBjOGGqDKj8G7I/hMHvg9NgsgPDaXZsQab/nMqWx3q13AAJAj6xBdQJQbR9ex8Fo+8RkpgFhlYWabowzs5noNGbX4bvwsbgCPZLoO4CViTUBSSitOoY+WsF7vadSB/Ww8VZYO6iyz+Lj0/spKRBh9w5g2p5f6ffzaOCxvvCOKA6S67nHyxWvSAGmvX14D0LvYOZG2MvUVDpzf7cDi4Kcqg+Bc13IXwD8/wf56vgNCQFh7kSmMGV5hH8tVsA7xkw7ih327xpyYyideLPFHtJMegW8yJwVzWT924fw82gwS//FJlCOGMXVHcjnQTi7CaTUTuZevc/+f3JJajMiaIqpCYEVB68MSCF+gGfYQUSZKC09tcZmnEpGBrMLA35gkhpF826Zryit4KpE8FqJDC+gjyJlQ87DRwujObmrNdECkY70Wn0qLMjJmMCMZfHIIkuwjLXVuxHVCArHvswxi4TV0M8EWw7sQpSBKsZ22oNU25OoTNGynwVOFt96BvqWFpcTnOtgYo4FY1+UhIcgLpD0HAds2ICGbNOcLUDxr78Aj4RpaLzG/ksyOxJyLqGelIDRU7V/KZsIQpH0HUbrPaRPO4WzOC3zbQ6BBP73CpMqCHuFVEHxtyF2tTAAxpIrwugrVSNwbcQRXdlL0CF3VN8d3oRFiv4y8DdagRTJzKLkYRbCTi0OGNRWWnFTJHFlYHdBoBEIv7XYrVwdvkflCsgpo9x0GMo1Nbz0vpXaJ8cRPzqOhj9FzjGcO2SLzfvgNuYgbwdks5uXMDGS6xcsZpZUR5MszyT913hKSe4bayF3BOijtegt+jscmDv+3NRt1TRFDYItayevk7on1NuUjLuNqmH4jh8XM8lKwzj/3MbBhwxwZh3dlP6GcQakzgeOY7ZSTuQOg3oc0/g3OREQGkAnWX386chgSFLp4P+HrTew8FjBqGF9ViETg5Zf0VilYBU1UOFbH/vBK+6QFG7hAT/87jK68VzQtZNUWox8HJRKH//KGKZHYIX9rtOnfNIMnIEJB0mBnhrGOdaL2oAAFpzDq5NRUgNEiIjy1BIeqvNpeYONmzawxtNEga++hRWOzekll70pd55MpnhfuhJZcuux6ge1r8C2aJ9lUvTf+CJX8fgc3Ie+ZGA++geFG5LdQ5HT84kLtmPRufGbuPfDVTjuxew8OCRGsqXLiXe/Tq+9E+03Go5QdSFZNoH5LPjxT84U54JAz9Ba7GSme9NW4OcMRXPonXZjL2LVnTInOIIIAfppbF0+dTTaXifNr0nGGaJiS8gVxqHxasA12t7UZ3oQhlgB/et6vk9EkeGURteQ6pPDldtDTwjSEXq0XvvwOx89vjWYNfizLnXE/CfeIN/Ny/Hu8x56EWmV8KtTpDJ+tNf7C29REZRPf6fjOJ1NWKQUKrghp2Gtg/foPTORyTs7kDWJRf3NYDzYPaGfomgaqI0uIMvnO+y2PlFXtsKr70Ga9fC7BtaWu2biLo1gdtmL8Y/bQ/RL0L2B9i5T+X3UZcA8JSCM90cwT6zwaSlPGsxN38yYQs4eXQRa2cDjd0BTud4sFqxeLtTZgqh3q+ZSGV/Z00mlZEVW056/G0W2kGAMAiinkOv1zJoWzGTz0zm+kor26a8x1XFQg4l/Q6AXCLHN7yebLUj3gMzGeeSTZfiexgm6h8JhVsouxmFvdBOxogyTneUie8OgCaM11yuM9pGYEKxDT7KTKRmb9G2sPUHiZy3S2KYmhpNydBbdI5NEYNW3jPBxhewEqKQcGZaNgP2DSU0Yx9yHgR6E2MFrZ7IL01AbmdLsnQhi1RmiooyqLV9g89LpAyKNxF38ilG+GRhbnkLHMXgu4vWlYl7/kDn7k9rlwNOnVaRnmvUXrAL4q1r66kK/pKNv73L9q+GM/7I1J41BQR2/DaekMIQfKb40m7fqz8BIFgtzDsyG88qHx7/8h2yDTlgMYrALwcx+KQNqeOrTTtoVHVhUsCmPhVfgkSCW50b/mX+TGv1IGBIgbi3ARU1yCWGHpaHf7t1QwprkF67R/YDF1i49DwVzW/xD5WsQqkhrXgBDtpcPn32LawSq/i8a8+CXQidNg/z/tPvYpa1cEsJbsIUlqjcYbK4L4svvcgfY34ksDaCmPL5aFT/WtxiYPScC9xZ+wjL07eR92p/hEa6JIyLw45QLzGhHnEbi9NjjAMUpmaMUy5xOjGdsGuzOHV3DVVfCUzymgpjj5J3s5q3n38cF8Nw5h5eSKmq//OWGWrwybtAcaSZN155g1Gqdf36/0oexrC/g3EOLuWOpxbXNxyg8QZ0lAACb3TZ0+FSzOjoB2hhESP+dVtKjBxa+xNNZjCbwdInKQagEQwE6dU8+N5m0uJzcJ7/MWizSNUFcCdFgY3JSsHaX/k0vBVF1REIF0WcwhQGxtvAeo92joecZZhyfT9d6kT7Virsu9BUeNPUqcKQ6NBNAwvW7qpEuUHOo7+uoHFAmQg82O8CFiP20e9xTWPPSzU6Ft7ayOSEMEYatWLCDnHvvOJ3F7eAVnyLIbt+PttlvQlrJa3MtYectAHYHljE2HW/g1u1+K0XBKwSKx1OXTQMScPNtfeaBWMLuz3hk2a4NuQKvjZtlDaEEjHqcQh/nMyr19hx0B97rZWk68PxsOtN0gAMEpJR/rgSfWwJK9aOpNO6DOjVT4xr+ouO3wyETJ7MA6u+osLeHy5ME4Pko/cjOM1lzIPX+fuWH1uPx+E8MYrxfebPKVmNgz6A0sA6Ds36m0GCmOzopTCz0jwkmyH2nUiwgOdECFtPyoE9tFSasO00IBQMZUhwnVi6Pk6kQh/y41ZmzdrH25dWMSLuGCF+40Ra1NxPwSGKFyLXUuYWyrDWYvTKenE9U4dIF65wYuvKtajktfhU6VCbvehPJggarZ6SX+Ywdt4FZsZX8+/av6cabChKS6DhxU9o92vGjIAU8Bf2M2CinjzDcqZO+4lpoV8DW0VEv0M0rpHPMWDwCXZeGYuX+wk09nH0rYU8dfoh6tJt2XwPCC1BtqK15+z3yT2P78psLuoNrIu6is7kCjYeMOEUmPWcvXOYr544z/QLwwm+rcWlK1w8+x2ie9hELA5OlAfaMiCogjah/3e43dDOgY8m0FriwXCAcalwcSYOchd+K55NWPl6HMIbcJeexdGpVxVPbRdFdtBh2pwEFOlhlCp1rJUo4E+xStkadpI9WVsprfFHe+w+Sgc0wzO96+pcLXy/9nvur3qZX1PT+DNvMEOHAjZ2CBLQqS0U++YyUgUPaCBZU4TRfSG4ixXt+5uG80THXoYPuU6x0kJm8zL6EpTOpI66l7Zg1kuICbqIq9kd5A+wsv1ljpV/SLw0hdHJkVjVjshm91ZHC4KAZ40nnQ5aqoRWOiyO/Z5XtT6RmyWJpMefxDBtN4vs3uzX72evo93eSEddKUZTB1q9H7a24lla5zicffeq8cgN5hFbDySK/rp6EkFCeG44/mX+eM+4jK2qtV+y70fVGJS/RjM48h754flI6PVdY0qGEHPJB/nIeDZGr+BWS3y/uX8ZrMcYruXp5OE8mNpK3Cqvfv17fVsoMsPNd1aT7F3LR/OUcNAebLxx0TzExB8foEDVzivbfuKXljCxyqfbxnW8totNn37Etz8uRpflTkXjbpj5vqh9rvKgtrWSt8afYPHdMHQdKgo7gnuYMwAMtvG4z6tkv901Tgcd4xG9G/jMorm1lPZlf3C13gUblQ+bVu2mltViMjvlAewDXiA4L4ygGjtyfP+pZLRCxnaQqnA127Hj9ihOVjgiT04kb1qZSAk59GtwHYab+XXiw3/hzPd7cS7PIuSliZD9Edx+BsKfwMJ1xoff4NbDEKXo9tuKfhSZYQY8xSVtLW1R96g/dY6AGwE4Nr8G9NL+jZNXM67ZnY4jHbR5ZCAYHSB7GwXKF3nulxdJSLSgr9pGi4OZVv1mvKbfBuDspvs588mT1Iy7xsE12djbjxDjC8cHgVsSD6slNE84wuOOz+EnP0CddoEoZ3PzMZydXyZ7ZDEnO+GJoB3YSmWEXNWw9cwFxgT8TlHzYKblDsTBX8GMLgtYLfg7ZImVmMO+RxAi8EGgUaek00GJVafEg/Mi8Nc5kTTzBdx8O8mov8PU44NIG2zp8Xc7HI1kjUxhvHQ5S79ax53FZxg99wrYeNLl5s4Q73XIlpUww8UZe3Ul+i6NSPd65X6GtuWxfuky3rvyG1qf16gLqyfDvB043PM8zSYJw8vHEP/pOCISs1EuERNnEgz8NXA/O8/EUtYFdwansTOghU6/tagHbUXfksWz331BjDUeW50Kz0oDd5QX6KHflWvIrHHHudye1kg3Smxn9nphQhfVowMJHhuN9ogBlbuF0LYnoHUPqMbTas3D597XtCTbcCPpGrUetRgtz2OD+J1vN9tz7vwg/G8mUDRoHs5Rfa0PcFTVMWXUn+S2j6HloCuNtqtB2fuxvx10P48dGYlBAYUeEsyWcKTdVPRoQkj2ukRKgpHrg23xVFiQWToAV/Eb4DmJViMUhTazcegV0luCCA3ZCCoD4yuHc7PrGjH8Yz/2t3I/Oz8Uu8NJtC+9wKt+8JpLFhz0E995q4WU+5O5lOTMh8l7uHpOzqfzese2yvw4mPQJMqsH88r24OLWzVwR/QKYOpBUxDNu+wm+TFZx6odhSO/LZ/S/CgF0vhWkjG+nzqkJudWh37UJgsDAuwNRGZWcHRuM4Cgm+yQWIzcnnSN5dCaPdizDoeEeOdj3gEFNVgPhOa74NLkxZt5JHnVu57axGhgEgIOgZbwNlHVZ+ebASCQ+bXy5snfdHyujSbodRJRXLacDSzht7h8LsAZ44hTYiPzld8iQOzPxw/5RFX/jJr69LxWHQnBoi0cj7V85uFRxlmcFD74PLiY6sIEGk9i/3OYYz/nBY/UW6qevJ8htNy1SMQ1utUJb1SmU2nskm2ZRtTuJzogahLngVy7GOcwyJfVVoA40YtNpg0Qt5QHD9W4XT4/SCi83mhmQMw5pfSxZ0VmsPfMqczc30NW4g+AWaOmU4VL1FybpMP6vJvv+yXfExsb+V+g8/5fs+1/77zerCcw6BEyU3fHGrrmM18wLmRV6ERt6UaUGiRO/WjuRWWSs1HlgVHYH236XgkyD2kfLvbHrkdU/w2Mvb6NoqwDzZ4nJPqfBeMplhBuiiXvwbzY5QbLlcbALhrRN4BCJYIpizsoD/LljKffdGIn7/EvQh10+wCYan4YVDEnW45oWxaOf7wD3+VTZrSXru3Tsu+w5u/QaJTHdaJBuCkN3SjFMukaZuoUFtlBtbBcpWqaKTnSq/U5kUy5R7yYmGoR/9NRqzkDwSpYufoHCmS4cP++Op8aIIJGIQd/y/aDNZVWXjpbsWG50GjFZ/lVeA5zWRbOl7hzbnrzBq2HbqWrzA8b19Ku9GpBHFFIQnsem+GLs2u8CoVgUEo7OOsqY5mjCTqeg9wqg70E9OvxT2prDKaj1wJhui3ZkXL91VYHVqMbcIEUrp9KnUjR2DSL1iJO1hXnxRyk8vxbPs5O4MSFfdLzD1oPSmXFp+3gzDO43ypgwNRN50FyxcqnugqhxN+In/jSb+DN1OCc9nbBxcuu3tl7pxcKBlwjSlLPy55XsdYepa5aIlFd3RG0fn0wFh358lMC1dwhd8QUk7KC6pBy32Zso7vRBdi2KVlUbs+2CewL/GV2dNPxei9y3lb9WHsBHNqTfurbSJ3lp3mVyquTsczNynCxI+ET8LQOXMVC5l1AFBPyxhFaHVoSXBEh7RqzUCVmL2lyFe7yefaNqKXI/w1JJN2VJlYiwV3RIWbx3MckjkjnlcwpvQ2+CaOH+hcjNtkj09Vyomkbih29heywOjFqi2zbylE0On9U+wGmPIALlodBZDk4J4D6GVstR6q3w7KmZeGRF4r6zvyGivhXH4NuxFEvMPOdXzvaHrCLyUmZHm6aa18dPY1eOK8a9CyiL6wD7MPF3coyFqC28RytXL88nOyKbXwbfYp/RHw54ioHbjjI+mPgBH6X8gBDQSLQCamZU4WUoIvjkm0SkDCPdppGA88spjGvovSiLkalNG8g57U3FzHUgdDug5Qeg+Geckh7gp7RPMEnaufnALe4qzSJ2d/D7AITUHaRtyk60FXE4tBQhUwsiQnFxO0iUON9+G6G5E7PUzMI9nTy35gpD/QNFOlnAt+QLHkoZw4V9E9gPLN/yK6Hje9FtbbJESs92Emsu5P5zTzDlgZOwx5Zo4IniByks8uLyczpmLNpFinw8A7tRacGSRt78cDPa0HbenZvC8U7Y3AdJPk0ezqDTT5FjMdPkWEqJPA/qVKIA+8jfyW8W6DBWkNPazsQKmOgbLZ4VtzbAxPNIK10ILxtJkVTGV8E1DI+X9mrbDHqLa7o3qa3/gkZfZ1Y/9jpDrPeA3oqGKyWbaWxzwfnYXyQJUobxn2dO3zYMGCeFJR+tIlE3kZZFd5nVGEtSaAzdhJQIAkTdf5rK+07g/NF33Ku2I27tQIj+CIZ/j6RGS+BaC7fbXZmdPhidYzfQo+R3qL9Ms+1kRl/yQ1UUy8NRe7BpK4bIA70XIVGwV2lLW3wBHpHxLEl4F7reh25dvpMdRrY+tB3n1uGMOpuOMFHOMxrxmasjHqHpgwwy65O47Slhd3svTtBsH4G8UBQz37H1Bx516eCqpTeRKAjQ4hlBl7WAUVUCgzX9tWdaLA58+/1MXGQ6kp76DVvP/hRFR6rkeCf7YZp5lqVTboF5tejgmrrAbQRNumaK+Bo7WQafhGv529KfZs7XfIbOiXfJcGzk9zZQOYtuqF7hz+blPyKzqHn20w95789tPJE6BQ9PkT5aioXZb33OJ2V2BO5eiFV2HvTdIfao5/nyYC3Z7UXMrXNFU6RCbenvmLwQ1k6lhweD6jXc6Ghks0QmJpsRQKqi/LdHqU62JX9ADidcqulPaALpphi+vL6dcu+XiJCL4I1+z+18DX4pV3EJdibYoamHH2Rb03Bu6UoZ4yZlzY7dqFU/gc0tGLUH7CMwXX+AZt90Ott7qXQiImD3bnGKOxclCEhZcG0PXbYStIancbSPBPfRWCRK1AIInSr2dIQi8eiEjnKxGsd7Bn9c28eVB3/D1zKOBU4jaHZOhFAH8JwsJvssBj5ZEcuxytE8ZHXAxqH/XvBTO/GNcRPpukY+CPuIO/pC+F2GnW0grw4x8A3n6FRPossKWqu6B8VqtI1hw9EcXFyg8e7D/Aj8Oh+4tRGKfybco5DjO21x83PgTdkOGu0cmO4Q2RNsMzz3Du99oGOqpQ3Yj/cDDuC6AkYfANtAaoLfYtu2bez1hEUaqJIIYlW4IAOrmVmSW6yZf4ytNfexNHYHElM1sL3nvhQ6CaMvj8EiWPhlZRw36z4lytaxJxBptBjQ5ii4WBtKpLzXgR1mswVljIFryqlke6xAF4nosfgv6vfcKh1dsXo7YpT32iaCIGDTZYOmTUMKC1C0utE3vSRru8dTU+5y4IqRPTuWIJ2Qx+SHZ4BPr47i/oAMDFYdMWUgtfbXFVNbdCy7NBOnzEAAooLrwWcOKF35/cZ7KC6cJzxewibXJPLsHfuNnR31IWmVg6i2aeZ+ByMHrf+i5YuZQaF0OM+7PE2ssjtI6DEBqo4zyWM56b8tp8RtJ2kBjxMns4PWe5C8HMI39gBBIm3rWBe/ETvDSuhDFFcicSKmUMnmf+5DMPRb2yi15+CQbjiNAQIEkVfNpj2d/cOqWVNdR4mpi1yvSwxUzARbP7D1Q5e6H71Kj452bg7vQibtP6+b7HsefXsXHzREsNcTbupK+/WH+K8ie3AJgXn25JXpGeg6B2Kmi4l6tQ/XOm1pRoK+cgI+5nH/kWiZKb/KN8HwyDuLkFlDCZrTn4ZuvTSLgOYwfvU3Eu80HU2eGIU53fU0w74djU+MnuRlBfxqC+v6UHftujCM66fCsZ9SQHkEmOjvpG/1sNK49mum/LQc/7IAhFVmmJUH5i6sTe3kmv04XGeHqdWJ0UKVaEdaRP1Ro9KDjz+fh2teMDPl+aTZXwe7T0VtXrkYoNrb4Ut7nZX5J94iNGpxv7VbZdGM/mU4k05Mo83Wkd/KD/DyxkaQyBhxZQPm5gAa3LR8Nu0I+9t7K3UEpQsbP56Dw90YHF7/hPf9Orionw7N7tCQgsriT+KtRASrQNR7HyD5F6CtFl/USw7SWBTOlS9iCV/V0K+/XRWHY3Qyt9tnc37vs7w41B+8povJPkMzRquZ4dmDmXJWiVv9IDIn9K9QGiAvYf1XexhVDmtswNkk7hUf/U1+SZ5Kcpcte8bdosW+jG9U8bjbivcmCAIfPfMRtjpfHCzNfO5rhj9kIm1f5LNkNaRS9f+w95fBcR1r3y/8W8OaEYyYmRlsS2ZmiJljJ3HAccBh3GF2dpjJceI4MccxM5PMkkHMzDwaDb8fliJpsu/3nG/Pfaqe3VVTdqmne1FPr+7rD1eOK1apnti0FtTOCeK6/MYz4BzNgV1LKdrvxPTHqgn2bkRi08O2XrLQxDNktsZjsjox8mword724xsgRzuW84t/I7dexsbTobw/SmFXrxt7E++MK7xt6eCrOjDJxesOMj9D4KBGVmR3syHrQ862vc67S90hTvyVXq6Yzq+5YAp5lUOh7zDVxX6NcPHmDLxzTiGNL2JSWiESq66vLkIh8FLqSWS315CiPE+7tBeYuvII6Moxs4wJ/kfQTDDz6LJsKty9wWelmBeyN3i6YlYiNWMt3F2uxKtmOHMHHLvHouevkBy83KpZv+okZyR+oAlGMBvYdexnAruaUMwez57aFdw7AHB2dw4nV/YjqgD4c5IKm2DkfuElcPAHfTXVEn9+vZiImTKG58TT5WavaFnlmMlXWvh2p43iW3twnBdMgKoAjr+Cxul51nlYcHIayqWvo3F31eOVXo+qaS/UXIT4l5AIEiYdmUSjZyPbU7JYaH7arv97qscS5l2HLiAUafQqPJQi+CURJNy96W5UEi1SowNhWi2RLbsAEUQcpmni+mt/kFc5lWmXhtEdYG+7BxBw5zBqs4KtQ2G/5SQD2bmXPCUca3Hhh4rPsKGj8aorvko9jNmDSeHKD/N2EjqoCctrT+I2LpJnVvX3K+/I44n6cBrPZ5Dk4kZ4WondcRslLhy9dwPBLe6caYukKOgo9L4JpndbKSxwQecRwsuHv8NpXAqDByDKRqszepMVj1v+hOf5YrrXnkzhVpBEQ04I8lYtVdpWhL8BBX0NNu8obrY10eagYd6p7wkPticFH2guRoeVzQt/oms+zHX4mEVyp77xJ0jl5GibOTvyLIOuDeK1Gh1nB7RvU0awlWVk+2zFSwoqwQTZ/0LpPZVHTR0oNCom6l7gyZxKno+bAB0bAdFJ4f3d6QSXB9Pwpg8OEtCbdTjdEnOLdbtsJ2vfCKwOejKmZuId3SnmChYkULWHK41v8NFXC2j2jqI2ZCQTlf1uCyjceKMumdzyStZFdDDEqfc3aWwV40KCQJk8lpcbrYSMPMsKyXWybGN7YQOxLNX7smfdODRAW/hHeBfng0ROhWoNLkW3sDhoSDsvJWz4bZR1GyFkLQBXpEFoovMp91IQ4pqBrypFJPEBNJ4HxNQGCmEKLx78gJgYoFB8o7Zql3Ow6F1yDAK5qamo9GXMPi9wu3EMtxtFAr5nzmZiCv7k986PCZxmpaR9rWgzbtYhkcD8sAp6Ys+z0ulX1nvdi0ywgGYYCAKn9TZ+a60mzdpIzDwTK6ILiDCK43SJaxUqBZjUChqTJ/J19f0EjwwgyAkkLb3UCcGK0G5j5h1vAtWzmHbxHlEpCSilIkHZywoWs5RE2d7+mzlmH60VJoz3ZAOwNaCY7vJpzAEEqYJ3S0fRvT0DabCUo4v24BsCPVYDOIVj7SxhgscRvnn+CJL6hyhz/4GPHMbDqVkwajtIlHjXuDJt1zBOzf2NqoTX8DDdDzyCwnqAHbMW8EKTHFO0kUoXGZ1p/ng6i+4RnvILTJt4ht11LqgmHmPIPzCAzPbRWK7nk3wnkGvT4pD72+dCvtw4kvSjvyKVyvEZ9xDLH0ZkTyPuZ54NKKJpRQ3q3xYwvOcJGspX4xvavz+93XOYavcGprua+NUHCvWZ0GtMmduYi/KHfUzwDeTbKOioS+dxQSTZ1pscGHJiLEEGJVMuv0rS4uNi3seAOeDgy063bjym7ye4NJyusw/S8cIWnNP66fhXa2ZQUjcMj8pcehw9EIQBa3fgRtoNfBoG8UH9fchUWmAHJIvEiJpG+PriSpRVT1OVHUzBVCX3tt2C80sh5hkIv4/zo7PZ5eGH9mIqXSn2+yypzcjyFzZx4kYEe79+EZ+Fo5gFeJetpz3SRnKRCVtMAt+pb+DeMK+vna/lDs+VJFJ8O4yqmUe5ZgBB0v+OT7Tmsaoqnn+3OeG/fzLX0u3fWXVBRt5+7W3W7niFbx8Yyegf7C0dV3U6UHRrJL9lJKGQDP8Py8ur5jiOtWRx3/dPEy3rYtR7l4CZffUWZBx8fQ3t8XdYmXCBd3vzQ3vJ3Wk8PpIM3TDqZD7sKX6a5QNEyE9WxCG8HY9j/AVcD4/livUKs6Y3MCGsgZaydmQmPV6ll5BJu3lh9wtcW10KvXbLf3ZB5sWHiW6agG9xMcnl88mLycMqtaKWqwk0FXP/51uYf3MUrcd/ICDJfl3+3/J/rvwX7Ptv+V8vNs9RULUDa0c+ivYefEoqMJb5cypwOgtX9oM4oU7RjMj6CbdGGaHny0iZ27uYlWvFoEPPdcZHHeW8tJbK+CpU6gjwndQX1Fp18i2WF7nxS3sEs0KKeDbBE5ydxZeIawpdneE8XLkL08NreMT7DtXmKXbnOdnRCeHrcLBBfdJtruPLlJJfkCUsZGjmUNyb3dkbVYFobE2fPYyadh68+wQl2RF89eiH6AdHMuQBoP402EwcdnHlxvA/WXBhJAlbl2J4sUHMMZT/KTRdYGGdjTZbOdcTxE3vTUEO7hkQ/QQo3dlyaii+p2JY+MYW1F6ngGN25x1uSsN4/mFuJO5kayf49bTZcSuW+lajX5LNMicY5AFdNnFDGCRtYZQKOjpuEpijpjEgH6wbxY0gcCOiFcfYbZh+Gou6rRqz0T7/yzKXDrrHHyBZAcFyEKxGmFsDxlZ6OiJY8FsbdZp9LH3xOSZqnESZfi8rzsVtAp+vDcDFIY43fNbyUDQkJU4WrcWCFoDUgd9yQzGfSSdj3hXCfOuBfhrN3yzg0rBS3nj9DQJtwyDmF+iu5MuSZkozA6jzaqNB38BlXQJ39y5i9N0lfBSznw+blJzNF69HowLGHwWpmrILRZwddRKz0pe6CCMVBvsN/Z+dbvxe54CqciHNrSUMSZ8LIQv7bBIX9dRT98sqCtzbiB93GalVLwaLAeTO+La+yIeTfuRg5r087wr+inxIfAvcUgF4LvAs6jVy9periSuMRO4vgh7RhvOYp57ncGs4l6O3YJA3MMlySWSKA02tUs5tH05ghMCSis0EDA+GnioRCFf7Ed3kQHLFIOqtNhzcm5FKku2uKzfjBkXx2Wia3dBpdEgkEqjYAUp3rP5reOKvQTjvmUSSQYUk6IYI3JZvFjc3cc+TG9aK7NUv2GXuwqg04tGdLdpNNGVSbTRSV2ako3M9DzcvYkLkX0g1bpD7Mq+POUaqykqIYSSxZ10wBzv2n5TNRmr8SdqaZnJH8Topzncob1kJAWKwTttTQsalDKRmA7WRL2OgFxMw68Bmwaadw0dn5uDa8StPnQkl8L4SsHSLeW8CZlOmeISC6S0Ybh8j5ug64qf+GzKTxeBRyDICghew/UIP0EVzzCD2tKXytHd/IOb3svcJKDTTo7TS7ejBnc77ifc/T4ssBPW968lsliGx3ccyJ4CTIs1KEEhyisMceBln1zhGnRlFp1Onnew/hSIOndfim2ThzhNfojSPFcGEkLtB7kRc5ddEHM7galMcNkkyhic8oUUEizHriKt1IemoGLy4EFrCkKfcxNx2vfkGD1re5uwTm0GwILWVUqmPs8sDt//SahSFOeQ4vM7H+v9noO/vstIEL1WdYJwhhNrGDtTmkxS4VEOvTkIQ4Dk3C8Um+NnPmx6ZB1abtc/CQmqsZc2QR/np5kNU/zSboBHZYGwTc2wBtpjpeJ9PJ/F2IsePwpGHR/DWwBOQSJG5nqHM38iosKcY6r0JzK/3V/fe35hcX9IOf4AlYCkgWvMud63AUwaXr7ny9nuv0Tg7l7t7CZbSXoWPpsuJbeHtXGs3slzar2oprvmT9FEv8pfOxPlxZZhNwXb3stgWhcdpLd1xdUTNr0ahHahPhoqgMjY99ykv+RjxdjHTAHBQnAsYtolmkhl2sQLHxvuZqzjIuMEz7Nq70cjSlcdYsX0IX+wayj0v9OZN6L1eGzbaQlIw91gwa+L5O5ZdIU3kPmMT+DTR/fXHnOnWiBYskzNFtnbF+zy74VkOTDvErqfrWaRWs6hFL+ZhSXiNL24fprpWyYSS5+lRdcBSesE+GwhSrlvvhuAL/Dl7HSqVvf0zwLm8Y3j+3M7C7jeYsPgYkn9sfHqcG7BqBX7JCuKtFhnNK8UdrtQkJbwoHJuyms0OUbg7qXhI7thnJ/yFdxkpSlh0wJvBBSn0zGvvG38Acw5MI+l6KsapJ4nzbkdnul/8bQz5Doerp/hx1zJyu1ScKArCd6qCEY80QN4noPTAKrVQGlaIuXEMx0vvY24KYtoETS8P3Wri53bIOeCA26VkZr77j/xf5k4cTmXhmhfBmhEjkQ/r4Yvwq0i6igmPhhLnSrQF41lYGIffoH5SQb2+muDL36GwSCFgDhaZEogVGeSmdiQKBcVp89HJLuP7+SpKxxbYHbdIfj8tnicwq2/g61VOpj6N2LxP4M57MLOA8Zpm2gX4tkHB/jYr74KYD7D0VwiYTZvTAvacjOT4iI14B95Erh9HwoD+Hwo9Q/AjZg59s5Bvu7Q4du4HzXJ0HT8yOKuQrixXPnvsS2yCjYfVn/W1a1CbiXnoLOvyPSh0mUIBw3md/gDOA+2DIC+ao0FyasO9GC+0Af0R5SErDlBhPsCRjk+RmV3oMdfhKOuVuincuJZ0k1KdmZDf53Mm2sgT+jrYFwuRD0PK+xxo86K9xsSzx4YTnVYgvst61VpB5noem3GWrZ5+fJj2Bd/JfakdIypUD2x/mahaJc3lMi7un4TnOHuw5JR3OC2LWim86MU9B1YQ/d5w5vTWmcxdrBw9meyOQNKuziMh4JaYa1ATDNokvHSbeT3wMzZnjWHMgTUUzegUnSlas6DuKKPQMWrnchpK/egIq6EqYRdJA8A+s9VMbLZ4j+JGZjEn9BbQb+cmw4SHFIRmLUl5nzEsUXyfmZ3iuL8ezvfYUIZ0M8TrEpGK/qfsZSljn9WH7ZZWsuPnYrQogP5A91F9AwfV9VRpdDxcFkaNi70d5sjwLjodWtnt9xRgYyGIYLZGZBeH5/kzffti4DTlSQ2APfBVZgvg31tG4XcrgbCEYiINl4F+8llP5Wp2/1BF8BQZj6+8RwzYKlzxVIRwYMYp/H0yuG2EZ5rgQdcBqhaJDZPcRFi5KyNuL6Jjij3I02pR0CCAz1ATS6bsQZBFiTmCANrK2LdlAp21HuS/8RGlOLJQ4Q6zK0CmQd9cwu6YzxjW4YCrtosIxx4x386R4RCyHFI/RF7/Lm5vZ+FGE8OjHwbzxj7FlSCR4NrgiUlm4s6oNSQGK0DkrvBglSM3fp/P2cUtrHGLw8llBH+H+CQyNTec2wkJrOS8TsqUangkMAZyPoTyP/B3fwnlmt+5YoCvurtwsPgwf8A1VwvRrIu5SorejPLORI7V6xjow3JerWH/cDUT9xbif+UmwicvwaBPxQ9g6+xB03yVRs8oymOTWeo+UNcHu5nAspLLjLSpeNvDynmLuM+LcI7hVnYNPhZvDGOusUcHK3y/ZYGfmIcozHqLm0HwSr2OPQYd3+nUzHUHCr+Gmn38XjqJoRtGUJ2u4JHWXJYsgZGGTeJBwx/kF3UxHrEGBv0xj2D/Wnwmu4DvFNGKWabm46ulxNwJYmJhFK1Db/LPMsb7ASr0S4k59W/y6jyR/IO9/abBnSZzLmGlYbS6tvbtUb7oGo3lfRX+bjdpcL9DU4sTCG/3D8HKcoJu3qLVxUyoD2gEewApd5CN60lDaA46wG6Zhe8cQuCP3vf76Cye/uss1R155G1aRUFaE2dXGURQTZtMbGkO06ujecNUzHvtIwn3cSKtpx5OTYfY5yD1QypMNor3jeap48P564lsu2MLCJweexoZUNsFgfI0ZriZEOqO41ZzB/+CU1hHPQx42wUvB+YMHqexEqKwIdiAuSIocenYdtJPFnAn7g6p329HprdXMRg6HbhxdhSKiKMsm9pMZMoblOgnQf1xnM3BhDTpOff+eFR6BQdjq6mqH4NzVu98FXYfgcoGQi4OoyQmn9y0LKQDUkwA1KkUHF+1AaXJG6+2Wfj2hDLc6keKrJALGfnIbBnkxB3lAe9i2hz6V5pPuxTQLfFm9aUHGXzwAkl3nbHrN1J7N7ayCDyq/PhQOQKHGSF29Z/tHgYHxjJUsNLu0o7WeLnPSlCKDQcB5BYBOgx06extgSWmdpbMP83bXrXc3DKHiw2L+LB/GOGhUGII6SDy9FjOHvfG7c07fXUjJhxk2tjbvLp/MW6XmujsMNn1/de5V+ncc5LKYcc4O2I/j0q+tKvffux9tDfP8evjX9Di3iKuL0f/Bc1XSHSdy1dJy1F1NeF56hYuk2NFEse5hRC0iDybF98VKjAqHOly6sIgqOHG85D7b5h+G8GmJhIZCrkZtV6Nptl+z32scjcnE+9myY0hLG2/i54VZ2HqdRq7mlAfPoPSoCK0bQwOFUYC1e6iEwMCeofhHJ+wDlWPisd/fABX526EJxEBg7abdBu8+WrVT6gUZjSh7VT39JpCFv0E3ZWcyDXjmafHY/Z+/rXwR3IMJ2DkJ5D4Ojj44/T9M6z+YTUlwOwphzi2RBBzjPeuIQQE7t1wL3oHPW8v2cqzwvN215WjehFDXBb6UW/z++hiDDY/CF7GkdwPCMx1o85sofjhTYwN6sQmndr3mzd5PcnWJVtJ636Ok9FfU9qtB9/nRJtDqYpJD47B8/o8HCx3iCvLwxZ+v3hATShmZShHfphD0q0kbr+vxsvPhESwEONxgfYeL6o7o3Bqr8DcacYjrIoxKaf5oexN0c7bsBNBeKxvXvuz61kyVBZi1eMZF3kvmLuRNwn8651/UZLeQ0T8RObE/YZFLe7ppzR6krN5Dap7wzkWNwJdZ/88ITO1M78+CHNuFH5mBWvOreH0fXoxPUXDGepTvuSrCefx6NrJ4KMPszmqlMkv3qLPtdJ/BnKFgeooV7JjfuNyylWSqkWbKkEiZWP5JGR3HWR2xgR0rSAphLZxz6NqOIO05QovuMFuHdT0ElvDHZqgrjflR3sOR8de4GNtOa750bzY44FtzgbgEfR488nFcKobtHQFHeBA+Apu5Q/l3AQxVtVl9eR6xhWud6p5ygUyVAMV7CAIEvbP2M+ZSdksMZvRW7wZqKI3yOpxrviBxFtJ3Bztyo0mNVFRw3vbwjLPGsoyrPxi2M0DCQ3I9AOIZcY2nEqfZdSFNOrqRvHa6OssiRX3vDRlEnlkGDafFUTlhXP9w3tZ+fRVHPZqYFYB49XpeJ52oMtbii3+Dgl+TSKBRBMCDr40eXVQGnKLKScDkXZpeLzpAX4d3mt7Kkj45OCPuNTcIqxgFw4jSjEaP+LvtbuHpYQDfrC5xsbZnaNwn+wsppA4PBQUWrJanqXdep6SIYfZPbOA+dp3oPkKtN+Ba49D+H382OpCUbE3sadGMyF1q93vylfji8XRmdtdg7HW5aPrDAT88AyczPmTFjJO3c/4xwbRXr+Kxkb6lX0SJzSL91M20Q++XshPyi7u/aM/Pl2iD6X8y0hsKfn8+MCP2Bzs3S385N04yKDaSYPELYha+Ry7+o+uxxCzdyQJc2+SFnsOwfw80K9KzLbEEvnMqygARWoXPYKfXfudpvHUjDvRLxrp/fGUKv7kxq5v8XAwUTnkHBaTDTNDABVWKwgWAZd2F3pcevj6ka9x8XFhQ8BcPMd9xnuOq7DurMOjMgvZ8JmUh95hQVgi8Cs/tcODDfDStihSVW3cGu7KyYytmGVmouWgOr8QCU6MrIQGkyPRPjEEePHf8r9U/gv2/bf8rxaVSgXmLmg8j9FjFJ8/+y57WhQc2naMIqmGeerIvu8GCPXMO5OLzuCCOcWffE1vstn5TSBIyP9hCcMay7GG1fDMokp+7J4NNqtoISBIqWuO5cY2Gx6OOnKHn8M2VS2+HOQuEP8yORUXOZ0gvigvAPON9ko1H9NfxKVXcsiWzoGhNhwU9zElTY5L02WefeAg3zSoGJp1lp0tWqYuN/A3X7VSSEEohMA2D5ZInVFK/KG7Bo6PBSBC8RF1NVJ8VM34u3dikEsh7GFxk+o5irL8UryrtTikZKFX68VJvP2OGOx28OV8dBsWr62EOZcgmASisS/j9T74HPWhtmA2z48/wo/pkf/4hvhS+KNT/NwaPwKApfKLjPlzIT85WvB87ieeDm4TwRCJ+BJb3+ZLtVDOxAWv8PZzLfxiOAMc+I9+n1VFcrd3IXVSiWgx5OBLU2M9YWd/xEdjYMHMVurNMtFyL/4V8BiKm3YGLTlv46aR0WO9jbUnSrScTO/PMPNZaDO+83ay4vwYEkI2MRDsU9h6eObKKFpaHamfc5AMeS1s04DPZFKkbdQVT6c5Q8mvU75knNtKuPk63H4LRcxGHv1yJtYmd+gleAsCouILCLEc4ueVZ3jv9mB+++URXFO8GXg3q2r9GPrvFwA4QBgZu2eLKhxDEyjd2N0WhledK96Tz/PUxNuUWI2Q/hOYO0AiI9MWyfU31rCg0JtRi0egnXobEvsVSs0yFfK0m8QcXkFcnT/TpmwG3iLQfJtpU7P5KVdOj82AUW4QF43jj0J7Lp/VlnDtgR9JLltO+rFiEuMaweXfEPEwOPhzb62Skt2TODjXRF54Bi972tu4ZI8o4Wg3PN8E7tJecGT6TRBkSHRGCio8STcq0Lz1KVtDOsXNms0CceK92Kd/iNyWEgZJFUT1OCB4hoG1BNyHkKezsG1TBWG1Hlze7cG5IbN5xzMJdOUc6/6JiNplhEfIeOTLSAT5ABa4VMFs23jahpWz+LQSWdZktGOzwGslRD2GIJGR1LmD9hYtTcGpWAUZguDQx9a2RVkJu7YDqVxJxNhKlO5p4nNqPAdOERxu3ccu399JbuskrScOidQqqhmbLoLvFGxKb66mPoU+vZpl6fewKvgkdJ4Ucz4AS+R7CH6/lAUFcwhs/AGbPzAGzK06/vWrCyajlThMbC1IZ+7EGygQ7XU7HUbzZ9hoIqKNTPymGfeQWmSS/sBRtXoJlvm3KXbO47A/7GgvERfZZZvAamBQYgEq53oaDi3Bq7abYdo54GOCrlJwDOexcZl4DL7GgkINnU6dPCl9A2b0b/yTVZdJ8i3iwt6x6MqG8+MamR1ApWxrwKf4PEaV4e/Y4v9r8Qb0snbOrNTgWdfC4r2L6Xqw3x5MEGDeNT/krY4oZy7nGd92yio/IT7uKQCKDRbePhdInekQQx/Q0Owk5T5BCs6x4DsFq1RN/vBMWrwa6JGZwdnLHuzrPUYPV3i17BDP6sKpWtrPMA5qOcEuXTCbdXL0YSFEeWWDOQBkal4uT8NSXg3dCm7H36bDdUBuMHM3i86NwfNmAuvv306+AywdN4D13HED72/HMV6p46OlZbib/2EnKBH48LkPceucSsmBXFassKsmQV3BIMdufi1149My2JvuArHPizlANcHIO7q4d/ZF/vpuLpE/rSZ2ykq79pnCNO4q3MjqS2n4GJR9G0mVtZF/d/lzvcOBwmFZjPM8hqXkBHg9BXJHJIKAY6cjZpmZMVU9SMzeTJGqRGV2TwNJ3uVYR5VT711Lt7IcnaVF3PADlG2i7dxEfC6fxJc0vEKrkVjNkPYppH0CgoyOUF/KQtqId9VjMdsHlgAkgg2bIJJ0smrGMVRu70d477zldGVocX71EU6P6ed5PyrNpWTTCgqTmvmtMxeDu6Oo6NquBd/JHD48jFMnY0gIbECRFY3bjEK7fp9NreCaXIrzlIss9DBSBmJuu8B5OKqfpqysALOzjIrUeei8vcHFBULvga5SZsor2BQJkxoO0mSZyvHGqazSLxYDp6ErIeYpvjiSzNxTw2l3VLO95SvsVhhKD7IrxyLvaMTn4ESORN6AoUpsnUWEF9YDsDzHSuSlRcxO+rWvWVd3Aeqo67TqNQTlSpG42DBbXwKFG8idUdjqGT7mFMeaJRx0O4jRX2t3zTJfT4qHLOZyxEYatAd4xjIZlHKRWGPu4HkHgY7SFPZ/PId6fzXqNWoR+FV6gFRJl+ME8teXEu+azFP3nOTRAHuiSIPNCZe4S1x95CfSgzswSaegUqnwMGwnJcyJPZ0emOXm/xgDH7W5cLijGE/JAZ4r96fNt8GuflCNkRuHIhj8wA+8OOocB1teB94AxIDJc1o5Vzq8CV63HI/ICoRXBjTWBHF/AxCeg/ZpJ2Jd0kUwz9QmrquAHTeSmbhxEI5AgHevnd/fxSWW645PURIUi1+7C4GeLqKivHg9aZFtvPfGReJb3yFQvZwyZ/ug7Dum03QpSpnZMRPfGr9eMz+xGMw6hnTU0FCuJ3vTPM55zeeT93oV534zMF5sYf+303FSSAh11zPTxaHftsk5GpntGvX5wWBUkJsXjE6vYvqAY0uN7YypCMcmWKlY8hedpukMfFqh+nN8d2A+ty8mAmWMXnQO+BFBHUz7z0tYmBfBoWcqWe8NF7ne104hWLj29kP4xZcwVp2Gwsde7VViCeDPagcSbg9m04GxNN5jbwOX7PQLUam7yGkcSYTbNaTWJfydNw/g9cgSqpM9qcmOJt1nPf8E+25ZY/E9GEh7WAfRs0IwalLt6ht8FlGSWkajoofPLwXwxFtDQOnBozt3oB+h4FjTSuLf7uJmainSl/ufR8bQWFpDSnG/6oTfxTgSJl6x63dkYzqy/Fi6/YbS4KsiSJMoWrmVbMBl8F5O+s3GFFDK4S5QWdR8L5H2AZgCpdxIu0FXxg0KQmBjR7K4ZrJZ+XvtHOk6jJvOdQRGVRCTesfu2FqZCufRMZwe0YFFsoZaBxVYPweJjINePTDnAN6ta5F/O5aIB/up3DKJjECvU7Q5QPLeZRgUbZhGuIq/d0Bh7eTjmDZeqJeQU+IOCvu59+8gUlZqFlmpWcxwe8quPsdwhHz/9/FLzSCwIY6i1ixCQ1P66iWCwILrGchKA7n7o41IBQMMGIU9VhUe2fEk7ljI80GVDPpJdM8wus/k+ODpeHnCtP1Z+ElTCR22GywpIFUgwUZLThjuTb4s71yJxFUJzy4Q13u6cnSeZv5Y+gdptxYQemwdLSPmQOjd4ge4GOuHLr6Wpzd8hr4rjDHqEBh3qO+8fGo9iSqMYt4TWyj0auGfRUMl6b6Z7F/xO9UOOiT/yOv3hO0Kl88Uknx2J6rxF5HYLICUIos/oR0GnLAQk+1K/tgb4hjobe+nO0VrRRnpN01EnX0Y8z32qti5Q2bRRBcnz11B2z2YIG1/nREB2eEbxHW00OGkRWFoE/fblTsgYjV3JIlc/jQdRXQ+P87cy+yOwTzaHSTO71rxmbx4OYGZlxJRRBUzwd0+V5sgCDylhfp2NWdKYnENjYZRr9N84yNmz36d3y4uIS5vF2OK8jDUx4OYFQ2rLo/V1ufIzUvCK9yHyS4magfcLm9TIa4rd3Nb2cYYB7BZ7cG44y2TCd9lxWnqKSJTRNXP+fbXCEsIoU21li+zPAhzPcp9W9J57qdoVD1g8L8PZfUGULgyzCmHxZ+e4u2v72L7qVkoFv4GLO/rP00RhLJ1HB21lbS77KbROAds0Tyr+pXfZsLnTbF4Z+9mV1kP81eE9LXLEtZxYnsrMqmB9id+4rpWaqeEbLLqGLPuU375dSqhuydROtHeivN3t3pCh13gZnIFzW7FDA3cSlCy2INf00lKFV48I2tg0+N/4u50ktfZ19fW7DEcr8ZuVMF5zM1YjkHjYdf3+/JcXgm2MSdiMAVR+XjWr+XvbNz7OnZw9qyZ+vA/yX3jFfyt03mP/X1tM02fEyVoGBdxjbeGV/GzYD8GO2MzKPMI4NPIt1niBjarGQJmQ8BsbNXidxx09fgVnkGW4gB6hbjPajyHoP6QtV+upcG7jiuTjiMPNUFN77E7cpEpg3lzz0IKrouRjdFB9mu2EGM27/77eUw6NTcEGy7LzOCWirx9N2/sn0dHcSCSgO14llTiaJ4DHhngkYGlso6U8DqqzVB1tZNalV5cH6eJmZTjMs0Mriih3Os7Xml+gQTBD+pPQstlSHqbrvJCLj+5Cz9HAzsalHQ63CJBObaP7OHm2Urx1CzCD00kMWc0MokMZub1nXeo5TYyZyM5EnH9K/mHlEfvkszNsEQaQu7ljWaYN+IvkpISeKw4mLIXHHi9YzYnPao4bLSQ5T4Dem9LgrKG0Wq4c9PAvz98Ht8lXsQ/JgOJONbmGSEvKwl9QD2OCgtSx10w5GsApDfNVAb2Ope8lITXmBjkWgPrJo4W/3T8JG/Nm0WeZhzvlB/lij5JBEPSPgGrAclRmNnuCHVpNHW6s0HRykvz3OHiSph8iSBVLfKYLtReKq7XTmXuVhPbe40aztR8hLH5JMm40eFwin0zx8FZYFYRDl35PNEcxIlTIymb003r1HMkePtDzAcQ8xQ7Ln+NvP467i2+yOKiqNW4cFn2fr/u6c77OBb+iD71ABbFJNLqvPHoSRHvuwSial/F1fVVFoflUv5sO9eHVONQ8gmU/4ICGFYkx3wjnuhWNUJICG93zWLO473EB4ULJ6RScmLy0J4aTmaFD+pphUwFdJIgrh8aSkhZCMXzmxlZm4xuYj/Rqc0WyAsNoLLICP3pbn6LLOHbxf3jIMKhgBe1RvbmdxO8qQefGVvh0X6wz1nRjrPSitqgZJztNaJPFUPqTdGqGZicM5xGh9O0RN1kg9VCqV+/EgxjC8sFPb8XheFeFMZBiZF5c3ujWQo3Dupl3HrgNxS/riS6xpVRHrfBArTd4kvPdXx374d0a+6nvvMO5V6/Ep2ySswbCRwIL6HECkfjRvHhQ09x2Wy/G9ebviK+QIJq3EXumXyDZtOr/A32KW1dNH+3BD+9hA2r/kStjhHhzd6UCIpmJyYcSCBgcBjed+fgLC8H//sg4VXwGgvAj+eTGH46nvolu9jlYWLkgGOrVFru25pLt+ePLP32WW7o/wKuIwmcx8kTrvifPExXVB0Tig5QGjgeQRCjH22yMNYY2tGqPZklsfbnx+0t9Sofzi38BtRB/J7kiFpuY2BZpSohvTmMR8JtFLkvY5g9Z4GTUVWcfuh7nvOyMd2vk4quRfgMWBcJCGQnZdPtEoDc5Rs0AgzU+AuCwJkxZ1ixZzaLWv3wTizv/TuUDFrIiBE2wr95Dz/nbtzNl/g7dYC/dxNnX1mPW8h8/tXYSIPUhGfoIl75cjiF1X5Ywk00BaQwNDaQ2tpBuAZXUeyQwuHaLDwcPKmNHI1VpuBq3KM0O4oEVmcJCHXHiAl4Eff92/CQuGGxf9X9X1lUqn/mVfg/V/4L9v23/K8VqVRKTEwvorKoE7Oxi8MXX+GGWk9efD0Sm4IeSzAgBk9tgoKw0dXktYVx2fEBwv6OofQGwgu7wqn6LY7aYRX85qDFP2k2ZL0AuR/BkO/QBd/HnVEjCbvyBSlZKaJX+N9gICCz6Hly472YexT8tGIjgrP94qtE+TCX4ipobhnCkJKnmTIiB64n4uw6hRLfOupUMsK3HOXm+VBmztsPVbshYDaDXCex9vutGDRumOZo8fFXiR7pAO5DeTi3lbEHfbni+SpX0pXMCvUFVydYIgaXwgvPMuJcOqOz0pCHVyDcZ4X6U1B3FIb9QptXNy3O5Xx2Mh69j/OA9OliCfLJIunpzWz5ZCnjT4xH+oT9JvThGh84OxmX/EiOTTiG9EGx/kR3ErbL8bgmlbLZvYp8XTCfS/onK78yN5Jyp9E+LJ8tHi3UyeyDWo9W+GK+MZzc+hTeD9Hw2HQvUQ1maKK6PZz6IadoR81DW9pISJaQJJeLdpUtVxCCZnB9xqtoq94g4vpOCovLwPiZmA/FwQ+kKsxyK91qA/VW+KRJy0D+TqyTL2UlYZTlhrJmVD7ufm1ihb6aHbLfqZ3jRav/Rt5wA09VDghi4N8qd6OxwwH3Tg3jSh4jzLMKXfc3ON16Hsp+R+LwL9qtILRpqD7pRI1ypB3r2eJg4+qgq7h2DsbBNgaJXAqnpoj2o2mfctVNw41nPkYjwHvFcHm8FjxDe4M8Nq6YXaky5hAC/HU+FvOIoXwzoP/vO4ZwXn+T0JHnGCJXk2cNJxm44HgvE3OOYLNIGKvT8EugO4a632H4OvCZiPXIs1QHVONhNZG8qhKrfzLYTFD0HVgNCAmTcZfX49H2MoLJAy8H7Mqv7Qpum4y87S5auTUIQp/NikRi4tC0QxyadohZDqBp92GN+xAIWgxKkfk0jufR5OlJvfwZCclZaB6eA4NFmwOh9ATHJxzHvzaB+JKHMagCEHqtOep0ici6OrEZNJwpX4Knt4KBYcIq1UG6HCs4nhZJZPQd5LJHRSVO3AvoG+pZ/uzPbCteSOrX/yYorgr4ua9tY+NG3GrL6AiXsMXxJ9Kdga4TYqV7Oq2GO0TXVyIN6uaJFb3WZg6+oK+FrhIkSg8QJHSrSrk76KRYP8DWoU1wxeZxGV2NCRr7WYoahYb40ueRWM24dH1E3uXpfDvEhyf+ERwSrAri7u8mwTcbJ2W/orHEKnBIJcNireLpP0bQHubMAmyQ9hk4RfAoTZQEn+S5hSN4MvIXymyJELJM/AB/2dSo3MsY4gLjHEBq1YGuXFRjOoUz2aEI9+vJ9FxPwrnDiXrsbeZGzJ1D/UQT5m8N1NmLef//lnogyKWd54c8x+s3V/HMI5fpDO2y+472UipDLw0h/+VvGOEAl6wdfXU2mRq/P2cQYlJSMvwzfAVEu5KZOQAMvrWZW2OqecZ7EOvle0h0mGh/AlYzi80jke2fCtaV7Jl/vC+YBqCydFC7czLBLRq83l7FUjcrGMpBFkRjkxuzNo7hyvASdizcSJLDtL52sp46npp2gz+iy1m0dTpt2haEVf3vjUZNKtcct2Awq1m09w+UkfbUtkBJJe8Ed5O1zZHgc+9iGPsg9PNRSaCSaXovzOtX0KXsEYWQqevED2DVZfNnXDb1U9yxVr4jUuXtmKGwaOsiTP51JM46i7dZtDyTWzpw3zuZ6Bp3dC+8x79CK0Wr16jR4D0GJ1szz3z6NPUhZZxLvY7B3Uns+9qTYDUwMVTCpMRSim94sr8ljqPaXAj6FFQ+4DGU7I3PE9Dqhv+CnYQFtiFInxCVYoXfwJQrBMXdR6Ali0F/LqIzvuw/xsuQkFs8/MNWVuWN5+qN4zz8D2Xf7k4njLJO8ubuJTrA2KeKne6Zz8GVe8isTWXxtsWcn5AtsshN7VCxnUzzPCIVeq4MqsR19jmSA+1zh2Wl3qQ48QaftcE97ZA7TioqXtvv4BYaycWR6/q+6+mBqIRsuwmmdmr0w3lz1Rt9G0unYRcgt0pUXTkEQMxTOHY5oVfp2bDya5zkB3ibk339NfW08e2ybRi5hmOXI94eKTDuMEZTNwl/jCf9cjoNyRZK797JTkVEn+WT2tzAh2sO8HqDI5E1lxjtbKK1ZzXu448CUHPkGx4e/A4Xyv15Z1Q1N/X2Skq9bRGTJ5zkYHtT75gRRPWQuRN6mjiV8w6131wDwOgyEoVMDklvQdyLIJFzpGYr1V5niC4IZ86uOQiP2z+sXe0juN5Ygt65kX16A1ucRzI2JoY9N6ayd/rzVHcPZ1WbL7O1ZmTdpeAqKjCkeoFJRyYht7qgyExg8MKzdv1WhK9BP70ea1UW+3+ahXl1v8WRIAh8sOc0jZXthMb8jHeQPVAoIDDh2AT8q/2ZstQXrTZKVHyDaNUF6Nx6OD/8PBXB9ez0HMpU1YB8mJogztQ9SlO9gUlSB8LdLXB+AliNPOMMf7SBXtlNg78aN3t8H58aDwJKgrkz5hxtC/exUL6p/7ykDry0ZxAxN5MoHjQTm0OvouXCcnDw44TxHQ7NyEWhDORfKd3gORqcImD6LVB6cubK12x6uV/O8aj8Q7tjK3pqeO/JXbzzy0RqX3yB0ndCGOAER4cskLzqFmzYiJ5Rg9FJtBrydvTC1fwoOrcKws908fyva3B6V9aXOy8m4HlOR5/D4G5Bfvkg+k4lywb0KzEIvPDhC9T69VCWOJPBkfZWbpvOjsNyOAHt2MeZ5VLLtUYX4gf00OXrifOqU3ioNuP4j/ckQKxTBn/MOwQSZ24WzSFJFfG30xAAMq0T7Z4RGAvXcs3SwArFLNxUbnhykdDgLZxq+Rqbnwf3hFSi7swTlfLAam0nUe57mNbxAgGTv6Rcap/r9f6uJi6diqEgYxKfdTzM+7OBG6KSUmlpB6c59CizSLyZSJcLYLVA9R4wtaM2yznuDz91wLBK8JPFstLU3puzTwQ3hvm+TtjrdxjdepM32vxokvWTndyoZ9OQf/Pd2YfparrDA6H5YHwdVF7schSoT7nMsjMW3KtvYmmNg79XTj1N/Dg5kiPF93M80wuDUiO+MuJfguAlVBbqefzCz7Q3uvL49w/SPvqa3TWHCbd5/7tHKQ4vpnveIdykjXb1Qw2lzDg+E0V8IQ/dvYFjuvFwdT0UfAUzchBk4axdcYyLrUpclBb0NnubLWehm4U7RCX2tag8kqQDRpIgwQoEtzoTJXFiUNN7UD8a/KZQJkvlxtdGgvVqoJbhy2+A7B6YcBKMraQ0fYcko4AcwwHCdGkUG78C2xei4lyQ8pZHJ/U2uJg+Ghd9Kv9IT0faguPIF+3jvh4zUnM08+yrcaw9jOynCt6/K4CMSWfo/Ieyz/PCZySf7cF1ylnmDc8b+Jbms6c+w6cjnTd6Wnk0tEU8p968qjGTXyZ9eitPb3mD1uouYtST7fqdZJJR0RhI24VMJM41YJ3e52aTmXOcK0O3YLNIUE45h6M5SgR1R24HxzBaak6xf+6vDL46mGc/fpbSdwToKhZt/xRaAFybnXHqcmLyjL2o/OyftUQQmH5uFOd3TCAK8HtjOewOxUNXxsQkWKusJOj8ZBRVWjq69vM32FfT3UhpuZbOSgNt391HgdrMfXdr4UAKtGWj1T7N0Ik3OF8cw5pvSvCO1jL74f7jzox9iR2jmihMzuTdIkgo2cj8mJGQMRJTITR6FqKLC8d5hJnAbbdp0IXQk/IzyjHi+j+zK5Fj6t34dKvIqXHF2ehod12femzm0KFJFJ0dQ9L4KzDuAghT2WBcyC8d27HZrLg3uWDFFUdFf9t6YQrS42/jF+zNp8OrcLVFMEBcx/6O4ay1/YV24hHUw89xl/pZu+OWhlVQGJGL0ugDChP+nVehNRxck9A7+PHbrpFEXk7i9R/fpvMfdgeCIKDSq1AYFVwa8i9UpghgQE5Avxl8PGUUCR7uFA35ktAB+OlvDU9Qoz2Ds9SFujD4vS3Hru+w9EN0Z5RQaZKwvVWKQWVv4xkU8TlKb2g69AKHfIuYukACF1aC2h+FbDJvBz3DBekYFn64icuSfHD+EaLWgkcGYTdO0T2lBE6nM2PTcoLH3IYh30L1fnBPB0M3xaEl0Av2jdKW2R1bgQHfyEYuaBv5behZHrBuAEBubGFsRh7bvJtwdtYyIuEMEr0Wqqsg8z7cw9/noD9sbwhn0UIxevCWRABjOwgCTuYq1qWvZmt+LKWbl9AySC1al7skgks8d2t/Z6JXM4vXzSP3ZhKq7UHQdgeyX4KIh1gYcJVpUfncHxVGcOc8lDIlnJ4NtQdhiRF3az2LX/yDJ95fxLqv15L42UEGEpZPd31CbugJpLci+UbbxtKhzbBNjQ0vNnpW0KEwIbEI2ASJuGZb2AGChLAP5hNydhq1FhONAdG0SwaDqQNyPwa3QdwJm0TPM/Xc9DjOoaACfqmfDy1RcCgNH593eHDmFfKNV6isciVf6MCUN5+qjmgCnPNRyZvZrpdwtO0I27rBs92HZz2PwLEpkP4DggArq725tlvUeje5N2Gc3yY6UakD8Hdr5oGXdvJE7jRS33+PpqA0BEHMmdbtl8qdcakkB+cSuv4PshVJJI+6CcZWvH3HcCCwg6bpKdQN3cLX/me4oQsT56nKP0lvOcL5+0+yo2oWC7RvMkW7ClebA/yd3TT7ZaRA6NB4tJ99yoqxx2iPbgJRM8ZHkzIo6RyOvvxuovLdCQkeh+DeCK2XsDr4U3LtMo/8NQeA1EuLOfbYgDWIOoDVNeHUS6/hNvc4BlsT4/T/4jnE+fHkuJM46p0YXfoEMefHMG/SenFP4CrGICccm4Bnszfa8kCqHTvslH2R6nzmC07ktLrTrG2lWFCwdMDYj3DJYd20szweU4r68wcoVBwlpffFJQjQZHLgkX+/Qm5sLtsWb0OQDAj5O/jxvJeE3Oc+wS0rgYqgCqR/y9ico7i7SU2HoQOXmRd5I6qQhh4tLnddRumg5Lb1BcqFDARHN2rKR7ElbxSTBwhT6y/HUrU/A+NgZ7LrxpPu8xocPwkTxPjKhEk/kB+vYa97DT9a9eyS9RO76pWpnG67jkVqoTzoJr7G3jptIlgMROiq8b2vjFydiWfaArjqkwcqL3FP8vecYFDg2OXIn/JO9D2efIx9ibi6Bam1i3tmgdHSH2eQxkRSWq8mpCWfsrP+OC9s6ovbSAQBZY8StVFJ54N/MljTjtzWb/8skUnIic8hsCMG2pKROdmTWLtK3mTTt3fwGVdFxGw1+W2PAB/11VvUBlqcaqn55Glecmln5QR7dGyG4ixBaw8wvQaM+r3E6r5kyQAbz9Wegxld9C7VRW2E6htwt4pgn5NlP0MHXUfneS+lYy4x0VMHQgggbpvnmq7zsh8sNVlZhoYszCB3orY7lm4T4AgGR4++9ZBNkBPelcV2X4BGNs3/gvW3/gU2ceyoBBinBsbsQWqbSOKlj7HKapAZL2NryQDG839jscM7/hfKf8G+/5b/tWK1WmltbcXV1RWJRIKAwIpiB1Q9KhLjZjPS0UJLeybhiPmAGiVevKtZSpfHHVocZ1GtjGbgZNnu5sTm+74jumQ20t8nMDo0FYLUULwevMdxRP8d+0d8ziuTmnjCBSpMi8UNbtbLUHsUtbmekd6d3L6YyMs/Pgxv2adY3954gB+Cn2RSzf08bqonUdcOQ96hnjTGXTqMxCrh4cQU5oSvRdlxDowPACCnA9f2YrokDTw39iUazV7gchOmXAF1AJKjb9FRrcXqCDWdUVgkiCxyix5kGr6Ye57iUTe5+etdhBgcxYVd0HzACt3VbA3MRlWtZe/eu7gw3n6RDlCuWcFvjePovHs1aa6dqM3Lgdi++mlNflzIj8TkAKMUqbhLRPAzWxbBrtfewdUSRUenjFyTF59L+8GMOR1WTJeHcsBdz5t1CgYNt1cMju/ScOlWHGCkgWCQKkXrDsAt6FN+ffA4r2bGIvvyAyzmScBwMYeiwhVDw2ZWDdvM5vIeFo6pIt/FCbKeh6IfRDVa2if86VnLEauMTx74AaXU3+7YVoU3helL6XSVcGAdeC6K4JGFj4NzNAfVO9Ep2wlokjCyKQ3X1AbReiPmaXoq69i+aDsyszMv/rqYUqmaaEMnPmVidrFKaRx71y2lw7mDcyuu4ePbyWt9IS9Y5neFyWuvsbLiCmMk+6novB+8e6N9ztEIXABg/i8rxZx9D0nhysNi4u6FHUgECb/c9wsqvbjRn2V73e66wvWDGLnOl2tp11k76QD3WZexGLBInRi5dT6OPV54Nav4zUnHnPH+Isu8NYvBsi5SHGX86H6Ee1AQKZEzT5D15WWqP5/KqU4YZvmZGOE4Gv0TQHrfcff+Op2UG/Fc8WjhWmglT88cEKow6ykKhp2NKjZduYtt/mGsCVkKan/+DlpFV7yN4bgUK1CkS2esY3/OLE13OWPTijnVYqU+dgcKCRimZqPsqab1ZStJJ79Axwz2da0gdIg7E/5uaLMxu9wb1Y0M9ow5z+2gQuKlrnDrDcj/jMb0C8S1tIDr97wxfBrNPr1s7N5ASPeOB3ht03ruLolDl7UcHY5i4DRlHbgPZapiO6U751EbUs6Y8N8Z4/MWb01YJAb3nSIovPU1nw9/i6+OJfHRW/dw38oj+Opr+q7riG00v+efw6PrIlM1Q1FYuoDbSHX5jCrdSEe5inMP9dCQsolQ1aS+drebdxDcs4HaUjd+Nf8GZbB9QEqRy51byfF9n6CKIFz2L6ZwRiU0nBaVwCnrsAkSrDaoR8sd3SSUcvvfRmbDSup0pbwcs5P7XboplAiwO6Tv3qxvHUr095FUJ97C682dPK+2/10P96ylxs1ISttIfhYusNxmz7r+n8pGOXhEadmT5YLe0cRat9v4Scb0LfsEAcoTc2jzq0Z9LoUVBxyY81k/H08iETgz+gwys5qggWvo7mowtqFwimF/wSPUtblzJnUPzpKLdsdHkFBabwOrQEVMHj4hteIcKxVzcRV7z+fToU8hM8vQdFkpN/vwuaKXvOBs4NDcv5C4elAUDIf0A94LKm8+9C7mhraYmRYl01y7kFr6Wf8WmTM/LNhOeOlIVvxaQJTPRegfwXgL9UzVeVLvXAwRjtRatjHQpuVswRDKvh2LzbMJ6cgrSG32GwhBpuadFhg9OpPTIcMp6PkUeLKv3sXWTGRZKObwMh6NbOGEVVSIGZSh/DDkPDa9mshdI9nj1MVdD+7tAzrilDYsky9xtsObBTsXcGX0bZGg0lkAzrF83ZrEs82lJN6IZ8fxsVherxHVACEiC/KXWTuonGZjeBW81wAXBSko3cEpEiRyMqqbaDo9k9zcUFo0/5n/qNHqyh+dUGdUI6UfKP+7bLsaj3uZjOfHS5gSdazvC8taZ3IhfBu+roHMCrzEAr9u0Q7QawwEzqeoaBMnU/5kSOEe1NdmMTHdvt/j59rp7hZYkn8/XY5dSB4UwHM43H4bqaUZ8KLJ6SRrhs/BYPUByy0Y/jvIXcj+5Rf0cVmotJ0szSjktnMqeI8Tc6MmvQU2K6vnnydzxjnGWsDR1mp3bKOlB7PLaTQC5CZIKBOkkPUSMgd/JuCIstUdo8M47tjSSfHtB+z0iiAmVEFVjwpP9wp2y+ADAag7AR153DJ6c3b1s3TH5pL2eDWtZnsbuMjOIhSnhvBgpQuJzkbUj18H/5dh4mlwSaBFK6c8wYuZo3/hoaS7gWIRKJeLQc7BPbuY8sJWfnlrFWMkCpyl14B7+/qfYXyJ0Z87cCOtgD0zN4PNRnNzM62CJ+d6wN8GAT8vo867Fcmq/nPLoBzbhRHkJhSRefcWLnj62uV2rFAGcV3pg0eeJzW5/tgeDmdgMWXV4lt0i3+98icWmYXuAYNIqq9hutGNjvJgUo07sRrDwWk+jN4NLvEArIkqpjOsms2XtpGqv2sghk5FewXl155jcKaY/2aE/0FQeUN3JW82TsZQmI/BcQ/Dx7xLdXs60K9iflkvp+L0WMpjyvk5sI1d1v7cHhKJhFPDLnAt4RbR3uW4ysFmW4/gHAfGVlosdWQO2Ypn+2S+vHyYiRNhshTQ9trz9p6kkwR8peCA/bO2aIJZ3GIk5Ww6TsAwV3spcasqhldWifNPWvE2nlSnEAdIWjJ5+6mV/H7rLYTzExG6ChkW1N9WEKAuYiSCaxlvjVuH1KpmIMw4SJmD4+hWqpxd8PXqwMWzi4HKvcs9V4g1aVHsmML3SiOx++xVKTfrMrmYW8qJpDCkVhX/1IFOdmhhaGoJr2fVUKbaSF7bJuIGrMskLGTl6EvceuV+2lyqsPTmtr4kbGD9qmSk0Wdwm+vIzMGbwdhPl7tVuZBdh5JwVi4lq+wDpk2zP25L0Fh8726i/uQefPP0NPWsEoGUlutY1SHcm/I8Nxu9kXw/H88RN0UHh7MiTBQ6cjdlu6cRdWQQjUMy6RjjIrqMxD4nzleA0ZKHi6YAc2c7Bqt9bjuzzJdvD39I2Tkr6hIfvnIz8srfFp+IgZfuWDmrF/xMvdoAvNd7M2ScPT2F4tse2J7+iK8Cm7mkHwSah0ETjLHsKvE/3INGpyF+5hlcIqrtjhupdMYkq8e3xYdFpfHcCiu1q/c06mg8OxizWscH8fkkhbpC3XGxsu4YksAIlsnqiNt+HxebwkjatpiQAe2nCGfxXdvGvYYOuuRT8e4F2RtbT5EYsI4ifTg3V+3l3+G9JKRuUYkiEQQOTD+AZ1s8M3SpFMkmMAnAeywAEx1e5C4NLByexbvzs/i2awZ0FsG+KPCbwbASf25fjuV0wHHanBvoNo6AS09B8U8wM5+7XPV4Scwc3XOJRF975SjACVMDlqgKih00qGtmEyOzz/MZF3MWV7OKNcmZvCPoaOjNufq20x986wwJeSaO+hXSKXfjWaE/NPNRhw8mWQ3dYSE0Oi0iOsr+uE/tHcGwU0Nwp5nwxEtITamAqB511eVw8p4LPNkIz7lCjbEapApRhWxoQpBN52byTSKcDUyrqcXd0QUCHhGdHtRiH8NH3KR6yHkesRnxbxg9QEsmAkx7LkfiDoycfQYPt3LQldHmlILr9SzUZgnnx9nIHfcB0x1fZnVvO7PKl7ULNyGzOHHPgTMYbTKcVS7QJqpl6pRxaEsgvDyQFZkb0VtHA/12r77yU0xKv43T/jBm5P2LA8ukfWsEQQB15Fo6tYc48c79rHHNZLPqCwShP8hYa/LmUqUW85LNdDl1sbRre6+hvFguNz5C0VkjVpkZy+CbmBgNEinnrRk4be8m0uSBX+EvuLo3Im8YCa53AXCk9VtypudgUVcjM8nEdAd2RcC1xZUIaxT/jlYic7MHzfb7Sbmld+OzHA/UCiujKt6DivdgmY12p0S+jv2SQMcmZGfisQS1MNB/xVUC72S+Q9fpLmb/+1NkrnuxA/tin6Yn4k96ZFrmxHyMQTMYeg14o4tkTNt+P6fvusjHXQaqXX0G6BzhPm09nlJ45sOF5OTE4rEnwu687wt6i6JuGae+epFT7ncx7VsZlP0GwO2gADK3jKVD08WmSU2U91QzQ6aBwZ8D4J61i1Urj/NyTAFuze40K2eA12jxA2Ct5dXES8wu9iE1K5UPmkcwasCxi1VDeWJOPyHg73dgj8dIhgU9gMTfhRBrFzscz7GsZzlpTZlgaERjrOS9Vz/AWG6EV1/r73CHVuwnKIvD60ag1CmJFXywxeWC1yiYKKZcuKwP53xVJVX+VehlJkZJ0uDW61C9F6Qqjhui2N+Vz/1RZ0lz2gHoxHm9F0TPVk5katYGpjm3kSqT4kSb3T1dZvyJUftGUVs6hEvpl9DOFdWOu8wT2Ne5haAeG6+uewVT+g0clhSBXLxfFxviaDziwq3JRzn9wLdEmxbzSfMDcFsEQ9rbbJSUQIPbFAZXFBDTM5J7W7MAMKkiWeYEp/VQHLoeH5mUjbkfs2Z/HmDDIhj4yFFOdH4C88qDyU2ETldvCJwPmhAkEriluZ+ItVK+NW7nIa+zLFE2QjMgkXPdFo9XyU6CdN6kBVdhjD/AlRI9I2PnopMfAZ9yarsCkes0bG5W4J94BA+3QSAIXOx4mFoJmOV/MLsW/PRjGHHrDShezxDgrWAY1WqkxVbMhNCNYNlIH9jnP4uegqN0fbwCj6pWrtwYg9UtkUmIyj6Nop1bX2hwCbxOyfxVbJgTDJ2Pw8wcLD2tqM96sW357wSVTaLW8zyJ6iFQcxj8xFQ/wwrmErdrBqenZ3M+ZXef4stFUskXg2vY0iml3kmLZfjrWL1NfaRgd0klU1t8aa3x4r2X3xfH3YB1alX7a7y7OY2Eokref/F9PB3twYKqrkgeqodLzk3ol/2Owc/Iwt7UF4IAQQo99cEVxOXGsezycKSmdqB37yBVkWV0plRTyJyJV/nI2QlHfTb0rpsEBIafH45UriU+sZB91fFMdIwEFVwwvcqdmnZizOUkFJ1hVdK7kPsQxIqBinX13gyx2LDWSTl0cB6xS8+hqO8nNj4ZWccOH/iuDjCBRNpP1JMIAr+t/A2JRcIWFyWV5gqxYtI5sNnY+dt77A79kFH7p7F+21xa3jMwx6wTgXZ1AKj9eH7yZW6NOEtjrRJn8wCSHtBpaEWZcY2mjlbiDofhGBDPg0Br0W8kJ73GPlUGPSGxHAz4Ca+O+5jd+zicrXW8cnAO5psxeHzxIY+5wRlrd1+/wxRlPBmoYl2WA6dei0SfMpTUJ/uPW62eSnliKE6Gb8k4Morqu+xzqp4PqqHdoOSFlBt0q7u5V2Kfn9FJ6CZEDuOuDiFVLkEbcZ2BOfuCLXso3WegKKGHXTO3sFq+WRyDprd4PC6LieU3yUzMYYe0nS+E7YAI9p0wJ/DzIS1ajZZvfniOWzOv4DErkzrXk+gMyTi1TQNslN/ejUdONpWFUgjtP69aj43kBnQw7kgU4YVD+f2lf7POwwonp6CO/o4Vj//Eputv0nXLhNrZhf9byz/xjv/T5b9g33/L/1qx2WxUVlai1Wr7/jb6zGiGZQ7D5tZGQlQV8lHNfXVlXflcjZyNR6MH7x2Zjma4vR++oJBQEVxB8h0JXuVXsHQNBvfBsEAMvsbZXuFXn2oKrWZyZV54Kt3BXAHdFWDpxiT34sSinTi4NjBNpqDuH9ZhgmBl2sGpZFwK5DqBDFuyBW6+gseQszz59bdoOto4/NSrfJ10TmwQJrK0HM23eOPHl/noyhrW/ZWIMdDMVzKNeG5ASdw3nBtvw60lj9DyA9i6RkDuV3DzFUh4nQsOPdT5tbN+zff4WYLYIFVASzHUHgFzN1n776JxbyiWOYdIirVnyQL82vwpB8I/5hktPOEJNwYEowHyYl6kea6amMDLPJLxBj2GeUAod/u9h/HQeziqtxCzLZ/TE+0Tzg4ff4Wocee4+Md4ph8cS1mCfUT2vbRcmgafQij6FK0uBJPFJAJ1dcfoUgTyVCPckVpJj/TFzb8DTF2ics9jKDV520jqyOFbpY6ZnnWMlDzC3F7GvxhktCLNDkO1YRZByzfREGovmRdkKrKbp6BQtdIZ7wKeYeI4MHfTszsOabsF7/w5nLtwFyXu5fwKIHdCabxBRzh81dLDK/f9ilViZJpsHYTdBzInbLVyvBq8sEgtPD9sG1XGcAYCzp02FaUmCNHU86p3PRt6wmDM7r76DzUf4lCaxoHqAGROOsAiAn0AEjmjTAe4EQTjq3oQAAfBPri/RKvlpns9Uom42frbKihM6UhudwgYnTmbfhKDsp1x7mvh8ggwtZPcOpzsf71C+sir7J24jxabmxi4PzsPkt4GxOCFqbyE2/lRuD1SwUCwr8W9jdLQUtyb3anSyUVVbG+RSKXU1LijP5HG/ONxWKb15g07JgbYGbWDqIATWMYH8KN8KR1ujiyWOveF+lw7svjJG6Z2d3N/wnoWOkGX5lOUpT+zMHorjw9Pw+zQRPpJD647ltOvzrPxiEXO0dvxqNOvohRAwNrHiBZkKpw6nJBYJbwx/iBSi5pHBwzRXNlgfPLW41uuY27+qzT5dIF8oxjkkMgpsfojX7GLqgI/vI6OZvXSb6HeCyLFkIVJpmHP75NwuRNJt4OeM60uLHbo1x0KNoHnPnoOgCYg5dE94ll3lzF2/EH2Z00nVjuHL+M/Jk/W1tfOyZyH7+lU3MLruBXYhU2QAv1SyxlkEv/Rc1Qn3uGHB3/Ay2UECFKQOoAgZdgdZ+Ydf5yD4w5zptONRyJ0A7I2wZhrs9Cc3E8dz/Osdyiv5MYysFTbXLi29A86nLp4RAXJCvtg37CcJHosBczyDGdr3XkuQS8d438ul4BTFhihH4nXN4NRP1RGphqSsV/4nRjWQp3BwicfDyKwwn6BrtKXcGdODu9UabD963X8hnSC5X34S9y4WCe0ceCXaUSVX+NPr8dwX36Dxwd2IEhYHlBF9ytvs87Lynp3m5gzsjefmlQioTCqkJDSEGI2z+XoqNo+MOPbiAsEx7Ww/OTjXFjviduwfgWzwsGL/WVLsdkEfMJj8AvcjYNL/4ZMghVHATq1nYxck4XJNcTuum6TQsmLARgTcvng9W/4tns+A8G+Zq2a4jGnuCfZxL0Zl6l19IUDySLgPPUqEsGFuX/OxcOgYscTuzBJWhgYC4zlGjN/2sC0AjV+N9W8P713IydVkBlRgMQmJ+XCKA7YnDFHn2KeJhgAR+c5HAwIJEffxJtD3iFRKxJPCLkbfKfQefQCMUcXURpQRdaC7SS5LBTn7/rj4BTF9k4lTdYeIosnIbf64KRwhqBHIepRALbfiiI6N5QvHv8CXL34/B9jprg9lCOHRjJRD6kj4hEEews7/3Iv0s8m8KevO/ndj7C2N9ajliWh6KhFpvBm2Ijd6K1qEWSceAqAp9zWEayAz7O6cWktxtLtD/S/593LivGpLsOh2xPj33Nc9BMQcjfGYh1LXVdwucaVulcfJGTGJVhZCAcSIOF1UKjYvkjcRP0GzGUl+M8QPwBWCyucbfTciMW7xg+vGTfsrklubuNSawwnWhWUSDxQBYRAwZdIgc/mwRtur6KRTMCnGiIGxP2tMkd0JzPwM2g5NeYwAOsEAfI/g5ar6FUbqXVrRtblxbtr36dndCALB6gktlSnE3PEFx+gUbARL5RD/Qko2QBDvqNAU8CVIWWMCLxNo1mGo83MwK2DHBNOEih49Gc+8oNDFnuLxRCnYxT6Wkm9HkVa0FDcJhRSWakFG2RcSsfZEsD+0acwKA3Md/66r90wj3yGfHac+EpXapxKCbXTdMM26xxq4i8jxEj5a7bAS7KNdvU9Q5yp9k/AKvlPMoLK1sPTT2znuy1vsO2jRTiumES8wgUC7ur7zjwnR3RtIRQ1+RASehSsY0XgGJA0X2WZtpV8iRVrmoJs9UQGjxU519nvfcPCn4dTHl+K5zezCZ1u/5yDMnJxjCnkaKWGZ4/FEDk9pK9OAngF19FmreMxr0KGqUAQfhaV/52F+FHBm26wv+0OlvZXuFUfDfSDbkMdXPAvmonJVMNDAV2cDbHPz2iTu3CqSksKoHQxEav+EnjxP+4PwPXwRVxq+YwVPAGCDJ1Ri9mioDt9BtX+sGDA9C01NzEq6BilVo//sS8HpYEpq/ZRU5vAq0HjOd3zAfBCX31dQgtnEjYSmhODultNlGSmfQf1dURdPY+6fCa3E+2vCUBjziXCdT+j/nqKGf71eD1wFQaAfZriZkp+Woxp6V98MykbWfst0IxnT+snqLyLMDt3cL3kLS7Wr2LLAPyztEhNz9k2VEklSNU6rHpfBs4X142ryW8Dv8DFeLSqaGveB4MfgKAFGJsbsB4tw6HURsyKejx9zCBRgedIULiiCpjGsWwDzsYcRp4fydVB9WIQUO7SN87uz9OSsH42b9T60broiKgM7HUy6RacuHbOkcCSBv5Y+gddLvI+sO+72iCK9o/i5JhblMmruaE/RvLfYJ9Cy29XPIm6Y6DyLiOHu0Ej0/ZdkwQbd5JuIjM5o/N6Cq1USj8VCbo1T3EwBaTtLyD9aiGtT9baqQ0OOCZz4vl1DLs4DMef7yI6wwMmHBdzlHukI5EInNSDNKASjWsLHc0K7qZfFpAvRHAxYiPjW0ewLuInKm1TgHjozCRgbxgSkxTltDLmXOvhk/c6CAsRV5IutkZGjLzN5dZufq0Yj0apYo1ZD8dGgd8MduYO4epXC2ideILxYzMJ10RBc69jg9XAO7dDSb+QwryRp5kc9ztNnd8TVfyTWN9VwpKbQbg0yZj91wGMyXXALLsxWO6sYOu833nxg5e5dTaNtx60H6PrfPwo1FyhQalDsEj7CCp55kDqzrjib3JjpxKOWzx4doACo6TNlc4uL8JMIsT9T+JLYUgd7VMO4T/0Fm7uNsp1T+OWsxkaTmN1nsOGDihr8Oe7A2txjtQyBMBmBquJYYYztITByy75+IzKp4IVorLp1EyIXANxz/GIVyu3TiXCidkolky3O7ZEEPju7t/QCgJjNbNpdRwNPlU4Nl4g/VI6o86N4cYKPa1eYGKA8rrvfzZag1Mw/s33mVcPFj3dl6+y9vO1NHoZYEQQbpH2uaH9zVsZEr2J7Ky5xCtaWTp0M/s7l/Xdnxcc82k5N4HsfYEUBUspGW2zu2+OePP4l2spi6jm2sgE/MLtraevml/gzuhGzsfPpkZdyUcysV4qSPBs9MTJoKA5vYm7Io7T2arGEfG9sUT5DalLchh9dB6vvPsKnjPt5/4kh1KGfvMousBO0l/+jiqbvQOF681ElMeHck+xmk7nNhjenxtvXOg4Ug3ZGG3nMP92heNTzzEAokLamc/MQT+yLXc+u597CuWCMyTc31//WeZnbFzwF/P+GIdpWx2xT1bxN9i3PPEyCnk+x1rd6fp2OQUzK+3O67XcJGQlBpxto2n292Wy+1i7+ve3T8T7WDpFyVGYnUaJ9zrqMSj7nQapL9/f9W8Em0BjmYYoh0DRUcmiB0HOdeVQ5mTvANdKcK3kOfkjol1m3XExHUPvg6v1rSWqIApL4z+s8Hvrh3W6skalpEMmxinC3SOZc9uCwQBXE8ZSaIS5giO4JoHCDYs2g1K5Iw5elXyZk467xmCnqiq2OpMZaQTXUjas2MNxXe867sQkMDSTf3shYd8MImDhX4ycuRcH6V0Q/qC4/0x4lYI9X+P9xwNsDVRxLjqZ51bYEGKfHnDeUpKup1LhXcd9yUd4WbrJzm7wdpcP3aV+GN1beHj8LTwbj4EmlGnWC3S+8zJ1AWZaU29yd0IVqu4cyLkI7Tk0+/ny1zMfYVAYOegHxd23wTECXFMhdAWSUzocW5pxv+1BW8dg2hO1UHcEAL3bHGa8PRePejmrHtxJsGLg2klAQMKQK0MYc0YcN7MzDnFanywq7LJeQCKZxIWO5biMgLbW9awMbaTYEkX4zK2gcMWsk5F6YDIWXzWd9+9mc0g1t9vHAnOZ5bwG51ot33dM5pcnP0OwynncbVLfZCGxmpAZjHifV6K8soqSmb5gEJ/1SZ/7eOfoTpza5LjkP09m/VCSp+X175rH7CE9M5kprS4UxDtzImoOrrWBPI44X0y7vJYY36skBjvQbXHlh+tf8OAE0WRfUv4bpVFmxqkLuemeRo1bNp979cDpDbDUDIZmVvvVc9krGG+jL78J7rSqRTW8UtAxVwM3emxc00jYoGnklmEcV1xEkphG0sFD/9rEC40yjmkgWWmfs6/H4kW16whU6eewSC12dQAtPb6cvxmMT50Peak30Dj3q8EEAbZHZ1L96DX2f7GYJzxNKFuugrtIZsNmw7FbTmRZGCmaOcQM2YRe6KVRdVexzrWHigsjUOg03Ff/DsnDs5la8AEkvcg3jQuZdzgDg9qGslugtdoZbjwjxjS9RiNxepLMqW1EZRXQc8zE62O/4/2X+klUkfm+GHU6XvrqEXKGXoZl/fN7gELD/ca7KK5uIe/tiQQt6HX0KNsMgoAFBe1WuBOdR7NLG2GKeVBzUBQSOIbBXcVMdexBfyaDZ7ZMJOMfeWY7DM28fc8PfHMgDZ+NKzl7j+h/a2m7yXSnMt5ya6RbFkSrSxWuel3fe0Mr6Wba4EIOOMm49udHrJLqeObb/vdGuGQYu56QEuTvR22wA/jY7zsEJ0eaggdhLXbm4oXh1E7Bjry45/gQdFsm0rRyI02RJQj/IFHvMM3g+++cGHR9EK4+zQTOLLarL1E+S2nFDqyyQu4Ywdhr23vYOJG/3kxhmM6Di4+Lv/O/43c2GxR2BePzewYMb+bykMu0+bdRfOdb9qdt5MnCkZg22PAqv0r1eIGQEoFbjcUQCkYbKItg8ZbFTLR60+jVRK2vgQ5bfyy2Qd9EsZ8TRbpmujSPkpjI/7Xlf8I7/k+W/4J9/y3/nymCIDA+uZQGBwv6glAOGiw8pAruq3ewtPLqnvnoaj2oa9eijLW3lxvMDXpCJHw0/3tGzPenQisF+hO1uFWbuPPuS9SlNXHF7T2+nusEhpui33jAbCwdl/mmHVRp2ZQXnWWS1N5ybY50P1evDaMqrJSNC7bRbrqLT1LuRSZVEBZTR0O9H8XUkamHZEfXvoVGodHCn2WpXOm5xMg9d3F5TK642D0yArDyXVMoVwZv5ckTM9DeGkJz/RlQ9bLU227yeK0TRquFEXk3cDIk4Pkk4CgGXXGOpNryDB2ex1k7PJ8m1//Mf+RUY+Plz17i4pjTeIy6wO5k+xm3wjyWUhPoG9x5/+wOnpohLgbiDc/xSsQVviq6D+mQbN5KygVLTx+YsrIpmkrhAiNT7+AcW0qo1j7k39qygk5jE4+nPsUyJ+joLsKtN+eeITuT/D+mYVUYeeKV1RQbtUA5tGaDNonLyjGc/TAJ9+h8ji7ZShoacfMZ2c+/fMkyFL/4HF5pDGNQ8nW7Y9foKsl3fxKVozPB6g2E/P0oOwvZ3OLJgQ0zuT1ZydEZbYwI7w9lCBIV+6+GU9fkhC1cDDBLJAIMFQEmlz3fUP/G5+R2SvhB4kCmzB5c3V+Xwpkdvvg0erJWo0fxcohdvbRjNEd/kBA8rp53HthJm/xHiHoc2nNAqkImmGm8lEBGgRe/zsyi1eU48Gpf+2jXXUxbt538V17m3R/XEPGBuKhM79nFlLe/4IlT1Vzz/hyTrEUEAqfdAF05B37egTk1G3mPgjfefI2euTdEJUL8y+A5AqfMVQw5EMiu2SepHVeM1PDlAB4tvDgBijN6eMGyCZnFxS4fiUTmwLt7BzMscxgeQ7tJSv0aGnsjHe4iYLhQYqFszFb0NtEupbzjBkHeKeI4cR/JE/c3kpEXyfX4AprGXOW+sZlgs1LhaePU1AP4Ny9k3IqzjPIbqAIS2JZSwO3Ei8zaO53QXXMJ/PwCJK6DiNXYWvQ8tHMh0hYtHz7zCSCIC7c94WAxYJa8hUNxMCHlocgKQokdniluWi4sg4TXuW0JY0vAJmbeCCflzChcF1wRVZhZL8DcGro1MXyX+AFOIflcHXIVd3MSiz1H9J1diiSPhlFNnJLrKAkqQ+MYzWxA55LGzKAbtAXc4NGuJ/jh2icMGp/Q165FGkjBU89SZglizFfvYw5qQG/+HFWvnXGjVIvH2Mvc8Krk9PAajnTdArde9/aWq8zW1tCg8WbciQzcW9zpmGBvYTc0ahvN7T0UFcbR4RKMIJHAtGxQiNFbhcRKVUwBLtW+rDkZSfiwCRwc0D4i34/Je2eSPSGCaNdEZhb9i30mw/8I+F0CZinkfPD+B/x46y8OuB3AqyyNZ/54hltL7OfvcyUP02V05eqiDbQ7F5Fk7WfP22RO7OiEOxYjo2LLUATUi1ZXvaWup5kar2t4lUO7QcYFXZA92AcgAYtgYa8eWhrceLeXYQvg1FPFAkeob3En+WYylaltfXWndT7IDC0IzZGUnHPHI3RqfzulE3cdWIVzXT77Rq1mb+6rfDOn/5C+XZc5VpzOpm6BJ5zH49/jSX9riHAcy7Hk/dSGVzCuCmKd/5FP1c1M9oRTbOtcQNmpA6we4yCC9ACdxUiJYoKgoik/mh9/G03GEwl2zaukaayp2E7UtyuJNXoQuFQEIt2UWhZVbEGidub3e5+kS1nAFKGfHW2ReXCjbgpNTie4Gp1HSY8P8xSuMFxkayeaQJtbRJ2viTsJF0h0WCQyua88Ao5hfH8phrACX5ZfzkDlZ8VXYw/UHBlWwK6RezEoDbhhD+wCyNtsTDwuBsIsLob/CHBOnHYRa/RtXP+4h8KkoL6/L5VMI+F7FXVx7qRpfsDDGkqjxQjZL4N7OuG1PjgUhzK2LBdJVh6O88MYCJakWTdRV6kh6qP1LPXtpkjyCZxfCs6x1OuHULEzABe5kXznVvbpPFik9ADfaSB3JF6Sz1ee8FErlJkBBNE+qWK7mP/IbRCBF725+9fZ1BhU5KfrGDrgmmwSB3ZuHYe5xpu9QNAHq4kL/RWsBj6+spuilkF4l51mRP5hoqaWAiJzUyoRSM5ORt3tyODLiTR5NCE8LPRZhUskAr+s2kBE6SxCd6ThbLXP6lvnJ2P/0x9jkpswyU08w88s6K6Gpktg7mK193Je8mvl2S/uYr2nmT13G3AasHW44DCPP3+pwCwzo0jN4qmgu+zC4JGuRxnz7HbWbZrCg5FNtBtLgCHMYCtBtwZRolfz0ePbAPtA9qaeUTzR/S1tKgdcJKCy2Qd83yhQUr7lVc4/uIEhCeW4CPYWc69Pm8Q1Azj8Ngm5V4sdQQWXeBbt7KZJ8ie6aT9g88jhGYbbtd9RsIeyT/fg3XWQlBnnwZQmAseIa9WkRUfZM/EoW7rA3RrD/aonwKzD6GrhwLQDSBzUqK1GauQOA7S2MK/Jgw55IYsuzsApJwTrnLi+OolgITtYzJ189ulnOR5VwocrgKEbwKzDa9/ndL/yHFN75EiMcmzRW+3G70QHG5s3DUbVOzoUm+ytkgVBYNJRcb2z8sUNSCyD7OpHqiXsKF9K5nF3XKwmfF8XAyU6p1hWHIjCs+EwwYPHsHNRMG0Nq6DXZLym5S88j19A72Xgd1UYOpW9WveiZQirK7eRUWjE+PHjtMwvYMwA/9B13teZ5gQSWx72tC2xRFp+QV5hwKdiMBH/sEcEuK18jF8eE5CZzUxYdAKT1V55ss2YjIdLMzKVnkabDbdeRberUEH2Iz9ibLoXr9OLyQtuoMd8EJVMXOP+5PMi+vuqmZWtxjOzG78xfvxtRQiwTTmWuqRMlrkaedHPxhFLDHQUQHcVPbJYdqJD7VzH2bb1eEkVjJABk/rtaGtHLqTA5XE2Rm/iL2mEOF/cfEW0uPOdjEOHAr9aMQfwExG1iE4J4vV3mls4OuYD3BPcKQ4vRivtV7bKrW7YzAqEDlcWZvoSEhRpp1zaPW0DigkKvPLiebrQxGep8X11sQoDp+87z/e5q9mXNRPzf95uAMoCoHDhNqK97AEgQSqgV+sJqArAt9ZXVDk4+PapvoUeC0v/WEpoaSjVH7xHqBBk117vPI/vMo0sPh7BL36DGP2+uAdsdUji7MjNNBrciKipwtphE/dJvVNRshJWe8ELLYP4MGQ1Sos7CIug5Rq0XKNO/hCW4HIUza7oDo2mY5KLCLymfw/eEzifMZ6j6cd54btHOFjmz6hH5TCnGvQ14BJL9OUkMjKTCY4twyO5nH+CfYIgYBNs1ATqERw8+Sd5+7LpMt36Mp7/6mW6kx3hddF6eqthHHGbBMa7dDNrRwZXRubatQvN9WLkvkcoGXeMKd4baK5MA/rtpD1Ca+mIqiHTCDtbIFPqCNdEIN3qupRtv0wlvCQGWaOelp6RIimn7jiELKerrZPv1j5NZ3Qeo6YdYIVvHFhNosV/7xp/ZWEwiefj8a+0ISOYfxal0oQOmN5xg2mqwYyd+iy18u1EOl+hy1mHW7OEKR3ROLv156qSWTp4tdOPvFZnHh42iZHep2mwdIl2bIDadhKVpgeTRwWvrvme450hQL+C9IciTxxee5GgqWdY+oxoDXeg6+/nAJsLk/DZl0p2UjYXhl8gpmmt3fNY6dqGfNo1NKWTmWXJQ+HoBwOyr+8WHuLGmD+wSvV9zxbgE9UXHL9/FJk8yw/d6XyjKuawy5d9NJQmmzO3DKCwRRIyuIrwYPt3VqpDIT6zzJz6azQfPvgSGX89b5fzfcv1D3EsPMHt5GIqA26S5fcwKf4p4j1rucra5O/5485Eti7aSqf3P/TNjhGUhD5H0ehOjLcukYeDHY0juGYrbwdkssvDEUunM2fafPv2eGdctFgyLlJSnMJfimoUgfYEQF2ZD2P/TKZzZg1jB+WgtA2FAeSbcx7tBA2+QsiE7aS5y8B2AwZ/CYO/xHr5MOUh5cTmxPLcR89xZUEtNF+BI+LqR9B+xIqNKygLKePCiGvIJHLI/wKq/gKfSTg7hrPlzizyDorvKhdFvd25OVsaePzseNyPj6ZcbiZjfa+i1tRBXPlf6OtA0hqMW/1yZO9bIGgpBC3E1GalLLYBs9QL7yMjMbm1i+uPZTawWmg7cZ1dd32KyhDI3Tv0jByJmO6guxI8hmF0t3Ep/RIx/o08roUt1lbwe6BP7SW1SHDqdMIl04WbTacwWlaIVp69JUAhZ8rZiXQoOyiPkOMid7e7rr+cArnwxhs85AzD3KEyZD3B8aPQ7s0gfugdshXBHBvzJxflsNF1I1wS5/AQ9ffUCMOw1uvpuOCKx7Bq0bFjmhgjGd69jJwL0eizg3HQRXEzwQxxz0PoCqTdMhJvr8SjKpsLayP5OSEfaaCJ3+Z6U90ZzbvntjA0vIQCpY1xY7PRO2iQ5CDaXltNCL2OiDYb2PQSVLcV7I2OIvzoaJhxC7XRyPCLwykZ3MqVeAUP1cOqZHH9c7YsEtn6YaiW9UAUTFCDY/WX4L2K9oZMZna+zbWj41ClDaajoRI340oYKgeznuLcfQRttjGkwR+lGYr0GlqnL6dPiN+eS7K2nY+e+wipxRGL9GkG1/wE3I9EAj3yGnbM3YDEazmXr6RwQNbC6taX4eQYJI4RfNoqodooJaL2ZSJqX+KKKZ8xk3tt381dTB38Jd2aZ2na5Ej1qQeI/1Qk/LdLEljy9kIiiiIQVpvRoEHq2j8R1VgTkRSCg8WZkAv3ovOrZebd/WNELunBFuRGk5ML2+6MoCXSfo0rCAKxubEMvTQUj0E5DHLSg1kPMjH6uK1yIu3SOnat+QGdBl5xHvC77qnjL3M9v/0mihLWXjnFm9t663saWe1k4F9rf6H7yxcJubmPJ59+DW4DYYv4xvMq24cLlDqFoc34lZRgg2hxKxMJsovcazFYHDg/qIDYsRcod0oCST+05XElnqkHR4LEyqsR7cgt7YAWgDCZjqd1heyqmMSltGvsl7uIoNiVh0Eix7XnUZ75+lFyI+upjh1OrHYkqHrzifTGQZ7b34A08wJJiYfwdLOfJwWJnGnVYPKqRDtnHwofcR7rCH2QmW+cYtDlDEwrDSwVPKmTdfTtSdzdFvJ152h8hmrx+3YDJpkLzsr+32yTcjqNynaaHZVIVi9FZvEA+p+Xn3YWb846x5wiGbunmJgtG5hEHA5KU/GMb+KBwE7WhsEtSxsMeN8mO4+n9YaOBu969i7dynTLR8wZ0P5cVxG/3FdKTFEC95y7wrBJ4rOsI4C86LMEmBK5Z+ts3Jy70EaV8/f7zuZqY8viLcyfPJ+fJj+FIHdm/53NfNUGF5XnEOI/J8P4ELWjcvhp5JsscBc139+3i8dVGpRIbRKuDC2h1VEkTz1lGcSn0mvUtN9hRuc1PB28yOBR/lv+98p/wb7/lv/PFAGBJ0cVcCitgNm9ceCHpP2qKQdrE4FtblQ56nlt9ff4mocN4INCW5M7Hzz5GpaxF0hYdYQco70S7ZQylY7Ua6RdT8PZ+XcE4WG49abIpg1ejCAIpF5PRWn0oEeWhOIfm9xqAgh+71N+aLOgV/Sglpnh5qso/Vfx+Jrv+SVrHdN3DWJrTwoO3xlI7ioDxxDqrbAttwOrxEre/T8hdwyCjrw+VmlKjyvJpTHcib3JlKHXqVDfDaErxAWUyocRrz7EiDPpVMTfwupsAxIh+xUxyfpSK7bEZC65VDDZ2IprTQr2sBeMdc7FMcYdxdHJuNf4ISy0zwx73vIMZrkOp9uRnPUO5VGZMyrAZrFx6LNxaKPbsD26C2e1moGczBE9T1JesAif2I08FX+dnw32eVhGVT1HR1ULbfpMbvu1Ezt5AEAkCEQVRKF30PNC83Hk5kCGyp0h/l+gcMXS2MO+mftIzUrhxXXPU//af1q9JWu+ot5pO2VHcugMbmJgmNukK+GVaxpKi3144Bslt3pCgHxQenDFq4mKGUeQeawjzuDNJP9+KxWL0p2De0bgW+vLgwtNRDj1IFhNUPgddOThYXXiYU94pCiAz15dxbVpBUwZ8A6TG6Uk3BHPRKfW0Sk4im2vrIEx+8hzepmiQSVIPG/y+y1v5szSwOAv+tqfVc6leesNhra58lvxUEa9a2/BeMn0bz46/AoO1nKCPbcw1kFcyOicp3D4hDM9Zkci5GYedAenjhsQtAwcQ2l2Ocyfs3cRUxPJIuR0+TWIAZhk0eZF5fk8QSl6qvwKaPJssrOSAJjsdYU7jGJ750oSvM/Y5ZCTSqSUh1nocs7mw9E9xPjnQ+dYsbJ3cdljk6G3weNXhzMjqBUnfSH0Zp0yKn0o1nSQqDAhz4lkU0gha06JeUneb5lCh/Uw7g4mFk3IptkyIJmgILChw4UOWRPeSj3+2nZaJaEiGO0YgrLuFIvG3uJQkwOv7JmPMqAX9OoS7RNUGj33/HoPJTH5/P7mu/hY5/KIoam3c1sfi6590hlWz7+CSt1riWZqh84CpBIJ+TH5AKz3gtGqfLt7lijJY8TDl/m0SEKbzUqkRFxcSeQO1PbGAGRHzlOb3UHjiP5Ac4/ElY3qJtzMPiwZfYVk3zYYEAK9Sjxbx75EeNcIetoi8RXG9DNlrUaOR/VA7HfUXIkjoNkHjcTe/is8sYChqbf414kncVG2iqoR16S++hfcTjBRDUu2jiQ2J57Lw+zBwgD/BspGXKTbMY5tq//kz+thjP49n9ESG/eabHgDdYjWnact8N7KAJ4aU4mf6hbVtjb2noknxs/IWE/7+fnInzPwKLvGmLVWvg2ycMLYn8zeqvJh++eLUPWoePSejTiZw1ghVYtWyEoPugrOsPz+N/hqgWhBk6axD8QBPC630nJ8Is6Zaeydd4F35f0M4Wj9bRZsWcyVSj/4+B12uEqgpwlUHlwvfZmA9UX4h3tyY/L9zB83wF5DV8nk8B/IaU0g4spmDGpXBlpr9Ci9ubB/GHKVnjsPrqXenAIs6auPVIdwNWY+Dfpi5Lud6LrLnrU8z/Eiu/zgsff8aW43wJtyWNAmBoXlTkiq85j/9GZ+ePZx/Gr87EB4gGjPxzj7yTR8pBeRBisI14jAmqOlgcX52yguTaB1TTeHwoC8MRBZCE4R6C06gm5sxcVNxeeK9QR7a8UOD4tqmdWDwoj9/Q+ysufwtSdUSmrAsVdJE7YKn9ozZFzOwGdIPcMGnwfesDsvtdKMQmJhY0cazZr/tEQL8KnB7bH9HK9x4SvfKrtNDcB4r2YsViM/aSrIc/kUEPOejVfOp9ZlHm2tKp7997NcHV0AFh3kfQxug7md9TmmA8domNBG8vwsrsi67UgNCx74muxFHrxj0nOsHl5GEIkYVgNdzivYMus9ulR6ikPL8LUMBQdvEVQr+oFYaTqSXTO4OzMWqV5F12sNItB36QEInAejdhKWE4vKoGL7gu0YrfZzu6Bw5Icp+4nJi8G3aSJ6yTVI/QgkMnK3piMtr6FbVo+DdwetQn8Q0dFQzIcvb+azZgecDg9jhEeH+JqOewks3cTkXcMaCaulJTit8MMrxJ4x6qLoxtO9k1LTgFlGohBz9pk6yCoKpfmXZaTVeZIflW8PmiEGHEacH4Frmys6jQ5JsP0Y/LItjJvNUeTO3cxJYyzvu07FA6gTQtCs3sK5xhQW1n5JiKYahbWHv5VTLRZnauu1hNYEcve+e/CZl2nXb4XWiOOQm8QJch7r8uGI1T4Q8m0bVFkg4GI6ofGl9qxoQYK8sQmNTUfhlDP4CiP4Z7F0W1B1taB3a+FoYCWTB+RLM2pTSKiwuwtw9VGo2M6uIAnDR1vJ1YWQmXSKgO4Zdv3KTFI0Bg1637mUKEOZ6DZg5SJIeP2rWXAliRq/Gso1beLfM+8DuSP50unUul2kxauRdyfkcMnVPrluo8N8LKPPk2M6T5H3BcZJV9rVO+hKeX7CHXaF1hHZ5sB93bP4akC92txAzQEPHOvdcfZqIQJxDrZhxa/STOLtIO4E/8G2/WMIXCDtyy5aZlFS1eREjVqH5JulaKLK4P0BHZtszN8xH4VRQZPURJ3Nfp67VhFMSYU370QV82JYB3ssVXb1MRnvMjzVRnF+IFrvtv94Vgcb1lM+6Bwtbi28oSplr8sEu3pdoJWDK37Dr8aP7Te1lCwVA4zzHc7ztRbG1BoZciUOf1W7OLf2gn2PB5YyI7qOFxUHuCteT5E2lIFg35NlAp1/vMTmZb/j2F3IVyPTYZ8IpsvHXGXrtAMATL+2/j8IC02t+fhFvEuVqZMLpkSUmpmg9ICp18V/gejgCgqfP06Bsps7Thq2DUgkp7FUsGtkBT+1VTJRAnMdK3vfC86c9fqR2xkg6/6QB9Y/wNVVA+6nWc+LwTqu9OiI+2khFrkj0a/2EwBVygBOli2noXwiUZm/Yo2NA1EPBoDReJuYzt10qXI5lZBDuJM92OdpbWN2uzeu9+xirLcOmXmESBzoLICgRUgkch5KrqLAxcS7PnDSlmXXfoz3bNq+L0Bu7Ca7MZE0hxTxuHJ3tsTeRGMKYdS3f5KoKiMs82twuwnaRFTauby+ohOZ3sQ6fzXqiZfFtWDko6CvwlonZcfSvxh2ZgjjT46nYDjgGAoRYt7WcAcDHYKJHxZvxibYGCl7GtR+4gfQR5eRra0ncEI2N4ga8JYXS6o+l7suT0Q/8y9qwr8H2hi4X0q77Uvo+QxcbDbmh/8FwkuA6M6xZfFWnHtCmJXvwf1hdhMLDyZl4aWsoabCi8KTI3EJsb9fW4IaaLHCgjMh9Kh6RHB16C/QfAmTwoOAqgDU3UqKBi/GJ8JTdIHQJoHan8HJz3GNDfjeUbO43QnZA0DLVZH05ih6danvjMC3KJQZj+7FM/oEcLrv2E4KJ27JQzh+JI0z5Y603F0Hbbdxbz7OprtKSYxpYPqxOajPL8U75WZfO7mlk8jDE5AW+XMqqIZbLncxYVQzAXU5oK/Dy1TMPet+4IF6WFUP7Uabnbq0UGbAlHYbPGoYpYcduU/j6yPWCQIccHCh6/Ev6FZ30+PQ8/9j7y3Dozq79u/f3uMTd3cPCYFAcHfXAi1eWqAtpe4tLXWq9K7QAi0VoArFi7sTgRCIu7tnMj7vhwlJhv4/P897HM+9jmM+kIt97T2z976utdZ5rnMRXS/YgH2dJjeCx3eQurGdFp2BhKHXgZ6K9Lc9TlNQ7kDV509iN+4qHn1LADBK3DDovVHJXbm3W/beV37rHE7kq8OJ95eyJ3w7/VywqdY62jqMu4PeRm7QEpUzAy/dXeglxtmRMIxc92EUeX5Fp9wdwleBf1cCvr0Q45UyfC8UsPrNAnQyW5k3rcyJj1vySA7bT0dcDq4dQ23GI42VFO6bQq1bPYemH2G8rmflP9YRTKr+KkFObbwxrYCbHbZ7eENwI38s3cV674HMiPyd2+Yn6A32FYWXkxF9mT+9YYEDGAXBWskrUWBvqOITd0jzbcJ9zHUSA8zW6mUA1yT8pFJMFQGEFYYxPW0k43dGWIFCz1HgORK1rh5TYc9+8Wq0bUWLryGP8gtDMQIlYVMIl3Xt5c13iDUfJbc2mmg3EfcaL4LNqdBVh9zaWc/WOT4caIcdiodBkLK+q2oaUYLS0soyB8jTgmNdAZZmd2jNAW0t+E5ncvBuAlef570DSSw5M5iktyPAqIGOUlB6sjHRCedNf7Bqnzu1ji3cb6Nk9Yx8fhfbfn+DFy/2ZfwkW9LbvRh8Wyv8U7iS8/0SIPMDDpkns2nK76j1K+iXmY7JZMHx4Qirv6erZ3GHhne8bvLBnYfJOm/HJQ9vFvWatzE4laAFRexRN/Clr5GzboNAPhHyv0ctlSEd6U29nYzW03/S6tKKGnBQNBGtuEa4y03WJZWwPbKEubXgWTeMtXY5gAghyxFKYJlsCtnPDGKhcRg1nmE0fKwH96EgUfNIRCgBXzzLzrzN1J+o4kKInKVzrWSPw3YetC/ZxeDgmUzN6MdPgypwyn8KIsdisoA8oIymYclETl/Ps8/4crc1DWS7QOqAKEi43fc2ng1SLo3ch07SymZheOCdRwABAABJREFUY8+XPhLLTgdQtolcv+PDsyGNFHteAx5BEGCW511y7MDhmh2rflnGTyt+tsbFgNCez8DwDh6athnRvI92Z39KvR+Be9walT+vZxZyO1vEFPMZt4NzSGtbxAxAKVWjdQigJdSCb3IDL157Eee3D4G2DpQe+MhimH+qgBi/dmr/3Ed+6Ehmbe559yKcP2bjvLdYej6J2/un4zD/ss0zEul8m+XzrvOTWztJHz7LtBX/gFkLqBAEuJhygMCMwwQHnOLIoFbede4lAypRkeZspm1AOg6pCYimXoUCznHEVblSrKxjwqD+POSzp2fMqKGfvJEDi/4hr3YpWWIbBwqjaX6vh6Qyze1T8u9GccNNysIhedSbamxkT+1cWymOysE47SwXwqqZKvaoJegVUezduRSNxcihp7/EV98VocVb7+f4q6epVMRjFF05Mu457lheA8/3u9uzAChqSnEtSebSwCcoaI+2UTYSBBH2TsFfr+SnF/azrdnqrwmiDLNFQDSL+FwTEC+vI2z1rR6/TVRQpwnE2QSdIydjNOoRMHGPfCVRyskduoJOt2t85wIacyu9TV/ZxPmflxE+9BJ3BtxEJtoSQRJ9viS7BQqPlbPpejvTBtr69okKM9cfCeFHuxOUOTUyVrBVRirSpFLpsY2V298gIuo/eGimAw8iCiIXR10kTurGlO/dGIAcD+EmMAGLBV7zqOfD+Y3kDZ2Oz6XR4DOVevkwXrkXTg06iYd4AqPyHSw6C62o+EenZpadhtWO8NYzv/BryQTu+ToCUGYwwZBvKS9t5IlvnsAiqjHbn0K0DwZsiXj/tf8Z+y/Y91/7XzUHh95l5wKBRaCzWJt8KgVsSpk7ZEF88+gKqk2A6d9zFSkDqYpLpkLRynfXPJgc0++eYjcARneRg7MPEuLiTLSotSbt49+G+ivQUYpKV8HytCG0lHvREfUrSmEy0MPcSDUP5FvNz3hpHfkSJ/zFYvCZSpvLGGbv19OgLydCYcBNqiehaTfUTerqgSQwf+98arxqWPzazzQYOkHmDKErwW8WE65/QtbPD5Iyez8v+NziUZmfNUh1tW5Ri0NDqGrUEngnl4DIUwjCD9Y+QiYNmDoZ4LoF34YUat/cwPXx/5Y4qnRzIPCxXdRvXoqT2hF7ia0j/2ZJKtf3jQXaOePxH1r1I3C082S7LpGbY86g9ywltSqY7cZ+lEp6WEdJmVKiDzVzjEG4KdOY69OfXmRt5hpXcOXKELjyIDsjcvlgvQP8at0QHMN+ZvknX7K1wJXmn+ZSF9flZGd9Ag5h2FnmoB+aRp0oMjKvD/1VRVa5y2sPw7DfIGgh/d32khxZy8+RP6O1d+glfgdmUYlxSBqdBgu7X3sU+dyb1ooatR8vSk1okq4wr17J52OS0HYmAScB8LX3pzFkJa1+JgYcNOLm2YTwmskK1gF+oSU8+1Yg9fpSygak0OBpG3Ctj55J9YtX+UifToHrXh6Rf2qVUwNoTkenGE51oBPkBZNT1J85b8ghbyvUX4XB2xEFgX1z9zHy8iykqkfwFvr1EtOE72s+wlSZCTFjqFdtwd9ZiifQ4jiB/Wc8sJdlMfXKMsJ8ynAeWtx9nEIw4iJCkUcDj836A3/LOHopvPCWr4ziWb/TxyxnukSF/X39A57efgVJZSVJ0SepCXG24feqJFKe95lKhlcjw43vE1A8gtJli8BvBkit73f/ajkTzz6IW1Y0pf2y8X285xkUBIEjM45wcnIyelktUsDs3A+xLRtZsTNLTywlP8lMqBK8LZH0TgPat6pxaQ3i0IBM9gy7zZvyH3ueAaUnW/omc1sPc97YgK+l616NPwuinMiLvxCwIo1vJE1k6UEuqEGqBrsgsAtmkPQYfbesp8qjlqrn/qBW7Eff+HlWiTu5Cyr9TT50g1MlrtSXeVHZp87GjbmheoTnk0UsplYmu2Xi3EV7lFh0zLswGrtKHxqCOrA02+FtyOReQk0QBOza7ZBbpGwbdxI3Efb0yhTeCwLdc+LZ+6k3kjkzmftoozWp5T0OiX4+yXcaSOi7nJcCMvlNTKK3Ta2upUpayPdjwnjECfJbc3BKWWvtVzh4O7m6KK6sW0CAAqon6ljsasvy3TT4Jk0D9Hy3ZRoUNvPG5CweHQzbitfx6jf/oDFq8PFzRaaezuq4YTw39gHI+YKBKjleJjgbNIPAfjtR+cyymVejrkOrgpxyV1Z3+jF5Tg/nWRAEzKIZk6QrAEOwypl1SSErjft4wAEOdcBmd3CW/rt36YIKDw6ctaZeLPclXj2jHiOv4wQGmYkOqZEmi4qgLuAsSpxJh+QPZDI1Ywatx0wSYNVBtCDQb1YWdfEDaf/lLi52LdS2RRPQ9SS0qSL4z4Ov4VXjxcaNG2mammJzXnfDUbbNXMiLW1dgf2U4xeNt61oUNY/x+aZBeJQoqfS9TIN2BD4Sd5B4AODlHMWMo+eoe3oMz7mAyhJic7wgQFjKn8j9JLzx0svUth4AZmGROVPW0Qet6IRTkTfFOgnBMSVgtpIHGur/g2+dBjethpBsL4xDo6wVAQ3XQJBQpn+Ukgp7YpIFOr59Ef9XW8FnYnfANW/qxzSNucLx7L2kNj/FJiz0Tnz+4FSHS0kkR76ZSfBEDffbQFdPpg8t4FwVNLTK/pUon54XSau+iG1vbeVNewv3wL4C9VNY5rlQWedGsCGVQMd2kLtA3/fAZxKS/tHcbgkiOfR9vvM7x1LVWJt5gyoE9EIpE89upQ5/XB52tRJvMjehsqsiPcYa0I5RgcrYVUoQ/TxYjEgzHuDGqc1IgcjEHJrtdKAPtv4fb2uqsm/YCG46pFEYWkigGGB7rxAoDSqlPriUl1xO0CoMAq9fQOZMSfuzjLgZxtkFbXw3/RumsKa72tYk2nNa3kau2kTc4gPYKbvej6BFYLHQlrOVb34fTavazGuz5nOjbQzQ0z9jrd01knBjWrGaaJkcz8hma2/FwIWg9OSnlmASTRb08w7jM+wm8L3NdfuZCihZ/Rt88gRzL84gdIat/HlRSw3B+4Jw9hyLzPEDIlebMBiKuS0Zwbum7whwhHm76ogKvotUX809hq+9xsAz/3mGcv9ahMBaKhS2zPcd3iIl0/9m8d75bNsWj+J3W7LR1Z9nEJ8Rz8XH93BHqeLBXg+RQdeKc+ObRN4N5o/VAtd02dxvGYHvcPOdI1w316Mwu/CJpOd7iYJIaEEoSclJJBhV9J17xQrsAnmmANCC3OyChGLu786wKN8Hr78fRDvnDmvnvU677j/ca1ovilJOOtQTElLEH4v+wCTp8n+N7SAIGKT2bFn5EwD7BEgwLLCRUNTK/Ulxms+tkP1UuaYz9j7w38MxkNoIHzrsDRiEUjSmZptxjftCrgX1J2/o+5QP+IUV6o8ZDUj1tQQ9eJRTracIObySgsxhJE9XdYnAgV7ixIfrv8RBG8No9TBM9r70ajPLUFUQVXc0FMYX893j3zHW7kmb817NjGHQ/kCMDu185tVA6FbbXlZragNpFDPxC8hGZlJzEVtzE5rJnneMTC1gobtnzj3zlzYz2WRP0o+ruDzscjcp4rK+Lyf2OOIgs0O/8XO+8NUjtheBygpjakwSavQiF12q+MMnh/HiCJuYIsCnHu3wWwSUBOHS4AajpFZ2eWMKFomSICkYOpQMOPk2usRKMH4Df1pB45b+23hGcpZv8vzYceMYUxY4gEQCrj3kh4dd8wjxaMC9EHI6PLqrrQBEAZxEcLieiPrPmche3GUtqQBKxDNUuNTi4FKFYc51Iv17JYbMWp6Xq/ixWU7xEFfi3O8i0xVwL3tpUgXz5/fLkGvK8dbkE+G8BzjUfbhR8xMOF+0Y2UfN/pkyLpiLbX7rJE0ptZsfx2fKJYYvOUUKerj8IHQUg7EDMXgt4gA36jp1rP5pNMErB9sQLjxatuITUMkZ7xL2j3uHAQ3R9GVEr3tqoXVqKg95d9Eaq8+AczwmmQdtghcKqjBWeuGu7yI1JllBja9KniDau4NnNXJWT/qW/U5DrHta6tPgNYbd9p2U1vsy07kevUJv9bNasqC9ALzG88WQFryUWfiUaHE0NffS27CaoVNO5omhDFcYmT4o71+EucU6E/k1XrQ8pKU8ZEq3X/+auzdF4dP5umImoevHMkFtyzLN89AjBp7nQHosdtGpxLvNsakU3nJsMLKL8ays9MYhNh/Zs00QtQJCVyDP+J0Rb3zP723wudNPNJp8QZpmlVg065BYRLJHPIJ7zTqiU6Mwm2rBYQJEWCv0AQa7bKAs/gp3FB7YabxtfH+pRMKeb5fQUSYjTKUlXJoLNzcjUXjzaA3UmgRK47WEh/xNqaIHfDIpvPl54FUMkU6EpMWTpREYLwBnrNX8rlEXePl8KeXVEH7IgZKkMhu1YYuTmaOzDhF3sx//+WUlJX7TWGDFZBEEMEuh0a2ReSqRJ51EdpbmIgg9FdQ5mnnsSp6H9+ij3NKEU9MynN5ecpBJS8oHT6AEJjgZabBYq+RvuKSw53QJ0aF5LEwZhcIlBtW8HpKJKIg0uNXRqbZVMrhnjWYX0gzgMOgqdiOvUq0zYQP2BUWj89XhlX4Wf4mcfhf8rGD0oK3k2iWwva4PnvJ2lthBi8yWAKs36WlKOcaszET6rajA6Gkrv7/VbjoOZ12ICCwlfeh1pEKPr+mZa8djRx8jc1I6c6MK0BhsE9U/RN8kIL6VB/505vLJpQx/v5K+9Pyel0MqqTHDmfUvcySkmB8WAafHAGDv/xGqD9YRI9Ox+v3v2dncB5yiu33FBWk/4rzjGX7asR59oRaFpQbUwyH6WevkSk/OJfyEXflfONfkUq+z1ceVuE2lfM12Ctr6o2gc3u0rNktdyFx4lyuTq5EqmpjvmU6L2NNJVhRF0s71p63YiwCHb5FYFEhF4OQoEETsTat54NAsbpW6YynchegTCm4zYeJlsA/DU/IDCVIBr5JAIjPiMQtya8++rE8hfA2i4A1YeG7BZcJk/Gs96LQfxsnGVzDWGlCXVuCtyeFeOw2AOZJ8FhX0p3HnbCqixmK/ogAKfyJWOoVBxnjqdWFEXi2kw9UPuUQC/awyzRVHDnP63SfRjjTy7cubCBPnQdMtONofpHa8JUZT0+caIVI47wZlLfOhswbyvkEZFMh7k94ntzWe5yMu4yRRYTwnY/3RW8R6XKKwKRGnAhA7VHwR+DOH0qJxjS239ot3jkMUR9LpVosiJp8iIQC5nRKdbBqMs1IjTHaJHCh7nLrOPgy6XoqZViRdKLxJDfkR+SRpTAzeO4cC1TE8/etA5YNZ7s1kh0KYVMgWVyduVk2iUdIXin6Baw+zCpgwH5JueaNT6DGIBiT3kdJMRgku1/szf88M3Makop5qBfAFAV4LO8HRf4Zyo0RF8tRj/BHfgHCvZ5p9OBK5hFb3EJxr8zBL7otHRAkHmh7Du8aEo/9KJJIvGdvFC3OSSImPm05ofDBnz15HMuwSq9xyQGcF+5RiBzEFh5HcaKOo3xxkni4211ykc+Ozv4fgdrUvP678EYWvn41fJTG6YO9kj8zfh7LwfD4TLfwudbCZw7MkFc8SF/ztnsdR0YtkJXcmxfVbjvbLxjDvOfrL30Y01AAeIMpYMOApTv/TQWnEDhaN2sXxivGMXXcQuVrNqKrppGkO0kdvIaLdjcd9WqDqpDUGBL47uwSXC52Uzs3nwSr43acVzk3pfuePTb3BoVE3mFcFu+phmrSHPN4pOnMzbhL1jmeYpAZR6FLPin8LgKzUAtpf3ELzX0/w2QuvkLjRVs0GwC+qigoxmGyPE3RIs6EXXVQUBbxqvHDssOPFOqgzerMGkOnqkY5KZmdSBgtaH0WhPkeBKYo1XS6XYNYRVn4S+5xmhq/azfzAY+R15nPPb8pq2kzE6Pe4VOPLB1/Ppjm8kb97JUR/b40iDgOBBhV77V3YJ7HNBUz22UCiVOTGxWBynJuZJrO3GQ8w/sKosV/xV/JgRJ0KKbbr/0Pyf3i+vg8/RTcgxrmil1tzEEvkv/FVOIwu0bJz5W/cVmtYLnkKsLqLgc5h0JlHuCaUv3a8TvwDvgh9bPvJA8haLARVBzHCqZRpip6Y/SN3uFjdifm2N3at4yiecoY9yluQsh61+wsE+tdR3uKFV8FlTJ7wfxns6413/E/bf8G+/9r/mkkkEsLCeqRnpKKUAac24l8Afn3P8EB4PSZdCfcWB7NERboeVBoV7zQE0+LeaTNfhH9/Do4uJCyjkXVb1lG20fZ8HkIz41WQuOAXljvCzbY5eKstkPsVeI7GXe3D8NUn+eedhwhouYaj6Qr04j0LgoB7nTsz/plOY1EIi7ZvBq0EjcSZ5qy7uDe5kDenhN+is9GLdsi7Gkx7WAqJXv4PB81a1CYpHRaLlS06xApMnHL9AePiPVQ41yDXyRHMWDXrsz6DuA3MHn2Q2v4+fH/bgtG+g4bOehw1FVD5D+jqUNX+hPPNWLJ8qhFc/504vW0M4d0G+GRlIS8EX6H+PoG74oAaLOOu0BSdy+4+tShb08FlIhWGHM6M+YHBpU+xatcXGGP62hw3rM+vtAqtFN4dx8gTryN717ZzvMYvDNdJJn63nKbGq8bG2RUx4CC1oNCoSbidQLJ3obUfSeQ6sA+j7629ZATBTFkKdRNTMCofgNquFE/JrxAwj7GGV2m/NJsrg/qh6CXJB2CROfOI/10iBuiZdGIS6a1+3QGy0JV+c07N4LMtzzBsk1M3a1SJlhVDj5FVP5y9IQeRKHSMlr4NKj8w61ApA7E/24SnmwPJTx/G3jLE5rx+4nEm9fuUguL+vB0MP5lvwYg91oSB9wSqzkxh+NhLdL78Eq0uSrTGV7BLfRrMOhi0FX9TPpPjS9nju4eowjkkGsqAnqSwvcFE4rmx5MaaaQ11wtzliNyoPYdD+V84trkiFeO53NKfiRE9fL7HLUUsOj2brQ7OHB34M+7YgpT3bNb+OSgyYnDfYyuxaK6/im9hNaWFjmR4rGLQFz1jomDi0cTn+ePyDHTfreXu6Fore/jUaCvg128TayQNXM8dzuWhV0gZmMIxhx4JgyG6TPRhMl69sglj1CoSFWCekoxYeZTJFz+nvGwYGo8G1h99htvDS2yu67s6e9J+m0/qxEQsipkMHdfrGZA68XOXn1jw5rvIjQ4sEwCvMQDUiWfISNxKZoUDAwsisPO1gFsSzC4GwPPin1j8JBSj5N0zP/P6vCIImAvx1jSPUn+chMNT0B0fQifw9LIr9O7QMdR7NcU/uTBck4dji4EBzxwAQNpymwfbPblTGsSpBf+wZdGvZGvHcC8B72vK5bVv1qPxr+f9JbcAW/ZwrKWQ9/YvpNAsoz6gH94eblbGadozMGgrfrWz6bxl4px9HN6GszwVZduPxLvUmUG3Z3O3w5E3+meyYrQMas9Z5a4Gb0cqPUCz06+0uQXy0sxn8RPq6d1H7v0afxotxSTcbCJbFU3s4Ey8XaQ8NekNLJ94UB06lDkvhpB98RitDhL0SX+gEFtZ9dRB4pL9sJ8h8lXWPubOhZG9rmvYww/SZDEQ8ukzeFY5oFzak0ywM3cw9dFr/NPWxpOnJqHx77B6qumvQ81pmoM3E3sghviMBGITGgkf2sD9tkGlh2nH6B/fxF8Rzd2VewCCcwzXw2Jolh2i+LY/e3y8KVRYe/ONCN6C4bkivi6P4F27IxxrusI9sK9N6khI3R2wf5lv3pTyhIuRi7rF3NuzhvuNYUhJHe3CPqKSfoMg2wRMlcWDf954CQefIp764gt+VUy2Ga/VDaexqhj3RXt4ecYNMnWPQtExaEyDvhtRyx0JLfAh3DCFx+cd4x+Lba8Vz87fGb/sDDfaRvPD3WcY18+aqutAwbYBzqhUMPS3EXwi6plyYjEzu/oNakUH4r96nd+z+jPo4GQcva+BuQv0DV3FpZOjuXXGH69aDWZFPUmutsDtHPd2qk1mNpqs3+ex1nxc3Xr2+osnByM5OYDiCC+03vfXSIBR9QKzf5tF24C+zLb/t7DflGtLCDjXSd4r3+Ic0tgtjfpWQzlnHTcRI1nD1g37aTB1NaOPex0AX49UOmNrudrdi8Q2ISBvdsRB60D/Kw1oHWTIJDLwGA5RT2MxyBmogKYGR17MSECMrIbOKut+FLgQi9yBWxNfoDzwLewCL9FXtc6aNA1Z0S2R+9WY9zhaOIQP7aVUd/X1uGeiWcNOL8jUw1tucLqzHA5Hg8qXX6Z0ssHPiya1NS0u9oKQfDxG8O6BWyhkFl7rb00WlYgCXHwAWrOplv0H/SELiuAiHjdqaPUzsbvXeW8UxXLjk0VMB0SJielTFeA3zfoBmn072bJuCwd9YKIaOu/7zaL0N1jWr5YnHz7LK8H5tMj9bcYdLAZismMgGypGpSITYwkMC2NYuStjd10n2MeOltZT3K4YzSSVb/dxL3hW0jA9g1/b1nHJcTaDbXHs7ntXEGRHp0Jkjv1Am9FG1ybKAsoQik/QaQ5E1iuHbuko4tUFp/hBlcSnz69HtyCdcdjak457UTm1E1bcc657JtHVsjAzAWV2DAaZAVErh/6fQEcJmSlvMf7zP2mNbWK2qye5obY9kldFtJE3Kotc5SSkoh6hV9W2QirH5P4md32q+d7vV6LvbdOmTtA3IsOApwRazFZSnEmwDeF+K9vE0QEbmG8aymtOQzB72Eq2Yh/Cxymn8N6zgXWeYThssJVEK+so4lriNgq9fwET6EVrwkGqreCzEA0v1MHJITVcGniQmdLHud+kZgdweRo7W7yXwQ6NxP/yDp8XjyIzAPZ22FbB14a1s2vJLgZfH4xSq8RXatv7r0mjxtDsRZE0BK8eca5uSxJu8r4/vPrsOvRujaim19uMT+/MwzkjiSOJ6QwYmIOoqwWlE3dN4YSfCMA3WMuRwa3UtMMWux5ZyRMXktD81Q/3+ReojM/5V8L2ObUL5RP388gvy/Gq9EH+cjtMSQbA0lDFoao4The7oQgtwy+wwFqB32V6u1C+3TMchzsx9OU/GEesgfv6Un7XEkPbnSrmJMeTNcC22lErC2bNjnGMujiK/LB8lrZ7kN21zrgZXmGExsJlbyXvTbnK3o5eYoFSR9Zum0hYaiL+T5axfOinlOtnci9p1aSrR96WjMSoQ/3xJopNtgmKSsKpe+JDjCnD2fL4Swx41Za0IHNag2TEZfbQwXtnQ/iuTwjEvAApT4LKD0EQ+OTSPoJuvIK9Tkfdw7YVzjJjBcte/4WUWgk/OJlxNeQDI/DuuMrvRxZyq13Ff+af5KS6kr2uq4jwtgJEzfp6rk7wQaswcCv4JZRGT2sNfeHPoPanoqE/V/5+iqaAOq5E1FBncYaqo1C4AyoPc/HUq1Qe1rFqza/0jy1CNHfCkS7fafRhrhVtQKNPZ+6ZEipDbMEQgGyXCA6uf4+7V4ZTeSSWf2bbjt8dl4044SS/5i7nWvUs5nX93dt0iJjwc/xe8AE/nyjjUoiUrb2O+74piOKWApTObdR4lRFni4NTXL2IoPZm7MdcYerAAoyWniSg3NTCCkfI0EGBUEOD2cJwgLsfgL4JRXAQS+OPUhnly7PPvMFl6WYryGcfCkarTN1TI6dTEiRhcb4etZ0TC+773hluCzGE/siva75gb8dAkKhBkHL8rj+x+X0pSVTwu/dtHpI90H2MIFVwOjgf0azkQnwaCBY28pW1Ar72At5uI2g4IMVFnotFuMH96+8a1Rn+joBn/5iGS64C3yU9SWxRhC+CL+NqduTchnXkBtSgHKuzqezL4Fecck5R4FPM2XFncTAdBHqew1U1U/AakILZJYoCPuFRN6tflNl2Bdeio+gz5Hg42NFfrcO99Sp09ewbJstF+/wBfswYzbS0hVS5t0EvEfwR4moit7jQHnKWbet+4qDZVofnfUUNV6vcuXz5BHKZypqjzt8Gg7ZSo2nkzylfElQchP0zz3FlcirneyWTJc132GBy405uFO5/QuA022SyRZCw+dnNjL/bl3P5gykccJh7fv0MVRFtlliq8/ry0v7x1Ex2YVkvxZorGh9kgh7fohAiMsPRWmwTzanJMdRcj6LJuZkir+p7NxksRpoVEVzyvUiH1IgiH/qLiTZEU4PTEL5N+ZpktxnUqYKItcU7MJgMVKou4+ypw65RRbHG9sWK9J1Geck0FE5nCQ76AqNpItAHncyVd7UZIBfw7BzO8Vp4z6vHBxUlcv64FklUThTDVv2FSmICVkCdNc/Q4fk26TfDMUuNPPDIQZK95SBfBuX7wKSns30/7757nsbBFzg59SSrFWN71nfHaNLb13Ls+8n0nzoHeXAdYfc9w3pVHIezY7CPrybO4xy19n3ovRJGa9pI3WldJYLcMnEp3QxSO4Jk3iy9CsmagZhLbjAxdAfy+rXgPA4w0+I0msoINWVeqUxx16LQ1YOp635J7bj3LhUZrVWzs00JkGeV49b4rOK9f76hprOATbECXjItTwLFzQkUNydgwUxoaj/mHJhDM5mMm9JKUdtS6/qua0AQ4HE3DR3zf+Np5VG+CJhKqbSJe2onekU8B9MCcfGU0f78Uj4JLkfaPAkCh7HG9Q6BStje1snuxbv52klHxjgL3goQO3t8qOS87/j9ri8efSKYmmuVD9CqQ7hZ4oRv42Li9JnEO8tQW2rpBhZmFzN720yS9liv44WgO3gWbWARVrDvucwFuF1SoTapyJ59DQdXwVol7xABSm9iW+ZjmfwOh832tKlvsz18Mhyts0qjmo3Etkvoc2kYBePb8As6h5MkAAhDTRnvjxvP1bYN3Ap4hzz5aCoTXyLKzurMOkvyWLb0K65XLkUS9RzeKg0m8yUkXRWm+doEymv20b/FEc/OV0nytq3+LG0ezvOHStBJaygb+SML5g6GXv1enxq8khZfOXe3+TFD+gLUPw/uPfTxfvZTSWsYQJ/459jsYaFKlwd4gEnHW4kPwtwj1CnMPBcDnVWDGCezbj6CIPLAXw/gpvHBp2g9I1cdgrOTYHYp2AXwV+wh5CHlJNweTMTh18j86Ddipz7Vfd7P6p2426bnob3vo3VKQPlwTxVbVvNN9o0bilu9G9/um4c40DYXFeL1Pl8fe4Lg5gxqWxvJMCTZkIUA1gyfwQlpGJJz/dH4/0pvsE+wWPj23Z/58Uw8+W9tQLLM+tbZVR/lSgCMLjVR5SbjiNM5wqp6mpM0txxgpOpP7tyKZ4/lEmUNAgO1Ld0yzJ76TAZe6YvGYMEntT91cts+823eBr57fAcr98/ly2cnIfnG1kd1rb5A1bkw/lx4lmqfKp6712O5y+4IM9l2xJOpf+rw9ByI5weX6a0SVGL249p346gLK+BlVwWfGJX0Aaosfuz4fTSRtcE0DRhEgVmHEG7dzywWaHZ9le/GuOA28CCVZyAnvIaA8Fr+8oavW6A2NYbAjMPoOyw8fPZh2p/P4Z6kR7pegjw9Aq8SDzzuDMWvREPmJKusNhYj3oZC5r7wFw9mJVJ1+SRRCfcFBv+H7H6843/a/gv2/df+18xsNlNbW4unpyeiKCIVpcQ1xqKszsVSPo4D4cVMWdvLicdMQn44HmUBmM+PJmHBeZv5xjq4E+ym5HCAC7UTz+HkYAvEDGjNZ8yBeeyJuMvGiBx+T3IBz5Ewch+4DcJL78sLyXlUPjONWQHHaNY/R2+Bmr7Cbby+X4NFJ+fauDN4mwaztjMLqUVLXG4UXuV+7B5nTbbKzR3WCiGszkTA4DuY7npz+vE3uD4mm7nrgPztAGSonbgWeYdHTk0m4NJQJJ+UQ2U5VB4BbS1f5aWQnS3ht77WBEi+xQSBC8AxCuSu/JKWSExyGM9u/Bmzly27HUDaKjDh5AQy/f3Z3TqZySO9bcZfcuqgbdQJFthDrhz6drFs+kiKeNwJsjoseJTdwjMuCywvdbOLb7qNoDWmBGlZX+TqfIa42kphLXc6RuuwAqapYbQCRLOum9njVtrI4xtV1DucZcu37xJt9LbKJtzeAEGLqZSFsfG5ddgFlPD2+AIWRw2DxJXgPwecYkCU8nVpOHbVUh4RvYlyse2veE9qLC8yj7zIPMKELjivs5ov61zIOzKZXL9raCJaSe907wb7RFM7D8R+zIGcp3lTKADBjCBKYPI1QEDINVPWbwqVLte4EgCnOmwlRe7q1vP9n1NoVH7PliI9mpFu4Bxn/QDD2prJfP8V8Kti5KKT6I3rrHIl1SdBlDNV2cQKL0iujOfp0RMwGucCf3fP/6TPORI2NbPoZBS+xjvcrhvHWNYR2XEU52XH+LwyBOdGaz8PoReeZzbP4tb5IkKC5CxtmEpTvC3YN7JCwfS7U8h1asRrcCpO4gM24wXz3uGWoYOmSi8koj/vGOdj36ULjyjjtc92o0jLw1NqRJCUgL4FWu6Cu/UdPBmqpf3TdznRpWYi6RV5S+2DOXB0MeJtM4nyMGaE1VrBrZozPD7sHP4+53AtnkdguRtealuAPyBcSs2UNjqbtURZnsFBtxi6ONOCIDDxxEQc2hz4e/7fSC1dPfsakkGipF4ayMZGGF8Qwsg983BfYyuR4ex5lG/9ICj9AAm7bxE35kO4lWKVvOzzOp2OcexsPkCYxEjG4Ghi7UfbHC8xteBalYlOIdLS9y5HJC7MAAQ7P3KX/cXpDqBjEWoBZL1k+UyiAp9RqdxUWqXPaj1rEV7smTdYrKLw1gDUCUUEr9yD4FQKzg9A8DKQuxFV9wn6TAdGtcVjkAZg2mQr29u/XULgLSsokCMYESQqmFcDorVqV5QI5A226rL/mfk60yZre/FNITN9LOE3JJyYfYDUuHyq5N+w7oEnsDv5AH0eDuFOYX+uPPMVEVFFPP3yU3T6W0AFYTo5Kl05npVuOBSdx1jXF+iJ7KfbG6kxwSc+M7HY2xPl3CMt5iGXs8qzHLF6OSUXQgkYddMKkGdagz0xxCoLHJsVzdUsKAtcxP0r4SVHLc2DrjHbtBp34TS9pVENJgN6SRu+pU1M2PcoKWt6HPEA9WF8nNL57MbLfLf9cYrm57Kii14pIBBSGIJHnQefDszghzYjn8X3VFFI29J5KuFz/q6N5eNhfyHTx/Zyz+FUaxHmOhMtzkZOKZtpuE+aQx7sS/qkF4kY8iqnOsFPlMDVrh5dniPBbQaRpfuwJA8hzuUOSydNsNmzijV5BAy/SuYOGVUVZuxnNxICaEztZAY8D0BQ3N8YzFqi3HrA1XpJKA9XgUrVxC9vf0xa23CQPwcLWkGUk/rTSww670rqEDPVYW8z1AuoPAbnpkLQYkbcicHSruGRY6PocG1GWGF7L/Z5duIy7jRlYV7083AHbNebztImEv/6G/7ayOA557jf+oYkUFV6h5SL8fxQX0GxVYkNnbaNxJRQHOzkbNN+j7e/Hb3hDl/FTOYPrSNn3wjW7H8M3RO2ErWzrycRfC2RwLkl9A09hCA8bO1h5DMZeVEzK79aR127kuRWBxpmpDHFpIO6y+A2CAVN+HpUIal/B13NpwxahNXDlvYEsnsKPiV9l4YbuVPo89Ulm3NLLGa0nz6KosCfh/vdpmmukfGxl0DlQ6pDGvmxTTjfmMCrRaOxn9JDelBKFfS9koxc14ay+E00dhZUT6usyVqTBlEhsuWx73BrdGHRn4som5Bnc95bylg6hqXh35FAhIMXTXJb2Oshx1KMArx/049P1VoOzrVNWpU4zuXj8wXUqv3Z17iEvlLb7p3L/S+S8FoLv3ywgtVhj6GofYpq6USC9FeYmpxKu8aVg7Ot+/dzvRm+6lkYBl5Ce2c86PhXdeeSOhNOt2Zy3H0c8oDZhDjYMmEfn3eFEuNlmg6+iszSBvRSkVB6sdm5nPw+anzLvMjQ2oJLAL9URqC9KWHjqZH0n3XJWgXUxbaVaUp5btFFPhpyk8/dSnAnjtqYF6xzp9aD2YzYJnDt+lA0U2yD/jPGz2gIEyhM/ZUvDs8meLfIvY5pEkHkx0HvkVaXRO2hMVT41cMjQNCDYDYSk5XGHzdHczjbF/usCPJX2PofY7lGn6+eobXBmXNurczYYyvZCuBy6xxqjZTpAwuwk9kCzo2td/AP+QznnCC8Gt2wD7WyZQTnPozKUlOjkSN4tLAuoAgXoed7eZoKuVIbx0GDgGvQb5gEB3rLGdfJpvDBqb+pMO8hO3MIlihbiaLp/rkEhOWzNiIf59YxzJLZroNR5c6M2GWdrynStlIdIJsoXvmPFFWtB6P6F+DSeQd6dUe9URWFw5Fg3BYc4c3BpWi01eAUgSCIbFuzDX8mUFrtQq5Sy7eKnmeh3aWTjLgMoks8mHFxHY1L7wd5rA+lbNFh1tvLuaN3B6yyraJEzv5TiRhKfTj51kfYmTyYJbPv9oGNNRn8lXSVYQoNI9WeqO1+BZ63mb20xJ/h31vBrFeH2BKdBEHAIDNQ6VNJVZ+XmebdIzM7rk7K7b2TkT6QQmIpBCv6MP/eoCghLaSQGrmWLHkd2+7048soF+5B9M0tacS/s4EDHbC5HuyNLrzQ67yNgh9fu5cywDMBtSqCWlXvLrQgOkzjuvM0+l75iUHtdcRvTISARCuRDxBM4F6SSrOzngMzv+dRG71X+KjFnq9KG1lTH8B0wYMss3Uv1stcqdBJ0BqtvmOGHnI91hPhbH172uu+5tL0t3m0Wsat3q0qr60EoLC1ksqLlejjo9ig8GHusEmgVFnjtIRNFB8cSW10Mf47jOBfi2SCCWJetKqN2Idw+6wnjhlVJGidiXG6xb9MItKp6qRP6gC0ln+vV7+0etLQXk//nHxK/V6DrjuSqtjF9RXbCXbYhatTIPaibUKoT24/Jl6K4+a4W3jHl+El2j6DCv/15DjA0f72vC7pIFXh062g4jTkFMP2P4q6rh7Z1WhSxhay4nEjKD3AczRl5T/hW5DDJV0C62tuMX1BgFWm8OYL0OcN8BjO7eaFpBytY+kNbw69ZKscozfpuDz4V4p8vuB4AYyyH8H86GAwmxhycxxxaQ1Uhg2hs3EoQ6L7dR/nKLcjsWEtepOCpLAviVV0VT6N3Gt9RkrBL+sUdb7OvPjJFs60u9qcN68pgMzfw4mMy2XxYydxcn+LL2q7ZDUFaC314M6XyxCB3/0KobGfzf0IUt7FocodpbyDZgW4SGwJGWYFHJ55mOjy4YjVI5B1udDundvJm32R600jKIk6SYtLJWEOf3QfN1Sex0h72K0z45nahz5B52zmHeb8GSnaZtquBPOyfg5u62LpTZP66EwCnkfG4iStoc3FE82AH1E7WBP8UlM7AxRgFD3Q28UwwX6lzdyCKCN4zhlOeZbDD4vZ7WtgQq9xb3MTia6dBKTHc7LdEaFPz0sSm9hM5KDtXD34A2JWBxGykTZza1L2od21l+ap19g7/0NW86fN+K7cIEbd7MPVtdvRBFRZf+uJF6G9CHO9Hb/POExAaQAL/1hIwzCltWL2SCxErscc8CVXLgrk+/9AQUQyKY3PMaKXaGubtgGV3Xw8ax5EZvCkoTkf6CH2CALIO1sIKvmFfmYzkoHFwBeIgoh3lTdqjRq1xQFnUyT0hBQIgsDBWQeRmCS8sf1J7D2asViWIwz7DUydmMoc+eDZz7EXoMQeTMYpPGYxQ8UhMBswtj6Fc20O07QBPJ2whUyVDBI/s36A679sIyKtiva0dXwy7DJjV9kuCOcrjxB0/QQmiYH3HvkHedsW+ne1tgD4ytKfltkHmOPjzysjD6FRzUDmNobq5hVcP3MKi281hU+WE+9XgUbmjsOfViDmiLiRQw+9h0InpUjvTZo63wrwdO07iw6NxunEY+g1nihbRBpfHgIu9aBrAJk9Bf8k0v9WP34YcQ1zSC0CZiaHbaeqPYzbtWPoo7HmteJGZuObeJVfWx+B8oMgUSCKr3cTUw1mKVtTvqTvyBju0WeuFB6m79mblMTZoenbitGuE0mXNLV3hZTbn7+B33wTha5LcdfJu99Zia6OFyojaL3SD4m5lHGZPlwJy7S2fWlMQdnnDc73u8jo+iac65cRnZCHqm8vcoxdEFERUzgxrYwy7yuUBRbhXtkV74pwvTmcisUfMdx3NAVtEFeswDR7Fqi8wdiOOu1lKkMaKGvxZIDfQUa6AS1dKlf6RnaNPsV3Iddx2/Mi/Vu/JWpkAPApOtGDT3d+AFp3zH0sNHb60qny7c68V5nkpDg5k+18kSc9zxAjB0uvuFQURE5POE3qiBI+aX8Uf8X9Uq/gn3US78Kr3J3wOA9H2lsRnK4fzs3zECrXNpJXLiY2JguLxdwDORs1hOS/wPDDXlTfXsHiBYfYlNRF2G8vhCOxBA56E0440P7WMwx9MQOh4Qq49WWUogaHrAm0uBnQT7iId1AXwN9ZAXYBaOz1VHoXE1IfiqZTyhbtDL6Ofqb7ulOOnMT7zm3cq8oJHv0XEvMgwOpbuxlyKA6Gz2t8ybzaF7W9LbjvIfUl6k4lGeHX2f3ADywTFGBT6w4bqr3QlLkQlB3BnNm2pAdRFDjdCRVGkJl9iFNY/UqD22BeSPNBkROApLurSo/8c5vgRP7ivVybdZaYD9dw16GVAeN7Yo5cYwzF+/xpTchk12sfoLbY9iLuo6wnQQYZocXUye0Il9j6Ta9kOzA8OYj1wQsYE56Lt8KWZHWkNQuPP3SIJimd/fJoEWxJslfNA7k6/0fa7dspdy+nXrsYiOOSeTwd6ZfxbnJlqGUBemQ4zbTupxYLlHeWo05Qox+uZ+OojUS4RPChaTjzu5SSFGUBeJakkj3SwtEpx/B0TuIN79d5/+L7vOWhwGvvWCI77MhLfIy/Z4bQYIICg0iYzIxBVOJaAAqNMwPtXOE+otL/Jbsf7/iftv+Cff+1/zWzWCxUV1fj4eHR/be3X1jJ3dpRPHNhEiZpHxY69qTqHAzFPHp6CvUdCs4u3MMpDw+W9ZpP0pSFcLGIQf4dPLryHNt0thtjSUc8litSoq70pWnwNYRFMjg33Zo0npENehCNeuo61GxstrDS3bbcOE7WSszMbD6gmeNxt7HnZdbOOY45/xLfvPwXH9aL+FwZw/P5StasCO5OMjZJIogrBSdTJ+P6ptPkabBWldywZigFwZpYlwdXkCBJo0TpC5GvQWc1BMwna8cfKDLacAxypNWp1RoUVRyyfkKWc7t/A6ejj+JnHopry1Rs6+sgrrMS4fIITFIzx+PnMfE5W7BPQEA0iRytUrO/UUdVl673KEk6GZ+so91bzog3cxkd+gdYXugG+z7u2Eex6jxPj5Xx3oCPuW3nCNyX1QXm2sOjTlBp6dEEd1G5knShAZ0iFHGyCcFisTKSBn4N9uFU3i2hQpqMd0Eci1ueY9amoSBXgn9PL67fQhvwnXiCub/NwsvzDL37QommDj48P4WmEm/Uz/xEtb4reW9sZ5DcQJlWRUl4BamRe5ju0SPM1IySV994HfsaKZMf/B6ltBMXhTOYrBU5HfU/8dMzj/DijYG89dkDtA9sZfKanu96uuIovieqcSaaOqKpG+EEbfnQVgDuQ/hZF4OXTyXhgzMYFFmJWRAgaQsY2kAQaHFfx5ZFAcxp6cR5igTXRT1N6QFumfzJd8glMns2MTo7Riz9AViHvbmR6Nhy3pKp8WrpwFVVCQYvUFrB8paAZaRN0SM3biPu1GC8Q2z7H41vEii+OIRDD++gNKiUTeIqm/G/o6tI1ln4zKMFf2ku9JKAMWHmkstu+ngFsvCF3cR46qCuK5HtaeVcHWj3Jd+UQ5JFwQC1Cbmxl0yAzySunq7CrqaYojvLmDr7ANfdrFvTQ5WTqTYeB7d2Xn7/b0oMtgndH5ouUCyFwfWfU3rNhYFPF3MP7HM2N7Ow2Z+KSk+0OXMxK7s22OPW8VFRdTz/xWdonGqwW7qfAvd/O9IAEl81fcPSEaUmqDxs/YQ+jJ3Kg+MjOlAMzuZpYxyRAbeBMd3HB7W8xlu7v2XKyV9JcT1Dvy7gU7AL4o2uwrPJt1x47p8ZuD7ar5vprZHH83y/r3EWAll7LIoAicKmT1a93UtcfOw8LQ6VfBV9kt3N14CFULwTZI7Y9y1ngk8ZWw/HElDshxytzfdaPP4K/aacIDRLiUFm4D1hMyh7wPJo6XZChlxizw/jaDlXzc3J823AvkF1z+BUtA9tYjF/xt4ky7gLeIKmhlPYDzGQb0qg1f0sM4NrOF46i9Fdl94nKZyWEE+qi6rwy81CqP4d+LZ73glnovCudGNC31s8MXAzutafuSdvhzqYx768TBsFNL36JDdED1YJEqsUsvswpCY9ndNPcyy+lNCcPwkL8+V+8xHCcC/14mzKFDL7f8Tnq3qAxsLrz/Om9zl+KUnAPHgcz0cqwaQFiZLFhb7YpYBBmsKZsU0Ifj3PiWBs58Hsvihu9OeAzEyzUyPM6AHSJfpKsn9wwtvUzs114GG8j9kmwGcvfIayU8meLGdmRNhClGbz6zwycTefpYRyyGzhj7FKGPIzFP0EjtF0aOt4+LFv2PH2Iww6MRHzFNv504hjekUH6/JCCRXN3fJ1MmMjv1bFkFJvT2dgX+LcTuBmauQe+CoKAsFF1h5ALoVmXE3NlAgCFPwAgpQx/nexf6gFjXksUdI7yAgETVdVYVsOD17dht1lqzRyjEfmvypi+kmOUuBznNHRM2jQ3+R+sE+nqEDjDepq2KNsxTYsgnkj3qLRsxPlR/PQqXtA+leUl7h2fBFVIWaKZW60D7ZNnO47MoRLhxPwS7yLo0mGEdvE6foRmdz2bqIzsR25ex2dRg32qU+Dyge968dUuzRS6dfO6bgqnJwjrDLdPlOgJZOmhrVsTDjPhlOzkBXOwaIZA3UZcHKYNWnc/2P+OeNJUG4hrW7+FGlO2Jzb1TGUW+bZeJBK8K2+FE/LhdlFICqY/a4jZkHPoiI5irSxDJv4V/dxEkMN0UNzqGyIpE96O0ovC2qZGsafApOWObfriPeew59mf35ZtgO1q61fM9X/HQ65Qp2rtZ38kPuKvieYmtHcTEKyczrZUdkI79jeyzj/VaR90UGoRMLRMUsImqCwGc/QB1LifZgzGz/Ew0GNxSAir67GXX+DYaMvcaz0cW6FLu3634u7jysVHuK31IdQa+uZa3gNh4ZQegf1EU0ySi4PYMLi67w6fSXp+gv0rhUep7ZwVw+mM38QEl8M/Nw9Jijd+bwZCMmFx3OJEHunRa12ozCUMXus2UFTh8o2e+8cz0qTjHSfYixGECyitVom6zMS3NdydMZqqhxKOT3wV9zUI2zmzW8dRVUdGO220OSix8+i67kuAVozof2uhbbLY/grOtsKCfSx9vcKq03hwN+uqNUmYgbkEOxtCwA1Ci4Y7DugwZkqUwwtdrb9AhuasvFw+JoOh2B+G3KJKNNqm34mnrp0lm17lIoCK/Qzes4p4CVEmSPuBycz/nYCB168xPMucJH87uPMooJTP8zCElSDou0aTiHN9Ab7qowqrueqENucuZ0yhbsr22yua7qdB0l2UJb1NQNcs4myt92LX4nJpm2qQO6VfgzuvxXuq+6rEAKxT5dTEVbA0SQFa1xtiTc5vn6krd5JhHk22gvf8MFCK7FhnuIcrwys44mbnTz95dNcHnbZZq2KjC7FMzaL2guj8at2w1tRaTPv6FoJLqmzSR9ync3e1QyR9VS5iVIFX0w6jMwgo0YLdvet/YJgle11iijltyD4qW0Q94N9OnsDKQNSGB3sjl0f26o/qaWdyBkXSZt4kVHZKxnhcBvMASBK+dFLwPjIzygqEun/n7WUPdCr74HFTFPfLCr63mHe9lVo1Bo65/WAwjJTG486QWmHnPysUDqdbclCoiAgmAUKIqrxMa9gaBD/TzPG+iEXzAimDqAHiBQEiDZvQ5vvRU2/dq5YeiqMun/Xc6PxPTeW5yJzmPmD1W9qU8fy/LzlqAz+rPxlBl5yPfFJ74HmC1D7oxGd2XEkCXOzkqVuDbT52D5jFxz+4sqqVEZcHseDPzoSEj8APN27lRzwg+pwPypdPsCk0BAvcYD+H1s/gI49mKQGnvj4ay6pddxvoWh4WuJA2vqt5Nq1/wvsc2i0wy+7L5NOTsJ/xoWe62o9S41/DUqzGZ/CchyCioCeoGKeJISqpjqcLw3CYfdoXF65bTPvU4nz0WiVHNX26iHnEAFteQS7hDHixCDsWiopCC2g015vJVVWHQe5G+foi/ZPf0T3TurCHNAbHUF7G9T+4GR9RzakZTP5RizKpHQWexbbnNtgNvBs34+oKvVkb7Erxv4KGPkXitxv2PTgFr7w3E5YVi5hmTn0G+0IWNVh3AQtb96uJutuXzxnJDDIyfb31Ncf4YFX/mB3U3/qTNButpU2vdIRS3SyHUanEpzcWzCZJd2/tyDATyXPE5CQyeV+P1IcUsyYDMHmfkQoSli7dTdvvPA4L/yxGKe1R+gduw5WVeLWpsIxQ4erOhXREguoGCucZOCQCmYW9aHcKLKtFZaoe3yM3dqJ/Ge3Jy4Ke269+yF5El8bz6bWkED8c/v54zctASn9qDLbrnMHAirwnHWA/PBi2hybmGA5xoyuPlcuzSn82RrC5moXzJOj8Op73zPo3IeBZSD1LmTi1IU4KNbZDM/svMrDggejxpyl1bGVAW3vd49lGVL47AB4uWuRPXAZ1zDb1KBgb0+rexjjg/bycrSOn822JClz/Dx2hufwaWIFM+696u5DwH0I9i0X6GMMw7M5iJisGEj0B1lXIluUszdvF6GpeYTctBA5qY32pCzoBfaJhg4WnhtPZq01RnF2fwq40j0uqfmdxIt3MOqDyfOsw9HS1TO1o5C3r4+hMiMCfZCAsnQgA5b1PEciFh737STHAHcT0jDZaXjUYkYSbKXjCWW3AGi3wG9t0E/iCrXnrSoqA74iUNpK4do+1NYp+fHYBvqtsa1QVrmepvmRdJx/eAj7dtu4FSCg8wJ2SVmca1QiGOvQmW2JrAv9t/FHahM/BXiTVgU7XID8bZg913FzzIvExJvZ6bCKnTooUPdUbifKc3jYz8hPJ4eydc94Opa02xAAR6vbKDXJaPBR4u7bSKhDvlUq1mcykhYNWdHZNLo2MP7MeByEYGqdLawbZFUyeenkJT6eeICaFyzsbArir3NDCFB2WPM2Km+Ec7C6UQ25k7HUp3LVFIDjyIHcE/AV9Efxn1xAhaMr2XljmVpVwekZVt/rS80A/EMLcPXsw7CGlWyf7g+nX4XZpUg7K5jf6crxO324M7uGoGGHGOYbDwEbrGo7gKy1EUlbDcnjz5PhUc9MsZfu74W5fNa6n2fcUil2yWeoMgtvpTUIFwTAItLu0I7jHUfWHl3LwVlHrcpZtzfA1HQcct5gzClHJtUF4j0gEfzSrH1RAaT2fN3uxym7Ynx0hdyt8aes040EoNkCySW3iM2KRSG7TcSdu+hrx4PFEwSBJouM94wpyHVygj5+ls1xd9i7tGehilbf5A9n2H3DiYodWnyWHoc1c7rHHVXFBMcfornNk7ENHyP79Q94PcfajgSYmTkBk+48BodDfNzZSJNDdJdIPmAxEijsB+NavAtDuJYVjEXeRXRSuNMSvp6OceGkNJ0mXNvJ8pDDmAs9QP0Kn7td5YP/XOVw1TyM6tPUy/3ZOuGitRcskBKTTpsAT0tu89niM/zUMcnm2XZplOBeVU7g1FZmT/+ddksP4cciSPn7rTXYa5SkTnuDwCDbjdS7ZRuBd2tRi3JWT4YOSQb327UbcYw/0Y/0JX+Sbu/AqF5jgihh3dVAAsVOdry/gzKlFFiN0TmeGzfDGX9mPKK+gxmlKykaVd+9b+ik3rzSAGqzHGlkIRqVbeyoUTix/YlNmKRysiP1VBptVXYmWaqIujmE94UJ4PkuozxshklLKORMwjF2VD6IT+dNBAy2/wGBM+POIEhjcBGPs8rh/lGB7JhsXvjmSQJiC/EekA1MJEjdh2+mptPPL4B+Hx/A17sYL+MMwAqins95nYL5B8lwGslZI2QKDRSqYxG6OKHyMedQOvxNZ/RHXDdeZ4rTQFKrUvGTwj+tGvqMMFPbEogoOoC8L3AWwT4UdPkozFqiLg9BpfdFJrYi6OTc68n+f83+X3jH/6T9F+z7r/3/xiwWC78XPExluw+OlvWIBikOveJMN/sYqqf+RnFLFPaaW0TY5pSoZiRpZ/MpSazkZMIZApxsE6dxfd/hp2lmQq8/S0iln7UfoNd46AKhjLo6ktK2kHgsnq1rriC4224yWrcdbDdpKfZ5kzj5bVzFLslMuQs/GLSkSGFAciwqXTQh876HisPgNwNBEFi9bTWtjp3ETotklIurVZYJwGcy69KKmfv98/w6/gTfjj3IWvVXYBcAw3YCYGqcSkjeEJ7b/BzKPnkIjwlg1lvnsJhxduxAajZy/q+x+CVM7QYN7llsIEz4/D989dzTmNs+prZzE570JMNfL/FC88diBK2SkxNOdgMLR4yjsBgttEjLedF0h/DCqfwp9DjLATnujL66gtoBHuyWvU3wiH42530j35v2s1OoqPFg5eBkPpjSK+DXN+A2wkyBzsCCq/44OyWwUBCtzpVLP/p7bObnqU2MuSbB984F3JtLoWOcVXvebRCovBAQaXFqoS7Agb/rttokrSRmPVEGFbcLg5mWlkReeNfGbB/GO9r91I0xo/PazjFfaOilnd1m7CDT/yoeCk/G7jfgH1CCSroajo8GbS0FDp9T2AoVzWqibsbh7m+bdKqRZZA7LQW75uHYG+ex3DcBbsy29nkbdYBKewV/L/sFgOeLoEaiAASwGMFiwaCKps65AM+WFH4sccSuc5pNz74THX24UnsNp+mHcXM1clOYzQAgw3kF/VJ+wdEgJcFhJC/7tdKYvwbf/i8B0Gqqp8W+FkfTJALX/YnM1zYQPBrfhBj5GRUW6zMt3Cc58k6xB1WdBtYFtjDB3kwvfhiCBa6P+oerI6HQAsG1sWx1irH2w+yqzlJq5ISWhzJ953JiB91FOacnWCxpLuHUtDTU1Y4EVI8kkNnc610g7YgnIV0BbkG8V3UHhTSY3nUj9xyxPUPzEYcV4mYY3U3etFM4MvqxDM4ULWfQF0qC+5VaB0btB5kT/kV38FdXUeXozNeOwxmk7v1Lg9pcTGLtObQOJcxdtc/6R6UXaGtAkBDl4sussp106J2YPHIeke6pQI9UxakOE42Nckoc/2Fo7lkmdiXFJKKIT6UP9u32eJQqCU8fSMnDPcm2Ae6jGXs3G1ffZgyfuKCQg8CO7nE/1QhU5SPQS2WsOWVPy0A5S+TOMO4kqPx5+tgLZNun8eiaNN73gj/Et2y+17ctfljkGTwbomeKGoT7JHlM5m1YrjvjUHuHRpcmjtZuYxU9vS7WzHmWjklSLlXVo9UoiFFfhV8F1knns6dqL3GSFq4lZpJkaGR06gHGdmGsc4JexDU8jwcUr7Fx/LfcdbR1+kIKQhh+eTD6JTIaOv2wE3qNi1IccjJx1tfz69RWnIzeIMq6pZAd0rZwPLadR9xu0uL5IRViFPCSzfz7Oyr57YcHgQyag8PpXVVokDmSfHYAbuXeDH/lMmN8fgDTYyBRItHKGXdyIikD8zg58wB91T3vvMTYxnMLLvPjkHQmbVtEvVcNQq/T3hUDOROkR9QWMvvINKojjbbf2ZLHMV/Yt3M2PhdjCPpiic24XJNOeJE9E3YuoMazBvFtAUKXWz9AY3UhrziUIT60By1uDJLYvrcCAo9veRwxsJIHn9iHXtsXGI1ENKG9NBDHQj+8H03liaQnaGn7k3tSbs6Walb+tphW9wZuheXRECxYJYrSngWpPfEefkyZkcPtiw0sqE0mpzwWpn9kTfCofHHZdZPmliia5jyJk3cDUYJt9caskI9Rai9Q+tk6JMNtK58AOhXf8NFnP7KoCv5udrqvSxycalpGsT6Tmid3MN695/n1c2iiz8s/cPn2XIYfPUdlQC7Q0+Ml3cWMR990rg9Jo927hPleb9vM+5NPOVXe+expB2qg2KCxVl5XHkV0E3hvwW/IDG5MTq8n9p7PbmgFLOSbY7j78rouIb5U/HWVUNUFDnRapWhuunyJENCXmugZ+N5/r0SBkj4zuDg6j9qgF+njMBtkVrJGv5sJTDwxjuTxrVQ+vYtK5cRuWkGbpohHln7Kd6UjiZ/qTKznZQThTWuvQkBhSGaM98/caFnJpuGFnGy3Bf/7yj5nwJgTrCiIpUZ5h7SmVxjbS9Tyg/yBJO3sT5tDK5n9b/0riQ3Q5BOLT/4l3O6u5UbLPGYxt3vsUls0+dl3aHVsZR31HFNF4mWGK7L5fO4iIvOr4o38vszw7ETsLAWVVeKosPk6/jfPYpC3UVbkRpDvMXqDfYcjx1P0WAFjUwL59NKLDNxuWy026XA8PpVejBhzFZ1/I2ZLD9gtIDD1n6nEZ8Qz6KPNlGJLqgFo89Kwb84+yv3L0bhK0Eh7/Bcvl0giqivpKDrAOKfXcAzthKKdoG8knA+pjP2dhrrnCK/eirctVkJj/S94Z1SRMyUN2cJcnIQeESKLYGHrmdkE52u4MikRi/0om2M1qgHcGRVIld/3dAz+g07zeFb2Gs8QYtj+8OtITGqCa58iuukqSd2pNrDoGnj9ib281wgfNsBa11ib+VsVkRS43KTTXyB8XAc4W71JUbBQFJVLh3078YeieevTYZh2Sruh1SZJADvnb0avEJl/chTNliYb2tfFht8YeLaFgtAWflh1gNhAWy/1zJkPOPznKNwSqghIuoRoagN6KuxOqQTkU08SOVqHwRz8r3slCiIfvfwR3k0LGHBrP+627g2zPd5DfvpFwm/8SnO4qUvODOwlKqStzqhNXpgnXOKNuBKEzmqws4Kdk9SNLHGAfnEFNAw9j5tkjc28s/RaslJGI7j6cbZ+ODNH9txLAYFGt0ZkehnjTo+j3kdnlapPfgyM7ai85lIdAu83gmMBDFMn2txLgK1RlbgEZ/Hg0as0VUfzfq8xpbmO773gg+vBuN7dw8i4n8E8G0QptxVySgKKGFU9CjutI169em1j7KAizMSOFjiv0iJTa5H08gE0qjCizvnhWOfBg/vnwPhkm2sKJ423P3idtsgCnB8XuN3wIA/wW/e4qvltpmdX4p5QxwPz9lFlGAXnn4eKgzDuFILXeNas/4WUMjcuFiTg2H+Yzfy+1OB6wQrWJiemMUu0vkAxrvFMSK/ATiXBwe1p+qr1BDbvg7LRELWeemkEJYdGEdpuzYIlPGKVTWdmPpgNRO5+klEDT3O9IRB3O3vc1XfBPAyaM0DpydiAccR5GHktdwYqfRAuSttq30HLV+EjbyO4GhzaQ3utcFYbYQzgwodP8OT0y8QtPPmvdXJVngMdx8bhseQgIwb0AOUa/Q2uP7KZxoZR7HSqZ4BnOb3BvqQRl4if9gdzdj9NpKqcOrmtiom+oYHaPA9e/Gc97dFlKFZYYGZXBZ6mAyHRi1y0eM7eiKMhAkQ5TL8LMicMFw+wdc1/GHt+DFHXd1JQNRz0raAp557ygdHYSLt9O7NG3EXv1GJzbgEBzz3TqT+bxINAUWxXhZxECSoFRhdXZAio3ZuRiR3dx1mk9lR2RqKX2pO1dwLJLk08/bzQXZGY5bMZt7C75JcXsW5vEh0+6l6eNRg8LLyz4R3sOuz4JjUCr5rPWNrlvgsCOJoewBgNQ0YeYEytmWaZBkkvvPCKJon/lOxmkEsDgn07jrqR9KZFPO+YzIEvVtJaDqPmvoC55SFgDb+zmp+qNmLBTGR2EG32Mpv+oFO8v+Dmmffw8nZkS9JhvIy2GdljHWqO1mqpmfcL4hyRdfKvbMYbPJup8M5HMEtxFSG8+k9wVYPnSDrtwtl1IRb3y0k8/NkSJM7uQA9pU0DAtcEV10ZXQpzrEGQSeq+hKcoESt6OZLhXFb+s+AXB0FNdYG5owr2kEEc7eH/WBK40PwNs7h4fGvc7rR6l/NJs4e1bPviM6tX7C3jCrYnKFkfO/TaRDJ8G3lgMnB4HUgcG+bzIt7k+5LQm8ujut7nS+poVTF7YAVI1HkdWE/LodVL3jif06EQCFhfazI3MnlN9buN7w1rB6yO22wynNOXgNf4ap1RN/NE3jVc4DFj3hMjxyWRH5RDEKsZP+w5R1rMXigJslDqwr1nOivFW4tUvveZ1MhaQbQpgZ6YPrWcHUzzOYI3/ghaDYxTmxmd5deR5Ht/wLqWFEvo8GQBl++DiPBjyE6Ndspk+Loen4jYSJPw7phaBgYtOcmfjI7z9/WMMTtoDPfXXOAvlxKjz0ZwdTrJfBfSxPsDldR/xT9JxCpiCd5U3GrXG+gx2Ve65vjOP5j+WE+yiojxqHJ52saCtg5PDIXgZn3mbaX/6K+ZL3+dJv085WusPTY5wcjh2EZ+zZVwOuYZsRn60hbKCQM7uEklvXUKC424MZgW/pL+P4/gKXsz+Ci/FHDY4JlglHAd+jSDEM7bZjrTzQwEtGp/z3G6X8VCX+K9aqWXW4pOk3pzEki+GURv+MLFfWX2QcPeHSBmWw6M+YTR+u4NkeRJJE5NBkGBxjOG52OuUvHOdYOkCXg7K4lizChpvQv536A0ahqyT8/WNN6gNLqe9U898WS85+fL9ACS5/o3k54HMGWeiLPEksB5BgMNJX1Hd5szPdY6o9HIWOFhA4WZ9TpvTkZglDLtq3adaa8YwKvghLizu0miRqtlSG06dphHtgp/RK/Q8pLO+1xJR4PCMwxyffJz5V5wIrGohLOMpGHkXVF6IgsCcfXMIKgmixb6NRoXWhmzkIatmVIM3Z8qcaI+Uccd9iM065WZ/l5cnf81LvqHUfruc7GET8e8l7S4g8vCPD1PrWctPK3+yfQalDsxTRJD74ie4VfhS59GrulnpQZ9Lf1Oh/Qqv6Xa8E9DB1foEkuZ/iyjq+K5jMefKG3CyhJNuhCsGNTj36QYZD50YSssfY+hcfIjv3Op5zOkEHAjuJtY8Ov8tsofG8YHGjg9vP8XxmT2+dasighPeu1DonbGIEgRbngctpjvMfvksX+c7MeBmf4oH2BKwAHRKA9Xe1aQJOiydtiC8m8qVeQefRq3twGH4NixYCwG87b2ZPuoXUrStTHW6Q03ZXRT1fXuRSKxtnNxbfAladJzxjp1IzT17miARqPesQ6Fz4c9MX3R2Upvc3amyaIp3RRMwsp3F0xZgMT0GjO8ej1DqaBFNZO+O5bbdeF5+wlbPeJb0CJuWXWHCbWf4d4txBkgyGOwJpXIDkSoTQpciwhhFOeHeAgWKJdRM+B3fgGS0gnVfsFggyZDDOg+Y3ASO5cG4uhuYEjufd45+S7usCL1Cj0muZKhiLZqMGTw+pwj7zh8YF9J14sCtLPyrhSbTbeTaDgJcIUhXBBIVJeoBjLzui7zDHplhMybtSPhXw4T/2v+E/a+CfRcuXOCTTz4hNTWVqqoq9u3bx5w5c7rHLRYLb731Ftu3b6e5uZnhw4fz7bffEhHRw2JpbGxk/fr1HDp0CFEUmT9/Pv/5z3+wt+9h0ty+fZt169aRnJyMh4cH69ev56WXbBOBf/31Fxs2bKC4uJiIiAg++ugjpk2bxn/tf84sWHix4Q9cG11Z7uvOMP/zCLrNgJX95uiYwN5aawDnqS7BSabsHgM41LmHrS99RHxGPDFfr6D5MdvFUpQIWEQJMe9sYa2ziXTzk1bgKHkt+E6jqNHMyJFnObd3LM8emovdwJs2x5sFFR0GFUNveDPq9MOEvGJdcS0qX17Pc0KulxOWNJt1QxcgN7VClxSRytKMh6MGi9rI+j5f02SMAvXjsFADohTDqYeQubehl1kQTSokgtTqnLXlgmM0Hz54htq5Z7m2YxYx3i1W5SCpg3WlNrSyySUbVZ0P/2S1kum3HWxSAiA67+CRwyZ8ng9kmZseue5J6AX2JehlXNMqaQ4pYWJkFVJdLaicqRK82PnEBhw0PtjXQYPSw4bd/qC0gea6ME7VnmBro45IfwXD6ekPNlClJVVioUViRtcxCLW8553saMvkhRXP8PG5KTy6/VFyp3aBHaGrQO3PAE0ef/U5wiHD4yxf8iIdjmvgzjtQ8D34zYRRB7jmls+hGm9ef+Ad7NX7eZ8eWTSz3IUnJu8j1L0IfprDreUZ1poBQcDOGE6nDqRFSjrvTsF1SnXPb4WZvMlnSDfKiNu5iUalaK1ktAsGuTNN0iBy3nwCqaOFlOkbGDbE1oF/yP06i5ak8FSmDzOVP+DGWhi225ro8JuJnfAr7hJ4+MMXqfStRHhGgNRnrNViD5nJaLzBoZl7KXO3ghirelWKAci0Ii998hIpA1L4cOZhnhGsv7dEFFnx0wrcGj2RWszsdm9m2SPR3cc11z3GiohrbKueyt8l3zPYtriTFBwokLSx4epi4loN+A61/V55h0cy8EY8acDJmBw+WtQL7MOCLhx2ljjxxcUwqoO8rNJICjcwWJMDr7eI5OxcjlmwcNHBn9FuvZrSV51jyYAv2VXvjWPuYjydW+CBJtCUM/KtZsL3naYpYTQdlxbhMTLY5rq8GxbjcN2f2zNPURqYR4upp0pTUPvz3AlromrU6M20OPexPr7+VkaTpOAgq977ka+uf0986esMsFVdoKP5LZzy3Gl1aMG9ANaGf8f7U+eAvsEaBNSl8kboEn7/YSl7jo9Ev3QqHy/vOf6mEMeeFF/CSuxYalpKaIcc2IFUU8TmtFHkpEZx+PkC7B7YSrBdT0WL3FjOmJIdtNf58aG79T1t6lXZJzU1EdRxl9oOA343E6gOqgKJArp61wgWa6Vue9Fr/F4H4ROX0ttuNbjT2erLcCcBF79WBMFWUuCz5hEkHnaiJDqLh5/9Ey9FKPQC+/Qmezo0MkZtHszxUf7MXmtdk+UtCl57/zWKotooCevFMu+yXbcmo8sfhiaqij7GGvp2zu3F3wX98JucS0jH6cogPr4Yy6SB5m7Jy5q2Eq499AEtJgOu9R4o7yXe669BRyladSjrShTcbJVxoM+PNAsu3A/2FWnGg8JM4bBLuA74GUyZ1qQUUOc+nvcmLcChI5Lau7vp8FnD3C5JQW+XJq6s20aD6I02HHa29+o3pfBkrqmeCs96nln1N0+46ygz9STqmvTN7J+6Fb9yP1Z/vxr/gIs216RGg3tGDEFyAy7DOvCyKwF6gtQzZUMo+zoJTVgxQVMuIvKyzfGiVMbf7TApIZvjAUbyFQ/ZjHtaStE6dCKodYTJIB8rG9ws9+bl0ceQD5WzcK8Drx58gxUnYrgnnudgaWDmiqP8XmnHqCMTyJHfBlMXk9ylP5sK4plRlcMDWaGcPz+Szv6tVonDLnB/9bi1NA9xJrq2E1M93Lov0aHOv0vmob60tysp7LTv6oDYY5X48no9VNdNZkbOMe637df/IfC2I0tWtDHcv7n770+3zuGM6ju8Y1J5e2g+FUqR3mBfZWQjl8IuIjM6YzBbWaS9bWuzEr1Ey1P/eYp693p4Auu7bmjFRaomvOo1pAY1m0YMRaOIANM2iHwS7EMoLjtJzuiDOLq0MS+xhmTpYsb6P2jdM4Os7/d309O4ND6FkbUinqpEoFffM4uJaPfruLkW8FOkiT877lj/3prHIp82yv2raHaWkuZSi4vYI2tWJzgzshC0xjSmFB5AWbqGr5abAGuU3GY/mZ8fe5naIDV5faDeZFu6J2ko5PaPvswrVOHtE4x70t/0DsjCfZ/n8sR8Xhi5mXeCihBE29/Mo+pNvnpzE298uphZAVXAReiVCpfqBJbvXE65Xzm7l+zuBtwKO0u47vUhzsYoxv09m+uBNSRs6Lk2R91HmOtiuNsnmy3rz+FtN8gGDBmo2ICuHIr1j0KjM1c7KhjTS94rKXsawRkqPnj1A/QKPY/0Jqh0lLAioJW7mWacLgwhOtE2iQ2wIfgW8uAqplWC2uz9r3H1if2MzUoHFjLrhb0w8g9ouM7jB87QfCMLL7tiVoU+ToVkHPTqdLXM+WtulUxHbrLwjSccMPdU55ktZv6cuxGZQcZzus24q2qhV0cSURTQOnqi7nyU5MvTmDDC2eaaxK7v6KvQ0Dd8E5UtDtAL7DPbBRFZDPH7pzPSoRVhqe17qVH4s2nGPtTaMMbdySexC9AWWzK5Oecub9bncH2/Lzrq8Jf00G4EQSAvMg+pyZHPH92OzOTED73mHSJNJnRpAdXKJpYH+yJ1sa1Sszi60eoeBukF7M18iMVP2/aB+7nVkyrKgAvITRncX1c1mOusDbbw8Xk1NN9CEPrZjPcV3idp8EH2XF2ATNBaiQuCiNbxIL8/uokkH08MT+/F0bkNQdKDzqbkP8zVk+0Y4ws5El7OZDvbmOJcSAC61w/Qb9tyHJvLcXuzB4hRSSScVw3mcrMn+osDCBmTaiXpFXRRF3wWcXDHdNzOJzErPI+mBfexFwGJfDQ5+b6417lBoJberGhREc76/U9id9wRQ6ucrUkP8nqXFLeA9e23DxrCZ4sfJ6d35Zwo55c/XqAi3RPP1Zv4JKSRK7q5gFUeVRSljD4yDZcmF0Y+tgc8m++7KhG/gZkUdsgYemYsZUPqbEbLtbmYan3Ja8vlbGECy+I1+Bi7quxac8BrPDG1eh74fib27ZHMOvGszfEDLTdJ+PAOi5p0ODq1IrNYE2pqfRHLpNvJrx7KvrmnWeLfpdZhbyV1CoLA7iW7cWkJY0brWK7K5ls79ThYCSwxQi0T9E4UROeybuFJLhEIjQo4MRS8xqK5aUfF8SSG9R2Cu3MMTnIHODXGWs0z8TI+UhMBSHAt9UFQ24IdAC0O4+iIb+e8qpnz5Q68et/4mAFSWl0uMMM3C2OTnHvw6jLFP3wVDHYdRXyoLCO0LdDm3TlS+x0fn/4PGYGfcGDQ78zxeNFm3jduODL2VBxKYIRLM76KnmfE3HKXjx6dzZe5C1jsCrnaGut+dGUpGDuQyR7DElCFOPES04fnkibJAf9dXbGpdS1ePuIWupGnGdEAfasX9YKXrFPtrVUTA8x9Yi/5btbnvz1gEUuuuiIac+iMjyfTzZVH1NHdK5FZ6shev6fAD/Kch2OQGXhW3GQlCDam0iiLYF6RFM9aZ9b8M50bY2x7fI5XFjPVU8G5r2cQnB/GP4t1NpV9Q/3/xsOxDr9z83Ay/cPfCtsVo8PsgCErnNNDr1AeUM5qvW0f+50Nc6DcizaPOtIS0wgyW5VOasUgonY/QEB1GA4aO3z9K4gz5APWeEZGK+Xj51Eiy8Gn0geFgy2hrcGSh11DCqP1fmwZUkqacBVY2z1+M7iJDD28eTYCP6mZ6LAd4OoPniPR2YWwZcAVAj3K0e1PggEtNl2dBF0da3MSUJwYzcz1zyP6OQKnu8dXzfybVwdcxigcY5cikmpFTy8su4pzBGfcpsBOygu/DacpvJDeELyD/EviQ6vZtWkelqw5aMfYyiwnqX6gvMGT5nOjyYzsApprrD3IzUGfcunUOPTY8VXsdhyDE6xzS3v6fwUOzeCwZzmiRoWdzHY9ECRKtgXlMHHYZfrf7M+WllE20qfVyjhWD97Y/e9u8NUxirEOBRADC7JXcbZkLS+pepADQZDw4tYp+GT0gbc2goAN0OJr35f9Xy1B1qwkMrAKf/tya7Vr1NMgCBxpDGXH3TLuxu5HEm6Pe/0IBul+tR5ce46D2kQ+bbjNO24QJ5P9S92iSDWcKSUwOSaThaKKDsG2v25k09vc+TWUiUzk8rDL2Dda46wRluOkqEVeKNLwwPePYTcyGVlnIXTRnS93JBBc2EnRIAPmiJF4ugPt16EtDzLepNgwlEozOOhrOX5iF0NDhjK13kpmNTsm4CIBRxOENzyGv2BiC1c5ULuLN47sAqC8JJB+N39k/c0nuTyukWZ1cHfeRhTho7A6Ol/6nGJ5J2cDzdQKzdzzf7KlffDMgXCtJ3OjI4mNTYf2cLAPZYjSiUCZP95O4RR7ZvNb5RpCRu7HXe2LaNBwXSsFi4inaMG/CAYapjEl7RmovYAcmOgHq4OLmNV4ioYGC07SXnFtv00Ycn4j/z8yHJFxriGM9trRLMYq45lfPYs733sQpnXE8sp3vOpXBlUnrKSH2vNMcNLzy3NfYF82inrnFJ5wGWNtwxNujS28S51ZtvM5Dk8/TEpSSve9VlsqSYnVsKMFrolS9Is2kuwWwJSuylY3STHz3XVktrWyf9WPyO4Dha+2zeD34zcYdnUYm5/5GHfPPrzci0ZV0hbD8DIo8yil7ekv2OHmSkkv/2W4Qw0t8Z0EpvTnmeQRSHQNoHS69wLQgZxGuw6WDcrjDRcwdKRDVw5PFERrGxSzHIe1R7jeFMUgqQIkCpzsd+N2oZg4VS0TMv15dNmXkPYcDNoOopQdZjXRsXcJyVxGReYytOteRmnf8+6F+5ylWpFDZUU1erkek/lVQNZ1WQLHph5DoVVwVgyhUOUE3Oo+9getDztUWcwonsGhvQOpjFD/q8Dh+9HXyIxK42KuKxKdbbyjkCroN7GW/DINc/72Q9o/i/GAWHmUR0LfoDl6M4pxi0mzgFYLa+8tJ+YK3kobQcPVfnglZjHTDlItPYoJfUmmzdOOZzIDaPt6HiWD5fBmz3lznFWcX/UDC9MXUvKbgqhXbQmGW+wbaa/z58WxxzDLA3lcPxePnjpMWiwuJOth5AUv+srX4W5eCPQoWIQIZWS9/hwVfmVMHXKcbxVWP8HNeJYA9V8UlD7MJwnr+Fg08IvFCkZbLHDSNJQt2/3xsfPl0JHx3Jx7kwfXOTJPksAJSRFtLXYo2ipRXath+IlUhvTfiGeALWEUwCdzC0OPTCby+DgktZvA1MlwlZnxG37h02sv0pjSD/eAgH8d91/7n7H/VbCvo6ODhIQEVq1axbx597+u8PHHH/Pll1/y888/ExISwoYNG5g8eTKZmZkolVYnasmSJVRVVXHy5EkMBgMPP/wwa9as4ddfrZtua2srkyZNYsKECXz33XdkZGSwatUqnJ2dWbPGmpS7cuUKDz30EB9++CEzZszg119/Zc6cOaSlpREXF/c/94P8HzNBEHB1de3eGAUEEtITGHlpJEaqKesjI3pJA70BPQC7pnJiL35N9Ix64MfuvysEHf5OnXi1ReNfGU24vW0rcd+OHbwd/T1H6xz5zNLEBIkKlO7gNhhkzpgUEt4adJ5oRRPLHKBQTLA5Xm6uJv7mARQVRirsgujjaBXoEAWBGYdnEFwcTNND9vgEd4FHHlb5pr52Hjy8YTffX3yLzZvG4JTkz/NrRKucismEJHoPp2NgTNE5HkjNZMi7M6xyaemvQsgKvm9X0EQbP87+CwmQJQggkVvZyYKEE8eHIzvcF781f6AO1XE/2IcZhI5OHNyb6eero72XnCbA2qhymt7ZyHoneMsdNPp78gbW++JT6cHiXfO4NCXT5jinobkMHXOO899PYfTFcVROtdiMLwtooWntFgblnsBTMwhlL3ZxkRFWV0CDfS4PjpzIhKgu9k3Jb+AzCYnEjvpkOflt14hudmKFxpWvfeOtiRKfKSAIpN0No/GH6SQu+R1NVIPNuYOdIhhw3Ui98hd2Lf0cZ9+uei+TjgjO4VgLLR0C6XeHkDWopVsmTmZsojQEtjSZ+H64N+DNMpMB5UQrkCicPESjayNatZLNU5Kok0wENnWfN9fozbYWcJJ1MjF0P1mmSaAabJXJAJ5V3kR1aRxnjVKcQyoQBUDRlbgVBELqPqBgyAGiisFfCl6SGpvvNcUhA8soFXWONch18u7EqZOhBLuQCqodLKTEpaKXd/Cg0xvdx/m2NpPx4RpiE5ppcanHorWnd+JoofAfUm40IslMJbuigxiVrdRbSXAlWlkL3tXelHlV2jikgiBh+80Aig+MZVZeKJfmZFt79qU8CRHrwHMkMSH2OC7KYEfNVnQqN1x7xYqqziKeUMq5ahRZ1e9FErxPgVwLmR/zUOTXvDF4D0apC/7p+xGCJkKvNuZTXDI4KXHF2WzE26JAaelhfRrNRsyd55AajJx3fAbBKPBkr3grXYzgpSsv0a+oiBFpr+MfFAI82j1+Ve9E/ZpfqLrcnzF/zsf0khxUXtYPIKh8OHTwQYx1IjLggHiaj3vJcAmCwMqfVwJWIpbbEGvCSzTrGTTpOlEDsrmg6MN3oZ0cN/VUNylMZVhK2lC5VjHvzGWMchUKaY8D72H4Ao+zIrKYRj5+Ywsxkh42KMCkOwoW/fUmd/s7cEScytpptvcyMM+D0QdH0wa8FZ3Fx0ttk4xGQcaX679EL9fzkAUM97kJy/PsEKrK6NfHQlBYEdfkixky5T3i/lpNVYhAu2M8a79eTFhkCcVxFxFFK+h8JT+DYadjMXnfQWevQieT28z7Q0wtzWYz710GmUFmw4yX6qu5OKKIT0rVaF59ibARVdYqiRPW9FHA1Fb033/D7MJydgD6J28Qv9JmelJ9vubovFM8PPId1jph7WnWBfYJgkCDewN+FR3Irn7KJ96VzBWtAfAL3jcZEFLFqFPBbHpzFe0TepiEglTJJS1ggZSQPOztIFLo+T3l5g4GK6Hcu40+ryfTJO9rc03ZYiJ539pjCG3g1bc+4YasP9DT+1TvLuHPBb/xQmwtD0fWk4EZfhWtJ5x0FfBj+c/L8WhyZfdjzsROHGgzv68ll9Xv/MQDlwMYeMmXLfHWcQGBOk9rkrbKMRe9rI1ifQf36AG10iieC7pJq5eUvNFZXBZk1oqAvu+BayLqIyd4fNeTpCekc+aJb5ji8ixoKiD7c/CdxucNrjTpO4jJDEWn0P2L1fxDRSihFi07125F8LBlXgLUN3nSvPUhpjV5MerBccAZm3HHJjsicyN5o/QCMp0792pPBAQkRgkmiZFW7xqa9baSGW+75TNCDbMvBBNR54yruy1tNOl6f6IzQ5G5tBLkX2/dA/ttgn6bcMvK4bOafO7kJrFnxxRi5jeTtKYRLsyCyKcQhRDOjT0HwEEzPGhyBdf+1k+XuUiAkyOQVbThtHYjvcG+js465hufJ/m6J18I/jQEebMQoLOcl0Zlsy46m5aCB/HqVCPvFWRKJDL6Hp2IY6sTp8Yswig1sslYgkMXdJvenMbd+HpMQhOnVr9J6sRCnu2V2/y6PgzfggaUOhkanQydYFshF+cYQatLEKUtTVwozmGcaPub3dF20tEhJXvZzzzkBrlCf5vxcc53Efo64H87ii9ronDpzMDVexz2xa08dGYCxjYvdj7wF3q5nrWqHn8uzKmC6T/uIzBfSa1Fi/N9lTL22lzCzFX8NjqbvfMu80EvCU+AfuPHURipwSCz+ju9k20CFpJmXuamTMPl32ejdFp8nzAkqAQT8k4FD2ZHEuPtC2ajVWocoP46cwd/wMnGGbSExZMqG0J/pxhwiqGxOZ3JR2bQHKQlvcSbgGlp9Ab7avsX4J20kRMXYlj5y0iGPdGT5BMFgZleGiqMMFX3MwEuhVib9lmtpm4bjwzfxIHUiaiS5+AUafvuhAnFbLozCG1BIBOiK8kLtJUSEkQFnYX+9E1OwiOilD7yo0DPw9Dznvb0vwIQFW6cLV6Komwe+Ym/g+ufREp6GiCNcU3gV91rXGuO5kuv5XDf+94uOqIfdR3/ur684p/MFeNKm/GO8FRqFhSxR7cDuc6efs0TiKGHEJRYMZWg/aOQmIycHX+V+81T4U/z7VhCzgfjH/01Qv1c8OrhwDcWSLj+4TRc5x7n3QdexNCYiswtEVGE+sD+2Ps5k32ujlIn2NXzcxOteYjq9H1ES9xwqtVjP8y2UsdJ8wd37kBI2HeEyW+h0iRzDzSTigLFO6Mx5gTi/OZXtCmk1jU07i2QO6G3j+RXOhgDhBWGUmiwBc0A0ptf4u7GL4ljC/ZBZ4Fz3WMKuRvmG1NQtabw9bqvUbgF8XrX8/l5RSe3d7zFnUkGTng/gl3Y0J7epRIF6XnjsC9LI71DwYeNMCS0J8aSWPSUj71Elk7Kd753sDcGUtHrmorERN6YsoHhV4cg2zeFitBWenGcuGYXy9/PvcGcfXNw+iGI1llma190TSmoAxAE6Gx0oiiwFPdO2348ALfE/nwvOUuiN3zqAZf0WcBYpLpSOpPbkTdlkT9MQkIpHJ2VzhS/LmlIUxmv9a3mYK0DWanv4tolvMKfDuAUy+8FI7j7mRfJ487gZQ+TXesZ0dkVq8ld+KhayqBmOxIz0hiQuA3RfBga06zj+maeSY4hoNCeR8+NpXDYv6n1Fgdf7vhPYPCeGm4WxiA8Yzt+wWk/BzWZGKX9kJh6mtekGOM5+qsH/ZBwK7qNvHtSal22v+EZdIZC3BuWEFxbzAOLbYHG/Kgyyr0zGS9/lkKHOQyT9jjX6W31/FUHyZ0nuLJjHO3hAodXA059wGKmf/0Vir0VPCerJs2xjAbFMGjJhH/iIX4jxL/FBLt20v+ayJqrsVxe12RzblEU+GPBX7gu+osQzev4OFlJIpVtlYj1nzD5xGTOrDhKcsRWYpo/ZC1Wab1aTTWnEwIQLBJGNDyDXDCikqoh3podFa79wyubXqHKtxmPRQrm+Nvmh6YoPZAUuZMTpGdM9BXenfIeXzf2rFkjXL+j5B8vkm+FI/jMoXpYuI0/qZAYWPjXAsoCy9i5dFc3SeKe2Ul/4Px0C2fiItCYWvleZgWMZYIJrX077U461IkDSehzFqWsp7VEvP4ppq/8mRm7drN221qC5l2wmXeg6g6J++ehleupcv8Tc7Qt0S7lQj+KDo3gwQYXaj1ruPL8UwwLt74jAgJNrk24Nbih2B/NCRfbucXOap4clc3WMl8OfTUDw9yaXh3grGYYNJyQH1Ooz5pKwms9aiLeEa8y8mUjS++EEHZ4IjlzqmyOW58ej2tKHzTu9Zwef5qFNlAgfHP0aezPVHJncCKt7l17XfxGqDqBWR1KcuIL6MnEkJlFlL3epuo7XxnPglJA1gROTWyQ3N9OwXpvtEot6k41ymbd/3N8QXkYj8oVVIekAtO6e8EBNMrvoLWYaTVGcU/RQyaVkeXjSYWqkh3nZuDsel8/VlUEmcEPofX+k6ce2MqZdqucMNdWgCBSUzaHwG3Lkc89woLZ56k0LYO+74DvNPCdStP+zxm6+VUOOAscGjCej5fb3mtBEBh7Zix64JFxZ/hY/pSNXP0ZnTN4NWPwL+W7B653/z1biOP8h4OIdTSjnX6aJX2qkBgarC1pai+i932Bj175iKjWVVyO9SWvczY4fgDx74DHMMZeKkObMZMOyQ2EzlxKAkVozQLA7JzI07+OIionHKdnknFUW6+5d4upRstajCf74YY7G7wvc8UYDDWnofk2ojib22YpbepWnEWQy6yqAN3fWSuwZNcSGnwd8Z5gZN3Ql6A5CuxD6W/5lMCyerJ8n2bzmidR6v15VWmNx9VSJc9VVlF5pwW5eQte6S+TvtYenBOg9gLHIz/mla1fEVzvQlTK+wzofwGV+T24pwMU+zLDz/1NTEghescRtKvewqXu3n2Ab5K34uGcwvxpGqb7dREqPYZbidrFuzno9wMzhWYui/XM9a3kda9f4cavVrDPqOHdyBTOjG7HR6fiZHMEVT7ngHVIRAntLXZY6uVYLEZ2tUGVQxJT7gHdCAxYepQrzbDbAUJk9/upIukJ6ZQFlCHYaVBgm2PTmR1ovBlNQn44Z8adwaQy2Yy/FXgT3YNm/hZEVsfUIHaWwT3FM4uZ+A4D/jcG0xJVym11DTGSLt+/s4pfXWo5VjQDWZUXq9StaHwXsqYXoSLy+m4EASRSA5o2Oyj8CaKfB+c46qOb+KvPPyz/fTTS7Hr+1NWyfFbPvhWeEYhbjYHndz9P+diLCGt7FG/UpgZecoHbd3w4sX0lDg/Y+nsCIJgFbgy6QXZ0NsOVtkQNAL1FpDk5nuUHxzN8w27b4wVYMfMjfvpzHtIDU7kS3NXb0WJCJrQhFQ1YLFYg7N7/B5BaOhk7Op3f/Wo58Od4djo08ulXPeSvZr07nz7xInYRtZRHj8ctwN/mvLPU27laa6Gq4w0UBcFcaxHpHa3/di0Bu9/HYr/8V4S4i+iNH0MvsO8KI2l7X0FIcRDagBoatafoDfbpXX+jXHmAKkeraoCkK07LVL7Guc+jUbcdhZesX0rSpSpjsUCD3oWQc8FUDqrkyLQjyKJk0HCdn6P28+adR8g/nkBUxg7MY4Yimk2Uiw/gyRZW10A/t/+Pvb+MjusI9r7R396DGmlGzMw4smXLlszMjtmJ7SSOA44TBx1yOA6jw8wcO2g7ZmYmMTMzjKTR8P0w8kiTnHPPfdc667xnPc/ttfRBu6ahuqur/13VXR2Py0vjSGj6kMNjGmn2rSTN90FotttE3V1D+at5A/XKMexd+Aa+Gnde4Od/jdf/Demf/o7/6fT/qrNvzpw5zJnzzy22PdlsNt59912efvppFi60g53vv/8ef39/tm3bxooVKygoKGDv3r1cuHCB9HT7tPnggw+YO3cub731FkFBQfz0008YjUa+/vpr5HI5ycnJXL16lbffftvh7HvvvfeYPXs2jz5q94S/+OKLHDhwgA8//JBPP/30f6An/u9MoigSFjZ4nUYQBG6flEdlZCtXLsdyxL+J4YKaa2eAjfoiJud8hq7BDUOwnqvYhrwKAgmWIo4L/hyefom5t+6hxCMeiHDQsysvUvfWLFoj4tk+9RDT56rsmzmfseA7Bpsum2N6OBaXzWfAncYHnd4E1LfcSYA5kuI4C1VxHzJ64KKa1NTC5InZZMVWU9X+BIdLVzI15he7M0+QYpQl8MShIzQ09BBZcomCIJPdcPSb2n5Db+DdoBDDJYylPbjqL9tvktk7ha9r/ZDo3LD5tKKXm+zKwmcsyDxApuaolwzN6HN8HN9Aiet/8PBnUwnDDm3lSs9UQqcdJHdarBN5keEiuYVGDgV8Trasnz+W2R3ct8q2cW/lKB7rUBC0+CBfjihzyvdsSwLltnpmTD7DqBlXyfe4zonea5XQJ8LDKe8wP2wP/eZuwG4ssUnd8PlyJT7IuffJZ6kzTgBusL814RJErdtiDm9tROrfCXHVdBqaIfReR6gKgMdd1MSMPseNBcNIS/7HxkewEt17kvDmKLpCD5HuNQBiDG3MlT/O9vOLyJnbyJ+ztjAtcPB9OpvUldd2D6em0ZsrE+wh9XpMzXgMhBT0NNcx/YEtbG1W0l4bRp3nKad6L7bGEfzh3fg3+/OkcjRBv/iROIR+pScIy/aJ+E+6wItrjtKOAGO+c8R/75QG881vE5h/PomHbt9LQ8Behr5FOFKjZ9y6w5Q+dDdLL4xB+84eYD0RvUf46I6DpJeEU2ptwio6h4PYLp9AxfQDePf6MvzoR7gEFMGQRTcz/zsUuz0pyrwZnTaC2QpnOfpg+iWKzEbWDUTfdAKkosjWM/FMKIki8oZDzE1vgN5KO1FlN0aedPmVvb1ce4PZKXX6Tufl10tJLollh2c/++e58lT6W9B2ngrXO2gNSUMidWfFhieQ+tfBkI3mM8F6Ku//iDlblqP+NpmIXwedZt091Yyo+hFrvT+vPnIdcrMPMLiJLey8gmvjryjbU/D0UKBROIdYrCOQb1xaWaZzJSVXizAkZANAsUHPm+N+5858+60dg79zeJlR1rN0rijmM6GbkoAe9rnPZDqARwoP5FRgs0GfuI5FF/1ISh3FrIF8F/R9/LnhL/oNCSx804JbpAV4wVHu8f4OJMvPUq5pozfRzA+dV53qnRBagW2sDV3uCNTdvyIIzmE870upRS5v5usiL2rDzMgkzjDAX+wmJrAD/eUkXs/RELV8OkPhsqZez4w/p3D0+t8pHp6LQr6OTLdI1irOULaxj9tObyah2kKA3EKHYtA4VJbYSEnwBRKrJrLxp42U3O18WvuVRhV9ViMVy7cTr7bhZhp8R8smdef5FoFzBgvLJrYjDffgmkEaQUQlc8PoPppu93pavEtolHnxz/S9eR7FsXvoaRM53BXOVumg4VajK+A5Nxnn+11RGKSI1h7HQ+d/dsXwW3sbxj4lbY1eFHUPyomIwIId1zHi8khe2/gaP/T2c1w6KOQefYU8f2wmR3pc+M59GzHOfldEQeDPJX9idOvlwPnVLBnh7hT6w91Nh+fIfF4ww3MVsCtTYT9d3JUH5h4kYj/LEhu4vDuKc9+nEDF/pFP5ZZLRjC3/iIxfFuOjMCCut89bCUYWKkWqDSLblxxCL69lnTB4a9BPFYes7gHkLrls9j1EU38ki0UJRN8BggQ/6R8opQr63HrsTkOZADV/2p19lT+y50IMqReTWFYRhSq6+l9jcWLMPv6abHfR+Vv/7eyT9NqIL46nH6goiR1yN8mebl96CMmwC4ib76R7whyHisyw1DHhpWfI09awWMjGSxo+xH0P7flhHLoymkUdiYj5SibPinAq9z4xjLLuAOKefo8FQT00CwLsywCbFTH8WwoPRWK2dpOdWsNJpSvXyz3sN4WlrgwXcjgSDI9VqCjTKRE8gb56KHgLguZA4AxmnYhhzfbplALfLNVxcEjdZkHCrmOpSGuC4NQIVO8P3K7zGUNcqQdN1j5uODqG4NwEFn50ebCvMBPdFIhLuyd3frGWppA6GDLlO/vLKFj5Ib2N3siM09F5OxsE6n1q+P6JwZBdz4i/ONFna1awbGYerzz8Cpe8lzLtH1dW8hRj2fbjVQDSZhxkU5zze2nTPWIJWv0D7xxuYl58A+2WVsLCwhhxYT8u9eEUt3rw20L7Kdeh68o3fXOYX3MeuVFkik2NXOF8mjS8/E3adocw7xaYngw9NudwPrcOW01VRASqT2/EJfIfTi91NMklgbj4dTN1wmLGRDhvzAFeuzKFSR8nkgBMWHzUjucGtkztVoGGjGJyo79Dd64Ml2vnFsy9uPno+PmW7WB1JbbSn4Oevgx9AfeJphDaZO0sz04hMj8Zw30eg+0S4AtXFbva5ex+bgo9ceN4c4gtw2osIvuJWwZQ7XkidFnA4Ns0oTRSvXcy9Kg4kJWEYsk/3omzWZh40e6MmrHuEhIX57dU/Pousa1kNFd/mosgeRbFnWrgUXAN5aef1qNpKWfGdRv4dMSv1KoGXUAKaxONn+sJ9Cpi96gMejycDaf5tiQeqIfZp0fy0v4lBN0/zMk1O1rzCC+OrSKiAuQCWK3OY73O1cLlVg3YBBY3ON8KBFD6v8but72waVoIvfV36vunMDTY4R+iB+2p51H6lfFNN9w0EOLW03KalOsqKe9eROzOH+mPToYhL9S6Z8RwqOI+4hp3o90vkD7qHwBG34eqsxPzqDImD9tFq/GGQZpEwXHrOggv4htbH5q+CO4RJZC6CYAwYy/yhGc4MuFRjiZe5uvuf2o5qGofeJ9NtLA83tnBJAjQGD2WkoRe2r3aCZQMPp5XqhJQDCvArJ/ML7+vZ+q62CFHSKBzzByKo+fQfeUQvxT5kDZ2UAe7GGvZPbeA59vgZDvY/vFeWqhLIkk1b5Ob9DZFMR8yKWQt/1GSWCRIzVK7A1nuDvJB5/Cy/TMJy08k4uX3ELtagcHoAw1iMH83qbj3izt5QdvKjDftzrxaeQx/ZIDF3EPapWSMkhCEhYP6Qm1rZZmLjGKJicqhDTH3QNt59OrRnB5zGrcODyZsXYp1lRT8J8OMk+AWTUHaeE6POMAzbz/N6bPjiMJmjzBhs4AoJbb4S8acHEba5MuMTfX+F79nOn8hN+F11A1zUItP/Ifhjt26O3nqladomjV4w/+8ZThBRzoZL7Uyc994zkzJdcoTUtuPdvs0uofXMd77Efza5sGQO86TQprpjagn55KW+uaJ3DzkIoNJ4UXNZ9cztcAeKu+MYgBzVf4Ioctoloby2rrHMMSVsG7FFu4IjbDzqwpxvOM8PyuFuefjURsVhMid30EUEBihAIMNfs5/jJtT7AZdl+4Cxkc1U5SWTWizL/c0T0A5fxALSsxdfNGQQE6jBzeOK2J4wAF6ZO8O9pOpGSE9B6O7jfULPuRqxwqGvuubb3yaiy/4Ezj9KtOutx8EGnqz750af1JzIrGugOyO+8HkfKBtous5xt5xka2Hh/F71XC6RhwBBwLH8ZSF1CIiWkQE7Puh2yxv4rq+iIwSKylX3yGrdBlbhlz6Ptmvorc6lkq/37luUQuW6Pah1RKuaCDtjqP8/OV1HHt5LQk7b3Wif9wZxDCZhayZ+6kPqmOh21LHHkrTdYUzQSIvUceH93yIoHY+eCDzSGK7xxucGXEGq/cJ6tXuvDT0B1ce5aHEfTzv+z6dtjjaW6dwDWH/JkbQ5buPogQr1UH5BHk6h1qTt7mRfiaZ9lV/smRSMXVCnxN9W/RXeCls3JFYxfCAZgRhJ2ifA+1zXM27wIGRtxBRKmHZH8u46HORoQfaBEHg/vfupz6onu0Lt/PPaSOYevjr4lSyDttDWge7Ox/s9TRV8/y25diuJnPJrZfgzAEHlVnHowfmYiuKoiXgE/wb1BStyGT2wLvACqkcb+VbGEJaqN7/AQ0BLSxyOsQK7SGplPnvZ3wtjJTG8oyuFHqrIHwFxgD4e/7fTIitZpgCWgQTDBz4AbDKoSa0hpiyGLqznN84BLstYWxRIj1WgSNTjzgO7l5Lf8n6OHb3l9yshmo1+AWtx03Wzs7aKRwb+y0qZRTHRpzgQwHyNelwcTcA3mI3M1TQXnacH16dizivmEy5J2ifAeAW4ypOn4zHIklFYhGpndcFw1+HlGcRTRpUfSo8Ozy59MgSCrVFEA5PRwkQBbdsq+WBzBOciC0kILGSPFMMki4gei1I3RDKYWbuQUYm9vNn+91Mrr/M2gkrHHEiPG06VNXhyNWdfKR5jZ0V3nw82YcE4FTDZLr21mFyL0NMhFf9u/C4MB7mnIDObJYb72TPmXnUJwt0BjWgVigg/X1Ifx9d/u+kHJlEWE0E/UqBsqYE4oQhJ4rr97HYpZknbznPqJJl+Hc56wuAlvB0Qqc08Pabb7Fk7ldE9FTAr26Q8SV3Ncspa1Ch6pUyXCoCVrujcSAND2ul4rpKOt+dzOVLowh+zb7nMUqDef3L2WhztbisMRHZFonUd3A97RKiSKoCqUmK8mAmitBGXh1yciZGGsxN5s+pkhbywqEL9Gc62+AEm0BYbQjpl9KJHpXPmLB2R9QCgNdq0mjXHCF34V+ckcEOzZDbozYLr1PGV7vvgN3wzNxytsyOs9OsRrRSE4c2fkr15tvIOJfBkcUD757qG7jOOpq2GbdRLFmJf1g+2vEv2CMLKezr4sP+dXRaoXZcD6tv+YkmTSBDQ1PHFYUxZ9cYpL5tPDqiBtGq55pxSGVtZuG5Mcgqvdk7ay969wAeGsJziKmJ5154jqJhWRSs+At3ubPdBeD2SyNJLwgmbdJ5ylXOGkVv7mP+qU10R/1A2r0X8QscDkCv3zTu+WUBMbtPcdGwm1S3E2T7TkIQNgPQo0ggRVmBMqmbe/beSoens27vVIzgatof6Ly8uX/sH0jcAmBIJJ4pns/w4KIPmZI/kZ81u7hdfN8p/3YPI94z93FLTAvrgqH4HxhYEATUOjVZqVn8teQvnrXuYehRKaUsgJIRa5EYd7KoOgblGHu0kuNdx9k16RNC+4bz2Hv34ubXgXp4BRBiN3F42vjovveJi0gg0nYBqVskRWYZ20tu5le3rzCMiMBP/RbmCSKHYvpwCVjK6ZhEviy5jzmWEmaFhVOvspE9rAKdKpu7xIch6lYo/wbR2MwfVx6iR1FEc/RudLZ/46b/W9I//R3/0+l/7Zt9FRUVNDY2Mn36oMHP3d2djIwMzpw5w4oVKzhz5gweHh4ORx/A9OnTEUWRc+fOsXjxYs6cOcPEiRORywdvEcyaNYvXX3+djo4OPD09OXPmDA899JBT/bNmzWLbtm3/afsMBgMGw+AJo+5u++bWYrFgsdiNKYIgIIoiVqsVm23QSXHt+7Xf/VffRVFEEIT/8DuA1Wr9/+m7RCLBZrP9h9//2cb/7Pt/J082m43q6mqCg4Mdbc6IbUQZ1siT0VkAWKxWLBYLoijSrMslOqyCLH94eu42QsyTeRocPDX1+PLZk3djGV1J6spyjB5uTvWecgmjcNZ2Zu+bzfxdK5A9I8dWtRWMHQipm7DZbEw+MhkXvQt75u7hmjH5Whkltjian9zMkeqZRBWB1WrBYgEXm5U7puXwS/Ea+OhF9p0OIvzXW4noLgV1LFaJG03nXelSV/Dq46/iI4zA0r/CHmTLrMO7+SdmdxdS7D+d6PUFmHyGYwsLxho4H0Qp2qeXMuHQaGoSC+iJbQYrWMt/QCz/AkvwElwjR3M46HlGVkrxUA6n2GJxGqcW8+NkzumHPWNw6XRHcq+zLAV3VyCW1iMcj6IlJhNRosJiseDltpq/vmsjJcKVwgceQy31J2QgnyiKqDuUJNYn4hJfw4joRiqs/Q6ZsFqtjG5zQ14zmvNtMXTFrGbRdYP1Wi0WRKs9ROamo7uJiFUTabEgqMJAVHGg+m++WfM5M45M4ZHNj9C9uRWr1eokez3uBlp9dNTvzqAyo5y0ITxZDR1kdn1GSVYsHz0wmSs9d2KxfAwSN37x01F864+UuQYyPVBHpLTJIWNIXck5n0RkRSRPK0zE+egQbFaHjKmsbdzkDjk5gez5ZBXn55ez+BFn2ZaapZglZhr8a+nrzcdimeSYT2XKVZy96QS+EUc52gCfDciY1WbDZrWSr5hE6SkB31Zfvvh9PL4P+TB6CF8lwgle+0VEGnkX03yLUNlsWCwWeryXsP5cGZa2EYwL/oSnvUHZdgqbj31xsagETow/RVy7D7f6dlEZ1uukCxShauKnV3BMmU+3x2Wa9ZlYLINGNTfRhkqAnUEwTgm2gXl5TUdcSr9EcXwRn4aOxU3jAzYLNoUfNrkPNosFi9WGxWphSs0zTBp5FGPbxwg+dqOaRVSSE1OCvEuDVbRyxJLDU1d+BeCptkQOxA/Hz5DJfQlHqDf7O8nY+T43WqUQHVHBMBcDrcwZ1L/Gdm5ZcorfmpW89MWduCVVYLVaHCe21NYO5u2bTV1kBetu+Y6R4hrmDuHJZrXi0udC58L9PPPwDv6W3uik94xmI/XBxZwec5qlggsno8856f4AWwOL519gUwV0W6Bf1GGz2RAEgT55OX16PSOPjMSnMIiw369zjEeX2cgZoQxPwZXhSzREBBditT7naHeJNZLvkj8gyKDkq6xgytz8HfVKJBJ+DnCjc+GfBKRmkWHwo6E1HFjjaLuHzwQ8vC4R43Y/49yqcZernfTBCrfTZAruPHFyPB5t3lQvs+vAazolPa6SluUVqCLqeMUH6LkH8nv5UTaa3XVHsWLmo3U/8aZFx5Sc93jEZteTLyXmM0LVyo37hzFi6kXSgiqd6u354zGSLlrxeeRLXous5Yip2iFjNrmG6s23ktThyZ+T3iTU1cYKQcC6rAcbIjSc4sW1d/Je9Wh+UP5AhnqZU9mCIPCwmEvNl2uR1ASxfdUpLEgQBnSKujuPqG9WYi4PJOHt13lXA1gNWJBxpjmCOW9OI0GrY+/1EcwZM8lRtk3fyDPD6zls8ebhd56l29Of4DVhDrpeGcqpy7HYDAoUqc3YelVYLC4OGVNZ24kdVYDuzDCCjluoDMzHap3tkLFxiiy+8IXHHnkYszEZ71v9sMzOcvAktFeiWnwAS2kgNapiLrd5McY6ybHm9qLBcmw0uoBmxPRsXKwL7GNp1rH855WUFYVT8cLHfBUMVzqOYbHYnYURLqGs3BJFh2c4Hy8+h0bqY9eR+zMR5F5cH6dn7tsFLKuQ86pKxVkqwP9ebF6jscXdh+rrbwlsCMR/wXFmDC/FKn3ZaTwSFHo0Nhcer9BS4N/zL8wUHljByLe+4t0mOT90L2DFwDg5fgcoNb20RLqh8R78frvHRY5nyinSBbDxjY2cmpXrVO/+shhGnYqlfqqckclmzH6JgzJmszFqQR4BE/u409TDmzXwB2ATldBbSY80iKOZS2j03MeVpG8JtUzCIigQuwpBV4aKZMrevZl5efZbALbXarBW/IhY9A5052Pxm0pwrT0s71e3fYWXxX7q2NE+Qcq71/9MZEUk6YUfMtM9fYAmY+bVXMw5hchUIv6jdmF1H9RzckMDn7zwPa+1g9/BCVzn141tQK/bbDaCjXnkhMPdUjMpK/sxuWqd9Fi0tAWlK+ztBfOAXh06HocKR1H52hRcqafV14zZbEYiER3jYbPZiC+Kx7fVlw7PDoQ4Z3xxuv8tdh/dzP40P141tnLUfTqVlZVUiclU3v8sv7WFMUMFoxX2ddsi0SCKIqJFILY4lqAWfyYdmE7IquMOnqxWK/t9CvGcV4SfqQdttparU7qd6t1a9CzlDWZcq+uJ8er7l4ytOHIEWXMdgeOr8HUzYbWmOcnYSr+3KHPdDj4uZPs9xESbCNfmtSKYhU1dQBcTFbm0WV2wWCIQyn/kK++vmJUMR3QeFEfnEqFfNIgvAJceJRGdEZwad4oTE07wlDDewZPNZuWpb6bhfX4kWalZVIXXYbEMHvQokGRyIf07jD7tvD+9mgLFJCfdXOOynN2LttFsy6ZL1cvNPOUsYz3VvLn8LN+OKuDBI1nckygh/B/YvOyw3RkYkVpKe4/e0XZlbxtu7VVYc4L4IPspZr+R7pAhvSyBJlUyXTIphi0ikpRS572GxcrdH98NohstwQl4qQOdxupwYRLnr8bzwLirbEhq5m9zsxPeS0i6zKhPs6jv1mOVW5x5wj4stQnTuBR3N6/ouzimSnXQRVGkzruSI3N+ISU3haNWA6tuD8Jms2HreZZ7Rh9i3onLjOoMIVJWhc2SDKIMq9WKVvIQK276jCd3/MmYjDxMATc41Tum8yGUJwP5y/cSz3u38mOsG0kDdLPVxvlReWC1Mv9MNf5BKjtPNiNWixmlsZN1QSc4ZAjjidbL6IWQf+2T+t0fQnj/Mh91wXZrNBeG8CQxVvDLmgS+KspkkcWX5WF52MwGkMi56vEceamNBFUWEpHdzO68Hm6yvWEfD4uBG0YmcKLTm77XptLl3oXNNrhvNUo8eLIVrtT48/RH92EeGeSELyKlJtZc6ifXawEHp3yMQj4QGnpAp4SLTTz10y24B7aw4pE/KDMsw1JVitCwF5v2BQQXf+6enEd1TDNLIo5wtEvEYlnp2OfabDYeevshpBYpjTUz8fEah81mo6W/kwOJn6Ew+fPUx/fi4dpCimRw7aiVJvPkg/fi0+1OsPpxEq/PwWLZgaB9CSx9qFvrKbxuPyF7pjHixATqDf1YJGokvuOwWq3MdO2lC3j1Pvup9Jn9dxKCKzYkYLXhOqqAyvhcvD1nonSb/i98Ed59jJcPpBI24giWuFqs1m+x2Qb37o2nHmLkEXdcwxu5I7HWMT8E4JO7P8FdF8zGmiAeG1btVPYNIy6THJrNV0eXkX0ohbGzJE7j8Zx/Kwbgjq5HqXJVUNv1Ex4eoQ4Ma5QbqQqroS11EdExM+0yELMePIdT1WfiYtoe4oviufeDezE8acHSU4WkrxaroEAEfOq98WnzYf7df6EMcnNgWLvut/JxWzTHvptHV/fnWGbfhcXigrL9Am9MKGNSWD1jtt+My7lp+M8uHOTLosf1fBpeOdHsPSDnqM8Ibp1tx4kSiQQ3cyMv3L+dR2pC2LDnLH6hngwfImO4uVOdNJP6iNNUtkBj4WYmJFuxWGxYrXBZrWLbMy8CoOr/kfGlh7BYgh16stUUx5n4bYjfzabVqiRgwIl/rX1L3acTJQsk/a2bCFx8EO+MAiyWUZSIwyjuKcKKlQrfD7CIelp6b0E58L7jrv4Awl9cRnpQB49eX0KoSxJrhuwptndO5w7vvxg2bQTakruYoAx1WotHhmyjYqRATkoKfS5ViOZeLEY9SOREa4LZuXUxow5pWff6Zno0eifdbxVdsLKKJomJ7OkfE6Cb48RTQcMVzn2Yil44T23cg6itg2tdnsGHc/0Q69bHjpQmdnblOcmYIrGefRv3cZNkBmmyWjRqrdOctwa1URZaSkCvF4HSAruMlX8LiKCD7z3rOOfvzaibd6KKUDmtxV7mZrq9u0jJSyElL4Xgnwodut9isWC1QYd+0H43zzfLaX54GOtwKYugW9XHjwv/YpX1AywWCzJDJ9M0Zq5KPJF6NJFosyI3FzvwHtj48wYXjtfMY2rSXmyCjSeFbxw8tXfnMm/CrRxpFIja9wR+o2Kx9NYjseixalJY6vMHY5ZfYt3nM3n6uyiSvorE0lOL0HYG0Wc0a8IPMP+Fqyz/ZT0e+uudeALwNRbyzMtf8tQ309l0YB4h2qtYLM5PBwlWgS1tUn7usGJ0+waseiIlYbw29gxhZhVmoMcGVgSYdhhbfytpf/3MhrpE3mr0o6bZney2OO61WLAZO0BU0BM0nNib6tnnf5GPorL4xbAcS9cwxOwnkIY/Qsj8IvIX7sf250w6grvxNg7a+XxVVSSEtFHg2cb6FghoT+AJ7xxsjQewxtyDzWbhdpc7Ob95BPPJoDY4GCYMymCoSw2rv/maBy5PZOzWDHK0l+m/TYXNZuNX9yzyNnxPmjCFtUdmsHRyIbLOU9gAq0RDb2ASwrggqsYcZm/aFv7uzMSivw96q3HT11E56RTFzX1cGLkDm2DmW/Ojg/Pj6GyeUIDJRUFp9WXuzniDAuUSYANWq4VJw2/H5H6UX75bg/KQwHPSw3y95Dhi7XasymB+6Jbz2NsPAlDhkcbaB+V8uvFJsFgQJS68X9HNhQvQnLGaA7IjZBgeZJXNhiBAUXwRXe5dBOdMYvylWxie+vsgNrLZSMxPxL/JH/djk7k48qKTbEe7HObRKTfzzN+PkHc2BUtYr9Ocj3a/ysu3HeP5iBY8P11N/Lq/nOb80a5QJmxZhVbTzc7rdmIVh9hEbfCXh4a6uQdwPTOXsPZNDpumqAojrS2cis4K5o5cTIp7NRNtwQN6SkKlUU/JnBfYX1+PoduNnZff4tRbGwbwhYWpTb4UHs+gOLQJuV8jmZJ1WAv6sMXZT67FRTRSPvkETSOyqAhuZaNN5pgfgiqdbYcmYDWYOfvgTkKNM514SpPkYcrwpjGwjith8Gf/aac+E0WRsJYk4kr82RtqIqjraW4fgr+79F14Hy4gWTeNd9/4nE86vbHZbOiNei6p92BKTCWu2AfbueuIWZvv6BMJUlT9MbhJPKlb6Y2HFDyl7o66x/ot5mLQIhSSPiZEhNNiisJqfdSBYevKgzm4+WESJ1bhOg/cBZ1Tu9v9uikOyGPMZ3ey0b+VW36TOWHYJKGA9k0f8nMHYAVsNidbmMzcQIS6EZ8t6WjHyXEz5WKxzMfV1o1xeD5lJm/SmvRkBOgQbQ0DMmjjesMBNme2M6Wnl61usNfYxvaaq2zkBzABkZVc9C9FpminMOQNLnQ9gF+Djq/94VaNlR3X7+f54nEDjQKpuQdbbzUC0IiG2DPvIDeY8StaSZG20omn/xP9Gv8ZT1arlZqaGid/x3/F039n+l/r7GtstIfX8Pd3vtLv7+/voDU2NuLn5/xYtVQqxcvLy+k3kZGR/yrjGs3T05PGxsb/r/X8R+nVV1/l+eef/9f3vLw8x3uBXl5ehIWFUVtbS3v74CmAgIAAAgICqKysRKcbPCkXGhqKt7c3JSUl9PcPevWjoqLQaDTk5+c7CVF8fDxyuZycnBynNmi1WoxGI0VFgzcnJBIJWq0WnU5HefngA8hKpZKEhAQ6OjqoqRk8/axWq4mOjqa5udmpH/47eXJ1daW8vJyOjg7Hie60CgEEGxFSCJJCWXEegq4DrVZLhy2YVdO2ca0HzCb7DYtrPOX3yqmcchiz1MzJ3ZMYv8CE65C+0dHH2TFnmeHhwyy5G63llZSrb8ffcAK37iJ09fncWBtDXVkII1vHEPmo3Wh3jacLrd780A3Duwq5JTCVmqogcnJeQ98Hj3/WiUqlwjV+BQtde4hu3k9Z7ih0bv1Ud7vgU7EDqXs7ZYkGSvW55BaWox32Oia3JLrfeYXW47O4NOcZfhRdeKAnjFE9Ho5xWjm8iC63bkK3TycMFXVldfSKs5CFjUCoa+Up73buKhxN9sfjOTE9n5ycHKdxump1pXjmdoydEtz9W5FY+qis7HaM0/DmTfTl2A2Pvf7e5GTrQJBytnkO+enQGXqG/Z3g0xNFwkB/xsfHM7FWhvfvN7Dlhi1EGiuZ5x3MXVarQ/Y2tKk5t3MyANu7Ghmr66arzH7FvKPqBF9v+okXrwbR8qOCXzu6GZPzKNqq3zBoRuMhn8O4xFr6DZcYFdNKk6Cj69wmPCteRBf6ABXqNTym6ePc+FzeSixALU/ktiGyV9vSQu3YKvp7PXn+xk0E3KJBGG1v++dWBZ3h+WirJ/OFmEehrZycnBzi4+Ox2azsWLADqUnGQ1/eiSnCvij29/dTVFTE5VZPPtgyBtEgp3f2HiyhQU7zL02dTdwrB3m5HZQCpLf7k5OT45hPaeJyyqqXEnZ6Cn0qJW0L2vDWqB3zqaGhnm/XfEt8UTyu1hsY2zPKqfy+rjCiLh0iJwaWDjvHmvYn0eTk4OY2hf4/AknTteLfa6M+qB2v0Hx8wvuRy+UodDWMV8JF71YWeh0hgetYO8ATwKdiMFn+ChK91zNDAleL7yLEfa5DR7z+/dt4lLURGHyV2th6Ribn4unu6dARb8V1csnYwQu7nmP8sEDe8nGnJv087R0dkJNDQc5DpJ0eRl+nB6ddQtEV5uHtbl9+9LIW2saf5/vM8/TaQLS60OJ5A56SZjjTz/rvb+B8ZhXj5Va8bb7M7O936D2PRhVR9aM4HlfM3yMu8qL1Vkd/tfV08oRnKQ0auH/vJNyAnJwcBMHOU6C5jDkb/2ST3q6X9Ho9lZWVDp7SDCdJ+eZOWuT9bNr4GQF+UU56r7H9FNsD4Y8RBdSUR/FXVRgZA3WHhoayR7aMiYWHUfWq2Ci6oZdWo9Pp0Gg0JNffj0tBLDVJc+kOD2esKh6r1UpOTg6N9bUk14Rhtgo8FFWCDIGfsrOQXAOrnfYTW26VYdT+soor11WTk5Pj0OUFeg0FNrgtroypPmX82m53Nl/T5T/mbqC0VMkLM29lUuRfFNdOpE8/eOr6aE80Zx5ehNSzg457viPVkEpOTo5Dl6+LKcAcbuazV57hQst2Rk2/CFcf40T3Etg2A3+pntowKYJVDjaR3Fx7n3eYRJosUBfewbnMY/QZRxE20F8SiQS1YjktQfkU5iaysMqbqYs1BJaUkJCQQHdXF+VR5bj2unI2Yj4l0hAQPqO5XUdjYyP9LTl4ia00q/dx3B/aLOed5k1AQAAjW12pr7E7W3SqPieerpqS2Rb2I2a3Flr0oBJ8WSJIyM/Lp9es5+qwq1g1SbwVcQlXVx9H2aJFhy4qHJlqIi66ZsICrlJRlEJ3hxyJRIJBEcRL6z4mpDaE27/WoFS2UVKy3rHmRve28lQQPNbjhaIqnA6TwWnNvVgYQsWXj+Haq6IrXEJhQT7NzVYHT4JCyfQ6GH3r97zpC8VNPnR0pDrW3NamJqYcmYIktIGXZ+dTRDf5+fnU1fZRoA5BH+GC+dRwvvbsxGOhAckAXx3lBUT5ZFPf48cjmx/h9ORccnJySDb0IpO6kt97PedqTxByTMXp4+kIz5aCh5b2Ubupqalh4/xT9F13iM27TrDr8nje6WtyGo+Hrb24XRnH4b/HY5l1wUG7hiMsxm7ifXuQAtWyXXR0dDjhiI258di6m9hy9/fE+75Jfv4BLBYLBT2vkxPpRovBRJT798SFtjjV25ap4P24HQRVPklx03RWtrXgl5/vwEYP7N5Hp85CgNiETq1DXCfQ75KIS8txLub8zZ5x9gBU693B2N9MTk4Orl6PAgIHqxvwrm5Hpexn0owLXLG5Ut0aYr+FFXkLlZWVJI7J5kpSAVJNOxqbHsDBU1tPHz3qHhrTcrht9DvoeiaSkzMFwWpE3lCLb2kOVRNW8ovwDOhaEAf4sik1vNgGJ/ohfvIJumRgLC4mMDMDnU5HcZuU5z5aSp+HlXtu+4ucHo0T3ptlu0JCXSIuxR7Eu+uR+VwlJ8d+E8TLy4tPzdWEJtmIysxhdHoh2bmjCQ8NduA914ZDRD31GZ2PPsHskiT8qXbCe/XlapIP76a/YzQHZ+ymoqICs1LDpRYfNreDm0VgzYuPEBNdT9Go89hc/NFqtfgZ27nx5xupC62mf9IZzqntAS2v4b1tKhu1o05yw5Yb+L0oHnFcqNNY79x3kBFXhvHpk++BS78Du3h5eREW6EGcbjvGUj0PPbiZ04KG5uaLThjWXXMJ1xtknKhdSaJSSUlpkYOntv42tNlaFuxYgFL6M7Pit5KT8weqPiVFhuFUma4ix0yiHIReHTk5OQ5cfkuTHNmWNbSv+Z23p+dywJqHTjeK8vJyrFYbh8LLCJb2c3TKEWwSqRNPXZ2d7Jq/C4DEHtB2GYkcgvdixenYqq+jWXsjbb676Ouzy9g1nnr1jVz0aCDbxe68a2gwkZMziONVXtdzYFglV6+7ncagPNZ1vI5PTg7x4R6M3bCe/e1euH0RTHuDjEuN8UQN4L0DeQf5et5erBI9LmOysMgU3D9kr+Guy0OFgqKoHH6Z+SZjbHcwIecewL7XKK0NJf1oEN1Hx/K8RofnJx3U1tY6MOxbVz7nYlUl1WMmIe/xom4ITwAt5aUMn1bIcXkBSr2S8vJyNAb7TiEqKooUWS3xghseu+ZzbOIx8nLzSExI5IBxFFs+UiGV9BP39mZWeXVhMd+P0WqhqKiI7yuPkaj356i5lpzaX5mRqHcaj5rAc0QsURNZ5k1IzTxqYmoceC8gOJCQ5LW0d7oy4ocG1CNs1NasJuxUNCJQFPkzSZ2/kHd4Ej8lxhOZGvyv/dMUeS2pbvBcOxhNEkfdUVFR6LGyp8dC0WXw3HM35zb+zDSDEblSirp5OO6VLnQGvkLgvT8iCQt1YFjBakDb009ZfS9nV//OYh8j3eUjyZHYjQt9ne60bX6P1NZeQoKqCQ2+TE5OmgPDil1nMBX0EB8k8NEyV36zngdwYKNwSzHy/ihM/UrqzNDa2o61/D5klnZq+7xwGfYY2+MK0F+K4o+3F+B3exoeOTmOfe70vpP0TzHxpz6ZGslYjEYDVquMmopC4mTQYTYjmeXN2JgDmK2rHH1ibVNTHeSBh9pIjKqJKmXlAG0eSqWS5aqVLPWBexYd5pm1R9jcNoea4vNEVN1Hj/dCHjPbqC0dzknPUnrUPQgCTnjv/oRKtC79XLf7GxJUyaQPkYPQ0FA6zZ60lXriG9FAgscF8vJySUiIdOzdR7pepCUolaPTy+lwm82wAbw32bSX1SmdLKyUwMg8DBKNk4wdkigoiM7lT2M28sRw6i3B+A7Be6/+NYmwXZnMtUoISSukuSYLwkNpbm6mtegUT92/jS9bPPnJ73cajKOorf0KabeI0HWVpiYfdl63E5VvB8NzUtHp8ylsSEYT8ARusiQ8AfeIWi7cuBWFdTx+demEDmDY/Px8+gx6tm8fh6LFC9/oIkzGe8nJeZC23nieroECA8gzdPiN+ZTq/kVkDvBV39XN/eP2oxipZNae5+iXKcjJyyYowIWEhATqzMGkVYOh2sINP+7l8ngd8JYDG+2t+ZGihftQnvKl+cQKCoYFEl1fR05OG+3tUmxWM1LgPr0f9/i0san6HDk5sQ68p2u+lf1n7sYzrY1zBinTdHY70jUbS21PI1nbJgMwLqqZqy1l5OTkUNr7OPs/v4mlahFJ3iHkwQ1kX3ajM2DsgANJ4EraFcxiLGlnT5KU1EfJAIbt6Oigs7sXgwr6YmsYFn4aN4ua5mY/B97L0OwgJCSIY1hJU8D0Kwupr7ufZp9bCQgYxaH+MCyBJ5H3uyBX6p0wbElxEc07ypl5qpaXNvrQFFzgxNM7HcG41rti9bpC41gFitaXycmx7/HUWVKe2foM5+cfQx/QTn+/XTav2Y2eDiggVdHN2pczKWmYxfivRdLAgSP2hVfRB/xxewbbwkfz8spmAs7b4yG3Kp6m7YObcLUIBL78EZfaRzrhvUBTFWuf/44PdqbjVh1Ca2ewE97T9el4J3Y04V0XSL84ioOdKYQOsYWdb/fhq8cfp9EMPqXPYml1se/zrP1cjX+EYrcw8mOXoPfLx6vtJfIH8F53dzc/f3U39cU+WG/bjU20679rPFVWnMX1hxSSS0Ow9SvwDJCQ06BCEn8ca7+CNuMOikQQTVJkJhkNDQ10HX0Pr+49EH0HRXpP+s3wzJKP8Zd8TEdHqxOGLeiK4Z3qCGKrgvEwqPDrueI071eLWaw+No+aY+mcGH+C4vR0EnpfJ816lo2esKO8n7v+vJ1WbT7lU3IInzgFXXc3pZVXaHlvOB2TzvLNw28zzHor3bXn8Thlj4jzXfNBqnpTsHrsYpOph962IGYUHCOi+RhdLot4K6CHyu5UPlu5h3hUXP6rm7fLcmitbKGgIYqwcimeHRpSa0PQuYso/PqwtGXRIMmiunoEUmkCodPM7DFVEBKehabDg5wce+jISlsg1zdAY4sXE6uCaQiupLS4GK2fFptEoNu9G3O1gcBj4zgY0MvyJS9g1ekoLetktaUJ/fC/WGHbxJaCAPqUabQe34h/29fMBmZOBZ+rgVw7pF9bU0tOTrsdG0lcsJn66dkyn+grUKtW0jvafmu+pqaK6QGHqdwXy9ELXTTfdoDPx76CrmgBHvo6Ggv2YsXKqbGnSMuaSqtnEpba4eTmF2MT5cTHx0NTFxFnD9A0xkKRtoi49k67Ad/YxeTxuWQbcyk/k8nE+H1M0ZQ5MGxnUx0Lc9KQF0dzYfpkJB6TnWSgqCWWzX++Dhf1fHDvB7h5xbJkCN5r6Vbxu8lESWAdurGn2KsQOYNAZWUFOp0OZXsyoTXduPTLKIovorysHKtp0LHwjTGcK6P3sDpyBLMVn1CeP5+ODg3x8fEICAQ0BOCqq2fD8vX8cvV+rNZ0TKhZXTuJy5Y/CddV4tNVy+NhOvpzRYosdp25M+s5rMcu0zy9jQ1lIlsSRMQrG7hqmIxarebTzGyODLMxvx7ohsUlFShk3vabT70eXI58gFbfv3jYA9p67U/ZXMNGh3rCqF72HW5bF/P8Lc/i8aKbU59FRUVxS9IISq02rM0LkbdYHfY9uVxOUWEBolsffVYpc6tFXKURrOnvpyz7KOMyCjmRWkh3wyqEyGO0m2cxqyCfxkYLlnoLD+1+EEmNgYz1OcxJ+pYjZUvp1ylRq9WoLTrWjfiY3wrjefLZxZSk1PHNnEEM+61Oh2dMJ66yKj5vjuOkd4ET3vvStxcbvmyTGWjHRElZBcbuNgdPGQodaV5woiAEuWhF71ZK/xBbmLVhE8vMHexKSacmbgRj1fPJyckhs/8XnomAUcU9fDD/BD9KYUPDk+h03bS2Ginsd6e4Rk1qfwZvfbOA8FUuNLo5+z4sOiWKFj3pzelkTijgqd79MBCVdYEbWKwmMs4m4N4Vj23URYTOQwCcKz2NdmoZ5yvCiMmNR+qtdcKw/yf6Nf4zniQSCWVlZU7+jv+Kp//O9L/W2fe/PT3xxBNOtwG7u7sJDQ0lOTkZjcY+C64NaEhICMHBg2FSrn2PiIhwKvPa99hY5zCL17y/SUlJ/+F3rVb7r+9KpfJf38Eu7P/Rd09PTzw8PP713c/PD1/fwXdv/jt5slqteHp6kpycPACUYczGMWSezcArpJnpI8pwX+hJXKwWURRRKF0wAR4dHrxekUpnTJ8TT74lR/l+0nGmHZzGhJMKGiYXoNUOhuZIuvgdwRIYP/4KkzRtdCd8hHdvNuKFLdCzGm//ULo2fEPbe6sJ1+iJ0XgCkxw8+eZeJO5CHHO2LSavX4l23HG0Wi16PSTUHcGtq568+y9wc3QNOnkYESOvB7kHFbW/8dBTX/BJu4S9BYH0uLpy77B0BHE0cuA3/5+ovfcTag0SVHqR5r4q1F0mhucvwjrqc6xJ1fQklrHXYwEGeRIpKcmoVMkOviqLa6nfGUdbcgGeUW1otVqncepyjeHzZnht1W42ekGduY3QiMEQBO9HKDEszSU8zsxzKZswhWcjuifR2Ah9/iKi+RR3fHEHeZkdDtkRRZGlwztw9/uTwiPjWbF1BQ1vKZ1kLydkES5LRL5weZ8Ozw40ajVhA071PqGOPeegtl9EY5AjE0V72SndKICIfbfyo4+EFSlF/CbPJ8h9HR4DYbw01KPVapF/Z6Dvy0e4Mv9PumJtzrJXXc11PqeJGt3IWMtcLqut3DHQduE3u6yNujSaD78OYdzHGWRq7TIm7dLzc1wPe/vgnTs+xyw1UyR528HT6ZYaRp4aS4dnB7YNX2M2ZzjNp8ILOla4Qr4BXvKB72h1oqdrfiRp7GlOvRZPp2804QPXuq/Np+q6rQRG9PKxREJQ3mOMCLYwdLqWns2jpSGfZOlsVJpPuf1JK1ot1NWJeDbko+puoii2iSuqLj4a+bLjbdN5kirGfHwH7gk57Bp1DkFw1hF5ux6iIvYIN/62EktxNBFbZU7tDmqrRdN4HrFuOAWV13PHd4Ohrvz8/LhVA9LjI/A99DsSj4XQJRB2KpWQlE3Ykp9mkXczJep+Pp3+G4UJhZRqNxPqHgpATfYeSiNgaQNMdYHhcgNeM39CrN7Keq832KUZhU+bhidffZKzswodPGm1Wr6zGTiwewHtnh10eXQhlw+2u6m7gwt2zMFjd32E3ORDr1brCNtxvCKGRz2qqO7y4+bzY9Eniw4Z8vPzQ+mqwTczm/w6dw5eCePmWVFOes+tooys5xYRdXI4AF8EZQ3KmCAgCCLrP1qPX4vdkJC5cQfqgdA78336ueLRiz5Nyea7p1JvfhRRfBmtVktp3XE8/1xGl6aHF279HIDUVK0jzMvk8x+S+dw9VGn6yRnpRUzUaKex8qvTEHPxOnaNPs/X3U28NizTwZOvry8BPWV4Ze/gyInhHBsxi0dPhCKXD4axe+ZoAv2ZZ2j1aWX7sHpMEoGIITrlwfokOkx5zK2x0Fg1EMPIbxKxslhcTrtQFu/C7XU/UlnTTImrC1ptBIIAj22cQvP+RPxvvMTLSsh0jeXNIe1ev3QheoOSrz6JxrcljcAHEx3rhpfKwrM3HeX3ZiXpPyyjP9EezsT//HgCesq4NC6XVe+HMeXIFAqHleA9t9ypTwRBYKpCiu3mH7gvqYnckF6kCREIcvtanZw0iQuxp5D05FF7NI0zqV4sEWUkJSXxemwfIXGXeenwUoKad9Dv7oNWOxhS6sY3x9HbK3DvLWuZFfMVdXHPERDqAUBr2SVsoo1u9y4mLj5KQ3QzsbH2kImenp6ER97OC4uVuEVU8uy3r/CxfrXTmtvuFkWvTyETF1Vz69xNNCYZ8Kt/HKFqC7a4y9QaIP1COsn9bsSvPkqPqwuenvYQdbGxsdxZeRb1Uz/wV+lDvHLid9ZOnIhXoBeenvChx2hcAm0EH+zgorKXxQ/PQJtk77PLOgVtK07RVBlCZs0xQhMU9v6MLwSpHFkW1Pwxh3aXXyhNv4Cv+ziwmvF0FfFIiqPzgg0JoFAqQA++vr5ERg6+RfTVU1p8do/gfGYHPtHrHWN1TcaqpSMZdXIfCxVS7vfvdeIJ7GFgMg7O4peHPidG5sO6AWyUn59KUbdAv/8uHl+8k/N97k5y8OTZhbTo4nnDN4/s2M1kuN3F2iR7uGa1Wk2H+glk/b2MPzKR2rBqBAQUcTdj9RtORE84t+a7UFHnzYS9GSgzStHGhyDuvQ5b+EpCPUfw4bod6NzO844AS9Wv8dD4R7AKjyGKIhEWC+/FtLOrMJDXjSE0hlxx4qm1o41T59QU94uMD93GWasrWu390H6ZV267kR+Tn+FMTw89shzwi3HwlV1Vy+dZISgMCg7E2OdFY3yCgyd54HD6LouofXrY+ML3BEyM44E7BvHeD7+NJPvdKJIBmcJI/HUdTuOh9zLx2/V/sS8IMpQgS9GiVNi3DhEREXTktZEusfDI0++zyc/CMbyIjR0M2bPVdzMufXrGnxqNb2wrKUv9kJhUBPcHEXgpEM++SJp9vWmWuLN0+ESkcjmiKDLT/TJhd/awztrG+ZAa4oW5Dp60Wi3Kv5VghZPjT3J1+FVuVb3hNNYV0ZUYFH00Y8FkFpxlrLeCe9e9yOc/3cjb9z5Mw9pzzLjLz8ETQFDevUwbV8CzNes4aw3j7djBzWRr6xVubIyg3yzD6tdEnk3BzVotoGXTkRxWPL2QhvgyJsihOd2OV65hozkpFXTftIu9hhs4WZxAyIxJTrg8Pzmf/OR8TodAqNRM4BCejlVeYkozlHQp0Xe74ubr6oT3gnwVxPvXM1LdyoOBcg641Djx1G0IZ9buUO77/EakYc8gugSi1W5ylN/QICKV+mN2DQTyCAgItLe9t4JVvjVUS2r4Y3Yr/aZ+HvJpc/Dk5icgBv9Ov1lOm2BEbvZ24ikrq49b3vmC9zvhZw94p7PKaawak3p4e8PbZJ7NRNmv5Maw5YSE2NekkJAQjrhOwUXRiXtRNG1+BieeAKoKXubBUb/TcdsTCJE1aFfJ0SYPYtURnfXY9s0gZ9IZVk27SmqMDxKlkhYhEE2emUBPM890d/FBj8BpmQqlYMej+W+k4v5LEqbrDnIq/hc8Lbdwj3YwMPDiP72pTD3E+itr8G7xJS7YzcGX0dTP01fHcKYwhPApBfjFdhMS/JQjr0/yBF5Zez2aPAkrs1ZijRv/r/3TsoOjEXb18tDVYWQtvuiEv81SK599O4uM8xlcSL/AFlMg+Sr7QdPprj+i1bexC5E7x5TyR5/MCe8tvHsMo86PYOR6MzMj36M9bgSBKXaaVx3stpRjUvXAAhWtmpHMGjJWneqx1K7S0VdSweb1j8LjVY5xCg4O5uMTo8i95yUMWQl8tSuVzZuSEZO/wlbyAYHaZQienrzSLuO2Wj9UgMYvyjE/AJJC5tI773umNqgYHzIVpfkzRHEkaf69PPrQHVS3q8mZtY4zV+/io7WgDbYbLyX0kp0WRJVYz+ex03E1h9Gl1SLkvwZWJd81xJK9JYl6bQ4vjcuh3yWSMGUFtF9C3VPJp+fSCN4+ng2L9zI+rQzRaiAkJMaB9+bu9kal70Z79CeaU5RotW86+kQQBGo9wvnh3kdYtG0RlSZ3DtyWglI5uHd/K7UD47BPyT/9AxrrKkTRrpPasgxEdmjQ9LmwvgX8LNHUD+nvw9tDKTHkIvU+itm/ixT3x9BqB8M/XnGx0B9bQtSwEmK11fgE2p8j8fPzIzI8FPcmUNkk7M2+GblvGPNDQhByjoLVxBPa11nRcB0nJpex8eZP+NJ4IwmJSQi5v4FoDz34bkItzYZgPjs2idTrV6BWD/JksVp5KfgWrCNf56s5+9lvnIFWq6W6y52WL6KZlpVGTaaKg8GNrPGKcMifuqWJVh+7IfPrtfa14oXYFiIC7Ou8XOND8B/zsEgsFCbn0RjU6+DJ19eX+dlH+MA3l1t3LCe8LpSoBecJCb0BrTaIlhb4JvYwfo3R/PnmzeyMr0A2zttJT/r5+ZJ5+VOMnn6UpC5lYEvhsLHcu2sK5tl7aPVpZZO6jG+DFwwY4KCkoQBNbjm9Cc1MCmsnys9E5ABf0y4+jftte3njSAgLj35AQKgnsbF3AQN4zxMeen8dFu9O7nviMc719ODn95wD72l2rORKfTr3f7OCPm89HT+V4B87A/9Ae9t7ogPoooqzz61n3+ITnLhnkKcYPxMLQp9jZ+8CCt+7Cf+bjzvx5Nvsy+b1m7nxyHReOT6LxplZaLX2MHNrIguxDDeQ1ejDjruf4uLiBp54ZFAG/2gL5y8daPrBxdRLaGiYgycPDw9ee28Exr0jqQqv5OrwXPz8nsOm8EMwNOMaPY33U55AsAk8WwPpkmE8OWQtblOPYtaVvTT7JhHc/S2Pp1rw9LTL2DVsNI6RdBu+o19Zhdx6vZMt7Iq+gtJCSDelcruPD1GRQWi1dufSux+ItOqh0OTK2U54Y3iKI6/KzYXtnbUktWrY1DIGpdLmxFNlN+wynkQVWs8jqw5TorodbeqNiMfng0TF+zUTyPooFf2UE9Rdv5s5gX/iET0HLu8B/6n4SF9mz9vHuDj9cfySynnwHxjW1VXL4c9uoC7WQF+gjgkxHmi1g5cX8vdCyTF71LIZ4RI0SXdilWixXPiWtB9uoKvXi6BOb5bLKkgLt+8r1Wo1wxKf5tDwMtr8BZ7y3EuPoQqNt12H2bwzuez/IjWBx1B3qPmu0YdpfimEKY8C4J64klffiKBH18hdK+5DJTfwkFoDrknkNIsIMhsZ50Yze9/Abas1p6k3ZiCGzCdYqSc6OoL1V0VcEkrplv7Fb1EGzjI47w2WMC68NAVvj2BaXk3nYFAvFT7LEUWR+zzO8JnBizuD9Lz7wLsIQhALQ9bgqbaRqk2ldsdecAV10bMcuXAdQcvH4iezPzBR7TmZH8qOYrNZWFWQQkZQJ6Gp4uCcv6GPcV+MY9YVLb2qXu6Kvkxa4wbuA8LDw1n4yToiyrIJ7FXw6pQ8FCLI1CqsIa8RaNLRmejHy7X3c9lrHp6KBn6aEgyFYLnBjCgIhIt/Y+toIEiv4B6NBH9lD6IoolGY+cAPfuqI470oF9ZpLrBTcj1/DGCjmjyBMbft51D1GkZ7/YqPphWtdvB5lR931HFV3kBYgP3taaWL0kmP9ZDInaWuuPVIaJtyhGCvCBAEB47YkvYExgAD+96axBfmABICrEg8Bue19LAM0SKS6VvOLbG7qI9cjX9sIqLVwCxlH7Ld8/GobeBZ/+00WeNYIRGRSJSoXdXc+eaduHd749o7njDNX7jkbEC7sB6UfpyZlsi+vin4NzxP4isb+O2tUyxb+Shaf3vda38PpK5bx4o/5pGXnEfimkRCQuy3dwMDfPHw78dmFoh9Zx3DZ9idstd46s+OY3sFzHQJwdcrgdiAxWi1g7GrRVFkfvQmjueNR15QR+wEd7Ta+x34Ijk5me8e/Juv/xhP8BPPcva+epRKJcmB8KU/3N4EhWEyTvvkEVe3jOTkJLy9wcWQi2fM75xrnMzp3MUU1aaz/MVhBATZ292Vux393i6iLF109bghmpVOGLYlpIQ9N33N6j+W8svmiVg+t3DLELzX9VU6V/4K49j0P6mMrOTNhC+JCfJ28NTr+QYPPLeKzNOnCYhowPvFBidb2HtZIXTvSkTna+Jgy5NMdoeUFCs/5Wfy4qcqxlTEcfGOK/SaISQ8DrVag7e3DZ/IJ7l400XCR6hRuCmITRyH2eNT8sPhqVYoO53K2PNGev00BF6Zz7jR2wG4ZFbRLg+DXRoWVEYiNLoTXOuF0j8ZOu08+ft4krjkGLsqk6gM/Y60NBtarc3BE/yf59f4z3iyWCx4eXk5+Tv+K57+O9P/WmdfQIDdiNjU1ERg4GA4uaamJoYPH+74TXOzcyxss9lMe3u7I39AQABNTU1Ov7n2/3/1m2v0/ygpFAoUCsW/vkskEsdAXkvXBOA/+u3/9HdBEP7D7/9ZG/+ffv9/2sZr7blGj1MbwF2HJT+WnRaRG+XeDpoUA9dfTcelJIqGvCRCVxxzKiOKCg4Ew84xuVhjGvF3v86p3uiuOvo/WMcfcX18oPie7+aD6JIBk3aB30QwVXFDq4Wgldu4v+VTNJI0p7aPEs/ivXMh/UYZW5f/SrXtJpZIJNhsvagVe5HrVWQZoNgIcVSDKIBEgkWi4YRXExWNwYhv3cm56cU88LQELj0EUlfaFCrO+zRx3/65eJ8ejXpYBUKX/XSBWPk971QcoKTcgEdPJjlWDbdIYWh3Hi+7E6HmAA8+tZOSkF4nnkVRRNotsvLnlZxIzadgmJFXpwU5/eZlVT5t2iLmuYKxE+5WeiGRSOjr3MjSKX/x2QUtGb0RpKoqnPKVuXxIib6d3uhv6NBcwkM5zKEEJRIJZ/QPUGHuZerI+5ngAqLViERiV2BWVShPHUhGsAlceP99Tvf5DZRtL79amcoL68LQhNbwzU0/ssYzGCH1YYi7C2FAJrY1fUiX7Az3oMbDq8apbVKpXbWVR5dTHv0hWukSO93Uzc+tUs59/TS1cVYaYsZj9Ml05PVxcWe8yYOeupl87PXrQB8KDhkb7T2CrDmtVKguc9gfduqc6zXLD3Dr69mI0m952q8E99UhTnQv4RwHXovHI1LH2geeQiK5yTFOAMHmEiaa3dnXK+GDOVr0tiVIJIM3iDXSm3n2hxzWbH6McQ3VaEyZSCQZqBveZcNT7/Laqe85GnYIq2BCJVc5xiPfGouxTYlfSTSPFsXTPFHnpAvmV0rRnLiZs0HViJFluMsed2p3wrxGmsdpuVo2CZtE+i8Zu/+VXXgXnMfq0wL+G4D3AQFRqgKJhB98NTTf9yl5fYPjc60MozqGl34bQ8gFLeYx2QgjSu20vkqmxuXy6Pos+sonESt2EBHa5SRjpVGtGO/5jimFETxWrsVNW4REMg8AmUTk+j+W4NPkz8frPwEEpNLBdvdKvfmtB+bUBhO9eybe4aedFuELjODLkY+zpHIJt31zG+YZgpPeM3sk86ZbA3MYDkC3n8GpX2JkFooEBfXBdXQlFJMvi2DRQNs/nXiE+nGHCG0Yw8mWNBQad8IGxsMk80G87mvKLVLu/+RumiOqkD4rdfDtIe2hW6LAxcvCnXOz0Hs6z+cNJhtXL48kvigRi8SE5Aepk4z1u9yPWkhCsLmTbTxBr3klLi6Dj2e7Sj04MCkPt95IHr3wIIunjiB6SPldhWGM2TOWy1P9aAl8lbDorxmW4csrB2aS/7KJZ47tovyLN0lMz4cwL6TSBfb+8dKTn5hPYvVUxhxfSPRdo53abZN0Y5P1cWJaCXKbjgzJ/EHdL9rQKuAEEqSVoYSF2m8iCj1254ZMKsW/yR+3XjfqT6exI9WbJf9Yc9pcLTRHl1GohGN97syUKhwvzyd5RTK18R1MxVuIv9rO3oiLDhkLkobibSlBqIniz9+WoVw7jFFDynavycW7roqH/Tx5oOlefs+0EjxAV/cVcdASyPcWgbnDj+JvHs2qITJmlkfQqYygW1nD821gcglwkrFJkQ/w53A9F+Jnoq9PYral1x4WEhB0hXhr0lmSOxVjlYqpXs2su2W6I79EIsGkiKZYnEzDXgU60Yrwoi8SCUil196qEPhz2QH65Y0sl6519PclfR13Sz5CHajluQmdlOrn2WkuA28h6hoJz99L22glh6ZncavHfKj5A/HUCvCbxM3Z0aibbSz/aw/4nUIUH3Ras/aG9iK9YQue1hdJ85zLP4aKWH0Kd35hNwKOuuGAE08AoSkVdBu6qN89gb0ZVawf+C5YTERk76EpvJ2xMeBmDeP0kMLDDVsZ6X6U3JNz8Tg/A9lL55FI7I9kCILAqNo8PE6kM2z9Ue6PzUEp/xKxNRtMnShsOpI3rye8U00RUOojskgqB1UQgsKT+b6jcOt9mF+sW8nS/IlNlCHBBL11oPBGIvdgU/H9mL81k9/khbD5AJOG8CSVSLn88Q20lYdwV0ADVRus7JFIwDWYr3uj2RP0N+LZMla16TBk6pFI/rb3iSgy/eB0fFt8qQuuo9mvGdlDUgdPUomEF595kZjydG78UUNdtAJRzHD0SaGrHzVL/sDNKsHNS4dU+pTTvLzPIw+1CE/vSqPbo5PsFVKn8cjTLOa2/UcxWwWKjfEsDZjvlH+0x58kP9LOls+X8pxXF4FiM+VCJEmmYzx2eA5Njb7kTra/OyeRDWKbi7Yk9iV+TmmHO2p9Mp5uEQ6eJBIJd9YZkR28nZ+n7UM2vAglzvr309lZ1Jmt9B4Zjuji6oxBXQJZ2xFGfeJ+/HQjqFO6/kvG3qlMpfv3EDZdTWT6yv32m9UDOlhu7ubB2/bx/LxDvGbQEyJmsmYgn4urhuL4csxqA6U5MTRHWJzqvsvsSXv8BRZ8fhNHftEyb/4wBGGw3sZIONgrsP/963GLruXhWwbzhlmLuWfbUq6UBiFr8aZ5XZFT2b59Wxl+OoeW3+bwrXQG/r8YnXjyVHly1x+fojRcYOGUSxhtLkgkLzryy4xVzA75G+9sL1xNM/HTNtnzukUSdcofl053xNCtvBl9gZ7OGgRhPAA+tjp+zZrIYcGKf7KKTtHLCV/oXW9j+gkPXHtPYCsMp3mih1O71/jmMjKsm6ke+9HbYLXkWadQN+mdIvF/LQGgKOOqE08AeeIojjzphqdBwZyMIlTmBif68TYt/le8cFu8jzuiO8DaB4KARJTw1iNvEdwzAbeceEq9OxBEiaP87sBeDk85THJ1APP/nkXvI5eQSNY5yhUGDuBI1v/As75w3BrlqFewSNh+JRJFSSRPPbwfHzGeU1KZ493ptrZivprxPuPj47lbHopLQMS/9kn6Vg/GHLUbfN/UtP6DLlAXXEdWahaHph3C3zXNgRHU3SfJu5BGr3cHilKYqMlk6ZDxyE0ppD6gBp+aHzhY8RwfTJY65p3cVM09j73BnuoFbNb14SvaWD20r/WVfBJ+H5GWSHQ6d0I9xznGCaAHD870w+0nxuHe5Y70RSmSsEUQtohrpaz5/Tdskt0cvONT1ovOOKDLZy3PHLiNaMVkkiRtWHUlhAnpmFxCqY/4lQZPNbnuj9Dn1kOTfhNhEvttfY3hAp/On86bWRvZO1CWRCKBnKcBSFA1U1b4OUoC+UrVxaS0EETvdAhfiRB5MwdOPoVs1U/c+Nv1lGYn47fSGe9577yP+DP2A4dh6af/NVaCICA1SwmvCqfJPwuJROI4WCaRSDjR70VniycLLjWhii9FEOLs+yyfo3y1djM3yazUBOdRkdrlVHZYnheLdz7GlTlHGJFYjtnqLAdVMY1kJdnfrXvFAnkKP8d4GNXRLPg7hfiieNxzvaiZ6cqCJ0VQ+oPnMFxMdbTsEamQjEQ6MZ+1YencYe6Eim9BHQOhC8ht/ZjCfaUkFJVSe/1ZBGGcgye7bphKrvdWAitguc94VkskyCQSAtp8SM1JoS2pHoW3AhFxCJ6zMlEqockscJO3mVS5s+0k1iOGtKvpNIV08/nL7/BHt5eTjF1pT+T0W6NJzcxiw5O/AfB5zzsObHO+Khrj+xMwyA18nXSFYP3zTn0WrjyBzL+CdmknmeEv4iJZCgwxwklEzmWec/xeNrBfCen7kGk3ZnG46SW2xMZzxqWLT702OfJFiA1MdYP3sKEwdBEqyUEiucfR9oluF5H6Sym5mshDG9fi/YYnGUNk7PnzaUTtisGicsWgGIVm4muOednQdIHAuCeokcE5k5wOn24nniRKb/LTL1Np6CfgtwX8VZnMagb1ZJypljs1oCiM5bRVSv94HwetJ8RA3N1byDmbiGiw0ufl41R2Vn48k75fSMmSnzGM6KS4awSjmegYj7+73Uk3CzTPOUxYRKP9+6yzYNIhrevm7JizxBXFcc+H91A23+ZU9nXJ93D87nQCZUraQgRkMuk1WG7/ndXEePMblOaYAYFw6Rin/JGaQGZn30rGrmDMXsV4jA9DIrEbYcu71uHSKSWm0AN5/1gkIwb3naIo8PeCv/l7wd+8+uYTeIR0wNuDMiaRufDlqp9QAn1Vu0jURjDV1g8dl8EtBokkFbPUwkgXKy8Fwm80IybcBwn2UIVSfTNhxflQvJqTqVk8dI8zvpBKIfryH4hWE7lT7gOls13na8kIim7/kkc9k7g/bT8tggEx6kZy8ywU1lxBULugWzqbal83RrvZD84KgoBMraIjRAvyP7m+SEtpSDOiWxissiEAd26dguW7Z7BZRQQEil6QIAbNArk7UoUrHVlS/Kqa+KL8Zpq1VQiCwBT3DciiUjhQfhtpPj30+rRx3fSrWCLkHAPE1lPgFoFMJqHFfR/dwhVSLPNZna1m4ZibyRxgzFVwIT37JK0hwyg0CfysgzGqQARBQNfoyV+v3U/YrCLOjTmGmz7AwZPU2Mz7tUnU/T0ZmXCRoMY2MGcgDH8VuteQ2yfh0uOe3FoZi3u3ioCMPFRTnPWoKAp8eM+HGBQGdBodkiY7TSaT0G705Pj1vxKhjmWKHma5gqgKBH0tqEKwybywWF0QEFipHcRLEokEbDbWjL2TX7XQ+/JzxFXcxqjn5fZQf/Jgnn73B5QdfahnK3DpS8HTJcKBjYzSGN4p3IrcI5w5snvxlTvjFpOki0OT3sV9tAtvmPyosXQ40WW4Mn//JhKze9nx4IeE+jvjvfb+YJSenbisjUEVl4XcxcdJyD7pa2Pni8/SDEy6u5KfJ8Ta8xq7+FjdyBcza6n/KoSeHefInxtkxxdmPatccymRjqA1QIdi7E7CE2rt9WICiYQu6TYkkW10ayqw+HeTLYnjhqAZjnrPZ8UxY/dwPLo8WB7fiFwwIop2x5WiP4uvp03j0+MLKOvxpKo9mJuG8CRBZN7Oeeg1NkpHr2SMynneAHxT9Bs15VmEyQuJct+DRLLBQZNKpLzVAdUeLbQm52J2sWNVfNJZfiwSISsR1yngpQKZYEUqlSCRgFUZxuWgWzHeNAzLxzspVysxvTzYrr09NkwXkihKKid/4/cozNFOGHacqo5lCvhx1EWyokvJYK3TWB5qfglTxz7WuoksCwcpJmfbotQXzflLABRMPEmsuNHJFlYmxLP19ufx6Z5NbJsNQbCvtxWS0VzuySfCKGH2kSkImh7kmeaBvALTU6YjeUBC+MRwEpfY3xstPCwiAbwl0KJXoewwUDHFzH7tzzRqpjN57neM9NBy5fi9XL3QR3itP+/du50elwImuByBzG/AYqCu25dpJzbjqfNknCAgisK/xur/RL/Gf9bGf/o7/qvf/3em/7XOvsjISAICAjh06JDDudfd3c25c+e4++67ARgzZgydnZ1cunSJkSNHAnD48GGsVisZGRmO3zz11FOYTCZkMrv3/8CBA8THxztOi48ZM4ZDhw7x4IMPOuo/cOAAY8b8+8H0/3/670uCINhDkQ0R6s3XFVA2rYlJVa1YRSs3yt5w0FSWZiZfGkWzzoVP7voEjetwbhtSXihelL9+O/EpRdy9/CSfGkc51ZdrSULs6SDoQCBd8Stp7H0br/w7oa8WFtfiqfRkWME9dLge5EvZVdYa4hkaQd0FPVNu/Zuneg0UhFYwbmCjbzG08NZTm/mwE7Q/X8fm03E88mQYseJAmEJpELdXy3Cjj+RpB+kNU4PFCANGW7XtJsKMSnSJJUwM6aBKOQGibwOJEvwmEvmRgoBj59g34x1a/LrRGfbi4jLo9f81400KM7ZRrQFj57Ahz4zbU6qlAn1xPBTHU1a+FM2GfzqxBZR6JfnlgRz3aWW90k73Np0gf+MNDAvrY9lzFwj3KHTK1WbRktsC0doveDflb34VYv9RrgWbzUymVMIqVxvdOMclnnRsEoJN4M8ZuTQYvByPOAN0SwPISjpFRGUE12+9HvFeQKoCtyjHb1r8p9AY0YBqy3wSwv90Klti7eej3UtpyY4j+d1Xye2vdPDq5tED8eXURsZikaZiGfRzIHfx5+kXP8G9uYTxG/fgo7QgDpFPH0sFny26jVcPruGxh++meVoT8wdtPwhGkYjsg0AIViGIljVBTu26q1WBavQ5xsQ0USmR4WU1Ix2ihq+6Lea7V/3IbPLnxOhSUjd0O+U/YkxgR082/hUyPN0L8dPlARlU6jvp0VRxPGUlc/PPEeN1CV/RPMhX6DK+WbGHqDYd8buTCLfVOZU7zGigoSaYkswTlEdWMkt0PshwQ+L9VOpC8Ha7hQBVM/C6E71HE05fZD533rsZHy+L/ZHyjC8g2O7oyTZ6kasXWNEWTKZvL1JjO2A/9RQesYCOK9141dXS+XsQdwl/k/2zvc9nNtxOnvErPLx1PHf/MS7qPZ3q3djvR6PvRZ7ZdxMNZQKLNYM3VqWGRlb793Olv49bro7C5KV3yisIAk+88gRd3u20PfgF1Z6jnekDj7VbhteTEmcm0DX4X/RzmeeYs9f+NsbGoAIn+jxZBbe+8w4zTz/JAd8vyFTd5KD91SNiE6x4XT7J93+lIN9k4dpq0ycNZENUFgqzLy/L05mosTnpyJOW8Xy59j7G9M5hc/Qezvf4AEsddI/0NCaHnePLfTICm1xQ2ZxlaNaYS0ydsQ+ZPaIun/KxE/1xLxsv2oL54oXbUHS1EbDsZie6RLRiVBjQRLzDR9PW01QzGjLOIapDsXj0YparSHlpFmW5YZR3jHDkS04rxjOzkJbfJ+NfU8twfS0weKLpllMGYou8WTynnje0FVw0lQMTAJC6hnLDVyuQGtspePpV3Mzxdt2f9iao45AZm4lctZtt8w7gffEQC0Y7h+UGSJH3YyqJpObyKF7PiGXmnS6DxPJvuFP8mV1t0+i882cejjZgf71a4EDz+1R/uhur31m2Xr8VH99cNrLIns/cS4zkcfqqRhGWm0djQCM9Jh1gPzWW4BnLN18vJ1Kvof/+lzDanEGdztzOxSmJ1Lh9gexsLBPHOOvmBOkffDrzdR7d7co3qJhyvQlmXYC2c+CejItEzn2Pfczmex5h5KWRiGucAWKPywTePjmGGPPXmF36MVlMgAyLqZMH1IvIKY+nZXgtSz07cDVUAiMdeUdeHIlV6sNt+dkkJ8NQNJQa/DaKDadpqn6KTR0jGOEzHoLUEDQfUjcR+fZTTNszBplHFyNHnKBet5x49eD80Xnr6Qws5HHxJ+RyG7Dcqd2JfsGcTRHo7q5ms2uP0zoMsD6lHYvYxd9vr6QmtnMwX6+W4pqV2ExmPD97B2FsqFO+E6dW0fpbHLKRtYQNL6JN6eFEv2PWBYpTS6hVXscVXRQTJTIo/x76qjHG7qJ4eBaNCj07R58hzDYZZBow90DtdrwsnixQvE/H3ulMMNyG7B6gfg+cWAKx62HUR3D8Rtzr9nB12FVqemN4dkjdglTJ9wlZzKkIQmFQIAj2jTuqIB5u6gBjNUsaU+nJSSZtzV5HvgAp3DC7mSv1AahO+jNcY3Tiyd9cwtEwG2+6XOaLRwvwlWcC1zvoVqVATmoOUY0P4137Fpn/mDrhvRba98xj3uFR5CXlOa2HAGaJG+N/W4J3mw+5Ux4jaLKbE/2UIYm9Qd9Qt64MH9MD3OieSoDEFbdmX7xmnWBryToUkl5kEgMCHoAdV3m7PsTuk7OZ2KJkfv12fCb7OpXrZhLoaXNngRDE06G17Fc444teK/TZYPrWRcSmlzozJVXxV185Mi8JfUv+JFE+j3+mi7UdjL9q3/yWKI2MG8K3xT2JkAYjvVYjNhEYWCtoOsproZd5NeNJDsnP89v8+4jqX+ZUbkzXe9S0dNLj5Y5E6PtXvUd2LKY6KwxKPTnn/PQDrdJI6s4psap1TFh0jOqBGy/X0rb2QnTJhZibvWgKiWJ8yHXOBfS3khG7jWyjlA2heczmOYauej3txxB35hNZm4QosZB874DBW5SQcjaDkZdHcHjdIZI6vSl1Hwx3Y5BoOLo3g14fHc2/+xOsbYUhrxxEaaYSekSDrc+DzjIZlUnOa7GryYpa78l9Lr3M8jTSZ+1wot+eVIjyjk6KrsQzdmTDv/pMJ3rRbRZoSs3mt/gLfK/5zIleG6bms2deQBRsvFQCfZo4ABZJt/N+LEw9bWPalpWcnnjCofcBxkVWkhybx4kzw0lWWmnUmJzKXdRgQ71vLQemHWRRUgV3RA0+GSGIIpuXbgXAIrXQaHbmSbBakHh10eWby2jfesqU0/7Fl9Kjj5ML/iY2uoGjUWbShtDktm5umZbN8bHZLBRgnLLG0fYnA/U0PvoW0Zcn8dgLT5N39+BYYbOSEFdLXUwf6S/sxKg6DwzeFOszdeDpVkS/Yg+BlwVaQ8zAg4PtBgKafTEpfTGGfsLkOOc2j3JdhGGrH9akL1EkVCK1rgBTt/1P6Q+ijLFue2ms0lAeCSfNe4G1g+ULEH1xKx7N03llxCXWL7NjBJNLKM9NOIDc7M2DH9+MwkWk/+5SwI6xO0Vv/vxgA41tcp6InoQuuX6gozzBZkER7MuVuU8hr9vAbd/cRmWUDTRxMM5+k2KFxziuVNrYOXMGBkU/YweMj9eSt3wurUGHWbdhE6eU/zYKhVhqeLU/gLqHvkTi3s8/7UbedRpGnJxKZFUP6aM2APZwvDKJjOaIDCTGTuIqTIyLd843w6cWRZgUc24iLn/Ow2tTthP9THgvFlsQqXWN2ARnrCgIAqPPjyasJoyzGWdpC1QAd0HLCZC60huzmUsnO/GzWVnW8RNjHh8LJruBG1f77a2cKxlIi0oJm36GaPc/AOf6n504j7ziXr45NhFxob1ut5qtHF95gbnjLjN6261M2foEPh/mOfJIrTrWb1tG4aV4PGacxsVXh7BwsN0xtkae/vRVPi65mW+7Ib9PPQThQqE1HI1eh60vATjD+br5CB7XeIZvehJQX7eDhsAGGoIaCC12XrO62M59L7zDi2s3EnasAWvyVmDwbdTr3PJJ6/amZ8cqfKPiiZ9jXxQlPTuYn36AzVdG0moRaOgHm9zbke978xLeey8QhVsrv95/kCPyRGYMqddmeYCeGYcpFcrxrtDQI3g5tWtv/AXkASeIafsdN3OGE03fcpbVzQ2Yc2JomncQD5XGiS64hrC6CYjJhyfyCdEtdqKP6s9hYk08i2ccoDiynIWGnwbb3ZPK6b6rRPu6k3x9JQZZlFPefi8bp8aeZXp0E49F1vOjyXn/WJJUx7nU/ewNgrHXYLUqBGw2ZNWHmecKggy8ZFasii7ndgsQUHoKuUFHonE3MsMqINxBt1oMuBRcBlJRufcyKvg94BUHPY1GJu4OwGiQUucaSoM4sAfUN3BPXQcF55Ppi+lHU+dHwD3FwLyBegU+94NsA5THj6da6TwWos1MqBS6zQLfBC9gZO9qHuq8C/qbIeFh7skYRq7+HLurEnngt2DG3+M8cSMDjiF/oIYjv0no8Ozkn8mt9WeWX/8Fe06txr83G4ktGnAd/IFUoDa0ll+aV1C0622enmQfk2EBq9k+ajWRkVBQAQWt41g+RFWFCX/w2fzHeXX7XP76cSlFN1ezZEi9qX7NdA8v4ZCiixne/ahVPeB1L9gsCOZu9k96B0VfE8t/X06bRz+EQprLe6RlQGnHSN6bnc2WCdnU+Ofw57eRZIzutO/D3BORZMGTbXKMf9+EsTqBTu8UvGbNcdStMR5nxm0HOdYcxMHsGO7zreb0UjvG+s02DGHcSUxBcib2eXCLTxvS7svgNQJMnUxVwe8IlE+u5YaYv2n26AKvt8FrBBEt+agUiRhcVfw+/3vM6h5eEN8ZZNrYwXpZFT9HtHJaD7e7gaJ3MFKDr1xHtAxSdqdw6ugKnly/lUtjkuHMahi3leqkS/St+4WRLc+jlBRDLDDtiEOANzSM5rTpPKkqBTXG4fgwl+GAIJHT2eSOT0M1aafHMLb0Vm7YOjivraIrOQ2TCLE0smdzJpWpIt/cO9jseJcz5HrCW6eDafzyJoatGQz3B+CrbGL8iEryOkNZ9edKFt28y4n+2ZUvUVYUYBOlFFluZJTKWQYD1INzsaPZDUEyYPeRecDYnxDr4jk65XVstmaMMnsYWCx67nS7xGdPXOKHmnHoXXLY2anlyn0NINpvuc2yrWNmhMjGICMvxHXxi1602zwldrpKr8BF74Lm5jxSMusRhUEbWFlfG1vevp6eDjUf3b2DCGGSU5vvdOtix8VRmOMtXH/vMNrNdwL3OP1me/mnTMmKYvviY/xq8uX0EJooCmw5nEqsWckPG/9iu96+FxKVftRVBjDr3GjM/c3MqHiUllVljgM7DaZeNqk24yJVETNBRp9Kxy365URgXyd7xH5e3bQJbCKtIZDf7xwKM7W/E99d1+ERUsjJ4Vlk/hMgxMeT3RdNRvOPlF5QkjjNec3SmyqpnhTBibAnqQiv4GUhxIkuCAK9rr3c92Es0RNuoK9tGXADElFkx8IdBNsyWfvmVCLC6/C3lQPh2GygaP+D2auOURa2lt5fVFRq0qnzXMhXLXAgGNbNPMu55HWYPHOpMFfQpPAADy00HqRXEcxTC19G1j2MsJZ1GGVNhKjDIHQyAJLL25i1dxbq3kD6vf6isdMDmML/jek/8nf8T6b/V519PT09lJYObsIrKiq4evWqI37qgw8+yEsvvURsbCyRkZE888wzBAUFsWjRIgASExOZPXs2a9eu5dNPP8VkMnHvvfeyYsUKgoLsG4BVq1bx/PPPc/vtt7Nx40Zyc3N57733eOedwYXggQceYNKkSWzevJl58+axZcsWLl68yOeff/4/2h//tyVRFP91e/Lbqz/RrlPTHzIDBOcHKo0ST0pWbyXHAE1u7Sj/8QhmQNDjnK35kDKNgo+qTjI9IMGJ3uXjwRf3Pc9dv6zCR61HwAJh10NfNVgMBCpdufm0Oz01K9l6/VbqjanAoPPgjDCT171+Ru0D16sgRLDfBhVl7qxvhlwDaIsj8MZGbPOH0HEb+I1HQOCej+6h17WX0Me/oN8cBYIILkEQspCbruRQ+NbjHJy7i+Wjz3CfYg24BELiwwD4Cj/QZmph6W+ZeCWWYxOcjVrDFB1om/1Rfn4/gWOdAT6AzN/AtJc/ZOdT9+Kia+afuubVEm/qf14BwM55Ox3hArfZ5tOceJkudQCbju4lLNTE22sG87Vf3o/24CWae9fwasdixi9y3pyMaJmG/99TqP77GW5IyeGrJYNANlyhJG1JM+c69Ly1Ow1pWDQPD8krCAI7Fu7glr1zSTo3ikiKoTMHSj6FyNXgk8EZ9SL6UisQTXfR333caVMlCDaiQ1tozYrH5fe5RI4b2PjL1CyWqWld/jPh5nOcib2Fc41jYAAOCAJ0BsSjd/Nk5rv3EhFbg/DwYIeZFeF8ffZlylv7UJq6aDANAeeAWtVM/8w0jrn9QmnAIW4VXnbubEFgz9w9nGvz4s0KE40257EcE7qUY/5+eDed5htlHcP0sxgCWck1RJCTHY5p1edM8w3B07aWG4AGz3msyH8btz5P4qr/Yrb7twjtz4B6kX0sPKaSVDuVRv/3OLvpQQTLNCd49E2yhdaE12g1Q6gIEpuzUeu2q/6YOgysjXiTUd79WG2vOuQEwDPJSFGaldX9OiIbxrFHECH6dgdd1aFgwoUJJByZSviEq0jW9ThoCqmCzim3ktOaTaHfAnReg05KRbeU8afH0xwkZb4XuFojmDykXcv791NUYuFyvCtirIxpLkOWNKkrLUt3k98jEP/oc4SOznXiKcKYjX96ISc0bXzgUUeGxNuJflvANOI+eAmDWcXU+1+jVelstJVY9MxSwc/PvcwW1xiCXfqd6F2apfy2J4YOUwsaEZRDnN1xhXH4tHni2iPDu01ND4MyNmhMt1G04TPaJDgZOq6BBV2NkYe/fhX5omRGD0bNI8v8KAcuQfKSEF4LrmOrzTmkwAvN8Wxob+WXAJih+vcJIhk9nPkmBZf2BtrChiFgZihU+HLycaQTe3nwfBBtDd54BeXYWzv6S7Z834XE2Mdbqjeo8DeTVjPfkW+d2pM0tYSb46Zxz82TaVEucarXv8mb4VnD0cXN4IDrRZKGD+oTD6UHaTmzkPfWs3/WfrzN9k0DiY8AoMz/kQ/94AGrSExUEWmuOsDZCvlGi4ydX68EQJ+4xolmdU/i/IUJSBslTE5tZ6Fnu8Pg22tqxKc2i243AZe0AtwUQxxIFgP333KAr+cfYcQzD9AVXos49FCDdyYX/VyxGY5z3/e3UJve4lRvS8tmfp77Cq99Nx3fIzdSl+y85pl1RnL2xKHdrcU7sN4+Vt7p9j8Ai41HT16g5s6b8PSowt0yxyl/YfdFko5uQxJUxzPPfUtZrUCg/02YbCaK8oLoyvdjxoQbuCvoK1qlg3Lgaa1h0eHp9MtNtLt+hE4WCUNcbrXSE4xOPYt/1Q9EXTKjHC0F2RKYbL9tFpdSRbVHI00xpVR49vLoP9aclwIqSOsIYOezwwmfvw82Ojv7fFyLeO6J57i1Cc62OTvsAC53buV870miXn+cOxSDTts6mZLIt1/l8OENTN3ZBQkjnfJt99+BaXIjOdqrtHlXs9bzDSf6wxIjnaFFqE9+j6sxhYmPAh4p0HEJCWY+n3wQALkAUsvAWPmMA4mSTt0Mdn/YjIvZhrvSjfFuPtBbPVCy/bcWbz+qY3wp0mpIljgfLhAFgQujL5CbcYEMJUS43uigJeTHMG/HVHYtzOOXpZ+zTPkwCwZoclFgXMYZOkpTCZv1OyluzUikg2Ft5DYDmoowwoz9/D28mT+6853qvV19hs+9BZa3mHGVdSDYXAH54FjVZDLlcCwtYTW0TD35L33hJdqozSzFf4c/kdnbEFjFNYcdQLE+lOYjGYiyIPbY7uZGNwhwh+zmJ3ih6gnUAXomVz3AmJhLYNgDUvvNlEAhipl/7MEmlVLf54f7cLNTve9EKCh58G0e+mkTL343jzn7JzrRn9k2hqjSMFKni+j9Y/hnmnJkMmPOZrJk8ztku+T9i94VrOfztZ/T5d6FVOXPmiE0QZRh6FCTUBrDrH4PvMYNOKD09XgbduHpchs2s33MBZz7K6XSlaRD2VTOP0ByximqW79kFINry/HLd+JTfYnNj76C8R9OhW5JIC88vR6FaOOyCjzFxU7tqiKU98a+R3B8EGkVL+L5z+Aj1n6WXP81rU1hHNWZSBecDyNVicGcTc7FFFaBdOppAqRPMwnAZqV39FVORlYRcXAs33w2DdNffo4DWnqJH+/c/gUuxjhu3HUf+YpxTuVK6CM8ZxcN4TZe2/gaU33uc6L/eXYcVb8G4hpVjT65GtcUuRP9W4sn1sSrjAxIQ2JbzT+TIAh8c9s39n+MgNQ5BI4ogn+rD2s/vYvjk87Ac/Z+bbYFsjUnBKFbQcCaP/kuthlsFhgwPmnl3ax2gzdjyjgcd5UM1WMMscUxRt5Jba8rYk0QV6QGzOM8BtuEgEVqQalXsvqPZbTEtNudcX8FgdWIIv1bGoJlvNBkY97RT1gyJZrMf/D1WMBZRiyvx6UMktuWsWEITWrt4yFPsFzQIj0+nZXrf3CsWTqJlHrXHvy8ummPK0PjNqgPsFnZG9rHLzrYo72Ch9pCU1c64QMHbxpQsW5PECG1KqaeHYNm9lmnNsXaznL/13dgDmtixu2LEMyrYYg5OVN+FEtOIW5+Kdx1y3Ea3QLh0HRovwDjf4ewpcy7fTeXCtx440I44ROdjeTnSjbg3qbGLLVxNvMs9wkSxxgD2LChHJnPVPd+lJZWR77LfW1UF9jw6JJAyRTSw/6yE5a2ATZit3zM69N/4vvzo4iPPk2I9xS7A7J2O7incE/o3XR7Skhpv4pNNP1Lz61ZtIkgt5ME1XXi053COpyTe183zZ+uZNzMc9y6cj//THc0WWnM1eJ6/7dY4gYjEqnNV4iY3sb5uut4+NEXcFf1AM856FHadqZmnmT+weFEBdWiUqU6lVt49ToaLyt59pwbxmH5SO4YxKG+MgkeCys4YT7BytQSmo0DTpz5BYDA2aJjfLuugJlHYki8UoKvaRyIUvshlgHDLUoFjUEwZVgpHUPWFQCrzcqlHwJoygljKqCbbw/NqfTUsq81HbfGOUgDVQRn5uKpGdQJCoUP+X7QldJK34kRXPRv4oUh/W1TBnC0bjXlTeFUHJpHc7zzPsnqJfDB/R8wvXoDn1wsZndJLAsX2mmCAEYlXB55mUkusM4FypStMGCUBTjRP4rbS2BR2mWqvLsJMTsfiljgmsuWV+/DC5g98nvcjfFAML9L1/LClaO4W39i/IlRdHm0OJkpBAQSChNo9mvnW6GKEJOzfaPNMpYjBWM5P3kmbQsP8DjfOtFlmj66POrR67uIkpyCMx9D9BoInEG/JoUPsoIJOD+CexYXIPXy458puDaYsOowYsdmo3drdaIdcx1D89caxrvpKIwpRbQOri0RnQn47VmLe2QMLy3dyMlO5/74KCUIWXg9j18Jp2SXN5mr3Z3oL/s10i/C7rdu5O/IOj68CdgRDVYTrqEvcdfeeRTrJTz01id83uGMyeQNW7l/0+v89NEN6PK9qe/4kxFDNJ1ZkPL+yFNMOZVKX5crLZ3/kAVVNAGzT7NNfZQd0Wd43ZAJRIDEBdu8I9SPPovZRWCJ2kC57TFHPlEQmNUWiKzWg59DR+IqOh8mUpnr+bs2lb8vR2E5P5z8Jc12B+awl8F3Al7FW7kx9T2yfvgGVUM10c9MhrxXIetJGPEOFvMhlo/eSWc8JNv+beDN6WlDl56PYU8hwzrLMNeZYOQnDnqKrJFwoye2/Sq63QUYOIArWA24SI1YjDbMuk/QeUgwWx/k2h7tfPkBTr6xCH1SPltuKMfDb4I9ZNPvnuA/jUeVflQt3MJy6Qwm+edwrGcxNB+HM6uRjNvO3ilHqTFbWZSah7vEl9QzcFL/HuMjtmOwebGyIgKDh41FeGKwuOIhy4GTy0D7PKL4LEkmyK4MRdpvRvAwOx16KO6vRJ1eTsO5AlZ8t4ijk4467AgmNRyfcZC5jTcx+o0HSb3tbyT9VcAIUMcz3rsY/fp8hokjGR+ez96uYKjbBfmvkqT05+k13Ty+729y454eGOAhusqkY5W0lpbSMYz6bBaTlx2masQF4G5EEV5M+h7fjiBeatbQHFfCbf6dYOwAl2CwmXnm/J1EdwYg2CTkn09i6vWHwXdQnx1tDUdX20aOVk1j/3riHVt4KxXDpuN20zQsBy/jqWokpGoFUANAj6Udn/w38d+t5MrwK1RGVAPPDMooAq1FEagvp9Ecm0lI7KBNBcBN1s71Uz/i0+4XaNrhQ6tiLdFD6JeD7mLZLneU/SpqYx7EZtM4rWuXo3I4uvgKxeFTUSiVdpsoEpC6QMQqmkygG5fPqfg8figPBuaBTMPdHYvY1bkNKxb6rBCv7Addsd0JBGw7OxLxx7G03vAHm7xz2OS9D351hZV2u9Kfc4soyJTw9umtHN7jxudDzJYGeRB/Jl/Ftc+b2QU5REQ6H5BVqZKY9fzP/HTRjeKdgfQvOck/nX1tnvWcH91CqVcrKku4E00QYOSlkfi1e5M70Rsfb7t1RRRELqZfJDs1m9VVS3Dp7aPO6O6QX7PNiFdzF4GNGhbMOs9ibwN9GBzlKmx6UuTQrJPx+t4E+vxsA0eV7Wl7UwojjoQSO6uLLyZXcVpS4dSuanUKyvEVGJ59koOaEDJeCHSiy1vv47vbdpJYCcqeUOT/OIw0RjjLQg8XjieU4hFdjs5kP+A13HqI/cGwsaGf/usmIQ84gqiw74dsNuit24tr1Q8cb4pC/v9h762jozq/9u/POePJxN3dEyQQ3N2dAi0thZZ6S90LdXcvdVqgheJOcfcQg4S4u2eSyej7xwRmTnh+v7/e9Tzvep/vvVZX2+w5fs59731de197dzIN8U0IEwTazLZEiBYLqK2eGHpiWCcMtgTYc8tIdE6g2kmHm7WbIWcXIjMZiLaUw55ZkPIGWlMzqSVRqGp8gUxOyi7wv5Xs+6/4jv/O8T9K9l26dImxY+0P/mYPvGXLlvHbb7/x/PPPo9PpeOCBB2hpaWHEiBHs37//lk4twPr163nssccYP368rY/Y/Pl8+eWXt+xubm4cPHiQRx99lAEDBuDt7c3q1at54AE7Ojps2DA2bNjAq6++yssvv0xMTAzbt28nOTn5v+Eu/O8dZrOZkpISwsPt8jU52QOQVZTy3uAFTIhJx8VqzwC2KNz5TNFk02RGA6I0q1nhrCJz4jNY6x9l2tvPoXtdinQ4yZxxNgQy7MkN3O0KeZZPbdVzeV9A8GxQJDJ59g62fL2Qxy+OwG3qCWDxre29VUH4tE5mxmEtyeVhxH+VAYCodGZ3Vgiuba5UPPYnnwV3YBZUyDS2yVpjbSJg4DUKxG4edoc8faMtoJpry467sWc6yuGXqPW1AcEigg0krN4PPqOYO+tFuqZa2LU1iTCfttvAo+VCBWJxAmeqjJg6pRnTAFfMKTxt+Isv3tvGKu+vEPSZ4GQHefxcOmn0aqYuroAnB5YidNeD2pduQcOG+VuILbubIVfyMTpLQdd+3q/SGZZMZZWK6rwwdJOkZIlfQCOd/a+RU+NOaWgZgoPX5y6zcP+of+g8uITgTWlcnFkq2TZRd4KGSPh4dhF33lmLIfxpm0Oa/60tEPUezD/OZ9nXlMCfEWrC3KXBBzItU1JOMKhTj3XfNE6FFbOsx3Tz/kWnR/HPlmG4PGAPivSGdlSTvqWm2wXtGS03PH2Yr7CnQ5UaTeT+YkTuouPrx78mQSYNipSm53hv2RZW1cOnWsiySKshQ2StjNbAyG8foTCqEPE96bN0surQJIWxduL7VLkcpb8ode4Ek8Dy35aT0SeDP+ZtI8psOzeZIDJ7x2xCykPRo2NPxSL6fWoPqvxbvmWV517Wtgxha5uWFIV00ak1a8g1wCuHphFeHYj2sxKJ3XAhiUFnU7gKbE24zm+PS8/7vXGpbM3z5bddk2nuJyVAAV4wlJF5dAaNPvW8E5HNbicHB6zuFE+nvsmP1+5CHlmOv6C1SV11FDHg8guoD08gs28xlsw51PWVkiEjq9/Ha7sTBQMX0dKL3Fc5+fNu+qt0KouRLztOoVolqQSO8RqKYfEudtW5oDAoEGTSa/IyncWptor29jDu2VbLSy/ZEvpuDmdjAx+VJrP1mwXsBIQHlrHa4XF1Og3h8mEZ/TqP86jyXvzvtr8Lr5QmUHQuhS0vHGLYfaeQq+yZ9R6WJj7ceC81ciufLvwdcMxjhcFO4biefpyK5lbM5hqy9WWA/T0saTtGu7WEfS06SkQYGyPNAGtr9EBR2pdcnBASypg8X3rdWeLb3Mj+kFY/T95afQ+m+gRwyI2ran2Ipgo9g35y5fzECyQ9ZCAM+PDgh6i/0+MWq2Rv8x5Mrkasot0ZPnz2Y7ZfuEF331KGX3qQaQP6SYjswEnnuD75EP6ffcaxncnELLU/T8HShddcNTeMrgwviMXs0vOOFf4K3fV0u0xgaroXuvIQnk15CzpToFc92F686IwtRRMfzNSBX4Dlg1sB4UWjhueWfISmO5wJB6voXGBiSc92dS4PYnzvMJc7/MkOhd877BmhKN0JqDTQjYHPXvmexR5mbjgACoIATUEpGM1XSDnlS4pYIjknk6DgxM4RBFrlRNy7HZXbSIn933pvNHtTsKRmMWX+EQTesVWiG9vBLYFuSzeXzScJ9Srkj+QWDluy4NaZg7rzFN6D0ulyaedyN7jLeuYyuRNPzfgHcZrI01++y2rV5zx3w17dpETPwmfX83O+N75/a6nWH5Lcz99bR7Cs+jz3FgdCuguaB+w9ZAHeiQN9uImI2m6sbfB8L+DU6cQd7P7HA51LG39paySEBUCVEMm6qw+T7vEd7r2SfgCqzgm4HNDT9yk3on0bb/39h66J7GrLIDrExAu/vU9J10VwCLlqAgoo8DmCvgfo6g3onuhU0iWDlw/socnnEEbLs6jd+4JTCDclptVdajKUvpS6lNiyVbWR4J7MjuIz7Jr1L2GKaOaFxtPlNgBC3cApCLxsxPVrs1LJGRHPSpMHFndpxa1apuSLprU0eGznzeC9/N3VM190VvFCYhlnms/R6i7DAFgcgQynYO7c2lO5k/keYGXLEvt1xYS/xKaZIPjJ+MbpE8oUvpLkAb+O6Wx4fQRT29RotO/j80Bf4M5b9g7fbj587kO2hxl5w9V4W7LQGGsmcxf9y5f1w5mV8gnKbitgrwYWTQJT90/FLJr56cHBVLRtpqHBiNUaDsjQC43Un/Xh3/oE+svtxG2U9RvqfYopdBlLdsRjqHq54zefXZWTO1YXb0wKaXDsXe9FVGEUR2Ln46xLRLKS6kr5YFApO0oiObN+MpYZvSr/gC+DzhAaWk9QMbhbpBctsxpZcGoSYRds4Ma4tHU2Q/idnEgfSsO/VxkUKPCOYiBZWmlQPjn4La44R2PU1PKwdys7zRUSe2XKFHJTY3gi+FVckZJeo/xmMjW9m0L/DzgQ9BpD1dJK95v3JNmpkUVJb6Ey3gXYqzQ6FZ6EX0tl5h8pTA2Mo7cYg0nhwbqhJ2/9f5I83PYfFiOHR5bxZ1sZX3eHUR6UyzDBXh2KINDi0YJB38iZYRFoNVJSwK1zHfe8tI6NnQF8GqqnwFIlsUcGzuZGUh6ReZGcqUzi3k+kxO3JTk9KrZBXMpMkce4tX+7mGGo5gjUGHv9wPnVOobjdL01hv0t2kjfxZ4OfnrGaR8CsB5maCwxH/p0Xic4WMp7/Aie1yN0OSUzrr6Rxct9EDCPPURJzjRSr9Fl+7a+h6ZnPSft+OUPPDoXlDkk7okC6youjlb6034hlbJBNgomejG+L0oPPf5mC4vQAxqtOcz78BCBNPjhlCOREmY7Jx19llGNGD2BS+jN6RzLjNs3HIor8U7uLh3ps4SYD3h3eNEdU8s2ck2zpdPh4BJHn/hyL/Fwfgp76hfei2rnQtZibVfYCAoPPD8an3ofRr39Hm0Y6/7YL3rhOXU9tYQSl/6gJu1v67XQIUUTNP0dm0wSe3L+CF8f4grqHmLDaCPvRGQuZslGPQWnAOr6fZPtwcx73/76Pu2ptK47KYqs+cO4q4LdrQ7jU4Mrvoy+wX9nCR6pP7ZcFfPPoNygNShINzmxzdrZRkD1yVWUtRwkqacHFtYNpgy5zzXklNJy3VW949OfSseFk/e3JjGU1uCYUSZN2gHK9N03dAfS/EkS3j7R6GaDUeQCnZnxNtV5Le1Yo63rNk/KBRfSLvcYdYi3KGn/ae/6uaV/NSyN2M+LEu3xwahPxcSaedtiuoCWTdXuhJuQJzo35ikVuUsWhoxnL8Dy/B0GrY0ZKGaLZHheHK5V8MvQoPxTMZ5FLPpc6W2yGYzNAEGhQ3Uts2J+4TxnC8oVVNKhiwX0WDF1nA7qBoYNHYEmrZHyVwMCG+5ntcGyr1cpm9xKSg0Veeng3x1xtRKQ6aAobToxAXtdMc6wPO6xqHnXgUlycfLgi/oU5vJsTk72wiBbe5Itb9ha5H5+fWozVeI6hl9I45yaNoybLLvJgMGz6U0l9+WaY+PKtdUkQ4FF3A/5OqRT+ksjg8AYaI5ok2wsIDD43mEK/Grb3yeJpq7Sc/d3mKfh4NdIZoOa8y/2MV9uqj6yimknb5pB0LQmLqpuY2HJcWk5DkC2xzF9s4PzqT8iu9WdwZh9MgRp6D5/iC8ySOfHjHbDdehwcZrMLYbXkd8l5reZt9FoDsrLz4DsMAiZiUfvw04ijxPq2YP7uCYLnxiKZEbpqeLYugtaDExiucUKVKp0D6xS+/DP9H1KavTlgCeCa62luVvjPMwWTnW6gtTWS1zK+x2dyf0Y4bKsWmnBV1+BxZgz+xX5Y7pXue0irL5XXk2m9EUOmqsfn77QRGlancM7kB9NtFFl2/mVCQqVk38mmIrpcjJx69FuajCJO5m+Z4WAXBBnHPWsRRx0n7WIaXzULkoTPbucENlle41hIHAIg3CTWlO7caWnD4t1ATNVqTrXIeGeEPU4SRYFn/xlCUkZfitcEY1VYAXtCkVnuxo6to7DWeZMyLBO34A4bbqIJBH01BcbH+PLT0TT6BNMdMtL2/rXd9NnKWdsaz7LSE/weaiDKTZoIClChjGNZrZ6RI08yWyNSI7tLYl9udWX7u6sAuDR6FQL7ARlBza/zVtRW/sj+kCHH9YRNPIulcyG42GLqTCIQ3fPI8m7EEhSOv0s8yHreQ+cwjJTRbYXG1nt56OxBRowAWl+y2bURnOmUUWO1UJYQRLfeiRVn4Zr5CUaMf4KG70Cx9zhpBRsprvsJ96kBdFgWQdid4BKDrAPuDzDS+tJ7LOJNfom6g9KWJ4FPADjTZWJNXRWhqgrGzbrBnfHVOOvLgRgGaUoYJIMKqzd1UYP5unAhX9xM0hJltFtkmAUTde0qgjK0jHGZyR3Zb0OjLRklVOuGTDDiKwM/GcgFh3uuCcQ6s4iWu2x+2g/qBjQND3APtvniYG0a4t5OorIiGPPj64x2AupP23C6thvEta5n05pOahvi6FKVENU8mqk570GKTZvDrV7NnD/v5t8J27iQvJ8+Xc8AYxGt9exZ7M8PrXCmXxN+A6rRJ4TfOi21JZP7hx/jkJ8f7fN2ENQLks/vHsHGC6GMu5JEzqh+yIOk81RZRwIT9m/D4KTAZcpkVtwr3X6+dy7CWAvi3vHEdk1DMOwClb2CtawrhzrPo6yKauF5T6jTl3LTX2zqakK/7V+GVo/lwOocKnU9mKEop9Dsi8/5VCIqEogqGM699+2F47NhZj6IMg54KNCMPoZ/SSiNV1PoemYrmnh7fNnSFUVbcxyetbV0OxkA+zmJosCltEt4N4axv3MEOqdE4I9b9kbXu3nv4t1oi1aRfc2Tqmk+SL8c2DPkJLnufnyeFUyXrzRxXRQEJt+3k9xLCfz21bP0eXQsEwCx6QrtCUZeaTRy1debPyMPEl77+K11xdlYxgflSeQeHoRxRAZd1m7JfiPIZWdHOB9WaXDeMJdrI6RKT/XeMt5/4X0e2fsou58dzPAfRkjsD5ibaL8Rz7dpESjlExBFqQNRLkTzRRPM//kRYj26iB/cLrE70cnld1bS6FPHzPArbFDb5tlAuQlFXiyD8yeTY5lEduEkXuhxz61W+KjBj6K3FiNzyiX8ynQyF2cyZXQTT3lAHxWkFC+g6UYkbW3nGHd+DVPe2QLnbGt1Z9hy5uy7TnNLDMHXD+HUVo1gGQCtOXBqAV7Bz/PuJ99yZ74zeZfjaA2UJmz+bxr/Fd/x3zn+R8m+MWPGYLXeDuTcHIIg8Oabb/Lmm2/+H3/j6enJhg0b/o92gD59+nDy5Mn/628WLlzIwoUL/6+/+c/4f3+0t0snLHn9FYJyr9Kdm0RmkjvR99mdSl+NP30KPiKgAgYe09FvcZlkW9FYwaDgM2R1t9Hi24laLl0Y71UaubvWn11dIitCKnhRpgL3PhC3CpxCsAievFn3D9b7lnGPXwX1LJVsP8oliK51o7E0GqiKy6fT2o9p2BaPoWeHkng9kYPJP6NUN9sS+XtK4lXWdp5fcZC3snz54OkPYGQIw1cCTVdAVHLcPZjDE9cy/8wIFh2ahPyVJihZb8sQ8xvPMw1m2qll31AbSFDXi+zbcy4Nr+3xTHl6G5bwRiBVYvcXwumbPYOCmHNc9Woj1WrFMQS5399A/eNf8KE3LHHHBoQAboKOBCWIuhwir26nPKQe+PrWdqWxbYQn/4rli0/wqsrGYpgkOe5yN5HG2ZuIVoCPjJ6sIduwOEWxbHsFxcpNzH70GwZ4SbPqfNyT2fnBHbSZEnnaYw3PPANEm0EbDX5jANieH0vjrrEMnHaDaO0hYIL9XehZodP7p5OTlEOcS0/4YDHzRvNMyk4vora9nqpKGVX6kFtgc2d3Iz/0+4lfWmFrkRkBEZXDLFnSns+V/ieRydzQR6jJ65YSnEc7ndhnAFntMPwt1SiDpbnai52zMGyeS6ZZTvSga7ct6G41X/HDjDeYcD2NWW4QJdyQ2Be5nEF7Vzcna5X41fgh9rMBU+5d2UQOyiErqIXTydmYZGYeFJZzM/ezs62d0+vSCA/1ZknJBZImSjP6vVvkpJXH09rkSrVeTrAolTTJSyil0q+AwKpAKoMqbwPJ3ziegGz9TIZ2abgYIpWaAKgP7iL82Z/5iCZ0Wh2izA5itrYVo+3MRmw/xOvCUBK8eqosyv7hjaSdzF9ZTlNVPJP29ONqkLS3akrIPlr6jOKa9xdEBV2nvPUl6BGyVStUDLm0GIW+nbrIIbeB1Grv6bx4cDr+rZ/yysmJeD8qBSAbNPMpHl9E88UMYk69R3XHYsAu0xXgO4B1x+8BbFJsFmfpPTtYuYmQnBxEq0ilcwiBVnuQq51xDPmIs7grgnjSA/b2ZPoBpLiGo1MZwRTJoPODaPaQyn8NVJZRcciT8Gg1B174CH8WSewuBZ8xdk8q2SUTqPSvoTteCggHVzkzeLstlW5zQRZTX5PemONtv/D3qu00a8+SrIdApRuOlPbB3GdpPpFF/cCvUCRm4+r0OGGAb/sJiseaqBQH8sqbLxM5OJus/uegp2qlo0aBKq8UU2gleudfuNo4Hhygo1cC2jBh5hMvTyxKKagqGFt4adwS1mfMJ/zDOwka1wOcnrfRt9bxBXAyjfHnh/DL3+DygosEMANwCjzN+VgLzwxeymj/jWB9F7B5vYJgq8IYfDyJPqc/omvYg9yU1JnnmscEZyNDdgTz+md30rL4BstuoiyCiKnVDW2Xiq2hDeRZLCwR7O92Zf1l4gY9xglrETWT68jsSnAQLINKWTxde7oxBzcy7Z7zXNW+JDnnCrdCDj7yDR+F65gQ0Em1AGzvSbqYcpl2RTB9L50iYMtjLF3zM7PTpNXV7tYaFj+wl/u29+evg4n8MNoWPIuCgFFpC4KaAmKxyBwkXIBGRRzxYgViXAXmP66yvU06zznXiaz5dA1Hxxxl27NdrJTHMAN7hcY/ZWspLRMZU7Sadpc2hIckm3Ox42GMAbv5Z/Z3KPykklEAxwpP4PSlG3O7X2f045tus3cLHQhmA89n+lGr9+WmsJ9oEQmoCkBmrGFn9Qj0qnAcUwDe8b7GTC2M2xdNn9JIVAulhNuIY2kMuJAMydeIji3BYH4C4h6DuMdQZRxi28nppBcEsLE0mPAFCht4nfESRNxDl3kImX1OUdfsg/X6CywbBDgBofaqrUNdneRs8CegyYcxr2ZIji2KEJe/hc6zfXhmcg2MirR92RYDk5NKuRZYguzqBKZWheEVYPebdMYOvHM/xb3RSkPkHRhVWsCemCOXyShPnIzBcpqGNx4nZ6o0o/SG+Cj1ss2oNSU4x9SQ0x0ruWdJTs14KTv5SQ++Rviy1zxqcB3G5msvcmr6z3h5lSPrasGR+p3lcQbFSA1FJ1P5U9WMuuUM7eZ42luP0rd0H52XtHz+6NeYRYEXlH/e2q5dqGPKK1tYdr2Nq6oXaTRM4HkH/2NGZxzWPBcuxmipiU5jpkmHowxW8OIDFCzayb8db+PSlQg4VO+JasqTcmkY6oJ12zSO9VfwsPSyyOn24FqNhWdO9Cekj3RtcDc28OLILHYQyO/9trJPDOdm/VP/in4cL3oaV103V07NIHCmlLC7FhSG+EA1F884c9eHcxn7vpTFnDZgIWWdbkwsHIGvV6PE5tZ1jNdjXufvS/0YfOgeamdI951ovc5Pl8dQsXsQrqMvUjfWh0TsH5/FasGrsJ3Q8lCmh7QQxO84Sjje9Edc2lxILHubuLE9wpGiklfrZWQYzVQml9LPKwMfi11eSWuqZaPGhR2WGs4ljwCTF2BPojrRUYYQVERbWzX3tfizQS2VchsS5UGVyYszIcsxm28PS33K3Vj+0+tAJlXD3EAi+g5Ngj8//tsX74wUhqXm4W/I56a8I4Bz071s+66MhIlNLFs6F7oKQRuJi9ydQyNzUMp8ONUKP7QoJGSfYFTg1uRBUl4MA/IjkE+RSuRWGp0pNMPsYVVMU1/DVZRKcZ7bNIG2q3G0f/YmmWZfG7I48TQoXLBYnVnvW8o8BiC6tROokO4bQN/5CtY1GQxFR1r8s4BdbUYQZWjKbIRMxvgHcAuwxzsr63RcW/sYh2dd4iXvLjRuA+0EvyByziyQqO7icqeV8hp4KNEusSy36LDesYfzXXI+oBb39lgcPZA6MY5n+55meDuIhyZwcLxBUpGYbvDlT+sgxp2qwKmzBj4cAKN3gdVyq/JQr99GeUgydcHhTHTrJ7nmi4oRPF2wj3laeMMbLptss3uwkyfnLvfBpzaQ6mGXKLWCVWb/3j2NReyLMPJVi5EdOh1aU4Rkv+srw0j5PJnMVCN3VTSwfDkgHrcZ3ZJZZ/HAM6CbwX8vJDn1Oto7pITG6qwbJF2JZ3ZuAtWjpSoRAKMDlpFpjCTg0BGKui7f5m++bw2kTDyLd4M3Zmf7vg8ZE/nzcxUBluvcCFhHbb0fT5Nwy26priUw7xpVztDgD7JeJGRlnIprnhOo6ruIEyqRtSqHtdo5lNcP7yKv5TrHNyioSDVw4gHALREUbkQ1ZPF7dxDvKIrYIAwnDGdS23Jh/0Do+x4kvQhCK8VrF/JSVgyH35LOR4IgcGT8ES6LR6hQQYR5ya30WJeK6/hd2kuD5z04+s227W7+l5UUbx2eIhJ56Iz6S/Q/fpiqoEqCP/yHZLM0XtF06+k+k4pz7FnmTmpm9fBX+M1gw5JEEQZTz9VPRiDUerJVJtLmL63sdpO1MuHEaGr9asnqkwW9YuoqixfrHv8cgOjql0i2yEgCPKy1WGKLyFFq6Rq3nyhvHRaVPbnqQdcu3lNZWXjyVZL31hO7+Ihkv57Kt4mvaqErpz8/uJegmCLFKP48MJiuvyYxHqgMrOTc5+8zJMxWQSQgYFKYUBkUiGVNNDRLK1YwtrFiYgafN3hwesdIzMYBksoSAYH82HxmfTOJ6xn9UT9jn2+S4o4y8at/ePaPf3G7kENbg9T/PpD3HmW/HqYuZgcX0o7zLNJ5bsOJDTj/u4MNSzZQGN0j8dn3PTC2kBQ0nvf7jEfd0YDr4ULcZkkVN27gzXOVOkSLAr1GzyRZL3lSwEUEuQBOXU7Iu6XE2bHyg+xPnc6qjEEsVEdQE3+BmwleokVEsMoIr7kHp+4gwhwSbwQEzg68yPXofJ74cxnuIbXwin2/JrkXP8zeiq/cSkBKNaX6Hnzj6kugjeRYwUq0l0vxmLCfh5dsp8ZwFPp/DDGPgmsM1p2fsuzjpzkCPDF/K+m9CtJFUWDOtjkoDUqeXbSJtWpp25ky9WOYY46hGfA326depVlfAkRxRS+Q/ut0umSnEO/fTf+wJkn1nNlTxu8rfiWueTF5A/4ivTXAhkUtNoIoJ2XDZEZdnoJbbSXylnUw4m7b3CzXImq8Wf/nZFIvp3D1+W6sPVKx7rIb0OmEIATj3FyBoU6PSqNndsq3lArLoNSGwcoUS+i2yui2wgHZarbrwEkl3Eo3Eszw1KdPURZhZNC4eFb2fcz2YIHJBjM1G5biOaw/ZxKm9NykmxfVzYhWTxQFwYRd609I0SQyXhYg5mEQZegHfMtzdzfhWnyMR/a9Sr+hWbj3reRWNa8oB20EtRGDuRr3FWejrzGj3u4j76wcR2ufdwmOq6efSQCsEP8MeA+Csi286dFJvh52qLu5x6ebqR7HIev4LbLv1757OXBXGNGF/nzWUkel7HtgLIhq1l33I6fajVzn79iq+YOy2jtuNc4wi1qqR50jLxU+cAcvWW8yRODSwEvkJl5jRrcfHcYIcGjUYxFNNLa+Qf/0/lQkaylrDyXEYa5d7JuLy6RO/lFYuKt/KZh0ErKvvvBxxlz25EbjIL4dkcs8eU/SvKEF8xY/lPLpeFf256cfZtKYPMFWb261EqeJw+WgLwqjDFlYBQqVAXTFgAWQ0RLQyXmvY0zdNw23oggebXqVX0baP66Pjm1FkXuJ2Ot/Il4bpmMAAQAASURBVBtdCZZvuBmLa0w1/OYHR8td2P3zLALnOOOY+nWj7SrNspOkD9vD3sn1LFZKW8oAHOrwpPJSHIMPD2fkiN8lNq1Sy+RQI51nwqDsKt26voArCrUPuR1TcLkwg1kD7keTY0skurUsiXJ8xp/jdFwuLb/M5UWfOt4bbUdSmwhg3bv3Yo27wYYlG7C494qZld34yvRUyLSI1mDqlNJCgZ8yEwnbOJKYaYXEJdSB5TEcKZp8WV90r67Co8UDa2o5VqSE0TEmktNvCx3ajp7ztp14h8+3bP3jL/zq8ymfkIdFJsdKBDc/LpNZRfiNGEpSS/ht2W8EJwYz1XcKwRWQ0Q3X8ruI+3cLTsP6UBvcSJfbDAgIguTVBLYWMO90FkV5JrIHRiA3+KLzGH4LX25yimdeFVyQdVI54Apu1lD+N4/efMd/5/j/bM++/4z/nWPFPXdT1qRl/5ZvKVTOZ67K7gx7C3ruO1ZJc4U7ymQtZU4zJNteLXqbOeIZElKbWXBvBd8bpZmwbfowjv48C4vcxKVZOxAXK6D8H1sVnUs0JVX5bE60BVn/AM9apTJbXt27iYu8yqngcNZPKOMJjW2ylpl1PDr3LMeHZuN7/DDfFW/ngXm/I+uRaNTJQ/EpAs96LxZ0KBD1Pc7ufhv4H6D8BM/20URYRXwMakRrPXj0EFd+Y7hU0oRLlT/O0QXotLrbgshLQTKM03czMbCIUlyR5n3CGGsi2i0DqQv2Z9Ydf1PtJJX/uFnp9nwDvFAvw9LTq2GycIjRJybzrb6bQY9vZEFoC45k36+tkeSa85h777M87COw3VALDv2/bu73TVdPlng2oXcIUttNXXgd+hNPawdPfl3IWZ10KvKPWkXRGT3uMiXN/YqxmkNBlEGg3dH5wtOF0JEnmJMTTeLwBwB7xY2AhRU3UrBU+FMy7V9chR7ZSKuZkeJmtmQspHiymV/v+Yilvq/at5OpeWr9SIxFwRB9eyKCl6mEt+49zB/58ezavhinlEQcRXUKdcEkPb8a0Sryqyhj8klpAsERfTTajGg8EwtZOTnjtirNAlkIR768l5Hnw5n/WCft3lIJJVFuJXbCBVKffIqFRfH0GfcXsBSPrkzemJHO6KIy6szSgBzgV6uMI0s2MKBsLHGna4lOseDYVGdFtZ6yjYv5d3YR7R4Ded9dWmp/aORVyswmltZkoLxdkYTThUGM7dIwctUGFsbX3WbfJy7hrOk8wapaYtoG4Sy3g5Q17oN49Y/hJOdEs299NMcXPMVz8cugeB2XlL/g2jQD8OLZH10pN02U7Pd5+QIKRpcw60gXzv+OQjk5nZvOsGBsZUDbOlpKnf9Lsk8QIPjaQdwMbkQNzsLFQ0qSn2xNZ4Preoa6NeFbH01l6y7gCfsPlB5cjnicpGybs2nx3oZjlm7frm04PX+WV+sTcanbS6iDn/NC7V10dHUS2NnCqLwuhswedatuyuDcn01hv2FxqmfaJgFtHynh2yAfj2bOMcrdrpAeCr+0SLOe+0aU4L2wmY4DE0m83pch70pJ+AcHXCA+5CrLsz2pCK64jXD2sJxjduBZynaPZLvOmZgnJkvIPllzPUF5R2maW01ofBkGuY2oTyOPpfcVknCiH8mNecgjKui22MG42n5tlLhH41adzopfV9D4pDTz7Ym8QIQOC+LSZUzwhJKavwkPt5GSrRaRZ66GUqY7y+il+eR59kSBogK8hyIKAjlJOaj1avRqPRZFwG1knyBAk/ofHqw4imfXIM459KV0bb7IH3hzyKDAEOCPpyaXm3331jX347tMA6ZOFcV+1dQpHSpXLWZmnx1G9NnBrF25lhMezSxxILK727IZvCcEdWsU7yxeT5BJGgCAwE/3/4SzIZXrW5u4/36pNUhdy9zoet6oc+LhBg0ZYwRbH8zKnSBzQhThobHZ7P45Ep+vPyJiz2zJ9oXKkbhnf8D9+8YxxKP5lmSOgIWXzR4Utam5PqYMb6WOVl0MHh420FcUBESz7bczqywYDQE3uxQCEOVSTVe/Llp862nRXqPZLCXhyy+4ojz6L6O7R+EdX3zbPGeIjOOidxbq4FpczLcHAIJZQN1tA2ZuFM+TZHIDjJ80C5VnKE4/zGPPHadv/X2CmE3/tQ+SH1PGc9xA7WHGEWO5fC6Z7KMzGKzSo86Lxm2GtP/X01HV3Gj0IHTJfiYEdGBwOG21ypfcy4lgUlCWOBmje4RNGSB4Dig9GGi9QF0ELG8+Q5V8FmeblrKgNQUuPwnRD0DofF49m8iSS30pV8jZ3/FSL6FZkbMZUxBaWnH9exZXhvRA7NpwAm+E0C4rY3l6MGHpI0j6xg4idurr0fqex2DxIerqZoyBWqzWp7gJYMqMtcSNKuJiSzcnvE6gD5KS/3J3F/KGr+B03AiaXU7ztln6Dr2g1eFUG8zvr91PaZQSVkmfhcFjDCd/bUTl1MDSpT/xmV9fid0qyoiZdZyrMXm4hFUh9LyDita/SfOs52ibD21ubYgWKXH1S4cnf9RU4q7cxL0tAegU0nluXI2FS1tTGLHsFxaNvp/Mxu/BQVxvjZecSr2ShLdW4tprDkXjx+xKJULSZTRRRaS6LaH32JSVxrBvY3AB+vkckhrVvpQELKFiQDBdVhVRyh44q6sWF492nvnmYz7PH07+sQ9odpX2el3dcYla+QUWly0mKj/mljzXzTFFcYKTbZ4c/24xuvBYPnRo7CgIAvu/HIOLSYaPsxPz1dKefy7oaM6KBavAxdN9cOqySL4dwWxgTLGtnE8/fy+VgvS6vdvPsv/8JM7tG4ZCVUfqQweAaBAEzv0zjpSsJJrvP8KuQDhmunhrOxWdFL2/kji3VuaH1dPVKymnXAjn8wZIu5yA065ZtDwqTe5K0vzC4qE/Ut95FFdlI4JlOmAHZp/0L6LGL5KOWm9inD+iN9lXJOtDxxaQBTfjNssNq1pKDtR730FZUjbd6jo2FwSzUGkDvOa7OjF4QDy7c+9g0CcBXB4sXZPiBkdwzTedsEsK/A+l4TFBSvLEtMYRka/BGJCAKv4KWo0dgJcJMna7TcE69Bp7W8Dd5MlHAD49vW+ai8jsm4nT4EzOhsBP7bfLG/VxG81leS6+IfUkxh6X2BSCDHP/BLYPyERu+ZRCcwj0iIxecFOimnwcz7qZtHz3GCnPSGOZ2KjtlKkF/A7XUOrahGmSfU5QmZv4JbWW9xoEzlWpkfUClW6SMqdHnOb0iNMs8lsjsV/rOkZW+KMExU/AqzWBWl0F/kIwCPb9zLnSD8/MJCbedRR/yxUcJbd1ghua02mIe6dz/5TDrHjfBl7KvIezL6o/RFqZfCAdg7oD5ttjCgV6gmv9CKnXMqn+aZzcpaSBycXK9tnbSbk2magTX6ObtcgmxbagCWQaTv2bSn3sdV789h2ulo1luEqavBVQ5U18bgLj7jhESYK0ZxmAwtpCuH8Bh+Zto9zdepu/eb/Tdo4dPsCQo7kY59ifZYUQjrzQSIxeTXKGnMuTTtx6jgDuTecw5V9loiDyZM7dtN0tncvikidjlFfTfv4gmraJ+Dxpt1nlWjp21BPf2IlZjKEpqidGqj4APsOpUo3g5HuBmP1reGXpHzzUMY4lBNvkCnuu/+nsPsy/FonVr55U9e39MhdqwdStYG+9jDu1tvffVHeacSM/4YRiKNqG/YzSFdDdMhGwgfgWQwuPau8m51ofotK8GKg1SRIIXQzlBM48Ro2mjfuCOtnc2iI55r+dE/H/1YzLmHOkDD8BgGOy/PeXvyFMv42Rrw3gUvoQ3Hth6InKa8x8ZRO//zSdg1mjqJxwChxq2YKUvkRVDqRLV0WV+xe0GO4C/BnWvZ4VCy8TeM1CeOHffFdtYvYTdqq7VrmInZtj6DYpcX5gI4W+0grnbsoZ/8x2Nn6xEPMfs7BOk0rY/q60ENcng+zkbGr8a1guX3FLfULbdJ5qLy2rI4v46dFMwv2bedlRG8MlmtCmTlRppxhneBqlShrPvK2t5a1okdkpzlyQ1xLT9sCtNJBM2dt8cfY5zqZs48aEd+ijLOZVB4nRc61riWoxMzy8kHWTb3DIIq2UNIVHkTFyIqsHfEg/zx4yLulF2797Hp1zSzmhOfuxDnED7PONIAgs/3U5ATUB/LhyLUI/6bMSLCb+PDKT9OM2vCTeXZrk7aW/wfs/PYS+wp/j3s2EzLGvLWv2zsB8ORm93w40rR0oTCsBmwKAKAp4R1fQbqmg9vwwqjukzyrQOZS4phMU+L/Pg3U/MUQRCm03bJXZoQup9yjkxH3rGBrUQo65i7rWHBLUC0FtO75W20ne8BziTg8lJWsMvYeXoZCBBi21Otv637sXss4ljdMpPrgMeJ617XIWKGxEzDVFKr9PfQknQskMzuUtM7Rq7OtdkKyOVe5Qc0HF29+vJmx5gK3Xq2jDWJaamrl2ZjoWUYeoLEQlPw/9P7D9Y4VW1w5q/Gs4+nYYTkP7gBfcoYmD7eCuquK9R+aSa41nd/cZDnYMJV4UYcjv4ByGrAJSuwSUNXG01XrykncDDy2ayM3mAp7WBszuHbg7GzlYeD9ZbXfx7WJbPFXQ9ho1uXvwTgpA7nIBP1UromUI4AKdFXzW6cfO3dMp7FNG3fDz+GsGQeRjEHkPp4oO0VzwC7E3YjEH+VJGKl43fTIAiwmhs4KW/mk0eQ0jvDYKZ2tPrCNCcOM9BHMPc4cc4JefwnhuuI4bd/asmX5jmVzrQX2GF8lNobw5uh1ohMh7b+2+UqOkecgV3P99hEPFkXiNsN1rQaFly57BpKan4ja7mUSDK43Rdt+oS/Dn7mpQ6VW4r59GV3QpvzjIFnjJq1gW2EnJdR8G7s9EpfwDR7JPKesmyqghrCyMhAEvY/63ABLt+3+uaBzV7hupGHCarxGo1Ej9omXWG6y7uhKupvCtUsa8u3riU0FGuUVJ25IdNJf7EFoWSlNyz/tp1vOV8zNsu2clu+pexCvoKLLI85D82q25arXnDRTAvhHhvP3EO5zhY8lxWzt+o//1ZjQJBSydeBiBL7lJ9inMHfjtmEx4sS9bFp5G65skSZBtaVzLxB1+lCen0rV8M/4yqa8J8EVmP4ZcC8F/6lGOiZ4Sb1EpV7Dw73o6g59n4gcfUKS8CmxDcI3h2Nk1sOEAsrZ/WNB1lmyfBQjCGAAMylDGG5qQ+eh4qM2VbpVe8s02iUHsn/Yp7a4CPw/XU2OWSvDfqclmqs6f+wc0kxv8OqN6tew7EaiDpX/wWnwLS0OaqTUtw6PXPFkVWEVxVBeXAn7iEWm4BMDp4adZeHgiQwuTcR7d02ZKhKrYMWjGppK6/Vc8IyqxdD8JpGK1gpurjj/f/hyDUyj3imXonOUEhc9nzdbvGe93hZY+59jkVsrd8efJSHeiIxS7EqfMhbbgMtqsCewdvZxOdSH3151m/J027DShtorWI78SJq+gcsBr3OYU/Wf8t43/kH3/Gf+fGlXOOpo07WQNFHDqNmO22vtFCYKV6IHF3AgZyGGvZ0juJVlfZQmm+Nv5dPQrROGiJzxRCmTovCeTNyiWuAsbSLuYZgMg2wuhI9/2A6uZe3bPRN3oydplv9/mfDVq5pA/IJr6utEMy/Nk/Bzb3wVRhld0OUIAxG7eR/1VM5Z74pD1SNaEuSQx5/dtWEU5tbMTiYmTgjv3dSoYcagf2e7zKJ8mMjUiCQI9bFKGQFjhOUYfSKPbvRXdoHTEZ6Tn1e5roMjpGisuhaDwdaM3bOWjzSZ17GWuHB1ASlYKMlEavE9rVOF+cCnORWFsn7vz1t9zhQRqj3jiGVrKplnHCDDHS+QEPWqcGJ0zmu6B2WR5NtIpl64+c2o1qDJmkVOZyhdJah5dYM+CqW4roirxKCaDhnu21TFgoFIiGyIIkDnuSYKK3yTu3DoyigXGslqyf5PKgs5ZR12dB2vrAiXZxb4aD6YWxpFzPpkPpl/loLrUZhAVHFZ/SsNsD1oD1vKKBzg7SOuJciXlZb7EF0QyQbGKqIhizNYdyHreQRFb025NnRs528JRKSZIxAKtMoG8uDz862IwuEoBFoAMUzCHn/2IYDl8XwbZvYiWLPzJrYFw4PvjiUQNGSohbzd1jebOplOMGHOMvj5G2kRbNmCd5wzmn/2Uym4Zd4la3vAzo2w4Ddo5tvupFLkRdwMvbX/uDG/FHC6VC1THjSF4YTEe7S/j0RCHXy/1mZM6DdXWdn73g5TefYCAC4PSOTf4BGKAAU9TNIm97NOcX8Z6rYuhGe8zdORxnEwtoLIFL6IgcrXfVUSLEr/WeTQyDKpsGuxF1mV06IwIgoX0hqmYXAdIqk7ShYNUeJ3Gq18ccVHF+MuevWWzmHTc9eDP7L8xB358g5DUGsDeE6G+4TAB1SfReSr52+dXxkmxHcp015B1H0U+QMa7j+1np9C79gOsKjWnJ7jyTPxaYgMrcCT7LIIM17hS2gnDpU7q5wyo/oja6laCGz4gKi+ZxlkON1wQsFjlKI0BDH84G18vKbBUZXFnp7IPJuEkr+7uT0t4mESe9Ct1FLl9tvGofxB3eGVg7J4O9Lc/K6UTVVE5xAfDZDnIejWKHqnKJLY0kt+PjaHTqROTILUH9F9Bd1AronsxS13hvNlWcbzLZTyL0wuxima+nLMVK1amX7cfd17UEqb1q6ff4eE8H2GiJkgKVBsvJjPmyGDan17LXRFVnDI6AApyFWweS1qtH+cmvkakR89cs9iWnRxScYSy8eW8FuPC293XGO1+e1+nsZ0LaP8nBXiIHSvOSx6IWl+FuH0SodkRDP76cSZoDdBTiVzc6cOEr1dCnzzWLVtHWk8fTADMXawekc9uzy4e+PEB8qPzER6w77fbOZJTOhFrq5YZh3/GPU4qn+tpreP+2AYqtnkQcv5zjHOWA3ZJ4gRZAQs1Trz3xePUeTQhrhJg9I5bdkHXwJnUK7RnB9HWPuE2X9oiKBi6Zwr60ErCZx5DZbJVq8sESNk7GWV6LD4fP8mXfkYy67wh2Fblo7R08NKnz9Du0s7uiQdReEjB+bGBWcx87ir6HE9+7w7nsFkKQF7V/kKYl4KY6Xvok1CBILwvsSv9VzLFeyeDjk6nKv72YC3KL58l6z7mtdK+5F27KpWrAi7rXXH1ruXGxCOEBNulq0c7ZVGxoJ2LpX6s+HUFZ6ZJwfsL7X7EN7lwfvYpgucfJNpPylzVp13DMPAySxpBVgTlDiSlq3sfTg/vySIVBAL9sfFpzenQWUGXMJhfn3uctDpbwKwdcxzK8qDmIJh1EDoftxYbAP33wt/x1hXzBD/d2r/ZamHtgjOYjcdxbXMlSWGn2INLA+h7fhgN/p34zt9LjjzhVpc3haWNrx7dyxfNEJiVSKxXB0bLo6iwTdR19dt5YfhDPFftzoJRLeR0SSVXG/XPMGjkDo7oC8F6e4+53NoXyHrNVoUoU0jlFQEu1Z6hRXOA6IJgxhwdg9BXuv3uzqE80HYQgtr5tNXCNY/BUN9MmXoof8+8j1PtLjxo9GCEi9mWeaywgT+iBYafGo5/iwc+lwbS955Tkv3WhS5GNjkf6twp2D4WYYU0OWfj+d0U36gjzP0wIe5SyXeA5OxEEq7HMHJsNMGRw2+zG1xMZCVnUR9WxpnQREY7GtW+HKp9nZISI2OVRfQP7QEK608iila07h38K/ihDU7CxVN6P1ybnQgtTuPCpENk3LmZ56zStfjlHWnEnk+jNKE/+Eqr/i6b3Dk+oh4nZy2vjs6l01Pa9/SSbBBf3PUcMrMMi2jhVVHad1wwtfDRsqP8POUM9zaqeSV4Oo57MMq9SM+0gWTRo8uxyOwVRjGtryEnl9Ajel7Y1Zfw98NuEYnRQfey2/cIOqdmhON5dPWRVvcLCLzy9it0OVmojBvL4DDpde3LGMKmnQH4j3yWu5OKSW/ZRrRDPrcmUMugt37ERd5Ni+l2lCPYOZofJp1BwI30jOcZ7iytmJG7OVMb5E1L+a8c0+hYqHQHwMN4htS4Lzl5417cnd152E1aZfyou5nYlAMsM87Hedxn1DrdKbE/1dXC2b19Kew/n9dr3+GzOQ7XLAiILvfSqDlCZOEWrNoe8qnxEpi7kJtE/vKHw50wqwp8VdH0yvUg0f07PD7Yy7T6Wn7s6Eumg82Jdv4d8TnrLszG0rGXeW6F3CSJ9mjduT50B3P2zsC/+BzGpngcAfZ3UwdywWcyJw/FU+9TJ/nmjUofZpeLNFT48uLXD1E97orknLytJby7+S5KPRopmrUfH1FapRkjK+btbx9D697BqjVfcVG/BK7/BdnvwMST4J7MypnnaBt6nX6+hTSbpHLbCoxM32uT/Drr0shyuXvP/QST0lYRl1YYgb9bJ06GMujxNhvVKez/ZBHezZ54U8/w+4+AgzpLqGs9zqOu0tGlRd7Sj4yWL7lbfAuUtvtyn1szOuDY+KF4t4/tXexF9NjLMOZfPlB20GmJchAqtg3n5hPE7sxm4UhvFIMu0nv4nP6LIUdr8e1/naH97OulgMhnT3+Gc7srm5sjmRQmJVKCBrzGHX2refuf+yjNDCdFJZ2/F6p0GNu8aDuTh9HNDaxp3Dz5zNpMLvT9Fdc2VxoX7MHZ1BNNpH4GGn+a8uvZN/Y7xhwbw+q3VtP4NbY5uLPCrvDSpEVpVDJrxnlaXaSV8AIC9+X059yvs+kLJG98BAB963UWDNzGxqBtxGy8j86sEGpaN3GT7OswtFFZqKEjU8mVvEe4FKjjyXvt37XK3MZj887yUkkgd/5USHR/FxzTJu/q8yLfDS4lK/EY3xdDZP5G7u/JAxRFaNQoqZ4wiQULFLhfr0Uu88IR7iozR/GmUw0+Vb7kaLpRjXeXXNeHnjs58FsSRekzGHzvTpyMWUASBcqx/FJ9DJPVin9DPyyiHI3C7ru3KEdSt/08Ea7NPDcsjzBG8KrDfk91juEh01oC5m7FWefMYuENyXErw2u5FmlL8HQXwaUtB/RRoPbBonRj175BBO0dxf53v6bFpwscyT5BRGeWodO2c3zoN0QYZ+P4/gvu/fjypWeJ74wjd+gHhDq83z/XvMdV9z1EunXTnWTk+ybHmQacYg9Q9v5ZPETIM4JVkEKHbv778BFb6E6/gyrfGkllv6wjh7fiVpCpSWH0jENkujaBg2Srr6EM/cDrdO4OYOVPKwmbIu0pjCCjyMve67qfukRiVptb8dXquZ6Yx48T9/GGzI6ADE8q5brMQqtzBP1VOdA1iJtknyDAmWAZJ9pdmXL/WgSLQoJvqM1NvJ+2ih2FrmTtnIkxTQkWg03G0yWa2dpNfDuujOXv3MG+/Ggi90rXnBleZ5j+8DkeSnEiqP0Zeg9XUyWvvPona965k4+3LCF00Dkc+1mdbfuba4G/IssL4riziflK757zFiiIKcDDaA/EHRM2w7uqSDg1ih3NMppcojFbpVWYxSGjkT2VRYbnQbZGtvNjYzNcOw65nyJMOMGkKRdomXSC0uz7sGrU4MCtapUtZMj1HDWc4AMdBDWFEe+qgEhbvCVWwRMtMjL+tD2D4vBihMX2c/MSG3n0vR957vwIfH/8jo4+w0C04SSd3v25PLk/STF/Ef/Pb9w36zyy5u9tie8qTyoC52KYmkhz3zOsSznI382CjXxtuoxXSznm+fvYWFtMbN1ZAFIcX1F9HeyMIK2vJ37/rOLhafvJdA4DbM9lcdKbxHufJSd9FamXIwiIfcq+rcqTKwYlK/9eiMqgYntWLA+99z30s1eUPV4zhqvmjXgu+gvBKnCn5SvA1l4lvX86pWGl9CmcyvCsaUycvvHWdjJRYND5QUQXROObH8uFXpV9gcobPOui4LlOF2q6lLTopYl6vupSfrvrLG/4tNP5+wJMwdLKoTK9B+O/fBCT3MQv9/1mS5h3GD+4xZJz3zrMWXG0unXY40eFC8s7o8lsziTgoZ941gMUhcHAJBDllMqWU+GUhtktktyySD7mPr5xCNKc83yo2DsRY0IoVTFRDHN5FjLboI9tvktJ+4ZqbzXngkrZJm9lu0OCrE4dw8YCH9ya3MlK+oMIoxRwihRKcZ13jZ1CJx8KHhwWb6+y1+hUBFQHsGvgJbo6I+mtDRhzYQOaLlee/6mL79rsc4smJpiqmFF41mVQcsEHryWlt+6JTBTBClarhYJnvyVObUGwpW8BIIgiFwddxLN9NKGiM1oHdRUAed0z/PDuDYKHV+M025/c5peY6ZA52elioNqlkBtvvciToZU8N0nqgKRYLjBw1WZW1UNrazrTun4Ch1KD2R598G54A8ulLgZ7G3Ay2TBPZXc2SckZdHoMxzz6HGlBbYjYYmarFUZ2nOG5MBjVaGSNF+w2tWBReKKvWsmo8If5NbCRfdpGBLUTaYG7mNyxHMp+gOw3sWhj2Or0MUKsnJt96Z26K23tVfzHE6ANZeBpPU5tZgaeeoIrvaRN/zP++8Z/yL7/jP+xIQgCISEhkoy+xeVOCBYDaYlziVNb6egqxxtbz6lOuRfvql6gOfEc7ZoVdMn6AE/e2rZD5cnWO74mvGIUretWMOsdqYTimY4d7B39AQPGlvOYqxnB+jZELYei36G9ALlFx2ylgqziCNb8sRzP16RyV3ub03nbZwXzy2bwhKqF4PaJwDOICi3JPQ7RUynezEj6GEVNOhhtjXaVohX/mgKMCpGXpi6i1BANZMCYfaD2RfnFr9RkeGPuYybXZRSTesn5vjT6Eu0xBWR9s4j42sDbwLg3fC8R0OHPwb+WcHZ64W33uVU7njNBAp1zv2RSdDkySydg7zlxV5M/p2/YZE7GWUbZ+hDJlOQKSfy46gUsogV9G4QaQiXO8IRmPbJjY9mn1rO8yYk5I6Mkx52jE7l0wZZdWCDEIcjs043aUM1fjxzg3VPxdP/wA51ICYnuxmPMG/Q310KymTZcz3Xx9uqPD3xyyBrVxPsDLhMkl4J1okxFUeIsLO2dfPfCo0Q+2yOhKAjsttZQ7nuBQFMpS0viuZFQat9OENiyYAsAr3/wImWiGwZzN0q57dybVLE88O1U5CY51XPKCIs6LDnvidp0hj6/jxdrNSR1plPRYgHsWuU3n92YX5dSEVyB8KL0WQoI/Hbvb3g3eGNQGnjcsS8O4CuL4PGvniM37ipL+//L++pXmQyYld747B9L36pQIir92RtezT0OBFKoUMkDrrCd3axoHc4Aeauk90aH7xp2dsFQ8WfiNe+hMq7GsdfPb1vH0fdSJIdVJo6kZfJWL6RjT1QL6W0iH+ZGkuQ1r1e7ZPC58S39DzagR0ZW9RhGKeyEhtzQQGK/As4mFKAui8bXtRRml4CxHcNntaTuX0t11HC+bvuc/mPdJJUK/euVjLwyjaP9rnAqJo++ot0ZNijcCetsxBLwK2/1X0B7uLTisKH5GM9+9ikvVAVhuvoUZhSA3bEcIZyn76EZlDY7syD8c8ZGSkGWmppzvD5iCuv39+PwiT6c9vSWOJVXVJOZWraeoI7rPOg7gMb6QGAXAIPy3qcty8q/C5qpTN3LIIcKi8Lmc7jL36DN4M0XZVuQV8E2h/2ea93O6fgHScpJQvHPQioWNeA4OqwqaszQqlVicPPG2quR86m2OWR1uLEh9gzD1eKtaq+b44gxjTOfa6kOrqDfU38SxGwcKxpH+WWjcjZT+tZyxvS/ylOv28C6VosHfX+9k1ZvOUX+NsdPwL7vs42+VFRBrXs77wdk4quaLXkHy8KqODjxIE4XEph4OZYnP7E7sqIgkN4/HW2HFjerHNnNXlbdjYCA0imIU2ULqW9xxphyjVMGaRUEQHNTEZBCm3sznl7tNk+3Z+1pDJjLW3F34+ZfSHa7njxL4q3KQKsMDk48iOhj4GwwnDA49OwTlfwRXMxV3wIG6OXMC2xGtNi1/K0KN75c8DcJRX1ZtC6ACK9SHHvfeVlrWKFy4iNNDUaPKqr0/8ItGgcy61IoXZ2MAjCPvXbb3C+IIk/Uw/QFu9jgdZr69h/BAb5XW3X0y0nGya+R1/rUkGNq6tlOzh/xGYhe5XhvmcAbrp1M/8je8yxMacBt8HUuV/lwzx/3cG6itKpqh24AX7RcJfZkGttPDkXxcYvE/v7U7xGmGehXBrRB71VpqO4Mhr2TuJwVTZvy6m3PSo8TF7uh0aT4L5MB119LQlnUzA/javF3SPr5pnsm//b/lg6/eKaElzM8QiqNV5rawL7k9fQr/p3OzHsYL8VB2JuZR3VjOzOur6HOtw7BIbHG1vpJoM51Pw8NupcmSyJYD8PYAyAquLHvEOVuxwnQ6Jg2+RI56iHgPwFqj0CiLct97tgrlI06SZTZgp8sRXpwwUq152aiFXDAR6DUpWe9NBsYZpQTlB/L2SAt+1wm8rB/P/t2Cg8WV0O+EUKjr+Epc4TSoMiiZceLK6kLrGLxE3vYbJWC89Gmk7hfDKLxehqRwfW4JkmJmnrniVTFuDNu+D/MSn4DOCixe7fuY+kLP3Jq1QtM6fDG33wDxz6J052ep9/TVjL6ZLBt3jbkMhm+ISGkNzeytQPkVgsBHz6CmFSM8JS9v0aEUEHEoYlUhZVSMH8nOT5BkgzfRtchnFGm4Z1zHlmZNz4rI3AcTec68U6/zuvPrqfLyeQgIg4YWlii86H7eiKjx/+DDdGSkg6Tw/KwPHiVjVe+YYb8EYlNZ9CRkXk/ww7YUk/G99sDXADPVC4aY/n1ehMF8q2MHruOkvZpwJ5b2z6iN9C6dzodd//DR0Ma2G3OB+wV6yeTcsgJqMA3qAZXxTXgRfuz0FdxeMz3uOkG8taJi8yYgWQ9FAWb1JtMaUIrgEx61QhyN57u7qD5WjTzKv5g9EQp6dbhHM9LK21CfEPyjvKeUwrhPbbH7n+Ngob+HPh9GPKaCpJD7U9DKVNQGzOZNnUG2ybfi9LijSO9HyXkY+xTQr67HEWAJ87+0jXnZNMlUqrdaflzJp/uamXoAemald92kc+vXOFk4gCcTD70pm5HaCwEpcj4sGwbmarfqdFdws3VDrZ06p9lZp/d1Hy8hAZP+9pQ4vwC3z8SDYo8xj55hWEhWwF7z/fSpnEc3O2OxbiU48pQ5s6VHlcXOJCYJQU0HD2Ca/W/tBtW4JisMSf+E240daL5+R6cx6Tb/njAlqDlM+kicVf6kfv3BAr7XKFhSq9sI0DefYI0nwt0thtptUrtVpkTG8+9wI29chSF/dnot+1WXevNdaItycjcGQcwhTsD9kSosrwgqq86Y372R9YEtdHYfQc3q+usMmc8v7uH1NJwEsZcol9iieS4YQqB1iYPQjuceKElghNO0kSPAFkDnnEtlCm6+LkV+ohqG9FnbIHG8+CezBvaRkJ/m8wfO5eyaMv9ONa+Djcexev+Cl7p1tHskYJWaYtVWtoLCQt7mbIOf64/9zOhLl2YZfbkKkEQOD38NEFV8YwnnEqZNP4bobzBHW7w8vRTvLvsFN/pl0LzVdjXH2KfYEajmrJLE7ngnE6bxhuTJZ6b1QYAc/waGObczvh9BxgVJFVLALior6fNVU+HUsC9adpt61ZsyAm0Q0y8PeQwP4pablKo08WdvBwB/UrbuRh2FY01WJLg95fRk42yLPKTQgnyfYTUXn02Xz0wkrRdaQTQSNKQjxBMP3CzB5Kys5QDy0/yQTO86AGnO3sSW+tPgahAEIZzcdBFYp1MjCmLxFMRCO53w/hj4GKL55ITiih+4QJrnHT4ts2RVgoLAntOJ+IFJI/KItL6PfA45qDZuP21kk4LxIyvxjrrN6ap3rFvqHTloUnbECfs4O5tBzCp3FHJ7b53s1MSynyILIhgydE/MIsTuam0AOAmZtJ/YC6yE9G4VidxYbKzpLKvK+wFWvx+4frPc3gz9iprW/aDg8x4gzWQn+sV8MzHGBVGXjBLI5YK5lCUbiM71eGVWHvegzL1UI4eHMiE4r7EXv8Uj2g9QscK8LQ9lEst+zgzPB+d8011ldvjO2W3kgC/Nl5xSsHDSSoV+5lvB1UmDR+WKhiltZCccT+of4Go5Rhck3nHt5DkgRp8Cv0wylolCbKYu/kg90PaN7WwZM1PZHldxpHsI+4xGl23YpabGBu+DpVy8K174lrexSPfLiNj9Fl+swgUaaXJXUtdixmlgTvfXcju0giSd0ozrReEL0Me3s1f960hM2g6E+2tNClqLeXopgG0dSoQx5WS1XVDohTiZqrnuSUn+DqwDrHBky5hgWTfgkzOUynnmFrpxuBzQznQmobjNFzjMpgVi1+3/97hw7srKoeu6It4WJ2J0uhY1LWSfrd+B99+/gYd1y0oV72NSVrYh2DVc/HLcJRVPgwAuuLywT0Z5lSAIFB4/izftYhU+9fQaYUoYYpk+yxzFJebzvH8wCO4cQkJawZUOw8noAgWWq3EK6y4maVS4WO6PyBhTxq1xUM5NtquXhFruMA6P/g8T+SR716kY+R5hKe7uQnn5nXEUPtPDDdGnmTvI/eRJj7AGgdFJr3xIy7mQ3HqQKZVXiZe6APGEhshJlNzn4eeGrOFMwN+BLQcLnuB9cYO7lqko2anJzNKZEQUJzCpPISyaIVEQlQmgzyW4j2vk13CQSZH5RBpigZs82WNMpqkUlC3eTHMrYjagF3UNK/F3yOKdgrRuZbS3FKKb0EEG4preNalR35X6cEF02oyZVCh+JW7a8DHnMqivC8h/xuSZS78ndxOhNCIsbGl52zcuPX9KbRstoRR//lc3JutXAkdTUtfm78oihDoeoOaw3LKTqRTN30a0YEXwTwSZEqwmHESBPbO2E1iTn8M/bI51TaYEXKXW9cdUzeAsevCOTHyIJcHXr61/soFAy8NKOecvpxrXiOZMGkjblo7oacWmplTFUF3QQxvv/I2AlJiq800mzVrzXieb2DtyrV4R0tVM9oMvrzSABd9K6mfvZ09XlouONi1MiOCCCEVIUy8lgwWs4Twu2gN5kjIUVYkFfGIIgjR1MzNpCBREInOjyapw52n77jET249cYGoIF31CwcLrPjJ6gnIv86DA1bDKX8Y8RcA39TF0q/cB72zMx9u/IGPl09G7mT3x56NL+RChJkfKgE94IBBiKKMdUv/RAR+9oVCnR2fAzgrDGXLgNcYuHkOX22ZjuUbmSQRBGDJ4HQa0nZytQnirFJc0mg2QlIuzc0V9D/rR1hwkk3CX9+Ae8BszHcEs90SRnHyedyNK3ji5itkbuGV45NRHBtK6DufsdynlRwH/zpSqOBkoIqfr6j46fGlmIeOoJ/DslKvmUVl3GVczD8z90o0+VOkJOXfQfl4yuClpByaPJsQe/V186aOUc6w90g/XDUdKE35OJJ94eZjtOdf5N/BMj4afoRVqhcAsDZ/wJoRfzKsOJyzQ0t4Fzjk0AD9hjyGvacaca+J4I1tD1J6VwYRE/MoNHeS2W1X0q3K2UZkwR6M/fQoT9nm6BplKEf6RODSlcSMnSNwaRuL++QcyLKRurLBu7njrl/YeXEG3RXdWGT/57Zt/38f/xXf8d85/kP2/Wf8jw1RFPHyksqaDT2bxrgjoxHU3QwdfwlhfsctW3VnOacThuFX48cHF0aiH/wXjmQfMoHrideJKxhL0I3jmDv74ygF5Gk8zYcRF/GRgSi4oJSpoKUQqvZCyhqsoobMxVtRW02k+XbQIkgdcbPVxMCLySTuT+Cixkj4Q7491yFw/7qf8K2opf6lz0mNTacw9jOiPPoBoDBWsPrLl/k9ZxafHoqmO1Bpm6IDbc5heUJ/Lk2z4tpSgk/5RSzdfaFkC5y5E2KfoMmjE6NHLbvffRNfqyuzek0WzZfSuL4hku5hFwlNlGZeAvzTsJm10av4xgcecQeTuQtHsq8i+iHMc40MjD/MrORvwLgaZL7MDLyf3L+XYu78nbTjtZydKl10Bw3MJi3lIhkbhzN1/1SapX23+TWukbbXPqSh4F08O+OxZX7Yzt0sd+PdJkiXGYkJ8MXdQ9rstqh0PVOdd3LQ08JjHg2MUkgdaYDYEi3XP7ufpDnbaOlVTSCIMk7W342zRzluMQV4uNklTfLEn+nQZJF0cgJ/pw+n5eN2Zt3cztJNYThs64BnX7DBVfcK9uoLAYHYG7HonboYvWItNcY0HEEUuWBFI8BQ5y5Wh+5hQ0ckjmTfE26VrKmN4XBhNGbnztskFKM6T3AwGB5WNlBovL3qaqFrNOcs1VQabAT4TaLGXaEioiYFn0o3zg44Q5trG3d52XsAxLYVYHnpZSalXeLPCQeptkozxG69UtcLyasJIPjRMhzJPoPSSLtLO26VwWRU9JYiBM9ODbKDg3n44AiUS2/vfRrheRjjUAXrFSvpjvDlIbnLLXEodWsWOwJhWQ3ck/Ym8QoZyJ+Gss2M1rzGh/1HUCtW0f90IJc9WwF7JPmi0MS/5+eSEVpCTUCNJLYWRRkGgxzRIvLc7E2oDYESBbo6VSKDmjpxrzQysmYp+VVawN4zqkHwQT7uBFWXYik5NoC2aGnmphm4tGM4qqODUAEHR6glZJ8gCLz25mvILDKqgeQ39t2yDRl4gOvqZOJ8BvLziF/IcLYTkTJ9LoPOxdAkWLiUau25JPuFpYkXefOdV2nxr+XPu/4kKFgqaRZRImPwvhXsHXKOzU3JvN/HhCOtMD5vKWP+UnHeOonDMbF8dLe0UqLaGkDW1L/pdOrkK28jxVYp0bOsbCj1beeYK1gxW5XIe3q9aSxmIosjKBHNrNgzk8DoCqo0VdDTOfL82X7Id8fgs+g4211hjEwKKOwfVoPVauKDt2bj0ewhCQBkxib+nZPJr+UuWD96C+8uOfAGbOnJdl1k5fffnyA49zAfBzyCduV1CSgFcJ+LgsanP+GHMD2feBtxnI8EUSQ3IZfYvFjct0xm8xTNLbLvJb/jTFyWx8wjM6nbNpXIYXZSDJmSXyon020ysGS0J0P8c7Fq7Pabjl2TVxMTVh6jzXug5JwqZbG8/fRjiIG1vP3RR3xreABHsq9J48GNgQdY3L+eV0ZdoUXlBifmQMUOmHEDFF6MOzyOsCZvrj6xB3N3I46hTaj5Csu++JJV9RBXJPDXUBsoJYoie2OuQww89enTVLi0YVDbIVc/jzFcTOykKLSGB4f+TZi/tAqzttkHzd4ZFKi6uTRtD6kaacXLxS4NRsGAR5MPWn0qSpn0Hdt5PZ6YrGi2zNtCR7wHn/R6VlXdYfy1dRBzRA0+fUcBJyR2v1IPBh8YzAYPHckpA29BWgZZDLVtccidnHhw9nmO6KQA+QJtOREKWJurR9XRgNXkAQ4ydV6lBlwKSgkqSKba0qv3aHc1C3zfJb0M9J/OJWhqOmCF3fEQegeCMIENd9l6l3wCrBQfAJ+hMOHorV2s9OrgyKkk7jMraR5yVbJ7URDJN4eQ36pE5iHHV9UjCWhq56cZp9kXn8jliz8Q2iAQ5+KwodyJrPOJuLa5sn2orSrgK4d1RS/3pNUooioP4tFn38Y6LoiFDj0UdzcNJegfTzyAruIwQu6TykBXqK9zaUgt81IuoFG33QZiO5ma6KuGqx9+wPOesM/aR2L3V1ymEuib2ZewUZdRGmvw8h+ITCYj/no8Lh3unBxyll0ebdzpkATSV5VL2ptZzGvt4oJLMykyKZn3a9ccrg3Yg0uMCwqjgjWCtOrQlBhFiWjGoDRALylCjO28eM9BfrT0Z8vXC/B/aiJS2gvudfPCqdOdwoaxePllg8MvLLoSFov1lBGPIsZCrmyareLcKQR5B/h98BiLAqoJ1jtjnCp9j5ITi9C88BvvNsGbl4KJmyxdV0xhVZSHl/BbcAbOgogj2edqqeMxNzjXVkh350fkNaZws1oGIFal4b76kQS3t/BATCNHzPmSfQsKZ9bVqXjpnzm4xf6Bb1cqOPR8dVSzOBc3loymnYzpgWY91NVoVWFU9L2b8j6wxAGXEi06+vidp9DURbe6G8EkJRndaOWOpzezpSGYJT4pnDSvA4faqLqkLt6L+Y4Blwag6lYxVHavZHtrSyvRV3NQVo3nRtLtlcCa7hymxXzHxddfZ1b/ayi7SnDsW+lck0PT+ok4Tz/Kh9POQVcNaPw53XKCWs1FZKI7n5z9kz8LTHzrMJVVFAfTuOc8ysQK1N5qMHniGDZnWB/jcjuEey/Ft0NNXctpHEnj4OsHaDjfl76L83AO6nkHFe6g8kTjNYANp9/BufUio06OInuk1AcGeKjMj5Cv7+b18lDKVh6V2MyiE4cOpBBaWMCBSQdoE/xukX2vtbhRv/sh9iQX4hx8iQsGNakOfupnJwYTd0FN+yAL1TIjctGOdIuCQFFkEQ3eDXQkT8DVw1WSlCO43snpobMQalez7t1ldL0inS8umQaxYeIjpJwchmzTaD6Y4ALjD0P9aQi0tV7Y0a5innM7XZozpDfFM8bhCDWyEApTd7JIH8F2rz00mWsBb7rbshhwyBnvShUXpt3gT6OVX+bYSQeVpZWwMZfJ1RVRmf8zvnIXB50FOFWdwtVPh3B98EUWTThDiGsE3JxvlO58kh1P3/1DmD/kBBOHf0+z/gj+PYmmAM8XReJS18HUDWcxDlPgWIUDUK6Ws27GJpatW0lz7mxGS9vvsjE0jEz3v8iiCTeD962/63DlbGYE3m1G1lgqCLIE85zDdtd1XhTptESZ/muQqNC7A/2Ik8Sk5hEbqqe5uwNtD9lnlWk43AUlLRq+OPMQ6oggW7e11msgUxFrVpIbBpu90hmlTeew7FFbVd/V5yH2CYi4i0cDKqj+ty+/XxuC+7Kltx3/15k7iEPOrKBkTErbXCbK1ARlJzDqxCguTGghPdrMOMEuf36z/xxAVcxQZFYpyC3KZCxedzcWuRJ93xg8wnsBn+1/8uywz3jsyhJSfYwMS/gZUbTNU6IILwefRnt1CIfX96Mi3oPmMdL5SC1qefLzp2nwruPk1AA8/KVSy9eFF8gbUszxlIm8Ti27NTYPWiaKBFcEE3ctBNnARob3OYNeNw5FD9mXaPieeXfvZNnxeN54/yVUc3Ik+w2UV/Hc2ocQVTDvoy/Jc6hCAxhUGcLlI6k8fT6YupByyjdrCemJ4yI8IhikOoRBvpv6n1vZteykVFG7q5qZYd+wgbvY+Mb9qB+W+kybczazed5mZq+PI/RiLprFOm6SfXeEXMFzVglX6zSUfrCSvIXSBMJfqhL5ucgNF/0AjM6h9PGUEuk/HB6K14bR1Aa50xYh7T3XJPfhu5GHkJll5FULDFVK1/FSp374FEF7UC7dgfCBXFoFd5M40Wm66NR00mZ0kdhvxtATTFpmO4G7tRywVYtNra2gslLgTNxIiq3ZzJFJt83VjcJFnc3HxX0RPaVJUM2CG8e93XELzuOrlVvZ1tWTRPePO3gNIbtsJH6vrcZv8mHCHt6E2iL9Nq61xGD+cQk/BKmITBnq0DnXNiyiGm1uNP/2SeerhFzWuZzAUQuoqtuThmKb7OLytHxEYwPgja+lhNqPVjKuVYtzRCVzw9oQLJ3c7Ffc6u7Kxoc+wKLV8WegkeIuKalgNRhQdbSjvuFPiSUB/1QfiJ4ILtHgHMbyvRPQ1ph5b9EpTJpuDgMW0RnUzliB/ukpTD5oizknpZyQkOyiCKdaHiEoCORh3/JUWBWnrPZ+9BaFMx5HhmNSu6B44Rs2+Vqob80Cjyj6ypawyK+NpRaBy6/k4tk+gmedwyX7Bgg8H4CxbBHlS0IgbikYmrjmPJAP972EUSeS2LaQYTHVWA3rgR5yTOHKT8QxqE1Lk7fIr8JqwprsKisvZ9+Pe9uf9AvQM7XPL8xI/h5MDSDzgqq9lAbXsExew8aEKgI0jVia10iI9uUe3VxWaInojOZebTaCzFadKgpmHnYH5zY4725gmUspdykeu4UyqOjg2Se38GmjyH43E1pB2o/VSAClTEUevw2dsw6fXskDnSYftub54FfrR2FcHlFe8RL797F7cV5pZduXi3jPXQRjq6Rnn8wkw7/anxHGeGYkHqPhZqKquZuVylryr47GPSeelW2fY/BPuKU88HvVcyQcFXDu7FHhaGyAcrvyRvzQu9inrKDP+QY8rpzi3fkHWB1tz6ycURSDrqWeJ9bdycXB5xEessfibnIlM7on01RVSOVbS4i+U6rooRe15BtBGZ1PvXMHMaI0IR5gtkszJ9dPZvXhwaS8JcUt9SY9ax74gL82jsPn64e58kKLzWDqYI7mCvkBVzjVNIXi0AvEV8y7Fe+oLe0sjK9hV2seJ/99irNeHbwyxr6O93Pty4knHyRSHUSDtycyV+mzkrk4Ux0zCkvePxw+N5HuSdK5KONiAo2/TKJkzjauJ+cg9OLFTitn88YPtYw/Mp7gmHK8pxZJ7HWaBVy9LGDwKuWI2cijPYoIRYpRbHq3i4lXkzj32ltYRestXMJigWJikH0XQ1BYDTmJOXT6Grie/Sn/jLSpgric78/zOyfS7XOZlrIQ9PPVOLvpIPwuGpz6s+5yB5dzoyjXdGGSC7S43iyskNFoNmOK11OiLWSd8zo8kRKv/5vGf8V3/HeO/5B9/xn/Y8NsNpOfn09MTAyyHo8hLbIGXUc6zhf6s63GmecV7rd+LzN38vSJCcgzEilp9MQlUlqyHm3OozICti5YR8CEFDoVUThmtnl31FPyxGtcT6nh14BvWb9YDsIQGPkPeKYi1BbxRhM4T9nJ/NLNzHSWArODzYfwuDSUBm0H3y/dQVf7y7wFCFYzcX5NdHRp2Nxdw84i2JQ68Na01mA28k1tP8615ZPw5zwyJvcAMEcmg1MQGxt8OdD/J545PBJ1Rj8a60+BZw9g4D+OR2r30UQ7w3JPIXYMx/lZyWlRo19Dk2obqyZkkh7Qu8EvWFr0PPHFE+xOvco3406RpZYSPaXiDNKN0F3jTbOYxLI7bItQWNfvvBa/k68ujsEzupxlIVLg6O2ueIosB0hJS0eIqsZXJe2nVtA1mTLTJZ4e9iTTncBs6kAmszkHZqUne3cMR26Ss/nDB/m3IwwoubVtsfNg9r7thrNHMxfu+4UBgrT/BsCqjklEhpXwbJcv/h5ZEluHoYNcvyeRe0KU+w/087J7pLu6G9j+5SquDqjh6MQ6xvvZyRJBECgp9kdX4wL++bdJ+agtbTS+8iU3jFa+94Dt7VLg6YA+ja1/CMQWR/JEcDUhr0gbw4erU/n35xjcg5pZ+/SB267JydyEZ1E4wwvcyZyYS5FsJ46VOqHiZiZ8/Q3PPPU0b56YRFT/dGAhUa3/8vOaL3npyEEOh7xluxaHk8+U96U5rAhVvTevf/oMzfNKpAe++DRDT6g4OOg0paPK8egeLwk8lo2RUzMsn9csP6Ewu9F7PLZ7CBMODccpvIu+4RuAtyX2l1RwbfomSk2bANAZW3FV2Z6p0TWZVb+MI/LwKM6kZVAx5ZINpJGpMXt2c/GOjVSVJzMs1JXoSKkH9G94J6Wvf8T0A2N54sowfPpnQU/zeMFq4oHNi/AoDuOdV96l98M0ik7UVnrT90YM6gv9SR58XmLPF2L5Mvg7Fp1KZvqe6fTW2LJognjXv4A7e+5Uh28vCQ5TIXl9r5FnNZMbe4NLytHM77HdH5FBW9QZht84zqacl+gzyf7tdMl9KL5zCzl6FcO+9aEtsQr44ZbdgArf1OvU+tSwbmwBR3RSHf45yhJq2gfjs+kODAoDzXOlFY2JvnupimijuUhLqyDdFmzfQGbfTHzqfEg+68edo+/nLQe7W4EXczc8yfWRaoaE3Em0k82BmWfYS8rv14g5/hBL1w6gj1KOsW8l9OTSNvnrOD7qOEHlXow4+jDly6THLa5fTLfewpk7duGramCidY7DScm5boA6q4UYr3rU7jbpUGJsFTc6YwdN2nMEA93Vvpxvszh0vbENi8JKu2s7u0zQ3BjCIw7vg6K7kdEacCkPZeC5oRwZYwdpio1e7NcBZX1JPyQnNF6aeTz3+Ouo8nPYNGABG1RafnE4beeO61xtSmBjk4LHhQeJsfje6l8BEOQykIvRRbQH53FHNUR6hEn2jZOMPTP2oDKNpO78L9w7TbQRfQCd5Qhu3kxv86U9J57nt5XyyHNSoqVJFsHrHQbc33yOse4yXFfawFGFTM7okvVYrCLfPvIoZlkbKxygI7Pci6Mld1PnsZm2kZep6pSeV5R5IB6X1ZwYeZqLgy6SqpECHR9nxOFfoGbV0XEoQhR4fS4l3c7EtLHv8S9pd2knQCEFjgBkbRam7ZuGAQjw3nObfcDgbJSdTbTtH02Oxi6wON9rKX4vqSgNtuLkfIUQEnEMifq1Koms7MuU7EKs2d+gvnsAYO/9m9S1nroCK33f+oqnwhpB+PmWrbn9Bs075Xg0uJPn1UheYwSLEcBvPLjEElhVzhpPWN8OBUZs0013ow3kdk8BbQRjc8KYv3YBOTITzSkZTHA4N1EQ2P7XeHT54ZwTRQbvesQmhyXXsqH4N64UxZFYsonAhgt4NaYA99q2EwVSr6QSWhbKsDPDKIosklSMi4LADw/9QELeeOb/E4lGL31Hap1d2fmQTd6426mTF2S/S+yD5ItYMqaYp36aSI2/kQxpkRsFHvO467fDqExy3u2Tw/t9xkjsoYq9RD+1hX/2pfFiUDdmXSG5uVpiOnbwXHYaRQVBvPPS+4hmjURa94x5EJ+rDpDtyn8pL/pYiY6W9S9yadFWvPrcQG2RgnUPTkikcYwVYfMITH7N0pPWBLDqSClFql3oRx5BVLXwIMMkPzle+x1Zq3cT2rGJwdqjwLFbNsFiYsbiw3w25TCvN0MME209LS0GkjQlNM85zr96CzXXoqkwSf2tZS2RVGnOsOT4EoyF0ZhmSMHP3AgDp9tlbHnhYUxJhbx3r93mbSmj79f3410YDHTSlfgrjmTfZBc54r8xlN8I5Sd1Nz5/uUv2LSAw4ZDtrZsxdwsaQyeOZF+MwsIftUvJOqBA7aLDf2DBLduCA+NwqxEIDjvMu2OXY2p8npuykTVNxxhcuBZnsxOzgv0pkUmfVaYwmLvz1zIu35m8MytonX+WkQ5k35MeF5nuDBHiZZqkeBcAwd276L5RRNiNkUQJ6bfZ61zu5Mc5HSjRMWvSZeSmFon9YFcaboZm3J068XUycFNyyGoq58z939Nd14/YK6UUueiBrbe2+1mzmsZF15mcbSH0xL94TkzB0R/bzjJy+xxk1cQaVnvDcbNE7JUNjXK8zQY+6fqB6M4AG0C40P4utg+fQtmIDbzhf5zD3r0qfQGFTkZouU3V4nF/aWWI2Wrk6JDXCAwN5ErqFcI1drLby6ihpModIUBJ2lUtg8PiJfL+h4du5kQfPUKtM9tb1GwbZn8H/UQ9e5Zd5O/S6Wy/uIqIXj6wVVDSYfCkMgjyp+8mwX2mxG4WVLRaIDE7GU1Xj7qEZ6rtn54x6cBoQstDuP7SF7ibpRW1bS5T+HhvEUsyojjSv4F+w2wxg0kTyoU+GdSHuuJea8VklaFwIClj5Ea2BMCaknDejJ6HmzEWsBPtNXjhoW5GrPSn8NgAXCZ5gDYCFulBpuJCQn+Ohn3Oqh8e4kijJ1OfkF64z6U4hh9JxNmtg8jInfQm+wQErIKVTmcDll7JLQCHui7SUWng0V1PohtkVyo5zQT4IYg5XRomaDvIGF0u2S4w1415m56lZORZxkTtoLF2Ko6NU2UhjeRGZXPYBN93QLHcruhhUvvx3uZhpF5JQ9/oRsfUMTaDKAdtJLSJHP7wHkqdW4ibsZMnIpNtUoVd1WCyJdqurg4n4Wg/4hvc8JT1p/fo0Oq4LFgZVdvAw54CaYBoaidKlKExqPBuktO3MgRFmL3KXsTIoyZ3ylqdeGLYXKLcCrCSy806Arm5C1+dC41+tbz3/F3sae0H2L/7rTXe1D+wmvDRF1jx6n4A/nHIi9xbGY/X+v60uLWwZdghRpjek5zzFGeBtsFlFJfGM87ciVohxRJ2t3zAjrHf0aWyAcU3kyCmmo8wYkU+b2a9ys7AwXwqN5OusfV1tQ0BsxWczH0Ii64i3FuaMJygvkbw8BJO7RzJ6uXvM+nfxQ71hrDh6lqU53dTGVhHRnIWdSHfEuJuSzJRdpaxInwt21v92OW9i1bvXqVoSg90/VZQO6sawxUdGe0RvOBgtpb+zcP+W8gW7uZ0mxJrp+pWGup1Fzc8Jx8m/VoIdS5NqPylPkJ1RQCjfh9F27gGUgfpkVk6cEwYPq6UEZ9wDZ9RpSQHHuKmggnY4on82Hz6X+nPmjfXcOWeFsm+LTIlSYdHo+3QsmfGntsqHwTgj7IpFB61+YlKUarWoDK1cnf6IKJ2TIOABgK/sy8gIQXn8SxohLokFIZwhMFSpYbq6BEUJYbiub0ct0jpd6cztbF53juIFgX1O3WMnyCCsd32XcidsWoFspKzSAmuZ40XbLNKr0s0QtSNWIQbAkfqjvNQLzF6V1Fg+tEpyHRySiM7Ucml1ey75Ykceeld3vG1cI+viSarTXrvouYOTrisx+rayYE71nNGAZuV9jfQ2zkAq0s4Pq0V+O0djCxVmgCe0rQKjvmTQhoWYQAVAwRwDrvVgy7h8lP4FJ5n/dnBZA65Bm4wTUyD44HIhM1Mi6uipekq80dnccNbSYfDd3eTkLNa4VyXGp8ieK//9FvEmGgxMvHQRIpjGjk9HF5pgMedbAkVeQ3hZHyWRMioy1welwsI9qrBrloGVc1H3D0RsyoSsduETFwIrh4wfANtpScxr5vLylyb7LihOwNxvpTgFwWRt1+7iUusZkHjJWAAggDd8hr2T/qZMzo3srt8OFuRyDvynuQEp2A2d2ooMnaRUPkBT4Uf4t6YN6DzLnC1kfxjYj4l6OUUOt4ZgCx/BYFv2uYNq6hhxvfjicvog3JpB4KzgOigdtMpBhNcYUJvkLG28A7MPtKkSkGAdp9I9GEiPzfGUOJe18suEp8bz/gj48l64gfClS0S+9nGwcg8jnHp5c+QqWSkyqT9WT7RH2frDw9RBtwxcgNfT+ghQC0GHlFV88Oco9TkxON1IZ2SGXb1rCec9nCwTzRXTFoGjj9KVFQLTDp3yz7F2Q0fJVzpfwK/gXm0yKq5icsAdF6LYPYmm4LEV4FmsJq4WUUfKHbzjeIamysmkx6dzxGzH8sdzllp7eaB3+6lKKSM0qShTHaR9iYF+D07j5YbJ0iMPUyUcy/paUHggTqwhhbROd6EwqUnic8pmJF7Eog9O4iwiU0E+TnRKeu8FZK4uvXn686fsKYFEPzLH3S4R+Ls8N21asZT2Z5HjYsM/XMrMZqiALv/rJQ/ypOTd/F4TAetHRrmC1Jx0e2W/gSG1fNkaAcrYqASqSMc7ZpC59lxWEQLW6fsZXYvkjOjq4ONizrxaApl0oVsYhbasMdmRQznvH8jMtaN+zctRgyqQTmxA3C1iRupBHbM3EHfgX145Z77Uas8OHdtE4c74VQXnJV1kuBXS91YD056/kSe8yo2LF4DogL5lfew1nrjUunDvlUb6FKVcZ/60VstqC7l7GdOcxZKZQ3xFe8THex+27P63zL+K77jv3P8h+z7z/gfHXq9XvL/jw7PpWnodQZOswGaL8h+uWVTWtqIL4mg2iLj9dVvEGmZdqv6AsBgVvHrwy+g7X+d+Y/s5HtTKo5kX7p8IA3R54m6Gkds2R/Aclu5cdNlWNiMIAhE50fj0u5Bk2UBil5fh050I3rVH/zdYqHGpRmD5ea5W3n2sec5UbqIvHVjMSg6YLGDjr8V3i8tQ21Uc3XxRhTePYF1jU0Oy0e/lBmNzpQmZzE2LZtK2b3gGndrwky+9APDDy6gIqaQbo8IQArQy1MSOSI/xxmncpwbR0qycAH6yq/T2BzO4MPjKPVslshpAlwwfUGtugx1ZjQlHf1Y1hMsKi01nPyyLwGeeoa88Scms7dku1hxKR15QwiL/Y7V487xU6+M/rFN71FyoxyleRJiUKukIkAQBJKzk1F1q3i98TgdRn8cqUKj3IWjI08y/PRwHv/6Mazv3Y4ADXb/gnL3ryg8EI4xTdo43mDS8fq1ZgrO9mHh955cso4FtgOQ52rGOP4U7Zo7CVDOZnSwPSATZGq++2cEyTnJ3LvyD3zddJJsd1dzDX8EWnn5eAxrXlnC9fklzHnQflzRKjD4wmDbNTS7oxOkQN81cQ1XQ7JJDTzAwYKHJf1yAHLdF3DqV5HImgD+yG9m1FtS2YZ88Ql+OzAPjT6DgfLrDFTbZM/MLsnsy3+Qxi5/UlUw2QlkXRWgsR2/xcWHL+98i+Hl4Qw/NRZvrTQbW6k6g79fEtVhJZSESIMegLneV2ns9COl/kl8tBW32RsDLRwffZkfpuXh4qulN9lXb5ZT2SXnmYI+jIioQzQ0Qw/ZZ3EK5JpeZARgudiXj2NrbGRf2CKebD1LevcXaNzaeH54Nns7pPIxv+qCKBXyedQi0NqpxF2wB6CCsY2laUWc8OzgpUPTMARJAV85Ju77dTkt3o0cf/NdPOWLeNzBfjPoNE49xtxZFykxz8Wx6koUbH0QTTITcrOc9/1PS/Yfbcpk2bNbSSqFawYIdLI7u/lGOUYZeJwsIavCnfBZY27ZTHJPPnOtQtQqWZCWzcAo6f3OoR9fTltJX3x5TXCjWikFCTOiBbRvfErBocFENnvhZJECNAmJF1kw6C+Wba/szWUDsEh5gB+i4Pkf56DocMY0VgpYeHq0Utovk4jgAN6bOJwK3RZgHtXOSVysLUG0ehD55TIOn66lrcUufTo1MY+YAddZt3sScaI/w52koO2u/SvQZJxj/v17eDupgzOGYugJE0W1B+/9PBmfeh8+vedHfE0DbDN72jcAdFSfYfzy59i6EA53wTiP5fQeUxVtWI+NQ54Zz/d3lPCIw3ft357BS8cmczQvBve3P+UjX3vgna5bRd7H4wjwDCJ7zAIS+zlk5FmMjPH8iuv6KKIv/kWrXyw49JqyKlw4uHMEshZn0p9cQoN5NDi0C4/UxkHU0zQKRbTtl+E9R5rUMEZ5mp+j4elP47jRFojwJjCvDrrrQRuJYNTzxHN/8c49rxFRHIGslyRrrP9ytn04jXDNblTu/vg72eZwmSjyauMvZGekEvKogfs8TWj0RdysXrJgIiB3H25OZp5WvU9sr8z3FVEnifj2JwrKBvK6J5iol9h9qz0Zd3QIbmEtDBl7HnhZYjepLQiadjZbo2j8LwBEd7dmjAv2UlDvym9qwaE22jbGBVagjlCw5VIDl4SPoad3Qqp+KbnWZPQVPqz89mGujZYGx+V535H55wHaB7WQOCWbbKsTwx3IvsnzvqN6gpYvFS0cb4K3HCrkjJpwdo86SpvcxMXEHKKtU3ui8TwwdRAlH8iAc8Ow7hiBRW7C/E6jjZg9fx9ELIOhv6EssIG8p4eco7tbqscmCALfDz7J+A4lyCOp6M4FEkCm4kr5fKpzawk0XkAQzLQb7X085cYmXlm5j62NSlx3TCI5qE0CmPkZrtEQCW+p8zCo1xCaLM0oVcpN6AJq0Vl7al17VZM3N7lx4McVDCsMITP59v4YgiAy+vhoAqsDWevRjLWfFOjergvniiaErLv/5JjZm19d+6Gv66Jb5o73gg38VenHzLq38VLqJDJDzVYvzrQq6Fsezt0Hp6CeLU3oaVGDU0QlqUqBJ5Qu7LFK17TdOgUtJiPRB8cTNUS6LaKc1gZXOozdXBm5m1jnGfQe+g4Vig4bUHDKU+cgTAq4ROErUXnvud+NF1CqDYxbeJSvq2DbkOPEG6RE+E3SsjVgElWKBOZ5SMGKrzeORX9qAN1mgQyD9FnUyBO43l2JOaKE+6Zd4lKoVmLXOY2mLsxAbfd1riScZ5ZMmvIgGNt4alAJp7VmppiUPGWdIpHcdjY1otvnjFNJAG5eLQSY7GSJR3kVgy6mkue+n0uXQ/ENbeRmDXOVyUJWThjt6i6MJx/CfeA1eEN6PycdmERQZRDtze7UdUtBp4omT/ZkxfFmYDULwtrI6kXc+kc9w/A3ZeReD0MeJZU7A/i3bg+ZCcfRq/W8oSmkwF16TzsDlKx/6HvcW9z5/JoPrT3V18M4xGvhMK7CwoiTceSppNUwM/0LGBuay59uuxke60qZphpHsm9BZSWWP1awe9ZOIvpm81WU9Jv+fdJ+mLyPiVdfvq0iVtdZj8rvMzTWBhq8tHg6S4lCgGCverIfPozZvY0GTx++crDJLG18N6KYPbpixspgsMr+jl8J/o7TY0HRsIZVX64i+0lpH7ilYXXkmdvp9/I7dLkGE/6yPeNao/Qgv2EEHSXjiLiyBaUlEbD3bjSaagjp2kOTczFn0i4R6zpLsm8tXSw8NxzP5AJiJ5xHZllB77E8pImG1kbeDbJy3nQOsDvRY4NmUP/zZQRgi24xfdaEA2BVevB90hVkZidWf/Ybzu4diIanuSnv6OY5ireWfoy5tosP4qIxjpbOVSZPWPvgWqbsm8LM3TOpvPlB9ygTuDl1Y3Bq4Y+FmzAoDUwTpQRRZ1AT2SNOc+f0i5Qqb1+zQoxFfF2VCAt3cN7vCNwS6rSN4EJXEnfNxbvencU+e4CnANu8v2/KfjybPJlcEM8UD+laOiciB+8JNegqPSg5ORTv6LM4kn0bQwtRCwIjL/qiV+slSRECAtEF0Xg3elCSMoPgsB6SURBBkJHS92MO1/xKcHMtM6wWeMgKnv1hjp1Ybr/RF22dL2OXHMLLpRj4VXJ+2c4JpB8OYXullqYeX1XecJpdM7NZMSib1D/vRbP3PgLS7AoRMquBsYfHk302hfMJBRSFGpmzWI8rNl/TxVDKx59+y8sN8Fw91HYbJD5AlsVIa8wNhJ579dm1+wnvKSgQRdiri8aw4mfaXdpp8WhBrOpFIAkCgxdmkLsqBuFiBa7DCsCh0/hM1W6GdQkU/nE/yulH0VjqgBg0ogGVugN3SxzdAugtIDgQu0eYyLa3XQlWeLE97n0GBwiSXq+Z+gFsHPEzYpuZkKLp1BrqADvxa0zoR7Y1mezQlzAo4/FycqgE0JXhV/wvgSemM+u+YvReUr8cpRuftzqzMWUjhtR6otql2XSRhhLCzg1me1weRwddYKnZTv8f0UdyvP0Y4b6N3J1UzvVOqY+g8zGwde5Wno4I5K6U/eRYH8aR7KsNaSYz/BDHgiBBKb3XKnMLz3lAY1AzmtRs4ryldlfRSkpRLN5lQYyp6cOE8dLqJACXQts9VqgM3BsqFYX30OeTdGgseiDXcwZJWjsJn9ixjutFcQwyeuLSGoq/6To4pFg9PUtGlsHKj/KJyBTekvhPZulmihNUtSlRtjRg1rnb/G6rBVyiGeCdictjW1h/IJU7v5/I1NelPvLjMR4In/3Oe3/7UuQn9U0A0uQNxK/YxV9bHuaZsx+QuliK6wiigEFl4LlW+LT4DjIn2kgxq6jgn7nb8OzuT0z2QeoBQbRXxt7hZOH1qEzWbF7GqcP+XHJvckjnAZ3vKeKn+/GvrINHgnVcdOoC7HO4uV8keQlteJ47gUW00e9ewiWoBB+nMl4cXMCu5AKGVUNw5b28bP9sEEWY5z6T2k3xPN6yiBsxNzAOtL9L870CkL31Fn/lPEz+4aH8Hl7C4yts13VcEU/l3K1YXRSsLI/hrphChLZroE4Eqxl/30qs/XIw9Z3B0yNeJt3ox811QxRECqMK6VbD8VEHMSgNbFU6JK0ZWniDy0SoFZytcuPu4A5aDOnAAEQRElxLCHUG8+4RDLyUxo9P7eEdWU8M6NmflAnVDOm3Hm1rBV5jdRADKO1Jzxsbz3DqnJq6mFfJdCqlr+UBZgOiTMBsiaZbK2fgIU/m31iDxw/2dcFdFUDaySx8TC3kHjlMR/JgJjn4Tf6Kf9i1ZCGvHupL9q//D3tvGV3VFa1///aRHIm7u5OQBEJwd3ct0JaWGhXqLtTd3SgtheIUd7fgCQlxd/eT5Pj7IXLOTu7ne98x/p1j5EvmWPtsXWuu+TzzmQtwWiPOMbgqylg1IZOTjR4M+vU+ZqzfLfJfz9iN7M4JOpy38+vIBl6VieOuSlslEtdGTPXOyPSdlhhFqmZlRwznVXdJGheInyEAtawJcAJggk0m9fdn8l1JGPttqvixeBQVrpbiiPCm93EXWslxdebRYSn80SEmvdvY6KnwK0U1+wyZ8YX4W61ZJhsPtm58iPY6I1tefptQvTi+GM1lcprC8XKR8O+Ir8mx8QHEsmL6qg6cSjO5EruW8uYkrLNoAgKlh0cT1ebIP08d4oeW7hhbIqOjzRa3Kk+8T9ohL3iRpodLeu+JIFOS1zAEFxdoSpyDXqpEsFIHArg7bh01Dkc55QlX2/sAsw3pFG6eQnBMCocTxLEBwISgn7gY00bO1TK+L/Fm4VRx7D9Q5UDW3MFs9niBQtdKZli1rAHI09ymQfkLK/59nHELn8Cx/U1gAhJBwuVRl0kbUcZzXy4l2t6I0lhJTx55hV09962uIS9iKPGXp0Hog+yTx7K3Cf71gdzEbLaGVTBYthpPaRuJNnldpIfCv5CYddw3cxe2EyJxlRqY4wgSK5BSIggs3rkY76oA2j3H4j8mqN91/79kffGO/037D+z7z/5/ZXPL1XQKlo4c1hsXndyVnY/8SrEBzPr+TO9yeTBFnlcoFzS8mqJm7QCxMJNJrWDzin945tAaAhTyLvpR4LIu+QKDBqlJy5O3RtCYGUpN/ClkwiisZUCzpMPZYPyKYBsVG+S2eJq7agYEiZT7b62jXqfCu70ZiY0aqbGjd5xEkLBk5xKMUiOT3/2RKl23L3A52IUwKf0i2T89wMXpx3h4+BVelPlZnzZLPWNo8DMTereQwNjXEYSNIn+4ai+rhbOUv/8sd6aIFxiANns7wp7fSNqmufjogxAwgRW76N7CnWTtmgLUsM//aaCrwumgeQIX4/+lwTmHb4ttCTKOseKpwJBMP0L/yufc3ClECVuYHyQO0qe33kfy2Vg6zj7Am4Nusftpy7200dcy54XNHC5wJ3nrNDrixCCO3KyneOQ1EvW2DMyKQiURNwwHCFRfpiq6hV1hu9DrfUXsH0GQYhObiyQjiNOfLcawwlLf8ZzZm7IxJ1leM5l/RrlS2PYe8BoAtja2tHstI8W+mUG/30NoQq4ocernMYOXf3mP4vpizJHZ1DuI5Wfu95xK5kPJ/KraQ5bzZd4SNov8ZkzU+0Rxo0pC8lnHfmCfIAicnniaOQcWUGi+nwjbBKs0C2xt/Jf86kMoxwylxv4bgjwccQeMruPYdDIMmbaJGXfvZUpYMfKFZeDSBQT1fCvX/IuoufcP/BD31fvC15U7KzcSIofpcrAxd2BtHx3YS2d+DROi/kEd3h/se8TtYTJdr5PQcpmBuqEk9/H71kPc/iXYZ0VimnwNwWwJCAQETk45yZkJZ5AZZKgllvffplnCgj0LyA+rwVtdgq8wQNRZSa6V497izm8TjqNVavlBZpG6EmR27B16hTsDZIx47XVCRojfIR/tXSLnXmYnGs6YdIwRxBWLoeYCPj68gKY6Jxpe+YNOQZxYlRjaeMYJrj//HY+YHLE1igHpIru5vJ2eQplQwAgleEst7OExqTG4l3jQatuMzEVOtbYE6KqeEgQBwSSAycyeJbtQIusDdnQ9S9X1afzwTiT29y/EWiPGJPmAPTcKmTnpPt52g13WFXLAiioTaWY4PdWXeBspRpMWqZWOf4PgxfevPYJ7uTclk1oYZestGv/GoMu4xGp4d8srfH/gE6a/1PWOTZmynXfvyyO+2B79cC+KNUHYWd2yoapG5jjAj0HDGDXlCO0uYmlevbkOO30L6RVOPK92YkaERWZFQMC+1b5L3tPqHvSYTFvPOicoMcB8O9BI8+hrDzSYOXKqC4iTGsWAtnvAbNIqDaDT4uLagosVXjFIsYiKtgpMjs6EJn6HWToIrCgVk5acQDXMjqvvm1CRTqs2FvduGS29OpCvphwipCiINz57gcqpYkk/x84zbF88lde+X4XN5SnU9GkF5K1cwi+fuOByx40az9N0GkeC0r3rD7BFysgTO9G+eQ8L7c046ucDFhkliUQg/OpWHNxaeOm1h2hvuwv2AxAEyK0cTGuDPV6V7vibVGhNFsZ0UeU/hNelotEo8c/3RDZBvJY2Ektby1Qibilw3bwO0ytiZvyE4WloB9zmetXHlJmXktAn0f21ezZRRaHs+Go5EfPFcw1AjJMjcxZcY10NNNX0r3h5vmoA5YGX2fTZLaYJjvSAfbXKeQQ9XsiR614MvlWGl0QvGmcTE0VukjNX4t7hc7/DPKGYJvJPa5TQKs0j4vxb1JtjkD9mYc8LUjmn4lIBSFCAc88aH/4YyBzwalzJvj0/I+nUEhBWil7SCpJuKWSbrvchxmsSV2Yc41ZYFgNFnP4uyw/Lpzkyn2V2UNyQDXSt9dk17zHkqoojs9K4mrSbNbq36eHZClI5tW711Cth0DN/EigXFchhFFQcOzkIk9TEx5PHcqVtMbCz13+v6hR/KVVMzFJjJzOhNIuf5Z9N8QypcKVt2A2UM872O2d7fSWtCw/D92tZcH4eLvPFrMX0zlpMR1UEOz6JyvYb3J41Ul+TRpl6HKuFDdi6OfLh3mYGxlwHkxYk3UlMs8BrH7yGxraNTmcN7WaxBMpWb3vSl25m7ZaVfLopEN89XiL/v7vGMzAlisoHN1PiokAEuZkMGJqeYfy5ILaOdeGG281+13XH/S+OvfY9mZJ63CQxWKvyCQhd8kcXR5MokROw7FbPwwCgxQh72wQQzP3i1AmFdoT9vQHdjCyWPrAZufY7rMGUQwYjMfbNbL1nK3qVOL7QSR34Yu2vyIDdMogT7hfFPXplKJcUoSQvmkKdw1XmCU+KxtvIFEQmSLkanEG+poR6g/jbM7iM4banirzY96kdeRhH5T+91AWPyVdJGX4Dx4OzuXxsAsq5Ab0pFr3Ugfce/xa53oXZx59G5z5EJDE3QOWN6vpwyoIL+eK5L5jr9Jrody9kxZG42R2zQsue+BzCvhW/Qx+2BtEsv4bv4ErMBncRMQxAZdZQsOggxfruVqx9AGsnoY3RDZ6M//EhTk22AHpFkijePRaDrM6dqFd/5glv8buvlhjwlMKBgFt86wMLZZOxhq4iXerRRRbjWuJHhwnM08TPSyWYUbXaM+TCp7QP6gC+h61d70PLuDO85fIj3+W78Om1dBavEld2AKx0SGPs6BKEXAhpHyAC+yQYGaGE2wdHYzg5ioEbLCBMtekudfbVSJUVtEwswt9VTNT7wEXDwVY5yUNsCXEtQaKrpQc0Mys8+X7j29jVpuFam07o4O3Ajt6xmuatBF+vws/Dg7+mKkmWidfSgfI0gm4PRlDoWXP/aW7TX23Ed7ScGmkWTx+PYdSqIb1zGYBd834cfDvIUpezef4HrO8sxxc3pN3VombMOE6/xSSvbJrbF+PZc9427jQ1u+GgKaLjVgyxw8Vr7QY3e/5SO/LVqCZmjPuLS05i+fOtrmUYNU4M9y9Fb6Pv991+NLySpKkZKPPNeHW0WjeQAECqN5G9ZxKDJ95gTfxx+tp9mhYKa+NR3ZtLS7ilinipkxcDw2fza00A99/3OJV9ZAo1vu1EhV1jz21PHAekMdB+gci/43oswskYHssPQpV0B8FoqQ6R6ZuIffIfzrcJfOS+kWJDArAPYl4HiQ0KqYLsUY/iUfIYQ24OAUN/UGKi09eUhR2ixV1Om8GJmD7+k7/NpSlHyUBALnRLpjkP4playNSBJrqaxOAs6qUWEEaQ27ItIh2jQxVBV4aSKmhZYHW7XVwG80HeFO6U63A/FkxRUp/qDDsJ21ZsIzYtlsl/zaHdeRUvdYc9ggBmqUCpbykr7WG+LZypq6VHTh6gkql8e/gurvH7KHUYwFCjuOd7vLyYXV89iDOwTC2nxdBFrqny/pS3P3oFF5sGVl9ZTLt/OZInLAk8iSBBYpJgNoHBpOhTmwGNJneOtYPDxLMETj1LaJM7WInNFjvPoXlINX7ZmxjrWE3QlYWQ8DF4T6HBLpJv8yfg1mLgQZWUerumfs8qOfknZqaNYOjsi1Q6F4mfk8Nkqo5oiBZMnBl6TdRPW1UrY/WO1TQlpfHo3BR+MIrJAW8EJTMuupQ5eyScurac+e+XE2N1P3/zKcBGkHDo+fVsjCrgz3stY+X6egZtWkFZky0vvPcbPzaOEB17kbMzy9/dyN/fr6G5SI3KLJYQBbgYvQl56hZoqoeO+SKfwjmJ+tUfk90ejqlhmnVqg9JZ5bSNrSTf7jarnNvQSX4TjTXfDUMo8MZH+BUns1i9Qm5q4pVrY7mYHI2+7BdMQcNBOQbG7AWHCPwzHmOaPZzIiMIrPxiTIJ777YUmvBwLeW7tLRqM4vUIwKCO4qpxFYZKHb6ac9i3NEJX9zAARkrzGF0wkNYdC6iMnI/kra4PxFOqZaIQTH1bJAnXJLS6BIpyFI1NXnz66EvoBrbxzZPfEOkolop9XxJEdtJBhiuhwgVaO8QS/BumTadJb8PqUdmoBW/sL8IZ4QQTYi6gOebCwHx72vRaVsvfo7hiItatpwUB5LaVKF28qDdLULq54qdw6vWrHSK40DGD2tZ4pp40dvVC7yYomhVdijJxuYn4/rUSHjyAhO49idqH8TYdtM3fyXPty6lr90dv1TPPr/Qv9i29wbiCEhqFHlKv1YRi0hMn1BNwbiyeR8YzZs1B7oZZlKpWBx0kvC6Ujyq9uTr0KkaZWEHIUeGI0cYBqKWgfACpLc7EKy3v/6bqz9A3pyHxmI2TaSOJjpLuMzATGvQ4LsMcqUzdgp9DClPlN3rHyQUTsVlXUZWUURo9BVWQeK/dZLLjj8vhtJ8ZzKUZh5H6+YniKqnESLyvjsvxNdTXurDR6IE1zUkQwD8nhRCdP+HeH9HXbnp+yY8rf8PB82uidLsRzDcBOUikRIYtJaeykPbRP/F19FU25rwAfALA8ub17K75GrURlB3K3mfYYycyZ1P7SyO5szOQucAqz2GiOPXQlDOkjDMyqRyEShlVEgsUYZK7kOK3hKawgyQqwA5xtWO54I/0vc9p3LuSPz9aQ8IGce4QwNm7mcLhQRS4ZyHFDFY6VYIgEFIQgluVF580HKJG6KlmNKAZnsofSTcYnzWbUK8icg0DLXspswmPmlRcckuYsPwHZkYcp9LYDnQBqDkNe3Ab8QLlDSae+30yzSFaJlkV8/7bnEBspZzAYHu0/jZskdwAHuz1Jzr8iE90NXlfxHMlsJhFgjgGdmnbzjtzX+FUrifF5n6iY4w2n2K53IVD0WW0ufljknfFRXHaPZQFw4LSTva88hkDFfCE7MuuSzJDuK0zSPWodUkc2vMiA5eFYfbQUGKAO9ouAqjcZI9NGzxs48bTsl2wezsA3k7d91Uw8UKZF6OrvAiMegmuX4MJx7HTljJQYcbQosa1/jhCwDTAm//sf9/+A/v+s/9fmcf1J/DJNPNIRDILEoqRGpvoYZMIUiUnOkDdruLFDneaHZtEY/0cQvh57jESrsp48dMXafpKvGDb0k6cDUx76A+m2UK5/nHsNUWQ9zNEPYOdQs3kpefZ+XYoQ3Wbse24CbzYO15AQNWuYuGfa7DtUBP4aze1WxC4XnIejxo39E+e5w8fExltQ+mpSlEYGxg16xLX9WaSFJDVs3iN+geAW0eWYZh/hDzPkt5fsrZFo05gTGrjrzOBSFyb0Rl1yGSWbLS08S2CcsKpNHtgshFLFwAUEsIrdiVsfe0wH3kUdLEyrFhJLe6t2A7KoD0ynz+G5XT55faUtmdzaPqXROUtY+3Wf7EbLt7mDQn4i4aJtZSkD2LAleew/cRf5Dd7e+A9ppGDmgKyorJEAanEpGO0ezs3cmzwSB7BDe9y0djQtktUhcCrDmdQLzpDvVKctAKYZbMGedYYUsO8cVAHiXyCRMpS3zSGjlQx/ux47jTLezfnQndgYpds5IPNrzPp50h6RIxkgoTFg4sobAngkvdBLjq1sNjqediqgxAOOOClDGDfs28QKBFLGAUK55gx/jVkVYGs8YADxlSsq0tv1i8lcNJuPJ/ZQFmQnL4VL/b6CgYMyuXPiC+JubuSTlMrWInE6EwdzDgykcLQDiqjwzB156Lzm3IQ6t4kLDsSsyyaE3XDGedpUZWfb8pkxo0p/GqrZVf0eez6yDmYuzlKq/5ajW2FFw47xKBxW1kuXulZpKWHcMFvDgkfiJ/FAwMf48LNGDTbZlA6sX8T3jXGQlKyZpMVmcX3EdcpVlkSsz4dudQFePBpzhw6vH7HW+jeaDalMcOcSdWd4di3laK8EkP+OPGm/+3GFnJ/fpwboz3B6VEGT7f8tiBT82kjYDZQ+sqnSE1uIiVOjdyNgjGXuFNpi0+5DzJf8XfnSQ1yG7hokvL8hS28s1ZU24FE38KK1CEc/mM2d4BT68tEncUG+d3HpT88SKxNxlnoJPS1C72+h5vcyUpJ4PCLP/JeeCZVVelAl2SgnaGMz357jEajwHuPfY/EJA76fIUyXj8znZpqe1rcgnG0txX5QzTxxKU6ckA9iP3+t3nGLUDkV9WYmXZzGsc1TlwYnsJrfdqwuHtsI7V9M50Bzvz+wBwa9KVYSx1ubwqis7GQgItwbEgOgwRHQgGT0YT57614e8egK0rCUS+AxIeeMOOri2PZe0mFbYwvbxdfZ+VK8e96LFwIS5px//gxODAV+TLL5ltq0jHogQtc1TTxwJXRtHqJ74nBOY6BZ/wJTY9heXwlwqD+2ug/SBxh/AUCBmfxQ6T4/ccxihuRL5Mb/S2/ZDoT5T+Jy92uSNud+D9fzJ+1NXwU/C27GgPoBfskcvwKbNCyhU3ftLDSAdJ166Eb7ItwjSC2NhWz7Z+E+KUT4Cr+rloEJ77/8imcJNXMeOsXTjqJJUIbZfOovpOKelAGTz+1A73xJXo2FwByqZywwhE4CLG8NeU2F01FovF27ecYNvsqtUZf9uU+zJTpTt0eM78MiIMBZib9toiPnBt4ZJmFomuSyBj29vfszYjEc+88nNXX6JFjBbjdvoDTJ4YQWJyNrrWcEa7il2ipdx02vloi1fcjN7jwTEcOvkrLM0m7GsOdf0bS5GpDjkv/yg97x1UsOSqjKfxB4pX92XAJ12cxd/9k6p/dhFOEZU74qVngb90RnMIaOHnv71xrHS0a5+zRiEd0Czp1FwjYVzKqQaPGYHBg0mGBRk+jCATHZCBQBsZGB34tCac8oLuavDUPnAZiljtzd/haqkOewjViP7HKt8FvPszKgO757ruk78mLssFB1cRNg5ilKwjwiSsoJfCkE+zQFfb+7scjfuewIpAyR1O/87a38+PVY+cwCI18MWA+AFqrNatJEUbWtunoXRp4Hh0tXvVYp9tymwO59cIspgMqu3amThGDDp2uRj556RMu+kFwH+UwAJ/2K2wdWcY3zce4L7ASjXQaYGH5CphJvNUFC6VNbgRz13oW6RzLsOxj2JjU1Bed4ypTGC+xxDUP2BdSPaaIU+ZxXHB6mgn9CP9d15gR0kijrT9x6miR1ygz0qnspKF6B6bmPlI/nTV8MuUYm8pmsPHrJZjW3kG8ksNixQ+8E1WLXT5g7lMZYmxnSUY8rndj0QL+i651OVySSLU/xPa1JTzhXcKciAtc9xPHNg/7d1AyJJsqx0hCnG9TYRYnMwwBj3A6JI3vQzXIzWKwWuj+MwBlBgi1ET+r3UW/cnTQc8yRe/KYYjKOTkHii5I78GFKKi4/v8kDMRk4xIgr6Bp1rZwbeJEsv8OgA63Uwh7+Krqaa1p4oj6NG4kt3GPVA64HW1MYPNB7voGLmHRMkrKNxV9/wY66CO71gVN6UVkkbT569s7fy7hz46gqcSNIKq5wLum0o6rNjgypnFB9/6rtKMN10vwkvPn8Oiojc5A9JgYxJxlusbwikuORecRHloKhA2QqiiSxNBxqJspkw87xp7htUGCdEjt2cwjlf83FdvZBiL3bDwDaoPYje+EWHvztQWIzoxFeEj+vFF0gZ3MD6VR04OEs7t9lUrjx0b6hqM+OZBCfYJj2JD3rRo8d1EbyT4qB8SnhNCaKt+tmG1em/DOUmUcmU+dax+LaEVzv9jW3P8dAeSnJHi18vqCM3R0JorHr/pxBwJkkwh/L5uHRP1JjWEgP2NdhaEfZcBF1SxsxP7xHtU58Ti1SP1qX/4zu2hD2v/o4g14VE0GUikW0rNjCzs5TvHfNmUPDnOhrX187id/hV+hwqaHh/mCRT6HN4tlPPubNenhPAnJDI+CGTVsee5KncrPchZfnH0Ai03BaYnlPOgwabkwcRKO6g7Sgx3DSR/XKpgO0d/hyZPd8mj2q0IxMRiuI5UPvXFpPwRYZ9z6whZBBufRNzxfXrqeg9BhTz7XSFNw/dVKmimDPPe+Qlx9OZfJATj4o9pcOrcAn6SPezF7A6MqZvaQ1T/1ZkmI/5dShFFYdvkpMuK3V7Al/tkTxXEM6eFSDRzUD1OIYIbN0OVElLaiCS5k7OgPBbNkDSg1tvOCpxygHibKExvbu+PjG4yB3RBa9mTkRJzFGybj/8Y+55vZlv+u6P34h7V7FTCjVEtOYyMw+/jvqJSiH/cA36/5gh2k+ABLbQDbnuRJYHMgNXxM7wpN5Sm5BgCQSGbuCsyE4G0Z0rYM/YKnE8XVPImvfF9gbL+BeW0h5hzgGmGM+xYchsOHPGbhUOlEwJ7R3DhIEeDbgIrEuUs4+/yzEFKIcLh6foz2PUPkbxXalnB17Fm/DTcASn7zVdg9CbBpmZxXV7S/xuFMXUFndUYaq8g98s0xANFG2oGgvANcuUk8wheSt/5PD+T5MuF1Odrka2NR73AHyWaz9ezdqn818/ci/7DXfxhrse8/5IkXV9vx76RwyNznMbuqSVQWa9Tr+GPMliapE9J8v48Y9KVh2FEBnDe/qHci4PZgAoz2hi8TV7Dqpmr/u+YV5ObHcbIziRuBpevqLjhNK0VfGU3xTxVNnJ9E6S9xPu1DvgATwTYvBvziQNqN4ftWUuVF6YRSqFnsq5GKZTa3Cj5OSdkxKM475MEQWbwVpgck+iu2Zr5MaNZMqv1BGqfq07BAEamyrEHw88Wtvp1o6Q+SO9J9KZtlUJPZXcXffj9E0BuhagF7XptOpbAC9hL21Jn4KEwN6X18YQOKtQSSuOYNa5QtWNBKTzIljlwZgU+nFxKWnuOF3GWTToPoUaGuRqPfy+NMXKI7ezeUpx3hbIgbhc6Wv8upvE1gydSFevmLwFMBgG8Pfae/hnXgbe48LtNuJ95dxxipu/7UIJeCnygDTAEDBeEkOQfnuHLo9E2neRebP/BzBsB26JQU1dsOpCWgn1yuZaN8GPPtUyfesYcmdMKcCHrIVk+l+qVRRbyplW4CEZup5E6iRTIa4yWj0QGYoa7fNB9rxmV6JNa9GIoH1dmoa7v2ND2VP82nwtxQIY4CuSmiTXQTfnPsQe5USt6ce492AFmStuWDrwXj1XZY5wodtlfw771/+MnmT5Zxgdd5dP3S55ixXMkczdHRY71xka+tLmc4L94Z5uKq2EWqrQ46V1KzSnSXCTIYc6ZpZH1ZUMqhuIffTNV/sLJmBw61L+JX7EbBuG195G0DfBnI7aErHKesLFLPv46DeHq2siqLqR7CeKV3bDCQcm07mEAOuEdkoBW/ABYnExHcz47hZt4zShm3k+F9karxFbUFpruT+RZ9x8e4sGPo59mop1sSaGmMsRwqcmVEUgCImlATfCNGzatCG8OCeZlqVdymdvZF7J4nXhjmhH2KYU0HebjemmZ8B4+9gVYk8xHkagbUKHojaxUOOadQbNYATmM28OfwJZM+cRH87iA+/vEFtmyW/pxcUjDs5ifC8aLyq3KheeQmMut5j/+l6irYleUTkRPDExfW0vCFeTXc2uZHX0sasY09htBuBYCVAUdJWwPbp0fjUuvP7uWk0xYoVmby8nuX9A1OJKyqmsrgFe/P4PgL8MD9mAjktAZhTA2mWGAFxK4hnXt7CpRNJpH3wPNInu99/XQOZgXq+b4JPwpvZGX6BAaUTe8E+rSaNFZ4vcOnaGI7pb9DUCkP0LXh378fV+lTW5bryT60NjmdHU0aW6Dc73E18/vznPLpzGR+ufRnHTWISlF/zHjpvOrBteQql/qV8Ifwq8teoRrP1wgam/gRrI4txG5iMNbG3TXDk5NZpNMi0rJd4sVlwIwjQS+xIPh9H1N1wziU4cl4KT43rWuvMZtCFvMrPi1wxqy/TeEdNSYwB1zHVfO4GcQqIrPJi1JXBhNU50JH1BC1ffoWzRxMA5X73sOZEGSUVXlRdHMkRjZmHH1GB0hZsnLHXVfHmc7t5OM+binObCI35D3L6v7L/7vx/9n9mEomEkJAQJFbRwqDKKdjnXESTM5EDZZksedyKWY+Ad4U30dmRqM+NJ2r1OdHxJriE8akykQt+Nygbfh0P5SSRf4A+jTHnp3HSvZw3B6R3debwWwC2QaD0wk3iyEM3i2h+JoGlXjkUGN5nvNV4P3MuL/2xFmpduTbuPHrpnN7+S1FZkcSlDeTApIu8U29ittKSYJcKRiZNuk3ObX9+fPgVbk4pYYoV4yPd1osjCTt4+MhsfH5PQPWjWJJyd/VObt9t4vfoLhbFIrMYTPk3fQhBlwN56LmttIYO63efBS2MuDyCa14uOITMYJZUzEDb4GRDxbwdvO0CMjsJPUk0L3MFi+2gtkWKd/4lHOOqAUuGvsJnKG0j7yCvTMSxppBRrpNFx12rTqN80gXGq2CsHCRmS0Whs1si0/Z9T739Dr79+AtcpUGisc0Kfz78agHVnVI+n3uWpwYOoK/9WRoNKf4s91QQ6yDur9gjvXlt2DWuDbvGKJVl5/2CpoWqI6soMmZS4KigpFFF710zm1iT8AJnSxbxrnf/3jCCABURY2lxOURBMOxpFUsFlUlX8M+OWGqV33HE20znMHE12FhFLpqNy8kGBi/8hb5g3whJOXN9YH6Jiknj4+lofBNrPaw1qqOEPdfAGxfDaLdfSnb9OmIYj0PLVabOTGZfbD6mzutdAbhVYtZROZF9Bz0I9DEyqxYMA8XBV2QtxKePoULZjmNkHh4SMdP15rj3MSQ1ICnzxaTOxlrOEuC73d+gO5hHGCAd01/qLTVAiuLjj9jZqscoM4qYYHK5PTlpiUivBDFtgh8xgd3MzvprPBxyjFnvnyQvYyDzDo/FydQkOm6snwzTlCI6mwIJ4TWUxoeAIKArIT78ynA8ajzYP28/Kp0YAGqRB/B4LSxKjeHhAzNxefqayO/l/xNfejURmHeB2X8X4v2UmAFpVnrwbZFDb2/OGHsxeiVgwr3kJhKThNawam4T2Jt8Kpp3iobZh1Fo/ElSwDksgIUE8BmYR02HwOwDs6n0FgNE/pJiCpMH4xxYhfLeO8gCHLCWAvKv+oGaFC0uxRMoCPLHNEScEA5rNhOR3JXuvyXt7N8jQ4Cc4asRMPFX6vuMnB4lSkEmFybheTaalAmnuRN3h+fNb3Wdd8HPDLm3gu3Z86h+dxNLxt0m2ekF6Obth3fOwqWyCKO9JxJuYGyNpqepPMASh3YUEnjTcyLYexHiaNnYKCXwqlc121PnkHUkEe8pN7A2iVSBV2Ykg5KHsy8Zoj5YIpLuADhvJ6dw/CkO244hQdVH1q/bYm6bmZu1njvvtPX+z1t6iNiov9l85VW++Ol5ytdmiViKTrVynOr9+Vyay+ZWHZ9Z9RGS62p4dODnnGqTs23kJky6JBHgfK6lnJpMG0zuaur9KtDLxHOzRCJwbc6LTJqmptIIIRLxswKIKNyNcH4eSS7FvDxHvO2pbssiafZ+3ts4jpvJTUQ9qiOSrmecEtLVMytg4GeYpCH42FtkiDQ2Acxt7MTOvZTWL98huWkuWPUcuVjxNYmnpNwZ1E5R1MeMchKf04qiQFoNpazYM49ajxrMz4jXrP0qW9yHJVMc5syIgGJATBTpqNMT+2spsIGhj+3td80JvhModz3BxushFEkMvZWvzfpqgnMlKMwqfr31JfbeYaLeo06mWXwxLZPqXUlEHr0XnhJLpk24Hs3As6twHVHHiNg0YHmvT6Er5/2dK8nLDOSQzobWJSldkG/RFvCehqB+DKWrCbe6PzDUKBj2MF1zsKMFhLpU9yTXPm1Hbeog9D1xD1xBgKB/FpJxOY41E87juCqWpQCChA4XDZ2jbhKcPIzFaUnY+lvY73KpjCHnynGoy8cj8TmanA1IrN4TQRDYes9WfMt9mfrHMrLnFIl+N5mBdA6+RWhNJPaBznTYJIj841QljJDBG9nOKBUmDvd5Fk0OY3jg5j80hnbi3LGWaLkYdFtsewanWRIuHxrFt+NjkDVuIyQkgZbCTBZfvURrsZwjy4YgFWS8ZMXwlSnG4jazjI6UuaChn01q1LAkdRLn7Kag9XuaQPFSy6y5l2ibc5K8U48gGPSIFkS5HVsd6smPS8P5xFhutgTyRp/jn24O5vh1T965nEjYbPF8I9XW8+r0VP4IL2WDZx7bJJPJBOisIrJ6IU7t65GXS7mSOpmOhZmisTnqV6kf2si1S+fJPbeaydsUohrPT8OOkN9ih+zcMOy8xUnCAP0tblYksPNiNMYSH4rvbxP5I3VX2XJpLJmnk6iILiRkbH/ylyrjJjKjlOUBzTiY9mPNLq5vK0IIeYn4/ADQyZGNsoBXq0vsqTdqMYSUscC9FF+jRS3BTl/Fuc5AjrTaoPQ5BAp3rNnUGptE/sj+knp2ElExEFWwuOZliHchLn6pvD8oFdoHsEwmbkztUaNi1Y9d/2tKFMf1ADWyYN75ezzyGjeWJ2Vj014IhPT6s+pDydrnhd+sszwxMhf0LSBTIQgC++btQ6nzoKxOgSBVi8A+o9JIpXclcTmRTLuZRMe94so/E1KMgMeSYzyollGqGya67n0nB9N+LZ47H7yHIPh3zSbLOroqQNtq2BacxaxCX4JVLsjlJ0AkuAZZ1QEkfj0SL2BY4n6RT0DARmdDs0Mzd0aPZ5qvBQ2f2thC1l9LUc89wjTfMoJsY0TA1223JhrjUtlpk8GOu2P4MkndW6fT2l6Mz3Ovc7VD4MMaEy4GT5EKRIvEl3d9chjhFcLY/FHUycUSuBL5FE7kTWH4+R8YadTj/1pQv+dlV5FNuV89pyYeZIiwROTb1OHF22n2vC6z4YkADRWmrndckCmoqHKBCk/03b3RBCsyRkXldrbPfpC3GyCtGXF5M1CinUfmwWLKYwdwj9qDtUPF8H6J42NUhqTjvxECBuYhLBTHyDeujEZxsZaRGj0ttv2VRgSZhHZ1O2FXh1Km7r932NIZQHldOcG1GdzyeRa66UTl9s/wy1p/vPTJyH1icbYXg6t+pf7EnHmAtFHnUUfkYS+IyQHuAc9ye5qBM0lyPhYgXW4BS5SOEYw/NxOhrJ6K5KHcndjAynWAUzwoXCkv+YcR2n1srx7O+437mbpM3IoBoLRzFJd/jWdVnYKU1/pXKF1PukCe3++cKoNZ3kNZAUglUsZnv8DAE53cnOaPQj+FmARLbCKTyPCpvweDtIXJ/gdxl/Yn3gSkHabV1cySbz/Fo00M0jRondmxZTrBAwpZuvYi3kGvcVjaFWMIAjibdJx9tGtG/9GmgUB9kGi8imwSClxpUSvJn3IWuaAV+c2ClN2LdxNU/RSxpfcj7w7LDE2HsRv/B+f8o9GPPY7ewcAE5eu942KFNJ72gLGpKgIuxRAWclp03ATVNhSa2zScT+AtZQu+D4nP6+dLCdj/MwkP6mnWBWCam9e7V5KYdATIQCERsNFFM0PxkmgsZhNDJ90go9mGvMMTSR6owLqrk61Rg7tnI55/JXI6UwNPWBgZcWHNzPztI5Zt/gSPc3J8+kC6QsUJLn7+D41J57k8+0feNov9X+VGMv38EI4s3EN+rLi3KDI1G+fsw7/EnzH7Z2Ec1aeC2SGavVcep9zxB0oDCpjV8jSJiGVyC5VDGdi+BLkukA5DEyBWYRGMegbmfcooFw2dmhZ6CLYOzfY4ddp03z9JPynlC2Mucm1oMk9tehDP8GSswT6z1JYv7t2EpxQaPDppNXTveXO+B/8FmDXz8ChJYUBAEI+P2Ey2rbgi93bFUeIu3yT78hN8O/U0l/r0fE+vv4Vb+i8oWuW89UAyEYbJvaRjgO2GJFqnnGR2oJqHJ2yg2TgZ8KTdYRInD5kw62tRPp6DjU8VglXcdN6wk99XvYqrpI5MW3uOmcR7nXH1WuYdXY25zB9QonxPTNZIPhzF0KtL+W7cZTqjuuJjH/MBqHFGEEYT2tClvOMVVIl/9G6qJJb5v6sVVddNLmz35Z+0NxkUYsndZDbeYczlfVT5KZHNKifSBSTmLtAjWFNP2SfPMmRIKZfCPbDtDLc8L0MHq5rs6UgehH2uBNd6O3JGZNGz1rolvMlXL0xjaFoOztXeJIy9jXJ0DWAhFcd5xHFyWhY1LrfI8SpnQKPled1pjCFn6CfYxqTwqLodMwrL+tFRgaryD0xBOaRp89HaVBEj7AGrp/VF7L8UvGKHZtcKprWtx824EFiHIEj49cAGWqs9EXx0ZDSPptPqNWlDygkPR646n+clv+vY9KGYCEi4Nuwad4fe5L2GX4my69PTHfAovEpCzllyhj/PQ2+Iq6b8vX4lxLOKDodxJA7I6Lcmupd9yowDrdxJWcFzS6/wyuRuIqlJB7tdiYl4nJTb7mR9ew9VEyzfZLikCnNqPAqNHepRN3kjOh2M7b1gX6eTnoyYDHwbXZDUu+MgFcdNpy7uxjU5jaTSavwnnaarUUPXtdvoKrjuD9vL3Lh6dATOSvE1+9mGEJ3Zwc3BW9i4egdPdviyurt/eY9tavJBlxaIx7VE5i//q889FaiTGGnvtMG1wYMwVTdxV2bLx7luZOR7YOMp7caJhd5bZhCUFE08T9aQK/h89ij/+pcyZLbledVKg8n5awatYXn8sf4rHGzE7EQ/WTPT1XA3sJAaqZQIyXyR/608J+JOxrJmZgCJ4U0o+/TwPNFcTvWWFhxwoD2kFJ1ETHpIkwzj8JgvMUqNFHhl0KS9D/CiQD2LA9cziL8zEHSL0amcsLHKATd2NmJSmOgc1Mkn0z5hqO9QHjVEMkEN21vhYoEXoy4OoXBYG+fGnsNdNoV1SRMgYCmyjF+ZcWUUGVdjuDvmESQmI/axvr2CeI0OhUQWQZVQScvYabgzgK8Rk97+X7H/Ce/437T/wL7/7P/MBEHAwUHM4nrm/nW0r1DyyOE5tNuGsFJhSdAr9DW8fWoWFfl+3J19hFuOHuLGrdpcfLPOMMdNw5wFV/nZJNaJrjWHkHc0CDtAPtkRycMC5G/qStaFrKFHcyNVDhc6OnhHEFeyhUpqCB6dxqZ2IztHXCBaaqlm+O2h4xypP09HWgRbZAamzg7q9enlXngUgodWy1T/Ujrtxcy3HqaVo08tEbp8mqQhIv/tgltoUotxcXahwbVBlMgDyA41c/GBjcQ5eSHTjumjXg0+2mq8j0/DJDHx54T1zJKKFxEBAcywoU7gg1pHOuVdWbN48zU8ty7nlNGO+z/8iw5VENZg33tNx0mV7eW1BRrei/mLdJtwsBIcNHZvoNY6wEoHMJuN9Ew5jgpHxp6Ug3Eiid+fo61dnLSqU0WQWZ9FaEEoq0zPM2+qGMwD2OomxXPiKWZfGU7I+K1g9TZIJAJvpA5Hlx1C2/07MMgtSd2xklb2FwaQNusQaXGpPGxv6c2hN5l48bPXUObJGLpoBFKZDJlVIN1af4Lvn5nDl6nhvPfLDFoGm0RSnBfaMpDuuoU3o0ixGYJqu7iHy/7OBGz07QyIy2FRfHG/azK6L+C9FTIGlXYw7L5DmGeJ2UHFJm+a3e4y4Oo9DHVKxqPxB2A8Kl05DyQWs8ejBpvLnShttGCyhe5zb/dcwJ0JE3E0fUbS6Yk4B4mFNqc1tFN2bBL7V/zDrchsfpWKKym3D8yi0aTn2eBybLkFbBX5jxtPMtjehXue3IU0VPwsAY7oQkjuuMogWwiRg2DSAd0Bh9sw/j36KOrM21xPXsvHDxzl3CN0JdAlZl7wMDC3o4xPv/ua/a3iYPZf00WuK2BU3fvU3gGbF8rpBfv0bdxfEUVVhj8lQ6qRSsUyb4M8hvLI71+hMBXgseAE5Y5iOQiF3BFDpyNyz+s4OulQatPp6WkGIJfbsW94Fc92K4dOchNft2v1h7z4/Yfcn/oIxxxvsMDZwuB9q0mJVtpG4q0wFpwNYNiTE3qFIXWqUO5NuoFgMvPqB+vxSBKDp+3K1dxaswOt62kODCji56azWFfe2QemM/vJu/y9dTQxGQOQIE74Lk66wdhBpxmWYUutQ3O/agVPwyHWJJzl3OYIrt52J2DBRKxn0kltL2K+uxm7iGy+imtC1nED8KMgfzuzpp3nDZUdNk43cA6uokXn1DsuNnEUhXaRSFLvEpSSSmf5EbBKsa+6GYljpYzpw1JYNuAw8o6tQHfALHPgmT9PU9d5F8XTH5Ct8BK1uxeMnTSPuMFptwZCCn8l1rf/pkgtOONbFsnPGWtJHr1U1FKqJONHHgn4lH/vjMYcP5pnAywg5Ov1alouJ6DnEu1D1ZhdxKznxdkROJ8dxoHOA6T6lnf1w+kxbS2N25vwrPTm++cgwNgHkQA+e/4zpEYp3xbKeThUPPe3d/7GknFf8UeaP5+bDVyfK35WeqOelSu/YetHqwk5MhnTfHHCIVsWxZjiTl6+NogI33J6NuISicAPLQGU1jhRHTqUMLsaHCVWVbGAW60bZomJkEJwMtaIOhANcspBNquKdptoXCU1SHACLIzROTf/QXH4AAAz1EK/NWu4y69civudoPCXKDUUQB/hXz3NmCRmJCaBQ512iLc1MDPpQ8ob6lFtn0xD5FXL79pcJGLvQmo8dVzJH03QePF7cPxMJFd2LSDYpxpFjRsqg5j1/PTgbArkOtSjKwjxasBkNgLdCWUbRwp0Ai2+lRwYeR0Hj+7EqM8sUPtRXfoSv0z5lZcOL6Qz7xFM+pGWsd22/+ZoPKvPUWfriYOIG9xll2sX4UQunleSqFjVvfEWJMwoDqdWeocnMoJxvh1FwHRLslkwdRAWm0NdhRuDrxuRhapFZIpRzkG877OEwzY6di3ahdpfvAmd7/Mm//hAnk9XO/kR4pZ7TJUUY58Vj/3PC7gzMB3eFfsj/Jfi8okGz85Wdkx8ipf6lMiVmTzRTtpDZdxNLtlKmSqzxcHBAW37bYYPPcUR2QKuRcxAZrQHK9JDtc0Cfr60AEdtMXOFl3BsTAAsklQxjUYqToxnyYorrF34Atm6W4CFpb7AXkuzToZk/36Ckn4EfrGclNyBV2tV6OOvQ/x1Em0soG7vsyiKYujWAEyAU6MYsEPpwTM2TdwNq0Kns0pUG9pRqrW8+PMnfHDrEbZWNuLjJiZg5WmmkV4CcvYjMTrRaRQn710b0ihLHUju8Rncicu06qYDIa5T2f13B4p2GQGRxYQ5FYnG6gUFLZqudTWjfBiJtuLKVrNBi53xU8ySCC4MS2aA5Dmsz86+/S5v/7uEjGsx2Dm1ErnkFD1ri+HABKZeHcyl5zbzkRucNuRYBgpwbeM85IIJH7t/UYfpsQa9GgVPjmUNQ9aQgnBjJHmPi9ek2bZ2jFWDUPw6flI5wbbi/kevBqXTMsSOnBvRjAnaSI9qRo/VykJpulaF4FPB+eFtDLEXz6M5TgEkr/6NaOUQJBmHWLfMCYB402U2DStnw3VHVn/0CmcmXhSN84yogidO43R6JN4XQrARxIns6EYDw29PIS3+Lru8ixgkEwM1nw++il3EHQo6jbiauvc63fG3IAiUBJaQue4PfvCDX9tm0BfsMypM5IXmMSrISFikuEJfigHn6Rc5Nekio3Nmk6i0SDAednJBunIXsmo3PH5eRf0KsYxnQVQRqTGXWPXnvdS619Jh1d5LatTwqo+OL6psSEkPQLAX9wbrIdKlJuTgKlvHaDFPo0vKraMFo68HUluDiOTXY0Ft32LI9mXD081UdB4DLJLeepOeaUcmoU9NYMGsw3w7uWsON9oG8cT8bUhMNqz9+x5MDi0IKywgjUliw+WrUVBty2zXOjTiR8FNyQmOLylh6NUo1n4fRuxgsZygxM+HsgFetNn+QrpzMyP6AAPG9nIUHXoWP7WDm/7idQPA0dzGDN8Wrjz8B8UK535+hUZO3J04Jp+ajMOKU73/v9NeSqVtBa71Kvwzq7ANjQfm9/qXGAdTnJNPaMs8nPaC8hXxO7gk5GkMvk0cNkKHGVFCN8A5lCH/Lse+poAqzyrMPWt8fTLYBnNTmUDW31MwGZQUJsYy3ix+TwDeuNvM+CJPHGNyGK/uz7qYPuBZJDVubK1W0eHTFX9IOqt5b9JeTjksYHCmkfF3aoiba5nnFJj4u7yCW9eG4T+zkCj3VlFvdF3TTaY+dITznZ64S8FWIiYHXDUm4nbKiGJkEd5BVV2/aVXZ9+XdN4kJTSY9+gQ3h9zkgVrx++crLWb1F9/xxUuP8fOF6TQuuQlW9e4BklomaJyRZclQSWsQcAMkhHUc5scxhQS4Gyg1GrjYCOvVln3FVWEsvxwop1MrofG1r7moDMGaAthm8iXpyb1s+9mE6txQ6h4Vxyf7HbV4TjpJxoAMGlwbuL8mkTjPrjhD2ZLGdbk7n9kZaJw0gRCfPi+oyosBFbYQc4MR5nW4ScUV0GPazvOYq5rJM45SYtvKlE5LO5BW20vM+RsUinZaFubjGyN+fwWZFJ3SgekhFRxKamSHIN7rOAbN4s+15/gzMYMQlXhul0ukOBtcCaxzJ+lGErrB4rn5VNFxHAoPElviyrUlKTR25EEfsO/5y+O5XdR1rxp5Dfi71ydtuMjI84fQagZy1b+CGKllj/jqtcE0XYmn00ODbYeUkJXieGyGXzN1ZgNVETkUeTRb0V665udOVSfFwDdNMEbh0aV6hBkkClx82ui4ZzStQjl7r64jKloMQGqFk9TOv4LPvzPxqvKkr6lbrzIytJiUOlua1TfoMIjjgCm+H/KjezJ/h80ks8KFDfKub6vTdji34iKROlSz0/050EnQW/Vi8zBn8q1fBaePJfHle7MwPNKMNZ1iqqyOnNLBaCRyHN10+CrFc1mpfxVyrZFB50ai7gik1APGmObCSXBSVLJp5XEaH2tmW8VgdqUMZ6zVpyUIMKvDgEPmaMw5Og7ajyV8qSUHUl19ApeRt6ixk7GlLIqttUYKl3bRMndqxuJiW4FZ5cOcljUEOqYjMTSDzBEMrawzyNmdPJTcpDs4TLxEkGKe6LwlZSU4VxeQPeoKxf7lrJFZ5QpMet6PGsVTHsv5M2gsoXJQSy1zmUSQ0OLYQmx6LHknJzNpST5lsu49oOdEYmuCsb2mZUHeeEImXSM84ihgqeQ8aQgm3+sW5lodl9tjCFlmSzxdfRf/zb/F0OuDUSQdJ6AGjNpp9BDPOgQbntJexa7VDpefHqY0Poe9Vp+tuzyf79wh9UoUzVuKkQiZsNISVymkbXgGHcWYJ2GW7g3kB+/Cw1d6/Y9mL6RY/hNm50s81aKi3YqICuDW9ifmtiW4ZUayLaeKV3tAIEGKIeQBpGOiybl1C728E2vFsbdtt3L0bTn3l3sS5V5EuSSCr6SWOOEzl9uEyOBx9UW+WHORPzrFcY17vSsepdX4DG1k8bw/0PG55VkAd39ZhDrXj9RJzxASIM5XOrXtIyz1Fh4aF5aOBaUg7uEJcCwzmtEZAdTMO8IObZhI9l0QBFakuBETkMcn751A4z4AmAUyW/69GcH0A1PxTCwjQvDg+mBL5arRxp2HmvWAnnu8qmh0bhTlTtoFZzbdu4EOVQfZg5tIbhfLiE+Q5DOsYCDvOCfSKfmBMX3ik1v+5Zx85jibdKMYZpf8P5CwBW4m3kSCH7bSEzzdtwAagWrPalYemsOD9gK2hlIgBheFOynD1JSPambmT7sIiitCYXgbCMBshkN3PiB5zTdcNaj406mDImU5RTazuacK3nWBzXFZ/OK+BX+XCVy1OYOf6h3WhT8G+jZsPUZxPvIW1Q5tBHgXMC5oG5LWx0DZRfg0SdVIU6MJ61CRGp/aLw/w/5L9T3jH/6b9B/b9Z/9nZjQaycjIYMCAAUi7KEEk18+lrV2CSrEBW4MMldXaZK/yxH5sEYZYPY3Cv8Q5iqsgOmxHcGHfWBqjqjgWexUPN3HD47jA1/lmSg0JJ38kIi+sazJV+4JLIhg7wKwiKf1LhhxcwBfPFPbToVa6b+APZQ2XQp/AQwo2WFjPx+Uabjm3MuXEPXQqtb1SkV3W1aRUMAu4zgkj2lOcyFjQkMnU3x9lX1Iyvy74h8eV4gbo5VV/MTxtKFFpT8GImwiv9FkEHHToOtv4e99c4pLm9JOjCnY24znzMlcOj6RIfS9wUOS/t1qKzU9vIpgl7F14pPf/6dJhVJTW0ulSz4udzjgbAvnJapxLsZrFFxdTGevDAfkTeI8Rb1zWVsgRjjxKTpUHa6af4o8VVlWaxnbs4iSU63RE3XAh2HG4qFYszDmCd6Z64HrOQMDdo6g1JkC8ORcEgXZ1OzW2as60fiHqJSEgMKTRndsZESzPiyUlylIt+bvNT+Qs6cTO7yJbvaBOsFRY6M0GUu0vE24XzqJ9WiIHZSGVWFTSSztqqdRqqahV4XVuGE5+YoZjfsctqsaex7s6GpPdVNZ6jBX5K00uHFrYxTR6Kt+mDwwDRttoakyZuJLGF+nejF24QpSMO69PZFfbAeLu/wtbvxbMNl36B42eiwk68gpmOnjFewhPBtSjrfkHhdd4AHSCllZHPUphOVGP/IE05DHR714PM6J4/ify5I2998/a/qh1p9FUx/sueoJs+vdguT74LGeHFdHiAC4tw6wE/7pMqpUQmRXJvG0rGDXnIpJ1HfTIk7ZqW7k+/DZ2Djqc2mIIs+nuoyVIMehkhJd74VLrzwduGhpNwaLkZ088dDipEgdzHZEGdW/3I0GAOY+e53zuYuZ/NYPgYeJeg97mGsJ0VTSoHXnX35eJdmJIQW6qJ7LtGo5eN3nu3o9I0fhgDfa5qlxYUHqTtKkdfD5tGDaOh7AG3VI7dTSZJJTa5TLubjoLrV5fpyZHZB0KojMDCM3zRfuU5X7Hug5iWko97fICAj6NRdFnlfazn4C2bAJtdS+z7sJtOpJ8u9uUd9mrehmXXVN4641U3nAW2GcSV9oc0vlwSlbEukENuP4PjGpN62/4l5XjUmxHYVAhF6vVjLdKWUwf8jnG0BI+l6dgJ4Cuu//iv46zeeH6eTwVRfw+okvQdHmBZU4Y57KBJ6b+zQye4o0ZB0l1c8Qa7HPM92PSseHI7mnH1qYJrOSokEgRUioIqm/kvYmVeBn69Ndoy+XaoCZe879FWs4eSiSDoU9HzF3mVrb/tgIoxBTfinU/VoCCM0NwyQxl0Zd/EybPpofY0GZSM27vfO5GZ7Fz2TbGOImBgVem3eJgeDFzfl1KVmQWwquW+1krc+Wsoy02+irGnR1Ha6gYjHMzlbHZCy7+NRfvK/GE/i7uzig1XmBMUxslPz9IWEAxvCdyozVq+cwjH/PME5Tat6PsQxoTEFixdQWGsCLmPrQXG/16enpDOl8aSs3VWMa8dJD74z5B05FKj2S20tzE41tWozUJnBl5CW2AeGcS61zEvHuu0nm1iBnNN8monAW81eu39TDTEBKGzZJn0QXWIPCdaPxol0NMlRylYeP9aEb236zV8Dtvb36H9bWwrXpIn9Fwt3MexZ6HqLl/O6MDLAmcEEkFMU//w0tXxzDsyiEaQgaAVaojXWmLe0Ax1yedwRRQwQL3L0THPeZXS7N/Dt82A5XQYTbRA9iZ5G68uawryTTrhokhYd3PuSkNDK1UMILPX34UtyZH4DTunXVYJwQAzsteZYJrImVR8YhTXl3zVV7kPWSPfowq5y0sEl7o9fkXerL88JPkJ+RS/fAmOuQWzr5e18rjD25gZ/lw/Ort8HIuQbAaqzJUsjLmM5rz5/DS8HTOasTzdxBbeHb4UV6s8KRCWklOywuMsJrBN1YMI/TnrjkkN7z/sxIE0Dj64NWQjKl6HgVt7zPaap68povjTM4ptAotPzc3UKsOIS0tDZ16HJ+4VFMyoYH3KyIY7akBk6GXoFKjK8IzYz9qXSUVZWqkvtuxBvsuBo+k4qHrDE8ZwG+fPULSd2KUcsn5GFxKHZk+OB0iqvqdd1zqAEZcHkzMQ7updins59e5GDg+5ThFQUX87ONFhZVPoXQhqL6DotyfWeD2Ic4BTV2OTsvvHNL4EtzyE5F9WlTU1B3HLSeH3PGXECJKUJgXifyfH36MwLtV3BzthdFbXGljtBtCeoIH9X7foRjzBw1mcbSXL4/lnRk/wIw9DCj9gjFtecRiJf1k0rLh0e1sXAEP1sBHMjE1TK/0JUPfBQg4zslF6WCpZq/0qeZ2wm18947nxY2zCfg7lJ6uZx02Pvwx7iSCWWD+iWnUuxeJktwXavcQdzaTNjsJ25ZtI9ZHXM2VfvdNTvw8Ap8B1QyduA0JT2ItV5yiVOL1wHZGrxqIWTuUviYIEn596FecNMMYfPUy9i+L/XN8X0C7714C0k+SPzQYupNazoKGwQoBG5UM+YhbPBUr7j83XlHB/c4weexlOiZdxlH+nMi/WN9A7vlpZNvaskkzkQmTEkT+Gs8ampykDL02lHaP7uRh2jtgNiLzmsOdANjaCqFFMNJxIA/1ua63AmrxeuUSq0/+i1w6RkS6kZo17PeBb5NDcCvfxDjTGaDr/O8oHUgJT2d09XT8SwOoNIr3SlUR5ZzsgKMaO7T2baIeyialJwk3nbErCuCBPQton35dNNbdmM2nv6yj0b4F7cN23Kp9nNl83OtX6Lcy+u51pIKJ9a9/Q4N5JiBOcixd/TfGWQ40a51wUolBTAdzEwmpXffxrlcFkm6Gu7e9J2PvpgAS/IWviFRIsdFbAE6NMoTTOybhVeeOFxDz+G7RcdXmAywe+zflrQ9gpwjAXl4IWL6vCMdx+I1v4IWyYOw6BqOSi6W+PRc8xIBVRcyqBbv2wdyL2MbbhKB9+znmjL+F78oT9LWFuRJMpyYTOf0SEYkWEl91+132PfgRxuZALqlldDjtxRrsGzzsEvdP+IkH/3oAcsAoEQMHzoabNBXJeW73M1QkZiF90qqK3mTAGOlLbnAlqmU/4a/v/nZG7wapEm12Jv8s2MOsQ7OIOf8zpY8vhj5d+TTtxZgEd2Ym5dKk6t9fd+j50dzdO5EHBDPNYy17Yhe3OuxDjbQWaZDVdWJjrLYMEqQUVcfQ3qwi49fFnI7I5SOrQrW8xky8k85SXqpm9sEYJN7+IulH7OHzZ7sSxN+lDyKw8Bs2dKuBCwJ4tc6jdsAMpGM3MlXrgkIqJtgWmQYwqd7ACMHIbS04m8V78vtVpzjx2xo0jQ4se2YcppYfgAnk2i9hfc4+WkxmQopC0NhqRHulJM/naNuvwlep4av41wk3idVErnX68E9pMLVLfkdmkPEi4rmswaOVXK8uYkmYHFwq9oKdM9j6Y1R6s+lyBLb7prD2jYUonXyAU6Lxao0am2bwcHDGVSEmlhUqwjnyziCStFKuPPmdKO43dXTiVFWEl185n86czLmWT8FKpjkq8BSeT+RwVKjiuyJ7XL3Fa+1qJ1dKdLEcO1SGg3cTL1qRB2JVcs7U+ZFjimPWTx9z3fi2aKxQf4Hps06S8eMyRvw7F/fVRfS1i0EF2DKEiUtPUq0Sx+23a1PxHnqHK8oWfh52kb+klv16VFIm5Z51NBtHMNbjBPY24njsKyeBmxoVs+bvRoramg6EjbGeFJUL/94KpvRSAvVTbUBiA16TwWUwhvSX+WjWTp555S3ay30Jf0j8XQ63vcPsJdd4ZcI1bLT9+7HKTRoW3XeUqvVP881fD+E39gzW8ufOEjMhBj3ZVxLY6aHlHXkXmHKr7iLHhq5FZXTCodkBrUIvKthy1jTivHkx/noHqoOH4WeXJPrdfzwcaXn7IxLbHuD5yI0cbYsDKx2UZ8en0j62A2f1OsrzEiEdmoUYHM13kUr0HMh5CkN8LutrviZA/hATpPf3jpVIYF67ifTDUwE9jRG/c7dJSU9XVqWxlfXrDvDa+ZE88/ly2sZMw0npBECA22QOzTjHDFkIblv3MmvNdgTtEFA6gtyRj3xqyXn1UzzVUt4LbmFjo+WcDTXnGfbQTTZdW8hl92wQXFgjswJfDW1wbhbjFI8j+3sh65ae44LrUXpkENdH/sFoLykvVHvhV+aHBKt4UCKjHSlTL4zBscURL0HB6MSzonu6qTmRq+2p8MS3mCQmXjXu670fl0ckk5JwgykX7yO4WIHs+nMQ2wUECQhMOD2BQbcHYRbMNHWKvysHSTVzTfaU1blh9JZTZC8mjtnb1PHWPRv4wD6Rwm1zKJsvwZqKahJkLN+yElWHih+f+pO+9q5qCJef+h5ZsS9NLjaW90giY3xGNpfqNhK7Bj5zg+SCRHpUeg6ZFrO/7S4yiQvJnUWU6r35ykoeNP92ONf+Wo1m7gHW22ThaBsk+t35Y7+lKMCZjQZ/Nt55iwPLLS+wXunHNjR4ODajVzkg9CEfajRXWfD4fnaU2bOoMJLU2KJ+1yUI0Kns5JKkA4lWnP+QS+Qs3/sSLnVtxP36KbfNljzBvdO+4Ux1A0P1GZTfbsItZEjvPZFKBGzbbLHV2GL/0A7G2RlEBBWJIFAcVIzUIOVChSNFgjjmuVEbRvkPIfgNr2PK9HXYGx7GujWGk0KHVtHC7Q3DueS2jLf6VAIP1J9h/n3nWJufCCn9ijQJNOfxpFqOqdmRcYICo7ErNoqW1vG2czR3tfdhN+1vgsKqevdYZjMMaL/DA94wIV/G2GZ3JGYdXv4zOH3zeXSAUdVJnVs9CxVP8KCxlYdUb0KGDaS8jL/vXMpa9mFQwPwB0xjpcxJTWhE4R0LUc0jMesYlj8CrJIC5B+ZyYUZKv2f1/4r9T3jH/6b9B/b9Z/+nZjSK4Y6HG7ajbjGzNsCVQW7lCKaP6UnK2tsFs6X2BGDGUdqOvUT8+u4u2c13j/9AdGYkXr8uQblenKCUSAV0KnsGfv0BL9qZaRO+g5hXuv6Ayvp8xiUe4mzRBN44NhfPpDTAIk1plDpT3urMjPMRTLVT4hB7gx5pszWFDkhMep4dF8wc/7PEKSybZ5lZR5BeRYdBwgcJn1OhGwFWYm4KwZFmnRyjUUKTCcyCmH3z7tRLyMdc4fg3ywizM/Sb5F93vUJIvTvbbhgo8Psb6yAdwNtzA7+pzxGybh3PBzSJtLUBhmoEUs0SDKoOpvjV9ybcaqUBfPj4W8h1clQtKrxsxBuimbo6NHdHcM0umfdaOpkakcIQqyTfCEHLjWp3zCYJrZWjRRUvuvZS3nx0Ld8cn4rPt0+RtUT8HsRLGtgcVEiREMaoRZvpEKLoC/b95n6FrHA5z4d9RaSykif42fKsBYHXpx4gXNYMv8yn8Lmi3kJ/hX488iaI0G3CN38MwnRLwkBA4PKkM1yadIY3f3ibyg6xRnq7wp+9n61AbrAnZfJzjIsXJ1Hm2FxiycNneKNAT6LWDg9zFD39A7p+QEBilPDMl8+QFZUL4n0PJa2FHJ59jXzvruTJKMlTIr+AwCsfvkJORA4vLN/OL8quzaJUKmPmwVlEZkcib1fzZ1QBTz9mqVTLKn2DmYm/satmBH8XHmGC+LK4JnPitl0Vr9+aRaIB3CaJF6NrB0cx4nwMJ4G6kTf49j7x+Nuh5WRWO/LiNS8cg4Lpa/c1aqnY1pWoPSX1Y6KV1FBHUxbDEzZwLBxcU0qIcOt+F2T2JLd9z6kNlczzjSMnOYjYGeLEkFD9DDF3pOwae5QLYfnMM1gJTskdePxEPkatgamj3qXNQ1zpK9HV8vwXH/Hj9e9JzHuH4eIiTFpqPiG+rpLcPF+ix8KTTk5iELOzmk9i4/nz+zX8lrwS12f8evtCAmTK43k3Q8nQPBvWCCvwb0oCngXg1Zsjabwcyr4HTqOb/y9JZstzlhobGVu/kZoOWx4caEBmVmMtICpBj1NHGTbVcjyvj+ROoLiPYU/yobJqOUebVYRHTxD5r7V4UdrixFZvM2HOYjYpwD7DECS7wmgIKOHh1/9Ep+/AuqLXXlFDp66RgW++ziNjz/HxD12AtkQi46mv1lPub2BXd/7aOqFwrjSJ45kulDu2MMOugMHmB0VQSGd0Eac883G/PZAbqSNZN1JCz/ZZZ+jk4vRv0OtasNHaIDWLUS2z0psNtQIpnXBwxPsc1UbRF+yrae9KYtWG56H3ng9YeqZp3MfzeuI7OIbn0Xr7exbM1dAjZOsk7eT06i0Y7DtpCYXtHWIZiqckGopCqrl31T7e9GkBKwi/Rd/B5lnfElsUzOJN9+HmKa4csaOJ4dV+1CpNmGPBUyV+ljeMw8j5yB8Z4Db5Sn+2HwK/tMDKSZe47AXFFGO9gbAz1RKgVeOmbGe+m5YKU2f3c4GX4i5jG5zGzC0rePPIazw+yaVXVFVlbGTGnMsczfJhxtEZpMwUVzb90zaeR6uvMut6DEeuDEWSKGbe3z/6ISQjmvGprIEaqOpz3q6150nbNIjWRgfKosTXDNCKC181Qm5TGNOrT/bzf3rzMM63dbyzuBbB09KP53v9Sv5yeg2HkTU8O8qJHEcl1mBfTWQrxwO65fD0oEe8k/ymwZkWWSNPfvMkeWF5CFYqy0qZioDarnT8Z1NGUiOZBLzX1XtXZkvFueucHfgnIQYls8YVkGFU9+sl8caINGqHXyGqHPxlKmCayO9ll4+nbz6/Rbeyw6qn31yHRjoUeorcGsnwKeIRq15qeqmSuGJoMCUzu+YQqloJ1hCmzjaRt9Z8QKuHjknvHkAmiCVVHbQ3yD7UwaybSnzj9KgmHMP6HQrwephLE6/x/Ig/uS+ojr6mqvqLt198lZ8+msUyz1bcDalYkyIEQeCxnx6j1K+UP9b8gUwqwWg0UqRt5YDHBgAWvP0WuYOzGLvBiinb+hX+hY5UBVWx76FTOLuPwLq741Cn9zmaDCX1j6KrdETQVJBoJYgZkzGF8KsOvPfMF7TbmREtWfoWnnHpJK/OnZA7A/FxF0slA6z0vo37kjRmV4CP2aefX3bxLNMvVgFrWPBBV8N6XBL53biII8cycTAUsiLidRpU0wBLP56xitcpypmFb6Ir73mXcMQglm3fN+kdlCNhteQtXJRiwEEQoM0lAIPxEf69MoVZo/36nFXX/fORgkv4sxQ0OYMV2CfIbBldLOB3aCpxHpVIEsTfpUHlz4Z5u5DNOsH0lCZGWmWO/p13nQydiW9/X4jJrMfd6h2UCAIZMRkA3I29i9rgx49Wxw0z38Fx1jUuKJsZE63E3VWcBNdJ3ehUOFNzs53N2Q/zwGNiUsTmliCyjdlAGvYGDX1njBBDCgdCOvlznyP6xgLoA6WHmv9m/aAdnLwyhvYOC6Nf4buRD2YNZ6ihhYBPXiWmTyvX3Jo5rD8yhNLIm5zyz2SZi7j6/6avN7KXtzH85/tQaMH1DXHV9y7bMaS16eHoZDx7pKfTumWvvedReGooyi0zGRNUSMfDfX4ccFfF0FFZgVtDHBitqowBpHa8eGYJ8m2hNLQp0Yxx791V9BAVFZFDePveTyhUPis67rFzD5Cd7If3E1/xXWgLd3RP9N4ziUTOwH2zCMsPY8zKI7QHiL95GTo8/WrQ1zgx704M6Yl5In+dLpXocddI08K7RUms0Glw6Qv2tcqZ/vkKOmz9eejMEyLfAP11It68zHMtemRuVUi7JVdsDC2sstlPUXU0Z9duoUWtJURmkVAUBIGDsw/iXuvFLP0Q7khnigiEkZIyFtnK+XXsTdat3shVZQzWYJ9DUQ0NxwYz0m8KDraTsZWLeyHHq9qYpJDyYI0DgkJc7QjQrhqINvoyFV6VpDY69VNYmRytQLfkNPN8U7FpCqFnxhln2s+boRBZVMxXKpAZo0Vjr7a/zs9nH+d02JcUjdjI/S7vi477xFU3pu5LwB5YLTPjJLcCFjrK+fLRSfxZNJX7POFoa3dV4M2nwMYFQXUvZUHFGOaeZFyzFwXtdcC3ouPPSMhEMeQwS5phaPN6UZUEwP4cL0KB6auOUiLpFrVXefNI6mZqak2oIh3oDNPgqwqhp3OqWSJnj9sXaCc1US6fgMZWw8dWMYJGFcLIYggoduOBXUu4Ol0seR0jFPByUBsFX67GrdKbMwvb6MmfCQIMcLuIu20JEdfXEOR6hF1qcS/NTsGJ7EJPshfsptqrmnfNn4j8l4wT0TR2vbPJnmUM7M44tykiaDoxkpWpiXjUuOIdX4a9oYGefn82EjlFCQuotT+KQ7MDUrV4H1VvKqDDvJ2JOntejKuixJSONSHoH99SGszw6qUQJrpo8cvcAB5JYOuP2TaIzwMySRypRn45HpWtgWesD27sZGVeMM67p7L06aep9hwFWPo+LZj8G29HXqBFt4Ov7QNpN1rUbuS1aYTdOEy1RsdbBxOpD77NOKtDK8yfsSj6FvvfWE15uS8ux8Xv/wj7L0noaGHXwSVcixWr0SB34tShGWgaVHQMeB1FgLg3ndysZejwTArtfqGtVY1G8hp97fOAXEwbNvA+cI/+PdF33WQbzf3jLHOINUl7oVMj7S5ZTMv6hJSW9TyrFFeuvv37dDySE1G98Ak6pXgtdFS4cfS7VVDqQrhTCwFCeRdJZGIXkH/x2hNsS/ckI+IU+LsT35RKrBVods0Yz1+1Z/jIDczG/tJttfajGFEAayNzCHfVgiDelEdq3iRqhztRzOf0RIvyhV/rPk54l/JGupQxvz2LbNp5BLOBnnRutm4AjslNFAyqRhsznUDxkkSmzpm7JijX6Mk6uY0xg4Zb1VXDIKWOdkyEub5OHeGcS9/CaWU6CxZA4xH4+8obRN36kgfSXLkxWt6vZ99GVzOyp37GLDPwpJ8Gk34MPSBRrToKIRdCdU4sCQ4gNDANdJFg40yCMhizVom/rZJaWRU7cp7hTVV3zkmq4JBOSZNNDS4GOwaXwEibIb3xYE3BLmZ5fct7Mb8z+FrXe21j/elJ1TBsI+krS3AE9pdE0hliAUFNbYlc+tuRAXeCSX9uBw+721qIZ/pW4uRajqzainNeCIaAcjKq3+Zrq8Pb1cp548s3ODvuLGcnnO0FgSQS2BtbxyWtid/VWYxcc4xCRUwvaUdJI8ulSsqkZr5b/zU2grg6tMwwnHmnIxh/dAJ/rd6Im/NgHrYS5G7Te7CgXEJxSCbVD1VyXBlutauFMGUdMqd2nCt9uDczkr5WhxPpLrV8FF7LfLUCTFp6FJckgoTYtFiGlgcw5MET5NhYJOWVrv/QtrGeWZpsVNV3MYwSS8HuNPoS4FqL991lNKTHkvCiuC1MYuDf2LvZc72qCpCDYIlPJILA0RlHsdfLOaA8SaYsDDjT69/W6c9HLndZdnox2/ZMp/M3d5FcMcCLCcm0B7axt0SFt0n87UklUiKGCFSU1JK0P4TQ4SX8C6BvY4n7K8jHzCBX8jQZPojaB8iNTbyfkUTT4fGEvvk9q3xrKbBas/xMmdQGw5sp3qS/t5aC8RKwJrConDm3bBsLciajP12BOr4A633W964FuLe5sG74VXS2tRjNK7GGaEzIaDfDmHMhhNo+i63xYXqVj4AACnC9MJLLbvXMDLrFVnVXbGPbfpEFQR9SeDOWF4a+illi5pasax40myFTmcjGXzpxrHPhxytDyXowi2/nB+DYGU20TSYT5DKS25R43Gyj/m40mlfU2KZ0s+k8xvTmo09/F8JN7ZOs2XQNh8wDEPIg8UI9sY8c5tur48ku1NHUR/3q/zXri3f8b9p/YN9/9v8rG3A3nMnHxwIdmMcWY3pGSw/Y1zOpODXmMbLkS4K8TGBVayZBj8GzBtfkicRkRBGuFvf2cO48ypsDP6OwQ8F2eRuz+1QvCVI53404R1ynmcl+zZT26SMkNbcTln0Ep1wP0p1cWeI+sdc37dgkElIHYvtkK0mDziNYNbD2Vqj59K3NbL++hi+/fh/3Eb4ihq7dwB3sSILIqnMM21HI+E/FCcDjJhvMqlq+eeQnbASY3acU+ualQVzbHIfjzHN4DdbQF+wzCUqqjd7MGFDMUmcTIE72fRrWQc077/CWq5lVTmboZvn2JJbDc8NZunMp5xcWiMb5D81lwtjjpHw7m5lHhtO+SAwcvOVvoubtd/HI2ot383yRr8pgYF0VVDvdZfygOUwOFAciko5S3BvusKvSmZft/HmiQ2K1rHVZe7UT5V8uZejkkzSOF0ti2trY4nmtkjy398levBNvR0uFnashncCmRjrSAjhTHE39RFVvZyZBEOgMg12t8NGUWGxw4BGzufdeSAQBwSwgkRp5fdoSWh1mAhbJyxqTCwc04KtoZXHoJ+QYxoNVAvIh9Tk+qI9hb5s9rg79gRab8p84P/ITlldCjRFsEWudJ8ky6EzQ0uBa2X2+XYGMTF+Pm0M7Gsdmzo4/S4tDC09bSVl5tldy541nGDywghLvDjDaYJ04mmP/Csob96O4foesVi3z3hYnEes9mrkz8A6BxYHclYn7WwBcL/OgYMtEVmSHcmV1ST//cD81tYvO83P1ZuxcfLDG6WUdJXzgBs0meHr0Qtzsa4EiKD/AEMV6zoz4Gl2tA8Gp+zFFzaFnYw0wUXmccw2zcelUoZOBDHHVilmbjqRDy1HnN1FJpayz8pXYhDPl+O8klLYyLus13BKSsGZUp2hl1M48TtmxUQQcnIphsDj5g1TFkWPLaS9VIQP2daSIwD6JIPDIz48gN8hJAwLmWuaEMcNyKXDRcdsNtsfXcdBkkeqU6qqxL89D3+LJ4rOl2DiKWaHOHb8Tc6WYZhctXz69kUEuq0X+2BIdE397mYIwFZsdH+DV+eJn6Vloz6K/n+Ya8OOgW+x/WOSmQ7Bjzz3fI5cb2CCHij7v4FPFA6jPaCcuqJAajxqk3bs9F2MDZXbtSAQZL2x6gIBB2eR4VABdKOrBrFuM2BOMedFVcvQQahbfz59jC3CQ6nj3/Ejs2uwwWj9LQxvXpl3j53xnqt58lcBJYukZe6cI8n/7kaHpVbwD8OqdftKPd/1+YM99G/lg1AYmqfr0RhAEqryrCCgJoC7rT76Mc2RUN9HjfofLzJ6ey5TTIfzxxTJapzeJxu5vU9IpbWNc0m3ylBBulWyQoCdEDm3+5SS9dIoKW3GlTplsIF9/vhSVcyfvfriBa+rhWDPrjYKSI9OP8lxQC7OH5aL5H6Q95uyfg09RMPuf6mRAhBhod9Hn8tEHv7P2tgfu151ImdHll0igxKcKhArGnKug2NhApa6jF1zV2Pjx8oCrdAQpODbjJpeU4qSVvkXFjI1rKPOp4NR9m5hj+6LIv73FGY1WT1huGB3/QzXAPyUR+DW3sWvpdrQDfPv5mzrdyPxhCVO1TsQuWg9sEvkda1Uk3orl5UF3kRDY20euZ55ut20nILiRUo14DV9rn0OSG9x3ywPXenfkAeLgOzwzkPD0oTi12ROmMopkRxzR8E1DBxm34tjxozdxq7ohh9OTwHUYEmE1pyed5jTwG/Cksb+U22CVlmt7JuArT0cXfJ6+YN/jbvPJTPZji8qRBj8LsPDSiJuciL/BlyVdmRvragKJVIb6/DBCqrzYPGUJBpmBL7DM0fmaUgqCm5B3NPLrk8+RMb2BdVaciN9bBuF0xQkBaCnxwhkxcBvjNJIs+2G0S2ooqG7oFkm2WElHHR3SJmxf+IVJ9pCBeD4aJM+inUD8y/z5WuKMtL1rfbBBy8SbQ7Ct8mDP/L1o7Iw8aNWDy1co4+HffmBchYnbRiOD5eIkoUzXgJu5gf3zL9LgcJefRCkrSBy5gkzXwu7qiz5zt8nAvEm3+b7ejisHJuE6sA97BQiQthChVTG/zJvgPmoMtFcwPWQDp66MxzzQi1z5B12wgaaY1fr9lG15DYNaT267nIAll7EG+4xh1Qz7/FM+v+XPo/sSmPKouPQv3quZZjQ85vQhmO2xljisrj/E0gnPc+bWWGTX78E9XgxCeprKea0mBLtrg5g1NIf0AeJqdiRScsrcmHJpBGMGZRJlsMh0gvV7Zel/1WMV9fNpa/Tn5qhU2h33EG+2xL+DnMP5tu1dbnco2ej+Qj9lADMygmafp7PNhxVuFVxEfN4dgZVkL1CT17kPuc6ZKW2LCbLqiRPcnsiIrX6o2xWcnnGLvuYhV+Be503I0QRCh72D0PYaOFnYO9qqes58nUTItMssW/kGaGtA4YIgQItbMFKjju1HdFzwgZ+tZLRi9WvJufgPA1s9sGmSIxsqXjecTbu5mg7DA7/GS56NtLOYnsppAOkRRziTQNiTW9C4dK9ngfeA3AFsA/iwTsJ0ILgomEJt/7gq2/wmV59TEsdX1ETfhq60FAASmYKWs4/g3XaRrSu2YivE9oJ96+uKqNr4PDeHCtwKmIZNiDiRnJy+GmX6OXKm2vJbcwtDpBZwVcBIyeBUKkML2DzgKnaGKKs3BGrkCTw881GmnB+L7s+ZtASJgZQs8zAejP6EufvmIimUMH9lfyWIgjY12WG5qEnsR14skUWzz3k7C31gpz1U6GsAOwRdLao7OSiyjZwZIeGUFpbJLCCiyljP6sHFXGrWcvNqMn7iYi7O1iSR/ep4UkZe4YfZqUzxLBapZrxXGM6YjGBG1qcxbur3gLii8YX0RNyzNKw/NZn08WKJLgCzypNU73gU28xcqMnljRfE/jSvv/ir4CDNTkvwNFgC4ApJKF8cGIxXh4lN0XfxlYorhPbUfEep4TqOjRMYXXudqYnib6fav5HdC3czxXkuda6jEaQWsmheay17G6RcabvG6W3D6Qxx6ppNbIPAxpHwlosYwuFTtxwa5Tm02YjnWIAZdvUUbprP8HoFzQ/Y9/NvWbSPsHt2EG+4jyAXC4m1sGwdc/fN4cD8G6TEHWF4w68s6n5D9SY9Bwd1JZXDG5Zjb3JAZjX3C4LAU18/hU5hRDnDgVkxK0S/ucDBHVmZMx/7axgbfoelww/TKu2aJyUSmBn5JdqLrVw5NIq7IcvpnOckGi8RBJbsXIJRauTHdT+KKjQAfFx+5cSUNo4O8qSzs50ryp64SYLUKEVmkOE0SsrQoSkorNCOoLa3+efxt7l/6zuM+fJZnFeJyV1B8jwe37cQY6Mjnu//SJWT+Jtvywyg5N+R3FvkR0VwIcUPDyfQNan3njS4NmCSmNCfimb7wEtisE/fwkuDCvntcAc7vlqG8GAn463cNlIbJKMmEfTVVezrJhK0zioPoHyGBetaeDfNF89/5lB+T5PovD4uCoFkNVL7VsqTrhPXR7Fga/KTGLfkUBzuRXvYeJEPlTeXQ5/DFJTLhTsNDHN2FIkVV9rFM70QsK0AW/ha2gedwrIu6YFSrRj47Xl2a2sCmetoRGIqpicJLnTLBzdIa9HpPOg0i+O9NCd33OLz+fbaRGwCxNX/UoUXt53uRR72F8/f+w07WyeL/GUNgTh/9hhRky4wavVeJPqHsK6Qq2zzxvfdp/nJVord+Em8+QDi4wsSEm8kkq1u5bekCxx0+Ezkv9ppmds+HJ2OxNgEOCHDwPXP72V4jTOOE68wK660WwWla04xuih4+823iW6bxsGQeIqElYAlPh/UuRCPGwPRG09Ry3fkxzli3TP74X1jCclx5+l1V9Arut4Ra4ncRp5EcTiEAAIY5nqqH9h31WxLvUsli+wgwRXSze1Wfglz9s+h3c6J2KWlrIx7CzTDwMaZKOO3BHVe5mDnbD558mXcm6fylo1lc7redIPUC62oaj7BpVqG5mWnXl+j9zyW7NhKR2UQ4zI3MHTYORSGTfSuxVIFD6RfgEApyCKobT1EmBWQsyf7dexa7jJ0Sh4PKfKZ6r3Zck8bbvKvcymPh8MPHl3VyQNMYtLPMo8U6pO88DfLOWDypFm4AcxGECBEkJBvkKGX63mkvZ3HFJN7CRNSwciiNUf4ZeERLtmBySymMZkFBXf9SmibdZA6tzrc+8RVerOam5kBhOUHUzD4FkaVOHZ/1v8sAfdo2LNNwnOB4v7PAC7aTkanD8QYUocxotFCJjWbeEuWxcmKYSiThzDLqR7XgIVWmS6IuLoZRUfXPnzWvD/BfKE3YGwIbeP0mm3ct/VTgnPuYmcSK9bEZcWgrKzijb/eIGPaaQTM9BDVpEYN9ztA851gzm66B5eVl0Vje84xech1MsNzmICYyATgLe8k/eAEHr2SwICvdvTz3zf/I3ZvHsfi3YtJj+8OFMxGHI1X8bSNIksj9KqW9twSiamDeYOK+Ec4w86jSXx/t5R/JlqehwEVPzz4Oo4ezVSFjMDPS0wun+32Nod0L1HT9AW3UuNJf0QqIvUcvRuJ9PvpyGYfombkYcxmsepMtmocP39bwuDbMUgGFNLWfgNrsE/q/i7JdQdpEG5QY6RXvaLJeRXffeGMMi8Fm5dt0Cq1vftasxnqJV74HpiOyb+Us+POYh9oj6zxMjtCuwi++pOv8/QhMLndRNNsg8nYvU4PfAeDfTRr2uZx68oQrsZUAlomDPiUODdfUPviYjRzRLuEGr8Q9gx+Bi+r+fE/+9+1/8C+/+z/VzYvvgC9upOqi4P4Vd7ICJOEnu2FUdfApNL3aU51oMlDRUZnuyihG2DIocTFjhvLzxPywB2qXadh3US3qOkyKe+PocM9mt+W/cPcPqCZIJGxqw12JZ3nTeBjyVciSczWqpcJ05goc1dyY9hnzLLa94yOL6DaroPzpV9wrvB1dlgpvUnk7nx0cQdlRTrcb+dxx1vM3umx8LaLtGcYcDRkYp0c+r7KG02HDo2tBoPc0C+RckHpgVtsOp8Mz+GCm5y+ZmquY/zef9lXOpNHp92gw6o5K8BwyS4u3S3kD5+NXLRX8VM3EjPScJSrTRG8r3EgftoVBkRViMZ90jSY5doCZo9PJmjMXZqUq0T+CoOSCjP8GPcVU7yWgkkD3ZrhZqmapm0zcdfY8/prT3K3fTXWzbFbvBbx4+8abNpLKYvbSLW2L3cHXupIICoyi1UNfoQ4nBP5pBIJvq0t+FU9QK3Lp8z3tUx104R72X9xHpdHFXN4+nkW21rAUQGBny+HkV/qwe34JZikJswYe++50lBP7FNbSWm2wd3oTGWHEWuw74YpluufriQ8L5znvOYwaouHtYgWRUZ3sg4FoHZt4oN7+1esGKS2XDo9iGHXonn5wWMU6v7FGoCKs6knbvkNOl9exyKljLBh54BpODRfYtODZ5hfcYZb3UGsNcPxjnI4qSMP4tzkzKDjn6D3qwG+7/UnpR9HstNMWcJEqmMHMauPBMzT46/TOrGRx2pAauyTOAW+uRbDjOxQ3JLSeTSmtJ8/xfVPdmgBJ7Dvk9wxOg3ivu9mEHJlGFudtXg/VNpVL2sXTLt9AvZD5DTkJrBg/I9IQpXQyw+G3z3tKHzjE6b9eQ/2R2fisj0f6K5kM+mJKHkfVWY477wxFNf24VhXc2U3pdOh+xh17QhkWmcczWJGapXUj4/V1axuV5GUEYXMLE5qNZvgnZBr3HumK8mgdxSD6AHaVLQzznPaYOCuj4kK6YpekZbvmi+R19JKi+Ye7s21JXGgBeDJ12s5ueIGta3eTPzhJ1TRSqwpYlfbK1FMvYBG3Urx4GY2tdwR/e44p3wMYVKk6aEY5Fux3ugBrAiqxGZ8A5eyfajy7Q8420o6UARUYpcWxaQjLjy4ag3WQjCmilLm7R3KzdmHGT7yLgpDNeDC4Kaz3PfxReKSpzLgz4FE6tSUyiw7qroADTsX7yS0IJKFe94k/0Xx5uPvZnvM0npSH/mTwQqQGywJHkGi4IcmuGrWMml4CUpfcULARmqDVDkK2E2jUyOlQn8AaVfrM9wJ/orPG+F0a7xVlwCwaS9hnSPUtTjjU+7CDZ2l2uaGNpg07R2a6x2oT4kkPUmcFBh+MZGhyfH8tvY3PnVtIEuQiY67JXM4x6vs+VF5nFgxARJBIuXUxNPIlFpuXX+YWSNUImEntdBB64ibfCKBT8vgyP9Q2bfUTUv+LRcObhlK2BIxmFhvE838Coj9ei2zAouRvGUZP0wJrWb45ZEu8aI3BEvvUme1H9UV99DqcIqd3iUUd0RjnW6zN7fjUe9KUUgBRcFFCHLxef15x0jE9ShW3Y1FFp3XryLx/IBz5A/sguhC6VudBEKHmZiMGJoBTVVZP//UCcnYGcrR/3kf0ocsCQF/STWvf/wSmaH5RC7eRagQKerrKy91oDR1Mqty4zDkOjBttjgJ/pQ2kqI8DxJe+ZWREZViUM3USelZZ3SV7eSHNnIb166Z2X0s2Ifj15LNDi/4olrO7Q5Jn1W6yyamRrH2wDhyBRMl4yqZan3NAuw7HQcZEbRdHUb4FktPknkl0eSRwfp9y7FNd2X0DstcJAEiioMIzYxmUMog0mPSET6wHLe+vYTjqz7BvsSXcUenYVCKE17lkiK+eeMdzIIZs8TMF/LDIv8oYSWL5u3n5Zc20eYRzMQ+xPsS2+G8szcGdbua5yecZYvXcJF/ir0fti9s4eDNENZ4diLR1wNO+LWd4KHyMDJTwzkw6zByg5No3CXzBB6u2E2bQSBCIsUZcWI0IPcDqg7Zc6+DE4EhoDI1ivyrBq6mOciM46Z5mELEsQsKFzyyvbD1z2HciDcY7ybu5wrwee5khn7kTwIw6V5xfGEUpEjH3KE9qoqU5Gs4WoEOMrmRRU/sYlOtDTW5AdxQi3tLv9sYTpGkhJVXE3EpCUC/TnzdWz20pDbasWnd/egGmnjfiqHS2Z5B7XszCWx0AM7i3ZyLtUqEl6kcp8vDyU2JYFdmKL6zxUkpzGaG3OxaP6fMTUWpHihyq9pz2F45gMyfl2IX8DxC0xR6AOnN/76NtDCPZRPj+SbxLIU2lu9SbW7F6Vg2kRpfTk6OI9e2U3TcImksD5TC3OTR5JyOJezlYFHXvVCbtbwy+Q5RRVBmBJ1RPP5JtQfXi7uqa2aV9+8r5uD7OpsW26OkifB5h2jTP4V1lH3EHEpR+EFaXEvwapczrTsOVepz8Z/SRnZBAsG394IhButKSIe4INLGP8GAmj0M3O1AUlIfTVaDHpt2Da5TM1kV+wdl+vuxBvuONz2L4HuD9+2LcDbGdcU1o7YAYG/Q4jzwaZJHPco+33b+0E+lr1U3W+TEpnvfEPkEAep94ygITKMoqIgYmWVxaZDJkXvWo28N5dudn7LwmUBRFyxt0mgynEZSXHCA5FoVO6daACaZroFz89P4tgmeqgX3PhKg7kofwipf4VrcD+QEFDPR/VGRvyfudKtzQ2qU9tebAmacHE70nQHYv7gHc6svYGH+V8sC2Z3pjc2pqVSNK2DFWCcA2hXuHIlwpcNHx8A7MXSqWkXrispQz7uu8Ia+kxT6/2yrQk11fAqKBif8901Hu1p8XfkDKzgfe4C3PnqDc2cnEWoSV1IG5roz4tQI/MNLGBLTp+EOkNZynpyA33GJHIi/4Z1+fpOgxLG0g9e3v0r+AgtgnSsdSsNhAxObnJm4dz6XZ4mBRGVxNUu2j6E1tpXRQZ/hpVkFVmB5lFcd7f53OZ/2LjkZ81hilXw3yuw5tWUyI5K7VFGuzOt+n8bsBKDt7Ft8+NzjVMt0vLj2V14ID+133k8WxDIjORZ/uzaQ9Cfu+Mm6egV+m/o2L4/ofvcN7YzxaqU+Mo/ARiUD0gZjG2pFGjMZ+aLVn/xyFx5OKsbfsQCpxKLMYmNsQRlYgdbWwEurvudG8yNY94OvsX2CPc+p8RqVxsz7ulpPXLTiI31b50jkpUAkahOpwY8ThHg/Eym9QdyCy5w4kcjfjSF0+mcjrkYHg8KOzm6iUg+hLa7hK46uucCA3FwG3qrkZO7rHLDKA2TrZaTXBZHle5DpU53R+IhjXB9ZGQnzLvHvTwvZ9MpzJB1ZI/J/XBnEyAZ7igfe4U7cHda7vwjKrlhX1prHAR/YPOYmG8NzUTj0EQKXO3LW/0muzz8D2UoqhWDetPbn/cL9ATv4yf1+0oUI/DotRKNLUk9OhV/ikrqIVP98/ALEJKTOZntGnYilZdZJhs1ORtJnL/SP80Y8xhlYHOVHsE8GWNUFFjcXcibxfkKy2pl9aDYpvmeBTy33Gph+ZDr+pf78+vCv/WJFgK25CWRunUnRqCu0LhcDImp9La+cno7i/HBKg8sJtZKSvefmAOzTZlKn+geVUUZ2w70Ms1Jrclf/TmNYHmX7/kRRL5byFgSoC0wky2cvvxbDOFU0otDHXsLpCaeZNqCEZfZwoE98IkgEjFIjThVONF/twzygK5cw4W4cklpXLoy90O+69xj1KAbkEZMRQ22Of68/134WOyI3oAi2I23CMb5DTq6VUpNK0BKvMuNcWMCp04kYZ51lgtUecK32BGePBwOTMcoMtM0Wr7Uu9Y5EZUdxdH0U6Ym54A3zNAJsBYVUw4MjT1Ls7s3AQbmk6mNw7NOzb0jeVpSKDq66ziCksoxf51vmdqW5k4S0gdT4N/C5zRaOlwbxi8ILBZDdmkj+DjUdozTgBdY94tA1s7BpOsKZubTbutBm34hMsMSxJqUH3ifGMzB9IBoBCj3Dieuj/9iqa2XXml3EFn9HUK14fRAEaPSJwXW8E4f32ZEve4DHevrb2YXwmcaday1NyHXgodZjLxGTTYd61uLx+A2+euoJyioC8Hi7a18tkcDDv09nRPIQPO7JQOfogTDAcsP0Uk/cCsBGa0NnUQwS72aRiLKjRMHMjo20u+zkjXR3qu36E6n9yn0Zd34cw2JK8JCJVSJ21w7C4LiLO6u3kCK1568+Y9frj7N7133ogeXLizm30HLPJklrqV56grs34khISaAswHLDhrZNonpcHJeqVjJ1+m5cAw2iGzpDVc5MBdSMqmXmvGT0NuI+tD4FHsza2TXHvBNbJRprY2riuYIY/s0O5ezoC7QpvUQgo5O5kVe+fIYS/1LOLNuFQiqeiwAeyxpDQpUDwwdn0NpHvcVkNvF4yvM0DvgNZ78MnOy75wMbR545+wJ2mw3UTX6ISJ9Usgyr6cnvmRS+RNsUYUwq4qV3Xye/SZzHbbKJJi0ymRZnBQ+OPYfgWI21wsQQxRZWTP+UJWG+/CW/xlPG30Xj98ht8Rp9gfsHVLI8ADR98tMSQSCwOJB6tzp+mredj6VPi/y2ilAKg9dQ47CFIW3OyMxd81FGexX/Jn6Gr5c/z3/7FJL4TGQL2gBnzGYQFAK/P/A7ert2onzrcfYaQYlBwZXy8RgNjpTpZiIE1WKcJpBhKCDY+wgPLOz6pvOy/6ZG1UybcznXJ17Gxa6WiNuZELQEBn0KDtH8cvNrahyOgYeFfPGf/e/bf2Dff/Z/ZhKJhMjISCRW1KAZ0aVIBhQxIKGbzWEFWDR2lBBgV0RmmAOvrtpElHkR1h002qTOfLH+GVTRZYStu9Iv2XaNMK6O/osxF8cw78A9yD4UVysICAy6NQj3WneOTzsuOi+AKokv+pdf4VRVHJ7i/DpPTknnarwS3Xt/0G6rp7R1PRE9vSikStKvx9GsPM3Wp77GUz0crDo+2DecYmLnTRoCIgkZk4VBHSQ6dsTdAMbvX0hJeC55Q+70CwqVIRFs9/qF451yAiqnshyxlbV+BDgQf2MIecH19DWfDgVD0oxIzy6EcWN673mw03h2bwwiTC1D9s57lHWIq+9sNDICywLxDKllVUgtf0nFQXhAi4yw0nhSWiKQDHBl+krL/RQEAdd6V+xb7fno4g4CogNFQc7ZkhPsnvshK7Yv46133sK4uf8mU2drosGlgcIrI6mYmydKHgHMN7xIavIgfvx1BFnyl4ENABx3EKhfsZdcj1LCvBtwlIoBj6MXY0lITeAVgwz3QHHwpDQ184oLvJkSyh+/L+Pm0gpmW1oEICDgUd2VOGmWVZKrKQAr/k6uZBlHJvzBcId8MurkiLszQpXjJM4frMWt3o3v9jcy5IUEkV9jt4MXd1SjiviQ+f71mIxdwIPJKYlXSqIpbbFlnssNltiBoCkGhy52u16pZu/U44yp9mWep4YSd3E5va2HmYhR+aQ5xFPteIs2QzxYpctilZ1ogEM+4PE/bPjLAmvZtmwbW+KVqN0H9PMDSLTNTGl7m1Fh58BwHnoaYMvt+f/Ye6swqa5u+/u3y6va3d0N6EYad3cLJBAIhEDsjQvxEHf3ECGBEALB3d2hjXZ3d6kur++iga7dfa7P+Z7n/647elK7dm1Za645xhwj072OUMDQKfBV622wT2aHq+UKTw+/wheq4fwUf4Oz3SXYAl+Xur3ItUKQbzXhKgN6qU2h2axn9ZQ0jobU8MHWFZhjxKCBwtzFwoOzaXdq492nvmSY6j8i8sCd98y6YhfrXaWcNz6FreSOFSvFAVcoCA9jrFLCl16nwcZdzN1cytx7z/NPOZTpoV3oLeaZJd0oHWD43sl4Vg/D/0yv5GSnxcJO9Q1Ugi/PzVbg4tsu+s2VRPLhoLeJsyiYXOFCs0LMtj7t7knHqs0434phqDaQutZR2BbTYgOG4LdkF3kpTzBNIi54AUziNO/5yXj50+l0KfQIK8TzTZRPDXUzCgiKqOQbLygxFAIxFNon8XPhBdoUZt569jNkwLKy3jljVWgmw+Nzee2omliJnDgXMShccvRx4k4JzHv0DzYML+W6qVdmSJCpOfXDIkJKQvhr8kvEubmJpTs6Cnnh/sf5Y4YzX0n3MdP9cfqOSdZjjPr5EaTV3ux78pYopurIYcqu+aReiWfCd+/zqMYebr+dV/SRJL76EhHBLfy9VmDRKHEhZENwM9cqWnj8+6eoDgnC+bHeji6rwpVjl2MwVnkiG9MFRgV3zNkBlFYtqhFpyG8koDhdT35ELtDbhTxAmsrHwbD+5UfpsI5C9Vj/+xW5+AQl12Opt99KtTaGWJveK4PMnfzrMfj51mIekoHU2lN0EAR44cgscq7G0/XexzztbqZVVwa3e86DHUJYunMWHZokXrzvRQLw5mub75znd4EFP1/i4Tp4TAOFiMEUh1Y74rPicRucxdiJqcj6SG57yvSEmTS83BjCDdf+0pCuTg24vfgrR5qUHJWvZV6fuL9ai1dgAykeCjxt9Hrmyy+SExFNfaM7b7z1BlfmiyXmDhXGk3gokOYRehLju7A6B4vi4+dcpXOqlDXaGhxr4aBNzKwJ5HT4I9QO3cWlpG+IvrN5bboKpnacrXHYHZzE9P1jmA4ofu4DMAGujT1z6olJJ3G29F2x4NvxZxip7MTZuJZZDr3l+YFVx/A9W4jcmI9XfBbujr1XRGLp4rcXd7KpRYJk3xRGhYjXLDd9PnWh8KpjFXKfZkKdh4jiHrJ2htpZSNWDwdrfr/VWfTSbXozDyVhIl6MJ+oiTCoLAgIwB+Ff5U+VXhTBK/PkC4WX+KX2QI+NCeLcRqlwHE+UsIyU1gvpln7N3poQpGgiTiAtiZkGFotyXySVhDDkxCc8HxQzfs/a3cB4vwYdu5lUHcyNE3IF8qfEp8kvrsU+1EqoRM54B7r+VjqWoCMcoE052ddiCNABLPF6jWLIFXJSUeK4T5xdKd0ZUtQAtjFdU3y7zOYLLQCRSC/EjMrleCediUkgwiSn9coMUj3YPLo28xKWRlxgoLBbFP94yBaczyVT6VZLpL1ZTaFAO4HLoeeylVt6ck0aW62xswb5Wh0nsTNqIybWSQr9KnpCI5emw6PlpRhb7wxp5IuUob00PxFYYW2rtpul4z/3196ykue08d8A+ZXMNzpXptKcmsKn2KSa/20tLMCv9yG8fgdas48LPC5EPFUsdC4LA0m1L8a4Lot0xGKNGDPCn1QRTdMiR1+KLuD+uhnNWcY4R4XuOyb/tI7/ck1p1/+KOIEBtyGCygp5lg6GRFnsxaFGlLmLPvL8JKgvin1thVMh71kxd0+e8OuZH7uu6D7djUThG12AL9kXwKVtWv8dPxz8iPvkGZpdBouMOaHgT8yklm9TZPKoxcCxBje03Zwxqp9sQyITMcsICxYCD0mpgtUszlw2D+Lb7DE3S/oBzkeE5ut47yxmjiXT5YHbZxCSWTr67fwB7qoN4QKUg0vX83Vi25wtcGJtNSHYZA69u41iOmoU2Bd8R4aNw8pQT//Y0yoLEqhhWmYYvWuB6mTsv/vI8+mHi7mh/hZLVmf5kqhfz7+hfEfrYUblIGnn6xAz8AxqZsGYfUut6+o5Hh+TR4axlatRRUhq9sM1PBEHg4Z97XIjT7Z9muaIHWOs0m9ia8BVYBd7+9A1cvZrwk/eupZ3KYJ5+cR0uVb5M9H6GgJA6YGvv9bY3cGbxHobtm8Xwq8NpvE9MekhUdhEMfPHoV1gkFpYYX8L1toctgCKqilKXWwSGBNPmJO5eBtB0nuO9PDmhI49R7KcEMdRCyY13GXrOhMreyENBpaLf+8/Sf3BrcuOJqgiWhovzokVRqUStvsX+S+NIPxCDx1zxu/G2dxmuSJirfZ1Cp+/pMuxCpbK7e2yNVkOXpov0UUkEh08UfbZeE8slvy0MuTmEB399EOFtMWENQKhzQWqWsWjuZWQuPv3if5pcufHjYqqN++GR23KKxna+GV3Et/FFOPx8P+rTc/Gfa/ObgdAbSbSfTWTPbivOkUbW2Zh8qg31fPTCv7xf78RbZw/gHRqI7aolcfCmJnw0N/3P4dAGqdkfc3/wnesJGR1+HHjkEywSC/aG49xvOYmtB6he8KZwQA66X+aiuJaIxhYJBxL0T7BS38qQt97E7ck/UBoqAW+0ikBOd4MBExVuvyMIAkbzcuS3uynPmYJofXcmSYKVVWt3EaOczBqb46YbRvG6/xeETw0louQpPFXieXBS2EEyxsLRQa4YZS0iwqarTIb7xeHEbpvMnKc30eXVxz9XqkSueZkcx04K575PXKfYAqKw8gy5u+V0qEuoCn0dT5tbnWP24WgbxLt28H10B8c7xECi1LuTvY/+xHr3MJY4msh2FpNEOly7yRl/kS8cYvBS68BG5FDfVcanAZcp1/sRNu80+IoVDzSmViIFGa7Vfrz32Qu4/t2fLGpq7PnMiKogzMpropidvhqXtAS0wNfJ53hX0TvfzMGJnDY3HOQQJLPgbC4Vffb76SGUdgQxJSgFQVDbqu6hNzQzeth9KFur8b34KOHDxA/JaLtLrFlxjrc2TeLe48ks+FnccTvN4SwjPznMM/88gKO1v3KAsz6X15/ZyRebpvJ8ykjsppVh2xkoEQT2zN/DoZmHQHCnUdmTO0oFCRdHX0RmdsRkhiaLRASWRFryeVQbyBdlIZRnhJM6RCvydDd6RJBwbyGn1bm8FlnMPuUj2AIxihnZnJ9+AOftc2jw7cLR5jmxk7cwK6KG8/7VzKuBoPpBfGLDl5NI4DGH50j9K4JE7iUnOgdhQe+5uZqqef3nj9mQEUXQvljOD0xHouy5t3uEYi6t+wmVXsW7qSNZNjwNumtA7QMSGWYPfxyGK6geeoSfhl/hp1YFd+wlJFYzhYnp1PhVcTn5KlaJlcsyGyFxUxffGQ8xROpIfkMBqxOnkyV9Cm7v+CP9f8TR91uubZ2DS4qK0uTXe2uPdoH8onNn6lf3MrvZjbCh2Qx++ENs/aMvGfPZvx+aRs6jzaGQe4XPWUjPban2rSV1UCqDrgQzt/gxokb0Pr8SiUBgWSChxaFIzo7nykixaoG3LINNk+fw1Z57ST2ejCRETPhxUlSza34KG7udUP2wgmGv/C2Kn2mNxvP4ZNw77fl33qV+YF+KJhjLgBwsWdFE1Xxo8xgJTDaO41TbWWbMcWCoLAR7G9n2TkM+6rnn2V+YzcXG8YSYn76rrgIwt9OBjqsT+du5GffILNLN+aLv9fdopmRwCqbkNP4MqOY5my2DTB3AH4fHo6hw4uQr7xNpEufHcZZ0WoNkKD1b2BIK202XsM2BAdzbfAkt8WS3fxmBbd+K5P8tVgvNl88R3TCWz977je87ezO2i917SPQYgP8tJfL9swh5pNe+QiqVYTI6IQhWUu/rQi33Efn6JrtP4UTIFGSCgYUxduR2TgMblZPORhm/blhO5JBSmmZcRC2IQfZO125OTr5E0h+rybtRyQtzxfsoT1MpJW98w44OqDX3dH3aDomlCy9dFgNPzmLeuj2YdXlAAjLBRE1cLkXR+YyvDGKEezcyaxd37IWS20+RPr6CudVw2h/2WirYV5tBtvQMW/0gI2wll5ULKfdaTXrUJtABpi64/ih2Cn8eSjqLMj6fx926+Mwb6C6DnE9h4Ie06toJufE7Q+o68E6cS9XA/rXD/1fG/4R3/G+O/4J9/x3/p0OhELMukkvc6JbW4yUFRwlIbPxlzDJXnpi+iy4rYOlflGqR+XF56DYsEgsHD6xixYN2oo4UiVTCycknGemsYaI6BKWsz2SJmacqoihNjWagYRIxfeRSamQxvNhsZWxLI0ujIyiteIABA3o8Ix4+WEWXTold4FrsHdrRd1XB7TJKl6kdt5KtuFk6+HxNC7d0aeLzrviM9hPJpE/6gU2yAF7prmakjazCrIhSzBONBJ4awXg76V2W4Z3xpLOae/JnUvT7EK4u6M/8qVM4In1yE+2nk4lyb+/p3bZJDhOrn6Ezs4fV2NY5AG6LuXW6PUxmZB41bkf5vBoizbHYumsMKrcy/c/V7Fy8A5kli5dixF0lr7dbub5rAUbgm7YqptvItMiNDXzw5mY2pgaQs9fAZYMjy2x6WlSmFsKiy7FMP0tcVSCdQn9JtHscW8mbcokfBqbjYR3JSzYxk8VIQ3wr0uwOPnviRRI3hN1Vx/7VEkBB1F4WWULY4qrltLy32CwIAmfHneXGkBs89OtDuA/LFD1nnfJAJu2LQ1nriW7sWaxe4gJhrLQY9Qe/sLEd2i0QbRVvrie5rqa4YQmk/EOrmx0itJqeZ3rHPTtISkmixXkQg53ni+ImwQF1zkUywhtJ8szkT6cvGQy4eo6gYuduBnaVEth0AKeEEoR7etlnGrqJlMNN7yrmeVUxTL5adC9P+I7jUHgI7t6rGCWTUtayhRE20Nfb+z9Ak13BAK886hKKGGK7gwUeDK2kKbyB/1w9w9Lh4/pJrl7NWknCRR+atE6kuziTaMPwkVgtFA5N4f0hKRis4GO9DY7K7Ohss+PnD5fjH5WLq52JeLmfzbYEVJ1yYktj2RubTotrC7/LbCQdpCpe8M2nxrGEe84k4xMgBrpdjKWsXnuAH7VGmiygt4rnomhLNt/tuZf6WxH88fU7eMjEhQ7BpOUXT7g6/RzGRi/yOnxEBYlMzSxmpf2BSqdkrVKGl9DbEWatWEhcZjANrhMQ5DFEOPWyiiWCgGuTKxKLgY/jmlFK1Jy0/d7bj6MsI4bUPQvIfkAM3OaZPDnXCe8OyeFB1xz2GVaJ4pc63+P06ff4fOoQnFRNYGtGDpQKYbz/TBz2rU60PPY7PuZgUfyBkFT8PLW88/MzvJEdw0Pf9BSHa+2iuXl4BN5KN8oH90juIPS+8xrBhJsU8mPzqR17g07ZHNFxg+T30+l6neKCYJZ2S3giobc7TyJIaHZtRqVTkRG8lm7lcGw7Fo3GTnycUnGzCpy0h3zEYB7A+A4jx6t7CqqWPpmPziWZXQ7bkYQXkaIHg8Lz7gwsESQ9BVFnH9730OHnIAZX3ZNcCfD2ouAXOYnueSikvQe3qjz4cNE24orCmb1TjcZeCvT6DMXKdVwOgNcO+SNPH0S7UVzIriGQt155CE25F/oApaj4Az3zVXK1hfEvfcl/nEGpnQ82YJ8gCEw9NhV3h26+m3OL1tuSOoIAOZIQ9N4StNcTuOTdScyo3udfoS8gVn2aqiI/3njnDS7NEksn1Vmn8U1FN177fcm6OBTXb8VEjwcmXkUx5TCbLu3hQuHPbO7bKawqw/1yMmf3jEOy4CZ9h52smyUDKrjZCCfqDtC3O/WdikRqrTe4/urnWFX7udOxa3X+kNYJRRQVWXEpO02Qh7jbq32QwA/uW/Fve4Xi9gdY36fJ/uOMFPIr60E3j2L3RhGxpl3fxq5JcQjA/Q5gNd6eT4Z8C3InclIbKEox4g8kjk+hTiZeGwAiQmtIWfMn+U6NjBJG9Yu3uLWSuXgfX1jscTFogJ71w66pCbusM5QMnEdGwFoG2DyCEqmSX7pNXLZC9KKDGCS2TsOgk7vz0abJdCoFvrpvD6ld4gLjdPkV1msDeSXLHRfXdhwTG0Txrzsb8fU14DkkG48xaSB2v8OxuwCPB3fCO08xuyAWjUXczWCVqgk6dZzhNZM4OfkkEqkSqUxKu2YAT3XqQQrvffAMCUn5ovVQQxdrfnsQvVqHMCSDAnuxp9kxVwdSx+9k9R8r+XmPD2Fjxb/rl93HGXJ5AAef+Qazs0ksLmoxEa49QHdmBSvm/EGe1AUQP4fBwQ2Urx3Ataq5yPs8J4IgEFoUytJ/luKg/oWhw3YBPcDcNskC6pt3c0UHPlLQmMUyhjMbLbj88jjd9+zn6VmpZJgrsO1WP+XaRPCgVE5MPoFFJf5iq0TBv3P2A7DJDJO6FTb9LjAmaDm765dzI3EMzQ7F/bskBCnnPKrINWiRphn7dT85+8xiT4SS7AmPURNQyWb783eh3aDlb1HfbcX8ky/FOV50O/cqMVR017J5Zh7d8mKc6wsw2tlhY3eJi7WOyAZvarzL+e2er7nH/nVW2fyPrJoQkvYlUbRvMu8FVzJmj1ih4kj7X+y5cZ6auKkodUGkik8bhbGMiLElHPe+ARZE8rsAftI67u30JHD7Uk7O6PU/ypHE8te2Eeiauhj39bcMsG8HG7HvHTVHiOuSckhdz4ma/SztY9XW7HaB6Gl2ZJV6odSORZgvjuvCF2Ft0jDyr6dwGu0MPAVbb5/b3BJGWd+h7Pw4vvcNY8hgMagGMEFZyITgbpSFoDeKfe8EQUKxUEvl6YFIzzyP78eb78Zi5TOpqZtJR/DTqCJ24e8qlpQaLclB6Hbg6H3NjPduR2Fs5I6Um4OdHze//5ygki58nKoJc80EG9dgtSEHu/x8kpRq/npA4A+TuAAZIBThUedDR5cKJwkY/oce5+txxXTdCuSv7cksXD8B217gYZYsqsYWcNEQQJP7oLt/l2DBRQI6qxWfeQLDgy+JpB89HUOpd/JE1WzAXjDSLM3EdkxTXOe3UPhiyQkeeuAsWzRiJZJXZc2YKiOYIa2h06G/t8y6mCxmD68n6egLzJAs6hc3WDTkXB6AQmZhQOIx+oJ9CZYT1DkNoWTpKazOk7mzM4ixXOXLxFpeaKwlcUYWFQYx6JvtoKFjyCX+VbriFDseZ0c/bFeOn84PxevvESwyKPAdnYLV3A23ZYulhhbW/mc3B9tVHPQykd25BFvVGIkgcGDOAdyUJkIz47CzVPX7XRqPdm7O24+fUyhB1nv6xY/tHYui3g1P31yaWj4C1oPChZmVSkpMeryG5zBg+GXk0l7IQZDIeDnuKiq/HMYde4oKnT/YSLnpVX5MrYK2AkcmbL7KxakZPEwvwHSh8R8uz9+Mc6qBvb/NojFyEA+IuLsCepWe16QOLPJJ45+KBrChNbjbr2XTjodwGF5GpsKO+TKxUkSVLp3s4z02IIs8dbTfltcTAl7h209GMra1jZCcVLSJt9DrZyDXeN2+nhJKQkpQGQKYkl7L6D58HoPVjgoTuAwtYEZEDUpzFdCb5w5w2oZDgJSjWHnYCSJPDOzxiPMci8xzJNvrf8PgcoB5dkYaZP39fzpu5DFzj4SRq/3IcxM//3sUSeSn56PRNZK9xIlQ6x7uqKCoG2U8+8WzVI+8RuSyC1yWiok3T3jcZGZQDcs+fZRLvz7FuqPiHHhnYAbeFjm/rVtKZaSCjTbd6HKLFuGn5cgqPBj/7SfUtIvlMF31ZXz1xAH+8ujCXOOBTEQ/6RlfDRlE2ZgN3OsA7YYJPGr7mzUR7HtuPaUmK/LGx3GwWazrhq2h00fK1aA1yJ2r0Ejmio577sQUKrJdMM66ilnRRyLR1ETcGQW6Uz1gg0+QuB4kwYpcAMdGd5yrfe9aatwZndiRb7Hw4/2/U2XcSk+VvHfo7AbwZJeKoMIgYjoccTWIO3qnSNKZcnEqDWeHcnzK5btrtaOpigcd4UqliiGHZlIRnyeSQezURbPpHU+ahl/lqzfeYYxMJPbKWckDnDe1oQ+ZwB/2DZgtYhWU7z0F9FZf3lq9E43Fl5azcN4+hTEDC+g64YpPrg/qLi1hTW5IlJJ+Mp7NshB8hrSS0tqOQ1I2dubefLBb4cnTrSbKmhwZmj6IKr8qpHfmcLmEGt8aYrJjMO2dynW3I4TcIaPK7HheSKBgcCEzhFUcLPXH3nX+3eO61B4kbXohM6sKuaMaKhXlAAIdgpLuv2YRmOuCKrgCuU8vETDZ9zAR1VZ+K9ZxZnw2rcqH+cj6893NtiAI5MTkMPriaOqVHhR2JxFjc3RrZydB6aeoGmwgOy5b5NkXm5xNTfIN6q4sYdHANMKEXnKNFD0L0gfjkjKQG2PDcQy0rVhCpzWQjTvfoOG4lb2L/sU5dACv28TNVgWZMj2Ncbk0dirIVgeI1gYHczADMnQ4dtiTMaQPOQA4Kh/G1oUfsXrWAibptiPwOCD0/G5BhqbTjlD7Nl6f9CpbCn7kjiXHm/qH+bP+dRytKXRZcigwx4JNm0Fe7duU7jlDyeQqokraWRWsFtVtvh92lcIkA2MqQdqkEZXBXO0CKQ58m/r4TaxwAINOTAYtlkZxa9XbWLfP4t3H1hP2QwR9x6qgWZTMacVe+wxBQp+8CQG1UY65W8mqWnBQB/cELCZikwpJi71Jfv5wQpotVBl7yd12MjnPnfkNaX4dEx84w7gBu+iS9eoDKU1lLE/4lZMVGh59dwk1A9s4YCN6cMDggtGtDHtpPcdMfhwnW3Rer7l2Yi+oudzqQH5zf0/geEk993lAbaUHpYIBOWLgTN78PmuijnKkbhQ3VCOY6NRDTgjr3ElrGIytsPDjQ7+wSwp/KXrWO6sVuqRO3OiSEtc+lR8+HUz8Yz4IWLmqgww9dAS8TaZwBv8aOdltMUQOKkCW9QGU/InDHSsrwUL46WROy2HC4tv+ioZmysoOEjt0H+UXEklKTcLdv78M+P9Loy/e8b85/gv2/Xf8nw2LxcKtW7dISEhAettdOzIrguSLCwlx7WTmpFQk1t6kU5BIqTaDR6szX9eFUx9U3+eIAkenH2XmwZmMOqSjYM51sNkguNDEfQ5w7/xThMnNWEUWuz2bZ98V+yhNjWaY/DqB1gCwcdcTEPCp9mHiz2tp9WnEa1QvA02eX4ZPdTGap47wQWA7+brVQI90itFQyZpHNnG4RUFRvSNNMvFEfsopgOo1f5Ip0dMlSaVWOw1s3CQmxhbhnpDJFr9YTKo+0h2AquoGunPetPvWoPRo7xdvxJsvXUvZ/ngpH/8Pc22Rtwa76XlERDcxY+C73C7TIwjQ5h2FxXyE+7beR9lIcSfkgqgWnFYeouJsMvN3LcT0m5g93+01F7c5TfxhOUFFQJ9NpCChzAidnRr8qjxo7hIXCL21GZzyh1dU17jONdzlb/Y771XWCnZvXEPJwCt09VHtM1vNrPM4S+KIdobcnEClwcDMO199G2jy3L6Kj3KtzNjf2yUhIPBhdAu3dC1sevBXdHZalttWxKRKok+NQ2IVKF3/HZq7R+0ZwZIq7nUBBwk86ATbLWJpSF/zLh4ZcIrUgx60BIjBUQBnXT5rYurY5ZFJeOoh/MU1cpTU41l+naHaRTiq/2b48p5kWBDAtSYbt6oM6j0s/NBpZoZzL0swTnuDN3cs40hAEZsHX+1XBDze9g1pIbt4648HsWtxQTm+VRQPq0vErqAVbcEQMir7IH3AWtc2Ui4NxHHvZYxuQdDH2WmSNJdCmTeHx5/mSsINVsl6C7fy1hTawuCZBoiUg5tQ2hMwtHHJEEddh5ro3AgmnB5D6nzxc/SxqYhLO5awb84+WlxbxAQAiYxDnVKsEj0pz3+MvSFcxPDqUvjxtX8+eeUezMwciCZOvBlTYsQjsoLCVjXf5Lrybh+JI8FqJObCSCq3TqUR2Bvcjg05GatExaw/eyRdAeK+7GXez1eaKNCrMSS18PboZ+lq+Yg7LDB7QzkfHFhETZsdG578ErlZnKwGWAv5eOd9VLZryE1QEOQ7XhRXNcuYeH4iP0cU8HpgBZt9xaCDubKGqEtH2Xp8PqrRHqxfKwqTL4kjM3o/UrWOg6PKSDecwRZs+aFlIJ3V6SRkS6gy+YCyx0NRJVMx+cRkavz0LLbbR53WQLuNity2y6PI3b0A9awTPB2Xymw3cZK+YMKXWEe28Pc3MaiuDMT8TK9crGA28Miy05yvs+eh/VNoiTwhOietJoQRx3yYdmoSEfFlOM4QgzwAG6T+GBbv5IHYGnYEiyV3VA6BnIizxxpwmiuXIwgeMfquxOJKxyZ8nzrLlhvrSLL8SWerElvJ4W+zjlBZCeufvYfRgf/SYqOhKAgC3ZpuWnxrGDL5OrXBtdiCfe7OE/nwoXdwcirm6d/e42eJmI1dTRAd5nKGjcziubVfYkXMchQQiM6JZkCnG4uWXeSGRQxoj1R0kLT2BBmN4/ju+g88dk/g7fOCq3ZfQTzE7jWx16uWyBd6J1GLzJHAVSYq82V4ZhUxJ0DcvqGwvsylfxaiM/0IsVn4ysWkhli1DmeplR+sarqMzv3uxZWsKJz2DKU4vBZZ4Op+8SprBB43nFhkDzM0/e+lX7Yr83c/wr+PbcE1RnW3T75FOYmjRZNod/uND+/7h70dYoD+xYAHya0dzW925Vz3mMfVpkcZbeOJlm/4Eq2phOlHp3Mr/pbIy0di0TFTA+2Vniy7nET7oB7/DoJ6ekzts/axdcVJBHUJHlJYasssvj0+Dy2jUeFKoFLODfvjwEei+HYPM86CkomuW0nrDucO2PefcdNIcx/BxbJ5dNqfp0EXyp0ioSBT8WqOK+puNX/6VyFYpdgK62nlPjRfSEIus/J69X34T4wRSfqldMeQ8vYUIgB75w7cZoo7fUz2Vjat3sT1AHCT9C/cO3fn8m5kK9tf+Ynpnt2kmRdh2/1X1LUDx6ZqxlwYQ+eYDKwmLbdyS7FarTi1OqHudqRVYc+5zniRbPpQyRU8l5byqaSV3eE5DJWv7PPNPedyYcRlNFoN/5GK1+Jan3rSB6STJevGZOrTEWvW8ei8p/m1YjmbP3iA9pdu9PP4jDB+yJgxu1HlOpNvGc5rNtxlwdjFypoQMCgxmGQ0W3rfj8uWZBQvhvG4WwujA1rIHSQGT6cGFyNfYOK8JpmmFjVBcWJt35y4QtIHXONGAHSYxeC/3KIjXgF13TI69XIEdZ/7YTFgJ9dyv1sNa7whwyS+l0jkrK1yYv37T6JIfJOW1iVAbwFULVejkM+h2eNpzIBgQ9a41+8yhUYLT0ycgErvzOj2fCJud8E1dNVQ5tXT+1vnCvbGYNHXelqreOrrr9jTCR+oYaNWXOjo9Dfz07qfGH9mPEa5kdES8XzzZ/tamiWX0JX4onXv00oGlFV+xTOJP2Bc9yba4TcRnm8HZe+6mdCdR+zFURSFVLBgUD6Y9SBV0iwNRJ8aS1irCy/P3Yen1pm9NsdNyYrC9P0kWmecpyxyPuFdq7nfps/4M7k3mSN28MTP6wir9kNqEatE/FATxpVrMTgNKsIr0gj0ykAY5a78cHItstMSVhCNMKg/+P+3djAb93mzMjuSvMV9OqBlGtb/NZaJZ0ZQFFrE0a6h3Ol9DejYxvS2bI476lg/LYNdevHz//DuCQw+lcTwR0pZM3ATjSob7zu5Cte2YBAq8F3Tic5+hOiz7fJwmqaH0JWZytsr3kT1rngfViXEcHHdh9RkhfL1qVAOThGD9AAb6l14IiUOuV8VBpmbKBbrkkDkffvwaXTGWTUTiWkr4Iysu5Lfjs8nPSuYvSPfZG/Gm/xu87P8ND4Q/TSXks5zPnwB7sZB2NoRF3f588GP8ygKKWXLmHRaBLEc7N9pCbhuGcPT848SP/JWP8B4W2sAhyqNxF08QuGAEmzXcYAmdSA/rXuS0ZdHUnd2EGefEIW5NlCHcuBnHL/yOTMbep8DD0slI5UyPjBDcoWJEGKxhRKPdwdzo/USeB8D72PEaT/Edn69YbZjsHsDoWHVDJuUIsrtJVYzY9SQrbVwrWIWHapB2O46Zrt4sV+6jPJ7TvDY40fYJHKd7hkfhGcj03nwVtos5ib0B/uyvJbgtOh9fp1/k52m22u5VElmkR+xGRHkxbRyNqiI9TZebIIgIcezFjxrSY3sUWH4umfGAcAst0d3ZAyebU7UepdT3sf3N6LlV9ITz/D4zgdwrfOiamguUmlvTvZO5EEGGz346+XHuZ6Ug2SqGIiRSCAkdRcSs5GioX21cOAn3TTKJh1BZ9/F55YaLtj1zNH2CjuCipX4FBWAXMlERwtSYzt3CLIR1lv4PnCY7VenM+X8h3gERQK9qh0qiY4Hdi3Cx6TkhVeXcb3rD2xVfoYKH+HW6c8Lvz6JProWw2dFqBS9krGG0Bg6io+y941H2fniZUQCQ4Y2BnXcR3ndPCp/Wor/E1fFv1mQsHHdRh4/P5GtxdGkJ13gDtg3RZODxTMAbV4436wZR9Yy8fW+2u1LVmsVLrVWHJpK+2nknssLRf/vELRqLRkRp4GXe09LHcgm/wLsHGt5rRQmqAZga2fZoYlgaZWSvIQ2PKX/8J6PeL8DMM74IZ8oPuKrVpjoJiZFmOUOXLAaGCoJYpEwHHdZL1Ekv3MmObVwwV3FMTNs6gPq/lnSxJBMP54cE4bZSWxPYZE6cbjSCTeHTpY9tJ9yHzdsPZYvtY/h4Hsu1Ay8ReHKfxjdJze3d/6e79+axrAR76OJqRQpFwGE+0ym7VQx5UMr0XtVMs1P3C3pY2kj42xPVjLB4IZgNQAKPA1Z3Hd6Mm4ZodhXevNgnFjJITbwEf6OS6Haq4O1joeRmsXxA4YXSIvfhlKrJL1TzTr/EFF8x40N1DXk897Mt6mUtPAe0ClPhKBETFaIzgljwe4eILx+VVY/sO9ziR9dydmMDfuN373gojGfOx10w8MX8tWTfsgcpBg/nsthdzOSjkJwjGSy8irPuClZG1zJxoc2IpUPZomy99m/Zd5Bg0c+TWdns/PUCwx7vJccbrQP4+c2qNZJubc8hBivNmTmdrjTmS3T8JpiGtG5PfW8SfJKlrXcc7cesLVkGbK8n4lqdebBCRdZEd4E/AQI0JJBarCSz5IWctH1QaRWJSYtolzVSZeGR0Uq4QGOhNuDEz1roiDA7z56LnW580BYNw85neVBu+d49/bnHKVW1i+6zPFYVwYGN2PnoASbmlKr1ZPdnVZiVc102nfi1MelR2t2Z3axPXK7erpn7WeE9zhR/A3fTJQr0jnz/Th+svQHkO6QR+/xvsUM9920Cb0U8HihDc/Lo3G/qOJz6y+0OPUyFwRBYNG/i4jLikNilcCaM+Ljxg0nc1wsAaUf8OqJV6n+Stwx/ml9EDUtDcw5OobcaLF6hSCA3s4VtcXAlN9XYJogBikbJP5sbIe5pgCchXDs7fvXoyb5vkmqxRdFbiHB0wdCH22Yb5/Zy+GtU8h7dT2lG27nJxY9f3m28Uc7POXXxMWgAgZU9BIbpaZG5od8zMnUaVzNnU65fgALbR4Cpakc59TrBNfE010UQlWA+LwL1Fc59tBnrNt2HxvfWo3dJvE8N7QthJStQzgx+UQPYNwn/7Dzf4qXvhxDzJGbzE4oxGNMDrYkwUyjlOpLMejKHdlb/A5Tb09nXcoIvtgzjNiUaK4v3k6VWof0tuyv1QqqkIc5My0KPxcJnVIVLg5j8TVsZp8PxCnhTGsjw0434NcUyo6GpVR/qeInUy3IHWkMfoh3dpWTVxhKSWo0bTILg550x8XeB2R2KExtPHTvWZ4ZlU32mXKGDf1/F3L6n/CO/83x/+6V/+/4/+UIEcBJp6Y7x5tdmm6ekPSubhLBwviCaEJTBtGcE03IA+dEn3U3VfOXN6QlZ9Ee0IiLUmzqHWApwvOP+/nNvZtyx9/ZvFQURiKRM7VdS+h7G3ih8Wt8ZINF8SBzBk8dnI8O+GnkGdZoNtwt+ar1h3CttfJvl4HiGnhzlC07TUZzcDlNzaFcXf8sV2cXs9amAalO5cy+gGKe2b8Ap5sDcRsqBjS+qdlGRn4brp2TUbX40Lc0ern8CbqK9vDgEztIi9bRbxhg7t657I0o50aSjo/6JOofqEopGX6OZ5zBqNPcTVmb6n9j0sifOXDNkajiZAJHpIk+p/V6h9TKUloi9qK2K8Czj7H2Ncvz3JRVEjf6Q+5XCqKOQovCg+Xng3EwWTi58QNOasW9YC2aKF7+cCnmUj8+ee5z1vv2lzg6VP81TR1nWSrRIHUUs+LugD6pSamkJqUyzf7pu7ENXRUU/v4M1Wo5DQFxGOx703CpRMoDajnnOxL4NiCFvqTkCKdw/k4eRJvzYQ77wvZ2sUyc0vUX1nyWisy0ifqQIjyWiD1z3CyXyNzRc4/uXfUL9DE6d9Fl84adgo4WM2smTMK5bQ22rGpZx0oeej2Lj7bfwxCTCqVhKBCCtOEE6x7+kG1pj7E57F8EaQ84fmfUCkHUZpnxqHHm8XJ/WsaKuw2GVhuZeHYRha6NGKPy8OlTOPUfb6AuciSFxXEonPsnjd9tO4b66BlU6GmUvYKtfBLAWR932jZ8wtl2wCrOWK1KT745G41wYjjOw7OJGH67i0hbwVSPa2x+7Rrns/1ZdGksni7iYlqLlw63B3cwssiXtVcm4ThCLJ03+cQEgkuD2LhuI1jFN1Mvd+O7NrgvL5yoQ9NwDhZLaOTKh/JW7I+suBHKup/XIfSRVRDkLjyv1d7d/pk14riLxIxfgzcWwULd8Bt0yePuvlv7R6fTPvYU8rLXqda5gtnGf0CqQTL6Jg0tKh76fTXloWJZGzehEUuTK87O3Yye2448yFkUf8xqIu38WMaeH0tJcAnCBHGho0r7PC7NPUXa643XQSTyAhKJlGNTT6KQmngkayFLpiwTxUvL/IncHcTNEQoE55F39/2rlY1kvXaaL89+jXbbPiZOvcoJxTigp6hmUlppdG8kNn86yemPMvCJZNFx3dVZqNSNnJkgQbAYWSjYMMGxsMQB2usUVN0YSIJ7X/8iAd9qX9zqvLhV50X24FbEZw1VajnZ8bdY6AENZg9sex18HXyZWH0UyfW3Ccu3cm1I73MUr3QmVN7BtsyBbP59Jb6vjhYVzBTVpfjklfCaWyTSyhfYPEWLCz3viFRXxx6lE4fsrDw6/iAepgkiWSej3J8GbSAm+0o2dhtptxd3MI90mc+3IxIJSlrBrsYg5lhM2KZtMomM+9LmYM2zI8izgp8fmyD+0XJX9F6+5P4VTq06HD6/I+/V+192LdxHt7qdF+klVKR1VDPHvAmPQH8qJjdyq1MMtFi7uwlL3cmtQQK7F+zjaTuxtOnyvFisTXWs3HUFk08lgiBetc64qBHm7EMpf4hZHuI5EMDXGMgzX/YwkhMf29sv7h5UR2eYwK1zA8mz02N7dr55p5B4aZlSBUpziGibF2T4gxHh/1CzfQ6OtyIwR14CG7AvsOooAYeGEzr/GosHi7sZZaYm5n73MLVlPlwDKp1ysU0hpngOp6r+N/a4vUOu3SmM9Aclfq9bgeEzJ+wdu7B/pdbmm3tG+x/zuXUzhu+HXSfwSbu73ejnLKFk+1XgUPk846uU5NbEMI9PgZ7ujEknJxGXHUdmXCYVAWK5YolE4LvHvyOyIIF5e+2oDBczY3IJpmT6fvzanZCHVeAgfUsUn6fOZoVcwhOnwtE7GElZJT7nRucJjD76LVZjNwfbfbhfKTZhj7BuwmGGiavHkjnu142ssxCQ4d6dwpvnptGeEsuN2W/Sl/RYLoSTNvxvbraDotsTtVLc5Ta3oZUVJ+9l19DL1MUV9PMRen7KTcyT9GSm+CHIvUQxpCreaR9AccJl7PQDKbf2iQPb66Oo/HMBr94YwPBHxc+gYNby3MIr/Do8g+doJsGy8O7cvs76J9f8kynQyrh1cQCN3uLO6c/xpXnYGZK/ncG2rUEsTRGTSE4HtlPSLefc79OQhtYy3oaM4W7K5+vrEzh7bAhClx3VT4tzRYfWXdxT+y+Zr6zgqFsrfpvpN548sxc4zcKYYjTdB7AF+wRjMyP9T+CQE4hZ4oJmUi95bEZaENYuA0LEtzzm20ZHXTvQ402mMdWSWjuQQ+0SZLFO1PfpzjDbT2fB2UKcu27SXBxJ90hxDjHNNZunvGp4KOBvGszwpOQDUdzP3MLszT1vecqY9H6/qVYRz6vfzsEJWBJXjcTQDDZeoBldcSgzepChe1R6sBhBqkQiCGxdthWVzhG7An+KHMQPod7VxJXkKwwo9mL6OX90L1zCtnB0J8/0fPwvnnISyDKLffd2pYShvJbAzqeP4u4wrIdAsqyn8KU3dPJdwjfMsCYyQxqGj0t/X6fGTlcGHunpIXhAdVEUEwRodWqjILyAA7MPEKnphasd2g6TeT4U00QL/iUw3i3ZBnKAwqAGmicfR1X7Bgf++ZGfJtn8bmM7y57cxOX6YWyo8SPI3VUEPuVr63gn7GlidbEM1CrwtxODgV14sLcLHj8wA6tgRfigP0FgybnvKQnZzIm5+5khomBBh/sKntw3nIm+C7g/vhKBnlzVKnOgUN1Jh2szeb5vYBXMdBhexP223Lxcm8XWRYP4MW8l52/fHdsR5vwJOVe/w9AYyOvuldybKJ4HT3iYUczbw/w98+kuDEdYLf686eJ9eB3sxAswxYilBKFHpkmj1RB3K4E63/7KAue7fagp82Jsmg7pkDqgh4hi8fmF12cmM7+znZCBadQmir/XrVjDf3b9h8xJZwgdWIiHVUyMrAlqYeO6A0APxNJko1hjVnmScCKYITeG0JkVT8OCAKbZNPpo9BW45pRwum4cvuN3sCasLyQBzdqPSdtSQrSploalhYC4S1ThtJDD/huRF8GLkWPvroluje6MuDKCWp+zEIQIhBQEgWh5j2fwQ47g3sfa0cfehxFXJtDh2sgnH3zP9jYxwFNiCuG9twNIGFTEQ3P/QqYwkSrtRVerdM6U3lac+ME/nwlWMaDsSA6hdjdo77ZHHbAZGdOA3r2aIAhcGHOh5x+Wnk5EAE3rAcZNO8fRjIWkTR5Lmp2R0creY/taS5nsDKf1EiwVXfjqjmIL9g1QXkUhl5OXEcnTnywidoNEpD70eVoSHlsDUQDtHXNRTe7t2+7W1iL3ep6myHq0MicsfZsEJDKaErPonu4IRyawt3yk6M1yN9azwA6crgzmmE8DkgG9JECpm4Gxr/7KX+dDCL0wAqNGvB5eLYlmxBeTqZ10isqRWkZ1zCbORr9lW2P+pahDAAEAAElEQVQgE2vdyJ1+AmFwH+87iYSzoy4SnRPN6t9W0zhPvB+ZPXAt2/8zicSuFhoD3VFI+88X8fqNrLs8jUNRWQwMERPL3NUuJBQuYvbOKMIG/Iti2QDuWFBkNL+NsbuDiFwHLJIEJOPExz469SSHpu9hw/uv4hvXADbLjkSq4Ptlf+EiAWPlLiLtxeRcwSLg0uJCuFXDywFw0CwmpdlhT2xKBZ0pKzg66jqP9nEWEAQIu7kDVWcD6VNfwNqHj7TZPO6uqOdwedtdsK/VfhSn0tsQWp2xfzScFpdO0csjU8upD0lG4voH6+vCuOpRKjru3MZmFvz+LNY2R4wyI11fiK9JxXV3HNNy+DR9OXVDe8hCIbrPodgdWEmc2oBVsDJodDpGb4UI7BMEqHXeS4PTEeSd7jzWOoD7I3qJCa4aZ0Zey6DTxY3KqQZSTDBF2gO2qLosnHjxZQYlX+PvGYfwbh/Ue2CzgfWtcur2rkJdegvIRiLpBfv0Lkn88/dEJqcm4tDpQNKk68jmdYKNDLOAwMaHNmKSmWh3akfo6j3xRr0PZ8ecQRh9licczbi1+zDzzvphNaIwNuIoyJFae8nVtiPM6wEm/1jFF28/ybB98wh61nz3/33650YsFQY8xxSTpbyKRW5Ty5I582fVl9RKwnjebxlWqRrolV63YOTMyHfIG6LkaYkD9dTRd0w9+yKDL5vYv+Z33IPFObBMYiAgsoGox7vwDnHq99lHjGWEb9jANeD3py/xy+ze2OfSm+waPJSsi1C9v5i6e3s7cpOlmZS0udBlp2XArAtMiBXXwdrk2XQH1mGwK0PrbMFTJgayj+SFM3H3dCKb3bg3VLyeSfRVbJ0bw+aTM8gvDKUi1iSqeQqCwOjzo2l10ZE7+iEb04vecaLpIwpLruPU1UCY3c+IcjZBYFsHNGraqfGpQZDdzr8lSp5IC6c11Q+nyFbaPBGRPa1SR7K9FmNYMQh+usQNXw/mv9Xr63tDb6XwxFAqg8s5vuEPnExiu4QoeS0JznAxMZVbgaWMtQFWAY41fE973iaWRWcx0q+/cp1F6Ytw7l8ACgZmYC8Rg5wV0li+m/ATHoOGE2IzJbTYjeRI5T8MrvBj0rlxVAeXcgdHtFphTOgYmpY14T/S/y6AnnbQhKcU1jfCsSp75uVGUTCsjeNj9zFCsxIGfwGDv0CS9zsD86MxnxrCN//5hU77Bu53eRcXjziQaWh3GUrgOR+86qMIkMgQZP+FnP6vxn+v/H/H/6/Gu9PzUc1IY2iaCr1Sz5PSXmBAamrngZThlOUEs+2BP3B2HyZKZv0UCrr+WMRw7wYWLj7HT8Ik0bErpdG05FfgkKFCO2Q1RvPfKG39k2RKQiqX0Ko+wTdaI28iboNzsDQyfd55Pm2WkBmdidRGtuH9l97kbDec3DaGEpUOy7JeRpJV7sSEKnCTtjFk+GXavcTHldCzXFjj8hno3UGNRHzeEaUJOB49z5lxT1EfoAUOiOIHwn8ja/1GdJ4GmrpGiLqLAPzNFTinJkFqEsVdjyMSpgdAQK1V81edA386Ot4tWsk6j6P5eSDxgpzXN71Hdoe4KNtoHcaFimEkjPmR98NP849K7NXWs9hYmWUH9ziIpUMlEoGJpybi2upC1vjPaDCJiz9dSj+uOzUypiOG2ftnI8TQb9SoJlLpl4H80DgmTvpLfE0lAl9cmUjH9QRcXvqJGmUvw91f3UGNcwfFIf7INXOQKm3OS5Dw6jtbcSjOIvGVXKRqMXjqZmlg233r+O70PNa/uZL2Kd0i7zClzBffy5uRGeNpTgtBvkxcUHi7WaApoIhpvnXg2Yd1D9Q5TeLP9TrcmtxoXnoW1wHiwm2GKYgrLTm4ZzsxbfDn2LcsBF6kTd+CzP86GYqHWZySj49TJRKzHm530Cm9p/LLwxsZUpPDoJ0zCU++LjruoK4uqlMGU7xkBxdjs5jVh1m/KPxRDEFW9ro+jaNKAMQ7l3ZJEG2+Uv6z6hc6QvsXrdKNvpxvlrLY6EiMswEsJri9gbZzG0j+zfW4F5eRXxzMB04XexirDj2A1GZvcO+s5LOJW9nRESw67juGODIDd/HOmam01SoY+4ZN3KzjcbWMjC4HFuXH0tWnkIcg8PCPD2OnV2JZt5ViFzH4dIf1Zhh8i8goA7GKMeKPSyRcHpTGtD3zAZjlJDbynCAtZcn3n7Ho1hx2qQ6xSv3u3di/7Q50yJoYd+sab52OZtAHvT3IBoUXa8NSkBllvHtuJGMEcYKfKYzgo4eXM9WcyFtRqVzUeWPreeMdHsu0V46wZYc/rjo5cqu4i21s4hUWbNmKW6FAh0kF/CKKr3NxZInDFHa/MxW91BGvZWLJKolFgkOHA44B//L+nFdpaTsDAWPBasYnrBFjphueK2IpxR6dubc1NSC6HOLP4PDXWwTlV5OgvwY2sNmiG0YCc/24b+JNXopuJs9YDXfEUqRqhv4zCccWK51vv4vSlCSCKAVjO57TLnI4MRO7vC9YkCDeXAC4SwwkF0Ww70Qg+0eP5rhtsGIP653eYEfXDLT3Hmehfe+zctPyCd++O5029SWuzzqAp7KKtfTO0R76JzEXDKbOPpOcmBx0pvVwG+zz0bjQ/OdC/Ep9qVj/HvI+FRq9pZv0MRPI9/gFS6of94wQkxqCuMGXI97mwwsytsq1zF4r7p6WSqSsXrmd315dzeCbg0WbEwC93VBeP3OUAc1fIHdv445UliDAwx7TKS8OwDAyn2R7MzJzr+eZAETmRSIzubE4vYKkJEQStaF2W/B+IIO2ltWsr1zI1FGJou/1L/Fkys5ZgIXBEdvpNNyDs6z3WdA6G6kZmsKb5pN4qmOA8aLPR7qEUuppQdJu4A+riw0c0TMejKvHXGbg3O5xlI3vfe/kDQvwL4nHsc5CbdNHqMcFiz53/cYcSn70RRVWh7d3C90y8fVaO/wWDZ71NLrH0CRNEhUorXJX8oJLaPOoYcfY04SoxFCdXdcZno54EsnW2VQIi1A8IyZUAJhy3kbR8isVcj31xkH0af7gR49qZhCDY60nFps5+Jl6ORXWKzxVsghjRhw+D/eyYWVWC4uGmSnwbCL+TDzxKvEzYm+o5I+ILv5ySuUHnxICXHN4h15QQpBJuDr8Kj7Ni0kou0iCp7idPN7SQMf2OUw/l0jaILE0GIBJ7kLU1kX4V/qTPvVNXKaJtxV5lnC6p39NfXInB6tf4B6VD9CAr8zK5FFZ/K3secf7FlAcHO7nlxN+JDdKmdFwE5eJ4iTAW2umITOCZUk5PBpcxok+smZeMgumbjnLf11D1ChxMQKJjG+1hehi6rHGpjNCvarf7zpX0crAGz3zU6OlD6Va6UZyaxfNyi7MZpDYyFLHCdnEvZHNaznzeIffSLCKwQwv48vUF1agc3Snrb9FFk1XJlB3NZCOzADSO3Ox7fXVynxIOzsAocuOpIk38HUXd2Ccbimgzr2neFJoH84I7z6dOFYLgzx3k+Nr4OvQLO5XP3cXUAbQt6XhefIYUQXjcfNpxFOXwp1uBq+rAxhxKZGM/2xicYCey+bedckiVXFo/0i0HXbY7ZYRm9iFDXeAcJfJ2J0KxNr8GxVFaiqixAVff0kr4wUlz9iZCNWYESziNesRr0yUC7SkXUxgTay4UxKgS+5Fe7Mj+sByXoi8yQGNOO+qd/Eka9o/JN8Ywn21Xpy/ra6RYDhBYVw3D17yYsSvqzg1RewLGRNQgcf9J6m9kESoRE6HWnzDRjZpmXV8GZeHXaNoQCGrZOJusW9GXUKSfJYOhw48LeLnU0CgxbWFhpmnWOp9gVSF2AsTQFCaOTfhFJOjqsnyt4p8IwWrkVnjM7g1Kp1ZEohQlN6NfetkovaJ77G7FcOSj5+h9inx8+sSVk1NeBn3vn2DTpdWBElvFmsydzLY9xBZ2ipUeVAidcBWoksQQNOlocZXT2j3z0zso9keajeQiVc2oov+F2V0MRJe7fe7hhvPUZMbw69PHiC76zB95//4S0fQtt3DzHkHOH7/7Wuq8uCFkSdgJLz0+bNYPZroWl/PncK+XmrPL5sfo7TCkaejh9IcLs7d7V3cSJn+Mpqy53ni2ydo+kZ8Tgu8/+R8RRtnxz5Jm1ObTe9+zwiWzKDDfQ+PPvEdp936AxKOlgbWuhtpevEHzPL+3naaJiUTTk8gOs/AwJEvAFsAUMvtafWKx0FWwKD0QThEiuffEapqlMognM6MR759Me6fijsh/vbtxFHqQVJtz3thWyQUEBhxeQRR+VFkJGTQ4FyA7b00+d7DvkMdOLbWM0Q3ndiX+5CFgPTiWZiafiRo2DWUwgVgvyi+esATTK1o4bviQUjuvNf6Zi7OSuW3pFym/D2PRftex/PPXt8mAYHPz87g+tlEAsbeIDyiSnTegdY2Xnj3O/bWjuZMN5QaxV1XhdJoKG1B7ypFprhAbmMytmT5jRVjsRu6H4vEQmpSKpN04vvVaTrDoy9+wYdrX2Z49k4Mw6XYkioTpIU83hhI3c1hOAdPwel2t5ik5Qj3jfqOXyQC6RhJ7wKJrPfcrktn8MkvVRjNnaS9tptSTbKI0CMX5mKZ9Tv1rWXIS/3pkojni8Pe2cjuu4BX9yd4SGaJYobWbJ6znOWn7iiyZp8kzkkMuiKzY0mdG63DziEZcoF4nVi+MbozhW91vqyYe5Ar/qWsNvb2vR4yJPFIwxl8A1qJX5mKi1rcpWbUWEhJTGF6RC2PRuexxyRWkKgPbOfj9R+TEgCOEnHXgsRiZKgSfE1q/JrdkFvE3roAHmUpuNZkES6/hMy8HHAVxV1z9uN7YQTPVQThOvYytlLicUIrK/eH0NWtIKttCEHSXqByWc1FSk6OxKyehkatx+ExMTnmJQ8dDVYLjXEBdLiI93dYQSVApwU+9F3IpM5nWM3nd8PLY5M5v9ZCWoeR9SeHMHaNmOzsLrmMx7IWqv+1wyLt7xmsaDnN7OlbuHB1OkptEwKu2BIUBJnA5vs3M+PKQi7aP8K026BYjO8cvh/Q01KUUwpnylaIJBKdjZf4ePLz/HIwji2/r6Di0VoR8THJoY5GzwCyXFuICK5FL/XGVvj9QsIWFG4FTD45mUo3I3hDdPdzcAXs5bN5b+I1rk24yvnmnRRdG81Sm2VLIoEHurtwPL8QQ/pA6kLH47WyF84W2jMZteAi6S3D2ZHryy4nLe12PUD+UctotInnKPOpYrFczmiv3g58rGbmKXT8Xe9BzahSRg0qpsMSwx0g3UnphI9lAKDk7OyDHPet5RtbgpfFxGxzDm2RVaToYa4KXA213AH4FRIjDnIL43fNg5IQnn6qiJl3ElLXweinl5P54imGFL5FxLJUfBzC4TbZDmCvLp4aQzGOFgUZteNxUYy/ez2qSgLwKbrIBMMIpncG8/Aym92MRMWF8qVIdIX89J/lVA9x5hcbcperNJ9TfnD6TAzdf81n6NPiuV8l7WRhmJaSbCVrzk5l/ABx/rI1/zs68qqQmvRcFELps+QRLq/lxG11hvQmcd4uGfwlirZQbgzZRKddE26SxdyR+l4iO0zhhjaGl0sIV1j4RztC5G8ep5/D/cNbeaBNz0dBJn4ziLvYNJ1K3JrdcJ+TwthJRaLcv1HfxrFvZtOaHcaHr3xFtHWh6LPz1e2or4zG4A6T/zMVk/FJYLbo/2wrOkZytppr065ysSVE5HUsIPDtlTCGyXV89Po/nLxTy5LISC/xZ9KJ8QQVVeFrHEX2vF5lLpNUxXrLUSwOuxk2zJF6z3q6TStwoWe9bjFp+eyZzzBLzehC4GCH+F4lmKoJPTeRbJdyDo+8zFjEYJ80wJers9cyz+Ef1IXB/Tr7DOYuyuPjSQ15nJzwXLbKxLYFIODR4MHCA7EMXTUaXduXEDAEiSDh2LRjnJh8gjfeeYN47JFZWwBXrFaQNJ5j4YspFDkINO/wps5jKhXyMTzXCH96Q1ZiHm9FvE2saTnrfDKJUh0GyzPQfBOLwo3Xk85jCM1nmGIMhxL+hqwXwC4E5hUj0Ou/nDv4CC1d4dDP5Oe/439j/Bfs++/4/9W4WPYVre2tNLn3iL6L2A1SNVVzjpA/QU6uZwVx1mGiz0aEPMMf1400BFXyzfBzjA8Qs4s7Je6889LjPPPd43iZLf2KS44KBx4/H0NnXiy/rPmCBv1o7rAvAXKVU5jh+SWu3jBBCfbW1ruxjxo1FFm0DLs2jBaXFrGUiiDw4K8Pou5Wo3n9O7qNg0Tfu7jpJgPefoPU8Wf5cuwJNijEunoO2iwMjaUs3B6H3cAc+g5PVQOqeheqf1xB0IT+xvFqjZ5BS89w4Z8pKHTt3Om0uTMeqzbR9VMPArhjxdG7f89UTiLF4ShdgjMPHyggNNIOW8GplswUYs6dp0P7LN+1LCF5gXjB9WuciHr3OLJ3b2DB2Ivstsk4XWQyYiZoyWot4pGTUXgEiQufggAnJ58kpsOFITeH4GsVswUBzkjeQB9xE6zjuWFIxzZVlwgCAxwMnG9yIfzkeDRTell3D0sHkvPAL0zRwPvuz3G17D+MGNGbBnW4haIz6Jj3/otEjMgQqfVY5a7syX2GvEYjmko7KjtaRecks7TQnjyEXNlGckIv8TRi1rMWGYfGnuGUTgVVmj6q2xDuNYa/XRuwbyri5TYtzygnibpHsi1RHDLswWPdL3R6BHGPLJphgMFjPAlFUqRm+KL5FyYF/APafeAYBcAgt1EMKh1FTcBLeHz6FiXm+aLvPRaloPn9t0nTW1EJ3PYA6B2vFqvRtbfwRPCreNtJ6Av2qUNUXA828IRTGQGtM0UeLACKTikTTk8g9uJoxi8+BdbezY9MIkMy5D4uuW6nwuUDDE63N03dVRgNMjIvJeBiaeQ+uwoEgrEtX85VfInbjXe5HOmIMsSB6RrbpFKCYskR2iaeJuHV9fiOSROdk4u+iKEBTWRLdbztm88UuVjqYY7HUCQXPkbd1MHKZ9+lQiZ+twSrhUQl/P7uBtL91dRZxGC3zn4kmzPeocqQCSpEfmveVR6ENTsTUuKPb7WXaL644zVgkpswvfcpJeY+HUKCFIMVKgpUvPDrJ3gtHyTS6S9QPMs/Gc8y4XFHnvXo4IBVXDTY0h7NN23F/O1rRYa4qAo92vMNp0HR3kZLcFS/efL1xFOEJ1bwXI4r++vUjJLfZjbHvcSWI89Rp+/g54TDNDVKmGnqPfcHnRSMs4e1caE8teo/1Lg8KjquW4MTIy8ng+9Q0p3SCRljkxQKAmNvrMW+Jpe3J53C3yKWBpPqavknyMgnjo0YDBISHPqbIH+gL+f47e4Q6fCnxUGNH1euTkRapeG+sXl42vWet9Gqw7PkClY/I7UjbuDnLAYd1s45xcXhV+Gtx3AxKkTXS+YYyTnlE8i8TrDq33uoGSwu+JbV/c4nsx/nz99m4nxxLe2DxO+dnGZ0t0zE7pqLKqS4H9sPYGPZFgoeeBijczMKq1jGuaKzmJArG1F4NfKfDc/R3lGCk2NwDwM3V03DZX/umZTMQ26XaJL3poMqSzMPnJiGocmFmqDNdHlEgw0HvZ5dTJl0ksIjWsLLJai7noO7LkTgF1xD8dzDCHEFtHs19ynlwoMuhYxtCODcV2EI83bQt9jrqCrn3S/e5qVGuFbbt+wKDebvuOr3J96vf8Mi516AtMvSzcR3vuf9Y4sYdlqBLEG8sThsdxXt4EouJx+kwbOBF51/FsW/UkBnbApN1z/FNWeciDUqKBz4YsKxu/8297kVek0Sm396EkWpkTACmOXUvxsdtZqqQDlpgxoZqhjWL3xzaDq3RpwnRA7zrB/e/btfsSurt7zBqalHqZ55iIWSXiRFwMLSKbs4XnIPHkv/wkUmJlsoLZ0kdrhSarGwY3Qdf7eLvemmKy7xXAA80i5BYXZB1ufV+cwwlmHnejaWVYP7d6woJVAdVkVwWTCBWYcRBLEXZ5E5jOvn4lFYI2niDe5Rm4EGJP4P8UrJw5jcO1lmvp8B/reA3q6tAE0kU/ddRKNtoFznjWuimPTwS7AHN19/hxf+eYmPjs5j5t4kUfz5Y0MJyvVk1KgOTKH9/dDiM6KZte9Rkp7cSrlnfxBT52Fmy/ItNLk14eQWLyZRSWQUdigILQ5ludERh+G9Xsmtjvfg3L6Dk81RmD339ntQouvi8NtfQ/3EP/Eelkptmz/xNrLtJ66sxzXrAhsf2kinq5gcoJV788majThKrUz26EYjW4Fttlgu8eOlwafxDNMwrPAAbn2bS61W7r3nS3ZMVfB2vYH5cvH70SRxJTWwBJXZTNs9B4lU/ng31h5VxmWHBoL/ncFbvzoSsH/Q3TfeJHfjqzm7sO8K5N4Tq+hEfL0FLASn76XDQcrXT3zN1EBxIXp75jgKvp+LzKuRkOlXsY4WF4yP6AOQjztN/DArehsflbvHFwR+fejXnn90gyAVk8cEqQTvWm9cmtzxPz8Abq/FRokd2bVOmEwSAhcf5dM4sed1rLSJ1a4wISmFi4NSSLJ/RxQfIzRRWjAQg2c1u50bWCkVF7V0Kj3KbisLD86mMVQ8t0u6qzGHw5etMPbwIVbPjqXvjPCY01kmPZSHXSEM0t4vJrhYjbzvYeXbw4moMwcy7eHezr9GuYbrbvUkOflicOpAIxODJQcDK7ihh3+C85G4F9HWPRYPeuarTkFF4kk/ovLcmHN5JLI5V0Sf9TBm8v7mB+kUrAx5ZC1OxtXYykrGSNJYWnCdzuY4nn30KM0qceEeYMGD22idYmVrrYbwSPE+7WbJ19i1NwESKgKq7q6ntmufU2wRw7w6kJp7vfVyutu4mWrBu06FOmsWI5/8V3Rcb9NeXp6wiUM3XfAKKMfXTryurHB7jqWjahjQ1tPR1bfYNmPCrwTP+Zvw6hp8G8bxUJ/f5GBqpfXLBxk+KoM5K0/Sd9xT005bXjJR9x/AHN8rrac0V+A8Vklx03J+nhVCg2UYtual8XFVLBp+iFkHYwkqDKOzT+e0qTKZWxetbDjnR8vYywjP9T5nGokU9fgqLg2/wYxhBdToxWo1WS157J9TyIRTzgy+WIGPVKwsAD2ktjY3PdMSSmiU9pdkbTzZSNWZGcyRG1Dd29bzR4mCFEMEzcYwJB5u+MjK8VaP7z2mIJCukqENbKT8+EiOVeSQbJs4yR0oNI6ioDWQLX+NpzvKBdt+c4m9wPsvf8igogdp2VVFqd6XD2wavgRB4OaQm4xXw+MKUBj1YNNpXyoNJ7jcwpyELJrdq4hHbI8xUXaNg98+iAewesq7qAwjgXhynJcz7Mp3VLl8QvKVZJrcmkS5u1mqISwjAZlJxjfjviFOL5aq7xRGsC9lBOcWJdKuSeNLQSwe3e1goMYxH+d8OfEOpyDlIEQ9DXYBYBfEx7e8cTsxhveSqlG69ffkcG5yJLTMA7/oUiRqcadZkX0Su34JJLHZiVPPfS7aj7gZfZh8cjFhXn78uOArTnWIO/vei4pF4nyCr0ss5F8MYUyQmDzwkHMlSmRs+X4OlpB6vljVG5MbG/g8fRR5Da6s+elTvm0Rd+YJ9ad44uHX+GvDfUiya6hvPU0cYoLhH1GZRBa7MWD6VSpEvZCAygOfMVoO2x3nr5h0hpp6wULL4Dw8A8sokxhY6myh2Cp+/pcbNeQ3ePFF6Ch8NOL9m9zaTpYuhH9PDaQlLZripeLc3UN3nCeGvMmrz32HpKmDsBfEs7dJu4XHZv3K1rFg7RbPvwAVXQ0ox13BuncwsxXP0105Avx7AdgASRMWv2Ya7SQ01g9AIutZE+88bhZLE0bDTnQOGqDXfzSj8hitnyZidatn35wU3J3E5PGvNeHkP/Ar06V+rPRt4KBBLHH1/uh/kNDNrME3cZaFMTgDqlQP4iccxYKM9ZWBNKlb8DAG0qb3FIHsEgkMMRnIvTEAGVaUXS2i/VB5eznhk69w4awD6zau48Ccw71BlZR98/YxJHMM8a++yuQnt4PVBChBqmSFMxSv/4RZdrDcC75vH8sdsM9LV8rLK+v46eoyTge+D1YBwUYVDKuZZaY0urKGk7RjLKuf+pfzDle4QzZdG/Ez2+0cebHFFa1G289/8ctrn6JqaQNkmPNUuE4Ur4k3tKGkF3njPtlEcOvTDL+9lRIEqI6aQEfSODxTj+CiL8G16GGI7SHsG616nIreJyJHQklwCTlepdhKfSskegJaXDEVBNEdGIAlUFyDUEi7WDXjfX6te47K435IHcXdXlkOXxFfkot/lTeNA18GxPlkYcRJji7aQrnrekyqPnv9qCcpPQrmhSvY7NvKP8Un4Hav6W+mpexs+BkEC4VGMJnEXYOX0gZw7sdkmhbtYrI0g0Q3cZ7546g6qgdc5Otrf3DkoA9/2jyCFoUrf/sX4anoYHpqO/FilXtc1f5Me3YXB9OcMN3QUuN1kL5gX6O6nMw4I5nOjbiYxPsCQRAYfmU4oWVBaMcIhHv2vh8lseV8/+j3LMhOxj4vkFaTo83nAH0avrXejF+QylC3btH1kmLC3lGLQavk4wvh1LnYi/y0L7TGUr3Xg9BxlzgzspT8O3Y1t0eqZSbywWnUv/EwW72DGdjHuail6Hn+eupHJlZCXpcKqUT8jIab01nv101lQC3BHuXoDD2y7+7dV9niBZ81qtEs8UXuWwV3vDABfc1JlNnvcUXyMG5n3amMzkSIHkflba7qAV9wKLRixcpsO5BL8qH+DJyagofPXNLt25CrJHzdbaKx2g133yboKgGrBYVNfhZ98ybn/DcB4rzsv+N/Z/wX7Pvv+D8bEomEhIQEJDaTVmrBFDpzC/lg5DSSQwqR0gsKCHI7XtPUItXAECW4mbpExxMkElKnv4Sp6UFiP30M/fvijYnktv/I9Pe+Y7hKLCtyZ0yZtIvdeYt4I3UU3jOugM32217mhFNXEiv3xjLM3oxXXG9h6pd8P5RaIwuf+p1X3Y1YOxdxx8xWYjEQGFpNk1bJu25wvkssNVitcEaTkEe9W9Pt8xSf18xJL2E/qZJ/N04kKFisvw6wXJGJc00EZwrUyBL7m3aXSmJ4OuwrvtvQxErX3UCaKB4u6yYdMISV8mh4byeZTurEb4v+JaB6MmPSrGh9xJsLF+uzKOwSyCprJ6t4CIPmi4ubYU4NmP3qaaxzodJBXIR2kMDz07bxw8EFBG+dwM17xYU8746b5ATBxWUXmfJAOt1e3/b7Xb/5fE+mcyhfRBpJVIvvtSAITI27wMQaJRwZQ+6Q1rtdoHeKBr6Xh5BSEwiPi4swujGHqNKacTrfRqGzWP6l0Srl7HfuSKwtfPTSRwxTipMvacerfPXoD7zbDP9RQItFXDxyFrQkoGDRRy+RPjBLJBsCoJHK6IqfzSf3xNKlqEHWZ0GXIGHNr2uoCKjg0xVbGC64Mez232cemklSahL1wJ7ylbz4fq8sjX3bYZ7w/pU/Oz3Z2GllpELMQqzAgasGC++cmkmMSY18TK0oXnsllhGnYjgBlCZl8vsjojBPDomipNSO14+NpnRw/yL3fV35lF0cj1lm4l3HOsbZJuKdxayIfYO/7AIweOXgfWdZ8hjFe21rETZ6MTqolOrsagyJYlnUIfX/ojrYQVHEHJoC+xg3ShU8mfUozdLrLF92iGoHJ1FhNNQpgtCH3uF6M6DvPx+4GdOI7rxGVmo887eZ+PAj8b2QWk3s6Q7k97cf5EvA78OVtzlyPcPgMIxjJxwYo23mafslmB7uZYk9V+ZF7akh7Hn2R3yDavEy9xYRleZ2Pjg5m7oWO15b/A8Ks9q2SYJItQ+r09bRXdqMor6Fa82HgV6t/urubDqkpexp01NihYkBzqLzrm5zobUoijK1AqfQ/vNJjevLnD+jwepq5uMNS+io/g/E9j6oaut9ZGVlE/f1QL6YdILh9/ZIPx0tPErDyxfx8zFwZOjv4C5AbW/XYEH5m5w+vJgqL2fGpLzP8olik3P3IXmkxl8m6Nc32XFqAc+vtLmfVgsOk1wo6NYQVumPRtOnS1Pjz735dlibnfkl9mFqdY9iK4cCcNHghdW9GUuEgmS/P8CmZ6Zc5sXzE7YiH2lP8al8Vq8G39uxtM6nqVu/jWpMVITAn91ZouNOa3akUVbDBy/+yqPuWrR9nqOWoEQk3qcYeCSOoQMuiGKC1ULDlQEECVKUC4+hUoqLP1fNUZRsm4RasJB079F+wKvVauWsrpno6EK2BhpI787EVsff1JWKR0QWrgoLeivcYQ4LAjw3/ALSoWd5/P0XeMV1BW8U9DLF5WYtSx88xLYbofielFA8aB+w4258Z/ck7q0+ySOZ4bRmuDP2kWDReX0SI0UVXYFXVTO0wpt9Tjy6dBxHv+x5l/dYS/p1o3dKvHgjdwFH1LvvnrPtqLjljm5HIFPXaeh06p2rDlln8L3+KOFxLfy1cDsllAO/3o0X++RybU7vZqPvkY90OdJhqeO1HeeoDEkHGw+jO+uyo0FBhpMDl2Ri4OtsWwnbky6QGOHLrMHO6JX9/XUfSJqBPtZAVVsCLs7yfvH1jT+iDn2FV31L2SX0ygqvDCiiJklChXctJaouTFIbFEeqYuEOEyaTgJ2iFanEwhabSnRg4Eq+nNRCl3MHO9d/RqFVXHwfoB7O+RdDmV3lik/Iwzh2LcbWk9IoyNjw5gZSg2BZnyIIQKIxm/3Lz7K5I4LpA9/HTueJreewIAgs3rkYs8TMxqemIZEcISEhgcbGnmN1y4sp3h9B4xAjtmVsD/1efOU5NPpEkhF5Py6R4ut1Zx2vMzti1vti7tNV5VXtwYBbAzgycwLuprE25TDAYuLb2HKOnuim7tQwVAFiGXCAR90ukDS1kMAScO77pFgtTL00hiFneyhGs0b1PmPlAf/w4pLHSfA5z9OqeNI0YuBqouPLXDL44yit4xXfCo4ZxXNwy8DxpEXZszT2LQwWZ1FsoFsyIwsLyPN7jS2q35nqIF7H76xh8ZpWpof9hNS4ELD5P4KEYbfmErvDkRFhRcjGibeAVqUr3405xR3m1Dhl8N3YP+NuUW0284xFimOHhgCJDWNfgDrvOlqNcHHkZDz6gJR2XftY/tjvnOh248mIZlrlYv+5cLfJlESkElLgzu5/V/DY21Gi+PnuAFKMEFs+nzHyp0SEH4AQ3WUqvRW8v2kqt5x9kPbpLpnIORwDO7iRDjG+8+9KyhcqxpD6Qw2D67xo+uRjSqVqUSn5ePEAzu+OombIDfLCioiyinPgXd5OtL37HuFfPExs+gAka8TPyT5nyG30pf3mYAI8+3hoyez4edcoOnZNYapwjgsDr7HOxusKINfiTXlpDUNvPMHq2a+Lf7RUxbQT4Yzc0kNSuqJbc1eVw8FiJESnoSsxk2+XpLFbL6ZffXFkBIbzsUSu3MMLsQ3k6R6B22CfIAiMvDSSqPwoFrz4JxVOYvKAWWqHY3IG3RlRGM/VIAnIxhbsM0lcCBzXSn59KG+eOcyzM/vqDcL8zBeY8H0tpcGluEwXy12569J48YdtvNpsZaSkB9QEORJDMz9VxZJZ4c6WOXs5L4ePZeIi41/L/8Kx3ZHBgpwLGmfRc9LQcZho3Q2iggawfN5hrtuJSZVVGc7c2BzGtDn1mOLy6TuMWGjvVBKRF4nMtb+UfYMsjNTx32NUWakrDxKRDwFcYqpJfGIb85UVODWPuAtnCE1f8OHkL1hy7Vk+urqLsChHbCGieksqc/6G9OCVHJq5hXUOYuGyI5nPYn9uGwD3BraJiHQ+SjW/jjvHzqpRLHIpYH+72F+xojEVWcznuDgPZom0C6NlKPTxIQ8Kn0Xs+9cZUQMLOz/oU1aFzcY2RgpWHnhuG6l3Osbl9uzLTaU4oxOjyp7sAAUviLdSpCj30RpXR/rkAeiVelFuY1T78NaZD7B272L82fFcdRD7Hw0ypLLbB0794opz41+UTn5BBDrMd+jmYU0U+r8SGRVbzolwMalNrlTgWBjC5cgcMhMy+UoQ7yv2GSdhcmwHmZxTHQ8zU9UDflkVbgSeGsPqUz0F4qgRGUgMraDu+bzG2sGVtX/Q2GpHeEE4ar8+a5YATrV5rLJz4s1ZcMachq0P3MmgbEzAo1fewk7dDrk5ELysB+xTurMxPJNRo52Q7FlDzGMDxBCl2cCjTc5od81g1Jzz6EeI5/ZmVQB7h+9leKM3O9WOlCqyuLNOL1UN48r5LgweHrxT8C3es0ZgC8m5CIUEe55B/e0KFPUusFz8Ts9ASXfmQE7dHEiGSexTb5W7cCwlDMr8eHykM14+Ysn39KYCur3r6Xj7Uyp1UnxMv9N37HNrxLBiE+8Dc7STbPr6QHCKZhvfcTCq51rbFsGfFeS0eOfh3j6Zba1R/Ogg9kV9c9doYi4PpvC5ERSo3YDebhxBqmT79gnoiwKJiS/CxVVsIVGvuI8fN0bS7OZJl29/GfC9+hDWlcjY42clwbG/vUqDKpKJNV0sG3sBTw8tXXIxKfMhOwk7P+oBfi6MWY8g/A2AuuUw3w19hD/PPIz5tBGfOWewBfvyCKS9u4lM50aKoxQEOYnJFO0WDZUmuFjyNPdefJ4FfbgzzWYZSKzUDZRQ0aVmfQbkOv2K3yTQHYD2o9cZUrGZ7pKD1E1vQCLp3bNIJPCOowOdb77LSss9vBP8DV3tDtwhAmbqrcyracTL+TpzpjWyJLYZ9M2gdCVIXsNKB2hwDWDgmDQ0Ljq4Qy4QJJQaVdRb4M9Wgb/bYK2bDYijLSfO/m887cbgKOnRVRHolVhEIoepV6hefgQZ8H6XgSBpb46aXp9AWUEdPmVB1D//PW/YqlvoGuku/JbTT1ZRV+tKt7qbFU0H+M7mmqnapTzy0yNcHHmR9MQ1VOnWA5FIJPDFYjUZBiu/2KWS7HMNg3dv16xgbmDloONcE0LJvfcfZBIx2aLRksj8az7ccy2RsiEDGeon3lNoTW7cf/wAHe4qzDOG8fA0Maic5JiBXVAdVHnj1j4FOCuKtxjqaXDP5sfB1UxS12KrRWK1WtFdSSPq8mLyH92B3oZgW0gETbfCmZ0bTUCVP7emi+t3ZzSuKIZew6U0EGlRCJ3rxOu0o0xKh9EZ52YwqroQBNvzFriWfA3fLnsO6wfRqB4HfHU32u68hDdSl+CS+x+MRW50z4pBDFfD57GnMCtVvFDkgELoXxMdtOA0HZcT+eSbt5m24faaZTFxLayMXV5mXnMpQjv6CoPqPrv7Gamphe/qI8nZORWXsI2EeFeJakZe5kIyle58WuRJ96Yl5EwW5xANDhoO/ucb1p1ZRt7HIQz+Qpz3rFGk4FjlzIdJnkhUk/vt9dvlfmztgAm/PsQ6fxPh48TzkTONdF4ejKEgmAGSCq7Z9dRlXCw1TG715NytsZzvWMW5wrV8c3tpsFrhL60Dp7+Yg9DaSVjhPRQ9VsSQyA6Cby9d3Z1qokq9SbjmyY+lz9P9sSfflvbY9Fg9xzOiXYm205lr//qRr/HgsY++v30bJdhpi1i/8QNeynYjJT2MDt/+ZKH/V8b/hHf8b47/gn3/Hf+nw2AwoFL1Fq5ktfmEph9Hnz6CxpFOyP/T+4hqZHaE1awnrKKTUYfdSFglXmAEUzux7hmUtJgRrAJCH2bmQkc7ZrcM4lZ3I2/6VPJB39lUENjY8TeyZWsYH1hDlyBODIc6hfLUv6uRlDSRMSCbQYregsTQawNJuhlPyVsfU+mgxc/GN0EimHhz3RF+vOXDq6++jWKyL5NsnOOvOEaxY8H7LLw8ihf+vh/VW2KZi49btBglbfw+fTdYJX16ruBi+kDsdoYzZOVp5ENcQOQoBQ4SV9wb3enyr6VL3ffT8LyXM0UbNnDIF8aqe6+JHAPuUnCvbyDy2l+UhArAG3fj3dFtDHz7W2598jZeRdewmMUMssfsPShb+z0+UrDvwygSVO48dSSFEvlWBt+/i8gQcUu6p50XRedGkpWTzB73z9nwPwhzX2kKonjLLJISW3CTZYJN36FwO9W7NPwK6Qm3GO7Suz1YYxpL3Zl4DFntlLXZoX1cvNH7IfF1TnfDV7V1OErFsYqOUrKjzuJk0lDn50uBTecnwHW9hCudoDJ6MdWphVwHcXI2W3GdldnJnAb8ovrLeKprN7Nt8UOsqnSiQwoO5gqgVyJvjOwqlkkN5GitKHVKpNKehUOuryQ0upxCCdyMaEJrb+RFRa98klVXz+VfEgjzd2BJUT7Js8W/S9ElI6A8gM6iAPKQECMRFzOqg5o4NeEU/pX+5PmLwWqAzXn+mL6cx2S9kqv/A4Bk9epm8OPbWCutp8lFL9JsM2nriFHsxss0jp80oTja337+pSrqHSI5s/RvPOqdmHh6FGn+4nc+ymkX9f4J5Lluwj3oNWo7P2KgzXMw+tYLWJuaKQkYiINVvMg6ek5k9f5K/Fo38M51I/bPiMkDXfbjyIlW09RQiX/6t7QZlgO9FQuVwoGDF9cCPclkt0QMcqY1XiHg1gGUBifycGLmi72bC7fRGShj8whw6eQ9dzhoLr0bi3DwprTDEWtdCPG34ul0EieriYp6ak870uZoz+9vvs9g1VOieEvBW0zYG0tVyv2cG3KDcUPFBUrnGhUztt1HDbBvaApLXxOFudZ4nD9XnQX3/UwwmFDpu0VP+YXaV8g8eYauuN+pCKi869lgbLiIYXAabWZHNny0nsiJ1zkf2CMbCdBS44qQUo0wsoiasF+43vQQ2PSAvupfjY9Mx9t2KtA7iJNdi4kN06exN30SkR8/hNeMPp59cidqzg1m4umJfA74fxAgKpgBuIVc4txw+HDSGILcNoNIIA/a7RuYcS4Z/4zt6B7o7TOeokpnXlwb8/fF8/Yv4zGsEgPh5V12GMx2HPOsolJtYZkNMNDWWYNn/KNkKk8zdGY7Gd1ilmGTIoKLOybgYKfnnUf3c1rzovjYumy2LvudH3xNJIfX9QOkrVjxurkZ3z9f4bHP3mJVVKQobm+u5Y3/7OepI3F8fCGALBt56S77bszSLprdfDCpA0HSO0d3q/yJ1RThOLqE8iUX2KMTgwqyTnjxoxfJisvi8LpKVJaRTLApPZ1teouysmZGFH5Hm1MbfUdqx3paXbZwZOpmiArvF79QdR3DJ5Es0G1g2Otb+8XN3Vqcmg2szXLFLEvgjoOSRBBwaHfAbGkg3RBHk8LNZvaEZx3TmO0FC0+F4VfthzpY/M5HZwQx/PR0TK4teA8+iC3YJ9XXsz97JDfPD2RTjRdRD4jXtHZjC2kDDlPeMY6GzDM8KSawA5BjzqPk59mEerbgFnEF+sCc0zp+4uJbS9iwZhuuw0Pu/n1uVB5lYRn8lO1IUosHSluZQ0FAVfwlnpU6aqMWYVCLQS+VTEl9yEgE7Tlynl1PymKxlFWB7BWqW79FRTM6OxNlhg5soRY/aTtDVfBnO6isSt7v85ssmgjOVK2gbtmfBGv06E3iTehw2Q0a7L2hU8PW4LPQWYxB5ou2s4Kohs2YyuDntb9hVmiw7ffSG0tZ/cHPPJI7lfPmP7EYZ/IkL92NDzP5E1cwneJEF2p1czFbxX6W9jMvkTb9ECcM7+HTPQVEQkMCQnglltHXqTo2lot1dog50dBhVZFb7cxTaVE4RXeKg8Y2Ph1Swf7qRk4mniXTkHgXCr9c+y9++dlYC63knl1M5H3iQkeVTwBRr2Tw1UULWT9P4t53xbnPoPBX8e3SsaI9HJ1SvDl2MOTzTuzz7LwWQMmZRRjniOcDX1MxP5QlUL95ArFzttHa4YeI9SwINNbmEFK0nEnORqI6/wCbO3oH0FboFYTVvoyXpnfm39RqRyPt5I+6SrjKgMbSKweoMLfzvauUU9pGjgyIQm0MwbZwerWtAGVUDspuBc8IDvxoFsvADQ0IJ3u8hLzABAxmVb/cXWaS8sp7r6AwttI0NQsQe4uZJPZcuB6N55UhrJ98DYWurqdAfue6WOZwZn8pAP+Z9iBYqkGqRClTkJlQQEloJecawc7sJgKITJ0aAnKDGShYCCwPQDpLvBYXGVxIt8DI4XkMl0nxEsRF3cZDY2g/mYzjq99TYi9ehwWVO1/Jm7gXkFglOCMmxAG0SZ+i8dUMJgBBgz/EVhtVkEgwFPcUcm9OWkCQc2/ev7K5mMLPXuTq7EN84px1VxL9zjjb6kJSmz1nOqXUNMBaqfPdmMRqQj/lPBeSU9igKcZLNwTbFa9dHs6DieeY1CJFf3IcXZN1oo7EciL41HAfI68UI7PWI3zav7hRbvqF/MhIykOseGl8RbFSZRIftP3OE27whT00W3SAHAepDKfUBNxuRpOVfIEso4VPbEgPalMdP0d28G9nBzs7wdMoLqb90x5E0JtryY/u5J+cAzzyiPgZ+7MrHgdZI6O3LyZ52lWUD4vn95dzqgm7NJSlWXGUThUDGgCjfBdzlnbkRyrIk2b3i38tCyfTfj/qbjVSYy8ydcvqz9+/T4TWIs773KTSI5iHbAg7luY2PEvycHaSU+Xe38unI8COnMlTaB0+nVv2Fn6zBbrljvyYtonrVdns+MeVtiQNtn3Xvto0LjrZ82V3KRflY/AV1H12j+AmKyXnrcdZJe+i8+n+9/LMxEvkzzhMrR1ES3rXcllJAQmn/6UoaTEtvnH9ipd3/u0VUI+8Tyy7IYvYC1vQa7pRvfovITakAwB3UwMeOdFkRacx1b+N58e9S42kN8+Y4lJD695YMtKj2Wqx4hYufvfUMh3TLoxF0eZAZkJmP4JtJX78/myP2k1sxedMvg0qyy0dSDyayYvKxX9INproKgSbvGmuqp5nY7U8+s9qvPd5EPDgGdFxNfzL4MYztN+YwGm/coxjxfusk9fjafp1KlMMCvwGFNDyyExcnOJvXy8BnVqHY7sjlvxOqozjxRfNamTdkBx+OJHExf1jUPuMFXWWCIJAbkwu93w0j/baQOzX9V4vf8czrHvvR77Y/jWK9Gzak8XX61Lzx3z38RSa3HZwLekqydwnih/L+hfjr9s5OfEkN4eJ5esivAeTFv4bSu8m1Mfqib4nRBQvEFy5t/Z2F6ICJknE3dF9xx3S9p1xpeoKh5LG83rOIGZ6aTGYK+il6vXc14jqN3DrHEOAWCGXG+EllDrV88rexdjF9vEalNnx0+hTRI2WM2BkAVk6cbfh5eoiZGfyCEw+z30rz2EyncTW91FrdmTRm6+xGTiycjeXxWreSCQCE05NQN/syiPJO7loN0gUb1KtBnqUlj6Lz0awGECqoETXysXNUzEV6/FZfoAJUeL9tmAv49snvmWobgyNcde52S2W5vXpNDHv+hg8CrpRmffAgvmi+Pd7xzPoSiiua2sx3l43FLSCUYIgOGLXVo2uuAMBmBL7gQjsEwRosKioF0ykOf5NugXk2jru0DYlgoQVf65Aay9n+lw3FsUcB2MbKF0ZJ3TQvWshN7xicJ2RyU3dCkLukMusVkK0cgKLY0k+MQm7Lju0n/YC6Rb/Bbx3+hbFl4p5oeoFRk+6jnxWI3AbhBUk4J5MU1QD2R5fctK1kmd0vfuVU9VTKfP6iYCpBp717mSek41UeFcpb6vL0DvDx9YesoSAeB5c5XEW00I1zSUevKe4QI0uDHgFiQTS6xwpaFST4/A+jxsqebMr+u7OUyKRoZp0mfbhl9nqBG3mVtFxJYJASUgJf676nWG6OLQme7DJzi3IuK5/mcTUWFqCTHRbRoKN/dB8jxsMnFfDPnsjo6PEhAmAlKINjMsp5LI2Ae3gBmbaTIOFf7uANh7n0il8tW0cqlG9RTg3hS8zD8/HtdkejWcTa3zExMcOXz1XZx1i3p55JKYlUt9Hxfbz68fpvnaJuKyfMU7swFYSVWbu5BN3KCr1Z/emBQQvEwOYNdoymlRXSB23H8tEIyv5ot/vKjM60HhkCHMyIxgyuX8n2cqYbg5ecEGSnwKmKfTsGwQEaQKq4mRm1H5ER5sTGpvpWSJA5OA8cuxbubR9Mn9EFLDLJv/ulrjywxtrMbs1sWfeHiw+YgKAVGaly6OJcpM9llo/QpRif+dt5eH4fDmFgOG1eA64CCKDIKjVJHL64xWEFfuDSz5SxLlmmnwq+1w/xT/e0lP/vv3uWP2e4+2N8fgWX6dxXCMGjSuC0FOftlpBL3Mg/MZgygPK2X7PdqKjohmuceDJ26efVeDLnN9XoHfR0GlnZbD7TIicDL6zcFa683rmca4di6M4ahiPjLpNSFvSs1/SqwN5qkPPRY9qssZWE9Snpv7/2uiLd/xvjv+Cff8d/2fDYrGQl5dHQkIC0tu0vGFTFiKNc+bMgWcp1s5nno1nn71E4JmTehoLPVBFSGhSipOYtJJveNTvF5qC9Qx+rIafpeKil1nqzP7v52A2Stm78g8+7JPgt+pa+TY0AQQrXwI/ysQTk4P2BHHu50nBi+9mZPCDuleGbtn4NErCyqi9fIwvzAPZ8YBNVinVEFcG9tV2TK8QkLeKpSSc5V7Yd8cS2+qGpiwAZ6vYk+R0rQ/WGil2PjV02fUVfoQMF1fkk06xICKXbMkZ+haxx9knY/n5YQoDKnh36Q0a+3z+zuZxaS14WkO5424zSHecg4VD+aHBlTlr91Ie3I0t2Le1M44r2jSWr3ifeRo1Gc1KxtgU4+4k2t97SJltLzYONlitmE5dJlwn582vMzjYKU7gA6Ie4btDzrjU5aEeW4fV6nn3eHfGV92JhMRlMqnFlRBhMdC7AZdIBGZVhmBf5UNO8nUcVb3XbYZqC9svTSN7WAeHp37OC8rvRMd9+/BAdPkhSLwNCH1mSJW5nsdXHeNihT8ZlwcjxIiL4MXWUKTvPoJ3nTfvKO2456pY0izTHIw5RYnKoYsn+wAWAA1SDzb9fD8hZ8N56O2NtHafAxvHKiehg4CYcio+X84j4c14tB0CRiDvzOWL2Wmsqk0jpwMEq/h6/muQsXfuXoaWJhF7qYPAJFewEU69v7qOmt/WcGn+BcrdJ/Gjg1gq6JsR1xBGtTOnGhRm537nvb8ohJl6JUnTLzFmQGG/+HHFAvY6/IJCqWO08YG7zHoAg3MCC7aOYNT5ZP4mmfCXY++WwB/wHU+cj4pshrPqo8XU2N0rOu6vmntJnZHLuMNtqG/EY5qfxV3Q12ImqWkjzRlymv0SoE+CLgjgWXIVj2Y3/CNT8XIQy8Nkahv4XX2e8bI6PFs8KG05x0hsaJASOTc9niHgdqJqUKRjC165tR1m6EM7+U3rCo0XmWCDr77VPZdacyUepSomlbUzb35vYcqiCuRI+Kc0h5WyeKccyTBxIq2VD0A5oRicrlEZAt+2iQtLA93LcJ7eztUjw5ndEE3kO+LyzcrI64StvMUnuR7kRok35gAG7XUGBu/GeGQsryt1rHtxpkjO0NrcQkjWdYpmVvPM4AokunIgkMCOND56Yj/DbwxgULsUR7d2bN/ZtjAr16eNxLV2P098+wSNb4pZ5h9X+oC+Bf3aJ5lir6GyKQxPz9tFL0HKiwVhFJoyGL9IS6V333spUBpcSoV/BVqNlmxLR7/iPUC14/esrCokSjcCWwhJ3pHP1yo7Mg1KrGoFKms10FNIPa+PZ09pDs1tdtRipsEqBnXHXY9m0Kn72fjQRrb61Ij8MXRdFdyTpiWwbBavLP6bMIv4swICOxftRG0IJe1vC0+IZfxxtjYzNLqSJ7oFakugrs8cKCDwcEI1F7Li4et/cJ8n7uBoUg0kohSWb1nMiOicu2ChIMAKmYw2gyM3prXjbGpBZ9SiUvXsbiSSHqnYNsHC413ddOm9ReUdH0UDZt9mOtwbKfO9SpNR7KpXdCuS7r1HmNYxDaeE/H6SaJKgQE7Pc6csJJdwwug7BJMFta4H6GisXNgv7jtsIRF1och2L+BQYK+0dSSFPPvtE1R617B4zn68ndyw/XRZTjAbTwxjRKs9QoUv7gvEwO3TXvVUOAURs3I/gyLFpAaZVE7hlYFQ40VV+AikfebIIMMtsgLhtZxMijT3kd62jimIc5Sn8qJ44FYk16QCwpMv9hEPg7NX52HubKd89yS6R/VGJ5eNpsB6kidPTMMhI47gETasUasVjf1J1Po4Yi7/gzYkCFhlcy3b8RxlIKvDQI3HTbrcxC0WEqlA5oQnOBsXT4c6i+8sy7Ddhj7rVItdVSibX15J7iDBFv8EwOw6gl1/dNKi6OTVpf9wKFYMLGgEHdOf3MaVMnei7I1YTHryivJQaE8w3i6NK13R1HrXIreI19J9Ok/eK2/DXTjAVLMTgl4sPzqnQc7Vv4YTuexPhs5+lOKmvQy2WSvf9jFg6ZaQ+P5q5AMKxCctkTKhzBPrwCuoI9NJdFtJ37GrcCQJH87FCRj90IE+n1fSFjKZtnkaMnVJjFH0ynU7d+xn/gdH2ZAzDOONVzA5isHs19uyyecIK7NW4lPjA1Lx5m+W84/kt3ry74blGGMjGWyTUkmsOq7+EoN9nQsJBBC7QJxXOZqb0eRHUGeScfLoMEIe0/aThkxq6FmI7CdfwiATR5WdORzNHcHlbdNw9mkj8Ml87kjZHzmaTHxaCBNX7mJ7RCMn9Sncec7kVh3Sz9eQ0OzAw+PSKPduFh23QRbCy3Uw/koSNUemo10vzs2j5L+xdPxHvCUcoNvogMBobNfrdc4ZVBp7rqO75EXEwC3UqhJJ2VeOu0aH0zgNyMRF9Bane7FIPkFiMXOpei2zbxeMJzi4YBc7g2NZETz8YwQpI8RsbN/Bvux/8Rijr6qIPTMet6nirm5XnS/xBeNQ+wcREZmJfR+gfYdiJgzN4Ji5DdeucGxFQCWChNyYXK5/soEDvvCTtm+/IiS7TOMCGciVRsI8TiEC+7DSHhHN3xEXkFvkFOib4PZ8U6ByQjn6BvZ108j/8VPmvCkmmcQN+J40lRHJzQxOlXmxZn5vgVBi6WLnyEp+boMT4iW653tvT+XnRl/gcvJV7vcQS5uW67K4Eb6AIO9FqIwRdBo6cURcZZ+eEoF/dhzL5t8k1FIKNrKvXTJ3tKeGk350Oice2MU703reD5l9MPs8vsU0xsjkE5dod2zD1rRdZtYyy+RAbpdATvkGAlz6rLUygeOTjxObOxrvK1uwrL4XW1mzs0H/UPr4JV7/+E2u3ZrKSBvCJoB7lT0xWYEMGJWOz+D+Oa5g1WPn1c65qcdo8HLuF7/P+WuOHYlh1BkZzSt7ZeAapBF0pcUQ1+hBHJ1cku0GGxlDZV0GgVmn8dba80bxIpruFz+DsuCpBEXkUJOyHX3LPaiftQ2qKdxuT2CZBhjMlaAK0WdbVUF88sNsyI5g7Rvv8FrnPX36+uDZkiSWVHvgHmDAWdb/gRipNiGXwU9t8Kb/bZDFrCc5+nPSaoJx7z5DbO3fGLqX0utMDQ/aLyU3J4yBUif8HXUiMFBubMRj7E1q5DrWxzbwZ6uY9HBOPh2+7MR+RCpDp5wAoN5my7M160Pci/biHKWgPOhNPPpglJ7mfJ5afoo9f8zgRG08jWFZQC8IZSd1wqHDF4m5izyfFzBbH/7/2DvL6KjOtvv/zlgmMxN3d3eB4O4upbSUCtTdS2lpS929lLpQCi0t7u4Ej0EgCTHi7pPx+X8IMHOS5/P7/td6n3stvuTinJlz5pz7vu5r72tvQEFw888cmXWZ2MRiior/5EiplZkKG5lDpxrNb9tX0oaVQXdvpTFAPLdbrWVMu3crf735ACd/mkNoP8PLnw1uJAZdJy6xnMkjriJYFvV5lgHSrhKK/OR8O/0Y69JdSTb/wNP2dgpSRyb2qtFN20Hi+Y/xFsQWEg+oOlkc5s2jg6o46KhnhN7mJlinupe110ZzKHMflSPWMcjHwvN2wqlnm9fhW1OPd2IDb83Mp1TUAwQGtxAuDcnk4VFf4+wjJlABWCVSnFvKCb68B/Oo+wBbAV8iCMzaOov0nHS+efyb/6i4tCk/nfObx6JNKkT7ZLso5qCt4t2N89FfjiY7toz0hbb39qG8UBz2zgfjYSSaE0jMjwM2MNEcUU1FdDsNx8ZgdhGTrB1ljgTo1nMu8CX21pUwUS0GKa9LStl2+9/MDW3FoGynqusakXZgn0JqoiKqhNCSKILz+xtbgKOhhpFN/hir+tZXab8cudNhCIUZZcRezaPAuYfkG8X7RkU4P2S+iWOCI7lRuciMLthnAc50sMgJ5Bc8+ezHF4l4OEyUAyyxlpB/YAZgwSrPQyAF+2ffIrEiNUs58UkslsxB4AsjG9zgH5BL9Ty94FEaZ0GuOpeczgmE231tiQQCjGZCa4KoqfdikW89vw2ykW/lpg6CjUpaTWY2Fb7EdfmDPKvqIzt1Wx+n8OS/OKRHs0qyGDcny61OdawWfjd3s27DYjrcW2lOvIK7YKuj1XTVceH8CtIK07C4uFHSMQgve8Ky1QrmXtqyRnHFZR3+rQuRqmy1RdeeDJx7i0D3Ez/udufHMT7cMrRxiuS53iiOlrYT3eRGVEwVwYprYOegK6gsREw/Tt5jL3NKpyZgWN96KgiwesdQRpzMJGBqDip1K3XdtjnYKnNmdDVoujQYzw9DF9XIars1TSl0Ms1Hi6ZVwZj8HejDVwGHbfdbsBDS5Uzc1TjcY97h7O5Gwh6wqVT8VjeBBpfPKB+9Dy9LAGLxUZhpOsq2Y32g0gpJDtPt52AsuEw+SfP+YXi0eNBrp+rxiuRdzt8eyrf5bzJh/Hc0uo8QnfdOdQnPa2D/ZBMvPv4B55w/FcVbOzYTd7kYuVsH94zagCDYwD4pvYw4PQT9mRh2TSnGyS1R1MlbXr+Gifs6aHedyJUnfsZZEMsVA3xZnkVmgxPhQ3LJ7Q1kar/4ssM1NKTdTdKYb7ig7WYBH4NEysni36n+dDMxqZsI8i+kQLkE6JP3FRw8SdO1QEgLr1fehlalFZFEumX+7B/+I13OnXw0roUys3g/M0t5gXUKJ5ZMquRy2XuIWyPguNIR2dxNvJHYwpSoBvqDfRJBwKAwUBxTy3mvP1ne390FgZqAGkJ0au4wuiI3dwIBSKRSmv1T6fF0IPnoD2iy8pCwHnDAagUFBn555QN6JWZu9zCi9nTD3WMGX+apiXbsYYWkCcn0HQxz3YpCdp14ZzO4p/X966mkxSUBc7QPO+bcQXJwJ5kAhybCpFP4+2Ti8+JKFpih7KUPEdQD5/b/K+M/4R3/k+O/YN9/x/9Xw9GjCzefBi50euPa443VausMETATnViIRZPOvsD3GdTPA71d6sGvn96BEFxPx0OlhMZNEMV7HIdQmGIh5vx+hpweMoDNJyAw5eQIfMtD+f2utQM0kbWOQ2gapqWpfjJjLn/FEDuprIyEciRR1+h6+hh6dQn2xTYnhTupGzYjMxq5MnsigzLFjNF7pP6kH5lGuXMasqUGYnzFMhd+5Z6M2zgJg7qHqolHBtyzXm8LxzLOkVfuiU+PI3f0iztJilA7d0NVED4NA73v0jotTN90L5Jmd7bfZfP9aJWHcmmvBleLlSN37qVZN1hkBO3YJiczPxMhsRxpQAt6QSz3M6TDwsRzU8lpSKV9sIT77JoYWrVN1AbsRNPhwuM7L5E+VCxfAHA9aToyy2USjn3HpcYoMkVldLDI+ha/hsv+FHcGiHzLBAQeLE8i92A6X42+xm65zZPnnORVrHO0dAVkc7+7GTdBXJi6XBRE5oVUpsR/SHByLbDxVkxu1TNZBfmVvpz8IwPnB6eJFm1BEGh3bce3wZd27+gBbNZLlnC2LPiEFIXAtlo1J8VhKmRBVF7wwQl471QEt48bLYrvNk9ms/4ME8ceY2RAw61uHIvbIMZVwxUDPOMk4W5nC2hr4QZrWiqTkJOeg5ePK7OjW7H6i9Mfl7ChKGcXctm4jMj6CXj367ZvNCmwSOB4ICj/AwO9OqyVr578ilXhPdRJ4kUSMACzPZdh+PcBxtS8z9ipu0WdfRKJjNLQcpJyU5BKIqm02J5/D8NeHh/0Mm8c2U23wg+TgziBOqw/SY77elwS0wmNKCdSNsUuamXBwp85EjMNxc438BnUC3YG6+2dlwmu3Y5eomRjyC/MFNfpqOwuodnxZ5SzHHk/uYn9xn4yoYBVpeb8kG5eijqGg3otYCsqy61ahmRd5dvyMJTNoktmVM+XXLrcRUTlG/jWqdDNt9OGl8po0/mBxZtJDy9D2k/ur1USwGbJbUiUp/noWDQNweKi1hZlHIWjfuPuUAVTlB04WhZj7z1arnJEMewM/mng0t8PEEi2XOAuszffHRpHVWDVgHlQ5rcM+dKr+PuX84y7mRZTIxBMqdsonsndToWDkRV3rkGwyHjMroEj3n0ury4u4o4j4YwMDaXRTdwlUXsxlknbslA8tI4nRxeTY7JL4iVSKjZkkFIYz4aFU0lViTfe8p7r5I1q44ukf3mzs4O57s/TfyT3vMisdRrgEXa/JC7WyXtrCd49gdbcJBZ8sxznzneAPsZsuSWQ6E8exRxQw+rHVjPNS6xh+3p6EWeNAg/+9CAFiQUIT9h+aKvCg31VbjiUhDLmzBf4x4aIjnW0tjM5oZreHZkEXPgJ032LAdvLFyqU8KIfvHH/y1QEVCN5YWCyXD7kHJa8MHqlwwfMNxaJEq+jQ+kKriZw0ikkVluRZsrBMVw9lsaYT17iKT89FR0XcHXqKwpLrXqe+fkBrHoFf83biIeL+L0b4ZbH3LcO8XaZEx/gxRmLuEu5wLqOYGkvobNOkjr00oBuhE75e0wL+IbPc8dQGNmffgKeLo2M/PlNfqwL5XT1Nfo70TablTiEVVE+9Azedv6MqZI8dJMU/HXVlydWPcGpBWI2a3ZTKAlXfCiYtpeA27cicxJ3UkpSSwlKv8iDTSC/LqHCLqZQB3Ek9hOEWDApVIT2yz8EAS7+OZWUg1mkAA4zT0A/sE/Z2zdfHx11kLpmJ6aLetlg7dh6HBL30K3pZqFgk8ZTNjoxJXsmTY5aXKYcoUWw21xbzfz0+G7Wte9FeTKzzzPBLv9oaTnGu+Nm8kmzI4rRvVQaxCSmhu4viB68jqNcBsvAzpFu5bNs/fYiYMXNKL4egPKOa1h0u4gvjSM5Lxlhgvj4o6ZRLFZsgKjrPFnpgME5BihAq0lgx8wX2T9uB49IHQiTi4lMgiCQlJ9EenkEYTmpeD0kLlV0+U9CMz4XU7sG5akMZCFi4OtU2ToK867hpS0jTDLwvfGp8yQxN5XM2EEkpvQvB4BFCVWBVfQE1VLiGyvenMtU7GpaRe5VGCbvYHyC7RlslPjxq6aRK34GnMKH4i9uosehR0ZCeQInxx9C79bOUKu4S+L9bcMJOzKY+qAABD8xMawUT/ZEKfCJ6+Gl2cdo7ud7ek2ZyZvDnsAp7gzdmm4+lP454Lp+nJzPjoRK7hSa2KCei+gTpEoun+8TVw5Lq0Swk9HP6HgJdUc+pl0zefVYM0lv2kDMcL9xrFN3YJbXcXL7SPSDxQQUAYFHVj+Ce5snDWFZjAjKEsXP1aaz/c/XCB38BrcPu8yV7kIC7AqQ3koV4355EzeLjIv6gWuWu9KLM8PbcDDKKMhdz3CFVPQUS1UOFKWnEHf+In+fUzBD0rf1VfWe5e7k1yi6sgVlrz9ZUvEa/6SbnC+ST/CyMBT/Ud9Q5yKeiV7V9XLin7FUJI/juYqRfNePl+Ds/gTFrv/i2r4NlVIMHkmMXXzlBeVGuL8B3NShA64rULaB1A+O83xbJxdNo0VuJwIGLoz9lg15I/C1/oSv7DO4QeU75BTE8Qmrmbc9g8CrhzC2xWBfYH8y5FFKHMI49/YwSqJKRGQMQabigQaoKw3gjdX3Uz5JTDZSWxp488hk6g0STs3YjVomfm89qeLNrbfh0unBI+9+QJ35eegH9t0/5iJC9HXSInJp188FO9FLQRCYsrcvj9tnFHj7puS7IGBQe6KTwsjDaQQGNyA3t3Kzi0eniuDFdxbhV+/H7XQy8slzYEdR8ZB14jjpJN09TnjnQHbjambY+SNN07SgA3bNS8GvvZ+2HeCfVoo+bg+73FtokSTxUL+4o66ACaevEpGsojploAeo14WTDD8iwyOogeHhtjxRIkj44aEfcGtzY31rFGmx4vXQLfx1Jq6o4IfNs7mSnUTmA2KJ28VOdXjqnHnzQj1WdRGCYItXdVSRE/oTAVIvzi/6B2+p2KewSxnClqhfmXcplmUfLEP4eaDyi1Lbt2ZNTS9FpzQMiL/QEMCFLxcxWOHA+AM3dp9WE7NTfqE9DK6uvgP9+Viq2//CHuzrrOihPduBo9nPYopv4A27/bTc1MnLi47yfr0j9246RnSqODl/IONJlqddoiBqN3tqQHN1HZ/ZYVutRjeqshZy21w5/GsdACD1yPxYo2lAUudFdVEwypHiHHu55hhpOyZTVRTCkNe+R66/DsTSrRnMx1dX026SE90yD5lCEJ27x3EohesLCDY3cP8LOSRLbxPRX6+axrJYWEbUoj+RG+U8Koih1aagDtbevZ9hSnjFH7SdV8Dc19ElSORcyk7E7bfZfP/cWrrd9iHyThckVJoVNAZW0qj+i1GyAOwBCYXSh3++n0Po1SQ6hz2Bj1298e/GzWxy+YkUj2KKQoz80H4W+6Fz38fZd7cxQWNBKgy0PHBwuYI6xoRrWxpOFvEzIugbeCNlIRWaQMLHFHI9yBGw5V0aQyNp4Q2QA8/8+DBuMyvoP6qkfWv7YI2Zcq6IYg6mNjx61VS7dvDeuN387fDxrdhov0auR9TQZvIiIaAYwTAN7MTCT4boqDLJGPLShziYPXjL7rwyjKxM/J6D1Z6cOzMexyzxPmisdB/vzbrCIx8s4MtfvZi0TdyBNFx6kpkrNvPs0QkEaT+j/3DU1/Dmij/58L27+PDYJNRzS8CO+JbbeYADY1azYeYJnHrjuevGmiUIAkWxNkBHsIp/C19jNeMuZbKpzolmYwiSfkS8eq8huD95gkKP06wK1/KbXg28cCueNO4CPaP30nJqKWZVLdiJGsikJnrcy7hmbeCFRghviWKB3W2RSOBFYyvFP/ZRLfOT8kX7IaWhlXc++JG3TqWhWbuZNvnoPolNoFcTz4Wpr6L0XsWismcZnHoGuEG4EyTU+4xHPs0RRdJufkwu4esuWx4rM3bQO/YU25Iv499cBBIJQ/tXtzeomewXht/xuTwycyfHrYe4KVk8MmgjgwO3cfznsZiuD0cxxk5OW+HKSYsbgzaMw6/ej8ShBUTcvwb7HPtn7QS2t10k5M71GBQGlsseuXU/yiJq6FG3EH85jaDraQy54wjcmBUEQSD+cjypuam4lkRz0mqT7gdwklSywRc+zVVTUe5Lb7dYNUMlb2ffnSf5VJDQunUCSZnrRfFyXQBeu2cxuSaAdQ/vo/84oR5Ey7QDtBaHYFCI2++ekYxja+NWEp7/iplOVozN87gJfHUoRtDpraTXPZPtOT8RG4sIUAurA/2eBViCQshzkyPzEBvveUV/TvUMGeXhZZxXdmMv3GuRe/H9xRCCr4Vw6q63iDeL635BljIix17nvNnER+4KtiImvwA49MrwqfdlR+wVzF2Z/cTRwfPINoIb43jzl42s7rUpnWhCPGgMySCwKZ+qXA3uS67fumb7Oe/YB2/hJQVBWHPrb4IgcGLkCZy1CYx1acKnW7xndtcv4bNnRxMwuJqumfGUtL/DRDsqqtHRTGFKPhc/eJkj8UV8LMb68NEXMv6pv/miHZobq2kz/Ia9RP9Ql2jaih5HfULPklElaE21QBxSUyMhkeVUGjzRSHMYm1CF5IYaldUKSe1HqUzQMb8OvvGGzdRjlPrQXbeEqUnf0Ki8nQ2qZ7G4+/FqSjpCVxToNsGRqViC72C7YiWdsVbu8n2Cp1xvfJnmbAC81d5Ib5R5nvniGY7fLp67/zv+58Z/wb7/jv+vxn213nRJKxiUMBeNDKxWEzd1ty0yNe9LV9Gacpwux5dwtGaAnVqzVubBtsnfE1STxvXfPuXuz9JE577cfpbdY55n0OBmbnPVDkhYBUHgIb0H+aURvLR3Ji6ZtUDqrfiZ7gZeVH3A4t49vODdg7rjabghlJFc6kuvvIaXkhSEu1eCrhE0fVUeuURK0PU6pL1dvPuKGxd7RgNHbp1X3Z1DwxkN5jgZ5x0WMaGflc9DyZeQqqsoWjOb8dXiIjfAPc7neakylPO/LOTsgpoBcZ0ynq4ZezA1H2Va4ECJxQc63cgu6zvvaIstgap2SOXzBX1JW2szJFrDRRJeg5u6cdw5g0OGfYztKeLJMeIOjUW6bnJOjQHgnHSYXWoEUmMrm5/exWdHY2hZs5d2J3EHnKmzkPGpW+n2P0xWt4ya3nboB/bd755L/b0XeGvGDjKs4phEIlAZPAmZfxurX36MwattXVuHTGouutYw1LGEV9uDOOIvvmd7puzl0LhDvPTxSzS7iGO9DqGM/304ns2edE7oIsL3HDeTAYA0SQGRL/zFl+3g3/gA9Z3BJNs9QwICFomFoE2zuB4ykCUrIPDNE9/gV+dHm1sbM5XirgBnmSdT9s6j0auC8ZI6drj2FUUER0/aTqUyrCwc14Jk8gZdJn2RbePubGllrhpOBx/mIeVixioUYomX0Hf45zgMk/5MpOtjSE1vYy9Z+c3eUSSd9OOKgxnZyBxWLBF/77eCy+g1d3JfvZSxrtMGFEJU1zaSdOQSLTiRnzBZJMIlmLT4JJSxIeYz1BXvkuV8FG6IDdVZ5rL2Hj2d/p28XrqNcTPU2HPIfLssTDs7mvPhpWwJushQqZ1kmiBlpLkbbdQ6Xgq8A62bWAK0tvE4T6xYzWdNbnBpxQCgJNpwlp/PjaPweBqPf/Au4/xDRXGzSc9Dmd7s7YzlcJk3zul+IgZludMURpz+hJHt3UwPTqegZgEjb6Sdkflf453dzvHRTRSlXmG+nXtdfVcFVs0zGLXOfFh7EHcdovOea93L4eT5pF9Ip3f7ImoeFPuRtFnVFBhACOgmwLMeXb/rOqmbyMrGq+SFdyBYB+rZF0lTOPJpABbHXsIe2YCfPgawEScyPY4TFqnjze9mMyYrlw3j+joHtQ5+OP47jXi1goYxVxCQiO5pRa8Le7QqLgVe5++EMlwd7xMRE1q9Ozk57CTKS6FMqnfm7UHiwmtpZBndmi6U+qWoECfSUpkjNZ0Z+Bu6qQ87z17zuQHXpW87ADfoCgqFGBjQe47gA98PcRnRQ4+ul3hD2q35SkDgTNYZZG6dbPODQkEMFB4ObqLctxQPrZxpUdXiLja5mk/mbSD1WiRz1i4mxFl8v53M9fzkAx/IOum2Xqe25xzY9VHWSGJ47bFnkesdMERWDLgmQRB4sN3InQ/8yQeqs0i7/sDeL00mGBmUm4anIGH54Gp6Lbbr3hxQhWK4Dtdto/kssJl542wgo4fEwJjwJnIvRPPod49ycq74ms8YU9jQfoiQHWPZdyYdlx/FRcL7R32Gz7h2MqsAPfT0ewZTzesRdo7lWG40vfPzB1wXSNAKVgwyC4J0oHTYX5VJdNUWsu/2s2idbev8Ucaxe8RKakO9ePhaDSkh4u/VkN7FlsifiK/6DFPxEyyaJz731msnKK69RkbxL1QFied+QQCzQkWb20ZuT15Ol2QY8NuteI0ylZ1dF8gCxizayzXERA2AKeGVNK78BB0GnBk6IF7u/TNJYS386eKLztEm0xKrMxN9MYOjYys5GxjFK+52hCBBwlONAsVGC6GDzqK0ykWduPUWGX98fBvdKi3fPL6LtR3iYnKwZRtZlWYkF+bjFXUd10HVoniDbAbFEeFMDPuRYWlroZ8blbTtFMOf+JmK5c9zW2sAjuYGbDJaMNb9Xlye76XeO4/1d9ngCoMymO87QGaU4f3ucoKG5YEdRu9GE3N2TUcvN9I89SBVruL5oMN9LEccx+KTW0JTu4rUe8TxquMKNEdb+PzJNfS6Sgb4vN1llCM9P4iJsXtwFFqhXw9cqn8puhd3s/nyCh50e5v+I7vwAYbt6OtIGjf+EDf9UBpkwXxW5YiL7CgpwzTkmu8GVt867jZ9O9Z/FyCdt4cVY6s5aK7FvgB5KqSOkon7kAQPwdvRFeycYBt769gz4WNUunBaDpZy222IfmtBIqVTraVT3VcU6g/cAnxuNFLVqGZi1RYC+0mqmTWRvDJ/HYJVYHDZOb53irhFE1k4/y9aJqg5uDoNOjsJD7Dl3kq5mpqwxbSqT7BpymwUkgBRruhlqQKfdqqDeugKfQKlt7jzblfTOVILNVQUzuS9XUOYfkRceLpuzOa9Iwc5GzMBd2M0Yu0LSFU68FRMIN/1/s5++S9Y6RBde1vXN6SM/gLD+bsp9rcpKnS6LuDlJ50xtbRz91s78HaqBWy61s26dE7uXEZj52IKhGjuulPMyNV7xhB/Wx7tx3Lw7yxBZ1oE2MDIof5/49lQwPTvHkWYmCs6VjD3MrM2lLV/TiAn5BrB8/qxjQBl735mBx3kHgNcNwSJjxek7Lp8P4UbXbhS6kzYp9tvPSk39zbtyZ2MHFmBu8d2sOt1tza3oC/0Qf/0Zp4K6kBuXMFNyTNBqqT1t7kMzk8hMuMKcWFiIoePxIB3WRjSZhf2zbrA9n4eXW7WOrzcdFQbtOzTcqsjxX785q7F9c8UVl+Yz7KNC7DHwxO0h1AtzuNbrYlOr9BbOYTB0I1T4MvoupxpeulbUjQWEF6xux8CeSl56B0sZGqcaZIkiT4zXXqZjwLg13v2sOSJPXxvfkYUv12nRZs3jCuGavSyBvok+WzP0IyAcua61JJ67DUm+zxH/3G1t4HGbjUhRhkR+oGkiDCXg0gStWwefZi9spBbGVWc8TAXI43c3dBIY0ojtQaxAPkB1Bz2qCR7mDsBISsYESK+n5+cHEbS2sFE0krKuJfBusn2vQ1t/LJkD1u6YY0bbO0Ur7UCAvkp+QzqdSGxMgyHfqAsQJRXA5WP/UKVWye+hrsHdP7tO5aCBxCdeBUP/W5gGkgdiS3zpcFaj+eUQwRP24WDIAY8lqQfQ5F4itu3fYteKnYC1KtCCCgDnytxzN54CKziuMpaRUhGC4azMShORFOS5Sfy7GvweY8yr7dYkjeCn4YX84uxFHvAuVfux9sGLY4vfoRJZuJL4TvR+TulGVQV9XV6hbp3I73h+9TpOoZvz4WSlZdGeu47uCZbwLLiFmBxrauAvJRidPIbRc1+mwqpREpyXjKjczJIuK+eKKV4HnzcrY4e4KMGOSajBVXjAeguBZc4cIrkaWkTw2OvENOpRG/qp41ntfJi3XI6vr/OhEf/JdfvbbD1J0HEEq7pPLBIasnw242jZBDQN+do25p56osFVKXlcnrBEbqk4u81V1PILG8Td30wnyst7szYJCYsj/KZxBLPHr5/ZDnt4WmMsjUF0tBVR972IGquhOC86k8u9oaLSMNOhkZWzjvD3yoT2jYnLIK42xHg2YRC1MmnSFeCi0H8LPS4JPDifQ/TawWDFRGJ+43AbtoX/YFSgEQHWKB9Hvuzb1v/Ms05FpwXf4Fe0099AhPNm3UoL6YwEujqp/TQIvFhcw90qnuQuSgG1JNqhUC+77bw1fh9VHYtBnLE31uTQnIlzG3wIFFjwMEgrs1EGt7lh6sOfK1M5mKkbd/gZLzOp56wu9SLtN/vpHGUWPGgwRxM+a8zqMi4wKYn3maiczvP20HOZsXX7CuHatcIljSVEaMR5wAPubeilhjYOPE7Wo3+tB16gDN+1WSltWE9qWRSuSeuzQqGVQXR7i9B0g/sqzQswGNCLWdM+SSlFqEy1XJTbUen9GdMNRi7exlmvcZl4RBY1oJEis7aSZeqEXlHC3nHU4nJuGrLIgWBfOV3ZEug3eEXnmsCV6Utx3RoO8fezCburmum/ch/8MQSBE7JIin5eh5qrYrqjHCMPrZnPCsmF9fCy1hr4mkbNJRERSH2dR1BEDg95DQzdszgSOg1GnoesKvqgK8uhIe/f5a85GxODz196xkUBLgjpZT61E5O+chY49KC1GrbZ0kxcFtVJJKSaL599FssGrGErY5IVv71DpIdJjbO24h3uJgkZbIq+arVgYLQckom7uOCcwJj7P+DVYJvvS8+jT7EXBfvmQGuSuP4dvDvPD3pBK8axeulIAh4NnmS1enN6yPL+KnXRlApdv2eHy+CsqsJTet1JIFR2HfM/lSbQcbZYLoMSr5oeIa7nhczI1+KKaA+SsfgKpB3u4piEomMNfP/QYXABx5QqxXvR0rkaWwY+huJv9/B+w8vQ/PbwLxpfNhVdO9t4GSPkaEDtC3AHFhJh7yGEZc0pIbYyDFW9RNI7lSxsUFA3duKg2BTRRKsep68nI7TzgmkPrmOqYnV6O1eeTdLPRt9JezId+SNhz9CPn4EKXZcvC7lAhpCD+NCMUtrFOSGXwA7sO9dz6vEKuCjgGqqnAd2K7oaq5niAiVn42lWF4C5HnuwL9h0hgVRG9lWl8jL0haWK/o6Zk0Nf/LRxOeYXgPLokFulXPthhem1QrNjqF8VuCL/lIwrx4eTc+D1/GIaWR/1TReTfqGe2N+4mzp03RXbOPr3LsY8UQI43JfhtYLdKtj2BClQWJ2ZPqfc9mqNjL7oa3gc4Pu33qRhe9s4a9tE7HkS5AZ/gs5/W+N/975/47/1dG/nTU+L4Kp2+5DkBuZfOd+kXdHl7GDw0lRhDf48lpxGi2as9iDfQICuWm5RJYNIuTSLiyGROwfcYsulxdCTjNLA9r/0NEiIFB9+1ac69wY5d+JxVyMvVm53tJLaKkjAdtSuRTSSvRjtqTi9n1v41XUiOczv/JYzGWaO8bheQPsk6Dn1beWsatkKN+ciqTFS2vnyAD1wcspmdqGtd2AS30RfcUd23U7+rTjG1DG2ohCCqWCqKUdQFEewOU/R2CILsUjeGDX1eHWbN6K/pDdY2CEcmAi1Ba8GM2cWkakbSMp9Dw3u5+G+U4guTYbofNnks7ryJ8l9q3JjLtKxisXuP7XCMbtn4R9swHAdn8rpudXc614Od6COJFAquTHDrhqNeGuUiFXiGVBikrXsyTyU95QK/hE3cEY6UCm7ThZFRsef4MZaRepX9gfuIX9TY/j6leIs7wMo5tt851n/Y0iv/Wkb5nLr1fikP8l9tTZG2qkQG9k2cvvYxVkIuBKECDhcgIqnSOKJZ+itUwGu1/EiR4yHGCxEyz0/4lNHdHYA8azldUsFnw5fykJk2wg+9dVm8svoXo+damgQg+yfl1V011SONZYjkTaB/BIbxRS5FIZ4RWZJBcEUhpeyh73WpaobMXPAN1VRn/0NH6xV/l2/FoqTc6i733r+s6VUmtSEPt0E/Zgn8wkRdOtQWh25HxR8IDjMuVaCraNZ/n+obi9OFBAMdBhP+2JWg443UN9RLJIElDSXcapIHitBSZkvYocd6CvJWxP5xlqQhqRmppIPnSF8z4ywOZF+LLkEocPL6BVoac6SJwUIghUGqUYFd08s+g3XPTx2Kskah1DuM3UiqLZhZiuKVyojeFBOyNog9QJc2IRrTWuHM6JYMjQfi0aCLQeTcD6b1/h/dhkQaRwJxUkPPXlU7i3udMBBGTaAJOU8AOUt/mREu3NpyPXcVFm2+iZdeXcluvG9Wovzo2RY+5Htg6zXOKznx6hUatg85zNhPiJu1KcWq3cveEOfk8s4BvvcazJsmAPGU9pfoTMTx35x2zElBXP2/28JJqlIeweuhU/hYUdYV3kGsRdGh+3jqK+6hCjWzzo7Vbf6i4VBIHk/GTaXXuIkIxDGVGNYDfNnspLxfj7NJxnb+GHtFwWOIqL899nNOA1uJmvXrsDWVE0gmDHerCYWTPnAnsrXbH8sAoXwQfsu6LUQXy57ksCzmxmbXA6iicGkhqeloZS8/B3/BPTzYPu4psqyJScjiskFgs9u4dQvjDzFtg31/E4o5ce5uUL8bgfH0LCYLHH0Df1I2k2VbBhUTnD3K9hltsYvjcLzl0+TYxffICeYDHA0yUL5uWnn0TtYOSd1e+w2mrFvnzfJfEmJ2o7t3saefK2KwO60QEyz2XiWx9E50O7kRnF87+HroC3Vqzllw4YWQUHpLbvtjGiFFPMRZa/t5ycoCrmyW3zoL9bCucyR9EWUc2Y9kO4R4ulmWp0fjRsG4vWAJ2jj5AlF8sgtpnlWIU+7y+VPhZJv4LvwcpoQnKjOZd1hupoRz7od03tVn/u2ZHEXY4yFJH3Ar+Lr6tSw6RN8/ldXk7iqMm35DC7pcGc63bH0buJl4ceZmOnuEA+yqGGOR6wsVKPXG9CYhXnAT5N3sjzygm5koGGflIbpm5m+H/FtaoaXH8bitskseSfRBDYPW03u6ftBuBV2ZP0H494NdFxPpLnnbVck/Xv64ZDKgku2kD8vHppd7D9Hp8OuUJbbCVnD58gst6JWPslS5CwLS8UtzYX9ifnIyDnC7uwUe5OV403mm4Nj6xMxWFSEIvt4idMg1D+okRjlaCoDkJzjxgMbDCUURJTz7IRu3Dor2sNKA113BPUxZnv3mGQo5ULVnG3orekGk1XD5FdkdyutoKxE6lUikSQEFgViEeLB4WpuZxyb7fL5iCCyyQ8fYCXe3Rs9KxhmMODovOuaXuI7LS/cA9yRGaSkWoTfQJAEhlBcWsLHY5arNaBPkArRufyR+Fg9q6dQlT8FDu4rW/c66Ih2FGgovVuHGgFbH6XmHXM0ZfQSBByNyM1woRbfWhSQcILn7wAMjMB/i00ThSTetJDKgl78h8+MGhZdc2NyBCxP1J9UD1lYXVUhZ2iwuAAdteltHZyuwZKeprpNf5ERUcKYCMy+chkTO1JYKyxi6WhHRwxDlRy+LrWmWd+u5OkjH/Q9OrAThNBIhFugcxHExOp7LxEzA1qToTzSTxkrvw4+HOwwgOOtndasBoJcbmKRNZMq0crzkaxVKwzrSxd9hdnup3I0vzBCetO7HPrHn8L77z6DiNO9CWRM/o9Z1aLlbDLVTjUjaYsWcxuB3DQFTI74HvKn3qO2WOzEYxdYCev59BxDMffZuKQUMLbQ8+B1QKChEvdVVSbsvHVuvHeic2EBBlF3R211VGUr8tGGtuOIsiAYBUXwS/LnuKYDqKUD+JjlNLYmUaEnUtoctM3GA9nEDGhFHWCeK4RlD78cfZ7LBXZjKkIpLI/igK80uiK8sN7ea4ijJKXxax/QSrj383zCCk9x7nMcxw1pnJzFr6vV86CLUs5HlXBoLHbOK43kmoH9n1wbA7h+3UonpHg5dYNdk7gAgJtbm2UhpcimeiOg0ZMpFO6jCd3eBTWyo9Y/fJjSD4Qd/bVyTL4bPxL+J0aTM+OdDbPHbhm/dXhwuJeBc0+x7jWNZE4u3vWLfOifeQZHrG68KmkCcFqABQYeuuZlt3A5TNe7F+wjzwc2XSfrVAtteiQjT3NkeHnKC48QqjEWTSfFJkjeXXFg9RFlnJh1iF8PMTr+A9XEon+O4s5g46QNflDdKYS1HZd9j82xvFzPkxZI0M57hog9gSstcKv0zdTt3c2VvMTjHxRfM1Hw8I5f9d77NR3EKpLtbvfVnpqvXBuErhT30i0EIk9lFhg8OVUj0CMRY5VIqXfloASlYApNYekuOuoUhsxWkzIuNkNKaXBDC3dSr67PBerf5zoWN+eAo4HwrXZx5mtPs5WWT8DZ+BRn1IMOYm8XeJNyIKlA+Ibhp1n2PBTLI5Vob9J3xIkCGVBzDw6kVPDz3Mo4gojBXHnk1bdS4/GxLW0ETgaA0W4mESqIHPTLFw6PeiKjMHPV9wVq6xbw9tj3+K50/cQYHUg1P8gUumYW/HnwnaRWBPN+s8nUJ8egnWOeD6RSWQsXnsvXk3u/HtPF45SMchZ4fgcV7NyOBt9Gys7uqhT9wGwEokEvzo/0nNTULj1MizjIFhtgLOmcz2v3vMpH5+I4q1vnkY/VwyUO0namLe5r8B7X+CflEtfEMXndgdSeHQor+6L55+0Iu58YT049M2lSpmSEe4bkJs3U/RTHJvfyhXv4CxG5jt9wBrDfRz5chGOK3JF586uymbXrH1M+9OREQ3/0pGyDOjr9JzqdA7nMUqar7tx7pVnKFoq9oPa3xXDroJuXNoScNRHEOMq3s9vLYpF9c0oelUyuoLFfqrdMic+iTuPi38Jb9TCJJUYzG7XxBJfCXVRF2i3wC/yNxg4BFossF8Lox3Fc7BEIqfDAlPlMrJUFhSWHm4WwbN0B7h0xcip2GEUGJqZ2+93vtQ4AVXvaVZWx9IdJM57TBIHdhnj8Iuo4aXH/mWzRJxfFGrTMD29DFVmDg6P/4nUIs73yiWxFP80i6YACRGpYxFreoBVpqG5wo+/p+1gVWQZB1zfEcUNVgXFO4cxFlj0/ve3rC/UxgYi/pnFsOxEND4tzPMQr4e9jm78fs+nyF26+NC/mUaTeN3AakFqNNBZ6cPhGiOBWWIQaMXpIchqdKyalk+RppPLgEkRAK4BSKQQcyWYGdv7QKfeh/JE85EgwMnm50EJ4WPVrPTVkmuy7cUsUhXN5xIwyo0EvPcRH7sBuk9AFYCD/mF+SDjJY9067v2hg2bzf/ZulB9L5LI+hOR7bN/b4hzPM02Q2ylnEiNIjaxHasnB3r/uV+UY3LCiBj4p2UiSs2093tL0AFdL2kl21DF36DeM8N8D3NjztF/ikOM53hxr4d3UPACeM4lBtwVOgZzubKZb58Vkd3CzVgJDEQR40qObEhP8G9jCIIdCXlN/cmu1k2DgpXsP8Oe046zxbqfLKq6xGQU3rrYvQR3+Ex0uHXj32/8ZLSreLPHFpUNCfVoOY1zFPnArQtcTekcHG7+5jc/VA+V1JfR5nM/RuJHhUS2ag8dbr+NRFkPQ7oksrwpHnmYjGO6t/x3vvGyCb0jQho0ow36PFjV4KuuEy2Qd7SEi9ycqH5oFdmv8szVJtDde5/5NMzg3uEAs4SyRkmoYhkttA/JvZpK4QAyS90rdOKUDU2gZdY49JAkDJfhnuV7n6nczWXktiKCVAyUTn3p4Gfu/m4vb+y9w/RvbnDDFvIXuUPhOrsEk7Sar3kYRlJh7eDRAy5aABg6cm8KxZgPvzLatK6kuMVz87g5i2mJoViiRyfspImmUVCVOxVq4g00/T0c9KlQU72p0Zefnd1KecYZTw08PIJ9Xuoznue+OMHnzXDIyr6CZVYV9p3yXcgQHz7chK9WytRuWy10B0KrTeXndCBKPZXHy/p/octaJ6up16gQ6P3iYeJWWWq8mTGo51TXfsnFyH2Ojtswfl20H8exuw2p1ZsvdAuPcM6Cngp6g2/mlOp+CCzFUVPrj4n8DpAzqkyHXmnTIPDrpnJLL59P2Ey4ZSIL6vzT+N+Q7b47/gn3/Hf9rQyqVkpQkThoTvdvojajAsTSUv6/6kmYHegkWM/fnDsbr6DAq21xxDxLLOnmZKskNhnPzNiMZMoxO02nsZbRcrE0YPniMXzx6yA/+jS33ib+PIAg81WLF+95fmdv0FrdrZoviIYazPHZ+OC3A52Mv4G+585aDV6yDgV6JhG97a1hTCTscfLjJN9FbzWzTxnK6rRXPNYs5PV9cgNne8jtbk1fxzN5kZLlxNLfGYa/X/GxjJNctZxlU8i/+PdPpP1p1H9Jh+oM7J13kbNhAGZZeQycP/PgAv8Zc45uxDfRzn6FaOZcjJhCanGjTjOemEKKX9gSvJ/zG2pNhBGj0+LuINy5/CLG8ofmXsCGOeIbX4CeIGTa55qHkK//ilYnPMMjhebDX+Zc7883+ZHy1Gn74+lG2d2cANsZ1qyaJF76djk9+Irtef5PBEtcB1/Vi/R2EeTcwUy7H2Xmg/12R7ysI3mbCml7BwdW2af/S9RiHn32FDucOToyuZKZ6oeg45w5nNE0aTPKGPldeuyG16Kh+6A8sCOz2hHWd4u6Pkwznq/UVjDg3mDOphSS9IC4opGmiOblJitzBwKcPDew+cjTUMqHDi/J6FbtTmujp3o29IXOAaQN3P76GVRvG8kJrEE76a0AG6tZs/nr2V77JeZM/XP9AYhVvjuvl0dQ5XEJZHMGKa4/SMk+c+PXmfcDwC61cCL5EcXgZAb2LsH8zp41W0DhsO2/3FqGwDGRSPXR4LNP290HYkW67oF9v349qOefv+oOL+j+QWzxYbeccKagCWLYzBdW6uVwcVEDIHXbPgaSe3+9ZRXiTN4uvpuDsJ76ui14qrC9/y/hDQ3l45wLUo2vBDkq/be9YIi4m8PELnwxg2VpkGs60OLKoKAqP40NxHZYnitcpknnKZyVLTEOZsWMGwuP9AGWpjCfljTaJwX4dSE6mWhS+zbQo9VzJvEia3PZOvxN9mc64HSRUrGbvtQeIibF1AZmlzlRNOkxxkxMpmxW0JnUCX9k+FzNufs20O2h5c2oe5wxiJuwcySXKS0YTcTWWNtc22u9pAWzAWpDyIN0qHcYuKR16sQRo3wcIXMy4iGeHC0kFGlZOmyNiMeqrXZnw872UZlpJdP0eCXrAgcTGLcxbtZXpF6MZ9e1tDFGdYJedKVmvq5lzmecIvh5Mcn4yxofE91Omn0iHvobDt+1AJdNzjz00K/TBZjKrBMy9OEjFc7/VaqVTkk0A0H3dn5IuN/pDLVYF1PvVs1cGUdp47LcIEpOWeAWkXkoi+nICl++w+XdorY5UmcBUMJoDR3yIS75PdN75+R9hOJ/PT3ETWKV2Z918u/PqmzhnDGJnh5oXzW+SJPHEXqTZXRNLmf9J5M5FPNEIvl5iQFkiSNg1fRd+8ih6Ct5iyZ0DC6ezqyIw58dx574SfhkpXk91Mh9+oYuW914iIKIHyQs2ADW55itMVgPfPvYCJpmJt/nQdrsVLmwteo42n2+5e/rnlHWLCzSBkhgij2k5n3GeI2OPMMRB3Or7TlECmmvdvLJzBkKoEw6fiN/bXC8FB5b+TJtbG8GOYrltAHqtzN80Hx0Q/fyWAeGI2DKsiT3UHkpAGj/rVpfydL970T2nocGrEv/bXiREkmDnfgQZvT2kNkRiPF2P6cpHyB+bjL3PZmTX3zRcaSX5oX+4J0MM5pkMrcgOl6EpCeKq0omSRkEkL+psbuQxFzighWIj/S1mAXi4KpwxqxZg8a/H+V0Lk/rFc7aNpOV0Mg1BgSzZePctUafzPcsprpSRUfsPPj0FqHsXir53+oV0UgqSGHVsFJcTxJ02giDw7ePfklwwgnkbJ6Ae0k/G0+rDscWrcNEr0Xs38YzsD1HcuftOPh91kcc2ZtHo1csVsXIkLR6TifjpG1x1jnTFlfBjpHg+8jHvYtj0k+QXhLMuUA+GZpKSkijO+Y23CwdRmZ3CO6++gxzxs18pSWKt1/ec0ADmgdJhs2srmb3uPnInH6Q7uRCpRVygHD8knmmDDXTsSUbnMVB+7pPL2RR67KcnrZSr5hMsQlzAudr7Bj8/MYiY7vWke5xC1CVhMfHI7cf4feoxljTAiO4HOH4jFGoqQT+qiGtNzlTXe1PbKwa+VnQnUuS5nYW/3sv1Bl/CJouLbfkRjdR1KfnxzTsxJF5nqJ28ncZYzaLts8k9mgbU0JWcjT3YN8lZieZcBDlH0vlWqSdqu7gwChBV3PdUJftfQN3riT3Y54GBVa33UrXXjCG4BtX0RrgB9i0+fgdUdZIe+isPpX+LY+fHcEOwu7k9n9sNz3Ku2oc5Mc4USMS/xXVpEi6lML7En0G5GWhvP8kIO7BvvtMFVnuZmOh4hHLTgKUaf9lxYvIriWEsQVLxOg3Q6zKRVW++gYIu5g++hmARd7ufNw1G09IDLa7E9Bzg5supNTawc/EqhDZXkguqKXZW8RY2D+cNlq8pnXaWUYUGAvcfRDNjDNh17O41ruBs/DbGTingOTc4ZxEDY39UqvFpVLE/6VUSpJlieXOJFG3SGBq9fmCxWyG5zuLnD8CklxNc0QdoPeoklqsSBDiW/jwRrsFkD80mQ27rkw8zCBzNDUZQOeNVKGNuaIqoI+tU7BZOe7bS2Wvks0IVORN9bsUcBRO/L77E/kY5m07vJCFWLDUiyByp6YqhzsdK0ZjDJDjNEcUtEjXXjDDt1BD0DvoBvyVAxulYBImVjolHUVrEBJUOp1G8cfoIj1QH4DC4gaib3o1yZ86GltFu1eHdcj8ITsjtPN19JHqOB8FHDW4six1JmXEocOpWvFbwQ91Rh6QkjIOnY5k6WfxeXgzScfSBH3nwpwe5pFOS1A+s40Iwg7YPBazEuO2iP9gHAjKTDKVOgc4yUDFhR08OzYUe3HlqDqZRY279vVg5kcKvehnX6EOMfw0l48QFX48SJW/89AbXhxUyKGkOzc2LsN8f9vp0s2/uUbZagV4Qlf7lzjy4I5nRR0cjbfVAcruYNClHT8eekRTkhbL0zrW8kSyWWwP4uT2M0J3DSHF1IfTu1AHxNvcONktbOXA9ic/C3fr6YaxWwowyAmsC8Goso8PVnf4009skDrRo1bw06BFc5Ubs51eJ1Ujk9RCM6h4+ef9OtneMAw7eih/XunPykZeJG5zLg0v3AFAnsXVfn+2IpPDfvkL/l4mneKCfL+pgtRohWEJVt5pXLb54SsXzxZGWv9g29jW0yj5P65t+aiHdJ/lgrJWvfJRkh7/Nb3ILRRJbl5sjPcTKwdPii7dGR4RGTGgLkl6iObCRpmpv3lr+OfO2i9/5rVe/QrdvJwhWQhPK6Q5/EI3yBmnY0MYiz5PsipFwKPAQVkH8/CJIsaTOo3dcGdpcFy61p9mJZULL9e1MdFuFruE59it9cTHY3p1GBzfkd/zNmdwAXPPjUbqJPT7LWwIY9FUaHYPbiBjW00eWsPtF9+s9SfWpR51Zi1dsI9iRJiWCwNW4q2Sey+SlD1+i9P5+T4Jcg5ATz5DrwRwcf5D/lDh90jyUhjVxbJq8B9MosSqHxNzLtOJ4Bq+7ncjkEhxH2OoMPjVaKCjDqWIsHc4NAyTG2xOGUxDlzLh/uvHKEEvMWSUW/pj/Bg4CVJ0tZMJ4MVwnSAXq/OpI9O3gRW/Yx0AZ8vScdMiBLbV5LO4nDqAQYNqBqXjX+PDhS18gSMXeGbtNg9Gl5pCWm8Ysi83jvk2Txl89e3D2bqDsyZ+5ImjsMkHQKJ1p8JeQ0qFg8OkhFCeIQcyQhjdI2ysjjcnoHHQIWeJ7knD5TZSnDrPqwDjyxhaCHKKa74GzaqTSbxkdVI85vJJJQ6+QFyCIQXq7n/avDjkH9PBDsm1PIpPKmLx3Mh0uHRxNL0aBmsdkfWBLu86ddW89QEB6DsPdtuLbMxT7nsXw62PJPDQMtEkYHBSEPGy7aosmjGtrp3Pb+b5cyHPyaSRCv+dEkPD1k19jFazolStJtuuM7RW07B29msNDFLTJQqloH2qzg5WpybM4U2VqJ7D5Prw6J+KZJnbbjnFdx8Qf/uaNJa8Q06nBbUXVrfvx4MZRhF6IxmXmBTr9EBEfBamaIW1aOix6fq/JxFMjJn4JAmhd/ekO7+BdhZpK3XX6j4jSEKbvHMflpWsJFNpFsQptLPrYNfR+8i4XBDeG9Tt2vnYbnp89z1Hg5+n384Od1M79lnz2DnElb/dEFBe16JNt+flw6xbO+lsol+q4d/4JPPs1GUx0SUBh9uRaylc4xjfSozdiD/aVlQUx+5e+XdviSWIwz0ViZq/nFTbkTeaypoPq3mDsaX6C1cKMndMwyEzkDfFnkrK/8zmcbjpBbd0eQj1ySHYcOJ980CpFFlZKlUWHj9SWld1xdjDKg8FED7mAOawbpcRu3yDX8HfPZzRlBuK/fhOt/gnIvrRBKDpVJgXFQ2jzNFD+6iNIjJmArcan07/PvWPX83FQCy1djiwSxBTX33uyiFZ1ca+3nnVhA/MHX+cIBh/p62rekXWKxfLXRfEys4L1qXL0mZ2kVR7ETdlHTjRqYjmIjnRNF4u2zqEltRCBPnssq7XveTw07hCqCBVPv/AUPhofCs7vp9oEH7fB8cuzGSRvoXR0Ledid7BQuQKiH+v7V7UX/3pfruck8eWT37A0orlvTyzrI/aUWNSk1pchUE543fMkB4vV3/4vjf+Ed/xPjv+Cff8d/2vDarXS1dWFk5PTrULO/VmFhI28iN8NPOwjey8J9Iwpjaa004lPnv+EKOU8UUFXwMrxd+9H49XGfU9u4gd9FvZgX4MsjkuOV/HLjyS86V8EQczUEhDwq/XDpcOFSu3TKJRiZpEgSElavJMvm2SU+lShM9mSu8fvf5XytiQKf55Im1srwqO2xM1gMfNIwxXcpe54zNqKLFAszWjVlZNsaKQtpYCh6VeoNTwsigcWeTF359M0hNRh8W4H3ERxRVwEe+9y4FLIZWTdM+w96wHws1YgqwmEmkDWRh2j/ygwbaLR4TLHciKodrjrFtjnYCiheI0r/m0ynvrme/Z2ixdVX4eZ5F3zJmzQt3zoXcgaqbi1b4rhczzyHyJsyBiSvcXgkiBAam4qfvV+rJp1igaTC/ZCniYHN06FlzA3P5H7frsPIWPggj1Y9gXlzm9Tc9YHl3sGakG/XHGRqgNDGfl+Ei1tdwAfAdAoKHEZlsNpWSZKp88Y1g/veO2f4Qw5k8WdS9Yh8WsRxZTmFnZG9/LRoTiWfzqdmoUt/cRFYWj2UJzb3eg8k4peKt6QlTku56iygOEj/+Rq83z6L33N7tP4dWUHql4VJ2cVM+J1cWdKnfw2Tl4IwLm6nkUODnhKbnw/pTfn6mdTbwhiqCMkyc1g0oKsr9jXo/TjwweeYnZJPBMOjcNDLt4AGHV7UFuSaYooJz+6iP704TvcC5GYdAyqfhKlaiCgrHW1cGbweVZPy8Psn0l/sK/cJHCpzZEXWoOJD24XxQSlNycbNUwEOs8l8e4o8y1xg/Ec55tIuEvTyJvD9vNHp7iQvF4Xx1nlOZZ3aWhq9sBbEDNnno6p41ibK0+dGUVbcL9nEIGlvyxFpVdydfkXqD36S8H23QPJ1KNMljlgMkRgX+gTECgPswH3813E75Z3by4fr1jHpJo+ACBFYePh5ulUtMogfZ+VA21ZhC60bfqtchded69EoVbwdVg1CSFlovNel6exfPpbjHeEV1VS9JJAUbzSR4P312+Tu2ECzjpHZFZx4TUq7Dxzv/uaB7aVYXEQg9EAmcbDVITC9489TK13A6ZZ7qK4SqmnKbidzCADL85IoK0nH0iiWxnELrORNrUThqXpHDRFijaCGWFleEcd5sKW0cQWx+GB2Ato26mXMBw+yYOL83k+s5o8QxO3gG5BwoPrRxBU5c+OR78mwDpWBNZZdM3E3fkIJ2cLHOwRmB/Qv1IHSdJ2krJHUnEthP13SUXHqzry+OVKFjsaffF59jcGKWzvbZGwlGMfRqEy+3B16EIywn1E503nO4rr3VD2tNIYNhh7sFkQZJzZNgprUQinXxlPq3UWsPVWPMIllt7gj7mino/sSCqT5orn9ijLBWrD4MPvMjjfmcHS9wdcFncv3cVvz8QRWhE6ABAJ9JnGFzvLSJRuxcOajFxiS/k+1v/Jtbwozi7qZaKHAYlZXGT0Lj+DU7Unt8ufJ9E3VtRtsNCrGM1bf1Hb5cTjLuBkbRcd61HvxMSdY5DITYwYshtBEMuemZVWzEG1fO3gSZd0oGyTyqGHlqHn6e1U87ExnLn94qP9K1CFyDhZaCSn93fgMQDCOpYR0CXg3KhG0XkPlWPFmyZt9bes+2Iv5phOIkcWcE0XyCC7Usnwkd/QGythrVMFuToLb9of7ODOrsQcdOFX2DX0FKmSu0Tn9jXX8lx9BF6f3YHWswXVpwO77Oua+p6r63IdCgYWVr8NqWLh6WQ6nVpp0zdxk6Ge17qYvLxWEtp+RtsmR2+oxB7se3b+SU4NuoL3jvGEq8Xzs0ZfSWEI/OR4iXrZ96Rl9CNrCFAWYZtjrBIxMOuIkYJVdzDuUiQX0wd2o0skMkbvn0BoZSifP/P5rTXn5jhlCCB7cCulI/dzrtqJ9Upfujo7QSIhbMIZDkZeZWTr8zg5iD+3Q/Dl324YXR7JGydHo5tZJYqbBVA4GBmthCXeEnb0ewYvG6RItQrSN88ifIxY6gqgui2JYvVOCib9SZbyvgHx7h4PpDc8Ui6oDWLRVZkaxTUlJvrmVsGugKOmh9vu38Xd9bC1BwaL+snBiAydFVoDklE4ZBLmLPbg2rR3MF37sqDVlYu+XaJYjyKIk9fdUQkWpi7eS56XOEc1aZKpcBqBSd7BycGn8ZONov94NaGGwrp8PvDu4A3VLOzL0Q7mdlT7BJRFkYQbHVEbyrmZQ0sr8xh2NIXWWSfprNWj01ZxM3VqMmo5dDYGS5033UcTCR6eD6/ZzmuVyIk7lYXvxXR6mz2p7xI/I0aDgoqqQN526iTZvxMd4vdWI3uMBU9bKM6NIjlzYK53pjWfS8FHUfnLGaEppV4hzrt0Tk6sffhT/Or8WNMeTemNeTKq9wBtEXD7UVcSDoRybvop0XHJbpdYPOckZ326GFQdSJvEhH0OMKQpl9F/jGP/GAl/Z+bxRT9vsd9HnMUwei+jL92Npf9222JC7/w7Sk0L3v6tBKjFHTEA7o6dnLtrH5E+7ex3DRL5UmM1s3J4CdmZVxkjA0957q3Q1aDPOD8d5PXLWP7BcsrfEitnDA+qpyaoguHLVtLmE4nHy7b7pZApMOoDkVcNJaTgAFJVLNj5JxrNWtwNp6lzbuBoylHiNWKjQrlgZOSVBBzjSgkYfxphQN82LHbupEcaxHsBAmU9p0T3dFTAJO77NRsd8GXbm6x66YZXlUzNx3EXIe4iK98fjNrPCOaXgb5nycsjiY8/eZ/OUj3vp56lbWg/SwNB4PPnPmf2ltnc8fcddPYzljY7mqgPrGHT3E10a7pZ1G8t1bobKUrL4f7Zp7nmMlCC2dXawKtSJxyX/Mth5ypgnSjuXKtkxJY5OHc5M3bGLm56PgmCQPbQbKJKokisC8BXIS7ojvUuwmlQI+ZyLxpORWGNP4A92Pd7UB7hcoGoAg0GhUHsvygIxF6NxaPVg5qYsQT7iAkV0YnLeO18NH4V5Yw/OJ7/sCRxuTGICL0Dg4MKSBTexV5ZA2CvSxz1JyV83yqledgNqUOrmV1Ts1k/IhtW3cHUfeMJHSl+b+85MYoL+7PIj6zEN+MKmXa320Ffy6dffc037X0Fx3Jdp2h/eNFopcaliQZpLw8Ca6/dwSQ7ztCe5jTax36L1CylPLx8AOAspZeR0w/zy5F7Cak4gIKxgG0jGGPeyco6FeVn7kR7206klh7AFUdzI4P98/EpWUGtYEYwykTMhCsOU/jiwxLcOgLJHrQCc5AKeypTozWRi/esxP3be+mVu9JuNmDvpWkITSF/bATRo8YyNDoPc72dQoWukaTOVVwumYXz1FKsTuK8HomUX/Wj+HjEQiyj9IwwiDvF/HqLubsyhlfGHGdT+jlesi67FcuxxPN8PUQF1TAzvoZao3hfYFJb2TtpL09HK1g46EuKBfH+rttDz29LfuNKCJgth7EH+6QWHfc5g4OHFnVgA0EqsQSozGohviKMxHODGNuYTsb8UPqPkBIFzV1OPJ07hKZJ4udIrS1n/O4pdAE5kvmkOdpAuUHalZwuTsCTBJx9fNGYqsHOpXbyUDeaaOcPYyZSx9AbWeSN24mVDAfo6JXT1abBbBSTHiIdi/B+YQ3Zx5K5+68s7lwhVgia7xZI04o/2bnJnyp/sYUEQAyNLJt/jC2bF/Pk6V0EPSHO/QUETg07RU3UNXbWjeSmi6Igc+SvGTuQmtUkX/qTLqn4fk5zdubhmCY+WDeHffvDKXqiR9QBatTsIHFMENk6K5Oi66mQhmAvzW5NjCFfXk5MdhH0OIIrePf8AddAJl3FsqwCLqXlMLIGwute5Pd+Mp7jve7FsNODzB3Pkj0kG2Ge7fkerFET+eQmtpSMYs2JVD4MMfDYjQ78AiGZwklr6NZ0c9bqj0f4AdF1xWpKuBQQijbGi7tG7eSaZDY3pTYlgoQGnwaKo4rJHprNVy69nJCJ9613646h9DBzolPGZGcJHl3XgD4So7u8kSEOEP/rItyavFi9vMVWR9OEETgrH9eUwzxQWc7whX/g4WRTwAK4InuDVdseRmU4xtlaV1IUfaCjIEBvVxTyHldmbB2Be/dYfP+yycHKZUo8z5wiqqmFY6cuoB06jjF2CL2LUMxv89P5a08CpWsW4vncGdE1KSS9vJBZQkFuHJn/zmFE4mZR/NL13zmwfSHtDmuoHmodQIC1yG3PtLZXLZonP5QOY6viGBGDlbhZUnGWGuFGx3gmZ4h8pI74yr5VLqp5hsg5L6LzPWL9TvCxhy8vpZ7mV1M89kOwCLS6teIx7Sg9gwtFn2uVKPnjl0fpvCLj2zdWkmgR+yQnGU/geD2SLrmFX+/dQ518Boi00qCz2Rl5TRMXoxbQ2jpGRFYGOHckjeEdbmx9ejer9bb9fFOzG6MuJRJSEYakW03jU7YGB0Gq4ELdVLBa6U3So1e50X9cGvckje5/ciYAtnSK13FL5zFM6xIIDS7hbMqFAXv124N+ZfOgKkKalnDoWBKLxFbexDv7smXoRP4IvoNSr3rukIg/v6z7EvkRz/K6KohnPf7ArD8BRCMVJJwfdJ6CpAKWf7CcycFtSCQ2sG+I0Ejqw1VUe01h6vnpEL+Mg7JMvu+Ab71hXNpqvowFf/1YtkU2YJBsAMPdULURicKPaWmHkCSfwCwz8mk7PLe0Bn+nvvVDEARGnBjBhIMTaAjLIvj2IfxfHf8J7/ifHP8F+/47/teGxWKhrKyMpKSkW+2tyxs9MUur/+P/FyRK9i78l+t6Cd1SLf18iWlXRnLBdJlOvZllZfBkbKgobpY48v3itTy37gG8XTwHbAAkgoTnc4bRfS6JkvQi7BnTADUOWTwkqSPWT8ITagFnS/2t2AclD1CvhdDrJpz0DgjYWEMSicDtG27Hrc2N1I8/pUIv3niP6S3A9/MnKRqezeKJe/lAJmYDznFOpcfBglteLcFpywGxv4Cv7BTTuotp/vRhyqb00/EHrDIlUQs2c3XHKDRGnwHx4aVfELZxPFDC38mr+ewGRz1HOYHd3o30+l/GsRRSGIZ9r2NmaSp+35Zzfvo8Ro7cxJwY8f0e1voYsl0BlO96iiXDT7PfDsOUmHoYu2Q756/58demETimiQteEiA/NY8JNaHEl0TiZx5owusqK8PZrKfFvZMvO7NEnjyCAG6htVQB5X+Mhcdt3Q4fdAznwqSfWG6W8Ha0jDLtTrDrt1F4z+dyai0Jvy4iaqSYdeTjlsJbxx+nuLYGhXsLrXIxY3S2x0hOTLewWfkTJe7lfNavd0MQoM0vnsPtj6E4qxH55gEIEhknRpxg0v5JbKx7g0zvaFF8Z+cl9jqtx2NaPF2Na3jZ50YRxjWJz/b/jMVwjdkXFjAhqRSMXbcKrzcLAIdiCmlOKsRPukQkOflPsC+5T3yNSoAMKUis4md0/YXV1F1oZE7q13gGD9QTv9/zFXKH/8MkUxGZLZH82S+uaoNFfy/EsTKU8MW7+90TgZMjTpKTloNSp8TPyVbSajZ78t3aSXS615GcWkCgLFbUTSOxSFD1qPhk7gaMCiMb5WKo+9SgPArDLxP65gtEjhHLmTj3FrFgTAH7tfCXQzszBPHmwNNUxVt5Q7EWxODxzHoa+zFOBUHgTlcLhz9cyTY/yDcmi+JtmmHcftGVMks7EXJwwMY0DLvmy6DKADC3YZJp6DC0cRPEv/lbGRwMnHvwd3RGF5Hs6U1pyJ6jc1l5aCRBz9j3LoFB+QJfnhjHowte4ElXOGAVAy2vt0o42SNwako4erMz0CGKmwQH9n47HwetCm2YM75KV1H86eiDZDzbwLI997D2+CtMH9sHSAwb/ik/HX6RiHMymqI8qWuEKLuaa4aynoVOsGCwgtvm/02tq1hG1trbjqa1koI6V96vVTAlU8xy9KvzJaYohn8sAoZ+qYvE1MkbHvC73MoQlRX+wxqyVFvD4b19AE25Rdyl7OyRyvlrUxA6LAwNb0AhsT3/g53nkFtdgd5ZjXfqYSTyJOwLBlPm/Uv8qAi2vDQOb9PfmCxJ3NwUWR08+DjjBEmOrSxb/RhVE8SSfkrtRTbdPpR3v1mAcHYOumlioCbUeQhHf70dt+xYtEE76e8rBnD/tRUIL71PnJMBB2MDN72XABRSOfGn9+Emb+SDV8aBRQ83igPXShOpv+pEcqcH09066bDans/ShiMktG+gq8YH/8p4nGaK/edMUncMzh7EnnHFbcd99L4sJg9kxJfR+dglunX3c1T6nB3lpm886ZrPqMoI1r+5iLi7TPQfIU4qHn1iB2+1QFNdf7c1+KEzgevDd7B67n7CDHHcBPt6HNIY/PIOvjmaRNRxP7oN4g2XPDKG8mQHjme+SVnwVl6Ti71iH9FK6FQXIxTcjZN5JG8+ZYtJpA5sSrmIAATKwAlxt1h0+FP8+IgvUmMT/gYlroIYnAKIV0/l1KiDlAZXMlFIHhAvi6zi+7dXMtIRippkTLohVXu15mcSjjeyd9IRsocdYUXXT3Y8WfAKbsDRq57UYYUo+vVQSASBmoIIPPUCy2clcEr7KPDtrfgc+V5WR8C4UgXtFpBZxV0Ov+pHI5VUkuCgxzROXGwAUJg66M7Ih8pQpmRPR3an+BnO03dwJVuGc88DKEJ/xCI1U1ZWgMVlCGOktSgCu1m9r4PoBHERTxAEnv3sWVw6Xehw7QR9uyi+1c+XU/d/x1M/388bG6aQvNlVFF+zZxjx50KRL9hOU3AP4pIBVLW/zPRNrnz9cAzXwgZe1yXNIX55+gcaHFsI14wTQ3aCgHOHMyOOTGKoXIrnXbm3QlcVGSy0s8bq75sX2wizf3sRyfByJk0/i8wUh733xpZWF9ItVtY+8TVGF/GG3iRz5eN7f0UlwDYFxCueENFqLKpI9ltf5NizqXSq8lgsLKP/GBEtpT14Mwcbrbwk6df55xRNget86sZ/wYlxR4hX2aQf1RlF5IRdwm/3eNYfnErsVI9b2IBV7sT7i9YgM8mYtfcJdK7JovwiSOnBuJPjaHNp4u3X3+Ye349EH3uoKo2WT/rWfcnkU0QNFz/Df3XH0RDwL6GhV+jUhQ4A/wWrjux52+i0gN4kgERM+nEQDAiB9Vz2r8fbzpe6Qx7CNxeD6G30IOuZtcwLEOc2nkIPYx3h3Zh8Pg3NZ5FsKvYQf7yqiW7nAFSVQRSpOxHm9pPqRMC1zZXM06vR6Z0BuwK8oY33/J/ih2o1z+48x6NLwug/5inPM3HaFTxKIVo3kpWic1tZ7GKm7Y/xmHLjGPrmxluxLms9napmBL8Gmge1460WE4I+8Kmm1AibU+QEeTYjsejhpmSlTM1bP6/BvfoUPk1nCUn9B/jl1rHd7bsYXryfuAtRrBrjSIFM7GUVYMnlwcvp1JQG8NxjO2mUDCxspI7XUeWUzbuFXkzPEEtLOnSdRuGhp1trYe+wexGEvnfTvkDiOfYSw4JK6DU8hMsNsA+5Gw2VPjh1Xkd/bDDpmRtF533YzZXXdKH8M7qc2MGllCjEyi1rfQvwt8oJMV/CIrUMkMxeNriYqZNO4VYKHh1axJRMUGCg/I9ZxA++zLwlB+g/7uiooborlNglx9EF29bT4Ro39NGz2BzUzp9DvuCoVrxHUwR2Meyxg/x6MgSnWj9iVeKn/8i1CA5tTuL5klCE0WLFA8x6Qhft4mrPfp7220i1MAZ7lQgXRydqEu/B7PgwI06OQPXkQILKnc5/Uh3yCx7x2XRZB67VV9cPo+GCmlGqXtytN/bEgoTPW904o29DGn+FIQGNaCVir7UtftexDrcSfHIIWx26ucvudrs4h7GqKo0j1R1IDmXQOEic70mVEn5+4GdirsaQ8u9gNA5PMNXulZcgUBlayRJneMEBurpM2Je79OpBPHegkKCEbew3xzEC8T4rTlLM3t/n4QM84boPuakJcKU3+GnuXvEwVuNV7jg/j7qI8lvyitBH+vBo8cC13YmK3kB0/XiRPQRQeyiLkHY3tjzwOqmdiYxn/K34ZeXz1GReo7loFUSPQKqtBIsZJFIsqmC+KpqPpkTFXdPUdKgHdvlsOf8u0y+OJGZoAd1O4hy41G00h3c6EFsWCunnRB1GUoPA/H/no44p57V5F/m0VZyn3ul5jlGLzrNsbyIH1y7krtgaorHZObzsWYSvyYEfX72P1rjr/Gq3DZOZu7jzwCSuFoby1Ls/8GWb2HZgsosTox88yM+14Rg7ZKj6SRkCXIr/gOvKe4lOqUDvKCZlKpzC6Zn/I1f1PnS3vYG9YnxVigm3wBNcUZ1jpKsJvWSK6NjIeicUlRG4sZogubhTV4KJn8pT2bNtOPr6XzBFiaUI/c3XudcVSs5k4NDkgfU1MRjoIXSQFXecMcthbbtYwhbAqvSjVDMOXYWFwZofkXVYwNvGQIiTVhLkpeewfhgh9T/c+rtSsBCLO+a2UNLPh6H3EdtqGPQqPln6LMawTn5f+jOJ3mKZw8+FwZwY8wO3acDVDSxWsVTDY5kzkQ+qZMykRlSmBALPQbnnasJccxFyBW4vi6RRVkysbh6eLbcNkPF0U1+hUd9HMnKShOIstxHfNaoA8tXJtGmjmLMznF3zj9iOlUvIHpZNSm4KP73+EHc8vw77qtB4aSDld/3Ge/pHcHVuRSKx1brkLWf4af553mg5T3kPSM3qAfW9IcZicrfOwP1kOg+89QNHlEXcBPtGuG7m2V5vPurRUBpRigQxWSjIJQilrm/+aKjyxbmft+mf9VtpLM1mKOn0qsYS4tzXQyeRgI/Hu9SPl2CueZMh3Y2k6G2KIVJBIDW/BtfyyzQFpaH0E6/TFsGBMxUqqi7Gkj/4DFL3cB63i0skZuZGNlA35BJNhwaxzxIu6t4TBIi8XIO6I47OqOfpP675vsXHL92Dv2MNwR3l2HuQO/hPwVimwW3ON/wY8jKrr62BG1n0DzzL6uo+VNIIGK3i+fnctZGUfBjJlcn5+DnDosBUUXzDqN20jNSSVQXyKg/sKfWCXM0Fzd30ZP1DkAyczeL9SIfEG88336N520wO/TaS2OUDYQxHJz0VSRFUefXgZK3BnqwEEF0Uhdu1SH67Yz+tClvdpyW9ko8j9zEtN5OEijCazLYasCCAqr0Wl4YrzJ/3GSkRuUA1NzuRK9rPoEh7Cp22imf/GkZbiFzkTXqoJ5XoHDmBqnYaw2Cn9TLY6c6EybcyJzKHlk9Gsiv+yoAmAofm7Xx+131cq4XSHkSkHoAo42mKQuBit5bqzgR8b5A2XToPkRMMzzXqOffuO3TJZQy1eznilAowXyfAPJojByTEe8ZgtQicu8ElORQIWdehvhOEHkdQAOVr4MJTqEPuxSK1YBVMfFQfyQw3E1J9C9wA++T6FrJCm7BIzfiUn8F6PQBwHfB7/V8Y/wnv+J8c/wX7/jv+vxpC8QI887v4POIS49JKxTGpnLUmHU4mBx7Qe2JUtYrirg4ebF1QzMiDnjz22krMv4snQzlmfKWweNlP+EulIFo2+8yzZ4zP4a9zScx1+BBV93ywK7Pf3Gze9vFzhPu34JpWcSv2b/VhXJtg1uu5fOxvoF77ELj2McykQNbwAio6HFnsDLu6xN+7xCEQy6RjlPhU3fgc8T1ZOPgULpm5rN00Akd/8bEAtL9E2nVv8hqSKbcM9KpqlgbxREI2x8df4aX/4Nkncdah9m3GEneND6KLbv29qqecrdPeIbxiJEs2HCF4inhTnuj2My6D6mksCKf30iNIvxSzRh08Zfil1HGm1Uh1kPh7CVYj90W00XotCK+DE8gNEF+Xd3s2lij4Y+lWPKVQJ3mL/mOBz3x+KJ8DeKCSywfE5/tfZcrgM3A2i+sG6y3HEuFGMVS3N5Qvf3qByRvEMlqT4zRU+AZT5PMzl3xaRYuuszqYrr9S8Td6sfqllWT26wgIsWQzedqzJLa5MUijJcckfoZP1y9DlfUNsW+8RHWcO/TjWslN7TD+FF+PPEVazkNYrGaw847SmtuZvDuTDlcrRelDb9VnDGYDvZ1LGXwuBR0J7GofRYbC9dZxqZYK9hSOYq26nrUhxYxxFAMpOqucRjO8s+4efK1ylMPFAFBjSQfOZy9y6OxwOqPT+VRM7uW20MfIyJXSdSCJpukDN2u3G/K4WjmOZp8G7nJuxx5qEborKAuM4Md6V9r8L+Ap2Daxl0nE80AI0a5tuF9MpmWcWOpkaWcZtR+/xIUsPTq/Z4mcLy6OLm9SY6SNCc98i0HiKy6MypxomnCMq41KXFpdkKjEL56buYYhBg3H6rxYeup9vhohLlYAvN4Ywa6DaUQOPsqYqMXY97bGBc7F5Xcv7rq+C2/XTlSf2Ayun+uQUZI9lBMvfMqSyFbKqrUMTuozUZeb2vhq60IaK31556kvcTSpsBfWc7Y283TOEExXfDEqnRDk4mU8QTaca5eC+VO9is0B5bwY6SqKt7f3EH9sNOv1LkiHXkLMuYPA8FUcLVuL2q+dDQ+spLU3DHtt+OzeUM5es+Cz3Yc/R3YxzcHGiK7/+A9CpR60+yxAYlIAtm6dv69msftAIi3BSTyXv48HxRZc9IyehWl8Fep3H6Nu/1hkC/ptuO49zMWeHczJT6HXR5wsCaogss57kFAUy0sxzbSmiYFbgC0EYs26gGdKMc8E9jMrd4rgQuxKjic8y4e1ZhaoF7P2RsjLepzYJ7Uc4Ay/JW5gXXMW9vJgGaUp1Fpy+PuLM4zRWNBZXucm2Ofm6EascTeWwFUEVUOwqkn0sXrBkb/X3o2HqZe4Z/7kklIMgJqcppN7vASNSxUzn18F4l4zAAKvL8U9/Cs+Ce2gzHgdsIFIcm0h6SPPIZjNnKpeyDC7zte10TPRR+iZ+KmJh8PK+egO2/0WsDL+6fVsP5OAy55JeHEe7PqQy4S7+XPnTCKLd9NRX81od/GPeWfAdcJDu/ApX4GmN5YVlonI7LoKm0v9Wf9lH5u43FksAQrg4zKRu/c8TaPfl/jJdQPifpdHkLI+EevDm/GIt20Gd+jceL+lHo+kZv65dz9HuieLjtP4WJHGuaF10d64TvE7X6pzpEen4oW/wmgOEa9nAqASwK/Lme0mf/K8i0Rxq8KTK8G30TP0HnzS95EgfEv/8V3MelTxV7lsNFEuOYnIdxJ4zE3LEA3c7gTbLbbzz4v4mqZZXlz3Lv2P3/vBk1tooYQrQS8gM6vsaAWgdQxj+6/TkVmkvCXbSLNHmagg0GT15OMHlzNZ54CbTytZ4/u9W8DmeZt5NwTusAzcpDh3neePqZfZI1Nyb3QNems93BIx7/uuU/f0FfjK7Pbc3mof0sr+RGaQUHWhiA6jv2j+HKuo5npcLbkWNed9PmVav66Tm/lYfngpXq2pDHcUd8Er9HJUWhUFPe/jUz9xwPf+bsgfrD16N4f/HYMsfCCZaIT1Hd4YXIvm2kAJUawWZhbHEJaTjh5IuddGJLk98gEaH/FC7n6MSenHOOor7jB61EtLdXATJhcHRoX8Tan1Q1HcEjuPv+N283FsCw16cbX4JgChtUKOHgIcxUWnI9V7OJT4BONdWlhkmU6U08DO7Y/PXMS47m2mDDmLdJL43dIjcDCmibywg2CE1+W2OeHDxBJ0GLmjR45SpyReGGP3vSQYHAwIEje6gz/Ez0/8mekOesa+9iNHO/0Y7wt5EjFAZHKGvZP2MnnfZE5cjCSqX4d+qdGb4z19156kmzPgmgJ6z1Lq5sTHn91GTuKlWwXymyPLcpKlzXH8Imun2af91t9bHVM4uWM4aSVR7P3sbdxwF8mXHy7NoGB1Bj2jj9KTko+1n5fgT5oIriz9gXlfP8qMilBYLv69TillFBRGUduswVfeD1CWqfhofzLWf6YxnDUY5z4P/YqI5yyRHLrQQmxRALIs8WcLEilpm9OYs2ckRrmRFR3T2HIjdrn9FTzcD1Pr0sbXd3ayxShW3Xhr63g8dmSQdG8+9475mw7zndxMJi1WM6rmY7g2NTP+00+4anUVHauXutM5/gSmI0PI//gekl7NEMWVynQMk3aS13OC6KsOnJw6EOz7oeAkvmveoiz+MroHxRLRcu0Vln/1Pj90wINa2zouGDvZXDKInCshvDnzX9RyC4fsShhmq4m80VNo0myi1X8ZGlmSyLNPIXXi6qkEcGwldGg2ZTIx2FeRcz+7v3Xi3oWbcRySN+Cd1/fex9Z6GWnZFnpCxfkzQJsslJ3Tfqa2w4PGohT6zzgdCZ0kvv01yysnMLH57ls0QA/9eZ5LfYuao38z+2AWyTF+om6wv3tSeLzxLARWQmAlkUox5f/c9TtIKelBkJmYl1IpAiklFiMfhnezoQtiVW1UdPVbS7VVjA45jltMD7OXfk2B+kv6jyl+jyJ5bB+ZLe3c3jnTzlG4b+RLF+IS/T0vPfoPRy2P9P1RkPB5RSCqGm/q/CpZF5fHOxIxqPBbaCnmsMsI4/ciWMX7Nz/PRE5s3YRMu4m4wi56oipE8Yzu4xSFwHffz0TZ5k/BhOHY189u8z3NCncpxW8+RkRmEfmjxSTYyu5i2k3v0u1Ww8FhB0my1gO2LumNlvnUJZ1FIRP4oOZJXnPom9D0ll6sHd+TfroDSOZOd7HmsKulDsOjazl43ZPkS3Xk10eCnQuopywUhfIFeuau4HKMnr3mq2AH9r3i+TsWhcAP/2SzofdBbn8k59Y8ZpUq+TzxU8Y1jKN90wgq/Ors+rH6xqt6E8XHRxDe5kfv3eJ7ppe5smncARb5J3PaFMgFWQE3JZpjLJUYLiWhvR7C47mDMM0SF8cNVjktZnA7ORinVje0ln5Ss70C3fvH4Fzrx+VQMVBolbuypU2OpsuByApIcYjnabu4oArgSP1SumaO5npnPEaHgbJ8zRovfkk/wy/A04jni8jAUZxtOIFMUYlKdRWsUdzcM7+mL6feKReA77pgk1y8ML15KJ2hp9JIWJiDi0aBfe4skcjYdCQZab0XWVOyyXO9DGy4FXfy+5onHr6NwvA/uDbhHBOlD4jO3ep0B9M+8ua1cQtJ8Bi4J7ZoYlh17nviRuxAGyDBqhLf8zTpNfLee4rRwT1YxhRys4stQdLFli5PthxagK5wHxnRBcC/t47Ta1Jp8ymlNOAsyogqNIqB8qIA/3b3/Xs9UEy03t4loJW2cCFQ4LSuii+BZrdHCBsE0s1QVR7A4jV9eo8NUwaIALFcGkLds5/yoXUu/0Rvpt1yD9wEhlWBvHf8N3TyfJIfeIjlkW1g6ACFC2HSCj70gH+Caxl7x35kPmLiwc26zS919WwuXsJdM2xzsFwixVWiwbnXmwSuE+BgRUAM8L/ovBTXk31g2u2dOmY42d6eq9oRHD9zHqdmT7Lu3spzvnZzv64RrnyKy5xx7Ox2wSIkcYd2vIh2adL2MDQ7nW5nE/qoZBwkN+81fDE5gzadL68fP0mzz18ExNmqEBJ0PDjzDU5eHIdp8C5UbqfArhO+hxCWl7ixuCQc3ThHMvzHiO6JwaJh3t9mmpwOUH3XGp5IF5MXM73WY0i+Rv1xFSNMrwGfi+Jp7iMJa3qR74Y/Tnjo39iDfctHLsdh9Xlatm3g28fX0WQHivVI3Ag6ncmC00Nwb3OjaIlY8eNXh2w6pl0mtCKUmasfxLRS3H16pseD2i6B4afnIZOL6ypdhk7+npJJRLsDfxaMpDS6QhT38FvKi5vCGZbXBnU6XGT9WvSBUYGjGTTYhVXVaqqFZOAbUfy2B7dTtWs4+z+/n7BXbc/BWZ8i9miszOYC/6SdY0ST3XczdvJM1Gz2nZjBhe6r9FprmYCFW/U53SXe7e7gz+JIZDvHUzxZXPvTu8h4+/W3eWbDnfz4xIt4/iAmhkWaf0VV0cELM7ooji4bUAfWqxP5NfcVBn1t5Z7Bl3EaeRX7bnQLUnqtsFrbQlPxDPYu7CMNCZjpKQ4hNCeYg6FuXHRwYNkNoNBqBWKf5beH3WiqqEJ73ZvGDDVK3ypm3OAndHeoCTuVwJBTGexqGU3HLz6M6Oz7vY2eI5hu2ExLswvdf8xnp3c7979qW0cV+no+GVfEsqBvKN+zlbjQgaSH/47/mfFfsO+/4/+rMaj+XtQ5W+jICeR0wwUS7IjJEkHApd2F4XlpBB4eQ8D9J0XHDnWP4S3jveT67aTKUEiYVJw0epuvsaU4iwulOl6OLWBPv89WSBWsKClB92Q4UX7ttFjFMl3OphpeXbMEWZeGo0Hn8VLZ5Guii0LJPBNPbloRq1UG5ipskg6CAA/NPMfP2ZE888TTXJnWzDQ7WmiJYyhrhv3CQ7tmMGP7DDS/tos+93j3J5wrKOHLQXcgsShvWqDfGgdKk/A+6cuUpdsZNeg/+B9ZISU3hWN+SohIZES/8GduPhQ+8g3b/SHMTjdebW1jlCM41rkTUngEt6FGsNuG9oam4TjjKLJfMwisrmCol7iw+qy8h/y5axiihBjEDHKVOpC0/SvpcVnNile+R+Yihhz0Cg9Wb8nickE438//h7f8B24A/u3sk0MV4jp51r2/+1FfkXH3tN3snrqbpW4rb/19hqaYSevnojjvQoV7G9vaSkiwkwG4J+4xStqjSQ84jcwiXpwEAZqDUrE4X6MrEv7sEneLtSom8PHhNXSoP8bXW4KlHyiRJjnPoONjOQ/4jfmO/mBfpKmck0GwrBkUY/3padkFwTYm5FjJITQzall73Zlyn4ep734NT89A6C5j/LAccrzraTf+hbOz760OHoAQ53R+29BNkJeOwYMPoEwWZxLenSam52bQppOj82zHRyq+7kMJH2HxrMWtJhCzdycgTip35b5I3aoSkoGrEwd2tFT7ORLw5ld83atF79BfrsSKqcEZzZl05g+uQulngwIlcoHPn/mciLJwpuyZik4rBktGeFu4NPoyus6xBEp/wYFHANu7F301Ar9qV46MOYLaLjEC0CkDWVAP95wczLP7x6F5RdxZ4hP4FB97FODlXMHEHVZ8V4o7bgE+uuJF0PlExuqjiU4cI74uAXxLTyIxOtDibEUvsYEHDaMKcRl6Gm9lF4ucYKfF1lEgxUyIbxttvQrGHhpLs694s+ZtrcRwKotWBz09dxmRhTQAtqKuR/NfRJ8pJ+L8XRwbeRzGixNt7y4zcUfHYAZOClaW97smqSClZMi9+KjL2Hq1gsEJYomMI/XJKI9rMA06S05iKYLwSV+g/iApcxo5cDGAwL8/Y/7MkxyQ7Lt1XIh+PK6XSpHoAzA6X8Gij6CPJtY3Frm0EqmA53zSUevD8NeIWY6f+zezN284pzdNxGtGP49OqQJJYRQheyfy9y5I/1bswwlw0tGTs1P/odjPD4//oOMvCDB6vwv+Vcuo/82WiHtb9zM25W2yf3yUtz96ne5lVSy2O87S1YhfhwOrHGCD2SjaTsksBu4J3cpZg5kzw7+jVz9aJBV0truRsxeVuEgdGJ9cQqFMvHMWBMgbM5sl4+Px0IgZjjdHYOE25NseZMryz/gsQywF164tYc5tv/DGn6N563wTlwTJLZjoTMgCDLJm/FOew6BKxslOttrk4E2Wth2vpAs0zjtJdsd9YFc63Ve7ivT9HVyNb6d74ueM7Vf7fLkxklZymL1lNvW+9fQfOwjAJymfqnAnRocM9FPTa6VEfu5GJCtJWbFxQHyQ5ywqZX/zZq4/PZ5Bt2ajVn09Ti3lOBnT2XD5FRz8xd35Dt0z+XnWKZZsScP/yHw0Ee2ieOKlAEZunY0ysIfRo3djP89JjC1szJ7MmT1D2QDo7y0QrcWCADonbySdO2k8YuWe5wcSay41zeXkCitxYy7gfp9iQHzcyTQubB7Lg3duIn2h7bd092zHfeYV/M4GMb4sDk2yuBCScUSCa2kPZ4Y/RIuvWAJRAPZM2UNMUQwZXy6l8g7x73HGMpj2iBPEX41HEuECDmKZ8XhZLW4agZU1cqxWjV25q28YNHGs6LLA4MsInfcRJROD9COkx4h48gLl3a48P/pL6OmT1HQ2N3Nn/nW6C80cnTURR2eNCPp0UcaSuHATOflPYmfxemskd3UwLH8IF71H0+u0kiAxTkLa1DNIpu9Amz3/P/qG7VMZaE8uRHFkCGc643ihX7xQF0ThCYEVFxJwnVQhDlqMfDCshH9kVt73vUZ299xb+aTCVI9XwyWoduVs/iyExddEh3a4PYFl+lX2HCzj6MrHeeAviUjOe7lPJTVdHkTnJeLdr0jooishpzeMf3dm0d7kRstSMbjk0nWWtdXu5H9wN2RcxXPGfyB3tTTh3OXMg1pvAjvXY6+32a1rpyZyKeFV/nRLTSLA451mLzqFWoyphQxxBLXZxsdWmDvYI/PgcJcjJo/TODh6YU8OMCgi2NX8HEb5P4ytCKfMXyypHeVWQ+S0bL4adpoevRcPCGI2kQQJb7z1BgCd48QgDUCv3J/vdgxCWhbCC+mlYOoGuxy8RhdM4ZdxOAHLPrMHNPqk0vKT8ylsB1ezB/ZOKIIgYJQbyMhPZkhJLI73iX+PbosD9WYIm3eQGY4CWtNi0XWfOJZC297htC3/mhIXP0S0Bpman12aucuxF0dHGRJyoF/5/pI2jJjPUpgExA/eKYoJEgF1uwtGmZFD05yZ6WmzJRjfWUnZ90tgyj4W+5wixDVexDK/qJAQGVzJUVkO+wsn8eF42a2syWTW4vjAcnK6Fazs7CbAmox9Gdug8OfFoELGBAUy/NRE/KXiPYdcPYzNxTtJOvQZ/golmmX9XkxA0VjD1ZhSzmSd4aF+DPX9Jk9eKJLxrZuc+zzbMdwICwi0lwYhuZhI29QtNAkWsAOFqxqO8dnM8fzcCV+2g79JXN5oUi3k+NoSagONvK9y4bkhY0TxGs0LNAYdI+RvSKgLgX5kpIuXJqPb1MDoTjON0wsHXJMgCOgd9PifHEKhz7UB8X8tURS2XMVbe5GThle4KfPZ5TyL1x+V4lBXj3f4KFy8QkXHuTRpmHlgLiUpF9FHVOIgiNfLyMDnyZ7ZREGmL2tlVgrt3lm5gyuTCpPgkpyDZzMpnWwWrVn1VVu4y/891l+awK/aLxk9eDz9R1evD/uWPczklFyEhwZOpJfSKrga+CMn9HCnzMbIGFG7mJi/ejkx1kB5tImQoWJQwbd3Ah2Sa0z0vIqin2wvQFjeVgyqHjJ/WIlJK+6CNyMld/tIPKOqGZZ0hba2KUgkNjJqpKqe7tWL0Dd68E2XhZGCGEw0GioZVG1BWZ7FyYkHb1Q9bUOHmj/nb8Sv9TYySt5AeoOzpu8uRjnoEy4qYvHNuIJboJikGmzK428/eCTHD79DMQREiqX9g6Wn+HnREv48FMJXR+MIXiQmnf2bk4T0u4n40IPVsxMC59yKSQBHAZx61GgaohimHKjpPj3zMnUJ8ZRdiuJKY5CI3CizmjD6NeL8wxByWr2wPmC7J1HOHcz9/TceW/swPoeCcTPfLjqvxLyJv1/aSEPYQXZM3sP4W9TZvvFuZRoz9iVzdNRRzo4U2w4gc2T15J0ExQURuXcs5pFiwibO0fx29iPKHb+i1TWbeb2PE9evE2ePNJYwo5JKiQEzA7tLzRItIxueZExEI0bjBqCPaC03yHHWOSM3yjFLzQNIUvmpRVwLv8T9GxcQmHEe+N52zVI5X03bSvR0GbLwFurMYtUYGe54FRYz3yGJR8fsocJRnPeUdOeTtf8k+/Y/xfa5hznfbz6p7a5GWfEW3Y0yXk2tZKLsKbudK+w3Z+FukuJU6sIjD48D+vI2qyqYHVsXorumJ/ThAtwDxR2eF7uP8us97xPsnM+VQCl/don3SnHdnaTumIdjUSQ9To54fiQmSW3fGcrQQ9P4eOwp2pP6Ei+1IQdaTEilg/DotCX7wyO2IghitZGb97io259dJY8ybJRt313dWU1c9m/olQYiXy9mugtg6ZvPAi019H7yMKkR1zg4oRlX0xBRPjhaayTr9BiiDiXTrWmlZWoFN39nVcAU/jlyGpe9p1lQV83QaaeQztNiD+B7OIewffoerFYDF1TdTLTY7nalPoM/Qq7isbiJdwKbCJbbScrrW+HKR3iGHeecLhuAKdYC7OsMD7nvwPdZHX+tncks1wdw1b0KTEEQYOeZ++isdsQiM3K44m7udLWd2mIVuBzoRIHXcZYF51FjUGIP9gkClIdWsPrlD7m/dRWjne31PPrirvVXSTx8lcqkN3noTfF74+/+I4Pm5nI2PI74+Bb6g32atp0s2H2drXnzUM0pY7n967HJh0HuYznUE8fpf8fhMM+2H3G1tjHkYgaeLZ4Ep11lfHC+6LwGNxPnBp/Dt80d19oAQEwy2Xn5D1R78phY0Yb3RDHRTmrVsi2oi1NVwRzaMB7fu8SdqyFOEcSVOHB85EMUReznlY5EZoqyG9jf44nlWBypDV5EzT5C/+Hp0kNNt4ao4mAyXWfd+vuGRn8ulMlQOTahVWtvAcwAEqmMxqRCOl6qgu8W8W3iJSbYlYh7Zd6s/2E23a7t/PPAj7i4iP3ZXCVaMhzgilc9tVYrSf1sj96rdSLs3zSmD27lseSB8vsnejo493sbPjoful06sQjimmqFYghzb04FQS+hNz8AKOn1mMaK7B8Zc3QMY1Km0+0ecQugt1hAb9LTWtFKb2gvq2asYrxmPONNQTx1Yzr7vdKFuO3TaHPv4XzGeaKlMyDpdYh+AnNPHS8WpXFk41iq4iaidfHneSebjLJBHcGIKqiQtVEzZRTRksm8OaDy/t/xPzH+C/b9d/yvDqVSzG64b+qLOI4u5bFdM2nxDBKllBKrgU8PzaA6P4qa8Ue5pvERbUJlhkqGab8jK9TEoLsK+LGfLEOv1IO9W0dg7XSiYY7kPxaAECTsdm1hh6GbDQrxBsPXWoVncgXbPVv4fcQR5srfvRV7fc4Rro04w7pyH041qZlzty2JlwhShBII77Aw0UGHUdpPDvBGcuTr3EuARxfSfrJ6OVWdVF2swVPuSZuHuEgCUOHjRP6dfzMkWIrVEjcg7mRpZ+6WueglFtbd98oAsO/m58+sBX8SuMlDD+k9wxvZY9lV6cHSlT9R73wde7DvraYTHLX+xvv3NfJy1C6umEeBXc+Alj5g5nV3GOkoso1HpVAzZnc0Ts2PcM+aN/mnS5wodzvFsbvUm4yrMdy14yzTFqQOuK71XQH4JucxqiaEGOmP2EtxAjxSFoO6KJyqGfvxkdjAlJnORWw//zCFcYVsWLiBd6ziZPX99U9iybOSPGEsgotYh7+nq5C3nprOxmJ3Pl8/itZUsWdOTm87df+W4tI+ny0u7SRsFoOUFywpdLVUEuLTwtjMYvoPuXsWH7z6DqoKE/et+BWh9xJgA/t60NBjkBB5OouPZn5HT10whL6KRNfAc4MreDasgsbDGlCIC8l69zFcGR6Fj/Fjpu2cgVNYrig+sa2Jui0zOT1nG3tSL7JFukIU/yn5LG7SLubVXUFhygE+FcW3tF9hkCBh6uK9DI8ZWJk9Yo1nh3Uvic7g1884G6cIftv7FYpjB9mx/TEKXjt9i6cYKZTxdlQHu/xzeHdhDr91its7jqiOs8sJRpW+jq5ci3RFG/ZJ+BONodRlx1OQchkHB3HSGOEWyaJ/P8OnrpLASaeoVYk33ipHPzpa/QjS/I6LqgaZsQ4QM0M3xtbyzDaI1buToRIDu6rG9Tzw5ne8XhPPX67neFVpu2fv93jQJKsi7nwcg447ct/TtrnK7ODD7Kx9KJOVvPzhy2iG54rvl2YC+fO/RuFxgNXx3/FVp9gHTqM5w4z7T7Pj51mk5qYOkKOaHZ7D0FcPs/SqM1d9xO8lgEp3nnlxh7i03pVtuWNJXCq29Z5mfZn2nO9om3KIZ1MrkOqrgUAqy7cyc/LX/Og6Hld3BdWBYhmh1NSpHL+9HeWlfYRc2EBXfSjYuUksK4lGUd/DpDF5TAzdgMSYCtjYhO/u3k5Fy3k8HvqTXOd+rSMWM82xxZztdsK36UkyPMW/NfTNc94N3jye/Sxzpz+BPc+8o+4o84Pu4rB8EoRn8ri/bVP1faeVorPhYC6mOaEJmTpUdN7x13wI3jWDndN2cja6mC/s77epB9WxM3jlx/LlsxBr7Qd0C/DL0j6JtPeuwyv9pJA7uw4wcfgKvqx0ohAV9f9hzZo84hcOXZuI/MAIrPeL56saeTjpJRJW7B9NTLK4W2GFyZGOpkjq4kcTI3NFJbUVfwQBVD0q9FaBrOugMVdx0O68EcpydKNK6FL79Jl9W/vu8M0xquhHJL/vQAAC0q8MKLKMCXibDWNW4uL/K7mWFmC7KG6xkxQ92enXbysHo5O+onzYdZT7R3NguK3YEWfNZcm/t9MlF9gecRcp08WFjOMXfDiz/mlC/x97Zxke1bn1/d/eo5m4uxsxYiRAcPfitIUKpe7uxqlRt9PSliptaYtDcXcNFkhC3N11MjMZeT+kkNkz5/PzvNf1nPtTm8Xes/Xe615/Wf1y6HLG2Si17XtlSCE1TUr8xxUTG9GGtd2lKHfgaq0HZrmRrbdsx9lLCiTWlH3Ejttf4IW982jOeR2ZKAXJAQ60rUBtWMvR6ytY6LfILn68dBmO1KG9kEj3ssEiyZMN48gx/8ULp0bjXBxJwC3SAk9YaDEdPY6MOBUAcVJFYpyzL48GLeakUyl7fPbgECAFhGd6P8R3UcvJjlQgM2sYLW2dx2RFHi556ez6eTYX03NB+mkgNGgq+k834NRWzQ8TH+ddqaswvRYXOodep9kA13S+ZIpK1Go18p58RsXv4GDbFE4mjkch+IOV5rvLcSqrDk/Fy1jEDNnbeHZlYa2CSOropWnbdO5cfInZcwNo7r+ANenhdpcuND2OOG7YT3DWBqz7CAE83ehD57j9MG4/Ex2lffUAztQlEfvDwNwXNcKmgCnIWaVpI3/USeq1EC8bXEo59Jzm9Z/e5KVz97G/vp5Qv/GSTcv0szleMBu3ruW4t7ugM0sLIeGGfRjygzj49yIuDytklpXzY6j7ULatm4eixpUh/i3I1FLQQS5CT/PAHJN/LZl0ldSeDsDs/DKQRFl4GWEqqVmQoreE305N59zeEQTFVOG24iI3wKu6fZkMP5aA75M/8m1aAwf6rUg5gpmKX2/BoTyA2KQvsIS7Ym03r1OGsOnqQ4ittdRme1P3krTgO91RZIoKQpofQNmXjL+jdH5/xv0o1cGjaa72Y5j772DThaVTHU1JThSe7u1cHtHIOJn0Ia5Ux9AVfIaQ6hD2t7x904rT13CV94dX8cuFcBa8/RpHZkh7O7qEt1P57G8k7pqG47GReJqlbG2/biO35GRROKQc75h6kmTS8/o0uAbX+VsotPTg22ujJgfqAxspevsDNvnDd7pqbME+RGhzbyMusJv0YGkhT8CCMOE8GyYcZVTxbpLUg0XYS44BCPN3o2t2gbXzqb1bej2rYps4N2QPS9ctpTKkCuPjg/OzYDbyaVQ3P7fIOVnsicpBCpTcyCdOjT6Pk9M+Jth8igUBRJMZfUAYFgcFMlvpB+DZ/CUehZE8dM9OXHvOgJW/QLfZyKhN87mSn8Dapfv57Z9pUlC6sWLqVpi6lTs33EqPdwvCQ4MgjYCFjpJgVHXOjHBtxuwoBQWudp7g8LQuUi94cP/qyaRMkFpTK4L9KEmbgiDuY6dZx3TbBWJHM45dJsYtOErhMPseXEr6iEwo54jz7zS72Fs0y4wi6RfTGXtiLDw06AxQ0d9PbX8dId0i4TnbUSVNxPo5uJ1JFF26QGpZFP4u3fCSFEgc7/sRo50LmGuxQL/UGcbbyY+EnV/gljMAOlUYpDnZdYsru7YNxzM7imvjp5Mls3dEeKfdkUwsJDj146wotYtHRK4gvcudn1vNLEgczD9eTj3Bqe5hjCwIZFZNNYnLpGv93xsNXD03g6hJAj5BUhKfsbeKkQtPUWyG8Q5QZ5Dmqfluo+nY0oopoY6klGNsOaaXKPu+K3oFR/Vm1PF57JxwlPE2a25nYx3P3HWQ9V8s4VR1GrVxUpWEi9BFRrsXYoUjgqkf4R+w0KnrFOtH1DIvpJY1vSDr1GDl9E2NMo279odSrxMJfuYHCtyl/YjNgprWvkCa9mfSptTicZdUeb1d4YjfyNPMGFbCgrQqdK393LhqQm85TV5OfDkin0sB7zJFtFdJJLZE0jd+H/EhMQyVSdGlIW2HaA0XWLhoO2sdurjD+ObNmJvvr9y1pheDoxrt/B4CY6V5pFp0RN5vYnpIO1vG1XHA5t0I8BzP2uWb2JxyEZOTdH13Y74Iqwlm3PFx6NKl1yS/OR9T7TfMuuzF4fmHaO6bg63t3r9ODyP3wHBkEVX0v2HTs7WrkMlnfqC7aRgHuwqYY2VN/WixK7pNA20MBO9uPJdJ12iJgS1og1rpPtnMBUW7tEWEKNDp1kk2kN0KM52lZFEHdQ/Ns6fj43OSkyULCcqQfnOqO49TPfoMwSdHElAjzckA6M5jtFcN1V0BNLntpt8svW6ZXs/x5+i1JNQW8Ut7IDd472ZNBGcDH0MfnMtK/5WodX5Y00FFXSEvBudQuXs07+yZDM91SPY7XSjl6uWZALi5CIQopPNzh3cPFaEVhB3JwtwZDAEQX5kGlSCXW9g5/RyGWcVsbpjE8YqxxNhM7+lmLdOvZGK+5sfvngsZ8fjgvaxqzMY34RKVooFn691Y1RJKnXrgu3jZmIHYVUxPTyQO1Q+icpOWp1+0tPLXodl0BNRhybiKRpggiRtLKvCqr6E1ppi97s08LErnm9cSZ9IancVq90m4mNWIwqDyT0Skw72D1MuprC2O5utbov5pZgM4RbCsfwQFV1tZlLuA8JnHiVKexrrBaZksgLqY3QjNt3F05xjG3OdAPAOqxz+KShl1IgaX2I24WcKw7k+LXMHt2nN4NXth+e0OGoZVsNGqqKkS2njNx0T/mQTc9+SjCm4AK3qugIkgr6Ooe1uZafkX8jMWmLX2Zvzr2gVc1ByjNeg8gW3jsZ29XTp/xdIQj2PDUH5P1vCK9WvtPx1NeiitHm30OPWgtnrn77Z8TcvTXUysFlgY3kauKUOi+L7P8QqrXeHjxfv40HcfPxmlqu6A9gicKo7iEdHFvCl/IQiDjjWCxQw7JqE8lcTVMXdg9JPOkeqeM6QV7yKyaAjaEXvwtNj05gV21CYxrNkRh/QrnNYOsVs/3nLdg2HjDvPiPAsarnCj59/anKGMXjuchxIKGeLdyZkUKwcKuYaF7QIoW3hArcMiWCRWmnrRnfVztmNy0HFsVC2He6VzTZZ4ndc6I1gZ7UOH859kSUueXFA2s/+BQ/zkGEWaXwmC8KkkLgCFsYVUhXaQbTnEm042deR/7s9EBxjjADLjQGsYpUxBWawD7V65zN8MUaMqEYRBW/bdV75j96PPUm2Ef7lBj0MNxcZMXmuBxc7wk6qN1uU/4+gcyPiI04TKk0DtA2ofFIIDe33kNEwqo0NMxqSWfpMsCifq8sNJzUtAO+oUgtd/Krr/3xm2eMf/5Pgv2Pff8b82ZDIZQ4ZIE7+a/uHIzH7I3D7Dt1+QLFyUMhVe6VU0+um57vYRI7ykCZRJE8Pf6xbR71XH7iHXcAuQFlbj/ZaxISORYYf+JrZQyjK8MRKKfiJu25N8+tTnyARpFuPpex+/uo1ma8IdCCDxvK9366XZpYkFX71EbWCtpKgqiCKTD0zGtdOV7ts0jAqUFs+zdDmkrb+Lk5GF/LniBx4WpUqGC1VfMuFCMkMvPEbPlBPYDqMz5Hk38NnhB5gy5ham28S95f04BDXSVOPLZfEFbIteE9v7WPzJSgD+fPLIzb+3qBM5n9OLrNmdL8Qg1CZPSRdDxwY5M47OoCYmgOPKW3EPk369ZnX04bT1Pk43erN9wX6+lchhjGjCRZq8daivqRjuMlrSc8/fyZ/LWe5Ed/YSkbsddb8zIFUcgIBap6az1Y0rpnuw1f5NafUjJzuVF1MryQ8ZTHN2tq0h795yWkOO8IUXqCxSpcMJ0zGyOoezdIdA0iRpD4yG3kbkYhMNtf447JyIY7C0/0Bxz0Vyko+RdmkY9REpLHaTFnwbCWTN3M9wE+HDSj+kZlZgdoqhs9kFNW08kevFqjulXf0uWrL4M2AnmUs6CFN1kPlPM3LBIxO/Mug2Cnw0ZBiL/drAUAH/WHmKokCPeyBuisUkrfgec8Qdkv0WBShxfuRX8m/YDNp8kw/2eGISu/nDDwzmDmxHfsRZ9r55EJUnGA1TmGwTF80iCbkJ3LpzDlOW77bbPi82B48aEyIehDsMLkKDaWKcwZ1TvQa+6+il3Ohlty3AqfQO3Okm02SRPCULFh/jXJwavr2fiLHSAo2v2E9Kaytd/c68mGhgrov0qEVLH/76IvwSc3hkzldUaDOwBftu7ajg+rh6vlsQT5ssG/jrZqxS102XRzutPX1MzWlijlURXK1T4dbvRtbpLDzaPOCpwU9xiEsosy4Y6VRfYvQ7M2mwcc4Lch1LU81YDF338cyVAgwZoZL4Kr2aHcGX+PL9Ap52Erlow9K9LPhT5HuFJYHQZLAvGHS1/EaWaR9VNbdTH3OU623+RIYNvvnpwWvRvnyaUy7ZZDgYEP4pVl9zySKz7N90u1xkx8QOFEY3nrISo6WqVnH33E941GUuCyaUkiv3xxrsM5UFMHZTBq5LKhky7DR6i3Tx237KQFC5wNuvlREmSK186KulOLONL2MOsP7aGBpk/oCU7PGT2oGN3zwCaLFMt7moxl76TibjVBLKfZ98hFNHMDBQIOuyuJG4YR6drp38eN+PLPSV9j54fHgOBZoO2DSLgLoAhDcHXx6d6MDB/gjUgp6M8xkQLk32NMYW3g3po2DdTHzPjyDhLykVw9h3kuWqYrq/eILQ6CKQOv4BsDXmIpbxGsr8qrAtqwoITDw8EUNIDZNulfYRij0/lOuHM8h6eR0LEjdgNLaBcuCbKTfreGrdXcgbvdk6dxv9IdLiT5yqhHkP7uH7q4Fk9SWS3/I48NDNuLOLG3UBIQTMXYVjfBnYdPFMdbhOiqIc85YlNIyQ2j4BVPZt4s11K3mvDXbW38FHNvFG81gaotfS7rqDBNfBdzLQUknSneV8fTKNqIsbaU8YjTVAlGvxwlfZR86MI8jiSpnl/I1kvxf8e2DpFu5uBVmNg4RDLihceHHhnwBMyqlhok3+0WJR8PlXc3E8k4IjO3DW64GRkn+zzuE2dI+UEdHyHYv+w7qnLOo+riXPpMnjLKnCoHOAc4uKFVtXUOPXhPPtW3CyKeQ9ds/zHGtIwbHGEUdXPVhB2RpTO0+kvItf7XA8M89xtk/6lASYj/Lg0P180qagzqyjpvcJUqy+pjv705HtV+ME1AXaF7kBBJMOtbaNNsMMdKbNYKXkv0Y69/2zHlebzPQ5BTJkCLQ0OfGT5wxyZzfzvsEXP2dpcb6rvx238p14t+TT3KhGFfYz1vfyWkAKzfccQZmXxIE/3Rk6THpeT16Jx6lMwdSoKpTB9v2N/Gq8WXJ4LGEzTtKTUGkXNzvBmRFnqA2qYbt/BBINvyjDtauThpLXmOa9Fl+XQaXlOb2JI2WZ5OJOsH4NGdJXh5aOHNzLr1I88QjyyCrmmKXdv77d9wSeZ8vIS1FhiZLmFzLnJLIDH0WX/hEzp31HpUVq6dekSeWukW/CyP2klP3GDEMr0TaK9pfGbuVU8i7mtvRz0GmUJIbag8v1AwesHlmJSjWoru5y66U8vByXjTN5+u9+Rvw4OAebFF6sTriIl385lpOjaVcWSfrMnmrcS+TZA7h0efD3nL9J8ZL6U7R0vMwb76QQElHGzHlvgyAtDtUYPQh9/VumCx7UdduD5IIgsGnRJjS6MMZdLrB2Pgdggt/tfDlsBjVpagTVYJ7qYmpkqjNsc+rDIa6UpSHSwlGmrIJvAuG+ufsQFuzD5CC1UV5sLqds71ROmg9xTHsvf8+Qkgsa/Vto9OwmpigGwdumP2J/F/uCdJzSwchqGOFpvya537MNt/f28vy5T/BwXCrJcQUBToRp+elcJOH6j4k39nPjm5Xv4Mf+5HVMOjiJ4VeTqe+Xkr8OheZTa4Kj5eHoVXqsEz5R4cDYQjWKokge/2sxrXOkfas1plo+27WQ1lYX6u8P4lLTS0yy6mgp6i4wtfJnOio9eP6HjzEhdSYAmDXrNzzGOmBx78Mslx6b0qwlIX+AkHBZbdWXSaZgUuM+urrNRDftJcbZjGgeBIH61YH8unYaQbVBTAdin9sk2W9vz07Sp32Eqe8uNEWxqCy1QNjNuId8GbdmFfNMDLj0JUrWdwD6kc8QO+EMb2uNGE3j7Xr2jXSNovWDJ4hJK0J912lsx/jCPuQnxhKdUoRHePnNv9f3VfDT3W/h1OXCRdGXZt/jYEWtiR9ygrnffsnrvy+iqdANOVIZfYh8H/0t7Ty09T4Khl2WEmssFsz+3pRaRLRzP8TNUQom96u82Dr0Ivdlp5J09CtaDQ9gm+OWy87R9PgFTiZ3cFC0V+DNyI+m4Jc53OXRimiVQkf5naRlpCudxQoMtWYUpkawsmCsrw2lvciF7MJbaUjOZZiVxURlaxFjp2zkYq0DiQfD8PRNlggtRVHg2we/RafW8c31dOJ1cgnYF6KfRXVwMlfTw0i0yBEEC9bPeJ8qhOe69URrVZxsciRYlIKcs8RdaL55EKNBwf2fBiHqLoAmGL3rcJY3wBWtiG+DNwab2l2410I8N/YQ0NPAay+9xTBRajFXY0ni3xtmMazNxJHbd5Fh00m2y0vH1WnHUTjCU0owm60SaIULv14Opf+Hxbz+2J24OIUAf0u2lxvlGJUGZC5puCukzhgdDiF88PESkq7GsvXNtyQkQLNFhbmhmaSQw7wx51YO9g725wIIdLuA/hEDtX7n2NQhw80md1nkksJQpy6OZneDt1aSeQcpVRTqYinWeJP5wddcdX5Psq225SIzs/ZRfXkZU3fOwvkx21UxXPNrQQBGp5RSKErPq6g5F9+oUprDTbw7aQ+3KH6+GUuJqKJn6lmquqIYEXUBP/Ujkm2/CWiny2whbfmPOBj9JZURmWDmhIeawxejuH42EfMMm/mg8S2+uv3fvPTc69S0jWDFHdL3MsxcwosP7ueXecc4120PoiuMHTx5/x5WPfwc3+ybj/qO68BgTumt9KYrLJqPJ72CR1/mTbCvpqeCw8PuQqssR2aUIZilx+Xc30Lgjhno6nxpDUgi2lkKih12C6Tjg7eJ75vAy9EHOSRMAgav6YKsHJTDG9E2/ISsagpYidlFEc7Vz6cz9CJPmL4kSu7EA6KU3bXE1ErptpmAiQtuL1Dd/SUeHgOAnwMGXnz0b1btz+SlVc+jXjqY43o7D+P724pIrY3Hc98Bpt2xF6y68/7spqLgma8Z6t3DWyF9rDZYkYq1taQvqmKD/zC2B55EtITwsGjjnHFwLJO7pqM8uohXVuxnh+4w/GOmnOm8hUtu8EtpFGKPIxfnWtn+ypQ0CBoyDifj3+DPMB8dPtEHwUo3e9CYyaGrF3gEOJl8kRjx7pvXKz/+OuUh55l6dDb+TcW0XF6Nv/96YGB9Nix7GBOOTMBR60hueIfkkFVCG684yfh3lys9MkcaVVJln1zsZ9WdD/O5eTKl+0fT3d2KNf28z+xM1sYFJOYl8vVrW7EdfwnxnL9vLZ0VARiUYZLY/c1K1lZ9TNaLfdznAuX1D3JDSZmnmMJJYSdtHo183wWxphAJ/Vtf5sG+tY/SPv4o72TlobCxkx8Tv5aqxbBfDOdQyes3W2YACEpXvmiWE2sx0+vqDQ5SApW+5zwzl2znULkXi7oCOeVjn7uLZgF5v4JT6FHpbOuGsGz78/iU9jF13b+4YLmLG2Df8ukvs+NaDUmNInV5ZsLjBmu1ggDyfgWKfgH9c2vIUEmVwqIoUDikEAeLQJtOTl+/FF4p6gmh7v3h+Kc3MGPKShxMd2NNapDJzbT5NXLurQc5ErKCd6UpMH69F3j77sO8VReC4rxoZ5/rYa7hSTfIUg+0gagxDYhGXMxNPO8wjxxhHMFT1xA1pPRmXd1igeDuCxwMgvtr5NwlqNhr6KBfHU2FCZJV8KifnvdUlaS2P8zXPqe5pu6AphNwcCyuyato1B2k2cHC6JA/GOJ1Dkv/+wiKgVxXsIBfoy/DLg5j2MVhnLjdXuDwf2X8J7zjf3L8F+z77/hfG2azmfb2dtzd3RH/mbnubvkbfV81j4StJMpBgyAM+niqVc780HwclOCmBXebye5k00XWLllPdJEvit9mMvI5aZVFJor0uXqS8d7nzNP8BysIs4kRIX9w7uoYXj0yA5cpxVhPxia5F/kto7m9aARzI/xR9F0CRgBwZ3UgXYoi3pwksMBHwFcm/QjEN/mhbvbig4zPyNcuAwabJrsJIqWV/sg9GyjuB4tNkvL6iNP4Rl5k6wd3kWa0ZwY84HKE942B/H3cgdLQrYDUbiLG915Wj/qYOIeNLIvSYQv2jdL2UvDPf4926bj5905VOO/e9SCiWUTUiww1D8GKZM5obSOG88O5ar7AU7167k2pIsmqpfIESxdXqgaSyIZyqSzdaOjg3cdv45sd03j5/ZcpvEe6iIxX9LPGywHdpA6GaHai6B+DLdj3vPdhupY08lHBVfK7HVmAlE37RdZJEhqc2b5mHsa07pt/V1lm09cIGeaNjK4OpjBQqu44M+o82cPO8trq56hvjZTEjCofVn43A9+mAHrHPsSkIW6SeIZwnhdWHGXN9BamtPrhJZMCFoIg4qB14La1d1MQb1+A7NC18/cSPeXeH9Cv6OddpRRAFQWBhZsX0uXSxfL4a1xwG1AlCnIVYaeGM+zCMJpbvPlr+DUef2yQkX2x4ltSxz/OkbYh1F69xhwbjOeSyocjPnt4JTeLDLkDfnOkbO6D+7MYuWs5fwHtM07yufQR45uAqyi6XLnnijOxkfZqgintraRtWowJOKyLY7h1sL8Ln7jHuRJhwfdSARkugwuPgNAf+XLpJwR7BHEicgxjlkgLEV3NXxJbVMfulINcj7/OQpPUjO3Z8w10NuqYmfkyOh/pIhFDK8999C7rrr3FyLwjjJG6LqBtWMtC+TbObh3JzFgNj8UKNt0i4M34MM6LmXxZ9Dhx6akSPmqtQzK3lfZza2U/sxV3odE9CQz0t3kkJwb9gRkcWLQFF+82hliSb24nWvSM0K6jtb6PyYktOOqCbioRYCDplBm0OBS745o9heIwqQURiJiBSmMcV7ReBDpLi4gX+kI50SmnINxMl8reGvI8CZT+YUDu1cqtz/6Jl3YMWMH8Pg5XESjl8L+eYMqkixyaN/BeWuSOTP51KVqNgW3zN2GLGBd2JZB96UEuq2t4ODiXdMV0SUFYG9jG/vnbCMyNZkfFHD6eaMGarnEs808sCeVYBNvzBRTOrO4Q0ZrNnBrzBn90HQOk4FaffmAxbtD0cc30EDDIbjN5jeDJ8AfwnF+DsvBt5g71vKkPVQpmjs/Yh5vaQEkoHLZckez3W41Ic/pZxhlEHg+XgiF6s4mvp3/KyIhYZv1xO5o7pOQBB0sHd+s9+FMlpytYjo9S+lGrVaTwzoe34go4ZuT9RzX6F3ozT887wHVvaDIUceObBKA095DQGIBbn4aHQrsl230cVonDvDp062ZSFPIiT81X3EwIFeZeZo69xpFjySzcspAzi6Wc0JOmLN6o3M2kg5nUXUzGJUOqOF+Sej/uqVdwrWqHVrjV5sB9u/dz/uc0uttdaA6z7wNnEDT80Q3XdHLG6j+0i6/KPYTxWj9/zKwhz3PQTvCUfBafBDyA69RKVo0MpiJoH9YAUWeMnu0P/jrwP/0w3cau+LPWQDoM7dy7bhkFQ6QKClEU8G2fi7PMyDsj76NLMxdrgLNdHc0f7n8w2SuEMYtPU6YNI9UG7HvWowaNdzO1lUdxNoYB0sW7g7KHUbFlfBjSwRnDWW40cB/m2IhTWxKHhxRwOiKft0Tpd2FGrYJG80Vm9P+Ic0uApO+YySGcN575jH55E8+vOkepjUrCw3SarstXmbI/k4Dx+fT3nAYrWMHb5S42jN7OO2G/MzHY/t0TW46x4sEv2P1VJne7dKI01mNtZWi9IBYQb+Z9rSYTP3q+jkqnYsb7L+M1+grWHooNbX8QVVWBVmbk7J1bcfYcLXF6GO73IZtOQkX1ozRp3ZH11BFvBWwF5o0i7rAX7z+8mh5PpWQOBXjSqZumylASrrfRl2jf63ic1zVc7t7PA00QJbcvGMhzi1m00Rl4jAnfDRbni/U6tlTVEmPOZU7MF5iV04DBRV6w6VE0eVNIjgzh2fFVnLAhz2xL+wRVZC/THZ7E20n6URIE6PKOpEN1P1+dHcOisVKV5o082lMEbeydlHfsZaQN2PdguRfyczEEBFZiO0cLDv68Mn89zIeZF/qZ4Dm4RHx36mks03tZ8/kinLudMcjdrH5X4GrygJXT4QlHce+Pl2SZPqYSXLOukUs/AcN68fOQ5uZ9Fj/6dXJqL3jzXenTPCJtf8RW7TAO9BwH2vDtv4KtMbCnvoivI3rZcdoRlbYR8JXEg0zHeWfEd/x+9W1g8N3xCFvJ4w+kE9WYx5SvX8VBkKoz6nuyeH6PC2dCLpLv28ADaum1LPTwRfPUOpJ/uYOhF0Nxf0tKMPzCaSSl9RVotizAeZZUNYjFjFtxGKYP7yQupJLeF+yJN9GaQFQdcny7p/zHuf/ti5MwfJ9JdZ+K7klww5Dqhv2UJS2MB5f+RKe/1C6+5vpcis544fPqN7we1UZT/0eA2z8bi/hsmUVSbhIjZ5+kc4i0+K4w9xCssNDX5M1TtW6c886RxHsMRaSkXaLY04fva1O502zA2q4b4DWdK5nvL6EuzI83j0kBj3BdNqNf2MV77Ra0HoP3Q7SYWEwRFR3+XH3pC1QaC8Ot2O+iKHB87HHCy8OZ6BBAmSgtckcIFbziDbsXHmKmx69ckI3AGuzzaT5L6/6hjNRMR62+B7lM+l1KdWpiibOJN5sVOGF/r0zyIAzhBtTB9eT3udkY3cOkcCf6pp1lZfRl5H3jbpIyY/t2Y4qCGXVdbHDoorPfFWsIvlS8n/Xnp7E1/d+UT/6Np1yl/YcevezOxF/H4gfMTaqW9ho09fHFnUkcbYlnvFcPmzqLJdsKAtQE19By2zbGtfrT3qoEG2P3aZoG3KObGNMM4z3GIaUfwu5rEUQAiybm0GHyAAb6HL1Vm8PZSzoUQ/U4pBoYLpeupfYFraF6Xhk9uttp85DaUvdrgnArg+iiIJb/sYzsBdK1kr++glvjGtB/ci+yPjWXJ3dKip8BTgUEiZU8cW05CWFH2WHjJGuSu/F3fz+KFT/Q7tHOOhsAqVhIw2gYWAPlKDqYIA48hxbHCE5ciWD6gSn4N/jjO1J6XBqFhsaoaTQ7rkfeL0ewURTqLd20OO9C753ByvgWFEap0vJt7xIQ4JVLAYxr7GPblH8Ndr1W+/Capp7xQ3NwLwvDM8HNRv8GU8vc8P9pIfMfeYU6zwKwMqfOSH+FAwHHaDP/xSuOfrjLOm7G5IYmhpz+mZ4qHZ+7DqU+9CKTrcA+Qf8JK1J38chL99LR4cqMY9LzGu78ARMDL/HDVw9yJa1AYr8oyFQc3TOV+hxP+hPCsXhIv2kqUzcrsorZIFtHVbcKgyjNmQA+iajG+K+VrBFhIe9KYr2aCO6YOghkWD//92tcqc7ay4yqnyk33MvjCmkrhi/WTcXjUAbeD39Dr4e9UqZoze1YckOJk5nQ66RfnRyDMzvKnCmMPo1JG0RdbyWRDBIvm2TBPN5i4St/HUmu9j37tC7pyIvhBd8mEr07EM029rx9K3myuAjf6gkcG25FmOi+xrrok6zLCSbqm9fpm3NMsl2TLIrC/RbKE2rpSFtAnI2gPM8UyJE+E0O7nKg4sp4JE6Xr9YmOnfgrjOj9P6PIo5CvT75DToKF5KEWxHdhzZlP8bv0OrcWh5GfLtiBDjscXHC/dx19ZoHboxtQ6aq4UUfTqvzwKQN/o4xZPh4EeuSCORJEBXEOSUxsewZXbSu9XRWYjNL5d6vJhyKXAq6Z4VwtTPAevJfalotM8X6EzVkw7EgvMrPG7rgY+jYXbu1CA2ypLUHvPri9i+hI2+GpuNcGcH35NiZqrFb6ZiN+gp6TM/aScC6TM5HXmKH7SUJ8l2vhkW8GgOS6gLqbRBFBgNfiy+mVdbDaczPfeffRYhokXMgEE3f2edHWr+Sj5z5C6SCdh/qEEKafCmXc7vHsmrmLC5oOlljR+c0oeLTWnYqkKxR7VZKny5TY7LvLu/HpGYD/xjbY95ltE/3ZHVTO92nlePdL5yKD2YB3rTtpVZnMnpHDb4pB9bXG/3uu/tDHks5sHDquo5gozUN36oYQaLSgyJvM5bJXmPeCtP43LPg7ov30PF7fjMrghyAMrqMEBI5MOEKPfyNPBo+lzjkO+PNmfL/Bi9d8clm4bxktB7JwWWvvZrY07CIObxbybaNAlmDf2igyzosW8TKJe+KYNqX5JjlhltsvGCbFsu3qC4jx/XjqB7/zgsXAv6qi0P0xm4TH1zE+o1gy17iaasgPhe8uBrP7Xysong48NfiblXI/ymbvYFZVBn4Fx1GPG4d1ffljz6uMRMmy1Dx0rtIeswByix4/mcCEUwl4aN5AbnwMa3KAj7GEz73hoSZ4qw12jR8gnim6L/FQyqO8q/uAB4Z8jdzsTqPVu9HolMBTexPpuR7OxxfSaXq+isQRnnzRC+YGD773beNwr4DriTje2Pw1s76bDMX/EPLNBgQBLJgwbTpLr3877c804uE6cF7+xgaO3XaB9/wEyo/F0K+xJwv9Xxn/Ce/4nxz/Bfv+O/7XhsViobq6Gjc3t5t/CysKYPrWAbZP8LwDwCDYd2Ne9e+6SFbL1/jGeGMtdTCY9VSEXSX5ylxSr6SS4CSBFFD1l/NayiOEuGs5bTGSjHQIgsCW0cdI7HJgXHw1WrN0kStgxq/8LOq8OM43BbP0zUEVxpgjKWScWUroUyXMy/gdTIMMYIWo4LM317E/ZyqfrXkXn9FDrExrwCt2Pb9NBI/Wgzy0bTkZU6Tsy2tyFXVBVfzxr3dQWRyQcrmhvCCc3NUJqNNyESMvYAv2WWQactShPDg6l0i5jUcXsD5IQ+crH/NBsA4v9WCCJQoCCJB+MZ1Zu2dx+h5pshsdX8r0rw/yxJdzmLt1GL3LpQu27/ycaVm1En3hD8T1vSmJdRuNvNUM9R75REfPZXKg1BINbSXDHLawuncSHziF86ReZqfcC1R0Eh7RSJ68EV+Tjc0WwLXzHIt+Ho3fbtJVg31eVGIXXr01GE6NYFunGte5UjCkILyP0n4Lr8xNw0cWLSkwCgK4t7vj2qvhzqnPY3GeA1aaRD1qyvthmFMbs0Oeo7x/HNb2MJPFY9ytDWN/ox+tcfa97QwNO1k35nlWtsKJPlDYWF6GU0Kbbzut6oHFw42PhmDS42+RozIquJB+gfLQOh6XDyYq7qZa5M+9xujECvLC7C/VZM97ab00FsdjZVTKekl6XVos63XRUxNYQ1BtEHm9Dnbb9/ZqqPjmFu6rCOL88+128XE+LtROP8u23pV4B0mfBXTNrPG18EEb3DN1IhZlEHBu4LwUTtRHZuHQ00J09p8Y0hZxo6k8wEjLGo4ULcY30Jsy4Tp2OJChE8HYxw6vD/FyUkjYwR3KAOYf+p2IyhbGVa7EuW0yWJncVpkVXE/OoanEm74LiZjm2SfL2ecmcuXPMDYs3kB6ry/WmgNBEHjsq8fQ9GnIFc0kPzf4foyOa6dVuEJrdD9fJtawu3+wCCMaOwhvOYI5P4rZXjV4+EqBV03PLibm7Ke1V8Oa+9cwIvgBSdyvq4cX/v049b5OfOH3PB8tlBYgHZsVvPbRa/wOXBh1iR02wG2f3IPN0/cwQinwjiMUmKS2rO80T+RycRFZ7u3oHAYXqEqTFk+tI649Lry8+Vac0wrptKqNrsu7ROov/hhnXiLbDyIsbpL9fpVwjbj0Ht58ZhSWZgtGG2XfqTmb2FPky9X3XsVvmk3SqHTn8KbPSTrSxr8A1Yfl2I4i/y/5a8VbbB/1IYINk1tAoDaoltDKUC6WHqFnypibJYXpyqOsmZvHiqORHP9lNvpZNovQLg+aZFWMmXoCUYXE2kMAnEVoDa0m66ldVLlLwZ0uZTj/+vBWPC0K3vjsFbJVU4DBRahRdObEsGxecBW5e6q0qHpjjDk2GrE4ikMPNJCYKQWBNPpyPvnXWlZWOhKc5yhREl8LaEEX0sDIvVO55FRGt0nPgH4LTApX3ko7BeHX+LwvkDBvaXLaY3Qh7rclVCr62b94A4sUUm/HI1pnTDovAms09Dn02dl4bm+Mxqm9kcOTDtI51L4Xps7szvZv5zJN5kDwnDWAdM7QNMlJOTGWF0MaUaRHcoM/bxTVdJhB7t7JjJROtvdICxmzVUU8FwBP53tg7HTCkiBdVHnXuTH+5DyCa4KxRErnZ8HUy+oeP8rOBnGqtoO4O23AQEEgOzOb7MxsVgHvmOzni8mOHSSogKFfcqS5Amv7XYCnw4ejy/fkspMD/W6D298VdQn/1Xt5p3zwt6xHQ3Y8AXVefDbhIQTRhdesmtw1GbqpdGnHp87Ea889TOVss6SXz87+YbTut6AwyOi7FoPCIrWVTHObxOneSbi7L8Tca082au9rwiX4Crd8eJZYJRTabB8ilPK2J9QbIVjsxWLopLq6Go2vG0MKYgmuCuHQxEO0+ZqsKFDgZalk/rtreLDFzF6xhwkam768FhMyi5H1t+9Cp2xglCAFDUYOe4yL5kt0unYis1oU3xhLM66zYcJ5rhzJJGzuv+3i0WI9o2RqNra74utjU9y3WJjs+RwHyEIVKqdPMdj3TkBg2bplOOgcaJK3EHznMazBPmf/dka9+zUfFHjx7LEoZkdLnxMfvza6A6t5L/BlqrtHAAduxlo7rzA161Gu5idjvLyCIJs+WM7GFh7SehN/OYVpyRUUmuyVmJeqfblj32RGTjyLp/YYsMTu3wyciEUCMLn1T0Cr7Wfv7Fx6HAqYaNXDM9w5gDc7VnHd0soG94+xBRFlFiMTlxwmzuBIhmMvp9VS8LTPo5cLc0bQ2fsaMrMbt/a14Mugit9dFsSijYvwaPPg2Gx7drC70EdMpzeFGycRN+0lMH0DssFnVaGtYMeLqXizhTu+ngkMvNuCKGJQuaJzDGDlNj1R0YIEpEyRP0r2rp+ITwpDm7YLwcZYwEu9mUMlMML7Y9zUuQjGLqz7BEVdNtC8bTbxCw/SEmeTFyndWFHqyTwgtCqUeoN9gafK5V8cmK0ghtWUpDQAg0pkUSZQuedfBPcdZN/UfZzrzrwJ9s3tymP6j49QkGikIzIAi0wKrp7OfxZO7qI90Y19AW2kWVk4i0B5dAlax172Zx1CbU6VuG70OsSxaOoW5ssmY/hqMeZPpflJmzKN6VFnmVk4k769apY+ZsQW7Lusd8QtuhizY7gdiNmmCOV4UAmvxsFXcmHwUTLp8b1+gPrjqWxPVrBRa+A2+WAuKjf1MiGziLzEOk6c6SbGhs+ZKwzl5YeCaQus47O7tzI+pEKiQPqoMomskylM8Cxhym1ZgLTX1TdVw/n+ijv37ZvG5ekV2A6z3JWLPn6Yto3jqFBjtXIdGA1hH/F98ydc8NpLonlwHdYrC+D3y6GoKzx4PekyibJgiZJ9W8tuzvcfQtUZSWbzbkaOlPY3b3PvZ++0vUwPGkpfQILkejb3tbOnXcO1g25slGWiHeInyVF9Oy/QFgH7/a7gJ79CkWjfs2+WUy0TNPBmG6Ra7J01tk4+Q8bc7WSpxhDpOQgFHihYwbj9yeybto+8IXmMbtvGOAbXYRuDg9HKa3DtHYaTTurCICBw19q7cOlxon+sK1NipGreuW6OzHZU8apfB2bRgLO8W6LsmxT8b5zyL7Bn/UxKEu9Afr/0IRNFgdk7ZxNdHM3bb7yNzKZvZLDfl2yanMuZIeNpb4Y+jc8/24lotBq8m71xC+thzMjzku3cu/7i2wce472tCxn97mvI75cC/K5CDRNmnCNu2mXmu5o5gBQE8mnUULZ1KfdejSFoWB7K5UfAO/1mvMO9E7VOTcveKNaNrbAjsLweXcgPTGLr6oW4rVRIXH6clE44j5mF5r2LDN3VheHWwcVSV8dLTL2jkD8uhtP53QIqHpTm12tb/fjyZBqOKj0VcdeZafPSHqt4kLr3LtHlpsIUblOhULpyyPUVhMwyTl1RMdVPmu/1uqYiFAMexeABf8jtVXAgoLNAnQnK+qS5/Y2C+5Nd/gz37EMh6OEfiNTyD+mhwajE0hWJWZTOVVfVPgREl7AqLxNdrDTfE2UyTvIw7iN+5LVHvuMbG0V5nTkKXn+W8PQrhK9YS3//02AF9nXJ/dB+dg8viRp8Z0618RkZ2H9UQSyHIwtZlXaOSjepEvm6UUPe2USSgPvnDM77cnMfrT8sJupCPD6ZVxkWLVWjm1UOfPzsx8RZ4lgTOJkWxQpg0K47hlH05IgIurOcV64ixBAKDCKCTx0dg3e+gk+WnkPlcHLgWEVAGHDaahffIm2HIwHEET/xit38fdbsSVXwfl5wh6VeUGK2qgsJcmKOjUJu8mD8w9lMjfwJjB2gdCXAtJVHnT9jgyqDl8Z8xy99t3HFar/3u/7Ggd1duJavAks/RityjNk9mZmFjjTWeXJn+bukjziHaNkJDOYAH7QYqIisw6UjiAu5lxgxcvDAr/Q8zPXiyQQFX+AJ7wBSPa16bWqrWCc7xQdp8FLogIpsFNI2ESOUeXQOUVMeU8wbU3JxN5UA0YgiTHLW0mOBl116mGJq50OnadygoYgiPHHvXv5YspciB6gwSMkDZpTkOHZjmnCY2sBa/GzyKgsytpcEElTlSEtMkfXpAnC3/1bC59VzeJucF/zs7S7lZjOxZRF4W3oJDeiR3MtF/Vfwaw9Es28Ci3QCcSMHATtBgJhzv+PYObCizJyRj3XtpCvcwPZH17D8t7cIu1qMh1yaYy6ojEdfXcjL617iwlSpCl60mJhtdmLo1nnkAz73SuNmUUmnGc6kXsIlsoSZwghsR5KqifzPbue1Nmfc37JX2S+95SN2rR7GwnWL6ZgyOA+6dKxnqO9otihewIwKibjf3M+ymAbWj7nApjOxfNFj4qDktAT2vPIoLv1yWoKSifGRkhqm+D3E7w7zaW7/jcN7swi52yKhel5s8+P0yw+jyMjmwviPJNamAI1uY3nh5/GMOz4cj5HXMPVXYQ32OXjdw30b3Njq/SptZhD/yX+N7qN5+6vvEC9W4/2gO63eBgkQ3qUOw+nPuShUes5nnMfX15dYoYnGCNjUNoHX908g+tcWYMDhQSePgrA7QeUFATNZLkzhxPnRNJkELju2MEMxuJ5xc4ngZOMi2nw8Wbv8a+Jks+3uxf+V8Z/wjv/J8V+w77/j/6sxLqIaWZaC/jOpfNdjvFnEA8BiYmLrm3SdUVCr8aOv24ahri9FHy6Sv/wY/Z1aBI3UqrC+O4erXw7haF8Snzz0I9LowALje50R9aSdOIvws1pqqdbV8AWjLTmUC2EcinyH21wGF2yp4bVotRZ+rriXbWVr2Xb74FdTJop8d+Eraq6ZUR5rpS5YZafUAUhqOYm21IKbORnrvmNft4VSoa/DJDMhs9irIM4KUfhGlPH4uGsc99LYxQVMzPmjkq8vLyBvbrckgQKIVn/KjrIDvC9fT6xLws0id4j2FCfxZa3BkfCROfgHSu3WvjWMYHHHNRZMPIVLVi6OFmlBrczgQiGwO+k7hjp+DFh57ctUnNk7nNiaYN568xFyDM9ibbtn9p3MO9/9G4+yAq49/yF1uuV25/Vc/WJOmz/GEg2H+g7axb06QvDQfUG3mwNLYt1u/n2651R0ZyfQJgisW7aOh4T3JdudLPGnrtKbvMClXFWqgcEijcykxffRP+gxyBjpquSIvg9rsC9PSOXPnycz+tRojsQamfe7q4R9qRWcuXg8BZncyAt3HMB2CIKZtktDGHU1gj+XHaGrZTuEPnozHiOrRgjswHwqkTVLDuHUnQNEIXTls/WuU7w08xQ720FpkgI8DeokqlL+wr3en2F5b9IRIWJdQM8ozcP4Wz2tkUO5HDiRWUrpon7RyIuIoyt4ohk0RhuFHPBxbgYzKoJQenSywK/ALl7u9wG/qY3o5Rp8bBnqDv4s35RJ+NaZfOeiI+Xd1pusaJmpAyEzGEOPB0NUR3GKvIo12LfXPZiaj1cy+dfbuP2XB1BPlvb+0NTdScTxdD5/8nPcFTOAP27GSrvKuap8joSa6eirA3E25CJJWFVRPK9u4P5eBzKyMxBtjxt4S97NHAYAcWGsFBBx0RfjmHWZki41lyMEfLWjbupd9ih/5Yq5i3ZxNk81Cgz3HbymbSYjZybV0jhUx7Dff0CZ4g0M2s9c6ylHlZGDEjPHsurYpCuU/O5Y9XW6HOPQ5KvQVfyOIEhZ/dM96zDGddBxPZImT/uefXJMtIRXcrkoirBjAfz82ByJUqij4TqLNs0if8Ix4lKLEUxaQENQ2zG+/PB7Zp8PxPff95Ia3cAxKyFmn4+J3TN2E14eztjjY2l/Q1rcPNDrxnlDDyef+JZQBYhmqV3m9m4FhRgIiq/BycumURDgzVRu2KgWmsPs4vs6vqIgZCXvd0KUMplXrGKioY1FTuBRHoFnlS/N+sGFf405iB87oanCn6pjwygfLQUzYq4Fs/jkKD6av5W6wDrarJVMxm7ONCewr9yTz4S9DE+XAuUWQcmZ9At4CAL3XHqMqeNlEsaoDDMlwy/wjQq+qZf/x9bWC3QutNcG8MuW+Xx4v1R7qlMH8WC1gsDXnmN48lWwclAKkZvok8MHLw0QZlYLgz22NGoPLjVMxuB2kOyQLq73SUFKpcVAXEEcV4fmkJ+Qb8dg/6JQj/+5KO4/PxzjUGlfMYDjnme5+uJ6jHIjiYK9LR9mSL2SShMQNclWxwPpyXk4Xm/Ff+tCvJIH3w0nunjix/tocG3HfdFmUhQpVkax4NVmwfFqFivODENf6cGMhVJu/NP98VQVepD20AbuHG7TG8ZsoOdsH715Jhr85eTogphpFXY1VPG1N6zrhtM67ABOgAU1Q2nlDCMdYGxQ+s1F/42x52ASxux4Ks4MZ/5vg2DKF20ZVJp38fzWZfjW+JD6h1SRG1kWRurloWSdyeJi2lV4ezDWom3kt2VvEVUUzbR90xDMUkZEia6KXx//DItgQafW8aPyuCQe1/sov834k7dWfY02KJ7RL0iPud0ljSV7I3Bvd+fU6FNcSpICOeOd3EmVw1U9DFX1YDIOFDgd20/xYVM0l04P44MXPkCmkKqmKhUZvNjzEfXygWxIgbQAGVD0Dqm7IEEZAuMrUZilc9miIfewMKgK140T6Atpw3bElUSiDr/GsO47CdL42cV/bZjKqZcWkgpMfyLbLu6XmY1/VBUbTx3GXxycj8IN10i57TB7y70xNPpRpDBKnsEvepK5bPyTFUfnILS7Y35IKjv5I6iBjg4nPnnoKYQ0FW9atRPs7SnEc0MYQVd8gV24d9aAVUcS9/5K5uemcuZAFn8eGkHSXBsPUUD5j2Il1tGIB1KClkxfz09doVR9eg/u6S9i0T7GDZXnphNf0pldzKvT9vF0QimVqkG1gYPQT2beCVxKY1k+N5LLSilRo0kZT3AFLMoeT/KpEST+S5pfuPcv48AtxxlaCdcMYLJI3ScedAvmRN5Aj5yJNfbzgUfww3z5iQFHOkkZZ28TfsycCuwDoLprkDgmMzbjNkpOY+0oQvKOoHRKwNrSzDnSh4Kse4iv30Li2uVkZErtqLBYEE1GEpZdYF70euqNT2MN9h1pfg6ty1nuC8jHWUiTkI0AghLvpXDo03ztomS98wJsR7d+8Lkd67TfLt7lHcVpn93kJOcwQTZY9DIjYjKJ9LU6sXLDBu591R8J/31oElf6gihsW8/6S35kLxgs0IgYubgoh3VdOdzRCMFm6fPponQluPleTiXuoNCvjmmu0n6XN3r0xRbG0u3c/R/nwYwTscQWxWC6/wwW7VislcBtqkh+yY5Al5tGxORylv7jEmcR1RwPiqZzjImokqiBfVtVJ+WmTr71gU9bDRwBuyJ0D67UBefg3eyNYX8mvSukxMeq0B6Ovvwer6x6heOHJhJvkVo/uha7kL5vGmpNH3dFSvNrgPKefJo882kM8CfV/J5dXI8nqquOPJv7JCVLBtXojeph7NibQVpeIkO3zeLcrVLQobeslNt/S6VnSDcjE/8kxPAQ1qCCr1c7bePPsrXgQS5cXm5lfAdmQc6vf41kzMkxOAHnb5MqmI1yJ7b8OYnSkymseOgbXs+ypVTCR20JPNfSiEYAuWDPzFcojWRj4O0LH/Hl1ME8Id2pB5N7J2HNPgSW9aNIlX6znjKZaGmI4skEPY5KqVWsaNbhodFjcdbzzoOrye55Aeve5YrIe3k3TYZbTCmvv7aW5UcXSAqY67uVeJyJRQ2c9nqccTbPgouhmPszyshu8+FbozcaSyMgvad9Li60erUimlU3nyWH1kPsnJPLEyNy6T/ex5naj6yM+6HJqKehT0ND4DFmZMXT6dkh2ae7UMkqL3jjocd5TuPGnC0zJfGVVYlMyotA4dnOqDFXEfus1lq6Fn7yNXN02gk2pF3CTeqPAsDF0GWUzN2FsSCSLv0InrQO1u1jrtcvrHfM5GzrIsLMg+uoAty5lHmWE54lCDHXCfW+RbLf9j5XUrbcQs+o86TfvgdBkM7Pq82/EDhMz8QoD9KjlTbbtnMi9XEi8uqZsmsK2VEWYOXNuEyUk3oplcTcRLbN2/Yf54uPKwJpWPsgl6bvRzdHWoeQ93fz2LkxuO+ZhHx4HsLCwfwmq8IZr3O3Yew6Qpvveko73iHWajUT4f0LJapsajbvxrFXSugRBGgOy+BwyFo+LYM5LlIFkaAQOZd5julxtTzuBkeRsgtEQcC/3h9lv5K6E/Yka7mph8k5aXgURXJuxDm73qYbuy3UjTrJ6FOjCewfvCY6l2S+8vyMoAQzubdswcPizSmr7ZSiGRe3HvxKOyi56E/31O1Yg3139V/Ef6sHMBOtphfZAuk77dzoSsqlIfx2KY0Lo4rBHSJLhsO184iihYUp2zBZfElMKuOEKtFOQZfSsoqw6hf4MnIknzYZuTpu0HFJZjEx/Eoq/aKJr8ac42JLFi//0+e2TYjg/A+3oI/rRR8BfRbpczStfgW9O8cAMdQE1khslgWVF+wdyy3nh9MJNPl5E2jT1KCpt4lf7lxDZP0LxNVKFYmCAF0+0XT4OrOryImOKMugglnhxmZLKCd0A0CfWrjhnWN1zi6NZL1+/Gafs1OWjpv7vW/ddMYeHUr4kr+pDTTY2EPLEYvBQevAB13B9PkYsG7yIwomRnR+j2PERzzTEk1Fr5ToJAjg2+DNjL3jGOvfhS9SdV52ZyqXow5T8eR1ikxhZEpvFbP7/0T89S6uAC/c00meFfF3rLEYx9FwYtMsAqtCsIwchAvC259mSLKZ6uMeDL/1Mh6hbtLroagnzhUYXcPw8W24ms9iXVuxVHsw76d7APgwSprPCRYDI79ZwQ2vmA2yCKumBKA293Lv1gWI7a78suJnZlns3Uaer5lJst5Melg9csHexe2LskepT1tNT3gBMVbZ+WdVn1K7qhHN8Ltwi6qgS/Y8/FONEuQOjKOLnnH7eOSdV9Hb2HT2KkI479qCzsnC7HFlOPg5YJ3XJMhOs2b0BzwR3M86SynPG6V34y99EKGpl1iaVM3qcGmNAUAmCsQUDZBX3xlzgH8rvpbEnTRx1FaGoPd6FS/ZQB9tAL3CjfXRX5KsjeXe3+6iZ1y29J21CGxcvBGjoxaHiBrCwhchiHFcbphMuKM/h/tmo/DPpiVWSbG/SLyhFgJnQuBMOtsLaLe0oDe3sXHBZuoC6/hesKpRO4Xx9ekfqeYXlAlKBM1/KKL9d/yPjP+Cff8d/1+NabGVpCfn4Tp1B4JZIVmw6Y19hIrXyXcN47WnP2WYeJ/kI2CUOfHx00+gce5jxZubKBOktghVeLI7Jo9Rp0cxe9d8+Fj624IgEFUchX+9P2dGngGZtDDbKbgiu3cD55r8keetlMSemFxAw6hGdK8fptlvP92GlTirnP/ZLxzLvoUu2Qau3fsDvj7XwcoMxUGXS5bxNMYIf2LGFWK0aZIeWOjNHeteozyyhHNjpExCAHVIID/cvYp9cojoWSZh7QOUNf+MvN9CzNWhXBl31W77AFMso850o9DG43j3ILwa7hrF7r/m4lfuQ+LqdzivlfafE/UiXs1ehAc38XRoFeuQ2sQ5aEUSq2MpN0TiFteNNZ9PEASCaoKIyE1kdfYoAhMzSLGKn667wNbxz3Nv2b088fVjKCZKi0f/7AULsKkbWgRfbDmpd8qWc/l4MstX/YpZ+wY3+oPt63VAmH2II2Iv8iFlKJHu+9eTCYw9NoqnJ51ANlSqGFSYe/jUGz7cncSH6+aRe2+rpOArCAKRpQMf+O7uRsq1tYyxMn8sVczlp4xnuS3zEr+0O9mx/bTuYzj8bTEKo4JvQ+u4day0cKr0+Iw99ceRue9luaeRBsM/yZ3an48a/Dmnb+cOZx0ZSi2Y++EfJqNJ5sRP87ayoCqKJVfTqXOSFkadnNuJSK7gpMcQWtxb0JucYdBAhjGaNnwVEK4AvakV29HrYWD3nL/5M7MZg9cMu7hFVGPqbGae7BVSxOtYqxWQa7gkMxAOmLvUvN+quwn2ddS/wjezv+GF2gBuDaxjc28NWGlbzxgiONoLT2t6UVoEzDZNuR/OKOSkSeD5vbfQlSItosgs/SzcOxNFqzsbXvuKMsdnsdaO3Fh8ud+zmedcRfL0Q8FmgZ0fdJCgdD1Ppldh0NRiDcq56Mt5Y8UB5tdBbi+0GwZpYHqLKz0GkfTvb0fVY8Dr5GAirAO+dj2KUuXAHZOMqIMdJPttlkfx5LBjTFDImNqrxGjToOiqYzDap79FeSKTUEMwHX3zgMFeQqMChxLx/Jd8fvZnJqukvZMAhuoO0BMFb751G8WRpYiP2/T4cm2lbvQ10odU80JYPdr+FiCELqcY3mqFfK9Gzqx8CyeTB8uspuCJ/tcZOv88m3dBvMkJpUq6cLl04Q5i/xa55/YtPDmugEKL9Bn94bepxBSGc2T0Q4wITJCoDTC0s2L2q2wY18rHHOVO75V25xVo2MxrP9+HUO3P4ffaJGCfrKeER0+P50iDHzPe+QZRCAEG1DrXzImY3r8HT5mZdSuaWRYvnWme9Gyj2BzFvT/eT0lSCMrnB9nDgkzFruOJaK/GwhyLXQFStBhoG5WNkJOA+UwhBSmFWFs7Bpqu0R4Jr755N7XyqfCM3WmRPu8Yhy4m0KL5Fb1pPNb0TpPCi/1F/sz3bsYYLWVEf3glgaJTQ+l76kcW+PcimnqBgW9PiGswi7Y9j1Y+hbuXvkicEIo1JWKi5jjzfv+bl1vgaznosBJ0IQABAABJREFULVL1p1OnA8PPp+Do38zI8dek1mKAWmEkXSHyuj6IsyrpAhXAQd6D253byO9W8k3/80yxicc6t+ATKmNHmYBG4TZ4LYwn8HFRUVQczXMfPcfVxdK5amdFGol/+dKX0E3irCbkTlKQZ/zIc8jSG3i8rxpNs9LKPAYEpRuH3F6iYfIG9o/+gAyZVBLrZGpmTH4izV8vYirgstl+njRYZLRaYGcvxAn2gPWPiaXMKwxE6zmKcJdB1Va4+Qd0G3NxbLiAk3ctQS5Sy+o19+/laMNJWnZM4PZ46TznaKjEHA2feRVTFB5Hio+UxKQR+vDx6KHGCCYLdveqGX82vrMC1/oiDI4KsPFEEEWB4eeGE1oVSmlkqZ1FSat6BUt3hXEmYhEeZl/Oq7yBVsxqHzqnHeVC6llGuOtwMUt7exkEJ2qaXZiRP5SU46Nxuj9XEj8rz8clw4KTUs9cnTfZNvNFoe5WSsvz8DoRSMQk+7xpUdMx2q9WoPJ2QK3s5aaN4T9juseDFPMHyKDJ10b9JggkVkEfBYyXdSJY3UuzTIXT6EvkxsM+LYwWpaQFwQSOPY6czjqNYBGYyaOS+E/bx+KwfQz9CgMnnc9LNK1adRi7VR0EBdTzwN37KXa7HWuwT+ecxo+hW1An5JETXUSszN7CaH1qGad7DvFFz9t85DVWkpeJxl5k+8YBEKSupLu3iBtgn7yxDr+yM+jPxLC7/XayMq2sWhUuXCybQn99F+c/uxPVaGmeKQoC0/ZOI+LqULrlXhhk0me/otefr4+n8nFoExPjaimweQaDZPt44rNN1JYGUuIrzTMBBFGgMSSCWvcveUfXS79MmgdUm4v47qnPcO52Zn1vKjfoB9rG3/lgyjM8sXMhngeScEzoxFrp6G9Zz+d3v8HeU0sJHnkR0UnqQBHR+Dmde7r41ljHHQKcGG6UdDwriHen2mc4ifXHSNJIVdeY+7nDyZMcQww71VdoF+wVLWdqn6PqxV10KfU0K8ZK7B0FAd5ZPIMjvSaedOvDID96M1bi+wg75h4l4UIbaad+4mhJGFOsilohvndhGd1Bylu3kpuQK7XaFWT83gXny3x55PcnMGVJc1BvpYa7C6aQL9ezZeTvTFNL75UCPXdeziQkroKQ5dsQBHvV7KPxhXQZTMwYuoe61jSIHnyWREHgrt8GNL6HVB+y7J/dW0SRT2M+hhhY+e4beEa24mylSDQrvXjs/cV4X0tgetTjeA5RAFY2n1jYumQjt2xewMQjE9Evlx5TkKoPUW7guwe+Q6/S85BNg1xZUDflsy4ye6gFi/dTdudk0Obwkr6KsKmXuRoQAzb92EryfmL4CSPgzl1eg73XBVHg5OiTdLp1MKvDj+m+lZLtbgm6SNhCGSfOpnNtUyQhi6Xf2jd984kX5WTmv02Rxx5g/eC+BQG5caDM0+soIyRM2l+r0zmBn4ybmdrtzJINS2CUfTGuzuSIWYDeKDhjuWwX/11j4vrHT5HnkYsgDOYuX4w4z8Hkczh9cTsOp7Pwni89r5E5iVzcPYpNCjO+w/t40KqdmtzQxmevrGdNu5LPz/6MT3SihASFYzAtIWnUhZxifh3odAGS3CqvN4Krs39EZpKhsVxngmgNyYEgOqCNrEDXPJMh1+MxpEu/HUHaf3NX82WyPngRhze+HlDsKlwQ5Q40mkBngRa3HfQp3MAqO8kXotj99Qh8G3257bHVjFDdI1mPtwoZTDjny+g+NSavdLzV0nlwRsw6Ts7WkTzRE40r9Idv4GYVwtTL2Opwqv+Yx/DFuzA42ZCRALXnpxwKvZX61L+YIZktoL7+OLrsQnq9wykMeZJgK9ChWghjdRsM92/hhfgWzuqk5C5BY2LDHb+zKtiJma5yqlyklqwGjYlds3exJsgDneE61omqTtfGM9GbMBgjcDWeI8pdmnMpzFqS+pyJKIvkhTUPoZlrT8xxrBrYZn5lDD3OUqBFqW8k4vwwuoA3wq6xSzlIXFhucuVCUTRmk4wMNaiNjWAF9j2bOoUug5YFXmWgcLMyPR2YX4cm3YNoyMb92jz8h0mB8DjLRUbcv4ef/xrPoi9n8PC3UrAw1FRIwuef8PPmW7CobanloNaWsHLFAX74ScZ9JQnIjB1Y5yCCIHBm5BmuDr3Kt81DuAHTi4LI2ZH/tAMwgkO/FEj0MZaSo/bi06IY8k6nkZ8O91jFRVcXUublcVWs55H4So4QB1YQU/+oYg4lHSBy8xx6nfU4AqKl9+Y1WRxdSmVEPlk1EF13N7fagH1Llf+m4DdnJvIS5zPOI3ts8B+oDM28+eZavsr1o+BgGL+mdfGybADU29dTys47fsNBr+Ln8hQaQ6StA1RuMnwyemkdtoevsnL5xTKVGy0LBEGgJKoEg9LA+cxsEH25atML9sGuTYThTm53A0uG3Ump4lVuOC64Ks4Sn/ESpq+yMGsDqJxqRWBRefADcfj/eyIrq0KZdOtBVNPeB365+U+0Xru484tWmjseIjy2gH+vGHXzenW5aCmNLGX27kk462YRMnmQDC3KBLyavEnOSab11GiOTb/E81bHrKGWXdPmsG7TTK5szcQhNk9yTqJgYu3Ei2zLDcfx51sZnvaHJH6+O4P6iymEVoZyYUYDX0pvFXWyCERHLeZeDXG1UnvctzUL+KVrHRm3VBItjsPdyhXM0HOUqMxeqo8v49z6VIqixkq6ss81CDhcyWC7poms4fs5Y5yCNdjn4aSlakghziOvcCC0TtIeRS7X0Ns1QDA6Mv4IXf7S/CPUcBlvZwslfWYM0bCB8wwamP9z3foc8K1xZ3PgZSK635KQ7QDOntvOiLphrHr9d743Dzqp7anexhB5BHEn/RH2RaJ49io3wD5RJlJtlGEUjRy+NReZPEjyzRnqMZw/ov5EI+/hvjQXrnTfA0y7Gbf0drPn3VFEDKnm8pTLyC1SENKssrBz9k4SttxKcasLr9hMGQ6GJnqe+Jlf+6BMZkCwqU8LmJEJBv7lCc+6Q5O+FnBBJogUDsmhOriY13++nzEqKREqpOM0f08t4qN2+MEXthqL6JLFUF4/jRWpz7PD5w4upi/CKSOD81EX2Nx+D3S/CsWrkftOY27IFXjsCr4yiBKRQOw6ow55426mnGsm7uJd5C60z93/O/5nxn/Bvv+O/9Xh7Cy1MHqwLoROWT5OAihskghBlPHWrC10zACM9sz5XmUAJ6M2oDQo2bx9Fc887igBkARB4ODkg2QoNIyxWbTfGM+XJFN/LokhfRNxnyVNFtqUcdzZ3ccsfQ9zUqOoa/oGH5+BRP+Ly6epbxJRO69C416HwdAN/4B9FosFj7JfCG/X8u7dNZzrkxZCWurewbA3juq0XL4PD+cDXSepVtaP4/xrkKXKCL+cwBRPe6/f5a5BjDy7gMaNCVxZap8od5gFXBfvofViPCFqeyXP0JpH6MgfsCUQjWO4CfL4LeOEUzqtGZt4uxoyZKlWHYogorKfUasf4+AtO/A0XuT1NGky/LahkYt/3k4z8Or4Mxy2ignGXl55cgvbcy5x/MB45HI986wWJzKjFteoasyTT5LY5IvSZN8jI0PdiIsAixtgpqf02ADaw7SIchN/rbqVW/4aZLhv6U7jeMKnPOEGX3jDLpn0muUlFtLm1sy87fOINkuLRiaFJzOPheJQ5YUh/QIWtzBJPNBSTesL3/Nnr4VKs4UxNqar47wXcb5hNMWtW7C0uCBZXTBQON06fyuZ5zO5bBjObV5SS1ejzJs2QyC5idfwrDZybso/wJomgGt7LhDQkk18435SRuXCw4OLIhET7iLsCy3hcFgJoxyelvRVuBg4h9/iUlBELyBKYaGm6xQJDNrHfHzmNQwX85nmW4NTsr1F4iT/anTBl5if8wNPBtxrZVo2ME4Uv07SMYFqsyfuwR2k2cSvJxSyMnElACMcBoskBSYXTrx3O8W+dcjG1jHa0QfrcoXMIBJaEcrPw0/Q4d7BHvkbkv1+4dtMw7gGMt97htAgqY2WWlfDo/PPsKFZxVVzJ1Fm6XziqS/g28tjaDw0gr0vfY+/zF4l8ZKflqY7duLoAl023UyanccQWwFqi8AtGgsaK8ueiopFxNU7odMMw6yOIMpt8IoJCKj71MhMIn9kGhBlblYp46CFn/bEMHbvnUHpE9I5odgUyJZO+GHyee51Pc8J/atYg325vMHrW97gp1uC6TYGAtI+cu3yYN5/4x5Ek5z+cWdRm6TXbXHAWVLmNfHalkW83/YUT8weKGJ2O8ez/kQ8CpmZtiEF9BjlkqTSX9bNGAf4bNR5Js04T4PyLsl+U2UPohMPcL7Ch/ucOnhujFSlbBEtmGQmigLfxEmcB9babJOODL/NdDiqGQZUi/ZWb5MN9Ryv/EdLZYu6Ocey3qhD7t5Bh6wfNYO/LRNFehz6cJB586JHKEnuUmVrwjB3vINqOPpxMunyRglDV5BpeG/CXoZ71pP1t4hG4wNWFJUQsZeSMHjz7yjEc6mYH5CyFHVyHz5cPQdlSTjOQZr/2LdptK6dqe++yzxHkOnfBKu+qQICE45MwKtPw3tzpTaghX1D6JOpEfOj6dQ3EWD9PdU3k8Ym6ooCefH9F8leKLW01ioy+K3xKm6bE+nNicP1O6lCaHbyNZxXHWRP0fsUtzxjU3aC+zTXCLmQwdHNE1DeccnunNQyPcunX+Gjdjhbvx8k5mPwVdtwcsO2UPz+BRqVtcD3APj7PU7N7CMU5hkIy2vEXyNdXPRGGvlhxc946h8ir/Ml3rT5rPxYdYyLhYXUcgftHjbfSkHg90lpKGQ6pjqAq7lTEm7WJPP1+XMkASGxlSgsUssogBilFjfgih47JjdAk38b+176jGd1naj1cdxQMTvLfIi7cIq2gAQuRL3ECBsn5SNCPzmeTcQ9sB69WVr8Mcsc+W53Gl1aDd8s+puLOqm9zATZMV52cOeZHB8M7p2ozNJ7+XFbLzXx5xjl64dhyVfY3guFrgHHiWfgl1BmlMcgt+iBwXM3ydyp7gmlwggtRhWICpydnTG5eHNnXw8qRT+rvnmAiHSpQlmJkQd+uA+nbhcYUkqtQqowOuTmwrFZP/Domgco3j+SkTukyozPtmWTfDiMvHt+53SA3qb0DiHdOXheOMesFTtoMPkAUkVYaIiMy8unUNyaQaCP/b30aPZg2S/34uXwLZZJV4CjANSr4llSM6AJkoOdIjyjq5dbPnkey4wjLFx8jA5jhyR+QGlmSHQR+6fux+gmdUuwiCrWzdiNAHwuwGy9KPk2pIXMRls9m12zh9DrIL2eN0aZUx9dU07QfczNbh5Uuyfze+AEmoa+z3fxhex3/OxmzGnCGjqT2zH+MonswlQyPx58jjqMWtZN7KFTfYaAiib07s68ZL1fcwfpJTH0urSz5p513KdRssTKbj6nI5r43+NoAXJTrjNttPS4LplX8WfecHqj78dsSLSCNweGzNiGx+hu9gcdgn6bBl2AK+1kBHRySdeJk2Uw+2iQB7P20BAaajq5d9X3KFwsWIN9hxp3kUo92z3a6S/Zx3Kb+ULrtJchoxwpqw4k9kIa3CE97iteM1GZLcz44WFcxoeDddnJ1McM9T20XRjBa5oQZmaF2R13liyPlUNbCSiDOJPUWl0QQHDKof2nhTRcXsDEHwYLfcmaGZTUzqA3/D76g/fhKE+U/PZY9W7qHeWsm99OZnAzglnHDUWiTK5k0y/vEpfTj7Oll0TVAck1UfZXEFm7F8eaAH5cBpuRgvAaSxuz68K4nh3PiEe22lkOA5THNGAs9eHhs8E8Oz5dEgszFKEfeY3rXd50ew5eE+v1XuSCRjLDj6AQB/u5Omt8aTWF4g30NGjQCFId/HDhHJ9Gw5alhxhhPsVOmZQW+bpjKT4tAWQ4d9Hj3GP3rV0Yc4k7hl9lwplnmKT5D9a3gpy8g8PRDy0lIXE3tmBfZNdfVMtG4LR8M2VugwCRt+E6rw5t4IfQBu4MgqPaMMl29W4OOE45wA53AZfKxUQ4REpIglvyEzn0TRZ36VV4TD0pPSSziaV376Pw9n28qAmgyKjFGpwSBIHTo06TVB9IaP4QXG1IOwAOgok+YHvdSLxDptnFT+8YjbzVjSiXkzS3uXKj8Ppcczh5/WU4ZJ0hPS2XXvFpyXYfh5SjntNB6sFFXG6QElBMCneWNUBTQQCJG8u5PtfAUquzPl5xhH1ztuJzvRtx72hkfi6S7QVBoNu5m7fcRbLUmyiyccgNC5zNR2crcRtWSEn7nSxVhkribb3HKD+RhAx42qsfwdQLuCDzncIzu9bj0lzM5Gunacy6IrmeoiDQ5dKFWufG7IsmptiwlEyCO/35UciNCjySupGZu4HBGkik+jz9vp283Q4ftUOnU9jgxppg/mrcSRsbWOhopkVlb9HcU1jLgl1+xM8KpyZQmgOf16Tz29kCknJVnJs5hCzTYW6omEUTPPTNQwjxxcy79xAlBmkOsMzlNAtnlPDA6ic4vPlNntonzV1W+Vwiqd+Rbx98lLZEN76yYmGLFgPRG2/h6slkln+/iu/6pN0sNdpy1i4/zha9O32djsgt9qqW1aOiUAz7kQUuzmT6r5PEDA7+nH7wJ4qNZrr7bpHMN9q0Zbi5t3I64Gn8XXowy++RbFuVF0BNrpre4dcwa6TfeEGA6aUlKDcsJg0IDpWuah0sPSSrwLM4Elm7K4IgJV2KgoDSQc8X925kb+dxsOk0a3II5W2xDY/rUUx10aPUV2HdQzRdls/wfh++NfXTqLBSMJu7meMINXUehB4bTVmcjVpX5s/Xzz9GV8J1fnzxfaZ6SHtw5jg8zWa0qDIiCXLrQS9I86YfgzQ4iUbu9/8FB10CnIPyITnExRgQPoFRuVlouYB/txOi0b5nn1HpjHdUK1WNShwjq5FbBklcJoULn1q6Ke6IYNTpUewOsyY9iJRGlZJxMY3KjbcQ8ajUrelD1W0cTStiietwzje54B1kpRhsPk3+zGKebC7mUCcobQBQACdzH70/LiGg2p/0zz+nwXI/N8C+MIeDzNOV8I02k6vxF6nR/cAjDDYOFhBo9G0ktCqUgt4gQizSFY3JpMbnwjGiWkbQ7dyFTBx4FkQR3JJLKRt+kPYTI3na4oyH3urDYrFwy5V0gk+P4OowFb6B0m+GUXBm04nHKDjuxOkxx3H1sQHhBQGzkw4h8wqNcj3b1EOxnknVuJJ5PpOA+gCKhksdOwBy1ON58/nl3GpJJ7luODLZoPFvj8yJTr2cETGtvJb0AD9XRwDjAfhN9RIfme/nLq9mSqJKqDEXgJVPSUvnS+SuPcq5iaW4+mp5NkItyVO/SjlMX0oPSVWgagrgUatLolQ6UDNkEq4NB3GYdgJPs9QFok0RRu7iL9Gvn8XHrz9E+EdSW2uAO33vomhiCU7GhwhX+NrFPVs9kFcE80KTgIfPIOHCN6Gao8/swS93CEOa/ag3WdUYLBbuO70dTWEZtyzeQFLiRmCQyCQ3dzAn+ncuNDdx/6dzaU9uZ9ODg5sfNoTQaqzA1VhEkbMrO5HOz3c4t+DuKFBVEMklmb24IdhUy28hBh4sdaTfICDaOJ2IrWvYueRhtjdFs7kijomTBuoyDq17METBPY1aNjz7BccsroyTvLMy9BYQKhL4cds0hjwcSoMRztfOISW2lpVZI3jXZxqnGuWs7QrBOMwbCj6D4q9RmAYBy1eyxzDGtwfBqv1KQ9MFho9/kpqyBQTVBmFqs6+h/V8atnjH/+T4L9j33/G/NmQyGZGRUoaYQ4Ufow8PJ8VDy+SZ5yQxURTJMUBAtwvvdQbS6i8tQIqCwJ6Ze5i/ZT6Tt9dRvOwSWHlrq9AxycnC8/fuQN+XA7xqd0zh005Rfy6J2apLePVfBiutjygIuHa4kvH5IzilFGG5ZdADu6vQgqrwOlEPbuDp0B7a+9uBAcsfQYB5t/9CUZsjGpMclUW68sgWg6leuplsZTc17n9T1zcR64LtxNhiUpOy+XmvE6KTvRrAue0EDpfcMKl1yJwMdvEe0YsXEs6RPf0cD4j2Np+17g54D68gfEgD8WFTgdM3j7vDbwi9Zgu3bL+FrnHSZHh2WAuO88tpOTuMzCPjsPxlkyy7zyNgUjmbeysojZKyOgVBwFluQdHmRnx+KNdGSu0XXbuvcS0UflxwkHozeIlSyTrA014lKHHEt7cXucy+oHCf1xnGZBlIys3kqF5z08ZI/Id7Iv/qOd6udmTRSeni4/7oRtrCatnt2IfBxWRlgAHIlPjsH49fox9H3viAUBvAIlio5GlfM9P6YIwD7OwvxzoJcus/yW2hB6nYJdAZZ2+Zo9TVMWpkPieGlhJ1ToeTTc4qow//ilO419+Bk2o9QW5WTabrCvHOv4IBH1ZXuzNWHASrA3U5/HL4Fg67N/BF8nmMgvRe7Wj6gvMxP/D2N4/hqTAhmyJVWcQ1j8d8RUsdYbTVPMrtUtIzy9wqaD88Aod9DZgSpQtYgOGmk+Sbx1MTXcL3XnqJ1RDaGnqj+/iyY8AiQy0MguF1sgiMpZBQGEHYxVTKF0mToEeM2Vz75R6OjT3GkYlH7FQp67sc6VXUcPZfK3HRp0qUwCaVBxsirnNF7smw0mgckqRMKwdzJ6levWxz6eLZOgU/qqQ+/QBza0LYt24aSyfuY0LWXKuOJGBWODNk0y2kXU5DruzHZ/Pgomy+pYuKyji0S6u5d9hPKHv8uGFjKze2s3rffKpyI1n52js4GaWFCDdzLe8dnU7r9XBKYo0Ee0nVhqJBYMSZEXzm38CrkRVsSJQWP809WqLOb+Obw/fjNcebRx6WhKlVJXPc8xxD3NvZOL6cXG0OMO5mfK82ma0FF/E77U9BchaicqACKiAwdf9UzDKR0c3vY3JWIlj17d5SOJpzvw2ncfRJliZfZXmQNEkfm/YHlugSdnyWTH9OPDwjBdpn33qI8hYVi4+OpD1qAxKwz8GfkKNRzDsyktkxjRjmdGA7frYkoJu1l0XxdYz0lM6DooMPG5N0iEHb2VLgyvyAef8sa2C0shn1Iyc4WzqB2Z4baOsxghXP/NfKHeTmwltvTSHaIxuQLq473Tppiy4lvseXZt+DWIN9/u4ZfPHoa3jqa7nr88/5Q/mSZNtuRSi59W6khtZzz4sbwK7MDcFVwWT0eXH/tKs0G6VqsihZHw/OKaalo4sdJe9KunPleH5De4aejD/U/Du2kG9eGnxOBJmSoQ82oM1Rock2Mt1DWuX2dH+VtV8uQ9bxIWrPZkJEqdprlEsriaoefsz3Qq61n+cKakMo2zyMXqceav1sO89Aj+CNR7HALEdIUtsXID0LHXnil2fZe+9mHIZrb87tvZpRbMgbRXfQW2y8/Wf+6JL27Hs+9hmOlcxjuyaHHLf7yO98kGFW97JAv40eXTYLdy/kfKYUhBQsJtLVBtwbfXm1NpbKRKkiUZBr2D3hNBcmHcHHuY/bRSnLFuBDn2KS1dBjhlPmPSDpCAvv+bYxwwWilT9RrB3PDbBvrvtcOh9t5K38OfSoC+g1BmBtF/hgiTeCUUunaycKszPLrfZpVPlx6fAwAuoDeK9tGcFTkrEur5dZIsh+I47wHkd8QxpwniHNqxDg5JiT/BoGHUZ70EulLebj0WUUeP1OsH8bLaYWYPBZquo9Q3vYi3jLwNliRiYKREZGUtvRiEqnwrnbmbYOL5qbNJL5M9yYTdK0C/xp1PFtynkmOT5lc1gDc/3pYdnIjXJGitJCXrtLC5WhAsecW+nHfpF5R+ZD/Hl4Lrt+noVpSB623SRCtR/z0ZRvCMuLpLdzDvfxmSS+tM0TTa8T2l6Qi4M5nSAILP95OX46DePTSjmTIO3DMsmrDM1kE5U+HrjrY3B0kn5XSmNruZRwhIvBcLXPZkGPCR8Z9Jqh1zJg1yi5Jv98/u726GCBB/Qa7d+d2+tdcJE3k5Y0B33fcawLjAq5ErVqOTmx94AJkA0mILOCNuIQ0sdtkyNx7UlgsbaBwH9Iae26NgoDB5TQDZ7g0S8FlD1Mtbz96dec11t4UG5hi1nKUDe5CPy+7Hdm7ZpFQZ/AdJt0bkfXb5xVvE9/rTtKF/sctqHuL16OWIn7Wy/QMP4sWMxYN1yJNecw8swErrk102vl1qBVh1N6IY7kq8m8PfMojj0BrLXa7/miMBrefoWmyYeoir2fgt7lYEVd+LcsjNNT1vDU148yvn24XQFmnUFFXk4KOu8mgv3qkIB9Mgd+zn6OjvUWVjAK9chEu/M6bklm/SaR0SXBdN0mXa4LAizdnMXsywm0erTySd9cblDDPLTHmNh3jkteXbwz5QzbTNJr9vLeySRsS2HEiovcPTybTpnSar8CwbXxKPXXGPFmBTpnKRjXr/CjbVgCelMxXz/yAo6rpDerX+bFwUVbqE4+zzdnfClZbJ+bv18XxorDI2jPEDEqpcW4oW7hpN//HjVdcZS3P4YgDOT+gtnA4WvjOHMmnnWpL7M+91vWW10SXycfeiI+YvuwP7gcsYxQ42SJoqUVL/7911gaVH2YJ2ejE6U56u6yWBw+ncjj047iM/2kpPcuwNm+EArq6vA5UU5R8jeANHHqk/vw1/wtpJREo8vLYLPNOZckWXD5/G1+z3ududq3bv7dxVjNY26wXwueZZAsH4a1oesRQywvdxyAoKMQdJSU3m+xtrI/3evDWHk/PgFNTB0pVXsJgoXZfe707Usnf4gcY4xUPZrl4sGv3Ivl4V+5y62fPyyrsB3veF1hqiMMPbCclRHz7eJ57vMIn7WG9xee52BfJDfAviM1wXheCeRydDF7Iiv4RPSWbHfWpxW9Xwn70rORm5z51tpSWOFE3olkIsoiMChaqDRWgZVGTlX0JcXDt/HMhofp75eRJ3XD5C6//XyBGzueeoqmMZcRlkrjggDeldk4t1ZSnLkMweZebzHP5NyEtTiZZSTXyyhwGDh2hUxOREUikZeuA24slkkBHg9jGVlL93H+ShqZp/6NY1I61qopGWYqkvIgtp7XU2op0B2WxMc4vkyUQsT8zuvkZFQjf8XqARdEzOFxdDkL/LhmDtmvtmNbvg+tWYJ36VR6flpMyMoKSUwUFeyYvZN4zx6OGeSckl3jBtgXL1yn26Kk68wwnr2URN3dbpJty4y+/NoFmhJnNJ32ZM+aLlfKfhmPzCTjiv8hJBYUCle+dWzCJ6oE1zILs9xTJAYVJodQHmmCU9O3434pjy+8wu32P03+KSvln3PcBH+L0p5/FrkLm5SdJKvdubt1BaIwaL9brJvPqTLY5/Iif+lgr1wKbH2VbyHzVAB3pnjQ4yQFFUQRtlwLwQ2Yfe/fNAXGcENJBpCjmMgv72npjC2gavRZ5vCgZPvgwCd5+jMn7k5egy5aWt8A8PdOo/yQAfO08xT7RDPbVTr/hwr15H57O3PcOml9ZNAKX62r5l9FqWzbMRKxwYfHYo9KtksJuYMj0UeoCm5iupcOX7mUrLmt400OpHwKetjZCB9GScmLZ2vvp6zyLH9k/cAhQz3fMtBfELkDoggODW4sXzfgVlC0vMGOFPGDOY36CTksTnqdDZ5Qq6sEBvLwIQEjefv8DhrNlxi68jN+8zOASQcyNRFiETsD4DNdKeuc1+EoGyuhS5zr/YtS//1cOv4tVd0vsPDlwdxeUHmxsxcqtXIWt/kQ4Gq2O64vPO5BXaNGBKJqnHnRyuGisG8m63O3ENXryLIRWxnuuRtugH26Fjb66Ph+ymj2BDyO0uTBXBvygMJQg0tLOfXh5ahHXURppYRcHdiKTFSRnlpEo2Mrjzl9fJO2IIqwcmouxwLdGBIWhMpmXdovuLO6Opbhukq6XLpwVdspIBhZHERfzGWIucxS1+WS8DL3UgxpnRTtCuBdF5uewYDsnxzpreACVL7PIQiDYJ+3pYfxeUNhWxrfLBhF31ArJwdBICM7A+8Wb7xbvKlPs1krRWVSOCKU0JIveOXEKxjWSEms6zvCaW6tZdKxVEqj7HPUlpA0ggoOMeLlNwh6/pAk1qEM5b12WNIRiKw5mVTXpXbbj3Z/Ex9lP47NVfgpxoFUF86rT27m6q8zOffBg/DJYP6x2uk6F+UwKaaAQzEFTO0YdN0QLAYejnuELYcXcblkPHXqVMl3WmFqJKnxTzpyh6G7mkq9r3SezLec5K8nVvLU2uV8+doD+Pwk/RbPV4Vx7Zcgvpuxh5yUYrvn1y9kEW+vciFwUxVfjrmCZk4l1tbTZf1G8rrhx04TmgvbmfKPOFVQ+/Hn2WhcTg6lcswR8lzkEoDeEnwbG+drCGqTc51urgWVsv9oBNqeek4ccSU5WYmq2cjwS9Mpx4JxQwSoBnJos894ntavo6EwlPa/J7EjupynrWqL8v4uHhtax8pnf+bavipGxtsT8f6vjP+Ed/xPjv+Cff8d/2vDbDbT1NSEj4/PTeunAL1IaFUo7VWww6Ub63KwKAoMqQlm1KlRGK4PIfbBU5L9OZra+NwLatMLaPJtJ0YWK4lrTM08f2wKP+sV5Hl/ww+2fpfANJpIffdfhPc8hUEhLVK69Zfy4r45GIB/R1/lE+dBWZal4xC+lc3sbVFT5tDDW4rB4pIoCngnlHLtSAp/PfAaZ+6o5B4rl4BmhQ8/x1zjuQ23s7goEqdR0o/flu5PeL2oBpnhVnw6Y2y0YpDbuoKaoo3MuWMvwYn2PRVEQWDioYn8FtSKf7qKl2zin6iauDJjIz/7Qo/R72Y3iNbWYwxL/4Rr5/pIujya9ixp7yXH0OcpqM+htfEUnrX9yJECjZfFFzngnkvQzCSWCDZ++HJnRl70JELdzg/fvIu3SdpkV+cQzJt/jKHvdBrfrPiR92LsQc6TnW9xvPlrvgrciklt3ydOQODIxCMcmXiEZ4w+3ABul7hkM2vN7fSWONHiH0+/DavzCVcNNToVq2MKUZmk1kt+jn40xmTSPvwvzoXAr90218TzVR74diLq3l/Ij67Bd6G0oOBqOIHmYj6QwISZu0DSQh3U2mLW+MJ3HUbism5D1v4EOA8uBi1tjzNhcgW7a2JJcsxCYXIF3KCnnDvmf8ThIaNZFbkamcJVwtrXKX24fCwFJwcddzZ4Yh4vvVdhHT08dmg6LQ69VEWXESGTfpR9MuQUi2NpqAnAMcTGjgo4dXU7nX8ewg0jpX1vAR9J4nkeAWi/WMmGTlCape8lopqteUE07x7JtLRSPEcNguiiIPDBSx+QUBrJ1CMTcVVKn2+Zq4GYpbvpKPdm7vUM5DN6JfH4q9HE5I9jx5wd9KilC3Oz0oOVbfDA5WRSDo3FbaVUCdHsPIZ5wc/zkCmT2/+8HfFe+6LVc5WujK7zIfVyKuIoaSKutBiJKxo417akfLwUg/am50dWIh97CHnT7SgV7eiNg8ctyuQ4JBXRr9SyZP0SasKlSbqzpQWXokhMgsiQ2e74REj7tN2h1HNl38BzdTnlMsI86Xnnt7yJW5MTFuBIVZGVQejAEP4hTRQpQFEyirtTpGBiQXMoHoeU1KfqMPkM9rGYpNbj9VgF6/ffisehK4xccJYTwuA5C4BgEUi8Pob4sgfIfF6qWg3XHMXP9RwvjI7ELDPxlE22e7+bnuONnpw9NpIItwvYDp9GHzxLIjlTEknNeFtePVQpXTiWsZUng8AsSJMujULDwubLsOsl3NosaCcO/naagyPxXrVc+XUM316MIvmriZL+R2JLE14VJbyTMxa560h+vtXMDTMLwWzgF1cll+RtvB+6ETfzLMn1Nqv8qKqOQqnu4qC6A4Mofe9i3Yby7chbGZv2JGd7vFjwH5R9d2QvRnHNhWSnZvZNloJuSrkK9/A+Dq8eTn3xBJasHIzduLzb526n07VTwoiu0LYwuvsYQVGOlM+qJ7vrCemPms3Env2VgiHww/JfeEsurbY9Up1CS1sud20rRh+4HSQwDpxS+mOeeAg005gfYK+S8FT68+TKAQPDhDd22MWdvTvo9RbZciGKzmjfm+QBQQCvyosIvXKWRoNoDpOQNYJ0P/Jq1uf0/zoPTZEXXWPOYb0QFBp+YfLuEXiklPDMJBvmvKmXl3bO49rpoRwBmp1LJM/YhIAR3FNzgt3Bd5Pdf5FF/4FYc6BrDru1f5Cqgm6l/TzqtXUsfxxN5+rUQ9zygvNNe5s8gyPy6C7MTfcwqs+Fa/W3kWVl/zju6AjSLg4lNyGX4lgpCCmKApsWbSI5J4Vxx8ZSEyOdD0qEOC4OX0dySQz1WRfwkEsLupliEaluMh4+70ev2oETSIfeZSjzGiw4uVXjZ3TnUbn0vDx135ObfIJynQPh6hrMuhaaOs3ITGW8fWYS2mOZXJ7yDC5+0m9ltyyA8gnHudAF6AQ7J4cR3U3MODCH3dH55ESUItiQqJaMv4DL+HZ2F7rh2Gu/yFzXn0j50CuYzPFUG8Ps4sd7Qsn7bg4PXIkj8tEjdvFXRuWz07+JB+SNTNU9e7MPs7upkWCZgKzRh7N7fOj0ljLvN8tD6Zy6i9B/v0XFtuk8cEF6vb71b0CvU3N960i0Nmp0J0Ml+2qS2f7TbCxGGfWvSEk5ipZ9vOD0DodffJgczw5it9r3K3nYVECTiye/+lVTqD2DNdiHsY9En7PIc1NodepGZsXSfboiCm1fNY4pXzHT3Uhd82+kRg8QLuTGLi73hXO4yZH6MHfa5FLrJcEpnTfueBZFmyNBmbmwVPqtTHUsYcaoEt6K+4IqI7zBF5K4aKnlkW8GZs6TU+17gHapQvlswxjUvRoeDO6wi1eZY8ndOpDn3f/54L5FQeDQpEOcG34OXZ0LagdpwdfiDNcSr5FSGMu4qyr6os5gDfbdIBdFP7SBxU7QaJL2r9t1JRrxZBrF9x/BNXi8FFAWFXzo+Ru3JKQRJwSQ5GIj1wXqzL6Ebh0Ag0fduVESEwQwykzU+dexfe52RorLb8ac2nfRdcgRxRgNabEw0T9DQhMp8zLQOfI02e2LOLJtNV/NsVJGWczMeWAneZ2RPFc6h/hQH4lVfV1/D89GPc+w9mEkGdIIVklVWcid+b7PyLN/LSTAvR3xM/uP1pTyl7g+ZB3nM8/bKf903guY/7ecZ6KeYXjE1ZtgH4KMAp2Irl9GhfdqRNEXg+lhNP84kQi6WrbdGsKfVVO5sx9spgvC/d/kxIE1dLrUc1dELs+kBEviB5QuOEw+wJR9U/Bs9QHpraQrdyrea4cRD2jDD2ML9gmCgGOvIzGXUzk0REpuBDilD6f4eiBpV0Qs4zu50Zdd7f84jz3pT3pRE5b0C3RkSRl+TnVy7vnrHkqyzuGano+jRaq6avPX8vHzAz0pXrTIpN2kBAUL9sQx4cBoGvZBz91DmWCVgDgZGhilP8wvO6aRnnmAO9PtgRbR/Bp/lx/GpS+JHlMrIJ2vXLzuYq3/Kn6p6eeTkJSbf3dtdWLS4Qx6HXtp8G+wI+K5ixa0IjzsCoJZ2s/PReXCuBMzkQndPPHxV2zrlpIHOpT+fPbtbHxjKlm68AQrC0ZhDcQYkXFu7cDb9rlzC/fa1Mhl/Y3EKQ7S3SEjwv0MoiUVBg0zEQWB4+MGoDR5vzOIA6UyUVvGmIy/yXEOojbrWWo9zBI/D3djOW94wqs9DlgK2/Dv3YE1mOcnnKNmWAcfrx7Dq9V+ZIX3SdyHvq9PRva1PwpgjC4JSYsjk4FO5dt0B16lT+mIRfgPhdMhV1Fl+NKXncz5jtEStxu1sYMM535kR7PY1Z2Lxgr4dZTrWfDRdzyxNxHXYyNB5ibZ7bnOOOJeTqF5aC71t1lYqFtGGIOq859b4pmdH0n+iHO0ZtqsxQUZezPPEufUycLNCzHPlxb+Y4PHoT1WTWJNG72eETgopCQ/gKC+ndxWlM7+oHK7IrhGoSG0fjxLNqSTNeEtBGEsN/Lv3PZf6DZVE1Pgj1bjapdDnB1xjjOpO3nyyycZltEM/5Lu+8vFmwhSGlA0fUmMp1TNJTDQl9TZpZ/nokwct1Ekejh4knW0j+Kjd3Jo+nlWSN26EQSIuLQJt4ZCzsx+g2dtTnuHaQJuii6cGr15w6nk5t/NmhDWXYjEucGHsMdA52Vjqy4TqIudiEPoar42+LFXJb0fw3pryfz+IcRaP7qdulH8IL0mheeisewu442jS6kdPUBUugEOiCJEiAO1Aw/fVka5i3bKvmqnzZQ6r2VPLxi7k3hqStjNmLPSmaQdlcSZ1fSP7aBfqebGRO1MF1cefZWk+EIOzd9EvF66Plyk7WDq1sW4XG2gz3E/gmBF5HOL592/hzHmxBhcu1wZectxu+dEEATW3r0WpUFJr9KMYEUG6jKHsSEqD/XTpbwboqXGGMvNmdCsx7H1NJGqdJQmD8n1uDFkPSt47JMc/ugK5NtoPdeFBiAGQYBtuz/HXNaFX9pxLoduoVchnT93db5AnVs0z8bdQZMhCRhca4kinE15k6r0Lm5VKTBbpARZQYCRF+5izCHYvngjykTpu+WuqiZhygnChzTjGGDTbxgYL6tg5cqV/K400PBkE2usYi9od5Ed78uJbVB0XIsmYXC+CDEXk9A58P8jZpxBHyT93mnVvTQEmHF3qMFNo8JFkBJ6NlbFk7VuDGOafVgcsldyrwSLgce95nPin3nzQkOYDSVSIKI0gia/LrRhD0har9wY13SPcj3nJM5UEqb+FVuwL7tPTidmTDKTJP/4uiaeimwVngHltHq1Yp1ECDI5hW4zUd8fTsf35RRci2OB1XxRZjBy9O/xdLq18Msr7xEmm4j18BfbuMsZribmYup2ZLpsnCR+qv9bKk99ydxMM4/727dbsjgE0rO/D0egJLycULmbJN6siOfBBnDuc2Acg+t7i0cm3xV6M/VKEqNEA7lDKyXXO803g5oxpRzrOsJPx39kwhYFr+p0+AENXTrWNAucMR1mIjK6p/QwgYdg6EoYuhJzdzl3toTw95Zb2D1jN4WxhTwrGxRm9DtF41EKBqOZCY4eiPZT+/+Z8Z/wjv/J8V+w77/jf21YLBYaGhrw9h5k/L0yroS0qUeIOxFAr6OBF63+vSgIrLySRcH1IRxdsglH10xJw3s3wUjSwQmkyE2MW36Un0VpEdEg9yb7bBzyVg8qx74MEpfpgeHXPY58/Rk+aArhB7mU8eRobmLmuKv8HFbJ2YRcsGLDPrniBRqXWti/K4WDtV68/ZA0EU+rhhBFG+lDc9C5SmM3FkFOcWUk+/bQbZGCTxHNM+nffZjzwz6m2l8OrJbED7vv5cqTX+Ec2EOVXvqBAXAytzP2xFgAWrxfsYsLCKh0Kh4rlRPgPNgLy9B2kIzTJow5cbz0/SquGqUm0q3iaHYXj2b4LV/zWvB1NtmYMwoD1X3uc4Ekpb196OiTo0m9koo5/T07ZqVeE8yBfiPT2t0Zc2IMdlRvoFI/iZOtR1idsJUTplq7+GuFQ5CfG0r/8o24Mhgfqq6jyBjG9bB+mhIX2yWFX/74O8LZi6Q88A09vtIE3g0de+5+lD9Ojue1jxfRO0WJtbbPySEGl2OncG4bT+7FFpxvk/Yn2Kj15JKmjUkJpfiE2dts6V1SeOnxJ3Bo82D8s38g9lVhvVgsM3vTkmfGpcGFN59Np791M7gsAJOemJj97PA4w4wLBwlzdQGL5eYXX+6SwZpH/mZqeTuRu6fiNFzKrE/XNlN/YgxXZ+xjy/AzLBSkFrbTAx5hcUApP1z+EEdHF5BwmqDDFEun1z7umr+LBr9+bMG+6/+PvbcMj+r83v0/e3ySibu7kQQCBA3uXlza0pZSb6m31EvdS72lDi2laIu7u4WEhAhJiLvr+Mz/RYCZPfM9/3fnd851ne9zXX3RLPaeLc9+nrXWfa97WaLY0iRhtkJOsEuXyIbKl6OZbxOUXcHJ7GSOx5felnxNMOdyKAQ2uZfw+ZQSfuwQO23fmkdzMP4L3j/1BNbrcYS9LnYqn5dYyb8Rw5gbcTSFOVSeIjBr+ywCs/vheve/NGiGiey3WG/Gfnn0MYURJnduBH05pJYRQIJPF/4y8TNNs5YzZs0nPFCWyK+mAr6U2mCHbR2BVEmLGXG+kssn4liarOMWtcAq92RJzGVcAwp44bPnGeMvZs01yFP48LGVTJeG8lp4FaesIdhXAkf6RBP6/Aa2rhlOeKP/zc6Wdtflf5I7fsxm+HUl5YIvOFSsTNGoOSBfzIn302jxTcH3ETHYJzVKCK8IRx5/mocW+SMYSkDlh8TQRnLyCWRlD+Gi8aRMM1L0bXlHNlO28k8i1j+Hf1YTyaZqwCbFMjvHjGd+KksGFXFnQhN6sziRPf7AUNwaJLi++BVmxQBxLs5swD/tOsfcm6F+LosjxFKDAHLBQnJRHM+f9SFi2mxEoWRrFk963Mdmyzj04y7S3xbqUa56hs/eGY1Ov42qsWcpNZiZbQ8QNT9JZG4SmfJMzg+8jMX6BreSDSqpHJe/78D7fDKZb64ixYH0YLFayB++iCthP/FDvidPDxPL9vlYm/kw9QfWZao4Jm9x+Op6x8yMixzIGU9QfgISqfi7Nbn2YeX+U6SVfIJSKpYRmu2+hOZWF7YOymeYmwHBagPSLVYL/vUaJAY3Fl4tJH2wXBQu+Zv2EDerji79RB4uuY/RY5JE5/as8Gbe708AZvoH/4Ij2GfUWMgfc5KHDf74ya5iPw8Aot1iuBXu7NHF4vg2FyVUoYvT4XqmH+eW2NaTjppVJFR005Ov5lrHW/iOF1f25eWP5a+3NCg1WjzculFZxOSAx/rk0TO/G12EK27eDgQTiZJMZRe6sCr+nL6TON/ZIrO8/SwfZczF86+pZFt8kaQ50H8Bq3E9qyuaafTYzxs+GeK+k8Baq5ThgFDjj8mOcPFJqw+nDIW8kr0UTWUIng+KmeIzkl2pNjWQkp1CjIe4+lluaOHdxBYOe15gXUQZ4QHVvIytObyAwKmRp7g6sJMBJZtZ7i1OgqdKSxGOjEG7YySXBonXOACUPnj8OZeEwgQyp7+FIH7V1BPGV23Q0xPEUKYxUqKiru4Gcd4dTOtXxj89sVgkcqd9WO05ma93vkPfZni0uQHvyWJQIUzbQ+PpDB4ObmZJTAknpGIiyACVDp92DU9+/SQJ48qcLvvjjkoahl2FYSeYonreyX6gppOo0wN7O/oapU72xd1SqnzL0BpsagEA470iuPudZ3ju2nh+1l5koJtYYFzDA1wpGkKoPAiDSuN0337lyRQdiqQ+M4HMIWIyhlnmye4j/cEoJyk9nwA3sX9S1N3MrbRgBQGM8h2L45jg8y25cni0AR5JSBbZTN3l9M/5HuWF2cT2LUKlL+NW9Yf0fAKTD8yj46mfeS66iiNmm/y5RCLj+M4MOvNiCQtoIbWfOLSM9R1DZstxAKoupNA4VSy7Fymr5y41GAGdBaeqqrtdj6EbHUrW8f48ENngdE96eTANDZ4EuHXycUwR6x2Sus2KUM5n/E5cTSjPtQy+3YvTt+cCJ1I7+OBEIEk/PcShWZdFxwUH1uPzwFbUB0cSmJ0KErGfmdDVwcD9d3AuNZ/ytOsslYqBmh/iK5CGrKHJt4kwkzOY1+nRQ8f9m3gyEC4qRjvZEQSupF1hanQjdR4KRxPpGTncyDjLKAkkymwVpBuVOmof2AJXExn85cN0viIG0q0RzWSHXuSeT0bT5Z+LILGLdaxWJsX8QldpKtoqI1ntUYA4rhAsAlf6XyNQuMj94qUfd4UX/Qtep61PFrq4G07vEmBwZxa1BUl8fN8RDO3HcKTmDDqaR/a/S3l3yQXO3Mr0SWQ8ln4MBh3lhS+fQht1AoP5Xm61HbBKFPy79y6KC31Y3qeCtnBxzOGh8KJw+MN4X3+LlR+vxOQQ/s0I/YjdVYXk9F/LkYBaJ1JlgnUy3ZoN3L18IyeinNd2hbWb8SkVNMT+jEU93NneJWX8kXFE3xBIGfUh0FtF56ryQyvrD+wn/XI66v5iwDnVWoekNZTwzXORbp5HwHdiMsdHfu0EyT0YXteOI8IpCBJc2m3E0yzFNuzVavAfw5rt7+OSV0hw5zgixoh9PYAC46P83PkXz4wZjr77Gez7IAJMjfyEwVWuvC+EwEjb/P9lyCnOh1xm3LbxLLqQgc9P4l6y6/L7cGn/UKKG5hCQKu7P5S+YeOyJnVxsD6bRDFqreP2tcOtP7WUNcn8tzSY9bobBIvumxkm0x/yEn3sDmQNyeNAht9bakcOdi3/kjw+XMkXyDF3mXwBbZVUgVdxbHk9trQ+K4Ad6fSNBitCVz4OjXualgKn8K2lH1R4gSpDXqUcwaV0flA1yIl/Zywnv8SKg3CSNYFfZNDouplIfXo5BKq4022wqQ3XHeZ4MimTZyH/BeCfIbpKG9E28oviGnwKSODToHBGyMTiOu1qSKB+3F/nIQ4xTiGOwwLYL7Na48diif9gbUMFKq430kCUZzL3Fv+PfJ5dBfToI0zggUzKBkpgSJiRWsyT9IpfNn4Md2KfXmHnrjbeoiLZSbRRndAWsRMqgT5svSQVJ1BrEfr1EkOBZVoJX1jHMSUEIlqXYA68AHuc3kHhsJoMTb+A54ypg65sdIzXyxMZ0uts0nL8xhVg7ECejah0Ne0cDC1CE1CNf0SY67wL/VvTWDrRJSsrdxZqrvflYgQoTvOr9JPPbuplvR5WeGpjKlifaKFWc4d3MCMYki9d+ubES5ZQe9PtcUGmdZSWFjlwmZmwk6+IQJEY9giD+N1ZBzu5pu5lQPIbN5Z/fhoDC/PpTFPAnwiQLl0pVeLQKIsBZoS/i1ZHPs3OvBz9++witK9tF501XlFFh8qdJZiQotRhPaQ3YaT1cCt0NA7IZmDmQLj8r+NmAA0GA78ceoXPsbr68/gq6kseY6PBtjTM3sWT3bIzX0qiJn43PizYyqlVbz6Axp8hvCufTYk9+9Iik4uaeeY0hVEVvp9q3jikuoMahfYRQx+ar0+iOriNmTBY60yRuVQxKBAkxPf3x6PCgaOh5svwa+Mph2+mrK+Bqn3Ky9DDU6o9SalMgEpBgUBqYt3Uex+VGDt6faosPXUJgsY69A88zMPMtEh7JwUeYC3Y0wmyiOcRJrjeMQNL4IKPjbXFHcVEfwnP3Mat6Fhb1IpY8anOQJVKBXUUrULZd4cO/nqZjQjDf2BWIyq2t/BXcRdWBobT/O4GEVeIYThBgtr+GJmknT2QPZuhsMUi0p+ZDvs5+DYWugwizHx8jHgmSBg7hi9Sg4ESneP65pb6CvMeF4phzVIRXMFh4AuiNX0cbNjBwbitbrqZw+vAA8kYIoq557m2jOTIvj2cb4TM/+N0qjmvV3TL8G/3xH3aNmcPFa7/FasEUdh0YzobFG3CNEvtFwxRdvFKYQdWFGNJfewC14QlwyD3+XVdCRJuJG0MKyOsc4NRG4v2sSMZGlfDEXQe4JrMBqGfKYxi0LY3H/JsI8+4kc5YdiClIeb61E636V0Yl+1EWsRdBsKnStJt1rLnvR1zlZrqSDWxsF8dKidYy+l4ZyucBDWweeIlpwovi+1Z6cuKOBTwW9hexLc79m61YqInty/XAZ7kYnc1lhfiZ3gIt5/hf486AgVj1+8DND0GAs0PPkZ+Uz1NfPcV9vg5Vr+3XqE34my2rd3PCbGaIWRxH3WW2ch6YxmFGWVSEWm6AqRt09Uikap7zLcFw7x9Uq2NRC2OQSWy+vyCV4309hgmHJtARehatIQFwvrf/F8Z/wjv+J8d/wb7/jv+rRm37O2zuuERl+JdILGKHTyoVaBp9htbkHI5F5TFEKk7+xIcv5KtDDehdy/lu6AlGxomDJrPMnY8f+oGVH63Ev8NZ7hLg6cNT0F4dzc/LP0NnWQrYHNcG9VBmBz+LSwgkSxFVsv2rc6NRqGPckXGUxJQ4NLyHOdvmEFwbhO7l7/A0ixMwI01niFz9HNeS8lk1ZSffyMSUUkVbCQEV15lZEYIsw7m/kURei8ag4NTfS0ib+IyTXYWOqOQblF6LBrMJ+346ALOaWrnj817puU2v2vp3NWsG8W9XJmpB4NVTx0jq7y9K+HbdKCfu3GE6tW/zZ3o+afPEFQHKlhmM2zyIE5tX8c0dh9lqh4opBIG+fVWUeR9nzoVgEmIGOvBk4ezwswwri2HY1TR8zFU4jsvmtdS6nmX0yXd5b+ETTvaBgpRL5SEMvTCE9kSbNNlbTXM4tOhj5mvgGbePya/5jtRUmz57pzoCmaaI2d89TOKUM+KTylQcLbuLnEo5qiuRVKaLHWkJetpjk6npOMHFgacYLiwS2VutMjaPPsxmwK0uxl64BgAfr0S6FGGo6WZehYwzIbOxD51KSWHt+LXEIceldAQPjIvoLeT3SMS/RIHV2sHf1tUMCzoGgu2Zpfr1Z0D5LsqilhPy9qeUu4rTzJeDvel6/TNO3GTYOrJwV9cZMfSYeC71IXSCL45gnyzAk8OLOmmMvoKfdjFTEQ/BKDB171T6XhrE+Ef+wXH4JC3icvtnNHrsQS6xBd0udONVEIveoONJoQq9RAzAj3d7nK6LczifpMLN6MtMpThB7zftHI0Dsxjw+tP4TRAn8hSGOqa5QnloLS/EZrFAISYHJLr68cKV1UQbi1m64Hc6Zc85Xbe7Zxcb3nmLvAiBHEFcyWN2iWPn9Se5YTkBMkRsHvcWF+Jb4knLSkNulIvYZbeSY92abhK+f4sSnafovBaJkioTZOb58/IvzxH7UJxIp7/K9Ql+KHiChR9JWeph4RziNeG0KYl/tOf5MlFHic65ElhjLMXj+g1k3YmYA5VOyeg7o4+S9v0l/mxQUdojwfuWcxd1N8eK7uZcTxVl8ceQm71EzdsXuxmY6APvDTfzwKLvaXL1ALuZ4takYcKhUchcU+kKvEKwqxi4HXHhZVwLr/BB/8tEOlTAYWxnT0IT64OaOJ+VQLKXGGQHeMl6nlPre53z2KUrxUapmtzMgQi1Hjw9Ppdmqy3QtAoy/Isv0KOUsW3JCaI8R4kOnTviCM3BV+HbRUikFvG7lLlyUvIsgt9BZu+ZTtsAccK2uG4HT82cw4EfZ+N+6WlHAiIySxuR7deIWn8vukTnPoQAJxWPUL3oE/R+dUitBuzX91Z9Ey4V7yANNjLpuYWALdGiK26n7mAi9w2O4k7/QvR2SRKpVccjB6dhKQulMv44PZGpgI2EUtm5jjsXbcF65CohrQrcTJ9hH3B5B7RSNPY4Qf0LKYgUVzYBTHS5wfvNUeSuD0QxewOOYJ+CRt5cv4ovWuFq3Qqn46XK98gb9C5uQ35luKvtdw2GaqY9s4ev946n7/kYNOliCbrD0hKaElu4NHQH9dFlvKLZILJvUaowDT/J5cytRFfMZZP4R/lg5DEso3v3/SgH79nqGsMvvz2H4pLAIGCS+38Oalz0Ubj1pOKh8HKynRl2jVOjdqKRWrlDsOsZIkh45vNnaHfp4dTDv7BAIU643j3+X3L6pqG5Nw+9TJwYlVo6WaZQ4uVp5psJN9jaJWYWD5RcZG8wvFHsj0Yfj0MBNFtMYyCnhSCgIsG5YkUQBLRqLRKrhICSM9iTUwAqhQQ+aATfjigW6b9mpKx3zVEGjuG9qhP0BJq53/VuvP06gAO3jwtxiWLi3lR8aq9SafYhYFCx6Lw7fcO58soHPPjvo3ydN4Ipm8TA7lOnBxN81ZVJ/Rsh1DnACqrw5f6NL5Kw4CANQ/Od7BY3gX9m/0ODfwPhwaNZ5GA/pVUSUhnGkyYNLv42EMjqM5an913ipMfPdPgexioTL6CpPeNw3WWkc+hmFAMLaenJIMaup+q+C1+hydzN7mm7qRHfEkaFP59N3U7IVCkTo5sQFGIgsUkVyf2Jp3B5qZZxucX4OuNLjPD+BlcjvNAEyx1UDQxyDy65tqGWmbgx+SgZqtdv21qDWzmXcZrUnxfz7J96Ru616zOrdOfTYccIiM9nxslFtGmdpXu1rj6oupv4bdlvTA8Qg6t764dT8OZwVC46Zty7x6na66opFhZsZNT8TLq6HRLR9O6Zvy37DQB5hyfrnWj9Eg5NPMQhoK8dCVCKFZlOAWo9YZNP83SyWHopTqjlXj+4Z9xJMsecpK/rFyL7GCq4cWki5epuNgaWsVQqfuA9bjoEoYuRJ0eijRIDnJh6KI6qY2sXjD64jifuGOW4/DNXeozRz52hTzkkGx7hbgf7d8FdrN2Tjl9dEH2X28C+BrkXO0NLGV0SRoxeidEq3uc3h16lwwg/u1ej9atHbxyN660EuyAh+owf8Vf9WHxyFMa5F0XHqo2VfLflbhpuhJD47CuojfeAXZdmD4mBZ0sqKLk4kBUPnkIqOAPlE2duRzKsiStSLUly8TqZW/0vmu5uQEaDu5i4ICBgxYqPXxv9g7rAYqtcrdB3sfOMhKgyT8IuL2T8i2IhTY3uLA+N/pVzbp24hNXgKRXP0XmuvzBrwCXSY0+A1RmgHDjoGIFD1jKnrRR1+x1OPcKV1m7cvlxOSmI5Ex8853T8+JoGtDf6MmjSObpCargF9knMHSjSPCiOe4CvxqRQpZwJdlXbifGlLP7hb5ZtScfveizI0kTnDe6OpeZoIKv2p1Ay/YDIJhOkKIbUU+h6nOjB10jwE1cBlXU3cXTADcZWSEg/3UGIi9i/vjW8ZEbudYc9pkpnY/ZFGv+dyR1RpSjutPkXBmUgVi81ah8pXnQRqBKvwTkmD3RyE4X7MthR544oKpbIwMeNyhZvvto6CEVCvIM4Onz+7OdEVc0hf89WXALFMs0SQeDKgCuMVMEcwbnXcYvMj3sbTYzya2QHPYySivfEdOtp2n+7kyjgjT8eAMNMUPlh9h7CoAqosh4i9WoqHV5iIp1J6o5XdgqJBYm8PeZtMoxpIrtWmsSu9auIMOzh3NBzzHAAZy0KyOqfxa6GBTRfXM7zi+x8BLk7nxUEIt82mQ8e3IGbj7gfMYCyW0l8TSia4AY8pGKwu90tmY/WjyH+/AC2vPy+aI1VSV0ZfH4sw7w9+WLaPxx0WL+fDEmm594bHO4q5f1r/kwZIL7uqW5VzDMoWPP7GPTh7QyxuS5ITD3sbkziOq5M+/kDvtGJ22bQepXls15iW9YUXMpKaevMAcTg7YHYAsIuhDNszBUEqwPlTaYmdHg9p+Wn+LbfJe6W2AA517g6Uu7ZQYlewuywduosi0WHPqwwYOj25PmU4SS4i/16gDMtMRz4bQadViu1D4hzRv49J3lr6ArefOozerolhK0QK9a0tR7ipaUfk7cQqpqc+4Z16NuInXiQS1v7sTj4bkztr6Pws/nBXkIHrv3z2ReYzhitje4mCGCRKTFbGjCa92K2emJPxihpOU3lWk8kJiuHxx0mwEXcZ3O7Sz+yn/6aGa7wnA/sc/AhFqX9gn//BpZcO4OrkErfUnFl37d1MdQpcynqjMZbG+JU5ZZBK2UXJyEDXNtrEQTbmtPSXc+Y2Tsp3jWOp798mr332NS5JIKEv5f8zeDMAUx5/RXiHhbH6++5h5L93iru84CHveBX44PcAvtkPVW8fp+aHxODWR+wF6UhyOmbn9V9BPOZkaSfT+GlN39nuyEP6CU3DNSso91HwfuN/tR5tGESxDe1JW8LyoprCIBfdRvuknPYg3351gROnUlk/tYBZCYPZsRDtvizNTyNtoBEEq/+iUdrHi43csH/7dvvUlXzCSmZPbS7t3NRcRrsehlLBT1TZBK+rfNFH6DGJVicgxAEeGz6C/xW+jiV5yKYFSRe36ukJ9B1f0fa2SS0cY8A4nlYG/gxRxauok72D4JcrDJFnxcoPGuGx57gC084VpcPNyHQ84p57G75hWigTdNFgV68h1/Pj+WtTxdSOmMPS8ZcICUwUmR/L6WDppfPsObaavbs68Nm+9I9iZzJ6jJYtYoJ2dUMc5B/DlJ5MHrGJQ56NhPSWE1R5y4cwb4GClEEtlOg7iDY7JwLGHJuIFG5CQQN/gncbL1g22I6WHvPWuZlDqaxLAKj1RavSyTQId+LZ6vAgDsvMcWzR1yRCLT4NmPRK/k5J5hSdYAoZsjTxVL7ZxrhgzPZlH6Ddos4p3mmawXxN7rJ3X4nh8LD+f418TWXFXzBN08/x0MNcLELpA4kKh9jEWv8IUUBw9WZtJt7139JVwGrA6xsUnQSvKQDpb+f6JutK9zLS59u54Tl1tfgPIYAezAz6mg30xfcgBtr4dLjyNI+46iyA6I6+KlpCX29L6OU2vIPgllPnEVBUF0QQXUHOJH0M6/wy//iV/47/neO/4J9/x3/V43s6tmUnI/j/VHnCA9oEdkkgsATbpVI3SBCBh6CWPoDiZzcjEcxtT6I7zf3YljtABZKhF72zs/voHbYzG+NQYkHOHF1JK8VDMRdfxX7fmtyqSs6bRgrNk1gTEo5mhk3gP4AfFkWiUFv4aHH/2BFQBeCsRVUvQGIIECMuw5pdw/fBMA/nWWi3+wRNEh9W+lyEVcZ3BqjB7xC4DdZbPhmNqHBzU72OxTnCGyN4nBmBG6DnBlkbdJQ3p78D5/NDCQ1qAj4U2RPtHRSCFhVOhZ42oI5k9yPnybvwbe5D+OurCQoQQyedjU9jp82mZpiOcerJ5O6RCwTkKCqIPNm+qLCIt6YlBKBtydv5Kd/ZhK59j5yHxLfu6Yzn+Oh0LFsHwMEBTqfv53u60m/D3g7uJApF5OwSJ178jycmMfYfgGwdxjcY59kl2EFIs6l06PzQHhU/NvtffOoCbQw/EwzN+Tie+pBzoYfB6NpreajN95inJsDyNjyMT899QZr2mEC4GIWS7IqMRLV6cHdq5/k8hDnBKNarqJoyHK+7X9LLkKcKBEEgQWbF6BX6nnxkTUsUYXcVu3OODWUcUdGkwfUD4zmiTfs7rj7Gg9HfMI2k5b3LF1MQ+wAVUq82Svt5MNTk+nrakIxQxxIXj/fhyE7ZvInUDEyi18eFJmZEZzMXJOOpccG0Joh7qkAMN6YTc2l3rnwjVQhAqcwdjIt+g061XXUup8jQWlLOOQqMrj4tT/eKh0XBl/AfaAYzEvtuoLxQCGlgRNoiI7G8bN+vOheyoRNPD9/D/k+YnZ7iKsvC5cf5tOOHmh1Bjg99XlMk27n+NExPCG9zDsznPW2j7hJuXg5nRXZTzF1fgb2rrbZPY2/Dj9PRk87T/gHYB1uS8Y9UCGjY+ednFj+OzGR9agtNgkiiUXH65kZtJeGsGTeJtxN3qLuogEKb6YV3YVntgFVcTun6vaz3M7ZbTXW0ams499OKDXBJIkY3K8w+nMlL4xlHgo8gtuc7qnd73627hBwcW3g9RceRtbyM/jZ1sEYxUQqzyqR/jiWp6df4tRDvTOwoKmAg/evJc6i5/DDq1Hrw8GOCavTPs1Hv48gSxPCIm1f7o1Px76uxC22hnOPXyJ+49N8d3EC7y0TV8GphgSRG5+NpkuD3MUhgSjT8FSlB54mM58NfYBS/RfYJ8wA8oy9gY4pppMQYQ/Yca71LpE8m3wRZdhVmjO38fDjtjl8vmk1mcs/wVVmQhsL603i5OdruiAaAq+x8vENLAoUg/8IAu1xQzF57CZt3yC8UsTkAanVQNCNKOJkAh0TTiOXiOd3tSSITb/diT8QPfmsUwALcMTky+CMLH70gWZtEdgJQ/V0lxDpW4yf1Z8IuThZsTK+Ekl4Dvd99DBvRPjz9mLb9yGxGpg/+xTb9w4mrBByh27AvhL+hDCeuUVbePxsX7zzApnyuDiI/CzRQmCfC2hKtQjNMt50uOaR2kSOfdLL2LzYc4OFDnaj4MZnZSPYJz/lJL0EUFUaTcWO/ry85DrFalsPrmzVJCbzM2lDS3l+SiZNPoDdrxf55LBv8U+9/2MBwWF93dHpR0dnHU9syqY2oRNRjy0ABEKsUg74yTlmzhJZsrs7+COsgFFWb8YMN2CROUtPT/K8m3nJV7hcO4VUN2cZrvs6PiGqz2ye9YJj+svcknme5lqAKdKH427NnPBsZI4DseCVs9epa1DgpmhG7SLc9Ep6R3DgKF5/+X3kumZCvnifDouYeZ+uieLq1wuZfq4P8YPvRqV/Cfsqhx7BlY2LvyU3VMI0pXNFS7iumB+XnGOf2YcJQ19DZtoI2AL0W+tqk/thTmmf5XbVt0ROp8GHDtkhsv9OxWtkFvbpNhf9ZcL15+hy9yG//90ER3qKftcok1Ov0NPY7omuJwyLVAymBFR6k345lV2T0wiW3IE4zQerQsvJ1o7CcqEfQUOKcRzTNGd5ZmY2k2rElXu3RmJuLDO39FbJDBi5//bfrTJPWnPkDPTT8LBXJHlyMWAxUPIGbbVexOubeTzqKmfNbWAH9unjU7noUUt6xru0mMXJ90iPGFJrj5IT/jir25qY7yOeY5KbG2CySsew0G0I5sncYmPfGksrJ3Hppg/oWHUlyD35ePRBGH0QgHvUtu/6x5Fn8RrTw0NrBORGOSPsqtgEBKrCqqgOrcXXeoiwUPF3pe65wPy71nGxy52ZfSvQqMU0pxiPEdQEnya0QsPffzzGC6vEvuZFUwqbmzYS3jiVufKVOKSL8ey5ykUPDb/tTedCgLNcVbLlPFd91LzR4E+6i43f3uI2nHd/msKAKwPw/uIDTArxd3WlMYkTG5RkJ+ZzNaaYaC9Pkf24ZxA9771P9CePcf/1WBzaNvGldzvd7T60nBxFgPcFsVGQcCEnhoZP72IcNzgxdDMLHE7QInhytFZDTP6dPDLnG5FNIoE7z8WS8tcMqgHvxyfcrkcXACnQOekEqx84wT8WccJ38+V+dB+Jpu/S7TzUr5pG8+vcrqYRBPqdyiAtO42ZD+ygNFTsC0oEAZ+EMtqq/fHLz0Q6aAj2YJ9EIiF8QB2qAAs/XvqeF5Y5r99vFT5G1PeVnB5+ms/miRN1Ll0XefqT7/mqWSDaGm3b86xWPu0IpazSn71L11PgaqK/g2+ze/puguoCGaS2kiv3ElXntXUeYJj3RlTp8YyOvkymp5hEYm5s4vx3qUzp103V4MtOe61K0oKvtRPPilBM7s5xlk7qQ1F6JhoPPZdbIpy66wZF1BKzrJz7PUrx7Fxye88zN/3F99Mf5fmcZfxc/B0RfSKxF61Uehxi6aZujoQ/QE2/v3lRI66y3F34LrL9PwPwgLdYttdD5c6mtBzO+cUzLLSVrTpxgvFGUz5VCa/iumQg0y2g1k8CxAoXaulcHgk7j3sJPBI2S1SpBrC2ycgQ4Ik7LlBpGsitPetkVx6HDhmwuMu44iVjlQP35UbgHvJGFFDmNg2tg+K1RenLy8f3093yNdOPJnJJI77usO5rfBds5vp30cgM66mdKxajH+rSRbo8mNg9A0lLKidHvJwgSFVUaGXsHHyWoogihkvEF5ctGY5Z3YNUq2Zn8VPMugniS5ReNOdH8ci6Xr8gfLz4m5Zh5NK0/WQNuUBQTRCyIMf4DQxqTwwpxaxP6KTRVAF2u/X3AZl4CgL3Z20i230jT1gn2Miecg3feDYxasBlVJenkzYkHQcuCPObLFj+uJvh00/TPkns7/W4RrM1uoAJSh3feEsxCQ1A7/4x228Uyr2lWDVKPi0fjv9MMXAVKBQSH7mFPS8+jLFbhbBU/HHMVOnxKI5j27Fh5KQ7KNZIFWzLjMJ8cjDnUyW4RYu747ZoW1CF5+P7egkFWgn1JscZBptCm2h/8SPeAPapxfKOuEaw3vM9dsWHI7EoRN/tO9IgiqOPIDOr+KZjBhvV4hjus50jiTuUTslDi7nhnQwcvG2TSGDfN71fqSsQpxDPwW73SXzw98+0+fjREewMkmcZpbxYBUdD4R5PR1ovdLvE0Ke6h6cyLkBcDUjEa8o9bjr6fLCMsxItJ2c8DKzpva6uAr4aeQebD96B9pArqvlHsAf7KiSBFN4IoiCxgMuDa5jjIZ7bTVZPMvXQVfEU12reZ8XTYn9QIzHjKzdzfUgXZ+tc+aFUXNlXcOwM6eXrCCut5cq4HARBLLG7ThGKfuVqplpH82LYJiTdseDeK+tdYTSTXt5NiFcmM0YamBpWc7u/rrvQzlQXMCuDsAa6U6MTr8/5Zi+uGeGdFviiDVbG2OUwtLUky74i3vMZpNxsGeGwfmtGbeXG4jNIgbfqvInxsvVK7jSHsfVsIupWLzzm7Ge0i51yhllP/tXPOXV/HrqSMNaEV/KK6WcRHC0xSZi/tReQzYvdjtGSxq2K8/unRuMib2V12DpWRF/B5C6e/4vjtpHf2JfDd/2FVC3u32wQAkg4H8Xyc2lUJUcjBDrIowvw7NEddIWraXJNQd7fARhT5+LuXwokYfRfij2ZDkCPQKlfE/8OKEJv0AFfiey662UE7V+AYdZppC42379CMYA95u/ISLuCV6sXbs1iEutpVSjyPteQlIfSvM2DzkfFczDWtZEgXwvu3aGY5QbAHlQTkJglLNozg3keT+Oq7A92VA+t9yyeuTCLgMLHKT+Zgs8EMaET4PHg4/gtaeXOeish/yF+jB5zCbPEwgs/fc8Dn9qyUav9M8kZ2sby6F7S17QOO2K6xcw3Pe4UrZtL3ENbGBSSKzqnxlRNV6Ca1ReiqP51IdkzykT2ZoUn2+5fw/KzM5Cum8vAN8WSrPPVOzif04t6toU4K3KYZG6c65Az+M95zItU4zqvA/sKaHdrPXM84M46yKiC9hm9OQyproL7caWiKoVtHU+j7Aljmt0jeX3DGTIQGOKgAOU4hgDDLVL+PtrJA/16gXhBKiPaMBltj0DtjxLk8QEMftV2crm2ir3T83kr6CfOnk5B6+X/vzj7f8f/7vFfsO+/4//YEAQBb29vUZJdUlNF0vl9GM5PxX1SJtgRWaQSCWGN99O3qp0xx6KIuUcsByFY9QR4V9BZLyWgPsBpie/n6s0VUz+EygbWurfy3n+4phOeL6Ga/yrD4uuxr4IASHKL4skdHyItKeK4XzPzVTZnIzUzkiEnpmBe+T2ewc1OlX1vPryX/dkRPP/h07hNib3dYwggW9Gfr5Y+y8zzw3hk9yzkQ8Qg5m/djbhI2vloyVqUZi8c0snklSdw7e8wIsbk4JeaAg4hgErmgskqEORXg9LFseMOfBcYSt7bq7gQDjqLLeF7673EF3jS59RPXO8Xwu2mxYAyvo1xX3/Mt+++QkjhUQRB7IS/o4qh4O1VWK2gNjss8hIV753ZT4FmAwF37Cc0SOzEeMkVUBrN7osj+Jkf+HiOM0W9weRJGxCrbkJlqQHESZ5Gs5R9I06iSbvCnQpbddMk9UASTr+OZpeU00odYU+Ke569mbKYJouF57XXiHARb8j1PXWUhewiSh5AfkAi5Wqx5FmuTs+1dvCVCMxzs5LjIna+hnGU6Q2J7LNI8QtybkwsaTzC7vmTebMFTmhBamzFXi4lgSy6huVw2Ngb1N9m9xi7iAtqpTa+iGthjXSH+IpkDqXGBq78EE5MoJwZSa8xaY5DktAo4NPkQ8/5NHJ92hgoiLeGtsAerqRdIbQqlHwPZ0nWI43udL5zF7OAywOdQWsXtY5h9+3kNUUDlf6eYqOphyHuX1Pcks7rvu50K23Je6lEyq4Ze4gpiWLioYnkhomlPWKEPylRRmBy24Ys+jtadZ9gz14bX/0+rUXPctU7HH8vcVDj5TeIOVu7CGl/jvfzrEifFQdkJtdUsn0W0R2qRVp6Dp0lHMct8+ylJyn+S4IxPhTdokiRraqrnNCra5B3RlGoDmXppzb5pJB+JST5V9MVXc8nkVr2m2z9wfxUrqTUhpCf04eo/lGYXcVJ7BR5JzNOuFPf5cbHr73LHf5viezXKt9g7NYQOg6v4PdZO5g8UeyIK3okLP91OeeBM6OymOugYHe9vZDtd1wmNXgjDfJ21NoGuzbQcLn7dY5e+ANZzFHavLtvB1XtLbnoQ69hrfPl7e9WEDDxAnV2S1FTczi64wfQDy7iQvLPJLS9yzK7tNbLYcWkx7fx0lYBmVmcIAd4ZcJYLl2LJ+WjFXjOFrM+kak5dT6VWf9M4H0g9bdgp0RIQPRB9swysXF2OFoysQf7rFip8c7hjhN3IK0uw/iczdlOEy7w1pBaVuzoz4fbZyJ9ROwU5xjcqMLMxbhC6uTwoF0C3WIxQchT1EVuZszYHvIt4sRmtzKSL/4cR2inB6u++5zDnmL+eoO+mj2T17PGQ0F4P+eqKgDh8mr48VXefGs1T44Wg/hKUyNvLz/AWxfDGXMpktplNluHZzdd8iq61Rp6TDFgD3zJPZjkW4DvolIOumnYLXFYv60SHv36CWoD6jm6xEKovpYh2ADSIuOT7CnMJa34H9o8nRnohcZ36HH5hjPDDmGIj3WyZ3dco+ajQYzWjibinYNOdnN7FyGlKh7N9yQgYAy3BJgkgoDUJKVTXY/Bx5VOpdgLWKDM5vtIePRUJKomH2T9xHttQKUn8/5ZhsIgwSN+DSKwz6RlR1sEmeums7nNgz6PO6yR+mYu99vI9bgUivJyeN05H0yt9SBJHg1Ee+ZRrffFnkwEsMj0DQWrH+CL+buJn2Lbz4aryxjx4iE+zlXhr1MjEXdmoiV/Pz6FTdTHTqPbR7wnqeSutPoNwbviEIcefIOr94jX0HqPV7l87QdcqKdDL6dR34r9jqgRegh2NbDbCl2dnk7VR4LKlxtCGjGPb8RNanFqaBJrzWVbEDSbIUDyE4L5bby9vTEbOwjt/A1JWTNb5+6ky0dl160ErPobvPDFh7xYOJ0T7WfRmGfzoJ3cX5TVm8nVQ7k20hPPrvtxHEJGHucGHOGY1EyU4U7ge5E9OLKOiuFXqDzTn8sdAU49PhWYocmLh66HYkpoxHFs7lPH3x7dlKZepcUw6DYAda3pAgHFO3DN0lB78D4GPCSunuvw82HiZwf59Ewn5zYN5tG3xaBvaOAfyNQVvKZ2JVcn/u5cTS28nfwuO6+Ek1frjmy2+JrcDFV80hGKZf0cho1eTXd3NIi6QsEB4z6WeMDzXmDVXcM+uW/vi4c0341KbpvEh7s9kUh6OD/1AD5SkFptb0tq0fO2N1zWm9ne3wtfYxo/ceW2PavtGu0dAroLkbw/1J01iElr6f5pHB7tS0mZOwaLcyWOBAmPffsY/o3+dC0ox77KGHrBxrqcWPwOjeb9+c5VVXGaQWy704s04LE1w4BepQeJRKA0qhSz1Mx7DRa8pVEiyWBdl4bwiwPRt7qjavIGf/F3d93sz0mjgTcGX2WwXI3GYe57nulL5aYJhN73D7WOy5xUxQtt7rdBaLVFi+Nocn+Q0rsGMxbwH/krIGZZ1VyPIAW4OnwIaVKbWNWs9kv0e+tNqscf59e5x+lxE/vGB2rDGXY9irNNF2lugYccQLOuQdmciazkq5AsAq0jmWdnM6oiWNjvJDPqXND/MxrXqV2iGaYTNLzc9gbpFy5iUagQxF0HADgn/IUu0Zea4DZc5GKkp0WZxKPmVj6JhYes3aK5kJibQufBwVyKzOe42cSnUtsaLDV38UafJk5ENfFrF0SZxZUKB/R+/PvG/ZgUcn4euZvlj4orq3/tmIa6KZ+h22cy0s158X7vRiOeB0cy7Wo/rs8sc7Kn+Ixks+UQrXsUXPKpd7KvVfXjaOiPSM1S0q22eVKFJxuPJtJwtZGLAToGhJlE5BdjtwShpBE3FxV44UR+sfi6c3XEOMwjZlHu2ykiTIDAH/vepmqznn0uPVx/qFZEevDuyKIiVMbfrteo6BmHn8U5JRQiu8gUV7irDiyC3Ml+cnQ2jdP/xeQKA2W2eWa+UU7//esoT5lGY+Sg/9BDC6wSAXN8KS4msY9b0V6B79n36dPoR88T6wn0TRTZffRV9G0J4vuY62gC8qnXDMC+Z99E9zrMxxM5dWIg13rkpIlbwiMRzIy+MISkK/1575X3nCqJayVRvP9C796fXraJWfLeNVoqWFFKev0/QW7EPyNbdFx/oZSSAe28t3UUpm0D8XhC3FtaY7nOwkV30dlUi1lmAoeq17puDWffvJc5HRqy+2Y7vWu92ohXqxedVSrKzbNwHPfH5vEzkzizO4OwcWKAX0DgRvQN/DfciY9ZQ/s9Nr/Mw3CSu1f+wZ//PEX3qRY6MzxFxxYKr/D1G8NoU22msF8mi4SnRfZrrVuo+/ZPimKLODtSLF/nowkhx3ct1owWZKfuYmyUGNDocIkkvqq9l52ggQmOqh0Ow5GUWdxSzJ6YBD6+0ZfBoa0O1Ta931lq+S+EtNxJoMOprwS10TD8NO+enISkv1g5o7cnqgmZWcagV3/kbLAYhMzV6TDsrCIgKZf778tGsO7lFsADYJS6M/Cd53iry42jj+zgmMN9SASBAZkDKOmRsToyix63KJFdr55HReEhQoBtT3/OLbCv1dDJke190Z5xJWbaSZKSb4iOE272uR9tjaMitId8B6UHtcHC8MwBxF9zRSGcQyYXkwe+ODaauBOeLF1SjFYx4PazgF7XTtXdjrmwCRmQkfAGEolYqafa6kahSzv9PXbQ4wotulZ8bn7eUomUqXumotZ5cNdiCyMjbCSpvooOBh2fyHFCOZe6kgQxNxelQUJMaSRzNy+gybcJPrFbi7wHsq40h8t78vmw0cigSReciZFBE2kYUU9nzQH+6XblPS9bDqTMMJmvFY+RPOIkr0c2k+BiJxVubOd1y1m8I+AJl97sm9WBbDdSdpaeCV2UBdXyy9irGHWzuVWdqtWbMbWrOOfxGZeaQvjF4ns78pRIIHHyeWpGneO4G2TrHCqnBYF2zy42L9xIhPZdDOYOwN3ezA7zeySX+GL0eRCJVLwmjPU6wqyFF9DPu0ix0bmHeGbNOgZV5JJlSUATrxTRPK7+0xdzrQLX7Jl8bDETP89GZXaRaRh+dgL9s3rX5Uke4r7qukAz/y7czPzN88k4k4HuUfHL+KX0JGU7TtEvZzWd45TYk4IFq5XnPQVcLg8gF0iIFe857YYWml1zyJqwA4VBwcPCeByHGSlZv93BS1Izvun7gadF9gf6dXPksAZr/jUEwUYdi3ULpbW+HxOy/kZlCiTarsBZEGBUQiVN8/az8+AA3m1QcMmm4olZomb1y4+gM0k5OuEg5lCxzyXIJFSGV1J9wB1deSJ+KrG46GF9OBrArLASnliJqJc20OXej1++v4OUa4moPDKROuSnq5TD8SqB7psuqnDTF7X6jeKVn7fim3UWz2EWdAFiIPzSgYu8Yfn/B/puPzfMvH0mG77fBj6DkAaMZXvHT2z7fh7dKl9OyWZwr93JrUpfXmuCU/7VHJ9STX+5I+Xy/53xn/CO/8nxP98l8L/jv+PmkEgkhIeHi+TtvFLnErlgP20+rmxvfkP07wVB4JVzngzanorVXU2ng4xnUdU/vDdgOJ/fuZvnfvgYT6k4QSOTKzn+61j+efNh1uaJE2K3xkf+9/BSygX6KcrReYgXY3XPOQa7b8Ps0cUHE06B2raYT++fj3zaUQ4VvcvKXbW4asTnn1YlZV2xL245nrh0iINQtdQFhdGP9IooQrNT0VjFFY17W0L5KTcIaZcLOovzJ5unDqd02HmGDs5CJ9vvZB/sM4xnVz/Lji8Ws6TMmTGitSpotsDEanimwcaSCeo4QpE2htGtAYxbeIjRoT+KjtuuH0ByBWgXf8ewV3+npH6j+LzI6bTAnhC4Fi0GUiyCQOnpHgJ3BfD21CyGBZeJ7GFxS1mz9UdcDkSj7mr8jxUtP7RO5c46ODRgI5Jax9oQ6N/sTXpdCKaQOhRSW5JmlvtmUnPr0SuMfPv4t0gdEjS/Xolh6/qJqLq7nH5XZu5m0d0HCb9nNz2l7qh1YiClUhLP9t+mkfvIm7y44kfUHuIos0ESydFrvXNj+aTzTtesk7jw58aZqF59ge1yb+Stp0V2X+p5/K6jDJp1kjOhoGq9WWWkq+fnaVlErFjPqZEHKAw/LjrulE7CgbHHaZbWkX5oG0HVh0T2yS1lrPhmBQXDz7JmqivuajFT8Zn+53nq8e18+8S3XOsnDi4A/qnqrdHyj6lkid8pJ/sJxQwmR17mYnAl8RIHuQilLzP/GUrZqql89+CzlGPTMl8QMpol/q+TkrqM6S8e4tHB4h5DBzULuLBYSkJtO2M2eKPViSUDE+p/Je7iBhQ94gQ39Do8HvXXCcr1x8e9m4EaMTzUgJLvrS3oDMV4dFykvDXX6RxXunqFtQRK0ZrECR5p21n6zNvD9TkHuDjyFcwaG3D8mTCVuz282NekZtYVH7rs5NSkCk9OJrxCzsIk7l13L0tyxEGgURGGPN0Lv2lZdCeZGOTQgL2vopSUYTm4t3kx798VBLiKgd1p7pfoO+ksTaFVVIc79z9q68rFI/ZnWi+msOhwHFoPsaSOtbOLxKxKXEPruatvIZKbjek9O3L46smdtM/bjUpuIlQpBuxMgRKyR07AvwlWfrwSty4xKLy+NZgXqxVUPL0S7+dX0tojvrZv6qPYKi8jcPIpWoOdE6Ot3p0YZUbaPNo40OPcD1MuUVLu+xV3VrjwbWeKyCZo63nTTUZ4jxtykxbBYgOBSkng2WoZZY3uNNd5U2SMFB2bdDmEVW+tYtulGJ5sFCfjLBYjj1VfYtru0TzSpeWQQfxdCYLAkXFH2DXpLHdvq0fuIiZMyKx6QpLKWBV8nUkVzklwgCWuEqQmOaUbfkdQi8E+vTKCSYWuyL67hwGXxVKZY1U6xslkHJqpomCAHxarbY0UJBKqTFAi0/OJoplTFjHo4G5tQyUzo/VsJyfhRxr04iC1qGw4N9YGMHv7bB7J7Y/jkPv7sONOFadGnvoPvMve5+Le4YHZKEfVNcH5H8Q/QHD/fIZsn0G7xRb8eptreOnbFUz9bQJDqgpZ1Siu2Oqu8eTQT3eQsW02af/OxMMk7oHxqHsNAUgZ9PAWHp3uuD5b6bqahKXNg+agKMrUD4isnvpijobAJEU5NzwfpqQzx+myn2pIw60EZpz4lEZPRyE4OHV8EtUloZzdMZIulQ1y+6J1AhFl8OTWRTz3zQqCzeLElMn4D351XaSc2kRg4WGRTbAaUA1wJ3+ilsKEQoxuYsBCECBv9CN8+PIvPDLndyod5NMfcinmpDmE1idf5/JWZ7lsPFP57o9nWf3NFPpXWLDIxJJonrQw0xWmuMB0TRcSwUJ4eDiG1kvMdjmAX4WM8ohyWvzEoPAZkx9CEfxiPkiSy0F0+rMi+916d4b9NIU5XTtYMl3CjVZxlcWqsGbWhBl4adsdzC13rqKfU9mHtwae5Ndlv2JVOs/C3S2j2bHyKRR/zGNMi/P+0RM6FNWzBfw7SIJVakuTWBt38sSz69DOOUtz/EAaPCaJjnu3rYyMzpP4nEvH99hwkIh9iIkBzzJH8Sdfr3iGoxufFtkkli4qtqhx257AkAtDmOVQDaA21jO4MgptvQ9HNk2gWOt83QFSC75SMFkBh+S90FPD7uq+rFq1ijdOuKOx68u39cwALn9xFw92+1MSCV56W68VCUYS1y9gwCuvcuL6UN7oECcP2uURlP09jY4bYax+8lm6qsUEq0jLOr6Ykk5S8lU8w539rknKC/g39q5BcoP4mQB0ufXn34O9vqtfurM0dbe7TXKroMMmYZ7i4s6D4Y/gHTKJRRsXEVIqLgHyjPPll6d2EOjVxbS900gw5YnsapMr4RUJhA6AcXfk4KYUgxYbzL39ed61trPW4Fz9UR/awo3PVvHm+lXEupY42ceF2Pr7hCp2iWyCAM3Bcfy59ADF0QVU621+QL3CC01aPsamfpz84RLJ/mtEx/Yd8CaH543meqUfv52NQ3Do9XpwbCkPTr1Mi2ChxyoGgG711z6acYpvHv+GFo24b2S3uY0zSRl0K3LoVGX9xz1rYIEfSQVJPB0QjZdZ7Df1KPwpPziUH1c8z5dX0kQ3fCroE8pGTiX9UjrJuWJfUGbR8aiLjFESObG1r5BkFieWLBIFFwdcRq6V43XmFBaHdgbHXf/gk+d7iVMXL0xzkqdzr5GRcrUfQVE1DO3j7F/IBAk6fylnM06hd3NOnk3zep7JR+7k1fdfpV+bLR7qVMZx/moUMRfSGbezgoL8taLjrDXFxFzZyoIjwWw6PYUonXjtr/OdhsucURQ3vEXTSfG7QBDIq+2N+ZQ9LuibxPNTpwzg079HUfLsSzxYd57LPc6khteahuBRAgOU4Ck4Uk0hXmFAArzbAlqFzdeM8VmPJFmBt+Eyg1s/xWIU++ZTJE8wp+0ndri58G+YuI+3YOomoG8RpmGX+WhYOUP8xP5FnvdEPvtkMbIGX1Yu2YmrqkJkP9v8OEdze2PCEt9fnas0deW8llGCh38LOyxBqCxiP0AmyJGapUgsEq6G2aoGpc0nyR7ZSxQojpLxfpMDZVidxPbcx2lSehM75zAGH7EPrKSRmYGXKP7wQb5cN4EeuThP8EFLMpUuncgjq/h0wVkEsx2x0tDKiZAugu7axbZF+Vwz7MFxPCdEcGbhFoqSfWk2p4lsI6TNlEZGUzv4Ml95dNJp1+e7QzmKTR0fciyjhD/vOcAF792iYy+3HsO35DIJvu28OfuiEzjaI48gb2AM42ae5vVIcQWcIAjIJG4kn95Awvn1mJvEe5JUkJJ+MZ2VH64krCLMuSQL+KEwjFWrVrFi1x0oLeLjrboG3tw1l551c7lxYoDo8CnFCl77+knuWFdE/8yvEaziasemyHrOjj1GZX4UZcXidyEIcGZyP/6Zu585liYKTOI4rLSzgJ0zduI6fgdhPidp0jr2UxPocO/9zj2zxfKKADJTC+OrohhwpRdQc8xDdCgzqIzs9cXK7IAarSqYT8JzODL2CI8MOM50k9gHUFi1THSBhKxINry3iI5GMcl1ieUsk3bMIrJEQUDNWQTaxMfrZYTUhHDks9Gc3N5LHZDexAslEnh43PMMeWIvYW99RZnS47bt1lCbLUQ2+XHgYhwZ2Z6gsSEmMquRlIYgoup8WZ/zNv/2nOJWn1GNZgFn92ZguZJ++7fsx2/Syyxdex+uPa4Y/JuQS+z2eamCv/a+SER2Ht01vtyozfiP+07X0DT2zq7FQztEZJcLKvo0nkXrFsSb+QF83G7fOM+d7yQD2VzrSmiTL+kyCRpBHJu6yXqYunQ/skFXsSJ+l9/szyD7taeIz5Og6LxITZf4++hbpuDNQi8+2zSC7YXiylMJRoZ4dTPBFZY1bqDx+j0iuyCAd7sr6ZfTmar6lO17xojsp1rHs6QWZtRZebVOXK0LMEC3l747p1H5/RLeKxf7qT7GepIH98YwZqn5tmoEwCLTWp4f2LtXhI0pQeonzsWOVNzgGz8In1XC0g9/Y7yr+GU0NB8jKqc3R3XfiO9E70IigQ8DzIQ8/DeXB1Xyj1qs2lVY+y9DKj7hgV8ewDe8DiXO5PM1DaO5YQRLSB11ZmcJ/1+vXeTEyCpaH/qdrMbfb/+9SPsPZ14cy9D8ncyQvoWbzPZNC1IpcS2urIy9QnRpNOEVYtKCUe7HkZRsTg09w6MzypgVL15rhkuzaIuQ0LP0CrnjnnRa504ZQilIKEAl1/L0CEftiptEVrOUusAmfnbbjkUdKbJbBQVtFoiQw2g1SKy9e48gU1Hv1peKuCgSz/7OgM5Zt58zQE93j4PO1/96BAB6rRa8+kHcI+ASRpV1MNIIOX8t+oRf7hiO0Y6orXGLZG/O67SdHYJfg99t8sX/i+M/4R3/k+O/lX3/Hf/HhsVioaqqitDQ0NsfQLh/A6Nn5PO18gVCupwdpPiwKxhaBnKwz8eMChYvll2CC99+thiFYGXSK0eIDBcfb1WFci5wPIk1lxlyTlxhcWsEVqgYfGERx0cfRy4Xf5QmVSw9o9LoiJ3G5KxPiPK02ab0K6Cznx7D8rF0BB1HYterTRDAe9d6XLp0XJoym/HxYsrSUv9Uwk4+QlOoO4oJXcR4i5ObfpXuLPjtYfQuPeTMPOJ0zSYvCX+PPsLOblf6NspFLFwAhbk3yPKvDENqqXA6Plhn4PG1yzBrlZx83BZUGRQB7DzUH+XlFEqXvUWFScwAk3dKSMpLwjOunn5xpRwzikHKqB4Dfc+N40RrMvUZCubbFUnoTXqKvFbTryqVVSf+ZtAEcfIdoC0mHZ/ik8Rf+JmiponExYkT4QISDFZ4uxncPfxJdzj+1epEMncM5PcPvuOU2cYyLzDci25wPpmqf5gQ1InSKgYdjhaGM+FIBhMi/yYs8Cfgp9s2idXAcg/4/EQcO/6eiv8LM7G/KokgILkJyGpd/Jxk4sqk/fhszHdMmnCUi02+iO8IWlWRlO/qnbfvZwfx8XJxgv68MI0Z19exQANL3XqdQQBUAdxdrSbHpOVtb0hXiJnzFgFOjThBSFQ342uCkXqLezKEBg7Aa1IOFdJlJNXc59TrRy1Y0EigNBJqjc79E3u8zWx84Be+iW+nVOr8bd0R+CTtv8xhquFths0QJ6ORSCnxbmcovSBwpd2349p1lJdHLuOPq+8QGNmAQSPesvZ357LL4wuWJ44iyKzAKhEHNlNG/85VZV9UJ7/GR6PEXtKvR1tPTPNfWNsFdmasYYmD1E+zroEb/qvwXqHktVAdl0zjcWyqZvILJHN8Ad+PXcMNwwbABrYrTM3cMyGb4zWumE6IgZqJls9xy9YRl/sSSr2AaYHdmiCRUdbWD8wxTLnvczo9xUlCvSKcreaXUHvPR5UdQkuwp8h+TtaH3NlfMSuhm6EyV+SIE/Q9ShXRC3ah7oG4HmfJs0j9Bfa5uvPVwYlk9812CkK1qi/QLdzHoKhqlgfXYLUaABVdHv1YVAunvRv4+8HvcdFF85LdPVutd/HF8qN8cSiVAZERdKjETMNrOXFM/nMhyXft4PFpmZSZxYmr0xv6kJY5n58XhzHUZ4z4og3tHBnWwNp+7/FFCzzs9pHTfYV1f8eiXw3AA1z5WgwWSvQNjDg2mtPXY1nx2eu03PgTBvRKbtYKUeg/eYhIi4RPX/iU+4PfER37UkIu14vVLP1zKWeGnUF42b6qW8bO3DDkmSn0iw6jT5wY/FcIWlJTS1EciMc/6yCWR8SdwVytTZwMhbcffpErcSXwOk7DOjITWXEIRqUzex65Ky0FkfT4NeE/QAwK33W6LwX7hhH+zsvcG9WGztCKi9Kz97qxcu+2eWgafPn+zr+I1ojDgf7STGZ+vpHv6pQ8KbhQbBYn8nI6/sG3s5aAgYX0n+kM/tcb/2KG39esLR3M6fAyJ7vS0kHGB19ypNWbq5brLHCwW6xGguIruGg14iaxBbGRpmyChtRw4nQyKz9eyYX7a0THnWlKIPG0D5XDzhPQvxCd/BWR3S+xktTPPueBelBWuyMSbJW5sFW5DutoPTpXH6KCxd+FwmpAf3ogqb/PJBXQL7gIIkG2XiDYYIXdQc8zpTWZNMTEhz3pKly8DlOYUMgCuwrRDquG8OPDMbW4Ix2ShU4yR3Tc14/s4OLiffQcGo4qchNgAwk62/P4Zlp/1rXJKRpnpM0iDiSrWzcT0G8dSmUlOkvvVYruy2MZf/3YD4lFSliXM1moTdeGpmEfsXV9udI/x2kOXpGMYe71HwDwMSfTINVQVVGBu2swp6bu59Do/dypBKVJvB4IgoTI0kimZA0kMDsVz6euiuxdnkPwGnUCpUnKwOI4ytPEidWq9p/Yc+ki8htW4mLFiTgAtzYXUq/2IzlgFKP8HbuhgSAV6HbpxujXzF7fPk59aHe1rOfUxV55mxF2n7VW7suhwEpyBHckxhlEOkTSUmPvfR0de5QO9w4WCuLS6q+PDiZiy0h6XJUYfMVAeaPUh01uscSl9PD40l3UBYvnb5f7QJZHvoz37E6qfRv5Vup41XAivI0qE6RWQPawJIebFqg81ztno2OKkNj1pR7R+SjK0kzat0t5L7eMwW/akih+HpGcMIxE5VnPkb+mYBwpJsZIJL3P0rXHlQ6fSEaF3iGyl+lS2P/ty/Qb8C5PTb1Ki7ENT2x7Yrisi3m/vIu105XjDmA0gFrmwpX0Oq730VFbeoAfHOyCIHBgYQZNmj0UG4feZrArenJYMehxPi7+Bm1FLMMGiYktj/q78uHgTL5SGwkYlUWV6j6RfaXFlZO/LuFaUn92xMxirYPzHdsxFtjFtD3TOPqAg99kMfGCdxta4OUmcPV0lv316NlH+DuX+aunAx0zbnZ5u3VPcGn0Bg6XhROv+IBm7a9A7zdySZPIljteZ+HW5+mTsxNTdzz2vVzv8vmBUbF6rr43iey+2U5yrq82SagoDOWlPxZQPFFcUS43d/B8ziA6an3ZNXUvcqkYXJVZjaw8N5pIuZQBz74HTiLOcP+AbCQedQxKvohZez/2aiQCMGV/77q4uTWY9+wurcOtD3Ve3YzdM5Kw1GIkdr0IrSpfHn5vMeHFsTzo1kP/xy5jX5mtpofO8adoqQ8iMlfJpSYr4+zqONNVLSTKYftdIYS23Od0zZ4xTbQ/eBJpRB1FCsdoA+TmOpZXFeEbbCUzOtvJHlhUyZDTsSjVOib626qfpBIJx8Ycoy6ynNfawoiIFPvuas9VLHiqgK07JpB3cCjDHhSrjSz1LKGvRc39uXpcVXWAzV/sMnTx59AHGNPch6awKqzR4oSv1iWKtZ41LAbu//V+nBY5wCJIiJfDhXDYYzmJo1btynYN+Z+vIjnWh9SFtucyLPJt3J42cuSzu5FnhlLa/C/pdp3Zhco8Wg+P5t/DL2IdVEaqXXG21NTDO0uPsaYNHj76J4n9xPHK/NS7OZ16nnMBL7C+DPobxT5yuymKa0kP4zMCaHYGDiwyT47KO2hv8IbKIKQSMTh7n/oGys9fwKBVMvCH98HYBXINqIP4s8fEpae/w1R4jtguh2oslzQO/TEB36ps7n95A0PVD9lRF6FVOpSEw1HcBVSHVDuBZmaFlTWPrOFZTyh0gTB7MobViqUoEvXHS3lv6S508tfBQVQ13+zB1T4n8AvZRah1MmArJ1ZLFRQeSSCgbCx1no/gIrGto8fb8/jE5QtGRJ9mpy9s6E4Eu66UdeYD5K78necCjKQooN7humWyNiyx8Qz2ldHW4wAqmHVMqnmWspuKK4Feh7HvdyYztDDSS49ap2b5r8tRP+DcqqT9Jql2sMENd3MpYJsPUmM7XrVBtACvJ2Yz3e5dj9I00qgJo8Wgp1/MEQTjArBLbR+NqEcmMdLntXfQmCNZZfebggAP9cviUpsH6swBKByqQ5N0x9m94DKrPp/Hiv1Defxf8RwKNV5j+Ttr+eD4IKIN63EcMm0tbz62ix+/ULEqdzASQwvIbKS6612ZbFq4nm5lLmHV0ym7+XeJIFCYWEhhYi/hQGIQT24Pcx1f1iSyqSSI2tow5KbpInuX50BCHz5Mi2spj8a2cqwrEeyyRiHD87kW8y/Fm1KpK/sJfYmVWePcmXfPnUhlD+IZeJ76sDKW10BiS6KTv/ck16n4preFwunhp8XKXcYOPn7vV1afSKbin2FUx4y7DXropeFkj30anewzlpkm067SAiduH9vpNQnvKW0Ykrfw54Aq1kuetf2oWUdr2hX2BVXhrtvNJcIcPGto3RLMIwmdhFVMYll0DuckBdySnw5RXuLJ5A1c/jQVrXIMfafaNZGTqjgmjSbx53TG1gWRMfMk7vP/BD68/U8OSibzwJkrjD88njmDL7BxoC1v0xjQyamMU4w4Oww3QSD8rky4WV8tkUBEWTijj2cgK4/kstSWpwKQC50ciGph3aUUrp1LQZgsJj0IAhyfeZyfy3xp2zGeQbPFhPsKfTwbbz7+CLMnjqNC0R9zSj5dVQEgiEHjb7zv56NrHzHsuS9JdjegkbZwq6rQJPXBKO8FsiuPxWAZJd7j41o6cTk0nZogD3QB5bgrxeukwfcTKsea6Y69wReuWr4V3ZOEQZXQGl7AjaACBiEmIvkZi1iaUsMZUy7HI83ssuSDg2i2xCjBs8mXAxFlqHWDecrhvs1HrhJT3pfXvzjMOmvZ7b+7+LnSFpiEb2U+dYXgtrKeW+uFIIBglaBT6jn42kfIZRLx3BcEjow/QrDZn7v8GsjsFMdZEYpZfLG0D8FpVWinj6ai833AVlAiQcLWeVuI+uFRPl6XwWoHsRK1tpy+j29kaxc0VHdh5Wd6y6FvPnPXUOY3v8Lc5PdZ4gZaUxsQgGDuwSu0gxo/Xwao99Ensre/uEQCFgt4qME5OvrPox7wUEug/jjkfQgpr7HT+gvFqS0Exb1Gmkxcee2h8qB/5UOE7e+9zhOPOeef/18Z/wnv+J8c/wX7/jv+jw2r1UpLSwshITb5r6frYqkWmknpM5dGqysgZvx9Jd1Ew/BztKk+Jdg8EHutcpPCh10DzxNekUj+b1t58gdxIqNeW8X+Me8zqI+WMb7i894+f1sklwuSCBEkqFY0gV3Cochg4knTTh7QneX5aAmSnq9A07uJZZSkUS8/z4uJAmFBlWDWw82eJoIAUUUm3JsrefU5Py7r5wN/3T6vRn+BphNSTBGunJA/z0QHBZm5UYWop7VQvWcU86uc2cFjVVeYXdOHknV3kH2/M/PSJPVBOusG1uYWMtyc5fEe1ys4W9ZbWZSmtEkgdrj25YMhL+OSepHyBisjVYnY94yNb2gnbdMiTk04RJTpEp+PF2/oyywVXD12BybguGySSB4Js46Dy0/w07Emyjcn0xzjoLutrWdw/FFQnaSPTkpPlyc4QGMTXLPpL8CbLXCPhzNrqS5gFBJFJz+8/BjzT9uYR5d0UexV6JgUnc1zUi+OOTCLc/oVUBlUzLLfl6GrEifILXIvBv2bTExeOD3DBBI987APPCK5TuKDu9iweBdu5W/R2pOOffAtlUgwS810Hx9EXYSY+QO9SbE1D60hrDKMa/7lfOIuZnopZS7IjT5s7mpmcxc0+/f2vUKu4UJODMlXY7Fkp6GddE4UkyssPYxQwcnwy1yQjmKOi5/IGVZHvcevChimWMcYr5fA8hb2kiSfnx1N7EENUWoz0nFZDBMXtbDIrwBNUCXjG2GuchzLxWbUHRdJOrOHMkLxG+Qilli0mFHFVPPdO6vwqHyd+wxZ3JIe07qN4u0nP6BLFcSW1BzmLZIx3O5QmQ4GXxjEgdBSKsMrmSQVfzzLhW50KXu4M+tuDMpqka2mMZPHHvye32o8ofRNp6DFs/sKmwqHkL11Am8/v4ZJ/cUVKwAj+/ejz8BS9gugsM4V9SJs8xqPvBjGdysYkzCIwoZ3iYjoldbxzdnAiHM3yIpvpi6gmAxh9u3jegxdtHiuQNot59uuTQR5uGAP/2c1n2bfgNFknMqg6tCD1LwoBjibrV4c0cLy1HKGedXjKKJ43jyKJ+p20RAFhWrnyo9mZQKrv5yHFIiadQyFoRZ72bQkzS7mJjby7pYR/NtQy87FvT2hjOpgKg8MIxyoHn4WEERJFr1VQZ0Zrqbm4DIoB41K3Gxc524kq18WqqIg9uxN5Nvx4vuqC24kx5CDRjcNL8TJHwQBhcWLMVKBnPAazgjOVVXGtr/gpuCjo5SC4J7IJ241eA7IZJtJj7819HaaRBAEChIL8BUEfvGHVkmp6Nj8iE4M92/E8ssCJoS2ic8rkfL+jG0Mjynkjs3zCbtH/DbUxka2B8PH3Uq6Wotp1BYBtn3LKPXmtY8Woeh2Ac/O/8hWfdiq566nf+IlaSZSbR/wirT9PpByLRlNmwfPDBVXBBz27EbV/yq6w4NZ26ecxfNtgYOrYGa2h5nsq0E89/lznLpH7KSXShJ4shb8/5zFmSvJhGwV7ysjkz8k5tsKMqpAYlHgWGsTYvqSuIMj2HspCdM9eTgOCdAnuJVMlYXOZmfHeHtjf+qU+zm+PJdKD1u1Qrkine3jH6MuLI+J1/oS4yc+tjPeyPsvv09I80PE1P7Me+FiKbftdTu4XHIa/7J9VIWJySsAJhd3WgI3MD7uS7qEyYBNQrfdvS8flLsyDuibkY0B5z6yo9VNDJPAkR6wmJyrKK6Gr2FQv2t84eJFlNQ2V7ylWsIvp9Oq1LEpvg9feIqZtF+1uFFu6SB6wlEMJjdRzrbFbOa7P0djavHgvSd38EuHGJj10v7NK5Y8/HfdgbRvASqL+L5bXGeTGxBPRvw6kkedAAehOFNnIdEL/6Xnq/tY2O6PFB3YOg0xyHsSSZWfkh/2PCDc9vvcfCN4twVcu1yZ9sUzRI/NBDvcS2XtZNGuWaibvdGNPkuJm5h53+U1hoPuYwg/v4LciwlMW+Qpshcd94EdMrY8/BPdAQoc4byJJhO+xycy8c4DuFgFHEGJSI9qyt/9iVM37uf1iLU4jhOVLzN4V+99pt6lB94HoF0ZyfIG8JJeIz49ggvGR7DvOTJSX4ty7X14Tz7JY1MOc9ksJhud9TBQNfw0bSFDiAwU21p0Teya/CZykxdFJ1tY6qA9KpGpKXbthLQswHmdA/izw416SztDrh/AReYAsLqEsHL8LuSj5ByrO80eTeRt29QpWbQPaOHk18m0VScQ4m+bByqFC2UhK2iK302V5hEUqkjetTuvxtyIW0IZbS3eXE99g0niFp9sasok9qKSixdnc2HvIJZNEvcea9McYtyOLVTHL8DHmIQYKoRolZI7Y/qySfYzW4UUfkAMNrZ0bGfQsDns6LZSaFkGN7u5mTyG8sYPf6AttfD0p59idgkEbL2CeqwxbD/zAPkNi9lhSuShDPF6gbsXqTMucP6sJ/6WfzBbZmGfgAkb1EGhVIv/Pn9mX3X43q1mHrNa+GntAna5N9D/HoeHAqi6DrEseifvlIJKKyZ+CQJcqZ5Azh9BZFd4kvKTTaJfchOwbxtQR8oANf7yo2DX6dCn5xzaZnf0Dxzn3shGsBix/2bPbx7PyFMZBITX0cehqkqDlv4lcRRlxVN49wk2WcRVbjLBxDipgrxOF+pMwn/csw4p5SjywvitfQKfzR0usgX1XML3rr38W+WBzi1adHyP+2qMQXpkL/xIjIfhtlRV7wORkh9dQmB5OL5+BroFcTV6giWXrAg4+eQWMlRb+c0iJu3cLS9Hk5fGC216LFKJ03WPCCxkWVIRY3JHMly6zumeqnWN3CgOQxCsDHXoAQcQItlLR3gHpZOOsFc2iLk3/+6jy2JXnJYv/PLw98qjVi8mqV6QSbkSk8/+ieMIvPEsE4LEWoS/5Qwm+pv+9KeH1KkvYh9bWqwWdiblUhKbyxI32NAuVlgRBIGCpALqM84TXhWKp0W8NwBEyDqRCfBSE4R7JzvZz5xNxQsY5XsZN8N1bvV7Xd6QzjX9Wdyn7SHBKsdfIpZRvj+iHJd71jL9wIvojGLQyqL0pn8FuGWlMn5jMSwTz31XSzdB/b1pO56AV3kQlmFivz9X9isH0x5hvSmBfolwQSL2By0qf15U1+L52Ld8pOlip0y8t5il3niEVdPWqGSSxgDW3vVI8Ejkg3Ox3L3+TmAdbhnuwDO3j2vVN1IUk01ZQCYWicVJhlOQSPFo6J2XE0K8CJCLfdw73KqZAnze1vufUWZnV3qzsFvCqJhihqn1WB2IpABLuh9k6ldpDJl7jKKINdzujwsQNpfz9e5YOjKJ9sxEykBuffNtxgYe/Wo0PbERaJ/cDGax3zRBdYUf+naw7LPZ/K1V8Pw/4vuKcxnIvRltrH7yWTqShzLCpspHj6GLkKACyghgzFvfcUI2jdl2YJ9cX8fnU3PY2+VDq06OIIj9C4BX+1Xhm76KJS1v0EctngsmlzC+WvE1rWZoM4q/jT8iFLQ9tgYJ0KgEX907RNiBfVeOPEn1WT1e037D4LC0CwIUfSqgMQ5lCtAeL86tGCQaCo1gNcqQG+W2lho3h17qwXajgTcnnya/7QHs+wECmDXR3FFvJiM/hn6+nUhNHdi37HBp/5BDCVe5uw5MXbb3ITe18ZIXXL0eTPTOqVQMF/v1Wrk/G79aSF3idTY89QfzNd48zH237a6+X7KxGjqHelNv6GaYVBzXai4Usn6rlhHSCzxn7IU66oprWPf++xw1vMWHw1Nw7RNE30Y/LG7OQHq3dRa+Q8oo7G6hz4BCJIYOUPeuh2aZOwtrwdCgJLW5kBNNV3j85hw1WYz0aEDVoiXv5HC8YsVKDuXev3FABjK/nzG0QGi4XS/i9mucH17Hy011XDomA6NzgWi5LIAL+lqs8gbUsk4kVhs46yPLJ5FtZBmWo40JRyk0A7bnIggCOak5BNUFccS7jsHW0aJze0i8Wfbb/SgNCvL65IkIsoP7lCIZUMy1qCredVFhMvWxOy/Mq4hGUx7JpgWb6IkQxwwWwZVPjj9Nx34Vx8cdJjRITLYTBNjR40ptvxxy1C00uo4W5UcEwf7lOG/ENao03pn/Ji94wfxuMTgllUhRa9WMMbjzil8T67qquSUtWRO4mq+KewiJOIpZpiRAJ/b5dzYOIOaQP0VpFl6v/pZHXhLf1wvRFxBi2kgoB5e2KL5zAM0u6wRCujU872mmQybek5pUSXwZ/yHRx1L48IXHCPkpBMcx0LuUtpdPcsygY/p/YLCYNVV0adqZUy0wMs6W2+iQfollaScHq5oI7BFQm2yZF0GA+Q3eBOyYxehFB0lJvy6aYwpzO9/5wYU8L1Y+8TZe04cywC4HZ3ZbRHPwdjyFIj6xVHFOdxl7sO9+TS4fqWC9Uk+R2fldqfXVrHSXYL0ezlnVTgSrHrD576HmXDYPfZ9f2uGJBvhU3pu7lrReYM2MsTzeAG/1BzdDKCXYvo9BEwby+7qD3GVy+kmn8atEyuBxU6BwNdTuA+90fvSbjMm/m9VbFpAWXSfe87oreWrcUrbeGIhQ5I5C67xn/b8y/hPe8T85/gv2/Xf8XzWCS324769XUcnNTFi+w8m+PcWf5MYAnqmKpz7wGPZgn0QQuDToEjE3UojJ2QzW17Hf4Hp0VSwNP8uzAyFbq3I6N0DVgGw8LyYzZVAJGuN1uN3yHnSmHjybGtBsnkbtkOsiSYhJmY/hnjmLxPv/YlnKNYyG+5CrbQHIM0+/TmFNANsKfCn1vCHSxm52v4fGKdfpanfDpa0aQRAvBhGBTQxekMtr/bKolcvtuwgB4NsuULN7DGbvVjR+jql9yO0s5PUB6yiIAMHiHIRq/eYRMDOHSUP/wjXQE27y2+J9Eolr2YakdQ0TD06kcqF4oU4Lv07qE5do/3sMQ4+OQZjgUJbu7o7ywb+4XPQAIb4OsnmClL3dUKGTITXqEaxiVlxByRZWDFjBtxFS/jKZmSh37skzXpNFtAI+au11TBzHgdbn8Yg9i2tLJTJvm5N0xbCJS7FvMHjtvayuDyB+h5h6/3lEM12hBt5y+wpkHqLnLZFKib2aSlxRHFWfvkmn7G7A1nDZ29rIbDdIVkC/4Dc52J4EdnUpqRTyncyTsvNDuShxlhaT95TzYt9atkXX0tMjc3JYx3gOoKP2b9ZETqTdgkhaIbIqhbTsRPQKPV9ahdvJBABPQykPr7+bPPdWPpi4mmK9G9jBfbc1+Y8WovVsxbSyB3uwT9UlJ6QmBANw0b9LxFYFGKaqpXH9LN44OYCo7xyawAFBlp34hzZxw30kuZELxHxUUwdXo1r5sR0SAt+hy/oP0Bucn+2ooty1kaDabgbs+4DLUYHcc5/Ng3pKupfje+ZyZtgZKsMrnZKbVw0K2lzaufDQtwQYhvOY/c8qfFghb8BkUONrmsTF+gkssbsziUSGOryW7vAqdlS5M07mDPb1d62g0gRjquH58NHYSbgjEQTu+vMu4orjUHl0oZ9pkz9N8tlLub8LLsN7eGboRXLt5JFMhmaeKeri2vkULswIwMMsTmR4mcv4asdCGvOiOTDxALGe4tSnxCwwc8dMPg2tYlXgCnaN1WHP6r/D/SnCV1j5zqhFNTOZEeI2QLQpotnX5xdGJRTzWVIrZfoK7Puh/KsdyQ9Fh+ibn0BWrO52rzcBgf5X+uPe4U6M0oIxvAeJnWrU6ZI0Wj4ehmHcYd4bdZLHPMV9l15PqSUqrYTfX1yGe2UIEgd9mNdnnqGsyo3uv2LwcMsH7CRN5O68t/sA/tt/xjO8FtVrztIeH5mSKF76B//2aWeaqyfY1UkIMhcOxOeTaLZy7WIksxPGcqv7QT/hIkvuO8LGwgAGXUuhIl38Pr5rHkKR6Sgnnj5AvFLr0DNEwCqxYghqYNScYxjCxNLQRoUvL7y/GJ8mX95eu4ofDT7Yg30WmQeXXduYl1bI0kVFTvcEEFUYQFD9IHwWnUJADLqptcV88dB+zrQqeazWg212to0x9XTGn+bNt99kd5MbiyU2dzDIMxzjqEQksSWEVjQxPlRMjqmTRHLu4CCitRK60i+ikoonkYvEjMfNpUnp2K8VyGyJwvdSEvXB1eSFOYNiRpkX445GsshVgnvw+4C4gklTpWD2untY251Dv9m2YK5DFsS/3RAVUcHbYyr4o0O87yQrakkKMHC03QWVMRiVw7YR3p2KObOKqLyRXBgu7reDxcyY4LXU1R4ncWc4HeNPiMwyiYQTo09wdvQJFAJ8JHNgRAAPet0g/abrcdxwGBw6xf3k3ch4qRcebh20KmwX94Z3HSGvH2bZgZMkV6aQ4JCY+rUgHHW7gfVRpcgEN+zT2Fa5J+X5kUSVRfHIpxFopoglFAuEfpz8NQhlvR+hrb6o7xQTADqMzVREW/l8yq8066JBRPkBmb6eZwZU0v3V56jUeuqs4sDfV6plvIsJPzX4WLrB0gv9Cgh4N3sTWBdITUQ5WUqraP30MxVwz90H+b7TwjfhhUx1Wyn63a1Nb3MkZR0B3hYEq8A04VGRXRYeTFlCGGXqLqxmZyDlldR6/olo4OBfk0ifON2hsx0sdpMQK4M7Lz3klNACmGDeQ8fNXbZTsH2zgiDwxNdP4NvsS0xqMYXjxYShNN8qIu9tYJ9LC9ualERKxGFYU1A7BWEnMcceZE/nN4DtfUisesaqoa5dTw/bqe9Jxr56Q4mFVHMg96k7medtpOg/AMoft3iTompnYdg+pNah2BPaBATaPXv3omN+CRgsPajprQCJc9mFOryMtYOLwWrF39Uu2Wa14KlqoEvVQ1VYFX4m8TfvYmnhmQf3UNulJjboRy6Yz2C/p1hdYPXTq5lwaAId7h3c77CPSyQQXNqG0DqQliTnyj6Ztoj5mj/ofPsxLHccdbILPQfp+/0y+ni14b7UJvdaazJS0HaJ5CovPjn5FxHJ7nape2ho6kPWt6GYE+SoY4NROPhjRaon2QX04SkCOvNp1Q3Dzy6ZnK59HWlJMq6p1XjHOFy3RMH66//QnXmCsSTTtsRZ+nF1m4yWd5awuDycqnfEILwgwK8bXyTyxl5uRN3gunH8bZ9votnKsIN3cT2yinnTjnGkx4A92PfR+YcJ2lhF4IMbGRve4pQHNEvN1AXU4X5PKXoPce8vjXsfspMfAOuffPbYC7j/4NDHTaZh1ciDaFSNlJ+NZNYKnMauxkCmlwdTEr6HDvOj2AttmSUuKMee53WZjLvr1aL9dEHWPi7sG8bO+W+xUx7NUTuwWmI10zL6LL+MPEtSViHJuIr4/vWScN74bjo9Vji6eDeegWIph/X1SYT/NJLJMZnE3+cHDn0lj/ckcqiwjLS/R+E2xTnOajTp+WviHgZmDsM/5x0cIYv8qCjOPfUYW7qsDNDbeispzF1EmlS4tqoY3dHBQJcUHrc77po5gn9bIFqixkflhmOoUyDxwBpdQmp4Iz4ZDn3HEPCRQIlezl9VQ+nyGiOya7qL2RYEmvv2MtEFNuEM5j3gkc90V3DL9+aTEEeYHbb3vcGs5Gwe6t+B0Wq78obqAPoemkt2v2w2xJYwWipWtOly11LvWUv2yKH4OUhOChIFmkMjSChIoi04Hm9vV5Gd0nW8NfQFXtz8OJ1yF9ok4kzlvf5b+bo+hHUfzqdpWCHuDurTAgIjT44kLSuNLUvKnEC5evdH+LvvFU4mD+Cd/CC0Cs/e4wRQ99jm3ICYndiDfeaOYzywZBV7zybxyp/30TFXvJfK6Eadnkf4iDyWBvVQLF0tsj+scSF/x1Q8dgzh0sBLCK+Lr2tKxFdY9Ls4uH4yV+MtohgPYDEfsLZlIVd/novHl20iW0V7BSdGn2ZcUwN3uG7mhmQjt4iqfSxXsKTVcj0nmh2PvkrJ0+JqmlxjNJdKruFRFY+3KZpAV7EiyLmOEC6+10vn1QU5VPEIMh7SlBE9QsJn7VomuovbJVg1cYypgtL0Y1SYYLf8U5yHQKERDlRPZJpMLMsnSJQUG2GmK4S5GkV7daJ0DacutXEmcQTHtDBPJl6DsypnoKo8yAv10dS6OZO7zMbevXncwkNkxYhL+8rVg8l99RGM4dV03r8RrGKyUIciil+3ZFDpoSSy/1TE1HMQ5G5cbHah6tE1/B1Sy2xXcVWWAgMBUnjUA3qUNolnmbGVmeeHofprMlKlgZmaTNFxJpknW+ZuRe3ZyYqQCsySy2AH9t16Pnm13uRIBNL72nJCqz/7jD+36ThhgSFitXfu0uk4D0w+k0PfUxMYxnCqHix3ylEUGF7khr+eUZNUvOQNHcYWboFEgkzNubww5Jo27v3ubd50hVuAdFP7R3yS8TkfNUn5YoCJWPVdoh7Ot0bTocFsU3aw4jG79UQVxPstcKFFxXCXqbgFmpE4APw7vebyRnYW4Y19uHrhJNPskhDXdfP4PjOTFImZBcM+Jk2oBG76bcYOvjbs5tdZ3byc0av2sgYxQWWOfz+2m27QkHaVYWOvoLS0Ab37y2O+dYQoISy2gIMC/Oi5UqRC9cId5zg8NJcPwuqpMIv3JKtEybmSVfh6f0SnW+dtCe1bQyKBFUVRyFS1dIwoJVwl7m062+8vVvtAoAwytQ7xDCAVJAgWgadd5XTLxWtVnKGMWVXRKDbM550xrvhMsoGUOa3n6NJ/jn95737hYzqLvexMSMoo/lp2ipFHPAjL+pL6p+/Fvgfd1839aKosYvH+UWT1FxNJBQGCdX158NNe9ZLYZ0+K7AaFL9u7YXxADR6qbtyk4spVgMmaPEp/mMlLLl0EPOQs637X8kc589598NEK9L/akhSxNV9wV6Ker29uN7N7xPHZ24FtbJDA/vwkDsvi+czOHK8JxfLPNDoLBtCm02I2ioMGiVxG6YB5xF05xu/f30HIr+KYxENiRNPkw4nxhymOcC7a6PIcyNRdyWRsnscj4y/elDO27UV6ZSK/5b7Il24fk6OX8Okt30gTwXuH+qLaOxzl9F3U+ctvP2eAoQuW8cSvhziPlSFOv2ob54ETFguDRg2BcA103YDgaXxp+JmS0yNov5zMLp2UUXYLgsmsI8TvGur7O3lJn8kgxT3/6x/47/jfOv4L9v13/F81ol26QaXH3KVhW24EQx3sdxQl0WfPJGraPPEPEfdoURvqORgCZTMO0ZU8hQ5dEdjp0ivpYsyWObzXrOFq6jo2OpYfAU9Yeoh64htKuxfygoeYQeZhKOD57KE0Ae9Ed5Eh9+YWHzDJqELXDj92NPNHFeyRqm6n1wUBMmXBXK2RY/3mYc4sr7TjecPhpuNsSP6CJ3ZHoigNpaVjOtg5E6+29CVXV0tSwyskGx2hPujhB9pbvmHs7BNcC3emZ3Qa25m/eT6fBDWhnKARlcwD1LrMZbd1Lp5dZsKN7tzagrz0N3glYQP7T2vw73YhRCGuaDmqSmBN2G/4DXMhpMkPpSVNZD9vTeN0yHHeTH2RQOFdwLaBCVIlT1wIpX+Hgs1fvsQeDgG2niRa11ge/3MkgfvHsfeR7xk12NPpvr5pnkeu5WOOhECDcMHJfsN7DdphnYQ234PMTpL1bvcjzPxrCbWlUZweUUCKUqy9lC5Aq9aNDvcOFIIDKGwxU3HHXqzmw/wdAL91iZ9JjjyDF3YdYM7eaWwac57hz4qzsv1d3bhwpPfdPjPTWQ5TbqjjIZkLsUY13wVZoOEk+NsqGv10//LR+CdorQetFSTaelB5Qk8Nf84/xfZ+bjyq+RW1XTUhQI8ymMJmDYrcaF6o96FrgZjh21W6noxrOVR51XEg/TIJ2kZisV37sFEqKgd8x8YmHSqJWKsc4J38iWSciEICBMmOg4OIxjaTijNPfc/xnu/x0q/mRezel0TFR5fi6Pl6EZL0a0Q9aEuONmir+fX+z0mrCmfS1f4Y3b2wL1ksdfEn6Km1DD05kDvPTEIyR4/96H8lmrST0/h70Uasvg5RidyVzV3weEECfseH4zlKXGHU4z6QKR4lPNE9lXFHxiG84cy2Si31o0fSK4LgWJovN7XiczMxcz7jNGEKW/pnXXwVuvhLRDc/wOWamUQOtcn9CTIFNYOu0OreRMopA81dAtjVScjR4iOzoo2qYtmcM5RKxKvkZMlZijOHMTBzICXRJfS82IV9QtfHchq5sVfGsrnFWcZTQODioIvUmBWcLZXy+wjxOtjU7k34vvFkD6khVPUpWK0gCHi1nOHZV/7k3UvhxK6fT/qSA+TaEaatLpCXlEd4RTjztsxD4pAEjJAMxsNgZs/cfxAkFl61ivuhJCpMyCxKLldLkWmPIwL7gC7jFfyBtoogGhwkAQEMcgXXYkq44gUWS5JIkFWwmvGVwpiTo1D2qJEstb1LGWbcJdB6egLbTscx9I+HROddWPU5DYeu8mVUP/Tuvmy0KyESBIFjch/OStSsUr3IcJcE7FOnbq5h3JD7oPNu5q1m8AgR99SRSSXsn7Kffm4ahOK3nCpmAebm90N+NYlxwaVcXCIGcqwKD3b2WCn6aS7atB6ww0tSOp+lXlfGjw/+iFatFQPlUhXrsj+gK+wt1i79g50OPQ79VaFM2T+V4pgids/YzUjEYN97lWkYCjxZtWUB1vhARGU+QIHKl8rFG2j1aiXOTdxU/tZzW/L7fQBEvL8TR7AvJLQeU4CJggtJeM+x9WsdFzKLh/84QIvnWdLmvUm4vI8ITutvrGFUZzA+h3owFL+J5KW7AdszC+3cBXmFxE48y6zpZ8QXZTWSXPkXrTtGkgeUptSK3ofS3M4CDVzRQ7ERrIIzAeWzliTU0jMkKyDCLdbJ3rB5DF9f6ENZfBDvH7WxSmvNC8hvimNAwykChF+R6x7DHuTpmx3PoLN9afVs5cLQa07nXXfPOvpfGcysnVPxGSFO3jdIQtk6bj1JlREcTc3jKbl4ljVV38dzow6waHcy9Z5Wh5opMHv1p2+WBwE6F8xhzayTidn13vqDfD30JVrN4CUtxXyzl4Ss6wafXh9Ixf4MvnjqC3AVk5w65VF8E5nPMS1gwCkhm9Z2ieGbx5CTlkXBoMsIVnFQH5s6ihF96ik5FYPOw3k9ONj5M/lRB2lVttGsLWK6gxB4i2oFq5ZFkdC+j9joL4ANIvt9g/M4HVfMnBYD5e2fcUsA0stQSXTKDdoyVRRfD6NpiFgyeK2xL7lxfzDl24do0aqJnCS+tn/CS5B1aVj96XQ6+tQzw661o9LYwNtXMjj890Qgi4ohJ4DPbtv7SFv5oSSag39N4ndg2Pn/FOIJzNXA02GfU256Bvu9QWrW8lb3QnoOm2mIK0aGHm6Cfa/m3UV9Xi1DI79jZvwhJN1fgW9v4rZH18QLfpM5WxLB9DQpuVpxsqJNEc0nTzyLxCJFGdyI7OUzDLHz7ocpc3gipZ0H/beSa3Bm5btbr5F+qhqYSY7ZuRLY7JbGj788ihwDi/s4SwXdkPan43oVEM7Ue21gYLepk81zv2TfRDf6FjSS1xrEM3aAx96ujVwamcWgPAMhhUdRLZwD2PbD4z3fcTZ+K4smHWexO5RaxUD4hkv+eBYFUzE4lDifuWKSkyDQEZBB49RvyXCtpUE+xum6uyUuhBX0xjDLleL+zYIAZ1OfoF0I5dSIU0wRbGJVSWYJh07HYTa7oCw+xsOxyXa0SLgYtB3rrCoqFY08e11Km0xMYPl0UQmZ7Xq2ntvNsP5iMoZEriavfSxNob/grbjAQBex/yyVyDmrgxf3TCI4QKyacWu4dilpDKzDEF2BxCz22brcBvLk6RTe1vqQnNIhmgsXvZrpTCnAt20OUmkSMjtCkCtGCiLhh3Z4NDWBKvNMwEYYbZKGUn8jBL9GP35JyOHB6WLQIEuh4fTiDSz5ewmtO8Y5chroyg8g9bfefS5cegAc0veCIODR7kFoZQBGk7OCyr+deZRfSGBCYQKek2zeR6P7aF75cwQjTo/gUd9GChaL1wuXCikvfvki9QPLSOt3L21ty7GvrtZ6m/njnl6ZQKlBjWME2Oe6glc+6N0/1Q/cKbIpTO3EFySw7ngyi0cd5p0pzgope7sjONBTTnTta0QM7+Nkb/Ht4Ct5Mb+Xp7Ezw49b0YFnj5S+Oak0+DdQFVrlJFk5WG6i3SrjpbQ3kFpCALvKP6uFxOI4/No8ef/HO9nVvRCwgSF5ePPv+4/iG9jIxy9s4rlccWVgqT6MigO96+oPgWW85EDWiFQqWeYSS6XMlRX66cjQYp84zWo9yYX4hSAg6jMkdBXz9gBPsv+BlrAq3owr5LjdeVXmZua7QYlZjaLTjaEKsX/tainmcKSWT964n1f9U7j7Z/Ged6TjVyp3/A3AsIgmcYcjs4F57vVsjbeQacxEITi3S5CnjEXoV0pXkT8lOrEkcUPdGSLl76O9+gq7FHGEDLeB9AbBlfCHNrHtTBzWzGTUruK0b5Exgvh3n0Sf3ILXiEDRdwewpyeGwTI9jMhHOWAPYllegfqAekaeHMmTXz3JDQemqEThQVFxMDElMbT3vfofq9Gf6YlHu2UCeYFNWK1WROwEi4nhBUkM/HsRAyefE4UF/k3+JF3twL10BvUB1QjzHarv0oZw3l/H+C1mYoaLZcIBDk44TlJRBOsUzzDBR9xeRSIImGQmUtUWng6Ciw5SsBJBYOKhiXQCfzTeYO6r4nNLJRLGHx5Pam4qb7/+Njj0TT3BSJaUbedcGIRrbGCf1TWc76pcCdF00bzyWxpUHth3W1PI5BQmVjKp04WFhalcjBcr2vg0/sKgPZWkW5bS4daBMKf3mdTV1fHSypWcMP+vk/5DgP0WCyM5yLz+CqS+cqeefb2vz8qP7bCvB7arbd+AgMDk/ZNx63TjxMDVgC9zbtosgpztrz+Kf0QFbXf+idWhuklTu4ghFyIwN4yi3duVoBfsgkuXYPZuH8mEQ+MBSJhzzOnab82rCr8f6VIVMENi+3LNgobtg75i+yBQuAajM461i9MkdAsKuq3duOiiiWx4EkWMOC/kq93Cm3+8w1+dcKcblJpruQX2fXR8KEEXAwkafYSakBokdu9ZIoE7dTqaXDv5x+qFtyBWrBEEMKg9qJ1WziNurdQbnX2b0Ep/5mybTfb8f/EbLiah9Jh8KDZCph4q9V44rhh9tAd48+03+QkwPHUv9hH39PZ9RES5cBwwXAvEPNmWDwrs2ctgryogmcGTz9ETXIE92DfBbzTt+lTa+r2MOUZLfacn2IlpZtZGMvmrXgL+ogwxiVUQoDL1Kh+5T8bQ4cLxrmiRcLREEEjKS0KtVXN5mJTpcmcyaY1xG9XN24hxzSNW6Qyc/dmuQh1bQoGPmmGCjYD7YeUwWne4EJucQ0tEJVLEiPeWrpdoneCP7/qrNBSnI3xuZ1RHcvbMeLQuWk6++ipewlFeZfttc3vXn8we/it/+VRR1yPnQal4z/pJOxrPv1tZWJDE6DXvO12zhyaYmJO9a9CBxFzSbhJQbo16vPiyJpF2zYMMapt1228SNFFsalMxtc2DWfunUDCyNz68la7y8PHBy8ubya3N7If/+O2fByYD6iAXPLy9IXIeRC4Bs57RXT40nhjGpgWbyEvO4xM7ZYFmiTuBFU1AEyFNS4kOz/gPZ//v+J8Y/wX7/jv+jw1BEAgMDBQ5d3el5TIj4wSaXCVWNHzscMydN5LIb/NkzUNriPddgD2XW4KJlrWz8TTJeGDFz/zZPRp7sM8k9+VIgwu+12Pwtx7klqSb/XDtccdcFkRB02vIXcU9NOQYGD79DD/3zSEv+hoGs62yb9niN9HPhewfh1BdVo3wrM2RFgRY1HqVIHdvwsabELzF0oxt2htolBcwDmynX/9C6rRpIrtXlSsPbnuQplAFrpEODEdAGeLL/sfaqIo5itVwp5PdlTZSrqXANdg13JnpfcN4hnbhKhvOjGRy2PjbMKNUW4rxSA0+J9N48Zf3+McoZn26qEZzPb+T0RO38JYPbLaKk7Yz5auRZN/JhPEZRCjFlTaCIJBYkMjA0yPYMqiECm+xx2hW+nDMs4HFwOT9k5Hc5RwApMg+4lJZE6khv5IpODNt72vcQPueDOKeHkt11TPExPRWWkiw4hVRy/muEJSeG+jn0Kvtkd3DmLx7NAsW/EtbkrgRs9TSzel+7fx4NJkXn5tPx11WltnZJRIpfa/0x7XRj8ajQzC/6Ck6vtH7aXbqxzE2Yw1Gq/O2avRI542VD+HW7on/04fBKnY22pWj+OHSV4zQ5HFP4g+YtWXglQASOW3WSLpUbgzRmAmyivudGRV+vPvQD9yfNYj4g+Pwd+jd2Ny8FuoH0jKyhKN9clnlAFzd634Db3c9o7T3YFU6PDCgWyWlqO9VXhpViEzdiiPYd90Ix+pdedLkhU9Qk/hgmZot5QHMMMmoPtePf++MZvxNU5zhIrVR8LtPBS+NreDbTrF8xk7LUP7xOsTbDZNorgomwWGavB5Sy2VVHAuvDqSmr3iOCILAgk0L8MtLpvnpn+jQiIXeblVNSiadZJSLFbVxHiBOdggIeEvgcz9QCGew733g2p3Dl2+sZ1k9bOmA+Xas0vM6JSVCF8v+CeEv7Qheutvm5EtkLjzlW4JGWc/H+UnEeokrQ9pUydw17RHucYO1HrDeKAaUG2Te9Pn8Qy7+PAdvtRbBoWAgQHGaN9evYsXeLAQv576ooboLnPFRceDJl7g84DLcL072SSVWDEo903waeeiOOLDoQKrEInOlzq2Dhpha/Od4cka5GE+7aRQVWI31kU10/zUDv/PpBDgkbfcVvEPtlpOsnP/8/8feXwdJdX1t//DntPe4uxszwwyMwODuElwS4oQYcc83SlwIcTcIhAQSSPDg7jADA8PAuLt7e79/DDC9u/NUPe8/9/2rer67KlWpWZzTR/bZe611Xeta3De4nAq7ub9o2wiCy9zJffZjfJXTEGp9zQb8JizlREYHp9uV3OXymsN9Bcg7WHB+CGvqvXC+Nx3b7nhSRz5HGhL506DBd94BwowN3ACVKzULWPuFjuCqEArTFzHGT6zwjO1ZC2UmXJorqIofhySJCbPynaPRnxzIyZdn0WG+jffokwCLcI/CEPIlezwnw+lYls4Xn7XG1MDFEIk/Nw7naNNUlv2LJNr4UZc4fikBzxZPhyS5t2cqf57YSXL3CRJ1Yq3BG90HqbzswfHpbaT4dAnV1ZIEHjW5aOqjGC6/k0EBQ4VqgxmaBpoeO8av1lYWuoBGEtd39wYnxm7qrWgemrQT+z4/VpVEd/88nnYHmeRIHlBgRBdegaLdlT+6B7PIzj4koACTr5JrhS7k647DdQFdr5ZfiG+/RFuRB1bdAtonewjHabs+54dX9yDz1BE+sIjK7lySbMC+tPhVDHxcz17PCqolE6m2B0tK1rjXI884z18jj5DidY9wbld9BV+affnmk9tocG9FmSEmsQEu6b3IvZ6N/SxmgIP9Dw8D44EeTRsmjHC9X0WJZTH/nJ1JetFKOixyLMYqbMG+28deoMy7Ac/dY5hhEfdShbGJw2FW/na6RIHpT8ISxWciSRL5/fLJ79dbFfK4TGS/+0ktdPwxndHHUzg/xBFokeQaBv4zmfi8eFY98xVyhRhWXDP78V15NK3qInysIbwvUxEQEICMclIzrnLVrYUI3WzcZWKio0cZyDdtML04jicupdE0o1SwazFjbHJnqkzDT6FwwG7Pa7P0oG5zYcJvtxIx9bTDdVf0jORA+GYKh3/LTLOLg73LEonU1nvOHFm3Q0/gQVX+NFqrMCkBG6FaJ0srTz2wiwfrdvF9O0xWi402O6xayk1QHxyAX+sIfDQiw/fy1Uhafh2HscGbC07iszQrvNh5KRwNMGrOEfJd7fZi5wiumXuTKJeTLjNIORP7sdKvFi85TC0K58vB4t4hM7UTd1BGXlYi05USMlML4AFA3bVjDPsnAeacJiDuIp091XhdT+p2GvXsOJmAMTsB9qaTPO4iAqNNrkF2fV7qq32p7xATmxqLAdcWTz5QG7D6dDkkfFXdzzDt7hYuHR9AbH9HRYSSnmYu+50mFH/ucuoSe20CPTJPfrvtQ4LrAjjaOI8bXYO92k9hjIUn94Xgu8uXI/NF0pivyyXuWLKFhqMVRBZH0mbywhbsC+k4wqObovkntYMNI8/xqUy8r3yLjAzA61oL+ul25bhWK53qc2iH1TAm5CInXRx9dy16js/ay+TQVi5qoxljZ78v4yoFg87xhBI85H2VDNX+T1AwvRlrycc8/+YrtH0l+nPRgfXUel7h0ddfojY8CuUrYnVemNJKVV0iYdeyISAW6IuFrBYLKl09rYFtnB68i3RncXWWSRKRlSFYk/JwH57pcE8Ad2sa6WhM4EU3DdrO89hWHQ4NGsVda2+jxGxkx5iH+MMmSf5W7FVMcZm8/kYCTtHbkXiZG4l/L9cQvln9KjWZVlYMPUZ9un3vL4nvH/yeRRsXce/qe5FmiHPMooL8uHz2TtpLs1ebHX0FTM5QFZfP3becoTTSvts2qKxdLO1XjzJlLaedXIAXBbumSc68LXNR6dWMnHUeru9qMkmiLLyMtAtp+Db6ojVfFI4b5FyAFOaG09kY2o6rqBm8E1uwb2XAMTK04FMoAztChGTTQxxA7u0h2MP7LeOpo36EZxUzAiPSNMcS5jP6IPQW+G3QJ5gN/tj2WwP4Vp2INSuIV9wqaDE0ckPy/eehOylO2YJ51R3MODUCz7FlwnHPXY7j7B+TKIisImSCuD7LjS189dqvbOuUWNMOuYZabFeyc3o5V3Ud+JgVfNYC3RaxavBE51Cyk1bgFVREfr9aB79IYe1hWOp+ivbcxST515ikadiCfS49B3guoJbjPeCm0oPFBDIF6JuYE3WY/NGDOex9lhKduFc2uY4j5fOBJObF4zdxMV4Rom9jkAfxSqkHThVB1Gkt6O3eV7sUx6VRi1g4fjq3xpWCrQyfuZtBXU9zrWsmypRSnFSO1eo7ZPfw+txeIPkuaYNgc+u8xutWL15Z/DfrI6+xyqazaYUiiXurICG2gIkDCjDLxAohZBLHRxznoX56pgxdSwdiFZCklPHVo1/RFAU5PTuBJ/tsWJndGkBUSe/+7qcUK+jkMjnBVcFMODiBSVfHEPmQXZNbIKNcwclif4Y1H0PRE4BtVbhSX82cnTPpBk613slwm0eW3PginblhhDIIj4hAlBYxBoyMiic4spD93YMxefjZdTeH6pGZFI46TGL2s4y2m2M+hmICX/iRq5kJ3P9Pfx4cJPpdQ9z9MC0/Tv6f/WhwF0nBAF76KlaMu8burnjuOHuuV/FAKNvtnRu31kCSYQ7br/9VptSyfvx+GL+fxJJP8NKKak0j3P0pC/Hjpw2D+ftwP2pfsQqktBbDauJSQslrcCNjyBWczVOAZH784QfGKZUMMTtWQgn3BYxRWnnMcwf9Gt7jQbtvK1L5EsGnDQzZvoJd03Yhe6IvZxWmcWb5wrMcKojl27Ox7IkKvAn2VUqJHBr6MSatjmfMHvQoRTJ0tHSQK/LFdKfqmT/tPAo7NRGdVkedXx05g8+z0b+HI3bXNazjBE96wNEeGKEuRS21wHVCs0rqJFoBc99/hkK/esqeHdYH9ildME4+TcPYkzyfU8aIew/iJPMBGx2KZtdb+enPEE6YV7E2qJ7PU/qqW1uboogv9eXOyjvo8a9HO7VVuK6u7C1EljWw9lQxhlnzGGkzCZVSF5/OjOf0zhgKf7gXv7dE30cmgwcSCqnXpjHywHiGzd0t2Ata3uerSxlcDXmOaOUouygMVNY+YkmLIUKw7XOfwm+azWgGdqFRpOFv8zzjdPuITalg7RY4u2coh4Z7YUv5CW/7gBUD1/OzzpMXY/L5RRK9Z7lJQq/S4zvlOB7/ElMcKLkH49PZrGAPQyTxqwzuPseHXcFkZg7itWlHkakd46xWXTTmRjNZLjPRm2YIxF6AvecSmKY0smH5UTbb7KXlTf4kn04g4nQGTm5dVD4pAuXHSx/EqjfgH+dFt7uYH5ZkcrLHP01n0IecjexmfZtYWdrZvoOAfS6E+bhwsL/YmxHgoZBvWeuzD1Xg39RdiXKwR7sHo+s/nT8mPUSubxkvKMVccEVXIdmRS3nLzYO7NCeQjCNB7YFMBlcTi6gKLuKxLx9jfEcpYAP2NRbQ2tbEcnq9irH08iX86e3ltxY4AjwMfF3bhUvTeegaDK2XwDuDcW5FSE9/TIumdy7Z+u6SJOFX58fyb5ZjDPQl6on/d8G+f8M7/ifHf8G+/47/tSGTyQgIEB271a1BrO9sokerQ2FylOs5NXk/1SOOUuPSQIwYV6PXhLOzyg0nk5IVZfC4t5iUlSk1fD//Tx5buxRvt39vSfpCZTjGDTOoiM1FoUgWbK3aVEYqyxkQD4tUoLTZKDfWLKGmvZ7Uiz6oDCqHD3r+5lnEFkbj8fmb1JrEgCrDegbth89SF1vE/MW/85OdTMU0zQB0jWaCq5sJ4T/Ap4LdjWtkNLTRvvtOWmY4AjESENi/kOrCUBR2TXgBghpeZsL2sUAdP079juVsAqDedRAbrZeQJeTjX2plsPNwIUE+rHkiLt81cHliF/PH/cNMX5FhM7D1NXSbFBzY9AqHppzisA0qJlksTJ2YRYlfO7tOxhI41BFEuZZwDX1MCYl1/ngaSx2uWyW1oZDM+BbDg3FTBXYbQJhvCxf1Koz/pNEy4izQC/b91jmVLan/4c2MTlb6B9KqO4wtKOznPp+SqDL6/TmHmKmibryncyCfXplLTm0TWqOcJrM4R0f7pFKb2MjBiLVcCS9linSvYJdkEi1BSezofgufa1qHJtIypYY1d64j+XIyZwsOsNNfTMKc7m5lhewPEjrDqDq5lf/Mu87V0vjy/sl9VDce5x5LLmlJYmAtvx407ko/R/OIc/ionxTsx33DyXrlfdotEAYOANHRqte4dqiG2wa9i9XNUQr2Tv9nOTTxCx5xLmRIR4gDSC9Z4c51d6KuCyDjqT8cjs9LKqYs4Cs0Og0p15tXA5gkDYf2DOaaRc+E1EsEaaMF0EF2nfu64t4fkKwSx+WihOiV/hUY/X7D863HiHUXZSyUujruH1jBCRV86VbNHQpx/moMtTxbFUdAXhwDFh2k06HzGIzUGvFQwt1ucMxSJNgMzvE8VCdxQWfFTQYyG5aYd7ULASUj8GzsAlkdBoueG7KpN9aOTtdOml/4htKeICHxdMNecHAyL56YQ/Krc4TfNbku4z9nwnjv8Q9Y6AqXZd8K9l873dhRpWX/pBRyu6MA8boVGCnf1psw7vYKQGPXB/Ee/12MeOUqX2YN55+cZUy7tTdAjYlfSmnBTOQbJXLVgdAIY2yWwQHKCpa4wn/mZvHQsuPkev5HOC+dnfhUXuZKhQ8/x5Uw2U6vKqQigNSsRA6OO4RGbl9xq2dlQAMH3GCTuw5/uSOp4VbTBU7t6JUayl0qvktJ7c253OnQ6MLC+FpqLX2OdJrneOpyC1Caumgf34RS6YFt8nP4qJ2kxXaw8Y0F+Oj+wJbhCPBG1CWGNKtY/utd1EwSGf8KfSmrZ4/kq6/HwYURSHMFM4HOIVQcG4Ny33BUUduxB5sBVmkG4/b490R7tiGZO7Ct1FHJVcRdKsOpu4Nld94H9LGXq0sCKTvjxegJrixx0mOx2bOqW4tJ6/iI9vxwgstmEHDbWPF5SRZC+lUxaG8ILqcX0PmimFjtF1JNwx1X0arHke++xGGdm+6Uy2el/dn06kIGPGQnAwd4yFW89+5PfNcGR+tedbDv0SXSddtG7nWWU2st4wbYZ1b4MuLJXH7anUzSsSQuDq8WjlOGJFAZb+LooLfIj9nLZ9aFwrW9qYeOgCtcrcggrOo2nrM9WCZnQ0QZ5ujLKCWQy0QwOilmAe/fb4XGBnwbvfCyiL2uAGIV48g19Vaw21c6AFxOL6Fo9BZilVDTOpbQoN4wNV9/lLALh2iklW+e+hzPznE2UB8Miq3ANbKEgQuOUWMUHSM5Znzr/BjcqmHJHWGcNryGba/BIZZDlEfAwlrIM4jrFMBf1ik0dGSRAnSmi/25ACSLHotvM+RB2qWhSIjM+8v6Hj7RF+HWMpB7rceRKZQEBATQY3QnVV6JOrqB385F4BNzRjwvEnf9chdRJVHUy82oxhcK9v1usex68mOe+fF+Lp2NZ/QmcZ367tgQYk76Ejb5KJcTHCuMLld+weJfvElbPJjaCEeJo6umLH5YtoZOlxbSvRbaCGH1js4eDeMPT2aImxH1wj5CRoPTADSFYLL23Yft8O4y8uQ3y3HpX0vGjO3IGIetzPIvlaFMaHVly8I/aIwTA3qryo0PFvyOl9zKKS3EaURAAZco/u56m30vBaNXVbPUriIW4BatH6WWUvZYyrDaVROg9uWSdhL6pL1snb6L2Zqfb5oUsTVkLrhGws4pvHc8gFumaW5W8UhyDe9N3InH4ONMPTGPTldRqs1L7Up9WAueNS6senoVDwZ9KtiP92RQ+Xwv8WTQor3I5onP7ISxP4UDPyZx8FmyuhOxh8UsVhPbZvdWcbkYHPtaS1gp7pdPQb98hiv7SGsmlRebSz1p6VGx4N5tjI4T54mftZk73WBxeiZf98/kVtkMwQceoKymsSsIZVEE5wPLkeyqeYsSKsg4kYxLexeN+Z9hy27HauHNwNH83qBm4T9bWXG/I/lwkvkEHy3OJK4UkoyzbY9GkuBhry7Wr52Cc70vsc9k37Tp5BL1KhXyoCY8qMNLLhJQ3vTNxWqy8EO8lUA/I3bqYDy39hAe+QcJrjtESMpf2MYc7e2ZLGpdRd7BNFK+2kMd4jNTYObz4hQyz6Vy/917He4JYOQQHckTPmRlC9yuEfulybsLUPu2Y6rVcinuA6CPPn/jW/IfVsCgfnlY+A83eyQqnCi4nIB7Sz78M47hqX8J553l5sYCrwEcHn0FjwFldFvFWOldr0xSA6wEyk4iszjGSfMSSrkl4zfGV4HUmiFIDvfet4nG1fMJjKhhzEMbsAf7JrWW0KhPZeiSQ+DbR4yMUjmx0P8Ozj5+jH3J+9nSKVZ8ewc1M/3lv3loXwLqikDkKqH7OfkNAeSsG8dr+ZF0Tj0i2CQkFvnpuPbRCu62jKXNSewx565xx5xwO7XNj5ByecD1Cjdx3On2I5eyZuI95DmqLY49nnsOxlFx2IWF4RWol/cphuzVB3PNUERHQj4hYfVYJDGe3u3ajSIhH67GcTCrRUgWa9QebO4MZl+JnvrjaRiGi3kCSZL49c5fiSqKwnw+nNF+Pg72sogyxg0oY6rVxUGG2eoSw4unThMVu5tdjU8zTiYC8VGWHO7yhrM6yNA0gkXfC/Z5DWLBn120eJ/ArzETa4kIAMkkiYDaAPpdjeNSUDz9rCJ5yyj35UBmP24BTg/6inldk7GtEM02fUFBSjbrSj7nfF0SHy2x+TAVzqypepCOM2puj82nU2tfYw+/XniD6aeHEZhQioebSKps90hn5e4BhB0bjubJTxySshP3TSQ+sIXP52byXptIch2jvkjK0v38ejKOo7umsHxcK4HXSSAAi12v8bTOmfffnktzQiOjbcUvrCa+G1hPwUN/ER5byRqt2EPLrbuYQ7dc4aNdM3Dq6cFZcoyzrkU8jWz4R8QPyQW5SMpUqn1QLDhCpc6Z2g6x93pzLPR/7AhXZEWM8O/CagccDO5uw1QTx07t6yS62Xup8N3lFE5uHg9sQp9wO7YEKz9TMW96w3P7RqPXqeA1cZJ5STqWjdgPI/bzRr0jMIvSjSbPBLpKlCyIWAG6r8Gpb+8Klmq42xXOV84nxtpXLSOXwFUGqlZXBmWNRRUlAsqSTM5vL83FpDHxx+INpLiLLVA2KcezcfZbPOoOGR5w9bofu/W333hSJxKm/k/jHiOczpbh6jve4dvyVB6lvKF3L3PviUdhI1PuqvXEOdYTQ24Qt/+WzN4H+mg5cpnE4XGHGXd6KK5vPcnAp0Q57uXmUZx66Cu+Vo0hNTCLq3KbedJZzPuzzrN67DmOdYCTLtoB4E/tOoePG3jK4DXvcjb11HID7ItU7+Si0oUPTUo6XDvwsbupOO84PGt7Zft7amW4IbY12NGcQ+alw4zIXELu4DSs9/SRx4J8viVnioWw0vsZbLEQ0XESrlNVZTIYeE5BYFExHV5hqDzsCEGSRKe+jaKCUArj8il3tReDhXv6VfBLei51Z5K5KIsTWgRJEgS2ziOwdR4DHLmFNPg+wcpnh+JvVZAk+QN9MhLdfuPJVlxm6b2f85L3x/xSd5Eb/VjPui3nq5ZnGA9cSbxCu1VUpMmvTOPcCx5cnpDNQGU+iyJF/+OD5D3wdQ0jKsGpJIZWu3f1+ZmfyQt6GU3wHhR28YhR7oJi2hG8LAoaLjoTNNpxzsqUEtXR/Wj1diPEjjQMEHc1GueL8RyachaDa5+P3d6vg5XPruTOc8MJrQqh206tTNXThktVLvNnfo5vSAPQ18qhrqsUXeITNFuP8vKugTQH+wo+QralP9WHLASmK7gy7QJZllLh3N7WU6Qln8VwzJ8Nh1MEPxNAajjK6qWTeaAOctskh+/OR3+F4yGgpRsM1pvyGFJ7Lv+EtfN1q5mm1z7hgsqZ+fSpZxzbdpjxcvjU0uux/ESvt9cGuANz6AX8AoBcBZzYf5rHUo1wbRUM+ZEmhQWLWzvfNkeR6i32GJdMXUz16M15KGsa8KvfAv8iFf7/wvg3vON/cvwX7Pvv+F8bZrOZ0tJSIiIikF/XAmipGY88K5zPIyuItgMsAD6TunBx6uQWSYur1CrYtEotmxefYvzeeKa//RzWLWISUSGBRW1g+TvfojP+hn2/HIDR7kYOAFODNyBvDQOvvq1TQkKlVzHvg6cYPOoSylv7GGrfV51AaqzkybereCm4E0w9N2UZJAmSkouxeHXylCf80SECjXWyEBTDsyh2b7j+78Wdb8nASwR8u53162bhFvEvuvIdLzK2SsalooF06B0r3HRybx5c8CtZQRrukg13sHuoe2gDtBGVPOrRJwFWq29mw8TXCagJYeo/e0hdKLL9Ivme7ogKurPiKK1wRfrMQ7C7urTgFSyjtt6VRi/7UnoLz8Q18/VVCd+/p5EXYxCsLm3ZNEbB1Rd/QSNBlfSjw3XP8LiNpYP24VKo+Neefbf5NjEr/irWS/E42yQUblRs6bYn8bduEON+ERNe4yMHkK9SUB/8N6V+TcKG7erkS8nqJYTWXOGN195gmtfTwrFhxgt8MesJDnSqcVHo6TSJQVNu/UascS+Q/Om91A6KBuyqySx6vAKbuOh3mGHnHJ2UNmMTza7Hye0cRmDdLNvWSFTW38+I/TFUM5TW5mmk2xznb2lgb8NAtvTo+Dosj7lK0answIkrBvhg453EeHYhmyWCEtXXtCgPXGXTgbno09J4V8iCw0S3/zCku4T/nAykcJGjbMI0+TGu1vWyMV+SeSOosJt1HA/x5Tf3EtpkDXip+hh9VU6plGzXE9nhRtvVKLqneAjnHd9zieQVK8hOqaQ55kn8nMXNdEVDAB3mAhY/tIYLmiABKESmRDX+NPVp51C0KJDsHreToYx5PV7szerPvaFx/HhrKvbju8AWGs3gVAgPhk7Flu8d4j+K7nWXmJv7O4kDC3Cf1Ndo+r6WDmr2TyJn+U+MTKwgrzqYiJDefiiSWc9nJyfScDGe1x/4BjfJnR9szqu2dHBXYSIeF2KQGQxYrOLcT/UYR1puIt+5f8dmv1Zescvklevb0Z4YwMZDnuiHFjpUC/hFvMhfl37Fx6WS/ff+QI/ubmwbSReZQik+a8CwdhifTnFl2vVgTiFTcPW5bfSvMnBl7CNYJRkym98+0pDC3i3OVHjHskz6lKfFT4ey2CVI711A+/7DnDyTytTF4n1pZ5/g3Lj9pFWFoPS1Q6PlTsy+Gkt8qYYXwjuoCHOU7TtlicIcV4RTv1Kmqu0SwtpAzse+z+b4OXzQ1cSr2re5kZrSmvIIuUNGhfdmvoh/hvU1CxjBnzcPvb8hhSuGnax553Ni3PRIkghOxbv9jC7hC0L2exMtzxZskkzJuaMZ+BlkeNyzhRbExKfGJZbtuxfjTR0p965E0OG8Pjyb/kP/8O28GN1Dt74OW7BPZqgjNe0U6pZ6qruTsBWN2hl9PzVulUx4F+4ecIk1d/RFDxaLmVl37WLL9mFwFjwMHmAjbNPsdgdv7bid/tnfoK1qYJLvQmzH4sACRkTWoy7MIaDrGo/bAPgAtCvZ9EXvMXVujn2AgtwG8ODVKRS57kFr148HQFU2AM/VHgTdegh5el+y+TzBPFLnQfCoA5x5IJvdPXPE5+nrREu/NBoCetcoe8AtR+9BS4Mnj30zleYkUboUen2A5B4Xtqu9OOAs9kqROfmS7bwAaeGtRKSfpFu+1uH4T/33s0UFO7pAbjyHbUULwAzXNh7wg1FauKovhOuc1Djle6jGtbG+UYlBsjqACo9cXkue/gjlft/hZPTDNuQyawJ5Z/UkYgtj+fL1L6hyvSjIoxslDds+up1pObEs719Mv+l21SEyiS1zt/Dt01uYqHckYKnac/lm3jmyPC3cl34NmVWURLvxjOUWZ9Qyl5t+X0hYCANrPkXRauHygXZ8uiWhz1WsrAGDp5kibRtZ0e8wP1kMV26ct8C3DrUxiClqcU9zaXMitCKUU/0mE94pzk+Ap2I+Yp/iXq4dGIzTuGIHe2TnqxSPLSW+1NEnA0iv9STl5HD0QP9b+ypTRoVN4raNO/DSbGHU8GMc9a0RjnvIo4VChRM+zi3cEvcn5dJKbME+bfRUvnj6Oz4MzyW/K8Lhnq0yK01W2NMNvm4egr2qo5Lj8QsZ61HNzJ5ZhGodiTmfVmTyvsUbGY6kHuRq/olScjFqI1hhjbLvPb6adJHI1HZu6VIjWSVmS33+s0yS0enaSadrJzXRnxAdIc7PJLWV/g/8QVaTLx+H6WiQi36sVQHHRxxn5ImRbD3enzS7x11GFD9fxxHSjOOxH66dmdRpffn6t3Hs7e8YN/Q3H6fIEsr3Bh3VLn3EBJ1Lf1bvyGDY6WFkv/0xOq0742yOy6pP4egqLwpSsyhNvYjJTtZ9hyaB3Gc+ZfzKJ5i9dTbyFeKFfxxfTNetu8ndMJX++j0IYJ8k8VNuLNXv38ZkTmK5Zzw3gavro1QWzsoLgXiXeSKNdJyDUw6kMnl/b4ySZQq7KcF0sukDuiO+xs21h6/iu/gbcY398eRw1JvjyZh3nlvHv4FJugdbgN6p5gSedVXMeHENOZ5KbME+i9yJtpQc1KUB9Pw8jyGvieoACoWGjkG1uPRbz4JmA+KO1zt+KTnBKk3v3FriIoJbiu48XvroA7Z2QUWNyARfXR9G8ZUIvhz3Fy4uPRy2mm4+M6vVSvHgedRqvkMKegeVPFXw3d0VCnRVMoK0ZsaMOMM2SaxIVPVM54dH/HhwUBaNc/Y5XLOv7BYuluYQcEFOZ7jKwW6Ue3JoyCn6G51oquvvkKyzRnYz9MWfeKNxGNM677+pXuFuyOOdQa+zMusNFp0Yx4CEBKHX4HbTEO6uOALRVyH6Kq9rxFYOR2tuJSm/9+NYEOtIYv3BH64ZYID6MOvbWrAF+9A30983C//ZVYx330mF2r5xBozUvs+EoauIrurhRZcwIa4AuKBYirf7TyxbdJQ684Sbf/+yfgC1pQZaY3PocOvgZ6Wo37g6uJHOhb/j1O2ETPIQbF5uEezdf57usu8YfMnIhX7iGhrRdprjIbD9kwVYiEJ6KEWwD3G5wFS0mL6+jX6Dr9FmJx7QomvmauAD1MmbyI7OZZxMpA9kKafySNFm/BUwoXM5L8iug78yOZWuhzAYDjP72GiSZ4n9pFSWdgyz97IpI4uQvEqyG4aBqEGBt/F5lFM+43hqCxfNNdiCfcu93iLQuYv/fNGPHn9RAQiZkrdcf2FUehr1uYFIGd4O2kT3dNdRuXsGUZVFdD4k7mlmtR87Ey5zt0xik4eGKqmRG76ir7WakecywKBieWEcijliQlYtGQmQgXzXeMx6JTpzN9iAffHKZpR7h+KcH0Oujx2RTlLwqL6T9uhL1BkhUSnel0zlRZ5hBiFPaihoGo1F6bhntbin8frk3tq24yoROAj2HUim4QztHRbAKuzV75qauOzdCxqtMMAphQhKvHtkKEP2phA9qxgPVSXYtb/oBfrAybWLZuUXwGc3bb6Rz/PYA6mUhm8mJ66Qe2Xi/Da6DWb2lt/4ZfQDjHQSk+AAkks4q879Tsaon+kMiwY7+fNkKZeXA+DW5mac9U1AL3ndy9JBniyQP/YvpfXSVvrH1WIrgWtxiqZGSqIsIJPa/tdQqsXA1iIp0FthVWvvf5sie2OOtvZ2/m9T0f6AS5Mz0e0+DqDaV4pkrryxgq81Q3g95AwY3gRlb7WvWe7CGye2UqnaTsatq3ggpo8c4GZt4FkPuBhWTfHoE+RKsUKGToeKLiu8UKnji+yVrIq3ye+ZdQzWyNnbZSVOaUZrMTpc1+9+9/Pu8ffRyiC7JY5ZwX2xVoslgc8uhiDXaRg1sJgk9d/cVCOxGCHvM7zv7Me2smj2y8fysDTqpmw7QIexhfTM3nlpdhkuADEvDRlHsHMOj+w+gz7sT2JiJmA7HpzyAufPDKEg3YRT6C7ABimXOzHtahQPXoumaUgO6b7iXiuTwdK/6ymN+43iwdt5LVxki0Y4HSGh/1F8nCoxOSci+B9AP88kvHtuJ3PI9xR0r8QW7Lsv7T66Ng+j9rOv+H32YQw2xLMuZSglHWo6nTvplx9HV5tYhbnGkkXDmIsElQeRtHoJhrdE0lqD2YnudmcSr6bgZhbbYkgSHO6fyNK1owntfhaP58Sqb6+QOSz5xcCkTKhtUzLluZHYjxSn8aQON/GStZU8/SOA2DNkzJzD6C3w6Z8jmfd8HwnlR6cj1CV0keHSu/8v1Iu7+FP9x7B371xKU0ox9xNlfw09Rax2yuLXcwPQ7xxPxWyR7KlXanj/hdd4ctNiDr67FM+PRLQuwvQ9I4euJd4QRIdd1R4AzmH8efF+otep2Dz8EjJT35oAoJAMRCrhmQYD1y69zNllvUQTydxFVKsbEecjWBNQjZNiBG/TV9m3a99lnrue7gsAXr7+37+Nu43w6vFC+PZraL8GwbNI5wl0bS7Ufn4X51MLybARVZIbWvglqYUPXv6KKzu24urhKBP+/8r4N7zjf3L8F+z77/hfHR0dIjg1vuVRLMfW03RsEK7TToAdqKDWq5mQmUbKvgkEPCoywZM8I/lP6wcUuq6l1bcRuV0mQ2Np5kpHHBUnNXwTXYEoctE71unXoXx4Fq7hzaBYIf62pYVH98yEHi3blG2kavoq8EKLfBi2bxhXAn9im28rs23YTJIE9806zfETiSx/YRlV8/SCLFmpPJb3xn/Cfbtn8PK6e1CNFCXRrlqfYvWlwbwx5j9oTUGIvE441RiF9oQ7KVNPkZ4xBfshkyRCKkO42OJOcnws9gXiPyqTOLFiBVWRUGfTIFhh6SFBBdF5ScSfP4bXFB+gT39dGZVI7D25XPt8BElX/BjiP1Y47wrJi/33v0WyCjwtohspl6sZf/k+usM3cesjjcSFiv0FrApnDmdGc+Rkf34ddowvE8Md7utETwhmfTBPGMbwcIRjk16Q2DJnCwqTgjdkX9z8a6KyjPCiJFSnI8hx78Cts4Eo+mQXFkU+SE+IjqiIEpxNoay2PaME7b5RmP1O0hkNv+nEigCdczrfZ36KzO9NxrpKdNkhSKG6gzxVGsUxQJ30DfZgn2t3EfkR8HMbnB/rDt2FAtuvv+UEOWGwrP4UV7pfw2R5rZc5Z+piZP+zVBvrqLU8S0CYCOpGuUdxZNN4fBtdSZj/J4oU8Ulp9WYGX4unszyAXE0pAXbg6Vb/j7BOLiewJgR9hBr7JHVO821kf3uNYcDVhY6yCs2SB/HP/8R7+m7aPewkYqwWwozVJBSkMjfxAlmqPidKQmL97esZcGkAI06O4NpQMWga427gyuBcenqG4i0/iVISz+1T40l8aQbbknJwchIjfrPah7FV8MCBibx8bBja90XteCffiaxxaccpWM/gsyH4OokVtwCvNbjTRjM9VjBJjttpYOEx0KsoqPEiTdEHwuhSqkmO+x6LRyNPeMIhc1/QI2EhUW3ltEnBkDNDaAsUz+tmrmLUqeFUd6upWRyPKkQMrj0Mh+l/7Cgce5Kts7YizRejHo1kZtruaTQBJ3DmGbtr1ijUFGfch2fgPxwuXULGBPG+j3al0X1Fj3P8VQrDWoHrqF1HIQlj66jf6Ur6rrdJWnSWCtnOm8f5WoajPO2JMsKdzsAyrOZQsOlKssiziFFOeh7yDUFtScZT6yH87g+hOvKuBrLjt4V4zLeTB5PJKSsKJe330fwCTNrp2Ns0UxnB9iWfY46BOuU8B7ukkDNvYwIWQmFNH7Pf03iMx0c8wKs/3s6KVaNQvyuCT1VdZZjbdfzi3IW70cCXdkHmfO9iLrurqB2zklaDyGq+3NXIr6eciCkPYeELa/lbIb5LmQzy0tN5ZcQ0lH6OIDpAUOZpjGue4M4Xv+a7W8R1tltXxfwZP/HJwf68fdlN6Le2z3cxbf7XCMhfSpdnAjLbb16hJcZUR/jMLZQ+voXTHU9jC/btq9lA4sGLNHubqJm+kukq8aa/a4/jvYZGJu+ZSF2AvZwaHDAl4h1VRHWYmqkRjskdSa4i6O1hBDGM+I8dq0NGeNxKQdsPPHnFm5ikRTevrKGnBqP5INb6fuxVL0MZKj5va/0CNi3axgPbB+JUNRVFhrie+FS4ccePj4HcwqCBh8G205XFyJm2ILZ9vpjVgGW5KK8rSaBz9UNv3MiFM0bmT3S8rwZTKkrrYSZqdVzE8X3OuxzMwd/v4a97N3DfoL6dOlDeyphbTvPT2QCm1oWgRGSzDjwbQca5Us5l3EFttLjnyGQS5wedx7nLGd54jIJlrYL9smIkWapNpAN6L1+UalGeJpAGpjvByhZo0Xkipt/Bqg3kO2UL4TP2o2i6lX5ycb2KtZznWjiYzBfxleLBepmOjg4Upi5uu6qkM6uL7NHTcInyF8gYPhp/Jiy7QOOVV5CXaVHYfVfBxlYeyE+mKDYKS/cnBNnFx2ETLtA8cRNt53cilxyrRwtdJBQDr9GdmcRZQ5pIBAFqrb78cT6Kp67EoB/rCCB93b+Y3cMu8ntQMc2mO27uiE7oCa26irwthOys29A8KCaqJfel+N97jK37vdmx6hH+k6HHlrXzqLuK6tbBTK7owcuugFllqOeUwpedm8dQ3uqE9LBdn6D2S3xvrif3+RX4Z1zB+w67EwBg5StfeNgDSs3NgFh1kB1xN15NXujV+pvEKIAN7YGYpXaaR54lXgVKS9+3I1kMbHDVcrZdSatLIU4KH2wTEVaVH5s23kvLZRW3PdvD9kSRGR+oasQ8bz8/jzpJg8XKm9IqwS6XZKxYsQIA/UL7mqpeedM/9qcgZSfy7sBSB3u7zJ/Vb96HEnhndd+5ZZLE1YSrNHs1k92lw8+YLNIpLHLcmz0ZenYIcTXBKJaJ310rLlw2wAMzjjLExQx2fSOrsuKo3jAZ2SNruRJh5z9IMj40et70AOXWUkBMhF9T9MPjw2imAhEjxaotAHN97xqz55ZGbnfvE2sb2plDwKeP0jDmBE977yPITwTUzps86afp4U9zBeeuzeQ9qziPeuY/T1a7kjeVVUT3TBarWtW+PBhyhWlRYfScHUKwXFwRVEo1664eJ3XPKsZ4udkXul+/9741yh5Iv2jx4NFy2BMMiSGiFJa2LATL0SHUpJymRdkjJHTr24p5bHYMu7rg41aIs4jEhDaXW9j4bQ0WSzn3KdW8NUqErUoVX9Do8QuBJxVM9dY5xJ7XaqZS9UUeQ1ugeoEjOUCSydCbZbgdG8qFBMfk/j/yJF7Wn8FFOs4x3UfcqK42uaTw7urv6D7bjUvYCLyjRKBF1Sox7uA4yqKLqI8od6i8Huj/OMen51I7eBB7VXJsOzvKJBm3lgfgsnMg3lfiyb/dJACgTVX7eTh6GVuvjGFn09OMdrUXPAMZRlSSFQmE9eDGKOvXxN5xq7ikgoeUfbFKSstogn8eyNGx+RQkqfFWi76JrzWJTqmGYX5dXNdCFu3ndqGtNxDy4Uq67PpSq8w6zNnxeEbUkZBaS656FrZqC7GaErRbx3O+MIx1Hu3Msbtsnb6JtLYGog9Oojiy3AEY6JZ5cckAns3D8av5qs9NNXUji19E8vqZyGK1VA0VJeg8dFc4GGbl46xIOvaE4JP4N7Zgn8ZaxWeT3+LqZQvbGzQEyURS2pHuQLoem08MPRisF3FQcpBL+Nb74dwcSbDLPQ7PbGr0NX7RTqA4J5oOo9gPUCbJaPVoRXNkGA2okN3d9837WRtY8vFXvPX7Lfif6EfgnNuEY9Veq1n1wF80u5zgxKQT3C+JqhzvNI1j8vZgcvrncHqsqBQiyRSs7YCwsjACagPoGmu3UbtEsCbvd461fYFefYbbTMmEI1ZqfmdIwEWCTuu/E286pIvMNj1HSkI5ktSnPiBZJOQmOUqjEovMgtyO+FgYXU/9zO0sOTaGIMO7wNeC/e95/zDiWgTJ9/9FsVoESlycI3A71844vzTWLruA0UV83rW6Bvpvvcpnfz7L9juOcl7ENOgxdWNqXU5phYIfkozcZteD6wIpfFe1i6+izuEiZXBDgUVSurJtx220XlKStCCbwCSROJPXksPau9cxwH8/LSHwU6coau1jbOOefVPxvZhMXaAM96m9e4i7mxu11aISxv9p1AFq1AwK2Y1cvlywmVGgs8LVTn/OVC9giLxvjnfoO3A99S3Jna2Me+ci47R9H6aPVIv/+kXg2szqQUYyXFOE80bSiWfmYNK3T6Mi5Bwdj07mJrHRPZG/igrp+HEnSyobSFm03+GbtmiDqDYDZmjq9GKG1AeuNlsGsNK5kfC5f7E2qQIPuY1vbjHChefoFzGAY+69uYkH0XFDiQdglHk/rkuLqdbKeH34Ckp7rnFDLSq7ZBzZtUl0NvuzrftFFtuEBTIZ1EapaQw/yBtBeZzo9kAA+4AO1x5+eexLJrW/yVgPR2CL+i5GHtESE/0p970jgimBLn8wM7J3Tpd2jcQe7FP15DDXN49NndDWkSjKXR6YwOBuF/bWpbHvYCoDYvryJ0oM9C+KwaXLBY1TD3f5iJQeo6eVw+MOs3DzPOKuxSOzijHz7trv0P2axaLCblymtAjvSpLg8/A8ymsW0wX41NwlHBvlGUV82WMcHHsL1f7ZBLQOZrRAE4RMnTPNv2ewbEAR8tHHsAf7UoObOd/oxaCSQaR59dGdL/b4k1/WjsbYiU6rcyCDmsLKUC3bwJW/JvFLgyvTbV6VReHKj1/Ow2BQsPm233APFMnhGsmIn6uOApc2SvRqRsgjBfvaLl8av1/MbdfC0dznqCxwRW9kz+8WQmt9MYNApAZo1A4i+LpasCxiGZJ0faf3HsyiY0nM3DGOqSkDMIVNvfmcATo6O/7/Avl13d3gngBjexVqfmkL449PF9Pq40+pXSscq9qHedVQ4NxAztzhFCqX8bRAXf9/a9jjHf+T479g33/H/6fGxJTPcPI5wdsHh7DLKRz7lO1nh6ZQc2YAHYMvUGvX60Smr2Wx+3OY0xX0eyCP9RY74Euu5e/No9GVhnD5tl3/+vttkj8b/Uv4SgdtHmLdu5upnFsimtnfVsr36ad4Q9WXvH9o5Bk6Iy/yRrOaX654MsdGZk6SYECxC7F1ajLaXemxqyC6sWhHWpR4GzWorWIiJKfJk8zzzvgM9KHb09HRrdIG0D3nT8b268IiD3awKzCy7KdllAMVb7zu0HvmhvMcXAIZ2oncgFDdO7L4ozqJv6oCmLt8Mz2B3dj2K3mn7jyb9Zv44KF6Hgi8QpNuAbaNuZutvTvPT/7gJxNl9WQyOek7FuCSE8otL67llIfoZOpc+/Fxrj+Tz6Uxq/sRhjwlbuYAf3b044zxB2qT19LU4gp8Kdhn13njV5hA3rAzOFv7FtmR2mx05dFcBL5e/gWDrXOE49Ydv5O2o52kpjVDmDjHDPpmnn7gDs5Vmdi4N536ZFEqttAo4/jObqLzHucz70Zm7LcLANQDuFBTi7PczIxUMVkMoHAJZ33eA6gDv+frAD205wlgH1KvkNHD7jBn6FtYGyaA/xgw9fBaRj7fxOWTe/gD/JxElqDVPYmzkY8R6voZi/9YjCZSTFaM6Sqk34ZbKR57jLVjDzBO+kCwv5twhOSBtWRUgNaYjb1E0d8tXUQBiUOuMMBNZHkBZJLGj04biHKHMLkdGKhw4rPjR+H3jXzCODq+rWTsdZO8IZeE8jquFe6jxGMfli1OvB3syrL77ycgIIACzw38HWok/eRKNEeLryfb+kCLh9t8ad2fQnVQNWatGGT6Ofsx7dC7+J1rJyTjMk0qMRmndfKnuvJhBpi/RqMtQzJ3AuJz/atDi1EOg9WQoPEQb6vlJLct/4Wv23X84J/NVlUfe/grUzj5qhySL8Ww5zK8+lTETZskd2Zq+gE8ojJ57LMnkcaJ+u4qlyRyh/+Fv1TJV+M+49OOB7BldmqMpxm38ACH/pzA4HODHYKeSerTxD1Qy6qrvpTGOSallPpixgcdpHqvkp89nmPwE7GCfaLLo1RnGtEPvsj8ARdv/r2x7hRTx65gp0si3scSaPOtFBiOQ+JnsX7qGAKvbSTk1Bo6WsbAzTcNX9T04/N6FcNuuUSGdzaSJArz/pr9DYXVB/BaspWiQDv31GqlxaeDoqginA0ZWJzsKvfoTex6NXkxPf8Onr3tFQRIo6eG0YEZXFDOxqLtz+LgvgBgX4+cl86HInW04ewvx0MlylENLHYm/s/l7Jy+k8tJuXxl97zDLv9K45lkXr5XyQCtKBUrk2DnjJ29hIRiWJUWJNi7dDUMzniB17p1ZF7zpx3Hkaw5SAnR5OaGICnF+dkq92HiBQ9u2zwN30li0H+XZMDc5U952nhiFSlCdbSEhGSRaLBI3FJtQW3Ovy7u3Du8LaW0DbhKq8IDhwkGpLd8TOsHe1AZzPgOueRgHxv4IJ/MW4rK4yQHu6w8i1i5bXvKa/pQ7MegmJ8xR1Ziykqn5da+6/Y1FfLo5kUYWzz4Je0Bxt0qrt9nCxWcXb2M4NLe+eH6gihL9mpIDjX9nYkanUtkShvwpu1VkVnae74LKRfocRVpM9Wlf7Nmwa18fGwipZc/RS0X3yXAie4V7N3/KpJVwaqVju535vk5KOmi/HwiklPf8T93zmRW9Wle3Tkd9xZPPBaIa2ygez7tWBh+PAZzl/i9BmjdWBh0K9lTT1BYWIiTj1ixOMZ/Dkcih3E83BmNMYgxdkUrI8lifmkyf309n6vDcx2KS929kij5YjcttcX8PmIZP95nl7iVZOitYJS66aQV7xsVRD2VjI34hf3XJnAwbTIKp0igb63Tu2TwxK4r+FuuMk71BS7do4E+ZvMAQw0Nv81n+ITLjJ47EMlyEuhb45e4NhDd7sG67ccIHH4cEJvev9nYn5IpW5FP3MEsmZ0cJnBel4bfZ72J97T03Q72XxRWihds4WgHJEp9vytvz+KF919n5fnp7KxrJdpHJCJVWm/hr+wZhJQtR62XY7SKVfRJyvW4FmtY9+fdXB6ZzyIbpqy/cwDnNs1EcTGcROceXGSilJvS2oNfkze5QO7Z/gxXiv4JQJP6YaIk2NABw+wkmump5Wj+cA5smET0kMtIxlZQ9VZMXj45gOR901h891+8PaScc4Y+EFPCirRxJm5n+zN6zPPolHHAypt2qzaElsu9E+v3j26n6wPxusZqdUzRwhftM2itvQM3tSiPN02+m1z3WHraXEmQbQBeEewG5xhOlfoSqzRSlN5ss6v0jhpFIjfm1kXDqzfXfq2ulKfTyzl0VsPIrx7hzAzRL9IG97Dv5VVM3TIL/zODCbhPlJJ10psYnj2Q2sh6GuOr8LMDQ35v8mYMcOpcP0p9HYHXHnc91R+v4Dt/+LGtB3jf7l/0fivO7p2M9Q50OL4j7Rqr04+SUvoXCc4RN/9eqg7BPOUITY2utGyYgvUhcQ2tj+7g5L2bmb5rOgdbyuxbvfFNUi17O2TsLVMhU4r3dCNe+Gf6P8j8djDDxxEAklvVdPvFYHX1cLABuLh8zHeeEKIAVXcekHbT1mo2k2Po7VXl35lhU0MPdw+4AAmnmLF7GtUhFWLi32ohstWLoEo3QrX1uCrFBGNx5xUyMxoYfFzDA98/SNpsUdNArlVxZfgEBh35nN3NGqG6DsDa3oFrC0QnF+I9sBD7IcOEfOBVjvm0oHdzlGSVrDD69DAGnRqG/uW+Hl7NkpaslhaS23WE5+zCWu2CbbXXEvUMzh/VwvkhREVVYY4RpZQHeqwnetBxlkgGFDrxu9EqtYRWVuB14p3ee+wW97sSi5rf9ibhvm0Q14bNZcQDjnKDH7fGsLdTxyJX8MFRmhr/2xlt0bKm1ciDyj7ZyoeiKjgz0YmRhcOY1VZP3H0i+WVNpzM5hx8gceRB5BEiIQJjB0MnHKeytZP7grv5vl0ENAq9x3Nsk5JgvSsznv2UwmKxR/Pe1gcot3xKeGA120dlMddufqtMzXw1I5vOEcU85t9IvlmsklBixFMGciyiXFpHAScjulnR7kGhUsfWLnchjdyjjeGRPHc6JD2Dlm6i2s+uPYUESnM3l9Ys4kzGGR6ZKz6T3zrjiBiQzUS/Lm6f9yFCf0RTNyVB9ay/Yxd/5fxJssLRx71LP5yqRRvxahjBTPkywebZeIjmeCNPzdrNWvd6XrQh7oYF/YfnN99Fi48fumnORISLe7in2hf3xhaiozr5z9gSQfIdIEyVyto7PuP3+GYWe4rfnYRESEUIS1f3gp49o0RCptliprTkOeb8486+SXtp6pkFdmDfT/UenPlqGVbvZlRTGsFWAUNXz8g9v1PZMJI23TlibS5tToGZeWt72QbmsCa0S8W1yieoCZ1vKR1nhnNR12GvP8HVxCKyB5yBNrjd165Hl1VH7bApJMSsp7o9Dl+7Z1LWepnaxCuEXxyAf7kjUdTcXcVoVT0dHQlUeX8G/CLY433u54czZXyn+o0Ya3Bf3z2NDwdVz6EbeYS3+72Nqy4B26+jp6eCO8L2I+0ay5tHh6F8WVRNGm/NJOtEb/16TKuSAFVvzDB7yRLWvfsut/9fSHn+ooABGZHkGvs5uP8hli6iclKozE/mu6DlDHm2794bOyoJCDxBe5M7C2sgUTP0Zn/dWimOuoJ6Wnw8GOP6KSmpon/8ivYMv2/vlWGVUq8gt5N77c4twb+yAbNHG8cVFkFlB+Cx5Pnk7NHyvfV+JGTYFtdIEnS4dhFUHcS9x+WMnPUEn98wyjW8pZnOjqvZTM8fg9+IC3hIedjmwrokd9TDssjpAff6ZOKVfevw6twmhu/2xTdiDQb3kUhSivC7s1qvEFDjRt3GuTQNbWCizTXLJDPLfFsJPBuN6+nLOA1XgU0HRpkM/J2ycW0uJynyJPKrYZDWx6452DKbl7u/ptUC8aZk9tg9E+fWrXwae4S0ctA0jxCJv04hqEMMmOQmagJrSLVZCEe2vseo0UWsOTIGfVAdmWofoZJ9qvISTwTDofv/4g1vWIu4Pkd2p9BTeByVs4E5o77BFoSUJLjdzcLW9z5h445dKL1E0EzRdY2JuvWk/DGG0mev4GR1VDvb1TGY/toeCrUdqEyOspG3FPsz9O5fudM6B5WpghvryW9FQ0h4vx+PRZUS1r+EnFRxv5xRF4DFo4KH9Wqcu0SihlXpyt+jDuOkNLNmQj5Hu0R/b6CUy8tSAK+ObaOyZDspnqL9VE8HekU9M5x8WRC5GVjhcN1l4WV0uHZw3PIHY1WOZFKAsVpIVJjBbABFr8/dEOhMcVQtcRch1Os4sOrmd6vUyPj3plaOow5wchJjhq0qZ1pGXaVUMQ+1r0hWk6vc2FPvxNDTQ/k0tRRDrGPO87/jf2b8F+z77/j/1DCo/HALdcYU9ikR9okIIDSxhk4lbA+/jckB4iZgUfvz+8/LsJoa8I3Ow8VPXIxDfZI5EjmOIaUFxF0VWRU3hr+2kEgFlJgcmRO+PlPY46diw9KFWI0gt9n8NP6tyH2buWP5a1xOuuzAPos6k4pfjRfnllUyPEwstY815fLZvlnkeDWxa94WlivE2r09eW8z8ng8g44/Ss3sw44XrZVxMLyQ2pzFLA2aIcgQAWgxMGrOEc41OXGGNWDXfSbe1MLE66zpfz7sCyQNThFsvhgOBRHsWlCGCldBukDRKjHm8BiaI3wpc1HgbNdHK72njUkb7mFbgxcdS/bzqe3PWq04V/RKhv742QKK3xdr3NzUblQMcKf7YhfRxafQmFMAMVEuIdFmgYcLRvDMtHuxT2stbHPj8uERfBBfT6Wxr2or3/w8lxKOUjtyFc/5GNHYNe3+s20vM4rHMqfUj/QlYvKnuacJT5er1BUMwbBpGi4viZUj+R0XuRrzF9F5M2kMjiPYRaxIbFFG88G0bfSbtY3Nzf25aHfNknMoP+Q+SaHqe9a2uLLHU5zjeYpRzCtfyWxn8JfD+BuyJCoPhhT7UWVp4NO0icQ4qYE+AFWSoMMnEmPwMFKHbcQYKjI3a1zdcL97M1dcekFZe0ZTrt6TOlMtp0Phqk5sWgxQ5HSO/S+t4XU/E+3muQ52CYmky0ncd3oUQ5bau5xQ6nsNPycLXa4KIl0fBuCTVat44fkVjJfDe8ZeiYFauln77ru89cYbvP/hh4SGPU2nAXIH9uCuMNBl6gH6stVzhhdR7tIOq+8mbKJYMeulUDCp2UC7TsNzIwq43cVOTtBqQatvIXbKZRaPWEO7bia2vR0B7u8uZrunmrNhcMr6F9jU3bYZ2lDGXIWqRGaetzLi0T4mrcwsQ2VSMeafqVglK9KzNjJuKicm5dTQpN7Lgpde45xaZN2H+QxBf3Q5urLzvN7shmG8+F1822pg84BjvJtwlQVOViTeEeyV1mCaBu5mQip4dTkmDDobtjPN+Vs2lt5KzpANtBt+x9nZ46Y9XrMDp4cz8Qg5wTTPPke73imOcWVQ5ZlLy8xc3LoH8o5NbB1t+YINdz7Ph4dHMUZnoKi7FFuwr7k8iAk/TSRoTg7jB21ChyiDWHbIFdeLgXzwVBERrnaMZ4uesqHnWJd4jjcvhjCNWuwled50c+HvLx4HwHSnXfBt7MQ3MxpVlS8PvfcmmtoLMKAXgGrFHY9Nk/Bo9eDjZz7mcc+PhUOXJV6kcbQeds3Aq8mxZ+r+hnSUjUr6X+mPNk50uZSWDh6KaqVx4zS8skcStzVFsHfqynjR9xyrHnkOn6Q8+Mjh9BQMO4JEB7WB1Q6Bt0ymwLssDJNPE0OHisDAsIuRXNsxgqnPr2HagH1gUzEgYeGp3+7ErTCKtXeuxSdUDCzCzHk89Pg2/s7zI8YQR0Xrx8DMm3YvbTg1rl4Ej/4N/+GOYF9/tY5VJgWuh2ZQmOFIDihsPsWTa1fwawf83fi2g71dSkA9dD/6DgP+sr6V39tYwswZp/lzxwjiT62hffhsIOWm/YohHL9mKJmyD1lGNoFKkWVY5m3B47F1zKwDbaUfQtcbmYJHki9hSD3MkIK9zAkQqyA6zGYO705BtSGDOH5DY7gV+/ViTdetZKf/RVLZV0iSyIgGKE95lBNDkmn2buR9mwo5mSRj7KGxSGXB1C/eTphC7MV298JnKR0fiDHfDcn7JLZJQo2lh7fS3+BAYywNQwrINYn7dIDlGvdEHWRNp54Co5Jm/TJs99os2QCa93WiBTrdxDUUepP/zi0lOLfXU6+4C0n6XbCXKAazqLz3/0Ms0ZRen6RWbQi7AmZxbnElb3rI0CvFoN1o0eNScwj/sqN0NKppj/8R6KvQL3ZOpv2O3bhfTaLwWChxdi13Vhb1Q5tnZoRTJ06eYtU2gFuLltv2zCVwyGVkkY57GkqJq/FX6fFpIse/v10tO5j1ORyqmE+i+0VcZX29rkrMEjuaUsl2DSCg7k9G2vES2nV1uFee48Lk/bTHlLDMrvRp3dnnUf6ZRaOfFWvEWMGmdo1iv+o/qCe9y5wFa7gqv1+wm70zGB+eieqFy6SUbOY2sw5biVCAZ/w2o7PCwHIos0nOAyDXcK6wN6npnFANNnKxZqWVLucupA0zeeZ0JXN+7gMGJIUT7wblk5QggyNptI6+KiR/LtSfpda/joA6f04NPUWSm+gjKJ2e5I3HoogOyWHe/EeRpFmC3SBpmPLJKgaq4WyzXYkEvf7KgQkHONXtwS1Fju86zWsMbywMQ25xxsfYB4arjHU84QklTia8vTqY6inKridaC3g3DN5ZsBvV3P20q98S7PNl2RT+PZerw0+xTjeTs/PFarLi+ErG7EkntCKUhlbHSt6v/UopNsGsasgICnOwz9A0Y/1yAysvP06S50rs9S1O9qviUHYE/Xyexb1rELCu93c14awd9jm3bLuFYaeH0Sbm/fnMPxNrj4xTF1OxSlaHWOfWSi3Wa6G8+MsSqufbAcrmVlaeHUvbtUhy7xhIdv0bDKavUl4mg+mtj9Jc58Gtj/8NXMF+zPJfSZoWui1gNIvkRpnVhJcMzuohoEMkTIyzfk1JcwMJVxqY6ySJfqrSjf/8PJHEq4ncB0S//LdwbG3rQQKmvkJPy914FylQWFq40bMJQGn6D3MHn+L5hHI8etIQOx1DbfC7BL65gZPKHmplU3nUzh7rHM6CLXPx8m+meeQZ7EdyYSuaU8Pw8m8iwLMvrmjsqWPzLS+SPcKHP3QhtAfnAX09uqIDj5Lx4Zd8+estlBQE4yOJ32y0aitKQzaLty4iJ+Wqw+9Kchnnxxqxxv2KJkYkTRrUvmwIKOVBIP7UGjpNT9Lbqadv5BsOE6+C1f6w1+q4l88pCqTil6XMTbyCfFGfwxfrtQX3OW5s/nA2+gI9CksTtvtKe42G+gt+1F+4lYah2Yy0kXWv76hk3vTv+bZWTdhJX2L8k4V6FwmJXdN3IbPI+O7sQhZ6iX5VpHwGV338+fPhwTgbAx179ik9ecHYyUifTrytEh5ycY3MMG6jORoOdp8mPj4IzKW9bTm0gbxYp+HoQz+R0+yExiDmETzcB9Gx+UuirxTxzsNfk6KYLj5veTCP7H6KYVRR71fvEGdJSPw972/cXKHLLInFpZKcw6VBNL53L68vegk/WQr0QRIA6KxKikIq0FjH4KkQiaZmtR9rtgzHe98wWpf9gFwufjsNNSZGRW7kiQnL2W/eA/SR7VwtV/BcHoR34haKjGCfPZnlOhZP1xbKa9ZSa8cfdJKr2RvszXbXbpKGXUbjL5Ll6K5iQvxmWv9ZyqR9k1FbHNfJCzGVpI0/T8akc9SZ78EW7KttLSEopIIirQtvDbzC/Ta3lRxYhWy4kuLaEIYMuoCvRpTx/CKwAj+FnrBHvsDDFO9AtfjFt5uCC/3JuhCLao5d9XPZJ/x0/0u88tQr5Gy8jWfsCs49DaV8/+RW/iw/wqFuR+KX3NjGY4uP8mN+NF9cHg7GdlD2XV+QUwRu3Sm837qeRGPETYinx9jDsdRHaHDbh0luQjLYKbdYWpl/egSHSqJpcUlgoMccwX5FG0vHio9JVoRzd+RlrppvA3oJtG+98QZn4KYc9L+NM8BhCxiHnaN/yxVkMlGWcg5FVG7q/c2jHkvRm7aiVvTm8uRWIyse2cFnO9N54v1PiHi5L1521cTw0Z2bSC8I5tayZQSleGM7v7dZoml8eA1Jvp283q+R/diAmMYOEifUs84Qxd9Rn6Ayh/KyPQfl0GReC20hvAuejT7FDksxXNe5CpCf5J9gPafODOVK4hVktio9kowSuT+pW2YSWBvICGcFHgmHsQX7LimGsvHQUWbsnMHTU46wY1bf+y4LL6RuzlFm7JuKR8c6qvIOET+oT0q537UY5v49HYVZQWa0WCEnk8ysCmxgXecAylp9aJM7kgfevHMWG1rmUnY+Ccko5rI6zH5cuv6YNBbHqs0zRPBDNRQboT/iA/tBNZJHuh5nyvs6ZmpBbWjiBgjfpM3gXHuvz2wtjOBQp8gQdKqTKFh7D3XpmXw34jImbzE719//b65NU3JF68e75S/yg9278srXYJS3MdI/CEkr+lT69kvMHLOOg7JklrkpOfsvvdFlVgltpzNZnQp8LI59DhfvfIigS0bmf/8++ZbHuLGe3DXpPtbvzCeh1EzzjjBi0lOE4ySrjG6nbqpXrCIU8brkMolLAy8RIIdQBbjJRRJfrdmf715+CL/katImrkFtWQT05SEkSeLUsFOMOj6K7774D2/bSY1oOgt55K79/NzshuZEgEOsrzE3cZsrPOQOo7VGzFY9oAJDKw9I92Es7lVJiE3s9RFugN2jxsay7rdKbhcv91/HLwoYN05cHeqN5yhyhzHh6/Fz24+tTLhMkuHU7cToY6NpPTaa1Jd28t/xvzP+C/b9d/yvDUmSCA0NFYCx5fWZlPUU8GDk60TIByAw24DfTSdp8YDANgiwI5AXdFSxYfw+YvNcaf1zDHe/JrIcVQoFnd6xpL/0PElOjn2AAJ7PGMVnSljRBHJ9Jbj0JYwtSm/OlE3lnuKJTEmoQK5vAOfeJOiDlQOpkB3k5fE1zA32RbJYQC67fp/QvzKY0Jx+vPrsG+QYg4A+sMVbaqMmKxGn4CpODj3GwzIR5FwWl0nA/Vc5/MNchreJ9wQwWXWCu1tjOLEzkOLYfWDHSU30mcQHCT8yNnQLQ/UW7MG+CbJS8q53qOqnbL35d5NzBO+O2odi2CF6lD1M6Em17fZAUkcNCYfHkZ9ykSU6F94Z3kwMfQmLKVIlOUW9Ujl5RaL8gNVq4Y6HvmH9B3ei6XFCXSru9lEqOV95DYJpxYRqc1AY52AP9s11PcRiD1hSe4KEtnIet+sm8VtCCYkB9Wz5di4RC/qY906KWdRWzmKEfDcLTDry7cC+a8lFFIVc4omvH6Wxxg5CVDoz74/BDLiUTE/GEqZHioBChDGbH+efZ8uYRgaXv4GXUpykcpkMjxYPMrbfQnGSY1Bjship9F5DlQnqTXJQi0l26ToFe2tX738mz+sOp0xBT04Msw7N50q9P43jMkl5vO+4q9VHCB0zg/OdgRRkFrDYLsd3RR3Cpsg/ebFgAM8bYnBVio7M9tNDGLJuMVeBzsWHGCXmGHnA6xzhGgXzipwYF+vIpEqWVxG8eQFtwJmuFKGJNFYL+tAHOfBqPZqLp5jknMYnq1bx3ssvc8xiZYhdJeztOh1ngFteeokxs3KJLA3lUL8D5CTncKdlBbbJih/rTlNW08CoASvB267q1djB8sffY8e1hxlx7SJjhovek7lxH88Fv8quNTN4SBvAE2MN2POeH4qZwnwFPNcA/f2Thftqc0kiuaqdZRXN3OP1AHLjO9xw7hZd9kG25SXOjz5KW1QZM619nFIZkNZ5lubKVgZGVuJpShS+WEkCk6qXyKDYMZHGFPFlGiQNtWZo8gCdLIJAtWjPJZ6fGqAjGkJljnOwSB7BgR2DcZObmbtkH2pDJbb9OXzlZ/CQlfDzJ0t4bUolmTeCXKUrwdumEK3TsHXOVrDKBIe00RzF32fu5KhbHqfizjJINUf43S5PPXsn7SUiN5z1P97OhlndaGx6f53rtwujZzZ6td5RykeSs7Xdg0BFK3mj3uO32irsmbIaa1/i+0TDx0zFJrvkGsOdHiWETzHgV30/syaOvDmLZDIZ5wafI9Gi5nQoZCMm1Pa6O2GcupcQhZ67+pU5OOEfTvyE8UEJzPlzHvL7xCpNtbWL153UrJGpqHeW468R13ezwoOXv5iFF6Ds5yglCLDKXc8Ld27hmqsbsp5ysJGrlVn1JNYFomj05o4wkT2/xrcb5/GHMG0eSlluMg8vsel3YjExO7WEQ4VR3LXuLk4sE8GQImUaE4vXMPSv8Vy+GkvoCAEWY2LYY9z+8Ta0JToUHS7YxxE++gMUrE8lv8GTjvASHIfEWT2UGCFVusvB+kXBFdqajPw0spRLyuM3/17uPIq3oj7FfVEV99RFUOdxFugLMHQRFj55+rO+67Trk/JlawJVTQ3M+mca+YmOQaS3fgge5gZeT/wBs6YbbHph6V1ied/YzQIgfc4JaroHE28H9s13KeEhBWR35qA0lANigt+oUDMmrZBnPcCpJ48b0mSh8gbC6oPQq/V82O8ivyrEfemxeiUN0mXGur+Iq3GQkKyRVJ689f7nmOoreePjD/mxo1U41rXnGKmdv1P6zTx8b91CZ+dwbPdaV8+5rB30Da9O2Eh87L/IQnYUMmn+TrJWJ3O7RyEySayuFr8H2U2/z6p05U2nV3A3uDNkxVNETj4HNhhmZfNhYqq2o2j0o3D+NqpcxnCvzZnGhK/k+z3gXPgsV6sV3NFTR7iNSLkmN5XkbUF8fvca2kNcHHpozVXUY8ydTJK7Gf2IUofb6q/MQ//oRt5rgUEqR2DWtU7GfV9MBaYycn3fd11mNPNRwxUSjd3MDf8NZ9l4sBHKMfbcSdzFEaQ2TuLZKd+SYyc3vzHwI1zmtJGsvYfkQJFMBNDuF0u9262cOTOQpRNFv0ouk2GRW5A56chLmkFl50XibKohAd5qcqKHXoDFYR1VefD09L9g5iYmZlcz1yb5eO+oUwSOr+KPFcuw1PhgVnrctElIZKdkc2nAJVQGFb7WQXxic1qNsYLUoTmUlDXQOPUybh5iZlSnjMZco6OoMpzi4kd53g5JOWadzPzyPwBIMLchiueBRl/NK5HtHMl3R97dgW3fUgB/cz4vJf3A7sIHhcS/V/AcHrlnC9rSy9z52keUu4jPo8WYwKu7hrHbu5jz3nW8KBf9vVKND54P/U74+kX4laXi9L5IMHwkNI7Gx9YS9MVdhJaKayTAOJOJi0+/hL9vPd0fuTvYU53dcXVr5K+OW7BzJQH4qXAw7V9MJNusYNIoz5u1Ojfeq3mUiblzd6MMFjNHmoahlJ9R4P38D3waU+ewZ5k3T2HQ+RTiB10lOkIkvMktPfTTuZBVHM6HhiMc77wANmCfJEF67GkKO/uxr8PHTpi0d3zSGEubPIuKy2s5OlsEoHy7LtAUDcvq4ECzGBfMkhRcaw+g8YMn6XY2A7/a/K5E9sBsnHucSA60UCOJSakgSwnrA+DKg5vo79TFJfMksIkZAi3r8DmRyJjOR9H62DUUBmKdyrg/qJURFWAwuTnYVSo32rQSoe6dVFgc18kJQc60DbvM1gFZWMyLuNFF1LPrPLWR8IZ7I2fVjdSY3IQ+yq3uc9laGsP6yd9SNG81bzn9Kpz3ldkFpdUAAQAASURBVDwvBn56JwnA0gTH2v+30/2oTpYR59zAujYReJWQqAmqoW36fpLbvKlu2AJ2ejppqno8ZDC3GiaHO8IA+xu96AfMcbXirC8GeivHt5DJ7/sNOGc04a7oQK8UoerTkWs4fct5LK0vURckAr5WpSeqAki4HMt9mxeQfZcIwrvraxiVWInHx/fT4tKBcYnox7oqakmRmXmtaAYhoeUU2iWTJbUb37TBN20gs6g51l98n/XyKL5rA60EPspmgm58IBo/Pm7W4lPvyTPfP4j7FDGBrparMXiM48zIHDpdOh3BPAnq3NZj1g7hzpAOlBbxvpe6XeMhq8TzBe5sUllEsE+u5nGDlnGx+YQZtbQZNdg1RKB/hcTkla8wZvFaupVBQB/YGBK1kHy1AmPQPua7aFFKBm5IEUoWPYnHv6chu51fveIpDctiog3YZ2r/iqeGfsuTL95Hj0nO41PE+0pVf8jUuC18/tSTZA4tFIBwSSZDE6hi6htnaeoJpkclzgOZsZUnBpVz4L4tFBokkM3EfnzY7op63A4CrPCjWlRU0GsCuGN2L8FIbQgS1rLPPUK5Nu0vhtW9TJvuNeLkorrL73uH4fRPCiGLNtIV6KhQEZiVRt4vE0kBDJPE9bvQrOBAi0RZ5AV0Pc206pvxpS9e16t8eKfdzOvRTfh22DfHBatrNMll8IBkJl1j7ZWLtBnBHZ9yfsxLjKoCg6lvApv1DXwQ/xfHsqIJWPcIdbPOis9DFcCBv8bQEFpP1dA7SfcUzBTKoviJdkZ0+FJw7DcW9e8FcAICAnj/gw+Y+eKz7DD9O+B3BpgqU9MvMZXwI95Uxlsd9o6zSn88F+xAp1cxJ/0qkrEdFNdBDaUr/cvAV2dhhEaOt6KUG6TMONckhpX/hH/VSYquHUMaL+61u0xxHPI/w1ANHK+Fu6NschTdVYx1WcypsVpSj+biZIhwFByJuodjhflUdh/kiNWMxbNvr9bIWwktjuOkzIw8upwweXnfcVYraiycHH2McQfH81NIIbfLRLUGmSRx+2+9Po1ZISYsFsQU4pNcwL7Qel5zl9HW00drl8lgSYc3RrOCbx/8FqWvKLctyRTMu9iP1N0jOTX0FOUyF6baZBkkCVbWBlEz+DxZwYWYOuZgS0dSyXorlP3l4G/uwX50KsPZ3gW/B0Cb02Ggr+9kh6EDo1XHKA085Qk/N7ZzA+xzjniff669gXvSNjBXkm4UAd893WkElUu0yDP4u/Zx7n1GVFkb4PUpAxdUElrRhktPPD9K9wj3pLPKGFAexpLwubRpJmKrEnHW5MTj2jKmFU+k8OJiotc5Fm6Md7uE01fbeL8B7rbxTW6M2IhYmlqOM+5cP+4NbrsJ2070Okv9VBnbs55CYdIRaxSZeo+1qVF++zoDb99J4gSxlYjS3MqRENh2Lppv3ryN4jkSD9u4XdWyEHLG/82kmgQy2lejMI/AFux7SHucNTEmHku+Qo+Hoy+oMLUzUaai/mJ/DM4fI+MRbKVkPY3F/BYAn7TAmw0a9iqufx+dRTw/bDEf7nqeA1GX+EZroJa+uGzMrLncu/bQ/x3Ib5Kxc9Ebwt8lCXoU5Vj+PINbSgG2YJ/a0EReajdfTTtM24lUfulMR6Tx/b8z/g3v+J8c/wX7/jv+14ZMJsPbjvHhXibx3C+9lT0D7lqNPdgnSRDReYwhnT/i3ZSEbWMFg0VPbsJBki4tYuzhcUR8KCbaZJYunk19l5kRVfzeKgblN8aqVjP3u8MKb9Drq7CvDvGpvIb58khy26sZ7Ny3UMddDODWA88xYOklFg7fDVIfA12S4M1HDlBVfpUf/3iewPFDbThBEBH3C9+PM6Nq28vC3bNJmCvKh7Y5SQQNyeZwTDbHTX7Ypz67O504sXYGct8mWn22YQ/2IVdxrDWUD+KhUXUZ+7FPEYH+qW/5OLKLCpe+5yKTSRjUBoacGcLI4yO58oQIkEaHlDP+vb2898UcZud60/5wq2Df4hxM16sfUVr0HwZpPkMcEn+EFFG/9Hdc9z7GlEA72ZuuCsZ7reIXt9GslsfyjNG+swwkqEtJ1fTOkPJOR1mdtoqNbB22HI9Wdxap+tApSQK5oZv2n2bxk8LIoNnihv5ncAOKUB0v3pZGvMtQIfS90f8wtDKEpDu+xdNlFtiE5nLJilaCxV51xAffTa1pLLbSj7H6o2wwBnK6OJqOMPuUFRjbrvDNoA/5uQ3+bHOk2vhYK3nYHXZ0QYXJpoeG1Ypfjxr/en9q/WvZ4dVj02oZlMZaMn5eiH+TG+czHJX3hnrNJOeMP67b2ukIqMPrNRFktNgkmi7VOUqSuGEga+W9PF7nzZUvHMxMc/cnZ9QZTnffil/M66LRamaNVwubdg9nSOzt1NVPZ9EL33DUbP4/Oh9DgO16PaM2/8TjlmeIUkeQk5xzEwy9MUxmNW0Esi3kc8J9ZEKyWKdw4+Hzq/EqrGdkzQeoO2Zh2yOjS+bM5ZAy2iPKuVbmBwrHpsl6iwvlJvioFRY6N4mgHBIPfPcAQTVBVAc0IXu/ixvO3dgIaB99Ac2IIl5IKuO40QaIsRjI6P6Zi6dTGeuXTUS0+Luq7ssEFRwFYNe0XQz0FXXSnaVulq+5hwsunhwLeY+fbhezhDJ6+x+tArJmZjLBrlBCr/Rl25CTLB6SzQM+ZtpNYsXNr10TOFB9jEFWMNlUfkgWE2HVQQSXh9NPY8E0sFZIrP6cf5nIz6PpGVXFzgkQK4lVcO/GnmF8Si0vPjwUbXcbFrv+R59P+ZaqQn8yv34cj2l2zrBMyZrj75OyvpYTgPc6UX4DoMhzJX/es4xzw3/jIv3AFuyTJKpCaggvDWPTpQI8DFNu0jGSTEc5P+8yn52IpWbLeBRzxOva1hHKNWs2mxbvIVD+b46cRFN4GWMe2kZJiFipa1b788Qncwi61o/X173IJWkh0Jd4kiu0nIjJ47mQFm6ZdpZ/GwMuJKIrnMiVJaWkyUWGpcpYx9dPbuPPth0sqg7Gljd6Lrid9sBrJB8cxz6Xizxse8UyJT8MPYYqPI+lTaH4R4n31Sn3wbpjPDVtTlya+g93K8SKl0sGGcoOHzxaOrEq7HQhgZNdEbQ1dFCQcJWaWMdqSLPMiS/XTWCiTEPI7BNgL7BU10XGrqm8r91J6PSMm2kps9yFfCPEB9SxfGgdv3SkCYdlyIuY6Q/vFLvS0qOyV6/DqU3NtH+mEZ8XjzW4VTRaLXzSmcy177SctUoMe+kCtmCfTJKRk5xDTnJvJcyP5pccquwnOlUzTAukfkNWtSski3LJj8Wn4V3Zjy5lGd42lX0TteeZ+coxwkoAk2Pl9YW8CLzLo3h/4JeolQk8zvybti6rmTxTJ7F1Ttz9yl10zPHCttAnU5bCgaO1eAOKM6korGL13jD/iezqmMigEak06u3Ti6DXtzBi+C6mj+rt4WCXn8fLWs3jHr3/72atuen3WawWQipCiC6KJistk50eZsG3cTZX89iTm3i/GX51qmeuq1hbd+MRrL3rd8zyLu7gWcE+PmUFR+v20eTdhBJHktTshCtkDgrh8qkk0p927PEZRTnjtSpO1qrxtDp+18OkZzlAb5WYq6vNE5Ukbt1wKwF1Acg8M1Hfpwab9xHg3Eox0FUZwJsXA5mXIZ5b5dFJk282ayOfIrPpPLaARmd3NUMCnqPzn0iaXSYTNS1FOFZubGW+1YnZBQmMia6nwuKY/P+rU81c106+8QPJ3A2IvrAkSfSmJsUE5QDVQFyNGn5euh6LvIe7ZX0AqLNSyxNtH5Arz2Kfy0bMJvFYlbWHuxccw4TEezIr57zE6g4D3ZyYfRvyxhWYFWqeMOnR2CQytAoXJuyfQGBNIFm3OMo2ac2tTG0KpumnJSTe+irwqWDXmIo583owgc5/Mnzc43C9s5lcqUVv8MPV5MFT+w4TP8QLW/h0sNuj7PlNRmRSNeWjtiDZNbyO8t3Mb/tgiOsH+LtkIUlixeIdnnmgaKfw3u3kBTrKXY7MT2eeQUVIVQjGf0kCtAS/yufjLARY/yB/ioYJdhq6V3Z8R5hhK+fTz1PXOYEbnZrTe66QtuFuamO68EnIp1suJiBPF71H97aNyD3yqImrItxuJawJrONy0mUuzdqBpBokqG5YtEHcPXYHc7qs/Pr+XbitsSOkSTA5MpPheV40ZHULfuiNcaLbhxIZpEiOyWKdyp9v2+BHf8hyFpngoVd+oGr7ENYEuvGlTxN32hwsWcz0H5THpYF5NJyw2tRzX78neQxPfrgAZaMXtY/+zWjXCgbYgH2rWlPI2DWYIXQw/ekxwBHh+EPtqWzZ08DkbTO5uKAehyHXcMLDlbqz/chO0Dj0QtZFv8Dq0cs54FTCRBvylFXhxMVqTzoKfHjSt4J07xhBwOufphzW63ZiMoUxsGENA8YLVDkaNSqOjzjOlBBPLAli4tNitXC4x0x+o5Hqqmh0wTHcaWN36cglLxzqQ47jJ4ccB/FSmOpcwixncC+CYTJHKdn9iZVoBp5momcwIe59+8OvF54g8rSW00NPk52UzbjW4/S36fj6q2oypekHUZo88e8eL5xTkskYt2cyaRfSaEt3Y0yMeM8TtWZmBsAL7p1YnbvRKETpwVTX7xjl8Rdbvp2HR0oa0Q6BhM2ks0oOczA0+E0++nMcp/tNwMMQQ4tMaXOkDL96P1x8OhgaLwLh6q5zrLz9Ln77ZwBDP38C06MiSUpOD49PyGLa3MMEKOAQYkuDQZoGCr5exJOZCZxPz7K/aHpc9bg1e3HuhDsN490cqs2f8T/PBtI5snECUeNFP9VZ5UxoynyMR68xrCiKuv56bpAimhu+YuLsf9h3II2ij26l/EXRZ/unx4XvjsbhblJQG1Dj4H/kG+9i4/VWBMoIsZoRScYHF89SVt2GRWZgcYgYz0juifgWQXfYRbqtcFThuFfLUFJnhhadL8jsjpckZGYZT8tc8HPVCe+yCzUNZijqCsDSMMm2UB2ALEsgsS4dvFo0gHpnR4WJNRXrCUl7gwdu2cf6APFpNzklkvv6I4Qozcj/8zUGiygtbVT5c/6Xmbyi8yJ6nmOKWyZXYW7wYsOQE7wSl4NZLebCKkxWCrqVvOFtpNWlz/eXmfWE7htL8F9jcfVvYlSwnQSuXMP39/1IgsqLN/yWYOR+sPFEQzX96Fe8hPbOy2zUvsbQ7nQGXJ8HTz3zDD9uXcPIYzmMU8Ddpt68Sy2wVqPhsMFIcOJsFuT0xsmRg/IEOUyALGsw55P+5DNfeMwDjDYgk1yhQpaZhL7LiYWvbCUx4FWgFQB3wwne73cfv3mn8Nbsg4xQPmpDiYfbgl5Bf+xB/PI+osq9DUbazFGnIO6tdKao0oU76z/Eu58Zmew74br+IooXWrahrllCRfWb3GZz8nLTdNYdG0Co2x6eDbcSrv4HuN6X0mrmm65f2DAebkvszdnMk4mOQLilkIrwRnrCKnl6WB4KG1ngCa5NDHKGp7xr2AX87TlWOPbBRQc5O+MIS12MnNGJ+7Qkk3HcpMaQcYbysHJC7PZpSYLVJRF41bdQHVaOyU4SfrLXOr66nq451SUSTaFPnWy2M2RaygXbgI6zPOIO77TAF60ynnXpIyZKEviVniUkr/d5pGmywMZz0odZWfn8R9y35hXiD9bg/4ooI/5MfTyN13Q8vvUujo8XlQMA0jUWpv+0lErA75lTolHpRIERPJIvodFpCJU7gnnJqjImOkF/FeRYMkHY9WD+jE/560QGyTsz0M3uWycVFWsZHylj89UXMOCYJ3soopJNA/LYdi2Ir9yb2WeTUJIBRV8swbU0kDaPeFI8xT6zo/3nk+WfTMvlPWz+eyLpi80CDa/O6sH+nDDqA6s4n7oKeFI4vsdzMDPXD2Pc4XEMnbYHuXUptmCfxmM8jxxYxC7nP6jUK5FuxI/OkXz8xxcYquqoGV6E7voHe+Pe5P6pjFCNYrr+JLv495zbGWA6csbIp+HhL+anFypmE5Ddn+56b/7syRc8CLXSlVLDMMxDVKwa8gmjtY6kyf9Xxr/hHf+T479g33/H/9owm80UFBQQGxuL/PoClOJfjzW6DKkonDV1MoHhCJBa+xlth1spIgrjcDHoV+trqYqEpvv2UVWhIkAtVie16+vp+MOZN3Jf5qNn1guOxI3xSSusbYcABWS6iwmJjsZdTNKuoYD+bHP5kLttKmaSvBohSMHqyhkcPbSGL24TPaBdhS/QcEaH4biR+vggwUmXJLDK5AyrOoGpwYS7NAzbhO/3nfFkduYD4GIS7wngrGUQ3v4FzByVw0VvR0apJIFVsjC6EgZL82wEYnpHkNOL/GxaxWs9B4hRzrgZ/mo7r7DbxYUDqAmMqsbXTWS2/SNP5wnTUSZMPEpCjytaq7hN5Fl8OCfvZOegjQTLTwHbhYv68VwkY/KjeefZVyj17URoBuSVxod/bEB2KIuse1ZTa2h1uK+VTcs4YXiSuki4ZNoLIg8S/450FD1r0XmamBTRFwQnyh4nTH2VE7qR7Jl8mEEyUTa1pcUVfWUo1/zvIF/rx43G2ACS1YzznVtoV/zNfD/Ya9BjC/ZVaAbx4j8JLP5jMbo0E/f+oRKa30qSRF1OL3vr/lmHHe5JZtYxWA2xPrA+oAvqj4NfnxMVaingPj8IvT71pe5KcA4FUyf7Fx9i9bRDrKgDT6Mo96DXhHDSq5bgqxFkHH6J+vQIbBtBj9R1YtjYQYdTCNn+jzDdTtZmhG85FiLITcglc0S+w3V/XTOOEXW9SZ8Rakd7lfYhdgQtpqkniAj7nJYk5/kj6Xj+OZkCoGzhCcYplQwxmx3OYzuGAKPkcjLHfs60msUs2rEYbhOTjOWF80nbncr6Jb9R7TwH6EuuV3XWs9n5QR4um4+uMAJnfQ62YJ/FNZ77FPU82eLGwGyxOuLGuL3al8vXd1E/jejQqow1eCUW02lUsWdIOyMNRm5APZf83iDTs54m52mYmmGETZWE0QLZqToaXEsYtW0TqmHRQMRNe15bPv6DrtBcFMJHMy9wVD1W+N0MKZu6zkjaSt3prFwHQmof0hSFNPsoMTZ60ah0ZJzKZTLKwstYX+/Pmiw3rs0R773UUMDUv2dTkXSF0Oi+ZIZz2wW+f281Tx6NxvO7O0mL2021TX7T6iJxZPQRooqjePjrh5FWir+dY/Ckw1rLgeXf46qwIkmfCvZigwqdZMbLuxVvJ8d1MNJ0C1xv/lwmxTjYT7Vt4VrEz3yqBy91glhdatYxVmsl5WIq3Qojdd19gX+XzJvdXZBzKYruE8PonOYhnNev2I2hh+/hmXGHqAirxX7WZhkDOV/ty9tdG5keKwaJMoWGzKhCetzaeeHi40wMsQv2JInMQZlsdAYaPPjL4a5g5oUEzFV+vH/wLv54TiQuWBSerDgXiuqbu3CfLvbsc5FZMCiNvP3y271SsjaRjVKhYXv9QJzcs1kQV099t5hsk0mQfDmZHm0P5wef5267b/r9YiPaw/148vgoOtMdCSZHLZkceexb9Go9Q5T3OthlMom0o6NoBtIWOgaDcVFlyHwCMZ0dgd/ivn4PCquBxTtuQdHhgvK23xntki6QNUK7W0m4PJBlW8eiq/Vk5N1iZd1yKZryPC0DF+5j7lRHKbaWXe1g7QVHdnZG2XDfQaVv4HUv2NsNp3SOgBzAq41DaLduI0kNcwITSLOzH/kjBcOVCC4nF/DjPX2J6JOGNDa2H+POzfMJ6fEgYpqY3Awp9WfkwRQm7Z/E4Qnie27Xt7F+8UsMvDiQqbun0q0TEwK5PXX8PO0bfDK8afVu4neV2Mzer/EDvhj7M198/g76yBSG2BW9GN0SSD4cTFCjL6VJJVydKoIagzUK7ruecDBZi276fTEBMr5riuHc4bF8tfwr8BTfRacqhkVSPYVugMmxCs2/4UeGHygmfegAqm/ZjWT35d0SvYwld53k2W0ptIc6Kjk8WjcUQ3QOQeYMBsod2cEHLWM4c98okoCZb5U72IP7HSDhoTLW5X/CMHVfJ2SPnkImTszk0pn+1OudKDOXCJS1dd3DGOjciUuXC40/LUD2sOgrrgnJx73Fk7eeuB9Nhi/pNq+jo7sC849xaIFgDuHarYCbEA+oOot5qbof29ffQjEwfqGjP+iML2O0TdzvDlUYEMA+Yyef6PxoXbkM97Gfg/nNXvk64FDBz+RuLeD9iT9xZ+pWTPI+GS0FsKhuK1mZA3lgfhBn5CLI2KmNYcjmgczd0guGjfhVTNi2Ny5ly9w/mFEFx3RgsQPr5rkHoD7e24VHUSFKXQF4Bk7h3Z8exJkuBgzciD3YlyMbAFxD16UF+tZ2ydKDYqAX1YFT8SqpRRUtkvucAtwoTF9EQvkW+n+znKHjRDk2SQIrVsY+cYzhwbvo5k1sgYS/qn7hbOZG/gn+AX/NWLudGPoF3UrtC6/wpEbDMavjOmixmpFd3yaTTL9h3zBT5xnImeH1nEk/yR2KPsKcxqKnqSKIdk0bj288y7PveGK7Iyqiw7k0ag6XzL/yUY47rXYyntvmn+akzsrCWog3iKUhaoUGVdMsDiZk4+tTx2ynJwS7JEGbBVLOD6Ii9F96vPX+K6KUMMI3E4y3YCun2aWJ5OH63hp8dXeYsEZe9EulfYge3wZfJKskkrssOjYHwpp22IQV+0aE3XJvqkyQVBXE1RNhpPYXfdwGScvXD3/N8m+Wk3VkEKJIM1jqncjY1lt1dFuwh8Md1ffUoFfrMWicSbc+5mDvkUKwHI/g3tL7aV3aB/J0Ow/grQOJTNo/iReBM4+KHXQqCy5x58/RdPZrZXD6KaJl4pW5uvVwZdp+yspfY1DeG9iqGUtI3FLXyooVK3AFCu4XK+AkSU7hmf4c2zaKtTO28e7tEQ7XvaEzgc9bCum22JALbYZJZWG1soWrWX+y6ba+nu3h8k5cDR6E1wZiqW1Abrc+T5YV0t0cxLMhPpg8RdBLspgJM2qQnLv5+OmnyTK/ja2spCZ8Ae/fIse1tYYPvvqYNysfFo4/0iNDnx2JDDirupMYu61YYWggNxy6LHCoy4TM2gM2smy9srbW68/Q5p67yjgR0cbf6+bQ3OXDBuMmbFO6OnMPFnkLioDLBMepaHcWn7fMqmO2Uc7Xy14ha1Apz38t+qlPNo5mamav7Nz8dDsCq8XE617NXJp2iBN6K0nY6UYC5eFzaRtxlvo6T5r0GWKKuzWH4U4/sFfvx8GiO4iX+nIMNVY1beOOct63hPbCUOLcFwrnbZR8iFlzKz3984i+exuSJPZY/rJ1F+79S0kJ1jItVpTNM5qNnJc9zS3be0ldh4e5sNQWCpcpcC6JYMy5dE4NO+UgKQzwslcnTnLYdPo7lPZ2s457T47B5cAYYqadRLJRtImslkg6NQt19SXy4yZQ170OP5tehUNj13HSsougv87hanLcK3V+YaxO6uQ9qYK7EHshS0jkx+Uz29XKA95QYF/FKZNIvZCK3CKn0MNHbP8MSFYj4y6kE3UhhasJjvK7W7qUfFrbzaUwSFTYVJBqg3jR2EF8ZDG1S/7mkDYQW6q0BLT41xNU4YappJ5a1z+xBfsWyatgbRwQR4tHC5JdFX104CCmMYULpiyedsrCZHYmIsKdeXcsYZ7PMspL7sTH6TKRkdUcdEtwAETSLQ9jKRrGS9Ji3m6Gak0fTdtiMjP0QhreNYH8MOwIw9yH3ySSGGR+7PzqVrq9dBjva8YkiaSzqeZ3qf4jEkihJKIEue1apHSjfO8wJhwYSTXg5VHoAELWdNRQLh0jQNOrWmFLRJUk6PIM4drI+zgmybGo3Pu6RkoyzijjONrTl9ewn6ORVLH03XUc64FRWqgwd3ID7HvnwDhS9oQRO3MHBbEFwhoqk8GAchcsOiNfS35Ue4iqXpIEMQ0rUac/xFudXpTrCxzs3o2uzN88gab5Owg3iZXGlfr+fHd9CSrvDhNjXiCm/W/aomBsJWhbHuCwjW1A+wmiPOGrNuiwyJBkfXCBR+N3TEq7wNW8QFLGX8DHQ/Sfw2WNLHAB5+ElJA5yxtl0BehTfGpo8mbs6t7V6auwKoc5dChMx0+Dcmg+n8QOY4JARJVbDYzM7U98TjLbZuxwIGwCfN8xnVUt2fwZCJX6Vgf7lrq7qRj9E6VWIyn0EbT2mF5n71vFKFPuRBHXTKvsHWzbQNyhc6d56k7mf/g0tXbzy6r2Zn83yALqSJ1kwS9W/N04WRlfpH7KW17lnLHUEWYSSRG7TGlYjpYxOjuFL+baCwr3xj+RJb3P+ZPEi2y0U6Vxc4knN/9tSpP/QGFLTlR7sd5tL/OsA1n8x2KKFvYSxfvmv0TzjFZURV6MuNTAaKWc+wzmmyD/TwoNJ0x6xjMB+dhH6TS2YaueZTZdo6emP5vnbSY3qZD1thel8eG7vL0caHqNtKiD+Mkd20D8vzL+De/4nxz/Bfv+O/5Xh86uGfD4yBIWvHkStyIwGvxZbffvg7sP00YK7734HiPdHxcqdSSZks0f3oG81Y3lH35Nvl2Do27kbFa3MNAYy6Q9o+C9f7+mJgs0GUCmEhnPOkmG+/w9FI84jbFAbDj76OgCZMMzWfm6L3URn4PU1ydLJoN/su6lS/E1ZYs34OdUwSs23e9UpipSpLMo4lyJGleAVSYmsj3LnHnl61coii7i5DjH/hdqF3e+X/4Nx1QQZXBkTpS1nKZg+Ff81A6bmhybukao0pl8+FmMlifxf6EPsAtQu3B131icDw5i4qqPOe6UIhxnsKjp7NCSNLCIlwOMbLeKYv0Ko4zQ8lC6VV4QYRfMSRLTd03Ht9GXb8vTGfTRCMGe21LKzzFPsOzQ3czdOgfZo46ghBU5JuCaEbqdHNmA80wPcnl3JNOf2Y9/9xtAL3s0W69DHVHC5TEWSlOzkSEm2D8+158Zm8fyyNBM2seITrgME+uCrHyzZQivb5pCxZM9TLYpR1DIZYSX9cqUdJW2Ua1rIs4G7mtwn8lrUet4OKiawwZn7DiQ4DUAz2KY4Qz3uSqYa8f28/B9lufWD8dnwAJe8AJ0tb1gn0zF2mZ/DhvquNcNvCx2lU0KZ36YtJtlYdUMz0qnSVaNLdjnTCl+AbW0e0fQ6q3GbBGl2IZ4VNKQmkdYei6j/OscnrVFLZE98QAfptVgdLUXTAOTzIO2GgMLvZ8iXm8FvukzSjIOyuQ3ax/2HLvIh/8XTcIBlhrNPHjJzGhfK/VdKpLsmnM80O8K+Umu3H56BPWj7JhtVgtTjo3BqTCCzKc3opc/ii2/+Abrzfvev1jmbsZqvA9sZOIAiuW7iFLCxgDQeIi/rdZV8N7S/Txav59TbdBu6EsAdVqCKWtxI/nzh6mUG3E/3pcksUgSr3vuwTPRme+bpqGMrcW2o2W3JpxfPaoZ4ashXG1BKYmJ7HJCMbz2KaatE/EiDJ3pDqAvkB3vFk3cZ8/yzom1jHFxTDAGdhyl1NOVH9+5n2MjjyG3q8zykrdTkZLD6PgqFsb0AQtmTQDftMLF8BKaX/6AXaY4ltk8kiT3Iibcdoirm+QEXInHC/F9HCmcQ8iGKTw54wBLxl+m2y4CWLltNDGXAylJW8zEKFE2BIuZ+UPfZlPSWT41Z/K8+iuH+zJ3buaZ9XdR2qWm+N1oHrQ19tTyUVEyO4wqJi/djnN3OzeklisVAzn843QiSiPYeEcBDwSJNJEl6lpqy0Zzz5p7uDjcH0SVC44cHUDL8XTkM40OQY1cgoKMTAxX+1F8oZ3gsQVMsknLqi2t6MIUfPz5PC6rboEXHW6L2MFXuFblx7WwT5GkyYLNqvRgc4uS6V7N4C0m4D8rc6Hy2DKabt/M0FBxvfBx8uHW3d/SYdzP9EWvMlgZJ7RYH2g4xCPv/8oX7RZWaMDJKp5b265i1PXk/KTBjj20ZHI5YR7dvCxz46IdoxR6+zbJJpygsVPDxpbXGWFnj3GtweKvpb0lEBdFX6I4ouc4cyzO5OX14/6vH6bydtGp3ls3mPzv3LEEdNFvQgVauybno2Oy0Xx4iTcNzciaXWxXKUCixGMSTuWXyUrNoitYlOBSGRpY1hyCtGIZE7U9qEaK8tAArVYtmXo4p4eJ6ggH+9aoVqZdAbPLBJxtyEp+6ufJ3D6TmCvHkYfXEeYqHvvW9GMUxRRQu3sEiwNFUo7c0ERjFPzhm82+BH/iQkUJIjlmJNcuKly6sNDLQLQdeknBnu8n4pZ/hXaVEyAmKCUkhp8cTsK1BH70W4/MrrrV7Dab+3ZWURC8An9S2ECv3yfJvbAOvUBdSDExfq0obSRtAMwKN060K7g9J42000Ow3C82dz/VcwancBORTgae0zqRbRXXwWrzOCqKlITtSSV8hh07GBgnfUd2dRlW30BkmLAPh0YF3ksWPwHQ5T3B4fgh1W4MP+2NT88lJKkP7JNLFlInnWN72jl+6YBZzu+Kz0uS+OjZjxhwaQCSVWK53aKwIyce2We9oMI+S6ZAYbKqvOj2asap2YuwfmUoFOexBfusbrF87dKEu3cTmcmXGK8Qk7YA64MqUctgyp6/+Hm0nRyhWUfw4RG0WuSE6/PBauJGD1xzeTUh1w5g1vQnz1LH4Jk2WguSnOPZs+jJ09H07gN4TxABOZkko8O173sw2nVY7uD/x95bhkd1RX3fv3NG4+7uEAgkENzdrQVKgQp1o0DdC3VvqQuFGhSX4q7BIUiAQAJxd52Mvx9Smtlz+r7X++m+n+t6ur/Bypk5c2TvtddflgfbT8fzoWcbPvHFCqA8wLyd+19bS97VaBJTRDCk/es1FIcHYwn6i/4GD2qc4gWt+Sx7cBn6Nj2nauay4+//N1Qf5vuJ41iyaRrs7457TxM4ZAE+9kO8cd9rXDydik9zHq5q0XIyuGEVfXZf54c6C/eOs3Pc3Iy7g4o0r0lP7BYtk+LfwWees84XZngkc1XyJN/nFkZJ6Tay+erLXH1iNaHubRzXjWWQU/yp0U9yTX2RR9xrsclHgfZKd4n/fSyft4reh62kHPyK08V96OsAA3l4vUVi3+t0+3guZ3ueVXzvuTYV2Xn+3LfhAVyGiKRHD42Ou7MXka39mt09N3KHXpxjJQlG30ogIC0br7GiOu72+CHsJKPcgegvaW6aBT4dZcjb9/7pwnimFJwVVIVLwn/AEF7D4iWL8UqpEgqnksqF+cuH479/MOO6LsSvRziOBEBJkqgKaO9JPWnbJKRHxWfMU7agCqpm5eyVGNxUPMqnQlzlaSF/cAbTUuvQdBaJhQCNhmIWRpwkoHMbF0LGACLxJvfmHvpktD/3Q9wzOn6vLNHg1QEKDXArEI4b6HUJz5EWrp3szrXr3nR7sAJHUtpz/kcYqNYQUfwDWe71QIeLym2ChEljAklDdJg4j7V6JvNSpcydpcGM3zEeaY6ydJpv8UEPNMTBRekMMFKIfxdawlRPGHHDKhTQX03eR/MnjWx4+wEGr+6C+13ie3tnfgAnVkxiMxA/uY6eDiQS2drKT4v+YkszLM/8kOAU8b3DI47GwDRqfM7wSAX4OSglAa6ZEtnW+zPck8/hKpmYK4t9q2VZxmqHdD2k682cxYQj2Ofdso0laZ+yXQse1hqw2/9G9m34q+xYxh/kwo0UvKRiHAufNdoEHt4VxMBjA1l6xya6e92H447cqvLhYo0XVrMal5aR+LiIrgZ3+XzN+umXWTh0LEM9nQgqdhtzTDLGjH7E9zuHrBXXQwC30G/4uX8PGt12sEASHT8aay4SVP0Xpi6TyXJZQJLckdc3aKKZXwWDo4u4J6WIPCdPAgmJzVM3836MlT7uAVj1TnUECdbPWM/vsWpyGmqBmf+EbHYbI/WbuU06jNA5KdGsRpJbvEjJSiEluzOqRUq7wRGuTXTXw2Nj7iDbnINjDqI21ZN0Op1W4COPJsY5PIPzLFrOnm0HH/t6WJBtYrF5auijjPO5yDzXaiS3QEevDwBCIt6jM5lo8nvh5S96CwUZs3jgvj0c3DKQWb8N5J0h4rvjYSplwAdfsHfzKLShSmBWbili8bgLbDR4MqEmBGwWcABUbi+B3Qoh1Tr9n92SLKs41fMcp3q22wj6tyYKn+tmKaMhVM9neztxLCOd4nQPYQ5Vq9tIHXuZ/GYz03pdp9p6J9DxGcaUKk7pj9HzrwnYekTi7voL69aBXg9r10Ka/SrRT5YSnAddC+7hfSf8f5RtByG/NwCL2TtyL9IrHb9JZTPy/uPb+fNsLKszNJwe0LGzPd5SysrxG/FvdePrigRqIkTSsIu9jPDuLtTEXuSdKSe4aRc3WhUhVVzofoELqRewuHRjolM9fXD1BpYFQgbFTPFahE16k9vtKVzlCiK7vsCVFk92FL1Ggs4xt5H5w200tT+ksTi7M8Pv3YE++Xcc1WKB0V8w753xlDR/TUlCJTsHdJCGmlUyDR5NzFk1B7N/De5Txee/td6TfsdTuHimN3vvvogDVo0s2Tk5YTTb/hjDuZ398O0mqk8lCd5Pz+T8jq74bZhIjzlbhfi11iF88/fr1NkWizgLgkkdwIk2sBhD6WF+WojtCZ7Hu0feJswWQL/ipUh+HTmypW4nPSOzuMY9XDiQRu3wOwQgcbytkaCCLhwKKeCuHsc4ahyFI9inV5kpDS8ipO9lMqOqBcIEwNtlYdyYtZ6NE9czSC2+lX7NF/ncXc++Bh+OBKm5ar0KTjtEIzp2tYLHTXgnQiQjAaw9u5bRpWFsfWw7mxx6/q3K2kFofQjdd8Yj77FS/2Y2jmDfxTY3WtQGttyxF5VGpGpGe8dQ1eUzAl1LeKZfT862PIujTbjWWsHJH0OIjpDZOuSywoVKlmRO9DlFWmUI2zO6kexkiaAyN1E1dwNnDGqyXU3I6n+Db2z8GgT3ejaC1QB/W3kWRFyBv3WEk//eJ92eX/zbLvHt+MscMcADMjybEcXLO3VgKSTIowWvuKd4KkODl03H1L4Pc716IY6uSCM9qml76X3cJfCwi7Vru92OofUad/yhQVa9QNonW/7lnP/vGc54x//k+A/s+2/8HzWW1saxpK6MFhsoTbbgxwm7qJmwGaMRnEmGFl0gO91q8DXaWbVhM58kO9nXqVTsGL6fRJuOAf5OtpF/j9VBakxY+OLqg0h2sQBk0MUzq9LGfSe6kZwwAmPbRnR6bwC2Fa4lt6AUt7aDNKhE+w1JAp/8X+mWZ2HB17lcRkyUy4o/xfOwmiLfen6ISOIHs4EEB6ZrD69yVKEeJN1IYkKQyPwBmOHdmYjd06nbE0nhA0rLtGZrGztaoMICniqjIt7N8ibFl29LuzvAPpewMaypi6M1/Wc+qDQxJKAPjq3afMtMvPjRi5wafpBOIw6zOFJMhhfYssle/iBXgC/HX+CISEomte8VSrYNxViq4q/8XEY62ADYrUbMkXmo0i+RZnDH06QsCEdoykkGhhTDK4GDFV7QxtB6AE6s7kniIx3P0hlzf372+4mPZ9xkvQ8cdipUl4fWkNE/gwHHB9DVX9x4SCodd171xK3UGyk6H6OryAT3tlUROfkga4cdI1vTyp2SuGKnBfYhoXoX+wx/4XZD7EEIHczZ7S1wvmQS07ycWJLqEC5U9+dsHSxrkMjx/rsQo9Jx+tR1Wm5sY1D1QZKHOBfb2n//6k6XONz1EgM8xUT5uv9Ufk3rS1NaH4JVD1PZWki4QxP0zY1zKXc/zrwGLzxrlUXsLu5l1I07ypTcZ3lLtRjnlsinqn6j65E8cgjCL2md4vjS6Cr2jtxLs3sz5kNtTmWS//cRBMgGmc/H/UWTZxOXZBHg3xRgRjdhD64fPkFSUpUQU5vreHfwDTYHNLFLn0ugXQR9dYY8vizpTMu24dyYu5UEWZxPAO71sBGug2C7D0ZZZC6bPboysKidH9xPD2o6Nu/nbz5FZF09amsK6HoQ4dnRx0JqlwvQ6NLK0dG12NThDmaB7Y2gfWt9cb0Rx/KHXqN6iVgwKJCi+K4Wtt21j2EuUGH7DEewr9D1GZ5d9gwbZ+rJaR6NoLgFTBof1v/xN2jUOReVzYhjIWSs7ghpM7L5+lR3fj/3Bo/8vVuzusfx3vlwNGYNebF5eBv1ArgVKlcxxxNenLyPLrP2kacVC2ZDNPOpaVzNqfxALuZ6ssSJPalv1eNT58MRn2Xk2m0g8FntDAj8Dn8vNXEGsP5L0+4hXOVEzhRcgVLnxUPnx591WtRqM4HeLVgc7JUkSUJn1KFDx0PeoxkWIPakHNLLD3vkFta+MZVeBqUa9b3u5xlkgP577agjUnC09POiiYY4eHdTNyyZXRCaotI+36zY3B/ThWQCo0Qm3+0xO+UoY9OPM8fdgmRpBF1HAV+WJTpf64xXrS/f9hOJInl1XWiq8sO/IJJo5xZENiu9LT9TdDWc+V/O58psMUm16+M5ZHXFa3tfVAXh6L4V54RBMbm4vnyA81UPYlWtQIQgYbTmPJ2vp3Ng7Ui8H1Mq/1TYeP2BvfzWCGurR4BTP9bthp4Uz/6KlYEe1FvCgHYgPSp0KlWTfqUoqBzfs2p8JXH+NgZJrLp7Fa72OwhvXkYvJ4Ho/rZf2ZN5iguqRRi8tCLYJ0n8NPYtrJOuEq+3ESeLPUDNHkksOhNGGuBil3GzivMNgL/cRowa8iz/rpLI6VSBtf9iZjQ9jGyeBJr2d9tXE0rilTNgktge9wVj3cQ8oMLNRGvqFXr3vkKJUXxOJFnDocxY7GV+bJi4h/N2MffpadpLVYie+edDuOVdg2wTc4QfGtVkRFxicG0aJROeByeKimRpwjO6FLI706vaX9n/SxNAZn0SmX4QY3fIfVzDmGhoxF1nZem6u3BPE5UMEhIPLXuI4Ipg2iLKMCCqwY6qfVg/62Ne/PRZzmRFc+cOMff5YN9N4rbG0TplJ9nJzU6diiGooZFuRw8ydPoBWprjgOVCPFjvQun0mVS1RZEeqQRi9G1a0i6kAUXUVC3ktpqsyT2Zgfkdf+cMXEVIdSxeshhVjyz6Pr5eEd/S4kfv0BIyBmTQkORUVFXp+OjpL9FL4K+CSa3vM9Qh7OfTibqCAn573A+zuvZf1aUmu0yLFRrNnsokWufLspAItJO3saXnBaapOtYNe7fNtOqLMK1KZMetyaR/1EF8sWBn3UAfygZvIuZGM8ZQX4GXoKON+JRb/P7uYqrre/KivgfjHPwtsuiKzw+BXDfpCBh2ijlOGGWR63x+atTDgLdpMfVRunJYDUi9rByMWofdlOQcRWdvwS+umAILJDr85la1H+9khpLbUMPLL6ykIWQTcOGf+JnyXaR7XGRneCyVWfuY7+Tk3WRdT2xnVyqKw9BmJSme/Q26O+iZHkPK2XR8s7LBSR0yxXUkluxk7ikI4LEh0YrzTjNf4IUBRXQrgH4qZTws8C/OfnIXhvrR9Pq047xTPUfSPX8krdFzaQw4hqvJCg5gXy/9jwyNMvDp2Gq6RpUpPvf7La+QcgRiWmS69z+Do42ybK5mcPOXeGd05/NJWzhhyxeOlWVYUB3NmVPdmXjnQcVnA5SYteSY4L06+MpdJFB5GQtYE6RmZZUdWTFFtl/ghCm5pMWfE9S+Oo0bZbVJ+APVWb5437ESR7Av2nyBT7tXcOBaDb0nHaXQJpILXvE+S0+NB2GhpRhdlHPzkKALPPLoXu66MppB5pmKuCxJZG0fiE9gHXFP7ANEgCqw+iuKSSdhzjYa3Dvuhc5YyjSfdrV1UGQ5/rEi2Gf0VeF/11aORTTjnnM/fTUxQq+2Y2VJHPlgBo8b9OimHFCc1zu+IH32HnfJXSjRib9ZlmQud7vM4ItphJaGoLM1KI6XkKizwZm6TvgEpCniV416JgN3pN9LfdMRQkLagYNVrSlca82A/qdIsemxyKJV53eeJrz6nSLmRB9O3Zws6NTsKj2LqqA4K5bAbUY85hQLq86likscGHEA+4V82o52Y3Q/cc2TJYl6n3peiq/Hx/41kiSWuX19OvHgX1V4aWvwlFU8vEAkizY27mN02CESzRCtcVDLuMcw6/gvNBh2MrDsBOWBW4F9Hd/79wMr2SV6XdvHyHSxP50kwXNZSUwGPP3rkZxIJlGaKtK865hRbUFd6Sr2OpY17GzbT23VeiZKemrU4nMC0FZez917uxLZpxkpSFT5lLgl8/yJSHpl+HLl3unI8q6Oj0Zm8pbJdNbZeWz+Vj5oFvfEw+TjfH7HJRb/Ppv9h2bw6jQRSbnPI4P38eatpx7GlBrDJw5qMUmSeKtrBTXv/ICHbyM/qpxm7+ZbbJt0iV8v9kCjNaNyIu0APF1+L0m2DCaq4ojrJCardq03eY/8Ro4FiqxipqnqPpm4hZfJDH0fb3crSCJJ21TSQkVWKDVx11F5KNf4ezU/seW9+xkIxMeLz76HpZJRHpB5vhtqq9IKVo2JLkFNjHlyIytrcoBL4h/ofPlD3YgtO5YnE0rAZhTAvjhu8qFfu7JKsortEvrqoa00AL/z3ShNFInQdpUbnyxYSKtPPT/O/5LJPouFeIX/8/ykthM21YU4TyN6WcybvouSiI3NZXDyUkx1Q3C/0qECkiRYdH0cNe4r8WzVIv3L79Zq2tB7GGhrcsEtoA7J7kAcltWsda0ht7Ybo/cOYV/3DsKnLElc7XKViYeHUP3dBOJeOyR87k88zx/pl3k07jK1zYl46Bxyo5YC9o26xNe9LrK5DjxaPRTKvoC2Qvrqocn/FFN8TrFNfo7bYJ+/KpNvUn7jwQpY47aHCa2rcNxzqGQV9r97Kxc0hBAjibUuq9oX7yMVRNeMYoPnXiTZ4X4lVnCy2zoG7x7FRL0K39aOuUqSYMTFZLqe6U1Op1aifEWXKUmWOHR9Bucyw7nc9TJ+XqI2T5IgxreBwv6Z3LoeyVm3Pg4yAlBJDvbD/5ILlnsMY1opjFNbCHHJ4na/VYAmXQjZZvjJy5OHkmazom4I/K2KPev/Mk80T2Ke2szp3qdpslYBX3dcD+NC1n91kLODslngAW91UQlEpY8TtuP1TiWR+eBR2kVQ7gFkHy/mRugiekZ8gT+iiMCoD+FUj3NYbwVwcOdgIvs6N/iBmZ73EJ45lsKG7sT8S5/xkJIA5HOdWFq9m+BQ73/+3z2xjtWv/k6/s+lEVwfQYhdhyCk3fsPtQhb3TPyR0E77hZhGsjIi9BC5zed58ocJtKVW/U0VbB9nbEnkllzDx7sWY4SGv6w3weGq9FXfIi2yCXNZEEezo3GmE3kZi9nauYnnquBGjRuy5ORe0LCHvyZNo8Tszf7yToy47VnckE1mVCm/AXXdrrAurYjpdLzTGpuJOA3sPB/Duuv9cA+px84tzBYXLGp3evl9g3vinViy4ynVt2KVxE2zhIxslfnhRm/iI5z6UpubCY3rhY0XsVlVXK8dhrIj63/jf2L8B/b9N/6PGoZyX1J2383T/gZS/8VG67BBTajBgxfafDCGiExBSZLYOX4ns/6cxfDN57i10J++DowPtWwnydPIB4s2caO2DBRLDCRooIcL3NP3Z2xNT4JfR4InIeHR5EHAyV4E1eeDgz92WX4IradL6ffcr3wcK9qLyjL0G/sTUqU3fV1tVJlERcA1ux+lYzZyTtXGtYifKDcMAgeriWHROYx86yi/bZ+Ki79odQXgZz5MyBV3TE2eoFay4myyK1PLoDoWilzXACLTvAwrXlH1hEWXkyY9BCz757wbA+Opt1sZeGwgmtFisjDGvxTV8BsYzvbEPzcW+1AxafT3nEJoeg4ZdUZKw8TeBQDTpx/mN7uduu3DaKgVNy4uTdcoj4Wjj23klhksqimK45/0vka8DqScDhWW43jU+wrjO+kJKY/mihz0D/9d/nsRrP78cT41+jDjQLpw3NiocgjK4WxQDUcD7QIrTpJVlJ/uyuiMfmx/dBmJIeKmP9B8ix+ijeSbjfip4IpFLPhqDVeY5LmPum0G2gbEKc5ZMjcxxQ2yTRBftFwZl0Br8adr1lm8ve3gUIzTltwi5XQu9USwMrhJgEJc2wo4XdiTHaUeLO57iHRJTIA2ln7D0eSPePv954gIr8Kp9QydKyYRdNTGeaCuqBt3OrXdm+R2CdWOoejP65FGKpeVTpZNXP6bIfWSNooxTvHMyDzOTLlFgd3K16c8Ka9XfMS/jgrAt8WXngcHsHHKFgVbam2jP5XaTLq8/DF6ey+BNWeX9eyLvcElfQUBlQHI4eKmSW2pZoQr/Gm3s7AatmqVftsLfBuwADP3bObTpwcj3FGNF7YD/RizZwyRSQVop3YAIhPacim6PAB5znUm9jiA3NaL2++8ZLPxy8FJ5B/pyYcvfIi7rhbHNog6ax09ctoLmi3uVoI8RaWNjESXrC68U9DKiwl5bHMCzSRZIuriFj4+8hLhc4McOOLto8EtlV26jfSJLWTf0ALMrfmgT/0nfsmSzPEzRrR/9ePsuAf/0YdKSIzeM5rg8mCujeqFycUX2QH1PVTTl/1fBXG5axYfpZ/jlWTxm7vH76T1xWyOfJJKbW4k8oviefeacoyWwQcYciaFVvdfEMA+SUWv84MYdsifsRH1lN3llHQC++yptA4/yuCEMrrIsQi2vxoPfupcgz38V76ptvO+56x/bLxi5Spc553A2FDIQ/4/Ud76E462qn/VruXoSfj0i9546ysB0Renxr+Oxq7XiGrxo9H1GI5gX4h3PL/+sQD3aiPDXl/GWbtTUyiNJ/tvBtIVmP3kChT9WAGbysYobxsLfcBkacLRAtpXMvJ43xaIz+BW3fMOWznIi/6Z60MK6PebC8+nnWeLY8szSaLP/dcwHzajOuuNv6s4X0RGvsCbHz+Ef/5r6M02OqvEDddAzxJGBFbQbV8EumZns0qoMvlRtrb9ChcH3qmI22UdPQrb1+MgtbLvmDrPhZGfvcGZu3fiN6lDx2P36svP5/pi7/QYy+Z+zS9NouXZY52fYN3Z8WRoTpLt9Qp5jfeT4MBqzjbepLL+OPfsvIejQ5WKlwC3Ejo1+PBGeQSZCdedTsqVPV0uURx/i9DAeqLVfymOf83vDANdIM8M5a0HcS4IP+JfyjthoJJ+wtb2JLi2g31d5E9InbWan46+hlHdiNXug2P6/nBRFA32G1jUFtxswWKHQ50Pf+5NIyUrhS+bZxIzvr8AnVarwvlm1TDCD/WhV3Qp+qmiLkolyxwbdIxt9x3jvEEJOMutxXw8/iL50Q08kVikKNCXtd6iIPhTVCh7YEh2Ca1JS+H1KKx+3jiupr7GK8zsf519jbksHrSP6X6iD9bt4sWpHucwuBiYIYvrToNcRotbMDv8SjGolI4Go8OfYo1Lfw5vHYDL2BKFyj6k6Wt+mLaYAVfT2Fb5ABOc3utxVe3Ij1prwtVBSyZLEhO2TSC1IIbBQy9wKFkkHgx3zSaHWKznu9JLdYBWnVhorg5t4rdHf2JvGBxrina6XuDa6oLe4EKZVwMmSUn+kmV40LuNIe6gtivzwTllnnioGwlPeByJazgq+JFkTPJT7E79HJXFVQAD+/p/S1p4NWMqn8CnJYZmcxNuf6vYLDYLZ0PatdIlQRBuFZ9rF3M5u8Pan/vCwHOcMl8DB7BPQmLrpK1M3zCd40W+3OO0Zp1ozma97lOkMjeC3bwVv6mx+hiLfd6h69dPkTlO2esq0nqRX/JTOatupaJzh2LA7hbB/jOJDD08lOU9vqHNqhMY6mfLwjm98Bly+52gNPEVCgxzcGSo/2JPYMsdH7Hws0WMODYIldN5b4y5ic+UGnZVhhOiPyz8ZoAdZa9w4yt4Eggaq7SSvaZJ5q+tpYQXBKJ+XJnjTtjZl+lX4ml2a6bCdc4/hT5X43UGmA9RFlvGpwMPsEklEoK+PD6cyOVdGTjnGDMdkem/R+eCvri0nGb0a6doC3JqpqDxoCYuFV1eNdtefxzvTxWHc3DAMRrCbzKuFP6lux2vVEZTJrfnxl/rRNJbV88g+ntaGK5uY3vb1ziup+uKI7h2bAKr4h9kW0Ga0JLAQ+eBOvFb1qUu5UrkQrrYZwoAkll248iuPrRW+FHvXY/N6d05YQzj6DOjebpTLraHlIbZBZYQPrsagHQYbvReB4iItF3lypY+x4mvCeBs1WBFv9aKOC3hn37A0hvTeUDVQbbTm0t5pms5H/Y7y+ex2YQ2DsGx5HvSnsLjJZsh6ihEHaVX45+Mccg/jjV3or9BjyTbGJ2Spzjv+d5QaQW9sQWVJFo4J7q484P2UXyeXsFEF5n9NiVQ/pzXSSa5waBjM/mw+1hFPKPwUz7XzOGVoAouGIq4rRI6WJtEzUkDJWEl/B5cwVq1mD8f9WmjZsxOGLMTb2MKXzqAIZJKz6YLUfQ/3ge3xiZuNBwCh6tizP6KXZ2X8dFXz1HuW4PsJP0f6HaEOWY/Tj/zOEHjM5CdfpYsyWgt/hgs/hgA2QkYOK4aSe+cpcRpIM7WiR0OC1ds81xqqrVEXM9iTIoIqOktVQwefInCwAa6HE/HvUGLowpNkqAipAGveRtY0L2QCstdOCpAe6mfo7NfHqovnuVCX7EfIJKENSyNRpe1fHU1EI8BPRWFU5+bDxByoTu6/EDce4hvnqR24Vi3iwzGhc1h17lOE7d79vnb8hlQG4ShIIwnCoOxPSrORfV4c8QA0rkINEbxN7cPFRfWjkTX6MYpl4M41lYkJPpXmmmWyiithjv8nPr+6vx5t07N6YdW0HB9Kb97KZ/BYdo3WdISyk+WK1zSbBZidrUr33lUEK9SMfHqq0L+UShNZ2vudLZ5vc1nrXauqMQ89pNLPqTtSGDyoxeo8VeSOZfdDCEACIsrJsC/EBxg9gLPEbzw3imCQks4MWofC2URSAwNncD49+fzScJfWCKVhAqdaygnzt0kZdImsgJHMNIJdIuwFzLZF0qt0Ch3AB6SpZGVDTGsWj8Wa2kQafGia0H3qDGsCrZTGrmBzmG1eKjFetPaos/Ynv4sNMOPzbAr1alXoHk6J68a2RN3hDW6W6y/0pG3yTLUNnoye2k7lefcvTUKUG2LPIyjs/awKO0NdnuB3dwAqnaSgY9bMKvOfUtlyB6eeO4n3o7rAHa97KWsCIJt3a+z06sBH/tIofZytHYtVyN+Yn3GWxxrfIjFwxzyJklDvkWm1gR3mNxBY1Sc1/bol3lo60N4yfBLwQhe793x3tXZuzGv2IujlgaOJNzAbn4V6ADDF7sU8+ecYNZlzuWs8U0ecPpsFS1419TT5tqKa3wR2My0G1HDOyG3GOQOESP3kquDlzwf+Oc4WYYl/Qq45OFOZEwn3CLEHFaW4dXL9zCi/Bytka0E6pQq+oHX0qgbsBcGwBKt2Du9n9tVxsm9iPY5z4kWZXsV6e/8bkdMJRfr3gLW/hPT2U14yXC40R/v2umYQzpyf1mlQ27TobFoGHB8ALljxM+2R/bmZg8/wnKX8di1J+EHMXc5ZoiivsxK3zMJlChLmsgyTP/BE9m6mOC3RXcAo1sUi6rhgVsxNNV2oYePsh9muvZ95oxdxmcnfsWdQYA4n82dt42yVi3bNw5hSmpH/vGS9SiLwiDJ0r73m2cXrXReS57Oyo1zuHi1P+XeZhxPXWVrYrz0DnsvDqDpYi/OB4rz5MWmQyx98Q1e+u4JPn31EaJ/FvdSs31CCJAMxLzwMU12Jd09MKgfH+1cgudKOzsmHUOyGXC04C8zNlFkbuWHGg3GjBOMuC2+U2nJrfKlLS6fPZ2yKG/y+ucaAzS5j+HjDxZx9doVDvAnw3Q6PjC2E+7LC+GXMjhs+YPhjKbm89kkrRTrljNQYbzZmZw1Y8nvdVkAmyVrK69HGFj6zmcc3nOIMYFd+W/874z/wL7/xv/akGWZ2NhYwTbKq1VF0o0kSm9Ak3ezom+Cb7Uvo470wTWrCzELzwkxtbWFV33AkJJLmW8jyZJoN6K2m1hxqwcrj/iRGf65k8ln+xiYF8RgrxIeNU1nqk7cIOutlTz6R7t0f12XC8x28ExurTxLZHYm+wt9yPWqEyzPZBlSe94ga1dvPn7sRTKfrMPB+ZEaVShfdDnP86vuYdSusahHisnZSelZvrqURaPqITpb+3KH0zkXMJeC7J/pPugC3glK5d5tdvdPDeBtT1Y0rv+yuZH9877mRDhktyX/szVpacojMfkDqk4WEpcxkJphImAXFvckN3rvo6omF5cqN0VPhiuaF1gdf5TQ9MEMsCobu/vkQUr6CT4enImH2zghZtUF8OmRJEp3DGLt6F1810/J3rlmepj1lTd4wzeLOKsykQGJNbPWANC9rif9/4Y1umiu8klJZ5pvBlHl7o9NIya7z3vr0cnNRFnO42URmdjuWnfsblO4NnUdB7rVs1YrqlICwubw9Dp3fJo3kBJbTniqmMC7NGeQUL2b0/QhpUsGOBkzya3FbA6FowaoT30BWl4Ftw52qKXqLbbPepMFN4ajrvgFrKb2njo2K1MHfsoJj0DejVuOzlVMbuxqNw5ndIHrsUww2dCPEjf93sZm7jkxiDYbXAgrJNWpKqtTdwBVcmx/nEdV2zfc2HSUMOBqw3fc4QSkFxKJ60cf8EOzFRfXkYrjL57vxOWv76TbgIv063uB33camfP/Q/K+QiMzKM6DMFnF20WdFdXkoGIf+pydzOnep1EFOi13Wk+erIKFBwfwyPHe+H1+VQjbffvQz+8aDxrGMHLfSOQlSmbco2Wh1Ev5uGJXbC4kIPlaO9qV61VLirZjc3KzRyNBvd9F3dKXBN+LGGwOmzlZJiC2hKJyX8btHEdJJ7Fgq7fWIjW1zz2B02JJiRYZp6Nd2gjeNgVrm5ZDQw4hzRPPO6tmOQFFRZiR2ZlTJcxF7ectsWvcLpr1oCvpxPN68d3LtnbCeK0WbVwjsncHUzxKZeKJB4z89acv3XdcpO+cTMpVDkVfm5qAiiC6WXWE1Mwiubc4u0dpd5GQsI6P+wzE4GJQ9EV40reO6ooINu4cidfMC04nLWGt88TtXHeOngPr3T44jyI5iO1DV/BeNGRK3or4vOYcpJ9fosmjFtXUjjWpm17HHYHlfPTLAj67aGTcdlFlRmMjnmU5fHzuDjTedr51+ty3/czU+uayuksu7swUbVr0Ppy9PBS/8nNUB1UgOW34ffQ+XO81mocef53KNqciyd9jqn0Uhw17+aEBLunFwqlGVhEZWM7OU3dSqr1HUD9LEthkmT2j9lAZKJISatvq6VtXQUyPIi7dXcjplleEuCxD3Nk1tKldWbrgU76XRcDuo9o0nqhqYvTOGlQRGTjbrFyREqjrk0GbRyL3xyq757rrPJny8mIA4r9Sgm469zbMWpnl+V6kmtP+UTdJEnhU59Fq9GZhFFjkSEHRFWVYxVej3uDNXyeiyvel4q4LOFoY5Zd8zYidiWh8Gnm0l8goBfjmbCoZWweyDWiaL4I48d5RzKm6xsaQAZwzm5mhVgJjF42DMdlX4yUDklKFn5oZzjsvzqNgcAY/T+8AfivMjYTGVNNS9hqJdpms8g+JiepYM1POxTN493Ruxt7kzEClQu7ooKPINpku27pQ2lVUdxRpu/Nz0CZm0YfcHhdI1DwsxGNtt3jeQ8VT1zyploMVRWxcw3nS3Ehi8iWsZl9ecLaqrfuZmtT2e2i2He/I+yyNLD47CPuOYWQOv4OIZNEu0KL2pmrMQbKagBYlMzneWsGSjBEcCivlVKKy583QPpcI7PsXvxbp8LIEKOIZcjytKddouh5Drk35bmXbgln5y0imX0nE46mtOIP4bw44x9E2Kw8FZfO0/P0/75bWXE/XRl80Vf6cWDcS81Oi6vWEKeGffjBvvvwNr0wQCyEv+Bfib3LBeLAb3sHiGqixVPPlptkU57Sr7qu/d8r3as/zctjjbH3uBSoC6og7olQaz9auoZfPAOZ55lBuawUH20lsVqLcshma04lSj7YO+zrg29oEmkyNqPt/T1f3bympPkyQd/v8LtlsnJADOV3hySl/PVaVeD01bjE8c7w3wfs60Sm+FPUdonojWiokcmgWP/fIItcMP0g/C/FWcwUvfPwCAIdmOYHsgEkTwMr9qUgV/izyUxI96tVRnPlxKgDzfnG0eJa4kHqBoogiKiwGghuciiwaibKQMnpdSMOeX071gLM4gn1I7Vlv2sMb6eduRXLqFbi/xQNX33IuPXycCpf7caasvdp2jEk+XfCQXejhpnR6KFeF4//bDPyBAY/uARYLcbVFTYNnA39N/ou5ckcfLI/GPRh316LtHcaYGBgW0004LtdNR123i1xpGciZvR/xhej4x8Dpp8mpcuWZ3Gfo5x4ovPMtdjuPRL/D4MT+dLnShSiNmGvKMnxiNvDq8rsZFZMHnyt+Fn1sz7JZnvX33zvZ/voNp/u+L9k15FnSIj/B8b0rr/WgtSiY0u47qXQvwm6/u2NesBr5fnQUO6oSmNMGzj37wkMf4drNH1Bj5cG2Jt5QicrsnW0J+KRmkn4unaRtI2CheM7V9QPQvhtEZ6AhaivOYJ8sS3jU+hB1tgcZQ5ROJGdtSZy5epqkm0HI3a3wdy84t4ARvLTlDUJONJFa70ptjJinamokZq6ZSXG3LKxpV9EhzglGHzuLFy/GXYKvbT4KcLV7gYYJ3zxAQHkI6heHCCuxp7maOzTr+fV4b6bHXObexFCch4/2EY6X/4V3w1hMtjYcXSIAEo2z+bLmYX5rbOX39I5r6lKnYfLWyewbsY+K4ArF/H2bmDffCwxmcW+pVWkZcnIysTf8GLdsMQdMInnAqPZi6+50fKLKuGPObvaqxSzWTW6lcttQrGY131b58LwTTi5ZW5iQsIIaQxg3a3uCPRwcevNJKg1W4IYZ6k0OSjJzM0PC13HzVCkWn3qOphcJwKyLuYSvAuHHI+GUXS4gsLkexwdJkuBh/2o8T8XymWszowc1keQA9m1v7Ur5h33xAcbWOAGrdjvFbSsx66tpkq1oJCeZMSAHZOAWGkJ1vRu1VnE9lS0GorV2DAf6cVSSCEjreM5c7U089+5PPLe+L7qDfWmxiZuZS6Rw7aX5tHiVcW1wM/MtrTgWmle1DKH7RRtlcbnk9RKJu5Ikcc4g0ft0b5LKg7E5IyX6AMpvZqHJbMRHl4CLRvm7AtTX6KGDzDZZqaCT1YRVdGPh4TS69X0C6FDc5DTso166TnxuHI2ejYq55nKXTK75lTH7h/sYPrgOXhM/e0VcMQnvLmZe0zNY/UVyoiTLJGV3IqbVgz9SWih2mm98XX0Ysy2QQ6Z57JlxlvucOqxIkkRM1nF0JeUcHPEMC52cY05L/VlwayNHQ11Re3TUGSSVO8vORaMrDaLz3EJ0MZ2dPheKuozFL/l9fnPzYI0s7mvDLUW8tOoe9DfiKA4rRp4gnndOyWhKPsjnfffu3Jx56Z/PhPa53Vfq6NXb00OZX1yTNnAj7GO2tUCTMZzn1R33003nxuC9bqjK0oj4ZjOhDkQnN3sDtR8/TCfXJjZMX80wtVgTSlcVE3NgNIFHbDQE7EdytKl1DeWxg6kM2zqIbg3edJ25X/Gc3HbSaLBBuckd2WHDbiCU1Q3utGkbsAIqSXxGvUs20tdjKjuMccL1uD3q8p/nwSV/UShJvB9XTJmthtsg/s2sd7l6uQB9wnb2RJziGa1YVzxjvZe66GCe7v0g2c3TcLTAlSS4FP8mxe9kMtQV/BCJ6ZIEnXJHMnj3QPaN3Ic8VJzo/LTZjPJvb7VhNYm1LIBIayHv+8EnddBa2/1vo8f2MbX0a8ZEqQjPO8XFknk8FtoxD3pYSkg3tINGOtc2Jvo41U7cJKrCI/FWVRJUqccF0T73z+oeJP/Uk7EVwbjM24HzeGvwENZvbFeCX6sR1dGyLOFT68PF7jeJMHykuBcARbYptNZk80y/+zjJ5zgv5vWSREu9Ox60CQStfa2duJ7RhJ97BTX+NQryeJ5+BAFz9ZT8qSazpBdTHQyyaixG1qwZj9li4scFX5DgJdIHvWhirCtc7ZRNsw1mq8V9b5HPEj58oRMP3ahH855zFQFwCaZihw+e1HLLv4qBTjaeLfokJpaA1qxlrONpu8fy/NV4pt2Mpr9RxQGf9lrp7esW49GJtfYKzsoHOGqz08co5hdzzHAKGM9+cnxzSHIirC1q8WDLqhlkpmZyMv0CQqVA50fnPDW19kZS/AJQ6ZWEiv9bxr/hHf+T4z+w77/xvzYkScLTU7RkmN/zBiN77mHE4ShKfVB4qC/NSiM7qwvXRx3gpns3HE0SXSUjd1zuQYusZ9BTO9hoEz2oZbWWLQe7I9+MInPst4Cyr5PeHMru1hKKC4YwzT1CiGktNXTpdosrJ7tyQ9+MrOqYuO4Z9zCaARKPnIhkv8UmgH2SBANu+ZNgbyQ1ogibVkxmby82ftGlJAS0oLabhHiy/UGKtu+hPnEzF8MO4dy46YjhBOcfXk5EWAvlNmfNFKgkM5/7w4VmX7qqlGoDSZJQWVT0K7IywnvQP2zB5qpTzKk7w46cfsx99RdKo0WLgUbdQP7MGsjw+wNYFFjNDqsI4rTPaXbe9AWdoosKqKwqAmsDiI9toUgnJgMWl3B+qdYwvSic9LPp/2pBUGgZy8aapZyOyOK4RVkAerDUh8AzsymfsA8PS8f3R6oKCTe785fOTH63yYpkYfOh5TRsOUnahJVUJIhFKVdZzb6p77D/fCe+WDuItlEiiOnj3QXTwTLkm2aO+tQy814RdMhSxbHeZqdXZDnR8XkomnC5hjOnHMa5wtzYn6DtYQHsq7S7sr4JRgedYUJiJDReAJ/ugJ3BnVZSFeRD/+N/0dVfBIC0HjF8PvwmM73r6HVgOO5ObO40SwElu0dQ2u80Pw44yAMqcUFKVC+njGhI9iMqJpd2A82O0SwNwOi+n6GDz9KEDWfVbJ6UwJetbYxzhxCNqHwFWOfyCqG6i3wUc4Uu6TM5+NcnnAL6KP6yY5wCDlvBMiGP9/+YiGd5MF5LvIS/edLeTMnFgZj8ain1EH+ThETvU73xPt4b31EZNOjEa6bWaGm0gSnxFgmuHnj+S+P4CyZQqWBu1wW4mn4DOopqQW25fPX6Kt6ocOETax13qDtYd9sNMVy0naLvCTMr1Il81t/wT8cqSVIxIfo8YR4FPPzVfPqFiGBHmz6C4/NeYYjRj+fHnuaY7TCO96OzWxiRj3zFli8nk3grTmkjZ97J3R8e5qHrnmT5BAOiTDNFJ/NbzSNkr4ojP3oE6ufFZ1ySJGJvxeKeUMC4QalAe6FdbW6gZ+JKzgzqji3ETJEmHa3DV7v7t3DojY/o+8d8Ik+00EklvlsL8lwwXE7njj459I6tVJz3vWe64V4sEfT4L9z0V6rFAkJrODNqH23GMB7wUza3kySJoOvx9LmiZuLUexH0CoYKHvSaxgZdX2xdcwgydbCx690n8/g7JzE3rqa093VMdYH0drBjqyt+hcRzAWRaMzne/xzf8rLwvZ23jOfCgVSOPvcx3X3E51OSoCBlAqtmvMV7pbA0QZwvXGR4PXo/67Jc2WWvRuzY0D4+Cy3AooWUnGAklVMy7RLKuztWE39hHXJoHdBRjOuneoYumgb+SM2kq7sIGlhtVtQt1dQYdTx4PYfeA3wFxp5L60mSBt3iam0y066fInWiWNy3l3kz+7NnAQtxSZ/iDPbJahVnJuxiosmKu1QIiPTOUNcIrk7JIolSLrYNFRQaAGNiblHmHU1SfgwqB1p+Temf9G44QMPJcE4YnyNxvPicFJX15tcnFiObQevaht5pE/pUxBlax9Sgim8mPELZB/ZkVkdh+5q/uGmXG7L4dnBfQjcNYW9bJPJw5XyhVv3JpJunadXfYmO3oYr4O74NjHNvYlenLCQHcsAGQzjLqwpYfPAxfC0aXJ8Q59BJkYHUBjYSdysOj0Qn+1BLK092ruSSx37Wd75GnGszj/HxP2FJksjunM1bb3xA3xt7ecBLLGol2rKJPz6QQxuGUddfuc5KWh8qNo4k9kQ/Lox4GNnJMKFZFcifTeBh8yRKnkzK7byvpZAZiZVsKy7HLvsq1mG9ZxovXnyaHo3N/FDlRcAU0VIn0lZFxd5BPDr4PLvHZ3PVaRPVlxaMG8fz0oFexI9V6ou+bGjjzPgtMB7u/ZcGzrtaTHjsHUgbEGa8qoi/WBNCScpZmg1in8M0dw+Gvvk7r19L5ufKOgaELBSOU2um8+f9Bxn3VxiyTWlZ1qvNh7zNvcg43p3zg/PElVTtQktUEfwN9qWoRJCzzWanxNRO5Kmu88HPQ+keMNjtLFoZ3qhS86TaSRloaeKuqq84tWsWyX2ugN0Gfxe+Gs/E03v9GKSHV/HG0BtkWev/OUySJK7t6k/d8VQGRlTg2UdUUsYG9qLm+DK8rm8k83osplEisBUhl3C3L4SrIVPJV2OYfSMFXd25lRXHHD+lPa7dJZgsk50UYGVgi6DOA6hTRbF83tskexswNk36x9tC15rLuq71rNkfRfin93Bwtng9ff3qKX3qN3w3TUBzKRk/SQRxAiyN3H9gFOdiSiiKvsqjTjnC51Uh3PnNLPKSrnNy/GmnlRZsOmh9YykfBMJF23icGeiyLHErPocBPkYsaiU5IKbnDTL7HCRGgkB7xzO+w15HyZz1mC52IvCHuTS8I64rLSFGzo/bybyfe9Ia2Or8sYyO+xQvtxC+bmvjcF0vYL0Qt8tmDg89jKvHTh7tJNo6qVQQXTKb0m5ZNEUrQS+A6Z5lfBrYDq/ITVfApQNAlWSJFm0JvYrNxDWOxpHXv6DbdQxdjzH/x0ep6rwcSZqNw4FcvtSLkotBTE0owRIqkiZ9dQFURfUh5OZxnvzmSaTx4r0a572QNckHCGy6wGl9k6iOBtK0Eynnc7oNuExJT6VSR7LbiBx2hus9rqJynaeIq4wSk3ePJbDYg4g7/4S/aVYuen9M1YOB7SRfS0ZrFIub0W0VmLM7k3wtGffdTfgvE8GUR7zzeVHWM7G8DbNNWUgqtqgIKG/P4U4bVwIOea53Cj8f/Ja2HddQ9xlF0ExnA37I5xm+avqSuwf0p7b+exC7HdMrYCWJfiaerwHkjvnkqZiDVDxtxfDXEKZVRKEfIe4B1zVKnF/9BBGd8/EadFGIeahkFt5xieuVagJUoHPqS13q3Zs9JyuIK4rC1yuIALvYt+yAYQKnwhaT0M1GZs8cxdrS1lbBY+kdPbTP04IjeOVuq2WgHq6ZIL3FwfLAVMujqQ/wycm5ZKps7KsPcOiQCGbXLoy94Itvq0TSvJNc9BujcDGZaHNn297+HLIaGKUW353fmsqxjs3hhWAPxg47g2NfdYBnpIdYNTKJ71wriZKcO13BC6YRnLtjDSqrivslESx0a8jkeKyZl+7exHeelXyr6vjuCn0PVLng1+MkUT2uMzDQqc8sEk0eTQyJL2fxsN8w294WrpeMii+f/oLyBCvHbK6KY6Mrghm3qx28KUc5wfsUWQk8sp226Ahk6T4EtTkw2ft5ntTCykYbKkstOLRCCVarePj7aVTbJU57dqafwysQnPM5I7f2BuaiTsxDelkkufYNqMbuWYo13kK5m5LEKkkSF82wUP8ZCxrSGOegNevtG8vhMd0xBLzD93leTHZ2ULE0IPVog5MaPGuVACaGMoZ3X811m5KkCmCS3ci3wPvFyQxzffYfUFmt9SDXZR2WoW2crQkkzE0jPGOSuZaner3Iqb3hfH5sOOb3xXcnhatcLOqBEYgOqkdlE+fJ06otGGNuEZsXS+21KFR+/ENklSR4P20HQ1bm8ciVqZhzv1KcemdbFf32TcB6PZWifg/Cwo7nQbJb6JF+nOxbrtxdoKGb7yBu726r1J3J0ByjVdtKTx0ESCJJarb+HBtPP4gZCB95EJNlOo7Eg8T6FLwbvGmIKWCDm8XBP6V9BLXlMdwFLpsgVteKLDnYiwLY2x+c3kXwfvJChJ3DjAZ+H3eFlPMfED8jDy/b4zj2civEj3M+ZVTUdiL/4hsMHdhB9LuRl47HkTxmZc6hOuwNBrzQsReSZVh/9WXcKk9S8ee7qO9MwFGbJ0nwYVgJqm1jKN3fl6SvxXdHlmG8NhFrWw6P53Sjr+GYED/V8Axf71uEza4iPV1ypCkBEEkFL/nCiGJQt4o5mW/CA5y9kA/8yLWI55Gkjnk/rnYF60Zd55WtjTRHlpLTlshdDsfWF4/h1zsPs6oJFvrASrvoPqQ2ygRXBOEZVcrUbkrXL9zPEvX5SRYU6unpLa43cVIzX1b05ebWPkTMeh+t+TFArNVua4HVzccYbYPOnrKiJ+CS69GMmv4X93Quw+Kwu8yoSCbqlyiecm8hMiWXrIliL/tXKhKocNvN8IgQrkftB4cGFS02Eyum/UGICgp61rO6SbTtjeYW0/NT+LxLLn8FFjJHElWYFnUg5R6hRIXdYID9XxBMoCwmjsudt3IwLJ/7VGI+eBuYnO1XxYS+E8G25R/JelFUe84cURzBGxWllJeXc/7cTxRcW8X4odWU1lSTwf97va0PsAMrg27mYa3OwdFV6X2dAcvMjWQE1KJxc8ofZDWFTa5M3zSRad0WEqTqBzzL/43j3/CO/8nxvwMx/jf+G4DVauXy5ctYrR3sIK39ZfbXjuVK8lVq/RsUxzSlXUGavp21fTKo9BSZ89HBfdj41zz2bu/PU5VglcVChkqt4/vR2wHwrFaCTwBPGh9mlCtcD39FETPoE3h10np+fWcxjZ1ykR1YH2fx5JhrHRN2TCAtM1U4TpKg/7GeDDzRn+Z5awkKFr87wXie79bdQ3GVFw+O2YhZLRZKpOZqIm9kMXqbC6OyjijOq9VeSpFs4Y9DPQhoe0ERl+12FvrASI82BbMNoKe2iNffeZ3FSxYLBXaLRwLLit1obXTjt5sfUaF+SDjOVNtE7Pn1NG37mK03nqJLkMj5b617mDFr9rPxgcW8eUhkJAE8aR/DoG8fZvF3E9nfIqp8JEkiKyWL1ogSehZFo7co2drXTQeoMLkx+fTThHVdoYgPMsuYriQy6Vo3fM0dZfJj5rFMiDtD6fvvcuecrhTVnhaOq7PFI7XambJ6Fp/ddGJbSzLZ1f24dCMO6+E+FNaLTFhJgib/GJrdWjgy+IgCsGhSebB84C4ef+B7ZjaJSTaASu/H8dz5PFoJcdkx4N1diJeqU5lRDkubm/izfAjc7sEhqfC/1olZZU285v8SDwWOEI6L9YmlR9ExbvYGj5e/o8ZfLGTfdA2m9rnvOTTw2N+/Qzzvwwm5yG99xlMLn6V/hLgBBVC56Nj+0C12jt7DFWuiIg4SUzZPoc8Li7m3rk4R7eQzi98fKeJk/A3U3kF88OGHTNLpUBr5to9TwCSdjmlzX+Clb9/BUOXPlU5z0apFsCMy/TqjXl1G7wPDuf+6OB9IlmbusbVfh1dTTlKsjRbieizMy15K33BXHp26Ax+t0jLNThNJWlgacxHv1u1iUB/I6eqp5Kjb5zdHq1lti4qw4jAGHxlMz3M9hefk9rWv961l8BcfYhwhJo12lQu+x3thXjWexR8upapR7BtZ5XE/L+cfoc+3b/HBkhWKd/6W1Im3tFW8OuAmD0eIhSMAN3MRMVVX0NYYkG1mRXy45hhjPvmMHo+vQ1Y7WCn79aGgv5U19ZP5Ot6H76yJgqXOOJc6TkVC+uhb3PPyb7ghqnENNTaGbp6Ia+ZwIkyd0TkVovtf/IDUzZNYr2rimtZbcV67O5/k6RnH8EruRbKnssg9Wz7A3JVzmbVmFlMjxA0EkkTxlSDs1V48OyCHOEPGPyG7yg1ddj5h+V7sHL8TyVUsRAxP3sXgmftIu5BG6gWRTQ1wUfMUBnc7g44OwrVG5FeV111j1iRXHtifwKvvvfovTbvN9CODiO/vo88JcS64PdYVPsQndRCosiEh9p1ssxio1XyBKqCGmAF3iQfeukTphkietgSyKtgJCJck5u8dzyPfPw6Xq2hrFDcWeWWr6R5yAI+TRTxjexZvuwimuHkYuNHrLLrH/6Dizp2Kc45XFbO2Pp6h68PQ5K75l19lZc3M9XQed5x8c7wiGuL1Auo7jxI0fQ/h0s1//t/QmsvUGbtpjS2h3xF3uhlE5fQhSy1FIW3snr6Bn1/9gFa1qPg6rg0kc8JunjFO5qvLyvP+cfRx9ozaw5fzv8Tm6rSW6gJZvflp5L8GMWbPGNL0yh4w7TbMgehMwWhVyuL9Ta9mPnnuU8pCxWKySlLT9XJXKAnmfM9ztGlFBdLcfvuZ9fIu7n73e2aOzBJist3IG37wbGItKydkMShQ7N0YbbnEb0HQy5KCb/NAXDXieZ/XDWP11fZicXWwMieTJInAykBUNhWeNSWK4k+1thOzy2FOfgxba37vyPu0gfxm2siF2Bd4KvoxxrmK60qgazCj902j28pIyvZ44VqXL8TPahLZuugrckqiWL3iEQLdxLX4u31DyDzQnlfoQ5V9O9xrtSz6+im+uNKbrqpcRVyWJHaP3s2fs/5kbagyf9nX6sklI0x1Az8HPY3OtztvHtrJvrbE9vvo1P+rl8dU4mo+Jv+RYzQ+9itGq8ik3X51JznHu3M98To53UWrejRevJ5yktPTN6F940sq1CLpxubVmclBmSxevJiz45/Dx1VpXzrY83u6aeG9Gh2S8zOo0nPqb+VQWY+Lgo1ni5+RzNRMLMtn8uTiuZh0HddbVmlYkpjJycFHqKwOpqReVJLJMtQHJbF12jU23LEByVecB8/be/DUSw9x8/HFzCmPVqxZ5XIc2qd+5/7l7xOTKOY20F7o2D5hOx8/9zFrmpWqbkmCwqhCdnlVkW3pUMyq7W2k68AzqJaQAZncFZkjHBduK+aXYDDP3M7VxR9TpxHX2iGqK0QfGUDNlTherlHWbO2ShIvBhfSz6XiVKZ/BdSGn6aWHcRlLaNCkKuJ9zSd449WVLJ22ngOuwxXxH8PLeSYrnSUX+jFB7gAqazVBrEjIolFrJKIoAskmntiyoENcipTQmGppddmj+NxBl71YuieYB1Y8wPACUb2htjWw7Mg43vroRSbHLEVtFp9RlQreuaUn6nwab3UX16Pbo4/3amI1UGUFJPFZuFV1ltCYD7HZocbsRI5BQrJLBKNijqv4XtWbWvj8oI7WQxGk/nQfdyEWEXWmG3TxPwqAHF+CryySKse7HObrlL2smL2KvT2UfWQjA84y7Ms/+WPqRrZZlCoJFTYSl81l+L6RzPA/roinlZYQWBxOXEouLerVHQGbBbVDOj42XvzuxMhbLP5jCU1jDlLvU0eTWtyTpOCJdk9/Xnr/JUZcV+Y9SSo/9j7xI+UvfUVotGgtVme1cSi8vv38TkOY+794qgHRWjPP+bTbPTqPON8/WehrIVwNaqnDCj/YQ090p2bcPdvwNmvx0YjP/7UWf5pKA7m6vze/7lPOr2GxJdT6ykza3oXdFWJeJSGxZcoWfrnvDIu2ZWJRiXO//Pf+sfjuTST7NyispY0qd4YUwxs18HmtBruTqiredJSjEXA9SsVfvRwUGLoAZuZH8vvgDZzsexL3RnGtlLU+lN4MI2n3SFa35nPcUi+etwQ/HvoBgBa3Fpx347IkcarvKXYGRLPmphMZWZL4qSSUwhV38F5hFx7wqFBcM5VJhbdFi7dnKzpZBHksrlF8sjMNvz+nobZohblKlmQiC6J4OK8P5+LqmOIt2q3N9E1g5ANmWoYf5/cSd8X+sJu2gAe8bSzfkc7RS+L+T5IkTndupUufLKY8uok4Xb540i1FTO/RTuv2rrxBs0H5u5bUFtBqgzmedmScrakluoy+gmVkBm8PWiv8roDwavpM34//0FNMG3sGlcPzCfCEewUf+krs6t2Tik7KPpw7/RpZum8STy99GnOxuG/1bz7DNyPmo/5hAaUfv4efi+gQ1NyQxdvz3+WhXxfjNToT52GzttGt9z7MRSbu104GJ+KPHiNRathWG8LVlg7qgSRJyHo/2tx1tKlO02gXiUjFdVcxHqxAKnDnZJ+TlGvEHCFTm86hJR8Q/d1iXn12PZ5qcT+T5vsNMxetZvm85dzoLNaqJAn2VCfyQS3sr42lziiuhwB9KcTjeC+8azRoqsX5xmBsYfr0H/D2KeTV917FxWFfIak0/HH3nxgDqpn76ULuNIuWxL8a+3H8hS9J+vx9npm4HIPVwXnD3Mhb98ZS/ZSaz+9dwc5ApStQ74rV/BoES3zhePe96KUOwlyw6ji5SaXc9/c87KzsO1t1HTlzJ7o6I3E11/GwZwvxEm0yf+zugW7xTLYfmouk6ahN2iJjuDrwYVytDaQ17kRfISrZNNXL6Xx6N/pqE/tMfwoxWYb7vGqg2Q2rDwQFK2svj49+jvAupZTdCic5VrS0bKGWjITRnEwc/a8KuBb/WfS5NIwTbR2Wnv+MLi+TYX+O+V6wPbwVrdTRTqHCcwwvNFgoDS0jy72e60ZxP1Jb4seyJ5/n8q5+LKiCBlmMPxtuIHL+KfalTuXVDCWRrkeBjnkNJjqXb2CEm1iLDdPqGNv7DEkjT9PLdysljco+tSXmy1RY4fcmqHECAgF6nOhCxFcP0cfsTWfvjvnKEmZh3fR1oDNSei4ZZ4ikSLsOo3yB2IfXMW6MWDdUyRKlYaVUBlRxqMSL8lYR2CmVItnw3R0EH+vLNwHgahPfraPlP3Gr93c0lgTyy2qxZQFAccEOlsyfQNT0ve15kDM50lzGO37tz/f0KIdalLGGBa4d70pGTT1R4eG4nX+PT5uzGVFTzTD+v4n1/B0fqoJTu8U61w61nY2dL/Gx91iWezo1hbHbCLerib2RyMX1KdysUYpN/m8Z/4Z3/E+O/8C+/8b/6nB+8K/VT2f5vmd53tSVJV7KQsWTXtUs6XoGncaKWhI3kpJKy7WUe7icUETZ8juEfmbQzpKtCqjiwR/f5Zl52/71fB7v8jZ7wmBVaBu0iDZdstqFG2Z4yw8qYkFyKIQvLe/KKyU65AfXcN8k0UNdliEJmYA2F1YHSYx1FQsKkixjMauwWlTU29r/7Ti6+b7IQ2/9hId/PSnuShbuUPURfm2JZMihAfg2eividllPeB74q6z0cX9HEU/VdgBhfXUdTFy7PoDv+hzjiye/50LVaBpt4gY399bj+JZeofniDTZseQi7kz1BgvriP2ypmy3KwueHsftQu7YRfzMeXbm4cmlbC1kfApMe2Mr9C9bRxStKcfxotw/Z0X0DZ8lDdlEmnW/HV9IYUcz+IynUuHQs6FbJjSordDrbg5izidgcLBQB6mLqOT4pBHVYLUVm8XstwHubHqMuI4o1M9dgjBTP21iynF+fSiT904+ZNfICart4v1SyTEhpCM9+8iydzinPWZZkQupm0GqHKou23aLTYdzeZO1thfsaMsE19HaAmOwIXnv3NfZ/OYYje0QLDMlUzbykBQRJRp7VVVCkFi2MSlUBfOlezkMXerM9pzfOY481iq/KdXz+3FP8tC1WEU+QxrPWI4P9ZxK4ZVfeqxQpi7QL7YzitVbl7+7j+RHT/G+SoAEPycCiZ5/l5ffeY5BKZhgqVgL7gD+A0Xo9g1UqXn7vPeZPnoLeYEG22Sl06adIgF6tnMk0Yz36cYfICxfvhbfGlYS/+zzF5MUoNrCa5iss9F3GzU2dWLJxLWiV89GxqHo2hcD4EmjyFm2d8Ejgy33fkfrXMDbeTEftwKCcXNDKw8se5taog9TdvxaVrWNzIiExP7cr9+yawLhyG181i4CEt84dlba9oCldquNw6V4hbrQaMGrK2WG08WurTQEglUsRrMz3JrgygGRZybI1+o5i65EHQIZFDz6jmAe7uHam+dBATs5/hXkZHV1D6o0NfD3nNTod+5prEc+RH/itcD88XR7g69Xvc+TW3bxSuZgCjWiHqQ9s4tC9K7l2Opn3P/lMcS9dukdzfqQVk9aEbFHuXD6riqXFKrF+wPN4GfYp4lV/N2O3eRhxMzsV8/SBPBNoZtv043xd+Dp+CR3z5JXaPZyZ8jZVD62kMQ46W0UW+kpVFEtTzpB871+8MVwEZgGaU0ZS1buZfif7MaFNVDqo7G2MN7nRSS1h73YNrdN8YULNO3+0Mw+D0pWbIoC99Ul000Fhp0okk0iKaDPWEu+Sh5tNTXcPceP9dqiFLVM3ceGbeXz105dCTEJi5vBLyC5tRF7dw+HyRUI8W9+bJbneAOxfMxKbWizKfpTYxsqF23g1KJdXq5Ss5kGuERz9bC4F2dHsqVH2GZIkmR8qItnZ0s4adx7V5l5cPJXMGJMnsVLHvSz3HEqMXMSxSccZNf8oSWHixjrbJZNlD3zAia6XyTUDkljk29GaxNfXAhm3rgH5hhLsqwlooGbocbZ1r2WEiwiq1UiufKtvwhRbQ9rcHLQ6ZY/PHpq3uBjbwLemZ+npqiz+D6l8hT+CwZ4ANHao6OLUeUz2tlAZWsq+kEJsKvGarri1jS9ObWZZznoOF28VYl7uUbzw3HvseWgxtiY3VIgFr56unqRkpjLuvQnMbrgPlVkstrWo/fhj/DZmffAVr4/IwHloWotYNC6XsLhiHuj/gaKwentebXS9yGlbe6HTarWCrKGyrRMltjMcX5ZObqYIGqhNxcQ37wNZomTYJBrCRWcBg8qFc141NJf6kHlpIJJKBA0kQztZoN5XT4aXsrHYPR5X8ar2x+tyMmGyUvmXaj3H4uknMKdcR+VcCPl7dNbCplBIsl/o+E+tDzmXO9Ep14f39T6ESmLhqROf89XAFF7xyuCbxH1Y7U7qvoAAMsaHI83aSmywqGLzc/Gjc/kG9iWV8IpcSyniWnrbqipBA8n+R5FQbixfqUrlrnKwqloU9wqVnsV9D7F48WIWBN0Q0KuXex7l52e2kJ10nWL3RlB3PL8SEgXRBewafoDjo5+kLl0snMqtN/luajKLhhyjc//L6HQi0BLs1pPWv0lu639+Bq2TQjlH3Y9Z5ZB2JZ2MeqVqW9tWwHoXT+7P7UJ4vTN/HUItWRwLh9GuMNCto2eOxTuVSb8OxHPpg6SOPE1glPjsFxhieHT5cLafi+OXRmiVxRwgSx1N3atLCb/UlftX3K9Yaxf4l5Pw+Bq0Zi1TqpQkJ7tBw63HF9P3WztHapSgm6RRU92sJ6BiOE+kPqeIv3gllvO/T2TvqjFowzvsLm+ThwyztvLmz++TrhbBp4y8BNb+PppeD2/gx4XKwlLw0XTS9g+n3/jjpCaKvcMku42wgAZsrS50rdyG3CLO3yoVRCfl039CBscr5vNv47PSe+mc366iwNOpR1fDWY5EwCIf8FeL6+ETJh2PFiVy/qll7Bst5j0AJ/plkDH4MI1Td3JQEkGxxsbjTB71G+FdCph3917SPKKFuC9nqdjgzeT9IwhSKUFKd7mIXq4F2Kp9MBqVc7ddVlOWnI2+0y3y2sIU8fCgMobN2M+yQQf4y9Jh8WWpP8ZLQ9rfF49BLpTrpwrHhUUt45Fdx1jZvYl37ltOq1N7iYM1Szm2cTBao54FLsrzPh3VypZe9XyXUkNfnXivShtL2Bv6KNLdOxh99260BmWOYTItYqxPKSkFcEurBJwfqi1kbRPkxYDWYe0o8LnIhzvKORW5kk0Jywn2F8mgjTHHOHDHKNZNX8fxTk6kM7Ub713O4a9TA5mxfgaeReI+yLclhxdjmpjzezrxp1cp5rF4XRNzcePljOG806xT5JIqtZ4jBni7Fp6v1CnygFJNN56uhD9qfThT5eAroHZhX7M3TUXB3Pv7vSysFL9YJavIi8ljzV2rafBqQHYC2WUZGgnGFlPIG3GNqJ2AqwWeZzgdrOZwyWE+KlcSWD+zeVISnc9VUxItmkmKeJ/aWu75/lFmf/gcqTqRTGp1j+EXrzJqB57iqfAmVA5qyYEBPbh/10L0K8bx3R+LaWoVi/PhUgFz4ldS8Nn9FC+9R/G9o1yLebIugrotI8lyAvsAtmgbaLhvPTtiDcgaJ921zURUxFHSn/2VlkdW0WBW2qevbpbwuAnaHLBqnKxm9f58FzqfdwbuxSK3CPf6R5dkxnU9ymvDdzLTxRuNk7X68n2DWP7ki9TKz7HXRbQvByjOjqbuWE9863wJsYlrg827J9/s+5wWnwiqI3oo9o8FZgszyyBMDfd4/Ysa3TWSXtWtqNKycA0vxNl2+E6XavJjIDJoByftDl3bzc281bcXI62vM3DbPqJaRHvxOtmF46c7k68ysW/UcdQe4lxVKweyuxU+uj6NB7bk4+UuqnGiNU1M8DOxa1A5z4a7IUmijWfmudXUfPkd0/8IxmZXArOH1PFkPbkMjycyeHDm62DpuJ9NZiNSDvzoe4PKHhfp59GR82lpJ97oZA8Mah8s9ieFz71mC2O73MLLLUZ6FILKsW+GqYFO5jfoH3QWJBTrMIB/r89YVXcfu1rhq9LO2FUdNSkreopMaprt8JQXxNlFN50zWT9wbObPHB98mIUROVRqJghxWZKYuL19L1wUfEiIdU4aycJ7OlE21073h4vBoS2GLMO4iOVIPg388dQ3tEYrc4ReF3pSerIbxb4B4OOs/Ielp3/iRqdJXOrzCFLiY0LcTVVI58CDjA/bR5VejAHY1F6UmNWcj4QHgpR9au12G7Ea6OtiRe1AHqj3GMjn9XAzOg+3FjfF9T6l6kSOXxWGG9Gc2tWHFlmsYfRwz2Fi14uozKNpsygdVEBiwtneLKr5mR4ap/MKHsEjx6o4lhXLr38Op1atJLhMdMngaDh4y0qbcICQHtexRBbz0IblGHUdrjT3uJzixRFXWPLUl7zx0nu02MR8b4ncxBPfPUG3zB5M9RRrJ2pbM1UxML88jsyXF3H+jEh4q1f5s2bmGlyafUjaM5zO7mKdrKfhR470OM61fvWY/JWtbuySTG29G322TGB53nRkp9zexVbBq77wcyNEZ4d0EPWMNSwMqEaOK2ZTeC4bMs5zxGrlgLWNOUAWCK0u/r/GfRY4fVisnQTZUgms7U/Bu+E07nKa50x1XO9ei+c9m7mUconVJmfPjf+7xv8W0Af/2Xj+N/4PG/bKSnoc2gOHppMy/bwiHmUaR9cbJYw63YWweaLqRJZsaDxt6ApdiLwVp1iAfLTuHNd1xbWphIOS5NQtrX18k/Ua47s9ygxPM5gbhViYawgLf13OzbxCli5ZygK54/WJvurD3VueJeDxVfSOF4tWkgTP3L+TyquRvP7bdAImidZit/Q9eW72vYw625u7jgxFHitOCBta8vD1L+GrhV+gtcTgpM+gzBbG1T+64RpYT2h0ISAuIiqVmhILDHAzkm/agrMx/cbWHmS99RaXo+2csXYce/v6DTraj+4nviB7eDrQkeS4B9Ywc8UqPnzjOcKv7AZE5clKbQ8y311Mnhm8LUp7mE2Vyym/cz324ib8vMSF0dVuYmiLN99nd+PbuhV8+biyYa0kgVoCP60Blb0FEAugt0w6GoceQG1Rk6rtMO1L0EVxR8GLtK5yYaPWyKhFIvPtwZh+BCW0cZf+MOkBYq+IJmMT10JfIzJnKJ8Eh0OAWGwrNLdxodVGshbu8oB8rZjIRDRsY7Mxgp3NHgRp/8WvqvkWW0ZN41cDrK4yg7VNAK2DrDf4OqC9/+K1NnGjmeDeSkNgJUUBtRyNihINYCxNGA9WEX6+P0OnLGbwzGjxWiKhN+hpPdiXW/GFRDsDXwYVT3z7BBIS151YsADZbSYOvP0gdwL5Q5XJlTcNJM3N5CtjG7n/AtwO8n6NTi56vtLDGW07KLHomWfw7BbOa/Of5qUcIxarFdlXxxOLFvDbQw8RHBxM3p+PAu2bv6aoVzBa38DR2mOy6XOunHiNk26+RLuIgLOnZzSrzT8SzQqm7xmHfky9eE3cIjkn3YU5SEdTZQvWf9lAlDY9zG7V9+xshQ9dRAVSi6kFvxvvIxWnUVoSwUS/jk1wbHwhkbMqCErO4bG4es7bOr5bkiRGl0dx7lQ6YYlXsQeICrdYjYWEy+3P7MZpGxkauECIHy/5mL6HarDtep737/2VWU7JrizJLFy6kF+AY+OuMsWpHldlamTHsJNMCtqBm3sJ2MX56Kb6Tdbmf4R/UB4mfUdxyWCoosFymeDqNN757QFUw68JfQwbjEnUbN1BddphNqf9ROf6PxhDB3C8IPQ8YxOKeH6THblJqeR9PH0kE11c2PDxs+juylLEV13txqQf7+USMOZIgJMhGviF/cn2u3NZP7E35c2NwNfi7/I/xqRDo6g2emFf0sEkDzEeJ2PIFT7b1pMVn92F/lnxnb5sCuC81ciw3udplDQMdPreGtfPKe29njED2rjsnirEJJdgpmzryvDznXnk7R/JdReV2QZbG3v7rePH/r7o+yj7kgHsC5iCjwHswJNOtlBqaxMfzMrgu36XmV4cL2gdavwMNLjfYNReLaUNyp5l94VlEbGokC/V3uzzFjdrVlmPf177e3xwjJHexkY60/E3Dbo5rLwQQvzNo7S5K98bg8t8bPLHXO1ymdYEJXmgsDmP79oKWRkMR5pPgKKzJISeT+BbWug6eAS3t0a3Cy4tvuVEBhnIdxELXoOlLJ6OgNfOhtFc74E8QiyOujZomLV6Fl4NWpoCPgBE0sSfQXVkzl/MHqD7q+I82Gpu5WTnnzidpGX8eSOfKIXA1Ft/Id0rjwfSnqeuOQCcjHU+SPmJ7h6wrlHNDAe5R5I6h6nDs0mPz6bYqNzA3rxVjHw8l7zw/vgkiWutVuOCW1k7uLJlwfNcWygW0I2hz/Djge4EcZK6UheajPU42gJrZTs2z2au+DRzyxCMWOYASdbiktDGQ28tI7/NQ1F4DbDc4vtAcJfAYvsQ+NshwGZFV72RhKuXOTDsCDWROhynIrupnIXvfcg3eSNZXl2Kv3UaMx0Mq3zUenqUduHMMCuu8lxFQbe1bx6H4w5x1tVMisaMc97TKbQEW0oOOdejqLImMls8HC8a6Nbkzb2lvhj+pVh3PtJOpVXN7HILAyNS//n/kqZifPKXE3gyCuPmBfR5IVU4rtWu4swvwym/FswP96zl9RHiift5HSM1sYgPElrYXSfmNHrJzBtJG9h3dQDnqurgXvFYVVsFr9i8Cf9jMul9X8do6gWIoPPq1m085wNfBIBsV/bguj18m8Tc55bFHZMBtty1Di2whA5ChCRJPOUFuWbYleZPjm0077P7n3h+Yx5Gayn9LHqme2v5DrGo1curD1m9EskPqMMouyueb5UsM3vlbBJzEuHpGsVvku0muJBMwOahfDZfZN0DxLlFEiL5sTushirbNODk358rUe1fzZXkKyytayNA7ivYf7UaPAndPxhTVD6tLXroLALlNwlnjaaOxanX6Osuo3Zq3DvSrYn4pHqOvfgzWe5KktN9lQP+6eOntitV9OboOeya0JlhgM9jR3G0jQK4dD2aSOBG91jM3h0rT4/Ws3z42XM0pF9k9117adSLRavtJcn0ORnB+agi6mMLcS4DNiTlccq9hrU9TxPoMlywUZZd/BnX+Th39fSD1aOIvbsex9VDpYLXy38i5exebAX/rhTbavuT4r9/rvN7a9BFML8SvgqEIaptOLZbGH4jiZMb+rPx0e8pD67gM4fjJLudh7qUcyW+nM9aoJuT+fN5izvF792HbJP5wvt77p0skhdXmp+mJXM3aSZ4prvyfV9Ra+GLP8aQltmDnNnKeIRHHIW14bgdC+NcqrJf6073dLZ0fY82O9znQB5oRM9eqYWCUQcpUi2izeAvrDqSPZjmHCt6nR+4KgvZKjcXrvTuhc+Ah9gV3uT0hEBG9dOcunKRm2XVSP06cZ9DzK3xMpYEOBNxmrbaYbT+C7HGX9rDHV7wWg1YZSVxp96q4c8mKLfCCAcwpa2wgh7bf6QscQhlCYMV91mSwKpRU9jlCq4m8TlpaGsg/+g8+p3vTtXdG/GOFtdJn7Y8Fqpd+TiqCE1EHgb7AKAj/x7iUk7vS+kc3jaYpuorTJ4oHC6w3CUkxZpVp03gqwaAambUz2et8PcqQktDcfWrJyxJJMK5tV2nqUczy7IjKFkzC/3rN3EehrgjdBm/iVS/VsrF6QQVds5/cxdzLidwuo9SDWZXQVh+NFkEEKEepYjP8jvP6r8N11XqVPE3SzJ1PnX4Hu9FT28zbWkduY+uLYv77v+KVT/cT+VOG55jvIVjK1zv4atXwzGrdnOp62UFsFUtfcfBL1dh1hq52kNU1EpInLx1i5KsFtxa/bhrvtgWAI84Im950uzRDkRPd7aWbv/l2AAb7Ta/jqOmtYZjwaP4tjoeV79W4Tm7rVYKLnyPmPKX8RbTYy57WWnuepmXM/tj0CvVvC+obEydtoP7e+eyJ1CcJQvtLmTtbia4ppaxr+9Blp3MArVe6L+9hyW34jj73A5l3iRJhJSFcMitjre88jDJ4run8RjNT1f3sDz4NIXGr8AhO7p+1IumncGEdMonOtqJlC6r+fjZTxip8eaGbzQl3k5tGIBO1zqRkhUJUgWSLO7HPyoYxC8re+DmfY5j+TupbQ2jV2dPpsyeTXTMw2jN3rjntIN8idHvgWBkC8X4cTigmAH+xQR6yO1W4Le/W5bpe6IvQRVBPPVAIVH+HbWyMLmBb6/3Yq+rK8fT38DFCQOSJBm/aj9m//Aol7pdQnLsNagP4qDlOHt3XOLral+CRyjvJWETOdqawzYznK1J46fkjrmslh7MvhVBqz4XewLk2HYCHXuxh6tXENHDyqTgEtQWT6ySeHLR5ixKB1/A7t3IuxPOIVvvAtqVXZ5yCeF2E7/5fsuuqqnsVgXiSG8ZPO4kuaMyeMMNdjSJZAtZhnJJxbYJ2/A1LvpXdd5v9q+JrTIiuc8FHhBi3d138Nrf3JCDzcreeHn1mbj47f0bCBDhgOuH5qBvWsXj1fBCuQ9fR3TkEBqVGrXVg6SiKGKudWKsj0hUsgZK/DHvF+b+PpfhB4YjPS6e+FbDAU58dYSeWe9QOS4G5/3dXR4m4raNJxvo0Vt0EzGYDTS75LNn+O722p7s1HAY8JJbGOgCpTGwyXIMmCrEH+7ZyPafg4kyVQlzf283V6oawumfvQZ3YyIRvUQS66SIYraNOMnWc3Fc0Bk453i5JRUrXnmc1pIgjvc7jsWpNijT3i6hbs9U9h+cysw3RRXcDaI5d9REYJMdbaJS1GHxSOKDX0aRejGVQVMPK9asZn0yifntbglGq6YjsXKL5KN962m7uZsrrOAYVkHFVwcoq6v/PoIAQ4tIXtxormfbsuGYgePW+Qg0P5Web+u0nOx+gY1xFxjvoiTq/Tf+Z8Z/yr7/xv9Ro9XvflwT2tUPe2uUTNYXzsbTfeNw6ur9MNtF4XF59XlWjA3j6wVrueuHd1HbRdacSlaTvaYHXz37NN/nKqXdAJ+7v8vAYpByAKfeNZq263jntZ/b0ps+4AD2DYm5iTzoND/UTGDBSXHBliR4ttyb5RcicTuagLdZXJxus3CHXutC+rl0tHaR2bbTEMOLOZ5UNLpQYVECLTlSMrXdsxg0+iRtslLS3tm7G/1bxpGYD4urlUXTRrsb12w2hpTa+bW1w7de23CJY+Vd6X+iP7LKSryLqP44RG+CCuw03rmSLo+vpqj2hNPnunPLDBciISNa2ePidMN4iiJr+fqhfXQPFC39gqMm8sn+I1jW9EdV04CsUk5Vq5vuYlIpXE7dR+NVpb1BcJMeT7uK4vBiwSaxt/YUi13bpegZAzIUDVP3FgazanN/3OublcmV3cbcwVeJfXYF3YKzCbKKu7VG924s3tWNjQ8s5qXnfsTDQ4QcTLowThW2F6ru/JfNHLKWApOGWe5wNvEWNIrXxcdWwpPe8LofbAhrgdYOVeZvIzMY+Pa3rJ+xmqyo/cJxRRYbK8LaE+uhW/YQVCYCBz0MObz04UtUxd/k84HgqhF3RbPdr+AaWsmBYQe40V3ZOWx7SztjR1JZGaQ6o4ifUY1kcPwp1nW5SIS2myI+oSiamVVN/JI7C7V7R7ltUvIQZg9YziP3LGb5w8P46ZNlvPraawQHt6cnF3XjaZ7TngzO2lGDySYmIsGlG+mc8TOeVcpNN5KEMSASXQ9P3PUmBng4NWB3C+XzlkBMzRWo27ZQ3Vqu+Ijtla/yfDVINg1OdT5sLTdJHr6b8t7n2B//GmqXDnuxn/QjGBNeyy9GE/fccsGmEa3HLsQ+Tek0F+79/V4mXnTarGm9iJlQS9jgSxyccJUoTb4QT5RySYsrw6XFjVnrX0LnpA5NtZ8mom+7Aq0+QMmirWstwBy5koO3Auh5MgTco50vG/HX2nBzNTIyquO6qlvy+XHBNmwPrkZT40Oy2ihuvD1U3Og5nqgCL95Y8gYqi6juOGQI5+0auPr6G5ie/gyrTQQZtzSHcURXS/jA81gClEC5ybXj73fX3VDEPTQ+ZHut59H8BHaZnAqvllYe9TIRWx6CW2O9cN4Nqijer4WLN4OpuZBEGSKTMOiGOy98+AI/HerCY+XKwvnDNcu482QX7jU1cNAmzgcqlYobcTc5Mmkb713YjI+HaFclY0fT6RYfdz3LUxX/xoCEzUEuPOIFv5zYCBqnv9H58MT+JJree5zQArFA3llnoIuPgZ0zfShLFwEiSZI4b4QTLo2sCyzkiiSCfXq7Aa+29jniWK+PKDeIisW8ugmcWtaL2X/O5oHLSqWNXe3GH4+0sP7Ojc6EZgBs2NoZxUCQrAQiS5sfQePegrreE4OqY6viYqvnib+mMf79h4gov8W7NeJ5qRslijaMZNDy+xi7eibuTlYqM3XX8GvypPuUQzw+9qTie/vrOgoX5S6iFazGUMLaYLhDCzmBL1PmpIgFWFw3FJ+bMOvc45j8xyvi+4on8lo1zCnx7FBtA2csQ+lRCJOXPsk3yx7H3SjOwQXNXxJSWEHX45twzz4nfqjdRv6dD5I5pJmS0BLsWueqq5qiTmN457XPePKeH6hyKviOtx4nRw7l6JuPcWj/fSiGWwQvr1/M2K/G0DszVjEPetjKedQL5njCNA+Ha2KqZrbLl4TnqsmNz6U2WARm8+xuhBZbeF+1G33QMioM4v14wiucyT/OYPAODaOHeFPRIl6TFyPzWJvQzNOHBtCzTKkCeqF2DEt6HWH9neuxScq86ph6FD88uxC+uZc+dUr1aZMthcv18ewo74nVpeO9ba7Yz/z7VqIalElrbAwtnmnCcR/XlvJdUnuupNs1VGH13dv/Th7wWM7b97zJ5U1PCDHJ1kZX23a81qYx/OBwRriJc7tsKGNOQwiV12LZsWIi5QYlaUKFnXBNu/pPkd9YDKwrT2bx4sW8f0q0clt3uSfLv53KG80hNMWDrq3jektITD04gsFvvcDum714oFUE89p0IdxzJIrfn1vABw+9gqFanIuCjGtYMSWSzulXsYUobTgjrNfagT6guV6pFrO4d2L15fY53TtUuaZZve9k3qZq9tx8kGp7h2W8t0riLv/noFMkQw8NxadCJAS5hnjyw6NrCNbYmbF+BiF20e5VQsKr3ov0qdncPfcwOqe+em9XvMyEzd15tsCVtZKS1GB3kWj46B1eWvYeIXblMzY1uUO949WwXhFvDYxk3Yyd3OhUT62xQ51qULmiCaqhoSKKVT+fJTFc7EnZPeVRtt4dwP5WePecEpDbMvISz806ylWVkRrrv4PB+/odZ8X9K6hSi9dbpYKDKQlom3NpNinbDgAEyl5MdIPPmAZG8Rk164L4ugEW18DuWnHtyA5fTGW/rsTnxhOdLxapJey87QczPCCs+h6iEZVkVtmV82ntBNLWco3C+eVg/c98Of8TAC6ccu74BFajge6Z7UTNHk52rwA6tZaqIJnM1EwUkyDQ33su47ZM4rX3XiHA2mEhZ3GJ5NdLEUTtHcbAXRc4fe174ThjaQVJJ39j7rZk/rw4CH9zvhC/Li3EcOdUdpqHU3JFeV77KhZjXZ5K9M6R1NeI75ZN68uPB7uw7eHXefRMOVcMyn7aPzelE58P7nK7+sZ5+KssXDLBwiqwO7hfhHofQxOjx6/hCqktK8EqFjBTze8yo3UV51x82BDppBCyWYgJq8DSOYcvxlyhV6D4bhR7D+KBlQOQrsexcOJOkETXggLbNA7fbL+HedpXFfOcbKnDntCuoN8UZkBCVFc7glnHXZ/vCLRVUZR8kXElMdQ3uvNtq1PxUuPHscIZtIXY6DT6BGo358+F+SHvcO21Bdy1JVWw+wNY2tyPg+b2Y54ZqSQ7/xCch2XhzxwZfIiCNiXh7ReXbuRM2cGF3mpanOabEHMF+2KiaU2+wecqg6BmtOjj2GV4jguTS9ky7QKXtWLvr2stuajyLhIcUs2saRmK69lEAtldAwm/axd3R4v3SpIkfJr86LNrA12P/PivgEVYYQiPffcYna51+tf4N4HN3IqG/nrAiRRhMTfx3PZpVH49F1VmonB8SlEbL6x4kFnLTHTOXonk5GpQGd3A/rG7Kb4SR90NJRkjXvUqX3c/TbqxlmInEnVeQy4ZfdcSMXE/fSNX0WwS8xdZkvBoas/HbZlK21/JYmBUfjyDjg1CbVHqLppcBvFh3lDyzCDgmxp3Frnmc6r3Kb6YuJEXVd7CcSosdHO10fliJw792AtLk7hXGmbbx6w1s+h8xYfQgqPYna5n2bFrbLq1mYLzpbxZW82atlIWZmdz9L33eOC+cFza7mT4/TtIfm4FjRolqUG2S7g3ufNBVhBRV9yFPYmMRHphDN0ud2PFhQ+57PLzP7EAj4Hs3jiRtkPtNHzn5+B1/Vbmfz0frVmLFkms26i0fLHhC6JOlFOdE0lLiUhUuj08VH54tHbDxRQpgCWSBOk32xVkg4vgrFZsWXPQcwS/1qrwNmmJkiXUklhb9LLV8MQj2xk29Sid9ca/+wG2j+8zhrD8sZfpe7wzTaq1VLV2rNOSBN1zA5l3IYDPdqSRUS4qwiUJ4twMjApo5SnpD27kimpHWQb3Zh1DjgxhhPoXdh8WVZ75xgE8XQnPVMHKf3E8CGncTU40LKiCVZUi3c2n/ixTb99euyxcr+5NG8kOisPDfRDdZ57Ey8kEMlku4E1fCLkzg0nP7qCnXryZlU25RGW1u4Q82FfZMunzoBZiZ+6iIKqO7ZK4Z84qOoKf6k5mbLwDb60VldPcDbDaMJ6+RXDRCBZJSR44VLqdyhlnaJj0PTfrOvYVrWE7+OXpBQzPvMJ47Ur0KvGz+5Un8UaX00TmxRCQ5wTm6dzYF3mTC31PMmbGRfomiOtdvD2bW9FQ/8RhsgY8rrAvvyh34ci1cPyyfLi/qzL/kCQJjbl9zv7ZtlLhAGeXXcgxt6/TnbQO563SU6ZK5zznGIyksOu0AsrK1r+PCkCyiXNonT0etZ+FDXfu5rdhTnVYtRurKuaTdSmWe5sDiFMrSVL/jf+Z8Z+y77/xvzZkWSYpKUlYtOP8byF7BHEDqIkYoDgmwu0cjUHJHEl9m4BIcbNmljV88dkM7EUhzPriF8Kd+w/IavZKI0jkFj1PKwuIAAatQ3HQKduwqf3osrCOy7f60atFTLTHdL2Cb1Izpvl9MCbmAWLx1HjkSxrr6sgcNoroYLHQMSMkHfnMRxhianCZWEuEl1hk0TdqeObzdk/7g/ftxnnIKhU/TNzMnzY1w+uGMdn5D+wWMlJ38mcTbGxQbkJdZDNBKjjZBt5uHRYxktqVU7vb2WqeyTepcxFZnbJZIio/ivioRmYmZLLfJFpheZmNjDvZn4MHY0ga6Ieztu8HIpi3YjZLIibS97kpOA97ZHtyHXtxNaUNc/H1FRNmCZkGG3xXD6E+vjjf0Y+rw8lc3ZN7Xv6VkpaL8PeVqZPSqbBNI2fmNjSxOaic5PBr8yKZunk4w/yPEB1/CljS8Z2SxMu+8O2BHvy8aRSJn0zC0YxQLcu4N7dnSGaTFsmJrdfo1Yc3U15kbtcsNln8uMP5R7uG06uinCEuMEbvyst60YrolmYwgbfe5RlvWORtBktHQvFaRRiXrSV85g+yVWRXWoC9PQ8S51/JuPxY3N1EUCHeN5m6gVeo06WSXP4Knk75UaRHHXd98B3TbXC+VWlPJ6lkMuau5u24Bq7LykbnE0MfoPizwUxzf4OUaUqFUpbJjTI7mG/cz0dDOoo0rs3n+XT4BM6UjKeT/wmueYtJ+N7GYr5N+Jzn+k9ApbEqCqf9En/l5jBXtGfM+MUdwpEdaTYbiPBcRX5oAC2hM+jk5dSfEbgc/QhPvObOvT42Km3KfphINnrr4GC4maZqkX2pNtewcMxFXux5kYJ9IkNsov4DXK68R6eMF2h2b4K54ruV3TIasyWS2Xc+Rn1otPidLiGsk5YS1L83gRUeGIPEeTBfTuL8zNcYEFVPopSkUBvIkkS/RzeSP3sjvk3KPnA+zefYo/Fjxeap7BupJA/Utf5G09gdjA8w0DO0o7hkdwljfiWcjMkhc9Gn+DcO4yuHfDS/5kkWPfEr+7f1I7ooErvdqTH8rUT6fzOcGZMOM3WaWGwAWLUlhNSjg9kzGUZEKK2EVnbJZctXW3m9BhbzsyLub9zKjO9dgTlcXesEilmamXOxK7savHjo5fdpu3oRQr8DoFEby6mfp9A1qyvvvvoub3j+JBx6f+Blilz7M2P9DPaOOqT43t2XYuFIOtHhahJ9hwgx2W4lOqYcXXEknC5BksXepWrZyoVAHZ+9/hAHEwrgdcXHY7HLmO2gMwcpi1pqFy63qgj0rSUsUCzAP37Zh5wtLxO66C0GJT4ODoUQCYnx+0YSXBTOC/f+Rj9XkRwTYzxDSLd8zu4L5A9PPdjEIuHl6r34lZfj6ldPr6FO4BNQ2XaOiT5/cndlFw7qlUC8hMSbvvB7g5YA2wJFXEMlUcmt3NQ0Cb1zfQ1XmBlTy6HzLryx5A2uLBKfsYymbkTu0NMUl0fgoHNYZNHCLj68lDuWv8NdZaCuCMF5tX3ldAvVI5uxqTRMChbfO5WlmV7FUVx9Zx5dgcb7s3BuHA8S9TZY4/kd8w2LCEIEUZfVa8h2B9Ti9WzFj4JSX+zlAbQl52JTi6DEFxP3UZV+ivIDfdCkbQQ6rGaNhipW3BHB9iY1Ww0WgnTzhGNL6o7hmvQ7LtpGmmxK7NU/cBJ//BiEd7meLiXKud1ut+NTtJvYW/0oiNmpeAaLdX3wvdnOLExSjebY7bxPZeZCn7Nc63KOcS5GWi1O/blkDVJZALOPDcX7Uhe8XxeLiG0unZE1hdjMEncY3KlyYJADtJR8wDdv3kID9Iw8qjhvrUFFdFkI4T5duCNguiLuWPC94J3ENKf4rqbV7D0Ig4B0h6Bd40lO7C2ueDfQnLGZJCchgyRJXEy9SK1v+z1+WfpYiP9+PZmQz9v1OY0+4vzcKul5/NJT9HJtZsCoUwQ4WTPJ3inMcy0itP9xsiMKuUOtJHcdjmjFXwUheVCvcQZybFiz2vOsMNd8ITKkbS5cPEt9sxdfll1i1MCOQp8kSeypHYK+zcyJ3ydgce4bKcvU+tbS6mJAY/FlcLjoT9GoTuKnNQvo5/s5cyc9iJ0mJAcuqr9UxbTvv6eu0ocjrkpLeI1KzaXuBVQEN9Nas440p7gkQbnXXywojGNm0Ih/MjqVoYDF/R/i6z9ep+ZSd4b1Ed/Xe4OCWNC/gK2edvTNZygX/RJ4wTuETl8s4kh0FN93vZ9NTsvSQM1rNO62kheTyzWndiYA09wK0WPhmzbQ+yhtwqk5w+iNVznb2sRh95kMcQpvTztPUWs5XpqXaMw+DAntgGCuazc+n/Mgs/+cT+fT29BIqcJxk7yO0C00hxufTuZ0r7M4j5/rXSnPDmD+hrEUjmkSYirM3F8djvVKIju6n1OqMFUwLyeZbnFl2GZuB95WfP63oafo4wawCYvpbcBf8TdLaiGtMV1Y8hp8+5HrX8jIrSMJ6i+S5SRJzb1fTiDuVC8WeLeSskjMgTVY8LC150J+RYe4XKMn1UGHH6Oux+7fwpZZ0NWiJJp66Zupu3sTncNruBzcQxGXsPG8rgBttJHDHtcV8dAyC93Ot+8W+rp0ADmSJFHj10E6GRkg9lS1Gt9m/LxM9v41mOubRjByvuggMdX9KH3MWqZlB+LpoSQtbNBNIr2TGyqrirZA0QLR6h7Lu9YmHrCpGLdzHDyjOJwWdHTVwuZQOGQ+AohWnq8HlXC3N/Q8soRA144c2sfyFPPeLGf5q4/idVVHXo0vsQkdmkVNxQbqt05h1dYFqCZcZICDi6KEhc9nHeev5uM8f+ENUnqItYDBiZM52z2Ii+oPmVprZHyECGI2S4mcC41FCrZiU2uVeZFKx9ommOkBI90sOGuBR2oquR4cxhl7CTaLA4Aqa8lo9qRs+jauZH5PN5M4j9ldwvlu8/sknvqD5ZO3EOcq+gdJEuytCcQFsEt2hUJOkiTWzFrDD4GAWnkvU9okTn8/l+eGnUZt/AxYLsSv2MPYmrYHnwQVE+xPAB3XRWc3oy8CL0MqbvVLkVUd332jrZ637HsY2nUzS73hgD0VRzXMjYaD1ERkcdfd+wh0Q7HPkiQ71THDeHDwIk61KN+NMb4vcoxgkOx42U4ADnZu5maG6M0EVwQza80s5A+VRD4PSUWMBjIi4IatCeiYK2WrgYDSEBqAxT5GHKnUfeUyqhvTaAIGhK1DkmaBA5lpRUgusREtpDzzKd6qLor0ep5PPSnVgzlUWofsKxa5g5rOsGriZb77dgqblo7j+wkiaOZqLuWFN35l+ZFudHdXWiRKphpenJDJ1iYfHm0IUcRLW/Mp8VvFmFIYJE8RqAu34vK4Gd+eN4eYxTVcY21iqymM3y4nkp8ThatdVIBKLp2Iv28vdnsrY1MKyW0eRQLtBNzPP/2UyqPnOYqdPk4gypy2Nk4BE47tpEeonRfcoXeJso42k3MM+bR9/tw9RmynoJJlPn9xLct29SBrbyqVEzrILDZtEJe6P4Ja8zFz9YOpMoUDq/6J6107rsB3c49S6vgM2m2URl2hdmwBOvXn3PDop/ACqd4zklcDz9DFnMr4wHPUyh1uUK5SKQ/EbKNH/mGu16YT3FcsgJzxGkLIR0ksrPVj9NxdaHrsATpqVldcR/DIht10v9idC+nVnBrb8d61eprJTM1kyJEhDLuZhPuTBdx+/mUZ/Ct9Gb+jP02lYRyeJ879sgz7ky5x8NRYzu/rjdc0kVgjSfDn4EPs2h1H67YRBDx2SIhXmLvx1d8loh6S8hlr1SfyWR3kWyDJCXw6FPcmszfOIU0HsToTsmThNmQg2wy4qhvxumwg48Z0QnqLdZmE5lL8Dw3lor+GiLQC6hysSwHyLe9gT29Bn1DAShcJ5xnj3lJPqtNOcjr5JCP1rjjORW6tN3kvoo29MWX8GdPGNestQFTVSkicaoN+xfBKtHLPUnu0Bv/rsbz/yjEOWjpIaxoXDSYPP9xuFGIobsM1vRnoIL4b7RqqPRvYvvBr7BqneqgksWfMHpJUGjYFmTlrEMkWyT79WfeojuCwOorGzqW8ZQk4aDxlWeZEvxOkFUXza0Y4HzulICpjDaGPrGO7YR0Nt2QkWZytgvS+9GyYybud1zLK1YH8Yreh17aQRxbv/AswagZ+AScfhH8fywGL02fsVf/K0b5ZyD0+INHmrThmjO0dbL+111uSv1aSLv9vGf+Gd/yPfv//yrf+N/4bfw+tVmTCfliXwo3z7RPg+SglE+VY/DL2jx5FQfBaiswXhJhd48mmuGtcDS3l1RWn8HUXJ1ujxci+IT/hMnUvqen/bol2IhwKo2GICwKQAlCLjifJhX5LeLbfXLB3JH+Plwygb7EdU7iJ4FBxkwmQeN2TrueqWDp5JPHqN4WYvuUc6swCjOe82XXzRzxcRVH1GO+beKS32xLcX6QEWpLlK3yS3Ysn338JbaESv5dkFUtL42mwQaxG2ZtmgZeB8ljoqnInVNehWrF6dOKDcVtYO2MtL43fSIZJBCGDmuqZ98s8TuzthToXjFoxmZgtn6XPrtHUH05g/1kxGQC4EGUisDQUTrVQV+W02Tc3ERd6Fr9RJ+k7/AxNDUoWYxfdNaa7w/wqKHDrqYjXeLYv8mu/v4vO4R2JYrE9ke/rxtC/z2V2JBjQOPXsqwitY9eYXZiqXbBXHBZiEjJpZ/04cSOUwlgNJS5iPxNP4y1mTjnO8Y8WkzNFotUkFkrUsgqbbOPClWiuFvy7uhTgsAHeqvIAFyeBvUpPlRVergHfG8Hg2WENuTE3hm6vv0TjU4sJPyb2TVDZrSRpYGPsFV6M68c5DxEID4pdwmrvtfj7xzEx9BOFRcB7jQMJv+jOkc2DKL2hPO8+LtcZ3ieb4f8Pe+8ZXVXVtf3/9j49J733HkIg9N57700EpSj2jgootohiwa6IBaWoYAPpvfcOAUJCqCmk957T/x+Omqyz847x//Q87xiv64v3fSZ7Z5e115rzmte8plzAMZvSudLbcmh1dh3XDiZSWatkUHpKDowSlHkcoszc5KjYPdqwbOl7bPp8PDP/KqfYpU/FPxvnx8N28Mag3Yqg/j1VI9s7n0aFBZtN7FtTWpPNg+rPCdl+GwpqkFwq4OSa6/xV2IaiV1/mm7MxOGTRDtA6YAKv+cKhBrDqxOdi9+qIz21IK3OjTfIgSmqb1gXD9YN0O/sZlT6VXGt7TRH0Z+oXk6Vbze+GJVz3EcGyzPJ09iV7ULUtikuvPU9ZuehIV8kh/FYLnQZfYMzI9Ypnck3dlzbZzt6jz/op1yqTPoK1vzu/l4SuGULPBYBQ+x88FVXNgbQIXr7exKCU9AHsO5OI9mBvbIANlfC3tZIVPxXkDzhF4dzfqXZpHG/TObgbc5cTWQE8fCpC8UzqPUwUhBTg1diHELVyb4jWVvKIu44DYRBuv6Owm6rXNF2r60PRePOpw0xJwm0uqxookkQguySghLK4u7zrBwF2kQdXHd1A/EtrsIUVMMhTCVR81vs8lwYeYcbPXRhc6JogquZUBAzKi8H/3mUqGsX1WaXSsmztUCyFAehsLbtrD+WFsKUOlnZbDBZxvZElmaCiINzKfZnvwrg+odFQG1JE/rm2XMoUv0mVrGKe5E5kdjRvvvMmHoUi8FSpjeJCtfN7yFy4AI1DrNIM8XmXHt8s4f3nPme6TllpQ/1y+h/uyZ4V07DeU1YsypLEGCN01LescX/Q1pXvJv/Jj0/tZri2qZK41i2BdX2OcOnB3zAlZxLqISbNrMHwxXNf8PvwUnbU/UKbYLE3wn7Tz/Q78DT1B3ty546y75KskqgJ30p89/sot4gJZbtXMhPTvAHwDy1Ba69VHN9aU8oEI+glMFlrFPZ7fj8xyQjf+hnA1HTtWslBZKkzMbg1zB2jhwiybGvwZq93MbXTt3EtUqxyqLfb+GDFWM4/8TpLbX60U7v0jSz9iRWhG3hhzxieLo5C5RCrNxoDJpEaMAvfrlbajshWXDPmSiLjnZW0vTyrFeZEn+7oyiZRZocah3P+a7Va0Hgwv1LFvhIDbd5/nhHnRf9Bdpi5f9sEvK+0Rep6hWIXKcIGYy/OjkjBorbwxi8D/u1X98+4dappffm+vZJR2sZSxZBdo3m4Jpdo+w6F3d9RzMWPUlj/4lkGtv1WYT9uWoF70mxWTwwisO7npsfhFsX4AthmLuN2cjcu1v8qHNdancciH/ixVy6H++cqkiXH8OBSx0scGe6gqrXInK+x1LGz4yLeXvgOX9r2UKIXU5CSxshZXS2bh+/letJ1BSgLcKI+gK110DbnC2XPPrWRZzpcZM3c1Xxh/EAwDehbRtLCBlQFcdzaPRk/H3ENLop7g53T4vn1/l/ZniCugxpbDQ8kFnPm9a+5PGIhOn/xm/+tLI2LZ0wcXDOaHz5X9qLyCf2QYSc+YGp9Ld/XuQo0g49aZlzEYCqGXGCzYYbCXlV7joTOEzBFL+aWqanvpGSM4sM/VnLvZjxPfPg1Q7uK88Am+XD+5lBOly7ko9J1hPrPFe9L7SB5aAaaslL8s06AS+WInyaDql41xN+OZ8BlFONlzww8N49gw9ZeVKuClP+g6BC9Gv7gp9xdnKlU+u6VphBOfzaFvUvnCUzxf957ddcsIifU4W4WE3qhlp100BzHNGMLY4crlRi27uqJ14q5hFT5MN5NrFBWOxp5oiqMuEP9Oe/ujb9N3GtVKphq9SI3MwqDWekzARyoDWF9Dcw4uxCVpxineTRkciUSHvQAjUOc/8XyTur9txH03E90GCX6c0gSN3ycVYKS7KKPCIRa0nlrcAZhbW/zxKs/obOI1f8zDKmsrInCo96GXadMKCcbb/L52Mvs86skWPuRwl5lriD9XGtuX4mnh05ZLRlo2oa7dw0+M7eRKTV9O3pTLov/Bg4jul1D4y9WjaRpLZzvep5DE4u41u9xfIJEYuSe4q58/NQC+uz3YjjKKon3Qvfy0Iu/s+at9UzzOyPYZEkmJyqHyla38PGtwWBT9ozylRtRSfBZBVRrlP2R9td4UWqDL1pvQ0fTnrfa1J+ZhXBy3A5uTtuMVeUpHPeym4Mjo3dS766nqEoELiVZx/A8+PZgMsYVRqQb4t/1Vmto27UdVPtgPdkZJHGepZv2s799DI/1b82rPScq1jlZ58n0QqeCj0emF5Isxs0qCbSynVFGmOrdTJVG68Ws7GR25HjT/8Q1ovL+wHXkBO0mPSmdKq8qheSqJMG+cmdM19VNhZta9Ae7aZ1V8I8Xw9h7IvkFYFBJInnh90gMqsJbqlLYBzKWlJQUll7yxGxxAVYD+7L73neYb5vwk7IE37zaUk7PIw5avTufBI3TVxeui/OsfXUDqzf25elvRir851DbIH7r0Z8PHn2V3b8qe4D2bv0Dr61+l4d+eIeL1X+KxsYivut7lbY90ug05liLe9aLJe0Ycg8+TX8YVC7fpsaLjc99zbq3llIcJD6TE9HenH7tU+58kkLG5D1UNIrzuyHjYbZ/+RjGOiOWFnzNjo0/4Ld8MFP+moJHmejbSJJznrgVBBFYHKi4bo1OS46hjrcnnmJq55cU50bnz+slEkXpcUypVZIdGsq+ZEP0PZK1TXKkANitPOzpYFxhJA/9dj+JeS7VdWo3fvh4BmVmmRUvHOOqRiQ6hcd8xG9lx9mSfJUf9WXYZWdiq7CwkFcWLWKX1aGo/vln9AB22By8+htEXY1CpVxmMWiG4t2qCLNfOa3j8sDeNJckSeb5ChtXb4USkHuVvcXfi/fsFYyxysL1DUNQuVRKlgWuJj+2C3b3WjbaG1A1/6gbSzjX8xr3jzuD2U3ZoxOgFhX5tmoCPTJJ9D+NTBM5zCjlM6fDYjoEnEJt16N28SVVkoo7sc597pi+jnpZXI/0Kh0zfp1Bx8sdMdYahPUmKqYA/YNbqJi0k5HDz6M3ifHjmNxQwvLDODD4AJWBIuYoSbA2fRZnzyZxsdNF7rnslZIE160GpAFnyOhxlgu+I0V7c4hfUr6sOmNHFpfBEAN0cHNRJJM12IDnvGFDZA1qqcmnq4lJ4bHtt8luM5riGOVsOVHVmSN/DeTEuQ48vTWNep2Izz0TeIT3nt3K14mXWVUWoDj+YJ2Ou40ysw0qYjSi795giOB9/7tkO+z8vH4IJpVSCSJWncf97mCQUMQFAFZrEXX1Gp4rkrCom769CrZinZvF5UmZZA8opMhlP+xW7+CNd99gzOfP8H2iGGfJDgtv+0LPwlBeXfA9Z9NEIojedzoF2k4YbBbWBRzC3CBKzfZ1XOK7IAemej2XKsW9EkBtKuJjLzVz6gLx8flOYfe3ZnG+6x/4qeCDkmbPpP4eKZ0704ipRblOLXAEONOCrfk4AxwDBXl8nWoIpxO6MH/3OJZku7TosdYxJaypYvR4nYgh/782XPMd/5Pjv8q+/8b/2rDb7Vy9epV27dr922zXo5mMzldms+KYz43dife08HB5BOVVicAv/9pkWeJc93O0utGahDO/IEmvCcda7BYGRx5jYQ84UtuyNM1NC/Q0wOFwoCEfPOKbHW8G0zmiCjzo0mWLUPnXJus+2h7uQK9pmxje7gYgNnuecd9ybBVl3Kqv5YrmJM15io0+w7jX0YuysgC09ZVIkrdwbBvPYkY8e54PUs+SblCWpIfJZWSddG6muhbA5vz6XF6ou4U5BK7otgEpgt2mmsy2GwZSWy0nX3UYeA4AH4MP8Q3fYatbzqRNHbDOFT2oZI8bxM8qx75uOHFXk5HGifYMXQi6qTu5kNOf8DBln7ZCtQXNhH007BiNwySy4vLzj/Jo7Fh2BzkTKeM0CxTH99Ufp48PrKpueUPfVfEKQRFbkOx2vIKbGLx3Gs9xsvVw+r73LO/ozYzdJzrLT4UU4hZQxfKIe0jugUIvH0mWseaEEH+tLbdm/UKuphSasYM9rEU87wOPeYEhbA5Xa9oRHtzEM/dpSOctmx/VB4dwSqeU3MFcxWIfONMIJ2qUHmtX7448mLOBLdFTsdvETbfj+ST0Jue8/qEqSOg9o7NV8OPVPhy+GsUbU14js94IdP3XLsvgwAGbr2FIyMIJWjX9fdkmMXT/ULIvd+TM4My/Z0jT6KS9TdWq+3jsbhzDf5qvuG6fhp0ERRZSWBXNrdBPFMHE+ZirlNrgZuiHFJZVAE5n5pbZREZDFdH5ZjoWp3CpczvGjm26swnyZtr+9Ci3yox8Mv9zRTXuqUYHWR5F8PIyYqyjebKZzSHr+MGjEFXb60iGtzhXPJGRzd+23UR0QDUXQgtZXQ2T1cp+JT3dr+KtgrgsuDV4omCTVBq67hlK9xN9CY/Ppf7FavBwJhCiVduwyBA64g7v9bpEgyQC7G/cu8SlPT05OzIYn0gxiai2lPBpXjwl15IpCizCaHTp0Smr6HWyFy+qbDQEz+LENDvNeT2jQmfR+GgDb1cW4DMvFkQ1QkyGOLbHXme8VzWr2pU4kw7N7v2CrQc7LlkIOtWNgx2zmu5XkuhwuQNtMtoQ6FdDnZ9NULO6VN2J9JdCSe90iS+H7WdphHjdD8XkEPXyKXa/8DyWCk8FUDJ36GksSWpabwnCK0oFiDKIn6ffwPrl+4SEFuH5sXIdXNuQQMGMDSyOqqXM3g5oJpGn0rI15h5htjo25+r4xGv4v9zjIEs6b007yb17NxlYHMWNOBEUWFXbmVPmq1x5ezVeDmXgYdZZaEy4Q1e1Gzpfcd+R1B4s3NWegHvBzH1tNacqk6HZl6tSaTgiNTAxrJiUaUqZN4BadRaddDDQ6wjYxXVUZS7jy4nnKe97i+WN3kJlyIaYBmoCNvDiZy/yY42O5rVNnjpPfMcE4BN5GW7I9PRxIb5o4zgl7aUnUBRYRIxmoGCP1NTQW2OnDlDZlW5mAaHcO+V0fFODlYxTWZaIyYLxRhtJqv2AmJSTkSmxweaSBCLjm2SBTbpQVlZD17bX2TzqOqvqxA6KkZoSxkdVcCUrFM+GDni4+N6xche6nb9J3LUADg9VguBtfXcSdHkHyel6Kib9Bcxrds0qLne4TE7ny3jJ8LHGtQscPOhxhYF/E0XTq/aBC6d1aWADzwUDNIC1FnTOBNcEt1ukDMzh1cYAfOoWEO0lAl5f50ZjqXRWT+k1gQLkK6ndMJ1wrvXLX34W+zIx0VKkbc2e3R3RnehGcm4EmifEd211WCiKCOHxKR9xo3oY//bc++f8lipemXiWqv7XUQWKsqgA/loNve2JZOsgSlUn+H3GOjcCSnwp0DZyyeIh9JMymHN5ZOxZNpWm8lbb88zy+1A477a8lVwKW4elq547sXeY75I0M3ZrQ/r1THYN/RCbXaxoAXg+qoSNxgZObOvL8CeUjePHuJl4MRQev/FAi+BRa917RMbkkdfQBoPc9EwlSeK+3++jTUYb2vW5wqUhF4GmBFR7TTah+waQmhEJL/5MO5eTN3haODZ5Dzeit7Cxag2Q9a9NJdlJ1kJtvY461VmqLLEIVVF2K77oeM3fxFA3kB3KdfCT8ijCdEVM8M1FQtwbAEoDcikNgCza0zx5FaPdTP/2m/mzWxEOScbPTUy+ypIduwYyYzOJsYvJDq2tkk8DgIBaqkP8uWU9A83kkCUJNk7ZyMz1M8k0tACSqzQM/7kCeJHdTymJenLjPYblHULzxTxK5m9V2E1V+1kfDJlmOGs5Djh9FLPKjaMlN+h+y8g3Jz8ldkCioEBRWZ/EgXc7Ux8bhlebjrgWThV4PsX3xhl0Zik+tT9ipwfNw+ohXg8xZvwdjtn7I4crlQMOm3dQdOAYIwD5aeUkW11p4ewH0+hY64aqrxKM/mbzp4Tm/UZRYBEGrzH/eqLtVFbePzGF6vA7PHTfNo5VaqCZ17Uq7XFUyzNInLmDSYlFivMaGvTUetQQ/9xBLMFDBJtO58N+1YvIMYdY++5DhG8Q/WeVCr7segKjtoxv8rRiD5e/xxcVGoqrAZ9l/KoRv2sJ0EjwczBsN4mA77ibbxO+uT8/DJ+IprEzzRsHSJLEjd5nuNbrDD1S71GFD82hzzpVINscDdRdi+OP1jcJ6i2ug3vNSWiWJzDE3kjw60E4Rauaxl1bDM+ciiRo+yR8pipBmypzFX91OkdIUSidaxa4eCeQHx1NYcpr/FwDU5v5HipbDVOSCvhs7H5ejbhBh/r7BJfsphzP1yUQodfTwStYsRZlWqOI8Csl2rue0M5iKwWA7gYrjTaJfaWhSAZR3k7bkMd3gdBp0S9008M+x1DF8Q8YLzHFCGF3YEWyUh5ve40fHWru8HTARersTTH7LUcohk0jKPUrY3e384xSifOkwGjjYvez1Hu/S1u9WIsryWrunU9i4OF+mIzeqFzj3ru/MF37BGnXXsUjzKhI5vXUbmSmyci212YQOTKNJJfbar6+eNd3UzzTap/7mXXcSmbcLOKsA7gqHKtCY9Hg4agnOeA3aDbDZVs1M4c/R657e2Iu9sDUWiQbSRIUhtYR8do3LIioptSlXcIT3iqqDnUiZOUEjgxWvsuJca/y4YOT+BZ41v6kSycqmFj1HT/Tm+IDvYiYLSaN6y31XEw6y4B+t5jZZRZ35KbqU2/zXWaG1pKR6sXyea9RvUQkf+VLYXx4LpKQM10x+vhg1Ij+x11bKKkrnYkIW5CSzOl314pNArMDHg0VK1PRB/FQQRAXJmzgqhkyXPpOg/N9HWyA6uzxjJFdsBtZw3G7hdFGaKsW14twtzc4f2wGF+PuAxp5wGWipOZOw5K6lcdioskJVJKzfsp5GA+pgI79L5MbIkatZd49eWXVYNqorWQ8uQpJ+kSwO9xCeGp3W+ZrPPBt11FRvYTajT/05XR49huO+FXhqqHi5ahgkAFe8IbsxmbEBIeVhfkJ/Pqt078cO2C7eKDGyLaBB/E0NjIpIh3ZNoDmsdI/j2B7qZ7tkpX9fydPf1i5kkEaDT1sLRPs/hk9gP4ONdLGWIKfzlfYS7UvszP5LnOHxbLUE2dM8m8yXeK3O754JKfy45xdRBgMgFOa8l7ZFhaPmcrPN/1Z2UlFJ/lRIV6XJMhJHkCq5R0qLkTSv3nPPrUbP9UYOVVmp4/PdPAIBw4J15UW/SzjTu3FuzaKrfvPML8ZTFElteH+U29RShn7Z6goqpgNrP3XPqN4JYaHbvP82B1oLQFsVotJi2GBXVmjvUi9XxFdk+/8HYc5nYUHfe4w3gt8O51lowybvJs6zcoyvDD4IrcSs0mIzuGOLBKZJAn2pn5PlOZNajxqkNUqhf2ZW+2pjzuJOclMf7XYGTLZuI0rntBOB39Vn8B1qGQZNwm+DISz5SKpx7sxlx56+L4Ksm8vIqpN0xpc0lDImYQ5TLgWTUReMB7qy9CsptY7tiu/P7CdYbsicb/2IVWLn6C5n7q1vgPFaTrGnU3mapdKxXW52UJ5+l3nHhn8jijfb9UFsrIaJmnrUMtqEtTKFFY/3QWWesGXleCvUUpPD7t/FmfPPcP1rYOxj2jmu6e+ysvtign7e6t6tjlGAHwWfJt1f//vP7P7CgoSoe5+DD7RF8PhPtSWFGOqFtcaWYZb3WeQfOIcK5ZPot1akcwRTCntjXaWPvAb2Z5KoofNGMv03Um0/30K7006rLBbNcH8cWcq32k3cLYyoCkGVLuzrdobraaYQmVbaPyAPsA4YBu0mOg/87f9PuCsUdzH50hZNJwaSNbRLlwzq8WewQ4biZ5bif3wKo9W5DJSu6iFs/+/MVrKd/xPjv+Sff+N/6tGsKoJnNiTFotraNE5z5s+m4ZQXu5DcESqYJMtNawLhpphZyi/HUK9uYLmMhYSEk+e6cdnadEcb/MxA5Rt3piX582KqkqGWLvzrk5knOisRTzz4yPkmrV0eTSH1GY6AW3qQqi/e4uf8238GiDzu8t5K/xs3L7kT8X3Y7iwuILm6ZCzlTdZE7+Gx64GEF/rTlX9U0DTpv9JXXfGld0l3DyC/jallKB78GpKS5bQquMNvEKV4E7d3xUEzxRDa81QurjYC6TR/H5hNF66EvwSuvEPLKY3lzO39AoFO51vwYZYEXBNF8+WDl+hv2fEvcpTASydlzqyKXkTi/udxc22GRCTW0Pu6ejR6wSbJ5/hMPnA8n9tZn0Qzx9MxPPnyWyfsIXRo5Sb3676cXxbdZKfg8Cz4RyuTX7z3LdzY3AlAdXDhGAwUXWeTQ0RXC7zo8K7GrVLn6uhbrWE4WBhcAE+kgjeS0g0JNwl+8nVfNu+hN9U4j1Vevci6awfE3aMxbvvBcb0EgOTaFUD8dfacBGYlax8V1iqWOoP5TbI8LJCXQ4Ym4Azr/rd/DxkOq+XQlqDyVl9+rekW8rkTI542Xk/ci/WMDHgcqjdOXDXH8e1VjzjmI56jhjYNFScpNftPdRr69nc+gp9G8vwbuacDTD4Il+OocS/hBsdlA7+n41DiDhnwBvwsKUBokzosUY1Z174lj11kFh+gwdcZBY31PihlkoYa4QjjiYGZUl9EWvmfEjfm4l0v9iFarmB5smQSnzRJN7BY1c/vr7aS8GuDMz1ZviOxzgy4AiWeFcAUc+X+lIWGBqJO94R/znnoHmyz6cjXeQCFpV50/li5xYZp1NyEyiRrhNXsFDJHrY3ElXjnLfHE9MZqmpylg+0Ksfy0l7iGntQXtWJKHfxfZW0uoW9Ukerc2aKNQHAC//aNPZ6JrYqZu+IE2i6XsLhKVb5JNvO4325K9VFflzstA8HT9Ac0HWzpONe6WQrlxcptcVkSeJsj7PUaeFino6NOpF1WkwY1fkB1MTnkCw1MbLV9bk8P+sAW/dW0vG3KbR7dIfYF0ElU+pfSqsbrVBb1cidxefZ3ZBIuP0yL4zbTq2uUcEeHuxWg9USybo0I3LHfbgm+2obs3Fr0FJ0O4IKjbI/UoNk5ESbNEwhYJK6K+wSMlM2TqEgpAD5qaYL11NLdz28t3MwueeTmHBCDADmWD+h3bo5fB4ZgDbMl29cipi/d7NTGlzHSvNMhvsMENJWGq0XxysSGKAxs8VswdcFyJAliYNDDjLh/oNkFLzdLD3fNKZ6WDndCE8Ug0MnAvCSWsvpO0FkHOzK1eHiN59kn066OYhfHviFGk9lQLQm43uK/J/lr5HLWVcjyot6670JLorFrDHzzVPf0EsSWYwfV/bkxfNqFv86E0fXJHAhexcTyfHx31HlVUWypxIOliSJ1hpYEQi/VGzGNdkXJFeQrIMld3qzrO2If3/vENiZ+7cfw+HYwLBpXxJpTBRa1rez3uUllRc/HAJb7muorPNpHoR61Zwi7tod/OJzWDxMyXV8pPU4lr3/JulA8SCR9SnbGhjgBlkWpywOslKG66+6ZC43ZuGjgvZ6ZYV0or6OUhtMO7KCQ/c3VRZaVD04VjSDyMJK2hiWIJlfFq474nYQQzaNwCbb2HK/UgJ367itjN/mlLGOM4mSlWXaGNZGX2dCbATX+p+ik1qkctzIfJ3p/b7ivv1RFHpXouhkaAihb1Ud8fo6LGVebHYxG+uOsrGns0qs3Ha4yWCpZVlWKwo2DuOHeT9AqAh02LV+/JV0hSMNQAOKBJCv+TiPfuesQM4dekSxXhjDJzL7pTSOpgairlWC83e1i6lpu5v8kkaO1Vfimu5z+Mzk/Xk6Qgtu4tflLRj4tmB/2KuUCht0zE1nhdmLf2rpa/MzKW+4yBoO0njChCZdh07tySOPPkpwcDAHTR3hbi4RGXFsfONxOkwSr/vDoAwS673Y8EM/ChPFb1Zlq+OEuyefLX4R2EXqFC1TebXpH1Re4XRdBOtTHmQTMP1eS9XAEs96w8yIj6njPYRkn93GE3VD0aeZuRkq9lleUzyZtJMdSG69iv4RaUimT8GtyUdeHJzAuZvRXDaquWOtFI6160MZcTKU3l8/hkdkEZ4fnKFzs2RfvCOTHzuU8E7EF5xrwS1SN+utGX5XSbpxGMI5eGIAMjDbX5m8KtO2ZlI+bAoFjdxU5WbHyr6Bn3O+gw/R+Q8SX2FjNk1r3YnKg1zqnE27azZCC06he+lRoClxfLxsI5div+e5vqfoZLAjSWKlwscVxajUZRQnD2BgrLIPS7FqEFVdVhNnrERnVyYDa1WeBP/d83RwxTZo/q6B0+Ev0S7emxN9TtCumXx6R42OrH3tsHb0JDzuEk+1jaS5oOAx991Yht4gx/0eT97WIab3Yf7UbO6OLODPM1sYH+iyTqm0nCm7j6q4jQS6leKjEyspZRn21MOb6+5Dk6iUswSn9K8GZ1IPhx2akfUaPZLpkiVRGCsRZhQVPVIlBw3+ZfjVPoCfRkxCqmU15XHwZw3c1yGccubyDqubnqUmihS5nF59jnNPXcmLKlElJc0eSVnPU4zYOwKPg53ABZcqswYTvty5m/g2HAeiXe5bIio7ipi7MYp+PABbqm9z7Uh72hcFEfR4E9HJ6tGOASdDmbp9KPO0/UhbJL4NVQk889UzlLfPI6ndAqprHwKa/DaHu8zyZ52xk8EUoniXyXcDWfC2U+43+G1R9ldtqWBCVSDf7u7MvKQrvBOvTNRcNEeSYUojKH8RoW5K0qYdC8+XwJtZnbk1uilekiWJ7me7IzkkLnS5oPCfI9UW3BywMPlzJOtZQCTnJmUkEVQcxNPLF3HeMx34/F9bjuTF4QNT8Qkv4oWndvCn6knh2Dq8OXPO6RevsVhZ5uK6GyQHW1QPYLIauFozSZHsy6q5QWrsLHBArb1ZotHWyOd+nlzcnEBNjYFvI+4K4KbkMPGkj52/qv1Iz4xjkF75Tb8Rm8EgvyJev/YU8118hLvapRzZvB0VFrp6NCiOHe9h4JKlN3s1J1skuBri4vGbvpdPdQU8hlh1WFuTja0+hYJjz7JF00D7ZuqOsizTZuoBjnmWUX6hDX5a8T3nynGUbB5IpN6CY1xPRXXHcVMv3GvS0cZno2t/E1yaalgsGmb/NAtDgwHr+66qGu4czA8m4EY4AZE5LbVwZoq7L7X1iVy1+inelQS0T2tL9w3T6DxLlI0Mb4inbZoD76zB5ETmKHwEddeOHHVcpdfGkXSsTFX83dXxK/B45yrjij5kXLjIipQlCWOdkZBKH3aEgMVV2hSJcVvHk2vWsbmikoni0g3AoEOD6Ha+Gx+99JnCdlEzjLHpa8iLgQpdM+KZrOX9m35EynbkZ9dwNlwntANRyRIXu1zkgTp/FhXFcchdxG3c6w7Q69hRum5/mRL/EuRZzuvesn49LzQq44CWxjysvCFfpptHJ4VNkpzE4eWVsKtWx/pmijgSTtJwm4w23OiWgsUe8q/fJElWrn0zncCMaPJeW0onl8SWtWQxY7pepvTNURREuqNtrsSj8eC7w10Z/pvTH2w9U5wH0EQIr3Q/y42Qt5HlJoUtG27sctuMyXiZP6p0tPJsRfM6e6OtFt+/P7fYovmKOWis2c/iz9+n0uEg2GimHDP/JPv+TO/IwZNd8G5/kbvR2eCSzH7DDnXhN9gVBCdqXQhUMtjVWu4+eJ6RHleRrQMVzzqgxJ1pv77M+eH70E8Q1wyVZEcvgd0BZVZlEt2r7hzrguHVYjWB1S/TPCrumLeaHaGQcKUXftc/oLnQmqb6FOMj9lLUqROzunhwT7Ob5sm+YaFjudE4mPJOj6FrsJBdGUdzwtvxugR6f9sDf2D6AGV1dGZcBV9LDnBInK+NF8gvkiThX+JPpXcleVF6ZqmVfSMbNcvYc/dLnov5lYtapYLK4Xoj+ogCKnwqhH1pa113bv5RQ2TYXXKictC69Dc/bnkGZrnBzzXU/vW6ENdqdR7s3jcNm7mCja99T5JnEfP58l97adUBBnX+gUOaG+Q6bLRWzxPOvU0zmo9+MzLpdC+i3xFbhQBo9b54nHEiAMeCil3EtKFO7c/7NydS6ulGcn2zZk46Xz4uiCHOEsYPHOWBZhWt4BSjPYzTs+wPDMSJpAbh7OX3E87Kvw+ATRK0HyrKAo82Sew63JtjfY9xrk8my5obNZ743wqjkltoVf4Ea5W9S/8b/zPjPxnP/8b/VWNSqzQGTTuAQ3KwPkzJEn8mKwpduQ/bpm5gn5coIyQ7LITt749/ajtenbOMnGqRASxLMlsuR1GdEYdXZssynlprAKcb4WDax6AVE0wqWwOPv/ct0szNXAv5TbBNG/Y2T65YQXVWGOnnlQD69Io7LPe/xamep7AZxHL6gtpsOp2pQ84NJq77VYrq7gl2yaJi+voZdD0Tj49NKaMiqdSceuwWV2duIMeulAKSJIk9oRCkhgyLEuQusGRS5LWTNy++yR1dM7mJxgLqTjhlMQzu9SQEiE6jbOjKgYIBdJ5ykBUvbUZnF2UuJvm+SY8be5nrCRO8xHsCkOwyfuc6cjwtjDyrS3NrfSCbHPWozTq6nevWYqIlVPsiNyvbMtUD/KxKOcJRlg94cMclJodOpKCgieVllGpJNFqojSimLG4e4R4imPH0+U6seuw1pp/qRyu1srr0RnIdY6yeLPlkKpkFIktRpTbgm56I4VYMRZuG/puI+2dYQ6exu/Zl4rrdJdhbmZBAH0znLDXra6C9zgp28e83aNuyLm0RY+w92BxRCs1k/zz9awgakkNgQi5JOjHkl3VevDVqC1XDjuCX3ppwiwjkZeZ+gjVDpib+LpuSLyuc2Ye985n7+mraToMFgYqukNQ6PChNuMXAKYcIsCl1uW9YZH4vNDDL7MNAVbrC/mZpDLOKwOcOnFE3ORPe9elciYRFAzL55PX1REWK7/mkNJhHehxAF1CO43x7VC7VHa94ZRNd5cvQa+1JqhK/u38Cv7O9TmN/bD3lWqWcBA4J2+CT9O93tcUqCbPDnSA1pITfxFAnMufUdbf4YdFGLF+ksKbbKeRmgf2JRi1fNRZx/tcufPnV28gGMaH2mG82v/U4TpjVTDv1LsFmdYskxp7PrdH7WNymlEgXlq1VMtL2hZ+xedQQhErx7XiaT/DmLynYHu4JnZX9NYz1GfxWH8Hkt14j73BXRXACEjqTjl46Ox93HtXsVwmfkDJMA0+jnljDJfsUYR75u5fBC6sxBpfR82x3gm1ipdrR0o95+b1VvBZi4sgQpQzntCN9efZUFAcWfM7mNuUKuzlsHmee+c6ZOGhhvTBKJjqe6slLG3pwVuNC9TZXkuHQ46m30Lp7OsF1TX3mKtyH0WvHEGpyIsmP74fKTdwXAko3E3rzKAFnj2IqULJi1Qd6UPXOMxzyfZI/60RpPKPWSJhhA+vGnmdZBdhVIpCtlmFPgJrSQ93Zd1IJFgO85ANvt9DuCUCnD2Rr6iLsBfEMbvhcsC2oz2BZhgd9o0uZ20J1h744G9XFtkRdHMn+RrEZ1QCDlc7o0Fq0DNLKqFxkPLUNKmb+6kycdw3foDi3LMnU9LjE5C536OOmrOyTkPBRORvLpzUqq646qG7xZwh4hq2l0tK0BhqrTzOGH0m64INuw2jKy72F4/y0b/PFC/OpSwsjMCSfwppbgj3R833GPbIV3ZQ93HRT0iBfKPIjr/cZ9gzfQ16AOA/UDbns8DDyyA+P8MDmiYpesABXzGG8UApziqC+BaDkgdvtCbgDZXYJmvV7LVEN4aszP9BwxYT9VhWSrVI47r42maha30JlVzHG7AI62RpZPPYiJe8s42r/hzDEiExYWZLID8vnm9lr+TP6BrgknAOsOfhvH0ann2YSeEEJ9qLSkWqCDbWwuwXFhAJ8WFIYwCcVsLa6Wd2Uw8aIdtm4jz0I6jiibQInFLsmgLfLwSejFSsOjKNVlSgh6kNTEJ8aZ1X2P6ot5/ALC5n54zwW3lMm+4rloazunseaWR+SVX9FYW/Ut8Vc4NwzMhuV/svswnim/q3IZLY7QbLPPvmEkb0fQJV3i3cp5RdqeL+ilGPvvUdUeDifffoplQ5PfpywibT2pdiNQ0XQCmiocWPfZzMpPNqVC7ejBZus0fPxbe9//3+05FLtq/Mno8G5f96NvousVipBzPO+TogK5txphcoFUMNhYfhJD7x3Dea+CyK14MLFgyRvBI/S4/QO+hOTSfTPt59ow42N/TEseJ0uJ1wqKdV6blU750ZNThD51eK34U0Vg1QaVnjo+DJAKbNcV/o9rf+W3h/TQamHWY+K09FnMLvVs6TRW2E3qf3ZXOesTvqirinhIdfeIjvRwsPl4Yze4klhzgHhOEmVRp+HVxM05DhRsXeorhOrbRz1+5i3259dh9uz0FSn2Gs31/mhff1l/LZaqHcoq0vrKcd7SjqPPLqDII0yiSlj58fFHxD14UcU6pTf3oiOF/F5eh3jO2URYmoinlV4TqBk5EjKq925b9l8LIjzwM+/ksauV5i9+hFmbvje9bT0NlbQqTqYiFtF2EqUe61ks3AnvpjNEzcj6UUYQZLAp9yHmk5X0fS6pDgWYL5fAeYEqIsHXNbg9oEdGHvBziN/2vg+Q+x19WFMCcum/UqvPcfpkC4C5Hq1nj93P0/Guy/z8oXedJZE+S9ZlqlzryP2Tiwj9oxQvCtZkrnU6RInep9gqY/ynlUaierwPHpNPIK2o3IOypJEr8mHCXruZ6qlXcrja2Hypsn0OdmHMc36YcoqmXo3p7+uM+uIM4kkklaOu4TpbLQ61AHbl+7klImVPHM89pMVDe6SkhABopSbyiiuz14hg3jjQgoc6Umr472QVEqCyllzDLvr4LOkjbg3KHufvukRw+5QsLqfx9EMSHxd/xdJPdOQHTJvLXkLvU18pu+VO5j1WgpZnyagqV4v2CS7helG571sPdqac/Wi5Oolq4GV2YVU5gewrUbG4bJnpdn68UGrS+wfsp/rSfcURDzJYWF87DqmtfqBJQPHKNxFW905HveCQQZI0jeLTe1Wpoduo1VSHee6XOB8g4vjpfUj7nwQv5dqqRsZSmOI2JVdkiBeX4fVATsqA11gV6hVt+dS+4F0mf8r8ycpK/t6mCayJCKdCUYIlJQ+20W/RTybdJK7sXcVyUBN7W2+C7NS98BfLEs6KzyTOn087XLgQPtUbI+vx81HTPpISBzrd4y+c3bzcm9ldydZUvHDk9/yxdur6ROqbNnRp9GLyNxIAkoD8JKUUt/e5Z6M2z6Ox/96CF+9Uh3jSe9Svo3OZMXQQcgOMaEhW6q5f7szJj2bLSbkwu59TNylbfQ/1p9nrnRXfPN2w3D69HiMjAHH2Beq9Nf0kkShDc6WdcAqif6e0ZxP2KxtVM7cxBunoxVy3GF6I56TnOtfhUacvwA0FLIouQZ9TCP905VVV/+8v6kFsLysWcWWJPNTx4sseXMJK20DOFckshJae4WS7tOT9ieG8suHs7hTLwLsd+6tIDTEieV07HntX9n2qurqFmX+WhpBQLl7ORdq2ilsbpbvGHpyBaNfTeHyhdaCH6tT63hycDaG3mm8mxrKB7VNdUS1mmh+CL1FRrurzNFqiFSLstV+detJvlGKhMSQgXdRyaLvItslTFoTN7qd52tPJbQdWnGC57yhsw76B51Blpretyzb8ZQcyMD0YhPXdBOFYyuHnOC3W5t5seQrfmp1Ak/bRcFucu/BJ1ceZuHVAOZdTwK5ab8tKgvD52IH5qx5iAe2j0PVDNORJLib/i1uf37JW7Pe4UiqKPkuSbBoUF/mXDHQ8O4z+NQrK4WnRDl9wJFpHelUISY5bzc8xPgT76PL9GBN5UDFM9FayxhsgFsWKLeIrV9ygsaxuBTskuXfv/XP8KvYyQ9BkHCkL5tPtGFLjVhCEFL5Md8PiWHYyFMse3oHXi6tL2SH82Q+7TOJaSVWFALcrOxL0AfbSElJIdfX5buruc4fal8GHB3Ae6YgWumVe1a9qierL6zghd3nKdaMUdg3ZiTQ0Okqq+47g5u9yU+4XRNJ8N5BPLz6YX5c/xDuGrEFxem810ktfIyi6G40+ovEGEmWSev6KOkzq7nROofxxqOCvahsM90z7xGthssdLyt83HmtF6DTzMEYWoxnldIXDHQLoDp6KlvHb+ED7yqFvaShgNTYB3kq4Ve+aye2HSgPqiOR1hzDrpDrHI8z2dcbyMaZ8Pscp+7RF8CAv3/vDZx0QO+eIpHjRcl5rQk3E7BqlWuoyqbmjXde55u/xvGIe4HC/t/4nxn/Jfv+G/9XjUP1YRzY2hfJIWHWKZuJpvW4QulTa0htnYHFILInHVp/Vl4O59LFBAbdc8r0NR+SJPHDmK3Ue1fipWp50XnMt4qnvMC19waA1ZhAUG05G5JT6e8mLmqnasfwy53h9D3Rl9jbyqB8wMHOTP/jPjqNOklrHzFhkdB4hh5nnA1mn4q9hE0tVpoN8+lA4o1E2p9vpP3V1xXn1lNISJGGu7+MpbRYySRUSTLd9ZCgAZWk3Bhtja+wqP8Ysjq2ZXPxx//+bnKL40ZnJ/t1++RNHDCLG/pQ96k88v10UjeN5IlisKnE646u+ZpR60+y/qEU5u8V5dQAXvS003njRM58OYPLdWLiViWpyI3MxRpUTLLJHb1F2XNHLWm5VReIfBMu+U5V2Nt61FJX6kNoWgj5xU3g/TVG0sZeiM/831jy8P3INlfpj2nUeDTQ6sBAnsl0qe5U61hf2IujRe7IuaGUmcVgL8Y9kEjvaVzudo5NkzYpWYhqA7n+fdng+y1H6t5TXDMqLZdNMs+WwNwjVwQZWYBMq5qX62/wYUEYv2etBF0TO/mLa2dZvXMGC/eNYVGNMlgD2Nn/MPkfv01xsDhPburiObr0HTZO3IReUoJtdzVP82f5O8zusJo+7sqAf5zPg9wa72BFuwr2NSozD7IkMWftHNzeeZ6BdS2AR80lQ5sFqbIEpVcSOHU8mfsLIMcu9tGS/97Cls5exaJpvyiuuyimiq4v/UTA1bYMznaRwLA2cLA+mkGbx7O+zk6FyiXpYCrnoboAkho8GB5TiuxwDc0hTmNijBs8GLsJXb0LAKQP5vVSOPx3bNr82tzK1XS81JH4TAfu5XdRrjcSBf5leL3/MTf7iIkvWaXC7IBj+3vz6svf4WefKNjNXpOZXdifYZ99wpIXNimC0ONmP8Zm+/DiwJF0CBykuCeNrQb9xXaoLBq0bjEK+zD1Lt6ev5HIiae4VdlUe+3u2476vplcvXSGU5ZlpBYtEgCFeOkuq4Kg09wtPPfl58QbxKDeUddA2K3LZGZG/vvMmo+gXH96HuvHzWo95SjZfB+EnOOjzgX0SLqHh0PJ5hslH6XnzpGM2D1K8UyQtVzMHIml0pP740rxa7z+r6m1dwd6X04hsNiIFO2NWi0mgZJb72fqc39gqCnBWK6UkFsaUEpBaD73b52AX7GLM2yuYtmgTsw72YZ5P85TzF+tWot3WjtUG0ajvycC0f+M17J78EYZDNJLQn8McOrr+1QGYbQ3MD7uTcFWmytx62As4yyeLPIV76nOXEd8/ut0O1bEtI2PM8A6V7CrbBV4GCsBePTP6bjZxeAjxqOc3PG7UM25SWUHMYkDECffZkNuBwJeX0hytpJxqlfrWR/s7J1bg5LAkmZPYEkZzPcGbePOf3+XZBWdJ1+jvn0Z3c53Q1MnurhuQZ0pjO3JX1OP8vD9a7hjFtfJdVZvPku8xJOaQn7JVkqO/F7jycrhuzjV+5SzYUuz4eGVyM9HFmPLCichtSNGmxKMC9U2EYEUcxCoMJ4iRAX2hCcFSdZyax61jq+w1snsis3kWr0ImA2Ny2LKwl+Yv/ZtBg85K9gk7Ewywvzwen5/NIYEjcgajW04xfkI6KZr+bqu+Uzhpwrnc6yLVu7DOOx4ys4+hLJdmVzKtFh5q6aEBcUaTI5mILjWi1b2Wt5qfYFfymFJo1glL8sSo3aOosOamRQf64KhXkwon5M7kJKSgjmglPdXDVfstX8c6vOv1PXtCGVwfLVwB48tb80PR0fTUVIm+7JrM/npwV9Y+chKdvooqzCzrQYk4DUfCLLd5bNPPuH9117jqM3Obouz4f0QnP/d09jIUZuN9xcv5s7BU7hLEDEpg56T7iqe9yf5SdTmBXGx00Wu9xBBEkmt50O3QrY9+T13Uz7iHi6+pjGSv2pW8PabS1k7d62yNykw0z2QVlpYby5AUqDgGlI9OiMFlJHaJ1W0hVdzduReajZ25ZklD3LPpYfoO53OsHnan5RHF3BdK5LSDGodNSGefLJgGV8++yWSn+iHpqp68+w7D7DjmVdpc1XZ1+OeFMz12b/Tf2UK6YEt1H5IDvaO2Mt7C5dxzgUEhCb/It8GNc2klmWVjswKA/VaM70nH+LxhAvCcT72Mhb4QMGIQ3z94E+kuyTS28m3cL8Vg3ytFVvzPJQJZyRKAovxK6ohN9NV7wNmaULp4X+RToeXEeg/W2HvUHmEnLaNfOeo47qHkqj3QkAJA7aPZtTuEcQ0NCXW7GpPstTdKA+oxexVgw5xnVvoc5ptETbqQh0E+yiBpfcPneePtVOIvroDn/zlgs1qqmBRRDce+XYgn5bF4mkvVRz/XUEbvC52YFaYUhoPYKjeG4C11YBG7E8jNRbRpct4qjt25pbhV5cjJeyyncB22fTpsFFx3tNnh+K45477tuEMkMRvp5NBw2GPXiQPuESrwecw2sVk9Vz9SVJbN3J0xD5uJ4jEAoBu+ns8/s5KPu99iG+qlBUwEhL2NVPRbhxFrPE3hb1rhXN96zr0nDBP3CWZIYY52HRmOg+6QLdYkYgaEljEKx99Q/HUbRR1v0ClSvSB6016tn/8IC+/lcLMDCWBcJCbmYylS4l9+wx+fiIRz03niU/EI5j0jcTei0B2KAkuk3zfpH/B+3T3ykPjKFTYk9T+jDDCSCNomvWFumUPpa63s4rWv/VdJJd17iB67B611JT68NV5MSGhkjVoh1/G3OkqOzJCuZivrAg43u84qx74hQUlYJXE2FOSJHKicogafZzRocVK7pfGgwf+KuHdo5v58swPinMHWq7xbSAcDIffo5rVsqvdePX0XTaq+1DuW45XmUv/OEnGVONOm9M9yL8bTIMkvitJgpm3utAqG26EpdDgEnum1x0jM24vPzU8w/obSmWBPdJCNpkq2RwKHeQLCvvPaR8yLK0Dk8tD0MsiUG33iGfR8TiM6ybjW+4tPBOVSibxeiKTj/dnXYCMnyR+G60cGXzV9y6XDKV8cbobFps4T7prU/lS7cXbn01mzyklienPhFqGP7CHuW+sIljl4hfV5XKgu/PbCNNW46ZRYhS7a1/gXCO4t9AkTlbr8bt/D9UTD3DR837B1hjWQJdHjxA7ZzMjpx1E5eKzdbOmM6UqiLtBD9EuQum77/PO453lzzF6/WkassTr9qi/yTeBoPljLNZNIxTHekp2Xhm/m7fWpdC+z1WFHUmNSe1L4109S8IfUpi9HWWMcoPSst7Ey2KPLgkJ9xp3el56nHaVoqqGSoKT3yZjyghn67it1Py91v4zzur688aDawhfkcL90w//68Z6eXqi/LpbHkWATQt6cweFTW3fjKXQ+T0G1YrfrVpWE9++Crk6kMdWPiasgypJ5kzPMxi1VmIWv0YnS5Zw7Pe20XxSokHvX03/9i7YirWOF4eeo/iD91k/ZjvXvZVJh4ji3SzxdcYMK5N2oZKbYiUj2eS2v0KK7z/XKfoQcb5xxFkmEKUy0D7oAFqHSMg4W1fGpcP1xH31KEUnPkdSNWGTPWKWc2HUYqxhRYxo9Ma9uqlaXZKg44Vw2p8rx6oxIBnEWEiWIdg9lXsFvhSH3yNXnyjYJQkejs8iMCGH0lo9RW6tFPb4wlcYeamaCdovFM/EFDgN/W34q95KmSTuO7VBI/ip1sjS6PP8MMkPuVkfz/KgBxiRB1faXeV80jXKHWJCrrCkLR8/tYAjW/sy9B5UqEU/9VnfA/Rb8S7zJ/zO2CzxngBWXf+FD7OcSgUKAoukwr1VFokjThGkuoXcQm90gAq7jfSaCBpQ9oRPuBaJ+rcJ5FUZkKSmd22LsLHiyRWYWt+kzCYhq0SMWbaZST68gtbyIWaMFGrYqDVVk5/4Mnd9vmfZ8XiulYjkrnvqeHbu6InvxQ4cCgO9TYyljNbbxEVlU5cfyPenlN8VVdf469ExjBt6qcXMjcGSy6YQeMHXQrjntSaDpZoV4TdIbf8XcTjlOJvvLFvh39+zcdbXnwNu/P3f1/7+fRxOvbnUE2Kv5BxJi022EWOwstqVIGu30d/NhMqmJjcjinpbC32p/xv/I+O/ZN9/439tyLJMu3btkJsBDnlVXZFNTsDo7WhltdYnagurg7LoZLTho3KROFKp+eP+nVxOTiPhi0cVC7WERLVXNa9/9TkTH2uhYTLwqG81XwfCT4OmQ32+4niAd/1gZ5hY5fNpQRpfF50kZuEPvDNbWdnUNqwYY+JdXg+00UkrssRr1QFIGuciaWgwKNhnD3rdZMG3y0jseY2AeJFRBFBb+i4TimpIutIelauGC07QzOeOE8x7PkgJGIepy5niDr8GQx+5aSGvtdv4tu/rfP3U1/hkfUoPt6eF4wLq1hFmzCT8ZEdubBqEw6WnmZf65r9NgYt1ykTJG4Em3CLz0Zv0qF36wmvr75IRBXPfXMX4139sUdqjv+4RSrscwl1GwfICeMRDpsazmt13AqjxbEom/tPY2b5xAGk7lUHk0IDRVIwIpbHzFY4ZRfBcq9JyYudqfLeOYNeoXVgixCvzb7jMrxPn8/oTO/mqVzYquwgKlFRfx+o1kE57PqT+SgvgvcOOl+xsLNxSQ+UaazmFPpvYJ6dyrvwRaCZBesL2Il2P5ZBxsBsZZ0RZG529gZ2qUMbmJLC2VEuVJEoY1ao9OWix8fKfM9l0fojCwSpX9+P85YF8/MRCtm0V5QcAusuf8H7NNcrP6bluUiZakh3nCS5y8ga/tbdV2FcFevCJP0wwgj9N4FGNRzt+/qsv2p+mUrFxOHVlIjDUWkrj7as9GHSyP+3ytmNwaXj/SUUcD1gqqJj9JxdixW9WkiSwyVBvQG1VKwGBumwW4kHO4W48cKo/eoMfruOz4Ju84QtdU4djCnlQsOnco7j+ZxrD31rMvssDkZsl9MaVFjBxy0QK5v6GLuUxssuvCce+k9GRZ1c/xIu33dlcIyZmZbuZkYURDN42HH1BATaruM4l+3Sn842vWJ7jzmtlKNaTXKuZrKuB7Nncj7wqZULCK3w6ezInArBrzDZoFJ3SKnzYumEgF5bO4KPscf/+rpJV5H54k35bfkdjdmbrmmPJNyyJPLJmAKuPTWX2niqMgaKOcrr7S2S+spyq453ZslyZvFf3yeDEkyvxMunR1ym/jVfze3E8PYJv7P500itdm1uOGIIWrcSw4HviHS4VMWo3zoR+xY9PbWGI5h5Z/k1VdGpbMZE9i3F/bSNfz+pOetY7wqFfm5OZYzQxcPGPzL1fScbonPQWde1zCb0dRz97jovVQUOJTKhNwn/AOTQOFwkvlYofdjnZysGjv6WlYc1axkA3OBjhAJuYJZVsDXROPEx067s4XFj7xyIXc25ge64ue5inVrlUVTnsTJ54CGNYEYE5lzE2isxkU+AUDupeBODGpUTi/EUyxxSPdH6Yfoalset4r0EJWrnJJjZ/O4mGWjcqDX0U9kC3YObnBbOiygkQuI5GVSInG2GCO/g4mp5pjjaS0fe6c37MDp776mNmJIjzW+fjxr02I7gT1zLZJ90azbFrkbzx7hv0OeMqlOL0Afra3Mgy+DFSc16wyXp/MnVPkDfmKIbXvsamUiak3/C5gSMB1gaBR901hb2nwcqJCEiLwtkr8++hNf/J7B5vk90plfSwXKFKGGBx1tcMSBtIXI6DxwvEfmmS2oOYA5F88eYjrD2axPZykayhwkb+vl6MeTWFtX/OJkjj4jepdGyctJnXfklhQT9lH0MaCqiKg4Z4uBH1f060uJli8VYHC35fx/qXSLr7Aof/HMTtayJQ4WWvYhLe2H1rOTnuAYLbLnM5r3OUqSyca/BF6zK/bVrnGpEbraJat1txXf31zq4Wldfi0ElKkFsuep+jw24RHJcnVMf8M3xkG7308K4/qMuu8sqiRWwzmVrseQHOXhjbTCbSNh/kWH0gD5aXMjXpPcW+E+I1kC8Wv0+X2dvp7yOy2yUkzDozF4LyWUudYh8HOBIwnSHuFhZLA/HWKYGObwtX0iYb1FZv5Z4nq9gUHMBbT3/F195iEufZhDPsmHWSE32Ok5F4HdnF56r0qSK17TWudV5MVDuxf1GE2kpxh1ROtJJ4LqEcg0oEotWympsJTsB31YH2uI4aTTR/HGvDD8tmcjFHWemrqsuiOMaZeG1phJguUBELi33AR26WKHePZuHeZHx/nkp5TDYFXuIDyWlM4OF3p3PwYCfOmsAii2Dceak9h95diqHEnykbpyj8pvf8s5h3n5PhHV0pVi8BnDSFc/qxFKb/WNdiX8hKXQg/V0OpTUnAAphzpiPXDnTnzO5e3PBuUkQ4UrSaa+3c0Y9bz/IPfqC1PVU4btftDnzz4Uz6DjjNK/MeRTEyz+GTl8vIh7ajafWnaJNUFPuXoHNrwHiwL308leoADbE1dH14M+9SqTw38PGVjcg34dE8XzC4VBo3ZJHSahspUZeIcBdFg9+s1fLm5V78PuovlrZR7itVnQdzbLRE6iM/sckhEpW8ZTttuIn9cA86u9kVe22EW3sOLXuYJ36eTS+Dcj1o7dGHmqIYvNJboatWrgeyLHGi7zFKBpziZmMrhd0QVM/AZ39jZbQb+fq5//7uZi3k+wFv0XrxXTZ3CaNI87xw3HFVH6SbsCL5At+M3obFZV85XjuT0jQnKXBmqFIN59vgCtZGWpgVv4tK89ei0VpPqFc6iXMu8fgTW4lQK0Hyruq/GNrmLaJzGrimUwKzv9x5l5VVsCEE1PYm/2OjtR+LTHV8/vznPHP/WixqUb3iJy8V7z33OSueXMHlTrcFm17rzqWGK1yX2tD+ansMpeJaE1iTyqYQeHjtXGZu+JAQQ7Rgj1Zf5x1zAGPW38+MQg/Ft2UHdkdP4Y2Qibyh+1phLzT0ZOg9Z+uLjcXNVA0kmbs2E3XVNxm1exQvu1RhStgJDC9j58NruB67gfQqsRObJEFM0XOsCoLyWJBd1ot+lue50OkjTFduUnhbfF4AL+XtY2UVPFgIGcoucAwsyKbPhkm0//JxvF2S3ZI+kC2++dgHniIltky4Z521ikm3k9DsHMz8b8dQWSeCsp5SNYMNUPXTBKp2e2JziHFxgvou7dKS4Xx77uUolRo+aHDwV7dTPOeZQ4ELYRNZS502kS4v5VI+cCBISh8i19qG7rnQ9mYAqMQ12Ns9jH21F7hhPYrs0plouaOBcWGHmB2TSmu7MpW14nQv/lz6MJHlpehUyqTB3tsx1Jc4M0D5pp8EW0js/Ty35R1yggs4PPCwUn3IEMrMox9iMasZ7O4CcADo/fnu1kq8+6go0Cn3vHhbKjvDoK2xRpyfDjvFfm58dPhh2p3YTw+z6Beh9SajuA83/M1c7HIR2SUWsshu3LPCI8XgcZt/cZsJM2fys16pzNDS+BE13UsHEWJRrnN7VJ1Z89a7JH38K1unHRYIiLIs89Xta+wJiqZ+3D6mGpriCq2jljke0BCZR2mXVC46xHNXSZ5s73mCDx7azOTLXyAQZO1mJhnr6K6HQBX4qJSFAjdjn2VwHnxTBfMzRyA1iyuskic/l/mTYYFvAiHK5FKZnf07rby38ld+b0ZvSadCJ0pIV5rLaHfVud9o5V5C3DsmdD7rpgVhGtsPzWR3osKaBLUlCWb2XEpA6zJqJ7mhDhbxPUmCfmm9qboeS4FvGV7ecQr7O/su8OtwieXzjtLoQt711dxgdMLXPNt9HgmGzYpn4mfwJ9YahS0BJgW9JdgGRA9gSdwhymyQZa8V/A+LWwJ768Gjwpu47Gg0LiomaxoyONvtLNJdb7QbR2GVxT1Lp7KibdQTWBxIqxJlJfA+vwkMPZXHF98+y3iXOMszuD8P3X6aM2d7sX3b/ZgNSnJYiGU2G6b6c697EJl1hxT2Dn0uo4nM55mzEaBvWo8ete1jX7di3r9/HQtnrUJyUU26P7o3aksDFXfcMfusE2x1pgr2xK7n0fTONHzzIFmZIvHGJBv59uEfkRp1FK6ZiFoS56h36Q+8P/At9j/wK5ltXXECQOvN1tQJBH09l1UFyhpcta2OsUZYUg73n2zms9lMTPYuoaSomDdpkuscAawDfoYWf98P/AIM//v3V4EU4OR+kUzqp7Pyx9Pf0JAVTpVrUYfdxF+ReQQ+8Ss32sWSoxIxsv+XRkv5jv/Rv/+/8lf/G/+Nv4fZLLK1R1te/vd/G08rF3FJkhh3qg/jFr9J70Kx4iXUGMCi4m34OAxIDqWgiSzJXCGEvIutWdkCMxMg5eQvfFcFFnWZQgpO7TAx1A3218MLJaL0UkCOgdk/z6aw2Jt7eqXc3wOjT9O9ZzovfzGBfSUia6PUkMh7L31MSatbTLncFcmlOqPYMINvsh9i7qg/mReq7Jd23R7EtfOJBLXKZkK76BbuyvksrA4ZrVbJmtvY0BfVTehlgGR1U9AlO+y0Lg7m6RVP0/noBfzM4rFqb38GLUwF9xB6nRlDO3+R0fcjvUh5cwmr301BilM62o/kj+Pk5K3w4F+E+Yr3pZLUlOf78e3GvgxK9cWqV25umWZ/DlYGM63+PiaEu3YiBDsyq+es4Y+Jm6CZbryPXMJ9khu2Ex24uKM95TZxLowNeJzIDRXk1OjZGKycJw6jJ1Wty/kusYoeKhfGnnscv6ctxmqHYW5OGabmQ1+6h0VmZzBT6/MlitFYTHmCmV2h4JUwAqwiKBZsTuVgGPT0ucN1jehs9AjeS+643aR3CqEwaY5g8zf4oLmYjMfKmXS43MHZKb3ZkBwQkRNBdVoCV/J8FNUGK/OfJunEKgDS/ZVAeYk8iC3fT6TvjtEtglJ2NPRYsIbz834i303JJO9tvMgL3rA5FFo5mp6pSpbZN2wfNxJu0PtUb1QuBRq9dTVEX0+ky6nuDLbXK3pJGGp0hGUkss6jiLMeLoGM2o3BHncojcli5qbJhFuzRLt7LCstU2nwUdH6zmw8dErw/psqmbfL4YJxL5JWTETKskT4rTQcJi2Xr0UhNwti1QkV9HphHQOS7rHUH9QOcZ51terR5YbR4XIHfIpdHGVrKZ8Z3JA0Fgp7DkAfKDqkbuY0+qX+SJslL3PsfILiXcqSzH2/zeD6X0PYdkYJGBg0eu4lzyXwaR9yG7opeo9dtPdhl8lGfkg+dwzbmgyWGiJiM3GoJTru+5ipfjOFoMdT24WIfYOIPd8aXW2ZAmAZ7XmS9e1Kqfc2EmSaq7iuVXH1LDe6MWPNXEZlFyvsp0tiKPx4HisWPkNAwASF/ZqUxGeeeSzsmM8IYwuuj07HpN8G0XHTOOzqpnfp1nCRN4ZNQX0qjqULnqTSKlauXmm4y3VLIZu9CvlDrVz7R7vrCeqqQftlCncjxTloURl56Jwn+dejGd3tFioXEEWS4E6bKCYvXs3AVi0EAIDDbmNdNbxwLwhU4vnttnoee2ApdU+s49kG8dyb3GdxpMtsbiTcplov7rWSJNFGzuO3x1ey8LsPaRMmSoedLjjC9v5T2TLlBlcHPYvBZX3eYW7F7BwN3Y71QcpzqXgFbsmdyA3P5XLHbByxyupRSYJCq8yeMOhhUJJnxoU8ydEqX7xvwx1Nk6xwWUMJRR4/UtsI12uHEOgpymjV5jzB7/dpeCBPZu6ZvqhcKhm0tTIPrXGyrJNilTJap2JyGPrOQtYsepaKEmWiBV9fbhiWs+PGPiJ8lUkJC94AzPYEg0W5jr7pbyVGAyllQDPyjJ9cxEPx5TDiGG4hpYIEEUD77EE8/N1Iuu64D11la1yH1qzFLzec3JXTyLwjVotneQzgnWrn3GjQeGNwkS93t1XQTecESXbVtKAXqzbyY6VEvhVuVSp9tgBLBsfCIaP1bUa4Od+H2WwGu4377sQz4mAAtzuM4ka8SFDx1RuZ9vh+4h/xRuuIx6j1Fuxe1PLasoWEFIag8v0efzcRHPXuncntpe+T2jcKo6SUaG7QafBulUVhnY7zDmXVlEUykJUZwaMXe5DUoKz02RiSwbt+0CEb1h+sZJBG839M9P0zegAD1BreXBvPtq8mMf+H2dhdKsYf8oznfdUIHm8MoLdL5Ydka+BwqMTrR4cxYfMEJeu5LocUtyP0XZhCnxMGvPTKZF+93Y3P/CGvdR6ypCRhpUU+ia5Rh00SK/BPmkL5pAJujtqPdegJJBeJ8c987TzjpkavqkYti+ugrHVn77VZlJ1N5KHSCPQ2MdHiLVXjPvwEvz7zNTtHKSUQZVnmvj/vo9XNVgy5N1dhl9R6rpmdide9kTcVdqvKnXMmWOoPT7o1yVbKkkx+aD6HBxzmCWslq6qShONsZi0R1xPpc2AwQ44MEAg7AKWSDwcaIHLIGcb1V1ZveMk2IltnEfTxB9zrVKewf1DdBM6pXGTwAXKNyTxXAq/5QmKtkuhXWuic83tHXSM6rAmATOISc99fRKu/xvF+OZSpRMmpUw2R1FZ6sK+6ga9yROAS4FavBVx48Df6B1/hU7vYlkBSG5mkyedcq+vk3ojEpE1SHL8+4zDH108l4bdPFDYAi+1vqNah9BULVT4MuQeD3GBOsAjUxef5Yd05iKxiL/Y2iHu4yWpiaD8fOgx/i83hd8h1kd0zefbniR03uW3R83KRJ2adSPSr8ltFZkknfG7F8vIt5Z5Urh3Guk8eoPWamYRlK8FkSZL4o90l3oi9Qp5NefwxfWfG6HI57L6XE5VNfYoc+mC+O/8lF9Mep/b06wSpXHrYNkL3M92JzHaSOFyTCj0DZ3N6WD9Ovfopn4YqK6/fKA7huZ/78+Qr89hbJPp7pvJUnorsRlRABWdMM/D3VLagcJPy6GSwoJVQ9DACqFRf55caZ89guZkUbbSxLfNWPcn036cTVD4WD624FnnIgVg1VqIji4n2URIE5b17aH9Rhe6pn4lwkXozmEvpWeWHe2QxXXpeQS+L/nOEfIP2J3tx40Jr/jjdXiHj6bDbCPE7SpgaGrQ5CrtJHciBBlhdHENG8apmBzrI83sOr4Y0NN7VVMWJyVUJKxcTKlmcm8Toba0IqF0j2iXoq1eRrIWLJieZq/nIdUSwYulsOp/0RZdzHdchSzLldjh652ECPWYp7L28ncfUdk2lXgpRHAsgH+6FX2ascM9GeymLHjiEo1UWfqe60Ekv9uE0Br/HMylfoS/35ciAIwrffatpIn/t6UKJfwlX+yhJ2isq3dh4MYbMC4nUuVRhYgjil8LDvF7ahz+kehptSh/6+xonqdCkLWyRFFHh2MpDbn15vL3o+7hKmbp+O9khjZzrf5R+V+6QfFtUvgD4wmDk9hNrSPrwE2pdXAhv7wTsp9oRf60tv3YMUMSHdTYLid818N5Dr/PJxY7KiwZKVAs4U5LOK7XKxG6uuh0PFcGmhKtM8x3YzCJx9MR4ii74Et/1JlEuBPDC2kJ+e2Aflse/xBIPHSTRbnA0MO50b1789EUmbnka3d8xwyOPPsohi0Uh8+c6zgDHsdGJzrT2O6KwN0oGsiQruRZ3ihvbCv1Y7Q47tcdW0upqMQ+MP8VYbRPeZLBX8PilniTdjWXt0JvU6sR36Sk1Ei7LvPDRbEIv/Y6j+V6s8WavLZ9zH3/KMylv8G6hkgza6BbNRROcbIStxclIzchKFsmP53Li2FkHT3hBoEWUn7adfZIe4U9yJLkNpxOHKNaLiMaLdJ29nb4P7WD1jFhRlaMmnPOp/cjJHszvV1ag9WraL2UZSKrG/8lNfDHuKRI9RSK1JIFN5eC3h1YjJfWmt28/hb2ixJvOG4cx7uQaJraeKNiDNCd5suszDI9bRbxBWW0um4vp617H0QbINbns42efZKZjHm+Vw6A7ocLz+ocw72NX01+t5gFPF3lzb4mdY3ZiU9lITlPGBAdsX/HT5wt44rsnmNTopbDPCzlMeGoHKgr98K2fJNgivCLoWf4Re0cc5rep33KnRtmSKcdqxkOG9ChIQqlE1SusCEtOKD1SnyG+2fuodhjJqnBD8zfWqVaL64XRp4kkvHhPT8EmSWpWfDWZ8m2D2TxhM9WRon8s46DBr4Jc2cyWUj2NGpH0cMIexptbRjLgyAhmasV7Bihx6PhjkzvFmdG4VysxtHq3tmhuwQcVcD6oWWGGzp/oawMorIdgYD6iXGcBLf/uKuM5H6d8b2WFGBd8VdPIjK+eAeCq4WPBhqxlQUkwf8Rksn7KbHbVKHuT/r80XPMd/5Pjv2Tff+N/bdjtdjIzM7HbmwCHDmG/EtvT6QB8Z4lXHLP0SBda7x6GNSqP21qXkmBLNU9HTSVlxHm++/h7ZAVwKnFw0wB+/+x+Tl2PbvGa8urCeaIYut4KB4PoLKstxewLg3A1rKoSk333JV0g4v6dfGerZ+YtpeM2JzeMb2764X8zjmqz6BTKkkRiZiIBN+JJKAtAhejs5sqd2HMuGd+8kBYTKSXqKLKHHCVpwGVC9EpQS0JijBGeKNBzw/ix0i5J2AG/2/CLtcnJV9Xe4LVjTb2t4g1icunbshy6ll3D+9GNzHn7Z1RWkQleYr0LkoND4bAmzKWSBmjv2ILePYm3Rl0hxt0lQPBsxZx0H9wP9mX4sY9J8FUyyHbWd2F+VSE/tvsDY0ELsjnVGibdaU281oHK3vRMo6XLfO3mTJAc73Mcm0METg/nO8Fjj9pQYmyjBJvdYWfu0Afpd982CmsMmOtFcL1WH8rvB8PZ9ehbvPLs80LlHYDVozXHq52ORP94sVcJAGo3NuUOIV4Lq6KuK6qqNI4G2ungkwD4KvEhoWffmxG3mTn+NLfbBoCXt3henR8/VqzF5tHI6F2jiakUwcsujqvMWzWPivbXWDR+gyKgesxwmKQhZzk/8AjnEtcoLvuAyQl8e/pV0U9SOleZqs6MMmZxMuoOHmola/r74nw6XO3LzELIUjX1AdBZShmTfI+2j25k5iefER2dJRxX4f0ep1q/jFpnJfz0DsV139egY+KWicTcjVEAo24aN9rXvIwUVEpYYjY62eW6tF7cvPsOujqZEO5ACzKe62t0rKqGaDWo7C6ARd1dJk9fS+nsP3h55lpBLuJ3uQ0jvG/yyU1fBp4Mwa72Fg59uOMZtj65kmH7hzH6jujY6dxCWBdkpdeKpXzz7CBulH8u2k2pDByyF4Cep0WHFKC97SjDH9hDddxdSiOUbFbJXEYXjy3c2h/CquOrwQVk7+k9kdisdgRbdTwc1CR/Z24opEvUS7g99iv+SXewed8Q3kfvqFGk9X8S73o9yceW09AokjU2V7dhVpo34TMv0e9+kc0HcLbyTTZXJeE9cT9Focp10KqHekM91V4yNo2SHCAhkW2FUVfGURnwhGi0mWjj1wYvm0yEaRDd/JoCgOtWHY/ciCC1RE2pVaLaxUkPzpVZ8PECvBe9xtdFYsIXIDT1I2L/DOK9bHfSLK6yThLprc+zeu5q2pOHVSUGPrIM8R2/4TmPHKbeU+4rAF/2fJKfgmFlhYciMevQuJOQBa+XQZksVpINVlcy1SCT13MArWJfUzwrkwPuYOXp2gZ+qHYB0E05DAzLpDHxCiajr+K7Sw74iNAlrzNk/zDm5SqZ3h28+7Fuzib2TFrNltJ3W7yvQqvEhhqobqHXVUjjz2S1qqCDDuRmPSX15iLmHxzJ6M9m8/mOFeTZxH4NVwur+P6j+wj5ZRpx+weid4jz/0HjEXwjC+h93wH69RL3O4AKW9M3nOoiVWiuus4z7XwZEDAf/2Jf3FxYoQCX7O8y8Tcz9/1ZjV/c4wr7+puzuL8A3i8KBE2Tn5AujUJ/C6JXzOX5LZPRWUSQ0UfOxFDXSM/TbZh2Q6zAN6i0jAp5kBOzt3G+y3lsLlhbt6A+aDw+5+C0yWwP/0KxZ8XVHGRldgfKn04h7aCSHIDWm/UH7jBry10+Ov6Xwqx2mIjTQITWippLTX6ftYFBfsswWIs41HUKv7tIcdvdYpm1qZjD6VPpoVuP1iQCvolSNpp65zNe2HuaYg7O8MrlPas/0/ZfoXXuR4rrWtswiDfH/MUPj/zQol+VrunJ/vfmUfjnKGJrlADm5jo/vq+CK2a4euQksxpbJpC5jrlmM+eqnIle3W0LrhLOnYzf0y6rmG9ffZJ9h8T1211rJDA3AfXBPnRK7USii/QS5kp61TvXoHN7eyBJyhCvSPUZRhkume1KKVm7lb23u/PqB6/y1YFxgulcWltuLH2Mp/LjOB0BKrMY9MdsH4b/wtd5pPAN2hS7VGEaAlhzcAlHVo5n9ZJ51FaK61wndSm7w+DZsFjm1Ct9uVBzE/gWG3RaYZfcwhiUBysq4Wy9cu2v1LVheB68UQYFqoebjjNVMLddHgHhxUz/fTq+98RvVhfoYO07S/CIzmfgsX4EO8QKJFmSiL8Zj6PNTTz7X1DMo2dLEpj34xDW7erKEZtS3l8lqTB9kcJb61K4XrxWeV+ShFGGuZ4wxFOZuC2LKeT7579A9phFok8TOFouh6Dte56iWh07tvShWC0mn+pCrXzz5DfcLjHwU0am4rzLO91i0cDrmHVmcPGb/rnH7aN2cXbUw6jclNUgGpsXFkMIaklZCQbg7b2dLwLg+/AyhY9b51BzsAFmF8JfJf0F29MRlax6+Ec6pnYk6ZoyyThGVtG9zhvZppz35aZibkVupNYnjE4ZT9ExQCRjyDJkdOuMXWNmd04L7RBU4FXirAZplaCMZwCi1BCkAo1rL0yce+p9WyfwwtfPECw1EXcsKg++L/HDvu8kbY7/gKVI/K6meQ9n9K7RPLx2Dn/snUyARfSbwjQHGd9tM8f01RyqU0pih1blErp3MMG5EVgbxHdVo/JgyblIjrw7mr0HJgltAf4ZP9f3JjnbQQcdeLjIOwIUBczAS4bV1SA16xH6kGcjxjoNIYUhPPlVVyLcxHOvVsfx3f5H+LowkXd9XCqfHHa47CQJtc1oQ5xRJHfl+g9n2p4kGm6HMbTbH6hlEUS7IU3ke/dCqj2rONQmX9lOwd5IWhTci4GrcZWoFK/LeYADhwjsO2wca72XATV+WCo9+azB5ZuWVCwp8CAtoJDg8QepckkASRLMbPsBiRoYmqdMRh20D2RXfAY6rxoenKKsgtjkn8raIAioHkmgPkxhX+nfnx/fTeGLkUdx14qJW1XlVY4l1VE37ChPGkSyXVTIKN67soGctgu4Mvh5tH7iWhPkFoqsiscYn82Hg64r9g2DKobdo3cxfvZuXgpSxrXutR7M+WkOM36bQbBOmQhPz13OpLU5BNxaT51VmfhdF5yPIwEe8XRWTwrD1siwYye4+scwbt8Sfc3Odyt464NXSElJ4aXfH1bIM6qDqynocYHKGjeulihjIbPBwc/BWUxvqKFcEvEmSYKS6O4kzSxEp7UpnklRbROBOfJ8tOLcWOsYYCsg4XYc93xWKszBvhPYVNCFVVVwvrHZtyNJ7LIsI7WHP0/128ZMsxi31jSUMyBqPXF7BvPR/OeorhfL3TuZj9Pz0EA8qz3pmBWBj87pLwQHB/PBhx8yUiX/HxN+Z4AxKhWvDPOi1/xsymzKvdbd3siAm4kc2d6HHy9vFpJ9FqsFN/VmvCwWxufDa3VNZBeTJojfL0Vju9COAdfS6O8uVnw9Km/nyN9dBbXBJWLiVpKouJpP+J1q7DYVly1K33tAeE9mhDzr/OcOlfBdSxJIqCixOXGwTG9Rwn9f0ByeK88nSg3DjQ1ocW1FIhMx8Dy6Pue4VReMrGo6+ap0C0e/7Izv1V9wL1b2oh9XnsPiCxpeWTmcnfdEzFOWYbh3Kc9UhTH+ahpulYcFuySBl1SAT+F1ejV+C3d/Fuy55qEMz4PB9+CjcqUUvbb6NGsjSllZBWdqXAiydhNqyfT38xLXqaDcj2iMA7tPFeUhBdyyuHx3pPN7MHR64RdSvvycIJuoDJPsNoSwHKffO6WT2MceYJ53HV0XriKjczJlRpG4TkMBvR2fcd9vA+lfpUd2xV2AI+Z+dMmBVVVQLSvjx8cqg7my8Cs6jLuIqlnLj82lg7j00kJeXvMwH93pgEYWMeRJeV35+qmvceDAv9RbsEkqDRuTL3F25B7eHp1KnzAxZo6zZVASYMQ45yjFbd8nwUtUuDprhpP15XjXq5kevENxzXaHnbywPAqCC/jc9rXC/o9f1lcPYz2qmxtQ24Lwr/H7V6Y3mCa5Th20+HtzGc9/vvIiwO6SsDrT0LQPmSPEmBhZzS/VfuiO9mBJQQJxqhYwz/9HRkv5jv/J8V+y77/xf9XQGuyYa5wBSxvtMoU9MaYMv87pbOofhVe4CGqhNvLTqmf5ZdmDbKhB2HDBGVjv8XLKJSRdbzkI9dBU4isDDuWnYTTG8HHq6/xVB2pEZyIipITO3TOY/fNs+h1R8ruljGhqLCp+e3ovrcNF2TJ/810mb3Iy3vc+/iO4SBF+fed5+uwv4vHvH2fEbWUwJksSf7RK45nqaIrsQxV2rUrLhmBYGFBPoUnpbATIVQw2gAWwNS+31/pRNsSZuPGdepVriNdlaqyjfWo05mI9Wg+bAugIV5fyzOYprH32Zf7IdNkEAJ1URfwH/Xjoo0lctIvVjlqVFlOk00mNKbiNwaWSAcDhkMizwqLcOOxBSs38eQ0OInYM50dTGJ71Tfddrx3F0pKh5Mz/lvhhp5FdJAhe42dO9D5BB+8a3qgVGehmm5lk34MUXIogdcV09OanBHtRfSE3IpwbcWVAEN4uevlWz7Ys6X6YoykfsEZSAgpoPHn32uOMyYPHc1qDS/I2X9+XgDuwvgZq7SqaL+GP5iXxUgl81Oc+eod2U5y60TsYeaQvXaadIylUnIPVsjfefS5S6VENDiUDssahJ3zGTrY9eojPgpXNxu80nGfnS18SsfAHKluQFpMkiS7nu/Dpnw+yyKFM5OTaMqmt7siZmwsI8B397+9G0z2+DYQu3iYKvaqwacWeUDZNIDnmRMp71HOvlayQl5naysagqQcZs3MMr6SJ36W7Ws9L9khMRPBan3MU6joqrku2W+j25DVeffklaKGfyXT+or8B7saAvuQP0Wg30bqGj3WTAAEAAElEQVTjYQKSZIZcvUuAseld/vOtDPr1PmK3jUTl0qeiT/F53BtnMO3534nuJoJxEQFJpB6/xepLvVl2tBV1FjHBdKhRy+NtTpPw4homz9yvYCHWygEU9T9FlwVr6RitbI5tqjjJGOOHeGamc6paKfEVyUXiJqYx5eHtjPNumgs2rT9D7sHKsOs8M/0nFhMh/G3/hj/5ZV4U7cZtYOCEwxSUiIFJdlEY8e+/QPRlFcOixR4VAGmH4mhY2ZmP/O9yxleZQNqcsIkuny3jzwezqLcpqyQe9/XgkPsAvHPfxKZzAUocVtpcN6Cqcue5iV/gUdDE5q5VeZF+Kpzoq8lsmrQJWSsCBjNDm2Qt+h9WJldPFXanJDOMuNtxqE0uVZZ2K9PDahlyM4mH1i4iykspK/x+yEEGvvoWffa2kGgBttgz2VoH9XqlI61SabhlgWnuMMxFSmhkpifR777IK14/s6Sbcj+ct20Cz7+Zwq4bwdwyeQu2IFMGfwbKLFeX8Wbn1pRWiezhWK9kkPX4JGTTaqSyQi5K58bqugS2pw6iW/Vthb2ssZgVEXlU2CHb3lVht8reFFolWmkQEnZu5hxmdHc+hzbHV1KfL1bPXbMmUpoRQ0H7NPKfW4lVJSbSTe5qur75HU93OMbEIteAHsbmRpKSksIfD3vTL0z8NuwOBznXvZG/6UX7I6tQWZQSuX+UL2J7R18uB3yK5CI7CZCRP5Xfa8HNFC38Lks6NNXuOKo9uKc24VCLIOKUvvO5b+lvDH1wHyO7ioGgGjtfdkrhw173GPbwduKCxW8+0FHC9KAMtL67yfZfSZ1FBL3K3Dux4aBzf9aalJKqACZNEVejniIjfKHCVmLoTOhdkG7Cg8VNUoPIWq5ET+LGY3m8nFTCNKPL85Ls6CvTCT/8C7aNN6nIEWW0itSxNLZzro3GSqXiwV8nu/PjW4/SWOiOm0Yp5SPZZUYc78eMwmgCJWVFjLPvUzZlkbmc9uuosFfb9/NRBXjL0FBXhxL2ankEASZHPd8u/IjNc39SgIS78l5j388jAXCEixWHWp0P75w/Q1X7PDo98ytWXOaQdzu6qStY/fCP7JsUg9VlPwR4IfBXBhqcYLNrHy2QyC7yBsDDW5QElu0ShgYDlT9N4tWfB+Bw6bW2zKOEipBCLh/ryOmbItBRWFvI/m6jyWidwa24W9jdRLKcn99sln3/Oe7rWvN88nzFNWskGy+vWcLiVUtJaJeusP+znz5dAndKUxX2WM8ketzYw8GTxyjQLG46zlLBu/7QSa0i1Kqjl05MKoRbbpMVA8kP7CBo8bc0yuI9D5dOMGv9TK5t7c/sa2IVMTgTPF3Pd2XAkQFo61qQmnc/S5wG5hVBraz0BztSxqroUL4PeAJtl08V9hPtrnHAw4Pfkp/AMzvl39+LVbEsHrqdCtnKkAPKyr2nPM+xwUdH3xN9icqKVNjfLPFm1eG2vPX2m3S47iLra7OwpC6El7ZPQuv7KNfLlev7YNs7hMtXmTGm5cq+B2Pf5zlvZxITq7je/API/1wDqbVidUdP74W4W++j86XOvFGuJM5MXz2A66+9wJvvvMmzqlTBXliVSmCbR/AxX8e38TaSXZzflpqV9Ov+AR++8imbuiqT/zcrv0O34Hu0b35JuoeSVBlgCCIrBgpjoaubsgozPLeMqBut0ACxUpNP12hr4ErUHLwqnSBajLsY90bqTzL39dWoQ8pJP5WMVeUSe0qbmWDYyIjdIwjJV14XwPm+NZwfvRt1oEszZEMIK/7uoRCZtgtTC1VV1+oPMtANtoeCr0nZ23SeTw1bQ8FTFsVw/Bq3olL/vf44QHKpUpar8yk4Hs6OL2awZVdvwWa2mhg41SmB+Z22nIP1LtWlkkRmYiZbp25gfNZ4KuxiDBdtHEGx15t89uJnlAXUKHxgSdbxVhncsUClTVLYo+v3Yo2HG4lZPBDdv/mB/FCSyMUZG1jyxhIFCC7JKg4eW4d1/QQ2FRu4qVYSllfdHc28v5cZ1/YTalnNkYFHaHznE/bqlI2rqxxq4jWwa+DjRDQqZd0rJS9yrRBZ8hheWrEKX1a5cfVWKJrUNsh2WbhnWePJ9YIB9ArdzuqZsXjKYlWh1ppH8KAa+r6yGpWs7KPVy70znY0P4OtmQl2lTP7vD3GuMSGx+Qw2uiikWGroFLACgP6neyEp+pfDhUbn31sZBPUurUrqGiuJDHf6NB/6iTFaokcBcfF5qDRWxiXfxqgV/b0PAtI51r6KtxZ9wOq+ygqhVwOzSSmJYcj+IciV4hyrvr2WXx8P49ohP1Z+8IwioawxlzEv5QeCRx4jq6/Sx8Vax4xO2UTH57FY3UKS3hiFZ30n5hXDD1WiatLJ2PfZOnwB1V7VinchO8wsut0OY3orKk1JxAeIrQUKNTHwwo/0fnstb3z5KhJNPsL8l16i+6QE+kowSFKLcn56Pf1lFcP7D2fKfRYe9dnAObtS8WCofJlB62bQ5YQvp3JcK9Uk3nlsN+EdbjJx+Vfcn9AUZ+l1gfw6tJT84TeZWzGNeLuYAE0lkV9POUnfbw/JEP0mu42YLhXcGe3N5y+s4ZdgpQKFdHwqa4wr+DYQjvdYgdysTYqaer6Lusk8Tyi3A2qXHnSeHTj1d9uATTGFeNhFH6TQ2IW5mzvx8xuPMeNCrCBFm+9/mz3D9xCUV0vn/E8oKWiqhpQkiMgNYtZPczAc7k1amehFShJ8EpVOZLU32XciMKkjFfZXZ/QkJDaP2ntqqM0S7A2OMPbVw6EGSLMofeRSbSTziuB0I6gQ16LM+EUE37zBZHd4KaBSWC8shlj21INHZjzZVxI40ygSb/yrK5FWTaP8YhJ7bBYsLjKeodJOtH3dqR3lz0/5ypZKg7KDmOeRQ158JCqDiPlYqm8yof37JPe/xJutKnCzKVXDJEniognmFUORrKwsHHZwOu0/eYoHIv9EZWtK9o3rNo6cZDd0+SHI24YQ4VL97+zBXMLlj95GP+q0wnap8yWy+59ihBuEqcQkZJ3Khy/eeASf1aMZHnQWvYuMpyzLZCZmUlPrxrY9SmlTlamEmdMPU/jSd7hZlfekttcz1A1+CIJvwpt9l3YrTzieI9rQg1Uo4zcTsEbxa8tjNeBwwapsITsBkHy1dPTfrDhGllT02jEK+3cPECffU9j/G/8z479k33/j/6qxpLyU7AwngyouLFVhPxpykj2hvxNd/gYROnHBqzA38GfHS1yPu8X7OzqDJC5KkiRRGdye9s+tY9gQZcIBYNmgcZTFwZyg2woJRUnrydasKTziCcciReB0UdEgehabaex9h7GJSsA2NtefXqd6czbmJg+4ixUWntYmgCH9aixIIhAyU3uFxLHOvh/DypWOdBv7eb6725Gpv7cn6+5ZhT3cI5LHCzyxAL4WpURSX10GB8KdgLB/834mbqG84J1NSkoKzyVvZFddpXBcpKWAsTvGkn0oiBFnO5BrFp/XEO0Z/FPbIdW4czZLmQB9KiwctVUmOrUD5lIxgHV3mPk2cBadJmcwZupmJKtS876b7gRv+8Fy020uW5XA6a4Q5/Vs2N8Rs66JlWcwdOdm+jp6alQsMRhQu0hZSUiYtWaKM0IwlymTjNFn/Dh/OY7CqHZERopJSo+6a3w+4hqmz94jemBH3DWiE69WyfgXB+KzYQw1d5UJC4ASz71cNsOmai9Byg2aknAfVcDQ7ETQN1X7XKrwY+LiFI4++Tr3dotOY2V9CW7dwznQdgXfNmxB5S+yuHLUcZwr8iDmRC++Leuh6H+0vrErEy8G8OHnkzmeqpSJ66c5xyuRtTxfYyJDaqewBzoKGLd9HFlX47ljU/YnuOWYg3vEcrzruhHXTEooNmYSb7z6ESeffJv3Np4j0ecN4bjy+utI7t+wPvln3uuzHJtL0H/csZLdtkUExZegNrrMEYeV3tovCN3RmpEnf2VwkIt8UvUN3knqxIUvWvHagSRqLEoZrocDl/NVAHxSasThKd63w6MVQdkNFN6rYa7/ZwKLt1emxFvvvk5peB5X+pxStGdMqmog4nYE3Qx3+EgtJk8lCdSmOgJ+7UPDdzOpqxEBSJvGm6tmkFtlERGoTLzm69oxoxBmesA4d6WjXKYO4o8LTr35cb3PgVUEHXxsp3hu8DYW+d3lseymeSSpDdxLbUXyHxPxqvRCQgRRzHIAZ28P53RMKev7HqZSJTINbUYHJ3qf4OBNTx7eLFbUAqQGHuZM9zPUurfckPuu2ZtBBjjXfRV3cr5S2H0poUvgMUpjnuZM4U7RqHLjSZWVjP7HOVY3AJtPE8guSzJ5YXnkDzzOZ3E1BNjEZEm6ty+yvzMxNCpZyfh/r9dqLo/ay6TNk+hRIT5v2W5iZSB0t+vxa9QR6JLclyR450cn8K/1q2zxvj/Mj6XKBrdDQsAFrJNxEKaGrwPhQW8xmbLZXUdRpyuc39uF7cdE9qQkSUyPdq65T3z3BLpKMRAsNyTx+JE4ji15nJ1LZlDcICbVEmoXMP+LT3n1wdXMaBCDNQAPayrlf3bh6KYBVJQpA3O1rKbODo0OCNQomeC/VJbwVrmdlUEQbm2qJK53b8OTEWmce3QNupFHuGUXKxbt/rD09aV8N3kD33gUYXfpwbWxcTDjL/nTeuM45FtK5ryXI5o2tmhejbhEK434riX3WB4udn7Iid3SqWhQJtI7azN4P6iWdl53kKyVCrtdsrLAB7bHZ0CzpJsbtXT/2y846leIQyOCEcsqHHygv05qXAS3tC5JB1nDD2eWc+hYVx40+RApixX42qpjTPJ5m/6/lPKg+Vuq68V3qQscwq89M4h6YBujhigr3LBU83qXhWxP3sWs2A0Kc3NQRqhkkNU8ZVrOwer9VL78KomnxfWgvC6LMMsryLVqKvqfIlUrAnljEz8gLcKZAHxvb3sqGsWqkzytkyxU4VPBgohKxXX10N6i7bludE9rT7ykTJSH2u4S9OJqNj/2IzU6ZcVusLo1M+lCRRx4u0koV9KWRxHg713EuTZ1XG6r3FO+4z32DdvLlVFGOkbPVthtHt4c6tiflyxTUHu67FmShFkD2ZG5nOgwh/JGZSXOj1USH1T8+8/FIat4rvNVli1Yxk/eIuN6eKeLDF32BVabTGFGjKIC9HS7DJbPW8mXz37J7o4i2cJhKuW9pAyMj/3OgVl/ofUQQQeLLp68G6FkpcXw61cPKa652NgL9yw7+hwLC8qUPrBsqSLF1ymd3lKvYz9HKS9GH6GHVvStVcZonl/7E0X7RzPvuY30bSMCvg2OYJadbMU3pRoek0upVYk+W6nkR+isrfjdSWDidpdqcWCWZ2tsM7aisWoYcEv5rkfrMyhe/CL1392P2aWqG6CnhwfDpXwe6zATVMp1dHthIr+/+zC/LJ9BnVsToe4fP7F2eDr9XzjPBE9xjU2wRaK/nozPo78z/z6lFFv2jj5E/jKNwLBS+ge7EAQdVmZIRnRXk3iywoGpJlVxfFLkXxi96rjqUN4zwG/FHXi2GLocXgpGsTrKrTYdWzws8kHRK3OUpg0DLH3Qvv0Z9rFiAl9C4k7MXRrc6vBonUslom/ubclim4+OSf1TmffU52gs4lqmr/qKccXZDD/yNMNLXPwDwIebvNIxnwyvcvJtyj1Lp9Zyptb5TehUSvb2AB8Nscm3aZyxhUypyb9Qm+5xMcaG7qE/iXtlJadUIrnL4t+fXXUvs27WCd55bSk1GrFSYVVxKCvfeIwOp3rxlUmUawV4NKo3q+etYNsDpxluFJN1siRT7ldOXddUQvucJadEORei1WWEqODpYqh3U8p8rqt0I88K5yMQEqg3fNdQEeGM0Rsm9sShESuMLgT/7uwZnHyVoy7EAiSZmR2OcmD8NrrtGYbqphgH6c1ldIguofupnvinZWFC9Ak1Uh2JtVFcqOrMQU+NMtmn0rKkHOKyoH+Wj8LeqA5kS52zt59a0+yZSTIfFXbl6um2vPnOm8zLEZN5kgRepuFc7FbK7bjbiso9SYIN6kNoJUjxBZWLvYd8kS/8JD4p0vBmkXKOPVbRhceKIctqp8ymlATzop5P/OFgt28wulQh6/w6sLlhFo0qG110LteFg4iMfeR/3cDRzBDSa0QZQ1PFz7zT/z42vDeXXz65X7FvJPEpyyK+Z8e7j7CzBcJbsG81i9Yuo9tTd7G5ECawNfBiq0x6jz1OwtwtLVbZLynX0j4bhudBQ4Not6kMPDnxVz58O4V6d9H/3R8cxwczf+bmizc4G/i44j0fvNCejxc9SeztWFwrmAF65vnB13Pod7wfHrWVgu2e1cqmOgelPrco8ruE2cX3dqiM7A+6x+OzDjC/jbI/M1pfFuToKS7wY7S2hZ7A1b9yovNv9NUrK0Afil7BE7lRLPrqOfrkuQD4soHffh1CicNBev8n0LiL+0aBPonXDEV8aQ3ki7Orhd51AC+PcZC7HMJG+vO6dzgz3UL5IimJAa+9xsYt96gx7uSRbUNJOtZb2acQuCsHoxpyAkv3SwzuJBJBZElmRD6cr9RjMJvwb9ZbL9Ijhu639xFwcyhpJ5Kpt4mVlMfowqcJV7j7yC987qgS56Ctge62YQwf9SbJ+RsYUrpRcV0ED+GMowNVJm8KG/2QVc39UivTfEvob4BkLRjsoo+s+jt5t78ensoNxqQVMT5ZkpiwdQKhBaFIDpVwbd1Dsxg19hSNj61j2P27qKkT/evJVc5E2J9T/6Q6UJxDkgQL7iZzcX93rsRnsKfhmsL+R2k42b3PsL7nMbZoRGxFpXKmgn1lMEhK4pfdEMnWOnjJBzoaDwu2f4gfD3rAm0Hlwj3pY59h5TYH5wZN4cDEKJK0Itn+SGMf0s+2JfNsOz7e+THeXoMFe7zjQ+xny3HbXcw32m8U11Vg1RBQ5svkiMcw278QbBkWFeMt5Zy7Hs6atSMwq5Ukqbbq67zj57zvltaTpICumIKqmXWrFbnmpr1jaGAJw6fI3Bk4g4wOs5Akca0a715ISkoKHReksDhA2a91QwiMvtuKV555mSMXRQyjRB3OvnapaGU7k9zeUCih9W08xJ4oK/mRWVwzKH1Y2VLDA7KBWTfb0sv9F4XdYMljXxjkWuG5/GYxs62Blzv05aEXazkiWTmDs5LvXaAjYAOOwP8v+d6jQEiAizy0Tsvt5NY4ys0EZ7sox9ktnIm4hyrUiT2ctCqJu/+N/5nxX7Lvv/G/Olw1660FJajszt96mJSsUFmGhPp9zKyeS0iFKHtjsVs4n7wRtVXNyD3jCXQLUhw/t/VvTOpxE78QJQgC8FutE1BaEQhYlOXhng3tCS19CD+XSjT/uwYe+eER+iRnMXOIUo7njalpzFjwC7suTKHULB6b1PZtsvqOoNGgpv3NhUR4iMGvSnLQYdIhCpa+z+4+SoaYCjt5h52VXNnqbUq7LPNXtY4p7pBk262wX7HEMr8EfgyCcXLT8f84mIMODuK5L55D5cKoi9LdY9arawm8F8aYTYFUm6sE+1lrZwrmf8e+ydl0CVuh+Ltra6yYpu6g1t+dYUH3i8bGQob7zMeRfJfvPRMpsymD51aadJ72Br0EuTVKhtmNrI840/0MW1tdw6xvmguyDDjs5C97lA9WjEeSRfBoif89hpzqTaWfG6eCxc1LQsK91p2onChaJ1zFxyAy2FW2errq4L1AMykDRyMjBmRuFadZVhdGu7R2eLYAWmGt44N2P/CIJzSolPChm72Uqe4QqVYGAEgyDXpn0Py7S188u7WOhTl+jPhyIsaKXEXQk+zbjbjbzsDVeKwV7jrxeEmSGLNjDI3n2nM+VykHESyXcPP155nz8XxlHyFgpFcIPUaeglhv4roqpd6+D07nahTsHvg4vtVN8nk6tyCKtZ2RbQ567tqB5p7osPpULuSB62d49HunJJ5KJf7tRimYq5bR7Gi9nJvRLws2Gyr+sLyFPcFAbMNhtHaXPnAqA7c9yjAHF3G40aEIigDcVeXU2OH1Ih/wE6swZEli7LaxBH39ELZj4rcxLMiDVsl3GDv+JOfHXkdtF8GKXlUpxB+soc/lrYySXCQbGot5KWEyvhVGsiOzkV2AUw12Rhztz/bP5pGy96riXatkmZeXvM7bD6Sw/rSyV5VdH8SpACcL8PmAWkVF41FrX+Kz4GADVNiaGHuyJBNUFETHyx15YvdYhltqaU4C+yXvBgeXdCZ3c1tWVkOtSpSlfCz4BD89vo/AvDA8MpRVlHO7fszCQem8+uds7suOVdh/zP+QFXNS+HZOCnZZ+W3l+77F8Izu7GtzlrjKVaJRksgMqeCetoHlOUXc0TStwf7mK5wYkMOYDlkEp7dGaxcBzIPmRDb0O0zCuMP06tlC1QkStfF3GDRrF3HBLqxmtRuztrej8FRHZjy0FpVdCY4eibhDqx5pzBvdsjuusrthBxx2HQrJNVs992Ig0wzvFUcLttPBdjb0Oobthic/3nIJIpHY2vcolqfX0mHyQYb7pQn2Rm0Q23RVOHCQ0ToDs0YEpnLMFeSa7ahNWlRWZUXLPQIoyQukwb2WO+HKagSdSsfTJU4HNVytTMRUWYtJN8NrpVCrbSaHovXiRCNUxWXxyqxDxPqIbNYEcljmD7E2LfoGvSLbobJKDNs3jC4XuxBZruzR+aXXKB77YwZZSyNwqxQJQ7IkkxWTRUpKCjPG/EGBXas4foD+Jgt8YEOnn6jP36Kwv9L5VZb5QytDLTSTq4qRzrG/QwVp777HoUGH/u2Z8c84kB/BlaMdeatsH+80bhVsNiS23tFTs74bX734PCcLRP8jS5fEF3unYa9zw+NoD1SS+O21DUgmqfws9w/PoI+ncu3HZmJk2FG662FegLIPod5azv3u8KQX9NY75ev+8ft8yr2IuR1Hdqsb/OkmEn60tnIWjr2A/vnVLO+/n0p3cY5JEhhqijjT/QxH++3B4SLvNXvAWnbc35Yvnv8CK8o5OECfjl9IKbdvhxMUsFhhD7Pf5h1vmS4aCR1K0k8v/Uu87GVk961HGTFqEj/rW9jPWxirNTJ9vGL5cW4Kq9ODUblW12nt5A84ycYHFzDAN0UwmS0N/x97/xVmRdW1+8O/qhU755y7yTnnnCWDBFHBiIiiKBgBtVERRREUVEAQJGdEyTnn0KQmNJ1zzt0rr+9gqU2t2ft/fUfv3tf1Ok8enx5UrVlVM4x5j3vcg4YNZtDe9Jh2916nqadTBr+lhu6yxF/lsRR4etcLxm2olMizwPx6yi8CWDVWatxqwGkv7e0WwUC9hh/f/471U9YL+/zL5vm0tnSixK8Ek6vyW6hsBl71cviYOQ1rGOivJH/Z7XYetu/Fw6ZVnIkU/R61SsWLa19kzhfzKLGLQXSVtYLP/OBIGPT0FokeWlMaE5p/xfeDOxNrfu3fv0tqPVUlrXEv9OGTc99yr0YJEnYOmkrtT8/ie2gkblVuOK+vzWNWs7P8FFqNjgirSCCc6pXJR21y6Dv+OOExIrN+WtnzmMo8Cc4LrtdvosE0NjdcypIry7matEcwn/nT4a/fdy3iqKUuWBhOMksPjKVXSjhdW5/FTVLuKzfLf+fcuoGQGYqvp5NjD1R71pAWm8ydlzZxLtjJF1S7MMzrMWXtb3HxQDdqrGIg/PWwu/wV85Cf8sX1AOB4RQTLy6HWplPIvAHYNd78WQ1f+8M74Uq1hKhbCwjacptVKT5MLlPeU5Ik/No85K9533K48beYNGsU9hptJO+muXJ841OsPN2JBzXKcbbP1pUNu3vQ/owHr4SIWZpZchPG/dWS0E/ex/uB6AtKEgxJaoaUBHdtPQS7Ou41/ppwhZ8a3KVWVQdQSkioVXYKWt9jsj6bcyYlme5MZRGLah9TXBtCo9yvifNSZlHkSS7cbnkbn/530DcV99JUWzZXCtUsuB1MUo2SwKKuTuVsOLwx4w8mTv0LGdHv6qtL5LsAOFgNZhcxmPiouinrKqDW5KsAVtdk/sLS5yYT/1k837YaRUplmuK6TbVvEJx0AJVVRUWkMtAtSTKB19rQ7cgASmJUdI8borB3UVVwNAz8THqCrVpUTsovUbbfme72Gn8uGcnlEwPrrfP2ZHP2kf1DXubDG3OISoNx6UMVNgkVGrMGd+9KeoSUOfUbPh84lJE+GUw5PJQIY7ZgB4kJ7jDH10H+fLI1kR/RaP9g3lswlw4XRNKkLKm4Z4LOueUcN4t78TvSX8zygWK7GaOT366W1XQOe5VYdx0fuqoVz1xWcprpwx3Shde/nUBakfKcdckIb55ohEdKNFaVKFmZJw1i7RKHPLJ7pFJ6F+DX7FuM31HNp6eOUK1Wyteh9WVITkumdD7GpIjbyM4fA9DjyR0THKtSo9M7ywrLVFgk3nJT87y3EkCvwINEE5wrj+JMurIOIcClmjAqqlx4O70pk2pEGe6d+btRx+UyePJB3IOVAekazxb8sqE/fikx6MevFMqBSFpv7r33HvOfi6cwRczKRlZzKTiLjaP2MBARj6qwGMm31/KtPwx1V5K4h6REE7juGVyKfRnro8yWkVQaNo7ZReLII0xtNxMPq9J3D3QLJrioL3cz8vnW+DElRqWCxZaqDsw/2JX4Vm589fZQRjyTzdXERObOm0dAQDClbpcY/mdbBh0dRKyriLskSnF80vMo7V7Zy5zGyvVfQiLtQSQJkam88dVyWhbWKT1ozems7BlOYN9dLH13BzstSrWFIWGTCCv/gpIbTblwyWkMqXTMLwtga5aKV6qWMcD1c6FfjwNHML8mhp+SJvLJgTTF+LfgwZhtBk6cu8qdKGhaowymjEr6gGNhkGiCjaV+WDVKbNHDnIepUTL2uHRebXFXsd50d8vmEz9YHZFES30qRi+l//3CkEsM++YHNg18yIxA5flRkuBIWTB3m98lKzwLG8p5KcvwS3pTtkpVJMclU2NW7uWN9DuxNYTiOHjf77rwTmRJxkt21CmMcVHWHPYuucTLnjC7EHo/jqx3DW1ia8Ty2AQGOKly2IIl4uPj8SmPpM8hCHVTZiR+XxGDxaRFtsu4V4h+c4Tayrilb1OxeDSasuMKm13tzgUDXG18nwuN7mNTiySpJqpHzPOFv0Ih1C7KlPfvtBpdjg/WG94Yn8wkTlnLiNjPKXFvRJ57G0VAGGBuaF3G88KHSpxWQsJtz2Dc105AQ0O6eylJa52DelMROY3qUnfWbxtJnkW5PttkNeU1OpJb3eFEyx+FPts8Yhm9owN3V42ng3GtYNe5N2RWeiM+LIKDT9YAlXXsTvueSzWdsQ2EvhJEAeeAhkAIEAeM4P8c8Lv8tz0SiSFjnZ7LNIuWFQ7fd2++87eQcZHD8H33NPHx8dRI4pn6f1Nzjnf8T7b/gn3/tf9rTaVS0bJlS8UEiHEvwebqcKq2ZIhM1pDULXiduMCjMzGUl2UobCpLDQmRMOuFo7R4tgR/F7HGUcjlIr56cxZvpYmBQICvSqFhGvTNApxkMCw1Gfww2otb2qv8kn5GYWuoK8VDZWd9Vld+eiiyIx9VvsDBU+NJ2hhOUpVSkkGjcaXIuwvebgZ65SbgqVFOy83mtjTLtLPSbOSESSzM+lDflWLXKhq1fUiUd/2HaxNmemXCHd10webt8go/l+j4tBjS1HUMHLk2j02ubvQ+0xvfUl9aO4GuCerWjPBIJWvYMdS9rqC2K1lJafYYVnrlMqv3CUbGKAMtAO8XeZHQ6Srf/vAecV5OQUi3KFZf2EXC50O5+TCJIqMIgu+qfIE+WXAuAmJLjwl2/4pBVEXMQefyDS0D6w5OfqYdrBrscD5Uhb7CAuyrttDq6eMc7fs7G92cCyJLNGqUhWXBIt587ktMNUpg1RzQg5anw5n/XDxf/DQXs13J0NHaKwn8m/37fKsEoc/YzDzr4ZAredwwE2qVgISf5T47QuALP3jdu1CRdXW26XlGLP+G+Ph4MpopDw+y2o2fqi1YS7xpdv53cjOUGZ59PD2Q7Q7H5qiHCHx21qQTadZT7llOUjsRjNtrGUVtpRs6gwsxiFKx5W5PcSTwc+63noK6HiW4ZaXRLC6FDKMLVfY6B0uSIKdJP6oGdcG7E3h4KdlUWXIjslvfRWVRs+bMU4JDeubhq7Q5NI8ic3euyEqGWIWpirmeUzC5XMeWVImbxSlQ4xbB5ErQ5gURlh1WL3D6ck407TPBoM0S2cOWKqICHBkO2xuZMFjqDpP50dO41fZ1Dgdn8ms5qNTKtS6pkQ56Z9Pt0CUiLyvXuczyDLwaXKLjy38w5NU/8dIr30mM+RIvloXS8GEEDa5sFt5JkDUZjd4RhC60iPIxEhJqi+MjjTvUEZwkA5NN2SSbIVQFIeq6IKVsLOTY5Au4jjuIy4OGDKOMJxMpZI3M1Q5XiUuOY9LmSQLjrszuSpIZjk39jYThR+rpF3jZVbiYJUIkkdXcVjfm3/8udqqnBnCn9CIprpdYXQ6F6mil0W6nidZM/xP9afDInRJDHWhgl3RkWuDE6dYc3/gUeRrlQVNXpqL99fasLVHTJ1kE27Zi50WTL59WLcHqt1pplDWcdamkoNVd9pT3oMpmFK6/0f4GF5/ZyXrRBMCMgCLUEjx3dhWolAEmSaVlSSnML4Ekp1qxEhIlviUseWcJV7oomdyyJPNbSSjbQ1OJGHYGq9O2I8tQEFTA/Pj5bHtmmzA3FpTamL2jKbO+msPzZ5T7HcDN6lR+ffVX1k5ej10nziutSkdDDbzpDR6yKBscKeXwjAf8WRKKyrVujZaAzlc6ErV2ElH3tRw3tVFcF2HMZXR6A6b+9CbzlsymlacyS2Kk1pPGjxrTqPNd5gwTa48OCPyEsoc6rBY152qjFTbJXM4bXtDl77NrfevF+qpuvJAHc4qg2kXMWMxUZXKqBiIfhsETMoklcjPeK7bTZttYVp4fiafT+PcpcGf03tG8tfwtPFOVxAKzzcyJJjPJCfk72FCtzHC+X1vJr11Wc3z4PpaP3onNab5rs9fzebvefPL9erZeEWv+ovNH9xhiUmFwmhhBaqY1sSXEQaCa4Xmuzu+T4NcqL/qf6M/pzpd43EBJrrLr/JnpksPmoHSsaqsQCvGoOsRE7Ww6X+lMN79aYT3pZJ/KHx1n8MztdjQrF486W0z9ON3uGndamFHrmwv2+5qufPnCp7T58DPeUotkI395O65uiXxzezajnvuck2bz/1/s2NNWOw/+lpnKXjVBAE6XB1/noiqIr+e+TcJl5TpmMFfyfvAGIrZKNDv3Gy5WJ/JXVSo/lARyfckUfp71jliTDwixt2ecB8zzrR8Bn1PrxsLVr9O7XEkcu1u9gWnfXSa+bCS3w4LxcQLYp5Vt5Z2T0fxqDqKfSul/2F1CaXg2hPnPxTP/uXiMVUqfKycnnp+eDqb0pe840ku5TwN09AojuioEtVVNuEb0BbXuUTxzegUWO4S7LxfsuZoGNEqDciu42p74SnYb5rgIUrtPwp7XGE2lsja03ktPWsvhNEuK5sMfZtHeQ3lu0Kv1aKxejPviEK/PFQGaq8Yv+fjuWMYHJbLRS9yTwtwGU/T1V0z/djk+tnokjlQ6ziQf4N2K7Wgeiwx4k4sX19uncrLvSQX5y9NeRuWNZpQmRjFpSyGV3sp6mGpfD+526cmP/mn0eyDO2e+Gn2PYBxtYoSrntsmJOS/JFJd35VCD+5zscxKzTgTbsk0y7Y/3o1mimAkGjr0nUg0tPdLBOUPJI5ZxuXDFADVW5Vr02Lc9tlY16Iw63KrFs9DxMPjaD3CutQZYtIGsppLTvU5z0CuFCpty/BdIARzodQqAtKuiKK9J5UnLrU/jUuPGcF/RXmOpwqO2JaHFz9LCva9gN6hiqP2rE312jkHzBAtKcouiWTqcOdaeqaumoitWdvx+6nVeWOHDC3sbM8daQJxOCY6q9TYOjdvNomZqtmU5+RfAuFw3rn4wG8s3r5NcqgRdZbsF79wgVi+cTLf9TTBqxWz2c6YWTMqFAms9BENAbXdjXjFMPP8t0hMlKLyooHWNB1PWT6FNQjNwOgtF2C8QkR1Os/vNWOevDCJKkkTvy11wMbjw0qCd9I5Sykq6hw1i34UluBr1fDn8BlpZCXI/NOs5l+UYt5dKxwljAZuZ0+FwIwJm+NYKNfu0Kg1aiwM/cM4u/TnqJMOzY6gq8+Cc+iuFTZYh2OsW0R5lBLoZCVIr10FJgh+i7jHcDVqmi1knf0nPsOe+I3NiUJR4lp/oksmEv6dEfd8iz2MUnxVDu+wasqxOD1WbTwPLSkqyvDh892XFuaDSJnGvxV0yXtrM+W7nQa+8tkwVTNYNRxBa9eyfAjHhQHUNCeGPMbXOpG8DkYh31fQTHfbNp8O++RwpdCLayWqyivxos28IcY/j6vWbZnh58bUftEndQEunGKiEneePDcb1vXmC/+1RbGP4if7021dCvr0/BieVkjFdNvBoSAxlZzoSfFMc+ybfKH4d84CxjS5TpFGOUVmWKPcqJyAmm3kBNqHfkiThXu34WGkHREIEdhvdb7Rn3IbJGOpZy85YA+mYZaWtDjrolD7wGwUuFAXlk/P67/wR4OTzI5EZmUm0TSYg5Qo5ZUpS2VAXAzNWd+fF319k9B+jBcJbscWT4H2D2fTNZH67ahVq2zUI3Y5vbBZNOybipRFVVtq4jCQu9yNmFkLnDKWyjFqtpt2Ndoz6cxSbij25YKvzu2SNnn2bxlK+uhPlHonU2MsU13Y1/8HyRvtof6M90Wkxym7LGo5fbkWHb2aSfiAG91KRFF9mKON47i4KvA78+yz/Xq6SsNh01NbEsen2fKo9lJlqJa4NuPXP+csuC+uJrzWLBZ9tIP7ztcwKTVLce8vdLsx8bxohtxzP6jxvJxX4Mq3SgKnMA8mi/BayDFFFM5D7X+CtqAoCzWJw1bNSy5QNUxiX2BpPk5KwXEMoh6rhrhFuGETM07XiMsfCHFnbxyqUJWn8srbwUwCkWuB2ravimbTFR3il7SwiPO/TN2YjfiploNBPrqC3C6i7phLXLw+tWUnQSrHVZYetjxDPdwejcvGJcRAlzpudfHO7ncBiXwIKA8gNya0vIZdj5gHEF0M3F/CqR6L/tnk4ll6XyI9OR/3E97jnNpOF775P4/MfI/u+RplZiXXNzm7Dubcdflj1VSffRlKxLs2bYq9y9OPDCXDqdgO1gUUNj5A0YT+/D95KiUW5Z91068/nJ5vSe/cYVnsqSQsAkqTGO9fhd2wMEp/J1SWKs4mLuGF02rNUWr6syCH5cgKDjgxCi8QZ4BCQBrwLpADPAr2AwaCU7/37788CadgZNXay4nctpSepyXDndsvb7O7iFFiVVeysuc336a0IRUZXD5nof0urL97xP9n+C/b91/6vNbvdTkVFBfYnDgHdQlLoM9hxGD8UJjJhA4scdZ5WTV3FNk9x+KYf6cKZnyYwuNUGwQawo0CPucyTzhcb1WsHeGyGU7UIwCmyRJVcRYn+LjYnsO21Do94Z95m3O4buZUgFlc9kvQWf3pa2TdsHweMykwd2VpBtPct/PvYiRucD04sdEmSeP2X13n7h7dxqxRZMBpZzYY3VrL12S2k2ES2n9Vu4VZcGdO8oFQt2hu5dmbQDQM3j9jxC1zw79+1diODJMfvdR15hgB3JQOsRnIl0QSd+l9j0aTzqO1KZpxKlvEu9SasRIW/qr6MFxmvfQNYs3MqNSolm9WmcuEb3acAtL/eXmDYABjtbhT9jTfVd0CYoJ9N/13HiXdbgY+pbtMutNTwUF1IbocbHBl0RGASzsxvyDMRt+keWUJflBuyhMTeCCtNEtrwyVvv8FeSchyoJBXhWY5DQfltA2UmpTNsDRvOW37J+I88zkMn7WsANF5ok2BWISSbNCAr/42v7wQ+SniHIBV8FpgFtrp3frY6gDO1jjoovd2cDpmyll/jEqkecQS/kCLKa08p7FrjY0JicpAbeVHi3UQIEPXQpvDx/LX0+PxnXvQTmfWSJJHZ7wxPvXCAgd4iiG1V+5PzOIQJHh8QWv6tYN9WGc57RdChIIejxroAVFHGXv54RmbaMyOYOTOechdlv5NU7Xk/+g6akALKswOFQ+iUoHOEhRYz4E4rQgpPKGz/1Mv4a9h+Mmb9wYUqp8w+wKy24vrydj4ckkA9uCmP2EcPPRwIBXWJMjggGfNZPvkM2mXxHGizRSExWmmP4XZuF9TfvsfJZW+j1ysBt9lux9jVdhcxTVNoEHBOYbNo/WhVUcGlNgmMji1G57QW1WrCKX91K3K7u7g3EmvH9PAK4dNVi9DO9KR3K7Euqmv1bb7vfx+b3kCif7nAvNdJBsLVcDYcFkfWyV1Jsgt/mawktU3gynvLWKP3VQT7gtU59HlxPz4NMmlbGEywRZkVe7RwKKsWTuGjqlD2tRXrnX1+theL/uzI/i7tKainrlMH/x/J+2IjX8z7ol7pjvyyv2ihhXcLId2lvdJoM7PoZiNkm0yL9BgCyurIHFUuLXjpWGNIC+dk/4f4uykBhQH2ZKIyouhxvgcjzoj1B1IutiBp5QS0BmO97MisyBwOt7rJ1vLLPKwQs9juB+iJ2T2MqiNfihcDQ7zzmeIBaQEiyI1Kx6wi2FEpOnvflRexet8E3tW48n2UMmCsU+uYeukorXZ+Qqc0mUO1Ssns4Mqr5MTAYn9408uRufNkk5Dod9JBHBnULE3sliwTFVnAV9HVxCGCWmpZRZgaphfAPWt7wR4l5TLfD9pam6J/AmD0rLzJ1KRWNHzYmJe/nIO9VBlEv1LTls3fPo+x1JO4Lpm4OwHVHb3u8/Jna8h96jh7a0UW4tqMk9wdkc7vU34nQa/MhpRNxXwlezHko3jeWz0VlV1kiudbvVlfCQtLweYugv/fpAyibzY0zFHWWdHq+3Dm6h4sd5pgfBhOiKuSyDS7zXU0no5g2TOeyn1aslm42ryaVnN+5eDHewlppJRhUckyFV4VnO1wjUT/QiTJyf+wVnNzdyPcb91F/aAeOSpJwmSHNAukmkUmh9a7J2/ee5pX8mF5Zd86vw/wb5qG7tVteITnE6BSrmOyxo2dVdDlSid+3vgKjcuVc+NK0RmCu19m5Bs7+S1arG9ULcexbckzNNk9kmdLxcy81r4fUG6dj8n/uXrnZYugEf/+t8mrnWDvkx1JXE4RlS73CAoK5utvvmGETvf/zY7V6Wj39FAu9b3GkYFHuNhVrHd2vTKAFR9Px5jmy19lVxU2SdbyRrpjvHsHlCIZndZ3l2A2aB3j7ny38/Vmi30ZkENDDYw/V0/gFmh9rSXGrGACU5UHd1NGPjG3/0R7pg1ybhCeOqWM5+Xbo0m53JjsBdOJuORce0ZFxRPBQZNFObfsspbcAm9+qIxgZ5AoYaQ33OaTJd/w8ZoFPOORJNhRaXlk8CAkFV7KEeuPZhqKSDKDdwr8avq4zlCTyfqRvnTymE307b9QFyulefW2h7wx+VPaj7xLpwFX8JKVQV8vw0m6X5rHTxv9GZgvAjTZttZ4bW3Es5vfp6fr24J9ZGAborTeVKks2OsB77deW8ajjCN8UQwZbmKdlsndv2DiC/uZ2zGDoPI65n6J+yiWv3GTDFU1rY8u5H6ZkswhyetpcekskzdOptt5UWK/2i5RmhrK2D+m0al0mNJotzM+fRPehHOz7U1UGhG8aFgSgEvH20T2ErMJAOZEHiA9BrZ1XAZGJXClklRYgc6ZcDlTCVR/4r+Fpf1XMGnrJD65pJSwlZCYs68tN6bFM/Tx10TbtijssiRj0VhocbcFXS51EXwEtSyRFpvCntF7+DJcHGMSkNPlKi0mnCS8pShlX22uYk+71VzvsJkQtQjGZVc8pMPVOFrfbk1by+En+iUh2SRG7BtBWE4YLVTKbLAWqjuEd7yLW5o3qRv0qK3Ke0/U76M8Uo27ywFu11PnXn5i54/xdfrWXs0ZmqpF8zCO3qd710sOSLeGYAbuRUFwzS3BPiswicxo8NGWKtbRZ+UD7PV0JTY1lpF/jkJnV0r7vlJVx16qeKSc8xISr45z+NOX0wZSoXIiDHk2ptpvECW+payotmCXlft8ptycb1qfZcszW7jb9IK4vtvtNNFAWz18HlAjgPf6mpt83HA3H/nAQA/leaeDWz6RPRK40KuUfKvyfCbL0OhBAJtuxPBXw0TuORHDZBmKTV6cqwV72nyhX219J3KlRyuGLF7MmB7iu57hkc0CP5jjA2GIAQ3vmF/4qtCxLjtnjNtqc2nv9wMufXO4Io1V/LZVH8wzeZDW5BE9nzmKh7vy3CtLEqf6nGLI5yv53CdCqDVoxcThcVtZ8MEa2nmvE/oVplpa10fnLDa7nchSHV0ud2HyvpH1jsFBbkV86AvnB74gfkubkXYPHfNxuVm5rzxjtNDhTE/CsyKYnu8vXNte/yUz2mxk99hdbIoVgWgX9z8J1CRTXeyD1knSz6U6hReGXiO9cRIfnIsT1hOVpRqfPg5vwOinJJUBYCrjk2ZFeDVNJlYjkon+2bvDUmFZ+XMK2+W4TJZP/4VVwalcqFGSMWSbibJYaHy7JUc2DiHBoLSrzbm06nkLz8hcxo85h1pWzj2N3sLxpxznOrtBq3hnkgR9/O/y1her+WDUdu5WKbPUADpxn8kr9bw6Lx77DeU6abfb+X7CRYL6XGHTLQs/Vtad9VONlSxtepaHXS6zoDqUBrJy3rkaE7DdcozL54dcE75lrVctqdGpHBl4hHgvkcASlL2DNYEw1quasU0XKee83UZg1DdUhi5mdfIEajwHKq69Fj2T2UUwzQtuN3mIm1kZ3GoYM53X/nqf0bvbMPDGAMW9K60uSLV6nt45nne2PYvKoiS1pVW6EbFvEAc+nMnvj5V+piTB5m5zeOZGZx6vnIjeqjxTyDLMbOxYJ5rdbENUudKfLLB146kcaJkBh2rE84YkayixQp4V3FD6TeWN32doDrhK0DF3geJ920vOMLrJEo6Gvkmf/Uc4VauUm+9jz2JteTS9W9zl5ee/w2BQEsBlJAr9C3HtcY00d1FedH1pIElv/0p8fDwV7kpCsmtVIjcD9TR72Jhv7L641lOLsBpPvigB/WMo1olZ+ssfbcVDZedMj2x0trJ//7721knUJVY8SvW0Wx9GrV25Vl2qCeCCrpJ9w/ZRGaGc0156Lyo6z6N6dC8+7zuMWNVWhV1jycF8Lo+4h7FkRWQJ41cly6RFp6ENLSA5W5QmlWwmsoYeY89rq9njKRLeJAns2NgeDGdjlf5eqeoBrS604gRHOWy3848XUA40Ab4GNgN7cAT2lgJvAD8Avf/++0bAUyNhlJVnwOlejr7KNrne84YFI5OXN+G1+E/prq3fH/zf0OqLd/xPtv+Cff+1/2vNZrORkpKCzVbn6BysacSZPX0AMGvFTeCPnre4+ukickNykdROuvNaT76otPBAZWTq2dUY68mS2NvnFNVNkmijF2tCAXznq2dZAASXjhGNuiDaZECyGTy9RytMDww/8OO1eYRnumGoEjP7VA920Ou4K9M7pNHRTQlaZaf9xkeNhpF0L4fF2X2pdNJ+bybn42/S41vqy+cPRAb66KA2TL40kajFr1NTKAKUKlkm0wJFVrDZRIc2xv47vwxrQpzPDcUGpPZqxkeJ+Wx6KZMRzc9xwqosoq4zm/ls/mdc/XUsPTPBqlL+9hD1Cd754R0OzZ/C9HNikPHHgHKanOlB1p4wThQpsxEkO6T73UYOy6NbSAUaowikeEhl2HEwJLMDxO9pd3UEb4xZFnjCSSrUdqNfgYVer/7JtXG3kIWC4BIxqTGEfj+N4TlKaTFJkngzW83pPHeqdAaqnQprq02lhHW9xeGXfmfVa6uEg4ufqx/hhevZJPUkofQtoc9IEmZgSRk8k9wPdEpmnE0XzrHcXkwvgDGZLUBdB7jdzU7n7ooltFzwLsMyxIAxwMZOF9g6czkFfkqWYoFHPxJculJekkxORw1VJqVDetw+kT4ZMh1LA2lpFrOqguQiCp86wawgH0rcJwv228WnaXxpNXfWh1NSLEoQPHlQerKORYnVztrDbfhpQ29apcPZGmXw9B+W3NxpPzNr0u+CjOcVT1d8Ju1HnRNM8zynw5jdTrotkgnfvcOdBxpyrE5zw1TKp1Y3XM91orrMo155mTGuQbzlDb31KmSrEmTHJYQxOQ65y1gNigPyhZRvURe+hF+RK75V/fBzdT6cSKT5lpA+KYk7zUVJVYCzu/rw0RvvUlyuBFbL9c14vQCGvbmTF19ZLDiVBv+xvPZXEh93msXTvvHCM8myDktYPkULv0YbUSgwsrtKx8mMgYsGiVsldZnCss6HKY9dOZDqzwlVNXct3opgn789n/d8oPnrOxi0eAnOpIan3N4hJq0RVx4HsyJfzDjUGtQ0fNSIMk6Sbrkj2NvoFvFd9GPmBFrxtIqyOO25yYEwiNbUQw6QVJzNdDit1lo9Lua69UaSJDwqPXCvcaOP62i6+CtB1xGNw/6tE9W8RsxyWBqbT0ZsMl3PnkbKdgoFWKqpbWDm1WvdGLZ/WL1Bh8sn21FxtiNRyeIBAGBErp2ZhTA++k+wi6ABgLEBrApV7jt5hc1JT2hM45wwBtQDNLctW0bLmypeWfMKmgplxzQ6b+6mhVE5I56IeR+isSvBoY7qLEwz1lH4cgopgcraXwARtkd8k9iJrA/fxydTXE9kWSLeF8a4QZFNDPwm0IJ3C2FO1HU8DXWB2fDAdmib1I1XV4vy3gYPPXtH7uWvsUVs8vwdtMr19bF+Dp9kfMziXB2bc8TsjWXmhfzVYS3mRqm4OAXZJZcIXrztCMK5Z4WhtZYL12uw4v73q1SpxHcuIdFJB4ODr8IT5AB3jSfBpSMBeGTvjV2v7JvOoxZzhWMviG6tBAElScZqh26udi61uMkzXsp5F115ipoYiam409SmFfbDC/o2rA1yZFYkt6snQGSz0FUPXfUQVE+tKrsukMPFDfmtAm6aY+v8Pjv0r9axzDWfz84M4s1CZyBDov219oT+MZSCxxGYbUpg9bLZi37mHP78eRzxq54S9trPisvJL3IExs5FiMHAEG0Q3Y5dYGLet5gK40W7SyCPhrzK7YEzqfEfIdglZDrrYXaPp8nP28i7s2fz8Vdf0UulopfaiR2r19NLpeLjr76i6+B2dNbD06Mu8PKwq8Kc/70yllLvUi52uUhKe6d1TFKxxa2Qn+ctYP3MZewrd5obOj8KyhOJj4/n6KCj9QKnQWoTaiC1pv6awaujLeT3P8OeQUr1itrQ85SPqaYiAdb88i441Rc90rEVW6bc5GrnSxzzU5LSZLuVsMgCVn8Zz0/vXiRJo5QWy9F25JvtPdjyxSsU7ust9Mns0YH3E59naY2ZkzWilCA2K97aMgw2KK9HPleNmRCVA7hSPwlUqz34Jd2dVFsOw17aR8uY9xXX3c07QfOQQyS0KmK1bicGnTKtpKB4C4G+ubilRVCQLPZreckwUuMu4V1iIPKhWBagS2UPurpV0z4TqtTi9RHllzgR7vAh8tzEAFO7mBXU/jiKgM1j8K6pC1S28e1Ox5R9VEcZKel4naIK5b7TUK4j9cU0EbMFfjw3k8c/vUSrhBCG1DoFFWxm3o/tz5BtPVhV2Ah3q+ibf1kZRu3VVvTW1s9ilv6uk/1BEYoMZgCNKZ8fA2CwKyA5rycSRp2RhoPuM6CjUtpOkiQeZzr25rJ7blRblyrsXsYkHkbCiOeOMOaZ47g6Bc2mSAcob2Ihp/l9Mp6Q6/qnNeQOK9/az9k22eRKwwS7JEm010OwGny5Kti1VQ4CR4Out7Hp6oBV2VbFNG87pth0eo05RcMQJ8UaVzvd3thJ6nPHyWwfSbFKCTKmGqJZ+MZsJi2aTe88sYzDZM8yXJfH89TSPxke4ESokGQyIzOp8SrHU2sVCJv//BuDHcoMAfi4iWtGob2KcA0Ma/wD0hNSygm04HfvHCoGnCFg3GGBPLtB8qI63BHYPPPgReG+e2Mekv/0PhLueHPpsbLf6WXpnGlwhBOtbrA5zxVUTtmOspoSvxJe6P6QYW2/Ef0qlZbJRxKZcnQf846fFYJ91RWXeT7iHAv94aNA5becfGsPX5f44FOTwp2KN5XvSgaQCMgLwi9jOF28XxLs36Z3Y2wu+GnLhX4Fa+w00QfRv6qSPhnienDP/yTL8qNY4A8R9cjTSRL0cjUz2xt0KAFhm3scz18LpvpgGIW6eYpnlmWZZvea0Xvb07wje+AqKc9ZjawJJLSu4PitVqzeukiQEe+mOcYOWyBzP5jG/pN9hH7Nj0znuQ828sY3P+EtOflFNiObm18CoENH8ZkAvi4YweqiAC4VdBRskqSh8MXtJL28iSRv5fj0bTmAZm/nEvnpcsKe/0vYD3XFibg/ksgMz6I4VJQzHuf+BsMWvsqspTNpb1D+ts6QywxvcD3WE49LIhlItpl5e+pBPtsUT8zwuYIdlY6TWV6U349jfT2S2v7WdN7zBg8ZDCixlWiNnWZFAXS52AV3J0IbkooV78+g4mZT1ry8BqtTJrAhYjordLupeX8lrp3uCJlm3wXWcOb5Kxz8Op6CNoVO4wS2ZvalW6qKAqOqXpK1m1RH3tO7KOetzWbjrG8+uSlhDDw6UHHul2Qb95sm4GdwwbR4KhGyMqvquOcnfFM4Gq8WGYS4eit/1GZmffvrNJn9Oxe6XyDfUySleZRd50VPmOhTwnOt5iqfS7KxpttHPNdyAWebdsDkRF78p2afwQbVxgAkldIXtWv9cD/kSdtdo8ks9Vfc2xxRzdbZP2BrlcigiFK0tcoM/k53Y2mb0Jac0HIaeilxLFmGnIoG3E8JIyUmhWy9UsIZoFtQLhGtH5HnXUael3KMKj9PPYEY7y50y4JHJnBXK8nOdu/WnKyFHcESB/pPVtyrIuJ1IlLhmvyY040GccGwWXGtl24a6796kV3HmhGSAjeNynEw130Hr716gLWhyczOUdY9BSjNzOFw8stEqhHOWTaND6cCcggeeJGaane0ar1wfR/P4Yx9dIU+CTmEu4vkr8BsH2pPdmVLjhuyug7j00aa2T35bwlXrwpkJ+JjX+MPjD66gTmWaFZ1TFPY1LKa7p4FyOZLvL9pCFseK0msKepYjl/phWtKKAUx4OLkf4SbUxgbYMCUE8j+JJEwrzEWcKFtKc+2zMJWT+TGXvOQ3b2m87Q7lBifyDq0Wdgfe4bbvX+mp6ou0AfghaN+37vAx8AYHHX53gF+BmbiqOc3BhgChPlIwhoqSzL5gfl0yIxmm0rE3IvMdX2pkkVywP+WVl+843+y/Rfs+6/9P9UKq33+/e9lISLjf79Fy10TvGLwI8ypxpaExLWO16jRGWhw6DLZ1eIh1s27mkWfbGL4U2L9C4ABLhIzvCG30x5w0jL/R19/pjdM81cCIdmFrUi9HUPvT1ax7BUxsy+2w4/EDjnPhKBaIlXKw3GaTceKLFe0j8IocP+Zglrlc/dWp/LxwlW0HHUZ74bK4BOAa80FmqTq8MsLrtf5kiWZp3Nhpg8MsS4R7FW2UiRVJQsH9CSy6glgVpKQ1Sp8izW0v94eyawEAXtpU2na/gFRCa0I3DQWSVYe5hp79iOoiYN1k+ejPMACjHavxbftfQByS5Wp/FJVEraG8MbCFfi9sRnJOcsSeM7zEveiwF9VP3A6X12IHTurygPJ19YdkP8Bex4seI0Nv7yJp04JNvRwqWDsUYeMwzY3MVPhUEoAYWe7kdgmAckJf9dUPmRnw1pO9U8loVUlst0pYF2TRZ+SRAbv1dIkR5T6wWall8aDhhroWinWrJQk8K3qRkDiQQJMK+GJOnISdhrdK6ey2IujD5UOjmwzk6jz593r3XiQ5YcRZQBpR+pmghLT8Sjy572Do4QNvXfI5zTZ8we/zX+FcxdFuapuqku8eWogcWfCsavEgLNf9W6eeXsHecOOssQqAiWrghK4FOFgrAZY68aK2SOWk5ea4n2yB61+ew5bvnIctLBe5JfrvXl97UtINklgFu2rieU1dR4X3/mZ402c6gipNJyhFnOIYx0R5o65gudd7RTlBDAnw6XebLG3fPPo5wpdTvyEFD5KYZM0bmSd6kCPD+JZmdBTcajqWXmWDtcbEjDuBuOnb1PIsQL8cK8l733zAT/lPGZDrZI8IFuNTC8No8/pPnhVeOHropTnUKtUBBQEMO9cHC9ki9mjklqP5/HzzH8untQssWCyyaM5nTId4P39mDywKZ3tHOJYUgpfpnfgdun8uvtKEo0fNmbq6qkMPPc1cdnvKmr2PaYlLy4cz/d/dKJNBhQ5FTmPDjxH7IdFqC63JfNXpewYQLMu9yj6eBnh6luk56wR7NPyBvPjhv4Mv94JD5vIrL8nt2BBCXzjDzEmJ9a0rGJZv0QODT7E2hfXkh8y8V+Tl72Eof2Saf3dSj6d8AyaWiUQmOi+mgq/OFy9anjuZaf6ikBxSBk1be4R6FlIuckpO09SceHWBLxqXWn31AU09YBtO1MdoHqv0acEG0CFDUa7w7JAhMAsVhOnPDsjS2CofEdhKmq8kav9B/Bw03Be2SHWWekyLoHg9umEVvjSwSl7o3HjN/nt0HkADNUuWJzqJnRVp7OgaxqXtZGkVSklcQBssgvHtzrYs2V+AwW7JMGEPFhVAQ1cREZqDS6km8FDW4btCWam1qcluw0/kN8vm1c//5Wu4cqg8HONptDI6zvMfkGkBC0hv1q512aawfVQHq+tfJ24ByLYViPfZ0RZEEeKmtBNUo4hSePKYb8cLk5dS2b8d1jrqSXxtudxKhvA3UjQl4qA8FCfLC5Hwrct18ATe4ev5QxLn+oAncIoDmspgIRv57UCoNq1mhm5ShktSdYw8HIwf74Sz+IFcyk1KwMWZq0/G3b3ICz+PaavnIYK5YFNrZK52PMcn22K59nW9TAzLdVciIALEXA8Rpx35aZisvzWA6KEkXuFG6/9+Da3z7cmOU0JmGktRaxqUobU8gErpq0g3U/5XLIsY5Nt3G55m7PBWcL6nW64xQ8zf2DhRws5ohb3pDbGV5EsEhkPIkg2lgl2n5I1bJocjtxlBGcLRBC9ubqKCe4wyQM0JgeA8+6sWZy7d5H0qAjmqnx4xVXDvMAAes+dS3pWFu/OmkUn9R02l8fQ4VQvRtrChG8pqSW2zP6BqS8dZqSHUz0eScIu2ylQm7lhsWGURH9QllS86gnf+1Mv0/b5XC+6Z0FNyFLBBlBofodfep7golPWa5TLAr4f9y1X+uaQ2cQPsxM55pDnCzyM3cv+pw6R10AJnKpMJdyIhMQo+LnxYWoMSuKBSlbxqJHDBzyZIpIa0k2VfK9fz4fFcNMoylFJxlxO9H2TbSHgX09Wlb/xDjmxsDUYIuUnflvny5I7YfjsG8Qxs4WjkrLf9yt8+Wjmmxw/YSPRdzX5RiWL/E9TBHMn/4Zc7Ub/k2KQ8ueAu3w9LAF370pCLKLUfJbXDHYXNMW3sgfeWrHsQKFLHB8VQYq5/m/Z61pbsh9HcOt8K9KeqDOnsRXRht20aPGY5W/vI9wpi35bvgMUDGiVzIKe4jmrfcZgXKs0DHjzHH4tnHwIScVd60hcfctJ/aMfWERS5oXG9/F6cwOz6vkWALNyw5CS4NtSQKPMEPWWLLzlDbsD3WnhpZTz/rZEYtGxEezRD2OzeafCJksyYa3Ws+NFC2s+/Jb11lZKu6yiuMSTC8smUfggRvDn0qUwvv/6WaYveZdn6qtjKPuy9kEgpWeCSU4/JdhlSSIoBQZnw31rtGCvUHvR4vNl/N7aiHdAHUlKZSnnlyAImr2WZ9udYp9R6ZPd07ajWTr83vAsvw6dyo1iZTbvDWsHbLUOctSQBqLqxgyfQt73gabeD9GjJAdorTUs9Xyd7u+vY86snWjqIaiM0Z3n10CY8XAgdn+xFuHW/N5srYT3QzKRniCKXJfa8kmZje97nODNFheFc8E5Fz1uWWGUepeyq7vSL5IkiXmVFq4aIDjHQkLWYYW95NFq3rG+y6g/R9HxakfUTiS/WK7we1FD3L98i05ZHqKMJ+BpbEpp0TAeF3cTZDxTdG0JSYGB2RCfryzlEGcbSovUCTS91p5X1MosTFmG4T5V5I7/CxdfF3zUWsEOsDEYjnUWz1ExVfF8FTSHyb++RkhKPdmj2vbsLPWhbQY8UPURnynrHY5HGvguAFwkJblAUrtyw7sIn/4Xie9wWCnLZynl+cQ22G+05PsZsymrVK5FNklDQY0G495O2G8XCnuWu1RFxZmOaLNDuGhUKqgAvFUUzZv+j+lqKSRHUkrJImk4qHfF9EM8X7cz4KkX9+popjK90MTYPJGsLKm0fOWRw9GYJLrlKIllpR4jWJO2iJflIgaX1Qpr6He3G3B03RD6F4USpREJQbuy6s7RHq7KuVPp04n2R2NICcvieP/jwr09PGOIudmLcit4u4rkRNRunLQtRB5VzQ3Dq4I5yJLEtwEwzh38VcpveczFzoTlbzLk8BA+tCrPteHeMWR59uVuq0z0cZnoZOX57UDqbo62CWBeMXTJRCCxmlwHcr04hrNBal4OfiAE+3INgQye9wkffvEJHvVIal9QD2bja2barYzn+CDRx1356As29rlC7It/8LK6jrCho4pv/KC23V2uDT7KNWsbxXWn8/dyuNcs1jXX8d3Zs8IYdJHsaGwy/dVqmulEedHsFl/jkQzvFMKka28q1wRJZm5WHOdr4WRUNb6l6xTX9q29wkq3yTy4dpSP9mdh0ivnpoQdlxoHOc/FXSkLPN03gbxYuPz8djYPPojVRUl8/6J1Ls0GJtD4qQCGhTjLGcO8G/Mw5/hT61KL7IRzyTIMvjuQV8Zs5pvJ66h0VfrI/pq77PBsTHIUTPIoE96JJEnEaeBOFLT3+FVh0/wd0Dxca+d46hTF+1bpfciygE5ylC1wHp8EdyK9xVACHkbQd8MkkJQ42mNrKHuXPsPE7RPRmEVCkCTBuE2BvDwvnr7SbYXN5hbNc3lw72pzbp4dS5Bfd+H6NvzCjmGdGBJ+CLWtVLAP6ZqAZ1QOPycGKchGkyv/YnMwGLVGdvU/Ck7BvoVNh9Ct4gz3zzUhv0ZUh3ve+yVaJJThfqAL6RnKb3Wl8DyLZ87BYtCx+evJICn9qqHuEl+3LOCvWT9yr62IXXt4RLHs6HQs781he7ZYLqTGUkOpOp+3C2FP1pNEKAlX2Uryg1pedhI4GQX8Q71/F0in/sy+dCALmVp9NEaNkmzUWldB2rO7MVS4k3BFlG3/2CeVRh/9yuU+EYR6zhPs/7X/mfZfsO+/9v9UU1vqhmTmTXExdat0Yej+YYQvmkGTUqfi13YLUz3Bs1E6eU0f1hv4+qsimjN/9GTLPbF2HUDvR82Ylg/Hc7uDU/BKttUy0xv2VMOkXCUVqzr/EY1v3uFEYgRbK0QApk+HB3hrLcyb+yr7S7oqr9WH82VoEoaIHMbdb6UA+QCS3abwSkZvFntO46zLD8K9a70HkpUXQEh0Do2CRGkPSYLeCVmcyhyJUS8e1tZUVNMkN4dyaiipvvTv323majx8v6DzLQuDjwzGz+kg2CT8ZTyG+1IYk0FQfpDgfGW6vM2a9sO4uCiexpEioNAkLYJdQw4SM2clcd5K+STUHqxPCmDh15OYfbwJdp2PcH2maTi/FfkxxRO8asU6cWV+tcz/bD4HBq8hrSzt37/7kMqnvqDKDOXB7eZITs7uLB87ET7V3Gt2j2teSvBclmRitVPIGXiStd0z6OgE+LoGdODj66+z6UYU2fleCGyq4is81dThVDUNOyn0GZuJ09GVXIyAp8N/Aiddb1PxFjaPjqJ7w2+JU7VXgPt9Ar/EbUoqq19ZzenOTlJukop7qcHU/DGIYQeGIhmVBwQXyYRF7ThI3fIoFeaOilrCHzucLu8wZYYngNrzfS7u707spQgSSw8J9gpVMFca3+daj/OoZDGA6irZaK+DBf4QYK0bC7Iks3fUXq52uUhURhRqp377kUtotQfB6VG8URglHgCqtHQ+2YfHRR5k6pQMSLVKy2S5EItHFSOODCbc7jR3XEIZLFez7INvudH+Rr3rydLC5jyfB3bJKgAGkgRNHjgOBecqVf+yAwEKm0jELlhCyw6J9I7aAk7F3aN9KtG519DzbE8C05Wgq2yt4tNAK27R2dj6NqBNhFLmMJoc5l7vR+e1k7GcE5mwScUX8ChMA+DAXRGo++c5D9XA+tI4weFNlVoyqwiyTDrl+7Ya2DK6FXhB+/NlvOfzi+JAZZVciLnbnA4XutHn4idEeSgZfeHs5cXmS0lsm0BKXJrQrzd8MnmrOphWu0fRLl/M3Mup9cd0pCcHfx9Kqa6hYM+TI9hXDUPdwM+SLdhfstxhyOEhdLzaUcFcjtbYmR9cSt6KV/hi8mdYnINPFgtZjTpQOsqLzTliVutkjypie9/g8uyfuOvvBOiq9Oy+FU9BegjuAWVI9WTYpbXozJglP+ESJL4TgFbmFuyogr4ZWkFyFWx087/BsZQXWX3mA4VFksCk13Gu+zmyw50KrNtt9KmQ2TL0GvErv2F0lLLfsgxVvpF8+8E65n86XyB67LC0oct9V1ocV6N9pMziASjRNeBKxytc7PaQLo2erfe5BmQvZ2cINFSLEtCy3cBZA8SmQYG+LlApSaA2G8jxlFjrnk+qTcn0jqndy9Yh/Xk66yyj7pwmp0LJJL9etIWgXMc7nNxUlH5MiHlAiyXT2b70GbKrRGmm11zuczgsnTVUYVeLdVjSrW2oskGACtSSCFyNCHGAI+NzURyQLfZKtPps7M9PpXJ8UzLLlP32z3IA5rUutXiWiv6HRe2Y51WJGgqLlazRMs8OLP4byCpt8UBBIAFwN6QyxU3m/UwtG4rqkaNS6ZlVCEeqYUOZ+MyV+bvJbZKDvSGs8VfueS+mOe5XHOSP1FYZKJdUWioaJ1P08lbyQvKE9dffXsKsG91o/LghKbEpwto/QJ/Hp+EGbHpjvYGS+9ZItDGZFLlWkyyLigk5+LDyl5EMPeGHZ6VIDlvgn8Rod9AkQYau379/D/JzY4xLV16yzuTlmrm88Pos5s6bR3CwIxvzsS2GNZcacHJnP779UpTdm+iZxhq9Ow1vtSCmwknezmYlL1zNF9/P5vVVr6FGuR9SlcrL+qcInxVPyO8T6iWodLC+xboguN3iimAD8JGKaZwVgXulchwdNjbk5XyoGrIK99HDKKpUBiI3u0j8YgihB140VSvrH8laL+IfRbJ7xSgCjvZGbVGOfV9bHsM7P2LLl/HsGL9b6FO1pU4CsL7sZ9Tu/FzmWNu/r8cHNqiC+KEURrjDYNU2hS0vJJ8d43bwjV8aR2ucA+USZo2Znud60PXmGfKqlGuZRdaTZ4UuL+xj2jixxmeS2ZOM2FSKP9pLZmuRPPBFsYFLmmvM8pFo4REl2EtcotlUCTtDoHmZqBhis6ixqCyc634O+xPZTe7Gy7gdv4p2f3uez4MCjXKvfUQQj+Mec8vfh1W3dwj3bTUgCc1kX+bmfstZq1JCDlnFs5o/OB37iGK7jRq1mAG9vNpC+U+TaXW0n2ADiK56pt6/A+DZjP5XpuGqqaKB3zLl8xZ6UXqtBXmW+9yrpzb0B206M6f5QTJdqjE47Ule/gOYdXkVd5vfZZFrHkaNMqh8Ue7IRX0Fco0LwxPEbIAabUuSv3mV2EMDKErbKdglSaLACkdqIMMiBgvv2JszO8Wd/Fxl5ofaJYTXrn3I9tuNCb7WBpNB6afKFhi2bxhdb7cmRg0qnDLhNTKff/I56+Z/zhs6kTX+al4sb3wyhe9ffJ+b5UpCmtqQy2vu6yjLDGNmngqLSiSoRLgOpqCiJaGFU7E7rzdAVO0Y4oth2MPWirOUJEmozWqCc4PxqPAQ1qJ/1mSfMh/ae94WbJG29qiaP6LN0oU0bq4kg9olFdlpIbjHZTCxzx1UkpIk5UkB6vMdKMn349ebrcRgn91Ot/BttAg8hUY2iHbZlTwrHKuByzVKcL6J5yGiakuRPCu57SQ5LEmwPDyHIbfbEnE9n0DbcYVdpYK+gVdppYNfKizCWpYkt2LTiokEZ4UyPucVnFt2zU2KXRNIMIJRFtPBqkr+BKBzBhhlJ3lHu52wCi9Kj3clLzNI8cwaew192tSdu+x2JRCdrmvFZ+v7IVtUXOtSKEiRnbFP5Nh9R+A22TkbHbhQ60v5ic74nu+ITXIuj6LivuEEZ85fw/XBDnRqkdjrpsoj0qUci3NQAcdYiUpvyE9HR/JGlDLTMrPqLmXaHYTkhOBe6S687zs+l9nQZCXXNueT/MVdOjZtypdffEFeniOrbJe7PxfnfUvgp5dQOdWVRKXHMzWKhg8bszfOIIxvV60b7U5PYOmUeH47IeIuALKuE1esi7hvFNUvMvVd6ZkJ3wbAO95Kf29dgWOeegRVEBylPI+oZBXFHcYSNuIG98NVRDtlgOqMWbx2rDvx8fGM3z5eUMMp8nyRfW+/wFcvzeNGqrI2niSB+omAfjO9SE5MqP2Tx6ELOFADWypEf7H74cY8t2kM/Rrn0kRVd+7Q2Gtof6wfUYf7cbDjJYwqpf8Rbs/nKVnHmG1BhCadFmr2jXrUhuxPZtJz3jxezBCdBEntSo3dUYPuflWE4rwuyTIrCsI4UQMNNaB1ChB5Z25lqHciAZUDHD/ntF7kp69k6vvrmPzxeq52PKScW+VzWbt1NimpkWysRMCy0nQjUXeOY1a/OTTRb1L2WYJHYfPZMucbDJP24ooYXA3Ob8GMZTPocrGLQKbzlNMYF/SQWC30dxVrUmpNuUz2gD1VkGlWkhq8r07F2AB+LIP5ScMVz6QylxOngVY6uBgBfVyUspE6Tx2F0R2p9CvH1+CK2qlW2y5TL7K9HRjVyy7iejEh6hV0pQ5/Jr3WueyACq1Ry+HBJ0lpK0p0ApTIXagyefFOl5fxMot+matsozIzGJ9SHzRPDIQH9gYcrYYNkzeQ2CxReJ9p1T3w6qmh1uTO/v3PK2xGi5F1W8dSejuCX1/9lfIGymfW2gy018HjiAz2+eZiUivXb2vkdN5b+QUjlr/OM5I4r9B6k3SgEWajllyjuEZa9TG0yYCfykFWPXF2lVUMTB5EQZWEs6c2EjgF/5YeCAbmAleBR3//71wcwb5z2KChhOSUCPCx332WNSpj68StrIoSv+WL+W34wjObvAjX+n33/9r/SPsv2Pdf+7/a9HplCvbzIYn0HO041K5Ri4yPBXejCLrTnKJ2t7inV4JWst3MZ1WhTAow8uMrx+pdWPYfac/JHf25k3dENDruwqoKmJk4ETRKhpnGXs3SAIczcNWg7He7Ri8y6N1NHKq188NDUbJyeFpDFlWbMegM2Jz07v8BsQK9q2jlXy3IaHWL+hjp0Nt4ZtwjUb1RuPe1yjRuj96L99ij1CLK38kyjIzZxrXMCeR7ThPsKllFlR1CUuGotk52xFqTzSchi/FNDyeiUQZj/JVAXq17N3659jNtZm/i5yW/oLY6MQn/fv9rg+Bjf5FxakaFvyQxNqwSP60T+8Y1lE+zPPBIbEybhDb1BlpSjMPYXOrFIn/wr7wl2IdVwJeHRvF8WQjqJ3TSva2pzPeDrGkbSO4wURgniZVrSRwVxI4JO4RYnSzJbG9zgGdalrDrRjQFVUoQXO8eRf7hKWR8/xJ7v3gV2UnuweDVii1xxXy8ZgHR4UeFPiNreC/PlUobTIj8CaxKtmGF1cY9s5GPY0/yYbQerHWgQmOvHxjUcxeBpsWM1Sj1wlVqPdNcTVijMolNiSPSoGQOxZkf8vzbO8l6cQvfdD8hHFyaVs9g3AsboWsE/uFOcpWAf3R/zF41hMZm41O9XbDnqRvxYj64yhAmlQn2n0p/wS8F2qRDpkud0IC2NpeFMdW8/vJhFqz9ivBoZZDmgnYCo3rtQ+ddQYsjgwV5mfFyBt0vdKfJgyaozE4BzL+DWEXNHxA24BIWnA+hGtJMrnTWwztuvkKdCoDztY4any2jVyGZnZ6rOpNvZhxDnvMTn3c6rXAcz9kbMcVczoenvGj+ZxMKjUpwaExoJlvH7qbl3ZYMKVAGHexqL17Q5BH30Wrmv/o8Opsy2yBCIzFqgMN9i00R5SD05YdQaRzg/7W4U4LdXzay3Dwd/aNfySp/LID//8zF5Q3P0TN4fJ3BZqax7ndaTsgjolsR+fbGigOCVm3i98/nI/tY6HtaTUMX5fq6pNCPwQeb0X/QVaaNEmtZfZLTjMX5Wgzj9pEWLUp3eOgNPGxxl2vtrxHqKTLMQOKyAXSP4Z7ncKXJbmeE1XFwCLdpcXmixqfRrS2zdp7lfpWBtOhMzhUrWbp30xfT7MJ2dOvN/J4trs+DLnbC45P3OJDqR5ZVzN4wRTXgx5lbGRF0D6ssPtfU0BSWZku8nimy7gHWBEqsCoSE4jYiEq7SM3a7iR8ui9K5Te1LGeezgoo+FwlqKjIJyyw3uF+TxtvHDpKCsiaDXP2IiS2+ZFzx13R7eJ6G3kqmaxGeDPpqNtFJ0NwSL9xbJclcHnEA44DrqO1l9T5XTmlHtt/7mGpZDNx2VSVRGAtDXJXyuKVF1+hW9QFt/wjlytbXMduVWYVlplZ89can2A90x+N2M0G2bJzHAeJaPqbr8HP0iqtHRrzSB5t3OSaNiRSd0z5flcrSRtEsfNyeRg8b1RtgklTr8UiGoFSwBIiZQLOLoG0G7KwCnniuNKk1kdkFBKlgb5gdlROYPMbf0Rf/Yn86lSvnqwQ806CIOzN+5eCQgxww/aawq1Uyjxo/Ij4+nj9DpuPtlCnsV36F5872xf2jOVSdE4NiqHQsK/JkcA6cvSAGkKwqN64bocAiUys7sgr/8fsmxzjmmcnFV0BQdK5BPHf7GVLu9GPpzdk85arMeAm15zExrIrGMbmcjpRQO7EteukL6H2xG3Pj4/kmo63Qr03VHsx5YQ0/zPyx3qDYaYNM3rl2ZJxvS6BdlCr8pSyST4vBAthsde88WGVjyYc70M5bwYbnN1Aa4JSp5jKA7dGOd6wyi3U8B7haMf4xkD0/j+PAdSWRQ5JVXM31wlrhQXBOKLHcd+qVHd+Yu6Azsj5Yi4ta9Ad7uhhwlWF1scgOBvjI/DOTVr/CJ2eVAaKse9GEfjCXUXfasCYIZKtyDBpPtyf/62lMWv0cE5wkLb09Ikg5f5G08205tasvVdVKNYUAWyaLA+C7QHjaQwwo+Ndex94Q7A3hJU9RJQKtN28WwodFsKPSTzBX6yJ5pwiezYNz9tF1BlMpqxqW08/ixozlM/BNV74vF3cDGbNX4N4wnQYPG+NpV+61blItw6914q4RLkWKa+gvFc1Y/sPTPNzUih1VYlasLMkMcYW5MWfROCmVgAPIVgOBKlDbRbnj6KASji74ksKhx9E9ERA9U5lPwbAT5JssVG4cSZVaOQZterjZ9iahJ3zJuuIt3LdP+AIGt13M3dAB/FkksrFr1NnsH76fa0NepWfUIMHuV9Gf5Na3qYiu51sBI1xcSYqCa2FaQUFFUmm4o3pEhwzYkK+UNZvTqJLFsxbT9H4OlkxRGq8qX0fNlVC6F4Tia1f6zl56T/xrJ+JbEkSLuy2Eedc7ZDiPm4dQGp7NxlonaTygc8AANCYtOhcDYY2zBLssSQxydWQ6OIOE/9if3TaRcUdD8TTVkSpVGjdyk75m4MpJjN47mo525Vrlbyqg47WODN49ho9/m4q3E1Gphz6FNaE2amQbWRYRBHxgciMoxRHEPl6qzN7ALYo9D1eTunIc1YdexddDPLvmqd9nSsV9ojr2pahI9Mtae11njDvcr/ZTAIGDOM0uvRevr3yd2d/PFgDf5W7JSLIN/4h8XnSS/5Ikid1SFE8tmoXH+Y74yUqQu8i3G7NvhFCVHEm03pVAnXLPStY9wxeNbvI47jGJscX1YAF23u34DAv79+W3p8XMPw21NNaApwzN5DEK25S4cbTyu0+B1sDBamWWmizD9IwoEsLTUQ2+xn2jWbBPdHfUI/6wSHTXzpmD2DRwL436X+OZkQtwbkNKunIqDLxkcFS2VLZl6ueRkuCKEVAp1zKp6jGnuuQhTdnFOx4Fypp9LhFMDL3Dgec2s/79xai8nDKIkMiMyKRB36t8NXSB6GbKKg5P3MFHczfyWaCYxaY36nnq0FMMOzAMjSDNC0HqRoSZYwnUiAQogOf9FpMcDZ8Hi2uki0rNK+snce9MOxIe9lHYym8u5qmdWUxbNY2ZJ4Yo9vklixdzb/kRIh/n8iVFrDNU8c6DB5z96iuiwsNZ8v33SCqZw+pq3pAPcckJY4j1CMIvdgC+H6zikrZK9PdsFhpnO/obmikG0bHU8F63SQyOW1UvVmVU+XLJAF/lBZBhU47BDJsjgJFBe05UOUmr221MbDKH4D/a8eULn1BqU679UYYEohMdcpCNVaIU56XiAxT7OtbkgntRQmbfa7H7+HDV1/h/tRGLQVz7A+ylvH5yINKS2eQ8uinYWza8hDq2lLZZNuaa657Lpg9gRZGa0lot4WrQO8k3DrSd5oe/Jbx9G4oZgyHmcLwqHO/5pFYkvOlqs+moAzcJ3GUz7k8cPx2vwM4DswMHKwl+T3nxgNP8lr6V2JBdDG82H41N6Q8Wo+ViRBq3wgs5/Hia4num5HUj4y8Pxv82nclH9hHuqfSNdjyIZ8/+Ecz76GtOZY9X2GQZXglM4eMzg+jz/QyiVVWCvZ+lF/7F/ryc1YjGpacU9lz7AF74I5vxOyo5V6skzgC4mHP4yBeSMp+nyKrca7WB3UiWHFlzD0M/Udhcs9bwOBq8ZZhdCPdMyjPagweT+HqszODZa1i+cA3eNiX+JyGxZ8weur+1lcmhFTg3b7c99Fz0PSumreChVrnG+tqq+E1uw6StY+hqXI1cTzb6eVMgnXPLmV0I9+tZUr60+JH14XKmjD2vqDd/pbo95T8/z6trXuXb7ZMVJWUAFuQO4wv7Y4p90rnh+YXCZrQa+a3Hbi48u43T3bMZ4q8kqYZY09lrDiOw13WODTwmSuy7RZJhD8MnoIzB7uI7AciODeZWq1t8FCI+8z+KZS94QGuf2QqbhIx/tR95Ttf8CTQERsD/d61xIAaJ4IJqNDal37+2ohGLj7RBbVFj8RF979tmP9rvH8w75bsF8sv/tuYc7/ifbP8F+/5r/9eaSqWiSZMmqJ4AaaLcZnDldCcAatxF/V+pcSZew06xeuBBijyUC4unSxDrj33JkU2DWZLqVS+As6HVNQD0VWLwCWBU1WS66CHT/zfBJul86ZEJS8tE9etivZ5871JG/TmK5rdFcLLRvQhi06N4NGUb3m7KDdvf8IAT3i6kdLrO/M4nkWSx6HfjG5fpctpMwyQRTC425nFOquWbJB+CdCIbUJZhYvMvGRi3pl6nMlJVwAwvCFOjyD6S9P58UmOkYMBpipo1xer+tPJ9SBBx9yClf3zB6YyxxPooAfai8uUMPXiCVS/Es+K2ss4VwBj31jRfOJN3Fk7kqlXMQMqLKMDiUkszuxaVVWTK5lkekFXWlmfvjCGyxSzB3r9SheVyW94sjUBvrGOQleh70SIdPJsmM2BkKyxOdQzzzJ15dKshKze+wrdVohyE0epJQkITKvb1I6smUrDXeDqc66sdrwobut0ljO9N6YRmm5lRLNaiQlazNqc93TNhaFI/0CgPCWX61vTJhnnFdi5W9VVkf4zNasvAdCO7G8xmnJsyqKtX6xmc/5D0EUmEzVxPjbsycFuiDmaryUhxoTdqs1o4AOyrKOBWm1t8NuMVWnu9JnRbp4OT0y9R+O6vFMjiO/lHYPN0OIySRKZVe91EqGrNLRPY1HXMN7W5mJckN6yFPqwoh2pJ+T6aeLWlRfrPJHcK5nHUGOF9NwzJZfTru+l4rSPPP1B+Z0mSWKr2xLMggC+CHlOkdmLW26wMLolnvktTvg2uQFuPlGyplMR4d9jc4BZUOdXKUutIuNOVyjMdca9yV6xHGouMd6k3Q/cNp/3VjkJ2KUhUBBYw7KvlaLo531fPkRr49XIci35ejMGsZArW+o5k6v1jDP3+O96bIWZJVOmioMVD9L5lDApNFew6Yz5vNvuFbh5l9a4XTbjK3hCHfJ2v7olDl9oNxuSy3vQiyxp0Yl1Vb8X17TWFpMTZGTHmAuOmbUVjTFPcN6vcQLeNE/A435Peksis725YS8SWsZwp1/LIVQSqVwRt4+NZO8nv0oAGXmIdgG62cyRHQ7eagQyNmyDYc684HLGOOvAsvfDv3yWNO2WP9QRl+HKm50lhvWipr8sUaXNDDDLmuk6nttqVdjfaIaje2My0aRXKK4fb8/aPb9dbY2uI+hRR371Bp0viOwH4M6s/34hKJf+2Kw0HMnKoRMv2QxV/t6btJHdjBHPLwtkW4jQ3kJh+YjBvLJ5N6Q0dNSYlYJCa+RcTGn3JpKJV/OA+A09JCazqJSt32t/Aq9dVDMNEB9/TXszOwiZMP9gGbZZIepAkiO/zFE38L1KjEjP8G/i/zJFq6K4HF2vdOlpdmUq0vgyA9lf86YhSIvSS2UqeBm4OOM6+T77BqlJKyKXZY0nL9mf/1basThEJQXOKwvn8nSV8Nfcr0DrNWbUbp29MwbhxBM9ueZboehx7OzYkmxrJXn+9kxQzJIhuz78BzXUVMDkP0CjX74kRdaDKpFZO6wUOiddvO2Sz4OnLNPVUHkK9jcks9ofOhBFSNgYXjbLf5f79+DPZMd8s6v9TcXHHs9S3XlS7NmZgNkQ9DOWm7neF33fS5RcAGlYn0EH1reI6D70bPRK+p+NvvSjb64G3QXlUTVc34N2Iu5R5WEi/MRAvJ9nJZVVNObrZAUZ5eYv9lmWZV7dP5Itjwwizi0EJWZL4bvZ3fPfet5xT9xXsB6oCOVLjGINqcx2ZSOseybpH6zhqiyG5QTKSmzII2cl7KHHFP3FjVgZpL4lyVLcNl7h73lHbIj/Oyf+QZMZXVZM09AgBs36jSnLK7nCPpXOJO/EfL6Tc7XVcNGKwb6T/NjrrYE62uE8DXNU4frPMSUbf4mYjNSaVyo2jmf7TUOxO8ovxgQUkN79HYVYg+QUiycTsFcDJfomc7HMSyVUZkK507cHEbZ24/lo8r+RGC9faNV7c+ntetNWNEuz/tEWlsL2eYN8/c21LJaTwRADVauAZTzuNg8vxa55MP3+l3KWvJZf9YdDgnY2ULvwao0p57w6263Q/0Y+Cs+2YXSgqFqglDWHZYbS60wqXUpEs9Ia0mxY6GHt7DBadmNkXVXmDLcGOTN97fmJd6nUxSSx72IFthY1p8UQwsVLlz5IWVyjXmP4ObCkH2Xuexxm/0wEs1laLChPPpGlYeiCat5a/RYcsJ5a53cYGUyALfn6L/hV/1is9/WlmU+JutWJ2YP11ZoeG7qKBFgLUNnAiN9osNfj5nSTdAtlm5RiTZAmVVUVUfjAzJHF9/eCIH9k72jPw59cYp1EGwjXWfEY2XkqgXyXhvrXonOSKu8oPWdA8h7Wv/crhFqJf5GJ+RLtFWyn98ltua8VvJUsSh8McmQ5N5AuCPUZKQ1friq1GR2H5wbpnejJzRrYxJFD529GaVN5fsQi5w21yazUYVd4KeyedhibnuzBl2Rs0zxD9IlcZ9r+ylqRp6/DzVgZW7WoPTrg61ogmib4Eu4mZOHY7dHaxssgfXAziOtnO5wwL/cFDp3RCIlVmOvsZ/30uvRNprNDijt0mU5QZxObzbYT7Rvpexat1BgvytJyqUNolJO43vc/6F48z//551Frl2q+R1Txo+oCrL2/ELbSkHhlPiXG5joyWHRVawe5fc5kH0VAeB4sjPlLYFuT0YUe3I+wctxPZLsrqHarwRX2kF4mJgdwwVgr2328tpHU6SBYPYe1X/93viwMSuW4Sa7qfNkdQa4eyOOivEYNqkiShAfSSI0CnaFpvtqgqud7yDqU+St9eVqkps0qMCTCR2NjIIL9Oikv7+kTyarMWBIw/xFm7Seh3iJTBIG8zJ+5E8ThdzCA6HHWdgLACOg+5SAgi9jLI/Vm2jvNl9cgIwb8G+KXSMZdf8RMzRwA6jLqEJiabH8OUmdfBfoV0GnSZkEbpjOr64N+A3JLFi1k4dy7nbHDCAs8B/XH872GDgTNWKwvnzKHn3cOsudyfV1a/QmW+MtDoU5XIzgHzKFnyGg+/XYbaaXybjCXM/eQ7xsWvpGKQqG6BzUzvqK3M6DSNfhGtBLMaOxoJvinwI92mzGDyksIpDYyhxl2LUVK+k3JDOT53zmNLDeB2y9uUqpVzOte1JbfmLmHAys+J/3ArGrWy3y6lS2n78Wr2TtxGZlStYm5IEjyqiGSH1cCqsjCqbWJWdxvVA4JPdycowwPylD6bSqVi7NO/4mYw88LvLyj2JZVaw86BR6htncjc9S/TTavci6+oO/PTA8f6NqLrOmEMrug/mPyp1SyYs4AjvmL5isDHP3AlEvaGwule8YQ+8VokCY41vsn3f3MxNU7EsQL0bJM+w6/lOKa1jkfrVCaiyqUBm38dzvWZM/n9ulKtQRUUwKNOz6I1GulV/Cfaaqe9peQv4m6eQpNpYFu5Et+TJHg1+DHqalcqbC7EBk4Q7K/3mY2nXznGKh+aBCmJInZZz8nATzkf+TKlNlFdy+LRiveOXODug0XiGtnsQ3Ywl499YEl4psJu8+nCwhK4bYLvy+CxRbkfmkplDs99g+v7uvNlCRhkJUYx2aOKBVGx/FjzPr/eEfGgrukB9K+pwNf8ET29XlbY/FUSY6Nu07bPdfq12k5hpbgOZhrvkfh336pUIoGl4a0wwha+Td+CMIJdn/ChvSE51nF+qngULUjcPlBvId/3JD5Tt9Oqr1PNYCQyojLIbvqQEoMas1V5bZUmmJWrh+K9cSyf+SIEzW7knSMteh+l2YHsPy3imdUVKcx+dRLhL+2h1kXEJdXWCmZ6w+og6OOzQmEb751GcLgvv6Gc63uBeTjq9fUCBuNUa/zvv38MxGOnKrcIrU25p50xhONxuDfP3hjIYnextrlaggbnu5L6Z3PSq7YI9v8trb54x/9k+y/Y91/7v9ZsNhvFxcWKgpVZ0jiMpS7Y/MoZGSOiXrP8jLzb8RQGF4MYcZNVpIc9TWLTh1w507peZn1qTCqvLl7K5NFn6u3Thy2WcTECvgvNAqsT21Cl5bwBNgTBvWjlxrmmohvPVlejfnYvw7uJjl2Dag0xGVFs8dHSU6u0a+wmPPOCCFo1GU1iI0GKLSRvBmPe2AVAe8R30tBwgTV5TRn510hCLOLGJsvw+Zl92O0yATXiYttMlc6yQPjEF0KfkDKUtd4sNxlY2eUMJ6zPU61SZm/cTPmQoPTLVJytZNUfi9DolWztQMsx5BLHJp9YKma0LAs4g2yXiciKQHZW5zAUsDzAxNDX/mDKhNOEuYuSJa1cfmJF811ckpLQe4mA8F67g921ptqKWV93AJE03twzQU8XeDUgWwBCqikkIrmY3McRFFaL9aK+SfiZ9JvRXOl4BZuv8ltZCi+w/NUOtFkZT4dRZ4QFVpYceZvFNqiy1b/oh5ZMIscKD43eYlbV38GAVRXwYr4GVHUM5FKrnje+nMu6z17l9mmnDAy7ndFxn6P2quZ1j3TKtEpphFJNGNcutqD1/qf4La214EgfMcey4FwDPnn/dTbeEEEWc8ZHrG98hnUVkCtHC/ZgazJrg2BjBVyW2gn2BtrtjPEuwksGzRM/bvRoypvLhpMwdyZLzjai1KAcR7FamdHFwdhL+vBIP0Do9/LasUx2ycK711XyA8RMnfF2dzxPdSMsO0xcL4xF7OnxKhpzMHNPiHUoALaElrDQH17IA9ycnlsfyP6jT+N2rhNLKpTAavuUAt754R1yOl0nf8QRJCdZnNEFAfQ625P+hSZ+tyiZgDqVjsiCnnRe+zy15yu5mKtkW8sqCasE57VVnKtHKrBKH8X3Y3bw4bKltNWLUSK7V3NmHb6Cr0sO4wJHCXXgmut9GOwq0T0T3s2eVGeQZBb8toagI+uoMX9JRsAqxQEhOnAsvx/8iKN5L/OL4WPKtEpAWO1m5siYPSTcbMHiX1cJ/fIJieFiLwPJccn1EjmOGZoRpIIr3Zaiq74h2C1osNihjXsJarMTx02S+KFpAPe6PiCpdSgB0XU1A8sNRVzuMpPKZ/ewt30BvhYl4HVJ3Qjb33I2E4aKzHd1n+EUdimm+4XudDc4HfTsVl5yLaeJzo4+uKhetvaPuxzsV+9IUXoUYF9+JzrrobRtPbJ8dhsdQ48x1Qva+SnfySpvXxJ6nOfa1vHsPfSN0+uQGNfBsRc0uHWG/VlKFmOea2MG3PDj0t5e7FszErtTRuKXPpXseWc/XwzYz4clIvjexseT08sncv9qM47kXhPskgQXMsdxK79/vfV2yrTt2F4FH/iCh6UO1DV5teCzuAQA1HoLYR7nFNfdMV1l2YzP2dvjLBfMNiSnzOsbto5sHnAYlUHGliKOIZBoq4O9IdBCrQQj0Aey0BgKriZaDk1E7xQMAWis+p37fu1YXr4Mf1ncy2PLxrAjGIpi1Ip5504Rr3g66jNurATJKWPrVHkdwJBerZQCkiUVs27MZcXmfvgnx+AhK8H5Rlob44vCeSp+KmPzpyPZlWuGzT2KzYMO8tSc35jYVwSxsdSSGBKCvSG80/xHwfxPoMWgzeJe9SmF35dV2ZEKXxVVha6k3VGusbKthn5/H1qNMeHUeiulsEwqD65VqSm73IiD5ycLa3+pTc+fI/7kVicvrvkuFfrVS3WN8MSmaG62IAgRCAk3J7KiQRUxvtWKLMu6JjHYFc5FgJ/hCda/xpMbSU8RlOnNTE8JV6dAeLB5F0v69mJrk81813ydcFer7M7VQW7ceGslvkHK9Vmr0tLNsIZNnS7wpmcGRfUAGTIy4WoIdxMzzQCWFkXwXD5UuCbUa3+/0Q3i4+P5sIvye0xodJkv526mMCiPLKOMJCuznzKjc9kwfgcnxo/B0HOiwoaxhPl9evHiqFNIT51Go1dmd7i6xFKV78j6PLf9daFPNW5NaZMBusQAEkzvi502lbE9GF72BFdEX9HTlMKeEBjmBo3d6pQD0Aeje6AlY8MIgrRWYsOVmXvl9iDe3tGZ5VdiWFQqglZZciTmN9fjmxzL2N0jhd8d7ZZB64kO0kDvHDErJZRCdBLscdnDjVIx809tNxGmhljPEMY2FSWrluREcGDtcHb8Op6AqNH//v2f/bH61S1M/WkR0SZlhkWOuS448tGMD4X7ymda0eBYXxq1fUi7MDHw1cfNhrnAj7iim8hG8bkaRSXRumcCSZWjBRvAL5mD6JMFMalq0Cn3B7k2mfvR8LEPqGQlIPaU1cTTZi9y5/zI9X7iWnSr+WWSmiVi73ORBKvSxzVVP2RKk4/oFFjC850e0cpLKdnqVnOewAd5DP9jJCJUDDpbLkODkqg1qam0iEF0SZJYUQ4GG9RK4p4Xocqh56gz5I0+QIZLHXlAMubz1ySJ9u+dI2a6iUo3pUx+46afMvfmL/z01Bk+m/YzBrXy3o/U8zi8cQjuBYHMUYvZn5ciK7nSL52NvdJor3ks2NeonoZBZ+g39giyKU+wGww/01xr56lsMPmINZ4/qr1IuhluNVeOMWvsUV49UMmdUe9xf/RMAj2dJHKDkrjZvSF3WtzhWrAIXm5XZXDEI4BeZ3uhz1I6AW61GbwWbOC5DX2IunNA8BG85FJGlQWx7n4XvvXIEUkoksSuKhibC2/legrX12ij+bIEzteoyTe1UdiOVbSm8moLpq+YzvgKZfBUlkGyqzgy6AiHhhwSMjxVKsirCWWMO6wLNQj96mA9xEFvF46nGllQIILgPxvb80YB7MrugqTvLNij7bncioLaBuDnlC2JSzCvF4C7DFO9TYpnDncLYurJnzEufZFfvp2N2aIkE4WRz7uNtnPm45k8/O5FgZTWVr7C7MpQktaP5PrtpkK/btvd6fz5z9wffhitViStRWscpMSfKswYLCJ5YE1lBf7JUI8ABcga5npNYu4Lv1Ltphy/J/zaM7TbQb6espa3A2qRJZm8vDw++vBD/jIaEd+go3UG/jIaWbe3lMSDbYjIisDDKQMZ9zg23/gEU0gQ1b6NhTNJldXKInMNzRvmMik0TfwRjSfqJDhRA166e/X04Q41DaCn/wMeGpWEjCkxTxPT7Shhyfcw576gsNWYa/njdEsKNEZ2P70bWe0UdNCHsbHWyttpHXjvyFU8dMpv3VDK5ynXGroZ71N9YDdb14T9K29aXJzH3aT5nF/4M903T8Iui+fLB1Ij8p7fge+kCzzV9XuFzWaz0SDNzMXATOyROTSU64JmatlOlBqs1a4UFocQ5j5VcW2qpiG/NLvByY++5yNtqTB3ooumMabpCcxaMyI4CG6NpvIbjTlQDY99OgvXu6mshKthojvojUrZ4AeZJyjVbmZVBQx60AarS7TCLiHhUemBxqKmzEt5DlN7vMGkyb3wmfGIDs89EsoDdQr4FjRm1r2wjlo/JUYhy/D8w148utWQHC9XrBplBrMkwfZ7czH2aE5Co1kQq1Rg0cg1eIetoUPEDjItom8uaTypMvnyQfeJNNZtEuxIdga6wmhvJ5wgqC9ziiHb8k8/lC/ztqYtyRY7xbcas/lsU4xqJW7TUfMIzz2RtNpYQolRJM4gSQxMacSnledornY6h3m3ZFFaLuevN2LdofbUqnyEy9uprrI7BCLV1EuqDAstwhSew9uXP1MQ6nuortFjzGkATvc6LVw7ze8hMw6Mwn3JVCZ4KbPrJEnidiS8lNSCA9PnceNma4W9RhPAjq5nUaltDL/ZgUgnwrJP2vec6HKU9I5pSN5i0Mxus+Bd5MeAU335qnC4YNdYy1kaALuqYEK2Ev+bH3KDaW+lcFqyKTL4ynFId77L/3e9vneBICCrGixO/oenHMTpPjcxpXrhckvEP86EniK4zxVSo1M5axH9h/8trb54x/9k+y/Y91/7v9bsdjuZmZkK/XfJ5tg95GIvhiSLm0Co1IaJd9vw2/YXmGgU9avx9MS9yo3m95rXmyWx2b05fh4GUmxibRmA9SnjsNphakCRUDtPlux0SP2Nk7VwvFq5eXmnaPhw0Yc0kdQMbyJ6pK+NP8rkd7fx4/kWVNiUDN0Sz7b0MRdREJJLWIUXzt3eU3yHpNZ3OPNNPCtaieC8Qe1DwilHAMXbSzx4yzLkVjagTfBxPA3iAeI2bRiTA9O8oJWlLpvhn+DH6D9G0/WvlaRUKQ9zrpZMZi9fjF1vIvLuAeG+16XeHPw6nm/mx5MdJzLyrpq+xDj+KrkdbuChc/qW5gomayQSjHo+vfMLepcQ4XovVQnNtaCRRIAc4KaHjf1D97MnJA37E8xPN5WG1qVTmF4AkamiM9DRqwVNqjMo9dNRHPiHcN9t2t64mmCknyfD/ZUsRpNd5qG9jFHusDgAXLVKUECVd5SSWJjvCwHqenS57TZ2dF7ATwHgozILgRY3cyaf+kI7nQPEfbJFqGvJiUvGoDNwOMgZaLbTzrSByAUjGHX0N9r5DFBYZVniYlfHRl2T2ECYO5IsMfjQENQ5wdyt9Ra6nWvIp9zm4GjXN++87EW84An7qyFJLR4GO7l8wLqobMrioD11NQYklY6EJg4w/9ktz6IqdKrZV7mFjhXriLh2gEKv1YIDP9HnG+Ju3+OI60rcw8VA9ybLVcpbZzIwKxZvyUkaQe3GgeS32Zc0lbtFnYU6FQA1NYMc7OCiaAG0AsgMdRziAvOUQEdUaC7dhp3n2SHXOdwpH8lprXm5OJroE70JyQ1BbVJ+5wC1ilN+Dhmp2y1vI3sot/GMksuEVb1I+Yx4Ft0SwWAXF5mXF81m/nPxbDvYT7AbULHN63sMYUuI0v8lpOsUuH9A0wtvc8EAqie/td3OiVtrCc0N5tktzzLRnqu41OLSlLSNelJvJLA8ZAo3y5QH3Fd8znJ+3C08KiVcCsQ1dLhXL5a028ubq6fS/b64zm2q6M4vL8Qz/7l4amVPwe4buZgfr/zCktjr6IvEjMd7IUfYMXgrh40jUHvXZSK71tzkTq+LeG0fwemVY4Uahg/tEWwZu4uoGZswRolgWbLxADmdj9HhvXVYfJ0cPVnHgDNNyLzViHG97hHqJq5zh9rtpufYk4zsdUmwAdyMfZYLtbBNLGcCdhv7QuFQNXxRrpx3uT52rjS9h71cQ0pue+HST6ISKX7/Fya8s42BvkpyjF3jwQ2XCpIaJHGnnZZCszKgofEewd7HI/DNDMOjXJTUiWrYH4OniYKQHPKjRHAe4GrOMFoHHcfHLkoB2exWjtbA6Bww6OvIA5JKw52YVM4sXMjMX5bholfK7TS3PmBfKPQsDSQqM1xYX1WyjEFvQG+zkKsRQfAF/lmM+WwON6fF0w6RYHLC/wfiP/iKn/RfY9eKBBejdQWN/a/wRsc3kCrEWoQ/Rz1mnAc8qvFTzDtvezKrg6Dv36/S+XD9WIrkVo/e3Ov+NJkGZcF6SZax5r6D65Fe/PH1C1wpULLIXSIn8M2hDQBkXgvEYFF+S7UsU+FdDnEZVKvrSd2TJFw0jsHXIESsbedizuULP9gcDKGmRQq/z5JxHc8Sx/690ddpzlpreXn0Qhp+fIE9fdN4aFHW6FJLMmG5YdxrVUpeXHdh7feUzeR3vsGBgQu4bl8j9KuBnEFQZB5F+hoeqNoKdh9rHs95wkQP8LKLsjk7g7V87Q8zC8Enrk7NodpUjSVhFU02dMZn9qd0LVfOLdley+NjUSx67SO+P9Be7LcqlTYtytnbJZdXnOTU1LKaWcEPeOvOMJomivso5kpe8C7nViSs6/0SNrt4sNxkOkorLWwKqd9v+qf5Gtso/r8RNSVW2PjmCvaN2CeMweFuJvq7wJnmrdllcpKLt9twdb3BJLcyjgap0KMM4gS5BKP3X0Jmi8HkNBaDWmrVP+/BXn/dD6uR8R6wJgjWBIpZ3aE6N4a7qtgXCn1tT2TqSBJWSSYlOo3fdCUsKVfOHaPFA78/nqLBof5EpkeKdanVMXzqlgONUujoIx6n+7rkML5FFj1mbEXbsEywL1DNZFQO6CTHO3JuNWFjeM3WhdHhrfCqTRPsl70d4J8mxB0C6mqBh1ke8POWF+m2exS5sgWLpAzMbqt1wNyq9ne4aBID2SVhJVzvconVQ/axz93Jh5BkYkzVZLS8y6ObjXlYJe7F36f+xvGU8STcqx/c2a/ex+lasMmiD6x2CeTnMpjlA2+EKCXhx2VEEPHzSxxJDGdhsUg6G9I4l7KXtjO/z2FuSMp+56u8GZKm5saJDmw//hZ2rRJsu+UznW2Hn6LRjXbssYoqDucMVib+NAS/jz/G65E4CD11nkwvAJdkyJdaC/Zr9g681PEE69peUUosynp2VcLe0GMsL+3NY7tSGcbdoynZ9wejKnQQD53PK3a1Cw/aNsQybSOXW4iZd48NY1lzpRuv72/JnVrl3JAqkyiJhWnPncC/W3E9VCPwsOxipg+kmUGlFYFVTcU4fiyD7WVKWfZ/umm0ulFl8hVq/cgyNLibRXBeMKWBovMyv7QhKa3XkjZ6P+7BSrtn9SO+9VGhjszGtcFBDLYyhb0Rt3n9QRsObxzCwYTm9RKG6jpqF+pt1+ob8Ekx9Mi2sKloodNzqdBZ1Lh7VxITUOhkgytx+VybdJ2NbQqJkpR+rCyDxfU2w91giLtZXPspIXfN00xYNRXvO+JZX5ZkHpjh0wcTKdf3EewjrcdpqoXdBQ3R1+N/ALztDR/7KX9bYylmXHNHzdzyhzpqa5Xf0uDVl/dWb0RT7kmh0zMDmD1ms2mdI1M4q4m4V16sPMn7e89xbH8WDX3Fep3tM/shJTlkzOsj8tmxU2yDnHoGqNlq5qH/m6wzhfCmjzKY/U8NbFvO6wTfeoxKVrH611/pq9H8HwN9/7TOQA+VRGnTXUycsxadU9JrtS6YPacC0Wbk06XjMeF6We3Cz2meTDvWmD3p0eIPSBJWoH829MgQ8ShXzw4cKgtgcxC01axW2AJcjxMYe4GW3W7TK0Q5xiRZZtUL60ibvo4bHn0Z4aMkLMuyRGh2KKHHmlN8V6yjNfegFxFvSjz8U2ZBSSmbq3P+lTcdPCCcrEdrCckoICgrD5WrqICVLwWxosE9GvY9QiM/Zaaa3W7H725jKrVG5k++xofudVmeHlItpw1RtIyp4lL3TymTlXuHLMtIdom4TeMwnuko9Dsl8hA79o9i5Z5nmWsS8SY5fBT7VM34vgzOB01S2mTomtiexWWwNQTcy/cp7B0S3+dQmCO4daPaRyDbeZiz6fXmdros/YbpnZSZaD6qWwz1yeOHkM1MLuxJhUaJEwwfdIXXVy8geWAaH/uK8qSpBk9O9T5FRqxWeGZJgt+sJ/jTsIEsV7F8SpB8kdtR8Gco9HEVMTrsNty0ZUR6JeKicgpmp6zDu3QYg7Mh5lGwUs71744018L1COitVxKVZD8VP7/5MxqzhoFHB4rBQNffuX+5GRpjNVpEklMPlxq6r3+WBztaoKlRZgLbJJlybBzvfYoTHa4gqUQ/IFDKZ4w7PI6GIKuY+TfGx442KxSPQnfFvtRFa+SZ2GKu9vREr/8KXydS/DivVFq2TKFctjD3WkuFTULiyurRVK0fS0LrBExeyrOlLMs8aPqA2io39hx4Ca2LEjss1UexI90LldpKPdssuIby8bbu3Nzbm6alSYLZ7hJBj0yHPPRjs/KdrEs5zI07V2nSZBFPqXT/Bvy84F9pz/9Tvb5/cnfzAbXdE09X5Tv5NaicZW0s1ASGcblWWTMV4FhtCBljDvD7i7/XK2/+v6XVF+/4n2z/e9/8f+3/yZZZMwN1sGPTOZ8nygW+f78bTXeOJiMxBj1K0MpuqeXH7ip++Ggrveb/Ui+jw76vMd+89hG/P6yn9gywTLMNnxQISgGcCgRLllKuDnoZPxV8VqjMcmvvn4q9zT0W1Ibw2SMxa/DXkmB+OdUc7b5eBLsqF0RZlqjyqCLU4Mqw3GhkJ037k7YmTMxUcaZCTapZBE4LXFtQ0uwhbXrfwGAXlZdlGSw2LS/vTSMrYJFgr5J9OVADT+fCfV0do0Qy5HPD14WWdx2bmrZmqeK6BG1nPAqrKBq/m+gJf1BYmaawV8oBXDbApQhYHSJKdzywTie/eQEr3v2Thq5KJhVuMWzJvUXBijHoHrgh1ZP6/Gf5S4zNhYfN71N08zPBXulp4mqnqxQFFCmcjWDzRRI6recpN8i1ItR5SzG7ETLlD+50DKgXWHo2uoxmXy1lypi9RDvJmkk+rXnhVBzzn4tn7txluDgF+ySNJwVW+NQPZgQowcu//wU6dQVveMO1pn+BRRkEdbdkM98PVgbClwF3wVZ3+lkRepWZczfw9cdfkxDpnIkmMbPSAQy1upxKcK3yvuHmVLa3Kubmu7+wqF2NsCkPUCcSHZtDqW8J+Q3Fw9w9t/68VQg/B0Ks1bmOEKRoOuP+GHZUgb+LKO00qzCUtwpgZ1YvPL26/Pv3WM9QOvh9jfxiNQ0nPGJAQ2VWYLauM7djHKewlxPFeedVfYU2x1cSfT8Bvewu2A2qUIKqonHNCaafnxNAo3FnYUYfPg16loPtdVjt4klzS/YMxuaCxlpPTQZzBZ3CTgGwu+ANhemEXyeea3eZb13zeKcAQbo3p9FUjENLmbR1Ej1uOIHgsoZC11YM/+Q34l85gqeTk+5Te5vXvBwDd8x+5e8CBFvuE9fCMfbKPMRnqrVUUu6/lSl5EJ4sShJLkuPw/ZQrtHJ5ItvMbuF4n2QCJx5Akm20U5cpJWBkSG/+FHGPQ3jj5+mC85FoCWNFOSR8+SX5L+wQfvey0YcsfTWhzZNRe9YTKH/ixy5XijV1/PRBJGhu8VlGc+7axGDgKF0tfcxuSFarst/6YH65HonNoqbofixVumjFdWqTzHObn+PWhmHMTBGDdX1TZjIlJZiJFHJJ5fQ+JYlkz2ru9D/JptKpBLmLBJeiRsms6HaahTViZirAMh9vBrjC51e/F42Siqn5Dnk7q0Y5xgLVRlyic/hjcgDG9iIoe6pW5i+vfK42vk+WShmQk+0WtJ7V7Ji8iV0j55Bf65QR4zuF/euG8/yvU3n+brRwb1nryraJuax+cR1SPW6oJIGHtpho7ztoJJHUk1X4Pd/6Q54VPAPq3rlsNTLuYndaL5pG2G073xUpsyE9TZW4nehG/x/eYPqW59DYlPdur7rJJ41L+HDBr7zeTwyujvYox2ZyAMFlrm8rjaZSlgWYGesOycGLKTeJYN1P5QNokgZTE4ch+YiA8KXSGFaXw7P3lZKrVeo2DMmGXi6QGQ0qp/q4u8yvkeY7i+bndyGniNkbeHvyqEkVtfpa4eAta91I9+3BovcX8cWcr6hyUjQIy/+DmzWx/Pbj0+y6PVq8t0rPiPNfISVB34fiM7la8pnn65D9HahRHuwnBDve4aXOlygPUc7pWllDmzwzc9yOkBv9JSnVSiBkWnA0q2rCaH7bh6Ztu1FtUn7LdzwecsUeyqyzHfAq2SD0a6ttHL90Pc2hPicxS2KmzgPXfgzIgrm+0MJ2W7Dr8Cap1oNV+RG4BtaROapKE5kybC2auHSsYf7Y3JSZ2avLK3nfw0FokY53F4DoUFU/XtRsYf4r8yi8KNZY7uWxBL9dHZm4fSJt9U5BCVMpS0PysNrh6zKT8E7AUb+jix5G1M93Y2uAGxVx8J2bMvC1N7MDc1YPYn55BLVxIJuVdUVeS2zAqB/eZFdye/oZHips6P3pfCmcr1+dw4IXP8FUpey3R9VBdj3rSYu+DzAEixKgHtZsVgXCmvAiHpvEzD/0gcgPNaSawUclrv1qv8GM2WbhTn5vbKon1n6bmXcrfqG2hQqVVYVLqVJJQeOh5bcXfyek2oNp25/B06okc0iShNqsZtRbh3hxsphZva12Bk9dDuHNIjv79SI5QCXLfOEHhgbgUY/M1pAGQzg4eiXPVB2GrL2CvWloHEfGHMF1tgu1TxIuJBmrTaIkK5iFm84TEzVTcV1Xz2f4c3w1OyMe8/E9kRC0pd8FPn/tECd0leRYxQxlya7icOeLbB+/nTxLhWC/EjsA/+wkjFYRgATwNbalqx4+tI4WFFRk1yA+LIK1FXCvRjkWiiI+pKajH/5F/vgXiiSR7wNgpjd4V3UiTK0E42SNB6dVNZztcYEsVVshG/1Q7gH2DHXUtk1JcMpMBQx2Ey2vOMoNNPMXyUgalRq9yTHXnSX9AJp7D6XMBrEaFMFulYsX43M05G8cwdDNlzn6+CflhTY7zc+s4NV1/fg9pS3uFqX8/72KNfjWHiL9ZF9S00Uy54WahWT9MIiQzU9T4KSMgUrP2VR/fnprFq/vcyG5pky4/oixCx0yoMAKmnoSnPuULuPUg7msPO5EBjUm0DlgJx32zafF/bVCMDvGvp6Q/pm8+Mwp1oalCfftZSjgmdP9eTfESCs/pTR1iXdHJh1qhj01khfbncFsVfa7UtedxNBBFDcO5JZxab3nuPwYRw3Qb4MqhTX4ScLiWZvyzLwuZjUjqv2pKvNgp0F5lpckyC3tTLlFTWc9uMpK4owsw5vtZ9NBD1FiwiyXXV5l/9+1Jp+LE9/J6/qbzPOFHN8tlFvEoNtJdQ+ez4N3H/fBpnI6kxgKOB7ekGM1MDE9UPnMGk8eBw+hol0p91oWk2y7pbg0w1hBWrmjw00GXhbeZ7W6OUlxHugGnqNnuDg3gqSG9P7zGJ0OioRMEImrzm2er407kRAjTivsdhtv/jGOtK+m0TBHuVb55VUzY8tzvPRzMGHJDrn4vZs3M9lQz/mhnvayxc62wjSaatMpk5XRvnJjOddaLCZ2wCU6N1sjEGskJIbvGkvo2kloT9SDN9ms9NBDI43jXzs3g0cP3ktvQo0dvGRl5mujVC9+SAxgq18Wq/TKoIGEhJdXNU0Tm5Lyhy8am/LauIrzvLPzGTpcjSI0+bTieyxZvJikHRc4h52jNlu98qZ5Dz+isPU6ej1/ALskfhBZklBZVIxJ1hOXJJ4f26ZF0e1iN35LeJ+ywLq6q4FeMez963lKtvVCslmEcTLUuI+LPq5EZkbiWSHuSR+e3UbYcRu5txrhXSDWwwbw1nsT7B6Mu1aJBahUEJf3AUkmBw5W46vM0s8LGsb6CtAALpKD+P9k01tLeN3fyuCAWgb5KAkXW+/34rMX5hF7YACPXb+l0knat/vjOLpeDeD3C415VO2U5S6Dl2yhR0gFbwfv5mGWcm+QJMeZZ/DhvnQoOsWtRCWprUaOYWmpQ1HpbG208D7UFddZPKgLu++/zz2zk2xwzgHe9nbUpK6WyxXfQ527l2vuAxhV9hb+dh9C1cp7h9pymO4FXuMPMvC56zTQKBfYUrvMjc4Q3DyZ19sriecA3wXlE9PnKtVuRk5YlXOnoCKdC7Zoel7sim+5Vz0zB47ZRjMxFzIt4KITMY481y8wjrlEdeefqH1Cgt+j20GWvPk+Hc9W0FX9CI0TLjM5ox8LfdNwqXJHm6JcDyRJYpO6nKzYFBq8cJi4OOV3DjCnsyuzFZJ3NY8bjhZUZbK8OvP7nXDCL8YxLOgloc+yJFPr4tgb9lTMFu2yK+cNUG4DL5Uy4za5ZiDSsaOMv19Fs45t6KVSMVivJxpYJ9yp/rZaUhPsN4VgJ0KyFT0mnZEdw35nW8slwnU7TW+yPcWP4RX+eEuir/hf+59p/wX7/mv/T7Ug9T169XKkbVd4tRTsAcZz6NyN3O77NhXhzjXkVPy+vwOL33yPsMIwvHUiPWJficMJaHFb1JUHqNGnUGlzHGycvQ1ZreN0+gSqk75jAkowum/sbaZNOkX/o22JuC46QBXJn/KYHtzp3h+1qxLo6B/chAXFK/B6Ngv18GIC3ZQ66JIk8c7Sd5i3YB7BhWKwTyVJrBi+l9mD/mRfhVgHTpLgl2FNmN31eWz1BDzUsozFDruroFTzJFtcApuEyrWWuMHnsQq1BGX8ivxoF13AtAEnqHYCN9WSndb3mnN99xDcKsQ6K18WtsO6K4ovf5pFtN87SqOsAn+H9KFP/nEqDCJwKiFRbIOdlVAgic7fjlAj9oZQGAsuNXUHELM2ir+Sp1BlgwYaMRPt5/IGzDBXMM7jFM3tq51vywI/qP2rH4umfUhW9WBltyUZz4q/waRc8UAhB/akQwZ8Vgwn6qktgyQRm1XJC3mwrtQbnN55qb4NLdKh2ApPeygDqNvKo1lVAUv9YbSHE+AlSRwKvIp5/D6adLiPXl2mMMe6B9JaraeBZhjN8vaiUyvHaEM5j9nv7Oa973/kHV+RVaSSZXxV0EkHrvWA8z1DR9HvRgWPA5oxXS+C0TdM/iwvhy/uT8bsWRd40FYls6bvUGb1XsmzozYT4qsEaC7UVPNJ1Apo/ogC9wrhgNDcZTkNWiURlHWVwAwxgOShOk9G6zakDJyAQS/WTbgZ9C5/hUKVqOYKOGpwvecNd5smQpVT8NZcxfPNctB0uENNgBJk+f+x959hUlTfGjf8q+rck3POzMCQc845iCBBQFRQghnFgAFQFEUUxYwBA4oBECTnnPOQmSHNDJNzDp37/dAoVO/+n/c5H55znus67i9ir6nq6qrae6+17nvda1TYApoeukn7uXMp+XoSklty6JY0llL9EDoNPkn/Jm5JLY0PX17dwJ96Jya1Badb3xCLV1P+7HSMoKGHadFFlHXSYWLS0+vx+nIBzmYiK19bc4W6JvBSAB6bcFXVH6Jpwudsi4JpQfcArJKKxVVq9nc6xddvv8P7zhTF4WmZH/DgrK7E9DpLz+gqgq1KgP9oZXOuvf4Ck4/04fUkMbH51g0NW74dzRofL0wpYtXVq0EZ+H+5gAULFnhkkBkbT9E3ehvf2a6QJYng7LiDLej73ivMKl9K7c27jFWnIZnvbrv+fsuILQKJZKTGRbDwr/an2xGxQuj4pSZc2tCPgKo4Yt3fMaeToLBqKg0NlGUEChLOAOd8fej+7WNIR0X5W4BIlUSSBrxMYkULksSPNVBhh3C1Mqn6XEEVz302i+eNnzAyVcz8dzjTga4rJzLzto4zVuUECK47S0USbI6Ar0JA5VAmtS5VnsS/xLUG9e4ggv9VjaUMC9rNDlsUiRaxGkGSIM7/MuvS51Ct6ynYNbYsHvSBaDUE3lNRo63PZGReEn6Vgbz4yYtoKpRB+Rk6c3DVYOxeDfjcv1dYX2OkPOo+f5RPFj/EskqRlLCyrJJr/adxqe+zaLzcgBZrHROs3rR+eQFTl/tTVS/+rkannWtW+F6zFbMHmc/llb7MKIFKgxIUc2pC2NkAF8yuAFblFiiuiDjBt82qSex4lZaJk4XzftZDxzMvfcr1hR8QEKQka1TV52KPW4zs3YBNIyZZAgJbsfnbCURcaEXX9HF4GjfDXTKwFo2YgKwzNqdpNnTIgU8sDyls11Jc70jL0nBStcpAUJLV3KjTMOnXycx+cwGJDcp9x6GPxJwSwX1PrWNmRBFOlMm26KD3+XHxVHSHutKvUdyLnZIWu8ZKqDqU/qGDBbskq0m3wLMlkKMS59a6otV8vKWGgZdyaBp8V5pMljVIqbcoePYnzrV9Fm2kMjiWJZlrza6x+sHVbBu+3f207Gtswm8fPAImNTV60Qd+7dp7ADTtkEEzL7ceXfpQxuZF0jMP3qnwXCWxMtxERz343BJMAIzybcBHhj4hyp6VvaXRdD/enbI/RrDudJIgMX60ZBAVBSFcWjkSzQWxl49Vf/daukQok2l2bRSbTk2mV8MXzGojJgi9KGGGHzzsCy00HqRkJQm1w4vkW0b+qPjak5mCgLU8fPERTnvNueeialjS6THGVgfQ7UQ37ncMUBw3OjKFU90c9HtkF50n7EZ2U5GYERzBO5/NYd3C13j1kMgg7+T7Oh22PcTgXYM9thVIdd4kRAUra8Cp9oC+1t7keNqHbPAZRrqXCKT/HFTLLw+coYP6ebIu3JViLtU157nJP1GskWhz+BDebmvNAGMu9//pxQMbHqD7EZHosavBh60nmjLl50fxLRCr3IbY9XQujKbKv0roawPQ2lpOs24XaT7kd/E3AV8m7eJYDCxutgE89OOuc8LjxbCrspvy84gBXArowpBdQ3glU5yTz63uzrEZC3hteX8e0yqflVoloZadtD3fCk3FH2TVKgFpP2ctgXGFbJ5wm3Ohc3AfWslM5bgtJExbR3mkTrBLEjS2yMOZDMGSuPaneMXxjJcfV+Ig2nJKcZyETIc0VwVwB73SLyq7OZ++Y/cjm3TkrhpGjFG5L7WXttPaZCThVgQh9UpiAcB283yy47Ix6Uw49W7voFcsU8sk1DW+dDkpStsBVDm96aKH8iTQV4gEmAkJSznb6z0S3JDAutxZvNbHVcXVOWIzZQ3KmMRgmcfAppvZ81cfLp8SfZsv/DSoMpJYe7kXfgFKgL9FTD96dn2T253P8I6zGkmtfB5mXVP2l7xIVvJTGO2pHn/XpTuu8QS/RgHsS5YqOBwaxtZImOR3Xnk/7F6oBxxj26hS2miVVTwAS0+uIulCL/wzIZO2Cpssw7GaUL6tdiXR3a9LJatYN24d/b5fQGwbcYHurc7hjQD4MukU9ppVgv26KoXfaqHS+xh2NzlubHX0N9ygafGLBJ7JUXx3HU7mNRZQ8fBnTHz2CxpRxmiXK06yfeSbPP7j24xvKuYYZBnqm7bhtal7GGbwF+ydDIv++bfBLqKc7XUVrI2AX8PwuE4202hpqXMRh92HBCTbXc//nTrlvtPClE3wNdd+0j/wWwCqa2oQdyfPIwxQNehIupmEZFH6kobaDH7smc3FKgOzD3qQ07abGNvPRRLyC5oh2HGYOBwDh6JhiFEk8tXbqrliPExEFmw2K6shJSQ6nulI8MkO5NqUxEVJkrjo40fgqfZcPtyKWkNfhV1vCCbpvqO0GnSSmS+8R+MdQvHf8qbbbfb/Ut50m8PK8kvZvBh7igJLovg30lnmvzuf1z54jcQM8b68N2U3Eb3OknGqGTW+d/0AWevFYb9ZlHYuZ1xIP8rqlK0Dov1SyTnniq9e7XdJmDu3yOBIjyOc6d2Ew96LcB+mc28wr3ETX0cH8bBJKe+vVsOEgDKeKFxB1bFs7EYlMacw5hE+roJ3gyC3/X401myFvcarFYG3QH8TOpyfqrA59HAr6RY9j/bklQ3jUbmRpLyqjQxf9SA5yyaxo1ip1iBJsLHlbpLOtOfi9u64R/OSBEtbuojdUTmxGCqVLRHqpURml8EjxXDRIhJRnbpg9mc9TE61h+rnTt8Qmemq3mtrrFPcb5W5kNTgk5iLH+KZDRVUqZT3O6HhJlPPd6KDFELfDlcI0ytzGEdK3ufK/QvImbmSTY1iHmxRaSRrR2xlySvvcw2lv6dpuM2eWAcpBjvvhTrQ28UchoTEmjpIyoYG/06CvdTYgaaDd7ChzwFk593YVeN9N7/q2JYlvGNldm8ygkrZMO1HyjxU0e/vv59LM39hSVgjk/2bK2xN/Zpw6YcxaIsCKIv9lIpG5XWrZZncmFyIKGFnkQhQSnYToffvZ/uCRRyJFmNqlSThY45hRxTsjlG2ipIkCCxy+VmPdpG5nZdH77lzOeaXzH5QSHt6GieBo0477/fcj8qiJKUdd3zPngWDmfLTQLRaUY54dMirPPzZc3Rc+izNVBn/f77p3/H/1vgX7Pt3/K8OHx9lELrN3I5ZPjmc7LefwyFiSXBR73e5Mr4zeREnKHXbcFFp+VBTRU5MDosP/kSAQQT7jvTcgnrAUaLbiBJaAL+Fw7U4SNUiSHxJGi8ezffHGvobQ4KUPY6+runPqEIHstFCgFFkFjfNSKDdyQZe6f4GYbafFDZD43UeC3ubyl9bc2D/fHz1/gp7ZzmbgBiXgz2rSJyygdbbvHZwEJPfnI+zXOw2IctwOGcCJQ1xqK0ie3iMoRB7Mgwygte9SQFDGB0qTcyf8wGPdNvNRacy8eRLHU9/+ySnVo4g6BY4VUogsbdzKw/8OZ6cLV3Zc1lkovwedp2ESy2xH/PF4nBjPdsthPjexLfFDQb3Pkp1nXhPY7RZtNTC1GKoDuoh2G8XuxzJ21UtiPC7yxA26VN5/+o0pvrC+Vgx4JIkifjseEr+bIK9SkwcDcvWcLLQj4pgNSWS8n7KpiKG9zvPvvff5vC45jjcqsEkSaLO6UrE7a/zAPbdGb/UwkvFEYqefABOtTdXLDC6EHrcvk8BDvxe04TKLx6h6tkFpF4RQd1ASc23rc/wYZcIMjVKsCMx+SXmL1yPz4exdPf9VTj2J/sA4k8Ec/FAe+RKsQoiwpZJsgZSbkOGWgwVtI5aUgMu4K8vRfIgFiTf2YpqDVewOO4BJQzhfPPDfD6Yt5BRf1ho0LtJot35/QvG/868gaJMxW82PWW9j+O0SzRWilIrD8pjCP3rLI3p2UppJQBbA6tiSmithbV1noPQFiGLmBMAhQ4ruLMc9aEM0eVwdvx6tB2UfYYcmVfokbEKs95MTmyOcO4L9m+5ajzLoZhHqU1QsvLrLHWsj07g+uqObHv2DRoa/JXnNibwRb2dYRP2MnjQWuGaqwM6kXIbXg6A6QHXBTu6IFbUuBor9zGIwK2P6Q92RMFXVbCm9h4gRpJZXO7FzUPt0GbGUmM3KOaW1llLT6OToEnbcM78A4vaDXBTSdR71XMiK5SFV0QyhlOGwMpAUm4kE68TQbUOhmyeD4BV4eBt89B7puIPFkblEP0fdPzXdbmJtdcpIu/fh/me6mqdRo1Jb6IqJo9+TYrR25VgiVN791wpgWLV6w+tC7mdms707x9kgFvvRpw2LiWWMTijJZGZ12iwib1rft/aheobcYSWiOATwLTsZD6pgpebf+PRbpTgfBw846tM4KTZAqiyyTReTcJSJoKfL5uCaHatGa8vfh1NnfJ+WXRhfHU9iOMzF1D6zAJUboml6tzXaf7Wx2yd+yFzvMVERkH+VgYc7MKhJVNQlYp7mixD37jf6By52aME103NAJ4sgR9CQVdzNxh06MO42fzu3h6gVrIrZS+J5dOX8/0jO/lFXkRCsBJ8zfZeyMFIEwXYOZMnpoTseFEee4TAjnMprleCIRgi6Z/uAl+0shO1UwSYQlUVdNGDGjxKLPrXd2KSD7zj56/4XHWH+b64EkZkdEOrUwbP502B7PmzH6VlfhRFKKssAf6olqip1fJHgJZmKmWyrTF/Pevbv8v8S52ZXByHq6vsPSP6ftKTJ2FoIhPeQkz04XQwMewqbwZCa2+xr2Skf1NumgykmaEcF+nhb79vdmUQ+fHZeGfGMzTLzd9yQuvcOPQZrmRdrUp5T2w+7fn67Cds+Xosb6zvLOzjJYaJ5Af4Y9dYWBfl4SWqqOTBPUMYe6QZsRaxKtzoqCZOA5tLWtI7ZaFgT3duIKHFND4d3gxKj969HcZIuuXB19VwrskortYqzx0qlfCYLywfnM7nvW8K133U7OpjdbFdA7ZkZY8WgK8DZ/POggX8Er9ISOSh0nOw0ZvrYiudf0a5NZE0M0SWP+TRHnQlgdBMeOuqkqXbo7Wa8Bl61AXhHNjyEEa3Cn1Hp0VsnOTLppGbOBDrBobYzYyKrmPb+wvYNXEgen/lXltvaMHnZ4vY8sP97FopJmcSk8cw4uTzjC6ADVUiGIjDymM1L9De0ZSTNnEdrGu4TWCr8VQ1mU6B+R6QR+3F7xmfUFDWnC5Dj9Mn+YDiOKekoqohlKOaUcwzLSY8SCnB7y3V0bzLVdSVjWizsoXvNcgl2GNqCaoIosXVRsE+wbmLLnp4tBgsag/+YMlBuhX/xjcZ20mrE/cGbLUESTYSNaC23z3/3/5URafrhPa1obMogfIo54Z//t2yg5iA+eVQR2zfTSIhK5GxskjuWqaTabN1GL/UJGDwUGH0uNWfjOOtia4TK+gBzteHkW6BmZcmgdrNV23IY2+Uq/+iewVFif06ZYHXSJ2+lu4eeqNn3AFjTdV69JLyuoyNGZgSZaZO3cnrj63H3qhcB3tbD7HHz4tAlQmnl+g/B9tv8+kDZ7jQ6hIRPi96/F1/jwStGIeFmtawKLyaE41Qp7uHPGNr4PNQG5p+x+n98Db8fZV71lVTLaWDDnJuahpX2k8jLFxJTLhJS7L6H0S2y3RpENep0T4/8/j8FTz73WJ6Gy8I9uqAGmyGRnwjS1E7ReBV7bRQbHcB0pJR3BOzajsD8EibNxSfn1Z3YUq5nSsfvs3ZsRsE/3hOeRSzSiXK6qKpLF8mnPdluZIrYzZjPtcEQ0lnhS3IEECvbsMpkK1czglBpVKeu95ezaHm7XhuhC+Pt5vt0YcYmA/Gm9DsRpBg1zkbaGmsZrgXjPJVgviv5k7j99OJ9DurI9Di4X4az1Pu6+qx5qln38bidiyvdhEj3df+EEcOD3jBsHyYnC9WOM/mASYXwWRf8HeK+3zLwE5M8YFP465gtrnF+l4J/GK1surSQrSSUg7ZbDeRE/IdrbQw2gu0br2+E81nqU9xsuznwcz9vZfwvd7Vs3k36hneenQ+53bNF+zNvT/jhc8+4fmvP+BixQbB/nnIacZ6w31engkqbxcPYkYxrMp6QLBJksyOR39j75xPKQtRrpG5TcIo+WQBxi8XUDTGBYr4+foiRgeeRzEQ0ODDI78+gqZSGQtpnVZa68ArrRVB2bHCsZIs07V/Gm/9toDhfdaJJ5e1zCqBMDUsDhF9xeqyP/kpDNrp7vp/f48BRhMlfY7R6FVPQKlyrZGQWPbRROoqfFkx8ybptcqK8xZt57G8eg+HRmxnT1j+P+/Bf0fetIsMF47jkegRYWiPzs/1HGLDKgT711YHuedT8Ck5yJHiuzG7SgUm31C8Km3krOqP2qnM+WhSl/F9yVvoBh3hir5BmDuHY0/yzKN7KI7xQ1KLlaJOSxUqczlR9Vdw5iqfh0oFT3SYzfgmv2K0xAnrgepOz+Y0M6wpixT3LJWGSgeYnSCj/O6A8CqKp/2BfsBR7ut6DZVbJXyv3GBCykK40PoCVi9lTCDLsC93JDnl3lwMK6BKEyfYLZJMSPurHElOpyR4gGD/e3iUxzUm8P25pejVdfjLbrkAXSCFdvgm1CUDeu/hjibPMGFtDdfKu945t/LQdEcbtv88gk0HujFpXSVWoxLEfEy7hbUFLTixtj/flIpqNrvqQzjWCEONECxXKWySPpz3rfVUGho5lR+A0z1vAwRIZQwwgI8Meq34LpipY1UtvFIKqnskWYsKD+F8wRWjOLysmNyqYsPVDQw91pMXTvfi01QlWUhC4kk/aJ4bwxtvLSMjR7lPh0SPpi7ARcZZ13k5jSYlOTym/hxz/GTsJYEcKxL3WclWy0+hEs8aNdSFixV0amseNS1z6aKHLXXK9ej1VCNdv17IsffeZUOgL+Hh4cydN49J06/Trt1s7lP/Z8DvJDBSDUNajUCdkITdbR/fXL2QY92O4RVQw8JGMfBofY9sfo7zP7Dm/48Md7zjf3L8C/b9O/7XhkqlIikpCdU9Eo1WWUeWTzUvRZh5x1/s1fNe7TNk+DzBaMNLlJSJC15WUg6yQ6bJhaMe2XwJSTuY+/huencQnfS/R4oWrsYh9OyTJMgJ+Y5Boedo6b1aYQsrHcCIT58jPrCICfc/K5xzQPdfGfb4ZvRhFzhfo9QDlwLa8OOFJZjtRiSnQ7juVKmIl2avJenpP8jyUInjZyskKDMB2a5C7aFkXZbhr/RX6Bf/GxGVooynn09fLhb3ZVcUDLQqZXf6N3xDlxNdGLFlhABIJNmuMeD+I8RcS2Hkj1OQVcrlpFSOxX/4QUriS5GCxf4BRtmJb09XBYPD5nbdjQWMkFrzwCu/cW7wXiQ3yR2ATsa9rImAUJUYVAGsS3+VkX84WbDrMr5Bd5O6NqeF4836sLrWJW/nfr/v9yri/j2DAPhELy7OV6u9ibnSnJomx7hiUVYgqUyFvBvq5GCsk/3DRtDo1v+Imhu8Fehy4J3uSdU7Y5qvC3j1JO0R6xVH1xLX5tmAMlqUkUm85dpMt5SKjLurvjrmbxtNbeVWrtYqK0ckCRp9XMm78IsHhGOdsop2h3pyesX9rLopVhvEWq6wNARS7AkMjBor2L0bD/PBoF78eO4jymPFd3B1xAGcyfBum0+pLLl7TyvQcbwAVPmVpO5bxJWrRxXHNbfsYc2Z/rz+4Rx8asVnddyk5aXgTD5/ewG/tqwS7D/aHZiSsqn1/5UL5W6/22lnsFcj2xpgSrFnZ7l3wCmMMnTJBbzcAj5ZjSkzhtavvsnDt5TvQUjtJnyriugw9hjfTryA2u3cD2X9wuBf7Vy5FUC1Q8lgVzmsvGWz0Oaii+kvuzmzWpWGyPxInlzVjdmn3KQ5gO4x3Rn9xxrenryA0ouvCXa84nis2JWMWBkuJu9z5DZML4b3KuCSTXltGruGUZtGMWH9GCIdZkWwka9rzSsvP8n3Pw5lchE0aJSA3rCATPq/uRyfjGTY3Ff43klRLrAquLgRXZ4omXY+4iJvv/Qczu8monOIIOUuSyQvlroIHTF1IkN9g0PHewO20buqgcbQu0GTwZbL623zaR5ZwYM1YQTKSn7ldmc3rqVco9mAU0x7QARPTb4WGtteIbXTVYIN7glKmQ8vJmKs8+b+sdvIqhKZcbutVoy+dTw+XCRqANQZ0hlmhNGxYpUQwJU7sdR+kxI42BJr5Kf711OxvyvvH2orHJc4KoiYMWfQNc2ktUEpzdTg1YS37mmL4dQok7pNpFKmRjWQrW8g3yZWhti1gVw65lqTs3w9JEokeO/wenZlTkfjrBLsepUPxTY4V++PfM8a7dCH8n7KeS498z0vfPYJLdzkvwLkatqk5BOsb0pIzSD89W6kB9/2mH36ElUQRXiumHwP1Zyne/3vjEvPRFX/s9Ioq7geWsHvr3/Iprkf4nQHs4EHvU9zIgasyaAqE5uZP+On4vdweDpGyWYNsl/AlKjhI9ujdLt2gCCjEuz7oiqV4wElHAwoZmm5GDw/ccOHky/OYdGTcyizK6tl7F7xfJzpR+Pa4XTdMBJJEvel+qgE5rz9Jh2bbBZsOB0sD4O3g+CTCDG5E6D1ol3tKOLV4CubFH5fRG4oUdnxAJypU76fsr2RnW0q8R1ymO9mfEeZW4+LIwW7OZfwGCc7n+RUXKZQoS/LcKJ/JAvnLiLdKSbvJ6gziQ2ox6QxYNI1E+w9tRUci4FxjePRqkW1hiDvBQxJ+hG1ugruIfVISPTb1483Fyzg64sOrKVpiuOSpJu8ktOMvO/HYq8Q3xGVpObCpHW8N+tDVB76MoWr1PjIUKMuwuw0C3YJicVBsDHCM0Hlu7I+zCuHAXrRpwKXukWpHf70e0TxeSQbeaLva+T3GoGt6dMYNUqVCVkGs0EmrUMatZFuHHRbHcujizkZC7/1m4TsJrunVam51tSVPDnkJ4JHktpAZkM4G+sh1+IvXrS1hm+7LGBLyjmitEcFc2P1MXZHwe0EaGG/h/Sj0vNJjZXKG+WcPtOTbPlhxXFmZwKrZo+g6M8uhNQMxs+tmUpd2HQ+91uK3dmAquQj4Xu7MIOX71tHQotMeoeIwFdt7DfML3S99yoPPuz2Ohv3F7iSjB77nfTfTefKJCKzoCTirgR/pNrG3PSBdPMr4LnH5lNeriQ67bQ9gV3tJKTlDab0FiVq/Sq8sct2El79HVUz0d/bUz8PXUglR1YPApW4vv+WkEHBjJW8ohXjFYD3qqtpfhuW6/8Qqquxm+lvdPVfnOSvTIK3uTaeh48cY1FVCq/UiKDC+Q4XWbBgAYfGTOGKZp7CJukC+K1Ux7ovx7HpYCscbjHFTW0qn60cQI/fmxIdI/bDbFSFMf9oAo0/Pkp4mYdknATj1tTx2MYc6rWiNFk+oXxY4fIlLV73sP4dZp4JsBMyfif9m5xit1UJdufqUxlfBCd9NdQGi359rtSMb32K6PToZhJSRR9goLGcXgbIsajx1bqtc+ZyFgbZeXTZB7w97zfUDhF0GKU/ytoIV6yk8qC2c65sHGcKhtE1ShkL5asS+b0W/mxw8l61U6hGz3cYiTzcFZNRhSNFXH9X16nIsIBTkkTf+/Yq2qT70u1EN5rcbIJapbSHmLZyOvU8g31r8fI/+h979jU6wWaJFOzmgIF0OPEcMVnwUtF4hU1GTaDJSHCNTBu/b4Vz9m/1JIuDYJQX+ErKZLEsg8Ecz9IQWBemF2LPLpzlpbM9mPPWAmJviUSPQYmT2FAP2htwQ3ufYB9j/p0V4fC4H6hlNzKRJHGuYQfvDW3OV8NTFb9Zba9lb5CO2mcXsPDhBdS6EQjr1SH8eNsX/cFutDqjrMoGqJd8OLSrI7Jdhc1XnBsPF/ekWU01AcWNFMlitdenVS0YVQD+mSKwBa57/n0NfJ89VLAhSfxFA5qgKpoalWB1bPDj7Ln6NnPK4JkKFxFv1EMPsfI/7H3u4ye1iiRa4R9SiZ+fMpEth/am8+5E8pvcIrPvEWF9llRGUm/DsUaot/mLJ5c1fFENg/NhjgfFJl97AVN94ZUACFcpSVZfh5cyTq8iPiuRZ9RKH1eSJDa3TiOt5zHaJv9KnVms8lFZTXxXYODJIgnVnbnz35E3fcIG0VfBz00hCMDh8zwbhw+j1/IFfN3pvMKmUqn4piCAlWPXMmn2d7QpvUuCLavN5On79Fju28jP49dwztHJ7Vio8/Pld59iFueKEs5HLH6crTDQM+wJZJ9HBbu5zSLis6F7LlhHuvV/VsPrZ95hd3UQv471x7tYSchPuLmUZSGwug6ey2oHOqWiQsfgVoyons4gI3TwVu55Q73S2RsNuwfv5slmp3DolRV2z3a5SO9pG+k7YQ99/bMVNkmCLWd/pcaqx6Q3Cf6cLMOLNwbyzP1r+L33AWw6ZZ7NX7rMtTiXXPGD3mKBgyRBtM815vSYRIx6h9LYWEg7gx9fVMGXWQMUa5Usw/n4KVxv2YluzV9B51T6XD6Rrfhr/HlirgXQ9MgnNNqV8Xiasw0bVw2g9+HeqJzi4qyR9Dz35gK6zllAG5RkI9kQzqJKyKnVcbhKi0MtFnV0kI+zJxreD4JglUiyauHblL8i4NMqUN1TFV5+8hneaPUiVX5V7Om+H4tTeez3Mfu4rzGA7OxwDlUpiaI6tY4nb7Wh6e8PobtZQn2FkvAmSdDg63r2Xy6ZiMOtdUCgKZvpLQrZ/eIXZLQWixskjT/PrOvErTmv8EOVGM84ZSPbCjvxcBF8XtVSYTtWH8Yxq51dVhuZ1rsxh0oFQbFvUtYfekjQWw2/AXuAX4HBOg29ZYn7Onal46WOvH2mmGxTleLcqeqDpLa5SX2lL8duxAvXFev4nYEffMdnsz6jDLG69P/K8IR3/E+Of8G+f8f/2nA4HBQVFeFw3HVKfSUz7aqCOfL7UE5kiCwA3+I6Bv02EcsPE4ioc2vC67DzSYiNFn3SoFMNNofIMph3sw2/fzmGVQc995J4PCeScYXwYWEs7tNDdpr4NhRW1EL7XKVT2azRFawU3YjknQoRcNNEFXPxagIbXn6RY41KyZKM+kqWhxxB7SyhTf3P1FuUzM71dCa+2MRsQwh5arEhcsuW8yix6AkIrSDeV+yNp1JBo82H3y6+TZVRdJZrjf1ZfGQtF4r7Yb03AW9rZKaPhWE7htHpTCciZSUfrkgbz5qeBylqewnZvwbJTUbrlrozL3Tej+6FrzBHjcR9PFDUlF0jtjL2m/dpsLgxO7X+vH8jjm9enMVfm7qBWmTvnGkYyLwyV/87n+rzgr3KeIYS3+00apSAhY/1KqvDXc2+3xFzkwz0KqZ9tyuc6nSKEtEHR9I7KXx0De+PTCPFzRGR/FrR5ZI373w7jO+OJ4vJtrpbLAiCtFiY4+3W8fvO+DoUdkXBiogKcCgBZ9+G4xzvsZh3gxBkz34JC6btOztZ9Poi0juKDe9XF/pTebwtk3+fjMamTMbZGrOJ7XWEqq5n2N/0Mla7cu60NBiJq3QlvysiRPbvLZ9+jCuE5motPk5RcjXdKjGnDH6MnMLW26L8zHlzENV2eMgHvBx353W1qZpfJi7k8OiNWALzyWlwIwBIatR+degajLySLSZoVE4VE3+dTIuDvbAgsq2X2DVoa73pczMOVYNbc2yND8HpTXjiju/uKXG6ILc/rW67esQIw+kgtt4FRGwoUb6/F5vU41z4Kz5t6wm0h+Pl1kPAHFmGPvUWiZcOcfuSkliglWBaZAOJPc9RP+M3dEbls/SrT+OjglR67hyCJs3NgQfU5nJi/V33uBxxXv6dWPmyGl4vExMO5XI8P9S4+l328FEmR1dHViOnZOJV5c/TuhtuAYKM7JBpn9aeLie6CAmcgT4RTPSB40N3cbGtKFMx2FBC5yEukM5ZLCa1zM5GKAriWlozajz0R6pTBXDW7FrVVW59UQGcjnuA83sCd42zkm7F0RScbM3Wn+6jidtaNi15Phr9kxx3PMXaDPG6XjJU0LddJtva9aE8yA0Il1WsKmlBWb2OS04LTg/v2InuN2i25CNKbV0FG0B371rW10NklmfyQLkdPqqEExYlCJSi60OjVyDrR6/ndoLIEl91+xc2J/rw2pu/EOejDGANGh31vgYWLHDJprrLj66kB8m745m5cD5Pn/pUOLfTGMHm+zazaeQmHN4e+n7I0C5iF7O7TsXgFAMfL3s9tQ549nYTZP+7gU2kTyQjzu7FfrU7z1bYyXAq2fEp1gx+UQXy0OoI+l16HezK5IbBmUd4QgG9n1vFkx1FQtAwv27wZweOb+1Blt2Nee+wkmq0U2Ns4KTDhtNDEvyIqQkHG+BgA8geurA3D3HJC726XwkQOVUBnC8cSiuthgdTP0ZyKOe8hMzF1hexqW34lYubllXrWs9lm5rWXkpJKbtfM+bqSrjw8GpOzlyB2q1KojhrPd0G+jCxEOZWuvlcAJKKIfmwqAKWlIrglab2LGc6rSIrAV7W/KTw+6bUudbkC60vcLyZW88blY5VhnLSBu+lIKpAqMY1VR3j+aI6upzqgtO3TkicFhXcz/z204iqCERlEUOd6A5TKB9n4uchGi5ZxQDMGDyUIznjWdRuMT6VYoX04/63aauD+IJiiu6RfVLZ6ul+Ry4vbV9H7A3KfecGLfk4PZSME63YtUQJqAFM8TnNJkK4uK4/jnwRILoV7WD2mwvov24rx4vcZBLN5RQ2yeTVQGij80xQkVHxfhB83VKs4AcY5kxltBfEu6kKHLAMZtqO94httZvx3ZcIc+d+qTMf8TvTVQYSNW69OdQ+PH6hHe899RJbPx5IRqlybhks6SzokMe+DxawZ5BYgY/dTKS+DD8ZJA/JIdRevJsxnlA1TPe/JphNuhhO37ncno2fKWwX/N5l5UNfs6VnCQdrlEmpS5Vp3EipICT3HF3OLkblUFZunyo5yOXYZxk090cen7ZJ+N61tSa2R9xm731mSBVJgPX+I/CS7XwfCjq3cwNUyj4U2iArHlLKdwr2R9Y/wo0Klw96r3/S2uhFyL4WmHb2o30OXHQok6Nbq7dwrOsR1iSlM+C2mDid/EA+/u+v44fMP1HpPhfsxyqf4kByAVnxWVjcKm4BDpcbiFz+CJ1PefCLALskJuD+GT5JJNx0SZrGGZVA5E2LF7UWFREVD9Da/JRwaEGilT1RsK91AutqlFKcTmMMj1dYOdbtGMvDsrCrlXMrW9uMY8YqABLS3ORxAZMmBPWyKYRdTERfKRJRAcx2L8oaYnDKYryyvTafV8vhuhW8NPdItmr8iL7hzbcHWjBo1yBkm3LOShJMWTGFAWdkWiZ8gckk9o2MKIxgToOJYWaxin5cYSfenryAXTPmoVUpAVAslcw2qNm+vhcD9sVj80BQybMn8W01aCtGYNCK8qUAHx5dzWMblbGnO/jh/v/ekh1ZYyPWt4COgWJcq7aqSQqsZ8qcL/AJ+UNhK5W8OJLXl8AmOcztfxW1m7yuStbQ6IR+eTCpwOCR+PuD7Sl+ckxncuF3AthXaSonM3wJeTYotfsrbA/7FxE98iDXF3zECllJggKYEnaVVwNhQyR091Lu8bIMY8PT6KiDz/Jbi73vAh5n9y4X0axzrRivDAuOoR0RWAFZJdrtkhdnTOBzE3Drs4zdzJmakZyVbrO+TlL65jhI0d3df90rtir0SXx/wNU7q7GdqKJzXTOC05V6zPpGnM3EuV1u82X4d48zYssIj6SF9Q1xbLqz9Hqyd9bpmaCJQWMX90MJic4nOzNs0cs8b1f6RUnqVNpcHkzX412JKHQlmqfPmMF+q/X/kXTdQbuDwLf2k/fEbdrFK6U0tSotvjW+xNxK5AOjON9lSabd9kHsnr6AjfvEdgp/j90NsLde9BULfQczON/V67il5rzC9lppCL8lZhD19ufciBF7BZ7ufJqE9hl8bQknxKH0n+Wai3Ta/hWvfvgqsz6fheoOSeq/K2+qt0CgukywuXq6O/isCt6rUJ7R4XDQ80AXHlvxGCYc2Ljra0rYsK4dgP9n08iLzhN8l+r8XxgS9wFj/xpLr5NThTk7v6Q56tdfxW/pTDpUiBK4f/dRtQE7sw4obCoV/KneyyrDas7aqnFqlUQ/75rLdLvziEOrh+Ger9c2XGZLx+/ZFQVfJCkVhk4VJ/PKd0Mou5LIOTPIbrmsH9UqtrY6x6z4WgYYRYAI4NLsjSRPW4/RocytuO6Bq47fKIHKTS1EIzWQov373x5AdFM+D6QuYV/WI+Q5lH3COfcKadHVrK+HPTdmK9eq+mzax/xKatAZ3mjzEXbbp4pDRySPpVX2h+jNKixN93OrXElsPOXsxA+j1uMzejdzo0SC7L6ou2D8NbMyzlJJKtRWNefbnie9RSEaD+ug7P00t6pTeMYfoiTRr8qwaNnbCE5ALd+9L5e82vL+tDfwr/bnfNvzAgHrXPVTVDWJhRofarcqf7MsyWzZ+wSmSgM/zNrJ4VilP1hVk05oC5fKx85ml0GjzPncCB3N0HXtGLpkNi+YRBKfrDay398Vhx/zFXNw6IKZe/F59mdPooNO2XP41cL+rDnajPs33o+hTk1RURHvLlzI6h9SOb2zOV5pYG8Pt1IieEbWMAaZF9TgaB3Jk6/uIDPkZa40v0JZcJkQh91vvMbLKeX8MfEP1sSJecfxZV0YaiqgMrASP41Y9PF/ZXjCO/4nx79g37/jf204nU6Kiopw3iOXOUB3hQ8JxCk52KQVk4CvZfujKw/kXL8DXLiXHQmAg1ElkSTkRzNj6HcUN4iVEDsOt+HG8dZ4Z4hBEYBvYxvW1cHvl9529Y27Z0hOGzP9oK0Oih3KxbZ/68WkdrpKSVA5h67FC+d9uLSUb4KzuND6Ak43FmJeTR63Q78hoMsF2nc8R3mj0oEyS3qa7utNv939iJTE3gZqnRcXR13H8vRKaiQxcJdleLjVfFSyjWrjEMFeZ68gV53FC8eWUR50TyWQrZb7I1zym807XyFOUgZzTq9Utl3sRtsBp/nxlfXIbgnIvuGjaZv5C/MDYbivmDiVkPHNjKf8dijlVrcKIq0/v1QHYagIJDEz0SPQonJO4rgJpvqCd2O2YG/feio/DhzO0O69qa25K9mjd5bzoA8kaWDw+VIh4FpU0ZIXBmzh2uhthKhEJ/1W02oeVOv4cm1PChrcmvRqvGi4EY/zUBeKv3pITLaF9ODN/Tuw2jUkuiWi/x4PZIdx1Qwj/YrArfrPpoliY+ZDTPfyZmX4PoUtSHuDlNBc4rwtBKlFZ+E1jRXnHfnaQLsyYZB243Pe7v48VQ9uYWXrU8J1T/cu5+VpO/B/sJDpYUopKwCLNgQ18G3CNbwbxZ462TaZTyuhtwGCGk8L9iVV7QjIBM0NqA24K3+gbcxjTxR8PeocSz9YTlC40tm9aRjImORTaLwaCbra1P20TPM+TvLNZLqndSCkQZzzEiquD9mLffxWTO4N7e/8xSx/WBrsOXFaZ04mUQNv+BnB6ubomCv4IcaVZD/upbzuUxYt8+pzWPn9YN5evkoAS17wsbG99wECKzREVyvBUZXWj9Z1VTgmb+TDvjcIduuvqFGrSejsYnPqtB6CzOpLzHvmKYKeCcMRIoJ5alM+K8JAL8Efdf6C/W9n72aslvGGBQpbS4ON6Ad34j0qg+v1IxUBmd5Zx+05y5AiSxhxogfedmUFx0V5KVMW/8lTSRUsHSCuF09XtOflrns59/on7IsXg8wjt2dzefx6/hz3J5IHKrdacnLK5JKaLfTvLdgvplzjTIyLSautuxssSgFtebDgnnVVpQRTQm1nSby4n4D9Z6k6L97vqMPNKZozh53SQn4pExOn/QM3sWzqJl4JvSn0bgRYGtDAueMt2HhGrJgFeC0AvgiB8v/gQ87cv4E/Tq7h2WhlJdpz1io+y21Gy+R8BiWKe6VcWU7N5TbEpbXnuF0JcHYyatjh3ZMpRl9aaBGIHg5JzdhVE9BYVHTzEUEFWZap63aWVn3TaKL1vGflVLfgt0sLMKtEJnig5QR7ouFyqzOopbt7sbYxn/H1P9N2f3P8v5xKWb3y3ElRs/jypVmYCwPwUZVT2ahcBwPrv2JU7kEqLjfBHCAmrT6pDKOq4znSm6VTpnabO6ZSjofXMev3h+l7pIdH5/q8JY6++dA3H/AXqyQ+Of4LD6w2cbtaycw0aVN49/AmVJKNKW3eALe9dpJPFjPqwuh8ujP9PVRnfBTbSMnrX3Cpx3A0vsr3928m/fom6fwq1SG7vYO62qu8GADFdrhp9aAPKUnsaoC55bCpVmSc1ss+fFmh4WADnHG2Ufh9ozq49qPgslAiZGVPMkml49lSsJ1rzmfHBxDi5iP4OaoYlVpAp8En2du0XhG0A3jZLqP6aygzPp/FnOoq4boM8QNYXF/GxYjF3KoVJQHt3s35pjScHKuGGzViP7aFZamMuzNtbPeQcmRbDe9O2Yf3kvdZOnsppghl4qhGCubHLofJSsijMVApSw0QKDey97sxpG/pzYlskbTwfZGLQKKyaQi3ZyuNso6bjYN5qgTisz0TVHr7HiJUBXMKPcuIr03MZH0k/BapBL52nN1L7EorQQVHGJ64FLtN+Z4dO+PDqR+HEz3/VTqkuYECKi0ny6Ox1fhQdj2OknplxYvGls80P1gbDi/4i9dUWXiAvd0+pioJZgaliX+g0jNf9SfjCmFZlSh3bNeGMLEIvq6CI4Hv3zXY6jkTV8fTeSkM3xDEjWxl5Wqd7RaRz3xOaNcLRPtdo65BWXldXL6XcRcs7DqeyvcasZp8n6UNJz56lIDlKdQh+nsObPTQ+jLND/w1YjJOQqLWAQcaoVEjPq+4mjSe8nMlhI2mu/PD4t2Ook4DqbCpafPNVGxuEvveshlTVCH9Nt7PwE1if6MR3pUMbPAhNlODvdZDYsnp5Hj3q/zy6C8e/SLJKVHe7iLq5GzxWOBR/1qcyVCfBNjE+9Y0LYv71+VxsPQrxefLYuGTB/+g04lbxFz9RTju4uXHOfvFTJ640Ik42a0njkrGprHR7lw7OpztgMrNR1DLMufaneNyu3MsFXPvyJJMY3wOiS1voW4iSp9KEoxvvohJLRcge5BwliQYYICjoYEkc0+/H0mmxK5m5Jrx9DjWg3A3slyUJYukqiCSj7Ug4Kc8bhQp5aM7SoeZfqwPQ/f39zjf7yW5uauvYIzlk5o3qNw0gIiDPZDVIph3wdaadXXwavQlpDoxMdsxdA1PdXoKlazcGwZYNpOfAN+GwpFohIrxpYGXWDR1L727HcNgWSqc92SAltY/PszZw0mkVSuVGK44/Xj6Uj4VN2O5XqdBdsu+V3kPo22Oa96A02Nl3yMpPzC16fe823uMEAM2NtziAS/opocglfL9fyD4T54yRLG8Bi6YRHjksYz7GZwPL6ePxq5VErRkGVr4FGCUYVO5KFVvNrblSJf29Jn5F48OEOGogEuj+SuuhO56MHpQPMgIm0unXFfPS7V7EtxcyuEYKLHBs3WZinuiMgQTU2Dl9/cWcOiDBaiMymelkmSyErLoNnkXMwaK/bLVsoo9j/3Coh8+IMFf7OsboTYTmxtLpzOd0CLODb3sTTsdDJaj/5FMvHc8FVDIqvhc5nV6WbBJksTIPYOor/bmwgVlywPN9TW0OL6ToTuH8sJFV2/S8PBwFn/wASN1uv9auk6nI3JQd9ZKsLGkpdCmRGWpJKnneYyzVrBRJxJcNZJM27ZlWI0NpCd72LPsFm6EJfGD6j7G1u8VzLJKyzkzjCuEdLuSNLGpzoewHyaR/tE0zjfOVNgCDAGcDBhB3JZhfDNvJhWyso/ntRsr//l3VNvr/ygi/HflTc1aKLWLKhJ2y26GH95G+1cWsCdDqfzidDqZ0rYObYfLPJSj5VfV3VyCrPdloa6ErJQb9Pd24C8r42lT7reM5DwAXWNtotTmPSSg35ziXqmpOMMUH+igA3uZ8smrVGBVV3PODAPywRyirF5tHHiI+dnfMbfxaZYmnhVIP05DNEuze/JhBXxaoJS5za0Ox/tgNx745VEGHO6F5JbTuVnwLrkr3mXujMVkVSljNFmGyR1msjvlGh8EIxCpZRlGBGayMhzqm0BksdJ/qVZ3xvdCS6QbcMk2W7gnkq2KrtGbKKpLpE5y64EbPYpv66KwO8EpWZXkgKLtbIx2oJJgaD6ct7VVHOpT8Dmt078H4MNpuzG6xfpqSU1ubC4jRh2lo1EE4xodQbRbuoKF8xZSqFUSb1QNuewJUzNsxzCeu96dSL3Y69iq6c17B3bz1oHt2H1bC/YvqwI4b3b1AFVx13e/ZbhbYX463iH4NherPuBk/qtURLbA4S/GraVdJ5P52G1udz5ON61Swe120TYiM1zVBTeSbwig2eCmI9FaxhEYVUKKVlwjNSoNAXzCggUL+MjpL9gBLsU9wbMpf/B0wFbF57IkM3H1RNqfa0/i6RPERUdzZNEi3ivPYI2pkAfKQH0Wmlwt5CuHlfU4+MwGqgu5LPtgGBeLptPiagum2/wFqe9v67ozd31nJq2ahMYq5i/q0dPNAJmRIYwUw8P/M8MT3vE/Of4F+/4d/58aNx1RbK8Dk7EBq07MXma3zkF6bA07uhzHZHBjsEtqpl+M5vzB9rx9y0+QdQL4s9s5HLIDvSQGmAAj/F39VJxuTEFwATkBt+CzSlfD6HtHjtyT9NPNCS8OJzpbTKCnnI+k7+FehA04SbRWufEF1FwmN9CbLO9qHvLNQnK77s7+KfTf358WF1UkXlosnFt21GMp8OfQji6UNYjMCUmCrtEbaBe+Ew9qVBSUfsnkvp0oa5/K7bp7WNEafz7IW8jF8Vv4vuMx0mmhOG50kwlM/WkIRR8/zvxykNzkeILN+xn9SyafP7qAD9M6Ct/7gFcFzZc/yr7Fj3GDloLdanA9g+7Rlch2cfPTyUZO1PoTlgmNMQ8K9gF+ZbTQwcuRmVRU3+1XZfIdiHzDBfbN7/6IEOhF6kYTcqQH0999jdnV4ntypK4tBzKi0JxrSbnDLbHksBNueJDc2NtsHLVRDL41PpwrGsKYNRbW33pHODfA/jpvWubA5AuHhD4XpaoAZhV589qtXpyxvKewravcx+zNz/Dd2T48acnBfUiSmkOjN2F95yMa3PpClmnjmb67Gebf78Orzku47mrfsRxsmMnzo76ln48oodgjqicFxffxVFYqF6yiRIpaJWOQYVkotLGIx0uSywW2AZp7KmIkh4XoonAOn2rKrEKZelnpYP19ncsfXcEHA8WqKptkYNJLv2Oo8qdvrvgOrTKaSD3TnpW5vtSr3NjcTgcDvBp5I8DVP8bTCJAsPOMPryQeAPd+a2oD22Jukf7OYi63VCaT1XUSTa81pdlVI17FSkkRcHH1bsfk0uabtynp7QbESDIFNi2/7mrHG299Soyhp9Ic1JcBjV5MW7GA554Sqw1u48vs/HAmdH6KpMCp4nfbapji66oMCVWJ/ZFS7Qe5GQfRGismlPcsu/8JNlRsYK/jR84Xvq6YWyG2HLZEwuRn/uLJ11cSoVUyHGVJIvHSFXJOt6DMLromtWi5ZnewRVVHmcZfsD8ib+PP0RcI7HgFrQd5uy6WPTQ2gc56z8BtZXVf/GX4MBi0dXeTebJKpm2Gq3dnducHBGAroH4TEfGuyji5RGT8L/OVqPOpZeCBfujq3PY0p5OZTQbzzMZ+TP9+usekVNvMJJwrxhOW7bnP7MfFEcwth9ayGPQAJNb3ZWWHT2knfaf43JZfyLVNTZlQFcEbwTXCcQE3n6P3DhMj/nyJgd5KOVjJVETn4A10+vQR3t9zHxq3BE4YtWQO24PP8AzsncWKRIO1nJ8zutDp42doXyvuWbIMj7d9hc6Rm7HJIgh/3ebHJTOcMoFKfc++47DirasCILg8GLVJ+ZwDwu7OlVemLie/URk8HzF7cfJmBHuvR7CiWKwc+bU2nE/v28jqiauR3BOnGh/25j2DJb0JffcMQu0mVQjgq7rrG3hisNudGnSqBnw0SjBbkqAgYA2PSz/TIVtFrZtqwQCvQrqq1XQddoz+zcXKppl+8HHzcr57fAThTuWaoK++xPZIGHzHnXJns5pjHiQhCy6ZQeuhItb9Ot1HvlPLc+VW+uaDI2S5wjbc5o9FbyKqIIKXbW4yWZJMSEkIzX59kMqdvfBxCzLzDS3oZyvlxPVI/jiRIszpTxvbceBiPAAFfiLRI6s2g2ahe9gcCTF28Z6VmYr4zfcLUvJqOOoQAed0iz81DpjpCxrTXWDBqQ+jQw58YjZT41eDpHGvcJFwyk7Uj52h6RARVfi2YTi5JX6YtWZKU0SgZW69jmNTfqP21a+oUrklmzXerCn/nW/u5E88rXNjvO201sGKKrEaAeCTa49wqBH+rFKe2x5g42T3Y5Rs6Myji8dS51AGrR/FlXJswF5MAZXc9kBksgT6suypZfw09Sdko9KvMfv3o/fvnVk2ZQHt9vYVjq1X+bKq1lWlnG0WK2L/Huvq4JhJBCwkJDKt8HQpVPkok3EGGYJCqkjoepGR4ecVNi9HJR8Gg++U9aydtoJss5IAkGy/TrP0VOyn2vBLqbj+qmQ1KruKgAoVOdmHBbvmUjv6BGSSfOZ+ooP7Cfb4qmOcjoUvq6DQX1xHp6myWRYKv4eDf9WZfz53qPy4LN9HSWAdBosOldueNd24j9fL4wEIaxArgf/I2s2vy6fS5OwavAp+FuyPRyfy4kfjeftsLzROkRSxsC6KoHOtGav3LFl0/x3/80Aj4F5xYKsnsdVUapOnUiYrfSMJGYfsIDSikC5xSqk1gMNnJtN4MZKI9SPorVL2qA2knnWGPoT3OE+L9tdR25Vzq4P5GBvalnNi7EauJ4hSssGOUh6Y/yNHp61kVa2HCmfg0TZzeajV2/jYRWnUplxjTzR096tA5bgnke10MLr2JZyyA9/AatoalVUpQfZi5n+1FPvQA5Qk38SsUfrAWqwkdsjApzyI/h6qNFM09Vya9zG+L57DW62U70elpUOz1wHwbTCK/VqBjl6jGVH5HA+G5CCZRVnWSMN5+sX/RrMgJSBnkn3IskKwChI1ojzjYXMIq9LD2fnzCJZdihfOW2OU0PQ6xYYiLy5VK0kPEhIZzTL44tkvmOGfhdotsP0bMBrjDV28Kj3uS7N2nOeXC++x5uobgs3LdJ2/IuFYDHwUplTW+C1vE2+kuwAtycMenmsOZHcDbCxtjtVtfZZlmJMfhfoGXPJbL1xXtbWc4vB8fvMeRLpJKYsKUBAxgzMWO0djINoqStH/ceNrEtQuQEPjfm0af54vhfUeUh9/7xPP+8PeaDC4Vd4GWbNZ1LaIssRbbCwQ1/Yox3lesgSz9MchnHEDeAC+DttGj5FHeODpdQQgzp01YddIi4U/o+s8+kW7alyyuKlGEVSTkEiasRZn/6OkSYMUNntQFZ0fOkzX+44wbNDdd3/2Sy/x+qJF9JJhsLt0nV5Pb5WK1xctIqlza57+9gkmf6fGXOZG5qzJYE2YTP0n08jz0HZALct82OMc7y7/kFbxHnodSxJ6VDyeuIUFyS8KZr2jhng1HKrxJdZrmnBsVkIWJp9YOtmVkpWyJFOyVab+Qgr7+u2nUaXcL3N925E+eC9qYyMPDj/5T2Xff0fe9AcN1LQEnSSCffWNK3HmuXz24FoRDO/WsQD75aYM2z5M8axlSeJMpzMESxL9lj5LkqRUTTrjO5Q383Vo9GY6Je8W5s4Tobco/sSlNHJBJ/pVmuxfWREOv4bDwHwliUSthvMtTnDgDv7ufu5gYzAdpBl008OgxJ9QoYwvs8wW3qwpZV5xABkFyhzdiJT3ODfAFUs8Xh6DyqRcQ9tc6UDz8za0DSbsboRjSYLWUWupc0CnHKjRKFseyDI8FnGdh3xgRQ2YjPGCvffVCwxLq6e1YYRwT5w+zXjwz2r+vPq6YCN2PO9X+fNzGHzQ+WVl/7/wAUwp1HC0EXY2QLlDGcOZSKCxzog1oIbHKi2YNMo82X2qneyIhGa3YXSmWP2/qfwPns7qiV1tF9ZYFQ6SvC207XOOTq0P/UdiTUlDNGcLhyDpxD3RiQMfybU3qVT3voMyPz72I4bXviFHYxIIQZIEKruZXn130r+bUlXD4XSQFvINF8L/5IfLkeTXKwHnWm0M2eUutOvT5BrBb/Ky1xKnt1KRH8qn6e6FLIClkr2DJrI0GDy1+JGsVXwbZuH1QEjW7FfYng46izR+C2mhmzhwIotDdjs7TCYmAxeBncAR4CAwGRhw5787LQ6OOBxIaVWktf6Bh9vdRuVUxml5jhBOReYS3CONmSEiaThOVUeyBixOFU4P1aX/jv+Z8S/Y9+/4/9TIk1NZ0PwMiYuXMiFVZE9+YtCwOP4q4d5m9JJbFYUkcbz5LS61vETD8bYez18cf4m3V77DkKlvebQ/GX6TH8Pgu47vCTZJlqhyuALr1RFKj/nL7Dxyo3MJ7nOaZwaKfLFU7xqC/et5K9hBc7cA1q42crrIl4BNQ4jPjheAlsn6XGYs/I6Y5ByCQsR+JPmZP/BIsZUOJ7qisniQbZLhtb2HcCITVvuNYA+VinnSD34OgxDTgbsGlZZ3eZNDodnU579Kq6BJiuMMlXfBla172iG7VRgZrXcBjnyHmIB5LSAPWWtFdsq4P0pMJfwaf5NnP/qCcWMOCwl2gE7G98lOrnI14fWwob90uyVtbkOL22A33GW3SbKEbDeSooFE/yuCYzfSfyI9zg3D3KjnNGIC50rJCaxn23KmwxkkX7cltPoSu4e9yaKFP/HkkPNi4OKwcqvZYM40b0+1WkwwgqvazAnYkQSv0+SsJSfkO5ZLB7lkVzIsj1u/odtWMwf/6kfVFbfgAPg+yEGX2kC+rFBhdpN4MWsCybkeQ9MzHVl+oZeQJGz07c/+rYNY8MjbHLs8RTh3VMkvHGq7hfPqdK6Z6wR7hC2dA1EuoPys13DB/lqgjs9DoLse9M67c8vuk8Jra7uS+/UkLm3ujalOmZgKsWfxYkkCrdObE1P2mftp2WQbxGhnCfZx27gVLjoaCZITMmPxr/J3L64Dp5318XmcNkNitnAoAPPjN/KAN/TLaAluevmovfg57WceCzdxOE558nYFWUxaNYmGEXtpeGIajVal4/d4oS8TV0/gqbRwdje4JZqdDtrWBdJz/Sh0N6twOpSJDoNWR6cbG/m0TMXKCvH9LbFL/GEuwuSEBtVZwa4LbE/fHRvpburP3mhRFtghaahxQJ98J+vNykCzc1Rn2i4/QMopF6Bw72tUo43h2fUdeONsR547cY3gEGWVcVrN+5x5+ieKzjdl5S+i3HCkqoG5i97guS+eE9cLYJfUk0wrHIqBGLVYgVQmR3GwAUZ7ga9VlK388PQuOuXIdMwBS0iffz5X00Bos0riXv6Tr6d34Vb+BsVx22zJbJhwiOHP/8GISWIPxC59Z1LV4QJNrjWlu0Nkxoc4zxOrtxLSNBsP7dL48cBUAHw7epaeLr/xIUOMcCzeszTa6KZLaR5yDL2kBJBuJn/E9TZNOL3ifuZuFxPJQ3q7JCXDcm6gsbg58RGDeOyPTMoKQkjb15EoX2UybijnWTPuLEu6rWJJlaiVrHGa2LFyGGUFIdTJYkWuJMFfGS+z7ebTHn9Tp+D5DLowiicO/oJ0z75Tq4vgw2AXkzO522UGhCp/s8ag5sx9b/H+G99S7UBYX4vUTfmz+yH8y0Lpe0UpheK6LonBKi1XgvW0ltwSuhofamPepbbLOUJmrgZZ3POeD2jAmQw/hIJcLxIyIrxv8uOoWJYNTVR8rrUXMrnLBJppIc1qR61WEmt+qJrH+kNt2Xu8OYur4oXzxmboePXD8ezcF8/hSmVFuMZeTw+Dq8dbSYw/Bo0y4SNr/al0QFkSvB8ryusCNCa5+oJsCUoUbH8HzGqbL6EGZSV8SuFEZLvr+ZXeUvZclZwOvjC6pLdyUsIJD3tBeWJZQ61Npr7cjx3FIR4Cc4m1Y9exd2gF9b6idLT95lscjoH7vMAbcc+qzP2WtBjoa/jPcph9DPBtGGga7ioHeOsDaXttHaOO9OZLSxgGu/LcfTTXKQ7w401bBWObvCucN1qXyupnvqFu7hc09RMrmJ1qFbsSbvCxoZR6lQiUH7F/Qk89TJSa/yNdde/4qeRpUm+D0SzK5AP8XpxKnzxYUaU890ORR9jy1G6ym6dzM6IAya0Cozq4jl29DnN48ERa9PhYeVJLNRdbbuJAx2rua3MbjUoJvsoqNY13fLyDV0Vg1WmMZlIRBGfC/lqRqY21jiuxsDDIczWjV2MG1+Jgui/oNPfcE7UXrbL8ubqrK0XXY7H7K0l+9XI40z4dyfe72rK1HmxuVd052lSu9DyGvtKfB1eNFr73QcMJpg1zgXC+haJ/UiMHkm6FWofssRrdrPblaCNU2D2/g0sdKTxQAA8UQE3g3V6cZ0sPsbdtID7Tf2DZR98SZT6vOO6CLYHrfY4w8OW1zHxajHXqz+XgVVBMl6HHafQgseg0uny0kEvN6RgsVihrwmDgpF386echyQ18dvUtwjLhkWs9QeWWYDYVsyx1G2822YNBo1xvHjbV8VJxEgfGreG7VNF3kdq341zvOvImbOCIQ0lUMkpmRoYcwzc3iq5qFSqUTkSyVwjVW/oz6ZuZJDvFeCOCVEJqQjDejsZR71l6+pszX3A8dzRmWZTo95PqqXfAg+ntsATf4wPbTazpNJ+BS7Zw+6lbNBiUPeKuGzsRlgkfdDvAx6P+wq5WrpNXNeN42SeXgOhiHg8SgZQPgy+zNrWW2R02kl/jVkHnsGNQlRA74QQvPrwff5X4uxOk7UTFfUGLbAkpuJtg//XC21wt7cHzXacqPk/TD6VnHowthMgshPd7VUM8fxX4cLHVRa7HitUdOb6VHLc2p83FNqjLle++b8NNvvbX8NyXzzF57ef4aJX3xNuWySx/WBcBz4fnewT71obPYopxLgvthwSb2diUSYXwfTWcMymrfAqsyRSpCrgWB1P9RVA3UG0lSIabEYsoMitjfZUK/Gv78Hs47I8SQYXAkndZ3WQB+bdOkFveTjj324X5zCmD2aVQpW0h2Fs5MshMgDOxoJHdCMsabz6vcpF6FgahIP6qJRsTDDKFS6cyd95jWK1K38XbUcFjvpDxySPc3iK+++H2C3S5lULt3m5k5Yjr9+b6FuQO38PnTS5RL4tVV1rZSJ3Fjx/PiRWeABmWVKQb0CNdrISXJInfczI4a9yF2q0f4FrZwvCUvQzvuIeOknItmv3ii4yfOQq7rR8v60N41DeSz1JT6TN3Lrfz8pj94otM014g9I785+0qZUUtvk2Zc/ph8lOucytJlI1EUjPv8lQA+vqIIDmyhqe2XuNA9kM4vUVSZWjDSU7HQj+DjM2ivOdpMUZmhIQQWlBOUuV3wrHHr0+kMqCew70PC++YXePHNUMtN8IK6Vxo/2df+e/Imx6ygappK0JVog9x2d6G3XM+od17v7BioIc46/Ie0lpYCe14lXbcjYckHAwzQn1EMcVxOeS7VcLXacNY1/I838z5lvcrXxOIkeN0JTzgDSpA7SGzLaU8x4g7cvNpgUrgS6WCI7WBVDjgnUDQ1p9XHlx6jGjDTr6+OJcx6wqFWL/aXEm94Roqh5FQh3LetjD8xBfjehM7P5/gh+vw8VICoIPil+IXXUPRsNbI3kpZYFmGYRfG4HMLzphBr1OCZioVrDi2gUEZbXg2PwZTsJL4q6WSwYk/8USbuUSoPahrySqSAtP4eXQkyWqR1PNQzFy8ZNBpy5XvkW8Kv1arKb6zxLj7Jn/UlJKeeoN6h4PsfZ1xuFXFhlBOxzvbfrxZ7AF60L6YgVfy+Wb1o3Rxu26VXzNG5k3i7OG2rNrxNE63yj8Ar/p3+XisitzuMqVu6i0AD3jXYAOSs0G6pwK6T9VuNvbKYZ6+iGEFIiGzV8gIOtS8ydVVkRgCvlDYHE4HfzV5k1kZbclfPJPM68q54dD6kRvk8ueDD/RARrk+S1k/sWTIYs6P2UhRtLgfIqm5Wt2E2QHwQaQYO8qOBqYFWPm8Ct6qUpIqx/ql07LLGbaWprHN7qTLnc+LgNeAzfDPZ+6jC7DNDlsv5dKyugiHG9FaI0no7WrKjrYnLFcsdHnXdz9P+sHo07Mo14qqYP+O/5nxL9j37/hfG5IkERgYqNgoHo57g6GHfqLwuTdpcilaPAaJyUf68OySl2lb5Sa95HQyx76XwAYvAisChTJpgHU6P3KuxbCxXmRuAnx8+TkyLNDEO1+wSTjo5oxnUz2sdZPZMpSoiMmLwVblg8FfpM6N6XuKgQPSeGd9Fy5YlayNOq8mTHSWURVeRPMGX1FSxLc/Wx1dmPfoj7wUJVZgFMne3M52ORDdw0VH+2+nKNQrG51DdDqva3vRLgfa6yHM5M4WlLh/80j67GggQq88tyRBQFQVAAMOTiPaV/m8jqt6snTBu3z2zgJMHq77o9r7yJ2yitpROwlw13532EjSm/lKW05YRS1ofITjyx0yhaYAujUOIlInspOr7VouWuCqRamTrnZW08O7gadLoOktL+F+d1U9S7I2j/LAcjb5iOeV1TKO8Boejq+kCUr2MNogdt1+lGAVTPHxUL1RdpKr7XazLuUcVVEPCecGyE29xc040Pm+L9h8bLdYHQ6DfRq4bVXKYYZoNpM/4CB5sRJ1yWIvoD52G/qlMxm6Y6iQENOoNBzr7pLfLKg2CPaVNz/GJ3cNksPJZQ/yMU7f5vxZEUqZHY/zTiPbCVTBcRPU6MR3tI/2Cs/5w9EY8DXdlUyTJZmzHc5S6V9J34N9kWqUyYjmcgkj8hNof6APneqVrDcAncqApSSIFb6FXAwQy/N6NwZh8qnjflMAvg63NUFSsfTmM2xIf5E2F24KxwL8Vd/Akko4oLoMbpWtAEZLHIV2KHYLnn1jauk7Yz2Te17lq/hawVnuY1ahv5RKy8stMVa5ndduYusdrDY3tSkabyXIrjLnM6RiDT4vz+fSLhHEkSWZYjvEZsNXZWLiVKXS4FN5P46GFGyIugs3Vb1onwsnTXDZ5NZHKHcDstVMQHEGgyKUbECjVwoB64cTfqgTpgaDIFnZRbWVzT1u49BLNK2dLHzvT6ENaFRO/Kv96Xk7W7BflFoxsRDezOxKWKDIFrylbsOnVfBaIPTwEp8VQKXDyVkzcI+0pNqUyRNx07n90Xg+mPkaJkl5v0/XF7DdkMGh5Gsc9hJRyBEBLfHpBpEfLKHM320dkySGpw0m93osPZNKxGoxIL9pIONnraFnB8/kACQnv9bCp4Ui0AIwruU7bKiDmUXKPe1703QOdnqVwtBiKh1txOO0tVx8cwnPLPmCRP2fCtuV0nT2th7Lob65XO/yMD4GZQInTUrkpWwDcWfaUVom9pqyecVzK/EWuTEFECZetyxDiDGXR9u8jsbDnqXFm063NhBdoVznai21HGu2gN3vv0PbCQU0DVIyvYuuL+Qp3SBe/uZBRl5pI1QyqGQVJr0JdUwp8cGi3PCPoRl0n/8Gfz77Gg6nmPTSBXpzNHEWv9R+Toif+Ls0uKpJHvcDySSyICe1fBujpo4DOUpChdZZzosB0N8AvjLIbutza90s0tv2ZPPQg9Q0igFXpV1F4KUWXP9tBBerleukNbQvvrfg40qJ0trmQg8M2d5AlBpW18JJD7K+ALvrXJG7zSkyww3mHP6KgPzkWhJrXlf4fQOquqO2ytT7RZIXp5SjkiWJCe0u0ndRBiUpTxBoVP4uvWSmS0UY1eFmgliJ1q1HYqxcQ/OmedRGJhGsEqsNtJKVTCtEZsJNjad10kFLHTznD8F2EbT4OfI4y0JhUD44/e+uoxpZRXy6itAd/Slb9BSaEiUIH+03kL07x/DTO4/z6a8i23qMXxPm+XbgvSDorBa/96fwKuYdHEzH0x1FAMhhZVLAdxyOga9jczyCfRU2f94IhJ2pntN6V2Nc0u0lamUF/i1HOL/WOTn58GqK+xwHN0Bkpl8NT/u5WNOCCyCruVneA6/bYbzjDEB2YwerHLVInS+w4bEVrB+xBffx9++UHBoG2DwkhO/Y5wXCkhAx8SrJalK0sDwMok3K5L6EzNlWF9nS9izrGpTJZCd6Yk93oNvWYbQ911bwbWq0EUSsdUkct08Qk0pxqjI6drxO03e+oCpFVAvZ6f0wL5RCa2MFMuLeUezTmmdLYZwPBNWLRJHLTn821MNxwvAPvivxH2S+wEcrJpH40dN8Xw2NamVC7LClJS/aa1gbkcm2+k7CeQ/FvcKZ0Zt5vPURfpdE1Y0RNcFkNb1GYV44ap0IbB3J+ot1O0bgdUCUCAWotespsYMrNes2dKHMLYN+Rujpr5SAbl2ixbb6PtIzI/irTiRTBManEj/mI75PPU+OU5kYlf2a8uj6fK7lB/Npji8Otwo5nxafse3UaLzyIplXJZ67zNqST5Y8jt+n0/EvEJ+VSgVbbzzLoiPraUCMXS+re+B9C/5Un6PYdE8fLVnD+hvzOXhhDrnHfybWT7n2O1U6gi6lEpkZ7/pzt3ewS/BgQuzfcmL21/weIcaev9Um8eq6zkz76AHOWdyeZWM+vcqiGdLzHBnaXgR7i9cdIF9jnA/okJHcm1UBDSYN228+yZorcxWfhxjD8at3zSf/us5o3HrBamQtrS+2JiknDh8/8bolCSwtS2h8dgWhkcr40MtczENaLV6JebRLyULj5jf5WC/zWYhrz/q0OMKjYoIUsAd/GYr9Ngs2uzaUVXXwRJGOXMsPCluetJWq4DU0OMHuIYX2e/M1lCVBXgLQqDy3LENTQuisB7VD7CVoVwew/88BdNk4CLKzhXOrJJlMK3xTEo3BR6wE7mp3JcZfK3PJQHoaY71dhLd7H6XsMLMyVEZ9Kx5tVhxtfZVku9D46Yz6YSGqOm8KI8R17pZhPL+ddO2xeS3FWH91fRfeORHPluthWGQxlt9q30n/Pb/yR7nnrnF7bG8CUOEjVkcDNEjf8FxYB0Z0VFZt/71vOACbh2dVGRVAbDs/XvWazFeTZnL66lXmzptHeLjrOm5oghj77J8MWPA19T5uVfb6EIqOvkzU9RRe1LcXzu0Efqs30S4HvisVK+AANOOnMtD6J1sSReC2wSuVzXWwOraK4BplFbPFEYchUkP/B/cSE6bcp002E9smnaJu/hLyEyDCroxdVdi51eE8eTG5tEq77x+S1N/ypsPVqv9S3nSorGHgKCjrcYlgjUj+apB8OGqsxuxtEXrbSZKE88YR/CrqeaXXNXrbz9+1Oax8XJBKilPLskGHqVcp/VSNDKFV/jz04Qzqs78S5nRxqyzWL1nA/Lff4l1JCZoBSIFt2NYAK2sh3V/5nqjV8Hx2K1bUwPwg0NQrZdsdp59hSPgDbG0Txcmo2YIcspcll8PR0Ngin1ktByiPtUlUFvtz+urTrL66EbVeCRSGtMlh2ILv+PrhMfg7lRK4rtdXhQZoVzSP9sFKSEaW4XZ1K/TnzjOhIIcOkUr/xZssZnWZzqhmnxIji34VdjNRPtepMQdjkdxA+KtLmOZ8j0lFMPp6B8VaJUkuv6m3AUxJ0FernJdWtZnc6Cz8q/1pn9ZeWOdO+nxCp2OzebzqVUaqRdWvDt5rSdjXh6L0RBLVfRQ2taxmkm4lG8ccZ9OgHyk1ibncMmsRTbVwIQ586kVS0HAjvBQAHbN+U1ybBhsnPpzCrI9dlbbue22Y7iSJbS7ToDXz0nHlvZaQWPHDcHLXD+RAnwOYwtz6yEoy+VGuGPyH68E43frjZmvj+PxcR5peacsgs0h2dqi96Fp5gYtm6O0rVjg79eEE3HKt/etrldXqD+dO4JGt0FNWK0C974F+/Geg7+/RBeiugvLTILuR/Ob5bmNDm3LSW9rIsK0Tjt1gbsmXVZAR/QYnKsW99v/K8IR3/E+Of8G+f8f/2pBlmdjYWEUz6FDzep6MW4NTcvBjXTPhmBcuBxO9ux/1DTqKZbE3x8tJHXn78d08N+9nj6DDra29+OmdaZy66tn5yqiLIvU2tDwnJmFwWDmWkk2MGg7VK3sY9Wp9hqip61ncZxeP5omgw5KyJvxwNp6wM+2ocCplFSQkLDoLgdX+9KwJFuRH63wH8su54Tjyw7A6xWDL7JVAYeezdB91iFijZxlPH205Cw5spyrmTdGuMnLZDM2y4aLqnoSbpYrGBAuxOXFozSbCTEp2/MY6E0cCKlElljH26U2o3RKnpdZyarBxOwG+Dlf2OgFoH/gXqlgjHz14nCi1G7hqjKRrYWveqoCkwjkEGsTKvX11vXmlupKVqbvR5IuSJ2FqMx10EKpCkdQ1mC5yMNoV9NQZMgRH5DYD0E8sZuP4yx57JPYJnsXQ6euJS8lFZXdL4HjF8HbmSLxvQWiWB7DPGMnOeuhugAf8PbD9gGMVzUjSwk/x2wSbwVnJgz6wKRLG6JWMqCcDzvLWo/s5NARUwaLU1VsXXWBBakYq/g5l8BxvuUlmq3puLnmbOUM2ChtSN8sOug8+RU7SLQ77rxDOnWloxmpzCQeiIc4isl3LvdoQnw2r6/DYk2FF3kYG5sELpdCgujvvZXs9nZrn0uSVH+k3/zsCQpSMJzl0Gjv1riRDswylrAJAF20RD/81jgH7+99pY60cbXVj0AZWkxpUj85djkqSuV7wJRQ8Q1Od5wbrP1cbeLMcvCXAXYvbVs+iTq9xxgSPua0Jx/070C/iIs8UqBl51VcAVxdElLCm01JuHj3PrS820Ck1lXcXLqSoqAhkDWsMamZ89x7fvjaZ6+VK+VJVfQbDkl1OVeItcS3yqrlMcYKrysFTFYRkbyQ58BRrrsxlq1kMrpv5dsTLlMyrAdBFq5zX1iMTSX10E8ExRRj8TyjmVpeIntzq9DhhRVpaHf0Ei1X5LM/YmjErV4PfC4dpPUZM/hTrnyR51i7Chx6hPkhMYqtkmdNmWFHUHNwarINLcumYCSZcbw+h/QV7h5b9eEYfQ4/sFYQY7s4f2SucXyvvAtRWrVsSUXISnxPDttX9We7hfvsce532e4J5I8fANSlesOcHXGfzhNVMC0v38DTAr+k+Xo+5ypxyTz0lYVGrZXwUDJ8Vea7U6Z3nCgDSanYoPm+hLqdTYjHHB7WkXZMvheOK7GpOaOpZoi1nQ70yeDY1FtEs4jhZvXZQE5IkrKEtU+ZTv+wFhm0cxZRc8bojfaNY+ehKfpj2HbvKvxXssgxVpjDya5riREy8+pv3s3mSxIpRykSzbKtnkg80mtW8sW23wCS8VpHJ6XU90VQE0GvHYFSSssKotf0g34XC1PG76dHhD/F776lIynBPglsqSa5KZnjIp/hXR+PlJlMLkMEHjF9XwuObr0KguLccvP0Q36d9zE8XPlF8btalEHtLTQsdVCe5+k/eO3zVubQtMDJx1Sgmly4RzvuYYQ7HR24lvVk6skb5sIKMQXS8uZ6NJzfx6p6jwrM0FGzmShz8VAOry8XEPsCSfcfptfEUP11eLdg09jpGekGo2omuYZvC73t84BRCmuexY8C3/KV/SXmgpOLRHU5WnHiP5oa9yHblehFuyeItRyCxN4OY0ERUahiru8VnhU155PANYsp+EewXgkaRlA2FdnA4xJlX5NONMYUuKbhoZ7ZgP2MK4vtq2NMA8r1Mb1MJT4+fhv/ANNK6HafRR+m71Bj684fpCQCsuWJFYRPnMlLTzSx99mVOnm0r2AfJEur93blv632Euffsczp4zK+AfBu8d2WmcCxAFQcIVYFJEr8b4Fi0q0rzo2AlmHemIJVdH0/muazm3Ip3SWzfO3qdTSZ09gJmZX1GXJHbeqL24oezK/npncf54sXnsbuRX3QNaZxqYmVGuzKGyiKYp2nI5UIsFCdZaeIlkgdQe9EiBw40QKldJGg1GlLolwcXzBDkdc/67bAx3c9EvH8DcbfjMJQpfRO1l8Rnry1GH1zFw4f6421TAvRqScXZ9mdRGUykjlFKJwEsa5xEj40t+epgKidlcX1Wyype9oddLQ9xu1zsdSxJEnFqVx/ZLl4ikP73/v3JkE+I878b05hV/liTM6k2q/h8TytqdMoqCUmSMNYbObWjLR9fF/tRftT2HB+OOkuhsV7R8+2f450q1g/fytFh7cEDoOztCEdn8cW3wYMcFWA0XuadQJgfcxEcyjUYjTfLquHlUjjVqCRMfBrixe5x64i7HUd0jrgWjTUWMUqtRXJIgg9rw8Etv+OUh0URUPUazYKURCdZhttNXec8UCCCDi1aQHCBa02PDRXJoCoV6FT16FT1ODyoHatVauLVMN4b9Pf2MJI1fJc/hNq1mbTe95mw/t6fMIBx6ycx9eep/HVmAEY3VQIfx1WGHjxI+Luz2FHrL3xvtNdxjH8NJ/ZcGxqcbuQtjQ9f5wax/Lmn2LBqIKhFJYjN5l7EZN2pFLOKa0aQIZfC2iS23XhG8flQbw1fSA/wTuFGXm+YjbdWee6fY7oxeHwhL7y8mvfCxOR8Uv0S+q4PYXBeArFG5byrCe5LrxPh1GdG0yVuqwAqNPj0o0cuPFsKJ+v9PFb2ZcVDZRKsixYVEe6+O2K/vw/iZrM+AtrlwB/V4h6+tbwF50xw2woOSbnOyTKMS/iLeA3MyI0SrqvAbxKvDtxMx8nbGdxfzEE8YdnOb+EQXf4Ifh56He/Uj6BLDnxQidj/2VzBhVhYWwddckFzz7T2Ngaz6MznVPVP4mb7sei9lWtopG8cxpoeaMPKGNxbjO8kTQRnup8g8eHNPBAsEiNVkpqpP0/lqW+ews9D79G0hg2cTh7JqRRR9QXgreBbOJNdSXpPY+Claxz/4X4aS5QPKzS3nDc/eoUFCxYwde99wnHOIDtXeh6jutyf9BwxgX7A0YxWwVfoRTEmlXi/G/2i6TjhMk2TxUobh9NBduAqzpsh102i3PXlTkIrfqaf0crio2J1tU9QdxaUqdnfAL5JynhoU/VGfj6ZwpcZIbzkJqdtsVsI8P+M+K2D+PO9qThkJegbWXGYmiToeaED3a50UhCCZr/0Ep2m3kcvJPqj9ihv2rnLBHaMh/z6KMxOkUCrlu2k5EXz+a6+bCpX7tWyLGM3ryCmwI9JORo2aAbftan0fH8mnpr9XRl4/ip9gpWtWfpWruZsgBcGkwEflV2YO9GJoYRdB5wSN/4DKS3ePx6ACJ8It+sCkNnXAKnZYA1V5veKE57gpbJG4tTQ2qsSGeWepXLaSbozn2xuLYLWZOtZv2AY6sM70FSJOZ+RJRWMPxjO26t6crpWmZeUJGhpLGNjJKT1ehetSRlvyzLIko3EgHNMaz4Fig8o7CZVPF+c+o5vz3zOLYdShQdAMuXxbOcnOJE3mgLc5kf9bSLJctFxnLLyfmevIjupgTZa2NUAVU7l3Iix3GDwbteznfHkJrxsyj2rlX93WuQtZXHz33kwUdlnE+BRv1t0fWwjN5vpqDM8rjRaa0jV/8aYtd3pcyEMh4ce5Wn0YVQBbK4D2YPq18KyJKYUwdCIw4oeigcCR1KckYBPrS8v+aPITQNMyR/NG/YaDCYD/h5Uk36NyOR6l1NMHnOYZuHKNSHAdIN3mlRjD6iiMm4aIW4VnhlSABvLqvDJjGSMz3+ur22TA1P3bhINkkyVA5proa1WuU9XOvwI2R/EDLvyvd0IiGUBnscTNgi4DCq3eXfNnsT2nGACytUeyco7rK3Y3wiz/MHH4aEC+v/I8IR3/I9+///Kt/47/h2Aw+EgJycHxz2RkY4yfExmJKdMe93bwjFtg+oIa5bFn+Py8I1yW6YkiQ0HX+a7eU+QdinRI4K+3e6SG0u+KUpkADjvbOKSB9Y+kpqPcjrwZx146ZUOVBNNEfkrHuDJz2fR6ozIIi+7HUl6QBnLnvyWeF+l7I3BXMw7ARLWBZ9wdPIqsffMpXl02neVp755ijYFYuJTo1LxdfIlHgutpkr2zNGY1+9BXurxGAkJok3naCRV60p6VZnv2YBkLRvrJbRJOYQNuUKOTRlR1VmrqfUqRm6UadrkvBAUeVHD5H2D+OK1JzlYKDYjl2VQfTOA5xY+xG1JlHIL0kTSWYrgfr0/Wg9azw6HzC0rLCrxg0APkicxGZyJheJE0JrushFthiTmZXXCCQzzEDzMKd7JOyVpvHMrhDck0VFoYljO9e3t2fn+43j7iQm141ETaHSCT+lEEezzTmRsoavp9roqUX4AYHb6WIbnw4eFvQRbo39npBvwey0Eq5RB6orqzrxeBl91fosgrSi16dR74/NAJR1GnKKZv/IdVGn8uFjuhaUgFNku3mutZGfAyGP88M5KXgsX5SAyq65T73AFuDZZTEqp1BJBsiuJ2K9qhWCvVDm5XDCKdTenEhBxVxJQZSpmXQRMja0lNLEAWa+cG3ZtJBcbOyN1qKMkTvzeEV51tOuUTnxWAk/ki9Ves306cr19Po/Gn6NKK04OjQae7/I4iwf0FmwAzWqfZlko1DZB7NnntNMj7BhdiaebO6AhgcquYuAXT6HbMkAx5z/5+GPOf5BLaFod71LG76ZGXsjI4PCiRcRFR/PJp1+QcSOXd2/Es+JKBFa3RLXDvx2vRF+lzcy1tHxQBIxRG8ixueTrUnWixKKj/hZLh3QhqVcMy8rHCPYw8nnFx5fFwdBTq+xR93SFga9apPHstG94v7azYk3Q1p3ls8d702nMXvoO2ENOsfLabjjjCXptLjFbY+kfpZTIAMjTz+DVxo482XUPF4JEssZDjr84GQNmTRE2hxhcD/fS8oNmKKrbC5CMYqJwWtQhvozN4cigqagL7wJjsiGYTcWugGDN+DX/9Lj4e4x07ufDK13pfbg3PQ60Fc57saQjt043c0k0i8Ud9A+uomVuHG32jSbUKCYZXwtYT5e35xB90vN6kaY7wQUzlPp5SILjSnB30ENLjXK9GJ7VQNs35rHEsYHZyanCcYNPtOWRxa/w26UoLpndZLLqrnE4BraHVvJG7+ZU1yuDudYh7dFZXNfbtpuYTA7SefFbVTtOZHYm3uShZ6UM8f4XqTKFC30sgH+SqUFuSUC1uZwV/hruX/AG7XYtwVKhnJPXjXcTvFWPrhV6CKglOxl/DOOn70Yys0SUIBpf1IZ35r/DT08V0DnALTh22NGWVOL4vAPdtq1GsouA9aG6H9jWtAsH/JeL8nXA2cJhbLz2IrKbTCeylgKrnk118EM1yG69XHsbp/BYn3m073eWgYliNc3nCe/zzeh0kp9eTbheWS2mt9Vyv7eDcGMBRf4bcbpJZjv9WvJhBWRbEWx/D4umlOzQr8j0FkE1i19bNDdBugFLeErh9x2te47S/BDGnG1HO32W8kBJQraaid/yC8Y/j1BXq2Qm12nC2NryHENmbqJd9B7he/fa4lj15ThqbwSjqhODY41KJkIFQ43g7RBlc2RZ4ngj9M+DXL3IvM92rGDGndzNvQQWq6ThnFdTCjv7YQlZR+8UZdJKlqEuIIhlTy1j7XiRbZ2pmcWeP+4kwYLEZPJHmRVYI8tpNeoAsvvzkLX0yOlCy9uwrTLKYzP45yP/YIQX9Mj3/Cz/Xt6aa5TJDJVZJiErgZLlD7JoR2uhR/Myhwa7zkzGqeacuiLO+VPBS6kIqKDatxpZozzWYUxg+d53SdzWg2fUf4nXZG+ktQ5C1BBlOCPYATQ2f/rlQ2b1LsEWYgij6sqvPL5/NZqoe9jzTjtfhNcx0OxLUl4MzZzKZJyvpYCKVibuf2I9MTPX4HCr6m5uOsXw2hAa1FYezxUrbVSSmg5nOzBw70BkmxiPtDCfpKUOXi8Dqwe/KdZezrTQED7XD8aYPF2wf60+gzMZxl6ZDll3q+DqdbG80GcHRd61DN0xVGjxMlSXxuJjAxm8ezDdD4rSeD/VhPLjlk689vFLRBeI835mQQABAABJREFU78l0lZnpx/tgN35EdrVYcd5Mv4XEsIv0bSmC/wBPpnzK/CB4IaRSqBAFqHLAx1WQblHGUk3CHqZK04Iup7rwZpVIaHty9QAOPfsGb73zFmM0ysrsRnMlluajiMgtJqT6MrLbZlxbfoCmXdbxxbz3+KupGP9VS8dRz/qJiBd/pNBXBCzUalj7oDdrH/RGZxflz/2c1XwcAmsiwNig7L97KGwihgYXeB7c+LXCFl5zlIee/Q1kB+kHOgh9vAMtu2gafgO/ykD8KkUlBp1Kz7nOpaS3voha7eaAaAN4p8EFYIbm3fa4vt8wn2aiN5yMBarFvsFDkpbz0eDuhHplKz73qd/FI23m8UjrubzYTUwpBtoPkWguYPn8mew71lawJzneQu9Vx/IGG1ecSplDldpAdnApR4dv552Gvthx8/e0odSl/0CZHTS2QI+Vfd/eWfLDPBSXetdfoDABzCkW2qufVNjS6vuw8Y5bISO+JztvLqV9LvTIgwqV0q+SZdhb3pRZJZCvvSkAFmqVinrveqp6nuScVew77YOJCd6wsf0P+DXsEOyNuiROmSGiYrxHgFMvgdUJDU6ljKes0nE25ym6dsng/RnjULsRQVT2agLa1DLtwy9p2kT0i+KNkST6jqB/2yxC7aLvvST4NgMn7mbo8zvp5C36uCGOjzElwSGxsBSASofrAX4l8rsBmPr0SsKGH+QDtXJNiJQLCDK67uPEcNEfe8lwjJOdS1kyfyGruoqAsyzJxGfF0/V4VzC7zY2KNL4a15Sz+1P4du2LHo91JsOuKNDgwfHHyTehsDsKHjaKgHOgPpjrNcn0z4faeCWx7Jj1KwxV10i8loLVLVclIfFVbQwcb09RXktSo5RS+A2GSJadjqNl2+s8Nksk+T04Kpn7p/hz1SuMZyQNYySZF7RaHN27s21nGv3a3k++VcWUynyuO8Qqzw7SGR76fjoDtzZh50VlqxGHw8Grj+4kvFk2obvfpn+z+f/YtGoDv3YspqzPBabpHifYqtzHywxJbM50kaqeaJktzGmNPYeMARLfP/4Tv3h5eFHSXiI9+DZ7oqDrTSW4qlLB7PBMnvCDDCug9VfYTeFD+LMOFgTB9lY7kd36R5u9m7ikim/AtNvK9hT52msc63aMoBIzLbLnYjEppZYDS30Yu/Jh2DyQM/XKCSDLsCjpMMO8YF1hW5xu77csw2uDm/HZ0PZ0j/gFapXV/3Z1ILtuzWDLjedokD0ov2iD+PnCItIKh4rrRacvaZYXyxAv6BPl1iNR402uzclpM9xfCFdsbRWHhlgLUWtd60BlUDlOSenv+VhO0Tl6NcsvvcGW/FnCZc0r7cnLyec420OD7K7J2ljEfYGP0qLrZWa0yUVzb//bO8PhdLCp3nVtTl+RcBTIEMZponmr1TdI98TzPWN7ktsZ4vpfZr6fLzq3GE2SJG4l3eLYmx+ga5cp2C60vUDhmG1MD7YTo1aSB5xqLzKxofKy0N7LjN6N9C5LMtV+1Zh9GjmTO0q4Zslu4b0gF/nQr0GMC3DYaKOFnVHwWYgSaJxifAGzZMV95a0G4bP/NMKAYBv4uD0O38BdnDzxE+EVKkI5LxynklS008FnIRApiaSI/yvDE97xPzn+Bfv+Hf9rw+l0UlFRoUhI/FynZbpRRf6IaoIjxH4lt9scYUfychJqV5DoJUqPfWs0kRWfxbJsX49gX21sCnFTNtC2qyiJA/BVt5dxJkO3aDHhgKzig5JAJvjA037KIPbrhrtMoJGxYvAcku9LvwP9OOzvxVhjnsJmsJQyoy4M1auvk7GrmwAQ9XBcomk71/WOrhYdaf/GGyw52Y8Zy0dSVJYl2AGCujxFbNMo1PViH8REWxoX42CyD3TvcA9LRm3koRIDbzzyI091+5N9DcoAwN9eRmstVDWo6HT8QSrderU1sRwl+VAPrPnh7MkTN6dhtmRC8v0Jzkih0Z1xamvk67jJrAkOZGnnN8AhVlZFqw/ysA98WlcNgeI931YZzmUzfFMNaO4y0By6SI6e/4UJPvBDmCQ4ObIk0zy9Odl7mqNvEAOA/jntOZMbRGWwkcRAtyRgQx5/hKt4LQCG10/w+A7WO2FdHVw1eaYp1umvsr0Bfi73BD653o0pxfBYuVIG9Lw1gYi3XubojAXUnRYZZLaIGaxrsY/5QZPw9VVqu5v92rBoQ2eaf/IU8/PFvhz7tAPovSeRr34cwvUS0ZEOb7zIOG94rBgKVGJi1MtewUM+cNkMjT4isHvW8QKGkI34NrYiIuQuoB0Q0Jz3vlzIBzMX8NT6nTQPU/7mems1ld5/saTf1yzsLPYuyDa+Rm5wDwIjKjB66EfS2fY0TVf1Yuz2FQyIFrXjX2sXTULIQRbWimsRwPTwMzzmC6tK48CNQYnGF+/0UI6QzYhwZTI66lYNr37xPNUBFaSnpt9l6H/8Me/PncsRh4P9doeyWbLJxCG7nfffeINL55ag+vE+8j6cgcXmlpQ1BLLHYaF5t8t0iBaZl2bvpowthFHeMNxXfL8duiBeKXXJBQ4yuMv6gq/1FG/Fn+WNMljl1s9km9mItL0P3Y51Q8JtbslaKhojKOybxv7++7C6yWjJssSF1hdIy/flzcPK9xNgb9kvNEu7RY8jPTwmMpBkOuthb+ttZBZtF8zB9kLui9lBfcQXXK8U5WeG3GrGxELYnDcQvO5W6EkSVPs3UNnpHM8lVqG1VymOK5TCqel5Ck1QFc3bionP95pu4EaP49y39T6a1YmJkh+jbtC+JJKk/DjCPSRCVm7qhrXOSIBFJHoAfFiQylULHAgUSSYA4Sr4LRwmeCsZkIcMwTTG5XLpZCvO3RBlQ54IrMdQ78WM72cIvWAtxnheuBLIqufmsGvOBEoblaCbb/7HPP32MpYtXMAUm794UaYSav7qwo5fhmOqFddBSYIgQz6BBs/9doqc/Ri3pp6Ja5UBlcMQxkN5d4OoM5XvKOyySuKPiX+wZcQW3g/KFgCLa4ZhrE2+QqVkpzJflMM0yL4003jxfJCNeI27rFMw99+RUg2NKabRKjLFo+RLzAzLIs73BtjFZB7A5FbzebKdMrEvY6WJSs1fdTC9BFRu/XFX1NSzdksnLpxuSbFOKXsDcLB8HicvpPJcQyhRbr1rqLrE/F5jeaHzEwzqNBqHW/JdFdKVhRXweiA86utWhXNnPNnyfda3+5l2Yb8Jtnt76cmSrPD7plf+SUazC0i3o2l1VawgKg9PgDtkhnSL0r8Y0moWl3afZ8e3o3l+V0cB2LrqCGF3v33kJ2bySaBYkRJqyWZlOGyPggib6A/6WIt4wNsFclq0IrAwa0IbZvt4YW0CmvK7wIJVZWBQ8TF+Na4kwf8iGkl5PzMKP2JEv2SKe5awt32e+2n5vfI4uwbt5lT3Mjomij0rVXoNfw2L5YXoJmj8lDK1SBI5NjNVDpccp/uzBNhvNvCZ+Gr+MwZdb4F0A767sUHxeduka8QvXQzAmQtJSG773amUYj548WO+n/Y9h1Ld3hO7mSkJ6ymY+zmfvPgJklsCR/KK5fSRnpzc0Y0DP4qENYdfCzQ3ICITllV68JscNp4MMDPAAA5EgNOXRp4OKaWPFK+oaEHWsPjmF1wsjaVl2xt0TlT6x3aVN8uvhrHY3MgEQx5mjZJw4VB7EdznNLpaH8YcmI/76GhQ4dfWdc5WmeJNb2E+QRc9LK7Eo1R9Vx9fHtKWMqvTUwof9u9xWx3JObNEiS4Gm+YuKUKWJFpfaE387Xia9yuju4+SMNHJYKBfh5toAqvoOkKsKDxzsD3e60bgY9HRz1ucG895V+J7ogOzS/2oqhPtURHvUl/tRREikQPgUElvllVBv1MzRfnzhnyKEiQWBEKzeyo/AIb7DqGtaiIxc7/G2F2sMMq5o/ygC6hBkpWEIFVjPmkxEuOnb+GpGYuFSuHSzM94QX+YgWkPMLbBg7Rj3W3mdrlNYXwOZQ7PCir//Nshru3d9A2M8XYR9aw+9yTvHXa2x5YSPWUDLZ/4k1O1Sp/L6d+ac7oJbH9pFe8/sQyLTukjHLdHsLOJy4/72CwqC/RUPcKfzy1j1at/0VVzTrDbdDasEUWE99pPUY3IvA+Sq7EDb5cDXqIE/9lCl+/wbOcZis8bYp5j1vZzfHvmSz44ukbYx9N8tvLGzVlkNM1gv1r0bYZWOjmYnE6zXQOw5ip9XI2jkSS/RqKuJ1NYfgyn2xorSw4iawZTEevPz2Eqj2Df9+eWIN2Ax06IVVWo9dy0gloCb5SEjM0Vs1laBc/7wyCDuC+EWDtiNLmqeDVu77Ysw3bHDUrtMEYsDCHYnMa8QJhdBovKRULbV14TGVEAOk0lFpuY5Fbj4ONg2NJiD7J7A2hdIDOrprK+DtoSKUos4qSI45Q11pBZpwR1G0t3s6z/KD5Z9DDLlosVhxG1y3jbsJXvX57F1mOiJHaM6ho9Rh4lOLkRSWiODk/4XEEng81Dz2qA9yu8GJ4Pszx3X6G3tZx3uu0HtTJPkBubxPqXPsf07maym4jExcvF8Xz47iM0vdxSLFkBJjplpv48laE7h6JpcAMLVQau2howO2op9hXl6yRJIs0Eg4zwiofKVZAYeGf7n2AUq5PUZXvY3ySTAQbQapQPK0m/iIDuaTSElZJYpswJSZLEb1+Nxd5g4FKfxwR581qfFqy6GMuVg+3Zcu1Z4Xsvrt3A5pWVtLLk85XTynqng88sFlTHjjF8SHtWHsilybVI9jZ6btlRThiGNukQU0CTGKVkttPp5MViX4oy4gkq1xPudXc/9dH5MKj0LEEZPbh6qJmwL2QH9GWuXw6myX+x269WWE/qt7Xm2fEfE934E51zRT8Unyacd/ijl1Woncr3RJbhocAqpvu5wH/JrVrsbzLX1DtbsKxxJ+bcvRiVG4EwLqCIlHG7iX5oC8MH78NqU8YNgytdz2d/3/04PIR43+W3Y0oRjKs7z5kaJYlEpYKj1dH8UA1eN+G6Xw/hd/1XQ9L5czx3DK3D9hEonRfsOY0ZvB3o6vGpu/faou5j0vmZnLhzm2L1SvJ4hk8PbBYNBZH5zDrzIn5+bRX2qNr3md9rInODnuIju/isbltDsNUZuD/yY+rNa5VGQyRji1Qcn7iWjR1O4ZBFfzDckcUsf4hQuYoh3Ecv9X2srwxhanp/xXvUXWdj9KRIToS9yxu7jwnPsr2hmGm7hjHw02d4KTxbYZOQ+CgYOl9rxpNzZnDyhnIdbDTE8WjcBcJGHOOZdjMFwDii/ACfhDqo0ddzziwSXCWnlTcC4d0gaBYoEt5keyXn72wZn1YrY9dJob0JCW3AHZr3A+Gz/zSKAUkOxt9LrIpNb1XGWz/Op0UHcT153TuNJ/2gVy7c9iAJ/39leMI7/ieHh/Klf8e/439vlJhzaNBeZGidgTB5EaDsFeSUtDRt3EJbeQP+tQ8CymDwYsQmUhv70+VUL0E2BGB48jGmNjvPihLPFLJ9JjO9veC3CE9MLIgtm0GT5ocI0inPLckyb7/5Nrci/dFrEwClrNScPlkEJ5zlWlULolXKgKlpymRevJKIt99pQqvHEOgWQPhTz+BHjrC673FOqBJwr9nSOhpI9m+kBMho2AE8g/uQQnvB+VegoC/4KeVRy+Ukfq6BZaFQU/AVxN5le6nQ8MBfD6C2qZHeUnpXobYcPnhyB/Oebk7fHY3UL64D7l57mTYV88ylpGc1Y2iU8jkC7G2woOlzksq0rgyJcHPErVV0ME0iXRXN/CuTecNuw+C2WiXoTjA+wCVL4mlcvL6AVk4XGPHAPTJbsuyqUHi7AvydXrhf2SSvGzRk9cIMnDB+QV83eyN6IvOikZMrMFAA3PM8rTVM8LUyAWjoPRlwCxDqslkd7pKzPNzoedF/JnkthTb41X5csGmpo48Bblldmun3DpWkQtvockK3SLG4pwkXR6xm5QcTORYsBh6RPlHYDC4QLulqvGBXySq67utHWV4Mh5Ju4p7W8jXnMs0Pfqn1IF0KdDL6MTIUvjj7MU89IrIgPwo7TCs9ZAR/BOW9IMjVR0ajD6LU0peAxr303XQc/QwlqGsveJcltpPs++gVlk8Xq0pM6gR2mudABxjkAQs5qV6K5J9FkF8+sqMeUM7rMmclq6tdLPOPxMNJ9srB5ICXbnVjosqDp+6UeNEftF5/AAv++fi+oFDSgq/RatgxFnbNQJIkioqKeO3VVzlkt/+XzZI3m830PvkqXZwvYtaaUevcEqcSxOa04JWNSWjCX6Sfm/8lSzKldri/AGhswmNu3yHrQ/ioCg5Gw0NaDz0+pXZMzYUMC4wKUSY/ZUmm3wFXhUPa09kKR3pfdREX5gwgLzqFXx/5lc1uYN9QzQmaP7ubDTPfpPC8KAXUyfwWPiemAhDaQ1y/0wLfYk36cJ7w8xyE5hme5v3cD1jXYjfXCv+CCGWPmesWPSfqYHtDGUU+qfwjwlh/mxWJBez7eQShWhWaicpA8KKmDZ/6rKFo6afcsojSjagk7M1u0jvUTIGvWEH3ysUuhBQGM2zELiRJTBgfCing2eZZxA7xXNFidIYSo04nURaTPwCZdzgDJt/XFZ+fC/empPthHvrjIRamn2CI23HHu1wmUFeHX240vnplcGs3RPKjycpLQFFYEWiUAVdZfTZ6bQn/P/beO0qKqmv7/lV1ntSTc2QCDDnnHATJiCLJiIgBVEQBM0ZARTBgAMSAgIAiApIk55zjJIbJOYfO/f3ReEP1ae/3eb61vvt51/e415o10Huq+lTVqXN2uPa1ZcDugZy0UdJTWOGLDBT5iVWWkgS7Mp8g2u86Ua0duOPS7HYw270wu1H2SWoffnWYcLwzn3UxdlapHlDow+yFjIqrZFd9JU7ZKcwTlSxTGVBJSFkIQeVidcZnEY9x9ZMWOEsbMXY+DyiBC3WhjXzw9jw0wADnKwJqsqP2BCONQMhWqLr4r3XuLxnf8m3Gt3wPk80HV0cFl+jsuVxPrmJhBcwtF+l3tzUkIKVcwTsqm501DtxhC6sruxP/mZ3rQO13SgfW4pPE29cmManJat4JAqdbhECv1jPyQimPjA1hW7EI1ACYELMfnQzmEJGuW+OoZ4ABItQQjfKeSpLEiVYXMVYbuaIOEyyXn5tX8PPjv/KatoQZ8pcKnUqWqTNc4UTnE1xKvSQAa95tu4Zpx79lefdptNQoaZkA4utO08ILFpRp6dpc7J8baE5jeRg8UIhQzQsQVbGcRcnRYExF7X2HuUBCov+e/vQ+1Bv/kHXYR4YBd1DVIRTQ/nIbPt7QD/mR32nldl5JAkv/I/wRe4TD9THAbIXex+sbutRpKM2bS8tAkRoyQW3h10hXFYcnsNHvDl86WSuZIuaOXOJ0HaNWK+2TAT6BtFbbaD3nYzKwCece4zWDP2zvkReTR7TeLcnosDIt5DrTbv/3WpCyT4ssQ1GsP7oqJ/ntxF5TkiRhA4rskO/0gA52WPks3FUd8V3xDkBp41qr85nafiZVphB8pMPAbfYASaaibjqqdA0XaoqJUj2tOK5n4gMcej8DR/MMeOAngWI/tcU83n6xN024QMjNg4AywDNWfxy6ZXLR4UVmqOhu7zc+ykfpLntI7SkyFz2STU3eoSR9H/218SSFtVWo73nwOhGLIijKucH5brH8BX/0thUx7lgvbMDg7tvROJXzO9+4kQVFp/j4o86oneL9dqqgNLwI/4c2cTVE7KnztDmUDnF5nDvQjk4eKpTneWfh9/ISjpvgJfGq2F7ejE/5gyZmIyLaToPKEs5bQYXsrtkH3KnqCj87m+TVKSwZE0pmyxLGup1XanGL796bR8zF0zzp6wYA1AXzRomM9qv7ULXLZOTkCkK4Y4Pc9O7Ez5tUpJ5vTre5MwElu4DNEMbjm9vge7Ar2sc82+4j1ro+f0Qk1sArYAyPXn2ZfY2w26Dc81rqrNzocp77S+AJx6MK3UULvF2upcjfl7C6Z4i4a60BKJP8WB93gdUvVRISLAL16uwXKa3Xs7laTU24G4DFUskP4ZUM+/hrALI8VCD10l5ishE65cBbhghBf72sO8fzRhHmo0zsnqrN4qv44djUNejNMTxrH4rXXZTcG4o+JOpGGkHl0ZyJFquuLltUmIJKCUm00jVaCWpIsORwOEpmYW4MdUHlqNXKOeRX9wfb73ftglU1rT0ChpoUzyKsagTe5mRBZwzpS4/rXanyOU6zvGe5GxqmkQyEyLAkBI7WiPdrQtMXeSY4l98awM+pTPC7XnGJlwPAKItTP6jxJO8Gwe91UOWBWlqWZXY2QFKOlQ3xbXAPnY6uW0nLAMi31FLmYc96KmoZfSr3UWfxV4Ae7NYGVo4J5ovH5/A7vfHboGT8uGlX8dnlEAw3kgiPFilsG/Tt2fq7a48NjBB7gG62XeLX2/mw18XlhNerunK2fh/nzHiAaoCXFMT2hnIPGpfk2GCaHxRY7/xNUVERu385zuk/4UzjFXxVn1DsXcQTU6f+qyffEXMyqRkhjNVayG8p9sdt5e2kPrCamORcqv3cHEhjKo+fC2M84JPomXZvSAF8EARVubNFpSSxpxGSsyHQy1/ok2cy1dLRy8wCGVTmdOAO08/T/mVcV4VztdZIV/1l5WmRyPerIr7Sn1FRS9DYpgF31gwfrQ+X2uRQFL6aWl+Y7pzxL5998aJF/LTmJgcd0MVtC59kMrl69t14hZRLFp4fCjkmkRnmFkksGTOf4zEQqxb3llO3wrk09hfO9s9Fur4VEm6zLjjsfNCxOdv9olloViM1DOTuqdIlujPBtQ9wMieDAoc3C9y2y59t4VzJrmWKbTl1gdng7tkmP03XZOX+/peoVPDqn8foEreSorbvU1+2FULuMACEnHmaI9HwYQUEmZoz0Y3ZS22vZ7wPtNSBo0HJatBae4tHA5z0aH2aZ81Q4Ua/O6HbOcJSr9I1oJwrWuV6IsuwrSyFbJWrytHiBiKRZViW04t02wEATA7l8T6ONLZMcNns5xzzgbkKvSRBpG8a41u+x1FSgLZ3lBXnGOPtokK2OeFbt63Fqi4nWQNPGaGVXplwdqh0/PjQjzy86mGG/BFD5PtKf32How07izbxd+Ktkhj2gasq1LbsN+AuNgaND1vr1Vhvs7BN1og+d4J8lZkhMNEXdJYC/mXv3ZbBAe/Q1fccQdfClAnRrB8YHrqSHxurabT5CcnSl4KOcMrYmesWNT8Vximiz5Ik0el0B/Z9PwK1Rkczo9LmSgpMonnOEgp/KGdFYh2PPm7jbn4AjaORpOgyvrhnD9d9swA3UIXam5RsOB0Dz3aeCuQJ+jdLtZyyWChACbi/0DCd4NSVLCtPY9JtHF4RYACW4QKT/5/kG8BufBxkZZwr3vYNc9usYXIRWC1GBrodl6L3I7vRh8OmOvrL/v+Fb/pH/r+Qfyr7/pH/qyRMrmKI2cilA+34/ZZoKHgVHMZr9xnSdsVgrhWrKNZGFfDoQ38Se48Zo16MZrTJTWPJnKeYd0lEQAKsvPE4PXNheqZosOJ0sKrDEjKzXuaopDTNouRK+no7mZPdh21lXwuHWvWjObh/EGe/bkeJVum46PWBVNiH4VsPQ8xX8dYqA5i7tb0Jz7PxvFTBnw4RSVgf2IVdfQ7w4heLCNF77gOHbxL0/AWixKCX3mcgL5XB51XgDL4rWGG38GmoiTYX29DiagtinMokUbGhGX3zoLLtJXStbwiB0xp9KvMj0xk78iBdQ0Ravreqk0kbuZ2Pvn6LKMt+pVIbwGmvdUyoyOM97Woa7GJl387KIUwphuWhQIFIaRJWPYJ2mWtpn7UGo/4OUkbXeImVA4YTrYZfG0REXaK6ml5DTrB/wG5WO8cL+mZaE7pXvmLm9C8x1+5VKv1S8brux/hC+CbjQeFYrNWM84VfI+D5EBHVDzDNCF+Ewr44MajlbU5nfzQsC4W+BiUK962ArUxa8R7z5s2jLEW8ri9L2lOXHUXr0zaqGtOV16yTmdoiHKlHFMeNYrI4wZlH4O1K2wajWF1aGDac3rnQxwD+NnEOhqV0YG3hVvwSxyroY/6SzbUJXDKDTt0o0DrVte1JY9+26Jrr0RmU86BBE82NMFe11KxMMRB9qXonoZcfw//aQ1y0iujLGWVf4Kyy09ZrBxqrGNwfkpPKLM9FfQA8kReLIROK/T00HnbY6eVfxGvl8FyOkkJU0244WX3acTT1CtvqXUmyFcuX00+j+S81S+6jdlLdaj1RM35Co3JDKZYdYmVFKO3PtSf2rIh+15hLeDkAyuyQZRbXk7/e42dK4PFiEV2cban5F6JPKynnwt6gIiJ6uRpij/LKUBjLdqeVG02vk5SZxOAdg4VIhwonlWnx3Gp1mYvdzwvfG6VuYMhD20FyEmv1EJQKv5evSiJJvQUWvZgMTK/M4TJFbKtHqCoECFBZ8Jehxus85rurrpwOHLf75V060hqL3q1PnCShsWhocdGXB27GC+edJ5UyLtDGO+pn8Y38TNDvtoRTEVHEcXsYZrv43p5rd46ip39gp16kGgR4LL6EaxZ4p+FTj/o/sx7lm4tv0CnqXcXnkiSRE5vDsqnLyEwRAQCfVQayuuklBjy0Hb1WOcdUkopav1rmzZvH109/LSSfPq7RMu7XZjz1xjzuPSlWKBea6/jh/vWsmrwKWeeBRwvoFbeOSa3fQuUBca2z5/BE+xfoGbNeuCYk2IOZToU2rqJMrkZac/D68GlGf/EMCz99mVg/ZeC0hV6iX+t6pn/8OXN6iFWtnVSzKL3gQ1lBCJccbu+G3cxwQw2ttK7qbZUHROmWuuZ8WAFfVgF6sYqzW6dGiuoTye2krJS0q/xZVBJAGx38GCYmcWRZwqQ3EZsTS125WL2/Wrqz75tQOscmjT8fqFczrAA65HhIlBfv58VuD/HGvl0sP7tEODdA4i0n9+bDrGLR5gq0l7E7GlaFw3C78nktC7/J3PZ5fDN+Ldlx4nOeXWfhl2aXKA4vFkAkxQWbecK4kC4nu9DCQx/Y2Kr3+Dl5ISMymxFiEivcbgX0ZUg+fJnbBVkrskRUeLdheTVsiIAJ/qKrVJn5I+b6XC6kvKUAUMn2RsYmujaNqtIAMqqVwIVbup7MKpapL/enYJl7ugIman9lXU0TPv/wYfKyxXXsydDpxK+rotPOn1DbxADl/pirdNa7KhI9yciUkTzvDwtCPGM9Hw+vZEUoPNhDGUSs8/ucx744xWRLV/6MCxOQos/XrmPl2dZ8aA2hldqtBEPtxagz9/P2pHm8PWkeshutk6b+Eq+Ob0vZ7Lf5rZsYGA3W6HnKPoiR3uAne6iIlbVMO/UqAInePwhqQ2gym288h7++FHXNeYVOpYK8ZmO52fxBvOxKZLxaq6a4eV9aXE3i3SWvkmhQVnT56fyQVHGENill9Izfhe/N1T/NV7WteK71EQ56qLz28+/EMC84Fg0au4f1XePLoWureLLoM2ozxesC/oUUvvv98HLWoDe5knAzDl/HGthfuGYndnpmqRmQJYItnh14iN7zvuZ97yLOW0UK/hum5mxrcYFjXU5gl8VgW5pV9S8bwZM47BIhKojWVYq9jvWhvLZ3P8V18WBX0tsV+bVHm1yLVWNFbRPn7x9RpawNB3CIOURdMPOrHRzreoxf4i7S6JZWqNFHcyrEte423vQweI0fcevGEFgYQQ+duHYrrs8DU1OjJp4fal2JCfXdvW1kFYnpKfy+chjDtwwX1vbzxWcY+30DLywaxlyHPwne/gq9SiXR6lIrvjnSjs1VK4XvfaGiM8ufnUPxq7O4iRs4wG6mn+Rg/uL76LiuHZJaXL8v2+N4vtQ17r+T9w9t4rntSltTpgFvbQ0yYNLlColym2kP9pQsmrYt4stA0er1d8j0b/Rnzth1dA9V1h94B3Ug3f4S/sFVvNEhFy+D27m1UfxaEcawtBZ8e+1ZjxUuL3Z9hGV9JtElSqRS1ms1+JpctkPzVOXBIwO+YYY/9MiFw7ViH7imgdvp5GXhg2AIkJUJKpUKngguor0e5pSLyb6yoEdpfcvFsjPJz0PyyZrJqNt4RHewJ0CuNoUfayD6lk1McNrNJDtWklbWnT/SXlBU9jkliQu2O++aO3We0yee3bdjJpX3KKu1AG6qUzlmLMIcWkWfJDHhfNNyiq0dJbZ2lDhXcUDQX7EFcc4z0QEAT/q2ZW4AJJeIvilAogY+C4XRga73d/GiRcRFR6P65U++KYPf6k18WHNXK4RPXH1hn+33KRkPNmC91Iy4c6JPcF7zHiue3sF7o37BqvMX9A5ZItJgZbK/Z2q2CoeKqSXQ48H7PeoBMqyQ5xBZDWpjWnFPPnTUg9HNX59SFM8HvvmUTtrIyVBlFZAkSRztfhRbQg7t7Nspqtqv0Hc3yPwS0pGxW0Yy9I9e/7Ld/wKb/mH992DTHQ4L17fBUCd4ubUSAUj160JgbS+eLoEXykRwY/OLKUxcO5GN2U3IVN1lQ0sSFw+3oeir3mBLp8audLxTGs7ynl8lvQ/1JvZWuDC/1zriCVgwnfwNTYmxewZG/p2oVFDT0ISsojFsSZuOw1u5Vzs1vlQ7XGtggNqCwW3L09tKWRsBrwXC46HKmNCxktY89/ZkHMfbYnYiVLa+3RDKK/piWhucBEnKai9JgtDqYYz1ge/DwGBXJlf/6jVolKGTDjRWNxtCpaO8wQXQqJVEUAPmch5u8yq7MqdQKLlBBNO/YmMkXDTDEZOyxydVV5gRcYN7vODFAPBHmeA0OBuYvNqVQopplovszoajacWaWjgcDUujxWrzr8K2oDW49uBLdnEOSaju+rfoN5y3dWRLHXTRg8YmggQuNXRlYQXYnGrFPKqMf4wvSj4lLfZ5SqLfEmitl1d3Zmd4DrJJT/YV0Z77/JLLlqq8vzWBqco4boxWYi6FNISUsrHFHhrc2MqKwkcxaX8S3b6fzFxblXBuJJl0K0wrgfcqxPiHWufFuiuvsqMBQjTxCt1X1WHYTcEctUmcABYDcbjShUdBABq4ywngGOCUbLjDMewNKxkdfZC1tXCmUWQ8uBi0g3vSWqECVB6e1T/yn5F/kn3/yP+YSJJEeHi4wrlpLmXwtNMPp+TggE40JAIKXgZcJe+/eQjwhOQEc3lTX/qGH/SI5lt3NZjqvHCaXxaD3AAtc5ZSd+17cs976HWFRJj3Tfz1JYJ/OsE7iz5z5tFiQVuOXxNpbU5UPs/6CBN7+u9hU6Fb4stuIdBQSMzAahK6irRlKklm+JbhzPhsBmqLeM0qScUOi42ZllrK7SL1mGsAU+HWGvATEyKRhiZ0PeFk13YnxqTJdxROG1P9G0ByktrpKoEojS+b1p+cKwm0iajmswePCpuuLMuorWpGedUTo9svjluWMR9pz0+/j8GqdkOMqvS8WbiHC7f3Q08I9alDHqW2uiPNNV5gqRL096cu4OyQCXzZYi16x12OjAxWbQESf21eSvm4vgeDIi8Q1PUiiapCQf9t+AWanm/Dex9MYlelW2JLkmhU1bCuDl5Ufycci39rYm/CvgZo0HkOtrW6Bb/UgsadhgUICu3A6zc70kUPz/nsU+jKHEbybTDaGxK14v3YaEpF1fEiXn71FNQcVujkhps81GEeIZ0sFPiJve3a2C/x5mO7afrO5/QIF5OQKo2eFC28EwSdfESaT0kfzITpXXmk04eQJwbFVlb70joHmuRVQfAdV6Ox+gYLe/myYOpo5r42lxqdkn6xzKcjU4MykdQ2pDLxne5pW0Inh56kzCQs5aJTL8syh2Z8zb7+e7jWIDrXEjJLQ2CjCGgGQG8L41kjrArViEErh4U/o2CEN+yS/1Co7F7JnModzvX5z7Ni3UQkJH5fs4aHTP8mQnaXPGKF/aW5PNs+F7XbHJb0QRQ1S0cdWYyps0ijlaQ38HYQjK97jC6mN8VrbsjjVAz0NcAFq1gdrZac+MlQmQiPG5TB+1t2DfbBB7n17Eq2WhIVt8TLXknylN/QRBUz4GYKXm6B6oOqezjw8cPEXWrJ953ENXR2bRf6JZ3g+DQ9plZiJY5f3Xaeuk3X58kBsBn20+CESUVQZ3Svp4HjydeoTHQ5Vaq7e9P4JPB0hWtdvRV7C71aeU9aOK7zwfH+zFg0i/5HxN40tutRXPzoMXyKTTgl8Z03+ztY12s/X/mc4VaN+LzWeXvjt6MfNw/NEHQAPdWZvBoIDdEemnYDn534jq1X3sGtTRbP1txg5bF7eFyv4Zloce7PuHWIkK0v0eyqgf02JQ2zd+01LsTC4mB4wAehd5gkSXQ97gKODI8VkeCyJBPfNIcnO2cR4BTRwQAn80ew5tJbqNTis/RzXGFU00+Z01MJqFA1FFARLzFr1zA6fjkFk1n5rLK8Wv2rcik6qQg/nTK42ZQsuv3elZ+/Gs1Jpzj3j/EdWaMusmX4Fs65UV1hq+NDbSkjXpnHE7+PQZbEAFC2OZg55fBsKR4p0Zrct4CwJzJITlXSWjvUgaw7/QOaug5M8hXvx9O+F3lR7U1SaRijDeKz3Bppwu/DBfzw4jJCvJT9pv6iqsm2uapI3IOymEtpEXoQnaoBmyeeISDfBjsa4KpJ1Gt9m/JSdipLKuG4+l6F3RejtlCzcCrTVkxBL4sO7DfVWpL39uabLeMIdavo+vXq97Rukc2wx7aytIU4h5ySnqM/9aXDqvFMrq4S9G1SxlF+cRMxefOxeyByiAnpwaEbc9hT1BWNt5iwfs3alOj0BnZmKHtlSk4rLWpde1FWQhblIW6UUBo/9rQ5x8lOJznaXQxKVRHCmiXjqLjQhDVFYsL5gxJXkEFW2ZHN4jr5ZXU4E4tgVKHndfA+KZsoNXxtcMfguuT14HymGCGwTIkCbyhqIOH4H/jubUszayNalXJBuXJzMMf+6EHD+88Sd9yNll2SKbLeFZR0QwfLWAlzqFkqBbLQS6wUkxsL+KrZn/weCaP9Lwt6ZBUH6vW0vAUfl4vBoZBIH2KHvkFe6mEIUya+3mmp5l7dsySf/hl7udLGla1ljHvoO9oNuUC7Tufxd6NT09RdoPeI77j49I/MUYnrXI2hH7U/DGPcpy/Rxlfcs7qFtSVFrSVaDSoP3vje9K1cKUtnaRXUG0TgDdcW8Zl/Fe8FgaH2TpLf7tueQy1d42m1fQm5dcp3x2zaQ8eyn3n9/dcZtFWkzA5RWTHcjGHAnhG0sIr6MY2rKQ9zcrT7UY+2uQZ4KxBGeqAqBHgwZQklTeBA5xUe9QW1KTyx5SaFfs8rPn/HZzdLe69j/LrxzDkv0q5/c7wlO599hZHXNhDqUNrHsiTjlJ10PtmZ1GupwrhVssy11GvsGrSL+b7igiBLMpU9TpDS5yxxiZ4ps9/sPYwv7m0p0MABVFvKec4fShLAq0Fpx+odgXQ63YmOZzoSJympvqMar6G1apEcMtUbypGtyrWupWUnT2S1oMOFGK7Ka4TvVd9lc8QYlBWvGMJpcysFy+nW9DrQB9lDVuyWpRk1DvgjEqgWgST3JC5nyeD2RPgoExKpleupSoSLsZCXIDJ+TDNc4e2emQTrq9GUiWCji9EmgrYMZuefQ1EZ3dZfv2SK/CZSZlWxo07G20t5bqtPe+4vL2abdIXCwA0eYwGdozaTHHSGR9q8IuhUlnyeizrHnADoGKQEAbbyPsgQLzhqgkq7mAgff30qoVnQLgeq3Xr2SRL42KK4YYHssu7CsV4+SVTlP85UIwz3EYPUD9hPsyECnvADH7u43jhjX+WRYtBZIjwk+0x0sD9FlaqcbbnDFKBLSdYyoAAOfziP8sXzUGuUvoxWpSEtJY2Rs3/g5RSRbtiBnQv3reaDxUto7rVKHJftK14NgJn+oHJ44pB2Ms7nDk2iu/T1zmR+MKxtvd6j/kIs1Nhhk5T8r1YIB+129litf9sKYfEnnxDduIb7U45zYugO9sWItkuDVEmGo5DLDSqcuIHPGwuZ0rSE2h6n+F7lGYi3JNjBNCM0b+65EvivPoUBHtZ+nVbmaCM0vwVVIcq9Os3mRdSufmi+fIgTJiVwXUJiVlYr1Ddj2fzjEDIdbjSHddn0CttM1+6X6dPzDiPCfwds2lcFw7dAlVMEI7XV1vLcogGMemUe+RluY5MkXup7npCOl3m9oJa11js2sM3pYKYxh8xOp3nAz0EQSjtWKj9OS6eLOWJwQpGQwJckiSpjFWWhxSyxi7Y1eVvg3MuQsQxylW16VCrwNh4m3XCU+Tfuh4C2Cn19+88ZWgAvB8B9oRngUPrc8eG9mJLWlYeL4P38CQpdndWPoLQkhmwazeRDfZHd4gSXGow4fhrNsmmvsKpSefdlGRY0X80vEfCIH+hQxkRVKng48iTnY+FkLPiVKxPxFk0cj/5ewIi1TvIkEVgmSU58tJVUNoZjk92oIxOfYG5NCvVOCKztqbzfleeZG32ZMjvE3IR0Ny6RdrbrGG5X+40fvRSrSfksNWqXHVXvgEaH6BMfa2yC+qMFzJs3jzrZbVGoy6a4ST2mRDgSDSo3OkyAWimEScWusakCRCDfszXvkaCB3KbKeOsP2aeZUf08S1t/z6Zu7wj28zFTE7b5lrooV4PEWKtuyPNUPmdg6fBhRNp/U+hUlmKSrfvxKQqnyr9KsD8kJBq8GvCJLMGm9rBeOJ1EqmBnA/xWI67BKhU4JCubIuBdo9InKXFepf/xexjAIAYAHwAHgSBgPDCCv0/4nbitfxDQ139CjduzXGoaiDETHIiJbHDN4bEBpdiSoanz5N98y///xVO+4z8p/yT7/pH/MZFlmfDwcIWDkS4ns5w6CuJvYTWIiZgd8RUUzl7KsW7HQOMhAXQriRslRj69NhqHU3TYrvUpoC46n5H9egg6gNHJy3gr5hKhPiJCAUni0d8LOFUwjBC7sgeX1ftOI+Lsol/cj6T+3B76/RnOiGaFNJGUjk1J7jYWdmxKVlkJi9PHY3Hr5RPhLKNNelOCKoJ4JVOkbWobEE1Lewp7G6DOKXI9A2BvAJuYPAXwqd/PggG9aB3mVqWmMvBRZhHbHjrDzF47uaFyo16SJFpfbE3hz0MZn6cWgoQxpqO8/v7rfDLldZ69KSYZn/C+SbONI8lc34ZzFnEpymu8zpJgyIoHlYfeM13a2lj6xB/4TKiCeLECr2uXC66/i96i6I9k82lFt2vtecAHTnqwCXUqAz2Od6XV+y/yYKNoxH9bncixYl+o8aFB5bbp2hrpoYe4vyNIliTeH/ILHzsfIaXnWx7/pMgODxTBPbdEI0XvF83avH4ML4BP6ocqdKXqQ3z3w8sM/vwp7qkQy9FUkpqf79vI0Tc+wqRSBnydgR2ZvuwHrv52lpuhYiIkw28YM0qd3OPtpL1OPLfeXsstGwzJisJs6C/os6uySVwUDOlfUnBT7Ifp6V0FaLTbOXwhlPk/92RQppaL9crkq0blutHvzJ3P3BFidWeRKpXUIa6eNKnlIt/rVv8r9PtpAjf2d6TYKgZoHgso4Rl/aCICLwEYbXqMhYF6JgRWeaSjmlbs6s/o42YE7b+yAa/qd4nKD6BFySNIkkR1Tc1/q1lyRYWWZ156Eptbv0s5oDmT9XnMWPAVo0YsEY5VB7blnQObeS71O8aFi9SnEk6CVeAvg8oDv3iKfTfViVBsA5NmjEL3ZE04cyoc/Kau4aI5Au1d/rG3o4KFwfDU+18zYf5SVG4JokkJs/717x11Hvq4IeFbFEp2/WUKPFSPBlQsZ0lEDU8ZQevBAWivPsbpWGin80w1u7oykKtml1OlMivP71vvilrqU1JpGtBEoRsaGE2nANfcadooBmS/CNJTEVRG27NZyDVionxP/D6m7RrKiC0jxEQL0HiuDSV/9KHVZbEXJsCTRRaeL4W6m2KgD2D88GzWPRBAN595is9ttU1IO9iejiWR3GcQ71dq8SK6HPHh8ZWP446JUEsyUp031dPnMeCLaciS8v1NcBRhnbAZ64STVCe+IJxbY63hlYP30PjcW/hW1ngc98Am3zEseanQWwag2NaZXZlT+OWqsgrI1yuYrMpOOA93IiYvBo0brY1Z7UfD7WtdF7QcyQ0wVO8/jIt1SVxpVPFjqYdnWb6Pn9pt5FKnM2KLF7Uvr91yBfeiz7VBg1iNHiiCxwWRKk5DqZLCWZJcleqf7zvG+I3is4pU1xNq05LaOpM2CWKFcrLWwcwoE1kdCnjA31857Kpz5MTDDKPrnRck9gF8bzawOX40Rc3ESgZwrY/xas99AWRDMN+XhzOzDDI1XRV23/C8VuRaJSLyYphYKwZ8jTWBBO3oT+GZ5qjd1tfWzaczMEdm89qBLN0r9oJY1hDEwduVstdUYtYhyi+KnhF7+HngSGzV4joY7hVFZeYCluw7hiNIDMwWmU201YH+wlyov/NeS5oA3qhz2Rs5sTnIbgEDH9lBghr6PLyNvkNEF/uAcyjp4QWUB5ZTmSKW5/3aGM2GWUs48foiDlaLz7pC2sPa24y+nhzLcEsBMWpIv4ty/W55pjaS9ytghUlpI9R5X6Wsp4HSCz4sXLpc6Klzpvk97Bp1jezm1zjv72aHOp0YQ+pZ+vr7LJizgOx6ZWLAGdCe0dub8/ms59D8LCZxMESwKLsHW+rgUqNnCn4nTq5YoBwPEWOnk7adjES36Qp6pW2/o05PXXA6rbpfJN7vdYUuregMfWN/oGFgJr8FfYFdq6yCKy3cymOpn1OebeRcvmgkHKhYTXngUbQWJ3FFom0ekPYEUwMsxGeDQxMk6P3KDrAjCvY1Qo1PiqDPvbiAcV5mXgsEQ/0dpofmIe3xr3X5JNWRhRRUKauuvGtfYZjBBVYI8hMpFBdfe5Jjnz9Er0MdGGET1+cXoxJ4+LvhLLjUFbVDXI9WRViYFwSTxZgUAHG3K4c+rUS0m6y1fD/rHZ57YAf3uxXESMjUe9eT2Osag5uKlMFnb7VCMutouORNRf18hU5uLORINHSdsJOxg86icij9oZbVO8nsWElD/6PcDBcZEfTmQt6atp3MMXsoRUyAAnSK2kac/xX8PFTh15Z+xachEKIGjcFfoRvla8Lx2ufMWPQZ8ZLyndY5Ghj01K84vBspSEyh0W2emNTBSG2uoWnwIi5/ufC93XV56D+fR7tPNjDAW0xoOG/TwDcm5KDyADDs2zYBLwnCVDrwYKdrZDOJgefoHKVkt2j0ac6yajhthnNmZe9WgG3WRD67EcyFPR3YlymSva43gWnQQS4Xh2H2U5bS1FvqORu+h/ef+YI5TS4KNMt/Tam3A6FD4kcek30Tfq3ilT37+Pr0F4LO1pDBS7HnWRAME1RKENWq4mMMPzMNNXDaLt5vH2cUpXY4b4YQr+aCfm32MFrcghyLVkhY6GUHSQ296ZgDM0tEn1nX+Qe+q1KzPAz8LVcFvVOy0c8AE/zM4jWrvRmYB3PtuzjXZKIi2feXPfyQr6tCTnajc/VruMHuFrWcLPNl2wElCAQgwrSXBdUxvP3uZE5dFMGiT/ls4f1g+CQEDE7RD1sftJN1EfDO39hHm2seJN2sJrfR8x8sqYJZZaDyDWbunDlsMZv/j60Q5s6eTUP+LppXl5EVlUdRtLjXXq9dwKvzX+CN996go96tYrGxiBkaNbafR1B1vplwLMB0fydfh8Kxo7M86rfUQy8DLPMXE7uaxgLu94FGB6BT7tXekpPC+FvkReehbVDasJIkYb/9Gq8btx6He+V17AM8vWQT53d2Iyz+jj/93wGbPmyFjDMgeQAvGhzZ//q37BQB4GWhVZSebkmHMx2Efs4ZsSeIKoogYu0YQlGCHsqSX2BO2hSCmhTQNEAtzO/X5Cs0+WAJXzzzFTkB4t5AyX649jGcfg4uva1QqVQwr/d9LOr2HMea9RVd+dvvx95GOFaeCrLyutV6f3ZWJLKqFtLNSmCCNRQ+nL0QTWQxTwQ4kNz6BSZfiaTllZY0GEwkeyntTEkCSXJwvF6PVwbUa5S2jyxDr8BM4jUwuwzMfq2E4z39+1+f6YOZtjWdrenTUeH27IM7k6UbzPlY2JeUqVyrIofQPscF8MuzwfE6ZeywZcrjmBtcoK2n9yeRZlLuh4Mal5GX4KK5nVOgBJIC1DoO82naWPSSqw+pQlR6zpjVNDohTuPyRd2lg+9g2l85RuL5K2h0YizBiZ1LFpdddbf89Zw31cGqapXAKtPXazphxYk8Z1SzoJm4V3bWRWKuKeSd3weyLU+5j9cZYllinYH/y8vZ3LEItRslq3djDv1kHXUFoazKjxfOjcNMfhP4NAQkp+gUq521bGi9jlE+4KtSrpOzA7fTdM4K2owqxCzBVu5U9j0KvAL0BgYDq4HdwE+4mmT1vq1/FDB72Metsp5GB+QnwJsh4p504tbDNDVmsLYWrGoPAdf/JeIp3/Ef/f7/kW/9R/4RwG63k5mZif0uWHWFHMJP4TcZ9/BOnk0UF46tkh9btdUM1qkwOqoEfUFiBWX+laiu36LSJCLIvBNNfLRwOcMGHhZ0AON7bWZM6iJWjhaTan/J0x2foZ1K2VupStObrNRI4jtd4d0R+4RjVFGf0KTnOR6OqSVKUiYsSh1qVhTpUZ1OoU76lhqz0rluZ7vMzLe+o+u9x4iMEQMKqpprrGmSxr3e0Krl37zSTV+A8uNwa52gctjLaRF6mHf6DYLstXcUkkS9PQwqvPEuCsXpFtxJMV1h+cQjWELKKNvVXaD/SvSJZswbq8kYvoNcvTK4AzBcX4Ah0GUE5Da4UbKayjgYepTnA6Dc7hkxwvXFhBwMQ1XrAekNvOGwI6VDX0sfBW2fLENAXU/ybVBeJ1IbpGjqaX/d5aRtk8Sk77q6ZDSXm1IUXiTQndCYz+EYyE6AP0XQJ9gaeSg6iD/Gv8EjbR/2OO5+hk7EqOHlHiKdlSxDVMV4Gq/9gI/fqwqdXq0n+bA3xTnhnM4WkcffxG3iqettOX85HqdKaQDdqi8jzfoTwbdimHW1iXDsoBaz0f64nM9nPcf1go8EfXD5Qf6MAo3kwKESg22mwj1cjYV55ZAR/oCgfzU4gyuxMNUPaLyDHJIMEXxzpCmWLQOJ+P5B7NVKIyfUcpVPi5N57LexBFb4C+dN197Do/5ZZD71PZcjxXHlOvU4/Gox6U0eN+FXg3PZXg9tPbQJAhidsARvtYlxGzwkLGQ1y2pgaQhsjVHq/bO20+FUFBFjTjBokgs4YPTz+281S/a1+BFWGImXRnldf72Gj5XAfE8gWrUXmZXtOVM4mAqzSGcs+cSTkA0jfOBItEiVXKeK4qcauO9mFNVeyjkoSRIT10xk6orpRFSMUST76nSxPPTNIMav70xMjgOzTmn4+VpOom7myiplfnWf8L3tNOU889UzjF+VzNmC7wX9T7YWnDDBV6GgtYvPo9InlY118Jw/eDWIiYW3iyNonQNeGeAM7XdHYa2lV+cMBi6dzxePzEFlU1LUmWNf52vLeh5fsJL7JojVvPWhDZjbXMOXegrLXxP0DdWdCFRBaoubHitx1ua7AszNOnhwYHFVVU3whW/EgloAJqW+iJe6CpWP0lEM6P0zl7s05/y6wbywS0xmdOp7HN+QKkLKQkjRKB2u2MT7WbjD5cwUZUcgSUrUcwdHOl/2yOZAaCR59WK1oyTJHN/uCp7UaUVnD2DVxfdYfPxHj8m+sNggPj+5gh8uLFR8rvaKZPnVY1T3T+fBmT/T2ku5N9zXdBSn7xvEuied5IRuoMasdELrNSGcjZQIK4gkMlMMvudaLjC5NpQMZzQtOatUqrTs8tGRPnk9jbO/QvbQw/PJoEM4k+FSLFAnVnFSchh2doJDyiS6xlnJvL5DGJy0DJtTrHx6p6YXP12O4tS5JF4paSvom2TD3Gdm8tHrz2NWK5OYsqwn3+YK8hUmeA4KBKtcyOEno86KSuB8LNxMgA1xYrWBzWGl3M9lE6lklcLukyWZMz2PAFBaKAZGt8bWoEnIZfuQ7ZSrlTZZ16iuWDVWsiPz+VMj9qu8Xn2GjSN/5/Ppn7Mv0F8c9OX3WNz2c4J0FRSaxZXXt3oTGx9UE9t+IFcrxev2t1cz2Q+eDwBMd8ABkixzqNtxGuYu5bmWhWjNyuBmnO00x9Wh9D/XgQkWcZ/WqNQcmvIjzd/7nI468X6qZA1XfKv4Q2rA6mGv9VEH8KgfPGf0jCBdqumGPgMO5h7xqB/UZR6vl0N9gJLKXmp4g8VTXiO7az418UFCo/lVtikcbbeO78etozDZLSngMLMr8WdKUq18Emuiyqy0gWUZbjS9QVpyGh92EmniGiUtL1mPMLIQztZ7gMQ4rNzo+xa7ogCth/eq+jL8rIVL8wSa8EcLI7l4sjmXTzfjpKysmspzaJjy2iN891soZ3RHqXFLfF1QxzNgfxwtlj7B0K0iyGk4P/FizxtIsoPgWpEtxBE6iJUlkRjr26FTi+91gyGaTyoh3eK5SnO2tSXJ2dA0G+pC7vp+eyMRdS56rDkzNhFoUVK+n3S04pde+5jw2Ue8NOm8cN7+xfeja9DSe8JhgqLEvTTbORhZZafxVGuhqhvgYKOKq2Z49W/abNWFHCLwXHcajUdFpbWWoIK3GKS/H4NNOe6ptSXMv9iNnaHNOeAjtgYYMuR7tkwsYseMr9hla6vQyTiJU0lc2tQP+4VUoZLSrPHn19WDGPXma/SziUBTSWXgYEYYpUdTKKq4JugB5u4+wGcnVhCXLCZ2NRofTppgdEYcBn8lOGFpzFl6xpfT11zBBdopdDl+nfj4aBJyvYHPB0/nUoWyuq7Aqyvv+OST1OUSw2LE1gATvU4zJxBGhl1RBOIBsFuYE3cPXeeu5L1JB5CdYsKjPQeZZoQ3nC3AXwQclQVOJ7uqJfe1Xqb4vC6kH9NK4NFiGFEg0kOfsMdwqsSHSy0vsauJ2Pri5QoDJ8xOwgus/H5GyQZSV7iXXjdnMOI2hsMdvKW3XGN5KLwZBKP8qz3SeAJcLunLpZJ+wud1XnG0vgUzSmC7U1lVZcdAgr4CazJM9hfjCK28SuhngGhTS7w89LOUUHE+Fg63OCHstcaiT9k78FECVVBq91BFH9qHeeUq7smHSp2YVIusWMveaFgeXSFes6xmTyN8FALzk3Yr7CoJJ28Hws2nXTTLdjeKXJ3sJFwF+WvvpeG0CA7wtmXjfb4FXE9iV9mfgn5N4xBmlED3XDCpRUP1nC2RPxtgXIFnlGys82Ve3Gjl2z1iZSnA65UaVtZAbHr0f706Tatl1s4G1i2eQI/jXfFXieCsPpo79qNW5bYO+jVjdJ4f2SlplIR6bp/yQIWrp7m14oxHve+AHZwy9icqVaQn1dVc4ftwGOkDaofyvdwaeYPZ/hJxOXE8YlHaFzqVjqzQx2kwNCD51qNyY5g4WXiGg21e4XiX44yvv/Ms/7tgU8kMGg/MSDfkPqx5Np/RK+fxU7ddCp3dbmd58Yv8Nvo3+g84S2fzHeYCWZJ50R8aO1zkZMcz5MnKqtjjhRfY1ullFtxzjWUZu4V3J0Gqo5UWWmshXO0hadniVRiZBf12QKevFCpZhtVlMaRZ4JcIUFUrn5dv0TYWqu+nvzqEMNlTT2Fw4kDGZePeLfdqT1LfphHnK1+xuedBwS94u2khqd0vox8VzNBAJcBbkuD90x/TrcBEoxNklfL9kGWYeXU8xkz4qBJsfsqks5ZKnuvyOEuHNicMNzD/bekcuYVVYyKIQmSG0ao1HGiEMzWRyvVEF8T1+gisThfQzyAp77cU3ImaYFfS05DeBEmlvGdmlf+/aOY90RGrZTWDAopoTILOajfb2xDOvdlGArJg0pYrBPnFC8e3kn5m/8hujI08iuwU+YGn+9vornex/Nwtvcs2YkmCH2vhtRIf4bhH/KbzbH4El37vhdkDrfV9upG0PleFc31Prucq/+BGdSbrmk2m6tsHuLD+HiGR2NpRyEe90znx9Aryw0V/BknNmuwhPOQHP8WIhpVK5STMJ5uPKmG1SRmjS9CUEp2cx4fVl+ipUnEUV7LPiat330zgFq7E3hLgGeBToM/tz2fiijkVqUBWK8EDYXIN7fWQY4Uqu7jfjZQP8JAvvHphBlFBz4jX9b9EPOU7/pPyT7LvH/kfldpa5aKmkmUSbiZw5O2nsF0Te0nIdplxm8bQYd4cmphFp36Ady0+wVU0BFaKvWeAT6oqObO3PZm2vh7H4zf0F2jyKJpAEVUH0DtuLfuzJ3PT7wPF5xZnAxc7eWF6bAOnPFBddWyVRpixgXe/HsZpRdthsHnFMEsuwxRSRhe7QajuqAwczqt1CcyLGsjpwCUeBp3KZYcfS0Mh1Om5DxwaXwhoBzoxgJkT60PzWwAOnDV3ObFOJyrd7wzcncCoXYMxuFEIdEoazQ17X2gwEJ8dL4zbEjaJybkvk9HzOE00Iprv6ZpunHvoZ4zTVxHscAtqyRrSbT68UAqdckFWiQYWgR0gehQU71Mg6/8S+20U6oFbBzDdVdmnspXxcNhVvqqGp9PFAM3D3rm0Cqml2r+Ss1oxmJaqG4il2xmm9bxBosPN+dAFsyCnPzlWCPfUiqqxAPb0gy1JrnF7kHVRF8hJgB763YKuvuQEKwYOYnD8TzTzU86jJqo16CdcY9u927iYIiaFtZKT8h9HM2n1JAHxr5GdZHY6gyO0jBvBYuWeLINT7UoWRASI9F+hscP5MD+FWlMEOTWiQ+ZU+7Gn0YXwlbRiICRZW09zHSwLA+pv/uvzv/oPZDbJJDErEdx6LxltBXSpCyT6SnMGl4i0ChpZg196E27cCqXIID7LmQ2tqJq5gmWTD+NrF697Qn4qL5YKH/9LTpRP47MTKzDZ/oavClhbBzvqlAaSs4mGNq8tY3Cf4/SL+hKAURMnskovGkue5Ac1NDMkU9pdRdMgZZBcqs/my0A/quxwwEMxb0ltAYcTh9K7fifrKj1lpF2yox72NYr8pWXatjxUDOfNIjL5z07DiawKxbtezdRAJdLbqTYQeaodHXf3p/3l6QS6VVgENmxmeF/XMUWR4rOYYrhOcKTrYbQqEd/3Mk00k4qgfx7YtSGCvsY7mT/q4X5f0HsI7g8s2okdaHSCdDdFnbmMj2OvE+5ozqqLH+B062sjaQxUSz5sLH+UQ/XKJA1AT101YaP3UPzeh+T4iu70htxfKc4JJ9zX7HHPMrdoyaPzvyK+ZYagA4gzGPmiCmZV/U0ZRdGfEDEYkp5QfKzSyMg2G06cFBjFiTKhoR0Hp53irdXzeDBYfLds3kYWv/AV8+fOF5zQE9q2jLxsJHp/ENKtbOFYSRfE0W5HudL8CkkBoz0O+3p5L84U3kuyh1YTAwbA44/D7dYrCpHVMlc6F3Mw+ToVsnIeRFXu4ckbK3hwmUSvom8pqFOih69Xn+Vy8gq8+x1nTKyI3NwQcoAmi55h5bwnqPEABJnRdDmrk66y0KsYWS3aAY0G1/vWUgdYxT2R7J9cv5sr0fMatY1mwQeJTZ1OfmdxjsiSxO6Bu9kyYgsak+cyZEO1kYabATTqlcFkObAd3fJgegms9FRk2VhEbwMsqYRt1Z6R9Ysqwe6Eg/UiitZUeZ4bceBMhsmODcAdu2+AVzV+rdP4c0wKfi0+E47V+NUT8NK3nOh6QgjoejnqGFsdTkiNkWqjWEnTSlXC1Oh6TCHl4A7KAcpVRrbUQeotyHG2FsetN1DmsDMtfg81pfsF/bNc4lE/iMwC/O8cLzuddNFD7Jm2XFw6Aala6fQ3apNYfiqBP1aOYPUXIuK/lXSeL9RG7sltQhMPVVUboo7x7e14qaf1YrRuMt+FwcsBnqnDprafytl4LZeixKAtgON2Msz9fl8hntdqbBTct5yCQT2x2ZUByNd9yliuMZLs1BCm9leeVFKxtqgzezf0o/PZDsLjkG3lPBBi5sJjayiKENdm2120We59VG6fgc11MMgL3gw5L6o1t8dz5T2BRlyWZPZ3Pcavw7Zy1OK2l0oq/CoD6H6kJ3E531JQq0w+ObT+WG+6QCu924jrs132Iq73WXSfvkNVqriXbpdCWWYu4AH/asK0YhC8zjuZTXXwQTAEutMGA1WSngwrpFmBu+wqrTmNkApXoHqhVEm9W0ViphTPMpONhSU92FMgvneJnQthTAwLa77lolYM0EyvNZAel02pSY3DQ5+3r6q0tMhx9aPyJDPHdebWm0eYO1ns74U+BFrNA1s9lClpbAOqNBTv60JRRS1XPLQ0uMfvIZ5vf5DjQcXUSm6MBz7xjD31BVdDC/lKVQVq5X6Z59+T7QW+SFYN9xV7oJL1iuL4oocIXD+KWwWe6QQfebk3rUZPoVkL8fhy73Z0yYU/aw2C3bQo+yXmlcMlCzRKynHJskRUvmvf8JKdSG7zX62SqfKvYnnPfXysF22AlY39mPHBg7z22JvkaccplbZapsofk9Ish3c15ag8AFRSAhNJNEYzu5vn6qQXXoDCiMWouyrnkU6t3IfEPrMuXz81I5kQHxF4ECDFYOl9nKj5i4gIU+7TaoeVBA18EATrw8Vzqy35PGF0VY6+WhjmEcDSPOQQTQLEflEAstabSxb4ohouoayWCdZcJUpfxZpayLGK7/Sz8V+wNxpyW13Gx3ZJ0EeHbiVGDUctos1l8W7DL7l9aVU8mxe8fhL0laZyCp1m/mwAsyzaZI0lmwB4oJC/TXCGqSBI5Q7qkXjzrtCAJCnfLVtgF0ZuaoOq0UBhhIggLAh8ihWFrr2/uIVY6XvW1oovquGYCRyyOMfyfTbw+r7t6M5e8Djm9m0a8dJUExTkeU8LU0nYk2Hvmm//69VpjY1cOXiQtP77ib3UkmnX2gl/c0WdSPd5X9L2nXWofdzeLbWBghoj8WkpPO+pmhzo1n4O2nTYruvkUT84aTCdhu0hrssiQWcN6MAXVa5KnhaNykrjHQ0BpDW7zj3PbqNZlEiV6ecdhyG8mv3xNoJsNxV6a0MxwS0vElzrR5cTd9Ki/12waZIPhGrERMqZmj2khSxnRz0cNYnvR+ejXRi0ezjjI+tItCn3zGmFiURfSeVwVDZWN9YkL1sVvapCmfr1AMJuivTm0xzd2TprJve9Oo8nbCJICl0Q+CRAWF8IUYIbVSpYWdSMjXUwxhtki5KqUJu1gqciDzBnWxqv7hHjNlXl57nUcR32ZFgWqwQ9xOmf5Jedk9iSEcInVS7KXIVE9qXp2AreHvESTSQlBa4sQ1bYImLU0FkHWmyC3m6Ko+b2luBus2moZlCT74g1XiOA88K4JaeVlOATZFa2pV5yA56df5XP5a95oxzm5bRXricOK3p1DYO8IDMe2qpPCeMy365el9V23AvGL/g/Rs88F7h7qFG5tgN01L/Lk+E3+LEGypyin/VXzz6nZPe4tv91LTO6TEWuEwHLSRonvQzg3iGiUhvBH/VgdYLs1CmqnwHqnb6okm7RqG9k+3mxrca6rUMouRDP5hGbMUcoL1qFjThJplRjZrXFAm6Ae23sWF7PmEaX1Dx6h4qtGJDVfHzjIXbWg90pXrRK70fTq82YXXbHhv9Lni+bQq888L4M0TY783HReE4H/ppx4cBrwCkg7fbv125/DvCNGupbgdotmT1Gs5fjMdA/H76qEMFAm5z9eLTY9az+t4t7vuM/Kf8k+/6R/6ukE5f42qjFKTlY3Sgm+14rtWK87EL85Lsh1AHe05UypXMmb40/5HET2LKnOVu/HcmPWzwjM9H6Q9fvoJ+nnn3wUveH6Jx6g47DlGhAa8V9fNfxSZZeiOSTPHFcn5R1YkWhAXW1L1aUjuBfvSSMkkQ3rSwEcHqmzqJk9xKCr+vJdIoJoCK7k8WlNZw1gST9Dd9gYxEkPQnhYqLGIZm5ZgF1Bkit37mjcNp5I240WpOewDpf+nspNy8puCMLj2yg+cJFvPemBzpACcDBoWh4MVC831V44RNRyrPtczA63ZAqWiNPVXfg06rb5/LU2DVyCCQ8AudmQamIsExwVvJKAAz0Avku+hmNNY+3Uv5giBfYVR4CrgHvkzZC5osXPvVYUbggeA/j2uaT7rTR6BZQQOvPycwvicuG9lkeaLL0IRzUugJKxVmeAwa/O1zBo7YFYu8Dk92ETV3BgpQ/6VurpHUKsi9mxr2/UBbXk2FB7wrHPp87FQCNTYOvQ3m/jfUXyexcQeUrXzC/uVhB5Jf1Csve7IPjic54+4nOnG94LwqdtexPPYu67g9Bb/NrxthCSLeCzi7e8+90zxB3EwblA3eh0yRrHTOaVDN77mrmfvs+xlBlgLIiaBhDkk6A5KR7ltirp5n9ABMudKTPoV4eqRtVshon4HDisbLvijmMGA0M/5tc3n1P98Pk250P53pAbjrsZEYbUQM/lfdSqDL8mvMAJYw/HEWvIy4E3BNTp7LPav0vNUve54C35h/ji2ffQHanrLTV8XRQDT0NYGxsKxxvKj9NZquLzPSHDM2v4hdY63i7cTpZNxZi0ouGskqWCJBhedxNfBqVTmqT4o30m3SEmHZF1KNMDKhlJ99PXwpaO6P2RBLtRmX1uyWETV+PQfaro/eYA8L3fmdqRtFj69BM/o3SUA+c9ZJMltVFzeHrJSbVJEni51qIvgk1RrGa7Onmr7A81NXoXLbddU/14Xx6cQfjixxMl3/kZr0SxXs+/zc0hnu5trqIpRkihfNjpeGELHyGHUeaU+gBOysZ/fjqqe95qPlRj3Owd3AVr5rqmF4qOtYAe5rGszYcLBGDPeoZnQe9xOccVLOR5mfSkJDQxYpufpbzT46ojjH7+AJKNG70jY1FDEhYztDGN2mVu55wb+V1latDiFv2EM0uGIip8dATR1Zx4t5dnJy4AatZRFQDrF0LP/wARg/M1HLJbsY0SiSfcgOYmMrpmjQQf/MNPj8wEpPeja7bJ4nrp29TbR7tLOy1Tep2sVkVTO/YCoYmiYGlK7ZAULsc7jzJrfrfWsND9f1ZmN+MsKIw1B6acKV0+gApHQJy/SFATDCRMgP67YRkZYBd6xtC3MVuOIH9HlgMR/hFc39aS8ZtGEeKB8jpNN9mXJnwCwd7HeQPt6rYv9bFpdXw2bn3xZOXHWddhIuO7UCd56DWwqIo1BlwLPN7QadSqTHdvpValXK/fCe4iGmXuxOXJSFpRdulUy6suxLJ8/ltaeOjvN/qhjSeD3SQpLezWMzt01nOZsyRXrw87y1m14s0W0fleEYWQo4NEhPFZ3XeqmdOGbTWgY9VBB8cIIpXy6DQDtwFRpKdFvb6G6g6l0pWQhb2AH/FcZJvJ5YHuGwDp1lMXLVVpdGwaRCr5j/M/ixxjjiQedzo6vcTYhMTTIHyKU6ZYEyWZ3aKNnoVbcJa4xM/zqN+UsZLOJPhabsygJNdFIf1jZmMPtmNLaJpTsyVaPJfm8nUxS/wcLEbO4CsIT1/J4c29WHrtyNwuPU/UjVmsyocvg2D7h7wLqrGfJzJroTxAF8PoAdZxdhC+KoKzniisveOgUGHIW4CeClfoLeCyhhdHcaAPQPQlyrngey0cmTuEvQxhfS/1gqtXVn5p8VOVsezxL65lJq+Yv/F496TuO+73pz++n7WV4vUkDqtzON+8HV8Fg6LaFfJkkyoGvoZQOch8RuMiRQNRKhA5bizjl6vL6cmzGXjlS4fh1mrTAzIkkREQQSGZR3JOKWkHQPoHPguDw+azcEWzfi5SKxGr5ayWPXwKjY+rKN1qBgkD3e2w1cG/d+1Jyk/je+hNrBvqKiTNS5fpe92CB+gUH0eHcZ3j35HWEEe53OnCIfq6jIIzPEisToAA8pMoyRJpDY8w08PredA3wOCT9E6vDUZTU2YAirZYhbXuZSgFHRO1xoVGuq5z2xq5FX6tfFcxWPQahnmDa9GFKFyKG3g4oqPMF9fRpvyJ0n2Uq5V3tYKgstdAc9v94zCYFUGAiPtWby0bTQ9lk4lzSraRRmOBIKvpKK1yOwq+0GpVHtz3e9LZqX155esB9FpRUP3ul83+mUUs/bQbLCIiR4f6wW6tcrGL1ZpUwVWn2B1OMwPgtcDxIrxp3VneO6ZzTw5ZzWPGMSA758xPRn03hyMm4bgjxLMYQ/tSdRNV5L7PqMIBLX49yEgE2aXQ75V7THx9Vqv0Xw6pD0f9O8r6NSSgwAZDBIkerdV6O4NfJx5SX8yqQjW1okgk01FD7O2FpZXg1nyF/T3aWIxqmDnxUUig23wEB5oPMDnHT5kaMCj4nXtas/F27kdnVa8qGWq7vhmwKZ6FKwaANgasSXBFQvc596KXpJoni3zw+vzWTh3PrJbixSdWkO1sZroDlcZMlRMTKlVKrL7H2Lo3O+ZYBQTnCpZRgVoJZHaESDakEx47SD8TJ7pMPv5Pce6+/356vklHvU3u/QBoLZR+m9Vp9lNZmxtr6IKbKDWKgK8bxLHPZQw2nGN6+5MDA4brYJH02z2t5g7ims7gNFSSITaDTx4t5ydBenfeNYZIviqGr6sllEZlZXAtZo9XFqzmN+/n8Q5tWizdeh+BG+fBlZ8PAGryl+hC6g5z7EYSC0LpWXNHR/tvwM2/VED3TvLeKvF4L63s57xJ7tz5Ztp2Gxi8jYp6CRe/nV0yHew1ThJoZuda6C8IAQfi06oGE8o2cp3vq7xGQNENieDxoBvjWvvP2nx0IbHUgl12eAQESgqFagcXvxQC6oMcEa47UudvmLhmc3U2PTUelhjTTg5a3FgdUJuo5KGvKyhL1d+TGbw59OZePV3IQ6xp+x9Pv/xVd55ZyFpttEKnSTBwMBMzsXCiVjwcyrjH7IMnRr7M8sngFPRKnzKDin0Nk0kM3ee4oUdZ7gpPymMW8bCmGafkF3VmirZzSfWBWPTJeFd34qo8knKNTRvExXJ9bTRwYIKKHIoY57HTr5CmybfIqnsLJ66C4NDaT9rbpcVvxMET3igsA1RrSHGUMYjxXDT6WbHWqpY5BfOYq9mDIncjeQU/Z3d9Vr657nGhl4EUT1XouL5UpjtFqK75t+LMYUwKwA+DWsQWGU+qR3GpyYzdpWdk4jv3RdJf3Bt1B98OuQSbf2VdqpXYw4bstsTUhxGenI6knvLDt9E9pe2541AmBjsIVmNi6p+SAFMuym22pEkMGkLmOgLyZKySlMlyzTXgr8Z1gFbcFU3PwHs4+/79f0lJ4CjDhjeS0wanXe25tUysDlB6x4PBTKlJlQ64IPEffjaPa+T/8j/9/JPsu8f+b9KekQ/SFl1NJJTxq4VAyE+wdXE9DvJysdWUqkXkW17bsxn09f3sfFME4/9Sj5qdZKz7c6yxui5vxH1t6BaRND+JVKvX4gdPFdAfEiSxN4FjzL2mydpeUIMskgVvqhNei4M3SWUvOtNefwSrKb24V9Y0uWgwIksSdDs9D46Hi5HzhFLGWrNtaysgQ65kBQ/zPPAL78DF9/wqNKYSnjQx0OfOUnms/I4HL1Oou5bhcG3r0ItyxBzZQcZ098kK6cfId5KA6uy6ghtWj3Fp1WwqS5e+N4OPlH4LJ7Kyy9P45Za7Llj9JMZ4Q0v+SP0vfmXBHWCHj9DiNiD8V7HTT4IdtFpyvY7KEqHVyIPXrkHbxmGxa8WjqvR9+LK4ZZ8s2kCM1Vi9aiXXMjxP7uQ/vMQilViYpe/aDOcHpZXjR+Drl4lNRsmXfJAYQSsdcbRLgfOJYs9/Rx+zWmRA59UQrGsLHn5pn4gE3JldqWsopfmB+HYfvJb1E7dTNOn1oEbglfyiuTzG0EUn2yJV71o8J9tqOO8Gd7uN5QE9Qsex33drGZ5NdRLYuRVJatooYWrcRBevFXQzxh4Hzk2uCbHuBLut0W21/OWn4oI1Gy12HC6UV0F+0QTkv8WlX2qKYwW+0lFONJo1zQP7zpfemeLCY1HvbMpt0PLHDBrRDexn+oFPveLZLm/ZxcyNhZmD5hJs3yxQhTAavPC4gS9pAxAqpzgV+PHkF/uJ/Scy3sPDw9nwcKFjNDp/n2zZJ0O/wGpfHY1gY/WTwF3pKxfU4bv/pEZ/vB1tFgB59QG8WsdPOALXb09GJWWct5s/QWjA0rQe6AWC7FfZV0ETPQFp1nZCJp+f/KV7SW+SR3FhrIRClWks4LitvU8OHkHox78GdmmNPLTrTYaDA1ExRbzgF4s5+rZZBWfW+J4LekCOb5itHlkw3c4kiGh5l6ifMWKxajKY+yNhs4BMbSPEdeLdsE7eMLooiuUq+9KeKsNXK8ezAXpFnWGa9jdqK68ir9l8s0wIgoiaXNSXPutoa9Qnh9KiystkG3iOhZkHMJDe3ox9texHkENgy0HSXn7ZTqc9LDWAFmGZnxR7api9SgOM1z5AIr2KD7OPfcxAM0GnGBtdJVw2LjjqUx/+03yDkXTIMcrdJWlp3m8xTO8ZP+WleFv4a12C97LMmkpaRhCKnB2VzqgALLDysZb7Vi6fygGk2caZi+vf9PjzsPeD2CzVPNQ6F6aL57C/eva0ztktEJ/VTJSHOHy4M5NWSUE2yzaUPb92ZHff+3D5nJxb1hY25V5r7/HvHnzRPCApKKkph2Ny8cz4/sp+KrFIKT9dg9BT8ADwEWTFnGPInkEoNOB7NSyohrmiHknJnlf5cnkMtr1PcsED9S7i0JvsmroZaZO2kugVQnmkExFzPGHLnqIKX9MPHlgex4tcqHynXhG1qscriCxl3sPWwDfZNrkgJQOGWFLFaqT9Wu4uH0QKRdukICYKG9VN4V+Xz5JwPLRhMhuqEi/VH6unkpw80qCG0Xnd7/UhfVFXuCUiNB7TqS00MIsf4j2EvcGSZLYUAdhWVDuARywX4rhm2pI0aDoCYys5Xjd29jKAikJLUHlrTSsYv0SaFq2iuNPWbk23j3qCrUhKzh30ZWQrgsS14NHC+9hbCHsbACbLDrXg4uG0DkXTLkzBR0AVz+EitPQTgzaAvxc6UpEXKhS3hNJLWHVWCn5ZTAz1ndBcrNT39T4UBleREONN2WFYoBF5efNhTZpXGh9QQA1SL6JjM5T00UPr3p456W71rY2NhGwBmADnimFbXbPawMhPaDHGpfNeJc85l9Jb38LIYG1tNUr7T2dtYqTsTBizg+YX1wu0J/H1hwmv3UDG31LmV0lVrVq1WpSr6XS8kpLVOIUJL58F30M8GxRPFGxYqIloO4aj/jC4HwoDRT7xL3JWW7EQ0ETCK68Y082qPzY0MqVrE3KTBJ8oaHSQT683pGQEi2OqyLt3ksl8PGvbZjz4RwSy0Q08lxDNh/8+DjD08zoZXfHARb6DqAmETZ7e6jcA0hfClUXoVDss4zTAbIWQnqCwc3u0oFTcpJ8oxlvOsTKvhf/bM/pz4bz0OLnuVer7HuKrZ5uEat50mhjrA9Cn9kWciXPtfJizaxP2Z3gAQRoqSZ17kYC3l9EvkasWAHgjxawoyPYxeqiWGM+WyPhtfAq7HblPZVlGKYN43zXZQzSKXv9hDVm/IvR4Hp2GA5ZOU/aGmRS/RvQmgwYK0V7TS2p2fTwKi6N24iP5HZdKj0n5b4UZ75Li5zP8dWJ64nVYWWowcq7XvnQ4MFeLNgGJ6dCg9LW9LMWMtEXXgyAWQGysNeqJCi6mMTKt6fwxyUR1JCo+YWgZrmssTWSplImgdS359zYQnCOEyuUZbWGKoerB62PbPUI/P3u/EfUW/y4WeWhN3rDdSoSoSEJJjmVlH/n6mfw/u2cp8Ottx3A0epuTCyCJ0vAphFttkM3H+f1vX9iL5wq6CTsjPeRWVkDV033CvoSbRRNtWBJgiRE/9Eqy9Td3qLdYxRIKjbXu9hVPMk1q4MPos00tDLTPlhpHwfJFh6Ia0G/GeupiBL3Sr29lHZGM7UNOuqqxTk0x3sZtmQwJ4EvIuVlvOkNfh+v5uuRnns6yaGutc9Q8K1HvbXkCEfs/jgMuv9WdVqQwc66DiVsnliPtcd44W9Ukky3o924f8P9VNa6jbvmOr+1eod67XDWnfXsy08pWMTFWPCzi0lhAOv1T+HUU5z9U2xbIEsyVy3wXKkEIcp1NMaQisYeglNW43Sr9nI4HXT1f5GQqEIuhxbS6ObX1nsnMvtgErExpTw84Q6o8ompU9lrsfzXwKZ2uNrMH7ssAmsSpWs023YPiZcjsBaJxuqAPj/h0yCTdKYddrd+gluTM7C2vcKCvFSSUNqplf6dWVHkWt+6NBfbJfze51nyHiliyfNL2K3xkOy7+iFsTnDRee9WgvVVKvg0xMnPIb40z10kgAMcfs1Y5juTm10N1CeL763KEMmgfNBmwP4qZa9Yrb8XBcm9kR1O7q9Zh+RWdVVXcJKYC4dx3mhkWbkSAC5J8FjUGYJUsCCrF4lucS5ZhintXuTjiEoSpGjCDcrslazWklHRkczK9tg8JGJktZ53D/7O5hsv4HQ37VNfpLL1Bd5nLI9FnVPeE+8Evq2G7fXwSjkUuFUFOi3lyGVGnHYVKzb0FuymNs7rfKjtyPQLU9l5a5MwrnkV00i+BSq7D52MyjgC1hqmR1/khajrvNL5RWpMIvVSruka+xpdYxNsCCDRMYyxPjDT6FZ9ftv/vscLhvqahHlwrP5H0pPTsb7yJQmtRWr/3Lg8SrufIt7bit6N4tauDWSvbxly5/MMD69TFCAAlNaXUuy/mf7ZRn645CFW67Azr8dkXvIHs9oDbbDDygMBFawOh55a5TrZRnuLMd7gdLqoOf+q5w0HFgAj+PuE3wlgiAomjoEtTRH6415VtWJRFbyra8Mcb7FFj6lRJkUDE8OvUVTjwd77R/4j8k+y7x/5HxNJkoiJiVE4oqbAYXzIJEpn/UDLZBE9+ZouhCl9tpET57mJVlHgBDKbZLK/wOiR4kgXUMuS5zczN+JvLN6TT7kctqsfetbHjIZjD8EBJXpnl6MPvzd1BS07pGQLh4VYrTS92px3bMG0k5RoJ42tnl4mP7QfT0N1qLPgmHvnfkLv2z0Ge9pEJ9RQe4W8BJgeoCU5yAPnGUDiEy5Ec62Iig42ZfNzBMwLwpXs/Eskmc8qk3lnwDbWhPbApFUi305eXUnTrh9g8TXz5a5v/+WA/SWqmoOsCofDjXCgQQy+L/Q9hH9xGAHlwTjdKxLtJp7S57AxwtVjQOuBEo283+HMTBedp7foVP0px/NNNbxQCpL6jrEhaX25XDyE4d7wgr94Wll2EpVuJ+d8U2xm0an/zbyf6xX+lISWiKH5xkIWdBjCrii4z79KPDlgccJ1K1TiuT/B+vvXM//+7XRu/7Kg+2tuzCqDr7yVQatKQmn/2mv89OKzFF4wCMe2Ma6mIKGQCeHXsLg18FUbm7JpXxuar72ft/NEpPcV7y70zYP3KmBTnWhoVGet4a3IXL6rgWpVE0GvMxXxYgDsa4AGo4gE11We5V4v0GncjC99KI+sGMAXT77Km3tTMduVBmuoVs2QsnupsT1OlpdY2XTR9wWWR2UQ3P4qjT7idveIIYdHboO4PYEDPot9jKa+BXx90rOTydlZULgTWr4p6mQVo08/T4AKpgYqqU1D04p47vPnqInNpaTz+X99PnPWLF754AN6StBPlpTNkvV6eqtUvPLBByS070WzZY/Q8HsMabVuwX1ZQ4klkAtmKLR6oL/1iuatcmijg7ZeHugidKHM2/8HQV559PARUf2JWieDvOCRIvjDOVahW5d7gfK0JcSdX0lekJKGqEVUL7aceoKTDGeT/CI2jdIxUckyuwfuJvdyIt+tFkENUV4JjP61P6N/G+3xWRVpXIm2rA7bkc0eel05rTTTQCuDSuiBATAmcwb9bjTlk7T7UPsqk3anfd/khZAqjseA2i1Ql6NJ5MLAvTS95xg9hoj0Mj5dB1PevoAWV1vQwpwm6CcZ99NMpSJY4xB4/AE2HHOhlkMlD5yswHnvDjTXwnzL3xjSRXtdyb4SZS+sLX7xFKRe58aF1ly4LM7fB2/vY03PZ7CjRIk+bvRuwojrPmxZMZKNi0YLgf9n9XlsfmY3Xz//GW/ViYlZb5WaQ9+O4vTuzjhbnfQ87n8n/q1gotP1c5dIuhCa3brzmZ9Vee4juUdYO24Ry59YzgbfEmEeVfl1Y12zizSo7VhyxSoGWVLRVQ8rQiFKuqVUqr1ZFvAYQWOuMva1jeg8JPuaWrKw9L6XWyPfAevfUHrsGwoHRik+0mlsDDGWc8sGn3uIG2VpZ7Fl50OcO9Aeq/SVoN9ds47vtg2k9dXmGCW3723IY0EIbA/zYXYHsU8s3rFsa4B14fB0iFiBAbAlKApnMkxNEl3GuysQvMLyFXZfrm0w+c1d48m7KgKwxvi6KKicshOzVhlQkPRBpKeN5equZvy4W3SOa9ShLO2/kzWPl3I1eKGgD6m9yKFo+DgEfE3ieuHdkM0HQRCjBjz0XgJ4xh9uxAM1d63BsoYbXi9x8M01BIzbgdotIOxVf5R3eo9me6/3eaONCOixqcM50d/MiQd/wctfTBp01j7MxjoYVQh1GjE4KksyfjL4eKBOByDxMej8jauCyoNMLXElZudVKW2ubqEXGPfmt5gNjaSVGAVgQnl0NZ9PXcau+ztAj9HKkzpsjIsey4DHt5A2/jeB1knW+3P+1lQezNfwW7aSbhhA8opCSneNa3fe4x7H/UUITPH7m0R6QwGceBLyxL40PW81Y+v1SHwaDSSGK2nw7WpfXtmdyuvnI5nd2ChQVpq8YviqSEeLd1+k7+6uwrlTHOdp09cVFGpdJL47/g1ZNNXChoZsChvFwKjeVs0gL4hQQ8fIjoJ+txTNmlq4EjAA/5A79qAsS0QWuNbdHnNX4m9S9sx2yCqie58ldfBR+k4Rk75Fx5oQuqc3vgE1dDDeFPT3ehdiyYqldXExOMQ5murt8jNuFYrBHwCSnnKB9MZ78MPMZfBrMJyfI6haOWroHtSAZv6HpHcTK4gyoq9QGVKKpmUa2U43n8NcwZMtJ/NVuI1fIiDGW5mQlspP0qv2Av133oOuwYPdZKvlwcQb2I211Dn/hjLb9zbbhiQmIsNxrXNLKkGtUwLiXm4Rypiuo1hd1IxGjZKxo3nqFErahnCjw1nefeRbbHeB4QBMofez0+F69gs9PKvPAg5wbnAmv4y6SFNEhpXPM55nRM8efDGkg8cqicrCg/jIMKUYj34WsbcrhE8rac0CWr3C6J8tTNucw+xtIrV/WPNLfJnzCNeaXeOct7gWXU4uYL+XL23PdMRZpryfanM5E31dQIu/ADR3i8pZTycdFDeBxVGe+6ntznqc8b9Ws/zsp4JO0gfxze191i4pfakb5kmcLOrJN6HwkK9YiRbl6IFPY3MiKh4QWlsAFNcn0DL0ICObLhF0xltvsjbSjgwsqBJ99aNhExicD3urIgmPFp9Fe78A8hJcvXeFXscqLfcVQpHNRXfsLqEquHakNcv+bI3WDQyqN+XwYdM/+OH5Fznx+YPCscGNB3iuNJ5Dn03geJp4Txpw+W0/1YDWQyIm2OJK3vxi9Wxf0OQRGHrZI0MFwBuBz9Azq4qIPkn/9eo0g4G47km89cIMEveVcMy0UqEvKiqidvuPFO5K5/CVnXz45OO89+67FBXdTidqAzlRMY28kqH4mUSwMsD6WjCqoF2DZ3rS2CzX3K3OE213v7KD1CXCjHARPNtWfp2Wzb9Db83lRpYI0HrgjA/brFZ2D9wt2Lhmr1j25hvJONeUdUfvUO+uXb0anSQxjH8f+B8sqYhs8xSNN7d6rJgtlmMxjdxF0IBztI1VMj5JksTEUgc5hjqCvCwEOZXV6gZJgpwoblxvTkqQsiVCjX97PgnJJv2Fb1gb5sEIPvUMM7v9TlVAlWcmqJBekPoSRA6DYOVerVJBgm82w70kRtMWyaq0/R22RuoMx/kzCj5pKrJ+ae5qY1AjK/2CKtun9JgynU7P7aN138tCibPR6vKxf73vVyQPbCDzMwfQMxeW5XcEjRtoWAX7b03ixwvv89bZbFd88m697CDIkEdiwBl0kujPSCoV8cZLPNB8Pk6HWFyhVsPgxOX0iv1Z+ayDOvJECZy6vX2721zZAT05V+yylVYkXsahVo47yXSYl+NOY01fSrFZfHdk2Z/mWlgVbKSp2m1NMEQwdesVBufD6AJweAAbNSGNr0Mh1cM2DtBUU8+vdXD/8VcVnyfUX2ZxMNybD62vpwhr6CNGFTP3DiHg9dmM1Yn+24YIG/ee6sJXU1/lQoay6tWuC2VuQDb+kaU8Yw3Ex633nSP9a6533ACSFVO+SBGKJDHU2xUPfcoxQNTbzawOh2sW+KBSGRt8wm8nD/lBnU3Po26HzQRewdWvbwAoYk69gN6yxJsPQkYneLAQ1G5+g06lw8vhxZzYCzTz2SgM6yl5HYtCIDAT0iWxEvh/i3jKd/wn5Z9k3z/yPyayLBMUFKRA+coyNMnpQrvN/ZlYISZagjXRPHa+Ez9sG8cAtRj8DIg24F/lT1JGkkdD+4eUVBI0EOTrGUFG/GTX72t/k+wDMLYAN0oplSxzsM9BnvphHvd2EisVxqWe5qFXfuCCTyU2nbJ9tMW/NfEljVQEVOBU2YUk5YHis/iO2kXJ4nls8pDLk1VeRKnhUV8rmDzA/sFVrVi8D+qzBVW7lk+y1hHPo35A9lrldaHmhcUv0PbQKUrNboGQ2iyeH34ItUUiIl0ZSAao9unIyAI4bvJMw2mJeQJN//Nkt72ADreAgcPCOOcNTlnU/B73itD0GHAF2HJ/AbNnSp0b+PNUCXxaBRrNnSSRLEN8yXReP/0eHx/ZJRznXfYwLQOvYtFZaDTuFPRrqmZgbIA4jUQ3PzGQLWnzGOQFn4d4WF7N5VyPg89DQOfBsQYIOj+TIZWbPW4KsrmCp43QXgfBRqWx4K2yURjtQgUf8tAbrH/QK7RZ+BBPLV9IvK8yoKCW1ZQHuSqtwhtFT9Bfb6TaAW+Uw0mz6GRW1uUTrXbR3niir9NZK5joC1vrwRQoBq3ibq1gWxSkBWdDxZ0+GiqVngvhrkDsuA3jcFYq75muYgePBj1N3OGNNNSKRnizwK7UFh/mt+g3CG4iBrXeOXia1eVhPGtEnINAmukBVl18lzOFHuimAFQGVy/Mps95VDskM+O91Qz0Vt6TcL9i2vY+x1MPHuDPLkpjduaLL7J+Sl9kR1+e89Exzs+LT1NT6fPaa9zKy3Ppg1xI0qKwIuzu88jpoK7ZWB4phhdrRJyrWlaTZoHEbFhTJlbEoDZwvPMPGMPXYWj4QlBbAyczLq0dK8IgRqWszlty6C26H+tOi6st6KlTOj2yTzTHv27OqS0VzAt9lMw65XoyyL6bd82uqgx1phjoaFZ6PwlZ3rS90JaUPDE4f9F7II8Xw5bKIBySCA6IbD2b3ISneVuXDeViUs4Us5P98g2qm49H9r6L6q2hgMOt3+fDYIhUgexW3VSga8L72nKajd9JcJIYrLhZd5WSrkfo9cwGNF5ikvGptOHkXY+nT1w5Rp2/oP8j6Rc6DTpJ50EihTPA7N2zSbfA0RrRmQPgxBQX0CNF6UAUBxrI8avCWWbgWmZv4bDv4ivRTP+BgY9toS3KYISsC2Sv00y9Vz03U3xpdKtkCA3rw9H8zqjLAzz2kDPog7AlueaOLswD3fH/S5FVWtJscPXjeby1eh56u7JKIsKUxeJmVcQF1uNb6yvstWqVCqvGil+NN+frnhXOP8nrOqM+eIm8Z+bRxiBWEO0qXcmMNuuZkP6qUJ0HkJX2I5rC7ficnQ4NuYKecy9D4XZoVFZ8qWUTPyaf4jl/z9ddrO7AxfCBZLUaTK5VdARz7KMoXtuTDZ+N44ak7EuGsTmvHTtMSWM4PWLE6joAFa4+P14qz30XOkS7UNgtY34Wj7VW8Zw/fB8GjuJVCrtPUtk4GwFHehxmq5cICJoZd5Q2b3zN58+upMgtOSpJYFdpKI7vTG2QuCfpZRmtDGmxS7lmEauX/OvTCFDB0yVQ6ycGHAymQmYGwFgf0NxFkfiXLAyReTMQ3vdAFXTWso77j3am43sv0gZl1YrGWYH9op23J81j/R9igkhDNR2b17Nt5GXu8xETduP88pmgjr19D0QboY93PtWJsKHLAkEHQFg/F6375fc8629LlJ/yeeixEmdsZNMbC9k1eJfw3W10jXT1dnC05Qj+aHRPrjqJ12/n2eBaLse5AFV3i0oFW5MvM0fuh+nWcmEsd9uPFovn6tJn/WFFGKxq2V1UWqsgczkcHAWVSiqhEruOHJWZy5oGNpqUdNsOtQ+On0cQv34UgeWBQoVQrW8rVu1sj6HGj045yt5JAMnmQ4xtl83gx7ZgjBRpOG8kTEdKh1I7OAVoPTSEDWZwTQItIzoTIIl7xx8kMKkIjodNgMA71afe1gIGXnPN6bDwclRue9ZeqTf91PnIY3ZRFyKOqz7IxLXWFzn98FqOe4u+0v3Vvahtkk1WTigFDSKy/obxG8ZtqOb3Gy8IOgCCu0Dcg54TzioDqL0h/Uso2K5QPVCko9nH0/n1aDPerfAXDm3ZJJ/LLy3ltfvXcNnhlnTQBXNfvoopxfDe8S+R3BL4tVGj+eL3B4g/1J2FHqjmi20w6dNh1Eyfh7bwb/rPjLjhAqB4uK7YhPt532s0NUnTUWmU9skFWyk762Fy7XWKVUowXHBIeyovtKLpmfZoLVphz5J1vpxNiSbso4Vc9tCns1bVmdN1/rxQCgWS2zpprePXgCPMDQSdbAEPldv2ot28FAA3LLh6v7uLbyLEPgBudr2kUmN3aihriKGwLll4d3QaNXHpjSRlJFEfKM7t126MILPVZtIG7Efvo0wKa+ozWR0OLwfAr/vEfVpbd46Tt1/HE5UiPeP/Ubyjeeo26OG09xJB3aPkdZ40Qi+96P891uRRaltfZU2YBq3Nc/KqV+zPHvdadaMLNBIgQ4Zd7A3mpfFifyMs0LbHJ07sAzfadpYoNVytDyTEA7U1wNJQeMWDyZWXAPKq+yj8/j4BvCX7N+fVTZ8hV/uhtovBdad/PzbucFVcmwLFPemC16903vYHy/aeIyGwr6CfK3UmMBOe/zf90fFvAX6en+W3Jz7mq1AY2sv5X26FsN9iIWBAH6o0ZlpkJhFTfKeqavGiRcRFR9Pw51Xeo4yfqOWtvDwOffABcdHRLP7kE/CK5NuiZ/EN+Z1eyZ7jRc+WwsxSOO30TFFRZHfNsSdqxViXt28caq2RD/3roXi/8lbYV9El9hpJSfn08FdWo0tIlOWG0WJPP5bWDKKrMUrQn+l4ho1jDpBV3cE1jqIi5s6ZwxS7nSqgJzAQMfDfE6h1Ohg66js+HtSdapvoR1XJkSxof5S+k34nVPe5QifLMg3FYVxsdo2Vo84xzqGkvk7319Ju8Gn2tX2YBq3SblLJKrwavLDt6M3N6yIgmfafsGrnvSzfN5xHvNNFfdRQaPcR9N0KbZXVd7IMb+zbzY8X5vN+/wHI1eeV331wNIUJsKgK1hSLc1CHiXeD4GY8xHkr2xZI1mM8HHOVxdEHmFL2JLglxgZ3O8sr337AzuFXGO+tpE4HuNYQyRETIDmENVSWYV1DPc/I33PY50VxXM5ivh8dw5IhHYl1eGYzi/W/TJeoTcjucZvCXQQVvstre/fy/qHfPSZ2u+phRyQkScqYp4TE3v57+WPoH2QkZwhMDrWJHzB9xwkcqiqc7uBDIFZTxhgfmBCcj5fTDcAiaxg2IZldDfB7PUgekn1NAwqZZoTLsUC9WBjyREAN34aBpUEZUG2tquWFAEitnEjL7O+EZN99Xml0D6/BqjexIk9sG2Pf3Z36X+6lIrACu165l8qSjOSUqNg0iDWbnkOnU8bZZK9IjpugHqvAOgCAJKO/YaDQBg/FiG2LUOl5JLM1k4sgUqds45MhLefn4xtRO41/S3PsBHKBt4EpwDtAHq5qwJ9qXcwu6+sQ5snzxjpOBLXgld37OVwlgoYLbCGcMEGlA5GF6n+ReMp3/Ee//3/kW/+RfwSw2+1cv34du/3OwnYh+zNaNnmSvIxozhVPEI75wDCJuE3DuHmyOWFezQX9IEMyb726huZPrfdY2ddB3UiuFfYaOngeVMIkGHkTBorUYwBsjHBRzHRYrPg4Uc7hWSP0uaVnbYVIoXjCFMePm7pT/ONYEsKUiQGVpMKkNxFYGchUU7AQRLlo6EyvPPiyGq5b/YVzO4zNmVcOHXROqqpEBCXganYPd5Cnd4naO5YJ95+ATl+DW9+nH6NO4V/tT0uzlaqy7xW6Mr82hGZBzfA/CbtnE/UWJZ2KTRfBlnrYFgmzAkVqVHP8C5QPusR3L/9GpMMtuK/yhmFX6TY+h1E9PvB8TakvQ+9NcGC4q8rPTe5GfN6dLFTXX+WP8VomGs1UVw8SjpNkH2IHH2H/vYc90r/01ufS+8PFvP7+cpL0bokxQwQdr7WhRy68e1ZE7YNEUy1M94eJWs+IU6ouQPpXsFHcllWmQr4Mha2R0LNO2dT+Cf1qli74lnnz5nEhZL9w7JLSJzBVexGe30ioRjluTcNN5vS+wfVpP/BylIi2TjJf4ZNgCFaB0wN9Ul34vTxRAlOM0DFeDGLbja1IuAkrakDrIQj+ozWS9bVwQ5cE2jsoMJ1KQ5+od3G0qcTYoZT2McrklNU3lUMa1zrQp3ydcF69I5dO5zbRY+sm/FQitdiIyakMr3+QL0KhY6D4bnxROYyHDW+wrf3foEXbvAf3nhVoUf+SkSmf83CJjaXFSmcwOzaVOYN/57PIDD6pFO+nb7snGdA3gefqXmHc4Ic4dfUqr73+OuHhrjnh9Alg4surGTtjA2o3Wkmcdq42MfNhMITWiNWOGnMJP4VDRx1U28UKUICd+evpkQtja8QguqTW4q1uRCNBlFuy72efm7S/15WI7GJQzgNZBovel6j8cCavmiyscxWqcBrH7ODa65+QO1FcLzItFsZO/4WIxDy81SInmkYl810NjCwr56apStCHeIWwsTibnbp21KhF57udrpb2OmjRwi3gpTZwuC6MJ4shNhucWuWxsiThW+PL2H1NeClLdAS9zt3DU/UGJtsquKxuK+gzJH9yu53goE8swV5B4nXH5PLzkG2s8vYM5Pg0Pp4wNewN91zxQut3oe1C0CsR1QbJTtcTt5MNwSLaeleDkZ8jb1LT7Qw1kjLIp5JUmPVmPpr9ET9MnEWFSQm4cMQ9wo8rJzNp8fOMuO4hsiSrOfHMdT597tP/d0i3mnT4LQrOz3Ubl0SECv5sAE06/FSldCT9rKU85CXz4I+TeWP1I4KNEGhO4+HeV3l12UKm9BADGUP1mfiFl2NXgdP4sFJptzDNO4NR3pAX/CNWuxjAPB8wiFEF8Im9KRg8ND7zawZRI6Gvsu+prNYzI7Mj7XVwwEPPvq9znqVUO5PwjK04qkTUs6y2UedzO2HlXnWi9ua6OZTmJRkEpXuAwpYc5HgMvFoGX+Z4SKQAi8OfwT8TlqhFuku1pYxPQ+ARP2hadUBh9/U0hPBRr8851vU49R7YF2flDectQxEVQbnkN2Qqr6nyNDObjiQs+yQhCWOFY0fa92BOgulGKLd9Lejzw0cTexO+rQa7hyB2tX8n3i6HVwIhqF6kRo1UOSi2S3xliVJSBTlsPGWYjKXURk2DEa1bb9IS3548bHNFNS0HROCLT8mjjC05yXsvvIC6VOxL1t17LmsScvjGpylxGvHdecPXtf7+2FAl6P4lOeshR0zMAnwXH0FlE/ipo/Ke7m/szMSfevB0aRw340UA1/MVdqZ+/ygrr3alicmtWlfWMOrybN6eNI+3J80TeiFLEtSYQ6gyheEBYI5sq2dhkKt6zxTwufgHQK9yV8DUq9YDDb9vU2h/22Z3oxuc5v0qjUm1XG16A+rdmAUkiU2jN2GsMfLy+oloHMr1RK1SURBZQGr/04yaIoL88iJnMCFfZrKtkkPeIqpZJWtZGgI1iS7KcnfpFtONA4Nf501OQoXYCy7AEIBOpcNsN+O4ix5Mo7Lja3Sdb+7mtUT/VX11WzqF9COydBjPn4lkdrpIPf1h50O8/sJG1gYUku8Q96SbxQa2dj7GloG7KDWLPXd+VT/OhrZG7Pf/TWUfQNlxKBTBdmh8XXZ95HBwq4DTN5uJs62E1qLFu1a0Xd4Lyuej21tZkNZtoVQb2FyvZmUN7M8bjqRSBgn3FVzgaFMXuM9WItpNFqeTlJOuBEuMl+eeOv9OJJ0/cwevYELkeOw2JUDrnnyYe/s27i0Vg7JB+a4qxg+toaitVQpdRsVJmtxaz7Fl95NRLdrAN7w+ZNi1HnxaBaW47TmShNkp8U45xBcUU2cTAW+XNIkMzHP1evtb6bkeOrpVyJlKaRZ8mEjfNPz1xYIvZWzYRWT3c3S/bz+vhYjvbEX+LsZmNGdGhIk4bbZy2P4tGV3g6u85plhk3XB4xbHy8ss8fvgzLlwRwQMA8wf0YcsEicGJywTd3XbBgUrl8xhofIonW81n/C+VXGgUQX5OXPtr3/g1qBCTxqOafkKUXzrfnf9I0FW3XI86HXxkmBUgglhbVO1jbgAcLfbw3gB5xq7MLYP5+tZizz6ng9+T49nVAF/Ui3b9mvJkTMlZ5MTkcKtOmSxxagM4W+qyub26i71JbYamXI3xwtH2CskRYoIzyiuBsOqhGBvbevSpzVqtKxj8d5L1PewfIYCg/pJRPvCUEVr5lP+XWyEs+PBDfAL9uD5yO3aTDu90l9+5eNEi5r/2GgftdnZb7UzCVfEyCdhpMnHQbmf+q6+y+JNP+FP1AuNTVtAr1QOzC1BmhyVVkIZnQFuSBqLVnsHQBHdF130VGlutkLC4tyiSx2sb2ZeYxi4vZeJLkiSyUtKg9wmCKsswSEp/PqDmImdiYNQf3YhMd9F4rli+nCbAGuAIrqB/P2AJ8AzwKdACMAJNJJmoy2a6GkDlXqLPHQBS7zx4pExp39jtduIzg+m/ZyDfnX0YVcwjCv3x84+QsWIUwSVOIbkUUfwHP3l7k3o9lcBysafq2xnn0P8WS/6BjiRX9RL0/07+SupcLu3Nl6e+RPJTxgKkiEGsq3M9y/Uloo+ncZp5PRDiNdDWoIzBnapqz8tPvwhb+nPQ63nh2OllbRhwxZfTGRHUOkWgtYTEi/6wvuOn5JQr6WJlGWyqakJ80mkXtpiKUqWNIGm8OVc0kILaJCpl0dbE6aRT5B8cznlQfC8LtmG89SZmmxdVpjDl86g4y28hkYz0hu4GiHIDr/hYSxl2riND9w1gs38KQWqlz2HVR7PTsJnv7gvlgRSxguEh7295ygg+GXDK6mY3Oey8n9YU4+3xeIrz5kd2YlGli0XLE6X2uarHeC+nFSo/5VoW2eUzjoTm0an8eTrrJIEK+bP6e/mWWjQmPZUlol30U+Zte+WeQwRFK21FnbmY1Y0xWMLLuBnYT0j62iOHMaQATsVbeb6n53ZMEhKjC+Cz82+LSllN2alT+J+6SZJWSfleohrJHmcaet9qgeZ4MTAfOASk3f65dfv3TeCQ00nGNuAohFWOFtpMaZyVGL0z2RD8JtssYuV1ZtSTPFwErbWgcXhmJ/rfIJ7yHf9J8cwj94/8I/8hMZmUC7FUd5rBhiq2OiWqVeKmGln/HdCBgqRe1Hj3E/QX031JXzSRtmP2eUwqbI16jhd3vciAlH8TYPSJ/3tdYHvwFTenVtJ1ZodC74JBxHiJ/Ot1pmlk+ZxF2yYJdzcyxsufN81fEvHELzh8JKFPlkqWmbxqMkmZSWyeLdJBqCQVK2pcPVxeqyhguCfoRvYq128P9DKAC5mePE34OFlbS/B7X2ExmLgkveD2xToCrqbQtCyU0ffvwmw3482dfhKq2wgQP1mmlUZ8VnMPPYNpdwwLzL3o94HbPZNVmLwTWHfhR2QJJrebKgaFJQk0fmBMdf12k3f9zWxOhkIbrg3/LySvxo8jOWMpa4jGoK4BlMem+T3LbMsvPBZaTqz0B6DceJ/xPcbG9x7nlcIoJh1WVmkCqJ0GjprAXCEmo9EFEn0T5gbAFY3nRAtDL8CFN6AxT9T5JNIvD9aHQ6tapeF33dGSjIYLvBoANR46F5y2eNOj20GSHFpkZwNwJ8CptlYzIczEDZMfVce3C8eGW2/8q8rhhzIx8ClLMk21rj5u1zz0J4gKSOTY5M0EXJ+PznITUN6bTF0CM4pgbuJ9zPe5QwMqW6oYcOonLl1oTVj7G0S4JSWqdFHMDF7DeyEzqDGIRl1I5VJahOaQltcU38rjgLKSIja8nE3ZkzmRN5oX+oroyj2W+eyPAi/5b2h/AY4/7upl0l+sAv0osoxf62BflXKrndBqFlsX9MXr1EaOdDoLbjGDEv8JlIdU0a3rXpJixWDcz1UH8YvxZpRfA5eE90LNkvymDAi+wSQPNG8aRyPjfKHBCdmeEGSNhdQnwtJqWOvBRKg3Z1Ksu87EIuiR0Fah2271p3jAXrZ0PIhX2XPcvaKcv/UHPlWuIFkXsy96qzJ5dUPXiXnFKylPqeGyaSmgDIa8WqlnT/BlvB5oZF6KsgIZoJuUyVehMLvMswNgqbpM5q3tfN0A14bFC/o3VFeIiIUrZx+F2A4uxDqANoBn83pzQ+MKpLkbuy3N51h8tjd5+ztxoavY4zOjLJTyH4aTOGA3wX3ETI1ab+NqSDH+5T08BkI2W6IoWN2Tg63F5BNAZz10UsPN4L9BsKdMd1GjWevgrgrnh0qvch5XoCChaW8gW3FYbEYE4ZcTmdjrEKNbK1GM2spz2JLglMm177ijxnJrb+JTWQBo6ZOcjbs4nA5i5IO80NKM3fzv4Nx/I40Frp+rC6HtnQom2VpDQRN45/FXcZq1NK5XBlUz/Trz4YpcuhUFE9rnlHC/fay5BC2fyKLcMArm7WGI29dmBF7mRGuoSI7iHnfmAKeVJ72zuEcL8+TdWO0WNCpl8uBKVT6b62Fz1g1edKNjAyBxiuvHTVQaNb+WJdAx5DStPOTjxss/khLfgQuF+fj5TACUVc7DvYwULmng5TLo7/Ze4bBT6bMBtQRm2UNjMVmHj6YTLTNeoXfrMaIe2HB1A9UOSKsRA31q3yRONV+M1lRIm2YPYeeO3XfT5kPGZ0OYXu5HxptiZWpZ7ThG/65nWp0vwZ1v4iKb+evE3pQnZnDfs7+gaiICTBJjHuPU+Sw+D83jZw/05U6NLwU20EjQNkyscHaq9WypBwno76GCw9hvM0Zc6Ne7paiomFWb4fcLu6lX7WTjo2sZ99hEnpg6lfDwcDQqNZlJmWwfsp0ao7hnFWracmxVK7CrqHSIvazW1nxDbP2XPBm1lxwvEbTwviWB46WZ7GuEVwUtcP5VMJfCEDF5BDBSXYq/BBGVSjBRV10/IndrqLxYxg2/DcS7vTyXKodTfd0I1+MxakS6Yklzx/ZsHiQmhV879QFrh3ZmoqEPcEB5rK2W2bcv1afDHuB14fhcuwY5HY73fhWhqEVWQfIzLjo4lbLn2Uthz9C0uD/X96QSeq/SnusaksTOvt6U6bditkvY3WirO2lNrEhtxkHfezhWPBb3q2oWN5K4Lcfpma3hmof2dV7mYqLUrj5azTwBhhqLOXlrH9aAoTRRBeNuoayKCeT9GjNnzj1LSUgk4fGjAbB6x7OszUnG3BxD571nMeqVfW96eNmYcb455iOdSE/NBTfW7EyrN7Unoxl7ohmOCeKmlKxqROvQci2k1ONem523gyux8FmlZ9o9zs2Ga7fXoYliwJjwga4fN5FT7uGsv4MhO51o/PYI+kUHuxO8si/zAxrpu0zsUcTtoPqZxAeoNu/AD/9/aTROK5XNb7BVF0Yn9c9McjtSJUHN/VtpZ/XiSpDnHs38Fg2N+Z6vyWFHPjmF5PzfsTet4+7wi85ppIW+mpcDoBLlWpV/7WuadEgj60wKlnXDCJ2pnL8BVTu5pyCUgqwonI62wtderN1Hib8LPCKgytXeTKzuyumKPbf14rMc2X46tswVbI2SXSAbPzf/99oncOEVGHRY0Q/TkreFjwa59rJSG0huiYGym89xf5TEn9u74EwQMz1bI2HFqTZsr7rF/W+NVui0XpF07vI+Lx5/jRi1hJDO84rht0sugF3A31C5OW73VO8V9zPwpELnY6vmlwgdAbIZa4iSLlYnVaPW1FBv9cfhwV5bl/sTb+/6AVmy8/Js0X4urk/kaO4YSjwk3Aw1+zkcDV0NINWJPkeTysPMD4YwlQWqr7l837sk3ZDCwkpoKntIijmdjCSbhpQRDOosJkh/yf2IqGdG00EL5eZHgDvPWZYl9vR4ne/H6jkri3uWJIE6NZK3e7/Hr7Vi/CPG+j3v9VtFcX0CasdH4Jb8CnZU80MYZIn4KJeUn4SCrbBnAAwXE8OfBsM5E/zi15LvZ80CSaL37Nn0kiUes9oIA4qAH/V6DlitLPjgA2a++CIb1g0iokUpzz2yCmN4L6bcrnA7aLcjevYu6QJsMZvpPXs2X39g4JtqWFVrJtPD326JdFXEdurjORl4Jgb0Moyp/ptMevQIj2uJXVLT/Fxb4i634nA3MSl8NdXEpt/C2H+8GU0WK20bg1qDvdqXLj0uoUveBjzMLz/+SKbdziHu9PN67fbP3fIY0MtpZ8MhKOoB8UYRJBjvvMa8efMAODpDjBXM7X+SK9U6NqWNY3ozJV39KZ93qGv/Ot1bPENBxTPAHcrYUK2OcpMrZvFgjGhnni48DR0gqLYP10tfZ6L7H9xcDbfWgNoXwgdA0p3eeyoVpAYfwUtTzfaMp3navX1u89k8tUGklf5LNIYImmW7dpcOqge5GxJnV2vxqvWh56HetPQVJ7i5zote3z7E2Wp/zn1mxr1OeUXLn+kVANlWuOXGMCHLMD1hO0/dzvdWVpyBkDuFDJLWjzf3uVgtHveEBZUkrpV1J782BS/3tSz1ZcqNj6HfWk+gIR9ZvqtCtCGf0f4F/FYEfpmwqa3y3oTVX2d0x0zKWhQwIiSNUp2SxjOj4iSl0e/zSx2YzBFuURk4bBnCyfrr1DvhUsNRuJsdw1TMqQhXtV+1HRoR3x2rpOKlMnipDJxuiVuAa7IXTwRfYkbkNVzprtui9ceu9Wd652Ho1fXUykrQw2lTOenxmWSNW0dDmNivUna49pQeF9rRaqxyBQnVeZNYGUJaUTBFLXdgcyQBd/xD3W37b00tXHbcxJOF0c+3gVIbnMQz2Eh2alkzuB+yLhi4UyWqUkF08H7u6WPiu80yk6yu/bYImAschH+73u2yQ6/dsHuoGBssDFvApI0+lMW+QRhiC58Z/caTd3UWX0QXsNFxBVeHwP+d4p7v+E/KP5V9/8j/VZKla80sdQnnOp/kqJ9okJalvEHNtASutjdSaxcRpZ/bm/4/7P11lBRn1/YN/6pdxt2VmcEluLtDgAABIkCUuBCBhAhEiHtChAgEEogQgiQEd3cfZIZxd2///igSqD5rkut6v/Xe97PeJ3utLLJ6T3VXdVed55ZjHweZgeV8dmUaFoPYTNmetYN7/CCkVtR7AODkc7ChE6igVQEZca81Q6aSi/yg1J1+uWDWOPHTimPjrRt70/GwhnEBOwhoUlJKGRpyeCL2KSp+68eWza8KTcp4ZyYpDXJD6sFKcbpDby9j+tW+jUfbTAMp9WG5iKhClUXRVvg5SA6GvC8360ZaOItpXVtNvkFJL6NzNdGhOIrinV0Ze8ks8HaH1+6gOgnmlVjYWS/SAd5s20La1gHY9iSgt3qJpHs81FVfJvHELKacF5uQADTkARL0WytTU3lZWoxcGPQzBcB1yDnJGsPre39mfMt3eLlvb+E4nVZL1yNd0H0xGXejOJ33Uf0QLrqd1JkaKXd4USA5G5gVXUJvEzQYxcI/QL4THi6F/S6xQfmXdXgZeogi1CZzKDsaoVMubIhVavodYCAbF03E8Oh82hSJKHGdpOfdUWt5fvR6qlE2fTUhvZj51nGYP5mOtm3CsRWJd9EvFwK1EKpCn2tsLKDKJVNDNgaqTHd4XEQ4SzBWHAa7WNR1XNVQK2vwur91ZuotcgCx0/QIeE1VGXR6PBoP8x78kAX9xGaI23aZ2FT5eayt+k3wBx+LYWL/bqzVHMVgEottzwRl098i86CrWsFGyPxGLp6qWJ9c+K0enLqfFK83FZZyw/kDWBot1PmKv9W5hq2c8LOR3r075rZilL7T/Rq7F43lo/ufxuOtZylJvJXbkXZGGBYuonDxbUmXHJjpB9NCVNZBjZHtjXIYmqCW2Feu5vdocHsAo7II/qk9gi/PR1B7OY5S7ylkZzW9x+4hMLKU1GcXI3mtc1qNlsCKQBau6cqyy2paVBJp6Wn0OnI/qV56lQDttOXc5w9fh4HeLTYtKrJXszQCWhsQABUAHzaGsLcRrO5G8BJR10g6eprgdl/QeE1SmmgiNlq+b6NUaPeWBwRQEVHEkK1DGGQQqT9+jv6FCQd7k3K2QaRkBY5lRpB7rBWhpeq8TF+YBvJoKbSvaWYavfoM/BoNFz9UvJzhCEMTUE3n6etIChL3lUcbXHQ+1pnHPnhMWNslYxCrSsxsvHs+PDJfKBKePPkS8Q9/wrnZi/jET6Q0s9lr+SrUxigrOLzX0P/EQnpCz+XQX/lMS1ozH1SCxybfW1aU94Gk0ZMbUciJDif4NvRmInyUyBgp8iZ+96uiyNTAnjLxHnQaYii0HMI3YSHljV6TPFozN1d1I88JSyJQw2pTUZ1Bkv76FE/FLn0GeeuUb60FvTOImcXQWYX9M9/p5kpBEI1VfjRaxT/IdPrR2gC7YiAWJb0u1WfY1mUeS8Ohm0WFWjqkO0GTtjFrZhWzJok0QwAx9Wd52B+qyk6ITp2Frh0fo0OPNyBASZf5UmVHsvWNmGr86F8iAlRauSagSW9BaV44Lo1y+k4KaMWj58L55ZNJvLxbBGDZw2dy76npTC+C7U0tBL/B1cArwdDYAgKqxMaX1tVEtRs+qNHSOfVmwX+m5Az7d82i+vdufyHv33vnHeLj4jm+0cUrlLHCVcmcK+kK+i+Lu5IxVnh30kHu7ytOg+WYhrK97y7yYkrQBYkTh9/X7eOmpm30OTgLu0WkH93qCmb73wFoNXowRYA+QNV9a1MH+ufBuajbFK/3TgrBONofT0kI366/TTguot+7rJnqZuugrRyO8CoCejx0Cs5i9cL5fP1AIQat+AQcShnDeXcNJw0iJbzWFMF8ujCnDOrCxQYowGtdpnNLcl/OlamwW7hsUHPuKuhLue8cb3icen0sbXqcoXeYciJRJ2loKoAdF7vzkOcRQgKVcarVUcrAiM3oT5lpyBILecEBdgxGGxqPhqAMka4qoXwz43zgzmLQqVEklu2nW9FyFp3/nUtNKj9q9Wni9DDWCjrbtTVcr9NgtMlxgbGdFa1DySLhX7eWfvFyMzcyUSycfna+E8VfTabdmXYM84jf59LE/fT5cSJvXOqE5M0sADzl20hrI4xS0QYD5PsPoMsnos9RB3umwGVx4qreXUFFYDWdpm2gf1sxvj7fKMfUtkozvh6vQnNjMWXJTXhSIL3dQRxO5e8RV76Nsy2cxMcW4TKKJ65z1PDOhCPUDttFfNgk9esK69esFiZl+5HyrzKQeGmUfxd4G3MDYaovBKHMPS/WFBI+bS1Fg85yIm4m0eHK0qjd2oLiEDn+7OrrNV0HJNo/Y08MHI+DNM9Z8bq0GoZY4F4/0KjsWh0iOvD44A8wRA0HncoP6pMAbjvkr1e+b2gPHiuFDyphscoWf3fOJV7OsZCfE05dlTiR+EI55A/ZiceZSKKvkoFFI2m4td2tbGyA5Q1i3K7RwPnE27l3RBgD0kTdaYB527Zz55os3t2/TPBpXfVM9LExyALta5XAg411K5i7dReJASewaMS4X6MBt0eL020Q6DABjhSNY9mpV/9qNl5v+roj9DDDc2VwV7EKs0v/NbxSIfFYIFCfLfgjfSKZ5Q8PW1Q0zSQNjM/H0nspoVYxltRIGoZbYJa/Cliu/DBVLZ0s39CFd7eIe46pdjP3er5gwQNPknfuccEf3fg6HSK2MSz5K65UiTnJbOd+pvvJ97+qdf4AUh6EpJmq7odL4blycFyliXt89myy8/IInTGYeaESd/nDS2F6hRQCQN/QLjTs7YZ/tR9NQS6+XLyYgXp9s4XvP607MECvo+BwHXqp+SblGCs8EQjJleqx+VtVYJDgMYuKFEn1eRkUUS5+X6n6BtwtssmOy8ZQL95Hq2s0VEzYwOrJ58mtV9IgJra8m+U7f+Ho1q5cuDr1l1dURH+aL/j/ad2R4VbplfBtLaryKjGma3TWwSq57dYmE7mH26Ct20pmpRdbg78fQcdCKF42CBzK9d2/40s8d/EVwnofoyFY3EsXevbz5H3rOdbLH0knnhf1WbKUTc4PAhOUVgu3tn+e2T1vBwRZvb9sfRS8mLBPeF2nM3HBAekOqPMoc0uLXxPZnU8AcHeayObUKsdCQHUAedF5qnrDF8r6sLwGErPAZVA+txoNBBrqcXvgxgJwBotahP9k3516ibOlfcVrtkTj9u/AG0P68GTPW5WTfVGjMF6W6R3VrMTait83dmX7/g6MW+kAo5JVqU3Ba1Qly7S9X5Z3EI4/4RzAylrobASr5JXL66z8ZEvkcBMcbAJJZYDB4KyjvQF8m+lw9Orj5Mda+MWZoHi9tiaLc6Wf8mlJNF9kjhW+vwLbEVpnJfJidhveiBbjD5/OMhDhUmlriqqVk5RBEb15u24bSW+8zW/T5uBwKq/LUHWcV4NlqZwvSlXqtMDGKFgQDFfCRUkaXDZWTAwg3CeLfJeyVdjB3oU3Uv/gWFvY6fL8NfX8JfIE73/y3PfVwkdbdMJ9cqDsD8rin6cqGWaHiPPU+oLvGe1Xw/PlUEUz8ln/2v/r9m+z71/7P8qceh9Ommp5sXMu90WLqNCXcj9jr+lJekfcxMWyLwW/LdyJpcFCwkUxyADIz17L26HQ0iYivQG5gVR5An5XLygA8jRBlrIxFuDXl3MlA/mh9Wb6+o0XDmkTsoqIhELCLFmcKvOiT7LEsjrzDRqqjehUaFTiHBk8+ORKes9cS0mU6NfaSlgYAsMtCEXqv6wmHXLU9XjQ+YLbCftvgyvLFa6OumtUqt6BSHD9BZbfug9nQDUhBzsLfrchgH1NUObyqAZPcVTT+4lv2TtYhSLRbSdkYzv6mWFZjbo2DTk/wdaBcjFbxUzt5sEtHqw3VypoFv8MWvbkTGF/gZi0R9oP0f6KPNWz1CGiL8+4kvBU+6LTNHCwWJng4qhhYdhl9sTCr32eVLkuB/f7wyAzeFT0MQAo2iI3YFXMoDWg1+jJd4JPkDLp0ml0xJ6SUXwHS8Xi5ieRr/LRiT4E7O1IrneiKElYqgqQ7E46Nv4kHKs1BrG7SUZxfVkq6gxZqk+yNEIe1W8VqiLCW30GDt4N7V+CRLFQOKx4GZ4UaJv7pawn9NdFWVnRPYYz7fIxXthCVo7yubVWH2B1RQpPL3qQhAwxQCrxG0WbIYeoffkttqjoguygBWvrID32KQoaxOBtjE8pB5tgRrF4ScA1KrQhYoESYG8TfB4GkwKUz5Um53cCSqvoOH4Xz48R0Y+pZ56j31fVHDpsoFYSkWmj3Jtpld4KS6NFfYpNX0a7bJifJ04F+pn86c9yvs0aTssklUa6MYgxBTDCCm+argjuWlMKz5TBKTtYjcpnVyNpGLtuLLesuAUTyu/bZo7hs1wfdoTnMbzEiUuvpMFIdV1kztmu+K4ajW75MOFzB+mymbZyGn221CGViThae+uF/FSjZZIveJxid/aYJ4RvauDNENDXiev/Grs/ffLkpEqBXnc2MDfsOJui4dsIMHpNQ5726cvwlIM8sWw+0yeLqGO3yY2t8yniW2bh7z4m+FfnpWKSoEf7kxTUiYXXxW0z6HPXr8yZqp5ZHcw/yH3+0LJyh6qfAzPlf4OUDdI9cdG88Nh7jE06ys35YrOk45BwYnueQ4osIVnyaiD5pXJnwXVau15rf5wjhyeTqzkeXMIlhzdMFjSSnlV18FgpVBuaUxH4G9MaZMrtaKWWpqS38niZhvw7v+euBYtJ8iiRmSZXPZYbznF2SAMhTdPxMymrSxGBiTRqBxBcFkJQgXjeZgoY0G423/d6jqZqL3CMpCFX48sjpTCmADQqSegg+ykyEsCeAtSJzxb5v8Ph++GMkqZFI7k53GkNK8196JkhoivrQr5haaOTyxobX1SJyfOcqt68N30+L902nyaD13piDObrarnQtlGFWRRAspcSknEnmjxxbwBZj+/DMHjWR0U30uOR4wu3Q4hPtBotp9qdxmO0calG/PBnW8cQ1ecEa8eupUEnru81vrUc7HaQfYFiEcXpcXAiYiHLaiHdJt5jHeqPMjcI6o1RqpSqKc5CLiXA2l7TMatM4R88/i6my1+grzoJriYF/ddOJ83Sfy37bD6L6mOo/mkEMeUivZdeq6dm+E6mz19EkYhPR6vRUOeGfIex2cbCa8Hwngi6l639Auj9Q7OUaH/kHWVXIzy1XwnQCrD/wRMTn6ahb0dCk0V9GK0W6vw87O63m4YosWn8SugyTsTDD23XyPeEl40JKOKPBrlo622SzkieTwferIRqowrfKzAt5xU+9OympHCv6KzPlkF8B+8GL+3HpbVtOXrOj7MH2lJu8pomM4Xxw4Jx1O8NJbFgLIEm5e8lJUxl6uL1uHfmYzmuBFMA6I7fxwODNhMeV0Qvb8p3IDD5dh4qgXKXzNDhbeecRu4qlotaaoVABm9Dn6FDfxlskSP/etlX62LgFTkOe3DWAiorlE36LN/JfBLZnvuWzmfaTTuEt/Wpku/3lDt+xSdG3Bs2Zc5Dq3OSdbCd6nn93GhifjlMby5uajVbnlhJfUD0ue0yzeyhWfIk2XVWsX0EM9I3sMDTwEL3fcKhOR1LmD9/Putv6chlg5LiGY2eI00aCp1wwYFAJVtnSeKD1T1o9/6NhPiI00mSzsrbu1M59+MwpvRIUL+u3t/DlGZYIEzhuBNmkBG3SJ40vc5GBy0j3BOM7hKc0iipkst82vL62q5EbGtDrU+s0ECy+7dledJ5et60nRg/ZZwJ0EV7nN5msEoQZPBaFNxOpppzWBUBn4eL3wkAFxcRcHI2mtZz1MGisTeBpJPlB64zjX9rPqiCx8pgXrlYQM/y+FNSEkhpaCnaBLEI+EUNnK/X4Spw4XZ45bXlh4nd1I4H/NWfC13DeX7ptJyxgaXEhIr75Z9W2hBPRaPKNfm1ZLRnGN1y4HDk7QqXJEFS4HE+HNmJVKPIMNHW/xdubfc8LUP2oUG8FyQJ5vcfyVO9xLVdXy/Hjz3NEKIRn7vgkA48PDMP25jLED5A8I92nOKzMHjAqtbskzhRU4J9TTK2baIG+YtxKzk1az6v3DZfzLd1vmyvNFD/+wBa7RUpsyVJS+bpRKj2ocElxk1fuqfRNQf8M6BeK07E/CHF83w59FABMgHyPtf1Y2j9tKp7ZR3kOCHguu87IiKCOQvexnbLeHIeh/x7wxVSCACNia+xbf1Auh3tTIy2ljXff8/tV6cvioBXgK5A6tV/X7n6OsD0Jhtv7TDybmXzTYV+xQFctoOkovkL8EoFzCqGH2wBorMuQ55+vviRMicGXva/xCN6M/E58UzxnBIOXbQvFd64n9aO41Q0iHGmMzaUdWPW8YKuCgDJbmem+iUIdgdgdME4K5g0Yr3JEnwnP98RxtRv5vNy2g7B/7HdyooZS3lo8kocp+YrfC0C5Hu+wVpPvk6swfkl6ng76Qxv2cSucDlmSmvNdIh+Cm24Cs1h23kwpQEm18INylF2SYKfzj7LhbIeLBrVSq4BXm/p7/NpGCTrIdIo1hYlPKQ5k+ljghidEkzUUXuOxCMyAHqeSQQH3Nv6GIOnbiZyxq+004uDAlvOL+HOPDlX1nqtdRoNvJp+C9rLsK4eMCtn/3XU8/mYFNZNk4hwK9k+/rQ7Oz3JnN43i5p8zka0zgrWXXyEHdm3KkM2jRa7B8K1MtBI61Lm43q/VH4feJY9PQuo1xXhQpkz1/l3Z3G1DBiWVNoQWo2WIRY4EgdpkhfYyODPp472dMuF4QWgUQGOtbZf4mS8nOv/JWF0ndWfvofng+DpAmUd4PKBR7nP9gC/mX7nM1um0OzbHRfKk7VRnD3YlnN1IkvP3Ggd5oAGNIV26mvFe1SSPGTOeZJPPhsraN/pq8/wTBAk6MCjxrgEzCqRKYs3pKgslJKGyxVdWHpyIfu9NLOLXBKHbbBagpRhoxhrNPI78BVwu/hOqnanA747UC/s47qG3Uz2geU1cLpJzKN0+T8zzLeOVyqgSlLRoPjX/kfs32bfv/a/ZhqNhqSkJAW1iEFy0To/lrWfTuTCZbHRUlB/iWGL78D84Z2ES+Io842mbOJ6nMbeqlxV8P6jSDkQtbZ9RvAB0HMJdHwTEqer+0+/LNOidVeiTvuEDyex5H7erHLyda043WHyyaIoK5K96/pyUqPEUdRKRj7xFKCx1dPaukqhvQGQ7t+XAbYKxiYeI0uv1AoE8A/tzlabifnB4GsX0fGAjDiNFYN0AEK6weBtENBO4JGe6Af3GRK53Rd83Ur0WaMpioX1dux6B1aPRki67EHdGVUAiaZ6SjxiU+FV/UgWRWQyb+ozlJf/onRKWtYTz4wiuLsZaTvCB0GLWXBlmYyC87K8mjyOFBwht9pLO8xRyqzOD3G5ojM/nRcn92Jsm+nd+RIFcTkU60QqIINWh3nUTu66eSfRbq9N1xDI+PwgfquXKe4EcztYFAZbY+C91v3Vr+vIw7BtCJwSqT+k6rPYkx0sCoUgSZlAzPQppe0LP/PRwx+S21Kkjzlrb0X6qiGM2DgCga3HUYvxhhwcbS+zNVkMJHw8Nlr8OZHiEYtStqBu3FcCFp0JMyrTIaYIaPUkBKtjiCquvuejgQiTfxv1T9AQdgGDu4ysKmUzxYCNCK2EpSSUMUViEbA+aBRD1rTjyo/DsXvEgu0KehCvB3cK2JvEBtCNxZ0YrMKm+pe1mQdD98nTviqmBeaUwRaNsticFdVE5MLv6DLyON31YsHXFNCAT1QJyecKyUnfLvhvDzlCQoeLFE9ei0alZ/x56lbu8oODfiJgQuu28+aMaIbdvIjhE2aIB1+19yphpUPUOWw0JpLhgGOx0FWnDPIfNxaQoJNvrlmh3voCWqKz4+hwohNtT7cVpsGGBARzS5/z2Kz1nLlBbOYN0WeT1FZ+3a2iWWK0+nJPsY6ITPB4NRIBavVBHGuSdTI0LhW6wubM7WB68DnO2+H+07cRHagswtzc6hYGnL7Iy3tWcKhIRInfaCii15h9HBkRQa2fOIH8fuYgKoqDqHRKqo3bllGt8Ol1grRoFc0FgOpzfFAFg7LEfUc2SV77I5Xo+GS/1gz/Yzhj1o1RTbh2e1ayaWAlL7y9iDgvWiitpKXR6mD+/PnMnz9foAc7FjCIblsSGfrRo0xMf014b63OwKRC+KCqmSL2P5mzEarPQZM4kXiHbhN7ErN52z+fEk2cwhdbf47DcfDo/gAGZc8XjjVq6zHGV9N5xjomthBBJEmlPXg2KptnyqBc8ioSejxEWxxkOORpXjVUf8tW91DnhjJMcoHU28quPjOtlbQ4Go1EZUMc3aw1TA4UuyEDwydRFGknvWU6Wqe4PuuuKy73jH1Y6bTEcFcJTCi4phulsIY85n+TxPPl8GC6yqQw8FAp7GiAXxtV1sHaS7BSDysNsPsmRdx3szWLrn1PseCZ1zibLE66n68fRfHk9RzrfEwodNCQz6N1oXQ/1B1rvVhg3HJkPIvDZJ3Zgnzxt/CNHMSZkNH8HDGLIp3YGQsK6wp+afQu/QkqxYJah8ottDOC9ZKdogY/5s6Zwzqb7R/pv17+YCkvHA3k0MYebFokgo1a2tazKRp6mMCqMlm9IvxTTsWBJuZDTpSKe8PW4GLmBsEYn7+hqt85Fvao66m1NurobAR/L02zy4buPHt4PsE3nGZgJ5EFIqV8It9YfuEmhx+hWq8CiyTxUdFEPnp2FhlL+1DYIG6qt/rbeUVctmXzeDC47RilZvSNgJ3GjgRpoX+jyj1qDAFrPBT8DqXKfWl709v8POJj1k64xHGbMt7LrLwGAGp/eQmSRxlzFTeUUGqUG+Bd+ojTofnGBIo7nOfkg7+giblD8DuDu1HigjmBoFMBf9XoA3F6YEs0RNvF72ze1nk43XK8dX2BPs3XlztHy03Pu06Eke1RFsTWlB7ge9/PGJwPDxaKBZjxvSqR5q/g4/q38Q8SNRLXXX6ck2mZXIooECb0AbY06VhQAdV/p8PVnBkCocvVRrMXlX1tk5WS7Aii86fRxSNSuf4YdoFt0XAkdTwbyz9XOo1BjMq3EHUF+ueBxos2tcqvLb/Z5CJbrzAVuQSDH/Wf3ULIzl64KtRI+65ac2MhfilIPb8htP2twn75cuGbzCgvxwVovaZSJEkiMymT0wOOERW1F4dD+WwZdDoutLzA5I47mVEkTmx9J80i9gr0PTmWwBAvKjqPk0eNF7jihKH5zezF1gSIHgvG5h5OYPRZ6KyupdmcGbUmfOutxNUE0MK8XvVvQm84z5C5n2Nzeq1zOh/y9FG8Ggx7IsRYztdiJ0QLD5bALaXqWscDE75lQsu38DOKBXiH28nvlzdx2AZlXmCkOMNm2obuYtnJVyh2icDH3iHvM7XtK7w1tDcGl9jtjvE7T7hPNjk1osyDK3466bWRZDqgk17cx2ksxl/jxuiXfE2W4nqzV4GkQxqmTvk+7ZvOHK+tZHWRSLPcxn/tX/8vTGz5t+SZXTKQtCFSBB86Agbyk0f+HTwqWIwKTRxHbFDjBkkl7tlAHK/83XpRdwVK94NLvWkWr4PT8TDarXwu2wS1pN+eW5m6Yip+VWLxXaOB2p7VWHJj6HxIT3VNDRHIxH7xwB7gMeDTq//uvvr6e0A44La5SY+HtaICBAC7a6pIyYbvm5obWZSb2nscKpTD4YPkptSVbwXw7/eNMWzqdIR2Cz6mIky8T6oDqqH9eV6KsuPj9KoJ1efSPXkDA/b2YfA2uQahMxr5TyF34YBBLwMfzVoRQCsvbR4+roJfm5K9fBo6HG3JtKUzCGq0YnIpc5Y2OfJeUuNTi6RR3icNpYfo7t7A9GXT6XJ8vPC5H2i7cHb2HFq8+jh9fP5o/gL0PuCtySfByeIh7M2djMujQyiNF27idl9olQ2zTykpfwGwV5HeKoPdsXCjr3ItK6i/loMcrhB/5+0+vmT238tLbcroYxLXMp0OulvszPQDnRdlpXczynv91mqcRPnKdQ2DR52tTJKgpD6BDt4YwdMvErQ9mB1Zt7Ep4x7lZ9mr6GGCCT6wNgrqGpSa2KNTRzNs1y0M3RyLrXssedXK2kpt9FTuLYG3Q+H2QHHafJbvWj4INjCnDAoR4xOj4drJqMWDN7S+G5B1PGkS1+AKp4tCl4g5K7Gk8WqFDL4yOsOF77fRdCsZIaU0GZuoyhJ1wPcceojGKgubR2Ry2EfJkNXUWEaLkM+oCi9mlX8xGq/7m/hppGXBHX5wW7A6TeeXNfBlNRzziPkOGj3Pb9/Cz+eeEa7rG/vNTLyawif1GUSvYcO4EaiG/+q599jE8Kaj8yCLrsqnrFNpgP5k7kerq7MFOlT2rP9LTK3f8T/6+f8rn/qv/WvISYyfn58iSU21neCTYA0eyc1mFZTA25ZLaBtkhGwWLQX/GGcR7fUSt/b8lUanSHvzh5TI46VgCRDp1P6y1k9BR7FACchIq4ojcuHgOouqXMjhkTfzez2srheXz1errkWx3qi5BkcDR62vE9TmMu1bHsLhUgZQdn0A7uOtGb72fsLqxVKSRu/DJncky2vArVXTqQDaPgc9RFrUvyy4C4w6BQlKtOHogGl8Gn+FbyMg3KOcBnNZE3g7J4C+D/7Ix/dsFgKNhIA4kkrvY3E49DWLxXmTzkigFlobPJTWKxG8aHR8IbXn22ZoAgAI7AChfeDy51ArJi8/bbkb24aurPu1j2KiQOOsYUzqJ7QM2a8GMCcr4F5e7LeJVXd/jREx+3jTuoQ3hp/ht7SzNHgXCbVGdjZaGFMAt6mhmrUmuZkMDAppBuXS6aqWSeYS0af3J8+/O/cHQNdzShrPUM8xJqadQRNWIUwfASyrm47/jF+Y+NBPGD1ejYGadN6beD+Oe5azOF6cUosrWs2lBHiHiQyQxHvQY4nBR4LvwpqgWgzesETLVLL2ClnfzsteawjF9zJEZQK+100lOhtZavBhyPHOjB1yDHOo8gfTxoyhn1lGLiY1iAGQSa+n9YU02p9uj6FJJTFBy3Eb/FgLOhXaX0OQjjv84CERjCqbbwtoKoK8X1Xd9ckwwAwHJWUGfM5j5PHKfO776mYWrhWfjXf8E9k7bAsBVSakAnGCeW7DBFo/8j2Lxh/D7CWODdDGCN1NoFrvbSpEu30gEbWfq9elHHW8FQKlLljtFIvgRp2BYhdsrdfjd/1vBfTQ1jLilo0EjzlCy27KKQmNx4FBJz9Pk9aPFcSa66Ie4v6s9xn7wVvMGLFf+NyPHV1ZNWE1+Y99zl5/8R7acuErYg02Kt3g7y8mwC63i4+rwT8ThbbMn/a5Tw45CTK9KY7rng+9LyMu3c7wXB3by9PQG5WF02B3CcPKtmD5/AKFR8TfoluNnSsPvMilg8dYXST+1iml77Bwzps80n+DarNvtLaETfVwPuxGwQfwfVobFofDgwPeUPUz6iT0Fae679X70PNAT7oc7UJvi0qjsKGB7MttaHc6ljMapeC91VHOVt9Z3OQDoVqR9smhs9L/x4mEFQUyULNUeGuNpCFNL9O8hag8d/9oVafgtzbwi1dx09nI3IiPuOl8X7btakeNTvnctUyZyrtrH6RsX2tcOfXU25WFU235Rp7QbcCZGUuAClXQb7XJzCmD1yuhRucVn3hc/KzbxWdh167R29LaPkjTTaUETasHqwqlSaunYHwexIxXnpdO4snNB2lw+DGzwxPCYfGuTczIacnAHQO5QSMm9dOtFyi+byl7BzX/Xf9aD5+pDARQl8X8YFkfY2elOvJmVR0MzIffmlR0x/S+ED9V1gsO7aOI+8Zbs+m1eTATfplAoEbcD58pSeHCgVa8cbETFo/X1GBTETMSK2jT4wxfpoo8cf6NZ7jbH0qT4N4IcaKWsL7ceP4cM7e8SFZVluj3S+V3bSvqXU4Kq0QgyE/uOKZeraP9N/Rf/fUGllWfxa63Yw8RJ6+DXJkEaCDhCmxrEqc09zem0MoAGQngp9IAcpujWVQFg8qbSeP/LBwmq4m4wOF4PUfi4M1A5fr80+md+HzqIODMMUbFiI2Wk+ez2fL+ZNq/Opu2l8Si1d66blRkR3L5YFuq7OI9OqfYh3X1si6wYLYyFjUso6kFpNSoF7KHnz3BQyWwwaFCj24Mgr6rof0rgubYO6EXeOJSB25cncLJLCW9V1ltHlfemU9QixwSzCdobFJOXpdVnMc15BuCPpnP0V4iJXZV9ETe/XoIhtdvobwpUfC7PW5uD/ZnbhDoVNZ+j8dDk0fei/3N4l7sKd3NDF8YYgHtdSApyRzBD7tlOr02X04Ho/I70WskWqWnMfnZ+bTdJq4nUwMauN0A8bkt8DhUwFvAuvFbWTpzqSrrhsdlp4cJElXwDAAU/AHfS/CbqJWJJEHiDBiXLdM1X2drWkTz2bTvaH2uEf9LokayrupGLnw3ickX22D1iM/G9YVB7/VZI2nY3Xc3y15ewJwaNRpxDa7kLABs5qxmLuzvTTr3Bn6nZwnfWYErmlo39DGBn0a5F/vbCphxQzZ9CgPpdOxXLhUpm9WB9Sf5LQo6GqHBLTZem3RB5DnBg0uM9zQGfoufy+RC2NLQDINK9ChZAiJzibq8RfFOmXram4K8eDsn4mT60MUiZoztyfEsvmUvN0/fiMn5suDfFA2dF8/g4vIx7Mj0+q39WxF/6gIraqFJJYfThnQg+gosqgYklT8Abmk3nzs7Pc1j3VWa8I5a+pggTQ8uL7adTub3mNxmIT+em0eZp5tw7O8lH7HkxOt8eexdnFoRyBdslmPX9DJRxFMTOYibjjzEwwEwzaSSR+2+CdalyhOvThXAWqc3YJoDAtWZic7EuSlwwrRLYrP6xbO1rHzhC35aOF91yrgiqJae43Zy85Algk+rlSgbsZ0nP30TnwCRuSXEz0GaHjoZQaeSm7YLa0eqXgbiqVr6e7C5Fxx/StW9MVq+b9u1f0zxet35g7Q+eoaWF1oy64zIdpOdvxzLDd/T0Cadi75m/P38WAK8hqxj9QdeE/pXX38NWAJYTHDQBmvViaQYa5Xzv3OlIuMHQG7nLuxq35UvbxRzPHQWOQ7sukjIV/6wRXD+h+GcfvEhzjmmCod+1WoE1jofVn4zCrtB2aSvLdnPzWHPkBxZQVqYHPBFhIdTJLyLuhUDMf6wuxEkTZDgtzeeZfzec/g++zSbypV1NkmSmJLqwXDDGYZWNJGR+JDCP6vOQlVIGblDdmKSlM9ddeZyJgfITbyuKpTX16/tG+rEuIray5C7Wm4aN6nLbmy9MpOHN5wWn5++q5iUvpCWea8TX6rChqM1saI8lLM2ON3UWeEq1V+La402sdlxxf4YGxc9yzOPvU6TRZS7GZj4LrsTavkmHHQe5Xei0UAH/ww2REFTMhhKlA0mjcGPcSsd3LjCRb5BfYarVXw+8XFOEhK8HCE9oMW9TJzqw913g+V6YoLyQ+yPhUgd3F4ERZIXtW/2jwzu/RPtu51ndyzoPMq1yqSXH/TpvtDHR9ynTZo6Sh1m3qyEUsmrk26v4mXTIbZFwxKfQQT7iAGjX0RfWZO657dgFAEwp4OHsa8Rvo1X+op92/JcuczG9GXqdqHZl215kiVuBwVRBXgM4hpZ45Abu/eV2ohwK6UYikqPMs2znIDicCqCKtB6NX30phA8Bfdxmx/081EHI38x5gvuKYEtVhUB6Kt2W7vn6GBQ0nxqJQ2DzZCXCNKhFezbtIm9QDL8V899qEVs9p009WdcAVS64ZhbzPWrtBZaGeBARBBdjf8Br+z/R02t3/E/af82+/61/zVzuVycPn0al+ta8FdriGB7vQbJo1EVvbmoMRE+5XeWTl+KTSMu8p/Wd2Lvur78eCJOPBjYLsXweTW43c1QrZQdgOwfVGmGABh5DDq8Cl46Q43GNDYWdaXarV7kM9dpqQwpJX/kFgJQFq309Vc4FaOhqP1Z7vfNERaDON8QJq2aRKej4ZhOK7UCAXC72O/05e4SsP8d+rI5q82Ak89DuajHs7rqY24tkvnAK43KIsngpMFM/v0Xjs5/gK/PhwsIG7Mtg2dizvNMGbx3RUw+2kpl5Dkh7Arka8XmqyRJzA6AT1USxb8segyMOq2q2RdXd5LeZnhAn4NCs88nnttXF+FvKmFaa1HPQWdtTcjvg3jhk0d4yCQifDOqW/JZtTwFUa8VYYwdDM3oeYA8Odn6KZnCqBlKEqLHwMQyGKGCULfGEjP6AHR8E12LuxSu8wHLuW3T3WwpaM+NbnHScYh1FzMGn2Jbq7NIWi9KKWs8jx4PJ/fXgYSUqvB/hfblQN44Hk5ZxRB/MXEJ9wmn1KctHzoSqFKhaQGgZLc8TaByn3k8Huo8UOhCSU3mcTHOz4U5sJYdbhtuSVnM0Gk1OHVONs1Yxqc3KPV2AHzdOUxKlCdS27tFms4HDLtJ1MGUIsAonneytpEPQuH+5pp9biecmAsnn1V1/1AHuxplilWF2TykXEqhy6FIdPlZwnEaSUNBcgb93n8D+w1iITvbFcvr67rwxKJ7SfAX6X56XkykpxkWqTw7bkMQPznCKD/zJmePLRT/wNXIk4FyAdGbtgwg2nmWN4Lh4yo3riAl5VRhn/Vk3/Uuk1d8yYDxShpbo72CoKvUFtMe/hkfr0ha0vtQYQvGqgGrCj1MqceXNdo6FgcUkmUTk8yUst85HQ/DLKBRmZKIr9xJbbJ8XWrWIbw9sXr4Jhy5Kf3XiWloX/8tvY84SCl8TqA70Zdtpq1GpopyZ4nP3RfGBNx4aHW+FXabOKkzo8Ncnv34IW5fdrtqIDjB6uCBYKOqziBA+57vUZ36BI/dIBatAKi5ALsmQN5axcvOnGvF6WdUkIS1Bx5mxI8B9P3ueYZGeumwNJXQN3QR45fczqrTfYTzNrsbOTFyMyG9z1MSLiLYJSTSE+C3aOgRLE7w/6NZribQIb28HG6STevRf9+Hib9MRKtVnld49EDOn5wCwI+3fkdJvbJ5VaELZNOpeE7u7sjn+TOFj/25pgdvXt2+hX1e0lAQMZ5pvlCd1PwEUoglpPlpRr0fmMKFqb8/v96vjr/DO/tF6rDQhp8Y1ukQHfsfZ3CwuM5NsaSzqO8VPp7yHtYmr3WyoYBvw5tpsgAEdqBrjryWNUs9/efpq1DqYI6E3itgmhNaPamI+56p6s2x/W3ocKoD92jFZ1or6WixYiKN34/D6PZqSPu3YVqxnswGHUdqxCajf8cvaJstT9g3uMVnp6KxgnjbFb4KA3NDluBvcDQw+tCv+FxsIksnQvfPS0Gk22G8FdZ8/91f9F//ZDOaGjGfhZMLFxIxQowfzgc/h/4yFLjUi++zj49haD68XgEOlanwS+0/5MFS8KgUbAGZIrHuMiSIRVmAnZauuD2QZVA+lxqLRGZiJoVbOzB9ifK5Kyoq4tkTFXxhfIfPeIfPl2/hlZdfpqjoWilB62fkp0k/sXr8atX7/4pdz00+cLPafag1c+ZqyF7jVn+ubB74pBp2NDaTTAd1kqm8fJIUL6ca6umRXERk1zP08FU27HRuG/cd70F1TiT7R26kwFal8PtVHWV3HOxshC8qxSajXqsnvDicoMoAqmpEuu7KM+/QU1vNfTURGA1iAdNacZCFwTK4QB8itpKnu8+yJAI2R8uaL9c+2IfzmuHXrkPnNUVv/523K+TvISpP/MLPWJbz9cf303L/EkwlYlPtjp7dmf/eTJ7ZPFp1r33Sr5H9sTJyX9UKr05eqNHQup1QeQxcjcL0kk6rQfJIhOuq6eIvspwcynyM4h1tafP9ZLrpTiidznq+CujO0nB4IQi0XjFGcM0JfokCP52HYrsIitC6bAyYuwTrx/NZX57VzIX9jdVlwclnIHulIucF8NM1cKMP7I6FFJTxtW9TPguCwe5fw2VrDW6dEtRmdNfQ3wxDLRCroqcdqKllkBluDc3C7PKi9JM09LrhaS5dxcE1t2dRshPS3xHyXgAqjsKF92SpiOvN7STQGUFHo3xu3hYUfiO/HEjjpw9v5qsMcX2NtQZjaHuRHW4b+RpxmgzggVLo1Qzzhg4YaIYWBvVpsLf3fc/xoiEcLBgv+DRNReyOhfQEGJGv1JU8xQIW7paZaNRA+iX2Dqw6P4c1Fx7HoxWf6fPl/Zm2qpwtmWK8ptOBX0NHnj9zB9vPqsTm8dNA7w/rU+VahZo1FqoCKgFerYCfmiGAKNOe5q2YAg7Hqkz2NeTxeNsr+Pc9ygHvXAbQO/OZWB/C2i03cKlMfOhvr3uN9AQ4FgdGjwiietqnhgsJcKJ1kuADIHaC/G+lOD0NkOfSsLgaGoOVRXCnseqv/x/QWXl/FhUV8cMnL7N9aSmfZ/3Ezt/2YwoKYiWwjuZ1rLpf9f8AzOzuoMwF71Wp/+3aKNgeA51M6qiHGFcZfRsP06f0F9HpbAQkeZ8WJtEkSkNLKQ8z0dvnVuHQTsadJHZLZ1fHYzi1AQqfK7ATU7ekYC8PYEgnuek7bMIERIVUdfscCGoHL5aDTuX+Lqpcie5iKMZGC6YmZb7ucrnoEV2A/Vhb2p9qL8SLpQGNBJSF0PW7KYS5lACrurDhvHAVa5AWJmqDDfFkUfL2Syx4YQHpHpUaRs6PcrN8cy+4IFJuP9Z9JovHNnP/6cw8M+wZZuiTeHCMShNeZ+GeggDa5sAV23CF68aWd3Kylwxq+UYSv68uef1JPW3AVNqETRIBWqkRciN4VD6gVcYYGg1MiTrACKssGYJXY9dilXB7dHjQNDtwrhmyBeOw30RH7E3Q7XNGBt3DuHgvJjS/NGaXyiDp5bVQ692Q0/tz+XQqFzIjeLRUpsC+3oKLVrI8HFKy4dEc5fcFcNz8PcPO9QNU9iS3nY7ufAZaYEbktmavi6AbIPF2GWzoZV/f+DU3hScwzF/5ff75WToJ9JJHWN+1WigLK6XL+J10ThTpoYvd8tBBraWeOqdy03Mbw/jQWoR1/Gbua1ki7LSSq4EEZyKRmfBUVkfVS5p4ZT6fhTWzT3vcPNJ/HlPavkqnECW4t7f+GLP84VgZbFx/9C8GknGAyEWkbt/o4Y4+YrOvQh/Lrkb4Khwe8BP3cYuniVYGmeFNcv2dmPj/t02t3/E/af82+/61/1XzvvHLTUk82eI4AR/Pp3tnsanwtT2eh1sdojj5imqSmRuhIyMpg4xKq+qCWFC4j4YWYDr7qvoJXfgA9k6Fo4+q+y1xsLEbHFLqRXxdVcyI2sN8HAZPBYrFthhDFeEGDzP8PYKOkEajo7behHH5TSSdaC+c90j3BUbf/gdGSxPJcWLi7ag8yS7fU8wJlKdX/murz4Kzr8jaTtXK73x5XTXf18p84PGh/ZTHVRzn9kHvYPerZWVWoFDA0TVmcFfcTgqdcK5ObEJOsO3n6z9rXSoFnDnuI7wTCrc1x4CR9T1s6iUjHPVioWWZM4aphTI9zfU7lEano6opnA7hW+kQLk7ahJj9STnfh8riIM47xMRlzYXtPJspF7uEpMhWwdbYD/GkwAs+6pWOV3e9yszVMzheeFzVD8iUOaa/6XK2fgraKAOwXFs6KUtjWP3pTZhLRLTfdJ+lBGq5Gnx5FXhMYRw7nkLa9v68VSEi0IkcynsHlvLWvu85US7SwVoqj/KR8QxnK7Ko0ahQDNRdgXOvyUhFlamqWWHBvBkCKXqUVC06K2FlLl4duIHVTjseuzI50NoKuM3lj395MH6VooaL2VNN206X6DdhJxeaxIpDO2c2g6++rLZevG64QL0HxqrLG8HhB+Sp0k5vqrrf87ThNj94zk+J1AouyGLCrxNwtztPwYiHheP6VNUx8vnn+eLTcRz0iAlIgMNF+1/H4Lc3SpU2Kq7kET6tgmOlIvWjpPfn3aISrBJcyBHvfwxBrG/1EROjWvJ7CzF48zdKxOjAonPhkJSJe//EQXjege9bfy8c59YHUD3+D0qH7+ed/PUEeU1YnCr6ndC6eay+cz5rPxM1cyy4eODze3noo4eQVOh+ynzlKZh1UWBR+YMGrS/FTuhrAuziCFNFj+9pmQXjiwxKNKDHQ6D+Mg+OCmDFzVrKapU0t6d10fzYNZ2xd6+l6ySx6XtjnzuouuEUgVWBtPecEPwDU38j1L+e6LBqDHrxt9wX+zBLK20cPPKKeNGAFDkU/4JfYdtQVT8ZX8uTp7XK5Nne5WVykwO5MvUX3q4QJxl6tZSps8JzC9DjVSEKaM9LRy6SeSaZbT8Owd+kXH87Ve9i84STfDzqBz6pEps41zcxnAbR/49miZYBE8O89Ll0Vm7J/xy7fzUhqTm01Yq4xeLEnrw2512OhIg+j38r1sZmUOVfRfcr4uST5NEy1gr7YiDS4TVNLmnQd/+C58thd3MaW1kr4AczHH5Q9R6kIVemvNwvonAntHybcGsWZ0pFYMuV4Jf59MANnNjZiZ+qRM3U5xqm8NwPvTm5rjUnKnYrnY4abveDnyPhQLgKB5felwo3ZCXAg0b1yb4LyRY8KbAiVWRbULM/4z4/80Dyuso0aw35YiJ4R8hmAJrMjQQGeNFea02UV/lRdyaFr3PEICE1vC1n7dAtF45mLBL8pRe/ZnsM3OkvAxG8rThnAxuj5OlTtd/S7XEz3RdWR0F1ddV/RYcT5vSwhGBujhLPy2rwIUILU3wgQlsl+FuladneCM+Ug10vxgh7cvZwgxG6mZpp9rV6CobuUS2CAGwwdEB7GQ4Y0hSv97Ec5o3HV1ORkE2m8Vpj87133iE+JoaUfVm8aatlObW8Vl3J7oULiY+J4b13ZX2cT0IX89OodFL6nFRtYpokDX1y4Sk11j29D+1yQLoE54zihAbIepOvNYd1a8iHTb3hovh9TyzqxdL9qRQebovFqgQBenQ+bFw+Apddz9oGDx6Nsqls92vDY2eDaPfkfMb9LD6XAQW/MqlTFgC6RjW2EDclTrhocwgU+gAOZGo7uweMOrGh/ZMmjXllsu6T2+/ac59ZmcmmbuNw+9TRbdRefOuVjcYyfRxl/fYz6ZEfmXLPYuF9q7K16AqqCAyvoNAlNtUSQouwNZiJyYqnbaQINgoxyeioNc0xS7d6WgbpDReLtrgaYEt/OcdyKRvo3eqLmIkPV+5azk+pJ4RD/TomcaVdGXWjtpLu9lqLnA3cFraF6X6wIFgsfsRqPYy1ytNHQSr0dL4GK13sVqwVgRh1KtOj/2RXJy8LQx8UXK9E3MvLwfBICWj8lOtcuV9HRq5pR0ZhEF90341H51VEjB6NTwbMDoDnw8Xp0knOL9gaA+8mnCOj1KvI7fGgw8P+lHBsKaIUAwCle2XNsIEbr4Fsrrf4q7mGl947kUN5cF0BHTctYcZ6sUBeE/MwnzkaSU9LJ1Pl63zZOJzNbhstL7REavBaLxoKeD1EQz8zfDr6U/FgexWz/GFbDDyqMqEPcKG8By9s38ymjLsFn2QM5ZESOGeDJq8GUq2+K3X2IN4Y0ptE/RrhWI3Gjebq9JpaIdoj6Xmk213cc8Nj4rG15/hhyASqwr5hb50I5CDtIej3K7R+RmAXAqDsoLwh/XGD6jW/WAHtDDBDZemPt27nnA1erkCYOqGpmCeDHWx45mEu/ioebLRn0PVCKy6uGsL5AlEDMdvciZM2eKYMPCr5YYBHHo2ztlHRuQcZzDvVDoNF2mqA1lOLuWlGGcmByomu7Z565s+fzycvz+fO69b2P/esiz9f4fEiD6vqXbxUdIm8gwfpRfONvj+tO9ATOFyXwHKV/vefttHYCYAUe5b6H4y9DP5tBQ1PAEp2wNpEyBF1kldG+/GYIYLgkiaCytcJ/qd+2srqU8nkxeRh0HtNMJvDOWr3kIGTOWXXbtB9gMpqrLCDwH5gWwPsrDNj0Ys5dbGmBacf/JIus3/ks04imKOk3bukd7xESlIhQV6MTik+19b7Or2SOcPlm8z3EVn8uuAV1pjFRvkI9xVmB7vxaDzqzZCoMTKrUsqDqnqXVbYwAk1FjEn5SJQHqLlE3zZHmdPvdsa2eEt8b8BxNVYzoUQGRzn2M3/iuwxduB5NexFY00r7JeaAJnJ7DEZSaYQvyLqV0EzY0ABar+9bq4VNZ1/npjwDw/N0SCHKSWN/Pzd9Yn9kcuuFSA1KZq6/zCdR0PpTWPkREchhjae6Zgenr5ZqvJld/mjwsN1aySn/cj6sAjRK0I6l5gS3+kGjG3xs4tNW4rpE+6h1HIiFFt6afaYwvkj5mLu1o9nY6iP1RbZwE2zuIwPMVSy6IZ341vehG6WswSXUHGRTNMwuhVmnb0Pn1aNPaHqfZ9LbUvHeHRjdS4T3nTr1SaKH7aVkdxeyChOUToM/X+sr0RWG0Ts/QaRzzFrBpn5zaG+EBrc6OMAp6ZnlD6M9Yo0ZYGjUQuxBg7GMVDZv++oO0s8MN66DPhrdX+vb3cB2/rPnfocTvlfB3Oi0GqwauMUXOhlFFojexd/xegh0PDGQCl1f8Q3+L7L/rUYf/Nvs+9f+D7Pb281k5I43aHx6Lm2viAGQJEk8smk07y+9hzREHZVPRr1OWFkIkYWRqgWFp3zlyOyX4iz1E0i72uQr2qLuP3g1MQjto3j5z82urQFidOLUYI/QdLp1TWePqZpcjVexwrcFA0oc1ASX42exCectBXYkp7WO3194nQUqnN4eQyCVLnjQH+JNzYyt/J2F9IBO78iNPi+0oP6qbktU+S1E+3oF8c562qYdx1DjS8/jswWUliO0J8lZMsWXW0XH7XjoOD6sgsk+YPaIEXN3ClhbBxHq+5psrkawlapy+VdIZn6ok+lprjcNDmL9zvHU5n3M2SrSBUYUPEnPRHlCZ7NLpIMYNQrGBZSwIQqC3V4FY42Bcw5Zx+Gu4CrVU56X9xzv1X3Ls+uakcbdNlRO2OpULrypRC4Ue03pALhsB8jrcZhGkw3ixGZfrSeRYqd8b+lV0JlHOx8lZMRu9F3FgkG9vZ6f24XxdOgtXPCo0DbpAzjQCEUuMKhouOCsl68nuJtcqPeyieZGngqEiwkodFqKiotx7nDiWF5F0ZoMFtx+t2JiQKo6znNNYXRfP4pOOWIT0ePfjZ7WTEZ23M4FSWy+vhR4Dx9VwaJQ0HkVlgDqk+5lV/B4Ppui0hQDCOkuIy/DB6u6WwanYZXAz6tQ53d1HdKcbsVbqeI02IB6uTDU8kJL9I3iOvihn4zDzEpqUEUXdw68wNp6+LFUjM4kSeJAE5gzYLepk3iwRsuYTg8RbvbHoELbZI4by7hC6GUCbYPXGlxxnBNLjlOTW0PBYSVW1GgOpXpHX4z7O1FQm4rkpQkVUr6SecFykS2hUvwtPwwoJqwwipDyENqpIOsr/DsxvgC+dkQRaBVL75a4CXxdAy8EA/WiaL3H4+GCA7Y06UF3HbLf7eCZNimM8K9mT5Mbp9fU9+GaMpZbLpHV9RiZ/uI6Nz11GNLAIlq9sIhalanXV2z3UJofRluDSpEFcEo6ZgdCD6k58VJkBGO8+MwDkLVMnoBrqZzO+/DiSxxu/xWbfcvIsPcWz8vPl6q5n3DHC18R1uTVvNUaWGF9nLNtS7jccQgWgzKZKzK3YGGmL77nUqkw/qB6WjFZWgIywKVTAQf8/2G/Nj7Guefe48bZ2+gUoryPCi5+y8KxLbi1MoTOOQnCXquVtLi1LixmG2E+4jo4O2YrqyIhVgeSN20Z4Gv0pfWA76nu8Z06XUdDrlzAvrRI0CYF4NLVwqVOLKjd2u4FBiUuVSUdqNUG8VPfNXx151dkS2JBISxgHPq1Qzm3vh9XvFG4fqnEZbbg+1rwc6sUGD1utMCeJsh1qSehfwI8AtSE5ZtK4eC9sCoULisbC1PCZ6LjBTI7jqc8Wpw0G+x3lGGfLGTp3Zn4eLMWuB10rl5IaVI8LSrEeyxEW8+Y2um0LnqaAJe4DhrtFTS5YWQ+NASKoAidu4FhVnggAAwusWvxsOcETwfBLUXg7+//X9HhGDxWPnnqYX74cYjgj9c1sMDYgZWR0NGQI35uzFaeuvoz6bTiPRZTuIqjcfBZUDOdFt8WkP4uHHlI1f3RIVl7a8mJJYrXa7V+HDfVsmfWN6R3k2lR33vnHV6bN49dLhc7nB4l5VlTE7tcLl579lnee/ddKmp9Ca/y40uLRZWu+NOoKvbEQnEz+bD+6hTYA10fUPX3NsPcILhVo1LU8jhlPcwjD8rFq+tMI2k4FZvNiQ4n2I2XBpfmWiwTWBkonLfbN5mdZ2UWkZYOcZ82Vx2n78DjDJj7NQ6VRmRV9CT65UGVG1U2kXKf1txfAi31oLeJXdDjUgQLK2XdJ73ftVjVU5/DmVaXCPBrJFdnw6NXrgkZlp7McFaxIfUcZ7Rp3m/L0rrHOT54GysnrGKHrq3gf007nvLofPJrA/FRyTk6jtjAG3GvM2m0GKcCYImCgLbqk30ak6wTV7IDKpTFuDZFtWi/nsKJYyn8UBcgHFrqmYD5zo95u9tu8vECsBiCePXsaTrnQNtsBE2osPZz6LJ5CZ+HwfsJKrR7eh8WvnE/JS88ilSnTm36txbQHtfEGopDRJ3kQ03DeacSlhS1ISVaqaXp0lkpbTDgU+WPzqkT9pUY/2jaZi9ifgX8WiU2gC4YevB5NQzLhzyN12/tceO7OpgeFOOOHCmCFwGqzkDWcrkZrfZ7GQLlvDl6tODSaiRiy2fg2yRO9+u0GpIzkuVmnlWMvae0mUJF1+MUT/8RX7NXEbypiDmBbt4KgXEaFSReQx4fh0GRE35Qmfq+evFIuFX3U8kQwEfV0CYHQrooi/tGI+i1TYRbszBqxOL8Cy20rJmqY8m4aLQuMX62GOroHrOGxACRNQZJT4VLR6MHPFIz91hId+i4EHxVqAqvXKVRU8n//rQnA2WtLW97PPY5upjgjXIjaYFev5d/W8aueQDJocepUzmvgA78UCivA3aT2Cg/ETyZjjnyhLJbqxLv9fhGBm+l3N/seaPRq99/wE8HX6dgdTJH9z6iPETSMH71eKa8/xClV2OX6/eszXaHYs8KcblQUWRTtVlAUZaBx1xj2BQtNvABPm3wY1g+nDSo/FbAk1vm0ClPx89GlfaiNREC2sHBuyD/d4UrSGegRVoW3UfsJ8pXZMrJta4g5VwSr7tC8PMoIxOtpOVyymVOtDmD1Z0AwLZ165gKjKX5wv/Bq/4pQHIGVKU2YlC5/226AFaF5hHVKhsfRLrY0owCPJ4GFrYtJbxY2ZT47GpeXO1XTZNeCTzTSBo8Gg8nPE7WR3lpjwKJo/fz+buP8uZ7TzArRGUCNLC9DJLu+jFEiPn6khNv8tO5Z5jV5RFoVNJ1c2w29j+6MiHPxjKnWL/AZeN0cgWeFGhvVl6Tjlpi/S+z6coC1hXsEA6N73Cepz95nS8f7oumTgRSN7n8KHNBVMUU4vy82BY0cKW8N47dNh6tc+BjUD7Yeq2TOX2mML3DPIKcojzL31r2D7BnKow4DP1WC+4Wuv6Ms0JOAkQ6lZqSNbYaikOuUOtbe/U8veKmrp8Rf3AqvevH0ts9QHjvENc67vaDSC20tIrxxx0d7yCzqYE3Tv1Ck1PcO3DUyiCVLf2goUD056+TGZkcSsCl2VFNVyP4aEDy6AUaz0DHDjoENWAzNfFGgQh8ueyRaEjLYF/PfXh8RZpwrUtL9cEOvHU8RvhOCGjLN8VpBGOkQ5OyvvynhZ/Jps4NrR1qjVsJxlzA0H+ZTFt/nX3vmsbgfPA/6cNdzmvrcwTwOv/83I/QSriHwkWj2Fvt3bCac3GQmgVHa0Qt7wLfDnxQCcUBf3ClTmUq9l/7H7F/m33/2v9Rpq85zO2p2yiPKmJFk4gSH2cswm9fVwoyo6mWxImAiIN38/wrXzHmie9U0T064LwdfmwOcRrSA6a5ZdSpmmVdHXr2SmxSPJdYFAoD8+DJClGH4qizDXvW9OPU4VZUa5Ro7D/58f3Kg5nk9hHRlVHDWXhyKqeK/Km2i3RUWmscX1RDmA6izf8PEKc6KyTNhCE7IWqkwvVJ+Ct4UuBC1x+x1ilRdeU+rWiTkYJ51gZGjj4gnrfeQr4TypPhg5Yiwmb8wE/R2mL4MRJ8beKG/LD//YwrhFD/BPXzTrgF2jwLO8fIhQEve2PAiwRr4LFuyskpqamIRaPbMKHlO7hUEDQ2axekgWWsuWMjgVoxsOsb+i4vRWfTxwxGvBq7eh92p8wiOhNGF6sXsXMcEKiFYXqVyQ4Ac5RMxbJtmOhz1MiF4l3j4Igysenh+InFD//Gksd/wRQm3gdFDCNc5+FUHBi8E8nqdA61bOTksK3cViMGT/qcH9gQYydeB3khIi93nSWBlXUyYtqk0kghoC1MaRCmEf+0PcETmFMGq6Q00AcA19CXffY4eKepnuXU8kJurnJiIKgzPxa+BEC76t+F99XrzZS6ZIovncrUVOuwNvS3wP0BoFUp3qf2eJfRAxczNEYdKUvyXdBrmSz6rWKTbQf5uBrS4+5RvF4Ude39nsgJEI5br79WqEq5KP4eJ4xtefT993jtyY/IqRW/76darOHDUNB4VIrzjcWciYM5gc1otLhd8iRm//UwUPxOfa16xlY8wROBEIJyIqZ69zR6jtiDT0AtF68ohcrbhbfDou+DX42JTpteA4+yqltpbcnGzofQv/U1yfeI0xs2v8kMfHg10b2P47KKoYtG0rCmHj6zRwv0XyAXi9fWw+M1ofKEtpelH32RuTHxvDTwJaVD0vJD0Y1MLYSRBSB5TXfotBpCSkN465cefH1RpCN27nuIYacCWXgqkixJbDqU6nLYN3Y9n6WdUN2zymtzmFoIS2zNcKKdmAvZK6DlY+r+7t9Ax9eFKN2/rpybVt/EoH13cru/qBmS6fLhd99SfgrLZXedV6PR1YTks5atNy+iKEEsZCS0uofLX85g9He3cEu5Os9KvsNFtRvOl4rN7n+0+mxZ82mrV9LucTPE4uayA6Zv3og9UDltc7nsHFpHGXFfTOeW9eOE71tffZr3xpzg1Xc+pU9LkeAkxVBMlgNisyAHcZ+v3jqS2PyVtAhSnz6i9dOyFtWwA+po2shhMmq/7fOC6+kte0kOOsano8RJGoPUCKEV5MblMslXnF4aFtqXtXccYPHdi5G8w35Jw1DLq7yx/yse2ayCnK84xsUEGTSzyqY+Debuv57fu/6GX+8lotNZCxmLwVYGRZsVrh5No3m821gOtXyArVqRSGp27jNMr7KTYK4Cj1cRsfoc89JmE3E5j8H+YrMvsGAp6274lvejS+mXtkfwl0dPxJwBfzSAToUi1x7ci6dKYbQVzE1ifFLk0bOjAVbUwrhbbmOZ6T8Tn19qMuMJlieXy/PEicLA8s+5N/okT5TCGYd4H/VyX+DNEPgjCgIlscU4pEkudG1vTtfD7YKai1AvNhIBdicF4kmBj9KUQJB0e2sWfzyBRy+1pyDZTlFREXPnzPmLEkjNugPrbDbmPv00D6z8jo9mP8r7Dzytuu+cLR7G6jpIqhonvpGzkZ3RLrISwFSbLvqB+CtybO9dQwHkKZgeS+RnzovCv7epgha+NiSPhN7uBbSTJD6b9RkA9+3vj96LSlYjaciOz6bn7b/RfqZIhVXd9hV6HQ5l7vEojrvF57a4vpiPwuBYeLmwH4JMoz3AAm+FguRdgOTaxOknoz7B73pdPo2R7UdTqCkI5crGXtjMysaXXqvFWmdl+ZYOvHhZpBd9Nnw7n03fxf6IfBpVJnEiw3X8PHo9R29yqSLru0V3Y06fOYxNGyv4AJmN49ybQvMfAK1B1tppPUduCl5nm6MSOTN8M9aqAILKxBh3qP4gjwTI/y/spRotF1w29Kc3EXmoEH+TGje7hw+qYGO1OquGOUeO89Seu380jRZq0rE2nhFcvzU+wHtVMijSe5qgR2QnwsKfxnfELj5uikTvUhbYJVsJfYNPsaIWfq8R45rAhA+YnZPM5gaol7zAHJKGQ7okHimBd7TNrBeJt8n5sDlGplj1NrcdUh+CmAnK1xvyGJb0BYMSl9I6RFx//ctWMGzMPqY8vpJ5wWIR8EZjHTMPDGd4dgqh3kAn/9b0Kg5GC4RdFidPscYxvgAG5MHuBvVm36JRrVk7TcvEVqLW8fXrkzd1dXfnNOb3H8nMNflcQZ0KGSDYUqA6dJIUcAyN5GFX9jTR6ZdCr4IGng+CezurTLmdeAb23tLsZ5JyP/Rb2yzTyG9RMo3ng2ViU/fH0gd5vwpM9lgsRq/vTGtEWzQBjbWBFp1FEKrW7M+ZtmcIHL2DDuEihajuuqq5USU2/0c7/64c7xXvUHUnlKynnVRNQ72S01VbWk3Hkx0JKQ9h+MF2/7hnVcN/NaHvsNcxOnk3QxLFNRRgTeZONjfA3gp1WtVzJ9/Br/oE83fMF53+LaHT2+DfRlhjMyJ/YM6pQXxbZuDdJhHoXNV9PtZB+2nID8Pg1R/Vl+6iKRkG7RzAHWUysKC6poaZwDNAP2A48B2wBVgODLv6+jPATEBjB1+NCJgAGYAUWBHIlF29yE7aLPg37f6aVic78EiGlcpYZVN6SUYCAOXR7egZMUjhC8z5juokGGxWZ26R/FtiPR1IQ7UvZbbmOOn/3rZdmcHcLTsFqm+SZjKvzMOaethWp0bX7iHVKO/fWi/wwQFiuOHoFFY5s1XXg+klWrpvbMG7G9uT7RbX70hjLTui4XLXn5CcyprRn9jQ1iF7GBP9lAAQl7Q6Vp1/iq2ZMyh2qzeQmrWqU5DzQ7PUzXdG3sBYK+Q7wea11/o35jIoI40B59pxMg60HiVrh4/Zn/aZK1jdejtTEh8T3r6LayXDLdDqQiIe35lKp8uOp3AjjqLtXMjdjkcNrRE3EW54D4K7g8o9SsvZ0OktqFbmnpfDRhOYCS0NMCL0nNDsSw/5gU+zRmBsMmG2i7/V/dmBnN/Ug35xFUQFKcFZ+sYC9pmCcEcXYTdMQOcNrAnpwWPVuXwfbWN2rChJ86fFXoGLLV8SHZIk0/06amTK8OusTBOOywMWV5Owvj3Oted+MMrnvi/QV6PF2TsFvz4QrBHDvXpdMAeaoM4Np13ilHF2YH8+qZbp+ZvqcgX/v/Y/Y/82+/61/zXTaDSkpaUpxpm1tjym9N9I2PBhdIn+RjhmgKGG2NRszrS9gtE8RvBXSMN5/5HZFP0h6ggB/BB4E62z5WS0WZMkdVoFgNZzofNHAoVGpKeA+wMgQqc+nVHiCOdA9wOcTksnzKREhGgcVdwbbiPglXc5MGyLkKT+fO5n+vxxicfff5xoFcpjjaThxQrwywCHWQV19E/msssJW1BnoQBZ7ArnnA32Nrmo8Up6XB4XLvcpDNtaE1SjpGMA0HtcxOhkIeeqQDExlyTY3uTP5EKwxovnHWgKpIPZyOOpA5o/d79W0OI+sIg0dD0r1lGWDO9VfqT8XIMfK888T73Dn3g/cTo0P2Akr+bncmcNPOInImVtV96kxuPANwNCIsV78NfM5RS44ExTleopd8uF+0vgsLuZxmzPpdDlE0hVQQtaE+CmYvn+K9uncGWah/NxFXwUfwFbk5js2SQZKdfOCEHGAMGvlVRlMgHQ2YoZZoWsRJiVJGoJljaUogesEhjUgiuAy1/CqjBZINvLoqL6Uhp/B8UtHgFjkAJ9ud1J8xMDn37PvurBWNJqqYgUC9HhoU7er3uZM8FtmWzsJ/jvjmvLepsPoZngMYjfCQCbesLWQeo+gPWtZDpZFRtvqOSFFl0ZmKhsOmi1Gi6kygliydYewnEpCQ9R1yKXmJRcwuOKBf+vJ3byfpWOg9UGXCpNytcrfInUwagkFY1PSSJECyMtEOhVOALAWQNrk1j8TShzt8wV3HpHPmNvaMFG51b8UpTNlneqJL4buIWnH3uH03Fe31lTKX65MmI+JfU8xV4aRlW+bXihAmZH5NIrRNRUKPZ7gGm04p6ha0i3itSm0eXb2RUD0dSL1wSkSbX81OVGJvR6WUC+AbQpWc1r5mweyZoLjdd95xotr2V144erdV5vUENq9T4+OdmHERtH0HGPuF6UBvTk8KbutD3TVrUQ3TG8nNj8GBoLQ1VpcdpmvsdHofBLjoqGJ4CkBUkd8QzICPCmInkq+DrrWSk3iUf7XGSMr1iUTToXwsBFd/PeiWjOOL0KozXppCdARjzM7jcap1vZiGkf3h6TU16TOySpB/g/x/rgbHGNwum/sj8/r3ib1+tO1kc3MOnZ+QxYtUJI1or9O3PHBhkpWt/xjBAj6Fz1dDHJ0+gLq8VGzMrwe0nNBo3bQJJFnIoNKd9Jn4q12I891fy5W+NkdL5WpQgZPlBG7XsVuQEyKzuxM/sWzpaK61ha2ZOUJ4MnBYb4iBTknUvas33gZsa0yccXL5CJq4kh1mCSNf7YDCooWGMI39TAGZtMXalm3aK7MSplFDF+KjRvlniYWA7jc6H7V4q4r9hwM3+sGMLNa/oSohGbOI0Eco8f7OuwGUej17SNMYRLPkN56rM3GDNMRJRKkUNJ1w5naNI3DOgqAlS0Gi1hWuhsBL3KVLdOo2NdPUwpBJc1QfBnx97BwHwZrHX3Pfew3eH4j+hwdjrsBLSZwtLpS9k4eq/wN/VBk1iQPoVfcgeDVpw4rOudztulvvQ0gUVFj+oLv0mkZMF+XTMN56OPQvUZufmlYj195ObO1Chlc0rrkOhwqgNXvr6Jr44k8+XixQzU6/8jyrMBBgPpFfP/ek0N1HD2xA9M8IFFISogKY+TniY38XrQV6kD8YIDU2mdDXG9mlEfSpoBEwogVDnFPDfgLJOrIulwqgNRzlKFT+tsYG+0RFhMMZF9jiN5TVJay/ezo2Ml77Y7zN0q9LkanZmWJzowYuMItCqMCIa6S8TrZO0bNRrPIHcNSWYfFjTFgZ8IfnzSdQBPCtySNV9Gsl81tzGE9/Pl6XSTzSSscy3sp3jxSF8m/DqBXpvE897mjODLX3rz8Kf341MnTuq0ceVxU0YrCuvWUdagwrtatE0uzp98TvQBnJ4PJ+bA0UfU/WF9ZYCKV56V2noEF3z19DjQk8erxDjz1e3D+fGO+cyfP5+eWq+n0ePmgHU0Ba2HYbKeF/bixtordAzfwhuV8EOJCMpxu53oH1pK0l2rqNQ3kx/+nTkb0WzrR0rWnQKFl0GrY5gF9qdexlyjBFiF1Rzn90FT8N/Vk9xt3cUnp/wwizp9xmAz4BHvoVbJVkwOeV32eMslSBJfW4byUbWcz6mazgoZX8JvreQ4wtuufAvrUqDCa8Ko6iwPdJnF4z1mMrun2BSzVG6jXUEsPy0ey7EsFUrKE3OgysD2EgsVJq/CqtbEiYYGZhTDR+aR4rF6P7JyXuaCA8Kq1BvOx4tkMGV8gLieSI3FnIyT91PdJSV4pk7XjtMlcjyvxqrxecEFnty0j7ErPLh14jhvhT2B5ade4kK5+sopASUuKNWoMDlUHJWBXduGgto6GNAOYsaqxxZAByNogRKXeOL7a8fR3wy/JuQLDWdcdgyJDp7/4k269xY/V6uFGL8BzBq/nwSjWHwfe3WdOh3cmlQ/FT21skPyerFfnHoFwH11fz6urnPf0ZbO7kbYoFWukRbbte/wVq3lH/csf/ivJvR9dbXUGcNIb/e+6t+ciIPfxVDuL/s2AnbGQC99MzFw5DAYfUYAYe8tW4exMp20I53JzBObpz9GOTBmx3D6eE/atFCChjWmUNZmB3JD35N07S7rP/v7+VGEXPjPRi78vw88AHwA9L/6+uNXr9vPDEtPvIZ0PQPKVUtwn+LRDx/l1mW9+eGkMo/TaDTc0eMECV3PEhk6mchY5XVtMMn3RjfHEcw25eSrxxTO/ibYEgOfJSglCQBw1HJ4SDW/TPiF1bXiNBiXF8v32E8BIuUw0CP6VwYmLONsaV8RvBs3kXeq5P9V1U7XmpAuyTTjuwKVz3xWXTpZ4R/Sr/0D3BA1QTjUUmVg1Le3UfvtTZx2i+f9cOwq+ltgb02IMNmq0cDtPW7ijaF9GRD+tjjFJmlYcuJN3j+4hBp3onjef2ftXoKpDlnzvcxrL3W7MGqq2dkIPfPgokcZKwY05fDsC98y5uVFGCUVGvzay7QM2c0PZ+exO1985o9ZHmRyEXhQ2ZOctVj2TGB3LOQkNgNYBhn8OvyAuhyONQ4yvoJjjyleTg1OpXvoYF4NsPBQzDFhfdfojFxKNbDpsY+xRFQJb6t1awnLSKRLTTARkjJ/1Gj0VDkltA0mEt3B4mQf4Ja0zCuDXYUi4wfAM2F+DDBDp7j+qn4aCuC3NrB/uuLlcRHTmFX2PtE+zmbXNw+QCywA7gJeAvKveqzo+TECrqjcQq06LubGQkjUQ6BOBDVoNVqGWWBFJLQIbIZK9v8CU+t3/I9+/v/Kp/5r/9pVMxiUlH+HpEhiDk5gs34HVo24LNlCTrGzw8s0JSwhVt9R8P9iCqMgsoDvnE2qBYXBEW34OhxuCxCDFACOPCwHBKViEQaAVk9C8VbIXKJ4+YxlMCEZMr3XnAQx+dC7tPQ42IN55cmM9FF27HT2chaazVS9/CBn1vcVNq+wqkP0mrKZ5HaXeb6HqPMmNRXxVRiMsECjU6QQ/UcrPwCrI+VgyAsl82HTI7TJgREFHk45vDT57JWMtHooKgjm3XSRi1lbvp/LCTKS8DetCo3F7sksS8jg5zqwBAcI7lfbDuFEkpVH6teon3fVWag6KU/3+YsTNYRcbcBEjlC+bvDnu9MvMbLFZ9zXRaSE0mk1tDveifJVwwmRRDqI1aEPMrIAIiomkhLsVVBz2XjeWsbsAHipuzolSbELPquGDM/foM9SHxBo9wAZpWQKg3FZMEJZ/C839yLv+Uc5d998ai7tEw5N9MhNz8Q9j2AxeU2V+LekdTa4gBk+YhFF0+YZUrJgTyM0asXGgqaplEQ9PFAKuqAO4nk76uTmpKSRiwdeNii2B1+PfJcHuj7wX08M5JlXMWfKu7zcRaWx1VjEo52ep03QGYI0KqjkPVP4NS2F87NLSQ4WkfdZG/pB3WXeLlApgoOcNNSky01nFdPcmElw0lQiGpTJib68lsRCOQssihSLZR1bDGJPT1++GreKEyEi1jQ3x0PtOzM59uyjqpRomsLl+GhgdJPYjMYUxrQi6G+BVg6VKUyNiTcrZORmQ86vor/yJJ3d9zO8dxZ46cfs8oRz4HAavXf1BY9YEOs5+TCpw/ZTcd8yYULOqNcSURjBsy/OYPFOsZFyqHQ7MZcz6bW3F26VnoPBVUdfMyw2naOsNk/wB9jyaFm6ln7BKg0J4AM68HU1FFpaCsUSV5OZab7wejBIXqCHJp0v7tYy2CGmlRjM3nFuE6VJV0i5nEK0Rzyvm+q2EHusIyNqB+FrFBvWy2thcQ0Ym+vEd3gVQnrC6QXq/nUpsOdmmbrvOrvoH4Ej7TL6tDLqUUEpGoqJzI/hni/vERMqUziv5Jv4YMZ8Ds2+kQaHF1fyleXMmLuM1a8s4OVmtEcnmurQSqDVGFT9f2u+LWS9k/FeyYVGxyOl19aXQzlfK92Shpy4HDYP2cxb7Q4JSagnbAA+GTCpEE41ijGCTqMlRQ8PabqSrLJO3mW+nfX10LVGXSuC2suQ87OclKkhUqvPwYG7VOMPs66W708v4KNDXwm+Op8BrKiVNbJKXO0Ff2XQnVQ5dXwWBsEeZUODxkKm+Q7h1X6TSL1hhHAsPgnMLYOBFrgpoBnK1QsfyYAIFSAHGq3cXLfE/KUT92fc90TxBbLMZWgazaTmirSm0foycpywsAJKHF4NOUs0z5SFkVDayKACsUnuCevPPJeGzjnwo1ss0JjtpSwLhyNxYKkTkfl6ZzVpBjjcBHqzqGXYIqgFb4VAUbKOiCATr7/xBmONxr+nwTIaeemFuQwafBOZw6/w5g0ilVu1Txted+2l0HiWbj6TBH9wZBDfFN3MDRdvxC9CXCd/Lb7MZQe8f/B99RMJ7gLx05otCGvH58BUB/quSi2s+IBcjHPkKbf1R1NY8/333N6khnIXbXpjI6byNZx5W27EaFXoRz0eDRsuzeJksQolts6H8EzokgMb7eqx+3BjA31NKjrKINOIZ30PFccE1z73As761tCm+xla+3vpcGp0bN7TlpK8cJ73VCF56RxqJdDa9SStHcnIHR8I722xl5Lklu/1XoEipV+Ks4heZviiGtVmX7LWwSxrHS/2f0GVPaBBI9/3TsmA57p0XpIkjnc6jt2nDt8OFaSalc9ta309I9vKIIzknuJU1dqqJFxrhhJcHE5fnUhpOaRhP6Fb+/F8Xiv1Zl+2XESmPkv0gcygEtoH+ovTkDgb4bd2cHKe4LolbRKxmofoPPtbotuKsUuBj/zbaw12LFqvvdRRw8XkYnISYd2oQQLTQ+6Fb1jS4TsGaSPpkrFKeG/J7eLZnlfQdDlNrfd7/yfWkIPkasKjAsicZP2QhcFg0Lg4Ve61hloTWHP4QU62zOD9wRvwGJTAmybfNGblhvJqCLxgFPckn8znKb9hJ/YWkOIS85nShlLu9QPrmRfVz9teBeZoeWpMhV6aP0FyFV7vHdyF+Ts38fmRD1l2aqFwWGObj3iqoZHM4FJWV4gNoMauizkVU4YhO4Ym74lyt4NATyM5Tnj82I+qp93eMYTMyGheCFQHQi0+9gFjV3h4d79Y+JckzV/xVlXWzwpflvVZPj2yiP7x3+GryRKOrXKlcqG859X3ET+32hFHTnUbzHoVoJ2tggf84fFSeFtF65hBm6Drp1B+SKaqU7MjD8u02So2uqE9T5fBkOiOgk+r0RKjgwRjI9UO5TNddGkP7etGkXCnLzMf9dC1VSuFnIK+7jgPF17i9XueYeUhkYI/uFqOZyJM1ej1KnHP5c/lf70mbf6yNs9C5w+grTp4YGwBPFAC1TYlkMkZFc3l2Z8SfOcGpP49/nHPGgeIfA7q9q3ZTKceBk6UXaLbVyJgEyBCCyOtMMqjkmcBj14Nxcboq0RnxTHYMRoK/hBcBQULiE8uoDguBz+HWPNZVGbhwsjNBM0bJVDsa4M68/yZMI7vuIEa/QAAxt1yy1/MBBHAPOAwcPHqv/O4NvH4lc5Mnc/L/Hx+rgCmA3Dq/TBFy2BJnX2j4F9bE0PW4Tak1acJ+c4MHzmHL7schhYlyKQpagwjCuS9cndNgPC+hWvb8cWtn6LxfYqOVfPFE9OaQWOUqRubxEZ6n/gfuL3Dc+g0DvXYHChPgjuamontr1psjHIfD6KKKT4yfe6gQLGW1b5UrkucbH8SjSR+oXuqu7GgHIaWllDuUDaFtVrIses50AjtspFju2bsv5YL02jletOeSXDWa/0O6cZ3tRksu7oE+euV63exX2c6FdvpW2Fj2JF7sJi8WGnOvc5bQ/vxmO8zvOwRWWWqze25bIfJITnUNHlNheusLDf15etq+KCqmXOvz5bjvWbYK3DZoM08aP+K4uUbfANY1fkhlh1cyaeHvhHub6vnMv0KCrjt58ncbRIpau+IqsIxdR0/ZwRzsVxZG/RYohmiLaTtax/w5sPjxL2haCvfRdaysQGSJovDCwDzfGv5OAyk+maaZr/GyAwRSTMVL3eumcZjXR6jKimFr/XKfPs94DVgN/LzfhG5sX8RyAR2u924953lnd/hx8KOwnnrdBJRWtgbC/2s4v3dsXoX9/rDtEKI69QMAPH/EvPud/xPWjMjGP/av/b/vrndbk6fPk27du3QXl1VSxpLqAhezaYWcKK6CVAisuscwUT67GJ6+3n06vAEeFFp7a3aS0JZAnE5capi44n1pxjpB795qtRP6k8B8sMPwCixEMOmXlB78Rr15VWTdFbK3bAsXKJevx0ZA3XNJoWU0fq5VfgG1FGtU05s6X1bsMyzkJLIbIyITTN/WxH9+qSz44Z0jvqpTBA5arnVD4K1kFN5mfgAFZTk35k5WqaOPPa43EhKuEYTYrgO4uetD2Ooz+S7tlU88srbrKkGmK98X0s8H1RpOWVzsXLE+8LHnq+rIKupib5xfekdK2pGebaPRHLbOB1zlwphGlC6R9ad8U0Fq8rmGH+z/F8z9t3pl6i3B+CtJhFcvYbOLfPIKAtmv+tNvMtpNlM4Jgn6B+ahpwa4bkLP7aBv03H6hkJZ1feASKn2dghkOeBwc3N0Gd+Asw7SHhZ9bgfUXgJDEJiVTSCjXo+5Qqbp2VGfhvdcYKEnkQpbJRW+O1WD9PiS+5gX+y2OJvVA47ID+ubBAxGpeKsNxpt9mBsEKdo26pp9DXmQ+Q20eU7mzve2/TMg92eIGMKXOzvL6Mt/iE67AwP0WlKL5zLx5xf5bZRK4cgQyIrTL5Be3hNnqEohu/1L6OyVhNiLwBQouEvLT5Nrg6fyy1CVjv9zurLja+onqdHD8Segxb0Qdo1GY1RwNJdry/ELrmbJ4BPCYe2qJnDLlhiuND5J4Lti8XNW+/s5+pvcsFVr9pmcXbhxhfz93ahyWul2uKsYUlNVGl86M3PKoTARWkoq9DNBnaHnMll70dmgaPhpJA0jNw4noDIIa70y8T5fW8akFpepNV6AMhhjUBboY2sP8HppC7IyE6iuEJGugZXzGLVhFADGgSJctiHpbj459TlTfUF1RjV2olwk2X87DD8kN4yus0M2iXdLoLPWwBGvKc9Fbb+m79V8psJrXyn068SMqPdxLD/L5hqRalPSSDi7nKRLUhnVBrERfsDShWe/eYVayYwcdittQz3kJ4GluWYfQOEG1elmhaUqNboyw8P5eppcoO5aHIw3cciVzsXEl22lrjSQcO9moDmSVytl+o86a53QDKytSqdd0AUctR6iEtUD3OfL5XXwIb343P1HpqZZI2lYWmNlyrz3eTvCxW678potjnKGpRWyJ76QGoc4YXR9o8Cbvgvg8ZSe9N4zjl4JaygPOYc36VNFUyV3FYNZ8pClds45P8PJq1TG4/NE/dKC3yHza7kQ7jWB9OWNiWRVteO57SLVZk3IHdyy8z4AbrI68CaAXkISnxTYSNHDpHYJSqchiBV1k5jm8zOvRno1Aq9a8cPn4LfW0FaFYhHgzMuydm7GlxDqRQfnskPNOWgsAv82uE1Rf8V9kuThcKuz9K3yp0QSG87PRXxCsRs65MB6lfskI+RDbC4oNZ5QPa1fLm0AoKtDBHr4VexnmBW2NUD3MLFAaay9xJooeKhEHc19i78RT7ARAjuARs/jTzwBkkS/p5+mp8bDPQ434ciTAt+aTOx0OHh94UIenNSR/M+/59sv+1I8TPwtTQYtfQPyWBEBp6QVyMRZ16yiZC8r7hqDNag9yUHihLOv5GZbNFSqD2HKcaveXy4melFlAXKFWhJTw77+FpJblTLusU85ooWQFdr/ivLM1VjLT5FQ4NBhjBCT/rHtvsOsr2VvzmSE2VRJosQlT7yk16hrl75vyUNjhSM1pwGv67KVw75bwRQhx/XXARA05icoPmrCnVlGl+eVxbLWEZ35dZP8HGYmZQpobP+Em5m1zIeOe08RaloDXtFkQMZn3N7/LJf8HBAgroORLe9lt1PLaz2bgTWF9JDp9S1x4HELDcGeN51EWmgB8qke2+avSFTjrOeFMBeuOh/iLafxQzkBGtD+be5a+is/fHsLl1WoeTUaiQbfGgJ7nKTQX2xObfPvj0bv4NyxVNqp7bUtH4fosRDeDCNC7E3yf2omSfLkafUZSLpTsc7Xr3uSgRutvNU3mMxgC95QJkdqFe+9NJ+EjG94M3Sm10UZ+bjCTCtTI1kNQdzusWPg2m9S59eGWcUwL6yco63eAJS6UJJWzxPL++HMiKXFUvVi8N+aMRR3h9fJqAolye3+K+cFaKXfy/kGH1rm1vFNy86Kw/K0QSw50oKOe0JJ94sjyKJcByVLLF80lfKMB8JUmuj1ZfvwA36rhwa/AME/y3WcYeHQ2Nx6kfMznHgaBvwBXo1GAHxaXJ048cq3jcGcKB7KUddQACF+1pkD8c+OJaIogmqLCOxdlH2Wk4ZiUkP9uTnWK4+ruUB+ErxSAfPFHjrUZvBtX3kfMjr+ZrSqGZPMEbTMBj8NfDvhaUX0o9NBatAhnux1G3s83wMJimP99bnc2u4LGhx+aHgUUD73paXw/k33kl3VFpmv5DqzlfJ2KHxVDa82JzWYcp/8n5qdmAsXPwZjMHQXJ5y3Td9K0K+hNEaJFOJPRt/LQTukZsOBXtWAHPC+9847zJ0zh74SvOqsJQIoSk/n24ULeXnBAl5/800evmsiF6vk7yEpRMzvXAN3MHkq2F1mvlNLyVPul6fXotWylauW1swUMJDhgGEWiHRVKV536vw5nRlBTm0cvcIeobrmjb/ds+4GXkYG5vzdxPpBYIfdTquuqZwvKycedV2YG0ssPOvXQJt2KqBLYGUt5DogMDBMzNPslXI82Fgo5yrX5Su3GLOocbVjd340faJFgNXanFBmfjyTDvf/Co4E0F+rUUiSRE5cDiumruBXu5njvMDd99zDywsW/EfXvdtlZ06vUDpGz0UrvYJ3GbnanMzcex4lNwEa9cp8yO12s0XvwXnHUran1sDxSwrK2TFJd/NzbDxHAzyke3wUqqsBpgCipE7MKjlOUG003txFO5wB1Fy0cwdrqQ+IRHi2Em+T/2vGlp1ciFVfzeopRqi9IFMi/mnHn+ZogoHjNjspiSp0xx4PqzqPprh4P9PaKGPkaPcVVkbCwyWwoS4A75nE0a2Ok/zEcVqEFFGrFSeB99X04Te7DLJye01fazTwbc4gntJcBTyoUOSumybvCRub1oGwY/6N1efITbMui4RcGUCvh06aAIb7V9HSqARUOPU+nLNDYF0v2md/QaA3PituCo8d+IomD3i0IvOLQaellxmWRsCS2h+B6yhItSZ261vxRYncdH1Ipc5L2QE53gvqLMdP3gDzk/Mg/R0ZNH+95a8nOuNRGhw7uVjeXaiTxVa8wLjAQnaV9+BQjQelui7c62fjaF0gZy+3pFvwLIXP1+BLx8bZrK5cSnFxX4QVvCGf4Rb4tBrePDWbu/qJGriD8z0ciIWm0/MgWSUXS3tEnu5OvlPxcqV5FKtyLnC5+yWy9mn+es6LgLnALpp/7rsD65zQew2cC+jInd7bfOUWngyE1yrg93oX3pw2LbU2nAZYVw9PqzAq/d9iav2O/0n7d7LvX/s/yqw00NMEX1bD2gax2N86sZCOsd8zOGkpbrtIHTnJc5nRt21k8K0GVYRveKVcnP+osZkCY+s5sli0imYOIDf6AJKVGly+ko02Bpidk0pJkEjhFRvYm5qgROaf2EpTgHJEW6P34bY7HuWuZ2wsefKAcOzlsLFoL8HAfDisUSno+rbg4yoYYYV44/8DPnzfZOj7C8RNkWkir7MRho18GgavBYOfyysh82/F5EIoc0MvtY/1S+GVajM3+4AhV0TKbg4ax/p6WG88hH+DiJIpbvksI/Oh//5f1M87eiy0egrOvwnV6jouquZs5JWBg/EzlrE313u7Bp+G4zx6037WPPwZ6xteF/z+1PNYACxLPQh2L9SzzsIrQffzUjns0IqFOIAnAuGjMBipU8tSgcufyfRGB+4UfY1F8pj+zjHCFEW72q+Y9PV83n9pPhk6Menfzl00uOGuqJNIGi/6L1sFcwztWH/yAz7c95P4ufW5jLfKSEU1KjfJNwV6/8DEdreDTaS/wxwB3b+GWJHGArgmmF1+iDU/rPrPJwaa7Gw+KzeGRjSoBBJ6H24zvMlL0SO5wFrB/UZBEUt2Pgm/twOX2GB6uCmNfuIw1jXr9A5MrgajCjUNyMFy10WQpixf+rWQW8g15f4UOMXC6hVPNAS68K9yEpstNpB6J39Pz9F7ke74UZUO4s60W1g8NkmB9v/LnI2M95GbtyUqNEN/2sxi+NShst6Yw2Xdo62DoGSnwjVHm06HSLnJl1yrpMi1uWzo6kt49P1HGbxlMAaD8tzM7jr69z5DTo9DbB8pTjC3NxbRqqv8vLU1iWjtjhEdeagUQjJB4zVxCMhTRYEdwTdNoEMBKK2XGx1HC0X0e3v/LLY0QHIWguaZhITklkjO0PB0rjd0ACaTyfj2uSwO70tamjitcNbcmjnVTnYbo1SfrQYPfF0NB5t7JIq2yuvg4B3q/oRb5UafRdnYvf6+aURci76qT+C7Xru56761BHhTPwJ2o4P58+fz9lNvC02zX6QUhn/dnZufnU9KuvokzisV8tRifq26nsnfmrNe1nEpFpslEjp+M1UxsbGWxkBlkhrYmM2SCJj99V288/1DBJgCFH69o4apPjJ918JokeJZd/oFesWuYY31WYkDzE0AAQAASURBVNwRYtmoDRVEaCFbZYgYgOgxEDEMokYJU7GArAHac5lMOehl27NuI87/LJNbi1MS+uuSh1zXYcFv1GvJdcK2RvAYvOijDf7cX7yHLjlwsxrHS0O+3My74V1Z60LNBm2WaffUqKdtJbChE+wYCSefVbhuNx7lgYHnePeuLymLFZsOZxuf5vNKueop0J/U5/BDdDVPB0K8CmxRyviSA7HQzoCM5PUye3AvphfBfRVWrAEiRaLHryVHm+DjMIhwiVOH5K5CctuQhu75q5jw+OzZZOflETkugPkmX27Hl7dSkuk/bx7ZeXk8Pns2BHdhzslwrpxNInuZWNwMLf6YTdFQ6QK3928F6HeNJfHABNZsu1OemPeyJYZjDLTA3xD7wqF74YyK7gdA3lo4+bys23qdxbSex30rfqO/IYEvEgP+ov76T6wY8DNJLLh1PksffQx/X/HsEv0PMyDhewxaFa56j5ubQyPpaoQUf7EpDLAtUC5k+ZdsEZ3GEAgbINMQetFHazRQFDeFCz1uJzpiuMKn1+ohWm4WvLDiUcItymaHUWckyr8TBrONjmPFCThiJ5DdT+Kd0b+SHaaCarZE09dqoGvOJ6rXhCFApsRcmygzCHjZ9YCE68GNWlcjdzeEguRhV+0dAvOA3mSmxmUhKhOm5or3/syAdNq9+S7P9N7K8Toxji31acWubgfZ3+mYOo2Wf2uIubFZLWNAXsNV6HPRmqDjm6Dzkf/mOnOHtcMT6KLe1IikUuT7zP8gh2PBg1tEzuvMzC42MSQf7q6sELR+7JZ4vqiBNKOdpEAx/kDSYNnTlaCLKXQOaEY/9+/MGISn5ZPUW0X95zVSDkMvynSq3lOvZ0vOsqPzAi5N2MuUwI7EWJWT8rqrtPmJWfDOYZH+fE/CE0iXYEIhFGrTBP8QzxV2NkBSVjPnHdRJnoLw1rH600J7wlS7KjixmQEZAAyuQmLMqYTGV/OUWyxCm/VmusSW88TEg3Tx8WIOMIZQGnMLzwVBSbLK/ac1caA6kcevdODTbJH2GuDmNq/w7rAuxPur0HhKEp0jO1PjBr2X5EGk7Tt6xf3Ma3t+okIj6mClmn9hattXuLPT00iIgUCc/xn8jOXk1oix4p9a0nf5Q3+TyjpYcVR10usvsyZC5HAYrZ4PB1Xsg+DumOPFiXGPR/cXaZ5eJ8cT18spbPkbOYUPv1zFGy1rOHn/Vp59RLyuUkclv3SysL6LRJVHZdomuAvETZL1OtWs/LAsA6GmH4assbUyEtq7vN7bARN+ncCgrW3YX/oQVovlb/esCOB1YCz844T+62++id5fz6ZoeLO59M+SyPhCcEQMVfW7gd1NkO1WiZEjBkPvlVB5HCpPKFxf2lvzZZujGJ/+gtpUMYdr8GtA2/k0ATV/UFrhdS/UXibvxpuZtnIabVfJtJERERGMHDmS4f9w3cOBlNRhPN1vHpNav4FGI9bo4n1bgEfi/lL4RS9OmkWci2DcNzM4mBmPw6QEklRfctGUG0xW7EGqHcqcxK/mDJ+GxpGgAzUw52/6thS9NovKb9uQFvj303dqVlyfyPas29mdfbOiOQqAvZKO/hGE33iauD7iJBp4uKnmN+43VxBQr2QHKDWlMbEQfq2HQpVcf60hkZ8TLnJrXA2RKtS9IVIaU33grRDQeq0nGg1IaPHXXI2BXSKt+19nqP0vdQwzvoYt/WTAUfgApa+phDamL5nu7+C1EPDzKFuYOo+TEC0M9q1gRPLnotZr5FBZGzcKXg4XZUp6lc1meQSMKYDLNhHUeX2sozbUQWhvWZKo4qgQQwAQ3A2S7gCtsgloDx3I7wH3sNu8gTrjRaHZVxlwK5vrJWg0sa9G/D4n5fux2L+AKyPbExCvzGd8JTfPBlnJ0ZTzgHuNGJ8kTSc4E54LgnuMKmwJyDWAeWXwZn4zRanO78s5pJdlWu/modoc/P2h242T/mIg+RIYyN83+Lnq76WFunOi3nB5/koeCIBltXC0SaWu3uMr4ooiqff8DeXqv/b/uv3b7PvX/o+ykIYLbI2Rdd5+qRObdQ2XBzI17XN8L8O66gbBn1RbRvbJVOLz1GkTHsot5cYC2HhlR/MnkTRTDjz/zq5H/QBdPec5Ew/ZThdHnGLnq9LnJobm+LAosScb81cqnR4PJoOTG9LyCDKJgbBGo+PPErDqYilp6N/nPc5F3UZcqOoM3D9bSHfosxJClZODQ7WbuM8f5gZBoFaJ+teZI/i5Dh4PgO+agctZdXoeCwRjmRh8aSUt5S7IlXxligUv+7TSxR8NUNmkUmgDWdfIHAVFW8Cu0ji78p1MybpPyV+NpCEx8AQhZvUNsyZ5PiEZEKMDqyRyw48peJcHAmB6EVxyeX2upGF1zkFerIDJ51QKBsDIfBnN52lusq/nMjnYVCnuoPeTG9EVR2XtnetdWg1BWuhjhkCt+GwYG/MYYIF3Q8Hp8UqaGgu4v+uDBFnyya5WuYdKdrA6CtZFQbxThdJSa4Ca8zKytFGleG8IAJ9EyF+vSqFByn1wiwcmV1Nd2/BfTQwUNMkJR6SfSnIMROibGGyGCp0oHuzyuPil1sWL5VDeJBZOq5qqWBQKbzTXE9OZ4fzbcOkzdf+2wTIy2t8rAb5uCuuVP0YJh71U0cjprjuw+tcR0CRS+t1y4S3c4zYzZ9A51TWh2hZBgKmYSB8VbQNnHYvC4P2WPbiviwpC2NnIb1HQ2gAnm9OVDGgPrZ4WCkBaCdrcvInaO36gOFBJi6px2bm1JJ7AqkD67umLQaMswBdGjCGxtooPH/ydF/uKDYuvfcbw0OifOfVINqlDxAKOu+YCU3wgVNvMOumokwshg7cKWkAAq/2y8KTISRVO5b20wPMpQ/Mh0wEaLxBJUFM2C4725c4XX6D9LmWxGCBGauDIgvtofcrDDd3EBo/b4+a9Khh/4QJVKjqfr4b7yFPbgeq6kGR9J68F7mYonHsthy4fCS8PdJ3ieBx8HAo3+YolkLl+31K/fQrhx305pvOavK67wuYYN2+FQD+zmHBpJA1djsiJ/mC7enllqAWeDADr/xMO+4Z8eWJWRUvzaGwFc4NgSwNIVmVxvtonjVs3p1KaEYuuRovVoEz29HWXWXEV9J7dpPLQt3qaZxrjGX9iIccqRETqC54DLAmX6bxVLaAtDNoIA34Dg0piZAiU0ccqCeyXx96nsimCQQlLBF9IxRf8HgVfhsFgi0jxMrRmGa9fvRyLQbwHdZg4aoNdakuoo1qm5bNXXqNs87bADjJQKkgsZKP3h7YvQGhfebr2OmutLcK84kbuX3S/KmjhnPMhwM2maDC7vRrOzjpG+9p5IwT+UJMqdlTT3QSn4mGY/YR4ymHd6Nv/C54f/ql4LGD1SyI/8iZyjQno9Cr0pZ3ehNFnhSm4iIgIvppQy3T3w4yMm8RnW5cx77nniLjaHNaZ/fi91RnOtzzP3t7i2q7XGzlpg0H5sKRGTK6P+/en3g2z7TtV99pNzhDmlcGNhaqXBdk/yGAHNZpwgL1T4ewr8oTIdeZoNBK78QCBm7ow3FSqoP76J/vWbGZgH/n67dU+oiYUsLX8DfblTqB37M+i0+3kh4BCDsVBav0J1c/4wR7AoDzYrhObV+gs0P1L6Lta2Iu72jtzI4+QdmAZ7jqvB8DtYMhTF4lKzqd1VCZBRq84te4KLTsvhle+ZGUrLwpQgMhh5K6axfDnnqdriHrBl6qTUCjSnQHyNJ9fGsTfAvoAwa3JXslXYfBIAGiu0+vRmiN4fEN38EgEnTlCuU1J+edoukKH/D3Mmz+frn+Imn0tNdW0zo2j85HOxGlFivJn+z7Lmb5n2Nl/p6pcAs56uTjvrfXzp2X/CD/6wFqV6WyA1k/BzbUCC8QbnOHr7hsYs+omZl0SWVC2ZLZi5WNPMPHSCXydKsws15l3jPDnHmbMgHuuiN8JgHbOl9y24AsSfNWpqf/JNHsmkpQtAiK0Wg1JxiZm+UOAWxkf+zVc4fNWVXTa3QbrrmNIHmWDVFO2jysJcIsvVPjuEt/7Omq4EKNIR3yXbhyjC6CoOSKNoM7yBPrRR2VKT2/LXw/bhkOVFzimZA+rJutZN01iwQAxLjKcn8fcfqu5oe05WphFKtlbshaQ+scQNv4wVNwrzRHkpz7Fc2Xwe5PKGmSJZkBhNe87T1IcIFKLAQxO+paU4KOMa/me6HQ2cpOuhIf8wWpTFqND7L/TP34lh/JvpEkrguHynINYd+FhPjj4FR5JPDe9xkZFYwRXqlQYTq7TQJsRpBJ7n5ovg2Zyf1UHVabMgoF/gEm9++TadxvVLg8lYeLv8XPpQZbWwHgrGA3a/0pO4Zk5TxMbtJNf+uwmpFYE9p7L3sTcQLjVFxwOleI7yCCTIhWgBkDOT3DoHpniXMW2XN3/T+sTFa+3dF+7qUefqCA4Nha1Vs319jgyY0UfYIBGw3fAFmA5MMxkop9WyzMLF/L47Nn00FaT75TB6Wo23VTBPX7gcqs/XEdi4Y8odbpXQNZwHrwDwpQaXZelECr/6Evja/dz2SbKT7wfX4m1JJQ1f3QVn+uGPAKzltJpwFE6d5Dzw6KiIjZs2MBMZL2+4aC87quvzwQuX9pEQ2M526/chkalOXtDoJb5L71Al6fmc1krMsfcnJhOaJcz3FVWgitFOf30XF08NSFldPG3YfGuUVQc5Ub/NeyKgbtDxBqEJEk0WOS6x7JylX2l4phcn0h/D2rEvdqgbeRowUje3LdSYE2i+2I047NpG6aiBSh/ukw122KW3By7zuyGMH65WlpIVAFhn3OGk/7e7Txx32w2u0UJoNvCfmdFpEwDqpGU6AmNBoYFFFCVDFmJyDUYLxu7wsPYFR5KNQOaOfdmLGoE3PC+zIbgbXWZ9Nbdgw0XnXMgB2WMkFB3gtIkWJp0kQe73YfHo96EjNFBqEr+V6lpyao6eRq9TgpQOh21LKhbgScFfoxoptlniYFhB2SpG7U8K/5mWTLhkHJoY1tFPqMPL+a3Hq/zzoiJQpzaEDCan0PyONjtIJJFzAnGhH9J36W3MrFmCQFNW5VOeyWTY15mjFUEDP9pGknWvguQ1NGiOuDNSvil4W+gfDtvhKNKUKbFYGaKD1Qlw/jhZp5ZuJB+Wi0fSZLA0tWczXJCSM4RYa067teXFlnItKtG9dX1dnMdFUlgVcln/rX/Gfu32fev/R9ljdZkHi6BQ03q9HR/NBl5tQLqPeBR4bf+2jCC9JPJHCruICA3AYKiBlHohOe7qgeN5K2DUy/I9HRqNrlaRrG7lZtXVMwUFmWM5Yz2MosvviwcVt1YR6z/KZ4OaSTQ5VUkrL0MK/X8UpDOrjYfC8daHJWMscLeGHgpSmXjdTXRLrIbrbu8LCKS/hOruQi7bpKDbS/b5N5Pj0sxJGdBU4AyAf4ziXytAuapDanZq/k0oJLXKuDtBjHJTKs7TowO2l4s44qKps6likt8Hvb3Ate0fAymua/p811vVSfkf7O8WPi1Rm79pZz4gNM82uMe4TCD0YpFI3NQT/IR0dpZgX15pkxGsni8CzAeD4kaOxF/M6XdpsMTTLB1pn13cWoQkBvJk6thmKi7h8Ef2v//2HvrMKnLvu//NT3b3b3L0kunIC0NAkqLgd1it4hio9iBha1YCIigdHfnsgsbbHfv5O+Pc0FmzvOLet/P77qf47l9H8d1cB17OjPfmfnOGZ/PO+YJxl9HT5WQLfkV5i6ZxMyvpzExXC74/nCgPzOKhJ2E1btI55/CvMw3ec38BQfTFOq7yIEsrIQeVkh3ykwsXA5xIOn3raQOPY/itXDoKWGpdRH8U8WAy9fFgnsXMKe9orDkdnM2VYR6d/CXDwjTShczKwDmVUCD4kD2bNv+3BoM40M0flcN+UL1cuZL9XiHxwSDzAvbs7dzvLU4IXY+eZ80rtfpOdD2GBHzvqAxI1ga7z/QyuL3xnPtW6OV8+Qx/cNYjE1MbL9QviZzMGTMpWvddtLrVA1pFwN9oI1Zo2lWvA7WDIXAtqIQeQEW0JleFLIg6RhVAZ4NNYO7iWf6nGH0dcu56YW30XvZJA5NGUqvEytYXWPgcIPckNAbjNQ1WKiJKSc0Tf6udIWr+CZGMOOUG5v8n2BpsmZBYZtdvOb9IUjze79evoQb4IrEXgRZPRtIA4NC6O0Wv6exyTJz/g06U2d0YDnULOx5vTCufh07WmpGqsJpt4BAOlvgqgwNC5r2D4vcPi/FynmcWgRbZkgqoAifEHx1cHswjPeROwMhBz5jxMp2XPXFVZKFM84mOmKk/o653PL9FOk+CbKVUDrjR3ymrqbHjBmosDoOXo6AYYmyhfNfwhop7KfbPSgP6RKwtdxa3tdlN4dyoM5ETWAD25Mls0AIbM33Jb0ZmA+/OGXSw7pmCy/k5yifG+ATXQZdrXDkHzpp/108ue53HlmzQfq7T8MuRvkJRUAPq1x5Smw6yEOh4GwFlwV6KeiaSlgbX8h9wRovGtiW4CxovfwZpn4lFwkBkR+kwbrHFACdnobLNkLCBI+heY5J7DodSVRJFAPrZYWFwegkwWynlwX05ylPLQhqT8CxGO4qgfdVxbZ295NwWqhiz3gXDIBASyCtanczMuspGkvltcPH5MP4UT+QcMVp9AobI3zjofIgHH1RGnrWfypH2x8lKy1LboTjxODXTPpt39JuwH7pse7Wd9MlF/Ic4NLL3ddDwUMYeRZmFcHxuippvLbdozxXCWkhGo2Usm3CGiy0u3q8z6dCeRo/wePPdZylMi2Ugv0RvPzpt9xw442ss9s1lQDncM7yrMuNd/Drg6/y+l2vU94g7wHuvAMuSfiJKUO9DYUBnYHVzkjy7VBvVW8I7RhY1wjVZo1GTECauP8snk38E80VxKQdo03vwxirPHNNC2vyCc9dRkFWHDt1M9F7Ncrt5bu5Pf0FTpQ5WV8rF7Eb7Y1sOy0yZzKi5bWB019CzjfQX507RvFa2DoDIgcIcpsXCvfPY3YQvB4BhoYz5/8eH5yMwyCyPasiSzhble3xuJIjzzA9QKz94XZ5HvvIdxY/LpjFuOXjGOmVrQvA1qu45cWbeXLVePUeYd9DopB2aK76fVW3FH0i5IYdLidkvqtcpw0GPXX+daR0yWJsnEzo3FLUCVd1AO6tYVR4fZc4GvkhoY7TyfBVtLzWBlVs53iSyD53R6mJejPialgRXoDNT0PldjE0FqE7u5TAenlf72z+npkxB0Smqv2Mx5jFUcmUAHB2OUl+eBdces+9u05vodQJk/whJmW+9NwhDZk8FQpvB4cxPlUufDboLQQbIO1iUuCaTOHi4LLLY7WZULQaGr3OBZZQdheModlhxeGSmwK6+PEsX381G34cTHm47KRT7NeWqrQznLQ0UW6S7/1teduYXwk3FKn9R43GCm4PgsT4j5Xj9X038W3mK8T2k/fnOGp51JrHm5GQnvO+x1BO+EJmLz2Dw2VCxVOqcmfwwd43+CN7NkaT/B9kVXbnmp8L+e2UgmjndnG/dRJjC+Bk5AR5vO3dwqZt00TxuatQshFKNiuHpuXVMuHwTu5aKVti2pr1PBkKH0SCj1XPh4sWiTgF9auch4hTMBG/r5aNjVAZKDcx2579nOfD4YtoMLgUhf99D8LGy2GnOuf+/BlKwwbu4xp4uhzKjZ5NzlYJHYloK+7LtgP3UZaXxxa01WvncK6acDjAh8cIZxYBvN62radCH3g6uS2tzTDkEvVZ/kFrIR9EQUG2wi0HqHbBCD94ykdBoG0qg/Ld4J8MXplnRr2B8rBychJzMBjleyzW4CJx7AaWD16DzttBIrwPG1nCvvXdiTeJc9S573ohIq9rALAQuA14HRjY8veFwKV6Ezd8fjlLT9yDXmEbHGLJB7f4u8EgX5vTDqW7OxJbECvZo5+uzCKwLJyrdP74Oz0/E1fyTO4tnE+CCQYqss0n2A+R+spLzJ07F2sHBcm1dIvYq+29F87KxN+bu9/BN1eG4GPUyMJ02YXjQZ7CaUqnE2ewXu9J1vzGls9gSTSsSpD3PeHHfUnLTiOwNpBIs7xnCzKL80b6GTB6KdH0ehgYJKola8qHeNiTe+MfOweG9xG2kCs7w06vuSqwLRv0v/FubRN7m6FR79kV69TqCjYU3MJNJS6G5UOlzeszPbaAYzHRDMzx4d1Nv0sv3Zy0kNlZIvJCOhPr9Nh0YrEa6ac+MwPglyj2uAoXH0C4PZg955NzNc3l9WBQnOX1eqhOtvPYTSt5LlFWMA+KupS6XnWcMpnYUe5Z53VaIrl+81vEGGBVnGLNaiigrxW65MK8WoVdNlCSqkFsPIdNk8W9Xe+5N0qoWciTobCoGrIczecdSHR+/v+IWO9W8Yl9gnC6YWciTPWXxSo7jnxKiq6WQzbQ/1fq0//i/wj+bfb9i/8x6PV6MjIyPOyZ6s2RfFQDb0TC9GD5wLVJF8frVdDDAiaXXAgpNVRSHlLJmsojykNoZz9/diXCCLfcDAGgYIUo3m/VKKzufxjWXibY0RfAHdKD97LHsi0BHrLulx62M+s5vkrO5flwCHd6sbFNAXxdC98VnGTgpwOlx8ZVbWVZLFziA+lmxcLWWAS/94NMDSugv4KtUhTCN14OlZ5s2OLGdhxqMpNtx4N1CGCoOkBJCsQY4SvV/sjtZIifAV8d1OrkBbdt7TaebFlrT1XIm7NnbKu4KUgodZTI+kQo9wpWqqlxXV6EK8phkjrjxWxowmqWDx5WVxFpJnikDNZUy8zNzKgJvNAiNpRYRS473/sepjAV3tFw+nmlz03snvEdk9ppZJbY68CmQRU8h6SpEO/p2R3kbyZ2eSdO7G1LvcLuNcjRi69r4a1q8PVmqBv9mOe+j+9TT/FDp5/l1/NLpNPwZXyX+Chteyny6RryYc1gqNglirveqM+FYwsg+Sqh8PPGyXdgeTvIXcLl06b+fcWA1Urf3tAhspZmq2IncsF94YyTbRf0CDYVqBliVzgO0mSOJGGy4jAGQtUH0EO2TwLE97T9Otjnae1bk7uetieDiO13iK7DZQb6CNMpJqwcxfIP+rOkSrb2TTHk0nZvN9ru6IXZIP+2rrougG32N+kxSbapRW+C6GEQ0g0MikKe0Q//LBjlCy+aFQ0kUzCE91UeLM59hq3DWtMrrpfX8wYww1bFdem7iW0qRe9l8aWrP82wmm0cvP0J8l+VrzvGWcGdH99A/0e6sWT/N9L4dmcIdrdQOZgURBCC2kNIV8j9TslQf7kpirAsSD6NdA9HNmRSkALfmuVCtH/qVfR/+2VmP/UR46d+K43rdDqcrUWxdfseuSjQOyqDXlZ4P1LdQFpk6MHGRkirURTBQZADjr8qcnVU2HmTUGa5PdmC3S75nqnrF9PuDDzXIDNW27YX911EWTiR3krhoHYcTBO/iYP1RunAlVS7my/655I1ykZyT8/cvPPQm8AajcGiYF7+FczBMDEfusqNlm/Kj3HcBrZWEFXr2UDqHtMdf/9nWDpdT1bqSWwK25srI3fwYSS0biMfyE5XneamQKhOheBamQjyhz6FG4vhjSqN6z71gVizfu2i3RzTwA3d7mF0q3cIjJLzdirSF9F20wNEZsNntfIe4pdW73N5AfxaHYbL5FUwc7sJMbp4JQJykhUvrNOTYISTydCnXuMe/CUNlgRJjFIVLtz3GfUGznQUqo6GSnnN6mYZRbwRgrNFjpI3jEYdb1bDwir1a+U74PoS2K2TPzNyf2BwwQdENJ2mvlnhHlC+G36KEwolFew1IvdD0dBw+MSQOmYT1/XMlh9XtpWiWCMzsttyVbVcMPD1hbYmmBcKyUaZWKPX6TlgExa4bsXhOcQnhGn+MCtEYRML0Pk5uLJCuCKokDRFKE8jPRsxxw4/xrx7HqOy21mybLFER0fzwosvnrcEUuFCy7OrSh9lR+cabm9VSU2zfO9Hh5bDsE2EjZRzjtEbGJFdQsIZ2NmsPrY+WrWYH2LgWJnMbqexEJYEi/27F16o788XBxM4saMjB+o83R4qmuv55tXpAOzzg2avQrUtpAfjDwWT8vxdTPtaztY1nljIBzf+zPVPL8Jc/at8XX6JQu1qVdyfIFj1yVdBgKzeAHjJnsbIs9A/D9zBF+TBul34RtjA4KBXXCWWRs/CU01QZ9ZdspnbXnybm67bKz3vOxNextRswjegng7psoIZ/1Rw6wisCFUX20K6iH+1crYy5sJ0J/T7Wh5zO0Vm+sZJgkRwAXoWn+KhkjT2ddRT3PZR6aGP3vwWm8ZksvemT9mHlzrP7WSkn51kE0z2l9danc6AzS0U5yPbqOfmRrcbuxsqGzXcRi4KN26/ZFwd50qWxKMN9zItAIblQ3WA53VXBfei7bI2VG7qxPepb3Ky0mtOCe9FrzyY6Ac3Rsnn2tT8L5kbBrdFlJMQckYanxSdzto42KsV+VuwSqj2hq6Rmg6AsAlvd79s9RbUnvmbfubKJY08s1EurusSJ7JSZ+BUq3wOhsv7uR1x17O/wofY7FhWtmSvnkdDATH7buOqAGEPL8FWxTuRwoJ5QqDa8SA9I4qpT91Hh8GKhrMpmGH58EkNHPHr4jlmiaB9xGa+viKEaPdq+aEXbMfNClfK6Gj4aHwSj/S/Qh6sz+WVph9ZHgv1FkXhPnqYIHr2/Rz8FWSOs7/CHwOFOluBpXWCzNa2TJ6LMqzvMtAXHioHX6uBpV999Q/iFJo4usvF9ka4bZu8XhaFDWFxDUwpRG1f7tNSwVbs5wAIaiecX3q9rxx+tiGKuRVwfTdPErchtDVfdbyc+Y/OZ7a/kYaGBq7nb9h0IvL7Asx6ruMObuE+dh096qHQBwjtsQDXZZu5o48y0Z1bW0oPPvWK5hOQlfEqOw3JtGut0NdUHYANY+DsCskP9ynTVuab/UnKTeLSJtnyz9Xxd9b8MIiA2gB5fjZYuefgfA51PMSDbvHYC7/raOAxYBdwsuXfx/gznXq2o4l9R45S0xyBQS8T6usDMyj7ah+zPnyGW5s9Fet6vZ6tbQazcfhqxmfkifv1AvSJL6Jh8jIWRmXRaPW067YZfHmt7jEsp+DtjfLvrr2riKn+ImokI11BSkicAoN+FQr/ONk2uMIwgDpbMDf3uFPUxC5E8QZBmjzyrLJReDEkNe+nPBWa3fBxdbA0fl1MMSmdTlFzuYl+gbIrwY+lr6DLFLEbJqPnhKLXwzeH53LTspP8cnKVsK70wk3d7+Tly/oSGyBH5fw13MIFxzumxBxMpXUErQ7s49Izi+kR5mVnHNqdb8+8xZe1sKYRdF7NQFx2AlwBDD16lOAGmXTWs7uRmQFOatOgnd5LDWb0Y3HMPegy4d7AG9Rko/xl8K2PqBGqvKRzvhONvs6ekQix5evJT4EPa+BU7HTpYVElT7OtVT4dzBBglPcAfRoG0SPkGPVrO3C01HPdqbI38nHUHbiB9hbFNRWvYVUc9LWiqSDND+rNUF9YlKjRoitcKeqCA37y+HNg80aSTDpuKoF4f0FyjY6OJjYm7h8R60sUjVOrrolEE4TowUmVNB5b8B23BMETjo6kpVwkk/X/caj6Hf/R1/8fedV/8S9aYLN5Hp6bHE0YddDTAnEmuYOk1+mZ5C9YBGENciElzlmHj96Nzq1me0yxi4PtaYuCqQ3Cbsrop21Xldli+eR1wBCsGTcnbFCObEnZyucsrc3weBlk67wOoT7RzCiCCpew0PCGI7w/r9qSud93OnR/Vf4PLKGCZX7sFcEG+6cI7w19PxP/v96TmdG1yxnC/LNJMILBq1isN/pxtrI35U4wOBUbeEsoj4bewVvVEK6XC6c7Yq7mkXKYGwoWu8x4cqJnUTX0VNQaAMF4C+slwpTtim6jTi8+G8UBtVfcL3yy72WWly+Wxnxz3mFdPCypg3WFcj5MfGA89wWLwqhvs1cTSGdglVXY112ijqqC5W1EcbRKQ9K+fhR8HyyyuLxhq4Q/BsPx16Uh/8YtOMYf5XCX/VSnyu/5zmt8mJn7HaMy12G1yM0Qp87Gjy3WCSoMSR/LlP7z6ZAwWB40B0GbOWqmNoDOIJotYb2VIdKUbhG2pVumccNNN/19xYDdxuej4YlQ6KIRaN+mojW6TLi812xp7OfYW9nbBBviQaewQdR1fRlrv88lu7/ziBsLnZ/XLMaht4j/JsCTJWbwNWAJrKNgSwZTOsrfZQ9DPn4n0ul4pCPGankum1j1Mmb/BvJbFxPjL1O9ghv/oG/XUiLaKsLEgZGrniH5RDlbXBphE8CeZjiLwuc/tKtQ1J58Eyo8CRkvtO3Pqd6DOZiko0uzl1WK3szKBigsDCUtdypmbwuYwt95Zupz5A3czO/D98vX3LAN3zKhyoiuklnNlaYwBubDtEIwGxX3WGh3iBoiVJh2uZn+6vBXqXBBmc5PslHsX/I1Jh3o28kqTMxBGBPaEjH2CozthkvDae4a/K5fgs+bc6k0yo0te4832dwI4zRYin2jO3F1ICS7NBSxbhd0fkHKhTwPvxTx2zN5sgUNBpH5dNwORW75e349cBD5D27l4fdeZnKYzDTs1dfM/Efn88Lkr6QDV2Fgd2YVQaFbI4MFRM7PpELwPgj+N7GueQG5DjGP2Yye1x1avZc1Q6/g1oq1pNgeobrJ6z4wh/BMOVh0MNgkk0QGFHzC+1GwrQlc3qHvwOyus/mwBh7XEi8bWz7nqgPKHLmLoU/cz4xMf49Qi8Lq22DA4dZT6oSjWfK9bzYa+aUeZp1uhd3qNVf5RDGmri3PV8DvKkMDZzOpJvi6FvY5NBa1pJbion+yPGarFJl9X+nOq+DO7ftS9WVUdz/IiklNdOz9kvTQcEMxqS3nS7O3SsJRz1D/Zm1FSuVBZgZAkF7YNXvDrjexvREuzQNncFf58QYLNBaIRrpKjb7rDqjYDcNk67xYnY3E1YPJeekGeS9qjeaLPZ35buE0Vn8tFxT05dt4NxKeCIP2QfI9OLzoUz5sqQOrCh2dzi7m6xiYadAgtBl94dfOsHmKelwDZeZYFulqyZm+iOxxQo075777zlsCDTXpLmp55nbD3nXdmJ6TrrZeOvkO/HGpaMxdBA12teuGzS3UTSPrZOUrerOY84++KK1Zep2eLW2O8vuw38kO8Cyy6PR6dvYUDe4a83xKGzyvzeAbw2+WaqIzMsmYvFZ+WWc9Pv6N7Io+S4lqLY28VNh/2avUbza4o9g/nPkcmkql4QKdP6saYEsT6C8kTdgqef+yNIJjyjkVWSw1+OuDujC/qZnndHpO2eQcWX3NPsa/24ZBX9xF1EDZdvIHY2earE3sNtWpi22tbhDF+VjZlgwQ66tOrybp6Y3CqcNRK+yaL0BkeQPlywZTmB3AaZfMvA8+9jSXrkjHtLU75Q4vG2aTP/f6PIkuE8ynZHJXdUgvuubCvSEwxaCwZEXsLdY2wK5cxT32V/CJgfHZNKfLpJztzju5t1QURl0+nnsEnU5Po08jZ+PO0ujTqKlkmFgIbxbLtnlnYydzUzEMrEqDaNn2enL+K7Q2w+s1GhOprVzEBmgREK2R0PVlNXHsL2Bq2E2rU/HkVf0sjRn1RirGr8L62NsEOr0sK50NjPETttWvqLaw9lquDoQ9TfBehcaaVZetnfVuMOOTMJbby31o09WzqWx2l+FrqiGrojtOQ7D00DSflSybrmPZdJ2y2derSwWRfrkYvXPT4c/9AXBT12vV1xbUXlh9q6w6S1oyjFX7VMCogyE+0M4kF5snJdwGQKIR7E471TU1/0j1cbwWnqpQr0mVIb24tlicqXV6xYfS9m4xXyRoEGD/At3NdspSITTfU0FXb6unz+pdXP/R9YCOoMBA+iNsOi9mV3nOxtPX348Zj3/MxGc+VM9Vgenod98Je+9RXldp1BjiT0N9nMItB7ix9xx6TT1Nmz4L5MGg9hA1FHbfDvk/ewyd1oVi7HyMEdetYGCq7JrULaYbsTojL4fqMXnZ0OJsojbgAGZ0RJaIjcQ//a5Nujw+vjxJqWzdWrCXt04u5YDTSaVeJv7ePOJmTg3YymOdC4Vy/QI80foE00ftYUlQMW6v84rJ2Uy8UaT1qUwOX7YM5+WbHuKWJ+bSp0FBSvOJEutR2vWSIw3A7oprWHP6GoamfCY7e518S3wPY45BJ1k9DYj97Vc6YRd6AdpHX8qp6jbMr4S3quTvKq1DDybf+xMLpjxGnF12CDqjX0OEAZKNYPY6K+n1UN0cSWFdOg6X+hw1rvVbtA3fTv8u6oazJk4tgpVdoPfH0PlZadhshgmxO9l42TWEOeTsdP0FjWDJfajDw9yy4iSdIrbSMXK99Fhd9ifcHJXDpkZoQr6HzuXUOtyqOwFRv3U2wY7Z6r1V5V6xp/Jy2nEb/cixQ5Nb5DF7w+CuR6fTEX8altbIyuw9dOdwx30sHb8UY6hnjU2PDqsOri+G+GzFXBLelw3hk+lghi4mte3pytCJFDkgBI2z45Q6GPCz9OfT8d9zxXe1DDl4hgGpwma2qKiIwOgYFmkpI73wvhGa2wdLf08uXsaGeLH/+HqfTO4qjriMm4qh3nWRAN//JfDud/wn8W+z71/8j8HlcnHixAlcrj8bQWnOIrbGw+xiePiMzG64UpfNs2GiaVav8JKeqt/Py6+9w8s3yNJwgNVV1fzeAF8VqzP98I0VE2YftfXHeYR6Focyj7/IW/1v4eFy+Mwh5z042nSnfY5goPtHKJRPIKxWFGS+Lu2vxd36Dtw+Mewv2i//B6ZAYXPhEwfeVld/F4lT4IoyiPEsVmec6UpuCuSmQJrLi8kS1I57N37F9AD4NV69+JidjWQmw8gGOZumwRJLoB6eCgOLXWbJPG0Zzk1qUZ5A1GDBYtl7r7BC9UbdGcheLIVbAzwy6Gqu6/0s99wjP0wXO4qXTvej3Ak9OwZL45eUfMcrEVDoBJ23YlFvoLL1fegyYY5e0RQDYZEFUKq2WiGuhf3ipQYDhMVR9SFxuNjsyd70Kf+GZ6Z+x5HRp5nVR1amdordyhdDp/Dr6MEYDV4Lb/Vx3OlQ74K5iigIClfDDxFCmeJUsD3NIUINtekKtZ2gbxwMXSuKwm7FPdr3Uxj0G/T/nuiY2L+vGHh+PqOKUxjtB08HqKWUu2/czYFbDjCilWxBZ9QbiTEKJYWyABk1EAy+otihQvQwSJwsZbydx/bZosjT6gaPP1cHpLGq624cJhu3ZcqWUJ/rR3N0oMi5TMyXWea7gqbxyPsv8drj7xOkcnw4+wscngdORWHUXsNv+lU8Yclh6Yml8rjLybqh9+Hu8DjDJmm878ZCcf96baQzqjeQVrEOC27pe9bpdKSfTOeuN+9i1sftCPBqoJ6xpHFVkYnB0/7gstZyoXqPT2cS7/yc4EmrafaV37RJb2JbE3xbBwaF2hEQOVUj9yqVFJ3KV/JuJGT4+UvXXt/+CR60tWXoob2SPfTJ4gOMe7gvV914hqu/lhmndwQ56fjllWxdPIZaq1yMc7vdXJoPcafVhZIAZzXTCuHZGg2lzs6bhWWaV67HeQxdI35bXs9tqtvPFXFrSTdBgF5uhmzPXc/BklO8cOAqqkxeVpv2WnyLf+ObcU/y4fgPMXkx/up8k/muDh52buXoOrmhAYh5xNGgZl7+N9DH7xMsOnH4IKSzx5jJJ4rVeUHULRlN11XD5M9bb+KtWjOJJpiisO51GP3Y1AgjC6DBT2bWDz31FJ9GQbA1WH1xyTNgaiNMrpHsBP8Kt6w4QZPdn1vajZHGzM2ZZMSsoJMZgk3yBB5ty6KXBar8d3CwfLc03j6iPY+Ww92VClJDYwFLY0XO7DaHBumhxxuiWKdS8rhdf64JTSUe+77rTHt5qTaR+Pwo/ELkIuBixxvcVgKX+0GEn9drN+Tzc2w5p5KFKlZC/s98EQ3r4uDmaFnFVhXSl775sLlJNL0lBGeIRkzNMXU+UvQwSL/tTxXTBbi9cjG1heHU+tei884iDEznOV139nXL4nCH1tJjKVzJIF+YWADZfjLTO9BRzvVBkJsMZkVmX0LtfgDedsuqQQAaCgC3R26sB5Ymi6LVCU+1epErmZPzb+LyzQNYkfRn4f+cJVDaOF/mWQOYRQDzQmMlyzO33sSyD8ezZMFM9VobO1JYh2rYfLvTxf8ebiPntQIMOwvf14LbOx8XxG/tkq9E8dRLDTnWmsXEqmg6Hu6I0Wsa1LnhyOW/Ygqu4YZjXdF7+RgZ9UacRicd7v4Sd4ZCxZnxNLpMwWl477T8uwPEOr1e/k2fR/kuyP5UuZab3Q6C9CLL5cI4n2p7M28vG0BVfhTHs6NxejUGjHojESUR5H0+nGNHekrPqzv4OO0CZ9FrXwIvbpHVNrnVuSy8eyE7btpDbMDFfPY1UJ8LKzqIyATpxfXQ823os1hSKvyR3oVfxyzHVVXCewfkXPXy/YJAl3E4A6dim/nkCKEydyMTa8J9w3EBNxTDCrdaaflgdCgr4iAjUC7a/h24sz6leOebHmdegBPuyXzbUneMD/T8PE3OepLb5tI3oYIPa5NkG8T6PB4IESrmTKN8tqwN7cWiGthYmsXKbLkoW9fmPp5xdmB7iEamZPIMGLYB3HZQKOE1UZ/LDd3u4Z7e1zIo+Qt5/NgCHpmymdlPfcTMUPk8nlG6lCk/XkHAp5Pxc1R5Dga0omNla3Y0w42himaeTzRtzsCoAsi2a1jDrBsJK9rBoaeVw0unLaXo/iISgzwJn/FlD3Nbz9uYv+knGn17SY+rrBKF6MrGKA+V3zlMGS3mCf8UhX25+c99rd8RxXXtmSPmZ63vof3DMD4LWqvdFLYnwG8NMK9ZJrF+k/cFn9eIhlyoT+g/jlOwWGFbAgzVy6SEC+d75dz/VzjwmFiTTn2gHP6+fRfCDBAV7mm9vu3YMgJqnEQXRzN1bbvzObNzuLhd5RxEzmyvCSNobcylszOfszWKLHqXUzQMXBoZW24bDS5wGTT27ifeEk0VFXxiIOMpUZvx9Tw3JPX9grd/nc93v1yHu7Xc2A0ybMU/ooodB1LB4PX7qNjDySQdbQ53pGuheN5/+l1H+IsagEGRQWdw2xmxrzvfL51E/0mejS+Xy0XDka8Z6Qt9i8NEzvMFmN/Qhjtbaj7+/p7Pbcj9hrwUeDAE2reWM93degPmJvFei8zyOQuXQ5w3NPITdTr4/MB8Zv5QKufUdXgYBi4TTUKFnbYHvEiAlqihPL5mExUHf2CcWSbrn/Z9lPtfW8qCN1+k0jJMGk+3ZlOYAqdTwGTwbOjpdOBrqmZah3nc1m6EmijV/3vosxhjtCLq5mJwO1sIiYqFtPIAI6ss9I3/idVZs3GaPT+Tw0c/5LL24WQng6MVGBTPERAAd/e+lkntXpPG7LlLaOtTy+gCCGzvRU502RnhOM7HrdIZHaDhABV5qbCjT71OEK690ekZYbta4bknqwofwKB8mBUAyZVyfa4q5SVSc+zUu+F3uxzXNOHU79g/vZLrq2KJt3oKAQxNRTS2ghe1+NUBrTjp343nwmGwS12ftrscxJyGL3w0zvoAeT9BySaPP+kNJjIidvBE90eJNB/itQULSIqPh51b2Yr7bxHrt7qgaz9ZCVBiTea1Sih1glEhdKkL7soxG9xgLhIOYP9Loep3/Cfxb7PvX/xfhY4RHTDqhLWeziBvoGJoxAi8WgWNZvnAFd5J2PT8oRG592BRLcPPwsYcjUaLvU5kD2hsCOi/RNhoeNkA+OobaWMCfz3gln9Wep8ATtnAoAN/q9dzN5XxfiTsbhKsD29kVWTx2B/3c/LQq3y7RXE4BpHjNjH/op7dmrDXtBTuayV2Zl7YEH6pg6/sUYSGyc1Xqz2WwJrB+Nf1kcZwOck+8SG/1sMmu1wkDDNZ+a4OOuRAg48ccuR2uxnjB1cqCNHnET0cur8pbYYBOPU+bL9WqAq8YOz3Ca0zYoh2ynYmhthB/FzahrVxMDlI0ewo3Uqmw0ifPHAqCmbnwrgNeo1D5iVfiNDtc009b7R/QGwsOyvsMq3hoikbPUzaVDrSZ/N0qY7vUw6R3Chb4FpC/2wqGU1em3RTID/UwVGts7zRD5rLYOfNHN6iYcsX0Fo0MhWKFwAOPAo/RkCTooOrN0HsCEgQDMgLFQMjTGgrBh54GEfe/TyUOYQ1x55SX9bum+m0prMomnkhve4AO5sg6jToVfYyAFumwO471WMAK9pLQc/n0ZCj3IDrdDpOp5zGaDdTc1Rm5LnMgehan2by3d9iiJAZ1eXJo7m5GL5pilEWFEi7CdA4HOtM7G+GQT4Q6VAUVt0OBuUuYJLuNBF+igZqXTbUnoTBv4uG+4Xo8gL5Pb9gT/evKI/0LB5Z3Q5WpIliacYlB8Vh4gKUGAL5pqGZmYEw1SgXwfv1mc+d0UXc02kr1Qo1Y3rdHspS4aoIjUJc8XrRGHM2gSJYPrjkD24Jgm0RxVI2TXVoX17OOc6m3E1Sg6gm6zOeKrCSsb8LgUflBlGrXi9yZEdH2h3KUBY6Nu+ez42B5yxl5fGeVb/zSRRCnaJCWE9InKqdTeCyQUOu1MDcuvdBnujwGSeT4WV/2fqu5+ooRn96NU2fJGGN8Po+Gs5i2jKVSdlPMLNgoTTX6XV69ECyCbKLNKwff4iE7/yk++C/iwdCj3NDy9cwOMXz/jxoN/LEXnHv5PffrlRJNGAg9TSUp8tFlANx1zGg5byieqzJYOKaQNjbQZHPdQ4GqyBG/MOil8Nl4ddTt7G9VM4bDsh/mZ/SD3MgCW5vtUka75m3kB2J8EYE0OjVlHA2Md2ngUuscE3na+QXNofyQCms0NhT/SUsYTC1QTQDu3ky2I2JL7N2+RV03llNsF0+cpYUGbk5CH6OhSCT137QGkVxe1EQvSZJkT+XNJWNAQPoaoXOVvm3odPpaG8WB3udVo5s67tg9CFht+iN1Kuhy/PKJoyp7b3U1fpTFVyl/E2nFz9BZdd+uEMUrNq063mQvqxqgLgwuXG1LPlRbi2Bsw4F2QhgzGEei3iQ64apLc848Zo4dGsqgZPFv14FTB06YgtjyVk6hJfWd/EYi46OpnTMYGY03cct3McdYxZJlmf6Li9AVCmZrTLVBd+QLkLBkKVR/GyBv13FRoI3J37H92FTGTdCtlIGIHk6DP1DZPddgPHW44w2mYh1WIh3eNpI6HRudpoiCPSx0dD1MHjNc4aClThaiczIx6rkfc+F79OlIjlVHhCOBq009g/Vx6GpGAb8omzOPskuqtLAng76ij/nWbfRl88N4jBREinvtyLL1vLckV5kHEqjed0Kafz1SnjmrVE89fST+FfJZKPWDYf5JKWJy1PipNwlALI/E8X5bYr5BOD051B9FDIVlq0g7oXUqz2aHwA6HyO49HQ80pF5iqbCcr8/G5dXRnk1tlxOTCXrmOYP/a0ysSbF5ObOYNjcCJua1GupBRe/1YPzv2I97WxCv3M2yWcfkob0GJgVIAqjnZ2eBbGAhiw2xEPNvnbsP5Qq3YPUn+GlcOhnhcEZMjnAoDMQqIdWJtApsuidbe9n2uif+Hj8RQiuJ9+EDePAqWH7oUJzKZe3eZ2hqYu5sp0i06whn8rN3fn46evZUibf262rt+ETWs0+v2qqfGViTbhvOGPOwoaOinlOb+KkHcqcgIrIAZDWQsBTNWJsVei/NhL4Q5DI6boA9YEjKa1P4MmBYwkwyMXLrIbLuOe33dy64jhOxdYmKC4NLv2B9sNVai+dWHOM/sK1xhumAKjPEQtilZxxiyVMWOxq7AUDQzMox8LnEz+Xxo7Wz6Rdw0BWhKTjZ/Y73xj7O/jMx4ek7tDGBEZFYT+54NvzZA29U0GMLFgp5ovfNZxhfFvW3xx1tmlT2R4yCWKL27P+YeVP5c4lAbUeObMXs6s8lzM7euoYuu/uTv9N/aluVihb994jshMz1HWZ113ryUsBp7ct5DkcfrrFZl9jzYq8FPp/K/b4F6Bvameu7JlElK4eFauhbNvNBFma2eWy4/bKJcM3nhdPBdN10F7GDRV1gn/yXX9ssjKgazTv7X5TmQMXUH+SvkvH0frXTucVWBcirfYX5odDph3JdSnLL4xeVqhNgxCbl7o6uBPvVcO8MBjYViYQvt//LlwvHODj6z7mjI+CRJX1kThvfGMU/98L3UM/485eN+J0G+U5NrS7qG00FWnb789wi/95qQYNBrA4IpjX6nfuDpe/56q8HCIPbKZueyMrFKT1UUHrMOjg/fIgzCb5A5/RcS4zOz1Fh9DVUtY8AIlXiLVUFa9yMaTfAmOOiOZR/i+eY6ZAKiwj+CP7Ot7c+RFOf0+Bg7Muh8sDqylxwi/1iuzGyv28fuf7LCt+n+hhsh13VeeXSW/h+w5M80oNddnoUvAp1+kyGVOkYbEPkHqNEG2o6jp6Exx5HrI81zyDzoBZJ3LsI2oPyg/TC9eXGQGQbpItx3To0Ps1kOZvw+p1LtCZ/PisBqw6mOivXpOOVeUzvcTC2SjZZhbA59Dj3B8MG3NlNxEAitbCpklwxFN96mM/wbCUTxiU/BVffvUWzz/2GBudTtY3N/Mif21rPNKg584r4ad28v4hxyedZytE1nG8Ra4jGPVG+ljhFp9yikvluuS/+M/g32bfv/i/C5H96ZQDTjf0jamShhcZuhOSDY0a+3dLwkR0mfBgufrWXjX+TfYmmdh7mUbx/sBj8FM0nP5MPR7eV7kJr40dQcxpuMQK/fUyA8yEizuCoToNYluY1+fhqOOmILAhWHfesGd/RkEKLIuFQQ5FKHdTqWhoHZVtsP4WajNh7TDRjLF7ytozE6/n8kJY0BwnZ601FjH8koG8UlfCawUKawOXjW+jXZQ54dtqeRMyuuQzjibpOGoDt8LacYq1luWxsECLCVO2E7I/hsgB6iyJuHEio6uDbFNEwkTI+0HYM3hBr4dLuoQQbDAR5qsosI/cyaUlQpWhaugNyHqW1XEQWa+2AsIcIhRjF2OIxY0VzS8tDPkd+nhuWI1B7eGep1hy45OUHK+THxOQ9ueG1Bu+scwoMdPLCg+r6hgR/Wh1RhS1fixU3IO2asj9VrDzVY1Xew2UbhUbYaOie1t7Soxf0GQ/pxioHJrGPaF6JvrpeLFVsqdiwGnjUvcEsvcvJTtHtpsCRF4ZKIvrvWrW80yLwEZZgFw/RjTr2qkzGTjxlthgh6vtMun3nSjSlXnm7lnt1VyeXMHJictxZshFq7TgZg6VBDC52MU+S7I0bjVa+KAGbiw4S1mDwrrXJwYC26obCkYf5pZDmhniHAp2gc4IfT4RFlrZinmwPlfYoVXKWT876ptJ+OoqeizqwfKTyz3GUsNaUxWQwcgX91DQc4Swdr0ABp0BN5B+IpoPt38lPXd6WDrDUmXm4zm4TEGEGeDzYMV7AlE0LVgh5jtF4fWn2Ft4oSmeLX6XSNd2oPgAT4bCN1EyAaXZN4ndfbYz9uFP6T9aVjAT0Q9nWjkBPjZCGmXGXnjRCj6IEs1XVQNpvbkT71aDj02+TwBodZNovu7VyEvbfRf83l9qYO4ypXLZWfi2qCNBkXJjq19rMX/5Njg55fbM3sA3lruqw9naCA4FMaZf81GyUwyMK4BPDF3U1+VosV5W5Sv+NzDkrAhYXxgO1HjOVZtyNpGbmMtX079icbv9srLP7eZAXBNzQsCusPI26o30s8JzYWD1tkcCDnV6lzy/jsQ1aVjmlO8WYfd5P2uywbWQEryfg8VDWJN/jzTmTpzGE6UmXqiA9HZzpHFr1+c50GjhzmAIs3oVK+x1XF69jC0J8ECAophrDuLHerEujDEr1hUQlpZf6eDscvW4BhpDxlDiI57z4K5PpfFWun2UOmF2Xjg6s1exwhxMVJcnYYYby2iFaiqwDQPGroOpjYR1e04a9m3IYWM8fBYNfjYFAaWhALI/Esx9LYu6jZNgaYr894wn2PPgt5y+QbZlovYUc/rOYsOQW7hfYS2NXxIn3AEE6sGkaIR3ie7Ce9XQNx9cvjLpIcg3gvnDnqdTVCf1NceMhE7PalvVD1sv9gjtPOeTRP2fv6U9ubLyyaa3suDeBaweH8TIBxXzdHMZwU++zaprvlQToQw+0P0NSNJQAk93wTSHyBRUYLLuDN90HY6vSVHcsdeJvX2+nLfzcuNQlgxYz5Ovv0Gcv2fxXoeBj4stlJYG8UWdG50XS1znE8PSeqGqUjXzdIfm4k6H4hSItylyf+tOC9VruELlA6K4eOYLQdZQrOXHTMlk2eCobycM1j9JLka9kQNdDnB6wVweuOYPTE2eug2zs4Hulx4kpVMmyZd/Ij3vOpsV87bu6Nx60hzyPja14QjTAuA5h9pBhaoDLReiwdSLGQFdXhIqe2+4XfCVXnKuAAi1VxPQ5Tjpz75OUYJcdD0VJIgMfh1P0uTvRfpx2wneNpWvY2BDoqIZUnWY1yOgp1Vt+wswyFRHqAF0CgXdX8Ih5tYGaztpaKzxet6MEJlqWD2vOyFuCC9suZkDnQ/wZr+16L3PSiFd+CjiRt6IhBsc66Xn7pn/AdVpkJkMofVyMe7tXW/z8zetKfi1tzQGQH2esADv/Jy6MaaFoAyu+fksD/+xgde2K/aR3V/jYWMpx9sc56BFbpRbRu5kX30MUdnJsvW/o5EBpiZijVClIsq7nPSzgisd5kdqNCjbPyjmufYKBxW9WZwfQdiYXoCG8CvZmDuNjpGbCNCfkR4a5ZfNwpE9eHzA5cTJ07M4/1UeUJIPcdRB0SrRiEyeIY93micUrwGt1YoVWxX8nKSpFE4et4frx/xANx/5/o+wnKRH9AaSA8U8f2Fj7GI41xhbMQ4+rYE1yG/6nBp1bxMYFEQ7ClvmEYtG0H3a9TB4NXRfqBye3tierqeqKa333CM3+8Xy0xMvUPTIG2wdm/WPc2ZT/KsZt3wcw9YMw6Dap4b2FJmVqu8CON1sx08PTdlycxWA/i3NS5XLT9kOoeLMXiyPZX2ErfZtqt35fLlZrr3cmF3PusRstvXdJu9x/ZJYUp7EvvXdcBtvBsR3vdZm+1vf9Qa7jYPN+1iReYfSxtNmjkTfbzfB7XOU137FmTyuKASrvVyyBjYbDeQ7oNyvA22je3g+MKIvd5bAnFJYbpPXFfOBh3g4YTm5Sbnqs3xgGwjuJLLavBugQLfkPQxM/prWYTvUe/PmCvgpFvbJZI2Lobh0HSMu6cfQ1A+JNO6XxjOPC4HC5kHrMRoU57+qK5lSCA8UhmCxyuOHSway9MTdTPu+QuTrecPRKK79H543AHFmPPg45H7v+Xf/FI5G/MLmPLEXq6ryHC4NG4AuE/rkCVcDs8Wr6VuwkojTtzBpdicSusjrjt4aTZQBXgiDRF2j9yBFfb/nrhK4W8v5q/o4HHhc6eoFCLJb15eljNCAxhzuCYYBeVCosNv2qdvCc/qhfBkN1wbJ88EzobUEZ2Ty29b21Pt7Em/0lnCuKYZEE3wbp5gDzy5nTsUiKuzNRCeo7c+vCdLzcgRcE6toZgNsbBEQdPBsoKbWP8nglC8Yvfg0L73zEcuamzn3qc9B29b4MquVAQYdT051wSXwZIms3Gu0NxJvhEVRMLWfLJ5IyvuMh0NFrnStT7L6uv/F/+/4t9n3L/5HYVBQg0LcAfwUCy8kyWxqP7Mfk/1hd6Kezmb5QHbuwK2VazDcVEVXs53W/hodpMiWoNlzDQJvbLsaVnaT/mwyGDEAT4fBpQa52ZdUt59XI6DaCdZAr4naN4HQLHhKg1weHxhLnQveqoIthmT1f1S5H7I+VFss/hV8EyG4s3jP5Z5NiXMb3L2Fe6mzeRX6moq5J3EnXUOPMHi8opChN3NdMSyq1mA1xwznN3sgEQbQK8aH123A5YZPrBpsv9oTIkOxXsOSNeISGLVX6TcOwKDlQsrvjcz3me5eyPACN0l9ZebxxpyNtHUV81AIRJnkRTuqOZfLfGGev0Z+4tarYdu12vZ1R56HHTeox5w2yF0iisZeMOqNVAVViacI+OcKT4POwPQAGKMhzMuywy0lcFCnOJC57MLqs1ouJgDiEFq5F2JGg0lRADoyH37vJ/IML0B0dDSdbx7O4OgruL/+KT7d8L2nYqByL+9cFseo7o9yxS0arz3oVxjyh2DneSGw3+csrIIp/hqLYVB7SLhSO0vCXi2am6qDOUD9GcGKrj7i8WdfRzk3VSQwqzGMN1Nkhu5sDnLLtoFc/uMkXC75ykbmvcbHLTUhb1tJQDB8h66DNvcoL2tjo9iIBycrCqd6A7MOrKGpaD1ZB2QLL8J6i2azKUjKCH1l2yskGuHmIAiyeTVDDFbm71hCn8Zl3BlyM3Ve2UthJaspSoFWFhvJbRXz8977+dm1lMVREGKTTWfqwvrxbAVst2ko4JKmQvrtsP0apbp0zdl9PJKXz+aw8VKOXHLmKzwdBsMUjlFNgR2YU+GidYczJBjkZkhxXTFNQ7YyZPJazAoV0J5AYUv3eTRKJuxzB77j3hBIKf1N/b5ANGi1tnNFqy/4b/5EvTmMPxrg4VNDqIuXVT57Y8TnaAmox1Lopf4wBbK02Zd++fCC/yTpHowKTkPnm4Ddrb0WM2CpuI9UWSj/DWxtApMO7g4BvGwO/ZqLeSK1lujWuTRbmuWigE5HrctNhAHmrb5Heu64mt0siYFHQqGVj3wz/HLiFxL3H+Zu62T1xZVuEYrzTROlvIi/wgOXTOfePrNoVjhmWxKH8FJpEI+UQ1XgIPk/SJ7BtOo4YrLBEOlli2MKpP6SJTgNPsSVyTa0AP46GOADsQq3BQByvxH/Fv0hjzltgkG7Z855O+Rz+z6jEbalNHC82z4OB8nkl8nR73J1IHzSVIbCufcvsSl3C18e+YGTlbLFom/9KcIMcCZkIP5hXeQHN+SJBlHeT2qnh8x3oXiNmFe8UFBbwDXbL2HKZ9fSIdLL1r25jFHxorj5R42iaeB2MSXARFEqZNjlvU0PdwkTAkVxUZmX5rTB1wZYpUFAiR4qCCoF8sH8YvB3/pmLvKe/vGYl+jSRHF7L1m5z2KzInuHo89wdDKWpGsSa018Ixr2igQmIeUJv0FbE7n8Qdlyvbjg76uHIc6IY4kVoKyKUT2og9TRk+nsS+XR6PY8E5vPKnNc4lHFI/rzDenBFIXS1wNQgRcG3RUEQaYRAp0IREDsSJhYJ9r0KEf1h9EGhdPMiawCsNneiVQ78HDYVAv/8XRtdNn6MgRfCoY8VDF6PrYifQjdjHhl3f0lEB1mdpNPpyGl7gtphGykKlpvZuyIm8ngZPFIGtc2KvOyOj8O4TEnJex5hPUSTRaWYRQe4BXms3pM9PzS/gkufv5dv1nbilUZZDRYbW8Luq7/i1UvWURjgaeGMzsQb7g68Ugk7ouRGIhH9GJoPb0fA7Ub1mWLCWQc3FENejVaQ+EVgCsR56S/kJ8jnEYPOzcZGuKII3F771NCwDA7sf5Q2m/oRXRQtr6emAPKNMfzeAGX6YOm5jdZwHG54rgIaLXL0xczSxTwQAvFOjQNoyQY4vkDshf9JLp/BTFB0LEdKBxCXIZ+ZAeLOxtH2RFu1jbJPNOWdSjnZcw9G7+ZrcynPG/bycjisP6hQh9rK2dxye1TXjf7713wORl8YuFQ0A3t6Pr/BAE0tdtb2APlMYUM0Eoz6ZqXyCYCjz4tznDecTaL5f2Kh9rWlXg3Dt4J3HQEE4bchV207DeCoR7dhrGTRDHBnijh7bc4Ve5d/1Bh7fj47o3qwrB4GJMo2yz7dXmRx+0/JveQnooOS5Sdq/yCMOSpUbCrojRBzmcgwVSCzcBsPhYDOy5YPvYX4lUMpPTKYt24TVuIXusYMt1oumjNrdDWR3uUkHfseUhNUUq8Wc7SqcQtcXSx+d9ZwjUZ65EDh1KNy8dGbhIpz7xyo9FQZVWR+SivdWZL8m0htlNfivbX+pP00lufKRpHoGyyNlyeV8/W0r/G57M+GmtvtZiwXV/iMRViej2+9kEf7T1Te383WaJ68bDlXP/wJRYdkAnq1OYUCB+SnAEc9Fb/vGHbxSJiera0ewxAku2M4EFE465oV81D6rWw0pvNjDLRuOCqPRw2C0QfgyvLzDkIXIu3KBZzWXce8wSNl9d7264QrUfqt4nlU2DwVfu0EjZ5n07rKPdyRtJXbyxxMXCVbVvZsd4S7XnudRdO20Q1ZsfXrni4sqQM7ToyKWL7tZyfw4d6F1Ns11Obf+cIPYVD+V61cL1QegOxPRAREp3nSsNsNXaJ/596+VxGIZ+1F7+2A4G1lnzRNNO8DvM4iLTDZKpjgDw+Fgqsuy3NQb6A5pAdvVsOX9RpuNrUnRU1pZVd1NvqpRbDrVsktJ8pewrPh0CEoiuSoLtLD/Ms+497UNVxfDL82yKSdgT7NtPGz0+jTiI+P5z7V1BIt8kIF2HopFIk6Ew1uIw43ODQas/cZhQDgSkWePCDIcoN+FQKIC2BsdQ0f7FnI6TOfMchkwns2moNQNjuBu4ErgDuA0zEx7Fj9EX7RCzm8exU5W3KklxzlY+e1CHig3EKHMXIdTGewkO+Ak3bQabln/S+Bqt/xn8K/zb5/8T8Gg8FARkaG5w+g4SzTQ2p5rgLq0uQMgcUD7uG7nhPoHhxN65Bkabxs28240+HREA1rsLqWoo+WbU7ClaJI3k321gaE57ci8NXXXsYV/jDqLNS3lVnk9dZY3q6CB1096NHR67X1Bipd8EssHFKcfd1JM0g6A3eWwkkUFS9rBLS6WShW7P+sgHj+8X0+gox54O9pl9I+522OJsEvMXA23zNMmcA2TG/qRJ0bCk8p1HN6Az/VwROhMNmocIRvcye5ThMlqeDXKC8iPkN+QzdqH09N1ZCsJ1wpVHt77xcM6X+CXzvBsVek7EUAHLVEGly43Q4lwzew9ijPh4lCSpTCQ9E0tY7cjAUk9JetIgARDHx6sdJKAoCiNWJs563ymLMRNk+BVT0ltpX+4OMseHshn8+fy6ZG2cqNmpNCfbFKYbkK/B7r4Jd6GK6IJqAum0dDoItFg/VsDRcWtzq91AAS41EwfAe0vUf52ufZ/Iq8ndt630iP5ACCu5eTHJzsOegTw/GQIcxOfRNjiUZTOHaUKHAqoA/vzbOdxvFtDAR5K0dAsL8uVRzKz6HjYzAhT82oA9EM7P2x2NxegEafRBbuSGXHb31Z1yhnd4Wkz6bMYsBis9C2VlbF6op+57pA+DJawy72yHNCoey9UQZw2ckePJM3L7mNkR1nKS/7i4Nf0CEXHqpRzDdGH8Gc2zYLqg97DD3g2EZOCrwXCRHedoGAT3MFc+fO5erFVyuKVkFk2iHMWkHHse9Jjy2pF3PI1YHg55TnuVMVp3iiHPrmXCRXJrwvpM5WWntEuupJN6nzu8wtr3eJwnLebDDjAgKz4E2DXPzZtnwAd/bbw/CE/ZQFymqbOr8UriyEpdbeylDwcifcVwortBy0znwFod3EvapC11fE3O51n5xrLLbr4KCdfG4hJzqKos6H6DDzV1w6+ZTZaBeMy/mb5sufWfJ0foy/hwwLtHZrFJ7ixws74v/DuL37zXxRC3HZsKXRk0TiYy/jgRAY//rtvPjdLGUjcmg+TAuAyY790lhS5UZijDCpIYOIuOHSeNvanVzhD2vPrJXGxBNMEwSTDo/LaoW/wK9nHiIp+Ah39JFdCYxGoUICCNbY+9iMUOQEvfc9ZjCzttlCdGYj4ysUFrjN5XwYBU9XwHKD4kYBuDwHRh9Wq+idDaIJc2IhZH/qse8r29ebN8Z/yJLRyyFe/jx+q3qY21rOtXYpqyqXyiWRrF6k457Pusive+wV+m0ewIY1V/HHYYWyJPoyGLmb5KE/CbWFN4I7CpviYy9JTWPgT8JNV7mgZds0hfIsM0Wno2Sbw9CeRB/ogjETXlVlrBx+lpl1KyByAF3SFU3jfffyU1QzH6al4YfcAGJfiwo9UmM9BLE+qIrN58ZWdJRyak/49mRv170EDN/M93Gyyvga90FWE8V7Jek05G2Vxs8kXs/TM+fy9My56mafy3bR/COOLYCDT6n3FwADfhb/limKWpawPzNNve6j8VFTsejgtANcZs914dz8cN97t/HkD9PQKa7Nx+jDi+Hwcis5q5sOj1LkgOAsOGZVWPsarMJR4zeFDS2IInfme0KZrbCC0+v0ys/SAIzzg2X1cOmaj4iM8VQORvlHEalLYPChIN60Z0iPv8W1nwce+poF/ddSaZFVnM2mUOZXwguVqO3tzCEQ0Oric1zZTkl5DYimbvfXBanIi/zS4ZJbMbRpoNZox+CQ3/fVTbvp8dkM+mzvQ6DFy1Jbb2CdMZ0HyuBAoCK/yBLGCV0YWXYYkX658pIn+YtzmqNJozF2MehNGOJG0rrHRKnos964hPEtU8Ha0/La4dcqhuKpvzCp/Vn03muty8m2nLWMOQsVaYozQ9eX6VXXlcfKodFHdt3wtZXwTS1EntQ4O0b0F3mXCrLcReFs5pkHT3LnDUXcdqtiTarYR+v2Z0i/bDuXW+TzX33lUbq5crg9roHgpjOeg+ZQnqyPZqwfPOPeLj0Wgy9no6fwhftORk5R2/uzfbY4D5UozkoXQUjZO8zMmCsuwyzvH2wEU94Qg8utUdirOiIa995OOSBiODo+KYhZ3moaEGSaoy+JOoQKEf2h7b2iUanCqUXiv0mR7XXrHKIR/MuJP4lfFzbGhppMF4lTeIhNhnTijBClaBpH+UdxTZdrmNB2gvq6fKIhqJ22FX3BKkEWUq2XwBhfeCIMAuo95xM9bnru7knGLxmc2XDmz/fV4hrT8+5reDHEzCwCmBsaKOXM+sRcxh2TvuLO0T+olX0gmgZZGlmC1x9A1/k5rh4iN1cBEZ2iM6oJsKHdhIrTVimafhfgeVsaN7Xfzva73ycvXra4tfnaMPc8QKzxJFZvW9WaE3waWMH0b6aT9Z44H364aBFDLRYeRa3wGd7y90eBAXoLNSUv0jfhZ2Wz79ya2TcPPjJ7zrMGg4HY0FiqXXAsbAREeK5LrVpNpUv7G5ieoVD3F61haQx0t/wZlXIhPi2vZrUjlMv9dYyO0VA+XQQGk5GUAVdAxtNyzmFAG+GI0P0NJbkLgNzvhLWuF5G0Prgzbc/A93WQGfuM9LAfDd25u8JJVZ0Vo2Jb1Gwq4PFQWJKiTTBJCjpEj9gV6saWTwt5yjuH8K9QuBp23y7Wcv9kz7HGItKq5jC+9UIGJ39Jl7ae79niqGWgj3ADWeTf+nyj6zz8U0Tz/udE2CrXIHz338djoYKAlW9N9Rx0u/ls34cYgEaHwhIYIGoIhLXUuVT1qpjh0PEpQVi+ABGtroIRu3j3ip8IdsufZVPindxwaAaf10C2Qya9j6iI5N16B3VDtuAf6Pl4k6OBj6OEsq8+WuHaFTuCq6sj+SEG4rPU80Wj3o9JBbAvUGNvnzZb1Lu8ETeayfdNxmr/mKubZEHIa0BPRPbz68CPwNtA/Jkz9B5+I6+vdGKtGo5F8Z71DTn094FTuiCMwanSeF2r2+mXJ/akOvd/QV36/wiU/Y7/IP5t9v2L/zG43W5qamo8VQE1J1gYIWxxjFGytU39kZcg/2dWt30NwuWmRUmdUJhN08p56zwfxp4Ag4Z8SacTzB0NBhnDNsBkmaEbUHOEb2IgxhLI2N5XSeMRccM5lHgz7drMlJ/TZWfNuFexWCNwBckFgwsXSi3GB+m3CqWESc6M+lsI7Q4ZT0gHkIjKHbQzw2W+YPDOazBYOdBo5+YgmKpTN9uGJvWntxXGxsqf56acTaysKOPTel9ax8os9FLfVqwoy2dLrsIaD0TTwRIuFQTO4+Tb4jD3Ww95LCD9z2wab7S7n74FfgzyAdM5O6IL0Cr7dXpYYUJVnNKyUqfTkZhxL+ZkBXsYYNQ+EbrtLy+MgChahV8CfjJzGaOvyIwEUQS68HWDOnKgGWYGQIhTkRFwTvWpwfBq6xdEon8UH09SqFprs5gfDvsS4ba4VurrLt0EJ14Hh4LBbrBA2TbY94C6mBczXLBoB/4iDXXdPAT3hghCmuoI8/VqjPkl0XX3BvY2QW69RgGmfJdgsjbJBUqdTodfxqPQf4naXgZgdT+hxlTBZYe1I0TjWIW99wklj1eO4dDWE+gweig2k40VJxVFwqQppI8OJaVDNtN6yooXV7BgrQ/1UavBzt9btYpmn9tFcO6X9KQInUpx63ZxJhnuDYZKnULK5mwWlig93hJWoRfglD6cNQ0wLB+K/D1/8/lVZ+iZMhKA1NOpUpulPmIgQ8/CF9EQmvm49LKL9Z0xZYLvKYhKvVIanxDdmo9T4nm1t8IaD0ShwBIq1gDFPHld4++cTIZLs5+TVLcRcaJZfFWErJgNrT/BjzEtOUCK5lFOQxUFTvGUQRa5eWrQGfihDtbrk5WXvSAc7g2BzVqi7ZJN4h7TUnW3u0/M7V7o07ALWyt4w+8dlv8kFz8NkW7em/gD46OPUBjoVeirz+VQdDm7EuCGQPl91zTXcLBgB19Gw2VuDSLGsVdFhuL/YbxW9ynfRkOBE+qdnnNNUupk+q1qRV1VALZGi1IZ1eiGVyphj1tuAIX1XUTdoLV8cfU2sf54YVjtOr6PgfgGDQtnnyhhsdL5mX/c7Jtw77VU05GMBDlLgsPPktu6mAOJUNliC3QhqlcP4iv/bNqYwKA4cBXVFVHmhA3FJ5Sv3TMsiZcGPs7G6zSIN0Y/CO6gttM2+sElX4pGYNr1Hvs+B272vH8lTzz7BH56+XBtCJjFVMcQihKCCPA+rej0FDdWM9wPZjnkdRof4YjwQRTocxWKAUuo2PeoGn3nrjv9NkGkUuWd9P5QrFmKuSS0MZuq0hBqExQFdL2BamMpmkmVwR1FU7jvZ0oyUn3GczQFtOV6fZbSWYDkmXDpD8omJCD2DeYgaK9hR3XgMaFEP+up/Ose0Z1u+7rRtLcD5hqZLGGLu5z3H72VondmYtonrysFSbPZ23UvGwZsINgaLL9u3DixTzinRPbGvvvh8DyhNFIhdoxQyanel94oyEhjj0sFnsn2hzmTEMiIohcYkuC5V0wKTqKiy1BiQ+romFyClP9VfZQTfUdyOuUO9F0Ve4DWt3Gf/wyqXRAdoLBtd9Qr81TPw14n1G9t71NaYi6Ij+J0kouoI094FIQNlmBMp+CmEoiqHic1vmLdtSzIncachXNo86XMrrdh5MTWDEb9OgpVP9nqthFrEAUEpXK7Lhs2XSnsilXI/hRW94Z1Glb1be6CEduFHfkFCOh0GcdMSQxaPZwZiobHHz5twbeR0SERTAiVrdqaHGKNvHWFoinmdqPTm+ieB2dTFeNAt/j+OAw+XNFBbSV7Ubic8I0Z+4YrZSV8UDDDdIm8HgEBzZ4uKZUF63hxTCrxP48k7o+B+Hnb1Fbs4TfzZu4OhsQgmS1aUl/CviKRlaP6rh7zm8x0BRfzPPyTReN/db/zVqR/C3VZBG1sw/D6GPw2Kc5gpz7gWh8TVAdgdCjuoXUjCdnXjvW/9SY2wmv/YfJHn34z1xXDi6o4NJM/cUO+5aqZb9CvldzMBv4kMxSslMecNth1u9jbl3qePy22LBrtfsz6qQizj7yG63Rw92/7eX7zj+rXNfpC1FAIVXwmOj0kXwVuB1QoLG7zfob9D8H+R5TnGeJGCzWtVcPZ5cTr4vx1zsHoAmyNLuSdXe/QPsrzfHiuMZY5LJhXIgxcE6jj9XbtpMbYpSERLIqCXibFPrR8F6y9TNh9q85/9jrYcw+c0sjgKlghyEJb1GfqR0OhwQVnAj2V2Rda3+98cqHHWHR0NPc/9jhdbvehx5RkRr9ytZQz2ya8DdSFUFPtpyZV5v0MfinQ6hbldXU6u5hHfIvVuaYAv3WD74MlNdh5JE2BSaXifHwBGgx+HLIJ29RSi7yufBNVgU9eHAeOjZKtUR11ZBhsdB20F1OgyLtc+tVXzGpqYg6Qg2jsLQRuQxT/B7b8fQ5wg6uJn7fCj8fuV9p4+jblsTIWOlqg2ui5t3K73byg20NmEuyOniniSi5Er/d4s7CY5d915Gyxl0qzsYDx/rA7ER4JlhelbXnbeDVzM/PjnsaaIZ8fKd0iSDMbJ0CFIjusPkdYm7d7QN7bd3gYBq/UrjWByNXtv0RS3OrMAZywQw8LjE2QVfA19kCiF9zMV0/exC6HLHC4v80SngmDsRoly5FtPuOt0Z14auBY0Tz2xsR8sU8NkJvCF0XiZKG+84kWVqAXwlZJfP1CTld1YdK3jei95pOgmkOsj4fnw+GGmJPa9394H6W6zxU3nhcrBAFra8F+r0EbTxQ+iyMddipKZIBonl+2EaY2yo1bEE3mw0/DHi9HG0uocJb4/RIp9w7AHdiRj6xfYQdO2tZJ4zp09F4zhBH7uzMlzus+cDVxXSAMC/RnZaZizQEyK8+wqxm+ylbY+gK2ygMctsGGuosEqP8UD1vlGnRIzRLqq3Lwbvm+BjwPbAR+A2YCQ1v+Xe92s9HppODEoyQ5uzBvsHx/+nR4mC/bf8yVA9QCmTZhbbgnRE9hKvg1yATw/y1Q9jv+g/i32fcv/sfgcrnIzs7G5brgoBvShWXxd5Pp05ZNOTLj7tmCImYWwdjvpp5XFlyIHRGTCM+CD8LUihV0JtF0qNGw/Dv9BWy6QnvzZTArCz+20O5MLoQsh1u54HaP7c57vWZyj2u3vNFoKmbI8Xu5tN1MOk06LD3WVHuSZ8JEePsE5xFpHGcTlO8UPuRaDYuLoeYkLG8rmmNe2NVjCbpM8MmC5iAvVYrLQbRJzy0lcK1GTNZbxr18XQe7ouVNenjel8wMgDdcbQgKkRVde87uInzTOCI3aKg/bNWCnT9ih5olee5QUaMovF76g1DPbVLbJ/m66vgwCk7ul20MC5NmM6sIlpaexa6aQvOXQuk29TWDsGTq/y1ED1GPm/xh+BbJdxsQzMekqTBip2A/X4iUmbxQKaxkY+2K+zekk7BKuUy9kQibUsrIMSuZlqiw3AnvTV1HYeUwNDJZHnc5ROF00K9/hqlfCLdbWJ7lfqdtw6WFuDEMvqWB7verCywOdHTPg8e0yNareolNnUKdUbiiN7ZVfRm/5TO1nWDWJ1C2VQ7sPofy3aI4qbBVBUTGRNr1ki2ISWfEqbdTGVIJiqcurC0keGIqAz4Mof9o2UK0xK8tq+sh+jRqxmlwS4GjWp5P0JvFvZP3I+T/pLhoHSVOCNRDsE7FyN4N64aLRqeP59ZxhakDw87CmkawmT0bszZnE3d0OcFtL73F6DdfUFh9GHC6hc3FZpdcgPc1+eIAxrS7kh5xsi2OuSGb64z5zHGpLXUo2QjrR6vzMYDP68Tc2VlfK90LQcGiqflQmvzb8HFUc7kf3BIEacgqh3vySphWCIN8YWhcF2m8X9nP2FqBqblYuREscgoLXU10fRky5koWzOex515hZ+KFGmMYK+qhlRnim+WG3GTnYd7/ZQqXLxsnN8X0ZnKcRnpY4eVw2U7wyInPicj/mrtKYJVebdXCgYfhlJoR/d9BflBf9jdDuunPrJhzuCRlKHXBvmS2yuTwNbXKQ+j1gbC1EQ5bZCWOObg9h5ps7D71M1UNspXK66Z+1LvgKy0SrdMm5oL/Qn5GVBQEzTiEYYSi2eF24qeHjmaoVapl9PS2wvFk8PWeB+21XHPiFlbGQhQKpqwlDEbth/DeavIAiD1N/i9SBgsg1qzkGcJOO6SLx77v99DxnCgTTYzWFfJzT58OacED8AtuL2c7+sZjxE2RAx5Xzf3JM2ibI1Sx+1wKEpTLAYeehm8s6uKPyyn2F90XqhuCOh0cnKtU4P/R/i1+6r6DDRkbKG/wujhHPb2sdtbFCUKFhIRJ0O9rTbX4slobrQ4e5xZHD/BRNJDCe2vbToNQCujN2gXhUfuEdV2KJzGtT3QigalG7GUh/HBQVoYMuPQNGseJIkRBgqwGM+gM/HL5L2RPzFYTVNwtjAitw3D31yHlagjprB63V4t7TaWSOAdTkLSWN4VNoKI+hd8ufZh2DZ73gdlgxn4qk+LcaI4m9iHE1+szq88hoegnWkd1xRqqyEisz2O0/SizAsDqUhRoSrcKtU7bOerrrdwH+x8Wub8KNbqpbBOJJrg+0AXNfyoe9To9uPW0M0Of2OVyk6ZwFcn1Qn0Z5is3cBYbOnNg0RX03tmbVjWyTVyPsmWcTYXDSRpWssdeFZnYxzUcUs59xykaJKrddykL/06Xk17Xm+gxfgPDIuTogL2WVrhsJnyr6yTrdFxOPmQDbq0iYeFq8mKKmB2oTaoc7uOglV8oPj4a2WIXQ4vCoLGm2PPMCwxNX8K82FLuCgYfm6dy9WztWYpqC6gbsJ0Ot11HuJ+X+toazgp7CDMCILREtlFuOvMdG+JhdRwEKYptOnRcaoVRF3PYMgW0zBf/YO9uCRdnmKCOQhXjjbTr2Vh0KZk7OzBgoLwPLYwex/6pP7L8wXWkJ8rOHENShvBZLfxm0FDxNBaLuTlfJhACMHy7+F+6olHjdkLmO2Jv70WqrE1fwJTv66hqisKicBNs1QqqmyOpalKo5EGQTPt+Jjc6ziEgDabUQYZCkdjufogZBSffUFrRA8Je+oxGDMmQ1eK1FbhsSDO39byNazvdJ41FBzrZNqWaewb1ovWkVHYdPerZGHPa6F30JUcIwaVQljSd/kLMc5lvU9WoWKyPzBeNyMOyXSAgbD5Du0OC+rz+Sq0Pd5bC4Dae657L6E/AVT+T2CaHwFEySSrQLxYHoymus2IIV6xJzRXc/dLdPPDKA+pzVsUeESOiyqkHMf+deF1tbw4Q0aLQOa6wO27IhxNvQkOOZJ8bTBNpLRxwVQM/02EkcPYP7Jt9mCqbVz0qtDtHWq9m3/pu1JSIM1p1Tc35JkA08BjC1u9ky7+PtfwdIArIrQxn+Ul1Zl+sNZB+PtDZDD5uz4OLy+ViU4OOn+th2cllyo+kofAPxjqO0FCf6zmQMovF6a9zygahZnmymt68jZ0J2nmr1GaJhl/+UrGueuOcgl7l4gDCUWDjJEFUUSF+HCReKZH4LDoXUQYR0zAnSG7gB53S49fgh81sw2yS77F2rTPYW96BJ4rVRLtpowVBryj8CTUp7b8K/2QI6wlLk2CPl5tIQCu2Rubx/dGHsLus0p6q0T+d20tg0KkErvlZ8XmefAe+NkH7RyDjSWlYl3oNz1ZAohF0Li8SoE7PAasgEtdrcKRwNgsyhNulruu43ZB6rXDuuBAuu/hf+4eVa5bRCHFGqEyF+4JklXEXk53qwVswtkpGZ/WqaVojyRy6lxBXHYMOXidfU00m1wbCHSXwfq1a4fymNZPFUbD4wGL1+/6th7h/fb02OAefhL33EBQcwYWVwSLgYWAZSNae59Ab+M3VzKJVByivkYkcEX4RXJeYwcwj1ypJ7/aidfS1uvi0BqwqAcP/Eij7Hf9B/Nvs+xf/d8ESyrbaBhb7HCc2W24+vXB8PfuaYUYAbDwh2xAZdAbKXVDr1rCDqDoA26+Fs+qNBmVbRRFcg0GmBZ/A1mRZ27AhupaqzTKrgoq98McAOPMlu7K8LDZMgdDpGbX8GjCUbeHxUDDooBHFhtTZCDtvgtPqTfxfQqcXOQG775ACbeOD/pycJbug2lOs9T/CZb6wR+EeAFCLmWon2JyytV5o7RFmXuRg3WfvdPr4wMY6DcVKwUr4taOwG1Ch7d2C0TRFoTQDwSQP7Sn/vfIAQ31haiGsdMoFsbrIYfxcL3KM8qtz5cdvniyYQevHqV+36A84u0I9BiL3rlS2wfJAWE+hpPDChka47CxkGRVFQBBWKV7WGR5YO1TYknjDFIh/xuMwzY4+XTHusonfzJmv1ey3piLxm0u/Td3sO/ysUGEeltlUmW2e4os8+Or9TdQ0e32Xddl8H+XkiovU90ic0mJFISsSi6yp/FgnDh6qe5Tc74TCo88n6uc+l3vUYusjQWcUv818r/mm+gh3RD/BvAe+46oup+Tr2nkvh+9exaKRfrh95MKpvv40A1oIa8rCaWAbGLpWbTmi00FYL8EgVhWUdTp65cGsQLhHf1we94mFNnMgRG4gXThHeBcBdToz6YV22jWV0au8STqk+jWc5tFQ+KQG1jjlYlpaczZPh0KSvUDNYIweJhSNVXJxEhDvOfwSwRBWqAK+afJHlwkBqn5GCztQXyjn5vmlXoVPFswMhFlmtcXcWD9YFotcgATO6gMx6eDe5vXKbNPlrlj8dDAvTqODZPIXRZKsj9XjJ15TBpWf8evIxELQZ8JLRjlnZWDIIAr3tqfrnu4Ee6kN8IlmSlUMnXNgrCKuNaTuOM+EwUEb5Bhkm1oAxmXBKIUi67+Jfcl3UuyEk8kQ2OD123I2c2D+Ur448jafPfcZZgU55pVIEzcEQVKQ4rfhbGbX2mkM2H8VuWfkIuIpfQSTC+FuBekegKxFsCQIvjFpW3FpYetVsF6jQJjxFP5ZYDgFn7nlee5sl7cYlA/PlIPOu8mjM3JQF8lIP1irUbOi5iRsGCcK+CqsHQYbLxcWkP8AwX6+5CUIe6JqvbzW+h2+gf7hC/Efs1zZ7GhltBOoh980PspCJ7xaBUddikr22eVwaK5Yu1RFkqoDgnV/XM5ZAcRncvhpyFPvQ++fsIOXJ+yW16ya42xIK2GQL/RTkI4BKNksiEgKS0q9Ts9ZBxx3+WlnaB14Qvu62z8MI/cIcpgKIV2EU4SX84C+bAM12WKvNufGe5QPrb25jrlz5+JOkdf4sIptuNPh7UCNIrWjXjS9tPJv29wFfRcLe0gVlreBX1LViheAHyPhj4HSn4vbj+VD2/Vst18NwXLDLvD6E4z/eDyz7rwef4vXfRI1mL29lzN83Vtc96Piuiv3MdO5n8+iYbCf4kzinwodn4AQBcEKRGOg8/Mif1qBsp4fE5QFkaeNInf7HFxOJgUG81I4zOs3m7pqz3nQFTOK0uvSeOrLuYy7Vt6L6nQ6nCYbvuGVRIXJ6067DkKRHabXUPYlTRHKAK3MvrTrxN6801z1+Mk3YeeNUpPyl88ux+/r1XzoX8Le9nIUg16nZ+GcVyl6OglSrpXGm3XiO1DqM3xi+KEOLveDP7Y8qL6u2JEQr7b4/EsYzDgnVZGdIP8ujcefJdVqIyYbaoI8m9nFphgG7g/B8vsAik7b5eKlfyqzCmx0tYKxQP4uQwqXMcAHelogWOET191xho0J8IlGb4q8n0UhPGOuUs2uCWukcFEYcwi6KDLJwnpwx49v8KTzSdp1kO2hC+OmsZBaakIU57emMnrsmcGToRr5n44GWDMITr0niLAqGH0EMUKVG2mwCnv+Poul6AGjEdJC9tApag1mBbd2xAiYPRsWLlS/LG4X/BwHO9VqMNaPhm1Xq+d2/2QRuTFyr9oZ5sxXsOs2QRBQwRIpiHoH5Uaibn1LcTtI4Wqk01NtD+XMkhEkHlScH3V6TI4aNlRVcsYmN1tOBvXnmiLolANOFbsxvK8g1lyqsb/wjYeRu8UaoMCT153hqek7aB3kWUyOCm3P2z1rmD39E54Kl0kNBr2BNjsTGfPrGAIUYcjuC4jCyvusw6NwZRUEaDScB64Q9R0t6+nuC4VtZLRsCU/dGZFDm/medN6Z3LCJU8mQnQxd62SS37zSMHKevxH3N9XK5qo5zMzawWv5IknIYoMCAz2aABdDMWAx+xMXeFJp45mQMoEnw+9hTgjMqJH3yA8U1fBEOcysXCI1pVetGIPbUU/qabD7ygRum96H9Bz4Uq9wwMJNjAE273ya7YraIMnThfJ/1H6InyCPx44RJKhNV8pkvDPfiPpY/k9QpSDQAtiqRIPW67GBtYcoSoVDzfBImVzrGhWxmz6jthJ63Q9cknJGGu845QGCxxxm7l1q68awIfNhhpvo4fPUc8aKjqK24mUF+7egNwuycoTXa+tNpHaMx8dUR5uwbdL96fJL5ovGIDa48ygyKwiAfskQN0Yo6RQw6A3MCICcFIio93Ib0Zv4MWQSukwYrNGXpXitmGP33K12TTj9mcjBjPIi3J9dJpqbwZ2EStoLvsfmkJ8CeQ7Ib5LXws/jfbm0OZrMpXocBu9zlh6XwZdMO5y0K344VQd5NUJE5Wipv1YbWtHXB1a20SDQGnwg7UZ5vXU2Q0A6l19zC59b/2xGfwgMRrvRdw69gUt0Vu74WuHEUJ8HZduFStMrBgrAmfcTY/xgbjn4KnI4/8V/Bv82+/7F/3V4b+8iTtthS7GsvgswBzDcFz6NhoBmWU6W0JzLK+EQ5dLICTrXEIuQbSwA6PKSOKQm/bNmX1xgHDM7X8O7VbCkXN7Y2RtE2NNtJVDg71VQMAXyuTOJr7bO4+Rmha1Z3DhOdn6Hj1JfolUfhZez0V9shrMWqT27/woXFk9cnkysFGcJs0P8GKGy8TSH8JMzht1NqI77AAzMg21NUFYgS96PtX6Sq4rgCf0hikv2SOM73BHMr4AbNGpDIlNnKBSvVxf/L4YTb4octw6KQ1H+Ut6NhKM2KNTLRUCzwcxzYVDbCsw2RWGpz2eCQa7Fslo/BjaM1Vb/7b4Dfu+nbgg6m+CHSNh+vfRdUbCSl2ICOdIMEWGKA5ujXmyuVSoG4MDOJ9jW6OLjGsVGxOUQzHm3Q92s01ug83MiGNj7ugAMvoLFrbDeFe+r5b5VBNOvP7Oe8iNFOA84yPVurtpruNzfzXuRMEKLmdz/W7iyQsn4Pxg2kkY3lKSocwDo/eHFGxLJM4V9rm+cejywtTgQejVmT9aWsGT1Zbz9wJ0MqpMr7IGOCgwl4ZirzZw4JlvrHUu8mTtLYZCPRmZf5QE4+Zaa/etusZ7r/JxoGmvgtUrYhUL94Z8ifnvbZknB9IOdZyhOAXc6JFZ7FlkMBiNnmwykbu9J8ulkqRkY5yhlXhi0NavVAu1qt/NkGLzCVhpKFL8dU6BocI7WaPb5pwjb0dIt4j72gvL7P4eqFlZwu/uloUi/SDbN3kpuu2do3/d1aXyMv4nuVri5GJH94IX9vt1YUgtlLnVBYXjKEEIM0DNKw1q6oUBkSaiY4CCY4N5WPkC0vyjiugEVRdfQVuRL9Bm1jdDGM9K4UW/koA22KLgYpeGD6JYLe5rgiQGyhSggbIpDFGqY/yYMOgM7m2B+BdgtXhXMyv3wSwrLJ9zDhnvUzZDPgi6ntQkuaZJ/97Ubp3CXbxXvVIHdJDcxdTodKxvgW63o3ECNzLu/A0edsLMqUStTzyExUC5eGvVGNjTCgvog2id6sVmNPiyNuoHbSuDtKsUTuhzCOjH1OoiVD8BASyE4QL2vcjSKYsNXOsEwvQA9KSFiwh88+9iz1CQqmtnhfYUC3pvdC2CvYX59BCM0D/wb+LVDZ2IN8NRAxW/DN06we0fsUjeQLOGATjQwVYrFc2z80XIDP6ruML2t0OxWkDF84nm+PpS+eTBZFUFUsgnWjxKks0a5kz7k1FPsSYC9eRsoqlV02nfcBEeeFYVfFdwu+ClWu9hsq4RmuUBY7d+OuN4Q2w3iY2QyUtO+x3ikejH9reCj+L78qsRaMcbaqCbWFK8VxJhaDQvcv0JYX7FHyXxX+79RPPdnBxbzfM1d3Ny4n7Nmz7W4srGSp3c/zcpvv+X4V+tk1aHBysKD37Ladx9XVilU8mG9eMueyKKGICxhisyzgDTRpCvTUGX7JYhm9K7bleSARfsWU+OCUofDk2TltvNDdAVpJj1XFUGRw7PgVW0KZ9K2zxiWDx86ZFLDCKsD33nfEfV1N3r3VTRqEiaiy4So0xrKvsgB0P+7i+4vLopzylSvZp+uUc/ZnR3p5LiM2wbK+UfXNG1ieZsG7tv3PKvOeCkh9AYeMA5DlwndVPFHIZ34xn8E4/yhp0NDwRzSRRCktGxX/womf+Eu4wVnz/eYcNZJkROq7Z4LaoO9gUafRg723U5YR7tcCHTauN63nttL4Npc+exZ0eZBeuRCYn4AbdvL58urmoWDzq1a5yyTv7hPvazo/09A56hFl/0hOq99JIh580p/eM1ykoYKL1WWTkeTW8/TYfC8WW09Tc1xQehsr9H4Kl4Ppz9Xq+x1OnGmTr1ayiqMCHezcGQP5g8ZRqRie2wwwMSJkKblmtfccmbUsgR0NIh9dYNibnfZxT43pLO68VrXotzUarI7aoT6WUGcoWS9+LfDI/KYTwyLm4UKNyFCoc7QG7nC0Y/bS8HulM9/zb7JfFYLh2xIjh6AyG8eslp7vrBVQu0pTav6wMqdJK7tDV523Wmhabw6XKiLlc26plKGBAub1ph9clP5RLOTbgteIm3Bi0T6Kb5svVkQgrYpCN4gGgaTq0Er0sMcLJRNMbI9HiGdhYV41oeSG8kRSzpL60Set0Mnd5zdRjfW9qe4rUMRRofXZtRWRdLp5+l/rAMxpwSR9PIZMzyaABfDJ2YfxvfV8eSAsUplH4DL7WJRNZywyPsql9tFgB5G+QG1ni4Urau3Mtlf2DdKjavGIlKbMhmb0IO+8XLsy0+BI5hWBH/EQ7NK2ao3CRv9kM7qJlNkf9F0NVjk32bBcnFvTW2CbhqRHd+HwM8J4j69AE6faN6sgmcr4I8m+bsqTgwmevJqnuyRT4pNobzWQWrgTgzFauvHv8Q5gqldvr8vihNvwJIAkXGf6uXk4LIT7XeKN+94k1eGXyI1QHvH92ZmpxnEGuB0e9m1i7jRcOlPYo+q2Kfqj77IixGwoBJMCvtR5W/5QpyL6Mn6UL1WOxvFnOJdq/JLgbQbNCN23IFt+a06hIH54B/zjjQe0v0l3NN9KXm2hH3e6lGXnfqiDbxVBROKFSyRqEGMOAtDfOEyq2J/DEyeuJnSgE5c0kYjN/KyTdBb4ZbT9UUYd5Ibbr6NdXY75yiESwENDzwJNzibqCpQnCkKlgvlZ8bTkDBRGq5ImEr/POFQJGU3/ov/GP5t9v2L/1FYvTcYJZspS4Vva+GNGnliWBjty8IIuLoIGnzlQ2pM+VruC4Eban5Wv2D8BGhz9582d94w+YtDqsra42Io+I1b859lXSPsMMsFvT8ajOgy4d1q0Ck2+Tcsu4HW1VswqYKefeNp3eFWru/zAINTBsvjehOk3y6YcbqLeIpfDJfnCrag92Z760w+Cq/ntzgwexeefKJ4T5fBgnDYo6HO/u7yd/k5Fu4Ikw8meqOFdBNMDoDCKtl6bJGhi9qe6xyCO4rvMfNtwaryRsUeYbtw+gt57MQbkj3LeSRNY1wB5DjUDJvEQ/dxb4hQIOkUOSokT4PJVdBPo9h2ThmnVdQ6x2xXsHRBL9R52R8L7/kLUb6Tq31qCDOAj+og2FwGW2eInAAFYk4+j8tWzfUHf5cHS7eIzey2a9QWt3qDaCztvFE9bg4SrH0/hd0qiAyrcafgMjmfsXfZz4w1G+nUOUve5IV0ISjbSLgBpms5WLic4n+K71Kv05NrbzmEqjaQvnFCiamVPeOfAri1WXNbZ4rPLqSLx5+zbTYe9ztE5sgT+I6V86SOJN/OkRSx8V+zZZE03uwTz1UB8G20ho1ncynk/wz1Z9TXtaID7L1XPQbcHyxUouv0ais5DL4iT0fvuWnt5C4m0gjL6iAqwvM+M+gMdCuNYcyvY7h28bXS562PH8e61i/wcAjMMMiFjppWd3JfKTxUBr8XK6wKnc2gt2pnYQL0el/4+Bvlm+Vxv3LykjXsrKKHCfVFh8ekIV1zGb0MdSS2mY0xepA0fntMHNcGwo/1sKFYVkrO7jqbKUUwskCvLJzGuyr5tAa2+PaSxgDRjNh9h3YA++Bf4Qq5kndNXDp7ul/K9YGQ4JYPgoFxgRx5dwvP9dtIaZBXodpey/beYxjpqyZ6uMzhZNvhbKqe9jnvq6/r/ydknP2EPlZh7dhs8fpMfGIo97+PPSuT2fh2pVJJecrlT5wRwp0yYajEry2f1sA9peBQWCguaF7GPoVQ4DyihwhFywy3uuB2MQz4Wfzm9isyyUq38H3H3lwTAF195H1TcPEqZgVoq+gfufQRLG3vYfwElUWoQzDMXTbtLOM2dwoFfbxCza7Tw7ks4pbP7Ny+r33Ndq4+2pXRv44mQEWOSbtekESyPpaLsrZKHvMrZVMC/CKLn6F8O6NsBzibCsP0ClJOWE+h2A5TZCeBUHyk3yKarC7F4TtpOvR4W6xtXuiU8xYlThhZoGr2RXEodATbm6BdhCKvtTZTvObAFcrDc70llm5WqEkDg4rM4Z8s1BmK/FsAqg8Ja9ZwuWAGwPeh8EM4HPMsGH+dtZMze+yY/Y6iPygrn3Zk/cKKR69i2MNziTwhX1de/ExST4MpU4NYEdZb7M+1bDzPNYyLZeIYIIg5HZ8UjSYVer6nzPMb3riddyLgYPFBVmWt8hiraa5h374X0G9xk//bb9i9m5SOenYc+5yf62ClwuIIn2ieKW3iprPV1Jk1bFNPfQAHFJbt52CvgaZiVDNtaeFGxvlBJzOeuTo6E/R8jwcqLXxZi0Q8C7QEknYqjd6PP0rr09Ol570h2MitfiWMSdRo7l8ApbLP0SiKnqomOQink690sENW5wHQ7TXRhPeyz93Xpiu/jfgNjlloyJf3XP3iehJn8aO9GRrrZRaAZnZQC3okDOCSPPjdqjEnnP6iZf7Vojf+NaQzL6AP78NxG7QxgdurwTkivhvXxDYwMD+Zyq/forjei+RafZiXI+DtSDjrkNczp38Ke5pRrnUA66Ov5dI8+Ekrji96mGiYle8U++i/i8ZCkdP2QzgcUTSM9z8qVO47b+LgHpmMkXbqZZbECELbgXyvc4EljDFVcayoh3iDYr4w+v6ZAa5l67v1KqGgO/qCPOZ2Q/VxaJDvIV3Vn1Z8xv/KcdvWEjKodRbq+TY05IlCtTcOPgnf+QsbQtV30fFx8b4T5UxrANYME+RgVa7piF3QdQFEyupnAL9gG92H7KZHd7VS0lSyHlsryN7zuDQWYAkgVC8s8DTn98biPz8bbxx9CZalCwcYFc4pGb3JVLZKemS/xIFEGGVQqOv0JnySCoholUehj6yKMvhEMr62gTtsZjWpsrlEfI9BinUcRA1Ci8gBsO9BYbOvgilANPsy5kquNAd9OjKhEHrmQVWsrDT+Y+A0DMURrFrfWc4sq8smrHgVMToDiU7RCLzhxhs9mgBa2AFsctp4fNxpmp2+SmVfc0MJkVXbeacadvnJZ5YX4hO5MRAsp5AsHF82D+DyQmHfqPe24yzZyNCcl7nTuZvB/nKzxKg3kmWHW0rgrI9CQdRcIaIcak4IO30VBi0TsTDe6PYqjM+SzrseOHce8FqznH6p3FUKm5ogOkA+o4V1msWij+bw6sdPkdhRYed99leRb7teYy2uzYLl7cQ8qyL7TyyCScVqxe7F4J8KcePVFvYNebAsnUB7y5zh7RBUvI67yj4gL0VYfWvi6AuiTuGN6mOYdXoeKAPbucbdObhdzGraxO9x8EKyxlk/qB0MXA5dX1GT01OvFcTECi+SSWhX4bSw/2GlU46p3c3MzOpIpQvMFsV6mjiZIw9G0f5jG7UlazzHbFV0O3gzD4UIAo8ESxiRsYO4OQiujVYXVAP8oogYdwBjRwUh4xwOzYMcRUZ51sdE7x3JC0/fyziLhR1ANUgZflqIAvRuRW0vciD0WiTcpBSIi+nP3F43kt29939NXfr/EFR7v/8U/m32/Yv/MRgMBtq2bYvhwh2DJYyf6kVGkRt5U2gzBrClEZbXg8Mk57D8bmzLr/XwqsaekYSJLRkswerxujOimPBPgsgBTIGUGCOpcqmL7waDkUgD9LWC2fu5m8tZF+tgUyMMzdd4flslHHkBihXFOBAKtSGrtbPFLgZbJawbIZpAdWc8hhwdHuP5Cri+GFyK4uacPnNYXg/7depCxqXO0xB/OYFtZOtH/6YCVjeALhNsCsuGseljuTcYHg3XsKoCaH27UPNYFRkaZ1cI24VtCu7KpUtE4VS1KAa2JtMGZakwzS035Ey4+LIGZheDzqJhUXcxdHlJhG6rLBYBUq6B0YeVjQUMZhi2QbCtvC0I2t7LZ64UDiVBbKOiGWKJgKD2ws5QgTnVoTyo4TaCT4zI4sv9TtifqRA9THiwK36XAOx7ADaM0XgBBHvYK9waoHXVJpwOA9nVPsrCkg0dffLgGQ0xL792gG+MypzOVlWbiTAI61Nls6+5HHbdIhrKKtgqRNP1lEZDIyhDqeYxGUzkJOdwZMIxgpLDpXGLwUJTv12Mnv8W5gR5UxltDWRRDcyrC1Iz6+PGw1SbUCl7Q6cT31XhSqW9I8CL4XB1oIadRM0JyPlKFIC88pMc3d/gq/RXiR+7h7TWnmxXg97A6na1+AbU03/ievm7NIdQbYmnowXidHJ2mG94T16tgpcqoVmhVKN4LfwULbJWVChYKQoGRX8offwHRyQTb4KfYlWFPB3UZUGeQr1RvgPWDcdZ+JuyoLba0pXuuVDlhDqbLPkyl67ntXBIUOQ1AAwzVTEvDLpHa2RVxY2Fjk/98+ZR9RG61WziwyiY5pBVbOv3PMdlWSHcFv8+4zp5ZQw46gnPeoOVcWKe9EazrQ5/PexpcuHSyB77/wvxZWvPh9lLvw2/RB44ICaKQ8P/UCqMXt37KcHZ8I5Ltkopj7uC64rBrnpuoDq0L10ssFXLDvO/i+5visOoN3K+44rmHXwaDRmNss1QcPYHfBYNy6IawDsLxeXA9+DjvNamBwOSFI0SvVkcnNs98F+7ZoMFxhwWRcj0Wzz2fbVtH2TFH93otq8b10QpGMAt742Dj5/PvDqPC9aZAlUNutVNcMmXogijUgYCFK2BfQ+pVRQgmnnTnS0qPy9EDRL2igpbp5OJN/FkC1FJZbOcEpTMpVaYkShbIZM0XSjRYxSWOcD+5LuYVggfVQMmhdqmw6PCOk9h/wwIxWBzOcQorMNA7C1AsuTR6XUYnAbObGrPHb/JP/rXnekU5YjXrG6UPy+nKYTTDnCgZb1XIwo/Cqtj4M9Cm1ax2CcKOj0tvhcV0m+GLnJhv5Uth1uDhc1ybP1RjzGdTseXvn74+DRzIi6fqiav5lXpFk4kw4p6eFujr1VSLxqfFY2KDUrVEaEa1yIXVh0Sqo4uzyuVPLM5yi+xcCAJqLtAUaA3QPrNbGgSa4r3Wms48zmvFLXGbDcTuFaxX+v8LN99+AhvtXlb2Mp7IXPffGrTROakcv9x/FWx1u5UOJWAWKtBNP1UcDYJG0i9J3FBr9fjX+ePTxY4tt0kP+6Sz3jLmcqRJIivlou2twfYeScCZqu2qDUnGF7xMxYdlBkUBU4Q98/wHeocoL8B5ZkX8RnOChCZqgF1nmcOc3Mxb0eA3WZiR51B3jddQILwVcxzBpeDTmYY42NT2gnGt7uFzU0Q468xX4BwBNn/kDwHXwwuu1CoNZerC7rWKHY1wYwimH9WLiL6NpxmbxP4ZUG9v7wWG3QGxhaIhocSziYxN2tl23V9SZC+FNa9uF2woh38HA+77vAcU8QB/CP4p8LY44J0rII1WpwRoxVZ9aEtBLrfeoisOBUcjdrNjOTp2nmuYT3g8DOiAeoNey3dLRvpNeM3jqaqVa+DfKDOBYVOeb1rVfAt5WmQnwI+RsV5PneJ2Lv/ruG4dK5RUaOIFQDRkIgaImxZL4DNVktc1VY6WWCcUXEfmIPp6ZfN7Vd9xJhr5fnknBJFqUQH0fyvP6224QRhfb66r7B3VOHUB8Jm/5RM6MTlFAX0Do9CoOf9f67xeHfvu5mRIdtHt6nZTqtux9mYfEo+F/gls9Jp445X3mLSNDEPR0dH88KLL55vAqiwAxhnsTD/2bkcq5rJB3veoF5RJjue8yuPuXcy2V9eGwwGA1dHhXFjEPSJl11+avQBTPEX9o2WBq8fdlhP3rbFMdwPIkvXSI9tY89lnB+8Xw0VVoXTTtEfwsZ7eds/4zcuxKlF8Gtncb71hjUSrFFi3TqnnvXGxLNij+vj6SbSKlTMF/sS4dNQ2YFqTOQYIrYEUbtGp7YUPkf67qqhKDy9WPwunE1yrjWI67FG/vM1K24s9PlUNNizF3uOmYLF+b/d/dDuXvCVa4Mul5MldfC8qi5TtgP23CNckbrLbjhc8hl9ajrgRrFXdLtJrVjDMF+43qg1+SNsQtvdp272ORrgzOdqi2dnkyDENcnOcXo9BBucPBsGGX6yG9kPR3+gsu0Rgtucwe1tV2z046EyCNTDQ6rthdtFq8getDsD2wMHqd/Tpith+2z1GIj55NBTcl2zZJNwerKVM+f2q3nkuecYYDBQrdP9I/veqACFZU1Qe0FiPfayuj7tdjEsLIbY6p3/XF36/xC09n7/Kfzb7PsX/2NwuVyUl5d7BlYGtePaIuFbPMQqs9AfPltK/3yodKkZpWf1gYwp0D54U7YDvvUTNo4qHF8Aa4aIUOV/AHtoT9ofPcMNgdC7UbZJNDsbWBAOWxMgpMGrgeR2kmpy0+BusS/wxqkPBdv6wCNkH1UUst1u+MYCm/+Z9eh5NJX82QjxmoyP+ffg0XL4uAZCgrzk9E2ldMp8lj3N8K7CCggQjE5nk9KeIzH3E7YkiPwMlbXHtYbTLIiApxM0pBIlG2HHDeL9e4UiA5AySzAY+yv8233ioPA3YU+mwMg2k1hZDwYFW8UyeCVXtewDlGy/5e0Ea/n0l+rrNpjBGq6+ZhAWEsEdtK0hQeQ2tLvP82+mACb2e478gG60jlVkvBh9YcwRNXMN2OrwY34YvKaoaxLYGi7bCN0Waltx7rkb7FVKpQP2OlHY0VLUlm6Foy8KNrAXVrZ+ibsmf8ETU7+QC0vOJu5I6ki1ExosGsWKc8Vtheo1pv4ItwWDUadRgNxxg/i3q4YtzqkWJa4ivw6AtNmiCFeyyePPRlsl70XCVPt+nlsmM+vDmnJpyI7nod87UG6Sw1SSjs3li2j4rl7DFkFvvHjjP3mmIDRoHEJXJj/KwNBoPm+nUIHaa6BwFdTJxYY+qSOZ0XMOXWPlxxn1Rt43VGN/7mX2tJon/3acNu5fdQfh2fBYndxEvzBfTfldnWsq7dNoSugtojlbsUdp7bEj7RGG5sOrrrbyY02BwiJuh2KjHZzBc/URGHZcz49fJkvDA9rPortF5GsEN8gHRWPFHu4JgTuD1EXs5L5vQ5u7mRCj4QsVPVRkh+1UFD8vhqTpvBf/OHNKYb1Jnud02d9R8Ep7Tj5xUmaRW8JoGLaVw4GXUBysmA9Of0p+CjxfCT+T/M+u678J17iTrCWOdQkGQmzygS0vJo+Fdy/kl057lXuIojQDq2Lhth63SWNGvZEr/GFpDFgb5d9OZYen+aQGmvQa2UbF62BpsmB0/xOVBAjyirNR5EZ5o/VtPGxrw6wi2GmS75OaLq/wUx10NrsprD7jOeh2ioLTtquEctsbOj34RMOG8WIvosKaYWLN07KOvAAX7vvMoV0gRmx6nI2Kg2D5LnDWQ9/PpML/hUXuW1T1XHOIUMlPPCuYvN5oKhVZg8de+tNa7ULU5wr1R/5S7ULJjhths6yiCGp1LaEGmBWgaPbVZHJ31WdsTIDL7fvl5zT6iIJtyXplcV6v0/NtnbA316lY1y2vITV1zyFpOlz6o7CpU6HPR6JolTDB82GNRwiNFdeTUyWrMCsaK5g7dy5z587l3vtelsbTg+KY7A9DfDTm79CeMHSdsN5TYVqzuK7oIerxi2VVXQSfhl7FuAIY6gO+Ds97UIeO+ypc5JkbWedqlvep/qk8WwH7tNzzS7fiThe21vsOKiz4nY3gtrc4BCjgskPjWaH0VOB3EvmuFh4vQxQjL8DEbydyj38declgbPIq6VgisAzbzuhrVzDyOkXhM7ANBQfFm1q35yVp2FF9DH89LIiAYGuw/PhzRUctokfSNEHSGyoXbQHRaNk6Q2rsxtYV0K5rJtaH3uejZvX9fdZl4alyqPKVG9KXlC7h1mB4TcEVorGQrvW76GqBvGqNRkpAGoRrKOz/BpRnXoBVvXkjUmSq6r0/s4B0bH1WcyS0lOV+5fJvJ6wHES3bimeD5fsk/PAjHEiCb6PsHM6UyY2+Jl82pQazKUaD4Fp9VBA2+nyqbT2pgl+icDi5shIGr5bH295Nrzz4uhZy3fJZKLvHYrq31HKl9+xs4p5g8ZtNCNSwlcl8F37N0HZwSZ7RokYfL4/p9CKXGmSnCHMQDFkjmr7/JbjFPJWjsBoEKPpdnP1UOZ2Jk0WuXdoNalJlbRZ856vdNGv/kDh3Fq9XPPaUOL+pGrr2GgaWnuTdGx5n8wdqBce9wcLS3Zogz9/GlnN0beRQrCbF3ugcoUBDVUjSNJhwVqi3Vbhsk3Iu2VdZgC4TOuTAC3Z1k3bsqrHMnTuXoDr53GqqO4U7HRpSGml2KCb5yEtFHILWWf1c1pmW9fqIHYJIpLDNpnwnfKeuVd0V0EhVRiIPhbilKAUALvmKr5udNPg2KJolLkqc8FS5575pzn33nW8CjDDBl8AfwBfAcKuVAQYDjzz3HPc//DhrG7/gcP0sUhVEP5slhquL4KogA0+EeyrhXC4Xa8Jm0ykX+hhqpDNkCM3kOIR9o8ubkO+fwuc2Ydt+KEgmpfVu3M8HLUugcn8R3Ek0uqMGq91fXHZBvMlfKo81l4vvcO0wyPpEHr8IrLYylkSLmibeedmA6/QPGM0u+tylEWnQ4TERTaFFDogbD70+EBmjqvm5aA1kf+ap/P+7cDYKYm/Fbs+/W0IFAUl1FgFOGGLpkAvTimCuqtlXfRROviFqMhqktIS6QyyLBXelFxFVp2d7n5XcbOvMR/4ae8XKA7BxIhSsUo83FkLPdwQ53ftxh+aKmqHKzjj/F15N28pjoVBXJjuGOTZdSWunhVMr+1PjbWFr9OWlSog2ajT7Cn7lqdJX6GWFBoOGGr02E7I/gXwN144DjwnXju5eERVnvhRq8FH7Ibgjc+69l5z8fDIGD2bR33Qp+Nhgwj/6OvVgw1lRM6+SScMcfAIOz4Oxx7Trf/8LoLn3+w/h32bfv/gfg9vtJi8vT1KPBOphfjiM8JE3ndXN1dwdDPVpENJwShqf2kGopZTWdiBUSU6FhPocwls22P9QbmwymDDrYEoAJDjlylNgYw5XBcJpO9RbvQ4n1kg6FYWzsAoiVJftl8SKerihGB7MUVyXTidsps5qLAB/Bd8LrsdrMj5XkA/zCSPEx2uFcjYQW7mFZ7tO5q3Rb6mfe+haNXMHqIkdz5tVMNAHDApGqi7nGzCHYhy2Vv3c9hrBArdXqcf9UwQTV2VpojOIxlfn+fLYsVd4ofkXniqHkwGynY8LuCoAFkeB0a4owpyzHdBqGK8ZIgqjKutRgD1z4Kd4tbrU7YaDcwUT0hv2OgJCOhA/cj0pyWPVz30RGHQGoo0QqtUjqjoEsaO0rdxMQdq5Hq5m0WjRspLI+VocgH+V1UtOSziVLeujybvga6vkZcM+3m/VmoXD5QIjIOxBx55Qhgfvi7+Bu0vh4RCNA0LcOGh7r3a2WEC6UGBoqQmay8VhzMuexuhs4uYgeDgUxtfIXvwJJcsYe7gLA9cPRtcsb8Yaw/qyqBqcGrZ8f4nkmTC5RlM9Mqb344Q5KghpVjDnwnrCsI1izvFWxDQWUnboZSo3zaSh1PMQatQbeaUhgpero5l1n6JBlP8Tp2KrGO8HcwfOlYZjDz+COx0OJ0JIk6JJGdRezDVah6LoIRA7VmzmFZ+b3WlnbSNs0CnIBaFdRfC1im3tl8QzhaVsb4QTNXJzaWLrUSQFxVPoBLfCBma1sR2lDrjCT+O71BnEXJK/TD2ODhKnaNvyaeDbEyu4dd2zLKyCokA5T2qrqwsA4e5yoei9EHoTi3P284ojnYIOsrVSh7RJfFQNBQ74ZP8/Oxz/d2H1jeaZSx9kUIA/ncO9Dly1WTzrs58rnAGE1AQqVSnr7QEYdWCplxuzEUUr+D4GxvuD0SkfnpOCkqjt+jpnumgogWtOir3FsZf/mUoCRFNqr8LmByCwDS/mnOCLWlheckYaNob3ZlIhhGVDrbe1qd4Ml+eI349CpQYIW0BrlLZ69BwLtkZhTe1yws5bYVkbKPrDY98X6x/NVU89wmUvX4YpRZFbU3NSFF+M/jJD1xLOqX6/8Vv7d9l7k0IhVLFXNFWrj8pj8Oc+MH6CyPL0hq1KFDpOvq3OdTr0jCiCtL5DGooJiOHhELGuSKQGt5NQg546Sww6VRPSXie+67XDhGrYC0mlK5kTLP6/0kIx70dY3ho2azgH+KeK7OrsT9XjGnDpjaSO3UBh353k9pQbLbckd2WaP3SLUh/orXnf8V0MrInXWGvNQWId9dMo3v8VMt8VhKG/0XD2gN7E8nrwzYLTwf08hnQ6HYtdjSy48QO29tsqX3dAK54shzuC4WYFxwnTn39UHrbDesCEgj/VlN4I7QZ9Pxe/UYWi8YA5jalFML8Sz0Ki28Urzct4KhTyHMhFwLgxjG6qJLfvLppS5cbtQ6vuY/tlf3By1O/k+MrOBNmx0+mXB/c0tfIg4ZxHh4dFY7bri+r35Rsviq5aDitul7A/d3gqIZILcwh76zq2/dGD35DX6dWb5hBYtZt5FVDtKxf383p9zdRCmFaqmMci+pF2Bl6NgJlNcibl/wlonXmJGsxHtUaerIDYGK8mjykAd+SlxJ9IJ6o4Sl6zdHpmdr2FhZWgj5GLr/awvmxpFE4ktQrFS6ut4+lvqCI+up80BogzVvYnwg7wr7KSVDAHqwmAF0DlIBHpF0krE4z2BZP3Wuts4sqmLXwSBQ8mqZtP523qtRpIF4NOB91fFfdwO4XNYvYnsFsmA/0tOBuF0rF0k3r88DNwRHEuPYeESdB7kbJxQM434l9FzhUg1rR9D8BZxV5yWYt6TEWOtUTwEVMACK4KVj51Zfe3qWt1J/MGz5MHk6fBDDcBw/5Q30MZT4rPuqeGK4feINRDWvNF2XbYdIXUVNO1hModtUEBit+808akFqGh7bj8fbgvIIgqLdBDu4tCd5EGaaHry2L+1iLIBraBK0rFfOkNawSYQ2HffVDmqUCKdtcS1JRLTNYbgvTshR9ydtJ+bxemZLVD75312lzGTUGCZH3SK7LsXBOgdCgsiIWrQ8y83q4dAx97jJz8fObcK34L80bN4qMru2JWTP1OcxCf10IroxNThed1u91ucsqrCdLDa+ajUsP7Htt65obC/WXgUri37Cjcz+oGGL9StqFN7v8xixxJHE+C1rW7pXGC2opG+dC16vN869sgoLXI8fTG3vtg/UhB+lWRAwB+ihN1nUbPM2BdUzlXBogM8Y5j5HvM13SYxz6Zx4jek8jertjf+8aKmosW0SKsB7S6UW23CWIfuf0aaFIFRF8EJZtERMSYI8JWW4WsT2BlN0n9r3Kw8UDSFJiQL0jkzYpuYMU+bgmCy3wgwtsrWacjOrQtVf5tOGPQsEZvLhNz7Hp1M5LiNYIc5q0ebSwS92StWr1M8Xou94dh+fB+nTyfuICA9FwOdDqATi/vzU16E9cXQ1+VINE3nn3mNugMFmKsGnW0IS3zjEqZCsKBZeg6mWTV5i5BuLkgviQ6OpovvvySLTr937Lv3ehyYQ1/Wh7M/V4Qs/p+IXLdvRHcWRAMFdEp/5uguff7D+HfZt+/+L8LdWd4MgzuLYXXquSuw9tdJ3BNACKDQJGX1qXoW9zp8GuChmpKZxBsElXRFkTRdFKptmReC7VZ3BsMvXJhgVO2IrT5JnJbCVxeAM1WmcliMphYHath/xVzGWML4KMaKHJpLPjRwy7uJ34xGH0FO639IxKDPXH/HbjTYXd0hRQ8jG8iTHeRGJJGr2qNHBWfaNHcOvSMNJTa4XbcRj/+iAeLTeEfOWKH8EnXYqDHjYWEK8QmX8WMA1EgUU2uawYJ5o9qg+SbQLYxFj+fCPzN8j3mKvyNz6OFzaEBOYycwStF/mGfj9TXdM4GQmvBrssWbO6DT6rHDz8Nm6eIIuSFOPMl/NoJyrapH2ev+TP3RoFfgvOpdsE1cr9CWFtsGAfLNQ7WIBb6yv3qzZs5FKbUQXcFux0g/TbRrFDYWQXaSvk6GhaGg5/ZaxNkCoYuLzDAcZIpDRr3oG+s2FQqlG7+fnEM9YHHQjXsqNJmC0sdjbwTkmfA4N+0c+Ia8qDTPOmAYLNGE5oFd5XAUrfMfq+IvYLsTodp7nACHx85C6gifjJBevglvEr9un+F9aOFX7538/Q8XHBllci0UqFkIxx4RNos1+26m/BDDxKS9xUHsj3JB8HWYA4NP8g9j9/Cd30UzZCAVnxeZ6KDGXr5KvIzWix6QgxgUFg8A2JT232hxnsCUq4SBzYFGWRkoA/bRzzM/8feW8dHdW3v/++xuLsnkAR3l0JxaQul7u69vXV3b29dbt1vvdACpVSQlgpWnOAJEEIS4u7JzPz+WDMkmbN3gFB6P7/v5Xm9+sq9s5mZc+acs/faaz3ree4fa/QcAcT4eoLCzxJocMLIXLi/WCH3s+Eu7vPJZVY+1Cl8WrKqizgpH87T6Wn4REkg7fby9ETm61CfLwH1EaCxoYR0G5yZPo1nJhu7N0r9I9gx+WdMpy5hS53H5s3p4M+9PzFn80dsKzYWU+LTzuPGYjg/EIY6j3CTebSo2Aqx06WrIMyjiNlSS5+WShL/fQUPLDlFWSx5vTGGE3whttCYEAuo2gxA6G5oUMjzeu14jvHVv9E1VDMfpF4hvqRj5x1ZlwTAgGdlE6sp4qSHSbLu6kHGDs8Q72ACbbI5NXSamVzyQSdvhelGaRpA5I/8k/T34MkZkqzr97Bi0AlZb4pHrWdHwYoLSC3rzh0hN/PtbgVZKXGWSJ5FKjoVTCbSkqcyzaeegQ2KIuXqK6WoumSc+C95wi8JzmmAE+ZI54onQvoIKaJwqSRoFd9PcC/ocqlhyPrzZKKtcFGh4vcO7sFXaU8TuOUAN24yEj0oWyvF/eDeSmJN1+r1vBAJH0eDWdXx5U7k6mTCHc1CDCs3KlAAcn99ZjIkEQ/492Zm7FYuu/J7vo4ySl2Naszgnt39uPnj7tRnGjsZmoL7sadZZJ2Uxb5D4Y9zhQyk61icvlFYzQ2KLs0OEGMvYbhru+B5XO754alwWJWojhHMwKVBQlozIKQ3pkyRqt/jpSliLpsGc40yWAex6zVYpWZUxwdK8eau0Z7JTxNNTjMvVsCoXHB4q5NiZx6AkzYZ48WZ5V/zw6V/MGfEcppsIYZxi9nCigb4rbmTCRx7oyQKizWx6vD3JZ702NPsiO5F4cBN1LaYMdmN1yKhYD6vuHLEqmsVmDCNK2b9xJ2nLTR+p9lGQQt8UgVrG/9mqaUBT3N7hew1mu3GPYXFz4Lppce4+toFymfnwvpfqHHCZl9jod2/502ckCse4y1eRsWEHFMwL5fDTKUWMuIFP2WVdJodSaLK3iBe19ma5OmBRXwaI12vV9iM2c+Yul18Ee/Lwnjwa/Ig0FoD+Cz6egLNcE6jxhNt2Nsw/ieI1hT7Fo+Ree6AouuwI1RsgexPRCGiM3DvVXV7hp4ugoqq6JD3Pay81GC3cRDxp8Dw92CIhny7/VnpNut+o3HMLWeqImxavEhMkG6qrt3VbcyxsSdybpAZv0oNYehosO8rmN9Fb2OS9baQXOrbeyxanA5O9IVh3uCH4v522umZILmHbbnGNc0vrN/B+Vup4mOvF4sHjUIQ/kmy5whIUY+Xb1R35gEEpgl50SfKSFAf/i43B17BSSVhfFptfCafX3YPPe5/k/Nu/wyT514pMJ39Dh/OCYS5ZxqJMTExMWwYARuuhku+upU127Zx3/33ExPjIok1V2PK/gRTxUblYbvXzO4lKSLT6IHXN92LvwmuKgTi2vvQrbam8VYl3DriVmMMXfgrWclwpqbpyStiCO/k76PRiVrO8nAw7E0YoshRxE6DHreJbYxCpQqQPRgYrlW9dyymTHi43Eqkv7GA+UiZF+N/7sJH65JpUMmPttTCz1PhO43MPci9v/0F9VjimVLEVEnRd4SKzbDrVSHyecawDSVyTDtfFsUbj/yIqS6H64Lh61hYqAptrP5yXy9IF795T2x/jhkBELgbfOKNhOSGolVs2PUVC3fOUx97zETJKSkI3jI+SfydvcKMr59WIH6ZKjnXvg/wWMiVLKuHkWnGou+c0NP5oSCAHuG1+Ds95OWaq1ibbOGCQBjYXaHCFjqAD+jGh5GNTKjVkI28QuDk7eLjqUL4MJGBbvaQiw3uJd2Yez9qRzyPiYkhNnY6U6BD+d5pZm+69XgGHx+Fw5/ZJveWf5LBGxqQwm6/x6Bq25HbYx3HX4bjxb7j+L+FhkKuCZYb87oTjey2a+OSGOgDCyLOIzTKyJaq9RW2Z5RFs3Hp9zCc1ywGriqYLSKxqJq0OkLtXh4Ml1b9miZj56DdO4o3KiGjSSF7Y2/kDN8mMpvhI6OktxwWEGACk1NRXAKRuZrWyc2H0ykbjP7GDg2La2NZ2uKk0e7BbDOZ5L/sT9ReViDSAbYgLevopwZv/lEEDsUmFK9QWRwU0o4HETpQCn5mRbIu8w343Cz/eSJmsrDqVN11yedwR1UoF/hWMdrLmOQzuzzrkveCl58mgRM2UN+yPu5HWax14yPeh/TrWrtM28JkEkkHaPU9cSNytPwe259VJ7xMusKOYKvdh3XGupKgbWearsOobp9IjzgVDEiTSbztVP6JIM/jlBVSGPPACZn3c24gnOQP/p4eRVZf6PZPSLkIwjXsyQ7QPziK1ythyP4OpAxmB8IyTadk9W6YlwjbNeSAzDeUfgxeVl/KHfBqJSyxhxnGIxJPorn/dp6893MCTcb5pFdkL3rFDCY1UtNxeCgU/SobBJ2G+pwwKSir0FwlgfroLw3G9MtagnimDFL2QlnQAMNbAw9cSmJ6DieeqvA4DBvMw5UBPBwOQTn/MY53/yfD64YRvxeq/RWSPBUZUvjXSXdUZwlLPW6asvAbvfdthu95mhMyjP6i1O4TiYxiRRBe9Ac7k+EszSbUET6U9yuh3qlmPMZ4eVPh6EAKbvMDIrGo8wdtKBZ5D9Vz1wG61GxmVwrM5kfpJvaAxdfCF6N/47rITPZYPKRkmyv5oGU+NWmQuEedWPI3w71hMNCp8cw5RmhZOp49P5zII8sULMTQflx64FQAShutysTpyqIdXF4IL+TnGcaKezyAXxZUOCAhyEhCMeV8Sd+ir9myQpFMA9k0R46ChFOPvEsiYph4w6j8Kjfdz67wTJzp0LNEkcheNIqqlDpODwCLZ+HK6ZR5rL4D94boEzsvwWKywEmbhZXqmWSMGEUmIcRXrqGqSuGFYvUXVvV8jZR3fYF0cOxWEGvc62djsUhiGo7LJAnp+jy1x5HJLAWHwa+o44s+90uB1MeYRLH6xeGfdDr3n/oNvgqvNbe3pzKBGNAV+j0uRAuFDKLXqA9YWAsXBoHZoTju/k9KgbTbP41jIGuSNVCSVyq4u3g9ujxNJhNhpWH8umgIfx4wEtYK485g7huns2dFH35ctsw47p1IajbcW6oh1hwKOV9KAkrn6RfaH84ogR6azm4NplX+yKpEmO4Hfi0V7cZMJhMZSdKBH2NRFCnLNvBTPJx3QDqnjhhNFdBUqVdLaCgRstuAp5VSsqfWLsOZDteWfdxejcRk4uSKhIMezIbfu3wjj4dDDxvYFR3Gu8yRvPT2ScyafQYmRYHHp7GISX6wp2iD2s8qd4EUUvYo1nEQu4TVl6v9wQBSL5M42EPqPjZtFAV2GLpyFBc6jFKbP/mP4aEyWBYPMZXG/dB3O+Zy8idTefL3J43f2VJLmg1uKYGPav5+X5Uz/e0siQdztUd3dMUWbF968Ug4jPJRd/N2a97PUG91Ed1msR0kgqieu+VJN3JzCdh1stJeIXK95h1hx21TJfx+mrD+VynkofN/5PxAWNcAOXYFOXf97Qy21XNpgZDj2sFsZZclktMOwDWFGiJe2lV6X1JoLSTrOr8XnyD3sOfe1t1ddqTJczdsITI3q/Z3IHtH/2TxlfJEZYYkbH+ZopR4JrCb7GtV7wXxLWsqU8vrnrRJCDuq7n2nk6Ckeh78+BF6na5JRNfuleS/whudwl/kt/xhoOa9++DrKNiokM4D6XCvzYYMDQG2dLV4o3sQXCz2epYlwOok+D5RsYZbvPln0mbmXv8mX/gblaLCfMP47PTP+OrMr/CxKu7Rgp+loKAj1qy4EJaMV4+B7Fd+GiYENRW6XAinFxqUY7YXb+fl9e/xQ3kZpU4jyfulgHymxtbwZve7CQ3yuNZmC1udQXTzghG71Z3XF8Sm09sLxndRHLvdlSRIV+yTAN/6feSkwCyy+T7LSGb6KC6ApQlQkXAmhA5oN/ardz/y7Gauq12IxVN62mwj0AyzY+EZxaP32p+vsaYR+ufA3iCF1HLR73IPfhWgVqCoyJB9VJCiqJZyLgx6Tk0Kc+PMclHL8ZDkjgqI5tXpr7J+zPlGLzXAZLcw7r1L2PvCpVSGK7qrN90PBYv0+7vMN+Q+2qCJ58bMhhk71bK/HaHrpTDTtcZ6kguczUIqj50qhPyQ9nmAkIZcXo+C0wNc8qWeaK6W/XjqlVJI9UT6NdxQHoAd9brUY+157EqBFVEdELuGvgYzjc80IGSjLY9Cnkcx2myReWb5uWqpTK9QKs1+RFnAR5GLtZgspGamE7m5N/6eDQxOJ9EWJ8k2ODdN3WzSZPbh5XLI9dbIn+f/IM0AvoqiG4ja0jdRrfYzbZH7rVh9tMmnvvj889QULOYyYCwwlfbyvZOA0UC3QVP48Z/v8OUlinxTwqkw+Q9XkVGTT9r7Efw8WU/UO45jjuPFvuP4ryIw0KOoFjqQ/3R9ipbEMxkQaewiaul+Mzc4R7O1upjEYOPmY5WtG6ZMGLhXU7Wo2iXSTh6yCAex42UXq1mjsa5D+Aj67YNCO5waZwwWLGYLpwfAlhQvRvp6PHbN1bwSXEKpHR5X6VsXr+SneKhOg0stqv5vRNf9gCbJfSjU7oUvrEq/k+Yx32DKhCH7Yb9nV6G9SZj6J3wt3U0q/HmlMBm7K9g7mx/iX8FVvF+FWp6jKlMM0n+Zrv7sunxhqwx8DmyKTLu7i8azuwMkcZm3APLVx+1nr+b6wEa8Kzcbxiy97yFn5Dx+umozvl6KjVHWO8Ky0pl6B/eAvg/pk6fe4SJnknSGejykD5xVKQVez9ejJ8iGTiVVa/WFcxpl86DAv5rTWNkgRusGhA8TlmxAml5+zi9JGNkqeRmHHUrX6s2tO0BD6jW87OjJnd6n4GdT/N5WPxj1H0hTBDiHQETWq/wYD19dvFz9D7Y/L+zNSI3EUfEf4nunC3KGvS3dPB6JdB+Thb5eEGeB3/YZZVgi/SMpi5nB5DyoVjDryXyLPhHdiDlZM48dCkP+LabeJRo+V8r5srlQbUKrs2DVpeJF6PHc7Q8czF2lsK8FnIqNkW/Nai5/+H029lB02gC59eVcVADfNqsTKX/myfk22hWVsYZiKWDqpDsqt4uEUalC4gUkee8br75/63JF0kQVRJss2IE7QiVhbBhOuYB3q0TmzVRvLHyNatxKZgqcE68hoESOhbRr9J3bfe4TKcKCJepxDap8k3jCvd4onunpdcv513N3ctG804wJRrM3C+yy4RjVrNg4F/3By5FwzgH4yK7ZuBwj5MVfwBfFB9i49mG25hqfa0eIgxUjV/Dh2KXK5Of1wTDcB1bUG5OIFps/sVaYFeRLlJdxovwy5jr2NMOMFk3ypqFIJKd0Es6HwvgfYIjRw4XgXmQ6A8lsghqTYgJ3KRl8HQu2JkWQsSANFvbS30ODX5XEu66bIfsLWHGxen43mWSti5lw8Nlyx32NqVfzalEFX8WCX1228b0t9SKLnXCa8muLf5R5+dIDChJU6uUiG51+nYFBDkiH2x9nwfxkSfKovtviK5tZlVQhSJLlV4WsU3AvfA8s5LS4nsZ7rKmC8oyncabDiRWKrhL/JHmmNQxyU0Aa/7KMYFz9cCyqTb9vrCR8dZ6tvrHy2YoiJADn1MNp+dC1fYI+pLmE63+eSv2XpzB7wwDD25qC+zHv1Hns7LaT7VHGrWV0gNF79ogwZRWM+lRiERVKVuu7JDpC+vX8WAvfx0NMbftEtQkTJXZ4tQJSshWFlpZqhvha8DXD3PMVXZr1hcyLhdkxEGNWJOtqc6B8PURPVB9b7V7Y+RJofFW8XQmnMHulQWrWYrYwyQ/m9RlGhM1jLa7Yyn1h8G0cXBJoLOYttXan4reh9NrSl7Qa4zMfWfY7i+OhIlXdiXZwPdElKEMGyLPV9yH1+B/nwFpjsfrMXmcy5dYpTLp2Dl1txpirwBbDmgbo6QX+CtWN87ddRXM6zDErVCDKN7MpWXw2n52skYT/C2DY8wLseJEXQqoZ6A3NTR7nZQvmV0ckZ7tkmFVr1r9jbqHBCROaFYWrPR+xKzyTyq4QWG0sxGQUZXB5EPQs00gR2puk6yv5XL13qQpewaJIkHy+JHU90esOXigX79EfFYQ3e58HObPAzEfV4GwjhwuA00molz+/18PcSk1hq2oXLB4Le43eSoDImp1VrS1aUOyKHTz31X4JUhQ748g6iA/CFiT2AIFGBSBACiAz90L6Ncax7jeLnHx1prrbvDZbipQ6csz0TTKPHqn8v6ORkatu4403Z/DeL0Z1CkA6dEIHqG0z3HFD+Ub1e/d9IaScvR+px3vfK//1ulc9PvgVZVzktPpyQxFsbbZhiVcQNk1mqvYmEL9mEOZGdTr0vKL3OKtKQaAC2WfbG9RkIBApwaJl0pGpQpKLULn/G+NYzR7p4lS8d8Xm15nlLwRws0J1ZmlLCG9WwoiEEUqZ5WccvXm9Ag4EqkkmnwRksiUZLCovY+9wyT8M0nSSmWzktcBgb8jbb4xv/jR35bEyGJ9iLCS+NeMtPph4H2nN+4z3d+Qoeu2D+TWwQ5FaObX8G1rSwAac1fss4z9wE6NaatXPTsES+PMa2deqULlNpOgzFURVkByWqknA0cwNST3om/8f2GO8v0NqheFfG1CjJn9FjpY5aoqmg9lkE1LcSEUn8NHA6i/z/dIJsMuD0OkbK36nGpnuhpABjM+F2D0Qr7K/zVso+42WWuiqIPxEjWVBRQ1T/KCqKtswvDLsFLY0wtdVmlxUU7lcz1pNvtQ3VhR+POOu5irZow16Ue0d3VhG96Zs8rvCiXXG3MtI5wHot50difuo8UtpP+gVzCXOcXS3wdQdNxs/u3IbZzSs5ptaWGPSKD2svwVWX6bPN8137bU9SS7rb5Pi5vD3JU8HbN68mbvuuIMfHI28BOxDCn4vAdcDLwPjgW+BjesXYPGqwK+LRhGv6HexDlDZC+UtlFxr7/v1Rcr/EShjv78Jx4t9x/Ffg8ViITU1FYulzQJn8eLiPufyUsMcJlUrpBH84tmat5xxlUtoqc42DJ+QJHJPM7ppjFsLf5bOpxJNct9tMLr3w8M/EQBbAKlxo5kfB7Mwrm5xjgq+joXetibjBtgWxJqud7PUksqVAxWboqrtTPKDAy2w3a5J0Gx7SnxYOgO3HKrC86+tjKUh0dFUDkvHw6Lhei39+BmQcqFyqKq+BG9nC2bE68iA30+Xv1007N+yddJ9pGLtg3TwnO+EaYrkvlsWSdVxWPQbs3xqGbkfssIVnmb+SSRF9qWHuVa9adr1mrCslmoYfdmfKWVND6LgZwkodUzbhhLpxlDJrfR9SIyafTWG4RYvdTEDuH3U7XwQbeL1VMUm1GSWAGn8jyKtpsKme2G3sYsNAHuteCvqCpwb7pIi+wpj51/YoCe56cJtzD1vgcL/yAkLusFaTSfNIWBOmAndb6JHsCa4qtgsRZaed6jH3exDVWAIsmFbf4v4R7VB37A4NidDXld4R3U5Nt7Dk47fmT7kHoYnTzCOl64SbXmHptP3UAgfBv2fEqkYFQa9CAd+VPtO+iXBkNekCOVxj9osNkLM0N0GVo+uE7vDTrcD4Zgy4TJVvbl0Df+JFl/Tm9Z+YRzf+wm/JsC1wRBpUSSdYiZA1DijNIcbUSeIhNHmB9Tj0ePgtFyYqEh6hQ6STW6dgpkWOZLh+2GoD5yh4ByYTCbG+8LTERDoMHb21QTJhvt1H4VsCMgzs+dDo2yvG/ZGyHpLLyWkQbV/KveXikTRE3XGRHyJI5L6Gj9SN/cltNajGGL14yVnL0J2Q899hrdCzR7OC4Q6J5Q6j1Cu8ihRnHQBtQ5RxnN4Fp+aaxgYkk/2KYvYk7pH2SUxyQ/OC2jtvmqLZJuJrcMnMTe6HmqM6/yakt3MzIeZeZq5O+87WDQS5oQemSQayJrxuUXd2RQ/k+XV1TxeBq+XGqVSHP0eof8+6UAqc3p0eJtM0tnfVK5MsANSdNh4t7qzFWDNtZD9sUi2Kd+fI8mS5pp2cZ/VbOWnOjmuUpuiY7F0tchkabogPquGB0rhoz2K4wobJPKiQ1+XAponmquhwCXLq+qard0Hi0crO7MBIYTt/8bgxwoIkcLRCAsVBfz6A/zTJNdQ+2RkfwbzkqUw7AFHYxlfnPIqn531NQHeis1jUyX8NFw2+CoknwO97tKTkSw+stZ7xBf9fCwMCZfEemVypeFtw+KHkXRuEp+f/znWCOOZhdXuwpkO9u4dsOM7QsRwIaHoOpyXTZcuCRVrvwN06XcbDHuTxRHnEpMyq/1X+kXgNXU5N7pynobOqaixbBi1iEtPfJ4hQYrjairj1AA4MxDGhigILP5JMPoLUadQITBdpNAST1cO+4/5DJ893vQtDDf4dF3iW8UbkXBq458EmDzmsvhT+KrLo6R7wVQfhV+2ycQfN7xF4h3vUWszHndhQB+qHfB7PWrPvsRZcEapFNxViD8JzqmTzhUVcr6SBKNHfN24+jF4cBOv7IzkKYvRT9tsMrOsHp6Kupn+I4zS1LsCB1Fph7nVirjdP5GvSeXyIOhW1UkS1SGg3PMC7P8aH59wlg35ml7dLzAc13OW4cx2hQ4G32rgxmHXMSPAwpjgEMOY3SXrX2iHFpPxuTyhZiXvRcNtavEVyF8oMZMuxtXB4gNdLoDRn0qXiGHcj1tDYUOS2rOvIGggX1fLfRsX6BGf2+u5KftO1vXrz/LLNXv5H/qLL57O4sFsEbKarlvn3BaYkalUGzkqNFdIZ94uBWkHRKp4gaYQaPGGoW8IaVNlb5FrlE1sh9p9olSisoj43Ka1eMDsxZLKBIqXDyIgQ7FOg5Bqo05U70vTrpZ96Tka+YqYKbJOT9+oHvcOh/5PQJxiPw6tCiMeijYmiw+vVcKU0kjoqV4PR+3vwrA1w4io0eylCpfqi5CJpwsJVteZPWau2FPovI573SUy4qoCaUOJfG/Gg1KAaINhNSuZGycE8H5VRmL6N/ZYvq6BiNLflTmKHSU7+EcxDP7NKOMJsN0uxxtQqyh8mczSua2SmQWa/BIZmSvKA2Mr2++lLBYL6/26s6AGLsh+CLLebf/mDXexdueXXOd/KaUW44QUE96bWQfgV4tRntEWPpSNjXBfdCAhDUZVDmImSj7o3GZ1HsK9xq5TKAPsfEXUfZx2vURo7rfSZefufHSjpUa6msKGiiyrB/o7NzLz6nmc8I+vCKrcaPzcpDPl2fDW7GvTrhSLFN1a6rZuUdmrdISmCvEf7PeYfu4vWy/X0OOzHV6hLKuHAjtMT1OQ9d3Kavs0+wVgpj/8EA+F+cb9+KrgCfTNgZsVzc0AlG+W3/x3NUmQ/B8kX+VZNCvfCL/NFKunYIWP9/ZnuKrmW76rhTeyjTL4Jzt2kpqXQOyWvkzodqpxPP1kfrOHsspL0T1an89kcujpBU8vf1p93H1cc8EmjdVI4hnQ5yEhmLdFcG8hBMfPAFsALz7/PIMGDGC008lw1z+JAe4D1gC7XH/vA04CxlnNfJA1Q3xsPVGXL+tOzBTx5/NE2Xr5XZPO1PtK/g9AG/v9TThe7DuO/xocDgcFBQU4HG02ovYGftz8LmvMCUpmvdneyDQ/6aJoqTcu6KkNmdj7xTNv1AWGMUD0p0E9KYHIK/Z9WM1E7Aj2JiK9/LizBLb7GQOJpOAU0U0e9paxS8jixdARTzFn8Mm8E6vYeCSfS8weYRYv0Xlk9L5fv7E+FHyiZBL2NW4ezDlz2DJ4ND8MmkRqgEcyo21Qr+vYSr1CAvVCI5O2pNttvFUJ65JMBNqNySO6/UOKCiqDdJDNRdxJYhCrS1wdCh4yEnJgq7jAVoi3CT7ZoWjltzeI3MiikWqGzQlfikxorGZj8ue1EsAX/Kwe3/KoJE/3qQNx5sbA932MyebyzeKrV7LqyP2ggHNtxVijRtNznIIJW50lfoALNMUhEKZf9Hj1tTD7wIB/QZKCcQcQ5AqsihUb9+3PSbCqkg81maTgtetVSZAeKRJmija7Tg9/5EdwbqO+syPtapiRpS9iBveU3yW0/XxjN/vygCsH4WM2LsP2pip8WiqYHGBjWLyik2HY2zDsHdj/te7MOkZNtsgJekiOHITFVz6/i0J21SdCdPqXjGklR7jQv+IXylNhRwqE1bUvEJlMJnaXK7xb3KjL5ZxASLZBi6qIXvgzY33hjShI9dYETBOXwuka2UivUJG57YxflNUXJv4Kk4xdmAC1DhicA4+p9lPZnzPez8TUPOiVOsswXB0ykOfL4evGYON7QRJSMZP116pmj3Rq9jPKMHcES5tNa161cS3d6j+A5sBqhk1ag7+nZ47r/ZUOKFOpaKWcj38WbGwEH4dOn/TYwGKyML8Wzi+AJk9/3Lpc7m5cw81vXMdTq6cpuySuK4JaJ9wcaOwYsO58GZ/CJeQkX0mdoqgc6aimyQmbdMtR+DApRgd1P7IuCWjdIOcrOohqs7k0CD6KgY0FGw3D5pw5nBkA39RCcICCCDLwX2LuPsCYIAdkM9f10lYmuidGfy6ekaoOOhDG6cLesOPFdnGfJfsTbguV46o1K5KE/iniJazqzkf8Tl+tAP9OqEJi9Rd274TFasapT5T4UWx9XL2m7Z8jf0criAmVrg6ayDHGMb9EHjWNID0b7lLloZ1OWHGBi1hgnKvKV1xK3JKh3PvZCDXZaOOdIqddqilYNFdJIkRF5ABh6e7+wBDbpPW4nC4vzqD7Rx/yz6HGZ8O26W7mO77lthAIcCi6bVxJd3Nn54Pa/ZJY0CHZ5SeZp4gTDoFp/a9h8pTPSU1qz4i2WWwMiRvCJ6MuZ96wM5SFrXGJIxm74zYishRePwGpXFPfnbMPQE2Awu/YK0TiOB1JyitEEr4/DVcOmzDRaG+ktslY4L/Ht4hyB/zZ9w2j9K9XMBVe0STshddNAwzvPdG+l/N6FnB9wH6K/dIN45U+SQTthrG5Ck9KN3TJycNBssLTBrCEplPXHECL3awkapxb+S32NHBoiHKzg2YQsgcuVZGN/BL4ztKdPt4QUXdkBePDhXLPCzB2PrZTtnN6T3VRN8QnhGHeMGf05UbfaiCgdjfmyb9jVfiEN/R9Et8s6LYPKnyMpIfRDRsos8N0RX4ckDk47Wq9/1FnYTLxY+Bktpqj+WzUpYZhi9nCLSGwNwVSLJ4WElZIPo/+9ZsYtUVDUAkfDl0v18v67v4AfjlJEtoqmC1ChvMkRzaUyH7kT6O/+GHBXQzQzYMh/YVE6F5D2n13sXQCOxrU8YNvnBDT3FYPnqjJkjxEmEJOsyMZeJOZ1yLj2DN4PbuHabr7R3wE6f9Q74ltQVKc1O2jwgZK95KGiEpNtsQgOj/WgqXw8yRRWml72K45Ir86n3/9oe5CenDsTrr22c2wVI0O/4Bn9B6IldtlL77rdfV47GSxp/CQ4TwIrxBR8FHZyoQNFvnmsnWGe2FL4EgeKxWyU4WXMXYxm8zcGAJDdj9Jg2enMPD76HP4ZzCkhqmf6ZNz6jj3AHyeq+gUdrSIHcefVyvfG+kna80/imBzYPt1y+FwsKt4Fw6goqXFqFZSvpHIul28ue5N6ls8uu/q8vh3l64M9IaRiSMN39uQdh33lsJDQdUiMamD2areA/onSwyritkqt0PDAbHK0SkI/XYqrLneWFSz+EmBfuCzEGRcS7ODurFn8EZO7bcf/zpVGxyS29ipIQeAyPbqpHvdONL9Rs5XQuQPH2b0PbU3SXFzy+Pw51WtfoUuFNfKczraB17tp5BSDu7Veg12vGgcX3crr0bB9UUQEzvWMKy7bw8iyEWW0CmR+MaJ2olnjBKQKkTnqBPU74s6kcVNAdxVAuVhxnvwa7/xzD7vR8J/CMDHc65zNHOKrYyfKsq5KFsRx0adSPdsmOEP5wdopKkTThUJ6CiND+3Q19S+6amXw9i54BPBi88/z1P33UcPp5PDzXJf3ORg3nyN2l3dfvFkj50M4UYCFj1vhzPLRGXlfxja2O9vwvFi33H81+B0OikoKGjP6qvcxrTsJ5hblMvcWmNC17L1Me4Kgwm5sLJKYXBX+ifm+jzMG25Xf2nCLCm46ZhY/knSHRVxhP5fpX/ytnMxyaHpOKKMixPBPTixOIKU755kbb5GRq74NyhSyMtY/Si0Q5NT4/EC0P8x2ZB1FmeUikSYJ7Y+Se+q5Uyrdmn9exwXE10Fq5WXqj+3sRQ23680QXcnCbRTX/q10O16/TH7xUkCP38heJpQA5Rvkk3ZOkWx8NR9cMpO8FVIS0VLR94XMVBaoShObH5QAqH0f6gZe0HdYcIi6KvxF+jv8gtpMXb5AGIsDGqDYJBCdF2ueIi1hb1BunvW3XjkXSMgnaF1+9XyYbXZrf+7WiNzUbpGuj9U52Xxgl53QpxGkjX1MjjPodZY3+DqqvtNIZcGMNlVINRJUh4K8TNEJlGHNTcIY0+Flnq5F8o3qcdXXyVdV37tE+xNePF4mXRVPdpkDFxr4uVc+2Y+Sku1opCe+aYE2asu1R93R1h5kchS6rDuRmEgRyk2PiCyTl0vM3RZBLTIxvKtSmjwav9smU1mpvrJc/WSSqUz4VSi9ogfwwMhChnmwS8xtTCQU/KhzltRsGiqEFnhek1C2OmQzfMMTSJv+XkyX+xUbOztTbIxDzRu1qjL5eZQMy1OsAUpiuFl65jk52RHE7QopDg3FWzi9hK4ZL+iQwikm8rqJxK9KmR/KhJKR5hEj67bxeJ4+DIGerXkGsZvOOEG3rj3bc4e/QOFoR4EFUcz95q38GyEdHEaYLYSbIb9XeB67w684I4Bum65g5tD4PNqcHp2nPvGUpf2JkW50fDbIOX7z+hzAXktEB3U1TiYMJNHKv3pvfRd9tRWGIavaVzG7wngrdtXh/SBM0vFU+1IkXSmFJ+KfzeOtfEdmjPuFuP48nN4IBw+m/G6Uv4cgPiTIUHD4M3+ROZ91XoJMq+P/kydRARIPFOSHuFD28d9hb9wdbBILzlVa3hAivx1+8h5wGq2si8FvjFayEk34aLRLu+wD43jFm8hEsVopGm8w6D7TfrCbLcbYOIvau8mR5MkjyYtM47ZArji5DmkJk1lwXkaAktQd5dvr5HoURUir30YlAstihg4eoIkBMYp4jkQGWPvcH3CYMlY8VPz7BLxCqXfoCs4N9yP07wVCV2/BJ664h4Cb34Yv22K44o/WZj153ciNgGRppsXr5ea7fuwFF5Vcm0doPqPi6iY30NZJAfwynqLC4rf59SK+dgsHpNdQzE7Nz/LAnscbxZXGN9s8aJ3n+tJ6nsbiZED1AeQv7BDhjvBfbXrsF/pCl6OhDP9mqC5fdx1fVNvLi6AT3O3U+9Z/LI30FSdTZkdypzGNEBo9RauCAYvE8QGGh+uIXGS1DGbzJ30X5wtz6VKAhdEov7sOkMXxaulB/g9Kovum/txdoxxfs73imNVAwxs2qnsvNbun1z4eOcPBO+G6w8cInHaSSj3vCDPY2OJq1PYg/jYUMzlLetZnQQzijVSbUtOhMXq7meLzY8G19f52ozKMG+FnMPAHNihE4kIGyidAVka+Todmqvhmxi5zpsVcq1bHmda9WJ6Owrp5jTGPqFrruCFSCi2g8MzVWXxkvUm7Rr9Xn7SMlAUPw9i411w4Ae19BiIV9++L42S282u/3+kv8dBmOS+1nU5pF0peyjVPLd/jnQv73hZTUDpejFMX6f/TTbeLeRIFenyjBI4Ve+pNC6mKx/d+i0X9dFUhQsWi5SbipTZVCm/c0edhwU/i2+aCllvi+e7ey/oCXe+x4O8G+7tz9ZkE850CM9+1/g+oKKfA58bPqY6VENk7XWHEI9V8ImCvo8YiyFu5M6H3e+rx0A8/X5WFENAis2JZ8AJX0lxpA0K/VJ5sAwuKIBcfyNR9UprHmN9hfBWZzcWcWMK5vNKFFzuU6H86rDIwXxZAzP6K3JKLTWAU4rKCiT7BvBFzwFkNEKOX/siptPp5HT7Jl6OhJRdFQYJx88iL2FqHkzzA4unVUPNXsaVL2B9EgyyG+9BX5svfzbAmP2QYVUoRdXli4JQ1tt61aTRn6nlSYe9IfLmXhpCJkihMHSg8d9YvCWXFTpA8gUeKPBO4sfHruCyR86nPNbYDcbeT4X8tU6jXlS2Dr7yl/9UcMdcR9pVFToQet/XaofTFi01UtxsrpI408MDdCBFONPhj0RI3aGY+802mOLK16hyJ7YgaoP6MmHCfxiRPsswPCPnOZzpsPfE89TH7hsrhdnxGmsjs1W86zy7vv3iRRHk5ylqv+HoiUz2qmFdijefnW4kl98w9V3+0a0//yy6w6DmhL2BLjse5oogte8uZht3nXAPU/zgrr6auMgrGIa8orf4cTTD72epC8Mb76HgTRt333UXCxobaUK6+Q4H0UBlyX7YpiBMBPeGyStayXaesPoKsWZhL6P34/8QtLHf34Tjxb7j+L8F31geK4XlDfDcyucMw6bw4bxTCVuawK5ilPa4WfxlRnyo/vyAVOm4cktXeqJ8oyR725iYHhb84iD9H/xjylvM6jHLMOxwOjBVbmeSc5+RndxUSctXQaxstPF1F4VkSHMV62f+i139evPxEM0kf7QwmWQB9sTwdyTYnLJS2DDt3mOWwtjA541t427s+0KYit2Mnn2+5euocsDAHCdVliDje8s2iBZbRxKNQ1+H0wrUflbuAtVOBXPIP6mV/eMJV5AZY4XrwxWyqeHD5Hz7PihSNZ5orlFLe7nR/QYJvhI0C3ry2VL40snHDHlNCoaepuDuIl39Ab2vXkcYOxcmKRLJIB4rkS62k87zrMtFMPw9tYzL4cBk6rjraphmc+0bI8bYAzTSBx1h12uQ8yX0vkc9XrFFZOTyvlOPV24VOYhCjd9JygWQYuyOC/YJ5rtJ37Hs4mX8dKExILWGD+PuElhUC6VNCn+B9a6EftyRJTYPIv4U8SIpM0pRAFJA0iVVq3bKxiN0oCHI35pwCaZMuLYIGhVSsl/FwDmBcIniccdk5oGxD7O/GcJUhRZbEIuralhYC5UqhkDFFvh5oj4hUPwHfOUnhVIV/FxFkK2KDrmKTeIfqmIhVmfxfISDpZNuY83liudnwFPE7vOl3AF2hb/RVZHhLEvy4v5+GsmRik1S7NCRA1LOF+mOI/RhCTY5GOYNZwfCqCYjizyicgOvxHgTZAY8k9xOB+NMB7g9FH5WqEnRUMLas94jL3wC/ft1TmK3s/BqLCTOFRoYNlVewZjTzsXqY2HQ1UZmJoCXs5Hri2BtkNGnszZsBA8X1VKjidf/sHUn2gqbNZY5R42pf6qTEXWt3hQ90GnbwBnbrxeiiCeWzYDND+u/d8pq6XLurGzwmNlwTq1Ia7fFkH/zUClkpUBYoyaJmP+9dv79V6SVYAuscoQYB+tyocQl76mTtd77scgzqX4TEDb2KTvUsVFgmnixqjrJks4SJqtDkZAF4v0j+XH4qZwSrFFqOGWHxDYKHIiYwJQ8uK0YGpyKpEHyOZIQUPkYg3SGNJbq459+j8nv5dGtXt9UQ07pDgqGfggTjGtDUeIFzLvwU4oGb6Shl0bq7WjgJtF5a7o/fCLk3IMUHXQdYF95JiG1OwlbPITKPe0TOE32Juo33EWjyZuWExRFgcptdN/xED+V5vNwpsIv0GHnxj6n8dyJ99IzUtG9UZMN5RugyyXGMZBE2I7nW2MvDzRse54bQ+DNsGrxvGqDty7bQq7DwodrX6G0zqMjJv8Hbsh/ittCYXeeUU3hhiInE3Lh+QiYFma8lt653+BMh/lxnWmppbUQp5NEK1wGRb8pi+zVgdV0iyvjQpORiLcqYBSPl8GldYvI2m5M7k8tX0BmMjylUoKt2snG1CBm+kOIv0bW/VihNge2PiEeoLUeutjNlYxv3sbqBvgmUONH7EYb0ocblvp8bgyBR8Kgd0CIYbzCGk6FA1nndSj8+chtIkyW1nhe5f0VMUJirqFvqrvvfKJZUAPD9kOd7pkf9qasSyo0lkqxTkeaHDtPutUj1XEAv58Oy881Sl76JcEJs2FqJ/xBAfwT4bwWWVtUCO4tXfYq6byQAfI348HWomNblG+E7wdIgUCFXneJCpAK3uFybBqcbc+g3gF14RoidP4PYAuGkN7GsexPxXLjt1nq9+79WGJ3XfHfTb7W2YUknCpzqIf6RUJQIlZXgb+vU90V+LL3eM4ugGaV3xpIoVIngegXL3kAFcEbxON79RV6aenSP6VIulPRFd5SJ3mEiNHg034ObqvKYZCWBnpZagk2C+HNrOim3NZDiJ4jbIq9JbA6PZKqMSczsetE46DFV/IP/TRWJA3FnNOykSn+6mNLtEFvTYPntuJtXBEs8o1eLRXtB0P7saGL7HtP9DfGY4H5C3gkHNY0woZKRf6ucitsf0Z8+VTEyNwFImWrUwiqy5frpPOdHP0ZTF+vzn84nTAnBFYqLGmaICE3kZRd3fD3VcRNFZvlryr+gFbVLJ2XcWcRPhTSroLlZxstJKwBQsjp/4TE9B7PTkhIdxpiT6Gpxx2SE/JE8Ur4yZWvGva2cbzfw/ifspkz+ymUhQCTK7eX0pHcdtggo3edG/ZGebZUe2aLr3TaquYa1/3s42wkusyo8nPvkrt5M/NXPqmCPbUepB2LLxk9niTcAo/5K7rJm2u4vM+ZNJ6ymz6TNaSIn0ZqVR4AKWbvnyP2B22R9x1kvsG7K2MZb7MxHAgGDpeGWwgE22rUxBhbgHSar7pEVEE80VAElVtcErjHS07/LRz/5Y/j/xZ8Y/l3JVwTDOeochVJZ3BHCfiZUDPbbEFSTIlRBCkgk978FLUUFkDGI7Dun3qfFh0CukpH4NobYOtThuHMA2t4I7iYd6PBr8EjsWS2sqnJwpx9azhztkK7Pe87Bm6/i/T6rfSyHaGx9tEifKgkriJGqAtbez6SYktXTbKiZBVU7zJ0NgH4583l7WiR4HK3/bfDj4OkcKWToypeDt+mQv53auZ9zBSYuEw6+I4E/klUJF1Mtm8PJg1VFIGCe8vmRcdS+fVkmBOmZsGAsKHqCzvuvuuo8GW2SHEqwYMF1rZA3QkZT3zj4bsesEIRkPpEwJivRfJMJ0nyx5kiOaXzwOgIed/Bj8NEjsUTbmaarnM1800pTFk1fpYdoaFQJDo0SdmDxR1dodEdVLllSD3hGyfFwiIPCYTmGrruvYSYX8exfLVx8+27/p88HQHPVcDCHEXHorsbpf8T6u89FGKmiKyOLnl/8jYpnmUoCs4mS6sXoYevWVs/meQQI7vy2mKYlS+SxAY0lpFsqmbWAfjOqva66uHlxAwsy15mHHd7FuV8qT4nnxghBSgSYgAMfAZO3i6+lKr3AmQoWIqhA2HsfCIynydkq5rFeF1APVWpsCtb4YvqqOFE7yYeC9PM7enXSYeRZxLQDd9Y2PKIXpZPg/5DHuCZuHuZkAuLfBUbiLI1XOBVwnR/sHkWGs1erB7zB1cWwsuVioRtwRLi1l1BfM/rSepz8xEd19EiZ8Qcfq6DvC7gW29kqntFeDF9z0lE3amqUsJZjav5LUE8Jj3RZG/iphDISgZrrbFz5HdbTx4rhbmauix530mnwyJjIfGQqM4SL9daBfs+7RqYuRemrRfpMk+cuk86xiNGgklRuCpdKffQkvHq767ZI2vtbjU7nrkJcl6eHiyHgi2APc2Q1QwOFWGn/oAkIburC8YDvJ002kK5/qJM46Cb5R8+XCRmPOFokeRL/kK1BFJdHvwwUF3gd2PzA7BFkfQqWyufu/Fe41h1FnzpLZJP2ZqkbNl6YefWG7fj/l7+LK6DFyrAZNGsebnfqiWxQe6VU3YaugUOos/94l3q0SWxccO/SPqpJ+99f66Q2zxQ01TDnwk5vD5jHg4fBZujsUy6p7d2gpQDMOlXiQNC+6nHl83QJ4s7wPLYy5iUC0XNdjYWtSc92B12RmfX039vI6ucEUZWbnBPzjsAi+qgsFaRwGkslo7Er9VS9li8pUMiQEFuAUmqRY7WFqvfaozi1mI4LR+jBJ6jhefCHVSmgsVTlSMglZfL4bFw+DjJyPi/bMg/iLTAVcEoJfqbXT/DKX6dIJWBdJ6eVqCWwAXxNFtmLGwl1+9iakopLdMLlHPC/WPvpy6gG2cdgFxfY6dOv9q1pHnBFSqykaOJLuYmJvjBoGBNcelYYdVlMheM+szoL+qfwu3+FzIxF85bt4CGFoXqwWmu2F8RA1gyX+flSHgwHKg1khq6+YWwLRnW6eo8ZesAsxSgjgRWP7CFyP9Ov844HjZYSCprruWrXcaYq6X3A7znaiDeXLi5/aDT4VLNUCSK3dh4jxTrdES9yFEw4gOtRDTD3pFn03O/Y/GSLnuVZNlfgd3vyb7Z3dXeFm0JGirCcvkmIYdVa/a8USeKetChvP0USKzdyvIGWOvUyPMW/iLkP5U1Rtgg+S17K9ZDADcpS7fHS5gpc//gl9Tjw96EkR8aX7d4031vC7ZMGL1HYRUCfLRJisX7KjTx9ZwQmb87AzfRs0qj5DD2WyFMq4rZVTuli3Pvh4ah0WXf4UyHubHQ024sbD1lGU24a9pWdRE9vvFzfLNgWo5R/hnAcuBHAgsWqpP3Fm/ocYtWEaHFL4EbHcO5IRjGlhn3O/dVxTIwB+4K9zKQTROa8qmww+WFYPaUnrYFMXD4MzBrP4MmGvd43iV/cGMI+JqgRUGqJHQAdL9F/KlVsaZfvEjZ5n9vHHOTftfdBIW/Ks9bC3sDfOciIMUY1VmmWg4Q2aWA+AurSA1QPFsDnhbyt8rXEaSjcPJyJQELEBuUxWOOmAwKSIGmueYgAf4gLF5C7o7QFJ5CB+AzfgFeg55RH3d9m3VIR/jZ/T7M76KW4pyVIwTv6euNYyD31YJuakUPEM/gU/fI37ao2CLrStfL1GT8tgoMCoWrS8v+w7tRcFEh7GnxiBXNVm7ftowwC0z1UeSbyjfCj4MJONCBDL1PlORDVV2HALlzZZ4d6aEAULwcmiuZv8rGRQ0SP5wKaHQCDPiPDWaNQt0p6XTIXqlkpdi7eGLPh7DnA7F6UHmnH8ffguPFvuP4r8FkMhEWFmaQgQk0w/mB0F9TM/hnCGR3Af+G/cbBpnKo3KFslweE1QzQpDHtdvus+RsT1YeG05X8NhZxbC2V9PSC9Q3QGOjBPLb6c49zKCsbYKSinkZwH2mnn7ZW34F0LOFoloBFVZza8hhs1zAUQQpEs/YbTYuBxtRreKRUunxMLeqgE4Cxmo2JNUCKu4qEmIz7iryGjsGuQ+63hOT8h5SRr+ETqwhoC3+R81k0XJ0kdEkwKoNGgJ+niu9enUYy5Y9zJXGqK0qsvAS2qIo8DvGy6KORDz0UGgqFGakyJgZYdQXkfK32NwJIPEu8xTqD4uVQtkbNfHO0SGFfJ72x/maRtKzTmY50gH6PStfpRs1z1fUS2fTrmMVRY2UDEaqRr7MFigyGRyHSZII4cx3dvWBI9QrD20yueydBV7N1J08yNV4Rh0LqZfJcKmTiAJHc84lUF24D0+Q+c9oNjNfBvjZ2DhhISf9udLMar9fn1TC/FqJCFHKYJSuZuf95pvm1LxoexOaH2JYM9nTo46WYi3xjhf2oYhKCzAN+iVC6Sj0Ocu+rkhX+idDnAeihkAT2ChbpvK6XQYRCSqs2B2+T+GuUe24AQKSKg3rofU/rCyWJ5ykL4oY1QKSEdH5pHcCBk1/qodBqZJSu9B3EwoAJvBsNE3w9WilNJgJ9QnmvCt6rDzF+cOgA6vo8RpEtmsaWv9mzz2yh3AF7mzFu6mtzqJ0dyZb5gznvffUzO7/OQqkDgio3GsZ8937AS5HSBWFXXMohcUN4sAzu1oQXB4vrJcZn/pBoKBRvBE/WJsha8W0XIcj4GK8l/kni5zBlhVqK84wSSRr4qQugOO0QPRE8Yxc33Jt+1XML8MNgWdP2z2sf99XlUmCNYlwuDOp2jvF9TrsUzlSdIYBp/A94n1VGhJ9CStPqJ89koqZj1mQWpYfuN+ljhPKNatY9wPrbxQOpl2LtSHQlOFSqAlZ/SbqmXiVJMxVWXynd04rCblLxj3wTK/egUkJx93sisbRO89mOZvj1FNih6BDtAE3eUfynCraX7eGCT4yyZYHFv/FoGASY2nceHETlNlF62HSMYtjSVdJleYQF5yDvIJbWw9D9UBzUv92Y1WxlQyPsbIYxH4zB7qmY4BPF/Fr4JAbuV+XArQEQ5CKuqORefWOF2e7p4+1GYCoknKbtTtrlDODFCphXi7GjYHYg1wQ7ebcSYau3RWg/PrIO4Kky8Eo+2/C5Ac3l7GuByD1AvFFabHfgQBL3Qlcj3+HwYPWVeUhHDOvzoHKtjW7KhW8nk7OwF3aFrLUt5yuu9S3j21po8PRrBRb0eovEvdBbFXqH9OWlsMu4LAgmNGi8fo4Suj0vXS6SbpmU84wdBWYrKbHDqXWtN6puGbxC5D5KudD4nfGn8EwZjMuFRoUM+XkFrxJvhSV2TUGjLl+SiA0aL+SO0PN2MHur92hmufarG+CFXx8wDPtk3M+8OPFoNnsmEZ1OIZ78eY2ejOFOzOpkfZur5Lg8CGsHkXalSGImqn0Ujxly5ui7KJ0OIeoNf09tIWFybRg0Eos0lUnyVded1AHeTryfU/JhS9EW9T8Y/6N+rx4xQn5LHTmxy4VSzOtsfmPfV0JU9SC4OFzXtgUwaQiwznT5b0qwphN++HvG5Pnhos/9QsDSSWYH9xArlBSFR6lvnHitbrrPkAtosshcPysATgw1xnOTuk7i35GQ3wVMTmPha0fecgLMYg2jxJi50k2uU+n5vl9rZ5YHtpbu5o3dq8lrMRYaTSYTl/e5nFQbPB3WZPBYnFyzjDtC4YMqsHp7kFCcTokHfaKVXVfWIS9xV1UI5anQs0bRdesTCYNfEH9q1b0QNkj2vZ7EGIBdr4rtxZi5+qLbZyb5T6XqZPaWQreC9DA4cQDXP/4mV05/Qd05ZTKBvV7vyecXL8QFXWdq5huu3/kI5QPzf5TC16hPRO5ehc0PwRfeUKUg23WEtvvkKoU3acESmdct3q1zmidsgZL/U8FkhupMIdEcEZzQUqsnf9fuk5ju5O3KuWpBRQ3zXalM1TptMVkYmwuxqrgpIEXyvPZaqFbYBwEMflnOrVIzB0/fKPLVnvFFr7thVh6VNQ0HpTuvBH4BDmV+sxpY5rBwxYt7DLYtgBBMVl4k97dqrY0cA/0e16ur/I9AG/v9TThe7DuO/xrMZjNJSUmYzW1uw5q9LO/di3+UejNo6kLjm3Jmc0MwfFoFTVbFRL/7PVjYUxuIED5UZCVDNOzg9Gsl6NR1qulQvVuY2v2fVLLXnL5xnJgLZxwAvIxZAZvFxrtR8I6qphDaj0v27if47Qm8t/EIpVSOFj8Nhy+84EtfqFcUUwJSZeOwXKPXXLMXvo5QF6eCexFkhhcjwaRihAx7S7rXdKbeof0lMb/ZuFEEZNEu/EXdCdER3HIIug4hX1cSQZfY7HkbnFUN4zTFPvf56CQU3YWhfRrWc/an4oOY4dFR4JcgzEdPec/DxYrzpZNCx76s2y8m1TqYvaT45OEdc1jodbd47gxSbNy/sEknxAKNp8Lw90SyZ5fCa+1wUPy7WooNpJgXPgwaNAXlsKHS0eufovnsP0QOzYM9bPYOZmBxDBNy4dwsY5HHNFruvfejoXeYgnjgNjvurG/Iwt4dd0HU5cCoz0X2R4VTtotUsk/7BGZC6VK61W4gvG4X2I2bk0CvQFJtsGTmS8bPDO7NMzXBTPCDoHJFN2/8KdT6JLDXK5mpPTQbrqQzxYxah5EfwRBNgTT7C+nE0W2q+j0Kg543vu50MvyDsVh++oB+PynGM9/grjB4qBQSYjTF1cl/SOeKCj6RMPITSQSqsOMF2PmSvrtUh/pCejbsIMmq3pisKtzObVt+5nNLf0yK9TKuYQ9vj72FB8Yq5t/gHiTNf4AVC8awdMkFR3ZcR4nkms2UOb04IRecns+lyUoFXlwcBF+pfN6AG0+ZTbwVro83elWYHfVkN4u0WIOPsbtpasX3fBMLg2M13QKJp0lHy0zNZq4jBLvksVQeF26mrG+8mujUUCIkKE+GbluM/gxGaZJaa64XTykFMxkQUk9HcZObZFWT1T7u2/IYiyOKmJgwBG9V4t8vAcYvUnsMubHyEtimIBw5moVRuvFutfS0ySwesbpuAb94SfTqOmpD+8txqX6TiJFSsJimKBr4xsqGfPjbIhekQoXrGiuIN5FNBZwWANlpfng5FWztClcioMfN6s9uqZH1UpfknhsvSavi9gXp+oB0LimEh8LgEYy/p1/p7zwQDtVpcGVXBeM7MB3SrhX5p85gQXc5Ll1iacISWa+P0JsmtTHrYKeXZ3LSYrbga4JvY6G4q3qeNAHRFo0Moi0ATtkmz4aHLOpBrLoMftF4GYPEgZmvKYcMxcc22OrbhztL4KoicCoSgVazjXtLYX+Y0Q9wfN1KViVKfvWbXcbOqFGJoxiUOpMrxigkrw8HldvknFX+XgD9HlGutRnBJ7IubSfbbPs492tFgrxwKed6leCFuqMlObI/uS3g7a8mNawq3Mq9JbCwA+7h0UC55wWRk9QVOhzNXJQ8mCXxkJGk8fupy5X1QUE2MkWP465S+LUeNpca152ymBm8WB/Dj/5qqVjiT4aTt+rnKh0cdpEwdDSq946Fv7CnGYb7wCyrsZBoqpFjvTYYLJ6+eSYznFkmRDpdDDzxZ0l+6mR9ZwdLa5Sqc+S/iV53SPy9+wPjWPanULBI37ESOwWmrNKrr2x5QogFve4+4sOqqdzFj3Ew2qHZT9dmC+nSY934S5D5hks5QNPJmfWmixDR/rkqrC7g+mBZs9KtHXc2BQYa4z1A9hNdjEX0w4J3mOQodF5vufP1XTq+0TDwOYgaZyBrhA1+mo97fcDvw7+H1CsNb41tKaSfN+xpBrOiWDI7vJLirpCuSWGQOAsm/y4FMBWqM1vlJT1gwomvGQbvh26T23f2mc1muvuU080LnvCeJp6EbbA8YCQ3uHhdVs+CXMUm+NIH/rza6LUGmL3DqfBN4/NqqPc6XDcyD5y8RU0W7XKxvB43TalS1R4e87PFB07O0Be6U69gdd53bM5+UF2kr8kWP765mu91OiTGXacpyE3fKPsNXb5Kh6YyIZiqcnP2JlHz2PmyyxbJoyhcuQPW/EM8WxcqZH19olqVq3Yo9sy5C6SgNWGpxNmeyP5C5oRKRaEQ5D297tH7yOkQ0hdO/FbyYBWKglr3G0UppGCR8u1rwk9ifi28GwW+TR52Ci11fG1axFs60QC/BLF92XSfKGSpEJAC59lFhUgFa4CsZ57kMK9gKF1FsFf9QenOGGA6MBV9wW+1a3z69GnEBDWp1cy8w4Rk5M5JeSJypJAdcmYLweZ/FNrY7+/6/v/Ktx7HcQAOh4OcnBwcjjZJB3sj0c0HeHXCI0xNV3QqVGQQbYUnyiEuSiGj4daT1+lXJ50FJ21s1YBXwensWGJRheZK2P+1loFrtvrzWz1ktygSBvZGLiGLzU3wiEYafs7m/9DTVMVbf3Sya6uziB4vC0jKhcJ68kSRS5po/9fq9zcUQGA3pVSK1WzltUphnLa45V7aIu1qkfup1EhgACSdre9kO7AIlk6A+UfYpenudsp6W52McJuIT1unZleCJHl0rLgRH0kHkk4CYcC/YMi/jTKdbpxZIYGBikG29gZ1cHU4SDpbb0QOkozIWwD7v1GPe4dL0aszutxewVI0U7DMD8o8KIrkQCsbLvYQfiYqVGyRDotZmoQuwA/9Yf1t6rHcuSLzovtNin5TSpM6HA4Kagr4pR42qPLvbbrLRsQqNgBuWRirhs13KDia5dga1B0zLJ0o95IKjWUiE6dgiTtSLiAj/X5WjlxMS4gxSF8R10BWCsStUWjxB6TwVZ0vFwfB5VGKiDhhJv6n76fLmdmYAxWyZxUZwu7VdXbU5QoZpFHDUN/3OWx7Cr5TJLVq94nfSLaiAF+yktUBG7gxBDKKMozjMZNZ1uRLjEXT8VKzV+RydGtO9mew8kIptqjgGyedpSppmg6QmfUFF1XOZV8XuLjOKAHj52zGDiw0pyqLDiG/z+CqAy8yq+QT9WGZ4WR/KMhfdkTHdbSwbbyLuT164HzISe8oj7nQL46n/E/jmkJ4pUL9/tCwPjBlFbYeCunInnfQJVvWcVWiPdJZw2kB8F5yB8kG32i9bF9HsPhIYkflQxExSogH9XmS+PLE5geEBPXLFGiuNo7nfS/SYTqkXXXkBKi2cEsx97y9fdwXPxOixvHT4Mn0Dtb8Zr/NgjUKGTiQje3e/8hcZhhrQ6bRdclX7hBJd12Bv+8jenmkLheJn7GnfCLI89LvEX2y7FCY+Iv4JCqYy6ah/4aedxDqr/m9Br8ov3WKJtGRO1cSOboufPc847Exd89db1fC9w5jN0Fj99t4uBR+qEW9LvlGC3kk0lhcOixUu3yP6vPV46H9hTCRdGS+1l2Lf+TdaLgtBAIa25OZzCYze1JgRgAsr1cUWkr+JCsFHi+DO/VWmXo010jXqq4QWF8gCZ4B6uTOw9YdONOhrCsGL/AX6c+zrhDRcNxlG3jStp1JfrC/0qiQ8mxOJs+UQTcvcHoWWgBr9S7m9+zDfb00dgmHQuZbcOBH2HZkkq52iy9+JeEM3deVa+zGTvcfg6ZwcYEov8QqPHW8azKZ6AtdLIrEf1MFgSW/sqAWntmz8YiO63Ch3PO6se1fkhz1LGbXHyB46Sgm+kGBXdPZt+UxWDxa7UUFeJsg2AyNzcYqZvyoN7nligPMO3ee+qBNZvFBXqLpTtLB3CbWiRpnHK/IoKsNZlfDKoUyqcm13xyWAyZPLzmTSYr6E3+GcRpLjvCh6kTxwXHXPk8XN80JkwLT7vf1n3EsENBVVBp8Fb6Rbjn54t/ViVOfSNlX6rztnC0i490Jq4Wb04Yzzg8eHflP9T+o2ikdOYr5ggOLWjufVChZLWMrNTGG2+Oz4Gf1eHO1qDF5+C/aLDZei4KHw+FZRWM1wNiGEZgzodJLoXhwtFg6ET436+0S1t4oXlfFK9XjXS+GSb8YvAi7hXfjogGXMiZ1upIMPaL0e4Z5wwm5YFIUeRY0Sm7iX5rfhP1z1XsdN4J6SGFCAe/6fKpS4ZloXwK82ndLOhwOehUv4t1oKPFNM8iB7/FJp4sN9qSA1VPNwStMimF7PlRbFtQXUuc0cWEBlAYqciAlf3Z8D9bshR0vqSVwI0cLIcNk0+/T3DGu6tmzN8LyC5TFaqfTyY93rmXufWb2WRSqHLtdxUddzjLrbVk7VCoSIB1wndlvpJwPMzJl7+vZCWwySw6q+01SHPOUda/PF+J3Y6maCG22tdoNJCqIfD1vFSUzTylXN1ZdKgTE3zuI9QY8CaM1MvkdoXIrZDysLvbZm6Bwqci5Kgpfr530GgO84Ipg8LJ7kN5NFrYRhr8JRqgU3EBUs4a9pfZrBdj5b9iuKI66Ub4elo43xlWNpZD5FqcOLudjH/nyAuAH4FJgLFLU+xRYAnwCTHG9finww/cLKfikB6xQPPP+ybLXcTr0+aS8+aLApfM+/R9Ah7Hf34Djxb7j+K/B6XRSVlbW3gcjuAfMysUcOVrNqO7zIPH5kWxv0mx6ok6QBXdEJwP09bdJcLZP09WlQ+hA0ZC2BSmZbRazhRuCoSkNAqs8GFGOZs5mFw1OmK1qiipeTm4XWJUIp3t3omvqaDDgaTi7Whj/3opiy6TfpRPs5K3q92e+LkkaReEqcNtj7EgW9pmfj+Kz54TDz5Ng4x3qz67LF2aPRjv+YDGi+83q8Y4w/icp1jUofFh8XJsCXWfe9udhYR+9KXdAinQguTsEDZ8fJUU3nZeELUAkGAcfmQzXIZF6hciT6oJ83zhJzirkkQDZfMZO0xdAO0JjmSQ5VD6c57XIMz1dw8Atc3UZRB9hMgLkOV9+jl4CdPPD8ld3j7l18FUbcxDZha6XGhJxToedy4J9GKUL+nwipag7K0+9efBLlA5lXVfKoZB6lRAU3JtoT/S8XTpjdYbLG24TuSEPFHnF0e/7xxn1n8lK+cY+iePB6o+1j9GnECA8rAeDc6A06dIjORsXTJJw+PMq9XBTpXT66J7bIa9C8rnqLqLmGmHhrlB013lH8l0t3BcGF6tqr16hjPOq5/dE8GpWFOjzf4DfT4M8o78FIKy4bjfqyQNx00V2JF/RCd8ByryTuLQAtrV407+nsRuyd80aMlNg355v2FpknN+XhrnmdJXkSMHPzI+FGfnwTJ0uo3CMMPR1WW+2PK7cXFxfv5hnIuBr3VK6+UEpdCv8BaxmK6N94JYQMDUar6X/5KUQ1JP+LZrnqnq3ePqqnqtDwiybaxWpwTdGPBHSr5N5wRM2V9Kl6Dfp7PLE8nNg0Uh9oTzuJNnwFy5Tj29+GD636Z+tNmgX98WfLEmMbU+pyStOh6yD4ZpER8YjkngYO9c45l5fA1L1UmxLxsCyk9TrfEut+FCovIrdmN8VfhqpHivfrJ9fV10mSae1N6nHI08QTyhdYmngM8LW7sxaGzpI5hMfTXLz3EZZb+Pa32eBDft5PxrWNMLXDqPBl9U/gUfK4KR8aNax33/oL90+ncHZNeJNFqH5vbO/gL1HntzJTbmKdyvhuUhoUkglf1kN95XArAMK2VSrL7uboVK3f3c0SwfRZyb1PGmvg5rd+mRcY7EkN+sUlgWA09VxnWuNMhQtbGYblwXBx9FgtnsUeRpLmORVw+J42L/KSOrJ8k5lUxOsSISoGkVsX7JaZM0Waa7FodD1EomLdAx1DXwcdWw6ax6j7viIBoxJ7M3lOWQ1w+ZGaFF4mvXedD1LEuDnOEWxry6Pz2LgjAC4oO+x6UZX7nlBOjM23i17Fs9CjC2Yuu63MS0PJudppHvd9hNbFd0jWe/SkAYVqZDsUMyxABvuFFliFZrKhbDZXTNXdQR3zKLqAkq/BoDTA8A/wUjsNUWdwNpGC9uaIMBXEUOUbwKchkLIQRT9AbNDIOsd9fjU1S4PUE1B0L1+VGv2cccKXmEw8HmIm2ocU8k9HglGfgQDn9VbX3QA321PQPgw0kZqZK13vyudlp4+WKAnqrlR5FK1UPhgAdDjNhj1qahrqDDmayHIeMBqsTE+F16rEJKKCr/vX4UThS/kX4HabPmr8qKHVhnvUoWSSc0e8bFXdTMeWMz3swdw/htxzN9k7ETbFT6Z61x5d9V8sSNqBrcVg6mLprj6++my19Hti0/ZLvKOCjhsAfynCgZ7NUvxtw2cTie35eRw7gFYl7/aoC4Q5htGrQMaLX5YPedB/yQY/4N0paoIQxkP87HXGkLNUFavKCroCuBu1BfArlcMx3wQBUvhCyvs1swnOjgd0r227zMloe6T30U9pyyyCKdKajP+FBj8qtz/KgR2kw7JKR2Q9TqLun3SSZnv4alqtsLUVVLkUSFytCiYnFUF4zT7Ut9YITlFjTWO+SfL77brFUPuBBCSXdfL9GtSfSHsePnIu7YbyyTvPOIjQ/wLtN+jKCRXUwrm0SMgmKDdUO+f2n7Q4s3nERcLwUXF06vcDr+cJF284Rplul2vwIbb9TKfy13rg6ec965/Q8Eirrz/U35pbmY18C4wHngJ2IcU9l4CrgdeBk50vf4SMM4C7/2C/OYq7P5AchgqIl7ObOlIH/JvvQXQ/wC0sd/fhOPFvuP4v4faHEnCqBJPZitPpPVnY/c4ItCwsY8Gbgaijmmrg8kk7Im1/1B2+fg2V/BqFNhMYPH07rD6cZN1Knfr4uGmSkIt4m3we6POxOu/hNVXyPkGaiQWu1wqBQ9VEBPUk+9rodGpKdy6tdNTNabdDQWw5RFhOaoQ0sdl6q1hPHWE2CmSXEozSmQc9AVYdYnaR674D2EHrble/dlZb8PvZ0oyUYWc2ZIE1I0XL9d7LrjZZZ2Bo0X8F3Veg03l0lkSqUnw7J/TKeN3QJh633WDnzvBFHc61V5ph4Pkc2TzoEvKWnykkJ+oYZClXAAnb9P/JkW/iuSqpwyo08FrYeUsT4T8oZpj9woWxpzq2QjtJwxEXUftoZB4mpjDBym880BYfW4PJE/4J0tAqmASbirYRA8bnOYP3g4jXXv3gNeJ3OdLt8VvGD+3eAVvWtbSxQZVVoXsze73JGm6/DyljAshfcQrK3q8+pyCeggbXCPBgX+SFEwGPaf47N4ik6WShApK5/JCiLDAMFVdoE03X2GFwpfPXUjedJ/6uJx2CfIPKMyxQVij1ZmHTqh4oMErjI+qoffeRuzdjAnfEt8urG2Ar2Oh5cBiw/iagBGYMuHkckWh29lCsBm8TB1LzR0LNESO4de9S2HzA7R4FnGaaxjQkk2w7lqBsMTrcpVsbHPlVr6KhRciIdSsOa9pa2GqInkD4re36V5YotjgHgr1efB9Xz2DN/szYduqZCXTr4e0a0SKSOUdNsyVUNJJIdfnSwddtUZCcccL0jWgW4v3zxOpzVpF0aL7TSI15GcsIAFSxNb523a/EYZ/oPZg8U9s9b2xGRP/QOszo/pNmqsk0aGT086dD7V71R63xSuksPWtpojjlmEL1cjJb3kcPre0qgi0RXUWHFis96WuypR58neN9GnYYPGC03XIaeDbXM5lQdDDSy0laGso4PYQGOANv+1TdFq60ZE/c0ew+kuyQCfr/udV0gGtkmztAC3BfbirBIbnwB/1xs7rm0vgSU2NhJC+TMwDP5PGbxtTaweO6vf2CoOTMsTbSYXAbtKtmDBLObw1ciamTLjfNMIgqX1O40rej4YLg8DkOZfFTsaWCVV2SLcaK5VfnPkFmxrh/hKo8FaQu+JOkgJ6ZwpAIB2vIz7QexVqMLJpB1fNPodvXzmbz2zGrtkARy3ldimKlYcY1V1Mfe6H6PFYeykIhP6JNA16ifui/Hm/x4AjOq6jxrZ/yd8RHxpJJl7BPFZu4yfXdle5V4qdKiQPRYETu8wTf9SDWVXgX3uTK+7XJGVLVolMuELJ4ZBIclkKqApq9dJFazFJR4QBufMY4m0n1aZJVP04GFZcqPcSXDJGCG0qn6HDwclbZS814Mi6T48aKy+GpRriotMhc0Gfh/R+VR2haodYnOhk4jpC73vF31yH5PNE7lCFlPM73pumXCjz3DijZDAgXeEp5+u9dRtLIe+7g/eUGzazjWX1cEMxrHCqSWf3h8EfCbB9v6bIczQY8ZGQaHVeyGlXi/9XF4XSicMuPvbrbxeiYhs07JvNSc2b+Cz4AOGlxrW2Jnw0FXa4LxS8FZ2rFpOFFypgp69Gmte93hxhjADg8Arn2iIY7d1C0RZjF9LVQz+gf9IEfvNbYyhkTiz5mn+GwL22qZhU3tI+UTDgKYhVqBLETWdhLbwcCd4qckFQN7n/ztMwc8IGyl9VbmXzQ6KKETNJyGMqbLgTfp5slMp3OmVvl3CadG15IKl+N1c88g733fUFfsXLjJ8bMRy636Am24PE+2Pm6JWiOova/RJ/jvhQrbgEovCz7lYjqdLiLc+s25JGhX6Pwkmb1bG5o1me5w13qIlOCTOlocNFGDGgLkc6yZZqSNo61B+QYlrNHnVx2G3tk369WmElbwGneFVT7QCTQj43MSiR5ypgga9Cec7RLOfakdTlENf+bI+mmaXnHTD0TYjxyKNFjoHe9xLTfQJP/+tfzPD25gvAPevEAPcBa4Bdrr/3uV4HuLgZ5m2IVedD6/IkZ5h8nsSrnnDPXWFDOiZOHscxxfFi33H830JLrRRLul4mHQueqMvl0mAb/R35xPoogt39cyXRsfmhzn1/wgyXv8YRSiG21IpU1PB3JanmAT+fcL6qhssKISLWI9FnMlNgDuajaPEGMSD+JIbVDmXEfujZQ8OsOFbY8SLMS4JVl6sXIe9w2UDoGP8hfYUFrCoCpV5GTgsUdlVINkCrX1ikphgS1FOSDvsVrP6jRdl6tTcSwNB/t7I+VZ45ve8VuRUVMwhEy3z/13oG2bZnJbmv67L4ZRr8MFBkKf5K7HhedLdVxQ6QrtdNGj8/gH6PubqMOvZGUCLKxdRTSXWWrZNneovGHyakH5SsUHtcHAohfeS9qzQMx953w/T1hkTaQZgtHXuZhPSTYrenn5rJwt6E53D6JRKbrDA1bigSScq1N+m9lbzC1NK6h4OCxcJg0/kb+cbC5BUS1HrC6iv+it2Mcj4peZ+zPQW+iQOrwqel2dFMSV0JJXWqwpQTe0sdIWbYWmDssDiY1Nn3hT4gHjNbpJ1UMFuk40vFJAT5TF3iCKSAo/LgAort4JMFN6tULEpWsLkRemZDk6oDNKgbDHoJ+mrkiP2ToOedUnRWoTZbOjUV3h0dwdvaypzNqzZeqxL/bjxWBg1OMDuNHbd7y6VAvrVY0fkRO4U+OfLeJFMHPnHHAA6ng3MzVjBgH+ys9yiItLlvHtDsnelxsxTnMx42juV9R5wVbigC32DFxiZnjiRNdV2YUeOEOND10kOfiCfcm0+d7Ni+z6Tgp0JTmXgbNRSpvTtSzhPSwgkaRYPcb0XmWXfcJ20Utm2ixkvz99Ng452QO6/96ztegj/OlGSYSlrMZBa/Sl1RIWaSFK5UUt/2RuhyiZybzkd2wmJ5phWeZgfXol2vqt/rZmn3VhTp3XN2wmnq987aD6cXSyJQhQxX/Kqan7PelsTT1ifUZCM381xnBN9YLDJEOp/abc/AslMMnZY1oUMIzIIxPnCjxdiB4b3uBp6NhA1JkFe0Uf3ZZ1UKI7wzyJ0vPi06yVX3dXB3+h8m4gPjKXPAn41Qb1Enpy4Pgsc0NYOfunXhwxh4q5ei+9RsFdLYrDy1tYDZKpJL6zT3t8VbEl6/KmIEwMcqyZOGFiOxZnjTdn6ug+j9wQQGJBvGIwNiCd4DjzcbC9JdC+fzbhS8VQWVKv8j32jppumsskAn0avn5fgMtLMvdY+yY2VUxVK2p0C4RST8DOh5q8QHve40jtmC8Eo6E5tXCF4Kv+Fjin4u/21N/OFwOjgvEP6p4EABsg87aaPa96/7P4nJj2ZMLrSoOm6LlgmJ6SQFsQBknzXoJfEqOlJEjYFBLwhBzBM+UdD7fgjuTUixImYLE5uOTclgUXn+9XtMYvelCnILQI9bZQ+p6+r+v4qg7nIfqLo39n8ta6iqE/1wULlNZPWiOqGCknCaJGt1xMjE06Vg0tAJPWO/uI79Viu3iSKCztIj5ytYd6NBWq/tHHDjMIUsOzKvj/aFk5M72aXcEaJOkJxQiMbPyuonZCFVIScoXTrwW6oNagw7I2dyer6QMUp8jfO3n82PcwPh8QhRlfLEWc0beDuqAyLesLdh+qZWn+gjgMlkotEp9iy/+hl/014hvbhzwgvip+ZBIOgZO4zk6KHMOesr4wfXF8KqK5SKMgAkzOT9KrggCIaHaBbr+V2EBKCC2UvmDDdBoS2s/kKSGv2lsZDixvZnJafjKaVsMsOk36D/k8rYe2/AAD7wKyYwtBqrSmIfJJ+p67wGyZHo/Os6i6LfRIHCFqxWHtj2rDxzO1807sdb6uWZ3f685BBVaK6Gb9NgoyKntOUJ2PKokBM7cQ8ezLdoZK218E+GMd/o81gxk2QeTD5PSWBpHvUFJ+x3MNAbrJ4EK0cLY8q+wwv42WRUjSG0H0xcKnuinRrSZfgwGPoGxBuV0gBIv1ZdAI2ZIHNJczW33HIz9zz5JNm0FvMOhWigsl6zn2gqF/nQoO7q4m7alXDqfnl+/ktdbcdxvNh3HP9FmEwmYmJi2m/aGkuEoesVpi7y7P0EDvwgCRpVkt3NRNr7n2Nz0DrU7oNfT5auREW3jJ9/HJ8EzaAx6TxjgOV0ck3PmQRbbYQGqKffny78iTlnzeGJCRqT32OF/V8L22TPB0bGErQG17qOrOYKSUKqGOrAbmcQX9eY8PdVBGe97pEkiUrzG2RhsQboPbiOBotGwg8D1FKytiDxiRj3vTpx+ss0SRb7a+RlTpgtSV/VBhhEa9wWot/4uDf0nWFndoSKLVI4UHkQAaReBjjVWuYg7J6tT0gn0pEiZpIU2U9UFIXd12DzA+r3uju13J2gR4rut2g9cXC0wOIxkKGRjzkUtjwqmwAPuTWT2YxvtwtwzsxWJ2icdpGk3PWK+rkD6SAarjGsPxR2vQ6ZmoAS4NeZsPdDkZw1HJtTOmoGGVmbjdZASu1wVwlKGcRvv5tGdSq8H6polQjpT63DwbvR0CVHUejufQ+cWQYz96qZsg3FsPNVKF2jPid7E6T/Q6QOVVh1OXwTLQxNw3sbJbGkmg9q9/FWFIz3BWWZ215PamAUlw28kpk9Zqm/u8dN+uMqWSkFJLPGZ6VwGWx9XN3t2AGGe9kp7grVqbBwkVGe1Gq28m0tJGdDZZhxLb6ndi7bkuEkTb051gKL4uEsb41+0jGC788TmRcHm5qgosmj2OcdwTP+Z7C/GWp10nu2IEi5SF1cTTqL/MHvcPv5G4kOTjGO/3EW/HG2dr3DL05YuCM6QUzwCoUZWXqP2qGvizRjvqIDtNAlceVo0pMHgnuqO1dBZNgqMtTrHUgyIOkso3eHG/0eh7hTIHpc+7ivsVjkLhuK9cfV5YKOO4AWj4HNioJb9S6RB+2osBUzSe+rZ/EWzz5PSRw3et8Hpxep13m3nPxYjZerX4KeQAKSpO56uXQnesKdvNattYmnC7t3uIYsVLbeFV9rkl4b75L5pqZ9F3JsUCIWr2D6e0Oq2didZ4o84eD/jvQ6ck+oQ+K3WaKWoFM16HaDFCySNCx0DSI23IAzHXxNMDrBWLDbOeoU3ouGm0MUb67axTiHJL8D+yjWcZBEoV+cupjtdEqSTtcx62iWxHyyQj4aOMevkZYBXfiuVz9DQfuZ8Os4NR8uGHA5vl4ek3RTBa8OPJVEK4xQnLOXvY4EK1gAb+VxO+TYdM/ssULkaErv7E7yuT8wxGEsAO3z68nsavggvTtjg0KO/PP94uG0XD3x5iih3POC7HXGL1J7njWWcW3lV3wWI11ISmx9SgiAKilkoLBWikMVDRXGwZM2wSk71N3RIDFgdaao1hwpdr8r5AEVcmZL3FK5VS0ruad1jQzwU+yLe98D/Z/Qeocx6Pmj68rb8jgsP9/ooXis4U5ue6qBQGtnfNabaqLHoZD7rezxVMSDQ6F8g8Q2hRoyXf5CmaOrFEWHunzx41Ml9kG6P3a82Kqa44mc2UI03a8p9LhJTh4kbS+LF83pFpzpcL+/2k+qos/jbIi7lPNGaSQJjwaZb4kvnw5LThQiq5bA4rJI8diTO/0TmFsLT5RDha8xjzA4731OCzDxdKBa9jWxYTdXBYNZpybiEynFh07IhFubKvkwGuKsUOXTnlxgMplIr3wZ87JJMOI9Q+GsvMfdvNGUwOKF040KAC010tH0x1laUttGcwJhu6EuRiXBWCH3vu4eczQLUT9OIUPb606Xh5xuAkZirt73GQm4JpNI1TeVKQvltbYwGu+5g38+dDGNKrJc5puSR9ih8WorWQ0/DpFO578SkaNh1GciJ6/C5vslh3VqjnE/XpEBC3tLl5y7a90Thb+IhLkKYQMl/o2ZrO78+65Xxx6ftkCREZ2hkQXWwRYg12jRCHUewTtCPK+XjFFazpTa7ZwVAOuToH+AB4HQ6WBQxRKuionh9pG6wq1ZnjndPsvRLF23EZr5u7lKfvcMxVy2+X5Rz2qp5pZbb6Vrly4crphzIeBvz4d1txgHg3oKia/X3foP2HA7fGETdZr/UWhjv78Jx4t9x/Ffg9lsJiYmBrO5zW3oHQmjPocumgRLzERhGfpqPEHSr5PgaFIHMkLHAr6xkmyLO1k5bDKZ+HbCrXyWEk1Qs0dRwulgwqYLGZF+Oiecf8D45vpCQrNe5YxtV+HtbiP/uzD2W5iVK5O5Sg4lzSWxqehmBGDfV5JA7KFYJPJ/4pGBZ9D/pGX4+Suup1ewFFBXKiQuQDY7Q/4NJ2mKT0eDeBeTWqe1H9JPpHNUkjpuSUldR2LiLDHdDh2gHo+ZBGeVizm3Cj1uleui+807i5EfweTlej8H33gJ4nTMzpQLYOTH+kCls4g7RdhWmq6qg9BIXXWIna/An1e2Snh4ouhX6TSuUph2Hw663SCJHA9Gk3LuawvfWCkK975X/3v6xuoLwodCxEg5p3KNT0VjibCHVQmFmt3wpa+y+Jo8+HEi9sB31l5KGYwLzPsIMEOSKq9k8aG/K68YlqDpivUKleST6jepzxemoW4D0FgkEkY7XlKPu31vVF3ITRUi0bL+VuNYQzFXB8M1wZCsOq+ul+LfVMSd5e9hVkhDkv05zI3Xd0c7Ha4OYl0x5CKRdVLIpnYEk9WPaksgAWYYbTNKAibUbmVRPAz11sj2YaenF7ym4gbU7ufUAHi2HBY2HIPEfwcwhfRmTzOEmKHRU+rQ4kVIaE9OzofpOoWiql0QPU7dERCYRlz6JaSEd8dqMjKmOXmb/F2hWbOOFoGp6qIByH1S+DPUKqRi3RKwGQ+pTdK/7w8/T9V3zJ74rRQodQbsdXnC5NUl//vcB+MWQEjf9nNf/yekW3Bhz851K6y9CXCq5cPaeUH3Ur8/4zFhe+vWvL4Pil+xCrYAkbfLVEjQHQpl6+DXU/Wy1wOelmSYChEjRPZ03A/qBH3ECGH36mTeGgrkOntp2oRGfQpp1xrY3F0DIvlp+hPc5RzCe2HG+9vkYjrfVwLNOj/io0GvuyQRrpNEswXIeEcJOQWagkWyvy4Nzgo1JpYSK1exrxkmlSpkxdwJ+eTzSOkyU/0F+74UkqJdMT/b64QVrfLZBJFs3f6s9pn3KlyEpXYvth3PHZRrdKPFGki0FeIb9xlZ6hVbOKPgTT6Mhi4KHzdz/8d40dmTr2Mhqk6RMNv3BXzhBd/8/R4sTpw8GwmjFb6o+wL68mk1nGTfqSZc/HaaJAn/UHRv1OyB2aGdJ3YdBrRxX+5c6dbNeFCkJ9vCaSe2pZh5NfCkbZz6gxtLpQheoCjEVGxlfiwsjgdzrUYOuWxdx36rzVVHTCYCpANBlTwHiUG9I4WA1U2hIJF2NQ7vSLIiAy8YAADCfElEQVQnriM6XNPd0fteNVkO5HpuflBP/joUNj8A+z5vV3T8W5B2NczcI96qnmibF1HtPQ+FPvcrve0OC6tcqkI6mfziFXJMKsJQ8R9CwN72lPq925+R2FpnfdHzThj+vnj3qdDjVujzgHLNsyaLnLVBytiFkH73MXDcB1g7khzsLDbcJsoAOoKsO15SkdObq4TYbq837HdsOAg0gwm1rG+IfyyVPglExE/EobBQyRn2GXubIclfs1YuPgE+t3aqoBxstXJREAz2NvoFms1m/CPSMQX3BsWeoriuGGfuXKZWL5HOr7bwT4FJvwvpUeUluOt1Pg4txYxYwxig65pzozoTvuuuj+fKN8Gf10hxTYX0a6D/4+qYrKEIFo8WuXsPmEwmbC02EnMTlV2YgEiHTlOo3UArKUs3z3YWASkSa83vIuRgT0xZJc+kf6LxnP0ShJQ44kOYoMmh5f8gf3so9tQJp8KwN6W4ptpTuLvqdDKzTqfkKzvjERfSV3JrKuJiWxnsLKMkq39TMfktcE8Jon7QFmYbjhmZTIjpyfCdiiJ7U4XYPIz8WC9Puuwk+KmDDvuVl6r3Yfu+EuJN4hkHpTSjUlLQ0AENeAeIiEBsCzxhtsC6m2F+irpzrzpLCCBR4zqvRPX/AA6Z8zvW3/9f+dbjOA7Abreze/du7PY2AYXVT5gkKy8RRosnwoeKPGLZWrU0k8ksiR0VI/pYwitUWCiLhqu9l1rqYel4SQ55MvZMZikK6GQfKzZJkq6p/IgTukeN6izI+1b+t2pzYTIJo2OoIhgASVI7nagCO0r/JCj7A9L8NUmnxjIpOuh0nmt2wzdR+g3E0WDwK7LoqiT/qrPgc7PeY8ttWqxjNFVnCatJlfwBSahW7dTLYVp8RG5W5/fTWez5UHyGFmoSo0lnw2n5rR5jnthwh3QbdGYTmv25JGFU8gUWL/GY03VgDHbJrBUtO/Lv9Y6E4L5g1txjB1zG1AOePPLPBikWrv2HIYFjt9tp+SpCztnt3+SJpDMlGa5jXB8NosdLcUvnZTXqU0lEq6QMrS7GWp7RXyPYJ5iae2rYdK1aAvekPOiRDYMVMvyYrZSeWszcgd8wa4qCuVmdJZuOkj/Vm1C3l4KOLWgLkb8qDwCAtKvgnHpJpHvCnRhXFeRCBzA0349ZAXBPhEJi8WCh2Kn21bMGyKZF1VEIIknin6Iu0oAwXrc8ovfa1CFsIP4zd/FD3w/oOdXoYzgiMo3JfnBTCPgouqefDjqfiblwvaqxumIzL0bC1kaoDtTMJ8cKw98lrwXKUyHU4bHBdzq5OvdxNieD/TqNv1zWW+JFq+tQ3nSfFLtrs41jTWUSf+iMzHO+lmf+82PwTAemi/9L2rXGse4usk1gN/V62lQuHdI6SbTiFcJ2VTzzgEgOL+xt8GBRwRD3RYySBKdO+rQj2AJE7k1FsgrpK4nH4N5qzyiQ5HptdjtfzYNoKpeChi4B2VAEW59Us8zrC+U6/zZL/d4fh0hcVbJSPb77fVhzg97fLrS/xIu6tXbLE7LBV6HbP+GcBr3Pbcr5MOwNI5GkfAPDt97AwtEX88r0V4zvixjNVbVdeb8K/HSFxKPBgKfh5C3KjnEAFvaF2cH6uEoDrz73kTnofYqjpmFW+EYOzvOn+z5YXaEoRkeNFRLSyA5UFpafK4Q1lSy8ySbPa7TmufMKk2dXd62GvA5T10hnlsf1irGX8m4U3FbzDS0enk8EprE45GQm+MGUlm3Kj+4WFEt3HxshNsWa5lbbUMnvH0Ms++NmBm67gysLYVu8kRBqNpn5uR5usU6FeMWc4F77Vd1HZh+R1C78RbpfjwGUe16QbhUQ+Twfj0K5TySfd3+V0w5AllMzR7rJlEUKkmvBImYGwCQ/GB2jKJqVrpH5SCdlX7wCKjaKdPuRomS5XoGkuVruH11yvaEQc2MxKZGauXvpxI6l7VZeDFse6/y1nLxCiHqJGinmY4WdL8uxq+L+th1eneoQMEmcua8TpGE3GceqIZE0lsh8pVKGiRguv+VozfdGniD+un006i1WX1GW0RXkuv9T7k9Vsnn059JlP+wN9XvrC6W7WtGpc9To7fJi1RGZhr8ruZ9ghcxnQ5EUWHe9Zjiv2N2vUpUK+V0goXqj4a1VvR8kZet+7lh4FQ5VscRkpms2vNSi8Z8rXi7xb/Uu/blpEB01hPOdE7gtFE448GG7Mbvdzm6fc7APfEn2xR5ddpnb38Vqgil5GMnlZosoJpzbACrP1br9jPKqZ6gPZBUopLz9EyXO1CkeuP1OVet0wc+w9gaJb3Vd+B1hnougpJB171m5nEHj1xF4+k9Y63KN702/FmZm6QnJiWfAmRVyn//V8AqVIqJKhSpsoBTXq7OMuSq/OMmDdb1Eb4nU8zYp3upixR0vwFd+6vnb/UxPWKx+b/lG6SRb24HHqAo1e6TBwCtUfc772yh1KMjM/htv49kIeLoc6s0ea7XJhDkwTXJOKjSVSZyY/Yn++MIGix/hLs2a2VID4SNac5BuNBRI/qP3vQf3fwV797Ic0JSuD2I1sAI4UAiMnWf8B/YGIUvaAtVrUsESIZv2uf+IiXj/L0Eb+/1NOF7sO47/KqqrFWwbp0PYTKpiR81eSSz9eoqa1VGbI0XCv1t6AyRZlXiGWmPa4QokY6YY5cFMJrK63sTi3YtZvVzh3RHST7odZ2S1dtL9XVh9mSS7cheoE+z75+p9gkBkDUa8L3KenuhxEwx8Dnb9W1243fUa1OW2Jio94RUiLeQd+Wx1Fn7xwqIMTDOOuYPCbU+rNxehA8AvSd8tsPx8SarqmLR/XiP3uFt6zRNLJ8ByjXTN0cAti5t6lXp82TT4Zbp+w5V2tUgvdEaX293ev1Oh795cI7+V7jp7RwiTTKUrfyiknCcdcms0TKr0f8im3ydaPX4oRI4RXyZFIrve23VvuRlubWFvEm+CzrKSD4W46eKFpfvNbEHSqalKkvtGy2/urZChK/oN/2/CsH7hpewA3dwEO9UEWwDC/SM4redpar+d/B9lg7houFqy1RYgvnoTl2nOKUAKlR0xPC0+6vvb4iMd5Sr5RbOVeZevJy/tFs6cogjU3d8XP1P8DzyRMEMKRD4dbAKKl+sTqz7R0uEcP0M93gGiAmKY3vdSbL7GaxnQ7RpuqYrkgiAIqDVu+t2J1R9UCkThw9gz4HXOmvAWr07XeJ4dQ/TteSUrvfswIMFDqrDtfbP3Q/WbUy4Uz1WV9GPmm9Jp45+sLk7lfivMygTNtXDPI52ROj4UVl4MS05Qb7jCh0mioded6gL/kFdlrtJ1RzdXQvREvaRPN5fEm4cHy0F8ZpL/XAWog3Ff+UbAKb5Ouk60jtD9Fhj4jLqIDtD9ZpEc1qHLxUJWUj17JpvI5ugSoztd9/VAhcdtlctbSNe554bbl9jw2a9IElzlz7jz37Cgmz7G3fGySPZs1Ui+V26FL6xqqZ8OYPdNorbXg9TWF0HRH8Z/sPtd3vHfwxcx0KDzSDyWMJnE3yj70yN6m81iI73HZURO+kGZUNtesY9gMyTo6vNNFVIUXqdgqIPEU2FDJKb0hMULYqeoVTNA1lpHQ/skU1v4RMgzGdTdUBy4vvIrxvnBrcVQ7/CIyXxjmL5mIRcWwD37FNLUZRvoHZ7O9z1eIr23Ql46cpQk287/ez1YvJorGOQNIdEjuHSM8f7uXbWSebHwxa6fyChXkHqmrZVjPkmhaOAXJ7JlRcs6Vww5TCj3vO7OTkeT3BMesJqt+JrA4jB24AMSi01eLn6znkiYBSd+B6cVHOy+bQd3QrSnhmyEU2TMVf7kh8LA52GKhtAQkCps/4RT1WohpS5P8i+saqJT7T6RtlNJiwHUu1Ryko9M1vcgIkcKyS98aOfe31lU7RI/WIem+DQrD07vpGefo0kKT53xpBz2lsSYuuOauATO0MSovglSjNNdi7jpcFqeEBw7gy1PwOcWkRs/UsyNEYKrjgR4NOh5B5zTqO+GDOousVfUCcYx31gpnu75wEBgaQ4dxJ5mCDRDdqmRuOB0Ovk4Bkq6qlU5vCq3cIIP+No0e/mx86W7qTPKMSYTdZiZXQ1lCj/B6upqKVjsfk/u9TYILvmNa4NhZxNG9RZHsxR+dN3JA57i3APwQzzYijXKXsPe0vskewVL16Bqr587X7pTT9rc+XsUlPdBtS2cyZcv4NYzVuJTpim9lG/Sd4davFzH/hcrKuUugJ8niRe8ijjTXC1k+wXp+txMU6VRltoNvwSZ44t+NY4dWCxk+fBh+tioI7gLcZmaAr8OFn8pbro9/zxhDZBjmrZWFHU8YO5yCU+6hBJ2l3uQjp1OWbOG/FvmSk/4xokfX3O1Pu/T4zYhMarIiQATFsFUxXrb/UZRC2tDmC8tLeUKYAb6gt9q1/gVQGmjwtccxMrpwA9qr0sQUvmYb+S4/8ehjP3+Jhwv9h3H/y00VYgecvwM6K5gZaz9Z+v/VrVo53wFa66TRervRF2uFGHChwsz2hO2QDi9WEztFZI8T//xNF3zPsO2U8GY9o2BlHNFwuvvRqWL9bvyQoNJNCAbwLocWKr5vaszRRJCJVPhZgXtflfNxo46QZgouiS4T5Rsct2dh38lVl4ismYq+MZA/6ekm0CVWM37Tn4TVYAEEsSDQXbpINxJIZ3vU80e8UHc/Bf7ikSNlUTIQE1Hok+sFD08pfHcKP4DNt4JCtmQQ6Ln7SIvo2IOFf0GPw4SPzUVytbK8+fTSUkpkxntUuiXIAGaiu13ODjwkzxDikB8d9Ib2GfmSIDnCXu9sL1/GnZsTI1/nQnfK9ikbhT/IT5Buo3NaQUwbqHx9aLfJBlg9Vcmqv1McHoAzNQ18ez9FOZ3VXelxE6F5HOhyyVqLwmnU5hvER3IXMzYCaM0zLkdL0sSXecJ1eMmKQ57oqWeeFM98X3vJLzLWcbxklXyt9+j+k7Kk7eqf0+QLoqBzwj7WYU9H0rCTtcJrENdnkiW7f6gNTHmgVXN3szIh9pA471yTag3J/uDj1XRKeYTSdek6Zyc8yx9yzTsy2OFXa8xNbYHI8/IwOTpi2bxocXli9FYrEtCpggTUvXM+8bLZnDst2oGe/q1wlb17M5ww+3ldiyS5PEzpKigupZV20UqTicVW5cnsmp97leP735XpN50G7YBT8s5eXiwHIR7/vPsRNv7Caw4X1+sOxSK/5C4S+VXUr1LYsE/r2l9Bj0x8iMYoFEGsAXIfKIrQkaPF/lclWxZ2ECR2ZypkFQFIW/1vq+1S9oTI96Xf6NK4JitLsmpbuq1wZ0IGPyS+rObqySJqCo+QWth1qOrcEd9LQHzH6U640nXOu+BGpF7PNEPAo+FAoX7uCp3qMdHfS5dckEKL++OsOc/4qukkSzr6wUbkiBD01DIclcCW0UMA/HVnbZGLz+64Xa9HzFI4sdd+PBEfaE8l3V5Bumm5p63c1EBvFgBZgWBJcI/mk+rwRFh9Oxj3+eckPcWD/34D8rr1T5w/w1kho4nbA9kOQOUXsYnRHZluI/IhFfXd6Lr0DdGYlCVLPCxxNh5QhoIG2yU4LI34Z2/gPKu8LpD45dWvlGK+CoCVkAX2Yv4Rqs7gXvdLSomOuuMyNEyl3TGnzEwVV8sq9ouhdXc+WrCW9tuVtVeaWaWPOuqZDHA9A1iP9FZqfumcklW/91Iu0quiU7yzC9O76t+KOx5X4panbE8SLta5Lx1vrzbn4e5Cer5uWwdfOmt9290OiR+7exeZ5+LcKy61utvh+/7SQFBBbfSwbHoOmk4IGu1rgN/1+uiiKOC1R8GvQjp1xvigKb4WaRmQ8BuyA4yPl/FWf+htxe8W6WW+UzMep7fE1F3AgMkzBQZxU7IIDpbGolrKeWOEsgKN+aFgqp/wVSyUgrWHrmyX737M3o/VDow3guNZRLnLR4teSMFZg2/n3dbkjh5iOY37QheodI1qOoA7XGLyFYGdtcrTXWE7jcJcVgB7/iTucY8iW/CzsYvXqHuVbYefhggNhIqVO2UuGjzw0d+XB3BFiAxjU5xY35y+3/bFrX7YMk4mBMC32v2DM3VInObM9s4Vr1L5t8Bz8j65Yldr8Gi0UIAViGgC/R/EoYcocS+bzQkniX+oyqyvn+yFF33z1Pvt7tcQIUDViZAcbGH7YzTIVKXbXPYbWHxkUJi1ptQoJmrgtKl4JyuUG8Byc3tek39XG/7F/w85WAHc0NdHScA9wBjganAp8AS4BNgiuv1e4ATgOb6OihSqOH4RMpvHau5PwO6SKy2+YH/znp6HMDxYt9x/F+DySpSU7pOGrcPWu/71UmY6ImSyFDpQB9TmCTJogvqTGaRIWgoUBa2rmn4hU1NcLmOsFdfKNJOOnbPsULXS+Vv/yfVQY67U6VUk0yrzZZkWMRo41hTBaRdCadmqwPt6PFyH+gMlUHa1YcdWjrsiLF/rsia7Z+rHu99N4zTSJr5J0PKRWoZAJBE3snbJLmqQt9HYOLPUthQYfIKKXzoPv9YIWK4aIqXa2RxoicKC0wl2Xo4iB6nTia7z9N9L6qOK7iPXt6rI5SsFoaxqmMLhAG74jy9bMKhYPHWbw5MJmFzqe59q5+wDN3/7q+G1V8YvjpG3vpb9J0hIHOZahOQdjWclAGnFymTyVuS4etYeFlTv2flhaILX6EotASli3zHyA/V3121Q9i9mx9Sf3ZLncjq6dh+patlY75cQdboCNWZ8MNAvYRhyvnSJalLSlVkCDNZ16lTskK6qyu3qsfDh4o0nyrR1xHqckUeevXlSuLA7rzfub7rcE4d8TD9uxg3od1y3uS7OFiVoE4CNtqbsFt86aCR89hg93uQqZGVtvrzhL0PU/Jgyja1fB1BPeCsaiGaeCJhhhRldRLKAV1gx3NSZPq7EdxLyAWqxOk+l8RP9idqacjcebLJ1KHnnXqp7sPBuU1SDPQkD3S9RNQO1v5Tns8jRYirCK3yPGv7POjYxQVLYPtz+u+esFi6hVWImSDPtcNYdMAWJDKbqkQFiDRcyoX64wobJAQvFdKvleJq3CnqtaH7DfJb67oJyjdIh6uuC9O9XrnXH/fLLj+ZfxY5eK5K0RHgYkPvsMQyLv0U9Wf/FdCRpEJ6S6JO55Wsw4EfxVdp1SVKgtW8OIizwr91eYr+T0r8oPIdOxScTrkHdIXAujwhMfXUeFWtv1kkGOclGDrWm9Jv4BPXS4aEb+U2NsTXcW0wnNVLQVDpcgl3lVqZ5AfOOoWCStkG+K6ndKD+jTCbzFiAeaYlsOpSw7it/2PcWRPN6iQIL1V0n+Z9Jx2YeQrJ66YK2PqUdPTqpMeOFQJSZK+0aKRR8s9ex9nls/E2Q45uMc2ZDX9erZcLrM2RfZRqngvqIcnVjop52/4F6zRJyqOB1V/sAVQkkT4PSvF+4rJWgqQnTtkG0zX7EbOXnO8RyvoexJwwSVa7O7j/LkSOhv6PHZu4P6S/rD2mTsiIz0uWooIOtftEoUW133F3Em7UdLLvek1k91Z2ssjuGyf3iGoPuPtdmdd18c059bJedrYo3BGWnyukSt06n/GQxB86FZ+uF8PQ1wxd222VT6amGknJUYXf081LJPY9ffMA3qiStfxqnZhCzhxY849OxWT7itbzum0Dt4aquwqDq3/FvP1psPgaiO8T+lzKRUFQkYpRvcUWIEXqulx1XqZiK+cmD+LK87cRFNUB4VOH+kIhravsgwJSJI9VtaNzBYvd74pSw+53DENT06by0fmLOX3al+p40V34STxD/dnueKgzHXAdIXq85NbyvlV3mqVcBAOflWfHMxfraIbKLa0KQSq4LUJSFCSTtKvh7DqIGmMcA8mFlqzomCTV+57OxWQVm2Q9VXU/m23y+tbHtZYGbhWI+ACPHLbJLJ159jrYq/EBD+wGk/+Q31aFjXfDqsv1x75/jhB/PcnS1btlnilYDDgpKCigzm7nQ+AWYB9S2HsJuB54GTjR9fotwAdAo9Ouzgl5h8ucf9CuRIHiPyQ/oovdj+OY43ixzwOvvfYaKSkp+Pj4MHz4cP78U7PRP46jhslkIjExsX0wYgsQlqNXiFrmsPe9srj0f0z9oWED4cxS6cL4O+EXLxIwJasgT9Gh4XQIe3hBujKh0LclBwuwSaWQUbJapCZWX6FnfBwrjPhAfu/e9yg7Ehn0oujsT9E0ghcslUVAxX7f85EkKmr364ukm+6DLGOABAjTa9frnTOOPxRO/Fa0r1XsyqZKCYR1njih/aUYp5Nq8w4XyUydHKZ3mARaOgalXxyM/1EvR9FZFPwsm7kVmkAjfJh0Iyh8bQBhnzcUdm6TWp0lbKkmBYs8pLfcg7qCXNJZcHJG5zyfin6FLY+KDIEKW11efbFTjvyzQY55opGNbTKZ6Nb0FebN96oZZGabsAyPlUxWmEuOT/V7gyQwmyrUc1lH8ImSBLzm3t7p15df6qCqn4bhO/oLmf874w/j3nBsfVzzD0xyj6r8ekB8Coe/K1KHRwLfGPHdyHgIthsN2GkoFGm5dTepf++yDSK7p9gIApKg6X6TPiFsDZQiju4e1iGkjxAH0q6FeGNyvnz/d1xU+TV7Mj9WGseXDXmXPFssXqrEaMFSTN/14NpdGVyXqSmqHSuMnSvdn8vPV8p/PZT/MIvixU9QiT+vha8jZMOqwr4vYdUVamkxp0OeZ12nWkWGMFJ1DPejQXAvkYhRdep0a6OSoGKkjvpM1qxNms2z2SqSaQeM3o4A/HGurB0dEXNcaBf3hfSVJF3egk52jqTJHDlekbz3Dpc1MuFUvR/yz5OFWa8qgNobpfO1o5jrpyEdb751+NIHFvYUCfPOoNddME4hM+tGY5laGh1EOm/YW5JYUMGd/PRIMPrWZLE6EWwm+LVZERcFpEHi6fQ46XfMx8Kzz90Rq/Ou2fYsbNJ0pnaEIf+Gvo/KHF2XZxh+ozaISwog4QRN/JF2tZ6IBK0diSr5MZNJkpo6CS57vdx/Op+g+JlChup2gyE+/zHrRx4Og/WJYPbsgjNZMANvRMGYknnGzw3pTVlLC+9EQ8YuhSxq7V5JfK77e/dZAc3FzAqA72vh80KFvxGQY7fyWgU0+irWyw23i1T8WkUSsLlKklkFComtvwjKPS/I/bHr35IY9dxztIlr32/QFCRiXQn/HYr4Y/cH0oWx+ARRBfHEupvF226hRumhLk+S0L078Wx1hMTTJHmf81WrB2RbhPaT4n30ier9Yc5s2V/qsPMVmBsrPrmdOj5Xcl217/3/KxJmypzRoLGY6AjeYVKg0aEuR4qIASnGsdAB8tdtgeEJd2FHJVt9OJi4VPzkVZiRCSfMluLx3w7Xc65Thhnksq1QrQ21++FLf0nweyCsaAnL4uHOUDDXGiVwa7rfyej9oIuoRox8jul5sDNKIze/8xUhzHVCccFpC+DBUpjsB6f5tV93TCYT5v6P45j4m+SFPPZDoxOGceqw+2hOPNt4L1j9Ydg70oWsUk3a+x/4/XQ94eFQcDRC4VLpQjWMtcD+r+GH/p1bH9w5E1XRt3SN+DOXayRok8+VHMKAJ9XjoQPgrCrxrfyrUb1bPEQrFXvmIS+LIpMKgWlwRol0Vg94Wv1vwofLualsj8w2uRarr1A/G2O/kRh24LPqz26phz+vE9WbI0FzlSgTnTAb/BWF17b3pEoyeMNd3JM2lF8G/Id+qbPaj5lMMOg5WU91hORvu8KOF8VfUoXceSLrW6rwpATY8ZL89bQuyF8oz/Kk38DizbvvvIMX8Csi1RkD3AesAXa5/t7nen018BtgdiC+q8rvfVFPds7+DHa/Lbmdznak/z8Abez3N+F4sa8NvvzyS2699VYeeugh1q9fT//+/Zk6dSpFRcfAE+w4MJvNhIeHYzZ73Ia1OSK9pE3APy3t0Drt7v8WfugvzAqV5I69odV4VcFS/HfKU5x5AJRKzFY/WRC7XtY5+Y2jQdbb0i2j21Stuwmy3tIzYbvdABOWqFlz4UMlSdJcoZZQyvteuiW7XqL+bEej/N7liqLw0SJ6nGhfxymkPO31Eggv12hUgwQpqsQ/CEt2TpieCZP1rjDMNNJ67P1ELX1wtHDJcGmfu8ptstnTddA1Fnc+0M6ZA7+fBj/+zf4YbmP0LTrywGBJwOi6JDoJs9mM394XMW1/BjYcYXHpr0CPW8QDVBXQgviEVm1Xd9h1hJZ6keLc/JAEzh6YcuZGhl5cTb+BGpmV5HMkOa9KfmZ/JkmxL3yUEl74xYtUiq6r2+oryQgd09tkkuJAR4lbFXyiJAj276Jmrh0qgeEutOkkdWpcGy63D5gnrP7iA6sz/tbB6i++NMPekN/cA1UBPVlYC09671Z2QpQE9uHlkAtZEnOZ8bO9I/ix1sn+FvgsowNP12MB/2QhgOz73Hit2/z/k3TcgIAuQjZqVMR9pWuFrb3nfXWBKONR6RAdoJFCrs4SRqqO4X40yJntSpwqnunAdBj/k0jAqBQRfCKEqFSgKebZG+Q31RVAC12EBt2atul+STZXbjfGfSM+gHObO0fWOBTqC4UJroN7nnF7fLSF0w47X5L4RoW9n8gcdzSeTiGaBHtHKFsvksM6T+CSP+HrcPhVk8izBUhC6wj9jaw4ibFAgEktDUbsVLnPdDLIxxqb7pFiTaGmg1oH7zBRajirGmKMsmOLzV34TzXEB2pkT5urZA3PPYScvM7r6jy7yPurEJgG439oLeZ4IuU8UcYY8qqho6bPrsd4KFx8nQzXK6g7t1kmsrIeSu3qlPDiOsmf51oUHeNxp4j8+KjP1cd1jBBfs5U5sfBCOfxiM8q1btz5Od4NedxbCnVBvYwfMOTfIpE4WNGt5Rsr3nY7X4QMTTx4lNDued2IGCn+S+3eZGPD6CV03QtLmzRzZOhASD5fraDiGyfEh5SL1FL3bsUAXSKuaqckTY9F0SvqRPH5Vkn/u8mHy89Xrzurr4ZlJ+slmje7ipMqounhYMwcIRekXd259/9fROV26YgtNBIQD4npG+CcDjq90q7Td/+7464umr18r7vF5qSz6gH75wpJS6VU4hMpigIqv0qAJePlPlORt44Wg18RP2KdBUuXC+UeU+U4LL7SBbTNGEt6VW3jRD/4VwT4VRrX8aSkyTzcewrbB6ljk/P7XcgbV+3l4dPmqY9r1Mcw4sPOqeVYA3ijEnp6gV9Z+44ws9lMaHwfzI0F8NNQI3ks4xFidz6Brf+jYn3jCZ8I+c2CFXN7ynkiv/zb6WpP4UPBL1GaCdIVRJAtj8HqK8WXTPXdh4J776bII9SXbZSuvx/605yt8Ir1T5b3deRprfqtjhaVO0Sxa9yPkKToKnTYW4lMnel27HKR3GOq82osldzjng+NHZ4gv0na1ZKn0yHL5a9+JGipk7120a9q0rq7UaPbjSK37Im6XEKai7iov4YwD1KkHK3ZE0eMkBhaIynPKBfxSldwHvSC2AcEeRD5YqfKe11r4fzPPqMZ6MrhefalAE2YhHjjibpc2QvrroVXuKzz/in6po7/ARwy9jvG6EQv//+7eOGFF7jqqqu47DJJXr355pssXLiQ999/n7vvbs+uaWxspLGxNeFYVSWJTbvdjt0uD6rJZMJsNuNwOHC20Z92v+7+d4d63Ww2YzKZlK8DOByOw3rdYrHgdDqVr3seo+71v/KcHA4HO3fuJC0tDYtFylxmE5D9Kc7Uq3EmnAWu97nPybn9BcybhF1gd0hxrN057Z+DZcW5EDsVx4nf/33n1FJ7sFBn73brweM+eOwmK87eD+IM6Q/esWC3t7tO1w++nstzHsXi2oy1O6fAXjBtU+uxexzPMb1OfsmYnA6cv56C47TSgxu+g9ejy+XQVHrwmAz3nr0J8/rbMPW4GWeXS9rfe2HDsex5H36dgf20knYJN4vFgqOxFBMmHLZQsNuN52SLxBw1HopXYoK/9nkq/AVT9S6cqVdjdt2bB4/dGoY5pD+mygz189R4AKdXCM7gPu2ulfs6UbUbc1M59pYmTA6H4TqZd76CqTID9nyAvUf7pLDJZML859Vgr8c+bhFETzj8czrU68mXYulyKU6TBYfiHnPu/RSqtuLoeXe7193Hboo8USQ+FefU0fUwmUyY406CTffgSDq73b10zOe9wB7CeKk/cPB72j1P8adD/OlyjK7P/ivm8qamJnLSv6RL/WyIOwWz6zMPHqPTieUrG07vSDi98C+fy027XsdUvRPTqI/V5xTSH7v7efdYzzp6nkw7X8W8Se5ZZ+rVYA00HKO/zV977M7mGhxlG8Ev6aAM6MFjtwTKHOtoxG63Y7EorsfgVzq89xw974TAHgfvsXbnVLVDCuyRYzFbbUd27/nGwszd8rrHM+8sWQFmG47xP4MlCOz29tfDGgyj54BXKBYU5xQ6FPOIj3CEDmk3nxw89uo9mK1BOAK6t5snDznvOezQUi1+TlZfwzk1ecfwQrkwwvo11BPT5vlwOp3sKtnFsyueZWjcUK4fcn37cwrqw6WFcHEQ1NF8TOMIw3VqKMbc804Y+CwOp7n9WmxvPigs9EwE7Z75g/dkzDQsm+7FsedjzP0eav/clG+R+WLkJzh8Yo3XI6QvzqRzcZj91M9N1CRMg1+XTY9rnvyr4j1nbS6mym04muvA5NX+nJoqsfwyFUfXKzCPeNc4R9TnYxn3A46ANOU95sydL98dM019j83MgZY6TFZfzBjXYotL/sVRvJoW31SysrJIS0vDtvNfsOMFHJNXtfM8O+y5vOhXLL9MhPR/4Bj8inHeG/wSjootOOuKD8olt7v3Ttkr18JpPrgWtx67F2azN9TsU8YX5oZiTN6R2BPONMZ7HVwnh8OBOWYKpoJF2L1jjDFsm3NVzuWFv8CG23HmzMYx8VfDOZH/g6xVQT3V91jtASxZb0NAV+zR7X02Orr3WkL6kZIt/nU1jrXYPWJYWhqwuBOTZ1XjtPr/pXsN05ZHMRX9gnPMPMw+YcY5Ytg7mFdfjqN6L86IVn/TQz5PtfnQXClJFqfJsBZbzVaeCIfemU/g7DrJeE7LToHi33HYG3HGtnYcHDync1o7aZXx3sIeENAF04RF6nlv1RU4vUJxTGklER7OHBHUUsFPtTAtH5qtfoZ77NPt8/gUIHc5LTM8ntftT7E5CYbshzEmP2NshAW6XqU9p4PH/hfvcw/49eK8A7ClCVJNJsM5OfZ9xZxY6LcPcLZ/Zi0WC46oCTgjXQlXQ7xnhqDemKMn4fSN+0vjPffrdrudrKwsunXr1rofAJiyFvOqS6RLXzEXNFr8uSQIEn2rjOfkcOC0hcKI/xw8r3bPU/QkmFWoXXPNsdMwVe/CPn5pu3ns4DkF9YUTf5REoeuZ/8vi8sB0yHgY5745OJPPa3dO5C2UtXbf59gHPIfFP679sY/4EMvvs2DN9TimrjVep6Fv4miuwRky8Ihi2KM+p/9DORbPczJV78bknwJhww17vKM+p5Z6TPWFmO0NOExe7Y/dKwLzmRU4TDZ1DIsVbC5C8JHEsO7Xd72GqXAp9u63gS283b+nahcOLO0kEtuek6VomXxtSxMWb+MzrzxXDvM6BfeT/0ARXwD1+ZhbqjGF9DKek3e4+EMX/dbuWpnNZuj7MCkLn6KfN/QNKKCP65k/eJ0cdib5AUVC1jQc459XkVKZgWPKauwO4/Vw+CbiTL7wkM+N6nWHw0GpHRKyvcg+7UlMbcacTie7d2yka2golj6P4Azs1T4XFtwfU8rFOF3xc7tzaqnHvGwSzthpmPs9bLxOQX2xxJ+C88AiHE1VhmM/rOepz6MH9/rtXg/uhznlYkz9H8fuE3fE8Z6p72OYyjfiUMR7H5RU8csBuD0UBtmCjLGzxYLz56k4feNxDnvnyM+pg9c7nPcqt2Ha+iSO4T1wmnwMOUvTmmtbu4WcLe3niJY6LF9LEc/Z7Z84Br5oOCdHfSHmebE40q7DOfjV9jFs7veY8xbgGP4hBPY2XA/Tzpcx7X4L04nfYffvajwnqy/2Md8eXLMO9zphDYPxS6C5Gou9CafZ1v7fR47HMm0tDr8UnLYQw17fMfJjnKXrMO3+CGf8LExege2uk/n3U3GGj4Te9yj3SaZe92JePBKHLQRnj9auyYPXyS8VJvwihTPP+BsgXMg+hnySfxo0lGEpWYkzfiaVVVV4A3sQ2c6xwDjgYiAaKAD+g3T+XQe8Dvi57h/DvdfSiNnih9MWqj6nmMmYg3tibyiF5iaxgOF/Z811v26329m1a1f7eschzumvxPFinwtNTU2sW7eOe+5pbVM1m81MmjSJlStXGv79U089xSOPPGJ4fevWrQQEiFRBWFgYSUlJ5ObmUlbWyk6IiYkhJiaG7OxsqqtbfRYSExMJDw8nMzOThoZWBnrXrl0JCgpi27Zt7W6i7t274+XlRUZGe1nIvn370tTUxM6drSxri8VC3759qa6uZs+eVikPHx8fevToQXl5Ofv37z/4emBgIKmpqRQVFVFQ0Mog/ivPyd/fn4KCAhobGw+2tnbvlo7vjuepCJrCPu9aIKPdOdVlL8cdym3LrqRvX9qdk9keR5r/IPz6PPS3npO1pYQ+ACYz2zJz1NfJeRqUA+Xtz2nnzp34164j1rsrdf7CxPq/c51GEtTtRpr3fsn2LVtwurpUDp5TzQD5hxnGcwLwadhFl7pyvB3N6nOKn0kdYWRuz8Lp8og5eE5+kylK/R5noQVncYbynBKaQvH3CsIX/tLnqfvuf+DbuIsd5ZGkDz3d8Dx5xb5ErxO7qs8p1ompoZDC4hIKXL9Lu+vkfS3W9LNoybETVpNrOCe/0DtI8voSn8ix6nMa9AKsuY6irfMpLIo87HNqiw7niKoq5b1XHzQcS0UWWesX02yLMdx7KfsXElz9K/sTn+rEvdePHYO2y3VyHevfMe9V1DRSF3MnjV5JVGdkGM7JbK+hV+bJ1MacSdC49/6yuXz79u2UlPtTHXYrpgoT3aMbDOc0ADA1FlPf0PCXz+Xpe7/Cvz4DRnxAbm6+8ZzWjqPJGcjOxFZpycN5nnwrg4gMHEdZ8ExiCMTL4Wh3TrGFLxNd+iGO0CFkxLZ+tvucmBuPpaWKA5H/oDDyyvbnVJoIvaSLN3D/AcM5WVrKSal6j8D008l1DlFep72Bl8t1ch1T23OK3H0v4ZXfUhE4EfOJ3xz2vWdryqdX+eM0J17AtqZWZr37nOrN0bT4DePAnhzqDwQYr5OzGYsjCr+gCFJBPZd3vZjcnBzKdrd+r/ucynPWEp73Lbttp1Ln13TY855Pwy567JHuZMeA58lomkBbmAPM/FwPP9fDF7ktdK/OaHfvTdx4Kc50uLN0M5mZmYZ7L8YKL0XCY2X2YxpHeF6n9D0X4WsqxzlzHxlbPOaIPn0ompxJxoJB+PilUZCRYXierC3lhEXdSHNTN5Kh3TmZHd0IHvAryYlDKSouUZzTGZSUFFG16TuqA8dozkm8WhPLy//SeC+voSfxBHFg7SfUhkxod04mZzOxYRdgqg8g0ukwzBHpB+7Av2IZReP3K69TVaMVqyOQTNd3d3ydfA3nlD7sa/wdB9hZlULD1q2UlZXR2NhIb59IbBZf9m/6hoqgaQfZtIc7lwfW7CQVYM8HlHd9xDjvNc3DvOF2dqV8RJ1fv3bnlJubS3lpITjtOE0FxMTGGs4pIvJGQsKjCADFdbqCoJ63sC0jA/v+1mtyOOtThLM/CSwie+9uUqNPOLK5PPkcHFuexFyy/OB3tD2n6sZheKW8T721B1FFRYZzCq38mURbKOawwUd075mccm0KW6DRBBkZGe2vk9NBV/9RBNWuABx/eQw7YNujAJTuX09E+iTFvTeT8PMc7Nqxg4Y2v/2hnqfKX68jrHIBmcnvUO/Tg94DRra79+5yruOsMKiu3KA+J7MVu28y25iFXRXvHWLe69VQjdPWiDeKGDYunPCYiRQ3hJKvOKeiX28ntuAlKgNOIDvhebr17Hvw3vvUdDbP5T8nb3BCQ5s4wmKv4sYQWNsAKxpod6/6+PjQwyeaIAuc6g+5+blkeMRGRXlZeDXn0WyLITiyy9+2zz1Q4c+cGngnGqyN29udE8DahiR+L4fnI6F2+xIySn1az+lQ996BPKwtpThCHiTEK5kk+MvPyel0UlZWRnJyMj4+Pq2/u9NBkimF0Ky3aIiczs7drXKyVlMLI7aOZEQ42J1VB9/T9pya1j9AdPE77Ez9Aq+IAUe0J0z2G05oczm5O5dT3tTaZXHwnHJKSdh+N1Z7JTvSvvlL4/K6BgcQQWFeCVVVGe3OKZ9T6Wl5j6qA0VTkldG1W5zHOSWT0P0VIiLj1Ncp9XL27t5NdZvjOZJ7Lzn3LkJqluGY8CsZ+e27Zv//kmPxPKe4wj+Iqs3Gbgv+y/NGSXnvE1a5AHreQlGx4pxqvqSmaCd7Qlolxd3nVLB5NsHZT1MYeTXVAaOOeI7o2yxk64w9VWBqPa++fftiXjwGS1Mx5UHT2JfwlOGcfLt8gtnZhH1/BT16xPyl18l3z0sE1a5gd/KbJCR1NZxT2t7LCajfAKcdYFtWsfF5GvMtGZs3H9yvtF4nOzktkNMCfe0JbNu2rd05JeXdT1jlInb1X0s3MJxTalUpgWVrafjtYnaFtBYV/op7r6WpmLWJMLumifeWf8XwiFb/vPT0dMJznsS2eS4Z3Zdhz7EDbeKIinTCmrri/+sNHEi4jz79h7XGsI5G+peuxlS6GvwTKQ+ZpbhOZ1HoNUauU4kxNur885RK1z6vEuQfJPHeEc57PTPfwG4JZvfeCvr2TWz3PGXnFzKnBubUQJH3EPLafE5gYCCpzQswFSyi2RrJNt+/8pwOseZGjmNPyvs4isw0Vq7GYfZrd069cubjBeTGP0yCd3i7vJGtuQi31pdj/7dkWFul7t3PU/WOLwgGDlT7UJzRPr9XW+JPUPQtVJaEEOZfajinAdtukw/7rhuZ7ryR5zlVpGAvrcOdQz7c9Sn+wNtEln8FpxdS3eRjnPd8MmlZexdZIbfQ5BXf7pyKioowb/o3UaX/YVvadwTE9G13nfodWEplTQuNkVcor1NyuJXQ5PPIrw6iRBHv7d30I7TUUOMfAlQYzsnWlE/vrJNxdL+Fxl5PtJvLu+Q+SHDVUqpPqcDb2xsbEAt8BswFNiCefZVAMDALuBm4FOnsq/O1Y/8mkYLhqwz3niXxI6KC44lGk4f9/9q77/CoqvQP4N+ZSe89oYXeCb0rAoIgYkEUFXtb116w79p17a69rT8La1l7RQERqdJ7772EACG9z9zfH28mkMw5KZOZuUnm+3mePIGcKWcyk3PvPec977vjKdh2foh1nefDYYus+bPXiNc1dK/JZrPh8OHDldY7anpNnmQxPL182EgdOnQILVq0wKJFizBkyJCKnz/wwAOYN28eli6tvNFVtbOvVatWyMzMRFSUnDBzZ1/NO/vWrVuH7t27V17pPrEOjsDoSrXBKl5T3n4gZ1N52rKEhvOa7CWwHpsHi6ME9rhBQPDJ1DcVkfe7v4Bl9ydw9HsLiOxQOdL7h2QgpgccI2e7vqbCw7AuvQ6WI3/A0e8tGB1u9s1rQv0/e5Z1/4Bly8uwnLcdRnjbyrdP/x22rf+GI+0pGHEDla/J8nUYkDIajmE/ub4mRxlwbCEsISmwxnb36GuyrPsnrJtfgH3oV7C2nlT5tTrsQO5W2ELiYIQ2d/3sWSARpdaQSnVvPPY+GaWw5+2VSMjy3QoeGSMKD8O28z0YyaPgSDyj8muy2eBY/wwsW16CY8xyIKK962s6PB2W4uNA26saxGfv1L5rd5ItuRGWXR/C0eNJGN3/6fKaLPu+gXXxZBid7oSl/+see00lJSXYunYBOnfpAltYirrvGXNhLTkGpF7i8bHcsuZ+WLe+Cpy/C46w1q6vadFkGIWH4Rj6VUV6J0+8T9bfusOSuxVGu+vgGFC5Rp3NZoMx6wxYji6AfdTCitpjtR7L8/fBNq0dAAscl5XV/X06Mhu2uWNhNDsHGDGt9p+9vJ2wzuwHhDaHY9DUSmn9bDYbjJztsEyTtBr2iwsAW1CVz9hXsC6+AvahX8HW5pK6jxFZW2DZ8ymMFhOAuL61f58KDsK6+m5YDvwAo8sUOHpVriO3bv0bSFo3BfcdBe6fvBK9knudfE2GgVU/9Ea/onXYWgJ0uKqs8vuRtxMrZl+KHw6sQff+z+CKwQ/7Lnp966uwnFgDdPw7HLEDKqWEtVqtQPofMAoOwTilMHylv6f938FybDGMtKdgC450/ewV7IetNBOOqO4wLCfH94pdcD+0AMJS4Ri90LXvZflSLyc4CZagKM+Oe4dnwbrqbjh6PQc0P9dljLAsvgrWff8DLiuBA5UjBy17voB13xdw9HoORnSa6/uUtQk4uhhGszFAaHPX9+PIbFiOLgA63QlraGK1r8lut2Pjxo3o3r07AgMDgYWXwLL/W9gvKa1Y7Kv1WO6wAxl/whbeEo6obq5jxM73YBz8FY6BH0k6L1T+7Fnmng1L+izYLzwOS3BM5ddkGEDeDlgCI2ENb6F+TXPHwxHbB0baU7V/nxwOoOio1D2L6gpbSEzdx/LCDBhlBRXnyHX6e9r/Payb/gXLgHdhr3LeVd1n70jmJjw4tQdWFQMHrDE4dt8x315rHPhBJvq6PQRrSLzrGJG/B1aUwR7eodLPa/x72vstrCtugaXkOOwjZsLa7KxKr2nh1GAU2e04PvhzTO4xuc6vCavugSVvJxxDv5Kdr1XeJ+t30UDi6bCMnKF4TbthndYRji73weh1svaN8zU5ltwI664PAQD2i3JhDQqv6PvUtVPx3z9uQIdA4IM7SwHrKTvJcrfD9pukwbw4Iwxf3VE55bat9DjwfTIA4POkm3DZyHcqv9aDv8K24Hx5rsvsPj3fe2buI3j8yPNYFtgRAy7eWun9+GL9F/hl9tX4uhmwv8craN79rkrvhzFrBCxH58GIGwDHWYsrv6bCdNh+agFHh1uB/m965TU5x760tLTKO/sKDsL2S2sAgHHRcTgCok8+iL0Etm+ljt8dGcBrd5zcKVrxGFvfhHX13XD0fglG53sqfyZPrIZ19hmAoxiW83a4nO9Zv42ExV4IR9cHYPQ8WRfq1NdkWTUFKMuBMfD/PHtevvsLGIemw+jziqQaRT3HiFN/fnQBjK1vSDaH8hrVdfns2b6SY7vR63k4TtllUeNragjzEZrXhNIcWB1FQHAiHBU5DjzzmixbX4Nl9yewnLMWDgOV+244YP0+HijLhf3Unc7O17rtfVhX3AxHrxdhdJlS9zEifQZQdBSONle5vqZNL8Ky9iE4er0Ao8u9dXpN1f28Nu+T9btYWMpyYR+9CJaEQS6vyfrXpbAc+A4Y8hnsqZdV7rs9H9jwFBzxQyqVb7FarUDBAew/tAhzsjNxWa/rEWANqPSaLDveA47+BWPgh7AFhbq+prJC2BZfBkfyKBid7qzTazqV8ufFmTj4TRJWFAMHOz2MW4c9XdFkGAYOLHwRrUL2Sr218tTtleZelt0E6+6PYL/wOGyhcZXfpxNrYV15KyxdpsDR6uLKfdzyMqxbX4Nj1FwYESfPATz293R4OiybX4C95/OV6mLXZtyzrJoCBMXA6PGYy2fvvSUv4u35/8DBMuDEP+2un7306TB2fQJHjyeBqC6efU01jOXG7i9gXXIl7EO/BlpNrPxa0/+Abd7ZcAz+FNZ2V1bZ2VcIy8YnYA1rJTv3LLZKj2+z2WCsegCWLS/BPn4bENHO9TUVHgIKD8ES3R3WoIjKfzdzz4blyB9A68tgH/xZnV5Ttccnhx2W3Z8ABXth7fEIDFuI6xgx50zZbdvxThh9/13pNTmOzJfry9g+QPJoWALDXc/3ZvQF7AWwnL/d9X3a8hKsAeGwd6icStb5moxZw4DsjXBMOAJYrC6vyTprCCyZy2EM/gRoe/XJa789n8O69Bqgw00wBryHp596Cp889RT2Oxy4DcC7qH5n39sA7j0XeH6y+nzP+mMzIKoLLGfNd31NWetgXXotHJFdYAz6uFJ2uKa4rlHdzr7169e7rndoXlN2djZiYmKQnZ1dsaZUH9zZ56bg4GAEB7vmsLfZbBVvpJPzDVXd1tc/t1gsyp/r+ljXn9e1j87+VGo3SmGb1kEKu1YpAGuLTAUKdgJ7pgIdbwZsUQ3jNdlCJT/0kT9h6/kM0OOflW/sKINliUwu2oxi4JT72mw2oO8rQFBcxWNWek0lGcARydFsDU+tdF+vviZAarvs+gS29jcqc97X+DgxPYAWFwC2UNf3qTQLOL4c1tJs9WsqOS71CiPaVbpfxWtyFABzRkuNrcEfe/bvpvuDQOtLYYtoWzEBWXF7Rz4woyeQMhqWM2epP3s/t5YUHFVyc9fqfcrfJ0Wbk4bBpqyjEQxbdCfFz+s5RqRPBzb+C5Ztb8I2yTUHu7XTLUD762ALq1y7pqLvW/8t9XraX+P6mmrqy7Z3YFtxm+RC7/+6515TOd0YYen+ELDrQ1ijOlX6DFb0vUCihSzlddw8OZZ33XUxArZnAX1eAbpOcb19s1FuvaZafcaSR8rnOjBaffvOd8Myayhs+/5X0bfavCYAQGme1CcKigEs1sq3H7casFhgsYUo65Nazpovj6Xqe9Z6YM3D0veu97q+pohUwGIFYnq79z41HwNcblSaAqnV+xrdSWqh/dwOtoPfA0mDK93WckqtPZulTI4Vp/Y9qqO0bXwaaHNJ3cfy/F3ApmclDWJi5YXGavsemSqFzgFY4Po7twSEIdEGPBYPWIv2wWbre7LNYsHPsRNwyeJ1sALYXuXkFcXpGFS0Bl2GP4XoXlWOhbV5TfX5m+92nxRn/+MM2CZmAIFVahnOGSPvcWC41HI55TXZbDYgY7bUi0h7xLWPZQXAqjuBg7/AeuEh1/q7h2bCYg0GOt3u0lebzQYcngfMl0lyXG7U/7We+vOQRCDtCdiaj60YyyqNEe2uARIGARabaw2v9lcB6x6EdcVtwJi/XB7benQ+sPwWqb8bcTIIq+KxF1wgdXki2wPtrnXto6MUcDhgK88M4OyXxWKRWsRJw2ALcL0cqXHcs9mAFmdLH1W/lAM/w3JsMWzhrrWqZOF3ljxMYChQ/j6fPM7bgd+6Sp2W07907UvONuDwdFgNu/L8pdr3LzxFvlSvqWofqzIcsAbHSv2aKu+j1WqVIKi/LgOSzgA63+nalxZjZcwKT63TZyyi9BimpgAPHQO+sEdWuo3NZpPF0YIDgC0EsCV6/ry89cXypevjvHOA3G2wTTyirD+mfa1tJgFRHST4LLqzy/lep0n7serwKkzueI76NTlKYS08BARGu9R+tFqtwLY35PHseUBQ5clNAHL8D22u7mNoMjDo/2CN6ak+Rx78f8Dg/5P7VnmtnUp3Y1pzIMIKAA7AEnDy8SNa48u4y3FZ5he4q4Xic2BEYm0x0CsYSIrpUKndarUCMeX18uIHV7wWn1zn7vkSU46+ifMPAe26nomBVd6P8OBwfJMHDC0cgkU9p7g8hiWmG3B0HiyR7V1fU1AkkHoprOm/Axl/AimjvfKaLBZLxVfFz4PLJ3NiesESFAub5ZSzkPLzBQCYMnGGen6h5bnA6rthLT5a8Tmp6HvejpO1hA2H62sKSQby98B6eAbQx7U+mO34ImDH20C/112vW938HThZTqyEZe9nQO9/qc+/DQOAUTHOVer7L50k6HDkdPX7NG88LGX5sLW/zuVvp1Z9vPgEkLUBloi2jXqOpdLPM1YACy4E+r8DW7ura/04tXpN3e6VawWLRcqxnKqsGCjLBVpOUD9Ox78BiYNhjUmrdEyr9WuN7Q2UZMJmccCl3nf3B4HuD7qcH1S8puPLgfy95XWrbZ59n/q9Bmx8DraAENfzCwDo+yLQbDQQP9D1NZUWAVtegS22D9D6osptOz9E6oYncQ0AJCVV1FSreE2db5MaWtlrgMQhrn20RQAjpqnPmWp6TTX9PCwRg/cDh9oBy3PmVmqz2+3IijkHreL3wjbvbDn2JZ7cXGE7/BtgzwfO2QBbSGzl1wQACX2BsVKj06WH5cEC1gPfS23IsOaee03ps4GFFwOOYpf5u1o9zt5PZYOC4ry8deFWbGoNbCoGrLlbXGsCtjgXlhbnKq+XvXrMdZTCEt4c6HwPbDHdXMd+Qza7WHd9CLS5FBZr4Cnn5RHA4RlA9kZYO99ZcU51KkvPJ4AeD8MWEAVnasdKr2n3x8D6x+XaOiitch9HzTrZF+UrcvN3YM8DVvwdaH8DEBAq18VVb9/vDWDTC7CmTnD5HFi3vAIc+g2YXLnGbKX3qfkYwF6s7suO94HgeNg636HsoyXxdODoQti2v1lpXqbicZLOAMJbwVJeA/Tk+UU0kDAUaHcDLBYLbvr73/HMUxKg+Dmq39l3DaSi7t03nAuM/Y/r+V5pHmANqKjv6XqdtAnIWgdr2hNAUJjLa/Lm/F6DOOae0kflekc1t/ck3VjvdxISZJfYkSNHKv38yJEjSElRFLWmerNarWjXrp3rH5jhABKHVeweqiRrPTD7TGDNg0BJlk/6WSuGcbLodeLp6tuEJAEd/g7E9nRta3cNcHQhsPE517aornLAm5gBtDzPc32ujZxtwLa35KDrzibglDFA83FA8THXtjaTpTD3pueAskLX9uwNQMmJimgmF7YQqbOium99BcUCcX1cJm8AANbylC66IrkOu/StfGKzzra+ISeW+79Xt//cHvg6Sg6ynpQwFIjuIUV+VRZfI8+t0+YKWaxzR3m0D465TjR7lVEegbNosrq9yxRgUjYQo/ibrQer1Qp76/KdRfl71Tfa+Dyw+1OPPm+FiHZAi/Mqdoa6CG8F9HoWSDxN3a5zaCbwTSTwXbzUQaoqIFT+NtyRtRE49Cuw+j51u9UGXJgOnPm7ut2bQpKlMHa7613bkoYBgz8GhnxaaeKuQvwAIGlE+WSDGyLbA/3fUhfProeimF646QjQNQgILz7i0t4zuSd2lQI7ShV3jh8EXLAX0V1ug7bYuDelTgL6vgoERLi2Df5YvuuKt7e9Rt7P8jp1lRyeARz8RcaDgEjX9qx1smMrsqP6sZ116U7Z+e8xuz4C/rpUfV5kLwFW3gkU7HdZIKqQ9hTQ+S51GyxyIVl1IsKp3+ty7Ijrp27/Mgj4KgTY93Xl874jc4Bji2RBzR0FB4Ftb8t5oUra48Bp/9Pfv/9bQO8XZVyqyvl7srkG9QGQscj5HL604335fe78SN2+6QVg/3fA1jfV7UcXAdN7688vNCLieuO8Q0DLAODyaM0Uy0+pwI8t5Rze0+xFQGluNR0sPzdJ/7Pujx3XB+j2IBDe2qWpWWQzjI+Lh+XIbPV9t70D/NRGfu8qgz4Euj4ABEWr2zv+XX9eHxgJbH8X2PSiur0aPQ5/iQgrcGdRJ5yaYQIAEBCK7SEd0HcfMD24r+udD89A2yArzj8E2OIVf9ORHSRYYaxriQuvCoxEpi0WR+2A3RLo0tw9fxWOtAVSi/eo7z/gHem3akwIjAR6PCqLYxkLPdvvctpr3qAYIKKDXHeoJnliZLd128hk9QOHt5HFqbQnXNtangdcsBe4JF/OFaqK7iHfB37g2gYAQXFyrhjeRt1eH+1vALo/oj5mOezA/6zA/2wSZOPSr3g5Hi+4yLUNAFLOAmJ6Ac3Hq9trEhQDJJ1eUTu6STDsQHCifM68QTdBaQ0Cev1L3m/l/awSTKw7N6nJxueA33oCxZk137aqmQOBhZMqJuA9qv31wPnb5fiiEtFOgtajFOeLQfHyXuVuc21rNgZoeaHsWA1JdG0HgNX3AnPOUrcdnCbHLS/JdQBPHAcOR1QOEK8Y/44uBI4tBhxFle+YvR7Y95UEi6o+S7s/A47MUz9phxuBXs8Aa/8h5+GelLMVcBQDZ84GUs6s+/27/0OC0hUyg1KwpQToFgwge6P6/rv+CxyepW7zlsMzZa41ticQ09213ZnVJGOunJtV5XwtKzXXFI5SYPX9wL6vXduO/iVzjvGD5VqsqpIsIHuL+rhQH7YQoNvDEqivE9tLgviTR7q2dXsQOO1L/X3T/wBSLwH6v6FuH/SBvF7dOWy76+RaKaqzur3vy8Cw71x/3vICCeJMkEweKSkpeOHFFxFis6EQwIUA5kMW994BcBdkV9+FAAosFrz04r+Qctbbci5QVclxoPAwEKJZJ2l5HnDuVjke+zHtuZ+vnt+UZ22AgoKC0K9fP8yeffKCzuFwYPbs2ZXSepLnWCwWREVFua5gzx4pk6KtLnS9075vTv5bdRAwS2nWyX+rJl2tAcDEI8DA9/SPsfcL9QRjQKhc7OlO6ryp5QWy8KrqV23k7QSW3aRfGCs8BGRvBqCYHIrqBgz9XH+QsAbKCUPeDvf6Vp0l1wFfWIDi465ttiCZJOz1L/V9HUVAkevkeK11ewjocBPQ5nJ1e1CcREnu/D/3n0Mlpjswfr3+gqzlBEATcQQA2P8DsFkz4VWTdtfJ38eZmgk1b4nsCHS6Heh4q7rdUSqTeOmePdG2WCwIHvymTDxV2clYYe3DwGJ19G29rX9MTuR1dn8GWINlJ1BdOP8WIzvK/avKWg8sulImMFVW3iN/d7s/d21rfjbQ7035UjEc8ndXll+3Pjutf1Kee88XNd/2VKV5wIEf5XgUrQhMCIqRC722V7pOujqNniMXqu448iew4nYgb0/d7ld4GJh/IbBgkuzIraJ7Ynd8lwe02w0YSa7HtIll67Fy4FisvlYxttuC5YLsu3hg47Ou7d70a3fgz1FyUaVaxGl3rUyiqD6fABAYJZNTqgvJyE6SbWDwJ0CgYiGx0+3AxZlAnGICHQCiu8rf/EVHa/tqai9xGJA0XB1YY7HIe1I16t1p+a2yGNhsrLo9cwWQMV//O+twkxw7YtLU7fHl6SLDUiuf92XMBzb+Cyh28/dxfLl89n/TBGOkz5Ydifn71e2dbgO63a9us1iAi44BAzTnbM3HA6d/I58JX3IuJC/7m7o9Jk0Wu0f8pm63F8k5RFB83Z43KBp7wnvgqkjg/mTFRb/zPN5RcjKIxpO+CgW+iZKJN5X+b8ixQbfgrLPlVWD2KH3w1IGfgd+HAIuuULdvfVW+Jw5Tt7e/XnZMuRvkEhSvDjoDgMzVwIZ/AYdmuATjbWxxDcYeBN7cv811Et0wYBgG1hUDxy2KcWz/94iyODAlMRIRQYp2k/xeFIh2mw6gzAD6R7t+fiMiUpHpADobxyVVbl1FdgDO2179eW49aK95AWDENKDnU67BMYYBhLeVfy+YpH7go/Pl/EU1jgaES9R9gGtUPQAZA3s+rT53AeS6IGEIcEqGAo/J3QlsfEZ5/lHpM6taBHKOMbpj0hk/AOes0S9A1eT4cgmMMCNYyVuOLQby96gXfb3JGgC0ukgZTOERB36U76pzsvkTqz+vb36uLFy4Oz5X5/gKYO/Xch2psu0d4I+R6nkGq03miwYorpUSh0pWjrOXSxBWVbs+kXO2U9KLV7LlVWDFbcC+b2v9UmrNXowHW7XFHwWApUXlRROLxYKoo9/AsuVl2elWdcGky33AORvKM0Eo/u4WXwXMHgFkrlI/d/IoWeyI1Syuuqv9jcAleXJ+7Y6u9wFtr1I2HQ9MxqD9QNpeSHB8VRnzgSXXyDm6L4W3lvN63fntklPmJqr+7ZwalLVLE5SWtwvY+aEsFlZl2OWY1+VuZZYGfBsL/NpVrl89yRnYt+5R/UJi3h4531IFqCQOlYXKr6PU5x/zJwJrH9E/vyVAdgZmb1K3R3UCTv8SaKEJYCk8LJ+Tg7+6ti26Gvj5ZHrbe+69F0+9+CKKLBbYAewE8CSAGwA8BWAbAIfFgmdefhn3nBcO/NQaWHm36+MGJwJn/CRzeCqBUfIZmHeu7lX7hWrP/XyAi32nmDJlCj744ANMnToVmzdvxi233IL8/Hxcd53mQ0z14sxhWzW/LdrfKLvBVLr/UyL2J+zXR1ybwRoiEyyqEzOnnK2yaFaa49r2x0ggsrN6V4rDDhxbIqnJjvo4kjYgDOj3qgzm7gxSBfslCiV+sGtb4WEg+Uzg7JUVudsrCY6X6KKMOfrHP3MWMLSayH13OXcKpGsWnzrdJtFaKtYQ2RWVeol7zx2SAAx8Xw6SKqd/JTvOdBPK3tLhRsmzr23/m+yScIfFKid1uuh3b7FYgf5vAgPeVrdnb5bFCncXuzW0Y9+pks8EohURdZ5gOAAY6otMQBaSd/+37o/b8RZgsgM4b5t6cmnRlcCez4GdH6vvv/U1+e4ocW0LigE63y5fKvZi4Lc0YMm1de83cHKyVLW7ujrFx2QieI9igdJpy6vACs0E4om1wF9XAEfm1u15neIGAD0er/tEinORcv+3yl1CYSXHcUcMkGADQkJcJ1Ytu6ei74mZ6L1e8brsRfJ7aTZOHRXqTc4LJdWil9PYpRJIohLaDBg9VyKuq4rpIWOgLlI7IBSY3heY6+ZugnoxgIx56klZa6BkHNAdz7a/K6nedn6obu96v0zCBLo5Po9dKouciUMqj30db5NxbtGVNT+GSnx/IDBGfywOjJL3UzeRt+P/JLBHF9UfHK9eMAbkwhsWCWjypZTRsju1u2bSoOX5wOlfq3cLAEBJpny5cf4cbAvG8APAhlTF3wYAjJ4PDJ4KVKnR4hHO3fW6jAmRHeTYoHvdOvl7JWBiRl8Zi6taW56GuO9r6vsP+VSClFpoosHtJe5HoBccAnK363dVbX0NWPcIMHecy27KZh0ux+8FQJBN8fsqPITHDj+ND5KBqav/49reSwI0RgTmYmC0YrIte5NMoLv7d+umEnsJQizA8lTgGusel/aINpfiyePAk7ElKFLthNjxAfBjKrD1Lde2snxJf3v494qUcJ5W7XnfxmeBP6S2XmUGcPBn+WeoJnr+6F/yWTix2rWtrADY+xWw+d9AiSLbQvYGmdzM3qzv+M4Pgd1T9e3uimgn18wpo1zbLBY5ZlxuqMfv6K4yEX2aZhEnZ7vskCg54V7fZg4EZp0GbNdcGzRGySNk92pdAz08Ycn1wBxNMFF9tZwgcwyqeQTnDiNdwPGIXyTtni4Qrz6W3yzZFvL3qdt3/EcWOw787NpmOIDYvnIOXVcZ8+XarsON6vYej8l33e+kPuyFeCR4N66Ngku6eLvdjuPbTxmXq2YAsAUBm18Efh/kuusPOJk56Pgy17aMhcCez4CE0yT9tSfZgiSLxL6vgaIMjz50oC0QOQ5gQwnU8x/OuagON3n0eWsUkyYleNY/IeV8qnJ+LkfNcQ0itARIanJbKNBfM37uL9+B1uHvrm1JZwAX7AJa15DxI0Ozy7NeDJkL0O0yXveonG/tUhwPN/9bgr+CYtXnqQPeAewFwHrNInxkB2D8Rv3f7eJrgenVBLNteVWu5TJXVP551npgz6cuc8/3TJmCA4cO4b4HHkBRQgKOBQQgJyAApYmJuOyBB7D/0CHcM2UKsKG8vzsUgY8BYfJeHvhB3aeyQlmc1+1a9RO1mvPzIi72neLSSy/Fyy+/jMceewy9e/fGmjVrMGPGDCQnN6AdZE2M8oPf+lJZGFNFN9iCZUI5KM69tJLeEhAqB8bltwB7NItPCyYC8yeo0/bZgmXnnmqBJ3ebRBbveE/SHPhSWb6kQnM3oil3u7yXqpPwY0ukfkDGfPV9HaWy4JCxQN1uGMC8871zMdb3NdnhpkohajiAP0ZIJK2K1QZ0f1g/+VNfEe2kxqMqoq8+9n0nEzgz+rt3/5YXuB8RfWwp8NflwDHFSbw32UvkuXW7opwLzc0UEXf1lJD+PmxfBaijmgFg1Gxg/AaPPy8AiZqzBukjTvv+WyZJ9ilSQlTHYq0+KKDLFAnYGPaNuv2cDZKyuL0iwMZeLBdaurS9zpPrI9UEB1Sn5xNSk2yQJp2VTkgS0PsFYMsrwNpH1bdZNUXSIdsVi5j5+2RX95ZX69xluf9uYMOT6guy6kR2kBS1Qz9Xppgpyd2BZxOAyZFAqOpMcexSoPPd6h3IJ9bK7rqUM4FWE+vWr/oa/qtcbM6foE77V5IlEe6n7sY/1fonJV2xbrfZppeAmYPVk4hFx+QCVxc5f3yFjLELLla310fySLnw1h0XHGXqRXRA0s8AciGrEtVJFm2tmkWcmYPlddUyKKLivC8kQY5n7kY7hrUEJp3Q77I/sVomsXVpU5f9TaLg3Uk7aRiSbnvzy3W/b321uxbo9bS6zXDIOZNuB1zz8cBZC4EERQBWdbI2YH74apwdDpyoWgfTKWkY0O5q99OxVeectTLxH9FW3b7ybmCeG6mQ+70mCw72QgCKz+GAt+Xvqo0m1XfSsOpTe30VDHytOP+tDWugpJdUpX8GZAKw7dWSaaLK79wCCz5LBnLblrjukggIQ17cYFwXBewZpoiODz9ZlxMFil2xzkWj6gJcvCC6cB8ejAV+LI1XBtNFh0RjeJ8p+DX6HIQkKj7fez6T17NNkR3AUSbBLwenVZ8utp60kz3hbeQc2mWS8JTPZJd71PdtJnVLcVSRfjR9tixirr5XfUxLGS3fl2iCmvN2ycRv1wfV7fWx9wu5ZnYnG0N092rSTgNYcaucg+S6GYzRvXyRX1dGojEKiJBALHcXQOvj2CLvPe+At/Qphc/dDFx4SK6Zfc0ZpKE7l+z1rKTRPXW8dSrNAX5uKxleqtr5EfCFVeY/shTXiP3flLS+qlTzgAR+XbBXf95UHwHheKKoNSZGAK0yXLMLHEgqrzV2YrVr+vXCdPmddf+nerGk1zPA2atkl2hVGfPk/LXocP1fQ1XFmRKMsejy6oMi3DAsLBAb2kdhaq+z1NeHzqxH1Y113lJ0BDi+VL1BodVEWdCzK67HA0JlzjAkRf1eAZKxq+fTQFiqun3HB8DMQUDebte2yw3g/J3A+R7O6uUoAzY9Lwv/ugBB5/HAKHNtW32vfA9tpl64bXuljIF7NXPEc86WtNS6587fDZxYpZ8TdQb7dLm3ys/XyPdBrsGcKSkpeP6FF3Dg6FFklpbiRGkpdmdk4PkXXjhZwswZjKML5t//g34e99A0yYzVXxFg5WfMWugDuNjn4vbbb8fevXtRXFyMpUuXYtCgOqYxo/rL3iQLOLrUSwsnycWzbleKWZaXRzvrdhQ4Fy/DWrq2jZwhkdq5ioNXUKzUEer3JtBaM+HgLUfmANO6nEyTUVed75IT7eiurm1x/SXtWeFB9WRb7jZZ/NTVS7NYyicJvbAtOul0YND/qesrwiInluuf8Pzz1sbG54F13nju8sVzixciHGuy/3s5AVqu2THgLcXHgN8Hy0WVStJIuSCLH+Dxpw4vKF+c8WLtBK1+/wYuK9ZHigfHSyqNsjrWhSw8DCy8BPhzrObC5Rq5YNPtQovprk8FeHgG8H0y8LUmHZXVJmlrhrixI9EpZVTdJ8EDwoA2V0o6IF2ttrQn5OJHNQnufD5301JHtJcdUs56cLVlscj42uZy5S7hkIRBKGp3A+6JBcIzl7vePyRJdn33UOwwCm0u6YjjTTh/OjxdLsR0x+GC/bITU7eLLba3XKSqLjIzVwNrHpALYNUxa++XQM5mfQ0652TX/jouotfG9nflb0+3Sy0mTZ8qtvfzkqrN3bqRA96WGjKqNESAjAdfWFx3TpXmSeq6MUvde15AFt10gV+RHeR1QdOecpYsBLqTwsu508XXO+yPLQGW3gSc0NSlOTRddgjpJu9Dk6UWa13rNgVEYF6BAyfswOI9mpoif03WBzx42/Z3ZAeUO7VtOt4s2UJU53tJZ8iOGJ38vbKTV5cm7oyfgIGK3XO1EZIok32qyWBA3schUyXTRJUF88RVt+KKKOCPQpvrYnpQLCJG/Q60vwEpbRTBGKW5EnRzzjp1WtT4/sCYJVKLxYeiC/fgsXhgXlmkOpgudwduTozC+CFPSxBBVUM+BUbOBIb/4toWFC0TyYenqxcDve3gL/I3XXWH0anvnS5wJqKtTNyr6tNFdpCxudezclyuKra3XF+2nKB+7MJ0OV4Ve3ZHCwBZaOz+iH7H+K89JJBEZc2Dkopw71fq9jZXyvVnvJvBi72ekUllLwT6mSZzhdQqLjzo++cObwO01pSmqK+NzwKzhql3UFus5ZPvmuPdFxb58kbweJ+XZLzR1TpucY6kP3cuuJ/KeU6iKtfhKIbs+P1FXdMvIFwCDP9U7JgFZEwJbeGdHczWQHxyeC/KDKBYEdgemzPj5H+q7kDa84XsImo+Xp1yPjBSsmqoStq0v0F2/s0crK/r567DM+R8r82V+vfSTb3i26G7NQdXF8ySY09VAWFyraXaoe9NJ9ZKZq1hP6jr07W6UBYD556jrtm393+yOLXiNvXjJwyR2nguO9khKTC3viE7OB2KRTVAju+quVRP0NUAB4BuD0iphk6KDEPOuUrd7s+D04AW5wNn/aVub3mhHI91tUedASi6lNqDP5Y05FXTGTcfD4xb4/4GgdO/lmOhquRB4WEAhvRbJbKj/F3q5kbIJ7jYRw3P0QUSsaq7wD5YfqHmjRzr9TH0C0lhpNvddPo3wJjF+pPOP0erU9CFpgBDPpEURYGaSC1vcQ7Q7qbGO7FG0ubsUEx2hLeSg/qaB9Un2rYQmaAMbaZ//PEbZcLZ0/Z9B6y4Sx3RZLFIdI/qBMgX1j4sO3kOK1K+1kfqxXJAH7vEs49bG+1vkIlm58mMr9SUNjS2Z3mNL83OkHo4lHw3HK2vBFpf5vHHrtGah4Bp3fQLA+Ft5HPQ7pq6Pe6h36Suavrv0E6wFx7WB2rk7gC2vC7Ft136VL4gG1BNDaG0R7W1EWqUsQDY9rZ7UeZhzSUdUDtNjcW0xyU1pCpVUFCsjGNpj9X9eQGZbE6fpR6rqmMvlvRfebuUzZagSIR0+BvQZQosqonTvN3a+yK8lezA3/1fWXzwpZheMpact119vHQuNOv6Fd0DKEqX301VpVmyy2bgB+qJkoTBEq2qW8xudpbsXj1bU3OkPkKSJKhGl45q1T3KdK0AZBGz632ycOCOuH5SQ0aX8jK9/FhVdfJn25vAD83dz1hweBbwPysw63R1e49HgDNn6neanfm71E90Z2dhUByQNML3k8HHlgI7PwCm91K3OycCVBOI9RHRBlmDv8DrSVY8FaiJbt/7pdTg8oYFF+vrKAPAsB/Lo67rOGl7Yq0EG6kmrJzmna//jP0+RI57quAAQNKqdtDUV6yNdY/qAxOqcdQWg1/ygPEH7erPf2CkBLS1VaTiXPo3qYO5/gl1Om5roNTzjfJtvcrMuCHouRf46NAevLJIsVMnd2d52jHN+BqeKn+vun6HJAKd7zlZY9SX0h4H+mgi5507cHQ7wi0BQKtJ6n5Hd5WxufvD+rp9Qz4Bemtq68b1A85a5H4gSHWKM2W80GVjyN4ogTUq3R6S71teU7e3u0Z27dJJba+VHTEJQ3z/3BfsBgbXfRyrlR3/V76rVTH2H/xNril0E/C9/iWfcW/UUopoK+ONriRHznapm6ea3LeFyI7dVoq/+Y63AJNygfN3qess5++XoCBdHdr5E4AvA7wWMP/plfPxdqvH0OesH13aioNO2cnV4rzKjUnDZPdQuGa31/cpUvJGFUgamiLBmomn6X/f7ortLQFxPR5VLzTWR7OzJHCmw83qdN3FmXLe4+n6dDUpOgLs+wrI0+yeW3TVKbWZXWsCV9CVAyk6AszoB2xWHMfz90h66d4vqtOyL5gkv5P9P9bwIurIGiDvcY9qrsWtgXLNrkqDf85aIO1JWVRTfUbnnQdsek4/B5wwWM6hnWm7q0oeCVx0XD/HERQv1/VV/66D4+QYq0u1WZO9X8kiuirNvTUISBiqz7gR21tqDOpSGZNPcLGPTGO1WtG5c2dYrVU+humzJLJCV1Okz0tAn5fVxZjN1PI8/WQvIIspul0juz8DwloA7TWTAr90An5qoz9585aozrIrRRUxWisWSWmm27XS5W6JGlFNRkS0A/r82/PpKmtj3aPAtjfUueEBSQ0yShPd7m1DyyPICw+Z8/zeENUJOONHIFWT8sFbAsIlSusizUXPkbkyGb37M48+rdVqRfug9bDu/czzFw+1kbVRdiDp0vr9OVpSStRVwhCg3+vAeTvUaVgWXS2/T92k1S8dgVV3S8BHVbE9ZTH6Ei+l2Nr3DbDidmBWHcebsgLg2zhgmaL2gFPeLn2aTUex7PhyJ5UgIGOnLRh13uFcfFQmsH9uD6x7XNEvu0ye9/oXEK1IZTX3HLnvyimaxz8mUcm6NLXesuxvkk5SN0kYGCVpTroqohQBubgc/ot6F0XySODSAn1Nhfj+khq3ulooMd31Nf/qwxYuqVSOK3ZhApJyVReVPKOvpI9xt25kTSYeAc7fDTQ7u/J5X1x/SeNzaEbNj6HiDPhyd6dC3i5JrepOVH/L84HRc9Q7n7yppgWWTrfJ8ayH53fYTU6bjJCOf0NwC835YNf7ZbLNG5y7YXU7dlucI+dldV183fEf+exvfUO94DfrdAkwDIpT37/15fLVw0uBSqPnyTFVZdV9Mun1o2twwfLEC3G+LqOZo0yOV6ogPODkQnHeHvWklUkcgdHYVQr80QIYeEKxgzNxqByv1j2iXkAqPi7nPqp0gg67LJS3v97zC+XltNe8gASXbHjKNeUqAESVZ0Y5PFP9wHs+A6Z10l+vVGfPF5K95ZjmeGkLlmw6f11R98euSUQbCfRTBRMBwIgZwGlfqtucgUYDvJQZY1o3+dvSpSdrjAJC5XetW/T1pqU3yiKTN8T0kEl4VbmQxVfJNcUGTdrr7v8Azl6hbquv1Q8C3yVIeneVTc/L31a6Zjf6yOn6kgeBETLBrnrNG5+R67uRmoC2iPI08xu8EJhTlo9hi8/Ck2GHEFAl8MxqtaJNQHmqwZg0CVA7VfwAySazYKL6uFOaLTUO93zq2mYvksXCkTM9f34d3U1S3XsruCVhEDDwXSBWEcC1r/z9r2swZ32ljAbO3QIEJ6kXo53XdeftUNTsO+VaVBck4ty0ocpIE9kJGPGbzJmq7P9Wvi/wwgJoz6eAnpoSPYDU+/3CAhxSHIuzNgDrH5ddoIbiOO6kq/Md2Qno+QwQq/n8HvxVjte6jS4Z84HfergGddpLJNhz43P6PlVn0wtyPT29t2tbcLyMoYmaYDhAArBW3OreczcR1Z77+eL5TXlWonJBQYoJ4XFrJeJUV9S+631A13vVbWYqOiYH5pzt6vbNrwA/tFS3b3pOXq9qJ03RUal9l7/X/ciM+ghvLVvX3XF8mdSOUaWPOfqXTLKUFaij6kpOSKHmTS+499z14e7uIF9oM1kWPRR1tuole7Nc9G9/37OPWxv2Eqmlpash501BsRL5pOKcONctitWDNXUijIH/0e8C8qaw8r9H3e+72Tg5gVelFa5OdDeg852ywK/6my4rv2DR5fHvMkUWmJopFhoNB7DsZskP7w2d7pCLT2d++tqyBkpKnB3/0e8W+7m9vhZm9maZUN7h5t9dywsk3VpyHeuqBsacjI6vetENSGTltzH6mmQdylPuHlEsbOXuAFbfLxNxujpD3tKzfFLn6IJqalK+rF+wO/y71FfU7RA6ulgW/3WT4FtfB/Z9rW4rOipRkrp6avUR3lqiwFXHWkAi352pJ6ty1lPYrNlZUl8hSTKxW56KqOK8r9lZsgt0m5v1HBKGAhMOSooad8weBcwcUP2FuY69WCLNlyvS+XiTrXyiVrcj3BYixzNVGqz6MBzAmn9IQEeXu9W36fOi1AP0hgv2SAS8KvodkL/LvZq/u+q0u1YCANY8qN7VHdtXUmkO10Rb930ZOM2Ltevi+krQnYozQlyxo7ZbYjdcGwW8okpKYLHK8WrZ34Fdish759h4YlXda8F6UYDhQIINaBkAhFkUf7OBkbIzW5eebtEVMhmm2qXpKAH+PAuYPcKjfa5Kec0LyLHKXqw+b3KmQuupWbBwLkSrsn2cWHcyVaFy96lVjkfb39X3K2WUdxZAM1fLl26nQ/OxQGtNSuyu9wNn/KyeIPeE/PLflTeO1f5o54e1rulbZ72fB0bNVbcNfE8CcnQpxr1p18cSYJCzSd3uDPTUZVjRyVovuxWPL1fPk7W4QBYNdLVe0x6XxavkamrNussSIIt22RuV9e2sqRfBCKimDm1JVnlqQIWxyyQ4IFJxPNz6pgRdeut4teX18sdf7dnHLcqQ3Z262qLx/WWn2UAfz8tYrLK4tOBC9RjY8Rb5nr9Xfczq+YxkajrjR/XjGw7J2pU41LUtKBqwBsscoSr4pXf5nGC/N2r1UurkwE+yK1bnaHnwh+q697fyEiQBkerNKs5rZt0uze3vykKg7pi25WVg5R3yN6KyrjzgrOrv1Lm46G7JgaCY8n9oAornjAHWaoLdji2TIKo2igwSfkZ77ucDXOwj0zgcDqxfvx4OR5UdDXu/BH7t7prP22nHB8Dia+t+guRts0dKzZxDv6rbV98nEeiqSZgh/wWGanYPnVoAN7p7/ftZFyVZktLS3Z1NCYOlJkNYC9c2W6hMUOrS8tmL5KDp6ZQMtdH1AeDSYu+cDNdX3m4gR5Gnv76OLZKTutX3ef6xa7LpeeDbWGB9NRFV3lBWAHwdCSzSnIiEt5aFHF3dSDc5HA6sP2CFo+315ny+214tARWqqFBAJuFPrHEvqjl7E5ChmfA940dZqO6smSDv+4rszlDVKCo8LAtif2kmf+orqqPsQNKl0tKxBkqKZgA4ukh9m9O+PHmiX5UzZY0z2tZXAiOA3s/J+9FJUVchOF4WjtY9qt6d1+Uuue85itphZQXlExH5vk+37Vyg3vyyerGv4ADwTaz+4qTggFxkqiJpS07Ixe/iq9S1Jo7MlQnntpod/tkbgL8uk6AKTzPscn6gWwzp/6akGVdpU15LR5eWtb6cCzGluZXP+0rzgJBmQIdqdsVWx2qTwIWKi9E6yt8j33VpPquTuUpSEfk64jqyg0y86yYvszbKxP66Jzz8xBYJStOd3wJSK0c3BtZXeGuJgNdl/Fh2kxwb8vbU7XHjB8gC5fBf5Hyzqv5vAIM16W8BoCRbUsXpJuu8qcc/ZQw+1zXtdY+izfg4GZiiWkexWCWdMKC/zupyrwTeVJdG38da5q7AnrbAnUeB1SmK8wBHqQQAjF2uTrPvrEuXeolrm/O6rPg4cECzsFtP2mteQMYSW6h6LBr4gQS56tKqx/WVY7VqkfPU8ckZKHCqNpfJ516XKaToiKRiq2v95tqwF5ZnNdDUZcrerD8mxfcHmp/tfkaEmkzKBSZl64/lVDfOtNfeUHJC0sSpduinTpKdey3Pc23ztv5vSY2uME1ays53y85t3U55ncMzZbfizIHqLBItzpFdaM7dU1UFxcgCqTd+J7Zg2Yl+bLFLcLrD4cCe7etgKctV70Le963MQwz7Xl2jLiZNrluTFMEasb2kfesbdT8HqMnRv+T3XXKi+jIS7sjaILs7f+mgXkiM6yc7zepak72+SnPlWqbXc+p6a83Pkev5mDT1/dc9Amz8F7DhX+r2zrcDFx7S13rb/o5cZ6mOh90eKJ9H0JRMqo/5EyQ1u07akxJ0005RE9sZaFmWq+53/zdkobKjZi4gc1l5OmKNlLPku24ncMdbJA151ewawYmy+679DfrHrk6/14HTvgImZbm2lebJeYXuPLE4Qz5LHp5Da2yqPffzAS72UQNkkZ0luhOkZTeVb5M2549GyzmYNVcUjgdOFiOPaOPaFtcPWPcYsFwx6RrWSh67/Q3uR2a4KyAcODQNOPCje/cPTpADX8EB17a4vpJSZKXmgG0vlvuaoeSETAS6E/HvbbNHAtM666Pf3NX8HCDtKWCkmynV6qP99UC769UnUF5lkUmMPZrI/HZXAxceUF9cNGaJp8l4ops4je0rJ/J13eV28FcJ1PhjWL276MI5oe/N3Z+G4V5Kv8j2wAV79TvOW18qaVpUQpLkwqX99XV/Xm8KaykXNtZgfcFwndiewMR0OebpaqV4S+Jp8vkdPk367sIitfd0O0QTTwdgqGv25WyVyc/EYerIZHuhHDN0i+hRXYGQFCnE7mlH5kgtV12aw9kjgc0vqduCYoFRc+SCzhtmDZWFmGOLK/98xe2SkmeDm0Eeebtkd567wUij5kpabHcW+3LLA25aKxYOvCmshaRW1+3g3Fue5rvq77q+LBap91RWoK+TNXsEMMtLaTyzNsiimm78d6YRynKj/mNMGtDiXPUEIyA1oVQ74ADZwTxv/MnUUg2Fc8HqPE2mkehuwKVFEmBT1ZZXgS2vAJ3vUl+vmKRdq7MwzdECu0qBUFV90GOLge8T9TUOO94sx9o0RdrqU1PO6VJAe1NQjL42dIvxQPwgWUxXieoMXHhQff6RdLq85ssNIFRTTuG8bcBp/1O3BcbIRKFzotGT4vrLTmHdQuKv3SQrgkppDvBlEPCTlzJjWAMkEE93jkx1M/Szk7UnPW3zy8Ciyd557PpofQkw/Cf9GBqaImVKdDuRdVpdJDun2t+gDxLc8LTMKans/FCypHhrDi2irQRAKs5zywLKo0+cO8NOZS+S8gK6TDobnwO2adL2NhsDtL9Rzn8K9rvZcQ1HqcxHnvaluoZcfUR3Pzm2FqW7ttuLgFX3nkzn6Ss5W2Su1WIFQhTHpexNEuhUUxmSdY+of154GJg5RNJiVpW7U9K2x/ZW7xosOCh1lvM9/D4DQK9ngRHV1JmP7iq1wFUlES48KMEh3f+hvu+XQZJBQpeVK+0JOQfQ1bBtdbHM0UX3ULe3vwHo92/Xn4ckSPBWvKaMVE0sgXIstCjqwdsL5Hpa1QbIefXkUnVtaPIZLvZRw7PtTaDwgP4EKX6gHHh9vWOgJqd9LhdUurzeZ84CemnSsDjsstVatYBjsQCdbpd8zr5mDZSUenZNStWaFOyXtGa6CZjw1vrdiuGpkp6r813uPXd9rLpbFtSyNek3zBRVvivE07UkQpsBaY96r+ZOdcJaSvH2SB/vbgoIBfq9CQzQLMT4qw1PycJdnVOMli+UVS287nTwV9l1ossd//tQad+jqNMSEA6M3+R+yr6arH4A+J9V8svX1cbnJcLW3R1GZig5cTK9V/ps9W063AhcVgSkKHY4L78d+KmtvkZc1gZgeh9gl2Y3mbdseUXSz8GQnV9VBUZJPUlV+hhAolkTh8mYVFVEe2Dwx5Ke1Kq4uGk+Djhrkb5+QWgKMPEwcMb3tX45tRbXVyaOHIq6Y4BEuCdVk+o1eYTslvCGDjfL8aXqxFT+3vo9btZ64MifEgHsjuThkhbbHS0vAMZvrP536i1r/wEs1lw8t5ookccD3/P880a0Aw7PkLo5Kn1eAQZ44XkBSZE0b7w+q0Ha41JHOaGaqGyVdU8A3zfTB09tfUvqgG59Vd3u3KHQbFzdntcTDs0AZg4CNismeHq/AJy7TV8PLXuj/P2pMo04o8sbWr2yuH54pSQVaUFAi5J9ru2hzWUMzJhXv7FFVa/V21qcrw8ccJQCS64DVmnq4x7+HfhzjL7uXnWyN8tEcqkmsDIwQhZpWmiCWOsjf3f1k/O6CUQA+CZavnurdvnOj2WSva6BTqRmGN4L/ApLlUly1cLArNPlHFcXAOBNh2ZKPWxP7/6PaCu7ugf9n9SArmrLqzK+99Gk4N81VbKk7PrYs/0CpB7s+qclWDu6q0tzeMFa+ceOD1zv2/ZKyT6Ru10ddLn2H5LSWJc5pu1VwPjNnq+jnDwCmLBXn1K4PkKTpc7gZLu6fMX+H4Et//Z91qPw1kDf1/TZrba8AqysxdycLgNFzlbg+BJZVKwqIAJoPl4/93d8qZQAmjO25uevq+4Py45xnawNUpJJmRIbErhSY208zdJLaZ5sMtCVsIruCnS7/+QcYFWZq4E/RshnpqofW0nmOXdsfQ2YOw5YqtgZGBQvdRvTntDff8YAYJYXAsCp1rjYRw1Pj0dP1hJSGbtUDryqE7uG7MhcYOOzkiamqt+HyACvmgS0l0iEzZoHvZZeplqDP1KnaquNsnyJOlUtVBYelu3faZqTGFuwpGYITXHvuevDuTvC0xFinjDgXTnJjx/g2cc1HPIZNKNunpk6365Pq+CvMleo06zUpMW5EvCgq2+06bmTj69yovxCUJcSIrqr92q0ONORHFYU3q7JpuclkrbkhGf75CuqtCBFGcBfVwB7NBH/uVvlwmTZTa5tpbmy4NbmyuoLd3tF+WmtLs13YCRwWbGMoSoBkRJ1nKBYDAxJlKjMGE1kJQBM6wQs1dQD9KbiYzI5r5tQG/wx0F1zXpW/H/gyBFjlpRTOA9+VlD1Vgzl6Py/f+2h2HNYk4TSgw00n0+j6lEUmyX29C6g0R47TusWluH4SeRzR1vPPnb9fLux1O0C7TgE6upmStSbty/+mdPV1Q1MkdaMqCr06wXESUf9Dc/WCx57yXaO6FLgjp0s9wVgT0hQd+EGO06sVO7rCmstCjW5SatbpsnNKlfJsaPmYv/d/da/b62XWsgJ8kgK0O7HAtTGyA9D2GtkVoArU2/SCTPyvedi1zWGXhdHOd3l+srg2drynzuwCyGspy1OnUwPkPUqfJXVqqyrJApZcD8w9V13r5/hSYOWdQLqi3p+3RXeX65kETYDh5FI5n9TdFwAGeWkRZ+n1MsleXdpiqr11j+oXq+ur/+vARUfVbcHl9ahV2YW8bdVdEjjpjbIb1SnOlAwSCYPU7f1ek+/e+J0YDgmMOfRb9bfTBYRufwdYci2UNbP7lgfcONOvn+rQDGD5rfLvAEW64vo6NFN2Snpj8d9ikR10yjnN8t+DbreYt4QkSZD//AnqFOUhybKjS1fD8Lztcp7oPMevynleofochCYDI6bpd8DFlc99mbHreuOzUupmh6KG4pqHZMehrt77wP/Id102s9BmwJBP9Wl9l94AfBWq3wm/4SkJdCqpMsdcnCnvZZ7ifawNZ8mKvYogbKsNmHeepGxVKT4uATlmlKuhClzsI9NYrVakpaXBaq3yMbQEADv/D8hWRHwAQM522U3gTro1b1rzkFxI6nYyzB4pdYJUi33NxuhPfk6dQDZj8WnLa5JL3R1l+UDhEfUCUuFh2eKfoUlNY6ZezwJ9/y3p4BqaiLbl6Ts0Edvu2vct8HU4ME2RnqAp2/mxuu6BF2nHvoaix6PlUddfe/Zx+7wMnDkbGPaduv3i48ClhbLjRmXmYGD1g57tk1Pbq4AL9ukXKqsz9FO5AN3qhYLh3hIUK+9Dn5fVEZRl+RJ1v+k59QXumbPkwkQVmFNwEFh+i9SPS6zjTpv66lC++LjwYokyrqsjfwLzLwAyNTsWl9wAfB2lPqYVHZOLYF3kZd4eOUf4SpGCrr5aXSQ7zdzZmR0QLhd0WxQp/Tys0tgX20cWATve6t6DhSQAA98HUi/2bCdroyhDdgHr0sF6S1m+BEsERavbC9OB7e8CJ9wM0KrOr92A5TebM8ky6IPyVISaQJBV9wHfxqsXNKrT+U4512s2Tr2TaNj3kqJZt5gX2syNHfAe0v2fUu97tGIHXmmOTMJM02QEaX+jpBze+rpr26npTOv6+/SiXTu+wm1Yi+cygZxUzW7e5uMkJXH8QNe2zFXyXXW+Z7FKWuCtr3ttYaDa877qUugHJwKd7gC63KNud475dsWu7twdsoPn0K/qHQPO3SS7NWlqvenw73Ke4M4uzPEbvJv+fOjnsrtEV1OK6ubEKrmm8LUzvten7vW2xPLdLFZNemh3bXtHziO/iQaOKwInez0tKQVDktT3j+vrvd+JNVBSo+/9AthZudat1WpFs16XyX8OKM6b8nZLn/u9CUCx8NX5Tkk93VqRjSF3myxGZK3zfMByUYZkUtjwtOdLy2Sukvfyt57qYNE2k+W9ctbV9qWgGCkjpMqi1uJ8ST+qq2H412WSul+XuStltJxb6cbXBRcB32lShIa3kt/JOM1Cozc5M3nEKIKOayr70/IC+Z65Ut2+5gHgr2oyjRQdrf7+AeFyDtu2Sm1fwwEERgMtz6++fzr935K0wSMVgdAOu5wv6c4Tj6+QxT5vlK5oRMye82ugM43kL0pKFLm5izPKD9iadFSzR0ph4oZmX/kEdmCk/jahzSQ9ZVW9npEJblWuZufESmgLoJMm8tObtrwCbFPk1a6NlucDY5cBCYpc0VFdpEB1kSYiz0xxfeXCOkxTF8dMK+8G5nvhwNnqQqDLFGCEn0WyLr0e+NML9UhqoBz7GoqgWMnvrqsho5O3Sy5cvtXsvkgYrE4J6WQLqT498/GlwK6P9O31Fd5Kf4FcncRhskBqxo6A+lhxp+xOUk2gh7eWtHxZ6/VpmNteqU5tE9ZcJqFTL/Jsf2tj76m7jtzY/R/WQqKiy/Jd23K2y+evLBfK0+fc7XLho/sMOY/lqknZ+tr+jtTLzNXU6KpOcBzQ+nKgh5cmxL5LlHGhPAq4YuyzWKQesC4VbE3sRZJWx9d1IQEgu/xvIqyFb583OFHSM/bQ1OPZ+rpEuG+qKZWQGzrfJeltc9z4jHnb4elASaZE+NdVl3uAkb9JWu+qwprLcdCuOV4f/K08TZwXUqLVJDxVglSSFCmS1j4ik0Jtr1XfN+1xGd+dE1BVOeu8eSu1rxsshYcwMQI4YIlBj86qCd+dcn5cmq2ug3X6V8Bkh2SHcXnwU44VW9/0WJ+r0p73xfbU10gOaw70f0OfXiy8DTDsB0nVXFVEW6Df68CIGRKIUlVoiowlHUzIbpE0TOrY6+o+OdOM66ycAmx60Tt9a3O57C5xZnyg+ml7DdDRhPkLM/V9BZhwQJnOsl6cc0ilOXLMU/m5vQRHqhQckGAgb9Tss1hkbidpBBDkeh1YlrVZ/tFGkYb8+DKZawpNUe9yK8qQ16tKz9/xNgmY+OtS9+r2Vid3h2TOaP83fa1kdzlrf2etPxmMcipHmRzXfJ1OuOAAsP09OaaoznHbXAac+bu+hqFzQapYc24eHF++GUCRhrmsQHbI6eqPm6nj38sXXxXnH72el2vH2D7q+y4ovx7udKe6veVECVIqVNRuBICWE+T8X7fAOvQz2Q1ftcRESAIwKcv9DCoB4ZI2uJkiJWtptlwP6wIAo7tJQKbPM/w0PGbO+XGxj0zjcDiwdetWOBxVTjiOLZYdVbqIj8KD8r2hpfEcMQ04a6EMyCrjNwNjlugns5fcoM71HBAptUhUURW+EJOmLhxcGxkLJLJYVVw4IExOcHZ/Uq/uecWu/8oCUH1rCnnD1tdlgjT9T88+rjVQLk50NSebsroWSK8n7djXUBgG0O3B6hfmVE6ske+6dJaOUkmHotu1vecLye2etUHdfuEhSSPnDQd+Aeac7f6FYsdbJY1pY2EvKU/PYVFH/Fusskti0IeSirmqo38BB35SP3ZglBy/l99WZfHNBwaVp0ppOUE9KVATa7Ac74qOKNoCgbj+UpdM9dhRnSVaVTeBHtoMuCQfuESThqU+wttKvSqbm7sGT/sc6PmER7tUwXnRbi+qPPaVZMnC1GZNbZmaHF0otdymm7ADP7q7/M37umafNQBYc7/U0FBx7jJr5oV6Jj2fltpH6/7p+ceuybRuMvGvSikFAP3ekN2t0ZpdtTpH5gDrn9JPphWmS8aDOZqAIGcaY3f/7urDXiSZQlQLkckjJJiur2a3bmCkpF1NHuHFDnpWXuKZCNgOfFeoqDMIyHnHvq+lDp2OM2WaStf75Xtkh/p1VKNe532/dJbUqyr2Qnldqs9gcLzsiGk+Vn887PEY0PqyuvepvnK2AhufkfpiKrF99NfTi66WdIFm7EikuhvyCTDAzaDh+tj8MjCti34C3ZsstvKaqB6eanWWS7jcUE/AZ66U89c4zXnRijuA6b08vygGyLVjYBRw2v+AVhMqNTkcDuw7UgxHl/slwL2qxGHA8F/12Sl+6yHptosUi0BWm6Qt7XSHewGb1YnuCgz/RdLgq2rc1vexW18uizyqcjfHlwK/dJAd0D5lleOKocmOsuBi4GvNohMgwSVpT+rTNxYdkVrb+xTZg5zP2e469X2zN8u54CI3a3V7i9UmNdnHKRZtgZO1O3ULxrYg2YGvOx52uBG4KEMfgFVwSLIxVU3N7rADCy/1ThCTLQTofI/s1FQJbyWL9McWef65GxGz5/y42EcNj2HXH2AAYMR04DRF7mCzRXWWkxTdhWREG9lKrYqm2vxviWZSpVWw2gBYgO1v6yc6vKnDTUDXB9y7ry1Ufi+qOisOu0xEnOnbFIq1svEZSfWjirQy2xk/yUWwGam0mqLLDeCiBhhBZqaVd7pXdyxxmAQmXKi5sP79NGDu2VLwWWXRFTKBn7dL3R7aTJ++rr6OL5V6fYuuqPt9Z50OzNEUIm+oHCWyCHN8idQhcGkvlcnTpOHquqnrn5R6DgsUuwkAmYjO36Oug+VNJdny/cCP7t0/fgBw/k7XVCiAHMPPXq6vSxYYLSnIq0vtGBAmkZKeVnJCava5M6nlKANW3Qvs/tTz/QJOTkxVTenqXATUnTPVJKB8IkFXy8qbojoDA952L21qfRgO+du0ao7/HW+W37Wu3kl99XkZaGPCJEtO+QKOrl5Jyihg2Ld1T7uX/gew/nFgyXXq9F8Ly8c3Z7rDqno+JamM25iwWLLqPuC7BOArxWeh1USg36uSikvlr8nVT5atexz4cyxQ6oXABDeFBIYhzgbsTDkqu7qqiusnNeDWPgwcVtShzdogr0tXZygkSVJT6yYZzVJ0VNLUqeoQArIzev4EqQuvUpguu3EdijRjR/8CvgwAtr3tqd7WXvwASaEbrSkdMG4VcIbmWOqsyzXwA+/0zbmrMH22dx7f3/w+1JygnPVPyKJyTTXkvGHlXcD3ye5lW6iPLa/LrqlOt6vbncfI/d975/l/6SDXkAoRBatg3fKSOrgmrDmw53MJbFAJK9/RmKHIgFWYDtjC5Hgc1tLNjmsExcp5ni3MO7shT/tcdpuHt3Jtc9ax98Y1Q3XCmstOMMOuDiYqPiqfMdXCKyDBJWmP6Rf7nDv/VAEqARFSB7mPJgjQOfY76yk3FEfmALNHAxkL1e1jFkmteEOTZrbZ2RIsGuNm/ec9n8rO1iNVNgIYZbKoqkqdW1/WYCkpFBijv83mlyWQm0zDxT5qeHo8LhHTzkm7qpqfrU4dZrbMVVKTL2erun35LcC3MZriwtMkQlc3cbTkWqnDsl9T68qbkkcBbdyYAAdkAjJnq3rnSN4uYHpv9a4/s3WZAkS0V0dama3l+XIR7OtJRvIfrSZKVKpuh51OSCKQPFIKbKs4UznoxpMhnwJd7tVHiXlTh/IFnPZuLHKGtZAo2aONKHrNFnpyF5CqvlHJCWDWafo6bt0flu+qHfYFh2Ryp9VEiUb0pZjyGhHu7tY9thT4Y4REWKrs+q8c55UccsGni8x02IGFl3kn9Vj8QDl3cqt+mAXY8m9g8dUe71a1QpKl7phuUqomCQNlYWu0YvKnqSrLl4lL3Q58w/BePesl15bX39DUl/amyXbgsjL9REj6n8C6x9Q1savT8TbJXnHwZ/VEXuvJQNoTJ8e7qtpfD/T2QsrU2kgoXzxP1VwPrX0EmKt5r46vkMmyLa+p2zc8BaT/rr5eMUmUpQyDQoAtJcCJAMX4brHIeXvqJHXKyk0vyuvS7ZJYfT+w4nbXNFhmc+7A1y1COoMlVHVmc3cCPzSTDCuqelMh5YE8ez73SFfr5MhcYO0/5Hyhrga8KzvkvVUT2DnO2Au98/j+puCAOZlyzvxDdhjpUuR6U3gb+a4Kdq6Po4skEGPL6+q5FWeKcd0Cfqc7gMFTZSe8NzQbJwEViuCE7MgRcAz4j353U1QXfcq/076U4ABVvbQDPwF/DJN6xt6w6j4ZR3XBRu4qygDmjJPAG1V6/xbnAqd9JYuYvrbvGxmfVZznHqWaedod/wGWVZMaOqylzC0mDnVts1glHfvWN9T3DW8tQR4jf9c/vhm2vQMcma2ugwzI3+3SG6WWssqGp4B1j+rTWtfEme0rflDln9uC5drQG2UajDJg5R3A7qnq9mNLZUejGXM6VKGBndGSv7HZFGlF9n0tg945G9S7OFbeLVEhZy3wev/q5PchslsisrM67dmuT+R7gKKm3xk/Vj9B0+clOZA016Ru8qYFE2VAP1eTeq868f2lWHOC4oIsKEYmgyPa1buLHtfxZvlqiDLmy4RWK/8ueOsxX1ikgPplxT59WuXY11BEtAfydkg0eUyPut33wM+yUNROsTOq37/lS6ftlfJlhvXlJ8Lu1Ano8HeJwHSm6WgMrDbZCdH2WnXh7oBIILqHBJm0u941dUjySFloUT52gBwztr0F9Pde/SOloFg5rrgbtWwLlR2kqojT0hxgSfnnute/XNtzd0qko+r4D8hk9L6v5Kubm7vldYqOADmb5Bykrpzp3apeJHrKtndkB9XA/wCBsSfHvuA4dTon0rMGS+083Xu1+UVgzUNSp6uzpjaIu44u8n2NQieLtfoSnBueBjLmypimS6OrEtYcOHO2TOJZg1zbO91a/f3z9sg1S7OxQKxiEtKb2l4hXyp7vwI2KsYop9FzJKhBVz/09K9l15enaxTVQ3TuJvzSHLjuCPBIq8sQW/UGZfnyHvZ+UXZhV9X+ejmn6aLYFQjIJHLWWtnh3NY7u1fdOu8LawlcXs2OksiOQMpZQIIiANC5cBkUqy4hEdFW2pxBMr4U1gpIHq3ffVodi8W7u13OWeu9x/ZHw3+pPmOTtyQMli8z9HgE6P4P9zMX6GStl0CM9N/lWFd1rEu9REoqZMxV3z84DmjnpcAui0XqXS6+SnY0Vtm1G16yFdblj0i2kKpBQ8eWSK3h3i+oHzuqkz7IKXGoBLyteUiyH3nyPCV788la8bqMCu4qygAOl9cZbnO56xxfeOrJGo2+ZC+Wxeo+L6tTl3a9XwJjdX1bVh482/t59fieeBowqpqsXpuek+AAXWkBXweR1kb3h+RvSxcw7HyfW01UtwdEKutc1v75/wGkXqz+G/FWiQZHqfRbNa8NSNB4TJqcZ/g5M+f8uLOPTGOz2ZCWlub6B2ALAVLG6Lfi7/lcUr01NL2ek2hAXZ2rLvcC/d5UR20ERgG/tNen1UmdJBfnZhQLLzig361Yk7CWUshWdeEdkiipF9Y/UZ/e+Z9558sCrBk1CJoqdybI60E79jUULc4Fztsu43BdHJoOzL9AdoC4oyRbiqGrolW9bdCHwMSjQDfNDo7qtL4UuPi47DpvTDLmyeSmSkCopKeL7Vv3OhUhSUDHWyTdcI6PUxgBkiplwkH37pswUFLqqIIpnIu5zcer7xsQJosNuglyi1VSIPbywk6g3G0SiVvXnU1OFx4CRmh2M9bXitskhUzezoY/9jV0tiBJw6Wri+rcpavarVtf526SvwHtzlYTOWvPqRbsqmMvlgmJsJb6OuBLb9RHmS+5FljzoKRSbEhytwOWAGCcZnwPawmcv0MmqFRSJwH9XvP8rpR6CI7rg/eMriiJ6Y22sYrJo7w9wOwRJwMrq0oeAYxdArS+RN3uTG93bEm9+6pSr7Fv9YPAzo/UbVGdgDN/V6eSDW8tQTkXZ6oX+yxWaRv8cd37VF9GKVCw3/epvsn3YnvJuZGvlebJtbIqha0veHqhD5DMKMO+l92twQmu7V3vk++qHXCA7HCeNcx7f3ctzwfGb5SAwFPYbDa06jFO6sGr6qIGRst9whTpLAFJ3awrLxGTJterJ1Z7/voxIEJ2Kw75r+fLp5y6CBKo2NyQv09qye/2ccpKR4kEC2VvVJ8XHZouwZy6CKyOt8j5hy7tac524OsoYIMm2C9/rz4lu6MU2PAv76WhdVdcP2Dg+5KeWmVLeaBz68vV7SHJQNY6INuNjRUAkDwc6PA3z9eVrJYhmQcMzfga3x8Y8Zt3aog3ImZf93Kxj0xjGAZycnJgVN3Rlj4bOL5MX5vpnA1yItHQdJ0i0YC6Rcq+LwOdNemq8vbIgTGivde657a4vm6mBquFVhO9V1umqXLukix0czKbKhs+TSL7fUg79jUUS66Vmj6B1RTgVglJkWjAHo+p21c/KDspdROnP7YCfuloTjFnRwmw+ErZyVZXax6Smk+NiaNMFoYO/Ajs+1Z9m7ZXAONWqnesLL1J3kvdfQ/9JhfeVesH+MKfZ7m/4Fyd4CRg7DKZZFEJbQZ0uUeOmTpD/6ufYK+PLlOAyQ7305qFNnM/9WlNBn0kKegiOzb8sa8x2Pa27OBTce649cZuBsMASrLMCcaoSdf7pVZssibYTmfb28B38TKOqT6Ta/4B7PxQP44FlB8jVbujvW335zIGL73Jta3HI8DkUiBWk/b00MzqJ8tKc/T1eExiiWyHmy9Zgs96jYL14DTXG4Q2k0XKQ9OAnG11f4Ku9wI9n5Gds17g9tjnKJW/942aIJHMlcAPLSV1Wl0VHweW3WJOOQVHqeyo1U0UmmnlFOCHFkD+frN7QvXxY0tJv7jva7N74jmBERKM1vFm9TWacwFFt4P50G8SMO/OeFEbCy+T42mVem2GYaBo3wxg0wtyHlFVdFfZlXh8mfpxt75afizWpG1vdy1w7jYgysM1nMNbSYaSlLM8+7iA7E6uqGmtSGm59yupJa9LDekttjBg2HdAB02Gq73/kz7pAqUHvCPnH7pgoaJ0SStdcEDd3v0fQOd7NJ2zAOseqb7mcEM08neg+yNA0nB1e1hLIK6/+zvW9/8IfJcIqM6NvCUwUrJiDdKMJYZDxmDnTk8/ZfZ1Lxf7yDQOhwO7du2Cw1El8qPPS8Dwn/V3DE3WF/RuyHZ/Diy6Un0BvfIuuejSbbWec075JLmPU6IBwKg/pViup5XmSJ3DhrjA2ZAN+hA4a5Gki6X6azFevxvXS7RjX0NhDZb0XroUXzpxfYALdgM9n1S3O2uO6nZfRJZfpDlrXfhS/h65qNLVaqvOns9lN0HhEU/3yntOjThW1ZizFwHT+0jdJ5XQ8lo/mxVF1MsKT9ZnaVbH3aGeYAvxzm5dW5AEvuiitcvygXnn+v7CHJDoW93OJLO1vw4Y/BEQHNfwx76GzjBkkiRxmO+f+/gS2QHU71XfP3eNLFW+15IzkGGRJtr6xBogMEbS0auMmCYTdSFJdXteT3DWJ9r1oWtbyQlgWhdg+3vq+y7/O7DgIiBLEzj5TTTwfaLsLGhISnNh2fKKevE1OE4m0jJXSm3JqtY9LtdRizSpwmPSgB7/1KeKqye3xz5n3bioLpoHLpPgP9X5R2kO8E0M8IVVbldVWT6w4z19AJY3tb5UdpcmaWp0mWnrq/IZOrHG7J5QfbQoTxWpquHZWJXmlGdAyVe3OxfuM1eq2/u9LmnAQ72Ukjv9d+DoApfxxuFw4FBuBBytLtYvAu3/Tl8nfNgPMl+kCg44+BvwSwf58oZfOsiisac57DInWKqpBZgwCEg6Q9Jh+pLVJvON885RtzuDnHQ1DPP3AyfW6csTOa+hdMe0jrfp52WsAfJZOFvz+W6oorsDG5+R8yqVdtdIatNwzc7Wmmx5WcqQ6HZTmqGgfFNCqBf+dhoRs697WbOPGp59X0t+7HFrT04oNgWLyy8wez3r2tbmCjmo6/R5SSYazapn5Q32EmDvFxIZ0v56s3vTeIQ1ly8ib0k5SyL30mfLYqinjFlUXgtAczI7zsST97zd8r315Lrft/sjwPKbgez1EozSGFiswCV5cmGgjCS0ykTXiTVAp9tcT9Z7PgU0P0e9cOs4pf6lGbn6g+Jk4dbTDAOYf6FEHk8udW13lEoKz2ATJv7JP1gsQN4uIKa37597WXmU97mbfP/cNdn4rEykjPwdaFaHCPyUUZKe68Ra9WL5yN+qv7+jTMY7a8jJ2pe+0udFiYC3Ki7l938v6feX36KuP93uOkmhf2SOul5bRAep2+uNNHTuOrFWFmXTngA6abKktLtedoeoUqI5U7A1pMmw2rCFym7yqK7q9viBwNDP1TXdS3OB0uzy/yg+384aQSFN6FrbE8atltS8upRs1DgM/VS+mpJ93wFLy+dMJh5xDTSJ6gTkbAbSZwEdFTtqYntJOmNv6fmUpPrOWi8BoJVYYCnNVmcHKDgg43b/t9WP22qCfCmVj+kWmwQqqlIWu6sku+bbuCtvhwTlAMCouZKK8VRJZwCj53nv+avTYryUYlBJnSTXhLpdaLNHAnk75RpTdZvYPsC5W/SL8AsmSrDFBE2wkfZz0IDZvZwRo9k4WTROMiEQUMe5c1OX8Y58got91PAExQHhbb2XUsosgTGyI1GVj7zVhRJ1uu9bKbBaVUx34Iwfvd1D3wqOl+idhjSZ0BgsmCSFtydmNNydHNS4RXcHOt9d94Wa3B2ShhOQ3Q5VmbH7obbi+gLDf5G6q3WVMhoY+j8guofn++VN1aULsQUBvV8E1jwgUZqqyDxdqkBnse6ozrJA5utxKvUS/UVqfZRmV59i1lEiF6jevqhrbL4of//PWQdENsKsDA3NRZnmPG/PJ4Fd/wUyFjSsCQXg5K5wdyKI214lXyqGAzi+QmpMq46H65+URcahX0h9al+yBqprgANAq4uAgkP6cgid75SU8FGaDBHnm1BrtUYWCagIa6m+Piw8DPw5RoInVamSu/9Dvhoba6B6wdbJYgHaaHamBicCY5fLpKpqMdoWIp9dM7IpNGSxvfU1o4jMFNtTFrVsIVAmaBv8CfDHGfoxI3Ol1ExrNcGzi2IV/esLdLxVuZATUHZcaqKWKhbQijKAPZ/pM3cdWyZZDVJGubYljZDvhl12N3lyccG5s7rjLZ57zAqnvH9Fip3ZJSeAjIUyBxjRzgvPX411j+kzs6RerJ6rdHLWDncogiIBCULZ9Ym8lymjXduPL61TVxuFiHbAJfmSOckbevxTvhqS6G6SHc6deRXyGC72kalCQhQnGt3ul6+mZtIJfZujFNj0HND26uoPoE2JxQJkzGuY9RoasqAYOZm1F7if25tMpxz7GoqiwzJpWtd0yYXp1bcfXw7smgp0uEldS2jFncC2N6Uuq2q3gTcFRsnErjtF60tzZNK1se1Ez1wNZMyXYJPwVNf2NldIZKnqc3Bwmryf3f/hWrTeOamYs1Xqh6Q96vm+V8deAAQneP5xrcFA22vkd6ISkizRqkGaFEX+KrIjkLtd6oCggY99jcGs0yQy3tcLMs3GAQsvAUISGt5iX7urJfNFXYPHMhZI3b5u9wNx/dTts0fIufmQqa7t+76S72ZELufvk3SWsX1c66oGxVQ/7gbFmpNiuT5iewLjN0m0eMkJeQ2nsgRILaCs9ZLirgGeH/t87LMFAfH99e3WAN8vUjcGBQdkjPVSSlfykeW3A9vfllrLTWWXZlw/YLIiJa9TUIwEVums+Yek2jxztndKWBh2OV4qMhAVNLsMjjMehc2mCDyI7i7vky1U/bgLJkhAxwX7XLPDlBw/+e/y80yPCU4Ezt8pAfueFp4K9P23ZAhoNta1/cDPUn9cd/7hTS3OdT9octQfQN6ek+k+qypKBzY9L5l+VIt9kR2BMMU1aWNmsQIBHv5snipzlXxe2l4JRHopnW1dBUVLHXEy9bqXW2rINDabDV26dFEf9JuivN3A0UWSvrKqLa/K5GTa477vl1kMBzDwPWDY92b3pHEZ8C4wKbtBTmRQ7TT4sW/3p8CaB/X59nXiBwIXpgOXaHY2/TFCLryPLVa3byuvSaqK+vS24mPA/AuAnR/U/b5r/wHMO9/zffK2P0cBq+6WaFqVw9PlvQhUXLAd+AnY8BQwS1Fv59TPTXhrj3S1Tra/5526eQGhwJBP9GmnrQGyU0a328ZfnbdNdvpGtm/4Y19jEBTj+yhvQHYBjJgOdL7H989dG+5kicjfKwt2MzQLIhueku+pl6rbB34ADPpIdsn52ra3gCXXqftuOORaw6EJpltwMfBDNTWb5p4rO3J1daHMUngQ+LktsOFfrm0hiZJSe+8XEshS1ZG5wM8dgf0/eruXSqaMfYYBHJ4F7P3Kd8/ZFPzYCpjWGUhX1IakxsMZhBHMc7IKzowc+bu98/gLLgI2PO3y4xrHP1uwZLiaqSlp4zzGZm9wbbOXAC3OB4Z95xoEUl9WmwT97PrEs48LyGvuco8EG6l24TuD4c2oeXb610D3h927b1w/IPUidYpxQBajkoYDLTXXzedtkwXDpsRRKpkHNr3oncff/h6w4UngWBPcFdnImX3dy8U+Mo3D4cDx48dNK1jpcwsnSUR28THXtuLjUvvIG7sRGipHmUwUK04KqRrWANmFRI1Wgx/7koZL8fasaqJDVWxBUrMuQBOZ2fu58hP8C9Tt41YDI37Tp4f0ppyt8r3YjRR5tlDZTXZsmWf75G1pT8iiQYqmxtXSG4HNr6hrHPV4TNL1tDjXtc1xSkBLu6s90tU6iWivv8isrzUPAYuv8c5j+4EGP/Y1BpkrzUnhvfASqfOm2pXdWLWeDLS8UH9O1e0hYMinQItz1O3Jw4H213lvvKlOi3OBqC5AT8U5dMZ84Ktg4EtNv3K2SMrhPV+q2w/9Kt/d2enuLaW5wJ4vgIShch6hkjAI6PG4eqf6vm+lRtKWf3u3nxqmjH2l2cCcMcBfl/nuOZsC5+J+UIyp3aB66v4QcFkJENHG7J54TvqfEogxfyJQVlj3+3e5Bzh7BdBygse7BkB2Eh9bLPXDTlHj+GcvkRp/upTa/d4ARs4E4hTBLXk7gYM/y45vb5wbLbkWWH2v5x+3JqkXS6abboq01I1ZQDgwei6QPELdvvw2YOlNvuyR9zlKpY7mro+98/jOc1jVuQ+ZyuzrXqbxJNMYhoH9+/cjJibG7K74RuZK+a6aVOjzEtD13rrvpGnMrAFSV8TX6fqITNbgxz5rEFB8VC6c6sJeLAtEQXFAf8XOqs53ypeOmXVSQptLmrpWF9b9vq0vk9+XbpGzoarp/UgaLul+TqyVi/BThbcCxq1U388aKCkvO/zdc32tE8M7k9QOO7DpBfm3r1PqNBENfuxrDPq/BYSYkDI4KBZwFPv+eb3JagPOqCa7RDNNIIRTxnxg50dAlym+XwRNOgM4d7O6raJekmbic9D/ARufU6ZaAwCct11SpjWklMQlWbKLvssUoOV56tucWCsZBlQTXl3ukR0Szcd5tZs6pox91mAgvO3JmpZUO6d/CUCzEE6NizXQ7B54ljNg/MAPUk+urtcdQTHqlNWeEtUFOPqXy6JbjeNf3k5g47MSSKgS1kK+VOL6SaCot2qDnbcDKMur+XaeFhjln3Nk+74B7EXAoP+Y3RPPCQiTz5G3Mr90vlPmVJtSMF4TYfZ1L3f2EfnKoI+AYT+oU6KVHAd+bCkXsv7CYpXor4PTzO4JEZ2q3XXARUf1tcl0cndISshtb6jbHWUSvakLashcJeOBKtWxt0V3BUb+5uZi3yXAWQuAmDTP98tMXe4F2lylTotTVig70lVp4ixWoNcz8jkwY3wf/BEwbq3nH9c5eZE6yfOPTVRb2ZuBI3N8/7y9ngXS/wBWmRDh7i3FmZKiq/CIur0wHZjeD9iqOabNvxDYPRXI3ea9Projuqukzr1cE0mcMBgY/pP+GB/ZQeoy2oK818e6CkkEzpxVfRDJituBHe+r2yLbAz3+CcT19U7/GqKAUOCCXcCZM83uCRF5QutLgPN3A2OWupflZ81DwJdBUvPVGwb9HzDZXvfaecHxQNf7gcTT1O1LbwKmdVW35e8F5p4jJSi8IbK9a11cX0ifDXwZAuz80PfPbaZ+rwED3ja7F54X2d57mbnCU4HEISzxQy642EfkK+2vA1pNULflbJEUnvEDfdol0/V6ViaUiajhWHU38E20LObURUgS0PEW4DRNRPT0XpJa7MBP6vbfhwDzzgNKc+r2vJ6Qsx34MVXqp9bVynuAnzuo0102ZLNHSTqg3ZqafS3PA4b+V50Cad0jwHcJwHpNndm95Z+BjHke6WqdzDsf+FUzKVAfFiswKVfS+hGZ5eAv8uVr1kCg2VipS9lUHP0L+OMMYNUUdfuS64ETq/S16zrdAUR1BVJGea+POtvelvH7jxF1v+/er4A1D+uPWVkbZGG3IR3TbCFAdA9g1T3ATk0qrA5/A44uBLI2urY57JJ9oCG9JmqYZo+Wv608L9U1I6qPiDZAwkD30ken/yEpBY/M9XSvxLSuwOKr655OMyRJdiLrFrb2fC5zZao6mkExkpI7uontgnOUSDaFhpRO2xfaXK5P50pEdcLFPjJVZGSk2V3wnY3PA9P7qCfQD/wsqRl0+aubqu3vABlzze4Fkc816LEvtLl86SY4dUISgQHvAK0vVbfbi+R7WEt1+/iNwLDvgRATapc6ioGC/UDBwbrfN3O5pKAp2O/5fnlTUUb17d8nA3POVrclDJHvRxeo20dMB8YuA7r/0/3+uavlBUC7a73z2CfWAMeXe+ex/USDHvsag4Awc9LyHfkTSD4T6NCEaqnE9JDve7/QtHeXnc3dH1a393wCOHeTeveztzmPp6odGrk7ZEfAXEVNVQBY9xiw6XngxGp1+29pwJ9nuVfD1pvsBbKrNW+nuj26B2CxAY4i17aVdwBfhUiQi0l8PvY57LJo9YUJNT4bsyOz5XvBAXP7QVRV4WFg79dSZ9yd0i99XpLz44RBHu8aAOlfzhZlU43j34nV+jmhYd8BLc6XHYAuD9wBOO0LfXrnxqr5OGDiEaDL3Wb3hIjqwczrXtbsI9PYbDa0b9/e7G74ztryyQJVVGm7a4DE02V3nz8pKzg5YUHkJxr82BcQDhQekou2uha2z90pk22q+52vmaBziuwgX2aI7i47EpPdmAhsNk52iJScAMJbe75v3jJ+ffXtZQXA4Zky4VV1gTb1YuCiY4BFcxoZkui92gQ16XKP9x77j2EALPr0eFStBj/2NQZBseYsLu38UBb3m9LEU0RbYOTvQFG6uj20pYzr+78HWk10bc/bDZRkyiKTLdi7fa2q673ypVKUIQEsh35Vt3f4G7D6fkl/pqrf1OoiYP93spuuoSjKAH7pCHS4WdJEq3S6Vb5U4vrL97qmJ/cQU8a+uu6uIXFxlgTghmpqWhKZ5dhS4K/ygMpL8uqeti95pHx5S2wfIGudy49rHP8Kj8j8WK9n1e3Nz5YvfxOSZHYPiKgezL7u5WIfmcbhcCAjIwNJSUmwWv1gk2nqpVKbTzUBGpwErH1E0te1u9r3fTNLTZPNRE1Qgx/7UkYDAz8AItrV7X7FmcAv5Yt1l7sRcWomi0W/I7EmHf4GtDwfiDBpodJbut4HrH9CJrtVuzFVEbZO+fvkgj9+kHmLfl7TyD7bDUiDH/sag94vSkCFr3X/B7DuUeDQjKY16dbsLH2bM3ijrEDd/tdk4PhS4JwNsguwoYgfKDurdTreKmnPguLU7cO+9U6/6sMWArS4oPraSV9HAInDgJHTXdvaXy9fJjFn7LMAl+RLCl6qvaBo+SJqaOL6ScCCYeiD7czUZYoslFdR4/jnKAaOztfXVD34G5CzSR7fwnNHImoczL7u5WhJpjEMA+np6TDcSUPQGJ3+pRSXVynLAw79pk19QERNR4Mf+/J2Sa01R0nd7ufctRygSVew6QXg+xRJMdaUhCQBMWlAQKjZPambHf8HzBgI5O1Rt3d7GJiUA0R1c23b9x3wa3fg6GL1ff+6TOovbvyXx7rbIPR9FTi9AU6ENxINfuxrDBZeIjVxfC0oTo4LxzR/841R5krgt57A7s/V7dYAmVxtoUkP1vMpoM2VQFQX7/VRJ3MVMPc8qb9XlTUAiB8gXyqOEjleW4O820dPCowC+r8F2EIlhZ1KVBfJStDQ0o/CpLHPYpG0v1zsq5vDs4AtrwH2Op4DE3lbeCtg9DzgrPm+301eG3k7lTvCaxz/wlOBM2frdx0uva58N/oez/WViMjLzL7u5WIfUUNw4EdJxdBmstk9ISJ/l7UB2POZ7ESui5AE2dF3SY66fc1DQNERoFCTMo18a9nfpN6grm7TpueAnf8HWBW7iE6sBrI3ActvVt+37TXyPWW0Z/raUHS5G0i9yOxekD/r/rBEt/taRBtg/Cbvpsk1Q9Z6YPGV6rYdH8iCoG4nQbMxwNBP1WOktx34GTg0DVh6g2tbWQGQsUAfWLP8VuCnVKCwkdUk+7GFLHTv/q+6vdXFsqM8f7dr2/7vpXbdphe828eGZvltwPLbze5F4zJnDLDqHiBrrdk9IWpcNjwl1w3uWPPQyRSlVTUbJ99LTrj32EREfqgB7v8m8kOhzSQC19qA6mMQkX/qer9MJns6avTM2UD2Rv1uA/Kt4b8CR/4EEoaq2zc8DcAAOt7iGqnb4xFJJRjbW33fjn+Xr6Zm5mCJQD79a7N7Qv7KrFSES24A9n0FTLab8/zeENdPxjfdTjFnza6s9UCiZpw0S9srgdJsIPlM17bsTcAf5bXpVCm1S8p3vh1f1njqzJ5a31tVPxEAWk4AIturU5AfXyHf934JdHvQ491rsLa/I98HvGVuPxqTtCfl9xaSYnZPiCo7ugiYdZr8+7JS2cXdkFisyjSeNbIXSe30hCHq9v5vSq3WYNawIyKqrQZ2hCB/YrFYEBcXBwsLiEuKoOAEc6KDicinGvzYZ3MztZfDDnwZUF4vaKlre8qZ8kUNQ4tz5Eun3bXAzg8lzWd0lTR1thCg55Pe7F3DdHwp023XQ4Mf+0gvrh9QeBAoyQKCNbXeGqMB7+jbUi+WFIhhLXzXn9qK7AD0e1XdFtYCCIgAYnqq23s+A8T2BWKqqX/X0DhK5Xu7a+WzqLLrIyC0BRAU69rW45+yUB5sTg1Z08a+jrdKQCnVXtpj8kXU0ChSZDYosf2U42+N419JFrD9balD2PtZ13ZbqLQxJTERNSJmX/daDBbO8IicnBxER0cjOzsbUVFRZneHGpsTa4HpvYG0p4C0R83uDRFR3TnKgC/LL8RUuwkKDkjEZ1Q39xcUyXMMBwCL1PVRyVgAHFsEtL8RCI6v3FZwCMjbIZPFQdFe72qDkb0FCIwAwlqa3RMi39o1FVhyrexqTZ1kdm88o/g4sO9bWTyK7+/abi8BjFLAFqYfJ8k3DIekVQ2MBNpcrr7Nt3Gy23zUnz7tGhGRz9iLAaOsaR2XygqAr8MlWONCRXrp+RdKyZuJR6ROOhFRE+TpNSXW7CPTOBwO7Nu3Dw6Hw+yumK/goESbRnc1uydE5GVNduyz2ICh/wPGLFa3/9gKmN4HyN3u236R2vcpwP+sQLpmYjS8NZB6qetCHyA1Of4YDiy5zrt9bGiiu3Chrx6a7NjnD2LSgB6PATG9ze6J5xQelrqj885Tt6+4Dfg6QtJlNjSbXpAadD+2qvt9d38GzLtAXn9jYbHKe7XoCmCLZkdjh78DGfOl7nADY9rYV5INlGrqKJPa1xHyt5W/z+yeELmyBQMB4Y1qoa/G8c8WClycBYxfr24/ukC+Fx3xSv+IiLzB7OteLvaRaQzDQGZmJri5FEDBPqD4KBAUY3ZPiMjLmuzYZ7EAbS4DEgar25NHyvemlAKuMRs9F+j2MJA8Qt0+52xg3nh1W2J5zZCAcG/0jJqoJjv2+YPsjUD+HiC0CdWxCm8j34vS1e35e+V7oabdTM66dHGKHYk522Wx4mtNVPDml4GDPwNH//Je/zzt1DFDd9yJ6gokni4Txw2MaWPftzHAN360+94TyvLlu73Y3H4QVVWYDiy5Htjyutk9qZMaxz+LRbKEqFIwA8CQT4FuDzHYjogaFbOve1mzj6ghSL0ESB7FkxgiatwO/AwExQFJp7u2MbVWwxLdTV0bw8mwAznbgPz9QHiV3SMpo9SpWomoaTq2FNj9X6D3i5JKsSkIjADGbwQcJer2hCHAkT8b5g6K1En6Mdgor29Xlqtu73QrsPp+IKQRLdwaZSf/3eEm9W0SBgFJw4CItr7pEzVNPLehhqooHdj1sfy7y13m9sWXmo+TLyIiqjUu9hE1BPl7gIWXSgH5dtea3RsiIvfMv0C+c7Kk8Wt3HbD9HVn0IyL/FpMGxA8CyvIAJJvdG8+J7qZv6/mkfDU20d2Ay0rK67IqdLhJv2DWUFlsQPd/AjE99bf5YwQQ2R44a6HPutXg8VyMqOmI7AT0eFRq3BEREVWDaTzJNBaLBSkpKbA0xIhZn7OU518PNLsjRORlTXrsO/1rfc0+aly6PwRM2AdEtHFt2/OFpInb+qbPu0WNV5Me+5q60mwgb0flHVaNXf5+GceW36Zu3/05sPgaoOSEb/tVG+l/St/XP6VutwZKbSeVvD3AoZlSz62xsFilVt9flwKZq9S3iesnqVeLMnzbt1rg2NeIbHwW+P20k+k8iRqKgDCg51NA35fN7kmdcPwjIn9k9tjHxT4yjdVqRUpKCqxWfgwR1wc4Zw3Q9gqze0JEXtakx77USfqafdS4rLgT+EtzTHLWsNr9qe/6Q41ekx77mrpuDwAXHQOiOpvdE89x1n7b/o66/dCvkrq0NMd3faqtzJXyfcurrm0l2ZKmc9dU9X03/guYezZwbIn3+ucN9vLdLLpag3F9gYIDQPEx3/Wplkwb+35oAfzaw7fP2djt+R9wbBGzGlDDYzjkeFRWaHZP6oTnfkTkj8we+5jGk0xjt9uxZ88etGnTBjabzezuEBH5BMc+ahR2fSwp+wb95+SkuFOHm4DwVEkpRFRLHPuoQQmOk9SQut2Kne4AgmKBsFbqdjN1+BsQPwAIb+3aVngI2Fy+86PdNa7tba8GTqwG4vt7t4/ekjpJ/fNOdwCplwIR7X3bn1owbewrPASUamo3ktqYxUDhQSAwyuyeEFWWvwf4uXx8a0QpennuR0T+yOyxj4t9ZKrcXF6AEJH/4dhHDV6764Btb0oUcdXFvsAIIPVic/pFjRrHPmpQej2jb0scIl8NUVAMkDxC3RbWEmg1EYjooG5PGgacvcJbPfOe4ESg2dlAaIq6fcUd8nsZ+J5Pu1Vbpox9lxRIvUOqvcAIILAJ7WCmpiOg8S5A89yPiPyRmWMfF/uIiIiIqLIOfwdSRgNBca5t+fuBQ9OAxDOAmO6+7xsRUX3Zi4HVD8gOt7ZXmd2buikrADJXAGGprnVVAyOBYd+Z0i2v6vA3ICQFcNgBq2IBK3O57MYyHFLjj4CAULN7QESeEpLQqHb0ERGRebjYR0RERESVlWbJjgBbsGvbgR+BlXdKurTzd/i6Z0REnrHtDfne2Bb7Ds8EFkyUf/vL5O/GZ+V7UIz6/Wo2Dtj+NlB0FAhN9mnXiIiIiIgaCi72kWksFgtatWoFi8VidleIiHyGYx81CuseAzJXAZNOuLYlDJbv7W/0bZ+oUePYRw2KNQiwBjfMmnw1ieoq3zve6tqWtxv4uZ38uyktBFoDAUepuk4hIOlJ4XBNO90AcOwjonorzQN+bgPE9gXO/N3s3tQaxz8i8kdmj31c7CPTWK1WxMfHm90NIiKf4thHjULOFtndp0qJFj+gaU0ik09w7KMGxWIBLj4BoBGOZdFd9GOwLcS3ffGVs1dLDdmEQer2wGig5QSpudbAcOwjonozyoDi40D6LLN7Uicc/4jIH5k99jGhPZnGbrdjy5YtsNvtZneFiMhnOPZRo9DpNqDZ2ax9RB7DsY8anMKDQFGG2b3wrNBmshDY1AIyYroDiUP0x6SVdwKr7/dtn2qJYx8R1VtgNDDqT2DEdLN7Uicc/4jIH5k99nEGh0xVVFRkdheIiHyOYx81eN3/AYxsXBMK1PBx7KMG5ZeOwO+Dze5F3WWuBL6wALPPNLsnvnPwV+Cvy4Hcner28DZA/l7A0TAnlDn2EVG9WCxA8kig+dlm96TOOP4RkT8yc+zjYh8RERERERH5l5heQNIIs3tRdyXltVSPzFG0ZQE/tgaW3eLTLnndns+Bvf8Dio+p24NiJc0nd6MTUVNkGMCxJUD2ZrN7QkREDRxr9hEREREREZF/OWeN2T1wT8pofZrOsgKgYB+w4z1g4Lu+7Zc3Df4I6HovENdP3X76V4Djc9n9QkTU1FgswO9D5N9NLU0zERF5FBf7yDRWqxXt2rWD1coITCLyHxz7iMgfcewj8oGQJGDAe0B4qtk98SxbiH6hDwD+GA44yoAxf/muT7XEsY+IPCYkyewe1AnHPyLyR2aPfVzsI9NYLBZERUWZ3Q0iIp/i2EdE/ohjH5GHFB0F1j8BNBsDtLygcps1AOj4d1O6ZarSPAAOs3uhxLGPiDyiEe7o4/hHRP7I7LGP4RVkGrvdjvXr18Nub5iF1ImIvIFjHxH5I459RB5SdATY/g6w6QV1e+FhoCjDt30yW5+XgN4vmt0LJY59ROSvOP4RkT8ye+zjzj4yFQ/6ROSPOPYRkT/i2EfkATE9gFF/ArF91O0/NJfvjXAXiNtSzjS7B9Xi2EdE9fZFeU3SRja2c/wjIn9k5tjHxT4iIiIiIiKixiJ5pL5tyGdARBufdYWIiHwgfjAQFGN2L4iIqIHjYh8RERERERFRU9D2CrN7QEREnjZ2sdk9ICKiRsBiGEbj2gPeQOXk5CA6OhrZ2dksQFtLhmGgqKgIISEhsFgsZneHiMgnOPYRkT/i2EdE/ohjHxH5K45/ROSP6jr2eXpNyVrvRyCqh6CgILO7QETkcxz7iMgfcewjIn/EsY+I/BXHPyLyR2aOfVzsI9M4HA6sX78eDofD7K4QEfkMxz4i8kcc+4jIH3HsIyJ/xfGPiPyR2WMfF/uIiIiIiIiIiIiIiIiIGiku9hERERERERERERERERE1UlzsIyIiIiIiIiIiIiIiImqkLIZhGGZ3oinIyclBdHQ0srOzERUVZXZ3GgXDMOBwOGC1WmGxWMzuDhGRT3DsIyJ/xLGPiPwRxz4i8lcc/4jIH9V17PP0mhJ39pGpSkpKzO4CEZHPcewjIn/EsY+I/BHHPiLyVxz/iMgfmTn2cbGPTONwOLB161Y4HA6zu0JE5DMc+4jIH3HsIyJ/xLGPiPwVxz8i8kdmj31c7CMiIiIiIiIiIiIiIiJqpLjYR0RERERERERERERERNRIcbGPTGWz2czuAhGRz3HsIyJ/xLGPiPwRxz4i8lcc/4jIH5k59lkMwzBMe/YmJCcnB9HR0cjOzkZUVJTZ3SEiIiIiIiIiIiIiIqIGyNNrStzZR6YxDAM5OTngejMR+ROOfUTkjzj2EZE/4thHRP6K4x8R+SOzxz4u9pFpHA4Hdu3aBYfDYXZXiIh8hmMfEfkjjn1E5I849hGRv+L4R0T+yOyxj4t9RERERERERERERERERI0UF/uIiIiIiIiIiIiIiIiIGiku9pGpQkJCzO4CEZHPcewjIn/EsY+I/BHHPiLyVxz/iMgfmTn2WQxWSvWInJwcREdHIzs7G1FRUWZ3h4iIiIiIiIiIiIiIiBogT68pcWcfmcbhcOD48eMs1ktEfoVjHxH5I459ROSPOPYRkb/i+EdE/sjssY+LfWQawzCwf/9+cHMpEfkTjn1E5I849hGRP+LYR0T+iuMfEfkjs8c+LvYRERERERERERERERERNVJc7CMiIiIiIiIiIiIiIiJqpLjYR6aKjIw0uwtERD7HsY+I/BHHPiLyRxz7iMhfcfwjIn9k5thnMZg82SNycnIQHR2N7OxsREVFmd0dIiIiIiIiIiIiIiIiaoA8vabEnX1kGofDgfT0dDgcDrO7QkTkMxz7iMgfcewjIn/EsY+I/BXHPyLyR2aPfVzsI9MYhoH09HRwcykR+ROOfUTkjzj2EZE/4thHRP6K4x8R+SOzxz4u9hERERERERERERERERE1UlzsIyIiIiIiIiIiIiIiImqkuNhHprFYLIiLi4PFYjG7K0REPsOxj4j8Ecc+IvJHHPuIyF9x/CMif2T22GcxmDzZI3JychAdHY3s7GxERUWZ3R0iIiIiIiIiIiIiIiJqgDy9psSdfWQah8OBffv2weFwmN0VIiKf4dhHRP6IYx8R+SOOfUTkrzj+EZE/Mnvs42IfmcYwDGRmZoKbS4nIn3DsIyJ/xLGPiPwRxz4i8lcc/4jIH5k99nGxj4iIiIiIiIiIiIiIiKiRCjC7A02Fc7U2JyfH5J40Hna7HXl5ecjJyYHNZjO7O0REPsGxj4j8Ecc+IvJHHPuIyF9x/CMif1TXsc+5luSpnYBc7POQ3NxcAECrVq1M7gkRERERERERERERERE1dLm5uYiOjq7341gMJk/2CIfDgUOHDiEyMhIWi8Xs7jQKOTk5aNWqFfbv34+oqCizu0NE5BMc+4jIH3HsIyJ/xLGPiPwVxz8i8kd1HfsMw0Bubi6aN28Oq7X+Ffe4s89DrFYrWrZsaXY3GqWoqCge+InI73DsIyJ/xLGPiPwRxz4i8lcc/4jIH9Vl7PPEjj6n+i8XEhEREREREREREREREZEpuNhHRERERERERERERERE1EhxsY9MExwcjMcffxzBwcFmd4WIyGc49hGRP+LYR0T+iGMfEfkrjn9E5I/MHvsshmEYpjwzEREREREREREREREREdULd/YRERERERERERERERERNVJc7CMiIiIiIiIiIiIiIiJqpLjYR0RERERERERERERERNRIcbGPiIiIiIiIiIiIiIiIqJHiYh+Z4u2330abNm0QEhKCQYMGYdmyZWZ3iYioVp577jkMGDAAkZGRSEpKwoQJE7B169ZKtykqKsJtt92G+Ph4RERE4KKLLsKRI0cq3Wbfvn0YP348wsLCkJSUhPvvvx9lZWWVbjN37lz07dsXwcHB6NChAz755BNvvzwiolp5/vnnYbFYcPfdd1f8jGMfETVVBw8exJVXXon4+HiEhoYiLS0NK1asqGg3DAOPPfYYmjVrhtDQUIwePRrbt2+v9BiZmZm44oorEBUVhZiYGNxwww3Iy8urdJt169Zh2LBhCAkJQatWrfDiiy/65PUREVVlt9vx6KOPom3btggNDUX79u3x9NNPwzCMittw7COipmD+/Pk477zz0Lx5c1gsFvz444+V2n051n3zzTfo0qULQkJCkJaWht9++61Or4WLfeRzX331FaZMmYLHH38cq1atQq9evTB27FhkZGSY3TUiohrNmzcPt912G5YsWYJZs2ahtLQUY8aMQX5+fsVt7rnnHvzyyy/45ptvMG/ePBw6dAgTJ06saLfb7Rg/fjxKSkqwaNEiTJ06FZ988gkee+yxitvs3r0b48ePx8iRI7FmzRrcfffduPHGGzFz5kyfvl4ioqqWL1+O999/Hz179qz0c459RNQUnThxAqeddhoCAwMxffp0bNq0Ca+88gpiY2MrbvPiiy/ijTfewHvvvYelS5ciPDwcY8eORVFRUcVtrrjiCmzcuBGzZs3CtGnTMH/+fNx0000V7Tk5ORgzZgxat26NlStX4qWXXsITTzyB//znPz59vUREAPDCCy/g3XffxVtvvYXNmzfjhRdewIsvvog333yz4jYc+4ioKcjPz0evXr3w9ttvK9t9NdYtWrQIkydPxg033IDVq1djwoQJmDBhAjZs2FD7F2MQ+djAgQON2267reL/drvdaN68ufHcc8+Z2CsiIvdkZGQYAIx58+YZhmEYWVlZRmBgoPHNN99U3Gbz5s0GAGPx4sWGYRjGb7/9ZlitViM9Pb3iNu+++64RFRVlFBcXG4ZhGA888IDRvXv3Ss916aWXGmPHjvX2SyIi0srNzTU6duxozJo1yxg+fLhx1113GYbBsY+Imq4HH3zQOP3007XtDofDSElJMV566aWKn2VlZRnBwcHG//73P8MwDGPTpk0GAGP58uUVt5k+fbphsViMgwcPGoZhGO+8844RGxtbMR46n7tz586efklERDUaP368cf3111f62cSJE40rrrjCMAyOfUTUNAEwfvjhh4r/+3Ksu+SSS4zx48dX6s+gQYOMv//977XuP3f2kU+VlJRg5cqVGD16dMXPrFYrRo8ejcWLF5vYMyIi92RnZwMA4uLiAAArV65EaWlppXGuS5cuSE1NrRjnFi9ejLS0NCQnJ1fcZuzYscjJycHGjRsrbnPqYzhvw7GSiMx02223Yfz48S7jE8c+Imqqfv75Z/Tv3x+TJk1CUlIS+vTpgw8++KCifffu3UhPT680dkVHR2PQoEGVxr+YmBj079+/4jajR4+G1WrF0qVLK25zxhlnICgoqOI2Y8eOxdatW3HixAlvv0wiokqGDh2K2bNnY9u2bQCAtWvXYuHChRg3bhwAjn1E5B98OdZ54lqYi33kU8eOHYPdbq80yQMAycnJSE9PN6lXRETucTgcuPvuu3HaaaehR48eAID09HQEBQUhJiam0m1PHefS09OV46Czrbrb5OTkoLCw0Bsvh4ioWl9++SVWrVqF5557zqWNYx8RNVW7du3Cu+++i44dO2LmzJm45ZZbcOedd2Lq1KkATo5f1V3jpqenIykpqVJ7QEAA4uLi6jRGEhH5ykMPPYTLLrsMXbp0QWBgIPr06YO7774bV1xxBQCOfUTkH3w51uluU5exMKDWtyQiIqJKbrvtNmzYsAELFy40uytERF61f/9+3HXXXZg1axZCQkLM7g4Rkc84HA70798fzz77LACgT58+2LBhA9577z1cc801JveOiMg7vv76a3z++ef44osv0L1794pays2bN+fYR0TUQHFnH/lUQkICbDYbjhw5UunnR44cQUpKikm9IiKqu9tvvx3Tpk3DnDlz0LJly4qfp6SkoKSkBFlZWZVuf+o4l5KSohwHnW3V3SYqKgqhoaGefjlERNVauXIlMjIy0LdvXwQEBCAgIADz5s3DG2+8gYCAACQnJ3PsI6ImqVmzZujWrVuln3Xt2hX79u0DcHL8qu4aNyUlBRkZGZXay8rKkJmZWacxkojIV+6///6K3X1paWm46qqrcM8991RkeODYR0T+wJdjne42dRkLudhHPhUUFIR+/fph9uzZFT9zOByYPXs2hgwZYmLPiIhqxzAM3H777fjhhx/w559/om3btpXa+/Xrh8DAwErj3NatW7Fv376KcW7IkCFYv359pZOBWbNmISoqqmIyaciQIZUew3kbjpVEZIZRo0Zh/fr1WLNmTcVX//79ccUVV1T8m2MfETVFp512GrZu3VrpZ9u2bUPr1q0BAG3btkVKSkqlsSsnJwdLly6tNP5lZWVh5cqVFbf5888/4XA4MGjQoIrbzJ8/H6WlpRW3mTVrFjp37ozY2FivvT4iIpWCggJYrZWnjW02GxwOBwCOfUTkH3w51nnkWtgg8rEvv/zSCA4ONj755BNj06ZNxk033WTExMQY6enpZneNiKhGt9xyixEdHW3MnTvXOHz4cMVXQUFBxW1uvvlmIzU11fjzzz+NFStWGEOGDDGGDBlS0V5WVmb06NHDGDNmjLFmzRpjxowZRmJiovHwww9X3GbXrl1GWFiYcf/99xubN2823n77bcNmsxkzZszw6eslItIZPny4cdddd1X8n2MfETVFy5YtMwICAox//etfxvbt243PP//cCAsLMz777LOK2zz//PNGTEyM8dNPPxnr1q0zLrjgAqNt27ZGYWFhxW3OPvtso0+fPsbSpUuNhQsXGh07djQmT55c0Z6VlWUkJycbV111lbFhwwbjyy+/NMLCwoz333/fp6+XiMgwDOOaa64xWrRoYUybNs3YvXu38f333xsJCQnGAw88UHEbjn1E1BTk5uYaq1evNlavXm0AMP79738bq1evNvbu3WsYhu/Gur/++ssICAgwXn75ZWPz5s3G448/bgQGBhrr16+v9WvhYh+Z4s033zRSU1ONoKAgY+DAgcaSJUvM7hIRUa0AUH59/PHHFbcpLCw0br31ViM2NtYICwszLrzwQuPw4cOVHmfPnj3GuHHjjNDQUCMhIcG49957jdLS0kq3mTNnjtG7d28jKCjIaNeuXaXnICIyW9XFPo59RNRU/fLLL0aPHj2M4OBgo0uXLsZ//vOfSu0Oh8N49NFHjeTkZCM4ONgYNWqUsXXr1kq3OX78uDF58mQjIiLCiIqKMq677jojNze30m3Wrl1rnH766UZwcLDRokUL4/nnn/f6ayMiUsnJyTHuuusuIzU11QgJCTHatWtn/POf/zSKi4srbsOxj4iagjlz5ijn+a655hrDMHw71n399ddGp06djKCgIKN79+7Gr7/+WqfXYjEMw6j9PkAiIiIiIiIiIiIiIiIiaihYs4+IiIiIiIiIiIiIiIiokeJiHxEREREREREREREREVEjxcU+IiIiIiIiIiIiIiIiokaKi31EREREREREREREREREjRQX+4iIiIiIiIiIiIiIiIgaKS72ERERERERERERERERETVSXOwjIiIiIiIiIiIiIiIiaqS42EdERERERERERERERETUSHGxj4iIiIiIyI9ce+21mDBhgtndcFtj7z8REREREZGnBZjdASIiIiIiIvIMi8VSbfvjjz+O119/HYZh+KhHJ82dOxcjR47EiRMnEBMT4/PnJyIiIiIiaqq42EdERERERNREHD58uOLfX331FR577DFs3bq14mcRERGIiIgwo2tERERERETkJUzjSURERERE1ESkpKRUfEVHR8NisVT6WUREhEsazBEjRuCOO+7A3XffjdjYWCQnJ+ODDz5Afn4+rrvuOkRGRqJDhw6YPn16pefasGEDxo0bh4iICCQnJ+Oqq67CsWPHat3XTz75BDExMZg5cya6du2KiIgInH322ZUWLO12O6ZMmYKYmBjEx8fjgQcecNmV6HA48Nxzz6Ft27YIDQ1Fr1698O233wIADMPA6NGjMXbs2Ir7ZWZmomXLlnjsscfq+uslIiIiIiJqkLjYR0RERERE5OemTp2KhIQELFu2DHfccQduueUWTJo0CUOHDsWqVaswZswYXHXVVSgoKAAAZGVl4cwzz0SfPn2wYsUKzJgxA0eOHMEll1xSp+ctKCjAyy+/jE8//RTz58/Hvn37cN9991W0v/LKK/jkk0/w0UcfYeHChcjMzMQPP/xQ6TGee+45/Pe//8V7772HjRs34p577sGVV16JefPmwWKxYOrUqVi+fDneeOMNAMDNN9+MFi1acLGPiIiIiIiaDKbxJCIiIiIi8nO9evXCI488AgB4+OGH8fzzzyMhIQF/+9vfAACPPfYY3n33Xaxbtw6DBw/GW2+9hT59+uDZZ5+teIyPPvoIrVq1wrZt29CpU6daPW9paSnee+89tG/fHgBw++2346mnnqpof+211/Dwww9j4sSJAID33nsPM2fOrGgvLi7Gs88+iz/++ANDhgwBALRr1w4LFy7E+++/j+HDh6NFixZ4//33cfXVVyM9PR2//fYbVq9ejYAAXg4TEREREVHTwKsbIiIiIiIiP9ezZ8+Kf9tsNsTHxyMtLa3iZ8nJyQCAjIwMAMDatWsxZ84cZf2/nTt31nqxLywsrGKhDwCaNWtW8RzZ2dk4fPgwBg0aVNEeEBCA/v37V6Tk3LFjBwoKCnDWWWdVetySkhL06dOn4v+TJk3CDz/8gOeffx7vvvsuOnbsWKv+ERERERERNQZc7CMiIiIiIvJzgYGBlf5vsVgq/cxisQCQ+ngAkJeXh/POOw8vvPCCy2M1a9asXs9btSZfdfLy8gAAv/76K1q0aFGpLTg4uOLfBQUFWLlyJWw2G7Zv317rxyciIiIiImoMuNhHREREREREddK3b1989913aNOmjdfSYUZHR6NZs2ZYunQpzjjjDABAWVkZVq5cib59+wIAunXrhuDgYOzbtw/Dhw/XPta9994Lq9WK6dOn45xzzsH48eNx5plneqXfREREREREvmY1uwNERERERETUuNx2223IzMzE5MmTsXz5cuzcuRMzZ87EddddB7vd7rHnueuuu/D888/jxx9/xJYtW3DrrbciKyuroj0yMhL33Xcf7rnnHkydOhU7d+7EqlWr8Oabb2Lq1KkAZNffRx99hM8//xxnnXUW7r//flxzzTU4ceKEx/pJRERERERkJi72ERERERERUZ00b94cf/31F+x2O8aMGYO0tDTcfffdiImJgdXqucvMe++9F1dddRWuueYaDBkyBJGRkbjwwgsr3ebpp5/Go48+iueeew5du3bF2WefjV9//RVt27bF0aNHccMNN+CJJ56o2A345JNPIjk5GTfffLPH+klERERERGQmi1GXgghERERERERERERERERE1GBwZx8RERERERERERERERFRI8XFPiIiIiIiIiIiIiIiIqJGiot9RERERERERERERERERI0UF/uIiIiIiIiIiIiIiIiIGiku9hERERERERERERERERE1UlzsIyIiIiIiIiIiIiIiImqkuNhHRERERERERERERERE1EhxsY+IiIiIiIiIiIiIiIiokeJiHxEREREREREREREREVEjxcU+IiIiIiIiIiIiIiIiokaKi31EREREREREREREREREjdT/A7ZV/wsABf1rAAAAAElFTkSuQmCC\n" + }, + "metadata": {} + }, + { + "output_type": "display_data", + "data": { + "text/plain": [ + "
" + ], + "image/png": "iVBORw0KGgoAAAANSUhEUgAABv0AAAN5CAYAAAArSffsAAAAOnRFWHRTb2Z0d2FyZQBNYXRwbG90bGliIHZlcnNpb24zLjEwLjAsIGh0dHBzOi8vbWF0cGxvdGxpYi5vcmcvlHJYcgAAAAlwSFlzAAAPYQAAD2EBqD+naQABAABJREFUeJzs3XmcjfX///HndWbMwswZxjZkGbuhQdamkiUaWaIIZV9SIXvJR1kLFaGFlEKW0p5CtpCQLNn3ncLIMmMwM5xz/f7wm/N1zAxzTuMcR4/77Ta3nPd5X9f79b7ONS+Z17zfl2GapikAAAAAAAAAAAAAPsvi7QAAAAAAAAAAAAAA/DsU/QAAAAAAAAAAAAAfR9EPAAAAAAAAAAAA8HEU/QAAAAAAAAAAAAAfR9EPAAAAAAAAAAAA8HEU/QAAAAAAAAAAAAAfR9EPAAAAAAAAAAAA8HEU/QAAAAAAAAAAAAAfR9EPAAAAAAAAAAAA8HEU/QAAAOBTOnbsqMjISG+H4TX/9fnfTtOnT5dhGDp8+LC3Q7njHT58WIZhaPr06V6LoWHDhnr22We9Nv6dZsWKFTIMQytWrPB2KLgLtG7dWi1btvR2GAAAAHARRT8AAAB4nWEYmfq6U3+YffjwYXXq1EklSpRQUFCQIiIi9PDDD2vo0KHeDi3LVa9eXYZhaPLkyd4O5Y7XsWNHp/s3JCRExYsXV4sWLfTNN9/Ibre7fe4FCxZo2LBhWRdsBubMmaMJEybc9nFctXr1ai1evFgDBw50tEVGRt4yh3Ts2NF7Qd8hUovbGzZsuK3jnDlzRm+//bYefvhh5c2bVzlz5tT999+vuXPnpts/OTlZAwcOVMGCBRUcHKwaNWpoyZIl6fZds2aNHnroIWXPnl0RERHq1auXEhMT/9U5b3Q77n273a7p06fr8ccfV+HChZUjRw7de++9ev3115WUlJSmf0b38ZgxY9L0/euvv9SyZUvlzJlTVqtVTZs21cGDB9ON45NPPlFUVJSCgoJUqlQpvffee2n6DBw4UN988422bNny7ycOAAAAjzFM0zS9HQQAAAD+22bNmuX0+rPPPtOSJUs0c+ZMp/b69esrPDxcdrtdgYGBngwxQ/v371e1atUUHByszp07KzIyUidOnNCmTZu0cOHCdH+Q+29cuXLFa/Pft2+fSpcurcjISN1zzz367bffPB7D7TR9+nR16tRJhw4dypLVlB07dtQXX3yhqVOnSpIuX76sI0eO6Mcff9TWrVtVu3Zt/fDDD7JarS6fu2fPnvrggw90u/8517hxY23fvj3N6kfTNJWcnKxs2bLJz8/vtsaQnmbNmuny5ctatGiRo+37779Pt/AjSe+//77WrVunyZMn6/nnn/dUmB61YsUK1alTR8uXL1ft2rUz7Jd6n69fv15Vq1a9bfH89NNPevLJJ9WwYUPVqVNH/v7++uabb7R8+XINGTJEw4cPd+r/9NNP6+uvv1afPn1UqlQpTZ8+XevXr9fy5cv10EMPOfpt3rxZMTExioqKUrdu3XT8+HGNHTtWderU0cKFC906Z3oyuvf/jcTERIWGhur+++9X48aNlS9fPq1du1YzZszQww8/rF9++UWGYTj6G4ah+vXrq3379k7nue+++1S+fHmn81auXFnx8fHq37+/smXLpvHjx8s0TW3evFm5c+d29J0yZYqef/55NW/eXLGxsVq1apVmzpypMWPGOBXRJalGjRoqU6aMPvvssyy7BgAAALjNTAAAAOAO06NHD9NX/le1e/fupr+/v3n48OE07506dSrLxklMTMyyc7lryJAhZr58+cxvvvnGNAzDPHTokLdDylLTpk0zJWXZvDp06GDmyJEj3fdGjx5tSjJbtmzp1rk99T3SqFEjs2jRord9HFecOnXK9Pf3N6dOnZqp/osWLTINwzAff/zx2xyZdy1fvtyUZC5fvvym/VLv8/Xr19/WeA4ePJgmL9rtdrNu3bpmYGCgU05bt26dKcl8++23HW2XL182S5QoYcbExDid47HHHjMLFChgxsfHO9o+/vhjU5K5aNEit86Znttx7ycnJ5urV69O0z58+HBTkrlkyRKndklmjx49bnneN99805Rk/vHHH462Xbt2mX5+fuagQYMcbZcuXTJz585tNmrUyOn4Nm3amDly5DDPnj3r1D527FgzR44c5oULFzI1PwAAAHgf23sCAADAp9z4TLvUZ4uNHTtWH3zwgYoXL67s2bPr0Ucf1bFjx2SapkaOHKlChQopODhYTZs21dmzZ9Ocd+HChapZs6Zy5Mih0NBQNWrUSDt27LhlPAcOHFChQoVUtGjRNO/ly5fPrXE6duyokJAQHThwQA0bNlRoaKjatGmT7vyla1vGTZgwQeXLl1dQUJDy58+v5557TufOnXPqt2HDBsXGxipPnjwKDg5WsWLF1Llz51vOMdWcOXPUokULNW7cWGFhYZozZ06aPsOGDZNhGNq/f786duyonDlzKiwsTJ06ddKlS5ec+l69elUjR45UiRIlFBgYqMjISP3vf/9TcnKyU7/IyEg1btxYK1asUNWqVRUcHKzo6GjHdq/ffvutoqOjFRQUpCpVqujPP/90On7r1q3q2LGjihcv7th+tXPnzjpz5sxN59uhQwflyZNHV65cSfPeo48+qjJlymTmsqXrlVde0aOPPqqvvvpKe/fudXrvVvdIx44d9cEHH0hy3v4vVWbvh9SxatWqpdDQUFmtVlWrVs3xudauXVvz58/XkSNHHGOk3nsZPdPvl19+ccSeM2dONW3aVLt27XLq48o9kp758+fr6tWrqlev3i37njx5Uu3atdM999yjadOmOb2X2ftPkiZNmqTy5csrMDBQBQsWVI8ePXT+/HmnPrVr19a9996rrVu3qlatWsqePbtKliypr7/+WpK0cuVK1ahRQ8HBwSpTpoyWLl2aZpy//vpLnTt3Vv78+RUYGKjy5cvr008/TdPv+PHjatasmXLkyKF8+fKpb9++6cadWak556+//lKzZs0UEhKivHnzasCAAbLZbE59T5w4od27d6f7fXG9YsWKpcmLhmGoWbNmSk5Odtp68uuvv5afn5+6devmaAsKClKXLl20du1aHTt2TJKUkJCgJUuWqG3btk4rZNu3b6+QkBB9+eWXLp8zPTe79yUpLi5OXbp0Uf78+RUUFKSKFStqxowZN70ekhQQEKAHHnggTfsTTzwhSWm+V1Jdvnz5pqvGv/76a1WrVk3VqlVztJUtW1aPPPKI0zVZvny5zpw5o+7duzsd36NHD128eFHz5893aq9fv74uXryY6S1RAQAA4H0U/QAAAHBXmD17tiZNmqQXX3xR/fv318qVK9WyZUu9+uqr+vnnnzVw4EB169ZNP/74owYMGOB07MyZM9WoUSOFhITozTff1GuvvaadO3fqoYceuuXWbkWLFtWxY8f0yy+/3DJGV8a5evWqYmNjlS9fPo0dO1bNmzfP8LzPPfecXnrpJT344IOaOHGiOnXqpNmzZys2Ntbxg/m4uDg9+uijOnz4sF555RW99957atOmjX7//fdbxi1J69at0/79+/X0008rICBATz75pGbPnp1h/5YtW+rChQsaPXq0WrZsqenTp6fZzq9r164aMmSIKleurPHjx6tWrVoaPXq0WrduneZ8+/fv1zPPPKMmTZpo9OjROnfunJo0aaLZs2erb9++atu2rYYPH64DBw6oZcuWTs/LW7JkiQ4ePKhOnTrpvffeU+vWrfXFF1+oYcOGN90es127djpz5ozTFpLStULSL7/8orZt22bq2t3s/KZpOv1APTP3yHPPPaf69es7+qd+pcrM/SBd2+axUaNGOnv2rAYNGqQxY8aoUqVK+vnnnyVJgwcPVqVKlZQnTx7HGDd7xtnSpUsVGxuruLg4DRs2TP369dOaNWv04IMPpvt9lJl7JD1r1qxR7ty50y20X89ut6tt27Y6c+aM5syZo/DwcKf3M3v/DRs2TD169FDBggU1btw4NW/eXFOmTNGjjz6apvB17tw5NW7cWDVq1NBbb72lwMBAtW7dWnPnzlXr1q3VsGFDjRkzRhcvXlSLFi104cIFx7GnTp3S/fffr6VLl6pnz56aOHGiSpYsqS5dujhd98uXL+uRRx7RokWL1LNnTw0ePFirVq3Syy+/fMtrdzM2m02xsbHKnTu3xo4dq1q1amncuHH66KOPnPoNGjRIUVFR+uuvv9wa5+TJk5KkPHnyONr+/PNPlS5dOs1Wt9WrV5d0bUtPSdq2bZuuXr2aZlvSgIAAVapUyangn9lzpudm9/7ly5dVu3ZtzZw5U23atNHbb7+tsLAwdezYURMnTsz8hbhOetck1fTp05UjRw4FBwerXLlyaX7Zwm63a+vWrelu1Vq9enUdOHDAcZ+lXp8b+1apUkUWiyXNL0yUK1dOwcHBWr16tVvzAgAAgBd4eaUhAAAAkMbNti7s0KGD05Zrhw4dMiWZefPmNc+fP+9oHzRokCnJrFixonnlyhVH+9NPP20GBASYSUlJpmma5oULF8ycOXOazz77rNM4J0+eNMPCwtK032j79u1mcHCwKcmsVKmS2bt3b/P77783L1686NTPlXE6dOhgSjJfeeWVW85/1apVpiRz9uzZTv1+/vlnp/bvvvvuX23p17NnT7Nw4cKm3W43TdM0Fy9ebEoy//zzT6d+Q4cONSWZnTt3dmp/4oknzNy5czteb9682ZRkdu3a1anfgAEDTEnmL7/84mgrWrSoKclcs2aNo23RokWmJDM4ONg8cuSIo33KlClptji8dOlSmvl8/vnnpiTz119/dbTduL2nzWYzCxUqZLZq1crp2Hfeecc0DMM8ePBgepfK4Wbbe5qmaf7555+mJLNv376mabp2j2T0PZLZ++H8+fNmaGioWaNGDfPy5ctOfVM/Y9PMeIvD1O+7adOmOdoqVapk5suXzzxz5oyjbcuWLabFYjHbt2/vaMvsPZKRhx56yKxSpcot+40YMcKUZA4fPjzNe5m9/+Li4syAgADz0UcfNW02m6Pf+++/b0oyP/30U0dbrVq1TEnmnDlzHG27d+82JZkWi8X8/fffHe2p9+/1169Lly5mgQIFzH/++ccpptatW5thYWGO+3jChAmmJPPLL7909Ll48aJZsmRJt7f3TM05I0aMcOp73333pbnWqX3d2Qb3zJkzZr58+cyaNWs6tZcvX96sW7dumv47duwwJZkffvihaZqm+dVXX6X5vk311FNPmRERES6fMyMZ3fup13/WrFmOtpSUFDMmJsYMCQkxExISbnre9NSrV8+0Wq3muXPnnNofeOABc8KECeYPP/xgTp482bz33ntNSeakSZMcfU6fPp3uZ2eapvnBBx+Ykszdu3ebpnktb/j5+aUbQ968ec3WrVunaS9durT52GOPuTwnAAAAeAcr/QAAAHBXeOqppxQWFuZ4XaNGDUlS27Zt5e/v79SekpLiWKWyZMkSnT9/Xk8//bT++ecfx5efn59q1Kih5cuX33Tc8uXLa/PmzWrbtq0OHz6siRMnqlmzZsqfP78+/vhjRz93xnnhhRduOe+vvvpKYWFhql+/vtN5q1SpopCQEMd5c+bMKUn66aefbrkt342uXr2quXPnqlWrVo5tJOvWrat8+fJluNrv+eefd3pds2ZNnTlzRgkJCZKkBQsWSJL69evn1K9///6SlGabuXLlyikmJsbxOvXzrVu3rooUKZKm/fqtA4ODgx1/TkpK0j///KP7779fkrRp06YM522xWNSmTRvNmzfPaUXW7Nmz9cADD6hYsWIZHpsZISEhkuQ497+9F6XM3w9LlizRhQsX9MorrygoKMjpHNdvFZpZJ06c0ObNm9WxY0enFXUVKlRQ/fr1HZ/39W51j2TkzJkzypUr1037rFq1SsOHD1ft2rX16quvpnk/s/ff0qVLlZKSoj59+shi+b9/Pj/77LOyWq1p7tOQkBCnlYJlypRRzpw5FRUV5bg3pbT3qWma+uabb9SkSROZpun02cXGxio+Pt5xry5YsEAFChRQixYtHOfLnj270zaW7krvM7n+e0m6tvLMNM002wzfit1uV5s2bXT+/Hm99957Tu9dvnxZgYGBaY5JvTcvX77s9N+M+qa+78o5XbVgwQJFRETo6aefdrRly5ZNvXr1UmJiolauXOnS+UaNGqWlS5dqzJgxjjydavXq1erdu7cef/xxPf/889q4caPuvfde/e9//8v0Nbm+z+XLlxUQEJBuHDdev1S5cuXSP//849KcAAAA4D0U/QAAAHBXuL7wI8lRACxcuHC67anPN9u3b5+ka8WjvHnzOn0tXrxYcXFxtxy7dOnSmjlzpv755x9t3bpVo0aNkr+/v7p16+Z4bper4/j7+6tQoUK3HHvfvn2Kj49Xvnz50pw3MTHRcd5atWqpefPmGj58uPLkyaOmTZtq2rRpmXoO2OLFi3X69GlVr15d+/fv1/79+3Xo0CHVqVNHn3/+udNWmqlu/DxSizSp1/3IkSOyWCwqWbKkU7+IiAjlzJlTR44cuen5Mvv5StLZs2fVu3dv5c+fX8HBwcqbN6+jYBcfH3/Tubdv316XL1/Wd999J0nas2ePNm7cqHbt2t30uMxITEyUJIWGhkrKmnsxs/fDgQMHJEn33nvvv56HJMfnld5zDqOiovTPP//o4sWLTu23ukduxrzJtqxnzpzR008/rVy5cmn27NlOxbrr483M/ZfRvAICAlS8ePE092mhQoXSFE3DwsJueZ+ePn1a58+f10cffZTmc+vUqZMkOT67I0eOqGTJkmnG+TfPmJSuFX3y5s3r1JYrV65MfR6Z8eKLL+rnn3/W1KlTVbFiRaf3goOD081Fqc+xSy3cp/43o77XF/gze05XHTlyRKVKlUpzX0VFRTnez6y5c+fq1VdfVZcuXTL1Sx4BAQHq2bOnzp8/r40bN0q69TW5vk9wcLBSUlLSPfeN1y+VaZpu/SIAAAAAvMP/1l0AAACAO5+fn59L7alFg9SC1cyZMxUREZGm3/WrBDMTQ3R0tKKjoxUTE6M6depo9uzZqlevnsvjBAYGplusuJHdbr/pirvUH+IbhqGvv/5av//+u3788UctWrRInTt31rhx4/T77787Vp2lJ/XcLVu2TPf9lStXqk6dOk5tt7ruqTL7w2R3P1/pWtxr1qzRSy+9pEqVKikkJER2u10NGjRIt2B5vXLlyqlKlSqaNWuW2rdvr1mzZikgICDDa+GK7du3S5Kj8JQV92Jm74c7QWbvkRvlzp07w0KUaZrq0KGD/v77b/34448qWLDgTc+V1cWMf5uH2rZtqw4dOqTbt0KFClkQYcYyijErDB8+XJMmTdKYMWPSLZgXKFAg3WcEnjhxQpIcn2OBAgWc2m/se/3nndlzesuSJUvUvn17NWrUSB9++GGmj0stIJ89e1aSFB4ersDAwAyvieR8/Ww2m+Li4pQvXz5Hv5SUFJ05cybda3Lu3DmVKlUq8xMDAACAV1H0AwAAwH9aiRIlJEn58uVTvXr1suy8VatWlfR/P3S9XeOUKFFCS5cu1YMPPpiplSv333+/7r//fr3xxhuaM2eO2rRpoy+++EJdu3ZNt//Fixf1ww8/qFWrVk7bCabq1auXZs+enabodytFixaV3W7Xvn37HCtkJOnUqVM6f/68ihYt6tL5MnLu3DktW7ZMw4cP15AhQxztqavqMqN9+/bq16+fTpw4oTlz5qhRo0a33F4yM2bOnCnDMFS/fn1Jrt0jGRWrMns/pI61ffv2NKvdMjPOjVI/rz179qR5b/fu3cqTJ49y5MiRqXPdStmyZfXNN9+k+94777yj+fPnq2/fvmrUqNFN483M/Xf9vIoXL+7ol5KSokOHDmXZ93LevHkVGhoqm812y3MWLVpU27dvT7MCK71rfyf44IMPNGzYMPXp00cDBw5Mt0+lSpW0fPlyJSQkyGq1OtrXrVvneF+6tjLV399fGzZscCq8p6SkaPPmzU5tmT1nRjK694sWLaqtW7fKbrc7/WLG7t27He/fyrp16/TEE0+oatWq+vLLL1365ZLU7VZTC/gWi0XR0dHasGFDuuMUL17csZo4dc4bNmxQw4YNHf02bNggu92e5ppcvXpVx44d0+OPP57p+AAAAOBdbO8JAACA/7TY2FhZrVaNGjUq3WfdnT59+qbHr1q1Kt3jUp8Zlrrl3r8dJyMtW7aUzWbTyJEj07x39epVnT9/XtK14teNK6hSf8B7sy0+v/vuO128eFE9evRQixYt0nw1btxY33zzTaa2Cb1e6g+cJ0yY4NT+zjvvSNJNCzauSF29dOPcbxz3Zp5++mkZhqHevXvr4MGDatu27b+Oa8yYMVq8eLFatWrlWEXjyj2SWkBL/XxTZfZ+ePTRRxUaGqrRo0c7tgBMdf21ypEjxy23QJWurSCqVKmSZsyY4RTT9u3btXjxYqcCw78VExOjc+fOpXnW3Pr16zVo0CBVqVJFY8aMuek5Mnv/1atXTwEBAXr33Xedrssnn3yi+Pj4LL1Pmzdvrm+++caxAvR613/2DRs21N9//62vv/7a0Xbp0iV99NFHWRLLrZw4cUK7d+/O1LNB586dq169eqlNmzaOa5ueFi1ayGazOc0hOTlZ06ZNU40aNRyr28LCwlSvXj3NmjXL6TmbM2fOVGJiop566imXz5mRjO79hg0b6uTJk5o7d66j7erVq3rvvfcUEhKiWrVq3fS8u3btUqNGjRQZGamffvopw+J8en8nXLhwQRMmTFCePHlUpUoVp7muX7/eqfC3Z88e/fLLL07XpG7dugoPD9fkyZOdzjt58mRlz549zf28c+dOJSUl6YEHHrjpnAAAAHDnYKUfAAAA/tOsVqsmT56sdu3aqXLlymrdurXy5s2ro0ePav78+XrwwQf1/vvvZ3j8m2++qY0bN+rJJ590bL+3adMmffbZZwoPD1efPn2yZJyM1KpVS88995xGjx6tzZs369FHH1W2bNm0b98+ffXVV5o4caJatGihGTNmaNKkSXriiSdUokQJXbhwQR9//LGsVutNCzKzZ89W7ty5M/yh7+OPP66PP/5Y8+fP15NPPpnpuCtWrKgOHTroo48+0vnz51WrVi398ccfmjFjhpo1a+byysGMWK1WPfzww3rrrbd05coV3XPPPVq8eLEOHTqU6XPkzZtXDRo00FdffaWcOXO6VOi5evWqZs2aJenaM7OOHDmiefPmaevWrapTp45TQcKVeyT1B/69evVSbGys/Pz81Lp160zfD1arVePHj1fXrl1VrVo1PfPMM8qVK5e2bNmiS5cuacaMGY5x5s6dq379+qlatWoKCQlRkyZN0p3r22+/rccee0wxMTHq0qWLLl++rPfee09hYWEaNmxYpq/ZrTRq1Ej+/v5aunSpunXrJula0atVq1a6cuWKGjdurC+//DLdY/Pnz6/69etn+v7LmzevBg0apOHDh6tBgwZ6/PHHtWfPHk2aNEnVqlXLkgJwqjFjxmj58uWqUaOGnn32WZUrV05nz57Vpk2btHTpUsd2js8++6zef/99tW/fXhs3blSBAgU0c+ZMZc+ePctiuZlBgwZpxowZOnTokCIjIzPs98cff6h9+/bKnTu3HnnkkTRbzj7wwAOO1ZM1atTQU089pUGDBikuLk4lS5bUjBkzdPjwYX3yySdOx73xxht64IEHVKtWLXXr1k3Hjx/XuHHj9Oijj6pBgwaOfq6cMz0Z3fvdunXTlClT1LFjR23cuFGRkZH6+uuvtXr1ak2YMMGxqi49Fy5cUGxsrM6dO6eXXnpJ8+fPd3q/RIkSiomJkXRtheT333+vJk2aqEiRIjpx4oQ+/fRTHT16VDNnzlRAQIDjuO7du+vjjz9Wo0aNNGDAAGXLlk3vvPOO8ufPr/79+zv6BQcHa+TIkerRo4eeeuopxcbGatWqVZo1a5beeOMNhYeHO8WzZMkSZc+e3bEaGQAAAD7ABAAAAO4wPXr0MDP6X9UOHTqYRYsWdbw+dOiQKcl8++23nfotX77clGR+9dVXTu3Tpk0zJZnr169P0z82NtYMCwszg4KCzBIlSpgdO3Y0N2zYcNNYV69ebfbo0cO89957zbCwMDNbtmxmkSJFzI4dO5oHDhxI0z8z43To0MHMkSNHpuaf6qOPPjKrVKliBgcHm6GhoWZ0dLT58ssvm3///bdpmqa5adMm8+mnnzaLFCliBgYGmvny5TMbN2580/mdOnXK9Pf3N9u1a5dhn0uXLpnZs2c3n3jiCdM0TXPo0KGmJPP06dNO/VKv+6FDhxxtV65cMYcPH24WK1bMzJYtm1m4cGFz0KBBZlJSktOxRYsWNRs1apRmbElmjx49nNrSux+OHz9uPvHEE2bOnDnNsLAw86mnnjL//vtvU5I5dOjQm8aY6ssvvzQlmd26dcvwWtyoQ4cOpiTHV/bs2c3IyEizefPm5tdff23abLZ0j8vMPXL16lXzxRdfNPPmzWsahpHm++VW90OqefPmmQ888IAZHBxsWq1Ws3r16ubnn3/ueD8xMdF85plnzJw5c5qSHPde6nWeNm2a0/mWLl1qPvjgg47zNWnSxNy5c6dTH1fukYw8/vjj5iOPPOJ4nRrPrb5q1arlOCaz959pmub7779vli1b1syWLZuZP39+84UXXjDPnTvn1KdWrVpm+fLl0xzryv176tQps0ePHmbhwoXNbNmymREREeYjjzxifvTRR079jhw5Yj7++ONm9uzZzTx58pi9e/c2f/75Z1OSuXz58pteu/RyYEY5J/Wzul7qfX2rzyl1nIy+brx3Ll++bA4YMMCMiIgwAwMDzWrVqpk///xzuudetWqV+cADD5hBQUFm3rx5zR49epgJCQlp+rlyzhtldO+b5rXPqVOnTmaePHnMgIAAMzo6Os180nOr+7RDhw6OvosXLzbr169vRkREmNmyZTNz5sxpPvroo+ayZcvSPfexY8fMFi1amFar1QwJCTEbN25s7tu3L92+H330kVmmTBkzICDALFGihDl+/HjTbren6VejRg2zbdu2t5wXAAAA7hyGad7iKekAAAAA8B/3ww8/qFmzZvr1119Vs2ZNb4fzn7dq1SrVrl1bu3fvdmyPCiDrbN68WZUrV9amTZtu+fxDAAAA3Dko+gEAAADALTRu3Fi7du3S/v37ZRiGt8OBpMcee0yFChXSxx9/7O1QgLtO69atZbfbM9wqFwAAAHcmin4AAAAAkIEvvvhCW7du1ejRozVx4kT16tXL2yEBAAAAAJAuin4AAAAAkAHDMBQSEqJWrVrpww8/lL+/v7dDAgAAAAAgXfyLFQAAAAAywO9IAgAAAAB8hcXbAQAAAAAAAAAAAAD4d1jpl0Xsdrv+/vtvhYaGyjAMb4cDAAAAAAAAAAAAL0vdQcZqtd72+hFFvyzy999/q3Dhwt4OAwAAAAAAAAAAAHeY+Ph4Wa3W2zoGRb8sEhoaKkk6fPiwcuXK5eVoAMB32Gw27dixQ+XLl5efn5+3wwEAn0H+BAD3kD8BwD3kTwBwnc1m0x9//KEGDRp4ZDyKflkkdUmm1Wq97ZVaALib2Gw2hYSEyGq18o8GAHAB+RMA3EP+BAD3kD8BwHU2m005cuTw2HgWj40EAAAAAAAAAAAA4Lag6JfFbvdDGAHgbmMYhsLDw8mfAOAi8icAuIf8CQDuIX8CgOsMw/DoI+EM0zRNj412F0tISFBYWJhHHsQIAAAAAAAAAACAO58n60c80y+L2e12b4cAAD7Fbrfr+PHjKlSokCwWFqADQGaRPwHAPeRPAHDPnZA/bTabrly54pWxASAj2bJly/BZp3a7XceOHfNYLBT9shgLJwHANaZp6uzZs7rnnnu8HQoA+BTyJwC4h/wJAO7xZv40TVMnT57U+fPnPT42AGRGzpw5FRERkWYLZNM0de7cOY/FQdEPAAAAAAAAAHDHSi345cuXT9mzZ+e5ggDuGKZp6tKlS4qLi5MkFShQwKvxUPQDAAAAAAAAANyRbDabo+CXO3dub4cDAGkEBwdLkuLi4pQvX74Mt/r0BDavz2L8lgkAuMYwjHSXvgMAbo78CQDuIX8CgHu8lT9Tn+GXPXt2j44LAK5IzVE3PnfUMAzlz5/fY3Gw0i+L8RBwAHCNxWJRRESEt8MAAJ9D/gQA95A/AcA93s6f/LIGgDtZRjnKYrF4tOhHhSqL2Ww2b4cAAD7FZrPpwIED5E8AcBH5EwDcQ/4EAPeQPwHAdTabTQcPHvTYeBT9AABed+HCBW+HAAA+ifwJAO4hfwKAe8ifAOC6xMREj41F0Q8AAAAAAAAAgDvM4cOHZRiGNm/enOljpk+frpw5c3o9jhvVrl1bffr0ybKY7gTDhg1TpUqVvB0G4ISiHwAAAAAAAAAAt8GxY8fUuXNnFSxYUAEBASpatKh69+6tM2fO3PLYwoUL68SJE7r33nszPV6rVq20d+/efxOyS1ILgjf7mj59ur799luNHDnSY3GlstlsGjNmjMqWLavg4GCFh4erRo0amjp16r8+94ABA7Rs2bIsiBLIOv7eDuBuwwNlAcA1hmGocOHC5E8AcBH5EwDcQ/4EAPeQP1138OBBxcTEqHTp0vr8889VrFgx7dixQy+99JIWLlyo33//XeHh4ekem5KSooCAAEVERLg0ZnBwsIKDg7Mi/ExJLUymGjt2rH7++WctXbrU0RYWFubRmK43fPhwTZkyRe+//76qVq2qhIQEbdiwQefOnXP7nKZpymazKSQkRCEhIVkYLe5GhmGoUKFCHhuPlX5ZzGLhkgKAKywWi3Lnzk3+BAAXkT8BwD3kTwBwz52UP01TSkryzpdpZj7OHj16KCAgQIsXL1atWrVUpEgRPfbYY1q6dKn++usvDR482NE3MjJSI0eOVPv27WW1WtWtW7d0t9WcN2+eSpUqpaCgINWpU0czZsyQYRg6f/68pLTbe6ZuQTlz5kxFRkYqLCxMrVu3dno+488//6yHHnpIOXPmVO7cudW4cWMdOHAgU3P08/NTRESE4yskJET+/v5ObcHBwWm294yMjNTrr7+u9u3bKyQkREWLFtW8efN0+vRpNW3aVCEhIapQoYI2bNjgNN5vv/2mmjVrKjg4WIULF1avXr108eLFDOObN2+eunfvrqeeekrFihVTxYoV1aVLFw0YMMDRx263a/To0SpWrJiCg4NVsWJFff311473V6xYIcMwtHDhQlWpUkWBgYH67bff0t3ec+rUqYqKilJQUJDKli2rSZMmOd5LSUlRz549VaBAAQUFBalo0aIaPXp0pq4zfJfFYsmwuH87sNIvi9lsNm+HAAA+xWazad++fSpVqpT8/Py8HQ4A+AzyJwC4h/wJAO65k/JncrL01FPeGfurr6SgoFv3O3v2rBYtWqQ33ngjzSq3iIgItWnTRnPnztWkSZMcqyfHjh2rIUOGaOjQoeme89ChQ2rRooV69+6trl276s8//3QqXmXkwIED+v777/XTTz/p3LlzatmypcaMGaM33nhDknTx4kX169dPFSpUUGJiooYMGaInnnhCmzdvvq1F3vHjx2vUqFF67bXXNH78eLVr104PPPCAOnfurLffflsDBw5U+/bttWPHDhmGoQMHDqhBgwZ6/fXX9emnn+r06dPq2bOnevbsqWnTpqU7RkREhH755Rd1795defPmTbfP6NGjNWvWLH344YcqVaqUfv31V7Vt21Z58+ZVrVq1HP1eeeUVjR07VsWLF1euXLm0YsUKp/PMnj1bQ4YM0fvvv6/77rtPf/75p5599lnlyJFDHTp00Lvvvqt58+bpyy+/VJEiRXTs2DEdO3Ysy64n7kw2m82jW+5S9AMAeF1SUpK3QwAAn0T+BAD3kD8BwD3kz8zbt2+fTNNUVFRUuu9HRUXp3LlzOn36tPLlyydJqlu3rvr37+/oc/jwYadjpkyZojJlyujtt9+WJJUpU0bbt293FO8yYrfbNX36dIWGhkqS2rVrp2XLljmOa968uVP/Tz/9VHnz5tXOnTtdep6gqxo2bKjnnntOkjRkyBBNnjxZ1apV01P/v6I7cOBAxcTE6NSpU4qIiNDo0aPVpk0bx4rBUqVK6d1331WtWrU0efJkBaVTjX3nnXfUokULRUREqHz58nrggQfUtGlTPfbYY5Kk5ORkjRo1SkuXLlVMTIwkqXjx4vrtt980ZcoUp6LfiBEjVL9+/QznM3ToUI0bN05PPvmkJKlYsWLauXOnpkyZog4dOujo0aMqVaqUHnroIRmGoaJFi/77iwif4MnceccU/caMGaNBgwapd+/emjBhgqRrF6J///764osvlJycrNjYWE2aNEn58+d3HHf06FG98MILWr58uUJCQtShQweNHj1a/v7/N7UVK1aoX79+2rFjhwoXLqxXX31VHTt2dBr/gw8+0Ntvv62TJ0+qYsWKeu+991S9enVPTB0AAAAAAAAAkEmBgddW3HlrbFeYLuwHWrVq1Zu+v2fPHlWrVs2pLTM/w46MjHQU/CSpQIECiouLc7zet2+fhgwZonXr1umff/6R3W6XdO1n77ez6FehQgXHn1N/5h8dHZ2mLS4uThEREdqyZYu2bt2q2bNnO/qYpim73a5Dhw6lW2AtV66ctm/fro0bN2r16tX69ddf1aRJE3Xs2FFTp07V/v37denSpTTFvJSUFN13331ObTf7fC5evKgDBw6oS5cuevbZZx3tV69eVVhYmCSpY8eOql+/vsqUKaMGDRqocePGevTRR295nQBX3BFFv/Xr12vKlClO3+SS1LdvX82fP19fffWVwsLC1LNnTz355JNavXq1pGvLIhs1aqSIiAitWbNGJ06cUPv27ZUtWzaNGjVK0rUlz40aNdLzzz+v2bNna9myZeratasKFCig2NhYSdLcuXPVr18/ffjhh6pRo4YmTJig2NhY7dmzx/FbFgAAAAAAAAAA7zOMzG2x6U0lS5aUYRjatWuXnnjiiTTv79q1S7ly5XLacjJHjhy3JZZs2bI5vTYMw1HYk6QmTZqoaNGi+vjjj1WwYEHZ7Xbde++9SklJuS3xpBdX6han6bWlxpqYmKjnnntOvXr1SnOuIkWKZDiOxWJRtWrVVK1aNfXp00ezZs1Su3btNHjwYCUmJkqS5s+fr3vuucfpuMAbKrw3+3xSz/Pxxx+rRo0aTu+lbodbuXJlHTp0SAsXLtTSpUvVsmVL1atXz+n5gcC/5fWiX2Jiotq0aaOPP/5Yr7/+uqM9Pj5en3zyiebMmaO6detKkqZNm6aoqCj9/vvvuv/++7V48WLt3LlTS5cuVf78+VWpUiWNHDlSAwcO1LBhwxQQEKAPP/xQxYoV07hx4yRdWzb922+/afz48Y6i3zvvvKNnn31WnTp1kiR9+OGHmj9/vj799FO98sor6cadnJys5ORkx+uEhARJ136zIPW5foZhyGKxyG63O/1GR2r7jc//y6jdYrHIMIx02yU5Jeibtfv5+Tl+8+HG9htjzKidOTEn5sScbsecIiMjnfLn3TCnu/FzYk7MiTndWXNK3Q7m+vzp63NKr505MSfmxJyyek6SnPLn3TCnu/FzYk7MiTndeXMyTdMpf3pqTjabTaZpOr4Mw0h39dztbneFYRgKDw9X/fr1NWnSJPXp00fBwcGOc588eVKzZ89Wu3btnI5LneONUttLly6thQsXOtok6Y8//nB6fav/3th25swZ7dmzRx999JFq1qwpSfrtt9+cxr2+/43x3Xi9MvpzRjGl/jm1wHfjeNe3Va5cWTt37lSJEiXS/Zwy+7mlrgi8ePGioqKiFBgYqCNHjujhhx9OM6cb55zRn/Ply6eCBQvqwIEDeuaZZ9I9jySFhoaqVatWatmypZo3b67HHntMZ86cUXh4eJbde974/rjd3093UuwZtafeKzabLc3POAsXLpz+5G4Drxf9evTooUaNGqlevXpORb+NGzfqypUrqlevnqOtbNmyKlKkiNauXav7779fa9euVXR0tNN2n7GxsXrhhRe0Y8cO3XfffVq7dq3TOVL7pO77m5KSoo0bN2rQoEGO9y0Wi+rVq6e1a9dmGPfo0aM1fPjwNO07d+5USEiIJCk8PFxFihTR8ePHdfbsWUefiIgIRURE6PDhw7pw4YKjvXDhwsqdO7f27dvntMdr8eLFZbVatXPnTqe/RMuUKaOAgABt27bNKYbo6GilpKRoz549jjY/Pz9FR0frwoULOnjwoKM9KChIZcuW1blz55weGhoaGqoSJUooLi5OJ0+edLQzJ+bEnJjT7ZjTyZMn77o53Y2fE3NiTszpzpvTkSNH7ro53Y2fE3NiTszpzprT6dOn77o53Y2fE3NiTszpzpvT/v37vTKno0ePymKxKCkpSYZhKCgoSFevXtWVK1eczhMYGKiUlBSnWLJly6Zs2bIpOTnZqagYEBAgf39/JSUlOf3wPjAwUH5+frp8+bJT7EFBQTIMI017cHCwTNNM87yu7Nmzy2636+2339YjjzyiRx99VEOHDlXZsmW1detWvfzyyypYsKBeffVVJScnO55Fd+XKFccYqb+oIl17DNbly5fVoUMHjR8/XgMHDlTbtm21efNmTZ8+XdL/FVNTr0vqeVLnd33s11+7oKAg5c6dW5MnT1auXLkUFxfn+Fl5SkqKLl++7Jif3W53Oo9hGAoODpbNZnOsCrx69apjzOs/J7vd7ogxJSVFpmk65pv6OV0/piTHZ5mcnKzLly+rV69eqlOnjnr27Klnn31WFotFu3fv1i+//KJ33nkn3c+pTZs2qlmzpmJiYpQrVy4dPnxYQ4cOValSpVS2bFkZhqHevXurX79+SkpK0oMPPqjk5GT9+uuvyp49u9q2beu0+Of6OV0/15SUFA0ePFgDBgxQ9uzZ9dhjj8lut2vt2rU6e/asevXqpXfffVeFChVS1apVlZKSoi+++EL58+dXYGCg7HZ7lt5718ec3uckXauD+NL3050+J+na99bevXsdRezUvHd9br7dDPPflln/hS+++EJvvPGG1q9fr6CgINWuXVuVKlXShAkTNGfOHHXq1Mnpg5Su7VFcp04dvfnmm+rWrZuOHDmiRYsWOd6/dOmScuTIoQULFuixxx5T6dKl1alTJ6ei3oIFC9SoUSNdunRJ586d0z333KM1a9Y4HtQpSS+//LJWrlypdevWpRt7eiv9ChcurNOnTytXrlyS+E0g5sScmBNzysycTNPUjh07VLZsWcf/0Pr6nO7Gz4k5MSfmdOfNyWazaceOHYqKinLkT1+fU3rtzIk5MSfmlNVzunLlinbu3OnIn3fDnO7Gz4k5MSfmdOfNKSUlRbt27XLkT0/N6fLlyzpy5IiKFSvmKBb40sqkI0eOaNiwYfr555919uxZRUREqGnTpho6dKhy587t6B8ZGanevXs7FqukHlusWDFt2rRJlSpVkiTNmzdPAwYM0LFjxxQTE6OWLVuqe/fuunTpkoKDgzVt2jT17dtX586dkyQNGzZMP/zwg/7880/HeSdMmKCJEyfq0KFDkqSlS5eqd+/eOnjwoMqUKaOJEyeqTp06+vbbb9WsWTMdPnxYxYsXd4ojo+uVOt7mzZud2uvUqaOKFStq4sSJMk1TxYoVc5qvYRgyDMMxpqR0x12/fr1effVVrV27VqZpqkSJEmrZsqX+97//pftZfPzxx/riiy+0fft2xcfHKyIiQnXr1tXQoUMdO0+Zpql3331XH374oQ4ePKicOXOqcuXKGjRokB5++GGtWLFCdevW1blz5xzP58tornPmzNHYsWO1c+dO5ciRQ9HR0erdu7eeeOIJffzxx5o8ebL27dsnPz8/VatWTW+99Zbj2YFZfe/dye2uuNNiz6g9KSlJhw4dUtGiRR1FwNScum7dOj300EOKj4+X1WpNf6JZxGtFv2PHjqlq1apasmSJ41l+tX2o6HejhIQEhYWF6ezZs46iHwDg1mw2m7Zt26bo6Gin32IDANwc+RMA3EP+BAD3eCt/pv4gPbXoB2dvvPGGPvzwQ4+uJAKQVka5ymazae3atapZs6ZHin6W23r2m9i4caPi4uJUuXJl+fv7y9/fXytXrtS7774rf39/5c+fXykpKTp//rzTcadOnVJERISka0vOT506leb91Pdu1sdqtSo4OFh58uSRn59fun1SzwEAAAAAAAAAgLdNmjRJ69ev18GDBzVz5ky9/fbb6tChg7fDAnCH8FrR75FHHtG2bdu0efNmx1fVqlXVpk0bx5+zZcumZcuWOY7Zs2ePjh496liRFxMTo23btikuLs7RZ8mSJbJarSpXrpyjz/XnSO2Teo6AgABVqVLFqY/dbteyZcucVv4BAAAAAAAAAOBN+/btU9OmTVWuXDmNHDlS/fv317Bhw7wdFoA7hFef6Xej67f3lKQXXnhBCxYs0PTp02W1WvXiiy9KktasWSPp2rLISpUqqWDBgnrrrbd08uRJtWvXTl27dtWoUaMkSYcOHdK9996rHj16qHPnzvrll1/Uq1cvzZ8/X7GxsZKkuXPnqkOHDpoyZYqqV6+uCRMm6Msvv9Tu3buVP3/+TMWeur3n+fPnnfb1BQDcnGlee0Bv6r78AIDMIX8CgHvInwDgHm/lT7b3BOALMspVpmkqLi5OERERHtne0/+2nv1fGj9+vCwWi5o3b67k5GTFxsZq0qRJjvf9/Pz0008/6YUXXlBMTIxy5MihDh06aMSIEY4+xYoV0/z589W3b19NnDhRhQoV0tSpUx0FP0lq1aqVTp8+rSFDhujkyZOqVKmSfv7550wX/AAA/05AQIC3QwAAn0T+BAD3kD8BwD3kTwBwnSdz5x210s+Xpa70O3v2rHLlyuXtcADAZ3jrQeAA4OvInwDgHvInALjHW/mTlX4AfEFGucpms2nt2rWqWbOmR1b6ee2ZfgAAAAAAAAAAAACyBkU/AAAAAAAAAAAAwMdR9AMAAAAAAAAAAAB8HEW/LGaxcEkBwBUWi0XR0dHkTwBwEfkTANxD/gQA95A/AcB1FotF5cuX99x4HhsJAIAMpKSkeDsEAPBJ5E8AcA/5EwDcQ/68c3Xs2FHNmjVzvK5du7b69Onj8ThWrFghwzB0/vx5j48N3Kk8mTsp+mUxu93u7RAAwKfY7Xbt2bOH/AkALiJ/AoB7yJ8A4B7yp+s6duwowzBkGIYCAgJUsmRJjRgxQlevXr3tY3/77bcaOXJkpvp6ulAXGRkpwzD0xRdfpHmvfPnyMgxD06dP90gswO1mt9u1b98+j41H0Q8AAAAAAAAAgNugQYMGOnHihPbt26f+/ftr2LBhevvtt9Ptm5WrgcLDwxUaGppl58tqhQsX1rRp05zafv/9d508eVI5cuTwUlSA76PoBwAAAAAAAADwORdTLmb4lXQ1KdN9L1+5nKm+7ggMDFRERISKFi2qF154QfXq1dO8efMk/d+WnG+88YYKFiyoMmXKSJKOHTumli1bKmfOnAoPD1fTpk11+PBhxzltNpv69eunnDlzKnfu3Hr55ZdlmqbTuDdu75mcnKyBAweqcOHCCgwMVMmSJfXJJ5/o8OHDqlOnjiQpV65cMgxDHTt2lHRthdLo0aNVrFgxBQcHq2LFivr666+dxlmwYIFKly6t4OBg1alTxynOm2nTpo1WrlypY8eOOdo+/fRTtWnTRv7+/k59z58/r65duypv3ryyWq2qW7eutmzZ4nj/wIEDatq0qfLnz6+QkBBVq1ZNS5cudTpHZGSkRo0apc6dOys0NFRFihTRRx99lKlYAV/if+suAADcXn5+ft4OAQB8EvkTANxD/gQA99xp+TNkdEiG7zUs1VDzn5nveJ1vbD5dunIp3b61itbSio4rHK8jJ0bqn0v/pOlnDjXTtLkqODhYZ86ccbxetmyZrFarlixZIkm6cuWKYmNjFRMTo1WrVsnf31+vv/66GjRooK1btyogIEDjxo3T9OnT9emnnyoqKkrjxo3Td999p7p162Y4bvv27bV27Vq9++67qlixog4dOqR//vlHhQsX1jfffKPmzZtrz549slqtCg4OliSNHj1as2bN0ocffqhSpUrp119/Vdu2bZU3b17VqlVLx44d05NPPqkePXqoW7du2rBhg/r375+p65A/f37FxsZqxowZevXVV3Xp0iXNnTtXK1eu1GeffebU96mnnlJwcLAWLlyosLAwTZkyRY888oj27t2r8PBwJSYmqmHDhnrjjTcUGBiozz77TE2aNNGePXtUpEgRx3nGjRunkSNH6n//+5++/vprvfDCC6pVq5aj2ArcLp7MnRT9stid9hcfANzp/Pz8FB0d7e0wAMDnkD8BwD3kTwBwD/nz3zFNU8uWLdOiRYv04osvOtpz5MihqVOnKiAgQJI0a9Ys2e12TZ06VYZhSJKmTZumnDlzasWKFXr00Uc1YcIEDRo0SE8++aQk6cMPP9SiRYsyHHvv3r368ssvtWTJEtWrV0+SVLx4ccf74eHhkqR8+fIpZ86ckq6tDBw1apSWLl2qmJgYxzG//fabpkyZolq1amny5MkqUaKExo0bJ0kqU6aMtm3bpjfffDNT16Rz587q37+/Bg8erK+//lolSpRQpUqVnPr89ttv+uOPPxQXF6fAwEBJ0tixY/X999/r66+/Vrdu3VSxYkVVrFjRcczIkSP13Xffad68eerZs6ejvWHDhurevbskaeDAgRo/fryWL19O0Q+3lZ+fn8qXL++x8Sj6ZbEbl1EDAG7ONE1duHBBoaGhjv+ZBQDcGvkTANxD/gQA99yJ+TNxUGKG7/lZnBdnxA2Iy7CvxXB+Ctbh3of/VVzX++mnnxQSEqIrV67IbrfrmWee0bBhwxzvR0dHOwp+krRlyxbt378/zfP4kpKSdODAAcXHx+vEiROqUaOG4z1/f39VrVo1w59Nb968WX5+fqpVq1am496/f78uXbqk+vXrO7WnpKTovvvukyTt2rXLKQ5JjgJhZjRq1EjPPfecfv31V3366afq3Llzmj5btmxRYmKicufO7dR++fJlHThwQJKUmJioYcOGaf78+Tpx4oSuXr2qy5cv6+jRo07HVKhQwfFnwzAUERGhuLiM7wsgK5imqYSEBI+NR9Evi9ntdm+HAAA+xW636+DBg4qOjma1NAC4gPwJAO4hfwKAe+7E/JkjIIfX+95KnTp1NHnyZAUEBKhgwYJpnleXI4fzWImJiapSpYpmz56d5lx58+Z1K4bU7TpdkZh4raA6f/583XPPPU7vpa64+7f8/f3Vrl07DR06VOvWrdN3332XbhwFChTQihUr0ryXuipxwIABWrJkicaOHauSJUsqODhYLVq0UEpKilP/bNmyOb02DIOf5+O2s9vtmX7WZVag6AcAAAAAAAAAwG2QI0cOlSxZMtP9K1eurLlz5ypfvnyyWq3p9ilQoIDWrVunhx9+WJJ09epVbdy4UZUrV063f3R0tOx2u1auXOnY3vN6qSsNbTabo61cuXIKDAzU0aNHM1whGBUVpXnz5jm1/f7777ee5HU6d+6ssWPHqlWrVsqVK1ea9ytXrqyTJ0/K399fkZGR6Z5j9erV6tixo5544glJ1wqFniyyAHcSy627AAAAAAAAAACA261NmzbKkyePmjZtqlWrVunQoUNasWKFevXqpePHj0uSevfurTFjxuj777/X7t271b17d50/fz7Dc0ZGRqpDhw7q3Lmzvv/+e8c5v/zyS0lS0aJFZRiGfvrpJ50+fVqJiYkKDQ3VgAED1LdvX82YMUMHDhzQpk2b9N5772nGjBmSpOeff1779u3TSy+9pD179mjOnDmaPn26S/ONiorSP//8o2nTpqX7fr169RQTE6NmzZpp8eLFOnz4sNasWaPBgwdrw4YNkqRSpUrp22+/1ebNm7VlyxY988wzrODDfxZFPwCA1wUFBXk7BADwSeRPAHAP+RMA3EP+vP2yZ8+uX3/9VUWKFNGTTz6pqKgodenSRUlJSY6Vf/3791e7du3UoUMHxcTEKDQ01LHKLSOTJ09WixYt1L17d5UtW1bPPvusLl68KEm65557NHz4cL3yyivKnz+/evbsKUkaOXKkXnvtNY0ePVpRUVFq0KCB5s+fr2LFikmSihQpom+++Ubff/+9KlasqA8//FCjRo1yec65c+fOcAtSwzC0YMECPfzww+rUqZNKly6t1q1b68iRI8qfP78k6Z133lGuXLn0wAMPqEmTJoqNjc1w1SPgDZ7MnYaZ0dM94ZKEhASFhYUpPj4+w2XXAAAAAAAAAIDMS0pK0qFDh1SsWDGKjgDuWDfLVZ6sH7HSL4uxbBgAXGO323XmzBnyJwC4iPwJAO4hfwKAe8ifAOA6u92us2fPemw8in5ZjIWTAOAa0zR17Ngx8icAuIj8CQDuIX8CgHvInwDgOtM0Hc/j9ASKfgAAAAAAAAAAAICPo+gHAAAAAAAAAAAA+DiKfgAArwsNDfV2CADgk8ifAOAe8icAuIf8CQCuCwkJ8dhY/h4b6T/Cz8/P2yEAgE/x8/NTiRIlvB0GAPgc8icAuIf8CQDuIX8CgOv8/PxUvHhxj43HSr8sZrfbvR0CAPgUu92ukydPkj8BwEXkTwBwD/kTANxD/gQA19ntdp06dcpj41H0y2KmaXo7BADwKaZp6uTJk+RPAHAR+RMA3EP+BAD3kD8BwHWmaVL0AwAAAAAAAAAAAJB5FP0AAAAAAAAAAECmrFixQoZh6Pz58x4dd/r06cqZM+e/Osfhw4dlGIY2b96cYR9vzQ/IChT9sphhGN4OAQB8imEYCg8PJ38CgIvInwDgHvInALiH/Om6jh07qlmzZt4OwyWGYdz0a9iwYd4OEfAphmEoV65cHhvP32Mj/UdYLNRRAcAVFotFRYoU8XYYAOBzyJ8A4B7yJwC4h/x590lJSVFAQIBT24kTJxx/njt3roYMGaI9e/Y42kJCQrRhw4YsGQv4L7BYLCpcuLDnxvPYSP8Rdrvd2yEAgE+x2+06evQo+RMAXET+BAD3kD8BwD3kz6y3cuVKVa9eXYGBgSpQoIBeeeUVXb16VZL0008/KWfOnLLZbJKkzZs3yzAMvfLKK47ju3btqrZt2zpe//bbb6pZs6aCg4NVuHBh9erVSxcvXnS8HxkZqZEjR6p9+/ayWq3q1q1bmpgiIiIcX2FhYTIMw6ktJCTE0Xfjxo2qWrWqsmfPrgceeMCpODhs2DBVqlRJU6dOVbFixRQUFCRJOn/+vLp27aq8efPKarWqbt262rJli+O4LVu2qE6dOgoNDZXValWVKlXSFBkXLVqkqKgohYSEqEGDBk6FSrvdrhEjRqhQoUIKDAxUpUqV9PPPP9/0c1iwYIFKly6t4OBg1alTR4cPH75pf8AVdrtdx44d89h4FP2ymGma3g4BAHyKaZo6e/Ys+RMAXET+BAD3kD8BwD13ZP6cX17a2Pf/Xm9//Vpb8tlrry/sv/Z67wf/12dtR2nJQ//3+u9F1/r8vej/2pY8dK1fqr0fXOuThf766y81bNhQ1apV05YtWzR58mR98sknev311yVJNWvW1IULF/Tnn39KulYgzJMnj1asWOE4x8qVK1W7dm1J0oEDB9SgQQM1b95cW7du1dy5c/Xbb7+pZ8+eTuOOHTtWFStW1J9//qnXXnvtX81h8ODBGjdunDZs2CB/f3917tzZ6f39+/frm2++0bfffut4ht5TTz2luLg4LVy4UBs3blTlypX1yCOP6OzZa59ZmzZtVKhQIa1fv14bN27UK6+8omzZsjnOeenSJY0dO1YzZ87Ur7/+qqNHj2rAgAGO9ydOnKhx48Zp7Nix2rp1q2JjY/X4449r37596c7h2LFjevLJJ9WkSRNt3rxZXbt2dSqsAv+WaZo6d+6cx8Zje08AAAAAAAAAADxo0qRJKly4sN5//30ZhqGyZcvq77//1sCBAzVkyBCFhYWpUqVKWrFihapWraoVK1aob9++Gj58uBITExUfH6/9+/erVq1akqTRo0erTZs26tOnjySpVKlSevfdd1WrVi1NnjzZsdKubt266t+/f5bM4Y033nCM/8orr6hRo0ZKSkpyjJWSkqLPPvtMefPmlXRtJeIff/yhuLg4BQYGSrpWhPz+++/19ddfq1u3bjp69KheeukllS1b1jGP6125ckUffvihSpQoIUnq2bOnRowY4Xh/7NixGjhwoFq3bi1JevPNN7V8+XJNmDBBH3zwgW40efJklShRQuPGjZMklSlTRtu2bdObb76ZJdcI8DSKfgAAAAAAAAAA39Noh/Pre1+99pUqtGTaPjHTnV8XjJUK3tCn/m/Or0v3uPaVhXbt2qWYmBgZhuFoe/DBB5WYmKjjx4+rSJEiqlWrllasWKH+/ftr1apVGj16tL788kv99ttvOnv2rAoWLOgoim3ZskVbt27V7NmzHeczTVN2u12HDh1SVFSUJKlq1apZNocKFSo4/lygQAFJUlxcnOPZj0WLFnUU/FJjTExMVO7cuZ3Oc/nyZR04cECS1K9fP3Xt2lUzZ85UvXr19NRTTzkKfJKUPXt2p9cFChRQXFycJCkhIUF///23HnzwQafzP/jgg05biF5v165dqlGjhlNbTExM5i4AcAei6JfFrk/SAIBbS90bnvwJAK4hfwKAe8ifAOAe8qfn1a5dW59++qm2bNmibNmyqWzZsqpdu7ZWrFihc+fOOVbZSVJiYqKee+459erVK815UotwkpQjR44si+/6bTdT74vrn/l441iJiYkqUKCA0xalqXLmzCnp2rMAn3nmGc2fP18LFy7U0KFD9cUXX+iJJ55IM2bquHfUlrPADQzDUP78+T02HkW/LGax8JhEAHCFxWJRRESEt8MAAJ9D/gQA95A/AcA95M+sFRUVpW+++UamaToKZqtXr1ZoaKgKFSok6f+e6zd+/HhHga927doaM2aMzp0757RNZ+XKlbVz506VLFnS85PJpMqVK+vkyZPy9/dXZGRkhv1Kly6t0qVLq2/fvnr66ac1bdo0R9HvZqxWqwoWLKjVq1c7FURXr16t6tWrp3tMVFSU5s2b59T2+++/Z25CQCZYLBaPFv2oUGUxm83m7RAAwKfYbDYdOHCA/AkALiJ/AoB7yJ8A4B7yp3vi4+O1efNmp69jx46pe/fuOnbsmF588UXt3r1bP/zwg4YOHap+/fo5FpbkypVLFSpU0OzZs1W7dm1J0sMPP6xNmzZp7969ToWtgQMHas2aNerZs6c2b96sffv26YcfflDPnj29Me101atXTzExMWrWrJkWL16sw4cPa82aNRo8eLA2bNigy5cvq2fPnlqxYoWOHDmi1atXa/369Y6tSTPjpZde0ptvvqm5c+dqz549euWVV7R582b17t073f7PP/+89u3bp5deekl79uzRnDlzNH369CyaMXAtdx48eNBj47HSDwDgdRcuXPB2CADgk8ifAOAe8icAuIf86boVK1bovvvuc2rr0qWLpk6dqgULFuill15SxYoVFR4eri5duujVV1916lurVi1t3rzZUfQLDw9XuXLldOrUKZUpU8bRr0KFClq5cqUGDx6smjVryjRNlShRQq1atbrtc8wswzC0YMECDR48WJ06ddLp06cVERGhhx9+WPnz55efn5/OnDmj9u3b69SpU8qTJ4+efPJJDR8+PNNj9OrVS/Hx8erfv7/i4uJUrlw5zZs3z/HswxsVKVJE33zzjfr27av33ntP1atX16hRo9S5c+esmjagxMREj41lmGx4myUSEhIUFhams2fPKleuXN4OBwB8hs1m07Zt2xQdHS0/Pz9vhwMAPoP8CQDuIX8CgHu8lT+TkpJ06NAhFStWTEFBQR4bFwBckVGustlsWrt2rWrWrKn4+HhZrdbbGgfbewIAAAAAAAAAAAA+jqJfFkt96CoAIHMMw1DhwoXJnwDgIvInALiH/AkA7iF/AoDrDMNQoUKFPDYez/TLYqkPWQUAZI7FYlHu3Lm9HQYA+BzyJwC4h/wJAO4hfwKA6ywWi8LDwz03nsdG+o+w2WzeDgEAfIrNZtPu3bvJnwDgIvInALiH/AkA7iF/AoDrbDab9u7d67HxKPoBALwuKSnJ2yEAgE8ifwKAe8ifAOAe8icAuM6TuZOiHwAAAAAAAAAAAODjKPoBAAAAAAAAAAAAPo6iXxazWLikAOAKi8Wi4sWLkz8BwEXkTwBwD/kTANxD/gQA11ksFkVGRnpsPH+PjfQfYRiGt0MAAJ9iGIasVqu3wwAAn0P+BAD3kD8BwD3kTwBwnadzJ7+WkcVsNpu3QwAAn2Kz2bRt2zbyJwC4iPwJAO4hfwKAe8ifuF7t2rXVp08fj48bGRmpCRMm/KtzdOzYUc2aNbtpH2/ND3cfm82mHTt2eGw8in4AAK/jHwwA4B7yJwC4h/wJAO4hf7omo8LR9OnTlTNnTo/HkxnDhg2TYRg3/QLgGk/mTop+AAAAAAAAAAD8x9hsNtntdqe2AQMG6MSJE46vQoUKacSIEU5t7rpy5cq/DRnALVD0AwAAAAAAAADAS1K3mxw+fLjy5s0rq9Wq559/XikpKY4+tWvXVs+ePdWzZ0+FhYUpT548eu2112SapqNPcnKyBgwYoHvuuUc5cuRQjRo1tGLFCsf7qSsM582bp3LlyikwMFBHjx51iiUkJEQRERGOLz8/P4WGhjq1pbLb7Xr55ZcVHh6uiIgIDRs2zOlchmFo8uTJevzxx5UjRw698cYbkqQffvhBlStXVlBQkIoXL67hw4fr6tWrkiTTNDVs2DAVKVJEgYGBKliwoHr16uV03kuXLqlz584KDQ1VkSJF9NFHHzm9v23bNtWtW1fBwcHKnTu3unXrpsTExAyv/8WLF9W+fXuFhISoQIECGjdu3E0+LeDORtEvi1ksXFIAcIXFYlGZMmXInwDgIvInALiH/AkA7rkT8+ek8pP0c9+fHa9/ff1XTSo/SZfPXpYknd1/VpPKT9IfH/zh6PN9x+/16UOfOl7vX7Rfk8pP0v5F+x1tnz70qb7v+L3j9R8f/KFJ5SfdxplIy5Yt065du7RixQp9/vnn+vbbbzV8+HCnPjNmzJC/v7/++OMPTZw4Ue+8846mTp3qeL9nz55au3atvvjiC23dulVPPfWUGjRooH379jn6XLp0SW+++aamTp2qHTt2KF++fG7HPGPGDOXIkUPr1q3TW2+9pREjRmjJkiVOfYYNG6YnnnhC27ZtU+fOnbVq1Sq1b99evXv31s6dOzVlyhRNnz7dURD85ptvNH78eE2ZMkX79u3T999/r+joaKdzjhs3TlWrVtWff/6p7t2764UXXtCePXskXSvgxcbGKleuXFq/fr2++uorLV26VD179sxwHi+99JJWrlypH374QYsXL9aKFSu0adMmt68LcD2LxaJSpUp5bDx/j40EAEAGAgICvB0CAPgk8icAuIf8CQDuIX/ePgEBAfr000+VPXt2lS9fXiNGjNBLL72kkSNHOgqthQsX1vjx42UYhsqUKaNt27Zp/PjxevbZZ3X06FFNmzZNR48eVcGCBSVd26rz559/1rRp0zRq1ChJ17bYnDRpkipWrPivY65QoYKGDh0qSSpVqpTef/99LVu2TPXr13f0eeaZZ9SpUyfH686dO+uVV15Rhw4dJEnFixfXyJEj9fLLL2vo0KE6evSoIiIiVK9ePWXLlk1FihRR9erVncZt2LChunfvLkkaOHCgxo8fr+XLl6tMmTKaM2eOkpKS9NlnnylHjhySpPfff19NmjTRm2++qfz58zudKzExUZ988olmzZqlRx55RNK1YmahQoX+9fUBUnkyd1L0y2I37oEMALg5u92ubdu2KTo6Wn5+ft4OBwB8BvkTANxD/gQA99yJ+bP7ju5Orx9+9WE9/OrDjtfhJcPT9Gk2vZnT65KxJVVyR0mnts6/dXZ6Xb1HdVXv4Vx4ymoVK1ZU9uzZHa9jYmKUmJioY8eOqWjRopKk+++/X4ZhOPUZN26cbDabtm3bJpvNptKlSzudNzk5Wblz53a8DggIUIUKFbIk5hvPU6BAAcXFxTm1Va1a1en1li1btHr1asfKPunaswWTkpJ06dIlPfXUU5owYYKKFy+uBg0aqGHDhmrSpIn8/f+vlHH9uIZhKCIiwjHurl27VLFiRUfBT5IefPBB2e127dmzJ03R78CBA0pJSVGNGjUcbeHh4SpTpoyrlwNIl91u144dOzw2HkU/AAAAAAAAAACymNVqVXx8fJr28+fPKywsLEvHSkxMlJ+fnzZu3JimKBsSEuL4c3BwsFPh8N/Ili2b02vDMNIsirm++JYa5/Dhw/Xkk0+mOV9QUJAKFy6sPXv2aOnSpVqyZIm6d++ut99+WytXrnSMl5lxgf8qin4AAAAAAAAAAGSxMmXKaPHixWnaN23alGZF3pYtW3T58mUFBwdLkn7//XeFhISocOHCjj7r1q1zOub3339XqVKl5Ofnp/vuu082m01xcXGqWbPmbZhN1qhcubL27NmjkiVLZtgnODhYTZo0UZMmTdSjRw+VLVtW27ZtU+XKlW95/qioKE2fPl0XL150FBxXr17teCbljUqUKKFs2bJp3bp1KlKkiCTp3Llz2rt3r2rVquXmLAHvoegHAAAAAAAAAEAWe+GFF/T++++rV69e6tq1qwIDAzV//nx9/vnn+vHHH536pqSkqEuXLnr11Vd1+PBhDR06VD179nQ8z0+Sjh49qn79+um5557Tpk2b9N5772ncuHGSpNKlS6tNmzZq3769xo0bp/vuu0+nT5/WsmXLVKFCBTVq1Mijc8/IkCFD1LhxYxUpUkQtWrSQxWLRli1btH37dr3++uuaPn26bDabatSooezZs2vWrFkKDg52bHF6K23atNHQoUPVoUMHDRs2TKdPn9aLL76odu3apdnaU7q2CrJLly566aWXlDt3buXLl0+DBw92uu6AL6Hol8VIBgDgGovFoujoaPInALiI/AkA7iF/AoB7yJ+uK168uH799VcNHjxY9erVU0pKisqWLauvvvpKDRo0cOr7yCOPqFSpUnr44YeVnJysp59+WsOGDXPq0759e12+fFnVq1eXn5+fevfurW7dujnenzZtml5//XX1799ff/31l/LkyaP7779fjRs39sR0MyU2NlY//fSTRowYoTfffFPZsmVT2bJl1bVrV0lSzpw5NWbMGPXr1082m03R0dH68ccfnZ5LeDPZs2fXokWL1Lt3b1WrVk3Zs2dX8+bN9c4772R4zNtvv63ExEQ1adJEoaGh6t+/f7rbsgLusFgsKl++vMfGM0zTND022l0sISFBYWFht2U/ZgC4m5mmqaSkJAUFBWXZnvIA8F9A/gQA95A/AcA93sqfSUlJOnTokIoVK6agoCCPjetJHTt21Pnz5/X9999n2Kd27dqqVKmSJkyY4LG4AGReRrnKNE3FxcUpIiJC8fHxslqttzUOfi0ji/HAUABwjd1u1549e8ifAOAi8icAuIf8CQDuIX8CgOvsdrv27dvnsfEo+gEAAAAAAAAAAAA+jmf6AQAAAAAAAADgJdOnT79lnxUrVtz2OAD4Plb6AQC8zs/Pz9shAIBPIn8CgHvInwDgHvInALjOk7mTlX5ZjL/4AMA1fn5+io6O9nYYAOBzyJ8A4B7yJwC4h/wJAK7z8/NT+fLlPTYeK/2ymGma3g4BAHyKaZpKSEggfwKAi8ifAOAe8icAuIf8CQCuS82dnkLRL4vZ7XZvhwAAPsVut+vgwYPkTwBwEfkTANxD/gQA95A/AcB1drtdhw8f9th4FP0AAAAAAAAAAAAAH0fRDwAAAAAAAAAAAPBxFP0AAF4XFBTk7RAAwCeRPwHAPeRPAHCPL+fPkydP6vWRI1UtKkql77lH1aKi9PrIkTp58qS3Q7trREZGasKECd4O47aqXbu2+vTp43jtiTkvW7ZMUVFRstlst3UcZKx169YaN26c28d7MndS9Mtifn5+3g4BAHyKn5+fypYtS/4EABeRPwHAPeRPAHCPL+fP8ePGqWihQvpt1Cj12b1bk//+W31279aqUaNUtFAhjX/nndsybseOHWUYhgzDULZs2ZQ/f37Vr19fn376qcvPRhw2bJgqVaqU5TF6o1B3/PhxBQQE6N577/XouLfD+vXr1a1bt9s6xssvv6xXX33V8b1ns9k0ZswYlS1bVsHBwQoPD1eNGjU0derU2xqHL5g+fbrjey71KzMFtxUrVqhy5coKDAxUyZIlNX36dKf3X331Vb3xxhuKj493OSY/Pz+VLl3a5ePcRdEvi/EgWwBwjd1u15kzZ8ifAOAi8icAuIf8CQDu8dX8OX7cOI0ePFi/2mz6OSlJbSQ9IqmNpEVJSfrVZtPo//3vthX+GjRooBMnTujw4cNauHCh6tSpo969e6tx48a6evXqbRnzTjd9+nS1bNlSCQkJWrdunbfD+Vfy5s2r7Nmz37bz//bbbzpw4ICaN2/uaBs+fLjGjx+vkSNHaufOnVq+fLm6deum8+fP37Y4UlJS3D726NGjWRjJrVmtVp04ccLxdeTIkZv2P3TokBo1aqQ6depo8+bN6tOnj7p27apFixY5+tx7770qUaKEZs2a5XI8drtdZ8+edfk4d1H0y2KmaXo7BADwKaZp6tixY+RPAHAR+RMA3EP+BAD3+GL+PHnypF4ZOFA/JierRgZ9akj6MTlZr7z88m3Z6jMwMFARERG65557VLlyZf3vf//TDz/8oIULFzqtJjp//ry6du2qvHnzymq1qm7dutqyZYuka0Wy4cOHa8uWLY7VS6nH3uy4VD/++KOqVaumoKAg5cmTR0888YSka1tVHjlyRH379nWcN9Vvv/2mmjVrKjg4WIULF1avXr108eJFx/txcXFq0qSJgoODVaxYMc2ePTtT18M0TU2bNk3t2rXTM888o08++cTp/cOHD8swDH377beqU6eOsmfProoVK2rt2rVO/b755huVL19egYGBioyMTLP1YmRkpF5//XW1b99eISEhKlq0qObNm6fTp0+radOmCgkJUYUKFbRhwwbHMWfOnNHTTz+te+65R9mzZ1d0dLQ+//zzm87nxpWSt/o8tmzZojp16ig0NFRWq1VVqlRxiuFGX3zxherXr++0Wm3evHnq3r27nnrqKRUrVkwVK1ZUly5dNGDAAEcfu92ut956SyVLllRgYKCKFCmiN954w/H+tm3bVLduXQUHByt37tzq1q2bEhMTHe937NhRzZo10xtvvKGCBQuqTJkykqRjx46pZcuWypkzp8LDw9W0aVMdPnz4pteoWLFiqlevnmbOnKlLly7dtG9WMAxDERERjq/8+fPftP+HH36oYsWKady4cYqKilLPnj3VokULjR8/3qlfkyZN9MUXX7gcj2maOn78uMvHuYuiHwAAAAAAAADgrjP1449VJ1u2DAt+qWpIqh0QoE88tD1i3bp1VbFiRX377beOtqeeekpxcXFauHChNm7cqMqVK+uRRx7R2bNn1apVK/Xv31/ly5d3rF5q1arVLY+TpPnz5+uJJ55Qw4YN9eeff2rZsmWqXr26JOnbb79VoUKFNGLECMd5JenAgQNq0KCBmjdvrq1bt2ru3Ln67bff1LNnT0e8HTt21LFjx7R8+XJ9/fXXmjRpkuLi4m459+XLl+vSpUuqV6+e2rZtqy+++MKpmJhq8ODBGjBggDZv3qzSpUvr6aefdqyM3Lhxo1q2bKnWrVtr27ZtGjZsmF577bU0WzKOHz9eDz74oP788081atRI7dq1U/v27dW2bVtt2rRJJUqUUPv27R2F7KSkJFWpUkXz58/X9u3b1a1bN7Vr105//PFHZj/aW34ebdq0UaFChbR+/Xpt3LhRr7zyirJly5bh+VatWqWqVas6tUVEROiXX37R6dOnMzxu0KBBGjNmjF577TXt3LlTc+bMcRS/Ll68qNjYWOXKlUvr16/XV199paVLlzp9vtK1Zwnu2bNHS5Ys0U8//aQrV64oNjZWoaGhWrVqlVavXq2QkBA1aNDgpisBd+7cqerVq+vVV19V/vz51blzZ61cuTLdXyCYPXu2QkJCbvq1atWqDMeSpMTERBUtWlSFCxdW06ZNtWPHjpv2X7t2rerVq+fUFhsbm6bQXL16df3xxx9KTk6+6fm8zkSWiI+PNyWZZ8+e9XYoAOBTrl69av7555/m1atXvR0KAPgU8icAuIf8CQDu8Vb+vHz5srlz507z8uXLLh9btWxZc5Zkmpn4miWZVaOisjT2Dh06mE2bNk33vVatWplR/3+8VatWmVar1UxKSnLqU6JECXPKlCmmaZrm0KFDzYoVKzq9n5njYmJizDZt2mQYY9GiRc3x48c7tXXp0sXs1q1bmrEsFot5+fJlc8+ePaYk848//nC8v2vXLlNSmnPd6JlnnjH79OnjeF2xYkVz2rRpjteHDh0yJZlTp051tO3YscOUZO7atctxjvr16zud96WXXjLLlSvnNK+2bds6Xp84ccKUZL722muOtrVr15qSzBMnTmQYb6NGjcz+/fs7XteqVcvs3bu30zipc87M5xEaGmpOnz49w/FuFBYWZn722WdObTt27DCjoqJMi8ViRkdHm88995y5YMECx/sJCQlmYGCg+fHHH6d7zo8++sjMlSuXmZiY6GibP3++abFYzJMnT5qmee3ezZ8/v5mcnOzoM3PmTLNMmTKm3W53tCUnJ5vBwcHmokWLbjkXu91u/vLLL2bHjh3N0NBQs1ixYubQoUPNgwcPOsW+b9++m35dunQpwzHWrFljzpgxw/zzzz/NFStWmI0bNzatVqt57NixDI8pVaqUOWrUKKe2+fPnm5KcxtqyZYspyTx8+HC658koV129etVctWqVKcmMj4+/6TXKCv7eKjYCAJAqNDTU2yEAgE8ifwKAe8ifAOAeX8uf8QkJishk3/yS4uPjb2c4TkzTdGynuWXLFiUmJip37txOfS5fvqwDBw5keI7MHLd582Y9++yzLsW2ZcsWbd261WnLTtM0ZbfbdejQIe3du1f+/v6qUqWK4/2yZcsqZ86cNz3v+fPn9e233+q3335ztLVt21affPKJOnbs6NS3QoUKjj8XKFBA0rUtRcuWLatdu3apadOmTv0ffPBBTZgwQTabTX5+fmnOkbrKLTo6Ok1bXFycIiIiZLPZNGrUKH355Zf666+/lJKSouTk5Ew/sy8zn0e/fv3UtWtXzZw5U/Xq1dNTTz2lEiVKZHjOy5cvO23tKUnlypXT9u3btXHjRq1evVq//vqrmjRpoo4dO2rq1KnatWuXkpOT9cgjj6R7zl27dqlixYrKkSOHo+3BBx+U3W7Xnj17nK5VQECA0/z279+fJg8kJSXd9D5NZRiG6tSpozp16mjixIl64YUXNHz4cG3evFnff/+9pGs55t/kmZiYGMXExDheP/DAA4qKitKUKVM0cuRIt88rScHBwZLk1halISEh/2psV1D0y2KpCQUAkDl+fn43/Z8bAED6yJ8A4B7yJwC4xxfzZ5jVqpN//52pvqckhYWF3d6ArrNr1y4VK1ZM0rXtCAsUKKAVK1ak6XezQlpmjkstVLgiMTFRzz33nHr16pXmvSJFimjv3r0un1OS5syZo6SkJNWo8X8brqYWE/fu3avSpUs72q/f8jK1OGq3210aL71z3Oy8b7/9tiZOnKgJEyYoOjpaOXLkUJ8+fW66deX1MvN5DBs2TM8884zmz5+vhQsXaujQofriiy8cz1m8UZ48eXTu3Lk07RaLRdWqVVO1atXUp08fzZo1S+3atdPgwYPd+szTc31RULo2vypVqqT7/Ma8efNm6pybNm3SZ599ps8//1yGYTiKoKlmz56t55577qbnWLhwoWrWrJmp8bJly6b77rtP+/fvz7BPRESETp065dR26tQpWa1Wp2uZukVrZueays/PT8WLF3fpmH+Dol8WczXxAMB/nd1uV1xcnPLlyyeLhUfNAkBmkT8BwD3kTwBwjy/mz6bPPKOZo0apTVLSLft+FhysZs8844GopF9++UXbtm1T3759JUmVK1fWyZMn5e/vr8jIyHSPCQgIkM1mc2rLzHEVKlTQsmXL1KlTJ5fOu3PnTpUsWTLdY8qWLaurV69q48aNqlatmiRpz549On/+fAYzvuaTTz5R//7906zq6969uz799FONGTPmpsenioqK0urVq53aVq9erdKlS/+rRTmrV69W06ZN1bZtW0lyFCPLlSuXqeMz83lIUunSpVW6dGn17dtXTz/9tKZNm5Zh0e++++7Tzp07bzl2aowXL15UqVKlFBwcrGXLljkV1FJFRUVp+vTpunjxoqOwt3r1alksFpUpU+am85s7d67y5csnq9V6y5hSHT9+XLNmzdLMmTN14MABNWnSRJ988okaNGggf3/nEtXjjz/uVBROzz333JPpsW02m7Zt26aGDRtm2CcmJkYLFixwaluyZInTikFJ2r59uwoVKqQ8efJkenzp2n10Y1HxdvKN7OxDzHQePgkAyJhpmjp58iT5EwBcRP4EAPeQPwHAPb6YP7s++6yWX7midbfot07SipQUdUmnQPJvJScn6+TJk/rrr7+0adMmjRo1Sk2bNlXjxo3Vvn17SVK9evUUExOjZs2aafHixTp8+LDWrFmjwYMHa8OGDZKkyMhIHTp0SJs3b9Y///yj5OTkTB03dOhQff755xo6dKh27dqlbdu26c0333TEFxkZqV9//VV//fWX/vnnH0nSwIEDtWbNGvXs2VObN2/Wvn379MMPP6hnz56SpDJlyqhBgwZ67rnntG7dOm3cuFFdu3a96QqzzZs3a9OmTeratavuvfdep6+nn35aM2bM0NWrVzN1Tfv3769ly5Zp5MiR2rt3r2bMmKH3339fAwYMcP0Duk6pUqW0ZMkSrVmzRrt27dJzzz3nUrHmVp/H5cuX1bNnT61YsUJHjhzR6tWrtX79ekVFRWV4ztjYWKftUCWpRYsWGj9+vNatW6cjR45oxYoV6tGjh0qXLq2yZcsqKChIAwcO1Msvv6zPPvtMBw4c0O+//65PPvlEktSmTRsFBQWpQ4cO2r59u5YvX64XX3xR7dq1c2ztmZ42bdooT548atq0qVatWqVDhw5pxYoV6tWrl44fP57hcUWLFtV3332nHj166MSJE/rqq6/UuHHjNAU/6dr2niVLlrzp183usxEjRmjx4sU6ePCgNm3apLZt2+rIkSNOxc9BgwY5vvck6fnnn9fBgwf18ssva/fu3Zo0aZK+/PJLR1E+1apVq/Too49mOHZGTNOk6AcAAAAAAAAAwL8RERGhMW++qSaBgRkW/tZJahIYqDFvvaWIiMw+ATDzfv75ZxUoUECRkZFq0KCBli9frnfffVc//PCDY1WaYRhasGCBHn74YXXq1EmlS5dW69atdeTIEUcRpnnz5mrQoIHq1KmjvHnzOrZHvNVxtWvX1ldffaV58+apUqVKqlu3rv744w9HfCNGjNDhw4dVokQJx7aFFSpU0MqVK7V3717VrFlT9913n4YMGaKCBQs6jps2bZoKFiyoWrVq6cknn1S3bt2UL1++DK/DJ598onLlyqls2bJp3nviiScUFxeXZrVVRipXrqwvv/xSX3zxhe69914NGTJEI0aMSLOC0FWvvvqqKleurNjYWNWuXVsRERFq1qxZpo+/1efh5+enM2fOqH379ipdurRatmypxx57TMOHD8/wnG3atNGOHTu0Z88eR1tsbKx+/PFHNWnSRKVLl1aHDh1UtmxZLV682FFIe+2119S/f38NGTJEUVFRatWqleLi4iRJ2bNn16JFi3T27FlVq1ZNLVq00COPPKL333//pvPLnj27fv31VxUpUkRPPvmkoqKi1KVLFyUlJd105d+OHTu0bt06de/eXbly5cr09XTHuXPn9OyzzyoqKkoNGzZUQkKC1qxZ47Ra88SJEzp69KjjdbFixTR//nwtWbJEFStW1Lhx4zR16lTFxsY6+iQlJen77793+fmY3mCYvvSrGXewhIQEhYWF6ezZs7f9xgWAu0nqMvvo6GieiwoALiB/AoB7yJ8A4B5v5c+kpCQdOnRIxYoVU1BQkFvnGP/OO3rl5ZdVO1s2tU9KUn5JJyV9FhSklVeuaMxbb6lvv35ZGjeQVV566SUlJCRoypQp3g7lP2vy5Mn67rvvtHjx4gz7ZJSrbDab1q5dq5o1ayo+Pt6lrVHdwUq/LJb68E8AQOYYhqHw8HDyJwC4iPwJAO4hfwKAe3w5f/bt109Hjh/Xw4MHa0JUlLoXLKiJUVGqNXiwjhw/TsEPd7TBgweraNGistvt3g7lPytbtmx677333DrWMAyPLhRjpV8WSV3p54lKLQAAAAAAAAD8F2TFSj8AuN1ulqs8WT9ipV8Wo9oOAK6x2+06evQo+RMAXET+BAD3kD8BwD3kTwBwnd1u17Fjxzw2HkW/LMbCSQBwjWmaOnv2LPkTAFxE/gQA95A/AcA93s6f5G0Ad7KMcpRpmjp37pzH4qDoBwAAAAAAAAC4I2XLlk2SdOnSJS9HAgAZS81RqTnLW/y9OjoAAAAAAAAAABnw8/NTzpw5FRcXJ0nKnj27DMPwclQAcI1pmrp06ZLi4uKUM2dO+fn5eTUein5ZjL9wAMA1hmEoIiKC/AkALiJ/AoB7yJ8A4B5v5s+IiAhJchT+AOBOkzNnTkeuup5hGMqfP7/H4jBMNkPOEgkJCQoLC1N8fLysVqu3wwEAAAAAAACAu4rNZtOVK1e8HQYAOMmWLdtNV/h5sn7ESr8sZrPZvB0CAPgUm82mw4cPKzIy0uvL3wHAl5A/AcA95E8AcM+dkD/9/PzI3QB8is1m08GDBz02nsVjIwEAkIELFy54OwQA8EnkTwBwD/kTANxD/gQA1yUmJnpsLIp+AAAAAAAAAAAAgI+j6AcAAAAAAAAAAAD4OIp+WcwwDG+HAAA+xTAMFS5cmPwJAC4ifwKAe8ifAOAe8icAuM4wDBUqVMhj4/l7bKT/CIuFOioAuMJisSh37tzeDgMAfA75EwDcQ/4EAPeQPwHAdRaLReHh4Z4bz2Mj/UfYbDZvhwAAPsVms2n37t3kTwBwEfkTANxD/gQA95A/AcB1NptNe/fu9dh4FP0AAF6XlJTk7RAAwCeRPwHAPeRPAHAP+RMAXOfJ3EnRDwAAAAAAAAAAAPBxFP0AAAAAAAAAAAAAH0fRL4tZLFxSAHCFxWJR8eLFyZ8A4CLyJwC4h/wJAO4hfwKA6ywWiyIjIz02nr/HRvqPMAzD2yEAgE8xDENWq9XbYQCAzyF/AoB7yJ8A4B7yJwC4ztO5k1/LyGI2m83bIQCAT7HZbNq2bRv5EwBcRP4EAPeQPwHAPeRPAHCdzWbTjh07PDYeRT8AgNfxDwYAcA/5EwDcQ/4EAPeQPwHAdZ7MnRT9AAAAAAAAAAAAAB9H0Q8AAAAAAAAAAADwcYZpmqa3g7gbJCQkKCwsTOfPn1dYWJi3wwEAn2GappKSkhQUFCTDMLwdDgD4DPInALiH/AkA7iF/AoDrTNNUXFycIiIiFB8fL6vVelvHY6UfAMDrAgICvB0CAPgk8icAuIf8CQDuIX8CgOs8mTsp+mUxu93u7RAAwKfY7XZt27aN/AkALiJ/AoB7yJ8A4B7yJwC4zm63a8eOHR4bj6IfAAAAAAAAAAAA4OMo+gEAAAAAAAAAAAA+jqIfAAAAAAAAAAAA4OMM0zRNbwdxN0hISFBYWJjOnz+vsLAwb4cDAD7DNE3Z7XZZLBYZhuHtcADAZ5A/AcA95E8AcA/5EwBcZ5qmzp8/r/DwcMXHx8tqtd7W8VjpBwDwupSUFG+HAAA+ifwJAO4hfwKAe8ifAOA6T+ZOin5ZzG63ezsEAPApdrtde/bsIX8CgIvInwDgHvInALiH/AkArrPb7dq3b5/HxqPoBwAAAAAAAAAAAPg4in4AAAAAAAAAAACAj6PoBwDwOj8/P2+HAAA+ifwJAO4hfwKAe8ifAOA6T+ZOwzRN02Oj3cUSEhIUFham+Ph4Wa1Wb4cDAAAAAAAAAAAAL/Nk/YiVflmMGioAuMY0TSUkJJA/AcBF5E8AcA/5EwDcQ/4EANel5k5PoeiXxex2u7dDAACfYrfbdfDgQfInALiI/AkA7iF/AoB7yJ8A4Dq73a7Dhw97bDyKfgAAAAAAAAAAAICPo+gHAAAAAAAAAAAA+DiKfgAArwsKCvJ2CADgk8ifAOAe8icAuIf8CQCu82TuNEyevJolEhISFBYWpvj4eFmtVm+HAwAAAAAAAAAAAC/zZP2IlX5ZjAfZAoBr7Ha7zpw5Q/4EABeRPwHAPeRPAHAP+RMAXGe323X27FmPjUfRL4uxcBIAXGOapo4dO0b+BAAXkT8BwD3kTwBwD/kTAFxnmqaOHz/usfEo+gEAAAAAAAAAAAA+jqIfAAAAAAAAAAAA4OMo+gEAvC40NNTbIQCATyJ/AoB7yJ8A4B7yJwC4LiQkxGNjGSabMGeJhIQEhYWFKT4+Xlar1dvhAAAAAAAAAAAAwMs8WT9ipV8Ws9vt3g4BAHyK3W7XyZMnyZ8A4CLyJwC4h/wJAO4hfwKA6+x2u06dOuWx8Sj6ZTEWTgKAa0zT1MmTJ8mfAOAi8icAuIf8CQDuIX8CgOtM06ToBwAAAAAAAAAAACDzKPoBAAAAAAAAAAAAPo6iXxYzDMPbIQCATzEMQ+Hh4eRPAHAR+RMA3EP+BAD3kD8BwHWGYShXrlyeG89kE+YskZCQoLCwMMXHx8tqtXo7HAAAAAAAAAAAAHiZJ+tHrPTLYna73dshAIBPsdvtOnr0KPkTAFxE/gQA95A/AcA95E8AcJ3dbtexY8c8Nh5FvyzGwkkAcI1pmjp79iz5EwBcRP4EAPeQPwHAPeRPAHCdaZo6d+6cx8aj6AcAAAAAAAAAAAD4OIp+AAAAAAAAAAAAgI+j6JfFDMPwdggA4FMMw1BERAT5EwBcRP4EAPeQPwHAPeRPAHCdYRjKnz+/58Yz2YQ5SyQkJCgsLEzx8fGyWq3eDgcAAAAAAAAAAABe5sn6ESv9spjNZvN2CADgU2w2mw4cOED+BAAXkT8BwD3kTwBwD/kTAFxns9l08OBBj41H0Q8A4HUXLlzwdggA4JPInwDgHvInALiH/AkArktMTPTYWBT9AAAAAAAAAAAAAB9H0Q8AAAAAAAAAAADwcRT9sphhGN4OAQB8imEYKly4MPkTAFxE/gQA95A/AcA95E8AcJ1hGCpUqJDHxvP32Ej/ERYLdVQAcIXFYlHu3Lm9HQYA+BzyJwC4h/wJAO4hfwKA6ywWi8LDwz03nsdG+o+w2WzeDgEAfIrNZtPu3bvJnwDgIvInALiH/AkA7iF/AoDrbDab9u7d67HxKPoBALwuKSnJ2yEAgE8ifwKAe8ifAOAe8icAuM6TuZOiHwAAAAAAAAAAAODjKPoBAAAAAAAAAAAAPo6iXxazWLikAOAKi8Wi4sWLkz8BwEXkTwBwD/kTANxD/gQA11ksFkVGRnpsPH+PjfQfYRiGt0MAAJ9iGIasVqu3wwAAn0P+BAD3kD8BwD3kTwBwnadzJ7+WkcVsNpu3QwAAn2Kz2bRt2zbyJwC4iPwJAO4hfwKAe8ifAOA6m82mHTt2eGw8in4AAK/jHwwA4B7yJwC4h/wJAO4hfwKA6zyZOyn6AQAAAAAAAAAAAD6Ooh8AAAAAAAAAAADg4wzTNE1vB3E3SEhIUFhYmM6fP6+wsDBvhwMAPsM0TSUlJSkoKEiGYXg7HADwGeRPAHAP+RMA3EP+BADXmaapuLg4RUREKD4+Xlar9baOx0o/AIDXBQQEeDsEAPBJ5E8AcA/5EwDcQ/4EANd5MndS9Mtidrvd2yEAgE+x2+3atm0b+RMAXET+BAD3kD8BwD3kTwBwnd1u144dOzw2HkU/AAAAAAAAAAAAwMd5teg3efJkVahQQVarVVarVTExMVq4cKHj/aSkJPXo0UO5c+dWSEiImjdvrlOnTjmd4+jRo2rUqJGyZ8+ufPny6aWXXtLVq1ed+qxYsUKVK1dWYGCgSpYsqenTp6eJ5YMPPlBkZKSCgoJUo0YN/fHHH7dlzgAAAAAAAAAAAEBW82rRr1ChQhozZow2btyoDRs2qG7dumratKljqWPfvn31448/6quvvtLKlSv1999/68knn3Qcb7PZ1KhRI6WkpGjNmjWaMWOGpk+friFDhjj6HDp0SI0aNVKdOnW0efNm9enTR127dtWiRYscfebOnat+/fpp6NCh2rRpkypWrKjY2FjFxcV57mIAAAAAAAAAAAAAbjJM0zS9HcT1wsPD9fbbb6tFixbKmzev5syZoxYtWkiSdu/eraioKK1du1b333+/Fi5cqMaNG+vvv/9W/vz5JUkffvihBg4cqNOnTysgIEADBw7U/PnztX37dscYrVu31vnz5/Xzzz9LkmrUqKFq1arp/fffl3Rtj9XChQvrxRdf1CuvvJJunMnJyUpOTna8TkhIUOHChXXmzBmFhYVJkgzDkMVikd1u1/WXObXdZrM5nTOjdovFIsMw0m1PjTcz7X5+fjJNM932G2PMqJ05MSfmxJyyek6GYejKlSuOedwNc7obPyfmxJyY0503J9M0dfXqVaf86etzSq+dOTEn5sScsnpONptNNpvNEcPdMKe78XNiTsyJOd15c7p69arsdrtjLnfDnO7Gz4k5MSfmdGfNyTRNnT17Vnnz5lV8fLysVqtuJ//benYX2Gw2ffXVV7p48aJiYmK0ceNGXblyRfXq1XP0KVu2rIoUKeIo+q1du1bR0dGOgp8kxcbG6oUXXtCOHTt03333ae3atU7nSO3Tp08fSVJKSoo2btyoQYMGOd63WCyqV6+e1q5dm2G8o0eP1vDhw9O079y5UyEhIZKuFTCLFCmi48eP6+zZs44+ERERioiI0OHDh3XhwgVHe+HChZU7d27t27dPSUlJjvbixYvLarVq586dTjdTmTJlFBAQoG3btjnFEB0drZSUFO3Zs8fR5ufnp+joaF24cEEHDx50tAcFBals2bI6d+6cjh075mgPDQ1ViRIlFBcXp5MnTzramRNzYk7MKavnFB4erj179jjF7utzuhs/J+bEnJjTnTmnnTt3ys/P766a0934OTEn5sSc7rw5/f333478ebfM6W78nJgTc2JOd96cLl265Mifd8uc7sbPiTkxJ+Z0Z80pdXdLT/D6Sr9t27YpJiZGSUlJCgkJ0Zw5c9SwYUPNmTNHnTp1clpNJ0nVq1dXnTp19Oabb6pbt246cuSI01adly5dUo4cObRgwQI99thjKl26tDp16uRU1FuwYIEaNWqkS5cu6dy5c7rnnnu0Zs0axcTEOPq8/PLLWrlypdatW5du3Bmt9Dt9+rRy5coliYo4c2JOzIk5ZWZOpmlq69atKl++vOMfDr4+p7vxc2JOzIk53Xlzstls2rZtm1P+9PU5pdfOnJgTc2JOWT2nK1euaPv27Y78eTfM6W78nJgTc2JOd96cUn9wnZo/74Y53Y2fE3NiTszpzpqTzWbT2rVr9fDDD/83VvqVKVNGmzdvVnx8vL7++mt16NBBK1eu9HZYtxQYGKjAwMA07X5+fo4fuqRK/cDT6+vpdsMw0m3PKEZX25kTc8qonTkxp4zGtNlsjnFvfN9X53S725kTc8qqGF1tZ0531pwMw0g3f/rynDJqZ07MKatidLWdOd29c0ovf/r6nFxpZ07MKaN25sScbhVLevnT1+d0u9qZE3PKqhhdbWdOd9acUv/t7ileL/oFBASoZMmSkqQqVapo/fr1mjhxolq1aqWUlBSdP39eOXPmdPQ/deqUIiIiJF1bevnHH384ne/UqVOO91L/m9p2fR+r1arg4GDHX1Lp9Uk9BwAAAAAAAAAAAHAnS7+s6UV2u13JycmqUqWKsmXLpmXLljne27Nnj44ePerYhjMmJkbbtm1TXFyco8+SJUtktVpVrlw5R5/rz5HaJ/UcAQEBqlKlilMfu92uZcuWOW33CQC4fTL67RgAwM2RPwHAPeRPAHAP+RMAXOfJ3OnVZ/oNGjRIjz32mIoUKaILFy5ozpw5evPNN7Vo0SLVr19fL7zwghYsWKDp06fLarXqxRdflCStWbNG0rUt4SpVqqSCBQvqrbfe0smTJ9WuXTt17dpVo0aNkiQdOnRI9957r3r06KHOnTvrl19+Ua9evTR//nzFxsZKkubOnasOHTpoypQpql69uiZMmKAvv/xSu3fvVv78+TM1l4SEBIWFhXlkT1YAAAAAAAAAAADc+TxZP/Lq9p5xcXFq3769Tpw4obCwMFWoUMFR8JOk8ePHy2KxqHnz5kpOTlZsbKwmTZrkON7Pz08//fSTXnjhBcXExChHjhzq0KGDRowY4ehTrFgxzZ8/X3379tXEiRNVqFAhTZ061VHwk6RWrVrp9OnTGjJkiE6ePKlKlSrp559/znTB73perKECgE8yTVMXLlxQaGioR/e3BgBfR/4EAPeQPwHAPeRPAHCdaZpKSEjw2HheXel3N0mt1J49e1a5cuXydjgA4DNsNpu2bdum6OhotgkBABeQPwHAPeRPAHAP+RMAXGez2bR27VrVrFnTIyv97rhn+gEAAAAAAAAAAABwDUU/AAAAAAAAAAAAwMdR9AMAeF1QUJC3QwAAn0T+BAD3kD8BwD3kTwBwnSdzJ8/0yyKpz/TzxJ6sAAAAAAAAAAAAuPN5sn7ESr8sZrfbvR0CAPgUu92uM2fOkD8BwEXkTwBwD/kTANxD/gQA19ntdp09e9Zj41H0y2IsnAQA15imqWPHjpE/AcBF5E8AcA/5EwDcQ/4EANeZpqnjx497bDyKfgAAAAAAAAAAAICPo+gHAAAAAAAAAAAA+DiKfgAArwsNDfV2CADgk8ifAOAe8icAuIf8CQCuCwkJ8dhYhskmzFkiISFBYWFhio+Pl9Vq9XY4AAAAAAAAAAAA8DJP1o9Y6ZfF7Ha7t0MAAJ9it9t18uRJ8icAuIj8CQDuIX8CgHvInwDgOrvdrlOnTnlsPIp+WYyFkwDgGtM0dfLkSfInALiI/AkA7iF/AoB7yJ8A4DrTNCn6AQAAAAAAAAAAAMg8in4AAAAAAAAAAACAj6Pol8UMw/B2CADgUwzDUHh4OPkTAFxE/gQA95A/AcA95E8AcJ1hGMqVK5fnxjPZhDlLJCQkKCwsTPHx8bJard4OBwAAAAAAAAAAAF7myfoRK/2ymN1u93YIAOBT7Ha7jh49Sv4EABeRPwHAPeRPAHAP+RMAXGe323Xs2DGPjUfRL4uxcBIAXGOaps6ePUv+BAAXkT8BwD3kTwBwD/kTAFxnmqbOnTvnsfEo+gEAAAAAAAAAAAA+jqIfAAAAAAAAAAAA4OMo+mUxwzC8HQIA+BTDMBQREUH+BAAXkT8BwD3kTwBwD/kTAFxnGIby58/vufFMNmHOEgkJCQoLC1N8fLysVqu3wwEAAAAAAAAAAICXebJ+xEq/LGaz2bwdAgD4FJvNpgMHDpA/AcBF5E8AcA/5EwDcQ/4EANfZbDYdPHjQY+NR9AMAeN2FCxe8HQIA+CTyJwC4h/wJAO4hfwKA6xITEz02FkU/AAAAAAAAAAAAwMdR9AMAAAAAAAAAAAB8HEW/LGYYhrdDAACfYhiGChcuTP4EABeRPwHAPeRPAHAP+RMAXGcYhgoVKuSx8fw9NtJ/hMVCHRUAXGGxWJQ7d25vhwEAPof8CQDuIX8CgHvInwDgOovFovDwcM+N57GR/iNsNpu3QwAAn2Kz2bR7927yJwC4iPwJAO4hfwKAe8ifAOA6m82mvXv3emw8in4AAK9LSkrydggA4JPInwDgHvInALiH/AkArvNk7qToBwAAAAAAAAAAAPg4in4AAAAAAAAAAACAj6Pol8UsFi4pALjCYrGoePHi5E8AcBH5EwDcQ/4EAPeQPwHAdRaLRZGRkR4bz99jI/1HGIbh7RAAwKcYhiGr1ertMADA55A/AcA95E8AcA/5EwBc5+ncya9lZDGbzebtEADAp9hsNm3bto38CQAu+n/s3Xd8U+X3B/DPTbpXujd0t6wCBcree4sMQWSJKAqKe+tXfy7coiCiKKAsQWSD7L0pq6V00L33SHfS5P7+uCQlNG2TNKNNzvv18kW584Dlae49z3MOjZ+EEKIZGj8JIUQzNH4SQoj6JBIJYmNj9XY/SvoRQggxOHpgIIQQzdD4SQghmqHxkxBCNEPjJyGEqE+fYycl/QghhBBCCCGEEEIIIYQQQghp5yjpRwghhBBCCCGEEEIIIYQQQkg7x7Asyxo6CGMgFAohEAhQVlYGgUBg6HAIIaTdYFkWtbW1sLKyAsMwhg6HEELaDRo/CSFEMzR+EkKIZmj8JIQQ9bEsi4KCAnh6eqK8vBwODg46vR+t9COEEGJwFhYWhg6BEELaJRo/CSFEMzR+EkKIZmj8JIQQ9elz7KSkn5ZJpVJDh0AIIe2KVCpFTEwMjZ+EEKImGj8JIUQzNH4SQohmaPwkhBD1SaVSxMbG6u1+lPQjhBBCCCGEEEIIIYQQQggxMqmnU1GSXGLoMIgemRk6AEIIIYQQQgghhBBCCCGEEKI9rJTFtonb4B3pDZcwF3SZ2QXB44INHRbRMUr6EUIIIYQQo3b4xcMQZgkxZ+8cQ4dCCCGEEEIIIYTohVQixYQ1EyCuEuP0/06jvqaekn4mgGFZljV0EMZAKBRCIBCgrKwMAoHA0OEQQki7wbIspFIpeDweGIYxdDiEECNTklSC1SGrMfi9wRj1+ShDh6NVNH4SQohmaPwkhBDN0PhJSPtVGFcI1zBXMDz6t6tvLMuirKwMzs7OKC8vh4ODg07vRyv9CCGEGJxIJIKVlZWhwyCEGCHnYGc8e/1Z2LjZGDoUnaDxkxBCNEPjJyGEaIbGT0LaJ7fOboYOwaSJRCK93YuntzuZCKlUaugQCCGkXZFKpUhISKDxkxCidbKCFt59vFGeUY4rP14xcETaReMnIYRohsZPQgjRDI2fhLQvJ949gZ+7/IyqgioAQGV+Jf4c+SfOf3HewJGZFqlUivv37+vtfpT0I4QQQgghRic/Jh9/9P8DeXfyAABXV13FsdeOobq42sCREaJ94moxCu8VGjoMQgghhBBCSBtiZmUGhmFg7WwNALBxsUHB3QJ6LjZyVN6TEEIIIYQYneyr2ci7kwepmJuFPPTDoRj64VBYO1kbODJCtG/D4A2oyq/Cq1mvUn8dQgghhBBCCABg+EfDMfyj4fLf88x4eCP/DXpmMHKU9COEEGJwfD7f0CEQQoxMryW9EDY1DLbutgAAz56eBo5IN2j8JADQe2lv1JXXQVovBd+cvicIUQWNn4QQohkaPwlp3yjhZxj6HDsZVtbshLSKUCiEQCBAeXk5HBwcDB0OIYQQQgh5hLReiqwrWegwsAMYHj3oEOOQdiYNGRcz0OuZXrDztDN0OIQQQgghhJA2QJgtxPW119H58c7w7uMt315VUIXordHw6euDjoM6GjBC06LP/BH19NMyyqESQoh6WJaFUCik8ZMQojV5t/Nw5v/OoPh+scL2Ux+ewsYhG+V9/to7Gj8JACTsT8DpD05DVCWi7wVCVETjJyGEaIbGT0Laj/w7+bjwxQXk3spV2F5bVotjrx3DvV33DBSZ6ZGNnfpCST8tk0qlhg6BEELaFalUipSUFBo/CSFak3kpE2c/Povy9HKF7V1mdMGwj4fBxtXGQJFpF42fBACGfzwcS64uwcWvLuLXiF8NHQ4h7QKNn4QQohkaPwlpPwJGBWBZ7DJ0mtZJYbtzsDPmH5+v0OuP6JZUKkVaWpre7kc9/QghhBBCiFHpPr87OgzqAKdAJ4Xt3n28FcqaEGIMLB0s4dPXB7f/vA0LOwuIqkSwsLUwdFiEEEIIIYQQAzKzNINbF7dG2xkeg8DRgQaIiOgLJf0IIYQQQohRsbS3hGcPzyb3SyVSsFIWfHP9NdImRBckIgmK7xfDJcQFE9dMBMNQr0pCCCGEEEIIUJxYDFsPW1gJrBrtY1kWpSmlsHayhrWztQGiI7pE5T0JIYQYnJVV4w8ghBCiqfKMctRV1Cndl3w8Gd96fIvEg4l6jko3aPw0bQV3C/BLt19wfuV5SvgRoiYaPwkhRDM0fhLSPmweuxmbhm1Sui9hXwJWB69G3J44/QZlwvQ5dtJKPy3j82nGOCGEqIPP56NTp04tH0gIISr6a/Rf4Fvwsezuskb77L3s4RLiAh6//c99o/GTWDlZYfC7g+Xlee7+fRfZ17Ix7vtxBo6MkLaNxk9CCNEMjZ+EtA8sy6L3c71hKbBUut+nnw/6rugL927ueo7MNPH5fISGhurtfgzLsqze7mbEhEIhBAIBSktL4ejoaOhwCCGk3ZBKpSgtLYWTkxN4vPb/Ep4QYnjnPjsHMyszDHxjoKFD0SkaP8mj9izYg+gt0Xiz8E3YuNgYOhxC2iwaPwkhRDM0fhJCiPqkUikyMjIQEBCA8vJyODg46PR+tNJPyyiHSggh6mFZFpmZmTRhghCiNUM/GGroEPSCxs+2LT8mH1KxFF69vPR2z5GfjcS4H8ZRwo+QFtD4SQghmqHxkxBC1MeyLLKysvR2P5qSQQghhBBCTErKiRTse2YfqourDR0KMWL3D93Hb31+Q8qJFJ3dY+vErTi0/JD894KOAkr4EUIIIYQQYuJu/HYDf436C6WppU0eE78vHmu7rkX2tWw9Rkb0gZJ+hBBCCCHEaKSfT8eW8VuQfj69yWMK7hbg9obbKIgp0GNkxNSETAxBhwEd4NbFTSfXZ6UsKrIrUF2gmLwWZguReDBRJ/ckhBBCCCGEtH0VuRXIvpYNSwflPf0AwMzSDOJqMU2GNULU009LZD39SkpK4OTkZOhwCCGk3ZBIJEhLS4O/vz/4fL6hwyGEtHMx22Ow/5n9mL17NoLHBys9pqa0BlKxFLbutnqOTrto/Gz7WJYFwzAAAHG1GOY25jq9BwD888Q/uLfrHt7IfwO2bu37e5y0T8X3i7H7qd2YuWMmnALa5rMxjZ+EEKIZGj8JaT8efU5Qdz/RHolEgpiYGEREROilpx8l/bRElvTTx/80QgghhBDSNJZlARZgePQAQwxj2+RtcO3sirHfjAUAZF7KxI7pO7D01lLYe9nr9N6pp1MhzBKiy4wuOkkyEtKS/Jh8bBq6Cb2X9sboL0cbOhxCCCGEEEIMTp/5IyrvqWVSqdTQIRBCSLsilUqRl5dH4ychRGsYhmkx4SfMEiL1VKqeItINGj/bJnG1GGVpZajMrZRvy4nKgZmlGcozyrV2n5STKbi14RZEVSKF7QEjAtBjfg9K+BGDcevihsjlkeg6u6uhQ2kSjZ+EEKIZGj8JaftEVSJcW3MN+dH5LR5bllaGo68fRcbFDD1EZrqkUiny81v+/6EtlPTTMlo4SQgh6mFZFnl5eTR+EkK0IulIElJPt5zM+++l/7Bl3BbU19brISrdoPGzbTK3Mceyu8sw9fep8m39VvTDK+mvwLefr9buc3vDbex/Zj9YqfL//1UFVairqNPa/QhRBcuySNiXgN7P9YZXhJehw2kSjZ+EEKIZGj8JafvK0srw30v/4d6uey0eK6oS4cr3V6gnuI6xLEtJP0IIIYQQQjRx9LWjOLzscIvH9Xq2Fyb9MqnJhAkhrWVmZabT64/4bATmHpoLS3vLRvuSjyfjO6/vELMtRqcxEPKoiuwK7JyxE+e/OA+WZVFVWGXokAghhBBCTIqjnyPmH5+P8KfCWzzWrYsblt5eilFfjNJDZERfdPskSgghhBBCiB6NXzUe9XUtr94LmRiih2iIqRFVinDszWPouainwqo+iUiC2J2xsPe2R8DIAK3cyynACU4BTkr3+fbzRafHO8ElxEUr9yJEVRb2Fpj25zQ4BTlhbde1sHa2xuILiw0dFiGEEEKIybCws0Dg6ECVjmUYBp49PHUcEdE3SvppGcM03z+GEEKIIoZh4OzsTOMnIUQrgsYGqXU8y7Ltdvyh8bPtST+fjhvrbsCti1ujUp575u9B1ye6aiXpV19bD3GNGNZO1kr3WzpY4oldT7T6PqR9q8yvRHFiMfyG+OntnlYCK/RY0AMAEDwhGDw+r02OszR+EkKIZmj8JKTtq6+rB9+Cr/K/U1bKIuNiBuqEdQidFKrj6EwTwzBwclI+YVMn92OpCLNWCIVCCAQClJeXw8HBwdDhEEIIIYSYHHVfLP854k8wPAYLTi7QYVTE1BQnFsPaxRo2LjYK2xMPJsIxwBHuXd1bfY+kI0nYOmErpv4xFRGLI5o9ViKWgG/Ob/U9SfvCsiy+cf0GNSU1eD33ddh52unlvvT9RgghhBBiWPuf3Y+YLTF4Pfd1WDlatXg8K2Xxve/3sLS3xPL45ZTU1xF95o+op5+WSaVSQ4dACCHtilQqRUZGBo2fhJBWSzmegi8FX+Lu33dVOt7O005vL8J1gcbPtskl1KVRwg8AQieHaiXhBwA2rjbo+XRPuHdr+nosy2LL+C3YOmGrVu5J2heGYTDtz2kY+flIvY1zLMtild8q7Jq9q9H2tobGT0II0QyNn4S0fR7dPRA0NgiWgsa9v5VheAwmr5uM6dum6zgy0yWVSpGZmam3+1HST8va4gMNIYS0ZSzLoqSkhMZPQkirmVmZwbuPN+y8VHvBPWP7DEzf2n4fbGj8bFtqSmuQcjIFlXmVTR4jEUm0ci/vPt54bMNj8Onr0+QxDMPAxsUGNi42kEroxZypYFkWNaU1ALhE85D3hujt3vU19egwsANcwhp6SUati8LqkNWoE9bpLQ5V0PhJCCGaofGTkLav30v9MGffHLVW7IVNDYN3b29a5acjLMuitLRUb/ejpB8hhBBCCDEKfkP9sODkAvgP8zd0KMQE5d7MxebRmxG3J07p/kPLDuEzy88gqhTpLabHtzyOmTtmgsenxz5Tcfm7y1jXfR2K4osAcMno+H3xenk5a25jjid2PYERn4xotK80VX8vOQghhBBCiPqkEilyb+YaOgyiBfT0RwghhBCdurH+BmK2xRg6DEIaqauow+mPTiN6a7ShQyFGwCXUBRPXTmwy6ezZ0xNdZ3dFfW19q+917edr2LtwL8TV4maPo5m6pqe6qBr+I/zlK57PfXoOO6btQOG9QoPE02tJL7x0/yV49vA0yP0JIYQQQkyJuFqMPQv2IHZnrNrnHn7xMH7r/RuKEop0EBnRJ0r6aRk9WBNCiHoYhoGnpyeNn0aMlbDY/dRupJ5KNXQoxMhd+fEKLnx5QeXjzSzNcOmbS8i5nqPDqHSHxs+2RdBBgMgXIuHWxU3p/t7P9cbMv2fCxrVxvz91ZZzLwJ3Nd8C35Ld4bNyeOPw16q82V16R6MboL0fj8b8eh5XACgAQ/lQ4Jq2bpJe+fqc+PIUzH59R2MYz48nHqGs/X8OpD09BItZOmdvWoPGTEEI0Q+MnIW2bMFuI6M3RGq3Y6z6vOwa/O1j+OZJoD8Mw8PDw0N/9WCrCrBVCoRACgQDl5eVwcHAwdDiEEEJIm1FdVI3bf97GgNcG0MMh0alfwn9BfW09Xrr/ksrn3D98H/4j/GFubQ5pvRSiKlGLDzm5t3LhGuYKcxvz1oZMiEZYloW4WgwLW4sWj72+9jqOvXEM84/NR8fBHfUQHTFVPwb8CBtXGzx7/dlG+1gpi41DNqIyvxJLby2Fpb2lASIkhBBCCDF+okoRpBIpJe/aGH3mjyjppyWy/2klJSVwcnIydDiEENJuSCQSpKWlwd/fH3x+yysWSPsRtzsO7uHucAlxkW9jWRbXf76OPs/3Ac+MCg4Q7aopqUFtWS2cAjX7LHb+i/O4vek2Xoh+AWZWZkqPkdZL8bXr13Dv6o5Zu2bB3su+NSG3Co2fbcvZT84idkcsFpxaADuPxquqyjPLceHLCwiZGILQSaF6i0tUJQLDY2BuTUlqY1ecWIyT751EryW9EDw+WGGfqEoEM0sznf7sldZLUVVY1eS4KBFJIMwWwinA8M/LNH4SQohmaPwkxPhJxBJUF1Ub9FnX2EgkEsTExCAiIkIvST9620YIIcTgKioqDB0C0bLaslrsXbgXO6btwMPzi2J3xuK/l/7Dnc13DBgdMVbWztYaJ/wA7vvWOdgZ1UXVTR4jEUkw9MOhyLyUiU1DN2l8L22h8bPtYPgMWCnb5Oq7+tp6RK2NQsb5jFbfK+1MGgrjVOvRZmFrQQk/EyHMEiLu3ziUZ5QrbL+96Ta+cvoKWVeydHp/nhmv2ZdDfAt+m0j4ydD4SQghmqHxk5C2qySpBLm3ciERaVZOvb6uHj8F/YR9i/ZpOTJSWVmpt3spn8JMCCGEENIKVo5WmHt4LsAq9rsNHh+Mof8birCpYQaMjhgjUaUIZellcPR3VKnkoTJjvh7T4jHmNuYY+PpASOokqCmpgUQkAd+CZjkTYOj7QzH0/aFN7ncKcMLrua/Dxq11Pf1YlsWW8VsQOCoQcw/NVemcqoIqxGyLgf9wf3j29GzV/UnbFTAyAB+KPwQrVSzm49bVDUFjgsDwdVdiO/taNqQSKXz6+oDHb3puccrJFJx6/xRGfTEKASMDdBYPIYQQQogpuvzDZUStjcLrua9r1NPZzNIMnR7vBDtPO7AsSy1a2ilK+hFCCCFEJ/yG+DXaZiWwwoj/G2GAaIixy7qahc2jN2PCmgnou7yvzu4jrZeCZ8bDkPeG6OwexDjxzHgaPXg3wgITfpoAOy/VryXMEuLoq0cx4PUBlPQzcsrKd/pE+qicINbU2U/OIvlYMt6rfA9oZh4EK2UhzBSiprRGp/EQQgghhJiiztM7w97LHrbuthpfY8KPE7QYETEE6umnJbKefqWlpXB0dDR0OIQQAqlE2uxM67ZCKpWitLQUTk5O4PHafrykZUlHksDwGASMCmjye7C+rh4x22LQfV538M1plRRpvaKEIkRviUbnxzvDq5eXxtdJPZ2Ki19dxITVExT6UQJAdXE1fgr8CYPfHYzB7wxubcitRuNn2xK1LgrWLtboOqtrk8eUpZWhuqga3n289RgZtzrw/uH7CBgRAHMbKvVprIrii1BVWAWfSJ8m+5LqSsrJFJSnlyNicYRe76spGj8JIUQzNH4SYjrqa+vBM+e1i3eLbZ1UKkVGRgYCAgKop197RD/wCCGGJhFLsHXCVux+arehQ1EJj8eDi4sLjZ9G5PSHp/Hvk/82e8y1Ndewf/F+xGyL0VNUxNi5hrli5KcjW5XwA7gyoamnUpF/J7/RvqqCKnj29JSv1rr601WsDl0NcbW4VffUFI2fbcvp/53G9TXXmz1mz4I92DJ+i54iasAwDEInhVLCz8hdW3MNm4ZuQk1J41V0ebfzsHXCViQdSdLJvQNHBbabhB9A4ychhGiKxk9CTEPSkSR85/2dzj47mhoejwdnZ2e93Y/Ke2qZRKJZk0xCCNEGlmXxR/8/kHszF12f6Nou6m9LJBLcv38fISEh4PNpxZcxmPrHVJSmljY7G6z3s73BMAy6zOyix8gIaVnwuGC8VfwWLO0tG+1z6+yGRWcXyX8vlUgBcKu33Lq46StEORo/25Z5R+eB4TX/M7fP831QXVTdqp/P6efSceDZAxi1chQ6T++s1rmlqaWoKamBd2/9rjQk+tFtTjc4hzjDxlVJ30gGSD2ViuCJwQgeH6z/4B4Svy8ekjoJuj7R9KpYXaPxkxBCNEPjJyFtV31dPX7p9gvCnwrH8I+Ht+parp1dIeggaNQrmmhGIpEgMTFRb/ejpB8hhBiR6qJqiKpE6PdyP4xfNd7Q4aistrbW0CEQLfLo7gGP7h7NHmPpYIkBrw0AQCUjiHZc/Poiko8lY/bu2bB0aJywUxXfgg++hWovMPqt6IcBrw7Q+F7aQONn2+EV0fIq0/C54a2+j7ReCp45T+XvUxmJSIJ13dfBu483Fp5e2Oo4SNvTcXBHdBzcUek+j3APvFv5rk5KamddycLfj/2N0V+PRs+FPVs8/tT7pyAVSw2a9ANo/CSEEE3R+ElI21RbVgszazNoo5ubo58jnr/zvBaiIjL6HDsp6UcIIUbE1s0WL8a/SDNxiMEUJxbDMcBRrZeKh144hPLM8lYna4hpE2YJkROVo5U+VjUlNYjdGQvPnp7w7e8LACiILcCZj85g4BsD5dsoUU1kpPVSiGvEsLCz0PkK+4CRAVh2d5na5/Et+Bj73VgaZ00Uw2PA5+lmRQbLsk2vMFRi4s8T9d5zkBBCCCHE2Nl52OGF6Bd0fp+sq1lI2JeAkZ+PbPPVxUwVvakghBAjxPAYZFzMwKZhm5B3J8/Q4RATwbIsNg3bhI1DNqp8jlQiBd+SDzNLM1jYWegwOmLsJvw0Ae+UvaP26idlakpqcOiFQ7i18ZZ8W+6NXMTtjkNtueLsvIK7BTj80mEU3y9u9X1J+5V7KxdfOnyJS99cava4lJMp+LXXr0g+lqynyBT1fq43us3pZpB7E937Y+Af+Hdu0z11i+8XI+rXKNQJ67R63w4DOmDxhcUInRSq0vH+w/zh289XqzEQQgghhBDtYlkWB5YewK7ZuxS2F9wtwIWVF1CSVGKgyEhLaHqdllEjW0KIIV1ZdQWOAY7o9FgnMDwGWVezkHc7D549PA0dWpN4PB4CAwNp/DQCkjoJ+izrAxsX1Wb6A9xKqcnrJkMikrTYC4sQfXEOdsac/XPgN9QPAMBKWfRY0AMhk0Ia9forzyjH9TXX4RLqApcQF73GSeNn22HpYIkeC3vAs2fzP28ZhkFtaS1ElSKN75V0NAklSSWIWBwBc2tzja9DjI+ZlRnMLJt+xE/Yl4Djbx6HW2c3+fhmKBKxBAB0Um5UFTR+EkKIZmj8JKTtSj+fjoKYAnR7shusnayVHsOygEgEWKpQ/INhGFTmVKKqsAoSsUT+uc3WzRbd53enVX5q4PF48Pf319v9GFYbRV4JhEIhBAIBysvL4eDgYOhwCCEmSCKWYKX9SvgP98e8I/PAsizqyutg5Whl6NAIUUlhXCHu/XMPvZ7tBXsve/l2iQTYsweIiACCggwYIGnT4nbHwdrZGv7D/bV2zZLkEuyeuxvdnuyG/q/0V3pMfW09yjPL4RzsTA89RC/2zN+D6C3ReK/6PbWTfpe/v4ybv9/EorOLYOtmq6MISVtVklyCovgidBzUUaufD6/+dBUA1+dUFTHbYrB73m48uf9JhE5WbXUgIYQQQghp3pFXjuDqj1fxSsYrEHQQKOxjWeDOHWD7diAuDnj2WWDKlJavKa4Ry585astqcX3tdfRb0Y8qNWlAn/kjmpahZRKJxNAhEEJMFM+MhxeiX8Dor0YD4GbktIeEn0QiQUxMDI2fRqC184hyrufgzEdnkHYmTWH7zZvAn38CGza06vLEyB1cehBnPjqj1Wvae9tDmC1EXUXTpfDMrMzgEuJikIQfjZ+mafj/DceCUws0WuUnlUjBMAzEVWLtB0baPOcgZ4ROCtX658Nra67hxm83VI8j2Bmdp3eGlZPhPqfS+EkIIZqh8ZOQtmvgGwMx//h8hUnUAHD/PvDWW8CHHwL37nEJwPXrgdu3W77mw88cUb9G4dT7pxC/N17LkRs/iUSC2NhYvd2PynsSQoiRYBgGLqGKpeVYKYtL312Cha0FIpdFGiiyltEDQ/snqhRhdchq9HulHwa/PVija4RMCsHiS4vh3dtbYXtBAfdrWVkrgyRGbdqf02Bmrd2PtubW5ngl7RXwzJqfJyeqFCHrahY8e3jCxlX18rbaQONn23D377tIPpaMUV+Mgp2nXbPHxmyLAd+Sjy4zumh0L6dAJzgFOml07qA3B2HQm4M0Ope0bRW5FYjeEo3A0YHwivBq8jiWZVFTUqNWKe6WPH3uadSW1bZ84AM+fX3wxK4ntHZ/TdH4SQghmqHxk5C2ycHXAQ6+iivIWBb49FOgtBSwsADGjweEQuDMGeCrr4AffgA8W+gIlHsrF5e/vYyh/xsKB18HdJvTDcfePAYAGPvNWB39aYyPPsdOWulHCCFGoiihCHVCxdUoDI9B1NooXF973UBREVMhzBbC3se+Vf2lbFxs0GFAB/AtFPv7yJJ91dWtCJAYvZCJIQgYEaD167aU8AOAhP0J2Dx6M5KOJmn9/qR9yLycidsbb4OVtrzi+cTbJ3Dxy4sa36tOWNfqldXE+BQnFOPEWyeQcSGj2eN2PbELP3T4AVKJVGv3tvO0g2snV61djxBCCCGEqK+6uLrRc0JWFpfws7TkVvc9+yzw0ktAWBhQWQl89hlQ28Lcrar8KsRsi0HG+Qx0f6o7eHwe0k6l4f7B+zr805DWoJV+hBBiJHbO2AlxlRgvp76ssH3uobkQdBQ0cRYh2uEa5ornop4Dy7KorwfOnwf69gVs1WwZJZVIkR+dD5cQF3mN+NJSbh8l/Uhb5T/cH2O+HQPffr6GDoUYyPhV4zHys5GwsG25t8Xjmx+HpcBS43v90PEH+ET6YP7x+WqfW5lfibjdcfDt5wuvXk2vBiPtj1dvLzxz5ZlG/Vse5TfMD1ZOVhBXiWHpoPn3oUxtWS2qi6sh6CgA35zf8gkPnPrgFBg+gxH/N6LVMRBCCCGEmDppvRTfenyLzo93xqx/Zsm3JyZyvwYHA87O3NcWFsB77wGvvAKkpwPr1nFfNyVwdCBWJK9QqDay8PRCWNhTX7+2ilb6aRmPR3+lhBDDiHgmAn1f6ttou1sXtzbdYJfH4yEsLIzGTyPBMAz++w/4/nuuD5+6bvx6A7/1+g1pZ9Pk22RJv5oarjQFIY/Ku52HlQ4rcfWnqwa5v723PQa+PhDOwc56vS+Nn20HwzCwtLcEw2u5t6P/cP9myy82h2VZdJnZBYFjAzU6vyK7AoeXHUbSEVqVamws7S3h288X9t72zR7X98W+mPLbFK0k/AAg8WAiVgevRsL+BLXOS9iXgNi/9dfX5FE0fhJCiGZo/CSkbaqvq0fEMxEIGKVY/UaW9AsNVTze2Rl44w3u6wsXgOYqT/LMeI3aC1g6WBqkr317xePxEBISorf70Uo/QggxEgNeHaB0OytlUZxYDIlYAo9wDz1HpRoLi7ablCSqufbzNYDlXibGP+jpHBOj/nUCRwdi8LuDFT5QypJ+LMuVnbC21kLAxKjwzHnw7d/yy25dk9Zz5fJUKQmqLTR+tg3Z17JhbmMO927uKh1fX1sPnjkPPD4PtzfdRlVBFQa91XKvPYZhMPX3qRrH6RLqgvkn5sM1jEoxGhtRlQg8Mx7MLPX7iO/ayRUDXh8Az54tNIN5xMLTC1u14lUbaPwkhBDN0PhJSNtjYWuBKb9OabS9qaQfAHTrxr1fqakBMjMBf3/V71cnrEPWlSw4BTrpffJre6XPsZOmZWiZVKq93giEEKIN9XX1+CX8F5x6/5ShQ1FKKpUiJiaGxs927vqa6/LekWlp3LasLK5GvDpcQl0w6otRcOvsJt8mS/oBQFVVKwMlRsm9qzvmH5uPLjO7GCyG2H9i8ZXTVwqrVHWNxs+2498n/8We+XtUOvb8yvP43PpzFMYWorqoGvue3ofYnbEq9QNsLQs7CwSOCoSDr4PO70X068xHZ/C51ecoSytr8dhDyw/h2JvHtHJf7z7eGPvtWDgHqfeyx8bVRq1yoNpG4ychhGiGxk9C2g+RCEhN5b5WlvTj8QDZ4rME9Yo2oCi+CFvGbUHsTsNVbmhPpFIpYmP193dFK/0IIcQI7F+yH+Xp5XjqyFPg8RXnc5hbm2P0V6PhEupioOiIKZh3dB7E1WKIxVyyT+b+fSAiQv3rNTSfZlBW1rC9pqY1URKiO47+jvDq7UUlTkzUkA+GqJzA8Aj3QLcnu4FvyYeNqw2WxS4D34KvUmnQwnuFuPDlBUQ8EwH/Yf4axcqyLOpr62Fuba7R+aRt8ozwRLcnu8HGzabFYzMvZOp1RbIyNaU1KLxXCLfObrB2piX8hBBCCCGtEbcnDnH/xmHEpyPgFMBVTkpN5cp2CgSAm5vy88LCgOhoLuk3bpzq93Pt7IqJP0+E31A/LURPtI2SfoQQYgTE1WLUCesaJfxkBrymvPQnIdoi6CgAAKSkAA9P+kxIUD/pF783HodfPIwZ22fAJcIP9fUN+6qrtRAsMTrJx5ORdiYNfV/sC3svw5T49In0waIziwxyb2J4EU+rPtCFTg5F4JhAiCpFALjeuwAglUjB8JhmE8clSSWI3hyNwDGa9fQDgJ+CfoKtmy2WXF2i8TVI29P9qe7o/lR3lY5dfHGx1vo9b5+yHc4hzhj3vRpvicD9rN+/eD/m7J+DsClhWomFEEIIIcRUFcQUIGZrDAa/O1i+7eHSnk09YoSFKR6rKkt7S0Qui9QgUqIPlPQjhBAjMGPbjIdWRjWNlbIqrSQgRB0SsQSVuZWwdbdFWpriRwt1PzgCXMkvK0criKvECqU9ASrvSZRLO52GCysvoOfCnoCXoaMhpGXH3jiGxP2JWHR2ERz9HZF0NAkHlhzA41seb3YFX9jUMLxX9V6rfpaHTgmlVX4mTlsJP5bl+kbzLdQv09lhYAeM+XaMQjlvQgghhBCimWH/G4ZBbw0Cz7xhMYCsZKey0p4ysn0ZGVxlJWs1CzCwLEvVbtog6umnZTwe/ZUSQgyjuR+ylfmVWB26Wmv9W7SJx+MhPDycxs92rDS5FKv8VuHsp2fl/fxkHxzj4wEV8tEKOg7uiGV3lyF4fHCjpB+t9CPKDHp7EJbHL4ejv6NB40g/n45ds3ehKKFIL/ej8bNtKEkqwc9dfsaN326odLxELEF1YTVcO7nKe+sJOgrA8BjUlLRcw9jcxhxmVprP3Zzw4wSM/nK0xueTtum/Ff/h7KdnVTpWXC1G4sFE5ETltOqeDMPgxYQXMWvXLLXPdQ1zxcDXB8I5WL1egNpC4ychhGiGxk9C2i4zKzOFCmCySdhhzRRVcHLiSn+yLNeeRR2nPzqNlfYrUV1ML2pawuPx0LVrV/3dT293IoQQohP3dt3DlVVXIKoSNXmMrZstLB0sYeVopcfIVCcSNR07afss7CzQ/9X+8BviJ0/6jRgBmJkBFRVAXp7m1364nx9APf2IclYCK7iGuWq02kSbqouqEftPLPJut+KbXk00fhqeuEYMsFBpxT0A8M35GPrBUMzeO1veV82tsxteTn0ZnR/v3Oy5xYnFyL6eDYlY0uq4iXG5+/ddpJ5MVenYuoo6bJ+yHVG/Rmnl3u11djeNn4QQohkaPwlpe3Ju5CD7erb89xUVQG4u93VwcPPnalri097bHh0GdICogsYEVehz7KSkn5ZJH25kRAghenDz95s48c4J8M2bftnN8Bg8F/Uchn04TI+RqUYqlSIhIYHGz3bMwdcB474fh+DxwfKkX0gIEBTEfa1Jic+823k4tOwQsm8oJk9opR9RpjyzHFUFhq/9Gjw+GG+XvI1us7vp5X40frYNHuEeWB63HH2W9lH5HPdu7o1KbKpSsvPiNxfxe9/f5f0ANRG/Nx57F+5FdRENqMbk9dzXMffgXJWOtfOww2MbH0O/Ff1adc/86HxEb4nWeHb35rGb8c+sf1oVg6Zo/CSEEM3Q+ElI23Tk5SP4e+rf8t/LVu15ewP2LbS9l1VqkpUDVVWfpX0w//h8g1fcaQ+kUinuq7uUshUo6UcIIe3cjO0zsOjMIoOvcCGkvBwoLeUaRPv5af7BEeBWzET9EoWCC4oZQ+rpR5TZOX0n1keuN3QYMLc2b7Mrqkn7cPL9k9g5c2eT+7s+0RWjVo6ClUDz77O8O3m489cdVORWaHwN0vbw+Dy1evX1XNQTHuEerbpn/N547Jm/B8IsYauuQwghhBgrlmXBStXsd0GIBvq/2h8jPhsh/71s8nVz/fxkZMckJqrfnoW0TZT0I4SQds7ayRq+/X1bPK4itwJHXz+K+L3xeoiKmJLoLdH4e9rfiLtcBgDw9ASsrBpKRGiS9POK8MLyuOVghwwFAMhaRtBKP6JM+Lxw9H6+t6HDAADkx+Tj/mH9zeAjhpd1NQtR66K0snKuLKUMBTEFTa7kCxoThMHvDFZYFSiWiCFlVZ9tP/D1gXi34l24d3NvdbykbairqEPa2TRU5lWqdV59XT1itseoXJr2Ud3nd8cT/z4BlxAXjc6ff2w+Zv2jfj9AQgghpL3Iv5OPr12+xu1Ntw0dCjFyXWZ0Qa9nesl/r07SLziYe+dSUgIUF6t+T5Zlce6zc7i25pqa0RJdo6QfIYS0Y2XpZShNKVXpZQ3PjIcr319B4iENai3qGJ9PqxTbs+LEYiQeTERmJvd7f3/uV1nSLyUFEIu5r0UiyI9riWsnV5SWcl97enK/UtKPKNP/5f4Y8u4QQ4cBADj78Vlsn7JdbzN6afw0vIT9CTj0wiGtJP0m/zYZy+OXq7Via0fsDrh/4473T76v0vEWdhawsLNot33YSGMFdwvw5/A/Eb0lWq3zzn16Drvn7tZ4ooJTgBM6T+8Mcxvzlg9ug2j8JIQQzdD4qTpRlQiunVzBM+Ph78f+RsbFDEOHREwAy6qX9LO0bHiPo057FoZhcPP3m7j5+021YzRF+hw7KemnZfSDjxCiT1dWXcFPQT+h5H5Ji8fautnixcQXMeXXKXqITHV8Ph/h4eE0frZjIz4ZgQ/FHyK7SgCg4cOihwfg4ADU13OJv/Jy4PXXgWXLgJgY1a5dcS8Tfnf2w9tTAoCSfqTt6/NCHzy26TG9JP1o/GwbZL0sBH6CVl/L0t6y2WTchsEbcGDpAYVtp1JPobimGBJWotI9xDVi5N7MRXlmeatiJW2HoKMA438cj4BRAWqd1+eFPhjzzRj4D/PX6L71tfUanSdTEFuAS99eQnmG/r8XafwkhBDN0Pipno6DOuKZy8/Aq7cXko8nI/Fg25uETdq/+rp6rA5djdP/Ow0AKCzk3r+YmQEBKn481LQ9y4ITC/D0+afVO8kE8fl8dO3aVW/3o6SflmlaGoUQQjQRMiEE/V/rD+dgZ5WOdwlxUSgJ1hawLAuhUEjjZzvHMAzS07nvLVnSj2EaVvtduwa8/z6Qlsb9/uJF1a5bfz8Vbpm34FrNLQ+kpB95VJ2wDn+N/gtR66IMHQoAIHB0IHrM7wGeme4/ZtP42TYIOgoQODoQ5tbaWe2UcCAB13+5rnRffW09pKKGUp4sy+Jk6kkAQIhzCF458gqOJx9v9vrFCcX4rfdvVGbKiDj4OKDfin7wivBS+7yBbwxUa2WpjEQswRd2X2Dvor1qnyuTE5WD428eR+6tXI2voSkaPwkhRDM0fmrGrbMblt1dhtErRxs6FGKExNVi7lnkwes+2Wq9gADAQsWPebJ3N7Jzs7KAkyeBmprmz3MOdoalvaX6QZsY2dipL5T00zKpVPV+GoQQ0lpBY4Mw7rtxaiXySpJL8Nfov5B3J0+HkalOKpUiJSWFxs92LOVECtIvZCLjQaWSh2eSyT447twJpKcD5g/eid9UofqDVApkevZBzPAXETjSHwAl/UhjteW1yInKQWlKqaFD0TsaP9sGcY1Yqy++rvxwBSfeOqF0tehzUc/hsY2PyX+fWpaKjPIMmPPMcT3nOn68+iM+O/9Zs9cX+Akw8vORCBwVqLWYSftWV1GHmO0qLsF/QFwtRrc53eDRw0Pj+waPC8bii4sRMEK9FYraQOMnIYRohsZP1RUlFGHP/D3IupIFAHAKdJLvo6Sp9uirrUJbZu1kjefvPI8R/zcCQEPiLiRE9Ws8nPR7+WXghReAVauAdeuaP6++rh4FdwtQVVClfuAmRCqVIk02C14PKOlHCCEmprasFunn0pF6MtXQoRAjceC5AzjwwmGIRFwteFn/PaDhgyMAODsD33zDlZjIzeX+a45QCNSb20Bk7wI3RzFsynNRU1anmz8EabcEHQR4p+wdjP6qbcyajd8bjx86/oDkY8mGDoXoyYaBG/Bz55+1dr0x34zB4kuLVTr2dCpXwqefbz98MPQDmPPMcS79HC5lXmryHGsnawx5bwg6DOyglXiJ4V385iLW9VinccnWI68cwe65u1GaqvrkCSuBFaZvmY4Brw7Q6J4AYOdphw4DO8DSgWaHE0IIMT4513MQvSVaIRlSnlGODYM34OLXKpa+Ic06/b/TWNttLaQSSkI/TJ1+fjK+voCNDSASce1ZZBV8z57lyoU2Je10Gn4J/wVxe+I0D5hoHSX9CCGknbr5+02s7bpW7ZJI3r29sSJpBQa8pvlLGkIeNva7sfCZOwwA4OfHlfWUCQsDBALA1RX44gsgKAjo3Jnb19Jqv9IH7x4dHID6rHx0Of8bzG4pL3lHSHN90PTJws4Cdh52ba6UMtGdgNEBCJmkxjTaFnj39oZHuEej76GKnApcW3MNhfcanrpPpZ0CAIz0HwlfB18s7LEQALDywkqtxUPaPlbCQlQlgrmNZiVm+zzP9SK1crTScmQtk4glNDOcEEKIUeo+rzveyH8DgWMaqivYedqhLLUMogqRASMzHhkXMlCSVGLyz17F94tx9aerKEkugUQCJCVx29VJ+jEMsHgx0K8fsHw58NdfQI8egEQC7NvX9Hnu4e4Y+r+h8OqlXpl5oltmhg6AEEKIZqT1UogqRbDztFP7XEFHAQCupISoUmTw+ttWVvp/yUS0p/PjnXFjC/e1rJ+fjLU1sH49N0tMVku+Vy8gJga4cQOYNKnp68qSfk5OgFNHO2SHDkedE1cCrLa8FhKRBLZuttr9w5B2pyytDHl38tBhQAfYuhv++yFwdCACR+uvbCKNn4Y39puxWr+mqEqEmpIaCDoI5NvyY/Lx30v/YfKvk+HWxQ0sy+JU6oOkX8BIAMBbg97ChtsbcDDxIO4V3kMXty5Kr//PrH/Asiye2PWE1mMn+jf4ncEY/M5gjc/3ifSBT6SPWuec+uAUxDVijPtunMb3BYCfO/8MS3tLLL21tFXX0QSNn4QQohkaP1X36PMJ34KPVzJeAY9P63C0YdLaSajIqWgzE0ANJftaNo68fASCjgII+c6oq+NW7fn6qnedceO4/2RmzADu3AGOHgVmzwbs7Ruf4+DjIC8rSpqnz7GTRhgt48vWvhJCiI71eb4PXkl/BfZeSn7qqkBUJcL6yPU4sOSAliNTD5/PR6dOnWj8bOdkpckfTfoBXOLv4ebRvXtzv0ZHA2Jx09d8OOnnEeaI3NBhKLH2QWF8Eb7z/A6Xvmm6fB0xHcnHkrFj2g7k3W4bfUr1icZP4ySVSPGd53eNfj779PXBglMLEDKRW1VYJ6nD4p6LMdRvKPr79gcAhLiEYHLoZADAjrs7mrxHXUUdzTAnjajTEydhXwJSjqe0+p7hc8MROlWNaehaQuMnIYRohsZP1VTkVCB2Zyyqixs3paeEn/a4dnIFz4yHo68fNenefkFjg7Do7CJ0HNwRCQnctpAQxSpMmujZEwgMBGprgcOHWx2mSePz+QhVZ+llK9Eoo2XUyJYQ0l5Y2FrAKcAJ9j72Bm0iLZVKUVxcTONnO5UfnY9vPb9FyVGu7KafX8vn+Ptziby6OuDevaaPezjpZ2PTsN3axwVhj4XBO9Jb88CJ0QgYGYBpf02DR3cPQ4cCgGtkfunbS4jfF6/ze9H4aXjVxdXYu2gv4vdq7/83j89D5PJIhExWLBlq7WSNgBEBcPB1AABYmVnh81Gf4+yis7A0a1ixP6PzDADAnvg9Td5j3pF5mHd0ntZiJoZ1a8Mt3NvVzA9UFWwcshEbh25U+fjno5/HwtMLW3VPABjxyQiDzA6n8ZMQQjRD42fLKiuBf1cmYtfsXUg/m670mGtrruHIq0f0HJnxEVWJkHgwEVe+v4L86HxDh2Mwtm628BvqBxtXG436+TWFYbjVfgBw4ADX70+Z3U/txq7Zu1p/QyMmlUpRUlKit/tR0k/LDPninBBiOsrSynDqw1PIuZHTquvM3DkT474fZ9BSCCzLIjMzk8bPdoplWTgHOUMotgbA9e5rCcNwJT6B5vv6lZVxvzo5AebmgNmDouS1dQxm/j0TXWd11TxwYjScg53RY34PjUod6wLDY3D8zeO4u+2uzu9F46fhVeZW4s6fd5B3R7srTUd/ORr9XuqnsK2+rl6lGcyTQyfDT+CHoX5DIZY0s5yaGI1TH5zC5e8ut+oaTkFOcAp0Uvl4hmFg42LT8oFtFI2fhBCiGRo/W3bwILA/LgS5faei3Mlf6THJR5Nxe+NtiKvps5qmRJUirLRbibzbeXgx4UV49vQ0dEgGU19bD6mES8RrM+kHAIMGAe7uQHk5cOKE8mMq8ytRmV+pnRsaKZZlkZWVpbf7UdKPEC2RSqRIOZGCgrsFOPvJWZRnlhs6JGLECu8V4vxn55F/p3UzmWTJPmm9FLVltdoIjZgYzx6emHN8MfLdugEAnJ1VO0+VpJ9spZ+jI5cotObyiqh+qEIKPSSRtoZvzsczV57BmG/HGDoUogduXd3wjvAdDHhtgM7vdeyNY/jU/FNU5lVCykpxKPEQhHXCRsc5Wzsj9eVUrJm4BuZ8c6XXyr2Vi+trr9PPfiPx5P4nMWH1hFZdY9qmaXj8r8dVOjbvTh5STqagvra+VfcEgJyoHOyYvgOpp1NbfS1CCCGkLcjKAsTWDsh2j8AXP1hj2zbg0YWRk36ZhNdzX4e5jfLPaqRlErEE3ed3R9cnusIl1MXQ4RjUoRcO4VPzT1FeWIf0B4tLtZX04/OBxx98RNy/X/kxC04swKIzi7RzQ6IVlPQjREvOfXoOm8dsxpmPzuDMR2eQdVl/2XtievyH+2NZ7DKEPRbW6mvVlNbg5y4/47+X/tNCZMQUySoU2NoCqvYljojgEnlpaUBxsfJjHl7pJ7s+0JD0O/3RaXzn9R29tDZxexftxSr/VY1WQEVHA5mZhonJt58vBB0Ehrk50SuGYWBpbwlLe8uWD1aDMFuILeO24OpPV+XbPLp7IOyxMFg5WSE6PxqTt0+G3yo/SKQSpXE15/6h+zi8/DDK0sq0GjcxDO8+3vDuo7+S19d/vo7NozejprSm1dcS14iRsC8BJff1V+6IEEII0aWCLBF49XUICgJYFti+Hfj+e8VjHHwdYG5NCb/WsHayxuN/PY5eS3qhvrYeSUeSUFVQZeiwDMI70htdZnRBZr4FWBZwcVF9QrYqRo7k3t9kZ3Mr/kjbR0k/QrSk93O9Ebk8EhNWT8Cz159F2NTWJ2MIaYq5jTncurhppayStZM1PHt6wiXMxWAlOuzt7Q1yX9J6SUeTcPnr8+CLatT6UGlv3zDzrKnVfg/39AMa+vpVPfgc7xTgBJ9+PqgqNM0P9oRj52kHpwAnMLyGJEdcHPD++8Dy5cC6dQ3fM/oiEUn09sBJ46dhCbOFyL2Zq/VVx9bO1si8lAlhVsNKvt7P9sbs3bNhZmmG06mnAQADOwwEn8dXeg2JVIJz6edQWFXYaF+3J7thwckFcApSvZwjaZukEqlWVtxV5lXi+NvHkXgoscVjey/tjYk/T9RKWeUOAzrg/dr30fu53q2+lrpo/CSEEM3Q+Nm8qjtJiDj6FaaGJeC117hkydmzQOEjH8nKM8px6btLEFU10SiNqCzhQAK2TtiKuN1xhg7FICKXRWLWP7Nw/z73TKxsld+x5GPYcXcH4ovilU4abI6NDeDry319/37j/UUJRbix/gaV+GyBnZ3+WpJQ0k/L+HzlD93E+Nl722Pimomw97aHdx9vmFmZGTokYsSqCqtQVViltSTdrJ2zMPSDoQbp7cfn8xEUFETjZzuVsD8B91afAsNK4KJmRQ1Zic+TJ7kZkI9qKulX82BhQc9FPTH/2Hy4hJh2KQ9TN/rL0Vh4eqHCtssPWluxLHDoEPDM8zU4cKxxGURd2T51O34M/FHn96Hx0/Cit0Tjt96/oSC2oNG+V4+8iohfIzBk4xBM2DoBz+x7BkXVRSpd19zaHG+XvY0xX48By7IoSlA870buDQDAQN+BTV5j6t9TMWzTMPxz759G+5yDnBEwMkDrKxSJ/uXfycfn1p/jwlcXWnUdqUSKS19fwv3DSt7kPMK7tzcil0Vq5XMjz4wHvrn+xzAaPwkxXqUppbj601V5fyuiXTR+Nq++HigV2aLEqyu8OzlgxAigSxdu36VLisfG743H8TeOI/OSgcqTtHPJx5Oxf8l+FCcWI3hcMEZ/NRpB44IMHZZBNdfP78erP2LOv3PQ+efOcP/WHWfTzqp17eBg7ldlSb+002k4+NxBFMQ0fiYiHD6fj8DAQL3dj5J+WiZ9tEgzMXpVhVW4+cdNhZcxEpEE+TGt67VGSHNOvHUC37p/C1GFdmeEGWKln1QqRV5eHo2f7dTwj4Yj8MtnUW9ho3bSb9QowMICiI0F/nukuqxYDFQ+mCTm6Mj9qqynHyHKREUBpbaXIR3xPqK6D8bOQAGeP/gC1q9v3E9DF0Inh6Lnop6NSo5qG42fhuc/zB8jPx8JpwDFFXM5FTlYdXUVbufdxoWMCziSdAQbbm/Abzd+U/naPD73qBazLQZru6xF7M5Y+b6YghgAQHeP7k2eP9xvOABgd9xupftZloVErN4sX9L2mNuaI/ypcHiEe7TqOvbe9lgetxzjV41v9jhpvVTrY1vurVwkHmx5haE20fhJiPE6/uZxHHn5CJWw1hEaP5tXUgJUOPshs+8MhA33AgAMGsTtu/DI/Jyus7tiyu9TEDhaf4kAY5ITlYNbf9yCqFIESwdLDHprUKPP5KbivxX/4dK3l5pN+vX06Il+Pv1gY26DkpoSvPTfS5Cyqv87DgnhflWW9AueEIyn/nsKnhGeGkRvGqRSKfLz9ZcroKSflhmqNB4xnIK7BTiw5ABSTqTItx1adgjruq8z2VrSRPcCRgWgzwt9YGFvobVr3tpwC6s6rkJ5hn4LdLMsi7y8PBo/2ylbd1tUO3oDDE/tmvEeHsCiRdzXGzcCeXkN+2R14s3MAFkFBFlPv4dLNWZdycLWCVuRcrJhDDYlmZczceLdEyb98+bsJ2cR+09DMiQ/H4gtisbFzgNxuOIL5FlcREfrrghP/wX79wNffQWIHsyXSClNwaRtk/DuiXfVuuc3F79B8E/BuF+sfEVM3xf7YuKaiQolR3WBxk/D8+3viyHvDYGNq2K57aLqInRz74ZQl1DsmrULz/Z6FgBwKvWUytcWZgtx7edrYHgMQiaFyF8IiSVixBVypYvCPcKbPP/xzo8DAM6knUFJjWK/tMJ7hfjM4jOc//y8yvGQtsk1zBXTt0xHyMSQVl2HYRi4dnJtcdVd/L54fCn4UqUyoKo68fYJ7JqzS69jGY2fhBivHot6oP+r/bVSgpg0RuNn8woeLHRyc+PKegLAwAeFGeLjgaKHijfYedih1zO9wDAMWJaVr069vem2vMQ7/T03bfA7g/F26dtwD3eXb9NW2fP25vam20g4nIyCAu77TrYq72Gfj/ocV5ZcQdarWbC3sEdMQQz2xe9T+R4PJ/0e/bZ09HNE8PhgrbQgMlYsy1LSj5D2xLOHJ+YemqvwoN1pWicM+WCIAaMixq77vO6YtHaSVstxmlmZwcLeAhW5FVq7JjF+FTkVKM7jMijqrvQDgMmTgW7dgNpa4McfGz48ykp7Ojo2PCzJyns+vNJPVCVC2tk0VGSb5vdtzvUcXPzyIgrummYZDalEijMfncG9nffk227c4Fb5AUCoSyj+mPoHTj63C+++Zg8zM+DiJRbvfiDG0bjz6Pd7Pxy+fxhfXvwSR5OOqnTPanE13jrxFpJLk/HuSfWShcR0dPfojpgXYhC3PA4zuszAK/1fAQCU1paq/PKmOLEY/734HypyKvDk/idh7cwtd04oToBYKoa9hT38BH5Nnh/sHIxw93BIWAkOJh5U2GfjZoOQSSFwDlFztgYxanXCOmRezmy2R6W5jTm8I73h6OeotfsOfGMgpm2apvPV0YQQ0xA2JQzjvh8Hc2tziGu023OXkJYUFgIeKZfheeOQ/DOfiwvQuTO3/9ESnwAgqhRh1xO7cOr9UyiMK8S+xfuwYfAG/BjwIy5/f1mP0bcvDMPAytFKPmEpP5oreX7pOyV/yUbuzYI3EfbBLABc7z3Zu5PcilwM/GMgVl9dLf9+dLJ2wop+KwAAn5z7ROVnk4AAgMcDysqA4uLG+6mKSNtCST9CWsna2RohE0MUlpCHTg7FyE9Hwtbd1oCREVPBssD77wNvv83Vj9dUtye7Yfm95fDt56u94IjR+63Pb6j9dTMAqL3SD+ASei+/DFhZAXfvAgcfvJd+tJ8f0LinHwAEjAzAe1XvoceCHhpE376xLIuus7tiedxydBzc0dDhGATDMHgx4UWM/nq0fFtUFCC0iQYATAubhsURixHkHIRhwxh88HEtYoMXYSM7DJN2jpL3V1seuRwjAkYoXFssBs6ebVh1KnM8+bj86zNpZyCWNH6ZlHo6Ff/M+gd5t/Ma7SPGZc+CPdg+ZXuT+3kM97jV2bUzcl7LwY3nbqg8Yccn0gdP/fcUei7qqbA9Jp8r7dnNvVuL15rWaRoA4GiyYlLb1s0Wc/bOQfenmi4PStqHS99dwsEXDkJa3/oya1HrorBh4IZmx66QCSFYeGoh3Lu5N3lMvbQeicWJKr9EChobhC4zu8hL2hJCSGuVpZdhTac1OP8FrWgn+lVYCAjy74Mfd1fhc9rgwdyvj5b4BAC+BR/lmeUoTSmFa5grZmyfgbkH58LCzgLmNuZ6irz9ybmRg8J7hfLfO3RwQPD4YDgHmd6kNjMrM6TmWgFQLO25I3YHLmddxva72xW+H1/t/yrsLOxwr/CevG1ASywtAb8H8w0fLfFZX1ePlXYrsW+R6isHiW7Rp2ot0+aqG9I+0CwGYgjbJm/D+ZXcA0xxMRAdDdy7B1y9qvk1DTV+MQwDZ2dnGj/bqR4LeqDcl+tMrslKPwDw9ASefpr7esMG4NQpxZV+MspW+jEMY7LfO0VxRVgTugapp1PBt2i+HJuxYngMXEJd5BNvRCLgzh2gwppL+j3a70zschtZTttQancZEogxOWAWqt6rwpqJa2DBVyyXvH8/8O23wIoVDQ3RAeCxTo/h2pJrAIDimuJGyRQAqMytxL1d91CWXqbFP21jNH4aXp2wDrVltY22P5rsYBgGXvZeal3bws5CaZmcAR0G4LfJv8ln6DYnwjMCAJosRUvav6T/knBv5z3wzFr/aB84OhCjVo6Cvbd9q67z3aXvELYmDB+d+Uit8/RZwozGT0KM176n9+H8F+dhJbCCpb2lfHt9bT2K4ouaOZOogsbP5hUWAon95yHwh5cUtstKfMbFNV4lxbfgY97ReZi5YyYYHoNus7vBvZs7Xoh5AZEvROop8vZn36J92PH4DvnvrZ2s8eT+J9FtTjcDRqV/oioRsq5kIf0e1/IjKKhh39aYrQCAueFzFc5xsXHBtunbkLwiudke4Y+SlQ1NSlLcbmZphqBxQfDo0boe08aMYRg4PTyrXcco6adlPB79lZqa3U/txkqHlY1m1x5adgibx242UFTEmNXX1SP9XDoK73Izmh4uCX34cOuunXszF4eWHdJr03Mej4eOHTvS+NlOjVo5GuneAwBottJPZsIEYOhQbrXqDz8Ae/Zw25Wt9Kt6pH1d8vFkpJ5K1fzm7ZS4RgzPCE+YW5ujOFFJfQ0TIK4RoyK3AhIRNwEnJoZL/NnyXeBs7dyo31l/3/74bcpvsIUbQrM/xuC8v2Fj3pBQEUvEKKjiSqWePcttKykB3nkHOH264TqRPpF4pd8rAIDN0Y1/1neZ1QUfij9Ep8c6afFP2xiNn4Y3Z+8cPH3+aYVt9dJ6uH7jisj1kfLVpA+TSFs3Yczf0R/P9n4Wc7rNafHYYGfuyTypJKnRvvMrz+PUB6r3GCRt0/zj8/HS/ZdaPlAFXr28MPidwXD0d1S6v05Yh91P7Ubiweb7+U3vPB0A8Nm5z3Am7UyL9y24W4DvvL7DpW/0V46Lxk9CjFfe7TwU3i3EkmtLMOitQfLt+5fsx+/9fqfEXyvR+Nm8ggIADA+egYqTtlxdgU4PHg2Ulfi0ElhRIlVNA98ciEFvD2r5QCNXGFuIPwb8gepLtwE0TJxOLE5EVE4U+AwfT3R9otF5U8KmwNdBvUpfslWEiUo+Cs7ePVthzCWKeDweOnTooL/76e1OJkIqbX1ZFdK+ePTwQNDYoEaza2tKalBdVC1vxEuItphZmuFd4bt4fPPjABoaRQPcir/sbM2vXZJcgqhfopB+Lr2VUapOKpUiIyODxs92qrwckEi4Mp2tmbTEMMAbbwBPPPgsKvs+Vpb0e3ilHwDsXbgXp943vRfX3r29sejMIlz8+mKz5QWNWcb5DHzv/T1ubbwFgOvnBwBvd9yLojeLEO4e3uicpyOexp15+QjN/QgXL/CQkcFtv5N3B71+64XZu2YjK4tFairA5wO9e3OlPr//HtiytWGcmtd9HgDgaNJR1NYrrvTim/O1suqmJTR+tk1JJUkoqSlBXGEcnK0bZkNUiioxbss4uH7jiipRVTNX0J5g52D8OvlX7HpiV6NVVAl7E3Dnrzt6iYPoDsMw8n6PulYUX4SYbTEKpbQAboXejZwbqKuvAwCEuITg6Z5PgwWL+Xvmo7SmtNnr2rjZQOAngLWLfv4cAI2fhBizpbeWYvHFxfIEiqxfaK8lvdDliS5wDnFGTUlNc5cgzaDxs3kF+SzsitNhJy1vtE9W4lNZ0k+ZrKtZOPLKEZ1XD2mveizogYjFEQrb7my+g11zdmml7Hl7Ye9jj9Ffj0aVmz/3+wcFG7bFbAMAjA0aC3fbpsuyA8CVrCuQsi3/nT280k+PBRqMglQqRWZmpt7uR0k/LdNnSRLSNgx9fyie2NV4xsSM7TOw9OZS6k1BdIbhcQ8xDyf9AOC//zS/ZsiEELyU9BK6z9dfjx+WZVFSUkLjZztUEFuAf2f+DYeCJDg5cQmS1mAYYP584LXXADMzbpura8N+ZT39AGDC6gkY8ZliPzZT0uf5PohYEtHygUbI3tsekS9GwrOHJ1gWuH6d296nT/OlX4OCGAwcyD2obH+QL3WwdEBySTLOpJ3Bj0cOAAB69gQ++ohLRrOQYllMD8zZsRD5lfno5dUL26ZvQ9orabAys1K4vkQsQfr5dBTEPjJAaxmNn4ZVX1uP62uvI+tqlsJ2Wc+9ru5d5T39AMDW3BbxRfEoqy3DxcyLGt2zoq4Cv934DVeyrqh0vLW5NZ7r/RxGBoxs9O/hyYNPYnncco3iIG1DTWkNko8no6pQe0nk/c9yK2GU8enrg3fK30Hv53orbM8UZqLP+j5w+doFIokIAPDThJ8Q4hyCLGEWlh5c2uw4ZedhhyVXlqDXM7209udoCY2fhJiG2H9i8Z33dyhNKYX/cH9MXT8Vh5Ydwk9BP0Fc3bgvM2kZjZ9NY1mgJKcWnS5vQsbmc432y0p8xsZy1URaUhRfhKs/XkVOVI6WIzVeuTdzEbszFhW5FYYORW8cfBww6M1BKLH2AQDY2XH/Tpsq7fmopQeWYsAfA7Dx1sZmjzuZchJVtrEwMwMqK4G8R1pA391xF/sW70N9Xb3mfxgjxrIsSkubnwinTZSNIERHaFk+0RVhlhAJBxJQmV8JoKG8Z1gY9+uJE0BdnWbXtrCzgHMQ1ecnqqnIrkD2mfswr6tsVWnPR40YAXz9NTB7NjBkSMP2psp7dpnRBYGjArUXQDvAsiwOPHcAd/++i34r+mHQm6ZZRsO9mzsmrp4I3/6+yMnhHjz4ZhL06NHyuXPnconmCxeA1FQgwClA3iNta9rXALjvP1ky2rbLOVRY38WBxL2wt7QHwzB4MvxJOFo5Nrq2uFqMTUM34fJ3l7X5xyVtTGV+JQ4vP4zYnbEK22MKuKTfoytNGYbBCH9ugsLp1NPQRHR+NJYeXIpZ/8zS6PyH2brZwsLWouUDSZuVfTUbW8ZuQfyeeK1el+EzjWbIlyRxbyctHSxh5ag40eF6NjfjItQlVN4f1c7CDlunb4UZzwz/3PsHm25v0mqMhBCijKhKhNh/YuUlPK2drCEVS1Fb3lCVwauXF4LHBytsI0QbKiqAWjEPGV3GoefcLo32u7lx721YFriswmNCp8c6YUXyCnR+vLMOom3fiuKLsLbbWtzedFth+/CPh+P9mvch6CAwTGAGVMm9IoS9PXAz9yaSSpJgY26DaZ2mNXteJ1eu7uw7J99psjrD7zd/x+jNozFgQyTsg7hnn0f7+mVfzcbtjbdRmVfZqj8H0Q5K+hHSCrXltdi/ZD/i9zZ+0JbWS3Hzj5uI/SdWyZmEaC7tTBr+nvo3Mi9xy8JlK/3Gjwc8PLiEyLnGk8pUJqoSIelIEipyTGdmFNFM0NggRB74AMW+3eHiot1rh4QA8+YBtrYN25oq72mKKnMrcXP9Tb2W4m3roqK4X5O6z0eXXwPx771/mz3ez6+hxM7mzdzD98v9XoYFzwK55hdRLriIfv0aji/w4WY+hohmK/QBlHm4HIqlgyXGrRqHHgtUyD6SdsvWzRbzT8xHryWKq5OaSvoBwMiAkQCAU2malSRu7tpNSSxOxMZbG3EqVfGe1cXVyInKoZUO7ZhrZ1dM/Hki/Ib5ae2aU9dPxTOXnlEoUXz5+8tY23Vtk/1zr+dwSb9I70iF7ZE+kfhk+CcAgI/PftzsfeP2xOG/l/+Tl+EjhBBNCDOF2PXELtzZzJWvDhwdiFn/zIJTQEPPgD5L+2DG9hmw97I3VJjESBUWAlIzS4h69UfYxCClxwx6MFfzogpFH6wcreAU6CSv8kQaiKpEkNZLG604tRJYwczSzEBRGcaN327gt8j1YEqKAXBJP5FEhIkhEzEvfB7sLOyaPf/Fvi+ii1sXFFUX4X+n/9do/6XMS1h2aBkAoKa+BiccZ0PC1OD+fcXjhn44FG+XvQ1BR9NLuLZFlPTTMlodY1qEmULc+uMWcm/mNtrH8Bkcf/M4rv541QCREWPWYWAHPLbxMfhEckv3ZUk/T08u8Qe0rsRnyokUbJ2wFfF74yGuEeu8LALDMPD09KTxs50qKWEAhqfVlX5NaSrpd/Gbi/jS8Uv5KgRTYO9tj7dL38aw/w1D8f1i/DniT9zacMvQYendjd9uYOfMnaguqkZcHLet3PoOUstSYW3ecm+ouXO5srTXr3Pjppe9FwbZLwAAFIR8DbsHz0fJJcm4VrELAGCX9DQersqxN34v+q7vi68vfi3fxjAM+r/cH/7D/bXy52wKjZ+GZW5jjsBRgXDr7KawXVbeM9yjcWJOttIvKicK5bWNe720RH5tNZJ+u+N2Y/H+xY1WWt3eeBvrI9cj91bjz7GkfXD0c0Tkski4hrkq3Z9QlIAdd3eo1KPlYWVpZbjy4xV5ieLAMYHwH+4Pz56eSo+XJ/18IhvtW9FvBbq5d8O4oHHynn/KpBxPwbWfrskrWegajZ+EGCc7LzvM+HsGus7qKt8WMDKg0QplGYlIoq/QjAaNn02TvZtxb6Z9mizpd/cuUFbW8jWri6qReVl/fcDaC+/e3ngx/kVEPK3Y5kJaL0X2tWwU3NVtm4W2RFQlQkVuJVgeDzweYG0NDOgwAIfmHsKvU35t8Xxzvjl+Gv8TAGBt1FpE50fL92UJszB9x3SIpWJMCpkED1sP5EpiEdvh1UZJP2sna1gJrGhsaALDMPDw8NDb/Sjpp2U8Hv2VmhK3rm54q+Qt9H+1f6N9DMPgyf1PYsa2GQaIjBgzp0An9FzUEw6+DmBZbjYZwH2wHDOG64V2/z4a/QBWlf8wf4z9fiw6DOqAtV3WYt+ifTqt18/j8eDp6UnjZzuUdycPeZdSwEglWl/pp4ws6VdbCzzcN97Oww7evb1NbnWAlaMV7DztYGFngdybuXp7UdqW5EfnI+7fOPDMeMjIACRMHXJFCQCA7h4t9yb19QUWLuS+Xr+eGzc9U94AWAYJ2I+4wjiU1ZZh8vbJqK6vhnf9QDhW9lcox5NXmYfrOddxPOW4Lv6IzaLx07CkksaziytFlUgpTQEA7P89HO++C0geep/YQdABwc7BkLJSnM84r/Y95Sv9lCQUmxLsHAwASCpRrMHjP8Ifo1aOgoOPg9pxkPbhiV1PYM6/c/DF+S/UOq/4fjGOvnIUGRcyAAAe4R6Yd3QerJ0bT6aQslJE5XBLrR9d6QcAtha2iHkhBr9N+Q2WZpZN3nPoB0PxatarsPNofja6ttD4SYhxshJYodvsbk1OUpCpLq7G+sj1OPbGMT1FZjxo/GxaYSHgnB0DwfZ1yLmhvA+fuzsQGspVGbl0qeVrHlx6EBsHb4S4hiozqKK+th6/9/sdF768YOhQ9GbAqwMw48KrENk4wd6ea0+hrlGBozCryyxIWSkW71uMs2lnAQBmPDMEOgUi3D0cf8/8G1umb4G7tRe8ymYgKUnxvUx9bT3yo/MhzBJq6U9mXHg8HiX92jOJhGYJmRKGYWDtZA1rJ+WrCToO7kjLmolOlZQA9fUAjwe4uAACATBgALfvgoafcawcrTDg1QHw7OGJsMfCEDwhGHH/xuHernvaC/whEokEycnJNH62Q1d/uorKtZvBk4ibXelXVF2EirrWrxi1eaiiYk1Nw9c9FvTAgpML4BKqh8xjG5FxMUO+stHO0w5vl72NIe8OaeEs4zNxzUR8WP8heDaWyM4GKq3iIGElcLJygo+9j0rXmDYN6N+fG0v/7/+AitQweJVPAwBsjt6M2btmI74oHr4OvvgkfBcYMArleGTJxeSSZIXr7lu8D7/2anlmZWvQ+GlYN369gc8sPlMosyusE2Jap2no4dwfydFuuHsXiIlRPG+k/4MSn6nqlfhkWVY+81adlX5NJf28e3tj8DuD4ejvqFYcpO3YNGwTtk3apnRfZnmm/PvlozMfqfX95hPpg0XnFqHz9JZ7CN0vvg9hnRDWZtbo6t61xeObYu9tDwcfB72VMKPxkxDTZu1sDZ4ZD3wLvqFDaXdo/GxaQQHASOrBE9eBb97095Y6JT67L+iOMd+OASsxrQmuLUk9nYobv92AqFKksN3CzgJjvxuLnk/3NExgBiLr5yerVNNcdYWmfDv2W1ibWeNG7g38l8SVD/O088TphadxdN5R2FnYYXTgaCSvSIZv3RjU1gLZ2Q3nFycWY12Pdbj5x83W/nGMkkQiQUpKit7uR0k/QlqhNLUUBXcLIBE3/WGnLL0MG4dsRNbVLD1GRozZv3P/xerQ1WBZVl4+wtWVK1EHNCT9rl1r/b3GrxqP/q/0x/E3j+Ps/51t/QWbUFFB/QPbo4jFEagcNhESM8smV/olFCUg+KdgdPihA/bE7WnV/czNuZWsAPX12zN/D/5+7G8A3AQUUy6hwePzkJPDQCoFRI4Nq6BU/TthGOCVV7ieqOUPqi0+7vQxjs47ihX9ViBLmAUbcxvsn7Mfk4d5AeCSOLJjg5y4fh0Z5RkKD1d8Sz7MrMx0ulIaoPHTkBw6OCBkUgjsvBpWJnnbe2P37N1YYduwHPTRFzqjAkdhYIeB8v5+qsoSZqG8rhx8ho9Orp1UPk/2PVpYXahRSVHSdlk7W8PKSXnJuodXH0tZKZ7890nkVqhWytXK0Qp+Q/xg62bb4rGy0p4RXhEw4zXdQ0csEeNO3p0m97NSFmVpZSjP0N/3KI2fhBifqF+j8K3nty2+f2EYBosvLcbYb8fqKTLjQuOncoWFQHHHCPTY8DI8uje9okeW9Hv4maIpnR7rhAGvDoCFnYUWI23/YrbG4ODSg0rfxw54bQACRwUaICrDiNsTh8Q9sQC4fn4A0P+P/hB8KZCv2FNFR0FH/PvEv3i659MK1RsszSzhZe8l/72dlTX8/bmv0xvmPkLgJ8Cwj4chYGSAxn8WY1dZqb/qTJT0I6QVLn9/Gb+E/4LqwqbfPmdezETW1SyIKkRNHkM0p+uXqW2RnacdHP0cwTCM0prxvXpxCcCsLCBXS216pv4xFY9veVw7FyNGo+Ogjsj2jgQYpsmVfp+c+wTldeUoryvH9J3TseK/FRrNOpNR1tevprQGZz4+g4QDCRpftz1hWRYjPhmBwe8Olm/LvZWLKz9eMbmyL0lHkpB3Jw8ZXAU6SN24VS3d3Vsu7fkwW1vgnXcaksozh3TH2KCx8LTzxKXFl3DkqSOI8IqAhwcQEsKV45GV+HS3dYedhR1YsEgtS5Vfc/Ivk/HMpWdMOiFr7MKmhGHO3jlwCVGc9cCywJUrDb+/fFmxxOesLrNwcfFFTA6drNb9ZKU9O7l2arZM4qPsLe3hYcu9eEouVVyRunnsZuxduFetOEjbMXvPbEzfMl3pvtyKXFjyLfHmwDcR7h6OSlGlQo8WbYn0jsRXo7/Cs72ebfIYYZ0Qjl85ouevPVFSo7z/bk1pDX4M+BFnP9XdJDNCiPGzEljBOdgZVgLlEyIeRp/RiLY93HqlOcqeKYh6Br87GE8dearJfp2m5Pxn55H4MzfZS5b0yxJmQVgnhJO1k1rXmhAyARse24AZXZpvVVXlch4J3v/Df8kH5NusBFYY/tFw+A3xU+8PQHSCkn6EtELo5FAM+WAIbN2bngUbPjccr6S9gsDRpjPLRNdEVSJUF1Vj91O78XOnnw0djt6N+34c5h+fDwDIz+e2PVwW2tYW6PqgupI2VvsBXPNzzx7N90UgpkckAmSTPJta6bd+ynq8OfBNLI9cDgBYfW01Bm0YhLjCOI3uaftguH046Setl+Ls/53F/UMaNrJsZxiGQfd53dF9XkNiK35vPI6+chQl95W/TDVGLMti58ydOP7mcfkMwwrbB0k/Ffr5PSo4GPjgA+CJJ4AhD1VKFVgJMMSvYYNsZq6shDLDMPKVVI+W+CSmp6i6CBkZLHJzudXJdnbcDO7Y2IZjNH3JOKTjEJxacArfjPlG7XObKvEprhJDXG1akwVMxftD30fp26V4b8h7+PeJf3FtyTWMCx6n9fuEuYbhrUFvYVHPRU0e42DpgA4OHQAAlzKVNzCydrbGoLcHIXhcsNZjJISYjm5zumHxhcVw7eTa4rGslMXxt4/j4tcq1FgkRAUFBYBTzl1UXo5p8diBA7lfWyrxybIs/hr1F3bN3qWFCI2Hc5AzgscFK/1cHfVrFH7o+AOKEooMEJn+TfplErxeeAwAl/Srra9FUTX3Z/d18NXJPbOtTuC+96e4UHBQJ9cnrUdJPy2jmUKmJXhcMEZ+OhI8s+b/Kdl7c1MtLnx1Abvm0A/q1jr+1nGs7boWokoRnAKdIBGZbi152UwyN7eGbSzLokuvMgDaS/oBXFPe2rJa7V3wAYZh0KFDBxo/26G1XdbA//Y+WFg01I5/lI25Db4e8zXWTFyDg08ehIu1C27k3kDErxG4W3BX7XtaP2ih+nDSz8bFBs9HP49RK0dp8Kdof5StcO4+rzsWnFoApyD1ZvK1aywwZf0U9H+1vzzp19U5Av18+iHCK0KjS/buDcyf37DiTxll5XiUJVTyo/Nx8ZuLKM/UXak6Gj8N6/L3l3H87YYSiizLovPPndH1L2dUWiagZ8+GktvKXugUVxfjm4vfqJwstre0x4iAEZgQMkHtWJtK+i2+uBiz/pml9vWI4WVfy8bZT86iLK2syWOsza3haOWIEJeQVvXb04bBHbnV6efTzyvdzzAMRn85Gl1mdtFLPDR+EkIYHoN7/9xD7I7Ylg8mcjR+KicScc8GXvcvIOan0y0eP/hB0ZboaEAobPo4hmG4frf0161AXC1usvKXha0FbN1tTWZim09fH8CfK6lpbw9kC7lGe9Zm1nCy0s37gUAn7n45NakK2w8+fxB/jfpLJ/ds7xiGga+vbpKwylDST8t4PPorJU3Lv52PpP+SUF9bb+hQ2rWOgzqiw8AOmL1nNp767ymTar4tEUlw8v2TuP8ft6JJ2Uq/Fw+/iHl3nRDb4RVEx4qgjZLRJckl+ML2C5z/QvmLmtbg8XhwcXGh8bOdYaUszB1tITG3hLMz1xdNpra+FutvrIdEqpiQnxQ6CXeev4MJwRMwMmAkurqp/wJSWXlPhsfAI9wD1k7WmvxR2p39S/bj584/KzzEuIS4IGBEACxsTafXA8NjEP5kOEImhMjLe/5v4Je4suQK+nj30dl9PT2BoCBAKgWiorhtIc4h6OCg+PIj62oWTrx1AoWxhTqLhcZPw7r3zz1Eb24ol5hflY+i6iJUS4SwFnVE//4NSeJLl7jvmYct3r8Yb514C2uvr9V5rC/1fQnH5x/HMxHP6PxeRD9ST6XizEdnUJnf+INeU2W0pawU17Ovo7SmVCsxZJZnYnvMdqSUprR47JCO3IrpC5kXtHLv1qLxkxDjdGvjLVz4SvVxZvGFxVhybYkOIzI+NH4qJ5uQndP3MTz+57QWj3/4maKlEp/zj8/HzL9ntj5II/KN+zfYPmW70n3d53XHc1HPwSvCS+l+Y8KyLKQSqbwCk50dV9oT4Fb56So5H+ruDwAoqldM+tUJ61BTUqOTe7Z3PB4Pzk31xdHF/fR2JxMhkZjuiiNTw0pZrOuxDqc+PKXyOVPWT8HbZW/DzKqZJQSkReFzwzF7z2xutpOJqS6qxoUvLuD+YS7p92hPv0uZl7A2int5mOrxIy6GDMeRS803MVeFo58jOs/oDI8eTTej1pREIkF8fDyNn+0Mw2PQY9XTyOw6vlFpzw23NuC5g89hzr9zGp3n4+CDQ3MPYeesnfIPoFnCLIz4cwR2x+1ulCh8lLKkHwDUlteiJMk0SlvauNjAxtUG5jbmCttZKYuaUtP7gF1XB+TlcV/76al9QO/e3K/RD/I9X4z6AhmvZmBFvxXyY8KmhOGZy8+gw8AOOouDxk/Dmn9iPp678Zz89zH5XCknm7pgmMEaffsCPXpwD99lZcC9e4rnL+29FACw4fYGVIub7g8NcKsClx1aht1xuzXqZ9zbuzdGB46Gh53iz/GihCJc+fEKytLL1L4mMazI5ZF47uZz8Oje+LPZuC3j0GNdD1zOVHyLOH7LePT9vS/2J+zXSgxHko5g7u65eO7Acy0eK1vpdz37OmrEyn9WXfjyAn7v9zuk9VKl+7WJxk9CjNPtjbdx+TvVm6TZe9uDx6dXo+qg8VM5WdLPPtQLfkM6qnSOrMTnrVs6CspISSVSdJnZBf7D/Q0disHVldfhU7NPUbr1MABupd/DST9d6ebDrfQT8tIhZRs+t83YNgNLby3V2X3bM4lEgsTERL3dj36yEaKhuoo61NfVQ1yl+nJxCzsLKoHQCjWlNShLK5OX82RZFuc+P4fLP5hO52Nbd1ssj1uOQW8NAssqNoqWslK8ePhFAMBw/+GwZgQotbuMRZcjcDGjdX0KeGY8zNo5C92fUr9Plipqa7VfNpToXnEx9+ujk5XOZ3ArQsPdw5WexzAM7Cwa6oH+fO1nnEk7gxk7Z6D/H/2bfBkIKO/pBwC7ntiFdT3XqfcHaKfGfD0GT59/utH2HwN/xLZJ2wwQkWHc2XwH3/t+jxu708GygJVTMaxsla9u0bbuD4bC6GiAZZWXd7fztINvf19YOljqNBYaPw3HwtYC9l728t8nFnMPcXY1ndGpE+DoyJWK7deP23/hkYUH44PHI9ApEGW1ZdgW0/y/3dNpp/FL1C/43+n/afWzZE5UDo6+chQ5UTlauybRD0t7S3hFeMHcWnECSEVdBS5mXkR0fnSjJG8/H+6b8UjyEa3EcD3nOgAg0juyxWMDnQLhaecJsVQsP+9RlXmVqMipQG25fsY1Gj8JMT4zts/AojOLVD6+vq4eGRcyUHhPd5UZjBGNn40VFABgWbg7q17ZK/hBG1tZq4KmVBVW4eLXF5FysuWV9aaAx+dh2qZpGPjGwCaPubbmGm5tNP5sqlQiRecZnVHnyK0EeDjp10Ggu8mnXTv6gJGaQcqIkV5CzxEyoioRMi5mNDkZW59jJyX9CNGQlcAKL8a/iHHfj1P5HGm9FCknUpBxMUOHkRmv25tu48eAH5F9natPzTAM7my6g5vrbxo4Mv3hmfHg2skVgg4ClJVxdeMZBnB1BXgMD5+N/AyR3pHYMXMH9ky4AYfqnqiRVuB48klDh06MTFFCEVL/PAerisJGK/2uZXPNJAf4DlDpWi/1ewnvDX4PDpYOiMqJwlcXv2ryWGU9/QCgyxNd0O/lfpBKdL86oK3qPKMzAkYGGDoMvTGzMoO9tz2KKq0AAEn+78H2C1v8eOVHnd+7c2cumVNUBOTmNn1cfV09RJUincdD9I+Vssi5kaNQWjG/iqu5bSX2kvfyAxp6tly6xCWJZXgMD8v6LAMArLm2ptkVfMeTud6BowNHaxYvy+KvO3/hf6f/hypRlXx74OhALDyz0KTGDmNRnFistFfNmbQzqJfWI8gpCIFOgQr7xgVzzy3Hk4+3uLJeFfKkn0/LST+GYRpKfGYoL703ftV4vJr5KmxcbFodGyHENDn4OMCti1vLBz5QmVuJjUM2ImpdlA6jIqagsBCwrCoB8/nnKlcEk1UpyckBxM2sJ5CKpTjx9gnE7qT+k6q6+NVFXF+jfJKRMbFxscETu55AeTDX3sLeHvBz9MPEkIkqTcrSlLOjGazFXFIxOiNNvr00pRRRv0ahNEU7peTbm/w7+dg4eGObeE9NST9C9IhlWWyduBUXv2zdqitT5dXLCwNeHwBHP0f5tqf+ewrPRbVcUshY1AnrUJlfCalEKi/t6eLCvXwGgIkhE3F1yVW427pjTO8gjMs5i1HR6Rhn9T8cPw58/DHw3XeAJpU40s+nY+vErci60vpyoaT9y72Zi/I9p2FdUaiw0q+4ulje20fVvmre9t74fNTn+H3K7wCALy98ieSSZKXHNlXes9czvTDq81FGX56nPLMcJ949gayrjf8djvtuHEZ+NtIAURlG11ld8ey1Z1Fsxq1kYW0KIGElsDTT7co6ALCw4BJ/QEOJz3FbxsHnex+klaUB4FasfG71OU68e0Ln8RD9qy2vxfo+63Hqg4aXOpmlXJ1ZS7GnfHUfAPTsya1SLi1tXOLz6YinYWVmhTv5d3Ap81KT9zuRyn0fyZJ+QiGwbRtQXt742MpKIC1NcRvDMHjt6Gv49NynSCpJkm+387CD/zB/k+mJaiyk9VKs7boW/z75b6N9x5KPAQDGBo1ttK+/b38ILAUorinGjdwbrYqhRlwjL2mr6kul+d3n44uRX2BK6JRW3ZsQQppSnlEOUZXqE64EfgKM+WYMwp9SXqGEEFUVFAAsjw+HweHw7OGp0jnOztxnRKkUyM5u+jh7b3s8feFpjP228c92U5R9LRv7Fu9Dzo2mV5jN2TcHs/fM1mNUhlX5YB6inR0wp9scHJp7CC/2fVFn92MYwBHcpMHYnIa+frk3c3Ho+UNK31eYAgdfB4z5ZgyqCqqwZfwWg05KN+43YwZAjWxNR+6tXNz8/SaqCqpaPvgBvjkfj214DIPfG6zDyIyX/zB/jP12LBx8HeTbnIOdG/W1Mma3Nt7Cd57fIeNChkI/v4d7AcnKfvF4wIBeDrCs98AXXwA//QTcuAGcOQPEx6t/b0mdBKknU1GSrN2+aTweD4GBgTR+tjOhk0JRt+g5CF0DFFb6ReVws2RDnEPgZO2k1jVndpmJMYFjUCepw4ojK5Suemkq6WcqiuKKcPHLi8i7nWfoUNoMWTmceosiAICrjate7isr8XnnDvdrRnkGcipy5AkVa2drhM8Nh3cfb53FQOOn4fD4PIz8fCQ6PdZJvi0xh1vp52nvAe+H/rebmTX0gbx7V/E6ztbOeCr8KQDAmutrlN4rpTQFKaUpMOOZYZjfMADAgQPA9u3AH380Pn7lSmDFCuD+fcXtwc5cDamHk36A6fYDbc8kIgkGvT0InWd2brTvWErTST8znpk8cXwkqXUlPm/n3YaElcDd1l3lnjFTwqbg3SHvItxD+cv1qsIqxGyL0UuZPRo/CTE+9XX1WOW3CgeePaDyOQzDYOAbA+HbT3e9r4wNjZ/KFRYCIhtHRHw6HV1mdlHpHIYBOj5o/5fRQkGwjoM6wtJe95Mb24OCuwW4vfF2s+9jvXp5QdBRoMeoDCP7ejYOv3gY4izuOcTevoUTtGgSsxojYu6jj1VDcrXjkI6Yd3QeAkcFNnOm8RJ0FGDgGwNhZmWGnKgcFCcUy/fxeDz4+/vrLRYaobWM+rWZjsSDiTjw7AFU5FaodV73ed3RYYDu6iqbGqlEivzofBTFFxk6FL1w7+aOPi/0gVOgE/K5n+lwdK+E3yo/rLqyqlGSZPBD+WXr4CjYBN4G0PCSWh3+w/3xXtV7Wu/rxzAMHBwcaPxsZywdLFFk7gWJhbVC0k9W2lOVUl+PYhgGqyeshjnPHIfvH8b+hP2Njmmqp1/mpUxsGbcFqadTG51jTPyG+eHFhBeVPkgWxhXi3yf/xf3D95WcaXwufXcJtzbckj8g18AwSb+YGK5k46MJFb4FH9O3TkfPhT11FgONn4Zj6WCJIe8NQejkUPk2QfkgeJROw6DQxv8+ZT1bUpUMUcsjl8OcZw5LvqXSyQ4nU7gS3f19+8PeknuSz3uQ9792Dah/qHVMQUFDr8lbj7QxaSrp93Pnn/HniD+b/fOStsXcxhwjPxuJHvN7KGzPLM9EYnEi+AwfIwOUr/weF8SV+NRG0g8Aenv11toYVJpcit1P7UbioUStXK85NH4SYnxYCYv+r/ZH8IRg9c9lWZNuE6AOGj+VK3wwX8VN9eqyABqSfi319WOlLIrii1BdZKKzXx/S8+meeKf8nWbL07NSFsIsodG3Wsi/k4/rP18HKxQC4JJ+laLKFs7Sji5uXWBbF4yyYgv5NjsPOwSNDYKtu61eYmirBr09CC+nvKxQblo2duoLJf20TKJJzTzSLvVc1BNP/fcUnIOdWz74ESzLQlpPHyjVdfK9k9gxfYfCtpriGqzrsQ4XvzGNkqmBowIxae0kCDoI5Cv96h0TUFRdhC/Of9Hog3dkJPD550DEC6vwj2MkkgPeAdBQjk4dPDMeeGba/7EhkUgQExND42c7U1VUjdJ8EcCyCuU9Zf19+nr31ei6Ya5heGfwO1geuRzD/Ic12t9UTz9Zw+SKbPUmYrQ3ZpZmcAl1UdrviJWwuPv3XeRH5xsgMv27+NVF3Pj9lvwBu7xev0m/0FDA0pIrr5ieDgQ7cS+YmipNqws0frYd9fWAQ8xbiEzegwXDhjbaH/DgnYSypF+EVwRyX8/FpmmblL5Ak5f2DGjo51f0YK5TVRWXeJa5fLnh60dX9TeV9Os8o7NGL0hJ2yP7fxvkHAQHS+UvFWR9/a5mX0Vpjeb9Vu7kczPIenj0aOFIRQlFCfj33r9IKEpotM+1sytm7ZqFLjNUWyHRGjR+EmJ8zG3MMe77cY0mRLTk/uH7+Mb1GyQe0P2EA2NA46dyZWWAY24c7n55ABU5qj+Tyvr6tbTSL/FgIn7u/DPi9sRpHqSRYBgGlg6WMLM0a/KYO5vv4IcOPyDlRIoeI9O/Hgt6YPG911Hh4g+GAcwsRXBY6QDBlwKU1Gi3StejZAnuwkcKNLAsi/ra+sYnmIDNYzfjn1n/wNLeEpYO3Mpc2aROiUSC2Fj99eWkpB8hGhJ0ECB4fDAsbC1aPvghyceTsdJ+Je7+fbflg4mCwnuFyL6qWOjc1t0WIz4bgfC5pleDX5b0q7HmHk7CXMOUHte9O7BowFTwGB5ulB+F0DoaCQlAba0G94wtwI3fbmh9FiQ9MLQ/e58+gK77vwQAhZV+Hw//GD+N/wnjg8drfO3/G/5/WDNxDRytHBvta6q8Z+DoQLxX+R66z9PuStS2RpgtRFVBldLVQK6dXfFu5bsY/I5plJBecnUJun84FQDg5CJBaS33UKOvpJ+ZGdC1K/d1dDT3kh0AkkobEirnvziPgy8c1GkcNH4aRsKBBGwYtAGZlzMBcL36qqoAgYBLCD9KlvTLzVX+89fFxqXxxgfSy7ip37KyjABQ3FApRiHRd/GhOVAJCdyKPxl50q9UMek36otRGL1yNEj7cfWnq9g6cSsq8xVncjtZO2F+9/nN9szrKOiIb8d8izMLz8hXjmrivSHvYfcTuzE3fK5a531y7hPM/GcmDiQ2Lr9nJbBClxld4BSoXnlwTdH4SQgBuH5prp1cwTOnV6SqovFTkUjEfb6zK81A/JabkIhV//tRtbynTz8fDHh9gMr9Ao1Zfkw+8u403+7Cu7c3Il+MhEMH/a2sMgS+BR9SGzuwfHPY2wN5lblgwaK2vhZOVrr9PGXuWIgE74+wufBVhe3feX6H7VO36/TebRXDMMBDczhvbbiFdT3WKST+9KXplDghpFniajHMrM3ULmkg6CiAb39fWAqoFre65uydo3T70Pcbz6g3VkdePYL6mnpMXjdZnvQr4XEzpUOdlbxlfCDQKRAzu8zEztidyPb7Bg7xmxEb29BjSFW3NtzCle+vwH+EP1xCmn5BSYyfa79AXLxrDzt7BhYPzX3o5dULvbx6teraj46rEqkEfB4fQEN5z6qq5s8xVv+9+B8SDiTgg7oPwPAV/8w8Pk/tiSjtmVOAE8oeVDL18CuFlOUmI7hY629s6t4duHmTS/r16dJ4FVXa6TRkX8vG5F8m6y0moh+1ZbUoTSkFK+Ee4K5clUDEF6JPH0fweI3HI4EAcHICSku5laFhyufp4F7hPdSIa9Dbu+EH9JUlV5Bamirvm8ayDSv9AODKFeCFF4CSEiDuweRvPh8QCrkyoF5e3LamVvqR9qckqQQZFzIa9fbp6dkTfz3+V4vnvz7w9VbH0FHQER0FHdU+L9CR6/GSUtr0zHtWyoJR8u+IEEKak3k5Exe/vIgBbwyA3xA/lc/z7OmJxRcX6zAyYuweVFZEbrcx+PbKEFg7W6l8rizpl5vLJQ8tmnics/Oww9hvG/frNUVHXz2KnKgcvFP2TpPHuHdzx8TVE/UYlWGUpZUh564EYJ1hb88gS5gFAPB18NX5OxKBkxj3vT8Bw/JRL/0GZjwuzRQ6NRT23npsLtiGzDs6T+H32deyUVVQhdrSWlgI9PuuhqaxEKKh1SGrNep/4hrmigUnFiBsShNve4jGlK18MTaZFzORcSEDLAt5T7/8+uZX+sm8OfBNAECq3XbkCw5rVOKz56KemHtoLuy9TPMHOGngPikSmd0mKqzy07bbebcxdvNYfHj6Q/k22Uq/mprGxycdTULqKePu6Rc6JRQDXh8AHl/5R7iSpBLE/hNr9ONhXUUdhFlCpKVwiT5vHynmhs/FY2GPwZxvrrc4Hu7rFyBoKO8p+/ufvWc23ip5S2/xEP3pMb8HXs99HR0HdwTLAidvpuBYhDM+rm66kUtzJT4B4I+bf6Dr2q547dhrjc91CpB/b1dWAmIxt93Ghkskxsc3rPjr3Lmhh+DDJT6DnLjVqFnCLIgkDf1Nsq9nY9fsXci40MIUc9JmTPhpAt4VvgtzG/2Nd9oS4MT9Q2gq6fd7v9/xa69f9RkSIcRICLOEuH/4PmqKlTwoEKJDsqSfg4AHWzebJp/VlHF05PqwsSyQlaWb+IxN76W9MeLTEYYOo004+e5JHJuyBgwrhZ0dFJJ+utbJ1xM8qSVYRoLM8kz59qnrp2LE/9H/HwCY+PNEvJH3BqydrfV+b0r6aRmPR3+lpiJ4YnCzTWOJ9sXtjkNOVE6j7VlXs7DKfxVubbhlgKj069lrz+L5O89DKORmgTEMkF71YKWfS9Mr/QCgj3cfzOs+D1JIcCNoOg7EnlT7/h7hHgiZGAILO+3NUOHxeAgLC6Pxs50peVAe/uF+fnvj92LDrQ3IKNfOi+P0snQcTzmOH6/+iPxKLsvdVE8/ANi7cC9Of3haK/duqyIWR2DMV2Oa3H/5h8vY9cQutfpItEf3D9/HDx1+QNaxewCA8EB3bJ2+FXvn7NVrHEFB3OrT6mpAUuKHjoKO6OvTFxUi7u/fws5CrYd+ddH42TZkZQGZZVyJIRdbxyaPaynpNz54PPgMH+fSz+Hvu3+jrr5OvoL1YbJVfgIB0PdB+9TLlxtKew4aBHTqxH39cNLP1cYVJ+afQNzyOPlMXACoE9YhdmcsiuIfWj5I2qWi6iKFhG5zjiQdwfJDyxFfFN/ywY+4ln0Nn5/7HBcyLqh9bqBT8yv9vCO94R3prfZ11UXjJyHGp+usrvhA9AHCpqo/wTrpaBL2Pb0PteUa9MAwMTR+NlZezv3qXJeD4vvFzR/8CIZpWO2Xnt78sbE7Y7E6dDVyb+VqEKXx6DqrK/q91K/F4/57+T/sXbhX9wEZUOeZneE7byhYHh/29vpN+nm482At4lZVx+en6fx+bV3GhQxc+vaSQvn9h98F8Hg8hISE6C0eGqEJ0dDU9VMx7H/DNDo3bnccds/bDXGNWMtRGS+pRIqdM3bi4lcXG+2z97KHpb0leGamMaTx+Dx5aU9HJxb3Sx6s9HNp+eFmw9QNGB8wFVJeHQ5ZzUFuccMPI9nCIJZl8eWFL7F432LU1jd+6GFZFjUl2p09adFUDQvSJklEEtx+azvc0qMUVvqtvrYaz+x/BseSj2nlPlPDpqKvT19Ui6ux8sJKAA0r/WprAekj78InrZ2EEZ+Z9oyyHvN74PEtjzcq+WZsnAKd0HdFX2RLPAA0PCjrG48HdOvGfX0vxhzpr6TjzKIzcLDkekdU5lci/Vw66oR1OouBxk/DSD6ejLs77oJlWVy7BtSZcxMTvOyb7rPSUtLPx8EH0ztPBwA8+e+TEHwpAP8TPqZsn4Ly2nL5cbKkn4sLMGAA9/WZM4CsL/zAgQ1Jv4SEhuszDINRgaPQybUTeEzDZya/oX54v+Z99FrSutLMRH/i9sQh/VzjN4NTt0+F5WeW2J+wv8Vr/HT1J6yNWouDier3HT18/zA+OP0BNtzaoPa5sqRfWlkaJNLGfU0mrpmIqeunqn1dTdD4SYjxYRhGo/LABTEFuL3pNgpjC3UQlfGh8VORbKWf45Ed2D13t9rn+z2oRttSXz8zK27SVm0pJadVkX8nX+nCAWPSZUYXOD3OvQN5OOnnY++j83tbWQH29dwDzt2shgece7vuYd/T+3T6DNwWJRxIwPE3j6O2rOHfp0QsQdyeOHlFKn2OnabxhlyPpI++gSREibzbeYjZGoPS5FJDh9J+sMDMHTPR96W+jXYJOgrwQswL6Lmwp/7j0iNRlQgJ+xNQmlIqL+3p6FEGZ2tnmPHM5C9RmmPON8feuTsRWDsTvVL+QWqCHQAgORlYuBB4730WS/e+jHdPvouNtzficublRtfYMnYL1nZbq7XygVKpFDExMTR+tiM1JTWojE6GVUWBfKWflJUiKicKABDpHamV+zAMg89GfAYA+CXqF2SWZ8qTfkDj1X6dp3dGwAjjXYFdVViFTcM34dbGplc1+/b3RfenusPSwbiTfj6RPhjz3QQUSLlSio6utRBLDDORRlbiM17JYpm72+9i07BNyI/O18m9afw0nMvfXsaBJQfAMAyuXQNEZtz/Yw87jybPeTjp19SP0N+m/IaX+r4ELzsv1Em4B+WbuTfliWQAKH4wgdzVFejVi+v9UlrKXTMkBHBza0j6paZykySawzfny18ikfbhwJIDOPPRmUbbsyuyAQDutu4tXmN88HgA3Io/dUXnczXie3j0UPtcH3sfmPPMIZaK5fEaAo2fhBifvNt5SDmRAom48YSClkQsjsCbhW+iw8AOOojMuND42ZhspZ/5yCFK31m1RDaBsaWkX9jUMLyU+JJJVx0T14jxS/gvOPvJ2RaPXXhqIZbFLtNDVIZV8aDIj50dkFWhv5V+AODC9wcAJBakybfl3MjB7U23UZlXqfwkIzXorUFYdG4RnAKd5NsYhsGu2btw+fvLkEqliJXN0tQDSvoRooGsq1nY98w+5N7UbEl9/1f6453yd+DereUHcsLhmfHQ9Ymu8BuqekNuY1OWWoa/H/sbt/+8LS+t6OPihPRX0lHxbgUszVR7yW9pZom3/P+Ba8Vw3LnDvSh86ZvzKCgXYlvZMqyPXg0A+N/Q/2FEQONVUwGjAhD2WBgkIvUfpohxsPO0A+9/7yOryzj5Sr/E4kQI64SwNrNGV/euWrvX6MDRGOY3DCKJCJ+f/xzm5oD5gxZGykp8GrPqwmoUxBSoVLrT2Hv6AQ0PNwwDbIj9ERafWeD5g8/rPQ7vB1XoCh+aGC77+/cf7o9xq8bB0d9R73ER3Rrx2QjM2D4DFRVAXFzDSj8P26aTfj4+3PhVWwvk5Sk/xtHKET9N+AlZr2XhwtMX8OHQD7F9xnYwTMOqBdlKP1dXboZtRETD+YMGNexzceFWRCclNew/kXIC/3fm/3A6VbEUct4d7kUpaR8e2/QYhn44VGGblJUit4J7NlFldve4oHEAgPMZ51ElqlLr/nfy7wAAunt0V+s8AODz+PB39AegvMRn+vl0HH7pMMrSytS+NiHEtF1ZdQWbx2yGpE7951RrZ2vYuNq0fCAhSshW+jmP7YMeC9SfEKNqeU8C1NfWQyKWoL6uvsVjNVn1295sGLwB+dtPAQAcHLgJ2JNCJqGrm/beyTTHy/pBr+aShpV+Q94dgneE78A5xLmp04ySjYsN/Ib4gW/Ol2/jmfEw8++ZBulBSUk/QjRQGFuI2xtuo7pIszfO1s7WRr8KQ9+Sjibh2JvHjHr5uJ2XHab9OQ2dpnWSv+x2eDDx38rMSq1ryVam3LoFvLUyFcfcJ+JUT1+ku68DWAY90zaia+H/KT138DuDMfmXyTCzpFUBpqy0lAHL48uTftezrwMAenn1UugV1VoMw+Dj4R8DALbGbEW1uBq2tty+ykcmjp1feR4rHVaiNMU4V1G7dXHDW8VvYch7Q5o8hmVZrAlbgx2P79BjZPp38PmDOLL8AACujElxDZcFsbOw03ssstWuJSXAjrs70OGHDnhq91MAAM+enuj/cn84+Do0cwXSHvlE+iB0ciiiorgVdhbOXBbP067p8p58fsNLnaZKfMrwGB4GdRyET0Z8gqF+iskd2Uo/2fgrK/EJcKU9ZcIeVP1+eBXq/oT9+PjsxzieclzhmodeOITdT6lfjooYRtiUsEaz/IuqiyCWisGAafb7UCbUJRT+jv4QSUQ4k3ZG5XsL64TyZJ0mST8A+Hzk5/j3iX/Rzb1bo32F9wpxfc11FCeq1xOJEEIinonApF8mwdzWXKPzS1NLkXFBO73JiWmRrfQTCDQ7X/b5MD+/5QoNaWfTcOzNYxqtaDUG1k7WeDH+RYz6fFSLx1bkVCD2n1gIs4R6iEz/6uvqIcwSoq6EezFiZwe8NegtHJx7EKMCW/770YaODv4AgKyqNPk2SwdLWNpbKkxaNHaslEVpSimkksYroDtP7wyvCC+9x0RJP0I0ELE4Am8WvYmOQxo3EUpLA65fb/kahXGFyLyUqf3gjFT6+XR86/EtordEK99/Lh2Xv72s0gqY9srGxQY9FvSAV4SXfCaZvb1m1woP51bH5OYC8elFsJQ6Q8yrAI/hYaH9ZvgWLcLmzdz3863cW6irN95kamuJa8RIPp5s6DD0qiytDNUxyeCLa+UJj+s53MCnrdKeDxvmNwz+jv6oFFXiYOJB2D3I6zya9LP3todvP1+wUuNe5dbch2eGYeDa2RWOAY76C8gAcm/moizGuO4AAQAASURBVCCaS7IIBEDRg6Sfq42r3mORJV7KywFzxgpZwizcL7mv9ziIfslWc8o+81k6t7zSD2i5r58qHu7pBwD9+wP+/sDQoYDXQ8+Tyvr6edtzS1NzKhT7mwx6exDGrRpnEquEjVW2sKG0pzm/5RfeDMNgfJD6JT7vFtwFwK0mdLFxaeFo5WZ1nYXpnacrHbO7zemG17JfM+nSZcYkfm88Lnx5wagnZpK2w2+IH/o830fjF8175u/B39P+1nJUxBQIhYB1eR6y3l+Huzvuqn2+QNCQMMxs4TVhyvEUXP72MvWfVEHGhQzsemKX0SbzzSzN8EraK6gdw/VC1vT9YGsM9R6P4TGJWGJ+Qr5NXCNG7q1cCLONM9mqjDBLiJ+CfsLR144q3S8RS/ReLY2SflrG49FfqamwcbGBubXiA3VdHfDee8AnnwBZWc2fv2PaDuxbvE+HERoXvjkfLqEusHJSvqKt34p+WJG8Ak5BTkr3GxvZSr+NpU9j+KbhOJvWcj3zh9nZAcHB3NfOtZE48lg0PhvxGY7OO4pNrz+Fbg8mXr909Hn0+q0XNtzaoHD+sTePae37l8fjITw8vN2Nn3UVdfhzxJ/4wuYLbJu0DYX3TOdD973d8fA6sQXWFQXyl87yfn4+2k/6MQyD/xv+f9j02CaMDx4vX+lX9Ug1sp4Le2L+8flwDjbOMhLZ17MRvy8e4prme9fN2TsH438Yr6eoDOPZa8+i19olALgVz0XVhkv6OTgAZmbcai93M25gvV98HyzLoiihCL90/wXXf1FhNpAG2uv42d5JxBJ8ZvEZ9j+7X568G+o/GI93ehxd3Lo0e642kn4P9/QDAFtbYPVq4M03FY+TJf3i4xt6CDaV9Ov0WCeEPxluUjNy26v86Hx86fglrv50VWG7rD+ej0PLpT1lZH39jiYrf0GhzJ08zUt7qsJKYAV7b3vwzHQ7rtH4qR8x22Jw8r2TOv//SYg29FvRD6O/HK10pQZpQONnY0IhwJeIALEIrESzCVR+DzrZtNTXr/fS3lgetxzu4abZLkiYLUTUuigUxRe1eGyHgR0wfdt0dBhk3L06Ze8HrWzEqKjT70IIPw8B7OpCUFrUUM2uMLYQv/X6DXf+uqPXWAyJ4TPo/2p/BI0JarQvdmcsPrf+HCnHU9C1q37KrgKU9CNEbSzLIuFAAsozyhvtO3++YbBt6WXOkA+GYNhHw3QQoXHy7e+Lp88/jdBJoUr323nYwSnQSaF2srE5v/I8VvmtQklSifz7LLHmIs6mn4WEVX/GyIgRAI8HLF0KDIkU4P2h72N04GgADR84BXXcD6SVF1YqrPbLv5OP9HPpWltRJRKJtHIdfarIrkDuzVyEPRaGEZ+OgEuYZrPd2yO3/oFIC58Ekb0LHB25bWllaQCAMJcwndxzQY8FWNhzIRwsHZpc6Wfsbvx2Azum7UB9Tcv9C0xBRSWXnDB00o9hAKcH803s6wMBAOV15SirLQPfgo/6mnpI63X38sjQ42fM9hj8MfAP5N3hVl6aQr9XSZ0EIZNC4NrJTd6b791hb2L37N0Y4td0+V1Auyv9XFv4dg8K4hLSZWUNPSebSvqR9oNnxoN3H2/YeSqWM5b9P1Wln5/MiIARMOOZoV5aj9Ia1UpjxxTEAAB6eKjfs0imrLYMe+L2YEv0lkb7WCmLkqQSvfT0M/T4aQom/TIJYIFDyw4ZOhRiAn6N+BX/Pvmvxud3faIrei3pBR6fXpW2hMZPReXlQKVzR4w/vALhc8M1uoaqff0EHQRw7eRqst+nBTEFOPTCIZVW7zn4OiD8yXAIOmhYd7WNK0kqwZ3Nd1Cdz70gzBDfhMOXDui6Vn+JJTc37teih3KwjgGOGPHpCPgP99dbHIbm4OOAcd+PQ+jkxu+snYOdETo5FJYOlnodO01zhNAhqZRmBBm7qoIq/D31b5z77FyjfYcPN3ydnd38dXrM74HwJzX7MEAak9ZLUZ5ZjupizfostgeWDpawdrGGpcASQiEgZUTIreV6qmiSaJkyBfjnH2DixMb7ZB84A0qfhZedFzKFmQovZmbvmY0VSSu00hhZKpUiISGh3Y2frp1c8XbZ25j1zywMfnuwaX3odnNHkV8f2HvaQjbB88LiCzj/9Hl0duus89s3lfSrLq7G6f+dRuKhRJ3HYAiRyyIx7c9pTa54limMK8TR148iJ8o4X+pX5lUidmcsilO4yTcCgWGTfkBDX7+qcms4WHL9+wqqCuAU4ISX7r+Efi/108l928L4WVtai9KUUvDN+ci7nYcfA39E2tk0g8WjDxZ2Fpizdw6C5w9AfT2XWGspAScjS/oVFDRerayK6mqgpob72qWFuSYWFkAgl4eW9/VrKumXdSULq/xW4daGW+oHRfTKrYsbFpxYgK5PKL7QCXQKxIIeCzAqQPUeLg6WDkh9ORXJK5LhZK1atYxV41ch+vloLO2zVK24H5Zelo7pO6fjtaOvNdpXX1uP1SGrceqDUxpfXxVtYfw0BTYuNug4pCMs7C0MHQoxAdYu1rB0tGz5QNIqNH42Jmu/4tCKNt6ydzAtrfQDgDphHXJuGOezXku8I70x//h8BI8PVvkcYy1fn3oqFXsX7IUkh2szUM5yJeccrRz1FoObG5Dutg5HrBfiWhZX3cbGxQZDPxiKDgOMe4WlTHY2sHlzwyKgR3n18sKcvXPQcUhH3L+vvzYgJvSGkhDtMLMyw9QNU9F9vmJJm8RE4OF/uy0l/WRybuSgvo5WbbQk+Vgyzn12DjUlNUr358fkY1XHVYhaF6XnyPSn7/K+WHpzKWzdbFFRAVRbpELCSmBrbit/iacuiyaev2UfOHMyrLCkF1dC70LmhYbzbOnBHeDKTspWl4qqRPhvxX8oSS4xcFS6V/Lgj+j8UBXNQKdADO44GDbmNjq7b2lNKb679B32SbkXjY8m/aRiKc59eg73DxtnPzWvCC/0WNCjxfJ7VflVuPL9FaPtXZB9LRu7Zu9CcRS3VMrQK/2AhuRLSQngZsNNdyysNo2Sv5HLIvFG3htw6+IGYbYQDr4Oelmh0xbk5nK/unlIUF5XqtILBTu7hgRhWpr695SV9rS1Bayaz/8DAMIezAl6NOlXWluKGnHDZyprZ2vYuNmAZ06Ph+3V6MDR+HPan3i5/8tqnefr4KtWWVcLvgXCPcLh7+ivZoQNApy47HdhdWGjMlTmNuYY/N5gdHq8k8bXJ21DbVktCuMKsfDUQkxcrWSWISFatuDEAkz+ZbLG51cXVWNtt7U4/vZxLUZFjB3Lckk/+6JUZO6/2WIrhqaoWt4TAPYs2IPf+/2O2vJaje7Vntm42CBwdCAcfFXLsK7ruQ5/jfxLx1EZRvCEYMzcNRtlNtzn+xIJl/TzdfDVWwzOzkC+4CAynf/CxdSbertvW1FczLX5uvzREawf/2+bSjDTUx0harISWCHi6Qj4DfFT2C5b5Scr8aVK0u/ujrtY32c9ojdHazlK45N4KBGnPzzd5AcoQQcB+r3cDz59VS9p1J4JhUClVQIAINQlVOs9eGQfOAsKgCABt3IrpTRFvp9lWSQeTMS9Xfe0et/2JPlYMpKOJsl/qMfujMW11ddMoqH25Re3IezSphZXmWhblbgKbx5/ExdqfkO1RXqjVTK27rZYdm8ZRn46Ur+B6YmqHyB9+vngpaSX0PfFvjqOyDC8+3hjxvYZqPPyBwDY27OYHDoZ44LGwd3WML0tFJJ+tg+SflXcWHB7023E7Y4zSFz6FjopFEuuLEHPhT0NHYpO5d7KxZFXjyD5Ejer1sorGc5fO8PtGzeVzpet9ktJaf44ZWRJP1XH36AHbSVkL5AElgJYm1kDAHIrc+XHuYS64Lmo59BjvuYlG4l+pJ5KxakPT6EiR7s9W8QSMSRS/ZTndbB0kE/SSC1rXOt21Oej0GVG8/0xSduXfCwZa7usxd2/7xo6FEJUYu1sDbDcRG9CVFVZySX+XLLu4NTLBzQu6+/nx7UNKCxULJWoTK8lvTD6q9FA28kv6A0rZdVKrLh3c4dLJ+NsxSLoIID3yE6ot7ABwwAFtQ+Sfvb6S/qZmQFODPdwE5+XJt++a/Yu/DPrH73FYQh1dcBnn3HvAKwrClBxP7/Jd7PXf7mO/Yv36zU+SvoRoiZlP1wqKoBzD6p9LljA/ZqVxf3gb07IxBD0XNwTvv31NyC3V0M/GIqlt5fCzsNO6X4bVxuMXzVeadNUY3Hu83O4u+MuRCLuh0uVFVfCMNRFeZ/D1rC3h7xXm20dVxvs4aQfwzA4+upRnHzvpFbux+e3v16Mpz44hX1P75P/UA+dFIolV5eYRN1yiYUV6s2t5Cv9YvJj8P7J97Hj7g6d3tfXwRfD/LleqDnO2xut9GN4DNw6u3EP7EZGIpLgC5svcGh5yz1xzK3N4RzkDJ6ZcX7Ms/e2R7c53VBp5ggAEAgYbH58M47MOwJna+fmT9YR2b+FkhKgl2cvDO44GLYWtgCAE2+fwJUfrujs3oYeP2/+fhMpJzXIXrVjebfzcHXVVeQlcCVmLV24xn6qlkeUJf0OHADefht45hngo4+AehUKPzTbz68yBbgwG6hqaAbj++AjpmwyGsMwOLngJOKXx6ODg2mU3DE2aWfScP6z86gtU5zdn1ORA5FEsz4hLxx8Aa7fuOJ02ulmjzuTdgZP73saf9/9W6P7PCzQqfHnS30z9Php7FxCXTDwrYGwcrTC6Y9Oo/Ce8U+MI4ZTmVeJc5+fQ/Y1FUsuKcHwGCyLXYYR/zdCi5EZJxo/G5RzHwdR1mMoZu+dDQs7zaoi2doCnR4scr92rfljQyeHYuDr3Phqai5+fRGfmn+K3Fu5LR8MYPqW6a1aAaxthXGFWmtLxEpZeUlJOzsgp4Ib/3wc9LsYwtPSHwCQUpwm31ZTWoOaUuWV2owBywI//AAkJXGJz8T+C5A24fkmj8+6nIWYrTFgRfrL1Bvn2yADoh98xm/T0E3YNHyTwrYTJwCxmJtNPWwYNzunpgYoLW3+Wpb2lnjsj8fg3s0wKxPaE1s3W3j28DTal9gtYaUszn58Fne335X/UK+25pJ+mvTzU4WsxCevnEukZguzUVvf8IJp0i+TMHPHzFbfh8/nIzw8vN2Nn2O/G4tJayfJf2/rbgufvj6wdDD+PhL8WdORHDlHvtLkes51fHHhC2y6s0nn934q/CkAQLbz1kZJP4Ary1OWXqbzOPRNVCVCwKgAOAepltSqLqpG1tUsHUdlGLLJN9ronaEtsn8LxcXAz5N+xvmnz2Ns0FgAwKx/ZmHCmgk6ua+hx09WyuLQskO4+uNV+TaJSIJds3fhyKtHDBKTPoTPDccrGa+g9EF5Q76AW/HnYeuh0vmhD+bq5OYC9+5xq+pv3lRt5Z8s6ad0pV9lCpCxE8jcDVRnA2IhvL0bzqt98CN8QIcBCHMNgznfXOH0G+tv4NqaFt4ykUY0ndGvqQGvDcCye8vgFKSYZO7+S3dYfmaJ2IJYta8pZaUQ1gnx+83fmz3ubNpZbLq9CUeTj6p9j0c1l/Q7v/I81keu1+nfraHHT1Pg2dMTY74aAwA498k5ZF7ONHBExJiVJJfg9Aenjba8fVtC46ci2TOJtbczOj3WqVVVmPo9aAPeUtJPRiKStKlygvrgFOSE0MmhsHHVXVsRXTm07BAOLDmAorgWlnKq6M8Rf+KfoasBcEk/WXsJWbsJffF60D4grypPvm3+sflYcGKBXuPQp+3bgYsXuYTfBx9weYCiEp58EsCjxv84Hu9Xv48effRXVcU0357rkKkNtqZI4CeAo5+j/Pcs21Dac9IkwNwc8HjwzidHxb66LMsi83Imff80ozK/EnXCumaPObTsEPY9s09PEekZAyy7twxjvx0rT/pZ87lefmGuuk36lWa74IdxP2D37N1g0PABNnB0ILwivFp9H5ZlIRQK2933//+zd95hTV1vHP8kgYS99xYciKK499a62rqqtlpb7bJW29ravdtfl5122NpptbVLq1at1rr3nrgQBEXZe4eQ8fvjkoRAgAQCCPJ5Hh/k5txzz+HenHvOed/3+wYPCiZ8omG+GaVc2aK9mbRUlpdLKRC87Hwd6v881MZdEXdhJbKmwO4cSUVXq3z+86ifW6Rmv62rLTM2zaDf0/1MKr95/mZ+6PsDZcV1yylxM/PruF9Z2nGpfoHtUEaZqmn7qY300343KhI8OBifrj4Nct2mHj81Gg3377yfwa8O1h2TSCVkXMwgJ64Wz6dmjJXMCudAZ1KzBE9uta1g9PNxMO0+9+4N8+bBQw8JkX5aI+C1azWfB/pnzGikn89IGHsG7ENgfQBcX4+jo94wXtu89PjXxzn08SGT+tCKQFFGEV+Gf8nZXxpPqt/GxQbPjp5YyfTyc3KlnKwS4eHwdTT/XTyv1zwA/rr4l+6dboyjycIuZFfv+m9YhLpUb/QrziimKL2oQXMVNfX4eSsRPDiYR888SuSMyKZuSistGJ8oHx45+Qid7+lcr3rSotPY+cpOsuNafp72utI6fhqSlwdoNDjJqt+vWnlmJdvjt9daV+/y7AxnzgiBBDVx/JvjLHZbTFaMkQVIC6bT1E7cvf5unAOdTSp//dB1tj69tUlyjicfT+b86vOoVYITk0e4Bw4+Dvh0s8za0LenL86dhY07R0fILhHGLXe7xpUzDXARNsGz5GmNet2mQqOBtWuF/z/2GLRxzia0KBqr0kKuXDF+jq2rLWJrMfnaTYxGoNXoZ2HU6sb19Gyl8Zn8y2Qmrpio+/3MGUhNFULxB5fvefmXR1LfMDHI4sAHB/ix/48k7Kya06IVgV9G/8K3Pb6tsUzq6VRST6bWWKa5IhKJcG/njltbN91G99DST0l6Ool7Ot/TINfUGv2uXxexsO9CJoZPRGZlGMWmUqgoSKlfThm1Wk18fHyzGj9VZSo0asNFjkat4V2Hdxtdp7uxKckuQf7fHhyyE3VGv9RC4XvXGEY/FxsXgh3aAZBSGlvl8673d6Xbg90avB03O5EzIxn10SjdAsOSlOSUkHig6byo3dq74RXppRsLD2X/jfRtKWN+GdN0baog72kMlaJh8mQ19fgplogJGhiEfy9DCZmHjz3MPRsb5t10M5CXmEd2fA7JScJ7oNRaGANNjfQTiWDcOJgwAQYOhI5C6lyuXq39XKM5/S5+DPnl46FrF3DpAu3mg2NbQD8v1Up8Hrp+iDd2v8FfF/4yqHvyL5O5b0fL9chtCHKv5qKUKxtViSI/KZ+iDMOktskFgkVXJpHhamOazGxFonyi6B/YH6VayXcnvzNapqSshF0JgvzniDYjzL5GZWqK9Bv9yWgWXluInXvDefE39fjZ0lGWKlkasZQDHxxA5iTDu4s31rbWtZ/YSit1RGovxbebL46+jvWqJ/NSJvve2ddiFTMsQev4aUh+PkhLcnH88n12vFw1/cmO+B3cv/5+Rv08ih9P/VhjXQEB4OsrSL6fOlXzdV2CXfDt7lurc/ytxOXLMGcOHKrgw5ZxPoPDnx5uEonpo18cZc20NahKhbVg78d7M+2vaUjtpSiK6ibJXpHRH48mZOEEQDD6jWwzkgkdJhDsHFzvus0h2F1YA+Up9Ua/tLNpHPv6GCXZLc8pXqUSUi4B9O8PsZtjcd25FtuCdOLijJ+jVqq5fug6Z7c1nqNgq9GvlVbqiTaX36BBICu3h1TOn1IbEVMi6P9c/3pPUFsynaZ1ost9XWos88CBB5h7am4jtahxKc0vpTC1ELVKrYv0cyx/XOojH1ETWqNfYjV7+2qVmg+9PuTvOS00urIGzqw8w3uO75GwS2+oF4lF9HikByHDQpquYY1A7rVcHI7uxjHrqj7Sr1CICjA1yqW+tHEWNrIzVFVnVH0X9mXQS4MapR2NyZVtV9j2/Dbyk0zzDOtwZwf6L+qPzNGycrMajYYVQ1fw+4Tfq+STaizGfjaWO1ZORVG+TipGkEexta5jLsfDD0DMF/Vqk9boV1QEa89txO9jP27/VcgdsebuNbxt83YVR4GWgFKuNCq/VzECqSWy9emtfNH2cxQKDWIxFGJepF9lgsvX5aZE+lXJ6Zd7Hk49A9Gv6Qs5hkGvL8GzP1DV6Hfg+gHe3PMm6y6tM6jbM8IT11DzDUa3Mv69/FkQs4BO0zsBwhipKmsYI7+WNdPW8HXk1wbHtEY/fyf/Os8L5/eaD8A3J74xGj29++puSpQlBDgF0NmrfpE0ACNDR7J66mo+HPVhvetq5eajKK0IjUqjUxxQFCnIvGQZObNWWjGGolBBaX5pvaPPwkaF8ejZR4m4K8JCLWulpaOV87Pu2QW/nn5VPq8Y4ffghgdZdnxZtXWJRPpov9okPtuOacucvXPw7924+duamuPLjvPvwn+Nftd37hTmylsrqJB3nNKRJxOeJHRkaKO0rzC1UOcY33NeT6b8PgWJVJDC1c7R/n7wb5aGL7WIjHnF/cHFoxaz/u71dPLqVO96zSHEUzD6FZOtyy8duzmWzY9tJvtKy4uallfYBpHJoOPkjgQ9PYViZ79qI/1UChXLByznyi/VFGgAmtTo995779GrVy8cHR3x8vJi4sSJxMTEGJSRy+XMnz8fd3d3HBwcmDJlCmlphuGiiYmJjB8/Hjs7O7y8vHj22WdRKpUGZXbv3k337t2RyWS0bduWn376qUp7li5dSkhICDY2NvTp04ejpooot3LLcOGvC+x4aYcu6apSCQcPCp8N1itb6fKnmGr0c2vrxqjFo/CMaFzd5ebEoJcGMeTVITWWaSjj183A+dXn+dj3Y+L+jdO91Bs6j5XW6JeRAbHpN/jrwl/sTNip+1wsEdPtgW6E3RbWsA25CbHzsCNoYBDOQYaSEuO/Gk+fJ/o0UasaB8c2HpwfNJfMgCidoUNr9KuLpFhdaOveFiulM0VlxdwqqjIJOxM4+MFByoqaVsZSJBIx5PUhjP5kNGXFZexfvL9JjH/aKD+pFHIVwiaih60xvcNaUORA/HJI21l72RqwswMbG+H/JUVWpBSmkFSeSD2gXwCR90Q2uCGgKTi69Chv27xN0tGqE54Lf11g37v7mqBVDU/HKR0Jf3gQiMR4eUFGcXlOPwfTIv0qExIi/DQl0q9KTj+XTjByL0S+Ve05lRUo/MrzbmgNRVpUZSoKkgtQyg3XUa0Y55/H/mHVuFVIpBJEIhFpZ9P4ou0XnPzuZINeN2JaBD0f7WlwLClf+A76O9Z9429Kxyl42XuRXJDM3zFVHbo2xwr5DMa1HWeROXewSzB3RdxldGOqKKOIs7+cJf18er2v00rT4BzkzIKYBQx5XVi/rZ66mmVRy1qkA0wrNwcHPz7I+87vk36ufuOGjYsN3pHeLd6BqRXLkZ8PCjtX/OZPouOkjlU+f2/kexx7+BhTOk4BYN4/8/js8GfV1qfN63fsGLQGU1YlZkMMJ749YXQuop1Lx8Wh2yewdbXFJcRFZ3hraA58cIAlQUtIP59OQN8AOk/vXEURwi3MDf/e/vVKDVOUUcQ/j/1D2kHBEdqxCWNIgj3dGX42gQeSi5FKhPQHEVMjmLVtFh4d6rBGv8nRRvlJJEJOP6cAJ3rc3xmVtU21Rj9rO2tGLxlN4J2BjdbOJjX67dmzh/nz53P48GG2bdtGWVkZt912G0VFermSp556io0bN7J69Wr27NlDcnIykydP1n2uUqkYP348CoWCgwcPsmLFCn766Sdee03v7ZqQkMD48eMZNmwYp0+fZuHChTz00ENsrWD6/+OPP3j66ad5/fXXOXnyJF27dmX06NGkp7cuNFrRE7M+hv3v7UdiLbwsTp0SvPpdXaFThfWquZF+WjQaTYNIsd0qZMdlc371+QbN/9FUeHTwoOdjPfEI9yA/H5LcfuNjdSCLti5qsGs6OgrPNsCqoxu5a/VdLDm8xKDM6E9Gm5xjrCZstLvlzYTwCeHcu/Ve3MLcmropjU5BiTUlzj6IXZywK1fdasycfgDvjXiP0adzaJOySBftpeXsL2f5+bafyb/ReFrpjcGQV4ew4PICXNq4mFReKVfy44Af+fepfy1yfbVKrZOO6Ti5I13v60rsllh2vLCDC2suWOQappCXmMe/T/1L3K7rgOD8kFVSbvSzq8OCwsoRxpyAru/Uq10ikT7az0ohOPCkFwlzyL5P9mXyqskNtnnUlOOnaxtXOtzRAafAql4op5efZvcbuy0iXXOzEXlPJG53DQcECaZBQYOY3HEyEZ4mRgUoi2DvJEgSjChBQcIzlJdHtcnfARQKvTevQU4/r0Hg1M6w8JlXYc+dgH5eqs3ppx2rKxv9jn55lE/8PyH5uIlJqW9x5DlyClMLdesClzYuiCQiaGAftL5P9mXoG0MNjlWM9KsrMisZD3d/GKBKFIJGo2FznPC8jm8/vs7XMJWcKzmsm7WO2M1VZbwtSXObfzZHtJuyne/pzMAXBqIsbXUqaKVh8OnqQ9ScKBx8HOpdl6JQUW/jYUundfzUo3VGrMkpu6dfT1ZPXc3zA54HYOHWhRxNMh5o0rEjODgI9VaKi6lC2tk0Vk9dzdXdV+vQ8ubJXX/cxRNxT1Q5rtHojX55eXpJfI1GQ971PHISGiffeOioULrO7lpjUMeglwYx7a9p2Hva1/k6edfyOP71cQouC2kG7OxVFCmKGjbX5vW1cOWHKofd3UXYKULIz9Yr77iFuRE6MhSZk2WVh24GtJF+NjZCmp/SglJCywNJ09KgsND4eb0W9CJgQEDjNJImNvr9+++/zJ49m06dOtG1a1d++uknEhMTOXHiBAB5eXn88MMPfPLJJwwfPpwePXqwfPlyDh48yOHDhwH477//uHDhAr/88gtRUVGMHTuW//3vfyxduhRF+U7gsmXLaNOmDR9//DEdO3ZkwYIF3HXXXXz66ae6tnzyySc8/PDDzJkzh4iICJYtW4adnR0//liz3nJlJJLG8RxopWm484c7eezCY7pBa1+5A/vAgSCu8G3SelSnpgrRgKaQeCCRJUFLOPfbOQu2uGVQml/Kb3f+xskfavacvvDXBdZMW9MiExkHDQxi/NLxuIW5UVAAJdLr5GpukF7csIsRbbSfrLj6vCv1RSKREB4e3iLGzzM/n+GPSX+0SMOzltSrciRlctxcNYhEwiRal9OvkSL9nB2kSMTCJlLlCVXe9TxuHL7R4rTjre2scW/nrttcrg0rGytKskssFrFz49ANPnD/gFPL9cklOk3rxLS/ptH1vq4WuYYppJ9L58iSI6SdE3IyODlBZnE9jH5iK3DrDo7tIfOw8TK552H7EMiNrrEqrdFPXOIFQEZRRsMuumj68bPj5I5MXzfdqDz5qA9HsfDqQqT20iZoWcOTIvg64OsLzw54lr+m/cXAoIEmni2GG+sh+nVAkIXxKVcGrSnaT37iXRb1m4lUKuSS5syrkF3N3KggBrJPgFppEOmn0VQf6effy59eC3ph59FwedRaElN+m8Ijxx/R/S5zlLEgZgG95vVq9LZoI4v9HKrKipnD3B5zeW3wa6yYuMLgeF5pHu627thY2TC8zfB6XaMiO+J38OmhT7mcddnguEdHD6atndag8npNPX62dE6vOE30r/r3ZtdZXRn6xtDWvH6tNBjhE8OZ8OOEem2ia/lrxl980+2bFqnSYAlax09D8vPBN3YvGcs3Gcz9s4qzdE6AIDhBvDfiPSaGTwTg8A3jaw8rK+jRQ/j/kSO1X//CXxdIOZlS5/Y3N2SOMhz9qq49MjKEoAwtFXOrfdHuCzbP39wIrYN2Y9tx53d3mqyKUNf1ok83HxalLkIVJTwsedaXcHjPAf9PGlDu9cKHcOaVKoe162C53FD6sjFk75sCbaSfTAY5CTm87/Q+xz/ejXe56Et8NdumEomE9u3bN04jgZsqXj2v3LXVrfxpOXHiBGVlZYwcOVJXJjw8nKCgIA4dOkTfvn05dOgQkZGReHvr5XRGjx7NvHnzOH/+PN26dePQoUMGdWjLLFy4EACFQsGJEyd48cUXdZ+LxWJGjhzJoYrZPytQWlpKaak+WWp+uWtHWVkZKpXwQItEIsRiMWq12uBLrD2uLVfbcbFYjEgkMnocqJI8t7rjEolEiCQzcrxyG6s7fsv3SQLuHQQ9pZISFQcPitFooH9/NRqNvk/OziCTiZHLhU2hgIDa++QY6Ii1gzWUb6K33if98aKsIuL+jcO9g3uNfWo7ri1OgU44BTuhVqtv6j5VbLu59ykvT02plSAl5mnniUajabA+BQSIOHNGjDqrDSAY/ZRKJSKRCIlEQtx/cRz94ihDXh+CTzefOvUJICsrCxcXF13bbub7pFap2fDABtqNbUeXmV0M2phxIYOYjTEUpRcJ3+dKbbxZ+1T5eE1jxKkv9tFt60FKHpyHRiN4r52Ze4aUwhR87X1RqVSN0ic7Ow0FBZCfr8bFRd/G/s/1p/9zQh4rlUrVbMe9im1Uq9Wkn0vHzssOe097k/v06LlHASwyRmhEGsJGh+HV2UtXj5WdFe0ntG/U91PIiBAWXFnA8WhbOA0ODmouFQkGQDcbN12dJt8nRQYaa2c0xxYgSliO+vZYsA8xbHtZIeLMQ2hS9yB2iUQd+x0az0Hg2M6gjW5uajQaEWV5wjy2TF1Gfmk+2YeziV4VTZ+FffAI97Dos6dWq8nOzjYYP5t6jNDi1t5N16ab7ftUn7mRSCRixZAVJNm0QSMbgo+PGpXKzD6JpEgGb0Dj1AF1ef2BgSKSk0VcvSoiMtJ4n6xS19Pd9wru7mrUiiLE135HVJyIus/yqn0a8DsqtQY04OmpAsQUF0NOjghvO2HdVKAoILc4F2dbZ0QiEf79/PHvJ2wUNNZY3iLmsKpqjjdAn1QKFb/d8Rvtxrej14JeuuM9/XpyX5f76OvfV3deXfrk7+jPa4MFxRxtPRKJBCepE4ceOESePA9bia1unl3fPn1w4AP+i/8PJ6kTYS56uXiZk4zwieGo1WqDcyz57CmVSnJycnTjZ7N89mo53pR92vf2PkRWIiKm6w23zb1Pxo639qll9ilyRiS+PX1RlCiQiqUtok+WvE9lZWXk5ubqxs/a+qRUKMm5kqPbS7sZ+1Sf+5STI8IpPY7cnDyD4x8c+IDPjnzGuyPe5cneT+qOfzjiQ769/Vvcbd2r1K/tU8+eGnbvFnP4sIb77qPaPnlFevF06tPYudtV2Ytuic+eSqUi40IGUkcpzoHOBn26cgU0GnF5eYiNhV69hDYOeH4AziHO1a4VLdWngqQCHP0dTepT7tVc/r7/b8InhdNnoT5FjDn3ydbDlgKlGNCgkAhrYgepAyqVyuL3ibhvEUndUEd9hEhtuN9qbQ0p3itJs9nLuuh7mNZ9BDkJOXwd8TV9F/Zl5OKRLeLZ01JcDCBGJhOhVCiJmB6BV6QXodZq0tJExMVBp05V+3R+zXnWzTfMqd6Q3DRGP7VazcKFCxkwYACdOwuJwVNTU5FKpbi4uBiU9fb2JjU1VVemosFP+7n2s5rK5OfnU1JSQk5ODiqVymiZS5cuGW3ve++9x5tvvlnl+IULF3AsF9J1c3MjKCiIGzdukJ2tT1zp4+ODj48PV69epUCr0QMEBgbi7u5ObGws8gqm8dDQUJycnLhw4YLBQ9ahQwekUinR0Yae55GRkSgUCoP8iBKJhMjISAoKCoivYHK2sbEhPDycnJwcrl+/rjvu6OhIWFgY6enpur/jrd4nRZ6CwsRCIoZE4BPkw/r1N8jIcMHVVUlZ2XUKCgz7JJX6k5EhJT5ehp+ftUl9GvLnECIjI8nPz2+9T5X6dF/sfQQGBNbcJ1UBRMCVlCsEWt38fTLnPp155wyOzo5M/n4yCQlZFGkEj25NvoaCgoIG65NE4g4EkhXvhdhBTImyhF3HdxHqGUpYWBgpcSnE/RuH2yg3Mqwy6vTsubi4cP78eZydnXVGwJv5PhUnF3Nu1TlE9iK6zOxi0CfXSa489vRjuHu6c+nSpRbx7Gnvk/b7lGFtTbJ3BPayIgoKbHByckKeIsdJ5URMbkyj9Ck0NJQDARPJFF1i25Fl9M5z1fUpKSmpxYx7Fe/TzwN+xiXChQHfDGiaPjlJmLFpBvn5+QblbWxs8LHxIeFEAqIwvTdjQ7+fsosFD7mSkjRuZAuJygrTC5HL5ebdp8znIH0v8QGf4uBhTebla1g5lAn3KSuN60lpgAzrsE3YWIcRlncJ0bFHKbTvS3zwUoM+QQ4FBVbEnMvD1tqWElUJGcUZxB+J59T3p5B2keJT5mPR+ySXyzl79ixubm46Z4zGHCNOvXEKlwgXBi8abLRPfm5+5JzOoTSk1GBx1tTfp/o8ewFeAeTE55Bl50CBXz4FhcnsP5FOr869kMlktfbJrvg0apk34T3uoCA/n/jy8hKJK0VF7ly75lBtn/5V/8OvqzNw8Usn+kIKoU6Dcer6Vg19uqLrk1QaQGGhE0lJUqytU7GT2FGsKmbPyT0M6zLsphz3buaxvGhfEQW5BTiPcEZUHnmu7dOmpzeRmZBJ9ze7W7xPklIJiQcT0bhpsBlio+vT1I5TCVeGgxJd3+p7n9QaNcWSYgZ2Gdhw3ycHQeLoaOxRekh6VLlPZ0+fFSRT63ifanv2Ll68qBs/m8uz11y+T5PWTOLymcu6z1RFKi68eQGfPj54TtFLnjWnPrXE+9SS+nT+s/O4Obkx9P2h9e5T57s7k5qaSky8vp7W+2TYp5SUFN34WVufdv2wi0PzD9HlxS60mdbmpu1TXe/TtWuBKPvNZvoL+nVSsbKYr499TamqFF8b3yrrJ89wT7Kysqrtk0yWTlFRMBcvwldflTFwoAdSaRIKhV7VStun9IJ0CpJvnWdv/fD12AfYM2j5IIM+7dvnSEGBKxKJBhsbR2JjRbo+uU4S8tao1eoG65M6Sc3G2zfS+9Xe+E7WKyBV1ydfD19y4nNIiksy+Nubep9KUgVVn4LcvqhUYq5nnRfarbHhwoULFr9Pbue/xUaRwDkXW3w06VX6lGH/HzfcfudIYnu6OQZRkFmAZz9Piu2LG3TPsinGiJgYW5TKUGQya9KUabR7oR0KFFhfuYZKFWTw7FXsk8RegthZDBk0CiJNQ+sOmci8efPYsmUL+/fvJ6A88cSvv/7KnDlzDCLqAHr37s2wYcNYvHgxjzzyCNeuXTPIz1dcXIy9vT2bN29m7NixtG/fnjlz5hhE8m3evJnx48dTXFxMTk4O/v7+HDx4kH799LmpnnvuOfbs2cMRI/HUxiL9AgMDycjIwLU8CVarJ1DL6tOFNRdYe/daJqyYQNR9Ubz3npoDB0RMmqRh9mxNlT599JGIfftEzJ4NU6bcnH2q7XhzvE8ttU/f9fgOsbWYR449wqJFalZqxpLp/B8/3vEjs7vNbrA+XbgAL70kwd1dw+6oNlzLu8ae+/cwIHCA4CVdpgSRPmKvLvdJo9Fw9uxZOnXqhFYi5Ga/T4pCBeoyNfYe9i3+2at8/IcfRGzYIGLiRA0PPihqsj65vtaeXEksX/bczqNjhuraWJRVROK+RNw7uOPe3r1FjBEqpYr97+3H0d+Rrvd3NblPZcVlHP/6OE5+TnS5t0uD9Emj0fBV+FeUFZXxxNUnDJKUN8RYnnkpE6mjlA27XFizRsT48WpiQhZxIeMCi0cupqtPV/P6FPslmqJrqKM+NDx+fTWa6DdQD1onSH9W7NO1NWg8BoKNp0Eb161T88MPIgYN0rBMFsbVvKscfOAg3Vy6oSxVInOSIZaILfrsqVQqoqOjDcbPxhojlHIli50WEzEtgsmrJhvt08aHNnLmpzMsyliEras+x0NzG/eqtlHE9OlqiotFPP3OJUasj8Dd1p2MZzNq7pNGjXhLJJRmIJqUjEYlR62RgJUt+/fDhx+K6dBBxIcfGu/T6tVqVqwQMXSohqee0tTcJ1UhqqSt4BAKrlG89ZaIEydELFggYtQoFRFfRXA5+zLb793O8NDhiEQi5AVyNszZgH9ff/o+1bfZ36eGHMuXdV6GUq5kQdyCKuV/veNXMi5k8Gj0o1jJrBqkTxqNRjf3aoj7lJSfxKz1s7iRf4NjjxxDjBhHqWOVNlbXp127VHz3nYhnn9XQpUv1ffr48Mc8v/157ul0Dz9P+tmg7d/3+R5lqZJHTj5icLyufap8vKysjHPnzunGz+by7DXX75NapeYTn0/ofHdnxnwxpkX0SdvGlnSfmnOfvu3+LWqFmscuPNZi+lTT8absk0Kh4Pz587rxs7Y+pZ5J5dtu32JtZ81TyU9h42Rz0/WpPvdp2jQxCgV88w14ewvHvznxDfO3zKedWzsuzr8IlXbetW1Xq9W693nlPr35poiTJ/X7LBqNhjlz1EycaNj24txiYtbH4BLiQtCgoBb97KlUKg59fAhbN1ui5kQZ9OmDD0QcOCCiXz8Nhw+LcHaGn35SU+HP26B9yjifwd4399L/+f749tAb/WrcC1NpqJx4zdT79N/T/3H086Ncn/okaSXO9H70e147/ghj245l490bLX+fVAoojAe7AESAWOZk0Mb+r73EUekHTA18kt/u/7hOfarteFM/e1oOH4b33xcTHi7i/ff1x0+ehLfeEuPnB199ZXztfujQIQYPHkxeXh5ONSUCtQA3RaTfggUL2LRpE3v37tUZ/ECwsioUCl3YuJa0tDR8yhNf+Pj4cPSoYfLTtLQ03Wfan9pjFcs4OTlha2uLRCJBIpEYLaOtozIymQyZrGoySm1dFdE+rMbKNvZxrRd4Zapro7nHW3KffLr4MOztYQQPDKakBI4fFyMSwZAhIipeXls+MFAIKU9OFn6a0qeC5AL2vr2XkKEhdJrWqcH71JjH63Of8pPyybiQgW8332rzzEgkEvKu5/F9n+/p9kA3hr893GJtr+54Y36fHj39KGqV8NIoKhKjcBa04X0cfXSTRIv2Kes4pPxLG//HADeyskQEO4VyLe8a1/KuMThkMABW1sZfI+Y8eyqVSnfdyp/frPfJ1lm/eV2x7rLiMpKOJeHg44BHB+P5xW7WPpnalpwcYUzz9BQhEsGRG0fYELOBnn49mdRxksn1VMbcPrmLwsgllvi8BCSSEbrjufG5/DnpT0a8P4KBz+vzazW3cc+gbisJQ14dYnZbRLYi9r29D9/uvnS5t0ud+3Rq+SnO/36esV+OFfIKVio/6oNRWNlaIbGSIJYY9sHS76cNszdQkFKA9IWnAXBxEfPpmE/NrkdH+JOIAN1RlQLKckGRhUhVjETmCpXOEQffZbRuDw9hXpCTI6J3z94EOgdiJbbCxtEGHKuWt9QzZmz8bIwxQmIv4eWSlykrLtOVq3zNLjO74NfTD2uZtdH2NJdxrzJ5eVBSIkYsBo19ubysrVu1bdTVoxFD76+hKBFurEd04G4kg/6CwMmEhQlja2IiiERC3QbkRiPNy8Na0hsvL6nBY2m0T4WpSA5Og47PgUcPAgOFhWhSktCWP6b+gZ21HcHOwbp5hMxeRszfMVjbGd6v5nqfLHXcWJ8eOPAAuddyjZaftHISNi42Bpt40HB90mg0XM29ip+jHzKrqutTjUaCVaXpWm33ycXOhYTcBK7nX2f4iuGczzjPrC6z+HHCj7X2SamElSslFBTA5s3QrVv1bQ9zDQPgat7VKp/59/FHpVCZPHbU5dkzNn7e7M9eTW28Wb5PxZnFiK3F2DjbGJR5Luu5Kt+L2tp4s/SppuPN9T7VdLw59unR04+ilCurbbs5fVKVqfh13K94RXox+pPRtZavb9vrerwp75Ox8bO68r5Rvgx6ZRDnfjuHSq5C5Fz9/kVze/ZKS0FdUIRTfipSpS8SibBfdTzlOAAzImcgEVetY0f8Dt7e9zYRHhEsHb/UaNsXLoSdO+HKFYiJgYwMEUeOSJgyxbAutVzNhjkbiJwZSZuhbRqsrxVpyvs08LmqObQlEkn5HBpGjBBx9KgwX8/Lk+DuDmdWnuH418e564+7cA5ybpA++XTxYdpf04yWrbZPxquuvnyF4+3GtkNqL+WbSw6ACIU4FxDy3Ov2Ayx5n8Q2UJYNf3WC7ksg/EmDuj3tfEAJqYWpLfbZ01JWJvyUyWDTQ5tw7+DOwBcGok3Xl5ICCoUEW1vDOrRr98aihser4dFoNCxYsIB169axc+dO2rRpY/B5jx49sLa2ZseOHbpjMTExJCYm6iLy+vXrR3R0NOnp+uSo27Ztw8nJiYiICF2ZinVoy2jrkEql9OjRw6CMWq1mx44dBpF/rdzaeHb0ZPDLg3ENdeXoUVAowNcXwsKMl9far5OSTL+Gla0VJ745Qfz2+NoL30LEb4vnl9t+4caRGzWWs3G2wSXEBXuv+ifvvhnRbqbn50OplTDmeTt413RK3SiMh4sfwNlXcVCe1yXl9bIWHvb4HP3zqSpTEb89npRT9UscrZVFbg6knk4l7WxaFS8iEAz3K4au4NSPp5qgZY1D4aq/8YvZhbu78PuB6wd4d/+7/Hnhz0Zth5d1WwCu5ccZHHdr68aEnybQ4Y4OjdqemxGxlZjZe2Yz458Z9aon71oeiQcSsfc0PrZ2nNyRdmPbVTH4NQTdHupGv0X9KE+ljEWd4xS5sD4ATjwF7efD7ZfA1rjzFwAXP4Idw6F8LNB+J7Kz4Y+7/mDvnL308u+FslRJ2tk08m/kW7Cxeppy/JRIJdi42FT7eejIUHrP743MsaohormSHZfN4e+jsZYX4O4O2aWC06BJ72ORCLyHQej94NwZgmeAzAsQ5pTW1iCXQwUFGj0xXzDBbhCO0izds1Yj9iHQ7xcInQOAv5CqTzcvjfKJor17ewMjkUgs4uWSl5m0chKt1Iytmy2+3XyNf+Zq22AL+vykfGI2xlCYWqg7liPPIfTzUGzfsaVUaaiQs2QJzJwJlXxka8VJ5sR3d3wHwJm0MyjVSpxkpg24Bw9CZqbw/1OnhDVTdYS6hgKGc0st474Yxx3f3GFew82kOc0/mxMHPz7IYpfFZMZkGhxvzI2uVm49RCIR1rbWtRc0AYm1hNxruRRnFFukvpaIqeOnolB4CQz/33CeiHsCB2+HhmxWo5OfD45Z12h/5BeS9+rXpZezLgMQ7hFu9LwydRm7r+5mU+wmo/sKAG5ucNdd8Pzz8MwzwrGcnKrlHHwcmL5uOqM+GFW/zjRjFAr9HLdDByEIAyCu/JbIc+XkXs2lOMvy3+my4jIurr1Yp3OLM4vZ8789xG6ONfvctmPaMuStEZSUCZ5dhWpB+tXN1q1ObamR9H2QfVJQEAmeocttXxEfB2HdnFGiD6g69Okh/nv2P8u3p4nRCj/KrFVcXHeR5GPJADg7g4eHsD2QkGD83IydjaTtSRMb/ebPn88vv/zCr7/+iqOjI6mpqaSmplJSUgKAs7MzDz74IE8//TS7du3ixIkTzJkzh379+tG3b18AbrvtNiIiIpg1axZnzpxh69atvPLKK8yfP18Xiffoo48SHx/Pc889x6VLl/jqq6/4888/eeqpp3Rtefrpp/nuu+9YsWIFFy9eZN68eRQVFTFnzhyz+lSdRbiVlsXJk8LPgQOhurWLn5/wU/viKSiADz6Av/+uvl5bV1uevPokt39zu+Ua2wII7B/I7d/cjneXmjfUZE4yHjz4IH2e6FNjueZGcWYxF9ddJO96Hmo1FBaCfWkowU5t8LZvAKNf7DJIXA29vga3ngQFCYcH2T3E2mlruT/qfl1RdZman0f9zJElVWWQTUUikRAWFtZsxs9tz27jh34/GP3M0d+RMZ+PIWJKRCO3qvEQxV7GPueGbtM5pUAw+Po6GN/8bCj8bASj3/WiKwbHbV1tibo/Cs8IT2OnNUuu7r7KT0N+ImFXNTPHGvDt5ovUXlqv6w99YyjPZT1Xo3EHIDMmk9L80hrL1Jeec3vS76l+5OUJvzs4qVCoathRrokbf8OucZBzVvhd6gL+48G9t/C7lfHIch0lKVCYAMWCQ4rWQSIrS2cHBCAvMY9lXZdxdKmZu+4m0JTjZ05CDjcO36CsuKzRr92UxO+I5/Aza7EtSMfPT/BmBWp+H2s0kPiX/lkDcOkEA1aBl+CtLJHoNyiuXTNSR9iDrE/8lBy5Lx7GA8kNkUihzUxwFjabKhv9qqOiRG8rxkk+kVyrET/9fDpbn95KSXaJRa99bc81fr/zd64fvK47lpQv3FQ3WzcDI25urhAhUFwM77wDlXxga2V029HMidKvhce1G1frORoNrF2r/720FCqlNDFAa/RLK0qjSFFkXgPrSXObfzYn/Hr60WVWF1xDXQ2OZ8VmcXzZcQOjdSutWAJFoYK4rXEWdbBaELOAST+3OsEYw5zxc/XU1Xzb41vUSnWtZZsjeXlQ5OxLRp/xBA4I1B2PzRaMOO3d2xs9b3DwYKQSKYl5iToDYU2UZ5EiO9twnaElfGI4jn4t35GlOLOYHwf+yLGvjhkcv3ZN+Ls4O4OLC7QVtgp0Rr8+T/RhUcqiah226sPOV3fy55Q/ufxP7fexMhq1ht2v7ebiuroZDYsqTJ0KVUJ+OndbU7wDzeTUM7BzlOAQO2AV+FedE/o7C2uhbIXeezF2Uywnvztp+fY0MdpUmTZ2Ep7Pfp47f7xT95k2MOhK+TZVerreGU4ikZC+I53GoklXdV9//TV5eXkMHToUX19f3b8//vhDV+bTTz/l9ttvZ8qUKQwePBgfHx/WVlhJSCQSNm3ahEQioV+/ftx7773cd999vPXWW7oybdq04Z9//mHbtm107dqVjz/+mO+//57Ro/Vh+tOnT+ejjz7itddeIyoqitOnT/Pvv//i7W3ehnplXdlWWgYatYbP237Otue3AUKoLkBoaPXnaDdX8vKEL/nrr8O+fbBypSB7Ux3Ogc6tXpCVcG/vTo9HeuAc6NzUTWkSkk8k8+fkP4nbEkdRkTCZ6R+zj7jH4/F38rf8BYPvgcg3oO0jYGWrM/rZ5fRhUsdJhLiE6Ipa21kzftl4uj/cvc6XU6vVpKamNpvxs9eCXoxcPNLo99Ta1po+j/fBv3cD3JebAI0Gzo15hrhe9+gMHCmFTWP0C7QXZvIp8rhaSjZ/SnJKyLiYQVmR+cYVjUZD1uUsko6ZEXZuBCtZzYrwl/6+xNLwpUT/VsPurgXRRvqlaaKRvS2jw5d1iOwsTIC0HSCqsGHRdzmEP2na+ZFvwh2xYC8s8LXfCYVC2GTX4uDtwNA3hxJ2WzXSAPWgKcfPMyvO8EO/H8iOy662TMaFDL7u8jXHlx1vxJY1LG3HtMXniakUO/ng6ysYK0Dv3WoUZQEcmwc7R4C8eu/O4GDh59WrVT9LKevDiiMLATBreaIWJp3aeWlqqjAPjU6L5vVdr/PtiW8NiqefS6/zxsOtwsaHNvJtz2+r9cwHSDqaxOFPD3N5k/kbQDUR0DeACT9NwK+Xn+5YcoHgXVx5Trhvn/DetrICtVqI+lu3zrzrfTL6E9q7tyfMNYzBwYNrLX/unLDJIZXCgAHCsZqiDJ1tnHUe6Qm5ho4tN47c4J/5/5B5KdPYqfWmuc0/mxMRUyKYtHISEmtDg8DV3Vf5Z94/JJ9IbqKWtdJSybqcxaoxqzi94rTF6mzdk6keU8dPjUaDa5grnp08KUgu4OQPJ8mKzWqkVjYO+fmgsHdF1Ksnrm0Ey1yePI/0ImFjv51b1YgoADtrOwYGCY5f2+K31Xqd6tYZFSnJLiEnwUgoYAtCUaQg50oORemGjkLauXNIiBCYUdnw0pAMfGEgQ14fQruxxu91Tdh72fPw8YcZ8+mY2gtXYsWwFex8QYiis7GBbr5RTOgwgc5enc2uq1a6vgM9loCoejNSkJuwQMlX6SP9pvw+hadvPG359jQxukg/maCUUlHOXPvsrV8PDzwADz4Ic+cKdgG1Wk3PV3o2WjubNKdfTQslLTY2NixdupSlS5dWWyY4OJjNmzfXWM/QoUM5dapmubUFCxawYMGCGsvUhil9aqX5Ic+TY+tqq5Mu08ou1bTpYmsrvJizs+HFF4UvOAgv6fh4dFq/lVGVqUjYkYCVjRUhQ0Ms14lbhFPLT1GYUsiglwY1dVMshnekN5N+nkRA3wDdRretLVXys1gMt27CP5Uc5Bn4+wsb2kblxhAib+qDRqMhNTUVT8/mEZkVPsG4RMetQGEhlClFILHSLTy0US6+jo1r9AtxCoN0SFfGodFoDBbmy7ouwzXUlenrpjdqmxqKjpM60nFSxzqdq1Fr+K73d3hGePLgwQfNPv/EtydQlanoObdnjRFAbYa3oduD3fDr4VdtmfoSvyOenS/tZMR7I8jPFyThFVbCZrBUUodoxvCF0P7x6kP2a8PaUJ5IKgUHB+F78vXh5Xx8+iXGtR3HDxN+YMhrVXMyWoKmHD/bjm2LtZ01Lm1cqi1jbW+NskTZory7XYJdKAh0QRkvSHKeKSyX96wp0s/aCQatBaf2YFPhXsV9K0QADv0HxFaEhAiHq0T6qZV8/70EpVJEVJTeOFgrJ56Gy5/DlCzc3JyxsdHLh17Mu8hbe99iUNAgHunxiO6UAx8c4OzPZ3mx4EWkDvWLEm6p9HumH4pCRY0bwhFTIvCJ8sGnaw3G4DrgGupaJXoqqUBw6vB3NDT67dol/JwzBzIyhM2HH38ER0cYOdK067nYuBA9LxqxSIyV2HDimZcHb74prGlmzxY2nLRGxREjoFcvOHBAMPo9+mj1Q+3vU37HzdaNtm5tDY7nJuRy/KvjBA8OxiPclPBW82hu88+WQLux7Zi5ZWaLdY5rpelw9Hdk/LLx+Pey3LMlz5UT/Ws0HuEetBnepvYTbiFMHT9FIhHjvhQigi5vuszGhzZy54934t6uASKRmgit+ohzBf90kUjEl2O/JKkgCUdZ9dF3t4Xexs6Enfx35T8W9K55L1omAzs7weCXkwP2lbIuKIoUfOTzEe1vb8/0tS1jDWwMl2AXFqUsqnK8otEPqkb6qRQqLq2/hIOPA8GDTZ1Im4a9pz1D3xha5/Prsn7WqDXkXc9D5CI8ePb28Fivx3is12N1bkeN+FSYOF5eCilbYfDfBpO7EE9hzisX5VCqLEVmJas2PUhzR2v0E99I5OpuNUGDgnT2Au2zVyELHQoF7N8PEyZoKLG1rApITbTqt7TSignYutry8LGHGfHuCEpL9TraPrWs47V5/dLThUFY+wK6WIMDtVqp5veJv7P//f31bndLYcuTW/iyw5eUldQe5RK9KpoDiw80QqsaD0c/R7rc2wW3tm4UFJQfayjlBnk6aMo3Zzd3hd1jdM95UrKatRfX8tHBjygpa7wXVXPj7wf+5qtOXzV1MxqEpLhinDKu4GJViHV5ygxtpF+NUS4NQBvXNlgpnXGjHQWKAoPPnAKdcPBtWfki6opYImbk4pEMeH5Anc4//OlhDn54EJGkZsOYzFHGnd/fiV9Pw0VLxsUMvu/zfb0jDUHwXtXKHGvHQrlYMPp52NVxQ1gsqdFjsVY0ajj5DBy8D9Dn9SsuEpFamEpyYcuNZgjoE8CA5wbUmK/PJdiFx2Mfp/eC3o3YsoZFo9HoFB98fSG1qFzes3JOP40G9k6CmC+E370Ggo2XYZmCOMg8CCXC90NrzKts9Lu66ycWBrjT2Xs/c+eaYad2Dgf/O0FVjEhkKPHp5yh8V7VjuJbuD3dn8qrJrTKfNdBlZpdaHZ5kTjJ8u/kiEjd8pIhW3lN7T0G4x7GxIBbDkCGCp/Ed5enxjh0zVkv1SCXSKgY/ENIdJF/N4eiuRJ54QjAyHjsmPJ8TJkDXroIzRGZm9XlNAEaFjaKHXw9srAwlpNvf3p5FKYvoNLWTeQ1upUlJO5vGimEriN1SNT+RU4ATbce0xdbNtgla1kpLxsHbgZ5ze+Lb3XJOiCqFis3zN3Nm5RmL1XmrUTEowr+PPzO3zKTt6LY1nNH8yM+HiL3LsNmgz2/vJHNifu/5vDvi3RrPvS3sNgB2Xd1lUrqCihKflZHaS+n/TH86Tq6bo2hzR2v0a1Nunw8NFeYjOTn6v9ea6WuqyILWh5iNMex9ey8lOfXbG1OWKkmLTkNRZHrKCpFYxBNxT9D1jYlAVSOwRVHkGWrK5p6D1G1Qahi128bbjeFnr3J3XIlObl6eKyflZEqDpwBpbLRGv6Ite1k1bpXBZz16wIwZcPfd8L//CXNwEHJeA2hUjRcs1rqaa6UVM0krj1S2txc8+mtCu7liYyN4wg4uV8W5dKn6c6xtrZm8ajIj3htR/8a2EKztrLGyscLKpvbQtju+vYNHzzzaCK1qGgoKIMVlHasDQnhk4yO1n2AueyfBPxHCSz10NgRNw7d87ZSWJuKBvx/g2W3PEp8Trztlw0Mb+Lzt55Zvy01I/I54Pm/7OTEbYqotI3WQYutmi0bd8iK/E/Yn0/7IL3jn6fvfVDn93JxkjD6dw30lJ3GSORl8NmPTDMZ/Nb5R29OQpJ5J5fzq85QW1G2y3HNuzzpHqD505CHuXn+3WRJHFROkn/j2BElHk+qdVxCg09ROLEpehGfvNrp1R6G6jkY/VakQZZV3oX6NEomh4DLkngVliS4CVlIqeD5rpX3+mPwHf8+pIalvK82GlSNWIvvuS0DI3zwoaBB3RdxFJ89KhglFNqTugLzz1VcW+SZMzQN7wdqnNfolJUFZuZ9TWRls3enEjfxwovoH6xzKTKLtIzB4LdgK47Mxo19yQbLBplzwoGAiZ0SaNOe6FVGrTI9aVZWpSNyfSH6S5XJMbXliC0tClqAo1G8MGYv0271b+NmtmxB9IBJB53K1J2ObhXUhOzWHHycE8XS/e0lJgU8+gSDncwzsU4C/v2Dw69ZNKFuTxGd1SB2kOPg4NIrhtBXLkZeYR/KJ5GrzvWo0mjrPZ1pppTGx97Jn1rZZ3PbxbU3dlGZJ2tk0loYv1a2b7T3taTumbYvLO5eXp6FM5ojUtZZc4Ebo6tMVTztPChWFHL5xuNbyWqNfTjUKniPeHUGXe7uY3Y7mRN71PM79fo68xDzdMY1G71ykNfrJZPpc2VeugEQqYeqaqQx8caDF2nJxzUV2vbqr3lLAJ749wbIuy7hx6IbZ52pz+tnbQ3FZccOoD+4aA5u76A1/3T+BaUVgY7j+dnMTYacIpijPBpVKOBb9WzTf9viWpKP1dwC+mdDm9PO6azC3f3O7LsoPBIe7e+6BmTMhKkpwvhOJICZGcITLv2K5dUFttBr9LEyr7nfLJGZDDEc+P0JZcZmBtGdtt3v8eOjfH956Czp0gI7lTjc1RfqBIAnUEAlmmysj3xvJo2ceNen75RrqikuIS8M3qhH5656/+KzNZ2jUGvLzQW6dRIH4GtklFtq10aLRgNcQ8LtdeLg7vQiRr+PpKby4yhQighyFRJYVjX42rjY4+jmatRFWEZFIhJubW7MYP9VKNVJ7KVa21W+Gjv18LHP2zWmRm1QqN0+uRt6OQ4SwMy1XysmRC6uOxpb3dHAAESIKCxv1sk1C9K/RrJm2pkruAnMxJVq6MjInGT5Rpkdx/jLmF77v871uwTHm0zHMOzcPzwjLyadpZY7t7SGntNzoZ2um0a8gFo7Ohau/1r9B/VbC2FNgZasz+omKhf5mFAn52wpTCilMs/zD2lTjp0aj4Yt2X7D16a21lr249iLHvracV21T4xbhTaGTYDDz8YHnBjzH6qmrGRBUKZpW5g5Tc6Hbh9VXZmVrEGnq7i4812o1XL8uHPv7b9h0dhrvnTjIHXcH1qvtWqPfjRt6R43ismLySxtv8dnc+WX0L/w44EeTNlUSdiawfNByzv9Rg+HXTGzdbbHzsMPazlp3rHJOP40G9uwRPhs2TH9ubZuF5pKaLuOLIz+AWy8GD4buvv+ydFwk9/X/Rlemd3mQb01Gv6u5V1lyeAnLji+r8llOfA5p0WlGzqo/zWn+2Zxof3t7Xsh7oVpno+97f893Pb9r5Fa10tLZ9tw2lnZcWu+Im8qEjgzFzt18Y05Lx5TxMyc+h5LskipGvpbmGFtQICK2z0yC5t2uO7Y/cT9HbhyhUFHz3F8sEjMxfCK3t78da7F1jWVBn9evNuedlpx26sahG/x1z1/cOKw3kGVnC87xYjEGznHa3Gpaic+IKREWlV0f+8VYHj7+MDYuNrUXroGQISEMenkQTgFOtRcuJyc+h9M/nSY7QTB+2tuD+wfu2Lxjw/W86/VqjwEaDXj0BZ8R+g3wSusXLc7Owj3QaPSyt4H9Ahn29rAWt0erjfRz7RpE11ldayzr5gbh5VOiI0dE+HduPInzVhdOCyMWt9pRWyJnVp7h4tqL9JjbQ2f0q03aEwSP7Rdf1P/erp0wCGZlCRZ+jxr2KFVlKsqKywwSgrZSO8pSJUVpRVU2RJozToFOuGW5IRKLKCgAhbUQOeJl71XLmWYiEkFUVQkKKyvByJ2SAj6yMKI5ZWD0u+3D+nk/isVigoKC6lVHY9F2dNsWJ0liDoUSZzKDe9CtPCepVCIl/ol4UgpTcLVxrflkC6OVsDBm9IvZGEPSkSSGvTWsRRhfu87qil8PPxx96+4Z+0P/H9CoNDx05CGTz4nfEY97O3ecg5xrL1xOYP9AnIOdybuWh3OwMyKRCK9OXqgUKgrTCnEONL2uypxecRp7T3sUwUKSdGdnyCyuY6SffRAM2QQOFsjRInXR/Vcr76kuLDf6FWeg0Wh48JD5+RRNoanGT0WhAhtXG8TWtc97j355lNRTqfSa16sRWtbwRD0/hm+vCWOQTW1TNJEYrGv53qbtApUC/EYjEglS8OfPw7vvCovmTOERZ84cIZeLWShL4Owr4BwBYQ/qNkKSksDW2hYXGxdy5bkkFyTjbCN8N28cvsHqqasZ8sYQuj/Y3cwLtnycApxQKVQmGYqCBgYx9M2hhN0WZrHrD319KENfH2pwbGzbsbjZutHVW9h0uHxZmLPZ2ECfPvpy2s3CnBzh2aqvrSspzY7o69PoM20az94HJ48NIy3lAXy66i2NPctVUGNjhU05bRsqEpcdx1NbnyLcI5xHexqqdawcsRIbFxvmnppbv8YaoTnNP+tLQUoBRz47QtqZNNqMbEP/Rf0b9HoikQiRlfEHLGx0GPJceYNev5VbDysbK0RiETKn6iXH60re9TzEVuJ6zcNbGqaMn+ETw2k7pq1OOUBZqmSx62I6Tu7I5F8mN0YzGwWtccOpgr3mqa1PcTz5OGunrWVSx0k1nv/tHd+afK3anHc0Gg2/jP4FK5kV92y8x+R6mxOB/QOZunoq/n30hhOttKdWZUBL27aC9LjW6AeCYoNILLKIw4/MSWaRfPbeXbzx7lJDbnAjJO5P5O85fxPywnTAGWu7YuTFwrvVxcal3m3SIRJBj08Nj6lKIWM/SN3ArZtB0cyAn0gU7+bvc/cwZ/BoIb+1GQ7EzQW5HERqFTKZGKj9WerfXwj+OXJEzNDnQxu+geW0WqgsjFpdt0iXVm5uxn4+ljl752AlszLL6FcZmUzvbVJTtF9eYh7vOb7Hnrf2mH+RFsiRL45wfrVpXtKnfjzFkuAlXD9oQe+WJmbUB6OY9d8sQPBgKrUWPJ4tbvSrTEEc7L4dElbpJD5dNcIL6krOFYtdRq1Wk5iY2GLGz6zYLPa9t4/08+m1F25maBc12gWHWCSmjWsb+gf2b3RPeQcHSHFZy1rfTjz4t6FBJebvGPa9sw95XsvYVPLq7EWnaZ3q5cjg080Hr0gvkz0/1So1q6eu5s+7/qy9cAWGvDaEsZ+NZdW4Vay+azUajQZVmYrPwz5n3b3r6tJ0Hf8t+o/97+/XRfo5OdXD6GftBP7jBWOIJYhfAbHf6Ix+ZbmC0U+ulFNUVr8IzZpoqvFT5ijj4aMPM2rxqFrLjv5kNPftuK8RWtU4VHz+1Bo1OSU5qDVG/v7X/oSMQ7VXeHgOnH5O96tWFSItTcgJ7WN/mTfHzGZop4PmN1Yig8tfwnXhu+dXvi+RVK6wU1HiU4vMWYajn2OLcZyyNBN/msiUX6eYVFZqL2XIa0Pw6tyw87V5vebx08Sf6BMgWPi00p79+hkaprXvboUCioupN3mZQnJV7/J9qu69ZHjf+QO49dCVcXMTnB6h+lyCoa7C3DIhJ6HKd6n/c/3p/XjD5ARtafPPmri0/hIHFh8g7t84EvclNui1Tnx3wiACozLD3x7OuC/HNWgbWrn1GPbWMB47/5iBxJolSDmZwpKgJRxfdtyi9TZ3ahs/tdF8arGVThHQSmZF6MhQvLuaZ9y42Sk8dxXfy7uRlqsmaDQaYrOEnKbt3dubVZdao+ZGfvXjZ0XnHWOIRCKsbYXUOC012s8pwImIuyIMHEkrS3tqaVvuq33pkiCDue25bbwtfZvizPpPgspKykg/n46yVFnvuupC6MhQ7t5wNwQKHn0aWyG/npXYCgdpLXmo6ouqBHaOhMtVU/zkOu/lhscKjiWdaNg2NDGlpeB57Tgnpi4m6Vjt0qX9+gk/z57VcPGi+TKudaU10s/CtNSB9VbH0c9RJ0tQH6MfCGG9sbGC0W/QIONlnAKcaDe2HV6dGtio00zY9eou/Hr60Wlqp1rL+vXwo+9TfVucVryW/HwotRKMSd72FpwwlxXClm7Q4XHo8IRwTGIjJOh176N73m1Lq8p7pp9L59L6S3Sa3gn3du5mX1qj0ZCdnY2/f+OFudeVU8tPoVaq6fFwj2rLZMdls/Olndh72re473DWz5vpdCYBu3vn0tRTCCGnqogCmwtEpxtObAe/Opj+z/RH5mh5b9/myvil5uU41Kg1jP9qPBKpxOxrqcpU+PX0w6OjByKRCIm1hMiZkdi42qDRaOpkINZoNDqP1QsVjC5RPlHklebRwaODeRWWFYKVff1DXbRc+hTK8nDzFqJRCnPssXG1Qa6Uk16UTs6JHDIuZNBtTjeLRp82h/GzJXl3lmSXcPTN3Tilt8OpQ1syijLw+dgHK7EV8pflSMTl3xeNGo4+DC5dYdTemivt8ZlghC7n7ruFuaKVlTDOeRafwu3iCpDfXkMl1SASwx2xupx+WgeevDzBQ9XXwZcLGRcMjH6eHT3NighupXY0ag1KudIihtSDHx3EwdeBLjON5+xRKmFv+SM3ZIjhZ1KpEKFaVCRsGGoj5uuCUglv9Q7nam4XvL23VC2QeQTceoJYQu/ewtrn6FEYPbpq0UCnQCQiCaWqUlILU3XGaKBBI4Sbw/hpKbo/2B3XNq54hHs0qMRWaUEpmx7ZROd7OhPQ15wEpK200nCcSjnF8JXDWdRvEa8MfsWsc70iveg5rydBA2+NqGBTqWn8LM4q5vve39P+4cF8dyyK0aPhkUeEz+7Z0PKiz8ouxeF/+QC2YiFxbkZxBnmleYgQEeZmeqR/kaKI+9bfx5EbRzj68FGDd6EWrfNOTfKed/99t1ntbwloI/1CQgyPt20rqKtlZsIHH8CdHTwInxSOqlRV72umnExh+cDljPxgJAOeHVD7CbWw/YXtxG+P5+FjD5u0Vnb0c6SDXwf2lAeKamTCQ+Fu6245Z2xlCWztCe3mQ/vH9MelLtDra3CNqnKKm0zYo0zJFwIVSgtKWTV2FaGjQqsoVTRn5HJQSu1wbOdjkiqStzeEhgr5JQ8ebLy8xq1Gv1ZaqQVlqZLijGIc/RwRiUUWMfpt3Ch4m1SHSCxi+rrpdbtAC2T27tkmSYgB+Pf2x793y1m8y/Pk7H17L2G3hRE2Kqxc3rMBIv2KEoTN74pa8rb+MK0YxBL8ym18VvlGjH7n09n16i48wj3qZPRrThz+5DBKubJGo19gv0AePvYwbm2NaFg1MkUZRex+Yzf+vf2Juj+q3vWVIUUttsLRVdjY3pmwk+3x2xkYNJBx7RrXa9veHuxLBfe9uOw4g89cgl0atS0NzZ9T/iT5eDILry1stGtKrCV0vrtznc6VOcqYtHKSgSPUyPdH1qs9IpGIwH5CPrND5cGHTk7w5MAXeGHgC+ZX+F8/EEthrIW8EPsuBys73MrnCNlZIvp36Y9SrUSlVhH9XTSnl58m4q6IFiHbfW3vNZJPJNN1VlfsPGrXnFQUKRBbibGSNe+lR971PBJXH8W+vR1OTm3JKhG8ap1kTnqDHwjaiYP/BkxwBgyYYPCrTGYoyQjToeMIIX9GXbDXb1RqJUnlckFq/qvxXyFCpMsF10rN7H1nLyqFiqFvDDV5QyXrchY/9P+B3gt6M/SNofVuw7539uHbw1dn9JMr5SQXJOPn6IeNlQ0nTggOYs7OEBVV9XxXV8Hol51tmPfGXDLTFZxNGUOWPITuldW9474VcqYO/BOCptKjB6xaBRcuGJcVtZZYE+QcREJuAvE58UY3OlupHxKphLZjGl6e3kpmxX077kPqKK22TGZMJvvf3U/nGZ1vacn8ViyHRqNh7//24t/b3+hz/saeN8iV5/LqrleZ1WUWwS7BJtctsZYw/ivzHOdudbLjslGVqTh4RIJCCf/9B/ffL8xvWiLJYYMoc+jMfZ2El6E2yi/IOQgbK9Pn/GXqMi5kXCCpIInJf0zmwAMHDOeW1B7pdyuw6/VdHFt6jIePPYxrG1c0GkHWHKoa/aRSeOUVeO45OHkSAgK68fCablXqrAsO3g4MfGkgIUNCai1rCopCBUq5EkWBwiSZYrVKjVgipqhcUKbMWliTuNlacA+qMB6UxaBWVP2s3aNVjwGett5QAqlFwqLY2s6anCs5yHu2DAUmLaWlkO0fycA3InEwMRajf3/B6HfuXD287sykVd6zlVZqIeVECp8GfsqhTw6h0QhyS1B3o59WtunKFWHTpZXa8YnyaXERU6aSl5jHoY8OkbhfkOIpKKgQ6Wfq28UUXCLh9hgIe1h/TCSC8omm9nlXZgjeagm5egmmsFFhzD01l7DRlstZc7Myfd10pvxes6yXjYsNfj396p3Qub4kH09m40MbOf7VcRK2J1ikzqL+I7k4eC4ODsKO3e6ru3lv/3tsurzJIvWbg1gMHmLBCJ0jzya7RO/yqFaqKUwrpKy4rNHb1RC4tnU1W+e/Mmqlmi1PbuHABwdMKl+aX38PNGOb4mUlZXVSRSgrKUNVJnhlauUVneueHhC8hggJyS2FWzdw6qCT98zOhu2zdrBn9h7aubej9+O9mbF5hi6vSXMnZkMM/z39H6UFtT8n0b9F857De8T+E9sILWtYvDp50W7ZU2SE9DKQl3W3reTwIpaA91DwHla1EmNoNEJ+jOqw8RAiU+uCIgcyDoJS2BXQ5pPOyhJkp9q5t8PO2tBwe3zZcY5+ebRu12vBnF15lktrL5nlQe3SxgXPCE+LKVA8eOhBxi3VO9mcTj1N2OdhdPhSiHbevl04Pnw4SIwEamujBHJz69eOtAwpXxz9gb3Zr1YNmA6YJMwnXaIAYRNOIhFy8GZlGa9PGw1R0akMIG5rHN90+4aEnZaZx9yKpJxKIfNSpu73hF0JDZZCQiKV0GZ4G/x7Ve9IoJQrObPyDCknUhqkDa3ceshz5Ox+fTcX1lww+vnzA57X/f+tPW81VrNuWQL6BDBm8xOcKhNUmkpL4US5j93xZcf575n/mrB1lkWthoJSGSXOPri4Cy/dy1mCBcpcaU8XGxc23bMJFxsXjiQd4fuT31cpozX61RTpp1Ko2PHSDqJ/jTbr+s0FRz9HvCO9dYo+V64IeYylUuhkRBgsLAwWLRL+v2ED/PuvZdrh1taNEe+MqFPAwfbtcPas4bFxX47jsXOPmZyX9Nfxv/JJwCc6o5/SujzSz66OTvhlhaCuFAHp0gnujId2jxk/R6OBSut6X0dh4zBLLmyciyViFqUsYsySMXVr101Kafmyrdb87hXoX55OOTa28fYJW41+Fqaxcxq10vDYuNrQ+/HeBPYPJCdHyIMhEoGnZ93q8/AAd3dhghAXV325zEuZ/DHpDy6urSH53y2AWqlGnifX6cLXhqJQwe8Tf+fgR3XIfXMT4hHuweNxj+vkjfLzwU7RhgC7UHwczLA8q5VQeBUUudWXEYlAXGlDOu8CXPkBPx/BQl2YFMTqqWs4+ID+72vrZotPlE+dpRRFIhE+Pj7NYvx0a+tmUrLmspIyijIaLo+XKVzbd42YDTHc/ffdTFw50SJ1FhYKPx3K1TRTCoQNG7OeRQviYm+PTCHo1V3J1ueZvLDmAh/7fMzlfy43SbsszajFo+qdjF1sJebimovEbIiptWxZSRkf+XzEpkcta8w9/+d5Pvb5uMZcP9Vx8vuTvGPzDon7E3VGP0dHDXJlHb1nen0J3T6o27nG0GhAnomLvfAlUav1xkkA326+tBvbzuKRbk01fvZ/tj+z987GKcCp1rIe4R50mdUFB98Gzi/RCIitxJRYOaGU2uHkBFnFggWjSk7J0ixB4tMU8i7Aame4+GHVz9QqIR9fcXLVz0wl7jvYNgByzgDoDNPVGV8Ajn11jCOfHan7NVso887NY8bmGWadI7GWMGfvHHo8Ur1CgDl4hHvg0UH/vCXlC3lE/B39ycvT580bWU1wtSnSYKagdYL0NuaPYuMJfb4Fp3agLsNalaGLKkyoxnYX6lJVSQKEzSJ5rtwkBwNzaU7zz/qw44UdfN3la50zz+nlp9n9+m7yrudZ/FqmrNm8OnnxQt4LDHqpmjwXrbRiJjInGY+eeZSBLww0+nn/wP4celDIsfvTmZ+Iyax9LlyRlFMpLItaxukVp+vb1BaDsfFz49yNrLtPyCG8Zp0ERGKk5UG/B8p9Di9vutyinIoKCkCWn45YqcCx3LcnNltwcmvn1s7s+sLcwnhrqGCYfnnny+SUGIb0ad/hRUXCvqQxRBIR+9/bT8zf5j3nzYWec3ty/677dUoj2jzGvXuDXTXiI/37w733AhoNGx7fxqYX9zdKW435uZ4/D599Bm++Wb+5mG93X0KGhuiMfqViYWJfxRHRFBS5sN4f9t9VtdEiEUiMRO9f+gzWuEKB4X5LgKswMcwtSzO/Hc2I0lLwv7iNa2uqSVhthMBAQWXDyowI4PrSavSzMGJx65+0peHZ0ZOxn48lsH+gboHr6SnkWqkLIpE+2u9iDfY8iUxCzMYYA8/MW5H0c+ksdlnMvnf3mVTeytaKuC1xZFzMaOCWNQ4SawluYW44+AibpQUF0PfyNnZNukJbNzMkcZI2wYY2cGND1c/ivodjj4HcyLN29Vc48hA+dlcQiaC0xIpR/lPo5tsNsUgY7zQaDaX5pZRkl9Sli4jFYnx8fG768VMpV1KUXmSSAfqzkM/4Y9IfjdCq6un3VD8WXltI+zvaW2RDS6PRYH1wD07psXqjX6Fg9PN18K13/XXBwcG4xKdnhCc95/Vs0Lw1zZG5p+YyZ9+cWsvJc+V0nNQRn26WNea6t3fHNcy1ThGYLsEuhE8KxznYWWdME9vnYvuOLa6LXVGoqln5NhbX18JaTySpG3SL/jwj+6mWzv3cVOOng7cDwYOCkVjXnvPRt5svk1ZO0smzNmcKUwvJTcgBtdog0q+K0W/3eNhoooe3XRC49QA7I1qLOadg32SIXVr3RvuMgKjFuvorGv3Op5/n1Z2vsvSoYf13/X4Xs7bNqvs1WygSawnOgfUJMa4fqjIVRelFuqhngKQCwejn5+jHrl2gUkH79hBUTfopSxn9AtKf5oGoZ4wb/SoS8zlsbE+/9sImb7VGP1fjRr/QkaE8mfAk4RPC69dgIzSX+Wd96f9sf2776DZd9MCglwexIGZBgzzLa6at4SOfj2p814mtxCZHMrTSiimIrcR4d/GuMbVC34C+3NH+DtQaNa/vft2s+u297ClKL2oxCiKWwNj4mR2bTcaFDOLjNRw9Kux7LVggfHb0qGCkmvDjBBYlL7L4fNhUtOt4ZakStcpE56wayEgsofOerwmN2aLbH6xrpJ+Web3m0cmzE1klWVWeVXt7sC7PxlKdxKdYImbB5QWM+6pxU280BWo17CvfJhw6tOay06ZBl64iXJIucP6P8/W+9q+3/1qjg+wnn8BDD1VVVtC2V6GANWv0x9VKNSd/OMmFv4xHLFdmxLsjmPzLZIqLhd8DnQOYGD6RfgH9TO+E1kFR6iIof/nepv8scQ2cWAglqcbPtfEGt+6gNhwXg9yEiWGBRm/0S9iVwMnvT5rermaAXA5eV4+Rsts8J/NBg0RIpa1Gv2aLSlX/hKCt3LzUN5+flvDydWtNef1cQlx4Mf/FW94LUuooJeqBKHx7mGZUEEvEvCx/mQk/TKi9cDOgMK2Q/KR8NGoNGo0+csSp9uAKAe2E2rULdHgKnAT5JwMZsdTtcHUVWBuJwgi+BwZvwNolULdRmFJJjUcpV/K+8/tseWKLyf2qiEql4sqVKzf9+Hn94HU+8v6IY1/X7s0T9UAU7e+o20TfkjgHOaNRa4j7N47YzfWT1ivNK8XtzG5cUy5UNfo5Np3Rz660qiSYdxdvxn81vkZ5qebEvnf3cerHU/Wux97L3rTE4L6OTF41mZ5ze9b7mhXxifJh7sm5hI4INfvcDnd2YNqaaTgH6o1+ZTL9IkRqzAOxOs6+BkcfBUsaCp07QdhDYB+sG5+XHPsQn498eHH7i0T/Fs07tu9YXOKyqcbP4sxiA8PDrcLhJYcpfOdzZMU5ODujy+lXRUrHexgETDStUmsHGLkLQmdX/cyhDfT5EYLvrnuj3XpAxHO63H5aec/MTMFZ4u19b7Py7EqDUzwjPFudJipxdfdVko4l1WmjsrSglPWz17PvPdMc2KojPTqdj7w/4uCHerWF5AIhCtTP0Z9t24Rjo0ZVX4dWGqy+8p5eqn/p6Lm/9jWRY1tw7Yq9fyQA8fHGi82InMGxh4/x2ZjP6tcwM2gu88/6EjoylD5P6BOFenTwwL19w+TgDhwQSMfJHWuda2RfySZuaw2SN620YgalBaXIc+VVxufM4kwe3/w4R24Ikev/G/Y/AP44/wdnUs+YXL+TvxNPJz2tU95pRT9+KsuUKIqE+fS9W+/lkeOPsHq18P0fOFAwxHh6ChvkJ08KaxFbN9smi7De8sQW3hS9yWKXxVz8q/6KWvl5GpLbDUbVTu+YsrDvQpaMXsKI0LqlEbASW+nehV8d+4pz6ed0n4lEpjnvuLdzx8699pzbzZET353Qzaeio4W/g6Mj9KhFUEEkgm7d4NKAB9A89HDNhWtBo9FQkFxAUbpxZSelEvbuhfR0IaelFrVaH/UKsGULZJTHKogkIv57+j+OfmFeJKxWiWlM2HjWTV/H8wOfr/kELZlH4b/+oCpXzRm1H9rN0ydeTtoEsV+BpBonnZC7YcROcOlscDjMW5gYykU5lCqFPcdjXx5j4yMbLWJov1koLYXoEQsZ+aV5+85jxqhQKgsbqFVVaTX6tdJKLfz9wN86Y4bW2FHTArdIUaSTvKuOipF+1e0diEQirO2szW1ui8MtzI0JP0yg3VjT5RFakkzPvnf28WnApxSkFFBaCmXljjSOlVPDxH4N5981fKAyDsH2QZB3ERxCoccn4NEH0vbAhlBI3yuUG/AbjL8AEiMeJy6dIOAOsHbCt9yuszf2FJ8c+oR/Lv8DgLWtNd0f6U7I0JA697OgoKDO5zYWdh529JjbA9/utRu4Rr43koHPG5eYaQyOfHGEi2sv6ha/a2euZfcbu+tXqVTK+cGPktp2EPblqaVSCwWjS1NF+tnbg4O8A6F2UbjaujZJGxqDgx8d5OwvZ41+dv06bN1a/bukImXFZSTsTGj2kdDaCLoSifD8mS0vm74X0nYZlyqpK87h0Oc78BygyzVYXKIirSiNlMIUnPydaDO8DTaulvfsa4rxc2nEUlYOX1l7wXI2L9hc/zHoJiBkaAiKnv0ps3E0jPSzrRTpF/UedP+o/heUuUPYHMH71kJUjPTzshfyJWcUGY4JKoWK/Bv5KEuVFrtuc+e/Rf/x2+2/1elcqYOUK1uvkB1bv/A6mbOMnvN64tdLLzOujfSzUfiTmCjktBlUg7+gpSL93o8+zxu7/6090i9gAozcTXCoLVB9pF+gcyA9/XoafZdfWHOBMytN36A3h+Yw/6wP+TfyjR5XliqJ3RJrcSn6Ia8N4fZlt9dabseLO1g1dlXrGNOKRTi+7DiLXRdXkY//8/yffHnsS+b9Mw+Arj5deaznY3ww8gPauZsnvdiS9hcsRUFBAXmJebzn8B47Xt6BxFpCUpLeqDFtmmA/GDBA+H3/fkFNJC06TWcobAiUSnj5ZVi8uOr6SOYkw6WNC06BThYxQJRZ25HcYRjSyA66YwODBvJk3yfp7NW5hjNrZkToCCZ3nEyoayj5pYbjuNZ5p7pIPxAcx3Ov5db5+jczZ1ac4ejngmFMK+05YIBpSmzt20OZjSOxV+pnChGJRMw9OZdpf00z+vn164LyAhiu08+dE5yuHB2hc2fhWf39d32d92y8hwnLazciyXPlbHp0EzEbY3TyntVJm1ZLyhbIPgEFlRxSS7Ph+JPQ80sYfxGk5u2xhHi7MuLsNSZEy5GWGwwHvjiQ+7bfZ2YDb17UamFfVim1wy3IvPQVLi7Qo0cNX14L02r0a6WVWkg+nkzaGSE0ubZIv/SidLou64r/J/48t+25ausMDRUW5QUFsHo1lFSjiliQXMCp5afIvZpbjx7ceqScTOHiuobPhahSqEzONVhXQoaF0Gt+L+y97CkogHSnrezoEszcfyu9NI89Bmde0XvmAGQdFl7kVFqkaCXESsvlPEUisKslIkol1xn9dl37j0X/LeK3c/rNrzu+uYPuD3U3v4PNCO8u3ty+7PabXqJOpVCx65VdHPjgACKRCLFEzITlE7jz+zvrVW+xXEyJkzcKBzfs7EClVpFWKIyNDZLTL/e8ICmhLK62iIMDtE19gfeCT/FYL32C6bLiMlZPW83hzw5bvl1NwCPHH2HCj8YXAMuWwZdfwvHjtdeTl5jHyhErOb38tMHxjXM38kW7L1ApVJRkl/DjgB85/2f9ZU+MkRadxtqZa0nYVc3ObzWsm7WOY18JUbbaSL9CUR3lZUfuhtEN92xojX4ylZD8N6M4g+DBwcz4ZwZBA6rR3GtmRM6IpMPEDrUXLOfypsv1jja+GWg7pi0Z3UahtpLi5ARdvLswNWIq3X3r+f4rjIf9d0Piav0xtRLKjG/Ym83u22HfFMC40S+9KN2g+OElh/k08FNST1cj6XMLMnLxSEYvGV2njV+RSMRDRx6qdhw3FbcwN8Z/NZ6wUWG6Y9pIv/Qrwjyuf390jjnG0Br9atosNIW0NBFFZS61G/3Kaeu8l3s6v0Fqihq5malY9727j12v7TK/kS0MpVzJrtd2oSg0bbM842IGn7X5jA0PV5X2v7TuEr+O+7XBjKm10f2h7tz5/Z0Nvo5q5dbAq5MXUXOicG1juDn9y9lfALi3y726Y0vHL+XZAc9iZ23eDnlOfA67XtvV+l6shEajIXJGJL7dfFEo4OuvBQNHnz4QEiKUGVjuB3v0KBz/7hTLuiwj9VTD/R0TEuDsWcHIeKNSGvER747gyfgnWRCzgMh76u9QVTnfvSX59vZviZ4XTf/A/gbHTXHeWTVmFStHmO6c15yY+udUZu+ZjUIBB8uFD4YMMe3csDCQqBQUX0ok6WL959jVzQmvXtX/Pz0dTp8W/q+V9uzXD+6/X/j/9u2QXJ66O3hwcJVxzBh51/M48c0Jko8l6+Q9rWxKzVOjiHwdJiRUdSxM/AMufy4ogTmGGT9Xy/n3IeZLg0NubiJsFUGoSmW6tvn19KPN8DaIJS3DBFVaCiJVGTYF6WhKzJzUAkOGWGh9ZwIt4y/eSisNyLyz87hvh2BgqcnoV1xWzJ2/3cmVnCto0OhyUxjDyko/+fn5Z3jgAVi1iiqL4OTjyWx4YMMtLX8SsyGGdfetIyfe9N2JvW/vZfVdqxt0IZkTn8O3Pb/l6FLBy6ihrtVxUkfGfTkOibWEggKQWydRIk3URRfouPMK3F6uF6tWCfrc4U/BhGtCBEpFHMPgzgRBeuzkIiEisCY2hcPOUTqjn7TIeN6VVvRcXHeR3yf83iQedhKphMcuPMb4r8frjnW4swPeXUzcmauG3AwFkjI59nYaRCIhwkWlUSFChLdD/eo2yp7xEPMZlGZVW0S7wCqspJAgkUm4sOYCKcdrjrpuLriGulYrtaeVBLlggvy/Wzs3Ri8ZTeQMw8m9nbsdxZnFSKQSMi9lkn4+HXmu+RNYU1CXqYn+NZqUk6bfG0Whgujfokk6koRCoX9XFqhNiPQrSRPGxMqY6bVoEqdfhEOzdUY/a0W50a+oeUdWGmPMkjH0X9S/9oLlzIuex0NHHmrAFjUeFWW27+t6H39O/ZN7Iu/RFzj7GhycZZ58rJUjXP+r3FGnnMzDsMYN4r6rf6M1KuEfhkY/T3vhGS0qK6K4TO9gEdA3gN5P9MbWzbb+124hhI4MrdcGoXOQPn+aJXMZJeULkX7XzgnRfyNH1lzelAiB2lBc30kbm/8Qi5QmG/0cc9YzI/JNvO3juXbNeJnvTnzHk1ue5GruVYPj45aOY/ra6XVvcAvg6p6rnFp+ir3/26tbe9SGnbsdne/pTPcHqzolhE8MZ8gbQ4i6P8pibby47iJrZ64lO672MNKw28Lo9kA3rG1bVW1aqT/txrVjwo8TcPDRW17ic+I5dOMQYpGYuzvXQyK7nOy4bPb+by/XD16vd10tCdc2rkxeNZnQ2yN48004cwZkMpg5U1+mfXtBWrykBIrcgxjyxhAc/SvLFlmOxET9/w9Vs82hNdbUN09j9OtraHNyrc7ZJj4nnt+ifzOQ5Kwr7nbuyKz00orauYMpzjtdZ3dtsQ7Zjn6OuLd35/hxKC4Wnq1OnUw7194eQiTXCT+4nIPLa8i1VAuJBxI59eMp5HnG18taVQNtyst//xUi/7RGykGDhJRPvXoJUWO/VRCSKM4qrrZeLV6dvXgm/Rm6PNJXF0U4ZcMIZG/L2Biz0fSOGMsn3nYuDPkH2j5S+/lx38KV7w0OSaV657OKz6hGo2kxjj5yOdgWpNN5z9ec/dEEz+tKODk1nqx8nYx+SqWS7du388033+gkMZKTkymsvOt2C9Ia9t8yEVsJX5W08lyklY1+ao2aWetmcSTpCG62bqyZuoYHuz1YY52PPw5PPAH+/sKG9e+/wy+/GJYJGhjE1DVTCZ9o+eT1zYWUUymc/fmsWROyPk/0YcpvUxr0pWLvZY+qVIWyREnCzgS+DP+StOi02k+sB/n5oLAWvPGrGFkcQsGpvbC5vf8uiH5DOG7jZbwyiRTyYyBtN9xYX/OFvYaCe2+d0U+dVdXod/Djg/w55U+z+qNFJBIRGBh404+fO1/Zyd9z/jY4tmIFLF1aVTokLzGP2M2xFCQ3jWyUk78Tvt0Mo59UCpVJGzHVcf7Xs3Tbuhj3POG+e9h5kPBkAocfOoyV2AQ9DXMZcwKGbAL76iMrqzP6iSViXlW8yqSfJ1m+XY2MWqUm73petZ792r7HmhBEJZaI6ftkX3yiDF9iI94dwfM5gv5/YP9Anst8jq73da1Xu6vDu6s3z2U9Z5bBSOog5eWSlxnz+RidwUUigcxSwXBo1OgXvwJ2joZ1PpCyVX/80meQuqM+Xaie7OOQtkOX009coo+iKskp4b9n/7N4FHpzGT9ljrKbvo2m8NuE33E7+i9QQ27d7JOChKw58rE2njA5DaLe1x8TS8H/DnCxwHdx2BYYvB7Q5/TLyQFbsSOycumdisbp4MHBjP1sLO7tGibvV3OjNN9M7+lqyE/K5/eJv1eJtjaVC2susGb6GnIS9Lsos7rMYrT3bCS5bfHygi5daq5Du1lYUKCXjDcX9dm3eXHgZOzsRKZHN7Sfz9dXT5FeFFytxOfXx7/m86OfE50WbXA8sF+gSdLq5tJcxs/sK9msGLqCtDNpTP51Mv2fMe39ae9lz6SVkwjoW3VTz8rGiqGvD8XOw3I5n1JPpxL9azQiyc3992zl1mDV2VUAjGgzAj9HP4PP5Eo5K06v4N6195o8tgf0C2Du6blEzY6ydFObJRXHz5ISeOMNIbrOxgbefBPatKlYVi/xeT7Xn6GvDzUpmqmuVDT6Ha4g7HHqx1PseGkHpQVCnrGLay/yif8n3DhSKRzQDEqzi7BWFOlSr/x35T9mrJ3BC9tfqHOdlVFr1Hx66FOmrZmGRqPROe/UFOnX98m+DHyh6VKNNCR5iXnIc+Xs2SP8PnSoodhVbQT18eF6x1EUuAbXuQ3Rq6LZ8OAGyoqMT6S085zx5T7YR44IOf7y84X1Q2S5D9m95UHIe/YI0X5Xtl3hQ48Pif41umqlFRCJRNh72qOxEZzzrKwgR55FmboMB2ktE7PMI7BjOORWcw2RGPzHmfZHHbZVyEteiTS/nzgdcj8bLwprpujfonlb+naLUH0BIdJPKXMgveMQggaZp+IjEokICDBibG0gzDb6Xbt2jcjISCZMmMD8+fPJKHcxX7x4Mc8884zFG9jcEItbgydbEunn0zm/+jwl2SWUlupfrJWNfs9te461F9cilUhZP309UyKmYC2xRqPRsOnyJoavGK6TwdNiZQWjRsFXX8Hd5c5nlT1fbd1siZgSgYN3A+gFNBOGvDaEl4pewqOjR+2FywkZGkKnaZ10xlpLoFFrOLPyjM67Vuog5dGzjzLguQFIHaUUZxSTddkwIunG4RvEbIyp13V/vu1n9r4t5N4rKIBSa+E58rKrYMxTFkPhVVCVgqYMlIVC4l1FXs2V23iDtUPtXjy9l0H3j3VGP3mKEOafVpRGkUIQEU89mcrlTZfrlJtDLBbj7u5+04+f1w9cJ2GnfqdKoYA1awTPraQkw7K9HuvFK6WvNLoUqLJUycW1FynJrqoZ/G3Pb/ll9C9GzjINKy83MgKjkPkKKw2JWEKISwi9/XvXuU6jKMvbLnMH//KZcjURM/b2kGt3grdy2jPgxwEGn1ny+9+UFKUVsSRoCTtermqoUqv1Rr+4ONPy+oEwnmnzWBjb7BBbibGyaQBDLoLhsS7RQxJrCTbONgZRVmlFNeSUvPA+pP4H3iPAutw6U5YPp56BS0vq2PpaGPIPTLyOi0v570V6eU91mZpDHx0ifptlI6SbYvxMP5/O6qmrid9uel/yk/KJ3x7foDlcGoP08xlIi3MRiYTxJ0+eh1pTKSfM0E1wex2MuzI3w989esPgdcJPC+LsLMxBNRrIyxPpov0qS3y2IqBWqfmu13d8FfFVvZ3JZE4yrh+4Xmd5uPRz6Zz/8zwqhd5D+IluL+N1cDm2ZYGMHl37Ho2Dgz7vTW5unZpBvNNiPjuyHC9vieknOYZh6xeFSmNdrdFPq5KSkFu1QFlJGUq5ZfO/NZf5p8xRxoj3RxA5M5LIeyJ18ljVPY8lOSWc/eWsScYMlULFyR9OWmRsHvbmMF7IfwGXYJdayxakFPBlhy/Z+crOel+3lVY2PLyBna8aPktb4rYAGI3yU2vUPL7lcVZFr+LA9QMmXUPmKMOnqw/Wdq3RqaAfP/e/d4D3Bm7m/DkNdnbwv/8Zj7rSGv2OHBHymDUkFY1+sbGCsgHA+T/Pc/SLo7oIY/cO7th725ssmWwM5yfu53Kfe3UOMLFZglGjvXv7OtdZmSvZV3hhxwusubCG5aeXWyRivzmztONS/pj2F0fLg95NlfbU0qGbPWlh/UksNU2qID0dZs+GRx6BL74Q8gh2mTeAuzfcjYOv8X1arbzn0KFCRJ9KJaTkAEGGXVI+fQoNhY4dhTn55cvgGeFJtwe74dGh5r3PnIQcMi9lkp8rrEHs7CC7RNisdrerxWEv8zBk7AeRBdb6Tu2MqufkOOzjhsdKTqQICibOgc60G9euxSiIyOWgsHUmr/tQs1N3iMVi3Nzcai9oIcye4T755JP07NmTnJwcbG31N2zSpEns2NFAntPNCJWq8cI0W2l4Lq69yJppgkdtevleiJ2doWb3ieQTfHzoYwCWT1jOoOBBBnX8b+//2HV1F0sOLzF6DbEYupY7cadVEyhWnFmMquzWfLZEIhHWdtZNrv+sVqnZ9+4+Dn54ULfZYiUTXpT+vfx56sZTREyJ0JWP+zeO1VNX18tgq5QrST2VqovOKiiAUivhQdTm4QEg6whsaCNIgElsYOhmGPAHSJ2NVatH5gYj99Su1V2O1tgtz3XB1UZ4uWuj/SYsn8DL8pd1fxNzUKlUXLp06aYfP+/fdT8LLi/Q/V5xsyy+0t63xFqCSNz4ns43Dt/gzyl/cnxZVZmBqDlRRM2JQq2sW9Jym4hQrnWdgJ2/BT0zC67A/ulwtTyDdd5F2BACiWuE38sK4b8BcOxRo6c7OIBYY0UWsVzJvmLwWerpVLOMEjcrEpmEvk/1JWRoSJXPior0hr6iIn0+gJq4sOYC7zu/T/z2eMpKyviy/ZccXtK4uQ+L0os49/s58pNq17PPS8zjA/cPiNkYg1Ku1Bn9nJ2hq3dXxrQdQ4RnRNUTR+2H247AiO3gVe7pauUI485A1/9ZsDcVKI/s0sp7qgoEY0pxWTEaJw1PXHmCke/Xor1nJk0xfuYl5nFhzQWT7p+WUz+e4udRP1dxjqkJS0jMbnlyC7/d+VvtBU1k4o7HudLrbhwdhflb6OehWP/PmpjMSg4+VnWIntFo4NoflpHzrEzhVbj4MeRGIxLpJR4zM/XziYxifaRfaX4pf075s9HHhpsRVamKjlM60nV213q/12WOMhbELGDs52PrdP7QN4byatmrBhGY33wjbPwFBMDEibXXISqIZXLnpUDNUQI1EZ/XiwPXp5os7aklLKQEb/uEWo1+leXjL62/xLt271o8Urq5zD/tvewZ+PxAggfpoxJWT13NiuErjJbf+/Ze1s1aR9yW2tNDnF11lo0PbeTENydqLWsKMkeZSd8TO3c7xNbiVgNKKxYh9p9YbhzSR2tpNBqdvGIf/z5VyttZ2zElQshz+/OZn02+jqJIQcbFlifZXhe04+e5vy6jvBCDtVTE228LBg5jhIcLUYDKzFx+HLyCE99aZswxhtbop92uPnJE+Dlj0wzmnpqrcwz16uTF/IvzCR1RfVqe2igoAEQiXaTf5ezLgGWNfu3c2/H2sLcBWPjvQuQyIVKgJqPf0aVH+XnUz/UyaN6MaDQaes3vheegcJRKQbJemzvSVNqX35rLl01zmD1xQjAcp6TAf//Bxx/Ds++44De0g1GlgNxc4Z9IBEFBMGaMcFyb326Q4XYx2qCv1FRBrenO7++kzfA21MS+d/extONS8tKFtZKdvUZv9LOtxegX/iRMTALnjjWXMwWVQti/qZSOxU0mTBBTC4QN7qCBQdz9990E9m9ch/iGolQIFkYmq7mcMVQqFZcvX7Zsg2rA7F30ffv28corryCVGkrWhISEkFQ51KGVVpo5kfdEMnnVZDzCPQzy+VUc27NKsoj0imRChwnMiJxhcL5IJOKlgS8BsPTYUnLluUavozWmZGQIkRsVOfjRQT70/JDMi5lVT7wFSD2dSvIJE3ayK3B502U+8vmIS+vrrtNdGYm1hGlrpjH31Fwk0qqezVJ7YUzUGme9Ir1wDnLGylYwgtXFO9zKxopnM55lwvIJgFbeU3hxGsh72vhA+NPgXh4NILYWvG4sRf5lOPIwdvnbdZvZ/naGGzMSqaRe8kjyygktb1IqGjUrTrTjKu2rqBQqEnYlkHqmcZO9e4R7MH7ZeDpOrjqJ6/dUPwa/MrjOEXDaiDKtRvvWuK28uP1F/o37t67NhbI8SPwT1OULEkUuiG2EZxqESFSJDCTGvcIcHMBaKexeZ5dkG3i1b316K6unrq57224S7NztGP3JaDpOqnpPK8uamiLx6RTohF9PP8RWYnKv5iISiyyaX8oUru27xl/3/MWVrVeMfl6aX8qBDw+QGZMJIvDv7c+ZFWfQqDU6o5+jIzzT/xm2zNzC+Pbjq1YiczeMkFIWCy9v5whwjbJ8pwBKs+HGRjxshH6V5DnS1bsrQ4KHIFfLcQ11ReZUh9VBLTT2+NlubDteLXvVrPxm7ca1Y9xX43D0My2Hy5VtV1gSsqTeOY3zr+cbyBqf+/0cq8auIivWdOOjQX0VIk2VaiU5JTmoNWpcbcudIQqvwrU/odi8eQsgPJ/n/gdnXxGkt3eMrD3nrqkUJQhRrmmCBE/FvH4/T/qZywsuM7zNcF1xK1srLv19qcFly5sD1nbWjHh3BAOft4xMltbLuSi9iOLM4lpKV0VsJdYZVbbuyWXzoXjUYjlPPSXkUTFK0TW4LBj6OPkUM8Mfx8fhSt0i/crySU8TwjTMNfr1zYnitSHjSUgwvtFWndHPrZ0bne/pjKOv5XNA3ezzT7VSbdTx08rGCqm91Kgj1+BXBnPbx7fRdmzbWuuPnBHJ6E9H0/3h+uV+Ki0o5fyf58m7XovKSDkSqYTHzj3GoJcG1V64lVZq4emkp5nxj34fJqM4A2uJNVZiK9q5G18T39flPgD+OP8HcqVp48DamWv5OvLrW9YZuzJyuZyAN+7nwuBHiYiAdjVsP4hEwtxDIxKTdTGNovSiBmoTOmf9ceOEn1qJT7GVmGS5G/ffD599JpStz/5Fwq4EivaeQFxWWiXSr52bBfdigKf7PU3/wP4UKAr4LPZxoGbHnZwrOSQdTWpxRj+RSMSoD0bhd0cPAJ2x1RxCQiDg8k5C135E0rXaw061Zo5evWDyZHBxUiPPKWHfPuPltVF+vr6CoXvgQP3+ibMzdO5sWF67F5xierp7wieGM+jlQZRZCXNKqUM+SrXQFzdbI1FkudGwuSvEleffs/E0/WI1kbIF/omAJMM8gt72wgQxvahlriNKS8Htxhl8tq4wy6FVS2POPc3e+VOr1Ua94W7cuIFjXb5xrbRyE+PW1o3IGZFI7aUGRr+K3BZ2G2fnnWXd9HVG67ijwx108uxEgaKApUeXGr+OmyC1o1Tq5Qe0eHX2Imp2FBKZGRI6LYitT2/l51Gme+AB2Lja4BbmpjO4WQqvzl7YulYfkh67JZYPPT4k+XgyTv5OzNk/B+9IbxL3J/JN92/MioqoiHYyWm2kn3NH6P6xxSXAdKjkQoLejEM6iU8vKyE6ULsxU5RRxJVtVyhMbZm5XeW5ci79fclgM6OmSD+VQsXK4Ss5+OHBxmlgOQ7eDvSc2xOPcNPlcE0l4dttBJ7brFvUbI/fzvsH3ue/K/+ZX5myfKHn1h0mZ0CbWcLvnv3gjlh9ZBbA8B3Qy/jY6eAA1iphYlumLqOoTL+A7Pd0P8Z8Psb8tjUj8isNKaYY/QL6BHD/rvsJHRGKZ0dP5l+aT5/Hq3pBNyTBg4OZuHIiYaONRxkn7k9k+3PbubL1Cs6BzszcMpNpa6ZhbWdNXvlXsNp8agDX14O8gif2kYdhU0dI3S7IIDcUudGw9068lcLCJz9PxOlHT7N79m487DzIT8onJ75laPGIrcRGHWCqw6+HH73m9TI5+l1iLcHeyx63MPPlT+S5chIPJKLRaJi+djrzL8zXfZZxMYNr+67VSV6mILmAy2uikRbn4uQEOSU5aBAsF7oFdvoeODAdsuvowd53OYw5BYXxkHlQcOKxBG49YOReCBGSh2iNfpmZEOEZQTv3dthY2eiKS6wlvFL6ChN+mGCZ6zdTCpILDKQ0LYVGrWHtzLV80+0birNMN/zdOHKD64euA5CXB2/+voFdXcK4NuAOnee6UU48Cccfh9xz0O1Dfk7ZQ2phWN0i/c69zSyZE55218w2+knaP8S+6zORy9GtqypSndHPq5MXU36dYjTivaWTsDOBD9w/qBLlOHHlRGb8M8OoI5etqy39nu5n0ma2lcyKvgv7InOsn0NKyokU1kxfw4XVF+pVTyut1AWRSGTgmOll70Xms5mkLEpBWk1+3SEhQwh0CiSvNI9NlzeZdJ1O0zox6OVBqEpbjX5azp8Xo5LaVjFkGMPNDcpsnei//jkGvzK4QdqjjfJzcRFS6YCQa/DKvmQu77jORx+oyc6G7dth0SK4GlvG2nvX1klq+OzKs4g2bUKkUeHgIESh3cgXIk6DXeqeL84YErGEH+78AbFIzI4bG8mzO0leniAbaYzRn4zmhbwXcPBpmWmCSsqzgdjWQS3S2hpcvKSU2rly8WTVlCiV0Srp9OoFc+bAuJ7pdPvvAw68b9zqp1Uz0EYgymQwvNyvbvBgQSmkItr9Na3R7+jSo/w09Kca1Znaj2/P8LeHU1wsvOdFdsKEztbKFltrI38Ua2chzYXawutg124Q8SK4GDqC+jkJm+ZZCmGypyhUsPXprbXmKmwulJaCdWkRVlk3v1HTbKPfbbfdxpIlS3S/i0QiCgsLef311xmndaVopZUWQsWFfnVGPy3VLazEIjEvDnwRgCVHllBcVnVxLxaDZ7mzRWWJz7Zj2jJh+YRadZ1bKr3m92LEuyPMOidoQBAPHHiAtqNr93A1hd1v7Obztp+Tey23xnLOgc6IJCJdXjudsS6lgJwrOWZHa+Yl5nHhrwsUpBQI9RSArSIYH2mY8RxWDYVzBExOh86v6CYl4xxe4fjDx3mo+0MAXN11lV9u+4Wre642XrsakbToNP6Y+AcX/9JvulSM9IuPN/RalzpIGf/1eHrO69lobVSVqWrNybJm+hr+mvFXneovPBOPU+ZVndEvrdxzS+vJZTJnXhY8zbRGGRsPw/Dpyotz7WcaTZXQAAcHkKjtEKuFzSqtrAVA+9vb03VWV/PadhOSdCyJ3+78jYRdVfXQKkf61UUpQiQSNXr+Q3tPe7rO6oqTv3HLXcjQEGZtn0XHKVWjGwsKyutwUFNSZmShVpwE+yYJm9y6CwaDSAI7RwlRVA2FS2fo8wOSwHG6tlZ8ZFcOX8nvE39vuOs3Eunn07m271qDGEPUKjUnvjuBe3t35l+cj1tb841+Z385y/KBy41uPg97cxiLkhdh526+/Gby8WTOvbUWx+xrODkJSg8ALjYuWInLNxu9hwny2u51HPvde4GdH4Q9AHflCI4RlsDaCbwG6fIGVoz0q46mllW/GVh//3q+7PAlZSVlFq1XJBYROTOSjlM6mvUs/vvkv6y+S4hg//57yFEK7ufd2/rXfGLvb4X8kC6dwbkjhbZCdFWd8gE5hXM5bzhZJf5mG/3EnZ7lWPHLAEYlPtu4CHJWCbkJjR6BftMiAu9I7ypjobF1p7JUyekVpyktMH9TLzsu2yBvtbl4Rngy6ZdJtL/ddEm7q3uusv3F7ZTml6JWqtn23DaWdV3GjpdaU8a0YjqFaYXEbomlMM1wUiwSifCwq37/RCwSMzNyJgA/nzXNwThyRiTD3hyG1KG6sOpbC3lWKfE7EpAoSkwy+rmWiyLUVVraFLRGv6Ag8PeHwEDBMPbvS/v47bbl5GQo8fYWDJCJifDsi1bE7bpB+jnz8xoPe3sYWWNnopLa4eAABYoCnQNqQ+zVhHuEc0/newCI9f1feW5mi1/mpibveh6/jv+Va/8I8r11MfoBBM8cyKUBD3Itu/bAJW2kn3/5VKtXPysyA6NIUPgbdWDSRvq1qaDQef/9sHAh3Hdf1fKVjX65V3NJO5tmkjN9Ubm/s8ZWmNBXifJTFgt7LnYBMCEB2s/HotgHQdS7gnNhBQJchQlinrJ8c1sEhz89XK3KT3NDLoe0sP6UPfUc7u1rkVNtYsxezX388cccOHCAiIgI5HI5M2bM0El7Ll68uCHa2Ky42ROBt2I6RelFvC17my1PCkmgjRn9VGqVLoy6JqZ3nk4blzZkFmfy/cnvjZbRLpyry+t3qxIxJYKejxrfPNu3T/9SbUikDlKsbKyw97KvsZxXZy/m7JuDVycvg+OdpnbiifgnCB1pnl58wq4EVt+1muRjgntRfj70jtvIz73iiPSu4E0T/RbsnQgmPIt1QmwlSACIRLpJiXVOJD38euAoEyZK/r39Gb9sPH49/cyvXiwmNDT0ph4/3du5M+GnCYTdpo9MqhjpV1AAW88f4c3db+pkfHs+2tPsxL714equqyx2XVyjB1Vd5cQA7J6ey4VBj1Q1+jmYsOunqmCMtPUDO3+9pKcppO+HLVFC5EsF7O1BhEgn8ZlVXDfJvpuZwpRC4v6NMzrx10b6aZ1G4uOFiPHauLT+EktClvD3nL8NpA8bm9KCUqOGI2s7a0JHhBo1CmqNfir7JOzetcPzQ0/DzWFrZ+i7wnBR0+llIcdf13chaJqlu6FH5g5hD+DgK2x6qtWGhtkej/ag2wPdLHrJphg/D31yiJ8G/2SWIaQku4SvOn9V62bujcM32PTIJo59dUxndDr5/Um+aPcFJTm1e+MChI4Mpd+ifrQbb1xaSeYkQ6PRIM8zT1rFv7c//k9Po8A9BCcnyCwWHHkMNhXtgyB4GtjWY7OnLB9u/A1iKYgseF/VSsEoDniUNzkrS8hN/erOV1l+arlB8azLWUT/Gl3nPLDNHY1aQ5uRbegwsQPWtpbPOxY1O4oxS4RodI1GI8gZ18LAFwYy4r0RaDSCXJncWpgfBjrXYvSz8YIAfdSmmxu0cTlNt9IHQG2mQTPsAf63bxNqjVW1jpA1od0EM2b083MU5pDFZcUUKAoMPtvz1h7+mf+P+ResgeYw/wwbFcYDBx7AO7LqXCtmYwy/T/hdNxZf3nSZv2f/Xaf8fP889g+/3fFbnY2t9l72dJnZxazNr2t7rnHg/QNkX8nm6u6rHPzwIGln00g727oYbsV0Evcl8uu4X+tktJ7VVVAa2Ry7WfdOb8U0xGIx+WckBOz4BbecKzVHm5ejdThK3hPL5U0Nk9PqmpDujuDyQLu+fYWf0Y79uNppHEilPPecIO/ZpQvIS0UcinqUIV/fbfa1nPydyHIVnMwdHCC5QHgnO8mcsJfWvG9UV14e9DIiRKS5bKREmlitATX/Rj6X/r7U4lSYSrJLSNiZQOENwWvJxqaWE6pB+7zWppKjVOr3Z/3Kt7na9vXAdfYECjxD2b276jnGjH4yGYwYYby92v213FzBmDTi3RE8n/08TgHGnWPLSsr4usvX7F+8X2f0c7KzZWL4REaFjTIsnLIV1nrB1V9r7qiFCfEU5iyFGuGPZ21nzcJrCxn3VcsIFNOqc9b0/B25cYT0oqrOBGKxmBBzE1HWA7NnuAEBAZw5c4aXXnqJp556im7duvH+++9z6tQpvLy8aq+ghVMfTehWbi5sXG1of0d73SLLmNHvRMoJHN9z5PZfb6+xLiuxFc8PeB6ATw59glpTdQNFW29lbxGNRsOmRzex5609detIC+XqVfjgA3j/feOfH/n8CEe+OGKRa/V/pj+PnXvMpE0fr05e2LhUHf3tPc2f+IUMCWHyqsk6Q1rFXEIG5J2H9L2Cca6hKL4ByVvw9RE2AyprjruEuNBzbs86SbGJRCKcnJxu6vHTwceBqPuj8IzQ659rPeTLJLlEBz3GuL/68caeN6psnDaWt7qVrRVht4Xh2al6jfb7d93PrP9m1an+wkLQSKx0mvRphSZG+mWfgM2dQVH+B2s3D0bsFgx/piJ1gZJkyDljcFhrgJSq9Hn9tJz68RRLQpaQerpx8ypamg53duBVxat0vruqC63WoNS+vWAAVSj0Hq41kXo6lbxreZz+6bQuMrmxOfXjKd53fp/rB68bHC8rKSPzUma13xvtOFgmE+6rjZWN4dhh7QCh94HnAP0xkUiInur0ohBJ1cBYSTS678nz257H+yNvPj/yOf2e6kffhX0teq2mGD+7zurK6E9Hm5WfUOYkQ61U1xpV6t3Fmym/T6HrffooXY1Gg1qpNlka1SPcg9s+uk2Xa9cY3/X8jp9Hmicd7uDjgCa8IwpbZwOjn7tthU1uS4z3B2cJjjy5FpbA2TsBNrYFjdog0u9M2hne3vc2f17406D48W+Os3bmWnKv5lq2Hc0EkVjEwOcHMubThpeJXjtzLd/1/K5Wqc/wieF0va8reXnChoNcKhhxtcayKpRmwekXq8jNurpCb/+NhEt/EvJHmkFhod6z3OylvzyTu33GMrnjB0aNfvZSe07NPUXqolQcpYbe99f2XLO4dGRzmH/WRNblLGK3xJJ2RpiPtRnehvHLxhM50/R8q1p6ze/F2C/H1tnIr5SbP5fo/lB35l+cj1dnL9qMaMMDBx/gyatPMmPTjNpPbqWVcnx7+HL7t7cT0DdAd2zi7xO587c7uZBR85gR4RnB0JCh9Avox5Xs2iNQFIUKfhn9C7vf2F3fZjd7RCIRxS6hXOs0Fp8eAdXnlK2ANtIva9W/7Hix/hG9x47B8uWGDo8VI/1Ab/RLkgSRGdyTe+4R1k0uLvC//0HXrqAUS1m71rxrlxWXUZhWpFuLOTgI7+L109fz9fiv69Wvmujo2ZEvxn7BvfnnsFUEVRuxn7AzgT8m/kHSsaQGa0tT4NPVh5dLXsZ5vJAKpK6Rfu3aavC+cpCsHadRqYR9nTfeEPYWK07l09OFSFGpVDBan199np2v7GTIQMFpddcuw/JKpf4ZNNWuY2+vz02YmipI7NeEPEeOskRJWVGZbj7W1qkT66avY/kEw30o7AKg7dwq8psWJfpN+CcS1HpH3rDyiJZScS4KlQKRSIRzkHONa7PmRGkpOGVcwepKjNHP913bR98f+tL5q85VnNK1c8/Gok47xFZWVtx7772WbkuLwFi+w1aaF4oiBVJ7KRJrCdP+mobYSoxGo/fwqChlczLlJHKl3KRov/u63sfmuM1MCp+ESq2qIptUXaSfSCQibkscToFODHltSH261iz5ccCPeHby5I5v7zA4rk3QnJQkaHpXfuGf+OYEapW60XNVVYdGo2H9feuROcsY96VpHi4uIS64hLjoftdGlrm4VCo48A8wYki2KOffhdivCeiSDPhyPUXOksPLSMhJ4NMxnyKuRzSCSqXiwoULREREIJE0n9yVubmQ5ryZsyEPUmqtNyxdyrwEwNlVZ/n3iX+Zvn46wYMsq+tvjOBBwVWu8/ffsG6dsKgJDKx73YoiBSXRiUhLPHFwcAbQeS4Z5Jc0Rm60YDQuiBMMLnV5Vpw7CRKzlTbmrKwEzzmHkgiCAqyRiPXPj5WtFXYedmjULUMizNimpDbqzckJ2raFM2cEj8XQWoKK+zzRh75P9aUorQi3duYb6i2BV6QXEVMiquReTdyXyC+jf2HsF2PpvaBqnlJtn0uthO+cj0MFTxxliZAv0qYJ5bAP3AMZB3BySqSoCHKLi0gvSm8wD/KmGD9DhoaYnVtLbCVmwaUFtZaTOcroPN3QwN1tTje6P9TdpI35zJhM3Nu5IxLXXDZ4aDAqhQqNRmPyhr9GoyE/Xyjr5KSPLDaI9Ntzp+CIMyHeWBWmEfWekHPDxsLOlP53gmN7UJXi4SFMmjIzYWD5GJ5RlGFQvMvMLgT0CcDO03wp1FbMo+Pkjth7m+4cpnW8UtoJm3n+TtU40WQegQvvg12ggfSSqyt8f2kRqZI7eGZmlOkNzTkDR94hwvNxkssGITM3DZzUBQ/NITzs2rK/mqCcKB/j7Zm+brrFJfVu9vnnmZVniN0cy6gPRuEc5Fzl8+4PdafHwz10Dhi2rrb0nFs3aeHwCeF1bqdKoWKx62Ii743kzu/uNPk8Rz9HHP30xt3AfvWYqLZyy+LaxpUeD+vHN5VaxdYrW5Er5Xx828e1nr/r/l0mX8va3prU06k4BTXehu3NikqlYv+FUtJDejFioGnzKLfyJUfpiLGMmVu/MVejgS++EIw1QUFCFBXoDS7aSL927cDdUUFWvjUdwkVMnaqvQyyGe+4R1k/HVl5ie1Y2I1/rb9L1r2y7wh8T/8A1aiJZAV1xdASp1IkJ4Q2fC3l+7/lk/QvHqF4qNWhgEBNXTsQnqg4h+c0AuVx45uzqOEUNCBThG38AuZ07e/dGsXKlMCcGwfCmjb5LSgL7nBv4+okAPxJ2JHBtzzU62UiRyQaSnCysvbWRg0lJguHP1tY8xygfH2GNm5ICQYFq4rbEYWVrReiIqot6Rz9HHo99HIAvvxSO2Vc3hXTv1fAOr8oi0ChBkS2ogwEh3q6MPJOEtdIVsUaYu+XfyEeeJ6+iitYckcvB9/IelBeyYUWHKp8vO7EMgEJFIe52hgoIKpWK8+fPN0o7oQ5Gv5UrV9b4+X3GRGpbaaWZkHstlxVDVzDwxYH0eKSHzsuioECw5oNeEgkEox9Ad9/ac67YWtvy991/V/u59qVgTN5zXvQ8pI4twyvCXJRyJeqyqgativrliYnQodJYO+2vaWZFIVRHYVohBz86SMdJHQnsX/fFqEgkIi06zWgUoCloNHBFuZejkffy7NG+/N3e0CPfohJgxgi+G1yj8C3fKMzJsuLZbc+iVCt5dsCz+Nn58XXk1wQPDeb2r2uOfDXGze4wsXbmWpKOJrEgZoFuMzk7R0OW405cJQEUyB2JsB3BMc0yruQInqIO3g41Rt3Vh0vrLxG7OZaxX4w1SFxfkcxMWLECyspg/35hUVOYVsil9Zfw7+WPb3fT5eeyY7OR/rkK9w7DcHAYjFqj1hn9qpX31GgEI13obPC5TYi0qis1bMo7OECP+D/59HFoG6I/HnlPJJH3NKBXWyORFZtFbkIuAf0CkDkajmlaA5ijo7Co1Rr9Ro+uuU5bN+F7bONcR00UC+Dfy5+pq6dWOe7o70ifJ/sQMizE6HnaPpdIjBj9kv+BA9Nh4BoInGTpJpuGfQiUFeDqrCQlxQqpWnBrzinJ4eiXR7mw5gL3bLynyr2sDzf7+GkqxVnFqBQqHH0NI3xMzTlZVlzG972/J3hwMPdsvKfGsqM/ruVLYoS1M9dSsP4KomGLcHKSYO8cyNSIqfT0q7DJ7tShal5Sc3GOgGH/1q8OY7Sbq/uvNtIvOxs87cqNfsWGRj/f7r5mvSdaEvk38vmm2zf0e6YfA58f2ODXi7grgoi7ImosI8+V8023b+gyqwuaocMAKK0t0s/3Nhh7BmwNN/3c3KBUZcf55CjzGpp3EfvsNThKZ+JdF38RsRUl47P5dpXwnS4qqmGjqhKWmNMb42YePxN2JhDzdwy3f2N8Xl3xHZ5+Ph1HX0fd+70xKc0vpf0d7fHqbN5GnkajoSS7hIt/XcS7izf+ffxRFCq4vOkyLiEurUbAVupEfE48cqUcGysbQl3NS61RGyKRiEWpi5ptdLAl0WggNlaY75iSzw/0Rr8sl7a0GVa/6yck6FV39uwRjH5FRXrDjTbSTySCbjF/kHM5nQeWLEAiMXyXdOoEERFQdvgYB/53jSHP9jJJ3cnR15GOM7tx7oYv1taYFOloSbRO4Fcz0oGqY69rqCuuoa6N2qbGIO96HhnnMyjK8APs6hzpJxKB5t5ZxCU68sknhp9duaI3+iUng3/MLpwOX6M0/xnGfz2e/Bv5yJxkHP9OePZ279Yb/bTSniEhNW5fVMHXV1jDp6QIShNrpq8hoF+AUaNfRbSRfjK7MjQaq6YZm7p9IPyrgIODCFulHxqNoBDh4gKrp64m91oui5IXNX4bLUxpKSSFjyCga9VUETklOfx14S8Ads/erT9HWYrMShh/GnPuafYu8ZNPPmnw77HHHmP27Nk88sgjLFy4sAGa2EorjUdeYh5ubd2qeIhnlUfkCh48+uOnUk8B0M2n/jl6asrpJ3OS3bKTy0dOPMKE5VU9pirmUzMmZ+cR7mHgPVpX0qPTOfTRIVJOptReuBYeOf4Is3fPNrn8n3f9yWehnwm5h+RQQDJy2XVyFBW0odVlcPV3yG1gbxGvwdD2EexdXPDyAjFW+NoKLnRXsq8gthIjdZS2mJD9yjj4OuAc7GwwNuTlioi48RHvhm9j8PloPJKFZPCphYIxInRkKHP2zrF4lF9pfimrp65GWaLUGfyu/HeFVWNXGTynv/8uGPxAmLwCFCQX8M+j/5idR8HRz5HM/neQ69UOBwdBRlOlESYrRiP98i4KUnLaXH71MfhpKYiDc29D3iWDw9oNw8KWlbJAR/Sv0fwy+hfyrlXN1F7Z6Ae15ya42fHq5MWYJWOq9QLUynsWIDzrvg4VjBIydwieKRhNmoqo92DoJhydhe+mtcoFgBx5DnnX80g/l46iwIx8ljchy6KWseHhDWafF7c1jn3v7qtWuvXk9yf5xO8TEvdXfalnx2Xz71P/knw8udr6VQoVvRb0ImJaw9x/j3APlN7+aMQSnJzgtrDb+HPqnzw34Dl9oe4fwaC/GuT6lkS78VZWBrYawTklvSjd6L1Rq269nH7KUiUeHT0a3YCiVqqrzWdWlF6Eg48DDj4OpKSABhVFImG+4e9YTaSf2Apcu1SJGtVKrBXklaG5thoSTXxmQ+5mo00hJ1LGGCifmIODo1jnQGls/v7P5X9Y+O9CNsQYjjHyPDkJuxLIu171XdhSmfjTRF4qfqlGB53izGKOf3OcH/v/yBftvkBVVreNpKKMIr4M/5Jtz20z+1w7Dzum/jmVvk+aL1/9if8nbJq7id8n/I5aqUZRoGDtjLWc/um02XW1cmuyeupqvun+je738xnCmjjCM8JAAaQ2CkoLUKlr//7cqnsylUlLg/Y7fifszFrCTQwU1s49qotOM4eTJ/X/P3NGcAi/fl1/He36UKPW0G1SMH3mRBDU1rjzyLRpcCPiNmJGLUCuNC2Hr39vf3q+dSdyRy+dNOOeq3v4/dzvJknF1hcXVxWnQ+5n/pUAbuTfaPDr3Sxo9zwKLwvrgboa/QDaDfJBKRMelC5dYPBg4XjFtXRyMtwIH47H7HHYOAspJZwDnbFxtmFYueF67169xKxWurxiPj9T0BoZU1KEMWbiyomMeHeE0bLXD17n1PJTyPPkOqPfz2nPIHtbxjt73zEsfPolOPygeY2xAGKxPg2Ldr+i20Pd6Pd0v0ZvS0NQWgqF7sE4964a5fdr9K+UqkqJ9Iqkl58QZVmkKGL4yuGNMjZUxmyjX05OjsG/wsJCYmJiGDhwIL/99ltDtLGVVhqN4EHBzNo2i+4PGUbuaY1+7hUic8tUZZxNOwuYFumnJbkgmY8Pfszea3sNjmtz+mVn6zfqtRSmFRL3bxzFmTXn+riV0Eb6aVBz5WrVDVSVQkVOfA6KovptroYMDWHB5QV0mt6pXvWA6dEKWlzauOAZ4YlIJCI3FxTWgrHP16nCTos8DQ7eA3Hf1rt9pqKVy/CQCJ5H8TmCjNnDRx/mto9ua7R2NCa3fXQb9203jGTXehcO6uWCRCOD5F7EzU0jep6F8zBVQuYk44GDDzD0raGA4CmddTmLhJ0JSGTC4jYpCbZV2LfRGv08wj2YuWUm3R82fcwCsPeyJ9WvOyXOvtjbg6uNKwlPJnD0oaNIjUW1ZB2BpI1C5JWlKLgCZ1+F1O0Gh7UTyspGv+LMYg5+fJBre69Zrg1NQIc7O3D7N7cbTeZd0ehX0cNQG5l+s3N502VWjlhJ/HZhDDEl/6XW6JenMhLp5z0M+q8Uoq2aGOdyJTaJojzST57DqMWjeC7zOYs4pDQVGo0GKxsrJFLzZZnO/3GenS/vpDTf+APq18OPLrO64NerqpNASXYJR5YcIXZL9VZtGxcbRrwzgq6zulZbpiK739zN7xN+N63xwJDXhpAyXHDuaMRUDJajJA32TYXYb7Cy0j+j4mLBICRXyikqKzI45esuX5ud+7Al4Bbmxpy9cwwk4xqD3+78jeWDlyPPreo57N7enQcPPUivx3qRkgJqURkTPV5gdtRs4xH3Gg1kHgZV1bq0EQJlShGao/Pg3FsmtzEpzQ6lWmaQ49wsiq4zsdMyfB3iquSHBth7bS+fHfmMXQmGknvJx5JZOXwlMRuM509pqdRmYMiMyeSfR//BxsWGAS8MqDUXUHXYutkikUosLqFaEyKRiH6L+tFrfi9u//Z2JNYSHHwduOuPu/7P3lmHN3X2b/yTpEndnVKjxSnu7roxHDYmbMyNub/vu737jQnb2AZ7mTsbMxgw3N1dC7S0VKm7Jjm/P56epGmSNlUK474uLiA55+RJmzzneb73977vJlHY3sSNAQcvB5z9jJLh02mnAejoa/veve9XfXF7x41TaTXv4bJisjj8+WHykvJqP9gbCKdOSpSpHXHxcbDZ6lkm/byObuYtx7coTCus/oRqcPSI2DMoFKDXC1ebqnl+IFRTg18bzLhPxlm9VvfuENjVn3yVB3//bfsY5L2nTDAuObyE2/+83axppTHg462i0D4GHeUGVVFlZF7MZGHIQna/u7vRx9KUCB0Uyq1f3Uq5p1i7OlTpiSnTlbHu4jre2P5GjWRo/356vBTZjB5cwhtvQNeu4vFLl4zHJCVBkUcQ7W43r5107SrW0rm5sHGjeKyy0q82kEm/1IrEmA5TOxDU23JD1+llp1l13yrKCsooqigPF0mZlOvLDUoyA9J3w9WttRtMbaEvh5hvIdH0cx8X8DHHw+5h1+X9AHSf253+z9lmn9vcIddaLM19Xx/7GoC53eYa1nD3/HUPexP2cjDpYFMN0YAG8YNr3bo177zzDvPmzWuIy13XUCob2WLvJhoN1RUbLZF+Z9PPUqYrw93evVbWEe/vfZ/nNj3H/w79z+RxNzcxaUiSMa9OxsU1F1k6bimJ+/85XTwgwrIPfnqQxAPm7/ubq4+yqXMg67o78Xfi92bPH//uOJ9EfMKVXRbaiGsBpZ0S79beOPvanrViDaV5pZz6+RRJB20LVB69YLQhzD4nB0rVovvbz6lSx7baAwb+BuF31Xt81UKSYF132D3TsIhxLosAjKRfXaFUKmnbtu11NX+WlEC2LgmdooSgILFQU0n25Kf6mRRojn59lC2v1j+ovDC9kM0vbyYnPgcQ1oie4Z6UFZbx+/TfcfRy5MWcF/HtIBQbS5eKDVBUhbtlRoYgS9SOaiLHRprZ59UErVa8ZxAkm0qpIswjjF5BVjziw2bDrRchYGRd3q5l+A6A0fsg8n6Th11cIMH7e2btacNT658yPF6SU8Km5zZxce31LX0L7BZIjwd7WLQGzs8HPVpK1Vfx9hbqDb0eYuv3lWwy+LT3IfVEquFzHbctjkVtFnFp/SWLx1f+HGaXWyD9mgMKE+D4y7T1EBssRanR3rMx0NTzp0Kh4P799zPh0wm1PnfgSwN56NhDVhXhrUa2YvIPky1aFrfo2YKHTzzM4NcGWzw3dnMs5cXlFp+zhuyYbFKOpVBWYHtzkEw6u7mJnAZ95Tzd8gI48jQkNWCzQ0PCzhkSl0OuKIjKaqviPGcc7USrtGzbLCOwW2Cj2VTfhDm6zulK36f7olBVT/SkpoJKcuDZbv/Ht7d9i53Sgs13QSxs7AfHXjR7ys5ONIvoJTvS2/wKQ237zErxv3P1kiDd6pwTnHOK21o8QpeAzYbiVmXIVqXJBaaqXv/O/oz5aAzhw2rZPl8NmvP6U5Ikzvx2xqryU0bLvi2ZumwqD594mAHPD6jz6ylVSh45+Uid8uM3v7SZbf+xPRetMka8NYLxi8cbMgUVCgUdZ3S8IW3pbqJxcOvnt3Ln+jsN/5dJv05+NnpOAk5qEQwmR7dUh4S9Cax5eA1JB2zbz9+oOHdexcV+99B63nibz3F0FCRNqZMHgf3D6uwkEL8/Bf3/PsM5O4Ehmn24ZsSyY4d5nl9xdrFNa0OFAqZPB6W2lC2fXaSoqPomRF2Zjm8GfsPZ70QBX25ATc4X9y2rltsNCE9PaJE9A4Dfzv5m9rzaSY1roGujWWNfK3i38ab73O6U2ovuO1npd/LqSeb8NQf/9/259ZdbicmO4Upu9XVA7ZloWq3+hMH+0djZQWSkeDwmRpS+ANJOp4Ek0cLCr1SlglGjxL+XLIGFC42N1rVV+smNVJWbofQ6vUX1ft+n+nL76ttx8XcxEM+FeiGf9XY0zY9j1E64pZGbpRQqOPw4XPjU5OFU5w0k+vzA2fRzjfv61wDFhXq6rZtP5vemXQLHUo5xLPUYGpWGOzsb70ty/vuptFMolUpayxZNTYAGW+Ha2dmRnGzdcucmbqK5Y+1ja/l5ws+UF5kvDCyRfvKisFtgt1rZPMyOEl3iq6JXkV+ab3hcobBu8Rk6OJRbPr8Fv6jrP/S0NihILWDd4+s4t9z0RpGYl8g+7RJKNanolaWcKtpgdm6LXi3o92w/i+qY2iD9XDpFmQ2jsCzOLmb57OWc/Olkrc/NzYVSO1GMM7FTVLtAyHTw7mnlzAaCQgH2PqDxMJB+ylxBdssZdhfWXGDnWzutXKB6aJraCL8WKMooYvNLm00UYzk5cDz8btZ3d2HN5T9pVcH7x1RR7J9fcZ69C/ai19bPHq0gpYA97+xh+7+3mzxemldK4r5EYjbGoHZUo1AoiI2FXbvE8w88YOwck4kgSZIouFpgk6pKxtb/7KDTtsWoi/Nsy99RqsE1EtQNqGhSu4BPX1CZkl8uLqBXlJBUcpH4XOPvyC3Yjfv23EefJ/s03BiaGQoKxOfwlm2BPLfxWcIiBSN2vVh8ekV48VTcU3SfK7onS/NK0Wv1OHhatjKTCReFAvoE92Rc5Dja+VTyFDr2vPhzLaHNh7PvEOYgyH6pyKj0y0/O5+wfZxvcnq45z5+V4d3Gm4CuARZV7zXNkQqlAv/O/hbXWxnnM1g6finL71heq/Hc8tktPBX/lE3KFr1Oz9+PrsUx9iwgSL8xP41B/aba2NFdnALRH0H6rlqNo8mgdoEZxdBzEWBc02ZmGtcV6YWmuX6Tvp/E+MW2F/RuFESvimbX/F2U5jetbLrjjI4M/c9Qs8zP8uJy/rrnLy5tEA0RclGoWrWdnRN0/j+r+aay4iJFGgFOLWseXFk2ij0zGOQ1H7Uaelnp+akRvv3YbbeGvQlTLZJ+ga5i0SIXT2U4+znTd15fQ3NTQ6G5zp8lOSX8MfMPDnxyoNrjlColnWZ2qnNmeEPg9C+nid3YsN1Gpfml9XZruYnmh6KMInb8d4fFektDQbb3rA3pJ7s22UL6tRrZitnrZhM6pGHjG643nD4tGidszfOT4ekJGaE96f3x7Fo3oco4uSsXTWE2/qpMCv7ciE/iCc6dM1p+yqTfjjd28FHoR+TE5dR4zf79oe2VzQTv/Jktv2ZWe2x+cj6Z0Znkp4hanmzvmVIgbs5NQfp5eUFA9lSQFOxN2GumanMLcuP+A/fT65G63qybN2SFm0z63fPXPXx/4ntySnLwdfZlfOvx9A+uXlUW0CWAXo/1wr+LKMIGB4NaLXLyUlMh60oBLVZ+RujJVQRZcVGfPVvYwyoUsHWrMYIotMr0sCV2C8ELg1l/yXJmt1yvSUsTTa7n/zrPfOf5XFhtHsni2cqTNre0QWmnNPwc8rXiM+vlaCFwub5Z4zVBoYShf0MvU9LP1U5sNNLyxdhO/nSSr/t/TdalBvD3vcYoLdRS4BWKYwvTBqXY7Fi8HL2Y1G4S3k5G8qCzf2cAg1NgU649LbQFVo9Vq0wlm5IkkZKSwuLFixkwoO7dZTcK9Pp/Xu7FjYLS3FIK0wpRO5n7eFsi/QJdA5nUbhK9W/Su1et0D+xOG+82XMi8wF/n/+KuLkaFlr+/6FCqSvp5RXrhFWlhAr/B4drClbu33m1G3K2OXg2AQlIhKXSkOGwhJ0+Lh5txSgvsFkhgt0Dqi58n/IzGRcMjJx+p97XcQ9yZumwqQb2srBoqQVuqZetrWwkbEkabW9pU2HuKD4ZFG6emwHDhWxBW0TRVdrUVhBiVfmeWneHkTyfp93Q/i98ja9Dr9Zw6dYqoqChUqrrZEjUmsi5lsefdPTh6ORI6WKzgsrIk8hxPICl0tPIMR4qAPXvgtwvf8cOyFczqOIvbo25n/Kfj0bhoam3tCiJjVJIkPEI98O/sz5wdcwgeYNpa7xroyv0H7sc1SOw2ysvh22/Fc4MHiy6ziAhRILx0SdhQrHtyHYcWH+LZ1Gdx8XexaSw6hR16pQqNqz0qFWyO3czm2M0MChnEhDYWFD9Zx8A5RGSsNST0Wsg9KwhFO9GV6+4Oap2YH7OKjYtIO3s7gvvXVYrQfLD6odXEb4/nsfOPmREe2XllJAcJa/XPj3zO/yLf4tghuFC7yMZrCplwKSsowzXIlXmx86wS0rKdqYsLvDr4FfMDklaD0gHqH7Nbd7i2hgnnSDkivIUUhf50atGJSK9IEvcn8vv035mydApRd0Q1yMs19fyZfTmbc3+eI3JcpNXcxepQnF1MaW4pHmEeJo//cusvlBWWcc+2e1CqLM+Xeq2eK7uvoFQrCRlg9G7yau3F8LeGEzY0rFZjqc19qjirmCNLDuEaqie3ZQecnCCjKAO9pMfdvsIn0yUcbos3a0xoVqi08ZeVfpmZsOr2VTipnQhxD7Fy4j8L5/48x4kfTlyzphFdmY7SvFKcfMR9LvV4Kid+OIFPBx8CB0aSlwcl6hSKHYoo0QbhYGfhM+cYCJ1etfoanp4QH1+RrVReAJn7q1fnKzVsK/uVjZda0Ls3tjUAWYLGE1qMJ6+U6pV++ZabiSVJarBMrea8/rRzsGPqsqm4h7g32WsmH0nm7B9n6fVIr1q97pOxT1KcWdxg47i49iI/3/Izt31zG13ndG2w697EtUfigUS2/2c7aid1g9i8FWcVs//j/USMiiBkYIjYNzl44Kx2rpW9Z49AYeVsC+nnGuhaZ7LqRkFGBhSdicX7aizBXr0B2+cLb2+xL61Prl+SaztODZ/H6MnOjPuvB1+sbgHRxkw/2d7Tt6Mvwf2CcQ+teXxKJQSO7cJphQ+RWqdqj/UI8+D59Of541cd/CT2JZIkGe5bcvNKY8LTExzLg/AuHECmy27+OPsHT/V9qtFf91pjyytbOPPbGUqG3g844eQkfvbRGULNtmLmCm5tc6tNeZ6erTxNGtvs7KBVK4iOFg20vo6QGtEPbVCogditCjs7uOsu6NEDPvhAkHYtWphnDU79bSq5pbmMWzoO6T/m+1xPT9BooKxMXMMjzINWI1ph726u1CwvKjfsYeRMvzxthdKvEtFEeT5c3QYencElrMafR73gP8zsIXeNF5RBRqEYW0luCTmXcyjJNbedv95QKmm42Gc2E+40fXxqh6nc0uYWsktMHX6i/MS+/1TaKfR6PWfOnGmqodZe6Tdp0iSTP1OmTOH111+nc+fOfPPNN40xxpu4iQaBtlTLwcUHrXaTT1k6hbn7LIecWiL9xkaOZcXMFbw86OVajUOhUBjUfktPLTV5Tu7YrUr6/VOhdlITPiwc79amxMGqC6L5oG3Sm6i1npTb5bD2xKFGGUOvx3rR85GGUdEpFAo6zexkk2VNYVoh+97fZ8i6ysmxovQ7Mx+W+0Ne02WctGghFjjqfFN7z8H/HsxDxx+qU9ZTc0Zg90Aej36cLvcYc6IuXU2mXJ2JQlLRwbeDQel3Pus0q6JXcSBJdGZ7hHrg5G26cSgvLmf/x/s58cOJal9334f7+DjsYzLOZwBC8WupGO7W0g2FQsG5c/Dkk3D8uLCbuLNiERIhfk0GpV/o4FC6P9i9VurDiLsHcHbIIzh6iIXntsvbeHfPu5a71SQ9bOoPe2fbfH2bceFTWNcFMvYaHvL2Bo3WnPQDUTi1lI10PUHjrMHe3d6syKnXQ7LC6Au/896dRLUXhd/rifQDsVn7cdSPLB27lMK0QqsFXZn0s5qnNuEcjLrG2RVKNbi3w9VDfO/tctpz6pFTrJi5gqDeQUz9ZaoZeX894eqJq2x6fhOpxyxU62tAUWYRHwZ9yKbnN5k8LuklnHydcAtys0r4gZg7fxz9I7v+b5fhvPKicpQqJQOeH2BTQ01VXNlzhV3za1bmOXk7MW3fMyS3Hoqbm+jozSgSc7Ns2YLSTjQ7ODRjV4b8SxD3C2iLDGvajAzRgRrpFWmW0ZoVk8W6J9dxeevlazDYa4fRH4zmgcMPNGm+mYzC9ELe93+fTS8avyfB/YJ57upzdJ/b3UCUpQR/SqcvI3l2w7N1eh3PiqVodjZw8AHYOkqoVa1Ar3Tm++0zOJsxkCG1d4A0QYC/hKdDCikp5oWvyqRf1QaQzS9t5j3v99CWaus3gOsAakc1nWZ2Irhf090v0k6lseedPaQcs/45sASlSmmSqVZf+Hb0peOMjvV2a7mJ5ofQQaG0GtUK77YN0xSYfiGbnf/dyYVNwulDoVCw695d5L2ch4cihG3bYO9eOHlSZG1ZMzmRlX7HU4+j1dc8v0iSdEMUruuKs2fBI+0ireL2YifVbj729AT7gkzOLNxkc+SJjNyEXIoyizh6FLT2znTvDmFDwhg80vReLdtP93igB7NWzrK5UcSna0vSwvuQVwPpJ6OwRNQ8XFwgrzSPonIhuwp0aRrSDyAgq8Li84y5xee+D/fVyWGqOUPtrMbe1Z7ictHs7+gI6UXpFGuLUaBgXOQ4VEoVeknPsZRj7E/cb9N15fWGXDe5dAmyy11Iaj8K955tqOkj1KEDLFok6i9PPGH6XJmujNxS4fKybKrlLHGFwqj2S0mBgK4B3LHmDlqNMI2R0mv1vO36NstnL6e8XJCEADmlomBtYu+ZfwF23gbxP9fw7hsIZdmgNTqkediL+kx2iajP9H6sN8+mPEuLHo2vhG1sVJfpZ29nbxY/IivPr+ReIackp5FHZ4paK/1uKtlu4nrFwUUH2fT8JpRqJT0fskziWFPjWCL96oM7ou7gP9v/w6bYTcTlxBHmEQZYt/cE+HHUj+jKdczZPqdhBnEdQNJLoDANsc8vzWfrZZGVFJAzGa3vMS7Z/c7a6A3cMaifyfl/3v4nTn5OjPvYenBzTej/bMOHzZbml6LSqCzmFslwDXTlyZgnsXMQx+TmgmNZMPZ2maaWEfa+QlmiaQIlaN5FiPsRu5aTCQ7uRmlcO77td4hbBwrT8qrk7I0ClUaFdxvT93YsRRB2voq2ONg5GBaIUmYEuBgtT/U6Pdkx2SjVSjzDxepcoVSw661deIR60OXuLlhD6wnC67umjbFOB19/DX//LTazHh6C/JMXjvLYZOvRjtM70nG67d2vYAwqlzMLrhZWozrVl0PUf22zC6stAkZAx1fAyVgE8/ICdQXpl1lkaseyuO1iHL0defDwgw0/libCmA/HWHy8oAAyXMVcOL39DLoHdqewgoxIuJpPbq4r7k0nEKgXFAoFA14cQE58jkHZYgmyvaeLm47i8jIc1VXaKBUKYWF4rVGSjo8mE2hnGDMIgr7TrFr6IDUzhA0NY+6+uXXKW3LydqLXo70I7GFaDFEoFUz+YXKNlsP2rvZM/Gqiwep8UZtFhA4OZdyicVZzAmvCiR9OcPSLo0TdEWWmPqw6Rq2jK1oHQTrr9DpDTqOB9CtJB20hOAUJ8rc54vKPcPq/MO443t7i/pNZjYtVeVE5BxcdxMHDgfDhDZel1tzh5ONU7VzUmHD2dab1+NYEdjf9nsikSsp58X+FqyiWBrlZILv15bAqEsLvhi5vWnwd2d4zOxsY+CBX9QPZ/qcjk6ZbLmKcOa0lM9MOZ2foWc9euJDsF/hh8vvMXXWZkpIwHCoJFeViaYm2hNzSXDwcPAzPOXo7EtAlgJLsElwCmsFcf4Ohza1tePTso3hF2L6nSD6STHlROS37tkSlbpimP49QD6Ytm9Yg17qJ5gV7N3vu2thwOfTHkv05PfgRrsbY0yPbSISkXVXy8suiqaUyJkyAhx82v05r79a4aFwoKCsgOiOajn7V75N+HPUjaafTeC71uQZ6J9cXMjIgufUQXPoF4BZcO3Leyws0JXmk/bWX5FEeBPW2vWFr+3+2c+LHk2QMfQqVsyudOwuypkNANs4F5RS6+OPjAxqVlpIcba1tjw3NMGlllBcpLDpCFKYXsmv+Lno+1JOCArH+c3U1Wnu627vjrGm4JghrUKvFnj8weypnQ+axL3EfCbkJBLsb98i73tqFf2d/Ot/ZudHH01QY/OpgBr86mJ0VvcWOjhCfI0j/QNdA7O3EAubLI1/y8JqHGRY2jK33bLV6veTDySyfvZyBrwyk6z1dkaPWLp3X4uKsAhRWrT2rwskJZs40f3zPlT0A+Dv7M73jdKvnBwQIBwZLLggyyovLiZodRVDvIIO1p4TOQCSZ2Hs6hUDf78CzCSxw4pbB3tth4B8QMhWoUB3mGwnJGwnlKRkEnTtOSVxHGCjWrTq9zqrC1NPRk2C3YBLyEjiddhoVTSeSaH6p1TdxEw2M3Cu5HPrfITpM78CYj8YYsoN05TokSUJbquXXyb8Svdq6Uqoq6ZdXmseV3Cu1ysSqjEivSAaFDEIv6Znw8wQKykRFvTrSz9nP2WYrvhsF0aui+a/qv5z65ZThsY0xGynTleFc0hrnkrb08xsNwL60jWbnpx5PJeNshtnj1xLHvj3GO27vmOTDWYLSTolnK09cWwgvgZwc6BH7O192vkjvoEqWspEPCGWLQ8NmnFhEcRKcfhPSdhIWBirJAcfsngYbAUmSKMkpsSkw+3pCYXoh+Sn5goSuwNksQfqF2IuiqYeH2MQ4lYoE6JgswbAVpBawuO1i9r4vlGml+aVcWneJGX/OYM7OOdW+bsSoCMZ+NLbG7sRt22D1akH4jRwJ//ufadaOrEJMTjb639cWZ7/ei2fyWXPSz9kC6aeyhw7PQ9jtdXux6uDRCbq8BW5tDQ95e5vae1aelzvO6mggT2805OdDpus2AIa3EpYaRaSzq0t7Nnb149TZps2iqi/aTWpH33l9USitf95lAq3M5SJO850IWVjJirAsB1I3Q3HtFWgNjr2zaXlGfAnz8qx3ll+PcPBwoGXflnVWdYx+fzRRt1u2NrWlE7vL3V0M1t3tp7RH46qhKKP6ia24GD77DLZsMX+u39P9ePDogzVa2Z36+RQZcWKt5uYmMholxC/WsMG+9DmsCoec0zW+j2uGkGkw4FdwDjFR+u2+spvXtr5m1inu086Hp+KfYujrQ5t+rNcQWTFZjZo5VROmLJ1C78fEWk9XruPM72coTBMeTnKeX5mDIP0sZgeVpAt7T5UF9q4ClZV+57KG8ehHj/HTrx78Zi4WAMDj6FgWjYtiwABRbKwPNEFD2Rz/IAoks+KWo9oRTwcxuKoWnwOeH8A92+75RxB+x787zvsB79e4X2hIOHk74dvet1aOHXvf28t3Q75DV6prxJHdxI2C9U+v57Mun6Et1da5jlIZJ06rKHHzIyHXnddeg5wcibQ0eOUVcW/z8YH27TEU7bdvF3lZVaFUKOka0BWwPdev7cS26HX/TEFEQQHo1A44tPKu1qHBEry8oNCzJU4vPWHiomMLIsdG4ju2B+UOrrRvLwif8qJyvu/9Ke2TNgMiS+3kjydZGLKQuB1xtbq+hwe4Zlym7P8WcHqZ5bXchdUXOPDRAVKOphiaYp2daVJrTxm+vuBQ3oKpIY/x0ZiPcNGY3hvv3no3E7+Z2GTjaUoUVzhKOziIdfhz/Z7j3q73Gp4fFi72xruv7KawrNDqddyC3dCWaNGViXtYpCjlkLXpMOcfWohjbgot6ilMk52RxkSOQamw/n2prPQDOPLlEVY9YBqvZu9qz+QfJtP78d6Gz5+9cwmT2k1icOhgU9LPwRda3QOeTUD6ekRB+D1i7VkBX2cxFtl6tDi7mNPLTnP15PVva6dLSSMwZg9lSUZCc/RPo/F819OY9V4FUf5Gi8+mhE1Kv2eeecbmC3744Yd1HsyNAKXyJo/a3HBuxTk2PLWBWS1n0XdeX8Pjm1/cTGZ0Jn3m9eHCmgsE9Q2i7a1tzc4vLzcWGuX8k78v/M3s5bMZEzGG9XdaDmOtCT9O/pEB3wxgVsdZOKtF8aw60m/K0il1ep3rGc5+zrSf3B6PUA/DY7K1p3/ORNxcFYxrM4YfD8HlsgNkF2fj6WhUHzx69tF65X4c+t8hTv9ymkk/TDKotOoLv05+dJrVCQf36jvPirOLKSsowyXABZVaRa5wBLi2yh3vXjDuBLi2JixMPBQXZ3z63PJz/D7td6b8PMVqUdcSlEolUVFRzXb+3PavbRz5/AjPpz9v6PyPyRdWGZEuxs1Kq1aQcNJoeaqX9Li2cGXASwMMWYAxG2P4fdrvjFs8jtBB1gPgy4vLUTvaVlU7d078PWkSzLXgUOzuLuaujAy4fBk6dJBY/cBqnP2cGTF/RI3XlySJ2C+34u3ZCheXDgBcLbjG+ZKV4OVltPcs1ZVSrC3GSS1+TyPfriaf6DrB4c8Oo3HV0Hm26YI9I6eYbBdBJg8PHw4IxVG5OgO9VMKm08cY2K+v2fWuZ8j2nnqnVCjE8HsGIOuosKfruRjaPHZtBigjbDZ6r4GAhFaroP9XA7icG8P6ievZMmQLHWd2ZOxHYxvkpZp6/rRFqW4LCtMLcfJxIvNCJivuXMHwt4YTMTqiVtcY9d4om4778kvYVOGUmJoKd9yBwabHp52P4bi002nkJeUROSbS5PysS1ksn70cz0EdwX0abm5GVbG7vTtqVcVc7dMX2j0rLD6bKzyixB+Ma9qsLNiXsI+3dr3F7KjZzOg4w3C4Sq1q0kyx5oCygjIWRS6i812dmfzD5Gs6FkkvkXI0hT9m/MGQ/wxh6OtDDcWgIlUy6CDI1UILulMLGFO9pZVM+kVHw+HDwiLKTZPBhr81TJjgZlACgtgLnU1qh4ZAhtzaAG8saAJr0ydwtVB8J+U1pYw99+3B28nbqKJtJDTn9afGVYNXhBcOnk2bEVqSU0J+cj6+HWxrJuz9ZG/ChoWZWeGWlYnPTV2zH6NXR3PgowPc8vkt/8hc+xsZunIdK+5cgbZEy+2r694gKElw4XAuqnJ7NG4OXLkCvd6bSYbyNK3LFtA9aALz54t9gl4Pd98tnHPOnoXOFmrgk9pOorVXa4MDU3UY+NLAOo/7RkBBgYRjQTqRYX61nj89PUGvUpOj9KK2griOMzqy4kJHOADdRR8/GmcNI98ZSYG9N+d3w5Ah4JzvjH9nfzPFfE3w8IAitwDK/IOtNrd1vbcrvh188e/sz6/zxWOurtDJvzMrZ61EQcNkztoCPz+RPXeX9yImWtjyBXQJMH/wOsexb4+hLZMoLxcfAEdH8HONYMHoBSbHtfZqTah7KPG58eyI38H41uMtXQ4Xfxfmxc0z1AyDg0W2XonenhKFE6UuPvUm/TbEbABgdKvR/HzqZ/46/xdf3vol7g6m62uZ9JOboa7susLJn04yduFYi3bzcjO3m6Mz385cXr9B1hceHaHfdyYP+buJe3eBTpB+eYl5/Hn7nwz5zxD8O1/7GlJ9kB/QmoRhTzB1pLEWkZyfTE5Jjhn5LmP+8Pm8O/JdWnu1pii/jp34dYBNO/Zjx47ZdLGGCtW+iZtoSHS7txtuLd0IH2G0JZIkiZLsEoqziwkfEc5zV5+z+vmVQ4bVaqO13bEU8Z2I9Iq0eI4tCPUI5fSjp01sa2TSLy8PSkowsbv5JyK4fzDB/U2zLLR6LXYKNf45E3H3gJ6tgwnYMBUvIijXm3Zl13dOKs4qJu1MWo0EXW0Q1EtkOtWE498dZ+MzG7l3972EDAghJ0c87uFR5cBTbwjpfsS9NDrsnA2dQqEVfNWmqz+Tuv4Q0zpMo3Xb1nS9r2udCoRlZWU4NNMPfPjwcJRqpYlFyJUyofTr4G0k/SZNgqPHQ1Do7SillLOJSXQKDjYhnoL7BTP2k7G0u60dhemFHPniCB1ndDSzRv1uyHfYu9pz95a7axyfbNvZvr31YyIiBOkXEwMdOyqI3xGPs7/tO63wdx7k1CoFoRWnpBVayJeUceoNobgavBLsG6FQc/FzkWU5ajc4B+PlBSq9C04lrYgMdqOwrNCUDLrOsfP/duIe7G5G+qVk5xGYNROtRzStvYSaUaFQEOXRn/3ZqyqsRK496afVwp49wg6ursU/GTLpV+4gSD8Tv3zX1tDrf+BXz7CphkCre7BD3MNLSiC1II2rhVfJU+XhEe6BS2DDqlSacv5c98Q6Tnx/gpfyXsLe1bqKqDoc+eIIax5dw3177iM/KZ+M6IxqFZ71wcGDRsIPYNky0R08dy4m+RySJLHyvpVcPXmVeZfn4RroanjOPdSd6b9P50C0B+wXSj+zPD+AgJHiz/UASY+3tyjUFRaCp0bM5elF6WaHFqYVknYmjbAhYY32e2pO0Ov09H++v5kNbVNjx393cOybY9y35z5u/epWWvYRltkpKSAhkakTCrCWbnWz0pZJPbnRcGzPAzwYMZSfT/6HZcte4tFHjccePgyL9y/G2xu+tb2nq1oEBIhipSUbq/a+lhc0ep2ePe/twdnP2eDcUl801/Vnh6kd6DC1Q5O/7tLxS8mOyea5q7bZFoYMCCFkgGmjgyTBiy9Cejp8+mndGhZLc0tJPJBI9uXsm6TfDYSxC0XD07JJy9Dr9Oh1+lorxWTEx4P/vpW0ykni/piXePVVBSnScQrVF/HzdGD+G8Z5TqkULiibN8P+/ZZJv2f71y0f9Z+IgpQCOu1YQo6mG9xXu04Qb29Aksi5kk9eErgF2WYPqi3VgsqOE2ILbiD9APo9IyJeRhl6/trSdqJ5M39N8PQEncaRmAF30+YWy8coFApa9hX33crxFz5OPkxs27SqOt+K3ox086UbIJqIywrKcPZtfLvRpsL+hfspL9ZBOyPpZwkKhYLREaP58uiXbIrZZJX0k4+VoVJBeDhEl3UjM7grKGy397SE9MJ0Tlw9gQIxnsHfDeZ8xnkmt5vM7VGmTQ8y6ZdcYXIw5sMx3PL5LSaN4NGrornw9wUGvTqIwkIPwFijNsORpyFxJYw7ChqPur+JOmJg8BBGfp9Mq0DRZeYZ7sn036cbYhquZ5To1JQ6e+FaaRuYki+68iw6cABdAkTdUJIkcspyGnuIBthE+m3btq2xx3HD4GbmIaQcTcGnvY/NKpXGhr2bvdmmSaFQcNu3t1FeVI5SpcTR08rdAlNrT/l+cCb9DACd/esnla5M+Gn1WpKKL+Pq2pr8fPPO18wLmZxcepL2k9sT0PXG69qxFUunLOVuryUsOuSMR5joxukZ+wcA9lXsOrJjs0ncn0jE6Ig6ZbMMfm0wg14ddE0aGgK6BtBnXh+DwjCm5BA7O0+h5HA3NnWukIxLEpx9VxS5m4L0A5FXVHCZsDCRS3VOt4otB34lzCOMAX0HcNvXt9X6knq9nujoaKKiolCpms7f2lZ0nNGRjjNMsx06lNzPxcIjdAvsanisSxd4+y07ti4NpUATwwtvx/DFK8G0rFSPc23hSp8n+gAQtz2Oba9to7yonBFvGRV3eq0e3/a+qF1qnkO1WrHpBaONpyVERMCBAxAbK/7/8ImHLeYUWIJCoUDr6UeJq1hUSpJUvb1neR4UxIK6kdQhKntQu0FpGjgHo9GAm6uC4adjWPwwVN7XHPvmGDEbY7jt29uazT2ptpi9djaWmkbty/3pFvcDXbuakhdDWw1g/5FVnCvcg17/LNdawLByJXz3HYwfD488Ur9ryar7EjuxqDax0HEOhtb1fIEGhru7IP1c7cQ8nqvNZe5eC3LceqCp58+W/Vqi1+otdpzaiqDeQbQa2QqFUkH7Ke2JHBdZKzs5W5GbC598Iv49ZYroiP7sM/GZLCmBxx83HqtQKJj681SSDyebEH4g1G4dpnVg1+fi/25u4GrvyoyOM/B1agJr7YaEJMHfbcElAsdh63ByEp3C9jqxAZcbOipj1/xdHPj4APMuz6s29/BGgYO7g80q0saEyl6Fs58zujKdCcGVkgJldhkU6wpQoCDc00LW4olXwT0KwmZZvb6s9AOxRpjzTDeKt48iPrcjRzaIRia5u33HDvH34MGm95v6YFjgQm4Z+Sc7U3aAjdkmSpWS/R/ux7uNd4OQfs19/VkflJUJUjU6WtiVWSI5LKHbfd0oTCtE0ks1kvx6rR6FSmG2T8rKgkuXxL937ICJdaiDd5zRkU63d6ozIXQTzRszl8+sdxPJmTOQHdAOl7YtCQlR8O//FrP0e9EJ+e5zHU3UygB9+xpJvwceqN9cVphWyKYXNhE2NIyuc7rW/ULXKYqKIDWiP57t7NHr9bWaP+V7j+/Sj1hf1I4Zf86o/gREw8cnEZ/g0Lk1Japb8fCwvO8tLyqnvKi8zpm8cnN1UZGYQzVVlrrn/zqPs78zLfu2RKFQmGXeNzX8KriTtDRIzEtkU8wmgtyCGB0h4m+W3baMpANJvJT70rUZYCNg2q/TyEjRsWmhEGXY2cH5jPO42bsR4BJgYp85qtUovjz6JRtjzWOAKqMos4g97+4hoFsAUbdHEdlKT3S00jBJ1Efp5+vsS8LTCRxMOoivsy9T2k1h/u75LD+/3Iz0C6go8V69Kpbrlj7HCXsTOPrlUQa8MIDCCtdSBycdkqQ0r1lqPMUfde1yN+uMcx9A2g4YImqVvh6OOJQ7UlzRtKtx0dBhWtM3MzUGynMLURfr0KhdACWFZYXklgprNmuknwy9Xs/FixebYJQCN1dRN9Gg0JZq+X7Y9ywdu/RaDwWAnLgc8hLzrD5vS9G7ap4fiBsLQDufdvUan4wzaWdo+WFLhn4/FF9/4Sdd1eIzJz6Hnf/dScLehAZ5zesB0aujWfvEWvJT8k0eLy9wQ4EKd3ehpJBvkDL5IePiuossn728Xr7RjUH4Hf78MEvHLa02ByB8WDhjPxqLawtXdDrIKkuhRJNIdlmV93LrBei1pMHHaBX758LaKLzcinF1BbXWmKX2T0JE2jN0v7yUyABTAr5DB+jZKhI7vRNp+ZksWQKnfjnF/zr+jyu7r5hkBIUOCeX2v283y0pS2imZ9P0kJnw6ocZxJCQI4s/Z2agUtgR5YySrAm0l/EB0CeZdLQZJwsUF8svyKdGWAFbsPbt/AJOTwEqQcb3Rag5MOAVePQwPyfNzZqbpoclHkjnz6xlK866vfLvK8O/sj3+U+c9ZJsBcTfkJJkQNACDDaQ9xcdc+TO7wYfH3kSP1v5as9CtSCmlIgHMzbYBJ3gCbhxLVQtjrOSlEhSO7JPtajqpB0POhnkz5aUq97o0BXQO4c/2dBPUSrbNqR3WDF3YlSeSb5uZCSAjMng0TJsDTT4s9/IYNpvbUAF6RXnSaJRpadGU6SvNLST6STE58DmD8zrm5iaavX6f9yuLxi40X2HsnHHqUZg2FAjy7g5tQUslzp6pEkJfphebt4u2ntGfMR2PqRfTeRO0x4PkB3L//fpMO/bIycZ8rshc38yC3IBzsqqjUtMVw9h1I+KPa6/v7C9eG1q3hjTfA2VWD262rkFrcil4PP/0kvj8ffQRll1fzQPd5jOif1GDvz9clBX/nWIoyU8ye2xm/k6fWP8W3x741e27Ojjnc/ncjZAY3Mxz58gh7P9hb6/OuXIGXX4aZM+Gll+Dbb+E//4HTNkaNdr+/O4NeGWQTIXP066O85/0eCftM96ZygxmI3Om6QKVR3ST8bkAc+OQAF9ZcMHy+6hPrd/o0pIf1psMTwuJecktAUuhx0bjQIcR83dy1qyBx0tNF3IEllOvKOZF6guzi6tdraic1J74/0aSZm80JBQpXEtuNxLNX7dfhXl6AQkFqq/60Gm+bGq8kuwTfqEDOJgjyYuZMc9L2mwHfMN95Ph8Gfcjx74/XelwATk6CRFLotPwycRlrH19reE7SS6x9fC1/3v4nFZHOBtLP1VXE//xy6hcScpuuVldZ6ffTyZ+4b9V9LDlsrAu1uaUNne9ugjy3JoRve1+cI8XnTlb5zfh9BkEfBrExxpTcG9FqBAoUnE0/S2JeotVr2jnYceDjA5xfcZ7cK7lkvfoB3gnHAfF5taYmtBUt3Voypb2IapL/XntxLcXlxabvzVeoksvKRPOMXqsn5WgKaWeMDXkj3h7B8xnP4xHuYSD9Ljp9j+b/NMxePtv0haP+DeOOQDU5gg2K3LOC9CsXGya3Cq4xP//GyreXJPA+vYMuWxaiz6vI2y4Qa1kntROuGler5y4+uJi5q+aSXmJFntsIqNNv//Dhw7zwwgvMmjWLKVOmmPy5iX829Fo9keMiid8Zz6UNl671cNj19i4WBi8k90puna9RlfQrLi8mLicOaDjSr7V3a8r15STnJ1Poux0wJ/1a9m3JwycfrnXg8fWMK7uvcGjxIcoLTW07q+bbhYSAVlnI8lNrDSHKIMKep/8xHb9OtZeQl+aVsu/DfaSesOA7VAW1vYllXsgkflc8+cn5NR+MeL9lanGzD3Cr9F4UCnBqCS5htRtAfRAyDaJeRyGVERpqSvpJeokVd69gz4I9TTeeJsCKu1ew8TnjIlKSsG63Cqy+6w/OzC4gMGcq8fGgUCrQlenY9u9tvOf9nmHhplAoaDOhDSq1SliW1AFyYSU8vPqO1YiKqKwrV8RCsiiziOhV0WTH1kxCXFp/iZyX38Mr6RQuLuCicSFuXhwH7z/YbGw05W7erCrc85gPx/Cv8n/h4n+NWjDrCUmSKM0rNWsQKC4v5kzmCST0ZqRfz6AeqCQNZeo0tp+M5VqipATOix4Zrl4Vnaj1gUy6FFBB+lW29zz4MKyJAn3dvksNCm0B5JzCz01YQDpQQfoVZ3Psm2PsenvXtRxds8He9/ey862d6Mp1DX7tXbtg715h0/Pss8Zu7eHDoU0b8e/kZMvnluSU8M2Ab1h9/2rWPLKGJVFLRPNDJdLPInJOQd75Bn0fjYKBy6CHyGCX17bKYqPST6qyqAkdHErfeX3r3Dl/veHEDyf45dZfyL58bUl6hVJB2uk03vF4h/0fiwYC2QrT3c6ffw3+Nw/1eMj8RJUDTEqCbgvMn6sEOztYtAg++MDUfvGee8DLMYVduyQeegi2bIFuAZuY2PYTQoKtX6+2yGv1FnNWJhOdYG5PeiL1BB8f+Jg1F9eYPefbwbdah5YbBUe/PMqBjw7U+rzlywUZotWKNWpoqPj3W29BUsNxtoBQxfp18sO1helCJCYGToY8zK72vdia/Q0xcWV1un762XT2fbivUe4RN9H0kCSJDc9s4NhXIibl8O+xvBD8Cx++VVzDmZauJbL5ADpWmLHI1mqBLoEWG5Ps7aFbN/HvA1a+WsN/GE7Xz7uakQdVoXHR8HL+y9z2Te0dbm4EyGSXo2PtXc6cnMSaLKn9SILGy3Z3ohlLbk41O8fHiexxtxMfNoRWrWDcOPNj/Lv449vBlxY9WxDUu25+jAqFmDcllR25CbkUphUan1MquOPvO5iwZAIKpQK9HgPp4uIC7+15jzuW38G+xH11eu26oLLSb2jYUEA0zegl8Xvp82Qfxi+ybmt5PaI4u5jCPHFPcHAQ84pcm62ax+nl6EWvoF4AbI7dbPWaGmcNj555lGnLppGXmIejlyPl9qLhqr55flXRPbA7Ie4hFJUXmc0zdnbG32lKimi8/qLHF+x5x1hbUygUOHk7oVQpDZ8/rSYDrV6LSnGNHQt6fQrTcgzKQldXONvyeQ61vJukbFFU/1+n//H9sO+v4SDrj7IyyPcO52p4H9wDxHpUrkG3cG1RbWPsF0e+4PuT33Mxtxkr/ZYtW0b//v05d+4cK1asoLy8nDNnzrB161bc62LYfhM3FDTOGno+0hO1s5r8JNsIjcZE24lt6fdsvzpljMmoSvpdyrqEhISng2eD2TppVBpmdBDWBuc1PwLmpJ+9qz3+Uf5onP85ndZDXx/KsynPGuykJv86mYhPItiTJrquZMIlNBQOth7HG5cmsDN+p+F8rwgvOkztYDWIuTqknUlj47MbubSuevJ6yxa4887adbIO++8wXs57Gfdg65/Lvx/5m1X3C2l8bi6UqmU7xUqkn2yjqGtCFVPINIj6D2jcCQ8Hja6C9CvJQqFUcG75OeK3177rsTnbKiXsTSDliLETfV/cMbKU55HQWST9XDQuBAaKm31uLrSZ3IknLj5Bjwd7EDEmAp+2PibH67V6vh/6PRue2UBpfilf9/uaUz+fsmls8uZIJvWswdtbFKr1eqGITT2eyrLblnFhzYUaX8MtyA19124Uu/ri7AxKhZJQj1DDItoERclw7kPR6dWYuPKHyA6sgJcXnG35HHcdbMPSk0aluZ29HUq767dTvCSnhHfc3+Hvh/82eXxH/A5eTejK7va9zEg/BzsHWjn2BGDLhWtLwJ87J4qNMk7Z9rG2Clnpl6ezQPqpHEDlCEqbnOsbFyFTYVomaWoRCKLRG5V+J386yd4FtVdvVIemnD9X3L2C/R/tr/d1JEli++vb2fbatkb5jq5bJ/6ePt3cAkpWRVvKEgOwd7fHI8wDz0hPhr4+lJHvjkTtqDYh/YrLi9HpqxSix5+AEVsb7k00AWQCU1mh9CvXlxvsaf6pyI7NJmZjTLO4d5TklKBx0eDZSswhKRVLkdY+4fx32Bu8Nvg185MUCnAMABcLtp8WDq1amwjTf8c3t4XQyXcnhYXi+9N61kcwMRaFU8NVvwJbCMeBtDSxNqkM2RqpciOfDF2ZjozzGRRlFjXIOJrr+nPmipncueHOWp8n32effx5++AE+/BDathVF+jfeMDbPWEN+Sj7fDfuerf9X832q06xO3LvzXjxCPUwePxaTyBW/z8l1PszJ8Ln0/imSTw9+atZQUBNO/3qajc9uJPV4zQ2YN3EdQIK5e+cy9L9DAdi1Ohfn5IscXZ1k1rRXE1JToexiPG33/4Bjmth3phaIz4mJ9XsV9K2Iut5vZRkT5SdCS4+k1GxP8Y9Wv589S5uDS1Fk1H69oFAYmzWzK3prDh+GxYtFE4olREcb85kfflg0dFXFhP9N4NEzj3LfnvvwbV/3Gp1sP9rri/uZ/tt0ruy5woZnN1BWWEZA1wBajxM56oVGPhBnZ+P9KtCl6fKAZaVfbi5EeffAWe1MVnEWp9NslHZfZ9CV63jP6z32PLsCEARydkk2+WVigxjqHmp2zpvD3mTr3Vu5vVP1DgFekV4olAqC+wfzxPnHKAmKBOpH+h1JPsL4peP56uhXhscUCgVT2gmx1PLzy83OkU2kUlJE/XfkeyNN1JpXdl8xNG3Ln8FyO1GwNskZl/Rw/BVIMq0hNCpUDiaLSo0GEn2+I8nnR2LTKubn7oHXfaZfaSnkBLYnoeNYnN1F3aEy6VcdovzFPSa2oOkas2u9m5k/fz4LFy5k9erVaDQaPv74Y86fP8+MGTMICQmp+QI3OJrrxqEpoNfpKS8uJ3RwKK8UvEK3+7pd6yHRZkIbRr8/ul7XqEr6ydaebX3aNqj1452dxcbuaPGf6JRFZqQfQGl+KVmX/jk2impHNS4BLobCy+Xsy8Rmx1JUKP5fWennUiKsqiwtciR97fXk/lH+zNkxh44zO1Z73L59YgO9cCHs3FntoQZonDU12uZcPX6VlKOiupObC6V2QiJjYqeYvA5WRUDSKtteuIFRVekH8Hza87W2XVKpVM06T+XJS09y99a7Df9/duNTbO/UnqsBP+HgYPkcFxcMz8nh2p1mdWLWX7PMCoklOSUiE0WlIP1sOhnnMyjOtq3rVSb9qsvzA7H+konBmBgI7BbIxG8mGjYu1SGodxC5QyZS7B5Yc2ZB1hE49ixkNHKX45Xf4dTroBcqYG9vKLNLJ7X8Ikn5xlb2wrRCruy+YvPPs7lBoVTQ9d6uhAw0XV9tuyy6DNyKu5iRfgC3hE8jOH0uhQk1sMGNjBMnxN/yrbK+pJ9crOzdoj/jW4+njXcb45M9PoKxB+v3Ag0M+R5lV25U+k38aiIPHnmwwV6jKedPvVbPqaWnSNxn3SLHVigUCu7aeBePnn20wW20CwsF4QwwYoT585UzM6yNbdqv0xjx1ghaj29Nr0dEg0Nl0u/xtY+jflPNB3utVKiaM3JOw+EnIfOQYf4oLXDERSMmeEsWnz+M+IHfpv7WlKO8Zhj6+lBeLXkVt5ZNlIFSDYL7BzNzxUza3CLmOpmoDqyupph7DgoT6u6l5DMAvc8Qoro58tLcvXz4xiXatVcKErEBv6ve3tCtxRZGhn5ORobpc3LRRLZLqoyLay/yaftPubC65qalmtCc159uQW74dqhd4To9XZCoSiX06iV+XRoN/OtfQj2QkgL/93+mzThVoXB0IG5/Kr99X0RdI2fSY1vQ//wuAlUdsS8LIEuXwOPrHufLo1/W6jpd7urCXZvuwq/j9V0gvAkBhVJBUO8g/KP8KSyEffkdOT7qOXJ9I23eQ8s4cwY0JXm45CahkEQDjjxfVEe6yN+L2Fjj/qwyugeKrNCjKUdrHENGdAYxm6xI025gSBLocvJwybpC+zYhdZo/vbwg8MJ2ts4VjZpHK37cCQmmZBpAwv4kfhz5PS5ZVxgxAtq3r+87qB5yQ29egXhfV3ZdYf+H+00agMGodnRwAJVKsq3oryuDqg1j9YCLi1CwAuRmqxkYMhCA7XHbATj7x1l+n/67ze5SzR2STqLb/d1w7yqamhwdIT5HkP5+zn44qs1dAEZHjGZY+DDs7eyrv7ZeIn5XPLFbYrFTK2gVUf88v7UX17Lu0jrWXVpn8rhs8bni3ApyS0yJc/n15PXegOcHEDFK7OclSeKHkT8YbGfl70qpShSsvR0r5VGVZsLZt5uW9NMWQtIayD4JiLnWQS9qhYkVRfXJP0xm3CcWpLrXEUpEyg0ajXFZ7OXoxdjIsfRr2a/aczv7CQI3Q5lR7XENiVqTfjExMUyYIHKGNBoNhYWFKBQKnn76ab744osGH+D1htp2sN1IiN8RzwLfBZz9o5EVHjZCV9YwN1R5IyqTfpFekTzf/3lmdZzVINeX0T+4P+Ee4RTrC0j1WGmxGLV07FK+GfBNg75uc0b25Wwyoo0TYlqhIL6kArH5q6z0cy0W5NyZtDOG4yVJYmHwQn6d8mutX1vjoiF0cCie4Z7VHidvGCRJdKftsVFYk7AvgfN/WbcBm7tvrqEwnJMDZRVKP7/KSj/X1tD+OXCPsu1FGwK6Mtg6Bo69SFiYOemndlLXuoArSRJ5eXnNev6U35MkSZzJFExGiMay1W5aYRpTfpvMrjb9kZBIT4dzy89xaMkhi8c7+TgxZ8ccRrw1gpZ9WvJc2nM2NU1IktHesyalX+VjYmLA0cuRbvd2wyvSq/qTKlA5qHzr5a28uOlFVkevNj/QfwiM3AktGtlKpOvbcEs0VNhYeHmBWmf6WQQRuP7toG9JPmTFx6+Zw8Hdgdu+uY2u93Q1eXxbnCD9vPOGWSRi3xj3NF3iv0IXN8CgjrsWkEm/YcPE3ydP1r0OLUnGz+G/Bv2HNXesoV9w9QvrawZ9OSSsINxF3BCcy0OJ8ovCz9kPz1aeNd5XaoOmnD+VdkpeK3uNid9MbJDrBfcPrlc3tjWcOAE6HQQFGQm+ypCVftZIP8BiY45M+rm7Q0ZxBhKSgSijJAPifhHq++aO0nS4sAgy9pvkbeyYs4NLT1wys0eCinwt9bVXvjUVFApFo2Q61xZKOyXhw8INY5EtaYu9DhGbHYvWkp3x4cdhTQcMoUO1hVtr1GM2c8fclgwoGY7q4N2NEsaiVMKMqA94sMeTXE02dayorPSrOrcFdg9kwEsD6mTdXxXNdf2p1+nJicuhvLi85oMrQc7ti4w0zSByd4fXXxdqlHPn4JDl5Sjl5fDeh2qOjHiBpHYjq93T6HV6Vj2wymzfX1AA6WlKvAoGcvrR09wae5mQNGFDu/7S+lq9H69IL1qNbFWrHOqbaL7QlevQlog5a9MmKNZq0NsL2+jt22t3rTNnICsoirAvXiJ8mCAA3O3d6R7YnfY+1lkhd3cjaWTJ4rNHoMgLP5pytMZ5Ydtr2/hpzE//OPvZsjJIDe3LsXEv49XBuU7zp5cXOBRkknMmCb1Oz7FjxueqWnzu/CMNVXICGhc1995bz8HbALm+JEd5dLmnC4+de8ysAbPy3jivNI9irWgwrU5pyrn3YN+dDXZPVSgsW3zuiN8BQMb5DM7+cZbC9EIrV7i+YOdgx8QvJ+I9RnxPHR2xau1ZW0iSxHeDv+PXSaJuOGOGyAEdPrxu14vOiObHk8LBbUzEGJPnBoQMoINvB7oEdOFqoelmRN63VI4gkCSJC39foDSvlDELx9DjQfH+ZdKvSCHqpSZKP40n3HIeOr5ctzdQF5RmwY5bINZYr3ZSiCL61bwbR7hSWgqhJ1YSfMqY+Tk6YjTrZq/jnZHvVHuurPQ7lVrPTuhaoNa7N09PT/IrKkhBQUGcrlhd5uTkUFTUMDYb1zP0Vf1J/mEI6BqAb3tfLq2/RNz2uGs6lh9G/sD3w76v90auqtKvW2A33hv1HvP6zqvnCE2hUCgMar8kr59ITRVFq8rofFdnej7Ss9ltThsL6x5fx+ddPwfEzS69SDBs2lxRJJRVFEFB4FbSCYCTqUaln0IhOgpr2ykrSRKpx1NtUggaVFydhD3RggXWcwIqY83Da1jz6Jpqf5dykScnp5LSz7mS0s+ru8hscW+YbEmboNJA/gUoTiE01GjvmVEobuSZFzJr3fWo1+uJjY1tlvNnaX4p51eeJytGvL+EvATyy3NR6O0Idba8qXTVuPLX+b+4qtlHuSqL9HT4bepvrH10rdXPlFKlRKURBJZKrULtaLS9svYRSUkRnUYaDbQ0j8Qxg0z6Ve7c1mtr/pnvfGsnbltWgCTh4iKyAt7b+57FrB3UbuA3CBwb2drEpRW4tTEEU3t7mxPQIEiFMR+NwbuNt8XLXI/IK80z2A555w+zmC/m6irmRYAL9RdD1An5+caN+6xZwopHViHUBQUFxu+C2XvWFsGpN+HqjjqPt8GxawrtFIsA6FD8ECcfOcmrg19FW6IlNyG3zjmeVdHU86dSpWz2NuOHD4u/e/a0/Ly8obZm72kJpaWi0AXi85dZVMVKJ/cU7L1DKPCbO7z7wm3x0OYxg9IvP18oHCK8IlCrzAvss9fNZtqyaU080GuDs3+eJelQA4efNRDkz+yi9NuI+CTCshql1b3Q6d+G+2Od4dQCenwMvT9vUIVfZezNf4MXNu8h5aqpLbNs3VymKzO5pwO4h7gz8u2RtOhZf6vR5rr+LEgp4OPwj9ny8pZanSer6Tt1Mn8uONjYgFO5wC5DpxN7mGPHMPy+5cYdS8hLyOPYV8e4svuKyeNyM5qfnyjsDx3oQFDWbAAOJNU+o1CSJDIvZv5j9r43MuK2x/GW41sc/uwIf1eIT2aMyMAn5RQxlyQSa2EicKaixzcqSmFo0rm3270cefAI/xn6n2rPrc7is6NfR9RKNdkl2cTnVh9X0e3+btz65a11chS6nlFZiZeSUrf508sLLnefSvCiF8jKVprkjVZVGF/26MaJkc8y7t5AmiJNSrb3lEk/10BXfNr5mDWDyU2Vrq5Gaz93e/ea8+4T/4K0htuvyBaf6ekwJHQIADvidqCX9Ax4cQD/0v6LgC4WOuCuYxRXGPhUJv0sWXvK2BK7hWc2PMOmmE1Wj1GqlEz/Y7qhsbFXL3jzTePnwVYUlRfx6pZXiVoSxcWsi3g4eDCxrWmzpFKhZNe9u9h17y5T1xqMSr/KpN+55ef45dZfOLjoIL0e6UW7SaL2ZyD9pAqln1OleofSDtzagrP1n0uDw8Ef+nwD4UaHLGelqM+k5ooxnv/rPOufXm9oALkeUVICzjnJOOaYu1HUBNlC+lJ29RFSDQmbdwMyuTd48GA2VRgqT58+nXnz5vHAAw9w++23M8KSh85N/GMQPjyc+3bfh18nP1bctYJt/65FyFkDQ5Ik3ILc8Iz0rFenriRh8Jj3boKa8ewosSlKd99ArjbdrFjb8+GeDH19aLPoPm4KRM2OYtBrgwDIKckxdDSX5YjVjdyJpdFAe2+xw72cG0NRubEBYcafMxgxv3ZzU2Z0Jp93+5xNL1pfGICY8OUF3yuvwJAhYtP8/vsiN606jHhnBFN/nmqxEbs0v5Qzv58h86K4OebmgkNZS/xUrQlyq1swdYPitsvQ/wccHKC1azeGnD7N1wN2AbD9P9v5afRPDaa0vdbIjs3m10m/GjL2Tl4VdgUuJe3x9bRsE+GodiTIVfyeCh1iSEuDBw49wIw/Z9Ro61oZv/4Kc+fCl1bckOTCSmio5WyDqmjbVvwdFycK2Buf28h85/mUFZRVe17splgc0q6AQoGzM1wtkPMl/c0PLk5tUNsSq9DroPgqlIs2Sy8v0GjFJF25QOjXyY++8/oackGvN6SfTWfF3Su4vO2y4bGLmRfRS3qcdIE4lre0arka2UZLjtMRdp66bPmARsDq6NWUaIXnxalT4h4aHCys6NpU7GlOnqzbteW5VuOgpUyq0mRWlASn/g3Ja81PvBZQqmHAMjL9nwXEHC5j/0f7+SjkI9JO1ZH9vIYoyigifmd8g2VpNQYkCY5URPFYI/0qK/1srSPLKj+1WlgpZRSJrlrDBtu9Iwz8HQLHWLlCM4KdIziHgEJpKKDVlPP1T4Gkl/jz9j/Z+WYtveaaCCkpoFMWkVUuCg0RnhZk/uF3QofnG+YFWz8EHo3nJqH37MWlrJ6kpJouYuzt7A02VZZy/W50qDQq+j7Tl/DhNecyVoas9LNE+gF0qzCRqEr6SRIsWiQiC+zs4KEp6fhdPkDc2SKrbgEeYR48n/E8g14ZZPL4ptNHOB52D6XhfwFCJeFR1IM26S/wwYhF6KXaEQRbX93K4jaLybyQWavzbqL5wcnbiajZUaQrfLh6VZAl7ucPEnZkOeqSPJvVfllZYi70TjqJS9aVmk+ogj59xN+nT5uuzwA0Ko1BiVGTxWfkmEi6z+2OnX0zyJJuQhQUgEtmHP75F+vcDyITKZmZ5vPRpUq1cEmCs2dBp3E0zF+NjapKP2uorPSzNc+LNo/DoL/Af2g9RmiKykq/ni164qR2IrM4k4uZF1GpVShVN45LQ+bFTJbPXk76PvEhcXTEQM5Xp/RbFb2KhfsX8veF6q0uO0ztQMfp1Uf7VIes4iy6ftaV+bvnU64vZ3zr8Rx+4LBpBn0FvBwtuy1VJv3kPUrbiW0Z9Oogus01/RLImqsCnQWlX1m2qJXU8p5bL6g0EHGvECZUwE0t3md6hUDg8tbLHPjoAMVZ12f0Cog62tkhj5Az5T7DY+U625wZWrq1xMPBo9ZrofrA5hmgc+fO9OnTh6ioKKZPnw7Aq6++yjPPPMPVq1eZOnUqX3/9daMN9CauL0xYMoGhbwy9Zq+vUCiY+stUJn5ZPwuqvDxj7oGXlyATd8bv5GrB1UbpOGzr05YFoxbwuMMBNFofQ+Hqn4qoO6IY/OpgwGjt6WbvRkmBCEur3O01aZQfmnIfJCTOpJ2r1+vaOdrR//n+tJ3YttrjZOtXJyexcXn6aWEDUFIC8+ebe9JXRutxrQkbGmaRBMqOzeaPGX9w+hdZSQ3d4n5gcdsLDA4dbDzw8DzY3bA2s7VFCx9nXEs6oiwSi5mu93blls9vuWE6ct1aujHph0mGz0JCbgIATqURhk2BJUR4iUJckf0l0tOhRc8WtJ9iewjBkSOwVMQcsHo1FucCWUVli7UngI+PmMd0OrGh8or0InJsJCW5JdWeN33NHM4OEtZMLi4YbChM8iVlrO8JmwbaNqD6IGklrAiAxJWAqdIvs+jGsY/Iic/h5I8nybmcY3gsMU+0QjuUCXmnJaUfwCb7h9ndoSfLLzf+2qykBIa//AkTl01k1h+z0Oq1BnKvS4ULblRF3biuuX4yKaH1PoHzfGfaLq40PzuHwNij0PqRul28MRA6E7W/yIKrXFQK6hNEn6f64OhlnjvR3BG/M57vhnxHzAbram6tXss7u9+ptY1bQyEuThQE7e2ho5V9u4+PsBbUao2NXTWhcp6fQgGZxVWUfg5+EDINXCPr9waaCsUpkL7PoPTLy4PNsZt5dcurrL1oTp5nxWSx6+1dXD1ZjSfqDQBJkpj26zT6PdP8rIO1WlHUK9SIjh93e3erRaPrBYGBYKcsozDdvHBf2eKzKna8uYMlnZeg1zUvhV5DwdnPmTEfjKlxH1IZmZmCCFEooEMHy8dERYkmsdRUcayMS5dgyxYxL770EvjkxhByZj0O+WnV3rOdvJ1w9nM2eWxD/CoSfX4g1mkZAO3aQZCfE23i38U/awrKWipQW41sRe8neqNSN7/cxZuoHQK7BzLlpynsviKUJ2PGQPf7utDx9Wno1A7s2GFbI86ZM4AkEXZyFYcX7qr1OFq0gNatxX5oiwUxbfcA23P9/okoLISg6O202P9Xna/h5QWaohxy9p7l2G5RMJFtV2WlX2l+KZ/1/gbVxfPY29u+360v5P19dnb1x1Um/eQ8SaukX2ECrO0CabugRUVzmKRvEJvPyko/tUrN+tnrSX8+nbY+bSnOLiZuR9wNk+mXl5jHqZ9PUZAgfjmOjsJW8bl+zzEi3HqTf5+Wgunfn2RB3tuA2Hp5KzHZMbhoXFg+Yzl/3/63oSZkDZlFmXy8/2MDCRQQIO7jJSVG4lmlVjH8/4YTuzmW/3X6H8mHxbpI/gx29enP4NDBpp+/i0tErSTr2s5jHhrRwCU7pAx6dRBPxT+Fs79zdac1a5RWONLbOxjruFFLovB815ODSQerPVehUNDJ10pnViPB5lXXjh076NixI2+//Tbt27fnnnvuYc+ePbz00kusWrWKDz74AM/aal9v4obB1n9t5ffpvxuKxx2mdTD4q1/PkK093d1F52NSfhJDvhtCy4UtKdfXLmfBVjzX/zkm9e6BAoXBokpG+tl0fhz9I6d/PW355BsYsrWnl71Y2ahUIptCxqhR4FEmJtCV+4w/n8tbL7P28bUUpBbY/FoeoR6Mem8UoYOql8PL1p7yYkulguefF/9PToaFC6tfy0l6iYKr5uNyD3Zn6i9TaTdZSPflG74ZyZR3DnKq8d5pLGQcgEtfgSQZiFd5jBGjI+jxYI9adz06ODg07BgbCE7eTnS5q4vBFkMmn+3L/au1e5C77wvtYywGxVeHtDShFpUkUZwG+PhjcyWGTPq1amXbdRUKUXwBiI4WyuFZK2fhFmSFNapAQQHo1A5oNEJVayD9qir9JAlCZkDwFNsGVB+4tYfWjwqbT8R3o6rVLEDamTQ+6/oZR7++PjfukWMjeaXwFaJmG5UWMumnKRGknzWl3+CI3gBcKN7TGHFMAOglPVq9luPHoTi+M0q9PSujV3Lfyvs4fkJsXjqLvGoT0q8u4zGoDVzFxtqQpQagsgevbuASVqf30ViQ58fE4gu0/7Q93T7vRviwcMYuHItnq4ZbMzfV/OkX5cfYj8fSopf1TuY3d7zJy1teZtYfsyjTVa8ibgzI66YuXYQqzxJUKuPcWl2uX2VUJv30kt6gKJbVSNcdDj0Cmwfh6izWsnl5wv5o/u75bIzZaHZ4dmw2W1/ZSsLehKYeaZNCqVLSfnJ7woaGXeuhmCE9XRSpy1wE6RfhFWHu/BG9CNZ1h7xr5OtcSwQEwOJxnZjsYZ4D/Nesv7j63FVGRYwye66soAxdmY7izPp3ijfX9WdtIav8IiJM90eV4ehoLKwfP258fFuFOc/AgUIF1W5yOzyfupsit0CrFp/p59JJOJbB4UOSyT39RKFo+BgZJgrbCgXIZlCbqjdQsYjw4eGM+2Rcg94zb+LaIS5OOD4olTB+PAT1CuLWlzqicbEnNVXsT2qCsPaU8H78dhOlaauPW9FmURtis2vO1h07Vvy9fr35mnR6x+m8NfwtJrWbVO01EvYmsKjNIoMbzD8FBQWQ3GYIRcPH13n+9PIC55xEVH/+zpWVYo82rcJB/OpVseZPPZZKxpmr2JUV0b69qMU1Barae1pDZdJvRPgIVs1axSuDXrF8cO5pKKzU3FKUBFtHwuUf6j3eyqQfwKDQQYaGtMT9iXw/9Hsurrto5ezrC+HDwnm15FXUvYXizdERxrcez4LRCxgTad1po0+QIP2Opx6nVFtq9bj6YlqHaSQ/k8ye+/Ywuf3kGt3ZtHotnT/rzFMbnmJV9CpA7F1k9WZylZ4ntZMabbEWOwfxZZCVfu/2/5odc3aYWoV6dofIh8E5rCHemu3YOQXWGLsuPR1M41dc/F1wD3G/rhWoRXnleKScw7Eww/BYcn4yOSU5eDh41Hj+D5N+YMdtTRdJYvNPetCgQXzzzTekpKSwaNEi4uLiGDJkCG3atOHdd98ltTbBGDcwVLZ4rN2AyI7JJvlIMvZulu3umhr7PtzHlle31LsDtGqe3/mM84Ao6GtUjZdp071CER0TY9plpHHVEL8z/obp1qkJyyYtY+3jouNcqVDSs0VP2roL2Yi7u2nEiIMDzAh/jKi4z0k9MMiwgE89kcqhTw+Rdcm2dn69Vm+zSq0q6Xc+4zyRn/tRMPwB1GqR7ffrr9bP/6LHF/wwwnyx5+jlSKdZnfCPEqSKrBIx87EfvhEmnLVprA2KmC/h4ANQnoOnJ8T4L+DTmKdJLajbfUClUtGuXbvrYv40kH5av2qVfpFeQu1RZB9TqwyzsjJ4+22xkWjTBhYvFvaI2dnw6afGjakkGe09bSX9wGjxef68bcfH74wnevUFFDqtgVySfwZ+zn6mBysU0OPDhrMVqw7u7aHXp+Ar1BhKJfg4+eJYGoqX2khIKFVKygvL0Zdfn2oAhUKB2kltQqL3admHV/r9lxZZMwHrpN8tXQYAkOFwgLSMxmlS+e+O/zJ+6XgOnc7CJ38oPWL+QCHZ8ePJH1mvfBIUkoHskzfsGRm1y1KTYVD6OYngD5NuxrIcKG9m98V9c/Dd4QFI6MvVnM84T3SGDdWsWqIp50/v1t70ebIP3q0tE127r+zm/3b9HwC5pblsj9ve6GOqipry/GTIuX51If1ySnIMHbkGe89Dj8GffsJO53pA+D3Q7QPcXIWdRX6+mEPBOMdXRss+Lbn/4P10mtW03alNjebsUiDnXam8RcePRWtPfRmUpIFD7bKsrxUCAmDL5TnsSZhuVnlv5dkKP2c/i8qwUe+O4vHzj5upzGqL5rr+PPP7GX6Z+EutLC2ry/OrjKoWn1ot7Kxwsx0+XPztEepB75nh6NX2Vkm/LS9t4ZueS3jjdYm//hKPJedkkGZ3CIAZPYwF2JEjhS3t1viNLNz2nc3v6SZuLBz/7jg/374KVXkJ/foZ98/29tC3j7inygR0dTh3DlAo6TYtgpCBIYDI/7ycc5mLWRdxs6++mRFg8GBBGKSkmDtQDAsZzSuDXqFni+oXEmpntSi+/zNSVwwoKIB8n3Ace3Sq8/zp5QXZgR2I7zSeK0H9cXAQ9a/Aikj4S5cgdHAo9i89TWZQZ6vODY2Buth7BroGcmvbWxkePtzywS3GweRk8TeAnRPkRUNu/es4VUm/yvDr5MfYT8YS3C+43q/TXGBnb0eJVuyLHW00TWnl2QofJx/KdGUcTz3eeINDOCF19u9s07F2Sjvu7izy797d865hDWop1w8gsFsgXeZ0wbej+KXLrmIWawEtxkLvJeDgY+HJRoRTS3A2Fqdua/EYI08kcYtS5NxrS7Rkx2bX6DTVnJGXVEDkkd+wPysWUvml+eSXiTpEoEtgjeeHe4XTtUPXxhyiCWpNrzo7O3PvvfeyY8cOLly4wPTp0/n0008JCQlh4sT6WSneCGhuQeCNibJKDdxTf57Ko2ceNXQzbP3XVt7xeIfCtGr8DRsRp5ae4tTSU/XuILBG+rXzaVev69aElPJzxHSew9mWz3K0kkDFPdidF7NepN/Tzc92qDGQdSmL3HjBePUP7s+hBw7xVpc/AQuqN+DN26cRkfMgV8+3MpAaXed05ZmkZ2jZr6VNr3nmtzMsDF5oFkxvCVVJv0fXPEp6UTq/x3zFLXPEAJYuhe++E93ZVdH2trZEjo2sMQD8cuFpNncO5pljo82fvBb5jq0fg6HrQOWEuzvE+S1mY95HJOQmcGnDJRa3XczFtbZ3lOn1ejIzM5vl/Hnof4dYGLyQlGNCXTSl/RQGlMzHJ2+UbUo/hwtkZNiubPruO7HRcXUVFkvOzvDss0KZsncvbN0qjsvKEmSwQgFhYba/H1npd/68GNPO/9vJ9je2Wz1+/8L9bH94GQokQ+e4IdPPkr3nNUQb126MOBXHe12M1nQ+7Xx44uIT9Hy4BgagmSIvKY+kg0mUFRpvuD1b9OTRjv+iRfZMnJ2t5zlGBbRHo/dArypm54U6BulZQWx2LE+vf5o3drzBpthNbIoVyqAo+1voevkHkBTE+31KXNQDODgJUsHe3kg618XiU1b6lWoE6dfStdKcfuZt+N0N8psuFLtGuESg8B2Ak0MZap2YLIq1xaTFpLFs0jJO/tQwv5PmMn/mluRy5/I70Ut6HO3ELvyv83816RgKCowNDT16VH+snOtnKwFdmfTT6rXMjprNuMhxxgYw51DRjKCu2p3TTBE8GdrNw81T/K7Ky8HLXvxQLDXw2LvZE9QrCAePG0MVZQ1HvzrKez7v2bQGbChkZcGbb8KuGlzqDDnfXoL0a+VpoeOn/bMwORE014cqKiAAfj/7Ct8f/Q8FhU2/nm0u82dVZMdkc2ndJfRa28dVU56fDJn0O3FC7E2OHxfrSXd3EVEgo1NHCZW2hKQk4364MsJu60xim2GgVPLHH8KK7NdDm0Eh4VEaRccQY2OOjw9EdI/nQJsxvLTrMUNGO8BPP8Hrr1cfiXD8++Msbrf4H9P0eqPi7No4yvcfQ1IomVJhCiJJEovbLsZx+c8A7N5tjFaxBJ0OrlwBhU5LRIRxcyXvTeyUdjbZHjs4wNCh4t/r1hkfP3EC7rgDvvii5vcT0CWAR04+QtTtjZd72hwhf1ednOo+f3p5AQol6WG9kJSqCuthidatxfPRZ8pFnl+sA5LK7pqQfoWFpvXOqpD3JdaaL81g5wjKiiZOjSdMOAPd3q3rMA2QVWHp6cZ6w4I9Cxj2/TAS7RPp80QffDtcH41ANSEvMY+EfQkU5YhfjMq+hH0J+0jJT6n2PIVCYVD7HUg60Chjq3xfqw3m9Z2HRqVhf+J+jqUKEska6efZypMh/xqCQqFAkoTST0KPk1Mzaljr+QkMXW34bwtPLxzKW1BcIMRBMRtj+CTiE6JXNnwjbFNBcnQittsU1N3ExCTb+7pqXHG1d63xfL1eT5at+RINgHoxIpGRkbzyyiu89tpruLq6smbNmoYa13WL5twh2pA4dQpmzIAFC0ShAEDtaPRRcg10pUXPFujKLTAdTYD7D9zPnB1z6n2da0b6FaRwTvM98b5fsPew6QZH7WTFr+oGxKOnH+X21bebPGZV9YZYpMldqn8KbhBHT0dcW7jaTABLkoTGWWOTjYxM+vn4iCycbXGiPdFZ7Uz7btnMmGEcy2uvmWcHDX19KKPfH22W67f9je18EPgBWTFZSBJklKRSokkks6zSgqY8D+J+Fl1iTQ2vbqJ7SGWPhweoK2wVM4szsbO3Q6VR1UplK0kSCQkJzXL+1LhocAl0wd5VLFRGtBpB69SX8S4YVC3p17NFT+yUdqj0LpRr9TXmAoBYqMsh9vPmGcnkiAixAQVYtAh+/90Yct6ypSBTbEVEhCCJsrOF4ur8ivMc//a41eOHvTmMTv+egl6lxsUFisuLDZ1MZvaeMd/A7hlQZJ6/0yjYeyeceNXwX6+KPb6l4tT1ilNLT/FVn69IP2PavikTENVtNJUKJS0RG5xdlxsmw+BE6gkm/jKRyE8i+ejARwDR9BM8AAEAAElEQVQ81ecZ7M6LbNHXXoOB7rfTOe4rkJSc1nzNtN+mGVRR9cn1kzfXxWohd2npVon08+oB4XeDY83ddU2GqH/B0DW4uNljp3NDUdEKnlOcw4W/L9RKwVEdmnL+3PDMBj7r+hnaUvON7eoLq4nPjaeVZyt+m/4bj/d6nDs739noY6qM48dBrxfqaD+/6o+trdJPXnu4uQmV809TfmLt7ErZdx1egJE7oJZ5Vdca9vZGG1QPZRAgrOwtoby4vFZW6dcjHL0c8evoh5OvU5O95nffwcGD8Ntv1R8n5xyVOFaj9LvOYG9vtFKrSsAfTTnKvHXzWLBngdl5eq2e498d5/xKG20LrKC5rj8HvjSQ18pew6e9bR36WVmQlCQawWoqjkdEiMayoiJBJMvKqsGDTZuIvur0MVEnBBFz0kKPSox9B1IjRYZzXh6sXQtrLwhrz/bqsWY9ibPHtsVO606ZVMSxZMFQHjkiHFGOHIE//qhm0BLoSnXkJuRWc9BNNHek9L6NY6NfoHtfNW0qXOgUCgWBPQIJ7+WDu7v4LFXnRpKSIkjBkNgdfNv6bbJjxQZLLroGuATYnBspW3zu3y/u8amp8O67gsDeeTyBP8/+ybn0c3V+vzcq8nP1dN3wHop16+s8fzo7G9ceCr0Ov1NbWP/Uelq3Bp8rR4l+8n9se/8ImZnCJURuGmwKODsbrURzq5lyZKWfqyv8cuoXfjn1i0WnBI4+K5oTq/6cNB4NMl4vLzH3a7VGh7BNsZvYHrfdUJu6UXD619N80/8bSq6In3MGF+j/TX+iltRMvDc26Tfh5wkM+35YrbNAA1wCGBo2FIAjyUcA66RfZRQViY9UjvNB/BZr6PVlL9MD9twBx63YzTYh5OxweR/v086Hfs/1s3l90xxRrrQnKygKx1biFyWTzlYzPatAkiTWnjTPT28s1HlnunPnTubMmUNAQADPP/88U6ZMYc+ePQ05tptoxjh9WnRa7dwh8WbfNZz83ZR46PVoL+7efHeNWVGNBaWdEo9Qj3pf51qRfsPChhHm2gadqoBVl38xUYnpynUcXHyQw58ftn6BGxjy4suatWLogP1c8fmKvYcLSEoSuXnZsdmGTUFN6Dy7M49HP45ri5q7NGTSz8tHx7MbnwVgTtc55L+cT7/gftx1l1BrOTqK78y8eUZ7purg7OuMR7gHDh4OFBdDIWJhE+BaqYqZfwn2zoYrv9v0vhoFkl6QflqjV3fY0DAeOfUIbW9twtV5I6LL3V144OADeEWK9yhJ1WQsVkKEVwSZL2RyS9ZGFChtyvXLzBQLIpXK2IktY9o0GDJEzLs//CDyIgEiI2v3fuztIbwibvX8eZi1ahaPRz9u9Xi/Tn649BVt4y4uYG9nT/xT8Ry8/6C5fU7OSbjyB9g1UTDz1e2QaQxLlufpyqSfpJc48uURLm1oRgqwWiBsWBgj3xuJe6ixy2H3ld2cSj2HXlGOWw232LbOfQE4cnVfvcdSVF7EyB9HsvrCaiQkxkSM4e/b/+ahsPcpLxeL+rAwePllaFd8Hz1j/kSjtKdfy36GAkx9SD+Z6MxXCEIiyC3I+GToDOj3fdN99moBd3dQoMRV7QGA1k/Lv8r/xbD/Dru2A6sj9OV6i5mtd3a+k7V3rOXnKT9zS5tbWDR+Ef2D+zfNmPRi7txX8TGvydoTjEq/2pJ+1c371xVKs2B9bxQnXzNsyN0UgkhPzEu0WMRbErWEn8b81JSjbHJ0mNqBOTvm4NO2aYoRly8bm32Sk627AkiSUel3X5eHeG3Qa/QLruL6URgPp96AnOsr97t1yxT+PXgCXFxi8viV3Ct8cvAT/jz3p9k5CpWCdU+s4+Cig2bP3ShQKBQ15gHJEBlnYn1Xk+pEqRSZpyAcJPZX9AQNq3JLajelHd49wwDMLD61WthYEf3Zq6LO+MefEgcyNwAwKNA8W6lXTyW+5SJr+Nfd+ykpgf/9z/j8ypWW7elArMXnXZ5Hyz62ubbcRPNDUhLs2KlAp3Fk9mzTz/XUn6cy7uOxhjXiuWp4tvh48bdLiCfB/YMN+3VZoW6LtZqMVq1ElIJWC3//DW+9ZSxM79a8xrTfp/Hbmeq7MY59c4xj3xyz+TVvBBTkaCly88feu+5rboXC2KwpKZQokhJJ2JNAeLCWfK9QSjTu5DmJ7qzWrUWmfFNBobDN4rOyvedr217jjuV3cCGzSp6urkzsjVM3W3ZnuvgZHHmqXuO1szPugeU5VCaRdp7eyZKoJWz7941B/oUNDWPU+6MorshNy9aLCSHMI6zGc/u0FKTf5ezLDT6ujKIMtsRuYXvcdtzta+/40d5HhO3KtWZbSD9Zcauzz0Cr15qv29N2QvbxWo+l3sg8DCf+BQVxAJSqUzgT/DQblPMA8G7jzegFownqFVTNRZo3SitiIeXG++R88YuylfQDcNM0HU9SK9IvOTmZ+fPn06ZNG4YOHcqlS5f45JNPSE5O5ssvv6Rv376NNc6baGaQu0g0RTkojx7mlzcv2aRkaQqkHk8lcX9ig6gMZWVWU5N+CoWCx/s+BMAFt8+IjjZO4ko7JXsX7OXAR43TpdJcoC3VcnLpSYOt4tyVc4n4JIINScsAy0o/gEe3T+Nk2APkOZxiwwbQ6/R8EvkJm16oQ3p8DZAXVvuKfuDk1ZO427vz/qj3TTbpAwYIgiY0VCwcV640vca6J9exfPZyk8d6PdqLuXvn4uTtRG4ulKpFVdKE9HMOg0F/QvCUBn9fNeLqdvjVCWK+xsMDNJVIvxsd685tI0t9Ar2ivMbir5u9m4nlRk24XLEGbdnSfHOjVAqbz3nzhCWNvNCrTZ6fDNniMzoa3ILcLBbwAUpyS9CWak384pUKJSHuIfQK6mVejOrxEcwsBk0T2dvdFg/Djd9rb2840Hosz8S1NszVKGDNw2s48vmRphlTAyOoVxADnh+Ai7+o4kmSxOgfRzNzZweKNVdqLO519xNF4ejC+iv9fj71MxlFGYS6h3L+sfOsv3M9E9pMIDpafA7atRP72cBA0Sn93n2TOPPoGV4Y8ILhGm3bimMyM2vOyqgKmfTL0VtQ+jVH5JyCYy/S2kdIJJyUHuLh0hybC7nNDWM+HMOjZx61+vy41uMMm+qmwNWrMHcu3HabUEPLuVS2kH6y0s9We0/58+ruDnmleeY2PsdeEOr76wUaDyjNAL2xecBJKzbgReVF5JTkmJ3S5e4utJ/avunGeINBqzUn9b77zvhYWZn1tUJamihG29nBA4Mm8ebwN+nkV8XHMX0fnHpdzD3XEbx8HekSsAVdbpzJ43LxXi6mVIZCoWD2+tmM/WhsUwyxyRG7JZb4XfE2H29rnp8MOTv+77/F5y4oyLyJbOzCsQz9r7BPOXHC9LO74Yt4Alf8jxb5F3jxRXHfzyhKR1Hoh0rnzJgOA81eU6WC3i3E/WHT2QMsXSo+176+0KGDcA768UfL463qiHIT1x9++QUcs5PoHZFptWFR3p9UR/rFxYm/W9zSnbs23iUy9TAqLQJcAmo1Llntt2yZuLZcX3DJFx7hR1OrV+3sXbCXve/vrdVrXu8o0mq40O8eWswy/57XBjLp5+2jYPZf05i7by5tOthR5urN6V5zOJkp1iRNae0po3akn2Rd6aPSwMRL0O8HyxdJ3QSXvgBtUb3GWzXXr3ugmORjCmKumfNaY6BFjxb0f7Y/xUqxAc7UxQEQ6hFa47mDQgYR/1Q8e+5reKHSyvMr0Uk6ugV0I8Kr9i4MMul3LkNMfkEVfFh1zWBFFR8ZhZPodvZxqtKsNjkRhqymyZF9HM78H+SJ92LnWMJl/4847/Rl04+lkZC57SSdN31ouCHJ69RAV9ubTsJcwhphZJZhM+k3btw4QkNDWbRoEZMnT+bcuXPs3r2be++9F2fn5tdZfRONC5kMGz7Vk7ipzxLtN4h//9s4KeUl5rHt39uI32n7hqWhsHfBXr7u9zXlheX1vlZlpV9+ab7B8qitd+OrmOZ0vQc77MlzPsZvew4ZHlcoFMxaOYv7D97f6GO4lijOLGbFnSsM1oNxuXHEZsdSWCRs4qyRfnIRJM/xNFeugEqtYvBrg+l0e8074T0L9rD28bWUFVRj4F4BSRL2iAAn8rYA8Nrg1/B2Egxxfmk+iw4sorCskKAgmCXc7wz2TDKyLmZx9eRVq7l+OTlQaieUfiZ2ivZegvBz71DjWBscDv7gNwQc/E3tPYuy0Gv1HPz0INGramc76upas7LyWuD498fZ/5EgTLR6LRN+H87Ojl1Ru+bY1HXo6yt+f6lXa7Y7jY0Vf1sj8hQKGDkSPv5YdD3a2RkLN1Wh1WvJL7WcfSJbpJw/D7oyHUkHk8iIzjA7bs+7e3jH7R2yL4mJ0KbMAlUtvEbrC6VpmJ2XFxTZx5Khv0R6odj1KBQK7txw53WrqqqK7JJsirXFADiUtaCmr83AsL60TXyLkUVf1tu+bFqHaSwcs5A3hr5BWx/jPVC2YWpfiQsICxOqgUjvCAPBJUkSao3e0L14uZaNlnL3dV+fMYxvPd40z2r3DDhrbgF3TVFwGc69R5jHcQCclMLDLrs4myu7rxC3I67BXqo5zp86vY7tcdt5ffvrjWadt3OnKBrLsLcXCpYONtwWZaVfZqbRpr46VLYWf3TNo9j/nz3/O1QhU9GVwrkFkPR37d7AtYRCCbfFQrd3DaRfeZEj3o5iDWPJ4nPIv4cw5N9DmnKUTY5d83dxaMmhmg+sJcrL4emnYc4cOFYhCjlxAo4eFfdyeU1rzQ1CVvmFhxst0cwQPAnGHoZAC/nPzRie/u5M/a2YzZmm2UZy8TSlIMVgEV0ZIQNC8OtUg4+vDWiO8+f6J9fz94O2zSclJXC4wvzFVtJPzu6Ts9OGDbMsQmnfXnw+MzJMFQcHd5eh0Ono1kuFvT3MnAn2Wj8Gnz3ByJOJtIu0vBac3k+QfrFlBwxNkI8+Cg88IP69fbtxLVwVSYeS2Pj8Rpv2aDfRvJCQIO7XkYeW4b17pdnzmRcyWfvEWnwKRd3o3Dnrhe4rFXGroVVq/HVR+gEMGgROFW7Odnbw6qvCucK9SGywarLqm7J0CjNXzKzVa17vMGb6WZ8/y3Xl/HTyJzKLrFvZy9bOXbuCi58zKrUKR0fR/ApwqOJW3NxJP70m17A3s/j5U6rByYqqqccnMDUD7OpnKV61yVh248kll8fPP37D7INlFIsfN2llcQCEuYfVeI6j2pEQ95BGabz845zwp57WYVqdzm/va6r08/MTjTJlZdZjS+TPn+QkajhyDdIEVWolTYLgyTDhLPgNBaClt6gT6pTFFJQUo9fq+fmWn9nx5o6mH1sDQatQU+bojoO7WOu0dGvJ2Mix9GrRq4YzjXCxOQy0/rCZ9FOr1fzxxx8kJiby7rvv0rYpjZWvI6hU1+CLdQ0gq/p69oS3F7mgc3YjLs44KRVnFbPzzZ1c3tbw8uma0PORnoz9eCwOHg71vlZl0k+hUPDZhM94eeDLeDrWnPdWX3g7eTPMT4TC/X75M5PnAroGGDLGblQ4eDgwY/kMut0nfA5lj3RFkVjVWFNZdfQVK8MCxzOGDeqw/w6jw9Saq4Cpx1I5t/ycTbmJ6VmllJXrUSjgl+k/svaOtTzR+wnD88N/GM6T65/ku+PfARiyC+LiTEOhZ/41k0dOPWLSxbr7nd0c/VpsMnJzoaxC6efnXKm4cS3zR9zbw7B10HIi7u5Ge8/0giwUKgUbntrAsa9ttzpRqVREREQ0y/nz6BdH2fOu6AjLKKogxiQlfq41h8RLksR3ulFs6hLAkZSalWZyoUO237SGFi3ggw9E12xIiOlz5bpy+n/dH/WbanwXWA7tljtpY2IgO6GAr/p8xcHF5hZZPu19iBwXSUnFfOfsDDvidvDCphdYcW6F6cH6cohbBnkXzK7TaMi7AAl/CesUxDyttqA6bTWyVYMUBq8FNr+8mS97fYmuTHRpJuWJQryL0huV5Fgj6RfZ0p3Wqa+gSR5W702Oh4MHT/V9inu63mPyuEz6tatGAP/NsW+IWhLFr6d/NXy+60r6vdz9Q9bcscZI+unKBNmS3czslfyHwa2XSNVMB8BP2ZZOfp1Qq9SsvHcl6+etb5CXacr5c9/CfVxab26V2/3z7nT7vJtRYYtQi439aSxv7HiDM+lnGmU8pytcDOfMgRUrRCbU//2fMYulOri7C5JQkmxTYlcm/a7kXkEv6Q0EGUo1TEqAbu/V6X1ca8jzSF4ebLprE5fnXW50R4vmiv0f7ef0zw1vj3nokFj/ZWXBv/8tFH7ffiueGzfOSFQnWY5TNJB+fhFJbIndQmKeBXZQ5SDyTe0tFH6aMXx8FIDCrLAlK3a0eq3FwrEkSZQV1o8Aaq7rzxHvjGD4/OE1HidJohEsPV0U0GUyryb4+hoL6wBDh5ofk3wkmb/v/ZMOrgmAyEwFoY4+mt+a08OfYNKzEYbzAytq3W4aD8O/q2J0B0H6FTieo0yZw+DBopYQGSks7CUJvvnG8hYnemU0+97fdzPX7zrEL7+I36nDmMEMeta8KFqaV8qhxYcoj4lHoxHFbGsNEPHxoCorJuPLFSYNpj5OPvQI7FHre5eDA9xyiyC9H31UEN3e3uBW1AUFChLzEi3ntFUgsHtgk9lBNxcUxqXTInobqowMq/Pnriu7uGvFXfi978eQ74awM36n2TFDhwrXhQkTTB+vrARVKEybCpsKMiFZnZOZTLoUKETBycPBA0e1o/GAxJUiU620Gickp6B6E35grvRz0QhCoaDsxsph3vDsBr7o8QXFhaIR6GqJaBSwRenXWMguzmZz7GYAprafWqdrdA/szu57d3PoAcF0q1TG5kRrFp+y0k+nqVD6OVaah4oSRY2k2EY7k4aEvbeoFdqJ70ILLzcUkpgjEjKzUNopubL7CpnRDZNtfy2g6NCe8wPn4tZWLHZmdprJutnreKrvUzadr1KpaFUXu646wmbSb9WqVdx2223NblHc3KDX16zmuBGQlQWa4lxyd53Ey7nUsLiXF2jebbx55PQj9H2q6S1fQwaG0OfJ+ttLlZUZi4ze3uLm+VDPh5g/Yn69r20rnh3yMADn7ZYRl5pj8lxeYh6nfr6+LHxqA7WTmvaT2xPQVWz6ZeWOLl+sampS+uU7nubqVWMXqy2YsnQK92y7p0YbmY/2f0TbL4PIdN2Gpyeo1QrGtR6HvZ2RiL23670AfLj/Q3R6Hb6+4OYmMtkqF7stWSvueW8PJ38UlnBWlX6HHoU//aDs2vrqOjiAk0IQLVfzslAoFNyz7R5Gf2B7p7leryc1NbVZzp9Tlk5h9vrZgJF41mh98Pas+V6oUChwd3QDhcSBrJrDeuXPRU2kn7i2+NlXxaroVexLFMFWpbpSdHpzSw9/f/H90Woho9yN4fOH02mmeXt4l7u6MOuvWRQWi6WCi4vIk1uwdwGrL1Sxiyi8Antvh5gmtG649DnsmgwlwlLFy8uoOq1qNavXNb/Pli0oLyynKKMIpVr8DuRCr4dSVOtqIv3ke3N2tlADNDQyMsQfpVKoT60hLieOM+ln+OHkD3Um/WR7T7P3rNLAjELo+23tLtjYULuCawSOLmLTc6fjz5x65BSjI0Yz4u0RDdZ121Tzp65Mx8ZnNnLie9NwJ61ey8mrJzmeehxXjfGX42rvyshWIwHMmwQaAFotnD0r/t29u21EX2UoFMbOaFty/Spn+l3JFVKDEPeKrguFEpxaij/XE7JPwJm38XcXRYG8POgW2I0wjzDslOY/0NjNsSy7bRmpJ65BEaGJ8MipR5j84+Q6n5+UJLLOqn4dt1XE6chz8p9/isYbR0ehkpKtnKwVumWXiHSPdYz8cSQPrH7A/KCc00J1ep3B2xsiPI8QjmlepFqlNjS7WbL4XHnvSt52eZvy4ro7uzTX9WfbW9vSfnLNle6//oLdu0WB8OWXxefJVsjZ0R06GIuLlVGSU8LpX04TbC/Wvl98Aa+/Dl99ZTxftklWqeD2243Xs9Zj5OvsS5CjKHSVeh8yKPwA7r5bzOOyArYqejzUg4dPPoxnq8ZvvL2JhkNhIeypcNO786NeRN0eZXaMf2d/nop/iiGvDjI0yVqy+CwrEwVw+6JsEtecJO2MkYx7rPdjHH7wME/3e7rWY7zzTmHvOWqU+L+3N9jpXQlyFIOpTu0n6SWKMooMzXn/BJQlXKXFxZ1or6ZbnT9LtCV08e+CXtKzM34n89bPMzumXz/48kvz/UPl/4eHi6bTpkZNSj+t1rivytWLbh0za8+4nyF6IVhQqpsg7yLE/VLnsYKR9JPdL2TSL78sn5NLT94wNUNdqY6ygjKKS8W+OLkoDrAt0w/Ed/m2Zbdx14q7GmxMK6NXotVr6eTXycQJpzZw0bgwIGSAiVqvplw/WXGr1VhQ+l3dIWok6Q1vZVojJD2UpIs/gJ2dAk1FfSYxU9RnXsp5iSk/XYOIogaC/N23VIuzBXq9nqu2hso3AGqV6XcTNaOxLIyaEyRJ3AC9kk6x7/kVXNlzxdAtmCCaAbFzsMOvox8O7vVX29UGunKdxd9Bcn4y6y/Vrqte7jjVaK7NYgNgdPt+dCp9gFZpT7P/WJ7Jc1te2cLy2cvJvXLjdzzqJT3pReLGoc2tXulXmfTT68XiJ2ZjDJ93+5y47XGWr19BCCgUihq79TbFbOLpDU+TU5pJstdvhkVWVczpOgdvR29is2NZcX4FCoVxEVvV4jN6VTRHvzJuKB46+hATv54IiO+aQ3kL/JRtTDOsXMJFF426ifLTquLUG3B+IQCddHcx5PRpnukg/h8yMASvyJqVcDIkSSI1NbVZzp8eYR4EdBFVDZn0sy/3M+QQ1IQRwaJ98Uz5mmqPKy6GFMFd2UT6WcOSw0sAeKH/C5S8WoJSoSSjKIOvj35NqVYUAxUKo8VndLSCQS8PImRgiLVLmmT6XS0UCxQTAhrA3gf6/wKhs+o++NoiZKbISFB7AGKTLudLphUYSb+v+nzF4jaLm25cDYhxn4xj3uV5BpWeTPq56MVcUJMzhLMz2Lvlkez5Owu2f2rTa2ZlCdXU33/Drl2w6UAifb7sx48nfjT7jspFmVatql/43tVZbK42xmzENVB80GtD+kmSaMLRKYtQOVroWlUomtZa1hZIEhQl4essfmd5lW7hHaZ1oO3EhnHMaKr5U6FSMHf/XAa9Nsjk8aS8JHSSDo1KY5ZlMLmdIE/+iv6rwccTEyM2XS4uwk62LpAL1jXtu8rLK82DrjrD99BA+pXlQEHs9Ue4ZOyDE68Q7CKIXLnRzRoK0wu5uPYieQl51R94HcPF3wWPMI86nStJ8NZbsGgRrK7UF5Ofb7RffPVVQc7I+4pp00QTjryPskT66XRwqUJgW+IUA0CEZ5XMmLIcWBsFByyQgc0c3t5wW7uF3BV5l1mukVxEtWQ3GzwgmC53d0FfXnfCrjmvP2vCyZNGtegDD9ReDTNlCgwcCPdbSYsIHRTKK4WvMP2dHrRvL4jsI0fgwAHwu3yAHk7nTY7/Kns2cSP70G3Ktmpf97upX/Blj+N8/vIwk72cn59R8bNhg/l57sHu+Ef5W82hvonmidOnxWenRQvr+xuVRoV7iDsKpcKgepabeiojIUHMs6rgFryY+xK9HrXdSq06KBRGi08An4pSQIhdzRafe97bwwLfBTd0M0xVZPu25vTgRwgZGm51/hzfejzHHz7O6UeEcv546vFqrT4ro7LSr7GsPWua8+W5yZrST1b5KRSQWhoHWCCeBvwCow+AQw1K0JP/gr13QIl51IatqGrvWbkJb+ebO9k1f1edr92cMH7xeB4+87jBlj8hPw6AUHfblX6rolfx94W/LdqG1xZlujLe2PEGANM7TK/39Sqjcq6fJciEdLnaQqaf7wBRI/Fpupx1A8pzYbkfnHjF8JCDJAjJlOxqVK/XEUqOnCHg0m40duIzVKKtXWe1JEk3Sb+baN44teISyrwcMoK7MX7JLbQa0YrgYPFc5c1qcVZxk1tw7F+4n/f93yfttKkNQ9+v+jJu6TjWXKi+8F4ZptaesO3yNvYm7G1SmbxCoWBeqy9ol/QW2XGmRfnej/dm+u/TcfKpvyVAc0Ts5lje9XyXk0tPklWcZbgxl2SKG5o1pZ/siV2qvkqpXTrJyaC0U1KcVUxJjuUJefWDq1lx14oau4WLy4t5ZM0jAIzxfohO8Z9aJf2c1E481usxABbsXYAkSVZJv11v7eLU0lOGBahHmAdeEYK8yM2FzvFf8H5YNGMixxhP6vACjNwhFAbXArHfweUfAWjhFoBrSUcoFjd0SS9RktsIsqJrgILUArQlQi5qVPr54W2je9at7ceJc+0OcbXA+s1dDqb38rL+2a4J0RnRbLm8BQUKHu31qEF52uvLXty/+n42xW4yHCtbMZ6vVLOpvAE6t/wcv0z8hfRz6ZWCykW2DlgIKta4Q9gsYS3WVPDpDeF3iddGFFEdJPG9Sa60qAwdGkqrUU1nodCYkAufTlpRIZazuKqDU0ACRyNm8PbhF9Hqa5Y+L10q7LU+/xzeew8e/XYJB5P3882xb8wsQm2x9gRo7d2afi37oZf0HCxZCojiTZmNzmylpYJ4SfH8g1ZfuXLbstuMTxbGQ8rGa656NocEK0PpUi5snwuabunQKFCqlLTs0xK/jqZWuXE5cQCoC0PZu8f0fjSx7USUCiVHU44a1HENBdnas2NH66qSmmAr6Ser/FQqyNUno5N02CntDPaDJK6CVRGQ0jCWrU2GlrfB6P2UuA4EBDF9OPkwr2x5ha+Pfm12eMfpHXmt9DXa3NKmqUfaJCjJLSHzQqbhnl9bHD9ubH78/XdjJ/CuXUIV0KqVyKHq3x8+/RReew2mV9SIZNLPkr1nQoKYAx0dIV1rhfSTdNDp3yLr+TqDtzesufA4b+5cSbnW1EVBbnazZGfa44EeTPp+EvZuzazho57IiM5ggd8C9n+83+ox+fnw7ruCABk+HMaPr/3r+PjAiy9aV+mrNCrUTmq8vcVa4LPPhCo1MEAiOHozedtNbeuPJB/hdM5BXF2rL6aPjBjB/bd0oU2kOXk3rEIAf/So+fpAkiRKckooyigyO+8mBCRJQlfevBRncoZpJ780lnRewokfTlg8ruBqAZe3XaZdO/H5saT0ixdOfoSGgoObfaM1l8t7PD9dzaRfi14t6P5A9xtuHqoOBWX2lLj54dXC/Of/x9k/6LykM2/tfAuAjn4dDdEr2+O223T9Vq2EgwjYnlNaG+yI20HABwF8cuATq8fUpPST1/ROThBfsQ42y5VTKMGzc80DavsEDFoBdnVXGVRV+vk4+aD9l5bCVwqZ9P2kejkYNDfIeX4SEv8e/DrP9nuWcE/bOqaj/KJwsHMgpySHi5kXaz6hBmhUGr677TuGhA6x2drRGnbG7+SZDc+w9KTYJ9ek9JMJ6TCnjgwJHWK6LnQJEzWSa+FAovaAyIdFzEUFZFew5BxRYE88kMi55RYm+esEumOnaBG9DQcnBZIk4fOeDx7veBCfE3+th2YRN0m/m6gVdGU6Vt6+jNBTf+Po40yvh3ug0qgMpJ+82QWhrvh5ws9NOj5HL0c8wz1NunRzS3JJyBMDq02uTGXSD2De+nkM+GYAu+KbtlNGLqZGR5s+HtQ7iA7TOtiUP3c9Qu2sJrBHIM6+zgZrT08HT/Ra8X6tKf1cNC6Ee4gbf77jGVJSIHx4OE/FP0W7SeaVab1WT+HVQgpSC7BzMG5A16yBhx4SHa0y5u+aT0x2DEGuQUzQvIcSO6ukHwirEXuVPQeTDrL7ym6DZUlV0q/7A90Zt2gcCoWC8uJyzi0/h7ZUFJ3kxaa193vNMHInjBDdvPLY5MLob9N+4z3v95D011/ndGVoS7R8EPgBq+5fBVRS+ml9bVb6dQoNxL1QbBpXnbNeEJZVT/Wx9/7ssMj+vKXNLQZve4VCwcQ2QjX6+9nfDcfKSr/z5+Hsn2f5oMUHxG2LMzyfEZ3BpXWXUDuqDRsbZ2dIya8g/aoGlTeDLnmFAtw04heTmmsk/Ua9O4pbPrvlWg2rXjjz2xnOrTAuiuXCp6ZUtP/ZkgHd3rc9dlo3inWFnE6rOatK/iy2bw/tOpVwxfcLAAbZP2F2rFyUsUVhcE8XkQX4x6XvcXGV0OtN1wzVQVbJldqL9+/lWOkLmPQ3bBsDWc0s00+hhPYvUOg5CYCDpT/Q/tP2PL3+aXa8uYMPAj+gMK3w2o6xFtCWai2SITLp51gaxu+/mz7n6+xL90Ax/x1KOlSn1y0rgw8/FCRJ5WnmVIVTUZS5U5jNkC3tUmto0Jc/f+7ukJAnyMuWbi1RKSsICvf20P45cKs5O7hZwTEQfPrg7C4KTXl5cOrqKd7e/TZ/nPvD7HClnbJG+/PrGTEbYljcdrHJnFsbrFpl/HduLqxbJ/69dav4e3iliDZvb+jTx0hYyx3dmZnGgpYMec3YujXEZovw3wivKqSfvTd0fgOCJ9Vp7NcSLi4Qm9eXg0kTyco1LZwvmbCEtOfSeKD79adgrCskvYRPWx8cPa17dZ44Ib6vgYHw2GN1b3yoCWln0ojZKIjmoCBhg/j55/DI0fsZ9e4I45glydz2uA5o1UqQkaWlQslYGYVXC3nX8122/bt6JeE/Ffkp+Xw76Ft2vdW8FD1yFmTrcLGGsGa3v+3f2/hh+A+EeIl1UXKycV8pQyb9/JKOUXDV2EklSRLe73kT+Ulktfl7tkJW+rUsvI1fp/3KglELrB7bakQrbv3i1n9Mrp9eDyXZRdiVFeHkZL73O5B4gFNpp0waNUaEi7liy+UtNr2GvT2MGCFcHGzNKbUV+aX53PPXPaQVptGvZT+rx9VE+snOCK6u8GCPB1k1axVzu88VDxYmwKHHhQOELfAdIO7ddrXwZ656iYp6VEGBWEMoFArDGrVln5YEdrMStHqd4cQPJzi3QnScatQK5vV7nPdHv4+bvQ1dsAjb8N5BvQGRN98QGBI2hG33bLN5DNZwMOkgC/cvZNUFsZiUST9rWc8y6Xdf+H/ZPme7qTjgWkKhgN5LIOwOw0OuKlFQT69wYtr55k7+vP3P69JlAaBo2HjODbwfBwcF+WX5FJYXkluaa6q2bEa4Sfo1MKp2wd9IkCQJSS/R9T8TSQvrY1L0tkT6dbu/G53vtKG7pQHR/f7u3H/gfjQuGsNjO+J3ANDaqzUvDHjB5mtVJv10eh0XMi8A1Nmrua5o21aiRJ3CwYzNFjOZ6pNl0ZwR3C+YuzffTcToCHSSjt5BvengJQIoHB2F7ao1fDT2I17w34JHYQ+r3TEylHZK7vj7Dmb+NdPw/c3OFkqX5GRh1bRpE5xNP8u7e94FYNG4RRRmiRt7daSfn7OfodC9YO8CQzdtYqJpUaf7/d3x6yTUEye+P8FvU3/j+LfHAeNi00T9pdfB4Sfgyp/Vv7nGhHOwQWGldsviQuB/+eyikPGHjwin65yuNucbKBQKvLy8mt38qdfq6fV4LyJGi+KaTD5rym1X+jk6Qsti4Ve08qx1pXFsxb7AFtJPkiTe2/MeY34aw9l04X9Toi3huxPfAfBIz0dMjp/WYZp4/fMrKdOJ1uk2bcSaLCMD9A5OuAW5mWzEB708iJfyXsI91N2wsalW6bfvLvgrGHQ2SrcaAjln4K9QOP+R4SE/hxY4loai0tZv4d1csPXVrex4fYfh/zM7zuTNYW/inSs20LYo/VoEKvEoFPYe+xOtKwdAkCqyYv/xx6HjzGWU2WXgUBrMxb8nmqjVysqMn9ualH4AMzrOQKPScDrtNM6tRKe3rRaf8mdQ7yx2Pi1dK3Uu+g+DXp+BRyO0A9cXXedTHizuAYWlxZzPOM/lnMs4uDvgEe7RIBkwTTV/nv39LG85vsX5laaWbjLp51QWRmyssN2sjPY+ghGOzqzSuWQDtFp4+22Rh7Z+vSh0g7A7lK2/GoL0q0npV/k+bLGw7d0Lui0At2qCLZsrdGV4OYqFUn5+9coqSZK4vO0yV/Y0rGqzucC7jTf9X+hPYPfaF8eSk+HQYYk8p2O0vWUdekUZf/wh5sjoaKFaGDzY+vkuLsZ1XtUCzwWx/aBNG4jJtqL0u46hUBiL7JkZpgWglm4t8XX2tTi/JR1KYuV9K0k6ZKUiZtNrN7/1p297X+7ddS9d7u5i9Rj5Pt2hQ/X7ofpi/bz1/Db1N5PCnEKpwD/K32B9D5BZnEmxVmxsgt2Cq72mJEl8vP9jHlr9ELklpqyOQgG9RT3WpOkSwNnPmW5zu1VrR/9PhrOvM6V5pZQVinV4cyimZmSI+UyhgEEzW/DEhSfodm83i8d2nNGRMR+Nwc3L2Ex+3nS5QXw8OOZdJe/HVWx8ZqPh8aziLLKKs4jJjsHDwaPe4zbs8TJbM6PjDJtVRP8EFBVBy3Ob6bpxAWptqdn8eTRVqCJ7tDA6vwwPH46bvRsqhcrsetbw5JPCKtupgQ2tXtz8IvG58QS5BhncoQAuZV0yOc6zIjq0JqWfiwuEeoRya9tbDU1uXFwCFz+F3POWT7aG4rpbxDo5GW3DZYtPGZIkGZrJr3dsemETRz7ZC9Quw7YyXugv6sEL9y+ss9pv2+VtnEs3Nog1xBqinY/YTJ/PEJ8bmfRLTTXPiQYj6WdRGLBrOqxuPq4cM52XMPJEMn0dxZ64/3P9mbJ0Clz721SdUKJxo9g9EHt7Y+a0m70bzhrb1LoKhQJPeZJpAtwk/RoYSuWN+SMtzS/lPa/32P3ubpz7dibXvzWVP6eyLU1OjvEmOPDFgQx4YUCTj7UqNsduBmBkq5G1Ok8m/Xx84HLOZUp1pTjaOdbKM7oh4OxRzJbOLdnXehT7T5suBv6Y+QfvuL1z3SuqakInv04cuP8AXw4UHWI1qd4mtp3IyIjh2OldDaRfzKYY9iywHmarcTbumv/8UxS0HRxEEfzjT/RM/fYhyvXlTGw7kUntJhkWVNWRfgDP9HsGO6UdjmpHXNy0+PiIa166ZPn4iDERjP5gNFF3iEpmQkEsmzu35KmTlXKUStPhwmK4urX6F29MlGZB9gnQl+PgWsyFoP+wOus9JEmi92O9mfjVRBPlZHVQKpWEhIQ0u/lT46Jh/KLxhsLLuNbj6JX7Nv65t9pM+gF0UAvSb3viBsp1lkl6mTypKc9Pq9fywOoHeHHzi2yM2cjAbway58oeHOwc2HL3Fl7o/4JZp9eAkAEEugSSW5prmA8dHIz+/8rwMB449AARo0QRUbYGUjuq0esVBsLF3V0itUDMQWZKP+dwoXJRNWL1qSrUbuDgB3ZGudtw9wcZcSqOGZ7vGh47vew0qx9afV02SNz23W2M/9To2zUqYhSvDX4Np8z+gG1Kv4AA8CzsC8C+xH3VHpudLXeIQkCAxKKDiwDoUvYoBXl2LF1qPPbiRUG+eHnVPA8CeDp6MrGtUJ3Ge3wP1J70K3UQlU6TfFP3DtD6IfFZaIZwrYjV0BV6AJBdkk2fJ/swd+9c3FrWn5xuqvnTraUbUXdEGaynZVzOEb9Ex9IwQDTJVIa8ia0t6afXC4WfnIUGImsSxHxZXCwKHHXN8wOjvWdNSj9ZbeDuLjLGZkfNZlSrUXV/4eaE9d1pnzQUEMqhIDchObNE+ikUCn6b8hvb/nVjKm0CugYw6t1RtVZsXMy8yNyfX2Zbp9bs7NCdD1LHs7tLB+J0e/m//xPHdOsGNe3vreX6yaSff1gWOSU5AOZF6EtfwtZRQmFwHaJj0El+nuKFQ5x1RU1VFKQUcPzb42Scq3sOUnNdf9YEuck2uHp+rd7o+1RfJnw2wWSfWZxVTGm+aX6pbGkV4BJgsJa3BoVCwYK9C/ji6BecyzBX1fapiCA6eNBU3a1QKpj41UTD/ugmTKG0U/Lg4QcZvWA0ax9fy7cDv73WQzKo/Nq0MRIS1tBqRCv6zuuLo6ej1Vy/+HgodvWl/yczGPjyQMPjckOit6M3Gmv7EEmC7JOQbr0WIEPe48m1oOpQml/K8juXc/DTgzUffAOgoADyvcPIDOuOm5+TyfwpSZLBCrVHoJH0G996PJkvZLJo/KJrMmYZ2y5vY8nhJQD8MPkHXDQuSJLEkkNL6P1lb4rKjdbBcq2poABDflxlVCb9zBD1b7gtDlqMtX1wR5+DFYH1Iv6q5vo9tuYxJvw8ge9nfc9bDm/dEMTfjD9n0OXFijqHcxoHEg/UOj5gfOvxjI0cS7m+nOc2PVfrMeyM38lty26j15e96uxiYgmGJsmMaHR6HT4+YGcnGiCrErkg9uwSEh4eFurATi3BJdL88abC6f+DnZMM/23pHoRDeSDFBcKxLWxoGB2mdbhu3UPKswtQ6spxcDCSfnIGtS1QKpUEN/YCrvLrNdkr/UOgt0TD3wAozizGv7M/Tt5OZFW4plVW+jk6Grs0bbXramhc2XOFlXNXmuX5yUVu2VrA1s63yko/WU3Tzqed0c6pieCkdsJHKbrHN5829cFv2b8lnWZ1uiFu4lVxedtltr62lfzkfMNjFlVvVhBYwUnIpN+J70+w+YXNJhvVfR/u47uh35F7xdhpmp1ttGR66SWRtyIptGhjB+Gs9GDRuEUoFAqbSb+2Pm1JfDqRX6f9ip3SzqrFpwzPcE/6PdPPkA1wtTCVEk0SmWWVJIsOfjA5GTr9q+YfRGPhzHxY1xWKEgmqmAz06Mgvy6/+PAvQ6/VcuXKl2c+fA4IH0jLuJXzzRtWK9Ovo0YvArGk8EfqlxU4wnc5oWVMT6Xfvynv5+tjXKBVK2vm0I7skm5E/jmTl+ZV0D+zOu6PeRVkl51GpUDKlvcj5qWzxKX9HUlKMx+Yl5rEkagln/xRzXm6u2CcrFKBwyDdsiAxZVjK6vAnDN9T0o2hYOAfD2EMQeb/hIfn3It+nAOJ3xXP0i6OU5pVyvSFkQIhZV7tWa1QKy4RSdQgIAM8CYWGzL6F60k9WmPj7w+Gr+ziachR7lT3vzhQ/4zVrBFF38iT89JM4tl07263F7u92P7d3up2x4SKTz1bST7ZXLFaLarhMTDR7nHsfn+NDUSp0KEpFxT+7uGGzB5tq/gwbGsaUpVMMqnQZAS4BeOra41wqbm7bt5tmMd3X7T4uPXGJbybabqMjSfC//4ksNDs7eOQR8Rk7elTMlZXz/Gqq1WcWZXLq6imLz8lKv/x80b1uDZVttoeEDeGnKT/x2uDXjAccexF2Xqd5KWF3UhowEzBV+uWU5FBYZm4/O27ROJNi6z8dSXlJ9PyiJ+sL36HIIQYHlSPejt4Uqa/gUB5oWCtWtva0Bku5fmVlxsxflY9Q+QW6BOKkriJ/KEqCzIPXLue5nlC7+HI5pwtZRaYNRYl5icxbN4/H1jxmdk7EmAheyn2JznfV3VWmOa4/YzfHsvOtndVm112pqHE2ds2ozS1t6Dy7M0qV8XO1+93dvOP2DlmXjAut2lp7dvAVrI68v66MqCjRmJaVZb1B8iaMSD+bzoq7VhC3Iw6VRtQo5Oa9a73ulfP8unaFxP2JHP3qKCU5NWe+y5bxlXP9CguFchCFkv73tTdZi8gNiWZ7ExNIsGUoHH2mxteXa1r5+bD78gHe2/MeO+J2WDzWzsGOU0tPkbS/7orj6wmFhZDVsjO5g29FQjKZP2OzY8kpyUGj0tDRr6PhHLVKjZ3StkbgxkJhWSFzVwn7zYd7PMzwcONN+aUtL5Fdkm0SgeDiItafYG4zC0bSz8G5lHd3v8uvp39Fp69w71A5gHNo7e7HvgOh1X2gq/n7YfUSVXL9tsZtZe3FtWiiNETNjkKvbT73uboiZEAITq3FHjDVbQ19v+7LQ38/VKtrKBQKPhz9IXZKO1ZFr2Jvwl6bz117cS1jfhpDflk+vYN609m/4VztwjzCsFfZU6orJS4nDqXSvJ5ZGdnZoFXl0Xu1Bv/3/SnVVprveyyEYWsbbGy1Rs4puLpDuJNhdCbKr32ZsFnC77fFRB5aZqL0qw3pp9frSWhC0uT63Bk0YzQHK4XGgEeYB3N2zKH3470NUuKqmVZVO1RP/3qaH0f/SE58TpOMMXFfIse/OW6i5kjOT+ZcxjkUKFh8aDH+7/vbnOtXmfQ7kybOkTcpTY02bkJpdOiKKenXd15fJv84GbXjjZfrF78znl1v7aIo07jprdxtXx20ei37Cn7hbMvnSUkrQ6uFgS8P5MGjD2LnoDZ0jhZnFZN2Og1nP2P7oazya9sWuneHu++GuXM0tEuaz5jzcQQ4hlBebpTU+9jQEO7v4m/4t2zxaY30M3kfWsgpFys3f+dKhVaFUmTxOFa3uWlktBgLUW+AnTMB3o4odcJjIas4i8T9iay4a4XNlkuSJJGVldXs5s/EA4n8Nu03g5VZ5W4/WzP9AAL8lfSI/Z32+hkWNz3JyUZlaWANrmL3db0Pd3t3/pzxJ0cePMKtbW6lRFvCE+ueoLi82Op50ztMB+Cv838ZLD4rLyTPrzzPunnrKMktoTSv1FAokIvdbm7g5uBCwtMJHHrgkM32BU0N+fdSuTt3xFsjeD7jeZx9m+eYrUGSJJPvRKm2lI0xGzkcfxapwg/DVqWfbO95MesimUXWW5flYnNQEAaV3x1RdzCklw8DBwoy5rnn4NVXjcTLwFrU/8dEjuHnqT8zudtQQJB+tnzt5U1CobLC3rOy0m/raNhiQ0X9WqAgFmXBGVzs81BrBemXU5JD9uVsds3fRfKRGvynbcC1nj/fGfkOk5PO0iJ7GkqlKAjtr+QiG+ASQIRXRK0apvbvhw0bBNH3/+yddXRUV9vFf3ckE3cXCBIguBR3irdYqdHSUqi794W6U3d3d7RIcS3uHiBG3F3Hvj9O7kgyk0xClK97rS6amXvvnElmzj3n2c/e+9FHYepUGCbErSxbZs7z61mLo+uG2A1EvBuB/5v+TPl5is1jXFzMxHltFp91rj2KYiD3oP0LtGb0WIii70sAlJeDs+SJh5P4paQU1byH957b26QKv9Tw48QfWfdo/ZpXvj3yLYWVhbiXdWNC/u9kPp5JwkMJrJzzN1EBoovHxcWsXqoNcq6fpdIvNlaoXn18oG9kB76b8R1Pjnyy5sm9n4drCsC1jTREVIOzTwhPbd7CkYKbrB6v0FXwwb4P+ObINzXmOJVGhcZTc1G2Wi09f9rC2dVn2fL0FrSltt0JDAbzvboZG8VNCL0slN439carnXlCbEzST62GAVUioeoWn/s/2c+PE39sFGvsSwU5Z3M49tMxilLM1dQrPr6CBf8uMDWQtgSMRrMld9++Ij/879v/pizP9l7FaDTy7chvWXLDEpPS79w5877rwgXwTjtFsCavhmrQlDdePXrAEuWZ0PNZ6PVinWN3dRV7MoAfDv3G/zb+jxUxK2weq1QrebL0SWb92EYbf+oJy5z36vPnwTSxDuod1Nuu4lJWqzcUeoOeV7a/wuIdi01/d0fw7JZnic+Pp51XO96Y8IbpcUmSGBIu3FCOph+1eNy85suz0asn/x60rkks3LSQ+Svmm5tucw9DaT3X9xEzYcjX4B5Zv/MsUL3xVV7LeVzjwVU/XWXlatUWYTQaMegNpiY9g5P4w/g4198mMTogmhfHvMhvs3+rNdvREr+f+J0Zv82gXFfOtC7TWH3D6jqV7fWBUqGki59ooKxu8Vmd9NNqxd64UpmDzqijuLK4Ucdy0Rj+K1yTB1V7vwzFQU5GPMTavPcBOPLdEd4MeLNNxgUYjUbyI/uSF9S1wUo/o9FInq2JpYnwH+n3H+oN+UZS3aameq5fUUoRybuTKc2qpX26ETHssWE8kvoIwX3NREhmSSaDwgYxKGwQlfpKMksy7XZ8V4eV0i9bbEp6BPSo5Yymw8B2gvQ7W3DUoQLppYDBDwzm3tP34tfFj8fWP0aH9zvwV8LnQN32nkpJyRPb7iEu+C0KNSfJyIDAHoGE9Avh628VXHedKPCNe3kcj2c+brKhzM01q/xuvNGsXpk5U5B7lYVe7N1r/mw4OTmWqQVikRqXF4dTiGD7ZLum2lBYCBUqUYkM9rQg/coyoOg8GFrQrjB4vLCvcA7Eywuc9IJtyS3LpTi9mGM/HSMnxgFflFaM/IR8Ti85bZrD1p7aQYHLEdy8KlDXg2ev3nlXHbLaqX17a9VKcmEyNy27ifkr5pseG9thLAkPJTCz20xc1a4svW4pDw5+kNU3rMZFbd/cfkS7EQS5BVFYUcjhNNF6a6n0i10fy74P9qHx1HBfzH2mzA252O3jIxSD4Z7hXBZ6mfXFS5Lg4EMOWeY0OmK/gTPvm392T2Nnt8G8VdzX9JCztzOufq5tzkJCX6HnReWLrLhVFBoS8hOY9NMkJvw+BAkJd/e6VU4g5i4XfHErE3m0xzKO2T1WLjaHh8MNPW9gXIdx3D/ofgAWLACNRhDUGo0gYj75BEaOtHs5u4iIAKVSbJodsU8qKgKDVEkxYj60Iv00fuBUD+ltc+Kyj5FmZyFpfFDrq5R+5XkUXChg81ObSfq37Vjx7Xh1B2vut901KpOyMgFc3eKzvpDvj+PHm685q6qetm2bmfSzl+dXoatgwcoFJotKtVJtZdtkCdni01HSL60oraZN86jlMLPtbVxluLqa55K6cv0uVRj0BgoSCyhOK677YAu4qtxx0QfROe1JHpl0LR4ad9yd3JkSNYn588U6MnTUWt7e92qd17Jl7yl/F6KiIMDNn3l953HfoPvqNca2AFOmX7X7gfxZLNeVk1Nm/aRBZyDtcBo559r2WrM6Rj01ijsO3YFHqG0pf0aGKPip1Wa1clMh+0w273d43yoeocc1PZj1wyyTqgzEHNvRpyOdfRyzE6uN9AMzSV6d9Ms5m0PSrqRaVZD/39BtZjcWFS+i2yxzuLJC1fIlvoQEce/UaIQjxMC7B3LjPzfa/VxLkoRCrUChUhAcLO63Op1Z7Rl7spwOR5YTtuvPGuea8sarRw9Y4tjTcOhhkcFbByTJTKAEq8Tv1ZYVrYxLsfnaHkpKIPLIcnwOb6rx3MFUQfpZWnvKOJV1ik4fdKLPZ30uqsniz1N/8vSWp3ly85O0e68d1/x5jV0VpgydQcfSM0sB+PSKT/HQWH8G+wSJOtuR9CNWj8v1Jlu5fvK6t0yTAAiVliRJgu1ePxj231Wft2WG0QBxP4Ch/i5e1XMI3Z1EZ2hxZf3WNa0VRalFvKR6iVPviTxPfRXp19Acz0UjF3Fdz+vqbBwyGA28vP1l5iyZg86gY07POSy5dkmtdZeGQs6ZlOcbmfSrnvUs/40NzsLe3M/FYh+sL4cjCyF1baOPz2FUU7nmKWKJD3qfY1rxPXTxcyGgewBKdfM66DUOJBKjJ5PVYRAajbnpJNTdcdKvudHyK4L/0CawcdFGDn8jCsXyJFMX6Tfk4SEsKlpE6GXN9wXwCPGwmjz6Bvdl72172blgJ70CRXXIUrpvD0ajmdy0tPdsKaXf6G5iMZKtPmJVmMo6lcXK21eSsC2hRcbVlHDxccG/mz8qjYrEgkQS8hMorup6rUvpJ0mSKUy5wO2QqTumtFDL5r9LKCszdx9aEgFLl4qCdrduoisRhD3sv0k7uPxysUDdsAEra09HG4zf2/MenT7oxA9JwpIzM9NsWWcPeXlQoRZMUbCFWpDzn8PfUVBge7Pc3PD2BrVOkH45pTlETY3iyZIn6T238SwPWgI9r+vJM7pn6DJNdF3N3zCVHT36ofarX3FX9tiPz0rn1R2v8vmBz62el0m/jh3Nj53LOcfwb4bz07Gf+PX4rxSUm71FLBe3KoWK9ya/R6+g2jNOlAolf17zJ2mPpjE4XFRTLAOiRy4aySMpj+AZ7onGw9wpZmlrZxcFJyDmfShwTEXdqDj/OZw2ZwAF+mnId99HpuKoSdFYnl9OxvGMGhk0rR0GvYHoWdEE9RbffbkAH6ARRVBHVMYgyLXAQOh14VM2TLnA2A5j7R5rqfSb1nUam27eRL8QQQAHBMBLL8G998J33wnLxYaoDMq0ZRzK2IsqUlTzHLH4LCwEvVTOGI87mN51uvXmZvivMLJmEahVoOoG4eGBSelXXFmMfx9/bj9we5uaI8/+fZYzy85YPWY0GtHrzXazM2eKf48etW5y+HT/p9y49EYOpB7AEciWw+0tIpS7dhWWXzqdUKS5ulrPmZb46tBXJBcmE+YRRt7/8oh/MB5XtSsF5QU1OsPlorkjpJ+3N/T4pAfOrzgTk12/jMJWi8wdSNtn0C9CWP/WRfrt/WAvbwa8SXZMw3PUWiMUSgX3xdzHVT9fVa/zguIfYuyRJLpqr2fMGOvnBg+Ghe8e5/2cqTy9+ek6LaRk0i811ayA3lcVE9W1ax0DSVkNaevrNfbWBD8/mNDxawY5/c/qcY1KQ2CV00VSgXWThL5Szxf9v2D7S9ubbZzNAVd/V0L6hdglbmRSOCzMscafix2LxktjUojYy0a+Z+A9xD4Qy+Lxix26bl2k32WXidtnQoL1vWTCmxNYVLTILnH0/xVObk41iKfMk5n8ec2fXNjZMg0pcp5fz57CJtGnow+dJ3VGpbFv8zhv8zxm/TALSTJbfG7fLubD5GwN5wfOIWhuzTxd+b5eq71n4BjofBc4+UClDb/GapBJPz+jGIisvLGF7Jhszq11wMLnEkBxMXhmxeGUWdMJwNvZmy5+XRgYWpNYbe/VnqSCJC4UXCAuL67Br5+Qn4CT0okwjzB0Bh1/nfqLMd+P4ZF1j6CzQ5SpFCpO33uav675iymdazo/yKTf0QxrR63aSD9ZpFOiTgAE6QeAUQ+9X4LIufV7YzLOfQZ75sG5T+p9qlyblccmk5vZu7JZedtKcs627QYZhVJB92u6ow4XC3etsuFKv+rILMkkPs/2hvSrQ1/xzJZnMGLknsvu4cdZP6JWNg3R381PNBnI3xF7Sj/5b6z2En9Tf1eLokB5Jpx6vWXXhGVpkPy3+BcI9hR1whKDKLB3ndaVW7bdQtigtudOodcLxwWoamrx78aUzlPoE9ynZQdWC/4j/RoZF2Mx0lqhq9Cx+63dxKwUBQ5HlX7N+bsozijmzIozVlaQllApVPQMFD5QJ7LqJv0KCsQXWpLEDf+die/wweQPGBQ2qDGH7TAuCxeTSIlzDMdOmb2+y3LLOPzVYdIOOW5v0FZQklVCcXoxRqORzBKx41OUCclUXUo/MHeZFbgeJDVVfI7f8nuNoINCqRD7zzl2vb3LZDNSUWFb5ff4hscZ9d0ossO/A8QmRs4YqCvPzxJd/UXF5lz+aZOFU10Wn7m5UKEWlchAS3vPgOEQ/bjwi28pFJ6DjWMg9ltB+lUp/bJKclE6KVG7Or4YkiSJ4ODgVjl/KpQKFEoFpdpSSvWiUy7MJ7COs6whf05OlG7iqc1PsXjnYrPvPxBXtfeR8/yOZxxn5LcjuVBwgS5+Xdh6y1Y8NQ5KSmvByPYjrT5HssIlNRU8wjzxCPWo8TewJP3+Of8Pj69/nNVnV1tfOHgizEiEdtdc9BjrjSHfwXhzh2eYnxcYxXuQs9NO/nGSz3p/RvLutqVacXJz4tol1zLkQWE7I1vt+ShFZbg+uZJBQeBfNBapSNyo9Xox31XvHLQk/WwhOhomT3bMVtQevj/6PUO+HsJx/2cBx0k/tcGTBzp8zorrV7TKucImyjLgwhLa+cWj1nsT5BxBr8Be6J31hA4IxcX34rtEm2v+XPDvAu45eY/VY1sTtuL/pi8HOgmipEMH6NNHFOg2bjQft+b8Gn45/ovDgfcy6Vfd7lhW+4H9PL9yXTmv7hSqqqdGPmVqknh397v4vuHL81uftzpefo3kWqYHeR5UuRWRV56HwWgw27gYDZDwK+QdqfN9tUpU5EDqGsJ9xeK9sBA+ueITEh5MYE7POTUOd/F1wT/a/5LIhrGF+nyPjh+H338HhVHNvXerTXZwlhjWqRe39L0FI0YWrFhAuc5+Xk9goCiOV1aK5jI5P1WhgFGjDXy07yN2JO6wWj+YcOgRh7KqWiv8/GBQ2EpGBrwjvlMWiPAU962kQmvST+WiYtwr4+hxXcNdWFrj+rMotYjK4kq7zzdXnh8I0u+uI3cx8B5RxP9t+m/8NPknDPqL+/5H+wsiJbEg0aYKxcNDzPFgrfZTqpWt6m/VGpB6MJX0I+k1HpckiVNLTpG0u2UcBWTSr5/oGzPlDDoK2UVi1Sr4/HNISJAo8u9Atyk1w89DPUIZEDLAZI1nEx3mwqBPYd0g2GS/+U2G3FjnXl71Wc1PtOsYsP3F7fwy9Re7pPilhJISODbhEdzvnltj/lw0chEx98WwoN+CGue5ObmZbDQ3xddUCTqKhSMWkvBgAifuOcHRu45yWz+ROf7unnd5dYd9Rb2zypnZ3WfbnD/6BvcFhBOKweL+UxvpJzeK1SD9FCro/j9of2093pUFOt0qLGg73Vb3sdVQnfSTlX4lcSUc/vpws0UuNRXcg9255o9rcBsm6qJaZT4APi4XR/rtvLCTXp/24rq/rqvp5AHM7zuf8R3H8830b/j4io/rFVdQX9w76F7SHk3j46kfA3WTfioPQfr5uVoUBVxC4cqzEP1Yk42zTmRsg+3TTS5QId6iTlhGbm1ntQmkncih04Hf8cqIQaOBOy+7kzU3ruGGXjc4fA1JkghqaqsGC/xH+jUyFE3dctcCUGlUPJb5GJPfnwyYSb/qmVby5iMzU2xYK0sqObvqLBnHammfbiTEbYjj95m/E7s+1vRYUUURRRVmf3tZ6eeIvadsL+PtLTbgoyNHc//g+2v3im9ChHmE4Sb5YpT0bD1lVtOEDQrjiZwnGPLQkBYZV1Ni9d2reTvkbTBCVomQ1hmKBWFRnXC2BZPSr4r0U2lUuI8aQJFfJADZm46y4XGzB1lKivjceniIoiWIxd+R9COoFWpuHjyDXr1EMXP5cvF8fUg/eYN7NucsnTqLjU9dpF9eHlSq5Ew/ixtD8OXQ7w1w8nZ8AI0NhVKE9FZk4e4OmirSLyU3F4PeQMr+FLJOZTl2KYWC4ODgVjd/Zp7IJGFbAnqt3vQZVBicCPV1gIAryxDWk6XJps+JR9JsfJx9SCxIZEOc+bNnqfTbEr+F0d+NJqMkgz5BfdgxfwdDwoc0SZFDJv1KSszZBNUhLyq9vWFz/Gbe2v0WG+M2Wh+kUIJbO9E929zwigYPc76Uj7cStd4bgMxicbMKHRjK6OdH493Bu/nH14iQVTceRkH61Wf+kf/WMpmydKmw5vzoI/MxWq1QfaZ7L+eXlBdIL65ZRGoMyA0ZaRzEiNFh0g9s2Clri+H4S5CxtVHH2GjIOww7r6an/yYklHzf9wLH7j6Gp8aTiqIKynLt53A6iuaaPyWFhLOXNauRkJ9AfkUeekUJzs5ivTRxonhu82azWknuXI3JqVsdZzSaN7fVSb/Bg82P2cvz++LgF6QWpRLhGWFVeIryi8JgNLA5YbPV8ZGR4t/aPoey0q9UJYqnPs4+ZnsobQHsugHOflzXW2udCJ8O11dyvlIUqAoLobNvZ9p7t7fZydx7bm/mb59PYI/6Nb+0diTvSeboD0cpL7BPylkiMTOXRz75B4NRz4QJtdscvzPxHYLdg4nJieGFrS/YPU6ptCahV1RFSA0bBkWqWO5fez8Tf5poynS1wmUfQb+3HBp7a4SfH3x64BNu/TsVo9F6vRPhVUX6VVP6SZLEyCdH0uWKWgr9daA1rj+/Hvo134/93u7zcnNtc+f56bV63ALdcA9yR6G8uN+Xn6ufaV9zLsf2ZmhQVZ+tZUZsZXEl59aeI/OEHb/8/4f458F/+GnSTzUeD+gewCMpjzD88eHNPiat1pz7LO+pvxjwBZ/2/rTW84wGI7ve2sXO13cyahTccYdowt34ew5xu9PBaLRyAJDx+PDHOXDHAW7r7wBREjga/IfWaC6oDrmxTlcQgJ+LH0aMdhX+fRf0Zfo30/9fENLyftHdQ2F3/rT3e7i8w+WA2E9eDEI8QvB29qZ3UG++nP4lf13zF4PCBvHQkIdqHJtTmmNF5NlCV/+uaJQaiiqLrNRe1e0yLSErkPNJBCxIv4uFUgO9ngGVa71PraH0q8r0qxhbwf/y/keHcTUJ87YI2V2kQtE4Sr9I70gq9ZXsT93PqztexWg08sfJP0wEoFqpZv3c9czvN7+OK108At0CCXY3E+lyE25GhnA6kSH/jSU34bphpfRTqMAzqmUzngOGisbsKjvl8KoJtUKRg9FopCy3jG0vbuP8P+dbbowNRF5SMd7pMbhUFKCyL1yvFQqF4j/Sry1Dr780g6VdfFzwbu9NebmwVYKapJ+npyBMjEaxWS3LKePXab9y+NvDTT6+9qPac+XnVxI5JhIQRYupT36P12IfHlj7IAA9AkXLYHx+fJ3e1pZ5fq0BkiRxe9QL9In/hpw4c0i60kmJi6/LJbnIjLoiisEPDUZSSCaln65AVLkdIf3kwnKh61GSUsVNO6HHVLIixc0nZ/g05u+Yj4uPUFrIRcbwcLPK74ejPwDC5s7XxZcJVY4iso+7o/Z6IBYUTkonynXleLcXbbqOKP002mACFV1MRY9WA/eOcHUOdH8CSYLBha8z+sQJJgTdiNFg5KtBX7Htxdo99mXo9XpiY2Nb3fy5++3dfD/me3RlOrJKBennpAvE39/G982ghcTfzVXuslTYeyscfx4fH1EMl/TODHe/GYAvD30JiE1xXp74zH1w7kHG/TCOvPI8hoYPZcu8LdYKz0bA/zb8jyt/uZLUolQ0GvMcl2ZHLGyp9JNJoBrNDwWnobSFVHQGrQhL14sbk7u72Wo2papDJaRfCGOeG4NfVCuZ0B1EfkI+a+5fY7Jvlkk/TaVYxNfn/iQXknekbGLSD1N5YfvzgFAty/f09HTQSkWcavcgr+97nq8OfdUI76ImegX1QqVQUaTPoczpgkOkX0GB6OhUuhZZP1GeDsefhbR/mmSsFw2ffjDsZ7IU4wDzvQPgTf83WXbzsot+ieaYP/WVehK2JlCYbO1JnZCfAIBLZaRJ/TlokMi7zcgwF6e7+QvSrzZ7LBkFBeIzKUk186oUCnjsMbjiCqE4tQWVQoWPsw9Pj3raKtR+ZLuRKCQF53PPW5EHnap6BhISzHYttsYEUIC4d7fzMq/DUDjDiL+g0+11vrdWCUkBkoRHFYdZVFT74Zcqjv9ynOXzllNRULcNtNEI93z2IxuDp3C85wzuuKP2431cfPjsis8AeHPXm5zNsR/qLFt8njgh8itB2OYeThd7qV6BYv6sgZAJEGrnS9EG4OMDeeVh5JYGUFhUjfSzo/RrDLTG9Wfvm3vT43r76kVZldxcpN+5tedY//h6JEniqp+vYsZ3M6yeL9eVE/p2KIO/GkxJZYnD190ybwsFCwtMFuLVMaSqn/XYMYip4lqKM4r5ZeovHPn+SEPeyiWJIQ8PYezLtpVrHiEtY4N65oxopPX2Ntt0dxjXgY7j7XhyV0FSSBz78RgHPj2A0Whk2jR44gkIid9Fj+2f41yaa5oj64WCU8KdJnkl9H8LBn5cI3OqOixzRqvnbFVHx8s70m9+P1TODawAtyEUZlXglXEWTVm+1fxZUllSJ7k2roNYC2+O30yFrn6RC3F5cRxNP2rzudndZ7P71t1Wrjhy4/+85fPo+lFXtiZstXttlUJF3+C+9ArsRV55nulxe0q/8nLzujCjIgGwIP1S18G6oZB5kbbTxXEQ8yFU5td5qAxL0s9oNCv9ylRlOHs7X3SzRksj7VAaq+9ZTWGMKFqUU0X6XaTSL9wznE+mCjvVl7a/xJW/Xsl1f13Hok2LTMe0VK3V11fsqQwGc7wQmEk/o3OV0s8y9qIyX9hq2nKFaC64tYeO80yuZBF+ojZjUFSQX1KGtkzL1ue2tklbZL/+7Tl4xTMUdb0Mo9FYq4OHPej1euJkq69mQNv+5v+HZkHaoTRyz+diNBpNKj9nZ2rY2EiSdQi9W5Ab07+eTp+bmt7f1qudFwPuGGBa3H73HZzTb8Io6TEUComDv6u/yevdXoaADEvSb3vidn48+iOxubG1ntPUeGrCfUTkzCcjLoAKi3VS1uksm7YebR395vdj8ruT0Rl05JSJP0hlnuNKv06+nXBXeWJQVHA66zR5edad/Gk5GiKGmQt3sq2dLKPXGXT8dEx0Ts7rMw8Q3dYuFm5s9VHaKBVKk+2IwU8UPuua63NzoWfSB7zTMYaZ3Waan9gyBQ61oGTfBtq5R+FR3gN9qSdKtZKJb0+kzzzHv/tFrbDS2GdeHya/PxknDycT8azRBtomW04uhn+vh6Ql4mevHqLzftAXSBKMrdqPF20VheGVZ1ZyxeLF3P2SyLjq3h0mR41HrVBzW7/bWH/T+otexNrCsjPLWH1utan4LpNBjpB+acV2MjN23wTrWkhtHPMhLA+DXBEer1SCs1EsLFPlG1YbRX5iPvs/2k/mcfHZk0k/dWn9Mv3ArPRLzstkffxaEt1+B4TN58kq8XhyMpwJe4pSpwtEekfa7JhtDDirnE122wWuInO1vI71cmEhnA15kbHrPXly05PmJ1wjYMoRiLrH7rktCpcgiLwBo5sodMnTnCRJDLhzAJ0mdqrlZMfR1PNnUWoR34/9nj3v7bF6PKEgAQDXCjPp5+wMvauiCk15ZFX21o6QfvJc5O8vNrrV0aUL3HWXyPSzhXsG3kP8g/Hc0vcWq8e9nL24LPQyALYkbDE9HhIiMhkqKmra54D4bMprrlyD6Oi2Iv1ULtBuNvi3jP37RcOgg/RNdPI5BIjvWmJ+Ios2LuLpzU/XOFxbpmXHqzs49VfryBRuLAx5aAjXLr0Wj7C6i+SbNhnZUSKaIm4bdYVNW8/qmNFtBldEXYHeqLf5e5Uh76NWrBBd3d26iTy/w2mC9OsXbJsgaetQqcDfp4JwjzPkplnnRcqkX2pRzS/oqrtX1aqKcwStbf057qVxDHt0mM3njMbmV/rFbYhj91u7KUgSVe7qxc+kgiTSitM4mXkSV7Xj6pTogOharetDQuByIQziyy/Fe/cM92Tal9PofWPbycNtanSf3Z0Btw+w+3zshliW3bQMo8GGQriJcLSKm+nTx9xIO/m9yUx6Z1Kd51675FruO3Of6XM2YgTMfrkf2b3G0GuMn811QZ0oSxXOC9rCuo+tgrzXy842u/WczrJN+v1/QlFCDlH7f6XioJByyvPnS9tfwus1L97aZV9xPjh8MN7O3mSVZjHxp4nklDqeMffithfp+3lfXt7+ss3nFVUkrtFoZPGOxfT6tBc7Enew5twazueeN1uy28GuW3dx7O5jpnUi1FTOyZDJFzc3SCpKAERmISDcH0oviGy/i0HqWjj4AGTucPgUmaTU6YQi85Vxr6B9RstLw14i/Ug6RWmt615XX2SdyuLApwcoTRP3ool+d/DUyKfoEdBwi28Zc3rN4fqe16M36llzbg0qhcqUb93ceHvX28z4bQb7U/ZbNUCmW5R75c9kmHs7xkSOsf4dxH4Ny0Ihx8Ibu4Xh7+mOZBRNEUnZubgHuXPX0bsY9fSoFh5Z/VFRAUgSGhcFxZXFuLzigsdiD7v2z/ZQbM9mqwnwH+n3H+rE6ntW8/WwrwH7eX4yLHP9VBoV/Rb0I6R/01piGnQGdOVmvfPJk7Bug44cD1HUKT1xuem5MZFjGBs5ts5OJEvS75vD33Dz8pv55fgvjT/4esDPT3R76PXWCrFfrviFFfNXtNzAmhjyglBCwlAsVuCOkH4KSUGfIFEcSSg/wr594JaXRK9jP+N/4RD6zBwKC80boOpZVutj15NRkoG/q78p9FmjgVEW96b6kH5gVjvkSKLwmZMjLFDsweb3zWiEorNQ2jIZDVZI/QeShFLFy0s8JJNEQx8ZStSUqJYZVyMhckwkgx8YjCSZ1aZOugDbpF/UPcKDP6Sq017pBF3uNXWS3n8/3Hkn+Bt74FM8FJ1Rx5rKJ8nwXsn48bBoEVzR5QoSH0rky+lfmrrzGhtRfuJvItsp1Yv0KxIHhbhXm9M739FyvvF+A6HLfeBk/qO4SoL0SysQX6DsmGy+G/0dR3+03SHaWhExLIJHUh8xkedypp+xsOGkn+bCVBRGNcUuZ3AKFfPQsWPiuc3ndpMQKPw+P7/y8yb7DIJZiV3hexCjERIT7R9rNAoiotxJvH8rq2OlBnz6CHvZVgxZRfVtxv1EfxzNijMrmPLBFAY/MLhlB+YgNJ4aJn8wma4zulo9bkvpBzBQCOpNpJ9870sssJ+JI8Nenl994OXshZOyZmVwbKTovrAk/RQKs8WnrUYcuZvbyQlSS2wo/do8jLB5PIM9XwcEMZ1Xnsdr/75mUqRbQqFSsPmpzZz8/WSN59oyfDr6ED0rus5OeL0ePlq2jyLXEzhJLtw/tmbuoT0svnwxEhJ/nvqTE5m288XlNWhlVaTbjCpR1ZGMIwC2VVHFCfCHh7A6bsMYGLmdT6+MRkr60+rx2wfcTuZjmXw/sya5V1lUSXl+/bus2yry8qC0VBApobXXsBsNQx8ZSt8Ffflj9h82M6EuFJjnxcZWQ9x8s2gkiYkRyleVRkX/2/oT3Ne6+ezMGfjnH1i3DjZsgP377Su3/78hfnM8J34/QdphOwv9JoC8ruzTgL5v386+NRRzY+aG8+6h0TzzTM3jy7RluL/qTucPOttXmgaPh6vzoX3VfH38Bdh+Va3jkPd6OTnwyNBHOHTHIRaNXGTz2OS9yXzQ6QOOfHek1mteCijXeBHfezpBwztbPX4w7SDFlcV4abzsnuukdOKPq//Aw8mDk5knyS/Pd+g1UwpTTHW4CR0n1HpsibaEb498S2JBImO/H4sRI5M6Tao97xEzaWiJwCqznep7ZDnPLzAQ1tywhpXXr6R7QHfxYPtrYVYKBNWdG1krwmfC6NUQNMbhU9Rqc+Z6fj5oVBpUChVZp7L4vN/nHP2+be2Dq6PHtT14NP1RtO1Fw+TU0Ft4edzLpsbCi8UnUz8h2j+a9l7t2Tpva5M1v9aFzQmbWRmzkkNpohlP3sNbkn5yjXB2h9vZMm8Ldw+82/ykT1+IuhfcIptlvDZh0MGKjrDnVgAUCgmNXkyqSdk5KFQKgnoH4Rbg1nJjbCCyz+bikR2Ps6LS5ARmMBrq1fTU3PiP9PsPdWLIw0MY++JYJEkydRVUt/aUIZN+yc3o9JayP4XFnos5+OVBdDr4+GPI9diOTlWAWudD5pEBpkLOr7N/ZfO8zaYgYXvIrmoy9fODk1misGG6mbcQ9EYdmi47SAz4jNOnzWTViEUjGPLwpZfpt/L2lax7dB3lunIGhQ2ip38/JJQ4OdVUmdrDZ9M+ZmrMBcJybmLtWpAMBpzT4ml/fDW9tn5E7GFzx5/c4S8XXL4/KooLN/S8wSrXZoLFWrPepF9VrlFC8RmcnEQxOzvb/vHy982KZJIkmB4LI36v34s3BY4ugkOPAlDucZKzIS/yV1zTWAK2NEyknzbQev5LWQNFseDsLzz41dWIktzDcO5zJAmuvBI+/BCmqBfjVdKfKN1V3HtNTx58UJCmCknR5LmhUb5VpF9u/Ug/H59a7D073wHdHmrkkTqIwJFw2Yfg1c30kK8yApeKSCrLRdC2UW8k61QWpdn168BqaSjVSjxCPNB4CIvChcMX8vLYl1FkiXza+th7yl2C2iIv/AqFvY7fcNEscvw4VOgq+DDxVpCMjPKcx8ROExvvjdiATPqVeguFptx0YQsVFaIAXu4kFhZWnZe6EmFjUkcjT4tiZWdGSbMAyKlM4Uz2GZuKldYMF18XBt8/mPYjrcN0ZNLPtSLSRGyCOYvpzBlB2Pq7+pusZ2qzNgTzXFTfgnZsbiyb4zdTqa+0e4xM+m2O34zRaF5HyRaftkg/y8aHpEIbpF/8z7DEH9I31ji3TUChhkFfkOQs1LKFhebvWGZJZg0LLqVayR2H7mDqJ1ObfahNBaPBSEWRY1Zj23cY2OEimlyu6XEN3s7eDr9Or6BePDf6OVZcv8JuZ7qldV1gIAwdKv5fVvr1De5r++L+w8C1ZbrSGwuVztH8cXIRaeXWxKa3szcBbgE2CaWrfrqKOw/f2VxDbHIk703mlyt+IXG77U6YC2IKIiREFHibA57hnniGe1JZXIlHaE0lbGKBGGt7bxtha7UgtSiV+9fcz03LbrJ7jK8vXHON+P/vvrPtCpCbKxrnPv5Y5BR/8AG8+CI895y5aeNSRerBVD7o/AHHfj5m95jhjw/ngdgHCB3QPCxxWRmcrbrNy6RfaXYpK29byZkVdav9AXLP53L0B0FQFKUWYTQaRUyCDU45rTiNEm0JKUUptRddJUlkkAMUnYOcfWINaQdyY11eHnT27ka/kH52r+/k7oSTuxMK1aVfWi3BjZx2/QjqYybejUYjB1PFer5/SP9az5/QaQK7bt3Fyjkr6eTrmNvFS9tfQmvQMrLdSAaH194s5+7kzsabNxLuGY6+Sm1336D7HHodAL2FJaJcE8rOtp575Dy/wEBxX5/WdZo557mx4BoGYVNBXb/r2lInerXzYsyLY2g/qn5zdGuD0kmJe5A7FXpx87N032oM+Lj4cOzuY8Q/GM/wds2fhSpDrhfKdsK2ajWW9ZkaCL4cBn4Ers3UGWQLCpVYkzqbi6WzC3Yy/kgaISrh9lOWV0ZxevOp3RoLccuP0XXPD7hW5pvqgwGu9SwKNzMu/TtTM+NSzFbreV1PLrtLSN3lG4gjSj+A32f9zsfdP27S8SmUCjpP6ox/V3+WLxevnRksbBn7aa5GQsmyesbmyEo/H1+DycqhxUk/g55vdOM43v5uDpwzq7wG3D6APjc3vYVqcyNxWyLJu5Jp792evbft5fdxYjHp42N70W8LPYN60DkgAgmJ2Fgo9o1g1v4n0V57A6mdR1KI6EaTcyhBFBp1Bh3HM44DMK/vPKtrdukirBoHDaq/EmFip4k8NfIprul+tal7TF442kJyQRobeodx677LrAqUrQb93oLBguQrdjnN2bDn2JQtyNJ/Hv6nzsB2GZIkERER0ermz6+Hfs2SG4Rd58iIsXRLfo2QvKvNZIu2GLZdAcdstJ7KOP48HLgHykRbYEgI/PTKaGIePUjMi0t4cPy1TfsmqqE+pJ/RaC6YuHpUmmx2a9h7tjJc4/wllx+PZ4jrXAACugfweNbjDH14aAuPrH4ozSklOyYbbZmQA1/T4xoeGfgUFAjCoT5KPzc3s9osOH8mAIfLlgNw7ryRReufI8NwGidtIE/0eaex3oJdDAgVpF+m8iBGjLXOg/JnUCb9wjwtgsnjvoO/fCDTsfzQFoFXd/TOwt5TpRWLp7zyPPa8v4ffr/r9ouf2lpo/tXqtyXLWpaKDldLP3x86dBBzyEFx66abfzeUktKkCrEHuQGnvvfXbw5/w+U/XM7tf9vP1hvRbgQqhYoLBReIzzf7fXfoIP6tTenn5QWj2o3ixl43mkhrANSews7ZqR4sfGtD59up9B4NCNLPz8UPjVI0G9giqEP6hbTJ7lx7KEgq4DXP19i4qHbi1miEZ1d9RK7HTpwV7rw6vv7KuufGPMf0rtPtfl8tSb9p04RldVpRGhklGSgkBb2DbNgaukfCuHXQaX69x9Oa4OQdzo/HXiW2oPkaGVvb+jM/IZ+4TXFUFttuXmhua08ZY18Yy72n7kWpVtZ4zqT086yfAlopKflo/0f8cvwXyrRldo+bOVMU13NyYMkS+PuOv3kz8E3TvXPPHmFn5+sLgwcLpblGA0eOwAMPmC3ML0XoynWoXdWoNPaz5Fx8XfCKsK++amycOiUU0UFBZqVUUVoRh78+7HAcydbnt7J83nIKkgr4vN/n/Hrlr3aPNTUkuofY/x5f+AvSN5t/HvQ5zEwClf37mJeXsB02Gs0Fdp0ODh0S/1oisEcgdx29i95zL33bWdmRzs3NPH9mlWaRV56HQlKY7PtrQ8/AngyLMFsY55bl2l0Lb4nfwucHPwfgxbEvOjTGdl7t2HDTBsI8wrgs9DKTY1NtqNRXMujLQXgs9iCvTBQ8PTzMeydL+3dZ6Vc9dxqAlNWQ2IiN2cUJoLPTtJq9B9YOgMOPmx6yJP32Ju9lzpI5LD6xmNHPjCZiWDPfOBoZhSmFZJ3OoqxYjwEdcRV7OZtztlFrZCqFqsXXA3KGqByJYEvpJ9fkvb1bYX1QxoTt0Pc104/hrp1x1gVTWizWEV8M+IKfp/7cUqNrMHyGdCGx5xRUfp5klQilX6BbYL2uIUkS4Q0KqG0Y/iP9GhkKxaX1K60+iTqq9EtJEQs+j3APfDvZObiREDYojDl/z0Ef3p5ffwW9VEaGryjUPzhWFH23b7dWVMnhvvYgk35a1yRKtCWoFWo6+3au9ZymhkaloYNH1U0gr23L8x3B/WfvZ8GuBaaf6yKc7cFSLeDiqiC6h4KgYZ1I7TbOdPMsLISSErNdjkqh4sQ9J9i1YFeNjjVJgkcegWeeEbZg9cHI9iN5edzLTOg0oU7Sz2iE9NJkKpxSya1INy9AimIh/icobQVKkeDLIViohoI8xfe8UCf8Box6MXfotXV72isUCvz8/Frd/KlyUZksZjo5D6Rz+v8IL5mOpxxBIilgyLfQ6Vb7F+n1LIzfCc7mxYDsz94Sa0rZ3qS6vaetLKvCQrM9UplS7HDUCrV1WHTaBlg3GDK21LxAc0BbCDuugZgPTA/Jf59Cx6M7WiWO/3Kcj7t9TMo+swxOvo+5udW/w1HeNAzxmQ7AgfQ9+LRLw2g0sDpmPQA9L3xEdGTT3rMBegf15s0Jb/JQ+O/gAOlnxEC5WnxIrZR+nl3F968lLUzqwuiVZLd7GwCpsor0K8sj7WAaMStj0JXpaju7TjTH/Lnv43181uczcs6a81eKKouY2Gki4ereaHSBVqQf1LT4XHLtEkqfKmV61+m1vlZD7T1Xn1sN1G795ObkxqIRi/j0ik+tFFodBSdLXJy491rCkvS7fcDt/HTVT1ze0WwbT/g0GL8NfNt21ppc2CoqqtqMVn3PZFthS2jLtOTG2i/StTkYoffc3nUqYZZsiWW3q7B3e23cmxdt81pQXlAjbsDNDQYMEOSf7CxxJP0IAF39urZq+6CLhWWGVnU8uu5Rrvr9KlNXtYyM4xkc+OwAJVn2FTu1obWtP3te15OnSp+ym/cqNyg2N+kH2FUxyUq/+n4fAt0C8XXxxWA01KoAd3KC+VV89tKloPTxILhvsOneuacqanb6dHj6aXj2WXj3XfE7ys2FJ5+EHY7HYrUptBvejruP3U33q+tuTD615BRLbljS5PO2nOfX24L/CuwRyP/y/udw893gBwYz5+85qDQqelzfg06T7CvCTNEDtTmlHHwAjiw0/6xyq3MTJknmeld2Nny6/1PGvH0Hj7+SyPr1Dr2NSxKqA3vpufkDdGlZpvkzuUhMTKEeoWhUmnpd7+1dbxP5XiSrzq6q8VxJZQm3/X0bAHcOuJMxkWMcvm43/27EPxjPvtv2oVTUbFaoDielE1mlWZTpyjiaYa6zyXV5S0cSec+S67aL13e+zrYEi8bDU6/DAceVhbUi9htY2cG+k0TJBSjPEFaOVZDrZLm5kFGSwW8nfmND3IbGGU8LY9dbu/ik+ydU5BRToc7g9n1D6P5xy4oymgJyJII9pZ/RaLb3nLimA0FvBXEy06K7Zf+9sM/C7rOVQN5nyPWZvrf0pce1F5/H2Nxwah9KVuQgnL2czUo/t/op/RQKBb72CJUmQOtY4V5C0OsvMrS1lWHDExv4uPvHlGSKzVRdmX4BAaK7TqcTXTBTP5zKnL8dz7toCIqK4LPP4OGHhQWYpvcqSvSFtPNqx/XDRtC7tyhcr1wpung6vN8Bz9fMzLwtyO8zm1OAKJRbWjy2FHoG9AUgody8GDny/RE+6fkJWaftv5+2CstOm1pl7LXgqOs77O80k2JNDL17Q9axNJxPC5sk+eYpEx4BAZgCwhWSgqER9VQGaQuFh3XWv1BSS0gV1En6FRZCqVKsMMO9LJQtGZth902Q30qIX6MRjEZCvMWNq0QvvjxTPpjC3cfuttkVXB16vZ4zZ860uvlz3uZ5zPhGBOrIjQC+vhb7RJUrdLxFkJ/24DsAAoa2DMNnA3KmX2xeLHqD3kQEFRQISx5LyN85Dw9o5x1K8sPJ7L99v3UHXGUelCYDLfT+FM6QtAQsGiHkfEl5UWk0GDn550m7dlmtFaGXhTLiyRH4dPAhvTid9bHrOZQoyNr6WHvKGDJEWCM/MD+UwWHCHkfXaSUSSqZWfk2XlBcJybu6WXKCnFXOPDbsMca0H4uEgqxabl8FBVChysQo6VBICmulafB4oTZ279D0g74IyBsdqdys9Jv+1XSe0T6D2vXi1hbNMX8a9UYqSypRuZjVBL4uvqy9cS2Pux9FQlGD9JMtPuWu+CD3IJs5e9XRENIvuTCZoxlHkZCY1GlSrce+OPZF7rrsLnxdzJutyEgxRRcUWFsigTXpd8li7x30SxLMpzxvyqSfrOa0xD8P/sOHnT+kLNe+OqctwTvSm1k/zqq1cG40wtrVCrxKB9DNeQz3D7vjol7zk/2f0PGDjiw7XdOK5Pnn4ZNPBAEIMDpyNDvn7+TdSe/avljmDjj6VJ3rztYOPz+4d+CdTHSZVeO5P079wbIzy0yWwjLiNsSx+u7VVg0J9UFrXH9KCskuwdZSSr/aICv96mvvKUmSyUXnVNapWo8dPhyio8U+P7fXGG5afxNqVzUlJeb8uCEWAtGICHjnHRgxQtQAliyp19AuSVzYcYGzf59t8HfFUdjK85MUEs7ezmg8HSOEwgaF0eXKLrgFujHl/drzj9OKxaKhVheSYT9D38XWj2XuhNO1O1tY5vp9duAL/i3/kkLXI8TH1zz28LeHOfD5gVqvdymgTKfCoNLg6ac2zZ8JeQkARHjWf2LKKs2iqLKIhZsWojNYN8E9tfkp4vLiiPCM4I0Jb9T72mqlul6qrT5B4kN7NN28p5QtPi1JP1npF6P/h4WbFvLrCQslav+3YeiP9R6rTQSMhM53Wlt360pAtiBtdw1M3C3U/lWwVPp5OInNR3FFMd+O+pYNT7Rt8q/zpM6MfHokJUZXdMp8QNh/t7Qyr7ER7S9EHhcKLlBSWWLaD6Wni7VoSYnYVxkxklmaTmZJJu5OFpuw7D2Qs7cFRl4NySvg6NOmbspYzRJORjzEzvR1AIx+djQjFo5oyRE2CHK9zNUVU6ZffZV+er2es2ftNzs1Nv4j/f5DrXByd0KpVuLqLzpLZTLMHjEtSeabo7wxaUpsW1PCi11/YefXMRgMMGwYvHvfeD6/8nOeGfUMCknBrKq94z//gLbcyRTUeyLzhM1rlpeLyRQgubJ15PnJGBguFiOZysNUys4vRjBoDWhLtC03sEaGQW/g7KqzZJ3K4pXtrxD5XiQ/xL0O1J/0O6FbTobPCvLd99K/P/wx+w/SPluJU1mBqbgoL+RCQ4XFRPUMG4dx8lX4TQ0bRsCJl20eklKYwobYDah8BNNoj/TLy4NydRXpZ2lnFzIZRi4F38saNsbGRMwH8LsL5B0mrGpSKKVh3f/ltoI6WhG2xe2mwOUInn4WRU5H36fRKBZgZXaC85oREZ4ROCmd0Cg1ZJRk4OZmLmZXt/i0zLJSKpSEeYbRJ7ialbApsHxME4/cDpROcH0lDPna9FCCtImd3QbzZWaVUliCJdcvYc+7e1pmjA1ExNAILn/lcrzaebE1YSuTfprEs/uFdWF9rD1lXHst/PYb9OwJ13S/hrGRY+nTUewkEvb0o0vaMwT4Sw5npjYG5OaH2ki/wkKztWewezAqhX0bq1aJ5L8JTnsClaICQ4mZ9FM6KRtto9rU8+fgBwbzwPkHbFqEyVZP1Um/Ll3E3FJaKuy+HEFRkfl6wbXU76pjzbk1Ypzhg+vdbQmi2Ufu5q5u8SnPg25e5STmJ6LVV1trxX4r7vdtWfXm5I3RORSFpKOoyghDttG1Rfp1ntyZYY8Pq/H4pYwjRyDzbAdGx2/ln1uWmvYSDUV6cTq5Zbm8uvNVm2smy6nBVe3K8HbDmdTZDqGdtVOsPytqCYluA/DzAz+XFNylmuSlXExOKrDeXHab1Y256+cS2KN+BRdLtKb15+mlp0neW/M7J+NiSD+j0cjj6x9n8FeDyS3LbeAIayLMI4yOPh3p4F3/5pvu/o6RfpIklHwA69eb7RUPHhTOQhER5vqDDGdnWFC1DExIwLxvvoRw4rcT7Pt4n0P7rlHPjOLBhAfx79qABaSDKCoy30N79bJ4PK2IjGMZJrt6R2A0GsmLy0NfWTshH5MdA0BH7472DwoaW7NB88zbcOQJqCUHWF5rZ2eDj04U4oudT5saQS2x97297HpzV61jbeuorISM8AGcGnUnIdHegJg/L9jKO3YQC0csxNfFl1NZp/j+yPdWz41uP5ogtyC+nPYlnhpPO1doPMiZuZZKP1ukn1y7ySMBgEjvSPOTfgMhdHLjDMgzCgZ9Br4WrlOHH4d1l0F5lpgYXYJFtnTSUsBcJ8vPx0QEFVUWUZhUSGlW28q2r47Okzsz7qVxlGrVaFWiQ8/HpZ6FwZZE+mZIXlnrnAPg5+pnyoiLyYkhMFD8qcvLrZsTnT1KqNBXmM4xYcpBmNwKGhCSl8PJV0AnNhaJ0mbig97neH7bnidjXv6Trru+xdUVk9Iv0LX+a9DmXHv+R/r9h1ox5rkx3HX0LiSF2H06YrEo3xxTU0XA9OanN5MXn2f/hAZCp4Pv387GNT2WELdCXnlFBHlHBvtwx4A7uK2/sAMYMADatROs/HffYfIat0f6yQs5Z2c4XyA2IQ0m/VLWQNJyMDQOITc0UuTI5LvtM42z7y19uS/mPkIva8Gw1kZGRWEFv077ld3v7uZCwQUSCxIpKhMTo7d3/a7VL6jqd+Z6kP794aaNNzFk8XQqXbxqkH5hYfDslmcJeyeMn479VP+Be0RByBTo9QJEP2bzkPkr5jPxp4nEKf8B7JN+ublQVlXotrKzc4uAiFlWwbgtBtd2YhOlcKJ9gFhsGKRKSrWl5JzNYd9H+5rku98cKM0u5d83/zVZKz599Fp29OiH1sdi3th7K/zdFfR1kMRp62H9UJE/1sJQKpSkPpJKwcICQj3EnCEX1+2RfvUl2psd1UgglUs5+e77SNGKXE5Jkrj6j6sZvrDlQrkvFnLh3cMobrANUfqByIcCeHTYo2yet5m7x4kqmlxAa0Z7eXJKc9hZ8AuJ/l+SlWWfMyksBJXegyGqO7i2e7UMzOMvwZ5a7HVbA9I34Jr4Jp6aHJRypl9ZHiWZJcRtjDM5KbQ1yB3Z9kg/SbK2+CwoL+CW5bcw6ttRNSwNZchzkK8v9SKfZWvPK6KucOj4szln+XT/pyTmm8mFTlXuYdVJP1npl60+ROT7kXT9qKv1AfE/wJl3Wo2au0Ho9wYVo3ZiMKpMHcThHvaVftFXRTPhjQm4+l0aVpPbX97OmvvW2C2cZ2bCl1+K/586RUH7oIu/KT4w+AFc1a4cSjvExrjaswTrRJd74cozIluyDcPPD17cvoonNh+q8VyEVxXpV2hN+vl08KHThE44ezdjt0oTwWg0snzecjY/tdnm80VF5nVZYEglj69/nKWnlzp8/ac2P8Vbu99iX8o+m1Z6DcUPs34g9oFYhrer/xrLpPTLrrszZMgQsR4tTi3gl9s2k7g9kd27zc/Zgr+/OEevh9jYeg+v1WP/x/vZ9vw2hxqIXP1cm3zOPnFCrOUiIqwbxI/9eIzP+nxG5nGx6XWEpNz89GY+6PQBh76qOR9Y4limkBbazDsFsUeztebo+YxQSUn2HWkslX7lyTLpd8amBfHMH2Zy45obax1rW4e83pMkoXKR0d6rPTO6zrDK6XMU3s7ePDXyKQCe3fos53PPU64TNZ9Z0bOIezDOfsNLI0NW+smW2lCT9JOJF4CsSrGGNJF+RqNZhdcUMBrByUdkSGuqGGlJCQfvh1NCCWml9NNUKf0qi3kw/kFmfDuj6cbWjCgrA62yivRzbu1FCgucfhO2z4C1fUUzeC3o5t8NN7Ub6cXpqNXmBoS0NHM93sVXFIM1Sg1u6mr5pBfZmNYo6PmcWJsqxdi8nAVxX1AhvkAHPj/AL1f+gra0bQlXDJIKg0KFq6tQZV4RdYX9+08rQSv4NPyHtoS67D0BKwlyxrEMdryyg6xTDbeeNBqNZMfUXF0dPQpZru2Ju24RL6zoY+UdbwlJgjvvFP//zz/grxetZ5ZdPJaQyTQ/P3hp7EusuWENc3o20KL01Kuw+2ZhgZe8wvq5kiQ49WaVNZ5jGBh2GZJRQblTMicvtIJMtyaCylnFjG9n0OfmPmSWig2Cskx0UNSXgBjTVZB+hqCDBAeDbydfRtwvsnfy88XiTbb39A8p4+fjP5NTllO7TYg9dLoVxq4ROW6eXW0eIvt0ZxtFOG9tpJ+s9AvzsGhfbcrFZH0RMRPGrAbvngT7uaIwCOu2nNJc0g6lsfb+taQdbHl1W0OQF5/Hxic2Er8lHqPRSIFW/KEifCw6eTT+osNOWYddTdBY6L4QwqY14Ygdh5+rn1WBQLZzrE3pt/T0Uh5b/xgbYqtZg8R+A6lrm2ysDiHvGKSYC1jBXlVWswZzJ3v32d0JH9yMjFYj4N83/uXHCT9SWVxpsqN20orU+IYo/WzB0xM6WDTnV++Ub0qczz3PPZtuJCbsaSoqjaZNdHUUFIB7RVduDfqcdydXs7fL2AypjVe8bBJ0X4hxWizFukA02kBC3cLxc/Ujdn0sP074kQv/XmjpEdaJw98c5tRf1kXZBSsW4Pu6L3u1QmVbnfQDM+m3f7/I0/vl+C/suLCjhlpHhjwH1cditlxXbiJNHCX97l59N/esuYeVMStNj1nm+llC/lyWqMTfyaoJB2D4rzDhX8cH3Erh7m7mLYuL4aEhD5HwYEKDLLXaGs6tPsfZVWdtFs5Pn4YbntrCBsOTKAPPctVVjfOa/q7+3N5fKLdf+/c1u8cVVRTxwNoH+Pbwt/YL5WpPseZUtm3iSy6wl5SItbkl7Cn9QOwTDXrbjQRtCkaY/etsRiyybXcl5/n5+8PK2N95a/dbXP3H1fx58s86L12uK2dT/CbTz7uSWkenvaP2ngAqlci5VGnLiP9+B+c3xHOgSsxgj/STJOhatR07c6YxRty6MPP7mcxZ5XiNojS7lD3v7SFpd9PYMR0XvXY1ajLtRrRjzAtj8I705oejP+D9ujf/nP+n1msNeXAI/W7tR/RV0bUe1yuwF72Detd0IpER8x784W4VAwAI9ZTfQKgl701ea+/dC9o0s9LPFukX3CcYvy4N7MhrIygpAb+kI4SlHbDqc5rVbRbLr1/OA4MfaNB17x14L+292pNalErUh1H8eNRsj9mcObay0u9k1kmTq4PcDJmcLDg32ZnEzQ2SihIAC9KvLA1+d4IjixpvUJnbYXUvsc+VJOjzCoxbb16wSQoY8ReM+B2wJv0slX6XQgbz33f8ze+z/0Cnw6T0s8znbvXo94YQBpQmQ2FMrYeunLOSwkWFTI2aCljX12XSz8lbFK2t6jraIkhaBsVx1S/Z/HCPFGvTqjnWx1W4xRRVio1V1sks4jbGUV7QetwWHIHbTbM4N+QmXF1F1vuqG1ZxY+/W3fDxH+nXyGgtQeCNgbK8MjY/vZn4LcK4XKfDZPtTW+6kZdhotxnduOfUPXQY2/C8neM/H2fDYxsozbGWpO/cKf4dNkqFq5cgG+5dfS8f7fuIgnLr6mHv3jCtqt6euEN0Iq6IWWHTxtGS9AvzDGNK1BS6+tsmcOrEkO9EQWjXTbDjaig8BxU5IvctZ6+wlahOBtYCdyd3AhCdvP8mCK9mbZmWQ18dIm5TK5jcGwlqFzV9b+lL+5HtTcVuY4lQttWX9BvaXthgZqoOUlIpFBVubuaMpfR0c/dWjLSc/PJ82nu1Z1yHcRf3JoxGSFkNpdbkrEz6pVaK3Wd2tuhArY7cXCh3qiL9LO091w+FNXY2Ni0Ib28JtU5MDEk5uUSOjWTelnlEjo2s81yFQkHHjh1b1fwZ0D2AW/fcSq8belFYUYgOYcXQPsBCYdnvDRi/zc4VLKB0ElkS3j2baLQXh+oB0TIsSb9159fx9u63+TepWnH70MOic60lceIF0TlXRYiH+ojPYRmNZ1/VEii4UEDK/hQUagV55VUr/KpMuMYi/cC6ONOcpF/voN4oJSWV6kzK1Sl2LT5l0sXTlrPP+C0wo5WTZq6hSB4dcXNX4V90OTuuSuLPa/4kfEg4Uz+ZSnCfBjSYWKA55s/NT29m99u7rR5LyE8grzwPXZno4LRF+vXrJwq1qamQkaYyZYqeybZdfW1Int/OCzsp1ZYS6hFqKtjUhQkdJwCwIc7cxCCT3/ZIv4Iqy8Ea9lUuweBVe1Gy1SP/BIrTr9Ip8Dwg1vohHiG0925vM4exMLmQX6f9yqGva1dgtBUs2LWAOw/fWePxzZvhySfhlOsnnA9ZTPC0T2vd/9QXjwx9BJVCxeb4zexL2WfzmKMZR/lw34c8u/VZ+2qe0mQoy2i8gbUQXF0h3OcCl3f4lvxk69AsmfSTbeRkZJ3K4mXNy+x4ZUeDXrM1rT8lhUSXK7vQ8XLbNoWW1p6/nfwNEJk+D617iDJt7fmazipnNt28iTv6iyzK3cm7az2+uSCTfoUVhegdaGqcPBkq3P05MepuCnoMpbxc1CSiouyfI5N+zRif02zw6ehTr4a2srwy1j28jmM/HmuS8Ryt4tWqk34RwyIY/exo3ALduHfNvRRWFDLl5yl2Vf8AboFuTP9qOu7BNhYXFvho6kccveuofaWFawQEjrLORZOhr4By+7bIciNCaip4lFWRfi6nKSwyUlGthGQ0GCnJKkFXoeNSRXExBCbsw/+MKMI11vypUWl4Zdwrpp93Ju28qOs1FJHekXhqPKnUV5rWqcHBgl8rKxNki9ys7ReoNTkhtPeS80wNED4TPLs13qCc/KAyR5DWsi1kdRVX8DhwE2OQ62S5uWbST2fQEfNPDCd+t+1y1lZQcKGA3Lh8ACplpV9bsvf07iWEAVelQcd5tR/q7G1lI2/pyiSTfgp3MXf5u1oUBYrOwY6rIMEiZ7KlYNCLtalWSIR93cRGvkQrwsMnvTuJp8ufxiPEo8WG2BCUVlESrnX1IxiNsGc+nKzZ2KdQKIiMjGz0sdlDy69wLzFcSkGihcmF7HhlB4nbRaFDLgCrVGbCxBYsJyUXXxcCogNQu6obPI7DXx8mYVsCSidzJ5ZOBwfXZRMUt5tBPQSRE58XzycHPuGBtQ9QXFlc4zrz5omCpiZlAp5SCNml2VZd3jIsSb+LhkdnCLtCFPwHfQ7O/rBxDPx7HYRMgtGrhDqsHpjt8RbDzuyko0H4hRv1Rv6+/W+Ofm9budjWIXsl6wuFwqq+9p5d/brS0acj5bpyKzsbuaiYkmJW+q3LFIqF+X3n1z+vJeYD2HktlFetBjO3wrYr4bR1l7wcznu+4DQqlQiYt5UNkJcHLpXtCFZ3sVhMAn6DwH9o/cbWVKjMg8NPwIUlqFQwOmUVo0+cIFDRFfcgdyLHRDpkJSNJEp6enq1q/nRycyJ8cDheEV6mz6BS705oQNu3M9ubvJcZv83gntX3APbtPeVFpbc3pBWLJ0PcLarxRiOMWQN9X2/iEdeBqHtgyA+AKB5E+ImqbKUy32RB+MP4H/i016ctNcIGYepHU1mYvxCVRmUi/fTFjU/69bHoIWhOe08XtYup4Jfvts+u6rmgAMrVqajcCm0fUJfStqWhr4TSZAK8BXtkap7q7MvAuwfi0/HiNqzNMX/O+XsOUz6cYvVYQn4CAMpCwZbZIv1cXCC6ig87dcrc9GKP9JPvxfL9uaC8gCc3PWmyerKFyztczqE7DvH5lZ87/DuY2GkiAFsTtpq6uWWlX1qaeUMH5rVvulZ05Ub5WlSXjUYoihWdtW0Z+cfh6FN0CxbrSHuqWxkqZxXn/zlPwYU6DmwjkCQJFx8Xq8cyMuC996DUmEumr9gr3DWk9iJNfdHOqx1ze88F4LWdttV+MhnYL7if/QvtuhHWtM6movqiT7uDPDRkAZUp1kVfk71nNaWfi58LnSd3xqdTw+bR1rT+rEutmFjlRuwblsv62PUAXN/zejbdvAkXtYvNc9KL003/7+7kzgtjXwDgeMZxCivs3FPrgSWnlhDydgi3r7y9QeeHeoSS9XgWKY+koKxFcSUjIAAGDFZR7hnIj7+Le/+QIbW7K1+qSj99pZ6SzJJ6KXj8ovyYu24uE96c0Ojjyc+HCxfE38Iyz686LHNxl59Z3ujjoDTFmsiLvAHG/gOaaoUdfTn86QEH7rN7KctakFtFFEpJiU5ZRLk6tcbeffe7u3kr8K0263DjCEpKIK7fbEqnXQeY58+88ryLVpLd0OsGVly/gt237ub7md/XfUITQJIkroi6glndZpkeU6vN++TUVLE2AHAOSMZgNKBRaghyFy4suIbDyCV1Ejr1gld3mJEEF/6EfwYI4YAtVOZD4TkT6VdUBM4K88J8+yvbWXt/CzvzXCTm/jOX2etE44pB08bsPSvzzXEwKrdaD7UF+TOYnm523pPcqpR+LhYTlWs7GPojhE+/iME2EtLWwrJgSPoLAH8PofQrNYi9g0LZ9qgoo8FI8T878cyKxcWFWvenSBL4DBCZ27kHqz0l5s7mQtv7Tbdy6G1Jdtoo/Lr4ce/pexlwh7BHlCcYb+/aF9dysSYzU5BzFYUV5Jy1wWo4iBtW38DcdXNxcncyhTkfOQKu544ScWo9fohr/3ZCdD2O7TDWWhlVBY0GHn4YlJKKwNT5AHx9+Osax8nvs9j9MM9vfZ5159c1bOD5x0FXVTny7Q+dFoDKXah9/IaA2kMQgvW04xkRMhHf4uEU5IgNnpO7EzeuvZFRz4xq2DhbIZL3JPNe5Hsc++kY2aVi4V6RJ6rc9VX6SZLEdT3E4vSPU3+YHpdvnsePg1YLF4I+YUfKJiQk5vVtwGKt4KSwOVR7i58Dx0CfxTWy/eSiZ1xeHL4B4uZvq9idmwt9E77ly94xjGw/0vzEwI9EqHNrgNEoVF5pwqKlo8sAPMp7UFYkPtNGgxFded0dj3q9nuPHj7eq+VNbpjWNXSb9NNpAc5d//kk4shDyHeyau7AElgSKfL8WRqm2lJUxK00qF0fsPU2kn4cF6SdJEDBc2OO0JIIvhw43gkI0l0QEmCeJjIJ8APy7+RPYq/4hy60F+eX5AFQWegONS/r16CGaeSRJKAiaE2MixwCQ5bnePulXaGBn9CBmH/Bi7TmLDavRCKnroLCVt+9nbIblEQwLF/efokbmh5pj/gwdEGqVG6wz6EgpEkp0ZbFQvtki/QC6VTU8nz5tbnpZuGkhD//zMGlF1pOOpdJPZ9Bx7V/XsnjnYm5edjOpRanM+G1GjSwqSZLoF9KPK7tc6fD76RvcF39Xf4oqi9ibIlwTPD3N36uEBPGv0WgmwBJLBOkn38PFL6IE/u5ca9GwTSBkIkw+SJJWFIKLiqBMW8aijYu4adlNpuYJGa7+riwqXsTYF8a2xGgbFUVpRcRuiKUs11opFRcn/v4VUb+hp5I+QX0cVpLWB08MewIJiRUxK0gpTDE9fiD1ALP/mM1j68Uasn9If/sXaXctRN3b6GNrCeQph/HitpUkacdbPS4r/bJKrSXh7kHuzFk5h943NixPpTWtP7e/tJ3XfV8n51zNPXNlJWyrMpZI91mKzqCjT1Affp39q9WcdKHArIQ8nHaYLh924aVtL5kK8sHuwXTw7oARo111aX2QkJ9AenE6xdqazbaOQJIka5WCA5g6FZTacpQFYsM+tI4+yKgosb7Jzjbv8S8FZBzL4K2gt9j5Wv1UUZ0mdsLJraaC+2JgMMDBqppmhw7WzeEGnYGfp/7MbzN+Q2fQ8b/h/zM99/L2ly+KLMovz69xf2L9MNh5Td0nK52hwy0QYNtOF6zX2u3DnejkK8J/SzXna1h8hl4WyoA7B+Ds07ZtlmtDcTFUuPvh3EHsBfV6PQeOHMDvDT/cF7vXcNqqDyRJYnrX6QwJt+PV20z4ZfYvLL1uKb2CzMy17IKSnGxRs/FOAKC9d/v6N4rXB5IERj20v1bc66vl2ANCUbW8Hey/Ew8Pc357caGK3CdyqXy6kokvT+SaPxz4XrRylFUt1cIqR/PUyKeY0nlK7Se0FpxcLJoMiqrCZRN+g7UD7DYNFlYUcvUfV9P3s77oDDore0+5PhPk6cO4DuOs16bO/tBhrlAVtjQ8uojGbA/RLBngIUiusirSrzijmPP/nKcore00TlYUVWDcsAmf1JO4uBjxXOyJ+6vuVut3K0TMggHvgu8Aq4f1ej0nT55shhEL/Ef6/Qe7UGlU+HfzN0luZdVHXdY2vr7g5CQWgFlZ8HH0xyyZs6TB41C7qnEPcufjbh/z75vCWm7nTkiNGo33XdfRfqTYCMqETm35e127wjXXQET2AjoW3cwjA5+qcYzcubVf9w0vbHuBX080QB5tNMKWSbCh2kJSoYZhv0D3x82PFZ2HAsfbD2V3QcvFZufJnfGLuoR85CVRVFI5q0xhr4pKUcivr9IP4Loe16FRatAoNabNhUx0HDgASX7fcixCFEyeGfWM2Zu9Phj0OVyVIawcQSzSeiysYScS7B6Mp8YTg9GAqspKy1ax29HvW4vCyQemnYf+ImdL/tsUFEBlSSUvqV9i1V2O5W21hoKLJfa+v5dXXF4h9UAqmXKemi7A3PWZswdOvQ7F8fYvYgmNv1j01BIY31zo4tcFEOporV5rWkhmZ4vCkgxL0k/uFrdS+ukrwdD6wpc93VWo9GJhmZwjKjxTP5rK7F9mt+Sw6o3Y9bHErhebg7yyKqVfiZgHG0WJXgVXV1i4UDTFNCaZ6AgmdxaK9Syvf8jMtF34OVv2L+VOKXiovaxtl/VlsHUynHi5OYbacHhEQdcHKVZ0xyBpuX3PUKI/jiYzI5NPenzCpic31X2NOtCU86deq0dbZv09Ty1KxWA0oFao0VTlTLrZaVy1VFnc3v92hkUMo1xXznt732PqL1NN9+TiymJ2lnxDmvcyEhVbuHf1vayPXY+r2pWFIxby8b6PWRmzkjtX3Ul+eT5Go5G159ZyLudcvd+TQlJweYfLAaxySmWLz9iqPXlJibDfNmLkfP5poBrphxGin4BQx7IEWy00fuDbH6cq652iInBSOvHmrjf56dhPpsYXS6g0NgpPbRBxG+P4aeJPpigDGelVAqk4T6E4mNencVV+MqIDovlgygecuPuEqWFx1LejGPjlQJaeXooRIzO7zeTuy+62f5Eu90Lv55tkfM0NZ+8g9qdOIy3f2uO3T3AfMh/L5Pz95xv9NVvL+tMjzIOgXkE2ra42bxZr68BAOFAuspuu73m91TFbE7bS+YPOvLD1Bc7lnGPKz1MoqixiS8IWK2Lk7Ylvs/GmjQwNv3jXEJlktHIkaSAcJX/694foo7/RbedXuLlBzzpErs7O0L5qeJeSxaeThxMD7hzQoLzqvPg80g5fnCLNaIRXX4Wrr4YZM4QyGqzdIwAqiysxaA14d/RGpVDxwtgXyHo8Cze1G4fTD9eZ7Vcb7ll9Dx6LPfjm8DfmQRm1ZsttXQn8eyNc+Mv2BQZ/AV3tN+34+Jgb3SdPhlVzVvGYtgC/4tE1lH6RoyO58rMrCYgOqHmhSwTFxUZUlaW4uZhVyanFwqJBQsJT03zKleaETPqlpJiVfoPDh3D0rqN8N+M784Epa+DAg8JyuzGhdIIeT0KvZ2w/r1BC9OPQfg6SZK7H5OUJ+0u1Uk3kmEgix0Q27riaGcd+PkbSLqH2j2QsL497mRndZrTwqByEd08IvRLcIsXPukIoTRSiARtwd3Jnzbk1HM04SnxevJUrk9y8Mq7dZDbdvIl3Jr3T9ONvCDy7wMCPRYM4EOQtlH4VUiEGA1zYcYGfp/xM4rbElhxlvaB2VVM053bSOw3H6FSI1qClRFuCr0u1gm1hDBy4XyjKZVc/o9FkdQrNu/b8j/T7D3ZRllcmbCMMYhEuTzB1Ka0kyVqCPPihwfSZV/8MsqK0Ina8uoOSrBK82nmh1ChRqpVotbBnDxiVKsbf1w1JkjiXc44j6UdQSkpmdptZ63Wvuw6i/DrRPeZ7EraNrPF8Tg4Y0bOnUBCV13RvQFeMoRK6PQZRNjbpljLJ8kz4O0pkUjmIgABI9fmLpWX3WxW7KksqazmrbSF8cDh3HLiDjjM7MiBkAB08o1DrvXBzE4RyfdE7qDdZj2fxy+xfTBY+8mc0IwPUOj+UOPHwkId5fszzDR+4yob1o0EHOnMHuSRJpqKh1luQvbZIP3kzYfV9K46D/fdCZst43deAJIFHJ1ALiUe2xybOhrzI1gsbUbuqib4q2kod0pbgH+1P77m98Qj1ICmvSumnCzSTLZE3wZVnIGiMYxcMGg0T/xWqtBZGqEcormpX9EY98fnxeHiYfcnlzQyYST9PL4OJ9At2t8gfS1oCv2nqlUvaJEhdC8sjIFlYsEkSuOva41IRSU4bC4e2xIbHN7DmvjUAPD7scRYNfA2Psh44OzvgI19PDB4MY1tAtDO6/WjUkoYyTSKnMm2Hmp9WigLnlMhZaFQWVp6SAgZ9AR1vbo6hNhwenWDAexQ4DUdhVBNbepgz2WcoVZZiNBpRqFv3Ujx5dzKvur7Kvo/MqhDZYi/ELRwJBS4u5s7i6pCVfklJ4K9uz875O1k/dz1Dw4fy2NDHTPfkk6nx7Am6lYOdr2LuhnF8cegLJCR+vupn+of05+lRTxPlG0VqUSpzl85l+DfDmfrLVJ7b+lyD3petXL9Ooomf+Cr+R1b5KT2yySvPQ0Iy5RICwrGh3+uiA7stw2gEbREBXvkAFBaCUqEk0E2oo6srMkFEABz6+lCbt/iMGBrB1I+nEjbI2iEkNRWKnE+TZNiHSqHixt43NtkY7ht0H9EB5lzIjJIMlJKSm/vczMl7TrLsumVm+7BLHPIaKy/b+t7tpHQiwC3Apg3n1he21lvt1Box4PYB3LLtFpzcrTc6RiMsWyb+f8YMeHviWywascjkYiJjQ+wGtAYtz297np6f9iSjJIM+QX1Ydt0y1EpzzMas6Flc3vFy3JzqbzFWHefzBAl7MaRfenE6036dRpePujhE/CkU0GlWL7IiBzFkiBGVA/0HcvNJjO1lRpuEf1d/rvzsSjqM61Cv83QVOj7p8QnrH7k455H0dNi9G6tsOw8PGDPG+jhnb2dhKfq62VLU39WfBwY/wPy+802NiA3BsYxjlOvKzXsTSYJZqTDwE/HFSVkFib9AfsMyDFUqGDRIkD7jxkGUXxRh/oLYqq70+/+AwswK+q5/E2m1uaE3vUzsDyO8IlqFTXJjwGg0ci7nnMkC2ZL0k2s2EcEu9A7qzdAIi+aJrO1w9gNR6G9u9HoGOgubZbl2JDeQyzAajXXaSLdWGHQGls1dxunvhDuHi21H69aLDjfBqKWCoAXoME/YtvrbVrYqJAVd/cWN63T2aVODdn6+2RXFZk3+2POiJlLS+oi0ER0HMPrEKYae2UZJiVBHT/tyGqED206tUKlWUuQRSoW7H2UKMRm4O7nXtFhPXg5nPzI3ABi0QoG+fYbZ5rUZcWm0af6HJsG+D/ex9bmt3HX0LoJ6BzlM+oGwZrpwQUxKUx8f3qDXP/r9UTY/tZmA7gF0m9mNu47ehSRJ7N8PqrgYAtoF07276Bj489SfAFze8fI6bUKcnOCuu+DZZ2HVKhg/3pzlAoJsyXX/l+yKNLw0Xkzo1ADfe6UGoh+p+zjnQOj9sshpcxD+/pAY8DE5rlvZntiXKL8o/rr+L079eYqnK59uk/7I9uCscmbf7fs4fhye3NwwlR8Ios1DY905G2LRRBxcMJ0XQg7x5MTuDVuwpq6DikyImG1N/GXtgo2jof/b0PUB08OPDHmEUm0ppScG8Q81ST+jEWK0G9nf50YUB8ewIloUvSk4A+c+EXaxgfbtSJoVZRlQmQte0cSr13A27B32Zj2BJI3nmj/bro1Etxnd6DZDVKu7ZA6mW/Lr+CkizaSz0gk8u7bcAC8CkiTR2bczxzKOcS7nHF38uhASItQtqanC4tHS1s7gnGPqErcqPDoHic+8W/0KDo0OlQe4hJrsPQHmFh0jIQFCq2rxMStjuPDvBca+MBaVc9tY+kx4awL6CtEFdl3P64g2wPFK8A+v3WK7LcHNyY3+fqPYm72BU4V7gG5Wz5dX6En2FB3a1/asRqwonU0b3LYA2e7KRfJGa8ygUF/IvadavyWfs7czvef2JqCHuXs9qVCQfkHOwtqztpxnLy9xv01LEwXX/v0lJnSawPiO4zFiLvDm5igIKJiC0SmfgIh8tAYt/xv+P1Mjl4vahW9mfMOob0ex+txqMTaVM+Ge4RiNxnrfu+W13eH0w5RqS3FVu5qUfuerxETyHOjpKVwAskqycFW3/VzXGtAVw5+eXO5zC2v4lsKqqK8QjxDSitNILUplANbWNMl7k/n7tr+Z8e0M+t7St/nH3Ejw7eyLb+ealgrp6ZDsJ1R+UzpPMRGgzYGV16/Ex8XHsdfUV8DWKyB8BnS9v+kH18Tw84MPJvfBWaMCDtZ5PMCJX06gdFIyYmErWRc3MvbuFWszNzeYOBGcnfvQJ7hmM+0rl79CdEA0d666k1JtKZHekay9cS1ezl5NNrZDaYcAbI7HUfi6+LI5fjOl2lJOZZ2iR2CPOs+59eMB9Dkg7MkdQdeusG7dpUX6NRQqjYrxr4/Hu733RV1HVsR36AAvvSSK8Gq1eX1anF5M0u4komdFIykklE5KDqcdJtAtkFCPUF4Z98pFkUQVugpicsQftFegDSu7Y0/DqTdg3Caxb7aFnANw4kXo+qDdpsynnxZ7InmocmNCddJPW6pl5W0rCRsUxpCHWtaisqlQWgpZ7frTtaeZ5JdJv3Ze7VpqWI2OKT9PYV3sOn656hfm9JpjZe8pZz4H2erD6fUCdLkPnINtPNlMMBrx8REf1rw8eGnbS5zOPs0NmTdw5KEjXLfsOqKmRtVxkVYICa5fcT2x6W7wN5S6nuZcjooIrwicVW3QUteBPPpo/2iOpB/hdNZppnedjoeHcOLIzBQOJBqPEqBatoLGF5xDQN109/16YddN4NYe+ryMl6sbgVI0ZTrxPkIjvel/Wy3W9a0Q+ko9ZYV6MKoplYQTmM21evQTEDQOfC8TPyvUYl5oSivgWnDpsAOtBArFpfMrDRkQwoC7BuARKqo5surDEbtBSwmyIzDoanadDH10KNcuuZYuV4oOMHlhuH1jJR0PLaHz4b9MCzCZ9HNUldevH4wYAQXOx5j99f1sid8qxmEQisZUX2EVOit6Fk7KBkjL7IXs2kLPpyDEcWLR3x+8SwYDsDNhDwDtRrSj99zepgJxW0fK/hR2vbWLwhRReZI7leqb52cL53PPU1xZjN49mQqVWdY0qEOPhm8+zn4EexbUfNy9A4ROAVfroKzrel7H/H7ziQ4Tj1uqq0BYihUrkqhUZ1JuLDQ/ETIJZqVBu1ZEpv17PWwQxL6vq/gD5ZTXL8NToVDQtWvXVjt/Bhn70Dn9CfqqLUiHvKMilLk+OPc5nHytUcfWUET5igX/uVyhFpZJ8JQqS/LiYpHJClCqEBO5v6u/9XwYPA5G/gk+DcvSaTQEjoBJe8V3rQpyNrJctI9dH8uuN3ZR3oaUf50mdDLd/8BcYGhuC86mxkvD32X80RT8k2+p8dzaM9uoUGeg1vkyrfv4mie3BRiNsGM2g12EIs0Zb8Cc03ixaOr5M6h3ELN+nEWHsWZy38fZh8mdJ9PTaxhgP89Phqz2O2PhZC5JklUOikdZDwafW8N8wy5O3XuKc/ef47b+t1ldZ0S7ETw3+jk8nDy4f9D9xD0QxxsT3mjQvbudVzvWz11P5mOZJiKvWzehWIyPh5MnzeveYI8AXhz7Ip9e+an1RdI2wJapkH3x2VgtCpU7dJxPoUa4X8i5k6EeovtWznS1RPtR7UXx6Io2WDxyAGlpoDBq8FR7c0vfW5r1tbv6d3WcZNQWQO4BERVwCcDfH05kjuZ8/rAaz3207yOu+v0q1sdaK5TmbZ3Hgn9trMEdQGtZf1YUVrDytpWcXVXTf3LpUvHv1KnCqrI2zO09l/237+fJEU+y+ebN1jnMFlh1dhWPrnuU87kN/9ykFqWSXpyOQlLYzrvMOwJx34l7YC1wUjoxPELsI7YkbHHotSUJBg503PVAVvqdOyf2+pcCtr+ynaVzlzZIuTP4/sF0nX5xjYtxceLfqCjR3OPkZN2Qtu7hdfxx1R+kHkw1PTZnyRzC3w1nQ9yGi1aFnck+g86gw9vZm3DPKovT3MNw8lURvdD+etGY6NsPnLxtX8SgFbn0xbG1vpY8VIPRwPLSx/i32zCSsq0DIlXOKk79eYqUvXaynS4BlBqcSew9jaAJYt+nUCjQugr793aelw7p1z2gOyAskwHCqz5e6eliX2nEyAuHb+P9Pe9TVGGRR6bUiFgXW7l7TQ2jETZPhJ3XWin91sWu49cTv5LnlSfyPD0aN8+zuaBQKug6vSuaTuKP8bfLNXT5qAv/Xvi3hUfmAJKWw9ZpkF/NyjP/pFDmFSfYPE12BjuTIzZPwRZccqUqi+4/edDu3XZo9RYRDF0fgMn77M95zY2MzZC92/Sj3CTa2Pn2zYVza8/R6bfF+KYco8goVBsBrjYsnSUJ/AZa3xQHvA/DfgWlBoVCQVRU8+2fWmeF9T+0CnS5ogtXfnolrv5iRV1fpR+ITbNBZ+CHy39g5e0raxynr9TzVvBbLLtpmemxYz8fY/vL29GV64i+KhqFyvwx3f/lEXIWf0FirysZunA0AOW6csI8wnBRudRp7WmJ226DlOAvOeL0Ec+vEoWc/HzQG/Sk+1QpC7o3wLJJWwhLAuD4S/U7z0ECwdkZQvSC9NuTLCTug+4bxMzvZ6J2Vdd2aptBwpYENjy+gaIUcUdoLNJvzpI5RH0YxW8nfuP29Vezo0d/8l33A+aMvwZh0KcwemVNe0+XEPF4xCybpwVW1XSysqwfz8uDMichB4/wsrCcUijBJRjUrcgzv8PN0O0RMBrxqyL9iqo+y3ve22OyJ6wLTg3xbW1CbHtpG2sfXAuYrVZN1p4GPawbCLvm1u+iCT/Cmdbhuy7b6cgWwfK6Q84Ulovdbm7QN7QHyQ8ns3Xe1uYd5EVAJv1kxcqop0dx//n7cfVreyqdCl0Fm+I2sS/pIEaMjZrn12SozBMh5XY2MpYY0bUHztpQiovNAe0y/jgpVM7tS67CSVXt/pa6Fv7uCqkNz4NpFkgSZG7HT3kUACeDNyBIv1N/nWL/J/sv+iWae/6c1HkSa29cy7zwV4GGkX7VkVpVFwyxXaM24bkxz1G4qJAPpnxgt6DtKCZ0mmDlAuDrCxOqerB++MHcNOBlr2G2NAkyt4C+5KLG0eKQJBjyDTnegjiR37ec4WrL3tMtwI1uM7vhFnDxFoEtiQ+jPmT5LcutHtPpxLqsa+oLnL4tjWldprXM4ByBcyBckw8D3mvpkTQKfH3hi0Mf8OXhD2s8tyd5D8vOLONI+hGrxz1CPNB41t01bw+tYf2ZG5vL4a8Pk3bI+rt25gycPi1sBvuNSeamZTex5lzt6+ruAd155fJX6OBj34Xh7d1v886ed9gS7xjJZguyyi/aP9q2Avrsx7BnPlTU3Qg4JnIMYC6y14WU/Sn8MP4Hzq1xLNM1PFwo0crLhRPRpYDkXcmc/+f8RTn8VBQ23GZMJv1kW+zqmPj2RCa/P5nQAWKDXVJZwtkcQWr3DhKkkVavZfmZ5Ty/9Xn0hvo1Lh/LOGa6lolAzNwGR58S92bvXjDiN5E/bw9+g+DaEuh8h0OvqZAUHCj6mzz33Zwq2Gv1nKSQWFS8iNm/tq388PqgpGqpY5nhnFoiFm+XktJPno+2JW4DRP3J2dncv2D0SuCHE1/z+IbHUVkSfPknoKSFJhhJAiSQJCvSz91JLNAN3Q3M+XsO7UdefP5qS0LeJ1ZIojjo49IIioCmRtFZ0VxQXd1XcEJEPGXtsHlatL+wfT+dJTLFLfdHZW5C5axUKK3su1sdZlyAy0V2vdFo5GTgc5yMeJi03CLKcsv4MOrDRsm2by64BHiQHd6HCjd/inR2lH6nXofsvTVPVijN9q4x76OJebWJR2vx0s32Sv9PYLhU2sdsQCb9bCn9lp9ZbhXEbJnpp1ApqCisQFdWU/2mdFISPjgcv27mKubR74/y7+v/otJYd8lotbB6pR5JW4lLhD9jbusMCHunVTesIvuJ7DqtPS3h5wf3DhXBmjtylrLm2J4qa8+dVKgz8Hb25vKODcjfKssQyhfnelgBbZ0Ga3rX2Q0pI8pVkH5n805SXFlcx9FtD33m9eHW3bdywuUEHd7vwGvnbgAunvTr5icqj/evvZ+9KXsxKktx0onPTFhYbWfWAddwK5VRXTAYDexK2sWytPcwSFqysqz/9Lm5UK4WnYJhHhYDK4qFwrNgbEXzTKf50PNpkCQT6VeiFwuxuI1xHP76cJ2XMBgMHD9+vFXNn2dXniVmuVhQ7U85SIHrIdx8q9qSDJXCwiOynhk/Q76DqQ3LlWhsRPlGmXL9AHpVOeOcOAF6vbno6+0tFpRhnmE1LZeOPSc21y0NfYWw8ElabnrokOojdkQP4q8LonDoHuyObydfqyaS1gyj0cgb/m+w8vaVpBWnMf7H8TyTMBwJiQAbDWWtCpV5kLkDds2BpKV1Hu7iYiaNLBsg9AY96xJFQ1B343U1TzQaQOHkkEVKi+OqTGLDlgOg0nkDgvTb//F+Nj+9+aIu3dTz58EvDrLy9pU2c4OLq5YfjpJ+Z8/aX+Y4Svo1Na6/XigWTp2CLVX18CL3Q1wouICh+r230wK4rgwCxzT7OJsCcvOTnGloIv1sKP1klGSZs7/bGvRaPd6R3qbmRhlZWUIN5OQEIQHOrbugIuMS8XwOCxNvJS/PvO+UEeEp3DGSC5OtHi/JLCHtcFqDFE+tZf0Z3DeYx7MeZ+C9A60el7P8xo6FjSl/8tOxn1i8c/FFv96wcKGk3JW8q8HXcFY5M77jeC7vYGOvnLkTvHvDqOW2886rwbLIXmOetQGlk5KkXUlUFDlGWikU0KXKOKG25pO2hBtW38AjyQ5EidjBH1f/wUfdPqo1R1FXYd+5SLb3tIxIsYRHqAeDHxhs+vl45nGMGAlyCzJl8CkkBTcuvZEXtr1gIgQdhUz6WVl7drwFJh8AHwct4xRKq2gAR3BZkLDujK3cXeO56rWrSw0lpy/Q4dASDMliwWYwGDiVcgoQmX6XCka2G4mERExODGlFaUiSWe0HUBksMmQHhA6wzvLaegXsuKqZR2uBsf/AiD9M9bL8fEyNbW29XpiyL4XFnotJWS4aJcuoIv2c2wDp1/0JuLYI3Kt1SIRMFPNVO9siE5PSL/sMRqPRSuln8BHzZY1M1FOvQ9wPjTb0i4ZMciEcXg5p3iI+6D2Sc7NQu6pRu6pRubSdedO3VxgJfWdS4hNGvs6G0q8kEY4shDPv2r+IQQuJv1OadHH7//qgbVS//kOLYNVdq/jnYTORJxfjqluLXSi4wKzfZzHl5ymmwFu5cJCWJgo8t+27jat+Mt8E9Vq9qYB0/YrrGfPcGNNzN6y6gQW7FqB0Mk8SOh28/jocMvTlzJSHuO/V0Br724bkrDxwbV+iKq/BKOm4bsk1nEnKokKdiZshmFnd6mntWZkvCBnPKBi/DaLudPxc/8EQPAF0jnWLR/qF4lwZjgEDB1IPkHU6ixW3riB+S7zjr9mK4R7kTviQcHLJJSE/gexy4Rd/saTftT3ETbVcJyz+rlH9gGtlB9zcaunkrw1Gg1B01tbFmvg7bL+qxt/2il+u4NldD1Psehydzrq4kZsL5U5VpJ+nBel3+HFYHe0wOdzc8Hf3BqDUkA/AVT9dxRO5T7TcgC4Ct+65lbuP3w3Ax0m3sqP7ALJdqiwkVC7QYxFEzqnfRT06C6VmdWRsg7zmJQPn9p5L8aJiPrvyM0B06rq5iayC2Fiz0q/WHM0Lv0PyiqYeat2QFHDkf5D4m+mhcnUaBW77uVAiusC1ZVoKkgrQlmntXaVVwaA1ENwnGM9wT/LKxMbGGTEBtnql37HnYOe10OtFYa/kACpCN7E3ahJPbzPPF0qFks/7H6JH4od0dx1T86SwK+CK4xA0tpEG3oSQJBMxptR6A4L0m/TeJOZtntdy43IA8ZviOfzVYatilnwPlUk/tzrEXu3biy7pkhJISjI/XlgIf/8NjzwCm6oaPS9Kdd8APLflOQZ+OZCE/ARAfL+uuEI8d1o01/JdybW0f6892xK22b7IpUC4nHqd6Py5SJKwEs7Px6SkTC1KtXnK1ue38lbgW+Sez7X5fGuHUq3kpg03MfGtiVaPp6aCAR0hIW3gT1tyQdyHyzLqPrYNwNUVhkSf5r6Bt5N80FqFJlv4VSf9tr6wlS/6f0FpdmmzjbOxIUkSrv6uVm4EZWWwR6Q4MHMm/H32b8DxKIvaMCxCkH67k2oSF45ifMfxbLhpA+9Peb/mk8eeEeuy4AkOkX4DQwfiqnYluzSbk5kn6zw+uE8wD194mJ7X9XR4vLLF59n6cUutGheTUR0yIIROEztRWVSzoQegPL+cDzt/yK63axLDubniHiFJEBlp/VzK/hSO/3K8RnTL0XThdmBpBatUKE0/y8pRR3E88zhgVg0CwtLOdwCo6+hEskTBKYj91uGG2pEdhgKQrt6NttqWIjsmm5i/Y2olUtsyKlJz8Es9gbLCPNde5n8Z07tMp0eAgwGbjYHUf6DgdJNd3sfFx/S53J64HbBem+Z7CtJvRES1HNluj0Dnu5psXHWiasEi18tyc81Kv6LKIjY/vdnm97ktQOWsInxIOAZXd/RSBVqE5K9NKP1A5NBXX1A6+Yj5yk7zahe/LriqXWnn1Y6iyiKrpsgKD9GY3tWvmk3zyVchvhWRfgWnIPlvU+3SWSEKrlmFhaicVdx19C5GPzO6JUdYL8hKU40GOvt25IqoKxgQapF57tYephyFXs/Zv4hCjWHU36QFPdSkY7V6yWZ7pf/Q5pC4PZHkXWJjVVFhVn4EVhOwrTu/DhB2Ip4a4akWECC66iorRbdmdd/28/+c563Atzizoma7ndJJSVAvczquXg9vvSWCzFUaJc88q6Bn1Ro/sySTxPzEBr9HpRLW3v0VHhVdKFYkc+f6GwnJu4qFTsm8M6keNnz6cljRHvY38Ebf82kY8rXDi9SAAPAuFt1ze5P3Up5fzpFvjtSwhWmrqCypRFeuM2UeqXTiBlErAeEAogOiTYu4hcMXMjZMWDaFhjawsJOyGo4/CydesX9M4RlIWWllc6eQFAwKGwSANkjIvzMzzadYKv1MOQUgCug9n7fqmmlxJK+EjaMh7wiBnt6AufvK2dsZtUsb6JC3AYVSYbKLKtILUjfM9yLZFoNWqDXLLdLftYWwaQxsbt68MrVSbTUvKxSY5tVjx8yWut7e8OPRH3l03aOmjY8JU4+LBoeWhkINk/ZB/7dMD/m5Ckl6foUoRh/76RjvtXuPCzvahq+T0knJzZtuZsxzY0zzoFovNjatPtPPqweEThX3NVfHGBxn7yKyvNazKWW51eOainA6ZN2Ht2fb6QK0ibyjBGuFZENdHkK4ZzgqhYrgPsEE97XRCNCKMPvX2TyR+4SVSjb8nXB8X/flfJ6ontal9FMqzRbCssoiMxPuugu++ELkLCkUMHQoXHZZU7wL+9gUv4kDqQescsKuvlooUAH0UjnZOtFQFR0QbX1y1r+i+HQpFPiy96JKXUJEuCh8njsH1/W4jsSHEvnr2r9snhI2OIy+8/tCayfG6on0dDje/k6+CfDjy4NftvRwakf6Jtg+U+T6XSKIjipmUuevKE2ytkeS1UEZJdYEZ5cruzD25bFtWmWTcTyDnLM5VmRBTIxQnAYGgn9IKf8micazSZ0mXfTrDQkXaqWYnBhySuuXw+0Q+r4OAz8ThJ+h7mYrtVLNiHaigO6oxaes0HVU4Skr/WJiHDq8VaOyuJIzK86Qn5jf4GuMXDSSmd/NrGGNe3b1WTY/s5mMYxm4+LrUUEKD2dozPLxmzuS/r/3L0rlLyYvPs3pctuWtnv84IEQUTA+mHazX+KdGTWV29GzTfhqA0hTQldk/yRbOfQZ7FwhLUAcwupP47uS77SUr29qSdOfinfw2/Te7RGpbR36Hfhyc+jTtx5rlnbd2uZWl1y5lYNjAWs5szEEch61TIO7bJn2Z0e0FEVE91w8gRVlF+rWrTvo9CJ2ts6ibHee/pGvp44DYy3s4mZV+x346xvGfj7fk6BqMoN5B3LT+JqTu0WiVYm6RkEy151aLkkQ4/yWUJtt+Xl8JuYdEdEw1aFQaihYVcezuY3hqPK2UfiUaO6Tf5EMw6LPGGv3F49QbsH26SfzgqhB/r+yigpYcVYNx9PvDRB5dgbuqnDm95rDqhlXcM/Ae64N8eoNXtO0LyHDyocz54nJ164P/SL//YBf3nrqXBbtEvoes8nNxqdnRvT5OFEqu62G231KpMFmQpaWBtlTL7nd3c/JP0b2n0qgI7htMUO8gakNlpVD4/fuvuOZTT0Hfvubnvzz4JZHvR/LQPw81+H12CvfkiwlLUOpdSXPZwNnQFwjwV+Lt7O34RZTO0O0xiLypweMAhE1d4u91HhYQAD4lgvQ7l3uO0AGh/C/vfwx9ZOjFvX4rweq7V/OKyysUFIobglQpSL+LVfqBsDP5adZPvDzuZToLh1ganKMaPk0EsvatxZO526NwbSl4W3fADQ4Tf78ir5qkX14elFdl+lnZe7a/Fno908DBNhG0BaKLpyKXIE/xB6qQ8gEozS4lZV8KlcVtb/OTuD2RvDixqCw1CuIozKeK9Dv6DGwcKwi7+iBrJ/zdGRJ/MT+mcocRf8EQi81L0fmLGXqD0buqWfbYMWul36pzq3hnzzscTqtm1apQg6aVyM78Bgqb3SoEegjSr0gn/nbBfYIZ8vAQPMNb+ebABvLKxedQWdlGSL+oO2HUUtFJUVngUJ7P4IBxSAYVqRXnOJR2iJe2vcS/F/6tPVMtZTXEfCQ2TK0dJxcTGHMVSklL14R3ufBQEvcMvAej0UhFUQX6yvpl2TQnJIWEi4/ZvqhUW0pOWQ555XkoykQXmBzMXhssc/2MRvjwQxHkHhICd9whMvSefFIofZoTcgFdVtGAyASdVRXFW6I5jwEDXhovgtyqrVmPvwA7r24DcjAHMPxXuLaUzlFia3j2rOigbufVzq7rRdSUKGZ8MwO/qFZyH6gnzq05x7YXt1GaY60QS0uDUk0cZeQ2yEWkWRE4Cob+AD79WnokjYagbn24cWkmPx/+n/Xj7uL7l1FsTfpFTYli1FOjcPauxj60IfzzwD98OciaYJYzlnv0gB2JO6jUVxLuGV7TzqsB8HP1MxUL9yTvqff5Zdqy2slC/0HQ7mr4yw/23u7QNSd1msTo9qNNf2dHkLA1gY+6fETK/pQ6j5WVfklJItuvLSPnbA6/z/ydYz81vkvIqT9OseOVHfhH+3PHoTvoc1OfGsfIpF97RRJxG+Osnpv25TSu/v3qGveFIxlHAOgTZH29/iHCirO+Sr8HBj/AX9f+ZU0ibhgu/qsPOs6HUSvAybH7WK+gnqgMbuiUReyLt1ab9b2lLzO+ndFmogTqi5ISMCqUeHq34PtTaCBsml1LxMaCKWc0cStgjoGpVOaQphOWprJiulUhZRU+2Z+gUZaSlwduVYKCoooiFvy7gAU7F7TwAC8OxcWgVYl9sbezNwqplX/X0jfCvjsgx05++7Fn4J8BQiRgA5bvz5L0y1NWkX7+1Ygjj07CWaq1oON8GPI9VGVfuqvEhj63RNTQjv54lAOftZ2mtdRdF/BPOoKLq519X2kyVLQ+95NW/i1pe1AoLq1fqRwOLRMSgYHWtQ29Qc/GuI0ATOwk7HEqqwpwsgQ5PR0UagWbn9zMoS/Fgq7TxE4s+HcBPh3sszglJfDss7B7tyD8Fi6EARbqWZ1Bxw/HhHy5V2AvyDkAF/5y2CLTEteP7cntwV+I9+q1Gk8fB4Ot9RXiPxBkTKf59X5tKxxZBP9eDylVIe0liaJoWg3+/hCRM5/7K1L5avpXKJ2UOHs711BUtlVEDI+g99zeFBjEezeWNR7pF+UXxY29b0SpUDJiBLz6KtxySz0vkrbB3JETeb0gfe1B7Q42CmZyZ2KmWpB+GRY1jOwcA55lvQnVdLFW+rVGdLgJZmdB8Di6BEUy7MwOhp/didEIR384yleDvyLjWO22UwqFgl69erWa+VNXruO70d+x5dktlOvK0UmiIGhS+lXmQtE5UDlQ6baEZ7QggS2zJiQFtJstrAoBUtfC6h6Qd+Ti30gdeGbzM/T6tBfLzywHzKTfqVPC3g0E6ZdeLOx15S57QBCeWf9aqxZbEvpyKDcz58Ge4m9VrK8ibAeFMemdSQR0b+2BeAJFaUVsfmYziTsSTfaeigpvwAF7z5bM/TRY5L/kHYMlfnD2ozpPaxfkiU+JKNQM/HIgz259lue2PkdhFa9uk/SL/wEO3i++Q60dUXejGyTIfr3ebA9y8PODvOb5Gok7Gu5Y0NTzp2UDBEBSgeiGd3dyR18i/jB1Kf3AmvTbuBGOHBGZac8/D9OmNdBiuxEwK1qwe+tj11NUUWR6fOZMQf4Vu4iNeDf/bjXXWD0WwaAvmmuoTQulBiTJ1AR17lzLDqc5cGb5GbY+txWj3lqpmZ4OJRoRWNXRx05gVWuBRyexDnNQVd0W0L2nE4UVASQmSpRYbOnsKf0uBq1l/TngzgGMeWGM1RxzStSV6d4dNsRtAGBCxwmNttcbGiEaRWu4ODiATfGb8H/Tn4k/Tqz5ZNF5sQZRuYL/UPDs5tA1Hxn6CFtv2WqKYnAErv6ulOeXkx+fX+ex3t7iXmU0WjdatkV4Rngy47sZdJ12cUqBE7+f4MuBX1pZ407/Zjp3Hr4TtwA3FEoF2lItR74/YhUhIuf5aX/8jaM/HLW6pouvCz2usW501Rv0pgy+6ko/mfQ7nH7YoTzHWhE5t/556779IHy6w25LKoWKMMQe/t8L1va4kWMi6XtLX9SubdPlpjYYjaBPzcAtLwlXF3HPrNBXENoptHnrT55dYPRK8GtaW4hR7UfxyJBHeGP8GxiNRpPSL89d2GN28+9GgJvFnjL3MKwbDEnLmnRcdeKyD6m8MoMKvSuVleCsMCv9PMM82+xnM3FHIpue3ERxcr5J6VcvgUZLIWyGyLYNHGXn+Suh5zOgrrum5OcHajUYJC3ZejEJWyn9dGVC7eyAur7ZEDQaOt5sqpV6OInNXl6ZqPHu+2AfOxfvbLHh1RfdF03n0KSFuHg5UaGzwRfsmisa/OuAQqGgR4/ms0RuA5WS/9ASqCisIGZlDPkJ+YA16WeJ/an7yS/Px9vZmzJtGYO/GsxtK4WsXe5GSE0VuRm3bLuFa/64xiGf85wcQfKdPAnOrnpue+I8gwZZn/fFwS84m3MWPxc/ru5+NcR+CTuvEYXfgtNCfSTDoBNF4Vrw8V03MtPwM8PO7KR7V9veyjVw9kP4uwvk150/4BB6LILeL4lg18ztsLITnPu0xmH+/uCk86c822zunHUq65Kx97zszsuY9eMsCqoIT0OpN3Dx9p7VoVBAr15mGy+HYNCKvKrDjzl+TvY+yNxh9ZCs9Ms0nEGrzDepaQHy8xQMObuR34bH4OdaVeHXFsHa/nC6HrazzYwAbxd8i0fgVtKdigpoP6o9418f75C6qrKyFal1JJj8/mR63djL1MksGZWE+FS9j4Efw6zk+qs7XIKFBWWghR1IwSnQWoRrO/kIclBqegvX5KJkTmSeMG3E27cXhfeKCjhY5bLj7Q1pRWJekfOdAGFFsWEEJP7a5ON0CFsmw+ruph9DfYTST1ZptjUUJhWy4+UdJO9OtrL3dHKqg2DJPQirusKZ95pjmNaozINloeY5yisaIq4G7961n4dYWwQWTAXAYDTQI6AHt/e/nfx8cd+3SQj1eVXYyyragKVb0GhUneegVIvNdlEVtxTYM5A+8/pY5Tg1BE01fxp0Br4b/R0bF240PZZUKEi/CM8ISkrEHOgI6WepsvjqK/H/c+c2f4ZfdfQI6EGUbxSV+krWnFtjetzFBV58EXqOFt203fxtFK6DxkLkDc011KZFWTqkb6RrB3HPO3cODAYjCzcu5KZlN5FbZnsu3f3Obn6a9FObzDCa+NZE7jh0B64B1t+/pNQKk9tCqyf9LkF4e0PPDon0DNhiIr4Ak9K2Ul9JqdZMUiTtTuKbEd9wdnXDwtpaw/qz5/U9GfLgENPPOp3ZCrlHD0wNthM6Tmi015RVzrF5sfU+92CqWCTWUOXpy2FtX2E5CzBmFfRYeBGjrB2BPQN5OPlhelzrWPFMdkqw3HO1RbgFuNF3Xt86HZPqgrZES15cHtkx5gY+hVJBcB9zk59BZ2DFLSs4/LXZ7SMuDjAYiJzWi9CB4iaeeiCVX6f9ajM7W2/U89kVn/Ho0EeJ8rO214n2j0aj1FBYUUhcXlyNc20hPi+exPzEmvedPi9D9KMOXaPmIB2Xf3Z2HoKmMpi8QgebxC8BlJdDSMxWuv37rcn1a0PcBoLeCWL0d82QyWU0ir0niKb4nddB/I9N9nI+Lj68PeltpnWdhiRJprVqqSYelaRieEQ1Rak2H8pS6/U5ahK4tcPZ3d1U27q+w33kPJHDp1d+SlleGakHU9GWtiJSyEEk7Upi5+KdlGaX4qwN4ZaOT3HHgDtaelh1w9kfwmfYd0YKHAm9XwS3djaf3pqwlX6f92PGbzOQJFFf1ytKGR9wC5d3uJwwTwtHsKwdsDwcYr9pgjfSOJDtWPPLRVfv9K+nc8OatrOPKiuXMKg1uLhKRL4fifur7pzIPGE+IGI2dLnPoWs159rzP9KvkWEwtFB3fSMj52wOv834jRO/iQ+xvDgOqCaSkDNQxnccj4vahX0p+/jr1F8UlBeYbo7pQiRC2KAwnL2dWblgJX9e+2eNgGcZp0/Do49CQgJUhmzn5PABXLk+igfWPmBa3OWX5/PslmcBeGHMC3g5e0H3RcLixjUctk2HTWPNxfT1Q2BpsE3VnAyFApY8dwNffeZiUrzUCZWHmMQ9Ojl4Qh1wDhA5SAoV+A2BdteILsnsvVadQ/LfITvbHCXz6/RfWTF/ReOMo5WgoEL8vVR6LySp5ZQAVtCVQufb6/ZqtsTO2XDwIauHAtwC6ODdAYB8t/01Mv2gmrKxMl8ozLStzANbVwIXlkDOATQa8T0CKC2F0MtCGf7EcLza1f6HMxgMxMTEtJr5U6VRMfiBwURNiSK7ivRT63zx9GyCTsbNE2HjSPPP/kNgyiHw7lX3ufpy6yDzuO9Eh1Fp3TZHIDbaAKeyRFVNkgQJDuYMV09vPRcKRA5ehGeE+WT3DtDvbQhsJeHL4TOt7JXDfAXpV6HIFfvE2Fx+m/EbJ/9opAaNJkZQ7yDuPX0vfW/pa7L3VOt8CAiog2uuzBP3QK/utRzURChNEZsWp6qJS6GGEb9BxKw6Tw0IgA4Z99Mn71n+uuYvjt19jOt6XkdRkXiznrb6Bjw62e+cbKXw8IAc921M+WsYtyy/hXYj2jHzu5kXlevXlPOnQW9gykdT6DPPbMclK/3aebWjuGqJ5Qjp5+VldoAoLRW22tOnN/aI6w9JkpgdPRuAJaeXWD3XqRMYfEXlvUZuxqWG1NWweQKR7ntRKqGwELKzJb45/A0/HfvJ9HevjtzzuaQeTKUsp545Sq0AGk8NIf1CrFQKRiPE5yWCZMRV5UagW2AtV2gF2LMAlre3mQfTlrGg32O8PO5yzpwyF9XdndzJfSKX8qfKrWxXDVoDuedyKc+rf7G1ta0/ZcTGiogLDw/wCyqjospV5vKOlzfaa1zd/Wp2zN9hN7OzNhxKF8V3OY/NBH05dLlfFL4aiJzSnJp28rWgPtnhlvvn/wC9buzFY5mP0W64KDjHbYojZX+KFZnm7O3MnL/nMO6VcYBwYkpPBxQKZnw6mcH3iybWoz8c5eyqsyTvqZld5aR04qY+N/HWxLdQVWvUUivV9AkWawxHLT5f3v4yke9H8tL2l+r9nm1idU/YNM7hw+eEPM/4Y6kMwrq4m3owlfc7vs/BL+qXT9gWUFICWe36k9JrEhpncc9MzBcuFVaKt6ZC+kZhg3j6bVF/S/oL8hvf3tYWjEYjzs4id3qMywNkP1bAq5dXi3YJGgszkyByTrOMqVaUXODK6K+QMKAt8cDXxRcnpRMHvzjIl5d9SeaJtid1Hnj3QO6LuY9C5wBcKzvwaL+XWTii6RpKGgXpG6HkwkVdQqPUcCT9iKnRplcvcDJ48dHEL9l480Zre1OXUHH/bU127ymrxRo1ZTUA3i6iJlhYVeMN6h1EYI9Wvs62QObhFJyLsnB2MZBdmk2JtsRacdr1fkHi1gGDwcC5ZrRUaQPt0f+hJeDV3osZ380gpJ+o0NhT+smk38SOExkcNpjuAd05lXWK3078Ru/gOwGRjQHihpkXm0fy3mRhGVHN79xohL//hm++gQzXLWT1+JxYl9+hylXqo/0f0cGnA48MfYRXtr9CTlkO0f7R3HmZeB3cI8V/IJQ45ZlmqwbvXsLuoQ71jEJRz67zqDuh8x1Nk+eidBIZKyDs/ipyRKeIpMDPT7zkscD/Mf3nc7w5+VVGLBqBUt306qDmwMaFG0GCwNGBRHpEodGG4OkJytbw9py8oN8b9Tunz2ugrlm1Hhw+mPj8ePLd9pKRYe7ezc0zAhJVvIWAWwTMSGjQkJsU2iKRaRR1L5LfZaSHfEO+IZnzmQsY5NvKrUkdQEqeIP2cdH4it6o8ExL/EHYFjhBz1XHsOcjYBON3CPujrvfX/GzIC7ic/cJGM9hGkcdoFMpPQyVMOycmhJwDIhN0YE11sC10DxDE0OlsM3HYuzfstHBZKFNfoEJfgUapoZ2XRReaW3uIfsSh12kWdHvI6scIf19UOk9UOm9KSozoynScW3OOsCFhts9vZVA5q/DvJlrSp3SeQlayJ0fP9sMvso4Tg8eLTWdLwLsnTD5g7kSpBwIDQWl0oV3cC8zoAoqqW6qJfLZF+mkLRSZmW7D3TFoGB+5lUMQ3nI6Dw9m7MapaP0mi0qgYdO8gq8csmwDqQ/qBsPhMSxOW7Q8+2Eru6cBV0Vfx2r+vsebcGsq0ZbiozfL/M9lme08r6CuEsrXDPBjQehX4DiNgBAz8FLV/DyIjBelw9qxQeGeVZpFWnEYfamY7TXpnElM/ntrm7OUNOgPZZ7Lx6ehjZXeVmwsFCqE26ejbsfW/L+cgsfdRtJIvUyOhJGgBn64fT4rGTMZJkoSPS02f//aj2vNYRj3cN1oZzv9znvWPrWfSu5PoNEE0kVpae7o6uXD63tOkF6c3KgmtUqgY0W5E3QfagFyArEH6OXlD38XmnzN3CCvu7gsdapD94+QfXP/X9agUKlbdsMoUHVIX9ry3hyPfHuG2fbeh0tgvb8mkX1tX+m343wZO/XGK2/bdhluAW4OvU/13tf7R9RSlFvFY+mNgMfV1udKcIxlf5fIZEGCd53v54svpf3t/gnrVX3340ZSP8NR40tnXsSyqY5mC7OkRYKHwzNgqCKGeT4P/4PoNIGCksLh2EMEBTkgIZypLaDw1OHs5o3S6tOZjEFlqhYFRSF7msteFQhtNoU0Fj87Q4WaRFarxhavzHbJEvFiczz3PTctu4tMrPuXJJ/tiNIIkuQKtOO/37Ifc3PUtDsX3Jy/PHCkSOSaSca+Mwz3YwUV7K4LGU4PGU0NxVW+Po/uOFoO+XDRiK11gemzte9VjzwnnpCtO1XCv6RHYA7VCTUpRCicyT3DXXT2ZO9dOlrp3T7jsg8Z9HxcLlZsQtVTV4B/u/ywFax8j1FNwDAa9gYrCCjSeGlOsWGvGucV/0aHcBWn61eiKRKRJgGvrj45p/b/Z/9AiqG4bYY/0W3rdUn6+6meT9P3WfrcC8PXhr60y/QAKEgv4MOpDIsdGctOGm6yuYzDAm2/Cl1+KvJuTXW8g1uV3JCTuGnAXz49+HoCVMSs5m3OW9/e+D8DbE98WHWOlydYWeSETocNc889DvoVuDzvs114nKnJFwRGahvCrjn5vw0hzB7pKJVRgWV7rWBW7jAsFF+h/a3/63FyzINMWcXrpac6vOc+7k99lyZizhOXOaZQ8v4uG0diwrKwON0L4tBoPPz7scZZN30HHjEfIyhKXLyuD017vsb5PEO8df7IRBt3E0PjBsF8F+Q3E+L/J2bDnOJ1xntzYXL4a/FWbCugFYRX1ae9POb3sNP7KDnRLfp2ovPtRqxHKuoP3i81lQ1CeLhRRuhJRpOv+P4i6u+ZxujLYdiXsuQWqclKJ+w7ifxb/L0kQdRdE3WP2bh/4kchXdHATJCv9YrJj0FepBKqrnLOMwi6rs29nlG2oqNjON5iZZwoYd+I8RUUSAT0CeEb7DCMXjaz75FaAypJKitOL0Wv1DG83nAmuTxBQOKHuPD8QG434nyFlVZOP0yYs74kVubDpcjhS+1zm5SXy3YxG6yKKTPrZVHkvbwdbJl38eJsDai9wi8TJ2Qm13hsQjgXaMi0rFqxg38f7WnZ89YBs71lfpR/A6NFi/XLLLcJOuLXgstDL6BXYi5ndZpocBmQ8OvRRFg5faModMkFfJho/nNtOh2qt8Owq7ilu7a1y/UI9RCecbPNcHSpnVesnxmwgNzaXT3t9yrYXt1k9npYGpVV5fp19G8nFoynRd7GwOb7EENRvCv/E3snpsy60AvfNJoW2TIu+Qm+lWLMk/WRY5So3MrJLs9mVtMuhYzOKM0gpSkFCol+IhaLA1h6pOBZiv4KCEziCq7tfzXU9r0Nr0HLV71exL8Wxe2NFYQUVhRUUJhXWepxs79nWlX7O3s64+Lqg8XScqLKHggsFbHtxGxnHMpjy4RSmfDgFSVFzTi/LLaO8oNyU5xd5YAl/XW9WiTq5Odkl/FacWcG2hG2UVJbYfH5g2EC6+nd1aJ+hN+hNdmq9gyw2LSUJkL4O9KW2T6wNgz6FAe85fLjJJjbbSKXePEH5Rflx5+E76XtL3/qPoZVDzld1s+CYLZ0fmgxyI6F7Bxj6vWg6hWYh/ACe3fIse5L3MPuP2eSX59sv+2VshYRfzHv2lkSHW1iW/RcpRV04nhbD3avu5qlNTxE+OJyRT46s04GpNaI4o5jClCKKi4yUq1PJ0J2lsKL2+b5FodDAkO+h31t1N6caDYIcq6jZjeKp8eTKLlcC8O3hb5EkKCaNMm3rbxwFIGiMaAgOnQxAt+D2eJRHU1ngDcCGxzfwhu8bFKUU2b9GK4L3tJFkdBiCXiP+Vl4aLzSqqvtw0XnYNF44oLUy/Ef6/QeHYI/0C3QL5IZeN5g2InN7z0WlULE/dT8P7Z5NfOAHFBWJ7iCv9l4M/99wuk7vWqMD6qmlX7B2/xlUKrjjDhjSsRfTukzj0J2H+PTKT3l29LN8N+M71s1dR0phCgFuAUzqNIkpUVPEBQ49Bkv8zURcUyPhJ5G3VxjTPK8XOhkChlvdNPz9hd0bYDdrpa3i3lP3smDXAgDy88VjjZ3n1yBk7xGF5uQG2qhWU8D0D+nPlb1GoDKKsOWCAtFlXqZOolKdiVGy8FzPPQRxP4giemuCQg2R14OP2Hi5SN4AZBfno1QrKUotorK47gWwsrVIPgBduQ5duQ6M4CNF0jn9CfpU3iOe9OkDl28WqtuGYOBnMCO+7gYElQsM+Q5GrxKqX4ATL8HRJ81Fla4PCLWd/DyILuvcg7BtRp3EZKR3JBqlhgp9BQn5CYBQOlsqTDMqzwPQxa+L9ckH7od/Boq81NaAlNWwYzYUmq0SZHVYYSFtrih94rcTvB3yNvGbREt1YdWtrdZ50GiEg49A0lLYfyecer3Jx2lCxlbR0ZhfrbDn5APF8XXaEkuSuQNfXm8YDJhIpRqkn9EI7a6FkDZC+gWPg4m7yFKMsyL9VBoVR78/SuLWxIu6fFPNn6eXnebTXp+SuMM8vl6BvZjceTI9/HtTXs+O2wEDYMkSmNHA6bOpIEkSR+86yk9X/VSjsH59z+tZPH4x7b2rsZRO3jB+a5PmVbUUulRN9+fOQYi76OBLLUq1e3zcxjhO/tk2rJNlOLk7MfLpkXSaaE3spaeDRhtMJ+MkhoQNsXP2f2hqBAeLtYhOJxSnMj4/8Dmzfp/FstPmyAOjwciJ308QvyW+Qa/V0uvP6FnR3H/uftqNEIVzo9FM+kVHW5MKTYG9yXuJfC+Sq/+4mnJd3RapB9OEyq+rf1fcnSwm/5IEkSeU8Iv5sYjZcFUmhDnm5ayQFHw/83smdJxAibaEqT9PNamta8PIp0byYPyD+Hb2rfW4S0XpN3LRSO44eEetqkZHUZhcyNbntnJ66Wnaj2xPz+t61jgmblMcb/i9wfGfj5tIP2dDCaVZjhFs96y5hzHfjzFliF8MYvNiKdeV46Jysc5c7XgLXFcOAU1v++7nB3GB7/GpawiLdyyu+4RLAMWFBnpu/gC/QxtMj8lKv3aeTUT66cvh3zkQ82HN5yrzRYOjnPPXRPho6kdEekcSlxdHx/c7ctkXl/Hdke9qHnjuM9h1Y5OOxWF49yDLeTblOndS8rL47OBn/HHqj5Ye1UVh7X1reTf8HfR6iA16iyE/d+WV7a9c/IW1xbDzejgmYqPQlcH5L8S/FwNJgtBJ0M4Bq+s+L8GUw+ASYvPp+X3nA/DT8Z/Q6rVcv+R63F51Y+nppdYHnn4Ldlx98WNvQsgKxYoKYWEeMSyCfrf1Q6FuG7SUalB/csN7o1WLYoWVtXF5BuTss0ne2kJzrj3bxm+3DaGlNw6Nhe0vb+e9yPfIT8xHpzN33lfP9KuOQLdAru1xLQArzi4lz28tIDpnJUli/Gvj6TzJ2rohLi+O10/ezdYe3ek9KpFp02D9TetZOWclfYP7AuLceX3noVFpGNthLGfvO8vX0782XyRkglAa2bBQBIQl3/J2oiDaGHAOAp++9l+vKWA0QL65qOLvD2q92NzkluVy9IejfNz9YzJPtj2f7upQqBQ4uQkiI6/K3rXZlX75x2t2rFb+H3tnHd/ktf/xd7Sppy60paUCxd2H2/AJY0yYu+/O7c7d/c63u92NDQYbDhs23N21LXVJXWK/P07TNm2SJm3aJPx4v159FZLneXIgyXnO+crnUwgyFagcrLbN3y5kwE7+p8lTcnl9kuWHHyA9HaqUwpPNzJw3/XfYeoP4LLsxvlLxRuWVFREYF8hD6Q8x9JGhNs+RyWT06NHDbebPhNEJ3Hf8PlIvT61LttTJCyrVQrffiuFyszRMPv1zJWyw4XcWfalIMpoY+hNM3GZHxZgRMpdB8WGbh8mkMjqHCp8qk8SnRAK9al9SpYL7h9xN1r+yeHdiI/k6g05Ii0pbH3BwCuVnhYRiZX1gumHSz2g0cmrVKTJ3Wg9cuxOhXUIZcM8A1AlqdpzfwRHNLnTSMtvJlapsOPYe5K6HSxaadYe3OQU74OxPYGyUBJZIYPpJIbndDI2DcaWl9XUSTWRMJBIY9AWkepakm78/yHVqAIqrijFKjDyueZwrf72yxddsy/lTX61HX6M3kw5/YPADLL92OWMbFD74OqAuJnXTnYenFQY4nRoNLEmFvU/WdfqdPAmRtUm/rDLLnX4Aqx5ZxaqHV7XDIJ1HQIcAxrw0hoQxCWaPZ2VBlOYK/hWxgseHP+6i0TnAgRcg7TdXj8LpSKrzeXt0D67v+RSHGuST92TvYdHRRezL2dfgYPj92t/Z9sE2h1/H3dafAOfPi3WLUgmG4OMEvRHEZfMuM/NZcya9I3sT7B1MVlkWX+/+utnjrUp76ipAGSJk/00o/GulveyfX5UyJQuuWkD/6P4UVBZwzYJrmj3HXlmwug4tD0/6OZMOgzpww7ob6Hl9T6ufsYgeEfS8vifBycGcFurHjPpmLtf/db3F4xuSVZpFZmkmUomUHhHWbRE+3PYh1yy4xqp/rAlT4rB7ePemnYESacukjnUVsONuOPS6XYeHhoLUqKBKnsPm9C1mz+39bq9HqTfYS0mBFr3cC4VX/XfN9F7FB8U774WOfQTHaiUKdRVQtAfy/mlqHVCZBVuuE3uPNiTYO5j5s+bjJfOiqKqIXVm70FRpmh7Y/WkYsci8ENeFBAVBsPd5jOVin15aXYquSse3I74VNjoeRvKUZHrdNRgkEvQKERy0JPftMJVZUJ0LRbVrijPfwfY74Mz3zZ9r0FtWAMvb0mwMxhEuTb6UCN8IcstzWXZiGcfyj2HE2LTDtmgfnF/skFRxm6OvEsnIjMUAnC47yPEOz3Mu9EtKS6HrlV2Z/uV0/KPap3O3tVTU1rlUy8UiwkxyPWwYXFVSp35mC5lMRrdu3Zo9zlm46dbbc2mrBXl7o/RX4hPig5e/F4WF4j5rkpQE8e+8ZsE1vL7x9SY3vm+mf8OKa1fw9vi36ScTlQkmiU9LvLvlXYwYCCuZyMAU+/SefJW+5gmRxFtsaxgrg0WizlkyTB1nw5jVVisy2oSd98Ky7kIaEBEgVejqk34gql21FVqrl/AEjEYjaRvTKDpdRK/Pe/Hgsd5UKtLbN+lXlQurhsCmq80f7zBF+KeFDLR8njW8I8C7g2jdb8SKkyvI7ncvef5/sXo1vPYaVCnEexwT0MATL/FmsZj0te870q78PU6YoAO+MjUABeUau083Go2UlJS45fx5JPc4Gp9dSP1qKx/01S2TeDWhLYUzP4oFob5SbGjsJXQweNuRcA7qA7OKIeXuZg/tE9mH7uHdMTT4N5kkPoOCRCA80i+SeHW8+YkDP4PJ+3Abkm6HOTrhtVjLxsB7+Cd1IJvS/wHgp0t/4p9X/nHVCB0iblgckz+eTGjnUK5feD1vl/ZH47vTdtJPFQkzM6Db06IQpj1lB7s+CpfnQaCFgI6dnnuNO/1M0p7+/u7j/dZidJVw6DW6Biyq6/QzYqS0uhQvf69WJZzacv7sfnV37j12LzGDm/qzmrowfX3dN5HnKEajkf05+zlRIDqGD+YeZM2ZNeSVW4gQF+0VCZeS402f80TkvoAEpEpiY0XCobISvHXNJ/0u/ehSZn4/s33G2caYfMgj205J0XkY9HDgeeEzfKGhDEKl0qE3KMySfqZO3Oyy+o2lRCLhqvlXMeIZxzt8XL3+1JzVsOqRVeTsz6l7zNTll5IC6879RYW2guKq4jYrTPCSe/HEcNGx/OrGVymtti2zNSR2CHf2u7NOcqwOdTeYckD43Tek9KQoDHIAfy9/Fs4W3Zx7svdYlYVsyLkN5/j76b9tvpemdUZ+fovsh92GTW9u4sD/DjjlWlKZlNghsfynz3+YN3OexWN8w3257IfLiB2ZSHptTq5TJ/uKZXZmCouH1NBU887QRny791t+PvgzOzJtf1ZMST8zaU+AzJWQt6nZ8VhE5i1UfHLW2HV4YCCEVAnfwJ2ZO80+c9s/3s7mN+2TyvUkqoxeHBlxB95Thc+8Vq+tUwAwi1e0luMf1RdJewXDuA0w7JemhQMBnWHo/6Dzg/ZdtyqvLvHgKP2i+/Hx5I/r/j4sdljTg9Q9Wq4C1AZ09v6N72fGkGwQxTBlNWXIvGSUpJdQpWm+o9vd6H1jb3o/JtRdDF61ST+VE4KDAclCwWlkrYpXwg3Q/2NIaKagIXMl/BEHOWvNHzcaYcedsHKQufVUc5z9BQ6/ZfEpuVTO9T3FeN7f9j455WK90ESFaeh/4aoKN/O6l8CeR4VnIXCs4CjHo17gfMh/KfUMRc86Ss6XUPzqp4Sf2UaltLbTz5Kfnx3//6a1Z3vhJiX6Fw4GQyuCwW7E4AcGM/gBIWtzqlatK6xBod7JwpP8fPBn5h+ez70D7zU710vuxcSkiUxMmoh0G6wBjqXnoTp/lgEdBpgdm1+Rzzd7vgEgMftRkuzzbzZHOOraPkYqh0mObTjcjg4zzJJGYWGgbJD06zW31wXh6aev0fPtJd/S/druHEg+gBEjXYyK9k36SZWQci+EDoGMP4V0RM/nxXMt2XT7drT6+Vt8bDErCz/luslK/DaNIyMDqpQZAHTwb5DY9uskftyRwNS6BEOAIghqoKhSLMgOzz+MVC6ly8wuVk83GAycPn3abaqt07ekk3swl26zuvHtyVfZ2PV7/OSvAU/AnkfgxGciuWJPAq4x2lLYcr3w4hu1tG2iDlIZSO0zGP9u5ndNHhs2DLZsgQEDmh7vtkgVTR7SyI9Q7LuDNE0GEomEaV9OQx2vbv+xtRJTYY1SF2Q76SeRgE+DOaNGI2Qmoia05fDqUYVaf+7UN6Lqse/bVg8xyYc3TvoFWGqoLzkGB18Wck6RY1s03HZn31MkKa9FZpyJHC90VKOp0qBL11GSXtKk48he2nP+1Bl0VOmq8FP6mSX9LhSe+vspXt/0OlekXsErY17hsx2f8enOT3ls6GO8Mb6RXG7+NpFwCR0CASkWr+dRSBUwVWQbZEBiIhw5AoZi4elnS96z4yVuWIzUDEvuWkJ5djmzF842ezwr24hWWkpUVDsqebQUiQSmHnOfjntnIpWRP+gI//tJqA7o9aL4I8JXeIaZgl4mbK0xbeHq9efZdWfZ8s4WYgbHENFT/NtMSc6uXWHBaSGlN77T+DYdx819buadLe9wuug0T695mg8vtV5IO67TOMZ1Gmf/xTfMFNX+0086NKaYgBiCvYMprCzkROGJOvUfaxxZeIRt72+j53U9CUu1LE0UEiK+Nlqt6Ka06BfsAax/cT1xw+PocY31zjlH0FZo6Xd7P0I62zaOPndOSK93KDzAud9rCL65D1K57QCnKenXOAbUmH5R/dibvZfdWbu5PPVyq8dZTfrtvEfESVpSkCiRwKV7wcvGOrbR4R19uoFRSmFVAdll2UT5iwKZmd/NbGJjcyFg6nAxrfmqdFXcM+AeDmccJtTbvv83q2hLxRpEpoJx/0BNUf1z3pZ9IpFIIX6O/a+x/1mQyIWKTgvumbf0uYXCykLyK/LpH92/6QEGfcu6TNuK8OEs3nEfGn0IKKBcW44RIw+cecDVI2sxpiSRqdNPrVK37oKWYshyH0i5x/o5xYdhz2OQcp+IF1qylhrwOZSeaN7KpSGnvxZFC10esvj5vLH3jaSVpNEzvCfrzq4j0i+SAC8L61R3+gyC6DqcsAW8xV4i0EvcdLWyYkpLIXtfNpvf2kzfW/sSPyrehQNtHl2lDoPegMRoINqvA1NTpjI4poENQO5GYWcSNbHZOcZgMHD27Nm2HXADLsBdwkWcjUkCo6G058pTKwEYFjfMZtVWVBTk+6/hzhOTSCjoyPF7j5tVhX2641MqdZUElvclrHw0CS2JeW28Eox6IWfWHvJM+mrYdquo5olruSSXw0RPFD+1NJb3vJCY+P5EVB1VGPeJhIhCr25fTz+lGnq/LhYDf48SUp8SmdBq7vWS8KlyEoNiBvHpzk85XbON9R/B7wsNLDskgmtm3awGrcXEhlvQv15rP0CphhoortYAsPqx1ajUqhYHZFzBkd+PsOXtLSSOT0RT66EYpKrdCKt7Qodponu4JXhHwvD54jrQdnNWZZbwuYscK8zPHcDbG559Fiq0Fcz4ZQ6dQzrzyphXUMhqP3+6CjjxKYQOg7AhbTD4FqArF5Xk3h1E1R4QoAgGPeSVi/ewz819XDlCh9j1xS5OrTrF1C+mUlQlNjdyvdp20k9zUMxNpsTf1puEzMflOeBlO5DTKspOi//7yPGiKtcS5/+E7DXQ4zmrsthxtSol+/aJqddUAGcxKFd2VnjrRoxq7ejbB7k3TNrJme1i0xNADD4B1VTqKtnxzA4O/XqIZ2qeMZPRdAdOrTqF5qyG3jf2RqaUcSj3EL3/05vk4GT+N0R0uNnr5+cJjIofxeubXmfBkQUsOFIvj9sl1ML9K34OhA0F3/j2G2A7kpIikn7qggmkPZhGhJ+VwFstRqMRjCCReoZMamlGKSUZTYM1Z3NzWdk3kp3rw8gemIncnRNqEumFkXC2QseOIsBcXg5nzkBSkuVOPxNGo9HjZHp7ze1FdP9oAuPqb3R1fn5dDaxdLboIxie2bdJPJVfxn6n/Yfx/x/Px9o+Z030OQ2IdXN9lrhDrgU43iXueidRHxD69BXw6+VMCVYEkBiU2e+yg+wbR95a+hHa2noAwqRYVFor4hqcm/W7febtT51qVWsWEt20XiJ1YdoJFj2zEO2wykWe2suYZDX1v69vstU2de/2jLCRKGtA3qi9f7/ma3Vm2Pdpu6XMLXUK7MLLjSPMn+r3fOiUWVTMeNo2IDPHGryqFMu+j7M/ZX5f0C+/ejiob7YjmeC4Rp0/hVdEFCMLfy5/3J77PgQMHkDbubMlcIbp7xq6x7/9131OQuRzGrRd7GGuJPkvUaABj87GZstNQkd7iIhmJRMJjwx6z/KRBB/O8If4aGGKHLGQ7EBkfxdO7PwRFBfQSXWLlNeX4e3mGjGJjFs5dSLHeF5iAVu4kec+DL4oCviHfNVXHqdGI7rSkO81jNeeXQtZy6P4MTD/dNI4jkYjYiKPxkQGf155vuYiiW3g35l05j//u+y8AnUM6Nz0o40/RaBDkZk0gofWJMVOiUicrobQU/KoqOfDTAWKHxbp90i84KZiyG+8l5xjcHgdPDZxmfsCRt4S9ztXVrhmgDdyp9/MibsTW97ey7wdRKWWqug9vMBcuO7EMgEuTLrV5ndhYUJcPRGLw4mThSf5Jq5dWq9RW8vF20SrfKftROsZJUBasEprq9i7ajEbQ14hEXHMbvaxVsPtfjrVaW6L8rAg2trFxsFWMRjAazOQ9i6uL0VZqRaB49SnXjMtJyL3kDH5gMMGjxb9NalQiM6qadvoZ9LBxNhx4ybndUnmbxWcKxGdqyA8w+SAU7RI3f3kLI5yZy2HHPaLatQGDOgh5kN1ZuzFKtIyanI9BokWChCi/BvKxf3YSMppuTmBt1VVxbZXe9K+mM+WzKS4ckeP0v7M/1628Dr9IP4prhKxnqG9t0iTpNhixsOWa/RKpMHXO3yJ8Cwy65s9pCYW7YfttYvFhBwajoYkk0snCk/x57E++3vN1fcIPxKZpz6MikeMuVGTA36Ph9Ld1D5kStYUVnlcUkb0vmyMLjlClq6Kmdj5SNNfpt+UGWNlAejjlXrGJkLaxtn/6QiGFXGTjntjvA7jsvE0f3H79RMI5N1d0Opg6/SwG5aInCgmT+GtbN/b2JLgfykAxp99UepL0h9LpEtqF3jf1ZsrnUzAa3E9rbO+3e1lyxxKoXV6llwhdL38v/7pOvwsp6Tc+cTyPD3ucAdED8FGIbmmZRMbQWAu+tIoAIeek8MwAikXSf6/z0jH5+qWf8iM2MBaljXveri938bLXy6RtTGuPUTqFOYvncPtuc9+N0lLI0wnDKh+lt3sn/EDIBldmi6KwCxBp4TZuHfQ8/soCMmsbTU3J55wy806/+VfP582QN9t7iK1GIpUQ3j0crwBxny4sFJYYEgl4R52lpLoEL5lXs11uzmBcp3Hc2PtGjBi5dfGtVOuaBq52Z+1mc/pmdJbWrme+F91WjffjnW4UNhwtYHb32UxKmmRXoDqoUxDh3cObTYaZfP3y81s0JLcgtEsoISltWMxlgZpKHWWnclBWFtP3rau5+o+rm02yG43Guk4/i91RDegbJRKIu7J22ZRondZ5Gq+Pe51ekY0C2x2mQsx0O/4lNshcCWd/tuvQ0FDwrxSdlgdy66VWDXoDZTll6KraaH/nIkoOpRN7eBXyEjv2VOm/Q/FByLdT5tQ3AfyT6rqB7CZvC8wPglPNe5EyZhVM3C6SORtmQuEux17LFrpy0QgQ1Nt512wlYWFCpt2o9a5LypbVlHF+x3m2fbTN4z6faRvTKDok9NdrZBrACfKeVTlQckR40Tbm0GsiJp21wvzxro8KhYXQwU3vdRUZIonYEvwTxU8z0pDHC0TBZZOkn0EHG2bAoVda9vptib4GqkU8LVAlNvU6WTElJdBxREeernyaAXd5hryUqePZx5KgVtcnRNzYreRVBe43oou4BRtf28juL0UAr3HSr0JbwdqzovJwcvJkm9cZNAg6xfgRVSC80b7eU39T/nzn5+RV5BEq70hU0ZUkJgLZfwnpvOJabZMzP8H5JdZfQCKBUYthlB2B7dwNcPRdKD/X/LG2COgs/LJSH2nddVpC9l+wKBYylxEaCjEF13HpnlJ+vfx3MMKSO5aw73s38tlqBcVVIuKr0IubQ5NOv5oCSPsVcv6yr1sq4w9YP912cKTiPPw9BjY1kHvy7Qg+0aKTdPKBlnfb5W0U3VGl5vI2ySHJqFVqqnRVPLv2WUqrSxmbMJahsUPNEy0RY8wqZdyKnLUiCVSZw9iI2Qw9upGxvAxAwpgEYgY1r/WvUqnaepR2E5wYTOKEROQqOSU6sUgJ92thZ581jrwJB18SHaRtQdhwuGQBxF1l8zCD0cDALwfi/5p/E8+mY/nHAAua8T5xMH4zJN7q1CG3Cu8O0O9DMz+FEG/xnhXVLjTnXTaPr4fasTF0A6Z8MoXn9M9RqaoEQGKUITf4206wpNwtFpwmIsdC0q2OyYu0hLirYPC3EGJjfvLt2GxyxMsLhg8Xf16zpr7Tz6K8J4hOApn7zBvNoisnxEd8xxrK+CdNSqL/Hf2Re7U8wdBW8+fIf4/k+tXX13UgphWLpE5sQGxd0s//Asp5SSVSXh/3Ottv207pk6WcuO8EJ+8/SedQCxW1VblQXejZxlCNOfkF7H0SqE/6nT4NumZiQ+p4NUmTklD4uqkagRUaB6yzs6HcSxTOJQa7qZx6Q3LXwcIoOPODq0fSNuRtZFzEC8QGHKlL0DTs9GuYGAhNDSVueFyLiidctf6syK/g3IZzZsFXU5dffDycLBGJhNSw1HZLQL89/m2i/KKYnDTZzOvZxBub3mDYN8N49Z9Xm57c6zXR2ePC+7KuSkfGtgybx5iUi0xKRp6GrkpHWU4ZBl37WsoUhnZm9/jHISWFSbP8iR0S2+w5acVp5FXkIZfKmybpGtEzoidSiZTc8lzOl5531rAdY8/DsOdfdt3XQ0IgoFKotpgkRwF2fLqDdyLfIWOr7c+hp6FN7MLRoTcR0lMk5goqCiioKEDpZaEgaOB/4Bqj/R53qQ/DqOWOq9+oe0DHq8EvEY6807y/rcJfyC5mrhAFi85CGQjDfhbSjG6CRAJPjLiOG3o9hbdM7ANLa0o59OshVty/gtJMzzJUe+D0A3R8bi4A1Tip02/ApzD1qGVJzM73w+DvIHIcpP0GK/pDea2hqX+tH5XRCLsfgS03ir8ffQ9WDRbJaEcxGoRKU0NpWwtszhCJdFPyrMEFYOhPkHyX46/d1qwZB0tSAfNOv5ISI1K5FLnKzQvsask9mItsz04UlSXIvWqaHhA2xCHJ4fZce15M+jkZd/CjcgY3/XMTU78QBt2Nk35rz6ylSldFXGAc3cK62byOXA733gtx+aLC79eDv9Ulc/pH98db7s0A3SNIkQs/v84PwIw0cRPXVcDeR2Hn/aKTrzFGI5TXVhXbs0hIuVe0YQdYCN44iiLAuoxZW+LdQci0aUtQq0GONzK9H+XlEhQ+Cq5bdR2jnh/V/uNyItn7svm89+ccmycSDnKduKk16fZQhcPVNTB2nfh7wwW60Qj7noW9DQLgmgMiaarZD5pDcNBCJYxSDT2eF35rjZFIRPKvpaTcJ3zgAs2/M1KJlAcHPQiIzfScBXP4ctqXbLx5o/n5Q76HXi+3/PXbkoLtcORtqEgjPiiO4LJhKCvrPX6MBqPNqk2ZTEaXLl3cZv5suJEuN4iEUURAbRXYjrvrOiFazJ5HRWHD4G/bTt5TGQixlzcrqyKVSNFUaajQVnAk74jZc6ZqsiZJP7m3WNj4Ny+31G4o/KDzfRA6qO6h0NpEbXGNqEpVqVV4B3lbPN0dkUgldX5+cp0aCRLbSb/EW8T/QWNqikRHSFvhGysq+ZtLLtYUwdEPbFbXjhkjfm/cWL/2sNjpV3pKSIpaWhu4K+smk3hcBIg0Guddti3nz9AuoXQaV5/8SC8WG96GSb8LqdOvIVKJlKTgJOLV8ZYP2HqzSLhcSPR5G8ZvBKORqCjReavVwuPLXuS636/jTNEZi6cljk9kzp9ziO7XijVSO1J4spDdX+2mOL3Y7PGsLKjwEp1+ndQekPTzjobke+rlwi80Os7hT+0+ThT2p0Asxeo8/fRGPeXa8rpDR/17FHP+nOOw5KEr158nlp3gu5HfcXjB4brHjtQuw7p2rU8k9Ah3jm+bPYT4hHD8vuO8NeEtvBXm6yW9Qc9fp/8CsOzp5xcPEaObPl64G5Z2h9PfOTyeosoi/nfgf3y+83O7jl9611K+Hvw1ZdnWVX1MnX6emvRL35zOO5HvsOsLJ3YqNYPRCIv+lIJEwvgBGmoK7VNNivKPYsstW/h+5veo5LYDnD4KH/pEChn+FSdXWDxmf85+/j79NwUVBeZPnF8CC0Jbn8jp/zGMsvzajQkNFWpWCZIRdA/vXvd4dL9o+t/VH59Q+7zVPYUKiS9lwXEERoh54dV/XiXi3Qi+y/iu6fxp795WX12veNOS/bDCTyTbIkbB4dfg+EdCCSpzRb1qEwgP8ExhTUTIAOH72PMlx1/Pw+gSvI6k4J28k3iAvEfzSAxKpO8tfZm7Zi5+kZ63eC8vl2DEyFjfB7lnwD2E+zpBSldmRQ3HpwN0ukEU++urRdNI2WnzYyQS0SlYuFN83jrdBN2eaVmRfv5WWBgNp76xeViviF4opArm9ppr/oRUIeRlLd2DXU3MDDE26j39jBI9BSUVGPQGMrZlkHfY/W/IZ9acIXTrUrwqipi5vC++r/ryz7laFUOj0aEiUJlMRkpK+8nzX0z6ORmDoX2rrtqKkJSQOhPsxp5+JmnPyUmT7fJOSE2Fq4cPwq8ylSp9JT/t+wWASzpewvZbdxB88m6SgnfSI2KDSOT41laOyX1g7HoYtaR+Qm5Y/bD1JiFlVtnU18Ei3pHC26q11ZJFe0UCyRXV3YGpYqESfw1SaX1rscnYNnF8IsFJLkhGOhFdlQ5tuZbyKrGZl9d2+pkFFk3/91KFuOEadLBlrkg8gXgsf5OZ1F9d0i24n9Bc3v+M8MBqiNwXuj0BUW3gneEdKRYQFr4z/x71b+ZdOY8gVRBnNWfrZMU8hk63iIS6uledwXd5bSxm2X3LeFH+ItUl1oPzBoOBgoICt5k/vxv5He/Hv4/BaKASkTCKVoeIz93JLyH779a9gLq3qExsj0BdZY6QHbFBapiovjqcd9js8eOFViQktGXmmyk3JcJfzIVlevEezvh2BtcsvcaVQ7KbjG0ZpG1Kq/PzU+hFNaPp+2U355fCgnAhtdMW6MpBa2e1aMV52P2g+A5ZoVs3UWBUWSkSf2Cl0+/4x+L+X5Xr8JBdRuwV6ONuAOBA4BsM+WooP+7/kbPrz/JJ6iccWXikmQtYpi3nT32N3qxgwyTvGRcYd8En/ZqlwxRRTOZhHmI2UXeH4D4gkSCR1Pts/n58Hj8d+IlTRZ4tH2/izNozLL5tMTn7zSUiMzPrk36JwW5U1GKNoF4w4GMRxLwQ8YlGEtwTrUFV1+nnq/Sl6PEiqp6usukpby+uXH92GNSBMa+OIWFMve+yKemXmio+g5OTJ3NJ3CXtOi5r/6+7s3ZTWFlIgFcAAzsMNH/SaISqfMt7Y0WAsDZogQxtVlkW1/5+LY+uftRm8aCJ7nO6M+bVMUhk1udlT+/08w33pf/d/YnsHdlur3n0KBw/DgFlmZx98APeiXqHyqLmi8mUMiWDYwZzTQ/71t6Xp16OQqqoKzBqzH92/odx/x3Hm5saSfnKVKKotrEvl6NEjIagnnbd10NDIaxkAleUrjfzeosdGsuUT6dccN5+5YXVYDDU7UOyy0X8zU/i13T+XD8d/holuqBs7UGPvgu/KFtvmaMMgtErYcxqoaq07lLIWCSeq8qD/c/Cme/qjw/oLN5jo7HlcowNOfIu7Hqw7Sw7WsgSWTrPrV1NdW4coT6hyKQyQruEkjA6AYWP5ygz6Kp0HP3jKJqT+UiQcE3US3w8+eO6rjHHL1gOayfDefssUOg4Gy7PgYiRTZ8b+j+hBCZTijV0r5dati/wSxRNB8348b094W3Knyo3KzRwe1L/JTxXEcUdUkSRQG5JMUaDka8Hf836F9e7cID20eXyVI4Puo7KgAgKq/Oo0FbUfwYrz8OvfnDgRbuuZTAYKCxsP/uZi0k/J2PPgtTd0VXXy0YYjfWLYlOnnxEjPgofpqTY79N1440SUspEt98Tq5+uezxK3o1ijZTLU98i7sSoJn5nBCRDYFfxZ81BWBhTHzCMniwWZ3I7o6BGowg6mroDW8ruh2HVsNZdw0l4BRSzr+Mt3LpqFkaj6KaqLvWgzgcLxAyK4b4T9xFxeQTxAUl4V8fj6yu6Rus4+CKsmSASGiBu3sUHIbfeM5Lh82FGg02DMrC+O7PHv2HCNnFzBvG5O/S66C5tK4xGUSVUajlodlW3q9h/135+n/17nWdJHUX7YftdorPFHVGFioS6TIlOUciZ8A/ZbHwXEIbmqZel2pShMRqNpKenu838GXdJHEmTkiiuKsYoEeOOCQ4Ri7hZJcLwuTUkXCsqE32bl8ZpFWd/gYWRouLRBl1DxRx7JN886WBV3vPAv2GeV+ulkp3Nmgmwrt5UOTooBLkuEIO2jT3t2oAVD6xg/uz5xAXG8fTAN0jIeQBvb7DajHDoNXPpERMhAyBqgig6aAvSFsB8tfAsbQ51dxi5BPq+Z/UQiaS+26+yNp5ksdMvZgb0fqPZTla3ovP9KAa+jUwGlcqzbD2/hVOFp4SspwSM+pbNf205f37c5WO+HlwviVsn7xl44Xf6NUvyXdD3HVePwrkYjaAtqSvqMCX9/Ayigy+rNMviaTXlNaz810r2/7jf4vPuRudpnZmzZA4xg+ulx3U6+PvvBp1+QR7Q6ff/gPDgUkJ90us6/QDUKnWTotOsPVmsfHgluQcdKwRx5foztHMolzx5Cf5RQiO5ulrI6YJI+l3T4xqWXrOU2/rd1u5jMxgNrD61mhsW3UCVTuzNV51aBcDYhLFN5Ua1xfB7GGy/vfGlhBTa9JPCE9tBEoMSkUqklNWUNZGgt3j8hEQuefISfMOsxwZMST9P9fQL7x7OlE+mEDu0jfcQDVi0SPzuIRXFsp1ndG4T5Yy7+t9F3qN5vDD6BYvP78+t7X6NaNT9GjkOxq2HMCfEZ/Q1QhGoGaJrG9tPnsRsfrpQUS3+jT4r36hb82WXiaSfvFJuPn8aDVC0D3LXw9F3bMcu/JMh9grHvfwsEdxPJH87zq7tthpSO/AwmHYSuv+76Tl7H4PVQyFvU+teO2MRnPul9Y0FTiYmVtwnMxopzRoNRo/y9CvLKWPezHmUrBXdza3edxTugpy/m3buWUOqsO7TpqzdpOZuaF1DiHcEDPhEzGW2hiKRmtv/mMj4AxZ2sD+R6SIkEgnvdtvMyIOHMJaFIVPImPDOBHrdYDvZ6Q4oggMoCUtEp1BSUCkWEHXdpgatsNbxad7SCMTaM6PxF7MNuZj0u0gTsnZl8U7kO2z7cBvFxVBTIwJxJjmMT6d8SsFjBUxMnGj3Nf384LmZ1yM1KKmqNnIsXXxRTtbam20tegjJwM9te/4YaiCgCwQLo2c6XiUC5834BNWhr4JFMeaSjy0h5T7o/Zrrqruz/4adD4C2DH8/Gelh37AqYz4V2gp+v/Z3Xg98vd11/tuCSUmTWDbpBP1Pz2/qGVSdD2WnwKv2Q6kMhPGboO+79cd4BYuqG0v4JUBogyrVk1/BviftM4JuKUY9/JkkvAKsEBMQw/C44U2fKD4IJz+HCsuVjy7HoBPJhuoCdHINh+IeYIffcwD0v6M/Vy24Cp8Qz+leHPf6OKZ+PhW5VE7PnDdIynyaEHVt4kjuLSoKPYGQ/kJ2sZnNlKnTr2HSz2g0cqzAStJP3Uv4uKkaJaddjUxlJtMxq8dMJu3V0Of4AvR6OLX6FP+8+o9HbHSGPzmcca+PIy4wjrmJj5GQd6/tTY5BK7reGifBVOEwaqnw92sLfKIhZqaQ5LaHDlPEd8gGoxspk1hM+kWMgq6PeZanH2LZEBgIcr0aAE2VhpjBMdxz+B66XtnVtYOzQNKkJBLG1XehmDr9PE7eM2/zheW911ac+BR+CxRKCdQn/RTVQsY0szTT4mlSuZSt727lxNIT7TLM1uIX6UfKlBSzdcnatcLTr9K71tMvyAM6/Y5/Cv/MghqNq0fSZvQ7n8RjQ2c3G1TXnNGw9b2tZO+1U/3Fxei1+ib+gydOgF4PwcH1iSlXYTQauXXxrfyw7wcWHF4AwMpTQh7P4v7foIPE2yDcQidEK/CSe5GgFvcgUyGaPdhK4priGZ6a9GtvsrNhS61F1ZRn+nLj+hu58pcrmz3PaDTy0IqH+HbPt3WJ4+YI8g6y4FVVf70DOcLnsmdEGyqlrJsMq4Y025nasaOQ4dXp4L/zyiisFF0bumodC+YsYNNbrUwkuRmlIfEUxPSo7/SrTfqFqELMD5RIYeY5uCwTphyG8BHWLxp3JVzym3MLE1XhotuqYWGtfyIEdml6bOKt0Onm1ivvjFsHl+5t3TXagE7qXfSJXMnaki+4a+ndbM3YiuachpdVL7Pm2TWuHp7deAd5M/OHmdSkdEcnLUMjO95U4tcRwkfAZVlCvtMZZK2Av0aKWKKrkCrBJ7Y+CelOnF8KG6+qa3wYED0Q/6quVJSJ5OWQh4eQfGmyK0doF6XFejAaMXgV1nkeh/rULij8EmDMSki82YUjtM7FpN9FmuAd4k3/u/sT1S+qzlMnKMi800olV1muMrDB9LHhXKVbTs+zX/L3YvEFOVXb9CSPHAxJFqoDGxLcFybtFJU8LUHuDamPQYdpzR9ri9jLIOWe1l2jNeRthuMfQskRgvx8kRjE+1BYWUjcJXH0vqG3RwS1rZGzP4ddX+6iLLuMkhLxWJOkX/+PxEKyofGu3Mdxj7G8LaJDJvkuGPR125rfSuXQ/VmRLHGUuFpZgahJzh+XMyg+BH/EwfFPiQpSA6CTlqPVOy7l4074yP2JS3+MLpkvi89gdSHkb/ecAJt/kvANDBti87DU0NqkXwNPv+LqYmpquz2SgxstxDrNheHz3C/hMvJPuGR+3V9NyQijEcrK4Nifx1jz9BqqNPYFH1xJlxld6Hmd2ITalVzp8RzMTLP9nrRF0iNyHFyywO7KNkBIZOdYl/GIjhZdDiYsJv08kfNLYcNMksKPoTAl/ao1Lh1Sc0z5dApjX6lPGE9OmszExIkkBCV4TtIv409YPQwOvQrVTirH15aIzuJmvDc8DnUPUSiiFEE8U9LPWCKSftY6beRech448wDTvmrl+rqdaCw3rtPBL78IJZNRkTOZkDjBM+Q9NfsgfT5IPa+b3V5qOt7F5owrKCwEk4LcF7u+YOYvM+uSUQCdxnfigTMP0O0q217z7sKRBUd4M+RNTq2uV/84elT8Tk2F0poScstdJ18tk8q4pY9Q6Ply95eUVJewJUNkfiYkTmh6gioUBn0BCddZvmDWaiF71YJ1SOdQITFvKkRrjg0vb+DtiLepKbcsQ29KqBYUiCSrp7Hj0x3Mnz2fquL2WcsuWSLetr59ocfoUDqO6Ihc1XxH08nCk7y/7X3uXnY3Monjnpn5FeZZ2fSSdIqri5FL5XQJbZTAOfo+HLWuIuEQCXNFUVkzntESCVx7LRzt8BQP5fnz4l/CYkSmlHF00VEyd1gukvFEjEZI7zictB5T65J+OWVCbSnEK8TySd5RwpbGWodUW2PQCtnQ80ut+5oHdIZBX9rfQGANibTtFFVaQVTu0zwx/ErSvZfw+c7POJBzAN9wX5IvTa6zcfIEvAK86HV9L8oCO1Dot5Ebd3Vm7A8tLGY12ZN4Bbf+fTfhEycU6FLub9119j0LG2a27NzoS2HiVud0OzubstOQ9puQwKQ+rmuyp/IUlt+ygL7LX0XqJ+a+IFWQw/kQV3Ex6XeRJoR2DmXKJ1OIHxlfl/QzSXuaqnpagkQCT88ZQ5Tmclavhtxc0emnkFaRlGhnZ1pru+v6vAHxc1p3DVeTdBtMPwPB/Qjwl6DUC8nKwspCBtw1gBnfzkDpZ6XDzQM4ufIkS25fguacpi7pZ9HTyZrxriMceUPovFdlicqMtpZl6PFcnZGtQ0hlonJN7qbdct4doPODEDqISHV9hD6vVEPO/hyW37+czJ22Nz/+TTK7rsFoMLL49sXs+2FfnS8h1Aa28/6BVYNEEPkCwrR5zinPoahSeMipVWrKniwj8+FMvBXOl/BpD2Sy+oRESYmoJLt91+14B3vOv+dM0Rl2Ze2iWp7b8uSKvgbWTIStNzpzaC1ny43Ck9cGJolPsDL/r50MmzzsXl6RDueXEB2UhUKnBkSnH8CuL3dx6Nfm5aSs0V7z5ydTPmHFdSuI9o/2nKRfxCiRyDr9HSx1UjdlZQ7kb4ayM865nrsQPkIUigSJggNT0k9bZDvpB6COV6P0df+1p9Fo5N2Yd/l52s91j5n2JMFBEhbd9gkrr1tJsLcH+GMP/A9crWu2e9qTUQ14nj+P/wuDATQa8dje7L38cewP9mbvrTvOy98LdbwamdLx5IIr1p9ylZzQLqEEJ9Z/zhr6+S08spCItyO4bN5l7T42Ezf3uRmpRMr6c+v5YtcX6Aw6koKTSAhKaP7kxqTPF9Lw1Y4b6Zl8pe3t9PMK8CIoIYjyXMteYmq1KGQ2GqEdLXWcRtaeLA79ekhIg7cD+/aJ3xPtF3gCYGfmTgB6R/Z2KDBaXFXMwC8HEvNuDMVVxXWP788R0p6poakoGyv5nPgMTn3l2ACt0WkudH8GFM0vbnr0gITQDgD8fVB0IUokEp4oeYJZv85yznjcgOrq+qILX1+o0ddQUCmKqDqGdDQ/uOQEnJsn1Ed0FbWyh1bifHsehYOvtM2g/x4NK/rB+qmiWN4WZWfE3r4lxZHl6ZCzzn5/83ZEmvoQPx3/DJlefJbLaspQeCu4+o+r6XNzHxePznHKykArF3GKIO8WqC7lbYLFSeIz6UzU3YSqjk8rZWpLjgiPSYMHVqPYIukOuFpb1/W7vuBXjkU/T1q1uLn8cfMffHvJt64coV2ou0ZTFNkFfEVBSphvg8R52nzY9wzUFFs5uyl+7biBvpj0czIyq4Y7nklDP7/iqmJi34sl9ZNUs0WYI3TvDr17i6raefNEp9+VXV9ncmUwlNgv2+EyMv6AP5OFxKar8I4Ev3iQSPHzA4WuPul3IdDtqm5cu+Javsz9kjt29yE95Nv6oG/ZWVg9HHLWOufFer8Bkw861qXiCoqPih93lSdThUK/9yBqAv5+MuQ68YZlazSUZJSw/aPt5OzPsXq6TCYjMTHRLebPmrIadn+5m7PrznIyJwuNz06kgVnCSy2gC/R+XfikeQo562DFQLEYsYK/lz+j4kcxs8tMymrK6h6XSCRE+UeZH2w0imTL8U/baMCtIHcDHH6zrqJTZ9CxMX4S/6QOJKuwhKCEIKL6RrUoKNjefNrtU5bcuYQPt33I7Tv7czriXevJlZLjcOAlMUdYQqYEfaWQyHYmp7+Hv8eJ7ldH6P60kMi2wfDhIpkUGCgCdE0wVDVbhe12JN4KV2spVIyq7/SrTfqtfWYt2z7c1qLLttX8WVVcxZ+3/snh+YctPu/2ST/TplkRIBJZyXdA7CzrwSdHCEiGWaXCH/gCJiQEfHzAq0YEMqzJewKUZZfZvM+7CwadgV439CKqn7i31dSI/QjArFmgbMu8Zd6m1nsHAZz5L1TVbtCk7n8/aw1SqVCbgXrfrEg/0VGRU27+eSvJKKHghGPdvK5af3aZ2YVbttxCUCfxjzMa65N+XbrUJzjiAuLadVwNiQmIYXLyZEB897fcsoUPJn1g+eCzP8Pm66HivOXnuzwCl+5rkTx+XdLPzk6/QfcP4tZttxKUYPm1JBIxt4FnSnxO/3I6z9Q8Y1e3XWsxGiGzdtpPcDDXa0r69Y/q79B5gapASmtKqdZXs+T4krrHTdKeTfz8AMZtgEsWOTbA5tCWNCvxKZHAteNEkcypsv11cTOZ4sKal4vztXTaPZ/Q8/vw8qKuC1kuldOvaz/z+TNrOWy6GoqPiCD4XyPFPsUSZ34Qx7cFnW6GlHuh50vCN9AWh16FDTNapgaRsUgkGAtatoZvU6Incl5xHXKDKGwprXG/xKQ9HF9ynPfi3sN47DhaWW3ST9WCpF9VjtgLu5s9iYnhv8LlWS1b1539WcRAnLHHcTYypVljxeJzP3Ai+gXOsxOjUXja67Xun+jseO1wzvS9AoO3mP/CfBok/TL+hEOvgJ1d7TKZjE6d2s87/GLSz8kYDG74RXOQXV/sYv7s+VQUVJh1+q06tQqdQYfRaLSqt24P114rfq9eLSrsCipjkAT1At/41g++Oc78CCuHQHlay8436MTEJXdxV1JVLhTtN0v6FVUVkXckjz9u+oPTf9lpTOuGqDuqSZqYxImaE5yp2otWrqmX98zfCgU7HaqisElAZ8sa723F2V9gRX+xEHaEXffDygGu85F0AKkUlAY1AJlFRcSPiufh8w/XSRVawmAwkJ2d7Rbzp9JfyWOFjzHhnQksOPIbG7sO4ECHB8WTAZ2h6+NCrsRTCOwOFWlQabtLe+0Na1k4eyGxgbE2j0NXDmm/QqENY3ZXkb4I9j4O1fWb0WyvDRT77iBLU4jRYKS6tBp9jXsvLI1GIwofBXKVnKIqsblR6NXWkyu5G+DAc1BqIyA2bp3wwHUmNRpRlejrYFAybhZ0nG3zED8/eP99ePddc2nxOsaugRG/O/a6rkYqB4mkiacfwOxFs5n+1fQWXbat5s+K/Ar2fL2H8ztEELdCW0F5jeicKCgQPxIJRLjj3rmmWHRln/lv/WOpj8CAj50nNSWRtL06QHujq4CtN8OxjwDxT4yLAy+tSJDZUvtYfPtivuj/hU0vLXdAppAx+aPJjHp+FAArV4rPcmgo9L+kgNLqNgyKrR4uflpDwQ7YMhd23uecMbk7mSt4cvBEOgXtqUv6RfiKSafx5/HLgV+y8PqFDl2+vdefZ9acYf9P+5s8npkppK4UCkhMhAO5NhIc7chtfW8D4Mf9P9I3qm9dErAJBTvg7I/W59eAZNFBLFU4XMDoqLynPZgkPvMcbzx0C9orqZSXJwoj5HLH7/U7MsU+oX+0Y0k/gCtThWfg/CP1BYv7c8X3pme4hf2kd4T4jDmL7L9gYQykLWj20OmDugNQqUzj+19EwwXTsAABAABJREFUfCL/aD5HFx1t4tvpqRRkVBCceYiAimwkErG/unfAvdzY60Zyc3LN588O02DYL0IuPO4KUSxrzWdsRjqMaCP1nMSbhR1M92eE3YUtEubC4O9apiIVMQb6vgvq3i0ZZZsTEwPyBp1+INRF/rjpD1cOyyGkCil+kX6Ua5X1nX6OJP1Mkp6xl8OMsyKe4460Zn9y5ns4+KLr5HRtoauA3H/qPP2CfERzgFZaTEUFzPx+JrduvdWVI7SLigrxW+0VyrSUaVwSd0n9kwM+FtZTdnSIg1h75uS0X6GkG34qPBt33+zaQ/bebA79egiZUlaX9AsLg2UnlwEwJXlKq67fpQv061e/5j9ScyvSCeudI9fYHDVFUH6mRfIigFi8TDkEoQOdOy5HWTMeNkzH3x8zec/qkmr2freXrD3WJZjcHdMCubhaLJzlusD6Tr/4q+Gqcugw1UWjayWGKlFlVONgV2anm9y/o2DrzbDjXgBUiIVYXqkGhY8C/2h/m91VRqOR7Oxst5g/JRIJ3kHeeAd5k1sqIkz+Mit+BZ6AKhRmpEHnex067eGVD3Plr1eyOX2z+RMKP5hdDf0+cuIgnUTne2HCNlDV+yp4I+bH7JICdn+9m9cDXufsurMuGqB9SCQSbttxG5Pen1Sf9NMFWU/6xV8DE7dD+EgbF22D5V6XB2Dq0TbzsYiIqJcWvyDQVUD238QGHkWhV+OFH961snyxQ2IJ7RLaosu21fwZlBDE40WPM+JpIcfy/d7v8XvNj9nzZ7N7tzgmJcWC5647UJUrfirbyFOncDdkrgC9+/uDOoRMBWd/gtx1dQ/FxYG6fAAfdEzn8N2Wuz4BelzTg1HPj8Kod/193BaNvyeLF4vfV10Fb2x9iYDXA/j32jZab41cAmNWt+4aIQOErGc/Kx1XFxrVBST4bSRIlV3XlWWt02/Iv4Y4LFnWnutPvVbPkjuXsPLBlU18JU1dfsnJIsFi6vTrGWG9YK49mJw8mWj/aPIq8vjjqI0gcb934aoys/WXRc7+D/65wiH5sj6RfVgyZwkrr1tp1/GVRZVsfnuzzQJYT076nVp1iuy9LbdbcYTztY2bUVGiqNNe9AY9u7PEQmFAB8fVUa7sKpJ+y08sryvEeHL4k3w57UumpjSKAWjLRCeZM+/HQX2Euovct/lDvYOI9BYFk39uO0BhIWx5bwvzLptHZaEVLzkPQxIYyK7Jz1A1VGjvR/pF8tHkj/h8yudN50+/BFHY5xUs/MW6Pi78/SwhU4rjXE34JdDphpZ5vKm7QZeHxH7b3TjyDtf6xRGnEPOt6buUvimdvd/tpbrUMxRTkiYmcd362ygJjkcr0wAOyHue/gEWRkHpSfF3maptBukMqgtEoYG1zlhbDPoKxv3j/DE5g/I0+GsEnBYSnkHeoghAKyups3LyBA6+tZywszvo4jWaP+f8yWvjGqgWKQIcagowGo0Xk34XcS1TPp3CMzXPoPRTmsl7bj8vJLzGdRrXsgs30Lo2dfsBJDVTfONUUu6Fy7MhuF87vmgbkHIfdH4Ifz8jCp246RVWFhLVN4onip9g6CNDXTzAlrPw+oW87PUyJWXiLqDQB5oHFaUyz62u73QjzEx33GQ3fo7oUnBnCnaAZi8AKokagNzSIowGIwUnCihOc1J3ZhtTXVJNzv4cqoqryCsXSb8ARW3Sb8+jsHJwfcWYp2DyvrAjyLLq1CrG/3c8i48vZsGRBZa7HqRyuyuZ2hW/TqIgo0EBibdUDUBBWTFhqWH0vqk3vhHNb+LdBVMnmEJvI+kn9xGBYKXa9sXSf4cNlzUrV2RGxp9w9L36z3xFJux+pF5asyUb5NwNsKijCP61hOpCOPaxSLx4ElU5sGYcXRRfE1Q+iKelpWy+RSTVjUYjVcVVblUVLpFKUKlVeAWI75MpwK72UrNTqHbR3/EC/vYhIBkm74fUx+ofKzsDm66xKXVsN8c/gXWXCtncCwmJFC7PheH1/0dxcSAzqijLjLHpy9T96u5c8tQlSOXuu7UsySjho+SPOPCz6KLSaiGrtkZu6FA4XSSSBKakktPpMAUiW7iHakjS7aKz5f8D8XP4rqKcXVmX1nf6+Vnu9Bv6r6H0u91993cyhYyr/7iauWvm1s2rJhr6+eWW55JTnoMECd3CurlgpPXIpXLmdJ9DTEAMJdXNROfkvs0rkuSsFTJ4FfYr7gSqApmSMoWkYPsCBvpqPasfXc2RhdZVVUJr4/OeKO/56xW/svox+4sHjEYjBRUtkC2kPunXoYNj5x3NP0q5thxfhW+dPKsj9IzoSVJwEtX6amb8MoNXNrxCz4ie3Nr3VrqFN/pO5KyBJZ1FwYqz8AqBSdshZppdh/eLEcn5IuV+0tOh19xezPxhJnJvD41XNKKsDIxSGT6BdngzWpIXtPRYTTHkbWmZpGZb0ZLiDzcoWLaK3BejVxQKgygwLNOKTr+J703kqYqn8PJvh4YLJ1FaG47QKzSAA51+Ph2EnGczikduQclR2HglnF/s+Lk+MRDspj6NPtHQ9726pg2TYqBOVkxpKWTuymTbR9vcukjCaDSSvXgHAXmn8fGxcEDRftFc5Ka4787sIi5FppAhkUgadPoZOas5C0BySAvkE058Dguj69p6k5NhyBDoE7mKqzte77jcYUtprTzigRdEENTVJN0KXR7Az19Ct/QPuK+kjEeHPopMIcMrwAuJB8hAWiOyTyRJk5LQ6DUAyPUNOv3O/AiFu1w2tovYYPJ+GL8RgHH6txl6dCPdfcahq9bxccrHrHl6jYsHaB9pm9L4vNfnHP7tMIW1JuVBXrVJP105VOfXJ9E8iey/YHGikMi1QoW2ghsW3cBfp//iZKGoiEsJSTE/qDxNSDS4oWE5IPz8Gni9+ZqSfuUa4obHMeObGUT2aqOArpOoLBRV6hlbMyiqNMl7Wkn6Hf/U/uRXyVHIWgWag7aP01XU//nM9yLZbSq0KD4IR99p2YbEhDJIBFQkLQyGaPbDrvvE59CTUEXCgE8pC54FQHGDOogVD67gDfUblGWXWTm5/aksrCR7X3ZdR4rJwyXMJ4I9e8Qx/SzF11sqn+4Mjn8KGpHQQaluuuY79zMU7W396yTfCQO/BIW69ddyN5SBZv9vHTuK32kufFudRdHpIvTV+jppPNMeR6WCgAA4ozkDQEKQg+ZV9qAtEfem88sguwXroew1sGGmSF7/f0IirfNfa+LpV5bjFgoRjhCWGkZEj6YJ24ZJP5N3WaegTvgqXV+k9Pyo57mux3W2ZRqzVlv3FW5I/49h0m7RDdRG+IT5cPPmmxn5rHX1A0/t9DMajUz+ZDKDHhhk9ZiX1r9EenE6IArHrvn9GoZ8PaROntsRTH5+jib9jhUcQyaRMaDDAGQt8KeSSCTM6T4HgLVn17L27FrrB/sliAKf0CEOv45d6Jr/f+sRLmR4S70PUFwMccPi6HV9L5S+HrhftEDh2RJ8C9PxlYv1YH5FPvkV+RgsJfMWdxaKVCYOvwHzQ0TBntlFd8HqoXBuXhuO3E4MeljSFTZf49h51YUwP1j4qrsjyXeiHbON7PJ4AIorxd7dO8gbhbcdCVw34dTqU2x7ZxMybRVGLw0AapXa9kmm723EGKHSFt5KafX2ILArDPkROrTA7qEiw33VRxQB0OVBCB0MQICXCOzqpCWUlsLJFSdZcf8KNOc0rhujHSR9+Rhne03Hy1tn/oS2FJb3gl0PuWZgdnAx6edkPDnZYuLUaiEbUVEB5bXzpcQvjwptBRIkxAY04/lkiYBUoZ+s1YgE3457ePhBHbdfuZPo6h+dOn6bGA2iyjvTPokQM7SlcOB54WflJvj5Ca+nqlLfus9e7sFcMne2kaRVOzD0kaFc/cfVDTpcapN+ugrhY3L4TZeOr9Wcmye8x+ylMgeW9RadBe5Mg7kvyac/wWXDkNeEoPBWMPyp4XSeab3SUyKREBwc7BbzZ3BiMKNeGEX0gGiKaqsPQ7xrI04DPoXpJ103uNbgHS26UioyrB7io/Dhx8t+RIJ4H7xkXsQFNvJrS5svJBo0TT1pXE7mcvjVR/jK1OJfG5QvqtS4ZkwtoOR8CasfXc3JlScbyHta8PSryoXdD8Luf9l34ZR7Yfqp5isBV/SD9bUbjt5vwPhN9fKg3h1g6M8Qd6Xd/54mqHvApbuh41UtOz+4H4xbD7GXtXwMrkDuDcl3IY8Q8uANk36xQ2PpfVNvaMEU2Fbz58kVJ/lP7//USaSZOv2MpRFUVkJgYAOlBn2VWF9V5cLiZNf4jRkNYo225QbLz/t2FPLgvV5u/WuFDBDFV25wz3I6FefFXFobMImrvQX8o3+bOb9dy97svRZPO7P2DN8M+4Yza9w3KdVxREcePPcgXS4TXs4NfcvByJmi2qSfug0SErsfgV/9YP0UEQR1lNx14n1pySThyRh0JPsup3PI1iaefhKJhNKa+gKkwwsO8/WQr8k5YL9kUnutP1c9sorfr/sdg75pkLysDNJFjoYuXer9/Fwt7WnCT+nHa+Neo1dkL8sHGPSi83nfk81fTOZV36XqQMJ2W8Y2Xlr/EouPNV9wJJVJiR0Si1+kdUUKU6efpyX9JBIJveb2ImVKisXnCysLeWH9C8w/PB+tXovBaGBj2kZOFJ7goZWOByQzarcMjib9Lk+9nIyHM3hv4nsOv6aJpy55il+v/JWPL/2Yu/rfZf1AdQ/o84YImDubo+/D75HivmiD0Qmj6S+7hZDSsXVrO6PRiLbCAWUNNyZj1SFSN3+Dd4mYW19a/xJhb4Xx3Lrnms6fIQNA3WDuUgYLCcyqXPOL+nWCPm9DmBskY6Qy0SmlcrCDXqsRXqXuKO1ZS0AApGhnM3r/KV7sK+QVa8pqOL/jPKWZblrA24hjfx5j95t/IdVrSdRP5p4B99A7srf1E0qOi+/tya/EOt1T1urKIEi41nF/Ul0FLIqFbbe3zbicTKCXSd5TdPr1uKYHN6y7geAkN5D6tYJEIqEaFXqlN9+Vz8L7FW9+2PdD7bNG6PE8xMxw6HpBQQ74UraSC6Pn3I2QOiJ47qb8Nus3ovtHM+KruYBILGVXnQUg2j8aL3lLTG5HwsQdYtLd+xSc+BRV7OXETHwKqu8Qk1y7IBHeYyEDIXqiY6fKfGDqMdxiw12VB+un08F3EvBvyho0B8y7fB5yLzl3HbCxQPYAiqtqPf1M8p4SmfBDaU7Czt3Z/bBYWMbOtO94bXFthbj7trwDUHwYig9Bh+n4+oo5wmR4O/aVsTZPlUqlxMXF2TymvQhJCWHkc6I6uHiJiDCF+nqwp5+JwK7CML2ZLsWxncby4ugXeXbts/SI6NG0QjdyjJBo8HdDE2zfBIi/TvyuJUCphipR7VyaWcrfT/5N8pRkul3lWsksWwQnBnPLllvwi/JD8z8NYKXTTxUu7qv2oggQP7bQVUJQXzFHAfgniR8T6m7ix5Uo/CF8hGvH0ApMnevLAi5j6Nc5/DbrN7rP7k732d1bdL22mj/De4Qz+qXRhHcXxoo5ZSLYozkvgiL9+jXYR+97Go5/LJKx8deJytp2RwKjlop7psWnpUIK9yK2Of0t7H8WJu2E4H4EBYGvL+QELOWXw+uY1mWKxWCLvkZP4alCKovce60ikUrqCltMdhoREaJzoVwrEp0d1R2d/8JBvURAoOPV4Jfo+Pk9X4SkO4RU1f8rJPQonMrlqTP54cwCALwV3hQ/UYy/0t8s2Kyr1KE5p3FIIqq91p+5B3MpTitGKmsaJzh2TPyOjhbFFH2j+nL/wPvpG9W3zcflHAyi89mab5cl9j4llARmnAVp8x0nq06t4rl1z3Fj7xuZ1rl5yUV9jZ6K/Ar8oy1LkHtqp19zLDm+BL1Rzzd7v+GhIQ8R7B3MDzN/YOwPY/ly95dcmnQpl6XaXzDVUnlPEB25rZFKVslVzOo2q8XnOwW/RJFUrM63OfdOSJzAqZAJLCsSBV3aSi3/6fMfovpEccXPV7TjgNsGRVI86V0n0D221ie9XEglRvpFNp0/hzWS7k+6Tfw0xi8eUu0sWmwPxqxy/By/TmLd666UHEeSsYjBCdNYvzeV0jygC2TtzuK7kd8x8f2JDH5gsKtH2SyXPHkJkl692LnQl/7ym3hz8k22T6gpAv9k8eOJGI2OJSqNOuj8AAS7q+cCsHYSyLxhxMIG8p6i0y8oIYighPZLgLWEmrIaSo4XIqsJpIICqnRVeMuFbC6KAOjhmBe4VColNrYFjVQt5GLSz8kYDBba3D2MSz+6FO9g7zqd+/BwkZG/q/9d+CgcDJgU7obT30HiLWLDC9DtKbHxDa2VpvBqx4C6RAJDfxLawo4ilUGA5cq6dkcZDJUZeAVUUOy9l30hH/HMmmheHvMSw58c7taeKs2x7vl1KHwURPlHUZ2tQqFXi6SfzAs6THb18FrPkO9B6cBnPiAFZlg3o3cbTn4Jx96HGWkUKYs4E76Of3I6cRlTmz3VYDCQkZFBTEyMWxVOlOmFFElEQIhYgB3/RGz+IqxLBrk1dsqSPnXJUyQHJ1uuMg/qLX7ckcAuMPS/5g95BSHXBVJdY0Rfo2ffD/vwCfNx66SfwkdBzOAYjEYjz454lh8XFOGljbAs7xlkpfLeFmkL4MibMObvpt6Mcm8Y5kRfFGuc/QXKTkH3px0/t7oQFIHinuxprJtCRE0V8DcF3lvIzMghvyKfDgEtD+S31fwZ0SPCTIbO1OmXdVIkAc2kPQNSIXKC6MIc7KIAgkQiKsxtUXJcSNTGXNbyyt+aYlFRm3Iv9H61ZddwZzpMFdXu3iLxL5GIbj+vKhHQzyy1rCSRNDGJR7Ld13v40G+HOL/tPMMeH4ZvmJBMNHX6RUTUS3tG+0ejkqucP4CUe8SPo5i6oSSS/4cJP0Aqozj1Oxat7kR+aX0szCQP1ZCe1/Wk53WOdce11/rzuhXXoavSWXyuobQnwIiOIxjR0YMKW6QKSGwmCNsYhZ8ImFfl2bUn7xwqCs2O5R+z6/K/XvErJ1ee5JnqZyx2cZqSfqWlUF0NXh5ibXXun3MsvH4h494YZ7FQaNHRRQBc1qU+sTc6YTSPDn2UNze/ya2Lb2VQzCCi/Zv/P6+pqU+KOpL0K6spw0/Zjr7fqy+BsEva5n7cYardvn6BIo5NcTEovBWEpYYRENdMoZ2HoAuLIqdTFENrPzYmP9Vw33DS0tLcbv9+kVpKjsLex+kRE836val1nbshKSGMfW0sccPdo+C6Ofyj/ZF39Acp1v3tGxI6CCZ5qB3Q+unCpmCGA6oZigDo936bDck5SOtUgyYkTuBfQZvYvT+SklqrYKPBiEFvqJPfdzey92ZT/Ma3hHSdSIVRxAeDvVvemWgwGEg3STy0AxdnZyfjad4Cluh1vZCNMC30QkPFYvvTKZ/y9oS3HbtY0T44/hFUNPhQK/zEZFyRCWm/iQV/exIzTQSmHKUi033MhqUymJmOrN8bVCtySA/9hj+PCsmTPjf1odf1LQgEuwl7vt7DkQVH2H3TMSbsy8VLFy46I/RV7m2WbC+R49zXaLc1xF8rZP+Uas6yjkNxD7BBIxIwKx9eyf+m/s/qqUajkcLCQreYP3d8uoNvR3xLSUYJA3UPk5j1OAlB8ULqbNd9cPobVw+xdWy/U8iM2UAqkTK7+2xSw1LbaVBtx8NdPmTSXg19q/5FYFwgT5Y9yfi3xjd/ogvRa/V1wcHHhj1Gl4zXkBv8zTc6R96GHBseJ7aoKYDSE/XeZ67g7E9w8AUhyegoa8bBki7OH1N7IPdH5uWPTAZynRoQXahFZ4pYdMMijv5hhydSI9pr/jR1+hWlRyCRQJ+Gt7GkW2HU4vqOjcpsOPIOFDjQidpaNAeEzJwtjrwJ/1wBNYW2j7OFrlysYb1bUDzmCQT1FlX53vUJ344dQaUVScDzJbZlztyVY4uOsf2j7ci96utdTZ1+4eG0rbRnYwxaqNHYd2zGQvh7NJR5QPFXG+HT7XqO5A+jurpeQcJZtOf6U66yXGt9uvatTXGTutJ2odtTMP4fu4twO4eIpN/xguN2HZ8yLYUBdw9AX2P5nuDjI7w8gboiZ0/BN8zXoldchbaCFSdXAOZJP4CXxrxE36i+FFYWcsOiGyx7sTUiK0tsu3196xUKmkNn0JHyUQpjvh/TPvcKXQVU5UB1G8WSHCgO8g2opsR7H+nFIrMye+Fsxr/h3vsNezHZ/fjWWoyakn4RvhHm82fBTtj1YFPv8OOfNpW13vskrBhgl2diu1B8BPY+4Zjv85F3hQSsuxJ+CUzcTm5gH450eJyvTj8DgF+kH8OfGE50P89Yx1YVV1GSXyP+7HOKvPI863OYXvhOepSsZ0MCukBw3wsj5tmQ0cvgEqHWEO4bTu/gofjWdKK0FNI2pfGi7EV2frbTxYO0TkBMAMbRoykLjqXc0Cjpd/Ir+HusQ+t0o9FIUVFRWwzVIheTfheximkRbKqGaxEJ18HM85alnjKXwsarIPvvVrxACzHomg8ONWbfk7Ag1P6NejugVIKPREw4BZWtCGK5EXfuu5Or/7y6rvLDy0v8O9l6s/j/d1eTWkfQV4ugjz3kbxXdsjVWJMvchZD+EH81KPwJ9hEt+qU6DQDFacUUHHeThHkzVBRUkH80H4lUQufi+0g9/zrxoRGi03TsOujiRlIkLSHvH8jb2PLz/x4LGy533nicidEA226Fo/UeIn5+YsFfUSFk3ZS+SrfwjrTF4fmHecX7FY4uPIpeD5W1SmV1ST+jUXiX7X2iZS/Q6SaYfhrChpg/Xp4GKwZCRvOeOa2m77stl8uOvhTiXCz51FKG/4Jk5CICA4UfL4ikn75adKG6kx/vpjc38e0l31JZWInBaOCqblcxUD0VL20UXbogOvCtUZ0Hex6Bc7+0z2CrC2F5b9hqxc/PRMKNMORHkLaircMnGsashs73tvwaHkZcHHjXiKRfRqllX1ij0cihXw/VeUC6G5f9eBl3HbgLr4D6976hvGdHdUdu73s70ztPd/6LZ60W+52i/VCeDr/6izncHsrOinWgvWvGCxClsn6+Me1Nv9j1BTN/mclvh36rO85oEJ/Bkyvdy3v56KKjHPzlIHqt5X1naa2tklotPNk2p2+mpLqk/QbYWk59A38mtWmRR3KIkGkrqCygoKL5/US/2/sx6f1JZkn+hkgkninx2fGSjty24zZSpjbNEK86tYpKXSUdAzs2kWBWypT8dPlPeMu9+ev0X7y/9f1mX6uhtKe9y+ZVp1aRVZbFwdyDhPuG23dSa5D7wLTjMPCLtnuN/K2wfgbkbbJ52Cfnb2FDt95sKf+vzeM8Ec2Xv5G89b91ST9TEZjJX7WOwh1w7AOoyjZ//MwP4vGG6KtFMb3MTWTXK8+LxGQz77MZxz9y72JgZRCEDEAZLuVU1JtsrvnM1SNqET9P/Zkjd32IESNvlHcm/O1wskqzmh5o0MKS1GaLm92aPm+K5JgjsYqs1SI2U7in7cblZExrutJSkYTuemVX1Alql47JFup4NRX9R1CujqZU1yjpV5UNhbtat7dsYy4m/S5ixrl/zvF+x/c5OO+gWaffmaIzLduASBUiQGLJRyV8FCTfAzFtsMG2xfFPYJ5KLEwcIXICpNznPp5yJSfg8BukBon3RVMlJqB9/93HJ6mfkLPffiN7d8I72Bu/CL+6TXBdYDGwG4SPBFkbyC61J6e/g3ne9nfpnPsFtt4kfP08hBAfNQDlelHBctX8q7jv+H0uHJH9jHx2JI/mPop/tH/dZ9DPDzGXRYwUht2ezMSdMHFry8+XKkUC1B2RSOHsz5BV78tg2qCaqlSz9mSRtcfCRsGNCIwNpMe1PZBHy9l4eheVyjSg/t8CwIRtMKiFG02pov4+VpldX01YfBhKjwlvgLYmIBn8ElpWhdnrFY+XVQwIME/6BScH82Tpk4x+cbRrB9aAioIKCk4UIFVIkUqkfDX9K6aVLkZhCKB/Q9sIfQ1sugbONujmDuwOo1cKH7L2wKiHrk82nwwOHw4J1zaVtb1IPUYjLOsFm+bUPRQXB6oaoe9mrXtDIpGw6MZFbHlnS7sM01EkEgkhKebS6iZ5z/BwGBwzmP9M+w+PDXvM+S9efBDS5ovPqU8MRE8WUuH2kPowXFUG/v+f2sAasethvpqkRiGtoqA233Mg5wB/HPuDvdl764+TwB83/+F2n8FNb2xi6d1LLfr5AXW+7H5+sObMGoZ9M4wx37vCF7WFSOQg9wW5rUoQC6T/Lgq17Oho8FH4EBcopOiO5B9pySibYEr6eVqnnzVM0p4zu8y0WNzWJbQL7018j2j/aHpFNK8I1DDppzPoqNZVYzQaWXFyBdN+nlbX7dWQ7/d9D8A1Pa5BIWveq9FptGUxn74KMpcJeXAbpIYI24BMfX2X2+a3N/PjxB/dQsmmNeh0okTP1xcqtZUUV4tC5CaejYm3woxzEDrU/PGhP8KkRgmJfu8K+xJ3KcQMHQJTjkCiBf9Ba1y6G4bPb7sxtRajEXQVJEUJycRqSuum25+n/cyCOQtcODj7SZ6SjLJ/T/TScgyI4hm1St30wKpc8O3YvtZR7kDZKTj/B+hKXT0S6+RuECpFukrKaspYWfIeJ6JeoaQEghODmfXbLDpP6+zqUdqkshIM0kpqDKKbNMSn9nPW/RmYpXFrCf6Lnn5Oxt07COzBJ8wHpa+yLukXHg6X/nQpxwqOsfaGtYyKH2X/xUpPAUZhhNz4/yYgGQZ87Kxh249/MnSY5ng2PuFa8eMuFB+CvU8wMPxDPjNAha6cGn0NEqkEJMJE2tMw6A3kHc7jpPEkN/99L+XxPbhC+oN4siXeT+6If2eIu1J4UtlDyr0QMRa8W26G3i7kboSNs6DXq4TWGidXGDR2nSqRSIiMjHSr+bO0upRz2mNIlOEEBMTVVtlLPdNHrCEm0+GWMnq5c8bRVlyWCYr6wNPxiq1sS/43gSQAn/PztJ8J6BDArdtudd0YmyFueBxxw+NYeXIlk36ahH9STyad2YfM9NGTSEDtBE/C/G2wdiL0fAk63wfRk+DyvPbZgBsNIuEo8/r/tTnL2wS5G4gMvguFRg2IpJ9UJkXpZ5/nZmPaav4c/8Z4M2kqrRb27RN/Nkv6VWbCuZ+F3GX8NaZBQdQEp47HJqow6PVy+7xWxh/Cr7rLg6KK+kJDIgGvMOEdXUtcXL28Z0aJ5U4/gCt/uRK/KPdLqB5ddBSvQC/iR8XXfU9qasCkrBMRYeNkZ9DlIUi6HaQq8f874nfHzpf+P9+u+0STXd0fL3kFBQWi8C/CT7xpJq9REHPhlfOuxD/K/uRTe6w/L/vxMjRnNGJ/ZgFT0s/fH3aeEvJWfaP6ttl4nE6nueLHUXI3wqmvRcGGf2Kzh/eL6kdacRrrzq5jeNxwm8fmHMhh/fPr6XNrH5IvTbZ4TEjt0qPQg4Ryji89Tv7RfAbcNQCFT31STWfQsfi4UGloLO3ZkNv73c7V3a8mUNX8HtSU9AuLqmLWb6II5LdZv/HC+hfYmrGVL3Z9wXMjn6s7vqiyiD+O/gHADb0add3nbhTdEHFX2vXvtJusVVCZBR1nt11RcNhwuKq8WV/0XlHd4RDkS+qTfppzGnIO5FCeU45fpPvdG+2ldNIsTpwQST/TnOsl80KtUqON1NbPn1IF+FrwifNPasfRthC5r/CGdwRlkHuvA8vPwZ8JdE0UyhRGqZbz2TXERCnRVmit3pPcjeFPDGezFLRbhV2UQqrAR2GhocSnA4xb2zLrCHeh5Bic/FLMlaF2eqQn3ynWmO5cXJC+EI69D3GzqZJ489nph6EDaIoeA9qxQKSF7PpyF36/7kPaexAgPoO+Ct9mzrKORCIhos03H/Vc7PRzMp5uYtvxko7cvvN2UqY29PQzcq74HEBdlZ3d7LofljY1mnYpURNgxEL7fdWKjwiNcncjYiSM38TpqrlgFDftosoiel7bk3sO30PMoBgXD9Bxqoqq+Lzn5+x9dS/HS/ZS5n3Ebh8BjyFsCAz/VXgC2YN/kvChlLr5DVEZCH6dQOFPuL8agEpERC3/WD77f9xPZWGlxVOlUimRkZFuMX+eWHaCY38eY+f5PaxJGsDWlHGi0+/sTzBPCZlunvRqjhqNCFqXHHP1SNoGZWCdUTSATlZKXuAqcpWi8n/U86MY/LCdi2gXU1Qlvj8KfZC5n5+2DGqKWr+4D+gMAanitwmZsn3mmsLdsKgDnPyPY+dlLoeNs0FzqG3G1dZkrYJ9TxGlPm/m6QeQezCXjG3WEyrWaI/5s1JbyZ5DZVRVQVAQJDS0PfOLh6sqoPuz5icZjUKRoLSNpfaMRvs3+DVF8EcC7GxF53nGn3CwnToYXcXYv2DAJ3V/VashVCnWlDlluVa9VDpP70yHAe5X6briwRUsuWOJWWLHtMfx9hYdVscLjlNe04beQnJfx4uGqgvgxGfie/T/mdRHWFr1F2U1wXWdfqYOk8bdRilTUojqG2X3pdtj/gxODKbTuE4WnzMazTv9dmaK/eaA6AFtNh63oetjcEWBXQk/gElJkwDYfn57s8fqqnQcXXSUwhPWM3pBtbF6T0r6HfrlEKsfWW3xue9mfMfd/e9mWNwwq+dLJBKzhF+hDWuQzFq18R+Kb2bR0UUsP7Gcfdn7uH/g/QB8tvMzavQ1dcf/euhXqvXVdA/v3kRelE1XN/V0cwYnPodtN9MiqXh7kcqbTfgB9IsV8a4S5VEqq0Xh9bjXxvFwxsMenfADc08/pUzJvQPu5abeNyGTycznT80hIWHdGINWrAWrG0jzHv2g5d7kbUVNkf0SiRWZYi/tztLbyiBImIsyvD7mlJEnusHm/j2Xq/+42lUjc5iyMtDKNYDo8rNZqCNxfTypxVTlwtF3IN9BxQKJmxemp9wD4zeDKoxAr/p7UFF5KUajkWX3LWPbh9tcOEDbVGmqkJeIuMz4uGlMTJpY/xk8/b1jssCItefFpJ8Ho9c76BPnphiN1G2sDL45VOmqkEqkxAbEOnahjteIQJAbdfA4zI47YfVQqGgHQ2pHUAZB2FC8fALrZMJsLd49AZlSxsjnRyIfKqqaFXq1kPcs2AlbbxGB4v9vmAyJ3R11D5iwCeKuJCJQDUC1RIPRaOTYn8dYeP1CCk5Y9uHQ6/WcOnXKLebPtc+tZfn9y8koFGNV6kJEwsU7GmJmgo+Dc6C7UXIcNsyE9BZIepSfg8NvNjVodydKT5otvCJrP4taqYaaGuh7a1+6z3azQpRGHPzlIH/e+ieFWWI+V+gaJf3S5sH84NYnoJVqmLBZFMJkLIZDr4kNb3vgnwjJd0Nw/+aPbUjxIUj7tX0kSNuCxFtg0i6Mvp1Q6NWoJH51CZQF1yxg4XULHb5kW82fR/84yrHFojjgpwM/MegPf3YmXk5srIUlndxbJNwbUpEBS1LaJtDXkKLdsDAazs1r/liFGlTh4NUKr6G+78Lk/eJa/0+QSKBzdBRj96WzdFgZUhsBFXeTMTMajVz5y5VM+mCS2eMN/fwMRj3dP+2O32t+pBWnOXcAunI485Pw5jNRcgI2XAZnm/G8LNgOO+6GbMtB/v9PhIaK3yYpRpOXVMNOPxMGnQGjwb7PYVuvP4vTiqkutb6Or64W0nkAPr6G+qRfBw9K+h39wFze2V68I8EruPnjarki9Qr23rGXP67+o9ljo/pG8Uz1Mwy633qBpSnpV9ROyx5nMObVMdy86WazLj8AuVTOtM7T+GTKJ8jt7Az+fu/3xL8fby6R24CMDCjy3cZfOT8jlUhZdu0y+kX344quVxDlF0V2WTbzDwtpw4ySDP697t8A3NjrxqYB+Y6zRcecs+nzJoxc0va2A8VH4dS3NguMunXoiEzvi0Faw74MUeyk9FN6TDeVNapLq1Hs3IJvUQa+vhDtH81Hkz/is6mfNZ0//7kc1k1qepGctbA4Wchcg7gv7n5QWJ64E9tug5X97Yu9nPoSlnQR+xJ3RRkIQ75HlnAdMqPohM3VlLl4UI7zx81/ULN2I1qZBrAi7Zm3RXhvtqG3bLsQ3B+mnxJ7ZHvJ3yrkM90Z/yTR+CBToZApUMmE8lRRRTESiYT9P+zn+GLbEsquZOgjw9g3/l9IlN34ZcafLJ4jOusxaIUN09H3HLqeXq/n9On280C/mPS7iBlH/zjK5nc2k5elE/rdEijmLAAxATGO67MnXOuesownvxKGp/YEJ/p/An3fd0+dXm0Z4QE5KHRi01RYWYiuSsfOz3e6nZG9PXgFeDHq36PQDRY7YLkuUCT9CncJo+Qaz05q1rHvWdj7ZPPHGQ0iuL/BulSLOxKlFjtpo0RHhbaCLjO6MHvhbEKSrcv4lZa6hw75xPcmMvXzqWRqRNLPmxAR4I6aIIyV1e6dMGqWwK7CCy62BRI7Rftg7+Pi++iu7P4X/DWybm43JaC1Mk1dpaq7k745nT1f70FTpgFA2bjTzy8ROt0MAU7weDIFZkqPwb6nWn89e1EGiU4iRyUgUx8RXWWBHvo99I2D4L74q73pcv413lCX8sLoFwAY9vgwRjw7okWXbYv58+8n/+bvJ/4GIKdMBNYVuqCm3fclx0RhToOKfwB8Y6HbMxB3leUX2DzXOfe2Go1I5KnsSORJJDBxG/R4tvljraEMFEUunlzM1hyFu8U6pfxc3UMdoqV4a2MoyLMeUF523zJeVr5MTXmN1WPaG4lEQszgmCYSf6akX3g4ZJZmojVokUvldPB38lq/aD9suU5I4JqQ+8L5xVDSjDdZ6BAYuRQ6THXumDyN8nMMVD1HauimZjv9NryygZe9XqbwpP37hbZcf654YAVvhb5l1XLB1OUnk0FGxUmKq4tRyVV0C3OChHd7cfBFx7v2TVScF95+dhDiE0KvyF52SbFKZVKkctthLk9M+gXGBhI71DnFhwuOLKC0ppQlx5c0ea60FEpKjRzt8AQAc3vNZUyC8JlUypTc2f9OAD7a/hEA3+z5hpzyHHpG9Kx7zoy+70C/90QxkOaAU8YPiEBy9KXOu541TtZ2FJaftXqIXCZFrRXf213p9cWReUfy+OuJvyjL8bxkC0Bxeglhu1cRmHvcfC9Si9n8mXKfsCVpTGA3SH0Ugmq9JKVKGL9RPOZOdJwDPV+2r3svfCR0fRz8PEC6FFAhFu9ZRRoAMndlsu2jbVQVV7lwVPZx8OeDSM6lobOV9NPsh8ylwjvZk5F7C+UsRwoZ9j4hEu7ujr66LqEe4CU+j2XaEnQ6uO/kfcxZPMfW2S6luro+beDTWFl29IoWzWVlZe13T/h/bhLgfHRVHlp9XsuBnw5w+LfDBE0UlXEhIZBWcgaAeHW8C0fmZDQHIGs5VJ4Hn2ZkMNXd3TfQv24S1waf5Lc1+7hsih9DY33QV+tZetdSus/pTtJEz1iINMZkEK3QB4oAY/IdQtta3nLtZLcic5noqOn9mu3jdOUQNwuCerfLsFrN4TdAFUlo8FwGnFiMQh+EDCUhKb6EpHiGb1fHSzoCkLNAdML5Sj1j3Haj8IPEm1p2bsQomLAVfOOdOSLnkngzRI4TCXOJjCBvNQA6eQklpXr2vreBw78d5s59dyJTuqcMxoR3JjD6xdE8t014pch1avONdsQo8eNMVJGiuMWdvSlMtNaX0pUYjaArR+0vRYIPxcX1T/W8tqfrxmWBKZ9NwaATVe11Hi66iKZJv6Pvwskv4PJckIWZP9frpfo/V+VBxkJIvE0kzGoKQFtMq4kcKzrv2gOjUVR1+8aB4kLTHm+AZj8cehlCBoCvuCcG1zbk2JLCC+0cSvKUZPTVenCT5VpJRgl+UX5IZeYJgNxc8TsiAs5oxD4nLjAOmbPlkQJSYNgvoG7w/faJhlklILfgSdMQpRo6THbueDyRqlwSK1+iW7g3uwqEdGGdp19ZDkajsS4RFNo5lC4zu7Sp2p8jJE9NJigpCIW35aJZc2lP0aHQJ7KP40W2rmT8ppYHWg88D6e+gpnpze/HHeT89vPUlNWQMCbB4vOmOU2jcerLtinleeWoAlVm69fDeYf5af9PXNn1SvpE2WldAkxJnsLi44tZeWolz4x4xuy5zEzIC1hFQcA6lDIlL4x6wez5O/rdwcsbXmZrxla2n9/OsyOeRa1SMzVlKr5KK5O/thSW9gC/BJi0s/USfPoqMNS0z724040QPkr43dogzNidArazP/sgMAuArF1ZbHpjE6GpofS+oXdbj9Tp+ESpOTL0ZrRefvj6Ql55HhKJhGBvC126nS0k/EAUzvd5s/7vUgWEWZehdRlxV9h/bFvsxdqCHfeCdyS+smDKDblka0SVw4mlJ1j373V0HNGRyF6RLh6kbZ4se5LrrzGglYniqSBvC3vV5Dsgfg7I3GTx2Roqs0WjQ2BX+47v+jhU57ftmFpL5nJYNxkGfQ2JN6P2DiS3IgetrJjSUggKc+/37diKMwSfL6OwQ1e8vBrsE6SK9vWwbyEXO/2cTHmeh7QSWGHCOxO4ZcstFGjEhzksDM5qzgItSPrpKmHFQDj8lnMH6Qx6vgBXFtveYFRmQeYKc/1xdyP2CtJkc/HSRVBT7otEIkGuknP9X9cz+qXRrh6dw5zffp7vRn1H6QZRNSY3Jf0AvELazqS7vRmzGqbZ4dGi8Ich30GXB9t6RM7h8Btw+jv8/CREFE8luGwY2moPClw0ILdMfO8D5LVJvyNvw57HXDgiJ2PQOu4JpwgQXpTe7adB7jAxM6DzfXW69g29S3KKSzDoDEikErQV7uvBIFPIUKlVaGo0gAVPv7Yg4Tro8kAbv0gjDr9V25XpgOF6+u/CZ9dTyd8Cv/mTIvkcwCzp527Ej4yn01jhQ2VK+im1FpJ+cbOh16vgFWr9Yvoa2PsYbL8D8jaKx0YtFZXeUOvL5+B8lLYA1k5yfI2WuwF2PyKSkI6iLYFlPWBXO39X2psO02DyQbONbHAwZAT/yOd517LwiGUZ2oH3DuTqRVfjHew+ifnvR3/Pl/2/bPJ4w06/M0Ui6ZegtpwcaBVeIULaLjDV/PHmEn4gvF3cTC7VJQR253zvgyw9fk8TeU+5VF5XKAjQ9cquXLXgKpvKEu1J31v6MuEt6wGhhkm/HbVJP4/z8wvsAuoWdiZ2uhmG/GB34ianLIe5C+fS74t+zUoJL759MYtvW2z1ebVa/C4s9Jyv2YeJH/LT5J/MHttwbgOvbnyVJ/5+wqFrTUyaCMCW9C0UV5kvRtLSDRyNEde7Z8A9xAXGmT0f4RfB1d2FH9hH2z9CIpFw/6D76RRkwbsydyNsuRHKTov4S69XcUpWPnsN/BbYPhKRQb0hdqbYl9ugp/xKumS8Rj+/aXWPpUxL4Zatt9Dr+l5tO8Y2osaooDw4Fq1fECoVPLf2OcLeCuOFdS80f7I19NXu7YV3IZGxEDKXcV/YQkbvP0WkbggAPa7twQ1rbyA40X6JZVchkUoprZTjW53ELT3uYWqyFfUDRYB7+9rZy/ppYn9jL9GXQsL1bTceZ+AbLzppawsJTb5+OlkJpaWgOacha3eWCwdom10fbyF+3x+kxb2Nz6vePLjiQfGEQecRC4iLST8n4xNixybOjQmMDSRmcEzdpqph0s/hzbC2GKqyRTW3u6FUN2/KnP0XrLsUcte3y5BaRJeHOKsWlVMN1RU6je3kETfxxlQWVZJ3KI/KkkpAdPr5+wNZq8xkpjwer+ALY1HSmHHrYeh/kctBWfv1qqiA3IO5vBH8Bhvf2GjxNIlEQmxsrF2SPW2JvkbPK96vsPiOxRRUiHaGQGVt4Chtfss8S9yR/c/BPB+oauqFY5OaIs/xmKxFKVMiN4oAdG5xMWNfHcvdh+5GpXbfAoK8I3nk7M+hqEpUYzbx9Nt+J+x53DWDcyblZ4U0pL0+gtoS+OcK4T3oqfjEQsINSNVdKPHezzz5ZG5YdAMAe7/fy8ddPibngGPfy/aYP03ynl6Wkn6RY6Dbk5blLvXVsLQbbJkLfd+DgV9C+CXmx5SdhTXjIH2+Y4OqPC/WZ6V2FNA0pGAHHH1HfPYcxgjdn4MOM1pwrgfhFSKC+A0KrUJCoNh3J/uN/2Pb+W0uHJz9GPQGus7qSrfZTRMSljr92iTpp6u0/Li2RFgNWPOf0VcJr8otc50/Jk9D7k1AbDcqdQGUlkJNDXjJvdA8rqH8qXLLUl924ur1Z8Ok3y19buH9ie8zq9ssl4ylRejKRQGFI8U7DQkbIoKVdib9AlWBzD88n91ZuzmYa9tfeuS/RzL+rfFWnzfJe1ZXQ5X7K9xhNBrpNbcXnad3Nnt8X/Y+AHpH9HboevHqeFJCUtAb9aw9u9bsuQNpaWhlhagkATx1iWXp9/sG3kfPiJ6MSxhn+4WK9sCZ78Wc1/l+iJ4EujLY/zycX+rQmM3wCoX46827qNsaXYXNp/v6X0pS9hNEGev9qlWBKmIGxXist19xgQ6JQY+fn1jmmYrAIvwizOfPnHWwchBkWfGg3fsE/FVbkH7mB/hFaf1YV6GrFGNsbo+lLRFrWwd9vFzCtOMw7h+6hHbBt6YTlaVCNjI4MZj4UfEo/ZqJh7oYXbWOY6vOoSjXEFQ+hE+nfsx9g+4zP6hgp1hPtZcvfVuTdJuYKy8kAlNh2P+EOgv18p5aWTElJbDs7mV8M+wbV47QJql3juRU31kYvAqp0lUhk9TGcY++B7/6ClsEB5BIJMTEOFfdwBYXk35ORuXvvoFEeyjLLkNfoyevtgA6NBTGdRrHXf3vYlisg2343pEwMw16v+78gToDzQHb1WGhQ2HA5xAysN2G1BL8/CBLvZBfq2/mfwdEUsJoNHqERndjkiYm8WjeoxhHGVEZQlHoQlD7lcPaifZ54HkKNcWi26Ay2/Zx+5+Dnfd5RAUJIHyOartni8OWcyb8Qw5kHkOlVhHZOxL/KMsVklKplJCQEKRS196S9DV6EsYmEJISQmGVKFYI8qpN+o1bD5c6dkN3W/xqPTD0tjevTdh6E/zq1/LgTnuQtkDIB+VtqXtIRRByXSCFpZ7Rib/snmV8P/p7ZnWdxTjVE6grBpon/bJWQf5ml43PafT/CC7PFgkGe5DIYejPkHRr246rLfGNhSHfIYudjF5aSYb3cjacE+brJg8iR2Xi22L+rC6t5mXVyyy/fznQjLynLWRewoPSO1oUW1l67+S+ULRXeIY2h64CSk+JPyfeKiQSQwc7MCAgYS5MO+n4eSD+DT1fEBX/Fzo1GrNiq6AgUNUIv7uMkgyLpxSeLGTlwytJ25jWHiNsFqlMythXxzL8ieFNnjN1+jVM+lnsUmkNRqNI3G28uulzldmw/TY48bnlc3Xl4jMe4XmqHW2Bn6KQ6EDxuTL5+gWqApsk62rKa1jx0Ar2fLvHruu25fpz1SOr+GHsD9SUWfe4bJj06xHRgwcGP8DwuKafV7fl/FL4PRzSfmvddbT2eduo5CpGJ4jvxPKTy20em3pZKqmXp1p9XqUC79qmZE/w9ZNIJEz+eDKD7h9k9vi+nNqkX2Rvh685oZPoQl15cqXZ49U58Yw6eJwXk1YT6mO5i39AhwHsvWMvN/S+wfaLdL4PriyCkAbj1lfCwReE3UVLCR0IQ3+A4L4tv4YjrBwsklo2MK2PGqs4VORXUHDcDYvg7WD3x5vpt+xl1FUiZmHyUY30izSfP2s0wrPR2h6xMgsq0oTyg0+csC+p7fpxG2Qqse5pLnlUnS86FfVWinrcCbkvSGUE1grfNPxsGg1GdNXubU1VmlnKvEnfEX52BwpFfVG5GWd+EOupqtx2H1+bkHS78LC3hxqNWGfuf74NB+R83hr/FleXbyK8eBKlpdDrhl6MemFUsx38rsInuQPFkZ3RK8U8Xidv7BMr/D29oxy6nlQqJTi4/Rp0Lib9nIxO594Tpy2MBiPvxrzL/NnzzTr9ZnWbxadTPmV8ovVqOY/k0GsiiF1txaDEP1HoQzvZY8CpnP0f3YsuQx64kQPyb9mUtgmAhdct5A31G+i1nmlm+8GlHzD7XB4JefeKYPfAL6FTC33I3JGctULWLmul7eMyl4uOUxd3wNmNrgLKRVDmWPB7HIp7gG3ntxEQE8ANa26g11zL0iZ6vZ6jR4+i17v286r0U3LNkmsY+q+h9FXOJjHrCTr711ZryrxAFe7S8TmNTnNh5J/CKNoRwkeLIGRrfTjaGkO1+KnlMXk6k/ZqCJd0o+B4Ads/2Y7mrMZ142uGfrf3Y+S/R3JVt6sYY3iNoPJGSb8Zp2HMXy4bn9Nw9HMk94H4qyF8RNuMpx1Rq0GhUwOgqdIA0PO6ntx98G46DOjg0LXaYv406Awkjk8kOFlsSOo7/cLNk341xfBHvO3uy5F/Qr93rT+vChNy171ebn5gZ3+ExcmQsVh8HqQtkI9WhYn1nfSirblNlvUUnbW1BAeDSivWw9aSfuV55Wx9bysZWy0/7y5UV9f7eJnJewY5udNPVw7RkyHYgs9WQApc8jv0tfLd8AqBgZ8Ln9qLIFk9mOdGTAHqk36WkHvJ2f7hdk4ssa8DuC3Xn1XFVZScL7HZSdEw6eeR+MaJAGVgC+U9AXb/C+arRXerHVyadCkAK06uaPlr1mLq9rPlVerOGIwG9ucIT9tekY7LR5okPleeWmkWbM3MBJnRizGdbRc+290h21hhSRksJKS7PwdV+aJgz+DmMbSw4RA6xGYhrloN5V6nWJe7gMzSzLrHF1yzgC/6f9EOg3Q+yg6hFEZ1wytETFJ1nX6+EebzZ+xMuOw8RE+0fKHB38H0U+JzED0Rhv8q7oPuhEQi9liDmnmv/DqJDrpulrtg3YqSY5C3mWPavznS4Ul2lS8CoOBEAS8pXmL9i26saAao1CoGPDcJTWRn5IG55JXnoWs8V3R/Fi5ZAP5u9nlqD/SVQjqzGelhl2M0CIuHYx8C0CeqD6l+Q1HqQygthW5XdWPYY8NcrvpljYraOnmdQiwW6pJ+8VfD6OUOJ/30ej3Hjx935hBt4uaRO89j73d7XT2EFqOv0dP/zv4kXZpU1+kXZtuv2DbFR+DMf6HSQQm59qLzAzBquXVfDTetNDCj5CiBpX8SKhELaZMcXNyIOHrf3NvhjgFXk3Mgh0O/HaJKU1UnV+of5Cu6A6IuoKRzcD8hddZcF+nE7TDBgzp6tt0Kf3QEfTW+UrGTzi/T2HVqlZtp63QzXk3q+dfoEdJfbETztzbfmXmh0+UBGPiZq0dhm7grxEasgbm6v59Y6pSVCd/Q5fcuJ3uv+76X3a/uXlfJbZoHmwQEZV7tO6i2QFcuAj352+073hPuyfaw/Q58zryCSqIGoLiqGEMru2edPX96B3kzZ/EcBt03CKPRyBWpVxBTPg0vbbR50k9bLIJ3klYm0LwaVDvaep/VPSHuKrPvd4uozK7vGHSE45/CmglQcb51r+8JJN8p/DdqUavBuzbpl6axnNSL6hPFA2cfYOC97qGQ8cO4H1j5r6bFVSZpTx8f8PWFOd3ncFvf2+gV4WTPJYUfDPsJulqRCou9DJSBlp+7iDmJt7K/THjWmJJ+3+/9nuk/T+eHfT/UHSaVS3nw3INc8csVlq5ikbZaf07/cjr3HLnH5jGme3yOchPf7vmWU4UOzkvl56DsjOvuj6GDYeB/QN295dfwTYAO0+3u9puUJLyONqZtpLS61OpxB+cd5J3odzi77qzVY0xJP1MRgDuTvS+b3676zezfc7roNOXacrxkXqSEOB7wHhU/ipSQFCYkTqBaX41Wr+XnA7+QkSniB9HRThh4zlooaRTclMqFhLR3BBz/CDZeCYW7HLvuP7OEN3R70fdtkQyyEZQODIR98TfzdemVrDmzpu7x7nO6M/ihwW7bxWKLgEFdOd3vSnzCxUakYacfODB/umkw/4Jn10OwZjzHKjZyKup1jhtEsYRvmC+dZ3QmLLU1wd62xzvIm5grBlEWHMfOyDsJfzucr3Z/ZX6QKgxiL79wPmN5m2DVMFH83xzeUSJWmPqvth9Xa5BI4cyPZv8m/9o8Zan127jb8M/sj0nZ8gM1skZJv1bQnrHPi0k/J+MX4amleiBXyZn88WT63d6vbkPsF1TB/pz9lFSXOH7BzOXCi6LspHMH6ixCBwld+QaeJWb8mQCb3dxLo/tz5I3Vcb5UyJcUVoqJqP8d/Znx9Qy8/D0rMHz4t8PMv2o+BWdLqalVw3FISsxT8I2FLg8KfWtbSCSgDGqXITmF6Euh80Ng1OErVwNQVKkB4J/X/mH7J3YG911E/rF8Vj26ivM7ztctQAICEN6kq4Z4tpdYYw68CAdecvUo2gVfX/G7ogISxiYw9++5xA2Pc+2g7GBf9j7OV5zCgK4+6VeZJeS0LgQJE22ZCPSc/ta+43fcCYviQOsBuwNbZCxCkrWSEF8R7DdipLS6lOqSarZ/sp2z68+6dnyNkEgkfDHtS/qf+BOlPqhOIggQXR6X7oauj7b+hYqPwoqBcNJGhXXoYBj+S+srWpd2hW0t6KCqyBDSutaKxS4kuj1lFkSQyyFMKZJ+mWXnLQYv5So56o5q5CrXd1FqK7WU55RTmd9Ufsu0xwkPF8usuwbcxRfTviA1rJk1mSMYDfYlYqryLPsa7bgbdj7gvPF4Ol0f47DxMaA+6Xck/wiLjy9mV6Z5siAgJgCZwj18s5urWjd1+u3S/8DNf97MF7sc7Aba9RAsTgI8L5lQR+d7YcTvoLIsI9mYpOAkEoMS0Rq0ZomVxqjUKtQd1WDjLTAl/TxB3lNzVsPh3w5TmlW/BjL5+XUP7468Bd3rfko/jt17jM+nfo5KruK/+//LNb/PYUPCaGRyI+GtFTgxGmHdVGFVYY24WSJx7Ij6iK5SqOVo7JAFb0cCAsC/UiTAG3pO9rmpD6NfGO22XSy2KK91RvDzg7KaMiq0ouUlwi/C/MC0+ZD+u/UL1RSJZoCCnXDgBeHx545U5sCRt21Lzmf8Aae/B4MHKGol3Qa93yAiQCQpyg1islOpVcz+fTY9r2tHT8wWYrpP6hUaAHMf37Iz7ttg0lKMRuFXfqF4FJq4LANGLgZgT9YetkneJydwGaWlcOB/B/hy4JfkHc5z8SAto+oUTWVAONXSRkm/nffZ3re6CReTfk4meXKyq4fQampq6vWez+v30OvzXvT8rAU3hNjLhHRNYFfnDtCZGI2Wu3cMWvDvLHwJ3RmpHF8/CQqd2LUUVHioPkkt3a/uzhW/XMFNm29kS8poqnxOotpzAyztLt6TCw1bASHNAchc2axpuFuRcL2QcZP7EqAQn8nCSrFg2fX5LvZ9516bs8YUHCtgy9tbyD+Sz7HSHZQrT+PjawCZD/R+E2Kmu3qIziN9Ppz53v7ja4phw+Vw5qe2G5Mz0JaKxVfuP3UP/aP9kG3JE1lf8Av+Uf4kjEnAJ9R9g/bfj/6eZfcso98X/fghKIlqRXZ90i9nPayfCrnuLcdiF6pwGPIjpNjuhKjDO1r4f8g9t7gKgOlnYNx6ggNUSA2i6KioqghthZbl9y7n8G+HXTxAyDucx6pHVpG5S8hTVVaCSb3ev60UZHyiofwsaDWWny8767zXSn0E4q9z/Lzer8KsUlConTcWD6JDoGj7qNZX1RWZNaY4rZjCk65fiyq8Fdx14C6mf9P0vt3Qz6/NOL8Ulqaa+cta5J/L4Z/Lmq71cjdA4c62G58HElqbEzIl/aL9xecxsyzT7LiynDLSN6dj0LvOf3j/j/vZ+v7WZhVXTMHMc1rxXveP7u/YC8VdCd2erpfLbs9OIqMB/hoNR99rv9esxSTxuerUKqvHJE1M4pYttxA/Mt7qMZ4k79llRhee1T5L1yvr4yqt8fNrjFav5eUNQmY7UnMZKckSZK3NnRv10P9DSLnb+jHq7kIiVuVAx5HcG2YVw8B2DLbWFMH2u+DkV1YPUastJ/08mdNf/EX0sbX4+NR3+fkqfPFTNlqLH3jediKvukA0A6T9KjxA0xe23aBbQ0U67HkUztrY7x77AHbd7/52FyDisZ3vJap2squSFNYV1nsCZ9aeYeO1nxGYcxytTAM0Svrt/zcsjBIFVBcKYcPgilyIv6b5Y/O3w6HXoTy97cfVWpRBddYKa86sYV7xQ2QG/4+SEtBWaCnLLrPpgexKIu++nPRuk6hELBZCfEKEP+nxT4QVk5vj+lLMC4y2MAJvL879c47tH24n5eahQAeUSsjVngVa6HPhlyB+3JlNs+H8ErFwbOgNI1XAmGb81tyB6gJ8SvYSpRAbW1PSL/9oPhtf30iPa3qQOCHRlSN0iLCuYYR1DWPzKxsoDyjD18+IRBkIXqEt8+5xZ3beDxmLYMY5y3IEp74Wi8rpp93/e2SBQC816KC41q9q7pq5Vn1NpFIpnTp1cvn8mTghkftP34/eT8/3Z8KgJ7ziWy6k55zRyeJOjFoOXg5ssKuy4fwfrfNtaQ905UIzPvluCL8EgDzDUfICV5FWMQQAo9GIQWdwm06AxlTkV1CuKUcfLipIFXp1fdIvdBAM+gpCBrlugM5CIoGEa+0/vse/xY+nU9slFhgICl0Q1cosNFUa4kLjmPv3XII6Odbd3RbzZ96RPLa8s4WoflEE9Qwiq0CPEV9UXhK8GgoIZC4X1aidbhZShq1BEQCXZYHUwvey9CQsThHdZ/Z4/zVHa3xYPLBSv0Xk/iOq8bs/CxEjAQgP9kKpDUOnKCCrLEtsehvx1eCvUHdUc8uWW9p7xBaRypp+L0ydfhERkFeeR055DgnqBHyVvs574eo84RnuG2v7uK6Pi6KaxsHDKQc9q+irrUn/ncmqz9nk9wn5+aLANspPeKhklWaZHbr+hfXs/Gwn/8r6F36Rtueltlp/7vxsJ3mH8+qkuq1RVgZ6SRXnqoQv24AOA+x7geLD4nMTe6XwyDIaYcddIjExfF5rh28fVblQfAiCWimLqy2B/c9DUG/hOW0HoxNGsytrF7GBzXy/msGTOv1AyNc25JkRz3BF6hV4yVun7KMz6LhzyZ2c0ZzBTxJOx7w76TGyVZcUSOWQaOe9QFsm1Jfs7ViUSNq3617mC6e+Et+xpFstHhIYaDnpd2rVKTa+vpGxr40lZlBMuwzXWWg2HiRQ54uv72hUchX3DbyvTpLebP4c9LXwF7OGT5xoBgjqBb1eEd97dyS4H4xaBhGjrR/T/1OozPSo9WBkoOhM0soKKSkRRTSb3tpEWVYZE9+14sPoBhh0BvQ1Yj9cLdGAsVHSL2aGaNJwpGjA3XHkc5W7DvY9CRFjml9vupqy06IrM2wIgSohG6OVFVNaCn3v70vfW/u6eIDWMXn6JauG4RV6nnDfcLH2mlVse96zglQqJT4+3rmDtMHFpJ+T2f31bkY/bOMm4cYUnizk8PzDqMeKL1xYGJzTnAUgXh3v+AWNBvevgIkcJwLfujLPklE0kb8FyfppjIx6lT+pl/esKath3/f7COsa5lFJPwC9QU+5TpS+hvioRYXghYgiQEiZaIuFwXljOt0kDIl9O7b70FpM3mY48iakPopaFQTlUKwVO+ngROva1xKJhAA30HGVq+QEJQRxTnMOAKlBRZjafTvCWoVPB8eOD+gMs2vA6OYdt16hMGoF+NfPe+rahWWpVkNxejEfxH/AoAcHMfEd99zk3HXgLjJLM+FdkBhlyAy+9Uk/vwTwc49gutMw6EAi86jNc6soOQGVmQQEjERZEIxEWkpZTRlSuZSEMY4XeLTF/JkyJYUHzj6Ad7A3Px/6mZv+uImIpGlMLv7T/MCzP4kfe4N6zWEp4QdiLRl/rVizuZJT30Bgdwh1D8+6NkVfBQXboLLevzA4GEauOcgNVwXTPdzyFnLwQ4PxCnC9tPzmdzaj7qg264oxYer0Cw+H34/8zp1L72RqylQWz1nsvAEk3gwJc5sPYHeYav25/w8ysvZSlUOwYStBqhzy8mqTfv61Sb8y86Rf6uWpBHUKQubVfGFPW60/r1l2DYUnC5FIm5f3LPHZh96oI9QnlI6Bdq75j38MJz6DmengEyPun5WZottBXyOCUW2Nd6ToRmitEovUC469L7oW7Uz6XZ56OZenXm7zGKPRyOa3N+MX6Uev6y0nJoNrtyae4OmXezCXKk0VHQZ1qCtaU8qU9IpsvRdpykcpnNGcASC14HFkBh969Gj1Ze3n1New/U4YuxbChzd/fP42kTQKH9k+n3UQr3PZeZsFk0LeUxRHnis+R0l1CQFeAVSXVJO9J5uKPM8r5FA/dz87l9XQ1w9iAmL48NL6uIzZ/BnaTDGiTCm6zkx4NS0acgskEmFXYovALuLHEzj+KZz4jMiUFwDQyosoLhZJv1MrTpG5K9Otk36J4xOJfvVeNs+HKkRMySzpF3eF+LnQyN0gVL+aU8PpdLNIULuzsp6JXQ9B5jK4uqbuPdTVJv3cGW2lltwf16IujuOa2HnMmdPgSYV/iywn2jv26eYZGc/jyPwjrh5Ci+lzUx+e1T6LpJMIOoWFwVlT0i8w3vEL/jUClrj5BJR0Owz4pGnC7/wS0S5eXeCacdmLuicM+JScatHRUlKjwWA0ENk7kidKnmDoo0NdPEDH+O2q33i/4/t1fw/xC7R+sKfT62UYt85ywg9EJVzK3e6fOG9ITaH47pSnEeStBqCsVqqtqriKghMFFn2A9Ho9Bw4cQK93rTZ+eV45xWnF5JfmA6ILJyAAodu/chBoDrl0fE5FXyUkIcrP2X+OVGbdA9VdkMoheiL4J9U9FOSjBqBMp0GlVtF5RmfCu7fWqKRtKa4SGttyfQASJHW+hBcce5+EeSrRSWqL8nOw68HmpfI8gX1Pwt+jCFLrGXFoH5/HlTI8TgS5jEajw9ImbTF/mrzZvPy9yCkTGZK6+bAhvV+HcetB7sQP6JF3YdfD5o/5dYKh/4WIUc55jdKTsGqoCIbYS00xbLsFTn7mnDG4O5FjYVaJmbxQcDB46cIp0VhPZA17dBj973BQotDJGPQG1j6zlt1f7bb4fEN5T1OgO0FtZ8L91Ndw5B3x56pc0c1nDUc8tiqzRac6QO5GyPhT3KcvIki6k3ODSjicP7yuU7NO3rM002xt2WlcJ4Y+MhTvIO9mL9tW609VoIroftHNHldWBsU+QtpzQPQA+z2/4q8TsvPeDQq4hv0CE7e0XxLERGuVWGReMOMsDHWufLxEImHjaxvZ8/Ueq8eo1eK3J8h7bnpzE99e8m1d14szMX2XAELP3olcDl2s5TSqC+yX2z7yDizpAiXHbR8X2B2iJtk/Zx77ANZOAKNt+Vynowq3WaDm7w9ehmC8akRBwuE8Idfe9cquPF70OClTU9plmM6kvFKKXqGyuA+pmz+1NaKAzx6qCyBvk/v7sJWeFPfixuir3LdL0RJGHRhqCFKKIqIaWWGdjdOs32bxSPYjLhycfZSVgREDlUYxcLOk34XKif/AzntFB7QtVKEQMsC5+7C2otNN0OdtMBrqPPG0ctF5WpFfwb7/7iNnv/vNC9XF1VT+vQX//NP4NKzFq8iEor0tWqvr9XoOHWq/uKIHRZM9g5k/znT1EFqFVC4lv1B8LMLCGmyGbcl7ag4KmZHGBPeH0CFtMcy2J+MPOPhi+3ojtATfOEi+C51kKGP3pfH32FIkSJDKpXj5e3mcYXRQpyACU0WiT2pQERQgF5IvmctdOq52x6DzDHPoxkRPhqu1ED+HfqGXMODEYiYivD5WPrSSj1M+thrQdnXCD2Dz25t5v+P7pB0TCQilPlh0WGlLoeK8YwE8d0dzEFYNEqbq9lByTPjJ6RyXMHAJ+uq6P4b4qgEo12vw8vdi9u+z6XNTHxcNzDY15TXs/mo3advTACHt6e1Nva/KmgmwapjrBuhs1D2gw/TmZeyK9osgT4nr/e5aTaebof8nBAYYkCCr23wDfDP0Gz7p+onDl3T2/FmeW47mrAaDzkBOudiAKXURTZN+PjEQPsKpr032ajj7Y/13OHej89diXqFiTtMWN3+sCZmXkHxKttOD0tORSJsEN01dMQVuXg8nkUq468BdTHhngsXnTUmj8HAHk376GjEPHf9E7HuWpMKuBxodUw0r+sPJL+0fcMafwpMm+2/x95w1sGGm8MO6iEAiIay2waa4WPjPm+Q9q3RVFFc78F1uhLPnz5z9ORSeKrRY5NaYsjLQ+NYn/ewmbKiQnW/4HW3vztD0hcK70hn4xrUoeajVa6nRWy+UuWHtDVz+k/WOQE+S9+x9Y28mvj8RhY/4f9qdtZub/7iZH/b90OprvzX+LWICYngy+X/IDD4kJ4PKUo1f0X5YEApbrrfvwlIFIAWldbUXQHSJjVoMoYPtu27n+4WcZHt/5muKIHOF2BNaQCo1dftdGL5+2kot5UfSUFSV4usLOWU55Jbnom8Qo9Dr9VC4A35RwrGPbF9w+13i87N6OJz8TxuPvhUYDaIwbMedTZ/L/gt+C4TTrf/etQud74dpx/ALFUpuOnkxhRrx/nkHeyNXuXdsI3NnJpq/d4IuFyPinlqX9CvYCSsGirjthUbXR2H8RrH3sEVVvudIwcfOhC4PgFRWl/SrkRVSWgrF6cUsmruIY38ec+0YLeAT5oPkoQfITB6BStVgXXf2R1jeRyT+WkB7xj4vJv2cjNSn+apCd+X8jvOkbUwjXzS5EBraoNPPlrzn3idgaTfht9Kwaqff+zD467YarvM4/CasvsQ8qNT3Pbh0r/tKDzTC30+KtzYWfZVvXaIve182mbsymznTvRj3+jgG/CA2vQp9IGGBRXDwBeF9d6FhNIqOhmMfN30uZw385g/n2smXw1k0CBLGBUUTUTyVwAqRXEmenMywJ9w7WdFxREcGPzyYfInQGVDqg/H2BjrfC5dlCInLC4WALtDzZYi0HBRtwskv4e9RzXdkuQMrB8PS+i7zMH81AJVGjWvG4wBl2WUsvm0x5/4UHZhyfWC9tCeAd5Tj0qzuTPw1MOJ3MzlWi0RPhulnINa2nJdH0GEypNxNgFoE7hrKiiVdmkTKNNdXgm9+ezMfJHxAcXpxXdLPS9so6Wc0QGWW+O1MBn0lJOtkXpC1Cv66BA6+5NzXUKrhinzo9qT958hUQvIpxLVdbO1KzjpIW1D31+BgyPdfw4+V1/DGxjcsnrLry118PfRryrKbqU5uQyQSCcFJwYR3a9rRXVVFXaI9IgLOFNlR3GhCphTy0eM3QkCq8JKJmW5+TKmQ73WoEyBsuJjb1SJQTKebIGamZ1Rutxf6avyKl9I1UiTI8vLAW+FNoFcgPgof8ivy6w4tySjhq0FfseVd13SGr35sNZ92+7TZjiyjUST9ylRCJahHhJ16ikaD9UKI0lOwaQ6cX+bIkFvGvqebJr1bSnWBsAhoULDVHDN+mYH3K96sOLnC6jGRvSIJ6GBdQsuU9CsuBoOTb2XOJmFMAoMfGFy3z9+YtpFv937LgiMLmjmzeYbEDiH9oXRCs4VumVVpz9Pfit/2Fvt0vh+mHhbdKM4kdLCQUG5v8jbDukttJhkCAqBTzr94a8BvTE6eDIBeq+foH0dJ35zeXiN1CpozGmTff0vYuZ34+sIza54h4u0IXv3nVfMD5T5ife7fzPo1pD9EjIWuT0KUnftPVyCRQp+3oPcbTedarzCIv77+fu0hhHiHcL9qJ2P2n6a0RMwhlYWVZGzLoKrYfVUFji89jm7hUhTV5cyIvoe5veaiktdWJGiLoSrLofuGxxDUG8KGNV8Ms34aLE6yfYwb0rDTr7TMSFBiMLMXzab71e73vZLKpNT4qMkN2cW0bd6M+Lb2/hc+Ano8D/7JLh2fPbh3at8DObQun4jZao/rsAJY89Qazm8/T+WDTwAQHKon7ZToNrCZ9Ov2FGj2CR+BygzwjmiH0TqRymwhHVaVLTbdAAq/1huTtwfVhbB6GBOjrmQXL5lpIs+bOQ9VkIo7dt/huvG1AFO1rlwfiJdfIEw+cGEGPiQSOPm5WFh2vtf8OalSBIF8410ytBZj0AoNcq9QfH3F96e8Vq2q65VdLXrruBMpU1JImZLCyytE9aG3JPjCtRlT+EH3p+0/Pm6WqMQ2zZHuTORYM8m18AA1AFVoAFj3/DqQwKh/j2r3oTWHX6Qf1664ls3Vm2GPhaTfkO9dNjaXIpWBX7yrR+FUAgPhfPD/OKT9iZDtk7ln4D2MfG6kq4cFQMeRHTEajPiE+tTJezZJ+lVmw6IOkHKfc713Gya1g/pA0p3Q6UbnXd+Eo5O7vkp4T12wNwUL7H0cKjLq/FKCg6FSmc4x5c/8fSafx4c/3uSUyoJKNGc0VGmq8Iv0a/J8e1CSUYLCR4F3cNNCTJO0p6+v+DlVdAqwo9OvphiUgeDTQLJx8Df1fy7aJ/YN6u4w45z9UmcAXsEws0HniG+sKIa4SD2GaiQbpjKj600czv6GvDzo0AEyHs7AV+Frtu+WKqSUZZehq2pn6b9a+t/Zn07jOyH3sh1mqawUiaa+p+fx6Gsn6dehp30vkLkctt4oOp0aJ51lXpD2m5BE7jC5Zf8Aexn8HdRK+LeaYx+I4o7J+4UCgB2o5Cr0Rj0nC09aPUZXraMsu4yADgFI5U1r3QMDxZRuNIoCnOBmGtLcib3ZewHoFeGceIXRCAcOiD9bTfolzIWAFEi8zSmvaUbeFjjwPPR4TgS7rWFKwrjiXhw2FPp9BFHWPdDUaghPn8hAP4huYPU0b+Y8ul3VjdihsW0/TifhE+pDyZAJFEtjRKdfbRFYpF+k+YHqnnDJ/OYvmHiL8zyg25pON1h+PHRQ8/6F7kTZWchaiSxiNN2C+nG6Bkpra5IO/XqIpXct5bpV15E4vpniSxfR+4beLN0fh6Eihmd6f8yAhg3xkWNFkeCFikEnZN+VNuyOYqYLRSpPIG0BHHoZBn5BsH8qAAZpNTWGSowKH7rMcE+fTG2FFm1eOXry0Rqr0ZnW96GD7e9OdzEXk35O5q853zBi+ot10guexOCHB1OWVcaXooiSoBAtL41+ibOas2Za700IGyoq8CWy+gVYdQHsf04Y1Ddnhutqer8Gfd+pH7tBK2R7/JPcP9kkU4FEikKp4HTEe7x3+gAds+6lb1RfRjw3wu1b9huz4ZUNpMvT8SUUpTYMP3+5x1VSOcSIPywnUSJGOc+7qD0x1MCacZAwF2XgV2QEzyNfUojOcDdyG9KYUqmUzp07I5W6R/N5TolIGPlJa3f/Z38WVYwxM1w4KhfjSZucXq+Y/dWU9DOgQ6eDw/MPY9AZ3DLpp/RVkjQxibLsMq4teIL9/3Sk8d76giN9IRz/GIbNs14NXrBTJJ1V7u3FaBfnfoWDLxAR8Q0VXmc4I1/G3uzmvZ+s0Rbzp6kAAuqDPF66CAIb7jslEki+u23uVWVn4dzPwsB+YBt56JWdhXO/iDWqPUVeO+8T3fczzzX1gb5Q6fWqWfIqOBi8a2IAyCjJsHjK8CeGM/yJ4e0yPGusfmw1B38+yBMlT+DlXy+NVFwMH3wg/hwXB/kV+RRWivt9coiNSl1dJfweLpLPAy1IkhXuhhX9YNw/ED5cVGY7KlX4/ymZ3BLk/jDoGw7+KvYEeXniYT9l08SyX4QfD5570K7LtsX82WWmfYGrstpm2EBjRyaldLT/BSRS8EsElYXFgU8MzEgTyWmjUQQMFW2UfA8d6LxrRV0q3mOl/Qo7iUEiSG0r6bf2ubVsfnMz9x6/l5DkpteWSkWSpqjI/ZN+Xw74krBuYcz8biYA+3L2Ac5L+uXkQH4+tv38gvuIH3sw6ODAvyF8FESNt++c3HVQcpXtpF/ZaVjeG7o/C10fs++6zkIZ1LRQtxGmdVJD6XaZQsbl/7ucoE6etXbwDfclp9MQyovBzw+yy4TaS4SfKPB3t/17m1CeLgpxPBXNASFTOvh7AgLEut702YwdFsu4N8YRnOi+E586Xo0mWI2hBvMi2AsdXTnMD4a4q4SnuTUcUSxxNUa9SFDqK/FV+LJkzhLeeikYiUFJaSl4ewtve3drnkrfnI7PF/8laogaukCIT+uVAKVSKcnJ7dch6FkZAQ8gdvZQuzT83ZHkS5MxGiGv1j4tJlLF4x2aVvH+H3vnHd7EsfXhV11uktw7xsZg0zsEAqEmQEhCIEB67ze93iQ3PfnSe256uamkkE4CBEInEHrvYBv33mRbVv/+GFu2sNxAtmSj93n82NqdXZ2xVrOzc875HQfmSqg4KCb9Jy7o12TDkXdBHen9Tr8TtZL1x8Rkst9DwiHozcj9YeY+Di+AAu0U9let5FDx2QyLHua1NataYsPLG4gZEcN9Zxax/RDopleD0SikuCTdcEKp7etpC9yLzB9Gvgfa/gQaJOxMEjUfymsvo2ZHDWufWcu4h8bRc2LPJocqlcpONrYpS+5cgrHCSJ+LxpKc9zCpoUL/nl2PiAe97ub02/eCcLacu1tkGnRTxieNZOZWCxJk1NTANauvQe7n3dOfIVFDuCxyCBVFEFiv2lGVAcc+gfjZEDLMk+a5F2MRFG8U9frULiSj7HZYMVn0eerqTjfP7UikYLcR5G9CYRHfu9Ja4Xg4tvwYu7/czYQnJrTrIbwjx8+GTL8I50w/v2gY2f76g20ib4kYdyMnd1ywQU0m7HpYOFva4vTT9oOICaDQdYw93kjUFKeXOh2ozSITM7vSdU0jbyB5RjKBUYFODr/8fHj8ccjLE/JrN90EB4sPApCgTcBf0UJtKHMl9LhY1Cp3RfpXkHh1114Y9HYkEuh1Lea6Mai+LqM7cOf4abPakMra9rxS7/Rr90JmzIyWn63rs1ErD8GSwTD0FUi5o51v0gpmPcj83FfrOnyM+GkHySFiclSfreuKnhN7YjVZUfg174QPDhZOP2+v6ydTyZApRYFni83CvsJ9gJgvuoP6LL9m6/nVr2+VbIIdDwiJxpaySWuyYd9zot5UW5x+oaNgbjnIWymVY7eI7Ao/D0rd19eRlDUdOzQasEprWJHzJ7lbC7l5hFBcGnhpG+V7vYx6xZ6AACiqEdEW4f7hjv1KpVLU56vYLyQxW6pBZtbDwrpB/MIc58x5b2TnQ7D/RRFIUX9/33iNSAwY8KhHTWszYWfA5OWgHcBW8xcciD1Az/LLgQFEDowkcqB3K7TZrDZqqiVYpDUYpDWYrToUsrrxPG+5eIbscbFQhOlOyAOEylKoG4NrPE3CfPEDSICZfWayUAYlQGUlfDnkVaKHRXPZ75d51MwT0cRpqBo0hqLw7UCDNCnLxorngpNUu+nMtc9uuIruWXQXjEMZ4PnF65OlulrUuwBR069Fjn0Ky0ZD1i/idf5fQlIEQDsAZudBn5ajobyG4k2iNoHNAoogGPBEi9IN3kZgICgsInqsxFDiYWtOnpu338wFH1/gkCntyXfwY6gomt0dsVmh8rBYzHdss4h6HF2tnh/UZX7cAhHj0QbJkVvExL6kphRjpZHM9Znoc5tKENhsNvbs2YPNwwU18nfmk7s1l17yCaTmPMeowLlix9gFMNyN8nXegsxPLNybK1pv+3s/2HRDx9vkDnKXCltrRCaKQi7F3088DFRXC7kab71PH/zlIM+qn2Xvd3ubLgiW7xGyGOW7PWZfh9DzSrHQ01yNGJsZBj0DyTd1qlkdRo+5cN4B/BLGo6xz+pVUC6df6dFSdn+5m/L08jafriPGzyV3LuHnK3/Gbrczp+8celnOR22OdXb6dSQJl4g6ju3NlmoPISNFZlZb56mp98DERadnRlbdYq9MBpF1C616UyWVxqZ162qKa9j77V6KDhR1qomNGXzlYKa91jCHz8qCBx4QDr+ICHjpJUhOhjhNHM9PeZ7bRt7W8gn9ImHsF5DcjKTd8NdgzGcQ0I5sLR8nRXjdWnN9pt+P+3/kgm8u4PWNrzu12//jfvb/sL/V87l7/Pzx0h95b9B7WM0t1/MD4fQr99/K/sjH+ePwH255fyesBoicJGo4u5u9T8P3AVCV7v5zt5F6p19LmX69Z/Rm+uvT0cS1XtfP251+162/jvM/PB+AQ8WHMFqNBCoD21aPtA20Ku1pyBH15ve/ABX7oLag5RP6x8LM/ZB6d9sMkMpad/iBqK8+eTkkXt6287qb7F/hBy3kuq6bqdMJp9+bBXO45Y9bMFq6br2xze9vp/fKD1Dri4TTr7rO6RcgBuL68ZOc34XjT9rKs5WskTfZ29W0QAR69bpeOJpB1FPN+Q1KtnjWrvagDoeoqeAXxeryzzkW/QLpNV3nOfLHS3+k97fPkqf9nuHfRnDBt40krQ+9Cf9c1z0TAwDGftVywE5tMaydDRkLOs8mNxNUJ4Gs10OPcT2IGuJ98kZhqWGUDD+HguhyAELUIWIN11IF1pqTOqfNZmPfvn1utLJlvDvUvQtSXd01FwNqK2p5b8B7JM4dCkwkKAgKajOpKK8gQZeARuVishx9jqjlEjNdvN75kKjx0mOemLj5ed+XtllyFolotNhZInNx0JOetqjtpH9Jb5kNpVWkGpfUCKff5nc2s/ntzVzx5xXoEnQeNLDt1Mte1MsOSLXJEHSjiKjqjlRnwO8pkHI3DK9bsKhKFw6/gJ6QcLEHjTs1AgJAaQ3FIq8kr7yECdPP4JGqRzxtVotcu/ZaAL79Vryun4i0N/q4y5B6l/hpDZtVZALKu4iuRsU+kRGXdL2QukJcjwaDWGTT5+nR5+iJGhrV5oj8zsI/zJ+kqUkYtAbSMo9hkUYSWO/1i5oKM/eJDPruRGuLPDJl267TLoa/P6jtwulXVOf0G3LNEIZcPcTjEvGFewupLqhGIpHw/nnvc+sfkG3B2el36C0o/kfIHSqCmj3XSaEMhokdsAjeGLmfkGL00TxH3hMZHZNXOqT8IoODkFu0WOQV5FTmoAl3fj4oPVbKj5f+yNSXphLeN9zVWTudn38W0n09e8JTTzVI+PXU9eShcQ950jQf7WHTjVxo+4UvKaSoSDxvZ1ZksujwoiaZmiv/sxKpXNrptaSVAUoi+kcgU7SedVBVBSVBazgQ+Axf7r6YmX1mtv4GVhNsvV0oT8S20j5kKEzqoKBJTT+Ing7+PdxzPrsdVkwUsqWNa2W2QL3TL6M8A5PVhNJF1lVbqHf6lZa23M6bqJf2HBQ5CKkbFrzbVM/PYoCwMRA3G876pfWTShXtV7SpPi5qVsbM8N4giqA+ED6+WaeVVgsKSyhy1FioJbsym14hvfj1ul85+PNBHix90Ovk65qjRm9BbqkVOrhyA9VmkfbXONMPwHbmD8gsZa0HRUkVMKcAVGFdw1FzYla1RAoXlYiAiq6G3UaInxjsyo0iwsFUbeKLKV+QNDWJyc9O9qR1zRIzIob16+yYFbkABKsbSeQO/j9RZqCLfJ/cjqkU8pdDcBdR/zGVQ9aPIjkobDRLjy7lgO4gFvU56PX9mPf9PE9b2CxGI5j9xSQhxC9E+DrO7TrO8y4w2nYtsl76lrQVaZ42o92Ya8wERgdSaxELTeHh8O6Wdxn0/iAeX/W464O0/UQ6a33UzpAX4cy6SANDnshgqpc/8HaSb4QZOyC0Gekeb2b/C/SseQGlRaRm1mf6SSQSpHIpFoNnCtm3F5vFRkVWBW+uepNFoZPICv0f8pizYPSHomh4dyQwEfo+KDIa6tH0hvl66PuA5+w6FdZfDEtHoFSC0ipW1nLLS7vMAw7A4dJDVCvTUAcahcPL1jW+Qx2GVAZnr4fhb3jakraRfDPMLXOSBdwYdRWbek8joySPNU+v4aORH2Eo9b6Hth7jenDZ75fxZs2bPJyXzPHwDxoy/eR+4r6rOnUtea+jKg0OvgHGLrTqdrKYyuDIe0iKNxAaUCfvWSP6rfBTeNzhB3D1yqv5175/OV5X1iV0OTn9ynYIdQdZC7KI3o7NKiSpLNUttyvbDRuvFk7O0wm/WAgf5yRhFhraWOKzaV2/sJQw5i2cR9/ZnpEvP7LkCP8b/z+yNmY5ttVnhc2ZcxI1u2xWWHm2cID68Cz+cZj8h6KU1TrkPaODRF3svKo8p6bnf3g+F3x8wYln6HBm/W8WF31zUZvaVlVBjUqsGSQFJ7XtDfSH4dhHQhK7rdTkwLa7IXtR249pjV7XwoRf3SepJpGINYN2zLejA6Pxk/ths9s4Xn7cZRtDmYHvL/qef95ofuyud/qVl7fH4M6luqiaDa9uIHerWPjOrMgEOrmen6a3yLBLurptJzXkizlde8relGyBLbdCwerm2xz7BHY/KcZmT6DtC5OXNStZqtGABAlBNiEHmVUp7kW6RB1xZ8RhNXrI7pOg92Wj2DP5LqThoRQbxI1UIVU0TQaQKoRyTFtQR3QNh19j7DahbGa3i7FK3oXmvdVZ8I0CdjxAWN0zR4Wp4ZmjKq8KU7X3rteOvPtM0kbMxywvB0Cn1jXsDB7cssRwV6dwHaybC6U7XO/X9IGLSt0v391RmEqFElPWjwC8s+UdVsjvoSxwo+M50xvZ/8N+olcuwK9SSDA55D27EL5MPzdjzi6gurCVxQMvJCg6iBs338jixcB7wul3SC/qdcQEudDbtlma6vg3rv1x+F0hQzZzn1ik9HYCEhoiyjbdJCQrJvzqWZvayhmfk39cgfLPFUCD02/kv0Yy8l8jPWlZu6jIquCtpLeovFBP0ZDVBCsndZ6UmKeQSGHoi023ywO6huyFKxRaUIYgkTRkseRXlGKz2Dj651H8w/yJGx3nYSObYrfbOfDTAUKSQ3i/ZD55g3YzXroMSgJh+ZlC3rOV4u1dDpsZDr8DfjEOjfVugaJpRmK2ahnV6gJyKwo5c1Yq2h5a5CrvnQJVGEW6s8KqbXD61eSIIBtlSPeLasxbDtvvEffh+NnO+7beASVbRT2/lmqFdBVMZbDlX9D3AaK0twJQXufstJqsFOwuQB2sbldNv47CYDZgtJip1AcBEud78hn/gxHvdO06Gkfeg213wMSlENOCpHv5bkj/AuLbtpjfbYi7QPw0IjgY/MrjqPY7SHFNcZND1Dp1p2dXNaa6oJrCfYVI5Q2LivWL+Vqtc9s/j/5Jgi6B3iG9kTV3HddkifGnq0RSd2cGPoEhCkwfCQeF3S4cPwB5emenX8JZXpol1Ai9vsHp1yu4V9sO0g2A2fnteyOJXMigWWsh7vx2WtmJTGuHIxMR3Dqn7xzH365QBio5/Pth1CGuitQJuoK8Z1laGcvvX87Ul6YSMyKGh8Y9xG0jb8NgcU/w2t694nez9fxOpPgfKNksFJ+am4/uuB8yvoZ5epfzcpdETIQJf7Rcyzfjayjd7rWqTDqd+O1v7kGZ6ghZFcLpN+GxCZ4z6iQx1imTqtWglCm5Y9QdmK1m5++b3QbFG0CT3HbHX1fjwCuw89/Q7yEhmRx+ZtdZp1FoRFa4th8RQYcBqLaWYbGAXC7h7uN3e9a+Vqi/Bi2ycqCR089uF7KrHVkGwNOYyiD7Z6GiFzLUdRuZ0mVtUa/ELwYmLoag3kCD88wsK0Wvh52f7aRwbyHnvHKOJ61sQnlGOQGF6QTVJHFGzDgRpFW2UwRfxV3YJcY9713x6qLY7rybgZdqW2/opRTXPb+HhcHKSuH0iw1yUSh513/g2MciOy6gkbSHzQq2WoicIAZiTxZZbi9Wk5jA1mSCselChtcSOgJlLSgsOwFcLsJ0BRT+Cs645wwWSL4Xr21aAtOfgpoMsbh4OpC9CKrThX58V5lMnsjoDx1/+kvFzbxQLxa0vznvG/rNa5q+L5VKGThwIFKp5yL/zNVmFs5dyOCrB1PdsxQkEKkJBrkS4uc4JijdCokcdj0iZGpacvqV74GcP8SkM6iNC1OexGqEyoPCOVZXeN1PoqOaAor15STPHUzydO+UDD6y+AhpK9IwxIuFHLlV1+D0++casdAyv2ldzC5P3AViLhHuQm7RZhFSOt3B4QdiXjT5LwjsRVxoMBgApBgtRoxFRj4a+RGj7hjFjLdmtHYmoGPGz/0/7Cc4KZgV8hVc+fOVhCdPZ/SRJQ2Sx/V0pWhnV0ROFNn2deNEsyReAbHnt16v5jQgJASGrfiemWcHcOlA73uMHHLNEAZf7Zz9Uu/0q1+MBai11HLugnOx2W3k3ZdHVGAzJQkCe8LcLirn1Q0JqYt5sViEk6Y+MPXETD8Am9WGRCppUWnCneNndVE1G1/dSPL0ZHpO7Nlq+5PK9ANRY7I9+EXCeYfdVyqheDPsf14okoSPdc85T5Kv5nzV4n6ZQsbDVQ+3KLfaFeQ9I/pHcP3G6wmKbbgJB6mCCFK5R1o7qy4xundLjzqH/isyNQY8Bsc+FRmnCZeIzC1XRJ0tpLrb6vADUIe1nrkz/ufW6wl2NEV/w4FXYeDjEDzEaVd9cInSEA+qhky/rsjRX/cTkmNFFT2QqMAo3prhXN9eKpUysE80sl9jIPkWGNVNM+KTbxJrg6YyWDUNzj/SdUrPKLUOOd6oXBFobpYLJ0twcAvHeQnrnllN1FEZuxLLgUZOP2Mx/BQBKfeIusrdkejpML/G9fOvpVoELsZd2HWuRZnaSS43RC3WCU111+PhZYc58NMBzn7pbCRS7wluHnnXWJ5fM5YoI3x7mSgZw/4XRWmz4GEn5fSTSqX079/f/cY2936d9k6nCfouuhaXsyWHDa9soCRd5NaGhECuXkhIuMz0C+gB2v6gbvSQXJUOCwNhz9Oi9tCQ58WNpqtw8DX4a7yI4pm+1dPWtB27nUB1NUqLGDjra/rVlNSw5d0tZG3oGpPNwMhApr02jYxU4WwOlGuRFK2F3A6qR+Et5C6F3/tC/go4/JZwwti8V2ahPQRJ6+pMVpcilUu58IsLGX2X6+hNk8mzfZYqpFz0zUUMu3EYNYgn/5jgEAgeBON/aDkLpKsikcCU1a3XTyn6G3Y9LCQYuwJVabBkCBxt7IDWAVBcVe4Rk9rK8bXH+ee1f6iuFIoBTpl+8XMgpfvVtgPEhDlmhuvacKPeg3N3drpJHYZMJZQRAnsSF6bh3G0m3o7Wo5KrCAgPYPJzk0m9sDl9Lde4c/y0GC0snLeQja9upKBKLKwpLCEEBAjpL/GGZZD5g5AN6sroBohs+7YoUii1rdef7G6YKmDnI5D5o2NTaKgYlyrKmnf4vRb3Gt/P/b4zLHSJRNLg6LHbG+pEN3b6HS09is1uQ6vSEhnQihNFIu26gVjdicrDyPY+yrCe2wAh21ov71llqqLKVOVouvTupTwjfwZDSevOWneNnxXHK/j7xb+dpGVbolJvxaDMANrh9CveDNWZ7TdO09t9CgH6Q5CzqHVZ5PZScVA4UqpdS3WeLK3VV+wKmX7KQCVxZ8Shje+YdZV6CeTwlsqwpn0qAr4lEuhzO0xdI9RdmiPpahjxdvuNsduFskVzsqBKrefLfliqIec3KN/bZFe9IoK8RgQT1UuxFu4r5K+H/yJ/VzszdT3Ivg/WEXtoZYvZnyYL2Ie+Ip5RuitKHQx9CXrfAiPf9d56k60Q4i8GO7OszDEvSl+ZzpZ3t3jQqpY5+N0ugnP3Y1WUA40z/SwiGFnXXBHSboBM2XzAa+FaUXM7x42y3Z1FnTSzI9Ovzul37n/P5d6ce8F7/H0ANJ4iquo/jp5XwqRlwh9y0uftvLVPn9PPzVgOHCbz75OYjHuYtL/SWP7AcipzxANTcDDk1Ml7xmpcZOv1uQ3OXuucTuwfB5GTQZPSGSa7n9jzYeDTXe9G/s+1aP8MJKZmNFN2H+f3uWsAqC2rZfFtizn02yEPG9g+ymvFLESr0sGUFTArw6P2dDjyAJHJYqmCs36DSX+KyMiuStFG2Pcc1BYxUnYDI48sYkLwVQAMvnIwPc7s0eQQm83GoUOHsNlsnW2tA7lKzoBLBhAxOgKLpAaoc/p1d8JGgX8rGdkJF8M5myB0VOfYdKr4x8LAp0TwSR2Bch0ApTXlpK9K55Mxn3hl/d3x/xnPXRl3kRcgshbkVm1DdlXvW2Hws54zrqOx24XD1t0Lid6I3Q5WI+HhEqR2hWPBTaaUMf7h8SROTmzzqdw9fkqkEuZ+N5cRt46goFo4/VTmSGdpz9JtsH4eZHcRGfRTJfdP0B/ztBUewC4yenIXO7bU18RrKSsmZngMoSmdX3vUarY61b0CUY+yfu248TV8sPggAH3D+7Zcc/jgm+J69+F5arJg3/8xOG4TIBwVgcpAApUiMqaxxGfEwAj6ze2HvZV6Yu4cPyMGRnDHkTsYdkPbpGBzq3KxSU3IJQriNG2UvV83B9bOar9xNisUrBJj2amSeCXMrxKZ0u6kdJuQhCxp3wK0xWYhv6p5R0rpsVL2LNiDsdLocn/9mObNTj9LrQWrqaEW3Nzv53Ltr9e6Td2nsdJTs0xdA1NWir+DB0HEWc0vSFtNJ1+jeccD8Euc+L6fiKVGONo8PU+MmAhzy4UKwAkEBQm/qNrkXNOvPL2cv1/4m/wdXcfp1/fB80kffCFqNZQaSimoKsDSqO6mzWbjUHoBtj53N1vjsFsRPEQ8i3U1Scldj8KB15ycLPU11LZ/tJ3Fty3GYmx7PdXOZOZvN3Nk9BVNnX5+0TDue1FftjtTvgeyf2u6PWKCKE3QY17Tfd7Mb71F2Rwg2K/eCS2cfoFRgQRFB7U8J/cAWZty0RQeQSqxNQS/+seIMa89meyNsNlsHDlyxH1GtoLP6edmQlf/xMZX26dJ7w0Mu2EY12+8njKFmO0pAvWOiEmXmX6ukCpg4h/Q6zpYNw8239JR5nYMuv5CpiJv2clFUXqK8HGQeBVatRo/Uw9stSIaWROv4aoVVzHytq5R1+/4uuN8c8E3qA6K0TTYvy56sLtIujVHxHi44AjEzRJZBOFnetqiU6NwjZD/rT5Or4ChRFach87WBSQh6ygz1D3126VEh2jg2P9g652ef8DsKOw2qMkWtV6aQxksnINdJXNboRGSO5EN9TOCFDoAyo3lWE1WKjIrMFa4XgDyJKogFboEHeXmcgAUFl2XkF9xC+mfw2+9IK/RomTJVtj3Qte6J7eFX3vCXxMdC2zF7lm3cwsyhYz+8/vTY1wPCqsLAVBZIpydftoBMOZLiO4GGdBHPoDfksHQVB4QEIuXa2aKxejTDYUWzjsIwxqkk4KDoUp9kJ9ll3L9r9e7POySXy9hyv9NcbmvIyk9Wsry+5ez/4f9jm310exBQY0yVYEDRQcASA1rIau2OhO23+2UNe7Dg4SdATP3kykTgWSFYngiOjCaAEUAZbUNXpth1w9j3sJ5BIR3XoamXCUnJDmkze+ZXSMCCaLUPZuvKdkYu13MbVLvPTkD180VaiLuQKZ2/8J39Nlw9nohC9lG/s78G7//82PiZxObbXPgxwP8dPlPFO4rdLm/fo5VWyt+vJFNb23iWdWz5GzOodZSy48HfuSznZ8hk7inpm6bMv0UQc4ycjYrmMqbtrPbYMNl8NdZYCxpvzERZ0HyzYALh33ZDlg8UMjaeRKZstnFXqlUBJiEV57NOxMW8sKUFwBImJDA7Ydup//8zpN0O1VUiTFUhSagUsHLf79M1KtR3PfnfZ42y0d7SfsMMr9ncuJkrjFuZWjaAsfc6MyHzuSatdcglXmnW8CuUmNRBRBvnM7lAy8nOaSLSFm6ix3/hvXzxbjaGLm/UKHyb2PAkLcQNUWsfdKQ6Weqc0Ib9UaKDxVjrjF70sImbHlzA302L+CPwYHEvhaL3qgXSiitBJV5E95XjKGLkz5oFrfc18W+fEBAeAAB4QGUvy9e1ypElp9GpXFEUDqoyYE9T0DCZRA12fUJq46JzKWuRuEq2HILTPjduVahN5N8AyTfgPxjQN8gMStXyduVLeBpKrMrObrkKLZLLBAB0bpAyF0CgclClqY7U5UBGV8JPXx1S2GWXYDEKyH6HNCkCs1roLrOX/bTFT+R9XcWd6V7n0RhxuoMfrz0R1IeFTJvCkswOq0Udi2BzIUw7FUPW9hBHHoLtt8jIngjznLdpipDZM91tcjGRmhVOqiG8tpykqclC/kIL6QiqwKrxUqlUYRgKm1aIUlXlQabbhTSMl0tqq+thI+HpGtA00hqMW8p7H5MqAh0lXtyW4i7AORBhAXAoZgn2SHfzIXHH2Z8wnh+ve5XytPLuXrV1Z620rGIrrCEOjv9/KJcRrh3WaQKqC1spi6DDUZ/Aup21tHqDkgkTdQ7QkPBKjGQEfgtfxzxrv+JrqeOa9ZcQ0Bkg9PFVT0/gIMlItMvNbQFp586CqauFdJePjyPPAC0fQmuc0zUOyp23bILP4XnpXcNpQYstRYCIgKQyltfQA3Rj2NS1lGuuaW8bW8gkYjaUieDVAajP4aAnid3fD2Wasj4WtQacvc9WR3RfH24ZojTxGGxWUgvT8dqs7p0nqZckEJwUjChfVxnH6vV4qe2VmT7Rbe/PE+HE5YaxsDLBhIUE+SQ3VbKlA1ZL6eAzdaQud1spp+xVAQIBvUWAao2K/ygE3OzCSdk/EukEDZG3FcVJ2Ff3AXixxXqKFFTMLyZ55XOpCZHBIrHngdqZ2+pRgMVFT05M7gnA+tuk6ogFaqgrhPIbLfbqdHbABlqNaTXiAE3PMC5r6Gl3yNddiOM+6br1BY73Zi+FWR+hCi19AkKodjcEBAVNbiZesZegN1up2hvIYpaf0aZHuKdxgqy2b9B/nLo97DIuuqupN4jEmrsVjG2ggi2qC0S3zcvy4prlVHvO/50kvcshe0fb2fZvcu4dv21LpXBPEWfy0eyNDsMq/xJCqpMwjfyU6SQ9py6ytPmtQnvdOl3Ycqi+qIb0PWcfoZSAxaj1fFwHB+u47nJz3H/GBeRzRX74dgnQtP/RPJXwvqL4cxvYdKSDrW5Q9ANFl/griJj14jAQDgQ+xAPrL/WUdfPbrdjquoa9eEGXjqQx8yPkdEnD4lNTq9QO6w+Fw52U2dLY9I/FwvbZds9bcmp4x8LIcNA7o9FXUB2yJesLPwGgKDYIEKSQ1zKLclk7olWPVlkShnBvYLRS0X2l9IaImqpjfkKZud2aYdXi4SdIWpzqFpwNi8fB8u6WAbq35eKe1EdWrXIUqwx1XjKojbxxy1/8F6/97h5wL9JKLyFkACdyE4xFEDpVjHJ764E9YIz/gfaRovwKXfC2X8LKanuxIi3YchzhIdDecAmcvyXcLhYZH1YjVYste2T2XHn+Jn5dyavRL7Czs93Ul5bDogabk5OvxMjTrsyyTfBeQcgeLDr/TK1qE0UM71z7fIWaouh8rDjpVYLaotYFS+sLnSS+qpn15e7WPHIik4zsR6Fn4KEsxIIS2m4n9U/12hPSFSvl/dsMdNPphQRyd25ZkxXo7aY+NBsoCHTz5XDr2BPAYtvX0zO5pxWT+mu8XPzO5t5Lfa1ZjPKTqSmSk6AsRcjY4e75f1bJX42hAw9tXMUrofNN4uarh2BzSICMOrJ/g3KdjfbPE4Th1KmxGQ1kV2Z7bJNWGoY/eb2wz/U3+X+/Px8MtKfYcvqvpw5JJaRffvy7DPPkJ/vPTKMKRekMOfrOWjiNA7Z7ciASLfIoJWWCsefTEbzyhK5S2DJYMj9Q7yWyoQ6UvjYhjZ2u6NWE33vg7ELRDt3EtQLBj0t1Ec8Td4y2HQdFK5usqs+yKTesQJiTaYqv4rKnMpOMe9UqSmqYeusZ4nftxSVCorqnX7+zk4/OUaozQeZ6++XDy/AL8qh1lM/F2p8bdqsNqxmq4sDPYvVaGXlvPeJO7C8oZZaPYVr4PB/we5dWWFuJ/ps6DHXeR0q+xf4vQ9k/djsYV2B4dHD+ezs3xmc8Ql6PcSNjmPcw+MIig5q/eBOJHhIAjlJYtwL9gsWJQfjZ5+y2k1nrn36nH5uRiKRUNUFE9y+PPtL3u7zNta68T4lNoqHxz/MYxMea9o4airMKRKZfidiyBNZMZUHO9bgjiJsNMzc2yRiy6spXAf/XEtS6F6ywj5hUfZn5FUJmaoPh3/IB0M/8LCB7eNeWQbnbjfRK7KviK7v6eI6625EToIBj7dL0sZrsdvBrAdLNXr5MXYmXcWP5Y8CcPaLZ3Pl8iubPKTKZDIGDhzoUcdf/Nh4rlt/HbHjBpKc9zDJ+utE8JRM2Uz2Rzch7AzhgND2c73fZhHZVz0v71SzThljMRgbHGS3pj7NzK0WzjS8iNlgZvfXu72y/m6/ef0Y++BYbuj5PAMz3yM8uO4pJ3wMzKsQmX7dHbsdDHWLbQqNWFSSqT1rUweh1YLKKiIdc+pC7ed8PYfrN7qWTXSFu8dPmUJGaJ9Q/IL9Gjn9dM5Ovz/6w6pu4gRrbdG0C8m3dAh/Xwx/jna8lMkgIjAciV2GHbtDArYxh345xPoX1mO3de7/rrqwGpvV2SHtKtPPZre1zelXeaR7Obi7A7+nMMIksoyLWoiB0efo2fLOFgp2F7R4OneOn7GjYhl1xyg0sZrWG9OgzBLY1pIwe56GPwY2L0XcFmwWqDyFWu9hZ8D4HyF+TuttT4YlQ+CvOmn2vGWifuHht5ttLpPKSApOAuBo6dEWT33i2ADw+quvkhAXh2bXc7xadZCPCnO5++BB1j33HAlxcbz+2msuzuRZ6jP9IgPdk2ldLy8eEiKkKV2iGyAy7EIaOahHfwT9/t3w+uDrsPEKsNUtwp+KQ3L3k7B29skf3xlET4NxCyFiUpNd9Y6VFZmLefOfNx31Rl+Pf50/bv2jM608aex2O0FjBlCjiUKthqLqppl+MpmM6MmvIpmd1b2zrbo6NTlQvg+LzcI620sciH2IkgqhZXz498M8o3iGfd/v87CRLpBA0g2TKI3qi1VVhNnayME35AW4MAf8ul6yzUnReC6q7S9qSzan0OTNZHwjAoesJsIDwrmw/0y0NcOoroaY0fFMeW4KwUneVdfEZBISpFCXnSiRwKgPoP9DJ31OmUxG//6dJ/Xsk/d0Mwl7FvHloFweKnnA06a0iz4X9KGsyArHm9a9aIJE0rwEYfwcERVbsFpEBmv6dIS5PhpTlQ5pnxGnnY0iLxSTothR2Lv3zN7NFi73Nor2F1GZU0lRXg8kKAiJCBLp7KcDEWd1zRu3K0q2wLLRMPRlwoPOA6Da1nIxd7vdjl6vJyjI88V7QyS9SM15jtjYug3Fm4Reun9si8d1W6RyGPysp61oP5OXO70MDlIhQUjNWo1Wfr7iZ4ZcN8Sr5CMAhlwzBIDldeaHhJzQQHIaxGqtmgb6o6KWWFWakC+RdrPpavqXULASyagP0SidnX7txd3jZ+yoWK5ddy0A05dPh5Jk1Kb4Bqef3Q66QeDXjRZ4ijbA8W9g8HOiblFjNt8ogqtm7BSyZqcbPS8X8xO73bGIGxYiQ2mOxKjMJU+f16T29/Q3pzPt9WnQybfzzyd9DhL4195/ObbVO/0aZ7DY7DY+m/UZB4oPOBwGTagthN9TIPEqGPNZh9nso530uR1zjVh0rnf6LTu2jLc3v83QqKE8PelpQNTPui/vPvxCWv7OunP8TJ6WTPK0tsnb2e2wQ/skdi2U268jnjbMRSRSsBpEneWTZdU0KN0OFxWd3H1Vqe04hx9Aj/mijwAREyD1Pki9u8VDkkOSOVh8kKOlR5mS5LqW6LsD3sU/1J9r1lzj2Pb6q6/y/H/+w1qrldE4Z7lcXlvLJuD8R0QNxHvu9awk/PoX1lOZU8m5b5/rlOnnDuqdfi3W8wse3Hw2PIgLuvhvKN8DNtOpq6PoD0HJZlFvvHHQ1+abxdh81s+ndn534B8jMnBcUD9f+l/WQ2Qd20NqWCrRQdGMuX8Mob1dy8x6G4GRgYTefBElPwj5W1eZft70/O6jBTbfBAWrkM2v5vuSR7BFW8mruAOIRdtDS985fQmMamv0SechV8mJueQsiopK+VodylvPgvFRI0qZUowxp4OjubYIlgyF+ItgxJtiW+hI8dMVKVwj6mQPehZk4U5BT1VVTVU5vIFVV3/GsLwSNqY2SJKeKna7ncrKzsv6Pg1WjzoXo1qLJiUGm6VrRYZOfGIiSTeKibJOJwrc7y7YLQpVnkjxZtAfc30iuZ9YpPvnasj7s+MM9tFAwnyYV0mV7jyUVjGRrJf3nPzMZGa8OcOT1rWZbR9t46tzvqIsRzzstfjw4cN78Y+tq8vVl8ggcWOspRyrzUru1lxWPbGK8oxyp0NsNhtpaWnYbJ4bN4+vO87a/1tLUbpI1dZoAKsJlp0B27t50fLNN4ufbkzj+pIqjYr5P83njLvP8KxRzVBlquJgQRomWVmD069wPeT8IaL0uzuRkyDmXCjbAX/0hd2Pe9oi91O0AdI+A1MFwWrxIRdUCqdf4d5CNv93M1X5bZON6Mjx88WzX+Qi648E1fZteBCTSGDcdzD8dbe/n8co2SRkgko2Nd2nDBXqD6ejww9E8NXAJ5yyNkJCQG0WGfC5+twmh2jiNGh7aDt9EbDP+X3oN885a92VvKdcKmde/3k8PuFxFLJmFqftViFRF39Rxxjr4+QY9BR+g28HxAKRwSBkZn8//Dt/Z/3taKbwUxAYFYhM2XIGn6fmn9XVkBH+DkdinsIkK2vbQQMehQuOnlrme8/LhWy2tbb9x1oMp5Zl2BYGPg6D/0/8LVPBsFdE4F0LJAcLR2tLmX6xI2OJGNRQLzA/P5+H/v1vFhmNjG7mmNHAIqORhx580ONSn4d/P8yBHw4AjTL93OT0q3eeN1vPrzlKtwkZ/cJ1dfOCH+Ds9aL25qlyxmcwO6fptV6TC9VeptJhyBM1DxtRP3cPtMUDkFkhbJ76/FSGXneKErudSG3dMKFSuc70s9lslG97BVtO18hePG3peTn0fwQJoFGIi7OoSlyzkYMimf/DfJKmNBMA5WGMRjDLygHwV/gLhx+I4BVvGws6AlWoqJ9b7+A0ljZkU3dFBj8HFxWLfgEL9n5JdtwbWKR6cnaX8uU5X7L7q+YlvT2BXBuAQS3miCF+ISIhYMNVItHhJLHZbGRkZLjJwtbxOf3cTF7yeFKeubxNBby9jcbRsI+uepTB7w/ms52fNW349yWwroUoP4lM1EiJObcjzPRxIjI1KIKIjJSiNIsZe4mhxMNGtZ8Blwxg2EsjWRI3h10JNxBd+xX8liwWR310HfxjRV2u2JlEaRuiYcpqy8jbkcfap9dSfKjYgwa6Jn1FOqseXUX68QyqlWmoAqvFot/QV6DnpZ42r2Mp3QElW13v2/IvWDe360ncle4QkWRmEbiSZdzLtqSLWRv0LyRSCX1n9yVyoHsWTNzJwvkL+fKmL7nzWC829TmnITvlwMuw9kI6PXXGE/R/GEb+V9Sg6PdviD7H0xa5nyEvwLxKUIUSFiDGyeIqsfCbsSaDJXcsoWi/Z+o3ZqzJYO2za6kqEE7H+rofTvKe3Y2el8N5hyHSRZbI0BfFQqYPByEhoKpz+tXLyTfGVG2iLK0Ms6FzFyamvjCViU9MdNrmSt6zTfhFw9CXIe58N1jmw534+TVIYhYVQXRg3bWob7gW7TY7xYeKmwSZdSR/3PYHi25a1Ka2eaWVmBR1qixhiR1pljO9roNBT4HiJLI68pfDzzGQ9oX77arHboN1F8G+58S8s3wP7HgQKvY3e0ivkF4AHC1r3uk363+zOPfthnWJjz/6iEkKRbMOv3pGAxOVSj75+OP29MLtXLP6Gm47eBuAQ3bbXfKe9U6/6HC9UGpyxZ9nwLYTsh0tNaKkS8Eq8VoiAXVE02NPBtmJBbzqmLgIZmxzz3u4g8K1dd+Jz5w2x9StzysNwumXVZnVyYadOsfXHqf0i99R64uQKY1UGMVk0Kmmn91ObP7LSA+/5SErfbSJnpeJoBGJBJ1KPHOUVLcx2MSDFB8sZt99nxKaK6RHdWpdw86VU2Hj1Z4xrDORSOGcDQ1Syqtnwo/hQn6+K6IKEQ6/OuWiO5feyc6oe6hV5qDX28j+J9vr6p72fXweGyeL9ZcQvxCoPAAZXzrXH/Zyup5nqgugd5Ec583oc/V8P/d70heLCbVO1xC5G6txIWk34D9CbqM59j0HaZ9DQE/3G+ujKdZaKN1Gj7CsJpl+x5Yd4+erfqYszftv7HGj4zBPDqAwdAVlQetR+8lAHuieiEEfHkEbJEduESvFpYZS+s7uy7/2/YuE8QketqwpI28byc07buZ743usGtSLzcoXRWZH3/sgbpanzetYztnY/EN09XGRvd3VZFuyfhLZizXZAFhlevJCvicvYCn1Af12L3Rk5u/MR39UTCIUVl1Dpl//h2HMFyD1XN3LTsc/HgY+CZETPW2J+1FqhYykREJEkPDsltaKqNuUC1K4auVVRA/zTC3RtL/SWPXYKqpLqqkyVVFRKb4nDqdf9m+w44GGuovdAXUEaHp3vXGuM8j9E1ZMFiofddRn+kmQUmls+nC+/aPtvNXrLfK2dXBWUBuod1o3dvqtSFvBH4f/cFmPEBCOBy+8P/gAjn0CK6aSEC2ecwoLccjLNnZAW01W3kl9h5WPruw003K35JKzOadNbQ/kpwOgsoahUbUhosJqhD1PQcGaUzGxgZO5vtVRQu42rANVEioPiozr4n/E66p0EfRUvLHZQ4ZHD+eSAZcwNXFqm9/m1wULuLK2bdmOVxkM/LJgQZvP3RFI5VLUWpH19vI5L2P4j4FHxj/ilnMXF8Mtw2/jCqVGyL+aTlgzsBrBUg2WE9QHws6AC7NElvzmW91iiwO7Xcw1cn5373ndTchIETR0gvRpfYkIe4Wz02/j6xv5fPLnWM3OcrLeSN72PEwbtiE3G1CoLNw56k4uG3gZwX7O8sLHEt7DNvAZD1npo73Uf36lhobs1KX3LGXj682PsZ7CqDdizCtFYhETuWB13bVnt4sao8k3etA6D5F0FSRdC4E9PW3JyWHWQ/leMJUDDXKZJlkphIbxcOXDjPv3OA8a2BSTCVTmSBIYR//w/kLRbH4VRJ3tadPajM/p52b8K/M5/slflBzuOplW+jw9B346QEWaGPyDgyGnUjy0xAa5cPr1ul4MOM2RcpeQZaBrSZx2WarSYekIepg+RGERmX5FdTX9So+WsvvL3Z0a6XoqZBeXA+Av1SFJvBTO3dlyDQEf3snmW2DP0wQEgMIqbualhlL8w/wJ7xeOwr+pnJZafQpyRW4gIDyAqCFRlFkbFeo9XWjJkTTxD5h28vIFHiPxCpj0p0MWKipYBwiJkJoa+HTcp7zb/10PGuiaOw7fQcBrItBBbtE2OP3Czuj+GaeNqUqDRX1g3wuetqRjMFUIaRpTGTHB4kOuMQlpa228lsRJiah1bR8T3Tl+nnHXGdy882aqQqsIej6Iz6OFLqLD6Ze3FA68AnQzp4i5CvKWieyFesr3wM6HoHyf5+zyNJZKkTltbMg8DQmB/llv8ITExP1j729ySOyoWMbcP6ZTa8QcW36MhfMXUrC7wGm7q0y/59Y/x3nfnMeSI0tcnyxvGfzaQzg8fXgX1cehdAs9IsVzdlERRAeJAIny2nIMZjGOytVyxj86ntTZqa2e0l3j542bb+TmHW2TSj9SLMpk6GxtlFSrzoQ9T0KuG2T09r8IvyY4j3VtIWwUjPkcNH1O3YbmCOoDI9+HcQtFEEbkJFHfN7H5dYcx8WP45qJvuG3Ubc22KdxbyOI7FjucshWVlUS10aRIoKI+esBDZP6dSVl6gzNOLVcTqHTP+GooL+acXh9jlobD8DdAcsIzmkwFM/fA6A+dt0sV4idiPISPdYstDiQSUYdsz9MN22qyhYJHcyVmPIHcD8Z+BVHOKgH1mX7SSlGrM6tCOP3KjpVRsKsAY4WxU808GUbdPgoefIBqbQy6gADenPEmX8/5Gmnj2uISCbaQMyBkhOcM9dE6aZ/B8vFQfZwwf/HMUWkudcR+7P5yN4d+OeQ5+5ohdmQsCW/fT1qqkNd3ZPpJJJB6j8hgPN3ofasor3CqdVM9Rc4iWDwQ8lcADettZnmpVyZO2W12Mr5YS/8Do/mX3zoeGveQ2CEPgHqp2ZOkM9c+fU4/N6OqKaT89789Jst0MsQMj+Ex82NYR4nIPY3W5oiWrI+ebN8Jp0PPS7ruYNTV8IuGwc/hl3QO6rpMv9xy8TA85NohPFL9CImTO1E65iT5fu73lNwh6hUEyr2wiquPtpPzG+QvIyAAlJY6GYmaUuw2OzXFNRjKDE7NZTIZqampyGSey2IylBmoLa+lwiScfmEBISJbbOlIod3dnakthOPfC0eLK7pidpkmRchCKoIACAvQAWCWVVBVbSN6eDRxZ7RcJ8ZT1EvoKKx1Tj+7/fTLOvFPAEMOlHuXrr/byP0Dlg6HglVM6zWNc7eZmFW61rHbbrNjMbatfqO7x0//MH+iBkeht4unL5lVOKEdTr/hbwopTHVbl0y7CEfeE1kOjTNKiv4WC+Q1XU+ay230mAfzyiB2pmNTaCjIbP6Ulri+5uLHxnPOy+cQktx5wTPFB4vZv3A/5poGSVG73XVNv4PFBwFIDWvGIWStAakaAuI7yFofJ83Ap2BeBTKdcDwVFYFWpcWvruZm42y/yc9Mpt9F/Vyeph53j59trWOZVibmWyHSNjr9AnrAjB1iwe9UkciEvFZN27ISOxWpHOIuaJB3VASJ+dwprilUF1Wz5b9byP5HqD9oNRramqteAGi1nnsutZqt/G/c/1jx8IoOOX9GXhiX/1REzuC666s90q/qCBj/I/S8wv2Gjf4URr3f8Lp4k1DwqM8C9TZsDdl7arW4T6rNzjX9Zrw9gwdLHsQ/zN8jJrYHqVyKSe6PXSZH1YzaqgwrqX16efT53UcbMBaD/giYqwirUxcxSsuoqkvevW3/bVzxZwd8h92A0QhmeTlwgrynj65J8BCh4qMR8+8TnX5HFh8hY3WGx8w7EXONmbyvVxGcdwBlvY+vbLeoaXsKazMymYw+fTowgOoEfE4/N5Mba0F5/+30mtbL06a0C6lMSrleLv4OLMJisyBBQlTgCYs6B9+EPwZApfdFg5y2KHXQ/2FkUeMZLrmRKbuyeCBVTJIVfgqXWVXeiEqjwqgQ2aFBKi2kfy0W4Xx0Pc4/ClPXERAAqTkvMPLIIgaHj6Aiq4KXw19m7bNrnZrbbDZKSkqw2TyXHfzrtb/yUthL6C3C6RceFCIkCGoLHLrj3ZaynfD3xZB3QlZD3nI49qmQ9emK2O1Cpo3G0YE2CsqqmPHmDGZ96l2yrWaDmX3f76P8oHD6ya1aUdOvJhu+D4A9p5F8jlQmxpER3bROSPBQGPx/oOlLVIQCqV1BcV2p08qcSp5RPsNf//6rTady9/hZU1wjAiBq653POiSShhpaSBXdUwoz9nwY+qrINKkn6TqYeQDCvUtqxtPUZzFkZ3tPPMLoO0bzqOlRYkY2BCvW1gpZHmjI9Ks0VjpKGDTr9IufA+cfBk3fDrTYx0lRN+6E15WVKiwUjrb6bL/Gdf3agrvGT7PBzIGfDrS5ZvXxSuH0i5C3cb1AphKLZYFuCOJMvU84EDW9236MIwhuc+tt3Y2pTCywtUJmRSb3/nkv1aamc9a4M+K4J/sekb0EzLrsMr5sY5T9F35+XHiZBzNK7DDt9WkMumIQAHO/n8s1v1xDUfWpB5ibzSIwwmDREBxTp+5kKhdqBPXkLa+rkd1CnaWOmA/EngshwxpeR5wFExeL7E9vwmaGZWfCBudrJDYW/EwN8p52u73NQQHeQMnhEkw54hozSyvIr8rHYnMORrNl/gDfKrFl/ugJE320lb73w5x80PV3ZPqZ5aUO+fOAiADkarkHDXRN6bFSKv/ei8wo1mYcz/J5y2HpCEe2mI8uhLYfDHwCdP0BZ3nPykr4+cqfWfOUm2TM3YDcT06P524mt89ZDcEPOx8UmbOnMJ7bbDZKS0tbb+gmuvlKZudTHVhMtSoUhV/XcLSAuKlnrMmgvEhExxpV4mE4IiAChezEfthFDTllaCdb6aMtJESE4GeOo6xIPMhYTVZyt+ZSerTzBpWTZdans9h3jXjoDvbTihoBe0+jRe7uhNwfJBL8/CBCfzaRFefhb4/AP8yf4bcMJ36sc/S83W4nKyvLozXWep3TixG3jKDGLuRzojQhkHQ1XJgJoSM9ZlenEDwMxi6A6BnO2499BJtu8J5V3fZQshW+84ODrwNCCklqFyFaBRXlHjSsefQ5en64+Acsi0W0sKLe6Wc3i8WO0y3rxD/WIc/a7dD2hf6PgLYvYUKVm8pK4aDwC/Ej5fwUIgZGtOlU7h4/f7zsR95IeIPy2nIA5FYdGk3ds43VKLLfTqz50x3QpkLfexu+Z3a7kG7RprYv86G7YTND9q9QtMGxKSoKTH5ZbIy7mPO/vKjJIWXpZSyYuYDdX3Vupq5MIUMqa3i0rc/yU6lE5gXAgSKhKBEdGI1W3UL2jkTS/Rzb3QFTBWQvIiFYfI5FdX6P6MBoAhQBlNU2jE1L7lrCgpkt12Jz1/ipz9Hz/UXfs/N/O9vUPrtaOP2i1W3M9KstFhLE7uBkrmtjsVCDUIe5x4b2sOlG4XC0GJptYrfbmf7VdF7/53Xe29o0YFThp0ATq0EiFX2/4cYbWWU205qOxyZgtcnE9TfccAodODVkShln3H0Gfc7rg9Fi5McDP/L5rs+RuUGFo7RAz/Tk94nQ5Ils/uJN8FOUmP+DuA/ueRK2N5Vx7hTMetj7f2CzgDocYmaA/0moUHUkUgUoNA5lkXpiYoTT747Ihay+ejV27FQXVXPg5wOUpXn/HGrJnUvwX/AxAMsKPyf61Wgu+9HZsWlXR1KmORu7v/crSvkQ3H3G3VxYuJXEgrsdTr/K7Mo216PtTNJXpKP/7EeiCmIYr7uMM+PPFDsseqjtOqp6PponRO2c6Xfeh+cx4YkJHraqAalMClFRrB56IQ8Xx7AqfRUk3wJDXjyl89rtdrKzs91kZev4nH5uplaWS1WenqoCN03MO4HtH2/n84mfo88TUk61cuH0i9W4qOeXejdccNQzk34fzbNiKmy+hWgR6EpeXaCroczARyM/8srivK4orRGzj9AALYz5As761cMW+TgpKo9A/kokEggQynBUV4MyQMl5753XqtySJxj5r5Gc+99zMSAc5FG64FaO6Eaow0S9uBOLQg99FSYu6ZoL3uoIiJoqJLEQmQBquw6AwsoKDv9+mKV3L6W2otaDRjoTEBHA3O/mkj1UPMgEynUoFEBgEkxaKgpH++h2BATY2ZV8OZt6z+BoThkKPwUX/3wxw64f1vrBHUCf8/ow7KZhDqefos7pB4D+MCwf53Cmd0tsZiF7t/xMz2S1eCNrL4RDDVm3MhlERkJeyPcszfgNm905S8putZO+Mt2pBlVHk7E6g5wtzotWrur5/X74dwBGxjYTzHPkfdhye/d0bHcHarJg7QUk8B0gMv0A1lyzhqpHqjivz3mOpvocPaXHOifo0T/cn9lfzabfvLbNb28L+Y2Jew5zRvCFbXuDHffBwiDhBHEHhevhn+vB2Mb/T/JNcFERBHhgcb/n5TDkebA3L3ktkUgc9UVf3vAyNeam9Qorcyod8p5RUVG88OKLnK9SNev42wScr1LxwksvERXlHXLWhdXigldIFQSrT/I5JesnRz3H2ow/uW3krUzp87PwBQcPgaizGzLeJRKY+LvIsFNomj1lh3HgZdj9KGQuFDZ7axDixMUw+mOnTbGxILUria2Yy+i40UglUgp2FfD9nO85+udRDxnadoZcM4SKIWLxXW8VzyXh/uHOjSImcjzuJQge3Nnm+WgPNblw/DuoSicpOIlk/+GoLOEOp9/yB5fz8eiP21xWoLNIOjsJ+aXzUMlm8e/kr7l1ZJ28dfwcuPB4k1qaProAxlL4a1JdbfhGmX51Tr9+F/Wj58SeHjTQGavZSm2ZAZMsnwpbHnKpHOIvhJQ7PG1au/A5/dyM3Z5P8KevserxVZ42pc30ndOXc16fRqlJRCgNjO3N81Oe56ZhN3nYMh9txlQK5goCwkrYH3cf72X+CwD/UH+mPD+FvnO8X6Jow6sb0OwGiU1OeJBWyM509wyr7sqeJ2HlFLDWYtYcIjvkS1alr/a0Va1itVkxSsoBiAsNgcwfxYPm6UKjehSAyHiJmeYZW06VgB5ioaLHPMcmP4kOgJKqSjLXZ7LpzU3UFDVdGPIUKo2K/vP703/wRBIKb6GHaoinTfLRUdQWCzmo/S8hlUrI1yyiSLuUY3ltk4brSEbfOZpzXj6nwelnaeT0U4bCkJeaZgV3FzIWwA/BcOgNKN0O1emetsjzSBUiEzz1XqfNKTFiEdxqt1BSU+K0L7hXMP8x/IcJj3VetO5vN/zGb9f/5rTtRKef3W7nu33CWXRx/4tdnyh/OWR8CbKAjjHUx6kR0BPO+AxF0lwASkpEhrSrrKf5P8zn9oO3d4pZaq2aQZcPImZ427KQaquVBBp7C1WJthAxAXrd0CSb6KSp2Atpn7ZeH622UJRasNuF1L0nsl/jZ0O/f7fa9ysHXUmiLpHC6kI+2PpBk/2/Xf8bn0/63JHVec999/Hwc89xlkzGNLWar4G/gK+Ac9RqzpLJePi557jn3nubnKszydqYxftD3ufgLwcpqC4AhBrTSUlF5vwB6y6C3Y8DcNx4Di+u/45M22yxX6aCiYsg9jyoPCy2KYMhwkMS1/0fhTM+h4RLRCDO4kGesaM1XHwWsXWx8zmNYlEiB0Uy+8vZ9Drb+8sADbhkAMV9xgJQYalz+gWEt3SID2+lbDv8fQkUCtnE+hKllXWKvQMuGcDUl6aCl/nUgxODsaT0w6IKbLaupI8uhlQJ5btF+RzgsoGX8eKgP0gsuAu9m2Ka3Enu1lwK73uJIdsSgK5bV9L7xHu7OHJ7EiW9YkmakuBpU9pM3BlxBKbGYVsh5izDe/ZhVK+HmjY062H/SxAzHcLP7HxDfTTPjO0AhK4qIC3qNdLMEqy2t5HJZYx7yPtr0djtdv568C/6RY9AZjLx4DUWsSiqDBa1nXx0LZKuhojxgIRC7WJ2xtzLgkOXctVZE1l2/zJMVSbOe/88p0OCgty0kHGSLLp5EaowNcklD2GWlREXGgybnhESEo0cR92W9fOhYJWI5AYRiWW3daus7rv8t7JpnT+JQ2WMvb+GUbePIjDK+7IYxwVey9bMaxkwpG7D8e+hfBf0e8h9C34+PItMJaTSTOUA+EtCqEDP8UKRdbHh1Q3oc/RMe61tTveOGD9dZvr5x0C/B9z+Xl5DYC8IGQkxMyHlru4rL9teel7adFMPBcoD4ZgUReRV5TktBnqibtHUF6Y22VYfxV7v9DtccpgjpUdQy9Wc3+d81yca9wMY8oS0qw/vQxEISVejsYuFy4oKSE+HlJSTP6Un5p9VdYJAgW2dgvS6Tvy4i4SLxTjXmmz4nqfgyLugG+T1z/4KmYJHxj/CjYtu5KUNL3HLiFvwU/g59g+7YRi9z+2NzWxDphTPlvfcey+XXnYZn3z8MU++s4DSsgpCgrVcc9tlfHHDDV6R4WcxWDBVmbDb7BRUiYXSyMDI9p0k7QtRF3HwMzDwSeh9GwD5JRrWZ81nSp8T2md8DZtvgolLIdKDUmsyJSRdJf6OmOjdNdaPfSqksMd9DzKVo/btnrJ/eOOfTYyMGcGZPc501GbsChiN4ne52XWmn2Tv0ySU5QPvdrJlPtpFyHAYtxBCR5JVkcUW5TekR/hRUSGylVIuSCGFU7iJdiBGI5illcgUfkBd2amsX8BqcDk39eHlKAJhbkOgYN/wvhgT+7LOCHq9COA79Nsh7i+43ytqoPqH+iMbNZS8aCEVr5PLYVEfSLwKBjx6SucObPME8NTx4jtn1ySm4mLS+55Lypz+njalXZTVqdhoNEK2xyWVh2Dfs1CwurPM8tFOesfXRYxK7E51LboCV667kczeU5AgIVJXBj+Fw9bOidD14Waiz4Het4BMhUYprsmyWrGYnfV3FukrnbMnZDIZvXr1Qtbs4NPxHPr1EFlrc0jNeY6Bme8RolUKqZbRn3jMpk5FO0BEkltN4nXap+I7WPS3Z+06FXY/CftfdrwMDQxCgozqavAP80cTp0Eq955p0M7PdvJK5Ctk/30cgND60rnZv8K+50Dii9PqNiiCYE4eDHkOgCC5GCdzSsR9+9jSY+z4ZEebTuXO8dNut/Pd7O/4541/SAlLYVTgHLQ1wx1Rwd2esNEwdRVETvQ5/FohPh5UZqEpn6fPa7I/bUUax9ce7zR7+s3tR7+5ztKKJ2b6pYSlkHFXBt9c9A1BqmYcPRKJ99WM8tEEiQR69xZ/HzkCiw4tYuaCmTy37jlHm6IDRez6cheGsuZrwblr/Nz6/lZei3uN3K25rbZdcmQJXxsuJyfkG5r4G3P+gPWXQMXBU7KnVZTBbasTnHpP3WLx6I61pyVsViEHtuVfrTa9avBVJGgTyK/KZ9HhRU77+s3tx+g7RzscfvVERUXxn0cf5bUP9zP27Byuu3M//3n0Ua9w+AEkTk7kzqN30ndOX0emX2RAO51+mQsh8zuwW2HgEyKgryYXQ4mQOw07Mb6vdAfoBkPoCDf0wE0Mfx2GveppK5qntgCK/xbrZQgJbJkMjgd9yz1/3s1vh35r5QTeg81q48PhHxKyR2SGlRldZ/pJs38iuGq1R5/ffbQBv2joMRcCEsiqzOI3w79Jj3jDERjlrSz/93IUr7/EvujZTF6t5OcDP4sd+18Uktc+ugX186DKSgiKDSJyUCQ2i63lgzqJ0D6hWM4/h2PJIvNdJ5OB9NTTTmUyGUlJbazp7AZ8K0huR+RFV1ZCSBsVOzzNF1O/wGBTQcDFBAfD1tytKGVKkkOS8Vf4NzTUDYJzd4sHBR/eRf4KqM4kPuZa5BYtFnkFWSXFhPmH8fstv1Owq4DrN17vaSubRSKRII+PpjZQ1IDz95OI+hERZ3naNB+niE4VAjYor6sbcu36a0VR3EbYbDYKCwuJiIhAKvWME+a+vPvIPm7jlzvENSiX410Pux3NQCH1g90ualppB0Li1cIZ2FXJ+BLkQY7MpPr6kno9mA1m9Dl6/MP8UevUHjSyAZVGRWifUHKMZZhkQWh1OkAKoz4Qn4/cr7VT+OiiaJUhZBsgr1yMk3O/m4tc3bYpujvHT4vBwuE/DqPSqLjk7kuo3HAJi4ppyPRbe6EYIyb46u2eVvxznagDNbfMIWMWFwdqczR6dpNX1dTp9/MVP6NL1HH9Bs/NPeudfo2d1gm6BBJ0zaixFK4HYzHEzBDZuD68k5XngKmMPn22sHUrHD4M/lE5LD6yGGWjDM1Dvx5ixcMruHHLjfiNcH3/dNf4qQxUoonToNK0ft2syljFPtkCEgK1BASckKlQtlM4ZwrXwKzjItOpJhc2XQdJ14oMPXdhrhQKD2FjQd1oMd9mFrUt4y6EoGTx40mkMjCXg7mq1aZKmZJJiZP4bOdnHC1tX920iAjxu75OpDdy0pl+E34V6gKN13B2PczV2i/YrNlLWNgJweqDnxGBZr5xsO30vV98R/2Es1guh6goOGwSzvWsyiwA3u3/LsG9grn0N+/NUjJVmTCU1SJF1D0vrXWd6Webto2ivKOE22wee3730T7qa6iZ5aWOOdKx5cdY+Z+VTH1xKomTPFC3tRk0cRqsIREY/ERmmEZV9zAy8h1f3eWuTO6fYKuFuFmUGkpZkvs7maEWFJXXMfHJSR5REW8JvbkcpCBBQoA2GWbuOeVz2mw2CgoKTt24NuIbnd2MSlNB7IG/+P2aHzxtSpuRKWVY7CJCR6eDG367gcHvD2bt8bUnNgTdQF/0szdy6E3YcjP+/qC2ifSQIzniBmk1W7EYLY4aBt6IzWIjN93Irh7XsiVpDgerSsRCty9tv2uSvQh+7wcFqx2TywqzWMw+0eEHIrskPz/fo9eoRCIhr6KKatUxVJoKIW1pM3vMHo9RuBp+Sxbj/ZjPQNmFU3ymroWpaxwvd9m/YnvSJawqXkD6ynTe7v02B34+4EEDnek7py/XrruWJ5VnsmxoKObAuoxYRSBovFN2xccpkPM7ZIq5YoifWIgr1Itx0i/ED4W/ok2ncef4qfBX8KjxUc7/WEgfNlaBAETGhd3q+mAf3ZfAJOEYsNY6NsXGCqcfQHpRU6ff9LemM/HJiZ1iXsbqDF6Pf5193+9z2t44069N34/Db8O6OeL+78N7CUiAwCT61EkSHj7csJBZaih1NEudncrFv1xMcK/mg1XdNX4OumIQN/xzA6F9QlttuyNfZHFraoY2lfcc8B9IuVt816rrMmUNucIJWJODW8leJAI5sk8I4ihcA9vuhLTP3Pt+p8KMHTD2izY1TQlNoU9oH/xOCJSqyKrgwxEf8vfLrhUsIuv8aJ24DtcmsjZksf2T7Rj1Rofsdpsz/Sx1dasl0qbO29gLWJt/G5mV/Qg/sVSbPMDn8GsvUoXD4YchD8yVxMSA3wlOv9A+oeh66jxkZNtQa9Vcs+NOsvsLiflig+tMP7tERl6JyavXmHwgrscfw2HnQwSrxf3QLKugrELM520WG1X5VZj0Jk9a2YTRd4ymcOY1VPuLeudadd2aRMgwiJriQct8nBK7HoLtolZuflU+d6y8mgPx92M2N0gKewvHlh9D/sNSwgvDCZBrkLpJYtput3eq08+X6edmfuuVyN2rPyd/c5GnTWkzly++nB9/BD6D4GDI0YuHipigE+RtDPkgU4NC65lC3j6aZ8DjkHIn2O0EyEKpIo30fOH0m/XJLA8b1zqF+wpZOuEDks8qZNXkxRjMj3naJB+ngqRO5sNmJsw/CqpBbxHXY1laGUUHiug5sSfKAO+omWM1WTm+9jhLj6xj1cBriLdMgsr/wh/9YdCzYhHmdKG2COxm8O/haUtOHf9Yp5cF7CY35DuOVMcQljKdM/99JhEDIjxknGvMVjNmxCJNXKhOZFWV7xaLnEqdR23z4WZ2/Qcs1dBjLuGBIVAKJTViwdpQaqD0WClhqWGogjp34U0ikSBTyDBajOTkKgGJozYNExe1dKiP7oqLuhUqFQQrosm2y8graZqB039e55Y5CIwORKV1/q40run38IqH2VO4h4fOfIjxCeNdn6T/IxA325dV7e2M/giA3pXiZU6OqIsKzk6/sJQwwlK8qy6x3W5nR55w+mlrhjaV9wQY8jwMe63hWTt0BMyvAbvFvcbEzoQhL4mMvsaEjYXxP0HwUPe+Xyfx0LiHeGjcQ022+wX7UV1QjdXoOnCl3vGl10NtLai9QwSCvd/tZfNbm0mensyLZ7/IU5OewmJrw7VgNcHS4RB1Nox4q+n+Hhfx3paLABfynj5Onqo0WDoS4i4gNvZ/+O0XTr/MikwALv7Zjdm6HUj94rtCaeeqwVdRUF3gvDZoroKSbcgsvkAwr0ceANr+4BdLcF2QIRI7hRUVQAi9Z/Tmnsx7PGpicxiNYJGJm71Wpa0Lyrb46i53ZYa+Qr06oiPzVFaOHRuHV+VRvOEQw24Yhi5B5zkb6yjaX4T//uNEDBpFUD+zkL4u3gDxc4RsbhfBo5l+a9eu5fzzzycmJgaJRMIvv/zitN9ut/P4448THR2Nn58fU6dO5ciRI05tSktLufzyy9FoNOh0Oq6//nqqqpwfPnfv3s348eNRq9XEx8fz0ksvNbFl4cKFpKamolarGThwIIsXLz7JXtnYN3YqZ/3inQNnc9RHwwZojRTXiGiK2CDnBVP+uQZ+ifM5/LyR0BEQNRUkEnQKMXPPKi5p5SDvQaVREXTWEHJjxIQ4Sr8D/r6842ta+OgYYs+F8/ZD9NliMRuosZVjtVnZ+flOvjnvGyqOe4+QfHVRNV+e/SU1v+QDECgLAYkCeswDbb9Wju5mJMyHWZkQ1MvTlpw6tcVCKqtucSRGJ8bGclMxIckhTH1hKrEjY1s4Qeey55s9rHh6BRJbnXxehAZMpbBkCOx4wLPG+XA/w9+EMz4FIEpXlxFdqwdg99e7+XjUx+Rtb5pB1ZFUF1VzbNkxqvKrGPPJGP4brKRIs7zB6efDRyPO1zzOuduMzNH9n0ft6DmxJzduvpHkac6ZLI5MVa2Nr3Z/xeIji52cQk0IHgw9L+lAS324E42mITtLXyQWMlv8fDuQ7Z9s5583/mm1XXZlNiWGEiR2GUGGAc6ZfjXZkLEATOVNn7UlEpFJ5E6UOiF/rj7B4yP3h/jZENjTve93KlRnwcHXoWz3SZ9CGajknqx7OOtR16UjAgIaZOC9SeJz1O2juGzxZQSEC+PUcjWByhNTRF1gKgW/GFC7zgqsrYX6JTOf08+NBCRCzEyIPV9kxJuEQlaePq/LZMSVpZWx54sdKGvKUaskvDbtNb6e87VjgV402ols5URCyn/3nKE+2oZCA1NXQ8odKGVK/OViLCmq8m6JzO0fbydg5zosUjFQBamCQH8EvlPBnqc9bJ2PkyZqili3BkfmKRI7ZlkFGRvzWPfsOkqPeGYudyKj7xxNwU3/IVH/Gx+M/xPyl8HW28WcpAvhUadfdXU1gwcP5p133nG5/6WXXuKtt97i/fffZ9OmTQQEBDBt2jRqaxvkZS6//HL27dvH8uXL+f3331m7di033XSTY39lZSXnnHMOCQkJbNu2jZdffpknn3ySDz/80NFmw4YNXHrppVx//fXs2LGDCy+8kAsvvJC9e/eeVL9qlfno9Sd1aKdjKDPw98t/U7xTFHImUCwwqWQq5xs7QOz50Pu2TrbQR5ux28FuJ9RfSMvklAvnbfGhYjb/dzPlx8s9aFzLBCcGw+xzOZQqvnMaQyYcXwCW1us3+PBuIjQNskrlteWkzkrlgk8vICAywLFdIpEQEhKCxEMBBaogFTPfm0nhABHWqFWGgKY3jPteLHycbki7iQjA3qdgyVCoK0CfECay+vTWImxeqN62d8Fetry4BbvUjszqT0SoQkgyDXwSYi/wtHk+3E3kREfd2memPM6520z0Tn8ZgwF6jOvBlOenoO3RuryuO8fPrA1ZfDXtK44tO0ZpTTl2iQWlPVAsrJfvgwOvQVX6Kb+Pjy5G+T7Y+TCU7XLanBivRoKMLBfPvkvuXMLLES9js3husK0PaDxi/JscfQ4alYbpydNdN7ZZTk9J765I8T8iU7om2yHxWZrdNNMvb0ceL+heYMOrG5o9lbvGz20fbGPjaxtbbVcv7Rlo6IdKpnY4mQDI/ws2XA5F66FoIxx+V2wvXA95yztOdtZSDQWrxd/VWVCV0THvcypUZwg5sPxlHfo23ljXL7R3KL1n9EamlLXvQL8omPwX9Gua9QhQVCdK5e+P83Xo49SQSIQUbfwcYmJAaREXldlmptRQypElR1j+7+WYa7z3fpO1MYv1D/yGf0Ve8xmvAQnYhr6KNOYcjz2/+zg56td4y2tLsVrBYrSw68tdZKzJ8KxhJ7Dzs52EHPoHJMJZHqQMErVGe1wMugEets6HO1DJVQQoxA3ILC8lYGQ//rX/X8SPjfewZQKJRILJKgeJFJUK6HklTFkJ2r6nfN7g4Oal592NR51+M2bM4Nlnn2X27KaLqna7nTfeeINHH32UWbNmMWjQIL744gtyc3MdGYEHDhxg6dKlfPzxx4wePZpx48bx9ttv8+2335KbmwvA119/jclk4tNPP6V///5ccskl3Hnnnbz22muO93rzzTeZPn06DzzwAH379uWZZ55h2LBh/Pe//z2pfklM6WT/dRBDmeGkju9MKjIr+OvBv6jadQwAk6pB2rPJDbzPbTD0xc420Udb2P8yfKuA8l3ckfoiU3Zl0bfyTgByNuew5I4l5O/I97CRLZNT0pD5pRzwH7i4tstKy5z2WKrFYkX+SrSBCoamfc2l9t8JUAYQPSyaodcOxT/U39FcKpXSo0cPjxUBV2lUjLhlBLlx5QDoVCEtH+CjaxAzEwY+DVIh+ZZQp91klBdSnGfm6xlfs+75dZ600InzPjiPUT8L2Tm5VUdICKAMhoFPQNz5njXOR8dQF6wTovEjqK6GX3ExRA+NZtxD40RATCu4c/yMGhzFeR+cR9yYOMoM5QBEh+iQy4GidbDjPhFl6+P0ojoD9r8ApdudNsfXPZO7cvr5h/sTlhqGpdbNkoQu2PXlLja+ttEpi8Jigepq8feKPFE7c3bqbFTyZuRyC1fDd/6Q1ra6YT48SMlW2Pcc6I/Qu7fYVJAh5m21lloMZvH87RfsR8yIGIKiXWloCtw1fs5bOI/LF1/earvG0p5xcSBr7MeJngZjvxbBIIffhq23gakM9j4Da2cBHbSwvm4urJou1BEOvg6/JQpHvzcRPBSmrIKk61ptarfbGfPJGMJfDidXn+u0L31lOn+//HezGVfe6PRrHDgxb+E8rv7lagqqWqkFVLFf/JZIQOraWVgsYoN9WX4dSGwsyOwqFBYxPuVX5ZO2PI0NL22gprjGw9Y1T+LkRMb891KqguORqavJr8pvKikbEI+0771E95vused3H+1g/8tw5AOgoY64SVaGXg92m51frvqF7R9tb+kMnc6cb+aya/x5AEiQ4K/wF0pE474V8oo+uia7HhXz7bo6xQ0Sn6VUWvwJ7xve5rr2HU3xoWIk2VlIbFbh9POPgchJoGh+XtkWpFIp8fGd59j02nD+9PR08vPzmTp1qmObVqtl9OjRbNy4kUsuuYSNGzei0+kYMWKEo83UqVORSqVs2rSJ2bNns3HjRs466yyUygbd32nTpvHiiy9SVlZGcHAwGzdu5N5773V6/2nTpjWRG22M0WjE2KjSZGVlpePv0NzjZP66m4KZV9PzrJ7YbDanyaVEIkEqlWK1OmtgN7ddKpUikUhcbgewnZCy0Nx2mUyG3W532q5L0nH9P9fzwn8DsJfZqZGJjL96ze7mbPfmPtVvP9HG5rZ3iz4FJCKJmYkNJX3jIlGbpBTnS7DZbPSc3JMrll9BxKAIbDabV/YpfWU69u/XEzElgsqYKhQyRUNf6967W3xO3fHac2W71Qhbb8OWdCNq9QRiSi4lJhzU8qY21gcXZGZmEhsb67DNE30qNQhJXJ1ahy19ART9jX3AE0jUYd3zc+rufYo8GyLPFtuB8LosaKO8iJIKCZnrMwmKDfKaPgXFBFGaKd5TZdcik1mx2U6Dz+l07dOuh7EffA3b+engF01oqITqaglFRRJiYtreJ5vNRlZWltP4ebJ9CooPYsj1Q7Db7VSZRSBOj3CNsD/2Igjqi0Q3ECmcPp+Tr08QPh7buQfAL9YxJ5NKpQRFlLEt6Va2mUp43LJU1IOs69O4R8Yx7pFxgLhWOrJP2z7cRsmhEkbdNcqxraQEQIpUKmFrvsj0OifpHKxWq+vPQ6JCGjMDApOwnXD+LvM5ndinFrZ36T4lXAxRU7H5xdOrlxW7XUrG4UBkvWSo5WpKa0qJCowiKD6Iy/+8vMU+WSwWsrOzHePnyfYpKE4sAFkbfT9c9anYUIwUGZqaIST0tWG12hv6qo7CFl9X76vPPdDrZmTyQOz9H8FenYW90f/BrZ9T79uRxJyHXaKCqGlI7RYk2n7ede3J/LGH1dXitFpb7VNuZS7FNcWklaYR6R/p2L57wW52frKTwdcMxi/Er0mfwsIk2O0SCgpETI43fJ8+GvERNouN67Zfxw/7RQDDi5NfbPaZXlqyEclf47ENfgF76v3N2i76KCUszHVfu/QY4elxr7YA6YoJBMfPR6V6lmHHvuOhe/3ooelB7D2xDLt5GAFRIrvFG/vkH+GPZngvzEuk5AYuJPrVizkz/kzWXL2mob1EgtliIScnxzF+drnPqRHd5tprpk/SQ2+BXzS2pBv45IJPeOxxCXZ9b0pKrAT1lDLvp3mE9ApxaaOn+qQO11AbYCCm5FJGjzFjs9m6/ed0WvQpoCf2iAlirm21EuIXQlZlFiZ5CYUFFipyq5Gr5Kh1ao/3af2L64n7cxff3/spL26fz3uxT2CVqIQSU+M+tfNzstlsHD9+nM7Ca51++fkiKyky0lmHPDIy0rEvPz+fiPqQrDrkcjkhISFObRITE5uco35fcHAw+fn5Lb6PK55//nmeeuopl/uORReROvNsjAHCKZidnU1paYPcSFRUFFFRUWRkZKBvpAMaHx9PaGgoR44ccZIwTUpKQqPRsH//fqeLKSUlBaVSyZ49e5zef+DAgZhMJg4dOuTYJpPJGDhwIHq9nrS0NMd2tVpN6uhU8l4yotdXcqyoLgJRKiSlCgsLyc/PR24ppkfOY5hiLyZsxN3e36fUVMrKyshqFH4cFBREr169HH2qJyQkhB49enTtPqnGka9NhuMmysr2U1ubTGGhH8ePZ1NRUQphoM/VE2Xzzj7tWr4L7eEM/Mb6YZMDJVsoLSkhuyaqe31O3fHac9WnuFgKBiygpDaI3Nwj6PXRFBcrAH92LNnBqltWkXxVMonzE4mPj0en05GRkUFZWZnDCdiZfSraXMTuF3bjN8IE/SFAoqbswEJCy39hj2w+2vDE7vk5nWZ9MpaKe7JRXsimzce4KeMmQkNDOXjwoFf0KVwVzuFDYpvcGsCePXvoYfydkKqV5PV4nqLqhgyV7vw5nTZ90qRQpTmL4wcOcMy4jWVBn1MbH0Jx8cfsWL+DDXdsIGpiFH2u69Nin2prazl27Jhj/HRHn6rN1dgQDynBfvZG76slChNRUZw+n5OvTyiVAezJqAWOOfUpJgLyQr4HYPU/G4nU6jzSp8nvTEZtUzv1KStLicXSm7AIWFog6oAFVgayZ8+eZj6nIAaO+1l8To22d63PqTtee830yR7Iof2HMBol6PU90eslZL9Sg7+qlrS0NIooanOfGo+fJ9OnkOAQdq3bhd3Pjkwla7FPL018ifAdL7IsvQxJv+Ps2SOCKwb274fJUMqhtPr/rwKZLISBUQr0fkNJy9NC3f/B7Z+TMRW9IRYOHAMiiU98glCJxPuuvbxclOZ8TIpoQkJDW+xTpDqSzMpM1u1ZR2BZoOPai54TTdCEIA5nHEaaI23SJ4NBg14fSk6OPzab1Cu+T+Ejw/H392fboW3i/SQyco/louihcPl96hWlJSh+LkcM/TA0sv/Ez2nHDh16fTChoUHU1hq73xjhwXGvuqKA3mYJZSV6QkNrMeZOxZSWwTF5wz1UVatCo/TOPtltdg4cCMRkigelGEtVFpXDpvi4OELXD0fvfybHZNc7xs+u9jl1x2uvuT7ZYv6LXarGvGcPfZL60MtPw56KSrZuPYpeb4BE0PbWYrPZvKZPllJ/JCVyksve47Fr09mzZw+9FNsJqtlGmvpyqq0NqlHd5XPqjtdekz4l38Bew2isx0qBUpRWkZxlkpVyaN1B3rj1R1JuSSH15lSP98l/lD+b99ZSFZBNVW05rJiItaqQ/cm/ndLnVFtbS3p655XLkNi9pKKsRCLh559/5sILLwREnb0zzzyT3NxcoqOjHe3mz5+PRCLhu+++47nnnuPzzz93uggAIiIieOqpp7j11ls555xzSExM5IMPPnDs379/P/3792f//v307dsXpVLJ559/zqWXXupo8+677/LUU09RUOBaPsFVpl98fDw8BD0qb+GmmHd4+GG83stvNpgBOXMvEQ8p97y8mU1FK0gNS2VOvzkNtpfvRrpiPPZ+jyAd8IhX96l+++kajbG/6ACzn/kUhSmclU//m4gIO3abHavJisJP4ZV9qqiwcf6N69jY7xz6RPTmQG8N9ppsbBc0REB0t8/pdOnTsWNw7WObIfQwnzw9mtCSUH669CdG3DaCIdcMQSKRYLfb2b17N/3790cmk3V6n9KWp7H8/uV8OXAZ/6T+ytP9v+eRWReAsRjUkUjqohe78+fULftUuh3pznux97kLacJcKg2VaF/Sgl3KgoGlXDw7yKv69JLuJUjW8P7Y3SQG92TFUw8i3fskksNvYzvvKHZlg9Rjt/qcfH1id8Fuhn00DKU5gk9SC7jofAPvpLzD0BuGMvGpiS32yWq1smfPHqfx82T7tOqxVRz48QBTF02l74K+SG1KFo2uYdo0oLYQlCFIZN45j/Bdex3YJ7sdW00u2K3gH+fUJ9UTIZhl5fw+Yw/Th/d19ClvRx6HfjvEgEsGENI7pNP7tG0bPPOMlMDkXXyrG4pWpaX4/mJHW+iGn9Pp0icJUFuEzQ6ow7nrLgkZGRIee0zCqFHOfVr37DoCIgIYfvNwl30ym83s3bvXMX6eTJ9qS2t5Ofxlht00jHPfPbfVPt16K2RnwxNP2Bg2rG57+S4kfw7HNuRl7Cn3iI02KzJTPnb/+M75nKozwWZCountldce2+5GeuRtrON+QhI3q8U+XfXTVXy550uenfQsD535UJuvvQ0b4MUXpaSkwCuveNf3aUv2FkZ9MoqowCiy784+5e/T229L+OsvCZdfDpdc0s3GCC8a9155RcL69VKuucbKhReC1WylpqgGtUaNWqv2yj79cvUv7P/xEJsnPoB+9P+xRvIUNw69kfdmvifamyuRrp+NNeQMdksudoyfXflz6o7XXkt9evRRCbt22bn3XhsTJtS1R4JEKvGaPr0U+jJFsijSxl/FDz+I95BuuwPJ0fewzsoBdUPSTnf9nE6HPq09vpZte2r487NhDIkOZIJ9Dckzkuk9s7fH+2S1Qp+77+R45LvcP+oRXo62YbMasQ95ucU+1W93ZXv9uesVKSsqKtBoNHQkXpvpFxUlMnwKCgqcnH4FBQUMGTLE0abwBNF1i8VCaWmp4/ioqKgmjrv61621qd/vCpVKhUrVtC7E6JBzqckfQlWVFImkkWSHC+oXZzpze30UeD3/vPcPfz34F4FnXo8hNI7JqaOZ0ne0Y7/D9tChML8Kic3qvL0TbW9u+4l9qqc5G9u7vUv0qToNDr8D8bMprDFzLOJ1Ag39KCx8mDCdmeeDnmfgpQOZ/eVsr+xTWZmUYOMkLkkz8cnjRsj9A4m5sl3/A2/rkyu65bXX3HaJBGxGNBo/0qPeIC/ke5Yde5M7R9/JzdtvdmprtVod73vie3dGn3pP703v6b1ZcJ+ZhMJo+k3pi0yhAkVsm/rapT+n7twnKVB5CIm5DACNn4ZXI4tY+UcIxp5SCnfnk78rn8FXDXZkmJ6s7c1tb2uf7HY7w24aRmZVKIMyb2NSL5DJgMHPwOBnmi3A3C0+pw6wsb3bPd2n8EBRb9IsKyU/346f1o/78+9vtv2JtrsaP0+mTwo/BTKFjFqFiMqUW7X06CET1+JfY0WNyRnb29Snjtju6c+pI7Z3mT793guizoaJvztt10iiKaGcg9mFzBw1ABB9KtpXxNqn1hI9JJrw1PCTsr0tfbKarVQVVBEQGYBM0bC9PuDYP8jE5MTJ6NQ65HJ5s+dh8y2g7Q8pd3Ttz6k7XnsnbjcUwC/RyHrfCiPfpU8fOH4cDh+G0aOd+7Tt/W0EJwUz8taRzdruavxsT58kUgmj7hxFj3E9Wp3DmkyQmwsSCfTqVTe2AsgU0ONipCFDcWxcNRkK1yJRaJANfQWSb2xie3N9as92mUwG5kr4PUlsuMzebF/dtf2krr3+/wZrNbLoKVDXrrn2icFC5SmzItPpfWQyGRajBbvNjsJP4bQdICZGfDaFheK3N32fig2iCF9kQGSTPrXVxsbbS0pEHyMimu9rlx0jWtje2X2Ki4MKvx18k74Gv7Q+9DnWh69nfM35H53PsBuGeWWfogZFkXXYBHIFRlkR2CAysNF1JwuGqauF1O6ePU3GT2/s06nY4q7tHu2TqUKM8wHxbMnZwo6AlRTr+qHXn49MBl+f+zV52/O4P/9+r+lTymXDObQ2EKXaikxWN38b8RYMfAKZOtxJYrElG9u73XftdXCfKg4iS/sf9JgLoSOZlDSJGDOssUBJLZz36XltOk9n9MlkAou8EoDQwGAYcr9b1mTqn907i+Zs9jiJiYlERUWxYsUKx7bKyko2bdrEmDFjABgzZgzl5eVs27bN0WblypXYbDZGjx7taLN27VrMZrOjzfLly0lJSSE4ONjRpvH71Lepf5/28NLor0kovBHFu2/y0+U/tfv4ziYsJYyeFwzEpNag04lJX4s0Uwzah4epLYRDb0DpNsL8RUVus7yEvDxQ+Cnod1E/YkbGeNbGFji+vYSAshzCQ+yo5WrocRH0utbTZvk4FZYOh8WDCAgAZV0B8+LqUpdNJRIJUVFRnXrzc0XPgrsYmPkew+MGQMlWqO48rW0fHUDoSLio0GmhLDEyDAlSSkpgy3tb+PWaXzGUGDxopEAikTDt1WlIRooaxcHBrRzgo+tTvg+23QPFmx1FzO1SC5n5Ve06jTvHz7MePYtb99yK0i+AqLI5hFdMJ7Y+9qHHXOgx75Tfw0cXRCKBvvdD/Owmu0JVIjDzcH6u0/be5/bmll23kDglsckx7qT0SCmvx7/OmqfWOG0vLxe/++tGseKqFfw4/8fmT2I1QdqnULi24wz14T5UodD7VoiYCECfPmLz50deYeaCmfxx+A9H02vXX8vFP1/c7KncMX76hfgx480Z9J/Xv8V2/938X0Z+OIqM0A8JCjrhPq8bCOO+hagpDdvi50LkFLFP3XwgsltQaOCMz2DC76029Rj+sXDGJ8JWEIXomiFBlwDA8Qrnefzxtcf5P/X/seOTHS6Pq68aU14uHLSexlRtYtHNi9i3cB8F1SJAPTIwsvkDijfD76mQ1foaVH28e3i4Oyz10YT0r+HQf4mJgWLNX/xYfQ/f7v2WkOQQRt05irC+YZ62sFnOfPBM4h4UCmgGmZD3DA9oeqF4y/O7jzaw8UpY1AvsdlZnrGaJ+SFygxc65koxI2JInNSx87X2Mujeqewals73qQqmfzVdbJTKwS/SpcPPRxfBkA0HXoLSBh9OaKj4XVra4q290/n5soWcs0iMfSH+WredVyKRNCkv15F4NNOvqqqKo0ePOl6np6ezc+dOh77q3XffzbPPPkvv3r1JTEzkscceIyYmxiEB2rdvX6ZPn86NN97I+++/j9ls5vbbb+eSSy4hJkY4OC677DKeeuoprr/+ev7973+zd+9e3nzzTV5//XXH+951111MmDCBV199lZkzZ/Ltt9+ydetWPvzww3b3KShIClIpxsBQNPEdm6bpDlIuSEEfk8IPT4FOBxuyNuCv8CclNAU/hV9Dw7KdYMgTD1hyv2bO5sNjhAyDWZmgCiO0pgQAk6yE/Hw7IGHud3M9a18r7P9gPX3/3ol00sOA0tPm+HAHMTPAXIW/PyisYkG7QC+cflve3YIyUMngqwYDIsKmpczqjiZrQxa52/KoKRkIcj+CgoAVEyF8HExa6jG7fLif+kllcTFccsMwep/bG4W/ouWDOpHsklJMMtAGawA5HPsENH0hfKynTfPhbgw5IlhH2xe/0JEopSpMNiOZxaVAEMfXHSd/Rz6j7hjV4oJKR4yfQabejDj2I35+Ym4IwNCXWzrER3dn8P+53BwdGM3hMjhekue03T/UH/9Qf5fHuBNFgEJkWY3v4bS9fiHLcf22hEwJ8/Rg1rvbPB8dgVQOI991vKx3+h2p2kHmkcVMSZzCTGYCENIrpOVTdeL8c2P2RnaXbCFFMYvEmDYE2qbcIX46i6SrO++9TpVjn0DecjhzgcvF3566ngBklGc4bdf20DLg0gEE93IdWRUYCGo11NZCURENQS8ewlBiYPuH21EGKCmIrnP6BbSwUGiuECumUnWL562uhry6IbtnTzcZ68OZo++B/iixA25HZRGfWX5VPiHJIcx4c4aHjWud+ipGBkSGabh/I6ff4XfBWoM09T6PPr/7aAdxF4ImBew2Qv3Fw7BJUUyFKCvLpKcnec62ZjAawSIV8zKVvE5hr3S7UB0J9C4HpY92EDYGzj8KfmLsOFp6lLWZf1OkjSK8YhoLr/iV0IRApjw3pZUTdTwWow2JzQJAtMwOG66EHvMh7vxTOq9UKu1Up59HXeRbt25l6NChDB06FIB7772XoUOH8vjjjwPw4IMPcscdd3DTTTcxcuRIqqqqWLp0KWp1w0Tm66+/JjU1lSlTpnDuuecybtw4J2edVqtl2bJlpKenM3z4cO677z4ef/xxbrrpJkebsWPHsmDBAj788EMGDx7MDz/8wC+//MKAAQPa3Sd/fwsmWRlHzrics188+2T/NZ1KmVA+IzgYLvr+IoZ+MJT9RfudGx35AFafCxbfA7FXIlNDQDzI/Qj1Ezdyu9RCRl6lhw1rG7LB/dk7LISf/C7hu61vw2+9YP+LnjbLx6kw+P9gxJvIZOCPuCaLqoRDet1z69jyzhZHU6vVyrFjx5poYXcWhxYdYumdSzDaDmCSlRIYYIWBT0PSdR6xx4cbyfoJ8pY5Xv5V/j7bkuazo2oxsSNjSZ2V6hVOv+JDxXxzwTds3P44y4aGslT/ElgMsOkGId3so/sRPk4E6yRehUQicWT7ldaUUVsLO/+3k6V3LcVYYWzxNO4cP7e+v5U93+whJ0e8jotrw8K0j9OahFCR6Zend3b62W12DGUGaitqO/T9dQk6Zrw5g+RpyU7bKyrAhgVZoGuFgSbIVKD23swLH83TowcolSCtrRtDDQ2fuaHMQMmRkmaPdcf4eWzZMb6b8x35u/JbbLcjT2SXaWqGOjtarLWwYgqkfXbSNpxW2O1QsEZkCRRtcNkkUZdI75De9Ant47Rd11PHRQsuoveM3i6Pq5e7BCHx6Wk0cRoeKH6ACY9PoMIoVudbdPpFnw3nH4LYc1s877Fj4nd4OHRwKaHTl5EfwOQVREWByiwWt/P0LY8R3sKf9/5JyTKRhVONi0y/I+/Ckfew2mwefX730Q56XSeC96SyBlUwWYkjQMrbqCmpYe0tC+hxTCxUBymDxI7VM0TWoo+uizwAgnqJ38DqjNVcv+gacmLEesfxVWlkb8z2pIUOJn1wMQsvWQ1AuKQWMr6CygOnfF6r1UpaWtopn6eteDTTb+LEiU2KLDZGIpHw9NNP8/TTTzfbJiQkhAULFrT4PoMGDWLdunUttpk3bx7z5p26bNHQr+IwDDVw9s4CjMYIXJT98yrWPL2Gg5sMIJ2ORmujSC9u7FGBJ0Tt9LoeQoaDMtQDVvpoFbsNqtJBpsLPPw61zJ9aaw1ZJSWAlq3vbyVnSw6zPpnlaUtdUhOTzPZRGaSZf+ZYcbiQbpH6Mv66C0EKZ3nPS3+7FJXGeXDU6z0XUDDy1pFYB2h49sBorDIJSpUR+t7rMXt8uJFNN4JuEESfA8Dhmk3khSwkOG8odvu5SCRicVoi9axno7qgmqNLjyKdJPSkwjVakMhg0jIR0eij+yH3Fz91hPqHkF+dh1leSn4+nHHPGQy5dkibnNLuGj/XPLUGXaKOwHv6YUdKTEzd9yJ/BRx5D/o/IpQFfJx+HHwTsn8R2e+yhvt378gYJEdkVNYYsFigvmxeRVYFb/Z8kzH3jeGcV87pdHPLykDvt4drDgzjrQ+Hse2mbc03Lt8LlhoIGQpSzweB+GgDe56Gsu1w1i/I5dCrF+ypaOr0W/yvxez9di+Pmh51qvnYmFMdP0uPlnLo10OMua/5siA15hoOlRwCQHui068qDUo2O+RKnVhQNwZf5kV6V55GIoEz/tdiyZHE4EQO33H4pE4fGQmZmQ3yl55EIpU4Mqafm/IcT0x4ArPN3MpRrVPv9Ovt2vfpwx3ohNxvkB0CaXD62e12Fs5bSOSgSCY8PsGTFrrEbrOz6c1NKAamQPxwRgddhF/sYBJ1jTKrzl4vFMDw7PO7j5Oj3ulnkhc7nH4ZazLYv3A/Y+8fi66nzmO21WPSmyjalEZAqnAoByoDxY7+/wFlyxn8Prwcuw0MuWKdwy/aEfRqU4m52/gf7+aMMd4RcWoyQWBtCjK5HXXkWdC/GnDPfKyqqn3lPE4FnxiumwlQCY+1Ur+PJff8SVl6mYctaplDvx6iYrN4CFFoS7DaxcAaERDh3DB0BCTf4Kvp563YrbAoGXY8AIBOJZyzxXVSn8fXHGfnpzuxmrwzEquqCswyEcFo9+8BM3ZA6j0etsrHKZH1M2y8BmqL0NY5/crqFmKih0UTkuw9EzZtDy3GZDUWhQW1PcRXm6A7MeZzGPyc42VcsLi3GaSF5Owt46XQl1j52EpPWecg4awEHjU+yvaB6QBE6bRCci76bHH/9dH9sNvEYm+NiGasf+gxy8rJz4fIgZEkjE9Apuy8edcVf17Bee+fxxdpz7N4mJKVyrr7sP6wyJq1er7+pQ8PUZUG5bvBVO60+YGJt3HRfiMDjr/nyBAFIe859IahxI7uWH28TW9t4vu53zfJKKyogIoA4ejTqXUtn+Tgq7BsNFg67wHcxylSvgcK14lxFOG8UFiaOv1SZqUw7pFx2Cy2DjNl5L9G8pj5MeLHxDfbZnfBbmx2G2pLJGpztLPTT9sP5pZD3/uaHtjzcoia6m6Tuz6nsB7x18N/8ddDfzW7v77GXVHRSb+F2zCUGijcW4ipSgSEqeSqhsVvV+x5CnKXtHre+uo6vXq5w0ofLrFZwViKxG4mViucfqW1xVhsFtJXppO/w0uz/iRwf8H9KGYLieTLY5/kqzlf0Suk0cWi1IG2r2fs83FyZC+C1TOh4oBDFcwkb5D3LNpXxJZ3tnjN2rWup44Jyx9l6xjhVHZk+qXcCYlXeNAyH6eMzQy/xMP2+wEIVovgZrNczN1Ky7xnHW7/F1uYtOkF5hfvYUjUkLqA3QBPm9VufE4/NxMdIGRulNVZ7HrvHwp2e0GYWAvcsPkG5LffAoDdX9ga4heCQuaLdO1SSBXQ72Gh1w2E+YkInlKDcPrNfH8mj1Q/glThnV95y5ffMvFPoS+iVbuvSKoPD1K+G9I/h9oCtHUZwmVGcT3aLDaqi6pbzPTuTCxGC7llwjY1IVCwGpaOgNw/PWuYj1Mn9jwIb4i+jwoSqzkmeRE1kgAiBkagjfeOMae2VoJRWg5ATKhWTIq95DviowOwGoWU9c6HAVh06SKeU5uILp/jqLVjs9owG049qr+tRA6KJHJQJIX6cuxSC9rAurlg71vhEiOEju40W3x4GcPfgLkl4OcsLaeSK0nuJRbgjxxp2K4MVHLBRxfQf17/DjWrYE8BB385iMLP+bmlvBwq/IXTb3j08JZPkng1DH3Fl1XdlRj3nbge6+q59ekDyjqnX1ltw6LlgEsGMOX/pjS5PtyNRCppUTFgW664FoOqhyKRCElSJ6Qyp8xvB2O/gsnL3WhpNyLndzjwWqvNTnzWOLrkKPt/2N9Ma5HpB96R6Xf4j8O8N/A90lemt964thj2PAmZC1ttWu/0S05uuZ2PU2Dfc/BjKFTsIy40FIldhh07RTVFPFj8IBf/fLGnLXSJRCLBP8wfc51zWX1iecjqTKg8JJyaProOhhyh2lFb4Mj0s8grKS03Y7fDoCsGcW/uvSSMT/CwoQ0YjWCR1zn9VEEetsaH25CpIPVeiBEy1PVBr0apcPrl7SzgyOIjzR7eWdhtdrY/vZiwzO0oldSpMmwRz+9dDO/0AHRhIgPFTDEz0cg5f9xJ73O9WzdBKpNSUStkeqzqZgpE2+3wYzhsuunEw314E0OegwQxgVxw4Q9M2ZWFpmQKJhOotWoU/gqvzWCyGc1I6/Tgk6zFQkbK4KURcD7aRup9ME8P2v708EtlSNoX3J74HgCLblrEKxGvYNKLyFWJREJ8fLzHrs/vZn9HwYXrAQiQBotof2Mx0HGR4T46EZvV4Tyrz2I3ygupMCi5ZvU1jLjF85l0JYdL2P3LMSTWagAignRw8A343l8ULffR/ZCpod+/IVYUA9eqtcRGi4Xp/HwoPljMs6pnWf/8+hZP467x02axUVtRi91mp6ymHKhzPtcjVYDUo1UBfHiSFq6veom4Ix54Rr/gowt4tPZRp4xYi0Vk+pUHbAVgREwrY3zkRNdZVj68F4nzEkafPqCwCqdtSU0b6zjinvEzZ3MOGWsyWmyzLlOUGQmuGktMDM7lPwpW+4LMToZjH8POB0WAlAueWPUE4S+H89LfLzltv2bNNdx+6PZmT+tNNf0iB0Yy4ckJhPcLZ/7C+Vz181XkVzXzfKwMhpkHRBBwC1RXQ26u+Nvn9OtAQkdB8k0gDyIqQobSLC6s/Kp8j5cUaAmj3kjR/iJqK4xYJUaqJfmYrKaGBoffht9Toeqox5/ffbSD5JvhklqInIhOrUNadw+tspVQWwsqjYqg6CCkcu9wD1QXVlO44SjKGqEwEqQMgpKtsGS4UB7x0bUZ9iokXg40OP2qbaXYsVOwcB0LZi7oUIWGtjLq3avJ632WmLMdfgf+HOWQNj4VJBIJcXFxp25gG/GOb3U3IjpIZPrpA/OxaYObrR/gDdisNjLWZFCZWQ6AUVHn9As8welnNUDwEPDvWIkeH+5jQGwSgbY4pHYFlZWikH3OlhwMZd4pzVU4/Up+vegfAJJqD8P2u6HWC0IsfZw8ikDxI5EQHhhKXOmVpMjPBqDnpJ4Mv3k4dptwxEilUkJDQ5FKPXNL6jG+B4YRIpQxSBYissNmZUDMDI/Y48ON7HgQvlODuRxocPqZFEWUlHjQrhPY9eUull72FX7VYoKrVWvBPx4iJoE6qpWjfXRJJBIY8gIkzHdsihZTSPLzITA6kJTzUwhNabmWsrvGz5LDJbyoe5FlD6+i2loOQHy4TuzMW+ZzPp/umMqFbHf5PqfNZquZr4zz2ZAynv1Hqp32Lb17KX/e2/HOjBMXqfLywGI3offbDbQh089H18OQB9m/Qo3wXkRFgbauzo/e0CD1mvZXGl9N+4rsTdkuT+OO8XPlf1byzfnftNgmKjCKEEUsofqzSDgxkWL347DRJ1fWbgY9CzN2irpALrBjp7immIzyDKftaq0aqaz5z9ubnH5RQ6KY+MRENEkaftj/A1/u/hIJzThYpDLQpoKm5YDztDTxOzwcNBo3G+yjgZhpMOoDCOpFeDgMTf+ae4L+ISU0hfxd+RxadMjTFroke2M27/Z/F8vOPVT672D+5mhS/pvS0CB6OvR9AAKTPf787qMdNHLMyqQy/rryL6Yc3Y7SGkJ5OVjNVooPFaPP844ajTmbczj65Nf0PtqXvvJz6R3aWwRlm0rBZvG0eT7cSL3Tz4oZq7SG2v7DmPW/WR5XBJNIJVQny/ht3FC+ChgKcbNg8POgjmz94FaQSqWEhHReqSPfCO1mIvzrsggUeZQcr6L4YLGHLWoeQ6mBzyd+jnKLiCKvRjhZogJPWGCU+wtpkYFPdLaJPtrDltthvcj0k0gaJvKVlXDot0N8POpjMtdnetDA5jEYGmr6lcXOhalrIci7s2R9tIKlGkq3gSGfgDrp6+q69cDBVw7mvPfPQ60Tjjar1crBgwexWj0jFTL+4fFkXiomFkEKn7xXtyKot5CPsNQAEO4v5D2N8kJKSmDPN3v447Y/PD6x7Du7LwMfnkmZThTF0qq00PMSmLQY/GM8apuPzmFb7jbeyLqC/XEPkJ8vFiYv/vliBl0+qMXj3DV+KvwVDLluCIqesVjk5QCEB+rEzo1XwtY7Tun8Pro4NVmwbg5kfu+0WS6V80/pH5QGredgTh7mRkk36SvSObbsWIeZZLfb2f/Dfgr3Oa/OHz8OevU+bFITOrWOpOCk5k9StBF+joGMlp02PryMwrWw9kIoEs+wEgmMjhvFuduMvNFnj6OZodRA1oYsqgurXZ7GHePnmPvHMOPtloPE3pj+Bo/6ZRFSdRaJiSfsHPQUjHz3pN//tEU3QPxIXC9nJWiFdzWjIsNpe1V+FRlrMjDqXUt01Tv9SktF1rA3UFxTjB07UonUIc3XhKp0MFW0ei6ftGfnEx4OYfpJBJSNJkAZwNpn1vLtBd9is3o+k+VEdD11nPXYWdSGxGKqq7FVvygPQNQUGPoSSGUef3730Q7MeshdCpWHAZiUOImeqqFI7Uohh368gndS32HTW5s8a2cdkYMiibzxfPxlN3Gb7g/O63OeUGWYle4ULOmji7Lj37BefI7+Cn8UUqF0Y5aVUqxJYsg1QzyePGW32SmpKsWozKNKmgsRZ0H/h0Dud8rntlqtHD582A1Wtg2f08/N1DvMahW57L33UxbOa11X3VPI1XKmvzWdkoh+AJzV8yxenPoiF/f3To1xH62gPwIVokbBuuPr2B19H9mhX1BRAbEjY5nywhTCUpp5UPAgVrMVvz2bic4V2vF+miSIGO+6toWPrkPpNlEXL3MhAQFQpFnGiuIvKDO4LhBdW1vrcntnUWEUdgUpgoXMUtoXvnoF3YHkG2HCr45M9fpMP7OsjKIiO2l/pbH13a3Ulnv2+oseFk3I9KFE6C+nr/US5wdsH92XbXfDuosAUYdqcdbXFGr/oLAQbO1YC3LH+KnrqWPWJ7MgJcURhOOosTv8bej30Cm/h48uTGASjPkCEi5x2iyRSIgOEs8+NdICMjIa9t20/Sb+tfdfHWaSscLIwnkL2fjaRqftmZkN0p7Do4e3LD1mN4NfLCh1HWanjw4gbAyc8TmENdQZTe0jR2pX0ngdpf/8/jysf5iU81NcnERwquNn8rRkhlw9pNV2GRkSJEjo2fOEHZGToMe8U7LhtMWsh9oil7t66noCNMn02/XlLj6f+DlF+10fp9WCUilU4fNOXcXrlFjz9Bo+m/gZOYUiICzMPwyZtJmF0H+ugV97tloL2uf06ySqs2DDFZC5kHARb0hR3SU38raRzP5qNnhh2e7QPqFMenoS1ZpozDLh9Av1a15xwtPP7z7aSPVxWD0DjjcEOOl04ndFBQREBjDu4XEkTjoxKsUzaHtoUY8dhjEw1FkO20f3oHwPlGwGxHPEgosW8Pm0P1BaQ71GialwbyH5c35h/Nrx+KFz+/k7c+z0FedwM6mhqQz1u5DarLEETB7N8LHeq3GtClIx7JbRlC0Tr8ckDOfs/i4kcIr/gdzFkHQtBHrHjcCHCyY3SCjtyN/BLv/XiNbOp7LyKsInhBPeL9yDxjVPTamR2F1LOKfsSda+PonwgFrhbGnuocZH1yAwGQY/B2FjCAiA3Qk3sqkik1tLNxGXF8eGVzYw8raRXlEwesldS0jJSGJf3C0MjBkv6hXk/wWJV3raNB9uJlYTy0+jS/jkHR0lQyTc8H9TmPrCVEfWqScxGmQMynyf0dEQpAI23wohwyH5Bk+b5qOj0B+BKpEJVb9AaVBlYLbYKS6WkPXLNjLXZXLhFxd2Ws2UnBwwy8oB0Kl1YqMvqtaHPKDZe2JUYBRpZWkY5fkcOdJQ46+jo3RlShkXfXMRmnhnjbqsLAisTeXskJuYldpypiwRZ8H0LR1opY8OIaAHJF3ltKn+uuvE4Ok2kV+VT5g6ksxMMYY7yXva7S3Wy/TRAoYC+DkKkm+BUe812V1/Tz1efhy73e64hyZNTWL6W9PRxLnWtpRIIDUVdu+G7dshPr7DetAqVQVVFO0rotgqlKMiA1qQFesxHyKntHo9HatLvvY5/ToYmwkyvgb/HoRHzqPSbxeZ9lX8ejCJWZMu8LR1rWI0gvnETL/CtbD5Jhj6iiiF4aPr4B8Poz6E0JEALD+2nN2a7dQETKC8/AxUQSqmPDfFw0Y6YzKBHRsqVV2eUsFq4bxMuFjURffRdZm02Onl3H5zqa2FhTZQH9nDGz1XMufr2fQ4s4eHDARFgALrSB3FYcX4SbXw1wSxvnnGJx6z6WTxZfq5mTN7nMkjST/Tq+ABbCNHM+r2UZ42qUVqahr+9m8usapwHex9BgzNFI724XXUR2SZ5SVUtK704VFsCiWHzriKwsRRhGkCkK+cIgpE++ja+MdA/4chdAQBAaCwigeGUkMpNSU17PtuHyWHvSOUZ+f/dhJ80I+Bme8xKXKucFae9atvIaY7YK4Sdf3SvgBAKpGSFB2CBCklJRAYFUhAeIDHi9B/N/s7dt/1KVB3L7bUwNH3oWCVR+3y0cFM/APOOwhAD20PJEiwSg0Y5QXk54vaKru/2o1Jb+pwUw7/fpifr/yZ7N2lhFSNY2DgpKZy7z58uMgiqV+INioKnBwupUdLOfrnUWyWjpEwU/grGHDJgCaLAsePQ2jVeJ4f+wG3jbqtQ97bh/fRuzfsTriZH1QzOVKQBYDZYObQokPk7+yYZ1irycprsa+x7IFlzbYZ+8lYIl+NoESxA7Va1B90ULEfFmrh4JsdYl+3Rh0BSddB+Jkud8dr45EgwWAxUFTTkNUXPTSa0XeMRhPbfEG7UXXLN1s8HA8w852ZPFD0AIU1QsI4MrAFp1+f22Dg4y2er6ZGBPaAz+nX4QQmwsUGGPIcYWFQErSG3TH38Pn2rz1tWYts+2gbn0/+HEtJRVN5T7Me7DaQ+dSYuhxKrVC/CR4CwDd7v2G5/SGKg1ZSXu5Ry1yy5b0tFPznLXZGDeKGdH92F+yGYx+LjGYPl+Tw0TGo1RAQAFa5CoXGD7vVs59zSK8QSq4P5UC/A/jLtGA1gs3c+oFeiM/p52akUqlTqrQ3c/CXg3wx5j0CS46jVsPG7L/ZkbcDg9ng3LD3LXDuXgge7BlDfbSN8j2Q8S1Yax16/0Z5ERUVUFteyydjPmHNM2s8bGRTTFY5+rBELMERKBRA7EzoMdfTZvlwIwEBoLQIR3RJTQlJU5J41Pgow64fBohxMykpyWOFwO/LvQ/bReKa8/MDggeJAuw+uj4yFRx8FbJ/cWwKq1M5LikR8sIlh0vQ53q2cLlKq8Lup8QkK0Wltgp543l6GP6GR+3y0XkoZUriNHEAGFTp5OfDtDem8ajxUVSa5rVt3DV+5u/MZ/dXuynINjMk4398OmElySHJkP4l/BwLBd43f/DRyWy4En4Ma7Lg0pzTb+NrG/l6+tcYSk94ruhALBbIzRV/92hLgPCux8Tc2UfXwm6DX5PENVlHSAiUBC+lULuYrYeELqOx0si3F3zLto+2uTzNqY6f5hoz2h5aVEGux+jsymzSy9Mpry0lwJhMQsKJ8WR2CBkJftEn9f6nNRKJiLhPvMLlbqVMSaxGSLufKPHZGvVOv717G+qRe5KC6gKglUy/NlCf5RceDprmfZ4+3IFE6shGUiggVCW8/dkV+exbuI/X4l4jbUWaJy10SVVeFfk780Wmn+wEp1/sTDj/MERNBjz//O7j5KlfK2ycIPDLNb/w0xU/edCqBmRKGShVmJSVmOwG/OR+0P8RmLDILTXVfHiYyiOQ9ROYKwHYmLWRz3d+DuH7qYjsw7gvbyLhLM+rgVWaywEIkOlg2j8w9gu3nFcqldKzidZ7x+Ebod2MRCJBo7FjkpVScSCDT8Z+wqFFhzxtlkssRgvmGgt2iRR/f5j/w3yGfTiMfUX7nBsqgkDX31djzds59j/YcCnUFjrqVpnkRVRWgtxPTkVWRadkC7SXmmo7Bnk2W3vN4V9//AsG/x8Med7TZvk4Vex2WD4ett4lMv0sDZl+UrlUTObqEOOmxmPZVspAJYW2EkyyUtR+ti4bxePDBVIFnH8UzmyoYfDlsVfZljSfLOk6sneW8N+U/7L1g60eNBIu/OxC8q6PYNnQUF4sr5PZVgSC2jtlmX24icrDwuFgEjVFE4OFhHqNKoO8PFBr1U5jpSvcNX6O/894Hq56hCyjuOZiYup2yPzAPw5UvjqTpz1BvUVWjc15LlmfEWpU5JOdDYY6H9+ASwdw3ofnofBXdIg5G17ZwGuxr1G4r9CxLTcXDJRSG7ydQF0r9TIsBtj3f5D1Q4fY56MDkUhFJs0JzjKNQoxT+9PFmOoX4sesz2Yx9Nqhrk9ziuOnWqfm+o3XM+HxCS73rzu+DoBE9TDktiBiY09ooBsAU/7ySSh3EGclnMXUpKlIJQ1LXoZSA+8PeZ9VjzevpBAdDbGxYLXCjh2dYalr9n2/j7QVaVTUilX5Zp1+Rz+GvyaJhdQW8NXz62RKt0PxJgBH7duCqnxUGhWaOA3/z95Zh0d1bX34Hc3MxN1DcIK7u5TS0lKoQAt1u3X3ftV763J762600BYoUMGhUNyLOyGBuGcmyej5/tiZCUMChGQkhPM+T5509sw5Z1EOe87ea63fT6lueluxw54bxqP5T1CpDcWiFoo8p/MZ9/f6XeYcWdALVovvGmfSz6IucHX6lRwtofhIsZ+Cc6fnrT1R3HUH+dHHAQgOCIbQjrKsbHPh2EzhaW8UhQ/vbXiPm+bdRHH4EgAKCvwZnOD4+uOEzigiJjeGIHWoR8/tnDt9RdP7pjnPsdvtTFyZyuIekWRbjlJ8uJjKQt9VuJ4LnSd3ZvT8+zBFJGMIdJBnEovmWnJOxnTXhpRME6bl9TD4F9BGEB0oNu0s6nxKSyXUAWoePv4wY94Y4+cga3N06WEGz/2SqENHmb9/vr/DkfEUCgWYC8FaWivpB+KL9MRGoTFjt9vZuXMndrvd52HaLXayNmcx1zyOxT0iOVK5HGZqYcMdPo9FxksEtRQdf9WsPbGS7IhfMOr2Yg8KZdCTg0gdluq/+KopqSoBIFgTJjwLircLKQmZ5svxeaJYp0y0R7UMq076aUWnn7XCyvH1xyk+evpnME/NnwqFAodKg9mqQEIiwrnHk3IVjN0AYV0adX6ZZkCX52DYfLf5FITknFqpRquzIEk13SQthrSg1+290AZpvRKOLkxHcGIw+oiaqu+MDMgPXcTS1r0Y8/3oM59ApYMrjkOPN7wSn4yXGbWs1t9dhCEcgEMnxLOmSqOi+43dSeidUOtw8P7z56pjqwBoqRoq4pNrJzzLiT9g+Rghk1oHP0z6gSXXL6F3Qm/XmDZYi8VoQTqLRFxTkPj8854/WfXSKl4e+TLmZ828OOLFuj9ozoOiLaANO+P55KSfj1kzRXjgAUlhYn+toCqHNmPbcNv625rE2qMuzNVLj8jyoUztfD1dY6u9cQ9/BUenuz7nz/W7TAPQRYNWfEc6rYAs6kJX0u+mlTdx69pb/RRcbUxWEyjEPB2sDZaLspsTyVfAwB+E1yQQrhP3pcJQgsJh58B369g3d58fA4TcnblEr5WIKehCR3047H8fSvd65Nx2u53du3ef/YMeQk76eYEIg3iizwip4pGcR+l+U3f/BnQGnJ5+SkMxNocNwNUl5uKvcbC4br18mSZERA+xOacJItogkn6S0kZeWYl/4zoLDp2e3KRIisOLaaM3wJprIXOuv8OS8QTj98CAb4S8p9096ffjpT+y5LElro/6a8FQdryMz/t8TtcNqQDEBhogZTJE9PRLPDJewFwkKm2rE2iuTmhNHmXmAEa/OpqWI1v6LTyH3cGq/6zCsUt4DgVrwuDgp7CgBxiP+i0uGR+QeBkMngXBYgeuZVhLlCixqkvIyRGeaF8O+JJ/vv3njKfxxPyZsz2Ho2uyMOr28mcvDV0/b9foc8pcGNza41bMz5q5JfJrAA6eudnEY/S8rSe3b7yd4Phg19ixY1Cm3wFAl5izJKoVCuE/HNTKm2HK+JD4UPGsmZ5bVO9jGjN/5u7IZfVrq0/bHbEqQyT94synSfql/wg7XgD7WbpSZerGUgIF68CUWe9DVBoV9x+6n5Evjzzj5/r0Eb83bwaHd2xJz8rE6RMZ/tJwQMiVBmmD6v5gp6fh6tKzqkM4CzLkpJ+P6Pyc+LsBWkaLpF+Fo4wKa4U/ozojhxcfZu98UYiWWngn30/6jtGtqgtodv8H9r7l9nk54XceMWIh9P0UcO/0a4qWVPt/249t9VqUdiVKlBjUevglRHSHyZz/hHWB1OsgQCSfw3RhYlxXjKRQkPf9Ev757sxrX2/T/cbuBL7wKC2NC7khZihsuR8KN3js/L6cO+WknxdIDBFSIyZllkvipily4I8DHPllK0gSDoPQig/XhaNVnVKR2/JGaH2LHyKUaSgB6gCC1KJlONcozMsPLjjo98mzLnStEtk0PJFjqcdoHaAT7d7lB85+oMx5g+j0q/b0q076jX5jNAMeGeDPsADhpTbylZHsbyuqhCMjUmHwTOFlKtM82Pc2LO4PRrHb4SyKMKvzm4R8hLnUzIpnV6DfKXTtQwPCIOFi6PICBNbHlErmvCW0A6Rc6Vr0PDrwUfZdX0XH42+QkwNhLcMY/fpo2ozz/g7dgvsXMP+aH7CqSpAUduxS9WJk9ytw5FuvX1/mPMCUAdseh5zlbsMalQalQknbtuK109fv8OLDvN/ufQ787rtnuowMKK9O+rm6E05H2X6oOF7Lo1DmPCF3BWx7QiR+qkmOElm14spi10bm92O+58sBX3olhIzVGSx7almd3dj5pnz25Itny7DSwQBERp56gl9g979B4R0J3GZPi8nC//gsPtzOwuZzIS1NrF/KynDzKvUlbca2qX832FkkFisq4MSJ6vPKST/f0HKauEeB5JhglHbRlX485zirX1/NwT99VCFzDqx4bgUrHv4DgICAU26rofOEj6bMeU9N0q+m0y93Ry7/fPcPFpP/7YB2fL+DoOXrkBQSenUQCuyQdAVE9vN3aDJewJn0s2tKQKFEefstXPzfi/0ak0qrwqoNRFKpMeoGwagVEH/mZ42mipz08wJJoUJCxKzJZs8fR09rHu5vNn+8mUPv/wkKBXZdtUF0UB1a8Z2ehLRHfRydzDmTuwJmR8ERYTD666XrGf3PCShuDcDaN9ay6KFF/oywTiorwaYSK/OsgESYbIZ29/k5KhmPkLsCDn2BwQAxpRfT/ch33NH1fkBotbe/vL2fAwRDpIFuD3fjaCuREIoNDfNvQDKeJ34cdHvVJWlS43maR2EhLH92udc2BOuDNljLndvuZF8/EwDh+nCIGQpdnpe9dC8wArWBJCeIzV+jEayKAAY9Poikfklev3a/B/rR6f4RWNUlQPUCTJJg96uQPv2Mx8pcIFjLYO+bkPdXnW+fmvRTBahQ69ReC2fZ08vY9dMut7HMTCgz1DPpt+UB+K0tSH5q45FpHHmrYe8bLk8YgOgg8T1vURW57sPgxGDCWoZ5JYTO13bmto23kdjnVLM+WJ2xWnwmpjMVhSLbV6vTr//XcMlOUJ7Zu1XmNCjVZ0x2rc1cS/Sb0fT81F2949DCQ6x7d90ZT61WQ8/qw/wp8QkwedZkrv/1erLKs2q/WbBRFOdUnbmKLbO6GTIyEnxoJSRTTWysAp1NdPvllOWw7Mll7JldtyytPxn92mj6vnQJAFJgLmbbSTYDYZ0hopefIpNpNMfni7kCiDSI7ySruoDycrDZYOeMncy9cS7lJ8r9GSUAo14dxeGrBiEpJQLVwaDUwKAZ0PFxf4cm4wkKN8GvCXDwE6B67wOwKEsAKDIkEZriWR+9c6XseBmW4/koHHYUugiIHV7LR/p8QU76eRilUkl8kLgZqjTZbPt4PX/e8ycOe9NbUI5+fTRJj18HgEUrZMVOaxAt0/TRRkBoZ5eef68WaeisCZjKVTgc4svzmjlNzyg+66/9dN6UTVB5EKEBoaDSglp/9gNlmj4HPoSNd6DVOIh0dCKp6Ho6hPSp9TGlUkn79u1RKv3zlZRXLiq0lQ4t0eYNsOlu0QEg0zyIGSyKV6of1GrkPUWnX0VBBaZ8E9YK/3gFqDQq4rrHkR8i7sMIfZhf4pDxA2X7YXY07PqPa0ing7Aw8d85OWc/hafmz45XdiR+fG9XEY5LauXS3dDn40adW6aZENwOLjvokitzYnPYuOrnq7hr0xBsynLy8qC0FFKHpXLXjrtoN97zUrG2KhurX13N/rk139U2m5B1rNIeB0Sy5YykToPO/ycnXM5X2twO4w+4+Y1G6EVWza4yupJ+V3xzBVf+WLckWGPnT324nsQ+iQSEBNR6r31Ue54a/BQ3d7+Fomq10VpJP20YhKY16Noy1eT9DRmz6nwrTBdGQUUBmWXu8p/bv97O4ocXY7eeWV7LKfG5caNHIj0njq8/zn8M/2H9/9Yza88spu+YjoI6EpxHvoZ/nhG+fmegsFD8jj6zAqiMJ9n7FsxvA5U5REdDt6PfcnHmBnq26cltG287q8SsP0gdnkr0oPZIOJidGo/uPzpyjDnCT81S6tYZ7+/1u8w5cmymmCvsFlqEtmDR1CUMOLACEB3NXad2ZfLcyQTFn0ZG2IdEtI7AFJFIdOk4+sUO93c4Mp5GHQyBqaARFSjONWcVYi+kMN+OMcfop+AEa95YAx99xKbUvhysWOrRAkGlUklbZ6WkD5BnaC8QFySqeMzqXFpcP5Qbl9/o54jqJqZTDIrWwseiSi06/ZyxuyjeLgyyT/zp4+hkzpnwbjD6L0i6HIDgaosTSYLyckjqn9QkDaNLd2TQdo8FjVVDG60C8teBrQnr4srUn05PwcilIEkEBoohY/X398YPN/JuyrsUHRI7IVqt9jQn8S5Hlh3ht9HzaHmkJRp7OAGmzXDwY7DU7c8ic/4THeiU9xSdfuM/Gc/9h+5HY/CPvJbNbMNcZqbCLu65KH0w/NZe+PzING80oRDaEXQ1BVe3zb+NFS0HU6k5Tk6OkN38sOOHZzyNp+ZPkwmsqhKgegGmUAiJ2WBZC0wGUZQV3AZUOrdhtVLNwkMLWXdiNWFJYj1x6JC3Q1Hx4LEHGfXaKNfYiRNQot0JQGpYKqG6s1QJt5xWK4Epcx6hj4OQtqIDoJoH+j/Arz3NdD32mcu/7Gw0Zv405ZmwVdUtHdkxuiOvjHqF2zs/hK36I+HhJ31AckDpHtFBK9Nw/nkaNt5Z51tJIaJLvqSqBKOlZgNx6HNDuW3jbSiUZ5bE7N1bfA2mp0N+vscirhcKpYLU4aloW2txSA4UKFzPr+5B/g9GLhPPEmfAmfSrlXiW8R4KNaiDwF5FdDREGoegzu2LVhFEYp9EghOCz34OP2A2g1VVCgqR4AvXhUPhRpgVBvv/6/ZZf63fZRpA15dg3D+gUKHX6LmozWhStN0BUagV0zmGDhM6EBBcu4jF11iMFgzGdvQ7+Cf/Gz4dirbCxn+J3zLnP6Ed4KK1wteP6jkGMDlKxOu/5vJ2/NvYzOcuze0p2oxrw4YhuygM28lIy1swUwNWzyUifTl3ykk/D+NwOGqSfpocpIREUganoFQ1vf/VtiobFdU+wt3ChvLG6DeY0nmK+4cqc6Fgg7wBfh6y+OifHGj5MDlh8yirXk9KDgnJ0bS8SwLHD+eX68spDS1ltP0ILBkIlXXIl8icf0T0griRoFShNVSSG/o7v+ybAYDGoCEwJhBblQ2Hw8HOnTtxOHzfEW0xWqjIMaGyq9Daw1GmPQQTcyC859kPljk/cNhhyRDYLKRlnZ1+VnVRk/D0O/jHQV4LfY3O2/qQUDSZHlHJoAoAyXcGzzJ+Qh8Ho1dCm9tcQyuPreSEag0m3SFyckChUqAN1J52Y9kT86fdaueDDh+w/5Pl2JxJv4Aw8exnyoAG+CHJNFOq8qFkZ61hpz1AZAuR9Dt8GKyVVjb8bwOHFnk+A6hQKghNCSWsRZhrLCMDyvQiti4xXU5zpEyzQZKgMgcqTriGdGodcdFiI8W59snels2yZ5ZReLCw1ikaO39OHzudD9POXJTh7PILDgbNybVFVbnwRyfYLieeG0WXF4XsWx3enCEBIYQEiE6C42XHXeMxnWJI7JN41v2Z4GDh7Qe+l/hM7JvI1D+nouktbppIQyRqZR1yyUqNWGudBed9WMtXUsZ7dHgQLtkOQakEBQmPPBAJWGulFVOeyZ/R1cJhd/BqyKtseeE3rGpxwwRqAglQB4AmDFrdDOHdaz7vx/W7TAMIbgPhXd3UDZzKIk5fv6bCF/2/IPk3If0YEIB47jz0qfBhlml2dI3tyvSJ03lv3Luo1VAa1ZrOt/TBbvHfXkjbcW1ZOWI1Vq0Vo74HJF8J6kCPnNvhcLB7926PnKs+NL1MVDOgbWRbumomEVU+mtJScNgcfpMNOx2SJPFq8KsUffIzAJ0ievLYoMe4osMV7h9MGAvXlLmy8DJNnF3/hsNfA/D3sb85EPkuBcHLKS2Fde+u4yX1S2RvzfZzkO5U2TSkFr/JZ21L6dvv39DjrfNWL1mmDiQJJAea4FI2tb2MpzZNxSE56HFzD+7YfAcxnWP8Gl6HCR3o9eflWENHk1p5lfAH0ceKjgaZ5oFSJZIXNlGd1Sm6E9umFDJqRwaFhULec/u328nZXg8tRS8QkhRC95u7E2i5jp5HZjKsw9VwyQ7o9rJf4pHxLy3DWgJQqT1KTg5c/O7F3L7pdq96o1lNVlQaFVaz5N7pd2wmzGsh/FllZADWToVF/WttsLvsAYLEPFpaCpJdYuEDC9k1Y9epZ2k0pnwTpRmlbvYJGRkQWT6cCfo3uaHbDWc+weGvYEGvOhOYMucJ9kr4NR62PeY2bKi2wnUWthbsLWD1K6vJ/SfX4yF0mNiBrjfU9o7MNeay+PBi8kx5p5f2VGig07OQcLHH47qgiBsJ8Red1tvP2e13ctIPRNLldMU0J9Ojh/i9b1/jwmwouSZx39ayYJEk2PMGGNPrdR6508+/KBSgjN/Jkdh3mbF9Dt8M+4bPen3m77DcsJvtJA9MRhMThlUlJi6nZDJhnaD/VxA7wo8RyjQKh12sh+0WAGbtmcXu8NcwBRympAQy12XyWthrbPrIzyamQLvL2lEc1wEArRYhxz4pD+LG+DcwGc+x53U4+gMgCgendp3KqFYjiYiAwuTudHrsEr93nZoVJQAUx94Jg38+o4dwU0ZO+nmBrrFdeThxNu2zXiB/bz7/1v1baMI2IRxWB50md8KekAzULJBOy3l6g19w7HsH0r8HaiTsLOp8ysogsm0kaRPTUOu9t3HYEEyZReiMRUQEBWFIGANpj4D6bDekzHnBoS9EK3zeKqKDRNu+hERpVamfA3MnSduJLhkfM7DqZSjZDWUH/R2SjKe5dJdYrAIalYaWcREoUFJWBsXHyph30zz2zfPPjk5i30TGfjQBU2gCUI/vY5nmxb73xE81zqRfRUA62T6q0dGF6bhr510ETRiF3pJKe+0IOkR1gNAu0P5BCOngm0Bkmj4tb4DOz9bqRD7Z2gCElLcmUMMNy25gxEue3yTc8tkW/tviv+RsqynWyMiAkKrO3Nz+Ua7qeNWZT2AzQVWO6F6QOT9RG6DDw5BwqWvoRNkJ/m/bNLa2nIqpuommzbg23LP3Htpe6nn/lGHPDWPEi7Xv7yVHljB2+lgm/TTp9Ek/XZQo7kkc7/G4LjgkCezmOt9yJv0yS2t8/Q78foBXDK+w++ezV9o7bXecHpG+QJIk5kydwz/f/UOusTrpF3RK0i//b9j+BBz4oF7ndCb95E4/H1JxQjzfVUsSVkSsZU/yw8w69C0dr+pI52vP4jvrYzQGDdMWTiP2yiFY1Kck/WTOfw59ArMioEDsS7+z7h1WqJ6iTP8PpaWgj9CT0DuBwBjPdDM1huEvj2bV0B0s6BHEk3/fLQp4ddGg1vs7NBlPses/cPSbWsPO76jC2uIMPuW3u3/j6plXABBhOItdQBOnae3+NyOcrdImZQjtxrcjsl3TesJSaVVMmj6J9Y8De+GQZQ3hWTrSotMwaE7accz7W3gNJIwDhZwjbvKMWQtakVyJNlQn/TR5lJbCwPHtaDe+nT+jqxPTT7+TlpmN/okn/B2KjKcJShWbGZoQEuMCUJ0IxK4yUVRZRKAjkC2fbSGyXSStL27ttxBPbDrBoQWFKG3tMRgCYP3NYiPwigy/xSTjfUJCQKkEhwNU0RFMmTeF2K6xZz/QS1RUCC81jSOYgOK/oHy/8CQZOWQAAQAASURBVJuqNriWacYc/hwkG3R4AICW4dVJP+1RcnOhPLucXTN2kTwomaR+SV4NxWSC1Py7mDzyLqb1qh6MGezVa8qcZ7ScVuewswulQik2qE0mUCgUtBzZ0ithJPVLot8D/QhvVWOSduyY+J2SUo8TtL9P/Mic3/R82+2lXbIzP/0HlOEBmLKnAwr04Xr04b7dKNyWvQ2AHnE9Tp/0k/EM5YdhQXdo/wB0+3ett5OCa3f6hbcOp8vULoQkn/0Zq021pe2JE+JZzReFWeVZ5eyauQt9pJ7ctqfp9IseDMMX1LsoR5b39AOmDNj6oFAxiuhJQmgclEN+RS6Dnhjk7+hOS1UVLnlPV9Jv//tQvBV6fyQnXs5XwrpC69sgQOwRRhmiALCoCykpgaj2Udyw9CwqCT7CbAabsgy7ygRKB5QdEMVmIR3kZpTmwpjVrn1rgPn751NcWUxwxNXoy8rZ/PgqYp/rQZux/vGVLzpWRHixiK9t1iOg6Qtt/+WXWBqLnMXxMEql+F8aHCxhURVRbFYwZe4UulzXNL0lnFWQz++cTO/Pe7M775SKtz2vwd9XAvLkel4Q2kFIE1LjW2Wu7vRrqtg6dmVHj3L+d+wWjAv6wRpZSrbZEDcahs6FiJ7ExoLWJlaahZWFKBQKFj+8mF0zdqFUKunSpYtr/vQlu2bu4p/nf8VhzyFAb4d2d0Pa4z6PQ8bLlOyCff+FqjwAXlr5ItvbXUWZ/h+MFi3tL29PWGqYX0Lb8vkWFtz5K391SOT3XmoqDn0Om++RfdQuFIbOg5HLXS9d8p4BRykqAlNeBYsfWcyhhXX7onli/iw6VMTGDzZizKj2cPF/ka/MeYazC8VETdIPhJd0ZVGlx6/XanQrLv7vxegjxOaj1Qrp+Xlkhf+EOWi/x68nc34QrhMbNA6lmQpLJfbqhtSKggpKM2urTDRm/izYV8DMCTM5uKC2OsT23O0AdI/rfvpky4GPYPlFUCH7mDcKQyJE9oPA1Drf7pXQizGtxpASWlMNEJ0WzaTpk2g54uxFCaGhEFPtRHDI8/akdRKSGMJTxqcY9vwwlzpKraSfQimkYYNS63VOudPPD4R1Fr7NqVMBSIkQf4dFFv/YCZyNokNFLH1qKSW7T7jkPSMN1TdM3ko4Ol14jlfjz/W7TAOIGQL9Phf3JTVJP6u6oEl5+kmSxG+3/EqrfVUAhOqCYdujsKCbnyOT8SjhXSEw2fXyutnXcdO8m1CGZqO0WylZu4fC/f5r9xv43UA+vvtjDI4ggvO+Fs1QHkKpVNKpUyePne+s1/PZlS4wbtvWicU9Ijlg2uDvUOokd2cu82+fjz09EwmJYovYCK0lHdHpaej/jVxRcb5gLRMVj7jLe5aWgjHXyIL7F/hNwu50mNp2Z9nIX1iQ8zU2uwWobcQuc/4TEwMau6gWLKosQmPQcPPqmxn9+mgALBaLX+LqeVtPDt5ZwsK+bVgf8CK0ugna3+uXWGS8SO5fsPUhl3/T4iOLyQiajSngEMXF4iP18XbxBplrMjk0awcOVbX3b5fnYcRit+o3mWZMcGswJLhepoalAlARcBSrFbTxEdz89830uavPaU/R2PnzxMYTLLhvARVHRcLGlfRbdxNsurtR55ZpZhTvgCVDIH2G23BsYCwapQaq5zGjsFDll6t/4a3Yt5Ac3n22O3ECCgxr2Np6CvesOEvxWFU+7HwJirZ5NSYZH3DkG3E/VopN9CBtEGqlEDKyqIpcvn7vt32fuTfOrfMUDZ0/SzNLOfDHAYw5RrdxSZJqOv3iazr9wk/9Sq84DgVr5a6ZxqLSwail0Oa2Ot++u8/dLL5+MTd2v7HBl3B2+/kq6Qeg0WswRBp4ccSLWJ618O+RJ3Uxlh+Gws2nlTQ9lcpK8QNyx6lP0QRDzFDQC/nrVjHid6k9h71z9vLTpJ8oOVbixwDdKTxQyJrX1mA8kofB3Ib++usZ1mKYeHPwL3BVUS3lL3+t32UaT6ReJHQt6kJKq2ti/nrxLzZ/utmPUQlvyf0/7yAqV3g2hwQEC0+/Li/Je9LNCVsFVNb4WITpwgBQB5VgCksg5K1n6Xd/Pz8FBzaHneDKTmgrOlJ6cSX0+cij5/fl3Ckn/TyMwyEmJ2dVTH5lDqtfX83Chxb6M6xaFB4oZNsX27AXlWBVFWN1iEV6bemIQZA6xQ8RyjSITXfDb23AVlkj76nOp6REwlZlY+P7Gzm26pifg3SnogJsKvGkcaz3NzBoxpkPkDl/qCqALQ9B5q/ExYHGVpP0A0gZlEJIYggOh4P9+/e75k9fEp0WTXqHHGwaG6FykqX5kjQBRv0FkSJx4uqE1uRRUgIzLp/B2/Fvn/ZwbzLh6wn0XXEjVo0VpaTGENoB4sfIC5sLBasRjEdcnZ0tw1uiVChRocGBjbIKDSmDU07rseGJ+bPVmFbcsPwGKqNTWNmxC9dsjmbD8Q1QvF10ycrIOFGqoWw/mN2rb2/vdTvmZ828OfhLoCbp12pMK3rc2gOb2bNFFd+N/o4Vz69wvT5+HMr1OwDhrX5GirbCzuehYJ1HY5LxA1V5ULYXzPmAkJR1ytFZ1TVJv77396XjVR1rHd6Y+bP1mNb8n+X/6DrN/X7LLMukuKoYtVJNp+hOrg6rWsmW7q/ANUbZV9JPLHhgAX+9+Fe9Puv09TvoI8vv9JXp5PxT0w2mUWkI1J70DHDwY1jURySO64Ez8azXix8ZH+Kwi+c8oG2C2GezK8xkH8hm39x9tYoG/EnqiFTuPXAv6s4diCkby11x33Fv3+pCWIVCJDFPwp/rd5kGUHEC1k6DYz8BJ8t71nT6bf5oMzun7/RTgAJVgIrr9z/DypFifgvWBkOLa6DTk36NS8bDrL4G5tV024frxT6c0lACCiXFpSo/BSYwL7Jx6d9/MnjfBrR6HWg95+vncDg46KsHCmRPP6+REBIHeVBiy+XQwgryduRy8bsX+zssF2kT03is8HGuvV6NWSPK1sJ0YQSoa1r2kRzCGFvp339wMudAwnjQJ4Jkd3X6SUobeeUlhCSF8vCJh11SSE2F0PnfM7Rld5ZctIhQ3fltkipzCg4L7P8vSHZi0yairU765ZaJ1afD5sCUb8IQ4wODjDNQZhWtXql6CRb0gDb/grZ3+jUmGQ8TmOwmIXFyUURxMSQNSEIXqsNhc6BU+7YeSqFQUGgxggICpFAU5gIIiJKTfhcKO5+Hfe/AhHQIbEG0IZrKZyp55EEt6QhJrpQkBxWFFQTFBnklhMDoQFqOaIlpDlg0+ZitBejUOrhku3gOlJFxEtoRrsyrNezsrgqqvkWdSb/e/+rt8RBsZhsFewsISarx4yoogDJDddIv5ixJv5ihMHYjGLzrkSnjA9Ieg47ukuzhunDyTHlYVcUumdkRL47wyuUVSgWqU9bJzi6/jtEdCVAHuNQETiurKH/XN57SfbDrRUi9HhIvqfMjVbYq8b1WzYHfDqAP1zP8+eFnPb2vk36/3/E7APfuP43ySNLlQg3CkFz3+6cgS3v6kdlRENkbRi4hOU6P2haKTV1K7A1xPPf4cyiUTeffv0avIbJtJJbqKU3n/OciSZC9GILbCHUKmfMThwXSfwB9ArSY7GpSsagLXd9Tt6y9hYCQgDOcxPsoFArsCjVmnfAnCtJ6Z+0j42cSL4Og1iLnoFC6Ov3QiZuxfO9xMtdKJA+s3/ecJ5Ekid9u/pXYyLYo4/Voy4+AqoNHE3++RE76eZqqfCCcxFBRyVOpymH897cRFtW0/lcrlArQ6XEowaw5jUF00RZYOgx6/Rfa3OH7IGXOndQprs5MHTBz6A6+eD8KS3IoSpWS4ITgMx/vYyRJQl1SQHB5IHEqiMn8AaTLhMazzPmPLhYmZEBAJIFqaF/+L2JLLqd7sGjVn3fzPHZM38ETZU/4LcTpY6dz0aZu7HxgEbE6LVhKwe55/yGZJoDDDjYjaEPdOv2Ki+Gap4b4LazMdZnk7s4EIFUVAnNioN290Pt9v8Uk40NiR4JCBUqxu6JQKNCqtERGQnq6qNCffvF0sjZn8WSJd6pcbWYbKo0Kk0mBNaIEqJFZkTekZc4FpzRsVRXY7aDyQt2gOkDNwyceRjopIV1YCOV6UZ3eJfYsPupqvavrW+Y8p475ydnpZzmp088bnNh0AovRQouhLVCqaoqFtudsB6BHXA8kqabLqlan35HvIKQ9RPlPvqrZoNTAsZkQ0rFW0q/CWkGL/7agoKKA8qfKXRvI/9r+L7RB2nqd3invmZsLZWUQEnLmzzeWUa+Nwm4RhpRTZk1BrVTz+ujXSQxJFB+IGSp+6slpu01lvE/qtaIgG4iKggBbHDZ1KUeKcumUULv72J9UFlVirbRSaQrCoipFqTEAAWAthb8uhla3QP8v/R2mTEMJbAGTK4UkMu6dfkVF4pktorX/JwmL0ULWxjy0lWYIg1CNXhRlp1wtrKdkmgenFNg71512TQkAgQtmseCogTs2+ycPcfHX1/DOx3r6JCxBtXSqkDhOucovsTQWWd7T01SKNuSEEKHZbdbkUCnpUOuaVtKv+EgxGZtyweHAFiDkI2r5+QFEDzmtMbZM06dnUhd01njKSsU/9eKjxeTvzfdzVCejYNPYqcy5cg5pWjDsfBbyV/s7KBlPoVSJ7iq16OTrGjyKpKLrCTK3A6DVRa3oc28f7FY7Km/sDNaD2G6xZCeJrgWrritMOAIdHvRLLDJeZm4CrLoCqJH3dHb6+ZPZU2ZT/oGQUNQrQ0SRzTls5sic5yReCj3eAL37M5hzc66wENpc3IZed/bCbrXXeYrGzp9/3P0HL2texlhWjEMpPIIiFBYhAWTKaNS5ZZohOcvgyLduQ1a7lSt/vpJxvwzGphRtfhUVkLc7jzlT53B48WGPh6E4KeGTXWDCFCCUS84q71l2oN5eWDJNHLsFMn+FvL9dQ86kn01V6ur02/3Lbr4d+S1Fh4tqnaKh8+ff//mb78d873YfAkztOpUvL/+Sm7rfRHk52KqVbd08/exmWH8j7Hu3QdeWOYWgVjApH7r8X623DBoDFrvwzjleViOHGRASUO8uq8BASKi23vWFr1/axDQ6T+6MQ3Iwe+9sftj5A1IjPO/lTj8/0ucj6PwMAGo1DC38mkF7NxBf1ZX0v9LrnJP8xbp31/Fu0ruYswrZ1HY8V+/QMXffXJFU7/sZpE6tdYy/1u8yDUChdCX8AAYlD2LxtCX0zPzSVaBSWVxJwf4CPwYJebvy+Gval3Tb0Z146yCSDaFgLROF2TLNlnCdeEiyqUsAON5uJIOe8s9+iEKhYGXC3/w+5GL+ClgD3V+H8B4evYYv50456edhVFFCxiYuqCbpl3vYyPH1x7FWWP0ZmhurX1vNT8M/QWUz4zCcptMvsg+MXATxF/khQpkGUbAeVlwCOcsBCK3uQC4tFcoMP17yI79c/YsfA3Snqgqs1X5++xyBMPpv4b0l03wwpgv/HyC2eorJqbap6HZ9Ny55/xICIwLp0qWLXxYOY94Yw7wr/wAg0iB7+jVrUqZArJD4qpH3FJ1+RYeL+O3O3zi00Ae7Oacw/MXhlI0U7TEVxELfT0U1o8wFy/Qd0/mwcgiH4l6nqAgGPjqQMa+PQaWpPUeqVKpGz5/xPeNpf0UHihxCSker1BJUvg/WTBEJHhmZk9n7Jmy6y036VaPSsPDQQtYeXwNBYl1hNILVZGXnjzvJ3Znrscvn7sxl10+7qCyq6co/ULIbFBIR2lhXUUedOOywoDuskp81mwUKBfx9Jez/n2voxyt/5CmbmeTCm1ydfhX5FeRsy8GUa3I7vDHzZ78H+jH+0/G1EkdtItpwS49bGJ463NXlFxIiNvxr4lbCsD/kIjNPoVCALuq0byeFCCnfk5N+ZSfKOLriKNbK+u3P+Eri02Gv8UcrrizGVu3365rXbCaYlwq7/l3vczrvQznp53/SggcQbupLyX4b3474lh3f7/B3SC6S+ifR9/6+mDVBWFXipgnXhYM6ENrcDnEj3T7viedPGR9TuFmouQHRgdGMaT2a1EDRcVpQAIseWsSHHT7EXOa/wqiQ5BDa3juGQNWDTLOsZkCby+Dyw9Djdb/FJOMFTvwJqyYKeW5qOv0qpGKUSihM7Erc8A5+Cy+zLBOjfi/77QgZeQ9KG6tUKjp16uSx850NOennYZxSMycn/fb+sIUvB3xJ4YHCMx3qUzpM7ECne4dj1wSQIg3hjdFvcG3na/0dlkxjsZZD3gqoPAHA0qxZ7E5+iBz9CqqqoO99felzT9ORNCrOriLq+B6i86JRBYRDzGAwJPo7LBlPsmIs/C1a4XVRueSG/s5fx9w3kSVJoqyszE2qy1dIkkSlJFq92umPwZFvwFLi8zhkfEDv96DLc8DJnX5FFBeDrcrG1s+2krku0+dhdb+pO5peHUgomkwHwzCfX1/Gz5Tsgr/Gw/F5rqF8Uz4HqlZTatjiqtA/HZ6YP/ve05fx312DWSOqe6MMUSgie8Ogn1yJchkZF11egOF/wimdJ87iQUWwqOwxGkVC+WnT0wx8ZKDHLr93zl5mT5lNaWZN1fcx014A2oadZRHtMEPHJyFposfikfEjSg0MmgmdnnINhQSEEBIoZBudnX697+rNE8VP1PKGacz82XJES3re2vOMnzmttKdSI2Qoo/qf83VlToO1DNJnQNHWWm8lh4i/98zSmme8zZ9s5ruR31FytKRep/dV0m/RQ4t4N+VdTPkmcoxiLg3XhaNVVUuRWopBGyFkweuJLO/pRw5/DWumgkMkl50F2RZDKOPeH0e78e38GJw77S5tx7j3xlGl0GNRi8nL2TldF/5cv8s0kFWXwyZ3r9BoUQdLfj60v7w9Q54dguTw399pSGII8ZMGUhGWQIB/7QVlvElFJpyYD5VZAEztMpXpE6dzbecphIWJj/hLjSnnnxzCrlPQf11/9ArP+/g5505f0bQ0J5sB0pEfoMfdtA5vTSfFlVhK0whIa82YNzUExgb6OzwXbce1pSymLbwAKZqePDaojkXLpnsgvLuo7JE5P4gbLbS6q1mesZCjsV+itUVRWjqCPnc3nYQfQN7+Yjpv2E5UxY+88J/W4LCBUp6WmhXt7nN5rhTq17Op7RXkFPXjbUZRdLiIZU8tI+2qNGztbH6pFlzy5BLG776Pza2z6MAyWP9fuHwoaMN8GoeMbxmcMpjdNxTx+P1hFOsgqn0Uj+Y9iiHK4Jd4UqWR9Dwykmf7vQdrr4e+n4jKWpnmj70KcpZC/FjXUMvwlgBUBBx1bRovengRhQcKue7369wOdzgcHDlypNHzZ0WF8PUAUf2LPh5aXNPg88k0Y06TqIgNiuVoyVEIFF19JhMo1UqUas/WmHa5rgtRHaKIaCM2IyUJDDmj6VU8m0fGn8VsS21wFX/INBPqmKcM1V/lzk6/UyU4nXhq/nSyN38vy48uZ0DyAHrG9zx9h5XkEN1+Mp6jMhvWXifWHRHu+xp1dfq1u7QdQbFBGKLr99znq6RfUHwQocmhGKIM5KZXqzGdbMFiSIJxtRObZ0Lu9PMjhRvh2I/Q67+gi6Y0cCuHY1ewqrg9r9073t/R1UmVWcIaeFLS79BnsO8dUQgW3s31OU/PnzI+oOvLoKl5Tvpy65fsCMnHobybgoIQhk5KI21Smh8DFFiEIrNI+pXuFWpm8WPBkODXuGQ8SOvbhKVJ9fNZv6R+9EsSHscREaDesoHZg9dy+5obfe41qdKqMLaqojy4nOeSV8LigTBmjcd87h0OB+np6R45V32QnzY9jGLbg+CwkRadxl2Rs2if9TK2uCQGPjqQ4Phgf4fnhrP6MbCufUWbCQ59ArkrfBqTTCM5ZSKqkbDLp7QJymBrosM41Osa7K27EXvwbZipgcocf4cl40na3wvt7gGgRYz4wi63iYWE5JDY88se8nf7z2dy43830mlXH3oemYkp9h4YOt9luC7TzMhbBasnQ/F2AtQBtIwLR4GCqiqw2JQERgeedmPQW5RmlPJBhw8oWbwBgFjVOsj4CVR6n8Yh40cieolinfb3uYZahLYAoFKb4arQLz9RTsnREuyWun39GsPChxay8b9rUTkMxFaMpG9iXzfpRhmZWkgO4ad2Ek6VE7uuRt4TIGd7DhlrPOcNGdk2ks6TO6Ot7uYyGkFdmUh8ySSu6Dr6zAdXS+XJNDMkh2vOWpm+kh/N0zgU96prreuwOdj/236O/X3MI5erKKjgzZg3WfnSSrfxhYcWcu+Ce3l51ctATbIl/FT1+P3vw8/BULDRI/HIAMHtYMB0SHus1lvOTr+Tk35J/ZPoe29fAqPrV2DVqpVYZhcV1fy9eoMhTw3hljW3oFAoyDWexoLlHJE9/fxIjzfFM55O7MlkqVeyN/lR/iqY4efAarP4scXMvXEuRosRSSm+K0Wnn1J0lsrFiOc/rW91s5B4bMljLLA+RZX2BAX+tfJzsWvmLv65+zO2x7bjDXsCxoy5sOEWKD/g79BkPIlSddokWng4OFQaVKFBXln3no3otGg237uP3Z13E6RWCDU9H+8ReRI56edhHINnu/7b2ZbaFJMt08dOZ/frvwOQr1/N5qzNVFgraj6gDoSry6CnbDB+XiE5IGsh5K8Dqqv1AbM6j7IyIWXyRb8vMOYY/RmlC5tGT0l8Gur4aFGVmXI1BMgrkuZKqziR9KuQqqsHW0fwrPlZhj7nH5NegPuP3E9eP1FpqQprA0mXgUrWkmiWVOVBxs8uj0mdDpdsSHExlKSXkLHacxvT9cFWZUOpVlJpsSJhZ0/4TLiqWO4AuJBQKGotJJJDxQalRZNHYakZux0m/TCJu3ffjUrr+Wrq7V9tJ/Ovw0Qah3B58TI+u+wz4ZM1N0VO/snUJm8VzAyAI1+5DTs3pi0BonjLmXD59fpfmXfzPDzFyZ5XULOhHRwMGs0ZDizdB3NihIy3TPNh50vwkw4qxPd3ZlkmGyp+oCBkmavTDwX8NPEn1r+z3iOXtFZYiWwXiS5c5za+LWcbAD3iegBnkPfUx4uOWX3jkjkyJ6FQQMupEJhc6y1np19mWcMl3HU6SEkR/+3tbj8nuaY6Ov1O/C6SxraK0xzljiTJnX5+RRMEqpp5IjpIeE8WWwr4btR3/DTxJ39FVovszdlkrM6gzCpumACVDr1GD21ug0t3Q3AbP0co42kiDWJSsKoLyc8HU56J6WOns/49z3xXNgRrpRVruQm7ykg52SiTJ8Gw3yCsq99ikvECViPkLIfyQwAUVBQwb988/jjwB2FhUJDSk+SXbic6Ldov4ZVbRBLn0+z74NKdfonBU8i7Sp4merBLnjAkRMKiKiRv/3E+7/M5277e5ufgajDlmzAXCxnI2dJU+nzeh525p9zM6kB5MXLeoYCVl8GeV4GTfKs0+ZSVQVVJFeVZ5X415z2ZykrIDf2dVRG3MN2khcE/C58LmebDgQ9h8SAwF9I6Uex6WFTFlJU7UCgVrg1snU53prN4DW2slkK1BQk7Br3cAdCsSZoAk83QYjIA/7fiWba0vhKjbh8lJbDgvgV8N+o7n/oYRLaL5O5ddzOz11v80VvN8pIv5UraC5HsJZD3t+tlpD4SnVrMiZXqE5SUcEaJxMbOn49kP0LXl4T3qkv9IaSdkHg/jysbZbyEIQkSxoHeXWbJmfQzq907/QY/PZjhLw73yKUlSeKNyDeYNXmWa6ywEDIjv6U4fhYlVSWnP9hSCIGpoIvzSCwyTYSg1hA31tXFGWUQm+oWdYEr8axUKZk0fRKDnhxU6/CGzJ+hKaHcsvoW+t3Xz218e852ALrHdQfOkGxpcQ2MXAKBLc752jJnwVIKlbluQx2iOjC61Wj6JNTYXJRmlvJxl49Z/frqep+6TXXO49Ahj0Rai8x1mfx535/k7xUKKKVVYtPRrdPv6Pew5f56e/qVlYHdLr7KnQXpMj7EUiKkCavyAIgPEfNTua0AtU6NWtd0bE1uXHEj9x26z6XIExZwaotybfy1fpdpINufgt/ag0N0T538fVlQANogrUj8ZvrOb+xUetzcg9A3p5CdkI0CJfrQdpA4HgJkU9JmRUUGLB8FR78DYFfeLq746QoeXfKoq1DKm131ZyJzbSat58UTVhxGoMrznn7g27mz6XzLNBNUSkW1wXI4Dx/owaEe/xCbPZ/QPBPWCqu/w3Nx59Y7+f574GeoQJTIOid9AIq3A0oI6yx3HJxPKBTQ/yswiFLEU+U9Jz05mMFPDvZnhG4cnbON0b9s5sfrlrMmI4BpXaf5OyQZT1OVB6ajYC0nISxejCkcHD5eSo+0cLK3ZVORX0GHizr4PDS7xc7c5fNYkHIjelsP4jZlw94UGLPK57HI+IBTCgoWHFpAumEr0QG3UFzcgR639aDNuDY4bA6vdFOdCZOjGI0KemoOQNlBCGnr0+vL+Jk1U0QF62ghqa5QKEgKSeJQ0SGqtMcpLGxFZCTsn7+fosNFDHhogOtQlUpFhw6Nmz81Bg0WlQYJCYOhOsnX/bVGnVOmGRPUCobNrzUcGxSLRqlBqRHrHWfCpcu1XTx2aYfVQavRrYjpGuMaKyqC3cn384+6jBzjXsJ0YXUfHD3onL2wZM4DWk4VP9U41z5mdX5Npx/QeUrnWod6Yv50UmWrYk/+HqAenX4y3sF4BH5rC+3uh141akWDUgax5Polbh/VBmmxVZ1bsV+7drBsmfc6/Y6tPMamDza55sznhz/P00OexnKylHLPt6HdvfVWJXF2QoeGglre+fM9eSth1RUw4HtoOY34MFEBYJQKuO6P6858rF9QYK8MIrHgei4aESSGDn4ifOBS3eP15Pwp40NUAWCvBGUQkXpxP1rUBeTni/XAU8anfG53cSplVeUA6BRBKJAAuQCx2WFIgt4fQmRfANeze0lVCeGpoLJUUvjbNo60i6PVqFY+DS1zXSadl7dnW9Jermi1FPIMEOM5ZTKVSkW7du08dr6zIX/1exhpXitIGQDD/yBSH8UhI5zQl/LdsQf9HVotKivBrjBjlsSq3NneDcD2pyF3OVxjkufY842W17v+82R5z6YoM2sP0FMUbaNKV8VNlnWw9x1Ie9jfYcl4kq4vih8gANBIgVgVJg6eKKJHWjiLHlpE3s48bj1wK+Hh4SiVvisyKDpcxP5x+xg4dCC7egVATBo00jdDpgnjsEPhBlAHQXhXl/+UWZNDSQkMnOD7havT60pVXkVMDFxT+QYcNEOv//o8Fhk/0vtD0EW5DaWGpZJTUIVdWeHaON700SbS/0qnz119XNXhDoeD4uLiBs+fFpOF/N35lGaHs7XVzazQLidtx7tyEY7MOXNHrzu4q/ddzJihYAY1nX6eRKVVcc2sa9zGMvNKsalFVbrTu0vmwsW59rGo8zGa3DcLpWq5YueGZkPnz8NLDpO1OYted/TCEGkQY0WHsUt2QgNCXXKSp/X02/40GBJdntcyHiKwJaROg6gBZ/2oPlzPfQfvO+vnTqZ1a/HbW51+Ax8dSNfru7r5DGpUGjSqk4rWDEnip57Ifn5+Jqyr8PULF4UAyZHiWa+SJmKgdhL75+/HkBBKYFVbeqR/x/tjq9/Y9TIYkmsl/Rr7/CnjB7q/Kn6qqen0K6S0AKxW0Gj8u/l7dMVRKlbtRdlaiU4bDMvHCGuOicfPfrDM+YMmBNrd7XrpTPoVVxYTHg5Khw374iXsa93H50m/Xrf3IlPfnn6L/8U1UXFwtMyjST+Hw0GRD9sY5dnZw0hJE0QlKRAfLDYUiyy5ZzrE51QUVrDl8y2UHczFqhZPgiqFitCAk1pX290tHlCUvu12kPEsLnlPdQGlpRKmfBM7fthB3u48P0cm0HXrwJ8Ts8iNy6V31S5RDSfTrDEoRblzeq74ohv46EDGvjeWjIwM12aMr9CF6VBM0XKsxTG09nCUA79yexCWaW5IsGSwWLwCcYE1Sb/iYv9EdHjJYRbcuwBtmZ0yB2yNeRZSJvsnGBn/kToF4ka7DS2etpinAjKJKbvYtWk35o0x3L3rblQBNc9mkiSRmZnZ4Pkzb2ceX/T7gqw/t2PW5FJBAQaHGbY8CFmLGvonkmnu/PMs7HbvBlUr1SgUCoKqmwOcnX67Zu7ivVbvkbmu4X5aZ+JwgThvoDKcQO1p5JHTf4R1N4LJOzHI+BFbJWx73OXV6NzElJRWiitqJMpWPL+CVwyvYMyuyUY3dP488NsBlj+9HFtlTafYsdJjgCjYUCgUSBKuZ4taCZcDH8Dxued0TZl6oFDAgG+FfGodVFor3bvmzpEWLcQlSkvxSjGtUq0kJDHk9GoTDhtUZoPkqPv9OnA+P8jdpn4iqCWkPQphnQBIiRLzk01Zwb6l+1j9+upz7jj1BnarnZkTZrLmtTWuMZf63IjF0PfTWsc09vlTxv84O/3sASIJXVAgClK3frEVh63+84wn2fr5VjQ/bkchKdCrgiCyX601kkzzI1wnqqPMdjOGkCqs2kByx9/CkKeH+DyWgJAAlNGRVEjhzCpZCe0f8Oj5JUni+HHfJbHlpJ+HkXq8A52eBiApTGwoljly+Gf6TvbP3+/P0FwUHSri9zt+p2LHISzVSb8IfYR7G3fieGh/btVvMk2ETXfD7BiQJOKC4vi46w5G7cigtBSKDxfz67RfOfinjxzIz0JlJVhVYtX0eZt3YOD3fo5IxuNUZkP6DJdJ7+TIV+l+5HuUZakAtBvfjs7XdvaLjERwfDBFUyo40voIesJl66rmjlINvd6D1rcB1HT6qUXSL39PPh93/ZhNH2/yWUhdp3blhmU3kB2TTrkDLCk3QPTZK9Rlmj8KhcK1UewsBoztGktEmwiPzpfBicGMem0UypYtsKrFoj9R5YD970Hheo9dR6aZcWwGHJtZ51tOX0hnp59aryYgJMAjG0iHlxxmwQMLKEkvcY1llIhEXozuDF1+BetE4k8T3OgYZJoYSi3sexcyfwXAoDGgV4nuu2JzvutjYS3CSB2Rit1qb/Qlhz03jNs33U5QXJBrLKM0A4AWYcKnr6wMbNV7+bW81CakQ/9vGx2HTP0Z/NVgDK8YWHWsRsJ/37x9bPxwY73PodNBXLUlaHq6hwMEsrdmU5pZk028dva1TJ0zlczS6mIF42H4NUEUXdST0/pKyviFxKhgFA7Rubn9l50se3IZFQUVZznKN0z8fiJp1/fCpjSh0JhROXPPYZ0gvJtfY5PxEKV7Yf//wCSKVJxFMgqD2BMuKIDt32znt9t/ozyr3C8hDnl6CLk3tMGutmNQB0P3V2DAN36JRcbLLBkMf18NQHBAMIpqZQaloQSUSnK1yQTF+f653ZhrxJRTjs2uoVA11FW0cb4iJ/28SHJ4taG9JofFjy5m9Wv1N4r2JlHto5i6YCrm1mmupJ+btKfM+U1ADIR2BHslaqWa7vFd0NniKC9XENUhimvmXEPHKzv6O0oAClftpuf6MNRWNcG6SNHmLdO8KN4Ba6+DXOFVNbH1VJKKplGRH+3nwAQFJlGG3cEgwcZ/Qd7ffo5Ixqu0vw8ShF5NfLDwmKzSZFNcDNpgLdYKK5LddxWrwQnBxA9OpqJ6sRVba2dQ5oJgy8MwOwrsVW7Dzsp8Z6U+gMVoIXtbtscuHZocyuAnBmOLTcSsFhvkusieMCED2t59lqNlLlguWgcXrXEbstqtTPppEg/tGoRNaXR1+nWY0IF/bf8XLYa0aPRlj608xsb/bcRiqunWyaru3ksIPEPSr/f7MPEEaMMaHYNME0OpgssOwuCfXEORevGMWWKpmTx73NKDqX9OJaxFWKMvaYgykNA7AaW6Zivl6o5Xs+qmVTw7RCRknMmWOr3UAiLAkNDoOGTqoGA9LB0BWQvchp1dwMfLaqrrt3yyhSWPLTmnTqXUVPHbG0m/6WOnM2fqHEB0AszZO4cfd/6IRHV8ygDxvRxT/84HWd7TzzhssLAvbH0EAL1ewcCjfzJ4z0a63DSE2zbehiHa4OcgQaVR0XVaVyJ7pXIg4QV+66bjyaVPgt0MlmKQu/maB4UbYMsDYn8GmNx5MkuvX8oojfjeKiiA7jd357o/rkMfqfdLiDGdY6hqnUi4cSCtA+Vkc7NGFQgqcZ8pFcoaT25dCQDWKjv56Safh/Xr3b9y7P532Nx2OAHaxheK+RvZ08/TmI7Bnrsh8XKXvKdZk8PgdybRon39DJe9jS5MR5uL22BaCFZVTaefiyPfws4XYOCPcsfB+UjXF9xehlTn0crKxN992sQ038d0Giq27qPv1hQ2jJVItZ4AcyEEyKuSZkVEDxg8CyJ6ARBbbZmXkyN+H1p4iPm3z6fnMz2hi29DO7L0CNGvGEjqn0SHFmY49JWQkDiHxbTM+cvJnn7FxSL5cf+h+30ag7XSSk6ZkB+bFgwt1vWHgdPl794LDV00hHYCeyWohJ7StuxtvHT8fvLbhNO9cL7ro9Mvnk7+7nweL3wchVJURAYHN74K0mhyuCTfo4LiIDCx0eeUacboYmoNaVQaFh1eRIW1gihNLkZjUB0HNo5hzw+j1x29CIqvOXdeVSYEQovws/j51RGzTDMhKNXt5aqpW7j3jhCUkgaHA85kN9WQ+bM0oxRDlAGNocZrLdIQyZAWNc+PzqRfLVlFSzFUnBCyf+rTyNHKNBzJAcXbocrdXsXp9+nqmgNGvz76nDs/W7SAdevg2LFGR+qGJEkMeWYIhiiRACqpKnFJkTrtOghKhT4fntN55aSfn1GqwVIEDisg5GHbqkZTUAHaKEhs6+f4TqGqCqwqMXmFBoRC/mpYPhr6fARt76r1eU88f8r4kPixMOovCBObLq3CW9EqvBU7oiATkfSLGxFHXLc4v4UoSRIJluEM2reGRy6xw8a7xN7MKZ6SMs2Ake42EmG6MIqrijHZizEYIHnp93zdO58nCh/zaVgRIyL4IesHJiRu5cYADeQs9rjEbFCQ59dIp0NO+nkYlSZIyN0YUogLETeGWZ2DvmNLErr7NzYnkkNCoVRQUQHBlZ15pOsbdGl50kJYoRLyNwFR/gtSxmMszv2W3cnbqDJNA3oD4svUH3KKtbj4Yr7o+QoDgu0M2v8EhMdCqxv9HZWMJ9HFQMqVrpcVukPkhu6lsiwFSepGQGgAwXHBJCQloFL51kPUmGMk6LAObU8tuyoGwKTXXRvuMs2UrY/A8Xlw2UFX0s+qLqTYT9a7P4z7gZxdeSRMnUyQ6hBKrUPeBLwQ6fSU+DkJpULJrrLVaAOj3Tr9et3Zi4r8CuwWO2qdGpVKRevWrRt86TVvrmHfnH0UDRqLFCw2P6MUFjCmgyFZ9naWqZvKHKg4DuE93O6R2MBYjpYcxazOxWgU96XD7mDTh5sITgim41WNU5pQaVSEptR4kNtsUCKJTfzWUadJ+uWugIosSJ4Eav9Urst4GXORkD4M7w5KDUmRkSirG1MqKiAoCKpKqljzxhrie8W7FE8aMn9KksQHHT4gdVgqUxdMPe3nTpv0y14Ma6bAoJ9O6z0n0wiiBsDVtY2ak0KSAPdOv9iused8em91+ikUCvo/2N/1OtckHkxDA0LRqRu+NjntfSjjOy4/5PYyJEQkV0pLJawVNhRKBWqdf7dlM9dmMmvyLNrfPQqLWtw0EfoIsR/Y+lYIq91x1djnTxk/oI8XP6dQbTVJgVD5R5IkHDYHKo3v1wAfdfoIZZEG+tyBQVMGhz4BySYn/S4AXh31KnbJTuuI1oSHQ0lce7r3iPP53nXo5aEsKlzEBHM8JxQXkaRP8uj5VSoVrVq18ug5z4Qs7+lhHAFRcHUZ9HidVuGt6OC4itjSyygtFYvepsDq11fzH/1/sB3PIcjcnnt7PMaN3U9KtLScBpfsgJAmVnokUz9KdsPOF6F0DwCLj8/maOx75Cq3YrXCe63eY/rY6X4OUlCpDKRz3kruSluFtdurENnX3yHJeJn5x79iU9vLORLyNWVlkDwgmVs33EpI7xAcDt/OkV2ndSX/0xCqwvsQp00DXRRofFd1I+MHFGrQhIK9kr6JfTl4axHDdu+hpEQo1xxccJA1b64562k8ReqIVFpc3ImeR2ZiOrAZxm2F8K4+u75M08W5QWnR5JNfXCP72e36bgx4eIBrg8jhcJCTk9Pg+dNSbqE0o5QiWwkABlUwAfvfgfkta3VKyMi42PkCLOojupZOIjbIaW2Qi8kk5lWFUsHSJ5ey7cttjb7siY0nMOYaXa+LiqBN9lP0PTqHKd0m1n3QwU9h3TRAlidrtux5DRb1dfkUaTTiB0TSD8R9uPrV1Rz8vcbXvCHzp2SX6HNPHzpM7OA2/urfr/LJ5k8oqSoBzpBsCUmDzs+JBKWM5znNxqCr068s023cYrJgM9vqfXpn0u/YMfDmsiXXKL5/nXMqADtfgjVTRTdjPZE7/ZoexSErORz7NivnzeeVwFfY+eNOf4eEQqkgODEYSat1qT5E6COEl1+/LyB6YK1jGvv8KeMnJAdUdxEbLUa+2PoFK61vApCfD5VFlbwW+hoL7ltwprN4jRbDWmCNTwFApQuFSfnQ/TW/xCLjZbIWwe7XXPLBkztP5rou1xETGEN4OOS2GkDKHRf7vFnF+Rz3V2ki24JmQ2iHMx9wjjgcDnJzfbfGlpN+HkaSJJdJfOuI1twc+AsdTrzCzv/M52X1y9iq6v9Q6S3CW4bTclRLjA4hHxEoNxU0L8r3i82Y4u0AxIUIXwuLJp+yMkjqn0Rst3OvbPQGVXll6KqUREYNQdPpSQhtOtKjMh7CYYPZ0bD2BgCiA8Xuh0Vd6JL4lCSJnJycc/LU8BTjIx6l55GZ9A+JFebW57CQljkP6fE6jNsCagNalZYWseEoUGCzgdEIO77fwdLHl2KttPoknOHPD6frU5cCYPC/pYeMvyjdJzbzSna5hiL0Ea7q/kJLFmZz3Yc2dv4c8dIIHj7xMCa7lsiykfSIGgxxY6D9Q7IcoszpSZoA3V4BldZt+GTZZJsNLBbRwXLj8hu55KNLGnVJh83BV4O+4vc7f3eNFRZCkLk9nVUT6RhzmmfIri/C0HmglifZZkvCJdD13y5v8Hn75rGt5VQyor5weUtqg7Xcs+8exr0/znVYQ+ZPpVrJRW9eRK87ernGbA4bz654lrv+uIsKq8gyOpN+tZIt4V3FPRnS7tz/nDL1I2c5pM9wG6qr02/njzt5NehVDi1w78Q6E/HxoNWKuc25jvEEe3/dy2e9PyNznUhKOjv9YgNPWrMXboTcZaCo3xae1SrsPUBO+vmVgvWw9x2Xb/Mh3c/sTX6UbcotdL2+K2GpYf6ND7E/dNv62wjr3wHryZ1+Z8Cf63eZBlJ+GGaoYOdzAFjsFm7/7Xa+zHwcu8JMQQHownUkD0wmqoN/VN/GfzyexaPWsKRrIj8ce0sUZcv2P82TjJnwz1NgM9Z6Kzxc/C6u3bjvdfY/up+LFl2Exh5GgBcc2iRJkpN+5z0VWZA+EypzCQurHouPo+PVHbFb/G8E2XlKZ66acx2VmhDKdLvYW7qJ4srqf03WctjykJDCkTk/iR0pOjUTLwMgJlAk/czqPPLz4cofr+SiNy/yZ4QudD99S7t136KX1ZaaL0o1hPeE4DZAzQLCqirC+V23+ePNHJvjYXOMepDzTw65f+9HYbcyIuIZ+KMjSP6fo2V8h0YjZL8ASkpg6P8N5Y6td/hUzqTMaEXCzsDEGbD/fZ9dV6YJUX4Qdj4PRVtcQwqFwtWZUKXNdG0gS5LEtyO+Zd4t8zwagra8HQMOLOOb0X9C0uXQ6x0xf8vI1EXCOCFJW51kceLcoLZoxRe8M+GS1D+J8Jbhjbqkw+ZgzJtj6HZDjcxYvbpYQtqLe1qm+RI7HDo/4ypU2Fuwl/SQHykKWu26BxUKBVHto9AGaU9/ngZyouwEDsmBRqlxJb6dc3Z44257mYaw6yXYdLfbUHJo7U6/yPaRdJ3WlaC4+qt8KJWQIppQPCrxaTFaMOWZUKrF9lxWeRYA8cEnSfEN/x0mZNT7nM7N0pOfdWX8QOYc2PaI8PIEInUimZIdWsDE7ybScmRLf0bnhtkMFtVJSb+9b4nCXYf/GxdkPEBABCRfBaGdAeGhpq5+1reo8ykoEN+V0xZOc5Mb9jVG8jBrs9BSLFTMrLWTQjLNgI5PwdhNoBKbwQcKDzBv3zx25O4gIgKCCo9x4N8/k7Uly6dhVWyrID47ngejc+hQ+bhPr+0N5KSfN8hZAmuvhYI1BAU7sKgLMPfqwtU/X01AiBdSxQ2gslL83p/0NMN+6Msve34RA6V7YP9/IX+t32KTaSTaMGHOW91x6jT/tqjz8WFBQb0obtOenV32k7m7HyzoBTaTv0OS8QYjF0EXUVHmTPpZ1EXk5Ym3N/x3A0dmHvF5WFs+3cLxN2aitFVwjCnQ9WVQanweh4wPKT8M+94VnVXAi3+9yIbUSZTqt1NcDNFp0cT3iHdtungTa4WVX6//lfnff8QfvdWkxT8gNqpkLjxihsK4fyDZXZ7Q2ZlQqT3uSm4oFAqslVaPFZFt/3Y7hxYecm2My+oPMo0hyiA2MyWduGGd95XkkDDmGpEcDe8IUOvU9H+wP2mTajr6jueXcST2HbLCZtd9kMMKtooGX1Pm/CTaUK1yos53yXuCkC3L+adx7VmHFh1ixuUzyN6a7RrLKBWJmOTQZJTVXVhOb6RaCekNt8Oq00jRyniGri/BkFkuyTAQ8p5jWo1hUodJ2B3i+zOhVwITv59IUv9z8+tp0UL8PubBesVu13fjoYyHSOyTCAh5MQUKEoMT3T+oqn/S2vncEBFxWtVTGV/Q+nYYsxr0CQBEBYrvyaKqAn9G5Ub6ynRWv76ashyje6df3io4/qtcBNZc0IbDkF+EnRPCP9y5V2jW5FJezmmVRXyBzWzjj3v+IEEs0+muzoY/O8OJ3/wXlIz3CGkHkb1d88tnWz7jip+u4Pt/vicsDNQWExVb9lJytMSnYSl+UPDtjd9yWUQOsVU/+fTa3kBO+nkYhUIhOq0GzYTIvrx4oj+Lu0ezvXS5v0Nzse7ddax6UXTy2bTiaTBSX70iiegFlx+GNrf7KzyZxiJJwtC++kHStfDViKTf7l92s+D+BThs/pcxPNoxlQUXvceRku1gLQGVLL3U3HF1+qlrOv2u/OlKLv7e93rdaTekMXfCXH7vF8kx9Wjo/KxPry/jB8r2wtaHoXA9AMuOLuOo7ldMugOuimiL0YK5zPsrnorCCnZM34H1gKhefClvMAz3j3+CjJ/RhgrJt1O6ppydCSd3+gHctv42Jk2fBIjnzoiIiAbPnwsfWMiaN9ZSVW0baNBLsGQI7Hy5QeeTuUDIXwOLB8KJP92GI/WRaFVa1BrRFWCsLs5e8MAC3o57282PzxPsyzvMnuRH+M1+T90fKNgAPwfCgY88el2ZJkbhJlg8CDLnAhDtVDnR5LsSzwAL7l/Ap90/xWISfkYNmT9LjpZwaOEhN8uOY6Ui+9MiVGSDJAmyq3OC8fGnnKAyx9XxI+MlYoZC3Ci3TFdwQDCLr1/MlxO+RKVsnJqD09fPk51+p/LC8BcwP2vmpRHVxWDmItExZso884EncXLST8aPhLSF6EGgFt0ssUEi6VdqKWD+7fNZ+7b/i+0PLzrMsieXYSqqIKFoMh1V44k0RMKw+TCp7qrxxj5/yjQNnEk/ySCqsQsKYP9v+5l3yzyqSqvOdKjHsZRb2PzRZqIzq7sPDanQ+f+Et6RM80OSRBdntcdkuE5II5RUlRARAaWx7bE99X90vKqjT8MKUAUQJCVx7dbb2Je62ePnVygUhPtQBkJO+nkYpVIJgcnQYjIYkojUi0VHxZEM/rjnD7K3ZZ/lDN5n5w872ffTPwDYqit5Ig3VST+lGoJayT4u5zMOC8yOhC33AyctfNV55OTA0WVH2fj+RqpKfPslfiqSBOWWUgDeqIwTyWb5obF5cvhr+Of/gJoCA6uqyOWFkdgrkbR+aWL+9CEBHQPY3mM7DhVEBMq6NxcEUQNFtW3CeMDdf6q4GHJ35vJq8Kuse3ed10MJTQ7ladPTHLtY7EiWkiyq3WQuPCQJLMWuYh0nqaGphCmSUUhq1+bdqSiVSlJSUho8f147/1oGPDuSvYlPsrhbNJ9vfwmMR6BS3pSWOQOSHUzpYC11G76/3/1UPVPFxfaPgZqkX+qwVHr9q1ejNgdXvrSSL/p9gSm/JouTXiw6rGICUuo+SG2AFlMgrHODrytzHiDZxbxlLQFqCh6t6gK3Tr+0K9MY/tJwV8dpQ+bP3v/qzbNVz7p1hzk7/VJCxX1YXi66XBUKiIs75QTDf4OLN57jH1DmnJEk0el7xo9I/HbHb6x+ffU5ndqZ9PNkp9/2b7ezf/5+tzGNSkOQtnp9UrwV/r7ynDpe6iV/LOMbHDaXolFcqEj6GR2F7PllD4cXHfZnZAD0u78ft228DSkogu7p33Jv5G81995p/HAb+/wp4yf+eQZ2/dv10inLrg0Xyd38fMjZlsP2r7f7vMNKH6HnkZxHWTvoAADWkDTRuR3q26SPjI848g38EiyUEhFyswDFVcWEh4OkVFFc6tv5xVZl47KCy7hx/zZanXgVVfVeuidRKpUkJyd7/LynvZ7PrnSB4HCc1D0lScQHiyd9S2EBmz/aTP7ufD9FVsMNS29g6De3AGBWn9LpV35YVCDKnL+oAqDt3RA3Gqgt7zn8xeE8fOJh9BH+NdIzFlnounItvTf1JkwX6tdYZLxM5mzY+ybgLu+ZmyfmS7vNzp71e6go9q0EV3GVaO0KtIcx2toDdr/q0+vL+IGACFFtW+2ncWrSLzQllK7TuhLbJdYn4WgMGooVJQAk6jSyZ8aFisMKsyJgy31uwy+OeJG3kzJonfuoW9KvJL2EtW+vJW9XHg6Hg4yMDPfnz3OgxdAWhHZKwqzJxaIpwKrSwsQT0PeTxvyJZJo7MUNhYhakXus2rFKqUCgULplYZ9Kv41UdGf/x+HPyzjoVW5WN8uxy9OE1z69ZRtH1khB4msVzRE8YNEPEK9N8ieoPk7Kh1U3iZbXMrFnt3umXNjGNYf83jIBgYbfR0PlToVSgUNYksE9N+mVV289ERoLW8xaCMmcjbzX8pIdDn9d6q9JaickibgqFQsHeOXs5vPDcki5Oec+sLM9J4S17chmrXztD8jG0Ewz4HuLG1PucctKviVCRBTO1sP0pAJLCxfxkooCHMh5i2sJp/owOgKC4IBL7JFJeKTqsQkIQifOsRVC2v85jGvv8KeMnMn6BjJ9dL517hcrgmk6/fvf344niJ4jt5pv1sBOFUoE2PBBjkNijCQ8M9un1ZXxMaBq0vAH0Yj/GmfQrqSoRfsiSROX+DLI2+87Tz5hj5KcrfkK3ezPtI9ejV3g+N+JwOMjMrH/XfmORk34eRnJqx89vDasmkBgqbuCDifk8kv0InSZ38mN0Al2YDkdQCBISZsVJmt0AG26BP+RKivOePh9Ca5HYTYtKY/boHQzds43cXAiKDSI4IdhtseoPSvMthOeXElUQxU2BFZC10K/xyHiRPh/DZQdBkog0RPLvQe/R/ei35OY5kCTYN28fvwz4pVaFq7dZeulSbvzmRqKkEDQYwVru0+vL+AmH1fV3HR8kdLeqNNkUF4MuVMfE7ye6eUZ5C2OOkRObTmAylqJXwIy4t2H9TV6/rkwTRKWFtndB7Khabzk3606W9yw+UsySR5dwdPlRJEmiqKio5vnzHHDYHUgOCZMJLOpqSXAvVDTKXHgEVef2TB60ah71yigeynjIzXM1r0osmlPCfFcxK9P0cc5jdpWRUtPplU0aMn/u/mW3m58f1Jb3dCb9EhJOOdhcJBJRpXvqfT2ZBmBIEnYrendt1Tt+uwPDKwY+2lQj9/vAkQe4YfkN53T6sDCRFJEkyMjwRMAwZd4Uxr4zFhD35djpY5k6ZyrFldXa8/p44cMV0rbe53TKjyadm2WhjKfRxUDyJAjvDkBKlEj6VSkK0AZr/b4nA1BVUoW1wkphiQW7wkxoKKIz8a+LYfcrdR7TmOdPGT9y0Tq4aIPrpTPpZ9eLTr+CArFfrAvT+Vy61WK0kLUjH6WtEoBuxfNh2Siwlvk0DhkfEdUfBnwrLMaoI+kHJC3/jhXP/+WzkPQReq747grsrWJ466IBROe97vFrSJJEsdNXxgfIST9vEdYNgtuRGiWSfmXaHFShQag0jdOQbyySJJG7I5fSLBM2VSmSQhhZu+Q9U6dB2qN+jFDG0+g1eoa270KALZb8fLBU2Sk8WIgpz4M7MQ1AGRLE/OsTWXjxQh7UHIGDH/s1HhkvEpgsfhQKtCotT464n5SSaThsQrIutmssLae0JKKNb00nlLFKSkNLKTFHs7v1Iehe96JGphkhScLfad31QO1OP1+yb+4+vuj7BQFHbKiAv63D60z6yFwg9PkI2txWa9iZ9Du50y+hdwI3/30zXa/v2qhLHlt1jJc1L7P7uy2upF+qohLSZ0CV/5UpZJowDru4T3LcPcvzTflcMfMKvraLbhRn0q/sRBmzr5vNjh92eCwESYIiu0j6tYo6TdJv0z2yP+WFgMMm7sfclQCEBoSiQgNAbnmNbLIp38S3I7/l71f/btBl7FY7c6bOYcVzK9zGv7jsC1bdtIpL2l4C1Pj51Ur6le2DjXdAluzf61WCUmHEn5A80W3YuaGYVV7TNRAQEnDOG9sKheclPhP7JrokY0vNpSw+vJgfd/6ITq0THzjHxIokwZEj4r9bt/ZMjDINRKmGIbNcBdmt42Lpe+BPBu5bQ/7+ItJXpvs3PuCniT/xbsq7bDX+zoJeOp47PBIUKuj7GbS80d/hyXiSgEiXvyTArT1uZen1S5mU8AAg5D0ddgd5u/Mo2FdwurN4hcy1mXzb5yO67ehHcGUnwjBC0VZQ6nwah4x/CNeLTF9xVTEhIaBSK8joNI60m/r5LIaAkAD+o/sPv3X8Fx/uvhdr7GU+u7a3kJN+3mLoHOj5Fslh1fKeqhyObCyg5FiJX8OylFv4pNsnHPp4KZZqaU+DxlDzQNnmduj0tB8jlPEIu18Tuv/VRESAWg0OB+xZdJwP2n3g0Y2XhlBZCTZVKQoF/Fs7Grq+6Nd4ZLyIzSSkg6urtlQqiK5uJsnNhfBW4XR9oqubP4ovCP13KHMnzkVjD8NQt12BTHNDoRCL12gh9XZy0q+kRHxkw/sbmHH5DK9Xrib2S2TUq6PIicjFKMFi9YPQ+mavXlPm/MJkMfHAP4NZ1iWVvKKabpWAkABSBqe4yRw2hICQADpc0QFVdIQr6de6ci+svQ6M/veYkWnCKBSwbhoceN9tWK1UM2//PPZbl2JXmF3yngqlgl0zdpGzrWEyORWFFax+fbWbN3pFBRhVos2mXWwdST9JEhJW+Q1L8MicTyjEvFV9PyoUCr7sdJxLtlgIMNc8W+pCdeRsz6GioOFy8pPnTGbgowPdxhJDEhnSYgjxwaKz7LSdfqFpMPxPSLqiwdeXaTiJwYkAnCiv8awtzSjlyNIj2K32czqXM+nn7KZrDDazDZu5Rt79RJmIL1wXjl5T/T2/fBQs6FXvcxYVQWkpKJU1cqQyTYOIUC2x5eMIq+jN0qdX8O3wb8/5/vM0bca1oduN3SiqErISwbpAkRhqczvEjfRrbDIepuI4FGwESciypkWnMarVKDrEC3nqggJw2Bx83PnjWgUu3iYsNYweDw4lxvxvLj6yi8gh0+HqYqGIItP8MBfC+pvh6PeAe6efQiG66gta9CK4u28rV7Zmb+WwbhM/HpmCohnMf2p/B9DcOLVSLDZI6CA7KGDW8A/pflN3Jnw9wR+hAWLRPfzF4RwojUVzJJQrDG/St/+ZTa5lzkOKtwm5TIcNlGo+2/oJB9vsJTr9LqyhyQx8bCCJfRL9GmJRppHoE6VkhgaTq2/jkryQaYYc/BS2PQJj1kL0ALbnbKcsLpPKop7k5SWSlqYgLi7O5xISTk+/bgEQXf4VVI0XEiwyzZt+NR4vzqSfVVXk6vTL25XHkSVHqCqu8qr3aXyPeOJ7xBP2fE+iSwNICpbl6S5odr8GxVthcI3PhkFjYE/xFqoCqjhRfgJJao1zmnTYHZQdLyMkOaTB82dCrwSumX0Ny5eDNUck/aSkSZA0AkK8L3Ercx6jUMKQX4WM3kmE6kJRKpQ4JAdWdSEmk8h6BMUF8UzlM6h1DVt65u3KY9mTy1Dr1MT3EImVwkKo0lZ3+kXWMX8qFDApV5buvhBQqsT9eJK3Y0JoDErJXWJWpVXxeOHjrvlSoTi350+VRkW78e3O+rnTJv204ZAwrl7XkmkkBz+Bkp3C8qKahGDxF3Jyp9/at9ey8X8beSjzIUKSQup9ek92+u2bu4/ZU2ZzzZxrSJuY5kpKJoactFYPagM2Y73P6ezyS0qSfSWbBIe/hqw/YdAMFEo1ISEiKZtyWTfajEpBcvhXInPQ44MAePLeNwCICjy7+s65zp8yTYQdz8GRr+HqMtDUeOY5C7Lz80EdoGbkKyOJ7uhbyf/IdpF0uX8EFfdDWIBPLy3jDyQHHPkGlAHQ8nqSQpL4YNwHLuux8HDxrO9LNaZ98/ZxxXNXMG/8PDS2MAK8cB8qFApiY33nlykn/TyMUlndPJm9BLL+pFXKzXSwX4OlIoUW1w+h/YRTn/59izZIy7DnhnH4K9AegMsiH+WWIdVv5v0N25+ALi9CfP1NomWaIAN/EFIS1Xz3z3fsDlpHL90wSqwdGPOG//9+czafYMRCJd35ladf6ODvcGS8SVQ/SHvclVB7aNFD/KX5ix5BM8jNnYJSqWT/Z/tZ9NciblzuGwkRi8mC4XsD/bJv4bIeuUQfvhXabpGTfhcYnWM6k/6vYu65LZQyBdhsMO69cYz/ZLzPFrGDTe8wtHgNV/ImFD4Ckb19cl2ZJkbRFrEhJDlEQgWxKEgKTeZQ0UGMiuMYja0Jrl6fz5k6h90/7+aZimeIi4tr1KVLjVas6hIAQqN6gFPyXUbmTCRdXmtIqVASqY8kvyIfi7oQo1GsexQKRYMTfiAkbW9dd6vbpnxhIfQ+/CuBCcfoFHMaz3SFErShDb6uzHlE8hVuLwMDxe+KU5r6Tv5uVyqV5zR/2sw21AHu93F6STqfbfmMjtEdmdZ1GpJUI+8ZH3/KCRxWUKhB3iT3PjnLIHM29HzX1SXiTKKdnPTreGVHotpHoQ06t8yYs3vOE51+QXFBdLqmExGtxUans9PP2ZkIQL/PzumczqRfq1aNj0/GAxRvFfdjVR4YEiiK+p103T4K2o7nisF9/R0dIBShyu1CCSwmOEIUkG+5H3q9V2exwrnOnzJNhOSrRGFf9VqjtKqUn3f/zPF8I/AQBdWKnkOeGnL6c3gRi0X8DggAMmaLYplm0G0lUwcBkSL5rBZG4CEBIdzT9x7X2+HhEHfwb5ZesolOe+8kMDrQ6yEpNUrKgsoYGWbhoQ5TCaj6AQJPs8Zo6DWUSp8m/WR5Tw9jt1e35hesg/3/JUktcUvwT3Q88SZhk0bS4YqmkdxwLoACT/53Yy4Qck6Sf+UFZDyA0n1B6uw4tWjyyM31R0C10abEcbTbBDp3KiPmtyQ48q2/Q5LxFtGDoMfrECxa8yP1YkPZqi4kJ0fMm5k7Myk8UIjFaPFJSBX5FWR/lE2PfyazfdeHVPT9HYLPXr0t0ww49AWsngwOOxqVhuToMFRKBZIEZWWg1ql9kvD7/V+/883wb6iogITgA0QZZwqZC5kLk0EzYLLJtQh3khIqOlcqtZluvn5tL21L/wf7YzaZOXz4cM3z5zmw/dvtLHhgAUUFpUSVjSJR1Z2wADlBInMO1CGD7PQJt6gL3Lqs8nblcXhxw2RjtYFakvonuSX9ioogtKInfYInEhJQR4dO6T7IXQG2hks5ypy/LC34lq0tr2On9Ve38fw9+Wx4fwMVhRXY7fZzmj/n3jCXt2Lfwm6p+fyO3B28uvpV3tvwHiCeI0wmkdertR++4Xb4OQis9e/YkmkgfT6Ca4xusnAnd/o5JdxbDG1Bn7v7oAs7N8+olBTxd1xSIjq2GkPqsFSu+ukqYrvGuuI7Od6GICf9mhjdXoUpZjCIv9O9If9jb/JjbMza6OfAQJIkZk2exao3N2BRiQfN+NBIUYSm0okunDo41/lTpomQeAl0fAzUYiPYaDFyx+938Mqmx5FwUFnp3iHvS3ZM38HCaV+wJakjc+I7I226C3Y+759gZLyPQim6TU+z7xIeDg6VBlV4CNYK36gTJoxO4PPbP0fTJoMofQFqjcbj17Db7Rxxfkn7ADnp5y3a3gUTjkFoJ8KFHyVFRf4NCeDoiqN8M+wbynakU6E9xglpEznGan+N5IlCBid+rH+DlGk8ldnCJL4qD4AYg+heMqtF0m/Vf1bxac9PcdgdfgvRERxKYXJ3lLGJkHINBLfxWywyvsXZsm9RF5EnblE6PdWJB449cM6Vtg0lOCGYmzbdTW7L/uRXtECbeilognxybRk/U7RF+DxZhFaEUik040HIR9itdtL/Suf4huNeDcNqsmIuNWOqtLPs6M0c6m2GuFFevaZME0ZZdxdUUoiQT6zSHifnJDu0btd3Y+w7Y9GF6Sgvb5h84eGFh9n4v41gC6f/gaW8lLAN1aKesGhAg84nc4Hx99UwK7zWsLOwR3T61YwveWwJP038qUF+qcZcYy3PI2cSPOJ0KmRHvoJlI4V/jUzzZ8UlMLfGvOygaTNZkTPIYovbxw4vOczC+xeSu0NUQZ7L/BnVMYqUISmotCrX2LESoe/YIlRc29nlFxVVh6xiWBeIHenabJXxIrpoULsbdscHidZLs91MUWXjNmZ0upqk7mEPW+C65D2dnX7F22Hro1Cyq97nkJN+TQxNEChrNo9DNUI2sWxtDh91/ogjy3y3AXwqdrOd3b/sJnP9Caxq8e8iOjBSJIcu2XHGLquGPn/KNB2iA8W9aHPY0IaItXFBAWz5fAsfpn1I8VHfaSsac4yU7cvDqsrHpMpEMeBb6Pqyz64v4wdKdom9mWrWH1/P/P3zKaosIjwc8lr1J+Th2whrEeabcKpKAPioWM0T6zIhxDtNAUaj74q/5KSft9BFQ2AKKNWEhTuwqArJ+uo3Zlw2w69hVRZVkr8nH7PRSmbUlzyyvy8vrXzJ/UOy5Mj5T/YS+OsSKFgPQExgddJPk0tuLlQVV2EuM1NZVOm3ECsr4XDsW3xrncnOts+KbjCZ5okxHf66FI58B9Qk/azqQlfnqUJZe96RJIkTm06Q/le6x0NSaVU44vVY9IHotFbUstj1hUPPt2CKBXRRALz696usSZpIceA6CgrAVmnj25Hfsu6tdV4NY+L3E7ls1WV8l6pmUfcI9IGa0yZ+ZC4AKrIgaxFUFbgNJ4fUdPod93DuYsI3E3jo+ENUmMUGtsEARPaFiJ6evZBM8ySsK8SOEN0AJ+Hs9LOq3JN+fe7pw/hPx0MDrIt+GPcDH7T/wG1se852jsS+Q57hr7oPajFZyJIFpp77BWXOP8K7QlR/18vYILGRaXTku32swxUduGHZDcT3PFV78+wMf34418y6xm0sozQDgJTQFKDGz6+WtCdA2iMw/Dd5re0L7FVQtBVMGa6hAHUAV3W8ipu734y9WtmoYF8BH3X+iE0fbTrnS3SoFnDat69xoS55YgnL/2+567XJakKBoqbTr2A97Hu73gUMJhOuIiE56ddEsFVC/looOwhAeIBYg5SYS3FYHdjN/uuWU+vUPGd7ji7PXo5FLappImWZ9+ZL1kJY2BfyVgGgVWkJ14kCLl2UqMY+2UPNXGr2WWgDHx1I6NdjKYguQKcIFrKyscN9dn0ZP7D6alh7vevlTXNvYsLMCezM3ekq6vNl89S+BfsYtHoQgRXRhIU2j2c1eXfJWzjsUHkclFoeOzKG9B676TLjfYpL/Jtn7XhlRzpe2ZFHHwVLZfWXenVFLkenQ0A0JMidfuc90YOg3xcQ3g04Sd6zutPvom8v4qK3LvJnhGT9vJqLF+bxzY3fkFE6ii6xXfwaj4w3cUDucrGZzEmdfqoi8vOFj5q9ys6eWXsITQwlZbDYOFEoFPw86Wf0kXr+tf1fHo3IYrIw7YfLWN9jFUdT4mBhIlzsf4kVGR9wSoX9qoxVHNYuJEh3OcePD6BfvwAu+/wyl8ySNymuLAGgS4CdEMt6kPrVkneUuUDI+hM23g7DfoPE8a7hmk4/96Sf3WJnztQ5RKZFEjHpdK1OZ0YdoCYkMcRd8r3fFw39E8hcaHT5vzqHo/RRBKgCsCur3CSi2o1veLVs+wntkezu2cJNBcvZk/wIUeYpwPDaB0X0Ej8yFwbdX3N7GRciNtUrFPk4HKKrHyCsRZirYtwTsnTHSkWn36lJv4SGKzPKeIKyfbCwF3R5AbrUyMP9cvUvbh/TGDQ4rA4kx7lXI3ToACtWwN69jQt190+7CYwOZOTLoqPq2yu+5cvLv8TuqL4/W98KcWNAV7/n0qNHxe/oaFw+wDJ+puI4LBkkPO57vE6kPgoq4VhqAdP3/8ff0aFQKjBWqokoH0KYIZiWYS2FalT5IWhzu5D5lGkeSHaoygZrmWsoJjCG4qpiVCG5QBrFxTDi9l70ut33z1AFZaJ7VK+UJ68Lgo5PADXJtTBdGCA67qLDQGWpoHzRNo4OSKDlyJZeD+f4H8cZs3QMbfv+QL+I2cCVXr+mt5GTfh7G5QNUmQXzUqH9A0ToI0ivgJ3jYnnr3av9Gp+Tigqwak6p5Nn6EIR0kJN+zYHg1i7/NDi50y+PoiJhkFtLcsbH2BRaKnVVXBlhpefxL6DVcFnuprkS2BIm13SVOgsNbNoCJAmKihQkxCXw2YDP6HZDN1fSb8+sPbS7rB3tLvN8W/2eWXu46InhFFybxc6w9qTEtvf4NWSaKJZSKN0FQa1AH09ckNBnMmtyyMwUH+l5q3c7nSSHxKaPNnEiUnQgPBdpJ2jdYEj1jV69TBMkZhj0+QTC3e+91LBUYgKS0dgiOXGiZlylVZG+Mh1Jkuj2QLcG+VBmb80mMCaQVRUzWNLtGVTZU7iMD85+oIzMGfhk/Ce8NfQLbrhBQYVC2P41trFp+PPD3V7n5MDhggyIhY6JybUPcEqIyh1VFyyJYaLTz6LOp7LS3cdeckhUlVYREBpAcnJyvebPoyuOsvPHnQx4eADRadGucWen36nynrWSfnYLbPqXsNFoMbnhfzCZ+hHUCrq8dFbZ9NCUUO7df2+DLtGxo/i9bx9uieVz5f7D92Muc++mUSvVqJ3qD0qN27r+bDiTfnKXXxMiMBl6vAVRAwGIDoyCIii2FJzlQO9TVVJF9rZsCk5E0SHr3wxoAX0SgTXXwbEZ0OaOOo9TKBT1nj9lmhCJl0JipttQbFAs+wv3IxlEp19JiR/iAo79fYzyNRmQDD0MKuGB2+VF0SUv0zxpdZPby3C96DotqSqhdwtQ2Szo/17Kvvn9fZL0m/TqJOamlXNjyM8EB/8f3kj6KRQKkpKSPH7e0yGXk3sYpfNpTxcL7R+C2JHEBIlKw8JK/3+pZ67NZM+sPVSU22va9/WRYnE8ZI4wGZZpdjiTflaN+CLPOmbln+//8at+vHJAX766bQaj44zE58wFpZ+zkDLe45TFQHyw0Dyy64X2TH6+kvjUeK76+SoGPi4WQ5Ik8ee9f5K5NpO249p6PKTItpHs6LeLoogifjj+MvR40+PXkGmiFKyHJYPhxO9AjcdLlSbblfQDcNgdmMu9I2lSVVrFgvsWkP5zOgDfFcYLGTq5y+/CJaQttL0TDO47xRe3uZj1kzPonv41x4/X5DIAHj7xMNfMuobIyMia5896IkkSXw74kj/u+oMSay5WTSFBqjLY8qCQ/pGRORv568T9Un7IbVij0hAUJL73JQlXJ2nOPzm81/I9Nry/odGX/vNPIXkLp0n6VWQKv8E98nf7BUP2EtjykMvPPD5UJObM6ny3jlOAj7t+zDdDv0GpVNZ7/jy+7jjbvtiGrcrmNl7vTr+qHDjyNRR4VzpcphpNiOhGjh5Y661Ka6XLt6cxpKSAXg9VVXDsWMPPo1Qp0YfrT/+Bkp2u+7o+OD0G5aRfE0KlE4mLaOGZ7OxENlYVsuWzLRxe7GFjyHMge1s23438juxlomU1NLT6jW7/htErQRVQ53HnMn/KNG2ce4UOvZhnSkuFosiWz7dw8M+DPotjzetrCPkuHRQQoA6F6MFg8F1yRMb/ODv9iiqLSEgAfWwIewfdStxVg31y/ZDEEKwxiby/6Ss2OD7xyjWUSiURpzUk98L1fHalCwSXTIhKC73egaTLiQsVX+qKwny2fPUPlcX+81Hb/PFmfrn6FypMwmcQqjv9FAqIGQIxvvnHJONlbCb4rR1sfgCA3gm92XXXLq42/g1AXh7MvWEuWz/b6rcQKyokrKpSpuXAidHb3MytZZohOcsgdwUA3WK78e7YdxmlehGArCw7+/bto8OkDsR0inEdMvnXyS4ZWrvFs14HSQOSmH/JPAqiCwgP8N2XrkwTIKwT9Hjb5f1zaqefJEF5djlvRLzB8meWn+lMDUZj0HDjihuRLhNyOWvKUlG0b1iluUwzw15VyyMtIUE8phmNUFajxoNKo8JuF/PnucrUSQ6JEf8eQdfru1JmE0VprfQBsP89eVNapn6U7RH3S3ntDSGNpkZRwunrZ4g0oAvTodGf2/PegT8O8NPEn8jdKUyAzWZYsgQqtWKXvUVYi9oH2SogvDvoYmq/J9M8KdwI+/8LFaIlOsog1t8WdYEr8eyk/YT2tL649TnNn4OfGsyjuY8S07nmnjLbzOQYRQFbSmgKknSGTj9DMlxVDJ2fa9AfT8Yz/GfVfzC8YuCppU+5xnb/spstn20553MpldC+WiikoRKfFQUVHFt1jMoisT+UY8xh+DfDuXHujUjOKp9lI+HvSfU+p9zp1/RJDBPzk8lRyO93/s72b7b7LZaINhGM+2AclsQU7IqqmqRfUCuIGXra4xr6/CnjZyQHHP3BrcAvxiC+1ywa8ZxVUgIKlYI/7vqDrV/4br9w8FODOXi5aMsv0STAiIVyZ3xzZ88b8EcnsAgjySi9mBsLKgpQKKBjZyWm8CQO5/hGEa6yuJKSAisHCvtRFXL6+a8x2O12Dhw44JVz14Wc9PMBccHixo07buL3W+dSdMiHTpSn0O/Bfkz84UoqLWqsJ3f62avAYTvL0TLnDSoDqINBI7Swg7RBdIrpRGpM9SRaquHa369lxMsj/BaiceMW+mzqgQ0IDpVXJc2eDbfCtscASAxJ5MH+DzI09jIAcnMVVFVVAWAz23DYHUIyZEAyCb0TeDv+bf6890+PhmO0GLFJVtpq4KaU5yHvb4+eX6YJY0iCtIchTPiIOpN+Fm0OlZVQWAhBcUGkDEkhsr13jOzVAWpSh6dSHC02efSKMK9cR+Y8Y9978EsIlLrvHgYECG8ewE3is+xEGbtm7KI4vficL6VUKRn02CA6XdMJo0Mk/aTgdnBFJrS/v8F/BJkLiJSrxf0S6y6fty17GxNmTmBXqvDidXZZhSSFcOe2O+l527nJJxceKGT/b/tdnlurVolEolnn3mHlRmgHGP0XtLrx3P5MMucvbe+CCccgrDMA0QYxadrUJZQZ3TelR/1nFBe9KYrKnM+fZ0OhUBAYE4hKo3KNaVQajj14jFU3rSLKEEVZmbjfFQqIPdV+TaEAbRjIhWa+Y/tT8HuaWyGNMxl8orzmy3TDfzew4rkVDbpEWpr4vW9fw0JMX5nON8O+4eACUTyRUZrBymMrWXZkmZBNlCRIewxa31av89lskCEUZ+WkX1Njy4MwvzVIEn2Su9P3wAL65k5n2uJpDHt+mN/CCk0Ope89fckOMrOgl56706vXPtayWkVop1Lf+VOmKaEQHuL733ON3N3nbpbfsJxJLe4ERNJPqVJy/eLrGfPGGJ9FljIohdzUKIIqO5Ia6n0pR5kmgMMqfCZtojorOlA8u+VXCAuUTp1A4bCze0O5T8J5Z+A7ZL/2LMdi3yEkxHvX8eXcKSf9vMmOF2DZKKHZDRxuk8ugD6YQ0cZ/D/sJvRJoPaEzKBQ18p6GSDj4CfykF7JnMuc/CgWM2yJkGU7CuQDNyYF2l7Yjsp13NrTrg7RpC0NXDaVPAAQ1AT17GS/T8x3o6m5U7rwf86oVaza8t4FXDK+Q+08uJcdKcNgd6MJ0xHSJISw1zKPh/P3O31z1y1V0lALoE/o1GP0nqyLjX5xJP5tOVOtnZorNvet+v46+9/T1yjUdNgeSQ6KwQiRrlrZf6urMlrmACWkHcReB5O7teP2v1zM7OZWC4L84frxm/Pi648y9fi4FWxr3HVqBOD42LE4kxQP892wgcx6hCRH3i8pdnt1oMTJ//3xyDaJT2tnp11AGPDSAZyqfIaZzDJIEv/8OdmUFVSpx3zq91GQucAIiIDDFpRwSHRjN7aU5jNtShblSdZaDz4zdYufAHwcoz3bfdFIqlKSEpjCkxRAUCoVL2jMqqg7vdFMmFO8Au3dkw2XqQLIJ+whrzd9bQrBowcwqz3KNjXt/HNMWTmvQJZy+fnv2NCzE2K6xjH13LEn9ktziSgxJFB9QKKDj47V8j05HRoZI/AUF1RQLyTQRVDoIiAJ7BcnR4cSUXYy6sButRrcmqn2Uv6Mjr1zsDQaohAoJ89vAMv8ViMt4CYUChsyG7jW2Tp1iOjGi5Qjaxop5yOnp13JkS5/vXcfn3sjw3bt5tdPFonCj4vjZD5I5f+n8DIzfBwbxnecs2HIm/Tp2hLYbpmN/7yM3iwtvYepj4li3zewY+Qhpthe9f0EfICf9vIkpHcr2EaMTPfIF0dnou7c/s2a7L8IygYREWu6LPDX4KbHhGdhCmLoa6qiWlWkW/G/D/5hvvZ8K7VFyRec+FqMFh+3MFVzeImtUL36Y+gMrkkC5+R6/xCDjQ5InQcJY18vtOdvZr/iVKnWO636Mah9Fh4kdUKqVfNHvC74Z9g0A1y++niFPD/FoOFmbsui4pyOLTOF8UVIgOhZkLhyWDIH1NwMnJf3UJUBNhbQ32TVzFy9rXiZqezTxZaOxS8GgaNympEwzIGEcDP9dyBKeRFFlEaWKY5gCDrgl/VIGpzBpxiSi+pz7ZlH6X+l80e8LDiw4TJWyOnkSqIXyw/KmtEz9cNjE/VKZ7TYcaRBJY3N1Uu7kpN++uftY9syyc76USqNCqVKybx8cOQK2QDFRB2uDXf4fbuz/H+x965yvI3MeYzeD8QiYhaKOUqEkWh+LEnUtT7+iQ0XMmTqH/fP31+vUhQcKmTF+Bhvf33jGzzmlPePj63jz0GewoFstD0wZL9LjTbjkH9CGuoacybSTk37xPeOJ6x7XoEu0ayf20HNzoagBYk6RbSPp/2B/18b6iTLRgZgYnNigeI4cEb9btaplqS7jb7q/BmM3gDrQ1UHicEB5uYS10nrmY73Ixg828lHnjyg/Lu69MGc3cvKVED/Ob3HJeJGEcbXWGgBhYeK3M+knSRJVpVU1UsNe5sO0D4leOgOAUMvfsOc1sJT45NoyTYOhLYby/rj3ubePsD1p1QqMyR0oSOpG+lHv34f5V+WzZcxyFuZ1grCOXr+eL5CTfh7Gzch2wDcw8QRtozvTRTmZyPJRDXoY9CT/a/0/frv2RxQo6GZ6hFdGvUKQNgiSJ8LQuWA41YBA5rwlayHseM4ly/DF1i/4o+B9TLqD5ObC2rfX8mrwq+T8k+Of+FTd6ZS/joyYJ6H1Lf6JQcZv3PXHXTyzcxLFQevIz1fQqlUr2l7SlmtmXUNku0h63NKDTtd08tr1h3w2hEWvHCCm7DJU+khQ+0YnXKaJINlxlou1Dm9NyRMlfNxKZJ+dSRVjjpHZ181my+fn7vNyNoLigmg/oT29Wl5BrwNL+DEzQ/gAy8jUQZvwNgCYdIfc5D2D4oLoPLkznfp3cn/+rAfmMjPlWeVUGB1Y1CI508WyAX5rI7yxZGTORuUJcb/sedNtOFJfnfRTlCBhd0u47J+3n9WvrKaqpP6yNjtn7HT5+f3+uxgb2y+VzbdvZtY1s4QE3qkc/AQOfX5ufx6Z85uC9UI6L/0H11Bg9aPdqZ5+kiSx88ed5P2TR6tWrc46fwbGBjL+0/GkTUpzG/9l9y88vexpVh1bBeDq9Kvl5wcQNxo6Py+6EWX8hrPTL9eUi63a2kSSJMzl5gb5hxsM0KK62bihEp8n40xGOuPk8FewZDCU1u/kzqRfS1kZr0mjVkNB/AwOx77F9Eu+4p1E/60BJEnCYXNQbCsFagp36PsxdHrytMcplcp6zZ8yTRSH1bUWLqos4rMtnzEr430ASkvFWwsfXMjrYa9jyjWd6UweI7RNFGZ9GADaLg/D+AMQ3NYn15bxE8Z08cxeJjzuOsV04t6+9zKqlbAOUKkg+tJ+ZHa6mL37vF/JUmIu4agNHttxF6pU7zQFKJVKUlNTvXLuOq/nsytdINS18OyX1I97YmbS5cCNbJv8GmveWOOHyARxPeLQJ4kv8kB5j7t5c3we7HoZzNWyXUFCS9GsziU3F+K6xdFlahfUOrVfwrOVWNCbkwjs8KrcZXUhsO0JmB3tkthxLmbNmiyKihTodCGu+VOtUzPqlVH0u78fACXHSvj9X7/XuyK7PrSPas+0oBmMLXyMOP3us3oWyDQzLlorCnMAlVJFqC6UlBRx/zk7/QJCAtgzaw852zxfGNFqdCsmz5mMMl5UlhsMHr+EzPnK8Xnw13hXtwpAmwiR9KsIOOTW6QfVPlO6wLoTH2eg/eXteSjzIWIHtyWkshthld3Rxw6EtMchSN4tlKkHAdHifolz9/SL0IsuAQkJq7rYrdNv6HNDue/QfWiDT9U+rJuq0irmXDeHdW+vw2SCtWvF+MTLdPRK6MVFrS+q+8Axq2HYb+f8R5I5jwlqCWmPQngP19Bm1f/Y2vJaNuWtcvtoROsInip/iuEvDCckJOSs82dgdCC97uhFQm/3bN43/3zDq6tfZWfuTuAsSb/YYdD1BZffuowPsBTDgQ8hd6VrKNoQjUqhwiE5yDWKYoItn27htZDXyFjdMKmHxvj6/TTxJ74f873rtdNr0NXpZykWm6Hq+ilFOZN+rVufeywyXqYyW3Sg568DYHfsM+xNfgx7Rx3tL2/vs26qU+l3Xz/u2XsveboyAKKD6ifxrlAo6jV/yjRBNt8PM7VgFYneosoi7vz9Tl7Z+DQgJIJNJkgekEz3m7v77N4c9tFkfhs/k1VdOrEocxWEtAVVgE+uLeMnSnfBprugYO1pP9Kpug9g927vhuKwO4h9I47em3qjtocRGnr2YxqCc+70FXLSz8PY7SdViFXmQvpMKDtIRATYNDpUyYkEJ/rvYf+aWdeQcvtYLKoiKiM2klGaIco41lwHh77wW1wyXqDj43DpbtCKzZeYwBgAzJo8jEaIHdCKSdMnEdMpxueh2a0O2sx5k5bbfpU3uy8UdNEQ2sklG+dczNr0WUiSxKpVe7Hb7ax4fgVLn1rqdqhCqWDLp1tIX5nusXCOrjiK8WA2N3R7hnGOzqLzS+aCJjlZ/M7MFF+LGoOGxwsf59KPLvXaNctMFmIDjzIg/N16V3HLNHNMGZCzGEprDIKcST9TwGFycsRi3MnPV/3MGzFvuD9/nsvlTND30G9MyNlGbMuroMfrwqdNRuZsqA3ifkl0nyM1Kg0hAWIxa1EVunX6hbcMJ6J1BEpV/ZagKq2Ka2ZfQ687e7Fnj7j3ExKE3M8ZCYgQHpkyFw6BKULOMWawa+goK8iKnMmRcvedIoVSgTZIi91uZ+fOnWedP+va8JQkic1ZmwHondAbqJH3rDPpJ+N7rOWw+V44/qtrSKVUER8s9FedCbaYzjF0u7Eb+oiGWbA4k34N8fVT69QolDVJE1fSz+npl/YIXJknrFjOgt0Oh6styuVOvyaIuQC2PQbZCwEIVAhp9opLgrnimyv8mjwrLwezSnj6xQZHCqnktdOEctRpqO/8KdMECesqiu4dQlbWuU9otBgJCBKt8SUl0HlKZyZ8NYHgeN/sX5eWgkl3kLKAPagqs2rJx8s0QyL7w8glEH8xAHaHndUZq/l176+ubvx4xwlab/6Jw8uPedXXz2K0EL05in75CTyTNgOd48TZD2oAdrud3d7OYJ6Ef1p8LhRKd8Haa6H3h4SGt8IUbKVqynV0nepf357iYigM/ovFuis5PHsga66bB8dmgiYEuM2vscl4kFMq9WMM1cm9wDxAeA+cddPES1QY7WS0SUTZeimKlY9Bv/cg6XL/BCPjG9IeFT/VuGRrQkRZdF6e2AA8tOAQWZuyUKqVjHx5pPhIUggPHX+I4ATPPXDOmjyLKn0kK8ddR1xad9ooNR47t8x5QN4qKNwE7e4DlZYvt37JrD1zKAi/FYonUVYGoaEQEOyd6sKNH26k8EAht0dcy/ieNoYEmqA4FkI7eOV6MucRrW6G1re5VfW7Ov10h7A7JHJyFCRV5+Viu8VSZizDZrahMtT/+fLI0iMYc41IHTsDSrkAR8ajROojKTOXYVG7J/0kSRIyUQoIig0663k0eo1LUnHp12Ksc2eYvmM6eaY8Lm17Ke2j2rsfZC6EqjwIag2q+nUUyjRPwrRRUAWFVfm13is6XETW1ixsrW11HOnOpz0+JSw1jClzp7jGjpcdJ8+Uh1qppmtsVyTpLJ1+S4aILsTe/2voH0fmXNEnwKjlteThrky7knJzOcFasa5IGZxCyuCGy646k36HD4PFAtpzmHaunHGl22uL3YICRYM8/dLToapKqDmlyCqyTY/gtjBmjet+DFZFkQPklBX4Nawd03dQVARWtUj6RQVGgClTSCVH9gMuPu2xcsLvPKXNbeKnmmBtMDq1jipbFdrwPMzGVEpKcK01fIG53MyWN1aTWKWjuBP0z3gfDj0jih5kmi+6KCF/fhJDvx6KhET2I9nEBcURF24mLGc/6dntyM9vQYyXelZ0oTq+fmM6/1If5apYO0je87j35dwpd/p5k7BuMPhnpPhxXL4uiMXdo8ks8U62uD6UZ5ez5PElZK9Px1L9pR6hjxD/0CZXQffX/RabjBeQJLHxYRFt+84KHkWQ+OLMy4Mljy9hyeNLfB5apVXDumElHB35KXZLvtxldQHiTPpZdWJOLCwUNSgTv5tIZPtILEaL67MKhYKQRM/KhxTcWMDMgW/wrbSVvOinPXZemfOEjFmw7VGX/PGO3B0sPPwnVTFCfjszU3zMXGZm54ydnNjo2e/u/fP2s+2rbVRSyhKziT9YWOuBV+YCRRNUS8arRVgLVAoVdmUFZk2Om8Tn4KcH0/u13qgDzq2Ob+MHG5l741yMFWJeDQwEtj4KqyY29k8gcyGx4Q7RDXAKkYZItAoddpXRTd6zJL2Et+PfZs3r9bM6sJhqngV27RK/O3eGz7Z8xiOLH2FbzrbaB2X+Cn90FB2zMhcOkkNII+94wTUUqYsGoMRSe1N9/bvrmX3NbKryz+wvKUkSMZ1iahUBObv8Osd0Rq/RU1YmvAMVCoiNrSM2SzFYy875jyXTCJRqiB1Rq3v9vxf/ly8nfEladNppDjw3YmMhLEx0Ijs77RrKyptWYvk/C8NSh4m1/L73IL9+86Wz07BDB5Bt1pogKh1EDxTqN1QXJQDl23KZd+s8Cg8W+iWsv1/5m63//Zugyo6kVI2na2xXIUd8TQW0ud0vMcn4FoVC4dorVIUK2eOSEjDlmZh38zz++e4fr8dQkV9B+verScoMA6As7hJRnCvT/JEkqO7qUylVLl/RfJMo2Gp3USrlDz5LYXIPr0t8alR6XiwO4N5/loOheVTPyI8D3kQXBSlXowhuSViAkFhUrV7Lxg82+iWc0mOlrH1zLSW7s1yVPJH6as1ulRa0XhKtlfEPRZthdhQc+gSo8fSz62o6/Y4sPeJRn7T6UlICNlUpyyrhxZAbIVneZGz2lB2EnS9C8XagJulXqRZl0cXFotMuqkMU9+67l7Fvj3U7vLK4kgN/HKA8u9wj4eQMymF/h/2oHIEEnb3ZQKa50e5eUW0bIL4Du8R2AaAyWPjyOH39KosrmXPdHLZ9XcfGciOYMm8Kl224DIAyqx5N1EWg873UskwTpfwQHPoMHKIgRqvS0jexLy2VQ7Apy2v5+jWE4S8MZ8rcKazIWMKi7pH8ETQRyg9AyY7Gn1zmwqH8IJTVliZec8safh9QSXTZRW6dfiFJIfS+qzcthp5dqg7g68Ff83nfz6mshEOHxFjnznCs9BgAKaF1LMjDu0HHJ0XxpcyFg0IJheuhvGZdE2UQm+ulttqdfl2ndWXCtxNQB565YEKhUDDph0lc8d0VbuMuac94Ie3pnJejo+vo9FIo4dJdLi9hGR8iOdw8ck/HXy/8xexrZzfoEgpFTbffuWxIZq7LZPVrqyk74Z4MVivVqJVqUby79UE48k29zudM+nXsWP8YZHyMw+qSLIzQiaSf+Xg527/aTvGRYr+EdNXMq+jw5ARaFNzBZPtvXNflOvGGWi8SlTLND+NR2HQvZNcUR8UGir1CZ4NASQko1Uq2f7OdzLWZXg8pJDmE1LfvZV1/UeRgbX07dPk/r19Xxs9Yy4S/5MY7XUPR1c9u+RXi2U2pVtKxs0hdNURGu75UFlfyhnI+F6/PJVw7QhQONQPkpJ+HUdZVViVJRAWKL/XofUfZ/o33KyXqIr5nPPcfvp/KDt1dnX6R+kjhG1OwHuyWs5xB5rwisAW0vl1odlPT6WdR1yT9blxxI/fsvcfnoWVsyGbgcjspx1II1cnJ5gsCUzrsfAEKRdGDM+lXJmWhUIBSGe82f57sbwGQviKdGeNncGjhIY+EU1RZRIAC5vd+n5TyVzxyTpnziJB2otq22hy8a6yYJ/OVIuHh3LwLTQnlyhlXMviJwXWepqFo9BpKg0QXdrgtjqhIudtZ5iQOfykWP6W7XENrb13LK21XEWRu55b0qyysJPOzTPb+svecLhHXPY5249txJDcPq7oIhc4Iw+bz/+yddXhVx9aH330sJ+7uBsFdi7d4W6AtUHejcm/dv9v21m/dqLt7C6VIoVDcCS4x4u6eY98fk3OSk3PiUkj2+zx5Qmb2npl9u+/smVlr/RYXdjJMQaZvcd5GmLPXplij1IjoUbCK9FOqlcxfPp+4ha1LGZtMJoLHBxN6TijHj4PRCH5+4OmtJ7NMRF+Hu9sxHnqPgeHPgXNohx5J5izm4gI45xvLn37O4uCozGBr9AsZH8Kwq4YxeNRg+/v3JjRVm9ibbZ3Pz6wQECq/dmcWf18Av4XRNBFQla7KEkUAUHiykMzdmVZKI+1hqFhGsnGjTVfNkrAqgQ0Pb6C6sNr+BWpXOG8z9P9Xq22ZTLLR76xg42z4XfwH8nES54OnBuTwYMmDRM+M/keG5D/U36Lj6OZWX1hxGgp2gb6ZdxNx7tm/f/82zZ8yZxi6ckh4Gwp2WIrMZ4VGR3FWWFoKWk8tD5U+xPx3ui+/vRmlWkm51p1SD9G/fEbYR1C5QNA84bBXj2/92q3xNzpIn4ZLUVq3Rvpl7c3i0N2fEVX6F2Feyd3Wj0KhIDY2tvULu6q/Huupr7JmNKybgL+L+KhvnR3E3C8v/0eGotQo8YzypKTWqSHSz8kbTrwC6yaAvmsiaGTOELR+MO59CJoLwJTwKRxZdoTnBq8GICdH6Bb/E0mjCxKKiUx0YGytK+fok8DQsrSOTC/AZzzMOwRhSwFxUPfa7Nd4aPB7mDBRUNByLqrQc0KZt3weEVMjOj2UwlOFDPzXIM7ZMwYHhQFHdWXrN8n0Pow6MAit9kG+g5CQKDPmUqvKs0T6SZLE4EsH4xHh0aVdZ+7JJDVRRKq8ElDFyJNqIf0lIwMQcSVM+RVcrBPvmnNrZDZSm1Wqlex/Zz9J69pnrDMajKKtYiF751e/TpWR6SrMUfSNjX7tQZIkzn/nfOa8OseyyR8yBDLLMjGYDKgVagJdA7tmsDK9En9XMa9VmprPmaUr1TVbV5FTwUcTP+LUqlM2dcnF4kDIbPQzO2PYzYGUtRoSPwBdB//PINNxAmdDxBVgasjd+NWhr3B+1pkrf2mQJp7/znzuTLgTjUvH8oBOnw5arVCKiI9v2z0T75/IDTtuwCdOvKe7M3cz9dOp3L/ufnGB0gH8JoPHkFbbysuDoiJQqaBfvw49gkxPELIIoq4Bo4EQL/HfPd+YL85kFD1/JmPUG6mrqKOsDAxSLR4e9RVJH8K68VCZ2uL9mvYksJQ5c3AfAIuyYFBDihOz0U+naZD3lCQJBzeHHjkv1FXpyM/IRqlXoga8ti+BE692e78y/zCSAqb+ZuXc0jTSD+DYf38k5Ng60tOhvJvMFt79vPG6Yg7Xzn6ai30WdE8n9fTk3Ckb/boYo9FoXeA5HDyHWyL9SjxLqVY49/zAgOqiakrTSykuNFCnbBTpF34pjHwFNF7/yLhkegY3BzcG+Q2iX4h4F3Nzoaa0huT1yZSmlfboWFRDB/LeHQc4f+Jx5hd+L6RXZHo3alexaa2XEXbWOPPv8f/m8mGLwSRx6lSJ7fzZCBd/F8YsG4NnlGenh2LUG6nWVlOs0HPF39+iGfNMp9uUOcvI/VtISSR9BIj3MdpLeNiWOx62eOybqS2rJenPromAMtQZ+HDsh5x+VmykD5eFoAtaAmrZo1GmHo9BELJAzJuNCA4GE0YyMhoiCTRuGmatnsX899ruhWvUG3na4Wl+u+43csrFYXiQhzec/gaK9nXZY8j0AUqOwumvbYwZK06u4J49F5IY8D9Kmywx9767l4/P+Zja8to2d3NYKC9bSXuGuoeikJpsZU0mWDsODj7W7keR6QXk74CMFZY/A93FwVE19o1+31/8Pe8MewddrX3DX87BHHIP5dqN/jp1xymS/pVkUQpoMdIv8T3Ys6wdDyLTZfT/F4x9DxRqS5H5cDurPMtSpvXonNHF2RlmzhT//u23tt2jddcSMj4EpUY4PiYWJbI5dTP7suu/w7oKi8x3a5ij/GJi7MjLypw59L8TRr0GCiULB89h7Kk19Et8nuz4HEpOl/T4cHLic3jO9Tkyv9/G+mFB3FvkxKnCUyLyZvj/wCmo2XuNRiOHDx9ucf8uc4aiUINjoNW8eNf4u/jr6r+4IPRaAIrrfVGLEotIXt99UU9mElYnoHjuY0YcPI9+ThEoSuKhXFYf6YtYjH6NIv1mvTSTqjFTgIY0LF1NgWsBr8bdw9cOJpJU3ZdP0mg0crS7kxM2Qjb6dTfjPoSx7+LjWO9BbSggPb4AU1t1H7qQPe/s4bWw16hNy2uQ93TyhoDzIO5uIUgv07uIfwR2XmdVZE4un5sLmXuy+GLmFz2e16+kBOo0pTxfamRzxP2gdOzR/mX+AUwmEclUYy2xZH4fKyuVVnl/mqO2vBZ9jb71C1vAd6AvX93+DQdGHsDbyUue+voizqHC4cUl0lI0xE94Upc5HqaoCKv38celP/LN+d9Qmd/5qFCT0cS5z59L0RhxSP5LziRUU78Tnm4yMo1pFAW/PX07036NYOuA8VRUQFmjFECOAY7tOqzU1+iJWxiH1wA/SurEnBzj7Q7bL4eTb3TZ8GX6AKnfwPYroMraUyKjLIMNGSspcd5FWRnoG322y7PKKUosojKv5fl08zObWXX7KqorjSQkiLLBgyGtVOz27Up76iugrkSOnO6rHH4ctjdEb40IGsLM+FzmnTpt9/Lg8cH4TfRrVtIxZnYM9+fdb1eOVpIkojyjUCvFoWmLRr+x78P0taCWk0ifCQS7BQNYZILNlKaVsvrfq0nd0nJ0U3NceKE4Ttm3DxvnsaYYDUYKThZg0DUY9czjMadA4MB98J0WapqPVDUjS3uefYyODSOgYjZOBaG8P+I9dr62s8fHoHHRMOzqYdR6+qJTFlNnqsbNwU2kQBh4P6jdWm9E5uykIhlKDlv+HOo/lOmR04nyFfOP2WHrr8f+4ouZX6Cv7dz5S2t4RnpSNWQskSWf8tWsFLikCMa81a19ypwhHH8Z4h+y/GmW98yrzLOUDblsCF7jRBh7Tk73DCO3IpcUw3beqCim0PPm7unkH0A+YeohzJrdw3bVsXXp29SV93z+vOCxwYy4bRw1GjfCCm7ioYmPMNhvcI+PQ6YHKdwNOestf760/SWe2X8ntZosamvBKdKfuW/NJXJGZAuNdD3FCQX4ZitJqFZS6TtdNjj3FX7yg103WP48ln+Mtam/oAkQskmnT7d8e+KaRF70fZHjP7cvd1VTTCYTpXVFDNPAooi1UJPX+k0yvQuXKJH3p17+GMRmx1XjioOLOIhufGAz/p7xzFs+D41z592nVVoVkx6chMekKHxLZxOkGC5PgTK2/DkFfh9g+dPdwZ20slSqHU9hwmQl8VlbVMvpTafbvCHXuGhY8uMSgi+ZQJ1KHCYGevnD5J8gVo5GkWkH4Uth0o/gaB0N4O3oDYBOLd6v4kY2uGlPTOO+3Pvwim5ZYSRpbRIJqxJISFKg14OPj3AUSi0RB/LhHnaMfmpXuOAkjHm7Ew8lc9Yy8EEY/7HlTw9XDQ56P2qrVXbzrE24dwIjHh+Bo2fzzodqJzUqB1WL3dbUQH69T5tdo5/WDwLObcsTyHQ1JYdh2xWQ85elyGxUK64pplrXkLNMX6Nn9xu7Oflbx5xhAwJg/Hjx79ai/YqTi3k77m02PLzBUpZZLj7swa7CKInncAi5EBy8W+3bHDQgG/3OcEqPwZaLIWMFKpX4puk1Tgy4fTr9zu95XVafOB8WfraQ/ABvkMQk6eUoq3/1Cf6+ALYutin2rBdVKikRv4dfN5zz3z8fujlmJXBkIBlD5lLtHoC7LH7Tt8hcKSSF65kfO5+35r7FVcOusrossF7RPyuLbuHUR6dYtnwZfvnBDflNewEtr2BlOk/eVkj7gYlegxjlcClFASF49v9nVmPRM6NR9YtGfwcMrLuG5+olKFg7Adz6w4RP/5FxyXQj01dbhe0v37OclJIU5vleBplBlOmdGXv72B4fVsWfO7j0wGSCvniWc8JsvWdleiGSBP3uALeGDc3Tm5/mmyPfMDv0ZUi4geRkGDq0+SYCRwYSdk4Yjt6diwxN3ZXKZfE3M2DgVq4ZeD9UnSsOZGT6NA9NeojHpz7Of/4jEY8w+sXVT0/RM6O7vL/p7jeyO+FGHpm5DI7lC49aGRkzflOgpsGVMcpT5PerU5SiUxWSmeljOdxL/iaZNR+uYdnhZfgNbvtclpGBxejn5RwAoRd13fhl+gYeQ+zmm/J2EofUBo1QFikqAl/huNvmqNRr/76W6qJqflkj/h40SCwl7hh7B3Ni5uCkdur8+GV6F00Ma871GTVMJqiqavi7KSaTibqKOqt8boe+OkRFTgWjbx1t4/Bz9S9XU62v5v+m/B9D/Yda8vm5u4OrtSqziKZQOgopNZmeR1cOqV+D91gImAEIJxontRNVuiqyyrMs8u7e/by59dCt7fqONmXBAtixA/76C666imYPr1VaFRMfmEjUeQ25ey1Gv/pIRGJvFT+tUF7e4Kg2YEDL18r8w5gMkPEreI/HYDSQH/w5CaY8tHPvIuo8h39sWPkVheAFzipXNEoNbF0C+mqYtvIfG5NMN9P/31YpdvIr8/n5+M8UlxqBZRajX8zsmB4ZjtHYkKvNQ5MB6XvAe1yLErMyvYRJP4BSa/lzXMg4xoWMs7pkz/I9FD+2Ge3Qq8nJ8e2WYVTVVqE1KvkiOolww6fAtd3ST08jR/p1MQpFk/9JS4/AqTeY7RfJfRHfoHK5A+ncGTi4/TMfdbOnrWfjtFiSQpYV6600MvgB+LsILUUHL5GgNze3x0cEQHnEENIHzuIaaS5uOy77ZwYh0/OMetUqisTsaavyyMLV1Y3k5JbnIWc/Z67ecHWnF5+ZmzOJ+cWftJ0fs7J8FbjGdqo9mbOU+Ifg0OOWP7UqLZIkWbz07UkzVRVUdToHavKGZD4c/yGpm1IAGO31GeRu7FSbMr2QYU8Lifh6HNWOhLiFAFDpkEhqvfqYQqHgnOvOYdYrs3DyaZsRJGNXBn/c+QfJO/NwrulHsHK4pW0Zma7AEulXn06gsLChzlBn4PjPx0nd3LKEniRJOHk7ceSI+HtwvTiJu9adUUGjGOBr53Q7dyMkfQL6zksxy5z9qNWQFvwy+6KWsvHUbpt6hUJBbFgs7w9/n9V3rraq2/fePrY8vQWlWmlVbjQZWXFyBT8e+9GSrsNs9LMb5XfwMfglWJac/afwHgNLKiHu35YiSZIse5DGef0A/If4I3VCfmHgQIiNBZ0OVq9u/jr3UHdmvjDTyqkso0y8SJZIvzZyvF4AJTSUXhWd0CtxGwhLa2Dg/SgkBSu5hRMhD3EiPb/1e7uBfR/sY8WNKyguFqo3nuaoUkMNGGtauFPMn0OGDLE9/5Q5O4i52cqpIKcih1tX3crL+/4DiAj2mpZfgS5l13vxBMW/yt7w0fy652bYchEUbO+5Acj8c2h9bfLYN8XB3QHnYA8kk5Hs7G4ax0Ww8d63iAgoxsmhqps6EXPnoEGDuq19m/56rKe+SvilsDAd/KdbDG3F/9Caf+09a9nxwC8YpTqqvHaRWJQoNiuztllJocj0IqqzIfN3qBbRAubE5Up3sbDLzYXfrvuNN2Pf7NFhFbpGkBs1nlqf2eA9vkf7ljlzMG+4dY5ZGI1GktqRq7kzeVGHXT0Mz3uvJ1s9lDKXeXKOlb5KxgpI+8GmOKTe9mE+xDNTkVPBK8GvsOGRDTb3tIfaslrKMsooKhaSUl/UlMCkbzvVpkzfIMZLODxUOiTSOP+37whfxt81HpeAts1lOQdy2PPWHnISKxiW+gFvDDzARKkQfvSG9F+7YeQyvZaCXfCTD5yyltM0pzWoURRiwkRRkfVt31/yPbvftDXCmMncnUnimkSqy/WcrFfaG2IbUGhL8qew63oRTSHT9zj5JvzkCyXCUixJUOz1J9le37M37YjdWxSOCjwiPHAOsA4DvGrdVVz919UoNdZGv6SiJEprS9GqtAz0FeHWZiehEHu+E6EXwcCHQONpp1Km21GoQWXrEGM2rJmj6xqTuTuTLc9t6VB3kiSi/QDWrm3fvZZ8pR7hYu++ZTFktKITipzP76xCobQ4ZUuShLtafCsL313HNxd80+PDSd2UyoGPDlCsFweUXk710p5TV8CMP1u9v66u51MWyXQP5nPCwupCVBqRLqC0FHIP5fL2gLfZ9/6+bu0/62A+AellVDqcYnetBBO+EJF+Mr2f2kIoOgB6YWir1deyJXULK082RBoPvWIoC1feQLWbf7cZ/YqqSoivhYn7LkMZd1v3dFJPT86dstGvizEajdYFGg9wCgGFCndPA8rKU9R9+AVHfzhq9/7uJPdgLiVHM6lWZ/CpajxD3xnaKU82mbOAnL+EXne9l4yfU71ciVOD0c/R2xG3UDeMemNzrXQpJhMUllVxNPQenjcNxjj40R7pV+YM4MRrsOl8zIlVzEa/CimTysoK0tOhte+fyWjiu4u+48clP3Z4GA6+DpR6BqLUGK2jnmX6FjO3wrzDVkWPbHiE2471J9vjFxu9eJcAF0bfNpq4hZ2TJB6waADLUpbxL7dBrB3uiYN7Lahl12yZJpQeh103Qe4mS1GMZ73RT5tIUpKQwTEajZw8edJ2/dkCI64fwT2Z95CtFmEpwcGAyhk8hrYpd5CMjAWNJ7gPBgcfq2KLvCc69IpyK6OfUqPkku8uYdIjk5ptdtfru/hq7lecOKxDpwMPDwgKEg4/96y9h1d2vEKVzo4X7uD/wNTf5Tm1r+LgDe4DrRRsfOrnuZTCDJvLzfPnkl+XcN5z55HwRwLfXfQdBp0BlVZF4AhbSc69WXsBGB4wHLVSHN63GOkXdgkMf7aTDybTKYrjIX+bVdHMqJncMOIGIj1s89rvfHUnfz3yFxU5FR3qbmx95oyCAqhsJuj4p8t+YsWNKyx/G4wGVAoVCklBmHsYVGdB+o8iB1wryEa/s4zieMgQh9k+TkKmrqasguqi6hZu6h4WfbmI21IfoFojjH5+Lm1fA3Zk/SlzBnH6G1gzGkrEubSPkw8SEiZMaL2E9H9JicgDjoT46Ub63T6Tb29UUO5ajk4bBJFXgrO9j6pMryPxfVgzEspE2HpRdRFTPp3Cgm8XYDA2OPEFBIjfFRUNUrBdScm6ckbuG4na4GEr1d6FGI1GEhISuq+DJsg5/bobkxEqU8mpyGPi6gn49Qtg2JY7Kc/seTm5qzdczUcfmdCtE5sVbydvqMmDlC/Bfxp4jezxMcl0M77nwLiPwWsU0ODBo3NoMPrd8dKsHh1SRQXEbv0cR/8MXgt5n2fmPNKj/cv8g5Qchty/QF8BaleL0S+/JptYFwNGI5w+Df1ayGMuKST01XokpYTJZOqQ48L3e77nyZprSFzggHfVEGBnx55H5uzGwTZRfW5FLqmVp9A4xZObuwijERqr1sx5dU6XdJ1dIVzUnFTVxLkdgpp+QtpCRsaMrkwkNXeNFWs0GiL98EzClAWHD8O4eifYz2d8jmuAK5d8d0mrTSs1SlwCXcmsl/gOCUHkOqrPdyQj02bc+sF5m2yKndROaFVaDAbQq0ooKrI2wg1a3LKszcT7JxI1M4qT6SKH7+DBIoImv7KAV3e+CsDtY263vdE1WvzI9E0iLhc/jfB3CoEaSC+zo9ldj3ktmb0/m4Q/Esg5kEPwWPsSi2aj3+jA0ZYyc6SfjdHPZBIvrsw/y/YrwKiHC05aih6d0rzT6aRHJnHOg+fg7N9MEshWcHQUjgolJZCdDTF2shIUJxdb5ZBUKpSk3pWKzqBDpVCBsx8srW01armuDsxnh7LR7ywh/hGxH15aTaCbL6dKYcsUL158/8oeH4okSdQpHdHo/AksP5+JoSOFtGfi++A1Gnwn9viYZHoIYy3U5Iv9BmIO8nHyIb8qH5V7LuQEUFIC/cd5cvsxO+utLqa0FHSqMpCEjLtMH8JvGgx7BrQiFZVZLcSEiaLqInydfamrrGPf23sIqfQhw7k/OTl2cih3Es1KNXMTZ2F6KAFJVwqa3vEeypF+3Y2hFlZE4ZP4KiZM5AZks3v+bYz91z8jaVhaKlGrFprhvk6+UJ4IB+618iSX6UW4RED0deAcDjTk9KtRNhj9eprSUjAo9bgoTbznp7DxvJTpxYx9H5ZWWTS7zdI6WeVZhITWArRJ4vOy3y/j8t8v73Ckcuq9qTz0/ENsLvWn1lM+5O6z1BRA3maoa8jRN8Rf6MdVOB1Gr4f8ZlJslJwuQVet61C3KX+lcOCLAygMCsYpPRlVOAlSZXlPmSZ4joCLC2DA/ZaiIf5DmBg6kTgf4Rlx8GDD5Uq1EoWqbcv65A3JpB2vpJAE1g734oLfR3Xp0GVkAIoeKGLNpGoc68Js5D0BjAYjJqN9qe6A4QEMv3a4TT6/1FKRBzDQJRAHVZP86Ea91XwuIwMQ6i40N3OqbCP9mjL2zrHclXpXswY/gL3Z9Ua/IGH00+uxKAPYyHvuWQZrxso5Jv9pBj0KQ55o8+X+Q/wJGB7QKUWkwPog0eZkyG7cdSNXrb/KplytVDf0q9SAyrHFfnbuFO+gpyf4+3d4uDI9SdzdMOFzMBkIdBMOf7Wq/O7LU9UCKRtTyDxSjF/ZbM4vW8mT058UQQH7/g1pHVfVkTkLiLoWFqaC7wRLUaBr/cTlKl7GkpKeG07m7gx8ckS065LqrfBrqMUgKdPL8Z0Agx4RComI76CnVshx5VeJwxhJklj/4Hp8ckVkanfMl/OeuJP+i408E/sT1NnZuJylyEa/7kblCAMfRhV6Ec5qZ5CgVlVAaQ/vSU1GE8d+PEbxiVzqVPVGP2df8BgiJM7CFvfsgGT+EcyRfhUmYfTLz4eStDL+fupvUrek9sgYSkpg53kRFF31E1c6VUB5OxK5yZzdKKzzopgXlpW6SvxDxYe1LUY/hbJzny7dUB0Hhx3k/5Jmoh8kSy71WdK+g/VToaghR8EQP2H0q3IRsp9NJT4BElYn8EbMGxz8/KBtZRvY/dZujtx9BJNkIqsyhNKol8FvcofakunFKDVCqq7RoeO82Hlsu34bD5/zf0CD0U+pVHLF2iu46KuLWm22qqCKL877gvUPrKNOlY9OVUxpbQmk/QSHngB9z8tLyZzlHH0WEj+wKXZUO+JVH1Dd1Oi37cVtPKN9hoITBaT8lcIPS34gfYcIl9JV6TDqjej1cFwo/TQY/UrEWjXcI9x2HKXH4EcPMR6ZvkllOhz+L+RvtxRFeYvwu4I6+0Y/pbJhbap11+Li33Ju1FOFpwAY7CdeypwcMBhAqwUfnyYXK+ojuVQdixiT6SIiLoeIy2yKq3XVHM8/bvcWg85A6pZUjIaOSRcGCTETu+tIMy0aFcsTIX+HiLpqhsJCeOcd8e/Zs+Wg0rOGwJkQvgQUKnEeBziVFrBr+T6qi3tuDWaoM/D5jM/Z/6LIVe5mDsh38IVzN0HsLa220Xj+lDn7CXMPA0DnJPKLmo1+R384yq43dnVr36c/WM/0NcLQo3fwAcdgULUtV7lM78M8N+ZXCtuF2knNjbtvxHOpUKnrFicJPz+26m/g89O/gmNQN3TQQE/OnbLRr4ux+x9v+LMQvsQSpupSeJJDP5y0va4bqS2r5YfFP6Dbuos6lTD4+Dn7iYgb33NkveTezKrBsHUpALOjZ3P0tqP8etlPqFTCMzAnpZpN/9lE8vrkHhlOaSnUqfL5vRIuNc2AsIt7pF+ZM4CaPMhaDVViB+ykduLteW/zw+IfmDY5CkmS2mT0A0hen8w3F3xDTUnzm+HmyJ9fzO8X/I5a74WHR7tvl+kt+E2Fka9ZScEN9R8KQJkqCb2i0u6CMnxyOHEL4vAb7Nehbs97/jwc/s8Zk8JEYVUEmmH3gOfwDrUl08spT7IbDT9kiDjcy8yEkhIlQ4YMafPmQaFWMG/5PLTjh1FXr/zg4+QD6T/DkSetcmHJyLSJk69Dyud2q5oz+nnFeBE7T6Q6yD2Uy7Efj+E7QGzw97yzh6fUT7F3ZTa1tUK+J0ycQ1ki/cwHU1YoHSDyavCUI1f7LDU5cPhxEcVfT6y/8BwvNdka/ZTK9s2fRpMRHycf3BzcCHUXe2eztGdIiB2jy+g3YM7u9j+HTLdTVluG87PODFw+kIo629x9W57ZwqdTPiVzd2aH2jcb/eytIzN2ZhD/WbzVHubdve8y5ZMpvLf3PVGQ8C78ORGq7Z9smkzw6qsibUZMDCxd2qFhyvzD+Nbn9AtIr+b4/36nOLm4x/o2GU3MWz4Pj6nDMKLH3axkp3IE/6ng1r/F+9s7f8qcYeirIeULq+9lmJtYW1WphdHPHKiy//39/PXYX906HIfpE9k6NQWAhIDFMHunvCfpK5QnwZ+TrBwIzXOjOdIPIHhMMMH9hSG4O4x+xQV6sspiyZQWiD1FN6FUKhk0qOU0B12JnNOvizGZ7MvUgDhUSS1NJTb+CDtPJTL15pY/pF2J0kHJxd9czKsfu1Or3grU/x/JUAOSEhTqHhuLTA/jGAxacTjt6eiJp6PwoPH1FZNlras3N++/Gc8ozx4ZTmFGNREnsihQB+MwLET2fu1L5G+FLRfDxK8sOVduG3MbJpOJxMQKwMTp0xJ6Paha+ToVJhRy6vdTpG9PtxwctpWc0iIGa+DJfhvQlu8A7YTWb5LpfXgMFj+N8HX2xd/Zn9zKXModj5KVNdbmNo2LhiU/Lelwt979vEkfVgCHwNkUhGPLyk0yfZndN0PRXlhsLQ+hcawjIsZESoID8fEmxowpR5er49iPx+h/Qf8WDdJady1jlo1h37tQl7QRqF8PjnoNBj3crZscmV7KeVtA7WZT/M6ed1h5YjW5nlcTVHwJdXWgqQ98GrBoAAMWDQDANdiVQUsHofXQAuAR7oHfED/SqoSzpDmfHzSK9HO3E+nn1h8mfNbFDydzVuE+COYetEhEAQyuT7Sno5LKuiqcNU6WOpPJRHl5Oa6urm2SclRICg4vO2xV1mw+P5kzh9Tv4eDDMOELS44yNwc3S/6qhMIERgSOsLolblEcSOAa1LGkQWZ5T3uRfoe/PszuN3cTmRppmfcO5hxkS9oWpkVMExeFLATHANAG2G3/119FtL+DA9x3X+v7JpkziKID8Pd8GPgQlw6+lOqkscR7+ON0iyteMbb5xrsLlVbFmGVjSP4OdpfM5E/jVgYe+5KL4xYIY0srBpf2zp8yZxgmPey4GiKvAb8pACwbs4zFgxaTui+OH4Hiehv07Fdni1tMpm77b10d2o8UlREvdZAlWEamjyApRXR7XYPTQ9NIPzNl36wi9KiS7AGzga59F/dc9l/8AsqpmXUr0H22GpPJRFlZz0nXyqbzLsZotCMBceJ1WDuOUCcPAA6PdkW9YH6PjkvtqCbuksHkOYRa5D39nP3g2P/gWw0UH+rR8cj0IDPWwug3bYrNuv8FxSoCRwSiddf2yHAKk0sZsUvPqJRoBmsUIgeLTN/AazSM/QC8x1kVG41GKiqScHIyoddDWlrrTY24fgSPVj/aboMfgPdbbkzaNY6LAvZAVXq775fp3Zjz+pU7Hm5RlgmE8bklZ5+m6Kp06Kp0pBcL97TnonbB2nHyPChjn9hlMPwFMDWsLRf/sBjHZxwxxPwGQHy8ieTkZAoTC/nrkb9I3dyyVLc5h1pGBtSpCoD6SD+tr40RXEamTbj1E4fTTTiUe4jVySstcsnFzQQwaN21uAY2HKwPvGQgyw4t41iCcEgc3Oi1TCsTCwS7Rj8ZGZUTeA4Fh4aD84ggV86Lz2be/hrQOVldbjQaSU5Otr9/byMZ9QGENvn8Et+Hw0+BobbDbct0ESonULmC0ToXcz9vkR/XLNnamIBhAUx7fBoe4R4d6rKlnH4T7pnAkp+W4BbS4CxhntssUcx+k2DAfXZz+qWkwOf1wdU33gjBzaeglDkTcfABpzBQuxPrHcv8/rNRqoeT4xzdY+cxjSkthTplITpqcFI7icj9b1SQtbbF+7pi/pT5B1G5wJRfYeADlqLBfoOZFjGNSF+xpjNH+vkN9sNvsF+3GnfLymBkytf8OjmT+fqjkCw7cfUZXCLgohyrd9FepJ/RYKTqRBqOZbnk5Hb9u3gyLoHzx+zlRi9bOfCuxGg0cvr06W7tozGy0a8nqC2A6izCHMWGNiO8iuO66FZu6nrMxuTA0oU8POkRJoVNAveBELYUtHLm577Cs1ue5Y4/7sDVT2gt5eRAXWUdRYk9k6y02tGLFRdnMmfaIe4v+RT05T3Sr8wZgHMYxNxoJaeYWJTILyd+4XjpMaKiRFlbJD5VDipU2va7tRoNRoJ2+lKYEMZVx76A4PPb3YZML8FkgjVjYddNVsVjgsYQ5z4ChdGhRaPfrjd28Va/t0jb2gYrdT3HfjzGc67PEXVsEL6lc3BXe4CxDhSyi7aMHcIugdhbrbytndXOGE1GFD6JABw6JF7lkPEh3LDzBoZeORSDzsCuN3Zh0Blsmnyz35v8cvUvtka/yjSoK+mRx5LpZejKhYduEwcIbydvABTOhYCtxGdLGAxw7Jj4d2OjX4s5/Q48APvuaXsnMr2TumKoKbD86eAAvo4BSCjIz2/hvg5iNvrZRPql/QDHX5DVdM4Egs+HefFCsrARsd7CcdCe0c+M0WC0+y1tDbPRr6QEqpukafOI8GDARQOQFA2Hlumlwgkx1K31kNFffxUpOsaNE7n8ZM4ynEOFdGHUNUCD0TYzw4RB33MGtMPfHGb5oOUUHcpApxLfaW8nbxEpHXqRnP6ntyNJELJAnAc3wZz+xJzTz2QyUV1cTV1lXbcNR/necsIPrhAys0efgxTZ6NeXuXTwpbw9723O79dwVqdQKrhu960kjVpCURGUF+vIPtA1Op8mk4mfFv1I4YytlKpHd0mbZwqy0a8nGPYULEwnLmwWi+MuxbUulpwcyM1te3RAZzm54iQfxL2Me+4pBkgLePbcZ4TRL+wSmPQtOMpGv15L7iY4+CjUCVed13e9ztt73kbpKXapubnw49IfWT5ouSUCoDsprdbgo3+O0NinKI+9C9Turd4j03v5+MDHLP5xMSvTVhJdbwtMbkN6SaPeSObuTPKPt+8ER6FUEP9mOVtnVODpPEh4/8r0TSTJTgIeePbcZ9l8xX5Ciq4kN1ccPtsjenY0w68b3i7js0ugC3GL4hgffRPjElZzuHYNzD3Q0SeQ6YNEe4qJskyViEoFhYUS+flqHNwcCBkXgoObA/ve28eaf69hy7NbOPjFQQx14iXWVevwjvXGwduZwsImRr81o+HvC/+x55I5i9l7B6yMBUOVVbFZnsmoFYeJhYVtbzI5WRyUOztDRERD+bqr1rH3pr1iD9OUrD8gd0N7Ry/T2/g5AHZeZ1XkKxzGKSiwc307eGfPOwx7dxgvbnsREHbuZuU9p6yAOfvknERnMP28RKRfQlGC3frTm07zkv9LHP3+aLvbdnbGkiOtcbSfQWegptQ2H3laaZNIv7UTbN5jy7hOi98zZ9pdxsqcRVTWVbIu/2MK1S/R/8dn2PCkbR7n7sKoN2I0GKmoUVFnNvo5ekPgLJj8o11jkEwvpJHaTK2+lg/2fcCHyf+HCaPF6Hfi1xP8z+t/nPj1RLcMwWQyoVdoMCrVYt6cvRvGvt8tfcmcoWT8Jn7qmRE5g9vG3MbIwJFWl7m5STh6iQj4z6Z9yldzvuqQY05TqnRVGNFzUx5k+L7S6fbOJORVaA+ybMwyvl/6DTNygxmx+jm2fWebULy70Lho0AZ4YFBpLJ4bMn2E3E1w9FmoEpsJP2eR60dyzRXVuUJKacJ9EywHg91JaaEeB50HHsF34DrmVXkz3Jcw6mBFLOy+1VIU5Coy3RfpioiKEkbntkT6GeoMfDjuQ7Y+u7Xdw7jI/SUWFb3IZF+VTWSCTB9j9i4Y94FNsY8PqNXCk7q5yACf/j4s+HgBwWParqsUPTOaJT8uoVwlcqh69VzqDpmzkeKDsGoIJH1sKYrxigEgpTSRASIlGikpQiJMX6un5HQJo24ZxZzX5yApJH69+leS1wtPCrWjmitWX0G/22cB4CGFMCJgBJEekRBzE4Qv7cGHk+k1BM2HgQ9aydBC/eEhYNCIw8Tm5D3tceSI+D1wICgaLRN9nHwYFTQKD62H7U3zDsPM9q8JZHoZsbdB0Fyrohzv79kXtYSvj31qc7lW23Y5vRMFJziUe4jC6gZDdk0NKJUQ0FThVuUo8kzK/POYTHDyTZHbrxEtyXuCyMHsEe6ByqFjagz28vrlHc7jBY8X2P7SdktZWW0ZpbXCOTfUvd56LCmwl6/IZGohulTm7OH0t3DwUWoNtdzyxw0cjnqSUt9IDK4ePTaEYVcN444Td1Dk6IlBWQmAl2P7NibtmT9lzkC2Xgrfu1jOQ5QKJbeuupU3Dz5NrTqX8nKxF/aJ82HEDSM6LHfcGkajxOEJ1/Ll0vtZuvYcyh0CwDWmW/qSOUPZfx8c+r82XWr+tgZeOIbJj07G2AUR0pnJmVyw4gJiT/XD39O50+21Rk/OnbKeVBejVCptC3VlkLlKTFzeYwgf5Mqp/SEkpdi5tpuInBHJwJdvYP0b4Oy5i6QiHyI8IlAefQb0lTDihR4bi0wPE3MzhF8KLkI70d/ZnyMcweiYBwij3/AXhvfYcHTb9zBqxzr0l10Pk+TdSp9CoRZ5VhpFd5qNfuWU06+fmBOTk8FotD7oa4raSc3Ml2biP6R9Uco1pTXk78vjytgHGe20CaSKdj+GTO9HksArsJx9lb+QmXkVAQEtu1K3J7G50WQkp7AGD20ZY10+h+LZ4DmsK4Yt09tQuYChBkwNXrhmo19ScRJ3DIfDhyWKikJRKuHzWZ+TsTODhyseZty/xlGSWoJrkCsh462TTWVmit/znZ7i+Vueqi+VDX4yHSR8ifhpglnes1bZfnnPo/WBNUOGtGMckgRq19avk+ndjHrVpqjS6TjZph84UOABXGspVyqVxMXFtbnpjHJhbQlxE3OqOcovKAhUjU9VqnOhOlNEyijlQ/F/HEmCQ4+J3OKN5iqz0e9k4Um76zjXIFdu3ndzh7sNDIQTJ6wj/dROaoZeNRT/oQ37F7O0p5ejFy4aF1E4y37EV14e1NWJ981fFmo6e8n4BdK+x2PQYyglJRWuFRydOJ25w4J6fCh5FYXgAwpJgbvWHU69DWUnYNQbLYaStnf+lDkD8RolfhvrQOmASqEi2DWY9LJ0ah3S0OoCKSsD3wG+XPhh96mBlJeDXllKueNR4nNAW1cIaq0sj92XGPsOKBrWSxV1FezP3k+tvpaZ0TOtLg0MhIQEUIwczrhFXdN9Xmoeo/aPYkpoKQHSRmBG1zRsB6VSSb9+/bqt/abIITZdjN1EtnXFsP1yOP0NBqOBwZf7kTD+Ko4UBfVokElxMRgUVXwkjSfmzRiqdFWQ/qNYdMj0XpyCwD0OlBqgIdKvVi0i/YqKQKdr9u4up0zlRUq0Dv/imdTuv7/nOpY5M5i9y8rJwGz0yyjNICDAiIMD1NY2HEq3xMR7JxI9q335UbP2ZVH++ids+OsCkp2fbte9Mr2Qgt1w7AWbXGZ6o54f/AYQH3kNa09tbP72kwW8N+I9dr+5u9WuStNK+XLOl2z9Ziv/qXWmcOQABtY9CPk9J+Ujc5bhGg0XJgjnnXrMRr+s8ixiB1ViMpnYvLmOH380EjAykKiZURaPR49wD0beMNIig7L5mc1sf2m7JUogxNoWKCPTpZjlPasQ4dJtlfc0mRqMfo3z+e1I38G/V/+b749+b3tTZTpkrYXanslPLXN2EeouJrucKmuVHaPRSGFhof39ux0yysT95rxrp+oDxGzm0oyfYc0oobYic2YwfR2M+9CqKMYrhmuGXcMDEx/AYGpe7cagM7Dh0Q3kH2tfSoGgevtNY6OfT5wPiz5fZLV/KastI9g1mAiPiFbbNBuaQ0JEhKnMWcqI/8HCdBRKreVbWafKb9P+t6s4+PlB4j87SGGV+G56OnihkBSQuRKSP2lVO7a986fMGcjA+0WqJ6WDpcgsMSx5CJUws8Rnd5KTUkVA0nZC0kOY7OyAemWkMD7L9B0CzgO/Bun+hMIEpn46lat/vdr20nplBfO3tbasltqy2s71Hyex6YnnuPP8XbgZD3eurVYwGo0UtccTspPIRr8uxmTPiucYBJN+YK/TKNRPqblhx3jUamFsaSz30J1sfnozWb/vp1YlFqsOSgfhSTZnn9BMlum9mExQk29Jam82suTXpqPViurE3UV8Ne8rDn5+sFuHotdDpmcoP179DCrXk0j1eQZl+i7m9zG7IhsTBqJEQGqbJD47Qpl3GWtmr+H9/m9TFnRX93Qic/aQux7iH4IK6xdOpVAx2mUhAN+lvtbs7W7BblTmV6Kv1Td7jZmipCJS/04lL0tEWe+pdCOl3w4IWdjR0cv0QTwdPfHUCnlYpXcyM2eaqKqq5tNPJfY4T2PSkzNRqq1PAquLqynLLGPv8r0c/e6oxegXbFamLUuAjfMgXXYCk+kAhXvF+5O11qrY/H2vM1VhwtTmSL/Tp6GiArRaLGsCgK1pW3lj9xv8csLOe5q9GjbNgUJ5T9PnOfqskCxrRKS3sMoV6qyNfiaTifT0dPv7dzuYI7JC3ELIyYEffxTlo0Y1udB7HAx+HDxHtH/8Mt2DzzhwibQqclQ78unCT3l48sOoFM0LYKX8lcLWZ7dy/OfjlrLK/MpWu7Rn9LPHhNAJZNyTwZ6b9oiCitNw4lUoPW5zbWOjn8xZjHM4OIWAJOHrLJKOhp7YR/rXW3psCFuf38rmZ7eCQY1/yQWcF3WeqJjyK1zQ+ka8vfOnzNmB2ehncLU2+m14ZAMrblzRLX3mJ5YRdWQ/UclR1CrdoN+d4D2mW/qSOYMxmSxSs+Z5saCqwGaOMct7ZmdD5p5MXgp4iX3v7+tU18UVNWwzKBmc3B+HuGs71VZrmEwmMjIyWr+wi5CNfj2BQg1hl6D1HoYJEyVlxQzO/QuvzMMcOtT93ZtMJra9sI3yHUeoU4nDRl9nXyFhoVCDxqP7ByHzz1GVDj/7iU0wDVECycVJFlmQwhIlKX+lUJJa0q1DKSuDWlU+lSYYkuGAetx73dqfzBlI+q9w9DnLnwEuAWiUGvQmPccLjhNd7/jaFqPf2nvX8vbAt9u12ajzqWPnhJ0U+Jbi6dnOscv0PiKugFk7wW2ATdVlUf8C4FDt7yQUJti9XeOi4e70uznn/nNa7SpyeiQPlz+M8Vzxvhpqg3EOGy+isWVkmiNzFRx70apoUdwirhx6JRqVmttvN7FkSQEqFezcp+bpd7ypbHQWWZZZxkt+L/H3k39zZ8KdXPL9JWRng1Gq45pjnsS+GUtleQrkbYKqHnQxl+k9GKogb6NYbzYiyDWIsofK2HJBARJSm41+jfP5NY5kSSoWC4NoTzsR/n7TYMy7spFFBgr3iEiVRmvDuEARmVdKenN3tYrOoCOnIgeAYNcQXntN5PMbPBhmzWpysddIGPoEOMr6i2cMJiPU5IGx/fnro2dFc93W6xhxg5hfDHUG3ox5k58u/6nF++zl9Iv/LJ4VN66gurja5nqFOc99cTzsvweKbA8xzUY/OZ/fWY7RAJWpUJOHr5M43PbNzEO/q3MH1+3hku8uYcILC3Gp7c+M3BV8u/gbUaHUynNXX6HkMOy5DQp2WorMRj+do5hszEa/rD1ZJK1L6hYjr97dmx2zhhM/PJ58tTeMfgN8W99by/Qi4h+CbzVQI9ZZ5nlRb9RTUlNidWljo5//UH/Cp4TjGdW5g72oouEs/DuZEYl7UTm5t37DWYRs9OtBfOqTzhfUFqDcsQWPnBM9YvQDuOPUHVTNOJ9atYj083P2E4uN/G1Q1XNWZpl/AK0/xNwCvhOBBqNfYlGixehXJrnxaPWjTP2/qd06lJISCEzYyZzVcxoMzzJ9i9Sv4eAjYBSRURqlhplRQqf75+M/W4x+yclta06pUaKvaT3KykxhVREBStgXk0dg2VvtGrpML8Q5XHh/q5xsqsZG98OvZD5g4s3dbzbbRHvmMYVKQVq5kFYONHrj6V7X7iHL9DGSP4X4B8HQ8K58tOAjvlj0BXE+cUgSnHNOOc8/b8TTU0gj72zYu+MW7MbIm0YSdV4Uaic1npGe5ORAnaqQcn0JycXJOAadB0uroN/tPf98Mmc/vpNhSRXE3GhVrJAUuDq44uUl/m6v0a+xtCe0YvRz6wext8gHlTIw6UdYWmklTTcoVIRF1SlKKK/tWC5noUhhQqPUsGODL0ePimjUu+5qVQVP5kwg/kH42R+qUq2Ka/W1HM8/zvF826g6M5IkEXZOGK6BImdobVktA5cMJGxSWItdmg8mi4qEgRgg9e9UDnx0AIWyhWM4/6kwcysEzLSpMhv9wlruWuZMp+QQ/BYBie9bIlq2nufA4al3WN6V7sZ/iD/qCCH54OHRqKJwr5DMlun91ORCwjvC0aAes9GvWmMd6bf016XclXpXt5zfVdSqKfQzUeZehrtD7zK4yLQR11gImmdx2HJQOeDm4AZAfpW1tLb525qXByhVXLnmSvpd0I/4z+I5/HXHpDn3vrKVQZvfpb/XSTD2YO6rHkA2+vUUWy/Ff41IdGtUGpn48+WkDr2AQ4fo9rx+kiThGuhKseRFXb28p6+TL9QVwp+T4Mgz3TsAmX8WpQOMfRfCLgFgfMh4Dt16iD037bEY/fLypB4xwJWWgk92BsOPDeIGNwXUtjHBi0zvYegzMO8wSA2fn/sm3MfyKcu5b8J9VpF+rc2Ns1+eza3xt6J2bHuS5xPvJrFk+S3U5Hui1bbdWCjTizHU2OT0A7GgjMz7NwCfxH9CcXWx3dv1NXo2Pr6R3W+3LCt34OMD5BzMIaVA6Dx9Grsf9e+yPpNMKwx9EubGg9R88h5XV1diY2Fqvd9OYqJ1/fzl8/GM9iT3cC7lZSYqK6FOJSS/vR29G6IL5JNrmY4gSS2+O2ajX2WlyNnbEiZTC0a/ImH0MzuvycjYRWE7V4YHuKEyCIPNiSzriGZXV9c2NVtRV8Eg30HEuA/gyy/EnHnDDVj2UhZq8uDXMDjxWruHLtON+E6G2GWg0FgVv7P3HQYuH8h/Nv2nzU05+Thx4QcXMuY2IT9XmV9pyaXbGBcXML9eOSJ4gQs/upAHix9E49owjst/upypn05lR/oOUaDxFFEuTZwYTCZZ3rPX4BIJ/e8C73GWiBZdcCEGSWXJadudGPVGastqKS42YcLQoH5jMsG68bDvzja109b5U+YMxXcSLMqB6AanLbPRr0yyNvppnDXddl5YWqjHaCoGE1yvLYYtl4C+qlv6kjlDib4Bpv5mpYBknhvzK62Nfp6eoNGI6SpPCBliMppY/+B6tjy7heT1yay8ZSVlGWVt7t5z8kD8p9bw38lDoLr7c7C5uLh0ex9mZKNfF6NsLqOy53CkoHn4aYXnQuB4A2pXLWVlkJpq/5auoiKnguLkYooLjVbynii1MOJlCL2oewcgc0bhonFhiP8QnDXOlo1qbi5k7c3i0JfdG3paUgKb5/rg+OjbPOGYJryLZPoWbrHgMdjK6DctchrLpi/DRetCaCioVOJwMLcbXo/S0kqoceDy5GkoB97V9R3InF3UlcJ3jkJGqQk+PhBUcx6uVUOoqKvgvX325YiVDkr2f7Cfw18271lWVVDFihtWsOu1XaSXiIXk3orBEHVtlzyGTC/GfSB4DrU5yNYZdGSXZ6NUKomOjkapVBIbK+oS7KjRbnh4A+8Nf4/sLOFNofEUGygfJx+RNyhrNejKu/VRZHoxWWshzzYX0Zu73mTJr+dT5Psb0Hq0X0aGkILXaLC8zyDe97RScQAV7WUn0m9FNGy/usPDl+lFVKZB1hrxfa9HpQJnQyiSUU1CfV5dwGr+bI2BvgM5ctsRLs6Pp64ORoyA2bPtXFhbCBov4XQpc+YQciGMWS7yqDWin3c/AE4VnupQs+nb03kz5k32vrvXbn1TiU9JktB6aK0Oz3dm7GRz6maMpnrDoa7CoojSmOJisT+SpEY5eWXOTjQeMOpVCJzJLaNuYe2Va1kYdANOpdnsWdX95yM58Tk87/48CZ9s43jIw/xP48gTm54AkwGGPQcRV7baRnvmT5kzFLOUa6OcphNDJ7Lxmo08MVAkrU1JEeX6Gj0JfySQsavrVeJyNxzhwq/SGJk0nkFqvZDoVmq7vB+ZswtzFHTTSD9Jspb4BFA5qFj6y1Ku33o9+cfy2f/+foqS2igxAqz0XMGmqRv4tfgicPDrkvE3h1KpJKpx0vJuRjb6dTFGo62XFwCDHoJJ3+NWb63Oy8tikJ94eQ8e7N4x7X13L29Ev4EpL69B3tPJD9RuMOAeCLSVjpDpZRx9Djadb1McECB+Z2fDjld28MtVv6Cr7r5w5tJSqNHk87G+hpfVU8E5svWbZHoXJiPUFonoqnqMRiM5OTkYjUZUKoisfy1ay+tXnFLMrjd2UXCioM3d555XzRv/fgOlp1tHRi/T21C7QcRV4DPepkqSIDBAIjr3PiQk0kvtS91IksR1m6/jui3XNduNxkXD5X9czujbRpNdLlanaysugxH/65rnkOnd6MqtDrD3Zu3F8RlHxn803mr+jKkPgEpJAX2T80LPKE9i58WSVyCW/o5eYt70cfKB01/Cpnk94tko00vZdikcfsKm+GDuQVYlrKLWUziVtWb0O1zvOzFggDDUmEktTcVgMuCociTQJdD6JqMeXKJlaU8ZQdr3sGkulFnLNV5Zu5V5+2uIVEy2lDWeP9uCwYAlCufGG5sJcHUfAPPiRVSZzBmP2eiXUJjQYHRrB/7D/PEf5o97uH1JuqD6oAXzwWTatjQKExqUbowmIxll4hA91L0+Ud+uG0VuoyYSYxn1Z+2BgaBuu8iJzBnOEP8hzIqexZSBgQzc8j5pX23r9j41rhqGXzscg28AOmUhempQK9TC+DPwfotCVEu0d/6UOUMpS4DSY5Y/vRy9mBYxjekjIgA4cUJ8+/Q1er6e/zV73t7T5UOo0npRGDyUh8f/yjlLk4RkvCSbKvoUZQmw7x7I22opai7SDxqMfuYoeoDQCaFoPbQMv3Y49+ffT/jk8DZ3f6RoD98p9vBVzRRQOXbsGdqI0WgktzuiG5pB/n9SF9NaYtMAF2FlOfjwQaTly5EM+m7P6xc2KYyht4yn1smLwKqZPDzpEWZGy4a+PkXpcZG/US+Shv98/GduWXkLiYpVgDggHHz9WC5beRmSovvkvUpLwb2gkJICdzJcR3T7hCpzBpLyBfzkDdl/WopMJhMn0k7wn43/YcG3CzA7vrRm9Ms/ls+af68hfXvb8w7klhcyWQtX+aWBru0h/zK9FEmCiZ9DzM12qwMDIajoUt4ddIq357/dbDNeMV4oVM0vqVRaFbFzYwkeE0yEagK+pXOJdOvf6eHL9AGKD8IPbnDyDUtRqFsoBpOB9NJ0anQ15OTkYDKZCAwEZ2eoq4O0NOtm5r01j0t/u9SyOVK7NzL6hS2GcR+Coxw6INNBxr4Lgx61KQ5yFSfeRidhUG7N6NectGdKsXA1j/KMspWXUqhgxjoY8WL7xy3T+wiYBWPfB+cIq+IQH08kFBQ08hMzmUyW+bMtZGaK+VWrhdDQLhyzTPdTWwjbLofED62KIzwiUClUVOurySzLbObm5tE4a7hu83X0v8D+mq5ppN+Xs75k9Z2rLfW5FbnojDoUksIyX+I7ESKvAoW1Zc/8XZffvV7CkafhzymWfBajztGSPmAmp71GUFrayr2dxKe/Dws+WUBtaAx1KmGE9nbyblcb7Z0/Zc5Q1k+CXTfZFIeFCYnimhpITgath5YFnyxg/N22jrKdpdg1jJQRiwgdXe+8ZUemW6aXU5sHJ1+Fogaj8o0jb+TteW8zKWySzeXmaPfTp22bcnBzwMnHqc3n2rpqHRNeCGfy5skEuvl2ZPTtwmQy9ajRT9X6JTJdQlkCnHqTG4NjCXMPI9Q3FCnSm/1JRo4cEd4T3RUZH3VeFHUhURjvh/6qmTx7br3BL/N3OPQfGPUa+E3pns5lzgzGf2L18dx0ehPv738fj4kehIfPJzUVslWhTLMNBuxSivP1zPlNywUTX+SKx2d1b2cyZybugyD6Jiu9bgClpOTFHS9SZ6hj7uCjwKBWjX6hE0K59u9r8Rng0+buQ/ZFML1yKnfOXS2M4Go54k+meYKCQGHSQFHrOaSyD2RTklLCgIsG2NTVlNbg4OaAJEnMUDxOSOFe7hn1DOTdLX9/ZVrGORwirwaPIZYiP2c/XDQuVNRVkFKSYimXJIiJEQoSiYnQWDnEbJQ27zEk5wKorfei9BwufmRkOkr4UrvF5kPsWofWjX4t5fM7L+o88u/Pp6i67VI9Mn0Uz6Hipwk+9UvFfFuH8TZxw283sClhD17uTzM96MLm01ieeBUcfCGydXk8mR5EoYHUb0DtDjTkr1IpVER5RnGq8BQJRQkN0XYdoCS1BACPcA9LWWMJMpPJxMwXZ+IS2JDLxyxbHOwajMossdf/X3bbN+fzk41+vYSqdKhMobAsld+S/0Jn0OF03i3kpoh13JQe2B6UlIBOJb6rXo5eULAL9t4Jgx8TkrgyvZ+Bjwi52Ub8dOwn4nPi8YpbSsXewRw7JiTXh187vFuGUFwsfnt5Aem/gGt9OhiZvoPXKFiQKtZP9VzYv/k5qJ8I0ueUHWVuo8FI4alClGolXjFerXZdU1KDwaTnciclY92+AS5v7+jPaORIv55CVwKn3uSakP58ddFXXHLbJSx6YzqO7hqqqlqPaOks5onUkqQXRHLUumKg+yK7ZM4QmnjLxHiJA+zE4kTGjhVlu3eL30ZD90k0lJZC+sBZLJr4JuGH7uq2fmTOYLxHw7j3xYe9Ea5qV2ZGCYeEo/wAiHmxJedBRy9HwqeE4+zr3ObuA1a7kPfLBaytWw0ObTcWyvRikj+FzQvtJgxvKsuUVJTEwRz7mtyrlq3i12t+pa6izqbuowkf8eFY4V1eWAgBzskESyvrv8EyMi2g8YAJn0HoQkuRJElW3/HGtJTXDxpkUII8vBkRMMLSjoxMd2A2+lUpxCRaWNj8tVlZYr+iVjds5s1IkoSPk49Fis+K7D/h0BNQnWNbJyNTT7HTLvZFLeHTrPs6dP+xgmMkVx7GJBloNhWLyQSH/wuJ9nMAy/yDqF1haTWMfcemqrN5/QBKTpfwdtzb7H5rt1V543WkJEmMuW0MAxY1OIeZjX5h7mGt9mGW95SNfr2EMe/CwnTy6qq5YcUNPLzhYYYNE1UHDnRv9Ny+9/fx2/W/UZxT2xDp5+gtVHBqssFou5eR6aXE/RuirrEq+vDAhzy95WkIElFXxxrUPzEajBh0hi7r3mQCh+0bcT79KDdvnABbLoJjcvqLPodSC85hbVaC618fXJ+SIqJRG2PUGVk+cDmbHt/UprZcA135aNm3TLxoE37q3qcEJhv9uhgbyRkzHkNhYQbE3WspUihgSL3jdndJfNaW1fLeiPc48dkuAKo89pFYlIjOoIPwJbAgBfwmt9KKzFlPXTFkrIQysZkxH/IlFSUxbpy45PCmQl7wfIHNT23utmGUVqoojBmF75Th4DOx2/qRObuQJAkvLy8WD1wMwPrs71EohJG4NSkwo8Fo18jSHMpF55M49BKq3OfI0hEygtLjkLkKamxlFsyHNVlZ8PnBz4l9M5Z/r/m33WbOffZcLl91OWpnazkmk8lE5IxIomZGkVeZR1ZxEVvTl7AnqgaC5nX548j0Dczf8eTiZLy8vCzrz7Ya/W4ZeTv7b9nP/efcD3/NEhJTMjId5cCD8KO3yD/ZCLPRr4zWI/3MUX79+4NG046+s1bDkSetcgXL9GFKDsOKWEiwNu4onIvJ9vqBE3UN8vLm9Wez+/dGmPP6OtaFNG/0A5h3EMbYGpZkzgCUWrvF/bw6b/RzD3dn8OWDCZ9inUPIHOlXUCCkYZuSXibeK4vRz2iArUttZEhBlvfsddTPO77OIrKluKaYEMVphq5/laQfDrTo+NpZUjenEv9JPCUVKmrVYmHo7+IPgTNhYXqbcvq1Z/6UObsIcxPzkeQpJp2jR4Vx7vjPx3nO5TkSVye2dHu7KC0Ft9wEgtIN7MjZyZbgWyHmxtZvlOl9VJwWP/WU1JTw9+m/2ZK6xeZSb2+h4GAy2e55VVoV0/47jYFLBrapW5PJRLUin2FpkDXoy048QNuQJAlPq2is7kU2+nUxCkUz/5MqHcApGBQq9EY9R3cc5fPzPiesUiQZP2g/cKDTVOZVUl1cTUVhLQAfGaYQ+2asxatMpo9QngibL4T0nwGI9owGILEokdhYE+7uUG50xiU2ALeQrpc7zMnJ4emnnuL7jwfw13ofRi5bzUPfFJCTI3tl9zn0VbDtCjj5pqVIoVAQFhbGwriFaJQaThQexznyKNB6FPQLni/ww+If2tx9ibM/en9vPD3k/AMy9Qx5ApbWgEukTZXZ6JebCxMDZ6BUKPk79W92Z+62uTZyRiThU8JtNr+SJDHvrXmc++y5vLbzNd5x8+Z48EN4eats8rXIyNjl5Juw6Xyr0OcYz3rnneIkwsLCLOvPmPrAvdRU2wNGgwHy8sS/AwIaVWj9wTEAGZkOo/UHz2E2hjez0a9Un4MJQ4tGv6Pis28j7Qlwxx93cNeau+zvX4Y+CXMPgpN8Ei4DKLTCuKOwNvDEBYn3o5SGPNDm9Wez+/d69EY92RUiWlWrCyHSdrkgkCThqS7Lkp2ZVKbD6W/AYP1xnN9vPk9Nf4pFcYs63LQkSSz4aIFNbj9XV5FrF+DAzyksH7ycU6sajIsKSUGIWwgRHhGioK4I0r6Hon1W7VRUCClGgJCQDg9T5kyirhROf4tX9WkUkpiDQkbq0Ts4UVqppDuPSRZ9sYgHih+ksEJvkfcMdAlsVxttnT9lznCSP4PVI8R5YT1mJ4RKVRpqtTDMZWWBR4QHEdMi0Li0xzOrZYqK4Pikm/j2+q3UmqDIf46c+qKvsmoQ7LnV8ueujF1M+2way1Yts3t5XJz4feKEbd3U/5tK3IK4NnV7cO1JRuwbimOVIzFB3a8EplAoCO1B7x15hu5ijMYWpBErU0lK/hmHpx1Y/O1isvZk4edUAYiQaZ2u68fjFePFXafvwjR5MnpFJXUI+TJfZ1/IXgdJn4BR3/Udy5xZuPYTef1CFgAiablCUlCpqySvKpexY8Gg1qK+8RpG3jiyS7t+9eWXCQ8JYeuzz/JC6Ql+rirnucwC9r/0OuEhIbz6yitd2p/MGY7CAVK/hoIdliKj0UhaWhquGldmR88GIN1feEm3ZvQbfNlgwqa0LokDIirwLWMgn1/sSv+S2R0bv0zvQ+XYbNSntzf4+gpjSUlaCFcMuQKA57Y+12xzWfuyKE4Wsp26ah266oaP+57MvQDMcK4hQL2rq55AprdTchhyN4Cu1FJkkfcsSiQtLc2y/vT1BTc30OuF4a8xBQVgNAr5RK/GKQ4mfgGTvu/up5DpzQy4B879C7S+VsV+zn4oJAWOSmd0quJmjX4mExw+LP7d1OhnMpn4JP4TXt/1OrX6Wtub1a4ih5scvS8D4BYL8w9D9HVWxYPqLSV1ymLKqiuBhvVni/t3IKciB6PJiGRU4WjwJzy8mQurMoTMbHeG6Mh0nKQPYPvlUHrEqnhG5Awem/IYk8O7Rv1IV9Ww7pOkBgeynEwDRp0RhbLhCO6u8XeRfnc6z8x4RhRofeHSOhj5klWb5nx+Pj7g2Db1M5kzndoC2H4ZitRvhbQmYAioQrHsFopChnVbUAAII7VOoUVHFf7FC5kYco7I6Ze1GlK+atMc1tb5U+YMx1grDNCN9hhmo19GeZpFbv3YMQgcGcgVq68gckZzni/tp6gIkCRqXITMrLvWvcvaljnLGPgghF9q+XOAr5DCPll40u7632z0O3myc90e/O4IF/x+AVcoXfBq5BjWXRiNRtLTu78fM7LRr4sxtfSB3HwREYfuxmgyctzrOP8q+BezHh2Du7vwxu7sy9oSOTkSdSqRuVyj1OCqcYWEd2HXDSDJr0GvR+MOUdeCu5g4HVQOlo95Y4nPXbu6dp/66ssv89yjj7LZYGBNTQ1XAOcCVwDramrZbDDw3COPyIa/voRCKXJqnPO1pchkMlFUVITJZOKu8XcBsFP3AdXqzFaNfhe8dwGTH27bJj33aC73PX877/0wGYP3pI4+gUxvQ18NuZuEzGcTJAlG1vtB7N8PD016CAmJX0/8ypG8IzbX5xzM4YPRH7D95e0A7H1nL69Hvk7OwRxMJhN7s4TR74kBP+Jy7OZueySZXsbot2BJlcjvV8+IwBFcNvgy5kTPscyfIN7Z5iQ+zV7j/v4w9N0hxL4Zy/F82/deRqarUClUlD1UxolrS9HofZo1+uXlCaO0UtmQp8NMTkUOVboqlJKScI8m1haTSRjFdRXd8wAyvYYwfzdUBhcAjmVkAtbrz5YwS3tqdcGEhiial589/CT8Egi1+V02bpkuJGwxTPgSnCO6rYvfbviNN/u9idHQYAgxR+AXesZwx8k7iJljm0vXSiVCocYgOfPHHw2SnubzQVnasxfhHAbnfAcxN1skPvMr8y15/eLju6/r5A3JpMUXoTF4Mj33F7bdsFW8gydegT3LLNKjLdHW+VPmDCfmZliQDF6jLEXmc8K00jQGDRJljfP6dSX5mXW4FiSjqdRxnRtM3LcU8ne0fqNM72PIf8SZdT2hbqF4aD3QG/WcKLAN5zPvF06csD3D3vbiNt4Z+k6b0gDFLplJ7axhvBtXBNlrOvMEbcJkMlFcXNzt/ZiRrT09Sb/bUAy4FxeN2HCkl6YjSVg+7N2R1+/U76c4/PVhsjONFqOfn7Of+KgP/S9M+0M2+vVRzBKfqaWpDBsmPP91B47w7RUrMRk7v3jLycnhoQcfZGVtLeOauWYcsLK2loceeECW+uxLNJNTA2B6xHTOjTyXiyNvQGFStWr0aw8ltZUcGnKI5S45OA19uOsaljm7qc2DDdMh8X271Y2NfnE+cVw88GLAfrSf/1B/pj4x1RIx7ejtiEe4Bz79fUgpSaGkthiFUcOvp19FGvJEtzyOTC9EqbE5gBkZOJKvL/6aO8feaXN5q0a/ABOJRYkkFiXirJDg0BOQs77rxy3Td6hIFu9R4R6bKmeNsyWytLpa/DTFnM8vNha0TZYIScViIRDmHoZG2cTaUlcMfwyFffZzrcr0UVK+EM6tjVAoJFyMwmJyLLN9HtYZZRkAaFvL5xc4G/r/G7R+7WpfpofwGAKRV4CDl01VRlkGK06u4HTJ6U514RnlSfDYYGqKG6SOW5Igs6EqA/K3sW1TOe+8A3ffDVu2NBj9wtombiJzNqBQQ/gScOuHn7OYM/Kr8gkhg7DDqzi2o4TuCKLT1+r54rwv2PnsXwB4eDSqHPGirPwgY2X0GzBAnAuaJdgPfXWIny77ycqxoTPkHM6n/84viDsYRpURjNpAEbAg0+eRJImh/kMBOJRrayyJigKVSsjP5uZa1+mqdOir9dSUtJ7vu0rtzmmXyXyX8zMEze+SsZ9JyNaeniT6BqT+/7JMoodWHeLUqlMMFe9xtxj9dryyg1XLVlFQKFGrFkY/X6d66R2PwRA0p+s7lTkz2XIxrGjwLPx04aeUPVTG5UMuR6uF4cPBpSSDU9/spzS9tPl22siHH3zAdLW6WYOfmXHANI2Gjz60TVgu00spOQo5f9mtkiSJdVet45NLluOg96egQHzIm8O88GyLF0+ph4GfL/6ZlOgcPN3lXGoy9WgDYdTrEL7UbvWwYaBQQGamiEZ5ZNIjAHx75FsSi6wTmUuSxLTHpxE4QuTFGH7NcG7YeQMqrcoS5edaPYx89WII7XjuGJk+hq5MSLKXJbR+LQ1RBYnWr6fF6OfpV0mNXmyCfBRGOPIkZHW/Z6NML6YqU7xH+dvtVjs6NkjS2Yv2a07aE7DMs9Fe0fb7Hvx/EHJhe0cs05s5+SYcfdam2EMhJD4TczPa1ZxSocRfGoRLzYCWjX5hl8Co19rVtsw/gN7W8+C2Vbex4NsFrDy5slNNT3l0Ckt/XoqTj5OlzGz0y9tykt3v7LMcltfoa4h8PZKpn06lSidSsJD+C/w5icwj+wGhBvW//8H6er8cOZ9fL8RQZzmfy6/Mx7mmEL/UvZCRRXJyN/RngvPfOx/v84ZjlHR4eDZy9vYcLp8P9jX01SKvX+5GS1GwWzASEjX6GnzCC5AkyM6G4mLIOZDDke+OUJpme0BTnFLc7sjPCqUbp4fM5mTMcb6rgKoZf4H7wE4/lsxZyNFn4c/JYGowKA/1E8aSg7m2escaDUTXbw2aqiZOe3wadybciVuIW6vd5qbVUl7rRYH2QnBtZq9xFiMb/boYqQ2h8Gaj36knTvHnfX9ajH4nT0JN64bodjH3jblMefMSkCQk5zwAi3yAnMuvj+EcIT6g9R/iELcQXB1cLdVjx0J2zCTKb3sAj3CPTnf329dfc1UbX+irq6v59euvW79Qpndw4D74+3zLn5IkERAQYJk/FZICR0cIDhb1LW14cuJzOPLtESrzK1vt9mhmOlMd4RN/BVKprTSjTB9FqYH+/wKf8XarnZ0b5CP27xeyivNj5+Pm4MaxfPtaJ/paPckbxItrfq/NRj+PylH4yUEAMu2hPBE2zoa076yKDUYDqaWpaDw0VutPc6Rfaqr1utJs9HP0LgBAq9Li6BYD845Avzu69RFkejleI2HeIStZHjNfHvqS+V/PpzBCRFNvt2MXNEf62TP6JRWJSD+zQoUVDl5CuaQ+Z7WMDADjPoDpa22KA7XRKAyO5BQJA0vT9WdzXDTgIi7MOMKw1A+IbC6VkR1DkswZyO5b4CcvMFjnBxrmL6SX4nPiu7zLwEBwdQXP5H2s/fdqJIV431KKUzhdcpoD2QdwVNV7RfhNxjTiZTbHi4WnOQVHRb2CsRzp18vYeyd8p+Whcbez7sp1XDzwYgYs6g9330Vx4ACOd4MCu0qrYtTNo1DExpAQ+F9ecnDk0Q2PioN2o671Bupp6/wpc6ZjhJ3XQtLHlhKNUsPm6zaT9K8kQr29iIgQ5ceOweRHJ/NIxSN4RnpatVKUWMT7o97nj9v/QFdlndO+JUoNrmRHxVEXJuGkdsLNoXUjjUwvpSpD7Hn1Ded6wwLEt9lepB9YS3x2BJPJxKlbX8D7+F0cUttXfepqJEnC39+/R/oC2ejX5SgULfxPmrUW1ozhXCcVALpbdcx9ay4BAeDrC3o9Xf5h9xvshxQrXL4dPBvkPQH4yQc2XdC1HcqcuYx8GaauaFajfcwY0Du4cCrdkZKSzndXWlZGQBuv9QdKWwrnkuld9LsdRr9pMUArFAoCAgJs5k+HyP3si1rC3hPNS79Oe2Iaj9Y8arPwtEfy64eZsHoWV3paJ6uWkWkNs8TngQPi9/L5y0m9K5UL+9uPLllxwwq+OO8Lkv5s0Kc1G/0uczVwnbs/5Gzo1jHL9CJcomHMuxBs/b4t/mEx0W9Gsyl/k9X86eUlfkwmSElpuN4sfaLxEEY/HycfJKUGPAaBS0R3P4VMb0blLKTz7EgyJRcn80fCHzhF7wPgp5+gvLyh/uRJ8W5KEgy049xtlve0a/STkbGH5zBLHvPG3DnoKWbHFxOctQxofv3ZlOpqEeUANB/pt3Em/DWza5Ojy3Q97kPEt7TJPmB4wHDAfjRBe8nam8UnUz4hca2IUpYkEe2XPnAWUf+5wmIkscxtXtENhhPP4SSq7iGjIAAnJ3joIbjnHpGGw8EBwsPtdilztuIxBMIWM9I3jpnRMwlyDULrriVmlDtIUpemuWhKSQnUqLPRU4tWpYWqdPhWAwcfa9P9bZ0/Zc5wVM4wZQUMetSqeFLYJKI8o1AqlJa12bFj4OjpiNrJVjHJJcCFiGkRKFQKnvd4nhO/tM0KU1QEaoMHf5ybReUl/0OV9EGnH0nmLGXMcrgoG9QNgSktyXtCQyR900i/6uJqDnx8gIxdLSs7GHVGsgfriBy2m9fd7hRGx25GoVDIRr+zGYPB0EKtCWpyCXUSG+K0uDSizo1CkugWiU99rR5dtc6ySRnsMYFHJz/Khf0uFBuSoHnNRjbI9H4KqwpZ9vsyFn0nJOa8vSEyEjTlhez4ObvT7bu7udHWLH25gLu7rN3dZwg+H6JvsBigDQYDSUlJNvPnGul2sr1+4POkF5ttSuOsQeWgalO3ht2llB+JZVbateA1psPDl+mF7P03/BLc7GGd2egXHy8cdMLcw1r0RDz32XOZ/Ohkws5pcMm+fsT1DNXdjKpqGHUO0eDg25VPINOb0bhD7C3gOdSqOMpTnD7vTd5rM3/ay+tnSZ3rJJzAfJx8oLZI/MgH1TKdpa4Yqm1XfoEuQu4Y1ywiIqCyEn74QRTV1MArr4h/T5vWIAHamKzyLKAZec/DT8Ffs0FXblsn03cxmaC20Caaa+JwH5QmB06cEN/y5tafTTl9Wvz28QE3e59+ow6cI8WPHPVyZtP/Dpj0nU3eRXM0wZG8I+g7qYbk4O5A1t4sipOKG7rtD7UuPmQ5NISKNhfFbHYwGzpU5CuaPh3eeQdefVWoT8j0ImJuFu+jo/UBcIhnJa4Fyd1i9Dvw8QHeHvA2BYezqVWLM59A1/rvdNhicB/UpnbaOn/KnAWEXADucc1WD6p/Jcx5/XIO5pC2LQ2A6qJqjAYjGhcNS39eyoR7JxA9KxpHbzsLOjuY1q1j4N/v4KqphZNvwKm3O/UoMr2LwX6DeWPOG3x3yXd2pWPNkX7JyUIO20xNcQ0rbljBsR/tqzKZUWqUbJ2dSdrknaRLoaD26MLR28dgMJDcLdrN9pGNfj1J0BxYmIZH9FVcMeQKJoVNAkRI6TCxzuRg553LLCT/mcyzTs+S8puwJJ4TOomnZzzN4kGLxYbknK9h8KOttCLTayg7CfEPQ6GINtGqtLy7711+PfErRdUiwcqAARC34xMOPL2q090tuPxyvtBq23Tt546OLLz88k73KXP2Ul5ue2B314jHAdihW05mWZbd++oq68jYmWFXV74paXeG89viFMI9JwhJRxkZM5ISfCeDyf7GNSZGSDNVVcGpUw3lJpOJv0//3ZCLpR73MHdmPD3DyhPyyqFXMiztPeJTbidv2HYbA46MTHuJ8RJKDkkltqdC5rx+5nVlZWVDdJVR2xDpx/EX4SdvKD9l04aMTLtYEQ3bbddyQa5BAGRVZHHNNaLs99+hoAA++QSysoTj2c0322924zUbyb0vl9nRs20rqzOhaK/wVJeRMZPwjlC0yd9qVRwWBi4uwtickCi+9/bWn02Zt3IYfw8cimvESfsXKNQw8QsY1zPSUDJdT5RnFC4aF2oNtZwsaOa/cxvxjvXmgYIHGHNbg4NhXBwo9HWcON5waGk3innLJQwvFjnVRoxoKPb3h9DQTg1L5gwmqzyLj/Z/xJeHvgQg56v19N/5BdmJlejarrjZJkwmE0hQXquhxmz0cwkE53CY9D1EXNbmttoyf8qcJTRxktmZsZP/++v/+Prw1xajX3Ky2Et8e+G3/HH7H6RtTeONmDc4+HnDIbZHuAeX/345MbNjWu3SaISaWgWSyYh/mAZmrINJP3TpY8mcRVRlQOp3UJluKXJSO3HnuDuZGjHVrpSwry94eoLBYJ3L3i3Ejct+v8zqO9wcpboCPi6D33wfBK1PlzxKa1SYNbt7ANno9w8wO2Y2X170JYO3DeZZ52fJ3p9tifRLTLSWvOkMjl6ODFo6iGKleHGDgrqmXZmzlKpMOPY8FOwEwFnjbPG+NnsaDhgAWbFTKRswrtPd3XjTTWzU6djVynW7gE11ddxw442d7lPmLCHtB1gRA/nbWrzshqmz8a48B4OihgdXPW33mvyj+Xw04SMOf3241W4jyq7jksonuSnqnA4NW6YXM/gxmPAZKFQiQqBJTguFAoYPF//ev7+hfN7X85j22TR+ONr6BqWuTiRAB3F4IyPTLrZeCqusva/NRr/0inSby8+pn+b27BGydGZpT3d38HB2ZlTgKAb6DATvMSLy2imkW4cv0weIuRVCFtoUW4x+5VmMGiXy9ul08Oyz8Mcf4pq77hLGGHtIkoSfsx/OGjuGvbHvwsUFIMlbWplGeI6A6JvAwfrwRpKAgT+zadAg7v6zbXlMDUYDadVHKXc6TEy4nZdUjpI++zj6POxeZlWkkBSWvH5dIfHZVP4uMlTPyDXP4fTnbxQJX1sreU8zBkND/j6zyoRML0ZXDnv/Temx17hx5Y08s+UZAEZfO4TcUXPRm5SkpnZtlyNvGMntx26nROltG+kn0zfZeT18p7Uy/B3KPcTTW57mk/hP8PISTjMmk3AmnP70dKY+PhW/IX54Rnni6Nm2qL6mlJZCRtx5bL5wABf+Opn7tr1pV5pbpo9QsAO2XQoFdpJ/N4NZPhusJT6VGiX95vdrNQXQ8VVJTFoTgE++D2G+PWPw62nkHVJPc/pbSP8FENbnsMlhKNVKvL0hIkJMpPv2dU1XoRNDueTbS8g0ic12meMhEgoTqNXXQnmSWOzm/t01ncmc+fiMg/nHIfp6S5H5wDCxSLhFDBgA+RFjOKkZYhUe3RECAgJ4/oUXuMDBoVnD3y7gAgcHnv/f/wgIaGsGQJmzHoWD8MpvJqrKjEYjscBVbH6+PfUBKcUpNtd4Rnty3v/OI/LcSJu6xpRlllGyP5nbht7IsLKLOj52md6JgxcoHcS/T7wMG86FuhKrS8yHL42NfpPDJgPwwf6W8w9sS9vGuqN7MEi1XD/yYVwKvu6qkcv0FbR+4BhidcDcz7sfAOmV6dToa6wuDw2FUaPE5StXNkh7BgTARQMuYu/Ne3l97usQehGM+1COlJLpPMOfhf7/sik2G/1yK3IxmPRce60oN0vPXnhhg1NFh5DlFGWa4jtBRN15DrOpigxTU+F4jD1F69vUVFZ5FkYMSCYlI2Lt7FWyVsPqkVDQmpujzBlD/lZI+w6ayHiajX7xOfGd7qLwVCE7XtlBabpQInFQGaiJG0aFZ6jlYNKevOc+px95fNMaAgPF91qml6NwgIS3CCqPByC/UsivR8+MwmvOWAxqbbfl9SsqMVCrzgPqI/1OfwN77hDSyDJ9C+8xEH45GKotRSMCRKhxfE48JpPJah887KphDFg0AK27lpv23ETcQmtp0JyDOXw17ytOrWpZRcTsDCt5pLE3cxs5eXvA0MlDSJmzF58JItrY19pBP6s8i8/iP+Prw/bPT8wSnyfspJHU17Qs152yPYeBR4O53kHDuKq2rQvPNmSjXxdjL+TUigP3wbHn0Rv1OE13YvHvi/EfKlz+x9RHnu7Z03XjqasT8jkA/9pxIf3e6seBnANQkQSJ70JZ2xKsyvQCVM5Cq1vlZClqavTz8xPh0Xq9dXh0R7n73nt5+NlnmaJUMl2h5StgPfAlMEurZYpSycPPPsvd99zT+c5kzh5CLoR5B8FvCiDmzdDQULvz58WjpuJTOhMDep78+0mbeidvJ865/xyCxwS32OWJlQn4r/2CXzYtQxf7cNc8h0zvpCoTMAmJiUaYZZYSE6GsTPz7uuHXoZSUbEvfxrH85jXjH9rwEBesGEuR95cs6PcCUuZv3TR4mV7L6DdgxlorA0eoWyh+zn7oTXric+Ntblm4UPz+808sh0byIaJMT+Pr7ItSUmLCRG5FLv37w/j6lOKhoVgkP+3xZ9KfLPx2IW/tfsu2sq4EEt+H0pbzdcjINOaS0VORTEqKSCStNL3Z9SfAsfxjnP/1+QA41UYTE6UUFSZTgwNGdSZUJNvkiJM5g5nwGVyUL9QdGnH1sKv56qKvWDZ6WTM3tp3M3Zmsu3cdGTvEWtLBzYHg2xdSED6KEyeExGKoeyiBLoFWkX7mfH6NpT1lejFKDSxIo3b8ZwAUVheiMwi1kehowGSiq1M/7XlnD0e+PUJ+ZR5IRhSSAj9nP8jdAAlvi5QHbaCl/bvMWUbsMjjnK9B4WIqG+A9BKSkpqCogszzTyujXOMDd3n9/lVZFyoYUihKKWuy2IN9EYMJmfAsKGKqBLxWb4cQrXfFEMmcjTiEir2gT9Zm9WXu59rdr+d+2/9m9zRzpd/y49bv5zQXf8JL/Sy12GbL0HP53/9tcOyCH4Px1nRp+W5EkiZCQnlPYkY1+XYxC0cr/pBM+hzHvEPl6JOGvhXMk74ilymz027dPSDt0lt+u+40/nxDyec7OUFAtPId8nXzB/1y4KA8iruh8RzJnDzUFUN5gzTN7FiYWizJJgjivfAZteputL3eNx+rd99zDvvgMsmMe5SFfJTd7wPNRgUx99FFSMzJkg58MCoUCb29vu/PnqFEQlymi/b449AXH8493qI/aKCf+mPsHL0Q/h3bQVZ0ar0wvZ+QrcN5m8BhsVeztDeHh1hH5ga6BnN9PHAh+uP9Du80ZjAb2Z4vwQG3FBF5LzYYR9hetMjLtQZIkxgULOe49WbYeY8OGiXe2pgZWrBBlAQFQVlvWcNHfF8Ix+X2U6QISP4CNc0FnnadCISkIdA3EzcGNwmoRQXDbbbBoEfzf/4GmhRS7uzN389vJ39idudu2svQY7L4FMjufh1qml2E0wPar4NgLNlVD+7vhWT0agJ/2b7K7/jSZTCzfs5xR74/iUN4hNDpfRme/1+A0kb8NfvaF019DzE1wUS64tKw4IXMG4eANClvDxriQcVw+5HIiPTv/3zJ6VjTXbLyG6NkNBr3GEmSSJPHnVX+SdW8WER4RoqKuGK+8l4n12iNLe/YlnILxdQ5Aq9ICkFEmDMW6n1cydMNrJCXaSgjnH8tHV92xZH+bHt/Ezjf2oDMYCChexJzouSgVShjzLizMBLV7m9ppaf8uc/ajVWkZ4CukNg9kH2DQILFeKyyEdNusAlZ49/PmwZIHGX/X+Bavy02pJPjkRvyTKyk2wnr1YPBp+R6ZvsdQf5EL7Vj+MersRILGxoJKJSJHzeksAEInhdJ/QX+MemOzbefm66lzqGVqjpGqc37s8rHbQ6FQ4OXl1SN9gWz063IMrVnrAmaA10iCXUVUyt6X9rLl2S2ACEt1dYXKSvuhqe0ah87A0R+Okr4jEwCfwEqqdFWA8LhFoQStL6ibSaAh0zvZMBU2zbf82TTSDyB2uBOSyUhObld6bQUQ2+8x8pa5kHIX/LBzA48+9pgs6dlXMerg1HJI+wkQ8+aJEyfszp/+/jDIcwxh+Tfzr36v2t2If7foO75d8G2LXaaoStk9bjd1bkZUyrZ5MMr0USSpIZqqrtSqypwnbcOGhrKbRt4EwOcHP7eRWAQ4XnCcKl0VWskFl5r+uPj4g3N4twxdphdTdgoOPQHF1rmGLh10KbcMvIWJIRNtbpEkWLBA/Lum/tX09zcx4O0BDH1nKEn5RyDvbyg5YnOvjEy7KT8FeZuhrtimKuHOBEofKrVs3D094frrIbCVNEJH848CMMh3kG2l+0CY9geELur00GV6GQolZK6EvC02VSoVDNCeC8Cq4+vtrj9XnFzB7X/cTo2+hsDKOUw5eohLx09rFGhtBNd+oK1P0GuWB5c5OzCZIG8r5Gxo/doO4uznTMS0CLTuwpCTsjGFvA9+Q1ueR0KCUNVpSmFqCouj72Nk4DqGDOm2ocmcadTkI+VvJdpdRH6cLjkNgH+UM9WuvqQm1GJsdGZ9+OvDLB+0nBO/dOzA8OoNVzP68bk46kKYmvMzq674XVQoVOAU1GbJ7Jb27zJnGRXJsPtWyFpjVTwyUHgfHMg5gEYjcjKDdaoLe0iShNpR3fJFQIXekaOTbyZjWgVJOljrPgf8p3XkCWR6A/pK+MlPOPQ1Itw9HDcHN3RGHScLTtrcptFAjDjW5ujRhvJJD05i0eeLUKiaN3tlbs9m4ZZE7neuxsN7aJc8RmsYDAZOnWpZ+rYrkY1+/wSGGiLdxEc97/c89r0vQgYUioacQXv3dq4LpVrJw2UP43OjOO1xDRBRfg5KB1w1rkK+rPSYrJnc14i+GaJvtPxp9t6prKvEVB8LPWS8M0em38kpj7Fdlps+Px+UtUXM/GMSMQkxQkJCpu8iqYTUcVJDHrSaGltjiZlRo2Bo6ntEF/zL4gXZGKPe2KIHD8DJnDTGOMD+qAJI/b7jY5fpO+y/F1ZGg67cUjRjhvh96BDkiTQYzImZQ4hbCIXVhfxy/BebZlYnrAYgWBpLgHMakb4pYGr5fZWRsaHyNBx5Egp2WhUvHbSUW2NvZVTgKLu3TZ0K7o2ctksc48kqzyKpOIlgzxhYXArjP+7Ggcv0GYY9D0srwTnUpsret7stmGWTB/oOtK3UeEDQXHCN6VDbMr2ci3Jg2u92q6aHC6Pf/qKNVFdX29QviFvAkVtOMqv6Y0YeX8Wg8ACuN6dEr8oCr1EwazsEnNtdo5fpTiQJti6GAw/YVO3M2MnL21/mYM5BOze2D5PRZMnpl3Mgh4Tv43FR1VJXBykptpvs/Yn9uW/ddtJVV+LkZFMt01s59Sasn8JED3E+Yjb6XfDqDNKnXkmNSUum8OPnxG8nSN6QTOy8WHwG+HSoO/8h/igCheO1h0ejioKdUNlKCFcTWtq/y5xF6Csh8T0o2GFVbM7rdyBH6A7by2/fHJX5lex9dy/Z+7ObvaaoVEm1eyBVQcJZzF3btihTmV6K0gncB4BzhFWxJEkWp8GDufa/zYPqfQOPtVPxP/2VH4k8+Av9fU8j6Upbv6GL6Mm5Uzb69TSJ78N3Tkx2FhryZY+Vcfvx2y3VY8eK312R109SSOSWCM9DR+96aU9nX6G7fPINWDUIqtr3YZc5y4n7Nwy83/LnIN9BZN2TRfyt8RY97uho4QVbWgo5OV3TbX4+qKqTGbN3DNMrvfF09OyahmXOTiQJZqyHse+36XLzAnPfPuwaoi9beRmXr7q82ftNJhPceYLhv5yPSlKDonXPMxkZ3OLAe7xV1Iq/PwwdKt7Dv/4SZUqFkuuHi9PAPxL/sGnm6yMi6XRE5RIuG/wkM6ujQFdmc52MTIv4TIC5ByGi+bnOHhoNzG8I8Gd/hZBCnBk1s8EQ0ySvkYxMh7Ajl9cZDEYDJwpEJMMgPzuRfvoq+4sCGRkAZfOG5gWjJqAwOlBGFinlp+1es/mXfmiOXoeri4KHH24kQ7tuAqyf2vXjlelZRr8BI160KX5t52vc9+d9rElcY+em9vHDkh94I+oNDHUGxt89ngdLHiR8QhAAj2x4jPDXwnlj1xuYTLBtG3zxjTMnCycQNURWg+hTBM2Dka/i7hoFNBj9FAqIrBe4MedlPvnrSeI/iWfh5wsJHNFKqLwd9DV6akpqKC42YZBq8fCs/4Ya9bBuIuy/u7NPI3M24hYnZKqHPG5VbDb6JRQmAA1nMkePQl0rsSPlWeWsWraKI981ryZSmF2LQl9HjSKfO93h8pKfRL5mmb6JJMF5f8Ogh22qhvoJo9+h3EN2bx1Y7xvYONIv72geK25aQerm1Ga7NE2fTlVcPy5UDIDDT3Z87GcwstGvp3GLg7DFeLqISL9Ux1Sr0OcRI8S7npraEEXQEbIPZHN602my0oR2hMpDNObr5CsuCJwJg/+vQZZEpk+iVCgJdLVeMKrV0M85g+ATGzi0s7JL+snLg/wAJbctf4El5x5FIclTT5/HdyI4h7Xp0iFDxGFLfoGJ1zZ+zrgPx5FT0XaLtL5GT4V7LSnqGu6rvE2WApNpG9E3iiiBJu/pufWO/Rs2NJw33zL6FjZcvYHPFn5mde3x/OPE58SjUqjwyLyEnRkLKQ562CpRuoxMm1C7gudQ8bsJxbXFrEpYZTGQNGXePCEf7+cHGzNE5Mv82PlQliDyodmRY5SRaTd1xZC1WshENeH3U78z76t5/Pfv/7a5uaTiJGoNtTiqHBtyXjVm0zxYIedRk2mG8iTI/F1Iyjdh6ABHgkuWEJJ/LSUlDU4PJpOJjLIMtm6FP+p9eO69Vzj8WIi8CsIv7ebBy3Q7YYtF2pUmDPMfBkB8bnynu+h3fj/G3DEGXbUOSZLQumvpP1A4R2w7foq00jQOHTHy3//C889DVVkVocF1zJ7d6a5lziZ8xkPcXVw57j7WX7WeW0ffaqkKSNtN8PH1JNd/Vi/86EJuO3obTt5OmEwmqgqq2tVV4ppEXvB8gZRfD3Es9G5e0mp5aftLYDLAyJch4squfDKZswWFGrR+0OSMblzIOE7dcYpDy4ShJSQEfHyEwe9IK5kB/If4s/TXpZzzwDnNXlO9cRcj1zyHV44TQ7UKIioOgdKx048j0/sYFiC+zc0Z/QYIATsyM0XwCkBNcQ0HPjzQYrTpgcgC1k9+mjVMhYCZXTrmMwX55L2LaTWRrd8UmPQdWj8x+aUXpZN9IJv84yISz9W14YXtjMTnrtd28dn0z8jJqBeMdxLtW2QVA86Dof+Vc/r1NXLWw/ppIgF9CwSZMglM3MrxjbktXtdW8vKgVl3AeSXVvKOSDc0yCGtJdQ4YalAoFERFRTU7f2o0wvAnIfHuvnfYnbmb5XuWW+qz92ez49UdVBXa3/ioHdWsuvwI62euJ8qnbYZGGRmrnBaNjCITJ4JWKyKhzd5k7oog/v50Bo//R0Hj1Ba/nfwNgFmRc6gt8WZn5kI0Y57tidHL9EbqSm2klxQKBR+nf8yC7xbw5aEv7d7m7g5vvw2PPZvP7szdAMyLnQcZv8Df50OZbX4EGZl2U3JUGOIyVtpU5VXmsTpxNTszdtq50T5mac8BvgPsO4v5TIDAOR0erkwv59Sb8PcFYq3ZBI0GLtN+zvDUj/E2zbCsPzee3kjEaxFc+4uI3l+yBEaPbnLzsKdhwH3dPXqZnsBksjEKDw8YDtAl8p7Drx3OnFfnoHXXkhOfQ+7hXGbOFHmxKh1E6NahjdHs3StUdp5a/CjLpzjgoU7rdN8yZx/DAoZxbtS5Vg7ZioST+J3eQ1KCSAsgKSR8Bwgn/i9nfclHEz+ypGhpC65Brgy/bjg6Tz9q1NkYqMNZ7SzyksbdDaEL29xWa/t3mbOM8iQotD6A1qq0xHrHWtZgkiSCVKANef0UEnEL4nDybl6ruETjR0HIMF5Y+CE33KKHpTVyjty+TvKnIod9E1qT93R1hbD6Yz6zxGfw2GAeKHyAcf8e12x3GTUnOem8gddqvSF4XicG3nYUCgURERE90hfIRr8uR2pj4tswd/FG5mXn8f7I99nxSoN+8pgx4ndnJD5H3TKKWa/PI79cSJucEzuUxyY/xtJBSzveqMzZj74KSg5BTUMYaXJxMnO/msuYD8ZYyoZeNogjU28jSd8gL2Iymdq1qGxMfj44lmWQke1FkaNtrheZPkjCcvglEPK3I0kSbm5uLc6fZjmJIeX3ArB8z3KqdMLIl7w+mXX3rKMosajZ+4uMqVzkAgtcj8lyYDLtY+ul8GsYFItFplYLkyeLqvXroaYGnnxSbH7i42HPwQqKq4WR8IFzHmDLdVu4Oe7/AHB2Fj8yMh1izShxiN0ISZKYFDEJgF2Zu5q91dMTtuSsxoSJ4QHDCXYLhuDzYdxH4Na/W4ct00dwi4NxHwo1kSYEuQpJu6zyrDY3V1RdhIvGhUG+dqQ9AYY/B2Pf7dBQZfoAYUth3MegdrNbLfK/SCQnO5NVnoXRZOTVna9iMBnQ1zgSHAyXXdajI5bpScoS4EcvOPKMVbE5muBk4Umqdbb5HjvKqmWr+GrOV/j4wLPPmjB5CqPf0NBoJk6EN9+EAROGQ+gl4BjUZf3KnAWYjLB2Auy6yabq3Ncv5ODMe0lKUZC6JY20bWkYDcIAGHpOKJHnRqKrso1mbo7gscEs+HgB1R6B1KrF97ip6lNbacv+XeYsYttlsHlhq5e1J68fQFlGGSWnS2zKjUbIdI3j9PCFBMU6I0kSCqWcgqXPk/o9HP+fTfEw/2H8cfkf7L2p+ciopnn9lBolR5MdufRSyZKWpTEnfk9kylcFxCTEEOTesRypHcE8d/YUstGvizE0dvFvjsP/ZVD6B1w+5HIunXwpM56ZwZDLh1iqzR6Fhw5BbW3HxhE6MZSQRWMwmcDJCabEjOSpGU9x3YjrxAXbrxQ/Mn2L4AvgkiIreUMvRy/WJK5hb9Ze8iqFMXDYRBdqXH1JzVBSVR88tfut3Tzn8hzbXmw5SrApJpOI9Bu3pZIHfn6Q12a/1lVPI3M24zUGYm4FBx8MBgOHDx9ucf4cNUr81h1eSLh7BIXVhXxx8AsABi0dxHVbrsN3oK/de4/8fIop6+dyBwGML//COoJLRqY1+t0GgbPBfaCl6LzzxO9t2+Cpp+D4cfF3hvfnnPt7GE9tfgoAhaRgUtgkgkxj6ee9i5fOHQ4ZK3r4AWR6DdE3QIT1KbTBYMCr2guA3Zm7MZqMzd6+KkHk85sfW5/kz30gRF8PGjnPrkwXoPUR72ijudJMR4x+14+4nrKHylg+f3nrF8vINMV3AkRfBxp3u9XDhwuHxm/+3MvQt0az+IfF/H5KyB9H5v6bW24R0VdWlByFvy+ErM7ne5P5h3EKEpLZTiFWxYEugfg4+WA0GTmaf7SZm9uGyWTi5yt+Zt396xj7r7FM+c8UAAqrCymvE7md3/xvJA8/LGTziLoGJv8g59nta0gKwIjRqOeTA5/w+MbH0RuFWteA8e4otBoqK2Hdwxv5YuYXGPVinTftiWmc/875aJw1LTRun5ISqFELybtAl0A4/Q2sHgVF+9rcRlv27zJnEf3vhMGP2hTvSN/BZT9dxkPrHwJg2DBxlJKeDgUFLTdZmlbKq6GvsuXZLTZ1JSXinFCSwMMDSP/V4mAr04cZ+w6cf8LGSd9R7cjc2LnCabUZzHn9zEY/oxE+eakAsrL46itbv//SYiMGpZ4xrnoeMf4JRQe68kmaxWAwcPRo59YX7UE2+v0T5P6FNvNXvrroK5459xkmPzKZyOkNOSnCwxu0kve1/btrwRyRlVW/rw4MtHPGXZUhfmT6FnaMHR5aD4sX9Y50EXHq5QUBnrU4lOVz6pS4bsyyMQy7ZhhuIcIrwWQyoa/Vt9plebl4l33H6zlvzncMM3WNZKjMWY7PWPFR9xSh+q1tGIKCIDgYjHoVM13vAuCVna9gNBnxCPcgbFIYDq725SAOfnucEXu0rIz/E9W037v0MWT6AH5TYPKPIt8BQObvDIgzEhgoovwOHRLRf5deChqdH1WmYt7f974l2g+EFKiLphgXhxJANjrLdJBBD8PAB22KI50jcVI7UVZbxsmC5qU6rxhyBVcNvYqFcQu7cZAyMrYEuohIgvyqfOoMdW2+T5IkXDR2UhEU7IKd10FRG13NZWSaMGQIzJ1rosL5CEV1efx8/GcA/ErOZ/bofhYJMyuqM0Xeypq255WWOUNROcN5f0PMjVbFkiQxxE84Yx/N69yhnCRJZO7JJO9QHkMuG8LoW4Rnd1KRiPILdg3GUS3nr5IBZu+C8R9x66pb+e/m/5JZlgkIx4MI53xcitIIvvpczn/3fFQO1kZhk8lkif5rjY3/2cjKW1ZSVGyiVi3msUDXQNBXQm0BKNpnQJQNfr2IyKsgdplNcVF1Ed8e+ZaVp4R0u6srxMWJOnPu2+ZwD3Nnwn0TiFsYZ1NXXAwRB38jMPNHJn08BrYsgqPP2GlFpk/hHA7OYR1y0jdH+iUliTOazZvBddV3RMb/Ql6ebXSq84h+/HBlAi79ThOqywJDTRc8QNvoyblTNvr9E0z5BRZm2hSbpRMlCaZPF2W/vJPNV/O/ZuUtK1l3/7o2NZ+6OZUXPF/g6LeHAXFYvjVtK8nFyRiM9S/XeZvEj0zfI3MVZFp/oSeGTgRge/p2S1nYlq/ov/0T9v2WgcloQqFSMH/5fIZcNoSS0yW8Hvk6W56x9dppSl69kqh+4GBG3TAYXGO67llk+gySBAsWiH/X7rgedwd3ThWe4o8E8S4b9UZ01fblTcJvns2xSTdh9B4MfpN6asgyvZGstfD3BUg5f1qi/TQaePxxWLwYwnSzca0aSqWuEq//eXHbqtsoqi4SC83sOfxYdxpCLmixCxmZ9qJSqBgZIPR2WpL4vLD/hXy+6HNGB9VLSqwa1CYpHxmZNrNmLPy9wKbY28kbdb3jRE5FFxhMiuNF3o9G+VZlZKwoT4IVMXDsRbvVkgS33GJiYey5DEv5FEziWKR/0d3ceKPdWyBwFlxaBxFXdc+YZc4IzM6wnY30A7jt6G1cudZaXSmpWBj9or2iGwprC8X3OPX7Tvcpc3aikBSEu4vUKqdLTlvKvVZ/RdjhVRQ5hTDs6mFW92TuzuSt/m9xckXbcjMnr08meV0yuWWFmBRi3xzgEiCM3wtTwWNIKy3I9DVGBAoPmBMFJyypVS6+WNT99hsUNZ9dBYBZL84idl6sTXlhoQmP7OO4FGewN3MvD1aFQextXTp2mbMQfRVUpIDeVl47sSiR/2z8D89stm8c9vUVwVMGA5w4Ad9+CznREykbIs661zQRaSgshDpVAR+UwY+DPhIKEb0Q2ej3T6DxBIUSvVFPWmka8X/E81r4axz78ZjlkgsvFAeJpXsSSFqXxLHvj3Ho80NtyqkmKSQChgVQZhKesf4BRmZ/OZvoN6Iti0yZPsze2+HQ/1kVWYx+GQ1Gv6iLh1PpGUrGEx+x9u61Vte7hbrh7OuMg3vriXbNRr/fHLfytC6WLJO2kw8g02vYdw/sau5kxZZzzwU3NyjOdWWm77UA/HT8J0rTSnlK8xSr7t1o976scgMV3s6EB5bI+fxkOodbLEReA05BXHghLFoETz8NgweLb/bYMRLROQ9YLv/s4Ge4aFzIrQ9w9vf/h8Yt0zvIWgMb59iVvxkbPBaAXRnNG/1scI4CJznPrkwX4hgAWlupbYWksOQNSi9Nb7WZhMIEBr49kGt+vcb+BbG3CLl6X9mRR6YZ1G4imkvZ/L5DkmDx4kJuGnsF406tZXjKZ/zrghn4tJTaRZJAoez68cr0PNnrRMRwtbUKzR1j72D3jbt5fOrjne5CqVZSllnG2wPfZteb4vvsrHZmYuhExgSNabiwKh0yf4cK+aymT1JyGI69wCgPsVFobPQLu3YG+eGjOXXSdg/r6OWIrlJHXXnbIuiv33Y9y47cRm6VkPb00nqjUbZfHlSmF5L5B/wxDHL/tioOdAnEz9kPo8nI4VwRVDJ2rIj2q6sThpW20NQ5u6hIIn72g6Rc7ooOOKgdAP7TuuBBZM5qEj+AFVFQuNumKqs8i6c2P8V7+95r9nZztN/HH0NmJtQOGMFdnwrD9e7dDZK0dRV1HHxmJV65IijKx6nncvr1NLLRr4tRKNrwP6m+EnL/5tHfLiP8tXB+z/sdrYfWKgmuhwfMng3ZsVOoufEObj9xO/dk3tOmRLnhk8O59u9rKXQVkqEOPllU6apQKVREekSCrgwSPxR5CWT6HqOXw6jXrIrMRr89mXssskuL/juS9PGXkB82Ep85o62uVygV3LTnJibeO7HV7vLzwSvzMCM/LOTdL96ltKa0a55D5uyneB/kb0OhUNC/f/9W50+NBs4/X/zbdHwRlw66lLkxc9m4y4migIFsOu6HsYm6iUFnYPXuT0kZ4cPdoZ5wuPMbeJk+jEsUTPgUPIag1cL118OAAQ3VEyZAUPESy98zo2aiUWrIzYX5sW/Rz2VVz49ZpvdQVwR5W6C6IS+aef4cHzIegJ2ZO+3e+vbutzmQfcDaeWzaShj9ZrcOWaaPMXUFjPvQblWcTxxejl4UVbfiFg4cyTvC8YLjHMk70vxFGk9Qtu58JtNH0frCvIMiT1EzKBQK4uL6s2yZxAOXnMeyiVdbVCXsUnQActaD0b6yhMxZRulxETFcbh0l1d+nP2OCx+Csce50F9VF1Rz4+AAFxwswGcT3d0HcArZdv42XZr3UcKHncFhaA/3v6nSfMmchhXsg/iEmOjsB1ka/GXcNxTd1L3VvvEtVlfVtXjFe3J1xt00EYHNIkkSdSY1Jryag+CLmRM8VFcmfQ/ov7RpyW/fvMmcJkgT6CnFWbVUsMSJAGE0O5BywXHpNvU/W2rVY0ko1x9fnf80HYz6wKisuFg3p3YXThdkxTKaP4zMeBtwnnAibMCJgBBIS6WXp5FXm2b3dnNcvJUX8XrgQ+vUTDtpGI/z5pyhP25pG8Z/78ct1YZIWYqqP91hwgEKhIDbWNvq12/rrsZ5kGig7ARumMUspzMxp3mncevBWBl7SkPi+uqiaRQtNqFRwOMOT9CIXFKq2/+fS6yEtTfy7ykksZKM8o1Ar1VCeALtvgoxfu+yRZM4igueB32SrolivWLwdvak11BKfEw+AkxOMn6wmdegF7E3tuOdDfj4YKScAiRt96whzD+vM6GV6EzPWw/nHAdBo2uZlOH++MP7VnpzKw/2+IbhkCR9/oSZ51GKSXIbb5EEtSigi4vkCBmycwrq6AeA1xn7DMjLtpc7WgWHUKHBQqRl7ag2Tg2by6uxXAcjPM3DDiHsJrVne06OU6U2EXwpLKiBorlWxRqNhavhUPr7wY76+6Gub2xIKE7hj9R2M+WAMJTUlPTRYGRlrflryEwX3F3BB/9Yljo/lC/UTs8yeDbkboSyhK4cn00fRaDRIElx0Edx8M6jVLVx84mX4ayYYW89pLnMWEHUtXFwocjd3EyWpJWz6zyamPj6V8XeNb/lihQpUco6/PknQfJi1gxqvcQCklqZaqkKCTRijYyn2j2PvXttb2xIUACKyJWF1AulHS3GpiWNK9k98dckXovLQo3D02XYPu637d5mzgKC5cGGSOCtsgsXol33AUjZ4sNj3Go3w5ZctN+3d3xv/of7oaxq+nQUp5bjlJ2Goy+FyV3i9+kextpPp2/iMgxEvglt/mypXB1f6+4jyfVn7bOqhIdIPwMUF+hmOs3zwciaGZwDCSG0wQPTsGPS33Ym34jt+jJhJ1PHHOpRHsKP05NwpG/26GGPTMBN7uPaDka9S6y8SAp0oOGFVbTKa+ObCb/jj0s+ZPlWEm37/rZH0Helk7Mpotfl19//Jq1cfoKgInJ2hXHMKgP7e9f/HcYmBqasgbHE7nkymV2EyganhXZUkiVnRszg38lz0jTayM2aI35s3C0NyU/588E9W3LSixa7y8iAtxpmbX3qTqTElXeI1KdNLqM/vYzQaOXz4cJvmTzc3LLnUPvgAXn5ZvM5eXqJs1QrrpLgaFw1HxxdxIDyFVU7XyfnUZDqPyQTrzoFNc22qHB1hxAjwK5vNba7riPSMpLISKiok7vtzJ8oRT/8DA5bpNUgKmw2Jef70dvTmuhHXMcB3gM1t3x8VOYLOjToXT0dPUVh8CA4+JiIdZGS6ipwNcOhxkZOjCS4alzYfTppzadk1+plMsOl82HtHp4Yq0wdI/b7ZnH7QvvUnADG3wJh3ZcNMb0HjDg5edqu+OvQVt6+6veVo4zbgO8CXi76+yCoSS2ewEylasAvytlrtz2X6EI7+4DOeIC9xXtc40k9SSAy+6zyy+k9nxw7bW6uLqln/0HoOfXWoxS4KThTw9byvWf+yuC6ssR/2pJ9g1BvtGnK750+ZsxZzXj9zpJ8Zc7Tfli2Q1IIy8eyXZ3Px1xej0qosZUX7Uui360scUiqoMkKl2kcoOMjItIA5L/3eLDseEIh5zUVkOWPBAnB0kjDUGYiL0uHmJvL4ffst3HsvxKd5YdA4URL0f0jjPu6pR8BoNHL0aM+pLspGv38CtSvE3UVwuNCpO5J3hJSNKfxx5x/UlNSgr9XjP9Qf/2H+LF6qRJJg314Tn077jL+f/LvFpvW1ena8soOMjQlIEtx/P6SUiUi/ft79xEUad+HB4davWx9T5gwl8X34zhEKrCXAvr74a9Zfvd4i9QkwbJgwppSXY9ezLPdgLql/p2LUN7/Yy8+HKk0q/inwmH5wlz2GTC+gtkhIiZS1Lfm4mYULxbl3ymkTRcpjeI1dw3PPQfjh36l57V0yMxpC891C3flr8n5SolIYEBjexQ8g0yeRJPAaBR7D7Hr7T6yfQrdvF2fTBw6ACQUFhhE41HtKysh0CJNJ5PXL29yu274/Jox+SwctbSgs3AVHn4HK1GbukpHpANlr4ch/oSa39WtbwGz0G+g70LbSZBCytP1u61QfMn2A5E/h8H+6TrLJb7LIJynTeyhLEAa3Jnx+6HOW713OjnQ7VpZ2oNKq8IzyJGldElWFVVTWVeL4jCORr0dSWddIRu/Q/8HG2UDPRRrInGEY6oh1cgOsjX4g0geAOI+pa5K+T6VVsfO1nZz42TqQoCluoW7MevN89pbGYpCqWbiw0bzoMxZ8J3T2CWTOdpI/syvzOipwFApJgauDq1WagMhImDpV/Pvjj9v3qS1zDSZ18DwyQ07zayVsH/CSkDmW6duYTLB5EcQ/bLd6VOAoAPZm2zf6SZJIvzJ9ujD6xS2M485TdxI7K5KZM8U1v3xURN7WUzhpdFx3HfSbNBnCl9htrzcgG/3+QeJ84lBKSoprijm5/SR73tpD7qFc1I5q5i+fz+xXZhMYCFOmgEmhRD9rHhPvP8eqjV274JFHxCS7fz/sPaAifub9pA+azQ03iJDrk4XiQN0S6SdLkvRtnMIh4Nw25UFRKhs+5H/9ZVt/ybeXcMfJO1qUns3LNRFx/ATRidEEeER2dNQyvZHyU7DlIqTMlqNFmxIYKAwrxS7b+XvwIFZrryYg0IhvmCN1ju78/kM1AFl7syjK01OlSuNWd1hU8xnoyrvjSWT6GqPfgLHvCCmmJowdCwqF0JK/7jp44QVw0RQRHlL9DwxUplchSbDtUjj8pN1qg9HAxwc+ZsZnMyz5c08UnOBQ7iFUChUL4xY2XBxxBcw/Br7n2G1LRqZD9LsT5h0Bx2CbKpPJxOIfFhP1ehTppenNNqE36jlZIPYug/zsRPopVBB9PYS0lHxNRgYY+QrMOfD/7d13fBR1/sfx1+ymkw4koQRC6L1LUWlSRey9nb1hb6d3nvVOPcvv7OU82+nZFbsiiIAgooD03qQlAdJD+u78/hgJhLTdzWaXzb6fj0ceR2a+M/P5mtwns/Od7+fbcDsJXvNPgUUX1djcp7X1ourBFxAaY/2n6/nq2q8o3lfM1tytOEwH+aX51avf9LwDhjzn0/JicpT5oiuDN97HrItmMeuiWdV2de0KLVtCaSmsWFH9sNCoUK5ddS1nfVh/Fa/o5Gh2Jg0mJyyFDb0vZtwPEby67FVrjVJHmbd7I4Hot9th/b9qbE5PSGf/Hfv5/uLva1RsuOgiqyz2ypXwc+3LigOw9uO1vDriVYqzrUoQ+82W7EsbipnsICIkQmv6icUwIGdJnZMCDs70q6u8J8CECXDrrVYFpmrbxzsJC4OWu1fQ9dd3ufishfy7eDK3fXer18I/GtV8WiW+seJvhG9+kf4tu7Bs/wYqx1dy0wU3sfvX3TgdTmx2W1VCPe88a8bAsopBDCuBg8Mm27fDY49Zb/usWgUzDr6UERrJhBMjOflk69uN2X+U9/yj/i2/XAk7PobT9kBotM+6LEeJtpOsrzqsylpFq6hWVX94x42zfrd+/dWa8RcTc6htRHxEvZcqLYUD2SUMXFZCa3tvEs9P9koXpJmI6wXD38RMPAa2l7p16HXXQc/5Qzl/RRT7ivexZu8apv7fOB54wGDPYjg3q5S3J71NVFpLSqft5rgISNr3Ddjr/50VccvBVxoP+wAUE2Otc7BypVVCIioK7prwN/pHvQAleyBSH2qkEY55GSJqLm4OYDNsPLnoSdbuW8t7q9/j6iFX8/7q9wGYkD6BxMjDypiFREFczVKgIo3SIrXOXYZhsG7fOrblbWPV3lWkxtXedmvuVsocZUSGRJIWn9ZEgUpQiOvhvXOZJnyaag02D33ee+cV/+p1F1BzekqfJGvQr7HlPQHaDm5L26Ftie8Uz4JtCwDonNi5eqM2Exp9HQlwnS7CbgtnfPr4GrsMA4YPh6++gkWLYOgRS9S37NqywdOXlsKnn1r/jkrKoLygnNjwWMicA3Mnw7DXoPOlXuiIBKzjPoTwmr9LhmEcWh7gCMnJcNpp8MEH8Oqr1qST2pYrK8svY9/afexbsw+zQ0fy9pZBSDizzp9H7N6X4MAvwMiaB0rwOWVHnS/ADEgZgIFBflk+uSW5df5eHmnWnbPY8eMOnnr/Mor3DCB3cST7+u1j5oczec+YDYsLYdgr3uzFUUMz/bzMZnPxP2lYAsT2ZEjrrgBscGxg56KdfHT2Ryx4dEG1pu3awSWXWP9+9T8mO7c7KC6GRx6xBvx697ZGs1u1gvCibPq22c/VVzmr/n/y4JgH+evxfz20LkZcb2vB6hCtrSbV/WP+P+j3Uj8emv9Q1ba0NEhPt9b0+/HHmsfs/nU3s++ajdNRs8Tn/v3gCImg5IYfeOichQwJV813OUxoLKRfjC2uO3379nUTwS2zAABoJUlEQVQ9f2Kt7XfKSWEc3/F4AOZsm8PgwQZt2sCBA7BwcQjH33M8EZO7YBoOLs4MxTwzt2odQZFG2/sjfNkDMmfV2HXNNVZJib/8Bd56C/qPG2bNrKpjsEbEZR3PgeTRVd/abLaq/GkYBlcMvAKAV5ZZH1xqLe0JULQdyrJ9ErIEEdO0SneX59a6u19yPwBWZtW99lBBWQH9k/szsM1AbEYt9wVrHrVyb9FWr4QszZjphNJ9UHmg1t2H588GOUohOh3Cal8DTgJU+sWQ/qcamw/OMvbGoF+vM3tx5S9XEhIewpYca+GrzgmdGzhKgk7/v0Pfv9W5+2CJz8WLwVF9CXtM02TnTzvZ+OXGOo9/9eyZdPj8Odq1KqPYngFgveQdlgAdznF76R+38qcEhuQxEN+33ia5JbnVSnwCnHWWtSRQVhZ89tmh7b/9Bv/7n/Vspu/5fblj3x10OL4jr74Kg759lKHLXyEuDoxNL2FsfLYJOiQBqZ4Z79Fh0Wy6YRP5d+W7POAH4Kx0YjpNEiOK6T48geE3DSe7PJtwA7YSD1F1v7TobTabjd69a6lk0lTX89mVpLqet8KEHxnX60LuPu5uRqaOpPvJ3Tn+nuMZet3QGs2nTYMB7fbR86sneOHCn3jmGdizxxro+8tf4MYbrRKfZ7aeT/grz0P5oQWiL+h3AX8f93daRv3x1kbP22HMlyofEcxWPQBrH6ux+bgOxwHwn2X/4fe8Q+v8jBtn/e+MGbBpU/VjNny+kYX/XMiun3fXON/evYDNRljYvygfO51j+033WhekGXE6KD9ygQIXndDpBAC+3/Y9hgGTji+m6+K3mf23eXydM4IZWzvTYd+VDAo9D1tYnDejlmAX2dZ6sF3L2lWpqXDFFdYH9LAwrIdKI9/W313xnsM+cB+ePy/qfxFh9jCWZizl283fklGYQZg9jFN6HFEK8cfT4NshvopWgkXeKvi4JWx4ptbdrgz6DWk7hOXXLGfBpQtqb3Dw5Z2whmc2SJDb8RF8kgS7v6yzicv3nyGRMGE+9H+o4bYS8A6uJ5p1IIv9xfu9dt4tubUM+mXNg0+SYfu7XruOBK7vtnzH/XPvZ+GOhdW29+5tVRMpKIB162oe98kFn/DNjd/UGJABa6LAxm2hmLYQTj8vlIzCPwb9ottY6/kd955H5d49/fwuR7GKInA6amw2TZMT/nsCrR5vVbV81EEREdaSFmDN+Fu/Hv7xD7j3XnjvPXj4YSAkBHuYnaVLYflvJvs7DeHY6wdYB435EkZ/0bT9ksBRsAG2vV3nsjydEzvX/lJgPU54+AQu//lyHBUOKkut5c72F++nzIR/RU2Bvvc2Omx3+DJ3atDPy5xO92YyndPnHB4+4WFGpI4grEUY4x4aR2RCZI12hgHX3R1HZUwCmYUtWLgQQkLgrrusGS8H2wy8qBej7xtNeGzD67VJENv+Dmx7s8bm0WmjOaHTCVQ4K/j7/L9XbR8zBuLiIDMTbrsNnnkGNmyw3tz5cOsg1hx/Fe8uaM+Rv/5790JI2QHsYb1oM+gBWreq/80hCUK/XAMfxbJh/Wq38yfAuE7WiPS83+dR6axk4imRhJfkUZZfwvLlYCtqz4j9L/POcedqVoB4V0xnq0x2p5prwYg0mXVPwgexUGi9geN0OtmwYUNV/mwV1YrTepwGwBcbviDz9kwWXraQ+Ij46udJuxC6XuPLyCUYtOgIna+AhIG17nZl0O+gI9eNqdLzNjhpPehFHmlIXG/ocrX1e1mLI/OnBKH8dfBlrxovKkSHRdPpj7Xo1+xt/Lp+B1UN+h1Z3rNFeq1l9SSI7PwE5p/Odytf5YF5D/D9tu+r7Q4JsdYNB6vE5+EMw2DyM5M5/e3Taz31V1/B1rRx7D/rGgaPLKSk0lpnvDHrqCl/NkOr/wEfxkDB+hq7DMPAZthwmk6+2fRNjf2jR0OPHlYZ2TvusNb3s9msl19XroSXX7ZmqL72GmAYDPrbVCLPr2DYf4Zx47zHrGVfRAB+/8Baa7dws9dOGRIRgmEYfHXNVzwS8wgVJRXsO7APgFaRrbx2HVc4nU42HTmTpglp0M9fnBWw9p/WCLaL2nQM46RPr2B/h0EAXHYZdO9evU33ad0Zc/+Yqu+XZSxj9tbZ7D2w19pQeQB+vty6qZDgdcL3MOnXWnc9OPZBAF5f/npVCZK4OGugb+xYa3LBrFlw++3Wmzu7CuIoiWvDggUGL75YbfIB+/ZB+7XfwSOPUlaoBaKlFjHdIHk8dmexR4cPSBlAfEQ8BWUFLMtYRkyswZTPr6X7rSdx7bXW7+07r+2n6+4Ta53dKtIoNheWRt7+Lsweaz1YEmmsyDbQaphVtq4OVwyySnz+b9X/qHRWVi16Xk3P26DXn5sqSglWYXHWmhjtT651d98k6+WvDdkbKKuseV9omiZ5pXlNGaEEk/jecMxL0Gp448+VuxJW/x0KfPegRnwgvCWYlUDNlwwOruu3Ocd7Dx5rLe+ZPBomLYI2E712HQlARdtg92f0jI4HqFZ16aCDJT4XLoSdO6vv6z6tO6kjU2u8MJORAW//8cjx3HNhX6k1yy82PJao0ChY+zgsuaH6QxwJTgkDrJcCbbUsygdM6TIFgG821xz0Mwy46qpDRW369LGew9x5p7Xt22+tmX87d1ozVs8+21rD+bfdv7Bv3xJw6Fmh/KHDGXD8xxCdVuvuzKJMzv/4fI577bhaZzbXJyopivQJ6YRGhrK/eD9dQ+HMyuWQu7zRYR+tNOjnL0YIrH4Itr1JRmEGs7bMIru44bVVjjvOSqaXXgonnVR9X2l+aY111Z775TkmvDWBl5a8ZG0o3gVbX4PsX7zVEwlEUe0hJKrWXSNTRzK5y2QcpqPa2n6JiXDrrfDYY9C1q/W22dCh1sy/m29wEJ2zg+++LOOtt6z2GRmwZg3kJMeSPmY1O78f7IueSaDpeSvO42fgsHv2xr7dZmdM2hgAvt9qvRE5aqydG2+EE0+EjJCfKDEdcMwr0PE8b0Utcsjaf8KCc+ref2A75CyF8NY+C0masbTzYdwsiOtRZ5NxncbRJroN+WX5fLT2Ix8GJ1K/9rHtiY+Ip9JZyfr9Nd8k35i9kVaPtWLCWxNw1jawXVkCK/4KWT/4IFqRw2T/DCv/Bge2+TsS8aaIJJi2EbrfUGPXs1OeJfvObC4fdLlXLmWaJsd3PJ7h7YfTtWVXr5xTmpFuN8C5FYQlWcutbM/fXqPJwIFWla/sbLjuOqts4uETRkynSfamQ88UnU7417+gvMzJwLwfSHNsZlfBLgDaxbSzGu3+DHZ8qCUIBNpNhZFvQWzt+enEricCVoWlA+U118rt2hUefNAa3Hv4YejYEYYNg0susfav/KPIwwUXQIsWkFGYQbcweDdkEax5uCl6JIEorheknm6tN1qL2PBY3l/zPgt3LmRP4R63Tj3x8Ymc9eFZAOwv2U+fMDi2YC7kr21s1EctDfr5i2HAhIUw8l2mvjOViW9PZMGOOtauOEzGsgzMzz5nRKeMGn+Xv7nhG17o/QKleaVV2zZmW4v5dm/5x5TAmG5wZg70vNNrXZEAVFFkPYQuz61194NjrNl+b618iw37q9fs7tkT/u//4JNPrD/oY8ZA7OZl9PjpdWL3b+PDD+Haa63B6TVrYEsPB3HnfcTK3Jpr/okcZLfbPT729hG38/X5X3PDsOof2IvKixj75lgS/9WRXUmTrTdpRbwtbw3sWwjlebXv7303nJ4FEb4tHSHB48j8aTNsXD34auLC42p/AzJ3OXw/DnZ/5ZsAJbiseRTmnVLrLsMwGNF+BMe0O4biipoz/L/a9BUO01pLptb1Og5ssx4MZX5fc59IbRZfCb/VPavZ5fvPDufA5GXemTUoAaFjfEcSIxO9dj7DMHj9lNdZdPki2sa0PbRj3ROw6SWvXUcClD0MDBtp8WkAbM/bXqNJWBg88giMHGl9v2iR9VL24sXW9x+e9SEv9XuJipIKAL74wlr/L9ooxr5gPhu/2EBseCxn9jqTiZ3/mFl6wjw4cbVnITfi87sEnu4tu5MWn0a5o5wfttf+8tWAAdbEgMOfVZ92Gowfb/27fXuYNMn6d0ZRBoVOmB8xGFof17TBS+CpYxZfVGgUvVv3BmBpxlK3ThmZGElYC2smq4HBzGKY3etZaHti42J1ky9zpwt1qcQdbv3wEvoDVumI3zJ/Y/Xe1ZzSo/YPyQcV7y/mt1d/o+3QtrQZeKgGt2maJKQnUFFcQUR8RNX2g4usdm/1x6CfYdQ5Yi5BZNenVp3k4z+23qI4wtB2Q5nWbRqLdy+mqLyo1lMc/oe8y5QujHlgDF0SWvPed7Brl1XDe8AAONDjd07ZCCd2HUntVeYlqFUUYF/9D/rG94Oda2D/TzD4abfeNjy2Q+0Lj8/ZNodyRzmd4jsdeptRxNuGPA0hsWCr5+9/SM21ekU8tvYxwIBed2C32+nbt+Z6uX8d9Ve6JHZhUpdJNY8vybJe/KnIb/pYJfjkr4a9c8FRCvaIGru/vuDrOg/9apM1ED2169TaG8R0halrISTaG5FKMMj6ASKSa91VV/6sVVgcJNa+VqUEuL0LIOMb6HUXhMb4/vrr/w8i22ud3WDnKIX9i+lsrwRgR/4OHE4H9iM+X3ToAHffbZVJfOMN+OUXeOcda72/Hqf1IKFLApUllezNCeW//7WOufiaSAY/ei2hUaEktEvgw7M+PHRCm92jFxPdyp8SGJyVsOR6a6JIz1tr7DYMgyldpvDikhf5ZtM3nNTtpFpOUpNhwPTpVsnP3r2timFgDfrtqIQlbc5nVJsJ3uyJBLKKIvi8E7Q/BYb9p9YmQ9oOYdXeVbz626tM6zat7nXA6/HpuZ9SUFZAiC0EQmuvgtcU7HY7vXv39tn1NOjnZW7VlHVWQNF2BrbqylvAqr2rGjykw/EduC3zNqKTq3/YNQyj2lp+ADklOewv3g9A18Q/pmgXbYfKIojtDrZQ12OV5qXlMdDvIYituzzYU5OfItQWSmpcaoOnS+iUwOh7R2Oa0K6vtYDviBFQtmsvz1+8i649utJxSEdv9kCaC1s45rrHqUyZRghlGPkrofdfrXI7jSwz8vUm68Hik6kpGF/2gDFfQ0znBo4ScVNdL9KYpvVyRZvJ0OlC38Ykzdu2t6w1mnvdgWmaFBYWEhMTU+0DT4gthAv6XVD78W0nwVn5Wr9Fmsbw1z36jFFQVsCPv/8I1DPoZwuFuJ6NiU6CzdS11gyaWtSVP2tVuhfskf4ZFJKmtXeuNYO4/WnQ8tAauKZp8ufZf2ZF1grePPVNUqJTGnWZovIiIkIirAeMh5u0BCoLG3VuaQbK9sP3Y0jpdj0hthAqnZXsKdxT57OY1FS4+WardOLWrVaFpX4X9gOs27unHoHycqsk6OQT7RhGUs2TVJZAzq/WIE+ke7/fbuVPCQy2EPj9fSsP1jLoB1QN+n29+WtM03T5Zx8SAiecUH1bRqG1vmSb6Da1HCFBK6QFxPeH6Lqf200fOp3/rfofn2/4nEcXPMrdx9/t0aVizXIrD4ZEQG0VRpqAaZoUFBT45Fqg8p5e53TWsv5EXba8Bl9249hI66HL6r0NT6sPjQytGvArKyir+t/KssoabQ+W9mwf254WYS2sjev/D77uC6VZrscpzU9sN+hzj1UvuQ7pCenVbjLzSvMaPq9pcvyxDiZPhrg4KNhVgLHW5LyQCAZG+e7tCQkg9nCcUzewNuEenCPehomLYM54+Mm9QZLMokzumn0X531srdtnmmbVItPdWvexHhSGerZuoEiDinfDb3fC3vmHth34HXbOgOzF/otLmqfRn8FU60Uxp9PJ1q1b3bv/PEgPaaQpuDjgV+4or/b97K2zqXBW0DWxa93rXRXvqrM0vUit6hjwAzfz58Lz4NOGX4SUAJR+qVXeML5ftc2GYfD5hs/5bst3rMpq+OXshtw/934i/h7BQ/Meqr4jqq31QrYEt/AkGPwMto7nkxpr5Zrf83+v95CYGBg3zvr3p58e2j5zpsn69RARATfeCDsW/F717DCnJOfQmrlFm2H2aNj8stvhNur+U45eJ2+Bsd/VuXtcp3Gc0fMM/nLcX2pfe9kNGUUZXB0HE3Y/D6X7GnUuaUYMA06YbS2RUofBbQfz7JRnAfjrnL/y3Za6f2frtfJvMCMFyrIbbuslTqeT7du3++x6GvTzp1YjoOcdpKaMAKxSnEd+AK7Luk/W8UznZ9i5aCc/3PsDz3R+hvwd1cs0HVyLrVvLboc2tjsJet8DEY17U02CyyfrPqHjUx3rTabb523n8aTHWf3eocHrLpO6EPniu9w3fg1dIzSxWOoQnY5phFozplp0tMowhbtXZsThdPDYwsd4b/V7bM/bzrr969iRv4Nwezidhj8FU1drTTVpOo5SWPc47DisXE50GpyeCX3+5rewpJmKTrfegvTUjo8hc4734hE5XHmetV5kwcZad1c4Kuj7Yl9aPNyCfQcOPeT5aqNV2vPErvWsq/HLNfBJCjgd3oxYmrOSDNj1mTVTrzHaTYMuV3snJjm6RLWD+N61DhD3TrJKcP2086dGX2ZL7hYcpoOEyMMqRJRlQ9FWq6yeBDd7GHS/AVqP4MOzPmTj9RsZ1m5Yg4ed8sfqQL/8AhkZsOilFSw4/V+EFedy4YUQG1HOuye9y9uT3gZg4MsDifxHJMsylkF4a2tZjTaTm7JnEkjCE+t9KbBFWAs+Ovsjrhx8ZY3Ss+6KCo2iT7idlrk/gz28UeeS4HPV4Ku4YuAVmJi8ueJNt4595MdHGPvmWBaVR0DX6yAsvmmCPApo0M+fEvrBwMdIaTeemLAYKp2VVbPzGmIPtxMRH0Fsu1gSOieQ3DeZ2NTYam0Onqt7y8PeHGszEfo/ZE3dluD204Uwt47ySUf4cuOXFJQVcPGMiymtLK21TcuuLYlPiyfkj8G9g6Vu38gvZeAuJ1Ep470TtzR/J8y21klzQ7vYdoxJGwPAu6ve5ZtN1iy/MWljiPJhjW4JUjGdYfJS64MzQEkmOMqtMmARtZTTEWkMZyXkrYLCLZ4dv+wW+O1278YkclDBRph3Euz8pNbdofZQSipKqHRWVi1t4DSdfL3ZKsldZ2lPgPYnQ49b6l9DVeRwe76F+afC/p8bd54eN8PAf3ojIjkalefCgZ01Nk/pMgWARxY8wvLM5Y26xJYc629254TDSpb9/j583tlae1LkD4PbDqZry66E2hueOd++PQwebJX0/OILWPBLGJX2cDomFHLSSWALtTHx/yYy8o6RVDor2V2wm3JHuVVSMTIFut8IrRoeXJQgUZIJmbOtZQRcMHvrbHJKcjy61G9X/8b0ayrgnDIIUelsOcyuL+DX6VBZXG+z5058judPfJ7/nvrfqm2T3p7EVV9cVedza4Afd/zI3O1zWRHRHYY+36yXPtOg31HAMAz6JPUBXCvxCdBtajemr5tOXIc4ht0wjAu+uaBGPeUL+13Iqye/yvl9z/d6zNIMVB6w1nd0wYtTX6RDXAeyDmTx9sq3a20T0zaGq5ZcRe+zemOaJu+d/B5z7p/DprwMVpZD+1Z1lxIViYiIaPQ5LuhrrV/1zup3qh4eTukyBdY8Cjs/bfT5ReqVOOhQLfgfJsKCM60ZLyLeVrwLvu4HG54BPMifI96CAXp4LU0ktjsMe9UaoKtDv2SrjN7SPUsBqHRW8sCYBziz15mM6jiq7nN3uQoGPOrVcKWZSx4Nw9+AhIG17vbG/ac0A5+lw6KLa2y+bOBlTO06lTJHGWd/eDaFZZ6tvWeaJltztwLQOfGwQb+E/tDjVojr7dF5pZlZdAl83d/tw0491frfmTNh/t4erBkznav+3gG7HULCQxh0+SB6nt6TXQW7cJgOwuxhJEcnNzpc5c9maOvrMGcC5K9tsOl7q99j8tuTOemdkyiuqH9wpi6GYWDY7FpyQKrbtwA2vWAtoVKP8JBwrht6XdWs05KKEr7b8h2vLHuF/yz7T53HHXyJp3+y+/nWG3yZOzXo52V2u5tvnm57Cz5L556B5/Lqya8yMnWky4faQur/8fVs3ZPLBl7GcR2OszY4HfBVX1h5v3sxSvM0agaMn+dS0/CQcG4adhMATy56ssH63SXZJeT9nkfhzkLyb1rFuqsWN3rxc2m+7HY7PXr0OJQ/SzJg+V9gzzduneeMXmcQZg9j9d7VXDfkOp6a9BSndJsKK/4C22sfrBbxOtMJHc8DZzk4K/wdjTRHLTpYZWNTT62ZP12RPBraTGi6+CS4hcVB58vqXTf6YMmyv875K/+36P8IsYVw1eCr+PCsDwkPUYkn8aLodEj/E7SouR6fy/mzPBfmngRb3SsfJQGk+42QekaNzTbDxpunvkn72PZsytnE1V9eXVXNxh1ZB7I4UHEAm2EjLT7t0I7Wx8KgJ611/URCWkBoHDnF2Tw07yGu++o6lw7r3x86dIDycsAwmDQJevSAnM05HNh3aMbW73nWGoEd4zpiM2zW5+1vBkF5fh1nrptH959y9GszGYY8D1HtG2zaJ6kPMeExLNq1iLtm3+XZ9XZ9BjlLPTtWmq+et8Npe6xqSm6wGTbuPs5aC/Dxnx6nwlHzWczeA3vJKMrAwGDIjmfhtz97JWRX2e12unXr1nBDL9Ggn5e5vZCtLQxsoZyYOpjLBl5Gh7gOTRMYQGUBmBXg8OwtDAluVwy6gtjwWNbvX19VOvFIBbsK+PSST9ny3RauWnIVU56eQosFp9Nj6SXWjaVILZxOJ9nZ2Yfyp2nC2kdg1+dunSc+Ir6qLNive37lpuE3kZaQDieugv7/8HbYIrUzbNbC02O/hYjW/o5GmiPDBv0ehOSxNfNnQyqLrYFpET+6/pjrOb3n6VQ4K7jtu9uY9u60auv71WrvjzB7LGS59sKaSDW1DNS4nD9L90PmLCjc1ETBid/1ewC6X1/rrpZRLXn/zPexG3ZmbZ3F7sL6Zx7U5mBpzw5xHQirZe1AEcAqMzdhPoZh49659/LikhcpKCto8DDDODTbLzYWTh6ZzVfXfcV/hv2Hp9OepiS3BIDtedsB6Bjf8Y8jnVBRYC1H4Ca37z8lMCQOhG7XQWSbBpv2SepTVVbxnVXv1DrAUpevNn7FMa8MpXL+GZqUIjVFtLZ+B918hhweEs7fRv2N5BbJ7MjfwTur3qnRZkXmCgCrhPL+RZDzq1dCdpXT6SQnx7OSuJ7QU3gvc/vNrw5nw7QN1lteXrQyayVP//w0a/auObQxLAFOWg8DH/PqtSRAHdgJG5+HvDUNtwViw2O5erC1gP0Ti56otU1YdBgr31rJrp93YQ+zExYdBmkXQvol3opamiHTNNm5c+eh/BnZxhqoG/KM2+c6WM743dXvWjNSDRvE97bKjYmINDM18mdDVt0HH0TDgR1NG5gEtwVnw+dd6tzdIqwFH531ES9OfZFQWyiZRZnsKthV/zmLd0PuMjArvRysNHvfj4PvhtfY7HL+jO0K55RC3/uaKEA52o1MHcn/Tv8fy69eTvvYhmfAHGlLbi3r+ZkmfDcSVj3orTClmUiITKBdTDuA6s/z6nHCCXDVVXD//RBmq2DJi0uIbR/L8FuHE5kQCcDv+dZMv7S4NOugAY/CyZvdfrAOHtx/SrM0pesUklokkV2SzZxtc1w+bnPOZpbuWcKzIcdYazWLHM5ZCYWbrSUt3BQZGsktw63fqUcXPlqjSl210p6nbINxsxsdrjtM02TXLvf75SkN+vnbYbWLs4uzmf7VdH7YVnMhZ3f/mH689mNunnkzDy94uNEhSjNVtAWWXA9Z37t8yI3DbiTEFsLc7XPZlF3zbdeI+Ahuz7qdKc9Yi56/t/o9rtr2O1+GaD0/cYNhQHwfjxbUPanbSaREp3BMu2PIL823bhTKfPcmjYiIT+z6wlqDKMv1D9gAxHSDpDEQqVJi0oSiUq2Xber5/GIYBtcMuYalVy0lMiSSfy/9d/3nTDsXzsyD5LHejVWavxadINq9ElE1GIZH96USIHZ/Cd+fUO/LsOf0OYd2se08On372Pac1+c8JqQfVlq7It9a0qA006NzSjOUv9Zai75wM32S+gCweu9qlw612WDaNOjaFZL7J3P9huu5evnVjHtoXFWbmjP9RI5gOuHrAfDz5S41D7GFcEZPqzTyB2s+cPkyGUUZOIHfY4+BlHENtpcgU5IBX3SFtY97dPi1Q68lLjyO9fvX8+n6T6vtW561HIABKQOsDc28Il3z7l2g2P8LLL2Zp+fdwwtLXuDmmTdT6Tz0FuucbXOIfTSWv37/V5dPOe93q/TNmI5jDm3M/hU2vwJl2d6KXAJZwkAY9z10PNflQ9rHtuflk15m7XVr6dqya61tolpFVf37+63f88qyV1i6R3W6xU2VxbB/sdv5KiIkgtdOfo1BKYOIi4iDVQ/Ax62sGwcRkeYiNBZCosBR6t5xXa6EsV+DLaRp4hIBa42qMV9Ve7mxLn2T+7LgsgW8eNKLDZ/XMJr9h3NpAsNfhWNrlnhyWcFGyPgOKgq9F5McXcpzrWclpQ1/XjBNs9aXX+szrtM43jnjHf583GFrB4XFW7MMhjzvZrDSbOWtghV3Q84ytwf9DmcYBi27tcQ44m/wsHbDOKPnGQxuMxicDlj7T8ia643IpbkwbNbakvYIlw85p/c5AHyy/hPKHeUuHZNRZOXaNtENlxGVIBTZxlrXr81Ejw6PDY/l+mOskt0v/PpCtX0xYTEktUhicOtu1nJCRVsbHe7RTJ+ajgZ5K2DD09zedSQJEQmszFrJf5b9p2p395bdKSov4uEFD7N239oGT1daWcrPu34GYEzamEM7ds6AX66C0gbWzJDgEBZnvVUTkeTWYZcNvIyerXs22M7hdFCaMZtv2sKw0DJPo5QgERNzxFoCO2dYpZgyZrl9rildp3D38Xdb60imjIeu17pUl15EJGAkj4apq6HtiTXzp0hztPUN2PeTv6OQZsal/Ln9HfhhEhzY3uTxiJ+kXQBnF1ifG+pRXFHMwJcH0v257uzM3+mda7vwcoQEieQTYNIv0GbSoUG/fe4P+tXl6iFX89HZHzGl6xQo2wfL74IdH3p8Pt1/NlMTF1rrS7rouA7HkRKdQovQFmzNdW0AJaMwg9Oj4cY9j8CebzyNVJorWwgMfBzaTfX4FDcNu4l/jv8nn5zzSbXtL530Elm3ZzGxdTrMPwW2v9vYaN0WHR3ts2tp0M/L7Ha7+welng4nbSC284U8ONaq6X7PnHvILckFoF1sO07ufjIAt393e4On+3nXz5Q5ymgb05YuiYetp9H1ahjzNUSnuR+jNE+m2ajSh3sP7K1z31sr36KiaDujIuHYdgM9voY0f3a7nc6dO1fPn0nHQb+/Q0L/Q9t+f98qeeKOjue4ddMqIhJIas2fdclbDQvOhb0/Nn1gEtwKN8OKv1rVTLzBUQ6LL4f1T3rnfBJcCjZaD7eP+H10OX+2PwWGvgQtVBKv2XJxBnFUaBTxEfGYmPx3xX9dPv3veb9Xq+QEQOZsa0DZoZdj5Q8RraDlUAiLqxr0W5W1qmmuFZYIExdD95s8Otyt+09p1uw2O4suX8SOW3bQo1UPl47JKMqg1AllEe2s30URL2vdojV3HnsnseGxte43WnSA4W9Cu2k+jctut5Oenu6z62nQz8ucTmfDjY4U3hJiu8Efa1v0at2L7JJsBrw8oGotvycnPkmoLZRvNn/DzM0z6z3d3O1zARjdcXT1Kf0tOkLbKW5N1ZZmbs54+LJbze15a2DuSVCeV+thFY4KLvn0ElL/lVpreZOSihLumXMP7xfB82n/JKbjGV4OXJoTp9NJZmZm9fzZoiP0+SvEHTartLLIKnmSv961E2thcRFpzvbMxFx2O5kZe1y7/8xfCzvet97uFmlKJRmw5mHY76WZeYYBY2dBzzu9cz4JLqVZVhm7fQuqba71/rM2iQOtl2dDa39wJM3E7q+tgbgGXDrgUgBeW/4aTrPhv70FZQWkPZ1G1D+iKCovOrRj4/Ow6GKVLJbqKouhPJeerXpiYJBXmkdOSePXpy+rLCOzKPPQ76w9DFodYz2H9IDL+VMCT/avsOohKMly+ZC0+DSrypKL9hTu4eti2DX8fWg1zJMopblb/Q/4dqhViriRHE4Hi3ctrv7yTUQrSL8YEvo1+vzucDqdZGW5/v+txtIdhpeZnj5kriiCzO8JcZbz1KSnANiRv4M/ffonALokduGGY24A4Lbvbqv2y/rr7l+rXbdqPb/DS3sevIYegsvh2p4IHc6umUj3/QgZ38LeebUeFmoPZX/xfsod5Tww74Ea+59e/DS7C3fTIa4DNwy/UR9mpF6maZKZmVl7/jx8W1xv6HMvhLv4NtjiK2D2aL1BKyLNU8a3GOufJGfnMtfuPzueDWcVWn/7RZpS4mCYutZaQ9IbbKFWSXo9GBJPJA6Gk9ZDt+uqba73/lOCz8+XwMr7Gmx2Zq8ziQ6LZmvuVn78veGZ81tytgAQHxFPdNhhJb36PwyjPrXymwhYz2Q+jIHFV9AirAUbrt/Agb8cIDGy8TOhluxZQpsn29Dr+V7Whooi99eFPozyZzO27ydYdS8UuPii9WEqnZVkF2fX2+ZA+QEGpAwgzB5GxzjNoJc6lO2zviryG3Wa/NJ8Rr42kuNfP54T/3ciqf9Krbacmq+ZpqlBv6C0+WVr1tX+n5jQeQJn9z4bm2FjcpfJVU3uGXUPLSNbsmbfmmq/pPfNvY9/LvwnYM3AWpaxDKhl0G9GW5jn26mrcpTreRsMfQFsdmtw5eDgX5er4cRVENEGltwIlQdqHHqwFO07q95hzd41VdtzSnJ4ZMEjTIqCz/ocQ4SzxCddkWZo/b/gk+RD6w20Gg79HnBjHUrTKglmD2/SMEVE/KLHLTimbqA8tK3rx4RGq+KDNL2QKGumfkgL75yvshhcmFEjUquQKIjt7nnum3UczNVn6GbvmH/DgIaXEWgR1oJze58LWLP9GrIl1xr065zYufqOuJ7Q7iT345Tmy2a3nsMkjQWga8uuhNq9Myj8e/7vALSJ+WOd+3VPwPuRVoUnkcN1PBsmL7NKzbrhgzUf0ObJNtw88+Z627UIa8H3F3/P3tMeJmbry40IVJq1wU/BKdtdf+G/DrHhsSS3SKbCWcGsrbPYVbCLcHs4bHgGPu1oLX/RjGnQ72jRdqq1UGWMNb3+7dPeZvetuzm/7/lVTRIiE7h/zP0AtI9tD8Cm7E20CGvBfXPvY92+dYTaQ8m4LYM5F8+ha2LXQ+d3VkKHMyBptM+6JAHENGHpTfDTBeCssMooxfWE/Qth47O1rv8zqM0gzuh5BiYm98099FZkQkQC/5n2H+5t347++z+zBl1EPBESYw3wrf0nrH7I+t0Ea+Ze0RGLRDsra85kHv4aTPRSaTERkaNNiw4Q3dn12fS7PoOCDU0bk8hBZdnW2n7esOwW+CDaOqeIJ8qyIc/FtbEKNsL342HJH2tdhbdq9EMnCQCpp0LS8S41vWzgZQB8uOZD8kvrn4VwcKZf54TDBv0c5dbLDCJHGvoCdL/e66fdnrcd4NDMqvje0PE8iHLjxTEJDpFtrLLWIVFuHdY+tj37i/fz2frPqpcyrkPcrg+tl7xFmpBhGLww9QViwmKqtvVP6Q/2KIhoDSHR9Rwd+DTo52XV1tBzR1wP6Hm79QAHq3xiSnRKjWZXD76aiZ0nsrtgN2CV/TxQfoByRzmXf345DqeDqNAoxnYaWz0WWwgMfx163eFZfNI8FW6BhRfAzo+t9VdKsw4NrACkXQhT10CbSbUe/sCYBzAw+Hjdx5z87smUO8oxDIOzep/FyHO2YYz/ESKTfdQZCVSGYZCYmFgzf3a5AqauhtOz4LgPrfI3Tgd81Rt+POPQIF9FIXzeCRZfVnPgz9OcLCISAIzyHJLD9zd8/1l5AOafBivv9U1gInPGw5yJ3jlXXB9ImQBhGngRDy08F74bWe0+sc77z4gkiOkCuVb1HEZ9CiPe9F2s4j+m6dJyKMPbD6dX616UVJbw/pr362yXV5rHiqwVwBGDfnvnwgctYOsbjQxYmrMN+zfwp0//xGWfXdboc/2eZ830qxr063AWHPsOhCV4dL4686c0D5UHoHSvW4cMbz+cbi27UVheyOu/vV5rm8W7FpNZlGl9c/zHMPa7xkYqzVVZDmx7C3KWNfpU7WPb8/iExwGICImgR6se1rPGyUsgOq3R53eHYRgkJHiWdz2hQT8vs9ka+Z/UdNZ7oxlqD2XmhTO5esjVgPUL8/JJLxMTFsOiXYt49pdnG3d9CS6mE35/x5rSfOy7MObL6m/0RLSGuF51Dpz0TupdNRv1i41fsGjnokM7baFae0VcYrPZ6NChQ935MyLJKssEVtmT9Esh7QIw/yhHGxoD0el/zPb7o/zXmkdg3f9pHVMRadZs806kzarTsDkamjFgg5H/g67X+CQuEdIv996aft1vgNGf6UUe8Vyni6H33WBWVm2q8/4zLB6OeQnG1762uTRTm/9jzSjOXtxgU8MweGLCE3x+7udcOajuPPfK0ld4d/W7wBHlPUPjoOO5ENur0WFLM7P5FZh/KjjKqXRW8t8V/+WjtR81eu287fnbAUiLT2t0iODC53cJXKYJH7WEn90bbLYZNm4edjMATy9+GsfBpYP+4DSdXDjjQjr8qwOzt86GqHbWjFOR2pRmwaKLYfcXXjndlYOv5PEJj/PfU/9LmD3MK+f0hM1mIzU11XfX89mVgoTT2Yj1JjY+Dx/Gu10KJzUutWrU+paZt3Dqe6dSUnHEOmoZs6ykne/+YqzSjMV0hrOLoN/91mzQ2tZecZRD5mw4sKPWUzw58UmuGXwN/xz/T9IT0qGyxJpJULS9SUOX5sPpdLJjxw7X82efv1ozo0syDpXGOeEHGPnWofUpNz4Hv7+rB4Qi0qw5e91FTse7cTb0LCgkEtLOg+SxPolLhO7XW4MsIkeDThdB779YLyX+odb7z/2/QMUfZclcLZ0szUNUe2h9HBh2l5pP6TqFad2nVZvpZJomBWUFVd9HhEQQExZDz1Y9mdj5sJnPrYZZL9y2OsZr4UszkbcKdn8J5dnWmn62UArLC9lZsLNRp62a6Rf/x0y/ny621vXzkNuf3yVwGAZ0mw5tJrt96MX9LyYhIoEtuVv4YmP1wZrZW2ezOWczkaGRDG87BIq2Wc8ORWrToiMMfAJ6eeezhM2wcfvI2zmr91lWpbAlN0DWXK+c2x1Op5OdOxuXz92hO1kva9QbOAdrJ1c2XP/4SFcOvrKqZMRnGz4jIuSIhcr3L4KtrwP6oyyHMWy1D/Qdbt98mDMBtr9d6+7k6GRePOlF7jz2TlLjUq03MVY/BLs+9X680iyZpklOTo57+dNZCb9eAz9Msmb3HXww46yETS/C1LXWh2kRkWbMbHcqO8JPxLRHNtBQ938SoAo2WqXos37wdyTSzNS4/6wsgXlT4fsxfo1L/KTtZBg3E1oOdfvQovIiSitLeWfVO3R7thsz1s0A4IZhN1BwdwFrp6+tdekWkRoGPg7nlkNkG8LsYXRvZVW7Wb13tcenNE2T3/OtQb+0+DTr8/KO92F/w7Na6zun25/fJXAMetKjtSVbhLXgmiFWVZF//Vx9vb7nf30egEv6X0J0RTZ8ng6r7mt8rNI8hURBz9ugKWbl5a20Jgm4MLPf20zTJDc312fX06Df0ST1dKuMSOJAtw+1GTZmXTSL4e2H8/Tkp2vW1u57L5yxH2K6eylYCRqtR0H/RyD1DNfadzgTxs6E9D81bVwS3A78bv2xTr+s+pvYm16CJdOtNTJiuvgtPBERnyrPhdJ9de//8XT4socG/8R39s6H78c1frCuYL1Vir54j3fikuBUuAVmj7ZK59XFsEG/v0OPW30XlwS8fy/9N2lPpfGP+f/gpm9vIutAFuv2r6v/oEWXwNrHfBKfBBh7eLXPtn2T+gKNG/Qrd5Qzfeh0zuh5Bu1j21sVns4pgRFvNDZakRqmD51OiC2E+b/PZ2P2RgAW7VzElxu/BODaodeCPRJ63KYKJNKwiiJYeivkrvTeOVuNhJO3QadLvHfOo1SIvwMQ7+mU0IlFly+qu0F4S98FI82HPQx63+V6e8MGbSY23E6kMWI6w8nbrXKeh+tyFZgVWrdKRIJGeNlWbJ8Nh+43wsA6HiK26ATYVK5OfMdRBjnLoCSzcedpfzKcVajfXWmckChrDfOy/XW3sYdD16t9F5McfVbeB+GtrHVEXeQ0nWSXZPP3H/8OQP/k/twx8o56DqiAnR9Bpful8yQIlOdDzlJrvfroNPok9QFgeeZyj08ZHhLOExOPKOXpSsUnCV67Prdeph78tPW8ZeubkDAQUk9t8NB2se14+aSXGd5+OCnRKdz4zY0898tzmJhM7jKZHq16WA0HeV5eVoJI4SbY+DQ4SuCYF71zTsOA6DTvnOsop09PXlZjhp27dnxsvfl1xKKnjeIot5J28W7vnVOCj7Oi/lkEAMW7IHe5VTJCxEWGYZCSkuJ+/jxywA+sQeoet1gPbkREmjnDMEjoMBzanwathtfdcPC/YNQnvgtMJGU8nJVnrSXZWKHR1qCNiKci28CZ2dXWmax2/1m0Fcqy/RigHBW2vArb3nLrkEsHXEq7mHYA2A07r53yGqH20LoPsIXCWQUw/PXGRCrNVf5qmHMC7LTu2Y7rcBwAH6/7mK25W71zjQM7rbWsyvM9PoXHn98lMJRmQdYcKNkNRdth9YNgc73M4mUDL6NX617klOTw6m+vYmJyYb8Lefu02pcMEqlT4kAYPx+GPu+9c+5bBKX1vATWhAzDIDk52WfX06Cfl9lsjfxPmv0zbHvT+uDhLYUbYf4p1jpXIp6oKISPk2DpTfW32/YWfDPQWkNSxEU2m42UlJTG508RkSBjs9lIadMW47h3rTLxIkcLbz0IzPoB8td651wih6m6/zQMWHA2fJYG5Xn+Dkv8acKPcML3bh0SHhLO38dZs/z+NupvDGozqOGDDBuExngSoTR3Md1hyHPWizPA8R2OZ0L6BAamDKS4otijU+7M30lGYQbOgyXed38B34+1ZhR6SJ/fm7n0S6wSsMljIGk0nDDHWvcUoGgbbHNt8C4tPo3nT3ye2RfN5q3T3qJl1B/V57a8BvNO1sQUcU3rY71X8cNRDt+Php/9sxyVzWbz6aCfYWrlVa8oKCggLi6OnJwcEhISPD9R6V7rDYqweK/FRul+2P0ZJAyAxMHeO68El58vh/g+1iyquuz/BXZ9Cn3+BiGRPgtNApvD4WD79u2kpaVht9cye09ERGpVI39WFEBobPVGexfArhnQ5RqI7eqfQCU4ZcyyyvG0P9nzc3ycBDFdYeJC78UlwWn/Ysj+FbpNB8Oonj93fwIlGVaZZBEPZBdnH3qgXZ/cFdY6vK1GqDKJuCS3JJf4iHiPZtXtKtjFMa8cQ0ZRBg+NfYh7Rt1jlTrO+gE6nA2Rnj181uf3IDb/dOuZ3ynboUUHz86x/G5Y9wScusvj30EJMvsWwfb/Qa8/Q4tUz89TWQxb/gNR7f3ywqzD4WDVqlUMHDiQ/Px8YmNjGz6oEbSm39EmIqkJztkKOl/u/fNKcBn+asNtWh1jfYm4qbCw0N8hiIgEpKr8ueZhWP0QnLobwhMPNdg7F9b/H3T0QplFEXcsuwWc5Z4P+pkmDPo/rTsk3rH1ddj8MnQ4C0p2Yax9kpKIKyAtzdomUlFoVVyK6eJ23nFpwA9g43PWA8fT92nQT1ySEFn/pIK80jzC7GFEhVYvg51TksOktyeRUZRB95bduXbItdaO+D7WVyPp83uQ6vNX6HIlhLf2/BwDHoH+/wBUHlZcVLAONj1vzYJuzKBfSJTfX/AqKiry2bU06Hc0KtoGhZuhzQTvnM80vVdiR6QuptN7U65FRETEPREpkDwOKvKqD/r1uhvSzofIdn4LTYLUwCettXY9ZRjQ6ULvxSPBrdt06HgOhMVB7m/YdrxLYlIrYLS/I5OjxZZXrZcVxs+DpFFNc43Ol0N8fwh3cZBQgs+ciWCEwNivq20uKCvg0QWP0i+5H+f2OReAudvncvGMi5nWbRrPT7XWvHrh1xdYkbmCVXtXsXbfWtrFtGPmhTNdH5gWqY+3qsfp2aG4I/V0a8AvqhEDfkFIg35Ho1+ugn0LrQWebV74Ec0cCuGtYOy3jT+XBC9HOSy6EFqkwcDHau7f9jasvAeOfdequSwiIiK+0/ky6+tINjtEp/s+HpG2kxp3vF5cFG+K73vo30mjcZySwd5NmaiwmFRJOh763AeRbZvuGq2GW18idQlpYQ36HeGVpa/wyIJHMDC49LNLsRt2DlQcAGDW1lkUlhVS5ijjjll3VK3/Fx8Rz7cXfkvH+I6HTjR7rFUKfvRnPumONEOmCcU7PSvv6XRYM++TRkFsN+/HJs1TWLx3lkGbO9WasDL2m8afKwBoaP0Izz//PGlpaURERDBs2DB++eUXt473pM52Dd1vgmP+Daaj8ecCaNEJojystSxykD0McpdD4cba9xt2CI2zaiOLuMEwDFJTU72TP0VEgkiD+dN0wt75UJbt28BEDjJN6wGPJ1beC593hgM7vBuTSEgkRmSS7j+lusTB0O9+q7xnUzBN60ukPqNmwPEf1tg8/Zjp9GzVExOT0srSqgG/KwZewbKrlxETHkOrqFZ8fu7ntItpR0xYDF+e9yV9ko4o5RnSwipx1wj6/B7kltwAn3WE0n3uH1u4AX650ipzLOKO0n2w98fGncMW7tfS2oZh0L69756ZG6apu46D3n//fS6++GJeeuklhg0bxlNPPcWHH37Ihg0bSEqqf629goIC4uLifLIQo4jfOMq09oCIiMjRatOLkLcKhr5gfX9gh/WhvNsNMOQZ/8YmwWfnDPjpfBj5DqSe5v7x656AbW/BpF90/ykigW/907DqPhj0pFXmU8RNlc5KMosycTgdOEwHLUJbkBxdc75yhaOC4opi4iLi/BClNHs7Z0DmbOh9t/sv/VcUQtYcqwrJ4TPwRRqy4BzY8aFVFTE02t/ReMyX40ca9DvMsGHDGDp0KM899xwATqeT1NRUbrjhBu66665qbcvKyigrK6v6vqCggNTUVPbt20dCgrXQrmEY2Gw2nE4nh/9nPrjd4aj+1muN7c5KsIVgs9kwDKNGe5vNVhWnK9vtdjumada6/cgY69re6D4dFqP6pD6pT+qTYRiYpsmGDRvo0qULdru9WfSpOf6c1Cf1SX06+vrkcDjYuHFjVf40fjoX2+7PMc/IxWkLh7L9GNvfwowfgL3t+IDoU23bA/3nFLR9ylmCbdV9GL3uwJk0tnn0qZ7Y1afA6lNFRQWbNm06lD+bQZ+a48/J532afxK06Ihz8PPe71P+WuxLrsF57IeYEYcGavRzUp+qxb7vZ4ys7zHTr4CI1kdln8rLy9m8eXNV/gzKn5P6pD6pT77t056vsRdvx+z0J5z2FgHZJ4fDwbJlyxg2bJhPBv20pt8fysvLWbp0KXfffXfVNpvNxvjx41m0aFGN9o888ggPPPBAje3r1q0jJiYGgMTERDp06MCuXbvIycmpapOSkkJKSgrbt2+nsLCwantqaiotW7Zk06ZNtF9/AYZZwaZO/yU9PZ3Y2FjWrl1b7Zepe/fuhIWFsWrVqmox9O3bl/LycjZs2EBcwRwiS9eR0/pieg08nsLCQrZu3VrVNiIigh49epCbm8vOnTurtsfExNC5c2f27t1LZmZm1fbG9Km0tLRqe2P6dJDdbqdv377qky/7ZDpIyP+GNm1TsXe+sKpPUSWrScz7nIRhf6M8sktg9ak5/pwCrE/x8fHs3buXsrIyDMNoFn1qjj8n9Ul9Up+Ovj6VlZWRkZFRlT/DW0yn51lvUXignK1bD5bjHk9EQQQ92hIQfWqOP6fg7VM4EW2fokdKD3Kzs5tJn5rjzyk4+7Rv375q+bM59Kk5/px83ad+xbsx7FFe69OuHVsJrdhLeVg7q08TFrA3K4vMTYfOr5+T+nR4nzJWzaB91uNsLkiiqMUxXu1TwcoXKd6/gX2J52HaIj3u0+bNm6vlz2D8OalPnvXJ7ijAYY9tVn1qjj+no7NPafTtO43CggK2bj3U3tU+xed/R1jFLhydryO1Uw+/fXY//L9vU9NMvz/s2bOHdu3a8dNPPzFixIiq7XfeeSfz5s1j8eLF1do39Uw/45erwFmOOfyNRo2IG4svxbb9LRyn7cce2bJ5jfLTDN9cONr7ZJrYPk+FiCSYsrwqRmP9k9hW/Blz/AJoPTKw+nTYdmgmP6cA65NpmqxcuZLevXtrpp/6pD6pT+qTm28Lrlq1qlr+DPQ+1bZdfQrCPhXtgPX/wuxwFrQc1jz61Bx/TgHcp4qKClavXl2VP5tDn5rjzynQ+8Tqv2NseRnn+EUQ1a5Z9Kk5/pyOqj4d2A0lGRDbE0Iivdonc/YJkLMY52nZYAvxuE/l5eWsWbOmKn8G5c8p2Pu06SXY+ALOE+ZDaIxrfaosw/ZJAmb702Hk20dfn5rjz0l9qtpumz8Nsr7HPCMfW2iE3z67L1q0iFGjRmmm39EsPDyc8PCaa0vY7faqhy4HHfyB19a2zu0jXj20IWsetEjFHp3u/nmG/Rt6/Rl7ZEvA+mWtrX1dMbq7vd5Ymmi7+uTjPh33AbToCIfH2OsO6HgWRmS76tsPc1T3ycPt6pN3+uRwOKque+T+QO1TU29Xn9Qnb8Xo7nb16ejqk2EYNfNn7nKMikLsScfD/NPBUQJjv/Eodv2c1Cd3t9fo09rHwFGCre99tZ6jzj7lr4SNT0F8b0ga6ZPY69oeFD+nBmJsrn2q7f4z0Pvkznb1yQd9ikqBuN7Yo5LhjzYB36daqE9ejD26PUTXXCfNG30yxnwBhZuwh4a71L6+2GvLn0H1cwr2PlUWQ0Uu9tKdENG3wRhtNhuYJdDpQozEwfBHu6OqT83x59Tc+vTDiRiY2P/4bOtKjFXbj3sfCjdjhEbU294Xn919RYN+f2jVqhV2u52srKxq27OyskhJSXH5PHX90njMWQk/XwqlmTBpsfsLndojrA/LIt6SdHzNbYYB0Z18H4s0CzabjfT0dO/nTxGRZq7W/DlvGoS1hBOX+y0ukSo7PoKS3VDHoF+d2kyGaZsgLKFp4pKgp/tPqVVFAWx7G6I7Q9tJjT9fl6usLxF3VJZA2T5o0cG75w2JgoT+jT6N8qfQ/UbocYv1LNBV4Ykw7D9NF5M0fyFRgIcFK0NjIHGgV8Nxl81mIy0tzWfXU3nPwwwbNoxjjjmGZ599FrCmYnbo0IHrr7+eu+66q95jCwoKiIuL8/70TNOErB+smVUxnd07tqIQCjZAXE8IadFwexFXledCSRbEdoOtb1jlPpNGQWjTTk0WERGRBmx72/p73P5kf0ciAsW7ITQOQqP9HYmISMPK8+DjltD+VDj+Y39HI8Hqq97grIBpGxtu66r8dWA6reeDhgbrRCSI7F1gPbeO7ebvSJpu/KgWyvSHufXWW3nllVd48803WbduHddeey0HDhzg0ksvdfkcR9ZybTTDgJRx7g/4Aez7CWYOha1vejcmCW6mCZ93gcWXAQZsfR1+vgSKtvs5MAlUB9ek8nr+FBFp5mrNn50u1ICfHD2i2nk24Je7Ekr3ez8ekT/o/lNqFRYP42Z7ZzZKWTYsvAB2fdb4c0lw6XQJpF3g3XOu+Qd83cd6gbuRlD8FgP2/wPK/gKPctfY/XQQr723amERqs+Q6mHUsOP2bsxwOB2vWrPHZ9VTe8zDnnHMO+/bt49577yUzM5MBAwbw7bffkpyc7O/QLLs+h43PwugvrLKdDYlOh34PQdLopo9NgodhQK8/Wx+IDANGvAlhidb3Ih7SBwYREc/UmT+z5kH2L9DpIoh0vVS9iFdVHoDCTRCVCuEtXT9uzjiI7QETFjRdbBL0dP8ptUoe653zFG2D39+xlmhpf4p3zinBodcd3j9npz9BXG/3/hbXQ/lTyPgW1j4C7aZB6xH1t3VWQtb3UHmMb2KT5qk8Hza9YH1GSD3NtWNME/o/apVMttW+/p4v+TJ3atDvCNdffz3XX3+9v8OoXeEm2P8z5C6HVsMPba8sBkdJ9T/eToc16NfnHp+HKUGg152H/h2d7r84REREpLryXPi6PxTvtL5vN02DfuI/e76FBWfCiLehk4uzFpwO6HUXhLdq2thEROpSvAsqiiCuh+fnaDkEzi4CU4MjchRoM8H6EvGW9Eut6iJxfRtuawuBU3dbL4OJeMoWCiv+CqlnuD7oZxjQ7sSmjesopfKegaTrtXDK9uoDfgA7PoBPUmD319b3mbPhsw6Q+Z3PQxQRERERPwqNt17I6XGbNUvKkxLxIt6SOBD6PmDNdHGVzQ49b4f0S5osLBGROlUUwWdpsOLuxp8rpIXWvRf3FW2DuVNhy+veOZ+fS9pJM9UiFRIGuD57yjC0xrM0TkgUTP4Vhr/m+jHFe5ounqOcBv28zGZrwv+kIVHWbL7SfTB3GhRusbZHtYc2Ew8NBkZ3tsp/VhY3XSwiIl5is9no3r170+ZPEZFmqNb8aRgwfi4MegJaH2u9ESniL9Hp0PdeSOjn70hEqtH9p9QpNBr63GeVx26MnGVWlSbT9EpYEkTskZD5PRzY7p3zrbgbPkv32sNv5U+pUnnAGqRuyK7PrOoPyofSWImDITTGtbZlOdakqF+nN21MLrLZbHTt2tVn11N5z0CUOQv2/wR7voLuN0LKeOvroOhOMG2z9dBHRCQAhIWF+TsEEZGAVGv+dFZaH8CjO1nldEQCyaqHIHMmHD8DIlr7OxppxnT/KXXq+7fGn+O32yFnKZyZ1/hzSXCJSIazD3hv/anwltaXF8u9K38KAN+NhIpCOGVr/e2W3wXOcjh5i2/ikubLWQl7vobt/4Pj3q+/raME0i+DpNG+ic0Fvsydei3Dy5xOZ9NfJHEonPC9NeBXFw34iUiAcDqdrFq1yjf5U0SkGakzf+77Eb7sBnODc/0COcr8dBHMO9n19pWFULQVwhKaLiYJerr/FJc0pixi95tgwKN6NiPuMwzvDfgB9PqzVRLP8M4jYOVPqZJ+qWvl2Ie/CUOeb/JwJAjkLofFl0HhZuv7smz45RrIXVmzbVQ7GPZv6Hi2T0Osi9PpZM2aNT67nl79DUSxvpsKKiIiIiIBJmEgtJsG/R7ydyQiUJEP5Xmutx/4mPUlIuIvTgd8NwKi2sKoTz07R/tTvBqSBJn8dbBvIaSdby31I3I06nGza+1aHdOkYUgQaTkEzth/6PsDO2DLK2CPgsH/d2j7wVKyQfzijWb6iYiIiIg0J2HxMPpzSOjv70hErN/FCfP9HYWIiOtsdmvALzzJ35FIsNrxAfxyJRRsaNx5trwOS2+x1rYS8YeybKsko0hTSBwIExfDoCerb19yA3zdB/YG72cQDfqJiIiIiIiI/xXvgnVPQv56f0ciIsFu1KdWWTBPZP0An6XDrs+8GpIEkQ7nwOgvIDq9cefZ8xVs/jeEtPBOXCKHK9gIcybBtrfrbvPrtfBRPFSW+CwsCTIthxya0XewLHfrYyEixaqAE6RU3tPLbDaNo4qIuMNms9G3b1/lTxERNyl/SkA48Dvs/gpSxkNst/rb7l8Mv90OEUkQ18M38UlQUv6UJuUoh9AYCIn2dyQSqOJ6eOfv4HEfQvEOsIc3/lx/UP6UKvZway3x5DF1t2k1AkJiICTSZ2FJEKoshgVnW+v4HfMypJ0HHc/x2lqm3mCz2ejdu7fPrqdBPxER8bvy8nIiIiL8HYaISMBR/pSjXt4aWDIdjnml4UG/NhNg/HyIaaCdiBcof0qDNr0Me762Zv25sy5Q20nWl0hjVRY3bk0/w4AWHb0Xzx+UPwWAqA5w9oH682OPW3wXjwQveyQ4Sq2XDU3T+p08igb8DiovL/fZtY6+3gc4p9Pp7xBERAKK0+lkw4YNyp8iIm5S/pSA0GoYjJsF7U9uuG1oLCQdD5HJTR+XBDXlT3FJwTprPaDiXf6ORILRV73h+7GeH//7B5A112vhHKT8KVUMw70XIkSaimFY64gPfeGo/Z10Op1s2rTJZ9fToJ+IiIiIiIg0jfCWVmnPiKSG2x7YeWgtDhERf+t7H5yeBS1S3Ttu/VP1r3El4orkE6DVSM+ONZ2w9EZYcoN3YxI5Uu5ya1Z0bfdvG1+AH8+AkkyfhyVBKCSq8eugNiMq7ykiIiIiIiJNxzShstCayVcXpwO+7AYpE6w3dUVE/C0swbPjVj8Ecb2h04XejUeCy5BnGnGwAWO+hvJcr4UjUqutb8CGp6Ht5JqlZPPXWiWSw+L9EZlIUNOgn4iI+J3dbvd3CCIiAUn5UwLCt0OsQb9pG+tu4yiBrtMhrqfv4pKgpvwpLineA1tfg3YnQcIA146ZtBgcZU0alki9DAMSBzXZ6ZU/pUr6ZdBmMoS3qrlv6HMw6Amwa/1HEfBt7jRM0zR9drVmrKCggLi4OPLz84mNrecNVhERERERkWCy8l4oz4chT/s7EhER92QvgZlDoeedMPCf/o5GgknhZlj7KLQ/Hdqd6N6xuSus2aY2zfUQETla+HL8SGv6eZnGUEVE3GOaJgUFBcqfIiJuUv6UgNHvQQ34yVFF+VNcljgYxn0P/f/uWvvyXCjJssoaizSGsxK2vArZi907rnQ/fDMQfr60ScJS/pQaTBMqS6pvy10BW/8LZdn+iUnkKHMwd/qKBv28zOl0+jsEEZGA4nQ62bp1q/KniIiblD8l4JTlQEUdH3Z/+zMsOAdM/T5L01P+FJcZBqSMA1uoa+23vAYzUmD/T00blzR/MV3h9L3Q7wE3D3RCn3ugw9lNEpbyp1TjdMDHLeGnC6pv3/EB/PwnKN7ln7hEjjJOp5Pt27f77Hqa5y0iIiIiIiJNK38dfDcSulwJAx+rZf9qKFgPht5LFZGjUP5aKNrecJnF+H7Q5WqI6eaTsKQZs9khorX7x0UkWTPsRXzBZrfW9Is9Yk3mrtdBfH+I6+WfuESCnAb9REREREREpGnFdofk0VapvNqM+QqcFb6NSUTEFaYJ86ZZ/25o0K/NBOtLxBuK90DuMkgZD/aIhtuX54MtDEIimz42kYOOfafmtqh20LFpZpuKSMP0GqWIiPhdRIQLH2BERKQG5U8JGIYNRn0KHc+pu42r5fNEvED5U1xmGDDgMRj6otbqE9/a/LI14Fyw3rX2a/4Bn3WEws1NGpbyp9Sr8oBV0l1EqvFl7tSgn5fZ7XZ/hyAiElDsdjs9evRQ/hQRcZPypwSs4j3VH0jmrYJt/4OybP/FJEFF+VPc1uEMaDPRGgCsi6MUZg6HTS/6Li5p3tqfCsf8GyLbutY+phskDoHo9CYLSflTatj/M/x8KeSusL7f9Zm1zt+Oj/0bl8hRxG63062b70p/a9DPy7SQrYiIe5xOJ9nZ2cqfIiJuUv6UgFS6F77oCmsfPbRtx8ew6EI48Lv/4pKgovwpHjFNKMmoe/+BnXBgG5Tu811M0rwlDrTWwo1Icq19lytg7NdNuj6u8qfUUJoFW9+wXuICaDMJEodC4iC/hiVyNHE6neTk+G4GrNb08zJTpR5ERNximiY7d+4kPj7e36GIiAQU5U8JSBFJ0PkySB53aFv6JRDXE2J7+i0sCS7Kn+KRBWdDxkw4M7v2csSxXeG0TDArfR+bNG+mWf8sU2eFVVIxLN4HoSh/yhFSJsKZORASY30f3hIm/gQ2DTuIHGSaJrt27fLZ9TTTT0RERERERHxnyLOQetqh76PTrLX+QiL9FpKISINSToBOF0NlUd1tDEPrk4p3zTsZvm1gxtTvH8CnHSBztm9iEjlcSCSU58HXfWDPTGubBvxE/Er/DxQRERERERHfK8uB0DhwFENojL+jERGpX9dr6t+/8QWI6w3Jo30TjwSHyHZg2Ouf7RcWB3G9IEHlFMVPKvKttZlLs/wdiYigmX4iInIUiInRgz4REU8of0rAWvUgzEiBrO/hw1hY82jDx4h4kfKneFVlMSy9ATY+4+9IpLk55kUYNaP+8p7tToJJP0N4ok9CUv6UGhIGwLTNkH6xvyMROWpFR0f77Foa9PMyu93u7xBERAKK3W6nc+fOyp8iIm5S/pSAljAA2kyxHpR3PA8S+vs7Igkiyp/isS2vw5e9rBkth7OFwYRF0Psv/olLglPGd5C32qeXVP6UOoXF+TsCkaOW3W4nPT3dZ9dTeU8vczqd/g5BRCSgOJ1O9u7dS1JSEjab3kUREXGV8qcEtPYnW18Aqaf6NRQJPsqf4jGzEioLoWgrhLc8tN0WAq2O8V9c0nyV58G6JyGhH3Q469D2iiL46XwIiYZpm3y2lqTyp4iI+5xOJ1lZvit/q+zsZaZp+jsEEZGAYpommZmZyp8iIm5S/pRmw9SLk+Jbyp/isfRL4ZQd0HJo9e0HdoKzwj8xSfNmC4M1/4AdH1XfHhoNx38Kw1/32YAfKH+KiHjCNE2fDvpppp+IiIiIiIj4XnkefJQAEUlwuu8+BIuIeMxWx2O0HyYCBpy01qfhSBAIiYKpqyG6lrJwScf5Ph4RETnqaaafiIiIiIiI+F5onFWqrNuN/o5ERMR1pfthyQ2w81Pre9O0clmni/waljRjcb3AHmH9u3gPzB4L+xb5NyYRETlqaaaflxmG4e8QREQCimEYJCYmKn+KiLhJ+VMCnmHAcR/4OwoJQsqf0jgmbH0dyvOtNUkNA/o96O+gpDmrLIaCdRDdGfYtgP0LoTzbL6Eof4qIuM8wDBISEnx3PVNFmL2ioKCAuLg48vPziY2N9Xc4IiIiIiIiIiLSFAo2QEw3a8BPpKltfQN+vhSOn2ENNBfvgsh2+v0TEQkgvhw/UnlPL3M6tQi9iIg7nE4nO3bsUP4UEXGT8qeIiGeUP6XRYrsfGnBZejP8eAY4HX4NSZqxVsdC3wes3zuAqPZ+G/BT/hQRcZ/T6WTnzp0+u54G/bxMEydFRNxjmiY5OTnKnyIiblL+FBHxjPKneEVJJvx0MWx4GgrWg83u74ikuYrtCmnnw8LzIGuuX0NR/hQRcZ9pmuTm5vrsehr0ExERERERERERcYctDHZ/Bl2uhikr/B2NNHe5y6FoMxh6lCsiIvUL8XcAIiIiIiIiIiIiASU8Eaauhah2/o5EgkGHMyFlPITF+zsSERE5yun1EC8ztIiuiIhbDMMgJSVF+VNExE3KnyIinlH+FK/RgJ/40lEw4Kf8KSLiPsMwSE5O9t31TBVh9oqCggLi4uLIz88nNjbW3+GIiIiIiIiIiIiIiIiIn/ly/Egz/bzM4XD4OwQRkYDicDjYsmWL8qeIiJuUP0VEPKP8KSLiGeVPERH3ORwOtm7d6rPradBPRET8rrCw0N8hiIgEJOVPERHPKH+KiHhG+VNExH1FRUU+u5YG/UREREREREREREREREQCnAb9RERERERERERERERERAKcBv28zDAMf4cgIhJQDMMgNTVV+VNExE3KnyIinlH+FBHxjPKniIj7DMOgffv2PrteiM+uFCRsNo2jioi4w2az0bJlS3+HISIScJQ/RUQ8o/wpIuIZ5U8REffZbDYSExN9dz2fXSlIOBwOf4cgIhJQHA4H69evV/4UEXGT8qeIiGeUP0VEPKP8KSLiPofDwcaNG312PQ36iYiI35WWlvo7BBGRgKT8KSLiGeVPERHPKH+KiLjPl7lTg34iIiIiIiIiIiIiIiIiAU6DfiIiIiIiIiIiIiIiIiIBToN+Xmaz6T+piIg7bDYb6enpyp8iIm5S/hQR8Yzyp4iIZ5Q/RUTcZ7PZSEtL89n1Qnx2pSBhGIa/QxARCSiGYRAbG+vvMEREAo7yp4iIZ5Q/RUQ8o/wpIuI+X+dOvZbhZQ6Hw98hiIgEFIfDwapVq5Q/RUTcpPwpIuIZ5U8REc8of4qIuM/hcLBmzRqfXU+DfiIi4nf6wCAi4hnlTxERzyh/ioh4RvlTRMR9vsydGvQTERERERERERERERERCXAa9BMREREREREREREREREJcIZpmqa/g2gOCgoKiIuLIy8vj7i4OH+HIyISMEzTpLS0lIiICAzD8Hc4IiIBQ/lTRMQzyp8iIp5R/hQRcZ9pmuzdu5eUlBTy8/OJjY1t0utppp+IiPhdWFiYv0MQEQlIyp8iIp5R/hQR8Yzyp4iI+3yZOzXo52VOp9PfIYiIBBSn08mqVauUP0VE3KT8KSLiGeVPERHPKH+KiLjP6XSyZs0an11Pg34iIiIiIiIiIiIiIiIiAU6DfiIiIiIiIiIiIiIiIiIBToN+IiIiIiIiIiIiIiIiIgHOME3T9HcQzUFBQQFxcXHk5eURFxfn73BERAKGaZo4nU5sNhuGYfg7HBGRgKH8KSLiGeVPERHPKH+KiLjPNE3y8vJITEwkPz+f2NjYJr2eZvqJiIjflZeX+zsEEZGApPwpIuIZ5U8REc8of4qIuM+XuVODfl7mdDr9HYKISEBxOp1s2LBB+VNExE3KnyIinlH+FBHxjPKniIj7nE4nmzZt8tn1NOgnIiIiIiIiIiIiIiIiEuA06CciIiIiIiIiIiIiIiIS4DToJyIifme32/0dgohIQFL+FBHxjPKniIhnlD9FRNzny9xpmKZp+uxqzVhBQQFxcXHk5+cTGxvr73BERERERERERERERETEz3w5fqSZfl6mMVQREfeYpklBQYHyp4iIm5Q/RUQ8o/wpIuIZ5U8REfcdzJ2+okE/L3M6nf4OQUQkoDidTrZu3ar8KSLiJuVPERHPKH+KiHhG+VNExH1Op5Pt27f77Hoa9BMREREREREREREREREJcBr0ExEREREREREREREREQlwGvQTERG/i4iI8HcIIiIBSflTRMQzyp8iIp5R/hQRcZ8vc6dhauVVrygoKCAuLo78/HxiY2P9HY6IiIiIiIiIiIiIiIj4mS/HjzTTz8u0kK2IiHucTifZ2dnKnyIiblL+FBHxjPKniIhnlD9FRNzndDrJycnx2fU06OdlmjgpIuIe0zTZuXOn8qeIiJuUP0VEPKP8KSLiGeVPERH3mabJrl27fHY9DfqJiIiIiIiIiIiIiIiIBDgN+omIiIiIiIiIiIiIiIgEOA36iYiI38XExPg7BBGRgKT8KSLiGeVPERHPKH+KiLgvOjraZ9cyTBVh9oqCggLi4uLIz88nNjbW3+GIiIiIiIiIiIiIiIiIn/ly/Egz/bzM6XT6OwQRkYDidDrJzMxU/hQRcZPyp4iIZ5Q/RUQ8o/wpIuI+p9NJVlaWz66nQT8v08RJERH3mKZJZmam8qeIiJuUP0VEPKP8KSLiGeVPERH3maapQT8RERERERERERERERERcZ0G/UREREREREREREREREQCnAb9vMwwDH+HICISUAzDIDExUflTRMRNyp8iIp5R/hQR8Yzyp4iI+wzDICEhwXfXM1WE2SsKCgqIi4sjPz+f2NhYf4cjIiIiIiIiIiIiIiIifubL8SPN9PMyp9Pp7xBERAKK0+lkx44dyp8iIm5S/hQR8Yzyp4iIZ5Q/RUTc53Q62blzp8+up0E/L9PESRER95imSU5OjvKniIiblD9FRDyj/Cki4hnlTxER95mmSW5urs+up0E/ERERERERERERERERkQAX4u8AmouDb7gUFBRgt9v9HI2ISOBwOBwUFRUpf4qIuEn5U0TEM8qfIiKeUf4UEXGfw+HgwIEDPrueBv28JDs7G4C0tDT/BiIiIiIiIiIiIiIiIiJBR4N+XpKYmAjAjh07iIuL83M0IiKBo6CggNTUVHbu3ElsbKy/wxERCRjKnyIinlH+FBHxjPKniIj7DubOHTt2EBMT0+TX06Cfl9hs1vKIcXFx+qMnIuKB2NhY5U8REQ8of4qIeEb5U0TEM8qfIiLui4uLwzCMJr+OrcmvICIiIiIiIiIiIiIiIiJNSoN+IiIiIiIiIiIiIiIiIgFOg35eEh4ezn333Ud4eLi/QxERCSjKnyIinlH+FBHxjPKniIhnlD9FRNzn69xpmKZp+uRKIiIiIiIiIiIiIiIiItIkNNNPREREREREREREREREJMBp0E9EREREREREREREREQkwGnQT0RERERERERERERERCTAadBPREREREREREREREREJMBp0O8wjzzyCEOHDiUmJoakpCROPfVUNmzYUK1NaWkp06dPp2XLlkRHR3PGGWeQlZVVrc2OHTuYOnUqUVFRJCUlcccdd1BZWVmtzdy5cxk0aBDh4eF06dKFN954o6m7JyLSJFzJnf/+978ZM2YMsbGxGIZBXl5ejfPk5ORwwQUXEBsbS3x8PJdffjlFRUXV2qxcuZLjjz+eiIgIUlNTeeyxx5qyayIiTaqh/JmTk8MNN9xA9+7diYyMpEOHDtx4443k5+dXO4/uPUUk2Lhy/3n11VfTuXNnIiMjad26Naeccgrr16+v1kb5U0SCjSv58yDTNJkyZQqGYfDpp59W26f8KSLBxpX8OWbMGAzDqPZ1zTXXVGvji/ypQb/DzJs3j+nTp/Pzzz8za9YsKioqmDhxIgcOHKhqc8stt/DFF1/w4YcfMm/ePPbs2cPpp59etd/hcDB16lTKy8v56aefePPNN3njjTe49957q9ps27aNqVOnMnbsWJYvX87NN9/MFVdcwcyZM33aXxERb3AldxYXFzN58mT+8pe/1HmeCy64gDVr1jBr1iy+/PJL5s+fz1VXXVW1v6CggIkTJ9KxY0eWLl3K448/zv3338+///3vJu2fiEhTaSh/7tmzhz179vDEE0+wevVq3njjDb799lsuv/zyqnPo3lNEgpEr95+DBw/m9ddfZ926dcycORPTNJk4cSIOhwNQ/hSR4ORK/jzoqaeewjCMGtuVP0UkGLmaP6+88koyMjKqvg6fsOCz/GlKnfbu3WsC5rx580zTNM28vDwzNDTU/PDDD6varFu3zgTMRYsWmaZpml9//bVps9nMzMzMqjYvvviiGRsba5aVlZmmaZp33nmn2bt372rXOuecc8xJkyY1dZdERJrckbnzcD/88IMJmLm5udW2r1271gTMX3/9tWrbN998YxqGYe7evds0TdN84YUXzISEhKpcapqm+ec//9ns3r1703RERMTH6sufB33wwQdmWFiYWVFRYZqm7j1FREzTtfy5YsUKEzA3b95smqbyp4iIadadP3/77TezXbt2ZkZGhgmYM2bMqNqn/CkiUnv+HD16tHnTTTfVeYyv8qdm+tXjYOmkxMREAJYuXUpFRQXjx4+vatOjRw86dOjAokWLAFi0aBF9+/YlOTm5qs2kSZMoKChgzZo1VW0OP8fBNgfPISISyI7Mna5YtGgR8fHxDBkypGrb+PHjsdlsLF68uKrNqFGjCAsLq2ozadIkNmzYQG5urpeiFxHxH1fyZ35+PrGxsYSEhAC69xQRgYbz54EDB3j99dfp1KkTqampgPKniAjUnj+Li4s5//zzef7550lJSalxjPKniEjd95//+9//aNWqFX369OHuu++muLi4ap+v8qcG/ergdDq5+eabOfbYY+nTpw8AmZmZhIWFER8fX61tcnIymZmZVW0O/6Ed3H9wX31tCgoKKCkpaYruiIj4RG250xWZmZkkJSVV2xYSEkJiYqJb+VVEJFC5kj/379/PQw89VK30se49RSTY1Zc/X3jhBaKjo4mOjuabb75h1qxZVS+QKX+KSLCrK3/ecsstjBw5klNOOaXW45Q/RSTY1ZU/zz//fN5++21++OEH7r77bt566y0uvPDCqv2+yp8hHvUqCEyfPp3Vq1ezYMECf4ciIhIwlDtFRDzTUP4sKChg6tSp9OrVi/vvv9+3wYmIHMXqy58XXHABEyZMICMjgyeeeIKzzz6bhQsXEhER4YdIRUSOLrXlz88//5w5c+bw22+/+TEyEZGjW133n4e/oNu3b1/atGnDCSecwJYtW+jcubPP4tNMv1pcf/31fPnll/zwww+0b9++antKSgrl5eXk5eVVa5+VlVU13T0lJYWsrKwa+w/uq69NbGwskZGR3u6OiIhP1JU7XZGSksLevXurbausrCQnJ8et/CoiEogayp+FhYVMnjyZmJgYZsyYQWhoaNU+3XuKSDBrKH/GxcXRtWtXRo0axUcffcT69euZMWMGoPwpIsGtrvw5Z84ctmzZQnx8PCEhIVUl5c844wzGjBkDKH+KSHBz5/nnsGHDANi8eTPgu/ypQb/DmKbJ9ddfz4wZM5gzZw6dOnWqtn/w4MGEhoby/fffV23bsGEDO3bsYMSIEQCMGDGCVatWVXt4PWvWLGJjY+nVq1dVm8PPcbDNwXOIiASShnKnK0aMGEFeXh5Lly6t2jZnzhycTmfVH8gRI0Ywf/58KioqqtrMmjWL7t27k5CQ0PiOiIj4mCv5s6CggIkTJxIWFsbnn39eY3aK7j1FJBh5cv9pmiamaVJWVgYof4pIcGoof951112sXLmS5cuXV30B/Otf/+L1118HlD9FJDh5cv95MIe2adMG8GH+NKXKtddea8bFxZlz5841MzIyqr6Ki4ur2lxzzTVmhw4dzDlz5phLliwxR4wYYY4YMaJqf2VlpdmnTx9z4sSJ5vLly81vv/3WbN26tXn33XdXtdm6dasZFRVl3nHHHea6devM559/3rTb7ea3337r0/6KiHiDK7kzIyPD/O2338xXXnnFBMz58+ebv/32m5mdnV3VZvLkyebAgQPNxYsXmwsWLDC7du1qnnfeeVX78/LyzOTkZPOiiy4yV69ebb733ntmVFSU+fLLL/u0vyIi3tJQ/szPzzeHDRtm9u3b19y8eXO1NpWVlaZp6t5TRIJTQ/lzy5Yt5sMPP2wuWbLE/P33382FCxea06ZNMxMTE82srCzTNJU/RSQ4ufL5/UiAOWPGjKrvlT9FJBg1lD83b95sPvjgg+aSJUvMbdu2mZ999pmZnp5ujho1quocvsqfGvQ7DFDr1+uvv17VpqSkxLzuuuvMhIQEMyoqyjzttNPMjIyMaufZvn27OWXKFDMyMtJs1aqVedttt5kVFRXV2vzwww/mgAEDzLCwMDM9Pb3aNUREAokrufO+++5rsE12drZ53nnnmdHR0WZsbKx56aWXmoWFhdWutWLFCvO4444zw8PDzXbt2pmPPvqoj3opIuJ9DeXPH374oc4227ZtqzqP7j1FJNg0lD93795tTpkyxUxKSjJDQ0PN9u3bm+eff765fv36audR/hSRYOPK5/fajjl80M80lT9FJPg0lD937Nhhjho1ykxMTDTDw8PNLl26mHfccYeZn59f7Ty+yJ/GHwGLiIiIiIiIiIiIiIiISIDSmn4iIiIiIiIiIiIiIiIiAU6DfiIiIiIiIiIiIiIiIiIBToN+IiIiIiIiIiIiIiIiIgFOg34iIiIiIiIiIiIiIiIiAU6DfiIiIiIiIiIiIiIiIiIBToN+IiIiIiIiIiIiIiIiIgFOg34iIiIiIiIiIiIiIiIiAU6DfiIiIiIiIiIiIiIiIiIBToN+IiIiIiIizdwll1zCqaee6u8wPBbo8YuIiIiIiPhCiL8DEBEREREREc8ZhlHv/vvuu4+nn34a0zR9FNEhc+fOZezYseTm5hIfH+/z64uIiIiIiAQTDfqJiIiIiIgEsIyMjKp/v//++9x7771s2LChalt0dDTR0dH+CE1ERERERER8SOU9RUREREREAlhKSkrVV1xcHIZhVNsWHR1dozzmmDFjuOGGG7j55ptJSEggOTmZV155hQMHDnDppZcSExNDly5d+Oabb6pda/Xq1UyZMoXo6GiSk5O56KKL2L9/v8uxvvHGG8THxzNz5kx69uxJdHQ0kydPrjZw6XA4uPXWW4mPj6dly5bceeedNWYpOp1OHnnkETp16kRkZCT9+/fno48+AsA0TcaPH8+kSZOqjsvJyaF9+/bce++97v7nFRERERERCRga9BMREREREQlCb775Jq1ateKXX37hhhtu4Nprr+Wss85i5MiRLFu2jIkTJ3LRRRdRXFwMQF5eHuPGjWPgwIEsWbKEb7/9lqysLM4++2y3rltcXMwTTzzBW2+9xfz589mxYwe333571f4nn3ySN954g9dee40FCxaQk5PDjBkzqp3jkUce4b///S8vvfQSa9as4ZZbbuHCCy9k3rx5GIbBm2++ya+//sozzzwDwDXXXEO7du006CciIiIiIs2aynuKiIiIiIgEof79+3PPPfcAcPfdd/Poo4/SqlUrrrzySgDuvfdeXnzxRVauXMnw4cN57rnnGDhwIA8//HDVOV577TVSU1PZuHEj3bp1c+m6FRUVvPTSS3Tu3BmA66+/ngcffLBq/1NPPcXdd9/N6aefDsBLL73EzJkzq/aXlZXx8MMPM3v2bEaMGAFAeno6CxYs4OWXX2b06NG0a9eOl19+mYsvvpjMzEy+/vprfvvtN0JC9BFYRERERESaL33iERERERERCUL9+vWr+rfdbqdly5b07du3altycjIAe/fuBWDFihX88MMPta4PuGXLFpcH/aKioqoG/ADatGlTdY38/HwyMjIYNmxY1f6QkBCGDBlSVapz8+bNFBcXM2HChGrnLS8vZ+DAgVXfn3XWWcyYMYNHH32UF198ka5du7oUn4iIiIiISKDSoJ+IiIiIiEgQCg0Nrfa9YRjVthmGAVjr5wEUFRUxbdo0/vnPf9Y4V5s2bRp13SPX7KtPUVERAF999RXt2rWrti88PLzq38XFxSxduhS73c6mTZtcPr+IiIiIiEig0qCfiIiIiIiINGjQoEF8/PHHpKWlNVmZzLi4ONq0acPixYsZNWoUAJWVlSxdupRBgwYB0KtXL8LDw9mxYwejR4+u81y33XYbNpuNb775hhNPPJGpU6cybty4JolbRERERETkaGDzdwAiIiIiIiJy9Js+fTo5OTmcd955/Prrr2zZsoWZM2dy6aWX4nA4vHadm266iUcffZRPP/2U9evXc91115GXl1e1PyYmhttvv51bbrmFN998ky1btrBs2TKeffZZ3nzzTcCaBfjaa6/xv//9jwkTJnDHHXfwpz/9idzcXK/FKSIiIiIicrTRoJ+IiIiIiIg0qG3btixcuBCHw8HEiRPp27cvN998M/Hx8dhs3vtoedttt3HRRRfxpz/9iREjRhATE8Npp51Wrc1DDz3E3/72Nx555BF69uzJ5MmT+eqrr+jUqRP79u3j8ssv5/7776+aHfjAAw+QnJzMNddc47U4RUREREREjjaG6c7iCSIiIiIiIiIiIiIiIiJy1NFMPxEREREREREREREREZEAp0E/ERERERERERERERERkQCnQT8RERERERERERERERGRAKdBPxEREREREREREREREZEAp0E/ERERERERERERERERkQCnQT8RERERERERERERERGRAKdBPxEREREREREREREREZEAp0E/ERERERERERERERERkQCnQT8RERERERERERERERGRAKdBPxEREREREREREREREZEAp0E/ERERERERERERERERkQD3/8gHqS8lapCLAAAAAElFTkSuQmCC\n" + }, + "metadata": {} + }, + { + "output_type": "display_data", + "data": { + "text/plain": [ + "
" + ], + "image/png": "iVBORw0KGgoAAAANSUhEUgAABv4AAAN5CAYAAADAfkzvAAAAOnRFWHRTb2Z0d2FyZQBNYXRwbG90bGliIHZlcnNpb24zLjEwLjAsIGh0dHBzOi8vbWF0cGxvdGxpYi5vcmcvlHJYcgAAAAlwSFlzAAAPYQAAD2EBqD+naQABAABJREFUeJzs3XdYFNf7NvB76HVBbKBUK6jYS0ii2LFGY48Ve1Rir1+NNbFEjZoYS9SoscSaqAn2gj12FBs2sEQQlSZK3T3vH747P1eKoOgweH+uiyvu2bMzz9mdG6IPMyMJIQSIiIiIiIiIiIiIiIiISNWMlC6AiIiIiIiIiIiIiIiIiN4dG39ERERERERERERERERE+QAbf0RERERERERERERERET5ABt/RERERERERERERERERPkAG39ERERERERERERERERE+QAbf0RERERERERERERERET5ABt/RERERERERERERERERPkAG39ERERERERERERERERE+QAbf0RERERERERERERERET5ABt/RERERKQq/v7+cHd3V7oMxXzs63+fVq1aBUmSEB4ernQpeV54eDgkScKqVasUq6FZs2bo27evYvvPa4KCgiBJEoKCgpQuhfKBTz75BKNHj1a6DCIiIiJ6C2z8EREREZHiJEnK1lde/Qft8PBw9OzZEyVLloSFhQUcHR1Rp04dTJo0SenScl3NmjUhSRIWL16sdCl5nr+/v8Hxa2NjgxIlSqBdu3bYunUrdDrdW297586dmDx5cu4Vm4n169dj/vz5730/OXX8+HHs3bsXY8aMkcfc3d3f+D3E399fuaLzCH2D++zZs+99X0lJSZgxYwbKlSsHKysrFC9eHO3bt8eVK1cM5h05cgRffPEFXFxc5O+hTZo0wfHjxzPc7okTJ/D555/DysoKjo6OGDx4MBISEtLNS05OxpgxY1CsWDFYWlqiVq1a2LdvX7Zqf5/H/sKFC+Hl5QVzc3MUL14cw4cPx/Pnz9PN0+l0+OGHH+Dh4QELCwtUrFgRf/zxR4bbvHbtGpo0aQIbGxs4ODigW7duePz48Vtvc8yYMfjll18QGRn57gsmIiIiog/KROkCiIiIiIjWrFlj8Pj333/Hvn370o17eXlh2bJl79QwyW23bt1CjRo1YGlpiV69esHd3R0RERE4f/48Zs2ahSlTpuTq/pRc/82bN3HmzBm4u7tj3bp1GDBggCJ1qIm5uTmWL18OAEhMTMTdu3fx999/o127dqhbty62b98OjUaT4+3u3LkTv/zyy3tv/q1fvx6XL1/G0KFDDcbd3NyQmJgIU1PT97r/zMyePRsNGjRAqVKl5LH58+dn2PwBXjZaTp06hU8++eRDlUgAunTpgh07dqBv376oWrUqHj58iF9++QU+Pj4ICQmBm5sbAODGjRswMjLC119/DUdHR8TExGDt2rWoU6cOAgMD0aRJE3mbwcHBaNCgAby8vPDjjz/iwYMHmDNnDm7evIldu3YZ7N/f3x9btmzB0KFDUbp0aaxatQrNmjXDoUOH8Pnnn2dZe2bH/rsaM2YMfvjhB7Rr1w5DhgzB1atX8fPPP+PKlSvYs2ePwdzx48dj5syZ6Nu3L2rUqIHt27ejc+fOkCQJnTp1kuc9ePAAderUgZ2dHaZPn46EhATMmTMHISEhOH36NMzMzHK8zVatWkGj0WDRokWYOnVqrr4HRERERPSeCSIiIiKiPGbQoEFCLf+rOnDgQGFiYiLCw8PTPffo0aNc209CQkKubettTZw4URQpUkRs3bpVSJIkwsLClC4pV61cuVIAyLV19ejRQ1hbW2f43IwZMwQA0aFDh7fa9ofKSPPmzYWbm9t7309OPHr0SJiYmIjly5dna/6ePXuEJEniiy++eM+VKevQoUMCgDh06FCW8/TH+ZkzZ95rPQ8ePBAAxMiRIw3GDx48KACIH3/8McvXP3/+XBQtWlT4+fkZjDdt2lQ4OTmJuLg4eWzZsmUCgNizZ488durUKQFAzJ49Wx5LTEwUJUuWFD4+Pm+s/30c+w8fPhQmJiaiW7duBuM///yzACB27Nghjz148ECYmpqKQYMGyWM6nU7Url1bODs7i7S0NHl8wIABwtLSUty9e1ce27dvnwAgli5d+lbbFEKIgIAA4ebmJnQ63bsvnoiIiIg+GF7qk4iIiIhU5fV73OnvNTZnzhz88ssvKFGiBKysrNC4cWPcv38fQghMmzYNzs7OsLS0RKtWrRAdHZ1uu7t27ULt2rVhbW0NW1tbNG/ePN3l6DJy+/ZtODs7y2euvKpIkSJvtR9/f3/Y2Njg9u3baNasGWxtbdGlS5cM1w+8vHTb/PnzUb58eVhYWKBo0aLo378/YmJiDOadPXsWfn5+KFSoECwtLeHh4YFevXq9cY1669evR7t27dCiRQvY2dlh/fr16eZMnjwZkiTh1q1b8Pf3h729Pezs7NCzZ0+8ePHCYG5aWhqmTZuGkiVLwtzcHO7u7vjf//6H5ORkg3nu7u5o0aIFgoKCUL16dVhaWsLb21u+9Ouff/4Jb29vWFhYoFq1arhw4YLB6y9dugR/f3+UKFFCvoxgr1698PTp0yzX26NHDxQqVAipqanpnmvcuDHKli2bnbctQ2PHjkXjxo2xefNm3Lhxw+C5Nx0j/v7++OWXXwAYXiZXL7vHg35fvr6+sLW1hUajQY0aNeTPtW7duggMDMTdu3flfeiPvczu8Xfw4EG5dnt7e7Rq1QrXrl0zmJOTYyQjgYGBSEtLQ8OGDd84NzIyEt26dUPx4sWxcuVKg+eye/wBwKJFi1C+fHmYm5ujWLFiGDRoEGJjYw3m1K1bFxUqVMClS5fg6+sLKysrlCpVClu2bAEAHD58GLVq1YKlpSXKli2L/fv3p9vPf//9h169eqFo0aIwNzdH+fLl8dtvv6Wb9+DBA7Ru3RrW1tYoUqQIhg0blmHd2aX/nvPff/+hdevWsLGxQeHChTFy5EhotVqDuREREbh+/XqGuXjVs2fPAABFixY1GHdycgIAWFpaZvl6KysrFC5c2OB9jo+Px759+9C1a1eDM2W7d+8OGxsbbNq0SR7bsmULjI2N0a9fP3nMwsICvXv3xsmTJ3H//v1M953VsQ8AUVFR6N27N4oWLQoLCwtUqlQJq1evznI9AHDy5EmkpaUZnFkHQH68YcMGeWz79u1ITU3FwIED5TFJkjBgwAA8ePAAJ0+elMe3bt2KFi1awNXVVR5r2LAhypQpY/Ce5GSbANCoUSPcvXsXwcHBb1wbEREREeUdbPwRERERUb6wbt06LFq0CN988w1GjBiBw4cPo0OHDpgwYQJ2796NMWPGoF+/fvj7778xcuRIg9euWbMGzZs3h42NDWbNmoVvv/0WV69exeeff47w8PAs9+vm5ob79+/j4MGDb6wxJ/tJS0uDn58fihQpgjlz5qBt27aZbrd///4YNWoUPvvsMyxYsAA9e/bEunXr4OfnJ//jfFRUFBo3bozw8HCMHTsWP//8M7p06YJ///33jXUDwKlTp3Dr1i189dVXMDMzQ5s2bbBu3bpM53fo0AHPnj3DjBkz0KFDB6xatSrdZU/79OmDiRMnomrVqpg3bx58fX0xY8aMdP8oDry8pGrnzp3RsmVLzJgxAzExMWjZsiXWrVuHYcOGoWvXrpgyZQpu376NDh06GFwOdd++fbhz5w569uyJn3/+GZ06dcKGDRvQrFkzCCEyXUO3bt3w9OnTdJffi4yMxMGDB9G1a9dsvXdZbV8IYXDPsewcI/3790ejRo3k+fovvewcD8DLe701b94c0dHRGDduHGbOnInKlStj9+7dAF5eErBy5cooVKiQvI+s7nm2f/9++Pn5ISoqCpMnT8bw4cNx4sQJfPbZZxnmKDvHSEZOnDiBggULZthsf5VOp0PXrl3x9OlTrF+/Hg4ODgbPZ/f4mzx5MgYNGoRixYph7ty5aNu2LZYuXYrGjRuna37FxMSgRYsWqFWrFn744QeYm5ujU6dO2LhxIzp16oRmzZph5syZeP78Odq1ayc3xwDg0aNH+OSTT7B//34EBARgwYIFKFWqFHr37m3wvicmJqJBgwbYs2cPAgICMH78eBw9ehSjR49+43uXFa1WCz8/PxQsWBBz5syBr68v5s6di19//dVg3rhx4+Dl5YX//vsvy+2VLFkSzs7OmDt3Lv7++288ePAAp0+fxtdffw0PD48Mcx4fH48nT57g+vXr+N///ofLly+jQYMG8vMhISFIS0tD9erVDV5nZmaGypUrGzT9L1y4gDJlyqS7lG7NmjUBIMtmVlbHfmJiIurWrYs1a9agS5cumD17Nuzs7ODv748FCxZk+Z7om7OvNz2trKwAAOfOnTOo39raGl5eXhnWr1/rf//9h6ioqHTviX7u6+9JdrapV61aNQDI9F6LRERERJRHKXzGIRERERFROlldxrBHjx4Gl18LCwsTAEThwoVFbGysPD5u3DgBQFSqVEmkpqbK41999ZUwMzMTSUlJQgghnj17Juzt7UXfvn0N9hMZGSns7OzSjb/u8uXLwtLSUgAQlStXFkOGDBHbtm0Tz58/N5iXk/306NFDABBjx4594/qPHj0qAIh169YZzNu9e7fB+F9//fVOl/cLCAgQLi4u8iXf9u7dKwCICxcuGMybNGmSACB69eplMP7ll1+KggULyo+Dg4MFANGnTx+DeSNHjhQAxMGDB+UxNzc3AUCcOHFCHtuzZ48AkO7ydkuXLk13ucMXL16kW88ff/whAIgjR47IY69f6lOr1QpnZ2fRsWNHg9f++OOPQpIkcefOnYzeKllWl/oUQogLFy4IAGLYsGFCiJwdI5llJLvHQ2xsrLC1tRW1atUSiYmJBnNfvaxfZpc71Odu5cqV8ljlypVFkSJFxNOnT+WxixcvCiMjI9G9e3d5LLvHSGY+//xzUa1atTfOmzp1qgAgpkyZku657B5/UVFRwszMTDRu3FhotVp53sKFCwUA8dtvv8ljvr6+AoBYv369PHb9+nUBQBgZGYl///1XHtcfv6++f7179xZOTk7iyZMnBjV16tRJ2NnZycfx/PnzBQCxadMmec7z589FqVKl3vpSn/rvOVOnTjWYW6VKlXTvtX5udi6Je+rUKVGyZEkBQP6qVq2aiIiIyHC+n5+fPM/MzEz079/f4PjcvHlzutzqtW/fXjg6OsqPy5cvL+rXr59u3pUrVwQAsWTJkixrz+zY17//a9eulcdSUlKEj4+PsLGxEfHx8Zlu89y5cwKAmDZtmsG4Pp82NjYG+y9RokS6bTx//tzg58OZM2cEAPH777+nmztq1CgBQP55l91tvsrMzEwMGDAg0zURERERUd7DM/6IiIiIKF9o37497Ozs5Me1atUCAHTt2hUmJiYG4ykpKfLZKvv27UNsbCy++uorPHnyRP4yNjZGrVq1cOjQoSz3W758eQQHB6Nr164IDw/HggUL0Lp1axQtWhTLli2T573NfgYMGPDGdW/evBl2dnZo1KiRwXarVasGGxsbebv29vYAgH/++eeNl+h7XVpaGjZu3IiOHTvKl5SsX78+ihQpkulZf19//bXB49q1a+Pp06eIj48HAOzcuRMAMHz4cIN5I0aMAPDyco6vKleuHHx8fOTH+s+3fv36Bpe304/fuXNHHnv17JqkpCQ8efIEn3zyCQDg/Pnzma7byMgIXbp0wY4dOwzOzFq3bh0+/fRTeHh4ZPra7LCxsQHwf5dEfNdjEcj+8bBv3z48e/YMY8eOhYWFhcE2Xr1saHZFREQgODgY/v7+BmfWVaxYEY0aNZI/71e96RjJzNOnT1GgQIEs5xw9ehRTpkxB3bp1MWHChHTPZ/f4279/P1JSUjB06FAYGf3fX5/79u0LjUaT7ji1sbExOJOtbNmysLe3h5eXl3xsAumPUyEEtm7dipYtW0IIYfDZ+fn5IS4uTj5Wd+7cCScnJ7Rr107enpWVlcElLd9WRp/Jq1kCXp4pKoRId8nhjBQoUACVK1fG2LFjsW3bNsyZMwfh4eFo3749kpKS0s2fOXMm9u7dixUrVuCTTz5BSkoK0tLS5OcTExMBAObm5ulea2FhIT+vn5vZvFe3lVM7d+6Eo6MjvvrqK3nM1NQUgwcPRkJCAg4fPpzpa6tWrYpatWph1qxZWLlyJcLDw7Fr1y70798fpqamb1X/m96T1+fm9D0pUKAAnjx5kumaiIiIiCjvMXnzFCIiIiKivO/V5g8AuQno4uKS4bj+fmc3b94E8LKBlJHXLxOXkTJlymDNmjXQarW4evUq/vnnH/zwww/o168fPDw80LBhwxzvx8TEBM7Ozm/c982bNxEXF5fh/QSBl5f4BABfX1+0bdsWU6ZMwbx581C3bl20bt0anTt3zvAfgl+1d+9ePH78GDVr1sStW7fk8Xr16uGPP/7ArFmzDJoiQPrPQ9+oiYmJgUajwd27d2FkZIRSpUoZzHN0dIS9vT3u3r2b5fay+/kCQHR0NKZMmYINGzbI74deXFxclmvv3r07Zs2ahb/++gvdu3dHaGgozp07hyVLlmT5uuxISEgAANja2gLInWMxu8fD7du3AQAVKlTIWdGZ0H9eGd330MvLC3v27MHz589hbW0tj7/pGMmKyOISrU+fPsVXX32FAgUKYN26demOTX292Tn+MluXmZkZSpQoke44dXZ2Ttc4tbOze+Nx+vjxY8TGxuLXX39Nd2lNPf1nd/fuXZQqVSrdft7lnpPAy+ZP4cKFDcYKFCiQ4b0hsyMuLg61a9fGqFGj5IYqAFSvXh1169bFypUr0/1yQ+XKleU/d+3aFVWrVoW/v798n0R9Ez+j+xkmJSUZNPktLS0znffqtnLq7t27KF26dLrjSn/5zNePiddt3boVHTt2lO+vamxsjOHDh+Pw4cMIDQ3Ncf1vek9en5vT90QI8Va/DEBEREREymHjj4iIiIjyBWNj4xyN6xsH+nvBrVmzBo6OjunmvXq2YHZq8Pb2hre3N3x8fFCvXj2sW7cODRs2zPF+zM3NM2xYvE6n02V55p3+H/IlScKWLVvw77//4u+//8aePXvQq1cvzJ07F//++6989llG9Nvu0KFDhs8fPnwY9erVMxh70/uul91/UH7bzxd4WfeJEycwatQoVK5cGTY2NtDpdGjSpInBvQAzUq5cOVSrVg1r165F9+7dsXbtWpiZmWX6XuTE5cuXAUBuPuXGsZjd4yEvyO4x8rqCBQtm2owSQqBHjx54+PAh/v77bxQrVizLbeV2Q+Ndvw917doVPXr0yHBuxYoVc6HCzGVW49vaunUrHj16hC+++MJg3NfXFxqNBsePH8/yrGYzMzN88cUXmDlzJhITE2FpaQknJycAL88wfV1ERITB5+3k5JThfQj1r33TsfG+FC9eHMeOHcPNmzcRGRmJ0qVLw9HREcWKFUOZMmXkeU5OTjh06FC6xtvr9b/pPXFwcJB/uSO723xVbGwsChUq9K7LJiIiIqIPiI0/IiIiIvqolSxZEgBQpEgRNGzYMNe2W716dQD/9w+q72s/JUuWxP79+/HZZ59l6wyWTz75BJ988gm+//57rF+/Hl26dMGGDRvQp0+fDOc/f/4c27dvR8eOHQ0uLag3ePBgrFu3Ll3j703c3Nyg0+lw8+ZN+UwZAHj06BFiY2Ph5uaWo+1lJiYmBgcOHMCUKVMwceJEeVx/dl12dO/eHcOHD0dERATWr1+P5s2bv/FSk9mxZs0aSJKERo0aAcjZMZJZwyq7x4N+X5cvX0531lt29vM6/ef16hlLetevX0ehQoUMzvZ7F56enti6dWuGz/34448IDAzEsGHD0Lx58yzrzc7x9+q6SpQoIc9LSUlBWFhYrmW5cOHCsLW1hVarfeM23dzccPny5XTNm4zeeyU9evQIAKDVag3GhRDQarUGl/DMTGJiIoQQePbsGSwtLVGhQgWYmJjg7NmzBs33lJQUBAcHG4xVrlwZhw4dQnx8vMEZpKdOnZKfz0pmx76bmxsuXboEnU5n8MsZ169fl5/PjtKlS6N06dIAgKtXryIiIgL+/v4G9S9fvhzXrl1DuXLlMq2/ePHiKFy4MM6ePZtuH6dPnzZYZ3a3qffff/8hJSXFICNERERElPfxHn9ERERE9FHz8/ODRqPB9OnTM7z33ePHj7N8/dGjRzN8nf4eYvrL773rfjLToUMHaLVaTJs2Ld1zaWlpiI2NBfCyAfb6mVT6f+TN6NJven/99ReeP3+OQYMGoV27dum+WrRoga1bt2a5jYw0a9YMADB//nyD8R9//BEAsmza5IT+LKbX1/76frPy1VdfQZIkDBkyBHfu3EHXrl3fuS79vcw6duwo/+N/To4RfRNN//nqZfd4aNy4MWxtbTFjxox091p79b2ytrZ+4+VQgZdnElWuXBmrV682qOny5cvYu3ev/HnnBh8fH8TExKS799yZM2cwbtw4VKtWDTNnzsxyG9k9/ho2bAgzMzP89NNPBu/LihUrEBcXl6vHadu2bbF161b5TNBXvfrZN2vWDA8fPpQvfwkAL168yPQSobktIiIC169ff+O9QvVnr23YsMFgfMeOHXj+/DmqVKkij71+CV7g5bG9detWuLi4yJeutbOzQ8OGDbF27VqD+26uWbMGCQkJaN++vTzWrl07aLVag/clOTkZK1euRK1atdJdfvV1mR37zZo1Q2RkJDZu3CiPpaWl4eeff4aNjQ18fX2z3O7rdDodRo8eDSsrK4N7LLZq1QqmpqZYtGiRPCaEwJIlS1C8eHF8+umn8njbtm3xzz//4P79+/LYgQMHcOPGDYP3JCfbBIBz584BQLpxIiIiIsrbeMYfEREREX3UNBoNFi9ejG7duqFq1aro1KkTChcujHv37iEwMBCfffYZFi5cmOnrZ82ahXPnzqFNmzbypfjOnz+P33//HQ4ODhg6dGiu7Cczvr6+6N+/P2bMmIHg4GA0btwYpqamuHnzJjZv3owFCxagXbt2WL16NRYtWoQvv/wSJUuWxLNnz7Bs2TJoNJosmzLr1q1DwYIFM/2H3y+++ALLli1DYGAg2rRpk+26K1WqhB49euDXX39FbGwsfH19cfr0aaxevRqtW7fO8RmEmdFoNKhTpw5++OEHpKamonjx4ti7dy/CwsKyvY3ChQujSZMm2Lx5M+zt7XPU7ElLS8PatWsBvLyP1t27d7Fjxw5cunQJ9erVM2hK5OQYqVatGoCXZ1z6+fnB2NgYnTp1yvbxoNFoMG/ePPTp0wc1atRA586dUaBAAVy8eBEvXrzA6tWr5f1s3LgRw4cPR40aNWBjY4OWLVtmuNbZs2ejadOm8PHxQe/evZGYmIiff/4ZdnZ2mDx5crbfszdp3rw5TExMsH//fvTr1w/Ay8ZXx44dkZqaihYtWmDTpk0ZvrZo0aJo1KhRto+/woULY9y4cZgyZQqaNGmCL774AqGhoVi0aBFq1KiRK01gvZkzZ+LQoUOoVasW+vbti3LlyiE6Ohrnz5/H/v37ER0dDQDo27cvFi5ciO7du+PcuXNwcnLCmjVrYGVllWu1ZGXcuHFYvXo1wsLC4O7unum8li1bonz58pg6dSru3r2LTz75BLdu3cLChQvh5OSE3r17y3ObNm0KZ2dn1KpVC0WKFMG9e/ewcuVKPHz40KDBBgDff/89Pv30U/j6+qJfv3548OAB5s6di8aNG6NJkybyvFq1aqF9+/YYN24coqKiUKpUKaxevRrh4eFYsWLFG9eZ2bHfr18/LF26FP7+/jh37hzc3d2xZcsWHD9+HPPnz5fv2ZmZIUOGICkpCZUrV0ZqairWr18vH3uv3vfS2dkZQ4cOxezZs5GamooaNWpg27ZtOHr0KNatW2dwadb//e9/2Lx5M+rVq4chQ4YgISEBs2fPhre3N3r27PlW2wSAffv2wdXV1aBJS0REREQqIIiIiIiI8phBgwaJzP5XtUePHsLNzU1+HBYWJgCI2bNnG8w7dOiQACA2b95sML5y5UoBQJw5cybdfD8/P2FnZycsLCxEyZIlhb+/vzh79myWtR4/flwMGjRIVKhQQdjZ2QlTU1Ph6uoq/P39xe3bt9PNz85+evToIaytrbO1fr1ff/1VVKtWTVhaWgpbW1vh7e0tRo8eLR4+fCiEEOL8+fPiq6++Eq6ursLc3FwUKVJEtGjRIsv1PXr0SJiYmIhu3bplOufFixfCyspKfPnll0IIISZNmiQAiMePHxvM07/vYWFh8lhqaqqYMmWK8PDwEKampsLFxUWMGzdOJCUlGbzWzc1NNG/ePN2+AYhBgwYZjGV0PDx48EB8+eWXwt7eXtjZ2Yn27duLhw8fCgBi0qRJWdaot2nTJgFA9OvXL9P34nU9evQQAOQvKysr4e7uLtq2bSu2bNkitFpthq/LzjGSlpYmvvnmG1G4cGEhSVK6vLzpeNDbsWOH+PTTT4WlpaXQaDSiZs2a4o8//pCfT0hIEJ07dxb29vYCgHzs6d/nlStXGmxv//794rPPPpO317JlS3H16lWDOTk5RjLzxRdfiAYNGsiP9fW86cvX11d+TXaPPyGEWLhwofD09BSmpqaiaNGiYsCAASImJsZgjq+vryhfvny61+bk+H306JEYNGiQcHFxEaampsLR0VE0aNBA/Prrrwbz7t69K7744gthZWUlChUqJIYMGSJ2794tAIhDhw5l+d5l9D0ws+85+s/qVfrjOjufU3R0tBg2bJgoU6aMMDc3F4UKFRKdOnUSd+7cMZi3cOFC8fnnn4tChQoJExMTUbhwYdGyZUtx5MiRDLd79OhR8emnnwoLCwtRuHBhMWjQIBEfH59uXmJiohg5cqRwdHQU5ubmokaNGmL37t1vrFuIzI99IV5+Tj179hSFChUSZmZmwtvbO10WMrNy5UpRqVIlYW1tLWxtbUWDBg3EwYMHM5yr1WrF9OnThZubmzAzMxPly5cXa9euzXDu5cuXRePGjYWVlZWwt7cXXbp0EZGRkW+9Ta1WK5ycnMSECROytS4iIiIiyjskId5w53QiIiIiIqKP3Pbt29G6dWscOXIEtWvXVrqcj97Ro0dRt25dXL9+Xb5UKhHlnm3btqFz5864ffs2nJyclC6HiIiIiHKAjT8iIiIiIqI3aNGiBa5du4Zbt25BkiSlyyH83+Uhly1bpnQpRPmOj48PateujR9++EHpUoiIiIgoh3iPPyIiIiIiokxs2LABly5dQmBgIBYsWMCmXx6ya9cupUsgyrdOnjypdAlERERE9JZ4xh8REREREVEmJEmCjY0NOnbsiCVLlsDEhL87SURERERERHkX/9ZKRERERESUCf6eJBEREREREamJkdIFEBEREREREREREREREdG74xl/uUSn0+Hhw4ewtbXlfT+IiIiIiIiIiIiIiIg+ckIIPHv2DMWKFYOR0Yc5F4+Nv1zy8OFDuLi4KF0GERERERERERERERER5SH379+Hs7PzB9kXG3+5xNbWFgAQHh6OAgUKKFwNEeWEVqvFlStXUL58eRgbGytdDhHlAPNLpF7ML5F6Mb9E6sX8EqkX80ukTjExMXB3d5d7SB8CG3+5RH95T41GA41Go3A1RJQTWq0WNjY20Gg0/B8nIpVhfonUi/klUi/ml0i9mF8i9WJ+idRJq9UCwAe9RdyHuaAoEREREREREREREREREb1XbPzlsg/ZtSWi3CFJEhwcHJhfIhVifonUi/klUi/ml0i9mF8i9WJ+idRJicxKQgjxwfeaD8XHx8POzg5xcXG81CcREREREREREREREdFHToneEe/xl8t0Op3SJRBRDul0Ojx48ADOzs4wMuKJ0ERqwvwSqRfzS6RezC+RejG/pHZarRapqalKl6EInU6HyMhIODo6Mr9EeYyZmVmmuVSiZ8TGXy7jCZRE6iOEQHR0NIoXL650KUSUQ8wvkXoxv0TqxfwSqRfzS2olhEBkZCRiY2OVLkUxQgikpqYiKSmJl/skymOMjIzg4eEBMzOzdM8p0TNi44+IiIiIiIiIiIiI8ix9069IkSKwsrL6KBtfQggkJSXBwsLio1w/UV6l0+nw8OFDREREwNXVNU/kk40/IiIiIiIiIiIiIsqTtFqt3PQrWLCg0uUoRggBIQQbf0R5UOHChfHw4UOkpaXB1NRU6XLAiwHnMn7TJVIfSZLg6OjI/BKpEPNLpF7ML5F6Mb9E6sX8khrp7+lnZWWlcCXKywsNBSJKT3+JT61Wm+45JX7m8oy/XMYbqxKpj5GRERwdHZUug4jeAvNLpF7ML5F6Mb9E6sX8kpp97A1rSZLY+CPKo7L6/qREz4hdqlyWUUeXiPI2rVaL27dvM79EKsT8EqkX80ukXswvkXoxv0Tqpb/HnxBC6VKIKAeU+JnLxh8REYBnz54pXQIRvSXml0i9mF8i9WJ+idSL+SVSL51Op3QJRKQCbPwREREREREREREREeUx4eHhkCQJwcHB2X7NqlWrYG9vr3gdr6tbty6GDh2aazXlBZMnT0blypWVLoMoHTb+iIiIiIiIiIiIiIjeg/v376NXr14oVqwYzMzM4ObmhiFDhuDp06dvfK2LiwsiIiJQoUKFbO+vY8eOuHHjxruUnCP6pmBWX6tWrcKff/6JadOmfbC69LRaLWbOnAlPT09YWlrCwcEBtWrVwvLly9952yNHjsSBAwdyoUqi3GWidAH5zcd+k1kiNZIkCS4uLswvkQoxv0TqxfwSqRfzS6RezC/Rh3Xnzh34+PigTJky+OOPP+Dh4YErV65g1KhR2LVrF/799184ODhk+NqUlBSYmZnB0dERwMt7/JmZmb1xn5aWlrC0tMzVdWRF35zUmzNnDnbv3o39+/fLY3Z2dh+0pldNmTIFS5cuxcKFC1G9enXEx8fj7NmziImJeettCiGg1WphY2MDGxubXKyW8iMlfubyjL9cZmTEt5RIbYyMjFCwYEHml0iFmF8i9WJ+idSL+SVSL+aX8gshgKSkD/8lRM7qHDRoEMzMzLB37174+vrC1dUVTZs2xf79+/Hff/9h/Pjx8lx3d3dMmzYN3bt3h0ajQb9+/QwusSlJEkxMTPD333+jdOnSsLCwQL169bB69WpIkoTY2FgA6S/1qb8c5Zo1a+Du7g47Ozt06tTJ4H6fu3fvxueffw57e3sULFgQLVq0wO3bt7O1RmNjYzg6OspfNjY2MDExMRiztLRMd6lPd3d3fPfdd+jevTtsbGzg5uaGHTt24PHjx2jVqhVsbGxQsWJFnD171mB/x44dQ+3atWFpaQkXFxcMHjwYz58/z7S+HTt2YODAgWjfvj08PDxQqVIl9O7dGyNHjpTn6HQ6zJgxAx4eHrC0tESlSpWwZcsW+fmgoCBIkoRdu3ahWrVqMDc3x7FjxzK81Ofy5cvh5eUFCwsLeHp6YtGiRfJzKSkpCAgIgJOTEywsLODm5oYZM2Zk630m9VLiZy7P+MtlWq1W6RKIKIe0Wi1u3ryJ0qVLw9jYWOlyiCgHmF8i9WJ+idSL+SVSL+aX8ovkZKB9+w+/382bAQuL7M2Njo7Gnj178P3336c7283R0RFdunTBxo0bsWjRIvmMoDlz5mDixImYNGlSuu0JIXD9+nW0a9cOQ4YMQZ8+fXDhwgWDBlZmbt++jW3btuGff/5BTEwMOnTogJkzZ+L7778HADx//hzDhw9HxYoVkZCQgIkTJ+LLL79EcHDwe21azJs3D9OnT8e3336LefPmoVu3bvj000/Rq1cvzJ49G2PGjEH37t1x5coVSJKE27dvo0mTJvjuu+/w22+/4fHjxwgICEBAQABWrlyZ4T4cHR1x8OBBDBw4EIULF85wzowZM7B27VosWbIEpUuXxpEjR9C1a1cULlwYvr6+8ryxY8dizpw5KFGiBAoUKICgoCCD7axbtw4TJ07EwoULUaVKFVy4cAF9+/aFtbU1evTogZ9++gk7duzApk2b4Orqivv37+P+/fu59n5S3qREz4iNPyIiAElJSUqXQERvifklUi/ml0i9mF8i9WJ+iT6MmzdvQggBLy+vDJ/38vJCTEwMHj9+jCJFigAA6tevjxEjRshzwsPDDV6zfPlylC1bFrNnzwYAlC1bFpcvX5YbeJnR6XRYtWoVbG1tAQDdunXDgQMH5Ne1bdvWYP5vv/2GwoUL4+rVqzm6v2BONWvWDP379wcATJw4EYsXL0aNGjXQ/v93dceMGQMfHx88evQIjo6OmDFjBrp06SKfOVi6dGn89NNP8PX1xeLFi2GRQVf2xx9/RLt27eDo6Ijy5cvj008/RatWrdC0aVMAQHJyMqZPn479+/fDx8cHAFCiRAkcO3YMS5cuNWj8TZ06FY0aNcp0PZMmTcLcuXPRpk0bAICHhweuXr2KpUuXokePHrh37x5Kly6Nzz//HJIkwc3N7d3fRKIM5JnG38yZMzFu3DgMGTIE8+fPB/Dyf0RGjBiBDRs2IDk5GX5+fli0aBGKFi0qv+7evXsYMGAADh06BBsbG/To0QMzZsyAicn/LS0oKAjDhw/HlStX4OLiggkTJsDf399g/7/88gtmz56NyMhIVKpUCT///DNq1qz5IZZORERERERERERERNlkbv7y7Dsl9ptTIgfXB61evXqWz9+8eTPdnOz8G7a7u7vc9AMAJycnREVFGWx34sSJOHXqFJ48eQKdTgfg5b+9v8/GX8WKFeU/6//N39vbO91YVFQUHB0dcfHiRVy6dAnr1q2T5wghoNPpEBYWlmGTtVy5crh8+TLOnTuH48eP48iRI2jZsiX8/f2xfPly3Lp1Cy9evEjX0EtJSUGVKlUMxrL6fJ4/f47bt2+jd+/e6Nu3rzyelpYGOzs7AIC/vz8aNWqEsmXLokmTJmjRogUaN278xveJKKfyROPvzJkzWLp0qUHQAWDYsGEIDAzE5s2bYWdnh4CAALRp0wbHjx8H8PIUyebNm8PR0REnTpxAREQEunfvDlNTU0yfPh0AEBYWhubNm+Prr7/GunXrcODAAfTp0wdOTk7w8/MDAGzcuBHDhw/HkiVLUKtWLcyfPx9+fn4IDQ2Vf9uCiIiIiIiIiIiIiJQnSdm/5KZSSpUqBUmScO3aNXz55Zfpnr927RoKFChgcPlJa2vr91KLqampwWNJkuTmHgC0bNkSbm5uWLZsGYoVKwadTocKFSogJSXlvdSTUV36y51mNKavNSEhAf3798fgwYPTbcvV1TXT/RgZGaFGjRqoUaMGhg4dirVr16Jbt24YP348EhISAACBgYEoXry4wevMX+v0ZvX56LezbNky1KpVy+A5/aWVq1atirCwMOzatQv79+9Hhw4d0LBhQ4P7CRLlBsUbfwkJCejSpQuWLVuG7777Th6Pi4vDihUrsH79etSvXx8AsHLlSnh5eeHff//FJ598gr179+Lq1avYv38/ihYtisqVK2PatGkYM2YMJk+eDDMzMyxZsgQeHh6YO3cugJenUB87dgzz5s2TG38//vgj+vbti549ewIAlixZgsDAQPz2228YO3ZshnUnJycjOTlZfhwfHw/g5W8Y6K/ZKkkSjIyMoNPpDH6zQz/++rVdMxs3MjKCJEkZjgMw+Cad1bixsbH8GxCvj79eY2bjXBPXlB/XJISAh4dHjtaa19eUVe1cE9eUn9YkhDC4NEZ+WNPrNXJNXFN+XZM+v/r/f84Pa8qPnxPXxDVlVru7u3uGf/9V85ry4+fENXFNr9ee1d9/1bqmrMa5pvyxJq1WK//9TwgBSZIyPIvufY/nhCRJcHBwQKNGjbBo0SIMHToUlpaW8rYjIyOxbt06dOvWzeB1+jW+Tj/m6emJ3bt3G4ydPn3a4PGb/vv62NOnTxEaGopff/0VtWvXBgAcO3bMoJ5X579e3+vvV2Z/zqwm/Z/1Tb7X9/fqWNWqVXH16lWULFkyw88pu5+b/szA58+fw8vLC+bm5rh79y7q1KmTbk2vrzmzPxcpUgTFihXD7du30blz5wy3AwC2trbo2LEjOnTogLZt26Jp06Z4+vQpHBwccu3YUyIf7ztPean2rI49rVZr8D1R/33yQ1O88Tdo0CA0b94cDRs2NGj8nTt3DqmpqWjYsKE85unpCVdXV5w8eRKffPIJTp48CW9vb4NLf/r5+WHAgAG4cuUKqlSpgpMnTxpsQz9Hfx3glJQUnDt3DuPGjZOfNzIyQsOGDXHy5MlM654xYwamTJmSbvzq1auwsbEBADg4OMDV1RUPHjxAdHS0PMfR0RGOjo4IDw/Hs2fP5HEXFxcULFgQN2/eNLjeeokSJaDRaHD16lWDg6Zs2bIwMzNDSEiIQQ3e3t5ISUlBaGioPGZsbAxvb288e/YMd+7ckcctLCzg6emJmJgYgxuJ2traomTJkoiKikJkZKQ8zjVxTfl5TUKIfLem/Pg5cU1cE9fENXFN+WNNly9fzndryo+fE9fENWW2pvj4eIP7DuWHNeXHz4lr4poyW1NycnK+W1N+/Jy4ppdrunfvHoyMjJCUlARJkmBhYYG0tDSkpqYabMfc3BwpKSkGtZiamsLU1BTJyckGjUUzMzOYmJggKSnJ4B/wzc3NYWxsjMTERIPaLSwsIElSunFLS0sIIdLdO9PKygo6nQ6zZ89GgwYN0LhxY0yaNAmenp64dOkSRo8ejWLFimHChAlITk6W702Xmpoq70N/lhjw8pZYiYmJ8Pf3x/z58zFmzBh07doVwcHBWLVqFYD/a6jq3xf9dvTre7X2V987CwsLFCxYEIsXL0aBAgUQFRUl/1t5SkoKEhMT5fXpdDqD7UiSBEtLS2i1WvnswLS0NHmfr35OOp1OrjElJQVCCHm9+s/p1X0CkD/L5ORkJCYmYvDgwahXrx4CAgLQt29fGBkZ4fr16zh48CB+/PHHDD+nLl26oHbt2vDx8UGBAgUQHh6OSZMmoXTp0vD09IQkSRgyZAiGDx+OpKQkfPbZZ0hOTsaRI0dgZWWFrl27GpwA9OqaXl1rSkoKxo8fj5EjR8LKygpNmzaFTqfDyZMnER0djcGDB+Onn36Cs7MzqlevjpSUFGzYsAFFixaFubk5dDpdrh57r9ac0ecEvOyDqClPeXlNwMus3bhxw6DRp/+Z+6FJ4l3bre9gw4YN+P7773HmzBlYWFigbt26qFy5MubPn4/169ejZ8+e6d6UmjVrol69epg1axb69euHu3fvYs+ePfLzL168gLW1NXbu3ImmTZuiTJky6Nmzp0Fjb+fOnWjevDlevHiBmJgYFC9eHCdOnJBv3gkAo0ePxuHDh3Hq1KkMa8/ojD8XFxc8fvwYBQoUAKC+357Jj78RxDVxTdlZk1arxfXr11GuXLl0v4Gh1jVlVTvXxDXlpzVptVpcu3YN5cqVg6mpab5Y0+s1ck1cU35dU2pqKq5duwYvLy8YGxvnizXlx8+Ja+KaMqpdCIErV67A09NT/kdJta8pP35OXBPXlFHtWf39V61rymqca8ofa0pMTMTdu3fh4eEhNwyUOMsnJ17dxt27dzF58mTs3r0b0dHRcHR0RKtWrTBp0iQULFhQnu/u7o4hQ4bIJ6zoX+vh4YHz58+jUqVKSEpKwt69ezFq1Cjcv38fPj4+6NChAwYOHIgXL17A0tISK1euxLBhwxATEwMAmDx5MrZv344LFy7I250/fz4WLFiAsLAwAMD+/fsxZMgQ3LlzB2XLlsWCBQtQr149/Pnnn2jdujXCw8NRokQJnD9/HpUrV87y/dLvLzg42GC8Xr16qFSpEhYsWAAhXp59/Op6JUmCJEnyPgFkuN8zZ85gwoQJOHnyJIQQKFmyJDp06ID//e9/GX4Wy5Ytw4YNG3D58mXExcXB0dER9evXx6RJk+QrGAgh8NNPP2HJkiW4c+cO7O3tUbVqVYwbNw516tRBUFAQ6tevj5iYGPl+fZmtdf369ZgzZw6uXr0Ka2treHt7Y8iQIfjyyy+xbNkyLF68GDdv3oSxsTFq1KiBH374Qb6XYG4fe3l5PCfyWu0ZjSclJSE8PByurq5yIx94+X0vJiYGBQsWRFxcHDQaTcaLzGWKNf7u37+P6tWrY9++ffK9/eqqqPH3uvj4eNjZ2SE6Olpu/BGROmi1WoSEhMDb29vgt6mIKO9jfonUi/klUi/ml0i9mF9So6SkJISFhcmNv4+VEAKJiYnyJUP1vv/+eyxZssTgbE8i+rCy+j4VExMDBweHD9r4M/oge8nAuXPnEBUVhapVq8LExAQmJiY4fPgwfvrpJ5iYmKBo0aJISUlBbGyswesePXoER0dHAC9PP3/06FG65/XPZTVHo9HA0tIShQoVgrGxcYZz9NsgIiIiIiIiIiIiIlLaokWLcObMGdy5cwdr1qzB7Nmz0aNHD6XLIqI8RLHGX4MGDRASEoLg4GD5q3r16ujSpYv8Z1NTUxw4cEB+TWhoKO7duyefmefj44OQkBBERUXJc/bt2weNRoNy5crJc17dhn6OfhtmZmaoVq2awRydTocDBw4YnAFIRERERERERERERKSkmzdvolWrVihXrhymTZuGESNGYPLkyUqXRUR5iKL3+Hvdq5f6BIABAwZg586dWLVqFTQaDb755hsAwIkTJwC8vDxB5cqVUaxYMfzwww+IjIxEt27d0KdPH0yfPh0AEBYWhgoVKmDQoEHo1asXDh48iMGDByMwMBB+fn4AgI0bN6JHjx5YunQpatasifnz52PTpk24fv06ihYtmq3a9Zf6jI2NNbjOLxHlfUK8vGms/jrxRKQezC+RejG/ROrF/BKpF/NLasRLfb6kvxed/l54RJR3ZPV9Ki4uDvb29h/0Up8mH2Qvb2nevHkwMjJC27ZtkZycDD8/PyxatEh+3tjYGP/88w8GDBgAHx8fWFtbo0ePHpg6dao8x8PDA4GBgRg2bBgWLFgAZ2dnLF++XG76AUDHjh3x+PFjTJw4EZGRkahcuTJ2796d7aYfEamfmZmZ0iUQ0VtifonUi/klUi/ml0i9mF8i9WLDj4iyI0+d8adm+jP+oqOjUaBAAaXLIaIc4M3NidSL+SVSL+aXSL2YXyL1Yn5JjXjG30tCCCQmJsLS0pINQKI8JqvvUzExMXBwcPigZ/wpdo8/IiIiIiIiIiIiIiIiIso9bPwRERERERERERERERER5QNs/BERERERERERERERERHlA2z85TIjI76lRGpjZGQEb29v5pdIhZhfIvVifonUi/klUi/ml0jdLC0tlS6BiHJIiZ+5/ClPRAQgJSVF6RKI6C0xv0TqxfwSqRfzS6RezC+RegkhMn3O398frVu3lh/XrVsXQ4cOff9FvSYoKAiSJCE2NvaD75uIXmLjL5fpdDqlSyCiHNLpdAgNDWV+iVSI+SVSL+aXSL2YXyL1Yn6JPix/f39IkgRJkmBmZoZSpUph6tSpSEtLe6vtJSUlZXvun3/+iWnTpmVr7odu1rm7u0OSJGzYsCHdc+XLl4ckSVi1atUHqYXofVPiZy4bf0RERERERERERERE70GTJk0QERGBmzdvYsSIEZg8eTJmz56d4dzcPCPXwcEBtra2uba93Obi4oKVK1cajP3777+IjIyEtbW1QlUR5Q9s/BERERERERERERGR6jxPeZ7pV1JaUrbnJqYmvnHu2zI3N4ejoyPc3NwwYMAANGzYEDt27ADwf5fn/P7771GsWDGULVsWAHD//n106NAB9vb2cHBwQKtWrRAeHi5vU6vVYvjw4bC3t0fBggUxevTodJcBff1Sn8nJyRgzZgxcXFxgbm6OUqVKYcWKFQgPD0e9evUAAAUKFIAkSfD39wfw8kylGTNmwMPDA5aWlqhUqRK2bNlisJ+dO3eiTJkysLS0RL169QzqzEqXLl1w+PBh3L9/Xx777bff0KVLF5iYmBjMjY2NRZ8+fVC4cGFoNBrUr18fFy9elJ+/ffs2WrVqhaJFi8LGxgY1atTA/v37Dbbh7u6O6dOno1evXrC1tYWrqyt+/fXXbNVKpDYmb55CRJT/GRsbK10CEb0l5pdIvZhfIvVifonUi/ml/MRmhk2mzzUr3QyBnQPlx0XmFMGL1BcZzvV180WQf5D82H2BO568eGIwR0zK/P56OWFpaYmnT5/Kjw8cOACNRoN9+/YBAFJTU+Hn5wcfHx8cPXoUJiYm+O6779C0aVP8+++/sLS0xNy5c7Fq1Sr89ttv8PLywty5c/HXX3+hfv36me63e/fuOHnyJH766SdUqlQJYWFhePLkCVxcXLB161a0bdsWoaGh0Gg0sLS0BADMmDEDa9euxZIlS1C6dGkcOXIEXbt2ReHCheHr64v79++jTZs2GDRoEPr164ezZ89ixIgR2XofihYtCj8/P6xevRoTJkzAixcvsHHjRhw+fBi///67wdz27dvD0tISu3btgp2dHZYuXYoGDRrgxo0bcHBwQEJCApo1a4bvv/8e5ubm+P3339GyZUuEhobC1dVV3s7cuXMxbdo0/O9//8OWLVswYMAA+Pr6yg1XovyCjb9cxv95IlIfY2NjeHt7K10GEb0F5pdIvZhfIvVifonUi/klUo4QAgcOHMCePXvwzTffyOPW1tZYvnw5zMzMAABr166FTqfD8uXLIUkSAGDlypWwt7fH6dOn0bhxY8yfPx/jxo1DmzZtAABLlizBnj17Mt33jRs3sGnTJuzbtw8NGzYEAJQoUUJ+3sHBAQBQpEgR2NvbA3h5huD06dOxf/9++Pj4yK85duwYli5dCl9fXyxevBglS5bE3LlzAQBly5ZFSEgIZs2ala33pFevXhgxYgTGjx+PLVu2oGTJkqhcubLBnGPHjuH06dOIioqCubk5AGDOnDnYtm0btmzZgn79+qFSpUqoVKmS/Jpp06bhr7/+wo4dOxAQECCPN2vWDAMHDgQAjBkzBvPmzcOhQ4fY+KP3SomeERt/uez1U6qJKO8TQuDZs2ewtbWV/4eKiNSB+SVSL+aXSL2YXyL1Yn4pv0kYl5Dpc8ZGhv/YHjUyKtO5RpLhHbHCh4S/U12v+ueff2BjY4PU1FTodDp07twZkydPlp/39vaWm34AcPHiRdy6dSvd/fmSkpJw8+ZN1KxZExEREahVq5b8nImJCapXr57pv00HBwfD2NgYvr6+2a771q1bePHiBRo1amQwnpKSgipVqgAArl27ZlAHALlJmB3NmzdH//79ceTIEfz222/o1atXujkXL15EQkICChYsaDCemJiI27dvAwASEhIwefJkBAYGIiIiAmlpaUhMTMS9e/cMXlOxYkX5z5IkwdHREVFRmR8XRLlBiZ4RG3+5TKfTKV0CEeWQTqfDnTt34O3tzbN2iVSG+SVSL+aXSL2YXyL1Yn4pv7E2s1Z87pvUq1cPixcvhpmZGYoVK5bu/nXW1ob7SkhIQLVq1bBu3TqDcSEEbGwyv7RpVvSX7syJhISXTdXAwEAUL17c4Dn9mXfvysTEBN26dcOkSZNw6tQp/PXXXxnW4eTkhKCgoHTP6c9OHDlyJPbt24c5c+agVKlSsLS0RLt27ZCSkmIw39TU1OCxJEn893x675Q4xtj4IyIiIiIiIiIiIiJ6D6ytrVGqVKlsz69atSo2btyIIkWKQKPRyONCCCQmJsLS0hJOTk44deoU6tSpAwBIS0vDuXPnULVq1Qy36e3tDZ1Oh8OHD8uX+nyV/oxDrVYrj5UrVw7m5ua4d+9epmcKenl5YceOHQZj//77b7bXCry83OecOXPQsWNHFChQIN3zVatWRWRkJExMTODu7p7hNo4fPw5/f398+eWXAF42C8PDw3NUB1F+YvTmKURERERERERERERE9L516dIFhQoVQqtWrXD06FGEhYUhKCgIgwcPxn///QcAGDJkCGbOnIlt27bh+vXrGDhwIGJjYzPdpru7O3r06IFevXph27Zt8jY3bdoEAHBzc4MkSfjnn3/w+PFjJCQkwNbWFiNHjsSwYcOwevVq3L59G+fPn8fPP/+M1atXAwC+/vpr3Lx5E6NGjUJoaCjWr1+PVatW5Wi9Xl5eePLkCVauXJnh8w0bNoSPjw9at26NvXv3Ijw8HCdOnMD48eNx9uxZAEDp0qXx559/Ijg4GBcvXkTnzp15Jh991Nj4IyICYGFhoXQJRPSWmF8i9WJ+idSL+SVSL+aXKG+zsrLCkSNH4OrqijZt2sDLywu9e/dGUlKSfAbgiBEj0K1bN/To0QM+Pj6wtbWVz3bLzOLFi9GuXTsMHDgQnp6e6Nu3L54/fw4AKF68OKZMmYKxY8eiaNGiCAgIAABMmzYN3377LWbMmAEvLy80adIEgYGB8PDwAAC4urpi69at2LZtGypVqoQlS5Zg+vTpOV5zwYIFM70cqSRJ2LlzJ+rUqYOePXuiTJky6NSpE+7evYuiRYsCAH788UcUKFAAn376KVq2bAk/P79Mz34k+hhIQok7C+ZD8fHxsLOzQ1xcnMEp2ERERERERERERET0dpKSkhAWFgYPDw82rokoT8rq+5QSvSOe8ZfLeAoxkfrodDo8ffqU+SVSIeaXSL2YXyL1Yn6J1Iv5JVIvIQTS0tLA83iI1EWJn7ls/OUyfuMlUh8hBO7fv8/8EqkQ80ukXswvkXoxv0TqxfwSqVtKSorSJRBRDinxM5eNPyIiIiIiIiIiIiIiIqJ8gI0/IiIiIiIiIiIiIiIionyAjT8iIgC2trZKl0BEb4n5JVIv5pdIvZhfIvVifonUy8iI/5xPRG9monQB+Y2xsbHSJRBRDhkbG6NkyZJKl0FEb4H5JVIv5pdIvZhfIvVifonUS5IkWFhYKF0GEeWQEj0j/opALtPpdEqXQEQ5pNPpEBkZyfwSqRDzS6RezC+RejG/ROrF/BKplxACqampEEIoXQoR5YASP3PZ+Mtl/MZLpD5CCERGRjK/RCrE/BKpF/NLpF7ML5F6Mb9E6paamqp0CUSUQ0r8zGXjj4iIiIiIiIiIiIiIiCgfYOOPiIiIiIiIiIiIiIiyJSgoCJIkITY29oPud9WqVbC3t3+nbYSHh0OSJAQHB2c6R6n1EeUWNv5ymSRJSpdARDkkSRIcHByYXyIVYn6J1Iv5JVIv5pdIvZhfog/L398frVu3zrXtGRsb59q2MiNJUpZfkydPfu81EOUnSvzMNfnge8znjIzYSyVSGyMjI7i6uipdBhG9BeaXSL2YXyL1Yn6J1Iv5JVIvSZJgbm6eq9tMSUmBmZmZwVhERIT8540bN2LixIkIDQ2Vx2xsbHD27Nlc2RfRx0CJnhG7VLlMp9MpXQIR5ZBOp8O9e/eYXyIVYn6J1Iv5JVIv5pdIvZhforzl8OHDqFmzJszNzeHk5ISxY8ciLS0NAPDPP//A3t4eWq0WAHDhwgVIkoQxY8bIr+/Tpw+6du0qPz527Bhq164NS0tLuLi4YPDgwXj+/Ln8vLu7O6ZNm4bu3btDo9GgX79+6WpydHSUv+zs7CBJksGYjY2NPPfcuXOoXr06rKys8Omnnxo0CCdPnozKlStj+fLl8PDwgIWFBQAgNjYWffr0QeHChaHRaFC/fn1cvHhRft3FixdRr1492NraQqPRoFq1aukajXv27IGXlxdsbGzQpEkTg2alTqfD1KlT4ezsDHNzc1SuXBm7d+/O8nPYuXMnypQpA0tLS9SrVw/h4eFZzifKCSV+5rLxl8uEEEqXQEQ5JIRAdHQ080ukQswvkXoxv0TqxfwSqRfzS/lOYHng3LD/e3z5u5djydEvHz+79fLxjV/+b85Jf2Df5//3+OGel3Me7vm/sX2fv5yn9+rrc8l///2HZs2aoUaNGrh48SIWL16MFStW4LvvvgMA1K5dG8+ePcOFCxcAvGwSFipUCIcPH5a3cfjwYdStWxcAcPv2bTRp0gRt27bFpUuXsHHjRhw7dgwBAQEG+50zZw4qVaqECxcu4Ntvv32nNYwfPx5z587F2bNnYWJigl69ehk8f+vWLWzduhV//vmnfE+99u3bIyoqCrt27cK5c+dQtWpVNGjQANHRLz+zLl26wNnZGWfOnMG5c+cwduxYmJqaytt88eIF5syZgzVr1uDIkSO4d+8eRo4cKT+/YMECzJ07F3PmzMGlS5fg5+eHL774Ajdv3sxwDffv30ebNm3QsmVLBAcHo0+fPhg7duw7vS9Er1LiZy4v9UlERERERERERERE9AEtWrQILi4uWLhwISRJgqenJx4+fIgxY8Zg4sSJsLOzQ+XKlREUFITq1avj8OHDCAgIwPTp05GQkIC4uDjcunULvr6+AIAZM2agS5cuGDp0KACgdOnS+Omnn+Dr64vFixfLZ9zVr18fI0aMyJU1fP/99/L+x44di+bNmyMpKUneV0pKCn7//XcULlwYwMszEk+fPo2oqCj5sqVz5szBtm3bsGXLFvTr1w/37t3DqFGj4OnpKa/jVampqViyZAlKliwJAAgICMDUqVPl5+fMmYMxY8agU6dOAIBZs2bh0KFDmD9/Pn75JX0Dd/HixShZsiTmzp0LAChbtixCQkIwa9asXHmPiJTAxh8RERERERERERERqU/zK4aPK0x4+aVnWyr9HJ9Vho+L+QHFXpvT6Jjh4zKD3qnMjFy7dg0+Pj6QJEke++yzz5CQkIAHDx7A1dUVvr6+CAoKwogRI3D06FFMnDgRf/31F44dO4bo6GgUK1ZMboxdvHgRly5dwrp16+TtCSGg0+kQFhYGLy8vAED16tVzbQ0VK1aU/+zk5AQAiIqKku8l6ubmJjf99DUmJCSgYMGCBttJTEzE7du3AQDDhw9Hnz59sGbNGjRs2BDt27eXm3wAYGVlZfDYyckJUVFRAID4+Hg8fPgQn332mcH2P/vsM4PLib7q2rVrqFWrlsGYj49P9t4AojyKjb9c9uo3aiJSB/21yplfIvVhfonUi/klUi/ml0i9mF8idalbty5+++03XLx4EaampqhQoYLcDIyJiZHPtgOAhIQE9O/fH4MHD063HX0jDgCsra1zrb5XL8Gp/77y6v3MXt9XQkICnJycEBQUlG5b9vb2AF7eG7Bz584IDAzErl27MGnSJGzYsAFffvllun3q98vLF1NepsTPXDb+cpmREW+bSKQ2RkZGcHR0VLoMInoLzC+RejG/ROrF/BKpF/NLlHd4eXlh69atEELIjYHjx4/D1tYWzs7OAP7vPn/z5s2Dr68vTE1NUa9ePcycORMxMTEGl+ysWrUqrl69ilKlSimynuyoWrUqIiMjYWJiAnd390znlSlTBmXKlMGwYcPw1VdfYeXKlXLjLysajQbFihXD8ePHDZqix48fR82aNTN8jZeXF3bs2GEw9u+//2ZvQUTZoETPiF2qXKbVapUugYhySKvV4vbt28wvkQoxv0TqxfwSqRfzS6RezC/RhxcXF4fg4GCDr/v372PgwIG4f/8+vvnmG1y/fh3bt2/HpEmTMHz4cLlRUKBAAVSsWBHr1q2Dr68vkpKSULt2bZw/fx43btwwaG6NGTMGJ06cQEBAAIKDg3Hz5k1s374dAQEBSi09nYYNG8LHxwetW7fG3r17ER4ejhMnTmD8+PE4e/YsEhMTERAQgKCgINy9exfHjx/HmTNn5MuUZseoUaMwa9YsbNy4EaGhoRg7diyCg4MxZMiQDOd//fXXuHnzJkaNGoXQ0FCsX78eq1atyqUVEynTM+IZf0REAJ49e6Z0CUT0lphfIvVifonUi/klUi/ml+jDCgoKQpUqVQzGevfujeXLl2Pnzp0YNWoUKlWqBAcHB/Tu3RsTJkwwmOvr64vg4GDUrVsXOp0ODg4OKFeuHB49eoSyZcvK8ypWrIjDhw9j/PjxqF27NoQQKFmyJDp27PhB1pkdkiRh586dGD9+PHr27InHjx/D0dERderUQdGiRWFsbIynT5+ie/fuePToEQoVKoQ2bdpgypQp2d7H4MGDERcXhxEjRiAqKgrlypXDjh075Hshvs7V1RVbt27FsGHD8PPPP6NmzZqYPn06evXqlVvLJvrgJMEL4OaK+Ph42NnZITo6GgUKFFC6HCLKAa1Wi5CQEHh7e8PY2FjpcogoB5hfIvVifonUi/klUi/ml9QoKSkJYWFh8PDwgIWFhdLlKEYIgcTERFhaWvI+nUR5TFbfp2JiYuDg4IC4uDhoNJoPUg8v9UlERERERERERERERESUD7Dxl8v42xZE6iNJElxcXJhfIhVifonUi/klUi/ml0i9mF8idTMzM1O6BCLKISV+5vIef7lMf+NVIlIPIyMjFCxYUOkyiOgtML9E6sX8EqkX80ukXswvkXpJkgQTE/5zPpHaKNEzYpcql2m1WqVLIKIc0mq1uH79OvNLpELML5F6Mb9E6sX8EqkX80ukXvp7/AkhlC6FiHJAiZ+5bPwREeHlDViJSJ2YXyL1Yn6J1Iv5JVIv5pdIvdj0I6LsYOOPiIiIiIiIiIiIiIiIKB9g44+IiIiIiIiIiIiIiIgoH2DjL5cpcaNGIno3RkZGKFGiBPNLpELML5F6Mb9E6sX8EqkX80ukbubm5kqXQEQ5pMTPXJMPvsd8TpIkpUsgohySJAkajUbpMojoLTC/ROrF/BKpF/NLpF7ML5F6SZIEY2NjpcsgohxSomfEX+/JZVqtVukSiCiHtFotQkJCmF8iFWJ+idSL+SVSL+aXSL2YXyL1EkLgxYsXEEIoXQoAoG7duhg6dOgH36+7uzvmz5//Ttvw9/dH69ats5yj1Poo/1HiZy4bf0REYNOeSM2YXyL1Yn6J1Iv5JVIv5pfow8msebRq1SrY29t/8HqyY/LkyZAkKcsvIsrb2PgjIiIiIiIiIiIiIvrIaLVa6HQ6g7GRI0ciIiJC/nJ2dsbUqVMNxt5Wamrqu5ZMRNnAxh8RERERERERERERkUL0l56cMmUKChcuDI1Gg6+//hopKSnynHr16mH48OEICAiAnZ0dChUqhG+//dbg0p/JyckYOXIkihcvDmtra9SqVQtBQUHy8/ozDXfs2IFy5crB3Nwc9+7dM6jFxsYGjo6O8pexsTFsbW0NxvR0Oh1Gjx4NBwcHODo6YvLkyQbbkiQJixcvxhdffAFra2t8//33AIDt27ejatWqsLCwQIkSJTBlyhSkpaUBeHlJ08mTJ8PV1RXm5uYoVqwYBg8ebLDdFy9eoFevXrC1tYWrqyt+/fVXg+dDQkJQv359WFpaomDBgujXrx8SEhIyff+fP3+O7t27w8bGBk5OTpg7d24WnxZR3sfGXy4zMuJbSqQ2RkZGKFu2LPNLpELML5F6Mb9E6sX8EqkX80v5zaLyi7B72G758ZHvjmBR+UVIjE4EAETfisai8otw+pfT8pxt/tvw2+e/yY9v7bmFReUX4daeW/LYb5//hm3+2+THr77+fTlw4ACuXbuGoKAg/PHHH/jzzz8xZcoUgznr1q2DiYkJTp8+jQULFuDHH3/E8uXL5ecDAgJw8uRJbNiwAZcuXUL79u3RpEkT3Lx5U57z4sULzJo1C8uXL8eVK1dQpEiRt6559erVsLa2xqlTp/DDDz9g6tSp2Ldvn8GcyZMn48svv0RISAh69eqFo0ePonv37hgyZAiuXr2KpUuXYtWqVXJTcOvWrZg3bx6WLl2KmzdvYtu2bfD29jbY5ty5c1G9enVcuHABAwcOxIABAxAaGgrgZRPPz88PBQoUwJkzZ7B582bs378fAQEBma5j1KhROHz4MLZv3469e/ciKCgI58+ff+v3hehVSvzMNfngeyQiyoPMzMyULoGI3hLzS6RezC+RejG/ROrF/BLlTWZmZvjtt99gZWWF8uXLY+rUqRg1ahSmTZsmNw5cXFwwb948uYkfEhKCefPmoW/fvrh37x5WrlyJe/fuoVixYgBeXrZz9+7dWLlyJaZPnw7g5eU2Fy1ahEqVKr1zzRUrVsSkSZMAAKVLl8bChQtx4MABNGrUSJ7TuXNn9OzZU37cq1cvjB07Fj169AAAlChRAtOmTcPo0aMxadIk3Lt3D46OjmjYsCFMTU3h6uqKmjVrGuy3WbNmGDhwIABgzJgxmDdvHg4dOoSyZcti/fr1SEpKwu+//w5ra2sAwMKFC9GyZUvMmjULRYsWNdhWQkICVqxYgbVr16JBgwYAXjY0nZ2d3/n9IVIKG3+57PVrIhNR3qfT6RASEgJvb28YGxsrXQ4R5QDzS6RezC+RejG/ROrF/FJ+M/DKQIPHdSbUQZ0JdeTHDqUc0s1pvaq1weNSfqVQ6kopg7Fex3oZPK45yLDx9D5UqlQJVlZW8mMfHx8kJCTg/v37cHNzAwBUr14dkiQZzJk7dy60Wi1CQkKg1WpRpkwZg+0mJyejYMGC8mMzMzNUrFgxV2p+fTtOTk6IiooyGKtevbrB44sXL+L48ePyGX7Ay3sNJiUl4cWLF2jfvj3mz5+PEiVKoEmTJmjWrBlatmwJE5P/a2W8ul9JkuDo6Cjv99q1a6hUqZLc9AOAzz77DDqdDqGhoekaf7dv30ZKSgpq1aoljzk4OKBs2bI5fTuIMqREz4iNPyIiIiIiIiIiIiKiXKbRaBAXF5duPDY2FnZ2drm6r4SEBBgbG+PcuXPpGvs2Njbyny0tLQ2ah+/C1NTU4LEkSemaHK824PR1TpkyBW3atEm3PQsLC7i4uCA0NBT79+/Hvn37MHDgQMyePRuHDx+W95ed/RJ9zNj4IyIiIiIiIiIiIiLKZWXLlsXevXvTjZ8/fz7dmXkXL15EYmIiLC0tAQD//vsvbGxs4OLiIs85e/aswWv+/fdflC5dGsbGxqhSpQq0Wi2ioqJQu3bt97Ca3FG1alWEhoaiVKlSmc6xtLREy5Yt0bJlSwwaNAienp4ICQlB1apV37h9Ly8vrFq1Cs+fP5ebjsePH5cvj/q6kiVLwtTUFKdOnYKrqysAICYmBjdu3ICvr+9brpJIWWz8ERERERERERERERHlsgEDBmDhwoUYPHgw+vTpA3NzcwQGBuKPP/7A33//bTA3JSUFvXv3xoQJExAeHo5JkyYhICBAvr8fANy/fx/Dhw/H119/jfPnz+Pnn3/G3LlzAQBlypRBly5d0L17d8ydOxdVqlTB48ePceDAAVSsWBHNmzf/oGvPzMSJE9GiRQu4urqiXbt2MDIywsWLF3H58mV89913WLVqFbRaLWrVqgUrKyusXbsWlpaW8uVO36RLly6YNGkSevTogcmTJ+Px48f45ptv0K1bt3SX+QReng3Zu3dvjBo1CgULFkSRIkUwfvx4g/edSG3Y+Mtl/IZApD5GRkbw9vZmfolUiPklUi/ml0i9mF8i9WJ+iT6sEiVK4MiRIxg/fjwaNmyIlJQUeHp6YvPmzWjSpInB3AYNGqB06dKoU6cOkpOT8dVXX2Hy5MkGc7p164bExETUrFkTxsbGGDJkCPr16yc/v3LlSnz33XcYMWIE/vvvPxQqVAiffPIJWrRo8SGWmy1+fn74559/MHXqVMyaNQumpqbw9PREnz59AAD29vaYOXMmhg8fDq1WC29vb/z9998G9ynMipWVFfbs2YMhQ4agRo0asLKyQtu2bfHjjz9m+prZs2cjISEBLVu2hK2tLUaMGJHhJVqJ3oYSP3MlIYT44HvNh+Lj42FnZ/ders9MRO+XEAJJSUmwsLDItWucE9GHwfwSqRfzS6RezC+RejG/pEZJSUkICwuDh4cHLCwslC7nvfD390dsbCy2bduW6Zy6deuiUqVKmD9/PvNLlMdk9X0qLi4O9vb2iIuLg0aj+SD18Nd7chlvIkqkPjqdDqGhocwvkQoxv0TqxfwSqRfzS6RezC+RuqWlpSldAhHlkBI/c9n4IyIiIiIiIiIiIiIiIsoHeI8/IiIiIiIiIiIiIiKFrFq16o1zDh06hMTExPdfDBGpHs/4IyICYGxsrHQJRPSWmF8i9WJ+idSL+SVSL+aXiIgof+MZf7mM//NEpD7Gxsbw9vZWugwiegvML5F6Mb9E6sX8EqkX80ukXpIkwcrKSukyiCiHlOgZ8Yy/XCaEULoEIsohIQTi4+OZXyIVYn6J1Iv5JVIv5pdIvZhfIvUSQkCr1TK/RCqjRGbZ+MtlOp1O6RKIKId0Oh3u3LnD/BKpEPNLpF7ML5F6Mb9E6sX8EqlbcnKy0iUQUQ4p8TOXjT8iIiIiIiIiIiIiIiKifICNPyIiIiIiIiIiIiIiIqJ8gI0/IiIAFhYWSpdARG+J+SVSL+aXSL2YXyL1Yn7pYxUZGYnvpk1DDS8vlCleHDW8vPDdtGmIjIxUurRskyRJ6RKy5O7ujvnz5ytdxntVt25dDB06VH78IdZ84MABeHl5QavVvtf9UOY6deqEuXPnKl1GtrHxl8uMjY2VLoGIcsjY2Bienp7ML5EKMb9E6sX8EqkX80ukXswvfazmzZ0LN2dnHJs+HUOvX8fihw8x9Pp1HJ0+HW7Ozpj344/vZb/+/v6QJAmSJMHU1BRFixZFo0aN8Ntvv+X4vl9TpkyBj49Prjf/lGjWPXjwAGZmZqhQocIH3e/7cObMGfTr1++97mP06NGYMGGC/L1bq9Vi5syZ8PT0hKWlJRwcHFCrVi0sX778vdahBqtWrZIzp//Kzi+8BAUFoWrVqjA3N0epUqWwatUqg+cnTJiA77//HnFxcTmuSYmfuWz85TLeHJlIfXQ6HZ4+fcr8EqkQ80ukXswvkXoxv0TqxfzSx2je3LmYMX48jmi12J2UhC4AGgDoAmBPUhKOaLWY8b//vbfmX5MmTRAREYHw8HDs2rUL9erVw5AhQ9CiRQukpaVleztCCPlL7VatWoUOHTogPj4ep06dUrqcd1K4cGFYWVm9t+0fO3YMt2/fRtu2beWxKVOmYN68eZg2bRquXr2KQ4cOoV+/foiNjX1vdaSkpLz1a+/du5eLlbyZRqNBRESE/HX37t0s54eFhaF58+aoV68egoODMXToUPTp0wd79uyR51SoUAElS5bE2rVrc1yPEj9z2fjLZfnhGy/Rx0YIgfv37zO/RCrE/BKpF/NLpF7ML5F6Mb/0sYmMjMTYMWPwd3IyamUypxaAv5OTMXb06Pdy2U9zc3M4OjqiePHiqFq1Kv73v/9h+/bt2LVrl8FZRbGxsejTpw8KFy4MjUaD+vXr4+LFiwBeNsqmTp2KS5cuwcjICJIkya/N6nV6f//9N2rUqAELCwsUKlQIX375JYCXl628e/cuhg0bJp8dpXfs2DHUrl0blpaWcHFxweDBg/H8+XP5+aioKLRs2RKWlpbw8PDAunXrsvV+CCGwcuVKdOvWDZ07d8aKFSsMng8PD4ckSfjzzz9Rr149WFlZoVKlSjh58qTBvK1bt6J8+fIwNzeHu7t7usswuru747vvvkP37t1hY2MDNzc37NixA48fP0arVq1gY2ODihUr4uzZs/Jrnj59iq+++grFixeHlZUVvL298ccff2S5ntfPmHzT53Hx4kXUq1cPtra20Gg0qFatmkENr9uwYQMaNWpkcNbajh07MHDgQLRv3x4eHh6oVKkSevfujZEjR8pzdDodfvjhB5QqVQrm5uZwdXXF999/Lz8fEhKC+vXrw9LSEgULFkS/fv2QkJAgP+/v74/WrVvj+++/R7FixVC2bFkAwP3799GhQwfY29vDwcEBrVq1Qnh4eJbvkYeHBxo2bIg1a9bgxYsXWc7NDZIkwdHRUf4qWrRolvOXLFkCDw8PzJ07F15eXggICEC7du0wb948g3ktW7bEhg0bclyPEj9z2fgjIiIiIiIiIiIionxn+bJlqGdqmmnTT68WgLpmZljxgS6VWL9+fVSqVAl//vmnPNa+fXtERUVh165dOHfuHKpWrYoGDRogOjoaHTt2xPDhw+Hl5YWHDx8iIiICHTt2fOPrACAwMBBffvklmjVrhgsXLuDAgQOoWbMmAODPP/+Es7Mzpk6dKp8dBQC3b99GkyZN0LZtW1y6dAkbN27EsWPHEBAQINfr7++P+/fv49ChQ9iyZQsWLVqEqKioN6790KFDePHiBRo2bIiuXbtiw4YNBg1FvfHjx2PkyJEIDg5GmTJl8NVXX8lnSJ47dw4dOnRAp06dEBISgsmTJ+Pbb79Nd3nGefPm4bPPPsOFCxfQvHlzdOvWDd27d0fXrl1x/vx5lCxZEt27d5cbM0lJSahWrRoCAwNx+fJl9OvXD926dcPp06ez+9G+8fPo0qULnJ2dcebMGZw7dw5jx46Fqalppts7evQoqlevbjDm6OiIgwcP4vHjx5m+bty4cZg5cya+/fZbXL16FevXr5cbYM+fP4efnx8KFCiAM2fOYPPmzdi/f7/B5wu8vLdgaGgo9u3bh3/++Qepqanw8/ODra0tjh49iuPHj8PGxgZNmjTJ8ozAq1evombNmpgwYQKKFi2KXr164fDhwxk2xNatWwcbG5ssv44ePZrpvgAgISEBbm5ucHFxQatWrXDlypUs5588eRINGzY0GPPz80vXbK5ZsyZOnz6N5OTkLLeXJwjKFXFxcQKAiI6OVroUIsqhtLQ0ceHCBZGWlqZ0KUSUQ8wvkXoxv0TqxfwSqRfzS2qUmJgorl69KhITE3P82uqenmItIEQ2vtYCorqXV67W3qNHD9GqVasMn+vYsaPw+v/7O3r0qNBoNCIpKclgTsmSJcXSpUuFEEJMnDhReHt7C51OJz+fndf5+PiILl26ZFqjm5ubmDdvnsFY7969Rb9+/QzGjh49KoyMjERiYqIIDQ0VAMTp06fl569duyYApNvW6zp37iyGDh0qP65UqZJYuXKl/DgsLEwAEMuXL5fHrly5IgCIa9euydto1KiRwXZHjRolypUrZ7Curl27yo8jIiIEAPHtt9/KYydPnhQARERERKb1Nm/eXIwYMUJ+7OvrK4YMGWKwH/2as/N52NrailWrVmW6v9fZ2dmJ33//3WDsypUrwsvLSxgZGQlvb2/Rv39/sXPnTvn5+Ph4YW5uLpYtW5bhNn/99VdRoEABkZCQII8FBgYKIyMjERkZKYR4eewWLVpUJCcny3PWrFkjypYta3AMJicnC0tLS7Fnz543rkWn04mDBw8Kf39/YWtrKzw8PMSkSZPEnTt3DGq/efNmll8vXrzIdB8nTpwQq1evFhcuXBBBQUGiRYsWQqPRiPv372f6mtKlS4vp06cbjAUGBgoABvu6ePGiACDCw8PTbSOr71PR0dECgIiLi8vy/clNJko1HImI8hJbW1ulSyCit8T8EqkX80ukXswvkXoxv/QxiYuPh2M25xYFEBcX9z7LMSCEkC+tefHiRSQkJKBgwYIGcxITE3H79m358auX4szu64KDg9G3b98c1Xbx4kVcunTJ4PKdQgjodDqEhYXhxo0bMDExQbVq1eTnPT09YW9vn+V2Y2Nj8eeff+LYsWPyWNeuXbFixQr4+/sbzK1YsaL8ZycnJwAvLy/q6emJa9euoVWrVgbzP/vsM8yfPx9arRbGxsbptqE/283b2zvdWFRUFBwdHaHVajF9+nRs2rQJ//33H1JSUpCcnJzte/hl5/MYPnw4+vTpgzVr1qBhw4Zo3749SpYsmek2ExMTDS7zCQDlypXD5cuXce7cORw/fhxHjhxBy5Yt4e/vj+XLl+PatWtITk5GgwYNMtzmtWvXUKlSJVhbW8tjn332GXQ6HUJDQw3eKzMzM4P13bp1K93PkaSkJIPjNDOSJKFevXqoV68eFixYgAEDBmDKlCkIDg7Gtm3bALz8GfUuP6d8fHzg4+MjP/7000/h5eWFpUuXYtq0aW+9XQCwtLQEgA9yudJ3xcZfLtN/UyEi9TA2Ns7yBywR5V3ML5F6Mb9E6sX8EqkX80sfGzuNBpEPH2Zr7iMAdnZ277egV1y7dg0eHh4AXl6a0MnJCUFBQenm6Ztp+nvwvdr8y87r9M2KnEhISED//v0xePDgdM+5urrixo0bOd4mAKxfvx5JSUmoVev/Lr6qbyjeuHEDZcqUkcdfvfylfs06nS5H+8toG1ltd/bs2ViwYAHmz58Pb29vWFtbY+jQoVlexvJV2fk8Jk+ejM6dOyMwMBC7du3CpEmTsGHDBvm+i68rVKgQYmJi0o0bGRmhRo0aqFGjBoYOHYq1a9eiW7duGD9+/Ft95hl5tTEIvFxftWrVMryfY+HChbO1zfPnz+P333/HH3/8AUmS5Eao3rp169C/f/8st7Fr1y7Url07W/szNTVFlSpVcOvWrUznODo64tGjRwZjjx49gkajMXgv9Zdrze5a9ZToGbHxl8ty+s2HiJSn0+kQFRWFIkWKwMiItz4lUhPml0i9mF8i9WJ+idSL+aWPTavOnbFm+nR0SUp649zfLS3RunPnD1AVcPDgQYSEhGDYsGEAgKpVqyIyMhImJiZwd3fP8DWmpqZIS0szOFMwO6+rWLEiDhw4gJ49e2b4vJmZGbRarcFY1apVcfXqVZQqVSrD13h6eiItLQ3nzp1DjRo1AAChoaGIjY3Nct0rVqzAiBEj0p3dN3DgQPz222+YOXNmlq/X8/LywvHjxw3Gjh8/jjJlyrxTk+X48eNo1aoVunbtCgByQ7JcuXLZen12Pg8AKFOmDMqUKYNhw4bhq6++wsqVKzNt/FWpUgVXr1594771NT5//hylS5eGpaUlDhw4YNBU0/Py8sKqVavw/Plzubl3/PhxGBkZoWzZslmub+PGjShSpAg0Gs0ba9J78OAB1q5dizVr1uD27dto2bIlVqxYgSZNmsDExLBF9cUXXxg0hjNSvHjxbO9bq9UiJCQEzZo1y3SOj48Pdu7caTC2b98+gzMHAeDy5ctwdnZGoUKFsr1/QJmeEX/C5zKRwQ0piShvE0IgMjKS+SVSIeaXSL2YXyL1Yn6J1Iv5pY9Nn759cSg1FafeMO8UgKCUFPTOoEnyrpKTkxEZGYn//vsP58+fx/Tp09GqVSu0aNEC3bt3BwA0bNgQPj4+aN26Nfbu3Yvw8HCcOHEC48ePx9mzZwEA7u7uCA8PR3BwMJ48eYLk5ORsvW7SpEn4448/MGnSJFy7dg0hISGYNWuWXJ+7uzuOHDmC//77D0+ePAEAjBkzBidOnEBAQACCg4Nx8+ZNbN++HQEBAQCAsmXLokmTJujfvz9OnTqFc+fOoU+fPlmeaRYcHIzz58+jT58+qFChgsHXV199hdWrVyMtLS1b7+mIESNw4MABTJs2DTdu3MDq1auxcOFCjBw5Mucf0CtKly6Nffv24cSJE7h27Rr69++f7kywrLzp80hMTERAQACCgoJw9+5dHD9+HGfOnIGXl1em2/Tz8zO4NCoAtGvXDvPmzcOpU6dw9+5dBAUFYdCgQShTpgw8PT1hYWGBMWPGYPTo0fj9999x+/Zt/Pvvv1ixYgUAoEuXLrCwsECPHj1w+fJlHDp0CN988w26desmX+YzI126dEGhQoXQqlUrHD16FGFhYQgKCsLgwYPx4MGDTF/n5uaGv/76C4MGDUJERAQ2b96MFi1apGv6AS8v9VmqVKksv7I6zqZOnYq9e/fizp07OH/+PLp27Yq7d+8aNEDHjRsnZw8Avv76a9y5cwejR4/G9evXsWjRImzatEluzOsdPXoUjRs3znTfmVHiZy4bf0RERERERERERESU7zg6OmLmrFloaW6eafPvFICW5uaY+cMPcHTM7h0Bs2/37t1wcnKCu7s7mjRpgkOHDuGnn37C9u3b5bPTJEnCzp07UadOHfTs2RNlypRBp06dcPfuXbkR07ZtWzRq1Aj169dH4cKF5Uslvul1devWxebNm7Fjxw5UrlwZ9evXx+nTp+X6pk6divDwcJQsWVK+hGHFihVx+PBh3LhxA7Vr10aVKlUwceJEFCtWTH7dypUrUaxYMfj6+qJNmzbo168fihQpkun7sGLFCpQrVw6enp7pnvvyyy8RFRWV7qyrzFStWhWbNm3Chg0bUKFCBUycOBFTp05NdyZhTk2YMAFVq1aFn58f6tatC0dHR7Ru3Trbr3/T52FsbIynT5+ie/fuKFOmDDp06ICmTZtiypQpmW6zS5cuuHLlCkJDQ+UxPz8//P3332jZsiXKlCmDHj16wNPTE3v37pWbad9++y1GjBiBiRMnwsvLCx07dkRUVBQAwMrKCnv27EF0dDRq1KiBdu3aoUGDBli4cGGW67OyssKRI0fg6uqKNm3awMvLC71790ZSUlKWZwBeuXIFp06dwsCBA1GgQIFsv59vIyYmBn379oWXlxeaNWuG+Ph4nDhxwuCszYiICNy7d09+7OHhgcDAQOzbtw+VKlXC3LlzsXz5cvj5+clzkpKSsG3bthzfL1MpkuCv+OSK+Ph42NnZITo6+r0fvESUu/SnfHt7e/M+nUQqw/wSqRfzS6RezC+RejG/pEZJSUkICwuDh4cHLCws3mob8378EWNHj0ZdU1N0T0pCUQCRAH63sMDh1FTM/OEHDBs+PFfrzm1CCCQmJsLS0tLgPn+U/40aNQrx8fFYunSp0qV8tBYvXoy//voLe/fuzfD5rL5PxcTEwMHBAXFxcTm6ROq74Bl/uYzfdInUR5IkODg4ML9EKsT8EqkX80ukXswvkXoxv/SxGjZ8OO4+eIA648djvpcXBhYrhgVeXvAdPx53HzzI800/PTbsP07jx4+Hm5ubIveKo5dMTU3x888/v9VrlfiZyzP+con+jL8P2bUlIiIiIiIiIiIiys9y44w/IqL3KavvU0r0jnjGXy5j151IfXQ6He7du8f8EqkQ80ukXswvkXoxv0TqxfwSqZcQAsnJyeB5PETqosTPXDb+chm/8RKpjxAC0dHRzC+RCjG/ROrF/BKpF/NLpF7ML6kZj9uX9+kkorwnq+9PSnzvYuOPiIiIiIiIiIiIiPIkU1NTAMCLFy8UroSIKGMpKSkA8s59OE2ULoCIiIiIiIiIiIiIKCPGxsawt7dHVFQUAMDKygqSJClc1Yenv9SnJEkf5fqJ8iqdTofHjx/DysoKJiZ5o+WWN6rIR/hNl0h9JEmCo6Mj80ukQswvkXoxv0TqxfwSqRfzS2rl6OgIAHLz72MkhIBOp4ORkREzTJTHGBkZwdXVNcNsKpFXSfDiyLkiPj4ednZ2iIuLg0ajUbocIiIiIiIiIiIionxFq9UiNTVV6TKIiAyYmZnByCjjO+sp0TviGX+5jDdYJVIfrVaL8PBwuLu755nrMBNR9jC/ROrF/BKpF/NLpF7ML6mdsbHxR3vsMr9E6qREzyjjFiQR0Ufm2bNnSpdARG+J+SVSL+aXSL2YXyL1Yn6J1Iv5JaLsYOOPiIiIiIiIiIiIiIiIKB9g44+IiIiIiIiIiIiIiIgoH2DjL5dJkqR0CUSUQ5IkwcXFhfklUiHml0i9mF8i9WJ+idSL+SVSL+aXSJ2UyKwkhBAffK/5UHx8POzs7BAXFweNRqN0OURERERERERERERERKQgJXpHPOMvl2m1WqVLIKIc0mq1uH79OvNLpELML5F6Mb9E6sX8EqkX80ukXswvkTopkVk2/oiIACQlJSldAhG9JeaXSL2YXyL1Yn6J1Iv5JVIv5peIsoONPyIiIiIiIiIiIiIiIqJ8gI0/IiIiIiIiIiIiIiIionyAjb9cZmTEt5RIbYyMjFCiRAnml0iFmF8i9WJ+idSL+SVSL+aXSL2YXyJ1UiKzJh98j/mcJElKl0BEOSRJEjQajdJlENFbYH6J1Iv5JVIv5pdIvZhfIvVifonUSYmeEX89IJdptVqlSyCiHNJqtQgJCWF+iVSI+SVSL+aXSL2YXyL1Yn6J1Iv5JVInJTLLxh8REdi0J1Iz5pdIvZhfIvVifonUi/klUi/ml4iyg40/IiIiIiIiIiIiIiIionyAjT8iIiIiIiIiIiIiIiKifEASQgili8gP4uPjYWdnh9jYWNjZ2SldDhHlgBACSUlJsLCwUORmq0T09phfIvVifonUi/klUi/ml0i9mF8idYqLi4O9vT3i4uKg0Wg+yD55xh8REQAzMzOlSyCit8T8EqkX80ukXswvkXoxv0TqxfwSUXaw8ZfLdDqd0iUQUQ7pdDqEhIQwv0QqxPwSqRfzS6RezC+RejG/ROrF/BKpkxKZZeOPiIiIiIiIiIiIiIiIKB9g44+IiIiIiIiIiIiIiIgoH2Djj4iIiIiIiIiIiIiIiCgfkIQQQuki8oP4+HjY2dkhNjYWdnZ2SpdDRDkghIBOp4ORkREkSVK6HCLKAeaXSL2YXyL1Yn6J1Iv5JVIv5pdIneLi4mBvb4+4uDhoNJoPsk+e8UdEBCAlJUXpEojoLTG/ROrF/BKpF/NLpF7ML5F6Mb9ElB1s/OUynU6ndAlElEM6nQ6hoaHML5EKMb9E6sX8EqkX80ukXswvkXoxv0TqpERm2fgjIiIiIiIiIiIiIiIiygfY+CMiIiIiIiIiIiIiIiLKB9j4IyICYGxsrHQJRPSWmF8i9WJ+idSL+SVSL+aXSL2YXyLKDkkIIZQuIj+Ij4+HnZ0d4uLioNFolC6HiIiIiIiIiIiIiIiIFKRE74hn/OUy9lGJ1EcIgfj4eOaXSIWYXyL1Yn6J1Iv5JVIv5pdIvZhfInVSIrNs/OUynU6ndAlElEM6nQ537txhfolUiPklUi/ml0i9mF8i9WJ+idSL+SVSJyUyy8YfERERERERERERERERUT7Axh8RERERERERERERERFRPsDGHxERAAsLC6VLIKK3xPwSqRfzS6RezC+RejG/ROrF/BJRdkiCdwPNFfHx8bCzs0NcXBw0Go3S5RAREREREREREREREZGClOgd8Yy/XMabqxKpj06nw9OnT5lfIhVifonUi/klUi/ml0i9mF8i9WJ+idRJicyy8ZfLeAIlkfoIIXD//n3ml0iFmF8i9WJ+idSL+SVSL+aXSL2YXyJ1UiKzbPwRERERERERERERERER5QNs/BERERERERERERERERHlA2z8EREBsLW1VboEInpLzC+RejG/ROrF/BKpF/NLpF7MLxFlhyR4UeBcER8fDzs7O8TFxUGj0ShdDhERERERERERERERESlIid4Rz/jLZTqdTukSiCiHdDodIiMjmV8iFWJ+idSL+SVSL+aXSL2YXyL1Yn6J1EmJzLLxl8t4AiWR+gghEBkZyfwSqRDzS6RezC+RejG/ROrF/BKpF/NLpE5KZJaNPyIiIiIiIiIiIiIiIqJ8gI0/IiIiIiIiIiIiIiIionyAjb9cJkmS0iUQUQ5JkgQHBwfml0iFmF8i9WJ+idSL+SVSL+aXSL2YXyJ1UiKzkuBFgXNFfHw87OzsEBcXB41Go3Q5REREREREREREREREpCAlekc84y+X6XQ6pUsgohzS6XS4d+8e80ukQswvkXoxv0TqxfwSqRfzS6RezC+ROimRWTb+chlPoCRSHyEEoqOjmV8iFWJ+idSL+SVSL+aXSL2YXyL1Yn6J1EmJzLLxR0RERERERERERERERJQPsPFHRERERERERERERERElA+w8ZfLJElSugQiyiFJkuDo6Mj8EqkQ80ukXswvkXoxv0TqxfwSqRfzS6ROSmRWErwocK6Ij4+HnZ0d4uLioNFolC6HiIiIiIiIiIiIiIiIFKRE74hn/OUyrVardAlElENarRa3b99mfolUiPklUi/ml0i9mF8i9WJ+idSL+SVSJyUyy8YfERGAZ8+eKV0CEb0l5pdIvZhfIvVifonUi/klUi/ml4iyg40/IiIiIiIiIiIiIiIionyAjT8iIiIiIiIiIiIiIiKifICNv1wmSZLSJRBRDkmSBBcXF+aXSIWYXyL1Yn6J1Iv5JVIv5pdIvZhfInVSIrOSEEJ88L3mQ/Hx8bCzs0NcXBw0Go3S5RAREREREREREREREZGClOgd8Yy/XKbVapUugYhySKvV4vr168wvkQoxv0TqxfwSqRfzS6RezC/paVO1ODr9KF48eaF0KZRNzC+ROimRWTb+iIgAJCUlKV0CEb0l5pdIvZhfIvVifonUi/klAIg4H4GD4w9i99DdSpdCOcD8ElF2mChdABERERERERERERF9OMWqFUOVPlXg1cZL6VKIiCiXsfFHRERERERERERE9BExMjHCF8u+ULoMIiJ6D3ipz1xmZMS3lEhtjIyMUKJECeaXSIWYXyL1Yn6J1Iv5JVIv5pf0zq84j80dNuN51HMIIZQuh7Ihs/xe3ngZy2ouw8NzDxGyPgSJ0YkKVUhEGVHiZy5/yucySZKULoGIckiSJGg0GuaXSIWYXyL1Yn6J1Iv5JVIv5pf0Hl18hKubr2LFpytwYNwBpcuhbMgsv4nRiYi5E4PoW9H4s8uf2NBqg0IVElFGlPiZy8ZfLtNqtUqXQEQ5pNVqERISwvwSqRDzS6RezC+RejG/ROrF/JJe05+aYkLyBBiZ8J+H1SKz/NYYUAOjokahfIfy6BzYGW3Wt1GoQiLKiBI/c3mPPyIisGlPpGbML5F6Mb9E6sX8EqkX80t6xmbGGHRtEM8AVZHM8isZvfwMSzcr/SHLIaI8ir/SQURERERERERERPQRub3vNiKDI9n0U7kHpx7gyHdHEHc/Th5Lik3Cg1MPFKyKiJTGxh8RERERERERERHRR2Rzu83YO3IvtClaHJ52GKcXnla6JHoL17ddx6FvDyExOlEeW9tkLdY1WQedVqdgZUSkJEkIIZQuIj+Ij4+HnZ0dYmNjYWdnp3Q5RJQDQggkJSXBwsKCv+lGpDLML5F6Mb9E6sX8EqlXbuRXp9XByJjnEqiZEAIh60JgUcACpZuVxo/Ff4SNow36n++vdGmUhYzym5achodnH8L5E2c5lyF/hCApNglVelaBiQXv9EWktLi4ONjb2yMuLg4ajeaD7JONv1zCxh+RegkhoNPpYGRkxH+4IFIZ5pdIvZhfIvVifonU613ze/foXWz8ciParm+Lko1LvocKSQnx/8XDXGMOc1tzpUuhLPDnL5E6KdH446/n5DKdjqdQE6mNTqdDSEgI80ukQswvkXoxv0TqxfwSqde75tepihOSYpIQHhSeu4WRojTFNWz6qcDr+Y3/Lx6RFyMzvaSnNkX7IcujXPLs4TMkP0tWugzKRUr8PzMbf0RERERERERERPRGZjZmGBExAg2mN4A2RYvr26/j+ePneHj2IVITU5Uuj7Ip7FAY5rnOw9UtVwEAiTGJiLgQwc9QZS7/cRlLKy/FvWP30j23b8w+zC48G6kv+JmqiRACSyovwZaOW5QuhVRO0cbf4sWLUbFiRWg0Gmg0Gvj4+GDXrl3y80lJSRg0aBAKFiwIGxsbtG3bFo8ePTLYxr1799C8eXNYWVmhSJEiGDVqFNLS0gzmBAUFoWrVqjA3N0epUqWwatWqdLX88ssvcHd3h4WFBWrVqoXTp3lDWyIiIiIiIiIiIp1Wh50BOxFxIQLWRawBAGEHw/Bnlz+xodUGLKuxDE9vPFW4SsouSZJgU9QGZjZmAIAzv5zBr1V/xdNQfoZq4vyJM+p/Xx9FyhdJ95ydqx2cfZzx4skLBSqjt5X6PBVCJ1CsejGlSyGVU/Tuns7Ozpg5cyZKly4NIQRWr16NVq1a4cKFCyhfvjyGDRuGwMBAbN68GXZ2dggICECbNm1w/PhxAIBWq0Xz5s3h6OiIEydOICIiAt27d4epqSmmT58OAAgLC0Pz5s3x9ddfY926dThw4AD69OkDJycn+Pn5AQA2btyI4cOHY8mSJahVqxbmz58PPz8/hIaGokiR9N84iYiIiIiIiIiIPhb3j9/HmV/OwNzOHE5VnAAA4UHhqOxfGeU7lsf94/dhXdha4Sopu9zruqPvmb7yY4/6HoAEWBWyUrAqyinXz13h+rlrhs/VHFQTNQfV/MAV0bsyszHD6Cej5cePQh6hqHdRBSsitZKEEELpIl7l4OCA2bNno127dihcuDDWr1+Pdu3aAQCuX78OLy8vnDx5Ep988gl27dqFFi1a4OHDhyha9GUAlixZgjFjxuDx48cwMzPDmDFjEBgYiMuXL8v76NSpE2JjY7F7924AQK1atVCjRg0sXLgQwMtrrrq4uOCbb77B2LFjM6wzOTkZycn/d63d+Ph4uLi44OnTp7CzswPw8rdnjIyMoNPp8OrbrB/Xag2vs5zZuP6GrRmN6+vNzrixsbF8E9jXx1+vMbNxrolryo9r0m/PyMgoXS1qXVNWtXNNXFN+WpP+dcbGxlmuVU1rer1Grolryq9r0mq10Ol08rz8sKb8+DlxTVxTRrVLkoS0tDT5z/lhTfnxc+KauKaMas/q779vWlNkcCRsnW1hVdAqT60pq3G1fk5cE9eU0Zpe//tvfljTm2r/mNd0dvFZ7B68G9/c/AYFShTIF2t6U+35dU1xcXEoUKAA4uLioNFo8CEoesbfq7RaLTZv3oznz5/Dx8cH586dQ2pqKho2bCjP8fT0hKurq9z4O3nyJLy9veWmHwD4+flhwIABuHLlCqpUqYKTJ08abEM/Z+jQoQCAlJQUnDt3DuPGjZOfNzIyQsOGDXHy5MlM650xYwamTJmSbvzq1auwsbEB8LKJ6erqigcPHiA6Olqe4+joCEdHR4SHh+PZs2fyuIuLCwoWLIibN28iKSlJHi9RogQ0Gg2uXr1qcECVLVsWZmZmCAkJMajB29sbKSkpCA0NlceMjY3h7e2NZ8+e4c6dO/K4hYUFPD09ERMTg/v378vjtra2KFmyJKKiohAZGSmPc01cU35dk5ubG2xtbQ1+SUDta8qPnxPXxDVltCatVovChQvnqzUB+e9z4pq4ptfXdPnyZWi1WhgbG+ebNeXHz4lr4poyWpOLiwvu3btnULva15QfPyeuiWvKbE2lSpWCsbFxjtcUZRSFqIdRwMO8t6b8+Dm9zzU9O/0Mxs+M4dTKCXEJcfliTfnxc8poTVqtFsWKFYOTkxO2D92OB0ce4LNln8HYzNhgTc+ePMPN327C3ssen/f/PE+vKT9+Tm+7pht/3MCts7dQpl8ZxCIWzk2d8SLhBex0dqpdU378nHK6pldPIPtQFD/jLyQkBD4+PkhKSoKNjQ3Wr1+PZs2aYf369ejZs2e6N6VmzZqoV68eZs2ahX79+uHu3bvYs2eP/PyLFy9gbW2NnTt3omnTpihTpgx69uxp0NjbuXMnmjdvjhcvXiAmJgbFixfHiRMn4OPjI88ZPXo0Dh8+jFOnTmVYd2Zn/D1+/BgFChQAwM4418Q1qWVNWq0WV65cgbe3t/wby2pfU1a1c01cU35akz6/FSpUgKmpab5Y0+s1ck1cU35dU2pqKq5cuYLy5cvD2Ng4X6wpP35OXBPXlFHtQghcunRJzm9+WFN+/Jy4Jq4po9qz+vtvVmsKPxwOy4KWKORVKN2anlx/gr97/43KvSujSq8q/JxUsKbNbTfjxj83MD5pPCABidGJ2NRmE0o1KYU64+uock358XN6fU2v//03cFAgrv91HUPuDoFkJBmsKSk2CXMKzUGlnpXQakWrPLum18fzw+f0LmtaXW81Hl18hBGPR8jfo9W+pvz4OeV0TTExMShYsODHdcZf2bJlERwcjLi4OGzZsgU9evTA4cOHlS7rjczNzWFubp5u3NjYWP6Lj57+Q89o7ocelyQpw/HMaszpONfENWU2ntfXJElSpjVmtp28vqa3GeeauKbcqjGn4++yJv3/jGU1X21rys4418Q1ZTaupjXpa311jtrXlN1xrolryq0aczqeG2vSarUZ5je3aszpOD8nrim3aszpuFrXlNXffzMbX998PVw/d0XX3V3T1W5iZoK4e3FIeZbC/y//QOPvuqamPzdF7f/VhrHJy21Y2lni6fWncK7l/N5rz2ycn1PO//7b/JfmaP5L8wxrsXKwQsCNAFgXsZYbSHl1TW9bS26N56U1dd3TFfEP4mFikr5to9Y15da4mtf0+i/afAiKN/7MzMxQqlQpAEC1atVw5swZLFiwAB07dkRKSgpiY2Nhb28vz3/06BEcHR0BvDwN8/Tp0wbbe/Tokfyc/r/6sVfnaDQaWFpayn9RyWiOfhtEREREREREREQfq0azG8HWyTbD5xxKOWD4f8M/cEX0Luxc7GDnYic/NjYzxqjHoxSsiHKbJEkoWLqg0mVQDpmYm8ChpIP8+NbuWwiaHISWy1qiqHfRLF5JZCjj9qaCdDodkpOTUa1aNZiamuLAgQPyc6Ghobh37558SU4fHx+EhIQgKipKnrNv3z5oNBqUK1dOnvPqNvRz9NswMzNDtWrVDObodDocOHDA4NKfRJS/ZfZbGkSU9zG/ROrF/BKpF/NLpF5vk98aA2rAs7Xne6iGlPAs4hm0qdo3T6Q859X8hv4dimt/Xct07vOo54i+FZ3p85S3JEQm4L8z/yE1MVUeE0LgyfUniA2LVa4wUiVF7/E3btw4NG3aFK6urnj27BnWr1+PWbNmYc+ePWjUqBEGDBiAnTt3YtWqVdBoNPjmm28AACdOnADw8vIilStXRrFixfDDDz8gMjIS3bp1Q58+fTB9+nQAQFhYGCpUqIBBgwahV69eOHjwIAYPHozAwED4+fkBADZu3IgePXpg6dKlqFmzJubPn49Nmzbh+vXrKFo0e530+Ph42NnZfdDrtBIRERERERERESnt6tarMDIxgmcrNgfzOm2qFt+ZfYdy7cuh/ab28vjdo3fx5NoTVOtXTcHqKCeWVlmK5PhkDL49OMPnl9VchoSIBAy7P+wDV0Zv4+ySswgcEIguu7qgVJOXV0jUpekACTAyznPnb1EOKNE7UvRSn1FRUejevTsiIiJgZ2eHihUryk0/AJg3bx6MjIzQtm1bJCcnw8/PD4sWLZJfb2xsjH/++QcDBgyAj48PrK2t0aNHD0ydOlWe4+HhgcDAQAwbNgwLFiyAs7Mzli9fLjf9AKBjx454/PgxJk6ciMjISFSuXBm7d+/OdtPvVQr2UYnoLQkh8OzZM9ja2ipyzWUienvML5F6Mb9E6sX8EqnX2+T36c2n2NBqA2oNqYXq/atnOGffyH2wdLBk408FtCla1BhUA07VnAzGzy4+i8t/XEal7pVgYqH43aEoA6/nt8lPTaBNzvzMzcr+lZEUl/QBK6R34fyJM+pNqwfHKv93+zEjEzb88gMlekaKnvGXn+i7ttHR0ShQoIDS5RBRDmi1WoSEhMDb25uXLCJSGeaXSL2YXyL1Yn6J1Ott8ht1JQpbOmxBzcE1M2383dl/B6bWpnDxccnNcukDijgfgcSYRLjVcYOxKb+350X8+ftxir0bi5B1ISjbqiyKlC+idDmUAzu/2YnUF6moM6cOHBwcPp4z/oiIiIiIiIiIiCjvKlK+CAZeGZjlnBINS3ygauh9carq9OZJRPTBxYbH4uD4g5CMJDb+VObMwjMoUEKZk8R4rigRERERERERERG9EyEEb4GjAmEHw7Cp3SZEXIhI95wQ4uU9xUgVZheZjU1tN2X6fMgfIVhdfzVi7sR8wKrobSTFJv0/9u46vqnz+wP45yapu7sBpTjF3TdsOIMNmzFhAhPmvt+cbWzfuQIbzmAwHIa7awu01IW6t2kbu78/bm/S0HjTSHverxevtDc3N09pn+Tmnuecg+Vhy3Hi8xNN7gsfEI7Hzz1O/TftDMuyWHhmIab9Oc0qz0+BP0IIAeDs7GztIRBCTETzlxD7RfOXEPtF85cQ+2Xs/C24XoDzP55HZU6l1n32vbQPHzp8iNqS2uYOj7SwktsluLnlJuor69W2ZxzJwEdOH+HSH5esNDJiiMbzN3JIJAK6BWjdt6agBvlX8lFbRvPS1olLxHAPcdfYX1PkLEJYvzC4+LpYYWTEVAzDIKh7ENwC3SCtlVr++anHn3nwPf4sWaeVEEIIIYQQQgghhJCWdPa7s9i7ZC8ePvIwokdEa9zn4m8XkbInBZN+ngS3QDfLDpAYTSFTAAwgEKpyQkpul2D/0v3o/URvxE2Os+LoCCF3k9XJUFNUA68IL2sPhRiorqIO574/h8NvH8bsg7PRZUwXi8aOKPBnJnzgr6ysDN7e3tYeDiHECAqFAmVlZfDx8YFAQInQhNgTmr+E2C+av4TYL5q/hNgvU+Zv1Z0qFN8qRkjvEDh7U7YvIdZC779t16qRq1B4vRCvlrxq7aEQA+15fg/OfXsOPRb0QK8XeyGmd4xFA3/0CmFmFEclxP6wLIvs7Gyav4TYIZq/hNgvmr+E2C+av4TYL1Pmr0eoB2JGx1DQr5XIPZ+LzOOZ1h4GMUHj+VtfWY99S/chaUeS1v3rKupwa9stFN0osuAoiSmyT2Xj9PLTqCmq0Xh/l1ld0GthL7AKOveyF8E9g9F5ZmdMWzUNXtGWz9SkwB8hhBBCCCGEEEIIIUQjuUSuN1BYfKsYe1/Yi+zT2RYaFTHVwTcOYv3k9RrvO/bRMRz54IhlB0RMUltaizPLzyDjSIbWfSqzK7Fx+kYkbkq03MCISVL2pmD/0v0QF4s13t//2f64d9m9YASMhUdGTNXrsV6YvXm21X5nTbtFEkIIIYQQQgghhBBCCIDtj2/H9bXX8UbVG3BwddC4T3V+Nc7+7yx8O/giYlCEhUdIjDHopUGoLqjWeN/NLTdRV1GHke+NtOygiNE8wz3xfMbzcHRz1LqPV5QXpq+ZjuD4YAuOjJii76K+6DC+A7yjva09FGJGkmoJ/p79N3z7+lr8uSnwRwghADw8PKw9BEKIiWj+EmK/aP4SYr9o/hJiv4ydv6F9QyGtkULkov0yYtiAMLyY/SJcA1ybOzzSwmInxmq9b96eeXD0cATLsjjy/hG4+rtiwOIBFhwd0YefvwKRAN5R3jr3dfJwQo95PSwwKtJcHqEe8AjV/tqcfyUfh989jL6L+uqcw8Q25F3Ow9EPjmLA8wOQeSwTwkChxcfAsFSU3ywqKyvh5eVl0QaNhBBCCCGEEEIIIYQQYi6yOhlEziJ8HfE13ILc8OSFJ609JKKBVCxFdUE13ALc4OiuPeuP2AdJtQRCRyGEjpoDRLnncrFiyAqM+2Yc+j/b38KjI8ZK3JSIf+b9g9n/zEbHSR1RVVVl8dgRBf7MhA/8lZWVwdvb29rDIYQYQaFQoLCwEIGBgRAIqPUpIfaE5i8h9ovmLyH2i+YvIfarpeYvy7IoSyuDQCigUnU2jFWw+Lb9t4ibFofxX49vcn9FVgW+ifoGk3+bjJgxMXAPctda3pVYXuP5m3E4A6vvWY2JP05Ev6f7aX3Mr31/hbO3Mx468JAFR0qMtXbCWmQey8SbNW9qvJ9VcCEc6vFnP+QSOQBA6ChEeXk5fHx8LBr4ozN0M6M4KiH2h2VZ5Ofn0/wlxA7R/CXEftH8JcR+0fwlxH6ZMn8PvnkQJz47oXe/Hzr/gD1L9jRneKSFyepkcPV3hchZc9nW7NPZAABnb2f4xPhQ0M/GNJ6/XhFeGPzKYIT0DtH5GL9YP/i097HQCImpYsbEoMdD2suyMgKGgn52hs/gzLuUh+sbrlv8+anHHyGEEEIIIYQQQgghRKOrf16FV6QXhr4+VOs+DMNg+NvDKcBg4xxcHfDE+Se03t/tgW6ImxIHBxcHSGulKL5VDM8wT7gFullwlMQQfh39cO+ye/XuN3P9TAuMhjTX4JcH690n50wOaktrqcefHUjYkIDAboEI7BaIc9+fw5mVZyw+Bsr4I4QQQgghhBBCCCGEaPRc0nOYu2uu3v1GvDsCPeZpz1gh9sHBhcvySz+Yjl97/4qk7UlWHhEhBAD2vbgP2x/fbu1hED3qK+uxZc4WZaZ830V9MX3NdIuPgzL+zIxhKOWWEHvDMAx8fX1p/hJih2j+EmK/aP4SYr9o/hJiv0yZv47uji04ImJJZWlluLHlBmInxiKwa6DOfYN7BWPkByMR0kd3KUliOY3nb/LOZJz/8TzGfDoGwT2DtT7m9u7byDyeiRHvjlAGdYnt2fHkDvh19NOZ+Tf8neHKvnHEdgkcBHhg6wNwDXAFAIT1D4NbR8tnTVPGn5lRY3NC7I9AIEBkZCTNX0LsEM1fQuwXzV9C7BfNX0Lsl7HzVyFT4M7FO6jOr9a775EPjmDlsJXU/9OGFVwrwIFXDyD/cr7efT3DPDHi3REI6UWBP1vReP5W5lQi40gGJNUSnY9J/S8VJz87CXGx2EKjJKa4+tdVZBzJ0LlP7MRYdJrWyTIDIiZzcHFAp2mdEDkkUrnNGufMdJZuZgqFwtpDIIQYSaFQICsri+YvIXaI5i8h9ovmLyH2i+YvIfbL2PlbU1iD3/r+huOfHte7b1VuFcrSyiCrkzV3mKSFRI+MxsIzC9F+bHtrD4WYoPH87buoL94Sv6UWXNBkyCtDsPj2YniEeFholMQUb1S+gfs33m/tYRAzuHvxS+65XHwd+bXFx0GBPzOjVU2E2B+WZVFaWkrzlxA7RPOXEPtF85cQ+0XzlxD7Zez8dXB1wJhPx6DjpI56953862S8lPsSlRO0Yc7ezggfEA63QMPKzh165xC+6/gdFHJa6GELTHn/9Qj1gG8HXwhEFAawZUJHIRzddJdVvr3nNr4I+AI3Nt+w0KiIKQ6+eRBfBH6BiqwKAIBrgCsCu+surdwSaMYTQgghhBBCCCE6KBRAcjIgoyQWQkgb4+ztjKGvD0X7eylDrDWQ1EiMy8hkueBvbWltyw2KmKTgegGSdyVDWivVuZ9cKkfVnSrUV9ZbaGTEWNJaKbJOZqHqTpXO/Vz9XeHfyR8OrtziCoVcQYuwbJBnmCf8OvrB1Z/r8ecT44MHtz9o8XFQ4I8QQgghhBBCCNGisBB44w1g6VJg/Xprj4YQQmxXeWY5rq+7jsrcSmsPhWhx8I2D+NjlY4N/R6M/Go1FVxbBLcCwDEFiORd/vYj1k9ajvkJ3QC/tQBqWhy1H4t+JFhoZMVZZahlWDl2Ji79e1LlfWL8wPHr8UcROjMWNzTfwmddnKEsrs9AoiaH6P9cfj514TBmgtRYK/JkZwzDWHgIhxEgMwyA4OJjmLyF2iOYvIfaL5i+xBxcvAkuWADcaKirl5Fh3PLaC5i8h9svY+Zt+KB1/DPoDaQfT9O6bdSIL/8z7B7nncps7TNJCQvuGovu87nDxdbH2UIgJGs/fngt6YsqKKXD2cdb5GL9YP/Rf0h/+nfwtNEpiLLdAN4xdPhbtxxmeWc0qWAiEArj4uoBlWaQfTkddeZ1Bjz309iEcfvewqcMlJrj02yWLPyfDUj6oWVRWVsLLywsVFRXw9PS09nAIIYQQQgghhDQDywKPPgqUlACenkBlJdC1K/DZZ9YeGSGEWM6tbbew+9ndmPzbZMROjNW5b0V2Be6cv4PwgeHwCPWw0AhJS5JL5bj0+yW4Bbqhy8wu1h4OIW3e2e/OQi6RY/DSwZBL5RA6CJH4dyI2z96Mcd+Mw8DnB2p9bHlmORxcHPDXmL8gq5dhcfJiC468bWBZFnuf34vIYZHoOqurcvv3/b7H4guLLRo7oow/M5PL5dYeAiHESHK5HKmpqTR/CbFDNH8JsV80f4mtKy7mgn4CAfDCC9y2SqpeB4DmLyH2zNj522laJ7yU+5LeoB8AeEV4ofOMzhT0a0UEQgH2L92PS79aPluFNEXvv+T8D+eRfjAdACB0EAIA4ibHYchrQ9BxUketj2MVLLbO34qfuv+EB7Y9gGcSn7HIeNsacbEY5747h7QD6lnyk36fZPGxiCz+jIQQYoOqqnQ30CWE2C6av4TYL5q/xJYlJXG30dFAQAD3dUWF1YZjc2j+EmK/Wnr+sgoWjIBKAduifS/tg6O7I0b93yiD9mcEDBbsXwCvSK8WHhkxFD9/t8zdgvwr+Xj2xrM692dZFn/P+htBPYIw4t0RlhgiMdLNf27i2IfHMPGHiYgYHKFz30FLB0FcLFbbJnIW4Z7P7tH9JAzQf3F/FCYWwre9b3OHTLRw9XPFS3dearLdGq+hFPgjhBBCCCGEENJq1NVxWXqOjs07TnIydxsXB3g1fFavqgIUCu74hBDSFuSey0VpSinipsbB0U33CyvLsvgm8huE9A7Bg/8+aKEREmPc+PsGPMI8DA78AUDk0MgWHBExlYufCzxC9GfXMgyD9IPpYOXU7ctWyepkqK+sBwxYL9HniT7aj1Mvw5lvzmDIK0Nwbc01OLo7ovOMzgC4v4Ous7uiK7qipqgGJcklCOoRBCcPJ3P9GATcYglN81Iqllp8LBT4I4QQQgghhBDSKshkwNNPAw4OwC+/AEzDBZTaWsDZWfW9IRoH/jwaPr+zLFBdzfX8I4SQtuDq6qs4//15LM1bqjfwxzAMwgaEwTeWskls1fMZz0NWKzP6cVV3qgAGBgWaiGVM/G6iwfu+WvIqZeHasO5zu6P73O7NPs6up3fhysoraDemHfa9tA/+nfyVgT+WZcE0nAhfW30N+5fux6MnHkXkEArsm1NVXhWkYim8o7whEKlWCiZsTLD4WGidopkxxnySJITYBIZhEBERQfOXEDtE85cQ+0Xzl7SEggKuN19eHlBWxm0rLATmzwe+/trw48jlQEoK93XHjoBIBLi5cd9Tnz+av4TYM2Pnb99FfTH7n9lw8XUxaP/Zm2fjnk/1lJwjViMQCuDoblxKfMG1AiwPW45z359roVERQ5n6/ktBv7Zh1IejMOWPKQjqEYTZm2djyh9TlPf9PuB3/Dn6TwBA9KhojP1qLLwiqISvuZ37/hy+6/AdSm6XqG0P7RNq8bFQxp+ZCajmCyF2RyAQwM/Pz9rDIISYgOYvIfaL5i9pCUVF6l/7+gK3bgESiSqDzxCZmUB9PRfsCw/ntnl6AjU1XJ8/fltbRfOXEPtl7PwN7BqIwK6BLTgiYimyOhlyz+XCp50PPMMNT10P6BqA3k/2ppKfNqDx/D33wzk4eTih50M99T5OUi1B8s5keEd7I3xgGz+JsUHph9NRllaGHvN7QORkerjGM8wTvR7rBQCIHhmtdp9vB184uDoAAEJ6hSCkV4jJz0O0ix4RDblE3iSoGtrL8oE/ilKZmVwut/YQCCFGksvluHXrFs1fQuwQzV9C7BfNX9ISCgpUX/NBwMJC7ra+3vDj8EHC2FhVeVC+zx9l/NH8JcSetfT8zTqZhV3P7kJZWlmLHJ+YriK7AqtGrMK5H4zL3BMIBZj8y2TETohtoZERQzWevyc+PYHzP5436HF1FXXYMmcLLv52sYVHSExxddVV7Hh8BxQyhVmPW1dRh/oq7gR45rqZmPL7FD2PIM3Vfmx7jP1ibJPMamucM1PgjxBCANTV1Vl7CIQQE9H8JcR+0fwl5sYH+Rp/zQcDTQn8deyo2sb39auoMH18rQnNX0LslzHzd8XQFfix248G71+SXIILP15AYUKh/p2JRbn4umD8/8aj46SO+nfWgmVZM46ImIKfv/P3zcfUFVMNeoxnmCfu33g/Rrw7oiWHRkw05PUhmLt7rjIjzxxS9qXgc+/PkbChaW85SY0Ev/X7DQdeP2C25yO6WeO1k0p9EkIIIYQQQghpFRoH/u7O+DMmTpWUxN02DvxRxh8hpC0K6R0CSZXE4P27zOyCDuM6wD3YvQVHRUzh6ueKAUsGmPTYiuwKbJy+EZ2mdcLwt4ebeWTEFMaW4O06u2sLjYQ0V0DnAAR0DjD7MXss6AGfGB/c3nMbmUczMeD5AfAI8YCDqwNqS2shl1DlBnNiWRYrh65Eu3vbYeT7I9Xus0ZfbAr8EUIIIYQQQkiDrVuBCxeAt98GXFysPRpiLE2lPvltUimgUAD62rKLxUB2Nvd1XJxqO5/xR4E/QkhbMuHbCUbt7+TpBCdPpxYaDbEW92B31JbUmr0UITENy7KoLa2Fg5uDUT3h6irqIKuTwT2IAvO2hFWwYARNA0MbNwKHDgGffAIY21rZK9IL0/+aDgDYvXg3zn9/Hv2e7QeAC0ItSV3S7HETddIaKSpzK1GdX23toQCgUp9mJ9D3KZIQYnMEAgHatWtH85cQO0TzlxD7ZYvz9/ZtYOVK4No1IKFpVRxiB/hgH/81y6pvM6TcZ1oa97iAAMDbW7Wdz/ijUp+2OX8JIYaxxPwtSy+jUp826Pq66/i558+4c+GO0Y8VOgixJG1JkywWYln8/GWlLJb5LcP2hdsNfmx5RjmW+S7D8U+Ot+AIiSl+7PYjfun1i9q2mzeBNWuAO3eAy5ebd/xxX43DszefhWeYZ/MORHRydHfECxkvYNLPk5rcZ41zZjpLNzNrpG0SQpqHYRh4enrS/CXEDtH8JcR+2dr8VSiAn37iAj4AUG0bCzWJEWQyoKRE9X1RERekkzSqUGdI4K+0lLsNDlbfThl/KrY2fwkhhjNm/rIsi70v7tXYI0qXlcNW4t9H/zV1iKSFyKVySMVSCESmXQ6m13zr4+cvAPR+ojeiR0Ub/FivKC/0WNADEYMjWmh0xFQxo2MQNTJK+b1UCnz7rer+4mLTjnvp90v4a8xfYBUs/Dv5q2UVZhzNwKmvToFVUN9OS7DG6ycF/sxMLqfauITYG7lcjuvXr9P8JcQO0fwlxH7Z2vzdv5/L+ONR4M/+FBdzgVuhkPu+qgrIzFTfx5A+f2Vl3G3jbD9AFfijjD/bm7+EEMMZM3+lNVKc/eYsUvamGPUcQ14bgr5P9zV1iKSFxD8cj8W3FyM4Plj/zhpU5lTi0DuHkHE0w7wDIwbj56/AUYDJv05G74W9DX4swzCYtmoauj3QrQVHSEwx8fuJGP/1eOX3mzYBOTmq+xtXrzBGeWY5Mo9n4sbmG6ivVF/9dn3ddfz38n+oLa017eCkifyr+bi25hrEJeIm91njnJkCf4QQAgraE2LPaP4SYr9sZf6KxcBff3Ffuze0PKmqst54iGkKG6rKBQcDbm7c14mJ6vsYkvFXXs7dagv8UcYfx1bmLyHEeIbOXwdXB7yY/SLu+eweo44/YPEA9HqslylDIzZMXCzG8Y+OI+t4lrWH0qbR+2/rVlkJbN7Mfd23Yf1E44oWxhj+9nA8fvZxbF2wFaeXn1a7b+ALA/HYqceoJ6sZ3fznJrYu2IqqO7bxQZICf4QQQgghhJA27dYtLtAXGAiMHctto4w/+1NQwN0GBXH9+YCmgT9DMv60Bf6oxx8hpK1hBAw8wz3hHuxu7aEQM0jakYRLv18Cy5pW2s+/kz+eufEM+i/ub+aREWNVZFVgy5wtuPXvLaMel3ksE3+O+hPph9NbaGTEWLI6GXYu2qksqZyby5WvDwwEpkzh9jE140/kJIJbgBtGfzIaHcZ3ULsvoHMAIgZFQOgobM7wSSM9H+qJ2Vtmw7e9r7WHAoACf4QQQgghhJA27s4d7jYmRhXcocCf/eEz/gIDVYG/W3ddDzMm48/HR307/7chkRh2HEIIIBVLkXM2R/+OxCaJS8QoSS6BQq4w6nGp+1PxS+9fkH6Iggu25Pz357HvxX0m95oSOYsQ0DkAzl7OZh4ZMVZNYQ0SNiSg+JZxzd+ktVLkX8lHTWFNC42MGKuuog4Xf7mIzONcfXo+u8/fn/vXeJspim8VI6R3CMIHhje5Ty6RQ1orNf3gRI1ve190ntEZDq4O1h4KAEBk7QG0NgIBxVIJsTcCgQBxcXE0fwmxQzR/CbFftjR/+cBfaKiq1CcF/uxP48Af/2clkajv05xSn87OgEjErcKuqOCep62ypflLbNu6+9Yh93wuXil6BQ4utnEhrK0zZv7e3HITO5/aiTk75qDjpI4GPwcjYFBbWtukpxSxrrFfjUVNUfMCPtUF1WAVLDxCPMw0KmIMfv46OzvjbcnbgJHJm+3HtsdrZa+1zOCISdwC3PBy4csQiLjX5NJSbruvr2ohW00NUFsLuLgYf/x/H/0XboFu6DBOPeMv91wufh/wO+794l4Mfnlwc34E0oBlWa0LK6xxzkyBP0IIAeDo6GjtIRBCTETzlxD7ZSvzlw/8hYVR4M+e8YG/oCDg7s/cDAOwbPMCfwzDZf2VlHD9V9py4A+wnflLbFvfZ/qiU14nsArTSguSlmHo/A3qGYRBSwchpE+IUcdvd087vJDxggkjIy0psFvz37i+bf8tIodEYv6++WYYETEFP3+FDsaXaDQ125O0HEbAwC3ATfk9H/jz8+MWnbm5cYG/4mIgIsL448eMjoGsTtZku2e4J7rO7grfDrZRlrI1+K3fb5BL5Hj62tPWHgoAKvVpdgqFceUPCCHWp1AocP36dZq/hNghmr+E2C9bmr+U8dc68IG/gICmQbmQhmvW+nr8saz2wB8AeHpyt229z58tzV9iu/Kv5qP0dik6TuoIRzcKFNsKY+Zv+IBwjP1yLGV3tRKy+qYX/401YMkAdJreyQyjIabg529VQRWyTmRBXCI2+hjJO5ORsi+lBUZHTFFfWY/CxEJlhnTjjD9AVe6z2LiqrkpTV03F/Rvvb7LdI9QD92+8H52m0Xw2l/CB4YgYrDk6a41zZgr8EUIIIYQQQtosmQwoKOC+psCf/ZLLVRdEgoJUpZF44Q1tTfRl/NXUcH8TgKqnX2P8tspK08dKSFuRcSQDh946hIqsCkiqJZBL5NYeErGgW9tu4cqfV6w9DNKAZVl86v4pNs3c1KzjjPlkDPou6mumURFTpR9Mx8phK5FxOMPox+54cgeOvHfE7GMipsk8nomfuv2EG1tuAFD18zNX4I+yPC1n4vcTMennSWrbzpwBjhyxzngo8EcIIYQQQghpswoKAIUCcHLiPmBT4M8+lZRwv0eRCPDxUQ/8+foCHg3JKvoy/vhsPzc3QFMlPD7jjwJ/hOgX/0g8Hj/7OKrzq/FF4BdI2pFk7SERI1RkV+CHzj/g0h+XTHr8yWUnsX/pfrAslXm1BQqZAl1mdUHksEhrD4WYQXB8MMYuH4vg+GCjHzvpl0m494t7W2BUxBS+7X0x4r0RCOnNlae4O+OPP6ctKjL/cx/7+Bj2vrDX/AcmALjFhMuWAcuXA1VVln9+6vFHCCGEEEIIabMal/lkGFXgTyoFJBLNwR9iexqX+WQY7mKJQMAFA4OCuMAuoD/jT1eZT0CV8dfWS30SYghnL2eE9Q+DR6gHwvqFwcnDydpDIkaoKayBQqYwuT/j2K/GwtGd3kRthdBBiJnrZjb7OMc/PY6c0zmYs32OGUZFTOXfyR9BXYNMemzc5Dgzj4Y0h38nf4x8f6Ty+8Y9/hrfmprxp0v6gXQUXC/A+G/Gm//gbYykWoIDrx9Au3vaKcunlpdznykBoKzM8mOiwJ+ZCQSUREmIvREIBOjevTvNX0LsEM1fQuyXrczfxoE/AHBxUQWMqqtVq22JbePLtfK9/QQCrjRSYSG3zVyBP8r449jK/CW2rTyzHO7B7vAM98QjRx+x9nBIA0Pnb2ifUCy+vdjk54kYpLnPEbFvhdcKkXE4AwqZAgIRvQdYmrnef/lMXCoDaVvq6gBxQ9vGuzP+WiLw98C2B6gHr5nUFNbg/A/nIXIRKQN/jYN9VVWWf72kV2hCCAEgkUisPQRCiIlo/hJiv2xh/t4d+GMYrswjQOU+7cnp09xtWJhqG3+hpHHGn75Sn/wHdH2BP8r4s435S2wXy7L4qdtPWHffOmsPhWhgqfkrrZWiPLPcIs9FdKvMqcSOJ3fg9p7bzTrOjHUz8EbVGxT0syKJRIILP1/Az/E/o+R2idGPP/XlKXzi+gmKb7VAJIkY7dLvl/DnqD9Rnlmu7O/n7MwtRgRaNuPP2cuZ5rKZeEV5YWneUgx9fahyW+PAnzU+O9Bv1swUCoW1h0AIMZJCoUBSUhLNX0LsEM1fQuyXrczfuwN/gKrcpzV6MRDjJSUBZ89yQdvJk1Xb27dX3VLGn3nZyvwltkshU6DvM33RZVYXAEDe5Tysm7QOKXtTrDwyYuj8PfXVqWb1ZWRZFt9EfYN/5v5j8jGI+VTnV+PSb5dQcK2gWcehDDHr4uevrF4GSbUEAqHxl/a9orwQPSqafpc2orqgWjkv7y7zCbRsxp+4RIzM45kQl4jNf/A2RiAUwD3YHa5+rsptjQN/5eWWP2emUp+EEEIIIYSQNis3l7ttnCnm4QHk5VHGn71YvZq7HT0aCA9XbX/kEWDMGCAmBti1i9tGPf4IsQyhgxD3fn6v8nuFTIHU/amInRhrxVERQ8nqZTjw2gF0nNTR5H5gDMOg3zP94OhBZeRsQXCvYLxa8iqEjsJmHaciqwIF1woQPihc7QI3saz+i/tj0AuDTHps11ld0XVWVzOPiJhq+FvDMfyt4QCAq0e5bY1bDfBBwNpaoKZGVZnEHJJ3JuPfR/7Fg/8+iLgp1PuxOWpLa1FTVAOvSC84uDgAuDvjz/KBdsr4I4QQQgghhLRJEolq9aymjD8K/Nm+a9eAq1cBkQiYO1f9PgcHoF07LhOQMv4Isa6Q3iF4q/Yt9Humn7WHQgwgdBDi6etPY/RHo5t1nJHvj8TgpYPNNCrSHAKhAC6+LnB0b14gNml7EtZPXt/szEFCSFN8xl/jwJ+zs+qzSYnxlV2VJHIJTmSdgFQuVW4LHxiOcV+PQ0CXANMPTAAAN7bcwA+dfkDm0UzlNir1SQghNkAobN6qN0KI9dD8JcR+WXv+5ucDLAu4uqoCOgAF/uzJpk3c7fjxQGCg9v2cnblbfT3++MCfj4/m+/mMv+pqoK1XubT2/CW27fLKy1g/Zb2yv5tAKDCpJB1pGfrmLyNgENA5AIHddLywErtSW1qLgmsFqK/SswJGj3b3tMPUVVPh38nfTCMjxhIKhUj7Lw0Xf7sIhYw7GVGwCnx16itcyb+i9/H1lfU48PoBJGxMaOGREkOkHUzDrX9vAVAF9hqX+gQA/4bpVlRk+vO8+t+rGLZyGN44+IbquHH+GPjCQPh28NXxSGKI4J7BGPrGUPjFqX55jQN/1lg0SGddZkYffgixP0KhEN27d6f5S4gdovlLiP2yhfnLl/kMDeWywngU+LMf2dnc7Wg9SSnmyvjj/zZYtm3/fdjC/CW2rTSlFCl7UpTlrgAg/0o+rq25ZsVREcCw+SsuEaOmqAYsyzbruWR1MqyduBb7X97frOOQ5kvelYyfe/6MjMMZzTqOfyd/xD8cD48QD/MMjBiFn79XV17Fzid3Ag3nr8cyj+Hl/15Gr196QcHqXpkkEAlw8vOTSN6RbIERE32OfXgM2xduB6A54w9QBf6a0+fvf2f/BwD46vRXph+EaBXWPwxjPhkDnxjV6kH+9wkAlZWWD8NR4M/MmntSRAixPJZlUVlZSfOXEDtE85cQ+2UL8/fOHe62cZlPgAJ/9oT/HTXO2NTEkIw/ltUf+BOJVL1V2nK5T1uYv8S2jfl4DN6qewuuAaoeYKeXn8bWBVtRX9m8jCPSPIbM3/0v7ceXgV9CKpZq3ccQImcRylLLUFNQ06zjkOYL6hGEkf83Ev6dKVPPnvHzd9jbwzBv7zxlJvWAsAHwdvYGAOxP1R1od3B1wHPJz2HSz5NaerjEAKM+HIWpK6YCUAWKtGX8NSfwpwnLsvh9wO/Y/sR28x6YAFDP+Csvt/w5MwX+zEzR1uu9EGKHFAoF0tLSaP4SYodo/hJiv2xh/ubnc7cU+LNPUinXpxFQ/c60MSTjr7ZWdTxtgT8A8GhIcqiqMmiYrZItzF9i+wRCAZhG6dT9numHB/99EEJHyhS1JkPmb8yYGPRZ1EctY9NUz958FtNXT2/2cUjzBPcMxoh3RsAv1k//zjqUJJfgm+hvcOrLU2YaGTEGP3/9u/ijw7gOyu0uDi54qMdDAIBfLv6i9zh+sX7N7vdIzCNqWBTipsQB0J7xF9DQgs/QwJ9CAezfDxQ0asXJB4YbYxiGFnGZyaG3D2H12NVgFdz/Z+MFhYD615ZCgT9CCCGEEEJIm6StnxsF/uyDWMzdMgzXp1EXPvCnK+OP/3twdlbtrwkf+GvLGX+E6KKQK3DlzysoTChU2x4+MBxxU+IgchZZaWTEUD0f6olJP00CI2D076yHOY5BbIeDmwNc/V0paGRlmoI1T/V9CgCwI2kH7lTd0fl4cbEYBdcKdO5DLItlVT3+tJX6NLTH3759wHffAd9/r9p2+anLAAAGjFo52CfOPYEpv00xddikQXlGOQqvFyrf88Ri1YJCAJDJLD8mCvwRQgghhBBC2iQ+cHN3mUgK/NkH/vfj6qreo1ETQzL+tAWC78b/vbTljD9CdKm6U4V/H/kXF3+9qPF+uVRu4RERa6rIqsC5H86hMLFQ/86kxZz/8TxWDl+JytzmrVrxDPPEkxeeRN9Ffc00MmKKFQNX4Ps4VVRn0B+D8Pj2xxHgGgA5K8cfl/7Q+fjdz+3Gzz1/hqzeCtEIomZ5+HL8PftvtUDR3YG/4GDuNi/PsGMePszdJiSozn0jPCOQ/nw66t+uh4ChkJC5zVgzA0vzliq/58t8urkBLi7WGRP9lgkhBIAz3/iFEGJ3aP4SYr+sPX/5wA0F/uwT//vhe+7p0jjwp62ikb7+fjz+76WtZ/xZe/4S2+Xi44IHtj6A+Efi1bZLaiRY5r8MO57YYZ2BESV983f34t3Y+8JeszxXSXIJ9jy3B1nHs8xyPGIacbEYJUklauV3iX1ydnZG5LBIxIyJAQBI5VKczz2P0zmnsXQQF3j47dJvkCu0L7Locn8XjPy/kWDlVObR2oLjg+HT3kdZ5tPdHUgsuYz0snTlPnxbgqIirtS9LkVFwM2b3NcymeproUCIaO9oOAjVSzhnn8rGyS9OQlIjATEf/vfp7a3/s0VLofoKZiYUUq16QuyNUChEp06drD0MQogJaP4SYr9sYf5Sxp99q6nhbg0J/PHXuFmWu2DiqKFCmaGBP+rxZxvzl9guR3dHdJrW9O/D0c0R4QPC4RfXvB5jpHkMmb8ZhzMgdDDP9a2QPiF46NBDCOgcYJbjEdOMeHcERrw7Qm3b2mtrIRKIcE+7e+Dnavi8PPf9OTh6OCL+4Xgzj5Low8/fTl+r5nB2ZTbkrBxOQicsHrAYn5/8HNmV2TiVfQrDooZpPE6X+7tYashEj7k75wIArlzhvnfwz0bvX3sDAOTvyiFgBPDy4s5l6+q4vn3h4dqPd/y4+vdXrwLx8cCinYtQL6/HByM/QKRXpPL+5J3JOPHpCcRNiYN/nL8Zf7K2JWFjArwivBAxOAKA6nOFry8XgM3KsvyiCwr8mRk1NyfE/igUCpSVlcHHxwcCASVCE2JPaP4SYr+sPX9ZVhW44QM5vMaBP5bVX0aSWIcxgb/GPfvq6zUH/viSPIYG/tpyxp+15y+xX3N3zbX2ENo8Q+bvMwnPgFWYJxPIxccFMaNizHIsYl4fHP0At0tvY9fcXZgYO9Hgx538/CQ8Qj0o8GcFmuZvamkqAKCdTzu4OrhixdQVaOfTDt0Du1tzqMRIfIZYnY+qTLZULoWTyAkMw2X9paUBd+7oDvwdPcrddukC3LjBBf4AYFPiJpTVleF6wXUsHbQUc7rPAQD0frw34qbEwSvSqyV+rDaBZVn8M+8fdJzUEQ9uexCA6nOFjw9XwlVTX86WRmfoZmaNXyIhpHlYlkV2djbNX0LsEM1fQuyXtedvTQ3Ar9m7O/DHfy+T6e4JR6yLD/zxgVpdhEJA1LDsVdvv1NhSn20548/a85fYtlNfnsLnvp8j/2q+tYdCNDB0/jIC8616YRUspLV66tORFpV+KB2Jfycqvy+oLsDt0tsAgI2JG3HhzgWDjzVnxxzMWDvD7GMk+vHzd9+L+3DxNy5AlFaWBoAL/AHAtE7T0COoh86yrgXXCrBq5CokbkrUug9peeJiMQ68cQBpB9OUgT+pWwYAYGbnmXASqVau8eU+dfX5y83lgoNCIfD009y2lBTunLlawpUyuZh3ESeyTigf49POB+EDw+Hg4qDpkMQQLDB7y2wMfnmwchP/+/TxsV6pTwr8EUIIIYQQQtocPlvL2blp9peTE/eBGaByn7bMmB5/gCrrr65O8/2NV+bqQj3+CNHNPdgdwfHBcPZu2kcu+3Q2djy5A8VJxVYYGTEEq2Bxe89tFN0sMtsxP/P+DJtmbjLb8YjxTn5+Etsf2678vvGF/7+u/oU9t/cYfKzg+GD4dvA16/iI4ViWxbnvziFlTwoAILWMy/hr79Pe8IMwQGFCIWoKuVVUFJi3jqq8Kpz87CSyT2YrA0WVDskAgI5+HdX25QN/d+5oP96xY9xtfDwQHQ2EhXHVSy5dlUCqUP2OMysy1R4nl8ipx18zMAIGnaZ2QuRQVQnVxp8rKPBHCCGEEJNIaiTY+8JenP76tLWHQgghdoPP1rq7vx/AlfakPn+2z5iMP0DV509bxl9hIXcbGKj7OFTqkxDdeszvgYcPPQzvKO8m95VnlOPSb5dQcLXA8gMjBpGKpVg3cR1OLTtltmN2nd1V7YIosbwR749Qy9JrHPgDgBPZJ+5+iFZyiRw1RTVmGxsx3kt5L2Hyb5MBqAJ/fMYfAJzLPYf5/8zHh0c/1Pj4oO5BeLX4VfR/rj+kYinWTljb8oMmTfjH+WPx7cXo+3Rf5WeOYpbLxBUwAvx4/kdU1nMnnCEh3P26An9nz3K3w4dztz17crcXrql/oGkc+CtNKcVHTh/h2EfHmvfDEDUU+COEEBvhcXeNL0LsiIOLAxLWJyhX/LU1NH8JsV/WnL980EZT4A+gwJ89MDXjz1yBv7Zc6hOg919imrjJcXi54GV0mdXF2kNp03TNX4FIgMm/T0aPh3qY7fmm/D4Fw94cZrbjEeNFDIpA3JQ45fd8oO+1Ia8BAE5ln4JMITPoWNse3oYvA7+EXCI3/0CJXp6ennD1d4WrnysAINA1ENHe0Yj1i1Xuk12RjbXX12LV1VV6y/oefvcwMo9l0u/TCoSOQvh28IVbgJvyvLZAygX+Pj7+MZ7d/SwOpR8CoD/jr74eSE/nvu7e0N6RD/xdTlAP1GdVZCm/dg92R7cHuyGoR1Dzf6A2KvN4Jj7z/gxXVl1RbuMDf76+FPhrNYR8TSBCiN0QCoVo3749zV9it5K2J6HX470wZ8ccaw/F4mj+EmK/rD1/+cCftmufFPizfXzGn6GBPz7jT1Opz5oa1fECAnQfp3GPv7ba4s7a85fYtku/X8LRD49qvM/R3RFugW46e0+RlqVv/oqcRei9sDdiRsVYeGTEUqol1bicdxkA8HTfp+Hp5IlqSTWuF1w36PHt7m2HPov6QCFTtOQwiQZCoRBR4VEouVkCcYkYAPDTpJ+Q/nw6JsZOVO43vsN4OIuckVaWhmsF13Qes9ucbpi3Z55Z+3oSw0iqJajMqYSsXobqakDO1KGwngvKzeoyCwCwL2UfAFXgr6gIkGqozJqayvUv9/EB/P25bd27c5VMsgq4DzROQm4VXGV9JcrrygFw78sz189E9zndW+inbP1EziKE9g2FW5DqQwkf+PP2Bry8YJXzHgr8mZlCQW96hNgbhUKB/Px8mr/Ebl1ecRmnvjgFgajtva239vkrl8ohFVO/BdI6WXv+Usaf/TM28Kcr46+ooZWVp6cqQKgNHyyWywGx2LDnbm2sPX+Jbbu+9jrO/u+sxvtYBYvChEKUppRaeFSEZ435e3X1VWx9aCtlFFnR952+x+p7VwMALt65CDkrR5RXFKK8ozAkYgiApuU/ten1WC9M+mkSHFwdWmy8RDOFQoHbZ2/jp+4/4cw3Z7Tu5+bohvEdxgMAtt3apvOYoX1C0WFchzZ5PcHabv5zE19HfI3UfanKShKf9l2Lj0Z9hAU9FgAA9qXuA8uy8PLizlFZFijQUC07KYm7jYvjgn0Ad87aqRMgF3AfaILcg+Dn4gcAyCzPbHoQYpKwfmF46MBDiJ3AZd3KZNxnTamgEhWCdPj4QG/mbUugGW1m1vglEkKah2VZ5Ofn0/wldmviDxPx8OGHkXksEwkbEqw9HIuy1/lbWak7S0RcIkbq/lQsD12OCz9fsNzACLEga89fCvzZP2N7/PGBP00Zf/wFlCADqhw5OQGOjtzXbbXcp7XnL7FtMzfMxMLTC7Xe/1P3n3DwjYMWHBFpTN/8zT2Xi2/bf4tra3RnCRkj92wurq2+psxQIpYX0isE/l24NKAR0SOQ/WI21s9cDwAYGjkUAHA867jVxkcMw7IsKiQVGPbOMMSMjtH5PswHdG+X3jbouPSebnl+cX4Y8MIA+Mb6oroaELLOmN1lDt4a/hZGxYyCg8AB6eXpSClNAcOosv7y8poeKzmZu42LU98+bBjgJe6D52uqcOGJC4jyjoKAESC/Ol+5z+nlp7HtkW0t80O2QeXl3G1yxFsYtLorduassso4KPBHCCGE2DmvCC9EDonE3uf3Yu8Le+mE3cYdOwbMmwfs36/adnr5aWQczVB+f+m3S1gzbg3ExWK4+LpYfpCEtAF8wIYCf/bLnD3++P5++sp88hqX+ySEqHMPcodfrJ/G+xgBg9Efj0a3Od0sPCpiDCcvJwidzFfKd8wnY/BmzZvwCKHeoNYyc/1MTPjfBOX34Z7hGBQxCIAq8JdRnmHQsTKOZGDzA5uRfyVf/87E7Jz9nTHivRGIGRWDjYkbEfJVCJ7d9WyT/ULcQwAAd6q0NIVrkLAhAR87f4zbu/QHCIl5hQ8Ix/ivx8O/U4DyvJb/DOLu6I6B4QMBACezTwLQ3efv1i3utmNH9e2DBwMCRoDUm+5gagOwd95e1L1Vh3Edxin3yT6VjevrrkMhp0oOpkjelYxD7xxCbWktAK7MZ5nbGaT7/4BaWS3a+0dAJLL8uKzwlIQQQggxF7lEDnGJGO7B7hj/zXg4eTlZe0hEjzMNFVnOnwfGjQNqimrw3yv/oeOkjmAYBpHDIhHSOwTd53XHlD+mQOTEna5ln8pGeUY5us3pRn1xbEh5OffhzBon8qR5KOPP/pma8acr8GdIxh/AlU4qLlb9HRFCVMrSy+Ds7QwXH82Ll4a9OczCIyLGCOsfhqcuPWXWYzp50mcUWzYwfCAyns9AlHeUQftX5lYicVMius3thuD44BYeHdEltTQV+dX5EMuaZtOGeHCBv7xqDelhjXiGe6Ldve3g7K2n1jlpMfX1XHnIYo+DOFkowRDXvghwC0D3wO44nnUct4q5qJ62wF9pKXdeyjBAbKz6fX5+QJcuQGIicPIkMHVq01Vu01ZNg8hFBIGQcsRMkbI3Bee/P49+z/QDABSXSnEt6gmAYfFQz4fQNbALRH6WL61Kv00zowtxhNgfhmHg6+tL85fYpfyr+VgeuhwnPjuBmNExCO0T2qb+lu1x/qalcbdZXM9uuAW44enrT0PkIsKqEauQeTQT7ce2x4w1M5RBPwA49PYh/Pvov6jMpqvMtiIhAXjoIeC336w9Evtk7fnLZ2p5aEk+4INJlNFlu4zt8cf37jNnxl9bDfxZe/4S2/ZDpx+w7aFt1h4G0cIa87e+qh6553NRnU+raaxBVi/Df6/+h5tbbwIAntzxJJbuW4qKugoAgKPQ0eCgHwB0nd0V70jfQaepnVpkvEQ7hmFQd6MOq8esRuaxTKSWpQIA2nm3a7JviHsIPBw94OGoO9M2cmgk5u6ci8ihkS0yZqLdhZ8vYMO0DSjO5jLFbod+iOmbJ2Jf6j4AQOeAzgCAm8Xc3A3hYrlNAn98mc+oKMBFw5ob9x6HcDX6Mfx4RvMHV0d3Rwr6NcOoD0bhmcRn4BbIfSjZkboFVa4JcIU/wj3CEbo8FIkB/2fxcdFv1MwEAvovJcTeCAQCREZG0vwldsnJ0wl9n+6LiEERALgMwLK0MiuPynLsbf7W1alO0vPzVRefA7oEYNSHozD4lcHwivJSe8x/r/6HVSNWYdbfszDsrWHY9vA25Yd2Yl1793K9Gg8f5lZoEuNYe/5WcNe6tGb88QFByvizTRIJIJVyXxtb6lNTjz9TMv6AthsYtvb8JbaLVbAY+OJAdJquPSCw69ldWDN+jQVHRRrTN39zz+fi9NenUZWn+wWOZVncKLqBEnGJ3ufMOZOD3/v/jlvbbpk0ZtI8kioJTn1xCqn7U1Enq8Nvl37D8jPLTT6e0EEIgch+X//5QOiR949YeyhGEwgE8HDwQGFCISQ1EuRW5QIAIr2aBu06+XdC5RuVOPfEOUsPkxio6EYRkncko6YhYVPswpVbjfXl0vamxk3FwYcO4tdJvwLQ3uMvKYm7vbvMJ08QfA3Z/iuRUHMEJ5NvYMHWBVi0c5Hy/rryOmQey9T7uk80c/F1QUCXAGXwNK08BQDQ3WGqslxrlWOyxcdlv6/SNkqhoFq4hNgbhUKBrKwsmr/ELvnH+eO+H+9D9MhoAMDqe1fj176/tpk+f/Y2fzMyuEARwN3evlyN1P2pUMgU8Iv1w73L7sXuZ3fjz9F/glVwO9ZV1KGuog4CoQDd53VH7vlcyvqzARIJcPYs93VtLZf9R4xj7fmrr9SnsRldtbXAmjWqrF7SsviALMNoXtmsia6Mv4IC7jYw0LBjtfUef9aev8R2MQIG93x2D3o91kvrPuIiMarzaFWFteibv+mH0rH/pf2oytX+Anc4/TCGrxqOrj92xX3r7tP7nAGdAzDm0zEIGxBm8riJ6Zx9nLEkbQlGvjcSpbWlAAAhI4Snk+okaOvNrZi8fjKWn9YfEJTVyZB5PBMlt/UHfW1RUWIRTn1xCkc/OGrtoRhNoVDAY4AHXi58GbETYpVZm97O3k32NTSrV1Ijwf5X9uP6+uvmHCoxwIRvJ+Ad2TuQCJwhE1SjVsStEo714wJ/EV4RGB0zGkHu3Mo0PuOvsFB94Skf+IuL0/w8ChH3nitSuOPMxVqsubYG25O2K+/POpGFVSNWIWVvihl/urajIqsC4mJVud2iWu6Dhb9zEDr5cwuhSmH5HpoU+DOztnKhlZDWhGVZlJaW0vwlrUK3Od3Q75l+kEvk1h6KRdjb/L07IHD5z2tYM24Nkndxq79YloWLjwtc/VzBCLgPahO/m4hFVxbB2dsZPu188FrZaxiwZIClh07ucuGCetbQ+fPWG4u9sub8ZVn9pT6NDfzt3g1s3Ai89hoFgi2hcZlPQ6vVacv4q6tT/T0YWuqT/7tpq6U+7e39l9iWWZtmYdHVRfp3JC1C3/zt+VBPPHbyMfh38td4/19X/8Lov0bjRNYJAMDZ3LPIqczR+Zye4Z4Y+vpQhPQKad7giUkEQgF8YnzgHuyuDPz5uPioBYZyKnOwM3knjmcd13u86oJqrBq+Chd+utBiY25Jrv6u6D6X66dub+6evxX1XODPy9lL18N0YgQMTn95Gim7KehjDQzDoLqaQY0T9//v5+IHXxdfjft6e3PnoCyr+vypUAC3G2JK2gJ/1ZKGwJ/cHQVp3MluSW2J8u8oqGcQxn0zDuEDws3zQ7Uxa8avwcrhK5Xfl9TlAwCC3IIR4xMDB4EDZNBQcqSFUeCPEELakJLbJdjx5A6k/pcKhZxWaLcG+1/Zj38X/qv8vu+ivhj90Wi13nDEdqSnq39f364zRn00Ch3GdQDAnfTPWDsDs/6epdxH6ChUfs0wDIQOQhDrO8Fd60IEV2UX586psjmJ7ROLuQ/JQKPMrbwqSGokyn2MDfxducLd1tUB770HXLpknrESzfjAH9+L0RDaMv74Mp9uboaXDW3rPf4I0aY4qRhrJ6ylko52zCPEAxGDI+Do7qjx/nnd5yHpuSS8N+I9dAvsBoDLACS2S1YnQ0VWBSQ1EmXg7+7AQkc/rkZgcon+cnRugW4Y9/U4dJ7R2fyDtQCvSC/MWDtDZ2ayLatIqsDNLTdRX1WPynruRKRx9mZjr/73Knr90gs7k3dqPZ6DiwOWpC3BxB8ntsh4iXZZJ7KQeSwT1dVAjXNDmc+GbD/entt78MaBN3Aq+xQYBpjY8Gtat477/HnsGPf5w9VV9dn0bnzgT6hwQ1ayNwBAIpegTsYFo7wivDDw+YEI6GLgCjiipufDPRH/aLzy+3vl/8OQm2cwOnQGRAJRk9+ppVDgjxBCWrG0g2nY//J+ZcnAzGOZuPTbJawZuwYVmRVWHh0xh5zTOcg5pXuFLbEdqVzvdXTjrpHgTq0Phr81HCJnwwO11fnVOL38NHLO0u/dWurruUAfACxaBIhEXM/GHPqV2A0+WOPsDDg6AgqZAstDl+PvWX8r9+EzuurqVL3ktJFKgRs3uK87duRKwX70kWr1LTG/xhl/htKW8Wdsfz+AevwRok1tSS0yj2WiMld7VDz/aj4ur7iM+koNdXeJ1UnFUsil2quHCAVCdPTriPdHvo+JHbgr0IcyDuk97l9j/sKOJ3eYbZzEcHcu3ME3Ud/g0u+XlIE/Pxc/tX34wF9KaQrkCt3VYxxcHDDwhYGIHNq0r5y9UMgUqCmsgaze/hp15+zOwZYHtkBcLEZn/87oEtBFa4ZYRnkGruRfQWppqs5j+sT4wMnDqSWGS3TY9+I+bH1oKxf4c1Lv78fbmLgRn538DAfTDgIApk3jgnwZGcDOncAvv3D7TZ8OaGu9zAf+HFh3VJd6QMBwO5bVlZn9Z2qLhr42FENeGaL8nqkOhU/NAMT4ceWt4/y0pGK2MAr8mZmh9ZMJIbaDYRgEBwe3yvl77a9rOP3VaeSe4xo+917YG+O+GYce83uAEba+n7cteuzEY2qlklgFi00zN2H34t1WHJXl2NP8lcuBzEzu65EjAbAssjKNTxETl4ixf+l+JG5KNOv4iOEuXOCCf4GBQPfuQM+e3Ha+5x8xjDXnLx/4Uwb3KurgFeWFwG6qBm9ubqoPz/qCO7duccE+Hx/g88+Bfv24YODHHwNl9Hm6RTQn8Kct48/QMp8A9fizp/dfYlkRgyPwZs2b6PdMP6373Np6C9sXbkdFNi1EtAZ983f7wu34yPEjg1oHjI4ZDQA4lH5Ib+nf+sp6SKolOvchLcM92B0DXxqI0D6hKBFzffnuDhRFekXCUegIiVyCrIosawzTYrbM2YIPHT7El0FfKq+V2AuGYRD/SDymr50O9yB3HHjoABKfSUS0d7TG/UPcufK6edV5Oo9blVeF4lvF5h4u0WP4u8Nxz+f3oLoaqHPk+vtFeUWp7dPZn8usvVl8EwBX7WLaNO6+X3/l+l536ADcf7/256mWVMNHAAwIqYSLqAbuQm8AQFmt6oPKH4P+wLaHt5nl52rr7u4lz/f5szQK/JmZQFtonRBiswQCAYKDg1vl/L3n83vw7K1nET5QVad74PMDMX31dHhHeVtvYMSs1EpBChiUJJegIqNtXEixp/l75w4XGHB2BgYMADxKMhH656e48MdVo44T0DkA8/fPx8j3R7bMQIleJ09yt8OGcb3F+vfnvuezAIlhrDl/7/4w5urnihcyXsC9y+5V7sMwhvdxu9owjXv04DJAX36ZK7VTUgJ8+qn+jEFivGpu4bJRpT71Bf5Myfhrq6U+7en9l1iHrqBwtwe7Ye6uufCKML0nFTGdvvkbMSQCPeb3UPuMwTuXew5ztszBrxd/BQAMjRyKJf2X4Nvx34KF7sDfE+efwMx1M5v/A1jI1dVXceXPK9Yehln4dvDFuK/GIXJopNZSn0KBEB18ufYDhpT7XDtxLTZM22D+wVqAuEQMgUiAPov6wNXf1drDMYpAIEDXUV3RY24POLg66N0/1CMUgP7A3+YHNmPViFXmGCIxQtzkOHR7oBuqq4HIosfxTNA63N9FPYLHB434wB8ATJmiWvwmEgEvvMDdalMtqcY0d2BF3MfoGnAMTqwPAPWMP4FIQAkCJihLK8Pqe1cj8W9uYXa9rB7HHN9AWuA3cHHjMorvaXcPpoY8Z/GxUQMgM5PL9a+IIoTYFrlcjoyMDERHR0MobD29s+RSOdyD3eEezF0RK75VjNKUUsSMjjHoBJHYvqo7Vcg5k4OIwRHK3zMAPHXlKQiEbeNCnD3NX77MZ0wM15Tb1VOIat9I1Gnpx6ANI2DQ/t725h8gMRjfqzE+nrvt1w/46Scu66uqShUQILpZc/7yWVqNf1dVd6qQuj8V0SOj4R3tDYALDFZU6A/uXLvG3fLZn66uwFtvAUuXAjdvAgcPAuPHm/dnaOvMmfFXUMDdBgbCYG29x589vf8Syyq5XYLS26WIGBIBZy9njfv4d/KHfyd/C4+M8PTN3/7P9Qe0XJ88lX0KGxI2oE5Whyf7PAk3Rzf8b8L/WnjE1nHikxOoKapB/MPx1h6KWS0ZsARzu89VlvprrKNfR9wouoHkkmSM6zBO53Hs+fPmgv0LrD0Ekxn7/hviwWX83am6o3O/ng/3RE1hjVnGSIxXVQV41cZjbEg8egar39c5gMv4SypOgoJVQMAI4OYGzJ0L/PYb8MgjQFRU02M2tnn2ZtTmH0XWxX9QXhcMpt4bEEHZIxIAHj3+qHl/qDaiuqAaOWdzEDeNK+eZV5WPm36fQeDjCG/P5wFw2fG17XvhX3xv0bHZ76s0IYSYUVUrq9NUcrsEy0OX49raa2AVLJJ3JWPtxLVYP3k9ChMLse3hbTj28TFrD5M0U+axTGyauQmZxzLVttvzhzBT2Mv8TUvjbtu1424D+0Tg9oD5kITGGH0sVsGiNKUUNUX04czSWFaVHRTc8KEsIACIjOTuu37demOzR9aav3dn/OWez8Wup3fh30f/RdqBNOV+hgR3amuB5IaF8XzgDwDCwoBJk7ivU1LMNHCixGf8GRP4c26IQdzd46+oiLs1JvDHB40lEu5fW2Qv77/Esm78fQPr7luHslT9dY71lYYkLcfU+Xu9gDvR6R7Y3ejHZp/Kxrnvz+nsH2hLYu+LRecZna09DLO4vfs21k9ej4JrBXASOSHMM0wZEGqso29HeDh6QCwV6z3mnB1z8OC2B1tiuESPg08exHcdvkNiYSIivo7AiFUjtO6rLPVZpTvjr/fC3hj2xjCzjpPoxrIsvon+Brue3aU8r9W0gLSdTzs4Ch1RK6tVK8M7ZQqwYQMwdar+53J3dEeAqz8ia1egR9Ah9E08hJrXpJgYO9FMP03bFTEoAm9UvoF+T3MlztOLuBWFjtIgeHmpMiidNa+FalFt68ogIYS0EdV51XDxdYFnmCdYBYudT+5EeXo5Jnw3AaF9QnF7921kHW/ddfvbgojBEZj21zREDI5Q215XUYfLKy4j6wT9jm0JnyXGB/4iI7nbLBN+TemH0vFd7He4vo6iTJZWXs5d5GcYwL9RsgIf8LlqXOVWYiX89U6vhipzN7fcRNL2JIz5dAw6Tuqo3M+Qco4JCVwPz+DgpoEjfp5nZ5tp4ESJz/gzR6lPUzL+XF0BfqF9W836I0STjpM7YsqKKfCO8da6T/rhdHzi9gku/3HZcgMjBtv/yn4cef+IxvuuFXIp7j2Ceii3yRQyHE4/jI+PfawzmHt93XXsWbwHtSW1Zh1vSxn+znBM/F7zRfGaohrI6mQWHpHpyjPLkfpfKuor63Xu93+j/g8Vr1fgtaGvWWhklldwrQDnfzqP/Kv52PzgZlxeaX+vQ65hrgjoEoDS2lLkVOboDOrxAV59pT6J5SlkCrgHu8PJwwnV1UC+97+4WLNNWY6XJxKIEOsbCwC4WXRT7T5jFsDBNQxs7HMolMZDIPVERpp6Icj0w+k4ueyk3SzOsDWMgAvyZZbkAwBcFEFq5VfrmBKLj4kCf4QQ0gpFDY/CszefRdSIKAhEAkxfPR0vZL6A/s/1ByNg8ELmC5i/d761h0maySvSCz0X9IRnuHqpSGmNFNsXbse1NdesNDKiCR/4i2lI8BMeP4KgtNMmBf6CewVj4IsDEdo31FzDIwbiAwR+fup9FHo0XP+iwJ994AM1fGCv76K+mLNjDga+OFCtdDKf8acrMYIv89mjR9P7whta7ObkNHPApAlTSn3yK20bB/4kEi6gDxgX+GMYVdCREt8IUQnqHoRej/aCi4+L1n3cAtwQNTxK7fW2NUs7mIb6Kt0BF1uSuDERKXuapqrLFXIkFnI9jBoH/uQKOSaum4i3D7+t1oPqbv2e6YeHjzwMZ28rpD2YYHnYcmyYqrmH3c89fsbHLh9DXKw/M84W9Hu6H96uexuRQyOx7OQyLN23VPm7bMxJ5KSzP2dj5RnlOLnsJPKv5Jt7uC3q9p7b2P3MblRmVyJxYyLyL9vX+AGg20vd8OD2B5VlGr2ctfdLDfUIhaeTJwLdAiFTaA9WJ25KxMrhK1GcVGz28RLNhA5CPH7mcdzz2T2orgYSI5bg1UvTkVLa9PWXL/d5q/iWSc/10r6XsPa/xyAtvQiPEO6CxK27DnVj8w0ceO0AakvtY3GGrcg5k4PkXcnKgGlWKXfBwB3qNVvfu2L5UqrU48/MDH2DJITYDoZhEBER0ermL7/aBABiRquXEqQef/avuqAadWV1GvujuIe448F/H0Rwr2ANj2xd7GX+VldzfcIAIKIhQbNk7wX4sL7IzBxk9PFc/VwxbrnuvhukZdxd5pPXvTsXCMjNBUpKuMAg0c2a8/fuUp/e0d7Kvn61ZbVw9nYGwzAGlfpMbLhu1rjMJy8sjPu7qKzk/nka19KT6GBKqU8+469xqc+ShsW3jo7GZQ8ChveAbI3s5f2X2KbAboGYt2eetYdhEZU5lVh9z2r0e7af1uwxS9M3f5ekLIGsvmmAIK0sDbWyWriIXNDeR9Vv2knkhH6h/XA86zgu511Gl4AuGo8b0CUAAQgwzw9hAQKRACl7UyCXyCF0VO+lxgi5/7v8K/lod087awzPZOsT1uNK/hXc2/5edA3savJxKrIrcOC1AxCIBAiOt5/PnfGPxCOkVwgiBkfgbcnbEDrYV5/axvO3op77gOnp5IniYuCVV4DRo4EFjVoY+rr4ouL1Cr3HrS2tRfGtYtSV1+nd1xaJS8Rw8XWx2/OS6mpAGlQOAPByahrI/XTMp/jy3i8R4RXR5D5D/HrxVzztVoMHQ33RoZ0M+Qnb8EHCRhRHjcKTfZ4EAAx8YSDiH47XuXCHNHXmmzNI3JiIt+reAgDcqeQCf56CILX92vt0xEWcsOjYKOPPzAQC+i8lxN4IBAL4+fm1mvlbnV+Nfxf+i+SdyVr3qbpThaQdSbSSx44deusQfuz2IwoTCpvcxzAM4qbEwStC+8q/1sJe5u+dhl7qvr6qjJOF15Ygtff9KCwEpFLrjY0YJ79hUfDdmUFubkCHDtzX1yjZ1iDWnL93Z/xJxdwkPPjWQSzzXYbKbG4HfYE/llWV8Wyn4bqfkxPXAxKgcp/mZkqpT/71VybjyrMCqsCfnx8XpDWGIRmhrZW9vP8Sy/tn/j/4ofMP1h6GzXALdINfRz+bym7UN3+FjkI4eTg12X6tgDvB6RrYFUKBerCks79h2SisgrWbMnJdZ3NBMUl100auDx9+GNPXTEdAV/sIZBZcL0DK3hTI6mUoEXNvfH4umlepPbT1IXT5oUuTkoJ3C+kVgsdOPob4R+PNPdwW5R7kjvZj28PR3dHugn4AN39v/nYTV1ddRUUdF9DzcvLC+fNAcTGwdavqHMkYfRf1xSuFryB8QLiZR9zy9r20D1/4f4GaAhN+cCuqzq/Gic9PIOdcLqqqFZAJuRNKb2fvJvt28O2AKO8oCBjjz7sUrAI10hosLwek/oMR778N1c63cI3dgNM5p5X7+cX6Iax/WJOFDq1BeWY5cs/ntsixB744ENP+mgaRE5dfl1/FBf68HdQDf7F+sS3y/LrQWbqZyeX2cQJDCFGRy+W4detWq5m/lbmVuLLiCgquFWjd59a2W9gwZQPuXLxjwZERc+r9eG/0X9xf64dNVsGirrwOcknr+LvWxl7mb27DOWZYmGqbf6gjGC/uqnGxCRVVEjcl4ocuP+ic68T8+Iy/oKCm91GfP+NYc/42zvhjWRZfBH6BDVM3ILRvKHos6AGFTKG8H9Ae2Kmo4LLHGEbz3wSgyvKlcp/mZUqpT6dG17H5cp+lDW1UTMnSNaQHZGtlL++/xPJc/FzgEeqhcx+5RI7/XvsPV1e37jdMWZ0MQkchnkt6DsPfHm7t4Sjpmr8KuQLZp7NRkdU0QyivOg8igQjdA7s3ua+TfycAQFJJktbnTT+cjg8dPsTlFfbRU23Mp2OwNH+pxtKkfrF+6DGvBzxCdP+t24rzP5zH2glrIamWKPuH+br4atz3ZvFN3Cy+qTeI6+juiIjBEXaXHVRTWKPsRZl3KQ8ZRzOsOyAjyeVyHP/sOK6uvqos9enp5InMTO5+qRQ4ftyKA7SCkN4hiH8s3m4WFfDKM8px8PWDSD2UBamgCmC4v0tdpVtNIZZyJYkVAJwK9sNLfhEOMh9uDHXlavtKa6V21b/UUGsnrMWWOVta5NjhA8LRc4Gq9EuhmFsp7O+kngnd0U+9EpslUOCPEEIA1NXZZzkDTUJ6heC18tcwYMkArfu0u6cdpqyYgoAu9rFCkTQVPjAc478er7WUxfkfz+Nzn8+Rfar1p5jYw/zlM/74wF9VXhUKrxcg0Jc7qS5smripFyNkIJfIIS6xj94irQXf409TkKdHD0AoqUX6+jOQiCmN0xDWmr+NA38KmQJdZnZB5LBIdJ7eGdP/mg6fdtyHYX2BHX5uBwQADlqqaPN9/ijjz7xMCfw5OKiy+vjAH5/x56v5+qdO3t7cbYX+Clqtkj28/xLLm/C/CXjo4EM69xGIBDi17BSSt2uvUNIabJm7BX/d85dNLsTTNn/rK+qxYvAKHPv4WJP7nuv/HGrerMH8wC/xwgtARobqPj7wpytY5Bnmibip9lGVRFwixt4le5FxOEOthQYAlKaWYsO0DUjaoT3IaWt6PtwTk3+bDLgCNVLuDVRb4I/P3rySf0XvceVSOcozy801zBYnFUvxZdCX2PbwNgDAzkU7sXXBVusOygTDVw/HpN8mKUt9ejl5KQN/AHDggPr+Hx/7GPE/x+PPK39qPWZ9ZT0S/05E/lX76nkoq5dxv0MWdvHa0lhg90A8cf4JREzoBpmQ+106Ch3hLGq62ECmkOHtQ29jzpY5ykCeoWok3Jzv5wRg0F+Q9FkNB7k3AKBUXKbcL3V/Kj5x/aRVLcqpq6hDxtEMDH5lMIa+MdTsx2dZVrmQgDfH61sMuXkWg31mqm3vFNjB7M+vDwX+CCGklWEEDJy9nOHo7qh1H7+Ofuj1aC94hlGzIXsjq5Oh+FYxpLW6gwpBPYPQa2EvuPjZ1wrM1orP+AsN5W4TNiTg554/I0jG3WFK4K/LzC5YkrIEMaMsv3KsLdMV+OvcGXCRVsL33D4c/PisZQdGDMayqgw+T09A6CDEtD+nYfDLg5vsy2f8aQvs8IE/fm5rQhl/5seyqh5/xpT6ZJimff6ak/Hn1XB9qa0G/ggxFSNgsCR1CSb9OsnaQ2kxCrkCTp5OcPZy5rI63jyIvMt51h6WXkJHIcZ+NRZdZmru0+cgcMTm1b5ITQW2b1dtj/OPAwAklyRDrtAc6PTr6IcH/nkAsRMtX+7MWHVldbi+7joSNiQoy4HzKnMqkfRvEjZM2YCdT++00giNEzEoAr0f741KObeSiQGjNatoSMQQAMDxLP1pY3/P+hvfdfhOWSnB1snqZOjzVB/EjOY+Pw15bQjGfDrGyqMynluEG3xifODr4osuAV0Q4RWpFvhLSlI/78yrzsPVgqu4XXpb6zGrC6qxefZmJG5MbMGRm5+sVobuc7sjtJ+Ok3Eb5ejmiNC+oVC4eUDaEPjTVOYTAEQCEX48/yM2JGxAWlma3mNfzb+KjQkbUSutRbWEO2l+xscBzMkH4eJUBwc5t8ixrLZc+RjvaG90n9dd2fe8NTj3/Tn8OfJPuPq5ovfC3mY/fn1lPT71+BT/vfafcptAHAKfmv6I8Fb/m4zxjTT78+tDgT9CCGllqu5UIf9Kvt7AELFPhQmF+KHzDzjz9Rmd+0UNi8KU36cgqLuW2nPEou4O/EUMjsCI90bArxOXdWtK4I9YHssCRUWAR3Eaqs7eAKtQX93n7AxE9/FDaUgXVHnZX3+MtkIiUfV305QtdvyT4/hn/j8A9Jf6zGu4hqsr8EcZf+ZXX6/7d6gLH/i7O+OPAn+EmMeJz08g8W/9F4592vnYXYlAYwiEAkxbNQ2z/p6FqrwqnPj0BNIPpVt7WHo5ujti0EuD0H5se433Jyaq3s8uX+bOjQAgyisKTkIn1MvrkVmRqfGx9sSnvQ/uWXYPkv5NQt4l9YBt9IhovCN7B6H9Qm2qd6Mh+DKfPi4+WnuFDY/iytKezjkNibxpf8PG4qbGYcDzAyCrt4/SgC6+Lpj08yTEPxIPgFtI2WNeD+sOykgKuQK1BbWQVEvw8uCXkfhMIh7vshRVVdwCp/h4br+DB1WPCXEPAQDkVWlffOAZ5omZ62ei+7ympXxtmbO3M2asnYHSlFLseX6PtYdjFFmdDJJqCaoqWUiF5QC47E1tOvhyGWMppSka72+cebb2+lo8uOVBLNq1SBn4W1/nCXRaCofK83CGNwCgrFaV8efX0Q8z1sxA+3s1v/7bo24PdMOglwehw4SWybaT1coQOSRSLdv07l7yPJFA1CJj0IUCf2ZGzc0JsT8CgQDt2rVrNfP36l9X8UuvX1CYoDuSsGLoCvw5SnupB2KbXPxcMPSNoYgaEWXtodgEe5i/LNu01Gf4gHCMfH8kQtq7AlBlkRkr/VA6Dr1zqEl5CdIySkoAmQwITjuN/Y//jSMfHEHaQW7FZdKOJNzefRtdeoiQ1mcWityirTtYO2Ct+Vtbq/ra2RlIO5CG7Y9vR3ES12wz/3I+knckQ1YvU35gq63leqbczZiMv6IiVbCJNA9f5lMoVO/bZwjnhupJ4oYqSeYI/JWXG/9Ye2cP77/E8liWxeF3DuP6mut6963Or0ZpSqkFRmVdjIBBWL8wLLq2SGcrBksyZf4Wi4sx6s9RmLdtNlhw553FxaqsIqFAiC2zt+DikxcR7ql98dPxT4/jwOsHtN5vKxiGQdSwKAx7e5jGnpUCoQBPnHsCI98bafnBmWDTzE34fcDvKBFzb3p+Ltrf9Dr5d4K/qz/qZHW4eOeizuP2erQXxn45Fo5u2qsNEfOqLa7F/vH7cfitw8ptfLZfSAgwYQL39aFDqnOcUA/uRPVO9R1cvAh89lnTRUsOrg7o9mA3BHYNbOkfoUVkn8xG2n79mXC25PLKy/jU41NkH8+Ae10cJkvW4pMxn2jdv70vF5DTFPj77MRneOW/V5TXBbIruRUa53LPKQN/aQIfoPAYcOYxeDhwGX8V9eXm/JFsjm8HX4z9YizqK+rxU4+f8N+r/+l/kBHcg90xf9989H+uPwCgTlaH7eLXkRb0NVzd1RdEWOOcmc7SzUxbryVCiO1iGAaenp6tZv5GDY/CyP8bqTc93zvKG16R9lUDnQA+MT4Y88kYRA7RXSZAXCLG1oe24sqqK5YZmJXYw/wtK+NKyjEMEKze3xmBDZ+riopMO/bNrTdx/KPjqMikdBNL4DMzaweOwvj/jcepL07h/PfnwSpYHHjtAP6Z/w+Cvbn6gTk5gLiY+i/qYq35ywf+nJ25eZl7PheX/7gMSTW3qn36mul4rfw1iJxEcHdX9YTTlPVnSODP05P7x7Kq7F/SPI37+xn758O/7vILLvhSn9Tjzzj28P5LrOOpS0/h3i/v1bvfxukbsWrEqpYfkJWc+vIUdjy5AwB3QT2oexCEDkIrj4qja/7mnMnBL71/wc2tN9W2F1QX4EjGEdysPwgGjPK19PJl1T73dbwPvUN6w1GoPQiUsjsFV1ZeMceP0aJqy2rBsiwGLBmg7PvLy7uch4yjGXZT3hIAnLyd4OLrgoHhA5HzYg52z9utdV+GYTAschgA4Fhm016P9ix5ZzL+mfcPSpK5iNiFny/gfzH/Q9FNEz+MWYGDswMGPD9AWa4UUAX+oqKA/v25hUmlpcDjjwM//QT4Oaky/tavB06eBLZts8LgW0DyzmTsXLQTU1dMxTM3nrH2cIziF+uH+EfioXD3hJMsEP1d5uL+Lvdr3b+Dj+aMv5TSFLxx8A18dforHEo/BAD4dMynAIC0sjT0Ce2D7BezsX/+fqDrm0D8Mng6egMAqqVVauWZdz+3G8c/0V/m1x7UlddBIedep118XaCQKSB0bNn34fzqfJwRfY5bYW/A21P9uaxxzkyBPzOTy22vaTMhRDe5XI7r16+3mvkbMTgCI94ZAbcA3bWvZqydgWl/TrPMoIjFMQIG11ZfQ86Z1t1Uyh7mLx8YCA4GRCKAVbD4seuPOPT2oSYXoI018IWBeDrhaXiGU79OS+B/T77dQjFgyQA8fPhh3L/pfjACBlNXTMXC0wsR05lLJ6rdthdfR36N2rJaHUds26w1f/nAn0tDhbmhrw/FqyWvIqgHVxpZ5CRSfjBjGFWZFr5sC69xNm9IiO7npHKf5tU48GcsPvM6N5f7HVKpT9PYw/svsTyGYRDYLRB+sfonVM+He6Lfs/0sMCrrSNmbght/31B+LxVLkXM2xyZKIuqav7I6Geor6psEtUpquRdLR5k/OnYEJjW0Z7x0ybjnnrV5Fl7MedGkcVtS9qlsrBi8Ask7k5vcd/Lzk/hz5J+4sfkGDrx+wC4qb0z9Yyrm7ZkHB6EDwjzDlCUDb98GFi4E9u9X339k9Ej0Cu4FXxfdq2LEJWJsnLERZ7+1j97WBdcLcH3ddUhquMVeIhcRFxCQ2k8Q19HLEWELw9BxakdMXDsR3X7shiOpXBuQqCju8+b77wNdu3KVSnbvBi4f505U71TdQXpDxeEjR1Slenm/D/gdfwz+w3I/jBlkncjCxV8uQuAgsLvFSO3uaYepK6dC5sW9Z+rrW83P29SyVLXtRzOOAgCGRg7FmHZcz8oIzwg4Ch0hkUuQV5WHcM9wxCS+Blx8Hoh+ED5Ofhh7uRhnJ0kgFKgCVLe23tL4umePti/cjk89PoVcKgcjYPDsjWcx+qPRZn2OlL0pOPT2IVQXcFmVBdXcBQMnaRA8PdX/Hq1xzmz54qKEEGKD6KIFsRf7lu5DaXIpHtz+oM4TW2dvZ7xZ8yZELq3/rd7W56+yv1/BZayffAsDXxoIRsBAVi9TBv5KSrh+VUIjF6D5tjchRYWYrKAAAMsiMAAAGIQPUJWyCh/Ife3OJfyhzCUMAyfWQFIladU9jJrLGvO3ruF3xJd8ZBgGLr6q31F9VT2yT2XDO8ob/p384enJBf3uDvyVl2vP5r1bRARw44aqJBppHr5Mp4sJU4vPzrxzB6iuVpVwNSXjjw/8VVWZ9hpu72z9/ZdYnlwqR31FPZy8nPRmt/Vd1NdCo7KO+fvmQ1qjqhF9ctlJHP3gKJ648ARC++hIE7cQbfM3emQ0lqQuabK9qIYrh+0o88P48UBsLLf9+nXuddTBgcsm2pS4CVKFFC8Pflnj8d2D7KMnnn8nf/R8qCcOvXkIzt7O6Dy9s/K+vov6InpkNFL3peLKqisY9uYwOHkaWXfaRuzZw1W0+OknoF07oENDG6zF/RdjyYCmfwd3c3R3RPLOZLgFmbASxwqGvTEMg14cBIEDlwcT/3A84h+Ot+6gTMDP36SSJKSVpaFHDRfBi2roBtKhA1fOc+dO4JdfgMKkaDBeDIrERahk78AZoSgu5np2duumOm5A1wAwQvsKno35ZAwGLR0EkZMIt3ffhme4p3Ixn72oqgIqXa4jRZSKW8Wd0Mm/k8b9tPX4O5l9EgCUmboAV365vU973Cy+idultxHjEwO4xQBSLkDl5iqAo9wPkjr153jmxjNw8rDP17O7hQ8Oh5O3/vOR5kjdn4ozX59B7yd6AwDyqvMBAI6yIGWveGuijD9CCGlltj+xHevuW6d3v+JbxTj45kHkX8m3wKiIuVRkVqDoRpHe1WwMw8DB1cHuVr21Rnzgz8eHK0XiHuyOp68/jbFfjIWPD7cqU6FQZZ0YSyFT4Nraa6ivouZhLa2gABBJxChf+qHW/jTOzoC/P1Aa1h3xH82kkso26O6Mv5wzOcqSTwBQnlGOtePX4vp6rkcV/6Ht7lKffLZfYCA3jxuTVEtQX6mak3yfP8r4Mw++VyIfvDVG48Af/7rr4cFdtDaWhwcX+GVZzaVgCWlrCq4W4IuAL3D6q9PWHorVCYQCtWBQ7MRYjPpolN6qLLYqr0KV8Td0KBdg8PEBJBJuYQvAlTh7Yd8LWHZymdbjyOplyDmTg+JbxZYYtsl82/ui7zN9IauXoaawRu2+6JHR6LuoL0Z+MBJLUpfAwc2ENxALO/31aST+nYjNNzbjpX0vYV/KPrCsqlSrTAZ88YVqcZShnyFFTiK8JX4Lk36a1EIjNz+RswgCof1eDs+7lIezL5xF+sF0VNRxJQeK73Anq3zgj9efazuG3DQv9Avtjzj3vqh3UF3/OXJEff+pK6Ziym9TWmroLYIRMHALcIO4WIx1963D5ZWX9T/IRlz4+QK2PbINVaVS5PquwQ8l0/HrxV+17s/3+MuryoNUrlpYcir7FABgcMRgtf07+nUEAPx68Ve8tO8lbHToDfgPALZFwdeNq3Vfe1dxGmcvZzCC1nENafDSwZj6x1Tl9yW3S3D43cNmvQY68oOReObGM/AM4+ZgbjmX8ecsDabAHyGEEPOrLalt8uFEk4rsCpz49ASyTmZZYFTEXGZvno3FKYsN2rfgegGyT9FVZmvjgwMdZvXCqyWvIqBzgPI+hgECGr7l+8cZK2FjArbO34rzP55v5kiJPoWFAKOQw29QR/h11F7GjC/rSNldtunuwN/G6Rux7eFtyvt92vlg0i+T0GVmFwDaS33m5XG3msp8bpi6AfuW7lN+T38T5nV31qYx+MBfbm7zynwCXIYf//dRXm7aMQhpTZx9nNHnqT4I6aOn/jGAyysu48/Rf6Iqr3VGzbNPZ6stKgnrH4bhbw23+QVBeZfzcHnFZYhL1PsUZ5dwgTpX+MHFhTuH7dWLu48PHvEXmYvERSgRa17RVlNQgz8G/YFzP5xrmR/AjMIHhOO10tfQ9ynN2alekV7waedjF0Gkw+8cxpUVV3Ag7QC+PvM1zuScQW4uUFzMLV7y9+c+s/x6V8yhVlqLvKo8nccWiGz/5+flXc5DzlnVyZi4WIzTX59G1gn7uSZSnVeNghMFqLpThYp6LvCnEHtBJGp6ThoQwAXo5XLg10HH8LLXeXiJeyM6mrv/xAkueG/Pcs7koCy9DN7R3rjv5/vQ69Fe1h6SwTKPZeLqn1dRLRZAKuR+l97O3lr3D3ILQuqSVFS/WQ0HIbfgoFhcjKSSJADAoPBBavvH+nKp2VtubsHXZ77GnpQ9gNAFcA6Em6sUt4M/wf8lzsHlPFWwtDyzHBlHMpS98VqTypxKHPvwGNIPp5vtmE4eTgjoHKB8HcwubQj8yYPgZAOJk/bz6mwnBAL6LyXE3ggEAsTFxbWa+fvAPw/gifNP6N0vYlAEnrnxDHov7G2BURFzMnQF5o4ndmDzA5tbeDTWZQ/zl8/4CwuDWjlBXlBDJRJT+/x1nd0V93x+D/o82cfEERJD5ecDUhdPjP71QfR6TPuHSj7Ik3I4G78P+B0pe1O07tuWWWv+3h00GvnBSPRf0l95v6ObI/o82UdZJohfrXl34I8P6odqqNhWcL0ARYlFyu/5wBIFh8yjOYG/4GDugnVdHZCWxm0zNfAHqMp93v330drZw/svsTzf9r6Y9PMktL+3vd59q+5UIf9Kvlp2dGuy+p7V2Pv8XmsPQyNd8zd5RzK2L9yOqjvqAdm8ci6Q5+Xor9zGB/6OH+eynt0c3RDpFQkAygvRd/OM8MQ9n9+D7nO6m+NHaTFX/ryCX3r9goLr6ifokmoJvgz+EofePgSFTIHKnErUlddpOYrteOzEYxj39TiU1nJZPr4uvsqAbdeuwNKl3NcHDqgCQauurIL35954YPMDKK8r13rs0pRSXFtjH9VH/nvlP6wZt0b5fW1ZLfa/tN+ueprFTY7Dy5Uvo+ODHSFTcD1DHeReCAtrWoGCYYCOXDwe6SmOyvOeKVO4YG9NDXDhgmr/23tuY//L++3id8lbM34NdjyxA4yAQd+n+tpVmc8Za2bgjao3UFMnhExUDgDwctK+OIRhGLTzaQeRQPWLPpPD9Xfs5N8Jfq7qJ7TP9HsGVxddxUsDXwIATGTTuMDf+PNgnYJQ5LUXx8s3qJUOPfHZCfw56k/Ultp3n/rc87n4e9bfyD6tWggfPiAcT115Cv2eMV9/4eJbxWr/VzkVXDahpyAId1+2s8Y5M52lE0IIAEdHR2sPweIc3R0R0DkAIufW3wOutRAXi3F55WW11cO6DHppEEZ9OKqFR2V9tjx/5XIuWAQAl19ei4NvHmyyD5/xV1TU5C6DCB2EGPLqEOoj18Lkcm5VNABlb0Zt+LKORZVOKEkuabXZDOZgjfl7d8Zfnyf76LwA2biPW2O6An+Pn30cU1dMhUKmAMuyauVCWbYZgycAVKU+TVlJ6+CgmsPXuWquJvX34/F/H20xqGvL77/E9g1/ezheK30N/nH++ne2MyzLYsxnY5Q9f3h7X9iLn3v+bKVRqdM2f3vM74G5u+bCJ8ZHbXuluB4MK4Svs+rCcr9+3MKJwkLgww+5gFGcXxwA4FbxLY3HZxgGQ14dgojBEWb6SVqGvF4OcbEYSf8m4c6FO8rtkmoJfNv7wsnLCemH0/F1xNdI2JDQ8uORyrFl7hZkHs806fHB8cHw7+SvMfDXqxcX/BMKuXMUfiFLfHA8ZAoZjmcdR/tv2+P7c9+D1XASk7gpEVsXbEVJkol9Cyyo/+L+uHfZvcrvvSK98OiJRzFgyQArjsp4Tk5OqJRyvygGDIQKtyZlPnmdGtrF3brFLXiSCaoQGFmOESO47Tt2cG0nACDzaCZOf3UaNQX6K0jZApZlMfqj0WoLYBVyhd1kqzECBo7ujlzP6YaMPy9n47LCy2rLEOAagMHhg5vcF+MTgx5BPcCCm7cjJDeA2z8B4D4HOci41/nGgf2us7piwncT7P46YfGtYtzYcgP1FaogtoOrA4J7BkPkZJ6fjWVZ/NrnV2yZu0W5Lb+KWyziLdLTAN5CKPBnZgqFfby4EEJUFAoFrl+/3irmr0KuwIVfLqitatGltqwWhQkm1hckFldwvQDbH9uOlH2GZQ91nd0V8Y/Et+ygrMzW529hIdczw0HEouxWAcrSyprs09yMP15NUQ0SNiTYzQcda6kpqsHeF/c2WcmuT0kJ96HYtygJFz/Zh+r8aq37Kss61gfgleJX7KrkjCVZa/7eHfjTZOeinfgi4AuwLKu31KemwJ9PjA+q8qrwbftvkXEkA+7u3HaFAhCLm+5PjNOcjD9A9Tvj+1I1J+PP25u7ragw/Rj2yNbff4l1JO1IwuYHNxu8SK21YhgGAxYPQOcZndW211fVQ+Ag0Bg8sSRd89ennQ9iJ8bC0V09MDjX+ztMvCjFfT5Lldvc3ID33+dub94Eli0D4vy4KIO2wF9j1v5/0KXPk33wTOIzOPzOYZz/SVVO3z3YHY+dfAxDXhkCv1g/9F/cHwFdA3QcqXlqimpQmlqKnDM5SFifgHPfGl8ilVWwqKuog0KmUAb+vBx9lYtfevXiMsPuPt+JD47Hzjk70dm/M0prS7F4z2JcyrvU5PidZ3TGrL9nwTvG25Qf0aI6Te2kFiQSOYkQOSQSHqEeVhyVcYpvF+PgbwdR2NAnwgmeYCDQGvjjM/4uXAAuer+Jfb18savgZ4wdCzg6AgkJwKpV3D4DXxyIJWlL4BVl2yWJeQzDoP9z/dF1dlcAwMVfL+ITt0+Qd1F3eVpbcefiHRRcK0BVFSDjA386Mv4AYH/qfsz7Zx6+Pv01AGBBzwUoeLkA3074VutjqiXc59a/Ax4G4j8Fbn6JQNdUOMi9AQBldaprFDGjY9D/uf5w8rCBOpXN0HNBT7wlfgsxo2PUtkvFUty5cMcs10xYBYvBrwxGtzndlNuejfkOQ26eRQ+HGU32t8Y5MwX+CCGkFakrr8OuRbtwZdUVg/bfPHszfh/4u01/6CIqQT2C8OD2B9Hxvo7WHgoxkLIHWCiDl3Jewv0b7m+yD595YmqPP97Z/53FljlbcOf8Hf07t2EJ6xNw9puzTXrX6MNfBPGrTMeZr89ALpVr3ZcP/OUXMJAr6HTb1jQO/CXvTMaPXX9E+iH1Xg/uwe4Ijg+GtEaqsdQny2rP+JPUSFCZUwmPUA/IpXLUFNbA0VGVnXZ35iAxXnMDf2Fh3C2fOWiOUp9tMeOPkLsVJRYhcWMiJDX6m0ZVF1Tj5tabKEtvuiiqtZr6x1Q8eeFJg8v2W4O2z4UlJVxmUaC/g9r26Gjg7be5bOqzZwGmVH/g79raa/gy6Evkns0127hbgoObA2asnYG+izT3+POO9saEbycgapiWiIsZXFl1Bd91+A4CkQBv17+NWX/PMvoYlbmV+Nz7c+x/Zb8y8FdZ6Iu6Ou49LKbhunjj6gS8CbETcO3pa+gXypXG01TC1b+TP7rc3wWufq5Gj80WyKVyuyjXyrux8QZOP30alWmV6OzfGd5y7tqApoVoABAbywV26+sBZ0kYWEaGA+n7ERoKvPgit8/WrVyZV/cgd/jE+EDoILTQT2NePu18EDM6xm6ub22dvxVb5mxpyPgrB6A/4y+jPAPrrq/DgfQDym0Mw8DN0U3j/t+f+x6/XfoNACBzjQCkVcDlVxDomAAHOZfxV1bbOt+HRc4iCB3V/5aPfHAEv/X7DUU3TCy31IhAKMDI90ci/uF45TaH+hD41PRHmKeWCWlhdCWCEEJaEUd3R8zfNx/9n+2vf2cAPRb0wNA3hkIhpdXa9sDVzxVxk+Pg085H/84ALv52Ed/HfY/iW8UtPDKiDZ8BoquMnLkCf93mdMPk3yfDN7YZNevagMhhkRj5wUh4hHAre3UF8Brjs7Tqht6D59Of17ky2MeHCyopFED69Woc+/gYMo5kNHPkxFwaB41k9TLIpfImHwpHvj8SC/5bAEd3xyYr4CUS7uJmXR13ISXorlYi6Ye40l85p3PwYtaL6PYAtwqUPw4F/nRbswZ46SXdmZH879CUUp9A04tjzSn1yWf8tbUef4RoMvT1oXhb8rZBPZbyr+Rj04xNSDuQZoGRWVZxUjF+6vETLq+4bO2hGO3v+//GJ+6fNLlwXtKQxKlpoUS3bsDcudzX6RdiAUCtZ9TdXP1c4R3tbfA5mDWkH07HlVVX0HlmZ4T1C1NuzziagROfnUB1gfbKD+YU1j8Mg14eBN/2vk3OVQwlchKh18JeCB8Yrgz85aZwb3zx8VD2odIU+AMAkUCELgFdAADpZeoLpewJq2Dxfdz32P/KfrXtP3b5EatGrrLOoEzQYWIH9HyrJ3rF98KNZ29gYi6XBaptEZOLC5TZgP6VYwEAJ7JOoEZSg6FDgTlzuPt++AEoKZShIqsCdRX2EQi9te0Wfu75s7IEbrt72mHe7nkIHxBu5ZEZZsjrQzDo5cGorVWV+vR29tb5mA6+HQBwr7EyhUxvkJMP+gFAgEAO+A0Axp5BjdsIiBoy/hqX+sw5m4OfevxkkRLGLSlpR5LG6madpnXCiPdHwNnbxNWDevCvn/zrqbVR4I8QQloRkZMI7ce2N7ihcc+HemL4W8NN/hBBLMvYD8cCkQBCJyFk9bIWGhHRp7rhmoAbI0bCxgSUppQ22YcP/BUXq/ormCKwayB6L+xtt6ttLSWkVwhGvDsCrv6uOPf9Oex4fIdBj+ODEK6eInhHe0Mg1H4azTCqrL/sdBkOv33Y7j88tSaNM/66zOyCxcmLETk0Uuv+/Ae3igpuRfTcucDHH3PbwsIA0V1tIrwivNB/cX8ExwdDIFL9nbSlwB/LAseOAVlZxj9u507g9m2ubJw2fKZec0t98pqT8cf/fVDGHyEcoYNQ53skL7hnMGasndGkDFdrIKuTQV4vb1JKrDCxEGe/PYvKXNtdKRDSJwQdxnVokpW4om4yLrSfCdYtX+PjJk7kSn7K0gfi9wHncPKxk1qfo8P4Dnj87OMtminXXFdWXsGOx3eAlatfVE/Zk4KDbxxU9o3a/vh27F68u8XGET0iGmO/GAu3QDdUF1Tj7HdnkXM2x6hjuAW6YcrvUxB3fxyqJNxJSMYN7o2vV6Nq9NpKmwNAjDc3T9PLmwb+Mo9l4ouAL2w+0C2rk2nsW9Zldhd0nGQ/FXWC44MRfX80XHxdwLJAacPHS13nMny5T7f6DghyjIZUIcWxzGMAuMBfSAjXnuLSP5n4Juobu/ncIq2VQlItsdvrWfEPx6Pj7HgAQNfsb/DtuB8Q7R2t8zF84C+tLA0OHzrAb5kfPj/xudb9Y31jlV/PSn8POL8I8B8ABzdvZY+/xqU+hQ5CyGplNr0wQx+FTIFNMzfh4BsHm9wXMSgCI98bCa+I5pezTd6VjL/G/IXc81z2er2sHqtyX0Na0HK4uEubfXxzsO9OjTZIIGi7sdTjx7kThfh4a4+EEOMIBAJ07969VcxfVsECDGy6fAwx3frJ61F4vRAv5rxo0O+416O9Wn1vMVufvzUNfdGdq4ux5cEtGPf1OAx8YaDaPr6+gFDIfdgqLQX8/Zv3nHUVdRA5iey+IbclnPz8JDzCDOvpwf8u3aoLUJXnqswY1CYiggtelMi98djJxxDSJwSVOZUozyxHWP8wuy2hY07Wmr+G9PirLqjG+R/PI3JIJDy7tgfAZeWuWMHd7+cH9O4NTJrU9LHB8cGY8O0EAIC4RIyDbxxExOAIeHjEA2gbgb/bt4EvvuDKh32rveVIE6WlqrlWrCNZ3Vw9/njU4894tv7+S6wj/2o+JNUSRAyO0Huu6h7sju5zu1toZJYV3DMYzyU912R79sls7H1+L/zi/OAZZr10AF3zd9ibw5pskylkSHfYCfgAfr4/aTymqytw333Apk0eSNzfD4+N4xa0MYwqo8yeDHltCLrN6YaVw1bCxdcFC/5bAAAY/PJgdJ7ZGd7R3gCA3HO5TfohthRxsRh7l+zF0DeHmpTRJBKIkPNiDkrEpfi/xd4AVAEhQHfgb36P+RgVMwod/ZoGyJy8nBDQNQDOPi2TQWMuDq4OWHR1UZPtYz4eY4XRmK7x/K2s5D5DArqrF3TqBOzfz5Xr7RM8ALuzMnCz+CYmxE4Aw3DnRXl5QJ2rLwa+NBBB3Q1bSG5t3ed0R/c56u8jR94/AicvJwx6cZCVRmUc/rw3qnoWFg/UvS8AhHuGw93RXdm3r6yuTGd5UD7wNyp6FBTtugGesYCsFq5OgKO8aeAvpHcIFt9ebOJPYxtYBYsZa2bofE1iWbbZ101rCmqQdylPWUGtsKYQ+2uWgQlzgLfHi032t8Y5M52lE7MoKOAaOX/2GbdSlxB7I5Ho70NhDxL/TsTHzh/j1r/6m6kDQEV2Bf4c9SfOfne2hUdGzCGkTwhiRsdQYPcutjx/+Yw/zxg/zPp7FmInxjbZRyhUXXQuamap+RtbbmCZ3zLc3n27eQdqpYpvFeOr0K9w6fdLAIBnbz6LhacXGvRYPuNPsGolti7Yqnd/PuMvJweIGBwBkZMICRsSsHLoSpz55oxJ42+NrDF/Gwf+Lv1+SeOqZqlYimP/dwwpe1PUSrUwDPDkk8DKlcCSJUC7drqfy8HFAZd+u4SMIxkaewW2VtnZ3G1GBlca1VCZmaqvdb0eNjfwFxjIvfYC3K1XMxb98o9ta4E/wLbff4nxyjPKcf7H883qjXTorUP4a8xfdK6qRcdJHfHI0UcQ1j9M/84tzJj5W1ytuijcLkR7dGHKFMDRkVv88cYbwAMPAIsWaS7dfH3ddRx8q2k2hq0I7BqI2Amx8I31hXeMt3K7q78rwvqFKTOMFl1ZhIWnDDuXNMX6yeux76V9AAC/WD8s+G9Bk0WE+uRfzcfWBVuRdTwLYZ5hiHLpjvo67nJw43Ll2kp9AkB73/YYHjUcwe7BTe4L7hmMR448gs7TOxs1LmKaPYv34IfOP2DFhRXot7IbkkLfg5dX0woUjcXFcbcCAdA5hMu0zapQlWUICOBuK4U+GPfVOEQMjmip4be4q39exbXV16w9DL1k9TL82O1HnPr4MABu8YQhBIwA62asw6uDX8WKKStw6clLeKrPU1r3j/Xjrj+IBCI49/sWCBoNbHJFZPXnCCm7Hw/lFWPX3F3N/nlsidBRiK6zu6L9ve013n/u+3P4MuhLrdn3khoJknclo6awRufz9HqsF14re005XwpqCgAATrJAeHnZxnkQBf7MTNGcGl02QlJt/Ae45GTutqZGdZGTEHuhUCiQlJTUKuavW4Ab2o9tb/AKUidPJ+RdzoO4SEcjHWIzxnw8BtNXTzd4f3GJGOe+P4fsU9ktOCrrsvX5qwz8hbihy/1d4NdRc1oJ/6G7oKB5zxfSKwQd7+sIF18dqUxtWH1lPTxCPeDkyTUGc3R3NPjipFgMgGXhMnYo4h+J17s/H/hLT1eVcA0fGo2eH0xDp3l9TRh962Ot+ds4aHT0g6M49eWpJvt4RXhh0bVFGPnBSLi7cxdEnJyAt94CJk/Wnb3w32v/4Z95/wDgVpe/UvQKpq6cCnd37v62kPHH9yxlWePKfTbeV1fgjy/1aWqPP6EQCG64dunj07xslLaa8Wfr77/EeOd+OIfdz+5GSVKJycfo90w/jPt6nEH7SmokWB62HLuebV0XHAGg4FoBLv56EdX56hdHPEI9EDU8Ci4+1j1P0zV/D797GGf+p75AKb2AS8F2kHnD31d7dMHLCxg3Dijw2oWNlUuQ6bQLd+4AW7Y03ffW1ls48ekJyOpssyUBHwC/f8P9mPzrZOX2ytxKSGtVJdwYQctd3GVZFsVJxSjPKAfAXcxud087uAW4GXWc8oxyXFtzDRVZ3BtVfkO1Vj8/LlDL0xX4aw3KM8px5n9nUHxLvaRA4qZEbH5gM2qKdF/ktxUiZxEUAgWyq7NxuyIR9aICvZULIiOBxx7jFq3F+HLl7RsH/sz1WdTSrq6+ioSN6gv4Hjn6CB499qiVRmQ4WZ0MrJxFXY0cUkElivz+xfHM4wY9dnLcZHx+7+d4tNej6BXSS+fnWT7j73Zpw8Jg5wAgej5Yz+4QKlzBiv0gEqi/rl9ZdQU3ttww7QezAfoWMDl5OsEr0kvrddD8y/lYP2k9Lq+8jIyjGShLL9O4390Ka7gPQI7SQI09/qxxzkw1oIiauvI6/DnqT8Q/Go8BSwYY/LiURn2by8pUJQIIIZYVMzrGqD4Zzl7OeK3sNVqV20rVltRiz+I9GPrGULtetWfP+MAff8FfG768p67SdobwaeeDB/99sHkHacXC+ofhyQtPKr+vulOFgusFiBgcAScP3RGEmhoADAO/qUPRY77+52rfngsmZGRwwaLx44F160Jx504ozteresQRlbqKOrBytsUD13zGn6srMHf3XMglTXtYCEQCtTJH337LBXANadSefzkfpbdV/Txd/bklvPxj28IiucYXjjIygA4dDHtc44y/liz1CXBlrXJzm1fmE1D9XmtqAKkUcHBo3vEIsZaIwRGoWVADoZPppag1VTbQxsHFAd7R3nAP1nOSZIdS/0vFfy//h6AeQU1+PoVcAYVUYbMl2S/+chF+cX4Y+Lwqqyy9kAsGO7P+ehdKzJsHbK46hPPV36FHdyEUu+/Dtm3AhAnq5ezH/288xn411mZ7c60YvAI1RTVYkrJEbfsv8b/Av5M/Hj3OBRZKU0qRfyUf7ce113suaSyGYbA4Wb3kHsuyqMyphHuwu8Fl4+OmxOGt2rdws/gmXt7/MtiSDgAWISREfT9dpT4BYMXlFUguScZLg15CoFugcru0VoqTn59EUI8gdJ5hu1l/+Vfzse+FfXBd7Qr/Tqo/xqIbRUjclMMeMUgAAQAASURBVIjh7w43OqhqDWM+H4PA+YH4M/9PAIBI4WnQucz0hvXDbjl9sLDXQgwMV81xvud8Ya4U66dsRuSwSAx5ZYi5h252x/7vGEQuInR7oJtym1dk83u3WYKzlzOevfkszp8HxF9ewnHfabi+OQR3lt4x6/PwGX+55RmoOzQBzmH3AYNXg21YAMB/LmrswOsH4NveF11mdjHrWCxl7wt7ceufW3jy0pMa53TPh3qi50M9tT7ep70P7vv5PvjE+GDNuDXwbe+LRVcXqfVuB4CUfSlQyBToeB9XAjmnjM/4C0KMjbQvpow/osbR3RHuIe64+ufVJo2odWkc+Cst1b4fIcT2UNDPPpSmlmLHkzuQeSxT/84NvCK98PDhh9H3acoushZlr6ptJ/BF4BcouqE5hYW/EFJi+iJ7YoJra69h7fi1KErUX2OV/1DkZuD1gMBA4MUXucBEQgLw5ZfAnTuAQCZB0qliZbUEonJ93XUs81uGtANpyD2fi7UT1+otsWIK/nfp7AwEdQ9CaJ9QjfvVV9Wj6Cb3t+HubljQDwAW7F+AJamqC4XVBdW4vec2XAVctKotlPrkM/4ALvBnKEuV+gRUff6aG/hzd1eVDW1rWX+WpJArUJZehqq8VpqKYmWsgkXn6Z0x/a/p8InxschzMgIGj518DCPeGWGR57OkrrO7Yv6++fDvrN64uSqvCh86fIj9r+y30sj0e/r605i1aZbatqyGlRjuAv0vmG5uwIwR3IVmuddtdOvGlXxevVp9P49QD3hFerVoxlxzRAyNQMzoGCRsSMCRD44A4IJuPR/piS6zVBfDEzYm4O9Zf6M0xTIXwo7+31F8E/kNim8avlqQYRiInEW4XXUbX53+Cruy1wBQL/MJ6A/8fXriU3x+8nPcLLqpfgfLVVBI+jfJ4DFZQ9SwKDxy7BG0u0e9TvvQ14fibcnbCOwaqOWRtqmqnns/dJB7GXUuMzB8IH6f8jse7/24cpsy8FciROq+VKP+vqxp1t+z1DJyAa7CS97lPJOq2VmDWAxIhdwJpLezt9mPH+TGTXRnAeBU8B9QdhkA1/JAJqjC+cBnMHfLPLUsuVl/z8J9P99n9rFYikeIBzzDPeHqZ2D9VA2P7/tUX7Qf2x5zts/BlBVTAA1vVYffOYxdi1RVC25kcR+APIWBagtdrIkCf0SpMqcSApEA9/10Hx47+RgEQsP+PFgWSE1VfU+BP2KPhELbXGlorCt/XsH+l/drzF7QJu9SHs5+d1atZAmxPUU3inDpt0soTTX8RVbkLEL0yGh4RdjHqjdT2fL85TN7PELcENA5AE5emlcC8yeGze3xB3Arj/+65y9c+uNS8w/Wyvz32n+4+Y/qYkWHcR0w6ZdJ8IrSP0dqagDPolRkv/8HMo5mGPR8o0ZxmWKdOnGBgWnTgL7Z29DtyI/4Z7Phr9OtWeP56xfrhx7zeyCweyDOfXsOKXtSUF9Vb/bn5INGIkhRX6n9+Dse34Efu/wIWb3xZcgaX8hMWJ+AdRPXKWtrtdYSWo2ZEvhjWVVvQIDL+NNWqYf/HZpa6hMA+vfnAod9+ph+DIDL7OX7/LWFoG5jlnz/Lb5ZjG/bfYvzP5632HO2JUf/7yh+6fULKrKbF73+uefP2PVM6yvdaSyvCC+0H9sezl7qqxNcfFzQeUZnBPds2ifN0rTNX7dAtyZZinfKuJVpniLDrmQ2Li332GPctkOHgLQ01T7SWinK0sts9uL82C/GYvKvk5G4KRFHPzgKlmXBMAzGfjFWrTpWp6mdMGPdjBbJMipNKcW1NddQdUd14hA1PAr9nu0HkYvhGaOVOZXIOZOD6oqGDyZS7mJ48F1/hvpKfcZ4cyks6eXpattFLiI8nfA07v3iXoPHZA0uvi6IGhbV5O9b5CwyOHvSFlxecRm5u3NRUc+9XovkhmX86cIH/opLBXi95m1MXTG1maO0jOD4YIQPDFfbduXPK/i196+4c8G8mXPmVpFdgfM/nUdxUjFkDYE/L2fzv44wDIOsF7JwZlEimDkyYMDvwLlFcMv6AoAAmYE/YX3COoilqrKXUcOi1Cqf2Juhrw/FwtMLdS4sSTuYhn8X/qvxPYhVqD6AtB/bHuEDwjXGSO794l7c95MqQHr7DpfxF+5tO/93FPgzM1u++KhL+qF0/C/mf7jy5xV4R3kry04YchKWn6/KaADMG/hjFSwyj2VCIaPeEaTlCIVCdO/e3W7nb2PJ25NxevnpJinoutzYcgN7l+xFeXp5yw2MNFvHSR3xSvEr6HK/ceUWWAVrsx+ozcHW5y8f+OsytxceOfqI1v6b5sz4c/ZxRs7pHFTntYF6gkYQl4hxatkpJO9QpdoF9QhCnyf7wCNEf41ysRgQSmshLSpX+zCgT0gIsGwZsHEjsHAhMOSJzsjrMBRnTsjtro+Gud09f9vd0w7TV0+He5A7pq+ejnfl78LV3xUXfr6A9MPpeo5mOD7jr/B4Mj7z+gyJmxI17tdpRicMf2c4FFLDz0Prq+qR+Hei2sr/9mPb476f70NAnC+A1h/4k8vVy3Smp2sP4DVWWMgF9EQiLpgmlWrPoON7/DXO+Lv460X8dc9fSNmXYlCwtkcPbl6OHat/bPrwgb/y8uYfy15Y8v2XZVmUpZWh95O9ETUsqsWfr01igLL0Mhx+5zASNiTo318DlmUhEAnACA3P4Lr0+yWc+qppn1V7p616kshZhNmbZ6P3470tPCJ12uavrE6GwoRC1FXUqW0vqxKDUYjg42RYdIEvLZdelo6Y9jIMbKgoeO6cap8bm2/g23bfIu1gmoYj2I5xy8fh2ZvPar0/sFsgus/pbnJmiS7ph9KxdcFWFFxXnTDGjIrBxO8nwi/W8EjP1dVX8cegP1CVwp2AKOqbGfgrUz8nYxgGgV0D4RZo22UyZfUyjXNTUi1B1sksZQ9EW3fqs1PI+TsHlRJutZGxGX8AIJaKkVSchGoJ93nRx4crVc6y9lOBhlWwkNXLmvRzixwaiZH/NxKeEQaW6rCSgqsF2P3MbpRezVFm/Hk5tcyC7QivCHQJaLiOxDBA1iYIC/bASeAKRsFd/y+rU/WxY1nWpIWP9iT/Sj6urLiCvEt5atsVcgWW+S9TW8Qkl8pRXdD02kr0iGh0nNRR+X1WCbfysUOI5sCfNa5ZUeDPzPQ1kLRVtWW16D6vu9oHqcPvHsYXgV+gtlRDwd9GGpf5BMwb+KstrbX7pqLE9rEsi8rKSrudv41NXTUVz6c/b1TJlJ4P9cRDBx8yKOOFWA/DMHD1czW6d8T3cd9j5bCVLTQq67P1+WvpHn8A4OrnitcrXsfwt4c3/2CtiIuvC17MeREj3jOtpJhYDJSFdsPIHUsRM8q4ov0Mo8pMGv18dwTNGQ250BH//mvSUFoNffOXETCQS+TY9fQuky9E300u50qOAYBPlAd6PtyzSSk2XrcHumHU/42Co7ujwccvvV2KzbM3I2GjarwBXQLQ96m+CGjPXYBo7YG/0lLu/1ko5P72KysNC4jxZT7DwwFfLkaqNQtaU8Zf1Z0qpB9Mx9rxa3Fr6y2Dxiow06dhPvDXlkp9WvL9tyS5BBumboDQUYj2Y9u3+PO1RSPfG4lnbz6Lq39eRdaJLJOOwTAMnrz4JCZ+N9Hgx1xecRmnvzxt0vPZsi0PbsFHzh8ZtVDIkrTN35LkEvzU/Sec+eaM2vbessWYeEmCJTG/GnT8cM9wOAmdIFVIkVWRhchIbnvj94KgHkEYtHSQxUrLGmv/y/tx4ecL8I72hn+cPxiGQd6lPGyYusFiwcoOEzpg9pbZCOkdon9nHdqNaYd7v7gX9f7cqhlZrebAH1/qs6YGkGm45h/joznjD+Cu3VXn2/aiw/9e/Q8fij5EZa56en7RjSKsHLrSbOeaLW3WllkY8+0YVNZxP4dI7qk8bzJUv9/6odMPnXAmh5vrDAMEBHD33dibhdT/UnU82jYU3SzCx84f48j7R9S2h/QKwYh3RsC3vZH/KRYWPigcC/5bAEFsO0iF5QBaptSnkkwM5P0HVKcBUzPAjN4PVxcGDnLuNbisVhX4+2feP/jY+WO7TMKRS+XY99I+3PpX92eB+Ifj8VLuS4garr6grL6iHuEDw5WBY5ZlsTxsObY8uEVtv7vfP+VyIDr5Swy9cQ4LBzyg8Tmtcc2KAn9mplDY36QAgC4zu2Daqmnwaac66fKL80PMqBiIS8Q6HqkK/IkaKg2YI/Anq5OBVbCoq6jDlZVXkHMmp/kHJUQLhUKBtLQ0u52/jTl5OME7ytuox/jH+SNmdAwc3Qy/sEksL/t0NsrSy/TveJdO0zuh/bjWe5HMluevTKa6OJ268QJOfal9RTu/SrO8XPMHbWMZk/XbVjAMA88wT3hHeyu3yepl+KHLDwaVJRM3nA65mmFB94wZ3O3+/aq/kbao8fzNOZuDFUNXNLmY5urvivn75pstkN34/7vDqEhMWzXNrKVsvKK8MHP9THSa2qnJfe7u3Ie91h7448t8BgSo+ugZUu4zqyHWEBWlu/yxQsFlAwLqGX8j3x+JpflL0X1ed/h1bGbNKyO1xYw/S77/uvi6YNzX49BjXo8Wf662zD3IHa+WvooJ302w2HNO/GEiFl1bZLHns5SQviGImxKncTHmyS9OYu8Le60wKhVt89fF1wUj3hvRZIFTSQnAgEFwgGELEAWMAO19uc8ft0tua1wcEdwzGGO/HIugHrZTEq2x8z+cR/LOZEhrpajKq4JcKkdlTiWSdyVDXKS6RlaZW4nlYctx4I0DZh+DV4QXOs/oDLcA9Uy6A68fwKaZmww+Tlj/MAx+eTDq3LiTIHmd5sCfuzsXAAJUixcb01bqEwB+H/g71oxbY/CYrCE4Phhd7u8CFx8Xte0+7XwwdvlYxIwxbmGftQR2D0S1RzV8XXzhIg2Fg8zP6Iy/SC8uGp9ZrmquzJf7vPjxXux8cqe5httiHN0d0X1udwTHW790silc/VzR7p52kDh5qkp9tlDGHwCgJgs4PBZIXQk4eAICEVxcAAe5NwCgvK5cuWvE4Ah0n9cdcqn9taYQF4lx5uszSDuge4GGq78rPEKbVv1x8XXBvN3zMOyNYQC4awjxj8Sj3Vj13qDl6eX4xO0THPv4GICGBYzVwQhFPwzoHH73YQFYJ2ZkeFFo0ub0mNfDoA9XfOCvRw/g0iXzBP52L97N9Rv84T4suroIAV0Cmn9QQtqAwsRCuPi6GFS2rjG+HKSTZzOa5ZAW9ff9f8M9xB1PXnjSqMfdu8y2ey20Zo3LYN9YexniwhoMfnmwxn09PbnyKlIpd3ElqJnXQMTFYlxbew0hvUOoLFqDgusF3Al+o9dHkZMIjm6OcHBz0Pv4mhrAqyAJpScUQHxnk8dRlVeFhDe2I6IoFtkh/ZGfD0RHm3y4VqMyuxKFCYVNtjMMY9YMH77Mp1CoWrSmdUy5lfj30X/RcXJHDFg8QPfODVz9XNHtwW5Ntm+YtgGl6RVA5FOoqVFlxLVGfOAvMJB7bcvN5QJ/vXrpfhyf8RcVxf3/JCVpDvw1Dt46q7fPgnuQO2asmWHy2E3l7c3dtrUef5biFuCGgS8MxIVfLmDX07swf998my8pZ08Ov3sYjJDBiHdHNLkgboyqO1VI2JiAmNExBvewC+nVvEwmWzX0taFa70vdl4o75+9g/DfjLTgiw3iGe2Lk+yObbOdL/xkTXIj1jcWNohtIL09HZzvsg/pC5gsAAxz/5DiOf3Qcz9x4BnFT4vCO9B21TE5nb2d4Rni2yGuSQq7Q2FeqJLkE+Zfztd6vDd/DSyB3hbOzatEKTyAA3Ny4oF9Vleq9jafM+CtrGvjr+XBPo6oOWUOvR3uh16NNT0Zc/V0x6MVBVhiR8ViWu27DKlhsnrEds2dzJ5NGB/48ucBfVoUqw5v//Ok5ZQSGDbG9RbV3847yxoy1ms/51oxfA+9ob0z6eZKFR2U4hUwBRsigpoZp0R5/Ss6BQL+fAZ9eQGUSICmHq+sAOMi4BKDSWtXF/P7P9Uf/5/q33FhakFuQG57PeN6gvp1yiRw3t96EW4AbYkZrD/xru6YWNTwKXhHc7yyxoXNE587mqyhiDhT4IyhLK8O/j/6L/ov7G987igVSGzLA+/c3X+CvKrcKldncWaGtrgAjxBb9MegPhA8Mx4L9Cwx+DMuy+LbDt5DWSPFc8nNNmtAT62NZFiP/b6Sy/yqxD3zgz8UFeHDbA5DWSLXuyzBchkteHlfus7mBP0m1BPte2Ic+i/pQ4K/BP3P/QV1FHV7MelFt+xPnnzDo8bW1QGTyMVz6tBqjnzM98Ofo7ojMIxnw6Mwtqy0ro8AfAHS5vws6z+issQQKy7IQF4nh4udi1AUuTfjAn4sLsPf5PRA5i7R+mHP1d0XOmRwEdgts1nMCgHuwO1gFAAULMAyqq5tecGst+N6VQUHcvxMnDMv44wN/kZGqrEhN5Y/5wB/DcAsmAK6HZ8L6BMSMiUFAZ27BYE1RTZMsiZbSFjP+LEUhU0AhV0DkJIKkWoLaslrud0uBP7O59PsleIR6YOR7I1GSXAJxiRgRgyKMPk5xUjH2v7QfE76fYHDgD+AWWdSW1po1+9qWzd48GyKXlj2nv3PhDo59eAz9l/RHuzHt9D9AjwOOz6G6fR5yFW+hBwzrT/jdhO/wx5Q/4OfqhytXuG2NM/7qK+vxz7x/ED062iaDLvxrTMSgCPR7rp9ygSzDMGp9LB3dHPH4mcdbZAy/9v4VQichnjinfq46a9Mso6p77HtpHzKPZUL8IRf4EypcERysyu5rzMODC/xpCtJGe0cDAO5U3UG9rB5OItWi4eFvUYsBS5DXy7HMexnCJ4Yj8FcuUcPRUX9bibvxGX9ZlarAH5/xJw6PQ+fpZhmu1dQU1tj8ovZDbx/CqS9PgX1uMULKZmHioPaY1im+5Z7QyReIfYr7+uAYoOwyXFxK4SDjosaNA3/2TCAUGFwFra6iDtse2obIYZHKwN+xj49BIBRg6OvaF/AAXKbwvD3zlN8nJCpwM+wNeAcFok72LJxFtnFdla4gElTdqULBtQLUlTetNVWRVYHtC7cj9r5YDHxhYJP78/O5C5sODkB8PLetrIwLCGo6iTDUvN3zoJArwAgYSGulyL+SD79YP7j6m79hMiEA4Hz3knE7xLIshrw6BF6Rxl1JZBgGIz8YifL0cggdW2n6gZ1jGAa9Fxr2IftuNzbfQNK/SRj3zbgWaTpvC2x1/jbu7+cZpr+5uJ8fF/jjV1XX1HC9yHxMaH3iFeWFBf8tQGi/UOMf3EoNeH4A5BLTypWwLFfqM6vbBMx9sb5Z43DycMKb4jfx3nsMcNm8vZHtUeP5ywgYMGh6AnnorUM48ekJLL69GL4dmtevgw8aOTsDt3ff1nluKXIS4bWy14wKNu5evBu3tt7C09efVsuc4Vcc//sgN7erqlpv4K9xxh8f1NYX+JPLgZyGyv5RUapjaMr4q2+Ygk5Oqs8b+VfysWfxHoz/33gEdA7Apvs3IedMDl7MetEiGQhtsccfYJn336wTWVg7cS2m/DEFg5cOxuClmjPnielezHoRtWXcqogdT+xAYWIhXi1+1ejjhPQOwaMnHlUrqa0Pq2DxY9cfEdQ9CI8ef9To57RV+1/ZD68ILwxY0jRb3Nm75edNfVU9knclo+Pkjlr30TR/z31/Dre23sL01dOVJdDq6oA8lwOocUmC0GWJwWOI8FIFjz0bToMbv0YKHYVI3Z8Kj3DjKtVYglwqR+ntUrgFuiF2YixiJ8YC4N5raktrETks0qBskuYKGxim8fO5sSX9pbVSSKokeGP4G4gVL8Q/V30QpOWjpacn93lEU+AvwDUARx85ihjvGDgK7a9NyIHXD8DBzQEj3mna7/uvMX/BLdANM9fPtMLIDMcqWPR4uAcc2zmqZeIae/1VGfiraBr4K2xagMMm5V3Kw/kfz6P3E70RPkC9tOJTl56y0qgM59/ZHx0ndcQVOMOnZiAeaDcQg41fc2Oa2EVAXSFcEwCnEm7BXJFYddKdfigd19dfx9DXhjb7s5eliUvEkFRL4BHqofd12i3ADbM2z1L7+7n06yW4+LmoBf6qC6qx6+ldiB4VrbEKDMsCV5JKkRq9DKmFwArG8PfKlmZDyYetg9AOa/ZEDo3Eq6WvIv7R+Cb3uQW6If9qPqryNDcj4bP9oqNVjWAlEvXyZqbiL7Lc2nYLKwavQMq+lOYflBANhEIhOnXqZJfztzGGYTD87eHo+VBPox/bc0FPjHh3BBxc9Je7I/al4FoBrq25hpoCM7ww2yBbnr/KwJ8bi/KMctRX6Q4Y8T2t+AyXd98FFi0yLYOEYRi0u6cdZfA20vvx3uj3TL8m29MPp+PI+0cgFWvPyKyt5U7oa3zC0XlS88tOMgwD34bPUG058MfPX2mVFCeXndRY6hPg+kz0fqK32gp7UzXO+Ft8ezEePvSwzv2NzTD0CPWAV6SX1rnn0XB9szX3+dMU+MvK0t2/ND+fK3Xs5MRlCfKfKzRl/PGBv8bXrEP7hmL+/vmImxoHAAjpE4IO4ztAUi1p3g9joLaY8deS778sy+Loh0e5DGAGCB8QDr9Yy/ZtbEsEIoEyO7bfs/0w+qPRJh3H2csZkUMiDVrsxGMEDEa8NwJ9FvUx6Tlt1cVfLiJ5Z7LG+8TFYuScydF7Xmgqaa0Um2ZuwtDXh6L345qjO9rmb3V+NfIu5UHopNpeUgJIRVyEIdTHtHno1ajUJ5/YL3IW4a26tzDpJ9srxVedV40fu/6I458eV9t+6stT+GvMX1DI1MsgXlt7DUc+OGL2cUz+ZTImfjexyXa5RI4bW27o7WHFm/TTJDyX9BwC3ALgVNEVztLQJv39eHyQVtN5CsMwGB41HBFeEWDuijSd+d8ZrJu0Dgq57ZaIvL7uOpJ3aJ6XADRWnbA1Dq4OmL5qOoa+OhT/z955hzdVvn/4TtKme+9Jy2iBMsree28QFBQcCIoIigNx68+F+nVvBUFFZagoArL33nsVCt2le7dp2iTn98dpOpM0adPSlt7XxVV6dnPynvO+7+d5Ps/kHeEcDu2Pi5vpBeJ1CX9ax5nCrXv52PtjCtILdO3aYMi4mcHZFWfJjmmcUVfhD4czY8MM8tVioKBdXRsZpB6FTaEQvRYC74WQBdjYQLv4j/mtUxrP93m+dNO0a2mc/fEs2bGN77M9u+IsXwR9we0zt43aPnRCKHaedhSkFaAuVrPg2gKm/zO9wjbWTtZEbo0kPSK9dNmNLTfY9fIu8lPySU2F5DzR8sTV2hVLme551TsxZ9Wc8Wdm7kShRnNQ2a5Ai4W1Bc/GPYuFle6vyq2SfkarVmJ6uZ2dKPplZpqeaq7l/K/n0RRrCH8kHIlUQmD/QIa8O6TJ+v83c+fRaDRkZmbi4uKCtCGZMd8BihXFXPnrCp0fNF08bKbu2P3qbq5vvM6Dux7E3su0h2u/F/vR/+X+TVbUbcjtVxsEYycv5ovgL+j8UGcm/zJZ7/blhb+sLLheMi6NiIBexpUXq4AgCKWZvI7+xk/CNQVy4nOIORBDh/s7IJFIEAShygSFlpvbb3L4w8N0fKAjbiG6J7QKCgBBQCYDubz24lPyhWQsLyQg0XQmM1McANw+cxvvLt56r7Mpom2/macy2fXiLmRymU5bzZDxIYSM15+1YArlhT+JRIKlreFnY058DhGbIggeEox7W/dqjz/g5QGlxeDLk5+Sz4lvTuCS5k8Sbe4a4c/TU/ysFQqx1l8LPc7D2ozAwMAy62MwXOOvvPBn7WRNqxFloryue1CXaMc95gh+bCzU5fs382Ym+97YR6sRrQgaFETQ3qDSdZfWXkJqKaX9VNNKVDSjm7SINArSCvDt5ouFtQVh94XV+FgqpQqpTGpyNlJDtHmsLc/EPIOg1i0inP/1PDue28Hsg7MJ7B9o9nMXpBbg2cETaxf9wV/62u/Qd4dWEX5T0zQUWYhRSm42xgt/imIFr+x+hZuZN1l3z9+ABRqNGBinDYJpqH0eub2cgW8MJKBPAPHH4jny0RF6P9ubbo93I3BAYJVx1eV1l4ncGsmg1wfVT507iWhh33J4S1oON83KNSlJ/KlP+NPeG1PrMSafTyZqdxTFBcVYOTRMi8Wnrj9FsUJ3oN9Dux+q56upORqNhpikGKLyLyGxs8Td3vSpfa3wF5cdh0bQIJVISzP+coutcWnlWkXgbmi0m9KOFzNf1GmdnHolleh90bSf1r7BW4MXFEC6/X4u5gq0LOxSt3X+ZFYgLfu8bGzASuWBZTGUj3UMfyScDvd3aPB2qbrw7e5L72d74xJsmm3SjsU7uL75Ok9df6qKVaiFtQUvZb9UIQP75s6bHP/8OD3m9yAtD5SW4gDI015/iYg7oRk1rBmyJkBjiBCpzNW/r3Jz502967Win7pIXcUi63aJgO5fkhVrjuj1wx8eZt+b+0o7TE4BTgx8dSAe7T1qftBmmjGAIAjExcU1yvZbnuQLyfw04Ceu/n21xsfY+vRWNjy0gfyUu2jmqhEgk8tQF6trZNUpt5M3WdEPGnb71Wb82dlBn8V9aD2mtcHtywt/N26ULY+Kqtn5Uy6l8GWrLznx9YmaHaARc2bFGf6e+TdRu8UP76/7/mJl/5UImqrfk+7zu/PE+SdwaqF/kFVQANZ5aYRveodDHxzUu52xnF91nrSfNiFXZJOeLlryLuu+7K67V9r269/Xn0f2P2JyremaoBWNbFS5RG6PpCDNcDRzWkQaW57cQuS22jlPaFQaDrx9ANt4UdFvqsKfIJSJdV5eoojXqkSPu3ZN/37aYMKWJfOX2oy/zMyqmYK6hL+C9AK974H6eD9oo7QLGnZwvFmpy/ev1EJK+CPhOvujO1/YyaH3D5n9nHcrp5ed5qf+P5GTYOIsvw4OvX+Idyzf0Zu9XR0NsS9XU2xcbPRaSQcNCmL4/4abXJ7BWJwCnZh9YDa+3X2JORijcxtT2m9MchZIxMlKN1vjhT8rCyu+OfkNm65vIrUwsfQ5Wd7uM+5IHNf/05+BdaewcbVhyFtDaD26Nfkp+Vz95ypZMVm0GNiC7vO6V9l+9BejWXh9ITrcymtMyuUU/lvwHwknEqqsk1nKmPbHNIZ/ONyoY1347QIRGyNYeXYl/2a/RY71pWqFP339lCNxR3h518usubimwvKJP07kVcWrDVb0A3HyvrwNe2Mk42YGGx7ZwPn15wGw0NjhVoNEXD9HPx4Nf5RXBrxCsVoUQ11dwcICkoL7MHHDoyYHHNc3Ugsp1s7WOhNVovZEsWXBlhq/j+qDg0sPsvfNveTnw4Wgx3l47xAuJF+ouxN69IGxFyBwGkR8DVs6424nPl+0gZFaLG0tsXGpfW31O0Hw0GBGfTrKZMHXp6voFmLjqvsZUdl2ecjbQ1gYsRAHXwcUClBaiBl/Xnb66xXfiX5O47uDzZidHc/vYOfinQa3yYrJ4ocuP3Dog4qDrMRE8adPSTKe9oVTG+HvwR0Pcu+f91ZZLghCkxoMNNOMuSnMKiT9ejrKnJrbxnSa2YnxP4yvYO/SzJ1n8JuDWXhtockR1ABFeUUknEwwy4ROM6ahFf4c3OSM/GgkHWZ0MLi9uYU/j/Ye9FrUy+RI4KZA28ltaTu5bWkdHUs7S+T2cp1R2M4tnPHq5KXX3QDEyXxBIkURGGoWu7nOD3cm/KMHKLayJzMTWo1sRftp7VEVqji9/HStj9/YsLS1pMXAFnozUwVBYOPcjex+dXetz6Ud2MoTo/l99O9E7TXcwPx6+vHAlgfoNKtTtccWNALbn9/OtQ1VFS57H3uevPIkdhPETIqmKvxlZIhCnVRaFhDYoeTRd/my/v20zzmt8OfoKLqJCEJZ3VMt5Wv8afk27FtWDV1VYbvo/dF82+FbovdG1+yPMQHbkvn9u0n4q0ucg5yZ9NMkQieGVlk3edVkJq2cdAeuqmkSdl8Yw/83vLQu37V/r/Fth2/1CkaG8GjvQdj0MJMn2zQqDct7Lmf9/etNPmdDRKVUkXQ+Sa9Nnk9XH/q9YHpddlNZP2M9u5bsMmmfS2svEb0vusKyxCzRc9kKB5PqukklUvwc/QBIyEnQWedv14u72PjoRpOusb4JGR/CG+o36DRTfz/AJdgFl2AXs2YwplxK4dS3p8iKztK5vu2ktjpdEnSx68VdHPrgEKvOr+KE7f+RZ3OldA6vMoasPgGOxR/jg8Mf8G/EvxWW10umYy1QKVXEH4vXW8Yo5mAMRz45gkppum1mfZKbkMvFXy+SFyMONKUamxoJf3KZnBWTVvDGoDewshA7VBJJWeBVcrK5rrjuyL2dy+2zt3WWawidGMqDux7Ep2vDdY67uPoil9ddpqAAVDLxwVin2X7l0ShBrcDOupA862usTF7Aa3teK11dXFBM0vkk8pLy6ud6GgC9nu7FPb/do3d9XnIe534+R1pEyTvRwQq3EDekFlIKCqCoJOPPy16/8HcnaBb+mmHyL5MZ8fEIg9s4+DhgYWNRQQwQhDKbAG2nwaUkk7Y2wp+DrwP+vSsWZj31wyk+9vyYjMi7uBBOM81UQ4uBLVicvJjwR8JrfIygwUF0e7xbc12wJkTyhWR+7Pkjl9cZmHFtpk4otfo0cv5LO2hLTzeP8CeVSRn9+ei7Uvjz7uzN9H+m49vdF4DJP09m1rZZOrcVBIHC7EIUGQqd60G8l0p7NwonTTdLVppXRy9CxrVBYyEnIwOsHK249497ufbPNfa9sa/Wx29MJGxPIPFUosFtJBIJ0XujiT0Ya3A7Yyi1+mztx7jvx1Uo5q4LKwcr2oxpozdzozz5qfkc+/QYN3dUddKQSCR4tPPAyVuMIm2qwp/W5tPdHbRlLMJKnAMNCX/ajL/gYPGnIbvPyhl/GpWGdlPbVcmqtnW3JS8pT+8knzmxKQkOLioyXMuwmdoTPCQYr04Na1KlMePfy59+L/QrjeqXyqRoijWoCk3/IofdF8a0tdNMFv6kFlJsXGywcmq4mUKmkHkrkx/Cf+D4F8fr/dzaQJkrf11h5KcjGfjGQJP23zxvMwfePVBhWVqBGH1hL63e7royfg6i8BefE19a56+88DfgtQGM+26cycetaxJOJvDL0F+4sfUGEqmkVND7stWXbHp8U5Xt1UVqsuOyzVq3sf209jx/+3najG2jdxtVocqoWrbT1k1j1KejyC0UxWgLwbZU4KmMVvjTZ/XZwkn07C5fGw4gOy6byG2RKDL196fvJDlxOazos0Jvu7zy1xV2Lt5ZrRPEnSawfyBLspdgPVbsBMk0tjUS/vTh6QnWuamc+uwAadd0FFpuQJxfdZ5lXZeReqWqL7xToBMth7UsDQJtiMw9Npf7d8xGEKBYKjY4R6s6LM+RfQUivoS8KGj3PEy4jsa2FUUWaRxQfMvaS2tLN02+kMwP4T9w4bc6zECsI/6e9Tf/LfjP7MdNvZLKv7P/5cZ/4kRN+vV08lPFCR+FApSWolruaWtcQEZ90VzjrxlaDNRTbKMcMrmMucfmVsg2yc0ti2rVFoGtbcZf2rU0bNxsSouLa7FxtcGlpQvK7LopgN1MMw5aT4tmmmlgZERmcO6Xc4TdF4ZXR9MnulxauTDsg2EEDjB/DZGGQkNtv9qMP1liHKvHH6TvC30JGhSkd/vy1nYREWXLk5LEiW7rhjtuaXDsfnU3glpg+AfD0ag1Bm1KcuJy+LzF5/R6phejPxutcxutWGRrutuuXlxdwfn2VaQx2RTld0NuZ8nYr8feVe4GxQXFXHj/Ajd/vClmNRu4T/MvzTeLbbH2XtoFuNJ9nqtR+wiCQG5iLvZe9gYzr23dbVkUvUjvNgXpBVgVKAEXk2vnNBa0EeJe5V5XbduKGYApKeI/z0rj4exsMeBBIoGgoLLl7u6iu0h1wp/UQsq4b6pOHHuGebI4aXGNsuVNpfyzQaEos0pr6tTV+/fg+wdJj0hn4o8Tdd6/4oJikNCkrczrA131b81ZU9UUZm3XHZzTGLFxsWHw24NpMUD3PEvu7VzWTlpL2H1h9F3c16znzo7J5uyKs1jaWTLmizEGt9XVfqf9Ma1K7dssRQ4SjQX2lqarC6UZf7kJpcJf+fdf61GGbfDvFIWZhSSfT6Ywq5DigmLij8Vj722Pg6+DztqJV/++yvr71zPtj2mE3VvzOpnlkcqk2Hvrt1qMPRzLz4N+ZtSno+j1tOFi4NpakrnHxAk8Zztb5HqSN6ur8aetDVdZ+Lv2zzW2LdrGIwce0fvdv5NYO1sz4qMR+Pbw1bm+58KedJjRwahArzuJRCpBbidHZi9GV9VG+FMUK4jNjsXG0qb0vnp6QmxuCjEr95I8ys2o+tZ3ihYDWjDk3SEGHUNUClW19bzvFHJ7OSjlCKjRyMQBip1lHdYjTDsGpxeBfWuwFyPtbGxArhLvcVpBmdDrHOTMkHeGENAvoO6up45IvpCs166zNvh29+XeP+8tnVf7efDPOAc5M+fInBKrz0oZf6mH4dRT0PEt8J9g9usxlmbhz8zIZI3LHk+lVCGzlBmVll950KWt7+fuTmmnobYZf5uf2Mzt07d5MevFCpM/YfeGma0D1UwzlZHJZLTSFqBpxETtiaIwq5CQCSHILGv2LFJkKvip/0+0GdeGEf8znAncTP0Qfyyeg+8exKuTV42EP3sve/q/2L8Orqxh0JDbr1b4s1DkcmvnLbrM6WJwe0dHsa6CSiUG18hkYrZgTg7ExEBoVcezajny8REu/n6R2YdmI7cz3p6psXN1/VU0xRrij8YTcyCGIe8Moe8LfXXaedp72xP+SLjBzK/8fHBJvIx1ehyKjEG1HkyoClX8FPoxrbOVqGWWZKd1wsPOskHb0dQF1g7WPHrwUQozC6utIWGuCX5d9eGq48A7B9j35j6euPCEweewVCatUgy+POsmryPxYjoMWFz6fGhqaDP+yot71tbQujVcvy5m/VUW/rRZzT4+ZZlzUBYMoU/4szIiOag+RD8Qn9dWVqINaX7+3SH81eX7N/ZALAknE3TevxtbbrB63Gom/zKZzg91rpPz3y2kXknll8G/MOTdITrrlpnK/nf2U5RbdNePIey97Rn0+iC96y2sLci7nUdRfvWZWqbiHOTMkowlqJVqg9vpa7+6hLjWjGLsmSLuuc/0TC5txl9CTgIBOqw+GyqtRrZiSfoSQKyptmrYKvou6cvsg7N1bu/Z0ZNez/TCpaWL2a4h9WoqMksZrq11Bym5hbgROjG01KZXH4IggCAKRvlFovDnYq9f3NJX4+/MGYiMhIFjxEnvxNxEitXFWMrE/lnwsGAmLJ+AayvjgqrqG1t3W4NCu1sbN7PY+dc1eUl5ZMdlY28j3ihZZatPQYDibJA7V3usN/e9yUdHPuKZXs/w2ejPAHF+N8e9Jc4vPk6rUeb7PtcFAX0DCOirW5gSBIEPXT4ksF8gD/z3QD1fWfUIGoGkc0nkCA6opWVBn3byOhT+fMfB8APgFAZ50ZC0ExfLIciLxQ53tjK7tE3be9sz8DXTMsYbCvMvzK+TQForB6sKzj89FvQoDRQoKIC2Ce9xf5sneLhzyUBHlQ+ZZ+HaJ+A7BqQWd0Qzarb6NDMajeZOX4JJnPzmJEvtlpJwsmrBYF0c/fQoX4d+TXFBcanwV74osLaWR02Fv/BHwhnw6oBGWUC0mcaLRqMhKSmp0bXfyhz99Cjr719fK399aydrkNBc468B0f7e9jxx4Ym70q7RGBpy+9VafQaMas9rytdoO7mtwe0lEioM3Fq0ECfLoeZ2n8ocJYXZhRRmFdbsAI2UJy8/ybxz89Coxe/Fsc+PVSnIrUUmlzHpp0kGazAWFIBj6k04ehzMUMLEwtqCoMFBpIUN5EavWRRKxUGDIAgUFxSXXndTR6PRIHgIBPSvPpo0PyWfiI0Rta5Xqs34S//kF34d8atR+wT2D6TbE90M1oEEiDsSV2r5oouOMzviP0WcXG+qGX+xJQkAlcU9rd3nlStV96lc30+LVvhLq+Q0pa3xpxVvj395nH8e+ofC7KrPubykPHa/spsbW25UWWdutLbOd0udv7p8/87cOpNnYp7Ruc61jSth08Nw8LsL1NU6RlWowrWNK7ZuZSKARqXh9PLTXP/vusnHu/b3Na6uv1qja8lNzGX3q7u5tetWjfZvTNi42PBs3LMMfnOw2Y4paASSLyaTdD4JGxcb7L3tOfzRYT7y/EhnjThd7VffRKlCARIkBsUifZQKf+Uy/soLfye+PsFHnh+RcinF5GPXF/Ze9oz9Zizt7mmndxvPME9GfzYa3266s8lqwubHN7Oi7wq96+087Jj+93SdtVDLU5BWwNuyt9m6aCuFavEFZW+l/17qqvEnCPDpp/DrrxB12QMrmRUCAgm5ZfOInmGedJ3bFQffxvtsVhepG3yNvyt/XeHHnj+ScDAVqyJv5CrP0nlYBAG2doG9hrN9tZRmb+aUZW86O4NabkOBk0+jLv8ikUhod087/PsYtvS/UxRmF7Ks2zKOv78HlVQcO0iQYGNh/ky1Umy8wHMAWLlC5jk48TgeHMNS7YJEEOfgy2f9NWbMWW+1MgVpBRQrihn46sDSoCmFAqxUnoQ6dCPAqWRc6zMSwl6FlP2QJVqm3ok5q2Z1xcw0NnsmxwBHgoYEGV1YWqVUIZFKyE3MLRX+yhcF1r5wMjNrdj3hj4TT/yXdmSknvzvJkU+O1OzAzTRjAEEQSEpKanTttzJD3hnC1LVTayWcS6QSnrz0JEPfGWrGK2umNlhYWeDV0Qsbl5p3Av+d8y8r+68041U1HOqy/SpzlRx8/yDJF2pW3Vyb0aOdDDamA1q+3kZISJntXU2FvyFvD2HRrUU4+tVhvYAGiFQmxcrBitkHZvOq4lVmH5xdqwFAQQHEdhiL27vPmK1WxIwNM1APHEKeayDpYvkcDr53kKV2S0m9XLVWRVOjKK+IyG2RRF2IMqr9xh6KZe2ktUTtqWFjKEEr/Mld7bBxM+65Gjw0mPHfjcctRH8kuKARWDNhDavHrda7TfcnutNh4WCgadb4u3EDDpSUhurUqeI6rfB36VLV/SrX99OinSSu/FlVztpMOJHApbWXRMukSggagUPvHyJiU0SVdeZGm614twh/dd1/1pel7tbGjWlrp9FyWHNAVG3x7ebLnCNzKkSwS2QStjy5hTPLzph8vLnH5/LYqcdqdC1FeUUcWnqIW7sbv/B37d9r/DTwp2rr15oTQRBY3mM5u1/eXbrMxtUGtxA31MVVs/90td+b22/ynu17XFx9scK22meaTQ2GIlqrz5T8FJydxWXlhT8bV5sGaSWYFpHGlfVXKEgvQG4vp8eTPbB2tubg0oM664nVBV0f60r/l2vv2iKRSGg/rT1enbxMEv5yckQdCUTbbe19O3lCqtfusyFz+c/LLO+5XG+7vLnzJu9avcv5Vefr+cpMI6BvAEPfH0qwVxeGn09kYPwmLLRxaRIJeA0Gt55lN88A2vsYkxVTuszRERAEslMKzVqzsi44uPQgK/utRJmj+zonrZxUb1lrJ787yeYnNuu9lsrILGUMfW8o7v3bopaJwp+tpW2dClZoVKApeR949IUh2ylyHYYEKVYacYyjFf40ag0r+69k18u76u566oD81HwurbtEZlQNRYlqOL/qPB95fETU7orjUe17skpZkOCHYMgOcBQDwO/EnHOz8HeXE3ZvGDO3zMTeS793eHn6v9SfBVcX4Nra1aDwl55u1HvGJM6tPMeJL0+Y96DNNNOE8OniQ7sp+iMRm2mcpN9Ir3WRdEEtiP8aubhd39w+fZs9r+zhxNcnxHpCJqIV/lTxSURujzQqgrR8xl+bNmUT4dHRJp/+rkWlVBFzIIac+BwkUgkW1hZ4tPMwuM+xL46xetxqvW2koAAEmQUO/k5mHZBp77c2YMo73JtOD3ZqsLUozEnq1VTWjFtD3OY4o7b36+XHxJUTS+vUCILAmRVndGZ5GUIrGrV9bRrT1k4zaV9DaFQahv9vOH2e62NwO12R9E0BlQq++krs/w8aBB0qJdC2L9EV4uOr2rxphb/KGX/awbOi0iuwsvB3z2/38FL2SzoDn+x97Jl/aT5jvjQu+r023G0Zf3VFYVYh1zdfr3V2bzM1QyKRMGv7LIb/b7jJ+8rkshoHqjkHO/PklScZ+GrjtBYrjyJDQXpEOuoi/XabEZsiuPDbBbOdUyqTMvzD4XR/osyyteucrjx66FGj7QvlDnKCBgVVyabdr/yMU63u4Wz+JpOva0LIBNKXpLPzwZ2l77/y74COD3Rk9oHZeHbw1H2AO0TExgj+nPYnGZFlVlZJZ5PY8+oevcLfn/f9ye5XdutcVxM6P9SZPs8a7lNorY8zbuq33LJ1t+XeP++ly6NdSoU/B+vqrT4Focy9pHy2/okTuuv8JZ5O5MvWX3JmhelBA/VBUV4R+cn6XRmcAp3oMKODQcv2hoBPVx/6vtAXiaPY6aggyCftBrUSQhaKImA1BDkHARCdFV26zNER5IosbD7/kP1v7zfjlZufgvQCMqMy683W3RBRu6I49/M5LO2MG8PJ7eUMeGUAjt1DsFS5MqL4K5YOW1q3F3nlQ1hrIWb7WXuCz0jkTuKEvpW6Yp0/qUxKdmw2BWmNq1ObdDaJ9TPWc3PHzTo5vk83Hzo/1Jns2Gx+GfILN7aKjiIKBVz1e5ntuR+TV1QyCZS8D9KOiFmWFneuduidbx3NNCrKT3YlJYk/dQl/RUWmD3rPrjzL8h7LSbms2+Zh6tqpej3Vm2mmKSIIgtH2fBqVRmc0Z02IPRTL3jf3Gh2t1EzdsqzbMtbe81etjjH558nMOTqnbiPImiB+vfwImx7GmeVniDtinDhRHq3wF/3nSX4f/TvF+dWLh+7lgp7btCnL+IuOrllAjSJDwbmfz9Vr1PmdJic+h58H/czxL48bvU/KpRTijsRRlKu73k5+Pthk30auNK9aU7k2csj4EKasmqK3nktTwinAiTHfjsGrn3G1Sx39HOkyuwsuweKHFnMghk1zN/Gh84f8ee+fRp9XKyKZmrmw7619rB6vP5tPJpfRdU5Xg5ax1zZcY//cX7HOTW1ywt+//4qZyQ4O8JiOhB8Hh7Ln2eXLZcuLikQxEKoKf/oy6HTV+NNXA1IikeAZ5lnj2semoBUq8/XPKzajg9zEXHa9tIu8JPGlmXQuiTUT1hi0jDz4/kE2Praxvi6xyXJw6UFOLz9dZXnw0GDcQ03LwhIEgbgjcWTH1qyAm8xShkc7D52Zu42NLrO7sDh5sd7aUwCH3j/EziU7zXre3ot6V2v7aIjAfoHM3DqToEFBFZbHaI6S5PIPGZoY3TsawE5uh6uNKxKJRKfVZ0MldGIoU9dMLRVNf+j6A1fXX+Xx048TPDRY5z5xR+Jq7BJSUwrSCri161YFgdIQS1ucod/Vo3jY6g+Is7QsC6zR9lWulnscZ2fD822/4+bCGOK2zuC778TlFtYWWDlY1cv7tiZ0md2FZ2Kewbe7bjtW91B3pq6ZSquRDbN2fGWUSnFcX6FmddoxiPweVMZ1MrXCX7oindySMY6jI6gsbcluGY5fDz9zXrLZGfXJKJ5PfF5vwOT1/67zz4P/kJtY953ue/+6l+finzPZfSs/H+RqFwZYLeTpXk/X0dWV4NQOAqeDvCwYxMZatJ60LKnzV97q89nYZ5m4fGLdXpOZ8erkxbR10+qsTI5nmCeTf5mMS0sXki8ko8wW50yzC/K56fMBPye+UBZIfHMlHJstZlrm3DB/dpSRNAt/ZqYxTaqqClX8NeMvLq3T4bljgBtbbrD/7f0klswhlhf+5PKyaFdT6/wV5RWReztXr62LaytXoy1Jm2nGFCQSCa6urg2u/V756wofunyoVwwvT+zhWN6zeY8zP9Y+wi56XzQH3j5gMHKwmfpBWSiQHNybs6oOqBp2uYE7Rl22X0sbS/q/1J8+z/cxuV5F+SjZzg+HM37ZeKycrAzvRFkGmFwOgYHg7w8WFuLEd2oNnIXyU/L5d/a/XP27ZjV3GiPWztaM/mK0SZNfoz4dxZKMJVg56r5H+fkC7Q6vJHm5eSeaa1sbuTFj721P93ndCe4TbFL7TYtIY+uirfj39mfST5Nwb+eOjbvxKp5CARbKfOJ/20fcUeMF/YzrGSSeStSbwWEos0NLXnIeqecSsCgqQKkURa+mgEoFa9eK/58zp8yiszLarL/ywl9MDGg04j5aIVyLVkirLPxpa/xZWUFxQTFX/7mqs4aVlqL8IuKPxdd5rVN9GYpNFXO9f8+vOs/hDw+XBqi4hbgxccVEWo7QP2kTezCWy2svNzsZ1AJBEDj8v8Nc/P2iznVFeaY9oIryiljZbyX7/m9fja9JkaEgLaJp1BeqjlGfjuLeP+81y7G2P7edL1t9WSUzoyC9gINLD+rMfDCl/RYIYifFza52QUnad0P5Grd5SXnsf2c/0fuja3Vsc+Me6k6HGR2wcRX7FxbWFti42eDT1ad0WWWejX2WBzY/YJbzp15J5YcuP1SxXa1M2PQwXs57mdajWuvdJuVyClue2kLiyUQ8NJ1wye+NfTXOEtqsP+290mb8afutSZfbcOFgIEcOWbBli/he9gzzZN7ZeXR+qLNRf2MzNWPf/+1jZd+V7E/9jyNtB3DV+bOylWEvw5QkuL0Ndg2qVmhwtHLE1Ua8qdqsPycn0FhaERk2iXbTwurqz6gX0iPSufDbBYN9xNpy7pdznPnxDJpiDbbuthz/8jinvj9V7X7R+6JZ2W8lyUdF2wvtPHqdEnAP9F8LdgGQHwdrrXCJWQJAeOQa0l5IZ2r7qfVwIXWHvbc9YfeF4dqqboNoW49uzZL0JaUBn2mFYtCHldQGe3mJo2LYyzBoE1z5ADaHQM61OzLn3Cz8mRmptPF8pDnxOVxed5mks0km7Xdx9UX2vbmPvFRxVOvtXbauMKuQ4NN/YZOTZHKdv15P9+K5+OdwDnLWub64oJis6CyzZTU104wWqVRKYGBgg2u/cYfFCUkLa4tqthQFitAJobi2qf0LLnx2OE9ceAKP9oat8ZqpexISJdwMHEKUcxciI2t+nIL0Ao5/eZyYA6ZH6jZ06rL9pl5NxbOjJyM/HmlyeygoKBtrtR4SQLfHuhkVARhQEhzeti3IZKLop11Wkzp/Ti2cmPHvDLrO7Wr6znVAUlIS777zDj3atSPEz48e7drx7jvvkJRkWl/EELZutvR6ulepJaQxWDlYGeyIK/IFEkMG4zvGvJMZlYW/7LhsNs7dyLUN18x6noZKTdrv0U+PcvKbkySdTSL8kXAWXFnA+O/GG72/QgFWBRncWL7fpEzeyasmszhpMTJ51Uh2jUrDJ76fsHn+ZoPH6PZ4N17KeokCjxZAWVZwYycvrywLb6iBEsG+lml02vkJCTvKlL+z62/R9vAKgu1SqrhS6RP+ylt9pl9P5497/jBYk+fi6ous6LOizt+B+q63qWKu92+/Jf2Ytm4aylwlyReTcfB1oMujXQxaNN/75728lP1Sgwvaa2w8df0pJv5YNZp/46Mbed/hfZNszqUyKSM/GUn7e9tXv7Ee/pj2Byv6rKjx/g2F2EOxXP7jMhqVRu82/r39CexnfD/FEDZuNlg7W1cRpFSFKva8uofr/12vso+u9nv2p7PsemlXlfkWBWIhYk/7mo0zX939KlPWTSFNEC3RyteOU2Qq2PfGPm7tati1HeccmcOwpcMoyivSG3AgkZrveVSYXUhhVmG1c18WVhbVZtilX0/n5NcnyYjMMNr1oLzwl50NCQni7zNmiD8PHoTV5UwQCus2rsYsRG6L5Pyq8wYDRrYs3MKRT47Uy/WkRaTV6HtflFdEXmIemRYZZDocRiEv17eRSMHGCwrTIO8WKG5Xe7xgZzGDVSv8lbd6bejuFJHbI7my/ore9d0e78bLeS8bzL6uLUf+d4R9b+5DIhPb/4mvTnDqu+qFv8LsQjKjMlHkqlBaJBNvsZ9rafU4/rN0BJ9RWLiKteesi32xlbgilZS9EyK3R3L5z8v6jnDXcu3fa6zou4L06+mlyzKUYrKGq5VnWd/UqR34jQfv4RDyFEjld2TOuWHNcjcBNBr9nbuGhmtrV17Jf4UBrwwwab+Brw9k0q6FqC2tcXSsGJlwc8dNrCMv4x15hPPfm/eFeXDpQb4I/sJoG4NmmjEWjUZDbGxsg2u/oz8fzYuZL5bamhnCr6cf0/+ZXsWapSY4+jni1dELC6vqBcdm6hbtIAvgkmnJ2RUoyiti26JtTTLrq67ab15yHt+2/5bNTxieyNeHNttPLhf/GUvnzrBkCSxaVLZMa49XE+HP0saS0ImhuLSs/jlS13z2ySe08Pfn0NKlPHPtGt8lJvLMtWscXLqUFv7+fPbpp3f0+qL2RukV3AoKpSS17kfwBP02jjWhsvCnKlRxdsVZ4o/Hm/U8DZEtT23hm/bfcCvilkntd+xXY3lk3yP49/av0XkLC6HA0Zuxm56g4wMdjd7PkHCvyFQQ2C+w2ve1RCJBIqkaSd/Y0QpdNjaGS8rYanKRK/PIUpRl1sYejcc+M56gLlU/O2Nq/Dn6OzL5l8mETtKf4Rs0KIgh7w7BLdS4Olc15W6z+jTX+1cileAW4sbfD/xdbXaLFrmd3KyT7HcjEokEO087ndbSAf0C6PxwZ6MymbVY2lrS57k+tBnTpsbX1HFmR3ot6tXoMzlPfH2Cv6ZXb9OvUWsQNLX/Wwe+OpDHTz9epU3Ye9nz2KnHGPha1bqJutrv1fVXOfb5sSr1sgqlYifFy7Fmz9D/bvzHhmsbSC4WoxhVqrL3hktLF+adm0fvZ3rX6Nh1xdZFW/nE55MKpS/+e/I/3nd4H02x7mde6tVULv9x2aR2o4+APgEsilpE+MPh1W4bvT+aqL36Bwkh40J4/vbzuI9wZ1P2u0R7fCMKf/Gb4NTTOrPCytcj1tp8BgaKNXwtLCA2K4Gzrq9wxV/MFiooAHWxmmOfHyNiU4Spf269cOq7U2ycu9FgwMjldZe5vrGqUF4XbHpsE7+O+NVggIAuRn48kkUxi0jNE7OjrWXlaoelnxIFv87vwaRYsNVta1qeR7s8ytuD36a1q5g1amEhzvEGXtzC3tf3mHRt9c3Bdw+yZcEWvevl9nK9jnLm4tEjjzJ9w/TSccKMf2dw/+b7q92v7aS2PJ/4PJLQENIcd/NZ5mAWbllYp9dK1G9w4gkozgW5EwzaiCxkLlotqnLg2v7/28/2Z7fX7TWZmX8f/ZePvT9GVVh3dlmqQhXxR+M59sWx0vabpRIz/txtypWw0JRcg9dg6P4lOLS6I3POzcKfmWlsnVRLW0u9tlb6cA91p8DGDSSSCjafAGH3hSF7aj4WxQpu/bDLaEsdQRA4/NFhgxYPgQMC6fN8H6wcTLveZpqpDkEQyMjIaJDtV6PWcPTTo8Qejq1+YzOizFGSe7uBh3jdBVzffouQY6uwy4irYI1mKo7+jjy05yH6v9zffBfXQKiz9itA3yV9CZ0QyuGPDvNdp+9MyjjXZvLY28OKvitYM2GNUftJJDBgAHh6li0LLZnPPnfO6NNXwRyTELXhs08+4f1XX+WAWs22wkJmAsOAmcD2wkIOqNW8/8orZhH/zv1yju/DvyfpnGlZhFsWbNE7uCkvbJgTrbWh1iXBpaULL2a+yLD3hpn3RA0QuZ0cC2sLsguyTWq/MrmsQjZn0rkk9ry+x2h7aoUCBJklvl28cPAx3sK3KK+Iq39fJfF01XqZdh52zPh3Bv2W9DN8jPwiIjZG4KwUv5sNPZLaWLTCnG01devbjwvm1Lg3iLduXTrPmN1pIGdHLiGojWg7Vv67UL7GX/mviNbq09oabN1t6fxQZ7w7l7MgqYRbiBsDXx1ocs0yU7nbMv5q+/4tLihm27PbyLiZgWdHsWZKj/k9WD1uNT/2+tHgvgVpBUTtjSI/5S5RWeuAnIQcMm5moFFXnYTqOrcrk3+ejLWztY49646uc7oy+M3BjT6Ts+/ivkz7Y1oVAa08Rz4+wjuW75B03nyOB5WRWkjx7eaLnUdVDzld7fee3+/hyctPVvj8BQGUUjGrwdupZhl/fo5inbAURULpc11b58/CygLvzt7YulXzAqlnHHwccGnlUuq8c/nPy2TcyKDbvG46M/8Bziw/w1/T/6r3MfTGRzey47kdetfL5DLsve1JUaewvfh1bvi8K96H44/C9a9AUzWz16tk7nrbtrLg0/btxfdcx46gluYT6fM+sR7fISBQWCgGcWx/djsXfzMugKO+GfR/g7h/o2FBZmHEQh7a/VC9XE/7ae0Jmx5mUma1FkEQyFGI3zMby3KDk/0T4PBMsKgmEqscT/Z4ktcHvU47j3aly5ycwCn1BjG7a2E5VA8M/3A4k3+erHe9Rq3h9pnbpF6pQb0MI7F2sq5QC9GjvQdOAcaXqMrPB5VU7MvYWup4DqqVoDHTOD5lP0T+AELZ8SQSsV1n2h3l2V1P8vmxz0vXDf9wOJN/mWyec9cTzkHOeIZ5GuWaVlPaT21Pu6ntKmR25qjEjD9P23LC3z8+sHd02e/qQoTi+u+3Ngt/dzFpEWlirZIaWGfGRxZil5lQavN57Itj7H5lNyqlCutAT+LbDafd1wuMHiwo0hXsWrKLC79d0LtN61GtGfnxSBz9HU2+3maaaWwU5RVxZf0VUi+nsvOFnZz76ZzB7Xe+uJODSw+a5dwatYYPXT/kv/n/meV4zdSc1Jt52GYlItWouHJFrINUE6QyKcFDgrH3sjfvBTZh7L3tGfHhCEInhqLMUaJSqMi7bbwvn1b4s7MTI65t3Ws+odGjh/jzypWaCQW/jfqNz1t8XuPz15akpCReevFFNimV9NKzTS9gk1LJS0uW1Nr2U12kpiivSO/EjD5GfTqKyasm61wn3Iik7aEfyb0UXatrq0z5jD9BENuqtbP1XZHJMvyD4cw9ObfWE7wpl1M4+O5Bo4ReQRBFKlmRAgoVJgkWikwFf0w1bClZ7TEyFKydtBanW+eApiMQGSOMa1QaPDwAiQSlEg59epJbu29x+zao5Tb4+Ig2fz8P+rl0H62QptFUrIeozfizamCxgHeb8Fdbbmy9wfHPjxO5NRKpTErnhzrjFOiEvY99teO9W7tusWroqgZXF6wxceKrE3zV+isybpjHTSdqTxTLui0jcnvDniiuD3y7+xJ2r+HaWG6hbrSf2r7Wk5N5SXnsWLxDr3V1UX4ROfHGpZdbO1lXqYuUW6BELRMnKn2cayj8OYiT4gk5CaV1/rTCH4iWd3nJDcv7uv9L/Xn00KOlfckTX50gJz6H8d/rtxbvOLMjU9dOxcal9lFiWltKY7LBRnw0gqFL9fts5yXlkXYtjbySwYlMYyu+r4fugkGbQVY1I+q++8R32tWrsLnE/ERbp3f4cLApEq0TVbI8imVZFBaKfdhHDz9q8FruJD5dfGg9Wn8tRAAbVxuDgr056fV0L6atnWZyIsaV9Ve49s81CtViZ8jWotwYs8NrELJA/H/KIbjwf2VZRybg5ASXBi2g+7LHTN63PgnoG2DwnmqKNSzrtowD7xww+7kFjcCF3y6QeatijSuVUkVWTBYqpeHPPf5YPBfXXCQ/qxh1ifBnJy8XpKEuErNyNwTA7a3muehuX8LUNNHmE8Tvx9WPsbGBfKub/HrtOzZfL3M7ajGwBS2H6a+33BAZ9MagOhfvpRZS+i3px9Q1U0ufF7mCmPHnZV8uettnNLj3Ff+fGwmbQpFEfFb5cHVOs/B3F3PkoyMs77G8gn2BsVx/+WfaHf6xNOPv8rrLnPv5HDK5DGtrUDh6IbgabwVh5WjFnKNz6PNsH5OvpZlmmiJJ55L4c9qfxB+P56HdDzH2m7EGtz//83kit5pnoC2VSen1dC9ajzHcMW6m7kn27sS50S+R6xZEQQFER9f8WIJGIDsuW2dkdzOGGfL2EJ668RROgcZH75XP+Jv+z3Qm/TSpxuf39BTtPgUBTlVfMqAK/n38aTm85R3Lav5x+XIGW1rqFf209AIGy+Ws+NFwpkd1dHusG09HPm1yXcbWo1vTYkALneuK85TIFTlYWZr3M9QKf0plmViQcinF5GzFu5k2Y9ow/9J82oyt3l6uuFgUkXyv7+PH1v8jN8F4Jd3B14HJv0ym2+Pdqqzb/tx2jn56tNpj2HnaMennSci6irUim4olZHUZf+oiNV+Hfs3h9/bi7AyyogL2LN7C5nmbSU0R25S3t+hEIreTlz6rrKzKAtbLi2nlrT7/mv4XX4V8Va1d3r639vGp36d1av2jLX/QLPwZR/up7bl/8/10n9+9dJkyR8nIT0Zy3/r7DO7r19OPMV+NwTtcf6ZnM4YJGhJEn+f74BzkXGVd2rU0/p3zr0n1p4oLilFkKnS2xfR04wKX4o/F88vQX4jc1vTFw9AJodz7570Ga1kaQ8qlFI5+cpTbZ3XX8lo9djU/dPnBqGMlX0imIL3iAywpKwuJxgIEKV5OxveDy1Mq/OXqFv6+bf8tayetrdGx64sxX43hgf8eMLiNbzdfOkzvYLKQo4sTX51g0+ObSuuGGaLdPe0MWuye/O4k37T7hvRIMXOzVPhz6Qx+43Tu4+kJ8+eL/1eX5Alohb8BA2DFDzZ42Irf3UJ5bOl7L6BvQBXxuDGRHZtN7KG6d1pKOp/E+vvXm1RrWsve1/ey7419KEuEvwoZfyELIHiW+P+ETXDpLVFwMECxupiItAgOxx4uXeboCILMgtzcxh2IaGFtwYiPRtDpoU5mP3bGzQz+efAfTnx9osLyIx8f4YugL0i9bDjL8OxPZ/n7gb9R5BSXBlfYWZYT/pJ2wIGJoEzVmZVbIyxswMpNrAUJEPULRK/GxgbkKtEVI60gzTznauL49fSjw3SxBIhKBQUSMePP17Fcxl/fX6HjG+L/7YLALhCsPalvmoU/M9OYbCk63N+BoUuHVikCbQyF/YaS1LI33t5ix372wdnMOTIHiURSGn2ryFVxY8sNihXVP6Rkchn+vf0NTtKlX09nzYQ1XP6j6RUX3fvGXtbds+5OX8Zdi0Qiwdvbu0G1X7cQNyb/MpmQ8SEEDw2utt7es/HPct/fhidJTGHUp6PoPq979Rs2U2cIAsSXlPny9BK/m7Wx+9zxwg4+D/ycnLgmUlSqhLpov4IgsLzHcvaU1DWoybG1E/r2Zkqy7NlT/HnihOHtdDH4/wYz5dcpd+wZ99PXX/NQoXHW3w8pFGxYvbqOr0g/GrUGRUbFgmIqFaR6hnFhxHO0GRls1vNZWZWJJVq7zzUT17BxzkaznqehoSpUse//9hGzL6bW7dfG1QbPME8sbSyr3VYrUOW5BhD+aBdsPYzPxNVmJHmGVRywCYLA6WWnubnjZrXHsLCyIPzhcOxai5FzTUX4qy7jLzcxFxs3G+R2cjw9QS23pddPjzP0+/vQCBLkctH2dsqqKczcOrP0+6C1H4KKdf7KC38Ofg64tHSpNkvWwtoCR39H8lPr7kO/2zL+avr+jTsSVxqEFDIupLQuTtzROD50+ZAzP56p9hguLV3oubAnbm3qtm5jU0brpqMr40yZq+TcynMknk5EEASSLyZXGzwUMj6ERbcWVREgsrJEAeG113SWEquAukjN7TO3yUtqWNlfpvJ126/5deSv9XKuFoNa8NSNp0onICvTblo7wmeHV7l/ldtvUX4R33f+vorluYPUi7Fniph0NQuZtGbTh/6OYk3ehNyE0tpx5YW/LnO60G5qOx173jlOLzvN0c/KAnq8OnlxcOlBjn1xrF7OP/zD4czYMMMsffegQUEMeG0AamdRwSsV/hTJsLUbXHhT536DB4s1/UAMVNOWIZBIwMcHAp1E23WFPLb0vaxRaSjKK6p6sAbA8h7L+TbsW4Pb7HltDz8N+KlG9pumkHIphUtrL7H+gfVc+1d3fXF9TPxxImO/GYtaJt5PO7mevmybeTD6DNgbzti6nn6dtt+0ZfyasmxWR0ewyksj8XANCszXI1+2/pLfRv1mcJu+i/vWqvasPuw87Zi2bhqdZlUUFQP7BdL72d7Vut/1XNiTe/+6l3y1dVnGX3nhT+4CAdNg9CkImGKei869CVmXyn4fcRiG7cHWFuQqcS6+vPC3c8lO3rV+t9HYqmvUGrY9u42rf1+t1/MqFNDm9usMuHKaJ3o+rnsjqQUMPwCt9ayvQ5qFPzMjrWFn6E7QclhLBrw8oEadidsOIcS3H4Wvr7ivVCYtjRa0Lnm+Ze44yepxq42KFCzMLqy2/pBGpSFyeySZUZkGt2uMHHjnANf+uWZ0Jk7atTTij8XX8VXdPUilUry9vRtU+7XztKPzQ51Lo0CTzicZjD6TWcp01m9opvGSmQny2EicUiMZVlLu69Ilw/sYInhIMD2f6mlU5Ghjoi7ab2FmIUX5RSizyzLir6y/YtRkpJZSq09ZIfvf2V9rOzKt8HfmjChENRY+++QTUlNSMDYnwwvILj8bVAOu/HWF87+abseoUWn42OtjNjy8ocLy8oKDuWv8QUW7T4D+L/en1zPV5Uc2bnISctj/1n5ubr9plvZbmFVoVP9QOzFVENyBSSsmVhtUowuNSlNhAlUikbAkfQlTVhk/KG9qApH279CX8ecc5Mzc43Pp81yf0olDpYsPxW7ik8HbW38pmvJ1/rRoa/xZWYmBSrO2zar2Gvu/2J+5x+eaVHfFVHRda1Ompu/fnUt2sn7G+irLvTp64dHegwurLpB8Idlcl9lMDfAO9+bFrBfpt6Qfu17axfedviftWs2yACIixPforVtQnZN34IBAXsp6ifBHwmt0roaCTxefal0HVIUqti7ayullp2t1LpmlDNfWrnot5Xs91YsR/xtRZc6nSvsVYPBbg2k7uW2F7RQKkCDBydr4mriV0db4i8+J15nxN+TtIfR7wXCN3Prm9LLTHP/ieOnvxQXFXPvnGkln9H+JUy6n8InPJxz+6LDebbQIgsCV9Vf0Zqt7dvCs1pZSy+GPDvORx0d6+0HBQ4MZ+s5QiuxFQU6msRXf15vbQuaZCvW+KjN/PowaBY89VvU9XV740/aVf+j6A8u6LTPquusb/77+BPQPMLhNhxkdGPXZqDq/lk4zO/Hk5SfJjskmem+0SfsG9A0geEgwcqktMrU9DtYlc0CJW2FLOCSJgavYtwTXLjqtXMsT5BwEQFZhFlmFWYBo9ekXsY+4d1cZZTd7p/Dr4Ydnx/rPoALRGjnsvjB8uvpUWB40OIhRn47CpaWLwf29OnrRfmp7CgqlpTX+Klh9evSDAX+Ca4nTiGCG+3D6adjes+x3W1+QO2NnVzHjTzvOcWnlQsvhjcfqU5Gu4Pjnx7m5s/pgTLOeVwFWKg/ci7vSyq3EPSg3Ek48ASnlbGYlkjsy51x31Q7vUtRqMxXdbMAUF0NqSdayj0/V9dqMP0lYO4Z31+DTRcdGldi5ZCdnfzzL4uTFejut7u3ceU35WoPKyjIXbwq6o6x0IQgC37T7BoA31G/cFXWA6hq1Wk10dDRBQUHIZKbVhKorNCpNBX/5dZPXIXeQM//C/CrbZsdmk5uYi1cnLyxtyzIeBAGOH4e4OJg6FUx5x0RsjODEVycY89UY3Nu61+pvaaZmxMeDX8QerNQFdOnyDGvWiBl/gmB0re4KhIwPIWR8iPkv9A5TF+3XxtWGBVcWVJjcP/zhYbJjsuk6t6tRx9AKf9bFuex7ax8DXhtA0KCgGl9TSIg4CMvOFr8HnTsbv+/ts7c59f0puj3WDd/uvjW+BlPR1vZrAxhrXJkMONXQSkrLkY+OkJuYS+cHTfiQEP36209rj1OLiufPzwen5OtYSYuxsDBcs6cmuLqK7V0r/HV7rKqVZFPDKcCJJy48gYWdBTdv3qx1+13WbRlyezlPnH/C4HbaiSlrw0G4etn7xl4OLj3Is7HP4uBbNglqYWWBhadxw6rfRv9G7pV86DyvyQhE1Vl9giiQSiwkYp0/ICUF5CVzUdqa4Rq1hiMfH8HayZruT4iuA5XFNEEoE/5qeh/rirvN6rOm79+AfgEU51fNppDby+n3Yj/+efAfMiIz8OrkpWPvMpb3WI5Hew8m/zLZ1Eu/6ylIL2DV0FV0m9eNHk/2qLJeZilD5iTe05DxIWhUmmqzqmMOxpB5M5Ow+8IqjEdiYsq2uXRJ9/yBlqYyzp+6Zmq128jkMk59d4rWo1vrtJA2lvTr6Vi7WJscAFq5/crt5Qx6Y1CV7Yyp4VodWqvPDEUGTiVz4TkN3IDkvr/uq2ANvWPxDgCGvT9M7z42LjY4Bzkb5ah14usTbHt6G+O+G0e3ed1IuZhS+swTNALqIrXR9R9t3Wxxb+eOoDacUltQLN7M0ow/39HgEAKd3tK7j50dLFyoe50u4a/t5LZGOX7dCcZ8MababdqMbWOUdbw5cAt147mE57DzMq3tCoKARqNhXO6PBFxwY1CXkuemqgCKc8TMIi0qBShTwE53KQMQxSYPWw9SC1KJzoom3DscR0dICwjHu0/QHSsVYQzGPGsPf3SY0z+cZu7xudi6Ge/0UR3FimKj3EaqIz8f1C46Mv7Kc/l9uPUTjL1UrZBrkKAHwXNg2e+KZChMxt6+E1YlGX9KtZK8ojwcrBzoPq97o3IBs3Gz4ZnYZ+qtTqcWne/J3EiI/AFcu1f4zNXFppdaqy3Nwt9dSmFWIT90+YFu87rR/6X+Ju2bkiIOuq2tQdfcnFb4U9k502+xcZFbfj38KMopwsZNfyepqQwEaotEImHGvzMozCpsFv3MSK4xhSfqkc8CPqPFwBZMWzcNgCHvDqkwiC7PxdUX2f3ybuYcnYN/b9FKJSUFvvuurB5YSIhpQoEiQ0Hc0ThyEnKahT8TURep+aHLD7S/rz2D3xxc4+MkJEB8u5GEBitp3VqcIM3OFpf7+1fdXhCEu/Y5WVftt/znOfbrscjt5UZ/zlrhz7GFC1PPzav1QEMigR49YNcuUdA3pT3n3c7jzLIz+PX0q1fh78flyxliaUl/tZpfgZlG7LPKxobJDxiuoVId45eNr1H9YoDx34+vsqygAHwiD2KtyATML/y5lEyCZTY9QwO9yOQyvDp6oVaribtoen2TynSb1w2MePxpJ6b8jq3nwHseDHx1oOEdKuEW4kbI+BBUyrLJwPTr6RRmFeLV2cuoDEKnFk5YZYqD9qYiEBmaGM5PzWf7s9vp/FBnWo1sVZrxl5ICWq1IKwRIZVJOfHUCOw+7UuGvstVncXGZXaAyJYud75+k3dR2+PfS8WKsxNHPjmLrZkvnh0wLCjAW7bU2FQtXY6jJ+3fEhyP0rms/rT0+3XwqCOv6sLC2QCpvOG4djYmCtAKUOUqjLPlaDGiht/5tec6vOs/ZH88SOinUoPA3fLjhflTk9kjURWpCJ4RWe87GjEQq4ZmYZ2rt2PLHtD8ozi/m6ZtP61wfvT+aox8fZcBrA0qfkymXU1AXq8lR51Q7qb87ZiunWi0n12owoPsc1dHOox3pS9JxsXZhwwZxWfmMv9PLTxPxbwTT1k1DbleLSW0zUrn2pX9vf4rziw2KcQ6+Dsw5Oseo43d5tAu5Cbl0fKAjO1/YybHPjzHv7Dy8OnqRHZvNF8Ff0P+V/gx7T7/QWP5YXR7tonf9ziU7Sb6QTP7blYS/fmuMulZ96LL6HPL2kFod827h2oZreHbwxLW1afUQ1cVq3rN+j44PdiTHvjdQru8VOFX8V54tHUBmC+MuGjxukHNQBeHPyQlyPFtT0B5ktde27igSiQS5nRxljtJswp8gCHwe+DktBrXgvr8qlttR5ijZ8PAGAgcG0ufZPnqP8UOXH7CwsaDAaw4+TOPB8S0ZEjy4bIMjs8CtJ4Q+DRoVSOVQmCTWiaspQTMq/n72eYj+HSd7BTKNLXKJDUWCgrSCNBysap7lfaeQyqR16uyhD4UCInzfwMXWgazCx3C2dgbvEXBPCsgqDY6k9S/DNQt/dymF2YVYOVnVyPItMVH86eOjO+tEK/wZWc4HgK5zuxqVRRG1NwqJVFKrrImGRsrlFE58dYLbp28TNj2Mvov7VrtP6MSmPRC621EVqgjoG4BbaFndkk4z9RckbjmiJUgoFegyM2HRojLhAcQJNlPo9GAnOj/c+a4VkmqDRCZBpVSx/639DHx1YI0jjhISINc9CL+hYGkpireXLsHVq6Lwl1aQxhObn2DLjS0UqYvQCBpC3ELo5tuNGWEzmBA6ocLxBEFgw8MbsPOyY+RHI83xpzZJBEFg14u7aDmiJa1GtCpd7tfTz6TjaCd+HV0t8O5srNGlYXr2FIW/U6fgcRPs4YOGBLE4ZbFZoxyN4d/Vq3mmsJBhwDvAccCQgeVxYF9REb/MnVur85rr89ZSUACxYaPxcjShY2MCzs7iT+0k2JFPjnDup3M8sv+Rer9n9YUiQ4FEKsHC3jxDkX5LjAs0UygAQcA69jq3T5numdtpVqcqtTxOfnuS418cZ1HUoioThbqY8MMENm6EfcubjkBkKOMv9XIqF3+/iH8f/yrCn7aL4V2uyc7aPgunwLJBe2Vb1PLji6yIFI58dAT3tu5GCX+HPziMaxvXOhf+FIqaZ+ff7VhYW5Ta3FfH7IOz6/hq7iyZUZn8O/tfRn02yigHHVNwD3VnUdQik/YRBAEE9Aae9nmuD20ntcXK0arC8ujokv0R+CFhPu9+9Ccn5p6glWurqgcBtj8j1phrrMKfqlDF3jf2EtA3oIptZmUcfGo/sdp1ble9dpEgvm8jt0fS+eHOpZ2wQ0sPcXH1RWS2Mtx2uhHYN5AzK85w8beLjPx0ZIXvW0TmZZJc/sFNXXOB0kJqgauNKHBoA8ezssrWp15J5dbOWxRmFTYY4S8vKQ9LW8vS73Orka1w9HM0+FmbgtxOzvAPhgPQ4f4OKDIU2HuVFAaXiMvM1Z/Njs0m7Woak1vez5pr4ViqnMvEoszzkLAJWs4GW9PGOg91fgjl2akcOu2LohEkBe1+ZTdOgU6lgUW6iNobxY7ndjB06dA6qQsHojC0bso6Os3qxLD3h5Edm01AX8MWpFo0Kg2hk0Lx6uiF8pr4LLayMrBDq8eA6i0ig5yDOJl4kuisaKCsnTbkzFx1sZrdr+zGv5c/7ae117td38V9jZpjNQWVQkWrka1wb181QN7CxoLr/13Hxt1w5q93V28kFhaoEsEzZzSvDBhd6hyBqgBi/6Q0qrHj6+I/cxNwDziEYn9efK7ZSdwpEuJIK0gj2CWY22dvc/H3i4TPDq9S47whosxVUpBWgL23vVmyMY0lN1/FDd93AChSPywulMrA2rj+bF3TLPzdpTi3cOaJc4btkPSh9ebXZ9OhffEolbD9ue2cX3We528/j8yy9hZsGx7agIOvA3OP125SsCGReCqR0z+I3v62HrYGX0qqQhUxB2IIHhpMVnQWRflFZp/gbObOY2FtwX3r79O5Tle2kW83X3y7lWXxnDwpin4+PuJk2tmzkJ5u2jVIZc0R1DVFKpPy8J6Hsfexr5XNQHycOMHi7y/e7xYtROHv9m04HHuY6X9NJyE3ocI+EekRtPdoz5XUK4wPGV/huyKRSIg/Fo+jv2ONr+luIPNWJkc+OkJxQXEF4U/QCBTlFyGTy4zK6tFO6FtJiynMUmPlaFXrLO2QEqfWpCTTJpUtbSzrtfOrJTsnB2/AG/gAmABsQrf4dxwYBXzwv//h7V3z95ogCKiVxtsjVaYwq5DNT2zGq7MXA14eAIj3ssDZD8s6csqtnNGkLlKjUWlEKzw3/fs1Zg68d4Bjnx5jYaQe/6g6orAQkEgofPZlpr1rHnv+9tPaY+dlV0Gsqo6mZglpKOMvaHAQSzKWlD7/tMJfampZ5l75Jl95YqFy+9AKf3I5tBndioXXFxotkM/aPsugu0ht0V6rSiVmJsobxvx1gyJ6XzTnfzlPn+f74Nmh4U8i3UnybucRsz+G418cZ/LPk+/otUTvj2bDwxsY+clI2k/VPcHq0c6jimirUomBbABRnl8S4fADKOBg7EG9wt+oz0Yhs2oYpRdqgjJXyZGPjtB9fvdqhb+85DwybmTg18uvxnMlvZ42XBM4dGJolXIpXeZ0Qa1Wc2vPLfKSxEjR3IRcMiIzqmQgpSvEQaSDzDwdEl2CwsiPRzLq01ENKuD0q5CvCOgTwKztYg3ZqN1R/PPgP0z/Z7rB+3rmxzMoMhUGaxZmRmWSHpGOXy8/bFxs8O3my6SVk0rXO7dwZurq6i0MteQl53Fm+RkC+om13yozba3oIJSRAa55XkgkIC+Og4v/A2U6xKwBl3CThT93W3f8HdyRCmXv5nM/n+PWrltM/HFijfvidcWJr08Q0CfAoPAHYpKESlF3BdUlMglTfpuCo78j257ZxtW/r/Ka8jWjngGWNpZM/3s6arWa+1+eiMJeQ6bwLRAM0atBIoMW08t2CHvJqGsKdha/N1rhz9ERXOMv4HDgALcfnlqljl1DoDi/mKMfH6XLnC4Ghb+6wNLWknt+v0fnOpmljNcKX6t23D9pxSSysmD5g+KYvkLwnIUt3FcAajNHCB59GNRK6L9W/D3gHgi4B5uSknjzrQ6y+ClHnKzFB3X69XSOfnIU/97+jUL4u7HlButnrGfaH9MIu9f8Lj36SMwWa6FJBCluNiXvyvxYUOWBQxuQ3tm02Yb1JG4CNKTOSl1x+7b4U5/wp623oVSKQpZXJy+U2Uq9tftyE3P578n/6PxwZ9pNaWfw3CM/HYncvmmNojvN6iR20CRUOyEfsSmCv+77i1Gfj2LPq3vw7+XPQ7sfKl135c8rTP5l8l3xPTQnEomEgICABv25Zd7K5JchvxA+O5zB/zfY4Lbnz4s/Bw0SOxE1Ef5AjHgrzi9uknXh6pK85Dxs3GxqHeyQcimFLttWoGg/FEb3xquk1E1KCngW5ZKQm0CIWwg/TfqJIOcg1Bo1l1Mv42ztTG//3jqP+eTlJ80ShNGQMHf7dW3lKtolVTrchd8vsOGhDUzfMJ22kwxP5EBZDarMfef4cOIWHtz1IC2H1a44tvb9KgimTyqnRaShKdbU60Srk6MjSSU2Ac+WLBsIDAYeArwQa/+tAvYB3p6ePPvcc7U6Z2FmIf9z+x/dn+zOuG/Gmby/lZMV8cfi0RSXRcfm5wsgCNja1k1AROUaZgNeHlAqOjZV/Hv5Ez47HAcfBwKca99+447Esff1vfR/ub/BIvRa8cjGhho9CwWNwK6XduHo71g62RrYP5DA/sbb7sQeiiVh1Q0slL3Iz7c3+RoaItXV+LNxKRPbtDX+cnPLJgrLjykEjUD69XQkUgluIW5V2of22WplJVrGurUxfjLaO7xug+VsbMR+lyCI19vUhb+avH9vn7nNuZ/P0e2J2tcyjT0US8yBGHo93avJjQ8BAvoG8Gz8szj6mT9gK2JjBMpcJR3v72hUUJJzC2ekFlLURfoDJgSNUOVY8fGgVkOB+0GuBSwG4LGg//FI+CN6j9N6dGvj/ogGio2rDU/ffFpviYbyHP3kKEc+OsJTkU/h2so0yz9j0RXMGTw0mBaDW5Celo6bu/gMHfTGIPq+0LdKoFhWoViA2NGydtf3xbEv2Bu9l8m+C4ARlHcJbogBp13ndsWllUvp7zauNsjt5RUceXRxduVZsqKzDAp/ERsj2P7Mdh458EgFG12NWkPa1TST++rKHCV7X99Lv5f66RT+tJR/V0sUsXD9a2j/Iow6CU41E04qv6MTTiZw8feLjPp0VIMT/hZGLKz2eRc8JJhFt0zLhjYVuZ281M1Jma0kcECgWJ/RBG1AIpGQZHOYIrtspJYlls2X3gWEisKfkQQ5BwEQlRUFlAj0EgkqQYrKwHP/TiJ3kLMoahEWNoa/Z8ocJed/PY9HOw+Ch+pvH+bE2GBfbaCw0vk8Z5PUtHFtU2axKZWBtOT9X5wjCruObcFrcM0vrCABNFUdbOxLhiOWBS0o12Wnzdg2LIpaZHIdyjuFWxs3+izug0f7+s20S8oV7dVsJR7IpCXjy6sfic/YKYlgUzbQuRNzzg3rSdwEkEobXqelMqmpsO/HSPxs0un+WBeT7RS0wp++gPzyGX/GTF5lxWRx478bBA0Jqvbc9ana1xdSmdToKHH/Xv4MfGMgHaZ3AKGiUBizP4bLf1ym34v9GkU0RkNCKpXi5tZw0ipOfHOC7Nhshr4zFJlcfHE4+Dpg42qDtYt1hW01Kg1ftPyCjjM7Mvz94QgCXCyxcO/cuSzKNi0NrqVdIz4nHiuZFSFuIXjZexm8ji0LtiCohWbhz0R2PL+Di6svsiRtCQeXHgTESFZTKC6GtHQJ1q6B+LUX23l5e7TRrUfzx7Q/GN16dAX/9QAnwzYhTU30g7ppvy4tXaoscw91F4UKI+oOARSVjMGcWrrTZW4XoywAq6O8lYtSadqk8s+DfsalpQtzjhhXe8QcTHrgAX5dupSZJbP7zwL3AyuAz4FswAmYDBRZWzPiqadqfU5BI9BxZsca1zKUSCTMvzgfK4eyDzsrKotu/32JrGAgYP7aJZUzmu4Gwu4LI+w+sU/nZlv79qtRaUg4mUBuouF6Y4WFYFGYh2X0bfKSfcpstYxEIpVw7udzuIW4VZtloY+4o3FErTqEVb/QJiP8Gcr4i9gUgVcnL5xbOANitqOdnTjZUVwsCmWe5bqtBWkFfNPuG8IfCWfST5P0Wn1aW0PGzQysHKyw8zRuQkJdrCY/OR87L7s6eR9KJOJnUFAg/tPa+DYF1q8Xn1VjxpQt0/f+VSlVXN90Ha/OXlWE2T7P9SF8drhZhLobW29waOkh2t3TrsnWo9aKfhqVhpyEnNJ2VFuOfnKU5IvJBksJlMc5yJmnIw3Xd/u67dfYutlWqHEWHQ3F0hyOB96HRqLCN/0B2jstrvZ81dmKNmSkMqnOfqQuQiaEYO9tX8Ue1Vgu/3GZ418eZ8xXY/TawQqCQNSeKCxtLAnoG4C6SI1MLkMqleLh6UFhtmivKbWQ6nSHyCoShT8nee2Ev+MJx/k34l86Og0ARlTo8xSkFZB4OhHPMM8G40wy6tNRFX5vPbo1C64tqFaIn7JqClJLw3OCrUa0YsLyCbiHVnxurZuyjqg9UQz/cDgpl1IY/sFwrJ2s9RylDOcWzsw7Ow/HAN3XdvWfq9i42LBXep1bnldoadEP3HqJE9IyW5DXrCaWRtDwa+pznG+RS3fF54ADIz8Ssze1cxgNCXNY65qD8kESppbwyYrO4uhnR2l3TztUErEROdmVdJT6rRYtIstTkADHHwO/8RDypN7j9gvsx/8N+j+6+YpBOY6OkOHXkQy/jniEm3SJ9YZUJjVqfK0uUrN14Va6PtbVZOFPXaxm87zNdJ3btYId6+5XdiNohFK73sokX0gmLSJN7/y1MlfJ3jf2YtU2GAjlRMAsui27xM4HdzK85XDIOAOqfHDvLWaLqQvh5HxoPa92wt+wXRV/T9gCVz7Ay/JjoGeFUkEAVg5WFcbEDR2frj53JDs1KTcZAAdJuQGN7ziQu0Cld+ed0IyahT8zo1Y3zGiI8nz0ERSuvohbwgW6zdVfBFgfpmT8GUNAnwBeVbyKRlW993RTJOlcEnaedsjt5cQeisW1tStuIbonwZwCnRjyljjp2PuZsoyenPgc+i3pR78l/Yye/GimDLVazY0bN2jTpg0y2Z3vpF758wrJF5IrdCQsrC2Yd3ZelW0Lswpx8HEojaiLjxdr/MnlEBpa1g7T0+HdA+/y+8XfAXC1ceXqgqt42ukXiUd8NKLBReo1BoKGBCF3kGPtYk3soVgKswoZ/sFwk2w/k5JA4eBJ/OBZ9CixCddOjCaL/QruDbtX574qjYrorGikEiktXSpmvhQXFHP9v+vYe9nTYmALnfs3NszZfjVqDTe336TFwBZVJiX9evqZVOdP2/Z8+wUTvqD6QYaiWMGllEv08OuhdxsLC/GfSiUe38GE8evA1wZi5VS/Hfe5jz3GO2+9VaG2nzfwask/LceBt4uLWVvL2n4Atu623PObbusVY6k8wFEUSsj0boe/f91MLFcWNnIScrjx3w0C+gY0eSs8c7XfwAGBvJT9UrVRlEolOGTEoNr1F9FjpoqBVCby+OnHSwXDtGtprBq2ikFvDqLb48ZlMIU/HI5N51BOf+6MdROx+tSX8afIVLB24lo6zuxYoV16ekKUGFSOm5tYx1aLnacdg98ajF8vvwrH1CX8rb9/PbmJuTwXb1ym8L4393Ho/UMsuLagyoSrubC1Fa+1qdRvBNEe7uefxf/36AHuJR+dvvabHZPNn/f+CSDal1WaAC6fAVobuj3WjXZT2pklsKYhcuiDQ6RdTWPiyomsGrYKmVzGgzsfNMuxx347lrzbedVvaAJBQ4KqvD9jYiDJ5W/ypUn42bak45llXFYYfk6fXXmWLQu2MGv7rEbZV1UpVeQl5WHrZlutwN1iQIsKGV+mUpBeQHpEerUiy6a5myjKK+LZuGfZ9sw2IrdFMvfUXOLT4tl9z27SrqaxJH0JNq5V22ZOsWgb42JVuyAdPwfxmZ6qjAcq1kKNPx7PmvFrmLhiIl0eNX1+qj6QSCVGZd9WtkrVhUd7D50ZKR0f6Ih3F28it0RyY8sNRn822qhrk8llBjPaNz22Ca+OXmx6+CBXAldgn/seSHuUZaGolVCYBHamfRelEilbUr9F5VFMlvJNwMGoTNc7gaARSL+Rjo2rDXYe+ufL1MVqLvx6AadAJ4MOErVhzYQ1ZMVkMf/ifJOzf7Jisjjx5QmcWjqhkYhRpk62Je3WJbzqDpZOkLIPnDsaPG4nr0508ioLBLGyEueTiorEGuTW1evP9Y5KqSLvtui0ZEicsnGz4cFdD5rkEKEl4UQC5346h7WzdQXh7+rfV8UEmg9073fk4yNc+PUCbXLb6HwPFKQVcPzz47R+SAqEopGJnUY7y5Lv5tWPIGYt3JcnCn9W7jBkOziaufatKhdyruHgmwXAleItzN+8iQEtBvBAxwdKg47kdnK9Dn7NQGqBmPHnZFEuscJ3tPivEndCM2r46WnNmJUbN+DqVYhvO5SI3g/zz2bToi01mrJJ5+pq/BUWQrGimAPvHuDi6osGjyu1kBolMOx5bQ8fe32MIrPphMX/PuZ31t2zjrRraawet5rLf17WuV1hdqFOcVQQBFb0XcG6e9Y1i361oLCwasr7neKhXQ8x7+w8ozqCtu62zD0+l8FvDgbgwgVxebt2IEiL+ObmM8R4fE9qejGWMks6enbE2dqZDEUGSw8uNXjskHEhtbYmvBvpOqcr478T6+vNPjCbhdcWmlzrL14cE+PvX1bHzdMTMu2OcqPoAEVF+gvLv3fgPdp81Ubn/VUXqflr+l+c+PqESdfT0DFX+006m8Tqcas5+P7BWh9Lm/FnTFaeRtAwbvU4ev7Yk603thrctvw71hR6LuxJ5wc7m7ZTLfH29uaDDz9kgpUVx/VscxyYILeqdW0/cxO5LZIdL+wAQGXvzK3u9+Ey0PCguaZUtknKiMxg87zNRO2NqpPz3WkEQWD1+NUc/ewoYJ72K5FIjHpnKpWQ7+yL/QMT8O/lX6NzOQU4lU6yFuUX4eDrYFK2hp2nHT4d3RFkFk1GHNKX8SezlDF51WS6zu1aYXn5DD9d44lBbwyi9SjR7k9fjT9ra+j8cGeTMi9bDGxBr0W96rTmaVPM4M3KKvv/oUMV1+lqv3aedvh088HBz4Gi/KIK6yK3RZJ5K9Ms1+Uc5Ixvd98mG6QWcyCGiI0RSGVSXENc8exkvkAQzzBPkye1NSoNOxbvYPP8zTrXT/hhQhWHi5gYSHRdA8CcrrORSWRst36UXsv6U6jS/ex3DHAkaHBQo72vSeeS+CLoC059f6rOz9Vjfg9eSH3BoK2ZRCJh3PfjmLl1JhbWFjj4OuDS0gUbFxsKCwtpf197eizsoVP0A8hViRl/Lja1y/jzcxSFvxSFaEejVouBbADenb0Z/8N4AvoZdi6pLxQZCtZNWce5X86ZvK+qUEV2bDYqpek14jrM6MCQt4Zw/6b7WZy82KQ2oMxV6nU9mPzzZAa+MZD8IvFlbWtpC4UpkHNdFP12DoAdfU2+XgBbCzEKMVshnjs/NZ/4Y/EUZjWcuRUQP59v2n7Drpd2GdxOIpGwcc5Gziw/U2fX4tzSGY92HkgkEtIi0ljWbRknvztp1L4BfQN4LuE5Ws8os0R2trMFTTGodHQ8LO3h3lzo8qFJ1yiRgLNlHh5RJ4g6nGjSvvVF6pVUvgj+guNf6htlikgkEloOa2lSLW4t7m3dmbpmKl3mVAxIWHBlQWn9T110m9eNaX9M0zsH5BTgxDMxz+B3n2gJrJKWCH/ykvncNk9Cj2/BouR3iRR8Rposzlch9k9I2l32e4vpMDVFPDaQoDnD96e/Z2/UXkCsR/pF0Bcc/fRo7c5bT5z45gSrx6+mIL1+IyvTCkWRxNmiYQbsNs7eVDM1ZtMm8ad7Sydu33bi998hMBD6GvmeT0sTO2gWFmXRnpUpb/Upk8s48M4BWo5oSccHdE+YRW6PxMbVBr8e1WdS2LjZ4BbqZrC+QGNCEAT6vdgPa2drPDt4Mu77cQQNCtK57a6XdnFp9SUWXl+IvZc9OfE5/DryV0LGh9BhRgfsPO0QBIGzK8+SE59TKgQ10/iQWkh1Wvkknkrk/Krz9Hyqp96IJa3w16kTRGZE8tPVL7DwdyDw7Dx+GfMTcjnsvLmTkb+N5LtT3/Fs72dp4ay/AyEIAoJaMFm4akakpjYnSUngkngJt4JkivIGILeX4+gI1wNeJ9V+N+/t/Zi3Rj2vc98QN9Ga9Xr69SrrrJ2tuffPe/Hu3HAEloaEo78joz4bRdDgoCrr8lPz2bl4Jy1HtKTTrOqtsbTCX+Qvh4lITBHrr+qxrPrl3C/sjRY72Dtv7WRMmzE6twNxsjs/33Th707x7PPPg0TCwCVL6C+15NHiwtLafsul1hwRinnz1aW1ru2nJeZADBd+u0DvZ3rXyt//6j9XObPsDB2mdyAvyRGwr7No18oZTd6dvZm1fRYeYfVbn6C+KMotInpfNPbe5rW5jN4fTV5SnsEsPqUSimxdcB7qQkk5E5NR5ihJvZKKSysXfLv58tjJx0zaX6PWIOQVICuSUYgNGg00gkoBBtGX8Se3l+sMOCgv/FWn9xuq8ddzQU+TrrP16NZ1Xj9M+xk0FVEXqFCL6+BBmDzZ8PbWztY8furxKsuL8ov4fczvdLi/A1NXTzXLtRUXFKNRaxqVFZWxzNwyk2JFMQATl08023FVShWCWjA5M0dqISX5fDIF6QWldpHVERUt4GzZE7nPdR7qcj/ZwVZst19P3O0cIjMi6eBZ9XndakQrWo1oZdK1NSTsPO3o/VxvfHtUbzmuyFTw28jfaDW6FUPfGVrjc1YX+KINpAAxsGLQG4NKsw4Gvj7QYMa9QpMDgJttLYW/koy/5IIEtEdSKMSMb0d/R6Oz5usDZY6SiI0RuLYx/W8++ulR9ry6h8dOPYZvN93fgX8f/Ze4w3EsuLZA573TqDUm27+u7LsSdZGahRELq6zTlu3Ivyy+SG0sbOHmj3D+VRh9GoJnQWEyCBpRYDABOwsHcoozyCsWXxTX/rnG5nmbmbV9Fq1GNpx2LLOU0f+V/nrviRaphZSZW2fqtU2tDSqlCgsrC8Z+NbZ0mdxeTkFaAapC44RimaUMB18HcrPLMrZdHKwhdT/sHgo9voc2lVyipMbNR9xIv0F0VjRdfLrgbuuOszQXx8tbido1hN731KyEQl1i625Ln8V9COhTfcCAoBFQZChMzlqzdbOlw4wOCIKARqUpnROTSCUGjxXYz3Dtb6mFWO5JEyn+XkyljD/PAeK/Cn+EAMVZon1kTTn+OLh2Be9hFRZra/xJFOIkf2pBqng9Hnb0eb6PSbXM7yQZkRlE7Y6qd5vwDKWY8edqVS7jb/9EsPGFnt/X67XoopEPM5upjoQEWLtW/JmVJQ7WEAQWPJzHhPFixsjffxt/vKQk8aeXl/5JCu3EWGGh6Lv8+OnHDdpubZ63mY1zNhp1/j7P9mH2gdkm12RpqEgkEno/05vwR8KxtLWk+7zuemtUeHbwJHhYcGlWn7bAqrWzNSP+N4K+i/sikUg4/8t5Dn94GGWOkV6rzTQosuOyiTsSR3FBcZV1mbcyOfHVCRJPlkVd3dhyg6OfHUWZq6xS3y8iLQIAe2UoEiRkiAGbDG85nKHBQylSF/F/+/9P77XEHY3jA8cPOL38tNn+vqZO7KFYfh35KzEHY0qXXfv3Ghd+u2DScfLywDkpAtWeQ6V1IqKzoki13w2ChF4O+ifMtMLfjYwbOte3n9reKBuauxF7b3t6P9Nbp12OoBY4v+o88cfijTqWdnI67VQMV/+5qrcDml+Uz5JdS0p/P5loOOKz/DvWFM7+dJbvOn1H+o1003Y0A88+9xwx8fFYDHqV5+3b8airL1+0a0dOx1cZPCyeB2aZR/QDMdL+zPIztY70G/DyABZFL2LnCztJ/2A5sqKCCjUWtShVSpafXs7ktZNRaUyP7oaqGULWzta0GtnKKEupxoiVoxUv577MuO/GmfW4+97Yx5YFWwxuU140qinR+6JZ0WcFe17dU6P9066lsTzkE7yijgFNIzNMX8afPht/j3Kati7h7/yv5/k69GuSzifptfqszT2sS+xK5mwK6jfYuE4pL/xdv17m/qIPQdDtSiCRSMxq5ZcWkcZSu6Uc+uBQ9Rs3UuoiO/XmjpsstVtqct8UYOraqTx+6vEqop8iQ8F/T/5HxKaI0mX5+ZCWKiE08R2uPnGLVq6t8POVYKcUrcp0Bag1BVyCXRj1ySi9wbzlsXK0Ij81v8ZBzWdXniVqj/HuALGHY03OYlvETUafySHMtXbtVpvxl5CbUOqG0VDff85Bzryuep1h7w+rfuNK+Pfxp/ezvfVmUAJYOVlh62GrU/RLPJXIu/J32f/2fpPO23FWRzrONOxMUVAsvpjs5Lbg3g/CXgHbAAh9Gjq/Z7LoB2AvFzP+covEF4V/H39GfDyiRqJpXWJpa8mw94bR7p521W7benRrPMPMm7mTfCGZL1t+WWGOAMQ6rs/EPEOfZ/sYdZyC9ALSr6eTkS4Kf1KNNbY2UtHSs8UD4NS+6k55t+DGD6C4bfDY96+/n5G/jeRonJjdZRfoRkTvh3EaGG7UtdU3TgFOjPxopFF1+/6e9TcfeXxkciauoBG4tesWH7l/xNW/rwJwbcM14o7E6e3rGENhdiGZtzIpzFMhIKCSaNumnSjw6eLog/CXq5ilW1P6/g4d/6/s9+JciP0TB9U5AGRFYnJBZqHozGDtbM3Ij0fSZmybmp+zHhn92WheVbxqNkt5Y+kvWcKAK6eZ6F0u8KIgXgyoaAA0Z/yZmTtRqFEX+fmwZg1s3izaKPz5J7RtK2brtQ3IZ0P/TwiZ2R0YR0qK8cdNLNEb9Nl8QtlAXKUSz11dfZqxX49Fo7476/vpQqPSgEQUTcvTc0HPCpHNMksZC64sqPLCuffPe7FysGqw/uoNEalUSsuWLRtE+73y5xV2PL+Dh/c9XGXA2Hp0axZFL8IpoMym4NKaS1z47QLdHutGVJQ4OWNjA61bw99HrgHgTltArPPn7S1OvLw/7H16/diLX8//yntD38PXoWoUl6OfI369/Jr9vE0gKyaL2EOxaIrLnml7X9tLYXahUVliWgoLIabjeDovGoyFlfiqXnl2JQDuucOwyAvSu28bN7FjlpSXRI4yB0erquJBUV4RRXlFZs+4uROYq/3mJuYit5frjbC187Tj5byXjX62ajP+xq1+ADdn/YMMO7kd6+9bz9v732Z31G5OJ55GpVFhIdXdRSufVW8KmmINxQXFFOUWVb9xHeDt7U3X7q8ht36NF16AgQPhqacgOrqijVxt6bmwJ51mdaq2rk51aGtGhT8aTvyPyajlthWEhvcOvMcj4Y/gaefJK3teIa0gjV23djG6tXH1WMpTOaMJKI0slVne+bqzdYFEIkFmKUMQBLO9fwe/NbjaAb1SCUHn/iXuTBzCAwtqFBEaMiGEER+PIPzhcI59fgy5g5yuc7pWv2MJDr4OdH+yO3+cFSdB8/PLxKLGiCDoz/j7Pvx7nAKcmLl1ZoXl1Vl9Si2kSC2kqApVpe2jstWnPDOJnwZupc/zfWg7qa1R11qQXsDmxzcTPCyYHk/qr6daG3S158ZOTk7F3w8ehGnT9L9//539L1f+vEKXuV0IHhJM28ni/bG0tTRr/S4HHwc6zerUJF0MNCoNN3fcxC3UDddWrmTFZHFw6UHajG1j9PddH7ZutnS4v0ONsuJt3coauSAIpcJFVkwWp747hYOfA6ETRGEvNlbczs0NHBwkpf+3jwsl2+5kaYBiZQRBYM+re3AKdKL7E91NvsbGhFQm5ZnoZ2q0r6AR+G/+f7Qa2cqoiW9BI/BT/58AMQjQ0s7SqPdvYaEEC40D9rUcDmoz/hJzE+lno6GoSFpB+FvWbRmurV2Ztm5a7U5kJiQSCRKZ6X2E4CHBBA8xfD8M1e5zb+uOzEpmstVt/xf761yek5DDt2Hf0mtRLxT25YQ/r0HiPy2FqXDmWQiaCb76nUcq41Ai/OWXZPx5dfTCq6OXoV0aBRq1psp8XG3IuJmBc7AzWdFZtarrefH3i2xbtI1ea4YiVdsgE2xEId2qK/T7XfdOKQfg5BNg7QEB+hMy/Bz9OH37NAm5oh2vk4ecXPcglA000MoUWg5viZWjFWqlunRuxRh+GfILt8/cxqO9B1JLKYIgsPWprcisZDx14ym9+yWcSGDNxDUMenMQPeZX7W9G/BvBhoc30PLVGWikAQiIc7p2lnZiTcaD00SrzxbTy3byHAwyG1AXgqyGN8VvbMXflWlw6D7kbRcjk4VjqXYGIFNhHkv2uwVpoRtOBW4EO5dbOEa3XfCdmHO+87PcTQxTi7PWFW++Cf/+WyK8eYoTkFoLwBEjoPv87rQZKXZIsrLK/NWrQ5vxZ4zwB+IEi0qpIvVKKkV5uicbQ8aHGD2AybiZwYF3D3D7rOFolcbCmR/P8GOvH0m9KqZSn/r+FEvtl5J0Lsmo/RWZCr4I/qKCJ7i9l32z6GciEokER0fHBtF+g4cGM+yDYXh1qtphtnK0wrmFc4WJyhEfjWD2odnI7eWlbbxDB5DJ4Fq6KPz5WpYJf1p6+vXk/WHvc+rxUzpFPwCnQCce2vUQYfeGmemvu7PUh0Vwp5mdeCXvFVoMKuvQj/12LDO3zDQpKkyhAI2lFQ5BYtRVjjKHb05+A0Bg6mMGAzYcrRzxshO/PzfSq2b95afk84HzB+x5vWbZKg0Nc7XfnUt28qn/p+Sn6PZnk0glyO3kRp1HECrW+KtugDGwxUB2PLgDRytHFCoFl1N013qFmgt/3R7vxtORT+PT1cALvI7JzhZ/OpZo0U5OFZebA4lUgo2rTY1tdivT+cHOCCPEugfl+zdrL69lxvoZRKRHMCNsBgC/XfitRufQiiVFRWK/DeBDlw9ZN2Vdja+7IZMRmcGNrTcozCo06/s3aHBQBTszXSiVoLa0wtLZrsY2MBKJhL7P98XW3ZbDHx7mzDLT6sDYuNgw7ptxqFuK2dmNXSBSKsuCk8tn/AkaAa+OXjqj/quz+ux4f0eevPwk/r389Wb8yVUK0q6mmRTMYGFtwdV/rpJyyYSoRxOpfL1NgbwSRzHtM/BgSRlcfe3Xwc8B5yBnTnx5gsjtkXV2XVaOVkz5dQph9zWNfmp58lPyWT1uNcc+P1a67MyyM8QejK31sQP6BjB19dQa9wdUhSo2zt3ItkXbSpf5dPHhhdQXKtg1nrqeQLLTZgKCytqouzvYFYrPvoh03cKfRCLhzI9nOL/qfI2u704TtSeKNRPWEH/cOIeI2jBrxywGvj7QqG0lUgmPnXyMYe8Pw9LO0uj3r1acq5zRbSo+DuL3zUJqgcw+s8KxQbTts3JuGAqDIkPBrV239NbMq0vk9nJeLXiVQW8Mqn5jI/Hr4YejnyMKtfhisrfSoeLmXIPo3yFHd7vUh6OVKPwp1PX/WZlCZlQmq4av4sLv1Wc6r+y3kq9af2XW87eb0o5HDz2q0/785o6bHPvimI69quLTzYf+L/enVetOjD1bwMRraVTbhfYeAYM2g5dhO2GtOJ+QIwp/jo6AIJCdXjNHk7om5mAMq8etJvZQ9e/FLo92Yfz340220HVv7067e9ox9/hc2k0Rs0XvW38fY74aY/DZaeNmg2trV6ydddeJ8GjvQd8X+mLh7V5a3w9K6m8COIWBdaX5wNZzoddykJteq1Av1t7Qbx2Slg9jbw+WKmcAsgqzSjf5874/2blkp/nOWYdE7Ykiaq/xWfDmQtvnrxwAqYs7MefcnPFnZrRe6XcSQYDIkjHWgskJeElSiHPtzC+/SvH0hKET7bG4ZxyCAJYboLgYMjIqDsL1cbtEbzMk/FlaisVgBUGcDLi0+gxbF25l1o5ZVfz6ja0PoCUrOou9r+/Fxs0Gny53bvLSXChzlWTHZZdmJjgHOxM8JLjKw2DP63vIjMxk4sqJFSxfUi+nkhOXg1pZ8XuXejWV3MRcWg4zrWj73YparebKlSu0b9/eYI2D+sA73FunzaCW/JR80q+nl/ps23vbl2ZtRZW849qW6OjaSNpAu1CyqCj8AbzU/yVzXnqD5dgXx7D3tufwh4fpOLMjfZ+vWfFyY5FIJUgoa8M1iepTKMA6JwVLlQNgw1fHvyKzMBM/eVt8MqdWm6kd4hZCcn4yNzJu0M23Ys0MO087wmeH1yrasCFhrvbbamQr5A7yUjtlXSSdS0LQCNVOlqlUZRPhSceiUbWw0xlZXz5aXiqR8uuUX/F39Ke9hw6blhJqavXZENBmjTg7iz+1wl/lbJLakHo1FY1KY9ZoY63IKi+XRJiUl8SllEsIgsCsTrP4+uTX/HPtH/KK8rCXm5ZJW34yTaEQ6yyETgxtcDZJ5uLKX1fY/fJu5p6Yi3dXb7O+fwVBAAG9op5SCXFho5m4oNanAuCxk4/V2Frdzk4UvRt7LTjtYFciqSiOS6QSpq7RbUtdkxp/2glibXu06xDM/G9eMOla5XZyXi96vU7rFjdFq0/tM3rAANizB27dEl1gvLx0v3+HvTeMoe8OJfNmJo7+Za4DW5/eys0dN3nsxGMmT77dbcjt5Uz4cQLuoWIJCKdAJ55Per5CHyX2UCyeHTz1TiyCGDSbcSOD1qNbU6wo5tzP53Bp6VJtkIQhZFYyEk8lVukvVXYI2XhrNSfbLKHYehLvsAEQhT/7wuqtPuccmVPhu9OYyEnI4ebOm3R/sixbsagI1q0DBwcID4cWLSidrI89HEvckTh6L+pt0ryIRCoxyk60PL7dffHtLgZ8GtN/vpV5iw3WzyPxD8bG5lOTzlUZuUxO5ouZOFk5sWiRhEwqCn+zts+q1fHNSeLpRH4b+Rvjvh9H93n6s07z8iAzEwLKlRjLic9hy4IthE4Opcts3RnOhz86jEtLF9pP1d3fr0lg0tV/rnJ2xVlGfzEa11Zl/UdHP0ce3PkgAMr3xBeTo40tnFoEBbEw8B9xQ88BcK/pkXiO1iXCnyYXjQbSI1L558F/6D6/u0luCHWNMkdJ4qlEo+r8Bg4IrHXJAFM4u+Isl/+4TI/5Pap9BgT2CySwXyCRkWpycnIIdLYHJHDuJbFGY5f/Vd3J1k/8Vw1a4S8+VwxacHQU6LrlXRJj28DcGSb/XXVN3u08ovZG0fXxuvuejf9ufIXfJRIJfj2r/yxdW7ny6KFH9a7XPovXrgWpJofxtu8Q3k2JTCoDryEwYkjtLnxLOGSdh5HHwb3ENU5dCOs9xazent+JyyxsoMV9gDj+tEwT6wdqrT4BEk8m6k3iaWhse2YbaqXuWqd1yT7hbbK8rVHKHgXcQZkO8f+Cex9wqmgvfCc0o+aMvyZIVqZAwKl/cE28RMa/B9n02CYGdM7m99/h+elx7H9jNxq1BolE7HwDpKYad2yt8GdokF5+4F9YCIH9A+n/Sv8K9oRa/lvwHx97fUxhtnEzmH49/Zh3dh4d7zfsX95Y6PNsH55PfL70s2k9qjUzt86sMqmcejmVuKNxVSwffHv4Mn7ZeHosqJg+/vfMv/nnwX/q9uKbGA1DtK8+I2zr01v5acBPKHOUaFQasmKySq1ytTVYnJ3FY11LEzP+QlyrZvwZy6W1l9jwyAYETc09zO8kGpWG7c9u58SXJ7C0sSQ3Mdfo4tk14cyKMySeTqyyvFhRTHac8YMpRU4xHQ58x+0VW8lR5vDJ0U8AmNP6DSTIqq2v08ZVtPvUN6kycflEOj9UNeKwsWKO9tv5oc5VOveVWTdlHZse31TtsYrK9Y3XT/lNZ5ScWqOm67KuPLPtmVI7jYmhE+nq0xVLmf6s7ZoKf8WKYs7+ZFotGHMiCGWTx3WZ8bf92e2s6LPCfAekal24YnUxaQVpAHjbe9PTrydtXNtQUFzA31dNKJxcgoWFGDQFZWLBlFVTGPS6+aK9GxIh40OYsHxCaa1Rc71/r2++zlK7pVz564rebcxR4688jv6ONbLL27JwC27H/gMav0BU3ubT2CBWJyeYMQNmzRInwitTrCjm5LcnidweqTfjz1q/1mGQuhT9oGlm/Gn7l35+0LFkCHbunPhTX/uVSCS4tnatMHaxcrRCbidH7lA7K+byHHz/IP/O/tdsx2soWDla0XVO19JAP4lEgr2XfWmwUNyROH4a8BM7Fu8weJwTX50gYlMEylwlRblF7HhuB0c/OVqra5NIJMw5MocHd4higkqp4uKai1UcE+IVoutEiEOZ+FFe+NOX8QdU+e40Jjo/2JnXCl+jzZiymkgHD8Iff8CKFaLN+TPPlDkuXV1/lV1LdpmcXaYuUte6XEp179/4nHiirTeQ4vyfUZkM1eFs7YxEIqkS0NHQcG/rzthvxxoMkhQEeOMNWLgQ4uLKrZDA9f+uk35d98BbEAT2vLKHC6tMr7FpiJz4HG7tukV+sv5oopm2q+h5fRutHTtCfpSY5VceS0fxX9w/cPh+UUiqhi/HfMWw83EEps2hsFAULRXpClSKhpUl5t3Zm5eyXqLvYt0BwHFx8Ouv4hhu+AfDmbh8otnOnX49nb9m/EX0/mid6we8NoC5J+aaJPiKfSGhrC8U/y8kGcjKEgSx5piB+abSOpwlGX9OThIy/Dqi8gnQu8+dJOy+MF4teNUo97jo/dH8PfNvks4b56xWmaRzSWx4eAOnfjhl1nmx4mKw1Dgywek13hn6juGN82PhyEMQ+2f1B84R6xGSdqRsmaARhSh73VbE9vaUWn3mFeWV1q5fFLWImVtm6tynoTHs/WEM/3B4vZ5TEAROWL/HNf8XESxKnr85EXB8DiRurddr0Ufj7E01Y5DYi9m4J1zAwtGWe36dzK2dt3AJFpX7mB3XOfT+IdpNbYdvN1/c3EQxzxhBQBDKhD9f3c6ApVhZiS8jpRKCOnvrrb3g2soV7y7eWDsZN3q3crAymA3VVJn+93SKFcVVMgEtrCx0RlINeHUAgkZA0Ag1trJqpv7Jis7ix54/MvD1gfR6upfObTo92EmcBJBA5q1Mvg79mr5L+jLiwxGlVkz29pCSn0K2MhsJEtp5teYEutv59fTrLD24FLWg5tcpv1ZZH3c0jvO/nGfIO0N0ivcNHgnMPTYXS1tL3Nu5m9WrvzKKTAWb5m6i80OdmfzL5NLlgkbgU99P8e7izcN7HjbuWAUCmSGD6DjEk7jsOLztvfGy92J6h/s4A9Vm/E1qOwk/Rz+GBhu29GgG1MVqCrMKsfOovsjWgNcGGGWlrBUXJAiM+XIMDr5VZ7Z33trJuaRzxGXH8b8ROqIz9aAd4Jlq9SloBDY+upFOszoZVQvG3OTmlo01Kwt/5qrxl5eUR5dHu9BqZKvqNzaBymJRcr6ovFtILXCzdUMikTCr0yze3Pcmv134jYc6P2TyOWxtRQG0KYkF+vDs4Fla/9mcQTcOvg60GNjCYPaLskCNb8QBcs8FwJCaZ7zUlvij8ciTNBDYdDL+KtvA7fu/fWjUGoa8PUSnrc1MA3MIEomELQu20PGBjgz/TrxPhYWg0ZQJf5roGK6szyd0YqhJtTCTLyaTeTOztO6cuWnKwp+jI7RqBefPQ7weF8Pc27kc+fgI7aa0w72tO5lRmfj1ECcTh747lKHvmrdfEn8knqi9UUxcObFBWPbXJXnJedw+c5sWA1pg625Lt3ndCH8k3OA+1s7WXP7jMpa2llg5WPHAlgeMylaojvJ9ofhj8fz9wN8MfW8oA14ZULo8tzgLAFfbsuwjd3ewU7YBQYq9pT25ylwcrKr2kVSFYqkQex97HHx0RAc0MrRB1s7OYhCUNms2MFAswRJ2X5jJdbdPLzvN9me38/C+hwnsF2j+iwbSC8TBo6XKtdZWn+XRHqt8EFv0/mii90bT57k+dzwj2CnASWddrvKcPw83SioqXL1alvXn4OvA68WvG3wePX76cWRW5nUZ6vFkD3ou7FnlvCmXU7i05hJh08PwLO6JZw542AODNuo/WNpRUUjKuQZO+l1IAAJdfLDXgLqk3q97qDuLohaZ4S+qX776SryPHh4w2vRy3QZJvpjM5XWXaTe1nc71priUHHz/IDH7Yihc6M+lTh9QYNsNeA/GXYFiA5GU51+BKx/AhEhw0D1O8nf0Byit8efoCNHhk7Gum+5SvZIdm83F1Rdpf197o+sCxx2N4/Ifl+n+RHeUuUrOrzrP+VXnSbmYwtivx1a7/6nvT1GUV6RTbN6xeAdZUVkUjb0XkJQGgIon3gBZF6Hd82BRPuJCgOhfwcYHAu81fPJpWaBIANtyoq2FLQzdXnXbLZ3Byg17+z1YqlxZ1S2SsUOdkUkaX535kHEh9X7ObGU2GokY9R3gWtKWHduK9rqODaPxNGf8NUEKrZ05M+olJAP6Y+1kTftpZS/rbo9344nzT5TaZHqUBCkbk/GXnS12ziSS6m1Bja1B1P+l/szaZrytgyAIFGYX1mvqfV0haAQO/+8wMQdjKiyP2BjB2klrUWRWDIErb/FZHe2ntifs3rBm0a+RUZRbhGsbV6xd9E9ahowLoefCnlg5WGFhbUGvRb1KC4iXF/487TxJXpzMkTlH8PEQj6dL+CtSF/HL+V9Yd2ldaQZLeQa9PogXs15snKIfIJVJ8evph2cHz1LRT6PWUKwoNvu5LG0smbltJj0WVhwoSqQSuj/ZndCJoUYfS6GWcztkMEFj2xPmGcbF+RfZPms7vj5iByw93XBt1omhE3l7yNv0D9Rd6D3tWhq/DPnFqDoHDYnr12HfPvMeM+lsEp8FfMaldZeq3bbrnK5GZZxrM/6srCV0e7wbIeOrdkJ/PvczAA90fAC5TMx+UGvULDu9jLkb55JfpFsNKJ9RbwpyOzkP/PcAA14bUP3GdYA2q8/OTsxwA/NafebE5/B16NfcPnubPs/1qf0By1F6P0s++6Q8MVrUy84LqUR8rszqJPZldkft5nau6XWItWKBNvr93M/n2Dh3o0m1Qe92fLr6MGvbLIPCb1F2Ab43DpB5TL/FXH3w2MnHsHrmCaDxC0T66lpc/P0iNzbfqJEYY2FtwUO7H2LIO0OqWOFqxxa5O0/w571/mnz8Q0sPsW7KOlTKuslG0H4OjV3QLY9W+HNwAH9xXrBihks50q6mcezTYySdS2LH4h382PNHo51dasK0ddN4OfflJif6Hf/qOF+1+YqUy2WRXud/Oc/qsatJvpCMW4gb478fj39vf4PHGfx/g1lwZUFpHzh4SDByO/NkXN7ccZPtz23HM8yTST9NqjDnAJCnFt0M3O1cSpc5O4NcYsuYM3mcfjBGp+gHYkbjsm7LuPyH/prHDZW0iDQit0VWsEbLyBB/jh4NwSWxV0kliSdubdzw7+1vcoajY4Ajbca1qVNL1HSFeOFylZtZhL+1l9Yyae0kLlj+AFR8/0XtjmL/W/tNcki5k2wsp5vFlJvOkUgkBp9HEokEr05epTa+5kIqk+o8b8rFFA6+d5D06+nG12ts/yJMjqtW9ANxbrAxlCHIvZ3LtQ3XdH6/MjLgWknyY1KSaJu6ef5ms7272k9tz0vZLxkUJVRKlVHzE1nRWcQdjSO+IIEM9+0ky0syuCUSkDvr39FzILR+3KA1Q+Uaf3VRksGcpN9I58aWG0ZZ7neY3oFXC181aT4m9lAsxz8/jiJdgX8vfx4/8zj9X+6vV8CtzLmfz3HsM921G9Mj0kk6n0RxsYRiWTa3hXPEZJU8SOL+hotvVN3JNgDuzYUuH1Z/cgsbcGgNMiOCKJw6gGNb7OxAghRHdavSwFaAhBMJRGw0rfZnU+FI3BGiMg27JaXki/00C5UjzvYlD0MrV/Abp1dkr2+ahT8zI5Xe+Y80PR00lla4BlTNXnAOcsark1epIOTmJi5PqzrfXwVttp+7OxUjEnRQOSNh8xOb+X3M78ZcfrV85P4RGx81EKHUSMhLymPXi7u4tLbiZHNmVCY3ttwg9Yqoxp74+gRX1l+p0eSfuujO21c2BqRSKaGhoXe8/Xp18mLOkTk6iz5XRqVU4RToxOjPR5d61WsnmeztxUGFp50nvf17G2znHTw70NWnK8WaYtZdWldlva27rdEZuQ0RdZG6grVn+o10vmz5JUc+PmJgr5phYW1B61GtSyPbyzPsvWH0fqa30ceqPDCTSWUEOgXi5CTWGRME457b+hAEgeQLyVVsmRo6H3wAn3wCCQlly2rdfiXQclhLg7X9TEUrFMn1zK1lKjLZcG0DAI+EP1K6XCqR8n/7/o8VZ1dwNulshX0ScxN5ceeLqC3FAaupGX8Abca2MftEg7FohT+ncjEE2sy/2lp9at+P/n3866R2ZeWMP63w521fFjHa0qUlAwIHMKLliAp1EYxF29a1k2Ax+2M4u+Jso6mpYAo/DfiJX0eIGeb1/f4tlNhyaeB82j5at7Veq0MilVS5540VfROJC64u4IH/HqjxcYOHBuPS0gVLy4pWuJklzStoVj+mrZtmsnVntye66a09aA6acsZfeeEvPl53+w0cEMjTN5+mw/0daD+tPcPeHwaINlk7XthB+o0a+M4bwNLWssmJfgAyuQwLGwusHMom7dqMbcO478fhGOCIIAjcPnObq/9cvWPXGLEpgmOfHUORoSD8kXDcQtwqrC/QiI3Vw8G5dJlUKs4/yAQbg/1Yr85eDH5rcJ1lstUl51ed5/cxv5OTUDZbrn1uubiUlUzRCn8gjleUuaZ17NpOasuMDTNwbuFco+s05v2bkiMKf5Yq1xrbK5cnMiOSjREbSZQeBypafWoD1LU24HeSK39d4evQr4k5EKNz/e3bcOpU2e+xsRXXxx2J4+bOmzr3VRerUWQqam3TWpliRTE3d9wk5VJFW5iQ8SEsuLaA4OHBHCr6mli3lUjlSoj6HdJO6D6YlZv4zwgOxR7iou9zxLn9TEGBGNx+9qezev/+O0XCiQTWTVlHzP6q9/TYsTJXktRUMYv59PenUaSbz4vWytFKr2tM7OFY3rN+j9PLTld7nAk/TODlnJfJV4uipJXUFvJjIGk3FBtQ6HzHQM8fwL6l3k0CnQJ5e/DbfDbqMwRBwMkJXBIvY7dtfY3rWdcll9ZeYvW41WRFZ1W7rUwuw8LKwqT+Qp/n+rAoahE+XX2QyWX4dPFh2NJhpUH31TFl1RTmnpirc939m+7nqetPUVQEGfYHeTWuC9P+nCau7PIRjDkLskoda4kULI3IDM+6CNe/gcxzkFkuyFuZLtaCvF3JIrzf79DjW+xLDq1NJtCy/639/DXjr+rPe4cpVhTzkedH1VqgG8upxFP0W9mPIb8M4chRDV9/LVqzViYxR3QCkqs8y4IgDczd34k55zuvUjVjduIPRmGVl1462W8IbcbfiuQn6PpDV/KK8vRuq80Wqi7bD6pm/CnSFcjksgri1a3dt9j0+CbSIvT3+vPzYdu2sgGnRCJmzrQa1TCU89pg42bDo0cerWLp2OXRLryU/RKB/QJRF6nZ89oeDn942KSXlCAIfBv2LWsmrjH3ZTdZ5Ppm6BsgB98/yIfOH5KXXLG9atuJfaX+gLaWZ0aG7nfQg53EGh2/X6wqzgsagZRLKRUijhsT1/+7zns273FxzUUAXIJdsPOyqxMxMzcxt1qB3tj6grKrlwg5+gt50QmoNWUCvkRS9tyuzu4zOiuaHTd36Hyue7TzYEn6Evo8a97MqLokO7ssO71ylnpt2q9fDz8e+O8Bozrxe17bwzftv0GjMjxY17777HNv83Xo11z4rWJm5R+X/0CpVtLJqxNdvMtq30gkEnr6iQW4TyacJCoziul/TSetII2JaybyvyP/45DyG6DmUbXFBebPdjUGXcKfs3PFdTVleffl7HpxFzO3zqTN2DbV72Ai2vup/ZrpEv4A9jy8h22zttHeo/oI6cpUzvgb+elIXsx6Ebl943k3GYuDr0MFSzNzvn8PfXCIg+8f1LteqZJR6OiJa0tns52zJmTeykR6/RpSVVGjzwzTl/EntZDqtDk2Fq3LB1ChHpR2srz1QF/C7g0z+bhBg4LoMKMDFlZ1U/GiKQp/2mj/8sJfWpp4Pyq3X5mlDJeWLti62RIyPoT+L4kONDe23uDox0fNHnCUn5LPrd23yE9t5A2pEt3ndWf+hfk4BZa9ND07eNJ9Xncu/3GZb9p+w/r71/PXfX/pFRCKFcXsfWMvUXvrprZv38V9WXh9IXZeugOnCskCwMvRpcJyYwKPbd1sGfTGIHy7V1NfpAHS7p52TPhxQgWLUkPCX05CDu9avcve1/fW85VW//5NzhUngORqV7PUxtVmFOVKxAi+8n1ZR39HvDp51dmz2SQkolCgL7Bk0yZxTK0dY1cW/rYs3MLmeZt17nv79G3+5/o/jn2uOxOopigyFPw26jdOL68oHsnt5biHuqOx0bBL/hQXgudgZZkHR2fB9S/1HzA3Eq5+CtnX9G8DXEi+wBWnz0h22izeTwlsmruJU9+dMrhffePbzZcpv00hoF/VenVHy5U9TU2FAS8PYHHKYpxa1N7xSNAIXFx90aA45ejvSMcHOpokeucpxQGDtcxGrMm4ZzhknK1mL8PYye14fdDrzO4yG4lEgp0d2OSm4BR7iey4hpf2FzoxlIkrJ+IYUH3Wc1F+EXFH48iONX7QKZVJcQ5yrnG9WbcQNxz99F+bRCqhuBjUUrH/YmdZ8i618QKXcN3ZmTkRkLTL8Ilv74BTC2F7L9jZv2wCsDAVrnwIKbrHSdo5xI3JXzJ/83wuJovzZ32e78OklZMavAuNWqnGva07tu5mKEgLLD+9HICY7Bje/P4027eLQQKVScgSJ+Wsir3KgiAvvAZ/2ENuwwiAaBb+zIxGY97IHVMRBIGkr/8i+Nw/Rgl/2m1OCj9wNuksP575Ue+22okJOyOSIiqn+4/+cjTT1k1DIpEgCGLtubgjcZxZfga1Un9W2qZN8M03sGFD2bIxX4yhx5OGPdcbAxZWFgT0CaiSfWHlUBYNJJPLePLyk4z/frxJx5ZIJHiHe1eJvGxGNxqNhosXL96R9qvIVBB7OBZBENj31j7O/Xyu2n2cg5wJHBDIlie38Nf0vxA0AkVFZREo9vbw1r63eGHHC1xLu4azs9hvUKt1T7CPazMOgLNJZ9FUKuKtUWv4Pvz7OzIYNQf23vZ0nNmxtC1ILaQ8duIxvTUUa4ogCCzrvoxVQ1fp3WbLwi183/n7akUjAE1uPrY5SRxK2YL1e9bM+rvMEtmrxDq8OuFv0M+DGPXbKC4kNy47T32Ut9EpbzlSn+1Xo9IgkUiqtWIpzfiz0CCzqjpxcCxB7DVOaTulSlCHVvhbvHMxLb9syR+X/2De5nk82/tZALZlf4ZKml8j4W/DIxtYare0zizuDKFL+NP+vzbCn0alwcbVBkv7usv6qGz1GeYRxnO9n2NS6KQK21lIaz5ZVV7YALBxscHaybpJZrJMWzeNKb9OAczffi+uvsi5lef0rldl5yMrUpglc6E2XPjtApnfrcOqIKPRC0S6hL+YgzHEHY1D0NR8kmD9jPV86PIhGpWm9Ng5OWXvPh+fhjkB0dSEP0Eoi/52cBD/aZ/dcXFV22/GzQxyb+dWOU7/F/vzxPknzFJfrjyR2yP5dfivxB3W4z3aBJHKpAiCwKjPRzF17VTQ0xSyY7M58M4Bbu6om4kn5xbOZEVn8T/X/3F+1fkq65USUe3ydqko/Hl4QJrDXp44NpjHNj5WJ9d2J/Ht5kvXOV0r1KnTCn+urlWFP3sve8LuC8O7i+G6U7GHYzm97DSFWWIHcMtTW2olHhnz/k3LFzP+bCVuhhwCjcbPUWz/2RpR+Cuf8ScIAoVZhaV/352k/dT2zL84n4C+VUWiggLYVTLvPrckmScjo2KWzOD/G8yoT0fpPLa1izVd5nbBO9y4OmPGYudhx/gfxtNpVqcKyxWZCvKS8sgrLLtAZ0d7GLgBQhbqP2D2FTj7PKQfN3heB7kocKtkOSVlgSQ88N8DDHpzUI3/lrrA0d+RTjM74RJc8XmUmwsXyg2VU1PF2qh2HnalFsm1If1GOn/P/JvjX+n/HJ1bOHPP7/cYVZ8scnskt3bfIk8pTs5ayWzAewR0+wqcOxje+drnsKMfaIxzBbOxgaRWfTk95hWcWnsYtU994t3Zmy6zu2DjUr0PccaNDFb2Xcn5X6u+q/QRfzy+VsFK6iI1mbcyq1jG5iTkcP7X82THZlNcDCqt8CcvmWgvSNCfvXl2CewdA4KBcVPQLBiyA7p+Bh1eBaHkftu3hAk3IPTpitvH/gVnX8DeTjzmyfy/+P7090Ski/aewUOD6TCjQ4Mfk1o7WzP7wGz6v6S71E15rl8XNQZDpXPeGvJW6f+jbf4BxPq8lUnIEjP+rNWeZc6I9q3BvS/IXapsfyfmnJuFvyaGoBFQDR9NUqu+pRFIhvCo9Pw+mXhS77baAawxwl/ljD8HH4fSSImT35zk27Bv6TCjA8/ffh6P9vpfItrC8Vqb0aZEUV4R6mLdL92MmxmcXXkWQSPg6OeIT1cfk49/z+/3MObLMbW9zGbqmJ0v7OSn/j+REZnB4Q8Oc3ld9bUsOt7fkQd3PIhKqSLuSBwSqaR0sCGVip20X87/wsdHPyYlPwULi7LMGl3RtcEuwchlcgpVhcRmVwxZlFnKGP7hcMIfCa/dH3qHCOgTwD2/3YNvt7qNGFYpVHSY0YE24/VnG9l62OLUwqnaGqVqNdwO7MW5US+S4ZuMSqPCqpw/uzbrOjnZ8DWFuImDh4g03Z7sEZsiiNweafggDQh9wl9tUGQoWDNhDdf+NRzNqmX4B8N58vKTFay3dKEViiT+fsy/MJ8OMyoOxC6liBbPnbw6Vd6VsW3GIkFSKsKHuIXw2ajPmN5hOq1cWpGrTiPWY1mNrD79evnRaVYnozNPzYn2numy+iwo0G2dYQxSCykP7nyQCT9MqN0F6kEQqmb89QnowyejPuGxbronLBNyEjh727SoW11igbpIzf6397PvrX0mXvXdywP/PcDjpx/Xu9753D667PgfqqyqwkR9EjoplIBFkymydmz0GX+6rD73vLqH30fXzuK/xeAWhM8OR6VUlbaPmBjQaMBKpuI73/f4b8F/Jh83/lg8n/h+YpSlVk3QjpOaivCXn18WLO5QksBU3u6zMhsf3ch3Hb4DxJpKK/uv5MjHR5BIxbpWMkuZWa/Pv7c/Y78Zi1cnL7Me905z4usTXFx9scryv2f+zclvT7IwYiFtxrSh/dT2erOSXIJdmH9xPj3m113ArNxOTuikULw6V/z8i4shJOEdQhLeopVHxT64m5uY4XA5fz9nks7oPfaxL47xWeBn5CXpdyRqDAgCZGWJ/9eV8Se1kDJt3TTCHw43eJwrf15h87zNnPv5HIJG4MyyM0TtrptsTi2ZiiwA7KXmsd/0dxQfHlnqqsJfRmQGH7p8yKEPDpnlXHXF1avidXt5Qd++ZfNp5bP+QieG0nZyW537u4e6M3H5RFoO02+5WBNkchndHu9WpeTE4Q8P84nPJ6TcFKNmpBpr7OyswH8SuBsoQ+HRH0YcAj/DAejaOp0qWW7p/Ww9ujXenc0rbNYVJ06I/QrXkq94ejoUZBWRcjnFLCK0vZc996y+h04zq475asK2RdvYtmgbBUXitVlb2IBzGIQurN6eVZEIebfEn3qIzopm582d3My4iZUVaCysEGSWDbp+ozE4Bjgy4uMRtBxuXLtTKVWs6LOCLQu21Pic1zZc48tWXxK5teJ8S9yRODY8tIHYQ7EUFYFaVinjb3sP2KM7cIDW80TLVsGAeGvjBT4jIORJsV6nNjBVJhfr/llXmn9P3AJXP8bJTnzXWmqcAbE0SVMkNxfeegtWrIB1VSsdleJt782qSeJYJsl5AwBROl65t3PEZ6sd5fpBrWbD0B1irb8GQLPw18SQyqSk+XQky6e9SRl/naJWAFSZ9C+PdmKispWPLvQV+BUEgbzkPCQyCc4tnLH3tjdYm0Nr56YtiA1w9LOjrJuyrsGnGlfHntf28K7Vu+TEV53BPvXdKTbO2ci+t/bdMVu2ZuqHngt7EjoxFLc2bjwT8wxjvjZerH1g8wMsiloElEUZ2tmBUl1IdFY0AG3dxUGHtq1rLXvLYyG1oLWrWCdQl0jU9/m+JhVCbgz8PfNvtj+33WzHs7S1ZNSno+j7vP66UYNeH8SDOx7E3suwN3v55+btAvGZHOhUVuNEK/xVl/HX1k2899fSdIta/z3xH/v/b7/hgzQgyg+oc800Z596NZXIbZEm2X4YQ3U1/vr696W3f2+dwl+4dziJzycS+VQk6UvSubrgKoFOgVhILXip/0sA3PL6hIJC02u49pjfgym/TrkjdTt1ZfzZ24NMVnF9Q0OlKpv0Nsbm6o/Lf+D/mT9PbnnSpPPoqvcmkUm4tuEaERsijMoUbgzkJOSw9829JJxIqH7jGuAU4FQhy6I8ggA5zoGkBnTBJcCIGhl1iHdnbwLGdUYtt230ApGujL8hbw9hzNdjSmuK14Qe83swacUk5Hby0mPfLElc8nYrptWoVjWqBWXtYo1ToBNyh7qx0S3flhv5UAUoe99aW5fVWgwoSYJJSKh6fzvO6kifxaKNuJWjFenX00k4kUDMwZg6qT3u1saNHk/2wKVl1ajqxsyBdw5w8tuqAbl23na4tnKt8E7QNyaWyWV4dvCsYBdqbgL6BjD97+lVJvnz8iAo9UlCk97Az825wjp3d7AuEsWJhBz97wILKwts3W1RZJivzlZ98N+C//iy1Zel96igoKxf6OxcJvwlJ5v2jNDWCj/wzgGQwEvZLzHp50nV7FU7Pui2jtFncumgmm2W42mtPvM1WailBRWEPwcfB8IfDce3x523d407Gsfxr47r/O5p64y3aiU66gSWDNEq2302FAL6BtDzqZ5oHMTvo0xjW6Umr06sXMGjX7VikjbjT11O+IP/Z++8w6Mouzb+25ZN772S0EvoVQRpUkQRCxaKgoKoYMPeXys2xIZdBKyIgKA0aYL0DqHXBNJI79mS3fn+mMxuNluym2xIeL/3vq5cSWaenXmenXnauc+5j/1xqalw4LsDfBjzIem7LT1WJJnP4cNBqRT75OGlZ/mi0xecXdtwB1nPQE+S706u05F/3ax1Tjn6jZg7guvfu54Kvbj48lK4IGvY+U24JRN8rCNZJbz6z6sM/3E4S44vQSYDtcqAd1EmWUdy7X6mqbBu1jo+TvoYXXnd+dC9Q7y55slriO0T69S1jVVGhn8wnOQJyfWuX0SXCPrO6mu1Vk0YkMAdS+8g4boES6lPKeIvaQok3Gn7ojE3QMv7QG47XyQAmjzbk4tBAxXpUFXL47DzmzDmPF5+4v2V+kAAijRFgGh//yDyA/JPuzdHs7tRcLaALW9sIftwtsNyP/5odkhesgRSU8W/izXFVmRnSMFoZEYVZV4nKFOfskn83Z30GAOOHaBrxZNuaEXj4H/E338ZjAajKaLHGeLP31/cxAWX9QfEnEI6g+2BsyERfxJkMhlD3hjCQ0ceQuFRt8enLeIv+2A2Z1afaZKIBXcislskHW7rYDMvQudJnUkalsTW17dyfOnxel1fW6Jl7eNrOfCtfU/K/6HpEdk1krtW3AWAT7i4mXcGFzZf4Jcxv1CUVgSYiT9fXzFxuoBAoGcgYd6iR48j4g+gV3QvekU79ghubgt4Z7D7k938+cCfVkbzjD0ZXD5SR8icm1HTAOpIKrKyEgKzThCYf570UnEXGRdgXqBLUp/Zjtc0pjxjx/NsjyE3fnUjw94d5kzVmwWkRRm4L+Ivvn88z5c+T7cp3eouDOSdzGPv53spvuSYpZLmPs9c0XBQO8rz41Efs/P+nSbCvTYifSNpGdySYK9g5DLzUu2eLvfgpwxC45HBGd3VQ9qCmdiTovxANJhI/9f3me75bA/b39/eaONTzXWMtLY5lXeKzNJMi/ybEgbED0CGjF3pu0grSrM6bw+2Iv7kCjl3rbiLqXumOnSSupqQfzqfra9vJWNv4xB/2lIt2Yez0ZZah8Tq9VAQk0xalzGo1U0vVyM98/+WiL+axF+LQS3oMqmL2+4hGSkliZ3IRC/uXnl3vfLUhrYNZequqSTfXX9jjiNI+6Sa0cJXMyTiT4r2A8cRfz2m9WDA8wMAMRrs6Zyn8YvxY8HABeSfad5Go+aEezffazPVw4g5I5iwZgIKlYLM/Zl8EPEBez7bY/Ma5TnlaIo0TbJ+l/Yl3t7WaYpCQ8FTLxJAl8sv27U99HywJ9MPTHeoDtQcofZT4xnoaZq3JZlPHx/RISw0VFRo0enM544vPc6vN//qUFYusEUgE9dN5M7ld4IASk8l3iHuyWVkDxqNDKXRF38v99zHX+1vimqpVGVYEEUevh7c/N3NdLjN9TzJ7sbpv06z9tG1NnOHSsRfTHVgnS3ib/enu/kg4gNyT1iTJceXHmfZhGWmPbw7seSOJXzZ5UuLY23HtGXUJ6PQeoq2M4XRG9+y9bAkEC7UEZlvrLIvOVgNU8Sf3Ez8fdPrGz7v+Hm92tBYUPupCUgIMKXUAbEPHqg2lfXvX0MNLTycga8MdMvY46zN8tTKU5z6w7ZKT020HtWa1je0RlMlLjD8PDzg9xA44ATpoPCozv9inyiL9RMn+PQScYL3VlXRYds37PtouxOtuLLw8PPAK9jLKbuyy9f28aDfrH52I3edQWjbUEbMGWGVq9Y30pf2t7bHP8ZfjPirneOvy1vQ7vH63dRYBcujYNsdUHJGzPF3tjqdV95O+CMOzi+0/Ix3NPgm4usnfo9ynehIJRF/XkFe9XK0u9LIOZbDP6/849DGd+4crFkj/p2UJCptffIJFJQX0/nLzkTOieSVza8w6qdRfHdgPmtX+hBaOhiAy0ErKCgwR/BL8DAEEVDZjUiPGnadQ8/BmS/c3ML647/DitCMIJc37Ve6+LbfSVozD4xGp4g/mUxcfPpo2xDkEYrWoOVAlm2iyJWIP3vEn+m+TngAG41mkiI/3+y0MOa7MbyoeRGVlwMvh6sAXe/tyrgl42xK3kR2ieTWn25l5Ccj6z3ZyFVydn+8m9TNqU6VNxqMDcrFcjVDLpeTnJx8xftvZUFlvb2fy7LKOP3naTL3iVINNYk/ydicFJRk0uKui/hbMHYBe6btYUQra1mB3BO5fNr6U3bO2Wnjk80b59ad48iPR6yM5jNPz+SeDfe45R66Mh2ftfuMPfNsG15qQhAEFt+6mF9u+sVumcpKiD+2lphTG7lUIuasqRnxl5go/j550v7zhBrEX65t4q/NjW1IGJhQZ52bAwTBvtRnQ/uv0lOJh69zkR+Xdlxi9YzV5KQ4DreUPLtV50+x9tG1bvNW91B4MCTyNgCOCg60Keyg8Hwhfz7wJ2fWnHFLfVyBrYi/mv/XN+Jv77y9HPj6QKPlHZDWMXK56AkMMOqnUcR8GMPuDOt8HVF+UVzXQsxrsuT4EqfvI62tKmu9KgFxAW6XxmtKxPaN5aGUh+g4riPg/vk35ecUvur6Fek7rRmJmmtSe9G4Vwol6SVsH/cR0ac2/9dE/EnknL5S7xaiIedoDsvvWc6FTRdM15bmgSjXFfCvGDw8xPECrn5SFxwTfxkZMqf6b9d7uzLs3WGNQuDoK/R83vFz1j3pPhWH5oCwDmGEdwp3WMY71JvgVsF28xtteG4D7wa9i7b4yjPQWQXFFPhuoyrQWnUiNBQ8qkKRC+JAnFX635XTY9g7wywkpyVyT0p1qFSaCQbJia/oQhFnVp9xSAbpynQkXZ9EwsAESrNKSd+d7lSkiz04M//aknJuCGQyGTH+MXjKvdEr863WPM0FPR7oweQtk21Gy9Ym/hKqt1I19yleQV4Etw62aVvJPpRNys8pGLTuj4D2DvPGN8q2okFRuThZK4zeqL19ILgneDoeY1gSANvucljEnOOv1KRaE9c/rtntMTve0ZH7d9xPRLJZju/yZdEpzNtbfI5Sv9T6hTL4tcENlpDWlet4P+x9lk9aXmfZ6QemM22f8zlPb1TOZdReHfdEPQJBXUDt5PyqL4VVHeH4+zZPS3k4M0rFF93DT016u2HEjW4cZ6mGYPBrg3lg3wNO7ZMEo8DXPb5uFmuFmvY/ixx/KicibAr2w8rWZjLP6uKV0HIqRA4DpTeUnAJddRSNVwy0fRyCulp+pqoSytPw8xLrIdMEAlCoESevrpO7ct+2+whp4wTB0IRIHJLIg4cfpPUNtlPvCAJ88YX4e+BAePVV0SHnzBmYvPA/XCy+iM6g442tb7D27Fre/Ocdzp9T0C7vOX4Zs4JrFI8A1nKftfdBCAKc+hTSV9qsR1NwRv8j/v7LoAr1R+sTjLev3OkFmmdoNtva96ZQl0fvmN5U6m2vwKTNqysRfw3Rgi4oEMk/EI2oUodSqBTNPrGoO+AT7kOfR/rUmUvKHlReKmZlzGLsorFOlU/bksZs/9k280n8f4BOV/+NU32x4fkNvBP4jk1vwrrQ/rb2XPPMNcT0FhdnNYk/aaEmyalA3cSfI/jH+iNTyFCorz7j891/3s2sjFlWx905hhSlFmHUG52S4pPJZMgVcodJqCsr4UKXmynuNcQkv1yT+IuLg06dxPFR8liyBYn4u1B4wSQHcrUiN9eSEKkdHVaf/isIAgfnH3RJtqLliJbcs/EeYvrEOCxnIv6u6cW9/9xLQJzZcJBTnoOmqv6T4w0Jd6KqCkRe5V934dr1Ktdx4JsDZOxunGgrR7CV46/m/7W955zF1D1TufvPu+tdr7ogPUtpXSMIAtlloqUu0td2/pLb298OwB8n/3D6PrakPiUcXnSYo4uPOn2t5gyVl4rwTuH4hJsXk+6cf2P7xnLdf64jsEWg1bm8c8W02bmQ0OyjJonZpoI6QI1XsDcGled/DfEnkddrH1vL+2HvU1nYMIuupljDkR+OcPnIZdO1q6od533z0/j76b8pPF+//CP7v97Pjjk7GlQ/e5DJbEfwXq2Qxm5bxF9mJmg05v57acclvrvmOwt5tPRd6aRuSaXfk/0aZf+m9BI9Mv6bHCSqNFVU5FfUSaAHJgRy3/b76DzRdu6oFoNa0GN6DzwDr7y894Gsg+xoN4BN4bdYnQsNBRly1DoxCkKKLLGFw4sOs+vjXY1WzyuB2sQfWOf56zWjFy9WvmiVn02CIAh8EPEBv978K4JRYN8X+/iu73dk7rWfq8sZOJp/NVUanjl0E0cSpqPydN88feCBA+y4sYyg8r5WtqItb2xh2YRlbrtXfRGYEEjCwASbjub2Iv5qEn+dJ3bmvm33Ed7Rmlgb8sYQXqx8sVEiaEbPG83EtRMtjm16aRPLJy03EX9KozfKqGtg6AYxD5gjJN4DEYMdFqkZ8VdRIY5ZIz8aaTNiublBUkoLDRXnbon4y7UO1KwXtMVa2t7cloTr6iZB1f5q5ArH5vkqbRXvhbzHmkfXiPtioxwPnxgYugk6PudcpTSXQekFSttGXcl+JMkwe3lBdqv+BHRv6dz1mylkchm6Mp3ThPvmVzfzXb/vGuy8+9eDf7FsouWY9nmnz/l+4PeASPyFlg5lcqtnGNRiEJSlwj83wsXfbV9Q6Sfm7FPYsRGr/KD3F9B6OnjHwG250OEZ8Zx/G+gxF8JqpaZJ+xlWtCBIvxUAQWMZ8Xe1QO2nJqJzhN1I+DNn4NQpUbr+vvvEvJ533w0VHmn8lfMpAM/2f5YIH5HwHxp0PzJkDIgbzF3dxtA6Udys1yT+jIKR948+yrmIOXh415jQxl6CvvMbp6H1wP+IPzfDaKzb8NuYaDVzJGd73e1UtJ8En+ASin324SX3Z/fU3QxOtD2528rhYQ91Rfw5g9oTriT3KQgCqVtSOfbbsfpfvIlh0BlYfOtiDn5/sFHv4xftV+cCQoJcKSe2byz+ca4bk692GI1GTp06dcX7b1S3KNrd3A6fMCfY9FpQqpVc/+71BCWKE3NNj2xpAy0lUAfnib8qo7UchdpPzcyTM+nzSB+X69nUkMllNkm2vJN5HPvtmEPJTWcR3imcR889Su+ZvZ0qP27JOMYtGWf3vEYDpWFJ6FoGU6YTGd2azxLgxuq91Nq1ZmKiNsJ8wgj1DkVAsJnnb+0Ta3nL+y20Jc1Xiyw7O5s333iDIX3as2N9DHv/ac+Z02+QlWXWOa1v/y08V8jK+1falciyBf8YfxKHJNYprSTNferwAFpc1wKlp9J07pE1j+D7ti/f7P/GpfpKuDZmMNcfvkyPfNuemo4Q1j6Mp/Oe5rpXr6vXvRuCuiL+6iv1qfZTE9outP4VqwPSs5QixEq0JVRWiZtAe8TfmLZjANhxaQe55c5ZDxwRBZtf3syO9xuHpLjSqMiroLKg0mTQdvf8G9klkkGvDrLplVp0sRTv4iw8jU0fYqD2U3PL2ge4nNTvqo8Kqx0REtw6mNg+sQ4dXJxBTO8Yni99nr6P97Xee1y8yM4PdtbbIHPwu4Ps+rDxyATJSVJyyrqaYSviLyxMHBP1etix45yp/5ZklJB3Ms+CaNr/zX7WPb6O8suN86LLZDIePvYww965eqTL60La1jTeD32f/V/tb9B1utzTpcmM7zmlItvlLbPOvRgYKOb39dKJa1vJYdEW9n25j3/f+rdR6thYOLTgEAe+MysoOSL+sqqDHVVeKoeS3ka9kY53dKTFoBbMaz+Pf9/+l6GzhxLavv7rn7rm36zSLPYU/UV68A/4ebtPacnHwwdvb9EJoPaaJ2tfFqdXnW7y9BIGvcFmtJ5GYyaLJOJPynlaXOz8WlbpqWxQDlxXkLE7g/Mbz9PGtzu9zqyie/4c5z/c+wvo8LTDIpG+kbyXcJQhR8812whOgHPrz/Hv2/+iKTIb52sSf2Am/i5fqGDh4IXsnNswtSO/aD9u/fFWuk/tXmfZ4kvFpG5JxaC3T0wZ9UYiu0biG+VLZaVAeXkZHh4urp/9WsGIfdD6QZuna0f8eVZP5w0J6GgsHP7hMHu/sM6Faw8zT83khs9ucKqsQWugJKPEbt5wZ5F3Mo/c45Z7wfhr44ntK85/Oh1EFI/m6W7vMqr1KNDmQvZ6KLeTLsK/Ddx4AhInNaheFgjsAm2fQB0sDmYKXSBgjvgrPF/I1je3knWgeUfna0u0VBZWWo3dvx79leB3g3lx8/MYZXqSk8220RYtwFuXwKiilTze53HeGfYOJ2eeZMOkDfTUiWNfUhIWv6W0AwCZpZn8kfUpJ2Oew8ezep6UycQcqV625UmagjNS1l3kf7iaIBn1XSL+AiqhHFQ4NmS6EvEnTRANIf6kiVhCfr64sJLJZKx+eDUV+RV0uL3DFVs0uRNl2WWcWnHKpke6u++TdzKPuP5xdXrCJgxMYNg7wzi79iwxvWNQqv83PDQ2ej7Yk54P9nTLtWpG/J0ssY74C652Kiy04xyvrdLS9auunCs4R+7TuQR4WkubXI1I25qGf5y/iSCVcGjBIba/u52Zp2cS0to9sgXOkux1oaJcAAE8vLXc2fFOijRFeKssx+e+fcVNSl4ebNsGQ4bYvtYbg9/AW+VtETEoISgxiBbXtXC4wWhKzJ0zh+eefZbBKhXPazREAtnaTL49+zYLz7xGxw7v8cQs62hOZ+Eb6cudy+8kIMG1d10QBASD4NBAY4r4Q49gtNzcH805ikEwWJG5zsLbS4FcUNRrfpUr5Y2eD8YWBKHuiL/6SH3mnRQXCiFtQhptLWAicav3fVK0n7/a36pfSogLiKN7VHcOZB3gr9N/MaXblDrvY0/qE+DWn29t8MazuWDjCxs58M0Bnsl/Bq9gN2mHOYmATrEcGvEswUH2DYqXyy7j6+GLj4frDjmuoibZKwjWebCuFtTO8df/6f70f7p/g6+rUClMa9faKibXzOrL8Fmd8Ivxs/HJunHLD7dYOGS4G0FBooSYvTXX1QSJ+KuZn1Uuh+ho0es5J8dMCHQc15F2N7dDrjLPj+3GtiP/ZD7qgP+OMexKwCfCh+7TuhPZ1bZzSU0cXnSY/DP5DHnDzkKwiZBfXgSAjzLQ6pxcLtoqvHQJRHhdQG+w74R309c3ofZXIwjCVaP4s2vuLvSVerrfLxr7HRF/l2ukIrq04xIGvYEW17WwuqbCQ8HN398MiA40giDQ/9n+jfqdmAz/+mgTUecuSGN67TXPuCXjkKvkTf6s/5z6J4cXHeaF8hcscsJJRK2fn9kZwtMTwsMhJ0fM89epE5TnlnPgmwPE9o0lcUiixbWzDmRhNBjtRnc2BDnHcjj++3E63dnJ5BQ3af0kBEHg9GkZEcU3EK4GMv6CvN3QfhZ4WJPzrkApV9IqoCNbdeY185Efj5C+K51Rn4xqNra6M6vOsPvj3XS5p4vJOUWyndYm/vIK5FQcuUx072gbV3IOro5ZOz/cye6PdvP4xcctlGJqwsPXg3s2iqlKnn3+P5ztmIJQ2gGOqqH9U6BwMrpb4QFGPWT9DQoviDA7hEr2o+yybKqMVXh6Kok4t4Mtt+wl+cjUejmsNxb2fraX0sxSej3Uy+3XHvbOMLc4FN27+V6r9+Dm+Teb/jbZDKRhJqQX3KkB6un8cOYLyN0BPT8R+3buDijYB20egZx/4ORH0PFFCK3hsB7SE0J6ohbE+TmyaCzfTBxA6xjRRlaUWsTmlzfjFeJFVPfmq7W/7Z1tbJu9zcq+9/qW1ynUFPJ79jsEtv2HxIhH+XxvIafyTvFo+/cBD/yyb2DuSJEUDvQMZGjSUDakip9PSoJTeadYrV3I2Uh/4s6bo2vPF4osoJcuAZ+Qanu7rggq0sE30W5k7ZXG/yL+rjJoS7Ssf3Y9FXnWLuEbnt/AiXkbQRBcIv68A8QVl8IorsDKdeU2ZeGaS8QfwMiPRzJh9QRoHusIlxEQH8BL2pcY/IZj6YSGYscHO1g4eCEll5xzPzv912k2v7yZnKOO81f9D80PEjHv6wvfjPmGrCezeKjXQ6bz0mazZj+qCbVSTbGmGL1Rz6l868TSBecKWPfkOi5uu2jj080TVdoqFly3gM0vbbY61+muTty++HYLubn6QF+hZ/0z60nbascrywYy92Wy9c2tlGaV2j6//QLd17xN2OnL/Hr7r6yduNaqjEIBo0eLf//5pzkHam082PNB7ulyD6He1h7BfR7tw4Q1E5qECKoLc+fMYfaLL7LVYGCtRsMEYCgwAdhs1LBNMDD7hReY++GH9b6Hh68H7ca2I6qb8wvYvJN5vKF8gy1vbHFYTlrEl32+iLnxc03HtVVaTuefBiA5on65EiTHmkqNwN6MvWirXJto88/kk7Hnykl9Zmdn88rLb7Broxi1OeLa9rz5xhtkV+tbNYT42/rGVua1n9eoUav2iD970X4SxrQRo/5Wnrat718bjqQ+4/vHW+QkuZqRcF0C3ad1bzTpOUEQ+GH4D6yeudrqnFYLyGSovWxvf84VnKPlJy0Zu3hso9StNtL+SiHqzFYEoXl6UjsLq9wWbkT24Wwy9mZY7D0UCohOUBGUFFRvJ7WQNiE2cze5C9Kaq74Sxs0JtiL+wCz3efmyZcJMhYdlSoa2N7Xlvu334eHTeIk1T/15ip0fXn15qO0hskskN319kykiwBFOLD3B9ne2W3m4V+RVsGjYIg4vOtxY1XSIggqR7fJX2iYVQkKg64Uf+P2adCZ0nmD3OuGdwgmID2hyIsgV3PrzrYz7zazs4YzUJ8CSO5aw/qn1dV5/6NtDGTZ7WKN/J5mlooyopz7areP7P6n/MH3zzZyIedZq7qs9fjQVYvrE0PGOjlZpLmrLfEqQ8vxdrN4ma0u0bHpxk4XssYQ1j65xmOu9Icg/lc+W/2whc7+lBKxMJrOMzs9cC8feBEMd6+cLP8C2O6DKccR27TXs+fXn2Ttvr1uUddyF/s/054H9D+ATYd77S/bG2sRfbqknz+Q/w/Xv1iGF6gBHfjzC1z2/5vKRy3UXRnSSGfnxSKfzzqfJN5MbsYwk3V9w9E2QuzjHCkbYejMcf9ficLhPOAqZAqNgJLssG09PEGRy5N6ezU4laOyisS6lezix/MQVT2tU13im10OFxwWytefNKbdkMpA5oGoub4GDT4PehqxE3m5RulNR/Z6fXwD7HwNtvkhGZa0BbZ7156pv6+sLKkMgYYqWBHoGAhDdK5rpB6fT6a5OdbS2aRHdM5ru07rjHWpp29p07yaujb8WBBlFvruYlz2eGatn8MmeTzhdKSrqlJdbcheCYI7sS0yEC0UXWHBuNhfDviYjw2zrkYg/b22Sea+SvQFWJ0P6n43ZXJfwP+LvKsPuT3ez470d7PrIUp5GEAROLjtJ8cELIJO5RPx5+okztMzgxdSVUwl4J4Alx5ZYlZOIBWeIP3eEhDsi/pKGJRHVPapZLAzrC7lS3qibYBAXECM+GuFUpMCf0/9EJpdx/677berR/7dDcYUT/vzzn3/47bbf0Ja6ZwFVM+JPLpMT6RtJsJc5d4AU8Vdaas6RUxvtQtsB2JSF1JXq2PXhLs6tP+eW+l4RCDDq01EkT7QmWCK7RtLxjo54BjTM+JxzLIcd7+8gdUuq05+5tOMSm1/eTP4p27qrVXIVpSEt8AhzLLs7fLgotXX2rGVeiasd2dnZPPfss/yp1WJPXLYP8KdWy3PPPEN2dna9+m99ZIS8gr1oe3NbQts6llaSFo7enZLocHsH0/HT+aepMlYRoA6wiMh1BdL8+k+L6+j9bW82Xtjo0ueXTVjGb7f9Vq97u4q5c+aQEBvLnjlvM6fsJL9pM3ni5En+ffttEmJjmfvhhw0i/pInJDP4zcGNmr+ovsTfhM4T+OnWn5g/xjl9/7pygunKdRgNTSsn7w50ntCZm76+ycID3J3zr0wmoyyrjIpc6y/y4oYz+OecNT3L2pi7ay7l+nI2nN9gN9+1O3FiyVGizmwFQbiq5T5rRvydXHGSFfetoPCCe0LdFo9dzF/T/7LYe4SHQ9G5fEozbTvPOAPBKFCUVtTg3C32IBn4/5si/uwRfzk5Yofa/v52try+BYPuyqsIHJp/iPVPr7cpzfffjpGfjOTxtMetnGHLssvI2J1BSXo9dbQbCEkizN8j0OZ5Mc+frM4UBMYqI6WZpejKrnwu9voivGO4RbSms8Tf9e9fz6DXB9m85oXNF1gxZQU5x0Tn3PnXzmfFfSsaXFdH829jEX8FlQWsTV1Jvt8W9HrLfWl5Tjln1pyx6xx5pdDr4V7cvvh2KzUXe8Rf7Tx/AXEBPLD/Aa55ulYuLaDvE30Z8mbjROi2GNyCBw8/SLubxT29scrI8aXHyT+dT0rOES6FLKDQZyd0fh1GHwd1mOMLFh2Bi0tA49gpfGXeHI7HPkmO5hIAwz8czpPZT9rMkdhU8Iv2I6p7lIUKlr2Iv5wc+061zqKyoJKi1CL8op1TJmhxXQv6PNrHoUx6UVoRm1/ZTPrudHSI69zNQU/DyH2OiSJbUKih3yLo8pblYbmCD4Z/wPwx8/Hz8MPLC3KS+tJ6znSCW7o/L2VDENo21GZkvCAInM4/jVGw3Df9++a/bHpxU53XNegM7PhgB5d2XGpwHUsySjjy0xGKUosAUXL2r4f+ouCcaNzW6+Fg0kQGLm3J2rNroeSMGImpdzB3526DEx9A8XHrc/0WwLgSMaoToO0jMGyLmPsvcRLcpYOYWnKnlVmwZQycm4+vr3ioplS92k9NZNfIBkv4Nzba39qem76+yaqekb6RrL/7X4YePUdUwTiuiR7Ire1v5dHejxIbHGqyrdTkG3JyxD25Uimud/vH9UchU1ChvkC58qJprL9QKCb889YlmudJ/7bQ6RUI6tq4DXYB/yP+3IzGJg/6P9OfR889ysCXB1ocl8lkzDgxA+MddwHmycsZqH2qN716L4K9gjEIBjanWkbICILZGOWM1Kc7I/6kwad2pJLRYLTyZrpakHsil7R/06jS2GFg3IT4a+Pp+1hfK6+H2qgsqOTA1wcovlhMbJ/YRpU/ao5QKBQkJydfUfKv4GwB6bvTnfbqqgs1iT9b8PMTJy6wb4iSiL9TedYRf+GdwplxcgaDXh3UwJpeOSg9lfSe2ZvWo1o32j2iukcx48QMut3XzenPdLyjIw8ceIDoXrblQ5Qt4jjTZwKK9mEYjPYNaP7+0E58ZJyzw8dqqjRsvrCZRYcXWZ3LPZHL+mfXNzu99m+/+YbBKpVd0k9CH2CQhwcLvv++Xv13ye1LeC/kPZcMhT7hPty57E6SxzuO1pO8wMLvHMzIj0aajh/NOQpAp/BO9XZckRanPpo2AGy/uN2lz/d5tA8DXhpQr3u7gppRm+tqRW2u02jYahCjNtetEaM2azv7OIPWN7Rm4IsD6y7YAEjPUlrXZJWJ/SXK13GkaKvgVoxPHk+Ql3MSSo6kPre8sYXZvrNN0qb/TWiM+fehlIe4ffHtVsePfbKRhJRVdom/3Rm7TX/nV9ZhjXYDRs4dQeoNYmS+PcL3akDNiL/0nekc+v6QQylkVzDwlYH0f6a/BfEXFQWLb1nMwiEL633d8xvO83GLjzn661E31NIadaksXE2wR/xJ6ReMxmgUCgVHfjjCkR+PWMh8XikMeWsI0/ZOu+L3bSxsf387yyYuc0qKPTAhEL9oP6s1RXincJ4reY7+zzRcdrc+KNaKm41AT9tzoGSrqGvuP/nHST6M+ZDTf512Z/UaFdoSrcWzc0T8FRaa7SXJdyfb3bNkH8zm0IJDaEu0CILApe2XuHzYuUiiqip4/XX49ltLkq2u+Vci/tS6aNPa0yby94KD/UptSFL3lR6iUb3muidtaxo/3/Azqf+kOn29Kwl7xF909ZYup5ofU3goiOoeZVMWscNtHZzK+VYfeAV5EdE5wmRfKM0qZcntS9gzbw9bslZxOHEKxz2/FfNPBbQHeR1rr+TXRKLAN9FhsT+zvuR85Ifk6EVruHeIN74Rvs1G5hPEfqkp0lg4fubmCRxMnMi3mTMB87ik0cDR5Wc4s/pMve/X97G+PJX9VJ12OFdQeK6QrW9sJWt/FnpB7DjefpEQWM9IrIQ7IdjahvF438eZ0m0KAZ4BpjVzc1Sm0JZoqdJa2lONgpHJKybT9rO2zNlhmc/yhnk3cOvPt9Z53ZL0EtY/vZ4Ty040uI6Z+zJZPnG5yUn84raL7P9yP1WVYr11OjDIRe8/b5U3pP0Km0dA2Xl7l4SW98GNpyC4h+3zNeUlA5MhfKBI9NqDIEDWWig9g68vVMlLef/A88xcPbP6tICmSGORH/Nqw/nz4KVNZHjxb2yftoWldyzl41EfkxzRyRQgUdMRSYr2i48Xbah+aj96RIvfd77vFi5cqC5XZI74Cwys/nBgMnR+DQLa2azLlQ44gf8Rf25HYyciVqgUdqVtZHIZ+VrR6u9KxJ/KWyL+vLm+xSgA/jr9F1VG8yCq04Ghej3nSsSfO4i/tm3F37U9AlfPWM03Pb+h+GI9wgSaGHvn7WXBwAWN5mnsKryCvXiu+DmGvDUEg95g8kD5/wJBECgpKbmiicRv/fFWHjv/mNuiViXiT6Yu444ldzBr3SwL0kgmwzQZ2TNEtQ0RO9vJfOuIP7lSTmjb0Ga1gG8Ick/k8kHkB2x7Z1uDriNXyAltF4p/jOPovJrwjfQlqluU3YhfaQO8XD8dz7c8+XLfl3avJXmYXrSjwFpYWciQRUOYsmKKlSRkcVoxO97b0eyIvxU//8wkJ3cX91RW8sdPP9Wr/4Ynh5MwMKFR3mmJLPKo9YhrEn/1hbQBCywX8xnszXQ+qTlA54md6TndPblF7cGVqM3P5z2DRpNNZqZr3rVXaryW1jHSs+we1Z0n+z3J6Naj3XqfmvluajctsmsknSd1/q/IvfvLmF/Y/r6ZrL6S82+rR0dxsdMom8Rfekk6+zL3IUNG9pPZ9c7B6QqCWwXjERkCMtlVG/FnNJqNQd7eYk6Upy4/hX+s83OiI3Sb0o1Od3Wy2HtERkL3B7rT6+H653QJTw6n96O9CU9uHIWL/z8RfwIXLugRBIHpB6Yzce3EJlFjCesQJirB/JesUdN3pnNy+ck687MDGPQGCs8XUpFv7T0gk8ncRsK7ihK9+PIHe9sn/srUZ3grcyDXLbjOZhkQn22vmb0ISmpYHrIriQ9jPrSQcrRF/Pn6mp2pL9fi72zNh32f6MuzRc8S3TMamUzGtH3TuGvlXU7V5/Rp2LsXVqyA2bPNa9S65l9zxF+MfftP2XlYfy38qoS/2jm1kEsMFEkkrUcmBpnGglCI6R3DTd/cRGyfxp+DHWH7+9v5++m/rY7bI/4kw3HNPbauTEdZtg0pvkaGplhjipj0DPDklh9vIXl8MmVacYzwVHpD+SXQOOFxp/QGed1Re74qcYIo1Yr31RRpyD2R26ykPldMWcG7QZaylhdKTpIR8hO/pc6jSFOEp6c5n+2GJ9ey4bkNDbqnK+Nv1oEsvuzyJYd/sC/PHNs3loePPUzHOzpSJROfZ4SiUJRxrC+MetDZt6d6eYHcoCfnz92cXtW8HDA+afkJP4740eLYM+ufMTk8f7brK4vxLbZvLHH94uq8rl+0H/dtv48eD9gh1lxAbN9Yxi0ZR9LQJAAG/WcQT2Y9SUjbEARBdMaoqib+fDx8IHYM9P4afFrYv6hXFPi3sSbuK9LFCN3a74OxCnSFUHrOdjShV5RI8Hedja8vCDIjiy68w7y989BUadCX63k36F3WPbGugd9G42LTS5v468G/TP9vPL+RIQuHsPjoYs5Uc/itWll/ztb4LRF/SUnmY9cliGuVfL8axF8Nqc+wOgKoJVxJm7OE/xF/bobR2HgyTMeWHCPl5xTyTuVxaOEhk3eDIAhsf287WQezTOSYSzn+vGV4VIWgqgqinfcAgjyDyK/MZ8elHaYykkFCJnMuh4crniH//APPPAPp6ZbHJeKvjRjUYEVWdBjXgYEvD0ThceUZ84ai092dGD5nuIXGeGNAX6nn6x5fs/YJ6xxhtaH2V+Mb4csvN/3Cl52//K+QFHMWRqOR8+fPN2r/lWDQG0zfrTvfXYn4K1eks+T4EuYfnI+i1mJAmtTqivizJfUJotTchc0XKM+5OiyUp1ae4ovOX9j0GvUK8iK4ZXCDvfAup1x2We9eEAS0pVoqC20T/3lr9xFzYgPFwiWqjFUE2fGWBnNOCXtSn5G+kQR6BmIUjKbcchLir43nkTOP0Onu5qXXXlxSgmMRRTMiqsvXp/8O+s8g7lx+p8v12/DcBnZ8sMNhGa0WZEYDWfOWc2jhIdPxlBwxr0ByeP3y+4E4D3t4QGC5mJR7b+beJllAOoKrUZvpl75Fo3EtH9b+r/fzaZtPyTroOnFdWgpLlpgN2o5QO+JvYMJAPhj+AZO6TKrzs2W6Mmb/O5uxv46ts6y0tqqqEmVfaqLtTW25ZdEtBLdqXhI7rkJfqSd1cyo5KWbJqMaYfy+nXGbfl/usPFO92iVQHNHGJvG38pSYi/GauGuI8L0y+RSNVUZ8hFLkVbqrNuKvZqSG9A77hPu4nfypufeIjhY96fs8WtcIYx9+UX6M+ngUCQMS3FA7a/w35fgrqbYP1Sb+YmJEG39OTgVFRUbkSnmTkTPSuqq25//VijuX3cnTuU87VfbS9kt80vITq7xFl3Ze4syaM05FDTYGWmvuok3Ga/QMtx1x2KoVKAQ1qcK/7Ly000qSTUJYhzBu+PQGYnrXTx69KdDp7k60HN4SEOd0qQ8F1eoeUtRfVvUy5sKmC7wf/j5Hf7GORJbJZHgGeJrI4Oge0U47Hda0oezZA6++Ktarrvm3oFL8oEOpT3UYtJsFwb1A6edYnq4aod6h+KhEG0ilR5rFPBIQH0D3qd2bnOg99ccpUn607FOCYJ/4k2xvNR3Vv+n9jVVkutFg5NM2nzaYUHKET1p+wtK7lgKifafzhM7E9omlXCcuNLyV3vDPDbDBCfUPfRnkbodyxzkl/NTiBFFRJS6s9321j887fE7usXrIeTQSEocl0v2B7qb1SWUl5BtSTefPFYjyOZLxvu0ToyxUW1zB+Y3n2fjiRkoynJdaVngo0FfoHcplq7xVhHUIwyvE20T8DTk7DnZPrVc9qUiHJQGQ8qrF4czSTP4+9zf7MvehVoOAjMKf11r1iaZGp7s70foGyyjpuzrdhbcQhtzowcWyc+zJ2GNxXjAKde6blZ5K4q6JI6SNC0Z1O/CN8KXD7R1MDnEymQzfSF8UKoVpvydF/PmofCCoC7SaBnZksk3QFUJRrbkiY5WYkzO3hmO7vlR8xvseM0cTltaSiqqxZvf1BaXBD3k1VVSkKULlo6Lb/d2IHxjvcvuvJC5svGCRV/WHIz+wOXUz/6T+w9nqw61tBNXbIv4kYs8e8Sc53ZuJv0Qz8bfvEdhcS061Bq6Ezbk2/kf8XUX4961/WfPoGg4vOsyKyStME2neyTw2PLuB/V8fMOXIcUXq8/YOt3F/fh69zq0gL0fJjW1uBGDFSbNufM38Hc7s5V2R+ly9Gk6cgPffN8tPaLVmg5wU8Veb+EsamsTg1wfjG2lH27AZI75/PP1m9bPSjXc3lJ5KUU60Dptw1sEsco7lIAgCHe/oSJ/H+jS6DOn/V2ybvY35/ec3KDeNLUj9pQSRQY/xt94g1yU9JRF/Z/LPWET8Sji14hSLhiziwqYLDa/wFUCVtooqTZVNbzvfSF/u235fg6RWBEFgwXUL+GnUTy59Tlus5R3/d/j7KWtPUoCy3ccIu7ifAqO4oogPsL/Iqov4k8lkdAgTc8wdz7XUgffw9RAjTho516irCPD3J7vuYgBcri5/JXHkxyOcWOpY+kOnA4VeQ8E/R8jYnWE6flObm5jYeSJ9YutvsAZxjvXTdEKt8KRIU8TZgrN1f6gap/86zRfJX7iUl9JVuBq1WZr9M2A2qDgDmVwmbp4iXF8DzJ8PixbBypV1l62d488V6A16Xtj0AitOraBM59jru6ZRzZbc538DVF4qni99npvn39yo9zn912lWPbSK/DNmC5yxyoimUlwMOSL+bm4r1q1c1/gOLseWHCPouw/xzz171RN/SiVocks5+cdJKvLc15j93+zn09afos8yy9xGOusZ0oT4/yD1qVZDeLiAzGhg/cv/kn+68eVx7eHQgkO84/8O5zc4kMa6yqDydi43VkibEPo92Y/IrpFUaas4+utRNEUadn24i59H/1znHrCxEFlyE22yXqFPTF+b59u3hz4do0CQoTfqyS1vPgRBQ3HT1zfRb1Y/QCT9BAHkcnMkkYSoasVwKc+fT7gPwa2CUXpZR/dn7Mkg93j9viOJjGrRQlxrHD0Kh+0HFZmwesJqHiwoIbJwrH3iT+UHXWfDyD0wci94BNR5XZlMRlKQaE2tUF9olmueiesm8vCxhy2OlZSIzvAymfnZSZAMx8XFZntW54md6XhnR4tyVZVVYkqVRgxO7vlgT9qObWt1vEIi/jy8xVxfSffVfbGSk2JEZ+ovDov5VxN/5dXEX9w1cQx4aQA+4Y3r5O4Kej3Ui5u+usn0f14elHuapTylvZTJeN+qFYlDHEuc2sOFjRfY9vY2dKXO5yYN7xTOI2ceofv99m0T2lIt5TnlVJYbMcgqkQO5sfdB7Nh61ROvGIgaAQGWTsALDi1gxI8j+HLfl3h5gaBQ4j1jMsPeG1a/+zQSRn0yykrOOsmzJwMPnSWyUJT8/znlZ9O5Px/4k9cVr6MvdxyJqi3Vut1pxmgwYjQYOb/hvCn3rhXx5+FCf9k4DDbXIqZjRkOf+RBWg9RX+UHcbRDSUzzf+xvwseH0dvkfuLwFX1+QIcdLLo7lhZWFyGQyxnw7hm5TnE9t0xS4f+f9PHL6EUC00y0/uRyACZ0nOIz4s+W4IRF/iTWGgGvjr0WOnArPs5zPE40Wh6YeZ8Dx/fhVdjRzMJWZUGFHjquJ0KTE3+zZs+nVqxd+fn6Eh4czduxYTp2yzC2l0WiYMWMGISEh+Pr6ctttt3G5lh7CxYsXGT16NN7e3oSHh/P0009TVWVptP7nn3/o3r07arWaVq1asWDBAqv6zJs3jxYtWuDp6UmfPn3Ys2ePVZmmxKS/J3HH0jvoPKEz45aMIzAxEBAX/Pdtv4/WE/sgCOLm21U7qOS1lJ4OY9uNBeCPU3+YvCGkiD9nZD7BtYg/qYOdPw+LF4t/51Xv7728xBwSIG6em1lQQ7OHTCbj4WMP1+mttP6p9XzX9zsAut3XjaFvD212ZMB/C/SVeqoqq9yq9y4I5oi/IqNI/NmSKasr4i8uII4+MX24rcNtNo3Ucf3jGD5nOFHdHee2ai7oOK4jj5x+hJBu8Y2iS2+sMtL/2f50m+raIkgdoKbrlK7EX2ub0FNOvItjA+6noEp8lnEB9iUpJKnPvDz7OaI6hNom/gRBoCSjxO0kdENx8/jx/OAwmYgZi7y8GHP33S7fw2gwsmrGKo4udj2/0/QD05m0wXG0l04HVWof+q95gevfu950fFqPafxwyw/0jG6Y1KanJ8gFFe0DuwJYeTM6gkwhE0nxysZz7nA1arPKKHotZbkQvNdjWg9mnpqJX7Rf3YVrwGCA3dWp3JzJK1hb6vN47nEySjIc5t+UEOQVRLCXOPBKHoH2IJebZdJr92VNkYaV01Zy8PuDdVf4KkBjS891uL0DE/+eSGhbsxdcyi8pHBz3Nv45Z20Sf7/e/is/3foT18ZfS+LHiUTOibQbgeIuhHcKR9anFzqvwKtW6lMy2Hp5wfn151h8y2K3OhXIlXKUnkoUgnm88irNZf618zn++3EHn6wbOz7YwTe9v8FY5f7nXDPi72reu1RVmZ+xrb1lTAwEFV3ixLwtHFty7MpWrgbCOoTR5d4u9XIEaW7QV+o5tfIURalFTpX3i/Zj+AfDSRiQgMJDQc6xHD5r+xnJE5IZu2BskynjSPsSHzt2TJkMpt2nQl0lRlj/e9i+58/S8UvZ9PImd1fxikDacwUGWjtPS3nhJNWj8E7h3L/jftrf0t7qOssmLuP3O39vUB26dIGO1TyUs04JRo0fCsHLtvOTJk+MCKsHEoNEa2qF+rwV8fdlly/r3VZ3wcPXA69gS7ZTek5hYdZS/v7+og0OzJHeA14YwKBXB1ld96EjDzFsduMRKEPeHEK/J0Tied2T6/go4SPKc8upqBIXlz4e3tDhGfGnLvgmQfcPIep6h8X8PcW1eKVR3FMmDEhgyBtDCIivmwhuKuTnQ1zefQTrRdKrNvFXn9zjEga8MICHjz/s9sjVg98d5IOID0jdlo5BXoERqOz0CiTdW78LymQwcDm0sowYDPQMBKBQU2jam1TFJBAQ13yfp4R//gGlwZ+YgvEA/HrsV5NDe2TXSDrc3gHB6HhhtuHZDbypftNtKlfz+8/nyy5fcmLpCX64/gf2f70fEO0FAgIGRY2Iv623wjrbDjMWaD0d2j4GNfcq3rHQcoqYw7MmrlkEbR+FoK7is659HmDXZDgwC9/qZZSXTHx3izRFrjW2iSGtefIr8ynRigRrp6DeJudiR8SfNC+WlZnztdYk/gI8A0gO74ZKH8KlsnMYDGAoDyKgojs+ai/Td8eApTC6cXKI1xdNSvxt2bKFGTNmsGvXLtavX49er2f48OGU19j9PvHEE/z5558sWbKELVu2kJmZya23mhNyGgwGRo8ejU6nY8eOHSxcuJAFCxbwyiuvmMpcuHCB0aNHM3jwYA4dOsTjjz/O1KlTWbfOrFG7ePFiZs2axauvvsqBAwfo0qULI0aMICfHLEfUFCi+WMy89vPY+/lefMJ9aHFdC8I6hNHh9g54BYmLEblCTtw1cRiDRQNHcLBzUXk1EVvNEaSnw/CWw1Er1JwvPM+xXHETJxmh7C3ea8PZiD9BsFx4/vYbnDplnmjDwsyb56oq8yZCwuJbFrNs4jLnKtVMoK/QMydqDpteaj4bmN6P9GboO0ObJC9Hc4Gnk0RDQzFs9jCm7pnq1o24Tmf2LszXi7NarJ994s/eZk8uk7Nr6i5+ue0X04KvJgITAuk3q59bZA+uFMrLYeZM8cdQy06//+v9bHu3/jn+FCoF1z57rcveTzKZjJvn32z3cxpBTUmAFiMGFDIFUb72iVYfH/NixV6eP1PEX14t4s8gMDd2Lmsfr1sK+Epi6rRpbNbr2V1Hud3APzod991/v8v9tzK/kn2f77MpA1sXfMJ96nSMkOY+b38VHr7ud6KQmtsh0Cz36Sxaj2rNI6cfodVIGytfN8HVqE1fH3EjmZnZaFUy4dixGhHSTijwSM9S+s4HLxxM7NxYU77GutAySJT7kiSEHEFyrqptBJOr5Bz89iBp/ziWWmruKL5YzOlVp60iwtw9/4a0DqHl9S1R+5stlV7BXni1S0DnFWDTgBnoGcj45PH0iulFdlk2ZboylyJp64OI5Ah877iBisDoqzbiT6q3lxe0GNyCm769ya5TS33QbUo3Hkp5iKRrIlGpxP2Fj6KSgjMFaIob5tFTnlNOWXYZ5bnuZ12lnMoGg3OSws0VpvzRMtt7wNhYKApJJHTWRLpO7npF62ZRjz6xjF0wluie0U1WB3eh6EIRv978Kwe+PeD0ZwRBlC6TyWREdI6g9Y2taTe2HV3u6dKINXWMi8bdlHgexcPLfsRLixYQ7il6H/+4MsMuSZ66OZWsfc0rH7U9aIo1LJ+03CS9Ku25ast8gth+MEcWOMKAFwdwzTPX1KtOUh2Cg81jk0QG1jX/So6TNiP+Dj8HS/xAWwAGHZz6FNJ+c6pOSYFJKAVvDPJyqzWPT4QPnsFXZl9uDzlHcyg4a7lhltaotWU+QRwjpWec33TBz1bwCfPBJ9wHr2AvKvTiXOfr4YLzsToY2j0BwY5znQV4icSfjlKqGs+vsEHY+MJGiz1vXh4ojb70VNxPr+hehHiLG2qJ+Ev/ahVv+75dLxUsD18PwtqHuWzvObzoMMd+s+9EE54cTs+He6IM8WfEgVKG7Ekj2t/9DtlSmpHCSjPxp6kUKM0qrZM0u1LQV+pZNmGZySkypzyHFze+xCfbvwIgrGQ4HvpQEgJakFUqzh+9Hu7FuCXjLPYHthDVI4oOt3dwm6N+dK9oYvvG0vqG1gz/cLhJql6vB6NMhyATjVQ+Hj6ixKeHE6kdWj0AHZ4GWTWlY9DU2xEDgG4fQOfXTeSVWggEzMTfhuc2sHrm6vpf/wrg/Ibz5J4QiYSMEtEmGuYdxqVU0RYTHg4BNrjr2jZSaU4OD7de+66btJobT+QQVDKQvDxL3qI5m9KblPhbu3YtkydPpmPHjnTp0oUFCxZw8eJF9u8XGfDi4mK+++47PvzwQ4YMGUKPHj34/vvv2bFjB7t27QLg77//5vjx4/z444907dqVUaNG8cYbbzBv3jx01YlZvvzySxITE5kzZw7t27dn5syZ3H777cydO9dUlw8//JBp06YxZcoUOnTowJdffom3tzfz5893qU0KhXu96rSlWrtSALpyHWWXy8jYk4EgCDYTRzuDb/Z/w9yCwaSGfUFGBvh6+PLCgBf46saviPETVzauRvxJE4RW69jTtazMTFj07w9GI8ybZ05yHRYGKpXZy7T2QqoivwJNYSOE8jQitCVaAhMD8fC7MhF1mfsy+Xf2v1QW2NfRaDe2Hb1n9Db9v2ziMv6Y/McVqF3zgEKhoF27dm7vvzVh0BtMEbRKtbWES0MgGWbkcsiuqFvq017En7NobvnE7OHkipP88MQB8nIFLl+2NvIfXniYXXN3NU3l7MBYZUSblk0VIkkQ6x9rlauxNqSoP3tyn5KEa+0cf3KlnH5P9aPNTW0aVmk3IzIyknfefZeb1Gq75N9u4Aalmnfee4+YmBiX+693qDdPZj/J4NcGu1y/kvSSOvPK6XSg0FVQcSbDlMtRU6XhfOH5OiUfnYE0x/YPuZkXB7zIbe1va/A13QlXozb7DRW9Mp0l/grPF/L3U3/XK7/fzp3mvyV5dEeoGfFXZawyyZFF+Tm30W4ZLBJ/zpBIkmGtNgnk4ePBMwXPcPOCxpXIbGyc+/scv9z4Cxl7zZEdjTX/CoKAvtIs5dNmdBtCH5+Ixi/MoWyrUq405eA8lH3IrXWyBWlDebVH/Hl7i85B3e/v3ihRV2o1/Oc/8Npr0HJwPE9dfsqhHJYzGPbuMJ64+AR+Ua5FDTsDpdIsjdnQNVdT4N9/YcECsxHEx0dcY0q4tOMSez7bQ3y8HF8/f/L8Wzqdb+x/cAyfCB/GzB9Du1vaOf2ZhYMX8rr8dYxVRjqO68jN3zXtXFFVBVsSB7G1UzJlMsfzdHKC6Kh4Ljcde/7WT6Q/wYQ1E9xcy8aBtljLkR+PkLlfXNBI0V+OiL+0NLO95OivR1k3a51V2a73dqXLpPoRuTXJR4n4Ky52PP9eKLzAjT/fyK6AWYB53WmBsAGiXKRHEMhVcPh5OPeNU3V6Z9g7PGcso+Xlp62Iv0l/T+LGL250snWNg59H/8zS8UstjkkRf7HWvrWAtfH42JJj/DLmF5OsH0BRahG7P9lN3sk8G1dwD44vPc6PI3+k8EIh1z53LdP2TkOukKMxivuPME+lKBF4+nO33TPIWxz/qxSlVFZC+u50Fg1dxJk1Z+r45JXDmdVnOLPKXB9JYWx0yOPsmbaHB3s+CJiJv3J1EPHXxjvMuWcPeSfzHNre7GHTi5vY8b79PPJJQ5MYPW80HmGBKPGkfZQK1b9j4Pwil+9lQtl5+Pc2uGBOXRLkVU381Yj4M6xay4fRH1Ka1Ty8mXRlOlJ+TiFzrzjWni88z9vb3mKPejYqFXiqVAw6eorlN+x2qJ5kC93v786438Yhk7uHyRn50UjGfDsGD18P+j3Rz0QoinSFQLv8p3io50NixF/f+TC4HgRb5hr4PRAuLrU+V5EB/94Oy6NhZWvQ2vBOiL8dYkabiD8PYyAgvgMAqf+kWvSf5gZjlZEfrv+BLf/ZAkBGqbjXjPGPMeX3sxXtB+axu6YSIVjm95MQ4RtORLi4IF588E9e3f0IlwP+MksEV5XD2W+h0L6edmPanO3BvdbnBqK42gITXP3N79+/H71ez7Bh5lD4du3aER8fz86dO+nbty87d+4kOTmZiIgIU5kRI0bw0EMPcezYMbp168bOnTstriGVefzxxwHQ6XTs37+f559/3nReLpczbNgwdta0ENWAVqtFWyOcraRE0unVY6gOK5HJZMjlcoxGo4WhXDpuqBV+Yut4SLsQHj72MDKZzOL40ruWcn79eQa8MIANz27grpV3UShvhSDI8fMTF4+1k0YqFAoEQbA6fir/FIdL/iHJowfp6UYMBoEXr30RhUKB0WjEYDBQWgqCIMfbWwDENpVoSlDKlXipvKzqrlSK5QUB9HoZCoVlW+VyOTKZjJwcA4Igx99f4MEHBQ4elHPhAvz5p4AgyAgJETAYBIKC5JSUQG6u0ST9CTBl6xQEQbD6LqW61yYobB13x3Oq2SZbx8H8PLzCvJj872TkcrnN52HvOdW3Tec2nGPTC5uIHxBP3DVxTrWp8EIhcqXc5D1aV5vqOu7uNrn7ORmNRoqLiwmysStzV5v2fbGPvZ/vZdzv4whtH+rWNpWWygEZPj5G0ksuARDtG226plTHgACxX+bnOx4j9FV6dmXsok9MH5RypUWbNr+8mYPfHOTR1EdR+6ibvD85Or5j7m4yd2TBiG4IgkBRkdHkRKBQKBizYAxyleW9XXn31sxcQ8HpAu5efbeFdJ0zbdrz6R5SN6Uybtk40+RvNBopuliE+vuviG8XBF0gzj/O5vhW892Li5Nx4ICMtDSZzXcv2k/0gL9cdtlqfhr27jCLMbQpnpOtNj36+OMYBYGBzz3HtTIV91VpiACyge89PNmq1zNyxJs8+thj6PV6ioqKCAoKQqlUOv3ueYd516tNax5dw8k/TvKi9kVkcpnNNmm1MvzzLrB98u9E/HIrHcZ1YF/GPgYsGECLwBace+Rcg8Y9Dw+xz3f0vo7p14qJpg0Gg1PPSTAKHJx/EN8IX9rd3K5Bz8le3afcdx9vvPYauwFH2QylqM0/Jkxh3jyBjAzr791Wm9K2pbFzzk5iescQ3jncqbqLcy7s2CGuLwCKisQ1jaM2aTQgCDJUKoGskiwEBBQyBYEegU6t91oFibuMswVn6+xnXl7icy0rM1hEKMvlcjwDPa/6OTfu2jhu/OZGwpLDTNesqqqiqKiIwMBA5HK5W9okGAU+CP2AFoNbMG7pONPxykoZIEelEte6Et7b8R5lujImJk+kTUgbukR0YW/mXg5mHeSOjnc02rgnl8sp/X4pUfnBlJUNMtWpqZ+TK20qLTUiCHLUaoEqvRGlSunWd89oNHLwu4MoPZV0mdSlWcxPzrYpKAhKSmTk5RmJi2s+ayNn2vTtt5CfL+PsWXG89PMzry8EQWDNY2vIO5FH30/DUOYruZgWbRqzmqJN+ko96x5ZR1TPKHo8aBmZcjX1J6PRiDpQTed7OpvOOfPuxV0bh2AU0FfoUfmqmrxNOQUajHLRMTfC389qnV2zTQlB4hpVo8ogL89IaKj1c0KGxTWaw3OqXUepTT7RPjxX9pzpXH6+EUGQERAgWI3xERECCoWcykrIyhKIjpZzZvUZzq8/z6A3B6FUK93Spvx8GSAjOBiKisQxu6BAQK832N3/Xii6wKozq/D1Pk9rYQ5KpRGDoda7lzBR/DEaxeOD1yN4xZokVhw9J7VSjafaiCBAebl47aYe92rWse+svqj9LPe5ubkyBEFGWJj1fksmkxESIq4nc3PF9hScKyD7UDaC0bzPyjyQydrH1jJ24VhC24U2SptKMkpI25pGaWYp/vFmh4xrDS/hc34yXTu3QMh9EsGvLUKNvYO9/iRsGQtGLcaBf0HRUWQB7ZErPSzq/ljvRziy8F4UFdGUlRnQlmrJOphFWU6ZlT3Cnc/J3nFbbZq6bypClfi3wWAg87KelLhZBMhbo9U/jFqlxmAwEBEh2ksuRPblzZ/6IZe79u4JgsDXPb8mtl8sE9ZOcKlNY38Ya4pGc9SmsjIBo1GGtyoHIfdfZKF96z9GCArk6SvBtyUI4zEajfipRM+lwspC1GqxDpWRCfSZXoVREO3DTb3XUAepeb7ieQSjuC7JKRc9R9T6CHr3NnLpEly8GERWlkBkpFjHtK1pHF9ynH5P9CO4VfAVe/cEQeDIwiN0HNcRpY+ZgtFoQG5U06vofT4dKbZVanOd456mEPm2sQhh1yIkv4FcFQgxYzD6tbeQuVIoFAhKf0hfgUyoQgjsgtEoQ4H1Owbg6yt+B0p9IKigoKIAo9HIlH+ngNz2XNwc1rCGKgM3fn0j/nH+GAwG0otFT41o32iOHROfQcuWRqu1qtFoJCBAMNlIDQaBc+fEftyihXnertmm8HAZqek6vjj6NifLd5Hk50lIyA1i2dILKPZMQ+j4AgR2ttmm2u2/Emg2xJ/RaOTxxx+nf//+dOok6ixnZ2fj4eFBoOSaVI2IiAiyq7MgZ2dnW5B+0nnpnKMyJSUlVFZWUlhYWD3IW5c5efKkzfrOnj2b1157zer48ePH8at28QwODiY+Pp709HQKamjrRUZGEhkZSWpqKqU19F/i4uIICQnhzJkzaDQacnblUHahjAEzBxAWF8bx48fNRqNYGa1GtyKmXwytJreiLLyM41svUloajJ+fHxqN1iJfokKhIDk5mdLSUs5LFDaivEOlXvRGqdIouHChlH37UgkN9aVly5bk5OSQnZ3NiRMBlJYGo9MZgSDOpJ5hwuoJyGVyPurzEUmxSRZtEqVtEvH09EKr9SAjQ2yThKSkJPz9/dm37zylpeH4+upITc1g9OgO/PabipQU8XuprCwkJaWI4ODOnD8vsH//BVSqsjrb1K5dOwoLC7l06ZLpuJ+fn0WbJDT0OdVuU83nBNC2bVs8PDxISUmxeFeSk5PR6XROP6f6tknZVcnAHweSq8hFnaO2alPa8jQu/HCBO367gxLfEjQaDd0/7Y5MLqO0tLRZtsndz0kiPqTjjdEmo8GIXqcnvSydrJQst7ZJq20F+KLXF3IhV4xN1xfo0Wg0Fs8pN9eD0tIYCgrsjxElJSV0+aYLaeVpzL92Pv1i+lm0qUhThFecF6cPnSa5f3Kz7k+5PbpyVnsdVVVVVFRUsH9/FsXFGtNzkgXLuHjpIhSZn5Mr715JbgkV+RVcTL/ocptObjlJ+tp0Du48SKeenUxt0hZoyW7fm8zAfEbEjKNfi44WbbX17hkMvpSXR3Lxoq/Ndy8pNol5Q+ah1ChN12pOz8lefxp9441MmDiR2279gicO/YqHuoigIB/adb+dQcWzaNXal5SUFARBoKCggFatWpGQkOBUm3RFOoI9g2nRtQVnU8+61Ca/fn60j2nPkUNHkKvkNtt0+XI8Cr9w2j0ygGKvYlJSUtibJcpxhnmHNXjcq6oS+3xa2mUCAswu8s48p6qqKtY+upbQXqG0GNGi0cbyp555hps+/JA/tVqb5N9u4EaVikceeQSlshCdLoSsLE8uXEilrMzxuyd0FLh3371Etop06d07dkxPaqrZaUv0tAxw2KbUVA2lpb5kZxew/7TovRfmFcbxY+a5wlF/kiL+Dl86bKqTveekULQH1Jw4kYanp9lDvG3btpSnl3NoyyFCe5rz1l2Nc66ip4LUvFTahojP6ejRoxQUFBAcHIxMJnNbm9re3BZ1tNr0nR9+8zA5uiRgAGVl+aSkmKMOv93/LeeLz9PRqyPaTC1hBtFtc1/6PoBGHfc0J1NRy8s5ezaLlJTcZvOcnG3TsWMZlJaGUV5Ywtve39N7Zm+uee0at757m17dhDpITdSIKCIjI9kybwt6mZ6IARENatPhg4fJ2piFyk9F+DXhbu9POh2Ulnpx4EAuYWEezX7Ordmm9HQFVVUy/v0XFAol/v4+5j3hvBMYvY3cuPJGjn62j647j7PNOJUDB/JQqYQmaZMcOYcWHqK8qByP/mZFlautP9X33Qu5PYQWU1qg9leTnZ3d5G06kV4k/iHIuXg2lfRqKTJbbfKo9MCnKhq54MHFi6Xo9WbdS+k5ndp+igt7LhA5MBK5Sn5VPaeMjApKSw2UlRWRklJo9e55e8eQkeHB3r2V3HxzJPG3xxNxVwQnTp8wtUl/Xs/v9/xO24faEj002uU2nT+fgELhT0AA5OeLtpczZyo5elTcj3p5eXHmjDmSQ6FQkCmIUTTKynBKS0s4dy6NwkIPx/3JkEj22WxADJuo6zlVVhZQWqri5MkC4uKKTW3aNG8TReeLaD259RV7TrXbFHZjGC1btrToT5cuhaPRBKFWe9lsU3BwJBUVFRw+nEd8fCHyTnLu/OdOAuIDOHnyJBqNBl2Ijv7f9ie8n+i01hht8h/qzwvlL5CTmcMvk38hrE8YEddGoM7pSEzhtXiQz+E21akuUlLqfE7lFZUY9BVcPLSD5FMDxZO3ZHEmrcji3QsxdKDC4MGRIyeJiNAwYtMIACt7hDufU0PG8j2ns0iL+ILsAm8+Kp6EMljJqdOnKC/TU1ragtJSyMiQERfn2runqdSQNCkJ7yhvUlJSXGtTICZbsq02ZSzN4PTG01TclsTe+E85ofDm8WtPEx0dTeq5c/Ub9wQB2u2hbbt2eBiNpKSkkFsqqpvkl+ejVotkSapXOHEP6kjNS0VR2Pz2GpfLRMk4uSaIuLgzXLrkj8EQTmYmKJXic7qw/gJHvjhCu7Ht8I71tvnuFWQXsHTiUqKGRBE3Os4tbcrdk8uO6Ts4teIUvef2NrUpNVWNXt8SlcpDbFNlGRF586nw6khY50mOx72TaXQoOkNxVTQZpJCcPBBdQN/qNqVYtKlUI3Ch7b8IcjF80zMth3btgi3aFJ39IcGla/FrcwmdTqDV+ZeYM2MyoYpQ0tPTiY+P5+LFi816zu0xrQf5+fmkpKSQeikVP5UfPkY/jhwBnU6Lp+d5UlJ0Vs/p8uVCU59PS9Nw+nQEFRUVyGTppKRUWrWpoKqYTZ1uQ1cu7iW9tUlotemkpBQgN5TjE/cpMZH9UVX3p9ptqvmdXCnIhGai2fbQQw+xZs0atm3bRmx1/PzPP//MlClTLCLrAHr37s3gwYN59913eeCBB0hLS7PI11dRUYGPjw+rV69m1KhRtGnThilTplhE9K1evZrRo0dTUVFBYWEhMTEx7Nixg379+pnKPPPMM2zZsoXdu63FxmxF/MXFxZGbm2vymmqoN9qf0/7k8PeHeTrvabxDvOtkxr//XsYff8gYOxbuv99575lpf03j+0Pf0yX3bWJTn2PuXCNJSaAz6vh6/9ccyj7EkNLvWLxYxqhRAjNmyNmfsZ/BiwZTqiulX2w/Vt29iiDvIIs63nabnKoqWLBARlCQbbZ/7VoDn30mp3t3gVdfFdBq5UybZpbfeuwxI0OGwGefydmwASZMEBg3zvxd5h7N5dL2S3S4swOegWYNiubs3Xlu3TkuH7lMz+k9bXrwX2nvmQPfHmDfvH3c/efd+MX61atNdR1vao+gutpkMBg4duwYycnJVnkO3dkmQ5XBQrrXXW3au1fOW2/JaNXKyHvvG8gtz8VP7Yev2tei7gUFMGWKHLkcli8HsN2micsm8vPRn3n6mqeZPWR2s3lO0vGabbJ3vLISpk6VUV4uR6kU0Ovh2WeNXHONue66Ch2lWaX4RPig8lJd0TYZdAbkKrmpbM2633uvjKIiGZ98IiYUruvdO30ann5aTlCQjIULXXtOm1/dTOa+TO5aeVeD2+ToedR13FF/mjlTIDUV/vMfI926wb598OabCpKSBD780Gjqv506dUKlUtlsq2AQ2P/1fpInJePh48H+r/azZsYaJv49kRZDWri9TZMnyygqkvPxxwIJCeLx7w5+x/RV0xndejQr71rZoP703nsyduzQ/UPLAAEAAElEQVSQM3WqgX5D8tmduZtw73B6xfRy6jld2HQB30hfIjpFON0mqY6ujHuffPwxzz3zDP1lKu6vEbW5yNOTLXo9b7/zDo8/8QQGA4wbJ8dolPHttwZCzdyWW9+9hQvh99+hWzeBgwfFwXjZMjGaz16b3nkHduyQMW2aEUPrFdy25Db6xPRh+5TtVnW09fy2X9rOwAUDaRPchuMPH3fYptmz5ezeLWP6dAOjRlnWfckdSzix9ATPVzyPQqVw2NbmPufWrLter+fYsWN07NgRhULRKG3Slet4L+A95F07sSf2NqZONXLjjWYvXL93/dBUaTgz4wyJQYnsTN/JgAUDiPKNIvPJzEYd99auNvDZ53J69RJ46aWrL+JvxQoj334r55rOpbRI+YtWI1vR6+Febn33MnZn4BvlS2CCGBX6cdLHqLxVTD88vUFtMlQZeNf/XeIHxjN+9Xi3v3sffgj//CNj8mQjt97aPNZGzrRJozFy++2Wa+FevWS8/LJY9y+Tv0Qml/HgkQe5uO0ir009S07rIXzyiUCLFq61yWCAI0fk5OXJyM83UlQkSqPKZPDgg2KEkrNtqqqoQuWjcuods3e8qfuT0WhkzSNrOP/3eabtn4ZXgNdVM5bXrOPqvScYsy4ZtSGY8lct9TttvXuzZ8vYvVvO9OlGRo2ybtOaR9ew59M9PJr2KP4x/s3iOdlrk7ZES+7xXIKSgvCP8mf2bCPbt4triBtvNJeX6j53rox//pExfrzA+PGWbco7kYdXsBf5J/NZPXM1Q94eYpLnd7ZNOp24vpLJ4Jdf4NQpI6++Kic+XuCjj6rs7n/n7JrDsxueJSZvEl0vLGTpUiNKZY26l55Hvv9hjC2nQ+xY8bjBgFCRDko/8Ahw+JwEBHq8exvniy7wTttNTL830FT+l5t/4ezqs7ygeaHRnlPt4870p9dfl7F/v4zHH5cxZIh1m5YulbNwocCQIQKPPWYZJVKSXYJPmDlZ1JVoU97pPOa1nUe/J/sx9N2hPP20jDNn5LzwgpHevesxRmjzkK/riawyHQauxBB1g8U9H3hATk6OjHffNdC2beO0qSHj3qUdl/AM8CSycyQGg4ExT61ideDNtPbrQnigL3sy9rDzvp10jezKgw/KKD6Rxdi2JxjxXDcCkwKdqrs72lSlqcLD2wMpwr5mm/6Y9Acpv6SQ+Nct3Lu3C2pDMKUvZ6NSqdw67uWU5xA9V3QyOD1Bz6wnFAQFCXz/vbk+Tb3X0JZqyUnJITAxkICYAF7f/Db/+fcl4nLv49jsr/n1VxkrVhrZ0ScJvaKQczPP4Wf0Q1euwyfUB4VKYbPu+WfymdduHv2e6seQt4e4pU3lOeVsfmkzQ94cgne4OXfW0aPw3EtaghNymDvbmxCFgOKPSIxJ05D1+cq1MUImA5ms3u+e7PjbyLLWkpa4mplP+BMYKLBwoXkcK7pQRMHZAuIHxptyVzaHOddRmwCOHDXwykse+PkJLFpkRJrqaj+nCRNklJXJeOcdgeefFyP+fvzRaJLtr9mmP/4QuG93H0p8DgDQ+/Qa3px8PUOGONemwsJCQkJCKC4uxl+SI2tkNIuIv5kzZ/LXX3+xdetWE+kHIoOs0+lMEkASLl++TGRkpKnMnj17LK53uTpBXM0y0rGaZfz9/fHy8kKhUKBQKGyWka5RG2q1GrWNJCHStWpCeui2yjo6PvStoXSZ1AXvEG+b5ctzyvEO8zYdr6gQN0r+/uJvW9eXyWRWxyv0YjKZEH9vZDIZWVkKWreGzOJMnvz7SQyCgajQh0DWix3GT5momUyPmB5suGcDI34cwc70nby17S0+GP6BxbU9PcWcJVqt/bYWFyuQySA0VIZCIeYIueMO+PZb8XxkpAKFAkLEXLsUForlJJxZdYZNL24itm8sPiGWmTftfe+uHq/rObl6/MyqM+z7fB/dpnSz+TzA9nOqT92l41WaKnRlOpOedM1r95rei17Te1l8Tl+p59jiY/hG+dJqRCu3fAfublND6mLruEwms1tHe9exVT77cDaeAZ4EJARYbaIUStvXbmibpNxA/v5yVEo50QHRNsuHhIBcLjp2FRdDcLDtNt3U9iZ+Pvozq8+s5r3r33NYx6buT/aOb98uUFlSRUy8nKgoGfv2QUWFwmL8OPjtQdY+tpbJWyaTMDChzrq7s00KL+tzUnmtVhzDvbycG8sTE8VyRUVQVibH1vrBXt3zT+aTuScTuUxuoWN/pZ5TTdjrf2VlMmQyCAgQn5+kSFRaai5fk0C11da07WmsmbkGfbme/s/0J7pHNNe+cC2hbUMbpU366tRiarW5jvmVogd0mE9Yg/uTlAtOr1fw0Z6PmL1tNvd3u58+cX2cqmOr61vZPO6oTRJcGcufmDWLu8eP5647v+XJAz/j6VlMWFgAY8ePZ9HUqaY1lkIBUVGQkQHZ2QpqCTBY1EUQBC5tv0RE5wjU/mqX6r57t9hXrr9extGjYh6ikhJpDWK7TVVV4me8vRWcKhM94GP9Y53+DnrF9CL1sVSb+TqlaxzNOconuz8hwutFIAGdznKsAug6uSuJgxORy+RW975a5txfxvxCaWYpD+x7wOK4VNean3Vnm7z8vXi+7HneeUMPx0RJVekyueW5aKo0yJARHxSPQqGga1RXZMjIKsvictllInxtvJD1/A5qwz9QXAeXl1uub6+WObegQKx/eJI/498e77B8fdsUf028xfFxv41DW6J1qh84Oq5UKRn3+zgC4gMa5d0LCZHmZgVSsaZeG9WEvTZpNCJJIJOJuU21WjFfoVT3h48+jKZYg1wuJ65/HIa+3sgKZGRlyWnZ0rW6LF8OCxdK/1nPc7NmOd8mhZ/tcnD19CeFQoF3sDdewV54+osOrc1xLK/reG6pGLGuJtCpd0/Kr1NSYh6ba6LzxM7E9o3FO8jb4nPNsT/lHM5h4eCFjPpsFL1n9KaoSOxPISGW87pU95YtYcsWuHRJZnG8Iq+ChYMWEt0jmglrJzDj+Ix61bG42NyXfXzEeshkUFwsM82/tt6xzFJxveOpj0GplKFW1/retVmQuw1F3G1IDZNnroB/b4N+iyBxklVba+NC1U5KvXPIrExDoQgxHR/+wXAGvz7YZCR2tq3OHHemPwmCwPz+80kcmsiQN8yWXINB/C5VKtttEsd8mZW96tSfp1h611LGLRlH4tBE5EpzuxqjTQa9gbPrz6IOUPPouUdReoppO47LFpMXqEQm64Ei+ygEdgYf8/xaZ3/yjoDh2yB/H4T0srjnqbxTHA35lUohAp3uQYxaPWfXnSUoMYjIrpGW16lHm+qqozPHfxv7GxFdIrh3070oFAoytecAaB3SmsKqDPRGPeeLztMjpgfx8XBubw5n5m+n563xhLQOsXl9d7dp7eNr2f3xbp7MehLfSOt8ybf+dCtjF43lm1UHAYiWq5Bnr4aQnii8Y+pfx5ytooEo4joUCgWhPqG8f/37BHkGofYSABlC9mWWT9hGjwd6kDgk0ek2NeS4o7oXnCpg4cCFDHtvGP2f7s+FXNGW7yePwM9PQUwMyJBTbihAZyinrKqM8KBwvAK8HF4/pHUIL2pexKAzODXfOHPcP8rfZu5dgwGKffexNuw69i1sw6mHj8KoQ8hVfuDKGHHpD/j3Fhi2BUX4QKuyMpkMhb4Q9jwAnmHQ+yvruie/DMkv45OLaV8il5uqwe6Pd7Pn0z3MypiFX7RlsEhzWMNm7stk6filXPfqdXSe0Nl0/OgRsUy3bjKUNmyx0ncQEiLaVPfsERscHS0jMNC6vEKhIDoaWme9xP5WtwJixF9ERPUcbzRAjf2+vbpfadh+S68QBEFg5syZLF++nE2bNpGYmGhxvkePHqhUKjZu3Gg6durUKS5evGiKzOvXrx8pKSnk1MgGvX79evz9/enQoYOpTM1rSGWka3h4eNCjRw+LMkajkY0bN1pEAF5p+EX5kTg40e75DyI+4HX56whGkdGuTjNoYqWdRWWVGL4aFiSSQhnV6kctg1sysfNEAH4pfIId7frzc8Hj9P62NzqDjt4xvfn6xq8BWH5yuZUHhJQI1lEkqxQpHFJjLh01SjQAenpCfPVapHayZAkd7+zIpPWTCG4V7EKLmxbXvXIdU3dPNZFwjQ1BEHg36F3+mPyH05+RyWSsnLqS/V/ub7yKNTP4udpx7GDVg6v4usfXpv8Fo8D3A79n54e284W6A2Wi+q0pGa89yOXmpO6FhfbLjWg5AoVMwbHcY6QWpVqd3/XxLnZ9tKtedb1SuHxRR/c1b5N0dCUBAeKxGgoEAMT0jqHvrL74RPhYX6AOZB3IYtdHuyjJKKlX/SoLKzm3/hxFaUUWx/d8toeETQuQa7NQezqn/+3piYkouXjRdpnNFzbz7YFvrZ7nbb/extO5T7steXVjQHpuEqEp/S6p8dXX1X/9Y/2J7BZp2kTF9oll6FtDCYgPcLk+x38/znf9vuPykct2y2i1EHd0DX/dMh+jQfT0knIPhHmH2f2cs5D8jrRa6BMjkn3bLm5z6Rr6Cn2D6+EMIiMj6dX7JXoNOs7ivzLYe/w4L770kpVjVXS1v0JmpuPrlaSX8P2A79n4wkbHBWtBEMzX7tAB/PwFKlXpJoUBe5DEHdRquFR8CRCJP2fhqfQkITDBivSTkFueS/IXyXxz4Bsy5OIzrKiwLtdmdBt6PdwLpbpZ+OzVC75RvvhFWfdVd82/NXF+43mW37OcwvPiZOfh44FeJY71HmYlQC4Wi4NmpG8kHgrxhK+HLxM7T+S+rveZ1siNBSEvj8CsE5QVX/l8D+6ApG5ki6x3J8oul1F2WVzsRPeMNhmdGoo2o9sQkdw4lZecVBytt5ojJGcyb2+YOlU0+rRvbz4vk8vwCjIbzuLjRXNCerpr9xEEkAR7OnWCkSPhrrvgnnvEY5s3Qw0VpzqRfzqftK1prlWiGWLIm0OYtmdakxiG3IWcUvGl95ZZ546zhbr2JjG9Y0gen4zaz9rpurkhKCmI6z+4nvj+ohFDsoN429n2J1T7HV64YHncO9Sb7tO60+fxPg16F2raWmSyms5zotHZ3vybUSoahDz10SabjgXCB8Ad5dDyPvOxoG7Qegb4trLxAWtEqJMAyKy0bHxI6xAiu0Q2WR8QDALFF4spzym3OC7Kw4vEny3Ys1dFdonEK8SLkDYhbHxhI28o36D4Yh2LzwZAMAr8NOon9nyyh6CkIJORfoPvFPa3uh2ldhtsuQkyVzt3wYIDcOID0OSATwLE3wZelmv4MwVn2OPzHy6GfktlpbjP/e3W3zi04JCbW1d/DJ09lN4zewNivyyUifK2HSJa0yrYnI8bRBtkUWR7gv/ziMvrjRPLT7BwyEKyDma5XMfontEkT0h2WEaukFNcKW4U+nrLUPw7Fi5vcvleFtg1BfY/ZvpXpVDx1DVPcX/3+/HzFl94Q1klxxYfI+9UXsPu5SYExAUwfM5wWgxqAUBGUfU+20tc00nrUpUhEBDzFeor9eSeyKU8t7z25SygUCnw8PFwWMYd0OvBIBfr4qPyAbkKgrqAb5JrFzJUgF8bUIfbL1N2HtKXw9mv7ZfBbE/M9zjAM38/zzf7vwGg4x0dGf3laDz8Gv97qQ+MBiNKTyVypSXFdVDkyOne3fHnpfF7Z7XJtmbUcm1ERkJk0c3El9xBVPEYfLQtCZPMO0dehMU+UHbB/gWaAE1qPZgxYwY///wzK1aswM/Pz6SNGxAQgJeXFwEBAdx///3MmjWL4OBg/P39eeSRR+jXrx99+/YFYPjw4XTo0IFJkybx3nvvkZ2dzUsvvcSMGTNMEXkPPvggn332Gc888wz33XcfmzZt4rfffmPVqlWmusyaNYt7772Xnj170rt3bz766CPKy8uZMmWKS22yx1K7CoPeQHlOOX5RfnYNstc8fQ3eYd6m85Lx32XirzrHX0SwF+ex3Li9OOBFfjjyA6mGHeALark3L1z7gsk4Mqr1KDwUHpwvPM/JvJO0DzPvDGsaJu0hXwyAMHU0EA0yc+aIn5MM9hIxWHshFdwymOCWVw/pB+Ab4YtvRB0MjRshk8noPq07QUm2N19bXt9CROcI2o1tZzqm9FQyftV4QtuF2vzMfxsUCgUta7opNwC9ZvaiMr8SROcoSrNKKThTUG+CyBlIfb9IfZg7lrxFt8huPD/geZtlg4LEjXVBAdhrcpBXEP3j+7M1bSurTq9iRm9LL9MD3xxAX6Gn7+N93dkMt6K4yEh+TGc6dIjFwwZRBBDbN5bYvs4b8Gvi7LqzbHphE3HXxOEf43qI/uUjl/lx+I+M/HgkfR41Z0ArzqrAqzSH/S0nE/rJen4b9xu3d7i9zuvFx8Ply5CWJhrQauO1La+xJW0Lv9z2Cy0CW5iON3fDkk5n3mhLC1FpjtNqxXMeHnX336DEIKYfmO6WOmlLtRRfLEZbantyMxrFSDK5QY++XItcIS5AcyvEfAnhPg4W5U6i5vw6usV1yGVyTuWf4mLxReID4h1/GFg2YRkpP6fwsv5lqwVyY6AuYwk4T/wpPZUMfWcoMb1te7bag1YrPhuA3KrzrI27g1LhMnmF52iJ/U1MTeJvcPRgDIKB6xKuc+ne9mAwGhi/TIySahPShj6+N/EXtom//wbc9NVNVsfcOf/WRNGFIo78cITk8clijt1yPdqKMEBBTcEOifir3W8W3bLI7XWyhYw/9tFq/25S4x8HXHdEaGpIxJ/hwGHWb8vhupevw8PXvUaB3OO5fN7xc/o/159rnrxGjPT1cM9+C0RVDJlc5tZrgtnAXlTk1ss2OqQ1pY+PSMYNHGgmLSoLK7l85DLhHcPxDhWjr7p2DeXwYdeJv2PHxPfHywv+8x8s+mVamhgFNX8+vPGG2ePcEf5+8m/O/X2OFzUvNvu1jauQotMDA8FOcEGzQl5ZEQA+isA6y+oNet7OGsKF5Ay6Fh4CrozsVWMhID6Aa568xvS/pABhb/3TooX4OzNTWtOazw2bPQyA40uPU5FXQY8Herj8bku2E2k88qsOJBEEKC+3P/+aIv500dgQuRIhk4GsRsN8E6HXZ07XLcozkTOVu8jWWjL8Bp0BXZkOdYDatIa+kpAr5cxKn2V1XHqWHnamOHv2qoD4AB45/QhKTyURyRF0uL0DXsFe1hdwE5RqJWPmj0Htr6YkowS/KD+MGDHIRZuf3L8LxC2AUCf38Tlb4ODT4k+Xt6Hj8yAYQWZ+Nn4e4uasSlGKRgPeId7c+tOthLZvPrakHg/0MP2dlwflnmbiz69EZLdrEn8GlSdZWk9ULj6q8svlZB3IEm1BLqLzxM50ntjZ5jlBEDi14hSh7UMp04obhTOGYOj7DoReY/MzTqPru6CwxfCbgzlKgxN4puwlvHzcu1aqL/yi/eg3yxykk12d4y/SX9xnR0WJxxX6QFBkUKQpImt/Ft8P+J4Rc0fYtWPlHMtBU6ghpneM29eFtaHXQ5Wimvjz8IGqSjBqQBVg0b/qRIvx4o8jBHWB5NchoIPt80UpkL4Sz/g7kctbUep5gg92vcPQxKFM6zGN+Gvjib+2bhtDUyG2TywPHXnI9P91C65DhReytG/xJJZu3Rx/Xhq/pX2NI+IvIkKMJu18erHpmClNiW9L0TFGbX/scxdn5AqadOn4xRdfUFxczKBBg4iKijL9LF5s/gLnzp3LjTfeyG233cbAgQOJjIxk2bJlpvMKhYK//voLhUJBv379mDhxIvfccw+vv/66qUxiYiKrVq1i/fr1dOnShTlz5vDtt98yYsQIU5k777yTDz74gFdeeYWuXbty6NAh1q5dS4SLLqy1NVzri9xjucyNncvWN7faLXP9e9fT/+n+pv/rG/Enl8nxUHgQGSrOalLEH4ih7zN6iUb/2Lx7+anPGe7teq/pvK+HL0MSRQmEP0//aXFdVyL+gmtxd35+WOT4sedBBeIkaNBdPZ7SpVmlVGmqrug9R30yyubkpq/U88+r/3B8yXGrc61GtCIwIfAK1K7pYTQayc7Odkv/TR6fTMpPKfxy0y8A+Mf4MytzFkPfHtrga9uDifhTnWDJ8SWsO7fOblmpL9XlgX5DK1G3f3PqZqtzdyy9w0KqrTmirMqLC91uIe7m7qYxsTbx1xD0mNaDe/+5l7CO9YveCusQxugvRpM4tFak+5ODODTiGUr9sxEQ8Fc7ZwCRPIYXLYJvvoEaQfCAmWySos4kFKUVcWLZiTo935oKUrSfXG42PHp7m1SFKCmpu/8KgsCXXb/k76f+Nh1bNWMVv932W73q1G1KN2ZlzDJ5c9eGRHKldRnD1P3mBag7I/5qzq+BnoH0jhE9WNefW+/U5+P6x9F5UmeqtFdmLpK+E3vGEnCe+PMJ8+HaZ691qIhgC1IUi1wOLUKiqVRmovFI5/fTPzj8XE3ib2SrkXww/ANuamtNYDnCzyk/c9fvd/H78d9NxwRB4PmNz7Ph/Aa8Vd4svWMpIb5if6+0EWSWvjudzzt9ztFfj7p07+YOd86/NdHp7k48V/IcrUa2YuecnXzV7St0xeIXW9OIealEjOJ0hjBvDHS8K5nzXW+hROdJ88i67jwEwbxBLtp1kp0f7EShdv9mNrh1MN2ndSfumjjWPLqG2f6z0ZXr3HLtlJ9TeMv7Lc79fc4t16sJydBua+/SnCGNlT7VYgg1I5UydmewcNBCTiw7AYj919s7H0EQXCb+NmwQfw8YgBWxMGkSKJVw+LDZW7sudH+gOyM/HmlSwrkaoSvTsfGFjVzYJHqKHz0K994Lt9wi/n7uuSauoJMIF5Jpk/Eafb0n1llWpVCRpkmhQn2BjJIMm2WyD2UzN24uez7bY/N8c0ZdZFFQkGj3EAT7qh1HfznKqgdX1YvQrm1rkcvNyhkFBfbn3xKtuGmyG/GX9TdkNyzKKNZHXMfl6i0jIza9vIn3Qt6jOK3xouLqg7rWstJ3XFZmLitB6SnGW3Sd3JVxS8a53UGmNrpN6ca5deeYGzuXivwKyvXmfZ5fYGtIuhf8HVi2ayJ+HFzzszkK6c+2sOl6iyJ+6mriT15KZaXY3uTxyUR1i3JLe9yNvDwoV4vEX+uQ1rQObg2IkYsAcXGA0UjGyRIqC+pWfsg+lI22RNww9HywJ88VPUdkN9spo+oLTZGGxbcsZvt72ynTiHXK0ftibDEJ/BroQBd/O8TcaHHoWM4x/j73N0X6aoUbmRxdVfMg/WxhgvcCrjmxnX4RosNEeLjom6DUi4uxIk0RgS0CGfjyQIfOo7s+2sX3A7636+BrDzqdtf3Fmc9YRPxlrITfg+HSUtcu5AwUalHOM/422+cLD8ORl5CVHMPX1xwpWaQpcn9dGhnaKi1b07ayMW0dcqMnCQnWfENt1FQgBMfEn7e3JecSFFTDuafVNBi8FlT2SRl373mdQZNLfdr6mTx5sqmMp6cn8+bNo6CggPLycpYtW2YlD5WQkMDq1aupqKggNzeXDz74AKXSMphx0KBBHDx4EK1Wy7lz5yzuIWHmzJmkpaWh1WrZvXs3ffr0sSrjTJvcAZWPil4zernEqksGUleJv7UT16J9ScukHuIgkJGBhfHh45EfM6OonK6pC4gPirb6/I2txUniZN5Ji+P1jfizBUkCpLbXbPHFYt5Uv8nGF12T/GpKfNHpC34c+WOT3NtYVSsBqoeCGSdnMOi1QTbLVxZUUlnYuDJXzQGCIJCdne2W/iuTyfCP9cc3ytfiWGPKs0nEX5lCtLrE+NtfzDhriOoULoaNSZ5vNRHaNrRRPRXdAYnkCwgwj4m1pT61pVp+GfMLO+bscPn63qHetLiuRb1lIHzCfOj5YE/CO1pGf0kGf71KfEDBXs5FNA8dCjExYqTQypXwyCOWEln2iL+za8/y222/kX0wu17taGzUnNfMyZgt5T7r6r+6Uh0yuYzjvx/n76f+RhAEilOLKTjbONbYmpv9msbMm9vezNRuU019qyGo7VgzPGk4AOvPO0f89Xq4F7csuuWKyJiAa8Rfhm27X4MhRdGVhP1NTvllrvN4EoCf0t6hymifAJXWMI7qXhf2Z+5n8bHFbL+4HYDssmxG/zya93e8D8DXN35Np/BOqDx16BVFNiP+lGolRr0Rg/7qcXSqiayDWWx8YaOVRJA759+a8PDxMMnCdRjXgUGvD0KjEJkMWxF/cf5xVteo1FeSXda4Y2PLgTEUxHZGJ1M7XC83R5SWmuesOxbfxiNnHkGhcr9RSKFScNPXN9H2prbE9Y8jeUKy28au4FbBtL2pLf6x7o8yulqlPqXxRyL+aiKkbQgjPx5J/ABxfyoIAkql2EfS03GavNZoYLs4HDJsmPX5iAi4sdoGWcPf1yHa3tSWng/2bJIIIXehKK2IbbO3kfpPKgDbtlmu10+ccLyvbi4IqepMm6xXGBoy2anyEdW5qbLKbS8A1AFq/KL9Gp0scQdOLD/B1z2/Jn2XuCerS/FAJhNzdYMY6WoLiUMTGfaujY7iBKTxp6atxTw22Z9/Ux5KYevIUgIqetom/g4+DbunWh8/ORfW9QGDA8/vasT7iURSvtGS+IvtE0v3ad1RejWNOJmuXMfhHw5bSTXWFb3p7W1eKzYHh4+Ww1vS+5HeeId6U6oVN1QyQYG/t4uSud6x0OJuGHMOOjwHIb0hsItFkdoRf80N+ko9n3f8nM2viA7NmTlaKj3E9V/rYGupz9hY8CzPI2n5XLa85zi9SdnlMr7u8TUHv7f0UqkPUZ9/Op+V01ZyfoO1zrXCQ8HYRWPpOrmrKeJPYVS5ff0s4aFVDzHixxFsu7RVXDcLRlK3XiTnqIvsViPhzOozfNXtK9N8WZWfQHD5NbSOFh1slUoICwNlDQLLP9afwa8PJu4a6zW/hC6TujD8w+Eu27o+/1yURz/qgn+mSPyJhjw/tR94x0PL+0XZziuN6FEw8gBEDBaJvyozYQpwZs0ZPmv7GWfXWdsGmwPSd6ez+5PdlOeUk1Umjt0KPFBVhdQp8wmWc6RKZZ6X7aEmJRXmok93Y/VZR7h6V8b/5QhpHcINn93gtKa0INRf6lNCZKToBabRmAk5ECctXbno7mlrEzg+eTwXH7/I/JvnWxyvi/gzGs2L0doMe21I3qZVVeZFF4BXiBetRrS6aiQpBUGgy+QutL+1fd2F3Yw/p//JV92/shho5Ao5oW1DbeZIvLj9Iu+FvMfB+U662v4PbHxhI192+ZIbPr+BMd+OAcR8eI2dc0Tq+6WIG+ZYP/vylc5G/HWP6s5HIz7ivevfszpnNBjJP53fqPkJGorKlLMkHF6JvKjARBLVJv5UXirOrTtH3knXtepLM0vd7lWuLdVy4ItdeJbkoFOIg3CIVx2DYzViY+GLL+C116B1a9Fw95//mD3P7BF/SUOTuPWnWwlPbrj8ZGPAnkOLPTLXFtT+aqYfmG6K/Cm/XM74VeN58PCD9apTZUElKT+ncDnFdo4/nQ4UugqiUndy+bCZNJjRewbfjPmGHtE9bH7OFUhGGGl+vb6l6Hm74fwGjMKV9yKrC64Qf5cvi3ln7OGf1/7hu2u+c8oDtybKy8Eo07EregItPm5Ba78uqKqCydKdZcmxJXXWXeVhZE/GHjJLM11esLcMFj1xzxWe41TeKTp/0Zk1Z9egVqiZd8M8JnSewJwdc7jjYABnot60SfxFdo1k5qmZdJnUxfrkVYCL/15k2+xtlGY60WndhOzD2WTuyyRpaBLXvXwdWp1ohKlJ/L099G0uPHaBJ6950uKzy04sw/ttb+5Yckej1lGtFo0T4Nx41pyQVW0TDQkBb3+lXUl5d6L3jN7c/N3NbrteTO8Y7lpxF5Fd3euZD2bjenm5dfRHc0ZNqc/aCEoMos+jfQhrb7ZyBAfrUSrF+eiy/dS3Fti+XdxvRkdDu3a2yyR0O8u6rsH8Uf6Miy24ehHSJoQZJ2fQY7q4TpBy0N57r/l5ZDdPPy0LOJt7XIKUNzdPa5v4C0oMYuruqXSd3BUQI3VzT+Q2tJqNgqrKKioLKk17hLrIIrCf509Cr4d60f+Z/rZP1gFbTtZSKpW6ZIgVBl/kgsq21Gf3D8Wf2hAEqMwW80nVgYRA0c5ViGXZ9re256avb7KZE/hKoPxyOX/c8wdHf7G04Evj+MmSvby86WX+Sf3H4rxMZl/u80pjw/MbWHn/Sga/MRiZTEapVuyUCoMvfrnfw4oWkOdCBK1BB1XlYiOv+QF6WD57KeLPoCijvELch3zW9rN6q6u4G0a9EUEQTHnXy4rUDD+Uz+M+uwn3CTcRf1llWZTryvH0hKBYXy4n9kGZ4DgtiORIKhF9x5YcM0VtuwpNsYaD3x4kc5+1/ImHjwddJnUhYUACFXpxozApMA/5n0lQ0EB73aHnYFkUVJk3IIGegYBI/Hh6gkwQ+GPM9/z79r8Nu5ebUKWtQluiNY210vqjpmBfVJTrkWsJAxPo90Q/l4hbQYA9e8Tf653zwQUkqU+xb/p6+EJYP+jzrSjLeaWhDoHgbqDyt4j4K9SIRkOFSoFcJW+2qgpn155l7WNrKc8pN6kHeOmjkSGrU+YTLOfIpCTz3swear5nFsTf/ll15lFsCvyP+PsvgUYjkmJQf+JPqTQz17U97iUjlK3E1EFeQcQFWHtNSItEe14/RUXi4CiTmSP67KGmp1lNCSwPHw/u/vNuut/vBI3fDCCTyRgxZ4RFTq8rBQ9fD3wjfNEUmR9IRX4F5bnlNo2Y4R3D6XJPF6uIpP/BPuQqOVXaKrxDxY5SWVjJusfXse/LfY16X2mDXWIUF4nRftaRuRIcyebWRJRfFI/1fYzhLYdbnSs4W8BnbT9r1pI7hosZhF06iKdCbxEdVhNypZwXNS8y5psxrl1bZ2Bu3Fx+v+v3ugs7wKKhi/hhuFlmMCclhwNvrcM/55gpD4OzEX8gjqXdu4v5cBISRHL3lVdEo2OEj7g6qU38BbcKJnl8cpNtruuCPeLP3jN1hAEvDuCx1MfwCbdhzXQBxZeKWTZhGSf/OGnzvFYLXmV5xBz9u94bv7pQe37tE9MHPw8/8ivzOZx9uM7PZx3MYuXUlaTvdlGbrZ5wxvAVFiaer6qydD6qDV2ZjqLUItT+rnksl5dDjv9aNIo8In0j6Rk6iMTLjwOYIu9sQSJXy4x59Pm2D7EfxjqMELSFlkFm4q9FYAse7fMoPaN7sv+B/Tzc62EAIn0j0Rk1FPj+a1Pq82pHjwd68PCxh13OzdgQ/DTyJ9Y8ssb0f03ZVgkeCg9aBLawmjejfEV5qrTixnXcyTuRS6e/5xJxbsdVR/xJRpbI0Coy9mRYrC/djXPrz/Ft32+5uN2OFp4b4G4PXB8f85h3NeX5qy31WReUSmjdWvzu/v67jsLV2Fgt1DJ0qP38fZ+fegG9spBjge87FUFyauUp5nWYZ/L+vxqhUCkIbRtqyh0tvTfh4eZ8RVcD8ZdaeoYSz6PIvZxbpCUEifNCqTy9zmddWVDJsgnL+PfN5mF8ro3k8ck8dv4xU0SJM+ufuiL+GgJbEX+S3aW4Dv9N6VnYjPiLHApxY62Pt3kYxpy1n0eqBloGJSI3eKEw+DZJBIQ9+ET4cOcfd9LlHkvju/Qs/81ezZv/vsl3B78DRFk5Cc7usxsbflF+RPeMRl8hVrqwQjQWKI1+KD28QB0GSicjmioyYbEa1tnPCShF/AGUVIqTSHDrYPzjm0fOTrW/mhnHZzD0LTH1ilYLHoYg2vv3RiaTEeQVxKAWg7ij4x2U6cTvKraNN5c6jkTforXDa8f3j+eF8hfodr/ILqyZucYUWegqIrtG8kz+M3US/V25l+sP5NFNOQHU4Xbz8zkNVaAo5Wo0v8tBXqL3UqGmEE9PEOQKur8wkq5TujbsXm5C+1va8+i5R0kckkhBRSEbjC+RGjbPivjz1bSjlbovod6h6Cv0LBq6yO3kZXa22Waxa5dloIoj6PXmiD9flZOeMo0FQQBdMehLq4k/c8SfIAgkDUvi4aMP03qU4/7QVOh+f3cmb5lMUFKQKUetSiOuLVo7UeWac6QjmU8JNiP+jAY4/QlkrrH5mabE/4g/N8NdycT/nP4nqx5e5XR5aaBRKq3zJNSFcUvGMeaXMVwsvkhstUNLbeLP2U1gTWNYXRF/kmEvKKjuROUKhfl6tjzh/4e6Mfz94UxaPwmvIPMib/cnu/kg/AObIfuegZ6MXTiWViNbXclqNglkMhnBwcEu919tiZY9n+0x5Wwc/NpgZp6ciWAQ2PTSJg4tOMS0vdPo/2z9vDSdhdT/iwyiNSDS177nujukp4JbBjPwlYHNNsGv0QhpCQM5OPwZ4nuEOszxV58xu0pTRa+ZvWh9Q8MWPl4hXhb9MaJzBL3n3cPlePH5KWQKp3P81YSPjxj5FxoqjuXbttmP+GvuqIv4Ky2tu//u/3o/B+cfJCAugMCEQKq0VRxaeIjsw/WzngUlBnHH0jvodJdtyU6dDir8I8gYPpkOt4uGD71Bz7mCcyapnYai9vyqUqj45bZfOD3zNF0ju9b5+bLsMg5+d/CKybVIXtKO1ie1JVztYfj7w3ky80nkSteWsOXlkB66EIAJyRMIDlTSIlfMwXgw+yC55bajB6S65+nFXHCRvpGoFA4seDYgeRKfLzyPSqHipYEvsWfqHjqGdzSVGZgwEIBin/0UV5bZvE7KzykcWnjIpXs3Fyg9lYR1CLOSaHTUfz/Z/QlvbX2r3gbBQa8Pot0t7fg46WP2fbXPJvFnDy0CWwCQUZLhMtHrCjyDPBF8fDEo1SYnnqsFEgERJsvj2z7fsutjx3JYDUXG7gy+v/Z7Cs+7VzuzIq+ChYMXsvF596YNkMmuTrlPR1KfyyYsY35/s8KL1H9vuUX8f9Uq857RHnQ6OF6dWnzgQPvlXhpkTmh3Kq3uL1CulIPAVZX3vTbKssssHDKl9yYw0GxkkiJtryQ++gieftrsZFwXVumeYWunZHaW/exU+YQg0QChUWXY7Sv7v97P7k93o/RS0u3+buSfyUdX1vxDaZ0h/uKrt1P2cvw1BLYi/swpVGzPvzsv7eTGn29k/rnXARvEn9FgX9dX4Qny6sbWMXe3Cm3BqIPljEzbb1GHi9susnT8UjL315H0uZHg4eNBu5vbEd7J0vlZWg+mFOwFoHd0b4o1xQz4fgA7L+0EzBF/jhzYrgS6Tu5KlabK5KRYWC7uP5QGXxQt74aReyEw2bmLeUZAzBjo8Kz4f+522DsTyszOjd4qb2TVpuWiSvFe4/8az8i5I93UIvfCloz/5ns3s/j2xUT4isxRXHVcw6VLdV+vKLWIxWMXc+THI9z6060MfmNwveqlUCnwCvZCJrdeE//79r98nPgxBecKqNKp8DAEU+gzHWHEbghooJpYx+dg+HbwMCs3BHlWE3+VhaYxIP6OPrS8voH5BBsBJ7LSOBn+Fmei3yC8RreNiIDWWS8yXbmTe7vei1wlJ+tgFiUZtjealYWVzImaw9Y3t7p0/9OnzX9XVMChQ859TqeDgIqe9PeczrXx18Kpz2DHJKhqAg/Q8gvweyAcfaNa6jMQAKNgpFTX/D0T/WP9SRiYgMpbRUapSGZ46mOQyZxzZKupQOgM8VeTYA6VxAdlcrgtH3p96fCz7uKMXMH/iD83Q14Xg+UkMvdkupRzSTKO+vvb95y0h3Vn1/Hn6T/RGXTEVDtiZ9ZYZ+l05oW+rYg/gNzyXG746Qbi58abDCS1pchAHATvvht27jR7QtUl8ylBundtT/g9n+1hzWPNj1W3hcx9mSy+ZTGpW1Kv+L1tLSCie0bT/YHuBLYIvOL1aU6Qy+XEx8e73H+3v7edv5/6m6LUIsvrqeQcnH+QY4uPEd0zmsgu7peQqgnJWFigq5v4c8UT8WTeSX49+ispl1MsjsuVcga/Npi2Y5xMCn6FUVYGAjIMHl4EBitMpFFZmfUeNOtAFseXHnfp+mp/NaM+HmWSHKovxv02jtsX327638PXA++OiZQGiGNosJfrZLSEkBBMeubFxfaJv5KMEj6M+bDZ5kl1JuKvrv67bfY2ds7ZiSAIlGaWkr4rnRWTV5DyU4rN8nVB7a+m/a3tCWlte/LS6cCoVGOMSyAgXtRTSitOo9WnrYia454k97Vz/AGMbjOa1iGtnXpnEock8mzhs3S7zwntiwZCEJwzfIF5YV6X4bg+KCgtJydAdKia1HkSAQHgURVKiFEkZ3em77T6jCCYDT25WjE6UpIkcwXxAaJVT1OlMXkh1n5OcQFxxPgkIMgMpBms6wLiu7zt7W0u37854HLKZTTF1qEc9vrvnow9vP3v2yjl9c/x02NaD1oOb4naT41CpbAioHUGHZOWT+LFjS9aeOwDRPhG4KHwwCAYSC9pvMhYvyg/9FOmkZfQ46qL+JOIv/BEb4a8PaRRjUFJw5IY8NIA5Eq52/N8eYV4UZpViq7c/STC1Uj8OZL6lKvkKDzMeRyl/tuvn5y4ONHgtaaOLdmFC6Kcc0CApeGkNrpHdcfPKI6d28/WnTSn9Q2tmXFiBi2HNz+jpLPY+OJGPgj/AG2xOB5JEVlBQWbi70pH/FVWihGaJ086f+9yo/jCh/o6J/8b4ycaIDQe6XajY/d9sY9dc3eh8lLhGeRJ5t5MynMaYbHQQKTvTufg9wdFCTrBLF3uaP0jGaoLCx1LndcHjiL+Skttz7+n8k+x6swqjpWIOdCtiL+Li2FJAGQ5CPFNeQ02DHBI/nl7y5Ahs7LtFF8q5ugvR632100NvR4EBA7lioo3vWN68/qW19mbuZfvD30PmL/npib+KvIqKDhbQFm2OKC38G1Pt/M/0SH7LRSupuKVK+C6FZA4Ufy/5CScmQfFJ0xFZDIZs1ttZ+CxI6h0Lia8ugKoLKhk7+d7TWTyjtJfOBr3KCf19t/h+HhIPLCU9AUbHF77zJozlKSXcHH7RYovFpM0LInEwc6la7KFnGM5Nh0zPfw88AzyxCvYC41G/M7j4sLdZn+uDRPxpym0ue9samTuz+TAdweoLKzkdIb4fXkbIyzI3NqR8gqVgmcLnmX0vNE2r6kt0RKQEODyOvPUKfG39Ci2OblV0+shsuhm7gn6kjs73Ql52yH1R5A3QT5bdSgkTYHgnvj6glzwRIlYjyJNEdpSMeChuaoqaEvNsq+S1KenPhpfX+f4kcBAs7ynPQn6mrAZ8SeTgUcAeDlY3OI+zsgV/I/4czOMRvfk1pl+cDqTt052urxkLHBWS78mJI1oL6WXSfO9ptex5Pkpk9kn/oK9gtmZvpOssiz2ZYqyhrYi/nbuFK+9bJmZeKi5EHUELy/L+kg49/c59n2xr1lJRNhDUWoRp1aeojK/aXS8Cs8X8tMNP3H0V3ED3famttz01U2o/Wy7v59dd5b5185vMo+7KwWj0cjFixdd7r9BLYOI7hlNYItA0nels+nlTRSlFiGTybhv+32M+W4MBn3jeh4LgtlIXqQVc9U5Q/wVFtbpiMmcHXO4e+ndLD2x1B1VvWIoLgafwgyCDHkolWaSqGYuVAnbZm9jybglGKuaPi9aeU45FRUCcqMnnWR3cmObG8UTGavqJRkgGe3KyqB9WHt+vvVnvhvznUUZtZ+agPgAfMIaJn/ZWJAiv+wRf8XFdfffyVsnc8sPt6Ar0/FhzIfs/ng3d628i86TOjeobvaMxFotKPQaPJTm+kiEa5iPezbDDd2AKdVKPAM9r4jHWU25E0c5/qBu4s+gN7Dro12k73KdiNmdswmjXEuwLJHOEZ1N71CPwtn8PfFvhiQOsfpMzfVLjqb+xF/NCMGlx+2Pp32jxPCXDJVtT9Mbv7qRu1bc5fL9mxoGvYGvun3FiskrrM7Z67+vb3mdy+WXOZ53vEHvaWTXSB48/CDtx5tJbml9mlGSwY9HfmTOzjl4KCxfTrlMbiJs04oaV+7TlZylzQmSISW2vT8Dnh9gkrZrDMhkMoa8MYTnS59vsFyzrWs/fOxhbvj0BrdeF8wG9qaWfXMFjlRexi4Yy72b7zX9L/VfQTBye7Uf04oVjnManj0r/m7Vqm5jTKxKjEg5mFE/R52rDUlDk+j5cE/UAWqqqsxr1sDAppP6rCk/6eyao1IQ2aZwv0CnyicEJuBnjEVVFWyXJL/151u5Z+M9VBZW0v+Z/jxx6QmTc1VzwrHfjrHyvpVUFlY6vf4JDBTVjQTBvU4COp35HbJF/BUWCjbn36xSMazUXy7uJ62i5FX+ENILvOynlkBfAroi0Nh/YSXbjk5nSXh2HNeRl7Qv0f7WBkYx1RMXt19ktv9s9n+z33TMYBB/Kj3SyKvMRSVX0SWyC6PbiATCb8d+Q1OlaTZSn0FJQTx1+SkG/WcQAAGKCGIKxpOouRUyVsPx9+sfVRR3O4zNgKgRFoe7hPTFvzIZvVZc8x5acIjt721vSDPchpL0ElbPWM2ZVWcAOFa5ntSIT7lYZZmOxSgYTbng4uPBryANbZr9d1gQBJaNX8aW17bwXNFz9H+2f4NtkouGLuKv6X9ZHe/zSB+mH5iOV5AXB/W/cCR+OukZT2E89l7DI8SKjsGxt6HEHLom5fgr1BSa+uqBl5bx/YDvG3YvN+HkHyf5c+qflGWXca5ae95faRmlK82bzkbKByYEMnXXVPo+bl/W1hakiL+R1QGuzsp9WjnG9v8F7qgQyfYrDZU/9J0PCXeIZBkyngvdzdlHzhLtF42+XM+aR9Zw/HfXHOavFBYOXsgnLT8BxHW9vyoIT12M0/yIUglPPAEzZmARNWoPNok/TS4UpVjkyrQFd3FGruB/xJ+b4U7ySaFyvsPXjPhzBXqDHoMgrrS8Vd4WhmIJ0gbQy8v+Bk0hVzAsaRgAf58TPWdsGSYl76eTJ80bP2eJP3sRfzfPv5lnC59tkpBZV9Hh9g68rH+56SKlZHB+w3mKL9Yh6l8NXamOvBN5lGVdZfpTLkIQBAoKClzuv92mdOO+bfchV8rZOWcn/775L5UF4gsaEB/Ad32/4+cbnJO5qS9qRuReejSXnKdyaBls39tZ8j6vaVCwB+k65wrPWZ1bevdS/pj8R32q3OgoKYGW+38jbt9yQJzIpQVrbcNqrxm9uOP3O1x69rs/2c3vd/7e4HxGGXsy2PL6FkqzxEp92eVLjj21AF9tayb7/Mr8m6vltDJWwtabxcWEC5AWOmVlonPG3cl3c12L6yzKqP3V3L/zfpcXuFcK0jtam/iraUytq/8GxAUQ1T0KtZ+aa56+hk53daLtTW2JSHbsjeUIfz34F+8EvIO+0npVr9NB4sHlhH/3jsnzTJKRlCIvGwp7Utpf7vuSScsn1ZnnTxAELh+5TO4J196p+qCmAbgu4k96Z+0Rf6WZpax7Yh1HF9cd/VEbewpXA5DsNQqZTGZydArKGcP1La8Xk6rXQs26Z5WLWj9x/vUjNx7u+TBeSi+6RdmPsuwfJxJ/lz1tE39x18QR2i7U5rnmDIPOwKD/DKLjXR2tztnqv/sy97HqzCrkMjkvDXip3vc9ueIkn7b+lNR/Ui2epdR/LhaLumpxAXE215AJAQlA4+f58zhxmMiz2646qU/JkBLlnkBmp6D0rH8EqCPIFY2zJa7pbHW1wJUcfzX778CBovGjqAg2bTKX0eksjW5nRLurw5wrK0+t5NsD3xLiLX6BJwqO1FmXKm0V+77cx9m1Z+uueDNF8vhkRs8bjUwmM0W+yeXi3NhUUp+pqea/nSX+tBQBEBHgXMTfyFYjeU59ia6pC+xG/IW1DyMwIZCPEz/mr+l/4R/r77Lk95VAjwd6MH7VeHwjfC3mHaWDoaumLLA7I8WkccfDw9J5W1r/FBbaXj9nl4lEhy/i4G4V8RdzIwzdCIG2Je8B6PwG3HAEvOxPEJ6ecCr6Fba2786PhxabjsuVYmRxU9l2PHw8iOsXh2+EeV0oGeiLfMRovy6RXfBUejKoxSBi/WMp1hbz1+m/TEpWTU38SZC+Qwtpy0tL4NAzINTTOdkjALyjrciJ2ra/wwsPs/3d5kH8BSUFcc/Ge+g8UXT6LDOIHS1QbVZvWXFyBd5veXPLYlG7OjYWjgx9guPdJ9p1zBKMAjd9exP9n+uPwkPB8d+P87b325z+67TtDziBgS8NpOfDPR2WuSBs5WL418QbViE//CwIDZSkLz4Kh1+EIvP+UcrxV6QpMq2bjYLMpopYU6Dr5K5MWDOBgPgALhWIDrYhasu9fXQ05PmvZ1lCPEO/F4nqs2vPcmbNGbfVo6oKzp8X/x4zRlz3OSv3qdOBVpmLTlFgTivgbO7NRoS0Hw/SdqVlcEuUciVeIV7cu/le+j3Zr2krZwctR7Sk/W2is8h717/H30MLSLr8pJUNyREGDjSTt3UhNFQc8+TyGiRg+gpY3Rku/+Pws00RsNT8Vkv/A7kncjn++3ETgeAM6hvxJ0X7AXipvGx620sRdvai/SQMTxoOmIk/W4bJmougLVvE385KfdqL+PMO9bbKGdOcIZPLmmyjEtgikOeKn6P/M/0RjAKLhi5i51zbkmIgEpXP5D9DmxvbXMFaXn34ceSPnPv7HFP+nWLKB1ClqaLvrL50Gu9gU+QGSH1VLgdvLzlhPmEOpdGUSrMxp66k7i2Dqom/Amvir/hiMSWXHCTiakIUF0Nmm+sw9upjOmYvz1+LQS1of2t7lxwtsg9lc2L5CVQ+ruX5qo2cYzn88+o/pG1Nw1hlpMMdHfDqIvY1r5prvsjrIXIEVLkmadSYsolXCvYi/pzNoVGUVmSSuQG4/r3r6XinNflgDwajAYPRemMc2S2STnd2QldqHdag00FpSDxVHTubNke5FSLBFubduBF/y08u58cjP5oi7x3h6x5fs+FZx/I17oBk+JLLqVNeyJbzkcX5cB8mb51Mjwd6uFyPlAqxrT0DxKgeyfBVXm4/b5G0flEqIaO0/hF/AJ/d8Bk5T+eYcvnZwnUtxHOF3rupsJMgWVuiRVtiJ3lyM4WHjwcDXxpIpzudmw9f3yLmFRqfPB4BgSfWPsH3B133MFZ5qyg4W8DeeXvN+TBVZie2SyUimStF9tWGlOevsSP+jAePEHl221UV8afXm8ffY+/8yaKhi64K5Q1H2PfVPv6c/qdbrykZ8+2RGc0R9oi/sstl/Dv7X7IO2GaelEoYUR0AklIjQO/77+GBB2CvmBbL5PjpiPj7dM+nTPtzGkUy0Zp2qfJknfWWyWSsemgVhxYcqrPs1QDpnQkMFMcsiWDPyRFzWV8pXDCn8XKK+DMaQacQGadIyUvLCZij0GyfN+gM5J/Op+2YtsT0jqHgXAGVhU2joOMIoW1DaX1Da5SeShNZJJPVvf5pjNxwkt0lKMjSeVv6ru3tA7PKxD7uY7RD/DkDpbeY78gBVCrQqC9S4nOQc/mppuO6Mp1JNrEpENk1konrJlo4atcm/npH9wZEdYAJyRMA+OHID80m4q82TuSdICtwOeU+KZD8GgzfLT6j+kAQoOy8RXQYwLaiXzkd9TrpWjEi6KZvb+L+nfc3tOpugYevB4lDEglKEiflMqPY0YI9zc50Eb4RaA1azhaIk5SXFwSHiB3HXqS1XCGnw20daDO6DZpiDSk/pVClqcIvxgW2oRZ6z+xN5wnWqjSbXt7EiWWivKrWKE7UK3UDMQzbAcoGKiFEXg8jD0DkcNOhXtG9eG/Ye8zoNcNkl4iZeQuTt0xu2L3chOCWwbQa2QoPHw8yi8WIv9oOtp6eEBwsR6O+RFqhKP+49vG1rH96vc1rnvzjJDs+2IG+wolwvWpcuCCOD35+ItHYv794fLsTnLdOBweS7uTBCyGiIkzeLig85PS93Y59j8Gh5yycxyUoVApaDGpBUKJzDj1XGkPfGsrwD8zvb1mZGLXoCvHnCpRKeOUVeOmlGnaq4O6Q/B8IdN7WdKXwP+KvGeLk8pMsGbeEgrPOrxjs5UGqC5XVYeEyZKgVatMmrya55qzn5/UtrwdgV/ouijXFDiP+wGxQczXirzbxpy3Rkrk/0yWitKmQuS+T1H9STVEgVxoymQyVl0hWVBZUkn0o26X37H8wY8+8PSy/ZzkVeRV0uacLfZ/oS2TXSFPuEw8fDwa/NphuUxo3h1bNXCzOOkbWlEl0BEcRf/dtv497Nt7jbDWvKEpKIC++Oz79zItmqc12PfZcMFjePP/m/2PvrMPjqtYu/htNMnHXpmmTuru30BaoUKClUC52Ke5coLhfnIu7FCuUQgWnFOrU3d3i7skkk9Hvj50zkvFIE+531/P0aTtzzpkzc2zvd71rLR6pecQvstAVes3qxQ1bb6DP5X2QK+VMfWsqgZPGYJI1EBBoRzalzoZzfoGQNL+235T4W3lqJZ/s+sTqey5h75d72fhSx8wNc6f487VAsvbxtbyW+JpDcWjzfzbzUvhLFO717pf16Z5PGfrJUDZmO/4+Q28ZyqyFs1xazjU0QFH6GGQzLrS+1tpWn+4Uf71jRF7d4RLPNhwymYxJL05qcU6lL5CIP29qP8DlRMMeqiAVncd1JraX/7/jPYG7GHLye4ZGnwuIc0q6Z/5wYAUP/PkAh4oPudz3gACsOW8uiT9DDVg8V2JlMplLVaE9+sR3I6nsH3TP/zc19c6kcua6TF4Kf4l9X3lWdP5d8e72dxn/+Xh+Of6LVe238tRK3tz2Jq9vfd1vYin9vHTi+sVRsLvAeq3YW5ZZFX9uVJznpp3L3IFzGZgwsDlfx2ek3DKNw+Nu+VsRf8XFovYXEADo9TRUN/wtnDc8IXtDNns/30tDTesR697uaR0R7uZ9pUdKWfPoGrI3ZbtdNy1N/G2fFb+/Uaz3559iTpjduHpGhutt6E16NmWLitkjQ1/mnANHOSfbew6xQq3g2tXXcu6z53pdtiNCV6Xji3O+sNoLSmN0iaSJjhaFJqMRSkvP3n75q/grrdJiVIibWVqM7xnn0vd0R5JveX0L7/V6j+F3DSdxcCLvZLzDocWHXC/cQWBv4+bt9tiWxF/TWovUkFBVJXMZ+yARf0EmN1afO+6E019634Gc72H/027flslAIxM7V1Jrq0eUHS/j87Gfs3+hd6Xv2YJ0LGuChOPE8OTh1veu6X8NAMtPLAeNuDjbO+OvKZafWcqujFkcDX8XglMhZrhXYtYtZDJYPgB23uHw8q/58zme/BR55r2AIGaiu/vY4d/GsFgsDmNIrUUcp6hA2/5lRImHUm51LvUGMW+M1BUQXnTcbROt2WQ39rfAid9O0GtWLxIHta4VgqHOwIbnNjgRf/WqOIhuwbGUEBAFUYOEmrMRvWJ78cCYB7iox0UdMuPP/niW1Il5dnK4s5tPWkIEgNXCdcqbU7jgjQuclgM4uOggKx9YiUzh+3hWsvns3l1cGv2EQzm5PqRSGAxgVIgBYog6BDZfBVuv9/mzWx0FKyB/hXX8t6t+KY+ufpStuVsBoXB15XbUEdGSKDQayoX1rcnzfKBfPxg2zO6FqMHQ7ykI7tyMD21b/I/4a2W0xqS316W9mPXNLL/snJpr9Sk91IJUQchkMpfd9r4q/tIi0ugW1Q2TxcTazLVOhUmTyXUXX0utPo//dpxPhn7CmTVnnFfqYFj/7/V8dd5X0I61EW2xlp0f7aQmv4YHyx5k6ttT3S7bUNPA4WWHKdpfdBb38OxDJpORkJDg1/WbvyOfI98fISAsgAHXDuCcp8/xOwi4NSANRGuiNnD5kst5fcvrXteRlC6+Kv6KtcXUNPx9KpKSSsz+fuhO8Vd0oIhX419l86ub/foMZUDL7cYCwwNJGZHicN7V18Ox5Ce4KUfFY6sfg+IN8GMnyFosFmi0G/QFTYm/R1Y/ws2/3syewj0Oyx34+gAbnt/QIdUa3hR/4pni/vrtflF3Rt0/iqBI0ap4eNlhVj20CpVGhSbG80Otor6Cx9Y8xt7CvezK38W2bfD0087nUFPYk0USrFafmtax+pQmYAaDY+d/r1hhcXG41Lv//uh5o89KfopULPGF+POmUjXUGRwn2X7AVBdGYuVMokLFcZfJbPeID3e/x6tbXmXlaccOUHuyaO7AucwbNc+ZBDLUwu+D4djbzdove6jVMoZkfkNG4YOoLc6dXNHdo+l/TX+iu3WMQkpNQQ2VWZVel1v3zDq+nPgluirnioH98zezMpMN2RsAuHbAtfSI6cFV/a8iSBnEweKDzcqbveLHK5j93WyPxJ87xd9V/a/is4s/Y0aPGX5/rj+I6haNPjjyb0UOSd3vCQlw6TeXctOOm9p3h1oBF7x+AQ9XPew297o5+Dsq790Rf0nDkrhx+430vrS39bWm4+ekxsivvDxBDJvNNhJw1y44eFC8Hh3tfv63I28H9cZ6YjQxTOs3mpCGHlSUKTzmBkroMrFLh7k/+gupIbMqSwzO7RV/IFTz8Y01zbOV82ex+E/8ZZYJ0khhCiYq2PeO5I/KrmB1vy7sL9/q8v2UkSkMv3s4QZFBRHWLYvSDo0kY4DuxeLaw+NLFvBj2IuAiv8kD2sIi0h3xJ419jEYIDU10Gj9LVp8BBkFeOLiQGGrgxHtQ6INjRNZiOPiMR8cSjUx88dI6G1MWnhrO+a+fT9fJXb1/RhugYHcBqx5ZRelRG8Mu3X/G5yznyB1HHMYFfeL6MChhEEazkVWFwrJUp3OuWbUE+4v20+WtLkxdOJVvD35rreH5ihq9GGAEykNE/qKXDCqv6Ps4dPmnw0uhanG9aw1iomSoM1BTUNNuDe/2OLH8BP+W/5u9X+4FoA5xvkVrbM+L6KBoa66d1PQcuns96Tu/Q1vjeu7x2+2/8XrK6+gqdQRGBPLPtf/k4s8vbtG+7nh/Bx/0+4DqPNuEUxmo5K6Td1kbWxoQ11RaVBgyUyuwcRaLOC8Mrie50ryz6lAuW97Y4nI8f7ax9PKlPK95HovZQnmDqFWmxjjPs9OTIwCoMVQCkDElg/TzXMfiTHl7Cjduu9GvOs+xY+Lv7o0GaVKdzVu9AMQzwiS3I/76PgW9HvD5s1sd0/bD1D1Wsuwwy3hx44tsy90GwFtd3uKLCV+03/65QXVuNcuuXMbxX49Tpaui7/t9efL4BZgxNk/xd/QNYX17+rNW31doHc7IX/yP+GtlyOUt/0ljesTQ7x/9CAjzfeLZXEZbZ9ShkCkIavQSboniD+D8dJvdZ1Pir6JCPFMUCuGZLaGlVp+JgxOZ+PxEYnu3jpKiLTHinhFM/3B6u3ZFV2ZV8tutv1mDWT3lmmiLtCyZvYQD3xxwu8x/A+RyOQkJCX5dv5d8cQn35d5nVfi1F6RCYV3IIZYcXsJfWa6zoezh64AkPDCc6CBxgZ6uOO3wXs7mHPZ/3XE6Me1RvD2TPuveR3XGZkEiPfSbKiqCY4OJyojySgJJaKhu4NDiQ1ScaZ3AHovFQs7mHOaPmM/3V31PXaUeg6IcCxY0Ko3IXwhMEIHLO++GX3tAvW8BL03z0iT7C0l9JmHa+9O47eBtrfJ9WhvuFH+SbZHJBNXV7q/fPpf1cbB9MNYbCUkIYdY3swhL8dwp88z6ZyitK6V3bG9uHnQ7778vCpebGzniVY+sYuVDzlYhNdkVdN21BMUZW86Q1eqzlRR/9rZL9oW43rGiIHuk5EirfE5rwB/Fn7ci+erHVvNcwHNU5/pvM+yqiUkqfvUJGw3A5hzHBgB7suifA//Jf87/j5VctUKugrpsKPjT731qCpnMvZoTIDQplJkLZpJ+vvsc17OJryZ/xfwR870uV1dSR8mhEpeEiv3z9x99/8GiSxexce5GPr7wYwAiAiO4uv/VAFy25DJmLJrh9DzyhMiukSQNTXJJ/Hmz+mwJLBYLd/x2B3N/mmvL7XCD4CAzqvpqasp8YDY6CCTiIT6w6m/T/esNwXHBVleMVtvmfxHxpw5WkzwsmdAk2wO56fg5IUHcx+rrRXNZUZHNStlggIULxb/dqf0A1mWuA+CctHMID5eh0Yj5Y5EPPYgWiwV97d/nOrJHZJdIHq58mInPTQRszbLhNgGGNUfmbBF/JSWO825fiD+FKZQeuc/Tq3KeX/PdKnM+9QGZ5GtdK0rTzkmjrriO/V/vJ7JLJOe9fB4pI5tnvd2WSBiUYCWs/Gl8ksi5s6H4U6slpxgZgYHxTuNnrV7cBAIM4oRzsPpUhcLsChj0ivcdGPAcXHQaFO4zq0IUYufK62yMpyZGw6h7R5E8LNn7Z7QBCvcVsumlTVScts31rMdSJadnTE+ighx/1LkD59Intg81hgor0duaKv4lh5aQWZnJipMr+MeyfzDu83EYTL4/e6UG3iBFCKw5X8wnW4LeD0GXqx1eimrMZNWaxbH884E/eT3pdepKW0gytgKC44LpNasXEWkRmC1mdIh9jA22FSJlMplV9SfZfSpHDuX04NnU1rrPkY/uFk1AuBhcdh7f2a8ariuY9CaMDUaHOAmZXEZUepTVZtHQSPzNNX+G/NdWmBPU58OScNj3hPUlo9nI9rztjXVd8f1rdp/gz/v+bNY8rLWROCSRjAsyQCaj16mPGXNkM7P6TXNarmfnCAD0aL1eMyHxISQP9+++Iyn+ejReUtLc0hfiT68Hoz3x1/VaSPuHX5/fqlAEgExmrSHJGyIAqNCJe2HPWT3pel77NGR4QlV2FQcXHaTsRBm51bkcKjnEKd0O5Cibp/jrcRdM+AU6X+Hfepv+ARu9r9ManJG/aJuE9P/HMJmaGZJrB4vF4jcx1FzFX6/YXhifNFpvgvYTVItFTN58VfwBTO82nWNlxxiWNIygBtu2wDaQjYyE0aNhcaOApaWKv5geMYx7dJxvG2lndJ3U/jfKhIEJzPlhDqpgFcd+PkaXSV3cZiSGJoUy65tZ1ty6/1aYTCYyMzNJS0tD4S2EwQ6BEc0JPWhdSNeXIUBUARJCvHe/+mr1CcLusyyvjFMVpxiQMMD6+tY3t3J4yWH6Xdmvw4Q8S9BWGZFZzASH2B6qUuGk6SQsJCGE6zf5bqlQfKiYpXOWMvnlyYx5cExr7C4/XPsDFacqKDpQhLH7TPRKuy7E+HNgSmMwjtkADSU4SIazl0L5Thj4ktN2myq43RF/HbU73mJxb2OtUIhnSXk5FBWZKC/37frtd1U/a7C7JxwpOcK7298FYFbPWVy78H5OyAbRibnWgfzpladpqGrgvJfPc1i3NreCyILDKGps9/vJXSejUWkYkuh/Np0rSLZRFosoxEnPx14xgpTKqsqiVl/r0Vpy7ZNrOfrjUW7de2ubXsMS8edLx3tTsrop4vvF02NGD4Ljfc+yqG6o5vyvzsdsuYBYniA42Db0DQ+HnBzIUAvib1POJocxmCuyyAmKABj/M4S1sIiC7bMqTLksPryB2+Mu85jZ2t4ISQghto8jme1qDDvt3WlMfWeqy7GtyWTizJkzdOnShSFJQxiS5HyNvH7B6wQqA3l/x/v8evxXCmoK2Hmz9xxLe7gk/qoE8efO6hOEM0Z2VTbdorsh98NKaV3mOt7f+T4A3aO688i4R9wuW7P1EANWf0+16jKgt9vlOhIk4kHx/RK+PxXKpJcmEdPDd6eSjoqi/UVoS7StNl7/uxF/JpON3GlK/NWV1aEOVqMMVNot7zh+VqkgLk6QdHl5znM2b/l+h0sO8+a2NwGYmDZRkIipv7K7YREfbZ/Ec508j9cWXbiIzHWZPKp91Nev3GHRVPEHNuKvwLf+rxbDXu0HvhF/wZZ4uhU+at1XX5Ecmsy+SijWufZHM+lNZK7PxGw8iwGHzcD4x205vv6Mf/yx+jx6FNasETZj4zyUQI409oC5OhYREVBba+HAgVwSE5Mcxs/59+ej1Wt55kkxz3UaA6kjgAjvOxrqgeGXFlGKL16h6zjemL1n9yZ1bCqhibbJh7cmtrtG3MVdI+4C4J+fiflJdbW4H7YGMqsyARjdaTT7i/azq2AXm3M2MyFtgk/r1zYq/jSKUEiaDsbWJ25iGtVzdRZxLNPOSUOukCNXtb/WJHlYMpcvuxwQji4WmbiP2BN/IOw+d+bvtBJ/wQMyqKwEnZt+kvGPj3e45lsDI/81kpH/GunwWn15PcYGIyHxIViQWYm/M4o+xKWktFzNo46ErnMhZoT1Jb1Jz4j54v9f9aoGQpEPHsjcR9OJSIto6Se2GGMfHguIe6a6vhOB8k70cjGc75MRDo198eV1lWy/fyuHFh/iofKHHOa/ZqOZ6txqQpNDfY5zqa0VYx2wKf6kOptWK8ZUnkoTBgOY7K0+2xvVJ6Aui5DgiYAcdJEQbmeT+saUdt09d+g0uhOP1T+GxWxhbcFaAMIQBG6ziL/AOEi+0PtyTaErAbwrnFuDM/IX7X8X/h+c8OGAD/nyXB980+3QIg9bQKUQI1Jpkmex2CZr/ij+pnabysprVjJ30FyrHYk0OZEGstHRgvgDoeLzVX7rTvH3P/gHhUpBz0t6cvyX43x78bfoKtzP4lQaFf3+0Y/4fs5+2f9tqPGjLe/YL8c4vPQwJsPZv2k3hUTqNCh9J/58tfoEeHrC0/x0xU+MTR3r8Pqo+0dxxU9XdEh7SG1iBgfPvZOUc20TTndWn55w8o+THPzuIMYGm1ojKiOKWd/MovuM7q2yrzKZjOkfTOfWfbfyYNmD6HQyDErRhdi0m5SUGTBmEQQ1HmOLBTZeBodfdmnN0bSRwx3xZzaZqcqpcsjB6whoaLApBVw9J+xtkVxdv+ufXc/8EfOpybe9J5PJ2PjyRja/5tna9dM9n2KymJjRfQaxwbEszXmHwoifAds5dPUfV3PnsTud1lX36MruaY8TMtZGlF838Do+nvEx53Zpnewhd8qwaE209TgfKz3mcRuGOgMmvQm9tm3VEc1R/LmzPBx0/SDm/DDHr3zNr/Z9xba8bRxTfYscpcNYRroXJjEMhUxBfk2+VQUGtt/WrK5gW+42CmpcVFtPL4DSzbbrsoVQB5j5q08/7tlwJfsKnbP8dn+6mwWTF2DUeVaRnQ1cu/paLlt8GYB1fxbNWMRX539FbaHjQXTX0JZdlc3478fz7F/Pun2ehKhDeHvq2+y+ZTcAuwp2seyXWh5/3PeOelfE366bd3HmnjNOzzcJRrORsJfC6PleT6v1ma94favNdvvp9U875UfaI2FAPEVpw6lRRPj1Ge2JghzRMBg/uS8B4QEtzrztKFh25TJ+vuHnVtve3434s59jNW34/OXGX3gh5AUnu+Wmz1/J7jM/31YQ69wk7sQV8Vepq2TygsmU1pUyJHEI1wwQuVmGiEPkR3/DX7nOCvumSJuYRp/L+zTbEro9UZ1XzdGfjlrHLNIYXcpjA0hsjI06W4q/5hB//jQM26NTpCjSVRjzXObOKdQK7su7j4s/vxiLxcI3F37DmsfX+PchZxnNsfr0RPzl5MADD4g/v/8O772Hy98KRP3l0CExXpTqLvaQxj+Fha4ParA6GL1O3NcdFH9VR6DqMJh9mANbLFBfBLWZbhcJU4q5TqXepvgz1Bv4aNBHrHrEBzvRNkBAaADR3aIdIjz0ejiW9DQb4690yv1uCm+58s1BZmUmAHcPv5sFlyxg9827Gd/Zd8JJa2gk/pQh0O8JGPSflu3QmYXw+xCots014kNF80+9rBSLRbiuTH17qjVuoaMgPDCcqwoKmXDwMCFBjqx2tyjxcLISf43P8JrK9h1z7/xoJ68nvW7NrJbIov2B07EMeq3lH6DUwMjPIO1K60tByiDUCnENGFVC8dWgiSR1TKpb0UB7QHIDiI11TbJ1SlaiNIki+YncSsJTw0keluxUx6vMquStLm+x+hHvmcISpDFOTIytVhESYst09Wbh76D4U6jg157CYrK9cPhFWHMeIYFin+SGCMBG/HVkKAOVqDQq8muEv7zGLAajflt9HnpJ1NbMBtBm+bfupFUwyffz52zif8RfB0TCgARievvXNdtcxV9TqNUiOBxsA3fpb1+IP3tIk5OaGvHHnvhLT4d77hEDV1/FjRLx17R71GKx8MWEL/j97t/928GzDEO9gdcSX2P1ox3jZtBtejfOf+18QhI7QHfJ3wyb/7OZn2/4uUMo3aRiUr2ibYi/qd2mclGPi6xkgoSUESn0uKiHR6vY9oL0vXzJ+AM4+O1B/pznbNO37c1t/PjPHx2Oc3BsMP3+0Y/YXq1nLZx+Xjrx/eNRBanQ6cCgsCP+TnwEJz5wXMFigfzfAQtcYYSZhcIKtAmke7bZLIre8cGCwG9K/GWuzeTN1Dc5uOhgq32n1oB0rJRK14orb0USU4OJ2sJaguMcH17rnlzHmsc8F4ukEOtLe11KbKMCpzbosMN+aaI1Lu8Bej1Y5AoCgttWqeXOErJXTC/kMjlZVZ4Hq+e/ej53Hr2zVfOsXMEf4s+b4s9fnCw/yYOrHgQgvex2AJfEX0Otxprdtz1vu/V9ad+LgzYy8tORzFjkIustaxEceUXkpdS3vBobGCAnslaoiV1ZN1eeqSRvW55D9kd7wmQw8d3M71g8W9g4DLllCAW7ClAEiNn391d/z/p/r6f0WKnL9T/b+xkF9QWsz1rv1e2if3x/rul/DY+Ne4xflxvYtw/+8u5uDbgm/lQKFWkRaQSrXQ9wlXIlSaFi4phV6fvkz2wx0zm8M8GqYEamjERv0nu0/Ow0JI6cvlMpD0zy+TPaG/rX36Pnps/of9NILvniEiK7Rnpf6W+Ac545hwveuKDVmpr+bsSfVKQKDLTNByWkTUxjyC1DvI77khtdsvLzIbdRvDVihCP558rqMyIwgnmj59Evrh9/XP2Htfu9T0w/AE7WeI8dGH3/aC7+/OIOOTb1hpxNOXx3yXdkbxJWl5Liz5XV59lS/J05I/6Wnt++EH9HS49THXgQpca/0NL0GGHbWavIpaICHn8cPmsSryOTyVCHqJHJZBTsLqD8RCsG4rUSNry4gU2vbAIcib9KXSU3/XwTm7I3uVzPF+Lv55+F2k+pFPUTrRZKXT9aWdM4zB04UBSlm0JSktbUuG/akI63A/F34Cn4rS+YfWgaMxvgx2TYfa/bRSLU0aiMkQTJbfMYhVpBQ00DZkP7EPi6Sh3VudUOxIDBAGWhqzmlWWR1C3AHd/ESLUFccBxJoUl0iezCzF4z0VQP4pVXZFx/vU1J7Ql1RjvirzVgboCGUtDb7FATwsSJpleWurSrb09krs9k+V3LKTtehlwmR1EfT6iul9PcZETyCC7tdSnDkoYBEBRkoc+698l9+RunbebvzGf5ncspOuCDD7Ud6g313PTzTXyx9wuX7+sqdeyev5ucLbbzLGlIEsPuGEZElwh0OhhzZAsT95+mf0zbZbXLZDIiA8XYTi9vJP4axLjfpG//5vf1z65n0382kVtcw9Hkx8iMf8vl2E0uhwTzMCJqR5Kba2H84+O5ZuU1Tjl+ygAlI+8dSZeJXXzeh+JiKAr/ldyk96xzNrncNp/11nCuM+ixyBud95QqkLWzy0va1TDsQ4I0KuRyUBkjAJvV5+5Pd/PjP3/sELmd9ig7Xkbe9jxMepO1ziVZVftF/FkscOpTOPM17LwLfkoDnZuH7N8Mf79R8f8DzPxqJtPfm+7XOs1V/G3J2cLM72by9Lqnra817biXJqv+dO4V1hayMX+VdRCbn2/zmZdemzwZhg3zfZvurD5lMhl1pXXoKts/ZNYTDFoDkV0jW+z73Ro4s/YMC6csRKaQeZ0cv9frPRZdtOgs7dnfA9PencYVP13RIQoL0vVZJxODTn+IP3/Ub38nGPcfJiZ7t0OxxFP35YnfTrDltS1OuTBT3prC8DuH88M1P1jfKz/ZtkWG+npsVp9B0XDsLTj2tuNCx9+B9TOgYg/IFRDkWpEbEGDrfKutda/4i+kZw/C7hxPXr2NZ+krPoLAw1w0iNsWfa7Jg4nMTuSfzHuRKx+t06jtTmbVwltvPNZgM7C4QyqIRKSPI3SOIP23ASUyyBut1Y9QZObP2DIX7HMme2mN5BFfkolKJQbHFYuFk+UmqG6pbVSErFWKaFuIWXbqIukfrmNXL/Xc8m2gtxZ/ZaGbpnKXs/XKvT59rNBu55odrqDPUcW7auSTnCQsmVxl/VVW2fMQTZSes70uFizqVmHynhLnIExq3FMb/BMui4dDzPu2bJwQGQlSt6OL+K9uZ1Rr/xHgern6YqHQffdLbCOWnyln39DpKDpeg0qhQaVQYdUZ6zOjB3afutnZ4y+Qyzqw+49KezWg28vnezwG4afBNPn3ugpkLeG7ic9RXiGLEnj2+7a9Ptq0u0DlcsBVSt70vkMvkvDvtXQrnFbLs8mVEBEawI38HC/cvdLm8NHZvaLBdLx0d2oR0qmPSHGwI/xvQ+9Le9JrZq9WyuKV7msHw9zi2ntRaI+4a4dP81JXiLyUFxjeKU+Li3Deq3jfqPrbftF1YnTdiULIg/oqMx2gwdrBqcisieUQyM7+aac0X8mT1ebYVfxJR6wvx98mJp/irbz8OBX7i12elRYnvrVPn8sUXsG+fILqMboQ29+ffz+zvZvv1GWcDez/fy4GFgqS2J/4eWfUI8/fMZ+znrhXm0phWp3PvbiTVUm6+GVIbo2mbqjJB1C4l4m/yZNfbckf8rTy1kmkLp/HKpldcPzfTroEBL4DSBxWXQg29H4ZO7sejaYGDuGBvOS91sxGicoWcu0/e7ZDRfTax7e1tvNHpDUqP2Aq+BgPUBgp1W48YZ2t3i8XCoI8GEflyJOYQceNrTeJv2eXLyLsvj2FJw3ntNZg3DzZuFDmcu3d7X7/OJNkJhsLWuXCkhSqx9OvhkiyIsVlSSoo/vbKU+no4tfIUiy9dTNF+/4ixtkDh3kJ2vLuD2iLxO7ibm0zvPp2lly/lhsE3ABASIkMbkYwlxrnht/xkOTve20FdiX92ZL8c/4X5e+Yz96e53PLLrWzfpeeNN+D222HvXkH8/XLTLxxZZstrTz8/nWnvTkMTrUGnA7UpmkhZGt1KXkN22Dnuo1nYPQ923+/wUmRQI/GnaCT+jpziOfVzPs/F2hJ7P9/LoW8PkVeTx8nEF9ioetLt2O3+mDWMPboFS6l7x6awlDAueP0Cuk1z40XuAgdyz7Aj4yJWqe90qKf7GqujN5hILbmZqSlXEqxJgOkHYUDL55LNRsJE6HYLMlUQwcGgMonjLyn+cjblsG/Bvg6Xpbz1ra3MHzGf+op6a51L2SCuWb/4EZlMHIPxP0DCedD9LrD4mKVaXyga9qs9Oy61F5pFKRuNRtatW8epU6e48sorCQ0NJT8/n7CwMEKa6zX5X4LWmij6A/scJH8Vf5mVmfx49EeqG2wMQHCwuElZCQU/FX/HSo/R872eaFQa7kwuo6wskLw8R8Vfc+DJ6vP2Q7c3b6NnEZoYjV9ZYm2JxMGJDJw70CcLz5ieMYR3Dve63N8ZMpmMTp06+Xz9xvfvONanUoG82iKqAJKqyxP8UfzV6mtZfmI5xdpi7hxuszXc/Npm1j+9nus3X9+hrGAtFgg+sI3ImlLCwgZbX/dE/E16cRKTX5mMKtjRiye6ezThncPZ8voWBt84mPDO4bzX8z3GPDSGSS9MapP919ZZMMTbKf7OXQGGJgeqy7WiE0lXIqx2cpaCQiOsQO0gk4n7dnW1uJ+7I/7CUsKY+tbUNvk+zUFhYSHzP/mE7z7/hvz8ajRBYTz37JXceNNNJNgFldhbfbq7fl29NuRmzzl7Fix8NfMrdhfsJi20OzvXyVB2C8eorEIbeJzqalGE1FXpWDBxAYNuHETfOX3Z+/leZn41k6qlq+iWVUjAXQ8BImOu2ztiElH3aB1BqtaxvHFH/CWGJvq0fmVWJaf+OEXauWltmvPoT8aNJ3WMtljLocWHCE4IZuA/B3rd1n82/YetuVsJDwjngwu+YN5SucNngOO9MKOPqGyeqjhlfV8qemnlOWB2kwWnDBaTg/QbIbblecMBARCdLyrkG7I2YLaYHbLlmnaothcK9xay/pn1RHeP5uIvLnawerTPv73ki0vcquO35GwhryaPyIBIZvac6fNn6/W2c+TAAe8ZGmA7llKRZ2P2Rj7c+SHjUsdxy9Bb3K7XOaIzG7I3eFXQukKIOoQQdQgLLllAWX2ZW0tRjQa67vmehsBQamvP8zn7uj2RPWAGOp3/biB/F5j0JhTqltuXajS2PNa6Ot8aIFoL29/bzuqHVzPq/lGc8/Q5Pq3jT7wDuB4/S8RfXp5tzJWcDIMHCzLHUyYZQKDSMT+7f+cUVNsiMCgrOVh80GUOqISczTns/mQ3I/41goQBrWO/fLYQ0TmCiM4R1v9XNAppXBF/Wq34bf22sPIDer2NuO3ZEw4fdm6+dYWSemGzFRPgn4JZaqzRqfJYKyJ6MJkEgSyRXH8HXL/xemujiz3xtzxzrXWZotoi4kMc506BgeK602pFzcQV+S7N26KiIC0NsrIE8de0kfrAAaFE0Whg5MimWxGQziulMsbh+j1UcojfT/5OaEAocleKv5QZTvMNjxjwnMe33blXtCeShycz4p4RDo4hpdpy9KoSALpHO5MHMpmM0rpSKnWVmMLzgORWJf4k5OfDunVQGbKVok4foirvT2XlfV7XmxH+GCuP5NJl2EA4MxuSp0Ov+72u5w/Gpo7h3DMboDYRnQ6qc6o58sMRBt80uN1rJ0NuHkLfOX0JjAxkQ9YGdsQsIVQ9goCAqzyup9FA5sCLSRzl/F7fK/oS3T2a6B7+zaHGpY5jdKfRbM7ZzMe7P2LZuuOMOP4ncpSsXQt33xHKlcuvdJubbE/IR9X8iay4AmgFe8jClULJOdhGCkcERojPlFUCoFOHM/SKvg7PqvbCTdtvwmQw8cFfogMwROb+HEtLE39nZkLWhixO/n6S4XcOJzSpZQ/RfUX7QGZBI49gQmdb3mZYmHh+emuyN+uD6J//ES/fDh3NMT8kBFTaCEDkYoJoip/6zlRUGh8m9WcRvWf3JqJzBJpoDSV14j4trxd1L7/HSYoAkU8bmgGpl/q+XsU+2HErDPsQwpybQ+zRHpyR33KVrKws+vXrx8UXX8wdd9xBSYn4YV9++WXmzZvX6jv4d4Nc3jIFUMXpCv64/w9yt7oOtnYFKb8J/Ff81RvFCD7IrmtLmuxJBJtELPg6Cewe3Z2k0CTqDHVoY0S3en5+y4k/d4q//8F/BIYHcvFnF/skZZ/zwxymvNkxg1xbC3K5nOjoaJ+uX7PRjKHex86PswCtVhAVtSZxgbW21Wd1QzVzls7hnhX3oDfZunvCksPoNLpTqxTGWhMNDXCm74WcGnK5g+LPk9VnWEoYoYmhDg/h+vJ69LV6BlwzgPvz76fr5K4oVAr6XdmPjKnew+qbi1qtkYTKSxifPFl0vAenQkQ/x4XUETBlOyRNAZkctt8CR1xnNdgTKYMSB7Ho0kW8N+29Ntv/luKN116jc0oKG194gYfPHGVxQz4vVR5lwwsv0DklhTdet+Vm2Yg/5+v35IqT/PXcX2hL/PdXUyvUXNr7Up6f9DwlxXLqtDLC9Y12n4GHrddNSHwIU9+dyuAbBvPHvX9QW1SLTC5DOX4UOb3PtxZ4JaI1RB3SaqQf2AoxzS2WFO0v4tdbfiVnk2fLopZCKnz5orSSzlf7fEcJoUmhPN7wOBOfm+h1OxX1Fby48UUA3p76NtFKUTVUKh0JSHv18y1DbiHrX1l8PONj6/sSaVkgE+3UPWN6Nvly1WKgb6qHEZ9A58u9f0kvCAiA8LrBBMo1lNWXcbjksMP7FouFUytPcXr16RZ/VkuQMSWDW/fdSvoF6R7z3TxZYku2OJPSJxGk9u3asFgsHM3Lp1KzAxBj1ePHva/XVLmwK38XCw8sZPUZz/braeFpgG9WnxaLhdt/u93JonVGjxlcN/A60qPSXa4nk0FoZQ7BVQVe80A6AsxmW8PBfxvxZzaZeTvjbZZctqRVtieT2eYvZ9vus/xkOfpaPeufWe9ScesK0vnXdD559KejLJy20MnSzNX42Z74kxRKycnifvv88zClyZTCbDFz4TcX8tLGl6jVO18AyckyImtFSNnaM569fatyqtj7xV7KjnnwS/wbwGy2jVftib+AAKyNAa2t+rNYLJTW2RRO2dmivhAWZovv8GW8UaoXbGF8ULJfn58SlkKouRNB+s5YsLkjZLm59eZtz+PQYvfZqe2F4LhgazFZGv+gruV0he2Z/cepP1yuKx1bd3afEhkcHu5YyG6K1Y2PtfHj3TcbRESI4qPBEOxw/Up5tokhia6tPlsZ7prYDn57kENL2uf4ZkzJYMqbUwhJsN0IT1UJFUeoJdlqQ9wUEnndECCugbZw1jl6VPwd2uUoJ4K/JDd6gfU+6wm9FNPpXHoLScGdYU4djG4FN6fKA7D9NqvNfbQmmhTzWIIb0tHpYOB1A3nS9CQZU9pu3uwrVEEqQhJCUAYo2Z63g1Mx71Ac/pvLpkSzxUxedR5avdarXXfi4ES/8+4SQxPZdP0mfrzsNxSmYMrC1qLpvQ4QSm+FWkG3qd0cLNQXXbSItU+J5oHaegMHUm/nQMJD6C/JQzbhF78+3y0u2AbTjzi8JFl96hA3nzpNDJcuupT0812PZ88mNDEaQhNDKdMJPiJU7t65yP5+mbctj40vbqQq27EQ9tfzf7H40sV+ZaifqhQX5Mio6Tx1zlPW14tCV2DG6JX8d2iOrS+EY++KeWV7IfcnkTNYtJaQEIjQDueL0Tv4+R8i+zogNAB1sLpdiCtP6HJuF8Y8OAa5Uk6AIoDooGhkWkEE+8yPaLNEvqKu2PuyrhA9DM5ZAUnem+lbyhk1B35/4j333MPQoUOpqKggKMg2SZ85cyarV3uePP9/gMnUMr/j0qOlbH19K8WHfD/hpBtKQID/naT1hkbiT+VM/EmTP1cZA54gk8mYmiFO+Cz1coA2V/xl/ZXFzg93tqqNWmsjb3seG17cQFWOD2zL/3BWYTKZOHr0qE/Xb+G+Ql7QvMDmVzefhT3zjtpakCHjt3GVlDxQQmq497ZY+2K3t0smMSQRlVyF2WK2TgZBdLld/cfVbrvR2gvV1aALjUWf2NmBaLAn/pp+Z7PRTMXpCmoKbKOzLa9v4cWwF6nJr7FO/CLSIpj51Uw6j+tMW8BoBKNexeDT3/LrnJWEyGVQlytUfe4gk8GY72DY+y7fts9MiwuO44q+VzAmdYzTcqseXsW3F3/bGl+j2Xjjtdd48bHH+MtkYoVOx1XAJOAq4A+djr9MJl589FEr+Sc9S0pKzE7X76HvDrH2ibXQwkeC9NyKkwnirybosMNEfvgdw0kZmcKsb2Zx0fyLADBndKes0yDr+Sd1nsVqWi8XEmwERtNiSb2hnht+uoExn43xaIuWMjKFq1Zc1eYTcn8Uf/Yd7q4m2Qq1wqdMwrWZa9EatPSL68fV/a+2jmck9Y0E+yaI+JB4UsNTHdR1DQ1gwUyOeRsAozo1afkt3gi/D4TMr71/OR8REAByi5qumkEATsSfTCbj+yu/Z91T61rtM5sDdbCa+P7xaKL98IFvgo05GwHoHtDd5/Hz7oLdDFiQzPZuNsvBvXu9r9eU+JOsO9Mi0jyu1zWyKwDHyrxbtzyz/hk+2PkB0xZOcyig+4LSK+/i+Mhr20Qh0No4/Vcunff9RFBVAUGt18vQISBXyEkcnEhUd1GB11XqWpxl0l7E35Q3pnDdX9dxw5YbwMcajTuXl4pTFZxZfQZloKPi2NX4OS5ONFlIzRsREZ4J4u152/ntxG+8uPFF1ArniWxkJMTWCxX06pOeib+eF/fk4eqH6X1Zb4/LdUSsf3Y9b3R6g8qsSmpqbGPVpm4+bZXzd9fvdzFi/ggKawvZnredlza9gAULqam2Obi35luLxUKFUSj+EoL9U/x1Cu/EwwHZjD62HhkyK9mYne16+c3/2czSOUsxm9onB84dyo6XOdkJBimDWH/demuj9fITy12u6y3nT2o8i4hwT/yZzbC5cYo60UOfVGioOF5ZWWUO129BrTix4oMTrdewlfgrXA2/9IA81/vvEqVb4Y9RkPuzy7cDA2F31yt4LHswR0pspMPaJ9ay4fkNvn9OG+NMo31bjMy9kiM5VJDddUrRwN9az/PP93xO2ptpPLTyIY40/kRT0y9EjpxqzT6frMgdxkBylW9Wrd5QdQROfggFK6wv2RO5MrmswxAEtYW1lBwpwaQ3UaIVF5jaGOOydjr2s7GkvJHC6jOr0WhAYdBh+W05u+eLJkC9Vs/i2Yv9Emq4Qk/FNOIrhXo2ZqDIlpfqroCVgDIbzeRty6PsqNjvitpasuI+4EDYfzh+4jQmWStZCSgCnbI1JKvPOosg/nyxez4bsFgslBwuobaoluoGcWPUKCLcLv9V/mOs6pfKJuPb9PrHQO48dicJAx2b5UsOlnBmzRlrRrkvyNGJC7JXrK0x9J7f72GBeSqnE171KePPoKhEpjBCzUnYdRcUrfP581sdMqX4YzEJxZ8pnE6Koda5Ul1ZHXk78tBVdZATwQU+nvExZ24rJaXsn4AfxN/pL+DQC1BulyGxex5s9LGpNyAKki4QTfte0FLOqDnwm/jbsGEDjz/+OOomd8m0tDTyJD+I/6HZ6Dq5K/ecuYfel/o+YZEGFc2x+6gziBmeJ8Wf/SDTV0jE3/7634HWIf6kibMr4m/Pp3v47bbfMNR1HCVWU2Suy2TNo2uoLfgbtHPb4cA3B1j39Lr23o02h87HkYwqSEW/K/t1mDw0qZAUFionRhODQu59sCIVEYxG9zkSEmQymVVFWFDTylWGNkBFuRm5scEpF87+OzftWC49Wsrb6W+z84Od1tcSBibQ5/I+RHWLQq/Vs+bxNeRua9kA3xvsCypBQUDBSvixE2S6zoWyIukCiOjr8i1PmWn2KD9RTsGegnYLay4sLOThhx7il4YGRrhZZgTwS0MDDz/4IIWFhcQ0cs5lZc7X7/QPpnPj9hsdbHp8xQc7PmDlqZXojDprF22Kug8A9epsdDrnrKb4fvFEpEUAzpaCJdpG4i+4bYi/pudzoDKQZUeWsTlnM8fL3EuhgmODybggw6GjuS3gT8afQmErMDYtkpedKCN3W65PnZizes3i2J3H+GTGJ8hlcrfFbG8ZDA0NUBt4hHpLFRqVhv7x/R0XCOkKfZ+EmDFiwrZqgvD3bwGk4xoqF52Krgik6R9MZ9KLbWM37Cuq86pbNPEzmU1szhHVyX5h/bwsbUOv2F7IkKFXldCgFE1yvuT8ORF/VZmALcPPHfrFi33bX7TfY3PZ4kOLeWb9MwC8NeUtYjS2phizxcyaM2v4cOeH6Iyuf7OQMDEl+zso/nJ3FRGbs5dAU91Zta48W7hs8WWc/5/z2fPZHl6OfJnVj7WssdWbYqAt0XlcZ1JGpvicSe0u133UfaNEtmiGsw9t0+evQgHxdo5bKS6iUe3x09GfAJjWbZpL4k8mg15BE5BZFFRrPd9zlIFKAkIDOkyx2R8ERgQSmhyKSqOyKrtCQwWJag/J9tKdEq45OFZ6jPd2vMeZijP8evxXxn8+nk8zH6Mm6ACxse5VWU1R1VCF3iIGs4khvtmO2yOyUeDSqxdMb+ztcEf8jbhnBJcvu7zFDV6tjfd6v8dvt/4G2DkeqBWM6jSKVdeuont0d3pEuyaPPBF/Op3tOWZP/OXl2SkLkcbE4rzp4cFtzNYQ6UicSnO9aLWtKG5tpDTVC6cRpR8NP3I11J6CBtdsZmAgVAftJ9uwx6HB9OIvLmbGJ35YirYitr29jW8v+Ra91jbYz6oVxF+c3P2PKin+amStS/ydqjhFVlUW1Q3VVuJveN8YBscI+/BDOtcKUnvs0v5MaehqlMpq0bSmbQW3j+QZMHUfdL3O+tKJiA84lvQ0JTWV6Kp0nFl7hsqsypZ/Vgux+dXNvN/7fWryayiuFeNqtTHapRtJp3Bh63+y/CTBwWBWqFAf3M3JFScByFqfxdEfj1Kw2/+6iNFs5NXNr/Lb8d84ftJIpFY0FB6t3QLYVL1fnPMFb6e/DYBcKWde0TwuXSRsByvrxINaZVESVLULtJl+74dL1J6GwlVgts2zruhzBf857z+M6yRsLHV1Jn699Vd2frjT3VbOCkwNJt7v8z5/3v8ntQbBrmnk7ovhRlktuoAcGlRFlNZpiO4e7dTIdOmiS3mg9AGfxw8WC5RYxAU5MLmX9XXJCrgo4hevxF+BaiN/DIpk+i8DhMPTpHUeM1HbHMnTRcZdwmQrYWY/Lzm85DDzh88nf0d+++yfG3w06COW32lrSJH2OTDQeQzlFn0eh3N+h0S7bNma44IItPjQYGRq8K6oaEf4TfyZzWaXDGVubi6hbWk0//8ECrWCiLQIh3wUb2gJ8efJ6lOrdW814g2Tu05GKVeSrT2ONuAUOTm2yUJrWH02vaZG3jeSa1Ze0+FsB+0x6IZB3LTjJmL7tG7xt61xeMlh/nr2rw6tpjybiO0dy6yFs8i4oP1tK8B/K14QBXhpEu+L3WdSqOjaza+xPeRLj5ay6pFV5O/qWA/+goPlDF7xEvGH1ji8HhBge/A3HYRFpEUwat4oUsfZOnR6z+7N7G9no1ApqCupY8PzG/jzvj/bdN+1WjBjRB1gEvsanCpChaMGeV/ZWAcG54px04Lj7yd+5+NdHzvl/F229DLuzb7Xoy1fW2L+J59wrkrllvSTMAI4R63m0/nziY6GgNoyuv7xEae+z6G+3MacKgOVJA/zz2YKhLXtHcvv4Pyvz6dSV2ktvpwTeR3F80oYnP054Hky35ToKq8X7GF0UOvm6LkrxMlkMnrFignIkdIjeILFYsFkaNuuM3+IP8DlRANgx/s7+HTkp9QW+saMZERlMCJFnFHucqvssz8tFnh548vMWTqHY6WiwNPQABUhYjI+PHk4SnmT2UN4T+j/DET0gYAYYc/SUOLbF3UD6bieH3YPyy5fxrRu05yW6T27d5spj33FsiuW8U7GO81ev0Zfw+xesxmcMJhu4d18Xk+j0pAYICyGgrscAODYMe9NLM1V/PWJ7cOV/a7kwTEPYrK4vlYsFgsPrnwQgHmj5nHD4Bsc3pchY9Z3s7jtt9s4VX7K1SYI1ZcTmX+YymK9y/c7EjrPGsye8x/Ektq+52BbI3FwIn2v6Mv4x8e3aDv2yvuzhaM/HWX3/N0Y6g2YDCbqK3zLSXBn9QkiX9TXgliy3ePXK/F3TBB/F3W/yO0y/WOGcsGeCh5J/d3jtsxGMwW7Cyg7/vez+hxx1whu3HojwbHBVtWHq7m3RPy5snhsLg6VCEvFIUlDuHHwjdbnTl7UN0RG+k78SfMElTGSiBD/FUXTpwt7yrvvhs6Ntxd3BGfq2FR6zeqFXHn2LbPcwWKxMPaRsfSaLcZg9hl/AKNSRnHszmM8c+4zLtf3RPxJ50RAgDge0dFiTGMyQa5dX6JkARsXB57cxKS6UV2dY+1EUvxFBwjiViazc2xIvhAuPALx57jfcFNEDoJLiyF9rsu3AwIEAQNQVm/74qljUps1jm8NFB0o4sRvJxzud1pjDTKzkgRlT7frSYq/KnPrWn1K45Xk4C5WIrxnTxgQPxCAEuMZj+sbTAY+r7+YrT0mE6g4DqvGwYlWiH1QBkGkY0PcNs2TnEh6huzKXEoOl7Bg4gIOLz3sZgNnD13P68q4x8cRFBVEqaT4M0W7zIfuFiXGpCfKTqDRgEWu4MzF93DZksvE+9O6ccfhOxh0gw/z8yY4XXGaB1Y+wOVLL+fUSTmRtaNJUHWje6yoK1VViflI2jlppE9xtNOU5uiVdeJBHSvX0C3zemRHXvF7P1ziyKuw5jzQ27xjZ/SYwbzR8xjeSWTr1uvk7J6/m1N/uB7Lnk1MeGoCPS/pSa1BTMo1SvfFcCmr0KCoJPuMiZr8GhpqnF1xfG2SAqiqslATIKw+h3ax3Rf6xolmbL2y1LvizyyOZYg6BNThED8Bgl3kybcDQkJEnNCi7Fd4dPWj1DTUkDIqhUkvTXKwoW1vmE1mJ1eLZvEjckVjjI7dxsYtg4tOiIYXb9h8FSwJ9ezU1Y7we6R0/vnn8+abb1r/L5PJqK2t5amnnmLaNOfixP/gH6pzq6nKrvJLddEi4q/R6lOjsnVu2StEJFs8mcy/7YcHhjOmk7CTK4343WoVERzsOeenRFvCSxtfcvDBlyCpAEwmx842gIQBCdYMro4KTbSGpKFJfvuAtzcuePMC7jx2Z3vvxv/gBlotlIau5f5tl/HW1rd8Xs+fnL/EUDH5kyaDAJWZlWx6aROFe1o5ZKSFqGtQUJoyAHWao8WQTGYr8jcdhKlD1Jz/n/NJP8+1X31EWgRX/3k1l37rR8BvM1BXB4WRP/B9XxXTFk6DqMEw9G3njL+mKN4Ai0Pg9GdObzUl/h5c9SC3/HoLu/J3OSzX3t3xP33zDdf4qLq9tr6eH7/5hsBACKWGoNoSDj67m/VPrwfg+G/HKTnSPPJlZ/5OYW0VnkpCSIJV8dcpJpLY4Bi355A9JKJLetZV6ET7pmSX0lrwVIjrHSNcAw4Vu89HsZgtvBjyIsuuWNaq+9UU0vPaV+LPnTqm9+zeTHpxEmGdwpxXaoTOqLOSdvaQSKGmKhapuG2xiGV+OPoDiw8t5mDxQUCQRTHVk7kh/gPuGHaH5x1XR8Dscuj7uOflvEA6b7oqxjOr1yy3xJTFYvE5t6st0HNWT4bcOqTZ60cERvDpxZ+y/cbtKGT+jd2SlOKeqEg6gLrTPtb27M+rf3hWRjeX+AtSBbFw1kIeHPOgM/HbiCOlR8iqyiJAEeCyoCuTyawdwO5UuOpj+0nfvYSykxUe96cjoL5ehkkdRFCor220Zw+7C3Yz8MOB3PHbHeRUtUzRkDAwgUsXXdricXt7WH1uf3s7v9/9O0adkReCX+D3uzwTZhJcNUmUnypnxwc7qM7zvYqdZDcES/ZQuz9RdoIjpUdQypVM7eY+FyUlSYnSHOrV3tKkN/HxkI/561nPlqAdHdLYPNLFsMFTtltzId2XJCXalf2uBCA/6lsiIs1+E3+B+mSn560v+OD4I7ynSuP3ok+sBGd+vrPLQkeFTCZj4rMT6X+VIEMMBqhX5fKD4VZ+OvqT17G2L8SfNH+TyVyfCxLxZ6+6dQVp/FNXJ3doqJYUfxFKMfcLdHb/8w9eVg4IAJVRKImlRjkJ7eVCctEnF/GE4QlUGptH/dz4d5m6p44xQTe5XU9S/JUbW1fxJ41XlLVpgMjcDA+HrjHi82rluR6vTa3B9vAJDEqBIW9B0oWts3MWMxSusVq5BiPcDkq0pUSlRzHtvWl0ndy1dT6rBci4IIOJz04kICzASjBriHF5emZECRLuRPkJ67Ow2hxKyaESa5NpdPdolAH+j4EkO9se0T04c1pORN1Qvj/3OO9dKGo4kujinKfP4eJPLwYgd2suR344YnU2q64Xx9NkCSY3/gEsnVqpPpE6R0SHKJybNqQ6rAUZd+fMY/bi2a3zmc2EMlDJOU+fQ+/ZvdEaxdgkWOl+fmhP/OVtPMPrya9z6DvbHLmhuoEDiw5QftKHwMxGFBaZGXR6EUPLXqVXnE0QIDn86JUlHu8BFgvoaST+AkIEYeSLsqwtoa+C4+9B8V+EhIjGxR/K/82LG1+kWFtMwoAExj40tkMRf3KFnFt238K0d6dR01BD/w/6c/WfkzDL9L7xF8Z6YelZ66KBQu5DTomEqCHivuqD+1p7wG/i77XXXmPTpk307t0bnU7HlVdeabX5fPnll9tiH/9WaGlQ46qHVvFm5zcx6X1niltF8eci46+uzjbIDA3FZUeMJ0jdgpWxNtmtJ7Xf4ZLDZLyTwSOrH+GDHR9YXy+vL8dgMjjkiLjKGLBYLB1alaYt1nZoK1J3iOgcQVRGVLsTA20JuVxO165dfbp+t761lV9v/bXNVTK+wGIRRZqaoAMsz1zKppxNPq9rn/PnDZJdj73VZ+q4VO48fid95vTxa5/bGg2aSDIHXkLYcOeOTHt1jyec+P0ECyYvIH+nTc2Yfl464Z18DDptJurqwKAoB5nFpd2VW4R2h9TZwnawCZoqDaSiTtO8qrrSOg4tPtRswqylqKquJsH7YgDEA1WNVbGgXmnsmvY4sRO6Edc3DpPexM/X/8w3079pVsFge952AEYkC6VYU4tqX4i/plafFfWNxF9g2xB/Ta0+QXTtA6zLWud2fZlcRvcZ3Ukc6r8dlz9o+nt4gzt1TOqYVMY+PNZjN+a23G30eq8Xdy2/y+F1d4o/tdpGBNXU2Cb7J8uFnY9eDxp9Ghcm3Mrs3i4muasnwvbbbP/3pSPQC9xlN9rj5IqTvBj6IoeXtV8X9ah7RzHxWQ8BQj7Cn+evhBiz6KYtVezn+/iB1GgO8Nx+98U4cCT+KnWVVOoqAegc0XLV2u8nBKkyIW2CQyOdPbwRf2EjenF60Cz0ge4LFx0F+TvzCawpaVZhv61x/5/3s69oH+/vfJ/0t9N5au1TLd6mUWdk40sbm61aaA+rz0u/vZSrll9FUGQQ/a7sR8pIL7K7Rrhqkji54iTLb19O0f4ip+XdXb/2ZJ8n4u/X478CMKHzBGthzhWkvLeCAqFecQeVRsWkFyfR90rX9ucdGXu/3MvWNx1znsJdDDslsqe42Hvmnq+QxoTSGHF6t+moLaHUB2SRL9/iM/HXJaILYxueJ7XklmbdH2r0NWRVZXGm8gyRkY2KA4uws2yKA4sO8Er0K5xe7dws3FFgMEBJ2J9s1n/ES5tesr6uN+nZkrPFaXlprFnuovbsigyWVJH2xF9R42Wa4GVQHRoqODm1OhijUVy/RrMRY6PNX7jcRvxZcfB5yP3F84ZdoWgdHHPtEBAYCCqTM/H37SXf8kLIC/5/VhtBrwe5RUVIgHsla5fILvSN60vXSNFI2trEn744DRBqP4DOUeLmqlPlWS0iXaGmQeyI3KxGrUmAHndD3NjW2TmLGTZdDgefAyBYLk7iEm0pwXHBDLt9GAkDfJ3hnR2UNVroa2SuC5OSBXxeTZ71+W3W6VkyZykfD/m4RYS0lNvdM7q3Vb2Zni5ciaQ5ZtNjuevjXSyetRhDfSPxp2sk/mShhA17HHniec3eHwfET4But4HKVlSu0lWxI28Hh+1yz+TBmg4lthivupcxR7YyNeZ2t8tI4wujohJ9cCTD7xpOTC+bJX/psVK+v/J7Di1x3zDbFKUlCuKrpnNu4P0EKG3KFsnq36CsoLLafTyFwQBGuTiWoQEhcGo+LFKILNX2grEWdt4J2cus8/FAxENHmjd1ZBRrizlQfIADFVuRW9S+5fuVboajr0H2Yuf3zAbIXgL5PjTP9XkExn7r0362lDNqDvz+xJSUFPbt28ejjz7Kvffey6BBg3jppZfYs2cPcXEdI/OqPdFScqT7jO6MeWiMk+ewJ7SE+Hvjgjeof6yeJyc8aX3NfoIqDTJdTTy84bLel/HVzK+4IvBL62ueiL+3tr5FdUM1PaJ7MDRpKCAyYC5dfClvbn0Tmcw2+Gxq6bTrk108q3qWzHWZ/u/oWcLXU77m/T7vt/du+A1DvYHq3Gq/yOi/G2QyGWFhYT5dvyd/P8mBhQc6xIBHsr1tUIn2TimLzxf4o/iTrD7tFX/qYDXR3aIJCPUg4W0HSISMq3uWdI90NRH767m/mD9iPhazhaqsKnK35CJXnd2HslYLeqVgmqKDomHrDbDD/SDWiqB4GLtYWPA0QdOMv54xYrZ4tPSow3LlJ8tZOmcpx39xnwnXlggPC8NX7WgREN54gKOjAZmcHk9cybDbhiFXybn8+8uZ+vbUZtmWbsvbBghbR7AVX6Ki4J1t77AiehrFYX/4pPiTiK6hSUO5efDNjEsd5/f+eIIngmhKxhQANudspkrn/iKf/e1sxj3SuvvVFE2trrzB11xKV9iWtw0LFgq1jmeTu9wqcLwvWO19yk8AXkhLixn0lWC0u6GUboWjbwnr3WZCGueU1Bey+NBifj72s9MymhgNqWNSCYry30qtI8BisbC7YDdGs9Gv56+EMJ1Q/K2v/tz6moF6j81f0rEMDIS86jzkMpGLG6L2Pis0mU0cKz3GttxtLt///aSYEE7LcO984o34i+oVT3lyP+osHf+Y7nnqRzJ2fuuXxfjZwNbcrazLXIdSrmR85/EYzAb+/de/2ZG3o0Xb1Wv1bHplE5tf3dysBsP2sPoMjg0m7Zw0AC754hKG3zncp/VcWX32vaIvV/x8BZ1GO9tPubt+7RV/nqw+N+ZsBOD89PPdL4Qg/urUWbzXMJIub3XxeBzGPjyWblN9tw/uKNj9yW42vSwa+DxZfYaGijEJuM+/8xfSfUm6TwWpguhcPxOADVXfWJtvvRF/6VHp9K96lC4ldzaL+JMU2KcrTiOT2WxNXX1PTbSGuH5xftVM2hq1hbUsmLSA3fN3A2L8UxouYgLO7yrOca1eS+x/Yhn92WhrBrQET4o/iQywPye6dBF/29uhSoq/2HgjFouFYm0xr2953eryJCEwEBQKGUqlitpacf0q5UoqH65E+6gWDdHW5QAwamH/45D1jY+/hh1OfQq77hbbaILAQDurzzrbF08ZmUKPGT3apak7b3seZ9Y6qj98GcuOTBnJgdsO8NG0TwFxz3eRkOQAiwWWL4dVq1y/32BssCppKzPTAJGBCdApPAW5RQXIPBJ/tXpxY1eYQzw6bzULciWMWQyDXwcgVCFIj7J653zq9sS6p9fx5cQvsZgtVDTmTYYqXBcm40OEXLZYW0xQkCDIlYZ65GolU99t3vxSghTBkKDshdEo7uexsWKcqYpqPM6VkLM5h59v/Jnig8WMum8UM7+eiSZa3FRrGom/QHmw3+Nnf7HmzBqGzx/OXSvusM6Fio6Ut3vUS9nxMr4890sOfnsQjSmJSO0IOrlogJZgVfwpK9EGRjP17amkjrFFvER2jWT24tn0vMS9lW9TFDempjRVV0cFRSFr9J4srnVvO24wgEkhrs3QgBAIToNOsyGobRtyPSIwDiatgZ73WseBAZYIQDgYlZ8sZ/6I+ez6eJf7bZxllB4rZetbWyk/VU5JnXimhisFL+UT8Rc7Bs7fBmlXu3hTDlvnwpH/tN4O0z5OW82qaiqVSq6++mpeeeUV3n//fW688UaCgjr+RPVswFX+oT/oe0VfJr802a91pCJ2WDOahGUyGYHKQAKVtnYu+6KbJ6sRb+gS2YWr+19NtyRbpp074k+r17Lo4CIAPpj+AXP6zgHgq/1fsS5zHX+cEqHF0iSiKfEXkRZB9wu7Exjuezbi2UavWb3of21/7wt2MGx5bQtvdHqDksPtowQ6GzCZTBw4cMCn6/eq5Vfxr6x/tf1O+QCpQKMP8IH4M5tEl2ae6K6W7he+EH+ze8/mpyt+4sExD9ptzkx1bjV1pc0vcLcFiv/YQ9q+nwiWO7dBSwV+V99ZW6yltqgWXZWOobcO5eHqh4nv58UnxwPK68tpMLqQYnlAXR0YlIJpigqKgordIjPMXxSuhjIRuu2r4i+mVwyXLrqUXpf2oj1w8ZVX8lWgb/fvBUFBXHLllZgMJoK3rSG4LJs9e7LF9WsRyrDuF3b3ex8sFou1sC8Rf/aKvw3ZGzit+B1t4DG/iL+Le17MRzM+sj7XWgvS5N2V4q9rZFe6R3fHaDay+kw7dg7if8afK3VMZVYlrye/zra3XRMvEiTF5vAkxyK3NGZwRVTYZwo2Vfxl6/eRFfMRRaajzivK5DB1N4z+2vZazvew+19Q78WPzgOsVpS6vcxZOocn1z7ptEzS0CSu/uNqt/bEbY2SIyV8PeVrjv/avEaBk+UnGfLxEBJfS6TB0ODz81dCULUg/pQyFQ8NedH6ek5VrrtVHBR/feL6oHtMx75bfbu/rji5gp7v9eTGX250+f6dw+/k+oHXM737dLfbsBJ/5a5/M+k8bC2FQFsi4dIxFGSM7XCKv5SwFO4cdic3D76Z9det5+r+YhL/2pbXWrRdTbSGGzbfwNwNc5s1WT/bVp+VmZVU51Y3q2DuSh2tidbQY0YPl3Msd+PnTp1EsTQoCGJjLVYVUVMEq4IJDwhnVMooj/uVmAgBhgRKlfvIq8lzGsP8N2DmVzO5+g9xzroieezhSunVEkgW2T1ielhfiy+5HIBNxb+hVotzSa8XVnSeID1vm1MaSm9USp2qEPlRnoi/9PPTuW7ddQ7F2/aGvlZPwZ4CqnPFIFGvh5pAofAZ1Umc48HqYKsiRFL/SJDqJBUVzoSRKxWodB6cseOoJOJvp+kzer7Xk/hX47n/z/v5/sj3DtuTySAkxEx1dTWVlY4fplFpaGgQ9zorUSQPhOlHoN/TXn4FF+j1AEzeAHLngaA7q8+xD49l9nez26VAuuqhVSyZvcT2/9OreDpvBMeSnvBpLGtfbPbUxGaxwEcfwQcfwNtvu35G5FTnYMFCkDKI7KOipiYRf2M6jeG+Oh2jjq/xifhTmkKINq6B5QMgb7n7FfxFwkSrgjBMKc7tioYy9LV63uv9HqsedsNqnkVUZVdRfKAYmVzG0sl7mXDwEDGW3i6XjdWI37m8vhyj2YBGA/qgcC769Wa6T/d/fmkPifgLrhcHMT0dNuVsJOLlCJZHXwCI678yq5I9n+6h9GgpcX3jrPbBADUN4kQZEGCh/vsemE9/Tasg/w/4tZeDukmKqajQVVibAFbf/RMLp3q2129rNFQ3UHSgiLrSOp+cZeytPqU6kLbEdsFpojX0uawPsb1iXawtriFTk+y21dnLyY9cbCVsJSjlSsIDxO9WWu++fqrXg1FuR/wlXQDjlkC46/PyrECugvhzISTNeh9Tm2yKP5lC5jYfsb2QuzWXP/71ByWHSijWCjY2RC6Oo0/CKEUgxAwHjQt7CrkCxnwHQ9/1vA1DNWy5TtQBfEBLOaPmwO8WqQULFnh8/9prr232zvwPzYNUJPCJ0XaH8j2gzYROM11afTZH8SfB3uLFHfG37MgyavQ1dI3syoS0CdbXByYMBGBH/g7MFjMajZzycmdrk/Tz0tutAOYrxj8+vs0/w2KxtPoAOWVkCiPuGUFgRMclVVsDvt6AZXJZh1FZSBMEozfiz2wUReptcyG4KyRf6Kj42/8kdJoFkQNdrt4zpqdVKSahtqCWNzq9wYh7RjDlzSmt8G1aB/VHsojJ2Ud4tHNWjPSdXRVWp749lSlvTaH4YDFBkUF+hTs3xcHigwz/ZDiTuk7il3/4bolTV2en+NNEw9g9vgcEZy8VFhFD3oY1kyEgFi4tdiJRpKJO0xy0wPBA+l7RftZYN950E88+8wzbgBEeltsGrNPr+fLGGyk+WEzDyg2EZ1goL+/Oj//8kYMLD/J4w+Mo1P4rcnOrcymoLUAhUzA0aSgWi6PiTyrUeAvrbpol1lbwZr01NWMqRrMRndF9i/6ez/aQuzWXCz+6sM2KK80l/uyLJfpaPaHJoahDPG+kqWJTgjurT3C0AM7o6kj8HTAt40Das3yXfw1z8Tz+BSD9RqG8bUG3pnTeqAzifCut61hd0wA1+TVkb8hu9j1jY7ZQ+fSM6YlSrvR7AmQpy6Cr9j5uvLgv/5p4FR9s+prqwEP8dewgV49wViWB83moUqisanZv6B8vCi5HS4/SYGxwsPQBuKTnJVzS8xKP2/Cm+Asw1NJ/1cfUlfaFhzyrn9obQSMHUHYMBnYwxV9KWArvTLPZyd0/6n7qDfXcM+KeFm87pmeM94Xc4Gxbfa55bA0HvjnAIzWPoA5Rk7cjj40vbGTEv0aQNiHN47pNrT4N9Qbqy+sJS3bfXerq+o2MhMceE9/92Q1P8+LGF9l0/SaGJQ9zWG7BzAWYLWavJGVMDAQoAoisHUlZ2DrWZ653GpdKWHHvCnI25nDTDs/2vx0NkV1sXbbeGm87d4Y9exyVXs2FwWRgdu/ZHCs7ZlW963QQUnYOsk4q6ow11FpKgDjre+5I/72FeykwK5DL09E0ozMgPaqR+Ct3JP5a43ueDURlRPFQ+UPW/xsMIucJID7Y1kzYM6YnpytOc7T0qEPdIyJCRKuYTKIGY183kc4JezJYIv7Ky8UYJjTURvytK1vI8bLjBKuC0Rq0fLrnU67qf5XD/oqiqMXlvEiqtVh78uQKCPddDeOASPeNzwEBoDbGEGCO8i/moA0x+sHR1JfZik37i/aTadhOYmBnn8ayCgVogi3UaWVUV7uuo1ksgvD7/Xfb/wsLBRFkD51Rx+hOozHr1TToZAQF2a4LhVxBVOM9wqPVp14cYIU5BLVSD6Z6oG2yxMLVMaCDyoZSFAEKZDIZcuXZt7Vrios/u9j671B5HKG6OILdzNOigqK4st+VxGni0Jv0aDQqtNqWP8MtFos1409WIsid9HThNlKrr0UrO4RBUUVlZThjL+7JQxUPoQpWYdKbHOa1PRXTOHf/KeZM24/cfLeo87QG5EpRL7LYnulSTEVFvSD+qquhy2VDiQho38bvpKFJPFgqGtJffOZzTsWXU2aeCbhW/cUGx5IW3JP6ynQq9bD/6/38esuvXP3n1dSV1hHXJ46ojCin9Yq1xTyy6hE+2/sZtw65lQ8utMVRLa96lcz0tZyr+gL4p8N6dwz5F0uWWDDVRGE0CivXpjAYwCQRfz64j5w1WMxgaiAkRNQ7FcYIUIhzILJ3JPfm3Nu++9cE3ad357r11xHbJ5aNWWJ+qbGI8YpPxF/1MRGX4y7PL9l9UycglOx1eXDmS9B0EjXVDgi/78L33HOPw5/bb7+d6667jptvvpl//etfbbCL/39gNpr5bMxnbHrF95wusA0Em6P4e2HDC1zzwzVYVgyDDbPAYnGYoHqyGvFp33RV/Kl9hd1drsKCxS3x9+keYYkwd+Bc5HYZOX3j+hKkDKK6oZrjZcet3YNNFX//31FRX0HnNzszdeHUVrfE6Dq5K1PenEJEWkSrbvfvCGODkTNrzlCT3/Yt+TUNNTz/1/NMWjCJPQV7XC4jDUD1ai/E36EXYdV4GPASjPwMsE1CFNqjcPgl2HGHyGDwEYGRgYy4ZwSdJ7Q8H6k1UTPpYnZPeYSIWOeHt7d8ttWPrOaToZ+QsyWnRfvwwoYXqDfW8+vxX50sNT1Bq23M+KNR8Qe+BwTXF0Dxemgog9GL4PzNgDOJIin+CmoLqG5w/iHMpvYJlU5ISOCll19mRkAA7jRd24AZAQG89MorJCQkEN8vnj7v3EJJp8FUVSlJGZmCIkBh7bT2F1tzRbbOgIQBaFQaKitFZ7tMJp6BUgeoXlnil+IvvyafSl0l5lYO7PZG/L08+WVO3X2KK/td6XYbmWsz2f3Jboy6Vpo0uoC/xJ8rW7y4PnHctP0mBl0/yO16+TX55FbnIpfJrRmHEjwRf/ZKq27RouiZV5NHnaGOXIvI4BkQ7UKNUrwBTnwo7D4lhHWHuPGgbL4USiL+FA024s/Vc33tU2vZ/t72Zn9OS9B1UlceqX2E/tc0z8VAutbGdBrj97oWC1RXKumd+xo3DplLkFpNglwQkFtOHXS7nnSdNIeQTwlLISIwAqPZ6Nc93R5SQb1YW+wyJyMiMRB9YBg6ZQea/LuBqxy4joiBCQNZevlSq9KmpagtrGXVI6so2O2fovdsW332uLgHYx4eY22UMOlNHPvlGGXH3FtOSWhq9XlmzRneSHmjWbZOI0ZA7z5m/v3XvzGYDdz/5/0ul5PL5Ci8jHXkcpFZFqkdDcCuAvf7Y6w3Yqg3tNt4prnQltiy4L013ko5f62h+FMpVHx44Yes/edagtXiIVlRAUpzMOedOEjxAyUkR8Qh9QZ5svu87dfb+KNrf4rD/mjW/aFrpCjaVugqxNy2cYrhSvHXUN3AumfWceznjqv+1Ost6JWieSc22KYk6RUj1D6S+keCTGYje0uaCEVc1WQ0GpCSds6cEcemqkrY4m4v/gsZMv64+g9kyFibudZKqEpoOkf4/sj3TFs4jXe2veNgjw2ASSfmGGb3GZseYTaKbTRBYCB0Lr2ZK7LKHIrqx34+xm+3/0Zd2dkv+nSb2o3+V9vGN9JzP0TXw6tt/eVLLify5UiqYgWj507xt2mTIP1kMltxusDFo6VvXF82Xb+JG5RrAejRQ9wPJUT6QPzZFH+hGGKmwIzjLuMhmo1NV8Ji8dCIDBDFvkpDKQqVgtsP3c7E51qeB92a8KYQU8gVLJy1kDemvEGwOthBFOENtbWujyMI9abWoEUpV1KdJRjejAxhLdologsWLFQGb6eiQuTVBkYEkr0hm+c1z7P3i73W7Vj0GoL1XVEEXcSRjJ+xdLnGx2/uBQmTYPohh3NDUspV6CqsddjE8/sx4i5PbbpnF1vM73Ck0zwKDO6fBX3j+rJixhGGnfyZykpIGJhAdPdoZDIZP177I4suWuQ011p1ehXd3+nOZ3tF7eyT3Z+QV20LnC1udIMZkOTcEPHvSU/Qo/BJAg1Jbu8Bej2E1vcntXoOgxMHQ+Y3oh5naF4do9XwcwasGm8dByr0EUDHzfjTxGjoPL4zmmiNVfEXaBbPW6/CKGM9/NYHNv3Dy3J14vnXFBYzrL0A9j8Bsyug17xmfIOzA7+Jv4qKCoc/tbW1HDt2jLFjx7Jo0aK22Mf/N9BV6ag4XUF1nn8Xu70Vmb9YeXolX+//mi2pd8DAl0AmcxgEtlTxJ5fJef/o4+RHf4M24ITLfTxRdoK/sv5CLpNz3cDrHN5TypXWAt72vO3WSURTxV91XjUr7l3Bid9PNG9H2xi6Sh3fzfyOvV/ubZPt/3j0R7Krsvnj1B8Ymjsg/x+sMJlgyRLY0SQWpjKzkgWTFrDtHc92cy3F/N3z6fJWFx5f+zhrzqzh8qWXWwft9pCKSDqFSHJ3S/wZa8TDqus/IUIUSKVrOrO8J5y3CdSRsPpcaHBOlDeYDHxz4Bte2/ya1a5JHaxmyptT6DWzfawh3aGmVoZZqSYszFm95M3etO8/+tLzkp7E92++xWeVrsohk0sKZPcF9oq/ZAUiTLjOvXWdAzJugstqIXYUpF0BoUK51FRpEB4Ybj1Pmqr+lly2hNcSWmaJ1hLce//9PPLCC4xXKLggMJCFwCrga+D8wEDGKxQ88sIL3HvffQDIlXI6DUtAr4mgqkrBkFuH8GDpg0R2bYY3NTCz10z23LKHNy94E7Cp/SIjRfeuVLRp8ED8WSzORNc5X5xD5MuRbMr2r6nHGzxZfQJOqiRXmPL2FB6qeKhNM3KkXJSWWH36Asnms09sH6fcNk9EhX3GX1RQFJGBkajkKrbkbCFfKY7Z0PjRzitmfg07bmvsnLaD2QSm5lugSEU2ZUPj+WZqQGtw/jF2f7ybAwsPNPtzWgqZTNZsZbRkd9knto/f69bWgrGRp5YKoLNjnuGcA8cYab7P7Xr2RczH1zzONT9c4zazrylkMhkD4gcAsK/IZg9qsVh4bfNr7Mjb4ZXYDw0IZcElC/jrur/QqJxPxPAoJUfH3khpdxfnWgdC9qZssh96j4iCIx2G+LNYLDy7/lkWHVjklGPVmqgtqmXTS5vYv3C/X+udbavPPpf3YfKLttiIlBEpPKp9lCE3D/GwlkDTJonQpFAG/HMAaeemNWtf9CY9I5JFgfB0xWmHwponNborJCaCRieKpTnV7hu0LvzwQm4/eHuLnBvaA2+lvcXi2YsB74239lafbRGBJo1/Ood0RyGXI5N5bzYCyG0sigYakpp1fwhRh1iVcacqTlm/Z2Gh81jHbDKz/un1HPul4xB/tUW1HPnhCJVZlQDUGKqxyMUDS3KNAPd522BTc33wQRPLc7FJp3Oie6Pz4NGjUCSmg5QmiBy+CWkTGJM6hvPSzwPg872fO6zbNPt8f9F+fj/5OweLD1qPs5X4K/gTlsVA1rfufwB3KN8N3wXAsbec3pK23/T45m7LZecHOx2Ud+0FyVo42Afir95YT6WuEpNGXAvu5guSinXSJBg6VPxbUms2xerV8NNP4t9TmxjaLDc8xMaeo9hevM7tPvWL68eQwvfpWnRv27iRhPeB+IlgNjEheg6jj27gAuXzbfBBzceplac4sfwEpytO88rBuzgd96bP8xJ/nuFPPgm33QalLsw6EkIS2HnTTr6btZTsM+JEkhSeUoNSRfAWKivBYrZQdKCIqpwqul/YnahuNjWabTzb9vmXktWn3qRHGShuCk3rsO2BijMVHF52mNrCWhoQF1l4oGcVjESSV1VBbJ84bt59M51Gd+Ifv/yDqW9PdXK+eWb9M1Q1VJESlkL/+P6YLCY+3vUxAJX1VdQpBMM7rKsz8SeX20gnd/cAgwFSyq9iQsm3Qo1dtAZOvC/IpPZEysWQMNm6/zKdzeoT4MCiA5xY3nFq7g01DdZGLynjT23wMePP3AC9H4ZOl7pfpjYTloTBoRec3zPWQVASBCaAOgLULbBJbGO0yoi4W7duvPTSS9xzT8utVP7ukMub/5NqojXcX3A/U99ytqjzBGlw3hziT5ogl8ZOgt7CmsK+q0UqkDeX+AsNCGV8Z2FxWRz+OzEunHISQxOZP2M+94+6n5Qw5wR4Katne952t4o/fY2ebW9uI2dTy5Q6bYX6inqO/XyMkkNtk5O34tQKAB4b91ir22QU7S/i20u+7bCkamtALpfTo0cP6/W7ahUsWADvvee4XFBUEFPfmUqPi3q42ErrwGKxkFOVQ72xnu7R3UkOTeZk+Unm/encQVJbi/D9l0UAjjYyDhj0CkzdJyTsJh1osx1JsOhh0ONuGPIWyJy7rhVyBdf+cC3zVs6jqLaolb5p28CSX0BQVSGu4uK8Kf4SBiQw+7vZqIObfw0tPLAQrUFLdFA0hicMTMnw3Qa1rg4itSPpqzmPnpTDxsuheKNvKysCHdWB1SegcJVLpcGnF33K5us30yfOsege2zeWzhM6Yza234Dz3vvuIys3l/GPPcaDYb2YE5DEq+m9mPDYY2Tl5lpJP4DCfYVEhBiRycBiiUKhUHi1gvQEpVzJwISBjOs8DnC0+QQ7xZ/KPfFnNNqKcNLkukIn2nClyZMVFgvUOHZf+wNfinAgiPv8Gtch7EGRQQRGBLZphoov2Qv2cHXO7vpkl1d1m0T8SUVme3hS/DUtfB28/SD1j9Xz7o53McrqiawdRb9YF8q2fk/DhN/EYF9C+S74VuWyuOUrpPPG1KCx5i+XaJ3HDjdsuYGrll/l9PrZQNZfWZz842SzHQZOlInxRLfobk7PX2+QutpDQ7EW4EZm9CCkoTtZme5VQ1KhIigIlp9Yztf7v/bLRlWy+9xfZCN99hftZ97KeYz7fJxPJMY1A65hXOdxLsdp0nnoKQ+oI8BYb8RismCRy11eT+5QrC1mb+Feh9e+2vcV7+94328CyNW2n1z3JFd97/p6OF1xmruW38WCfT7Y9XpAwoAE5m6cy/n/8c+K9WxbfTaFXClHGeC9ucNicc5DTRyUyCVfXEJ0N9eTTG/Xb6AykNXXrkatUJNXk+dgdTvxy4l0fasrG7I2+PQ9kpIgSC9YkewqFxKwvzEsFgsDrx9It+lCGSzdB9zZVKWmCqVQTY2NEGouimqL0OodT07pPisVSi0WC4FB4n7vbsxhtpgp1IpCqMaY5PMzvylGpoxkdKfRmMwmwsPF2N1igdwmfXCB4YHcuv9WB5K7vVGwq4DFsxZzZrUI3asyiGdMkDzE+jwH94o/gJtvFnWX06fhqadszy53zdi9G+OgNh3MYeZPE1jbtzuHY58D4Op+IjPyhkE3ALBg3wKH53ZYmIzg4BC0WnH9SmqWxNBEZ+JP0wkyboawZth9BiVDyiUQ4hzLIo15mp5XYx4cw7zieUSmN6+ZryX44pwv+PZiG8EpNUqG6Hp6Pa+TQ0XGjT5InLDucnuleUR0tFAzg2vi7+BBC+82xkvNmQOjm/QGFVsOUxmylaxa93WaLpFdSC26jaSKOWh0u+DkJ67VK81F38dgws8gV5AankpU7VgCG8S9eu+Xe/1ulmkLrH5kNcvvWM7J8pN8n/suOTGfezyWBpOBgpoCKnWVPiv+6urgxAnRQO7KnlitUDMkaQiDgy/GYBCEonTs+8WJ7Oq6gDOC+LNY+GjgRxz/+ThX/HiFQ5bpvrrlHEl+mALdfHqrNyGvz3P+sObAWAfH34dCWzZ8iDoEGY1zxMBKAE4v3cVHgz+iMrOydT63GTiz5gxLZi+hcG8hDYiLLDzQs6+jVAfS68X9Rpr7dh7fma6TnS1CnznnGd6Z+g4b5m7g0bGPAvDx7o8xmAzsyhb37gB9EunJzoXyKl0VpqjD1Kuz3dYMpEZhazPBkHdgVhGommHj15oY8gYMfNF63qcW3MOW63Zyz0jB9fxx7x9seN63cdvZwJLZS3gl6hVA1HJiNDGoGkRN1KvVpzoCBjwHaR4Uf8GpkHoZRA52fN2oBX05jPlW1FD9QEs4o+ai1T5RqVSSn++6sPQ/tB10OttEsjnEX51BPMFiDSWw+34o22G9yA0Gm82Eu4wBXzA1QxCZ5q7L6dLF+f0QdQg3DL6BV857xeX6UlaPveKv6YM3Ml34DY97dFzzd7QNEdklkieMTzDx+da3OjCYDPxx8g8ALuzeirYNjWioaeD4L8epOO3BQ+K/AOrG0Z9eD5J4ubzcpiwACI4NZvidw+k0ynWGUGtAJpPxzLnPcOKuExy6/RALZopC1Ue7PmL5Cccg7tpakCFjnuYg227cRmKoh1wphVp0EP2YCpuvJiqkihcnTSBN1ZhBl3i+IP9cdKrIZXLiQ8QDtKDW5l/xwzU/sPKhlS38xq2LqE2/kLHrO5eDeWmy7MmmsSWwWCx8uPNDAJ4Y/wRKuX8qKq0Weue+ynPd/yS921XCsjO2meqPjbNh89UEa8QEv7bWRkhN6zaNUZ1GOalOznnqHC5fenm7ZzEkJCTw2OOPM/r8w4w+L48VGw/z2OOPk5BgI1gMdQY+HvIx2x9aBkB1taLVu96bquklxZ+njD9pEA+C6LJYLFTUi3un1b5VwsmP4LfekL+iWfvnTfEH8OvxX4n5TwzX/uA6f1lXpaNwbyG6qpYV3j1BUvx565KW4Crjb9tb29j6xlaP6w1PHs6V/a7kgowLnN6TxkneFH8ASaFJ/HL8F348+iMyi5J+mR8TEOCCGA1KhORpYE+aBiYKb38XxS1fIR1XfYPMqhBwRVBFpEW0W/bu+mfWs/Typc0ijOsMdeTViCKFZH+p9qNCLBWk7VUPku3dmTPu17O3+pSU2GkRaT5/rkT82Sv+3tshuoNm9JjhUsXnD0JDITL/EJG7VzuMOzoauk7uCnfdSVV8D78UPRd+cyGDPhrEj0d/BGDFyRVc++O13LH8Dnq/15tlh5c1m0g+VHJI7FtkV4JUzhnMX+//mnd3vMtvJ35r1vbtkTomFZncv/P+bBJ/mesy+WTYJ5xa6dhUUnKkhH0L9nn8jevrbeOE4GAwGXzL3vR2/Qarg61NoL+fFPZ3DcYGdhfs5kzlGc9jVzskJjoSf+6+S9nxMna8v8Oquvo7QCaTMe2daQy/YzgWi/esYLVaEKHQ8vy725ffTsiLIXyy6xPra/aNT/9a8S9S30ylJkTYq7oj/krrSoUriEVGpCqB5vYT/XjFj2y6fhMjUkQTj3R/P33acTmZXEZ8v3g0MR1Eeoywj5u1cJY1BiFYn8bkfXm83d9RXS4p/rKrsp1I1+RkeO458Uw4dgy++EK87i73sU9jD1/msRCmhM8DixyjrI7ooGgu7S1UDDO6zyBIGUROdY5D80poqChASuMt6V7aI7qHs9Vn1CAY/pFoFvUXQfEwbhmkznZ6KzAQ9IoyNnadzLBPhluv68DwQIJjg9tFuRsQGmBtJKw31FOkFU2vwQ0ZXseyUhO7TiXGOe6IP+n1sDAb+ePKInLKd+fyR68uJI5ay1UueluSQgTRWFTv3iHGYrGNxTUVv8D2m0VERBugaVPiX8/+xZZXt7TJZ/mDyS9NZuo7U63jabUx2iPxd+2P15L0ehKf7/ncZ8Wf/RjUleJPwtbG6Uz37rYphH2OfEUFyBVyJr04yaWl/gnjGk4lvoyyYRnqvXdC9WHPO+YrzHrYeYewnGyEXCYnPFAUUCwB4ibUUGtEV6mzWlO3B7qc24VLF11KwsAE9DIxKY/UeCbMJn8zltX9U9EGnPSpYWZil4ncOfxO0iLSmNlrJnP6zOHD6R8il8nZdkbcKyONvVzeE17e9DKLY/twKv5Vj4o/k7wOpbpx0K8MgsA4kbPYAaDRiPMzWN+V7qFDiAsWKrqZC2Yy5S3fG9vbGp3P6UzfK4Wj2SvnvULJAyX0qhSN4l4Vf75AJocxi6BLkxtw0Xr4qTOcmu97NE87wu+z6ueff3b489NPP/Hhhx9y9dVXM2aM/1kd/20wm5uvlCg9WsqhxYeoK/Xdy1wqTAYGNi9ro94o2sgGH30Ujr4OpdsICrI9hCQut7mKPxBFZoAzlvXUubCr8gZp4N9gaiAwSPy+TSXmCpWCsJQwVBofK4vtAJlMhkLV+jeFTTmbqGqoIkYTw+DEwaw4ucKqfGguTlec5kCRsBDrNLoTTxifYPgdw1tjdzskzGYzBw4cwGw289tvtuvKYvHsmd+WSApNQilXMrHLRO4dKUJ0X9z4osMy0gA0JETG8OThDvmYVmy/DfY/Lf4tk0PGLZByEVGy3WRE7URpKvGJMEkMEYWZghrbRCHrrywKdrXNxKG5KMkYSUHGOJeDeW+Kv5aioLaA0rpSApWBXDtAEC1Gs9GaZ+UNDnaEwZ2EZWdwqsd13KLfUzDsfYKDxT3TYvGuDOtIsC98uTqWJr2JCU9NYMDV/UTmV3U1tbXNf/7uL9rP3J/msnD/Qutr0n2gqeLPKK9xew5J+yyTiSDvGn0NpsaQdCkg3YqoIRA3AWKalz/li+IvIyqD6oZqNmZvxGR2LuIeXnqYjwZ9RM7mtlPLS2Sor/ZCroi/q1dczeXLLndatqHB9v0v6XkJC2ctZHZv58JSUxWLPZoqreoN9dz9+90A9Cx9kDBdX+d911dBvYvWbE0SjFsKqR4sQ7zAntCVzjlXxJ+uUkfxweJ2UehOeGoCF37cvEajk+UnAXE9RGuiHZ6/vqCpEgVEYfh03FusCL6KA3nOKlr7+4leXmVV4XaO8D2jVrL6lIqmZXVlfLX/KwDuHn63T9vIrspm/u75DvcZCcHBEFF0jMSTGykv6tiW7f5m/GVXZbMjX3inX/391VTUV3Dbb7cBogP+TOUZZi+ZzU2/3ITepPe0KZc4XCKKXk1V7BImdJ4AwPrM9S3OwTY2GDn15ynyd/ne7Ho2ib+6sjpq8mucyMnNr27mx3/+iK7S/QND2j+lUjRqLL9zOR8O/NBjY4in67e8vpzvDn5HRX2FtQlUIv72Fe2jwdRAdFA06ZG+NUokJECQvhMxhkFM6jKJBjeWynnb81h+x3K/sxg7CkwmGwHrqTBtb/fZEkgqzE7htqZG+/tsVlUWudW5FIesAtyPOSRngQBjHCGtOB+XrPBOuBA06Sp1zc51bguEJoXS78p+RKWLgaPJqCDQkER6WG+H5aI10dw+9HZemvSSdYxoj7Q0uOsu8e+9e0UTqkQUNa3JpKWJe4y5LpK6Q+cyIGs+TyZu49idx6z5XEGqICZ3nYxKrnIg/kJCLNTUVFNVZcFsMXOwWOTk9o/v36JcXH8QEABySwClYavZmb/D2pTeUN1A8aFiGqqbb53eXPzjl38wa+EsAIeoDaUpzGfFX61CEHHu5gvS62FhoqkB4FDZXm779Taru06lrpIC+VbqAzK5/MJYl2R650hBNJYb3Ku+DhYcpzR0LXXqTGRd/wnn/gHBaZ6/iD8oWAm77oP6IszKWs7EvcNak1CdXrroUi758pLW+6xmouvkrnS/sDtldWJy5434i9MIkqNYW+zzM9y+OcEV8ffyxpd5e9s7/LJGHN+JdnqA6CDRZapXllpJqTEPjuHoj0fZ8b5j9ky9SezIMUVvTqW+jznCff65X1CFwaQ10OdRh5cfHfso/znvP9Z5bOSUEdxz+h5ie8e62spZQWTXSPpe0ZeA2ABMMnGzigjyTPzlVudSr85Bryx3G/viDmqFmm9nf8vFPS9ma+5Wntx+KwCJSteRN1aXIA/xIHo9bO1+Hh8nqERTXPVx8ae9cXoBbPknMrPeSpzZz8nTz08naWhS++ybC4x7ZBwXfuA4J5Wel14VfxvnwNbrm/fBmiTofhdEj/R71ZZwRs2F38TfJZdc4vBn1qxZPP300/Tv35/PPvusLfbx/w2O/3qcpXOWUnbCd+l9S/L9wGb1aQhMFP60GTcjk9km81KhpCXEX8+YnqRHptNgauD7I987vPfwqod5e9vbHsNCO4d3puKhCvbduo9gjThlXUnty0+VU3GmY6rSagpqOL36tF+krq/47bjoYJ7WbRovbHiBqQun8sIGFx7EPqK6oZqxn43lyXVPUlSq58cfZdTWtp0NXEdCXZ3I9rOH/cBt9WOr+XDAh9SXt765ucVi4bZfb2N95nqn956f+DxfzfyKNdeucXi9orYek0zn2W4r93sosbOLHPAs9JpHUNq5XPtDAWtOXSkmdxYLrBznNtxW6si2V/z9K+tfXLvKtZKovVCS0J/S1MFeib+2yERJCk0i619ZbJy7kcigSKobqkl7M43Rn44mt9p7Vl91XQMmma51cpM6zYJOs1AHKFA2Cg+lCUtZXRkf7fyIlze+7LBKVU4VK+5dwckVJ1thB1oGV5aZ9giMCGTCExPof0Vva6ddS4qqa8+s5Yu9X7DooC2ruOnztXdsb07fWMvkA9luzyF7aySZDKvaL0AR4KhEKd8NumIY/1Oz/eDd5aLYo1tUN1RyFQ2mBpfnYPKwZM599lxrkaot4GRl4gWurD7DUsJIGOCYYWowwLx5cMMN3u13PFl9Sp8nTRJyqnPoE9eHLhFd6JLzOODiHMz5Hn5IhNyfaW3YE3/Pnvssyy5fxqBE50n9+n+v54N+H/idC90a6Dy+M33n9G3WulFBUTx77rPcPcI3sqwppIJ0lN0pGxoKxfGLyI/+hpUHdzmt09Bgu16LG4Q8JkYT45QF6Ql94vrw5PgneX/a+5gtZj7Z/Qk6o45BCYMYmzrWp23sKdjDTb/cxOtbX3d6Ty6HmmET2T/xHmrq2y5zs6U4+N1BjDv2AL4TfytO2lTNG+Zu4PUtr5NZmUnn8M5k/yubx8c9jlwmJ7sq22Yp5QcOFYvO694xvV2+PyJlBAGKAIq0RQ5Wk82B2WDm6wu+Zttbvmc9S/ed+npB6rQlel/am/vy7qPLREeLlcE3DuayJZd5tPy0v0/KZBAQFkBAWACB4c1TFq84uYIrll3BpAWTmJoxlb5xfa1WzFJD1MiUkT4rh5OSQGEJYvyR3fww50cH60R7dJnUhWtWXUPncb4T++2N6txqFs9ezOFlhx2e6W1N/JnMJqv1cvfo7tbX7RV/k7sIK83cAB+JP33z8v1c7RvYMuxcEX8LJi/gs7Edt/4kqbddjX/em/4eD419iLAA18Xqvo2P2Px8mxJMLncuZMrl0Kux9nzqSAhRtWMZlTqcaI1jcejtqW9T+mAp1wy4xvpacLDNFSSrMosafQ1qhZru0d2drT5zfoCNV0BtE+mlrzj6Jmy/xelllQqUlmDkZnGyl9WLwffRH4/yQd8PyFyf2bzPayXoTXqSQ5MJNicgQ+6V+JMUfzUWz4o/e+IvvjGpY1XkZXy460OrbfXXe7/DLG8gtK4vozNcN7akx4rPq7Lkup3bfrz7Y7b2mEhm3HuoIrsIlx9Va0hhGlG2DY69AboClGoDh1LvZlPAEzQYG0gelkx8fzdRJO0A6fxSG2M8ktqS01Gxttit41hT2Cv+JNc0CRaLhRc2vsA9K+4mt1Js0962tVt0N+b0vIaEyouoqgKzGfS1eg4sPEDuVsf5m84sHtbGgERqQkZBgIsspeZAJof4cyHUsRnngTEPMG/0PGIbydCO1Ehco7ddYFEhnlkeqRHCqKj0SvzlVufyzYFv2JG3w+m9YcnDCFFEEVk7iklhd7pc36rgVLl3CTIYwCgXjFqwKhi2/BNWn+t5x84GSrfAmQVgqCY0FOrUmby18xU+2PGBdRGz0YzF3PYZk82BxeLdLt0KbTbU+WCVW3kQ1k6F7GW21yIHwtC3IdJFHEgHhN/En9lsdvhjMpkoLCzkm2++IVFqV/kfmoWeM3sy+7vZxPT0/eYtFSZdZef5AqmrKmfEIpiZJ+wAcS6OOYWLn/gQjvrmZSuTybhu4HUAfLrnU+vrxdpiXtvyGvesuIecKvdqA5lMZr1RSw9eV6GyHw36iN9ubbmNT1vgzOozfDX5K7I3tk4mRW51rrUr79cTvwIwvdt0Lut9GQC/nfjNr+wae8zfPZ+C2gKyKrP4/VcVn803s+Tl0xTudZM4/V+EFSvEAD05GXo0xviV2fHwMpkMQ72BgPDWb3/87cRvfLjrQ6Z9M43y+nKH94JUQVzd/2pUCseZ49qKz1k5IIE/6xyVgA6YWQjjf3R6OTAQTPIwDOZAMfiRyUQGoMx1dT4pRHT2uMsK6wgwGsVAGVwXSyTiz2hsu2BqlULFkKQh4vMCwkiLSMOChV+P/+p13SPG5fw+JIiHD0yHbTfD94nCb78FkBmqnBRUlbpKbv3tVp5e/7SDCszUYGLbm9s4s8aDX95Zgn3hy1u3sVS4aAnxtzVPFCFHpdjUd00z/hRyBYnR4sfU610Tbk0JJuladsr3O/kRrL8QTHWCBDz4nFCR+QF3uSj2UMgVdI0UuQWS0soe8f3jGf/4eKK7N7N7yAdIxJ+vbo5NO2t1lTqqcqqcJhgrVoiCZ3U1bDx8kiMlRzC7CEQ3m23Xuy8Zf+mR6czqOYs/rlyL3CzI2qCgpitlQPqNENXE7x+Ewnrf416/pzvYE3/Tu09nVq9ZJIQkOC2XfkE6E56agCro7DodtFQxlRKWwuPjH+fpc55u1vquFH8AqQEiH2VH9kGndaRrRCaD/LpMwD+bTwCNSsMz5z7Dpb0vxWwxW20+7xlxj8/ERZdIQca4yycL7RSBXhNBZVXHbbba+vpWAresBVxfT64gqbyePfdZBiUO4r5R93HT4Jt4f/r7xIfE8+zEZ/nz6j9ZfNlip3GOL5Ds6dwp/gKVgYxMER2567Ocm6v8gTpEzYxPZjD0tqE+r2P/O3krHLYWmp6TnUZ1ovfs3h6dUZo+v87/z/nM/Wtus/dBsqe/IP0CesX24sBtB/j3uf/GYrGwKWcTgPW4+ILYWEFy6PW257MrhCaG0nVS1w5lAekN2hItR5Ydoex4mYNzgKeGmdRGQ4iWJK1kV2XTYGpArVDTOdxGlEq/b2QkTO4qiL88+UZMsnq3Yw6puSjQ0DLi73DJYbq81YVu7wgraIn4y8x0tFMH6HdVPwbOHdj8D2tl7P1iL68lvUb2JnGPP2pezqGU+9hW8Yvf2woNhThRZ2fnTvF3eDguVV9pPSs5lvQUheE/Y8FCgvOQgbSINCeS0X78IykBe8f2RqVQORN/lQcg+zuRa9QcFK+H059DE/cJmQwCA2SojGKwLY2bEwYmMP7J8URltF1jmjtsfm0zx34WuX7JYcnk3pfLDZWCffXWxJYcJhR/5SbfMv7CwsR1pgiqRRsoxumrz6ymoKaAL/Z+CUCXquvQaFyPCzLixOfp1Lnu1YX1YgKotoQgl7WB0qTb7XBxFoT3ISY0HJlFOFyV1ZdhsVgw1Le/i8G7Pd/l20u+tdbHVMZoj8dSsjUsrvNd8XfKznCiqeIvtzqX6oZq5CgJ0fVg/HjH+W3fuL4snL2AbkWPNDrZwJY3thDRJYLzXjnPYVsNjcRfWIAfQcu+wmIBo+siibS/tYU17Pl8DyWHnfPHzxbWPL6GVxNepeCMuC7l5kCCAz1fnFI9Wa8o9+rotTlnM1d9fxX3/nGv03tqhZonog4y5uhm+ib0cLm+LR7EveJPWH2KazNEHSIyVHs94HnHzgYGvQKX10JAtCD+AjJ5/cBDvL39bQDWPb2OZ9XPdgjBjbZYy9I5SznywxEsFguDPhrExC8nUWcRBVyvVp8XbIFzf/f+QUoNFK0FbfvXyJqLjmEg+z8AEJUeRZ/L+xAU6ZxP4Q4tVvw13tg1Ko1gu6tEUKn9JFWtthv0STjxARx/1+fPuW7gdWhUGpJCkzCYxMP/6/1fYzQbGZY0jH7x/XzaTmCgKDi5mjiPeWiM1d+3oyF5eDLT3ptGwkAXI3E/UVFfwZCPh9Dvg35syNrAnD5zGJE8gvPTz6dPXB8GJw7GaDY65cH5AoPJwFvbBKF729DbqK6WIbOYyXn+Kzb/Z3OL972j4/RpMag+/3zbZMue+Jv43ETuOn5Xm+QNvL1NPExvH3q7cxaYHQwmg7VouF33NUZlFcGBHpgRmcyxq89igW9k8I2MiHAx+Ld2PU1aA6MXuNyMVfFnZ/WZvzO/Q6jDJBgM0GvDx6Qc/sMlWRQQYBu0+mvx4A2ldaUurRQlq2Op8OkJZcZMAMICQkHTCUK7gcL354ETNs6Bn7sQ0mj3KU1Y0iLSUCvU6Iw6hwJ0RFoE92Tew6QXJjX/M1sJUnFHLgeFC4fkdc+sY8GkBehr9a1io2avPpDg6vkaEGCb9LsayDe1wJMsBZ2u6fSbYMSnoI4UIer7n4DKffgDe6tPT1xMRlQG4Jr4OxvwV/FnnzOs1ws70jdT3+TEclu7f309fPedbZ3ntz9I7/d788SaJ5y2Zz9e8CXjTyFXcNOQm4hT24qgTsRf3DgY8QloUpw3mPeTQz6Gv5COq9GIx6y3jAsyOOfpcwiOa4OJvweUHC7hxbAX2fqWbxbGrQ13xF+vKDH+O1x6wGkdx3w/MWmzL3L7iz9P/UludS6h6lCu6HuFz+tJltmldaUuLS0jw82o6qsozWmjzpRWwMVfXEzeaNFk5gvxpzfpWX16NQBTMkQmSGRQJB/P+Nj6fASY1HWStSjjDywWi434i3VN/IGd3WcLiT8Q6jl/sp6VSlvjQ1sTf1te38KZte4LE56I+4ONnHlLXF4kmMwmq9LT/jgDjJg/gsWHFgP+EX9KpU0Zk59vsc4nXcFisWBs6MBhmU2QOCiRJ01PMnreaIdmGU89BVJTbkvGs5ICNiMqA4VdRo1kNRcVJZSASaFJmGQNVAbvcEv8jUgewfWdXyC5/KoWEX/RQdFkVmaSWZlJg7GB2FhBjhiNzjmuo+4dxTlPndP8D2tlqEPVRHaJRB0sLvhs2TrOJLzBnsq1TssaTAaOlBxhW6579XA3wX1aiT+nRuxGmON3cyLp3xxO/RcyZNbrxB2MZnFtSOMfrRaqGqqIDoq25tk65Uz2exKu0EO4+/usR4z8HC7XusxCCgwEtUkMtiUrxvj+8Zz7zLnE9jq7doIWi4WV81ay70vHMbmvedUpYSn0jetLn/CRmDG6nCtIxA6Ic1smA0OKzdWnT2wf1mauZVfRFmQWBf1lLsL9GiFZfepUeW7JjJoGQS4EykOFs8+3aicCtkUIiBLRFHIVmiA5KqM4lqV1pXx78be8HPmylw20PWJ6xBCRFmFT/JmiPTaWWok/O6tPT89voxGy7Xq6mhJ/UsN+sK47coua88933oZCYWtSrqgAo86ITC5DrnSsOzVYxPGcYV5O/8PDmq/CdYXl/WGFY1Njfk0+O/J2UKcSZLY2u5yfr/+Z06ta8XP9RHBcMFEZUSTFJDH++BaGn/jda4OpVMvSqfO8PjclUYq9BbY99JXivuSuBm+f2eiO/NfrwaiwI/7S50LPf3nesbMBVSgohfVDSAiojWLCJTn0xfSModesXn7nXbcFqnKqOLT4EGXHy6hqqGJv4V7WZa1BYdagVPpoVe1LpmJwF7i8BnrNs722eiIc+Hez9/1swycvmfvuu8/nDb7+urN9zf8nyOXNJwQsFovPXcMSpIdKc4m/0gdKqTfWE1myBjbNgbAecOFRh8m8y+6yYR/CxtnCtsGHG1RKWApF84qstkpGs9Gq/rth0A1e1z9edpybf7mZ/LIaerDL5YN3/GPjvW6nvRDdPbrVFBXP/vUsxdpiAMamjmVc53EOnfMTOk9gd8FuduTtsOaM+Yqlh5eSXZVNXHAc1wy4htdX67AoApFfNJ3BN7WSjUAHhFwup1+/fnz3nTjRY2JsxUV74q+tcLzsOCtPr0SGjNuH3e52uW2525izdA5hAWFc1vsyctgCFjnTUl3bc1J7BmpPQdQwm52gTAbpN4C+krBwOcUlvmXeWTP+7Kw+1zy2hpzNOTxS84jP37Ut0aCzIDfqkZuMbidmYWHCfqO62pap0BqY+9Nc9hTsYf5F863FTYCpGVN5bM1jrD69mgZjAwFK9yOQCjKBRiVKvyfEn5YgajAoQ4gM1ZJHqJUYU8gVdIvqxqGSQxwrO2ZVociVciI6R7TsM1sJ9vl+rh6L2iItRQeKUAWrCA6G0NCwZqs4C2sLyazMRIaMYcnDrK83VfwBPLH2cXZk7KdL5r+prh5obRCw7lcTxURccBw3D77ZOoG0Inqo+APQ+QqR9eenVYQ0mDWbhX2c0s2IrluUqBydKHf2yaotrGXJ5UvoNasXI//lv0e9L5CKJb7mxEhh4haL+D1jesUw/K7hDlZBP/1kK3bWqTPZWPITAFf2u9Jpe9J4QaVyXbCxJ/4sFtv5Jq0n2bb6jImrQNl8CyX73+lY0RkOVe4gLjiOc9LOafY2WxNyhZzk4cnNJhy35GwhWhNN18iuKOVK6/PX1/Gz9GxuWgAdmtqPheWQrXOv+AsMFGoCGTKrErY5kBpg7h91v8d7elNEa6JRypUYzUaKaoucCgqhZZkMWP0VWV2mwSXD3GylfRHTM5YyDWB2QYi7wJacLdToa4jVxDI40YVCtgn+OPkHiw4uYnbv2VzY3XuOZLG22HpMe8S47rwGmJA2Af6CdZnrmjXfailCQsQzpS1z/nRVOv68/0/6X9OfLuc6Wn2WHS/j83GfM+zOYUx4YoLTuhUVsHSp+Pe0aRYWzfiW9AvSGX6n53xvd9fv4ZLDlNWXEaIOYVQnxxzbjKgM9hXt44L0CxiXOs6v75iYCH9ZXmD4jy9wd+EdvHyec0G5oaaBV6Jeof/V/bn484v92n57QiaXoZArfFbJtwbxd6xMqJp6RDteO/aKP5lMxphOY1hyeAkVIZvQ6VzPtwckDODiqAEUl4OmBX24ccFxBKuC0Rq0nKk8Q8+YnnTrBrt2CbvPHu4v83ZH70t70/tSm+Ww1iIKNdFBznPoVadXMe2bafSJ7cPB252fWwAZGbBpExwWMaZuSfkS5W4AwrSDkcmEOtYV1meu54GVD9A5ojNLLltCeLic0NAwamvh2gHXck3/a6zZmU6KPxDOMM2FOsLtW4GBOCn+2hM3bruRgDDHZ7uv12VYQBgHbjvAnj3w5J+uFX8NDbaxsUT0lET8DhYYF3k562/7lsfWPAZAbNUUUiLcN46nhKUgt6hRmiIoLNWRluZsgVzTIHYiUB4i8sXNBpcEbLNhrIP6AgiMJSgoDLUxBr2qmNK6UjqP70xQVFC7PHftccVPoknr9a9Erdprxp8L4s/T8zs3V5B/0vylpMRxTiERf6F1/ejcWVzbTdFgbEAVXYa5OpbKShWTnp/EpOedm3ENiB2p13RBFjUFWUCEh2/uJ5IvhCb5uY+ufpQv933JVXEvAQ9hio7jH7/+g/h+7WfhOuLuEYy4ewQWC4RVi/mrt2uzU5gYc+vUuT5Zfdqv0xTexDf2GX9V1RZwYWOv1zdR/HUU6Cug5hSEphMaGonSFAHYiL++V/Sl7xUdQ2yTNCSJx3SPYTaaydKKKIdgZQgKSxAhIV7m73m/gq4E0v4BCi+W9jKZoyuaoQZqTkJI8+aSLeGMmgufiL89e/b4tLH2vJn/N+CTYZ+gUCm4YYt3IkxCSxV/wepggtXBsPMO8UI3QTrYE38uu8siBwqrB12xz58l3dCKtcXMWTqHwyWH0ag0PnVLhwWEsT5rPTJkdJMZqK8/u9ZWHQUnyk7w7nahtFx+5XKX19ywJFEs2p6/3a9tWywWXt3yKgAD4gfQ9a2uyOuTGcQOanoMpYPUHNsMer2eigpx04+IsF1T0jVmNpnZ+OJGOo3p5FRQaSk+3PkhILqiJRLGFbpGdqVYW0xWVRYHioWqIbb6fDpHuWGwcpbBngfgvE0Qa2ckP2I+AOGiCd82+NHmQObXED8RYkY4bGpKxhR+vuJn0qNsvu8j7xtJ/2s7jq+13iDj0Ll3olK5f9CHh9uIv9ZCaV0pK06uwGg2OqlIBiQMID44niJtEZtyNjGxy0SX27BYoFqeCUCXqFbKpen9EADKRgdk+wlLz5ieHCo5xNHSow5EZW1RLRWnKug02nc1Q1vAqdO4Caa/P53p708HhNWn2WxGq23eIEpS+/WN62u1QTIYbOeI/fN1beZacoM3Ex9wHdXVA5221ZT46xvXl49mfOR5B4I7iT9+wr4Yo9O5t7PwpPhTqBWUHStrk/xZCf4q/qScYa1W2NOmjkkldUyq9f3qavi+MS44JQX+5F3MmDmv63kurf6kY+JOgSD9bhaLIPvs87jABblRfQw2XwN9HhZZmk0R0LImH6XSVjj4/eQKHlh3OzN7znQi/ipOV/DzjT/T76p+DL7BO6HSWojpGdOibNfZS2aTX5PPthu3MTxZkAp6vZ5AJ2sJ13Cn+JvQqy/shUrZKbT6OoLVtgNufyyfOfcZHhv/GA1GD+GY3r5D79mEBoQys+dMv9aTy+QkhCSQW51LQW2BE/EXmRHN6S4jiI6Ic7OF9oVeq6euyojZLH5bXxR/Y1PHsvn6zeTV5CH3oat25emVfLnvS2QymU/EX2xwLCfvOsnpitPCvcQNRqaMRK1QYzQbKakrcW7G8ANrn1rLtje3cfepu322kwwOFmSKZLndFlBpVNy4/UaUgc7Te02MhqiMKELiXT8oFi4Uz5Hu3WFwt1o+2JRNRNcInz7X1fW7M19IlIYmDUUpd9yf1y94nc8u/sxtRp8nJCaCIi8QnVlLdrVry1x1iJruM7qTMKjlLitnCzX5NZSdKCO+XzwNDeKh461ZRiKBtFoxZvHlGVteLuY4Us1JUvxJDUIgitfS+Ee6z47uNJolh5dQHrLZY5NVU9eD5kAmk5Eelc7+ov2crjjtRPzZ4+C3B9m3YB8XfnQh4Z1aQarayqiTCSu8GI0zE9crVgTznSg/gdFsdLpOwKb4k9T/TZ97EvYWCeIvvG4wsbHuG8GC1cHsyN/BkdIjNBgbCA1VYzabqakRJ4RMJrNel07EX9VhaCiHmFHNI40MNcIuNLgzaJId3hLEX6Pir1GRVXq0lF9v/ZXBNw2m/1Vnb64pk8lIHm7bv7+y/uKhVQ9RETaEbkXv+jyWbeomYQ/p+lKrbdd5jmIdGGGQ/FrMFjNf7f8KgJSyfxLlwRgrNCCUR0z17Dsgp26y62Vq9OLBE6QIgd43+/YF/EHBH7BhFoxeSGDklagbj2WJtpQ58y5v/c9rAb6d/S2vfVjMzsp4n4k/6X7mifiTbD67dYPjx8Xcp7bWdh4cLGkk/ur7MnGK6zpFwmsJVEZVck7+ESore7r9LIn4K4q9EF3vcwgMaF4Or0sMdI6PkdwYGuSVjX8H0X16d6fl2gP29s++En/1qhyrqt0dcqpzHNZpCm81eEnxZ5brKauuBZzD5hr0ZkwK8dAMUYfA6kmiaXvQfzzvXFsj71fYci2M/4nQ0ItQNRJ/OqMOnVHXrLFbW0IZoIQAKC4TvERkgLh2veb7nf4Ccr6HNPeKagfoK0X2YdQQiB0Dl2SDi4iRjgqfiL+1a53tCf4H1zCbm3/wEwcn+p2d0tKMPytGfinsAONEF59H4s9sgoZikQmo9K/j22wx849l/2Bd5jqCVcF8PetrwgO9D9TjguNQyVUYzAYalIXU1TnfhFc9soqivUVc9buPF+9ZxOrHVnP0h6Ncv+l6v6xcm+KBlQ9gMBuY1m0aU7tNdbmMpFjZW7gXvUmPWuFbsNL6rPXsLthNoDKQp895mjGfjUFpqcSCmerq/25XYLPZzLFjx6io6A/IiIx0Jv5qC2tZ+8Raht05rFWJvzpDHZ/v/RzAo9oPRIHrvWnv8fWBr0kKTeLAX12Iyb7RffEtcQooNBDey+Xb0rWdJ2XaNhTDvkeh39NOxF/niM50jnAkpDIucNGu1o6Quic9DfykzsrWJP4WH1qM0WxkcOJg60ReglwmZ0rGFL7c9yW/n/jdLfGn0wnlEkD32DTY/ySE9oAuLb+fScSGfcFR6u6Wij4SVs5byf6v9/Nw9cMEhLZ+lqWv8Eb82UOjsaDV1lJTEwL4X4xwZfMpkQsqlSOhZrPucO3Z73PR64dkSJgMo0R+ByY91OU4Bap7glIpbGFMJs/E3+DEwczuPdulsiIoKoh5RfNcrNU6MJttBStfM/5AjEG0WteT7D17BJHTqRNMnFLL/B2ikeGeEfe43Ja0DXe/j1R4aWgQBRqvxF9dHmgzRce0K+iKRadk5ACRCeAnZDKxPzodhCnF+eYqs1eulFO4p5Cuk5uvXDvb0Oq11pxYiZCWnr/9+vVD4crXtwmka9NeiQvQPz0OlTESg7KCPZmnGdvd1o3a9H6iVqh9Hhu5QnhgOJf3aV4xKzEkURB/drbZEmLSw8npM4W0s+ts5jOO/3qcZVcsI3LwbCqT+zjHALiAQq5wUnx5wpSMKby25TVWnFzhk0JALpOTHpXu0JTkChqVhhN3naBTWKcWN6qGxIeQNDQJQ53vuUXSM6EtrT4VKgXJw5JdvhcUFcT1m653+V5mJvz5p/j3jTdCWHIoDxQ/gF7rbEfbFO6uXyvxl+ichegqs9RXREVBoF40grjLypTJZMz5fk6zP6M9cOL3E/xy4y9cufxK9KmC7fH2zAwJEQSe2SzGtN6agI8dg3nzRJzBXXeJ1yZ0noDOqBOK2EZIBVF727mxqWNJVQ1CXTfArdXnusx1HK+MxizrQVCQi523mAGZTxL69EhB/J0qF9V0KeevKfFXmVVJ5rpM6krrOgTxd2bNGXK25DDstmEERQVRLxPPbmnsaI/U8FSClEHUG+s5U3GGbtHdnJZJb3Jbc6f4211gI/4S0tzv3+DEwSSEJFBYW8i6zHWMjp+MVluLXB6GXu94zknPTet9/uDzkPUNzGmgOWNtyrbBmvNg6LvQ/Q6HtwICQK2LIUwVZc1qNhvNFB8sRlvchjJpF7CYRSadMlCJXCEnvyafrblbiWssdvs6lg0Lw1pDsVd+ga3hVrL5BHi3325e+e4vInqNQy6T8/XMr3lq6SI0lTOcxjtNERUp6jTurD61VuLPWyW8mQjrCb0ehLCewrbVKM73wmrnsWt7wGw0s+6ZdSQNSaLnJT2JNEWhNnk+lvHB8VzV7yriguMICDICSo/Pb8mGuGdPKCwU9+TSUjvir9hG/HVy0+sZHRRNpa4SvbLU7bE0GmHIyR8wKCoZc3tnv8bPzYWV+EOcuNIzoD1VnHu/3Iu2SIv68khOx68gWNcDlWq6x3W6RnYlJagbCmOCd6tPifhzYfVpNttU8e6eu8HqYP7Z/V62rI3EXcmptsF2QoWoQ4RTV1Ar2lE1F1FDYMCLENaTkBBQmkKRIceCmUpdJaH1ofz17F90Ht+Z3rN7e99eG6JofxGGegNJQ5IoqRONNuFKQfx5zfcb+DJk3AK+zgf15bDrHujyT0H8gW82oS7QEs6oufjvrub/zTDj4xlc9MlFfq0jkRLeBgQu160rY+5Pc7nn93sgeZqV9AOcrD4doCuCn9Jgr//2fp/u/pQ1Z9bQI7oHO27awSU9L/FpPblMbg1L1qlzXXYbVmVVUXSgyG/y9GxArpQjk8lQaZqvVNyau5Wfjv2EQqbg1fNedbtcemQ6kYGR6E16DhQ5Z924w2tbXgNg7sC5jEgeQYAiAKOsnrqA04T8sogvJnzR7H3/O8BgkKHVisGLK+IvODaYm3bexMh7WtcO79uD31Kpq6RLRBcuSL8AEIq0wkLXy88dNJfV167mq5lf0T3v3wTpU90/2CL6QvfbRY6YCwwZIv5es6axOB/eB87fCj2dg4z/Dqguric2ayehtflul2kL4u/r/V8DcHW/q12+PzVDkPRSzl9pXamTlY1Wa6G+kfjLiE6FQ88LxWZLkbecGxL7kxp+0IFEke6nRdoih8X7XNGH8151DBFvD3iy1DHUG9j54U4K94qLRDr/6+qaN/mQ7GtHpozkiy/gwQdh/37xXnS042Tdat2hKnHZxdtU8VdRX0FFfYW1kAGA2SiuNft8uHXT4PdBfneOSQWZBg/ipVGdRrHksiXcPeJuv7bdGjDY1cX9If6kY6rVwhcTvuCv5/6yviddu507w/aGLzEqq4gwdXPbDOMLGeuqM9st8ZcwEWYVQeplrjd28mNYORqqj7r/QC+QjmuoXBRPpMmMPcJTw3mo4iHGPeqfVV5Lkb0pmzVPrKEyq9LvdSXVaVRQlMcsW3cwGm3HqKnyQaGAUGNXsMg4WZzn8J7bY9kOsObl1joTf9J3clfwaW+EpYTRbfYA6sISrJa8nuAq99YbxqWOQ6PSUFhbyP6i/c3cU9dIDU9tlSLVsNuHce3qawlP9Z1osL+ntRXqSuualWv3449CYTxmDPRq7F2SK+UEhje/q3tngU3x15oICYEgL8Tf3xGdRnVi6jtTiesb53Pjk0xmm6P7Yvcp5cQdOWJ77bI+lzH/ovkOOYyONp/i30OThvJSl930zH/OJfFnsViYunAqTxX1p16V6/p5+8cI2Ohbw0R6pGC8TlU4En+5uY7k+ZgHxvBY3WMkDuoAxVLg5B8nWfv4WnRV4kfSycWzOy7YuZtDLpNb7YmPlroeL4SEOMYSuHJhqmmosTbxjek6mIs9uNvKZXIu6i5qTT8e/ZGgIHGMKzU7yXg3jbk/zbUua5+NC0DXf8LgN5tv9xneVyhZYsc6vRUQAP2zPubnMWXWJti4vnE8WPogo+71vXGkNVBTUMOLIS/y5/2iG0KrFzdtuUkM7H0Zyz62+jF6fBbJqYSXreOWTz6BzZvF+/b5fhLSkoOIq76A8iINMpmMCWkTuJAPUVgCvdb5vI0dag1i4KRRhsCBZ+HQS96/hD8I7wWDXoaowajVNuKvqKaUM2vO8NPcnyg7cRayU9zAqDOy4bkNHPtJWBv7YtsarBYChdcveJ2wEKGV8fT8Pt0Yd9e1q02QIUUymS1mjpSIG29ofV+3x9PWXFrmVpGm04FGn0Z4/UDiS78jqfA19zvVHOT+BFv+6eDqJhF/9RaxU7o6My8Ev8D3V33fup/tB/Z+vpdNr2xiS85WDne6n+z/Y++sw9u4tq/9jsiyZGbHseM4zIwNY9O0aaApM9MtM2MKt3zblJlTTsOlcBpqmMlJjDGjZIu+P45HYLEt27m/767nyZNEGs2MNHPm7LPXXmsnvY0/7nN61+ksmnqQPife9K/4a+jx1z7KvZd7ebkg/yTJe99VgGfHvELX/Eepq/RMuNfVW0ktnUsv9XShojv3KIz8wveJtQaiewpXm6iuREaChAItItgoM5Rhs9rY9J9NHPvTez/p1sKaZ9fw4fAPsVlt9lZYWouYb/32q47sBKlB5LwismDcMhgyH/J/gxPfeS8CPg3RJOJvy5Yt3HfffVx44YXMnj3b5c//0HqwWBwTfFOsPksMJXyy/RM+2fGJeCFvOSwfAafW+Fb8KdTQ7U4xye9/HUr/CfiY1wy8hj8u/4Mt129xU8b4g/zgNXgh/uZ8NYe7cu46LS1nxz85npv33CykyE3EiiMiAJ3Tc47P306SJL6c/SU7btxBv5R+Ae9/VvdZ9ErsxZ3D70SpUNIzUVRwVIXvpp4w1G2o/mkNVFWJaEGtFkl7Z+LPZhOWeO0GtSOucxNYdh+I1cbSO6k3Nw6+EaVCidksKnLvugufdjoWi+P9QOy2PGHECDEplpbC5s0If+uEYaCO8rj917u+5qX1L1FhFBmGtc+v5dX0V6nMCSGL1gxUnKikw67FROYd9LqNHASEivgrrC5kQ84GJCSvtsWTO03msTGP8cGMD3hz05tkvJpBn7f7UG9xVNPnl5djVomTyozNhHMOw6BXm3+CSi0RyjxSIo66LFjkxUVjJVHX6V0ZeffINlX7gWNR5inxVZ5dzuKbFrPnuz0AAVmw+MKnMz+l7P4y5vY8n19/FUmx118X7zVenDk8+4t9Kv7kMfngHw8S92IcT6962rGRQgUTVkC/Zx2vdbgQuv3Lra+CP8i/j7cK/EBwcPFB9v20z/+GTYCzBUug9kjgSJIfOWjh2NYS1i9yEOXy7x4ZaePHXGF93an4Nq82gtu3i799Lc48qWJ9kkWS5L3KL2WySG5pm65qka9rhELcb54Uf22Fk+tOsuaZNdQUBj/g5D6TzrZywUBeqKtUnis5Z1Qv4qx/jPSPmOryujw+rNpiRn88mqt+uarNisQeGf0If13xF7N7uK+ZYmMhY9cirD/90gZn5h8ZZ2Qw6JmZ1EXEB2Tld9Pim7joh4vsvVICQZgqzK6Ml4tlfOHJlU/y9KqnT3sSSJ4TWtLq89frfuX56OexmDwTrru+3sVv9/3m9rpc4DZ0qOiPt+rpVRTvb94z54tZX/Dl7C9D3ptUr4fwOkH85VXlYbJ4Trpsnr+ZxTcvDumxWxKJPRMZeutQotOjA+4lBo6Y1l8SExxqucJCsa7xBm92ynJBiqd4o8RQgtEs3tCa0tyfDzYrlG4Bs4eKKQ+QFbwy8RcdLfrW2Wxw2Mm1XFKcXmv+EXeO4IZtNxCVJtZRdUoxjpIjPVszdU8Qdn7eiD9w2H2C5zhmR+EObNhoH9WeV55OYqjvtpzM6iEsqn858As2rOj1Fip1OzhZddzlWe1m9Zk6BbrfHmTTYyeEp0CPe4QbQiNotSAhNSuWDRVUYSr6XdHPbvdZY2og/sziIR5ILCtJEuV15dSFid/zyy9h4UL46CPxvifiL6UhZCwocIxPf6oiGRssb7G2+3AWn5rv8f1LMh6ke87zJCm7wLFP4cS3/r9EEyFJ0L38TkbuX8sFna+n9HAp2z/ZTvmx8hY7pj+owlXcsu8Wxj89ngd/f5Cl9Q9TrywLuPe4PH/X1np+dtpsrsSf3GOzqKFmTyEpyL0znzP2r0NX19GrZW+8TlxoX4o/eYwolaAq+IWEsu8D+xKBomKPsDOsdTwLZOKv1loOgKFOQceJHUnu13Y9/mZ9NosrV11JuUEMJo3Nc+6qMQIplqm31FNQLYqL96xPx9IopJJjpthYfJKN8viuqXG43zhDbY1i0NEF3Jaw6LTMX4NjrRVmiwFEnz99op47c+7kzNfO9P7BVsKAawcw9bWpKDVKimrEgKs5JRR/PudCq1n0MgwW7aYKN58Dr8P6S/lv0tEFzUJ88803XH755UydOpUVK1YwZcoUDh48SGFhIbNmBdfr4n9woGhvEVvf20q/y/uROjCwqrWyMjHRKBS+E1reYDCJrFaEOhwWRDpsO+tLfSv+tIkw6BUo3QrLBkPvx4UfcQBQSAqvVnf+IBN/Rk0OteW42Sb8X0eMNobeSb092rU1hjflgy9cPeBqrup/lX3i6ZPch20F26gK382xgY/w5GdB7/K/CtXVIpKXK1zlhL/JJBI1YYr6Zqs2PWFWj1nM7D4Ts1VEBDk5jkD/5ElHlWtjOFe8eiT+bDb4OQ3azxSVKR6gUsHkyfD997B0qSACsdQLpUpER1C7VijdufxOCmsKmZQ1if4p/dFEaohMi8RmPT1UtuqkGA4Mu4z4TjFet5GDsECqowPB+pOifLN3Um+7kqMx4sLjeHL8k9hsNl5Y9wIGswFDlYGNORsZ3UGM54rqetoXX4lCV4ZOowdNiOxkk8aySF3AplwVk53cGCZ0nMDf1/zt9ZzbGnLFu6fEV3R6NJcuv9SuthD3v9QsJUWMNoayMleiCtwX2zJhWufF6rNxP7kyowgqY8O9rPJkdL422FMGHASRL8UfiGrTvKo8IjQR9gWcjD8e+AOLyUKPWcEV4wQC+fdUKn0vjhpDfqZ98bUS2/i7Uakcc76cOLeEF1JWXYTCqiEh93KMRtysB48ehUWLxL9n+DBUCErxt/cFSBwDiV6q0BOGuVklBwv5uuoQ91upoRSL1YKyUV+dY38ew1BmoOec1rNaGXTDILqe05WYzJigP3uopIH4a2RpFqhFkVw9HRPjOf5LjUyh2Ob+fJcTJTWaI6w9sZbs8uw2W2TLduyeEBsLuopCFBXBq7ZaC4EWHG3N28oH/3yADRu3DrnVY9W0N5zZ6UwWHVzEssPLeGDUA163s9lsvLHpDUoNpZzd9WwyojO8bguiUvmSHy8hpzKH7TduD6jnoCcYK4z8/drfJPdNDvi52RpWnx3GdSAiNQKl2vN4OvjrQXZ/vZvxT4136QPobOl37I9jrHxsJeGx4SR0D6yPhKfx2yW+i0frwuZCp4MwcxJKmwYL9eRV5bnZ0APk/5PPzi92Mu7JcegTm1gd10bwVfjUGPL6319Ma7OJvlPy/svKQK2v4kTFCbJiswhXOyY6eQ3SuPBJqwWLZCTXkAO4Wv3L6gg9yShtYe7En6SAiwNfK/RK7MWI9iPoneiwbO7SRSTSDx2Cvg0t32qLa8ndnEtiz0RiOsQEvP+WQkRKBBEpIlNqspgwKcWFSYn07N/cMUbE+r4KFzp3htUNpgee8j3b8rcBwsYzEEzoOIGosCjyq/PZmLsRvb4HVeHCIahPkqOZnBvx14Lw5F5hNVvZ//N+ItMiSR/Ren3HdQk6Zn4y0/5/WfGntAau+JPnO5NOuA8sbahhKSoShbtyrBkZCZV1lYz9ZCwTOkzCpngWo1FDRYW41t7GYmMYlPmUR2zkhHGQx/fHx1/GjgJITAPO3Bx6lUptHmy8BtLnQOdrSVb0QFMB0UpIuyye3hf1RhPRdHv15kKhVJDQPQGbzcaLH72IVbIySXELarXvtZnJYqKotgiFMgKIwmwWeaHG90BRkVj/qVSiDUFjxR8AddHEVo90UWo3RmCKPxv70x5BqwynauC75B45SEi77XW9Dbrd7tLOyZn4i0OM04sWXhTKowYNOQdQcUQMpjApMBtb+RlaWSnGYuPwxWyG+x+QGFywEIPmJN9tSaRdtMiXyQiUkK9TlFIdno/alEBlZbLbOJZdcdRqwFQNOT9BTF+PxRGtCkMBrJ4JGecRGSnagUyt+pp771bRPaE7kkKyF7e0NTpN7kSnyaJQSCEpiAtLxJyXgkYjXCy8omIPLO0PfZ+B3g8Hd9DaXIjpA52vb1q/2zZC0CueefPm8eqrr/Lrr7+i0Wh4/fXX2b9/P+effz4ZGb4XXP8/oKn+yvnb8tn4+kbKjgXOPDvbfCqasHY1mMXqXa8Kh4SR0O02mF0A7c/1rfiTEd0HJvwmqr9aAe0jG4g/dQ42m3uis3h/Mbu/2Y2x/DQoF2uEbR9tY9dXgdtuesJtw25j1027uHXorSE6K3c4J8LkxVZVuPAkDxVRcjpCqVSSlNQNSZLsVVhqtSMwKymBTf/ZxDz9PHI353rfURMhSRJqpSAUZY94ECSgN8jJ77AwL03czTXCcz/Md+Jm6lSRQN22DfLzgUNvw9J+sO0+t21lkkju0TT0lqFc+/e1QdldtSRs6jCqErNQJXlfIYXa6lMm/kamj/S7rSRJvHv2u/b//370d/u/ddZk+md/zLmGn8FcCzUnwBKCZ5lCic6DRUmCLoFh7Ye5JUpri2v5YPgHrHxyZfOP3Qz4srrSRGjoNKWTPSkZFaUgKioKg6F5VVeFDa6ncXEwvMHRN6tR+7REvaz4893jz9nqE4Sy146ynbDjUajY26zzBd8V+M6Y/e1s0l9NZ8GeBW7vTX1tKme/e3azz8UTglEuOEP+/eTKWrPZ8R3lhEl6bAqn7ilk8rF/UFuj7JW1Mmw2mD9f/D16NPTv7/14ARN/pmrY/oB4TrYg5OsabhOrSqtN9FVojL8e/Yult/pXRYUS2mgtiT0SUYcHXwTjSfGnVCoD7k8iz4lpntuYea3ktd87alGSLSdbTzfExsL+M65m96gb3IoQ2hq1JbV8NOojDv8o7Dd9Kf5sNhu3L7sdGzYu7nMxZ2T4Wn27Qy5eW3dyHZV13ifrUzWnKDWUIiHZlTO+EBkWyYojK9h1apc9jmkKJIXEqidWceDnAwF/pjWsPoffPpzp8733uJn0wiTuzLkTZZjrWHOeb7tM78IVK68IuG9LMOM3FIiIELZTeosgArwRJhOemcD9pff/15B+K59Yydt93sZQavBZ+NQYgSr+Tp1ynd8KCmDNiTX0frs3Iz50LWJxtvp0xgHjapYNiOIz8zlu+5f7IUU2XBefiuCCP+GvM0XFvReM7jCa9des57lJz9lfkwshnRV/BdsL+Oqsrzi89DCnA+oq6zCWG0XvK5uKSTvyGLNnB4mRngmGtEgxmeX5aFHgrPjzRBjcPORm9t2yj2fGPxPQOWqUGqZ3Ec+JhQcXkpkZT5VO5Cj6Jvd1fJfGPf5Wz4a/zqJZWHcR/OFeAB4WBiURq3jowARuXSLyHFaLle/mfsfmtzY375jNhKz4U1oCV/zJ17VOI4IWOZa1WkVOwVnxt/jgYrYXbGfJkUUkxYtBn9/gBO6sLPKFDnEiR1Zq8pw4cFlTaWJBm+T/SwQFK5xaLfpf47hnDAZQh6sJiwxrU0WT1WyluqCa2upae+sFhTXcb3HFrG9nkfZKGouOfWcvNvM0h59omIbatxc5GU/En6zgi472XggZH+5f8VdlqOdw6jx2Jz2KNSyKHgMnhXb+VUe4kH7gIP6qzeWAb0eq1kJ1QTV1lXVUGMVg0hIYEXXVstms6JdMiX6tx5YdOTlw+ICalMqzGWS9CQkFGza4biOPS3/E350rbmdlr97kxH/uMWdgrLNiwyrmekMebLgcjrecGjdgKDRC8Wmqsq+P9eXDGNRuEHqNuDcqTlaQt7XpcXRL4MHRD/JM9Cm65z3DyJF+4hClFrKugnjvxZhe8c+dsPf5ZhG0rRUzOyPoLNmRI0eYPl0ECxqNhpqaGiRJ4s477+S9994L+Qn+t6Gp1kG9L+jNHcfvoNMU383pnRHoQ8cbak0iQ6lW62DCcuj1kP09n8Rf7mJYdS5UHYCUSV77h4UaHWM70jGmIyqrWD03rprd99M+frjoB8qOnn7NUVY/vZoNL2/wv2GIYLFaeHPTm1zx8xX2SjVveHvz27y9+W37/SCjT7Ko+qsM30Vk8VE2v7oWs/H0rUJvDmw2G7m5tYDN5X53tvtM7JlI38v6hrR5fFFNEdX1rr5PslUE+Cb+5MDTa38/dQRM/BP6PuXzHFJSYGBDkejy5YgCgIGvutoQNiA1oqE3UZV7b6LTAXVGG9hsPhdloSb+ZnafyYOjHvRo2+YJSfok3j/nfQB+P+Yg/lwIo1Or4JcOkP1lSM4xWlfJxb0fp7PSv2w3LDqMypxKzIa2Heu+iD9zndllrtXpbJjNJqqrg59/b1x0I2M+HsOKIyvsxF+7dvDQQ4I0mjPHdXvZ6tOsrPSYaJMJeXkOlXs5uvQzK90Ce54RTbxl2KywehZsvSOo8w+U+OsQLRQRco81Z2RNzCJzbGZQxw0UTSX+5OdwsrachILdqI1V9jEr/8aRkaBUKOkS3QvAjfhbvhwOHBDE3bV+BJW+iD+XhYMyDCavg573et9ZbQ4sGwr7XvF9UB+Q73uLSW1fcHuy+xz/zHhmfjqzycdpCmpO1VBzqqZJ8a58/zkTfzabjcrKyoD2d1Lklkn3Uvxfrz/K9swrePHQ5S6vy9eySinGXFZsVuOPthoKqwt5d8u7vLPlHbf3dDpQa0R2KRDrvtZExfEKivYUUVMkJitfC+pvdn/DupPr0Kl1vDDphaCPlRWbRbf4bmTFZnG07KjX7fYVC4vijrEdXRRL3qBSqOzqsGNlTe9NEhYZxo07buTM1wO3OZJ/r5a0+vSH6PRootKi3BKwzvOtUq0kc2ymXbXkD57G7wf/fMBL61/iSOmRkJ27DHluTaqexIxuM0RfHA+ISIkIuUNHS8NisqAMUwY1bwaq+JNtPmUUFDgU2J3jXNV7csK5sTqhW1xPbAoTJdJ+Smpd+3XJij+d2QvxV3kQ9r8GVYfhxALIXw7V3se2JyQ3uMo5PxsTeyUy48MZZI7LDGpfLYXFNy/mhdgXsNRbMJsltKZUogx9CdN4TrmNyhjFcxOf49oB3oOUrCyHwt2T8kupUNI9obt93R4ILux9IXN7zmVsh7HUqY9TqROqQZn4s1o9KU9tQHB9qN0h4Sn9qNWCSVXOntq/2JInmlEqNUrmfD2HYbc1z0EhWJQcKuHnK3/myG/i+eVQ/IlnYiDEn6z4q1U5ioXla3jqlCvx98M+0c99To859n6OBQUibpFje3+Kv8wG4q+Ck27vma1mtpaspFy3BbXGCuW7haInlNC1hwtqoJ8gn83heRxL+g9f7H8Hs9FM/j/5TeoLHSqUHCrh5dSXWfPcGvtrSqvO7zM2SS8I0lM1hT5V+/IzSc4bNbb6fHrV0zy85nYqw3f6vJay4s+k9K74K3NiHnWGXKoKdofWut5iFK5uNcftL3WK7cTDox/mmt6iV3xdHfz9+t/89fhfoTtukHin3zt8Nf0rKuvEwk2rCEzxV2YspV59CoPmhMffWF4HtmsHTzwh/r1jh+s6O9AcfEK4rOAs8kjk7qpdweLBSp7KHQnhqTDmZ8i8JKDv0aIIi4NZOdD3SY+tMAAWXrOQT8d92vrn5oTKnEpezXiVDa+KHLvJ5FDHT5zo58NR3WD4R8LCOlh0vwdG/9islh5t0W4iaOIvNjaWqoYRkZaWxu7dQg1UXl5ObUv6l/yXwGptWkCkUCmIzogOqq9Sc4k/2epTp3aKzk/+BDm/+Lb6rD4GeUuEKsVmE/83BebZ3xzcPORmjt5+lAEVTwLu1SbdZ3Zn7ndzm2Q/1dK44KcLOOd99wrJQFFVV+W1j4UnKBVKnlv7HJ/t+IxtBdt8bvvhtg+5ecnN/LjPtUFvn6Q+JNSMJbFyMjEF+9n7xh8Yyk6DEp8WgNVq5eDBU9hsrlV18tgqLoZuM7ox67NZASdDAsFTq54i6rkonl/raLIdLPHX1P5+zpja0A5p/XoazPnvEJN+I8jEn+x7XrSviLUvrKXkYNs17HZGzrLdDFr8NNps770yQm31eUbGGcybOI8pnQIPHCZlTQJgY85Gu5ohv7wEi1QnFha6dNFHNbZ/SM4xXK/moj5P0Tnc0e/GZrPx6oZXefiPh6mqczy/lWold+XcxaTnJ4Xk2E2FL6urxTctZp5+HnWVIlup1Vqpra1tEvH357E/WXNiDRarhYKGdXByshgG6enudoITsybyz3nVjNr/N7kexL9yGOTT6rPDBTB9LyQ52TZLCqg84EoGBoBArT7lxJ4n4k9GSwSh8nUMpr8fwNlnw4UXwhVjj5O55Qd05Xn2REllJdiwEBEhzrfxAlvGsmXi74sv9p80kYk/54WNfC1dFH8KNSSOFBYf3qDUgTFfqK6bCOfr+vb0t/nx/B9JiXBfYHQc3zGogrFQ4Nfrf+Xldi836bN3jbiLJ8Y+4WJ3abVaOXr0aEDxsz/iTx9hJSfhMzZVf+9yP8sL9jLaXvGXW5XLjYtv5MlVT7q9J0mQoConNm8P+UdaUBrWBKQOTOW+0vuIniSunS/i77Odosjk3pH3BmXx6Yx9t+zjwK0H6J/S3+s2MnnXmLjwBZn09UUoBoLkvsloYwL3wHPuEdQSsJqtLDhvAZvf9q6OsZgslB0to6bI9d6S5w+l1UTRviJMhsDXG57G7/zN87n3t3vZUbgjuC8RAOTfsdexd/j5gl98WueWZ5ezef5mrObmkhUtj3FPjOPW/bei0WuCsvoMVPEn23zKKChwxAPO48dqdahXGquMUqMT0Bu6AfB3zt8u78nKS229eDi7WWSXbBRV8hV7YODLcEEdRPk3qDOYDPaxKu/TeQxFpkYy4OoBAdvStjQyx2cy4JoBKNVKu40beI+B+qX044FRD/hs0aHTwU03wWWX+Y9lAsWMbjNYMHcB3eO785lqHCZVOQnqDHolikIq5yS3XfE35icYv6x5Bz7jK5j4u9vLYWGgsohArKperEkkSaL3hb3tvfZaC1V5Vez4dIe9z6lSoSRSE4XaEoVKFVibmbQocc61UiFWqZ7MTIc9bWGhg/jTRNTae9nO7jHb7maQne1Q3oaHe+k17YSuSaKgpVqV7daDrqS2hMeOjmdtj6GEaepgSR/45y7/X6IZMIYfZU/GbXx++BUqTlbw3qD32PL2lhY9pi9oo7UMuXUIcYMaBpBNQmHTBEH8nfLZT75xz8bGir+vdn/FN9lvYFTn+1RvDkwdyAXdLyO2ZiSVleJ53BjlDSegsKlRr5mFat3sJuefPaL6mGjldPh9+0vp0ek8M+EZrh94IyBysHu/38fWd7aG7rhBov9V/ekxpwdVDXmUcEVgir/0aDFHGTQnPeaCqqqgTL+RnNivMOj3k5ws1rJyv3hwdd3zBYdLkGcFZ41ZLDpVCpVor9P+XIjpFdD3aC3I6+OTitU8v+ZFVmavBMTvP/aJsW3a7qe+ph59kt7uQLNpk1jHx8c7nrctgoShkD5LFAM3ESEdswEiYOJPJvjGjBnDb7+JxuBz587l9ttv57rrruOiiy5iol9q9X/whuL9xVScDC4TLT90EpoY68qVsp0jEmH7g5C3VKgNdj3lsqB3I/663QoX1gtp7LHPYGEWFLgHcS0FT4E/QGKPRHqe15PwOP9Vv62NlP4pAfdu9IR5a+YR9bwrQeQPQ9qJxfDmXN8WGfKCql+yq1y5XWQaI/avpNfJ1yjsOILur1yHLt6XZvq/G5WVwg7ReRJ3Vvy1BLbmb8WGzW63aLMFbvXpl/jLXQK7noY6/yffqSFvXFzs1LTaVC0USHtftG8nV6KVGMQ+C3cW8scDf1CwI8SVg02EFBVJeXI3VHHeq77k51moFH+BoPEiLDMmk06xnbDYLKw+LkqT/n3oapYO0rJH8zHE9BZ9VOM892oIFrrIcK7/9RAf7P7Y/pokSTy+8nHmrZ1nJ3JPJ/iyukrum0zWpCw0keJNuRot2IRqSW2J3XpwWPthdsVfio8CLo1SQ8/OeiRJVMXLC3MZjcelR6tPlR6ie4C60SJl+m4Y+0tQ3yFQxZ8v4u/3B3/nmbBnqC4IvRRFTnwFksB0Rnw8XHIJ9J2RSfWZ51Eb085F8ZcX+x0zV2Uwb808j8RffT0cbyhWHenfhdd+Dzk/F+TiIpf+Nhaj/94oYXEw8yT0edT/gb3touH3MhpFZf6sHrOI1npWm9tstlatHOx6dleG3zG8SbZNM7vP5PFxjwdF1DhDnhPbe+GSsuIzwKbAhIHCmkL76/L4KLW2veJPLqA5VXMKi9Xi9n586SE6/fM9OVsK3d5ra0iSRG2dsKfxFnvYbDY25mwE4OyuTbcQDuT+yi7PBiAzOjPg/WbFhIb4q6+up2hvUcBjr6WtPutr6tn3wz7yt3p3ZMjbkscbnd5g+yfbXV6X59vqQ/nM7zm/WQlao9nIrlPCNnBwu8FN3o83yPedxeK/4GXbx9tYcssScjb6CKZPQ/hyPGiMYBV/MrFQUOBuvWw2w8svw759DTWAjdxztVqIqxET6oYcVxcb2epTY/Ci+Gt3FkxaA4mjRAyk9C9nXHpoKTEvxHDJj5e47PN0sJnzhoHXDGTGBzOQFBIbTm5kT/qd5CZ8EVSPY0+YNg3OP9/99adXPc2cBXNYe2Jtk/b7V/ZflCuyCa/L5N6kPwhTiZtOnjMlKXjHhqZAq3Uo6ho74bQ2OozuwIPVDzLoerEGe2nKS+y/ooLOBQ8G/Fsk6BLQKDXYsKGOy+e66xxrC2fF337TcmpNtWTGZDIgZYDd1vXgwcD7+wH0bifmNZOqjPxyV4ahqFYEx2pzHDqtEvo8CemBudQEhbylULQOgAiNWIfXmKuISIlgwrMT6DytaXFfKBDZLpKz/nMWCRNEHkNlC0dCCor481W8Iz9/5RyDvC4pLobquhq7ujrS0Nsn8Xdm5zP5Ys5nZBRfI7QVHoZCRa0IItTosXW+kZLYEF9LXXvoNw9S3R0N5PWQ1QozvziPm3bfFNpjB4FJz09i+B3D7YUC4crAFH/pUWKOMqpzvBJ/OfGfs1h7CZ/v/Mze+uNvp1qXgBV/cs9GtWfFn6GB+AtXhk5UEDLk/AInvrff94UxC3nwz/tZcmgJAH0u6sPIu0ciKdrOwjehWwLXb7mewTeKWPPCP4bxd9dJ9B+d578N2o5HYFPb3b9tgYCJv759+zJs2DD69OnD3LlzAXj44Ye56667KCwsZM6cOXz44YctdqL/1/HDxT/w8aiP/W/ohOYq/g4Ui94U49v1Fz61hX/BsA9gyHwX+8AoTwUUkiQUCvHDRBNYfev1d5SJv9M58HeGzWqjvrreZ3Jg6aGl3PfbfR77LwFsztuM0Wy0e38HgqFpQwH3xZkzKowVdkVKx1jXCniz2UFY1OtjMSe1Q6n572lgGiwqK8V386T4KymBpbct5c9H/gzZ8cxWM9sLtgMwKHWQ/TjOVnP5+SK54Ql+rT5zf4Fdj4HNyw6cIH9nk8kpyFTpIH+FSx+yeJ34QWTLuaxJWVy3+Tp7U922RniPTI4MuYCwjt6rQ6OiwCIZ2BRzNz/tDY5kaYy1J9ay9NBSj723QFy7556Dq692Jxqv6HcFNw++2a6GKDRmA5AQ1nTbAG/Q6yG/ujNlVa6ZGLkSrbGFYO7mXNb9ex11VX6yai0IXxXvw+8YzkULL7InhuWgNBgLtRMVJzj7a5GU7hbfjbjwODvxJ9tJeUNYmIN4ONood+ys+HPuy+ai+KvNBWMjeRqIOTVIBKr46xIvsgmHSw+7zUVxneNaTDXWVMWfjOiMaLSDe2HSRtrHUFUVnIpZTIEhhwpjhUfiLztbzGFRUY4FuC/IcY7zPeSxx9/B+fBNGBS7qh1CDXlx7e+6bv9kO/P08zi+6rjvDUOIgdcOZMpLTbBGaSbq67Grcr0p/hJiNYTXi8HpTOzIScwic4Pir1G805pI0iehkBRYbVZO1Zxye1/XtxNHBs7BEh/qHjzNw7aPt3F8zXG3PqaNcaj0EGXGMsKUYS79oloC2RXZQHDXU972aHnziL9ldyxjfq/5GMsC68Xb0laf2mgtj1ke89njL65zHCPuHkHaENcYSX5OR6dFMOrBUaSf4WWABYCdhTsxW80k6hLtCbZQIizM0de+utrmkyToe2lfLvjpAlL6hT6uCjV2fb2LHZ8LhWRTevz5Iv6sVkdfvNENRgONFX82Gzz/vLDJUqngvvsgM9N1P1otRNX2B2BP0R6X967sfyXPTphHVOk4wAPxFxYPSaPE32YDFG0Qtp8+0DOxJ/WWejbnbqaqrspj4W91QTWvZ73e5n2pPWFr/haOJb9GYexPPrfbfWo3Sw8tDZrwstlsfLbzM37c9yO5lU3rP39V/6uYpXqWkfvWoDM6iBln8tleg3HobUHuNAfVR2HfS1Cxz+VlF8WfkwvJe4Pf45NxnzTvmEFCUkho9BpUYSr7a8HGsgpJwYSOE5jWeRovv2Kmb19IapjSnRV/f5cLt6XZ3WcjSZJLH0tZLRZIni82Qk+YSTzndue6WiwX1YjgOMyciDpMA30eg4zzAvsiwWDtBbBLOBlEaUVywmCuJiwyjNEPjW6xlgLBQHY7U9rEw6Spir/6elc1njPxtyl3E0tyPwfJhtkMr6x9C4vNQoKyI1pTO79ErkrliK88FSlXGkQCSIMeW7fbKY670PcOg4U6Eno9KJ7XTjhUcohdpZuxSOLhoI6LPC166F7f8XmGHVxOF2lqQNvLcYlBc9Kr1adRnWPfdqhIp7J5s+OaB5qDl9uDeFP81TYQf3p1BOT8Ct9Fw4nvA/oeLY4dD8POR1Aqxf2otsQAjoLm0w21plpybJsojvqDQX09CFVsVijb4Yg7Cn5vVeHS6YCAs0yrVq2iV69ePPfcc/To0YMrrriCdevW8cADD7Bw4UJefvllYv11nv0fvGLQDYMYftfwoD7TXOLv/Rnvk3NnDrMH/QvOOQw97oHUyZAwjORkIZGdNElMQC4o2iAacwNEd4fBr4dMmeILNpuNUR+N4vPEdIyqAjfi78TaE7yY8CL/fPBPi59LMKgurOa5yOdYdrtnewyTxcTc7+by7/X/drPbBJE8lv3ug6mele0EFx5Y6NaLQcaxclH9nqhLJELjyiDJQa5ZUUO9sojyU/VY6v2TSP+tqK0VGXRvxN/+n/aT/Vd2yI63v3g/BrOBCE2EPSkvq/0yMkQwajZjJyMaQ04eebXb6v8inLkVwvxnvdVqh5Tfrm6UFDC3EkZ8Yt+useJPF6+j3eB2QdldtSQC6YsSEQFHUl/kaMor3Lr0lmYd75UNr3DWV2fx/tb3Pb7/4YfCPrW4GHbtcn3v0bGP8tb0t+if0h+bzcYpUzYA7SMyYf+roo+qKTSyRL0e9OoyktVbsTkxyfL1bEz8HVpyiN/v+71N+6UGk/jS60GhUFJbK7mpK932a67DZrNx3oLz7FZVw9uLuVcmFXwp/gBuWXwLK9udg0Gd40b8OSv+6i31XDfwOub2nOuq+Pv7SljogWirr4AjH8OpNe7veUEwPf6UkhKD2UBelWsz7oHXDOSiXy8iMjWwaslg0NQefzJsNptLX06rFaprLJyKFhWH07tOtxN7p5w4FDnJ2aVLYLZMcgGFc+GF/Ju6EH+RnSB9jrDj9YXcxXD0E/8H9gJnQndv0V6+3f0tW/PcLXUi0yLJHJuJKrxxoHb64VDJIZYdXmbvBeUMrdb/HJKXJ4qR9HoPvacbEBUFujp3RZfRCDasKBQSElKbKv6UCqU9mZRf7a7OiusUR1m73tRIp08FsMVkYdENi1j3/Dr7M86b9VisNpY3p73Jg6MeRBOAqscbthdsZ9gHwxj/6Xiv29gVfzGZAe83VFafXc/uyuiHR/vfsAEtbfUJImHtq0BPn6hnyktTXPqh2WyO+TapexwT502k/bDg7Fmdx6/sMjK43eAmqYL9QZLEb1kasZaMdyMY/oH3tXN8l3i6z+yOJqIVJEvNxLrn17H6KeEAEcy8GYjV58mT4hmo1cLghmVkfqHJPn46x3Xm+HHYuFGs+R95BEaNct+PVgsRhp6AmJecMaXTFO4Z9iBR1eIAboUBpiqwNvSONuTCbyPh8Hs+v1uHmA50jOmIxWYRPUM99NhShikJjw1Hoz89rvHKJ1by6w2/AlBcI2LrcKtva6apX0zlrK/OYn+x9zYFnrCjcAeHSw+jVWmZ3tU74e8PUxMuQ2tK8xj/uEzNW/7l95r5ReVB2Hav6GXuBK3W1epTLlBL6J5AfNcmJrqaiNriWnI352IsdwTWsntFMLHs0kuWsuSSJXSKE/G+TPzJij+rVM/qQnGvyH3i09PFb2E0OqwFA0mvShLEmLujM3aiuNKVQJYVfxpzYsuqN4e+Bz0fACCyIadk5w5cygABAABJREFUsFZjtbW91XL+P/ksmLOAkytF/KmwiIdJUxR/2dlwww1w112OAnmZoIuItHLuN+dy1cLLUSYdoF5ZxsubngNgvPQEEpLX+FVGnbkOVYxYkDqPSRmVRhGEhTXEiIHEz6HAoPcGMfzDoVj0wta5LM9A0d4iLKbWzw3WVdbx7axv+efDf0jT9CKxcgoJmsDiFtnq0+iF+KusFO/J2/bqJdaIlZWwv+ERHazir85Ljz+DRYxVnUoPmmghqNGeJkV/g9+CYULUFREBarN4EJXXlQNw7K9jvD/0fY6sCH0v50Cx/5f9bHh1AyaDyV78orToaBfnwSGn8iAs7Q8H/iP+P2UDTPPdDuv/GgIm/kaPHs1HH31Efn4+//nPf8jOzmbs2LF07dqVF154gQI5Y/b/OZRN9HIYfMNght/eusQfCA/yhIhUkcySHzRWMwosPPss3H67hw/tfBjWtkClkB9IkkR2eTbVihyMmhzyXHOXaGO0JPVOOu2sPhVKBf2u6Ef74Z4npF2ndlFjqiFCE8H0Lu6B++HSw1TUVaBVaemd1Dvg4w5pN4SBqQOps9Tx4TbPaly5P4qnaum6Ojia9BrLBkZQX/M4xXc9x+Hlvqsz/1uhVCqRpDgkyTUgcyb+7jhxB5euuDRkx5STuANSBqBoUPrIJEKnTg47Hk99xAC/VfdooiFuYGBZbxzf1cW6UOH6PJucNZnFFy/mxUnC/tNmE2pWU23g/WBaEqXr95O+eymqOu9eWgU1uRxJEeefV53bZJtLm83G+pPrARiZ7u4l+Ntv8Ouvjv8f9yLIqaqrIrcqF6NNrBrSozqISti8JaAITTAfEQEX9XmSV6cOpq7ccUPJAam8KJTR77J+XL3uauK7tO5i2xnerK6M5UZ+vuJn9v/sSJBERSmJiIjAapW8KqSMRli2wkT6qxlM+WIKT49/mrhwUXY5KmMUZrOjutaf4m/RoUXssyzCqMnliFO8W18vyHoQhLxWpeXdc95lwdwFdvskQFjsdLvNfceWWth4NRwLvFl2oMowtVJtT4776vMXajSX+Pt25rcYn38NEIuumhoo1W3EpColRhvDyPSRHhV/sq2ZbJvkD3Lhg/MiWy4ucimuaH8ujP4OdH56zuz7N/xzd2AH9wBnq89Ptn/ChT9cyFe7vnLbrtPkTlyy9JKgE/XNwXfnf8fKJ1YG/bkFexYw7ctpPPLXIy6vK5VKunfv7jd+du7v521ai452EH9yfAPiWkoo+GrYUQwPG+x2m20F+fj5Ve7En5zk85QkaCtIksTlf1zO6EdG+409EvWJ3DL0Fh4f93izjqlT69iUu4nNuZu9OmYsv3Q5B289yJmd3S2pvCErNovosGgiNc0rdOg+szsTnpkQ8Jqjpa0+DWUGjv11jKr84HquO/chC9aSGdzHr+wyIruOtAQiIkBjTsBgruVExQm/dqsVJyqor65vsfMJBc795FxmfT4L8O140BjOij9vP4M8H3buDO3aiX/n1GRjsVkIV4XTLrKd3Ua5UycY5KWWV6uFCGMPQMQRdWbXwEO+tyXJw7mvvUCoGWw2CE8TxYntz/X7/cZljgNgZfZKe7GByeSItcJjw7l+6/WMvCcAT+9WwNHfjnLwV9FQUS6q0+G7ADMtUsQTjYuy/OH7vUIZMq3zNLfi3UChVCrp0SMdSZJ8Ox7YbDDxL2ET2RwkjBCWrx1cVUphYaBqsPo0W83UW8QgmP3FbM5575zmHTNIHF5+mA+GfkD2ymwAbvj1Bq78Yypl+k1Ndq8Ax9pCVvyZFVXM7HI+/ZL7MSJ9BCDUzHLcurmhU0ugfR1nV/3JhN2H6akb5/K6rPjTmBOJUx+G38dCtns82WxkXggpEwCI0Tnm11pTLQvmLOD7C9tOyVSVV8W+H/cRXx3Pnpv3MOywKB7094z1RPx9+61YLx454sjFyMRfkWKnPa+QGd2JIykvUFlfTu+k3mRUCMtiX9ezoLoA7bNavkpJw4bFs+LPKAZqmEKH8q+JdDctaHL+2SvWXQIrXXOSMdoYAKTwcgA2/Xs183vNpzq/9a1566rqOLDwAKd2nwp6nSk7LflS/Blk4i8qHaUShjS0Ev77b7Euk6+7X8Vfg7OSSVXs8Vgy8afXREDSGJiwQvx9OiB5LCSeAYg1sqqR4s9mtVGdX92m7lA7P9vJirtXoFApyGkg/rSmNKKjGy0Sj30pHNB6Pw4dGjyzJQnUbVdgGfIxGwCC9pXS6/VcddVVrFq1ioMHDzJ37lzeeustMjIymDFjRkuc438VWqtRo80WGuIPAEsd1OaAuUbYOHwbBiWbvG/f434Y9Ibj/3ueh+UjhIS2hSE/rI2aHDY0crBM6p3ElSuvpMfsHi1+HsFAn6Rn5icz6XNxH4/vb8oVv/XI9JFc1u8yQFgRyAsquXq2f0p/1MrAI05Jkrh1yK0AzN8832MvGbniuWOMZ+IvzCyi1MKkk1h692sRRcjpAIvFSmGhsGP1pviTJCmkFaVb8wXxJ9t8goP4y8pyWAl66/MnL7A9Kv6sFijbCZbAEx1yIOpC/FUeFIsDs4hw0qPTOavLWfRKEo2Hi/cV81zkc6x9oWm9JUKN6j3HSc7ehFoye93m4T8fxqJwlAt7UtH4wta8rby47kU+3/k5hTWFqBVqBrVzzZIUFcH8+eLfMoHrifgzWUysObGG7m+KRioaUxKxEToY9p7oo9oMtYQz1Gr4p+BsPt/xDNVGR5LSm+IvNiuW9JHpqHXNWOE2E96Iv4oTFez4bIdLX0m12orZLMavt6TqsmXw5CerKao9xY6CHUzKmsS6q9bz0qTXuLzf5RQXCzWZWu2/utbu2a8qdlH8yQsBSfKhxAXochP0e8b99fBUGLMQ+jzh/l7VYSj9x62/nDNB5A/XD7qeu0fcbSc8ZRTtK2LFPSvI3dQ0qyhfaEqVtDPiu8ej6yRIkspKsSCr1IkKvTEdxqBSqOzEX0mJw4ZFVvx1DrCliC/iz5uyySf6vwhjFzbhgwLOij/7ODUU+/hE6+HYH8fI/8d7HzFvaNxPSobVaqWkpMRv/OxM/HlDdDTo6gXxd6TU3eozPBzCVGEtokQKBu0iRfbdU6I3OtJC399epvh93/ZwrQmFSkGH0R1IH5HuYmfcksiMyUQhKagx1bj0a3SGVqWlS3wXe0IqEAxKHUT5A+Usu9SzC0dLwdkmrCVachZsK+CzCZ+x/yffqqEV96zgq+mOpK9z0cgfdy1mwRzPLQe8ofH4XXdS9Hg6I/2MoPYTDPR6CK/rAAh1kNyywBN2f7ub1zq8xr4f93nd5nRA6oBUe4FoU3r81dd7jwOcC2EiIsTvp7JEct+gZ7lrxF1IkmQvppWJQU/QakFrakdG0XU8NfY5TA3xSFVdFUsOLWFbrrD/1Ok8FGckj4OOl4o3VOHQ8157UtEXPBF/cPq2+7hq7VXckX0HAMUGQbjo8K34k+eDYOw6bTYb3+39DoC5Pec24UwFrFYrFks5NpvNJf6RSUB7OwlJgqTRENtM+2ZNtLAQ1MS4vKzVgtISgcKmIlYbS42phSokAkBK/xQmPj+RpN6C9NmQs4H1hSswKyuaFMuaG5Suzoo/iwU0lng+OPc9tt2wzV4ADNCtm/hbvh6B5vmio8Sga2wnLVuKh5mS0KmrRHFpXcvGkxFh4WAT51NVV4Wx3OiioGxtdD27K49ZH2PwlYPpGtuTiFpRSO+PyG0f1Z5L+17K5f0ut8/hzvO3nDORrT53VgvrwLO7no02/hTHkl8HYN6EeVSUiUS/L+JPXp/ZJCsmZblH4q+jchSj927lyqj3sFXswVh6MPT5Z0utyAs7wR5nNRB/CcM7MebRMaj1rZ8riEqL4lHzo0x5aQoL897hRMIHWNR+Gt02ICM6gzRtF6IMfSktc8+NllYaqVeLZ7esDpTtPjdudOTftVr/cXBqRCoXd7qNrMJ7KCl1D/xqTGKwRoefPg4fLrDZwGYjMtJh9Sm3MMmamMWdJ++k55yebXZ6Z75xJlevuxqlWkl2SQPxV59mX9MDUF8Om66DDZdCn8dF3GEogNwlntuutBJaizNyRvANZZzQuXNnHnroIR555BEiIyNZvHhxqM7rvxaBNnl3xsn1J/lg2AccWnIo4M/U1DgWBk0h/m5cdCNnfXmWaAZdth1+TodD70BEZ0g7F5Q+lCbtporgXYYhHwx5UOfZTjKUsFdpqHPYt68RSfFfio25GwEY2k7MKjctuonYF2JZeEAkDTfnCeJvSLshQe/7wt4XkqBLoFdSL0oN7j+WbPXpjfiTF9bZHXZTPHom7Qa3Y+cXO/nz0dD1ujsdUFVlo7parCI9EX+1pUYOLj9GdUHoqprsxJ8TaSRbfToTf3KyszF8Vt1XH4al/WDHQwGfj0fFX/aXsP4Sr304dAk6+l3R77Tpn6I7ezw7Jt5JeLJngtpqs6JSqJBsCiJr+6JRhAVdYXvpT5dy/+/3c8XPVwDi+mlVrs/Lo0dFNXJGBlx3nXjNE/G3YM8CLvz+QvsCN75qjON6hjAxLUlwpGYSC/Y+TLXJUXmcEO6Z+AOw1FvatELeW8V7ct9kHqh8gOF3OCvkbYD4Db0Rf4WFkB8rrJTHJJ2LQlKy4uturH7xdo4c1Lj09/P30zs8+4soKHAc05mMlyRRwFFmKAvO5qb9OaKxemMcfAuWDYIa1xspUKtPgPvOuI+XprxEn2TXIpSq3Co2vLyhSWSOP8hxSlOrpCe/MJnuj18AOIi/Gq14HskEUlycqJI2m4XdmdHoGG9NUfzJYZz8jHUh/rbcJoqd/CFhaEBJTW/wRPzJVdvOMNeZ+f3B39n+6fYmHytY3FdyHxf+EnxPEVlp2pj4s9lsnDx50m/8HAjxFxnZoPizSZTVOrKYHm3L2hB2xZ8Hq8/4RCXGyCSMQZBZLY26yjpsVnF9nO2MG8NoNvLBPx+wq3BXk9ZDztAoNWREi/7hR0pDZyUUKtK3Kr+KL878gk1v+SiUdIL8e1mt/hXaTUFclzimvTmNDmM6+NyuPLuc4v3F9usjn4tKBRXHyyk9HNzCynn8FtcWk1eVh0JS2C20WwJ6vejRFKsWxYmyZaUnZE3KovdFvUnsGUCz1zaE2Wh2uyaBkAxhYY7tvPX5OyGc2cjKErFJcjJozSmcl/wQz0wQRUiBEH9hYSAh0ff4e9zS7z67ymxv0V6mfzWdS5YK5a3HYpme98HQd/1/oUYY22EsAFvytmCwVNm/qzPxt/E/G9n9ze6g990SkCSH3W5JQ7GOPkDFX25V4MTfjsIdHCw5SJgyjLO7nt3EsxXjt7JSXHxPxJ/9OW+ziaLSUMBqhpoTLi+FhYECFTeX1lN6f6mdANn11a4mOQw0B0m9khh1/yjiOotzkNdoSos+qFj20+2fEvN8DJf86FB6ObfP0WrF2G08JzWOWwNV/HmyrAdXq886/QCYc8qz60hzsXo2LOkHgE4n2RWc1fXVXP7H5Vy6LHSuSU2BJElICsm+vgT/xRUJugQ+n/U5L05+0SXmkWNJ2ZlBfvZuKRXE36SOk8hIjKNb7jP0185gepez7dv6svrUKDVEhYn+BvWqEo/En9IcRXTtQDL1g7HOLGB/9D3NjrfcMOYnmLTS5SU78Rcmvmzc0M6Mf2o8uvgWrgLzAkmSUCgVfJh7Jzszr8OiKnfdwFQNOx+HetfCoLjwOJaedZDhB3/nwH6li+sBQF7Ds0mr0NnbdAwaJMZuXh7sEK14A8q/R2ujeX7s63TJf5jyMvfYU1fbjcSKM+mR0APyfxN99Qxeevy0Nv65B75RgfGUIP5kq88G4u90QFRaFOkjxKLwWAPxp7OkuY5rTQxMWiWsSyVJzGV5S2HVdChsu3x2yMdsAGgy8bd69WquvPJKUlJSuPfee5k9ezbr1q0L5bn9fwNDmYHKnEqs5sCTg/JEEB4efCW9zWZj0cFFLD28FJPFJCw+u90h+vSlToYxP0LcgMB3OOhVmHkctC2/qJKJP32qkEH9/bfjPYvJwsonVrL729Mj+JeR83cOv1z1i9ekqqz4k21xwtXh1FnqWHFkBRarhaWHl7q8HwzC1eEc/tdhFl+82C43d8bjYx9n7VVruXrA1W7v1dVBeL1IuhjVOZRVWLBarPzx0B9U5VS1yQOrpSDL7yMibC5BvV4vxpeuIp+vz/yMvd/v9fj5puCKfldweb/LGdFe2HvU1kJ+wy3SsaN/xZ/PqnulDno9DGmB93yQFxYlzvx9xlwY9R3oxaRqspj4dPunvLLhFUwWk13NerqobM2SBlN4FGFaz1ObQlLwwYwPuEt5lBEH/uL7gVVcN+i6oI7x7tnvkhWbRYIuAYWk4LK+l7ltIy8AkpLEtQRxbesb8WiX9L2EigcqqHiggqsqjzDw6DfieuYugpLNQZ2XP8gLQudKUG+KP3OdmWfCnmHRDYtCeg7BwFfiKywyDG20awZfpxMJCW/EX0WllYIYoaCx7Z3NwoXCjtVqFX87E3/+ID9LNTFiMS2r/hqrcBceWEjci3GuParqSuHPyXDEs/0yAMZiUaHmjPQ5oI6Gw679JJ0JoqYifWQ6tx+7nX6X92v6TryguYo/wKXHX3U11IYJEqBTrOibolQ6FmBFRaKAwmYTz7RgEyY2m+PZ6lHxd+yzwJuBW81NTpQ5X9dkfYPy3oPqSalWsv7f6/2qfEKNppAndsVffIBsbCPIc2F7D7y4DKUSuphnMe0fA2+McthKGY1wJPklrls/ls93fN6k44cStw27jZVXrOT6Qde7vRcbCweHX0ZhT++97Vobv1z9C89FPofFZPEZe2zL38Z1v17HxM8mhuS48hj3ZE/8T/4/XP3L1by9+e2QHCtYaPQajq8+TsWJwKrMw8LE/QnuioxQIDo9mqG3DLWrVLxh7ndzue3IbfYx7Kwuu2TJJdyw/YYmn0OCLoGKByrYfN1mIsNaziFETsAma0SA5Wzr2xi6eB1zvppDu8E+GK02htVi5dnwZ/nhwh+A4CyyJcmRSPZG/MmvR8da+HLnl6iShRWlc5eWQIg/SfJcbHSyUlRlJGvFWiEgNfCmG2FJf7+bOff5W3tirX0+du7zt+qJVWx9Lzj3jpZCzsYcCraLH7a0TsSIEUrfir+0qOCtPj/8R8SQM7rNaPZY0+tF/sn5ueSm+Ks+KpLA2x9s1rEA0S7m186Ono842dYbXWOLPQv2sObZwPtetwRq6kVwr7Lqg4pldWodFXUVdiWnJGF3qKhXllEXv9WjG5Os+JMRSI8/gDrdEdZ2H8HNOwa6vH5ez/MYa3qOhMrJLdvjLzzF3v9aq4XBh3/mlvA19vu7LVFxsoJjfx5j0/5NPLn6cXJjvwGCK0rs3Vtsf+ml0LWreK2kRKxzDAawSHVsLBB9WidlTSIzLZxOhXdzruEXjEZHKwp/6xJnVxlPxF9bFbLJxJ81rNzlPNoChlIDR38/SunJUkw2cSJuz8G8xbD7KdjkHtP06iWuQ3W1w1JXRk6dyPV1jOpuj5PCw6FfwxJ5qUjLBiy8kcdvTY17Hqh9wU0MO7SUy/peDqdWwp55YAospmxxRPcQYiAaevzJVp9ODgu7vt7FgYUH2uLssFqsVOZWYrWI+et4mXjORkkenjfxQyBxpFCxLuwIRz8SPUkTRrTmKbc5giL+8vLymDdvHl27dmXcuHEcPnyYN954g7y8PN5//32GD2+56r7/y+g6vSt35d5Ftxnd/G/cALeALAgcKz9GblUuaoWaYe2HQURHQd4lj/P/YUsd/JgiGjPLkJrMHwcNmfjTJokMkDPXrFAqWPXkKvZ9f3rZuRTtK2L7J9s9qsUq6yrZVyTOVyb2pnSaAsCKoysAeGzMY4xMH8mMbk2z0o3WOhqcVtW5loEl6hM5I+MMj4m4ujrQmlKRbCr0NVpi137LngV7mfPVHLpM74LZYGbRjYvY8OoGt8/+t0GuwmocXEuSmNiN+jj6P3wWmeMym3WcgyUHmb9ZeEBeP+h6Pp35qb3pd3a22CYhQagW5OSmtx5/vqru0acLK8HkwBOHHq0+Y3pDxnmgET+MQlJw1S9XcfeKuz0qSNsahvxywmpK/S5sMqI7oLHEUVsdvAxpTIcxHLntCEX3FlH/SD03D7nZbRt7kiVa3FMREYJg8nQtJUkiKiwKdXUWEkqRMFl3kaj6CiHiIyt4buJYYk46+nNc2vdSNl67kWcnPOuyrSpMRd/L+pJ+hg9pTQvDm+KveH8xBdsL3ApltFpXNUpj7K/eSJ0mH5U5CsO+CXzwgeO9jRsdpHtAxF+D4k8bJwhTmfhrrMKVg2O5YhAQyviSTW7KPTsK/4IfE+FYI3IiaZTo2Xn4PRefmWAUf/WWenIqczhZ4SojVuvUxGTGtIi1a3N6/JlqTSy7cxk128SiorJS/Imq7Ue6bZTdchhw6fMXbH8/+fzkc5SrpT0Sf3OKYHQAfUoOvwffaKDIT8Lq6Gew+1m3l517NyZHNBB/1e7En6SQ+NehfzHzk5n+zykEqKus4+Cig5QdC64BXWVdpd1uqnNcgP6rTnB+fvoi/gBiI7UobWEuSXCDASp0W9lStNqrbWRrok9yH8ZmjiUlwl0tL8ch5eUtYwnZFKQNS6Pn3J4o1UqfxJ/sYDGs/bCQKOvke+VImbvib3vBdj7e/jG/HPgl6P3OWzOPnm/15J0t7zT53MKiwnio5iEmvzA5oO2dLaBbqs9fYOfhel0a20o297ppVVoGpg70v2EzIM+xccpMwLfizxnVha3fiygQWE1W+l7al/RRIuYKpscfOPr8eeohBE5qlMpFXPrTpbxh6sOWTrPZenK3vZdaIMSffE5WycTugn1szBHjXY4pEjReiD9Ttejxd+xLx2uSEhTqgIpj7hx+J69OfZU+yX3s+3ZW/F264lLOfqfpqrdQ4vsLvmfhtcKxp7zBTjFS4Yf4a4Lib3C7wfRP6c91A4MrXvQEmfgzGh3FWm55JqUWMi6AmGZafQKkzYCu/wKLI3D11q962hvTuHnvza1abLzt4228P/R9iveL62dX/FmDU/zJOaucSkcFr93uM3oxv6YOZvLn7vNHfLwrORQowRAfGUV5xN8cr9uO0ez4bSd0nMAgwwPEV48mUnEMji+A2uDcbgLCkPkwThSMarWQUDWBduZR6NQ6cjbmsPntzZiN3ltxtCQOLDzAZxM/Y8uqLby48Sly479oUFv6/6zNZuNI6RGiM4+wYAFccIHj+pSVOcQYlVEbMJgNpESk0DOxp30NcvCgI7ei0/l/rseHiwtu8qL421O9hsMpz3HM/AvSie8IqwudI4IdZTuF04yTFaKcT5SVdQUbj/PphE859qf3wpuWQt7WPD6f/Dm7f3GIPdyIv4y5MPIrGP6p2+cVChg/HmzY+LOR6CuiZAxDDy7hnkFPubwu233K+bqAHffUNdRE7MCgOeEyR5vNjuedXg/0uBfOPgD6zAB33MLodI0QA4UnCzeV+kxuj/2N3y77zb7J8juWs+7FthF+VeZU8mr7V1lxj8iX28waNKYk4lVOuSubDYo3OlwJVXqI6QfJE6DzdaDPaIMzbzsEzNhMmzaNDh068J///IdZs2axb98+1q5dy1VXXYXeW4f3/6HFICemIptQ5LXmuEhEDW43GJ3aw8o9ZyH8MdGz1NhSC5GdQR3jeM1qEUFEbsurQ+QgyhQugqjdux0TrqSQuGn3TUx7c1qLn0cw6H9Ffx6ofICsSVlu70lIzJ8+nzuH32lP7o3pMAaNUsOJihMcLj3MJX0vYd3V6+zS/6aiuLaYYR8M44HfHwgogK6vBwklEdb2KC1Kkk8cIntdLhmjMuh5Xk9U4Sr2LNjDsd9bf8IPNcrKQKlUebRfiI0FU3g0iWcN8VtJ7Q+P/vUoz655FoPJvTmFbE0nK8Tkxbec7G6MUPfZ8Wj1KaPBqlCpUBIbLrKSskrsl6t/Yf3L60NzEs1E7YJF9Fz9rhvJUG+p5+bFN3OoRDACcpLE0+8aDJQKz4155SRLVJRYVHRocN7yZPcpw07k6mww7AMRAIYQqvAI0qL2Y6tzJOzTo9MZmjaU1MhUt+1nfTaLITcHby8cKnhT/K1+ZjXvDngXc53r4jE6WvjneEuobjOJnkadrWejtIlV15gxIqlRWQmrRZEmKQG41srEnyJSLIiONKy5Giv+5AbY8pgBIKoLzK3w3McPRFKl4xWi0k6G/Lwe8QXMzndZqXpLlnjCx9s+Jv3VdG5deqvL6zarjfLj5VTmNHNAeEBziL/qgmo2vraRqp1i4MiKv255T3NL+BrGdHA0QJeJv2PHgu/vJ0NWFsp2n8594exQqN1603iEviOkz4YwPyvDv6+AnY+IoioneFL8nao55XHuju0YizamdUp/Sw6W8PU5X7Pn2z1BfU5WbCXpkzzGMpF+gtnCQpGQVKv9k/Oenu91dVCjFQPVk7X56YSYGIguPEjStuVUlrZNkqwxzrj3DDu57KvoSHawGJY2LCTH9aX4k8mezJjMoPdbZihjX/E+9hc3TykbLEkmJ9Gd1Uqhwtb3tzK/13xO7T7lc7vqgmr++fAfe9GePHeEm6vY9tE2yo4GR+qD//Ebasj3XiwNir9y/+uQny7/ibe6v4XJYPK7bWtDpVUx6/NZDPuXGDfBWH2Cb8Wf1erIGRw3iCSpmXoKYn/iyaI+bDi5gZoax2f9EX/h4VAc+TuTf+nJtb9eCzgUf3FKL8SfsRBOLICKXY7XhrwFZ24GL3G0M/417F/cMfwO2ke196j4azeoHfFdm9D3pAUw7slxjLxnJAAfDN3KmD07SVb67oHUlB5/V/S/gm03bGNS1qQmn6uMxEQ9ioaMoHyvyM95O/GnS4NR30DmRc0+Hp2uhoEvg9pRvS7HPH9H38mkzybZSeXojGjiu8S3al/e+up6qguqsdlsond4vYP4CyaWlZVuuVW5dst/mfgrjhLOEd7cnGRFGQSu+GsXnYDKEokNm5sK2q42M62CdReINj8tiMZFibu/3s2Sm5dQU9Q2VS+ZYzM5662zMKWJ57/SGh4wifvUqqfo/J/OPLf2ObtVq3OxtPzsrEhosPnMmoQkSXTsKOwhq6pgX4MmIZBr6az4a2zbCrDX+Dv72z9EruFrFBsuItm4MrAvEgwKfoctt0LVQftLMWExAFjU5QDUVtRTsL2A2uIWCGj8IKF7AtPfnk70YBHwK6xadGGNLqikEM8rlbv39JMrn+TewmQOpzzHli2O9YLNBqbKOJIqpzGju6tr1rBGYW2gxN8dy2/nr+79ORn/sV1oAOIZa5XE/ajTIdaWUV1B2ZKy3KYhMhKUVh3t6ye5FHbN+mIW095om7y7QqVg6L+G0nG8iAOvzfg3U3YUMlJ9i2Mj4ylYMdy1mH7sL9D3Sf5/RMDEn1qt5vvvvycnJ4cXXniBbo116P8DAEql/wC2Mfb9uC/opuNuAVkQWHl8JQCjM0aLF4rWw6oZcKoh81lXDMUboNKDraEmFiavhd5OA0hSCBn13heDP5kg0SG6A1mxWWTGtyMrSyxoNm50vJ/UK4mI5NOrQaqkkAiLDLN7/jsjMiySGwffyCtTX7G/plPrGJUxCoAVR1aE7DyWHV7GvuJ9vLDuBd7b+h4ltSXcu+Je3t3iueeCHCTGkEFlVCV/zuhA2pm9Hd9Lkrjt8G1ctCgEi4A2RmWlEr1eT1yc+yPRm2d+sLBYLfx25DfyqvLsfRudIVtsyglsrdbxb09KscYkgwt+GwVb7wrq/DxafVpN8GOq6PPXADkgLTGIDfcs2MPR344GdayWgq13Hwo6n+FWTffmpjd5e8vbTPp8Emar2Z7gX1D0GH3f7hvwOFtyaAkfb/uYo2W+v68cQMoJ6ECIPzuRq5egwwXCdjmE0OmVXP5TIduVr4V0vy2FxioEGX0u7sPE5yai0TsCY6VSSXp6LJIkeST+quur2aP+DIB/jbmSjh2hZ0+47TYY0sBtnmrIlQai+JPHgBQuVnsy8edN8Ren9eDr4k0pHxYPIz6BFKdkTt4S+Km9WKg36r8r/z6BKP5ki9LG1q5Wi5XXM1+3V82FEs0h/qI7RHPHiTsYdpdIolVVOcZW4zyzHJIuWOBwAghG8Qeudrh1dQ6+1U78GYvg1Fq3fhEekTpZKANj+vjebm4FXGQFpeuN7kz8JelFpshkNblYrMioLqym+IB7n86WQHSHaM79+Fy6nBXcjysXXTTu7wdi/Hbq1Mln/Cz390tLw56g9HqO0XCg3WPctHEsq4+vxmYDg9FKtVbEtD0S296autxYzrtb3uWl9S+5vafRQFzFMVKO/U3+4dNLoWQ2O8a0T8VfiIi/bgnd6BzX2T4GnNEc4i8rVhTiBUIY+cLxNcfZ813gJLg8N7SE1Sc2UcShUPseIDVFNSy+aTFFe0Thir3ivDyXhdcs5MTaEz4+7Q55/Naaaxnw7gBuWXyLaCPRgpB/xxTrYGZ0m0G/ZP821RmjM+g+uzv1VW3XuzhQhFLxV13tmMtO1ggCfXDCOFTmKFRWPX2S+9jVfjEx/osJtVqIMAoi60DxAcxWs534i0ZUz7vlJiI7wQV10PuxwL6QD8jn50z8Weot1FW2QOPMJqD/Ff3pfaFYK0dKKUQZ+qBTe2p66ECvpF48N/E5Hh/7eNDHay4hplQq6dy5E1FRYj9yjOXW46+FId/rxWGb+ePYH3aVXH1NvbBzC6IdTnMx7F/DuPPEnST2SMRoNmJDDKBge/ylRqQiIWG2mu09mpOThcqoOOoPACZ29GyLLce0en3glo6RkRK6OlEs46ySX5m9knzbdqxSPfUx4+CMb4Nr6RMo8pbDrifBXItWC6eilrLO/AZ7i/Yy6PpBXPbbZegS2qYfXFLvJIbcPIT6WPFwVVjDA36+Dmo3CIA1JxwOHs7EnzxmBtqu453p73DNgGsAQfp1EpfDnqsMjvjzrPgzmMVC16hKgjO+Ja7f9U3KP/tExhyY+CdEO/J+UztP5aFRD9FVMw6A8D5duL/0fnqd38vLTloO0enRDL5xMIqOIt5RWaLc15l7X4STPwshi7PaHLDYLBQbT6FJPIHF4ij+ra0FS4MIPapRjWJ8vOu6MlDiT1731GgPuczR1dWwumdflg+IYVPe3+I8a/NOH6uP8t2w5XYo3uSxXQxAp8mdSB3oXjzeGohKi2LaG9PsjolynjYq0mlOVKih33PQfpbrh5f0g9UzW+dEvSDkYzYABEz8LVy4kHPPPbdNTvK/CVZr8IHJqidX8du9v/nf0AlNtfosqS1hwZ4FAJzZWTThxpAP+ctERR5Ah4vgvLLAbQIlCc74RlTvtTBGpI/gyG1H+GrOV4wUuUAX4s9QZqAyN/Sqheag9Egp+f/kYzEF3utncpZI+t+27DaP6rCm4NK+l9oXFV/t/oqDJQd5acNLPLvG3WYMHMmArtJ0uhmvofNeC7+e84FLr8LwuPBWrcJrKRQXW6mrMxIT4z5+IyOh/d4VbLtuPnVVTV9YbsnbQpmxjOiwaEamj3R7X1baOdt7+Orz15hksMNiBFMVmIMbB862FfbHmEIt7AUjHZGObEEhkwf3nrqXS5ZcwukAU89+5HcZ47IwqzPX8czqZwDR01KlUNmDuYK6I+w6tcuuUvCHd7a8w9ULr2bpoaU+t3O2+gT/xJ/J5LDXCZWCszHkucKZGKusq+SVDa/w5Er3yqe1L6xl0Y1t1+PPW+Kry1ldGPXAKJfXrFYrFkulqMz1QPypFWoGFLxOaulczu45kTfegBdeEPtu7FAeiOLvsn6XUfNQDd+dJ/rx5OSI822shPGo+KvYCzm/gimI8Wmzil68YXFQ+g+U7bC/FYzVp7yYlBMQMpRqJWfcf0ZQduOBojk9/hRKBdHp0aR0FjevxQI5hQYsUp0b8XfWWTBzpvi3fO80VfFXUeF4vkqS0z1Y+Cf8Plo0YA8V1FEevYacCd0wVRgfzviQny74iXAPlas/XPQDH47w0TMyhNAn6ul/ZX+S+wbAkDthZPpIPp35KXeNcC9IsVqtFBQU+Iyf5TkwPQD34ehoqAzfwe7q1ew+tZu6OqjVZGNR1qBRajySj62Nqroqblx8Iw/98ZBHFad56Eh2Trgdk7Z5Tg+hQM7GHL6/4HtyNua42OuFN7oVywxl9qKYIWmhUYvP6DaDQ/86xGtnvub2XnOIv46xokLYXxGPP6x5dg2/XBm41WhLWn0Oun4Qt+y7hYRuvm0Fk/skc3f+3Yx7YhwAxlor2KwoOqRz0a8X0XFCcIpYefyuP7me7QXbWXJ4CWpl6G2jnSHPsV3Ms/nlwl8C6tU86LpBnPvhueiTTj+XorKjZSy8diFHfxf3YygVf3LiWK+H6oa446q+1zNx1wkm7z9ErDbObnXuT+0HIuYIr08nXKnHZDWxq3AXq7JXifOwiGerRwGoUgMqpwC3+ijsewXKAyPOcypz+HjbxxRphcOI87Pok7Gf8FaPls9BBAtzg2DbH1nULrIdD4x6gLm95vrd55HSI7y39T0q60KT65DHb2SkmIcaE3/2PFPZTth0k7BNay5MVbDmPNj7b/tLciyrtIibp7penMCqp1bxavtXA+6lGmoYzAb0aj0SUtA9/tRKtd3OWyYyk5KgJuwgRk0OKsLshd6N0auBS0kNIqceGQk6YwPxVyqIP6vNysTPJvJj0gDqVSUoojKhw/kQ3gLJ+rxFsOsJMFUQHg7ZSW/xZ9jtbDi5gcSeiWRNykId3rJzgz8YzOLBobTqAr6WZ6SfgYTEwZKDFFSL/p3OORP5uds+ogM3DL6BcZnj7J+VlZvbtom/AyH+RmeMZkbG5UQYu3sm/ixibCg08VjTz6PAmNik/LNP6DuIPLDG0S7o7K5n8+zEZ+kbIQpT27LHn4yqesH2qCyRrtfTUg87HoJjn8I/d8KGS6HSoV5MjxKLibBEUbQi232Wlps40O5xTiUuQKFyd9yQ7T4hcOJPtqyvCTvspvirUxdgUlYQHRYNm2+AX04j68naXDj4BpTvsM/pmwxf8uK6F8mvcuSCLSYLNmvbk5V24s952RQWB70egHZTHa/ZbFCb47D/bCOEfMwGgNZrzvb/CZriQX7OB+dwzvvnBPWZplZivb3lbWpNtfRP6e+YmDLmwIX1wgsZhCRa6aUMpny3aDzq9PAExIDyV9keYshVNMVOhe6fT/6cj874qFXPwx/WPreW9wa9h6nGvQL2k+2fsL1gu1tz52mdHbJpeVILBa7sfyUA606sY2u+aIQuJ0AaQ154jtfcxzm296lqN52OV40hZYAjM15XVcfBxQcp2lvkcR//LSgvh7q6OmJj3cdvRATYJAVWi81FZRQsZFXZxKyJqBQqt/c9EX/yIjw/321z74o/pRbO2gFD3w/q/GJiRP7Zam2UPBi3GPo6fM5l8kAm/tQ6NZLi9CB/PamLthVso8xYRqIukSv6XQE4CLkYo6jik8eCP+wtEooR595intCY+MtoiOO8EX/Olcu6mnWij2rjHm/NhF4PnWK3kmF8095Xo85cx90r7uaJVU+4Vehn/5XNzi92hvQcgkEwiS+bzUaF6RgWRa3HhKpKCiMp70oGHV1AdJRr2DNwoOsxAlH8aVVadGod8fFioW21wokTHqw+ZcVfuNOgzv4aVs8QAbU3FPwBK86Aoob+qe3PgWn/CF/65UNg99P2TZ2VYf4gW5Q2VvwBTHp+En0uDv0cHmwC0xm1xbWUZ5ejlCz2pNCfRV+xdGA4H5Zd5rKtUgnXXAP33it+k27dHOMvUMjJ07Iy1/5+dl4uph8MeBniBwe2w2NfwJ9TXfrYuMBYBHueE300nBJg4G7hevWAq5nZfSbhHpQDfS7pw/A7Tu8e2+nR6Vze73Jm95jt9p7NZqOgoMBr/OxcjZuZ6f9YUVGgrxMk9q7CXRiNUKUTFnM9Enq0OCkRCGRrd5PVZFfPOyO6fST1uhjKK9t+mVa0t4g9C/ZQV1lnf8aFhWG3vJIhz4/pUenEaGNa/Lxk4q8p1q2y4u9o2dFm9Y4a/dBo5nw9J+B9tKTVZzDQxeuIah/FngV7+GPqv4koPYkmVk/Xs7sS1T44slkev2tPrAXwmsgOJTwVMv03o+JkBds+3EbRPrGWClYp70vx52w9/8P5P2B42MDlQ2ahsUajrE2lpCTw/n4g5iYJBRnhQjn91OqnKKotIis2iw4WUbTqRvxVZwt3IbNz5dlB2HY3FAfWH+iNjW9w9cKr2akSDZqdx1DXc7rSc65vO83Wwjv93mHZHcs4Xn6ct4/czdHkV4JSifnDJ9s/4YZFN3Dlz1eGZH/y+JWJPzl56kb8VR+Gw+8Iwra5UOlFwXmFoz+XHPOoLOKAcu4jY1QGw24fhlrfevP2iXUn2P7pdsxGM3HhcVQ/VM3X3S1IKIO+ls52nyCIP9nms5vuDI8xHUCPHnDffXBXEOY9kZGgb6T4KzWU2m1GNeaEgFVuTUKP++CsnRCWQHg4qKziWsokrs1mazOCYP1L63mrx1sYchuIP1vgVp+x4bH0SRbrI3mekwm8khIHWd5YIQYOhZj8THfO8XjDdYOu4z8TPyW54myPxJ/RIp6jEWF6v/Fzs2CzurUhAMdYra00se/HfeRtbYF+kX6w/ZPtvNn9TbsYwU3xp1DB9D3Q71nocQ+M+g4iHLGiTMYVsQ+FQvSGLyqCXXkHOdTuKbalX4tSchc7ORcLB6z4i3co/pyJv9JKI6aGfokpESmi92n3OwNrPNkaSBoDM3Og4+X2OX29+mnu//1+DpUKF5c/H/mTZzTPUHGydQszbDYbH474kFVPiaKjUzWnuONwd/7uMgW93s9YkCSY8LtQPrchWrNvrYy2X1H+D6QNSQu6wrKpir9hacMYnTGae0bc41upVXMc9r/m3vy3ZJPwya1y77eB2QDm1lvNepId97mkD/2v7N9q5xAIes7tyYR5E9BEuq7gcitzueqXqxj83mCXJswA/VL6seC8Bay5ao1HiyMZZjNs2uQIKPwhMyaT3km9sdgsvL3lbcB70sRZcRMdDYaoFBLOG+9y35Rnl/P12V+3KUEQCsgTsacefxERkNtjElEP3tIsgmvFUUH8Te001eP78jk4B4X2JuCNWraYTI5KUq/kf5CBg1LpCGQ99vlrgN3qs1YkK0/tOUXupsD7UrQk1D9+S/rupS4Lm/UnRXXwiPQR9p58cgATWSWIv405G7l7+d1U1Xkn2WtNtXZ1QM9E3wkG50QLOIi/U6dcq5Tt+254bGq1oFAq3fuohgB6PYzK+I6Rmn9BjbDyig2PRULcJ6UG14t+/vfn80D5AyE9h2DgSfFnLDfyn67/cWsk/fKGl3lDMZITie97tvp0miMaz5laLfTv73gvmGIaSXKocgsK3BV/9oW+8zO8w/kw/GM/DaUlqNwHhkbjShUOA1+Fzje4nD8Ep/grM5a1uBWbDFnx15TE1z8f/MPrHV+nYHuBfSydrD4Cko2oxk3cGzBmDHz2GTz/fPDHc67glcepi8VSdHfocRdEuPfr9YjqY1C0BqqOeH6/cr+oSN1yK2y/T8RdDQiG0B14zUDGPjY2sHNqJra+t5XXMl9r1YX+woXCTlevh8kBOCBHR0NMjSjL3Zi7EYMBqsIF8Scnb9oaGqXGPh6dK2dlREda0NSWcyq7jRkiYMBVA3io9iEyx2baYxFPz8l9xaJlgb/5salwXiTXW+rtz9emKP46RHdAQqLWVEtRbdML1zqM6UC3Gd0Cdr5oSavPg4sOsvntzUElE6LSo9CmJyDZLGiUzesnKcdao9JbnvhzVk7abDaKa4upt/hfCK16ahWfTfyshc8ueGSMyuD+svsZeI3onePN6twbZOLPl+JP3kar0hKh1doV8Zs2BU/8AbQPE+N8UOoglly8hNemvoahRlQDuOUmsr+A386AqkOO1xKGwaQ1kD7H/0HB3svusPV3bNhcYunRD43mzNfODGg/LQmbzYbFZMFqtpJdns1Pha9wPOG9gAjc/cX7WXJoCXlVvudW+Tk7tkNo53x5XeRV8Zd2DswugvYzm38wSSGcpUZ8an9JqRQ23iqrq+Kv2zndOPO1M1u1jcv2T7bzy5W/YDY6nokmk3jGB1vENjpjNGd1Ocve2zg5GbvN56BYzzaf9s+ODszlQEZkJG5Wn7LDh9oci8KmJubEY/BDkn0dGFLo04UIQKFGp3NVbx5edphnNM+w4/MdfnbSMpAUEpJCwmgTiyVlEFaf4GiPtOa4sPuUSR9Z8Xc0+VW2q+dzqsY1YdO4M1ag/RrlNU91tZMLUwOMVrHYHKw+huLnFKIrfw/8iwSKmpPwjcalN5rRbORQySFKFaI3sqG8jgVzFrDto22hP74/SKLHW+/U3lzCUnrmvOw6NiUFRHWD6J7CuSrjPOFi1YB+KcIe/HjlMRLalwOirc7OAlGMEG/p5TGu69BBtAqJjXXkdvxB7lVtUpWSU+LItZwsFU57SluYKJbrdDUM+LenXbQNVOGit6syzD4PqMwxgGhXAJDQI4Ge5/VEoWxdSqm2qBZDmQFDmQgETlacpNBygKrw3URHO123nY+J9kf1jYKjuAGgCyDg+T+G/xF/bQyLyRKUBaSMphJ/kztNZvVVq7m4z8WOF2uOQ/4KqHNK/BZtENLo/OWuO0ifDWduhcRGVoUFv8MCPWS7eii3BK78+Urav9KeTWVLANeKzxF3jrDb15wu6Dy1M6MfHO32UJT7vPVK6oVe455FmdtrLqMyRnHkiOhbZPawJv/sM3j6aVi8OPDzOaerUJfK1dneiD/nhac2qoYazVG3yqPEHomcNf8s+l7aN/ATOA0hV8l6Csi8+VoHgwpjBRtOCvXOlE5TPG4j99bzRPwVNcpLOVe6NrbbIvtrOPKRqNQKEs6e9XYUroQt/7IXATS2+lx4zUK+nd22VTMyFKXFhNWWuwR/G3LE7z6i/Qj7a/ICV1MiehzkV+fzyt+v8PRqoaQqqC7g6VVP88fRP+yfOVB8ABs2EnQJPsl4m8090RIV5bi3TnhYa7nYtiYMF31U2wenAveHiAj44+iVfFnwpwjkAJVCZbehbKwC00RoUKjaLkTwpBQzVhhRapRuFaORmkjqqeVoysuUN+rf8/hfj/PahtcxKcvR60VyoTFk2+i0tMDP7/MdnzPuk3EcjRcFFHl5Tn0aG5KSZ3U+i3O7nUvvJEePBGL6QNaVouLZG5LHwZwSsVAx18DmW8QcC9DtNpf+j8EQRHHhcV6J3t8f+J1vzv3G/06CRHN6/KUNTWPE3SOI6RBjXwSXK0QyIzO6k9fP6XTuaqRAII/R8nJXxV+T0eMumFsJMV4UwnEDYeomGL9C/B3e3v5W496Nu0/t5utdX7O9YHszTqj5UGlV6BJ0QavfP/jnA3478ht15uDssvPz4YsvxL+vuSawpEl0NMQ2EH87C3dSXm1AsinRmdPok3R6EH8g7N3AUSDgjPCKQvr++Tr5iwJTo7c0isrUPPu8kkceEf/3pKad2X0myy5ZxoOjHgzpsW9cdCOJ/060tykAyKvKw2qzolVpfc7H3hCmCiM9WmRU9xUF12O9OWhJq8+t725l+R3Lg7LfTx+RTtcXrqEqIQvp44+Z32t+k459oOKAvX/82MyWL0KQCdSaGug5vyeJ/05kW77/5GP5sXKK9hZhMrRO4UugUCgVaGO0qHUiMRnsvBmI1WdjNcpokcdmzZrgiD95TkxVC+JvX/E+pnWZxjndzvFstQWQMgUGvAT6TMdrmlhIGiX6GgeAURmj0Cg1lFlPUhN2qM1Vs54gSRK37L2Fs948ixqTGOQqa2B94W5afBPTv5rOyuyVPreT54v2Ue19bhcs5GvmVfGnUIM2QSSDQwGF648iSYJUVlpcVWJtgWG3DePChReiiXAMQHlMBlvE9srUV1h88WK7y1Z4pIGiKFEIPKpdaHu5C8VfV3R1mSTrhTuTTERpzMLxQ6FPhegeoGyBvhJWk7DPs9S5KP6q6quIbBdJl7O6EJnquWivpTHirhHcvOdmaqLEuFRYw4Nal8jE3+oTwn5CjkWNRsjNN3Mo5Vm+qbrFLZ5ITXUtlAqU+FNpjdSrirHZ3PNP9baGZ4smCqK6Y1G0wG8aFg9pZwvirAHrTqyj65tdefOUcIirU4Qz+8vZDLx2YOiP7wf9r+jPzbtvptvQbnSoP5OEqvGu19OQ79qP3WZzEafEhceRES2YO1uSEC8UFMC+EmE9naJwWrc7QZLg2Wfho48C772p1+iJV4vJ9UiZQziTWylsYyOllNOzbZLNKoQ+NSfseTNFfQzgaGXS95K+zP1ubtBOEc2FPknPrftvZfKL4hkqz4taU5qr44CpUhThqtvmuXO64X/EX4gR7MA9vuo4z4Q9E3S1hByYefTRDwAu55mzEP6a6mK5QOoUGLfEYf8pQxMjklWaGNfXI7uKJKUuiNKkJqK6vprcqlxOGg4AcsVnix825Nh9Svze/VP6e93GbIZ58+Dzz+H3RgU9ViusEgpnDjsJMI8cge+/90wUgvDodoY/q88y5QHuKY1gTc+BbotKhUrBkJuGkNgz0et3+G9AWZmEWq0hNtZ9/EZGQsrhtVRvP+Thk4Hhr+y/sNgsdI3v6rE63Wx2LM6dib/Ehp+1seJPThqFh4sKSRfsewl2PiKqnYKEfGyZhASgfCccfBOqhL3vFf2vYMnFS7h5yM0ADLt9GGMfbx21iT/kzbqFw0Mvsgd/NpvNXoXu3FdRfm5aaqPokyRI6/4p/Tm327kAvLLhFR5b+Rj/Xu+ovNpTJIJBf2qGujrHAtE5OSr3+fNE/Hm1bQ0h9HrIqerO3pLxLqRTY+tWGdUF1ZxYe4L66gDlxCGEzea54j2mQww3777Zpcfft7u/JTY8Fo0UjlGTw0aDo/ikur6af6//N09svINq7X6v8+W4cXDFFXD99YGfY05lDquOryJH/RcgCIrGir9Hxz7Kzxf+TPeE7q5fzh8khUOxW7YdDs0XantnNOzH2RLS366VCiXxOpFka6xyKTlY4tK/NVRoDvHXcUJHprw0BX2S3p6Uqg0TxF/XhCAb+AUAZ8WzTPy5jMm1F8IfkwLfoUov7GZ8vR8/RBC58UNA4WClG1/Xd7e8y8U/Xsx3e75z282ur3fx6fhPKT3sQ6odIvS7vB/Xb7mehO6++4gB7N4N33wDJTXlXPfrdUz5YopHVY4kScTFxXmMn+fPF/dQ374wKcCfPjoatPXphFuSsdgsbCvYRueCB7mkMId7Rt4T2E5aAXLS4USF+6QQ2T6ago7Dg6tGaCHs/2U/X79RxObN4l7s3x9uucV9uwRdAlM7Tw058WM0GymuLeZwqSPQzYzJxPCwgb03721ywmRk+kgGpQ7CZG06CbTlnS08H/18wK4HLWn1OemFSVy64tKgPyfPteqsdDLHZzbp2K/sewWrzcr5vc53ne9aCM4FebKF9bHyY34/N+OjGdydf3eb95lqjOrCagq2F1BfU4/N5r3HsTcEYvWZr/uNcZ+Ms/e8lom/PXvgWMNPFwjxJ59TskJYfcpFpOCjKDlhKPS42z13YDW7Jmd9QKfW2W1ki6N+d1H8bft4Gz9c/MNpRejW1IuAUBkg8ZcW2WAJWen7WSIrAmULyeZCnn/lGEtei8rxrP1aGgqhfJdwdwoFak7AsS/B6FjgarUOxZ/swHJ89XG+mfkNORs9NLtvIST3SabbOd1QqBRsy9/GtC+n8VXpnUDTYlln6DRazi79nS55jzEiI0Db+AAREQEJVeOYsOsYb00RfZ/lWF9jEs9JqetNMGmVIHFDjSMfwg8JULRGEH8NJG6lsZrkvslc+MuFdJrivWivNfDomEd5q/9G0ouvDI746yAemDsKdlBhrBC9Ths48L9zN2BSlxChjOWMjDNcPidJjj5/EJjV58rslUS+EM7fPcYAjpyvjHrE4KyMGoxt4io0GdNCTxypdDDmZ6FCa4Bs4W6wlovzMCvpc3EfUge0QL/IIOBxnbn9Qfg+zqH0+u0M+LWry+fk/GtNlMjBFxTAwXKRm20f5pn4A1FYGmxxaXqEWLOerHHkEvMqxZo7WtnQQunvq2FrEN6+LQ1rPfzaBXY8ZJ8H1GaxUJYVf20NpVqsm+V5U1vfiPgb9BrMym1STrSl0RZk7+n3K/yXQ+GWhfeNsOgwes7pSWynAEtAGhCs4m9v0V5e3fAqB4oPuL+ZPEH0A4t0eiCGxUG7aaBudIC6UtGUuXGGUZ8BoxZAu5a32pAT70cqRSLeZnMsord9tI2vpn+FoTREgWkI8M253/DjJT+6vS4vlnomeCcS1qxxkD7r1zf6/F6HMquw0PH6e+/Bp5/CihWe9zksbRh3j7jb/n9Z8VdbC++/D/uFgt+eDEhpUCGYVRUUlHv2cG6qcvV0gNUKVVUS4eHhxMW5j1+tsp72+//AumO3h08HhoGpA3lp8kvcMewOj++Xl4v7WKl0rZSVib+SElcitzHB4ILRP8Co75t0nh6Jv46XCY/vRBH09k7qzbQu0+gUJ4L3Phf1YdB1g5p0vFCjcfBXZizDZrOhUqgY3M6xwNLrHbzKR2d+xw/n/8Dm6zbbA/YbB9+IhMTyI8s5VCKCtEDGKziSLBqNazWYTPx56vMnL7QjI4HcRaKPqil0vT3BqUK+2uqycPfW9237J9v5ePTHFO937wfX0jCbHVOMv8TXQ38+xEU/XsTAeLFAWsvz9p6piw4uwmA20F7XiZiaYV6JP6USzjvPdXHmD8PaDwMgu34jIIg/F+WmN/ycBusDTM5mfwNrL4DJ6yDrGvFa8Ub4OV0ssHHcY87JQl+4dsC13DPiHrvtkIwLfryAO0/eGdh5BYHmEH/OiIwEGzZqwkTyv3tS6JMHMvHXuMefA02oMCrbDkc/8fxeXaljLFrN4to2KLXl+95qFT3u5J5wBdUFbrupLaqlcGchxvIA/F5bCfv2wWOPwZdfwqL14pqlRKQQ6cGiVaFQkJGR4RY/HzoE27eLBfattwbuXh0VBRIS8QYxRv8pFKS5VguK02jxlxHlg/hL0ZPTayqWDgHayrYQzHVmvp31LdW/igKHa68VLhON7ataEnIvFrmniAytSuu1cC0QfDn7S7Zcv8VuIdgURKRGkDYsDaXGg5TcA1rS6jOxZyKZYzOD/lzlwQI6bvsJ/fA+nPXmWUF//vt937OpcBM6tY6XJr8U9OebAvl3rK11WL0eK/NP/J2WVfXA7m928+6AdynYVoDJ5Ih/gu3xV1npvjyX48tS9U5WHV9lLzpNSBCWZc6xQ2oA+Vv5t0+2DCYlIsVeMAdNKEpe2BH+muZ/uwZM6ijGanHUHy7ked7mPHZ/vbtNCtWcYao1semtTZxYd8Ku+FNaI4Ij/jwowGVYbVYH8RcZGuJPnn9jYsTcWFkpYnA5B2CPZ49+BEv6Chv6UCB/BWy41KWoLSwMVJZIlJISs1UseqsLqzm46CBVuaFdEwWK3Kpclh1exiGj6O3W1H6Nh0sPs69oH5Ik8djVw3l01JP07Bna51FYmOP85LEoW31qzIkoFIEXEzQJ0b2h8/WgTSE8HJQNJG55bdupN2UcWnqITW9uIi0ijU7aoYSb0oNal7SLbMdT457ih/N/QKMUH5RzJrvqFwIwMnE6Kg/Ffs5ry0AUf3JfeJNKJGMau231yX6PEftXMi5rjNf4uSUgE381DcSfp9YlrYUjK46w6c1NrD+8nm18RLluq+v1TJkEXW8FTcPkmHqmsCt2wpiMMUzKmkRGrCDeCgogu1bMjx113om/puDCrtfQPecFwsoG2F8rrBHruVh1A/FXtA5KN3n6eNtAqYVeD0H7mahUYg2ltsQAIr8Gogf4kluXcHL9yVY7LWO5kcU3L3YpuPOq+DuN0Rpj1u2YrX7E/+OwNjZi9oO0IWnM/W5u0Iu1YIm/7/d+z10r7uLhPx92fzOmF3S+FsJT3N+rOuLaz2/TDfBdVJNsBEOFXonCNmt/yV77Q14mQooPFHPsz2MYK06fBFhdZR11le72Vv76odhs8KMTX7hzp6s90Jo1jn8XOOUCTzY8e2U1YGMoFUqen/S8vWltVqxILP30k+il802D25sc9EeF64lSCZVITpX7g33Pgj08F/kcR38PQcPvNoDBIJRhBoMBrdb9vo6KVbFn9PWU9Brd5GNkRGdw98i7uWnITR7fl/v7xca6JjdjY0XS02Zztd+UF7webegiMt2teAOE7FnvYvWpiRXWkIrAElttBUu9hfDDuwivKLA/F+LC48i9K5cTd5xAp3ZIdxQKp+SFqiuze8x2CdazYrOY1kUkI+RemLLir1eSF9u+Bjj393O+lilyXOehnZCLNWjOz8JT3xZaIj0iAvTqcv49JBy23mZ/3ZviL2tSFlNfnUpku9aPoJwJLOdFavbKbDa/vdn+fC81lNr7Lt7a4WlU5hjKFAf5fq8gvmVruNFx5yMhhTQYHNxuMBIShXUnMKoK3BR/p2pOUVhd6NpvyVIPMf1B3yGwg2hiGm4iG4QL4ofwVAhLAKX4YZx/n0DsPp+b9Bz/nvJvu9qopVBQUMAzTz/Np/N7sP63NG69qgfPPP00BQXuxJU3/HzFz/x6/a+AGE8mZSlmlRhgPVNDT4j4Jf5GfQsTg+ylsfff8PdVUF/u/t7mG4VFutUk+hCsGC4U1rhf15QI8QAprCl0282w24ZxX8l9tBvc8v0K9izYw/qX1/vsI5aXJ8ghub/jthNCLd4lrovH7a1WKydOnHCLn2UL81GjAktIy5CT4BGVQ0mPSqfe5KqOPV3QIUY8BzwRf7LStM3t7Gww5+s5WAYMATxY+DWgur6ax/56jK93fR3yhvWygsxZWRQKhIIE7n5udy5bcRkp/T2snzygJa0+zXXmJv32dVX1xOfuxJbTtF7NZ3Y6kxt738jjYx6326e2NOT4zWSCjEhB/maXZ/v9XE1RDXsW7KHkYInfbVsT6SPSGf/MeGI6xniNf3xBfuZZre6kshxfyjbZcr8hED1xZcTHB/aMlFsQ1BW3Y9kly7h6gEMR4pX4+3MyrD3ffWcdLhIJ2QAhF+eV6za7PBunvjKVR+oeQZfQgrYZAcBQZmDprUvZs2CPXfGnsgSm+JOtn331+CuqKcJsNSMh2WOC5kKefyMixPxbWel6D9mJv8RR0OcJCA+RCj11sigYjxtifyksDDoV3Mums028Nf0tAHrO6cmjpkfpMbtHaI4bAL6a/hWvpL0COJSbasQP0ZQitlc2vEKX/3ThiVVPADBwIFx2mQfHnmZCkhxjr6rKRr2l3q74CzMnEhEB0rFPRXFpSyBpFAx9F2J6o1BAuEIkKSsMVVjNVpbftZx/PvinZY7tB9s/2s7Sfy1FUkhNLkh8dOyjzOoxi3C1WBjIxF9BtCD+pmSc6/FzwRJ/ckGuUVGMDYsL8VdfD7qansRXjyXTth/rzifIPbQ56PxzQNj3Cmx1FIRGa8VEY7BUY8VMXR28P/R9Phn3SeiP7Qe7vtzF0n8t5ae9P/FHxDXkxX3jej07XgqD/+P4f5/HYOjbLvu4e+Td/HbZb8zteQEAOQUGCk0NrjIxoSX+rhp4OZ0L7oOinvbiHK0plcSKaXTVi9YEnHNAtHo5ndDvWeHoh3i2qBqIP1nxV11Qzea3NlOwPfC1fXNxfPVxtry9xcWlyE78OSv+TJWw/zUobZtnjj+0yJj1g/8RfyFGqBe83hAs8bfs8DIAzuwchCKv5jj82tk1QEiZCJ1v9EwC7HsF1l8W+P6bCDnxvqdoD/oI8XvLv8ek5yfxsOFhYjsGp6BsSVzx1xVc9OtFLq9ZrBb2FwtpXY9Ez8Hs1q2QnS0WYqmpoup/U0MhiNUK69Y5tq2oED7jVVWOhdfeve4WkTJKipRM2n2cqblriFWnYrHAb7859gWuVnvt9CJJXFDrLleK7RRLx/EdTzvrnEAhJ3gtlnpUKvfxGx2rwBCdSrkyMFuMZYeXcfvS2+1Eisni33pGJtoaB4SS5Nnu06viz1QlbFOsXnxe/cBjjz+bTfhjV4rkbZmhjE+2f8IH/3wAwJp5a5jfe75Hcrs1YSg1kPb3jySc/Mcl+JMkidRI96yxY3HkeX+3DBFeZh9v/5haUy1vT3+b5ZcuZ0a3GT7PQx4/jXsg+erB4vKZfvPgzC2gDq1ful4PNaZothVOhxhHT05vxF+7we0YfsfwNiH+5GePJLn25Nv97W6W3LwEs1Hc3//ki2CuU2wnYiQ9HQtvB+DxlY/z8/IylhwSfWCH6ERQH0riLyosyj4XlUdspLTUMW50Onhj4xukvJzC7ctud3xIqYHxS0QgHQjanQnnHIJEJ+sYfQZM2yaUuIjEgZxUMjaj3qX0cCl7vttjb5TdHLz68st0aN+etfPmMa9kPwvq8njo+H7WzJtHh/btefWVVwLaT9HeIor3ifsyKgpqtGJBFlafSlJs6BN88vO3qsoRUzSrxx+InozjloHSw46SxkHn60Svm/RZ0Psx+7hXKh1JIaMRkvWC+PVE/LUmtn+8nZWPr/SqnDEa4YknxG8on/+mU0ItNjDVcx8Qm81GaWmpS/xcWQmrRSsVzj7b48e8QianOuU9wJ5rTxAhJfJb31TW6E4fm0+A83udz8orVvLcxOfc3tPpIGPXIoxf/9z6J+YElVZF7wt6Y0gV5Io3YmBf0T6eXv00d624K+SqKrnYb2/RXqwNRYfz1szj6l+uZt2Jdb4+GhBq6mtazbaoJa0+38h6g49HfRz052xpaRwcegnWvALytnonHbwhQhPBDZ1ucHESaWmEhzsKq1LDxb0ZiNVn8b5ivr/gew4tabp1f0sgbWgaYx4eQ1RalD0p3Tj+8QWVyhG/b2vUOUROGhdbGoi/OAfxd8YZjt8xEJtPgOSGGqTCQuiX0s9eRGS1erCHtENq+NMIA16Evk8EdmBgQMoAJCSMYScpMThiVpVWhVKjbHNFpy5ex+V/Xs6Qm4c4Kf4CtPqM8q/4k99L0iehVoZmzS3PvxEN+RRn4k+vdyKnkkZDn8cdRWjNhb6DaCfjtD+tFiSU1NU5rqOkkFr9uib3TyZjlLiv7b0abWLB3RTFn9wfbsGeBVz202X2FhQtgYgIOJzyPEMWJPHy+peZnDWZO3o/T0rZLLEGOva5KEhrBXQ0n8WQQ4u5tfcTSEqJTf/ZxIGFHpzHWgHjnxnPlauu5POdn/Nd7r+pDjvUbPVjXBxUhx2gRnsQhVXDmV2metyuWzfxjI6MDGwdmqRPQikpsUlW6tSFLsSfHDtIEoRVrEKx+0mqS462TP45fxkc+cD+3+gwR2LDrKzEYID4LvFBu9aFAmMeHcOVq66k0iZ+HJUlqsnOMnJh9qGy/diwojbFkxYTfO9oX5BzQCaT4xp2Mp/LsENLuCTjoZAeq6UQGQlqi6vVZ/rIdO7Ov7tV+zx2Pacrt+y/hV7nOwrxc2SrT1Oao0Cx6jD8cyfkLm61cwsGrcUZOeN/xF8bY/c3u1ly65Kgk23BEH+lhlI25go7sqmdPExKe/8NC7tAdbbr6/oO0OsRyLzY8VqXG90qJuwo+RtOfCuq11sQXeK6oJSUVNZVQqQY6PLv0dZBf6DILs/GaDYSpgyzW202xg8/iL+nTYOxDS1TNmwQf+/eLciCiAgH+VNY6GjQLkNOnDXG3r0SakMa6vxRLF+m4J9/HPaO8m/pXBHVPlIEwKfq3SvT2w1qxyVLLyFzXKa/r31aQib+tFqrRysxrdKEwlxPbY0Nix8R1qmaU8z9bi5vbHqD0R+PZlX2Kjq90Ykf97lbvTpDJgw8eb/LxJ+zUkw+Zzfi78iH8GMylGz0faJe4FHxB7CkD2wRRFhhTSFX/XIV9/52LyCUdlaTFXNd08jGUEEdEcbhQXMpSR8QUPDnj/g7s/OZZMVmUW4sZ+SHI9lZuJMpnab4VUq5qPecIAcivoi/qChAmwRxg0LuRy7uFYlnVv2Ireu/7K/fO/JeNl27iRsG3xDS4zUHzkUHzmNy1AOjuHTFpegTxY2/JW8LIAiF8HArHQvuRG2O42jpMe785XHqLHV0i+9GvEkQnaG2fxiWJqwEa2PFeJOVu3o9HCwRRLm353vAUPovwXfuB+cPRrORkxUn3Swj9/+8n+/P/56SA81TQrz68ss89/DDrLZYWGY0cgkwEbgEWG40stpi4bmHHgqI/Ltu83VcteYqQIwNpUVHWvFltK+eGXR/hUAQGelItMpzqZ34M54S/VNLg+vHTMIwaDfVrtB0QdebRXU0iB5/fZ+ECKFklCTX6ypbfRZWuxN/hlID+37c1yq2vGfNP4srV17p9f1Vq4TtbUICXH21sGfdXbcUgGmdA7d0++03sUDu1Ck4C14QiTmdTiQQKyvhYMUu6jQF2JSnjwU8CIvCsZljPfZq0ushvKoIWxAK2ZaAvDCViwq8EX92K2w/PXCbgs5xnVEr1NSYauzqyK92fcXH2z+22w01FY/8+QixL8Qyf/P8Jn2+urCaPx7+g8PLD/vfmJa1+ux6TleyJgevhK43K1Ebq6hcuY3qgra3ZAsEkuRQTyZpMoHAiL+k3knM+WYO3Wa0oldtkPAW/wDUmmrt5HdjTGt4vC5Y4Gr3KceXBfXiHnVW/MXEQL9+4t+BEn9ykrSgwPU4zipWt9zEhBVCMd9MRIZF8u7IP5m8vRBqHcWY1YXVnFx/ss0LEFVaFR3HdyShW4JLj79A1iSB9PjrGNORH8//kdfOfC0Up+sC5x5/zsRfi8PpfpbnF+ciNlOtiaO/H6X4QOu1HZj47ETO+1aoXOzKTWvTFX+D2w2mfZRom/LFzi/49cCvoTlRD4iKAsmmpqy+mC35WxiRPoIL0u4nqXKaGJcjPoUpG1rm4JUHRS/sHPH9EtWZJFecRXpYHyRJ4vZjtzPnqzktc2w/SOiWQIcxHZi/eT6fF95HtXZfk67l5tzN3P/b/RwqOURcHBTGCLVffNV40hI8LzKjo+H55+HZZwOzrFcqlHYFsEGT40L8VVfbOJQyj7zUdzF2vALL1G0YNQG6yASLM76G2Y41h1qpRq8W48CkLMdohNlfzubcDz0rHVsScZ3j6DCmA1VmkcRRWSId17NoA/w+Dgr+cHwgdxFsuBJq3Yub1FGlmBXVqEv7clX1AQYf+YGoqNDmk5UqC6a43RRG/2rPFbgU8VstYtyUN72tUItg6x2wSlzfiAhILT2Pl/v9xiNjHgHEnBeREhGw3X0oIEkSCd0SCI9zFNRGKGPQmJIIr2/viD8iu4p+ph0vabVzO93xP+KvjXHsr2NsfmszNmvgrK/Z7AiKAiH+fj/6O1ablZ6JPT3bsCjDQKUHD77U9Hta+CQHguEfwwVGUb3egghThdElXlhGGSKF/Z4cpFYXVnN42WGq8trGC74xTAYTm9/e7NaUOiUihaWXLOXds99F6UE9mZcnyD2lEmbMgBEjxOtbt4prv7ZBCT5ihGMR5kz8yYGFN7vPQ07Frt9/Lyw+Zci/pfPis2OsCCrKrCfc+kfIOLHuBD9e+iOFu9pWkRAs5MqbsDDPi+mjC3cxcNlzRBUd9mvPVG4sJ1IjAr/9xfsZ9+k4Tlae5MNtH/qs7PBF/Mm2Op4Uf7rGopeYPtDlZtA1zW7JY48/SYLej0LHywGHQqzcWI7ZambcE+O49cCtdjKmrWBTqSlP7UltdCphYSJB0v6V9sz8ZqZ94eYMmQTyloBTSApenfoqerWeHYU7yK/K97xhIzRb8VdzEsyhlwPIC3ibzdWXv1tCN4akDbFfVxlFe4t4d8C7bWLL4vzscUZMhxg6Te6EpBAPuK35WwEYlDqI8HAramsUA49+zY31hzGEZQNCVVNdLbZvKeKvKtqVaNfpHP2o5LkKgLzlwsbVENi95BWlW2HzzXYVrvw7BaL4e3rV02S8lsGzq11Vh13P7sqcb+Y0q3KzoKCAB+6/n1/r6hjmZZthwK91dTxw331B2X5GRUGUsTcDsj9jbHXTkvT+IEkO1Z8b8Ve5H7bdC6e8VNP4gs0mLEeChHxd6+pcFX+N55Kyo2UsmLOA/T/vD/7cgkRsx1iflqLLl4u/Z8wQ/aOqwndRJeUSrgpnbOZYn/uuqRFqlUOHYIkQ6zJ9euC9/ZwhP38rKuBo9S4A0jV9gt9RG0GngwMjruTU7Bvb9DwW3bCI1zJfo75KPJT9EX89EkJvx6ZWqumWIIiaPaeEhZ5slT8otXn9hVMiUjBZTazMXtmkz9dX17N23lqO/emfdIKWtfo8+52zGffEuKA/V1cHJen96fvFvXScEFyhyrw183hv63uU1TWPgG0K5JgmXinO+Xj5cXt/X28Ijwun9wW9ic06fVxhANY8t4aPzviIuso6e9Fl4/inqKaIrv/pSofXOnhUDJ19tpivjh+HjU4hSWUlWCUTBQbh2CL3zJRx8cXQpQtMnhzYuSYmimey0ejae0ouotPpAlcqkrtIWIA2Ljz2gdHp4wgzJ7nEsbu/2c1HZ3xE4c62XX/abDb7/HzbsNu4O3w3nfMfCEglJlt3nqo55XW9GBsey6wes7iw94UhO2cZMvHn7HjgkmPa9RT8PhYsIWylsmwoLB9q/29YGFRqd/PorjnctEi0xqjKr+LzyZ+z7aMgi65CBGflJjRN8SdJElOyptj/f34vD7a3IUJEBMTUiJ72m3M3A42upy4Noru3zMHNVUIAUCFyc/J8J4/VqPZRaCKa2fC7ibCYLKKtS0NfbaUtvEnE3xOrnuDF9S/y3d7viI0Fi8KAZFWRXD7DZ062WzfoGMT0KhPFRvVJlwLlosoqDrR/mK3tbsQWFg8xfbApWsjHPiweVK6JJrnPn1lV3iyXmebCbDRjs9qECIRGij9Dnuiv7ixEqdgDxz4V7znh/O/Op93r8VSm/YiEkrLDXYmvHuvV1r6pMJqNLM/qw+YuMzheKOKlyhox2UdEAOZqWD0D9r8a2gM3FzXZYv2LyKHo67Popp5kjyNsVhvFB4opzy5vtVPK35ZP+XHX4705dgFTdhSSYZnoiD/UEZA0xl5Q+z/8j/gLOYJVnE17Yxp3599NeGzgflLOC8ZAqrHsNp+dvNh8drsNztoOuvbed2JskButPV9YenqCSh9ylYo3jMkYw/jM8ei1IgKTf5MTa0/w5bQvyV6V3Srn4Q+1RbUsuXkJe77d4/K6XqPnzM5nckX/Kzx+7nBD8XDnzqJyvmNHYa9SXw+PP+5Iro0Z47BdKShwJCtHjhS2AtnZYhHYGAdFvhhJEgu37dsd79XUiDylc/I9K6HB8kJ13KuqJOfvHHZ9uQurqe36PzYFckAaH6/1OH7jO8VSntmPOl2cV3WYjK7xXcm9K5f1V68nNULYS8ZoY3j/nPd9PhsCIf6cFX8yWek2/lMmwpC3hB1gEyAfv6JCFBjY0fN+u7VgrDYWqcGyp9TQWBrYdjA5xXdqtVCD5Vblsil3k0t/PxlygN64abYzZnSbwfE7jvPCpBe4pG9gFUMu6j0nyInoqirclKMuKsHFPWH1rICOFQw0GvG7jO3wJYqNl/slFxVqBXVVdW2i5PSW+DKUGVwSIrLib1DqIDIzk4Q1buUUju5oj8IajsKq4dzOF3jvO9NMDGs/jPjweOLDXC1BdDqbXfHXNd5JrpS/XFhnNzdxUnMSDr0NxSL556lK2hvs1q4G1+rphO4J9L6gd7MI/A/ef5/xarVX0k/GMGCcRsOHH3zgdRtDmYFdX++i5JCoQnAeT4FanDcFXom/2IEweT1kNKFaedlgWD7c9TVjMayaAdnfOF7bdh8sHWCXUTgTurLir95S72ZLGNc5jtlfzqb7zBZK5jjBUGrAUu85uX7smCDtVCqYMEEoSIqjRE/EMRnj0ao8JyckSaKuLo1bb1Xw2GNw112i0CUiwrUHVTCQ75e3d7zEtmqhOOyg9UH8NdEeu7n4aNtHPPzHw25WyzodIElt3uMvsl0k0RnRGCwio+LNGmtvccsp/gDGZ45nctZkwtXh7CzcidVmJSUixaONdzAYlzkOgHUn11Fvqfe9sQdEZ0Rzy75bGP1gYD2gW9Lqs6moqwMkiYgkXVCW/SaLiWdWP8PNS29GHa0OrePK0c9g47VQ5z3GlH/LCGt7zu12LjcNvok6S2CKr7awWPIFQ4mBsmNlKFQK+xqrcVL6852fk1uVS05lDmM/GcurG151+R4REXDOOeLf33zjUONVVIBBcwKLzYJWpXUbMz16wCuviMR0INBoHO4gzrU7cpzlNj/X5om8QdlO951VZ8OJ76DW3U3GGzz1P+0wpgNTXp5CdIdozx9qJeRtyeMpxVOsf2k98bp44i29CDelB0z8vTjpRd45+x1stN79KUkSKSkpdoVLba1jHeNyLY0FULEXpBAWd8cNEn8aEBYmLATXlf3Ib0dF/5GI5AimvzOdnnNaZm7xhDXz1vD3a38DoVH8AVwz8BpAKG77p/Rv9jl6Q2QkRNUKS9yTlSf5ef/P7Dy1DYtkENfTkO+553QoEDsALqyHXg8AIOlKORn/CT8d/RSAytzKNisO/3j0x7ya/iq1JvHgUFqbRvzN7j4bgB/3/Uh8PHTNf4zxuw/Tre7SkDqRyMRfY8VfTplIBqlsOnS2eqT6ElKSk1vG8ayuFEo2uxQu3jj4Rv7V7yE05gSMRtj30z7+eOiPoAQsocDHYz7m9azXqa4XrLbKGuG4nhlz4LwySHWQ7XS9FeZWuTxvwNFbtSx+BTas9txXqPMFeo2eCJuYe3fniyTvB+FdWdY/mhOmbUKEM/xTyLoqtAduLsb8LHoP4tkpy2qx8lb3t/j9gd9b7ZQ+Hfcpv17rqpqWx4jLdTNVCiXlaYq2cCn8H/EXYiiC7NSrChMSWVnBEAg8eq97gc1mY/kRwRIF1d/PGesugV+7iERxzs9QusXzdvXlULgSapvWID4YvHvOu/x5xZ/0iZgAOH6TdoPbcc4H55A2NETNp5sJXaKOy36/jIHXBed9fKyheDiroUhBkgSZB6J3n9UqEmJ9+7r2W5CTlV27wqCGua2x6s9shqNHxb8vuMDxepcGYYrNJgJ/58XniIwhdCi5lsTKqV6JkpF3j+Tu/LtJ6R+ahuOthdpa8fCNj9d7HL8dJ3SkcsJM6iLiA7JnkiSJEekjWHXlKq7odwU/XfCTPbDwBln278vqMyDFXzMRFYU9cC0v97yNUqEkNlxkx4triynaV8SWd7ZQmRu8oiWUOPpnNv2Xv0hC3g6UStiYI0qeR6SP8Di5yolhf9c0XhfPfWfch0YZ2ArBm+IvMtKhXGk8huyfibJC5xtEv68WgF4PXeK2oM3/HOoEoZJTmcMrG17hzU1vumwb3yWe2w7fxtBbhnraVYvCU+LLYrLwYvyL/HixsM0tqS0huzwbgMFpg0lNTUGvFz+whMSgo98yeXsRCdZengPCEKBPUh+K7i3i3qyv7a9JEpSa8qg11aKUlK5Wn/2ehen7mqzItSNlEpx7HDqKwpFgrD4T9eKBUlRT5GfL4PHLV19xWYAloJcbDPz81Vde3y/eV8yPF//IwV8FgRoVBUZ1PlbJFPLr6AxZmSs/k+3EnzoCEkf4LpDyhnZniqIMZxjyIG8pVB9xvGYxignYLB5KztdVq9Ly4YwP+fmCnwlXuxaKaWO09Lm4DwndA+tD2xy8lvkaX5/ztcf35IKk4cPF80+vh/7GOxi1dzPXd33C6z63bVPw0kuJlJVJxMSIeVCjEfFJU3uwyM/f8hpHdrhjRG/vH9jxEHwXCzWBJ6BDgadXP828tfPshQIy9HoIqylFfWA3htK2sygd98Q4rlp9FXX14tnq7XrsKxIKvJYi/t6Y9gYrLlvBhI4T7L1dm6v2A3G+CboEak219kKSYKBUK0nonoA2JrCKe7lYq77eUeASChjKDPxyzS/s/X5v0J/1prD3hz1FezCYDUSHRTOy28ig178+cWqVsK6XvEvH5N+yzqDk5wt/5vVpr3ss8mqMj0d/zLsD3g3VmYYEU16awt15d6PWqV3aLMiw2Wx8uO1DALondMdsNXPXirv45cAvLvuZMUNcxyNH4J9/xLWtq4N6VQmpEe3oFNsJRQgKdJ3XnTLkWNptfq7cC9vuthcquaDztYIoSAq8wqOWYvanPczaREfhbOqAVEbcNYLo9LYl/rTRWnrM6UF8V8GMygWJgRB/4epw7j3jXi7vd7nXa/Tbkd9YsGeB3fI4FFAoFA3En8K+RslvMKVwIf6GzIc5ReDBpajJGPq2w+4cEfOorOIGqqoX2WVNhIbBNwxu1bzO1ve2sutL4RQgF4QoLE1X/AGMTB/J6itX8+cVf7ZowjciAtTWKJJVgsmf9e0s7j40kGrtfnE9lw2ClWe1zMElhYvrl0mbz46OVzH/sGgNsvDqhXw44sOWObYfZE3KovvM7hhMIp5SNJH4m9FtBgpJwdb8rdRqsgHQ1XcgITK0ErGJHScyKeEKIozdXHIGBZWiSExnS4Std6D4KZmUhMjQzr8yTnwrFLklm+wvPTLmEZ4Y8yzh9RkYjXBg4UHWPreW+poQBjQBIGuyuJ5eLZUlyVWMotKLdVyjsSeT8PtUX7Ku+0iqw1xJrlAiUSkSrQeKD2Gz2ahVFGBWVZIaEyvaemRdDkmjQn/gECEyEuqVZSwueJ+3N4u2X0q1krGPj6Xnea1TmGGz2Rjz6Bj6X9Xf5XU5/nAput9wOXwXedqSfy0yZv0ds9WP+H8cFn9NwBrh1O5TlB0NziIlGO/17PJsSmpLCFeFM7qDl4rUvGVCWuyt2jVhOCRPEHLvC+pg+Ceetyv+G/4YD3lL/J9YiCAHpfJvEtMhhoHXDCSukwcGpQ2gDleTNTGLxB6JLq+/s+Udvtn9DWUGz9deJuZk4g9E/4a0NBg6VFRn3nuvIH49WX22awfjx4t/L1vmqgQ5flwsRvR6kVhLa4ilzznHsdCsqXFV3YzNHMPYivdJL7nCJ1Eik9g5f+fw7axvMRvbtu9bIDAYxERiNJZ6Hb+N77PGOF5+nAd/f9DlenaJ78InMz+xV5T7gmyt2Wyrzz3PC1Wulx4g/uBsdedi97n3BVg+zK5Uig8Xi9qS2hJOrDnB4psWU7Qn9ERCMLApVdRGJSM1/CjbCoQtzODUwR63l6+pPxVnsPDW40+hcASSje0+7SrBaAUMfEn0Um0B6PXw5a6n2NO7BvSCfMqpzOHuFXfz8oaXW+SYTYEnxZ/ZaGbANQPIHJ8JCMujvTfv5YfzfyBSHcmRI0fQ6Vzve7U1isJCHwmpZkKSJCRJItWpgF6vh0OlDf39YjuiVjplCVThwmLHk612MFBHCFWv5JqMD0rx10hhVLy/mJdSXmLt82ubfFoVlZUEWvaRDFR48L212WzYrDbiu8Yz97u5dJkuFkpRUbC2+zCWDNRSod/a5HP0h8bPYPsz1lDQ9P7F/Z6Fwf9xfS22L1xYBz3ucbw2+A3hvqAWN2q/xKXcf8b51DXMo1cPuJpzu5/rVTnX0rDZbPS9tC+dz+rs9l5dHaxcKf491amVdPs0BTG1g0moG+Jxn6Wl8PTTNkpKaujb18o778Cnn4oexzNnNv1cZQK3s+Twr4vX+0gK6zMhPBWqDnnfpgUg94xtnMjV6SC68CAZm36gYGcbz6023z3+DCYDR8tE0NoSVp+NIVs8D0wNrqDOExSSgjEdBOlwx7I7eOyvxzCag1Nk15yqoeKkBw9vDwh34uxDqfozlBjY/tF28ra496/xh6YSf5tyRTJwcLvBHDt6LOj1r08Me1+0j8hb7HWTptqmJvVJOq0LFD1dj815m9lbtBetSsuGazZw38j7uHP4nfRK7OXy2ehomNIgcli/3hHfJtYNJfeuXLbdEBq7ROc+fzK8OivED4XJ6yBthvuOlFpBFJxaDVWB9cmM1Ks4nDqPk3GfUVDRvJ7EoUZ813jO//58us3oxqfbP+UP85NUhu9oMlnUGC9veJkLvr+AP47+4X/jAGGxWDhy5AhWq8WeNJVzCa3S488JWi2oLGJxVlXXdi1brtlwDRf8LCqjX5j8ApbHLAypegZouuIPYHSH0X77xDcX8jVMk1zXvhpzohibWVdBxtyWObjVAqfWCmUoEB0urqXBIq5l38v6MurBUW2iuJ7wzATOevMsh9WnNbxJhWWJ+kR7zPDJsafsr4faGvKGwTfw5IBPSKo8sxHxJ+JBvSIBkidg7XQ9R47nhXb+lZEwEvo952aVKP9uFguMflS0e1HrWrbNU2NMfHYiZ752JtV24k/nGJs5v0Buo1y0uUa0yWjU489ZfVut3YfaIpJgob6eAGlasZ49UnaQcmMFVoWY7DvEn77xCBX7IfsrqK8gKgrqVcV8UXE99/9+v32TcU+MazXiT5IkRt4zkj4XOxxcVmavZNafXdnR4TrX+CPhDMi8OLTFKiFEi4xZP/gf8dfG+GbmN3w7O7iG1x69172gY2xHSu8vZdWVq7wni7K/gn/uApuXG7Dbv2DEJ+LfkgTelC8xfWHQG5B4hv8TCxEU4VXYsLVIv4xQwGa1uQU4NpuNe3+7l4t+uIj8avdeTzabqNYEV+IvNRXeeQcefdShzgPPVp/t2omq+9RUsRhb4jT/yTafXbsKddeTT8I998C4ca4EV+PFZzBEydE/jnJo6SG33oanI+TkiyS5J3tsNhtfnPkF0ftFgsMb8ff70d95ft3zXPD9BZ438INAFH9FRQ7rHtme1I34K9kkemY0o6JXPocyZ07aWCQUKmYx0JzJgy5ndeGy3y4jdVDzLLeai5je7Tk44grqO4jBIRN/3ixVPFkWhALeFH/OrzkH8Tabd7Iw1IiIAIM5kmqD48aRr6UnBdiOz3e0St+wxvCk+AuLDGPG+zMYdL1QeSgkBT0SezC7h7BdqaqqcklSyM/IwkLH79sSgTxAaqqNivDt2LCi0+HZ5hOgdFvoLHbqy4RizFwbnOJP16D4q3W93mHRYcR1jkOX0HQZcXRUFIF27SsEoj3c8MtuX8ae7/agS9DR87yeJHQT92eYrh6jJgckK2mRTVDdBQi58EGGnej460xYFGJSQ1IIexcvuDJrJqMyvsNU659UeKPzGyw4b0Eoz84NkiQxff50ht8+3O29detEAj45Gfr1c7wuFxblejGCOHZMuBDExhp47DFbyBKN7RtukbCiEVwfvoJxu/Z77U8HQNebQZsEq89tVdtPb8RfeDhUJHXhyMDz0HWIb7XzccbxNcdZevtSTu0rwdpQU+HpNzxQcgAbNuLC40jSJ7lvEEKUGcrsyrxQEH8A53QV/oib8zbz+sbXCfMxJj3h/aHvs2B2YGNPoWiZPn+xWbE8UPEAYx4J3hvXm7W2P8jE35B2Q6gKdSAlKWDfv0VPXC9wXq9YbVZKaku8FlM6Y/r86cz8ZGaITjQ0OLz8MHt/EMlyT4q/jjEdeXHSi9w94m5itDG8MPkFXpn6imsP4Qb0bMi/ZWe7Ws9LEq6FSM2AL+LPLTehjoLEkaDz4Xzy99Xw5xTv7zshNTYGnbETABtPiDg/5+8c3h34bpMUry2FL3d9yWrlE1SG7wyY+NtXtI+lh5Z6VfTlVomJNC0qtOo3efzK6yKPir/cJWJ9GUoUbYAtt0GlKLhxVvwZzAZ7z873h77PDxf/ENpj+0BkaiRRaY4Fg0JSYDWJixgqErelIF+zZLNrsZXGnCje6/csdL+zZQ5uM8Hvo0VLAyBGJ06m3mbEbDXT99K+jHl4TJtY3MmQFX9Kq67J11K2+/z24MecjP8EaOL60k//b3mfzjkDea0epUiErMuxDZ5PVXULeYfH9hO2rU7EX5mhjJO1h6hTifPQJEYT3zUehbJtKIWXxr/FwCMLiDT0dlzPbffCjgdcNyzfJdouHHd1mnF2qehUeC9hZtE2pCWKHrrFCrLqUPU/nCwXk6faHEN8tBaKN8HPGdBgi3vaIOdHWH8JVB8lOho0DcRoVX0V5jZqkdAYR0qPkGs8hFFz0pX463kvDPPeUuT/R/yP+GtjDL9zOENu8VwJ7Q3BEH8AOrWOIWk+jtHncVGN52uxqw5gRtO1EyRhdMuz/harhazXs7j+WBRGda79N6kuqOaNzm/w1+N/tfg5BII9C/bwtPpp9v/iSJ7nVOZQXV+NSqFya7IOgnCpqBCLtA4d/B9DXoCdPCnbVorXlEqHleePPzqSwjLxJyfGk5Nh7FhcJjpPxJ8uykildjd5pf4X1qPuH8VNO28ic2ym/y/QxpBJtLAwd5VcdUE1OX/noKkWVaXechurT6wGYGha8LaIFovDVtMT8ZfQ4N5WX+8I/uSEkVtgMuZH4WHeDNgt0sqdXhz4Esw8KRo98//YO+vwqM71a98jmclI3BMCCe7u7lJaoMXqQr097akbbU/l1N3djbYUSqFIS3GKu3sCCcSI62Tk++PNHsl4MpNwfl/XdeWC7Nmz95vZ89rzPGstIX8JIvEX3iKc1mNbo40JsO6on7APXlUYKjhSIOQaeiX1cnl+sBN/rjYBrj7b8nKsQdUIwwbh+5XvQhIpANDpQKMsQ1a0TTCYsCX+KmornJgOfz74J5te2xSUtnhCQxkIUkC1SxfxA4Ip6++c6Q8sFgvXrhvM+i69KNRvQKeDbgnduHfgvVza0U6y1VACy3vDjn8H5sbHPxFyPee3NZjxZ1+UEpYUxpwNc+h9U8OD6VOvvJJvPGZXbPhao2HalVc6HT+94TR/PepczZ5TeQZkFuQmDcnhDUwuWCxw/FPY/ajbU+on/qwMnRbTIO3qht23IhM2XQ+n7QJXRXsgd7Wj0oLFAsc/hlPfAvDy6SpuXnyc0mox3u7L3ccP+35gd85up1vEdoxtVn+jLUJZmdGjbUo6j/31GAstN1Ci3WktSqqPgjriaXx8bUD9UdLSxL8ZGdCydhz6mg6eE38g/K57vwGWJkz8hbtO/CmVQEwMRcldQNvEtIs6ZKzOYOvbWykvtH1HXY3J3eK7ceyuY/w6+9egBfQsFgut32pN9MvRHC8UzKBASH0CXNfjOjbcsIG3Jr7Fo0Mf9ftv6HNLH7pe6UFGth6kdVsgE38yuQx1uBqV3n86SmMZf/2TgyAHnr8ROt4n/GXcwP5zvH/F/cS+EsuLG14MfFuaAOufW8/SO0SFpqtEbJwujgeHPMh/R//X67Xsxz5prRnooie/pD4NJZ7Z8hYL9HoZenj/20Akz6OrRd/flr2j7hIWakqax5PaHjm7c1j6r6Wc3X6Wilo3EnQe8NDKh7jo+4tYcXyFy9ezS+sSf2HBkb2sz/hzWDPveQx2BDhhVHYUjr4j5GAR33mFyXZTyb9LpVc1KaOo8EQhFfmOA7Q1IR9ihKzfBKvoAoTU/2KqBpKkFwW5aks4Cos6qP7YAMjVYg1VZ0MQpbUNBtKzbC788cAfbHxlY6MZfwCXdrqUEHkIcpmc1OpJQAMLd3/vDKvGuX1ZrauhUpXpkPiT/NkjQuLcvCu4ePDPB+n0fnuyEz4GoLzYSNnZsiZX+Fpy+xK2vLOFgYmjSC6aidYSh0IidvX/CHq/7vgGXbpgL8Y5qt+pFCrenvg2V7S5kzY5D4lTfbDSaggGthBrpUzjVk7li+oKdW2i2GfK5BCaKCRJLyS0uBSGLQB9mrADMtm+6CXVIuC17N/L+H6ye+uOQCJndw6f9P/EochH2j9pDK2CagXyfwH/JP6aGf3v7E+fm/3bvPoSxDxeeJyn1zxtrZbyiLA2ohovALr/TQWFXIFaKWbscs1B6wZaqVGiDlM3OeXcHXQJOtpf3N6hcuxgvhis2kW3c+kbJvn7tWjh20Y8Pl4E2qTkQWysrVJ0xAixQSspEZKfAMfqlKzat3e+lifG30+hY1nXtRtrs//w2ia5Um71OLjQISX+QkOdE39hSWE8XPgwkTPGAu4Zf2szhJHiiFYj/L5/SYnY98rlrheOISG2hKAk9ymxFF1WJDWS0i5JpLnz+AN4dOijLLtqGZPbTwZs8nzNibObT5N8ZDXqmhL25u7FgoVEfSKJetcSCs3J+LNXOJQW9FotKGtOw7nlUOubbJi/0OmgY+wm+hf2h+zfRJvUESjqfHTOVzrKJs34cQaT3pkUlLZ4gqvA1+FfD7Pw2oUUnRKFB+9ufZeXNrxklZgDrJKbY8bYAlPZ2baEWDAYfzKZjC4JggmWHf09Op3w8nh9wuvc1Psmx5O7PwuplwXmxikXQ593IKytNaHhS+JP8vgzmAwB34jfdPPNrK6tZYuX87YAawwGbrzpJqfXrvz9Sq783TkheKpYTIxaQxphYQ1MLshkkDkPjn1oO1aPkS+NfxKsib/uT4mfhuLUVw4+GRx+A/4aDWY7mqZMBvueFq8BGo2cnPI21jnqw+0fcuWCK/nloHPl+5VLrmTCaxOcjgcSxRnFzL98PkcWH3E4brHA4brapu7dpWMWvt37LWuKv6RGmeuW8Scl/qKiAhs0kILfZ8/axlu3ib+aQth8Iyg00PZmIT/XRJAYf5klmU6vBYMZ5g+GPzGcu0/cjb6NSLQrlbhMzirkCtpGt3VvJxAAyGQywtRi0v5xxo+UPFJCi/DAMH9lMhlDWg7h7gF388hQW5X48uPL2Za9zev7hz02jEH3DvL5ftK6LaBSn4VVnNt1juoS/2RKwTXD3hvKDeUcyD8ACMZfwLFqPGT+JBgHbmCf+LN619ZjsrtC9tZslt+7nPNHLxyZyLEvjmXaV9MA14w/VzBbzKw8uZL7VtznUMSTnCzeazDYxuWl0RMZ/dVoDuQdCEh7/ZL63HIj/Kh1z6SWycS6KM153neHeJOIm+zKEX6fqYNSufvE3XS/qrvP1wgGzh89z7b3tlF0ssjBe8pXZlGCTixc8yrynF6rqq2iqFqsf735xjcU0rOTxiaHOFOfN6Dvu07vaRRSL4VpWZAs9pJqNcgtauSIiUZao1636jqmfOpCKjYIsFgsvNP2HX6/TcgMP7HqCWb9PItsuSjIDCVfKAPsftjTZZoN0jPUl/Tnp5mCiR5qEeNjuLZcFJfar4EDCZkMOt4DSSKZFa5TITOLL3+5oZyD8w/y+dDPyd2X6+EiwcGer/Zw7Pdj/D3nb64y/kWIKbrBsq0twluw/ob17LxlJ8kRos/6nfgrPyUS36EJLl/OLs2m/dehrO7alrJykzXGV1gt5rgodSzsfgTZ9jsa9kf4AosZ/hwG2++yHtKrxKBgUYm+ue/rXbye8jpn/j4TvHa4wK5Pd3Fq5SnX82XCKEgc6/gGTYJgL8YOcLrWXQPu4t2L3kVuERcJVvJoWLteyMxKKuW5bDotdspaS6IolozpCxO3QssZwbl5QxHRSYzTqijCw0FuCUFpFouv4upiAMqyyvy2LWsoaspqKM8pp7bKVkwk7Z80NXaJv5JDsGE25Kxsknb9ryCA9bX/AGgS+rpbOQ073LDoBjac3kC5oZxXxr/i+YKmamGgLg9AsmzVeFHxM3Jx46/lBV3iunC44DBlmgOUlwuJkNCIUG7ddWvQ7+0r0kelkz4q3eHYoYJDAHSKcy0b5krm0xNCQiAmxhZAS7bbDyiVMGsWvPMOzJ8vJLhO1xWW28uFSnBM/FkAmTX4nqhqx2nDRk6WHHF+owsUnSri6JKjtL+4PVHpUd7f0EyQNjjJyVEu+69MLiMsum7h6iJOnlmcSWZJJgqZgkGpvgd/JEheelFRTp7DVsTFCS+k/Hzx3Nx6/OWsAlUURLtmufkCKfDtYL9VkQl56yF+OOhaMjh1sPWlrC1ZfD7kc8a+NJbB9w+muZC77TTJx9ZR0qcDRrORIalDPG6Ovfk2NgQGgy354mvizyFRmHYltLqiwR6N3qDTwaHSTuzmBXrGiMozmUxGtCaa/Mp8zledd5AQai7GrqtA5NkdZ9n7zV5GPCmS6+9te4/DBYfpl9KPtIg0UlNTuf56GDYMeveG7UINjuN1tjH2EmuBxlXdruTL3V9wLvpnVCFvAy52k6oI6Pp44G4a0dnKrvdH6lMbouXm3jcTGRqJud73bMs7YiMy4C7njZEvSExM5MWXXuKSuXNZXFODq6tsAS5Rq3nx+edJTHROyoclhRGW5LzryijOAEBjSGtcAnfAJ0LBwGSAPwZCRFcY/LX15fqsa3tPrgZDmwozikAVaTvW5ibhfRRS728dtgA0iXDiCwZF5DJg6DaKqx4D+pCgFwGCnHJfBVUDi7JzZRz48QApAxzZBgUFYn6Sy6FtnYjB0fNHOVN6BpVcRUz5CM6aRIKw/hyXXxenb9MmMqDr58hIkegvLbWtqdw+y7LjcPJzCG8PyZNEgLqxPpw+olWkkHVwJe2ml1XQ8s8P2PSfrrT+ZmKTtMceMpmMqNZRZNUptje0Oj5Q6BLXhb25ezmQf4BLOlwS1HstOryI2fNn8+DgBz0rpjQA9soagcLJlSeZP3s+M36cQZdZXby/wQ4NYfwdyDuA2WImNTyV5PBkNKmawPVfiwV6vSLGQWOVkLV2IRNpn/iTJGZdJUzq4/zR82x5cwuthre6YAoUUwenWv8vrSGl57Ho8CLOV51nZNpIWkfZNoaVtZVMmzeNitoKpneazpCWwmZDLhdqMceOwZ49YMHCGfk6TmZUWQtmGwupsCo/X0g1K5UeipLjhkFIhPcx9fR8OPE59P8AdJ7lbhItQplgX0Hw/H4bgg5TO/BA7gOowlRUfCo2akqzzmc2u6fvsSTzqVFqiAyNDEh7QYzzqampyGQyp7WVw7NMGBWwe1oREu6gKKXRgAwZKvQYZKVW1mRTwmK2MOSRIcR1FsmyVRmr+PvM3wyRXYEKiMyvS5r1erXJ2+YL7Itape+R2liX+NOUQ+Zyr/0rUNBqQWnWUysvoqymjJqyGopOFFFT4sNmJcC4+8TdWMwWQiNDWVgDJZbG+TUOaCF2ONHRosDM78Rf4Xax12/luuAhQZ8gJGblRqqVeVRUJBEWBoMUd1Fy5CImTEqE3OuQVeeROvT54MSfZXJhT2G0LVbCVHVfMJUIRuvbJtL/rv7ok4JNJ3XEw0UPYzaZeW/bV5yN0tLePBWXe28fERVlK5gJli1Iy+RQOp95A42hJSUd9gKg5wL296sH6TseYozCqKqwJv5m/TKrydrQalgr7j3tyDy3Mf5a2hJ/xXvhzPyGK/U0AZpD8vh/h+L1PwK5H9zgsrNlfNL/E7Z/tN2ve3hj/BVWFbLh9AYA/tX/X94vuHYK/BIgyrhCA8pARMq8QzI1Lws9cMF6/LmCxPjrHOtaElVi/Pma+APbJgwcE38gpLeSkwWD6777xJ46Nta1rKT0nUovvJsfLtWjlBusm8+W2o4AZFb45vmVsyuH5XcvJ2NNhu9/SDOgqkoMvgkJYU79d/+P+8lcl+nR33D96fUA9EnuY62E8gee/P0kxNcp20lBUreMvw0zhV9nI+CS8VewBTZdAwWbnc7XxQlWa1Tr5k3uJl/Sh/0jbkeREMewVsPYMGeDtdrRFYLB+JPYe0ql6ySTp8SfdaEpkwXNiFivh4LKVLZXPgJRPa3HJfnH+ow/AFNt05sPu2L8jX52NI+WPUpkeiQWi4XMYlHh1TKiJXK5nJiYGCIj5fTpIz5Cqc9I/Uuvd59YbyxGpY1CTwK1ykJ+UczinuX3OElpBg3mWr+kPgE+vuRjXh73MhGhjjvVHR/tYNt73hkunnDv/ffz6PPPM1yhYHxoKN8BK4FvgfGhoQxXKHj0+ee59z7nccpYY6TwRKFDJZ+EU0ViYow0p9Oxo5+NOvG5KEoqOw76dCFZrFCJYGQ9GXOXUp8nv4Z1l0J5hp83roNM7pj0A4gfKnzl6iN2AKhjYfsdDAx9lIEtFqKoEZkXib2cW+FcLb3/x/2sfjK4Euepg1J5ovYJ+t/pKO13pK4WKC3NloT+8+SfAAxOHYLSoqWiwtGnREJBgZh/W7d2nn8bA5nMxvrz6mEW2x+mn4e2t8Df18Ditk5M0GDBnccfQGhkKNXaaJTRQYpCeEBNaQ2nVp2iuqTaY2LodMlpJn03iXe2vBP0NnWNF3KaEtMsmMgozqDGVMP+/P1ez932wTa+GfcNNWW+BTODIfUZ1yWOUf8dRUIP1+wBT2hI4m9AiwEUP1zMkiuXWOdfj/336HuC9ZW7xvvFZTIxNqZcAr/EwLbbXJ7mwPhz413rCh2mdOCezHtof7EL2ZMLAPUZDO9te48bf7vRqiwiQa/SM7PLTAC+3P2lw2vS2Hf0KNSEnKOWKhQyBa0iAhPwl4KkFottX+KW8dfx3zDwM+8XrTwDuaugyjsbqGWISPydqThBSXUJhnIDOz/b2eSsk/pQqpXo4nWEaEJsjD+T71Kf1sRfpYvEnyTzGZ4S0IChff+tn7wIhs+VAywWKD8JZaI6R9o33VmdjfEJo9Un+/Cvh9n0RtPYDsgVcsa+MJYe1wi2scQ6tNSID8MUMxy6PAYxDSuQCzak/ldeDnl1fnA1FvE3hEYlwuUG6PNW8Bqw5hJYNw0Q62elSTSo3FBOrxt6cf+5+2k5tGXw7u8G6nA1oZFigdoQlrs79OolCvA7+WsBnjAaxq51m1BXypVWqdbqkCzr+llVlUps2UjaR3eECVuQXXLc+/zbGEzeBwO/sP4qKS+YlHWJv06pTHp7EnGdmlZ6VKVXYdaaeWDj9exsMwulum7irDgDP2pg75POb/prNGy93eX1ZDJbLDVYjD+NBrpX/4vEkiloKjoQVzKJFPqKF89vg4MvQ2VWcG7eUJzfDvNj4Mjb1jiV0hgJYGWgNzesjD9DK1ssrdVsmFECSb55BzcHgtZnPd2zye/4fxwmk+9BUkO5gfKccmpK/at8caujX4fNWSI43z6mvbWa2CMSRkOry/1qg1uMWARDfTO5bywkxlxF6FGHytkdn+xg1xe7mqQN3rDv+30sv3c5hnKbR8rR88Jkr2Os6+jlyTrluoYm/lLqSf8rlfD886ISv7YunupK5hNEcFwXUkSo4QgZxd0IVZZbF0bpYR0AOFvjG+Ov1YhWXL3iajpO8zdK27SorBTSHgUFZxz6r8Vi4ffbf2flIysdFtL10RiZTxBMCfCc+IurW09JUp9uGX993hS+KI2AS4+/uCHCayVuKCCCfV/u/pIFhxYQ1TqKy3+9nE6X+rvqDSwsoRqqw+JR6XxjLkvPtKpKVCwHAvZJPFf7cldsSgfGX/7fgrUZJLhjGkiejeerHBN/C69ZyH9V/8VsDA4D0R3cbcpUehVyhZyCygKrT0NqeComk4nDhw879N+EevHPRi3kLWbwIJutkCsYHiWqNg8YF/HWlrdo/057LNglDw68KDYdNQGUFzvwIvwcTqRaBIR8Yfx5wuwFs7nmz2sa3ax777uPzKwsut8xl/v1nbhCk8xbnToxYu5cMrOyXCb9AAoOF/BO23f4+xVnj8uMkgwAbpqZ5jTHeUXlGeEZpa7H7hi7Gvo5SlfVT/xptUDZEcheDMpGUEYrTgtPGE9eRxJqy2DofFYp1jN1nonD5VMBmwyYq8TfofmHWPfsOsym4PZVuVKOQuVYmCDJydknZKXE3/g246zzV3a2iPPZ59QKCsRcW1Z2yq/1sy+Qgt8SPLI31dGCLa9NhcjuYAqgDqMHtIlqw5rr1rDzlp1OhQK6cAVHhswhflrTM+nPbDrD12O+Zs/Xe6wFBa6kUv848QfLjy/nh/0/BL1NUrHft3u/ZW/u3qDeq1tCN0D4anpD4fFCsrdm+7yXC4bUZ3yXeIbPHU5sh1i/3mc22/YF/jI6I0Ij6J7Q3eX864S9T4KpShRf+Ap5CLS9FRJdB24ayvhTh6uJaBmBIiQ4BVYNwTvt3uGb8d8Azok/Sc7cnu0nYUYnIQv2d5bjnJleJzRjMkGFWnzmrSJbEaIIjA2GfZBU8vlzm/jzFe3vhtlVohDDC6JCowkxRqGQKckpz8FQbmDxTYvZ94P3/hpMVBZUkn8wn9rKWgePP1+lPn1h/AXa38++/9Z/dg4F5r/Ew0bf5Vh9xpLOsONuwDZH11ZqHZKbuz7bxV+POHs/NwWkBC61YsCRJY2BHs8JtYZz3q1PmhrSMzMaQaeIBCCuUMikWp9vUK19zNZFnlYLXU+/z7TKxbSNbhvEe3qGxWLh3K5znD52mtc3vc5ehShECETib9Ys+PFH93E1t1DHQHhn2POI8Gx3gdQIwQSvUp2xJv6kdYMU/zFZ8D7/BhAS489cl/jzteA0kDAZTJzdfpa8TNs4qQmxW+AnjBaefvVhKAKj+2pvSYSmQXPY/v/CqW+8nibdI6FgJgOOLWWY8n5xIG+tkA++0BJ/qkghua6OJzRU9JkQUyRgk/rMO5DHnm/2UF0c/C9D1pYs9nyzx1poZ7aYOVMiCn40hpaOc1aIHhTNLFXiAU3VZ+3xj9RnMyKmfYwTXdUXeGP8bTy9EcBBjs8jujzi/ZwLEJKMX01IjkMg++9X/iZEE0KvGxoudxgonFhxgj1f72H0f0dbj0mLd6nS2x5VVXBO+L1aN26+wF4xrT7jD4QU6IsvwptvwoYN0M+NcpFeD53i/iZV+QfvnfyQcoNNA71ddAc4A3nmo1gsFq8Vh9oYLW3Gt/H9j2gmSP5Jcnm9CcsCsxfOxmKycK4u4OWKHbbxjOhvw1sNb9D9pcRf/YCzPST2Ul6e2MhLAX6nxF9644P2Lhl/2hTxU4ed53Zyw6IbGNhiIJd1CpBnWSNRnl+JssaCMkRNuaHaK/tSpxPBC4tFjKn1vb0aAif2Xj1Ix115/EVEAHsfh8KdMLO48Y1xASlQdln4cNjWDfq9B8BbE9/CZDZZq2slJPdPxmwyYzKYkCubrk7IFQPh1KpT6JP0xHWKs1Z3JemTUCvVmEwmquvtPkJDbVJ/0IhgVMlh+GskRPfzKGH98VVPMndBAimtDIRpQxidPhq5/ea6Jh+K9jgxzBoFXRrEDUN/vgRI8XkDVlVbRUFlATqVjmiNreIgkLJniYmJ3HzL4xw5+jjx8fCZDwX/oRGhDH5wMC2HOc+No9JGIZfJ6d8Q2b1u/4Euc32Sb1SpRD+Riis0GkSQp+uTIG9ElODoe3DoZbj4MGiS4Lc20PY26PGs43lFe2BZT+j6JJW6pwHbHCVJfeaWOyf+xr06jjEvjAmqdEjJ6RJKTpeQ0CMBdZitc9ZP/BnNRlafEuzDcW3GUZks5q61a4XsuEwGb78NCoWNLaLRBF62oX7iz63HX9Ee4esX3gF6Ph/wdniCWqlmRJrroiFpfg9kgshXxHaIZfzr42kzrg2ZdY/G1ednS/AGv6q2c5xNJcMqNxUkSOzCk0UnqTBUoFO5p76Mf3W8X/6awZD6bCgMtprERkm51p9/nTB8IZQdgzY3er9YzirY8W/o+ZLwFXMDV4m//ArvjD+LxULJ6RIsJkuzK1VISO6XbJVKs2coG81G63qnTbTzfkr6nh47fwyj2Yiybo6z3z9WqgWbqk1UYPdjiYlw5ozN589lbKK6ALbdKoqLW870fEE/lC60Whiz7Qx336alQ6wMk8HEVcuuIjI90q+/IdDY++1eVty7guvXXd8gjz9Pib9RaaNYOHshupDA0/Ck/utW6tNigeg+EBbg5I1MBl2fEGtZbHOetOaRMPalsYx4aoRPsYfGojy3nIXXLKTL7C70vrG3LYFrEp+79VnuvA+OfQCTD0HEhVPgrFaLNtbWwvjkWeyc05Mn7hDPTa8qhHPbhU2ANjAeuU4Y+bv1vxoNJJRMJikcojRQVVTF8eXHie8aT0I3/9npDYWpxsTHvT8mdUYq93e9n1B9HGO5MSCJP8Dn/u0AY6VQSDv1DSAT/tL1IPkYV6tsjL8Nte9QEKfEGDILcveBIpzq6iDuzwt3Qd46aH0dqCJtjD+FGPCLThXx44t/0GV2F7rO7hq8dtihPLecT/p9Quc7OkM8yM2hhEoFibpUh++gAyZ5JoUkCYKl//Egswn2PgFRvb3G4WLjjWzIW8FPedtI5HH0+rp9afq1EDfcauFxwSCsLYyxFaSHh0PH7Be5+fYqRrQSrOjDCw+z+onV3LLzFpJ6JQW1Oft/2M+Wt7Zw75l7UYepKa0ppXtCdw5kZRFqSBFzmKEEshbWJYCbnl18IeOfxN//IKSAlLvEn1T5NyR1SBO1yA6FO+HcCmh9vQhuBRFS9XuNMo/aWrFZUqlg5k8zUagvjErOi96/iNHPjSZEa1sVLLp8EWdKztA9wdmEPCNDrK9jYvzTC/ck9SlBrYaHH4bbbnN/bZ0OMou7svDMO+zJHY1KZWMutY9tg8yiwCAr52zZWQcvMHewWCyUniklvEU4MnnTaxn7AmmDERrqyJSQyWVWj7OyuuJyV4GaLTdt4e8zfzfI3w9sib8YDzF3e6lP+yBgMDzLpASkg8dfPcRoRGMLKgswGUz88eAfJPVKouf1PQPfIB9x4r0V9Fy9l8LB0wh/oRd9k/uy9eatbs+Xy8X3vbxcJHQDmfhz17+8evx1fhSqg2d6Ls0ZIZYSB83+3km9XZ4/4K4BDfZ7awzqJ/4sFgs/XPIDLYe25OoVV1tlPr0x2hMSApD4C2sLZoPXApmUmEi+vPlh9yf0fk38BBJpl0Pa5dTW7XF8TfzdvPhmvtv3Ha+Me4UHBj9gPV5bWUtFfgXhKeEBSfR6lVish8i0SMa9PM7lazf1vombet/k/SK15SIgk3618CSV4CrpZyiBw69DeCfxWUrtiBTrLLncbkPf2KrBljOFSbo6DiwmsUF0VXUd3sHq/6fPq6Bb/FZCa1OBtg6Mv/oBsMhWkY1rnw/YP28/Kx9eyU1bbiKlv5j/DQabh56U+NuavZUyQxnRmmh6JfZiewrs3g1Ll9qudfq0YLJL35HIyADRru3gc+Jv+51QehimFwS8DY2BVgvRWXvI+KwAJo9p0ntHpkUy6F6xpjlaRyaq349NZhMrT64Emibx1ya6DTM7z0QpV5IWmRbUe8Xr4onXxZNXkcfB/IMeff78DURL67ZASn1ueXsLB348wMyfZxKW7PtkZ88S9zUIWllbydR5U+kW340Xx76IAh/2W/HDHcdjTzBVizlX5vm60nqmogLidIJWXFFbQWVtJdoQz4vjdzu8S+sxrbny9yAwmBqA6d9Pt/7ffv1zpuQMRrMRtULt0rM6NSIVbYiWytpKThSeoEOsUGaxH/sq6hJ/gWbc+MT4q8qGMwtF0sgbjFWQvx5CEyHKeX9sD+EdprOueRQqBW0nNh+jSEJK/xSGPzGciFYR7Lx1J3c/UIHaGO+/1KeLxF9SWBLTOk4LYGud4ZbxJ5PBqGXBuWnXudb/Soy/7apXmPHTFu7sdyej0kdZ/faaArUVtWRtzrJ6GdsncAHUJ16C/JWimEzfVkizX0CQycRzLCyEigoZKfoOyBDju6p8N6yeAH3fcy01H2BIz1OKs5SeKWXBlQsY+fTIJk38yeQyRj83muLEYjgDCrOYH5rVt3jLzZD5A1x6DjSuP4sWYSLxV2WX+Ptb/TiGVqXUKMbAuqnIo/tBTBD9Js8thz2PCeWnmL7WoqtaeR3jr8zEkUVHSOzZdF516jA1Y14YQ23bWjjgn5yyJ1x8sfiuTvTbUtsCI5c7e7a7QGKCnF0VV2JUljJccyk6nUieERovfi5wRERAdMEQumggrq4GpfOMziR0TyAqPfiFVH1v60vayDR08eLmkaGRbL9lOzNmQA11c1jBZth8g5A07nB30Nv0v4R/pD6bEfkH89nzzR4q8vzb/Xli/NWaatmaLYLdPjP+9jwOe57wqw1ukf+3mCBKfZODbAwS9YlMaDOBxJKpWDBbP5fEnolNrjXtDiqdSiS97AIDneM6M6HtBCd/JWiYzCfYGH/28ivu4CmhqNdDfmUr1pycydjWXzAg1Rapi45Qoa0RDTty3rfnu/KRlbzZ6k2KTl0YOtCuICXS1GpHma2qwiosZnFM6muuEn9h6jAmtJ1AuLphTB5fGH/S883OtrVBpcLRML78FCxKh8NvNqgdEqTvR1mZnQRmdR78HAXb/w1gZdNVGCqQKWRsfXsrx5f5IeEUBOi6tiavVV8KQvdiwYJa6X1FH2ifPwf2ngt4kvoMDweSxomERZAgVci/uHcPDPoqaPdpLJwSRhaY9M4k+t0pgq9SBbw3vxr7sbDBZt1yJcwoFJueCxRSQsNXqU/J07Gg0jHRsfrJ1byV9halWS7M2BqAhnhHNRrnt8KJT0QREgh/giPvuvYNUqjhwPNwep7tWG0pCTFikNVoQGYshZNfWT1oGoyYvqIgSpKUHL1CsAid2hQKAz6BlMlEhpzm+TGj6aoTfVVi/FUbqykzOA5axmojpVmlGKsDn0CTkD4mnQlvTHBgVZw8KeaJiAjbPFVWU0aXuC6MbT0WhVzhIM0qbc4zM21sv4gIS8Mqpr2gZUtHyWW3ib92d0LXp2y/n/kVVo4AY9OYR/9x4g8eX/U4q045yjzrdBCVc5iCXzda1yJNBXvZUXdSn7tydlFYVUi4Opz+Kd7l+RoLuUzOTzN/4vvp3wed9QE2NtX+PM8+f+W55Rz57Qglpz1US9lBWk8GkslZVVRF0ckivws27GW1ff1ID+YfZOXJlXy791tC5H503PxNYo/oDSkXwSVHIHmCWHuuvggOOQc27ZmT+pAwLu96OXf2uxOj2fMYKJPJGPbYMLrM7uJ725sQ9s/kRJGYd9Kj0h0VBOogl8mtthGHCg5Zj+v1NouAylCxNg8G4w8E489isa2jHWITUT3g8hro4IO6UW2JSEoc/8jrqVJCoX4fahJfZQ9IHZzKqGdGEdkykm7x3QkvGYTcEuK4V/OAVhGteGXcK7w+/vXgNtQN6q+Tg+7xVw9SUUSOaiO/HPrFao1iMVswlBuCLmUOENU6ikdLH2X0s6OxWCxWxp/SrEOpBEVVhlhnxg6CTvdB6IWV+APbHrSwsF7MMLwD9P8I4htmTeITzv0BR4Tnr1YLRbrNHFR9zYG8A0SmRTJz/swmH3sVKgXDHhtG+GjxBZebxQDidm9iagL9ytgBwofMTdIP7Bh/IWcoKoIaYw0GudifpUTFQu/XMbf7V3Db2epy4UUYLuaZdjHtuLPfnfQOqSuWjIvhCeMTjHgyiN+pegiNDGXoI0PRDxKTjcKstSX+8v+GXQ+5lhXPXQPHP3Z73aQkuPtu9wQKt5ArxXolbjBU5Xg8NSlRTmSliGWs69KTUnXdvF1bLligzTyHucS+Z+GEkO2R5gh7z/bYjrF0mNLB6qEZTMR2jKXjtI4OlhMGg23dFBaG2G8P/Un4RP8DB/yT+Asw/DFqPL78OL9e+yuFxwv9uoe0uHa1IDtccJjK2koiQyPdesg54cwvkPWrX21wi5bTYcI2iO4bmOt5QERoBMuvXs6wgq+QIbdWz5qNZqqKqjy/uYlQcKSA4sxin8/PyBD/+iPzCSJRmJgIgwbh8wbDFcSGzUJ5aS2zurxAr8Tl1tfCwqBV/u30KHjOZ4P49FHp9L2jL3LFhTvUVFWJgEe7dikO/feXK37htaTXsJgtDh5/gZ6TC+pi7548/lJSxAK1uhqOHRPHnPq/2Sg04xWejIy8IyxMMF3AbmJXhkF0b9CLxK8ke1VRW4FcIeferHuZ8tmURt23sdAN6cHpbpM5J9sNQK9E71K/gU78eWP8SQum8nJbUtXbewIJdxJj+3L38erfr/LzgZ8djmdtzmLJbUvI3Rc8FqIr1Pf4k8ll9JrTiw5TRDW7lfFXNw7J5XJat27tNP/G2xXPuWPIu8Xux2BpT1vCyGKBnJWiksxfGCuF1GPRbv/f6w1nV9CzdCqRoTk+M/7itCIiWF8WLW1kGgPvG4gyNDBiEP4m/ja9vokfLvmB6hLHP6TCUMGpolPUmnzwx0scDVMzhOJA5k9w+mfYcRfUuPB+UoTCRXthyE+QvQQWtoCfIxiUJBKBGg1QtBc2Xy9kQwIB+wnEi7SZXJ/CB9veY2eeGFu1IVo+veRTfrv8N1QKx/LWHZ/s4I3UN8jaEjx/iOQ+yQy8ZyC6ONvkYy/zKSUOJrSdwP479vPtpd8CMGIEjB8PjzwC4+oInfaJv7g4mcv+6xZlx4VXhxeo1TbpHvDg8Zd2BXSwC55UnxOJ42D0VxdYcnQJz61/zsqek6DVwpkuE4l+9t/QhIIJNWU1vBT5EqufFHKt7hJ/f5wQ/kaj00dbJQb/L6FbvPD585b4O7vtLPOmzuPUqlM+XTcYUp8j/zOS+8/db62C9hUNKc6QfA+7JXRDJpO5nX8d8Fs7+HMw7PTT3iIkHPLXQeVZp5ekQiqDAaqqZPww/Qfevehdn4rwRjw5gh7X9vCvLUGC2Wjmjwf/4MDPBwBHjz9P/n4SOsUKf20pSSJB2kcqTeHEaRIDzvizT/xV2sUrndZa8hAhaecN6liRlEi/zuupWi1kxn7Ma1mXMf/gfABeTXiV7yZ+58dfEFyYTLbPxFc2SkRoBA8MfoCrul/l9Nq8/fP4cf+PTgVbjYV9/7VP/IWG2sUTagph9yPB8bQ7/jEs7wdV56xztNwoNmdSgdOap9fwQtgLFBxuWlZ+jakGs0UkGxUmvRgn+38AM0suaO8oe3sQh5ihNgXa3gKRQUy8nfhceDaaTWi1kBH/LluTr2PZ8eWow9V0nt7Zby/aQEHyhpebPCT+So/CqnFQfjK4jelwNwz5Qcghn/kVKs44ndInuQ9D9dcRWzaGoiI4XyU84mUWBUlRkdDmRuSpU/1bP/sLfbpg64eIgb1zXGfevehdxoaJQvCaGlmTFGK5gn1S3jrGnt8Kh15xrZ50/CPYeqtvXuv+wGwUg/2eJ+DXFi6fpYSEBIiosDHg4/R1wb/td8JPuibzF/cLR96ADDG3RkRAWegBfjn1CX+e+LPJm+KqCFIa4+TyuuIRdYxQ2dH7GUxvYgStz3rA/72dWjPDn8Gv46UdiWoTRWwn/yZAT4y/bgndKHyokOOFx11WB7rExB1gCVCVuCYp6BKf9aHTiU4vfS6/Xvcr+77fx9zquSjVzfsV/+HiHwjRhnDbntsAkZhdcGgBXeK6MLXjVKfzM0U8m1a+5dWsCA2Fjz/2vWLXHcK0Bn6YnsBfJ6/ntiWHUUW2QBJeCwuD1rn3IpeDr7YYbSe2vSDkVzxBVIzKiI8Pc/j8UoemEtUmCplcZu1rkqSstFh8dOWjGEwGbut7G+1i2vl9b6NReGSAYCe4g1wukruHDsHeOtlRJ5nP8HYwcbvfbagPmUxM7EVFwucvOhqxaR9jM1aXPCYqDBVYLBbCUwLoW9ZASMGSs2ah4e5P4i9QAbiiuli0O3ZZWJjNV7C0VHy2UnI1RpcLv3SD9ndBtwAxsOtB+h63DF0LR/dDuztAJmNL9hYe/PNBJrebzMwuNh+WolNF7PhoB63HtW5SWRZvEpHPjn6WOb3mWFnTMpmMcBcfeqMYf3IlmKpsMj5V52DNRRA7GMau8e9aFadh+7+gy2MQ1dPPhnhB1Vmiq34nPXIPJVW+Sa1YGX9VjkGU9he3p/3F/jrUu4d9ANMXnD92nhN/nnCQxgZYf3o9k76bRI+EHuy+bbf3C2lbwu6RYnN30R5IGAnhbnwTIkTQlHN/CFmyxLGYq8VmQaNB+C0MWwARAfCuWN4fCrdB0kTBuPUi6xIaFs7S43dYmRsAN/Z27ZGV1DuJAf8egD7B3wx341Df388eIQrxHMPD4a67xDFpvMvMtDFS4uJkhJMLW/4N3Z4CnYcFUG0pLOsN3Z+Bjvd4bV9aGpw9K8Zdn1mF6ddD2tU+yfYEAqnhqYCNySxBqwWDJoIadePXdv6gIq+CxF6JhKWIv99dcmjZcSH9Nq61a3ne/3VMbDsRhUzBxLaedZ8SeyUy9cuptBzqm5dIMKQ+GwopqetP4m9vrliEdo8Xcozu5l8H6NPBWAZdHvd+g6zFUJ0j/AAVoTCzzGUHUKttfqyFhU3PTgoUaitr2fTqJnrO6UmXmV0c1j8nCr378/139H95ceyLpIQ52i+kpcHWrdA982O+egaiogJbtSitr7KzbQVsanW9+b54v2DyxfQXCUBPkCtFUsIHaDRQqt1DZvVC9uZ2ZUbnGbQe17rZPRu3vreVQ/MPMeKzEXxx6hsy4mJJy789IIz2R1Y+QmZJJn/P+du6hgsE7PuvfTd2iDHV5MPBl0CmhKQAyzrXloogvaEYjU7Ej5QmcfNyg9icJfdJpuf1PVHpAmTK5gEV+RVkbcoioUcCka0iqX2ilgNHK5m7Pcx5nNx0nWAYTTkW9Hb5A3sZXqmotMF2B/6i6+PQ/k5A9FOlSdy4pLL5TG1LTpcw//L5WCaKMdAj46/8BORvgDMLoNMDLk4IMAo2wfpLof/HTj5/I9NG8nCHkXy0Rsxx50pFtVyIMQa9TsR4fZp/GwuzSeyHQ2yDgr2M66nVp1DpVaT0824BFAic3XGWpXcsRXOtaITCPvHX9mZoMQVCXcSiOz0EbW4m4JV0h1+D/f8V8r+pMzwyRhMSQGuwJaQSw+vG8rih4t9GFu8HBeM3QYgYSCIiID/iDz7Ivo+SPVcyrs04sjZn8e2Ebxn9/Gj63xlc9Y/3u75PaEQoN24Se+EH/niAn/f+RmTMI3Q3zUGGGaoLL0gmdn00R8L8wqXh/I/CZDL5fG5UehQdp3ZEE+V7JzcabZtwdwyGKE2URz8KJyi1oqoyUDBW+VSNHSiE6mswyWqsm+i00Wn0uqkXZmPwJSG8od+/+tHnNltlx7bsbcxdNZd3t73rdK7FInxvwHMSyB0CMX6EhRZxrLAv56uSyS7rgMzORFxaNJrNzsbb/6uwWETgw2KxcOrUAYf+O+KJEUx+fzIgEquKOnKGfZLoyz1f8vrm1ymuLm7Q/bOyRJ/Wah3ZSa7Qti5/KiX+ghnkkCqpi9x0Y0nq02QxYTAZKM4o5vzR88FrkA/I/WoZqfuXUWQWmVRfErHSGOoL488b0/PUKVi7VvzfXf+Vy51lEqR/w3U1QkpD7YH62UhI35mhyV+IRFSdlJ3k2ShVE0roOLUjDxY8SMepTWtcX5/xd/T3o7zT7h2OrxDSHeHqcHok9rD6PJlMJvbt2+c0/9on/vze9HZ/RsiNSawsbTIM+KJhEqnaFjD6T2HeHWi0msWRbkXsypng87jsTuoz0PCXTXLxBxczt2ouihBHJlxGcQYALSO8TIy1ZZD1mwgkDfgURq8UlX/Jk9yz68y1Qg60w79hykkY/SeV4cJPTVQORkPqpaKworGQTMbLT4DKg6lrHer7o3hCyyEtmfjmRGI7Bm+zs/iWxXw64FPr7wUFsL+ODNWxI5gtZp5c/aRLfyIJUlFTZqaN7R4TYyZn29tw8ktYPx32PydeKNwBK0fCry0Fc1M6Zq7xeXMseV2FhrpZI2UvhSWdBZtXglLTZEk/ED5dAFmljmxNnQ5kJiMV2UXUlPqo4xsARLeJ5vo119P3VqHcIfVje8afwWSgRXgLVAoVF7e/uMna1pSY2HYir014jXFtPCc2w1PC6XldT6Lb+DZ3B0Pq89jSYxz85aDf72sQ4y/PxvgD9/OvA0b/AZflQAsfpJeOfSAYI1JwzsPmRvLGPn9ejD8FlQWUVHuXXN3xyQ4+Hfgp5bnNF4yWoNKr+PepfzPmeTHv2K9/7hl4DyuvWcnNvW92+/60yDRahLdwCiLZK8eEhwc+yJSaKsaoqirYJertnNdZh16FP4eKoHEAodFAiFEk+YqqxEblsm8vY9QzowJ6H39RnlNO7t5csouzeWbDkxxPeh7wo+gEOJB3gKXHlnK2zMZyNZqN1t9TwgMbXLfvvzqdrbs5xJh06XDJ8eD4JXV6AKadhohOKJXis1KY6xh/NWJz1mFKB6Z+MZXItMjA378ecvfkMm/qPKt9hVKuJMQcjgyZmANz14p1A4jCQF1LkRi5gCDtffLy6pEFTn4Fi9tDwdbg3TyyK8QPA7kCtRqUZvFFKqosw2K28HLsyyy4ekHw7u8CxmojJZklVBeJhIzCU+IvfiT0+xBaBdn/ddP1Yq6LHQgDPodE1+sMyQamqAiyCkXiT22MQ6sohN/aYd77jPf5tzEwVsKPatgmPCHNFjO55bkUy09gwUJNDcybOo9Vc1d5uVDgUFtZS9nZMlLVqTzQah7tzz5tS/wpdUKdyhXLPLqXUIWRK8FQDH8Mhn1Pe77Z8U9FvynY4v4cTQsh/dtmDgyd53G/GB8PyUWzCDFGEVU+iIiwur1p25tFfMFX0k5TIrwDaERhcXg4hBgjAdvcq4nWkNI/pUkKT1OHpJLc36bFejj/CKcrjmGRGRk6FCg7BgviRCL2AkfQ+qwHXIDfrv9/0BAtemkCl8nEgttisfDa36/R9+O+fLrzU6/eBi5RuAtKA1StZKyAn7RWL7BgY/pP03k3MpSz0d9bP5veN/ZmyidTmqQyzBsG/nsg/W63JWGlhbsrk/aiIlG5KpM1QF86QNBGJ/Dk6j9ZePhB9KpCWkXstb6mUkGIykS5+gi/HVzh8zXXPrOWb8Z9E4zmNhr2AdWQEPd9RyZzThIVVRWRUy60vH2W1a2HU3XKUOnp3hO3beoKfs+dE/86Mf6K9sKh16A8o0FtsYdUIWjvRceRt+HAi4BN6hOE1MJ3k77jx0t/bPR9G4Oaw5mEnc+gFvFQtSH1PyBnSEk4b4m/3Fy46ip4802gaI8ITFeds75eXQ0vvSQYof36Ccldd5A+2+JikUyUPmNdfEsYt85aKRkMSN+ZXw/fR9WQ1VapmhhtXeKv0jHxF6INQRuj9ds3qLGoH4y0mC0oNUpCNO6jJ64WUPbJdKeAlLEBgaj0qzyzkdwhRA+JY8XiOdBQ6lDrxR/na+IvSiN2kvULFvIP5jNv6jwO/3o4IE2rn8D1Ba6Ck6eKxECZHulFtiNvHaybCqe+gsQxEBoHFZmes/alR2FFPyE3VScLIn1vwsMRm8NAYdh8uNIi5EW9yHyCCG6+MaEPN3adY/0TDuQd4Pt937P9bOPZ3f7CYrZY160HD8I994jxKyYG2reHp9c8zbPrnmXo50PdyrJKRRH5+TaFg9hYCzkxN2Iat1V83mfrgmo5qyBvLVSesfl1JIyCy/IgzVkKzRWkxJ9bmU9ztVC8UNSbLywW4Ze762Gf7tMYSD4u2aXZDse1WojMOQxvvm0temgOuGKFqRQqfpj+A2fuPeM9If8PHBAMqc91z65j+d3LvZ9YD/4m/iwWC3ty9wDQPaG79XhAAxi9X4ORS22L4qK9cPwTl1X09om/GxbdQNwrcXy0w7tHXE1pDaVZpVTmN7+slkwuIzIt0ho0s2f8JYUlMab1GGuS1R+0ayc+wujoxllAuINCAT3q1FLXrxf/OhUkt74eer/he3HxputgkXeTe60WVCaRaC+s9s8uJZgY/exoHjr/EKYY0R8UJh0Khc0+wRf8a9m/mPz9ZNZmrLUeO3b+GLXmWnQhOut8EUhI/Vcutz1Dh2epUEFYG68qBYGAPUtMYvw1JeI6xzHt62mkj7atNx3Gyb1PwJYbxIHerwk1HB/Wc00Je8afg/emTCnWOsGWKTUbwWJGJoNQhfgilVSVI5PLSO6TTHS74BW4ukJM+xjuy76P0GtE9ZLCrEWhcDEu7noItswR60ttEANxpmqxT8nfKPYpbW4AfZrLU3UR1VSoT1BQVMPZYlEtpzbHIpcZRXJLpghuAkGphZazIEbEMktrSkl8LZEHz7TFLDNQVQWT3p7EoPs8BD4CjFbDWnHvmXsZftNw+mlnk1Ay2bbPrDgjVHbc7fssFrCYQakX+8NaLwGg/c8Iu4eVw4UsqyukXyWKm3wYH0NCIDEimjF7Mxl0ZM3/hlJBbal1DxYeDkpTJGCLIcS0j+GaP6+h8ww3yjoBxJRPpjDprUnW3/dniY1kjLIlV0nbwvRrIdoPAtT/R/gn8deMmDdlHq8mOhuWe4K0UdRqQSaz8OCfD/LAnw+w49wObl58MyHPhvDwnw/7l1T8axRsv8OvdriFUgetb4D4oYG5nhdIzKOakNyAbqKDBWviT++8oJDYfsnJ/gVKAwn7CejOfrdyT4eeYDJYj4VEFLCmW0eu/H0SVbW+RZnLzpaRuzfXybfpQoBUcS0WgLY+c/7YeX658hdO/HHCekzaBEnfs0MFwpA3NTyVMHXD2AFS4q+1932uNfEnwSnxV7ARdj0QEF16qcKsuNju4KlvRXUaogLypxk/seSKJehCdPS7sx99bw++r6cn6B64jYPDb8NoEd+zUKV3k2FfGX+LF4tzVq+G2r2vwt7HRcFEHT78UMgdxcSIYLinJK6U+CstFQHV2lrH48GESiV+Moq7U6YZaZVecsf4M9WayNufR8lp7xX0gUR9qc8Ol3Tg9r2302p4KyoMFdzx+x28tOElq/eGO7hM/JmqYWGyKFBx9/7CXbD3Sdfm4MYKqM53Pu4J5tqgGnbr5dmMb/0pCqNvDL7I0EjAOfFnrDFybNkxik4GhrHvb1B53w/7OLfrnNPxjJIMACvD0y2iekLfdyGljoGU8QMsSoOzy9y/J7wjdHtasPoATn7J0JrRXHNFOVdfZYbfWsP6me7f3xAofDNA12rBYAql1qiyjhOf7/qcqxZcxbz98xzOLTlTwo+X/cjeb/e6uFJgMOXTKdy89WYyMmDuXJH0S0sTRQ+rTi/nmXXPAPD48MetMp/1odfbgvX7BHGI2Ji6vhHdG8ZthHEbxO+dHoCLj8LMUujyqO0ichVsucmnpFzXruJ+vdwpP6deBpcchbjBjsdlMsheLDxBjMFNDkgSfVmlWQ5rd60WqsITqOw1iOi2TRcoO/jLQdY+uxZDhRiI3Xn8AcTrgh8Ibk4UVhWyNmOtExvTHsYaI2+2epMlty/x6ZrBkPoc/9p4pnzuv8+yv2N0bkUuBZUFyGVyOsf5GOSpzIK9/4HNNwgf1RIvzMSITiLBLyFzHmy9RQTp6sE+8RerEWzn+t61rjDwnoHcl3Uf8V2b//trMpgoziimpkw8jIYUzDy95mmm/zTdoXggIQEG3vQTq7qlc+9yP70VfYQ0rh4Q9oTOBVYJI32SZLYiNF4EwL0wqATjry7xVyUSf1ve3sKqJ5qOdeIJUsJKadb5LfOZoBNZG3vmvJRs75bQzXcblwZCKoh0CEgbK0U/bkjBnDcYK+DUdyIJghgfJZaY5PGXuy+XX6//lYy1GYG/fz2EJYfR45oexLSP4dj5Y1zxyxW8tlesNUJDgW7/gX4fBL0djYG098nNtc0zYWGIBMVFuyEqiP6m+5+DeSFC5hfQKsSgUFot+sTVK65m5H9GBu/+HnBRu4v44aI/aHf2SddzXtkxKN4nYpm1ZVCZ7eKkAEARCrOroN/7tmMFm6HaWS1j0rI2rO7WloyqfeTUSX3qLHFirLxoL5bOjwSnjfYY8j10EHr9UtwVwKQoo7oael7fs9lsfaS9kXW+3HYHLHbDuDv5pfhu5q4S64mxa6HXy55vMHql2CN2e9q7XDUI24gVA0QhqhskJIDSHIbcorIVWOx6GHY/6vY9zYotN4vP1FwrGH8m18XDTY3CQjhbIYLnV1/cSnyW4R0EczJ5QrO27ULFP4m/ZkRCzwRaDfOPPSAlHXR6C7ctuY3XNr0GwHU9rkNdV8Gz7vQ6/yQ9uj0FrV37xjQIAz+36fQbq4Qpe8UZUVFf5cJstRGQFsg1yjzr4ubUqlMsuGoBeQfcy001BaoKq/h8yOdsfdcmqXC23D3jT/J6S01tkua5hPrES0zvLJLRG07PYm3hMw7+j3G6eJTGCCxYOF7oWwX6xR9ezAO5DxAa4VuwsykhMWQ0GotDsqYks4T9P+ynOKPYeqy+H9zBfBG88Dn44QIn63J09pI87pCa6hgIcKoSSp0B4/6G6D40FvasNCuGzYcJm62/zuwyk8ntJ6NWqun/r/70/1dwdb29wWAAZDL6hE9icrvJ1uSGJ0jP1FPiz2CAv+rsDc1mUGZ/J6omE0ZY3yu9/uADFsIVZzyyhOw/W3tfFHXxaiFNEOAxsj6sbIMyszW4IjH+iqqKMNkFXCoLKvmg2wdsfGVjUNtUH56CkRnFGXyw/QNe2viS1wCIWg2xdcqH0VLcvPykja3pLqCf+xfsfxaqzjoeN5TAT3rY9aBvf4iE/c/Cj6FQEhgmXX3oy1Zw14Cb6RS1Cl8KP5PDkpnTcw7X97je4Xhiz0Qer3k8YJWbriQC3cFQbmDBlQvY/MZmp9esjL8oLwOlNkUwZvV1lRQh4aBvC3FD3L9HroBuTwq5HYDKLORF25g1+TStW5ZDi0uFhEszQKOBh1du5P3tH1rnqqQw4VtxrtwxQWo2mjmy6AgFR4Ir3wpC1s1ohA4d4JVXxCZWYtnc0vsWru3hWdJWkvusroZ+yYvpU34xIbV16zVNgq1yQiYTkjmS7GbeBlF8Yq6Bol1QdtRrW8PC4Isv4N8NEaEYMg+m54uK5yBCWg/WmGocii+0WqgOi6Og13iSejWdd/bBnw+y5sk1VqZ3/cTfF7u+4Nj5C8vTKFi4/tfrGfnVSH49/KvbcxQqBWHJYWhjffueSIGeqiqxpggEUgen0naC/4E3fxN/WaVZxGpjaRvd1idVBUAEU/c/A6VHhOy12TUbGBAFMrXljoUy6VcLBqDGed8kzevnz9uS0HmV3vd+csWFE/7IP5TPW+lvseUtISUmFT4Z5CU8ufpJvtr9lddi3h8P/MiCQwvYn7ff4bg58ihZFRkUVQfHfqN+QUWjfcR6vQJjVnllUGm1tuCjJDd2cP5Bdny0o5ENaBzO/H2Gg78cpLwuyaFoQOLP+j22T/zliMRfz4SeAWmnJ0iJPwfGX85f8GuqTXI7kDBVw6arhZwejow/KfFXmV/Jnq/2kH/Az6K7RiK7LJt5++exPu83oG6cTBwjCoZAMIv2PQP5m5q0Xd4gJf5KS4W6Ari3Bwo4wjtCqyusfnBapfi3tNoHT40goexsGXu/20toXigD48YRWdnX9Zw3fCFM2AI1BfBLDOyZG7xGKUJtTOiKTEHC2HiFU5FoiwjB8C2TZTEgbBaDDq+hX81jwWuXFyjlSjR1MppGeZl1fdiUKDhSwJ5v9rBj/w42Fv1CqWavLT7Wcrrw8nMVB9e1gpRLhF/dvqdFMssb4y+8vdgjdnkEVG4qtLfeZh2/sJhFkYSHOJC9DYk1lpe1EM7+7rktzYUWU6HzI2CuJSLCTuqzbl1hsVhY9fgqdn2+y8NFGo+K/AqW/mspx5eL+PPPi0uoVYhA2uyJ/yiP+IIgiD/8/w25H3oOo58d7ff1pQC1IqyAjWc2IkPGZ1M+44ZeN/DmxDdZcGgBw1oO8++i/lTj+QOLBdZNEQuj9nfCjn/D4HBIuzxgt5AWyPaMv+LMYvZ9v49uV3UjvkvzVXMayg0UnSyiIs9W1isx/qTgnT0a4+8XKMiOvc+o9Ch+OfgAG8/MRJ4Go+xGifAwGWHVnSnSb2J1xuoGyc9cSJAYf1qtjA4dOlj7b+uxrZlbPRfs1l/12WFS4q9TbKcG3dti8S/xp1CI844ckdpc74TQOPETAEgefw6JP92FPanWHDhOaGk4d7b/hokTfXtP/WSuK6xf7/j6kvzPueRSvagIRPg0TunwJuaQeLpERcCii6H/J9D2JpfXs5dRlRJ/4eGIKrGDL0LL2SL4HSTo9dBBt5D0LTMgZB60nEm0RkTPLFgori62JgI10RpG/GcELQYGXlrIFXJycvj0k09Y/MP3VFWXctXBcGZefyXDEoajqdXQ745+ZJaIqv9WkbaiGblc7tB/7XH//eIZWQsq1LHQ5y0hAxHiZifc/i4hzRnW3vG4KgLa3OR/IkjfFhLHB00mSdnqIl54/2f25o6mutq7/2eiPpHPpn7mdDzQHkBSANMX5oI8RM6sBbPQJzo/E8njzyPjrzofzAaR/JPQ+lrx4w86PSgM2qXPYqDz59RUkMlEsqW6WsxVERG2JJG9/w9AZFokTxifCKpZ+NZ3txLdNpq8PJFg6NZNtM9sMbMuU1S3zuk1x+t1WraEnTvF/zvHbURXtZnWbVrb+m/uGuHVmDReMH9kCjjxidVjhOTJMGm3aw8PF/D4kRx4QTBFkyc5vxag+dQb1Eo18bp48iryyC7NtnpwSv04kF5wvuCi9y5i6CNDUarF4s8+OZRdms3Ni2/GgoWj/zpKm+g2Hq70v49u8d1YfHSxU0LFHjKZjBs3+V48ab92q6wMbkDWYrGw4NACeiX1onWUs7SEv4m/vsl9yXsgzxqQB8/zLwAxA+Ci/WL+89anjBXwcxi0nmMbeyM6ix9Xl7Zj/CXrxLV9YfyBKBLN3ZvLwHsG+nR+sKCN0TLogUG0GCTWWdIzOVtzlGfXPUuSPonrel7n8Rqd4jpxqOAQhwoOMaGtrdL9RJFQLWkbHRw2RkICJCXZLAgcvsu15bCkA7S5Gbo/FdD7umL8zfx5JjJ58OY/X7D5jc0cnH+QljvFfilQib/dubsB6JEYeKZW/f7rMvGnT4MO90Bkl4DfH1U0DJ1v7eNaLaRkXM0r113O2BFisGw5rCWPlj1KiNbPD7MB2Pf9PlY+vJJLv72UiiQRv1HLxGTsNE7W5MO+/4g1SlzzFIi5gk4nnl95uS3OoNMh9pklh0Rxvo/rJ7/Rcrr4qYPEECurY8FufW8rtZW1DHnQQ0FegJGzJ4eFVy/kkk8vQTdUjBtu5zylrk697MbgFf1VFwiv7/AOoIoUCaluzwj/uXoL1vaxbdl+bisl2t1U5U8jpjyBVikINuLpn5HHj/I8/wYCp3+G079A/w9BFUmYOowqYxVGRTnV1bBoziLObjvL7ftuD14b7HDqr1MsvXMpvA4flD5Fy7hbUak+FC+2vt79GxNG2dQEWk6HqizBAuxwt+uNQnW+YPmpIsXvxkowFIoCJgm1pUIZxFwDra+D5IlwqWemqMvE3+RD4hoXItJsfpeC8RcJ2Bh/MpmMLW9tIXVwKr3muJNXaTzKssvY9t429Il62k5sy4kCEQ8KU8SgV+tEYHXNJEi+KDh+tAFGUPusu3s2+R3/QYOwLnMdn+38jLIykYmI18Wx7/Z9HPnXEW7oJbTGI0MjmdNrDu1i3JuKNgmOfwxrpwimWPJkEVBpOQuGLRSBnABCYvwZQmyMv+5XdWdu9VzaXdS8n0NEywjuP3e/g9n4uTKxO3LF+JP8bpoz8cfkA3xy2Ga6XD9gGxYGKeevAeD1Ta+79fGxR2lWKQd+OkDJmaaVC/QFNsYfqOr9sUq1EmWoLetZP0kkSX02lPFXWCiSiHK578/cXu7TKbhvrHCQZW0MpMSfg8efoUT4YdX5iK44voIvd3/J2bKzrHtuHZ8P/RyLOXhyhp5gNpoxf/0dycfX+RzAAtszLS11f86yOpXAYXX1FF+uu56a2EusslNZZyxc3uUZprd/CiK7Q7s7RcWjG9hLfUr3jYhAyNpN3NEwDzk/oNdDfkVLCrVTQS0CDCqFinC12O3bM06UaiUjnxrZJBIeb7z2Gq1atGDD88/zUslhfqo5y0MnD7P++ecZd+tonnngaWQyGZnFdYm/CMfPqX7/ldC1K46J4NB4sSD0tElXqEUywBXTZ8AnwsDbH7S+FkYuBnVw5PpCwhLZem4G5YZon33+3OHU6lNkrneWVGsI/AkqK9VKOl3aidRBjpT3CkMF+ZUikOsx8Xf0Hfi1hdicNgYKtXfD1SbE0FYLuajt+zbGn76O8VfmyPiTyWRBTfpZzBaW3b2MXZ/tslaQx9XF8A/mH6SwqhBtiJbeSb29XquVXdf9eu+LmCcdQhVhV/2y6yE48obYwNUUiKDawZdAkwQjfhdFKIEIWtWWwp7HhIy1OxTtDg7LoR6WXbWMk3efpEu8LbCq1QIWC4lrfmTlo38FvQ3W+8ZoSeyZaP3dnrm749wOTBYTXeO7/p9P+gF0je8KwL68fQG7ZkiIbW0dKLnPF8Je4JcrnMe+ZceXMePnGfT6qJdL1pi/iT8QY420XpDgbv4FxDwa2cW3RLqlVviz1JfetVhcrm/tE3+uEiaesOOjHay4b4VVYrO5EN4inPGvjKf1GJGYlQpmcqpFxN5VwrY+pALEQ/mHHI5LyixtooLXV+1Zfw6MP2MZaFP9Y0yXHBJMjOIDHk/TakFVl/grrRELaX2CHl1c8xomDbxvINPnTaeyVlRrKEw6v/oWuGauSoy/HgnBkWi077+S1YO0DxS/dIM+bwREUcYJMpkIwkeI77BGAwpLKLJavVXVQxGiQKVXNUliN0QbQlhyGCHaECpqxQCtwi7xt6wPrK4rFArvDJN2WWUQLyRIrL+sOpXqsDCEbPLOexyUnIKNNqoh9Dr5HVe3eAqAPV/uYdu725rs/gBJvZOYvXA2hR0L+f7wJxRrdzj2y9oy2Hgl5K62Hev/gf+Fg74iZyX8MVD8K6Hzgy7718hWIwE4H7bK+ix1OgSDfue9ULDB8/wbCBTvh9M/WqVPw1RioJcYfyq9CnV4kH0j7dDuonZcvuhyKlPFOKs06/y3SGoxFcLaif5Qc971OQdfgvlRUHZCxMB+beFsMRASDjOKoacXyVA7JNqW17YCC7ki6OoigUBEhI1tX22sptooKJ+37rqVy767LKj3ju8az/3n7qffncK/L6daxCri1XVB1OpcIZnryqblHwD/JP4CDrMfui3L71nOnm/2eL+mxczl8y/npsU38UvmJ4AYKGQyWeOTfKYaWN5fSBUECsUHhKdOTQF0+Df0eRM0iZA6LeCBzwS9JPVpY/wpVAprpfKFBIvFYvP4q5f4s1hsUp/NmvgL0VOtFJvMJP0xZkaPhBM2xkNYGKQWXE+4Ip7Mkkx+PPCj10ue2XSG+bPnc+bvM8FqdYMhVdJrNBb27dtn7b8FRwrI2pyFyWDTzavv8Sd5aXSKaxjjT6rCa9HCdy+Ptnb5FyfG37Y74Ee1SAA2Ei4Zf/ufEdW7VeLvfvSvR7lh0Q3sydlDWXYZhccLMVY33WaiPgyTppCf2svBq9Eb6j/T+jh5UjAslUq45RaxkTIYoHbpEPhTZAKzsmXcsuQ4K2uXgC4V+r3r0ePUldRnRASgjhEeV4rgLuD1ejhR1Icd2gVWuVKA3y7/jW03b6NlRNMPQG+89hovzJ3LOpOJ5dXVXAWMAa4CVlRXsx4zW0wreeP1122MP7vEn9lsdui/XlFdAFtvh4zvnV8z10LhjuB4mQQRGg2Eq/OpLPdejAFQVlPGmZIz1Bgdg54LrlzAX48EJsngD+PPHYxmI3OHzWVOzznO8r0lh4T3QPE+SBgNnR+GpHENv5mE3NWCcfbXWDj8VuOv1wiMT3+fa3s8ap2rpLVDfalPgMx1mZzeeDpobZmzYQ7DnxxuTfxJgSWJ7Tc4dbBbbz97tGoFcpmYK2JiQKaNd+y/vV6BQd8I5rQ2RQQGh/8KFx+GlItsFzrzq/DibCgUOph8ALo+7v6cnffDpus8yxMGAL2TepMelY5Sblu7arWATIa2KIu8A8GXcAWxTj1/9Dy1lba/V5JyUqvhdIn4fgWLQXShoWOsKOLxJm2/74d9bHhxg8/XlQq3ApH4M5vMpI1KI66Lc2JtwSFRyFdaU8qajDVOrzck8ed0f2/zr6FYzLm15XDwFTG2uoMqSviztLFjUJoMMD8SNjuz3lwl/qRCEW8YNncYt2y/pUlYRP5AmjfPVgu2ni8Jdmvir8Ax8bf97Hafr9FQ2Cf+HFhimiRhD9D5Id8vVnYM9j0l1mAeoNGAvrojk/eUcfY+MRdW5FeQfzC/2QoQAVIHpdJ1dldrwkhpbkTizy6Bveb6Nfw046egMP7q99/LLoMZM2Ds2IDfyktDxJyjqavpsS9iM9WaOLv9LIXHC4PejI7TOnLTlptI6ZdChUE8xxD7xJ8+zaaAo9SIIsGQcJfXak5IrCKp3kOvR6x1xq63KtYEBdUFIh6R8QMASZpWpBReSdsQsWee+fNMbthwQ/Du7wL6BD0dp3Xkt9LfeGb3LeRELnTsl+e3QOYPrv1ni3YHvkFRPUSiKKoeO6qmEI68DblrrYdGpwt1uCLdZpbkfkBG3AcYNGdEvGDsOsxJl/i3/20IOj0Is6utjF+JxWlUlFFVBZPensScjX4WxDYCkWmRdJjSgQqt6J8Ks9a2z1w1HvY96/qNNedhxz22AtH2dwmfv5Aw1+fHDhIMQl0rofiTOt11fEcV4VjYlLUIMt3HR6W+qVTW7Y8tFuEJWHbC7XuaFQVbRLFDzkr0eggxh9H7xE8smPYnCpmQ5Y5uG+2z3H1DIVfK0Sfq0UTVSc0alERU9KFdWE9xgiYRZhRBz5eC2o5AIah91g3+Sfw1E4zVRra8tYUTy713crlMzh39hMTRl1kPUxC2BpU+QGWiFqPIkHvQIvYbvV6Gy2vEor9+BXrVOaHnHiBIC2R7xl9tZS1nNp2hOLM4YPdpCIozitk/bz+l2TY60fZbtrPi6hWkhKU4nFtUJJIPMhmkpNS/UhOh6hwU7yMyTDwfo1lNrGKvoLrXISwMFBYNY7T3APDihhcxWzwPXKmDUpnx4wxSBzejeaEb2DP+7LHp9U18NugzakptQfH6SaI9t+3h7H1n6Zfcr0H3PiVsq3yS+ZTgkfEXOxhaXQmKxk+8LhN/CWOg86OgEB+WTiUaUFFbweT3J/NAzgPNFkCRK+VUdexFQVIYI1Yp0D2v8+qJAm48/uy+7yvrivEGDRKfyWVDVvHRxe0oKguD9GvAYiE7G8oN0eiT68lCuoFbqc/KbBEYCzLc+RqOSBtB3+S+hCodDdkWXruQn2cFj/GSk5PDIw8/zOKaGga4OWcAsKTWwCMPPcThDOGTZy/16Rd2PwIr+sPxD4VvSX2UHoblfeHAf12//9yfsHoiFPqhZb/zATj8ZoOa6yumtn+d7y6Lx1K426fzu37QlZZvtmRv7l6H4xPenMDwJ4YHpE3+BJW3vreVV+JeIXuro0RKRGgE/x39X5fSpOSshBOfQnUeJIyEni8GJgCz7U7htVG818rsbS4szn6dJ1avdPL4K60ptQalJCy4agF/3PdHUNohk8tIHZxKQrcEJ8afUq6kfUx7hrf07XuTmgp39L2dNyf2IiXehb9Gwgjh62Uvl+wquJbxnfBFrWlgMFCuqJMR9FC802UuDFvg/vUgQq0WigB7x97HxC9nN8k9q85X8W6Hd1l+z3LrMXuPPynx1zL8wpb+DhQktlVeRR7lBvfz896v97Lu2XU+XzeQiT+5Qs4Vv13B8Med+599scSHOz50et0fH1aj2UiHdzsw5YcplFT7oeBx6BVYECekzXY/5DEg5hIKFSSOE6yjepASf8XFEKUWA1JeRZ5P67+E7gkk9U5qdr+/0xtO8/3k761Me+mZZFXUMf4ifWD81RUgrj+93nrs18O/Wqvxg8n469ZNjFMQAI+/+OEwaQ+kXurxNK0WZCiQ1eoxGkWc4a/H/uL9Lu877NuaC9d0v4YP+2+gTc4jPvUte7hK/LWPac/MLjN999VsBJKS4Lrr6jH+Ts+HtVODx6TYdD38qAGzEa0WqkKyeP3EHG5ZfAsAtRW1fNLvEza85HtxRSAgJXClxF9oKDDsF+j/ke2k2nIoP9Wk7fIF9nKCUNc39a1F4sKLP3qjIJMJL+Y69pxUpCytYSPTIolIdeOVFmRYmbgWjeO+JHEsXHoO0q5yfMOJL2BZLzj5dWAbEtFJMPzC6o3LxnJhi5T1q/VQ66jWRMpaYpHXslJ9B/tb3UGZ6oiQn4wf5mhvECyE6IUaSh3C1HWMP0WZdb5qDlifp8T4MxmE/3e5m3HKbIQjbwkm39LuotAkfrjD3+aAltNh4BcgFeQN+ATa1ZMzrcgUxaf2igR7n4A97gsK09JEv2jdui5MbjbAyhFCNvhChLFCJCYrs5DLITxMTnLRTHpFjLUWe1aer6TwRHALMyryKjh/9DzGGlE42qJqEsMObefJHp/bTpLJgidj/H8A/yT+mgkKtYL7z93P+Nd9k758eMjDdE/oTqW5mM0dRvFEUTIbTgdgAaTUwbRM6PN6468lQaF2vag49hEsTIb8jQG7VUpYCoPjJhJfPJmycrHRKzpZxOeDP2f3F7sDdp+GIHN9Jr9c8QvndohqRJlMRtf4roxvM96pKl7y90tKahw7olE49S0s7U56pAho51e2ZD6FwtC2DtKmro/ldsJUYeRV5HGqyPOCN7xFOF1mdWm2hZ4n2Dz+HI93vbwrY18eS2iUbcdW3+NPJpORFJaEWtmwUmkp8dda2tOba4WRdMEWt+9p2RKrX4QT46/drTDku4DI1Nknp6zxk5SLoOfzVp8yXUhd4s8QoCKERsJgAJO8CgsWzBazT7J30ve5qgqM1RXw1xhY1tO6gJPkd/vV5Xa7dDZSY9Lwzc5HMHZ5DmQyLEV7SNCdpEVK3Qd16HVYMcAtQ8RV4i8iAvhjMPw10v8/3E+EhYFCVkunyn8LWWYvKM8pp+xs8EzZP/3kE0aFhLhN+kkYAIxUqdi1RJiD1Zf69BkWi1jEX3oWBnzq/HpIJHR7SmjEu4KxrG4B7Aez6vjHcHZpQ1rrM84Z+vHXyeuoMvgWZZICwpJGv4Sus7sGTNrVn8SfJlpDXOc4NDF+LNg73CVYYPEjvJ/rD3q9CiMWw2W50MNNAriJUEw3jhX2swZNwlRh1rG3Putv7MtjA5a0rQ+z0YzJYKK62jYHSom/W/rcwpF/HWHu8Lk+XSs0FGSqcKpr9YRFNyJS3OslmHZasIQagvIMIfHkCYmjhWeHPLhFLTvP7eTxVY/z2U5bglsms7H+AiUJ6RUyGPLIEAepfJeJv2ZghjcHIkIjiNGI7NLJopNuz5v0ziRu2XGLz9cNZOLPE14d/yo7bhHsqQWHFpBbnuvwuj9j9KpTqzh6/igbz2y0Bv58Qkx/aHurqJofvxl6v+r+3JyVsPlGKDnseHzYfOjymNPpkZHC/9piAXVtArO6zOLGXjdispicznUFY42R4oxi3/+WIKDsXBmnVp+isqASi8XG+DtdLgqDfZH67BDTwfp/yY/y54OiYCs5LNnq4xwM6HTQpU6hOM6O9EDOSsHw9KcwQxUJUd3dMzDqYF+sKc2N7S9uz/AnhyNXNl9o69uJ3/Jhzw9JCkuio3YIYdWd/Gb8tYtux2vjX+OFMS8Ep5ENQfkJOLdMeFwFA9G9ocWlYKpCoxF7ubWlXzBv/zxASAmOeXEMXWYGwWOwHk6tPsWGFzdQVVhlTZwrzOIL5/JZrr0YlgdBArWRkBQZJOj1iOdn9m1sbDBU0WLt3O8DABShFeRE/srqfFHwUV1STXFGsU/FGYHCri928WLEi7Bb/K4wa5yfpSbR5uUmIWUypF7mv5LI0feFn6K/0CTBmDXC+qMOMpmMzqGjHU6L08aJJJa5iVSWTAYRmyoRjPJpHaZxXac70NakUVUFGWsz2PjyRgzlgbGb8YY/H/6TF8JfoOacWMBYGX8KFUzPF6oBrhAaB9POCOZrbbmIVZtN/hNS7PvQsQ9FErHCLhba910Y/J3bt+v18Nln8IL9EN/nbefE84WChFEwu8Lqn2hvWSNh4TUL+aDbB0Ftxq4vdvFuh3fJ3SvWsdLexLoeyFrsuqD7H1jxT+KvmSCTydAn6tEneHZ1lybGEEUIH07+CCx1wWy5OWha70FD7EBofYNYFAQIcbo4Ph+zjB6Zn1FRLj6b8BbhjHt1XJP4UnlC2og0Zv48k+R+zn5+9XFByHzGDYWuT1CjtjGX6i+MpESJsTySFVevIPOezP9pnxd3jL/0UekMeXCIQyWwZHjuyQ/OH0hSn9bEX815yF0jdN/dBCOVSuhcZymYlBSYdriCVOlpMrmXwbRn/BUcKWDfD/uoKmweecSc3TlEff0W8ZlCpqM+a80ddDpbnrS8zCS831Ivg3Mr4NhH5NbFyKTKyZb9x/PE33vZlDGBQ4fAaIRLU//N25N60qJF3SamOgcqswST2gWkBVNeHqxaZXcs/VrB2Awy9HowWZR0kL8P2b9bj6/NWMsrG19hfeZ6h/Ov+eMa5mwInoTHou+/55pq3xbd11ZVUbJJJDsazPjr9RJcctQ1Ix2EXGu3/0CcG+P5lCkwq1z4A/iKS8963AQEAmeNw3hzy5cUmp1ZEa7gLvEXSPgTVO52RTeuX3s90W0c1wcnCk9wqugURncb3PAOtmrMQCHlIpHwkcma3XNBq7WgUlRRWSGY9TKZjE+nfMriKxZbPY4ldLuiG+0v9o157C/O/H2G/6r/y7rXhDeLTudcfCL3o4p8q/E1HvlrvWOg2F/oW4O2RcOLXTZfB7+2tKtucQOLxYEJHgzsy93Hc+ufc5JP12pBU5rLoZ/2YTYGXxZGG6Nl7Atj6TjN5lNr34//f0v8gS3x4inxF902mtiOsT5fM5CJv9LsUlbcv4JTq10X4fVO6s2AlAEYzUY+3/W5w2v+jNFf7v4SgCu6XuFXX6fFVOj/oQiqxg4Qc687FO6Ek5+LAhsfIJPZPMmqSrX8OONH3r3oXQfJXE/4sPuHfHdRcOdmb+gyswtzK+fS6dJO1NrVi2WWiu+bL3ssnUrHZZ0uIz0y3Zqs+ObSb1hx9QqWX7U8qP6vAPfcAw8+CD172h0886tgePobVDUUQ5Xr9bMEmUzs2w62eIjLF17GkYIjdJzakVFPj0Klb67qWYhMjySmnSgUaKiMbpwujvsG3cesLrMAeH/b+7y44UWOnT8WyKb6h84Pw+UGl6zbgKDD3TDsZwgJQ6sFpVnExsoN5VgsFuRKOUMfHkqb8cGPNxxffpy/Hv2L6mKbf5XcLPaUoaEID0p7L+lWVwgWkBf1o6ZGfcafXg+sGguLgjx3y2SiQFguJABN6iK2t72Uj/KvxmKxsOKeFbyV/laTJYkAdHE6kvslU6mWGGJaW78sOSxkSWtdBHdC4wXD09OcVR/mWth+J6ye4FzAImHHPXUFwvWSsPIQoXhRj8U3Ov5K2mc/bf09Th8n5sl5IZATHJUPBxjLRWzqsCCJ3D/4fl4b8x6RlX2proaji4+y8uGVVJ4PUmFAPUS1jqLFwBZUKCSpTx89/mRysWdoMQWmngRdGvyosv5dDijaI1h42fWKdvc8Ab+l2xh+SROg65OgtetX8cMhtr/Hpuj1dkQPhVoUsiZP8uGPaAbUWz+Eh0NB2F98e+gTThSKAqXOMzsz8J6BQU3opw5KZeijQ4lsFQk4FiUCsPtB2Ha7y/f+A4F/En8Bhlzu20dqKDdQcKTA68T325Hf6PJ+F97d+i5pyoG0yheSn9f3vN6/ikt3qC2Dk1+KzVawEdUDBn4O0Xaa1uUnhc+gsaLBUgkSE0vaQIdGhjL4/sG0GNiikQ1uHCJaRtB5RmfCksRz2py1mefWPccfJ5wnaYnx17yJv0HQ/RlCdDHWQ0mK9XDCFiiwlwkclDoITYjImFksFh776zGWHVvmJP1ZVVjFqwmvsuzfy4L/N/gJKfGn08no1q2bx/5rL3/55uY3mTpvKr8e/rVB962uhnN1ZA2r1KcmUciHDF/ksdr1oYfg9dddSITuuE9sSAIApdIWmLLKfRbuEBKHZ8VzlFgnlbWVHF18lAVXLuD8MTcGyUGGxWzBFKLBqBSLaI2PNH+53PZ3llWHiwBV33fg9M+w7TbKi0XWU6qclMuhRw+I0WQRtW8KZbs/4fejdzDv4LPExNZ9d3q+CJdmi8WlC0RFiTWU0SgYf2q1kEqix7PQ6b4Gfwa+QoyXMj44mwlD5lmP/3LoFx5a+RDLjjdtPy0pLSXR+2kAJACxijh237qbLnG2yl+5XO61/zqh6pyoSDf5ufmUK/2XyQnRO+r/BwH15XS8wV3i76/H/uLVhFcDIpcVCP+oh1c+TOu3W/P+tvcdXzi7QmzIglHxarHAgRchb733c4OM8fGP8MssLbLKDOuxy7tezsXtLw7MGtBHhEaG0vWKrlhiRHJDStgVVhW6T8p6wPDhYj3Rt28D+6+EqtyGe7CkzhA+1N6C4qsnwNJu3hOEjUCLcDFfZJVmORzX6SAucwebHlhAZUHTBFXq4//3xN/dA+7mg8kfeCy2NBlMlOeUO/hCe4K07nBXWOUPSrNK2fz6Zqu6iISiqiJrEOau/ndxVberGNN6jMM5vo7RxdXFLDy8EBD7T3v41X9NBlEY5Q6dHoQZxULa1x45f8HmOS6ll+19/vxFzzk9xbjWhOwTT5Ceh0lWTVapqAj1Vabzl1m/cOLuE/RN7guIQozxbcbTLSFIyRo7xMeLMd3hK9D1CRi3EUIT3L7PJRamwObrvZ6m00F++DKWZy7kTOmF4SF/8QcXM/Pnmfx25Dd+OvMmpZq9jVr/AHy4/UMe/etRDhe4SSL4CpNBBK3PrnA47Ff/DXICGURCV2kSaxsLFqqa2G974D0DuXn7zYSlhPHQkIcoeriIi5SCpRyqMgoPytN29gftboUezwVXPrMBqJ/40+mAxPHQcmbwb16eAfmbAIjWiSCdGSMGk4E2E9sw6IFByOTB/y5JaH9xe65deS1FLYoAkNsz/jK+gb+vhOID7i9QmSU84b0UJAjIhUQ8wPmtrk8xVYtkWl1y1AFmkxNLemTqONLy/mX9PTEsBnTp0HI2cl1qw9fPvkIVJTwJW11hPSQlW0wm6H1bf27aehP6RM9klkCh7619ueaPayhViWStwqwV7anKFbLEniwayjOg9Ij4vyYZUqaC3gVRpDJLJP9M9cYfpQ60qVBdt95KGAndn3aWlzTX+h9fuJBxdrlQO0IUqx9Lep5XDt/ClmyhUtbrhl6MeX5MUIuMWg1vxZjnx6CLFwvoX2L6sqprO45X1OUw+n8Cvd8M2v0DjaD2WXf3bPI7/gMAMtdl8l7H9zjwk4eJBvjp4E8czD/IyaKTHDkCXU6/yfSqZbw24ZXANKQqBzbfIAbKpkZtOayaIKpIFibDVt9lcuyh14tNUmllDc3gk+kzVp1axeOrH7dKV9jjgkj81cHenL2t6S3YcpNVtlBK/NUPVnyy8xNe2PACF31/Ee3fac+enD3W11R6FTHtYwhvceGZX0tSnxoNGAy2CfqTfp+w+NbFDufaJ/7WZKzhtyO/caakYZvN48dFDDE62sYAEzfpKiqRQCz+XAQjwsOhXTunw3B6nmCqBQjS3yvJUWKsgPwNUHUWwOo1UWGooP0l7Zk5fybRbYMnJ+QJSb2TODX+Fs6liaC0r4w/sFWLr1hWY/u4O95Lce+V1NSqUCptgSVO/8zkdu9RbdSTaF5OVd5xNpyZxZ7qf9v2w142f3o93HknTJsGTz4J334LHTp4fEtAIfXhc8XJDgtVSc7sfKVj9KzgSAE7PtlBRX5wNMkiwsPJ8fHcXCAqMpIeiT2sRQcS7PuvW5hq4MALYvF6+E1YNQ4qMhzPWTEQtt/l+ToFmyHzJ98abSgWhTUGPzyRGgCNBkalfU2v/ME+3UtK/BVVFzkcD40KJaZ9DMbqxifUpCCmL5WY619Yz7b3tzkdP1UsCoLSI+tVOux7CjZeLnyKA43SQ7DnUVGV28wokvVl1alrqKzxHj3848E/eD3ldav/QSCR0D2B6d9Px5ImnoOU+Lt3xb1EvRTlxCTyiA2XMzzhbb77Drp3F4d86r+usHI4bJjVsPd2uAu6P+X9vMSxkDo9oP7U9ZESLiq8s8scPS61WihI7UHXZ2aiCgs+k2X7h9v57qLvHOSd7atqP5/6OR9d/BHtY4LDLL0QcXX3q7mt722kR7k3ZN7y9hZeS3qNszvO+nTNQDL+knolcU/mPfS6sZfD8UGfDSLtrTR2ndvFVd2v4tvLvqV/imMVuq+Jvx/3/0i1sZoucV3ok+Qsa+ex/27/N2y/W/x/8w3wa6r7viSTgSrCWVq37Bic/MIWsLODfeLPZDaRX5Hv0Y/RHkMfHsqIJ0YEnRHnCeePnefY0mNUl1RbZT5D5CpO3H2CP67+w+r55gua8+9wgiYB4ga7Dm57Qvs7IeVir6fp9RBiEnuOwqpCDi04xFejviJvf56XdwYfX+7+ks+y7qVQv9Fvjz+APTl7+OnAT+w6t4tDBUJer0diI1WeavKEd7UL2Xmv82/2UjjvvD4LGAp3wrZ/QdFutFoRyJcg9eWfpv8UVL9xCWFJYST3SUapVqJSqMRauUbEL1RqOVy0XyiHXOCwl/oMDRUFvXR/Cvq8Gfybb78T/hoFFgtROltQqcxQRtfZXRn/ynhUuqZn5to84ewSf+3uFLYPMR4YWjkr4e+roMAHqyK5Qkh1TtoNLWe4Pqf/hzDZTfx31VhY4ri+io4Gg7IAAKUxnAi9WsiPDp0HEV0avn72FTKZ8CRMFJKjBpOBUmM+BoXYP6oTIknpl4JSHWD1FS94eMjDjCj9mMiK/mKcLdwBG2Z6lntcMwmWdISs30RB7vAF0MrFHiJlMswoghbTHI93fhjGbxTS5e5w+heYp4bsxe7PsUfxPiEXeuob385vDmy83EouCA+HEFMkEFzVIG8oDTlGZehxovR1C+r4YUKx5x+4xT+JvwDD7GPmKTI9kqGPDSWxl3uuQ1VtFb8d+Q2AWV1mceQIyFEyse1EvwLbHqFJghFLIP2awFzPG7KXwIJEschThEL61dB6DrS/S0jsNQBXLrmYZX005EQtsCZyPh/6OYtv8XHADRKW3rWUl2Nftsofni0TAYEkvaNkgMVik/pMTW3SJjo2Ylkv2P2oQ+IvK/whGP2nSBAX7bUmDerLXY5tPZZ7BtxDhDqCE0UnHIKACpWCG9bfwJAH3UjnNSNsUp9mjhw5gtlsdlv5KyXCSkvhYL6QlOwc19nj9fPzcZmM3lOXF+3Sxe7gwZdg9UVCbmLXw/BjKBiKnN/sDlNPw6jAST7YJzoBIV0wqxza3AjYefzVVhDbIZbO0zujjWk+WTzJ4w9wSgp5wuzZ4t+BleMp+76dSP5F9yLLOAajWUVcnF2R67EP6Gh4nIraSC5fWMn60hcBSLFX5bCYRSFFfXkIO0yYADfeKLwDQ0OB3Y/A2imiGCLIkPq3wpDlwPSO0dYl/qocE38n/zzJkluWWDXVA42pV17JNz5GRr7WaJh2pbMcqtls678eUXUW9jwGWYvEYr7vu47+YKYaYbJt8sJ22/ukCGD6Iu2Tt174f2Qt9H5uI6DRQLi6AJ35iKPXgBtEqiMB50X7kAeHcMP6G6wVdY2BtBf1peJ969tb2fvtXqfjGcUZAKRFpjm+MOgrGPyNWEcEGhGdYeh8GBW4QoqG4lzITN7Y/DXnK22DzNHzR/l277eszVjrcK46TE1Ey4iAJG3dIb9O8VJK/K3PXE+5oZyUsBT3b7JHzXlRoFK0xzqu+tx/XaHjfdDx3qCy8ej8EPR7z7VhfHU+HP/Ec4WxD5A+v9KaUkprbAssrRYqI1MIG9C5SQJlxRnFnFp1CqVGBHAsFltyKDQUxrcZzy19biEi9MLzbG5OJPdLpt+d/dBE+7b2qK9U0hgoVAoiWkagibLd+2zZWY6cP8KZkjPOY6cdfB2jv9zzJSDYfvWTS177b94a8QNCyqrDPW49kCnc5Zp5kXaVCMIljXd6SUr8FRbClHlTiH81nh/3/+h03oWKAz8e4PvJ31N8qtj2PFRy0qPSGddm3IWVzPMHFWfcP2dP6PWySP55gU4HIUZb4q+yoJLcvblUlwSvQMMb/n71b3Z+tpOKWtGxlWZdgxh/t/9+O7Pnz6b3x70xmo1EhkaSGt7IAEFoAkzYKlSXqm3JUZ/m3223w6brGnd/T6jIhGPvQfEBNBqQIUdpEeOZlKypKavBUBZ8Bk1NWQ01ZTUOsQDrHKiRQ2QXx6B/0V5YPUlI215A0GhshZ76piFi2dDmRtGPLSbCdEqrVKqvBRmBxvEVx/nrsb8wFoq1sUPiT5ss2uupQCFhDAz+AWIHe79ZdYHYl0T1aJhVQMpkaDnbYU0bFQUVoUcBMCpLrYVD0Mj1cwPx7NpnSX4znuOpTwBQXWWhtrK2SeToAXZ9votVj69iQtsJtCm+GV1NG/E8o3qK55Qw0v2b29XNLUff9X4jmcz5eyHNxxazmOOWdHJW2wprI4oF1T7Kv5tqxI+P3sTNgv4fCW9E6hJ/RhE7KaoSccpDCw/xzfhvyD8UPFuE3+/4nZ9niuKP8poKjAqxV2oZlSTICReY3LI3NGWfldC0qfl/YEVcpzjGPDfG4zkrTqyg3FBOy4iWDEgZwE91hY4BZYeE6MUk01RQ6oT3VWiikEzr9p9GXzJMLVY0NcpcysrqGIA1Jp8ld4KFyLRI4rvGW/0GpMRfcpij519pqZDOBGjRXOqkxgoR8DZWOCwQDWH9IbYKFiZCeGf0/YR0Q3m5WJNI81/rqNa8MfENusR34ebFN3P4fCMlSZoIUqLYPu8gk8m4edvNTueGhYm/t1ZWxomiOk1rD4m/HTvgqadg3Di4+27H16TEn4MfRnkG5K8HhQ7C2kHyRc4SA54gV4I8cKt7KfFX5Cb3eFX3q+iX0s9r8rMpkL39LFGHM8nrIJhO/hRGjBghvgfH/u5HdmkJlhUwcSLk5liQy0wkJNhNk/0/RladT/TvUFioJOb0vbwzaSXbItYAEi1QJliyUT19rzyqzIaSg2J8DDKk/n1pizthxVLh2yGT2Rh/9RJ/7S9uT1SbKJJ6BcdU8qabb+bZp59mCzDAw3lbgNWGGpLSjvPzgZ+Z2aUBcjWaZBH4UEVBWFshb2wPhRom+SB73eUxwViwWMBbTC68PXT/L0T387+9fkCjgd+P/YuQbvdyXZT3QGGURizaLxSPv1t23oKxyjFhVVpTSmGVkLxxCl6Htxc/wULL6cG7th+Q/GftJVx/PfwrD698mKu7X82ItBHW4yOeHMGIJ0cQDBxacIgTf5wgN3EEEEZ8PJTVlFkZmQNaeOq9dlDHwPQ8tz62fqPdrQ1737EPhVzXgE9B757J5RX5G4RaxYDPxKYzYaQYW0D87qP8V5g6jHB1OKU1pWSXZhMeJxgGUoBHWqsEG2NfHMuYF2x7E6kPAw1irvxfQLWxmh1nd5BXkcelnS51eU7aiDTSRqT5fM1AMv6qiqqoLqomLDkMZahYr6w+tRoQ/n7SWG+2mNl+djurT63moSEPIZPJfBqjD+UfYnPWZhQyBVd3v9r/Bl60x+ZllH61+HGHLTeKgrep9YpXPMjf2zP+4lqLioT8St+CT9XF1fw882faTGzD4Pt9COoGAR2mdCAsJYyIVhHk1pH1ffIrupBhMsCiVoLxMtRHdQQ/odOBqlwk/oqqiuhzSx/63OLMRm1KbHptE1FtoqiYU+c9ZWpY4u+NCW9w59I72XFuBwA9E3s2PgEsDxGWKptvgIFfQmsfE3kWi5gn/dmP+oukCXDpOVDHoKnbgoRYdBhlVVQYxGd5zR9NU6C+5JYl7J+3nydqn+CLfV+wOWszRbXTgQmoVSaoPg/KMLtCILMobEi+8JgmCQkiviQlANl6G4R3hI73BPfGdoX8knSrQV5NWU0ZR5ccZfsH2xn78ljiu/jOZm4MTq06xd8v/81T65/ilx1lnKjsK/qlsQosteJ5eupfulTQXe7bzXbcJYp/Lz0rCrnD6kk1Wyxw9B2I7O46QdXpAadDUVFQHSLUINSGRGHtkPkT5P4FXZ/1rV2NxcYroToHxqyy2gyYQ8Q6fvfnO/n70SVc/cfVtBkXfB/OAz8d4PSG04z+72iHwjS0yZDm5Tm1v0MUsktJtsNvQNlxUdxnj+OfiiLQODfrgk3Xw5lfxDhQv1g4qqfwLPUVMX3hEmc1gwsKrWZb/+uK8VeRW0HW5iwr2SUYKD5VbFUjySwUUqsKk5b4iDDYc68owrzkqJNH5j+w4Z/E3wUMie13acdLqa2VcbLOV74pZeECjoRRMDXTvd+RudZZ4sXbJXVCyNwQkkdxMSQl4TJx09QYfP9gh43kuXIxSNVP/GXXKTvFxTXjZi9Eb5Ud0O+zHVarEZNat6dAl0aY3gLIsFhEwKJ+FVmn2E4ATl4E2z/aTk1JDUMeurBYfzbGn/dzFQox2R1lDWaLmbbRbUkKc58MWbVK/Pvnn3DJJTZPvqoqOCoKt+hhr9zS/wPo975YfLa9Sfz4CpNBaMnrWokFagAgSZBapT7NJji3TFQwxQ6kd1Jveif1BiBjTQY/Tf+J8a+Pp+d1PQNyf39wcmUGqQf/5FzSAMamTaBDnAu9dg+YNAm+Pf8qn/8IPQwwsddqRuZN4GDr9yDebiwJa4ssrC1du8LxXUfpFf0TBlMo8Sl2EqcyGQz+XixAfcXgb4RXWRNUdksbwNWnrqbXuCFi8StT2hh/9aQ+I9MiiUyLDFp7EhMTefGll7hk7lwW19S4TP5tAS4JUTHqxhF8deIrVBGqhiX+FGqICUACzlM1YX2Ed4Cucxt/Ty/QasFoVvmcIOib3Jc5PecwJNVxTC48Ucj+eftpP7k9iT19dV90DYm94Mu8Jnnh2kNi+8VoYhz97KoLRJLcRy/P/2UkqnZy94B3yTHdAgwEbGuIc2XnPLwzsDi98TQ7PtpB9e1DAbFekZJ+0Zpoq3SsT5CHgLp5ZKGtqMoRc6Y949cdzCbYfB2Ed3Luy/EjYNhC8X1cPV4EVEMiYHnfOlbxWz43qUV4Cw7mHyS7LJtOcWItpdWCtvgs+6/7juTnRtD/Xx7kqAIE+wCzfeLvaPE+9uTupntC98bLzv0PIbc8l6FfDCVEHkLV3CoU/koXuoC0dg6Ex9/+H/az9M6lXLvqWtJHiYXm6gyR+BudPtp6XmVtJcO/GE6NqYbJ7SfTNb6rT4m/OF0cL455keyybBL1DZwTfP3MOt7nPsFQclAUKNbz/7NP/MV3E4HkvArf5B4VKgXZ27KJ6xJcD15PSOieQEJ3sY+tqWt2RvSn/Gf1GaZ3nk73hO7N1rYGw1wj/FPrezX6ghOfw+mfYMiPQvbVDXQ6CDGJ8VsqEGpuXL/uemRyGe+sfAcAhVnfoMTfgBYD2HbzNpYeW8pPB3/i1j4NLHCxh6FYBLJ7vgyxg7yeboVMJmQFgwml1sqOkvyqFWYdyAusjL+mQqsRrVBqlMiVctZkrOGbvd8w1NyBSCYQJj8NC1qL5EyvOsudyB4wq1LspTbMFoW7SeNFcqGZkZAgrEWssZqTX4gka7ATf3bQakFp1mNASDDXZNVwatUpKvOb7rkOfXgovW/sTWR6JId2KsiurZvzMn8QxSYjlvhGgjBVe1cZiR8uEolrpwj1lcvqmVkYK2DHvyH9Op/3kuHhkF50C3KLmqjyQaJwKH8DHP8YOj/h0zUCgjoWol4lvlAWpVjAaFvG0uPaHgFRivEFl359KTVlNSw9tpRMpRK9fAihoT7eWyaHKLs5NecvIeXa911bDKY6D7beDGlXu0/8JYwCUyUM+rphzM7/YURE2OZeyS6k72196Xtb36De96plV1n/n3FekGlCa5NRqWSioCFhlFAy/Adu8Y/UZzNhy9tb+Gb8N1QWuJ74TGYTS44uAWBqh6mcOCEMVKOibBJLAUH+RvhJL6qfmwq6lq6Te3v/AwsS/Ja7k/wPapS5FBQEooHBgTvGn5T4S7lAChTsk3nWDUvHeyG6LyElm61V32UuCvY7xnYE4HTJaQdJhz1f7WHLW1uC1OKGQ0r8abWgUIjARHFGMZvf3EzBYecvU0QE5IevBGBs+li31zWZBONPwjd2st3794vXExOdzbcbnPipzoWVw0TlUoAged9ZpT4B1l4Ch15zOlcdoSahe4KD1FRTov3M7hwaPAdlyCiWXbWcdy/yQcKhHgaKmDonT4IlNJmMmokUViXbnpHZJGTqLBa6dYOBLX4lRnuW1zZ9S0qLes8t5SL/Ax7ypqnDsSb+TszE2P4h633dMf4ALBYLtVUNkGzyEffefz+PPv88wxUKxqlC+Q5YCXwLjA8NZbhCwSPPP0fSBDF2topw1taX+q9H1JbavIUsZljWBzbPsb1++hc49oFvhtwWywVl3C0VL0QYt8Bh78mGi9tfzGdTP+Oq7lc5HC8+Vczqx1eTtTmr0W3ylfFXU1rD+WPnnbzpThXV+fvV99ba/SD8rBeBrP/jiFCdZVzrL9CbbdWgkly4tKaQUHC4gA0vbqDgSOAXQmNfHMuD+Q+SVyWYaHFxds+nvv+iOxgrhcdmySGnl3zqv65gMghfzi1+ekR3fwpmloAq0vu5cgWcXQYFm5xfU0dD6jQRGO3/sWAyqmMEu1jjX5KkRbiQfMgqtfU9rRaMIRoUCbE+y0g2Bnu/28vZ7bbvleTvp1LBkmOLufbXa3lzy5tBb8eFhBbhLQiRh1BrrnXyYJRQXVzN/NnzXfqUukIgGX+JPRMZ/OBgotJtSWzJF8zej0+v0jO2tVi3Ljq8CPBtjI7VxvLw0Id5e9Lbbs9x239NNUL2vOy4+L30CKy7DM64kb5Ovxrauinc/Gs0bLvD6XB0XQ3B+fMQp/WP8ReiDeGR4keY+OZEn84PNqRimZNh3/LMumfYl7vP8xuCDbNJMKPP+WkjEBIGfd7wnVVmj7Ljwoe5xvM8ptc7Sn2W55Zz8JeDFJ5oviRgTLsYottEW/e/SrOuwUxpmUzG5PaT+WraVwxODQAb9eh7sLSbmKvqqSV4nH9rCoMrpQ3ie1Z6BCoyrWvZiae3UfJICX2SxRiWuS6T7R9ux2IOblv63taXqZ9PBQTbG8BSKwZIpUYH7W53lHyUycRP4U7I+hUOPAd7Hg9qG32F5PNnZfzNKILB3wX/xnnrYGlPyFqMRiOYryCsQfrc2oe5VXNJG5kW/HbUQROtIaZ9DIoQheOcp28NrW+wqTR4wqbrYX6UKNL1hHa3w4CPhYVS21uc5QcVahi7Djrd7/r9BZth4xVQsNV6SCaDqEgZqeevR1/TQSTHe70Ml+VDaELD18/+YMj3MFYUFIWpxBfKpBSBwPBurZj21TQSezSuWNRX6OJ1RLeJZvb82WxIn0BNSE6dbcpjsCBJqCi5w/7/wtIeIp4DInE3s8Qx/qaKEs+o473ur9PmBhj6s/uk375nYNeDvv1Bxfvh2EdQ6ZtHdLNg9yPwcxQYikXizxgJNJ/H3+kiUfiqMSWJR9fuNhi5xGeVlf9f8c+nE2D4OviWnCkha3MWcqXrR7A5azP5lflEhkYytOVQDtcRqDp0CDApRBkGccNFkKK5oUkSwfLqHK+n2iNBX1cpGZLH+bpx/NiyY+z4eIeHdwUfG1/ZyNZ3xcRttpit1fnuEn/NJvMJkFdXOWQoctAOd2Bq7LwX/hxCRLhY9LhK/MVoY4jTxiGXya1BQYDLvr2MGzfdGKTGNxwSO0avV9CtWzcUCgXndp1jxb0rOLvDeQKOjISC8D8BGNfGfQXkoUMioKPRgFwO27aJY2CT+XRg+5kMcOILYfALIkC66yE4+aVvf0hIGPR+A1pM8e18HyAl/gqlPbRcAYO+sVYKZpdm88vBX/jr5F8k9UriutXX0WFK89CRQ6L0VESnYglR06D1b+5q0vNvJTXiMGVlUGjswGfHfmP72ck2g/SqbPglFnbeT7dusOnMpTy7bhFZpR1dJ+1NNbYkkycc+1DIEwR7Y10H+/5tH3S0Z/zV97l8s+Wb/HDxD0Ft17333UdmVhatrp7L/fpOXK1P5q1OnRgxdy6ZWVnc98ADZJYIH61WkY6JP4XC1n89Yu9T8KMGKk6LxaEiFOR2Ec/jH8LOB7wnYSuzxQZwz6Pe/7CdD8Cfw21SZ0GCFCzpqX4ddt7j3afQDVL6p3DLzlvoMruL95M9wGLx3T/qxB8neLf9uxycf9DhuFt/v9gholrWl6TN/zgqI8Yz6+cStuTa5PGsjL9yR8Zf3oE8/nr0L87tDDwTUBGiIDRay/kisWaNj7c9H6fErDvkrRcem2cd/U997r8uG6YCLA3b7PnznkuzxabSHhazYA5KaHszdPiXuO6ETdDFh/HBDu9OepeTd5/kym42H1OdDgy6KLR33kC3K7v5dT1/UVtVy8KrF7L5jc3WY/ZBstMlpwFoGd4yqO240KCQK6xj0Mmiky7PUYYqOfDTAc7t8q3vBZLxlzo4lXEvj3Ng5mcWi7my/tg5tYMIaC864nvizxs89t/qHFg7GY6+L343GyF7kQjy10eVl71ft/9A+7udDjtIfer8Y/xdCFhx/wreav0WhnKDdc4sVQpZkPYxQZSz9gazEY69DxtmuSz4Cxq6PwuzK50l8upBpwOVSST+SmpKyNuXx88zfubkn677aFOgPKecmrIaqzxlQ6U+g4KoniIRoU0Vc1ddQsLr/LtqDCzrGdy2mathSUfY+6SV8WcujyNcHY68bp7e9fkufr/9d2org1eEWB+2xJ/I3obo44UyT2o9yef8jVB5BqacEq93e7LJ2ugJXeqW8a1b1x1Qaj3KJgcMMgUYy8BUhVYL7c8+Q78z39AlrkuzeJZWFlRSdraMb/d8y7aqHzDKy0W/TBgJAz8XyizeENVTqDgYfZy0298B3Z9xXmfKQyB+GES6Wc9V50PmPCg76nh7O3EKnQ6xfw2NRaEMafj6uYGQFFhMChEIrPYh1BFIlJwuoexcmfM4GxovpPs9fccrMqF4ryhoAFG8p6g3SEvPKLq354Z4+i6fWwGnvvX+xwDkroZtt0H5cd/Obw6EJonPw1RTJ/XpaBdSXVLNkd+OBM3jz2w0s+ebPdb9bXaJiNHqLRdA/qKBaMo+K+Efqc8Ao37A1B3GvzKe8a84m5RLCFOHcV2P6whThRGiCLFKAwZc5jOqO4xa6v28pkC728SPn7Ay/kJsjL/t72/nxJ8nmlXvf9t729DGaun/r/6crzxPbZ3JuZSolJBVV9zdrIy/0z8JzfGkSej1ttWFQ6Vi+jUQN4TwrSZy85QuE38AW2/eSpI+CbXSNpFGtfZBTqsZYJP6tFBaWkZYWBhpI9O4ft31xLSPcTpfF1FNaG4ytbpMRqWNcnvdrXWFWgMHQkgI/PEHfPklvPCCG3+/qizYMgc63g+9XwW5Cg6/DsmTofX13v8QVWTApTukhFdurt1BO2+Wv8/8zaz5sxjeajhjWnv2Kw02KoprkZnlqLWKhhVGFGxGcepjWibfypkSwfqT/m4r40+mgDY3Q/xQkpOhRtWOrdntiI52IRWbvQTWTYWBX0P6VXjEgRfEZsxdlXuAIZeLTUPH8KVo1j0DA1+DuCEkhyWz/Krl1gSgPdpMbNMkEh6JiYnMmPk4uXmPM2AAPP64kBgkB0i0BTPrM/4sFgtlZaL/etxURveGVpcLj1mA8RsdXx/4hdgUeEsIqOMguq+oFvUGQxFUnfNd6qyBkL6Dq/MepvPU273+DRaLhXJDORW1FQ7ybepwdUD8HKUAJngPKke3jWbwQ4NJ6u143/4p/Xls6GN0ia+XhPRXCvl/GKFaFVVGFZV26neSxHRxdTFVtVVoQsTDTx+Vzs3bbia6beBlNPP251FSJsdkikWhEEEISerTZ8ZfwkgYs0pIYdnB5/7rDhP8VBOozBbeHMkX+VbpDa7lnUqPwO+dodvTAQn0tYtp53RMCoI2hcefTC7j8kWXo42zVTBLAZ3QULvEX8T/X4k/ED7WxwqPcaLwBCPTRjq9rgxV8rjhcRQhvo3zgWT81YfBZLCygesXyVzS4RJkS2RsO7uNs2VnqakRQRNXY7TJbOLyXy5nRqcZTO88HaWbghiP/TckEgZ8DhFCupaITjDb4DwfGithcXvhDTXoS9d/WLvbXR6WEn/V1RCpFAu2nHLfC0hPrjyJscZI+8nNk2RTh6vRRGtQqAQbxSgvp0IugluuxoQmQ/YS2HE3xAyAni/6995T3wj2U+83hMqPP/BxraTXQ4uC65jd6Woem6mjIq+Cy76/jJR+zbOZtpgtvJb0Gp1ndqaid11A2nwBJf5SJoufk1/D9n/B6D8gdqDn/muxQPxIsScNJhRa6DIXovtY17IGg1DHkWKjg+4bRPdruqNQB3ctveqJVRjKDEx8cyI1dQV05rrEn9tnufdJOL8NZpW6HaeaAwMGwFdf1SWNTDVQtAe0LfyzomgI4obAlBMAaEshqVgkShN0wpM2a3MWsR1jHVjqwcSyu5ex/4f9vPzUy1RSyWhlBmq13vsb7eFLjCXnL6G81O0/7m0lzLWA3P04lzQBZlc7JaPsE39aLYIZbarGEtGlcetnX5G3Dgp3QbvbrFKftXIRCCw8dp5F72+gy+VdaDvBP6uVhuDrsV+jUCuwzBAxd4VZK/pmx3u8P6e+70HXx4U1DggloJKDoEsHTV3Ap/SYKDxpDHtsxGIICfft3NRLhVSlu2TwhYCO/xY/CAZxdPlQ+mf8xHM3ivm95HQJ86bOY8RTIxj5n5EBv33l+Up+vfZX+t7Rl8m9JxMqiySiog+xls5QlQu77ofUGUKB5X8EvuaMAol/En8Bhtls9n6SD+ie0J0vp31p/f3YMfHv/7S/X5Bg9fhT2hh/I58ZyeAHB2OxWJqlughgzoY5mGoFyyNKE8X+2/eTV5GHSuG4gL4gpD473Q/Jk0CTjEYGSiUYjfUSfy1EpXBoXUzIXcDCiZ0BGCoMVBZUEpYc5nNgpCkgBdPUajMnT56kW7duaKI0tBrmLCUIEBcVysD1K5k23UCUxv1GaFud2lP//tCxI6xZAwcPwrPPQkaGeK2b/fyujofhi2wLEbkSLjnerFrVUuIvL0/s/ep3I22I+CJUGCowlBvY+MpGkvskNwvrb829v9Fn6X5+uzWMsBf+w7Xdr+W9ye95f6OEzo9AmxtRnY2CQ3DiBIxJeJraGDXx8Y+Ic7QpQr4DkCGe39q1bpi6Ye1EZaAmQbD+Sg5AtJsihIk7RKVoE0KvhxC5AVl1llj0AiqFigltJ7g8f8ongWOSekN9BsKi6xeh1Ci5ZfctnCkVn1P9YKa58hwnT+TSrXt3zxVU6Vc7JK+doG0hfrxBoYIxK72fBzDwM9/OaySkBMHJot5QX0LYBfbm7qXnRz1J1Cdy7n4bS8VisWAoM2CqNaGNabhvgb03mDePv8SeiS79BAelDmJQqh9eNP8HoQ010CZqP2pzNJAGQIQ6glBlKNXGanLKc6yMO020JmhykItuWERxdgX0uYeYGFFA0DOxJ9M7TWdAiitnThdQqIUHQz2Yzbb5t0kqIM9vEx4rIRG+J/4qTkPRLkgYbasmlimg9RzXfkmFO+HkV9DuVuGr1EBI/bpqzRY2aywMvGdgg6/lDUq10mn+/ifxJ9A6ShR5uGP8AX6tbaXEXyAYfxtf3kjGmgxmzZ9FiDaEamM1t/a5layyLGthpIREfSIDWgxgc9ZmfjvyGzU1otjSVUB70ZFFzD84n1WnVnFJh0vcJv489l9VhJDEkiCTiwVUfdSWQatZ/nmP1SE0VHyeFRWgM4mNVHapB6mvevjj/j+orapttsTfyP+MtAbKqquhQi02/LHaWP+8UwON5Ekw6FtIHOO3bDElh0Tir6//svuYqkWQWRUNMe49g3Q6UFg01FaI/Yk+QU+3K5ovcGoxW+h/V38SeyaydOxS3v6ojLLaFhdO4k+CLhWie1klCz32X5lMSLYGGzIZ9PgvAJo6Ql9m3Ifc8Os2buxzLSPSRlh9MIONY78fo6akholvTrQy/sw14iFqaw/A+qdEoWaSXfF+l7nC78t+s+xq4xxIHH0PFBpoM8fjaZIUMlXZ8McA6PQQ9HopeO2qB/vC2KoqOH8gn+8v+p4Jb0wI6nrGHm0ntkUbq6XSUgkyu0TRgeeh/CQM+DQwNyo/ATl/QNcnxHW33i6KTe3nwMwfYdO1MOI3SLnY+RoK1xsmKfGnUokYHdvvhry1mGeUNs36OfNHwQBvdblV6tMoFwuYioIq9n25m9jOsU2S+Ot1Yy+qLdVQt8/UKLW+dzWFyhZrA8hdC+umiMLf1teLYt0lHSD9WvdFSL7AHx9zX2MPFwj0etAYWqIpaEmvuiVmRMsIIffaKzhyr+owNbN/nU1EqvD9nZR4A1sO3UCbNkDVTsj4DiK6ANOCcv9gIFA5I3/wj9RnM+Hwr4fJWJvh8/mSz1ZA/f1AVP/sesgmMdjcyPwR/r7aL1m0FuEtGBgzibiSiVbGX1KvJFoNb9VsST+A8Bbh1mompVxJl/gujEp3DHqZTJBTV5jarIk/XSuxyZMLxtR118HUqRAb63yqtIiT2HK+YN1/1/FW2lsUZxQHpLmBgMViC2pp7eLbhgoDZpPrwTiizme+vMR9JPvcOcHiVCigVy/xGT7wgFiobd8uzklPt10LgBC9kOmMstP/1Kc5yw+4w9nlsKyXqDYLEKSxprraTtZ1wyxYLAKDOpVNs99Ua2LdM+s4uuSoiysFH9E9Uzmf0h1LaCXlhnIru9ZnyGQQGk9aa+E9um0bDG/5A6PSv7Ftmuph5Ejxb19XcYnwDjDsF0gcC2Un4I/BQr6jPiwWCI0Vm/AmRFgYbM6exu60LNHvLyDYB5oBRj49kmGPDSOnPAej2YhCpnCUSzYUo1iUQsuzT0DuKj9vViB063NWCr+4qnNNJrkaaEjjcmUlQsLJSx+QAoqu9Plfjn2ZRTcsalR7JMafUimSRAFDzipYfZGD/8X/ZeiV+bw5sQ8jk960HpPJZNY+YO/zZ7FYMFQYqCltmMyrJ/T7Vz9azB4C2IpCru95PfNnzWd65+neL1BbCsUHgtO/qgtg/3NiHvQFCSNg9ErHwJ03ZP4A66aJymAJ4e1FYj/Jhex3xWk4+jYU+M5GPF1ymrl/zeWJVU9Yj1mlmXfu9Nk/rqFwVX1qLcQItVillv9/TPy1iRKygyeL3Sf+cvflcmzpMZ+uJ0l9VlWJfUBjcP7YeTLWZKBQiaBfuDqcDy7+gMVXLLbK5NljWodpAPx88GePUp9vbRFesbf2udVa6BUQlJ0QMvZGu02EJkEEX9t4sAQ49gH83hXKM5xektZpoYaWzOg8g2u6X4O5vreSG4x9eSyT3rkw1kHV1VARKr5DzSrzCWL/kX6VSPqZa/2TK+/5PMyusakr+ANTFayeAMc8F+8FkzXbEMiVcia9PYlec3oxpOUQUqom2hIMFwL+vgb2zBXFN2PXQvzQ5m6RS4SEiHVjfthKvtn/Ofvz9ltfs1gsQWdJ3LztZm7fL1h79aU+Qy3n4Mx8qMhwfFPiaJHEkcmEp+n8GMhy42MaCNSWC9bm1lt9f09IJPR6RSgdNAVOfQenfyYkBCr1+8mJXMSe7MPEtI9hymdTaD3OB8WUAKHHtT0Y8doIa9GJwqwR/TJnJWT95ttFLGbYcS8c9JA0bXsLzKoUijAKDeSvdy7s1SRB6nTQelhHFWyBQke7ImmOs8ar0q8VzMKmQoe7he+dKpKksCSu6nYV3ZUzAVCkJvNo2aMMuq9pCjWHPjyU9reL+VFuVqENrStKOvyGsE/xB1E9oOfL4pmBmOs6PQAplzSukYYSIQHsTcIcnH0gL0SUZ8De/0D+JquNEdiK10IjQulxbQ8SugWnQCNEG0LHqR2tykA2xTaEmtOsSvEd/Qce8U/ir5mw5NYlrH58tcvXNp7eyM5zO62LG6PRtvm292cKCEoOwKFXbMbrzY3z20TWvv6iygNSwlP4ctxSulsUVokAAQAASURBVJ/+0Jr4AzCbzEE3gXYHs8lM0ckiaso8B+Dy8sTzVamCkNT1B/XMu6dNg5tuqleslr8JlvWhe9QCwH3iL6M4gzmL5nDVApvEYavhrRh430BUuiDLhfgB+/bbV6T9ftvvPBvyLLVVjoFzi8UCOqH/WFLi/rqSzGeXLrb+OmgQ/Oc/tmSGg8wnuN5QG4rE4s8XnzhTtaiYDiBUKttC0yr3qUmxyhvqQuoSf4YKQiNCuX3/7Yz+7+iAtsFXpM3qz6lel2JRioeqUfrBerFY4OwKqMwivU6x7uhReOyv1by8Z6utD5z6VhQlVAl2VN++MG+e6CsecX4LhLWHyizH44ffEJruzbDgc+cvtOjwIl7Z+ArHzjsGL09vPM2iOYvIPxgc7XZ7SHOd1Fe6XdmNrpd3tbJNWoS3cGQemGsxt7mN6JLfka+9CMpP4YTaUiFzs26a46bAVCW8a059K4pOFibDOR+TBxk/wBYXxu32MJuEt1H+Jt+u2QhIY1iKeiPMUwvPVg+QEn/VxmprcANEUqnfHf0avSn3xzvq51k/s+Q2R/80i8XC2oy1ZBRnOAZwy0/WJXj/BzZKAYAqLJqv9zzHljOOlcFvTniTJVcsoXOcjU1WU1rDC/oXWH6vj99hP9Dzup6EDBHSRQ1aq2T/Dku7QoaPnhd+wQx7HxeS5b5AFVXHYvGDUZ88GQZ+KfxDfEHiWJiaAa2v8/kWRVVFPL/heT7a8ZH1mNSvz4+dzfVrrvf5Wg3B36/8zUvRL5GzxxaosHq3qEsoN4gJIzUiNajtuBAhMf5OFJ5we87qJ1Yzb+o8n/Yd9nu5xsq4TvlkCnMr57r1jK+PWV1mAXA4/zAGi7h5/XE6pzyHdZnrALi9byOk6zK+h0WthY+4/bHNN9gKTs1G3woCLGawmASzph6kYoTCsxH8PPNn3pj4Bn/+Iefzz71fuu2Etk3CUnCHvd/utSb1KyuhIlQU0LWLbiaZz/yNsHKkKIYCOPwm/BgKxXv8u45c2TDGkyoK+n8C7e7weJpOBwZFEUsUNzJ7/mzKzpXxauKr/DU3cAWQjYE0dl4wib/cVU7JBK/YcjNs/3dw2lMfO+6F1SIBr9GA0iwGycpa0d+3fbCNZ0Oe5fSG00FthlwhJ0QjikBrjGIhK7eIh6hsMRauMEFrDwUKmkTBPJG7kAcPFEL0ML1AeA/7CnW0SGgkjAheu+yxd641SZaZ+Dbb205jweH56OJ19JrTi/gu8V4uEFhU1dqCPtbE3+i/YKqLPaMryORCIv7MAs/nyZVCxjM0EWZVOMvAJ46BYT8LqyV3WD0Bdj3ocEhi/FnXDmmXQ+eHfGt7IBDeQfjeKdS0jGjJt5d9y4zwVwCoqZWj0quQK5ourSCNCwqzzqZOdvhNOOEne1PXEjo/CJFdxe+h8dDrZWjpQ0GjJ+Stgz+Hwtll3s/deivMU9nm3AsR1bmw/xnIX49MBhq9gbNR8/lkx6c+F1k1BvULPqT51Rq/VWpAGXxLmv91/CP12UyY8vkUlKGuP/4XN77IkqNL+GDyB9zW9zaHaraAJ/5aTBU63OqmnYDdovMjooLFT/NhiZlWWAhmM2x8aT2rHlvFbXtuazJ5CHtU5Fbwdpu36X9Xfya9PYkdZ3ew4sQKuid05+L2tgCe5O+XnBxcRQiPMBTDLzHQ9lbo76FSRq4Ew3l0ajHZemL8fbH7C0LkIXw17SuUciXtJrWj3aRm9KpwASnYolCICsPQupVDysAULGaLdeEv4WD+QWZv70pkx4G0Lf4b13pFNlZf//6Ox3v2FB5/K1e6SBbtuFsEQy45CqF1UdXDb8H+p2HSHs8LRBCa1kHQtU5IEH0qLw/atcNB8sWe8SeTy5p8EW8Pa3AyRHwpQ5V+bLhqCmDNRGhzE+kdPrEeLqpOopV9gLtwuyhK6PWa9ZBP43HrG5ylWMwmwU6pqpPaVEX63t4AQK8HpbyGiKJfIK8FxA8H4I3Nb7A2cy2pEakO3jKlWaXs/mI3bSe1Ja5zcCsU6jP+JAxKHUTRw0Wcrzzv+EJoHJa+75KpGEKqNgdZaKItGSeTC++g1ZMACxRsAo0dW1CXKvzBonpC/gYxBkZ66WsS8tbBiU+EPFGom+9+TT5sv1MEsOKCWwkpLX5zilOEj4sX2ZAwdRgyZFiwUFJdQqje9oFPfHNio9vjT+Kv8HghujjHzlRUXcTIr0YCUPlYpdXHjrY31fmeNh+bvymhDdPw88HHAFEkpKxbNl7SwbkaVR2upsd1PWg5JDiMrPy6vH9cnPARK6gsIFGf6JJV5ISIztDhXpdSn2CbfxuE0HiYsLVO5sUH1JwXwWV//Dsiu9oCAyCKbf4aDenXCTnP+gjRix8/0CJc9Nn8ynxqjDWolWprkUaJIhp7onMwoE/Sk9gj0aEvSv24Wi2CrbHa2MCyv/5H0C+lH+9d9B6dYju5P+fOfnSa3gmL2YJM7nl8UirF2FhTIxhLYf5teTwivyIftVJNuNq1v0x6VDqbbtxEp4i+XL1GDCj1x+nfj/4OQN/kvj4let32X5lSyH0q7b4zLWeK8SC8bo1x4lPhMT50vs0L0BXa3yl+XKBPH9ixA1avFmolBQXw/vtiPzjs/7F31uFRXWsX/41mLO4CBBII7l4o1mIFWmip05Yqdbm9t/bd6q3d6q27UxcoFCmlFC1W3AkQiBB3mWTs+2NnLBlLMjMJbdfz5AHOnDlnk5lzzt7vetdaYxvnrx0U217fRsWpCobdMqzR6lM04rYb8VdxUDSslWwVquiwDGFdL1V4fasNp74VDW/e1i7u4EOGr04HEiQcUn/Aof3w9vi3ie4eHZQ8aleoLa5lyY1L6HxeZ7b13cY+eTjhXN1xiL/ZuTZ7T/JWwIlPxJpfqnV9/VosUHWUoM216nKFVSJC1SQzN8ZJGEQRLLJrJBkzMwgJC+wvNGdzDiFhIcT2juXXq3/ldHEt/7xVBIkqlbi3K7YiajCcuy4wgzNUC9eBmOEQ0jyPvUNh1Ke2jDOVTFyTFXV+8LZuBVb9axVFp4sgHaQWBRJk4rqUSFpGFkzZZq/RNIXZBMc/FHbVEX1cF/V8tX8d9ByonGuX1oa7MBeP9TbNn1sCsxGQ2PIJraetqzGT90c+qggVUWn+zxh3hMVi4fPpnyMfKAcVzqrqc9Z6dbsJGqIGiXpR9HDv+0YMgMRpHZu4iugH0/eCRswHtToTO5LmsmMTLBg7l7CQMP6X+j+6ntOV898/3++n3/zSZtY+vparf72axMGJXLo1hbq+avoqf4Fqs6jlRfRzncf+N2z4W/HnZ/jqr9zjvB50m+S6o96aTWC107GqMjQaP1tmgbjJ6Lq1uEARMKhiWkz6geiEMctqMVjqKC+HmJ4x9J7bG4WmBQsVP0IWImP0P0fTdZLoDt9wagMP/foQn+1x7nbPa3TpalebT3ODsNeJPcvzftHD4PwssuUiI8sd8dc5vDMahQaD2eAxD6W94SgTl8tl9OzZE5lMxvBbhzNn4Zxm+284JTqW5aZQKspdT9wsFjh8WPx9wIDmr6enw4IFNLePDO0uFgtKhxcSp8CAp9t1cu+Y89cUjoo/EIve6oL2mdTveW4liUfWIpEL1qhFxJ9UCcPegtTLiYiwd9WFKkvom/SH3Y5qyMuie88dyeMOrib5UhmMWwyT1gSd9ANRZJRKzAzWX+GkDIvRiA6KohpnZV/GzAzuK7uPPnN9LKy3AY5d0vm783kl/RV2fbQLECq1tKi0Zu+RyWR0GXol0t73iq6vQy/BN2FQky0+39B0iB0jOnUdiFtAXHcSqSA/h78lshx9Qf8n4KJSCPFAhCojREdp9zYoJnyE1f7lVGkqlrGLbJms7iCVSAlXCb9hV3afbUVLiL+bdtzEFSuucNqWVZ4FiAxfG+lnhbWj9i+ApvkoniCRSLjgowsYdK1/rYMNtQbeHvQ2hd+tB0QBYnf+bpJfTCb9FR+VMpEDYMiLLglpmcz+/G01ooc5kwuesHyg6MRtC2pOCqcMvYuHoxX6QqGe8RFR6ijbc9WqcLYSf7UVBkoySzHWG1s9ZG8YMG8AV6+5mtAk+xzcej+OD0ll+RXLefO8NwN2/o6MlLAUbhl2SzO7fkeknZvGgHkDfFbe+Svn79iqY07REff/cj/hz4Tz7Ab3lmQjU0ZSWy1IP4VCNMA5YsmRJQDM7OHd7srj9dvlYpi2UzxnrQjvKbrplY2TLbNB2GK1NEfOAePGCTL12DE4kWXmyyUF6CVlgH2d5Q6rH1zNM+HPUFvcRullK3H+R+dz2dLLAHGP73/yXZ5JPsK1gzxndwUMadfBjMN2K+SkacK6PsLH/DxjLWyYCweeads4LBaPck2tFuSmMLCI660upI756+cz8s7g5IY1RX1VPYcXH+bkrpPcueJOtkTcDTRvYmtXWN0yCtcK++q6AvfXr0Qi1iijPgnO2MZ8LZpfEfMeWRPFX/rUdC754RISBgQmQ8qKz2d8zrJblwHimRyjTEFmEQoxif40FPwmXJLaAye/gF/G2K3usxeJJmFfULgOVgzzTYHkD8SNtcWXqBuJv6r6Guqr6nkl7RV+vvfn4IwDyN2SS9568SCQmcWkOkRpgdOrhPW0r1DHu28Yy1sGW2+Ak1/at5XtFmtsK+F+4lNYdbZorvCE9BtE/IsDBg2CSy+F+da4wPUXwub5/pk/+4LMd+BLBRT+Bgg1rEFRggUTdeX1vDv0XTY8s8HzMfwAs9HMqQ2nMJ0wcV+fN+mR+7j9HqtLtTcUtQRrZ4nfp6Eafkz3/ZryBE0K9LpHkMDekHGbqAm1pLkm2JBrRPOjUtQNInRqpGaxwC/XlyORSAjvEh6wxhtdgo7EwYloYjTUGeooMeRSq8oUdYyjb8LK4VDbAgV0B0BQMu2b4G/iz8/wR1BjfrWw2UnQicmNVfHnd7UfiMVWzUlhg9ZRUHFITKxagDlfX8CyQVpOR35LSQn0mt2LuV/PJSo9sJ0n7qCJ1nDuf8+l5/k9AaFeAIhURTrtZ1X8tSvxp4oT2Rpd5/m0u7eMP6lESka0yIE7VHwIENknX8z6ot0y4FzBOn6NRly3JSUlHq/fo6XC/jBU34fyctfr0bw8cVylEjq1xA2r510wabVzMTt2FPS53zURYai2TyShUX30vlhw+xHxjQ1nNqvPwg2w52Goy7cp/upN9ZjMJt7s/yZfzf7Kr+f3Ffm/7Ce8KNNu9dmUKPAEZbhQbDQqUbo19mOc1/11Lgkf2tjx2gi5pnXS3LoC2HytmDiXN+ZVyFQtJxH9hNBQaDCp+bnue+h9n2279Zlzuvq00/4KjQJVRHCqF46EkanBhFKnRBbiYXJ0+FUsG6+gtCDLfv3qukLCZKg6IgodIz+Egc+KBZsrG9jqLNh2S8sWgKqYRtWQh++DTCWyPxyVQgGC9b5ssdh/h97gLufvt8d+Y/G1bcv4awnxBzTL4z1RJux3ukY6WCuajY3X0P42je1MgkwGj4yfwZ0j5js9c4+WHOXT3Z/yy/FfAj4GQ62B+sp66irEhxodDSfKxeeTGNoCu0w38OX56/0gRnFvdZWl6giLRahXWprfYTHDkh6wsZGgDsuAOQVO989m2H4brBrr83NZIpGQGpEKYGuass77Y47+zmvdXw2K3bIjrMRfhDqMqelTuaj3RUE9/58ZVlK3rRlly25Zxoo77Pa+1ixGb9dmdjZYMBGfUuv0GNMb9aw6vgrwjfhr9fVrtafPuB0uyLYTge6gL4Kjb7u0KwwLg2HCiZjLv76G208lkB3zPiBytz0hNDmU5OHJbrO9A42YjBgSB9lza6QWBV3Duvvl3toiVGeJZgWJRNiftRYSKZz1FXRf0PpjHHkDvgmF8j1ud9FqQYIUhVF8b0rr2omMaURk10geNj1M2n2iOc1KXHUIxV/dachZYs+b6n0/XFQGoemur1/rAlciEc4YQYaT4q8huCGOE/8zkaG32MPbnVxI8pbD6glCEesJxz+BHfe2fhDle6F4c/PtYT0hZY6Yf4DIEd79AJgavB/TWCtcSMw+7OsvGOvAbLIRf9UNNchVckLCQlDqghf9cs3aa5i8RTQySBuJP5WsAtZMhv1P+X4gQ7Wwrd94efOYh+QZMOozYadqxfGPhI2jNeqjLk/c05qo+XyBXA5XXAG9rKL42lyoy/fP/NkX6NKh88W2ZvTIZyO59WQMdcpT1KNk0tOT6H1Rby8HaTtkChkPVD3A5V9ezvS4BXQuuVZcm2ajaPY1uilOeoK5QTQg1WaLmoGljcHLf1bUl9rItdBQUJgiAHuNe/66+Zzz9DkBOXW/y/tx9a9XE9453FajkprURGnCxXpuwNMti2/oAAj4NesCfxN/foYvocPZv2fzTPgz7HhvR7PXTGYThTWigzgoxN/xD2BxKpQ2H0u7YdNlIkurBbAWMOuU2U45fx0FZXWNxJ/aeWGb29ickOLZla3j4ORXpCm+ATznkvSMEYTnwSLR1WSoMZC5IpPyrPJAj9BnWMevVovrNjs7G4vFwg9X/cC2N7c12/9YWaMFiT4Ng8E18Xm0kSNKS2veQe03ZC+C7+Ph9Er7thOfwZbrfcsDbAGaKf6KN8G+J6DmJOEh4bwz4x0+m/0ZFiwMvmEwvecGftLnCr3evpMjI+ZhkbVC8dcE1py/3QWTOBH2pF1xmb9aWKy0BoowYX20/xlY1k/kyrUjrAXHfRWznbq4kxq95JoSfwBFB4rI2+6ldd4PcFxkJw9LZsGuBfS7rB+P/fYYty27jT0FTQpBJVvh1Necyiu2P387zYGzvxdZCtBozeOBoKs8JDrGjr3rfp+mMDUIoqEm2/0+ZoNv2UV+QEiI/b9o3vdf2Ok9+8Ed8Ze9MZtDiw61aTwNjbUFpZe1ffHhYnZ9vIuq084ZpVbFn5UIAYTCautNcOLjNo3tTEOYqhStotzpmbvq+CquWnQVb253VmCt+886lty4xK/n18RouOPYHZQOEgs6nc6BmI3wIfOu4Df4sbt4drmA4/O31chbLu6t3vJXJBIY+ir0eaBlx5dIQdvFWZUkkYDMwxc8dR4MfqlFOa5WottKrKpUwumjOrIzg28fjTqyBU0tLUBDTQOL5y/myE/OzVktJfDtbyyFgy927MySFmLn6Z0s3LOwWQauFZkrMnmp80scWuzbvdO6pmsr8Tf5xclM+I9diWgl/rqEd/H4vg93v8cv/TtzIuFFp+351fkMTRpKakQqAxMGej2/x+v31DdCgd/0tY1XwPcJ9nxrXxqq6vJELnLuMpcvT2p83BedEMUfvVIUW70p/obfOpx5q+ahi28f55v6ynqbkrdZbk0wsedh8Zk0VTFbLLDjH3Dwed+OI1MJpWejhXyroEmBmNEe751WlwOFyU78bXp+Ezs/3Nn68/oBtY2NHlJTByL+ijbCuln2pmpluHCkkEhcX7+7/gW/XyOIjmCh8rAgzPTFzoq/xt9n+clylt+5nGOrWtCg1woMXTDU5m5y78/38sjvd1MvLxKfY8xIGPKqd1vxvGVw6IXWNeMaquDn0bD1xub3zbixMOZLm+qGwS/Cefs9z0OsSJoqcoe9uIH4DXufgK81UHkItdzuECRTyLhp502Mf3R8cMbRiE5hnfjqoq/pe/J1AJQqRaPbzxVe3umA+mL4406x7rQq/2pzoaFMPMO6XmH/bEBEfYz7yW4P2ud+0TAW4kWUcPAFWNwVaj08vKZshgnL/TN/9gUJE2HMVyIaA9ApxfPSKKtG3yBjzP1jgp6Va52fqlSI/PfFnUVETksxYQWM+1FYjc84JJrx/YE102DdbO/77X8a9j7un3MGEsv6CWUkYv5qbboJhGuQJ+RVietCZUhErZaI+2Kf+313fekgCPg16wJ/Z/y1A2RKGcnDk9ElNF9kFNcWY7KYkCAhTiuq7tZFoS4Qa5LIwdDzH77bmwUDvR8Ai9F3L2zEAx3EQq+4GEqOlLD5f5vpM7cPqeNTAzhY18j6LYtNz21i7P+NpdOoTm4Vf1bir10Vf0deF/llQ/5n82N3i90P0dOoBeZ6tB2z5qAcKhEFkOQRyfxf/f81U3W0JxytPq0w1BnY8+kesMCwm4c57X+sVCw2Iiyim7O83L7wtCJTRHOQ3pK5j0kP2+8QdjqdHCYIFovoLAzrBcPfFBPB1CvFpCtygLAwtKLXPwTZoQhvdvi2oJnir9s1oqtN2xWFTMENQ26w7TvhMfcWWIFGg0mGWS4jUdEDXdJZtvuBT/j9atGBN2U7SGU24u9g8Vno084CDeKzWDsT4sbBhFbYpMjVIkvVWA277nf+nNsB1mdJVRWisNK4gEnUiYLZ6armxN8XM79AoVVw857A2la6y/j7+sDXHCg6wPkZ54Njs+ToTzEPeQMOtqEYkDgFRi8UhSZfUZ0pJsF9HhI5f65w+FXYdR+cu1FkcgQQEom4l9XWgqxgGdTtFQHlHjCzx0wGJwwmXufcfXrFsiuQyNp2r/aVMDj28zFW3LGCq1ZfRWii3WLQqnhKDU+176xJhnFLBQHzF8Lzuzdx+jT818H5x2rLW1zr3Ol0ct1J8rblMfOdFirafICVeNRo4MSxFhB/ZoMoBssCWM2OHi7mjtFDve/bWkwUKigsFlEEjzsbYka43z+l5Z+B9fdpJVYlErHAropJZcC9qUQEJr6RksMl7PpoF5FpkfQ4r4dtu/V+vN34EQv3KJicNplYrQ85r/ufhEMvinmNJzvr3J/EXCbx3Lb9B4KAJ9Y9wQ+HfuC1aa85ZeBaoQxVoovX+Wz16S/FX8bMDNvfzRazzSa2S4Tn+2RZiYJ6ZR67+Ihy/W22RpDUiFTWXrOWemN92+fsx96HwvXQ827n7TEjhK3VoRfBUC6uXW+RE6Hp4hoMzXD58pAhEB4OqgbRSSmLzIVs78Rfe+PV7q8S3SOa+evnk1W3j51dn2VF2XBmcntwB9LpAmFl19SJQiKBk1+J52+vNqiYWoKUWc2s7ppCKhXPIqUxilqE6uDI80eI7h7NoPn+tbv2BfpyPXl/5FGmFOt9eUci/qKHCecLa9a0sU64mahiQdnk87ZYhBWhPj+4mVN5y2HH3XDOOjSasc0Uf/oyPVtf2Yo2Vkvauc0t/wOBN7a9QZ2xjonSOwkJiRXZpOE+NLgOfh6Gvta6OY/FLN4b0c+5Bmasa+5Y0kjCdEhEDoAul4MsBI1CC+bgqzetOPHrCZQ6JecPmMun5WJbiFYL0S7ymT1BlwqzMu2qMosF1s8RDU6TNzWPZonsb885LVwvIidkPtwQpCFCAW9qH/tpXxAaEkpRbRFGaZXPLjP+QENNAyd+PYEx2cjmwtNUqZIICektSJ+MuyFufNtP0pL8b88HwqeM1KzPwWKAfg/76bwBQvpNIBfzNJ0O5IURgJ342/nBTgy1Bobf5v96x5pH1qCL1zHslmG2GpXKkNQ+TVJnMP4m/toBSUOSmLfKta2i1eYzVhuLvNGL3Zr/EBDFX/w48dOR0OXiFr/FGj5fp8ympARqimrY/sZ2otKj2oX4q8yt5MSvJxh2myCPbMSfg+Kvrg5KG51J/EL8nfpGyJ1bGmxa8JvoThvxvvd9R35A5j7xRfRE/DVV/HUkws8KxyKmFQq1gv+r/79mOToWi8VWiE5Sp1GHIP6SkpyPaVX8dW+JxXhtjlAaKSOcCSGJBAwVYKyBkm2w816hTBrxrphgOiIsw2794Uc4Kv4sFpCo4trNntIdzEYzZXtyUdSFcX70Y1x/bQu7veRaQXg32qxarT7BTnwKJvgNCIlp/UBVsUCs6JhrZ4Q28itzYufAt2tgrrg/Wa2lXCn+xj40Fpky8H7kjh18J349QcHeAgZeM9BzMbOtXV4SCaRe3rL3aDpD339D/ET3++i6QuLUNmUXtWhIGnFfy+32Dd16eM/KfXyC6w5DXwvXnuAr8ddrdi/CO4WTMMj5d2S1VnYqsCtCIfm8No/tTIN1YeOo+HNH/F26+FLkIf6d2pedKCPrtywMRV1BGSGIv3IXVqzukHgunLfXr2NqBnU8DPTBsunEQjHfGfSsy7xBn1BzUigi0hd4Jv5aARvx1/j7BbHArqpqO0HkCYmDE/ln8T+bbbcSf0sqH+ODH7JYfdVqJnb1cM+zoufd4nkZliFUXRsuEioJxyaJmmxYOwNCe8DMw376nwQOKWHi+5JTmePy9c5ndeaGbTe4fM0V/JXx54iC6gIaTA1IJVKSQz0vLCJOz0YR+g8KOMbZH57NiitXkBSahNliRiqREiL3A2Mx7A3Qu7BhybhD/Ll6EpTtgn4+dLvLtZDg3kZKLhdZf4fWif+3Ol58Tt6sPgv3F7Lvi330uaQP8f1absHWVvS5pI8tVzOvYT+50Z+xsewkBJ34myN+XGHqdt/zxvc/DUdehYm/ijzHAEKrBYVJqGdK60q5avVVhIS1D9NWuK+QT8/5lPj74kHdwaw+tV1E46YV5Xvh5xHCAj/jH877SiQwfqmIggnm2j15hpgzh/dGrYaU4qu5dcIcrpwlGpJje8dyd87dAVO9g7A1f2/ke/S6sBfjHh6H3igegFKzqmVZja2dWwD8PEo0w0zf5bx91Rixlhj/U5NBV4nPM9ZL42LZLpE5lzzTu+LMH3Ag73XKP0APdUYxidnx3g4kUonf86jd4fsrvicyLZKLltlzU9t0XToSsN1vhYJfQeJhfVy4Hn45Wyg0mzbBuELGbeLHHSwWOPGJaIaJClKmafUJOPAspMyGpCmEKsUzyyiroq4OFk5biC5Jx/nvB1ZRWplTyZezvkR7rZaHO/+T+OQLmK36QVxzQ170foBgYoJrd4JmOHd9x4rccgcHYlJYfYqattXVbvub26kprAkI8bflf1tIGpIkiL/GGlWIIVHcl38ZD7puMPIDv5/3z4a/ib8Ohqb5fmBfFAZE8deR4aBE8QbrglzfaPWZdFkS9xbcG7Rcqqbof0V/+l/R3ybjtVl9Oij+rGq/8PA2kLoHXxQqorixsOFi0V111sKWHWPsN6JbyZffddzZmBtFZd6sPiVIqGoQ9m0Wi4Xjq44TEh5CyoiO4WvaVFkU2siGyJSyZgRHYU0hNYYapBIpKdpUjiKIP0eYzXBccIMtU/zp0mCOm9yeaY2WNRYLjP0eogY7v269Rkz6lhO+PiC2sbFfrxeFxzCdERpKQaYBhY51J9dRWlfKuC7j2PfSPvJ35TP367l+H4cn1JbUkv/kByR2GYZaPb3lBxj2htM/k5KEh354SD6R2y8SZGyvfzgvms9wWJ8lJ8oH0a+f1PY9slp9Wm0UHBGsBZojYXTgkwNsf2M7KeenUN0gHoSdwx0kL8YaEc4e3t92/QYNCh3091Ks7DQ7qOpOK0FUbYiFNnC0ZcfLKDteRucxnZGrWjdN9NXqMywljLCU5krzIyXCcrBHtF19hKnBN0ujPxn6xP5GN45TV2cvWrgj/hRq/4fD527J5cdrfyRk8FxqkxqJv5ZYffqAoF2/ZTvg5Ocw5OWWv7doI+QugR63weTNgoj2BItZZPyFpsMo3+xpr+x/Jef3PN9Jta7Tgby+hjU3LKbqsnSG3xoY9bAmunkDRX091MuLKDJmATAkcYiPB0ux26lKEHlFTeeY2k7CBqstRdIgwkb8Vbkm/loK63O4LcRffVU9r6S9wsBrBnLuf8+12XwmhyajkLm/F5jNUJgdxij5ag4Nncrewr30f7M/SpmS0Z1G8+3F37ZoHG6vX1038eMOE36GmiznfGtPsFjAXO92vjtjBizeLoi/MmMuXYDKSkGau1tnlR4tZf2T64npGdMuxN+0V6bZ/l5pKAclRASjMO8Is1FkG7lDS5qXQqJB06VtjXIgVNWmeuj7kNtdHO3GSutKiRvYfo2Jkd0imf7GdHZG74SDIDPrkMsDGPvQFui6Qr9HhQIJN9evo2VhMBCaLn4Qc1mFOQyVIYzQRoJGppQRluzFlaiNMBlMmBpMmOpNGM1GLIgajsyiEkTR4Vch8204ezGEelAdmk3CelAqF7/rliBxir2ZsXAD5K8SZFFYT9fX1MbLIP9nuKjccxNkzo+w9xGYvic4xJ8D0tRD6Xv4DaaNEo2bm57bhEwpC9q6cvILk6mQVvDN/m8o1SWR0HAWkuxvYc//wfB3RQ2tNZBIoNtV4scTwvtAnwchxE/3J3MDbL5G1PxGjgzO/LmhXHz3tZ0haYrN6tMkraa+HurK6lBo/b/+aApdvI4LPr6An80/w0mRBdoiUr4jwpMrRgeFTgcKYwRgV/zN/nS2TwLH1uD2o7dj1AtRht3qs1HxZ9QHN7v0DMbfGX9+hsyHGd7B7w/y679/pb6qObvfPbo7z5/7PLcNs3d6BDTj79j78NtMqCvwvm+wYKyFH9NFZpmPsBZJ6hQ5lJSAPESONk4bFIWKJ1iVbk0VfxYLrGyMaOvUltzs4x+Krp+Y0TD8bRj4TOuO04KHjlZVj1Ri8qj46xPXB/3/6dl/y35A/B6+mPUF655Y17rxBQCOVp8ymYy0tDT0JXqyN2WjL3fOyjOajcwfOJ8Le11IdIQoPJeXi4LN6tWiyJ2bKwgylaqFCk6JBFQx4sfTPp1m2y3uLBZhUblmqlBsfqUWHS9+hkIBUY3rg4ICoGiDyBc8/hEA8xfPZ/ZXszlYfJC8bXkcXXY06J7VCrWCkFmTKUvo6RfJv1QKzz4LD/5bg6T6qPDuD2bORRBgLTh+feDfMPZbW0E2PSqdZZcv45d5v7Tb2BwJ+TH3jeGadddQIBPPp3htvHN+Y+URWD8b2cmFpKWl+fT8/TPD+v1vqK6A4q1QX+Jxf4vFQlV9FRX6Cqft297cxqfnfkp1fuu/974q/pqqq614dPyj3H/W/TbbaAB+mwaLAuR12IExJu41bh9+PXW19rwjK/FXUluC2SEHyeo24Gp+2Vp0GdeF87++nKoo8bsPUZltBINPir+sz+Ho225ftj5/23z9Zn0hclHKdrvfZ/ALMLeqdUXpku2i47nqmFD6ebP8kkhFJkyyZ8s6R8RqY+kW2c2JtNFqwSKVUbz1OGXHylo+bh+QvSmb0mOlzbbX10OFZjsAGdEZhKt8KAbXnARDpfO22bkw9jvx96LfRVanxQzJ0+1WWB0c1nVGdoX7XNffX/qdPQv3uH3dEf7I+DMbzMT0jEETKwq+1mxUbzafp0+DwQAxpgFsun4j6VHplNSVcLr6tJPa1Be4vX6NNSK3yx123gc7/+m5gN4Ui1NhtXvFaWIivP2cIGjza04THikyBD3ZfXad2JVbD95KzwsCq07zBTUm8SwOVwWW4GiG1eNFHpE71JdC0SahAvOG9Bthyu+e1zS+4OSXInvZA3Q66HfqDVaOq+LukXfTUN1AZW6lx/cECqFJoQy7eRgNqaIAKTdpO4baD2D9XPjJIZdOFQv9HoHY0c2v30Mve/29BxpWJx7HOoPFYqE0s5SKUz58B1sJVbiK2w7dxqSnJtnUfgBSc0jjZ2kBi8mzugvEc39pDzFfaCmGvAQDnhS1sM3XQOZbwl7vrIUw9H/N98+4EwY8AyYPRRkQribjloA2teVjag0aysRz/vgndNalkVp0Mz1lojn3oq8v4qKvLwrOOIB+l/cju0821/98MUeSHhGfpcUsPkdpEC7SkCjxmXb1MU+wvlRch9ZMzqaQyMRn2fNu/82fvSGir2hS7ynsnkNDnBV/12++nou/bbljW0uhilAx4KoB6LuK61Nu1gri78jrIk/Py7o3qCj9QzSwuHI9cET5XqjzYk3QEXD8E/h1CugLCQ2F1KJbucjyNTMzRKxBTM8YYjLa+Nx3A22slvBOYv2RoEsgwTwUnb6n+OynbIbRnwXkvIFEe9Ss/ib+/Ayz2X0QtRVHlhxh/X/Wg4v6eLfIbvxj9D+csrMCmvFXlQkFv4hMvY4CuQY0nUDle4eh1erToCihoLQWi9lC0cEiyk4EpkjiDfm78slckWkran4z9xvWXL2GQQmiu+mnn2DFCsHnXNSWuc/U7TBhpfAMT79RdFC3BDWnRBeYrw/K/U/Tc7eKTmEHPBJ/UokUZRNlxsx3ZjLqnlEtG18A4WgpaDabyc/PJ3NlJh+c9QEnfnUufCSHJfPB+R/w9dyviYgQ2yoq4L334OWX4Z137DafaWmCPPIZ1VlQcUh03DZF4Xr4XCIKZY6QSERuksUIUUOF5diI91pwUt9htbssLEQQj+kLbAVPrcIe2H3x9xfzYPWDQbd1DQkLwTxiFFWx3Xj4xFgSX0hkw6kNvr25fC/sebRZcUoiAYkyTIRw938CfuoF6y7w99DbDdbmwKoq5+x4jULDtO7TGJAwoNl7Mldm8lrP18hckRnQsVmJv5AQCO8cTpexXcipEQoLJ7UfiOfEyI8xJ80gPz/fp+evX7HzPljpwepv64KgFlCsxJ+mfJmwcSr41eP+T6x7grBnwrjvl/uctvea04vz3jqvTYp5X4m/lzu/zMLpzVXq1wy8hqfPeZpojYO9WPQISOj4WWD+xvba+3n0txVOKvtotfi9mCwmJ+J278K9fDLpE4oPellotgChiaHEj+mOUaVDpQKDuZ7bht3GnF5zbCoojzj8Kuxzb8Fsff62+fqVa8WPwUthUKFrnYVZ6uUw47BQQrh6XrvC8LeEPW19ibgXNH2W+wCdDkzyEAZ99xBTXpzS4vf7gm8v+ZZvLvqm2Xa9Hsq1WwEYljys2esu8cddokHI6MBoOar9jr4BOYuE0ktfLPKLC9a0euxe4admJG9WnwDr/7OeHe/u8Ol4/rD6VEepmb9uPmf98yxA5PPdNOQmZvXwTDafEs7ZdO4MaVHd2HHjDpZdvoyt129l5ZUrWzQGt9dv3nJY2hOOu1G75i2FwhZ+7skzRNayB8Tr4pFKpBjNRsITCwHPdp8hYSHE9IxBqQu+mtyoN/L9Fd+z433xnak1CdLKJ4LdX7CYQZ0E6kT3+5z8EladJfLgg4Wx38GMgx53EVafEZj1OiQSCd9f+T0vd3k56A2IjpiVMYv3Jv5Et4J/dhziT50g7OldoNn1e+hFyHw3iINrROVR+CYc9jyKWg01yuN8WX4Xj/72qG2XV3u8ysq7W3Z/ai2ciD9LI/GXcYf4TupSPb9ZFQe97xe2mq2FXAPn7YeJqz2rcRPPhV73eLfiDU0X909vTgV+gxQy34HC32yKLOu6IGFAArG9fMgK9iMcbVtDQhCxQuftD3j+eqtgrBZ5lzmLXb8ulYvPMnqo/+bP3iBViGaOxtqeo9WnXu/pjYFBrUEsiGTmxgaLioPCkSOQWeItRcEa0dxUddT9PhYzLOsP2+8I3rhai5osKN4I9cXodBBVfRZdqueSHiWU2oY6AzWFNX5//uor9BQdLMJQZwDgrpF3cXH5NroULTij1Z5Br1nxt9Wn3+HLl33yi5MZ/c/RKEN9W2QENONv4NPip6PhnJYtBsNDwpncZSYHt8dRWFqPyRDCG73foO+lfbnwiwsDNEj32PLqFnZ9sIv7yu9DHiKnd2xvescKsmT3bni3cU59zTUikL7VkIU4d8saa0S3uzevdyvylsG2m2H8Mkjy0O1pRXgf9HGXYDCHUNdCVfWAq5qTCe0JK3GpUonrNj8/n8RBiZz73LkkDHRPOluJv/x82LhR/P3nn+3WrS2y+QTY97hQbs7OFzlFjqhvLN7WZNlD2a0Y9Yl9MTA8cORCXBwcPNio+NN1dTqXVtlI/Blq2jXH0fpZVpgKyK/L9/2NRRtEQTpmlPuMRFMtxE1oWVd6B4eV+EuL2Ipxy2coei3wql5RaBRIpBKb1UKg4Kj4q6+qR6FWuM/3U8VAt6uwmEzk791LbGxwF5IYKoT9iSurXVO9WPR2vhi63xyU4Vi7pIvMQ4X6O7yvx/0jVBGAXZFuRadRneg0qi1SdN+IP4vZQtrkNKK6+2g75EuG258QVYqh7MyH3g6L6xB5CDqljuqGaopri21uAt3O7cY07TTCOvlPMWKxWKirE/d3jQbUCjUvTHnB9wOM/tRjc5H1+dvm69chU8YlTHrIXSqaZbwV7lxBFSuKH99GQZfLfLNVj58g/qzNhW23QI/bmz/Lm+C5jc+xu2A3j4x7hO7R3cXcXyIJWMafxWJh/OPjXTpkCOJvGwDDk3wskCXPgtDugoR1hRHvChWoIkLMWXfeK+6R1t+VP9FQDhvmQtr10OWSNh3K2mCYU5mDxWJxOeeZt2oe6mjfCk/WZk5/fq4jU0YyMsV75o8j8Qeie39adx/WAC7g9vrVpUH6Te5zcAe9CBX7W3ayYa973UUulXPdoOvQKrREHZBy6oBnxZ/FbKGmsAapXIompo15wS2EodbA3s/3ogxVMvi6wdSZRdNClCaIxJ9ECmO+9rxP3Nkw6DnPtq1W7PyXuMd2aaP6Q+tZtQrNr6EeM3oQkRqBxWRBIg/umuTosqOsvGcl016ZxqiU6SyqgWDytx4x9NXm29bPBVkIlhEfO1+/U7YJtVawoYyAqGGg7YxGAw2KQjYa/kf27lQeHf8oEomEs//vbGJ6BUZRAiI64sC3B0genowpTaiF5RIlEqQtKzBLJK2rreWthKyFwhYyvKeo8UR4nsd3WCjC4KJSkIciyamjOHQzO6vqgamYDCYMNYagxPGYjWZeSX+FhlEN0NOB+OvIUCfCOevEM9QL/DZ/9noiM1QcEPM6XVfGdh6LoV5B3pF09GbRHFxbVEv/KwPr4HB89XEWz1+MYZ4BlA5Wn8NegyH/8902PBjofAnEnAUR/dzvYzGLjOMA5+H6BX3/z5bzZ+UkqqrsLy+7ZRm7PtrF/ZX3ExLqv4vs+KrjfDP3G+Z8Pod+l4nfpbVWFCrLhsOLRP5zeC/3B+mAaI8Gpb+Jv3aAOlLtNpx4x+kdmC1mekT3ICxEFG4CavXZ0VF1DLK/hV7/8tihLZFIWHrFj8xpdBKqrYdxj4wjrl/7+P0Pvn4wnUZ3ctlB+t13Il9jwgSY3Zb4p+LNotgc0d/eTb3+QmENcGGRb11dCecKf/MoHzu5U2bREDaLvEa3CZPJfX7Bcxuf46ejP3HHiDuY08tNYHw7wlHxZ0VMrxji+zbP+MivzidCFYFKriK8cSG3YYM9x8pigX37xN+7d2/hQFIvB1V8c9IPIOUCuLDYdSefpw5AP8Kq+Ctw4QbsqPgrPVZKyZESupzdBaU2eJ3TWWuzkLyzkvAuEzFYxEzAyQ7SE7peLWxy3RUz8ldDbTaM/sRPo+0YUCrFT1JoJorjr0LSOBvxt+zoMvYU7OG87ufRL94+We0ytgu3Hrg1oOOyWJwJo/dHvY/ZaCbvNVG1c8y9sr2hHQlnhr/l/jVZCFxcI4jjIMGq+Cuu7w697/O8M3biz+rP70/4QvxJpBKRCdAEewv2UlJXQp/YPsRqg0zmdkBYP9faWguOAQofzPoAjULjlAmdOCiRxEEelButwE+3/MSuj/YgO/sutNpWFMYdsnvaFVVHBQnU99/e8zldwWKBujxIvRI6XdCy9+q6wbilEO2dPPv6wNdsz9vO3N5z6R7d3VbcLt2bwyFVtd8tCSUSCYPmu87aqdNbKA8VxJ/Pir+0+Z5fl6kgrTGvUhkJk37zXBhpDSwWQfQaqwW5VPx7m4m/pNAkJEioN9VTXFvs8t6UONj3a88fVp+lx0rZu3AvPWb2aNF1f1I49dqIv4AgapDnZ2TSFPETALwz8x0Avq6ErXhW/NUU1fBC4gsMvWUo571+XkDG4w6qSBUP6R8SDoIW0FsaiT9tR2GMGhHR1zcCwlANB5+D1HltJ/5M9VB1RNgyu1EjarVQolvL/7I+Jntzf+66/q62nbONkMqlSGQSnx0P2hX1ha5tDtVu1qOBhioWJomYAXWuUPKAXdkDMOHxADSHOKDiZAU/LfiJiU9OJHGB+M4pJOJ3FBKCyBSvzhSNJFIf8swsFtG462vOX9lOyPpU5Mr7ioYyYb+XNA36u3dWYNOVQml/UXlw6gcSiXi+A/WyXDZnTGR7nYp3qOO7S7/j4A8Hedj0cMCbhs1GM5oYDXVK0SFsU2+e+lY0V3dfENDztwpShefcwYoDsHI49H0YMlrwXWkLLCZY1g+6XApnfcHtI27nsrTbmf8V6OUit/H0H6cDTvxJ5VJ0CTrqZOLzlJm09jpeRyL9QLiweXNik8qh37+DM562wsG5IzQU6hS5HGATKzPDmJI+hdSJqcjV/r+3RPeIZsyDY2yiDIvFgl4v7hthpl3wxx0w8sMzjvhrD/xt9dkOKD1WSl2pa5/Ee1bew7B3h7Hs6DLbtoBafZbuEHYsZlMADu4H7PoX7LofTq/wuqtCgY2UKS6G8Y+Op/eFXjJYAoROozox+LrBSGVSKvQVPLnuSd7eLjJuysvFPuPHt7Fmvet+WDXW2cYofQEM+q/vxwhNg/TrW5TF4Jij5knef7jkMGtPrmV/oejo/fH6H3kl7RXfxxZgOCr+vGHOV3PQPKnhx8M/2hR/VtLv3HOdF3ctVvwlnOO+M1Ai8W7fEWDENXLnhYWAqQE2XAIHXwScFX+7PtrF59M/D2j+gisY9UYsdXrATINZfKhquY9WD3INRA5wT5IfelHYNbajbVCgoNPBltxZnByc46SSefuPt3lg9QNszN4Y9DEZDPZftUoFPWb2oNeFvXhq0lOU3VfGQ2Mfcn7Dhotgaa+O+/nI1UG9fq33Zk82zI6IVIlFeVPi7+iyo7zW8zWO/Xys1WOx3h+VregBeG3ra0z4eAKvbnXoUq88KnJ/C9e3ekxnKgapnuf7i5VoG5yz6+b2mct5Pc6zZW0ECrG9Y4kc2AWzPASNBsrqyiioLqDB5IPs32wQar9gXaMFa4RljquxqZNg1KfQqbWNSBZYPhgaSlp+DIlEWH6qvBPZXSNEkfB42XHAThCVffcb31/5fcvO20aUGnJoUBQil8gZmDDQ+xssLbSukUggfpzIwPEHrN+znMUiY6k2B6buhMEvtfnQSpmS92e9z5LLlqBTul6QmQwmqvKqsJi9f9+ta7q2WH0WHyzmt0d+o2C36Mw6WnKUynrvGWdNFX9+h9kQmONmfQmb5zvbyHpAUpL40xPxp4pQMezWYaSOS237+FoIiUSCPESOXCWnvh6MMivxF8SMv8OvwO7/a/m16wpyDZx/SuRZtRUl24QFWtbnbnfRaqFGlcn66g/55Xj7ZVMDdJ/enVv23UJuei7fHf+QSvXejkH8NVTA7oea2ymfsxYm/uy8ra4Aqk+0e01IoxFKHhCNpcFCVHoU836ZR99L+9IlvAvH7zjOv+OFva1Khci333aL79fKhrmwYqjvOfF97oe5lRDex/u+VsjDQF8gGl08IbyPsEkOUtMwIAiqsj2Ea8Qkxogek9lE13O6Mui6QZiNgbe6k6vk3Lj9Rrhe/FtmVfwdfQN2Pxjw87caZqN75a00BGLHiriLYEGqgH6PCRebRljXnEYjjHtsInO/nRvwYaSOS+WGrTdQOkBkUtusPk994znfu71gavD9+u/oaKiA3J+g8gg6HZTpfmdD0sU8se4JAAbMG8B5b5znV7UfQHz/eCY9OYnYXrEYTAZ0T+v4LiUNg6wCYseI5sHEwDSR/dnwN/HnZ/jSufLu0Hf5Zm7zLA2A09VidZKos3e3BdTq8+Bz8Nt0XAYOdgQMextmZgpyxAeEx9TSICujpANlu+ZW5fJ/a/6PB1Y/APhRwdnnQRjysnOHS6cLhAe9rx7uvubUWNFQjmLnzUzq9ingucBsVedkV2YDorM1NCnUp4JIMOBoKSiRSIiKimLRVYv4ZFJzddexsmNYsJASlmIj/qy46CKY2zjX0WjshQafcAZMBpwUf1KFUOCWiMwfR8Vfz/N7MvO9mejiA9Gh4B7pU9LJufBOKuIzaDC3UPFXcdBzAanPQ2ButGz8k0GnA71RR3l9slPnapJOfIHzqpp7Y+36aBe7PwncxNqxkUClgnOePodJT05CIpEQoYpwznsDsejRdUMilRIVFRV8u9m6Ash8z/Vio+YklGz3vhj3I5yIvzVTYdM8j/u7U/xJFVJkClmb7tW+dLxnrshk6c1LKTvuvLg9WiryELpHOcinKw/Csfeh+nirx3SmwhDSjT9OT6OqzntDQ8mREt7o8wZbXt3it/OPuH0EvZ+6AotUhloNL/7+IgkvJHD3iru9v7l8H3wXA/vdF4Ktz1+/XL+F6+HIq1C2q/lrIdHQ9UqIHNi6Y0uk0OveNhCHQG2eyLXzACvxd6JcZA1bCSLLWaOZ/clsv9vDrLpvFW/0eYOaoubPQkVdJybtzuG9CSu8P1ctjcTopqtaNgCLWbh7+EjmuMXex2BtY6aSsRokMtB2FeoViUSMz+hjV4QbzB80nxk9ZqBWuL4WV9y5gheTX3T5u2wKfyj+UsensmDPAnrM7IHFYmHIO0MIfyacw8WH3b7HaLRb03fx7qboFS6v390PiqacOg+MW2tQuk0U3+s9X0Nmi5n86nyk4WIe48nqUx4iZ/pr0+lzcQuK7X5CQ3UD2ZuyqTpdRW0tDDqxkHP2nOLKAZcGbxAnPhNxAxIvJaFfJ8MGL+OSSH1TOfiC8F5CnR3jPrpCqwW5SdwgqxuqOfLTEb684EuKD/sv47al+HDXhzy571oKw39qP+Iv63PYcgNkfy/I0/1PQeE6l7s6Xb/H34cfu0GZbzmlfsfB5+Hg86jVQskDQvFnfeYtvXkp3176bcBOHxIWQrdJ3YjsFolCpqBrZFcizT3EayGI7+PE1SD1saMt6TxIvQLMLchGUYS2jJyTyuCCkzDoWc/79XkAxv/k+3H9gXUXwO9XEqGxF7xqDDUMu3kYs96dhUwRPIWWLePP0kj8DX1DuDB0VKw6C5a5icgJTYMJyyH1Mv/On72h38PQye7SIpUbMEmFIjd6QArdJvlgBe0nzOs/j3H1zxFdPRaNsg42XAwHWiB8CAbqCuCrENjlwYGn8qgQcXhocOkwqD4Oa2fAqW/Q6UBhjACgtK48aEMo05dRa6ilRnkcuUmHKixSNA96yijuoGiPiKS/iT8/Qyr1/Cu1WCwMu20YfS51vcDIrxb5VI7WTVbiLyCKv/QFMOI9sUDuiFDFiAecD5YKT657kjcitRxKfoDiYtjyyhbe7Pcm+orgp86+P/p9Fk4X+S9ldaKoac3gqW10rbDmMbUaiZMh7TrXr5Vsg6pMz+/XF8LXGtjziO/nlMgh8y0GJ4nOSk/EX0pYCmAn/iY/N5n56+cjkbajNZ8DHIk/qVRK586dwUyzDrSq+ioKawoBSItMcyL+MjIE0Td7NsyYATfd1EIV5y9j4Zfxbfp/BBpW4q+wECxI4JJ6GPMl4ED8GWpIGprE4OsGo44KfrCy9bPUmxoVf26Kck4wG4RtxaYr3O+jihNWJbU5fhhlx0JoKEglJgylmVBzyrY9MVRMnk5XNS/YrX9qPZue2xSwMVnJIoUCvDxKBYa8DON/sl2/3p6/fkftKdh6g8hKbYpjH8DKYVB5JGjDsT5T6uoQRXST5yK3O+Iv7dw0bt57M+lTW2/P6Avxd2rDKf54649muZE24i/agfhLngkXlkCn4Gf2tjfKQ+fw5PrF5FU555Duyt/FJ7s/YXvedts2uUqO2WT2ey+Xdd6i1dozIa1zGo+Q6yDtBo8Wl369ftNvgFnHINqFLWVtbtuPP+A/7udd3lC4HhYliyK7B3SNdCb+rARRbXw3es3p5ffFokKtAAlooptPSvV6UBuSmZg6yfuBjFVi8d1S9V7mu7AkHQrWtux9TVG0AYobn09drxRNg1bLuoYKWJQC229r2zm8oOvErgy/fbhP81x/KP6UOiXx/eLRRGso15dT1SACV6x5hK5w+rQg/1Qq8EcskMvrVx4KMo2wsfcn+j0iFDEaz1LF5zY+R+ILibx26H4AKir8m6XoLxQfKuaDsz5g9ye70etBbtYRJe9EhDqIVp/nroNzfXB4sJjEjyfUnRZNT/5QjIVEC0tmD5moOh3IzKLRtbqhmopTFRxZeoTq/OA3Veb9kceWV7dgLBHzGaklpGW5cP5E/mo4+YVogKnLgWFvQfqNzvuU7YbMd5GaauzXb/RIyLjbfeZ5oHH8I8h8F7Ua5I1WnxYsNtKmLLOMksOB6+x21VRjXV+GhCByuBIm+r7QT5sPQ19p/kzMWy6uk6bIXSaIgD8Lev0TMu4mXKsCi3g+BFPBCaAv17Pp+U007Bbkq03xF94TYt03FbQ7Us6HThd53a291r9vbX8LzTNKdncTjV6eHMD8idytuWx4dgNj1GPoX30v4bWDCVE15tR2NNvWkBiR8xc11P0+xiphaV1fGrxxtRa6bjDiA0iZhVIJakmja1Aj8Ze5IpNvLv6GooNFfj3t0puX8tWcrwAoqRX3f4UxAgkyVIo6/7gVtAOCXrPi74w/v8Ns9vzlk0gkTHzCddB5raHWZtFiJf4slgBn/MWPEz8dGXUFwlop3LNtZ5xWeBLqlTkUF0NorQFDnYHaolpU4cGdfevidYSEi4qnrUimisRicS6gtRomvchJcQVDtejIkGlgxmGQuelMM1ZDwhTfwtqtkGthdj6f3CasQa3/F1ewFh6yK7J9P34QYZ2kqNXius3JyWH2wtnNbsRWu61odTThqnBkEfbXJjTGDSiVgvRrEcxGiBwMig6W5dEEMY0usPX1okAVGmp/bFze73KGJA1hRPKIdhod5GzJRX2okJr47pgaixI+Kf5M9dDn/zwvcEPTYU4RmIPfPBBo6HSgkOoZero77LkKRn0M2NXmVvW5I+Z8Nsdlbqm/4EjG6yv0LLtlGd2md+Nh6cN0CuvE85Ofd0nqWq/flJSU4E6kwnrB+GWu86kSp4qcv5bcX9sIexYccK53S0xH4s9isfiVUPDF6nPC4xMYcecIp8zjWkMtOZWCaHdS/Ekk/rMDPMPgzsL1410f8/KWl7nvrPsYmiQWluGdw7ntkH/Jjd8e+43s4yHAKDQaKK0TC9QotQ+fR1h3GOFZMe3X69dd12dtLixOhZ73eO+MDxQiB4lsoKjBHnfrFinuGSfKnIm/QBEX4x8dz/hHx7t8zfGe7BWKMNGF3lJFYuxZkHEXaFJa9r6mmLgKGsrt/3ZUTSjDxe+/jffjzNJMtuZuJTk0mXGpzddOvS/qTe+LfIsYsH6uDQ3C5lrhQ2RUU9RX1WM2mFFFqCioEXaf4SHhaBTuOwsd8/38cct3ef32e1j8+BsK3ywwk8OSASioyyUxXBB/+fmQluZ6/2W3L8NisnDeG8HN+AtNDmXyC5PpPLZzi+IH/AqZCnSp3vebtNr7Pof/BweeFWvPsB5tHpo3NFX8Dbl5CEMXDG2XTvrjvxxn9f2rsTwl7n9Sc0j7Kf5Gvg/9HhX21l2vFg3UTZH9A+x7DHPUCHKqI8T1mzBREFvthfE/gUyNptxu9QmiuVStUDNvlWcXi7bi0KJDfDP3G+Z8NgfFOQre3/k+R2q6ALeKz9JsEqrWln6/LBbhHiNTCdX5b9PF9guy7c89Yx2sPU8QBY3NtT6jaCMUbYKMO93XfA48K5r8e93bsmO3Bek3AKA+KIhco6yKGkMNh5ccZv+X+5n09CTCOwe2BlKdX82qf66i7x19uX3g++w42IeQbhZoqBTPk/bMifeEPh5sSMv3CqV26hWYw/sGb/276SqwGOCsL2yN32aFaDb67YGVHPxoK/fk3oM2LhAFa4GstVmsvn81qeNT0evFfCBEEwKdA28z2mJIZd6v5ajBMKcgOONpK5ThtgxvCRAREgFAeWOduzyrnAPfHGDITUOI7eWHrrJGVGZXUpkt+BHr+lNhjBZlgd03iiaXuZXCavwMgjfOKBD4W/HnZ7TFgseq9lPL1YSFiJuZwSA6MyFAxN+ZgF/PgbWzvO5mJZrqlNlUV8OY+8dwR+YdRKUHv1h4yQ+XcMFHFwDOir/6erBe521S/G28FBZ1dm0jp9DB8Hdg6OtiAmg2wO9XQ9YXzvvpusH4JdDtat/PK5GAOh61RihEfbH6tBZx8/7IY91/1lGZ4z1/JBhw7OKzWCyUlpa6vH6PlYmcq7QosXDSaiEhQSimxoxpwwCkcrE4G/JiGw4SeCgU4v8KUFoKlP5hs4yZ0HUCC4YuYFDiIDJXZPJK2iscWRo8hRPAvq8P0GXXj8gMVQxLHMGA+AG+ZfwpdND/UUi9zPN+UpkgvP9k0Omg3qThkPQB0VnYiKRQYfXpivhLHp5MbG//TeaawvGarC2uZe/nezm29RhLjizhvZ3vESJ3qKJUHYOd/4KS7R6v34BCoYOkaa4L1rGjxMJNETzr25Zm/EWpo5jTaw5X9rvSRpoD1JXVsfW1rZzacMrDuz3DF8WfRCpBG6tFKrdPRTNLhVI9UhXpbO1atlvkhfwFES49wnWD7iFR7qzKiNGIrozi2sDamu14ZwflG8XvXqNxbmbyB/x+/TZUQP4vzgSUuUFkk8Sd7Z9ztAYKHYx4FxI8q+ccrT4tFovd6nPHTp6LfY7s34PTTGUwmtnSeS5HEh/HJGsB69jSQlpEXxjyEkT2B30R/Dwajr7ZsmNYoYxw/9r4pdD3Ifev+4AfDv7AFd9fwbs73m3TccB5DeCpic4TNr+0mf9G/5fCfYU2Z4pYredntNX2spOf4oGC+vw1VEPpTuFY4gHJoYL4y6nMsdnve7L7zNmU06bnXWsRmhjKqHtGkTwsmbo62N/pLnbF/YOiGv92zLtFXb5wiDG28gvYFHHjBImv9VN45MEX4Mc0MLheN2q1QiUJgviTyqTtQvoB9L+iP9esvYbKeDFWqaUdiD/Ha1DbSaxfXJF+INY+41dg0XRun/mzK2i7gCoOtRokyJCaxS8wWCoxXbyOjJkZhKWEkVmayXObnmOb8SOgkZBfPQF+aKGlXG2uUPrv/j/7tqRG4q94s/O+Iz+C9OtbPvBT38Guf4Heg7Xy0bcFWdQOUKns1q3VDdUUHyxm7+d7g6LMDe8SznWbr2PGP2YwVnctkTUj0IXUwLcRIi/2TET5Pjj4X6g6Etznb12uzQHJmi1ukgniT9s5hu7Tuwfc1WvAVQO4fuv1ZEdmk8tWjNLqjpGl+heE1fWlzlRLg6mBwdcP5mHTw363fL186eUs2C3UnDbizxRFSAhIoocJJ6AzjPSDtnFGrcXfxF+Qkbc9j08nf0rmyuY2jI42n9aJq9UCRiKxF/X8ig0Xw/IhATiwH9HjFuhxq9fdrNaSemV2qxfRgYBjkczasS2Vei6IekXs2UJl4k71l3I+JDdOLI21cOITKFjtHzl0zUkyon8HfLP6rKivoKq+itytuaz59xpKjnSMAEZHxR9AQ0UDm1/aTN4fztWBY6WNxF+kWDxJJPD88/DaaxDescV6fkNkY323rAzYejNsurLZPrIQGSHhIUhkwV1097pyCEeGX44pJJbN129m14Jdvll9/sUhyFwJv9c95ZRZZbX6dJXxZ7FY0JfrMTX4wcbJBRzJoqi0KB6qe4iYWwS50SmsE1LHDJri30VGbXtnvlksbc+m8hOciL+yPXDof6KQ7gZapZbvLv6O989/H7mDOkZfrmf57cs5tPhQq8fiC/GXsyWneb5fiQubT4Dtt3Z4W+RAQSfL54KeLxGncM7dsRKjTYm/3Z/sZu/ne/12/gW7F6C77mKgkfhrYl/uEYdeElmTbcxWaxF2Pwi/ngs1WfZtuq5w1kJIDq6ipzXoHN4ZCRL0Rj0ldSU24q8WDbG9Y/2ai1NTVMOah9dwamNz0uNE0WlOR33L0aTHCdd6mbAWroNfp0BpG7Ohak+Je3tL56o5S8SPPywGPcDmZFHpmnwtzSzlm7nfcOA7700KUqmd/Gut3Wfi4EQG3zAYbbzWRhZZ3U/cwbo+sjZ0+RWmelgxTNi3BgIlm2HFYKFW8gDr+iO3MpfERFFcOe2hJn7D9hu4ec/Nfhtma1BbayEr9g326F60WRsGHDk/wMrhwobYG2rzRAZbU7LCEUnTBInvbm3aYliEe42beYxWCzIHxV99ZT3HVx+n9FjwbdPCUsLocnYX6hTiWSc1K4NTkHa8V+5/CpZkuM6dboqwDEiaIjLlAOpLYNlAOPpWQIbpExoqoPIwGpXoeD97/x4O3HDSpuA9veM0e7/Y2yySw1/oNLoTl/xwCZ3HdKbeJCaxUrP4LoeEAPHjIdl7I7oT1IkQ2l2oLwHkatGcPfZ7oXa3Qq4WjdgJ57R84N0XwOTfPVsrT9kK4xa3/Nhtwd7HYPkgVEoDskbr1pqGGkbcOYKH9A+RNCwp4ENQqBWkjEghvHO4bV2iCjFCt2vbtxHMG3J+FJmqNS4aUlJmCSvzxCnBHdOk1TY3GZ1S3HfNjU1hnWYP4dLFl6KJCSwBo4vXkTwsmXkr5rEseQSVmt1EFb8O3ydAyXbvBwg2Dr8KGz1EylRlwsmvRBPOmYCVI2GNqC9Hae1F0Ap9BVK5NODEr5X4UxqjRc0j4w4Y81VAz/lnwt/EX5BRU1hDzuYc9GXNJ/XWTCVr4RXs1j46XYDU6IoI4aPfkdH9Zuh5t9fdrAozg7yM8poaLGYLuz/Z3abiZWtQX1nP2sfXkvVbFuBQJFNFOtl8tunz7HUPjFvi276KMJhbAcPfdQ5v3/Vg64Jw/7iTm7uPRYLZI8EaGhJKlDqKRF0ixbXF9L6wNzftvInk4cktP2cA4OTbD9Rk1/DLP3/h+CpnIsGm+Iu0d02Gh0NUW4SktbmwYmi7dd+1FNb/a1kZ0Od+GPgMIO5Zq4+v5o+8P+g6oSs37biJ7tO6uz9QAKBKiqIyrjtKjdy3XDgr9jwCa6Z2GNIm2LAW/aqqnLdbrT4LawoxNSmibvnfFp6NfJbcbX7IynKBprZycpWc3AZxrmaZRalXwPR9otDUnlg9AZY0+c5bzLCkB+y6P6hDccr4K/gVdtwFlYdEESjH9wV/aFIoV/92NcNvc5/L5g2+EH8Lpy3kx+t+dNpmzffrEd3EJqz7rSLf6a+I6OHMX3yK1Secs+Wsir+SOudmmrWPr2Xjf33IbPIRmhgN9Upxw2ix1WfRJjj1tR8LwT6gy6Uw+CWRLwiimNgR1AwAxVtEs92p79zuEiIP4fidx9E/pCdGE2Mj/goiMrhqzTUkDfVfsazseBnrnlhH9sbmRNahfOEJqTakoA7xkgxRtgcK14C0lZXuA8+JonN4P7io1KdmPyfsexy2XOs8x20Ks0E0Lu15tHVjxE4oWZ0smsJkMHHguwMUH/RNhdvWnL8eM3ow852Z6OJ1FNUKciRW41nx1yL71pai6ijU5TlbrvoToRnQ/z+uMzwdYCUKagw1hMZWAMLq0x3aSyWWuTKTtwe9zfHVx6ms1WORGgAIVwWpqzBmlPh9Rg70vm9DCez8J+T6uPb0B3rdC+ftdata0+mcFX8lmSV8es6n7P9qf/DG2Aiz0YzFbLETRsHI+CveDMsH2fPi1AmgSfZ9XWOx2Jsl9AVgKBcxIO2FvY/C0p4ojHnI5aCr70G0vLOtMW37W9v5/vLvqa+sD/hQrOS7VXUYEoLInPRiXd4MEimcs1bUbQBMDeIz6jTbvTV5SxHWA2JGep5nqWKEojKYMFaDoQqVoo7upx+i76nXSI3oijxEjjxEHpT7rtloxlBrYGv2VraXL6dOkYMkJEI4LqVdG/DztxpVR+DUV+J52hRyrbgnKgLRveMbrMSfUSruF746zbQVhloDDTUNNhWwzKRFpo4AXVrHjIMo3gynvnTt0AYij3XjpcK+9UyAJlk8Z4AwnQy5UTgUlunL0FfoObnuJBXZFX47nb5cz84PdlJ0QMxvretdhTGq/TJ0z2D8nfHnZ3h7iHWf3p0HKh9wKe/sF9+PFya/YCvmgH0xGDCbz5ZOYDowwlXhaGSh1JqqKNDngKQHy+9YTuKgRHqe3zNo46g6XcVvj/zG2IfGkjo+1a74U9sVf62y+TQbYN+T0Puf4qHvqcDhCInEnothMQuLHHUClGwBixF6/6tl4+h6DWv2TkAmNVJX5znvq/DeQmRSe3d6IH2/WwpHxZ9EIiF9VDrd1nVr5jd/dpez0Rv1jO7kxxDoigNCjaDsgJMUF7Aq/kpLcVKH/XT0J25YcgMzesxgyWVBLAY4oLbWAkhaPgGoyxPB97Izzx7AH7ASf0PkD8LqLbb8ljhtHMuvWE6iLrHZ8yxhUAIDrx0YsMxUx4JkdUE15SfKOVkuChqdw5tYR0kkENFH/NVsJiEhoX2Kd7FjRdafxWy/JxsqG0mH4I7HKeOv0xyRHRA5ALIWiuwdB0tXKywWC9UN1YTIQ1A2ZoPIQ+Skjktt01i8EX8Wi4UJj09AE+t8/Z2fcT5R6iib3aEN3ix5/8RQaVUU13ZC3WTa6M7qc87COSjUrQgMcwGz0Uzx4WKqi3SApuVWn2O/Ea4DHq5NiUTi3+s3bqz4AXFdrpkqLM/OWef7vClQUEYKuyRDucfdUiNSbX93nP/X1trJIn8gYUACN++7GU108+fgsWJx7w01+1AszLgNul7p2WrTEwyVopBVXwyaRmLTbHTO6fOEkR+KznhP3yGpAvJXicIFj7ZqmI4W9maL2VmFDsRkxPDvhn872Rd7gj/zG21Wnz4Sf/5SIzldvxF94fws8dkFAtpOPtm1ahQaIlWRlOnLMOtygAjRuOYGRQeLKD1aSvq0dL8qar3BVG+ivrIei9lCcZ0olkmQ2IqqAUfkQN9IPxCk6znrXGcag1B7rhoDXS6zkxwBhlYLIYZ4ztmdyxcf69A0yJn+xnRSRrYxL7QVWHnPSra+uhXJ8+IeFBSrz+zvRbNFxUFB6qRdJ358Qfl+WDkUSa8HSEi4EUlEnLh227NJJuFcYdkmU6PVimxOxwbjwTcMJm1KGgqNf+Y3TXF89XEO/3iYkXeNtBF/EpNY7/ityLykO0QPhbHfOT/jdv5LfJ6TN4HKs2rbJUwNIkfQFRlk0gt3FE2KzzmpfsGg52DQc6j10KlE2GrGhIhiftGBIiLTItHFB/Zel/VbFp+e+ymnrjvFB50+oF/424SE3BjQc/oFPW6HHne4zmxsKBO21+oEJBJZ8Na/Rb8LQrLrPFvGn6GR+MvfdooVCw8w/LbhAY1YWvmPlfzx1h+Y/88McpCZtUi7XQH9PKjq2hMj3oFRn4g1iCskToYx34i1+pmAsfamxdBQ6LfrTSaMUxCvjadgWwEfjfuIKS9PYeSdI/1yuuLDxfx43Y9MemYSsb1jiVZH0zdyBIbcXqjVFth0tVDutsYiuZ3RHjWrv4k/P8PXYFVXH3aP6B7cM8p5smxdDP5l8/1A5Dlsv02EFne52OOu8eoUTlQfpLghG4kkg4u/u5jQpOB2xESkRrBgzwLUkaIKe8+oe5jdczZJoUlUZol9WvV55v8K+x4DWQj0eaDl77eYhVWaoRKm7YCzvhJqkJai0wXsN4LR7L3Dx5H0s1gs1FfWI5FKCAltX0Nui8VemFapxHXbKa0TuGgqvbzf5Vze73L/DiDxXDj/5BlDOjlZfTrAOvGraaihrrSOXR/vImlIEl3ODl5X4corPmPAjgIOXzudzi9NpVN4JzZe64PaZcS74ovQUYO9Awwr8acy5wirCbMJpDJkUhlT06e6fE/quNQ2E0Ke4HhNHv3pKD9e9yPlj5SDxF5wBUTBQp8vSDeZEqlUSkJCQsDG5REDnmi+TRkh7rFBhpPiT9vZnrNTdQyiRziTk40Y/t5wtudt56fLf2J69+m27RazBVODCbmqddPEhgbxp9JNb4hEInGpKOwV24tesb1adc4/K1QhFuK0J5FKLFgsXW23LHfEX8oI/xU9q05X8WbfNzEOHQEJU9FoYF7/eRTVFpGg8/Ga85K9ELDr12wS3/eYUaLzvL1JPxCWX7NPt+i5I5cLksZYVcvax7aScU4K6VPS/TIcuUpOXB/XRcYTZcJiKkLi4/O8taQfQNd5cOJjoYAwG2HVWcIa7WzPlo42RPQVP94wdZtwOmklEkMTkSChwdRAcW1xM1tNiVRiszqqLa5FHa32uLi3rgVaq/hb9+Q6Sg6XMPuT2QxNGsqCIQsY22Wsx/c0tblvK5pdv1KF+GlndA7vTJm+jDrlSaCvaFxzg62vbWX7G9v5R/4/Al6IdkTGrAwyZmUAsPIrkY8dQmgzQrlDQKa0N1S4Qn2RmJfV+zmf8PhH4s9u1zR7SaMBqUSGypCE3ASaaBh2s2c1aKCQMDCB3nN7c9bks+i07GZO1wwNPPE36L+C8HFFDniDKhbiJyIN7eZ8/bbnmih5ui2mRKOBPcq3eHTTUe7VzKdvXF+ShyWTPCxwrkF52/PY+spWBswbQL1FLEisxF9ICMIpSRkpGrBbAmMN7LgH1MkQMwJCe8BPfcSzaHLjelURLhoGla1w4arNg0Upgiwa+r/mr5fvg5XDYMBTrasftRGO14FeD4Wbsvn8vM+Z9f4sBl07KKDn1sZr6T+vP/vi9wEgs6hIVqyF3z+Anv8Q2cIdETIPN48jr8Oef8O0nUgjBwZv/Zv5logO6nyRrTnFIBGTl/LDhRx/ZQvdz+seUOIvZWQKxjojFYhGGZlZ07Ez/uReCr66ruLnDIROB8mll9NfBuEqkHWXMfWVqXQZ678aYHSPaC5bchnRGeK+ePXAq+leezVPr4KIvhWQ9alonjgDiT9fOSO/njPoZ/yTw2TynC+RsyWHI0uP+JyR5Gj1GRBkvgvHPw7Qwf0EqULYxzR49+w/t9NsOhVdj6ROFMS6TepGbC/P3a/+hjxETny/eMJSRFdV5/DOjEsdR/fo7rbOtVYp/uLGCE/41FZ2tUikIng9foLo/lLFiGO2AqJgYGmRtL/4YDHPRjzrVwuy1qLewSVEpRLX7eG9hzHUGwJ/cqutirytfq/Bg5PV574nYXE30BejVTYSf4Ya9OV6fr7nZ44sPRLUsYX2SqYyJg2pqorsymyX2XRucYb8/gOBsMamz/cPfAIXnHTfjRZEOCoRkocnc+7z53IqShSfnRR/R16DX8+BmhOAuH6PHTvm9fn7Z4dTxp8jBj8Poz52SXyEhYgvQrm+3Gn7s5HP8sWsL1o9Fl+sPn1GyXZY2htOfeOHg5150Gjgjem9uHHwbU7Prmi1WAiV1pU62fJaLBYaahr8Ehyu1Co5++GzqUtKs43lxSkv8unsT4nVeplbmY0id63qmMfdAnL97nkYvo0AYxUMfgGGvua/Y7cFEolPz51FhxYx74d5fLjzQ8CaZWVg+/NrObbS8++zJagrraO2pBaLufl35VSFUPxFyzs3e80JpTvg8CvCwry1COsBA54UvxupHFQJEBLj/X0ANdnubZSaQhnZpue+UqYkXidylNzZfYIoIL/W8zW2v+k5c8a6tmut4u/UulMcWSLmXNO7T+fNGW96bVTzt9Wn9fo1735E2FubGvxzYHf4ZQJsv93rbpf1vYy7R95N9wTRNFTiIV68/5X9ueDjC1DqWkGg+AlltaKQqZYGyebTpIdFnWHv476/x2yCyqOuP2NNClyQLcgFf2L/U25jKSQS+3raH6rZtmDQtYOY+/Vczul3Dr3Nl6A2JAenIN0a0g+Eqmz8T5g6Xyau3xNfiqypDgKtFnKjFvLx0Rc5XHw4KOcccfsI7sm9h/j+8facTZOD1eeJjyBnUcsPLNMIu/3i32HM1zDgPxAzGqKG2vfp+xBM39W6tZgqDjpfBFFuSDRVLPT5P9EsGUxUHILM95DoT2MIzaREt5ajhaeI7RPL5BcmByX6Jb5fPLM/mU1hd6GIl5pVREn3CQKrwYMMvL1hrBFW+dVZzV+LHgEZd4MqIbjr34w7RMyQVEmEKoIZPWbQWzYLCxZCR/fj7uy7A974PfDqgUx/fzr1crEYCpFqkR9+JnC5wm1FQ7nI0K1tQW2qI+P0z2J9ZaxrZlWvS9Ax4vYRJAz0HxGtjlTTY0YPorvbGyKsNQ5JSDhcUgdDXvbb+YKJ9qhZ/U38BRlbX9nKFzO/cLnI3pKzhW2526iqt4cuBVzxd+BZOPxygA7uJ0T0hQsLRXixFzww/EkGnHwXdcVA27aG6gaM+gDZzriAodZAXVmdy8+4TZ+nXCs84bVeijCeMOAJEbzupQPfI8wmZmtHcu/oy70Sf4sOLWL8R+N5aPVD6BJ0DJw/kMRBfvK0bwNsDw2JXZGy+fHNPKV+ipoi++qxuqGaw8WHqTf6KU+gNg9+7AonFvrneEGCk+JPqhTFM3ODTfFXa6glNDmUazddy4g7RgR1bKnXTuTEoNnIVGKRppL7UM3adgsc+6Dj5D61A6zEX2Vl89dWH1/N0+ufZlP2JqftZpOZRdcsYsOzGwIyJkfFX1zfOEb/YzR5WjFZdlL8ZdwOQ16BsAzbpqqmYYXBgtkAW24QhLgVBWtEITzIi0or8Wc0gsHHHoYIVQTQnPjrfXFvuoxr/QLOm+Lv8JLDvDXgLbLWZtm21Rnq+GjXR6zNWutMWhmrwWICSfuT0+2BEJWEbw88wPpTl9iK9iAUfx9f8DE/Xuqck7jstmU8rXuaupK2h26oo9RMeGwC5bEix7JFTUt1p2HdLDjyqtdd/X79KqOF1a2pQTzoW1scDQTyVsAhF135DthfuJ/P9nzGhlPiXqvTgSFEx8RvFzD2Qf8V79Y/tZ7nYp6j4lTzTI7cxtyoeKWX+0D+KvjjzrYRf00xbrFQ5fuC36+CH5KFotkbLBbx+z/5dauHZn0WZVc0z0W0IrJbJOlT02mo9kyCtdXq88qVV3Jv4b0teo+/rT5BXL+S/FWQ9Xng1X71hT49W+8bcx8vTnmRs9KEoqOiQjwbXaHTqE4MuGoASm1w7xOnd5xmx3s7qCuto1wvrkFNsIi/+mIIifXdThfgwDOwtAeU73a/j78b6kYv9Jhpr9XC4aR/c9ev13Oq4hRvDXiLxfN9zzQOBPza+OQORb/D5muFK1IbUVVVhWTfI7Dbu41uQFG2C36dDDk/otEINQ+I5lKAXR/t4r/R/+XUhlMBOb1CoyA0KRSZUmYn/owOVp/nHYBxP7o/gDtIJHDefpiw3L5txLuu1XmtgVQuCEUXqlhA2MAOeKLVzd6tRuFa2HoDlO/jcOJj/N5zPD8c+YaILhGMumcUcX1bYWnaSljrOFKzivzQW+Diaoj1Y4SLv1F1TDgfnPik+WuJ58KQF21Za0Fb/0YNgeQZIFUQqY5kyWVLuDbsCyRIMEhDCEsJQ+4tD9oPsOb7AWjlWjj0PGR9FvDztgrFv8MvZ0PeT65f3347fBfXsUloR5xeCfuegPoidDqoUO/k94pvA9acYTY1n9c7Na/JVMG1Lz7D8bfVZ5Ax/I7hpE1Jc2mfdeuyW/nj9B8suWwJM3rMAIKQ8eerjc4ZAierM2D3p7tZdNUiLltyGT1m9AjKGPZ8toelNy3lypVXkjY5jXf+eAej2cicXnOorU1wGmcz1OaK8PRBz9tzTgDqCoR3e1tIP39BKsMi1VFv1Hol/sr15aw9uZYQeQjqSWrO/6B5xlR7wLHwYV2jRvSKoNeFvVBH2f2PNp7ayNSFU+kb15e9N/sheLfygCDOvEn/OxicMv56/9Nmc6JVisVXTUMN8hA5nUZ1cnOEwMH6HZQqxV/Uci/+VfoiOPWtyLrsyMHeAYaV+FObsrAc34AkYUJj9hF8ue9L3tv5Ho+Nf8wp21Iqk3J48WFqCmoYc5//F5CulAibrt1ERX2FM6Eb3lv8dARIFZC3DML7AI1Fk1PfwdHXodNFgiQPEhyt2+rqQGGtv1ossOtfIFNDf+fufmtOW1Pib9a7s1o9DqPRXmB1V/gy1hlpqGlwylM6Xnac+YvnEx4STvn9DuOJHw8zg9Px3REhkcCiYw+j18NldRARIbYrZAquGnBVs/1TRqZgqPavet3qViAL0ZNfXU6kKpIQuZeqpiIMRnwAYcHLWLYh/QYxh1IGqYjeEhx7H7K/gx63uCVJojWiu9UaZK/VgkUqQ5YYj8ZHIZwvSBmVwuAbB7vMXy7UC0VbosYL8df1Goge2X735OQZws7VF3tEiQT+uAPMDdB5bqtIiicmPIHRbGR4cnOrYivUUWpmfzrba4ZHWxV/gO0emlWeRVhIGBGqCI9Wkf62+rTCPGkdMmNp4J0Uztvfot3DwoRdrtEomtdig2sC4xGHlxxm7aNrSRmVQkUj8aeVB+mepUmBaX+07D3xE4Q9niu73ML1UJsDKReAt3l4SxDt2bpTp4Nc7UKOHj/BP6quRxmqRKYKfpPQtje2UbivkIbbG9htMGKQnUtISETgTli+B45/CKltiKI48F8khmpgDuYx3yNrKPTb8FoFUz2UbIG62ULlXmNvLgVQR6uJ7x/fagt6b6g6XUVDVQOR3SK5cciNzOgxk3tuE7kIISG0zc46JBr+uAtU8c3tNquzxJqhy6WCXPmzIGkajF8OkYNQSUU2WKU+uNLcrLVZ7P5oN4oUhciEs6gIUUk6fh1G2xkGvQCxZ7X3SDzCul6vrTRQmlmFOlptizoKBNY/tZ6843nQCSRmORqVAqbtEveOjoiIfjD4JaHwdQVNJ7FGkgc3lqrVyLgLul0LqgR0OjiW8DzrDZ8z+OgLdFV15f1R79NjZg8mPTXJL6f76eaf2P/Vfu46eReqCBXjPhrHwbyTdNd9RpQmHYpPQnivv8k/H/E38RdkpIxIcZu/kl+dD0Cizq6IsslnA2X16S6ku6OhcJ3wKO9xi8fdVCoLRmk1DSYDRmMUcX3i6H9lfzQxwctSi+4RzaDrBxHRNQKAx9c+Tm5VLiOSR1BTI4g/t0SusVrYSCijYJiDNVXm27D3EZj8O8T4JzC1Ldge/gtvboXRXpqlfOmMbg+4Ihi6zOlCv379kMrsBZPjZccB6BrhJ//thHNg5lH/HCuIcLL6dIBG4dyNaWowYTaaAxa87gpHX1tJzMkYjKPqwOCD4k8VKz4DU9vVMGcyrBl/vWLWI9l8FYz9wUb8JYaKZ9DpqtPN3nd3zt0B64p3JOR/uf8XTqw+wfz1822qNABqTokcqA6QIWTDeQecCYY+D0LnC23dmMGCVCp+d/X1gqixkrtIJKJLz2JuRvxZf7dldf7rNmxwELm4I/76XNyHPhf3cdp2olxYt3aNPDPzDgIJjUZcH77Yaw+YN4AB8/wTFJ+1NovfHl2LzHA2RHZlf8XvzP5kIj1jenLw1oOe36wMh7T5fhlHiyHXCGKnI6Lfw9D7X3gyXXG0cQX7GqCiQE/VaQOhif4pEvS+sDe9L3RN2D0Y/Qdfr8hlwAwvTIk6Xvz4G0deh6INcJYXy+Fe/2jZcYe9KazRWokp6VN82s8b6Qdtz/jLWpuFOlIUw0e9P4r86nx23rSTgQkD3b4nEIo/QDxn2vB79TcsFgtFtUUU1RQRGdmHoiLRvOaK+Du85DDLb1vOeW+eR/fp3YM2xv5X9idlRAoRXSJIN89g0p6TzL3YB+VqeyF2tHuVTOa7Im9nbgDUJ/piQeyHNM+OEjbI4gZZ3VDNtRvap6Evc3kmx34+xitdX6FIWcTZyj2BJf7Sb4Qul4iGrtbi1LdI9PmQOkcUn2V9vL8nkIgZAXMbla8akJntOfIAGTMzyJiZ4fbtbcW6/6xj+xvbubfgXqLjoglTRKNyvF+W7RYEnqYVOcoWM5z6GlSJgvgr+l1cLxl3CaXjwechYmDrib9jH0LuEhjzVfP10aGXheJo9OdiDRwsOGSNq2Tis6zW11BTWMNnUz+jzyV9AtJE6ojig8Xs+mgX0tukECMUfxHsgVJDxyZZlRHQ6x7Xrx18XtRFx3xDUEv5R16HXffBxF8hRjQ+KVVGLEipPJDDq9d/wrTXpjH8VvdNUW3FsZXHKDxUyF1vPMPyFSZRx2vN9RgsaFKg513uX+/9r8b1wBkCrb3BPzQUFCZRaKiqr0KmlPkcZeYrYnrFkDwimZAwMWE9WX6SIsNJelgUZOiWwM83wvhlosngb3jF38Sfn+HLQs8VzBYzBTUFACTo7MXCgGf8GaqEAslTiGxHwNG34OQXwsbAg03lN0c/ZsXg+cRWTKWubjmJgxOZ/ens4I0TSB2fSur4VNu/y/SioBqhivCe8RfaHdJuaF6wihsrtneQSYq1U9j6/1m1SpBCF1/svF+n8EbirzIbi8XCz//4GYlUwuTnJwdxtM3RtONZIpHQqVOnZtfvsTKRpZMWmea/k5+BuXJWhUltLTQUH0ZZ9BMkz7RZfVoXZU+HPU3a5DQu+/GyoIzLYrFQuHQr4THdKVNawABqhQ+LYGU40AGVIEGEQiGI732F4yjt+z1RMXaL1qRQoTY+Xd2c+AukFZajPZJRb6S2pBZZSJPO7a03iU7nC3Js15K76zdoaKoq0iQ5K7aDCLVa/B6bEUQTV4uCRRO4s/rc/PJmSjNLmf7a9BaPwUr8SSQOqkMfcKKskfhr2miR9SWY9e5tjP4CmJb2CgMGf0l99c+AfUK4KXsTmaWZjO40mvSodL+ft660joI9BVjSxIdah5jPRKmbF2Bbi3a/foMNHxrumir+rGuAXbe8w8l4JQt2ebe+bytqq+VoGroQ7a2Rtr6kMTvPz+kRZTuFmlpfLDKp/YUE/3Qie0NFdgVLrl9CxgUZDLvZtWqpaUZKS/HVBV+RODiRK3+5kqKaIgBiNZ4Lu/7O+JNIJKSGFSMpWA3x4wK/nixcDxUHoPtNHnfbXbCbQW8PIk4bx7yoAoqK3Of8KXVKtPFapPLgJqBEpUURlSbupQY9qBs60zlYjesnPoWGCuhxq3/WJX3uh5TzQeHngkXxVvh5BAx8Bnrf1+xlrRbk5Xbir71w4ZcXYqwz8tx7zwEgNYcE1upTImmbAg1g3GIsEhVdCg8jMdWBLFDFppZDq7VbfVoVf4FG+tR0VBEqlKFineOYqRyiMMHigdD5EhjzZSuOLoGMOyGmUcFVfQyOvikagjtdCDMOt42UK98DuYtBX9CcCKnLhZKtwhavPWCx2Ii/yvpqpAop+nJ9UGJ4Bl8/mH5X9OOjdz6CapBaVHQpuw9WrYNL2jkYtLWo2A+nV4BEjoQgzp/VyRA7xqbo7va/bpwoP8FY9Q6Muq6cdd9ZAY/zuWr1VZgaTOw5qCAzH8LSq6C6WIytI1n6/1lhNglbUpkKnU6HvJH4q6yvRKaUcdvh2/x6ulF3j2LU3aNs/7Y2QyqN0VTIhsPAZxsdl848tMea9++MPz9DKvX8K30h8QUWXbOo2faS2hKMZvEAjNPaOyYDnvH3QyKsuyBAB/cjev0TzlnnVeURoxW2ZQZZmY2Uak80mBpsE9ZIdaTnz9NQCbXZwvO9qQ97/AQY8U6HUbnoVNVc2udxeqoWYjTCG2/Ap59CQYHzfilhYvJZ3VBNRX0Fx1cdJ3N5ZjuM2BlNO54lEglrbl3Djnd3OO1nI/6iWkH8Getgx70i3Bog+3vYdtsZGfCr0dizumry9sDOf0DZDuK0cfz3nP/y3Lliodt/Xv825YK1Bp1euoeT/WciUQimw6Pi7/Ar8Mc9HdcSIsgIC4Oi2s4UhcwWKrpGWFXneVXNv6vlJ8s5uvxoQBZsjgXJqS9PZfja4cz6chZPr3/avlP8BOg0x6lQJZVKiY6O9vr8DRgM1ZCzRHTt6guhbE/7jAN7M0Mz4k8V67IwbyP+6sudth9beYxdH+5q1RisxRKl0n09cc9ne9j35T6nbTbFX1Pi79DzsOffrRrLnwXRmnzitFk01JQ7bX9y/ZNcvehq1p1cZ9tWsKeAxdcu9ksOTq/Zvbjp2L+oiM9AIoEasyD+rBaxHrH3cfgxHWo8K/7b/fptD5j0Yo7gBlZitaTWbvUJoD1rIH0v7eu3YXx/xfeseXiNy9essTGh3sSFy/rBKv/lDtow8Fm40Avpl/kerJkOjXmEPsPUIMijViCvKo/P9nzGdwe+87ifKlzFyfUnqcx2EaTbiLZm/E1+YTLD7xhOub4ck0V0Wsd48YL1t9WnVColIv9jpL9NAWMQCqmZ78K2BR6vH4DUiFQACmsKCY0S67DSUtf7dp3QlRu23kDaZD82+fkAxyzbQFmwujkx7H0MDr3YctJv14Pwy/jm28N7C6cDfyOsB6RdBxH9Xb7cVPF3aNEhfn/pd/+PwwuUWiWaGI09S8yi9Bu57hIl26DCi+reG9SJSFWRRO65BunKwf4ZV1tgqoecxVC6Qyj+TI3NpY2uMhXZFax5ZE3AMv4yZmYw6clJKNQKFh9azH/WP06pbiNyOcjlFuj3eOudBCQSQVxb6zsp58PsfGGNK5WJ73lbogEG/AcuqXetfhr0nFBSKoJsJ1i2C75Sw4Fn0cjsjcLqSDV3Hr+T8Y+MD/gQpHIpIaEhPHHuE4wqewVNfSqVcTeKRoKOjl8nw4ZLmm8f+SFc2gBSWXDnz50ugAkrbM1rCpmoSRpl1Ri0EZzzzDl0Gh3YyBepXIpCo7A9L/vG/AI/dhNq2o6K1ZNgw8WuX9v/DGS+E9zxtAWFa+D7WMhaiE4HcrOd+As0GkwNVDWIhYnCGIVePUCoJTtCDFYr0B5r3r/QKjs4MJncS1xNBhNx/eII69S8nc9q8xmjibHdSCEIGX+pV0DCuQE6uB8RNUio3rwQX5HqRuJPXmoj/tb9Zx2Lrw1e0Pem5zfx/RXfYzaZnVQU4SHhnhV/2d/D4lTIbsxdrM0Vf5b+ITosOhBCNCHM7fMUA8IXUlxsz3NqSvxpFBpbASu7Ipsbtt3AzftuDvJom6PpAru2tJb9X+8nd2uu037HStug+KvNgUMvwMZLxb/zVkDmmx1fXesCEok9569IMgHO3QgJ5xAaEso/z/onNw8Tn+msd2cx+h/BC8uWSCQ0KLQYQ7SEqULpHdub1PBU92/IWSSsVaR/d4WBvaBbWYkoBDUiXids2wprmud9bH9rO59P/5yKUxV+H09TJcK+wn0sPbKUDdkb7Dv1/hcMfdXpfSaTiUOHDnl8/gYU9YWwbpaw2jn8CiwfAIUbvL8vAGiac2uD2QgFa5qNq1dML+b0msPIZGcL6Qu/vJB7C+5t1RgclZvusO4/61j3xDqnbW6tPkd9Cme1psP6z4NfS57imsV5VJmcizrWIn9xbbFtW01RDbs+3EXB3iYP5FbCOm9Rq6GssdvSJ8WfIhwkMpdKU0e0+/UbbBSsEcWwEx+53cVq9VlSV4LFYrEpwxSTzmbM/f6zxjr+y3Hytjdv8Fh/cj1fGi4nK/ZNz44jFovIMk0JQH5zSLT3ZreaLChYDfIWKlV+PQdWjWnV3HpPwR7m/TCPJ9Y94XG/kLAQHqx+0GPWSVutPgddO4ie5/e0qf3CQ8K9Zm/6cn9uCUwmEydUF2Ae+pZLK0a/o+fdMHEVSD2bFkWoIggPaVTjhwti2B3x115YfM1i/qP6D4ZaA7sM33Ag5V721a4O/IklEpi6rXXPVX2+WJ8aHbprLebANdQpI2DEe26tvETx0U787Xx/J6vvD8LvsAmKDxdTdKiIepOV+POi+LOY4dD/hKK5eEvLmxd+nwfr20i0NpRhKt5OkXYy5rTAq8i9wlgjmtGPvoVGA3Kzc8ZfdX416x5fx6mNgSH+HLHo8CKe3foIpboN4nOUyqHfv/1HbitChUW2RCpI3Ibyth1PrvV6Tww6lFEQPwl0XdE0KoGtDkHBQkV2BXnb87i0x6Wkl95OiDEOQ/xsyLg9qONoFSxmwOJxl/acP+uU4jM1Satt6/dAI3dbLsd3HGdP8R/UhGRSI+kqcmcjBwZnAK2BRCZ+XOHwy0J9f6ZA1w3Sb4KwDEJDsSn+KhqJvz0Lmzf1tgUr71nJjveFKMMaSSJBgsIUHpwmqQCiPa7ZDvaE+HNDppAx7+d5Ll+zEn+ONp8QBMXf8LcDdOAAwGwSdl8eAnmtxSiDrMxW+MzdksupDaeY9d4sJNLAy2pPbTjFsZXHmLNwju0mFR4Sjkwq8/x5hmZAt/lCRr/lesj6HM5eBGtniJtsk2J3e0KtUXD3yu00hKRzW2Ojd4Qqn+KCcMD5TtwprBOldaXkVObQL75jZEo2VfypIlTM3DKTXj162faxWCy2jL9WKf7CuotQ67jGX9Dwt4UVjpciaEdFVJQgdourYujRz4+2W22AUW+k+kQpsoZQxsXM5ZULvXRiTvwF6vLPSLvVQCAsDFTyagZmdgbzHFFcwa46L6wpxGKxONkR9Jrdi6i0KNTR/p9xWQuSKhXs/HAn2flCKdQ5zHs3lz5Yqw5X0HaF4e+K7BuTXmS1usvBCTCa2jDbYYE104RiMm65beukbpOY1K15YVoV3vpWdUfFnztc9OVFGOoMTtvcWn2G9+KvDndKzhh1c+Kvy9gu3F9xv82uqi3I35XPoQ1lSI1paDRKm3W5T4q/nneKHx/QrtdvsKFNhS6XifuGG1itPo1mI1UNVWi1YnHdWmWYO9xbcC9mY/NMsR2nd3BQ/gWJoQZCQz00a0kkMPQV/w7KEZVHIOcH6HGHzV7KCQP+A/0eE4qJlqDbNVB3Gsz1IG1ZBni8VjTGWOMZPMHbmsO6FmirQ4m1SSdW69kqzmi0N+r5U41UIU3DknaB/w7oCVGDfN61S0QX9hTsoUGTBfRyS/zVV9Wz/a3txPePJ32K/y2T3SF+YLzN0vwYKzme8D5HaqKAINjRKiNtOU0twoj3m8+ha04J1UXff0P/x/wzPkeYjSLXquuVzRRNTRV/F/73QsY9Oq7Z3DXQ+P7y76ktrYVrxL+9Wn0efgV23C3yl08vh/7/gb4P+X7C3g/ijRTwiqNvIdv9IKVdFxKV4UJZFGwoQmHUJxCagbYCuhTdwuSky/j32eK+Ftc3jlv234IuMTCWpL/c/wuF+wq5fOnlduVmIC1bqzJF5M7PI4Xyb6xnFblHmPRQsh1CYiC8p/NrWV+K66apm1Sgoe0M45eKvyo/Aj3UNqrC9yzcg1KrpOcFPT0coO3Y/tZ2Njy1gVsP3Up9vZgvB9SC15+Y9Ivr7XkrBKnaeP8O2vy5KhOOvQfJ50PsKBvxZ5TVoC+r47Mp39F9RndG3D7Cy4Faj69mf4Ul1sK9F9xLWLdBnCvZAYMHBux8fsHEn92/NmUrWM6gpkddNxj+FgBao534K6sVxN/ax9aiUCv84kxiNpnZ/PJmes3pxeDrBtuiD1REIkFGf+M9sGIDTP695WuAvyj+Jv46CLwRfwHL+DtTUJsDi7tCj9thyItud7MWowzyMmpqLICEC7+4EIVWEbQFwKWLLsVYL1bWtiJZoxLRo+IvdpT4AYibAEggLANSrxQh3h0IajWcquiLVmuhoAASdMf47zlnUVI0C3CWrHcK70RhTSENpgbKTpRRerSULuO6IA9pv9uPK0sdqUKKUmcvlBbUFFBjqEEqkdosgzyiJlsspBU6MBtEp3rSVPvrEol4YJ6hsCr+SksRNlmYQabij7w/qKyvZETKCLY9sw19uZ4pL0wJyphKjpRQ89+3iOsxDrV6vPc3SKTtlr3WEREaCnqjlnLJYGJDu9u2WwubdcY6agw1tsk9QPLwZJKHJwdkPI5KhJ/v+ZmGzg0wx54VSsUh2PMQpC+AxA6kVJdIIP16+7+j2s8yya3Vp1QBI94VObI+oDq/msqcSuL6xbX4Xu2LoiRhoPNcx2KxuFb81RWI3Ia2WCD9CRChKWFM51+QV/cF7HkGVoLIkfiTKWXIlP5ZBO36eBdbXt6CbNJdgvirc57T/I1WQNcVzvrc4y4ahYa8e/KIUkcRIg+xZ8Gt28HCXw9y4RcXtomcd4SrXLOTFUJ9om7o4t3qM5DIWQS77oeIgZDkZl7RmgV/2rWtHpJVEV9UU4TZYkbqIduw+HAxWWuy6D23N5ro5hN/twptH1BxqoJPzvmEoTcPpWhqy/L9wI/En7Eu+IUriwWweM2VTI1IZU/BHmoUWYD7jD9Tg4lf/vULQ24aElTizzG/ps4sXBQi1AEO+as+AXnLBYmmaMW5XK6lzcICMVBZO6e+ht0PiFy02mwI6wVDXgIaM/7M4iZV3VBNbK825KS1AUNvHkp1VTU0Op55Vfxl3CGIkZiz4MTHEDe+ZSfsdlVrh2pH3DjMfR/BWB8Epa4vkCqgq2iQ12hAZUhEW5tIbGODhEKtILZ34D7fsmNlFO4VTRR6o7hZyiwq8TnW5sHGiyF1ntd8UZ/x80jQdIYBT0NoG+87+kL4ZazIERzysn27xQKbrxFZgnFL23aONiBNNZSemU9x9rAeAKy6dxWR3SIDTvylT0lHqVOyrWYbeXIdYZKRxO4aBzkDYPibAT13wPD7PHEPPHed9339ibo8OPAsqJOciD+TtBp9PWRvyia2T2Dvv+MeGcfuit1QAzKL+swhcd3hDLWpBJDLQSMTz96KOmHBOfuT2X5bf0qkEv5Z+E/MJtGcaM33U5nF80opqwVDxd+kXwvwN/EXRJSdKGPnBzvJmJVB8jDnwunQpKG8OPlFEkOdQ1Gt9i8BIf5Methyo1AApM0PwAn8CFUCJE2HCM+LCqvizyIxUVxVBYQ5kTnBgrVQaiuSNRKSbhV/ZpPzjavrFeIHYOQHgRxqq6DRgFRi5P7h05AWjqCw5jEOFI2hInwiTaeuSy5bYvv7yn+sZPOLm7nzxJ1EpEYEdcyOaGopWJFdQcmuEupT69FEiEqMXCrn0XGPUqYvQ+ktMNhshGX9wdwAc/Lht2mgSxc+7GYDrBwmgofH/3TGqs2sxJ+h7AR81U3kbg76L1MXTqW4tph9N+8jc3kmFScrgkb8aWI0mMeeTbUh1bvkv+KAWBhFj3CtHPgLIiwMQMIKwy/M623frlVq+fnKn4nTxnnOTPQzHK/Ly5Zcxt2/3g0WSAptJGtrTohCcHIAbOXaCosFijdB1NB2tfN1S/yBraDSFBaLhVpDLVql/cG0+X+b2fjMRm4/ejtR6S0rCjU0iD/dLchMBhNmgxmFxtnG79u533Ki/ISz4u/Q86LTf8YRoaL+iyJec4JLzrqUPfqncCT+XFl9giAFTu843eaiysBrBlIbkcwfW7RoNHBKLxZeXhV/p1cJJYO1uPk3WgzH9YB1zmjILyFrbxb6Mn2bib+awhpyt+aSODiR0CRndu9kuZ3487j+KFgj8kl63xcYq6Uul0HkIIg7u/lrDRVw6iuIPbu5wsFXWCzN52QWC+z4h8izcXFeK7lmspgorSv1mKl3bOUxVty5gqjuUXSb1Lzxy71C2ztMBhMKtQKpXGqz+vSm+LM+Y0VmVcvP6QqSzDfpd+gRSPkNYluhIGspTn0Hmy6D0V94td2zWr9XSDxbfaoiVFy3+TrCkgNMunmAvpExitKGB/ZEx96D/U+JTLGEc1r+fmMdZH8rFETxE8Q2XTcY85V/x+mI+AnC8rvzXLHWUth/R1ot9Mh7lNkxD/KvsyKxWCwYagzIVXKXTQ2BwuDrB4tMVhF5jtQc4ppct95zJFKRVw3Cwr49EDsayR93kmDeDXSgZjpcq6EtFgu1RbVIZBKXjRRtxdxv7K4xNstWcyPxZ6oTiqf6YjfvbgX6PCQUem7m5i2COkmoRpPPa/KCBcZ+3zqS3x/Y+S8ITSct9EbS8/vSv3Haf+EXF/rFlcIbupzdhfjR8aifVEMPmLKjAqm5VtRrOjoK10Ppduh+i/O6ctgbLbc39weihsLMo6ASzU82xZ+0Gj1qHqh6IOBDGHLDEA7vPQzfi2tzZPh/4bd1MOabjlvbqSuA4x9AzEj7MxNEfbA2F1SxHt3sOhx+v1o88/s9QrJkCP2z3mXBeNGgnTLSRcZoKyGRSNDE2O/zcqmc4cnDKc0S58pNeovEoX473V8Cf2f8+RmeghpLj5ay/j/ryd+V3+y1XrG9uHvU3Vza91LbNoslwFafpjqRd1WyOQAH9zOkchi3WAR8e4BaoUaOeDgWVgnSrTq/mmOrjlFTFBxf8azfsijcJzrGRqaMZM3Va3hlmrBCsk5gm32eux+AZQPFw+EMgFoNMZoc+sX9ila/DbNFyjMbv2VjzqUe39f7wt6c99Z5qCKCRya4grUobl2UHfruEBvmb6BoX5FtnxhNDI+Mf4SXp77s/YASGQx+QVh1IAF5KMhUjYs7GWARnVJnMKIaa/8F5VGiGBcxABDKBBDh6/N+nscdx+8I2phCk0KpGT6BqphUvjn9NL1f783Lm192vXPmu7B6whn/OfgTYY3rwEoXmcznpp3LgIQByJtkRtQU1vBaxmusfsj/GSqOFrydx3QmJykHcCAZkqbBJXro4mxJJJVK6datW7sEJduQ9ZnIjNpyQ/uNAR8VJA55jnlVeSieUBDxbAQWh+3pU9KZ9MykVt2rvSn+Tm04xVPap9j+1nbbNolEwrlp53LjkBtRKxwWb9EjhS1faCvslv9E0Ct78MyGr8lsuMhpuzvi77dHf+Or2V9RXdDK8LBGJAxIIHxMPywyORoNjOsyjiv7X+ndtjtvuci4tRi9nqNDXL/Bxp5HYLtvNqhgb/4rG3IOD9U95JfGqZwtOXwx8wuO/Xys2WsnyhqJv3oXir9DL0Nx47qh4iCc/FLYlQUC2k5C3e2qmaJ8L2y9CXKXNH/NF+x7EhYlQ30TNqh8Dxx+CX4ZBwebO4woZApbk2FBtec5e8asDK5YfgVJQ1w7DVjv13o9mJs7rnpEVFoUC3YvYMTtI+gV24tbht7CtHTXOWhWNLW59wckmmQsUUORhvXw30E9QdsZYsf61FBgdesoMWWJPxsVf0VFsGaN3fZUKpOSMiKFsJTgFsd/f/F3fnvsNwDqEYq/GF2Aib8+D8KozyB+YisPYIHfr4LM9/w6LI9QJwqFoiwEZh52Ihl1OggxxiKrSUaj0LDh6Q08Hfo0BXuCv57WKrW8O+0z+p94Hyly13bnB56FzdeKxoXWouA3WNoLclp577PCpAeZmtCw8I7z/F3WH9ZdgEYDVaqDbJA/ytvbRSyNqcHE8/HPs+LOFQEfhlXxJ7U0ErihaaKxtyV2rN7Q827/kH4g6mR9H2regCORQvL04Nt8WnH0Tcj+3lZvsT6DUsenun0u+hvWzxKEgtM8ZReMfD8o524TTn0NO+6BhiZzlM5zbZmnQZ0/yzVCmaoQk0KtQhQzTbLqVrkWtBY2Na5ZTbTiiMh59pYH3Z4w62H3g3DqG+ftVcfgx66w/5n2GVdrcXoFFG0EIEHdhc7F1zMw1N7wb2rwjwOEvkJP0YEiGmoEST8yZSRbrt/COSXfAv61q28PtMcz92/Fn5/hyU6y85jO3HrwVjSxvnUpOS4EA0L8KSLgkiDeqYOEQfLLyT8twVgvHgLHfj7GoqsXceniS8mYlRHw838x8ws6ndWJK1dcSaQ6kvGp4wFnIreZ1adMBZhBFRfw8fkDKhUU1qQyf3EOyrB4QHzvS4sNUHlcEF8uLBU7je5Ep9Gdgjza5miq+Os6sSuTX5xMTEYrs+skEmEdZbWPmrBcqDhBKDknb0HYEp2Zaj+wK/4KSsOdbMqsE7+ahpp2UddaJ5ulxlwOFh8UVgB5y6F4C/S4VXRSgVhcaTsLm7W/AWAr6Kaa34fdJ0RekheoIlRIFdJmai1/wFaUVFowmyyU68uBJraCLib3EomEsLD269QHROd21VFhs9OO8KggyVkMm66AkR/blBLhIeGYLCaw4KT6Sx2fSur41FaNwdAY3adw8xVRRajod0U/3yxhOl/oVdXxV4BcHcbG7LmEN+HbrMSfNfvAioHzB5I6PtUv16mjRfktw27hlmG3eH/T4Och/QaPOXZWdIjrN9go+FXY1g39n9tdXt/6OptzN3P9oOvppB0HQHWN/+YQCQMSmPXBLDqPbU6gnKo4BUCYuYtz8bqhXORSAVxugR63iHmPJIBWOxYLVB6EkDhQOczRIvrCuKXCEr810HYBVaJoBgpxUDVHDoBz1sH6OcK63QXitfGU1pVSUFNAHwcFblNEpEZ4JGkdnQrq6lq/1ju7y9mc3cWFKrIJXNnctxWSrpcj73q5/w7oDdHDYJJvjUejOo3i7pF3MzBmJN98I9Zg9fXw+uvwxx/i9zBypNjXqDdiqDOgjgyeamDfF/uoKaphzEPjMUgFERStC/C9UK61O8q06v0aOPtHoRi04uALwoJz0AtBt92yXjNWh6SEQQkMvGYgIWHBdV74YtYXhHUK44LHrmBxiVDUypr+KiwW0WxdcbBx3d+IbbeIZoppO3w7manxQpa2cc0lUyE5dx1KV8rn9oI2FdTJaLVQozrCrojH+HDXCG4aehMypYyhtwwNWNxA5opMpAop3SZ1sxN/VsVfoLDpKtCfhgk/++czsJiFksjaLNPen+15+0ERivRHPRWaAxyqNgAjGodq8ZqD21b8+u9fObziMLKpMkwyM1IUflO7Bxw9boMul4o8PzcI6vzZYhbzJWkIqGIZlDCIkykFlB1Lpd4IR5cfRalV0uXsLgE5vanBxDtD36FqWBV0FsTfdtl7ZFwSxCaU1kDTGc7d2JyUV4RCr3shbmy7DKvVuCDHVouxfvWsz9+vL/qaQz8c4t/Gf7c5Yuv4L8f55qJvmLNwDv0uty9+rfPYuKp34HQqJE5u03naC8HMILaig7T3/HlgMrlnuRUaBTE9Y1zaE2zK3sTW3K1UN9i7s60kkVyO666xtqD0DyFzl6mcJ58dGYYqWDcbdnmWks+P+oABJ99HZRDEU8qoFM576zzi+gWHVJvy0hSG3Tqs2fb6eg9Ebv/HYdrujjPx9gKJRJBmZfpECgrttxF1wz5Y2lNYyTRi8aHFjP1wLPetuq89huoSTYm/uP5x6M7RoYqyXwt7C/aSWZqJwWTwfsDqLKHWdFDMOC1+5WqxWD6D4ZTx5wArUVBrqKX8ZDnZv2djMbcxcN5HHFl6hLDvP0ZTnotRIhhAtVwNuT/Bvsec1SZRg0V3pZdMmL8SrBO2NOX3Qp3jgFXHVvH0+qfZnOOsCJcpZdyy7xbOfsh7kbGlsCrF6k4W8oT8CdKXC+Ngm+KvcINdaeIAk8nE3r17PT5/Aw65VtzHQ6Lbbwx4UfypEkWejNKuKNAoNDZVp5VobSu8EX+JgxKZ89kcuoy1Lw43ntrIR7s+4kDRAb+M4c8Ge7e08721d2xvPrngE16f/rrT9i5juzDgqgGEhLatYrVw+kL23yZyUFxmE7uDRArhvXya03SI6zfYGP+TWDx7wJqsNXy25zP2Fe6zKf70pbVkrsikIrsNapFGhHcOZ9D8QUSlOReVag21lOgbrSNDOjt/hMoImHEYLqm3b5OpAtttfXoF/NRHKAsdoYwQtmatzUZKvRymbhcEYlPEjYULi9zma1tz/rwp/kDY0+nL9S5fUyjslpst7ZgvPlTMlle2UHa8zOf3BELx167Xr8WzTHJkykhenPIi8wZfbPs/5+fDnj3i72UOv7o3+7/Jx+M/DtBAXeOKFVdw3abrqKsDg0xYL8SGBVDxZ6iEmpONOd1tQMpMZ8I9+3uhTgkG6VdxQNgLNyp1tVoo12xnrfoeXt/6Ot2ndef8D89vsUV5W1GeVU5VbpVnxwOJBMb+AOeub65ilsjsDaPekDQVZhx0n3vaAphMJvbu29dxnr/jfoRhr6PRgMxkX1+CKJSe9/p5DLx6YEBOvfyO5fz8j58BqDeKD1JmVon5V10BZH0hVDr+wsmvGp23tvqn/lO2B76Pg8Ov2Lfl/ghf68Q12h7QdgZlJBWSLNb3HsK7tUKp9vmMz3k26tmAn76+op7qvGrMUjNSSwgxujIkR9+AUh9J9vZEWAbEnuV8ryjZDt8nwjGhWAzq87ehHBZ1gt1C9Xr7iNv56vyfSC67lLo6+GHeD6x5eE3ATm9qMGGqN2GsF7UdqUV1Zqi+JBKIHd28DqhJhkHPnXnElcN8XxtqoCjsZ5ZnfYfFYqHTWZ3oc0kfLKa21wGje0Rz9r/PJmFggtN2MVe2EHviFjjyusv3ngloj2fu3xXQIEJfrqe6oNoWUumIG5bcwIj3RrA1d6ttm5U912r9zAfl/wo/j4btt/vxoEGAXAcV+6HmlMfdrMUpa5d6dPdoht40lMiuXjJp/ITB1w8mY6ZYEK0+vpo3tr3BztM7beORSj0sCM4gNC0CSiSQU55GXdf7nHJRyvXlbDi1gd0Fu8nZksNrGa+x57M9QR6tM1x1PTe9Ad+49Ea6v9qdRYcWeT/grn/BD4nCPvdPCivxV16OyL/Z+wRgt/qsNdSy9vG1fDD6A+qr6l0fxM+oLa5FUVqA1GTEiPjdq+QqGPi0INIbfegxG51J2b8B2Im/Tw+/AzOPOf2Ovtz3JQ/++iC/nvg1aOOxXZdhcnrP7c3z1z9PzYM19I5tDCDccQ+sv8jleztM0aKd4VHxFzMcJq50yvWRSCREqCIAKNPbq6An1pzgvRHvcWxVy4scVuKvJV21C/cuZP7i+Szcs9C+MfM9WHU2VB5u8Rj+bFCr4flzR3BphHOgQYwmhnkD5nFON9dZTdYFcmsR0TUCaZxQWWk0guiwFsTcouYkFKwVWVA+4i93/SrCvDahRKtFE0FJXYmtWUxRlMfCaQvJXJEZsKHlVeUhk8iQm0KJ0kQ03yGsB1hzjwvWQunOgI0FgJjR0OMOofRyhKG6bc91idQ+7zY3XielOyF3qVdi5JFxj/DT5T8xoesEj/sBfDz+Y97s96brIUi85LJ6QM7mHFbcuYKiA0VkV2RTUluC2QsR1rTprc2oykTy+zzUFRv9dEAfYTbC2lmw0TeloURit6vfsMH+jNI78LF9LulDxvmBd4VxhCZaQ2hSKHV1YJQJMj9KE0DiL3cZLE6F3MVtP5axxk5UnbMOpgapkJ63TFj8Vh4CaFSGHeVQxEv8cOiH4IzBBW7eczNTv5zKkqOLKA5d7f4ak0iau/sMewOmbgu6WtKKjvj81WpBZhbry+qG4ES1TH15Kuc8I+ZSn835jP/2+J2ImpGi+b58D2y6HPJ/8d8JzUbhijDuJ/8cLzQNdN1FbqAVijDR8KdKcP++QKLuNFQeIVwjJjH1FlHcTBmZQvfpgc/tnvbKNGbtnoVFakFmVpEckQXbbxUNRWcCzIbm8xFdV+Ha1oigXb+KUMi4y2kNab3PGY0w7fUZnP1v/zcFW6HUKbnt8G2YbxTzHJlZTbL8FzEP7egwm6B8/xkT6eQRFYdEg73Fgiq0ni09pvDMiYvQG/WMunsUF35+oV/ydeP7xTPh8QnE9hbuQPesvIcuL6VyQPMWEixUj1gN/R5p83n+SjhTxM5/Cmz+32bWPrqWBXsWEN8v3um1/GqR+5egsz+YrcSfzrXTTOsR0V8EALcm1Ls9IZHA9L2usz4coFZbMEqrKauRAu0blrpw70I+3PUhT058knmpgwBRPHPi+LbcAKE9oPc/22eQrYQjaRYaKh7+RUVhnIx8hp4OX29rFkppXakIW1e0f79B067nTyZ+gkllot8yu5T8eNlxANKifMiWSrkAdGlnvKrPE6zEX1kZWE59h0QVC/3+bbf6NNQwaO4gYnvHIlMEZ/E68JqBPLZ4IEYjWBAfapohW9iopZwvFlWHXoLMt0FfAGd97ZcO2T8LrMRfTkkyNPnqxmlFYaKwprDZ+44uO0rOlhzGPzreb1YFZrO9GJfQO5oeX89tvlO/h9uWi/IXgE8Zf00QoYqguLbYSfFnNpipyquiobrlygBvir+N/91I2fEyznvzPNv350T5CQC6RjpYQ9YXi2KLYxHjLwq1Go6XDSJWZiLRx/csunoRR5Ye4Z9F/2y1ndJ5r59H1uvAClBrzCS+kIgFC6f/cdppvuqEk1/Crvth0m8QP65V5/3To75EZNSF97HbUTeBde5UUltimzfWhcUz8eWZdDnbe76ZNyxdsJTjq45zy4FbkIfYl4PpUen8Ml7Pky8XEtajyfemeIsgdkNihD3m5mtAGem7RV1roAx3bYm6Yog495Q25pRvu0UoLqZsg/xVsOs+mJkJFfugaBMMfKZZY57Vwt8XdJvcjcrsSre2Zmo1VFW5adbwgO7Tu3PNumuI7R3LqC9GsadgDyuvXMnkNPfd4363+qw6hvTUFygTA1/AdYJUDqZ6kMi9WtkV1RRxovwE2ugMOB3OGgdBguNzcuITrc28az1KjpSg0CqoM4Yxfv8BQsIqSAlLCdwJQ9Mh424Id6FybQn2/Qf2/FvY+IX3FoSVOkjEQspscd8MFw1hOh3ITaJQUlVfTcnREtY/uZ6+l/YlfWor1cCtxNGSo9y8djaq1E70L2nSpFxXAH/cIezgY0e37UQnFoLFIPKP/2w48SnoC1D2vBelxBolYb85rrhrBWajmemvTff7qR2JqJ4xPdmvBKWp0XUroj+M+RaiBvnvhF2vaJvtblPItTDld+dt8RPET3th42VQsZ+IGNHAZ5YYMJgMnP1/gSOImsLRtrXM2B0mrgZdatDO32rk/gRrZ8DIj6Db1WJb9FCYvKl9xiNVwJCXnDapVGDBggQJ3Wb0Dkw0VROM6TyGsZaHqKsYSj/9bbBdAeftDfyJ24KC1bBmCgz5H2TcIbZlLxLKzYHPQIR7y/gOhwNPw4lP4OJqYsO0UCQBiYXK+krUisBZpedU5nCq8iR9JAbSu0sJTft7fdlS/E38BRHJw5IZestQdAnOTF69sV7kUuFM/JWXiz/9TvypYs44kskGL6QfwLuF1/LL4I+g+ln+xb+ozK3kk0mfMOCqAYx9MLA+ygV7Cvj6wq8Z8+AYBs0fZFNQRKoinXJybDDWQs4iiDvzbl6OhYP4eEGiFRVBYSH07Gl/zZH4SxiQwC37fMgHCjCaFj+kcilmmb1Tuqq+ykZ4pEX6QPylBjHfpJ0QESFqKxYLlI/eTWSM+CI7Wn2mT00P6kLbYBBdZgANZlHBGVq2En7/EOaWCeucU9+I7LXoESLT52/YYM34a6ipxlJ5GokmRdjS4pn4O/j9QXa+v5ORd430Wx5OvYOIyG2ndPIMv5zrzwyv6pG9jwuLPOvCA2yKP0fiL21yGndn392qMVivSXfEX+byTAr3FzLjLfvneaKskfiLcCD++twPvf/1tz0v4nN9cftbZGTA0Cav/XzsZ05XnWZmxkzb8xYgplcMhloD9ZX1qCJaL/Gxzl0sygosCIWVzX7XFZLOE39GNx3p37Ahbzn8Pg/Gfg+dZrvcJVpjV/xJJEIBUW0JJXXOYGL8EJWsjlKjidU4kX5W6GvlqAxJzdcfx96HY++Kv3e5rJEUC6JKxWIW9wOLBeInOmfztRZSpVBFNJSJonzpDqGcOPCMsK3v/a82WTh7s8Vu6lLiK7RxWrRxYv5lfU7Hajznpnq0IWwNkqZgurCKsv37CEzqlgeMX+aTQmriJxPZV7iP68KXA1MpcGi4d1T8tQfeH/U+8f3jGfnW1agbOhMvA2UgL6foof65L0cMgNQrBPFamyfm2FGDxHUUaISmiZ9GqNUgt4jroFJfTX1FPbs/3k1Mz5igrkf2LNxDgUp8uaSWkObXWNF6YYeackFz4q82B059K9xyogZ7P9mh50UT3J+R+Dv2AZTvQdLrXnRKq6OMXfF3cu1JTA3BUThZ57JyOaCO/ztvujXoOg/q8omqt08magw1RMgignL6o8uOknUqCxDWkBZ5KCQEv8mjVdB2gc6XgMYPE74AYOGehdy49EZCu09k2NElbcop9gW1JbUc+PYAPUf05Kz68ewrh6woOX16nQGOTtHDoNe/IHqkfVtNFuSvBPOj7TWq1qHbfNFMIJERHi5BbgrFKK+ksr6SynWVHPzhIOMfHU9YctvmA6sfWk3ullyuXHElUrmUklqRZa80RjN9mhks/F0faCH+Jv78DKnU/Rew+/TuLmXt1gWbQqpwKqYcF4IjuvizVl28ReRZyNtXCdcmZP8ABb+57gAGwhuLmJUGQaYq1AqkcqlfZMfeYDaaUWgUNsVTWV0j8aeOtGU2Oj0U5RqYfVoUHM4wOBKYCQmiyLt/P0SdfgqWfQVT/wCp3In46yhoqvi76per0Ov1tuvXqj6JVkcTrgqg7c4ZBKkUwsNFQ0JZdTiRjU41l/W9jKGJQxmZMtLj+wOBrE15RJwupyKuOw0W8aEejL+QpOTGjkyJBMZ8JSw/z5Qs0yDCSvyd0/VtJEvvhXM3QewowDPxN+aBMYy8e2Sb88McYb0mJRI49dsxtn29le/7f09yj2Renvqyx/dKpVIyMjI8Pn//KvBo9QmicK+K9Ur8tQXerD6v+vUq6ivsTK/ZYiarPAtooviDvyf1jbBn/DV/7aalN5FVnsWmazcxqtMo2/Yx949p83nXPLyG2gOxQF9MSjFP0Sg0hMg9XPsRfV3nprnBX/L6jRkJQ16BiH5ud3G0+oRG4q/a7gbSVkx6apLb16qqxJ/WZ4QNGXeI4nRDKUQNEfkzwUD5flh3AfS4FXreJR4Uw13bZ7YYg54XCjIrzvpc/Nn339DnQSdLLStyKnP49cSv6JQ65vSa06bTt5b4M9YbkcqlSKQSimuLAftz2x2sDSH+zMWRKrX06DUg+Nevj7aIqRGp7CvcR4Mmq9lrjvfT7W9vJ+vXLGZ/NjtorhUj7x6JLlFn+1z8psQMNFJmih+Ao2/DtgWCiE2aFpzzWyxgbgBZCBIJ6JSCVKhuqCZhYAIPVD+AQhPA3NEmMBvN/HDlD8TMiIGhIHNF/HW+SOS6Kl00zdTmwI67YeB/fSP+Rn8OBv+4X3S45+/wd2z57KEqUTCpM9ZisViQSCTc+MeNrXYw8IYXkl4gfUo65394Pi/9/hLbisw0yOajUAQ3L7JNqMkWatzEqZB6KWS+KwiG/v9pn0iZtOsACD0IErMci9RIdUM1BasKyFyRyYTHJ6CJDpxb0von11NyvIS7/vcCq1doCAkzgUV6ZsTrRPSFMU2yjfNWQtkO6H4zKCOCf/1uvk7UkIe+glwqp9ZQi1YhJoxL531FXU4pN++9OSCnrjhVwU8LfmLikxPR64VQpjZiBgRQJO83KCNhUJNMy553iWazMw3x421/DQsDuTkMI4L4q9tTx873djJ0wdA2E39VuVUUHyy21e9zSkQdOUwRxbje6+GLCcImu/uCNp2nvdAez9wO8pT/a8PR5tPRNu3oUfFnd385qBjr4LdpsNr9Yv+MQN5PcOQVMblxgSi1mFRXG0WRSh2l5pZ9t3DWvwJfoEgcnMiC3Qvof2V/AJeKv2bdMFK5W6unjoymir/Yxv+CobZCKBnrRbHKSvyV68sxmU1se2MbB749EOzhOsGV3ZFSqbT9/VSFsGjpEuED656zGFaOgOI22k2dAbBmpNQUnIAiYScyp9cc7htzH4MTB7P3i728M+Qd8nfnB2U8uz/aSfof36CSNhCvi6NzeGcMMaOdFRTaLn+Tfm4QEiIsbA4Xj6Qm9X7R0doIT8RfVFoUcX3i/NpM4UjG523N5cj7R9hwYAMrMhtzGPJ/hR+SRFe0Czhev39leFX8nfMbTFzltGls57HM6TWHRJ3dRLK+qp49n+0hd1tui8fgTfEnkUicFGgF1QXUm+qRSqR0CmvsbC3dAQeeg9qWn//PCLUa+sb9xvkpdzX7nTQliPwFi9nCuifWYdx3EACjwj6f8fCmVuWu/eWu39B0yLhd/OkGNsVfY5drWBhgNrN08qssXbA0YEO79+d7eTHrSso125oTfxF9oeuVggAMFukHoO0sCnWSAPSrSpsc09psoO0ssnRcEEw7Tu/g6kVX8/SGp70evr6qnsXXLmbzy67niK2xZwZY9c9VPCF/gpzjORgbMwpjNJ5tkf2e8Ve6A0r/QOnuZh9oZL4r8t48IDU8FYAq+fFmrzkSf3nb8tj/9X4MNQZ/jtAjzv6/sxl83WBOlJ7iQMq97A99ObAn3HU/rHdho94WxI2FwS9CpA+ElT9gNsG3EbDJbpEYGmIn/qRyKUqt0m829D5BApcsuoS4q8W8WWoOcX2NaZJdR0KE9xU5ib4q+MJ7ieYRP6FDPX/Duov/HxCmFr8rk8VEQ2POWaBIP4vFQkzPGHRJ4rv08G8P80XJvRjkZaKJ7cSn8G0U5K8OyPn9BrlG2PAVN1p+nvzq/9k76/Aoj7WN/9713bgbCQkQ3B0KheLQUqzUjbrQnnpP9aucnsqpu7tRL21pC4W2uFMIFhxC3JPdJKvv98dkN1my8d0Yua9rL8hrM7Oz887Mcz/P/cCB11qd6NLpQOlwSreaSN+UztbXt1KW20iPl0Zi6rNTmf/xfM6Pv4OuuTcwPuYd+FIF2X/Wf3NbRPpS2Hk/2KsmrxYdv/kboWAbUOVw4VAKbzRNqD9BXX3nLB/aPZRLl19K6NmhZFlTsSqLverE1GKovkeSpFYfm81BYCCo7ILgKzGXMHLxSO4puIfowc2X/p774VzuSL/D9Xd2iSD+xg0LRW0Igvj5de6hOlETncSfl+Fw1J5YfdWDq/jhih9qHPeU30+W4fBh8f/uDVAabDAGPSESs7ZnDHoK5mWJBbQHhBqEUcrkaP0oOk8Rf65IOVsZpDwOJamtU7lmojppFh1dRfz9lPY0nHvQRSKEVBKxMjLF5mJWPbCKrW9sbenquqG68SN7VzbrnlnHxt82usavk/hLCGpAHh1zgfCmU7YXV92mw5nnLzLjblgxVhh5q8FutlNeWI6t3NYi9Uk6byiHhyxAE6BjyXlLOH7bcWb0mNEiZXcESJKI6NiXdwYZEU+CfzfXubqIP1mWKS8op6LIezpZ1SXIzrjnDPqu6UtOZI4rGg0AQ4JHb2mHw0FKSkqd8+/pgnqNyP5JNb7Dhyc8zLfnf8vkblVOQRVFFXx/2ffs/mJ3o+tQV44/u8XO0VVHKU6r8lZ3RljHB8ajVlbeVLgD/rlH5PjrBAYDdAv5h6kJL4HR3YB9KkFUHeufXc/PNzaRJJJg8YHFlIwWOcMsCrHpqi4nWgM5fwvj2PElDS6mc/x6xqmEblAQoFCg8NejCWieoUeWZf58+E9Sl9Zcf/504Cc2ln2GVVnsLvUpyyJvrhPpv8C3kZC+rFl1aRDUATD7APRaLP7ecY/wPK/I812ZskPIGJZn1jgV5SfWt57mx1Oh8dOw95u9nFh7wuP5eqO0a0H0kGj6nd+PElUJAIHawLojcamaZ71mLNtxD6ycRMru3a0zfnP+hkNvg81U6yVDYwQhta+iytDrKYJ61muzeMj6ULNkkZuKY0VHORL9HDs1XopirQ1FuyF/U/OfI8uw9RaR6y+oL/S+3c1xzKdQKCF2lltkXJBevKjKbEZkWSZjawa5e3Nbpj6AQqmg95zeqPuL9UsNqc+slWJOtNeyZlb7CwK1IU7AdgtYiprkYOMJbW7+tZVDRQ7IDoL0AZyxbyPvj9jlWhvm7s3l4K8Hkb3UfickSeKKVVcw+QmxDq6eF06tBjShEDIE1G1cBUgbBvNzqlSxxn/j2xy89eHwe/DHRAxSRhXxZzUx9u6x3JV9F6HJvo2mjB8TT/ep3V37khJ7AsTNAV1Ds2W3Ihx22HwjpL5adaz/wyIfsVa8K1p8/J69B6atA6oTf2L+7XfP2Vz8s+9S32gDtXSf2p3HDz7OlxG9yYl4lz77omHbHfXf3BaQ8Rv82E3kkgbhQJ+5vHXr1BRk/C7acXJpJfEnPARLzCVo/DXoQ/QolN6lmDIzwegQe6GzJ4VByGDxboue4tVyWhKtMed2En8tiJMbTnLkj5oeh07iL8q/atGcmyukdlQqSEz0UgVUehEannihlx7YStCFiw1GLRJgEf7CqFkmV0lL7vhgBzs/3unzqp3ceJJNL2/ClCMmwToj/jJ/g5SHxUTQDnFqxF9kpcJQzil2EI1SQ7ghnBj/GErNpVy87GJmve79pNyNQXXi7/jq46z890rK0qusLk7izxV9Uhe6L4L52SLxdwdHcLD496DtcuHhKzvIMeWwLWMbhwoOMfjKwfzryL/oMrpldBcM3WIojOuPIaCaR/5vw9t/VHMLIrBSicEp7eaEk/jLLcvF7nDPp5G5LZNnwp5hy+tbvFaP6mNSqVFi9DPiUDpcjgNET4LpGyG6s2/rQr0Rf5ZiKEoBu7mWCwT8Iv244IcLGHpt47343fKinAJnzt1NL1cZH50yn4nBiVUXdpkrpGe96NXenqHTwcojV3DNzydqfCd1RfwdWXGE3Z/vbpKRTJIkwpLDMCqFoatCqnJkquMuIV+pa763Z4fH8jNgbe3r8eGxw8m8M5N9N4uIy6BKe2PCY9cw7X/TmlW0xWhh9eOr2ffdPrfjVruVI4Vin+Jf0cs94s90HJboRZ7Q8mz4+xww57ZMXq/qsJULhYWKnGbl3qsXZSfhhzjY+3SNU879WrYxu96xJSkkbj9xO+d/c77H802V+hyyaAjnLTmPwspxWZ/MJ/hA6rPXLciDnvLSw5qAoS/A+aY601fMTBbyk6mlW6lQif32sGHiXPV5UqVVtUhKCCeM2UY+Ousjtr2zjaIKQd4aFD4mFSb+DHOON/85kgTpP4lPa+CML4QMbyWCDcIAbXGYsdqtvDfmPVbe3/KRWWabWFcpHKcQf4ffg3UXgr2OsF6HvWGODPmb4JsQOPBK8yrbVrHtX/BdFFiLCfBTEmIaRYxiAIpKm8/ap9by+azPsVX4zsHU7rC7oqiVTuIv7myYvLJ95C7WVYv81gS7OXW2OMqzoHg3OmUp3bLvoFf6f4g0RKEL0uEX6ed1gsATckw57MzfgFF7kKPmWXDmdxDU2+flNhsKJRz9EDJ/rTqmjxK/wQZKXfsSTuLPphARf41VLWgsnGstJymvlVXYDcnuv/e2DKVerFUcInqZlEdgTTvMG6oyiPeKpHCL+Cs2l2AuNZO1M4uy/OZF8sqyzI4PdnBy00kAtu8yY68kmHsntCPp5TaGzhx/LYjLV16O7Ki5QRwTP4YXpr/gRjI4ZT67dq1dLqtRkGVA7hj5chxWMB4FlT8YYmucjggQL4QKqiL+1j65FpVWxaDLB/m0agd/Pcjqx1aTNDkJTZgGo0VMhh4j/uLmwKQ/2i1hdGrEn72SGyjONyEf/hrJv6tI/grk3l3N83IMrY7qxo8BFw8gclAkuVJVHWf0mIFOpWtc3rp2HKrfUDhJogOmcxlXuWZ+f8f73LfyPq4cfCUfzPmgRevj0YgVkCzeDZ1oEAICINLvGIlHb4Dgi6DbFYAwIq64bAWRfpE1JJOCE4MZcs0QogZ5z8O7OvGXfzCf/P2CxKhTVrATNeB8L5vN4p2sPHVvuP952P0YzEpxy8MmyzJWhxWNUkQSqbQqes9p2sa4rog/XZCOma/OJHpQFTE0pdsUfrvkN3SqagNZG+bKN9kJ0a8mawgmawg2B1S3T7uIPw8Rf+ctOQ+Nf9Nkz+wWO6ZcE+UlekBdRfzVNSajJsLU1Y0u67SEUgfK2iO0tCqtmxKI0/Gm2AupndQGNTftvQmVzn0beLjwMDaHDbVsQGeNcyf+ZLswfgYkgy5SeNrGzITI5ueSbBDM+bDnv0JScMpfYDP6dt2lj4PkmyByQo1TTpLNbDdTYi6pNxd0XVFkTSX+nMg1ibVrhKH+aCGvR/x1mYNst0NKipce2Eg0IEIq2j+aYTHD2Ja5jdygX0koWMSIEbBunXvEX2lmKQWHCogeFI020Hv5i2uDxWghOyWbrhldKYwTg9qgbIFoIm+NmZk7hCHz2wjofg0Mrl/21lcINQQz4Z+9XHWJP2qlminPTCGkW8utHUvSS3i97+voztdBFw8RfwMfh64XeM7v58SKcVCWBvNO1l2YJhi6XSWizzoioiZWyi8rXO9GU7WA3sGLBpM4MdHrhJEp18TmVzbTbUo3IsZUvVcUsrbWfNVtGjlrIPUlQZB7UPpoMfR/APo/gLYCumf3AiBCJySwS9JKCIgLQBfkuyjrV5JfoSy4jH+f828iEmYwWf1r/Te1JcxJA021eaE8Sygg1OHs4lMUbBfpBrrMdhF/VknYOk/+eZC/fk9n3L3jaqwtvYF93+7jmwu/wXCTAcKgwhpK8cg1rsCDNo+oCXB2tbVS33s8Kkq0eUSOd0URB1ZAYs5iYgoXMjRiNMf+OsaX537JnA/nMPiKwU0uwmqysvSqpQy6fBBdRnWhpKycYOMoVAGFYr194lvI+UtEwLbDdFmthfY4lbVreNIm7x/Zn/6R/d2OHTok/u3hLelac57wXO19Z6suzr0C41H4uRf0/bfHtkQFisWNWVGILIs9zoIvFvhkEjoVw64bRrfJ3QhJCkEhKfjrir8orCj0HPGnULbr6BWngVmShMynk/gzV9iRNi2CpMtdxF91yA4Zc6nZpwu9uiDL7sYPfYie+LHxFKUUua6ZmDiRiYkT63+Y3SwM6dFTIGyET+rbluA0AlaPDvNTix90mbWM0sxSDv16iC5juhDRx/cT8YZbvqDfrgLkvjcz8cOJlFnL+PK8L+kW0oreje0MgYGQjYNA81owVeVsUiqUTOnmWULBEG7g3HfO9Wo9qo/JZTctw7jBCHdXIxlSXwFJCT1v8mq5HQ3VHTIqKjzklI2aBEjCeFSJ97a/x/U/X8+c3nP49vxvm12HuiL+9KF6Rt480u1YpF8k03tMd7/QWgoKTZ3EyOkEnQ4Ukp0Iw3EqCnX4R1Q5PTmlPgvKC2re1wzZuuxd2bwz4h38+0ymtPs4ksO7cenASxkWM6zJz+xENUxuXESKM+Ivb/Ve1hUWcsbdTc+xp1AqPM7RqXlC+jPI1hMJhbvUZ0B3OPOHqr9PyRXqcyh1Isqly3xIusT3kYYKJYx4zeMpg9qAv8Yfo8VIjimnXuLPXGrm6MqjBCcFuzk9QNNz/K16aBWSJJFwQwI3j7i5QdL0Xs/x19pwWIV8pcoAgb1qvezs5LPZlrkNw6Bl/GvgIsIqA0WrE38pn6ew4q4VXLX+KuLHNEDto5kI7R7KPXn3APDBa+J35qf04W/aboG07yBkoJDnbC40IcIIHdin5UmFjF/hxFcw6EnQRxPgryCgog8Gm9iTjrm9ZZ2GFEoFcaPiCOsdxpW6N9lzMA5ddb+pgO7iUxfi50F5Bi7DRW0IHgCj3/NKvdskEi8WH8T69Xj423yens3okquIC4wj6awkqGlaaDZK0kpY/fhqVDoVASOqPF5cUp9Zf0D23yK/bXswNOeuhbRvxafHdTDyrVatTnUivKIC0n49yLcXfcvCbxbSd4EX3ke1oMuYLhx2iNxJCoeOYcHvwdadMORZULah3Ja14dRott9Hid/fjFZKmbP7P3DyB7jI7iL+LJIRGZnsNans+XUbI24a4RObq1+UHz3P6cmWAKE2pJB17Xst045twE5otZBQfi6WYohUgapvARMfm9jsHH9KjZKLf7kYv0hhxNA4ghm3fyOTJoFCQpB+B151i/zvRP3oJP68DIWidg+kg78eJDAukKiB9UdJeJ34c5ghejr4ezNhYCvBECdIv6hJHk93C48junAeOks8FRXCEBo7rGZkoC8QGBdIYFzVxm1CYpWnsFvEX8E2YdQMHtAi9fIFnAbm8HBh4FWphGGquDiQrF4/Ed3Tc7TItxd/y54le3jI+lCLyuo4YbFUpUbQ6cCYZUQXqmPAgAF1jl+PKEmtTLJcfloRf13lT+HX52DMxxjUwmplspjIT81n6dVLmfnqzBYh/tSRwVT4KwjWw/bM7ZRaSnHILa+Z3Z4RGAjZpm4skY1c0oqvI6cBTquFQVcMIiVBeMW5ZAVTXxLGPQ/En0KhaNr47YBQq8W72GYThuSaxN8E8akGvVqPXbZTVFHkdvylpJcITQ7lsuWXNaoOdUX8NRibroUTS+BCCyi8IXvQvqFSQbh/Du+e052yfTdDRFXOj7qkPs0lZrJ3ZROcGExgl8YZlfVheobdNJLU7SKf8tm9p7Jg8NS6b9pyM4SPFcRMA9E5fmvH/Svv50TxCZ6c/CTBwYKMKN+4kz+XHGbsXWObFMkJItrIWm5FH+qeiyM1XxB//maxfnOL+GttqPzgnP3gl9TaNQFEnj+jxUi2KZvksOQ6ry3NKGXJvCWMvmM00c+5G0SamuNvz5d7UBvU3PDYDQyLbRgZX14OKoWZBMWv4Di7ee/Woj3w92wU/R5gwIArW2f8Worht6HQ/VoY9Xatl13Q/wLCDGGcnXw23UNh/35xvDrxlzQpiRkvzSAooeVzeJWahdSnvy/zh1VkwvqLoPcdMPS55j/PnA/Fe2D8ty1PhBTvgSMfin7XR7vWOUZjy1bDCf9of9c6KeNdyCuuRnRYjYAsonTqQt97fFrH2tCW51+DAQ5H/4+UwkPcXnQWcYFxPisrom8EN+29CX2onhKbGI8KlChQVRJ/q2Dvk8KpuT0Qf71ug5hpcHIphLais1Z5JuSuQwobgcNPRakjl7SCWCIHRHLmQ2cS3su3Mo3zPp7Hy5teht+EbGsPwzI48D0Me9Gn5XoNxmNC1jy80mEy6VJQB7tOt/j47bkY4ucDMoHaQMYnjCcvwx8ZO8HnjGfhMyPRh+jrfUxT0HV8V7qO78qzbz0PWRCnKcVw5HGIn9k+JHgBjn4i1CqSb2ztmjQdDpsg3fwSkeLnEhgIeXlCjSQ5OZQJD9VUyWgslBolybOq1tVOu4LGydUP+R/0vQ+07SXcsyZaY85te7N8B4XskPn87M/5+9G/a5xbc3wNm05uwmQRzJAs+4D4M3SBiT9Bj2u89MBWhMpPRPrFeDZA9YhIYOTR7+if9pJrIy07ZMwlZq8ngz4VFpMFh90z8eAk/vz8gH/ug9+GiU1rO4XTEzy6mg0jonItfNx2DgRU/XifWfcM4z8Yz+cpn5N4ViKDrxqM3eKeN6ylUH2jr9XC28Pe5oPxH2CxCM1tu8POyiMrOZh/sH4SKbCnyEPV7UrfVbgNwWkEtFZYxW/XXoGfpiriL2pgFBcvu5hes2v3vPYmYq6ayeHhF6DXV2m+Rx5+FY5/1SLldwQ4+7SkpOa5Xw78wn/X/JcdmTtqnPvr0b/49uLmR4c5UT3ib+ClA8menQ1Ui/ib8hecsaTW+53jtxMNyPN3CoJ1wQA1iL/4sfFN8tqrK+Jvy+tbeGPgG+SlVuWy+XTXp3z4z4dkllaTPIkYJ96rnaSfCxYpnO/33YkpwH3tM7X7VD6e+zF3jrmzxj3pW9L5YPwH7P9xf6PLC0kKYezjMykNT0KS3D22PcKcDwdfh+xVjS7rtBy/RSnCe9p0otZLluxZwmcpn5FWkuaK+CseOZVrt1zbrKJ3L9nNs5HPcnj5Ybfjzog/nUnM4W7EX+rLsOMeaE3nGv9uLSurfvBN+H2Mx7V69Tx/9SEsOYzZ785m2HU1DbBNjfi7MeVGrvjrikbdYzbD9cNuZahxnvDabw7s5SLqUqltvfGrDYNBTwgZxTrQN6Ivt466le6hwvnVGSVQ/TuPGRLDqFtHuTlv+hIlJ0vY+81eSk6WUFL5+wrQ+JD404TCuK8gseFOGXUi41dYNQVyWkHaucd1cF4RRIwFxH70SNQLfJJ9J0cLj/LzDT/zwZktm3rACedQcBkoT3wFXweK78sb2HgV7HzAO8+qRJuaf9OXwZqFUJKKnx+oHFV7TIAdH+zgpW4vkbnDuxJ5Kp2KiD4R+Ef5u3I1KhGLHpUKQZifvRf8unq1XJ9BpReE38BHoYt3FVoahfytsHYhZP9FSvytrOk3hO9TvyGyXyRnPXYWkf19b7h32gcUso4Vxo9h7sn2k/po6y3wx7gqj/VBT0Dfu90uadHxGz1JkI+SggBtAKsXreaOqGUoUGHzCyKyf6TPnfrLrGLiTjTkodr7MBS0UvRjU5D6Mux+QvTnD/Gw+frWrlHjISlhx51w9CPxd1AauQF/sPXkP14r4lR7vdNG5JpXlTqR7qsN5LpsT2gnb732A4fD84ZYlmXmfDCH4TfW9Ei48scrGf3eaHZm7wQgJ0d4ralUIsdfJxoHSaq5kV56zVKeCnoKS6lvJ8fPz/6c56KFJ+XhgsO8vuV1fjv0G1DlzWswAP0fgmEvuet2tzOMHAkTJsD551cdc+ps5+QgpNoqDURHCo+w9sRaDhUcYvj1w5nz3hzUhtYx6DqJP40GJEmm/0X96bOgD6mpqTgcDjKNmUz5ZAp9X+9bP1Gs1Ik8VK2ZOLsF4TQC/nliEcw5AmEjqiL+rCb0oXqSZya3mMe0c3xrdXasDisSEHjwJUj7pkXK7whw5m0MtayALHfpuY92fsQDqx5g7Ym1Ne7L2p7F4d8Pe82Z4lQJsg/nfIjpfhM3jqj0ijN0gaA+Hu91OByu8duJeiJIKvJg5WTY96zrkJNcPZX4m//ZfKY+U0+ElwfUFfHnsDuwVdhQ66tOPvb3Yyz6cREHCw5WXdhrMYxuHcNdW4VGp+b9f56lwDDH7Xjv8N5cNugyRnUZVeOeyP6RTHtuGl3HN20xWX3dUmIudhnEPEIbBgvyYdB/GlXGaTt+C3bArofqNFqE6kXO6vyyfFeOvwJFOFEDo5oc7QcQ1jOMYdcPI7RHqNvxMlsZSkmJvpL4c5P6PPGViLJpLwYzb6A8C0xHwZxb49TjZz3OLxf/wviu4+t9jKSQGHr1UI/RDU2N+FPpVOhD9OSV5ZFrysVqt9Z7T0UFfLrrcTKC7hSSqc1B2HCY9Q+OhItab/xKkpB6aqRklvM7r+4I2NI4se4EXy/8mrQNaRhtgvgL1PiQdFQHQMJCCB3qnedFjBV55rKWt7wzgDrQbf/s5wcnwt9hVcXzHCs6hq3chsXYcsbw4rRiVt6/kg3LNpBS9gdG7YGq9Y9fAnS9CIL61f2Qgu2w9gLI/qv2a2QZMn8XqkFeQpubf41HxB6uPAuDAZSOKlUZEPmndcE68LIft8VowZRjwmFzEO0fzeorV3OR4xegci2rCxd7kPYgD9mWEDoMxn4BkRPQSmJBUVJhqucm70CWZX6/43fKfxXGAqVDh6T2E4RBe0G3K2Hw07W+Y9vC+HXl4iy2Yco1+cyx/9Dvh/jttt9Q5Io16EHjEJi5E+IX+KQ8n2DUuzB5Fcg2oV6hCWvtGjUekgRT18HQ5wE4Efglm3pN5YPU5zFmGflg/AdsfGljs4rY9cku/uv/X5dz4priz1g5oCtLSheLC4r3irmiHaM1xuxptHtrXSiUCgZfMZhuU9wJAlmWyTJmASIBOcDBSttXYmIzpbKq4+hnsONeQcZ0BOy4B/6oXeRdb5CxKUopMYrQg4TxCQxeNNjnEX9Jk5Pod4FY3G/J2MLNy27m6XVPA6dE/EWOb99h3ggS6K67YPDgqmPOiL/4wgeFh6PpGOBuvGptODf6ej1IksS0Z6cx5s6qfBAnioX3fVxAHMr6PEkq8kSOkdMETpKoenRY9Rx/LY3MJasJO7kTlU50qgyUTdsmtPs70SA4ydxpQdfCP/e6nYv0E0x+jimnxn0Lv1nIPfn3NMv4XB1VhLzMOyPf4e9H/8agNghi2Vwg5E46ZVwbhDojSFR+wmhULaF4bRF/TUVdEX+jbhnFLQducXMOyCjNACDGP8Yr5XdUNDaSE8A/yp8xd4xpkMT8qTjwywF+v2oJupIcdDqY/9V8dE/o+HL3l7XfpA0FfWc/NgixM2H6ZoiZXusl1WVcXRF/RQ7KCsqbZVzpOr4r57x5DmHJ7kaHLxZ8wYnry4gunI9C4Z4zlEmrYOb2JpfZLjHg/2B+lpuChROTkiYxK3mWa55sCGwVNmwVNrdjTYn4s1vtnNx4ktKMUm799VYin43ktS2e8xFWR0UFFJsjyYl79rTzlDZajLyz7R0WL1vscjAym6sCKdI2pPHGwDfY8/WeFqlP/Jh4FnyxgC6ju3Cm/THO3LOTOV2ua5GyvQL/biJlxckfW94ZwGEXEdPF+wCxr1Y6BKlgtBiZ+9Fcrt/ecpEUxSeKWfvkWn7/5XfeNE3laNSLVfab6ClwxueCAKwL1lLhXFFSR3S+JIlopfHfea3ubQ7JN8JFdoiaUNmvYo9psgojyoCLB3D99uuJGerddUbK5yk8G/UsR1cdRa/WM77reLrYJgKVa1lzgXAE8bEdqcPBEAuJF4J/IjqF6MuSCiNFx4v4aNJHbH/Xd2sKu8XOxhc2Yl8v1koKWUuEZg+YjvusTK8jYQH0vl3M18Yj8Pcc8c5tLaS+DN93EaluKuFcw5Ss2sqzkc+SsTXDJ0Wf3HiSTS9tYnbEBSRm34rG1lPkrG0P0rtOhAwSamEKNUxdDYP/29o1ahrCR4O/kN0P0gnjYElFCZJSIv9APmV5zbMJGsINJIxLwC9KvDMKzFmUa09QTqG4YM0C+Ht2s8o4HdFJ/LUyjBajy2Ae5ScMM4crlXe8JvMJkL4U9j0jFukdARU5UHZCLP494Nu47vw2NNAlUTdk0RDmvD8HXZBvs8BOeGgCs16dBUBhuXg5OaMpnN68frryDrtwDKu0IR0rHgaJlwGCFHASfwUVBZzceJLvL//eZwuD+lA9l5gnpBWnAZAQVM8mDWDtAvi+HXmONRNO73+lNRv50PtQlOKW48+UY+IJwxMsu2VZi9Sn+Jd1hKanoNRVWcy0oYPq32B3wgUnmfvtsRdh0JNu5+oi/pRq7xoNnUolatmKKcdEWX61ReOJJbA0CTKXe7XMjoo6I0hUelhY5JbjpzrxV905Zvt72/njvj8aXX5jcvyVmktdBp2YgGqGnI2LYN/zjS67I0OvhysG/ZuuR2e6HTfbzPx84Gc+2fmJV8srPFJI+spUlHYLajUUlBcAEKStJaK7KAVKDnbY9Y3XoYsQuYFVpybirEKYoZL4K6si/sIPb+Z/Yc+QvjndJ9WylGtQyloCAk5R1VRqROT16QQvyooe+OUATwY8yb7v9rkdb0rEnzHTyHtj3mPd/9ZRbK6MFtPWES0my/D7aM6OFc49OrVZyJge+ahRbXDD8a8g9ZVa92EthsPvw49JLhKoNhgtRq77+Tpe2/IaSnWVw55TQkpSSNgqbDisLeNgFJQQRP8L+xMUH4TKHEFg+UCf5jBj37PwXTQU7fbeMyf+ApMav0ZoPhywbBDsehgQxJ/KXkX8tTRihsaw+MBizJPFj0khaxvvuB1xBlxghuQb6r5OkkS+644KhdJFJBsMoLS3jHNpeJ9wht80nOCkYNcxt7Xstn/B9zHgqEP1oBN1QqcUfVlqFmv+7J3ZGLN8N16VaiW3Hb8N03WiPIVDx0zdDBFZ2x5RngkZv9QpD+9zqALc1oEj3hnBRSn+FPptxhoaxbAbhmEI9837aexdY7n95O1cNeVe+qe9RJQ6HCyFrb8GaQxkWQQNWFspIa234LCK7x4I0Yu1Z6mlFL8IP+7KvotJj09q1uOTZyVz6W+XEj1IBEUZ7aKsQHVl+pdet0LPW5pVxumITuKvhZCxLYMXE1/knw//cTvujPYL0AS4cmUdr3RE6eZN9cDRH8LsQ6CsL0lLO8GYD+Hcw7V6rGoRYSw5xsIWrJQ7CivciT9nxF/X43M6LFnkJP42ZcyDsR+7vEFcxF95AaUZpez6ZBf5B1on+q96xN/BXw+yZP4SclJyUCrFb8kZ8RcfFF//w2JmQFLjcqy0ZzhJonD9MaTNV0PGbyQEJXD/uPu5bfRtqP3UJJ2VRESflvG+8rvzOo4POBuFRnSqTqFCac4BW8vIiHQEOPt03fG5NfKmuoi/sprEX1leGQd+OUDxCe/kKXVusLUBGm47dhtfTf6KK3+4UkSDBQ2AnrdC8MBa73eO3040PoLESfw5ZIeb0Sz1x1Q2Pt94uY66Iv62v7edPV9VRVQ4o/0CtYH4ayo9C2SHSICeW1Ni9nSGTgfR/kfQW/a6kWsVtgpmfzGby3+4nHJrzU7/4YofeGfEO40ub9Qto5iz/SFMwXGo1VXOTM75vAa23Q6/NU1G7rQdv3ZznXI11SP+VCph4C4PjKbnhUPQh+lrva8+LFu8rFYHndJKYRA3mU9LEeRtbNd5qZsEuwVOfAvZNfOznyg+wcc7P+aH/T806FHhvcPpMbNHDaOY833dGOJPE6Bh6v+m0mt2L0rMQoKhTuLPWgz2clSIztUZVJDyKOxvhnPF4XdErjGFsnXHr1IH6iCw1W1IizBEoKgkFIptuS5O1zlPdhnVhVsO3MKAiwf4srYeUSMvnC+gCRORq5oQ7z1TGwbB/b33vIZCoYbBT0HSZYB4V6nsYt9vtBj54+MMnl64haKsltFyVevVhCWHYfarJP4clcRf2UkhrZ7WgAg9hap+Gcn8rSIHnq2RCUHrQZuafy2F4n1bllEZ8ecu9VmcVsyG5zeQ9U+WV4vtOr4rZ792NmHJYZwoPsHLm15mDyJfvFoNRE+F5Js7jgN9S6EiRzgc7Lgbg7KSnDebCO4azD3593Dmg2f6rGhJIRGUEMT0EdM5x+9RwkumsJ87oEfzciS3KAp3wbLBcOgd4RxwobWGYliLjt/ui2D6RggUcvDl1nIqHCbsCiPl0Umc88Y5hPX0jXylxk9DYFwgdkR7p3R9E74JhYItPinPJzj5A3wXAftfgJTHheR/e8SqabBURCiF+ou1p9FaUtcdzUKprZL4c65fkm+s30mmEzXgwSzTieagrpevX4QfKr37V36qzCdUbbqd+Ty8ApUeArp78YFtG36KUHKBfJPwUD+58SRb39jKqH+N8ro8hBOyQ+aHK34g4cwEhl07rCriTx+CLFdt6uXQUUADSKV2iPDK9CV5ee7HqxN/yQuSud90f42x0FKonkus4GABqT+mMv7+8QwYLDb7TuIvIbABUWP97vNVNdskNBrxOVnSm8JBywhJ6EOcfxxPTH7Cdc3Fv1zcYvWxBYZhMYBKXUJcQByJKgSp3ucuGPK/FqtHe4ZT6rPEw3qtroi/jK0ZfHHOF8x+ZzZDr2l+zhgn8ec0fH25+0vMdjOPTnwUIseJTy1QKpUMGNDyxrq2inojSHLXQ+kBkTsC0Kl0TO02lQBtADZHlQzdOW+dg+xofPRWXRF/f/3fXwR2CaTf+UISO9MoJEfdZD4lBVxQ0elZfQoMBnh63VfcOBBmVQtECtQGolKosDls5Jfn00XtHpWl1CpR6VXIstxoaV67XQJJ9KXLmUlfi+E4+XqIO6fRUVKn9fj9cwYU7oDzCj1+by7ir1ImPTgY0k2JDHggkQjPKU8bhOOrj9eI2v5qz1c8v+F5hhnmA/e4nEIAyF0Hf58DI9+BHtc0veB2BxnWngddL4SoCW5ntqRv4YofrmBs/Fjm9p5b75NCu4dy0dKLahxvCvGnD9Ez9q6xAJS82QDiTxMMs3byzgfCM16rU8KEn1zGuyZh1LtQntX64zfxYvGpB0qFkki/SLKMWeSYstHpYikvb708f38/9jdb39jKVeuvYpPyA3JjysizXoHP9ofdF4lPR0Hfe1z/PVXq84/XDqDf/Ddb53VlysW+VfoBsJZZKS8sx2w6JeLPeBTyN4H5woY9KG8zyFZh4PeEQ2/C4fdgfo6w63gBrT5+T0XeZvhrBox8G4Ph2hpSn0XHilh+53JmvDSD6MHRdT2pydiXu49//fYvwrWDGc35womt2+Xi04nGQeUnHA500RjUMjjAaGmZtEN2q53iE8WcGXYm+zXTWFEKh9VTGNaezKG6SJFj2F45UUkSSFX2s9Yev06HTZvC2Og8xY1FSXoJFpOFQ3lFlKv1ZFX0he7Xgr4dBVME9YMe14n97Z4nhBRu6JDWrlXjEXcOBPUFILyS+Cuzi7Xo3m/3og3Q0n1a0wfalte3YMwyctZjIq2XySH2n0FaLzoutTJaw+Gmk/jzMmrLIRc7LJZrt9T0MKmL+HMaZJsNh00YFrzt6deaKMuArBUQPkZoJZ8CP2UIOCC/TLwoSk6WsPPjnfSY2cNnxJ/FaGHXp7tQaBSC+KsW8WexgL0yEl0a9Dh4Z73e5uCM+CvMtyNvugkpoDv0vceN+FNpW/e1U534G3XrKEbcNAIZmZKSEgICAkgraYTU52mIgADIzw8iTzOTEP/6r/cVHHYHlnwjCpuOOP+unLzjpPAs3P04hNeyae5EDTjnmYv73oP89btIc4+DWhysi/iLHhzN7Hdn0/XMrl6ph5MsoriYLe8fQJevwxxsrp1kqAZZliktLSUgIMBrOQfbM+rNBZf6spBP7XoRKLVIksTyy2rKqAbENG0R4uxLTxF/F/98MQ57lZSaM+IvNuCUjZtCJT6dcMGZl+rUfpUkiVB9KDmmHPLL8ukS6E78zX67aXkQsnZmkbW+DByJKNUOV2SRU8WgBhIWNqmc03r8Jl4EEWOFbI6HaA+n1GdBhXBiCw6G9HQoKmpesTfuuhG71V0eaUfmDjalbyIyahhwSsRfQE8RYVObQbqjQqmFcV+Df00DRpS/SM+QbcxuVhFO4s9mE+/OxkoENijiDxEkXG4WhgadDggZXnWiKePOryv4dW1X4zfKL4osYxZZxix0OtyIP4vJQspnKYT3DvfauqYuGCIMhPYIReOnYaf2ZUrjjlNsn0ZHdQz1Jfz9QVVJ/JVajJwIGQijEzjDqx7UtePgrwf5+ryvUdypgIBqEX+R42FhibDDNAQbLhURrLN2eT7fczGEj/VqTqs2N36D+sKwVyB8LAYbdM+6i64Fi1h0eyIA0YOiuWrdVYR09649a+NLG0lbl8bcD+dSYRMvBYVDLLoaLdvaiSqo/GCqUO/wU78J5ioSd9/3+9CH6kmckOiTootPFPNKj1cYe89YbFFC0abd9aU+GuZVyroX74fydGH7rJT7bfHxW3IQ0r4RxE/wABfxZ1casZzM4ZsLVjN40WB6zPBmviqBVfevYufHO3ni/iewDrLSo7AQRs2s/8a2hMCeMPItIfWZsLB9kZbV0edO138jgsTas9wh1qK/3PALEX0jmkX8pXyWQsHhAhfxV2YvBAUEa0OgPFvk9+t2JfS8qeltaGXUxhn5Ep1Sn16Gw9G43ACeiD9jpVqJv7eM6mUn4feRIqS4o6B4D2y8ErJXeTwdoBILQmfUXc/ZPbnPeB/9LujnsyppA7U8UP4AM18Sk5CT+AvWBbtkPiWpynjXERFaqQBmtipFkvLKnFyh+lC0Si0apQaHzUH6lnTyUvPqeJLv4DSaOvtBoVKABEeOHMHhcDRc6jN3PayeD7kbfFjbtgcnUWQ0ArKMQ3ZwIP8AOzJ3YHfY+euRv9j4UuPlARsLY6YRx/9eIC51VZU8ki4Shr8C8XN9Xn5HgXMcFJZHYw8cLKTnKlEX8ecf7c/Qq4cSluwdSQ8nWWQ5cpJlVy+jy8kuKCUlAUo1/DoU9r9Y670Oh8M1fjvRAKnP3nfAxF9deVRqQ0VxBfkH87FbGpc/oa6Iv+jB0cQOq9roeCT+LEWQs0YQ+Z1wQa+HGP9DRJV/IXJEVEN1SUhvYd3T69hyyydIyNjVRa7jDSHjG4PTevz2uA4GPVGrxNvlgy4n885MPpv/GQBBQaAuL2HHg9+xe0nzcnWdGvGXmp8KQKRSRIH5VU89GJgMfe+FoGaEGbZXJJzn0SPbOT9mmxpO/GXuyOSzmZ9x6PdDrmPV9wQNlWfe9/0+3hnxDmkb0lzEX625N2UHbL0Va9oKl0Kw0zkEWYb1F4v9odyI8eewQnkWyI62MX4PfwAH36r3MudeO9uU7frencSfrdzGz9f/TMoXKb6qpRtG3DiCRWsW4Rfph1kSEroh+lr60BvY+4zI69hRsPsJ+KUf2ExuOf6yC4yUqkMpDe+GrYVkGUO7hzLqX6MoixEhLwpZU7X+kRT1S3g6MeBR6P9/tZ8PGQzdr2pWXU9Fmxi/1eEXD70WQ3A//PzA39yL4NKxhGnEGlEbqCV+bDz+Ud71PM3clsner/ei1Cgx26skW6HSie2f+2Dj1V4t83RDN90wumfey3DDeQB8f+n3rHt6nc/K0wZqGXvPWDTDNKSZUzAYDjLRMgYOvO6zMn2KI+/BqilQUSVz2+Ljt/Qg7Lwf8oW8ZvWIP4vRwp6v9pC33zf2vZ6zezLi7hHYVMKRQq9ux1EUan8IGeRVJ47WQlQl8WdGrEXnfjSXyU9ObtYzL/r5Iq7ZWKUuUiZXS59lM4oxYG3fqQdaY87tJP5aCGkb0lj/7PoaSWwnJE7gxekvcunASwGxB/M68acywKD/QtzZXnpgG0DoUDjzR+Fx4gFODeAis/CSVmlVaPw0PveGUelUaPzFAr+61KeT+OsekYq0aRHkrPZpPVoLarUwTAEcH3wcJq0AoF9EPyoerCDlxhRsFTbeHfkua59snfxN5kpeQ6eDA78cIHN7ptv5u8fezf9N+D8GRtWeTwwQi5/0pfXmFeloCAwEhWRn4H4DrLsQu8NOr1d7MfTtoRSbi9n+7nb2fLmn/gc1EyqdCtvwkZSGdm1/HnxtCEolaLXwY+od5A9eBbpw17mEoAT+uOwP1ixa4/N6OMmioAEJjP1oLCcSThCsC0aqyAZzPlgKfF6HjoJ6I/7CR0LsDJEnpxpkWcZeLUn6+mfX82rPVyk43Ljv3pnj79RxabfasZgsbscuHnAxv1/6O7eNvq3qYME2+ONMSPu2UeV2dOj1MDx2GeMUFwvnp2pwRoY5JSGro/hEMeueWUfGtoxGlTfsumH0uO1sZIUSm0qsZwI0Aag8RWIe+Qh+7ityEHXCawjUBhLtH42m0nAcHAySbKfg7xSydjQtx1FZXhmpS1MpOemu73yw4CAA4QqhotGRndQaDQ+kmJP4M1qMrgiR+qDSqTi66ij5qVXj1DkHQ8PlPq1lVkw5JmRZrj/ir3gvHHgFOWOF65CzPKxFULwPinYCjdgflaTC9zEiT2BbQOqLsO+Zei+rHqV56jypC9Zx6fJLGXP7GB9V0jNkWcYqVUZTG+qO2iTlcdhxj8iD1ljsfwGOftyEGrZROCxiXNpM6HTQLed2JuzezTT/u8V5WcZc0TIe9dGDo5nx4gxKE4Rkk0vq8/AHLgN5g5B4ESQs8HzOYRef0wg6XVUwcvV3o91ix1pu9XxTEzHv43k8ZHsIhUrhOeIvdx1k1VTG6EQDkPoyHHid3oEj6JP+FMM1lwAw9+O5jL9/vM+K9YvwY+rTU3nG9gwvWgZiDP0eveOE2FO2J5QchF0PQ2AfGPE66HyjXNYgRIyFGVuhy1wAt4i/QkMsD1kfYvRto31SdN/z+jL04aHIChlkicnRr8G6SxoeUd1W8M99sHqeyJndClFfXkH6z7DuIjAdp2t4JH3SnmFY3vPIskzyrGTixzZPuUAfoic4Mdj1t789Ab+KZCL9okTqsrknTrt0S95Ap45SC+HIiiP89X9/0X1ad/yjqxi9gVED3QgGs7lKEtJrxJ8usuMNDm0YdDm31tPB2lAwQknl5shusZO1Mwt9qJ7Q7qE+qVJJegklJ0uI7BeJxl/DM1OfIa04jRFxIygSQWQkR+2HIx9CVPM8IdoywsKguBjyigNJrFywVydc1X5qJj812Wfa/PXBucnXamW+vfBb4kbFccnvl7jOXzSgZg4Wj+h2RYPyinQ0+PuDQ1aSq5xGdMgQ1Eo1aoUaq8NKmbWMqzdcjVrveybOEG6gYuJMivbDzpJV3PXufSyI7sk9QWXQ43qImebzOnQU6PVi7jmVKNKpdEzuVvu76qNJH+GwOVi0uvl5Y5zEnyEqAH1/PaVHS+mh7wH+iTD3ePtdHLcC6s3xB5Xfp+yK+jv/6/P5bt93vD/nfS4fJPKYJE5IRLbL6IIbxwDUJvWZvjmdD8Z9wPQXprs2hrEBsTVlPgN6wNAXIaL2vI6nI/R6WJcxk7jkOM6uzK3gRF0Rf6UZpfxx7x9MeWaKW7RlfUicmMhBayK8DFqVlssGXoZSUUtOAtkBsh1Uraj/3B5hKYT1lwtJuGo5q2pDUBBY9MGEvfAAU25r2hYuY1sGX875knPeOodh1w1zHU8rFjLnQY5EoBo5BPD3uSKvzJnfNanMdo3N18ORD+B8k5uzRKA2EIWkwCE7KCwvJCagfmNceO9w7im4B42fewSQwSDm4IYSfwMvGcjASwZic9i4tvBaSswlBOuCPV8c3B/mHKMkT4xdjaaasqcmBKatF+O3MY6RKgMk3yhkB9sCxnwM6npIMyDaT+w7nFKfUBXxp1Ap6D615ZI//fPhP5hyTAy6bRCyJIjlUL96Iv7MuXDgFSH52NjUHTO2uCk6tHsMfFR8EJR1hLYLpaVdyDkGQVmp9Ni2hNzu82Buy+W/unLwlZzcOBpV6TjUKhk2XwNxs+HMH5r/8Ny1IvfdiDfF/rOjoiwd/pwOSZcj9b0HS0gKaeo/+G5vEteMm0t5YTnPhD7DkGuGcO47tduBmgKFUqyHncSfZK9G/E1d3bkPaSoOvAYKDVqtkOVzvnP7Luhbx03eg7M/M8viWReZzuQ2lNKyQSg7IdKYDHkO+tzRunXRBENo1brRLeLPqsCBb6OKyq3CUKFw6Ojmvw5O/AhjP/VhiT5AwTaRruqbYJh9SBBZ7Q0lB+D4l9DzVqLDutI9+26P6T2aAofdQd6+PALiAtCHCIPG1KKvOHEChjcwXW4nPKOT+GshDLtuGN2ndSe0R92kkzO/n0p1yqa7E57hsHnMA9QjsD/Rh+YTXZmrpCy/jHdHvsuIxSOY9cosn1Rl33f7+O3W37jizytInJjI8NjhDI8VOTROVDpWH7PNgfOKOnTuorAwOHIETDnHIOswRE6EasZCSZIYd2/rGXRdEX9amPT+uWgDxEDTNcW9XXH6hZoFVtpW/nL8wIWVyrl+Gj+KKoowWUx0ie9S+81ehqUyeCjPeoLN6ZuZrHdAyTaImdFidegI0OvB37EPvyNfQNB5EFJPtGslArsEItu9sxGuIovkqmjp6rnE6jFKNmn8dlDUG/GX/Tf8NQuGvSCkBgGFpMAu2ymqKHJd1m1KN7pN6dbo8muL+NMF6Rh0+SCiBkbV/QC/rtD7X40ut6NDr4dMYzJ7SpI5+5Sfu4v48xDxFzkgkqs3XE1Yz8bL8jrHZYQmgefn1REt0n2R+DQRp+34VQVCzl9giPN4usxaxgMrHyDTKOQ+g4OVIEmUlDV9DRnZP5I5H85x88g1WowUm4Vsjp9d1MVtD+KwgXSaGj0De0PsLLCVgaaKmFFICoJ1wRSUF1BY0TDiT5KkGqQfCOKvsLDhUp9OqBQqXj+7AbJlfl0pq1Tfcsl8uh5SqQ1tt4AlH/QNiCbw7yYiDwDs9tYfvyGDGnTZDcNv4IL+F9A1qCsvV6r0V5wSrOmwO1wEgC/xzwf/kLs3l8QbEgGQHCoCdPVIlw17EXpcC35NyEFuaLm1eWvAz0/YUA4cAKsugKLIXsR5zYO6bhz4+QDb393OpCcm0adoFjnloFY74MylwlG5oUh9VUSuTl4lHKCqQ6EW+2m/RG9WHWhj869CDbINEPNNadAG9obdwZf753DNuLlo/DX0v6g/XUZ59/d8fM1xVDoVcSPiMNuEoUCqLvUJTcuF2gmY8BMotah+tWDSnOSY0Qz4XjY8a2cWf9z7B/69/CFUkEXtUiEofAzM2AZB/T2ebtHxK8tC6UpSgMqPnmE9GdvlDIzHYkGWObQqjeBIjU+c+5ctXkZWehYMBqWs57vcb+l7jaX9jcuJv8DJHyHzN9CG1399W0TPm6HXLaBQE1hpi7PZxBr2p0uWkL45nTvSm0ZSl+WV8caANxh+43DOfl2oFTr3ohoNItdlwVaIngr6euwJnXBDx2UfWglKpWdvaP9of7dIPyf+PPonerWeAZED8NP4ucl8eu09tuthyP5LvGjUAV56aBvAsoGg0IiQ81MwMeZcdh0+l8GVPKs+VM/kpyYTN8KzccUbSBiXwJSnpxDep+ZLvKhI/BsUhJvhoCMirHKPE1P4NKx6E+bngC6Ca5ZeQ2p+Km+e/Sb9In2Xa7E+OA0reoNEv4VV9ejduzfpJensz9tP99DuJAYn1v2g9J9FUt7Qob6rbBuEM8dfSTWVMIPaQFFFEWXWMkozS7GUWppkZG4MMrZlYPhpDQFRIylzFAFwVNcD5m/GuWHsRMOg10OYdIiIrMehsJsb8ffj/h/ZnbObOb3n0D/SfdMx7+N5XquDc1F39KWlZK/eieYWjcgllva9kHRKOL/WSVGpVNK7d2+v1aW9o94cf/poiDwTdFULZme0SHXir6moLcdfZP9I5n401+3Y61tex6A2MKfXHK/njutocO7tPfXroiGLOCvpLIbG1JyPNH4auoxuvJHsw4kfUliug8gLvebJ6Qmn9fhVKCudwTzvHbRKLS9vfhmH7OCF6S8QFCRImZJ96WRuVxAztPGST4FxgQy+YrDbsfSSdHFOG4hkFZO8G/F31rJGl9Nh0Pt28fGAEF2IIP7KGy69aC4xs+uzXYR2D6X7NOHp7XxnNzTi7/DywxizjAy4ZEDdJJXdLHJtR02gokJ4bXl0KnXYhXRn6DCY1AA5O0uxay/TJsav7ICKXFD5idw5tSApJIkkkgBqRPwBPB/3PKE9Qrny7yt9WFmBeZ/Ow1JqIdecC4DKEYhGU8/GX1JA8AAoyxAOA+FjwD+p/sJs5VCRLeZ+ZRsieZoDUxpkLBNrmaA+VATs5YD0NScLY+kSfC2HR1zAwBYKpCg8UsjBXw5yxj1nVDk+aRSNT7GiMoh1mcNS81zEWDjr1+ZX9hS0ifFbHbpIOGe/608/jXg5Gs0iZ4pSrWTB57XIoTYDPy76EY2/hhv+ucFzxF/WStCEesz32ol6ECjkwwulPfw5sD+bysJ5mlw+P/tzCo8WcvPem31SbHl+OSfWnkCKkiAUumiLiDF/DmUTanW2apNQGYStacWZIuJuwlLXqRYfv2Vp8GNX6HU7DHueO8bcwR1j7mDBMrAg8/U5H9FjRg8uWtpA9axGIP9APkVpRYL4c+gFCdTQ/KltCQq1yB2dcF5r16TpUFYtJDUaKA/6B5Mjn5O5wz3awRv1aLWSCf83gdgRVQo1zqANjQYhubztXzDl73ZN/NXGGfkSnTn+vIzaEjWaS83YzDU1iC/57hLGvDeG1PxUwAf5/QAsRWA8JDZEHQkRZ9YqM3NqxINKq2LcveNImtSADVITETMkhjPuOQP/KH/sDjuvb3mdz1M+x2K3uEiSHqHboWi3z+rQFuAk/naXnC88gisnh83pm1l7Yi0ZpRn8dN1PvD/u/Vapn3OTX91ByuFwkJ+fz6+HfmXKJ1O44ecb6n6ILMP6S2DH3b6raBuFk/jrYX8Zdj4ECOIPwGQ18cPlP/DOiHd8Xo/SjFK0x1JRV5RitFeLEJMkl3xhJxoGgwF250xgW8weiHffUL+17S0e/PNBNqdv9mkdnGRRQFIESWOSKHikgC8XfAl7noTtt9fpCeMcv62RKLktot6Iv8BewoDUZY7rkDO6sqC8Kp9f+pZ0Ppn2CQd/Pdio8p2Gr4aQRfesuIdFPy5yl6jc+zQsGyyMm51wQa+HGP9D3JUUXiO31riEcVw68FL6RniWTrJb7JRmlDaqPI2/BkkjNtWyqtzlBe8Ru5+A9KaRQ6f9+K1NPhVQKpRE+YmNbUZphiuHsubbL/jtX795rQqlllK6BnUlKTjJtcHuVB2pH6/MfIVfLv6FPhENj15w2B38uvhXtr5Z5bRY7zv7FGx5bQtLr16KxWEhvywfi90DUQAiT+rqc+HQ2x7Xvi4olNDtSuFBXR8ylwvDX+V4bxPjN+17+D5a5N1uIDx95z1m9iB+XPNy0zQUQfFBRPSNoKhcbBDV9iA0ddkw8zbDr8NEpEDBNrEHyf6rYYXlb4alSXDo7WbXu82gNBW23AA5fwNQbkjlQNwjHA/50HWJuY4py5sYdesoHrI+RF5SHpnKDViVRU2LLOp+lZBkDfI8j/sCbWL81gF/jbBdmawN9IpoIiY9MYnxD4h8cwv7LWTZxb+SmCkcPtRqYM0C+Kd+Oe5OeIDVCGUnCak0blpkYewM6R5CRN8InxWbNCmJ+433c3CU2MMMC00lOfcSKNjuszJ9BrsFSg+C5L5ebPHxqwkRa4WwEW6HDQZAkhj5fzMZfuNwnxR92fLLGPX7KEBEbyYY1kPBDp+U5VOY8+HkUig93No1aTqsRshdDyaRy2pz4nls7DWF7Sf3MPm/k7ng+wua/Gh9qJ6Jj0yk59nCYSC9JJ1v4ruyps9IsUaKO1fIZ9cSAdte0Bpzbqd11MuQa9H//ub8b3gm1D3xuN1hJ8eUA0C0vwiJdkp9epX4G/4yzMvoeMbwEa+KtnmAwQAyMkVlxhaulECxuZibl93MJd9dgoTkivibHHgD/NmxZQidxN+u7LNEDpDKvBthBnGioLwAu8WOraJ1kvE6jR/mlAM8F/Mc+3/cjyzLpKWluXLcJATVJ6Mjw6j3oc/ptwlwEn/d1d/AobcA8FOLjVmZtYz+F/Vn9B2+SexcHb1m9+LYZQ9REDcAk70IgK5qFWStAnNB3Td3wg16PZTbAimw9a0RFe6cm7KMWTXuy9yeyaoHV1F4pOHRDrXBSfwlXzGWy5Zfhp/GT0SAjXpX5O+pA87xW9v8e7qhsUZkgHCD8NDLc+rBAVaTlZMbT2LMatw8WlvE3/Z3t7P0mqWYS4U1rtRciskqvLhj/KtFLjlsQkqmo0QmeAl6PZisQWSU9W+YHF81fDzlY94c9Gaj7rn454uJvnk+AGtsz6F7QsdNv9xU80JrKex6EI5/0ajnO3Haj1/jUTj8HpRnejztzIGZUZpBcLA4ltNvEqNuG9Wk4r679Dte6fkKsqPq+x4eO5xjtx1jx/U7ahJ/xftg/wtgPNKk8to9Sg/D9juFRPIpmJk8k1nJswjVNzx3uD5Ez6XLL2XO+1WOFw3Ky1oNEx+byIVLL2TDyQ2E/y+cIW/VEoUSMx0GPwNJV7rWvjWkPp0Y+hz0rebMVp7p7qiY9QfYTCK6Th8tcgfSRsZvUD9IvklIkNaB4opi/rfufzy46kGPEX/nvnsuk59omRzspZmlmEvN9AkdyJl7djLs8Dd1k0W2EjDnifkxfBSM+wZiGkDUguiv3ndC2Eiv1L1NIGQonLXc5cAUqK/KNaW0lBO/+1eMW/a2aJWu/elaVnQdS4H/egxlm+Arfzj4hncevvUW7z2rGtrE+D0Vh94WhnHAX+vcX5pcp//49x/8+fCfXi2y/wX9XSpAicGJTEmcQVD5YKDSiW3YS9DrNq+Wedpg87XwQzxhBrGosEkV2Bw2Zr48k/O/Od/nxTsjOHfkjOZo/Hc1SKt2gUNvQUUW9Lje7XCLj191AIz+ABLdI/qcqgXx84aTPDPZZ8VH+kUyLfhm4gouZlbQJbDpGp+V5TMUpcDqOfCT774nn6NkP6w4A459BoBOErbenOKSuu5qEvLL8ylTn6Bcc1wQf/6JYt7XNnzd3RbRGnNup9RnCyFpShLBScFux/LL87HLdiQkIgzC48VUua4J6ECKnK2BQvkYy4Ymo5BUvC6XIUkSXy34CiR8tsj4cMKHBHUNYt7H81yyP35qP9RKNcUidQrHdHcQPsD7L8W2hPDKCO+8PPfjTsNIfnk+N33owXjYQnB5PfupCOkWgi64yrh8suQkAPGB9Xj8SgpI8L7USHuA8930wcGveORRMYU4I/7KrGUMuarlZFCsNgkkMNrEeBvsSIdVk+Gs3yFmWovVo71DGAJlbKZ8qAB0VTINzmiTbGN2jfuyU7JZ88QaYkfEEtKteTKNtZFFDc032IkqNMiI/M+/hZG0Msefk/jLLct1XZI4MZH7Su5rVNkOh/hAzb489ucxUj5PYdZrIs9uRqmI6AvUBuKnqaZI0P8B8emEG/R6KDFH8L/tf/H2KUHp+WX5rE9bj0JScHbPmvJifc/rS/GJYmSHjKRouI68c1xWSOId66/x4JWmNMCslNMy561XkLNGGC/Gfwvx82ucjgmIgUzINGYysVL5Jj16KMnnNq04vyg/ghKCPP4OJEmqSfzlroHtd4hcd/UQKx0S5nzY/zxoIyBqglce2W2y+/dYrzzzKYgeFE30oGj27hfERpC2lhQC2jAXmedc+zYoklOWYfVc8O8BZ3wGJanw19mCSJywFGL2ti1n0qDeMOK1ei+zOqzc84dw2Hsn4WFAUyPHX0vhtT6vETs8lnnfX05guVjn1En8RU+Bucer/m7MHiSwFwx9tmkVbavQhroRn0F6fygFu8KI5LATdWwz5lQZ8H30XP6BfIpPFGMrFw6tSocWpUYHEeNESoiGwlwARz4Qsn5RZ1Udt1sE6ddlrnCo7ejYfqdQdOpyLoF6A5RDua1qQbv/h/0o1UrOeuysOh7SPDjXPlA5Lrtd4bOyOjxiZoAuilDJ4DpktBhdKQZ8heITxWTtzIJK39S8sq6Uhw2BelKptklET4H+/9fm1mA/7v+RG3+5EUP4aPpkfNfoPMWNQerSVPShei4NfZWvMmCbNZmz+rZDqc/gAeJfRTusuxN+iTDkOYgUUdJ6pTAO5peWcnj5YQ6vOMy4f4/DEGao4yGesefrPWx9YyszXppB1IAoCsrEAFbbQgTxJ8vtL69jG0En8ddCGHtnTUlKZwRFuCEctVKs9r0e8WcrE57EEWNF7oaOhJzVcOxz6HM3BLgL+XcJjkJW2LBjo6C8gDBDGBZTLVI4XoIsy67UYoUVldKDlTmLnMRfWcSF0MPT3R0Hzog/f+s/8Mul0PsO6H4VoTpB/FWXkmsNODf5UWO7seAesYCy2+2AMKxBlYd9rTiNJx0n8ZdRGA2VnOklAy5hQtcJ9AhtuR934dFCNCcLUejjKLUWAVAaNAi6T2lRmZyOAL0etMoyZpZFwI7LYcxHrnOuiD9TzYi/XrN7cdOem2o4tTQFzk32/tdWsjxsG3tH7+XK/uczMfEsULXHXVrroUFG5AOvQ/hoF/EX4Secj6pH/DUFtmqB3KdKfc77dB6zXpuFSitOOIk/t2i/TtQKTxEqTuzJ3cO5X55Lz7CeHom/Ubc2LjrMVmFj44sbKc2LBbpRLot52ykJ6waF0hX904kmIHoynLlUjEcPiPWvivjz9welEux2sa4MD/d4S52Y/tz0Os/XIP7iz4PAPqdvH4cMgnOPiqipU7Arexc7MnfQK7wXo7s0XOnAYXdQklaC2k+NX4Rfo3P8OQn8YrPYXARqA2telLNaREVV5ryrU+pTVArWXSByjA17URhrrZWbl4BkGPi4+K1C2yL9GoFQfSgqhQqbw4ZZlQN0cXufbn5tM4WHC5n+fN1jxBsYes1QghODXWsfSRJju1Fw2EBxGptyZBkcVlBqCDYI4s+mNGLTGtg55Q6GjGkZveLt725n/f/WI/2fcEZUyFoUYYMgqZFyzDYj7LhLRJZVJ/6UGlhYIs6fDpj4i8inBwTq/KAcKuxVEX/Xbr4WpdZ7+ZEqiit4a/BbDLpiEBMfmcia42vYlX6QEv1wAssHNk22tRNV6HYFcAWBh0ByqJEVVowWIzkrc0hbl8aZD56J2uD9L/nw8sP8dO1PXP3s1XyZUYTWGuPTfNU+RVAfGPhIa9dCYMtNoIuBAQ/hkB1kGjPpohLOwTse/5nNR9K4cZf3HRR+uOIHYobGYLvkcgCOchlndfV6Mb6HNgwucoi5q71CFw597nD96acKBDvkG0s4sfcEG57dwJBFQ5pE/JmyTWTtyHKpkuSZKok/eyXxt2Y+5K2HeVmnrS22qWifK/cOAifx5zSsgg9y/BmPwrZbIe0HLz2wDaHkgAh9L0mtcSoiRI/GKgyZxwqFfOOlv13Kpb9d6rPqLFq9iHmfzANwRfw5vZmcxF+gh715R4OT+Csq1eOw24QsEFURfwXlBWTvymbzq5sxZrf8JqY240dAQIBrTMYE1GOI3vcMfBcNRXt8UMO2DSfxJ1fki/Y7rNw88maenPIk/SP7s+3tbXw06SNMuaa6H9RM7P5iN4mrP0FXVoBWrSFAE4AU1E94wxq6+LTsjgaDAcx2A/vsN0KUu8xVlH/tEX/6UD0RfSNQ65u/YXMavw59spGs37P4aOdHKI9/Dl/5CfnWehDQGSbvQnWpz1qVJM5OgXFLXH/GB8YzMXEio+OqjNd2i53UpalkbG14rr0aXtLVIEmSW4R1rY4WRz+BE183uMzTBc5+PSv2Rdj/otu5ML2YePPL8vEGygvLWXnfSkzb9ou/K12mPUoaVuSBpbhZ5Z3W49cQB11mg85zrhvneiSzNBNJgqAg6LJ3OZ+Mfs1rUjHX/XQdY98by/LDy2sSf9pQ4dWraV5Ud7uFUiukhTxID3+5+0uu/PFKvkhpnMxtxpYMXkp6iR3vi/w0jZX6fDrkab6c+yUlZqEgUoP4sxSKtAKr57oO1Uv8KZRgPCxytijUMPBRQQCCIPr63lOrA2mbGL+bb4Stt9Z5iUJSEOkXCUBFpaGyuoPMwV8Osu2tbT6rYnVMe3YaIxePZPXx1RyM+Q+FoSvqviFzORz5WJB9AH9MgGUDGlbYjrthzUJB7nYUyDJ8HShyrwEhBqfUZymSQoFVF4BFbpmIij4L+nDOW+dQrBPjUZI1TSOL9DEwdb0Ya6dCZQBdZPMqWgvaxPitjsgzXY4mQQahBmF2VL0ctYFalwOZN2C32NGH6l1k4kc7P2LxiqvJDvoZpRKksjT4sRvs62BRsy0MvR5UDjFOjRYjh38/zNon11JR5Juw664TujL3o7ncctEtDM59hlsGPkn3HaFizdqB0OLj9+QPkCmcGpwqIFaFsOfZZAUqrconMobnvncuI+4ZQX5FNjaFqe6cuG0dkiQcOjoIAirTOhWUlTDqllHceuRWQpObJsU5cvFI7i28l+hBgh/JM55C/AUPENH0naRfo9FJ/HkZylrc9X5c9CMbnt/gdqxFiD//RJj8JyRd7qUHtiF0vQDm50BszZx5wcGgtwq5xv2ZaS1csWoRf7qqiL9+Eavpd7S3SMzegWEwCMNCemkvsobvhx5Cf7s68Xf0z6P8esuv5Kd6x0jZGFgqAz/z1u1nzX/XYC2zolQq6d69e8MjUDRh4JckPKNPMzjJ67MTn4Bl/WvkJSo5WUL2zmysJt96MnWf0YPjA87GrA/i09nfUHJfCQv6np7yq82FMDpK/Fn6OnRznyvqyvEnyzJleWWUZpQ2uw5Owmj+6n+x87KdAEiGeOh6EQTUHUnqHL+1zb+nG5xGZFmuit6pAb+ubob8AVED+POKP3nt7Cq5NLvFzpdzvmTzq5sbXHb1iL9Tu+PY38coOFQV8e1839Yg/nY9DHueanCZpwuc/Tq56zvIB93z9Tlz6BZWFOKQayYMz/oniyXzlnDo90MNKssQZuCazdegnTgGgDLZXcXADbsfhW+CwdS0tVbn+K1ELeSpc3w4ifKgIAAZWVJgtzTOkG8uMfPHfX9wdNVRt+PbM7ez4eQGLHZLTeKvIrdjEQZNQdlJKN5f47Bzje9c8zcUYT3DGHXbKOJGxAE0KuJPlmV6zOhB7IjY2ok/hRaGPg+9b3Mdqpf4A5i+ReTVtTfcENtmxm/eevGpB841jUkSa5rqEX/zPpnH7Wm3+6R6teGvE3+QGvcQ2cH17A0PvAabrq6KtgwdXmuUcA0U74PCfwS521EgSdBlnus7CAsQhhOHspzoWDtaUwGW7Obnn24IuozqwrDrhmFSCodHpUOLpmAl/HMflKU3/EEKNUSMqZnDt+QgFOwQkp9eRpsZv9XhjOQEYv27MPLAb1zr/7PrdOHRQk5uOum14vwi/Lhu23WMv0/I1pntYhJUyjpB4Mp2sV5WtEwEaYfDiW9g9Vz8OI7SLsZpqdnIuH+PY3HqYvwi/ep5QNMQlhzGoMsHERAbgNUK2aYkrAFDXFHwHQGtMn5nH4apa4Aq4s8mCQN21JWzuHbLtUg+IGX6zO/DmvA13JMbzYHu53OuohekPOr1cjrRQKwYBxuvBqrWoEXlJRjCDYQkhaBUe+c36Yz409hCRMTuwMdEaoR2jtaYczuJPy/D4ahpcJFlmZ2f7OTE2hNux50RFNWJP6fUZ4OcNxriTaHyg6iJENiOE4jWBnWA8JD2IDmjUEAggvg7kCWMUWnr09j82uZGG0oagtx9uWx4YQOFR8TLqaiiCBBGMocDSkpAo6xAoVB0+IWjJFVF/eVX4/VC9aFolaLtfeb14fJVlxM1sOWJM6dRK/vPfax6QEQSORwOTmacJMeUAzQg4q/HNTB9g1sutNMFfpXr8+2Z0zEnPwwqP4oqijhccJgcUw5nPXYW9+TfQ3BisE/rEd4/htyuw7FrDFWetfueg5/7gOl4nfd2wh11RRu4cvyZakb8ATzf5Xl+uu6nZtfBSfwFxvqTrRNlmaOmiPxCfgl13utwOMjKyvI4/56O0GqrHOFqNSSXZ3mMlq8OtZ+aOR/OYfgNwxtctrMfVSp3ZzzZIfPxpI9ZcXdVVEOtjhbjvoYRrze4zNMFznH62N8/UXGGexSs07HGITtc64/qsJgspC5NpfBwwwyhSo2SuBFxOIIEsVHmcHdmckP4GdDtysblMqqGzvGLiMb5PsalkFAdF/a/kMw7M/n+gu8BQfyd7DudwW/d2Oioh6LjRax7ah3H/j7mdtyZ3zguIK4m8ffbUFhRM13BaYWVk2FtTcciJxHeWOJPH6pnxgszSJqUBDQux58kSZy35DzOfODM2ok/lQGSb4C4c1yHGkT8KVSw9SZY2h0sRQ1qS5sZvzO2wYyt9V7mXNOYEOuM6sSfIcyAPtT30uLGLCNfzv2SlM9TKKoQhL+WeiRhBv1H5Fd07nmHPgejP2hYgRN/hnP2NaPGbRRjP3blA3YSfwBxiSZ6r3sP9bKWdbR1EkYKWYsifw3sfUpE3zYGdkvNiKQDL4v3sDnHSzWtQpsZv9Xx+wj4uTcAwX4GIkumE2M9w3V6xd0reG/0ey4pOG+jwiZeCgqHVhia/RNh5jbodYtPyuvwKD0I6T+jV+STkHcN3bLuJlAVTkBsAGE9w1CofGeOdsgO9uXuo4ij/Jh6K8XDVnqM3m+vaJXxq9K75iEn8WdBEH8NVS1oKsqtYpGkR4ssaYHOqK9Wg8MmnCKAIL0gLkrMJdgqbBQdL8JcWpvncd04uOwgB3896Pq7oFJFTyuHdKggv9aYc9ur0nGbhafQZkmSeLDiwRqE09TuU3lR+SI9w3q6jjU44s9hg29ChW728Fdqv648u1ZyrN1DlsViQrYL7etTEKaKJw04ki+Iv5TPU9jy2hb6ntcX/yjvevucWHuC5XcsJ6JPBCHdQlxSnyG6EEpLRVV3ZE1DPnvvaTHqwsIgPR2UaZ+Bnw26XcFVQ67imqHXuLyAghKCWqVuTsP08PumMPWhUaj0KhwOB5lZmbwx6w2yy7KJMHiW3AKEYa4jjqcGQqUSRqodWdPJi51OnBYe/vVWXtn8CvePu58nJj/RIvXwKCkoKSo/p8Eg8yKchMIo3UOw/hiM/cR1LikkiT8u+8PNQcUJSZIYfdtor+X4U9itGI8VYywSE6HH6CIPkGWZrKwsIiLqGLenESRJjFGTqQ5D8vpLoWArLCxyO+yMFlNICiRJYvAVgxtVdnXiz+25dgczX51JUHzVe/+20bcxs8dM4gLj3C8OazjReDpBoxF9m23qRjlQ3TytUWrw1/hjtBjJL8uvIcnZZXQXHrI+hKRo2K7JbrVjt9ixWNSAhMkhIjU9Sn0mXig+TUTn+EVIaaoDwWYSTm3VEKgNdCN2goPFv8WeAwTrRHivcBanLkbjXyUxZLFbXE5PXQK7uBN/sixy/J2G6gZu6LkYTwYmV8RfefMiixqb48+JWok/D3D2a53EnyxDxHihaqEJblAd2sz4bWCuO6d8eamjJvFnyjFhzDIS3ifca97qnlBeWM6Bnw8QNSiK4m5iIOukevZEwQPEp6no4LkAQwN1nLFvPUqHPz0XGFjVbQwBUY3PL9QU/H7n7xz46QD2S4SdR6PUIvW5AxIvqddxrQZWjIOKLJhbzVk8fgFoI5vsXFMX2sz4rY7oqWAV7zZPThEDLxtI/Nh4V67T5qLgcAH7f9hP8qxkIvpEVBF/zoi/TjQPfe+FfvehkaFX1lAxzahFLunygnJ0ITqvpIw4FeufXc/GlzfyxNwnKOhdwMztJtTqlnkntBRaZfwW7wNLAUSc4SL+zJXEX3FqFhueP0r/i/oTEOM9CdKKogpeSX4F81Qz9IJiaxCr/XcxuxlTYieaiekbXf+dGDeLvVsiiA8fyaHfDrFk3hLmfTKPgZcObPRjV96/EovRQvIhEbSkVwTiV9GTAIcI6GHbHRA6BJIu80ozWgu+kMOtDx17FdiGoFApani0DI4ezODowW7HGkz8WUsgYqxYBB77EuLne9YK/uNM4fk5c0fTK9+W8esgiJwIZ/1a41S0Ph7skFYsiL8RN42gz4I+6IK87+nTZ34fIvpEENFPTLwL+y2kV3gvYgNiKRFrV/z9axpDOyqcEX9xxU/BHgt0uwLlKRIzskPGYXOg1LRsqLNT6jO4SwCxcVWLErVCzTWDrqk/9Hr9ZSLny8i3O/xGujYEBAgDlTNC2U8twgDLrGUUnygmY2sG8WPj8Y/2nZzGH3f/Rr8/D7F7wtWM+3AiIfoQvj3/W/x6t6xMU0eAk/iLUm+GrJ1u53QqHZO7TfZwl8CUp6Z4pQ4WC/gVZbBk/IfEnx3P8RHH6XbsLcgJh0EtQyZ3JOj19RB/iZeIPCrVMPStoezK3sWGqzcwIm5Ek8p1Sn2eaixRqpWMuNH9mQlBCSQEnWIUkysTnis7dmR8UyBJIuLaXlFKRV4uBHd1k24L04cJ4q88n2TcVR4UysY5qxz+/TBfzP4C5YI5wGBGBJ+NKiTDowNAJ7yAXnXnJquOoCDQl2ST9vUxSkf1JSC24cYVpUZJWM8wt2OZpZnIyGiUGsIN4e7EnyTBsBca/PwOi1oiPZzOKQXlBR7P14W1T61l//f7uWr9Vej1Ynw2hPgrOFzAxhc20u/8foyIHUHJgBKGRA9xv+jPGWAzwtS1rkPOuaBO4k+S3ORB2xVMaVCwTZDo2rBaL4v2E++wYoeQ+qw+R657Zh0bntvAv479i+CuwT6rakSfCB6yPgQy/Oe9lwHQ10X8yTLINiEF6UTRHjj4BiRdWrfkZ+E/UJQCsbPq/F7aJY58CDmrYdS7+PsrCDEJaeoe3eDzHuNw+CYlXg2otCrUejW3DXuIr5aW468IE44c6voJ+RpIWCDklasjaqL4nC4Y/KTrv3o9pIV9SLmllFLzlQRoA+g9p7dXi8vemc2Ku1YQEBNARJ8IzLbKyE1nxF9JKqT/DHGzIbBn3Q/rRE1UOktLkujPsjLx3j36zU5+vv5nLv39UrpP6+71YjUBGvRReuxKQcgv6PkahhMqCO20EzQL2/4FeRvh/JKqiD+5DBk7xr3HWf7tcmKGxXiV+JMdMhH9IjgWfAwApUPfvnP8dTBMTZ7I79kTMcgQ1iuXMXeOIaxX09YbM16cgbW8yrv/6t53s+uNuwkNRUTEp74g0sC0c+KvNdCqYSurV69m9uzZxMbGIkkSP/zwg9t5WZZ5+OGHiYmJQa/XM2XKFA4ePOh2TUFBAZdccgmBgYEEBwdz9dVXY3SyZ5XYtWsX48ePR6fTER8fzzPPPFOjLl9//TW9e/dGp9MxYMAAli1b5rV2WowWTqw9QWlm/XmQGkz8aUPhrN9AGw7rL4JDb9a8Rpahy7nQZX7jK90eIEkw4DEhMeUBvYMHE104jzhGARDRN4Kks5JQ6bxP1hjCDCSMS0AfIizo3UK6Mbf3XEbGjaSoSFwzs+f7IjH7aQAn8bfK9L6QbTsFeal5PK5+nL8f+7uFa1bl9WwrMmIxNjJfgt0CFZlQkXPakn4giL/+EX8Tu+8syP4TQ6UHXZm1jKN/HuWrBV+RsTXDp3WQlEpkhRLZUMLmjM38fvh3dKqOI9/RknASf2/u/xXm18zl52vY7WK6suoCGH7nWI7GiPxTATnLIaOmU0cn6oezT2sl/rovggEP1zhsl+3kllUZnT6e8jHvn/F+g8t1Rvw12UvaeASW6GDng018QMeGnx9cOvBBYrd2F1EB1eDM85df5jl37vE1x2tIztcG/xh/Bl0+CHuIeOYdyW/x00U/1YzOLM+GlVPg6GeNbEknGoO7lt/FBd9cQF5ZHsHB4F9wgsIvfiN3X26991aHKceEMdvoJpGWXipyUMUFxAFSTanPTtSKpub4AzBmGynNLKW8oLxREX+FRwrZ8toWcvflsmjIIj6d/ylzes9xv0gXBTp3CeUGSX22Z2T8DGvmQWHdjq7XD7+e7ddt59ZBYv6rHvHXfVp3Jj460S0i1leQJAlJIVFqqYxsUtZBEtmM8KUGNt9YdcycCwdfg7xNdRd04mvYcDmU+3ZN3irIWQNHPgBrCSGVAhFBQeAMfqk1x7GXMfm/k7lh5w1c3f9uemY+jF4RKHL7lTXhO+97Lwx91vuVbKcwGGBPwq2s1N7qMde4N5B4ViJXrb+KblO7AdSM+CvYBjvuguLdPim/w6MiBzJXQHkWSkMxZZpjZBUVEjUwipG3jCQgznsEUXUMv344s1fOpji4GMmhZnr3d9GcbKA8cidqR/INMORpQEh9DowaSB+/sTgkC46efbhq3VXEDKknZU4joQ/Vc+VfV1I+T2xqu6itJFlehuK9Xi2nE41A5nJIFaqDoZViMAUFEN47gmnPTnPlsG4sEicmkjyzynnVOY9rNAjnp3kZIod1JxqNViX+TCYTgwYN4rXXXvN4/plnnuHll1/mzTffZNOmTfj5+TF9+nQqqq3SL7nkEvbs2cOKFSv4+eefWb16Ndddd53rfElJCdOmTaNr165s27aN//3vfzzyyCO8/fbbrmvWr1/PRRddxNVXX82OHTuYO3cuc+fOZffuxk/wnpKZ5uzJ4YPxH7Dr011ux1ccXsGGtA2UWat2es4ImnqJPyeSLoMBj0LS5Z4qA0P+BwMeamj12x/63g1dL/B46sy4aQw//B2DzVWeurIs47B7X1O3LL8Mh83zc51yTDMTn4J9NUnnjggn8bc/dwSEiDDvcms5c7+cyxnvn4EqWEWvOb0I792yOfIcjqqIlG+mvc3HkwURK0kSJaoSVh1bxfGiOvLDKTUweRWMW9ICtW27CAgArcqEvmIXmPPx04iIP5PVRNczuzL/8/lED/FtZMiwf09l74QbkXVigAVqA1Fm/g6pr4LD+3k8OzJcOf7KPS8Jftz/I/9Z/R9SslNqnNv58U4+nvwxxmyjhzsbBidZZPYLpf99A0nvko6EhHTOAZjyV733S5JEaGioT5KJt1fUlbexNoQbxPs4r6wqv0xgl8BGyTI736+nRrcf+OUAr/d/nSMrj7iOvbzpZd7d/i7FFdU0CxUa6HoxhAxqeMVPI/j5wT9ZU8gKubtGvuAHxj/AJ/M+qaEk4cS3F33L8ruWN6ic2GGxzP1oLrZoIa1SK5FbkQn5m5plVO4cvwj5/s3Xw56nPJ7+LOUzvtrzFSeKTxAcDEVRvXBccSWxwxon/fb343/zXPRzGLOq3teu/H6BcW4S2lotwvFi7fkiuuh0xomvYfnYGt+DK8dfE6Q+pz8/ndtP3I5fhF/9jhrVkHRWEndm3smAi+vQuBrzEYx3d7prkNRnE9Bmxm/UFBjzMQT2rfOyxOBEhsQMISZIbFROJf4mPDwBQ5hv5eBKM0s58scRTLkmSi0z3JRtAAEAAElEQVRi/jMo6phnHRZIOB9CBlcdCx8Nc9MrZWjrQI/rhQNmQK/mV7ytYehzsLAE1EHExEDP8z8kcPZjZJmPELfvD6L/+LRFq+Pm+LRxEfxSMw1Jo1F6GH5MhINvNf9ZHtBmxm91HPkQNl0LDjt6PSjtVaoyAHu+2sNbQ94ic3umV4rTh+iJHxOPX4Qox0n8KR2VxF/MdJi6HiIneKW80w45a+DPaZD9F1uibmTVwCS+TP2QLqO7MPPlmUT2811orqsvZR0PrPoTx/ilPiurNdAq4zd+PiQLJxS9Ws/OG3by6tB1KGU95epA4sfGow30jedYuU0sknrrS+hZWhl52InWweH3YNut4LChNJRQZNhCvma7i8toCjzJXzrnVY2GyrDhGNC3f/WZ1phzWzVkZebMmcycOdPjOVmWefHFF3nwwQeZM0d4Mn788cdERUXxww8/cOGFF7Jv3z5+++03tmzZwvDhIifMK6+8wqxZs3j22WeJjY3ls88+w2Kx8P7776PRaOjXrx///PMPzz//vIsgfOmll5gxYwZ33303AI8//jgrVqzg1Vdf5c03PUTS1QGFoqbhNCg+iOkvTidhnLuk1QXfXEBhRSF7btpD34i+yHKVkS6gPueXE99A4U7oc5dHr/1OQHglp5Rf6fy+5+s9fHvhtyz8eiF95nthMV4NH038CLvVzuL9YgO2NHUpJouJCYkTKCkRhpnv8pZy/blNN463Jzi/+7w8GaxGUOrRqrT8fOBn7LIdk97EBd95Jmx9CUu1AL/+Fw8kIFps7hUKBX/l/8X//fV/XDPkGt459526H6Ty82Et2z4CAmD1jlksM+QzJwEMOeI9WWYtIyQphJCkhuVmaw6cCwGHpiqfJkc+gLRvoOfNPi+/I8EZbeAnH4P0PVX5pirxzvZ3+OXgL0T7RzMgyt3YWJJeQsbWDEw5pibnTq1ubI4LisR0v4niimIUSjUo6w8dUygUJCQ0Mo9KB4envChuOPIxHHlfGIj9ugIQ4Sfc5HNNVVFEcz+c26hya4v4c9gcyPaqfCyyLHPn8juxOWzM6DGDIF2l0dMvAc7ojB6rDf7+sOXIbCb6zSb6FAP+/D51qztM/d/URkudW63gwIZDYQc8bORDBgvDq2xr1HOro3P8IhQETv4ocnj1+3eN07EBsWQZs8gszSQ6BKz6QAoDA9EFN66YxAmJyHYZv8iqNYxDdpAYnEhScJJbhIxGAxTvEaRXvwea1q6OAqsRjEfB4k7wRftH8+m8TwnRhyDLcqM28tWvrfd9XQ0KlcIlo15cUYxBbUDdgHmyQVKfTUCbGb+ByeLTQDi/h+rEX0vh2J/H+O6S7zj/2/MxWiuJP2UdxJ82rKbDoVIHhgYQ/34Jjc8z115wSh7Kv8tfZduRbUxIHo6mvBxtSS6yLGyFvsSer/dgzDVyYpCSEr2WaHV/SDivbgnW2pC5HA69AwMfg6A+YCsFTYjobx+gzYzf6sj+C45+BMNexGDwQ+kQL0iT1QSI9aS5xIy1zFrHQxoOW4UNJCHZCvC/qf9jY0oOf+wchiocMf4ixnilrNMSoUNhxJsQNhy94k8ASsp9bw9L/SmV3Zt3o0SJQtZRUB5LXYHV7RFtZfxWVy2wW+3IdtmrCmslJ0vY/u527H7CsXtXcQ/2xq6mb2wPr5XRiUai/4OVaQoktmRuYG3fGQSWDSbj8Dp+efhHeszqUSPFR30oPFLIGwPeYPwD4znzAZGO5JrVU0jtm8ds+4dg7SEcTQ1x7d4W64kz8jXarFbd0aNHycrKYsqUqvxBQUFBjBo1ig0bNnDhhReyYcMGgoODXaQfwJQpU1AoFGzatIl58+axYcMGzjzzTDTVhICnT5/O008/TWFhISEhIWzYsIE77rjDrfzp06fXkB6tDrPZjLnaDrmkMpGb1WrFbhcvJUmSUCgU+EX7MWKx+OHb7XYkScLqsLqkYSL0EdjtdkwmCVlWIMsyer2DysegUCiQJMn1XADpxLcoTnyJ3PffOOx2sBQL2Y+AHq4fkpzyKJTsRx75Hih1KJVKEfHmcI9MUyqVOByOGiy7p+PONtV2vHod6zruqU3O40CNOtZ2XJn+HXLK4zhGvicWFtXqHhLiwCFDWkEu5ZZAAuIC6DWnF/owvft36YU29ZjVw5Wrzm6389jfj7Etcxs/XvAjpYWzAbDoemIPlnF2bK1t6gD9FBzsQJYVjA54Ar5+CGbuQAoeRLghnGxTNlmlWUT7Rbd4m4ThQwFITPrvWSgUor8cDgcHMg8AEOUX5XaPazwd/gDp5Hc4hrwA/t06RD819bfn5ychyxIlJRIOhwNd5YbUaDHicDhapE37vtxBaJqCjC6C2Q/WBSMPfAxHjxtFaGcj21TX8fbaTw1tk0YjI8sKBocugb//DdM2Yg+pmlej/KIAyDJm1aj7Gfecwfj7xmO32z2Om4a0SRjelEQe38znM/Zz7gfnEhmhx565CmVwXxzayDrb5HA4SE9Pp0uXLqhUqg7bT41pk1brQJYljEaxlji1TVJ5FlJRCpiLwJCAw+EgXC88NrJN2a6yGtumigrxW1KpZOx22XU8+Zxkks9Jdj2z1FKKzSHIomBNsGttdLr1U2PbpNfLyLJEcXFVvza0TX3P79vgNu36ZBfH/z6OVT+dQr9tTP5zPH1T+rLrhl211F2BVPnub2ybbDYb6enpxMXFoVAoOkQ/Nem3N3MPqIPAbq9Rd2desozSDHoFibFdmG/HUuFApVE1uE295vWi9/zebscX9lnIwj4LUSgU5OcLUl5E7Dqw97wdRfJikFRir9HYNp1yvN32U/dFyN2uFMerPUun0nFR/4tqtLchbbKWWUldmkpgXCBBA5OQZRmTCWw2B5JUe5tM2SYsJRYCuwYy9O2hHCk6wpor13BGwhmi7qXHkI68ixx3LoQOd7WpvFy8O9Rq8e7wVj8BpKWlERsb6yqrTYynWupeYi7hzW1vkltSDDyF2SxjtTpQKERu03VPrWPKU1OIGx3ns99ezPAYZr4+k8hBkfyf6lve+7yQmOT+tda91jaVHkOuyK2x/3X1k8MKtjIkbXDbGk/eekfYTThKDiDr40Eb5so3bpaNHBmyEACLxY5K5ds2bX51Mzl7c/j34n9DX4m+OVbsSVeLulde2+A2lZ1ETvsWR9Ii8O8JgQOQpm/zWT85HA4yMjLo0qULp6LV3uXDXsI++FlAi0ZjR+moVJWxlGG32+l7QV/6XiAie2tbqzbmt/fnI3+y/un13LTvJkKTQxkXPw5VOqyxKFCrwWEtQ5YlofrT1Da1h/HkqzbpE1D0EMEWeqXoy+LyEnL357LyvpUMvHQgvea6RyR7o027v9zN7s93w4OgdGjpEnQIR0Wo633YEfrp1P1vS7RJSn0J6ch7OCb8ijIgAVmWK/ecCswnsviP5m0mPjqR8Q+O99pvL/9wPn8/+jfJdyTTW3M5lIzG5DcWuwY3u2pb7SdPbaqO9vjbI0C8gyUkV65Hm8JIXp6DwysOE9JdBAE0pk2yLNP1zK4EdAlwnT9UkkKJIQdlmYw960+Ua87FMeJtFMnXtut+OvXalkCbJf6ysoSOd1RUlNvxqKgo17msrCwiI93Dw1UqFaGhoW7XJCUl1XiG81xISAhZWVl1luMJTz75JI8++miN43v37iWgMlwvNDSUhIQETp48SUFBVeL36OhoLHoReqSSVJw8eJJ0KR21OgEIxWotZd++Kjmsbt26ERgYyN69e10/EoXhFnpOvhu1QkfKrp0MSJ1Aua4XhxLfZcCAAVgsFszH1+BXvos9ew6gVKkYMGAApaWlHDlS9WydTkfv3r0pLCwkLS3NdTwgIIDu3buTk5Pj9j3U1abo6GiOHTtGabUY3/j4eMLCwjh48KCbRKunNgH06tULjUZDSoq7pJyzTampqa5jSqWSAUEgW40cPbATo5/arU0qVRGr+g2nwnCUmWs/ZUzX0Vzw3QVkZWW5Pd8bbYq4KIJu3bq5fgPZxdkAFGQUUJxnRUKB2biXlF02V5LjWtvUAfopJ+cQpaUJbDnUm0kjziVM6UdpaSkBigCyyWbLni0cffIoYWFhDLx1YIu1qbBQic3WG51Ozf79VW2SZZm0IlEHuUR2a5ezTcVH1hOZv4K9AVmgqegQ/dTU315RUQh2s4HIitUUHA4gL1NIA+YV5bFr+S5WLlpJvxv6ET2nKhTf223a+epfRBaq2TdUkI7BumAqNImkHjVDpSRlRxlPvn5HHD9eRGlpV1btH8fshS8Q7J/k1ibJJIx72cZsn7SpqEgD9EVdXkzaxjRSD6YSkpZKz2NXwJD/URi5qM42ybJMQUEBkiTRtWvXDttPjWmT0ZhLaamO1NQCunQp9tCmadBjGr30PdE4HKSkpGArEUTcgZMHcDgcWCwWVr+3moJdBfS8uicag6beNu3fn0dpaTRFRWaOHSuttU22AFGWTqnj0P5DVW2Sd2Ha/T4ngy7Gokno8P3U2DaVlGQSrT1M35xHOLZ+AYljb3S1Kac8h33F+wjSBnHlWVc2q007f9rJ8e+OY79+BmUOsZ6RrWJurN4mOWslkmyj1G8M8QkJTWrT7t27KSgooLCwEEmSOkQ/Nf23l+axTTqrmOcyjZlYDbnImaX0+OUzvgvqzdh/j/Fam8zmQEpLS9Hp7KSkHK/WJqlDjqfWbJPVaGXZpctIPi+Z+Z8kUVZWht1uY9u2Y2i1cq1tyvgkgy3Pb2Hq0qkUlInvJvt4NhWRFWg0Go7t/IXuaf/lRKGSghCtq00ZGSZKS9WkpWXi52f1Wpvi4uJIS0tzzcGt1k8KiQHH5mMLHMqe4KoI1VPbVGwp5v5V9wMwU3oUm1lm69ZU9HqZjL0Z5OzLoaKowqe/vZDuIWjKNKSVpmE5GYAmsysBfQOoqKjw+NsrO/ID5Qe+ICfsciyaLq422VefB6bj7Om5wmM/Gcp30/PoZRQmPUzImEc73niStqJYs4DjcU9QGDQL2SwMdxUOo2s8bd9+HL3e4dM29bytJwMCBsA6kBwaSooLSElJb1qbEi8lTTWRgrwSyEvxeT85jZ3h4eEcPHjQN/3UlHf5/v1UVJykvFwCq1AbKC43+eS3J0fIDLh4AGal2fWc1FQ/yspiUan8KNt0D/4nXmNvj6VYNPGnxfzkqzY5zIAKMvIy2btzLweWHiB6WDSW7lWSTN5qU5+b+6C/UI99m50gm4o3ZiWT/+c8rENf7zD95Nz/ajQaYmNjW6RNEfknCaso59iBPfQelsCUj6awM2MP8dJnaIt6MmRWAhF9I7z62yvUFHLWN2ehDdGy69MbKChRkplxjBR1VVvbcj95alN7/+0BINuIjo5xEX9WqYTdh7M4Z+M5xMfHAzSqTcFJwfR/qj8AKSkpyLJMqU0ESlmK7RzIsBIadhUlhUEkQ7vup4pWkJuQZE9iqq0ASZL4/vvvmTt3LiDy7p1xxhlkZGQQE1OVIPT8889HkiSWLFnCf//7Xz766CO3HzdAZGQkjz76KDfeeCPTpk0jKSmJt96q0kbfu3cv/fr1Y+/evfTp0weNRsNHH33ERRdd5Lrm9ddf59FHHyU7O9tjfT1F/MXHx5Obm0tIZYZpJyu84/0dbHxpI3M+nEPUwCgkSWJr5lZGvTuK+MB4jt56FIBDhyTuuktBWJiD996r6paGsMjSnv+ANgK5x/Xu7LK9wiUPcbp5MFgsDqIeGEuR/yY+nPk1lw6b12Jting2gsKKQnZdv4ufP+jPiZT9vHFOXxx970Me8HiT29Re+slud3DeeQpsNnjnHQcxMaJNkz6axF/H/+KTuZ9QelkptnIbN6fe3GJtysiAm25SEEgJ5yiWMeCSAfRZ0Ae73c7QN4ayp2gP3y38jnN7neu5rbZyUOlddWzv/dTU395PP8G3n2bzyfw45OSb2dnlGt7Z/g7JoclcGnYpSxctZci1Qxh42UCftemvJZm88ZpM9qgfWB2wmLm95vLdeV/iQOEi1xvTprqOt9d+amibzGaZhQtFWV9+KePv7173Vza/wu3Lb2dh34V8Mf8Lt2fbzXaOrTqGIdJAzLCquboxbTp5EhYvVuLnJ3PzU2t5b8d7jI/oxqKwIBRRZ+IIHlRnm+x2O3v27KF///6o1eoO20+NadPrrztYtkzi/PNlLrlEblCb3t72Njf9ehOze87mxwt/BGDZ4mVsfX0rt528Df9o/3rbtH69zFNPKejdW+bpp6si/g4vP0zhkUIGXTEItV7tcQ0kSRKKfU/BzgewT93simLoyP3U2Da9+67MofVreWLKdBQjXkBKvt7Vpne3v8sNy25gVo9Z/HLJLzXatOG5DWx6YROL1i+qkbfx1DbJsozVaOXO+3SsL/uEnd2uYHLSZH6/5He3uitWnQVFKTjm5zW5TVarlT179tCvXz+USmWH6Kcm/fYsRZC/AQJ6owzq4VbHR/5+hP+s+Q83DLuBl6e/xgWzjMTv+Z2L/9ObEVf2a3CblsxZQuzIWCY+PNFj3Y8elbj1VpnQUJkPPpAhYxkKvwQIGdghx1OD24QNOe17HLpoiBjvVvcVh1dwsvgkk5MmExcY16g27f9+P+F9wonsG8W558rIMnzwgYPQ0NrbdGzVMY7/fZxRd4wi9PVQLHYLR285SteQrqLuFhOYjoAuCrThrjYtWiSTlyfx7LMOkpO910+yLLNr1y7X+K2rP3zeT3/PRA4eiGPQ07XWXZZlDE8asDqsTEk5jrYi3vWd19ZPvmzTzz/DO+8oGDcO7r23lvG06xGk3Y9in74dggdWtenwh2ApRq6W58+tn4p2Ie1/FrpdjSJmUtsZT956R5iO4Dj2JXLsORA8gIu/u5iv9n7Fi9NfYu2/L0BfnMNT33cnKl7r8zYdLjpMr1d7obIFcmtZIc+ecw5y2EgUAx9p1ntPOvweSKBIvs4n/eRcPw8YMMBF3FevY6u8yy0F2I0nIKAnslJP9L3TyA9cxbszPufK4edTUVTB/u/3E9EvgvjR8TXaVFtbG9qmJXuWsG+vgi2fz2Lk4EAeWfQZpP+CPOxl0IS0vfmprc+5pQdRrJkDyTcx+zszv1jvZULQZfyx+H3X9b5q09Gio/z3j1fZvsqPx+IszLpsBFLCgg7TT6fuf1ujTYPfHMzO7J2MTP2NRPtUPv9c9ulv77rrFJwZ/hRXDLof+7QtEDLE621q0+OpjbRJOvQ2im034TjrD47pkuj+cneUdj8+7l3CBRfIXmmTyWIi6BmxX72PYh5/sEres733U2FhIWFhYRQXFxMYGEhLoM1G/EVHi0iR7OxsN+IvOzubwYMHu67Jyclxu89ms1FQUOC6Pzo6ugZ55/y7vmuc5z1Bq9Wi1dbMd6JUKl0bHydkh4zVZEWtVbvOZRkF2xwTEOM6VpXfT8EpjwBLIcrsvyF8jNB6L9kL/t1BHSDuH/h/HuuC0l3/VpKkGvWDqh9mc497eravj9fWJo1GQSDxFLGJQ7npWI1Wlj+8nPgx8fS/sH+N65vapsztmax9ai2jbh0l8jhKUFRRBECkfyQlJRJmu4FMw9XEhI/m1M5tTJvaSz+pVEoiIyEzEwoKlMTEiDZF+osI3fzyfK5ddS1qg7pF2+R8H+vsZRz89SAJZyS4zudViKi1uKA4j88S48k9h1l776em/vaCgsBoDeerk59z/sw+DA4ZzGtnv+Y6f9W6qzyW5826B/SIpTwYFEqZAE0AofpQpD/Go6zIhrnHG92m5hxvq/3U0OM6nXgtORxgNkv4+7uXGRso8shkGbNqlGkuN/PF7C8Yeu1Quoz0LBVUX5uc6yG1WmJ//n4+2vURhb3O5epxgnzyXHP3NjkXY3W1tb33U2Pq4u+vQJJEf1a/xHV9RQ7krBE52gK6o1Qq6Rnek7MSz2JQ1CCXAWjcv8cx8uaR+Ef6uxl1a2uTLIMkgUZTVa5CoSDl0xRSPkth6FVDUSgU5JflAxBuCHd/Vp97occNKNUBoKh/rvTW8fbyjggIkNibN543C8tZ3NP9+qgAoVyRVy7mslPbpA/RE9ItBOyen39q3VXBKqw2sKqEt2WoPtTtPqVSCYP+CxWZNY83sq3Outb3nPbST0367ZWkwOrZMPQFCLrNrY5dAsW7NcOYgVqtwBAVyBHdQiLPAudl9bXJYXeQtj4Njb/G7fi498fhkB18OPdD7OaeSJKETiehVMiwbiFET4aJv3TI8dTg43Yr0vqLUHa9GKInup26f9X9bM3Yyo8X/khCSEK9da9+vN95/Vz/9/OTMJnAYlF6fmdXovvU7nSf2h2zzYzFLqIkQv1CXe9spdYftAPd7rHbobBQQpIgPLzq+d7oJ7vd7nH8eqq7z49PWo4EeLq6et2j/aNJK0kD/2wkc0KN7/zU671d9zVPrmHnRzuZv2w+X2V+wPHIYCZpbkaSahlP/R+E5BtQasPc5kVF9ys9lumqe9gQt5y5bWY84aV3REAPFAMerPpTKxSXTFYjkdl7iNy/huITNxGbaPBq3U89bquwYbYIZ3CFrEWnBSl/E5ImqPFtsltQ5G8Stp7g/nDgeUCC5Ot81h+SJNXaH63yLj/0FspdD8LMHRAyGA2i/wrLTCiVSsxFZn6+9mfG3jOWhDEJtT+nluP1tenan6+lzFrGJPUR1OpAFEmXQNIlzWuTj4+36TlXpQOlFhRqArQasEKZzYRKpfJ8fSWa2yZzqZl4bTz3DX2B2z+B78wwO9FLbWricV/0U/X9b2u0yU8jbM12ZRnlRkW9a9Km/Pas5VYsRgsWvYVyu4b0kh4YQxfib4hukF21scfb9Hhq4nGvtymoFyScj0IX5or4sytNFBTCidUnkBQSiRMSG1XHQ78f4thfxxj9r9H4R/tTYi0RdXeoCNQHoFS6O6e053461dGmJVCbPa3VkZSURHR0NCtXrnQdKykpYdOmTYwZIxLsjhkzhqKiIrZt2+a6ZtWqVTgcDkaNGuW6ZvXq1VitVQmAV6xYIaQ2KiPzxowZ41aO8xpnOY2Bp04ces1Qbj10KxF9I1zHnMRftH8VuWiszHPr784tgLUENl8Pa+ZB5m9gPAS/DoG9T9VekYIdcOwL4UHc0XHsc/HxgHCVWBAeyU8DGTa/vJkjK494vLapyD+Yz96v91KWJ5jbwopCZITXQKg+lOJiyC3rSm63d6HLuXU9qkMhIgICNPmEHrgKjnwkjhnEGMgty8UQbkBtULdonVxBujExPGR9iDF3iDEuI5NvEYboGP+Ymjdm/CqSrVt9n4y6PSAwEGwODevSLxLEQSugvMQCsswIeTEl95Xw7rnvQvQU6DKnVerTniFJoNeDQrIR+HcybLjC7bwzx1+2qWYEvD5Uz7nvn8uw64c1uXzn9ByUlUr+WjEOg7RBddzhDkmSiI6ObpVFVFuFM9G6yVTLBUUpsPY8yFruOjS522RWXbGKxyc97joWFB9ERN8IlGrPi9lT4exL9Smv9gkPT+CyFZe5kr3nlQlyKswQ5n6hQgnaUFC07NzQXuDnByB57FfX/GrK9XjvsGuHcdW6qwjtEVpvOSc3nSR3by5WqzvxVwOR4yBhYUOr7xGd47cSIYNgxJsQO7PGqZgAsS7JNop3cOX2hcLChj9eoVRwb8G9LPhigeuYQ3awOX0zG05uQKfSudZIWi0g22H0B9Dz1iY1p0NBqYNx30Dfe2qcco6LwvJGdEY12CpsyA4ZvRCTcDmB1odic7Hr/06jCwDlmWAugGrey/n5wsFGpYKwU165zUV7HL9R/mJN49CLvbhTcam8sJzdS3aTszuntlu9AoVKgVKtJN+Wz+fZD7Kvy79rzJmn3AD66M55sR44x4HJYqKsW38ODVuIOjTA5+W+OehNVkwWcqsKWYtKrYDz8uGML+q50wPs5fDHmbD/efH3xN9grGcbhzfQJsdv5AQY8BhohbPw4NIHGHHwF8aEzwAgMC6Qi3+5mGHXNn3fUR1r/ruGX//1KyDUDsqt5QAoHHpUbTZEoh3BPxFm7YKeNxGoqyTnbUYcdgdHVh4h65/aUys1B1+c8wUvJb6ETWQW6JB92Srjt/QQHHwDjMKe6sytaleYkGX49bbfWf2f1V4tcu/Xe3k28lmuvO9Kvkjy53ujmsJ+X4GhpsNxJ1oI0ZNg3BIIGey2Bs0uMPHjlT+y4q4VjX7k0VVHWffUOiwm4dTmXFer7SFotRKkvgyrpkFlsEZ7xmlH/BmNRv755x/++ecfAI4ePco///zDiRMnkCSJ2267jf/85z8sXbqUlJQULr/8cmJjY11yoH369GHGjBlce+21bN68mXXr1rF48WIuvPBCYmNFhMLFF1+MRqPh6quvZs+ePSxZsoSXXnqJO+64w1WPf/3rX/z2228899xz7N+/n0ceeYStW7eyePHiU6tcL2pjjE9FZmkmANF+VcSfU0K2BvGXuwFOfA2BfSDqLFAHwYBHIHpa1TUOK6ycBJuuFX8f/wLWXwzlvplM2xRSHoE9//V4KkovJCDSitPQBmq5I/0OZr5U06jSHPS/oD8Pmh8keVYygCuSIVAbiFqpprhyb95CUbxtBhER4JCVxJR/ALlrAYj0i0Sr1GK1WzFmG0nfko7dYq/nSd6DpVJCXqOp9JBSifFaWFGIzSFWhk6DgBsOvgFbbmqparZ5VKYxpUQ44mB32Mk2ZnO86Dh2q53VT6xm77d7fVa+LMusmvwk3bZ97TKWSJIEg/8Lw1/2WbkdGXo9OGQVVnUX0Ea4nXOOCafRuTokSWLIoiHEDottctlOsihk029UvCoscBfL+2HZIDDn13u/QqEgOjq6wfPv6QC/ymD/Wo3IwQOEMSl6ap3PsVvtmHJM2CpsDSq3tg12WM8wuk3p5vo7v7wq4s8Nxfug2HfvjvYOsT6UiZF/FWvDaojwq3KsaS6+Xvg1P1z5gyD+lGLjFaILcb/IYXcjF5qKzvFbCU0IJF8Pgb1qnJrSbQpZd2ax7qp1AAQHQ5c9v7P77fWNLqb6ZjOvLA+rw4qERIx/jDvxp1BB4kUQO70prel4SFggyNlT4BwXhRWNJ/5W/2c1T+ifIP9gPk4RmWpZJDzix6t+5NuLv6XELBZgAZoAFNXkzdl0HXzvrljjTGkSGSkcfbyJNjV+s1bCjnvqdXp1OvhZ9RlAFfFXklbCtxd+y77v9/mylpxx9xncmHIjFn+xKVHbg9Bo6rihcCeUHq55PG8T/DoMTnxT85zNBH+dDfue906l2yIcNlgxTvQ5VcSf0WKEyEiKYvoia3U+r0bP2T0JmyQYdYVDWzeJWx/UgTD0RUi6XPztnwihQ5pbxVrRpsavE5HjYMBDYBD7igTFaKKKZxEkCSO/SqcieVZyg5yYGoIDPx1g3zdizJvtZpfztlLWi77c+zRs7XSA8QYS/XuTkHM9fRSzQYZPpnzC2ifX+qSsHrN60Pui3qQVpxEVvIXbBs2EtO98UlZroVXGb8F2YRPL3wpUj/gTHon7v9/H/h/2e7XI0ORQht84nMJQsc5SOHTNe892wqvQq/QoKmml7CIjU5+dysRHJzb6OWc+eCaLUxe70lE419Vqe4hYI5lOQN6GDuEE1Rpzbqv6PmzdupWzzjrL9beTjLviiiv48MMPueeeezCZTFx33XUUFRUxbtw4fvvtN3S6qkXcZ599xuLFi5k8eTIKhYIFCxbw8stVht+goCCWL1/OzTffzLBhwwgPD+fhhx/muuuuc10zduxYPv/8cx588EHuv/9+kpOT+eGHH+jfv6YcZH04VdsVYP8P+6kormDwFYNdx87peQ6h+lD6RPRxHXNG/AWc6pwWMhBGvgNRE8GvUkZmwCnSngo1WEtFTj+A5BsrJbySG92GdofRH4Las0dfl4B4KIFMUxqSQiIg1jeef0pNVTREdYOmwyEI3RGxPxF77CsIewQCuvukDm0NERFgsgbxblER14wUrOcD4x/goTMfQpIklt+1nA3PbeDWI7cSkhRSz9O8Ayfxpysv5MjKQuJGxKEN1KKW1Dwx6gmU/ko0Sg+771HvQdFOUJ/Kyp+ecJLYDw4fBitDODnsPRJfSkSr1FL27zL+fPBP+p3fj74L+vqkfIfNQeS0QRxPjyWk/c/9bQLOaINDXf9koLtKGInBiay8fKUr8s8TZFlusveSk/gzTTwbecRr4ACdUgu2AlDV/8622+0cO3aMxMTaJSVONziJv1oj/nSRwqjvAQ7ZgYSQfkr5LIUfF/3IRT9dRM9zenq8vjpqi/izlltR6VSu38j5/c6nf2T/mmTS5muhJBUWNJ+86ogQxJ/ERV3mwZ4pMPFn1zlnxJ/RYqTCVoFO5W7wLMsrY/u724kbGUfSpKQ6y5nwfxNQ69X88WM14k9/Sl+d/AE2XC6iGpqhZtA5fk+BwwqSyo2hMagNGNRVcnUhISCnp5C9OgwY26DHGrOMnNx4ktgRsQTGiUk8vSQdEE5ZaqXanfjrRE3IDrccwlCN+GtCxF9k/0j6nd8PSZJcxI9znVobio4VYSu3uYi/QO0pXoVx50BQH7ffjzObRR0ZLJqMNjV+c9fCvv9B4qWgCa71svhA4RRaoUkjiCriLzgxmPO+Oo+oAbWvdbwJZ9Smyh5UtxFz7UJQaOHsFPfjCg2Y8wTJdypK9kPBVoid5b0KtzUoVFCWDhYx9q4ecjUze8wkPiie/wql+HqJdG9g2rPT+PPon/CxiPgL0BZA2t/CDuNf91xbA5IEvf8l/n/sc6Fkoov0ep2daFPjtxY4FSxOdWRz2B0olM03nl65+kqsZWLx6oz2A1A6DGJcpv8MpQc7HUubCocdDr4G/t3oH3oOA08MZ0SCGL5nv3k2YcleDkOvxLh7x/He9ve45qcEZnY9gx5BO6CsY70PW2X8Rk6ASX8IB1KqIv6UOjEPzf3tBrome9c4Ez8mnvgx8dz12l2QBwsTfiP44FKIes373kydaBjKs0XwTfQUpIQF3DLwEX5fpqHUaqDfQg/qaQ2ANkCLNsB9AxKlSsZh7ibWyEOfFZ8OAE+cka/RqsTfxIkTayRerA5Jknjsscd47LHHar0mNDSUzz+vWwJh4MCBrFmzps5rFi5cyMKFzZMrqg0bX9hI7t5cN+JvWOwwhsW6SxTUKvWpj4Ee14j/28pBUoIncmL65qqXn39S4xeb7RURtRs9ksIE8ZdrOQFA4ZFC7FY74b3Ca72nscjcnom1zEr8GfFIkkSvsF58f8H3SEiUlgqH+G4h/6DJ+BQG15QJ6qiIiACQSM8NgsqfpbJaborks5MxRBhqvOB9CadBxXB0D59MWcm1W64ldngsAdoAZkXPYsCAAZ5v1EWIzVcnAJHjD+BkSU8SDYEuby+z3YyskLlu+3X4RfjV8YTmQalWknT7XJa9BvsUt7Lx0wM8MP4Bxud9J5wjet/us7I7Kpwb6/Lymud0Kh2TkibVeu8vN/3Crk92cW/RvU3ahLuUuHv0IK9/HuyCbRFzmTj2zgY/o9QZMt8JoAFSnx4gyzIxz8WQW5bLydtPEhMQQ0S/CIbfOJzALg0LWa8t4u+V5FcI7hrsyv8ZGxBLbICHKNHkm4UhsxMe4SR0P059l6unukvcBOuCUSlU2Bw28sryXHnhnLAYLay8byVj7hxTL/E39OqhAFi/gaDywfTtWkCf8D7uF6kDIOIMMMQ3r1F0jl8X9j0PO++HWSkQWLvjXkgIrJ54E2fPrStMyB1pG9L4av5XzP14LoMuE5FrmUahPuKUEnUj/g68JlIKnPkjhA5tWns6EjZdI2TrL6hwy7PmJMSbEvHXe25ves/tDVSRrfURf1esElLcB/IPcFH/i2oSf8nX17jHGfEX5SM+q82M3x7XQ+Il4Ne1zssSgoQTbbkmTfxbue7RBmrpt7Bfbbd5DbuX7MZcbKZ4vCD+1PURf73vFHv/UxE6pEZO66pzw+Cc1AY5T7VrzDnq+m9SSBJJIWJu8884wODff+DY+HPo3983TojVkRCUwHmRD7J3axhRcXthzXyRr7X3bU17YMlB2HAZxMx0c/DxBdrM+HUi+2/Y9QD0/z+ImUqpYRdpYdvYlJnMSMYB8ErPV9CH6rlm4zXNLk6pVqIMEuOr3FYp84kShawWa9nJq4QEayeaBkkB2/4F8QswGM4Bqt65w68f7tOizXaxqPmnNIYHdq7l2ct8WlyroMXHrz5KfCrhJP4krdhw2tU6lD5yynYS82Mjt6BN3wHS674pqBP1w2GBQ28KKfyEBTw84SEOvw8mSUjLNyWgreBwARp/Df5Rggw5s+uZ/CfqAD9uBE1NwY1ONBIdUO247WHmKzMpL6x/weCR+JNlEcWn0sPxr2DDpaCPE1JAE39193KQJDj4ptBcHvgfz+RgR4XVKFyHlO4e7r2jkojePo+EwERkWebTGZ+i1qu5YecNXiv6z4f+5OiqozxQ/gAgchbN7T0XgOOV+7FlJx7k0v/cKTw2TxOEV3KrCmMq5ORA5Hi380lnJZF0VsuS006DitSzB7Mu1BKcFFz/TcX7hQSXvmU8gNsDdDph2H92/Rf0vQb81FVumGXWMmKGNM3TpzFw9mUa69l4eBuLRy6GIx9A2MhO4q8JcEb8qYvWwD/LhKFJ1zAHiaCuQSSMS8BqsqINbPw7zkn8qVRQVFEECBKjE01HvVKfNhP8lAxxc2DkG4BwtnLIDhyyg9yyXGICYogbEUfciLgGl1tbxF/Pc3riH92AiOlaohA7IeDs11VHLuXqU6YkSZIIN4STZcwi15Rbg/gLiAvgms3XENw1uEFlybLoz27Zd/DJ3DsIPvW2mGni0wnvwT9JSPp7MDL+35//x768ffxn0n8ICemJXWOgqBFph2OHxzL347kkjEtwHXPKNzujuZ2RT1otoPIHXTRovCOn1u4R2AfizgZHBSiqHJuaI/VZHQ2N+HOiZ1hPPl/QsNxfzog/XxF/bQb6hoU0XjLwEqZ1n8aStxPZQ8tEhVXH5lc2U3CogIjfRJR2vRF/HsjcGnDYBSEty0IiXRdeZ9RjR4cyQE9ZYFSLhC//ctMvhPUKY2HM43ySA0YpG8Z82nSHiS03QfafcMZXp41KkBvsFSKPmFVENR/R/sDOpP9n77zDo6jWP/7Zvpvee0ggdAi992oBFAXFLvaGXa/96r1Xvbaf1y4qdsFeQEREUUBAmvQeahJIQuqmbS+/P052N0s2fRcCmc/z5IGdmZ0zm82ZOed83/f7PsXS47dwV43w13FCR1TB/lEXjq0/RlBsEFEZURisYtCslolJkUqFcNU6C6zlThsyGUxaDdo4NDsdWBR6ThgNQGDrsy29ZynlVeXQAeROyRrSrzidwgFBrqBDeAd6xfYitFKMFUsO6dEUVpA6ItVvdcx2frGT/Qv3I+8iBxU8teEVFr/bicAbOUvUiy4JZhSKEmSIEgRyuRD9Ft60hKO/7ue+3Pua9Tcw/9z5qIPVXuv0rvGZWo3IvlYGi3mSRLNpQ4beZy/xfeJJH5vute2Xg7+wLncdJpvJvc2n8Fd1CL4Jgz0vCuuWhHPEgp3heN3UZrtZeFcf/qh9RSZlfwXfhMLxJXV2ZSTGMujQ9/Qv/B8ymYwhdw1h4G3+KQbtYvCdgznvtfN87nPV9wsPl4EyyCtK+GwntqZM2AXJ98BKUVcxvzKf6V9O57z5vn9fgcZd4y81gcG3DyYoWqTF7CrcxfrC9RyrOFb3TbufFfVSbPWtoLc/ZDKP3WdFhfD2dmGwGjBXmDEUB+73VXGsgiOvLiL8RBYmWa36UxflwKhvAtbu2YwrQyyoep3I8qg84LV/0b5FPL3qaXac2FHnvaMeHsVVS69qkegHQlzQVJUQ9NpzhH8vBpADK9ZA3tIWnU+iCRl/Ch2EdIIgb1HPVXOv2NCyrLv6Mv6mvTONcf8a5379+c7Pmbd5HrnluS1qp71S28LVl2HGi5NeZP7F890ZLbVRqBQkD04mOK7hbOzSg6W83ettNr2z2b3t5O9TIkCkXgzjlwqL/5NYtH8R3+z5hsNlh4mIALVBT/nevCafOjw1nL7X9PWyVj9RXSP81dRx9cr46zQbztskakxJQI8HYMxCsehQC3fGXwusPp1OJz/f9TPr/reuSTX+7BY72z/dTv6WfN8HGAvgj8lwZIHXZlfGXyCsPtsUTgcY8xutb58SlkL/xP5u0daVfeKwOXgp9iUWXrcwoJd5wXsXcPmiy2tZfYY1XOOvMQ6+Bz91BUs57PgnLM6Agj/8c7FtnROr4OD7ABwpO8Jr61/j0+2fouqYStbw2YT2C7xwtu2jbRxZfsQd+GRTxkPHq8S6TUuQa0Stv5Tpwi60vZF0LswoEHVVgZCajKJqq2deOe2daZz7cuvrzzqdTj4c9SG/PvAr4MkoUsvEIFqpBIo3CiFSouXEjYKwrpSTw6/9o3lfJ0oHfD71cz6d9GlAmtz77V6sG0SnTFba6Bnxk7AnlGgd5fvgSyXs+jcAj495nF137GKIbA4AO19fyUejPnLb5/qDwp2F7P56NzazmGTabJGoQvzn3ibRAuQK4YhWk2iUU3EUe/zfWBQlODRaItIjcNgczTrlwFsH0v9G75q2rueqWg1smgNb/+GPq2+XSMKfn/Glatst3h6uTqeTGV/NYMSHI8ir9EzaXZnaXsKf3Sz8+cO6CS/lcT/BzMK6Pv8gvP5TZ4ioGnW4Pz7OmUFYD1HPwUekZ3SNbXhJiVgkG3rXUAbfPtivzXc5vwsDb/GIiX/l/sXnOz9nX/E+t/DXI36rKIbbjnAJf0v23Ya518vgdKCQK/hx/48sO7SM3L9zeX/Y++z6atcpuyavqJFazNs6j9vX3c47m9+p+6aUC6Hnw0K4lXATFgYjUr8l+OBjyMBdf6jaWs28wfP4ZMInAWu76kQV+hXb0FUWYnTqgZoMMVVY+7r3+RFXxt8B+7Uwda+wiarFB1s/4MmVT7L+2Hq/t221AjIZspQU7j7/bo7fm02/vI/h0IdNer9MJiM11X+RhWcDtTP+fDqqy+QweQ30fsJrc2ywuHEXVYsae5X5lXwz6xt2zK8r+Pqivoy/k3l+zfPc8tMt7CuuVQA++2tY0huK/mpSW+0R1/jw9gE3ww91MzGv6XsNV/W5iugg3zVTrAYrlfkN2wJZjVacDid2m/jDccgsvr/PTXPg0AfNun5fSP23abi+0xJDCZGRkLr7F5SffNBgyYTG0Cq1pEekkxYurBFdwVFSjb+mMz59PPMvns/jox9v9ntlMhm7vtjF/h/3u8elDQl/hmIDC2cvZNvH2zDbzFjtJy2smU5A8QYwHvfaHMiMvzbVf82l8EOSEL+agLYmXcCV6SpXyonvE980N5BWENszlpShKZSbmmD1WbYNlg4Qz8f6kNf88VQfEWsGMSPajz1v1puiNrDDxv6S/dy77F5eXf9qk/qTv/hH0T+Y8MEEsqv3YVIWtD6zaOArcO4G4WQUYNpU/62HILWYXxqszfCubypOmPzSZPrOFj5yKWEpfDHzCy4LewOoGcv+MQn+vtP/bbcnnA6wm4iqGcTaZEbsDjuqYBXqkMA4lN2bfS+2l4VQNCqsiGtSL4DSTQFp63RxWvqvOlK4xYR199rsCjiNHN2Lc/53DjK5/65p4n8n8k/bPykIFUE9XUKPojAc8tv5JVpIxX7Q7wbguoXXsSR5MMVhy0m9biI3rLkBhap5CS8j/zGSoXcPdb/+18p/8WJFP3JiPhDP9MFzoc+z/vwEp43T8cyVYnj9jPwkQ1u71c4zmmfoc00fLv70YgAqLZVuD3GXvQ54Mv5Ca9vxR/SCsYua1rhMBl38Z2F5xhDZB0Z85nNXdDQ4sFFhLadIryUuMnB1x1x8tPUj3t/6Pv8Z9x/6V4rJ58Ud7oKVB0UEWztBqxV/y+uPX0RBCKTJIFoXjQwZTpxUmCuoOFaBpbKJvkZ+wLWoZf/6O+Z9X8rNG28GoKBKfC8+a051uFT8SHgRHg7DghaSULIAHE8SrArGYDVgsBrofWVvvw74TiaxfyLp7z3K3z843cJfpDYMSreCNq5OFpNE47iEP70pAcLrBlG4nlUua7jalOeW8/c7f5NxTkad7PamYLWCOTgK1Y3XMOQGhGXVuRtFVloTkMvlREf7FjraKy7hz2YTv9+mZhPEBtUIfwYh/DlsDvZ8s4eI9Igmvb+2bauLyvxK/nj8D3rM6EHXaSLK15VR6CVQOcwi2El1ltckagUaDSgUoDfFY9N1QeWwNsuC6uNxH1OVX8V9ufXbIcdnxjNn7xxRo/hXJ7/0DyPy/+Rk3ZXlsQ+1VcOBtyHtSsi4sVWfSeq/J3HkM8j7GUYsEAJ9DdE68TsqNhQTmQQlKX2wJqbidDiRKRp/3i6Zs4SsxVnM2TPHvdB277B7uXfYve5jvDL+djwlFnYk+11B6WY4Ml/8vUf0dm/OiMogI6rlWUVz9s5BF6Xj1dfE64asPrURWi5fdDnhHcL537r/8dgfj3HLgFt494J3xQGRfWFWhVhkrcFsBr1e/D8Qwl+b6r+aaOh6t6g92givrHuFxZYjqBRPYTR6rv/a368N5BUCYKm2oNKpuKH/DWT/OZaDJ2LqF4vsFrDowd6Ai0b6NeJe7CrxMb4duSX0fNhthRqiFqJClaUKtd1I0v71FK5NgXPqr5fqD9QhahZnLebx/JuJS5/Gs+rJ8MOL4nuIqKd2fBuhTfVfF5Zy8QwM7wmRfQnVBEMlGGwe4W/XV7s4uuIo5712HkpNy5czZXIZw+8b7n4dqYvk8t6XU7IK8qkR/vo8DUE+1gckms5P3UGhI7r7Bvemams1l34duPUVuVKOSS6iOraWp/NT8UdMO8syaE9L/9XFw5jv62x2CX/azC4Mn+b/e65MLuPCzrNYu9HIc6Nmw7p0OEcKFD2trDgXNDFw3t+EasTc3aaoorTUP6c/WHqQfOd2IhR6sY6RPMU/J24DnKwZnZI2T3mLZzl2u3d2n91sp/cVvUke4lmIdokMoepQgtUeIcqn1efJHP0cPpdBjmRn1xS0WtjRbRa/9o/h/U2fsfbFtXww4gOsRv+knzsdTl7r+BrLHljm3lZiLAHEgqYr42+3/V4xcGxnuLL+isQaMgq5giid8AB3dnFy/7H7GXDTqYtKdS2oKHTeEWauzNvaQrxEw4SFwafbn+P3oAMgV3sy/izVjHtqHGP/OTZgbcvkMuwKNVaVGSdigStCqYRfBsDOfwWs3bMZl/BnNDrBVCTspGuRECLEQJc1XG1MZSbW/HcNR1cebVHbdbLE5AqIHuy1uNoQdrudffv21Xn+tme0Wo8beL12nwfnwd7/eW062eozLCWMf9r+yeQXJzepXZfVZ+1FzMq8SrZ9tI3CXYWAcD1wPSdd7QHQ8Rq48ECbXyg7nchkQtSdv/MZ8nqurCP6HSk7wo/7f2Rz3maf7+91WS8yr27a79dqBYfciENuxmgzEq6plU2tDIZLK2DA/+o/QROR+u9JlG4WY3yDt/W4q6+UGEXGnz6xJzkpI7E7mjaVC44LJjQxtMG6SC7hT6uxwe5npLlGbSoPwv5XoXyPX08bHBuMXCF3Z1k2JPypglR0u7AbCf0SqDCL+leusZcXtQRjV7ZfcHAj88sW0qb6r0wGg16DtFmNHvq/9f9jheENDJrDp7zG38sJL7NgygKSw5JJsY8h1NSzfuEvZghMPwwdGxAk5QqP6NfeiB4ECZNArvQW/mRWkg78if7vwFo0OuwO8v7Ow3hcBHTLHRocijAIShF1Uts4bar/ujDmwV9XQs63AIRqxVqZqVbJjZzVOWx+dzMmvcnnKVqLVxBb93ukAODWkjIdks4nIkSDzCmygKoszShS3AJy1uZgyRYP1FxDAnvN14l+eRbRFvrvon2L6PFWD35w3AA0UFu+FZzYeYKc1Tn8b+RnDDr0HT8dehg6t8Nkl7ZGj39AlzsAT+CNTV5F3sZj/Pnsn406zNSmJKuET8Z/wu5vdru3uWpnq2yRrbNDb4Ocjj4rCX8BRh2iZubnMxly5xD3Npfwlxia6HVsHeHPZoSV0+Dol56DdDURR03MhGg37HgKNt7uc1eoKgKA/DI91YXVlGeXY632j/BnNVoJiglCqfVEm7mFP120O8q2JOgS6HyzX9o8k4iNhcy4FXQ/2BOOiczVk63kTiWuBZXo6y5k9h+z3dtdfdIlbrixVsAvQ2D/G6fqEs8YwsKg2JBKflVnkMm5ovcV3Dbwtnot5vyJodiA4WAeDqdY0VIr1OhUYdD3v5ByUcDbPxtxCX9mow2+j4fN93jtr73ofDLR3aKZs28OIx4c0aK2rVYIKcnG9utybnr/Jv6x9C4MprrtNITJFJjJ/5mKTNaEOn+HPoD9r3htcmf81dyfZTIZckXTh4q+rD4T+yfySPkj7nFQlaUKi13cjL2EP4kmUbvO38l8tuMzpn85nXc3v+vzvSMeGMGk5yY1eP7jm47z9zt/U5FfjUUhJl0KmcI9qXOjChWRv35A6r+16P1PuLQSgr3rNLoy/koMJYSEeLJqXQFmjTHuqXHctOGmBu1lPHboCrjwMAz4v2Zf/llL0lSYniMWMGthtpn5cf+PfLq9ZXWKqgqqyP4zG5VMRE00VYRyCX9hmjDPxpJNot653XOS2jafgXIWOhP7b2pYKgBGdY67xh/Axjc3su6VdQFtu+elPek0qRNQq/Z4Y4taMmnJpl6cTnA6vIQ/TXQIu8beQfiFgQtCBLBUWpg3eB6V74sFTrlTwzH1dXDuegjpGNC2/UWb679BqTDqW0i/EoAwnRjMmhyeQc/4/4znH0X/IDi2dU5ORXuLeLf/u2z/dDsAueW5fL/3ew6aRFmDVtu2Sgj6vwT9nicoSIbCLvppmaGSg78cZO2La3E6Wm5Z7gunw8lHoz6iwzcdmBh5MxGGwWftd3la+u/m+yHrLQCMNiP7iveh5wgAJesP8G7/dzm66qjfmvvzP3/y8diPsdYEl/5x/B7oFPjsfIlG6DoHMoTgG6IS/dquqKRkWy4rnliB/qi+yacylBgo2F6AodijHLtqZ6vskYTJDsE3EbDnBb9dfntDGkWeBvIrRVH22iKDzeapMeAW/sp3wYk/vGs1xI+Dy4yQPO3UXOyZQtFaOPG7z2JG4doIAAor9Jzzf+dw//H7CYrxT702dbCamzfdzMRnJ7q3lRg8mQwu4S8y0i/NnXHExIDFrsVmBxziaV3bSm7Xl7s4sPTAKbseXxNsp9NJfpXok4kh3mI85hIReWhungjRHggPB7XCiK3iGNhNPDfpOeZOm0vnqM6sf209X8/8ulW1hxriwNIDlDw3j7CSo+jkYURqI5GpgqHXo5A8NSBtnu24RSKjCrrfLxY5a+HK1HXd32qj1CiJ6RaDOrhl4VhWK4SWZmNZvpafN/1M3t43Cfo+xh0sINEyGhKIABgxHyat8trUM7YnEzpOoEu0x6Yl969cjm86fvK7feLL6lMml6EJ07izrF3ZhFql1pOt4nTCjn9BwfImtdOeCQmBjhHbCM/5N1R617g42aq1JRxYcoAlty+hIr8aa43wF6mL9BaMKrJAv9P9XJfwI5poUNYN7nPX+DOWIJNBcukueqyex5ENhS1uatB7gxj6/lCO6o8Ctaw+tTIIToOQTi0+91mHKgSCU0HhXQDRbDcz/cvpzF44G6PVWM+b62fT3E18PPZj5HrhjdRQxt/Oz3fyfPjzHPzlIBUWH8Jf1puwaho4PCcpqKkwEAibzzbJ8SXw64hG66p3CBfCulGTQ+110y3vb+Hvt/8O5BUy/cPpjHhwBF/u+pINvEm15pBv4c9hh22PQeHqgF7PGc3RL+ErDRxf4i386WSYQmOxawJbn12hVjDx+Yk4R4v5jsKpOWsFhlOGKgQ6zITwHgCE68Rg1uzwLAjronQExQS1uqyEzWjDXGHGbhHZF39m/8nMr2eyzCJKtYQqcmFpfzgwt1XtSAh0OlA6RD8tqaxi15e7WP7wcqwG/wTku3A6nZz76rlMu3saN8W9x+0JB7ghOl1k7ku0nkPz4NiPAASrRP+0ysRk02R0Yq4w+/U7HXDzAM5/83wMZgtOnF5zTIm2gdvqU16FISOTW7feSnyfpg88U4en8nDpwwy+fbB7m2suq7bV2KFHDQRt3XI0Ek1D6jYBpmBbAZve3sSAmwa47T59ZRdV1cp2dy3WET0YZhZ51WoAQKEN5CWfmYz4DLS+w1ljgiOgGooqy07JpdSuXaTXC3FkfFU/2HEF9PnXKbmGtkJsLCwpGc6HJ/Zwf03weu2MvyW3LyGhfwJdzg9s/QUXZjPgdFK1dA0H4xPpfF5nig3FmGxi1p8celJtuJCOcPGxuieSICwMzuv8LtdF3AclqyFulHtf3qY89i3ch91s98qG9RcJ/RJQT5kIxp4s6F3O9Iscjb9JokE8Vp/4zPJwCX+lRt/G7Sa9ifKc8mYN8lxYrXAifShps8IpcjxDkVMN6ZdCSMvrJkl4xNx6bVdCO9fZdEXmFVyR6V3T66sZXxHdJZrrV1/faJs+rT7zK6kqqCKmWwyqIJVvm09TAez6t7BuSWg4I629ExwMUbbtJJf+C8r7Q6inn7ier65xyMkc+u0Qm97axPj/jK+3r/a/oT8dRnXAnhSJTZkFQKT2pOilPS/A4Q9hZgloolr/oSS80e8GQy4knefedLINb7DWisJSTdmJptVJXvX0KqK7RtP7MmGhbHPY2JK/BSdOdDVCo0v4C1GVQLVBuIzIFf76VGc2DjsYskGu9ar3FKoORSFTYHfaKTWWkqxqXo3hLud3QRep46BW3LAbEv50UTqShyQTFBtExQEfwl/XOyF+vFedVFfGX0J7WStxmMXirqmuLXlt3Bl/qlyvjL/Lvr+sSTUz/cHrG15nnWYdg3Q/oFT6GO+U74Y9zwEOiBt9Sq7pjCM4FRLPB3WkW/hz4gSVEaXZiaHICQTOclMVpGLUw6P4feXvsApkDjVpzs9g7wno8WDA2m1PdI/uyYBDX9Ip0eMoY6myoM/WE5oUii6y5S5YiQMSufvQ3e7XRpu4GSic4pwahUm4/9jbWFbkmUbWW1C2DcXQeV7C35jHRzN4zmC/rxXIFXKG3TMMgA2fgckWjIlYwqQ1VP9wwQFh+Q/uslUu4c/WqSv3Herq1+YyzsnAPtBOtze1hPQPZ0dSN9g9HXo95td2JJrJ7udFSYDJqz1Wn4oqSswhJPRr/XPX5T6kscXiDMmAkb+3+pztGSnjz8+cXKixaG8RW+ZtoTzX48XjFv6CPbOwshpNKjwcvE6hDPaawEnUgy6xXhuUuLAIAEoNesqOlLHz851U5jXdc7ghivcX89fLf1GSJRYya9cucll9ahQGZEo1EJjsp7bMyTX+AOKC4tAqtZhsJmZ+ObPJtaP8gcUCcruV4q/+YNeXuwDILs8GRH/UqSUL3aYSFgYHSweytvge0MZhspkoNhRTbanmoo8v4p+2fwZE9AOIz4xHPmYU5uAolEqQy+RQcUBEeR9umd1We8cl/NUnEjUm/C28biHvDngXu7X5nuVWKzhUGmzJTmwqGzuJEtloTazxJ5fL6dSp02kplNyWaTTjz2YQdcQcDX9nk16YxPAHhzepTV9Wnzs/38l7A96jYLsY+7iDY3S1bIE1sTBlh8g2lWiQkBDYdHwaK4O2Q/wEr30ucag+K+2qgioOLDlAxbGKes8f3iFc2NCpVFgVegAiapwT3KRdBpn/8YvoJ/VfH2y6FdZe4eVicXH3iyn6RxHLrhY1pYNH9mfnxHshpfGaNU6Hk1X/WsXuLz21M4oNxThxIkPm/rtxCX9ptg9gUQco3eS/z3SmY9XDjxkiQKEWMpnM3T9c9UiaQ8qwFIbdO4zgeLFI0pDVZ+fzOnPNb9eQNDDJt9Vn9GDodJ3XewKd8dfm+m/KxTCzEJLOb/Awd8af2jvjL7JTJBFpEQG7vKqCKhZet5B9C/dRbhZrA0p7uO+Mv4jeMGWnVMuoIWJHwthFEDfKq96lU1VFr5VvUfbed6fkMsw19rpyp4YO1g9h93OnpN3W0ub6L4gx6bdRsE6U5EiKiCGp7DLiqjxBYQeWHmBu77kc+vVQfWdpEa6sbYWjJhBD2wUuPATd7/NrO+2OEyvg0PvgdJJmuIQORbcQIosluks0yYOTkSsD8/dXZiyj3KxnycGbWWLbdNbV+Dtt/VeX4F6fdmX8WZxislk7mcWfuIL0daiJ0EhuXG0CazlYSsFmcAt/dnklpSVOTHpTs7I+s//MZt+ife51JIvd4h4jqW0xZ12Nv9PxzJUy/vzMybUzes3qRcY5GaiCPKtgM3rMIDksmcy4TPe20pq11CjXGkrhaijdLIp5S9HUTUO/C7K/gN5PelnxJEZGQj6Um/Vk/5nNousWcfmPl9MtqVurm8zblMdvD/5GZKdIortGU2mpxFZjfeXK+DNZoikdtpOkpIbPdTbiEv4y5J/BYTt0uo43przB3GmnxzLDYgGHXEn3V25h9FTxBOkQ3oEPL/wQm8NWt/ZN8QaoOgzJFwjrEQk3YWGwp2g0VftHMzIMrv76Er7b+x1vTXmLOwbfEfD26wgMdgNU54jITIlm45Xxd2SBsPEYMd89Seod15uVs1e6M4pOpvcVvUkalITD6kChal6GiNUKKlMlliohVtQRGRpBJpMRFhbW+IHtjEYz/rY9LKzhLs4TwTO1cDgdQlAH+s3u1+Q2XRl/tW1YOozqwPhnxhPZUWSNDUkewvJrlrvPD4BcCRGZSDROcDBUWqIpMEXDSZZijVl9Zl6RSZ+r+jRoj2Uz2VBoFFitMrfwF64N9z4o8Rzx4wek/uuDno+IOm1OO8hEZwpWB7sjqwEiIsS/ZU3RmmQwZ98cr3qdJ6pERlRMUAyKmqw+l+hkCh4ICXdJWde1UUWIDJ6YurVso3RRlBhL3PVIWoJrUaOpNf7KTWJBxC38uUTik8axtWv8BYI213+bWMgwNbwm40+T4xUcYyo3YS43E94hvJ53to7qwmq2f7KdyE6RlIe4hL8w3/aQMnmTA6AkRBDgsquXEaQKomBzOJvTBpLWK7Bzt7IjZXx3+Xc4hjsgUgh/+yM/ZvhAfUDb9Rdtrv+CyDKPGuR+/vgKTIzPjGfMk2OI7eF7TtJUCncVcmzDMbpO7UpIQggGq2hE4RCNSratfmLYRzDsYwBGGJ8hPx9SNCJg3mqwotQo/Sr+VeZXMv/c+azPXM+nXT+lX9R8lMqr/Hb+tsJp67/V2WCrhvCe7nGpuaYGZ7XewtYPdxPdNZoOozo0dJYm8+VFX1JSXgLjoMoWxLO7j/J/jZvQSASafs+JH2BE6ggeHPIYKz4fiPPIUV6I/JQpb01h8B2DGzmJYN3/1nFgyQEeNz0OQKW5km7R3cguLEVliyLMvBa2/wKdb65TA/1MpKF664FCEv78jN3uHTkvV8gJivb2lx+cPJjByd6doI7wd/hDOPwxpFwkCX9N5fhPsPu/EDvayx4pNSYCgCqbnvRx6cz6fhZJg/yjwnWZ0oUb/rqB6K4ic0Gj0LDwsoWUGkuR24PcUaSuBZr2hkv4OyfleZx7HMg6XYdS7n3bcTqcrfbobyoWCyCXE9kzEVcJq7jgOK7tcy179uzBbrejUNQSLY58CgfeFgvjkvDnRXjNmkh5TTKzK9K22lJNeW45JVklJA9JRhOqqecMLWfD6xuwvbOTkqEansxawIHIqdw19C7JlrUVeAl/phOg3wGmQrfwF6oJZWz62Hrf77KPawlWK3Tc+gOqv3LgIbg3pAr+vgsGvdGk99vtdvbs2UPPnj29+287x1UvuN6Mv/jxIFOA3BNGV1BVQObcTKosVRgeMzR7YOor4y9laAopQz1RtlG6KCZ2muj9RkOeiB6VHA4aRWRyOrFW68ECqD02nC5hvtRYis1hq/O8bcrCyifjP6Eyr5LxP96LxhZPunkaw5L7+e36T0bqvz5oQh3vMKWB6GMHKNwZD5c17OMok8mI7hLtta2wWtQGjA/xKEIu0ckcMRH6n9RH2ztyBfR/yeeuSJ3ogy3J+DOVm/h0wqfIu2YAExu0+tyxYAfFe4sZ++RYxqaNJTE0keSwGmtRSyksTIXuD0DfpwGhBboy/gJl9dkm+2/BH1B9BDJurPcQV8afSZVLZS1n5CW3L2HXF7t4wvJEs4OYmkJcZhyPVj2KTCaj/DUxgFbVl/FXvAEi+vis+SlRg90k6iBG9oVOszknQwSkLN0Nx7tPJLVpa44tb95spzK/ks66zgyy3YuscjQOXRpEpgW2YT/RJvsvwIRf3f9Vaa3kRS6iUG3A7rgKhVxBTPcYxv97fKubOfTbIX69/1euX3M9IQkhbqtPeY3wF8IhOLRSWNAHnxnfaZuk1ti+tpC7/tX14ve/+nq/iUQAdosdm9GGzSKiESfE7CBTmQeO+88q+/LT1n/XXC6szy/Oc2d6mWqEP4Peyo83/sjA2wb67TuVyWU45KKsi8KhkwT5NsioDqMYmTqKGe+BRVdKj6v6E90tuvE31jD68dFkXpnpDlCMDopm3537uPJKqASCDavh6DOQMv2sEP5O1oxOBW0or//spOxIGSd2nsBha7gGVUlNtrJb+Bv0Fkz4DULSA3p9ZxWdZsPkv+pEoffp0JGkksuJKplCcFIEPS7uQWiifxYXdVE6UoenusVdjVLD9O7Tub7/9ej14pj0qP3oct8UmWPtjKgoYV376vqPKc+sa7ey6MZFPK1+ukX2gC3BYgGZw47CbsHp9LZe9XkD7nonjPoaNHGn5PrOJMLCICk0izv7XIQz5zu31YPBamDngp18NukztwWuvzFXmqGigmrdATaVLWNv8d6AtNOecGWHGY1A93vhklKIGnBK2rZaoSyxB8YJYvIwRlkJxxY16xynYwDV1nF9p/UKf6kzYOCroPEMzINUQe66py7rqqV3L+XNbm82qU1fGX9NYtPtwtqpJmNeon6CgyFMU8w1uiix4FmLaF00MoRYW2Koe/+1W+0c+eMI+Vvy6z1/+vh0ul7QFasVYismM92wmKcnPO05QL8LfuwMRz7zzwdC6r9NwWq3MmfJHC7/9nKMViNBtgo6bltI+casxt9rsFKZV+k11jpRLVLB4oPrCn8a/8frnNW4amC2JONPE6rBpDchR8wTGxL+9n2/jzXPrUGulPPKea+w5Mol9InvI3baDBA31svKrLISdwBiXACHsW2u/+79P9g0p0Eb656xPVk2Ywtj9+ykslb1h87nd2bY/cNwOgJTnkEmk6EOViPXyqmyCE80pT287kKm8QT8Ogz+DryDxhmNTAn7X4G8JV6bXfewhvqTP4jpHsN9Ofdx03M3Mbr6FeLLL0AnOyH64xlCm+u/J6HS2NiScSkbk2ZTbfHv77XnzJ5ctfQq4nqJG6TL6lNuF+pUuOUv2HATlG33a7vtDmM+lGwCuxmNzoZFUUZJZRUJ/RLoe21ftBH+rb0XkRbBXQfuYv/k/QBMSVpLX+fD9ZYFOpM5Lf23800iyAhR57hDeAdSQzrixEGFVceVS65k2L3D/NbcZd9fRpd3RcR+tEzFoOhPRS1sidNL1WGRqFSdCwjDBY0GzMFRjHzhQjpN7NTkUyUPTqbXrF51trvmJZa022Hqbgive4xE0zj77n5tjNXPruadPu9gqfaMPH/K+om1OWsx2zx+Li6rnmjX+psySEQXSTQdXSLEDq/zUB/WqTcj8r+ga/5TFBb6t0lzpRm7xfcD1yX89U9dh2zzXe3yASWXQ0wMHCgdTIGxJwB7i/Zy4RcXctX3VxGfGU/36d2xm0/NoMVshrCig6yZ8hzbPxWD+N8O/cbyw8upsPiwiAzvAR0uPauiw/xFaCioFUYGJy3GUnLQk/FnrSbjnAymvD2FsJTA2E+MeXwMVbfcT0WYHqhZdKvOERaV1dkBafNsxyvjr56J0YdbP+SZP59xW8TVprqwmk8mfMLal9Y2u22LBYrSB1N2kWj3P+pzRV0biVbhqvFXr9WnD1yRm4C7hpRCrUAdom40gAl8Z/z9fOfPfDLhE/frZQeXMW/zPPYU7fEclHQ+dLtbWH5KNEhICFRZItlWeRPEjvLap5AreHvq2yyYscDLFtKF0+Hk04mf8tdLf9V7/on/nciUN6f4/C4BsBtBoUOaQgSQ8r3wQxLsecG9SSlX8t6W9/hq91cUG4qJ7xHJgcGXU5XeeLb10ZVH+V/y/9jx2Q73Ntd93FfGX/qR82HHU376MGcRf98Nqy+tszkuWCwau2q4NweZXMbdh+6m5z2i3nVDVp/T3pvGnL1zfLtkBKfC+KXQ5Vb3JpfNZ1QUZ119lAbJfAom/t7gIVqllpGd+qO2xWAyeZ5dfa/py7kvn4tSE5hnUXVhNcc3Hae42JNmqPIl/Mnk0Pe/0OHygFzHWYNcCdOzhZUgsGjfIt7Y8AbF9sMkZq3C8fW3p+xSXH9D/bLTYM2sU9buWcm+V8UPEBWqBae455VWiUg2c4WZTyZ8wurnVreqmfAO4XQ+r7NbeHJZfcpravwZwyfCuKUQPbRV7bR79r8Oy4ZAdQ6/6+7g1/5RfH74VTqO78hFn1xEXO/ARKa4Mjjf2nM7q3Xrm2wFLdEIGTdCDyH8RQdFk31vNn9dvRsZcqqNcjqf34WYbjF+bdIlyndSwcyk2XB8sV/PL9ECiv6C9ddD6WZMNhMHSg5gDBHB+E21rQcxN/UVbOV01lpXCAqH8J6SA0IrkFZYAkz3i7sTlhLmtruzO+xc9OVF2J12jt9/nKRQYTnpyviLjKRGIHKKP+6zMDIloDidULJRWF+FdQXEMz4uDnJy4MjmUuZnzmPIXUP8YhGx5LYl7Px8J48bH0epVbK/eD+b8zfTNborNv0gALKtU2DiCghvn3Ua4uKgsNBJ0fFyunfRYXVYWZy1mLjgOBY8uMCvEUGNYbGAVRNKytS+7roADy1/iG0F23h96OuMZKT3Gxw2aSG6HlQqOGHuw0Vf2Xj3XRnB+icBMWlKHJBI4oDERs7QOqxWvOtPlWyEdVfDyK8kO5YW4BL+bDawmm2oin4W99G40e5jnvnzGY7ojzCx40SvxWIATbiGgm0FJA5s/vfuGtRNS7yFpy6dKWq/qQNTX6c90WjGX9Fa2PsS9PgHxIp7n1wmJ1QdSqWlkgpzBXHBcZzzf02v5eYr489UZsJQ5FEfP9j6Ad/s+YbXznuNnrEiIIQutzW5jfZOSAg4nEq+yZlHv4519982qP7fpVKjZOrcqcT2arwujs0GThyoVCeNQ6MHw1RJmA8o6igI6uBl4yqTyYgJiqGgqoASYwmxyamUx3cjqIHTuAhNDmXwnMHE9/HctzVKDR0jOpIW7nlems2glFvQGHdAVdMtetoNlQd8Bhfd2P9Gzs04l6EpLV8cbkqGUlB0EEHRQTicDuwOOypFw35XrjIS0e3tq4xp2vcQFCQCFB0OkR3pdt0JIFk/ZfHjjT8y6SsR3Ct3aJA71XWFP20s9Ho08Bd0NlDL9uulv15ibe5aXhiQTLDejKI8sMGAlXmVHF5+GG0fLWX2IBzySMqibiE6uWdA2z3rOTQPnA7ofi9arQyFIwi7opqyKgPpMaDUKinYVkBcZusEI6fT6WVpf1Wfq+gT34efP+yHDZAFJUGSf8rDtGsSzwN1BKgjCVKEgB0qzVUBa64yr5L9i/ejOaYBDRRWd6JaOyRg7Ul4ykvYbGIco3DaUGr9s4a26e1NFDlE7fJsUwTfFS1iZmp3v5xbohXEj4OxP0HUILYVbGP4B8MJTezIuLz9rLpnMcXnpTBkTuP9rmhvEe8NeI8J/53AiAdEHe2Ptn7EK+tfxZxwCV3z/4nGkQ8mpRgbSbQIaUXbz8jl3gskXad2pevUru7XJcYS7E47MmTuCFE4aXK24wkRxXCZURL+msuJFfDHRGGVGub5vcfG2Tl4vJwSg4qEfgl+y0RKHZmKTCFzP9iWHVrGPb/cw6xes7g+5CsAlCFxEN9+rSI7dIAO5rcZfeJOKF5BbFA3AIoNxTicDrHIf4qwWMAQkcTgZy8iucYd6aj+KAAjeo3w7r8OG3ylg45XuyNJJbwJD5dhNIo6f7Uz/gLNwWUHUew2Yu8tJg2h6lCIGQFjFoqC8BLNRlcrgMpodKL6czp0mOUl/EXpojiiP0KpsbTO+5UaJQ+XPtyitm1WJ13Wz6csOYPk84dD0WqQmSDEh6rhA7lcTrdu3eo8f9s7roy/eoU/c7EYa3S4zC38AYRpwtzCX3PxlSU2Y8EMr2NKjCLSKSbIv9Gg7QXX91rVwjWTQbfVf4+0Gqz8POdnMs7NwBrXmy2dLmOZegkZW97gxgH118tqDVL/9YEuHs5dX2dztC5aCH+GEnrWaIJlpU6g4Sj2hL4JTHlzite2O4fcyZ1D7vTaZjaDzaGmdOxxYmMCY3V4RjN+qc/NDdW/bQpHVx3l8KJj4BiBxVJ/Pyg7XIYmXEOxspi0V9OI1EZS8lCJWLg+8hlUZEGvx9wR0a5s7+C6yb9+o832X4cN7AZQ1T/f+3bPN+zruJqowoupqBhPVBTs/X4v2z7axrmvnktUhv+VwKRBSUx8biIZ/TP4fdAKHvunUHrrZGQ6rCCXChk1CUMeWEogItPtWmCVVXFw8DV0SAtshs+JnSdYOHshubfksiDpAzJNcylOf91dR76t02b775gfQSGiIWQyUDqDsVNNSaUY0CrUihbPOWqzcPZC9nyzh4dKH0KlUzEsZRjDUobx15tQDCgVjT9fJZpA/FjxAwSrhPBXZanCUm1h6V1LSRqcxODb/VeQs2hvEUtuW0LUxVHQF4LlDjRKE+BfS9HTzWnrv/tehZxvYNwSIegigpdcgTQLzvuMsn2FPJD/gF+aW3bfMsJHhTNk2nTK8jLJTrgQAmMqJdEcglLc9vIhaiHM2uRVOGUKcn7aSViIs0nCn0wmI+OcDCI7eoIdD5cdZmfhDtJUYn1Cu/UaKNsEl5YH4IOcek7HM7eNPeXPflw2MLHBsShrZRK5rD6jooBO10O/56UBf0sI6w79XoDYEe5NTqeTp9Hya/9ojhoqmf3HbAbePNAvzQ2+YzAXf3qx+3WxQVi3ROui3Vaf0REmEbXWTklLg6Nlfdlafj2oI92LvQ6ng/3r9rP0nqUUbGu+PVJLcEVSuyKrK8wV6E16ADrHdPY+2G6ElIsgos8pubYzkbAwJ33if8dWtNld46/aUk326mze6vEWe77b08gZWsa6l9cR+udP2ORi5TtEHQJBSaLgb1ByQNo825HLPf3CYFLBiM/d/v0uooNE2oAv4a81WKvMBOuPYz5WDLZqWD4Wdv2nWedQtysfs6bhyvir1+oz+QK43AbpV3htDtOI2VS5SQyu87fks+5/66gqaFxpakqNv9rPSdHAr/D7JCje0Oj5JTyL+JPinoR119XZv694H4v2LfK2Um0ihmID2z7exrENx0RWtVKPTWZEo6xV9O34T5D1Ntj9VzxJ6r9Nw3UPLjYUExEBfX/9P1JWf9ksS536cDg8wr1Gg2SJdQrZ+91edr3yOypzVYPf5XsD3+Pby7511xJUKVSebJXsr2HvCyD39CWjcMZyPwsCRZvrv6Yi+DoItj7U4GFLDy4lK/INykL+cgdS6I/qOfTrIYwlxoBcWnyfeEY9MoqELgmMTB5HbIXIqPfK+KvOge9iIOutgFzDWceGG+DX4YDHrtzsrAKZzC/3xoZI7J/I5T9eTklPEdAkd2jqZm+2cdpc/wUIzfCqV6pyipuYvtq/Nf7i+8bTcWLHOplJrrFsXN4/RBBwdY5f223PhGhEH622VqHUKNn28TaOrjjq1zYS+ydy9a9Xc6DbAQDeHHUzw0vPTuet09J/TYWivptdPCenLJhC77m9cESKmorR/VLJOCcDp9M/AWTX/nEts16dxQPJC+mW9/QZd48963E6RRA+YJVVglzO5FWPM+PzGY28URDbM5YrFl9Bjxk93NuKDEJIVNtikclA1mEGdL2zvlNINAFJ+PMzDoe3wLNgygJ+uu0n9+v8ynwAEkIS3NucTk/GX1QUkHIh9Hgw4Nd6VhKUBD0fgsh+7k0ymYxghbCNyykqC2jzJQYx8I/WRbvF3PMjb4QvVWA3BbTttkp6OuwpHsXrGz+EyL6oFCr3wnJeVh4bX99I0Z6iU3ItFgtE52xl0+MLMZWbyNYLC5goXRRH9h/x7r+qUBj9DXS/75Rc25lIWJiMf409n4TCf9MrrhdXZV7FmLQxyJVyUYMmQAkDE56dQOHYS7ErROSnr1pWEs3Hq85f+hUQ4x2lFaUT0e/1CX8ndpxg41sbMVc2b6XFqtCy7bxHyLo6h4eWP0p25weaVdfG4XCwc+fOOs/f9k6jGX8yuc/Ffdf92ZXxd3TVUX594FdKDzUu+PrK+NvywRaOrjrqfu16Troz/qpzoHid5HDQRFx2Oh2CN0B+3QykV9e/ykVfXcTXu7/2+f4VT67gpbiXMJXXHZOEpYbxSMUjjP/3+Bo7ZSH+RmgjPAcd+gA23wUy/9S+lfpvPRz+FHY947XJ1WdKjCVotVAZnYYhPB5TI8PL9a+u55tLv/GqN34yrsCobtHr0RV9K4IwJLwp3wdHvwCLd8Sx2WZmSdYS3v373RYtdA25cwjTFt6ITR3UoNXn4DsH0+uyXpSZxAQjUuuJjmbkFzBll1dNalfQhy6AJVHaZP/VxIjAvcj+DR6WGpYKgFGdQ0VNgvuw+4bxhPkJkocEPojM9byEk4Q/cxGE9QSdZDHYJNKvgZ6PgtPpFv4sVKGpLkV55ABWg7WRE7Sc4Lhgul3QjcrISgDSdGUkHJgpAmTOANpk/wWw6KHqiPulSiYGtGW1BrSHlx9m7/d7W9XMiAdGcOVPV7oDKNblruPXQ79S6RAFUh0h3SF+olgTkGg5x3+GZUOhcDWhao/wJ1fKebjsYS756hK/NqeL0pExOYMpI6fQw3YF649PoSS4bn3eM53T1n/7/Rdm5INOlPfYV7yPPUV7kAeLsUmXm8dx0ScXednotoYOIzsQnxmP1QojUr7j1qhQOLbIL+eWaAWVh+DrENjxhPvZa5OZcGDDhrJV379b+LPGolKBrOsd0PdZv1x2W+B0PHMlq88AU5VfhSrIM5p3ZfzVFv7Ky0WkrUwGERGn+grbB2HqCCqNJeSV6lnz/Bo04Rq/WAosun4R8X3j3XXqXBZm0UGejL9q3RCIt4Hi7LIXaCppNeVjiovFAnRwsBAQKswVaEZouP/4/eiiT02hVosFYkuzOfTddhSfTiX7hBD+ate4kWg6YWHw9t/vMGBUKhM6TmBCxwnufXfsviNg7SYPTqYiDpzYAUS24b7XYOdTMPF3iPJPRm97Q6cDvd6TJXAyUdqGhb/9i/ez4okVpAxLIWlg0xesXAudfxYvZNP+3+k3YwFpSec259IlfOAS/urN+LObRJ0/XaKoKVzD4KTBBKuDRe1MoNelvUgZmtKkWionZ/w5bA4W37SYXrN6kT42HafT6cn4q8leovNNkHFDsz9fe8X1vT61Ygk/LFTWMaGKDRL1D4qqfQfU6KJ0xHSLwVJlQRvuPS6RyWTumtRWK9hcdVQ1tWpuDnhZBMTI/SP8SdRD9hdQuBJ6Pe4W6F1ZsiWGEmQyyB12KVZrw3XhAAq2FbD3h73M/GKme9uw94fhcDr4YuYXZERluDNjJnf6ANWG9yH5BCiloBovcr+FHf+E87ZAlEdUsjvtTPtiGgCX9rrUHSTTVKK7RmMNA+f7NJihNOFpMcZauG8hcJIgrwoBVVev413P8kAKf20SmQxG+Q58qE2HcFEbzqjOobLS9dbAZrr++o9fyVmdw9gfx7Jwz2+cCE8joWIqitq306iBcO66gF7HWUXHq9z/dS0+mpxVxORuJfHgGspz5xDTLbDW4ma76Lixqmp0JT9A1fiAtnfWs+l2yP4SLreAXIWmRvirqDWgXfHkCvRH9V5ZIq3lkd8f4c/sPxmq+ZpYLsWSehPE3+S387dbHGYw5oHNQKhW9FGjXaRZnzwO9Sdzp83lgRXwdRZ0mw4pjb9FogW4ArBVQdWYaXkpAl84nU4cNgdypRyLRUalJYpCxwhSNFK5iNOOKhxiR0FIJ/ezF8CuqKZ0j5FjGhspQxvvdRvf3EjF8Qom/neiewzmmsNqbLFuRyqJ1iEJfwHm1q23er32Jfy5sv3Cw0FRsho23Aj9XoTUi07VZZ5d/HmxsNYc64kEidRFcNwIJ8r1bPo8i9Dk0FYLfw67g+2fbqfHjB51hL+YoBh26sVx+vh7YNQ9rWrrTCY4GGJinFyZcSNVG7sQPP5RIrWRHOUolbJKQpNOXRSd2QxH+07nkVXnowpSuTP+XJN/L/J+gbyfRfZtsI/9EoSHww+HbyCsH4xu9Gj/4XQ6sVpljNq3nnfedZCYCOToIXowKCXT95bilfG34Rax8HxpuTsTy7WY6brPnUzvy3qTMiyF6K7RzWrXrq8kvCCPvFKxSuklMki0mEYz/ixl8MckYZ0x6A335jemvOF1WFhKWJPr4tbJ+JPB1b9ejS6qpuaU1eBeIPOq8Sdl+zUZ1/dqdygxGuva+MUG1wh/Bt/C37B7h7nHLCdTXVhNxfEKortEY7OpsdYIf14CQ0gn8SMRWAa/VSersnbGH4i6YFZrw2IRwEUfX8SFH1yIXCH6mcPpYHP+ZmwOm9vG1XWOJYcf4NxrJ4Am1o8f5iwhdQaE9YCQdK/NQaogYoNiKTIUkVOe02zhD0DhtCG3ObDI1DidDTutuqw+I3U1GX92E1TsF/2yVmbKqbL6PFNJDhNZfWZVgVv4M5WbOLb+GFGdowJS489SacFQbGBr0VYeWjWHqIQxdDBN9Xs77RW38OeopCyhB9bQKILjAhfAsGP+DpbcvoTQG0MhEnaU9qf0HBvRUW0sg+5MI2mqsPp02gEVY3mSrMPlpA71BFxMeHYCdou9Vc2s+s8qVEEqRjwoSsQYrEJYdFrFmFWyFPQTqReLHyBc+z3gEf5KD5WiP6qn00T/jSu3fbKNxTct5vJFl2O1ioKb0nfpR6oOi/IM8eNAl+gu96LQiQln0Z4iFn+wnswrM0kfm96qpirzKnkl5RXsM+w82+c50tX30oNlXCMNUU8/2hgY/wsAGkAlV2F1WLHJq8h64UeOqe3cub9xe85dX+yi5EAJk56b5N5W2+pTrXLAiimQcA70uD8gH6U9IK20nGLcwl9wXeEvKgrhlSxXgaK9hWf6EYdV/NQiNkRMjsuMeq5YNpvLFzbdRq4+5Ao5T5ifYPrH093bfNX4k7I4IS1NxpCkxSiKfgeEgKBT6qg2VlNyoITKvMqAX4PDUZONIpMREi0Wuo7qj4rr85XxV7gKst5w+5dL1CWsRgsor3G9stgtlBpLsVRZ2PjmRo6uPBqQdl9OeJmUP78AQKOWI5fJIe0ymPAbhHUJSJvtAdfioNEIhHSE2NFef/+NWX1GdY6i08RO7oyhpiI7nkuXv78kLCuIUVoYt+ceycLDD7i+z+pqYSleB3UUDH4b0q7wsdMbu8XepAUWl/DnyviTK+RkTM5wZ4C6npFqhVpMFO0m2P8G6Hc1em4JgVotFjBignKxHP1FWGLVIilU/K6PVRxr9rn3/7if9wa8R86aHCwWZ13hz+kEc0k9f1ASfiWkEwSneSlAD418iOJ/FPPyOS8DEJW3iw47l2A2Nf59uEQ/EMKRzSHSc+OCRSavS/grtnQXVs9Sjb+6hPeEDjNBHVlnV1qEGEe6AsqaQ0lWCW8nPUviwdU4nZ7M6doU7y/ms8mfsff7vXWtPsv3wNJ+sP81r/ecCqvPNkvOt7Bmlqj3Vw+u7GizstAt/BXtKWLBeQtabSFYH9PemcbdB++m2iIWSBWOYO8F6eyvYM3lXjaHEo1w/Cf4bRSUbPIIf/YqDBFJFCb3Rx0WuA4QFBtE6shUqrXi+5Q71ajUcpBLsfWtouPV0P8lt1tSX91UkkuvJMThseDtOL4jnc/t3Kpmtn64ld1f7Xa/NlrFnEduF38zQTmvwY4nW9WGhDdp4R1JKrmcDrbJAKx8ciWfTfoMq9F/lrxhyWF0ntoZWZQMs9XGXUNuJEH/P7+dv91zYgX8dSWUbQM8GX/ymvtgRZGJLe9tIW9TXqubUqgU9L6iN+ZOZhzYkDkVkojbRnE9f+3yKmIuGsWYf45p0vuuWX4NN2+62WubK+NPbYslLKhKOBRVNL92vYQHSfjzM3K551dqLDOy5YMtFO4qdG+7vPflvHH+G1zY7UL3NpfwFx0NJJ4DU3eDZHPWcsb9BON/9toUHRwBgFVRhj08ipCEEB9vbD5ypRx1sKeorrvGXy2rz/TiO2D/635p70wlPR1u/PEo35T8BsAvV/+C4XED58Wdx5td3+TPZ/8M+DW4rLBCSrKpyhaLz9f2vZYPL/yQy3pdRmZmplf/pc/TcFEuhGQE/NrOVMLC4MHhVzIrbBD7i/ejeUZDxusZWA1Wlt61lN3f7G78JC0gbWwa1WEieEIa/PkP1+KgwQD0elTcR2tZvc3sOZOVs1fy7IT6PdadTmeza/wZIpI40nc6h5MPoZGBEjvUMTCsH7lcXrf/SrgzwxyOejKCFBrocjvEjmjwPDlrc3hG8wyb3t7UaJuuBWtXv3Q6nV41r2KDY1l+zXK+ufQbYedRvgc23y0WaSWaTHAwjEj9joht54N+p9e+jEjxzDpYetDnew0lBlb/dzUHl9XdnzgwkQnPTiCmRwwGiwmnXCzEuIU/cwl8FwOb/GflLPXfenBYRf1La4V7U4Q2guigaBQ1NqshRUeIy/6byqKGA5QOLD1AwbYC9+sT1aJ+UaQ2ErVCjGFNJlDKLQTr2mc96tbiCiDLLm++8BeaFErPWb0whMYDvq1bjSVGjm88TlVBlSfjzyX8aWIg898QP8H7Pacg46/N9t/yvZDzjbCXqweX6G1RFlJRIZ5TMd1jmPrOVLpMCWwQWbVVLJAq7cGo1bV2lG6G3O9AJblXNBlbNVQeBIueWb1mseTKJdw3zJMV0JgVcmvofG5nrv7lagqSxT01UaNHo19RJyCnrdJm++9JeM1P/Mgtm2/hisWe4DejTdw0FQ5x01TnfynqGku0DlOxqFus30nfuP4MOPIFg6qeAqDv7L5MnevfrOdOkzox7rNxdFnWhffi1YxPn0+IYZVf22gLnLb+Gz8RRn/nrqMbpBL9Ra6psZhJTOIfRf9g+APDW91UcFwwMz+fiX6MHoCRYQX0kz0KxvxWn1vCD+x82r3OPWfwHM7RPYLSEYZuSCZ9ru7TpFOodCoi0iLcr51OJ/Eh8USqY1BbY7HLw2BWJQyZF4hPcFo4Hc/ctv2UP8MpO1TG4psWk/VTlnvb0JSh3DnkTkZ2GOne5hL+IusGkEr4CdeilU2pJyfLSNnhslaf01hqJHt1NtVFHh+1d6a9wwcXfkBaSJeawamT4Px5ULC81e2dyaSlgdkeTHa2WMxX1kRCaiO1jHxkJJ3Pa13EXlNwZaJ02fQ5fzywFIDM+Eyu7389w1OHYzl5ZihXCpsRKWqzXsLDwWgLodoSTphGLFJUmCvQRmqZvWI2Ix5oWFBoKdPnX8rxbuPY2Hka1/98KXqTHnJ/gO2Pg9WPxvLtjJCaeIiKCt/7O4R3YGz6WDpGdqz3HC8nvMxXF3/VrHZNmgiKU/uSF5zL70YomrAaUi5s/I21qNN/JdBowDWubM5iyf/W/Y/oF6O5Z6mwqA5NCqX3Fb2J6tKw7VntTBWX8Hd0xVGe0TzD1g+3AmJyOLHTRE/wU2hXGP8rpF/Z9AuUICQEtuVP5njyPAj1fn5mRAnhr8hQRIW5bme2W+z88fgfZC3OqrMvsX8iox8bTURaBEarhTj9NLpqRteq3eCAjJsgzr/mzlL/9cGxRbAoTWSz1INx+Hi2TX4QtPVntDidTr66+Cv+eOIP97bCahGQ6BI+QAQH9Ir9k3cnBMPhj1t//WcjpVvhu1jY92qdXS7L+JZk/KlD1Fzy5SXoU3oDvgM1Ukek8kj5Iwy+YzCdozpzfufz6RNfs6gS3AEyn6wTxHGqMv7aZP/t8aCoDxbZt95DXLbITrmNwgo9ALpIHYNuHURcr8Zr2raErR9u5fDyw14Zf8ra04z+L8KMAtA0zzK9XZN2mfidJU6ma3RXpnSZQp+kHoQVHiTzj9fY92PdZ52/mdntcjoU3czE2B1o1kyoE5DTlmmT/ffYIlg5TVgYA2WqXeRHfM+Bck9A6ernVvPf4P9SkuW7/EBTCIoO8goGd2f8OcRN0zFmCZy7ocXnl6ih+iisnw3Hl3iXlQAyzslg0G2DUOn8G8lrsokgJqVTx6XfVFHc7Uu/nr+tcFr6b0i6sD7XiSBsl9UnavFcqzYpCIoJ8mvNXINNDGiGhufR3f48mH27D0mcYg7NgyPzAXh6wtPMin4OrTWxyQE3lioLeX/nYa7wDHxlMhm779jNsklFaG0Jnhp/khNJq5CEPz/jcHg83aM6R3HlkivpMbPhosMlNeOVqChg3ysiQlGi5eh3wb7XvCJBhqUMI5PLCTH2YNM/F/N6xus4Ha2zqspdl8vHYz72EnandJnCDf1vQGYWKq5SCVxSDsM/bVVbZzrp6RAblEN49S84rZ7IdKVGyaTnJtHtgm4BvwazGXA6yes1mUF3DPLa53A42L9/v1f/pXQzGJpvldaeCAuDtza9x0tbfncLfw6nA5PTRPq4dCI7BSaawWoFh8xMYcQSfsj6FhkyyFsKu/8LTv9ZhbQ3XLbE5eVA+T7Y9giU/N2sc3Sf0Z0Oo5pXE9NmA4fciM0pVCOvemJNwGf/lUAm87b79MnvE2CVt8jqdDopNZa664hFdoxk5ucz6Tq1a4Pt1bancy1kasI0dD6vM+Ed6qnbqAqBxMkQ1vC5JbwJDoacil4c094EukSvfWGaMLegc6j0UJ33hsSHcPOmmxn373ENtqGyhzPk4GKeTP3TM3nXxsHQeX4VaqX+Ww8RmdDtXgj1ZB4dqzjGHUvu4M6fRc0MVUQINk0wVlsDk2EnXPj+hQy9e6h704kqkZ0SHxLv3mY2g8kWwo6yiyGsu38/y9mCOgLCe/msf9iajD8Q92vX4kZjCyaz+83m56t+5tZBtzZ43KnI+Guz/VepE6UzGkCr1PLp0F1M3nYCa1Xgaws7nU5+uu0nNry2wZ3xp3CclPEHkujnB2QyUGiU2JRabPbALRYe/v0wyx9Zzu0pD9An+z32583COegtr/t2W6bN9t/qXDjxO5iFQ89ay1w2d57JWr0nsDA8NZy0MWnIFC37fp0OJ4W7C70CuD0Zf0KdUgRFiSBgidYR2gXGLIIOlxIUBA5slJvKA9bcgaUHWPfIOkIqQ1A4ddidKhTqs8/zuq303/jgeFLCUgiuGcRUVUHpwVKObzze6nMX7ytm8S2LkW0X/Xx+9rn8GX6gTtCjxGli8hoY53Hac41ji79ZwZvd38RqaHhdLu/vPOYNnsfWj7bW2ecaC0fqToggSEPrrWPbCqejz0rCXwDRRmjpMqUL0V08A/gf9v7Ampw1WOyeWV1ZTfJZdJQTtj8BB8+eNNbTQtEa2HKvV82gGwfcyJ2JX5Con4miZzdGPjwSh611HS62RyznvnouKcPqDgg99f1kyFRBYrGgHZOSAud2fp9Hh5+P/vgRlmQtYdrn03j2z/otA/2NxQLIZFR0HUSPi3tgtpn5YMsHLD+8HIfTx9/C8vGw7rpTdn1nIq4afxUVIpNHLhOPlApzBU6nE5vZR6GaVlKeU85v9y8luNQjuAerg6Hvf2HaflBK9kgtxZV1rtcDxmOw5wUo9dg7VpgreHvT27y49sV6zzFt7jTG/Wtcs9qN3vwrfX+bi86gY5ROTnD2ArAEbkLYnnDZfdab8acM9bJzBbyyd5tDbeHPlfGXNCiJK368gk6TOgGwNX8r8zbPY13uOnGAqViqF9cCXNm5VfUkOL80+SW+vfRb0iPS6+yTyWUkDUoiKLquGrDsgWV8NPojnA5nnexNiVNMWDcY+ApED3ZvMlgNzP17Lp9uF8FkGoUNTVUxlQX1Kfvi++5zdR8yzvHYlrusPuODvYW//SXD+DznW4gZ5u9Pc3YQ0hEmrYSOV9XZ5a7x10Lhb8MbG+iw6TvAt/BXtKeI/Yv3e0VFu9l4O6yYUude2q5r/NlNNTVhDjR4WN+kXmhscVRVepZEFl2/iFfTXg3IZV297GrGPjXWd42/8j3CDs94IiBtn7XYjKI2YtE68ivz+Xjbx3y9+2usyensHXMrCaMDJ8Llrs1l7QtrqSoUne14dR9kXe9wZ8JItJCuc+AyI8QKhyxXRlG1xTOY7XN1H65aehVRGQ27UdSHSW9ibu+5/PG4JxveYBXnVzh1KJUgq9gHxoL6TiHRVNThwsklNAMDRfw8SMWXHSOwO+wc23CMt3q+xc7P/Zclm/tXLgfeO4DWpEXj0NAzZg1ae8uezRI+KNsG30aJhBXgtfNfI/e+XGZ0uA0Qc5Pvr/6+2Q5AvijPKWfLvC3IcoXwZ7DEYNV2FuUqJE4/wR1AGwNAqbGUUg5iUZRhs4n6jI2tBYZ3CGfCsxNIG5NWZ59rLJwRtg5WXSCCQSRajCT8BRDnSRMwk83EjK9nMPqj0e6BBZxk9Xn+VhggFZ9tFckXwKQ/IXqI1+a4GteW6s59mfT8JBRqRauaiewUybB7hhHbQ0T+FlUXsWDHAlYeXekW/hKiK6B4o6iL045RqeCI9ULe2Pge2SfiyKvMY8mBJWw4voFFNy7ih2t/CPg1uB4ersjao/qj3LT4Ji768iKRMVYbpwMy/wUZNwT8us5kwsKgd+wqzuvwAnZTlVswqLRU8nLCy8w/Z77f2yw7XMb29zaiqxJRP2qFWljHamNE1pC8df26PePK+NPrgZjhMC0LOs527zdajcz5eQ6PLH8Eu8PulzadTrAodNjVoWy4ZR+LRl6LbNNtYJWEP3/gEv7qE4gYuwhGfuG1yZfwt/Sepax5fk2DbVlrBfUp63FIXnpwKbf8dAvvb3m/ZkMf+H18g+eVqEtwsMiiH1rYBXY9U2f/tX2vZWbPmUTqfGddW41W9Ef1dbabykxUnahCJpdhsYgxrJfwd/RL+OsaMLQ+ilei+UTrRCBhpaUSi92CuiSfzJVvceznHc06j0ahoVNkJy9h2GUvqZHWUlrE0OShzL94Pm9NeatF7z++4TihObuR2W0+rT53frGTLy/8korjFXWfv+YiUcvuJBskV8ZfuxT+jPnw2yg4+E6Dh4WGin8rKz3bguODieoShd3in3GOC5lMRsfxHUkalOS7xl/eUmGHV+W7PqtEPTjMsPZyOPQ+WSVZXL/oep5a+VSTM2hbw9C7h3LXwbswxNqxKMpQqdtY5tyZykn3shCNCFQy2OoPcmkucpWcsU+Npes0j+PEa+e9xlPDXkZlixbj2F8GwHppLcCfxISFuv9fbqoUFp9OWu3EVZtRD49i1IZRlEaVEi9X88Lk0YQef9Vv52/3qMIgsh9ovC2xXXPO6moYctcQxvxzTKub6jihIw+VPETOkBwAYpRWgmXHxFqdxOnHVOS2ZL5+0fXcc6gL+ZHfIp84ntt33o4usuEBaGSnSEY/NprE/h73mp8P/EyfuX14PeteAIodA2DYRxATmBJC7QWpeFUAWfvCWlb9ZxU3/nUjCf0S3NY6GoWGcI3HUsRt9Rktk+yu/EFQsvg5iehYOza5gcLCUB9vaj07C3dy9Q9X0zO2J690Fh703eO3wK/jYeAb0O3OgLR7piCLHsSvawaRdAwie4jFyDJTGaUHSv0+ufaF2QwhpTmkr17M3h8mkNsnFxBR2jKZDIWilmAkk0OP++s5k4SLkBAYmLSUS3q+QHnJJYRpwtCb9FSYK+g+ozsh8SGNn6SZpI5M5YpN93H7M0eAWr7yxhPgtENQkt/bbC94CX/KYAjzjpJ2iQhOnJSby4nS1Y2yPbD0ALs+38WEZyfUb+9YC7sdCrqMpqDLaDrGQ0j8P6HTlaCNb/S9tfHqvxJuXBZvzanx50v42/vtXsI7hDPqkVH1vs+VISaXe2oL7lu4j+zV2Yx+dDRBMUGUGkWkU5QuChw2SJkBwalNvzgJQEyujbYQrA4dKJq/qv/9ld+zf/F+njA/gVzhiQGc/uF09///Nn/Jz/1vpDRnChfzrdhYugmOzod+z7f6M9RG6r/1sG62WGAZ9AYgbJDlMjkOp4NSYymq+EiOdR5Fz4y6Y14XB5YeYPHNi5n69lS6XShs1W8ddGsdm0izycnDIy+jQjceuD1gH+mMZ88LoIqALt6/v8TQRK7qUzcTsKlMe3cavwZdhDNf7lP46zWrF1GdowhPDafPO33I1mez7Oplomb86G99nvNUWH1CG+2/ukTo/3+NZq/+ceIbdqWuJrHyApzOychkMOn5SQG5JKfDCTIhAN415C7SrVNYtqsrKpcxUIdLIaSTsPmVaDqqcBj5FYR1I8QoBiJVlio0ThPxh7dybE0iGRnpAWlaG6FFG6El4Wk11v5WFvAg/LQEJq4AXfPGsaeLNtl/zaVQskHYTod0JFQr5npGm2cwW3akjM3vbqbrtK7NLjEAoAnV1HEouW3QbeTmwmZHTdBTj4dEn5RoHdZK+CEZ0i4nZuB7yB1qHHILBWUV9OzTgTl75/i1OXWIGmLAoXBgMOmYt+UVrhjWz69ttBVOS/8N6QQT/6i7uZYbSZ+r+vilKblSji5KR//0/hw+JufBLt8yrOBmsBuEpbfE6WXTHZD7LVxhJ1Qt1tjtiiqf49imkq3PZmfhToLDM4gBjPIO0Ok6v1xue0bK+PMztW++4WnhpI9LRxctbkoFVcIqICEkwV0vxeHw2ELGhBZLhUr9hcMOVk+Kwy8Hf2HMciXruo3DsHU/C85fQNGeolY1seKpFbzb/12MZWJmXWIQCm60Ltr9nTp06WKBrMaqoj2Tni7+zc7GLRiUGku5/s/ruWn9TQFv32oFmcOGTC4GEbnlNcJfeBoKhYLMzMy2Oflpw8hksLrgdu5btgm9NdlLMGiJ5WNTUKgUqGPCsGhEelGwukb4W3ct/CTVJWoNLuHPZT+NMR+qc9z71Qq1e1DnEnBOpiSrhB3zd6DP1jepzdpZYmo1YjKROLlZFh5S/62f2tGXPslbBruf94qc9CX83b7rdq5ffX2Dbbm+y9oZYkf+OML6/613B3d4CX9yJQx+E3o+3IxPJAEiS6XKEsU31TugxwN19pcZy1i4byFf7vrS5/u7XdSN4fcPbzDoptpWjkNhxCmrdcyAl2FWVZ26gq1B6r8NULoZ9J5sPoVcQaRWBGCUGErQRIVwvPtE1F3qX/SUyWWEJoWijdA22JTDXMrQ5EUkB9WtsyFRi/2vw+GP/H5adbAatVZMy31lKMVnxtNvdj/UIWpKjaVUW6s94x8fOJ2nxuqzzfZfhVbcGxuZf607sZyj8W9QovvLLZQGipy1OTytfJpNczfRK64XQyMuJMTU3fPMDO4AqRcLsV+i6chkkDYLIvsSohYrz1WWKrSYSN3zK7m/ZzVygpZjKDZQml2KrSbySSFXietppL5kW6HN9t/y3bByChz7EYDQmow/k90zmK0urGbtC2vJ/SvXr017jWX7/As6XevX87dLFFrhJBPWDaUSVA4RGJpXGhh3l8q8SsoOliFzyKiyh/Dj/nuRxY8LSFunk7bSf7/f+z3DPxjO3AOPAA3MOVuAodjAiR0neH7k81xc+Tu7cq/hROQ9ktVnWyF1BvR6Ahw29/PXJq/EcryIjW9upDy34T7+w7U/sGDKAq9tRQaxRh8iF656Z6MTyenos5Lw52dq23tmXpHJVT9fRXiqeLgdqzgGCOHPRXm5EP9kMgjPfxm+i4byfaf2os827Gb4Sg0bPdG4roVMq0KPubiS7NXZXsWcm0rh7kLe7PYmOWtycNgcmCvMqIOFR4u7ZkpIvFv4U4Sli0XNqP6t+khnA+kJ+bwztSu9nf92L1yVGcsaeZf/MJuhMqYTFdfModsF3SisLgTE9+V0OqmoqPD036Ofwy+DoVRaAGsMqzqNg6WDKK/Ucl7GeVyZeaXPTDB/UXWiitKDpTicwpfJnfGXejF0bd9Zta3FJfxVVIjnEj/1FBkntXB9t65Ah5MZePNAHqt+jLTRdb3afWG1QvzhdWgLv+ehP+7js20fNrvmW53+K+Gm0Yy/nK9h+6MiGreGcK0Ys9QW/nSROuTKhoeMroy/2jafE5+byL3Z9xIcL/qpl/An0WIaq/G3v2Q/F391MQ/++qDP/f1m92Pyi5OFxVINhmIDG17fQOFu8Ww0OPQAhKkivN+sDBZZ8X5C6r8NMGUnTFrltSkmSNTSKDYUuyfDDUXWdj63MzdvvNln/YzaVFmiueSbataZXmrVJZ/1TFwBY3zb06/OXs3cTXPJKmm+0GDSm9AVHkVlrGjUmlBv0gMiAxTDMTjwDlR620OazZ5HaSAz/s70/psYKqzKzKoit91nZV4lK/+1kqMrj/q1LU2ohm4XdiOyk5gDuQQGt9Wn3eTX9todTqeX8KeMCmHvyBtIuWRowJpc8eQK3kh/A51RqOu/HHsMpu4GzZkxxmmz/TesGwx9HxJE9m14kBhDmhyewWx8n3juzLqTwXcM9nmKxsj7O49Pxn/CgZ9FHVCD1cDvh3/n74INQP2W9RItQK6CCcvcgWoap5hnnNALUWD7Z9vZ/ul2vzX3+2O/s/vc3czsOIuYignA2Vmv+rT1X4cddjwFR4RgU2IoYf2x9Ryt3gOIuUn2n9nM7TOXfYtat66959s9vNP3HY5tOIbFAquyr+JE8qt+nYdItIL0K6Dv06BQe4Q/RRW2I7ksvWspRbsbTrSxVFrq1K4uqq4R/mRC+BsZ+hQsSgdDnv+v/zRxOp65Uo/xMw5H/X7Drolgl2iPfZqrvl9EBMhjhkKnG0TxeImWo9BA2pUQ4xnoR2gjALAq9RSmDuTW7MdIH5ve7FOX55RTklVC/tZ8Jj47kbsP3e2uFejO6AxOcGfMuBbSJSAiOgiLQ0tFtcYr469gWwE7FuwIuN2nayHFtVBWbCgGIEYXg8Ph4PDhw57+a6sCU4EUTdQEwkIdaJVVVFdaeOmcl1gwYwH9Evqx/rX1/Hznz35vb9Nbm1g8+Q201aKTuSPeu9wG/f7r9/baE2FhIgjF6RTiH93vg7TLvY6p3Xd9oQpSoQpq+uzKaoWEg2tJyN7HaxteZcieB+HH5j0D6/RfCTeNZvz1ehTO+1uIOTVE6aIYmDiQAYkD3NsqjlWQ/Wc2Dnv9v2NfGX/qYDXhHcLddpJewt/B92HN5ZLTQQtwCX/pzIcDc+vs7xzVGYDjlccxWpuWxnJixwl+uecXjvwubJTdwp86wnNQwe9QvrfF1+0Lqf82wEl1jgCig4QvYImxBLUa0rb/SM7L3zTrtJlzMxkybwg55Z6MbrMZHE4lMnXjFs3tmrCu9Wa8PrfmOe74+Q5WHV3lc39DHFt/jKCvPiG88IBP4W/+efOZN3geJpsJk00IRJHaSCjdAptuhyLvGqyu7DWZLLDR0m26/264CZYObPCQ2GCxsGRRFrqFP3OFmVX/XsWh3w759XIS+iVw2Q+X0fnczizat4hf8j/DqDounplOJ3wbDWtm+bXNdsPKabAozbPw6LCh0NmpjkxFHhG4e1rHiR3pd1c/LGrRabXKhjOr2xpttv9q4yDjRojoBUB4TfSC2eEZzKp0KqK7RAtbxxZgLDVyYucJTOXifppbnsukzyZx08rzAIgLyYXl4+DwJ634IBK+0MpEnyysEMLfmv+uabSOeHPoekFXRj06inkXfs6Fxpm8dl4/VPnNGyedCZy2/iuTw+5nhcUjnrUYs1P0T6MRUCiwGW3YTLZWNZU0OIlx/xlHVEaUO8D0bBRxzwZcz1+7vApjUgazV84mdWTD5Twu++EybljjXUfVlfGnc4rxmUMRBupor7WKM53T8cyVYlkCyLpX1mGuMDPuqXEAZJUK4a9bdDf3MS7hLyoKSL1I/Ei0nhGfeb10CX82hR6nDIqLITrax/saocv5XXjS8aTbqrU2ta1cXRl//WSPwbIVMPF3UAa4yEYbJzIunBuX7kCphMlakUlitpvZ9P4mtry1hY7jOxKaFJj6iyCEv7DCg2hsemzmfu6HimvS70XnW8SPRKMMS/yC5/tezdbSH4CL3NsP/3qYI38cYcqbU/zaXtqYNLpcP5Ktx0dye5GV/3uskdB4iSajUAgLwYoKYUEdkflknWMaE/5sZhuFOwvRRmqJymg84tligX0jruN4wjwADiqT6ZZYfx05iebRaMZfaOc6m5JCk/j7lr+9tq1+bjV/v/03D+Q/QEiC79qdLuGvdpR04e5CdFE6QhO9LWKjg6KheAXkfAVD3mn6B5IAPMLfoNA3YHcedPGuyRatiyZME0aFuYLDZYfpFdfLa3/xvmJ++8dv9J3dl56X9AQgaVAS1/5xrbvfGhx6UNQS/pxOWDkVEibCuCWB/HgSLioPQenfkHgu1HwP0boa4c8ghD+1qQJrUf3i7oY3NqBQKxh06yAAjFYjuwp3AZ5JOkCEdS09YkCjGY4UF9oA1gowF0NwxzrCbFq4yKqsLag2lbjMODj/PKoMqT4zOMNSwlCHqN1OGXKZnFBNKMSMgPG/Qrh3H3fd87Van/px+0CmFJkmTme9v4S4YFfGn0f4i+wUyW3bbyMiPSJgl/bvVf9ma8FWhuiWolIli2y/1Ishsl/A2jyriegNCp2X/a1MU4XMocRY4QACs1Lcc2ZPIs6JwPY/GzKHkoGxS+BAvghGlPAbI9IHkfnZuwTZUr26c2V+JQ6ro0k1xU8m45wMHip+yP3aaBPPUY1cZG+GaCqgfJcIBJZoPbv/C8pQ6HYXQQrxfRXVCH8XfXJRs4JGG6PnzJ70nNmTkhJQyi1oldXInK0ToCRqIZMJRwq1mC+43JdM9mpcRtVRfVK468BdrW4qaWASSQOTiHkxhsoEBwuSZ5Cc64AeH7b63BJ+IHchZL0BA1+vlfFXiUkTTvrYlgXduNZog4ilEtgne4Ch59ctayHRPCThL4DsnL8TQ4nBLfztL94PQNforu5jvIQ/iYDhspZ0yhxgL2Hf4lJiZNFEd22++ucS/ZbesxRNqIYJzwgLgdrC3996cWyQqgpM+aCQis+6/sZtNnCaQlHIFKgVatIvT6fzhM5owgObXWexQPSx7Sg37kIm7+/J+KuxzZJoGUZlF34/PBu6JgFgtVuxO+3MWDADmcL/q02dJnWiNKIT9n+DRgVBqprH2IZbhLg+8FW/t9meiIjwCH++aEz4MxQZmDd4HkPvGcp5r57XaHs2G5hDYiiPFZO/5WGTmDrklZZcuoQPXAJRvRl/TqfIcJZrQFF/1HTPmT2J7hqNUlf/sNFXJOaC8xYQkhDCzZtuBk7K+Bv8Ngx8HWRtrLbMGUBoTYzMx3vf58lZdX9/MpmMzlGd2ZK/hUNlh+oIf06Hk4PLDtJhtKc2nCZMQ8fxnmxbk1P0yXBNhOtN4jvTxvv3w0jUT+73sO0hmPwXxA4HYN4F81DKlURoI/hyHxwYejVdGoivWf/KejRhGrfwl1cprHJ0Sp17bAzQS/4SUyb+xNdyKZimQTbdCUc/E7UuT4o+TosQwl92eXazTxuWHIZmzFBMa33X+Lvw/QsB2FMkrLQitBHIZXLQxoi6uCfhyvgLpM1nm6cJQSUu4a92xp9CrSC+j//vc3t/2MvRlUcZ88QYqq3ioaxwBItnplIHI+b7vc12Q7/nAbGwpVVqMdlMOFVV9P9lLnuLMjh32hUBa9qVVS93ahmd+DZs/UsS/lqLzQA/94HkC2DgK/RJ7URasQjIra72jG3f6fsOMd1juP7PhmtQNwWDVURLaGrWbEpsvWBmcavPK1HDwfdAEwvd7qKrehzmEzEE2ZMBSB6S7Pfm7A47ZouMvcUjuWf5Ab65tfH3SDSD8B7u/7oCLgy2arRaMJmE3Weon+L5nU4npcZSnAon3SJ2oas+daWCJBrBXCScJyylhKrFF26TV2GxiLmmsdRIUIzvgag+W8/+H/fT+dzOXmvyLqtPrf3srfF3OpCEvwByzfJrsFR5Zm/PTniWnYU7GZYyzL3NZQkZGemEP86FxHOgh++aLBLN4MhncHwJDP8YFFq0Si1qhRqL3YLCnMu2f/xIpHk8Yx4f06zT/vbwb4R3CKfvtX3Z+PpGr0Wz2sKf63s19nodUl/316c6o1EqYUqPz7BbjJSW3kL1Y9VolKfuTm42Q1638SRMGYBCpeCJMU9wee/LGZk6EgCttpY9y8H3ICgVks4/Zdd3pmIMGsKry4YwIxn+8es/+L91/8fDIx/m+UnPB6xN16KYl9VD0RpQhfk8XqLpRERATk6N8HfwPVHvcvT37nolj49+nDuH3OmVuV6boNggJj43kZRhKU1qz2ywozQbcchbV/vNq/9KuHEt+tYr/B3+GDbcAGN/guSp9Z6n44SOdJzQsAWrr4y/ofcORRPmuc9/PvNziqqL6BTZSWyQS8PQluBa9MoqzIR6Aipdwt/B0oN19sV0j+EJ0xPI5J7gDGOpEW2E1r3NiB6A8BrHBOQKyLiBQCD133pIvgCCO3hl5saHeAQJ12S4oZpw1628zmsucrzyuDh1WLKXe8Xmqrv46cAFpIyVsv0aJGkK6BJ81qJ1Zfy1RPgDT623hr5Pr/p+IDLF5Jo6GW0u4U93CuIOz+T+6874UxYKi/MazJVmKvMqienmv+DAoyuPsvH1jYx8aCTVFvFQVjqCPTX+JPzCghkL0Cg07Fgcw/GUviT3bIHFTxNZ9Z9VZK3PgqGgcGhZeuJ/dJtxZi1Kt8n+K9eAKhSUYrCjUol7mdEI5eWeMdDQu4eii2rZTa5wVyHF+4rpNKkT2gitW8BVy8T5pBp/fmbSKpCLv7UZ0f9EvQ7SRDwSTqcTS6UFVZCq0XriTWHZ/cvYuGUjT45/ktS0mxhVOq/V52yrnLb+ay4FuwGCUtwZf9WWakJCPMLf7q93Y640M+DGAY2crH42vL6BXV/vInhYMFWhVTz0x298+E4ovg3XJU45nW8WP0Afs5Jru89h2y/9MFtFYIbD7mDOnjk+33p843F+ufsXZiyY4SX8RemiiAmKQecQ47Ou8vfgoMzdjkTLkB5pfkah8ERe6yJ16CI9g5HxHcczvuN4r+OrqsS/kaFVULJBiA0Srad0q7AQ6/8SBKcik8mI0EZQWF1IVaidyOsvpNuFSc06pdPpZOMbG0kfm86QOUO4c/+dXguaL01+iSP6I3SP6O9eZI2Rksm8mNblVdTOUnJKbqFTp1MbvmG1gjk4Cm0PISyMSB3BiNQR7v3du3cX/3E6YfM9EDdWEv6agGvyVVnpifiqMFdQVVBFxfEK4nrHodT471Hz020/ceywhfx4OKT9mm5bJnPjgBth2h6/tdGecdUl1euBqGOg3wmWMrfw1zehb4PvV2qUjHqk6VadJVkl9PttLlSrOHAOzKhcCofToNO1TT6HQqHw9F8JL1w1/uq1+gzrDh1ni4XsWgz/YDgHSg6w/Nrl9Evo16S2fNX4G/HACK9jxqWP87woWiey/WKGNOn8Eh5c911DtR2nqRyZOqyOiJoRmQHAodK6dapqC34uPh73MQ6rgzl7xQQtwtKbKnMVySGBHZdK/bcBwruLn3pQq0FTXYrhrxwqZ2W4LXW9TnGSBdqximMApIR5B2ccrJzIX4fhtnP8cN1nM+mXix8fpIaLvuLKqmwupnc/IaNCh/kq7zpvdoudVf9ZReqIVIIGBDGlyxTig2sE4DWzoHAVXKL3Ev9c9/xAZ/y16f5bkQXHFwsBPayrz0Ncwp9VVYK+woZraeSHa34g66csnjA/4a5R21omPT+JkQ+NJCQhxJPxZ6/J+Mv+Gk6sgMyn6jyPJZpA0VrI+Q663c2MHjMAOKqFv/pcwKjGzSdazIntJyheW8y5F1zPvl06ymJ6QVzg2vM3bbb/yhVw/lb3S4vdQnXcWk6U66mouJjkmgSxMU80L4C7Nnu+3cOqf6/i9p23C+GvxupTXWP1GROUAzkbhZ1yUPPWjCR8EJzm/q8rE8wVbLHmuTX88fgf3LLlFhL715V0frr9J3SROib+d2KTmirJKsF+0A7joau2kkkd54FhCgT5P7PwdHJa+++Kc0W21/Sj7vWfaqsQ/oqLRcDppv/7i8q8ylYJf4ZiAyVZJTiHi2AruUMn1fhrowxPHU7HscO5eT6YtdBjZg+sBmu9x3ea2Ilr/7iW2J7eZZdWXrcSgBdeEK+7Ov4H+5VnlfBXWzM6VUhhnX7GVajx8O+Hyd+aj9NHRGhtXBMzdVAoXFoOQ98P9CW2D/r8By4zQbBnweqCrhcwKuJyHMpwzD36E5/ZPBsXmUzGgyce5IL3LwAgumu0V62j8R3Hc0P/G1AYxOAwIsyKLvs1sbgpAcDPJR/ynz8Xuy1uAbJ+yuKFqBfY821ghRuzGeQ2M2p13T7pcDgoKSmp6b9OmPAH9HkmoNdzthATfJx7h15HiuNbwjQi467CXMG6/61j3qB5VByraOQMzaMkq4Sqo8VUBG1nr+IrNudv9uv52zuRNc5vej2Q+W+4pARCMwLXoEZNYdog8lMKCZFDr4p1YhGzGXj3X4naNJrxFztcZMZHDfTaXGYso8RYQrlJ2D3mb8nn3f7vsmPBjnrbanbR9Y23wLprmniwRG1ciyaXdH8a2ffRULGvzjGdo0SW2MGyuhl/AMfWH2P/j/s9x5/Xme4zPAsIg0pfYsT+1YxKFnbmZH8FP3aGwj/99CkEUv9tArXmEtsLtjNnyRz+s+o/qNUQWnIUy9eLKNhWtw6RudJMeW45Dpvnd3u8oibjL9R7AcyVZSZZ6rSchBAh2LgcQJqLXKXEIVfUqfFnLDOy+tnVZC3Jol9CP5ZcuYQPp9fUt4keCsnTT1vGX5vuv+W7YOuDULKp3kNigmL4b/JuJm8rpLrSsxDT85KejHp0FHaL3W+Xo9KpCEsOQ66QuzP+3FafhSvh4DuS9XVL0e+A/a9Aled558qk9FUz01/M+m4Wj5Y/yr0dPyQz5y00KovPbOC2Spvuv7UwWA38HDuBzZ1nUFTmny+016xezPxiJuFpIjjGlfGnkYmBc+fQP2HNpVBa//1DohmYCqHqCCDGsE7slFaYAEgckEi/6/qhDvad/nxk+RHWPLemyU1d+dOVmD4W5x4eVsQNvW6ByqxWfoC2x2ntv+lXQidhsRuiDiFcE064JtwdcFpVBVPfnsqVS65sVTPj/zOeKw5cQXVINXKHmoEJf6Ctlvpkm8FSBrk/QLmYh7rmEGYzjH1qHJNfrGtF70IXpaPj+I6ExIf43O8KKN4atghGfuHXyz7dnI4+Kwl/fsYl9P1y9y/MP2c+Trt4vTV/K5/v/Nxd58+FazHOdZNsvxXY/YwqBBTeqxfvX/g+zw38ghBzF4pbaNmuCdUQltywneCJE+LfziknYMu9kP1lyxo7C7GF9iWnvDclJfB/f/0f0z6fxjbDNhL6JXhlTwYCs9lJv19fomrel1jsFj7Z9glLspbgcDpwOp3k5uaK/iuTi8Xw6EEBvZ6zhbAgExM7fUK0bJOX8Nf5/M5MfG4i2gj/WlDM/mM2vV67Bbtc3DxdhYTJWwpl2/zaVnvEK+PPx/PocNlh3t70Nl/srH8A9uNNP/LRmI+a1J46NoKczKkc7nqICgf81v9rGPBys67Zq/9KeOEaW9Qr/NVDuFYshFSYhXAvV8kxV5ixmWz1vufkjL9Dvx3igxEfcHTVUQBOVJ3gvc3vsSRriTgg80mR3SDRbNRqYUOVVTIEY/LNdWqNAUzsOJGvLvmKl8/x3Z9+f/R3Fs5e6H49+cXJTHzWE03tU8hVBrmtt/yF1H8bwFoJ3yfAJk+9qIKqAt7++22+2/sdGg1UxHTCftnlJA2qm5FwaNkhXu3wKru/3u3e5rb6rC38Oazcl57IDf0elIS/xihcAyumQOHqOrtcWXhVliqqLFXNPnXSQ1dxZMDMOlafQdFBzNk7x3c2fe/HYcSndTafqoy/Nt1/48bC5LWQXH8RTLlMTvfonmhssVRVecY8fa7uw4SnJ6DSNRzJ4nQ6MRQbMJTUl1bvoXhfMWVHyrDarVgd4oHpFv4GvQUX54NGsoppEWlXwPRsiB3DyqMr+XT7p1TIj5CYtYqi+csC3rxr/HNraiYsbdgZoy3RpvvvoY/goLBoDNOEIatZtjxe4rFS3TR3E5+M/wRTuanZp4/tGUvvy3ujCRUPPVfGn6rG6vO4dRSM+ByipPUAv7D2SljaH4DfKl5nySAlXxlvAkTg2fSPphPdNRqH3cG+hfu8vtOZX85kzt45zfo7dQm5K4t78PrOXyDizOmXTeW09t/u97nncJ0iO6F/RE/WXVluR5KqKkgalERC39ZnsLvqbyocQTw0cha6rEdbfU4JP1F1BFbPgGPfY3fYKbcVYlTn4HR6nov1YSo34XTU/7frCtqx6bpBRKYfL/r0czr6rGT1GQCcTifT3ptGxbEKt0/1N3u+4bk1z3H7oNt5e+rb7mNdE7MITR4c+xuiB4NOci1uNTYDlG0XdikhnrpE0TX2wap33+SHvGQu/uziJp+yPKcck95ETPcYFGrviMzC6kKWHVxGWkQaZQXCdiIoMkb4mWtifZ2uXRId5SBYVUFpSShbtVtZcmAJE86ZwP1/3B/wti1GB6XJmXTpFU9hdSHXLboOpVyJ5QkLTmrdfG1GEXGrkIpuNAVVZDqXvl5JQnIQoyZ+AwixoOP4jnQc33BNsJZitYrCwYDwlXc6YeUUSJ0Bo78LSJvtBS/hz6KHE38IO8jwngDsPLGTOT/PYWjyUK7IvMLnOewWe4MCUW1cC5wzjEu566FiYZWmlmo1+otGM/5MRbD1H5AwCTpe7d7sEvHLzSLjLz4znrsP3d1gWy6hyFUXxVhqpOxwmTsAak/RHm796VZ6xPRgatep0OHSln0oCWQyETG9OX8K+alT6ORDi0uLSCMtIq3ujhpGPTYKa7UVp9PpVevNRR3r1rTLxI/EqUMZAmHdvCyykkKFwJdXmYdaDZagCCzpEQT7GGpGdIxgyN1DiO/rcbgI14STEZnhqbMJYKumxNyJams4yZLw1zCWMihcAZ1m19kVog7h60u+Jj4kHo2i+b/I+mr8yZVyYroLQai+/noyp7LGX5tFEw2xIxo9zJVBXVnZ/CaKdhcxN3MuY/45hvH/Gd/gsV/P/BqH3cE12zyZ7gpHkPjeZTLJ4rM1qCPED/Ds6mdZfng5cxLnE15kwnCiHDg3IM3mrMnBYrFQaU7AgZZsy/lEJ7XBmnlnIvtfBbsROt+MXCZHJ4vA4CwlX18KiL5SnlNOwfYCTHoT2vDW/d4HJg7kpckvUXQwlT2AQZYO6emt/BASbtIug5jhAITXTE6q7eXu3a5n264vd/HD1T9w7qvnMuyeYRxbf4zEAYl11t4a4sDSAzi2iIyaQlM8B5zngjS2OSWcHHBq0ps48scReszo0aLzHVlxhANZBwCQO4J45++XuG9ipD8uVcIfhHSC4fMhqj/7S/bT6+1eqHpGce62Eoqyylj3zO90v7g7vS/rXeetH4/9GJvRxp3773Rv+zvvb25YdAN9E/qSbP0MGQ6ClSXgiAC55PHaGiThLwDIZDI6jOzgtW1/icj06xrtXWPAdVOMl62BPy+DkV9BmndtB4kWUHUIfhsBvZ+CPv9yb46MsmOX2TCpQtFFN282vHneZlY/s5rbd95OXG9vA//tBdu5duG19I7rzd3KnQDExGshruXe82cjw7SPcPklL/Hq4cNEZoiHdpnx1BRBt9gVHO07nREXQFG1qBsQExRTdwHl0Aew+W6YtFL6/ppAaJgCky2Eikq8Mv4Cgd1qZ/fXu6nIj8OuEDdP4SvvhGEfSTVS/UB4TTmosjKgOgdWzxSWn5lPAqLgMkCpsbSeM8DFnzY9oOL473vptHkXQYkT6BWZKiLHZGnuBRyJ1uFa0DQYwOEA+ck+D04HHPlEZHL5EP6a05dPFop6X9bba6Dv+ptx/Q1JtI6QENFPq5qfWARAxmSPhW/WT1nsXLCTsf8aS0y3GMw2M193iURpi8Do2AdIYvxpQSYTAWS1cAl/xYZiUJgBDRYLOOyOOrXIkgYmkTTQOxPw3+P/zb/H/9u7HXUEL21bS24uPO27fJ2Ei+RpcJnR5y6ZTMalvVoe0GDPPkbCwaOYygYCnjmKpdqCpcpCUHQQD/7+IO9tfo/HRj/GY0Nugy33QcpFkOr93JWEvxocVhEMqg6v95A1lfPZlbqBoOrLgZEA6I/q+enWn+g5q2eD9YmK9wkLmagujT/XBs8ZjNPpRKvU8sNlP/DV99VUOtWolA4oWAFhPaRaYi3FYQNDLih0bicQm7yKrKHXM2Z84AyufrnnF4oLi3n8pseJ6jaataY/GdAvYM21L0bM97K+DVFEYbCVcqLCM/+Y+N+JTHpuUotO/90V35GzNof7cu4DoFdcL3rF9eKHE7CHZtjWSzSNWvW5YkLFmNLgEMKfSW/ihcgX6DK1C7O+nYX+iJ5es3pRVVDFByM+IPPKTKa8NQW72U5wXF2Hi5P56dafCNGEwNWgcGik7zIQHHwPchfCqC9B5Zkj1M74A/jh2h84sOQAc/bOIbprdLOb+eulvzi84jCj3hjHwV1R/JlzJQ+kNP4+iVOEOgI6XgVASHkOAPaa4HybU8Hur3YTmRHpU/jrfF5nOGkZ9ljFMXYW7kSn0hFWCaGaEobkxsHft8GQuQH9KGc7ktWnn3E6neRvqVvbL6tE+Ep3i+7mtd2V8WePHALDPoaYoafiMs9+gtOg/0uQdL57072/3EviO0oOJT7P/mGzGfav5lX7zpicwZh/jiGyU90oE1c9j4SQBLfVZ2KcCRz+qw1xNmCPHMavh26kuEztERDKSlnx1Ar2fr83oG27IqjV6poFM4Tw5yLUtUIe0hE6XOIVYS9RPyEh0DnqbyJke7zEgkO/HuKD4R9wdOVRv7VlLDXyw9U/UPLrZrfVZ7AqWNizdroOEppW9Fuifrwy/kI6iiiuWplZTRH+mkPl4WKi8veglNmg6C9Y2g9yvm32edz9V8IL1wTM6axHINLGwqUVwmasFr6Evx0LdrD7m93Ux8kZfyfjJfzZDPBjBmx/omkfRKIOISEQG5RD4pGr6+0zyw4u4/UNr3Ok7IjP/U6nkxM7T3Bixwl2fbnLnZ1Zbi7HLjdiVucTrqtZZDkyHw68E5DPIvXfphOli3Jnk1VSAA4HoW89z/dXfd/ic+r1cOyY+H+KtKDSMAEsyWDce5SUfb9jLvAOiMtanMXLCS+zb+E+ykxlVFurkcvkYDgGRz4V9c1O4lRZfUIb77/fRsLahtXsjWU/cTT+TXJtf7u3KbVKsldnU5HbcPDL4eWHAehyfpdGL2XwHYMZMmcIGqWGi7pfRG/nVciQEaLMgz8mwZ7nmvCBJHxizIMfO8GeF7yEP4dSjcUeuDj3sf8aS/S9YjFb7tCekQJDm+2/EZlutxGAcJWYfxRXe+6PTcl+ro/wtHDiesXV2e4KYhsR+m9htV2d3eI2JHwTHyECMczOcpxOUAWpiOkeQ0R6BEqtkjFPjCE0MRS5Us6EZybQcUJHXoh4gVVPN60G/NS3p3JilliQeyhjGS8OUItg1rOQ09Z/y/eJ2rRW8Yy88IsLGfnhSCyaPMCT3DL2qbFM/2g6UZ1bFvQ59qmxXPrFpXx34QoGHfoOlUqqjNVWCVWLv0WH3IJDZkEZFcpj1Y95lZGozaTnJ9UJ3CiqLgIgNiiWwkJwOBVUJd4mrNslWoWU8ednSvaU8Pnoz70sP+wOOwdKRIpyfRl/qoh06JR+Cq/0LEcVBj0e9NqkU4qwV0WwHoDiYs8id1NIG5NG2hjfYpAv4a+P8r/w1bMwZTeEd2/W5Z+tKNJn8MbLMwgNhcFaIaDqjXr+/M+f9J3dt8U2AE3BdDiP9G0bsR4ZRFFXz0MFQKFQkJFRk/2QPFX8SDSJ0FB4fuIYdpwYT7D6daZ1nUZKaAo2s43y3HIsVZbGT9JENKEaZn03i7W7wrHra2f8SfiLyJq4hvJycCpDkdVEcblwCX9lpjLsDjsKeV3rlaK9RRz5/QjdL+pOWErDmUJJl43mixOZlCf8m4S9Tu7I/Dey6MHNumav/ivhhVIpbFeqq6GiAsJO/jpkclDVnTSGqesKfyueWIEuSkevS3v5bOvkjL893+1BoVLQ7UIR8OQl/NlNoAoHuWSp3FJCQ0EvtxBbvQDK0kTAykk8s/oZ1uSsITEkkY6Rda2Xl9y+hC3ztnBv9r0MvWcoSq2YFpQZ9QAobWFoNTV9POsNsbja5bY652kNUv9thJxvRVDEgJdBJkMmk5EUmsQR/RHKbHkgT8OUkE5cZt0FzEU3LkKpUTL17YbHNHv/2srMHr+QK7ucmJjAWHSfNThsUPA7aKJEeYaT2Hh8I1vytzAgcQBDkoc069QJkzP57VBHekZ4+7ZGdopk0B2DiOkeQ9kusegdqY2EyD4icANnnXOdqoy/Nt9/O17XaCBffKj4fetthe5twfHBPFb9GDKZjMq8Skx6E7E96/rpjn96PANvGYguqvm/aNczU64KhkFvQnjdiHiJJqKJhe4PQNxYQvJ/BsAqq0Jt0GM+WInTkYJM7v/V4m4XdGP1ltWwGBROLeOCboUDA6DLrX5vKxC06f7rsIKtClQRIJMRrokEI5RUewIPDSUGsv/MJq53HNFdmpdNNOl57wXnI2VHKKwupNCUAiRjkcdCaGdQnILoifbAgXch9zsY9RWJkUL4s8jLMZtBq1Vwx547vIRcm9lG3uY8Rj82GqfTSc6aHFJHNM3Zp+u0rnQJ78LQbeeTe7wLByKD6eHn+tRtgdPafwe8DAP/5365NnctpcZSruiuB5Lcwaa+nCeaQ8pQEY12/DgEq/S8eX4f2D4b+j7diouX8CuLOkLMcIKHfezeZJNXYbFEoQpqXjRMkUGs0UaoYxFlPqNQj5wLZ9lygULRdOtifyFl/PkZuUbOiIdG0HWaR+DLrcjFbDejkqtIj0j3Ot4VkRksrV0HnAhtBAAynZ7IvN1senWt387tEv7igz3CnyamGyRPl2o21MJVY7GyEkJVQmEopZQ7dt/B5JcmB7Rta34xMce2Q1VVnYw/h8NBQUEBDocjoNdwNqLVwvydz/Hb4RuJU2aw+IrFzJ02l24XdOP+Y/d73QtbiypIRY8ZPZCnJGGrnfFXtg0WdoCD7/utrfaKy+rTbvedIRYbHIsMGQ6nw92PTib3r1yW3rWUEztONNqe1QpmTRHr5a/y6MYPkWU+CZHNK8Au9d+GcYl95eX1HFC2A0q3eG3qFNmJAYkDiAv2iAkXfXoRF35wYb3tnJzx98fjf7D8keXu/V7CnyYKzt/itpCVaD4hIVBQ1Ykf1Ebo+6zPY5JDkwE4Xnnc5/4eM3sw7P5hyBQy1MFqt1VkSZX4Y1HZIzwZDMM/g9E/+PdDIPXfRslfBvtfAavevcll91lmFZHV+WMvZ8zjda3JC7YWULCtwPO6qoCkl5MY/sFwL2cS/YE1zO77GCP7SZkNTWLlebD3/3zu+nT7p9y+5HYW71/c7NOGpYZjiEjG4vReKEkekszUt6YS1zuOMlON8KeridJRhXrZbLk4VRl/bb7/Dn4Tev6jwUOSwsVzrspZiL3GqEVWI7IDbH5vM3Mz51JxrG72X3BsMMc3HeeT8Z9gNVjrbaO6qJoPhn/Aprc3UVBVwIIdC8iyieejXBsJXedAvBTR3mKUOhjwf5BygSfjT1ZFUtYqZB9+2OB301pMNpO4BIeG7sr3RG3sM4Q23X/X3wjf1gSKUctxxOQR/or3FfP1jK/JWpzV6ube3Pgmwz4Yxs8lrwGQ5bwDJq8RzhgSrafqMBSvB2sVsaFismlTVLhrq56cvbn45sV8ddFXlB0uQyaTMf3D6WRekdnk5h4d/SjP9vyZjbtf46uCH8W84yzjtPbfk76vYJVYzJZpxPpM7dryTqeT45uOc2zDsRY3Z7WCQmaj3JIIqrNPxD2jCe0CQSmoFWq3I4lNUYnFAmWHy9j11S5sZpvXW3LX5fLdld9xfJP3/NSV8RfkFPfd8HBP/euzidPRZyXhz89EdY5i8guTSR6S7N62v1jU9+sc1dkrO8Jm89gPRh66Exalg60aCT+x7jr41VPU3SX8OTV6oo7vImvu7zjsTet05gozr3d+nbUv+RYLC6rFwkqUKsEdZRuaeRWM+V6qVVWLEPteHhxxDf0TfkVl8wzgY3vGEhwbWPXb2j2TLec9Suzwzm7BwpXx53Q6KSgoEAthG26pd0FHoi4yGawsuId1x2a0uNZUc7FaYXjWCt5JL2Na12nCx1CbAGdhNN+pRqXyBKLo9cDycfDLIPd+pVxJfEg8AHmVeT7P0eX8LsxeMZuU4Y17xpXvOU5QmbDKci9kNhOv/itRB5eYW1GfY9nqmbD+eq9N9wy7h823bObeYfe6t6WNTiOhX/2BLCdn/E3/aDpT3pzi3i/V+PMvoaHgRE5FtbbeY1wCUX19NWNyBpNfnEzuX7mUHChxby+p1gOgrC38hXWF6EF1T9JKpP7bCJn/guk5IkO2Btf3WlIj/FnqSay/dcut3LDmBvfrYxXHyK/KJ6c8x73IZjTC539dzb2//E3nIQMD8xnOJuRKUVO4610+dyeEiHukKyCwOajVILNbMVeY6z3GVRc7UhsJlYegdCvY6/4BnKqMv7Oh/6ZECOHPoiz0OY4Njg9m7L/GIlfWXTYpzy2neF8xhbsKqSqofxBsqbRQnluOocTA9oLtXP3D1SyXC2eaM9Eesi0TqhEuBlZ5FaVJPakccY7P784fzBs8D/2jevHCoeVb9DDkvYC0FQjadP+NHwcZNwFirWZWt9lkHn2HqDJPoHBsj1gunn9xi4JMVz+3mm0fb3O/NtrETVPpFNES9dnWS7SQ/i/ArAoITiVC5xH+Kip8/+0NvXso2ggtNpPN5/76qDhWwYvRL7LqP6vqzEvONk5r/zXkwfGfwSiCfF0OTDK1WMs++Vk6/5z5LH94Oc3llQ6v8OL4FxnwZSyrEx/m2b83QM+HW3ftEv5lwq/Q/0XAUyrEphDZvFs/3Mp3l39H6QHvEjFFe4rY9cUuTHqT9/aajD+1TazRDum0FtZcBsUbA/0pTimno89Kwt8pwF3fL8a7vl/tSAhlcAxo4iQ7AX8ik1G7YqhL+LOpSjnWYxLJz97RZG94Q4kBpVaJ0+G7k7om+BqrmPBHRp6d0QmtRWYrZ2zafNIidiIziUV+o9WIocRAeW596Sj+wWyR4VCq0YUq3dEktWv8AeB0wNHPoHB1QK/lbMNlL++K2rParRjKDOyYv4P8rfl+a2fXV7t4IeoFqrYdQO5UEaGNQKPUQFR/OG8jpDdcx0WiaXjV+QvtAmHeFryJIYlA/WJCaFIo6ePS0UU2vuJ47OPl9Fsr6urcEmqH30aftXUYTheujL96hb9ej9axxq4Pm8lW73Pw5Iy/1OGpdJzgsQ10RWlH6aKg6gjsewXK9zSpXYm6uOo3aozb631mNZbxB2AoNvDNJd+w8qmV7m2lBj0AKnu4+D6dTjCXSHWLTwdByRCcKmx5a3j9/NcpfaiUW/vdCYD28B4W37rY5wJZbXu7YxUi2tr1dwGwZQvoDZEYtANJ6dhGaz21NTpdB3GjfO5yC3/VzRf+HOUVDFz6X5SrvTOGFl2/iF/u+wUAvUkP1Mxp9r8GvwwAc1Gdc50q4a/Nc/QLWH0pWPT1HpIQKoQ/s6rQPY6tzeDbBzP2n2MJSagbXDY3cy4ntp3goZKHfNaAdxHZKZL7j93P2H+OxWAV6ZhKp1goTdM/Bj/3afAaJZrAX9fApjvcGX9WqqiI60JhxnC3jbW/0UXpcOiEMKVw6kAdLn4kWk/GDTB0HihFP5nW/VzSim9FUdIb15qpLkpHn6v6EN21eTafAH+9+Bc7PvPUR3UJfwqnuGl2VX0Ke15o5YeQ8EWENoJU87kklM2ktNx3Nm7SoCTuP36/22L58O+H+WT8J41mjTkdTuL7xBMcH4zVCudlvMvkmIazviVawInfYdVUKNkAeDL+UNXN+JPJZEz+v8mM+MeIk8/SKEmDknCkOigzF2NXVEvrq20c13q7tUb463lpT2Z+MZPQZO/5xYAbB/CY4THSx6Z7bXcJfwqT6PedYg9Bztdg9u0yJdF0pFgWP7P84eWgh0u/vtS97dJel9IxsqNbAXfhsmHRakHe91/Q91+n7DrbBcM+8nrpyjIwy0oxh0RTofZaR2mQyI6R3LHrjnr3u4Q/hUlM+OPjgQ03Q0Qf6OY7KrhdEjWIJ3Ya2L5Px31TbZgeN6FRanin7ztYqi3cffDugDVtyy8iqNyGSpnALQNvYUTqCDLjT7KMkMnh0ipRU0CiyVzX40bUnbOprFxO7EuxFBuK2XreVhZds4jRj48msX+iX9rRhGlI6JdAoUYESEiDv8AQGSm89PV6YPS8OvtfO+81ZDIZmXH1W644HU7sFnujiy3h5w1n7TqxyJmoVkNlFshOve/52Uyjwl/GDfXs8Oa3h37jr5f+4t6cewlPrbuwVTuy1mFzgAy3dSTA46Mf54reV9AvoR/od8CW+4V9ZHjPZnwaCRcu4W+89jpYVw7TD9c5prGMPwClVklc7zh3LUbwCH9qR4SIoTKXwncxkHEzDD1zshnOChxWMBwXlo4asbjpEpcqavqcrjiXLe9tYfRjo4lIiwCgPKecY+uP0WFUB0KTxIT7eIUQgFPCPNnYGzZAmLqYUcN0yGRS3YHW0pqMv9D4YEqSMlFEedfDyV2X6/4Ovaw+U6aDNk78nMSpsvps85TvFnWl+jxdrwOLy9LaovQt/NWH0+lkwM0DGhT8fFFtFSuiCrvob3KFDKxmUErCe6vQ7wB1FOf3vYuEkASi6Mxb33gvQPubq5ddzRN/PAGrQetUESPfCsZEqcxHAHC5V1gs1NSFa935bv77Zq/AGKO1Rvhz6LABGfKPYfdWKbvIX1RnQ9l2iBlBkDaGq5y/sOswmBron7W/H7vZTt7mPGG5PLT+94R3CGf2itn0facvWdsO8Ud6b/oF7wJe8t9nkYCYETD0A3d5DlfGn0PlO+NvwI0DWtTMZd9fxnOrn4M/IEku4/y0F6BkYkAcSCRayJH5YMiFXo9yWa/L+HlFKRpbLGYzJAxOIKGv7+ehSlc3FTdMEyYc2aqFu1Rh8LVw+RXUTuaRaBmS8Odn9If0WAu8I1cSQhKEHd1JSJOyU0t0kFgwMThLwemkOMeASa9AG9HKkSPw+nmvk1OeAwfEQnhCvA0OfwgpMyThrzZyJeFR4rZTXqZEU5Ma0ufaPtiMzbNyaC6atSvocWQfatUT9E3sT//E/u59MpmMqKgokQEql0vRms0kWFON0lnJiUphBQlgj7Vz2Q+XEdMjppF3N50u53ehy/ldeOEF2G67kXeOyxlV/V9iq/dD8TroNNvnAphE8/DK+PPB6LTRDb7fVG7ixagX6Xd9Py58v/6acACazK4cKhXWH0s0/blhxnfNvNqT+q9EHRoV/nywJmcNsxfOpmNER5ZfK76fhP4JZF6ZWe/vuXbG35EVR1hw/gIu/OBC+s3uB8CAxAEMSKyZ+IVEw+S1EHKaitKfBbgyrVcUPsrlY31HTCeH1WT8VdSf8acJ1XDbjtu8vtcwVTSRlSMJt/T2HNj5FoitW0eutUj9txGK18HysTDgVeh+j9cuV/BLfpfRPPvnaMKTPeldR1cdZeG1C5n13Sx6zBBZ267MT1fGn80GGzfCE6Nn0j14NyBF1DaJ9TdAwe9wUd2aiK0R/rTBCo4MmEHMScOmO/fdid1ix+F0MCZtDKXGUqJ10RDVGRIm+jzXqcr4a/P9t9fjwi5XXv+Sh0v4M6sKfT4nbWYb88+dT8qwFCY9P8m9XSaTcc5L52C32tm/eD/aCC1po9N8tlGSVULe33l0nNCRakuN8OcQCwBlqc8S3913nVaJZjBlOwA9gB6xPaiuhi+O7SD+8DqOb7uY5H6BmR/0iu1Fb/mlpDnSGGcaAPsfhn7PB6Qtf9Om+2/+b3D4Y+j9TwjvTpW9BH3kdhxWNeXlo9zC39w+cwlLDuOqpVc16/RRGd62865MXLlD3DS3qD/m/InNiASQaJi8pbDpdpjwGyRMquMW1BgZ52bwSPkjTf5brTRXYnJU85/N/+bSYb1oWojjmcVp7b+hGeKnBlemtVMpnm8VFWKM6Q/LXFffTNdYmJ72CBS9Kgl/bYmjC6BoNfR6lKcnPI3iT9hsEgEaLpwOp5eQf3TlUXRROuL7xHud6ptLvwHguefgMBAbC8jPPq/e09FnJeHPz1z2w2WEhjYtYs8VgRYcDOx+HoLTIP2KwF1ce6MiC/KWQNI0COtCQkgC52Scg86WhK4sF81rH7EldTIjHmg87XzvD3sxFBnof0N/n3UCJnYSE++568XruHglzDKC3eDXj3Q20C1mIzkRWkpK+ri3NeU7aC2VnftSpE1BG1T3+5PL5XTo0EFYDFYfhcj+Irpeokn8ZviS33+H2UkiUqegqgCDwsDAiwJTL8hicXIs5hNyS+xY7P+Cgl9h19OQPE0S/vyAK6pWrwcK/oD8ZdDjH6BtmoirCdPQ/aLuJA5oPNPTagWrslbNohbg7r8SPmlU+Nv1DGR/Aeds8CqYfrjsMPJaafGZV2SSeUX9WZ61M/40oRq6XdCt/kwIdSTEBv6+fzbjyvhbd3wWl3f0fUztjD+n01nvROPk7eekzmDh/hnuhRk00TDkXX9cdh2k/tsIIZ2h+/3C0rqGbH02L659scbS/k3s6iCUYTUva0gfl87ML2aSMsyT3eey+nRl/B06JOYi20suoNeYYafk45wVaOMgpKOwvpV7Z6i7hL8TVSdwOB1e99DG0GjEv//P3lmHR3Wlf/wzmkkmM3FXgoTg7tpiFdpSobLUXal32992a9vubtvdert1dxegQGmBFijuEiyBkBBPJpnMJKO/P07uTIaZ+AS9n+fhgVw591xuzr3nvPJ9A9VsVGnFdeZdNq9NbR2t4NLjfvyqW/d8ZkZmcivb2bM1npoAcU3qEDWmAyaiewauT6tQKPh81udkz8xu1vG3b9E+Fty+gCt+vYI6nTAAKBsz/k7W+lPHmrAwULodqBwNmKuCH1zqdrtZ8fQKhvQZwkXuS9lVWsGeMAc9E0+cd+lxPX7N++HAJ0JaOaI3qw6t5I/u5xBRN5yamjVCXQmIzY4lLL59LzqXw4W5xIwuUodWLyJoJKlPpVO8MxzadIgM1s3IkHAajP4AjELlQ9SpdlFtcgOtK700VRBpicMbD5P7fS7aei2EQl1DAjZV6zXnT0SOp/FrDDESrg0nNMxFSIhw+pSUQEqjsnzer3l8e8W3TPvPNPpd3K/lxhqpK61jxTMraIgRHqTcmnRe3L2WO2adnM/zhGX4K+DyfmOluazk+Htr5Fu43W6uX3O955hv5nyDIdngs60ppaXi7/TInVBaAbEjTyoHoFJ59CvuyY6/IONyufwMKK+vex29Vs852ef4yH36LMq2PSY+iLLjL3hUbxVSYroEMPYk2ZDMwjkLKS6Gmz82U5E5hPh+Ca23A6z/33oO/nGQIde3nKZeIurbismoSiv+yPhwZsgkkgdO4tfK+dzw4w0U1Rbxxsw3PAbKrqImMZuqELHA/nTrpxhCDJzW7TTCNGG4XC4OHTpEau0nKDc/KDJRZKN0m5GMw2azt6hvTUM70ovayLbPtlG2s4x650jcGlFrSq/VQ89bIPksYYiT6TRSxl9VFVC2AnY+DZmXeRx/eVV5LNi7AIPWwOUDL/c7X6FQMPvr2a1ex+12U/zQywxKqCP3QhitrIKC7yDtvHb11zN+U1OPyUTqeEdy/JmaK6Pqcogabk6rx/HXkXEsOf7UakgdlcrF317ss/+tDW8RERLBzOyZ6BQISdeTaBJ/tJEcf0fK6TQlzZjG17O/bvf3takTt6uRx28rhCXDkP/4bLI6rLy67lUiQiKYrH4ZhwOqDtZyYEs+/S8TzvmItAgiLvFVL/Bk/DVmgkpZ3Rut9/KXjqkwnZq0kM0jZY/ZXXaqrFUetZG2oNVCwv5VGGoPA+cDULSuCFOBie5Tu6MNb7KmcLthwSBImg6Dn/Zpx+0+ehl/x/34ddqgch2owyFqQMBDNCoNPSL6cMDR/Hdybt5cv20Fqwr446k/GHPfGM7/+PwWJT97nd0LQ7KBhP4J1G1vdPw5hOMvpvQ/oOsNKWe18+ZkfCj/E+pLqIwZz/IDy1EpVFh7z2Rb2hD0PYN/ObfTzZK/LqHfJf2w98im1hZDYeRD9DyBVD6P6/GbdbX402hLkUq22FVVPuP0oi8vCnR2i9QcquGFbi8w+p7RTHt2GuCV+lQ6hRMxTFkM9jDQGJttR6YdGHuJP428VDeOPcNW0KP8Zy5jepuaKFxTSNH6IobfPLzFY5Y9toyQa0MgDdJCS4nUHgSODwdZMDmm47d2L/w6DXrdAjn38ukFn3p2bU2B/fuhqMjr+AuLDSM8MRyVpu3lPGqLaln17Cpcc1zQAxockZQ5h8GpXrv4eCM8y/NPu9OOXWPCrlLT0BAJQNKwJBxWh08A6tSnp6IKaf53QXL8ZdU/A7+8CxeZTiqbgcvlOurXPM6+8Cc+Wz/dSlVelednt9vNHT/fweXfXk6VtcrnWG/GnxtmbIShLxzNrp78xE+EqX9A0gyfzTEx4NCFk9dvJglj2iYzdsbLZ3D54ssDRssX1RbxweYPWH5gucfxlxJdJBYgdlki4kgORD3Dgr03UVEBP+3+iXl75rH2rbV8fMbHWCq6LkNSiqDWaF1c/u3lzPx0pmdMut1uKisrcSdMhcHPQmTbIpFkBBnhfzKjx/+oq7H6OAyejn3ak7IfDHZ8uYPlTyzH4rR6tuk1elFLI3YkqDov2ytzhNRnzxth5l4w5nj2by/bzq3zb+WF1Z37ZrkcLtCFYKwfzbNZ27hcuR/Wt7/Op2f8ut2d6s/JSqsZfwMehbN3gC7Oe04Ax191fjU/3vgju+ftDtiMJPUZyFlU76jn+h+vZ/ZXs2lwNMDWx+EzLdTsae/tyDQiOf76GL6GH7pDyTK/Y0LUIZyfcz6jUke1S1akqRMXELKGK/4iAqqCjDx+24/kyDU1mFA1Zg+tfPI3vvnLN1TurcRmtlGxp8LvvBRDCt2jupMRIbKSpHdCeLjfoTIdRKvS8s3sb1h21TKP9FWbz9WCvqoQ46HtOGwiuGnTe5v44vwvqK+u9z3YWQ9uJzgb/Nqx2UCyKXR1xt9xP36dFlg8Fnb9t8XDOiKJXVNQw96Fe6mvrqffxf1IGZ7S7LGRmZHknJ9DWGyYp8afwqFHpbATmXcv5L3f9gvLBGbr47DiUvZW7mXW57O44+c72hQg01EUSgXXr72ecY+Mo8Emfv9PtOzN43r8HhFAHRUqHOt2dWXzgWxtRBOmYeTckWRM8GboXjfkOv4+4e9Eu0RG2riagbBUdsZ3FRq1MPpXWtr+MNe9to75t8xv0V7U75J+3LztZvKT8wH415ibmRYys1N9PV45puNXFSrUW1T+k4zkxljDwiZVBhIGJHDDuhs80vNtIa5PHHfsv4PyKUKGXucOIUxrBvfRd5rItICzHqyHwWXngV8e4JG6OPYm/tOT8XfWK2dx7jvn+qxD+1/Wnz4X9PFp5qDpIP1e7ceMD8/0zMXUPefAkP+K4K2TiGMxZmXHX5CZf8t88n/L9/xsajBhcwqPQ0K4b3aZN+NPARG9wdDjaHXz1EAXC3FjIcRXmkWhcqA3CAtlVVWgE/2J6RlD2pi0gPvWF63nyu+u5J5F93iiE1L5HhaNhorVHe7+yYot81bWFJ5DRYU3eq98TzkHlh/wN2wECbfbTeb8V0nZ+QtWdzVOtzCoxIYdIV0YNQhy7pGj+9pJD/Wn3Dr8ZpzWSh+HQfrYdGL7BK/G31n/O4tbtt9CnUO8PNUKDRqVBmzVYtIhExSiGoPWq6sRkmaG7j6L76bygc2x9tW1/HD9Dy1eR6VRETr3Bg73nklWeF+0w16CEW90tvsyR9ARg2ZEiMgUqnfUe+YwdoudDW9soHB14HpxTZ1FK55ewS9//cWzTwqyUClU4h0R2R8yLhESkjIdwlMfxRKGW20QToAgccPSs1k8IIXC8B/FBtNOIbllqw7aNWTaweob4M+rPT8atAYR9AI4Qg8D0OOSYVzw2QWEJ4Wz4e0NvJz9Mnt/3uvTzAezPmDvHXs9dVpra0GvqWJO+rmQ9+FRupmTgKrNsPUxEfEegFk5s5iQMYEQdUi7mg0JgfxB57LhzL95JESH3zqc894/D0OKgYV7FxL+VDjTP5ouJCzP2gbD/ANwrN7YKE8drFMWbaSoj9m95QpPK+wvsy3tDnZX7Qi4v2RrCWteWYO1yvuf23d2X/7W8Dd6niXSydwuN25X68acS/pdwgfnfUBK5aW4UVI9ap2oQyjTOfrcB6Pe8zjczTYz4a4aYg5upGR78OuXKpQKkoclc8uWW3gmTIkh668MKBkDh34M+rVOSexmKFsFFiFR3TTjr9rkNfzvWbCHxQ8sxlYXQCO5GfTxemY8P4Psc7I9264adBWPTX6MSHtvAEr0V0HaBUG4ERlABMR/lwb73gXAqBXrjCpr2x1/I+4YwRVLrvDNfj8CXYSOuD5x1KjEouenfVexXxFYTlCmE4SlwBnrRcbfEQRy/HUElVZFVLco0tLS6BE2lHNjD/BgdwMUtk3yXOYosf2f8G0y1O7x2A/sKpNPjb+m2K2B69IXm4vZXradrSXbABG4Fpp5GvS+C9ohmy8TGPl/MMjM+nAW3U73ys2VmEUKmDHEiE7tu/qSMv4M+gawloAr8CCQ6QQuB9i8E4qRb41E84SGmqjlJO1exuLrvmi1CbfbjbXK2qxnvthcDECMNhGbTdRY0WeOgUFPQ0Tf4NzHSURMo523stJb08t4q5GH6h7yK7QdLOz1TnC7Uboc1DrF4s+gNfgaZdxu8Uem3VRE38Tffl1MmSnax/F3yfeXMPmxyUG7jj5OT1xOHFaHeHmGqoXhk2XnwHcnn4THsUJy/FVVIb5LlkM+Bv+kcFG7r6SuBKcrsLPhwLIDbHxrI057y84IH0nB+HGQPKPF42Xaj1SzsVnHX+0+2POaqKfSiCHEW+O0tkFkrsf0iuG+8vuY+MjEgM00fZbbP9/Ots+2efZVWEX2UVRolIj4y7wUxn7qF5gj03b0ja+/9YfPoHbcJkg8LeBxvx/4nRf+fIH1Revb3HaJtZAGbRFaVWPKX/ZtcLEVYkd3stcyHcK0Haq940mhUHgCMOyhIgDDkJ1Mv4v7odVrSR2ZSv/L+pMxMXC9MYnaWojQlZEVMl/UxZZpG9VbYeujPs8kGISEgEulAYXCYzCJy4lj4BUDUSgUlFvKqbPXNfvdlZAcfzqdb93HU5becyF+QouH/Fn3CfkJL3HAnBtw/94Fe1lw2wIqcn0zaRUKBUqVkrWvreXJ0CcpWh84IOqLC7/gmfhncLvdDEocxJwBl2OoGYHLrUIVNxQi+gQ8T6YdJEyGjNm+jj9rKd22/EDRnweDfjm3y43NbKO+QQQe6lUuQhwHoHGNItNJTNth8Rg48BnQpA64wk1xlde2k7ckj5VPr6SuNDj/75J6RUHMv6H3nUFpUwaRsaNL8qjzRIaKxUl7SgokDU6i22ndUIc0X63KUmGhqqQKGk063+fOJV97W8f7LdMmvt/1PTM+msE/lv/DI+9ZdMTncOe3O1l076I2ZzvVm+qpPlDNI2Mf4Zme62gouYxtlqtA3/LcVuYoEzsaet0GagMROjGuHSqTR23N7XKz6N5FLP/HcgC+v+p7Xur1Eo5639q7ZXVlABjVQoEoPv4o9f8UQXb8BZleZ/ciMiPS83NpnUgBk2o+NEXK+MvQr4ZvE2HP/45GF08tvo6DPy70/KhtzFpxh1YQWltK6dp8XM6W08XtdXaejn6aH64JnL0iOf6MSmEMj40FdexAEXkYmhSMuzipiD14D6+c2Re73U24Rkziq+rbmHrZQdxKNdsn3UpB3xmYHOKjEqf3ytopFAqSo5Uov0uAHc90aV9ORlRROWwumUJ1bSiDEwdzZs8zyYgM/qSsOr8aa5WVemej40/VaPlOPhO6XRH0652qRDf6YqqqwF3WGKG53ytDFa+PR6lQ4nK7PN+4Izn79bN5yPJQi1r+NYdqsP22knLNv/ii5HEKTAUd6q9CoSAxMbFdUoanElLGn9Xqdc75ULUB1t4CFWs9m9RKNWEaId9iahAGFqVaSVhMGEpV4KljU6nPa1Zew3V/XufZV2mtBLwR2zKdR6XyyvjVtqAq/vbGt7lz4Z0s2reozW1XNwjjdriqSca2SgfK4JcGl8dvG5j6B8xY67NJcvzZQkRItbTArimsISY7hvM/Oh9NqFd3LpChpbYWimp78YXSBv0e7qLOn4QknwFnbIbEKQF3ry1cy6trX2XFwRXtalalArXdiqE8j4r8WlxOF06b18lXbhGBa7FhsVC1BXa/Chb/kHqfGvJdzMkyfiNCIgGoslYH3N/nwj5c/svlxPXxrh0KVhWw/5f9uN1uIjMjyZqa1ez3Mb5/PCkjUjz/T55AGWU9GkXXlTk4FZEcf/WOekKy4tk94jJC+wdfVammsIZ/Gv5J8sfiXby+fAi7+xRC5iVBv1ZXcVyP3/BuMPg/ED8JENLlIQqx7is2VXoOG33PaG7fczvG1Lar9Rz84yCfnfcZBSu9647NxZvZUbaDerv4mJ5osq3HPZH9YMYaEfgHRIcJB0Gtre0ZfyCcCE0zr49k4Z0LeSnpJaanTqebcjwqd6hXtv4k45iP39wXIV845g+bD7Nw30LWFa1r1vG3Z/4eVv1nFdaK5p9fU7Z9uo0XMl/g4O8HsdthR9l4frW822y9XpljRPIMGPYS6NO8GX9qb8afQqkg94dcdn6zE4DIbpHE94tHrfMdmGUWYaPVI+ZZcXHAktPgj4uPzn0cRY7FmJUdf0HmyMKqJXUi4y9Bn+B3rJTxpwiNg563QNTgLu/fKUe3yyHBGwUfEyrSzdwhlewffAETfrq/2UWahMvhYvC1g5uNnJYcf6FOUc1bjk5oGaXChVKlQq20o3UII3BlUSV75u/BVNBJ0f5msDVR/zDZmhhOpD4plcRH6VBE5MgZKB1ASM65qalxc+eoO5l32Txm953NmlfWsPzJ5UG7zuuDX+fLC7/E2uj4C2uUOqPvX2HIs0G7zqmOVOPP6YRadyZkz4WogZ79KqWKxHDxvmtO7lMXqfMxOgeiPLcc96LFKCxL+OTQI6Qs6Amr2y/HolQqSUxMPPqFzU8Q9HqQ/msCOojiJsBpv4ho+SYMTBjIkKQhuJrUUijPLad4c3HA6zTN+FOHqAlP9Orx+zn+tv0DNtzTsRuS8SDqF7lR73/Js/g+krZI8x6JySYcfwaVlKK/Ecq7RrpcHr9tIMACUXquDVrxXBsaIO+3PJ5LfY4tH27xO/6DzR+Q8GwCd/58p2eb9D4wGBU+cs4yrRASIwxPmsA1Rz7f/jm3zr+V73Z9165mFQqINBeQ/ecH7PpqG4fXH+bJsCdZ88oawOv4iwmNgeJfYN2tUJfv146U8Rca2q7Ld4gTYvzu/wC+SYDyNc0eEh0mAhFNDdUB90dlRZF1ehYhRq9SyPLHl/PpOZ8C0POMnlz202UkDQkc8DnpkUlc9tNl4rwDy/l2xw/Ua4oYk/YNuh/0UPBNR+5MpilbH4Ov4wl3ejOItPFuauJ7YtMFv4SDWqdm0NWDqMwQ8xuVW3fCOYuO6/Gri4ecuyFmmGeTUSPmkKW13oBhQ5KB6B7RLQYaHknNoRp2/7QbS7nX6T7+3fH0fbUvlc6DhGlM9Co+D/a90/n7kAlIlF6MSbOjfbafF7Je4NOZnza7v9uUbgy9aSgLrl3ADZrlPDV5OjnOxzvV1+OVYz5+tzwC+94EvDbWSmulR+qzvBzqm1RimfjwRO48eCeh0W2bnMT3j2fU3aOIyoryVQiSOW5pmvHXVOrz6uVXc8O6GwCY8q8pXPyNvzNPyvjTOoTjLyEBUKhOSpnPYzFmT77/xWPM68Ne9/lZyoY4sr4feCMynfocGP6KkDmTCS7DXoS+D3p+lD5KDm0FKJWeZ9ASukgd57x1DoOuGhRwf3GdMIKGuYQhPCoK+O1M+F3WhQ/I0Of4sGILDpcWV52YwJs2m/jkrE/Yv3h/Kyd3jKoCM3H5a9HXlVBZLwwncWHeqF2n08m+MjXO05ZB92u7pA8nM3G17/PdxRp66Bf6bN/68VbWvbouaNcZdvMw+szuQ3TdWKZtrOCTqcuC1raMF7XaKw9ZWZ8GQ5+HhEk+x7TmTKg31VO4prDFAuwpw1NouPxaduRsRqsAa8wYMPRsd3+dTif79u3D6QxejbOTCYXCWw/OFGh9HZoAiacLI0sTVl67kvU3rKdHtDdS/tOZn/LtnG8DXkcyNmuwcWj1IR+Dip/j79B3UPB1h+5Hxotw/CmIKXwE9gZWjUgxiNDbwtq2Fduw2q3Uu8SzM0gZf5segF+ndra7AZHHbxuo3Qf5n0K9t06V9A6uV4s5qM0GqaNSMaQYUKr9l3e5FbmU1pXS4PCuxGtqICl8L2maJWBvIW1Uxhe3G2xV0FAZcLcUGCOtD9qDPTmDwz3GkXXBYJQaJdkzs4nNFuNQkkyODYsVWROnLRb1Uo/gaGb8nRDjNyQa9FngdjR7SKw+EoBae3Wzx7jdQtpRYsx9Yzj79bPbHbn96NJHueT7c6kwLKPcmgaZl3do7iNzBCFxYOyNVqlFoxTWYVWoeK+ZzcG/nD5Oz7nvnEvekDwAMkPLiTJ9LMq3nCCcEOO3Cdf2+Dv98/+H0pzq2eZyuKgprKG+uu213vtd0o+H7Q/T6+xenm0Wu3hxum2hhGlqiDD/FHQ551Mat1uoKjUGqcWFi4Wm1WWiPb9+OefnkDkps9n9g64cxNmvifeyrd5BZsRWDIr8TnT8+OWYj9/TFsMIYfuW1ncV1goMBu+68/Bh7+ER6RFEpEWgULbtm5k+Np3p/5nOecvP4868LAb2+Cen668C6+FWz5U5ilRtFnbvw4ua1Pir9nH8hSeGt/rcpYw/dUMTqc/TFovSICcZx2LMyo6/IJM6MtXnZ6nGX3yYfxqYlPF3NBZmMgLpo2RTV6CyWTm8NJeK3RWtnNUykjxdiF0YYYSsmgvcLUuInsr0aLQjO0zixW5KM3HO2+eQPr5r6rSV5VaQsW0+kVV5no9K04w/gNqWtNJkWiQkOp01hTOpqov26nm73Vz0xUVct+a6lk9uB6c/dTrDbhyG065C64wmJSIRXE74/SLY83rrDci0GanOX0Uzr0fJ6HzYHHjyvffnvbw18i3yluQ1e40QYwiW+DhMkWVY3WAb9zX0ub9D/ZXHb8tIcp/N1vlzu0VN3FYYc98YRt01KuA+aU7jOFzO26PeZsNbGzz7/Bx/U1fCGZva0nWZFpAW1pujf4ZRgSPT25vxJzkXFC41ek3jBXrdBoOf7lxnW0Aev61QtABWXgamrZ5ND41/iKoHqpimEM/FZgNNqIY78+9k2E3D/JrYXSFq+GXHZnu2mc0wIeNTBpRP8anxKdMKtir4Klo4xAPgcfyZ2+/40+hDKOx9Ou4QHUmDk7j424vJmpIFHCH1GZokpEY1/plMRzPjD06A8ZtyNkxfBXFjmj0kzhAJQJ2r2iNb3RS3282/I//NV5d85dnW7bRuDLzcq4aw/B/L+f2p3/3OLdtZxk83/0ThGhF8UWcXH0uVU8+e6vEw5oOADlyZdtLrFpi6HMKS0WsbFUHqShky/0nKPm671HV7qXcIh9Oo2K3E750DNbu67FpdwXE7fp02WDAUNnrXBZf3vY6M8htxmhI92wpWFvBc6nNsen9Tu5pXKBQeQ7TdacfpFobYenMo5ZY0Doy2w2C5/EfQUChg22OQ9x4A/ZJ6EWeahsHar12O+en/nc5p/whc0/pIauvUzPm2jIKUkzdz85iO35hhYBBGvZgwb8Yf4Mn6K2wSc+h2u6k5VENVXvtK/Bw0HaTMkUfvyJ30Ur8PzrZJhcocJew1UPAt1O4NWOMPRIBG/rJ8Xur1Er898lvAZqQ5LpYmUp8yQeMkVTw+dsx4cYbPz1cPvpoRKSM8hpemSBGZmc534I+FQhtXJ+tEBpWCb0RR6KEvQWiC56NkU1Wiq6sg76nP2BUxhbH3j222if1L9rP5/c2MvX8s8f18n4/b7Sa3QhSCNzSIqDGjEZj8c9fcz8mAaSejon5hQdi5KArmYn35LnRqXZde0tA9gdxRl6NNiOa83pGkGlNJj/B1MsZVfIBiTzfofXuX9uVkRJs2mX+vmozTCR9t+Irbf72SMWljWHz54i65njSR0GgARw0UfAXaSODGLrneqUhMDOTnizp//DEbDL1g4D88+x+Z+Aj3j7mf3rG9A56fPDSZKf+e4vfObIqj3oG5vhwMoEDhmSzKBJ+ICCgoaMbxZy2B71Kg581iHtICQ68f2uw+aU4TnRHOlH9P8YnIPa/3eWRGZnrnQiqtLC0YBMIblQYP20ZAYNVBz/95WzP+KizC8ad1xKINa4zOTD2nU/2U6STJMyD0S4jo69kkOdF1jcqDUmRtoGw/wDNX7RXjzXCoqYG1prOZNjOGeH1m8Pt9sqKNhO7XQfzEgLs74/gLaXye9RYXq19cS8bEDBIHivZ8HH/OelCGBJSBlRx/cmBp24kzRgLgUFVTU+OtdSyhUCjIuSCHqO5Rzbax5cMtKNVKxj803mf74Q2HWf+/9WSdnkXKiBTqbMLxp3bp0euDehsyjbw440WUCiWKrcmsje2GJiIm6Nco2VrCymdWEhEZATHwZ/FEqi6dRFST97RMJ1BqhEHZ6U0bkdRImqpXRHaLZMTtI0gY4K+u1RxlO8uoLawlbWwamlANVofXkWCtCUNBowS2su3yoTJtYMrvnpIqF/SdxZdFs6irE3ORiCAtAb+/5nvMejO3pNyCNqQ7o9ggv2e7CqcNnBbQRnoz/iwVuN1uUlIU5Ob61vlz2pw8l/4cOefnMPur2a02/8e//6BgRQH1Y0Rwxdvb7yZu2GtM1x+lqCaZthE3Fi51gEJJUk0hUxMvY9+mRBoivYe4HC7en/Q+AKVbSgM2E6oOJV4fj7NMzHkTYmph55sQO6rFwC2ZtiE7/rqYzMhMMiMzA+6TouMj3Jvh4Bcw7JWj17FThZrdcPBL6PNXCE3wfJQalBXUh8cSffW5ZJ+T2mITpVtL2fLhloAR1G7cfH/J9+SW51K6qDsQvInLSUvFatJK7yAjMot1hWeJ+XwXv4mcGh21sVkkxkHv2MiAzoq4io9R2NJkx18HUCiEAdpkAqdNi8VuoaahBmuVFWuFlaisqDbLOjRHeW45P8/9mWE3DaNId4hDCV/y9b4x3JZ0JVziALc9SHcjA96Mv8pKQL0cHL6SnUOShrR4fnSP6BYDKgBWPLOCtFeWknBTAkndrCg3PQBpF0LsyM50XSYALWb8aYwiI+KIjIO//fo3Ptn6CfePvZ+bht3U6jUkx19clpG+o3yffY/oHj6SoZStEhkr4ZntuAuZI5Ey/sy1LiEDqY3xM1SlGIXU5+Haw7jcLpSt1EpQKBT00Y+joioajTyfOT4w9PBEVR+J5ChqGll7JC63iz0VewDIjhEZf263qPFXaR8MPQeD7IdvOwoljHyz2d2dcfxpG59DZW4ZS+f+TPa52Vzy3SUA9I3ri9VhFYFrv0wGayGcd9CvDeldfLQy/k4I9r4FDWU+5R+aEqWLBIQ8VSDHH8C575zr+Xe9qZ6XerzE0BuHerJPrvj1CnQR/oGM/S/rT7fTuhFiEIPVk/Hn0nNB76dgTQGMeK0zdycDohZt4Y+Q+RcuH3g5AEsrYe+Iy9D3Cf7lqvZXCfvAncPYb4qkomYAruQe0LWxrKcOCgWcs8dnk4mDlBv2EmJPwm7PQaOBiLQIznjxjHY1vf6N9ax+fjV3HryTiLQIrHbh+FOgAGcIRm05hrpNENZXzFVlgkP0YJ8fjUZhD21P0lrt4VoW3bOIrKlZDL56sN/+/Yv3o0xVUhNbQ6LCxNi0L4lU9AW64CVwqrN0BlRtggsrPeWU7C47dfY6UlJENGLTjD91iJrx/zeeuD5tS+Uq31nO/sX7MY8SKaEKl4FQgx46Z1KSCTZN1pUpxhSeGvoxj/0EDU3moGqdmnPfPZeY7Jhmn/8rZ73CC9Nf4fzzwQ0kGIpg+T3Q9yHZ8RcEZKnPIHN4Q9s1h6WFmanHC3CJXRSLlwku2XPhEhtECyN1z+ieTO8+nR76oTg1OhRDBhHbO7bFJkbdOYqH6h4iebh/1qZSoWRS5iRuHHYjllqxoIvXH4Rdz4FpR/Dv52QgaTqcvpRy92gA9u8HW52NF7Je4Oc7uyZT0lzjAre72ehnhUKBefzvuEd/2CXXP+mxHOLaQXMZnLgQlUN4GGoball8/2Je6vkS9aa2111oDvNhMwf/OIil3EKlbgMH495gRdGvYqdSBSp5pR1MJKNXZSUw6zBM+ino14jvG09Fz0Tq9HUMDA2Dnc+CaXu721EoFKSlpbW7zs6pRIuOP3UoTPgOetzgs9lUbyKvOs8jZw2w9rW1/G/g/6gp9G3I4fA6HlrNMnE5YPEY2PTX9t2EjB9Sxl+O7SH4JgHq/KV1E/QJKFDgdDs9dadbYkDCAJ7I+p3h+74XWdUuO/zQAzY9FOTeC+Tx2w7cbs8/q+uruWXeLXxsvwg37hYdfwdNB2lwNqBRasiIzADEeLU3xstIDmSZ4CA5/sot5did7QtKkhy5IekJzPpoFkNv8GZZv3TmS6y6dhVj08eKuqyp5wVs42hKfZ4w4zf/Y9jxb58x1JQL+lzAJeW7GJj/TvOS2E1wWB3E9YkjNMb7n2xMMaIN9/egKxQKDEkGzz4p40/l0jMgdoGoeSvTeao2wdZHoCbXs0n6RnZFjb/sc7J5qO4h/vnoc4zYswB9Q48TLrPohBm/jby7/WX+zD6dgti32jROm6Pfxf0467WzCIsVE1apvp9OrUOBgt4Ja9H+MVXIbMsED2c91HvnoQaDCKJvj+NPG65l22fbmrW53lVwF30+EE6+ZKWav46bTZTpk051+3jlmI/f5LMg41IAwjRhhKhCMIYYMdWbAkp9Apz2xGn0v7Rt0tbnvXceD1kewtIYfJwRVkSCeq1cTul4w+WEkt9ErT+8AWz1R5j/Bl01iLTRaQEDpCTKy8U0TasFQ0IaTF0B3a/tqp4fM47FmJUz/oKMudB3Zvnm+jfRqXWc3etsokJ95UF8iq8r5UfRJah9V72Tu01mcrfJ/PgjvIF4Bm63u9XBpwnTtHopSXYiXrMRNtwNo96HCDm6yI/QJAhNIjEDcsvKuOO3e4nZXse0qGmERIR0ySV3f7iaofMWY7/8an7avRm3283Y9LGeDFClUkl0Uq9WWpFpFpuJyckvcrgiigabkISraaihx4wehEaHNis91h4yJ2XyYO2DNFjdONd+D4AhRC+yXGp2gLEP6Fp24su0HR/HX4D3Y7G5mK93fI0bN7eNuM1vf0NNAx9O/ZAeZ/Rg0qOTAl4j5/wcti/eh9lgZis94ZxVoG1eRqs5lEolMTFy4ExLtFrjLwBxehGR19RZ5Kh3YLfYsdf5GrMtTRJCN/9vFTu/2s7sr2djTBEX/mL7FygVSiZnTiYmxABD/gvhgTOYZNqOZNTMN49l4PCbQOlvdNaoNPxw6Q/EhcURpWvb+JIcQmo14KgDjQEUXSN5JY/fNuCog++7CcnVkW8BoFVpeW2dyBJKVJloaKqpcwRSfb8e0T1QN643JEPbk6edhu5PHUye33X9PxnZ+hjU7oExH/ntig6NRq1U43A5KK0r9WTdtgXJYGKzwai/DGj+wCbS20fis77sYk6Y8TviDfEea4bo0Ggy9NGYnc1/J/cv2c+mdzYx6bFJRPeI5qplV/nsr6+up2p/FTHZMWj13nfxodWHMCQbiEgTKdRNa/x9VLKch68+TmusnWikngMxW0CfyYbDGzhUc4gwBpKyay9UGoHhQb2cQqFAE6bB1FiW6MpBDxLy42tw1jYIa1lN6HjhuB+/Bd8IZ1HmZQDEhInFiU1dickkyhIAfHfVd4TGhDL9P9Pb1GzqqFRSR3mfkST1GaIUdqMqZx8Y/hrEylkmQeX3C6DkV7jYyq7yXbxiHIl7oI65NSVtbiLEEMKDNQ8GDLKQaHALediyBj3//uNzbp8UuCzFic4xH78593j+qVAoMD9k9swxbY3TnqZSnx3B5rThanT0XdP3P2Tnfw+jnZ1rVCbIuGDJaZBxCYz9FKXGjk1twtoQAbRuQ29KVWP5x6goUGjCTtpMP6Xy6OffyRl/QSbjtAyfn+9edDdXfHcFZZYyn+1Op9cLbqxfBhXrjlYXTy2cDVC2Akw7fTZLi2HLB1/xQuYLLTZxeMPhZqOK5u+Zz7sb32Vf5T7PQlGZOFFomCe1bfJ5SuJ20at7Awq3kqVVH/B17tdcveZqJj82uUsup4iKojoxG11sOHN/nss5n53DrnJv8XVn7UH2bVqI095CyLxM8xh78Xx+Ad/svB8ahKG/pqGGPhf0Yeq/p3rkjTqLQqGg3qbEqRRGE2OoHsp+h18mwuGFQbmGjMBH6rNqExz63mf/4drD3LbgNv6xPLDxUR2qpraoFpu55TFlrJjMxG3beOa094Xso7b92oJOp5Ndu3bhdMoLgeZo1fG37UnY8qjPprgw4fhrOn8Zfddobt9zOzG9fBeaknS5TgcOi42aQzVoQr2T/XsW3cNFX17E/qr9orZf77sgdWZnbkkGb6bW5oqZQipOnx7wuLN7nc3I1JGEqNv2LpYcfxoNop7ZGRth4BOd73AA5PHbBlRhENkP9N08m8I0YaLWG2DVHvDU+AuEVqXltG6nMT7dW3tMehfY3BEoNMYu6fZJTcUaKArsLFUqlHw9+2uWX7XcU1u8rUgZfy09z9Y4mhl/J8z4NfaE0MSAgUyeQ1r5TpoOmNj6yVYq9lQE3L/hrQ28MfQNSrb4GrE/OfMTvvnLN4CQ3ZWyi9QuPeHhCiG3LdN5QmKEZLnGwCNLH+Hcz85lQ/Vi4g6sQ7t3Z+vnt5Pq/GoK1xZiqnAAUOtMRxE9TLyvTxCO+/G77UnY/DfPj1LArl1V6VPnr2BlAYfXtV1160iiQ6P5v/H/x6Xd5wJQr8qAnjdBxMnpMDpmJJ8l6uO6XYRpwmigBoeqmurq9jXTnNPPbrGzZ/4eTPvFL0edw8CKQ7MJTWohiOYE5ngbv+omiSxJjQq5tbW+Uq65P+Ty5og3KVrXukdw78K97P19r+fnX/OupLbHsz7SkjLHAUoNDH8Vul8PwGk/prNoUBxltF39rt5RT79X+zHnl8k4FVYxH3PahOrMScixGLNymlmQaRrhZ7FbMNtEBmCC3rfgcNPo+LAtV4E2Gs5YfzS6eGphq4bF46DnLTDcW0NRE+LAjQq7PpL4TBsupwulKvBH5Oe5P1Oxu4J7S+712/f6+tf5IfcHXjrjZWpqbgUgPDoSosd1xd2cHFgOwXfpTIi6k/edz6JwK3ErXJRbykkydI2OvnZgb/YN601Gmig6DHi0yAEUe1+h+86ncaZvhuiTc3LYpSg1EJZKgxNc1kapT1ttm+pJtZXcH3MJMYSg7pHpcfyFa/VikT/0BYgJbiTvqY4UQFhVhVh4F3wlaik21g9LNggNj9K6Uhwuh89kH0ClUXFXwV0tXmP5U7+TtNaMo98ZDIsrg7oDEJrSoQz4+iP1JGR8kAyaTQ0lPhR8BfYaGPCoZ1O8Ph7AL3ApEJKhOSwMJv59IhP/PtGzz+12U2IWhtCE8IRAp8t0kK6QMbt/8f28mf8RyfH3o9HcGbyGW0Aev62gUMDpv/ptzojIoNxSjkV7AJttYLOnT8qcxKTMST7bJEPMO3u/ZdjdwezsKcKE71v8Vp2TfU6Hmm2a8deUvKo8+r3WjzRjGrtu3gxrbhRyn90u92uj6fv4aHBCjF+XEywHQRkCYf6lG6rrq/ld/V92p1qoqXk2YBP9LulHv0v6oQnTcHDFQfJ/y2fQVYMwpooPbMbEDCb/YzKGJG9modvlZvITkwmNFl5Yt9vNu+e+y5Lf66h26cmOWAJ1PZsN2pBpBy4H2E2g0mHQimfgUJrZPuEm3GotbneLft92s/bVtax8ZiUvzX2J6sH1xJUVcfXpNwfvAkeJ43r8Dn/FR55XUi2wq6s8mSEAt+Xe1i7ptC8v+pKawhquXSkk5BLDE/nHaf9gxQr4F7L8dZfR6xbPPyNCRKCnS2mjpKKe9hTHNBebObD8AOnj0jEkex+W6aCJT876hIjrIiAVVC4den1wx/3xxjEdvwe/hIJvYejzoIv32aXTQWyskG4sKoJsUV4ap91JbWEtlgqLf3tH8MM1PxCWGsbAqwexL9/GusJZqPp1wX3IdJ6e3m+fURtBmbWYOmdzRgd/yurK2F62HbVCw3S3Ttgt8j6ANdfDpJ8hWU6o6Syyu7wLkeSxJL3jpkiOP60WFIP+Bf3+72h379QgJBYGPytSj4EGRwMR/4pg4hINDlUNNSOmcNlPlzXr9AMYOXckk58InIkmySelhWZ75qVGnUksPmQCo42CtPMxpPZHgRKNQ3gYtny7haWPLu2SS0rjTRdmx9QgPkJNo7Dd8ZMpjZ4jS891FLebFOMe4sIOehx/AFu+28JHMz6ieHNxpy+x8M6FLLhjAXV14Gh0/Ok1ejD0gOw7wChLtQaTphl/7p63wpiPEaWWBXH6OFQKFW68Tp32suv73UQdFlHY4Yf+A99nQl1+5zouE5BWM/4mL4IzNvlskqQ+y+q8jr+60jo2vL2B4k2+Y1rK+AtkaK6qr8LeGLGXoE+Ayg2wYAgc+KLd9yHji+T4c9dXwIrLYO+bAY/bcHgDz//5PAv3tp4ZXVRbRLXzMG6FU2T8mXbBrhfAvD+IPZcJBlK9PmvIgRZr/AVCcvwZ5WSjjtFFJRqac/yVW8qx2C0iW8xWDXnvQ/mfAdsoafwkR0Z2SRdPTGpz4Ycs2P1ywN12p52frU+wP/E/VJsCR2JrwjRowjS43W72L97Pbw//5mO8TBmewoT/m0BkZqRnm0KpYPgtw+l3ibBWqpQqrhp0FaOUtxKrq+JM3RRRF16m85T/CV/Hwt43CdeKj6NdYcYeGoFDE+pxiAeLXmf3Yuw/xlIbVotTVUeEXq41HnRiR0HcaM+Pnow/daVQJGmkvfWStOHagGo00ndxYvLr8GM2VG9rf59l2oQ0RgGKKtruIADIX5rPVxd/Rf7SfJ/t+ng957x9Durx4vs8KbyW16fHw4HPO91fmQBUb4MDn0K9WCc+t+o5zvj4DL7ZKTLcA9X563NBH+4uvJse01u3uZ3x0hmc/sjpLLloI5O2b0elOjpKBjKdI0InnPoWl6m5ssp+SEHGBlUsChRibRKeKWz4cmBUUJAdf12I5PiL18f7TUgkI5leD2RcDGnnH+XenSIoVUJ/Ol7IG4WoQ7A7hQHSpqrwybxsjj4X9mHoDUP9tjtcDvZV7gMgUSOcDno9qFdfCl9FNltA/pRHrYfxXxHW/2piYkDrEMbl/d/tZ9ljy1qVBuwI5QvXk7ZtAU6NKMSgQOFb6yhxKkWJ94Cqa2oMngpcqMvmuiF3Yq0NYWLGRM7seSZ1FXUc/OMg5uLOp6Oc/frZTPnXFMxmcKqkjL/wVs6S6SiS48/hALN+kqiv0cTQqVQoSQxPBISjIBC7f9rNlo+3NHuNc769miXnprMv9XFyVbHQ+26/iEGZ4NCq408X51f/KJDUp6nAxI/X/cieBXt8jpW+pXo9bP5gM3vme/cXm4WTMFIXKaQmnVZhuD5J5TuOJtJzragOEYvvysDKEQv2LOCuhXfx+fbWjR/lFvGd1DpihBOi4k/YcCeY2i7ZItMFHPwK1s2FxlonIDL+oHWpzzpbnd+22lqIDi1kRuo/hDNepn1YCqFoATRUBtxdbinnf+v+x1O/P9WuZpuT+pTGZWxYrPhOzjbDoH/5ne92Q0GB+HdGht/uU5fw7pB9J8RPDLhbMlQBlJiaL4brdrtZfN9iClcXcv2664nN7lhtabMZrHYDW3WvyjaAYBGWAj1ugMh+nvWB1Wkm1FFLSF1lUDPjATImZNBvbj9sITZwKxif/Als+XtwLyIjvnmNNhXJ8WdTVfpk/FXtr2LHVztoqGmbRvK5757LnIVzPD9XWivJLc/lcLVQBdKGKIWaTYC6yTKdoOhn+P1CMO1CpVQRphLrjuLq9jn+MiZkMOvDWWRM9P3IhUaHMviawSQPT2ZA1Gio705pfW/QRAbrDmSa0vchuLgBIvsCsK10Gz/v/dlTTiehUeSltLS5Blom5/wcep7Z0+OQf2DcJSh+n9XZXst0BSsuhYWjAIhsnE/ZFCbaqmgpBRmHK4TtwWgEEqfA2E8hIifo3T0VkR1/QaZpocaWpK2OZuF1GV+kTC+7uhJ7aRW/PPgLB/842O528qvzsbvshKpDCXOIAtFGI5AwCTIvP7l1BYJEjx6gdYhFc/j14dyy4xbUocGPorZs2k3cgXW4QsUqISo0ClWjZCGIcZuVlXVMCq2eFCgU7NM9zIqCi6ipUbD0qqXMu2weY64dw0Pmh9oU1dUaWVOy6HlmT+H4U4rVu16rh61PwPyBUF/e6WvIeNFovFI3FYHL2XjkPptz/P3+1O8svm9xs9ewNig5FP8DOxMf4SezG4b8p0O1buTx2zoRjTbNmppmYlKsJcL43yRbPV4fT0ZEBj2je+J0iZl7TK8YLpt/Gf0v6+9zujSnCQ2FhXcvZMXTKzz7pLmQ5Cgmbiycux+6/SU4N3cKk9j4X1pWFU7tmVYY8b+Ax6UaxRyloKag1Ta9jr9YkVGYfDZMWQ6xo1s+sYPI47eNFP8Cu1+EBu+3rqnjr7mMP6vdiuGfBlL+m4Kp3mtcq62F9IgdTIp+GCpWd2nXT0oOfQdLzxQ1cANQaa3k5nk388TyJ3C0QwWkpYw/aHT8KRQiiE7jr0dXViZqyKvV3ho7XckJM35VITD0uWblorQqLTqlWJSX1VY324zb6aZqfxU2s43Y7FjUOu+axW618+6Ed1nyf0s821Y8s4LXB79Odb5o01Rv4qfdP7HDvII6eySVMTd7glNlOkl4NxjxOiRO8Tj+zDYz3dd9Qfaq9zxB18Gk3iFk9pQuHYMiv4Cd/wn+RbqQ4378rrkJPlWBQ1j+o0K9Up9NM/62f7GdLy/6kvLcjq0Fv9j+Bb1f6c3LB0WNqnz19XDWNllNJthYCqDga7AeAsDYKPdZVtM+x58h2cCAOQMwpgReM56fcz4vDVqJdeeHvFuw/KSVCTzm41cVIuq2NyI55iutYnDGN8byHun42/X9LrZ+srXNl5Ecf+G6WlGWQub4QxUm5qVAVKgY13ZVdZvrVUtz3FC3cPyd7HLLx2LMyjX+gkzTzL6SukbHn97f8SdNPjOi98B3UyDnXsi+/aj08ZRj3Vwomgcz94BCQXRoNIdqDmFTV2CvSWbFv1YQYgghfZx/GrGl3MLbo99m8HWDGfeAb90+SeazZ0xPamvE4I2IAPrc3+W3dMKz9QlwWklOfgptuXD81cTUEJcT1yWXc553AVuTGxir3gT41vfDYUGxaAzGbnPEOJTpEBUpj7H8APRsEhzZXumV5nA5XChUChQKBXV1MHj/ZwwcYeK83tGQ+19wWEAlS+wEm5gYMdl2538GG++BMZ9AgjdavjXH37Rnp+FyugLuAzi4ooDQujqI9S4WOoJCocAo69W1iPTfY7eLTBLdkcNl2xOw5xWYVQShwlocExZD/p35PoeFGELoeUZPv/abZvxd8t0lKDXeCa2U8RdoLiTTOXQ6iIsTxv5Dh3XkRAY+Li0iDYACU+uOvwprY8S7I0YsvHSxoOs6w7Q8fttIv79Dv4eFhH0jktRng6asWcffnso9uHFjsVt8yg7U1sLOsjH8YFvPOan+Nc9kWiHhNBjxJhizA+7uEd2DcG04ZpuZ3PJc+sb3bVOzUsZfi46/hgohix3eA7QRPscdbIxjTE4Wzr+u5mQavwZNJPUNFirqqoBuAY9RqpVc8OkFuJ1uNGEan31qnZrqvGri+3mVC1wOF5YKC6ExQp8styKXmZ/OxKBIZyIHPHLNMsGlqeNPmzOQ2rL6oGf8fXv5t5QWlcIEULlDWW57h6wLa4N7kS7muB+/UYMh/SJPpnu8Pp4bsp5i1a/RVMS7kHIYes3sRURGBFHdolpoTFC8uZhd3+5iwJwBRPcQa4+aBuFMUDvF/8XJbnQ+ZmRdDVnXeOrFR+oiKLYcorKuJig1OLd+spXfHv6N8z44D7NZ2PVO5nfsMR+/tiow7RRlV3TxnuQKaR0hZfyVHFERZNmjy7BWWf2CSJtSU1jDm8PeJH5OPA/FPY4zaxAfHpzH07d1yZ3IdJZRb3v+GdHo+HOoTTQ0NCoctoKkLhTibJLxl/ui+P0a9rLnnXGyECwbaXs4TsN7TlycTfJZm0p9HonHSBbqEJItajn1r8tQhYA2GpwiKk9y+tjVldTp47l+y62MumtUwFPtFjtqnTrg4MwtzwWgV0wvj3xaRITfYTKBKPwB8j/CaAStXbzgqyxV1JXVUW8KfpFiq1OLXWegQdXEcCJhq8Rtr6WkcK/P+JVpH9GNfpum0iuOBge75+3m8IbDnWp77Wtr+ZfxXxxafQizGbTOKFLDM4URc8CjcM4e0JzEM/tjhCT3abIYISwdFL6TLimDSwpyOZK0MWlkjA+sNeZyulh6+TuMXCG+j5PLv4Q/r+5QP51OJ1u3bpXHbwuEhHgzSQLKfabMhIFPgqptxRNcDl+HbtMaf+nj0kkdmerZJ/1+eDL+KjfCvneE8Vqm06Q3xixV7N8JhfMCHpNmbHT81RTgbkWGXHIwaKSMP4cFXF03tuTx20bCkoWUncK7dJvefTrzJ1QxJndps44/KUgtOybbZy5bWwsNTj228CEQmtilXT8piciBHteJZxIApULJ4MTBgKix2Vak93SLUp/Fv8DPw6B4kd/5kuMv3T+WsUs4ocbv1sdh4ehmSzFE6CIBqLJWt9iMOkTt5/QDYci5q+Auznr1LM+28Q+O566Dd3nqiUmyuyqXnhk9/ke/A/2gensHbkbGD3utkBvb87qP4085YhjFPcYFPePPVmfDVidevEqXDpU+EYz+gVHHM8f9+O15I4z7ArSRAIRpwrh98INklN9IdZX3WxjfN57+l/YnLLZ1e1rer3kse2wZpgJvlpnk+FM5hBMlK2Qe5L7ko4IhEwSUah8D/tj0McSZpqFwhDdfiqAZXu79Mh/N+OiI5pWEGEPQhGqoq4PsmFVMjn0M6loPejsROebjt2QZLB4Lh4W6z5EZf3GN8fxlZb6nzXhxBhd9cVGLTbudbqK6R2ELtXHIuher9pDskD9BiAyRMv5Mbc74k6Q+NbYmjr/CeZD/4Unn9AOOyZiVHX9dyJwBc5h/2XxuHX6r3z5p8mnR5MCMtdD92qPcu1OIwU/DjDWgFgZNr9RnBW6VmtDUWDSh/gs4gIj0CG7eejNj7x/rt6+pMUWarCRElMAfs6Hgmy64kZOISfNh5h4iIqDPoWf4G/VcrrqcZ+OfZdO7m4J+ufqCMrSWagYlDOPDWR/y13F/9e4MS8V19m4Ox94U9OueSqQV38J9Yy6hqgou/OJC9E/p+Xzd53x69qesfXVtp9oOTwgnZWQKxhSjJ2L3ZI7gO16QnLl76s6E6asg3jfree7Iuay4ZgU3D7u52TbcLndAR4Pb5ab7rdPZ1k/UgotuKOxUnanj1mhxnKBQeLP+TIEUdZKni1oNjcaVlng+43neP+19n20eqU+dyy/L89zsc/nqoq+4bURjmGbhD7D6WrB2LiBARpDa6GNNqngMlp0NTv9VliT1abFbqKqv8tsv0eBowGwTL1mtI1b8zqy6Ej7XdrnzT6YVXA4w7weLN8M6VBNKbHgk4O8okthSIuqs5sT51sioqQGDtoLI8C7Qv5MB6JDjTyoBcaSTwsfxF9kfBv4TIgf6nS85/o5mfb8TZvzWF4O1EByBU7+iGh1/1fXVXdaFOrt4sAqHHgVuVAq7HPwbNNxw4DOoXMuEjAm8euar3DTsJs96IdgZfxd/czHDvhwGgMqlI1ZfCPUdLGZ1DDlhxm8j0tqksrJZH36LDLtpGDesv4GUEd6gDUkGW2EXE+VM93uw/g6fQBuZIOCwQOlyqN0LwFvnvsH0soVE1Y1stqxEcyQMSCC2t2+N1b6z+3Ljxht5ueplbshNIaHbk4zSPwrWwMo0JwPHdPxGDYAh/4VoMdc50vEnZfyVlYGrydIwY3yGz/gLRER6BNf8cQ1hF4nvo8oVxsi4d+Gwf8CTzHFA2UrY8TTYqhmeMpzMmsuIsAxqs+NPq9ISr49H3SB+aQwGYNI8OCe/y7p8qiF/zbqQ9Ih0zuh5BkOTh/rtk2v8HTuideKj5A4VM4zKQgtlO8taOiUgj0x6hMWXL2bOgDkeQ2qi8RAc/BJq9wStvyclujhQhWA0gtplwFITQlRWFENvHEpc3+DLfRq++5CsDV/TPTaNOQPmcE72OUG/xqlOiG0PacadOJ1gbXBgsVuo1dRyzjvnMPQG/3dge+g7uy9X/HIFxlTh+NuZ8iDf1t3L4drDsP99OPBFkO5CpilNF9eByInLYUzamIB1bAGWPrqUx9WPU7nXvwGVRkX0maPY1VtEuucP/xDO3ByUfssERnL8tSeq9tZ5t5L1QhZf7/jasy1rWhapo1N9jpPmNMqCAp7QPMHa17zO/m5R3bigzwWMS290HHe7HCbOA31mR25D5gikzJ4/Dl8Ho94PeEyoJtST6X6o5lCzbVnsFsalj8NY3xeNM0IYTOPGQuackzLi8oTCnAc/dIfcF3w2N5chJrG6UNTvG5E8wrc5M9wy/GZOrzSAyx707p70WIvhh56w5ZFmDxmSNASAjcUb29ys5KQ40vGXGZnJ6NTR9IjuARF9oO9fA9afkhx/aWltvuSpw/BX4byDAWsjAkTrIwGwuttel+ZIDq0+xJpX1uByuKg31bPsiWUcWu1950oZfwqHngV7b8Zyeq6oTSfTedQGuMQOI9+iX3w/bh5+M1OypqBdv4peq96ntjL47zm9Vk+WewpRdWMYb58CSyYH/RqnNGUrYO0tYNrl2VTq2km54VdqnKVYrWKbrc7Gc2nP8eMNP7bapCZUQ9KQJLR6b22KGpuYGCsaxES5NOkpmLJMdvwFG0sB/DIR9r/r2SStNdvr+Lvoi4uY8fyMgPvKLeVUO4v4sqQ/CxSbIbJfR3ss0xLhWdD7LjEnwauqVmERDzM6GlQqcDj8bQlOmxNHfesZtRa7WGCqXGGcZrhFZOLKHH8cXgibHgBLIZf0u4TTqj8mpfKyNs+lHpn0CCX3ltDz8P8BjTYLpVqUm5AJCnKNv2OEtKDrb/wMdhVDr1tBGTjrTKaT1B2Ag19B4ukQNYhBiYOY0WMG5jKx0Fp07RfU7CnhgcoH/E4tWl/EoVWH6HNhH8ITfVOMEsMTPdJlPzU6/hzGoXCuTZaGaI2GSqjdTaS+D2CkpgaMKUbO/t/ZXXK5il6jsbh0gR3tZatQmHahdJ5Y8izHG4rTF/Nw4zxei1g41dhrGHz14KBep64ODsS9xr5KEzUN15O09VEh5ZsxO6jXkfFKfdZVVcP2/0H0EEia1ubzY3vHkjMrB5UmsMPAYgG7ulpcS9d6XQ6ZziE5/qqrA+wsXQ6b/gr9H/V5xuXWcvKq8yisLfRsO+dN/8AJj9RntJac83OI6Rnjd4yH8CzxRyYoSAb+pbumcGUL/61pxjTKLeUUmAoYkDAg4DFRoVH8Oud3zj9f/GwwAEl3BrW/Mh1Enwa974GEST6b39v3T9b0WIHC9hAwxmefy+1i9SHh+BuZOtJnX00NbLafRs6gWGLk9Uf7UYeLTK0WsrUGJ4n5z8bijbjcLpRtMCJLtVCOzE56cPyDPDj+wRbPdbuhoFHR7GhJfZ5MvHTmC9xy239RW1KoqfHKlLWHLR9uYe0ra8k5P4fyneUs/ftSNKEaj/y1lPGndooH3ZbaNzJtRKEAhb9pS11nIqympNHxF5x3ndvtZu2ra4nvG88s82Jy86DyzH8Rni47ioJKzS7Y85qQo4/oDcAdi2/kz+zfGbLvM6qqLiYsDDRhGiK7RWJIaVkL0O1yU7y5mLicONQ67++KJPXpsoqJsia6J8TLdoGgE5oMQ1+EGG8gUkwM7N3voLKy82bpXd/vomp/FdY04RE226JxGgbIFu+jhJTxZ28MJlOpIDZW1PgrLRX/Bti/ZD8fTfuIs/53FkOvDxwcXrK1hNwfcqnp1SjD6wrjD9VPTBoQ2eX3IdMBul8DyWd5Apmaq1fdEm63NzjZaEQEfoSlgV6e0AYDeXYSZJRK73/p2xve5v1N73uiHpoiRcfnaN+CLQ8HnKjKBAlzPmy8Vxg2gZuH38yCvyxggPsKAFLPGsjou0fjdvnrRexbuI8Fty+gtqjlYt0+Nf6UGo+sqEwz5H8Mi0YTrdiAOWQ3i8Ov4JZ5t3TJpdxuOJQ2mor0wWwzreDH3B99Mx7yP0a55hp6dkvyGb8y7Sem0davdQltb2kh1RlcDhffXfUd278QmWFmMziVwnCi1+ph/Fcw4n+dvo6MP9LzrKuxwuYHodA3krbCUsFLq1/i2ZXPBjy/3yX9mP31bCIzI/32Fa0vIvfu18jZ0R0lEFu+FKq2dKifSqWS7Oxsefy2QqA6nB7cLiEj2OAbkhkXJiyfUs3i5pCirmP7JTH7q9lkTfF6oD7f9jlfbv/SI/3SIW0mmWaRHH/l5d7nEIiXz3yZ1detZkLGhBbbq22c7igUR8coLY/fNqLSwZBnIfkMn82bKv+gNHIeZYptfqfsrtiNqcFEqDqU/vH9ffbV1sLPe2+itverXdrtkxZNuMhS7+MfNCiRE5tDiCqEmoYaDlQfaFOzzWX8+bD1cVg0Bmy+us1lZVBfD2o1JCW16XKd5oQav9YSyPvIJ3uoKT1jepASko3a1f56UxLDbhrGFUuuQBepI318Ojesv4F+l3qzTbzZC3rO6v0OqoLPOnYhmcCU/wmV6zHbzPyW9xtL9i8h6eoZbJp+P/XK4Mks2S12Fty2gE3vbfI46WtS/wp97g/aNY4Gx/34zbgEZhVD4hTPphSjkAis1xZ6sogUCgVXL7+aSY9MarG56gPVvDHkDRY/sNhnu7RedVrE+tUQZhXzYpngojFA9u0QKwKRPtj8Af8iko1ZlzarLtMc2z7f5vcct3ywhUX3xkUlUwABAABJREFULKLeWQ9AhNJNRGjFSfssj/n4tZbAgiGw498A9E/oj+1vNvbc7lU+i48Xf5c2WUZGZUXRe1ZvjKnGZpsuWlvEb3/7DUu++GaqXXosxtMhunMqUjJdhD4DYkd4guE0Onu7avyBmPdKJgJDmBUWj4NNLQe8nagcizF7nH7lTw4e+OUBrvr+Kopq/XWlpQXdptAP4fRfhYVFpmuIHgJTV0LGZT6bpeyvuGmDmfC3CSiU/s9g0FWDuOLXK4jp5Zu98Fveb9w671bWFK4BvDWT4kO2C41jZzvCG05F4ifAoKcJjc3AqazjQOSHfLfrOxbfv5jvr/0+qJeqr/d+RF7d/DTnfHYO8/fM9x7Q+07c479Fa5B1kTpF5QbGp3+KSmFH5RALJ1O9ia8u/oqXenZclsFUYGLz+5s9Ukkmsw23UmTU6jV6MQGMGd75/sv4IWX8HSyNg+lroe//+eyvrq/mjp/v4JGljwSs49cSjnoHLqudPgef4cmUeejXXAV7XulwX7VabesHneK0KN2aMAnOL4bMS3w2S44/qeg2wPYvtvPjDT9it3plszwZfwHsancvupvZX80mrypPbFh6JnwdfEnnU5XwcDFW043bUc/rBrueD3jcmLQxjEgZgSGk5Yj4pnVUFQpg1VWwu2udQ/L47ThpBlHMzaTwdyxplBpuGXYLcwbMQaPyZrq43V4Hr7F5u4tMJ9GoNPx+9e9U3l9Jt6i2yTk2l/Hng71GqJmofIMMJZnPlBTh/DtanDDjt3Y3rLocDv/c7CEt1sJtA/H94ul2Wjc0oRpUGhVJQ5IwpngHmST1qXKFcWHvx2FX4MApmQ6y/DxYP5f9Vfs57YPT+Ms3f+mSGn8qrYorf7uS0feMPuFrjx/X41eth9AEH1Ws5PBkAOo1hYED2VpqLkTNhL9PIHtmts/2i/pcxG3D7yCkRtTCjd8wFOb3D9SETBAJ04RhdZuo1xa12/G3+4fdrHx6JY4Gr8rW1Gemcs2Ka7C6RBTc/T1/ZHxJLNhbDuA/kTmm41epAbvJU1tcqVD6zDWhGcdftyhmfzWbnmc0n1Wbc34ON268kYbejW27wuT56vGM2w0OKzgbWJq/lBcjtKzoPbpNjj+X20W/V/sx7ZPJ2FXV6HSgUQNDnofMy1o7XaaNyI6/IONqrFxqd9qpsIpMv0D1j6SMP7UhSTZadzUaA8SN9tMI1oWJYrjSswiEIdlAt8nd0Ib7flTf3fQur657lQ83fwh4M/7S6v4Li8eCs4VGZSBqIPS5D318N0KcwgBcbinn0OpD5P+WH9RLlR+oo/eKt4k/sJaqBjEmJQ1yAAw9cCXPZOv2HZ7xK9MB9r7OhUmXEaapQdnQ6PhrMGFIMRDVPardjiGJqG5RPFD9AOMfHA9AtdUbhm3Q6qExqk8m+EgZfxWVatzRwyA00We/9G2z2C2Ybf4WlYo9FSy4YwH5y/L99qWPTSf0gTuwxp9Bn5jJMPYzyLqmQ/10uVxs3bpVHr+t0JE6GnH6Rsefxev4K1hVwIY3N2Ap937npO9o5W+bWXDHAhpqxUzf5XZRYi4BmsyFYkZA4tQO3oVMINLSwGyPot4V2aL0YGu8s/Edhn6axLa0O4TMp7Me8t6H8pVB6+uRyOO3HWz/J/w8AlxOz6bMyEwAzCp/x1/36O68ctYrvDHzDZ/tFguEqk38fcLZGMsD14WUaQP5n7Zab2Z4ynCiQtsuZd0040+aNrndbiL+FUH6c+ki+3rIszCrEFS+a5NjUd/vhBq/kf1g/DeQel7A3euK1rEx8u8civ6wwxl/IJ6X0+bk8MbDOG1On31Tu0/l3r4vklJ5Ge/t+wZGvNnxC8n4M/Ap6H034VoxkMw2MypTJZGHd1BbGry1uUqjInNSJgtdi/g8K5KN3eaQtOc82P5U0K5xNDjux6/DCqYdUO+dg3oy/jS+zqI98/ew8J6FLdYNMyQbmPzYZB9VCoCbht3EP8a+QIR1EADKlDOFvKhM8FlyOvx+AQApBulZFra7xt+Up6cwN2+uTzmJqKwo0kanUe8QtoH1Ff2ojLmxU/Pi45ljPn5DouGcfdD/780ektC49CspaV/TukgdiYMSiYiJwOjsTroijLEH1bDh3k50WKbLKF4MX4RB3gcYtCLAtK0Zf+WWcraXbefPw8tQOcOFg1cdCr3nQspZXdvvY8SxGLOy46+LKKkTbzeVQuXrZGikrg5C1bVEaQ+ctOnnxxVOGzQ6fdYWrsX4TyP/U4morvJtxbw38T12fL3D7zSb2eYnAWqxW/h217cAXNb/Mtxub2SoK/1yGPwf0ER04c2cPCgUEBMmxofdZee8eecxd//coF6jpsKO1lJNGBbKLeUAxIY1cQI7WtBGk2k73a9nuetL6h163PVeqc/p/53OnJ/noOhEVrMuQkdYrJi0V1mrAQjXGFDXF8PnobDxvk53X8afyEjxt90O5uo6qC/32R+uDRdZl3i/eU2xlFtY89Iaitb5Z72D11kUGh4KGRd7pF9kugbJkdtsVG3Bd+JPE+L1IlSzqeNv4t8ncn/F/T5ZDNKzrFi7nzUvrUGpFtPLSmslTrfTpy0GPAZjP+nUvcj4kpYGldZkvrZshB43BDzmoOkgz//5PK+tfa3ZdorNxZTXF+NU1gnHn0oHFzfA8ObPkTmK1JdCfQnYqz2bukWJjL869YE2q+jW1EBESCmDEhejtu5p/QSZwOx5DbY9HtQmJcefw4HHYGJqMFHTUENBTQHGkOZD3iXHX0ZGULt08qCNgrRZEJ4ZcPem4k38oXqCw9FfdtjxV7qtlH+E/IMFcxfwxpA3mH/7fJ/9Q5KGcFbs7cTVTKNaOQSig1sH+5Sn+zWQdr7H8Vdnr8O8MZce67+k/mBZKye3HbfLjdvlpqquFofKhFNhRVu5GKo2B+0aMkD1ZpjXF/I+8GzyOIuaSH0C5P2Wx5///bPV8izNIWXB6/WgGPosDPpXh7st0wJKLShEVliyQcreLKKisn32UGOKkcjMSB/FrnpTPS6HC6td2HZ+LpxKbe//+WSMynQtt8+/nRkfzWBXuZDUDpTxB7Dm5TV8e/m3zbZTb6qn3lTP3aPv5rwDe0kvvhdL1EyIyOmqrst0Bn0GZM6B8CwidZEAOFSmNtX4KzYXAxCljUOJWpTOkgk6suOvi9hftR8Qkbgqpcpvv8UCgxIXM/RQpqg3INO1/NQLfp0GiLpgtbZaLAhHYH29KCBbV+pfUOO9Se/xYvcXfbbN2z0Ps81MZmQmo1JH0dAgDOMAoZmTIOduWbq1NZw2WDgKNtxDtCEUVWOR+wpbO8O92oAyOpItU+/BOmKiJwtXcjbitMEXehSrO5ZpJNOEmGGYoy/E7tKhrktnTNoYesf27nSzpdtKKd1eKhbZbqi1C12XiJBIURs18y8QNaTT15HxR6tFGP8B3dJhsGSi3zGJ4SILUJq0NSVpSBJ3F97NiFtH+O0rXFtIxZ/zyI96jBWmT4PbcZmAeDI4m3vNbrwHtj7qsymQ1GdoVCih0aE+i21J6nPyf2dyd+HdaELFIlv6vYgOjUZ7RHaKTPBIb6x7XlDQ/DF5VXnctfAu/vvnf5s9RgqO0TpiPWMflVYoJ8gce4Y+B+cdgBBvQGFWjPDyWLQHfBbYVruVlQUrPZHvTamthcPmntzwaz30az5SW6YVhr8Kpy9t8RCHy8H/Lfk/xrw9pk11j0NCQNW4bJTeq9K4DNeGo1PrIP8TKJzvd67k+JPeBzLN0EzArWSssquqO+z4C08Mp8f0HsT0imHiIxPJOd/fSGk2gwIXUUaLXPO2i5AcfwBx45PZN/gCakNiWzijfeT9msfj6scxfS1+UdSEwuw6GPd50K4hgzAm93sYYkd5NjXN+Gsq9Tn67tHMzZtLRHrzVuOPz/yYH677wW/77ord5JUV48Z1wkq2njBMXgDjRG3TJIMoRutW2imqLm/pLD/sFjvVB6p9yg683Otl3pv0Ht2jumO09kfriJGfZ1eT9zEc/Nrz49IDS1m4byGHakSJluYcf4WrC9n22TZsdYE9Q4vvX8y/I/+NtbKe2lowNcRTN/Rb6H5tl9yGTCcxZsOYDyHxdCJ04h3sVNVhaSEDW+Jw7WEAItXCpmQwAIU/ifqRJcu6rMunGkexAsCpxd7KvYCQ2QlETQ1oFBlUJ9xBZMywo9m1U5PMv3iii6QMTKu7CjdO3AmJPFD5QMDTuk/r7qMdDvDJNpGpcGm/S1EoFJ5sP61WLNhl2oBKC7ZKcFoxGkHriMOqqqMwrxDnH05SRqYQnhCcmZqUiRKmd1FpFaGBnixcp1VoR0cPA1kxstNIUoJxphm8//AMAIrWF5H7fS6Drh5EVLe2y11JLH9iOdu/2M7fbH+j3qbCpqoW1wqLgrBkGCMHTnQlUVHCSFwefhlJCU6//QnhCeyr2ueRc2yKOkSNITmwwyD3+1yiF6zj8G0vUVORBN/cBSPegNRzgn4PMgJpfFZVCVujX3zKiNdB7fveTQxPJDMyk/QIryXZVmejOq+a8MRwwmLDcLu971lDlBpDtPeZS78XkoMYgA33gLE39Lg+aPd2qpOaKv6Oq/8Ktu4IKLuTFiH0/w7VHMLtdgfMwpaCYzTOGLHwqi+D2r0iwlYb2VXdl+kE3Rsdf/XaQuqsdkJCxFx3bdFaJr43kczITPLm5vmc463vp/CTi5RpB5H9Wj1ErVTz2fbP2F+1n+UHlnN2r7NbPF6hEBknNTXCQRQTg79axYZ7IDwLUs70nOd2ex3/suOvBRYMAYUSZqzz29XU8dfRGn9hsWFc+uOlze5flr+MdaUqwsPiuSc1G9bfDsNebPZ4mXby59VQk0vo1BUoFUpcbhf6bC1VKf1QBtHHqovSkTMrhw1Ru8EJISqdHPPbFYQmwQDfrGpPlpi2kIpKNyD+4w1JLQcoud1urBVWT2CahN1pJ/tlUfNvmqoCoyESVl0DceOgx3XBuQ+ZgGhVWuJC4ymzllJqLcLhiG9zfdpN721i/q3zuWLJFXQ7TdTQ7XtxXwwpBi474xoq34ErBz5AxO7PYXTX1qk+pdn8IOgSIV3It0aHisVmhaWx5FWj1Gdpqe/a88xXzuTcd8/1KMQcSfq4dFwOF4RoPQkWBjkG8YQgIsQbfCFK9ES3ePxhs3D8GZUiEMBoRNho7SYxX5MJCvL/ZJBRKsV/qeT46xHVw++Y0lKorob8mqHoxr4AEX2OZhdPTQY+CQMeBfDU2nDjxq6qbrHG3+lPnc70/0z3/FxdX838PSLK9tJ+YmEnLQ6jIuwovk+HTQ8Gv/8nIzN3w/BXGx1/wpiR92sen537GQUrW0hbaCcV+6qJLtyKylGAqzHK15Pxp42AMR+hyL6N/v37e8avTAfY/z7DDsTSJ+53H+mV4k3FLH9iOeU72xfJJ9F/Tn9O/9fpqDQq6uogunYC03fm8fXsr4LUcZmWiGr01e5SP+x5hzalpYw/t9tNxe4KqvKq/PYNvmYw68+JosZYg1pjBEOPDjsWlEqlPH7bgPQsHQ4CZzMkTvGJqgbIjs0mb24eiy5f5NlWsKKA1/q/xs5vdwJCjk5KWqjJPYypwGsxlX4vEvSNKz+3G3KfhyL/bBWZjiMZ+vsbPse99VGRzX4EHnksR73HkXAknow/e6yIkj68CBaPgZLfuqLbgDx+24XNJKKrK9Z6NiVHJKB0aVG79BSZvJm5fx76E4ChSUP9mqmthXTjdgYm/AIOuSZ1h3G7oKEy4HhrypRuUwBYsn9Jm5ptWucPvAY0j+Nv7Gcw+Gmfc6qqhIKJUglJSW3sfxA44cZvzEhRZzYAwXD8tcaNP93IYwXjqQ7dzj7X5aI/MkFEWJUVCoU3608ralCb/UtRd5jkocnM/no21T3F2I/QqKFwHtTsDt5FjgIn3PhFzGVuzXmC/gde9ZGHdDld1ByqwVwS+EErFAquW30dF311kc/2ppnYapeBKINF1DYu+6NrbuBUp/hX2PEMuERgfWqEmJtaNYVUV7e9maShSYy+Z7RPgOkZL57BuAfGeb6dgxKXoCpb1EwLJz7Hxfgd8xGM8JYDkILrPcH2MWJe4nDgk6EbYgxp1ukHMPDygZz79rncvOAWlvcZjC71TXTb50KpPC6PS5w2+PNa2P0KGpUGrSIUgApz65MpyVagdzfJ+Eu/SNSPjB/fZV0+lhyLMXvifOVPMDyOv2h/x9+2beLvHj1ApzuavZIBEV0kFR21qSuwWIQu/JaPt7R67rc7v8XmtNEvvh/9E/oDXgNqfHQN6BJAFdplfT8ZkTL+AJx9nZz77rkkD00OWvul6wvI2vgNmsp9ABi0hoCSc7a2iFDLNE9ILC5jf+xOnSejCKD3eb25ZfstZE7K7FCz2TOzGffAOEAs2lXuEBJCMukZ01Ok/6++XhR+l+kSJGdRc4sxyaETqMYfwKv9XuXnuT/7t5sVxYFUC3atnXxdL5j6B8RP6HA/5fHbOmo1Ht38Zuv8uZyt1h2OyRYSZkmDhXVZCp5RKuGTKe/y0w0/eY6Vfi98Mv5mFYvsQpmgEREhFkrvbf4XhwbtBqV/yHSIOsQzXiUJniORHAxaZ4yIuIweDIOfgaiBXdZ3kMdvm7FVwqo5kO+VR1YqlJyTW8b0jSaiNd650+rC1QCMTPF3LNTWwrTub3FN5lRo6FhQjgyw6zn4OgYq17d42OlZpwOwJK9tjj+9UL/3OCr8Mv4SJkLcWJ9zpHd6ZKRXKvRocUKN3xGvCYnWAHjq0qir22WAPpI/X/iTxxSPseT//J+3ZOCy1uWwOewD6PaXjl9Ixp9R78C0lYBX7rNiwwEGLP4P+n2bcbSuOtYuzPVCLiYl1AXLzoZ9bwb3AkeB43r8OuthyWmw7R+eTSHqEB4Y/TfSKq7GVOV92ZkOmHgu7TlWPruyxSaPVDuQHH8hyjCUbg1avR4uqoVhLwXxRmQ8HPwCNt0vsnmAcenjSLGcgdppaH5tEoDUkalMe3Yasb39JXwlx98jq9ahOHtXMHp93HLMx2/8BIj2Bph5Mv4aFURUKm+piaZyn3aLnQO/H6ByX8sPfXfFbmrCNpEamY9i94tg2h7c/ssEB6UG8t6D4l8ACFMKg0NpTeuOP0nqM8zZJONPJujIUp9BxuUSBrOnpz7N1YOuJjs22+8YyfF316BpsHkkDHziaHbx1KTwJ9j/rjBghWcRGxZLra0Wm6YMi6UXvz/5O4WrC+l/WX/PhNBcbGbRvYvIuSCHnFmiRkNNQw09ontwQc4FnqalqFB1WAzMWOt3aZlmKFsJFWuIMN7KkP2fcs4ZoVx7ZvAlp9TdM9g7dDbZPQ18NOsjGpwN3p0F38Hhn3Hl/JXcfdX0798f1dG2mJwspJyFMuEs9rwK9epiUv4zlHqnlYr7KwiLCQvKJSQjmGQUo2oT7HsLul0elPZl/JEcf9GmD2H5tzDqbdB6JVvvGHkHcwbMoXuUv6y1QqFg4t8nEpXlL/HqdLiwuKoBiNFHdqqPLpeL3Nxcefy2gZgY8c2qrIRu3Y7YmfsSbLgLpq6E2MAZEQCRGZFMenSS52dpgR0W6mbioxOJSPNKfMzqPYvMyExvxp9CAbq4IN2NjIRCIeQ+d+7szoFKSGtGciwtIo2SuhIKagoYnDTYb7+3xl9jXZSIPl2uSiGP33YQlgrjvoLI/j6b9WojJkT2LYDT5WT5geUAjE4b7ddMbS2sO3AZkem9mKhL9Nsv00aih0D361vNVp+cORmAraVbKTGXkBCe0OLxR2b8+Tj+pKiqI4zXkqMqyv9z26WcTOPX4/hT1VJZ7aCjZpJtn4iFvtvpqy1Z76jH1CAWjVp7glx7qot5ZOIjOFwOMq2JLAs7iEsdQl2dNwCqM+z6fhf7f9lPQ7p46dqUkTDyHYjo2/nGjyLH/fhVaKBqMxh87WnSe66uDmy2xprkyQZG3DGCjAkZuN1u5t08j+7TunvqbB74/QBl28voe3FfQqO8QdqS4y9UIazNBoMCNPLg7DJ63w1ZV4NaBOG/eMaLaJdArrmFoMQ2UFtUy6J7FtFndh/OO3Axhf0amFm9EJSZwen3cchxM35dTlCK6x+Z8QdC7rOsTDj+evcW26ryqnhvwntMeHgCkx+f7Nfkz3f+jEqrojZVTIT21Q+F8w6BRvYKHZcoFHBhladkyOiYc9i8sxYb+lZO9AamhlhF8KLRCBz6UQQmZl0VoDbJiY/kMzqayI6/LiIzMpPMyMyA+7ZvB42ynljtLqiTF9xHhbqDUPAt9LoDwrOI08eRV52HTV2GxQKTHp3kV8uvdHsp2z7dRsaEDM+2uaPmMnfUXJ/jpIy/YCwkTikOfA67XyQu/CI0zhQstV1zGXuokeokI3Ep8JcBR0TWlv0Oe1+H7Hu75uKnGGq1+FjbzWEcrisCwGqz4q4R9aTCYtvvAHx79NukjExhxvMzMJuhOPI7SqNW8cv+qUzJvh2yrgBV65MKmY4hLa51th1Q+D00PO3j+Osd27vF8yf8LXAW3wdTPmTWOi3P3AfjNKWw7UnInisvtruY6GjYv7+ZxbWhB6TMBJWvFMGlX1/Kn4f+5L1z32Ni5kS/07x1VBWMvc83C6VbVDe6RTXxMDosYM4TDgyt/NEMJnFxsGuni5rSUrCF+IxTiTRjGuuK1lFgCiyn3Tu2N6aKELT2BLmWxvGIUuOpo9IUqb60td4JqFhbtJZySzkRIRHNZvztqRzOwZDhcBzaek8YEiaLP60Qp49jUOIgNhVv4te8X7m0f/M14MA/4y9OH8eYtDH0ie0DtbthXl/o9wj0f9hzjiShdbQdfyccJcvg4JfQ537Q+xZDbFqXpry29bo0zXHZvMvQhGlQh/qaWUrrRLqDyq0lJ2IvA+tfg4qbIWZ4h64jE4DS5SIosMcN3DD0Bs/mw2f0o6ZGzH2CsV4/+PtB1r68lsgnU4k0jyI8uhd0v7rzDcv4olTBhRV+m8vtB6iK3o2mLpPKyp4kJoJap+aMF84ARKmB3O9zOfjHQY/jb/sX21n78lp6nd0roOMvBOFQiDLUQcUO0GeALr6r7/DUw9jLb5NUg7w9jj/TQRPzb51Pn4v6MPCKgZiLzWz7bBvx/ePJd+7GpmtgSPyfUG1uUz1emQ7y+0Vw6Du4xAYKhSfjr6njL75xGDXN+IvMiGTaf6eRPjZwUeI98/YQYgzBcqlYZIbpIiEspSvuQCZYNHHK/n3w6zw5HzRtyOl4eurTPD31aR58ELbR6Pjb/SKUr5a/q0FElvo8ylRWQlERONw6HGcdhNHvHesunRp0v1Z8kBKE0XJs2ljGxM1A7YzEYhEFZLNOz/KRf8g6PYu/mv5K/8v6N9cq4M34y4rcCDv/I5yMMq3T82aYshxdhIgMkhyor/Z9lS8u/CJol/FkowTyOQ1+Bs4vgbDAkw6ZdmA5BFseYXjG76hd4Sga62yUFJXwbPyz/Pq3X9vdpNvlpt5Uj90iqjrX1UGZcSFrtE/zx8E/RMFfbRQEkG6VCQ6RkeLveQcfh0vswjkUBOKHpnEoQ3j7Bzn2wZa/AUc/+ulUQ1pcV/jbUSD5DJjwLUQN8Nl8uPYw+dX5nuLbbrebD6Z8wMK7FwJNHH9t8etXbYT5/WD/Ox28A5nmiI+HXjGrObM+Cfa9HfCYVGMqAAU1gR1/P132E5dWbSe8oadw/K29DRaO8mYZyRwfHFFTzhlawrqsC5k+rxdOl5N5u+cBMK37NDQqjd/pknFNltM5epzeTch9/rL/l1aPPTLj74qBV7DimhU8OP5BUGpFgMYRxlPJ8Sd9s2WawbQd9rwCtXv9dmlUGpb+ZTWTtu7CbjbSUQW1sNgwNGEaP0nBErOQvta54kk17CbB/K6YO8sEjwNfwPq5YKv22RzI8NwZJj8xmbuL7mZCyj2M27WK0yOvC07DMm3iieWPsyJrGkXRn/nUDZNQKBSMe2gck5/wBmaMe2Acf1nwF5+acIA3C9ctPohJYdtg4QjY/37X3cCpjrPeZx4THQ1ORUPgtUkzuJwu9i7c65GKTBqSxN8a/saou0Zhc4lM3Lt6XgmbHgxq12WOIHYkpF0AbpFAERMm7HpWh9VziPT+LWlSFUQbrmX0XaNJGRHYmXfb7tu4dtW1WBvrUCeH26Fmj/jdkTk+qcmF8jWAN8CmPfWSpWONRmDwf2DCN8Ht3ymO7PjrAvZU7OGJZU/wY+6Pfvu2N8oSd+vWGNGpkB/BUUEV4lPz5r/T/8tbkxcQWzsJa+N3ye1y47Q7fU7ThmvRhgungrsZw5fksOqmXwob7wVLYIOazBFE9Ib48YRH6KjSr+ZLx+U88tsjRGREEJ4UvKyfqi9+YdDCf1NmWscPuT+wu6JJ4XWFUkTzKVXHp8TJiURDOWx7nP4Jy1CgJEwlFlAN+gaG3ji0QzX+FEoFt+64lZlvzARE9LtdVQ00SjKZdkD11iDdgEwgpOyBiipNwO9VpbWSF1e/yL//+HfA83958Bc+OP0Dv+0D7jyNX6bvBGB78mUwY4NH9qUjyOO3bbTo+GuGOL2Q5iyrKwOEQcV00IS5WKSjSI4/Q+UB3hzxJnsW7PGc+8nWT/hi+xfeyM/QZOj3MMT6yw/KdI64OCg2d2ed6WaIDFyT77YRt7H6utXcN+a+Ztupbcy+NxgAV4NYZHexzIo8ftvBn1fDF2Hg8qpUGNRRVBiXcKhuPysKVjB/73wAzup5VsAmDh5088456YxV3xBwv0wbqS+HlXNg37utHnp6t9OJCY3BENL6d05y/EkZf747u4kAjYyLfTYfK6lPOMHGb+ZfYFYRxPtnrwNM6D6CSGc2StTtMli1BanmbYg9geUHLiV/uAmSzwzuRU51su+AKctBG8Weij0sy1/GvoJ9xGxbjrF0T9Acf5pQDYYkA3VWMS/uFr4Svu8G+Z8E5wJHkeN+/Bb9DIXzfDalGIWzoEFT5JMltvwfy/lvyn/J+y2PEbeN8JRqATCmGukxowcKZeAaf2qXWLcq9Skw6F/NviNkOkn+Z/B5KBQKO+nyA8u5uyyCFTkj27U2icyM5GHbw0x+zOvcVWlVONTeudHPVc9Az5uC1vXjkWM+fnPuhXGfCUUKRKCS/WE7n1/4ueeQQI6/1lAoFKi0KqxOscg8LWYx/NQLKjcEresyQWb1dbBc2OyMRnAp7FTWNLRykhef9WfUAEic0gWdPHWRpT6DjEqlYnXBav6+9O9MzpzMzOyZPvul+n6nD/gDDhSKqE11cOpfybSAyw7lf4o6HI21UaTsBIsF8pfm89H0jzjjpTMYesNQHA0Otn26jczJmURmRAKwrXQbk96fxPDk4fw852dP09LCsMI4B4aPkuUE2oPLQUS4gwZNMbtCPmLhvpE8Nv+xoF7CHmbEYkxkn/Mb7vnsn9w+4nZePONFsbP0DwhLRRWeSf/+LWd2yrSCsTecuZXd74uMklBFBHWYqLXXcvb/zg7KJcxmsKurAYjSRYmo3vI/YXYX6cTKeIyINnMVFG8UWQZhqZ79pnoTc3+eS4gqhPvH3u8X4V5bWEvF7grcLrfPYruuDvodfIWw+MNM7N0TWql51BIqlUoev21EKrAeUE7H2SAiYyNyoMf1ns1xYcLxJ8mUAdy++3bPvyXHX4jChvmwGZfDm7l5z6J7KDYXs+GGDUL+JbwbDHg8eDck4yEuDkwN8Xy461WGJQU+pleMv8SSRIOjAa1Ki9ksxml4ONDzzS7oqS/y+G0nUYPBYQWnBZTCUBmq1ZJQfS6HYt/nqx1f8d657zFvzzzO6HmG3+k2G5QW2yg2Z9EjOuZo9/7kQqGE/I+FvFErckTTuk+j9L5SlG0I+DxS6tPldrV63rGS+jzhxq82Amhe61GhEFmT5eXCmRoXxJK0xeZiADQNCbhREh5llKV2g42xlycb9m+//Y0vtn/B8yOex/lLNZEZwygr6xmUy1TsqUChUGA2i2gqXZgSQmL8pNKPd06I8bvhLvGuTfEGsqQYhOOvXlvoM5+tyquitqiWD077gJu33kx8P+FxcLvcWCoshMWG+a1TekT34PYRt7P9d6Fooo1MhT4PdPFNncIYeogAjFBRzytSF4nVVYNTU9guqc8jn2PlvkrqSusIyQ7xbNvHzZDir3pwsnA8jl9tABWmlMakvoIjciO+u+o7SreWcsN63yA0l8NF/tJ8IjMjqXeKiVC1ehj0+auQ4JU5Pul5K9irAXhs7a3MH/oqvQofx25/GE0zw9BsMzPsjWEkhidiqF0EaEXGn9N2Uit6HQuHvZxuFmTcbjd7K4V8SI9of0k0KeNvVOwbsOIST1q0TBfjbIBfJsDOZz2b9Hpw48RiEVFgWVOyCE8UYbbFG4v5/urv2fz+Zs/xeyr3UGmtpKreV1OiSJQyIyI+DuJGg0YujNMmqrfB5yEkVj2D1hELQJmlLOiXsfYfwe7RV2IOE2FkUtFh3C5YMhnWz8XtdlNTU9NsVqdMG1DpILIf4dGRAGgbayVIEiodoaawhnWvr6N8VzkgnEV2lRh/kbpI6H6d7EToYiQjYqJ2Hfx6ul/UbUKjw67B2eCJmm3KrA9mcVfBXX4Rtn8+PI/0vTV014wjQeUGu/+5bUUev22nRcefUisk0A5977NZcvw1936W5Oj0g3pyV8FdZM/MBqDeUe8xdEoSkzJdh2SgLuvgZ/SeRfcQ90wcuYb/AUdPBlIev+0k+w4RXd2klkZICCRVXQjA1zu/pm98X/467q/E6/3rEx06BDZHCE+tWYpu1D+PWrdPSrRRcLEVhr/a6qEqpapNTj/wz/hLfDaRzOczyavKg5KlsP5OMO/3OedYOf5OuPHrcgq5MHN+wN2fb/ucHQl/pyZ0c9Az/samjeW5qS+TWnYd8fp8DPYNfrK9MkHA5QC3i/DGmtF1ujqyn7+Rop4Tg5bx992V3/HuhHd5x3whvwxIZbmjFGasg7Tzg3OBo8QJMX6HPAdDnvfZJGX81WsKfaQ+z3nrHG7bfRvTn5uOpcLCSz1fYsdXO6gtquXZ+GdZdM8iv+ZHpIzgxTNepGflHYD3/SvTRcQMgzEfCZsZXieuTVNOSXnbs4MA8n7N4+AfosTOhjc38M6YdyjNF4Nc4dJgDD95nX5wnIzfkt9gzc2ifnszZGaKv8vLvVldIOpyqkPVfv23Vlr5cOqHrPzPSgyKRLT2WBr0U2DQP+U6f8czmZd4Mmyjw8UaxaYua3Eudbj2MLkVuawrWgdO4egzhFrg8xCRQXiScizGrOz4CzIul8vj+Ose1d1nX20tHDgg/h069H4Y/7XPwl2mC9GEw9AXIEtE5H6942uy3jSypudZ2O1gyIjmsnmXkX2OMFjGZMdw4ecX0nd2X08TgRy6ZrMwogD0yqySF3DtITQJUmehjemF1i4cf+WWcvKX5rPkoSVYq6ytNNA2JKN0navR8deoPY7LIRYTWVfjcrnYv38/LpdcY6xTNFSSGCHqgGmcIqLaVG/it0d+4+tLv253c6VbS5l30zwOrhCT+qZSn1GhUULqqvddwem7TEAMBlCp4KCpL+beL0P8eJ/9YZowDFoR7CDJWLWGo97Bga/WYSzLE5nXi8fB4gkd7qM8ftuOJPUZ0PGnUMA5+8XcpAkeqc8mjr+CVQVsfGcj4M34k7JUJPKqxCLQoDUQGybe8ex/H5acBrX7OncjMn5Ijr+zMx7F+dt5AY9xupy8uvZVbp13K1a77zd2a+lWKqwVqJx6FAoIC3XBruehZFmX9lsev51Hq4XYmqmEqYwU1Rbx56E/mz1WWodkZHS5guvJj0LR7gwft9vNrvJdLR4jvUvr6oQMXZmljAOmA+I9WrEacl+ABl9NNEnq82jX+Dvhxq/TKuTCtj4ScPcHWz5gTegTVIet9/yfBoucuBzm9LqVpOpZnNf7eUJ+GwoNQfJEyQhyX4LPNFC2gnBto+PPVUfmyEQcuvCgOf6GXDeEMfeOocZZQr22EE3IiRnIfUKM3+QZkDTVd5NBZIvVHyH1qVAoiOkZw6g7R6GL0KHSqnA5xb0NuX4I6ePSm72MFGgR3/AdLBzpqVUl07VEh0YTohJZevkVh9tVUvqbv3zDLw+IurnZ52Qz9ZmpuGLE8+6mDOPCkF6w67mg9/l44bgYv6YdsPd/UCcmlxa7hSu/u5LT3j8Nu9MOCJW1hEZhn7wm/sGz/3c21/xxjV/2pkavYeabM8k5P4e5ir1M21xGSlTsUbkdmeCQavRmZde0ENstBQjHhwqpGp0OtGobZFwK0cO6vJ/HimMxZmXHXxfQXMZfbq74OzUVDKn9TriosBOe7DsgYRIAoZpQam212NTCkCnV+XM5XTjtTkKjQuk7uy+xvb0fmT0Vom5Rz2ivTMjuxnJxSUlg3HA2fN/8hFLmCEJiYPxXaHteTCjCGVfTUMO+Jfv4459/UFsUHPlGxdo1JO5dgbnR8ecxQKu0kH0bpJ0XlOvIAPNyGGEXdWeiGgYyJm0MhhADh9cfZv8v+1s52Z/k4cnMWTSHHtPFu1RIfTbJ+JPpciTZq0prMocNt0JEH79jEsMTAe/krSkVeyrY8tEW6srqPNtUISqGfnEfS0/fzpbQF7Gnz4bMS7vsHmS8SI6/qipwOgMcEJYiauI2Qcoaair1ue7Vdfxw7Q/YrXaP48+xcg37FnkdevurxJjPisryLurqS6BqE3AcR5efoOj1YnGdHrEdyleJzJYjUCqUPPzbw7y67lUf54Pb7WZriaiXarD2JzwcFPYqIbF14MSrWXRSYz0Ma2+Dg14HvVYLKncIQwwzADj/8/M9BpcjOXAAekav5fxej7cYoS3TRqq2tNk5Xu+op/uL3cl5JYdDNYeaPa5pxt+BamFMiw6NFvUBs+fCeQUQOcC3G8co4++EQxMOfR+C1PMC7pbmlg51ddAdf+DNdthWPRMGPgna6OBf5FTGmA3pF4M2yuP4M9vMROrqUdmsQXP8Db5mMKPvHo3NVQ9AVkihUBVqNH7LdC1SlliDpoSiksDfusRBidyy/Rb6XdwPY6qRmW8IR8KRlNWVUVxbgqlOZJuFasxgLZJVuboKey2svt5TG1ehUHiep1lZ2K46f9Ofn86Eh0XgaNqYNMbcOwZlqJJ4RR+MtizcSh0o5MpWXUq3K+H8MogbB4BOreOzbZ/xW/5vFNYWeg/rJv7Oa8O0U6vXMuS6IXSf2t3jkO/jfBR+nR5wbSNznLD7VZjXH+oOeOWYNYUtZ/yZRcJAtFbYkoxGRGmusZ+c9PU5jzay468L2FclDF9HOv4Oi99r0tPdouaczDFDki6zaYTjz2KBmkM1vD36bVY8vQJLhcXvnD2VwvHX9LlKztzevYHkM6Db5V3b8ZMQhQJi9FHgFobhbld249ZdtxLTMzi1Z8JyNxJ3cB0mu5CL9Eh9ygSfHjdSHycCGoYUv8KKa1Ywrfs0Lvn+Eu4ru6/dzYXFhNF9aneMqSIz2lznxtGY8RepCYX5A2DbP4LWfZnASM6i5oxgktxnidk/42/fwn18e/m3lO3wZospFArqlEq2dv87891zseb8Va6ncZSIiAClEtzuZp5nfamom9lkYZVqTKVbZDdPhDXAyDtH8pcFf0GpUmKxgMLpoOrzRaz67yrPMZLjr3t0E/WDPvfDhZWixodM0ImPh2dWfsbm7iWg9K8foFAo6BsnlAy2l233bC+qLaKqvgqVQkV4fW9RWF1tgCm/C0eDzPGDMkRI8h721pqWalYP0p0HiOxrVYDnD8Lx1zd+OSN0jwgnokznWHcr/HFRmw7VqXUkGURU8w+5PzR7XNOMv/zqfAAyIhrr2qh0os5ukwCNhgZv5rXs+GsDA5+EtFkBd0XpxH+gXVUddKnPZfnLWJb/O3ZlDQcbThcOSHVYcC9yqpM0TUghR/bzcfx9N/kFuq//EpNJ1DkNFja3iBzuptgLG+/zk+CVCQKrr4cvI0WJjkbi9HFolBpQuMkrbZvaSHPcu/hekv6byL6YFwHQZs8RwRVxYzrVrkwzKFSw7y0hEdlIslHK4Cz02EvbQr+L+9Fjhu96ol98P65r2E7qlg1sSt4C2bc3c7ZMUNCEgy4WlMLBqlQoSTOmAXDQdNBzmOT4y8/3nlqxp4IVT6+gbGfzNQqkYBmDYj9UrQ+4tpE5TnA7wGUDh9UnK7uluZQUNB6lEXNjg1wxq8uQHX9BxlRvotwiHAw+xi68jr/MxDL4LERMEGWOHhvvg+/SweX0SJfZ1GW4cWOxQFhcGA6rg4pdFTwT+wy//f03n9OlTM6mGX+S469XL6Df32DwM0flVk4adj4La28l0qhC4xSLbavRSmx2LCpt5z/sbjfsGX4Zu0deTnXDEVKfuS/B/IFQI9I2dboTqyD7ccmAx1H0vhMQUoKSXIdS1bFPjcvh8tHANpth0rZcvjhtI0m6cHDWg6OuhRZkgoEkHdYrbySsnOO3v6WMvx5n9ODiby8mvq+31tS+Rfso3pMPgAKlxzjTGeTx2zaUSq9hOKDc59bHYNFoH/mxMWlj2D93P59f+LlnW/LQZHrM6IFKq6KuDtxKFX2ev4HTnjzNc4wUBJUVmdUl9yLjT1wcuNyqFuv8SY6/HWU7PNu2lW4DIDWsJyq3Tiy8VFqIHxcwyzfYyOO3HYREw7kHYcTrnk1ScEa2/RJemPECy69a3mw9uQMHYOHe69nXayNEDTwaPT656X13u+b+52WfB8B3u75r9hifjD+TyCDKjMxs3JjnJ5UsBXFotRAa2uauBI2TafxGh4rBZFOXBz3j78afbuTq5RMw6dfLxq2jgMfxZzcz+JrBWFLF+r2jdXAl3C43b495m2WPL8OByPgzxU+DaX9C1JDONX4MOO7Hrz4TYkb4BM0rFUqemPAMA/PeobbCQEMzpeH2LNjDkoeWsOyJZXx31Xe4Xf5qE1J9crXTiEYj3qMyXYgqVAQAjnrXs8mTHaQtpNh/Kdkm3hj6Bj/e8CPglW09sgTBycgxH7/OeqjeDlavAz4jUgQqSYoFEDjjr3xXOb888AuFq72ZgQAL71nIKzmvsGrHKj6LGMrGzCuo6v0BXFDedfch03my74CZuRDR21OHtUFzmKrq5mUtD9cKB4lB0STjr/QPWHOT+L2SCRqy4y/IHKgRL7gEfYKfMVP6kCXE2yHtAojod7S7d2qjCoPQZHCYPRl/TmU9TmUdFguoQ9TcsP4GJvx9AgPmDCB5uDe7wWK3eNLVpYw/t9vr+MvOPrq3ctJQ/Cvsfxej0Y3WIRxylXWVmEvM1JvqO918QwM0aA3U66Ookhx/Usaf2yHkJrSRqFQqevfujUolRxF1Fsmp4HDg0fSuOVTD7p92Yyn3z6RtiUX3LeIfIf+gplA0ZKlTEGbLZGjKIDRhSTBztyj0LNOlSM+03hEmsk2OIEHfmPEXoMZfdPdoep/Xm7BYEdXudrn5Zs43VL+yEIA0tQHl8vMg78MO908ev+0jpvEVGNDxl3oeDPp3m+tWud0icAaFgpiceJKHer+bTaU+PRxeDId+7FjHZVolLg4M2gp0Zd96glqOpG+8f8bf1lIh85mh6w80Rlw6baIWbhcjj98OoE+DJo4975hWcMfIOxifMT7gaXV1wuhtdRhJ6D0I1KeAVayrSZsFWVe2+fDzep8HwG/5v1FdXx3wGMlYabXC/qp8oEnG39pbhNpBEySZz8jIo1+38YQcv9uehJ+HB5QMS48QJRssIflBd/xJc6QQeyJ/yboCls4M7gVkwLRTGAxLl3vsMLUNtUz/zzRU40YDdFru0261YzpowlRkxqkQa1V9VBLEjgRtROcaP8qcEOO33//BaYv8ZOgfmDiXnIar0Tgjms0Sy/0hlz/++QfbP9/OvoX7UCj9X5CmepGOonYZhcx5+SrY/wE42rdmlWkjCgVoo3wyt0amjKSv9kx0tjSKitre1Pzb5/Of5P/gcrpQqpUoVOL51tVBYvg+UuteaHYufDJwXIxf03aY3w/2ex250nwlUMbfwYPeUhPp49K5dtW1ZJ/ra0jV6rUoNUrKFRVUhmzAHLpTOIRkThgS9AkoUOBWOiiobP6jW1wnHCR6d5OMv6qNsPd1aDh5Hb3HYszKjr8g0ye2Dztv3clXs7/y2yc5/qKSU2D8l+1aKMoEgQGPwfQ/QRtBmCYMnVoYNm3qMk+NP5VWRUzPGGZ9OIvsmd6PkKnexPTu0xmcONiTMVZUJCKKtFrollwOv18IBz73u6xMC4z9BC4yYTQqGLvzT74e1EBmcSb/SfwPG97a0Onm6+pAW1eFxmnlnXPe5cUZL3pkCel9F5y7H3TxuFwuKioqju/i5icCe15D88cZJMRaORD7Br1eT+G2+bexe95uPp35KYc3tk9WLK5PHD1m9CA0SoSwSxF84Z1PEJNpB5Lj7xvTbzDqbb/9t4+4nZXXrOSOkXe02pbb5WbmGzMxn54JQJLGAEULOrUwk8dv+5CygwLW0UiaKuQ4tS3rxe38didPhj3Jlg+3YLFAqKkYndrXSfTvKf/mq4u+YkaPGd6NWx+FNTd07gZkmiUuDpINu5moPB8OfR/wGI/UZ6m/4y9JJQLSDAYg7wP4TAPFv3Rpn+Xx2wEaKqB4iQhewuv4a602zsFGG0yvlAOE6+Rs+WNBz5ie5MTm4HA5eHfjuwGPaZqlkFchAkqlCHoyLoW+D/ocfyzr+52Q47ehHBrKwFHjt0vKrLRq84Mq9dngaPA4ekPsCei1JrBXBe8CMoL6EmEwrNrCsORhPHXaU1w/5HpASGEDlHROGRKtXsvdh+5mzD/O8jj+InUKcFi9UicnCCfk+G1CcmOsWXPOorH3j+W23Nu4afNN3L43sORj04y/8HAg733480rZ8deVmHZB5XrPj3eNvosn+8wjuerCdmX8hUaHEpUVhdPm5LrV13H2a2fz/qb3+TK+D7ru95Fw6M7GuuInJ8fF+A1Lg35/h7ixnk1SAI2kWACQkAA6HdjtUNiY4BcaFUrqqFSPnUdi8uOTuXnLzZTXiTmu2mkkwvwDlC7v4puR6RSWQtj7BlRvR6PSMMJwIanlV1JT2/zvp16jJyk8iTCHyBCMiAB63iLqRsaOOkodP/ocizErO/6CjFqppndsb8alj/PZ7nZ7J5qJicegYzI+KBQKT9Zfg6bMUxujOZIMSfw852c23Oh1RknZft27g9peAgVfQ82ururyyYk2EpQajEbQOqOxmrVEpEcw7JZhJAxI6HTz5loXA357kaxt8/jLgMu4feTtHodvU9xuNwUFBT6ykjIdwLQTSpeTnmTCrbBTai2ipK6EbpO7ce675xLXJ65dzQ29fiiX/nApmjANNhtUK/eyM+UBvtj3BtTkwt63oK6gi25GRkIyJjYX/Z4dm83otNHE6+P99pXvKuffUf9m+T/EZF2pVtL7vN6UdheRu+WKOLjEBv0f7XD/5PHbPiTHX8CMv2aY9fksMp7PYE3hGgBSRqQQ3y+eiIwI6kwOcla8xeaHfANfcuJyuKDPBXSL6ubdOPApH4lCmeASFweHanL4ouBDSD034DF94oR05/6q/VjsYvIzOHEwU7KmkKoYCTQ6/sLSIP0i8XcXIo/fDpD3Ifw6BSrXAW13/B1otME8MmYMLDm9Czt4CpH3EfzQAyrWtfmU20bcBsBfl/yVtYVr/far1cJABpBpyGF06mhyYnPEhqwrRGmBJhxLx98JOX6HPgfn5gcMcOkWKb5XFm0+VdXBu6fSOhHxrkKDxhnFr47vYeofQWtfppHYMcJg2OMG+if058HxDzIrZxZLH11KyEJRV7OzUp8SZjMY6vsS0dCP1AOvwxdhUHeg9ROPI06I8Vu6HDY95FeTtqi2iJq4RVSHrW/W8RfVLYqYXjEoVUq0+sAanpLjT+OMEEEXvW6DCd8LG4VM17Dqclh+vs8myT7anoy/yY9N5po/rkETqvFsO1RTSE3ITpaZw6gZ+TskTA5Gj49Ljovxq4sXyRXxXqWJQBl/CkVguU9HgwNLRWBDbGlj9E2IOwLNhuth8/8FufMyQaV2D6y50VO/85GcLxiU/x7UJjd7ystnvsy2q4uoX30F0Bigo1SJupEqf5Wpk4VjMWZlx99RoqpKFJNWKCDe8hGsuwNs1ce6W6cWdQWw4xnP4nxS5iR6Kc5A5Qpp1fEXCMnx17s3ENkXLrFDnweC199TAVs1lK0kzihWYTU1YEwxctYrZ9F9aveWz20D5ho3h7uPwZ51hBarywlbn4CSpZ2+hkwThj4PF9cRFp2I2ik0GUz1JmJ6xTDoqkEYUzqu02A2g1mXy76kp3l/25tQugzWXA+mHa2fLNMpJGNitP1X2P5PcDZTTCMAIREhJA1JwpAiCtpYq0R6tckmLJVGbaT4MMrFuo8aLUp9mvNh4ShRA7UJh2sPc9B0kKJasSI3phi5fs31ZE7MpL7OSVGvifS6qA31whImQuo5nbsBmWaJi4M6eySL984BY6+Ax8Tr44kJjcGNm90VItP2zlF3svjyxXRzTgcaHX/J02HcF2CUtcyPOxKnwtAXIFzMk6QxbTKJaOrmkBx/+7kGMi/r4k6eIqhChGSqu+2yuDcPu5lZvWdhc9pYmr804DGSssEtvR9n5bUrmd5jerPtSUE5x8Lxd7KRHpHOj+cvY/L2XZhMwUvgkmQ+Dcp4FCjkGn9dhUrbaDD0dfIUrS3CtnknuN2dlvo0F5vZ8vEWindWMiZ3GRdXbMWYOBEyLz/hpD5PCMr/hB3/9Av0/HDzh7zVMJ28hOdbdBaV7Sxj9UurMRebA+5vmvGn1wOR/cQ8VakO1h3IHEn2XCHh2oSkJDf1miIOF7vb/d4t21nGqv+uonJfJZV14nmaG+LRpY0DXfuCjmU6T0ZkBgoUWB1Wn+2BHH/Ppz/P5+d5A0fNJWaW/N8SDq0+RHmteJZhKiOMeBP6P9LlfZfpBFEDYfJCIYFPY/Ye3tI/gbDb4amnhE0iPR2mT0ckEsj2vaAjf9GCzNMrn8ZgNHDFwCtINaZ6tkvZfnFxoCr5GfI/hkH/Oka9PEWxFsGm+2HQ0xAzjA9mfcDzB2CJlVYdfzanDe0Ri4hdjcl9vSTbmlKNPKTaScmv8PsFZIV+SHGkgTfKv0S7YTLXDrk2KM3X21UU5kwlvnsRP+T+QKoxlSFJQ8BaCFv/Dr3ugIRJQbmWDJ6aQwkJoN4kvvamho5rJS35vyXoInWMvW8sdXVgVwlnUaQuEpLPhkkLIGZYp7st0zKSMbGH7jvY/BJ0uwLCUjz7K62VfLj5Qyx2Cw+O95UgMyQZuGKJiOJy2pw8l/YcObNyqEmuBjX00GmFlGDUEAiJPkp3dGrTYnaQOgwsB8FR67M5MVyE4paY/TWybFV1VGaNoc9sr/N2b+Vevtv1HQMTBjK1+9Sg9V2mZSQps/JyIasbqJ6NQqHg1yt/JdWYSnSo75iTFmeynPJxTmRf8acRo1FkiTkcItAw3j/5GvA6/ipTnwDZnxsc0i8Sf9qBQqHg3XPf5boh13FmzzMDHqPXi3Fcd6Qi6y+TIH6iiLBv5Fhm/J2QWIvh8CKIGQERvX12aVQaZuRM4HUbuGnM6gqCk67YLNWxEWomfbRvQEm2CIaRCR4uu1Df0Ubh0CWyvXQ71fXVXPTVRaxaq2bNM4pOO/6KNxfz7Zxv6fvXs4Fo8b3MvFT8kQk+WdeI+tP6dJ/NPaJ7ACIotLkafy6ni1f7vAqALkLHwCv8A9SkdarH8SfT9XSb4/Ojw+VgzFeZFA4s5PTNh6ipSfE4DVqieFMxu3/ajdPmZPkTy4nJjqHKKtYvemUYWrUTkANLu5xfp0HkABjyLADj08dj/T8rIWrfjK1Ajr+BVw5EF+VV4yrdVsofT/1BeGI45VFibIZrIuSg0RMBbRQkTfP8aDSCU9FAWU09EHhAv/mmSKbR6+Fvf4PQUGDlbUIK+KLqo9LtUwU54y/IvL7+df7v1/+j0uobSi/pVScmAqPeg/MOCQObzNEjsh9MWwVZV3s2hTU+gtYcf9M+nEb8M/H8vPdnABoaID9f7OvdGzDvh9LfwSHXTGkX0UNh8DM4jUMw63awyfUxKwpW8P0137P4/sWdbl56rhX6Pzj3s3O5a+FdYoMuEWZsgGyv3r9BDr/tPPVlUDif9NgCNM5Gx1+9CUu5hRe6vdDuZ7rx7Y3kfi9Sa81msKuqAYjSRUFYMiTPgJCYoN6CjD+RkeLvb7bPxT19HYT4Rk/WNtRy58I7eXTZoy1KFzTUNtB3dl9SRqWQYrqAMbt+56+Zw+HXqT61HjqCPH7bTlzj4wtoKNHFw6wi6PuQz+YEvTBWSsZLidwf99Br0cvEFGzyMZisLFjJfYvv498r/u3d6HLCF0YhAyLTJURFgVIJ94+aheu7zGaPG5AwwOP0W1O4hj0VewBvHVWjEdj+L9hwbxf3WCCP386hULRSuxORuSTNWzMyjkq3ZFogQhfRrNMPhPPdhYPqmiaZhC67kFKyHPI5VnL8Sd/qo80JN35rckUNr+JFAXer1d7gh+YkztuLFDQT6kpErbQxsOFGUYtOJrjUl8H8AbDjGcw2M4NeH8Sk9yfh1DhJSBCBMJ11/CUOSuTiby8mtG8WcOIHyhz341cXKxQMVL6lOvrGi+AXs24HhUWB6yUpVUoGXjGQtDFppI9P99vvdru5bvB1jDNcjsYRLeaxS6bA91lBvw2Z5lEr1Z45qSlsfZvr/BWtL+K3h38jJjuGq/+4mrQxaVRZRATbLelr4TM1VG/tqm4fFxwX49e8TwTVN6JRafycfuB1/ElzUYCpT09l/INemdD0cencsv0W+s7u63mWBm3HFaNkjjJuN7jEvPWzg8+yYKiOZaF3Bzx09cEN3Hsog43d5nDffZCU1Lij+3VydmcXIKcnBZkqaxXovFFIEj6OP6XaJ1tC5iih1vsVCTUawY2L6uqWfeC7K3ZTZikTDgegoABcLpHCHBMDbH0ftj0OZ24VDkaZtqHPgJx7UdeDxiFqXVRYKziw7ACG5M5PZMp3ltFz9SLKDeWQjef5odJC9GDPcSqViu7dOy8tespTsQaWnU235HdRO0VUpanBhNagJTQmlBBj+7S6b999O456MXkwm8GhrgYaM/5cDlmG5SghZREcrOxOfRiEHhE8mRAunEI2pw1Tg0k8nyYseWgJFbsrmP3VbM59R9Qdc8+BaHM8CWkxkJUKEX3pKPL4bR+pjWIEJSUiQ0jdhmEkZfwd6fiLHZaBXatH5agXUXqN7K/aD0BWVBPjiasBYkeDPrMz3ZdpAZVKzEkKa3thCXFjcLuFV6gZ7E47V353Jfsq9/H17K+prZ0JNBoyC76B+hJPBG/X9Vkevx1i1ZWixt9Z2wHx3EtLm3f81dRAbS0MTlxIt/znIfJxiBl+9Pp7smKvhf3vQkQfSJzSoSbyqvKwu+z0ivHK8+r1UG5cwum/ncXUg1P4ec7PoNTArEK/84+l1OcJOX4j+8OE7yBqUMDdS/OXsit1IeriUVRXn0taEMqcjk0fyytnvsLCr5JxuZXsTFpETk5s5xuW8UUbKWpGx4zEGGJEgQI3/8/eWYe3cWV9+B1JlmVmxsR24jAnDWOTNk1DZdi2X7e8hS3vbpmZadtumblpm0LahqFhJidOHDMz25K+P64lW7HMIjv3fR4/sWfujM9kfEZz7znnd4zkZuVSs0+PW00IRUW+nX73sYZ3mDfJi5PZ+k0Rfw6LZ7Nax4OHrkMp3QsT/tfuZ66r0Sv8V18PdQXg5gduzVHWxMBEtGot9VSRVXmC2tp+5t6oLVn8weI2T60oCq/Mf4W3s+AHg3juEjgKdGG2vw5JMymvi8SH6T+Bl3jAjokcw978vZR5bSc7eyEDO6FKMHDhQCJ3RhKYGIjWWyhzlTZV/OXWx0PsBaDtuwnCLuO/C1M7NSwuTjwei4uFNL21qk6Nu4aQwSJD1djgjrYhmCQ/Bb4OhIH/hGH329BwiU0xNMCX3hBzDkz+lMgA4XuVSrbVz9x1h/dR455Ogz6T0aNb7DgFqufVasdXIsuKPzsQ4xuDp5tlNZ858BdmFJVhNW1oEkjsi75eSLwAr299nSuP+LA7/u8UFrZ9SGpxKjmVOWhUGnN2mel+RkQ0vd9HngWjX2glQyHpHL6+oG0UHw5F1UXcmHIj/7fu/zo4qmMq86vxLkpDXSVeAs2yZrWFFtWZBoOB3NxcDAbrGYOSThIwEia8gy5mMpoWFX8adw3XbLuGafdO69Lp3H3d8QoVZUStpD7XLYWv/GzXgEXSJjodTZNpI6WFVdBoWSKt0+jwdRfZeCcHhgBSf0slb7elRKRJvkwbMgiSrhUVnN1E+m/XCAgQ99NgaKPqL+snOP6JxSZz4K/K8v42qrTsn/EPSgdMsHihtxr403jCrN9giKUcrMS2hITAB7ufYrff920uQOZW5nL7b7ejfVTLocJD+On8mBY3jYomhVcfH2DuBjhzh93tlf7bTdz8hXqBQTT1a1fCl2Zf7xeWjqpgFRjq7W/jqYChHrbfAulfdevw5zY+R/+X+/Pg6gcttnt7Q432BAb0uKnd2j2HM6U+e6X/ugdC9CKRfGiF1Wmr2eX9JPl+yynrvlq9BcnBydww7gbCSxdjMGowhJ5ukYAosREaT1EpEHkGKkWFn07MRVLXp7Ls3I8JKD6G0dhGj+MuUlpZTY37CUqMx1Fy/4ATn/aqoB/0Ev/N/R2+j271jNWoNCQHC6neCo99na4Ss4ZpTuLlBYx6BiZ/0u54SQ8x1Iu5pL55Pjk6XKz8d6XizyvEi/CR4RgaDRgaxd+wqWfjztoZMOXzHs0vXR1X9t9H1z7KjPdnsCK1ubJep2uWos9oatl5aNkhvlj6BaUnSgEoTSultqwWgCVezzJ3dwFzAq8Wa0we4Q68AkmXUblB1AIIFL7cP1j4Xq1bltU+f9vTReJilHZwb/vo7DHO8FkZ+LMDLTM2TZg+wGJCC+GPabD3QccaJRH8Phl+Gw+AVq2lxlBJvaaAgoJ2Djkm5AknRk/EWysyzSwqOAGCx0PyP8FNlqJ3mQ0XE390Bm56sWqVX1mISm2jR1NcHDvn30PWFPFpY67423GryEhpELpmRqOR3NzcdmUKJZ3AMwoSriQwLgmtIQDvmsEMDRmJ3qDv8qkaahrI2ZlDdZGYFLSS+gwcJwLup9qbgpMICIBhoauJWOctqhtOor0ecJeuuJQr1l5h/rm+HjK8v+F46Muk1xzssW3Sf7uGojRX/WW1Lh6BfY+JfrgtMFV1nnx/q6pA76bDy88yjc8U+EsIcIFM1FMMk5Rre+81akXN8389b/754RkP46fzMwf+vL0REzgHSClL/+0mY1+C2X+K+0THgb/sbPHvUePVcEGNqL6V9BxtAMxZA0Pu6dbhk2ImAbD8yHLq9c3BWC8vqHFPAyDOrylAVZ0JaZ9C1QnzOKPRuYG/Xu2/RusLL/38hR5Ztftxm0l9mhByyka8vXrh/1cvxKRAoRqg4qz/noV7oqgu6onc55qH1/BUwFOUHhcncVN0MPVbWNr6/dfV6RX+650AA24C3+RWu4aGCpWlCo/95s+4rlDXWEdeZR6llSLQ0NtlW3sNyf+EhUfAt7msb0zkGEAE/trq2XgyRoOR2rJa3p/xPs9Hi3daLyUYXX00Abq+3zfeZfw3bw2kfW6xaW/+XtacWMP+/P0W26OaRO9M97g0rZTDyw5TkSUmIF8s/YI3hr0BNPcd1/jGwOyVkHiN/a5BYhumfgODRJuIaD9xs2u11gN/B4vE38aAFj3LqSuC5cPg8Mt2N9WZOMNnZeDPDrQX+AsJc4PRz0PMuQ62SgJA7PkQdyEAIZ5idaxeU9BuxZ8p8Hd6/9PN2/Ka3u3DZeJJz1FUqDUqInyF5E1hVRGFhwo5/MNhc/ZWdzH1+KtXi9ROc8Vf6HRIuNpCMkRiOzQaiPAPYMb+/Xw0fSNqlZqd7+5k6+tbO32OopQi3hr9lvmYykpo0LSo+Bt2H0z+1B7mS6wQEAD5VfHkeV0GPkmt9pt6wOVVtV748AzyxCeiWbq3pgbSQ95if+wteB64BX5KFtXYEocR2ZQAm5lpZefo52Cy5QQu2jeafv79iPSxzJw19YRr2d8PILVEyL5YVPyVH4Fd/+5xP0dJ+4SEQIT3EfqV3QUFm6yP8bLs03n1mKupqxP9iwF8vRugYANUd2MlTeIUOlvxFxmJiP4rcgpoExQVhE7rtuLHhOgJhHuHU15Xzqrjq8zbvbygWisCfObAX+Fm2HiJ8M0mqquhQRR9Oq3HX6/kx4Hwxwyru+L94wGo1qbZLPC3Om01a9LWUlpTxqDgjcRu0gq5O4ntWXUmbP8n0JzwWetfy9hrxxI4oCnJtEXgb9Pzm3hIeYgd73Suwt032peI0RFUNgWOtWoP0UJCJv/aB79kGPsyhLROVhkSIhaMKzz2dSvwtzlrM+HPhfNKw0gAPD2B3ffB0bd6YLCkO4wMH4kKFXXaXI7kdu5mFh4u5Cn/p8jbncewS4YBcHPEl8zZk8Hlocdgy/VtJnhIbMiBp2Dz3y02xfqKd6L0snSL7ab5p8lfx10/jnvr7yVmkkjKGHbJMMbdIGToTRX3vvLR2iuJ8hGBvwZNMXnFNa32p9ccAGBEZIvAX30p6GtB33q8pGfIWZ8dODnwV1/fLCkRGu0PybdCxOmtD5TYn8F3wqinAQj2FIGmercCamqaZR5aojfoWXl8JQCnJzTfs1YVf3/OgbVL7WZ2n2bSxzB7JQkRYjJWVl/M5pc38/miz83VXt2l+K8UfAqPU4cIGJkDf4lXwQT5Um9zDHr4IRG2XEtYU3sEU5B862tb2fjsxk6fyjPYk5mPzqTfTJF5XVEBQ9Nf46GIHSwdJH3N0QQGQl5VP7aoPoCIua32t9UDzhpCtrUUAI2bD5gWTSQOo92Kv5BJEDrVYtP4qPEcu+UYX5//tcV2U+CvZZZ0dUO1+e/AMvB3EA48CaWW2Z8S2xIaCn66AkZqn4HCDW2Ou3DohagVNb9e8isalcY8wdZowFOVB79PgYNPO8hqSZdpqIT9T0C68ElT4K8tCTvTIsuo0B9FywGJ7TDoxWJFN1ApKhYNFL1vvzv0nXm7t3eLij//psBf8ESY8jWEND+fTYEpLy/Qyo/RzhMyGQLHWt3VL0C8d9ZoT1BS2nXFCmtcv/x6ZnwwnWLdDmoafTCEz5f9bu1F1XGoEQ88U8VfaW0p0Cwz1zLw99cLfwGgr+/cvR515Sgu+/MyypqEDtxVOijeAWU9V7CQdI2FAxdybdh7JOT8q1uBP5MspJtBRBa8vIDDL0H6Nza0UtKK6mw4+jaU7jNv8nTzJMFvEAAHyzqXIOgV6sXIK0dy/rfnM++5eUDzvCTB8zdI/Z9McnIEQ++BKV9abDK9t5woO2Gx/eTAn1qrtlD6mnT7JKb8awoAH9YvYdOAWRgM38Oe+6HskH3sl9iOQy/BTqEa5K/zR2MUjVePF1iW8VbVV1HCcQAmJbUI/PkkiGrgwXc7xt5TiG62NZa0x8mBP9PCt5eXlBBwJUwZ7/VuQg+rsLB11cK27G2U1pbi5+7H2MjmCWKrwJ9Ka5ZbknSPQfFBkAYGDPQ7vx/RE6PRevVsFaN2+Z/0K6/m4AViJSzAw7oOkqIoBAYGokjZyJ6hUoMuFNz8CQ2F/fubn3+LP1zcJQlX3yhfpt3T3BOwogJ0DREMDY4gzMsAm68WWfb9/mbrq5BYwSQh1lb2u7niz4rU58lUVUGDRpwoJ/EWkuK61vvxZKT/dh2T1IrVwB80ZcgqHUrpmhJmWr7bHC8RL/L+On/LZ2746bAgBXSW1WYS29KvHxwrGcVtq4/y3PlRtHUH3134Lk/OftI8OTcF/vz9QXHzglHPOaQHlfTfbqJygz33QdRCiD230xV/Yw3Xwc5omLfZMXaeCvw5HSpSYWn3+rcvSV7Cm9vfZNnhZbx+1uuoFJW5xx80V6DhGQmx51gca5L5dFa1X6/139PebXNXlE8UakWDXtVARmkOEN3jX2d6N3JvCCOzZjCqGcto8+Es6RkLmheHTe8gJbUlfDj7QxqKVBBzKSdarEUven8RdeV1DFoyqEu/prK2BrSizzXrzgWtv0P64tqSXuG/tYWw5WqInA+JV1vsGho6lIsGDeXZ5W30rO4AU+BP3SgCf97ewIKDMlhkbypTYcs1MOYl8B9q3nzpiEt5/6s8jCWxVFc3VWC2g2eQJ4veWWSxzSRZv9XtM+IWW8nq70O4jP+GTG61yaRU0FHFn75eT8amDDwCPAgbHmYxNlPZQK1vAeHqmbDvEQiaICqAJa5L9s8i6XTU0yiKgq8SRTGppBVnAc3JwLuyRKKMtiGYkUmn3rqAM3xWBv5szF9X/cXg6MEW20xBorAwUA4+JbJzpy3r081mXZaibbD/MRh4MyG+IwFoVFWiV2opLNQRd1Kf92DPYO6aJLIWNCrhLnp9c6agOfA382cHGN9HqToB6V8zPGoOc77PYXhSIINn2CZtuWTaIorTK/j7oEEQcI4I3lakwrYbIek6iBYviyqVitjY7sk0SU5irqjqCzsAWxLPZv2hnUSmf8HkIa1fCruCKYPPxweRWZ/6P7FBBv4cginwN7T2ZtjjD8Mftth/04SbuHT4pZYVXm1QXd1c8WfKxu4J0n+7TrsVf3sehH0Pw9kp4JPY7nmsVfwlBiay45odFNWcFIHQeIBva5lYiW1JSACD4sGRnARyCyAiwvo4DzeP5koiTpLU0QbAoNvsbyzSf7uN2h3mbQVv8cxtGfgzGi1j9kZj8yJLyYC3CQtTO9jYPk7kAtF/r5vM7DcTX3dfcitz2ZK1hdOiT8Pds55aN3HTzFKfVnBmfz/om/6rVqkJ08WSXXOMjIrj9DTw16BvoKRW3Cj3xlC8vGR7akdx4ZALGRMxhglREzgWeAxfjQjotAz89Z/d8XuridqyWn77528MuWAIlbW1zYG/IbeDyt3W5tud3uG/Rsj6AbysPwdN7zjdqfgrqxUvPqqGFhV/nlHdMVLSFfyHwYxfoWVvL+D+mf8i5R0oqxGB3IROtAkvyyhjxW0rGP634SSdncRjJSOoS/bmMs9fQRdspwtwDVzZf2P9hF3tVfwZjdBY18gHMz5gxGUjGHz+YP56/i/mPDWHyLGR1CsiMK+PWgoTzgHPnifhSOzMpI+EklMTozwWcTijCEO0n8WwrJwGAion4kUIfi135a2CyjSIuwA0HUT+ezEqleOTS2Q6i40ZGDQQH3cfi22ajA/5eEkIQ2IOCM3a2lypA+8sGqvEy2PFUVF+3BTMq9cUUFDQenhCYAJPnf4UT53+lHlbQQEYDEJSJ7Dv9w22P5VpsPMO+nuuQtcYTla6Flv1Oy3URlIWNpCzk+dz4/gbSQxMFIsz+auhtlnnxWAwkJ6ejsEgdeBtRVgY1GvyKTVkUVRThL5eT1VBFUZD527uzvd28u6UdylKEcGDigo4HPkA3xY8SalBgXOLYeRTHZxFYitiYpr+VS+HrOWt9g8IGsCE6AmteodZo6LCSGNT4C8m+yvI+L5Htkn/7TqmiVd5eXN2rBnfZIhZAoplbth5X51H3ItxbMxoluy1Fvhz17gzKmIUc/rPsTxvbSFUZwlZPIndcHMTiyV+7vlk7DvQ6eNaVvw5Eum/PSBwFGjFjNn0Plpf3+yXJioqmqtzA4bMh8h5DjTyFGDIv2Dcq90+XKvWMi9B3JPlKeLzVeVeTVzBdQQ2DCPUq0mf8K+/wzfB0Chk8OvrYUdTgZGzKv56rf9m/Qxbb4A669q4MT7xAOTUHm/3NJnlmVz303Wklaa1OcaUBKNChVtjIGOi/4Ttt4rER4ntyfkdMr4F4Lwh5/Gfqf9hTOQYzvvqPM75TFTMZmUJ/zGRvz+flfeupOhIGyXTTRQeLGTX+7vI25NHfY073jWDiPPtLyrR+l9mt0uyF73Cf92D4cJGGPOi1d356h2khbzG8Zqd1HSxLZSp4o86sS7nqWsU/ajrS3pgsKRDtP7iPcRKIMc0P2lTkeQkPjv7Mw58fYCSYyVU1leSZ9xHqfdfROmOQnmK7Wx2QVzGfw8+B196Q+le8yZTYmFhdSFV9c2Vl6GhoFaL529REbj7uHPmK2cy8sqReAR4UFNcg9FopKq2DoNKNB4PDYkCv8Fy/bw3oAsFt+ZYyP9FPcfItPfxqx1uMSyoZiKTD23kMu0yy+OPfQCbrwRj314rcIbPysCfjTG2iFhkZ8OaNbBu/1j8dIUkheyDYQ/A4gxwk5qfTiFkClxQB4lXoygKc/rPYYh2PkbFQGFh505hUcGpADV5cOAZoe8v6TqBo+H09fiOuAy1WixO7f4hjRfjX2TvZ3s7Pr4NCo+UUFEmHqoWCyJh0+H8Kuh/hXmT0WikuLjYwn8l3STjW9j/JGFhoGnqmVBWW8avt/7Ks6HPUlXQOdmNmuIaio8Wo6hEWnR5hYEjEY/w3yP/plZfJypS3GXk3VGYqqFv+XU3hrnbenSuwrJaDCqx4uJ/6Ek4/n6Pzif9t+vodBDclAjbanIdfyFM/Qa84y0251bmkl6WTlZ58wGmAMPJMtlWOfAkfB8N1RndtlvSOZKT4d5pCxmSN6fjwU2YAn9+fsDRt+DXcQ7ppyH9twfo66HsADSUo9U2VcTTWu7TJIEWHGyUfeBclFsm3MJX533FHZPuACAy0J9h6a9zZtqeZkkgnwTwHwFqD/btg+uvhz/+ELsS2y/Othu91n+Lt8GRN9qs1Hxm+hvM2pNGYNbF7SYjXvTNRby5/U1O/+j0NsfkV4lEQz9tEAoqkoM2weEXu90XUtIBex8Qyi5WCAgQz0mDATIzoTi1mCf9n+THq35k3WPrOLG2/WBs9GnR3Fl4J6OvGo1v8XRm7D/Au/O+tcdVOIRe4b9K+7Lz7+x7hX1xN5Ln/6NFJWdnMAX+NHqRQOPtlg8/DYA9D3TbXEkXsJIIGBRZTpH3GlLSSzt1itrSWiJGRzD2urHm+6kYNIwuuxjWn2tLa10Ol/Ffz2ixxqo0tz3y1/njr/MnwjuCgurm6gq1WqyhQnOV7vgbxxM/PZ6YSTGMumoUEaMiyC4uMx8T4dUAtQVNbSgkLk1DORTvNCdPmNZgy8sth5me1a0KVofcA9N/sgge9kWc4bMy8Gdjfkz5ERBSZrfcAs8+C79vGcz5X5XREHG+k62ToFKDqrmK4ZdLfuGBxOV41se1CvwdKjzEL0d+schSASv9/SpSYNddkL/Wjob3Ydx8IGQyGq9AShL+y85+l7ImfTdeIV6otd2TozIajbw//T2SN74LqkZW5/zA+vT16E0vmIpK9mS0F8c/ht3/ISzUgKZRTKTK6sqJmxbHmOvGdLrP36TbJ3FH7h0EJorgXkl1OSjiQ9Jf0UPJHmiobO8UEhsSFiaqnMtrvMnNaz0BL64p5sW/XuTxdY93eK7cpkaBCgrGeZthxBO2NlfSCTrs83cS4d7iQy+3Mte8zVrF3/92/I/nNz1PanGq5QlCp8KAm0T2tsSuJCfD78f+zsqsWzp9jKl/p58fQh2hNq9Py6z0CU58CsuHQO5KgDb7/JkWV+YO/Aq+8hfVThLbkfsHbLi4R4HyybGTOXfwufjpxHuTKZmiqormwNOQ/8DsP0FReOEF0XYgKAhuuw2WLu3hNZxqDLwJluSC3xCruycOGIA/cTTUadrtHbY+fT0AR4uPtjmmoEosevpqhCLC9qrbYFE6+Hatp5ykk4x4HCa8B4jAzu7c3ezP30/OjhzWPLya+ACxoJyWBkaDkcgxkQy7ZBjX7rqWEZeN6PD0nkGe6AI8qBaFt3hrS+DnESIJWGIf8tZA4V9Wdw0JET5c4bGf4+0X6LbCFChy0/ui0YCbzgMG/0v0pJbYD6MRvvCC9ee12vV2/Rw2Jc/gixMvdupUNx6+kau3XY1Gp6GiTkiYaAy+lEfeKuYcEvsTdwHM/LVV/738O/LJvj27uU9xEyf3+WvJ+H+MR6VRkV3UFJQ3eON24EH4NlQE/ySuTeYy+HW0eGYj5pR6pZacMst7dyxdVHO2Cvz5JkHUWY6w9JRDBv5szOaszYCIYtfWGpmb9BGzplZw1mJfpk01wp77zfITEidRuFnIgDRhqno4Werz99Tfmf/pfC7//nKL7XmiP3tz4C9gFMzdBLGtX14kncTQCDU5lAesJSvoE3a6HePqrVcz+JzBHR9rBX29nv5LR1IcORRdYDGLv1jE1Pemip356yH7N2ymJyqxZORTMH8vQUGgNYimM1nFxQy9YCgL3liAZ3DXF5KNRiiuEZlDOrUOXd6f8MsIyPvTpqZL2kalEnKfIZ7plB5eKSpNWlBVX8Wtv93Kg6sf7DCLyVgdyKRD67g19GdUgaPBTy5+OQNT4C/z5IKHiqOw7WbIW22xOcxLpGjmVeWZt1kL/L229TVuX3E7hwpPWgSPXgRjX5aKBw4gORlWpF7N/zbeTV1d544xZWP6+QHJt8LidPByzd4hkiaCJ4pFSh/RCMcU+Cs+Sb3QtLjiFeAjVBY8wpHYkKoTcOIzm0k3ZpRlsL9yPW5aPY2NrZ/RFRXNvcZfeQVmzpQ947qMNgA8wkRCqBU0GujXT3x/5Ij1UxiNRhSa/+NPThQ1Yap28FEJyVZ3L0/wigG1LL+1C2EzzHLGP6X8xMg3R3LLr7eQszOHNQ+uIVwtnCctDYKSgrjsz8sYf+N4wkeEo3ZrP+H0xLoTlBwvobq6eRrp6V4D+hrRzkViH9afBzvvsrpraOhQACo89nU58DcldgqL+1+KT80w0XvTPQBGPgHRZ/fUYkl7KApELYCgsa12/d+AOwFY2/ACxTXWpZhbUny0mIyNGegb9C0qOH3Q979OSPBKnIab2nqSfXuBPxO5JSJBw93oByFTIfE6s7S9xIUJHAvDHwHfgQBsqfiWX8Z48LX6HPOQqvoqHsebVUMHEBzVIonfaJRJ/XZEBv5sTGKg0FrJzIShIWu5aexl3Dr7Pi6/HHSGDNj3CKR94mQrT3G2Xg9bml8EQkLASGupz4xyIUcW7WupP95S6hMQi5jBp8lm0D3hz1nw8wjCfMWqVWZx+z0WOkLjriHu/2aR3/803P3ES6O/zh+1Sg37HhITiBarJIqiEB4e3iynJOk+vkngPwS1RoW/m8huzijqeobW4R8Pc2iZCBzU1EC9UgpAgEcABAyHYQ+B31CbmS3pmLg4OHvgSwzOmQ21linwph5EDYYGSmrb741RU+FOYOUUpobNEpIQPQzCS//tHtFNH22tKv7qCiHlFSGF1gJrFX+mvmEtA3+mXkcnZ3hKHEdQkOj5ZjC0vWh9MhYVfw5E+m8P8B0oFin9hwHNff7aqvhrCD4TZq8UwT+J7Yi/FC6o7XHvxIyyDB5e8zCxL8Yy88OpHB7yNwD27WsasPNuOP4x6enix9DQZnlXZ9Fr/VdfD+WHodr6ymN+VT77w//Dvpib2nyGZldkY0S8v9TcU4OX1rrm9djIsbw+/3Vmef8DgDDvdKhIlQmIDsBf5w9ASW0JAxcO5Npd15I4W2jXp6VZjm2oaSBzcyaGRgPlmeXmHuMmjEYjny/6nO/+9h1VVZAa9ixrhg7h1QOfw9kpMOw+B1yRbek1/jv6eRh8t9Vdpoq/KvcUjhzvZKZTExcMvYBHR39EZMn5nZOsl9iOKV+IKvaTuHT0OfhUD6deKefZjc91eJqfrv2J96a8R2NNI6U1TRV/eh+LeUlfxWX8t/I47Po3FGywuttwkkRnZwJ/pRX1uDUE4akEiRYU498Atc5WFkvshd8gGHqvOam7X6hYHyozZpp7sO7MPIhRaaRBXcqg/i0ctaEUvvKBrf9wsNGOxxk+KwN/NiYpMAmAjAw4XDSBtbVvNZeZe8XCtGUw5mUnWihh2AMw+gUAHlz9IBOWeXMo6h4KCy3nYKbAX6yfZca7KfAXEdG0oaES9F170ZScRMxSiL+EmKZVq7zyQjY+t5F9n+/r4MC2MS1iuvmKwF+gR9OK2JD/wPj/WoxVqVSEh4ejUslHYo8x6KGuCPR1hHqJwF92WQGZf2Xy9QVfk7Gpc/29Vj+wmt9u/Q0QVUUNmiatcJ0/BIyEYfebqxwkjiEuDv7KWMKKktdbNdh217gToBMVni0DQ9YwVRbFeqyHr/zg6H/bHd8R0n+7h2ni1ariL2AULM5qJZHTntSnacGktLaU0tpSwErgb+uNsOVaG1gu6QhFgVnD/+KpOVMoO/BTp44x9fjz9wdSXhOV8Q5A+q/taEvq0yRVaPJ5iY1Ru4uvHpJTmcMDq5t7S02OmA00Bf4MjXDwachq7mNl6r3rTHqt/1amwk/JcOR1q7sNRgO/Vj1BWujr7E+psTpmb77oQz4oeBA6TdsLkomBiVw/7nqGKKLf1Eyfm8TvltiHrTfCl97QWGV+Ly2tLcUrxIvwEeEkJItKyxMnIGV5CqvuX0V1UTVrHl7DO6e9Q/a2bP474r98Mv8TDI3NC9ZGg5G5z85lwi0TqKqCavfjVOgOmN95eiO9xn/7Xdqm/Fu0bzTBulCMqkZ25m/pcjzdJNnq5QUUb4eV8yD7l57ZK+k2UZEqknMeBuDlzS+ZpZLbYvilwxmwYABaHy35TRNMrcEH7w3jYc+DdrbWubiM/9bkij7uhZstNhdVFzHv43lEPR9Fg77BvP3kwN+WLXDoJJGYGGUi83YX8k/33fa0XGJnJiSJtbpq7QnWbhIP23WHxfpuQOMQywC9oQHi/wZB4xxtpsNxhs+6+Kd87yPBX/xxZ2ZCg0FHZfjVlovT0QtlZZiziV4EMUsACPYMprqxinLPndTXC/kcExllIkAR4xtjcXirHn9broEvdCLYIekeyf+EMS8QHy50V4vriljz0Bp2vruzW6d7d/K7HHxpBQBqbxEwMk3+CJsJ8RdbjNfr9aSmpqLXt24yLekiqf+Db4Ihfy1x/rF41wxCWxdBZV4l+7/cT8mx9qvBTJz5ypks+O8CQPhlg7oUaM7elTie2Fg4UDiFZQeuFzJZJxHm3SQFWZnXal9LDlVt4ljoi6QZM6H///W4clP6b/cwVfzl5IjKMDNqd/CMbLWQbQr8tSf1eaJUrEiHeIa0rn4o2tJmjxaJ7YmPMxLpk0JRdufeTUyBPz/fRth+Cxx9047WNSP9t4fsuR9+GQ1Go9XAn9EoFleCPDIZVnkZHHrJOXb2ZfT14tlWntKj04yNtJQ9u2iMeAfavx+MqGFxJox+3lzx16o3ihPotf7rGSUSAcNmWt0d5hVGoC4YFAO7sw9g7fImxUziz8v+5Nm5zwJ0KHNuqpDP0Z4nfrezqzT6Kt79IWQKGA3NFX81JRiNRurK6wjWiReX4mI4sOwoax9Zi75ez8CzBzLzkZn4Rvsy5T9TOP2Z01FpmpfKVGoVo64cxZDzhlBVBfUaIRXUTwMcfRsqjzn6SntMr/XfFiiKwsz+MwDIcV9lXqfpCKPRyInSE5SWi4CElxdiLadwk+wl5giOvAnbWveh1mhghG4hflVjqGqo6rB3/NjrxnLRjxehKAp1NSp09VEEGENRavOgvmOp0N6My/hvwEg4+0gradUAjwC2ZW8jtzKX7TnbzdtNgb+cHNi+HR55BB54AOpbdBExJQn7+gLb/wnbZL/GXoHRCH/MFPcMiPAJx1cdCoqBb9eLgN/aIzsAiHMfaXmsLhQmfQj9r3CcvU7CGT4rA382xrTwmZ3ViJdbKdFRUsbDlZkUMwmAUu9NreQ+TRV/MX7Ngb/KyuaFTrPUZ+h0SLwW3IMcYXKfpl+YCPzVqYo447PLOfutrmvsN9Q0UFdeR21pU68Fj5Mq/tqgomXUV9J9/IcLf9CFcf6wc5ix/wCjC55nwFkDuLf+XoZfMrxTp4mdHEvCXJE00TLwF+ARADvugD9mwEnSERL7YqouyMqCxsbW+631gLPGIf1yDsTeylfFf8Fp70Lo1B7bJv2364SEgFYr7mVey1tmNIoF7JMWsaJ8ougf0N8sf200tg78tSvzecYWOHOXTa9B0jahQybyt+/y+Wr75R1mwBuNLQJ/fgqcvkFUVTsI6b89oKFc9JdqKDUH/lq+y1ZWioBDo0GLV+mPUC+T1GxOQxmsmAiHXujRaVSKytyvCmDSiDA0GhHIzc1TRLDKM8qlAn/QS/3XzRdGPAbhs63uVhSFkeHifbVYs4cMK2IVvu6+zOo3i8kxk5n/yXwin4+krrG1AsymjE2sSVtDfoXwvRK/S2H4Q7a7Foklg26Dmb+Cm4+YMwBldWUYDAaeDnqa32/+yazaE33ZLK7fez1eoV7ETIph2r3T8I32ZdLtkxi0REiV5e7OZfWDqy0Cuy0Df0mUiCTg4u30RnqF/266An5ue/44q58I4Bf7rONYJ+OvJbUlxL8UzxnrdeiVOhH4i5gL55dD/8t6brOkfXJ+hSOvWpU8jolWSM4SAb9Xt77KwYKDnTrl5KDFzNmTyYyy72HxCdFXvI/jEv6r8QCfRHCz1B5XKSpmxM8AYNXxVebtISEiwNvYCC815aJVV1tW/Vn0Hc9b3arvvMRFURSozTUXxCiKwqjwUQBsTt9JQQFsyxaflbMHjXGamaciMvBnYxRFob4eNNWH+PzcAJLqHnG2SZKTyfoJvg2HzB8YHjYcLzcvGtTlVHjsp6ApwUtv0JNVLhoftaz4M2WRBQSAu6kYIunaVtKRki5SdgjWncug2sMA1GuKKNZF4h/v3+VTuXm4cf3e62GhCBoa3Zsq/jwCoK4Yvg6C3ffazHTJSYRMFP4QMJzkJiWjlBQwKirUbupOncJoNGLQNwf1KiogonQpV9Tt4JnTn4G6AqjOAEV+hDmS4GAYGLaHl+cNoWzH/1rttyYFaY1Sg9CWjA+MaXecxL6oVM3B3KNHW+xQFLHAsu1mi/GjIkaRenMq313wHQB1dZirIDoV+DOdW+IQEhLExLq0lA4z4GtrmzNt/fzVEDxBZPBKXJ8xL8KCg6ANMPvziRNN1UUNlRQdFcoJGu9QlAWHYPjDTjO1z6INhJFPQ9z5PT7Vx0s+ZlT4KH695Ffc3SFJdJDg0N4K0ReuscalpD77MsPDRKCh3HNPu71Sfd19zVUNO3J2tNp/1x93MeODGRyo/RPglOg95SqYKv4MRgOVDZWMv2k8CWckmH0np0RH6NBQVOq25xNrHlzDusfX8cGMD3h96Os0VDc0tSAQC5sNwZNhxq8Q0vMkNkkbaLzAre0GxIuTF3Oj90rGHfmR48c7d8rMcjEX8VYHoTa6yx5/jmb823BOodVdUVEQUj6XEbqFTImdgqqT8/2TkxElDsJohOosqMlptWtmvAjKr0xbad6mUjUrp5W0EILatav5+z+K3mPTgFmsr3sN5u+C+VLys9ew4CBM+sj842nxIwEodd/Fw4/qKdHuAuDimScF/o5/ApuvkRXXdkKumtqB7Gyob9Sx6sSV6KL6vkZtr0MbJGRAVFo0Kg0ToicAUOK10ZwlnVuZi96oR6PSmBezwYrMp2zMbhuMesj8jhC9mETVawrZuU1PZV5lt09ZViYWmBvdmir+dIGgrxUVabrQntss6ZCYGCGdUlcHqYcbSV+fTuFh6y/5LSnPLOcx3WOsul9kh1VUgJvenyTvUQwOGQwTP4CFqfY2X3ISigLBYR4oGCguav3sM1f8tSP12dAAlWqROn+G9jhsulwE5CVOYcAA8W/KyQp1Q+9rJYl8MqYJtkbTnAhzvFSsuLQK/BkaIeN7KDvQI3slnUerhQVjfmVq7Ofs3dv+WFO1n1YLOk2NkC6U9DpCQ8XnrsEAO3cYYe1iog7NItgzQ1S4eIR1eA5JN1CpYfCdbcpGdoUR4SPYce0O5iXOA2BoUwFg5dE/4MdEqlO+pbxcfB6b5Jol3WTL9bB2SZu7zYE/j9aBvwZ9A3f/fjcf7/kYvVFvVpDZkLGh1XlMPapUNWLuMbR4Mey43QYXILFK4WbYeTdUpKLT6Mz9F0trS5n3/DzGXT+O+Hgx9OjWkmaFmDZY8NYCrtxwJZHjI3H3ccfN082i4s/HLxEi54FHeLvnkfSAca/B6eva3B3uHc7cATNRG3VdDvz5q8SD1MsLqDgKmT/KeYkj0AWLthFWEgKjmroiza/5hJWXrWRg8MBOndI0LwnxLYZjH0DpfltZK2kPowG+jxZtAk7CFPjbkL7BoiK+Zc/pUaIgzCLwl1l7iCLfVRTTtN4jk717LaaKvzLPnezNPoReXY2H2ptBIQMsB+avgdS3QeXmBCv7PtKDbIyiKGRmQk5lIsuL3kGJOtPZJklOJmQizN0IkWcAMDlmMgAl3s2BP2+tN28teIsnZj+BWtVcpWR6mTQH/lbNFfITkp7hNwguqCV03HOsOiuX2XvTyH39O54Lf86isXpn2PfFPna+u5PSEhGYOD1uAa+c+QrnDTlP9K2aswoGWlayKIpCTEwMiqxG6Tl1xbDhYjj6P/TGRlYlD+f3EeFs3ZrHe1PfY/PLmzs8hVFvJOmsJAKThDyrzOBzHbzCk7jh54NsKb661b4bx9/I5qs2c9vE29o8vqICarUi8JdENhz/sMcv89J/u4+pmqR14O+eNgN/RqORRkOj2S+9vJrn7Q/OeJBtV2/jurHXWR5UXwzrlkDKq7YzXtIhi/o9wN9H3caePe2PMwX+/P0Rvf2+cIeC1gvY9kD6bw8xNMKx90VgHRjb1CZu23YFhvyLLaU3U1gdTUJCm2eQuDCmwN/2lP4w6C4yK0UwKjy8hfKIE+nV/ludCVVpbe5uDvztJuWIZbLT4aLDPL3xaW5YfgNqRW0O/G3M2NjqPAXVIvBnqAwBwKN+P1TK5DW7UboXDj4tgjjAv6f8m8dmPYa3tnkSYQr8lT39Fl8s+aLd03mFeBE1Loq5z8zl75v+DkBlpZH6poq/4A5aSbgyvdp/T6JfP/FvZwN/GWViLuKLCPx5ewOZ38PahVDRs36tkk5QXwIle6CxqtUuU1JLfqZ3l/423zvyBBuSJ1Pp8zL8dQVkL7eRsa6Jy/ivSg2D7oDIBa12DQ4ZTKhXKDWNNWzOal4DMkmVJyfDzU3LckePinUCgIp6MTEJ8vSBrJ+hrHNyrxIXoGgrHHnDnEQ6LmocSxMvIbrobyioGVB9ORcMPc9ijR2A8W/C0oJ2q7v7Cs7wWY3Df2MfR6VSkSkSiGQmZi/BNFkr9t5glvr00/lx9RjLhe116+Crr8T3ycmIxRZDPRha93OQdBFFBYoKLTBlZBivuUOhfwJjp3uib9BbNFfviL+e/4vStFJKThfZJZPix9K//9h2j1GpVASZmuNIeoaighOfgZsvmsSrqFCnUadUsCernCUvziN8ZMcZsf7x/lz4/YXmnysq4Hjoy6wz1rCw9ALiSzaARwSEz7LnlUis0FJK7mSSgpI6PL6szEiNm/iQrBn1Ivh9AWrPHtkk/bf7mCr+UlOFbKe6AzXe+1bex5vb3+TRWY8yyf0awDIg76/zZ0ykFc1+jRdM/Ai8ZfTBkZT1f4b/vW0kWy8ECtqaZzT390Pco9jzwcsxOoLSf3uIohayvEHjIGYxY8fCd9/B9u1Q9n9zeHbFHABmyY9L+7L5KlHRPLd14KcnDBokZLG2p46gMGYEhzeJ7a7S369X+++MH9vdPThkMCpFhV5VTUpGEQ0Nwbg1JaLvyRPZFMPChqEoijmJdH36evQGvXlBq0HfQHGNqB4yVoiKv6JJR8w95iR2IPZcCJsBHqJs6P7pzf1q93yyhz0f7eG0p5aA0ZOCfuNYcGHXFxnLqurxqk1E7VNE5NEXRJXC2UfAq3dJ2Pca/81fC4V/QdJ1oj+nFdSBJ9gf8wKNqgoqK9/pMFnUVPHnpRcLdp6eQORZ4B4K3om2tF5ijWMfwI5bYc46CJ1isctU8ZefL2ToN2avZmfOTi4ZfgmhXm2rNh2vPEiJ90bSlbkw/Ufw6VylYG/Fpfx31DNWNyuKwsz4mSw/spyNGRuZFjcNgIULxbvNmWeKViIxMZCRAXv2wOTJUNUomvxFeOtgzVnQ/wo47T1HXY2kJ6R9AodfEs9Tr1j6B/Tnm0s+5q7dcLAA3jzzfWbMsHKcoohK4FMAlcrx9Xey4s/G6PV6MjLgnEFPsTj8SjDonW2SxBopr8Eh0U32tOjTmBB0JjFFV1BQaL26bMMGePZZsXA2Z474kEKlgTlrYNInDjS8D1O0FXJWoNHAsGFQFDsKt0XzcfPoWrn3OZ+fw7lfnW9uCuzv32Jn/jrY+5DI8m2BXq/n0KFD6PXSX3uMmx9cUGfuexmkExnO+zOLOO2W04ifHt/lU1ZUQFroq3xd+i/Sy9JhyzVw0PoLpsS+xMbCzPgPCa/5qOPBVsgoLMagrgEgyi9aBIR6mPUk/bf7REeDh4eQ483IaLFj973wx4xW4/VGPQXVBWzN2ip6iNHJSlyNF/S7VFTcSxxGzJhpHC6dTnGxkKFvC4vAX/TZMOUL8HRM9pr03x6iKDB9mciUBQYPhvigI8S6r+S9d+ppbITEROjf38l29nWMBiFbb2M8PDBXa27aBOnp4ntX6e/Xl/3Xw82DIzce5byjFajrgs2VRMU1xXyw+wMAhoeKqsBxUeMI0AVQUF3A2hNrzecoqhFVYQoKhipRGSbVK+yM1h98EkHj0WpX2YkyTqw9gZexEo2bQnriLGKXWElW6oC6KnemH9jL28nZaANGQPjpoO19VQq9xn+zlsOuu9vt/aTVNXI87CUygz7i4NHqDk+ZWSHWAjwaWkh9+g2C/pedMovPTiV4Egx7CDyjWu3y9xeBWKNRtNm56ZebuG3FbWzN2truKSvrRbmYmy4cohaAb8cJqb2Z3uK/s/rNYmzkWK4e3VxUERAAf/ubCPqBpdyn0QjVBjExCfQNhAnvQL8rHGu0pPskXA0zV4C75XP07rvh4YexHvQDyP0TKtPsbZ1L4AyflYE/O5CZCcPDVhLFMlH6LHE9jr4FKS8DokLhvdk/k5RzL4UFwiW2ZW9j1fFVFFQV0NAAr7wieqbMni3K0S3WqaXmtG3YfgtsupwHVj3ASv9LqHQ/ws6dXT9NQL8AAkbEmqsb9pStY92JdZTVlkHeKtj7INQVtTqutrb9Hg+STqIooNaaf4z0E5l5uRUFFg2c22Pvp3tZcccK6spFNW1lJdS6iVXrCO8ImPoNDL2/vVNI7ERcHJw/5HHmRD7JyS5TUlPCC5te4NG1j7Z5fGqhiC55GkPQleyA0n02sUv6b/dQlGa5T4seRjXZQorspOSlcZGib/HW7K2tJHjL68q55ZdbeGHTCxhl/1uXQKuFQckGVIq+XbnP0lLxr5+T1i2l//aQsJlioRvRc/Pi8W/z2KzZpO0SzR3nznWmcacIp70L8zqWMu8OM2fC+UMeJfrofA4fEO9FrhL4g17sv+VHRNVJTdt9ifsH9mPQQDGX37dP9Imb/eFsVqSuAETFH4BWreWcQecA8Nm+z8zH51flAxDoEYSCGnd1FV6Fn8veU/bE0CASPJv6tOVV5rErdxe5lblMvmsy/6n6DxEjw8w9piySnjqJKfHJywtIvEpUj7ZRiebq9Ar/HXADzNtiNUhkon9Af/yIwahq4IddHUuVmyr+3OtaSH1KHEfweBh2P3j3a7VLUZqr/jIzYVioeM6aKq3borKpSszfw8e2trowLuO/O++CDRdZ3XXxsIu5fMTlBHkGUa+vZ1v2NtJK0yzGjBwp/t21SySjNijiXgYHBEPClRA23X62S2yL/xCIOB00zYpOeoOeHP0+9qo+pNHQ2PqYhnJYOUes00rsgoxY2BijUXxAPbD6VwonpTnbHElbTPkS5jRnZJokVwoKROb7E+ufYNaHs/hi/xfs3Ste8AMDWwT9GqtEb7/M9mViJF1gyH9gzEt8e+hb1pR8inv5Hirf+4p1T2+iONV6k21Do4Gakhrzz9WF1VQXVZsDTD4+cP3P1zDt/WnszN0pevuduRt8+7b0g9Mp2SWqK4EwH1HxV68p4NMFn/LZ2Z+1c6Ag5acUNj23CbVWLLYUVlSgV4tZdoRPhOjPKSuHnEJAALxz4EOe2vAFR49a7qtuqOa2Fbfx4OoHMRitV0/76wcy+eBmLtB8CuvPF/JoEqdikvu06PN32ruwOKNV8tK4KBH425e/j6JykVHt5SX2HSs5xstbXubJDU+21q7PXAbfRYs+DRLHcewDHhnqzvCwle0G/ix6/G2+Bg487QjrJLakKh1KdgOgj7uCd3c+zbGSUWi1MG2ak22T9Ij582FAxFEGBGzgeLpIrHIVqc9eTd6fog9U2d52h40eDaWeW7lv1yXsyNlBUmBzFYmpDyCIxc1JMZPMLSQACqpEhVKQu0iCiw7KRLXpIjjxqQ0vRGJB2UH4PsbcU/i2Fbcx6s1RfLr3U1Qalfn9JNqQTtJfH3Hw5042hWuBReBPYn+84oSctVrX5hBFURgXLDStl+37HYP1aYiZs5LO4tLhl+JVJRqpenkB22+D72PFIrTEqZhaJmVlNT9n9+a3/6yu1ouKv+ke2+FLH8j+1a42SlpQugcKN1nd5a315oqRVwBw/U/XM+7tcby7812LMUOHinYTublw8CA0qIUPBnn1zoSKUx6jUSThNPHervcY9sYwLv/+ciKfi7SSIKyCca8LdSCJXZCBPxtTWCi0qDUahdCoUyfbpNfhO9Aia8zfH8IScijwXsmOHc0Nn2N8Y9jclMA7frzQogYgf73QL65Od6zdfZmoBRB3PnF+Io3ZXXuCgOwDrLx7BR/M+IDCw4X8fvfv6Ov1rHt8HRU5Fbw//X2+vuBrjAbx4bHl1S08E/wMJzbnAuK+mnprBHoECvmXgOHtThwkNmDrDbDhAgBCPJsCf24FVFYpKKqOZR0XvbeIW47fgkYn2tDmVYtqPy+ND95amZLpbLQR40kvG2pZIQbmvgt6o97sdydTX+VBQNV4RvnNgeEPQ/Kt9jZX0gGmij+LwF8bRPlEEe4djt6oZ3/RLqA5S9qUvRnvH2/lSJXoy+ku5ZMcilc8lb7zqWnwYe9eMQ+zhlnq09cIaR9DwXrH2SjpOY1V8Oto2HgxGPQkTxjMd4fuxIiKKVPk4rRDKN0nWgicJCVvC9RqCJz/Phd9WwYoqFTN1RCSHhBxBkz7AfxHtDkktTiV98ovZv3g8RxQf8r/tr3PJ0s/4fqx17NgwALGR403j53ZbyYbrtxgXuAEGBA0gDfOeoP/G3gXAI3qcJjyFcReYLfLOuXxiICBt0DQBAACdAGAUKUAyN2Vy6YXNhGorcS7JJPc9Pou/4o99d+zeshg/pv2T5Eos/chm5kvsYLRCI3VFgvJ1vjbpHkApKl/Zdu29k/5z9P+yUdLPsKjdDTQ1OPPPVD8/ajkOoHdqTwOf0yHo/+zutsU+DtypLnir6PAX41eBIvUnuEQMhma2o1IHMDMX2FRWofDTH3gt2Rtsdju4QFDhojvH31UyGMrRjXR9cdhWX848YWtLZbYi5Ld8IXOoi3PyPCR5u8HBg9snSDs5g1J10P4HAcZeeohA382JitLhYemgjmDf0Fdl+VscyRt0VgjNIT14mX/ROkJ3gmIZHPSGazdXEZGuQj8RfvGsKXpc2n8+BbHR86Ds1OEhrHEpowKFyLfBRP3s23+fQTdeilnvHwGO9/dycanN7L8huWsvGcl29/aTsjQEEKHhWLQi7S+iDERjLxiJI3+otGxn7/RPNEL0AWIBZn60la/U6VS0b9/f6c0Wu2TDLoDRjwJQIhXc8Vf0dyLuHDZhR0ernHX4B/vb/65qC4HgFDPCKhIhS88YO/Dtrdb0imSEo14aCo4kmIp1eCmdiPIQ/ieSULnZEy9N319EdIdcT1f/JL+2zNMFX9paSJxCYCKo3D841YSaIqimOU+D1WIXhudCvxFnw1nbBXSPhLHETYdj3nLOF5xGmVlzf3BTsYs9emvwHkVMOljh5ko/dcGaLxg+GMw+gWoyyc4sJFBg4RCxfz5zjbuFKFwE+z4p90kHJOSYMEC8X1UFLh1rf213ejV/usdLz6b2lkcdlO78d3RZqWKie5/x03txutnvc6PF/2ItoW0vTVi/GK4bux1nB5yOQAqnR/EniuSECX2QRcCY14Uc3VESw8QMq0A297cxorbVhB/Wjg7z/w3+f4DuvwrihozqPQ4SEljtkgEPu64z0xb0mv8N+1j+NILsn9pd9hZA+eioKLCcy+f/NhxEobRiGW/6qH3CsnmDvxaYgMUlajOrSu0unvsWPHv9u2Q5C8Cf4cKD1GvbztQX4eo+KsJnC0CUYFd79/Zm+g1/tsCU7LM1uytraq+rr8eYmLEXHTagV3cWNLAoOBBInFfJu33HnRhou+tV7OM75CQIebvw73DnWGVS+EMn+09T4lewtNPK8T67eMfQ+ZDmpTxcFkOPQc/9IPyAwDE+ceR4DsIo6qBn49/R16lWOw0lMZQWAju7jDi5IRQ737yxdCWHP8Yvotito9oxl7osRNUKo4aEhi0ZBBT/zOVy/68jLPfPpulnyxl0h2TWPDfBcx7bh5qNzWNtY0kzU9i0XuLqKgRKyKeflU0NGUHBnoEwqp5sKK1RKSiKPj6+rbOPpF0j5ilojk60M+/H0n+g3BrDOLoUWi0IuvdEqPRSObmTKoKqpp+huIGUfEX6R0JKg2EzQKveHtegaQdpgQ8ypfn+VKZdbDVvlERInD/29HfrB67uuhTjoW+SIX2sM3skf7bM4KCRHW0wQCpqU0bc3+HTX+DstY9GE2Bv6PVloG/4yVCLiveL97OFku6gpsbDB4svv/66xbB3RaYAvJ+fgh5Vwf2KpL+ayOSroWIuUKG/vsY7v1PAy+/DAOlsrljiDxLtBAInmCf85/4kssXbGXRIrjKhRSy+4T/tiFNDkL5pSX1RzrWzS2qLuLNbW9SVV9l3mYRXJA4FHPFX61IBB1/43iuWHMFSWNFU9v0DKXNavi2qNCLYEWwVxDMWQOnr+3gCNek1/ivTxL0uww8ItsdFuQZxOgwEVhYnflrm/0bq+qrOFF6guraBvO8VFbGOxivODgnH4b8y+rufv1EK576esg9FIOfux+NhkYOFR5q85RqvTcavQ+hfqeGPKRL+W95CmR8LxQo2mF42HC0ai3FNcUcL7WUWY6OhpdegvPOEyprQwYrKKFT4cwdEL3IjsZLbIpHOMz4CeKbez56uHmYvx8dPrr1MUfehOVDTpn+x87wWRn4szEGg5FyfTzHgt8SfagkrknwJEi+Ddz8zZvOH7YEgMN+r2LEiE6jI2WnkCUbPRq0WkT1w28TIPcPJxjdx3HzA88YBgYnA5BWtQ+DUs/x41BbCzo/Hf1m9UNRFIZdPAytl9biobn8huV8dd5XGPQGc/WCm6+Y5Lmp3PB084T4S6D/Fa1+tV6vZ+/evej1entf5SnHtWOv5fDNBxhTeR+6nDR+vm8T+oa2/59rimp457R3WHnPSkA0eK5Wi4q/KL8IMVGYsdwcWJQ4Hv/+o/nz+OXkFniYfc3EuYPOBeCrA19ZPXZj7dsciL2VeuOP8FMyHH27x/ZI/+0ZitJc9WeWb42YB1O/g6Ys25ZMjp3M9LjpBDeKIK9pIfNY6TGgjYq/3fdBWsf9PSU2xmiAHXdwxeTXAVi9Gu68E3JyLIeZ/DjQIxPy1zq0v430XxsTNhPiLsDX3434eGcbcwrhGQmhU0Vmuq0xNMKGC3A/9gxXXSXmJK5Cr/bf+lL4yg+2XNfmEEVRzNnpsfnXsn270mHvsKnvTeW65dex/MhyduTsYHXaajJKRELphLCP4ZsQyF1pq6uQWGP9BbD7HqB1xV/okFDipsVhzMjELz+FqkqjuTd8ZzAaocpQBECYT7DweY8IGxrvOHqN/wafBhM/gKCxHQ5dOOhMfInCqDTy00/Wx6xPX0/8S/FMeEecT1FApwNS34WU12xouKS7KApMniy+37BBYVhYk9xnnnW5T6MRZu47zBk7yxntdhB2/QvqrLed6Cu4lP8e/xDWLelQ7lyr1jIiTFRTbM3a2mq/mxtcdhl88QVcc41dLJU4iZWXreT2ibdz+6Tbrew1glF/ylR2OsNnZeDPxrz5poE3P4yg/9yrrS6YSVyE8Fkw+jkh89LE0sEi8FfmtR2AaN9otmwRgaUJpgTeoi2ij0d9mSOtPTWIPhvm/UV4/BICdAE0GBpQR+zHYKBVL7GTMegNqN3VlJ0oI3tbtnkRU+XV3N9PURQY8h8YfLfVc7jES1NfIeM7oceetwoQL+/JyRCQs4+dT66grqyuzUNVGhWzn5xN8hIRAK6ogNiCq5l5aDcPzXzAIeZL2kfX/yy+yXqfnMrEVn3hlgxagkpRsT1nO8dKjrU6tsQg0m9j/YJB0QipFxsg/bdnmAJ/5vvp3R9iFoMutNXYWf1msfqK1YyoEi/uXl5Q11jHmrQ1QHPVpxl9Hex/DDK+tZP1kjZRVJD6Nglu3/Lgg0Ji99gx0T/DVOFgNDZX/IXULhM9V4paT8btifRfGzLkX0LmTuJ4DI2ilYA9mPotDLjZPufuIb3Wf918IWi86PveDj9f/DP3Tr2f8cUvUlEBhzsQLJifJPR1fzv6G4+te4yZH8xkZe7XAChaX/AbDNoAm1yCpA3y10LJLgACPCwr/kCoi3x38Vckbv8SFKXNyjBr1NRAvUZU/EX4BUPJnlay6L2JXuu/bXD35LtZsyiDuILrWLVKJA+fjKkdQZiHaCTn5SXmqqS8Bgefc6C1pziZP7Qr3zplivh32zZ4eOpTbL5qM0sGLbE6trYWc1KGZ/kfcOAp0Fu5+X0Ml/HfmHNg4sdC5rEDTMoxJ/f5M1FQVcCCL+dwwdfnQ8EGOPg81FqXhJW4KAefE8H3FszsN5Nn5z5rXSI96TpYcAh8Ehxk4KmHDPzZGD+/phcHSa9jTMQYQt2bJV0CVDEcOybup0lnnOizYUkmxFh/6ZD0HEVRGBUxCi83LwLjxUzsYGtFQQtUahUL3ljART9dRNS4qOYqJI/mwJ/Egag9QesnMneaGDQI8vqdhvb6K3H3dW/zUJ2/jil3TyHpzCRABP7cDL5Euw0nOWQgFO+EnXdCafsNviX2pVWgqIlQr1Bmxs9Ep9GxK3eXxT6j0UilSky2A8Knw1n7IOHvDrBW0hFJwt06TLJoSWWl+NfbG/bl76NeX0+4d7i5f4MZlRYWp8Oop21jrKRrnLUfpv/ImDFCQkejEb3+cnPF7urqZglm95hpMPpF8BvqNHMlkl5JbT587gY777D9uVUaMe8InWL7c5/KKCqY9TsMspZ93syoiFE8Mushxo8WmehbO8iLmJswF4AVx1ZQUFUAgLZB9BEs0C4U0pCBo9o8XmIDlmQLdRBaV/wBbHxmI1V5VajGivLZrgT+qqqaA3/hnr7wywjYcZtNzJa0QVUG/PV/IrG0A9w17owYoRAeLoK0mza1HmMK/AVrmwN/AEz5Aqb/YCurJR2x7UbY03ZSb//+EB4u5D7dcicxPmq8UHCyQoVo74ebG6hHPghnH7GavCixE4GjoN8lnVI9aNnnzxpFNUX8efxP/jj2B2T/DDtvh/oiW1orsTdZP0Dq/5xthaQFGmcb0CfZcLFYlJ6/R0YBXZXqTNh6Q1MvsisAEXBaOngx/935CgCGv24EYOjQpr43JmSWpn1orIGUl8FnIF+e+yX+On+W/6Tm7Y1wqG05dwt8InwAzJItg0ITeSX2FfGSWHEUtt0EiVeL+y6xH5HzxBcia2vWh7MorChjjPcJUmqULrXGbBlcAMSz9eCzEDJNVlU7i9pCzo2+C0P8DFJSWkuuvrngTcK8w/DWWjazKaguQK/UgVEhMSzKUdZKOoEp8JedLSbPPhyFX8fBwJth+ENWjympLqPKvRBv7wT6R46h4M4CjhQfQXVyFaeigGe0na9A0iYt/u+Dg0X19b59sGuX6J9iSpTx8AC3kGEQIp+rEkmX0QaIjPcAGdDpq4wbB2vXwrp18Le/tT3Fnxo7FZ1GR2Z5pjnA4FYvFqBljz8H0eLmJAQkcOekOy36NSaekUhVfhXpIaNhfXcCf2IROsQrCIY9BH5DbGa6xAqNVXDsffCI6lTytaLA5JmVPLPybR5baSDFx5urx1xtfj81+WWgJppqWgT+fBLtY7/EOuPfBE3bD0VFEVV/X38N69c3VwBa42DOcTYkX4ovUSjuX4K7XK9zVWbEz+DJ2U8yKWaS1f1ltUJZzdfdF5L+IXooe8U50kRJT5nylSgE6CzHPxEFA7KVj92QFX82RqVSicmfLlQG/VwaRUgLVFiWN1wwXASEtA2hxFYvZvZsuP12RJ+cVfPh2IdOsPUUQVGJkvD0LwjyDEKtUjNokNh16BBdarxuWsgcGBHDjeNv5MpRV0JtHuSvgZqcVuNVKhUDBw4U/iuxKd5ab/bl7yO3JgODppyifD35uW03SPnrpb94Z9I7lGWIl76KCkiJeIT9AU+SV5kHsefCwuOij5HEOajURNe/x+CQDaSktPbNhMCEVkE/gLRiMdF2bwgj1LgDUl63iXSH9N+e4+MjgkAAR48ieq4GjgbPGKvjvznwDZ/382dj8mQUd9HI3cfdh9ERVppPVWeJbG1jB42RJPahtkBIdxpEWd/IkWLzrl3iX5PMp7+/ow0TSP+V9AlUbjD1a0i8yvbnzlwGXweJf12MXu+/xz6A7bd2auhpp4Gnp6iW3rmz7XEebh5Mi5tmsU1dKyr+Brh9CHsf7tqkRtJ1irdDlqj4i/OP4+nTn+amCTeZd4cND2Pus3NJGB8MiCr4zlJZKSo4PfURBPtEw7D7IfYcm5rvKHqN//okwXnlIsjaCTakb+C29DgOxNzGn5o7uG75dby38z3z/swKMR/xU51U8VdbaD+5ZklrIs8UvXHbwdTnb8tWIy+uf5Mblt9ASU3rppxZpYWUeG+kyGMzVGd32GuuL+BS/pu3Gr6LFgGcDojzj+PuKXczNc76vS+rE2tAfjo/0T85ZNIp0/utz6ALBbcuZDodfAr2PWI/e1wMZ/isCzwl+iDjXoPZfzrbCkl7eETChfUw4jGLzVNip/Da6e/z/en7+ehDFf/8JwQFIQKERZuhooPGDpLuo3aHeVth9PPmTf36gVYrgj/Z2Z07TWMjlDW1YAxomewVMhnOr4Kk660ep9V2oQxN0j5GIxx+GU58iYebB15uYkbVv2YzY35+lI0fHG3z0LqyOkqOlaD1EvejogJSw59mrfbf4kVQ4yl6c3blZUJiW9z8aVxSxlu7/ktVFeS0jqWbMbQI9qTkiZRqj4YYdMU/wbZ/QJ1tNPul//YcC7lPXYh4j2ljEfuMfgvxrO1PnVseHxx6qf0TH3gSlsVCTScf4hLbsu9R+G081AptT1Pgb88e0Q/FlCjj6wv8PELIaTkY6b8SSTtovMF/OGiDnG2JVXq1/+b8CodfBH19h0N1Opg9W3z/88/tj53bf67lhmoR+IszfgIHn5HJwfZmzwOwruNgXExTblNXK/5OO7KC66qyrSc79TJ6hf+q1ODmI/7tBKMiRjE1fgrRxskEVogysQfXPEhto+j5Zqr48zGeFPhbFtepvxuJ40hIEJKfDfUKj615gje2vcHe/NbtPgqasth0io94j/1psKNNdQou479ufuAZZZMAnUmW2c/dDxoqO/X5LHEx6stEAk59WefGT/oEJn1sX5tOcWTgz8YYDDKjvVegKFYnXRqVhhsmXc6Z04ObXwJBNH5fnAGD7nKcjaciQWPBQ5SdXPPjNQx/czCBA4TOZ0d9/kwcPw56vZDTyTccZN2JdWRXNC04K4qoLDwJg8HA3r17pf/aCkWBPfdB6tsAhHiJBQ/vQXqKIoeSVebV5qHT75/OHbl34BHoAUBBWQV6tdD7jPCOENUrVSfM1SsSJ6AoaDx8SUgQz1Brvrny+EqGvzGcxZ8vNm9LLRArKz7GaJTEa2HWnzaR7pD+axva6ttojboaNwZkPwjAQ+vvod9L/fjmwDfWB4efDgNvEQk3EscTfTaMeALU4pmamCiqViorITW1uV9VWGij6M2hbvv5bA+k/0r6DAefE20EbE34bJizyiV7/PV6/x39AizNFxWbnWD+fPHvli2Qn9/2OFOfPxOGKhG0PRH5Aczb3C1TJV0g+Z8w4R1zZWV6WTq7cndR11hnMSy6SQm7rKy5+r0jqoTIgVgnKN4Jf0yHzN7ZF67X+K/RKBaRyw50arinmyfLLlzGxzPXMyHldzwbo8ksz+S1La8DzYEFL/1Jgb/+V0DkfBsbL2mTrTfCFx7QWN3mEEWBCy4Q32tLhgOw6vBWHngAfvyxeVxxpWjy56HyFRW4A2+xm9mugkv5b+Ao8dnWyernzPJMPt/3OT8faZ1Fk1spEhXDvMNg3RL4JtimpkocQPqX8OtYKLTSZNUa/sMgeIJ9bXIhnOGzMvBna6qzYfd9QlZJ4toUbICcFZ0fr/EErV/H4yTdR18rZOGAffn7OFh4EHX0DqDzff5M4wYOhFtX/JNp70/j/V3vQ8kuIe+qr7W93ZLWzPwdxr8FQKiX6G3iM6We46PP4Vhd5/u7ZZaJcjJ3xRsfdx848BQsi4eaLJubLOkC5YeZO2odIHrenIyP1oe9+XvZmLERY9PCy+yQS5l8cDOn1T0AXjEQPgs0Ho60WtIOLQN/RiNw4Bloo5qvqgqiii/Gt05k1KaVptFgaLB+4uiFMOZFq0kXEgcQPgeG/AvcxcKzWg3Dmtr4/fgj/PGH+P7shRqYswbGveokQyWSXk7un3DsPSnj2JvwCBcV7p2swIuOhuHDxS3+7be2xw0NHco9U+8BINgzmOpKUamk8w8Hv1OjEsWphM+BfpeY7+vYt8Yy6s1RHC6yVO7R6SBUTFE6LfdpEfhrKBXBqPpSm5gtaQNFgXXnwZqFXTps4kTw9dSRmCEkQu/59XFWbSzjp4t+ItAjEI96UfJp7r057jUYeKMtLZe0h08ihM0GQ/sVXRMnQmws+JeIkus3V/3Mjh3w7rtQ2CQck1UoIveeGh9IvAZGnDqygb2RP479wUXfXMTzm55vtc+UsB/pHdn0LP+bo82T9JSgCTD8UfBO6His0SAqO+W7s12RqzC2puIQ7H8Uirc52xJJR2y7CbZc2/G47bfB/idkfyJHsOFCIbNhNJjlUyq9RSONzlb8HW6a03n128+K1BUoKJw3+Dw48l9YPV98sEjsT/B48O4HQIinqPjzCCoARJVJfRvv+Fte20LamjTzzzkVIvAX6NbUgCx0OiTfDtpA+9gt6Rzbb2G2+kxA9LopLrbcPTxsOFq1lqKaIo6XHhcba/0JqBpPgtcIkd0pX/Bciv79QaWCkhIoKgJS/wfH3rE6trISFNSMq3zQvG1+ksyS7i2Y5D5XrRJuOGkSJCc71SSJpPcz+TM4t8z2Mo6p78m+cPaisRrKDkJ9655RbWGq+vvtN2hoI99FURRumXALb5z1BvdPfYiSptP7a45DQydLyyQ2I8pXJBxmlbdOGuzfX/zb6QTT4n2sHjKYzwxLRb/xcwqg/2W2MlXSFsMfhpFdW4/RauH662Ga32UEV8wgKfMh3nrdg4H+I9h57U4aq0XEz9PTXkZL2iX5nzDjJ6E20Q6mqr+wsgUA5GrX0qAuo7ERvvlGzEG37xUVfxGBPnY2WtImB5+F4x91aujAoIEArZIxAGoaatCoNET6RMLgu0VAXtK7CBgOQ+8B36SOx1ZnwVc+sON2+9t1CiMDf7Ym6DSYvw9iznW2JZKOGPaQqEJoD30tZH4HeStltYIjiFwAA24CQwOjwkcBkGPYA4hMTFOWZXuYAn/r9M8CsHTQUpKCkqD/lTD+bXCXASOHYNBDbT4YDWapz5qKfJIO/4hvxn6rcoL1VfX8cuMv7Hh7h3lbXrUI/AW7N8kERp8No58VvR4kziPxOlSjn2PwIANGI6xZY7nbXePOyPCRAGzJ2gI0yyj5+AB/TIMfOpEFJnEY7u4Q16S8euQIMPNXmGm9Kr6yKX9iqPpcXpz3Il+f9zW+7r5WBh4XUh+dnAhK7EDlcVgxGY68Yd5kCvyBCPZedhmQ9ZOoqO5sPwaJRGKJ1g/Udui3k/YRHHpB9oWzBzm/wvLBkNVB074WTJgAgYFCHnLVqrbHhXiFcN3Y65iguoG6OggK1BO+NQH++rsNDJe0S8pr8F0UlO4HEAvINFeStGS4UA9k9+7OnTqnIodKj4MUGo/YxFRJJ+l3KcSe1+X1mOnT4ZWXNGQ+toqJmn9QWablhx/AqzGWdUK4hIgIRCuJ9RdA2me2t13SY6ZMgaSgBLxrBmFUNTLhYlFy/dtv8OqrUGMU767xkb6w4SLY9S9nmntqsv8JSH23U0MHBovAX2Z5JhV1FRb7XjrzJerurePWibfa3ESJC6Kood9lEDTe2Zb0aWQkw8aotN7gP0TIhkhcm+izIXpR+2PUOlhwGE77wDE2neokXgVjXgC1O8nBovzgWHmKeCGnOajXFmVlkJsLNW5Z/Jr1CQB3TrpT7AweL85vZcKgUqkYNmwYKpV8JNqMHf+Eb8OgJpeEgAQGhwzGV+2D35EdeBens29f60PUWjVXrLmCSXdMMm8rrBOT9NCm3o8SFyFmMSRdy8xZwmf+/LN1McL4SPECtyVrC4cKD/H4oUvIDvgSX1+EdEf0YpuYIv3Xdlj0+fPuJ2TQrGAK/Pl4K9xy2i2cM7iNng61BVCTC42dyNqQ2Ae1DiqPWUiRRUVBkFD+ZN488TMnPhcLJQ5OcpL+K+kz1BVBwSbby/5N/gLmbrTtOW1Er/dfv2Ew5J4uyW9qNLB4sfj+669FX/H2MAUXpk5pRBl0B0Sd3T1bJZ3HzQ88Y80/RnqLwF9WReuKP1MizIEDbauRtKSwRmgL+rsFQ+k+SP+q10p99kr/7WbFrLs7XHKJ+P7bb+H550VCcVISTJuGqPpN/xLKrExQJfah4ijsvFO03+kAlQpuuw1OCxJVf8c0PzJ4sKi63roVjEoDHmpPonwioWCjaPHSx3E5/529EiZ2bs000CPQ3Aompah1NrhKUaFVa2HjpZDyuk3NlDiA2gKRdHrgmY7HekaKv5v4C+1vl4vgDJ91kadEH6K2EOqKpRxLX0KtFQ8kiUMZECRWoE+UniAhuQboWO7TFBgsSnyZBkMDU2OnMiG6c41i6zsz25N0ntAZkHQ9KCrunXYv+2/Yz41n3Mioz+8kc/Bc9uxpfYjaTU3ctDjCRzYHGwprRcVfuE9T4G/HHSKTT+ISTJkCbm5w4gQcP265b3yUCPxtztrM8pTlbKj4lIzg90Tgb+STMKa1rn93kf5rG5KaFDkOHQJjXRmUH7Eqq2QK/Jn7orRF8HhYkgmJnZDVltgHjwhYmgND/m3epChw5ZUweTJcemnTxjEvw9zNTqmmlv4r6ROkfw2/T7J9n3ddCPgNsu05bUiv9l/fJBjxKASO6tJhZ54p1AtycmBDO2vW9fXw11/i+8lT3WHU01IW0hH0uxTmbRLJ2LRf8RcdLSo46+s7J/dZUlsEgL97sPD59edDdabtbHcwvcp/N14q+rwbGrt1+LRpQtmiqgr27BHzl1tvFb2P8R0AFzbCsAdtabGkPeoKhTxkJwJ/AAMHwn3ni8BfQXUBF7VYDvh74gNU3VPJA9MfgMUnhGrJKYBL+W/ACPCK7XhcE+3JfQJCeS3tEyh0zcQnSTuodVCZ2iUZdYl9kYE/G2Pccx98EwSNso+Yy5PyGnwTapYBaUV1Jux5wPYTeEnbFG6BtUsgbw3BnsH4ufthxIhvXCrQ8YTs8GEw0Mhh77cAuGvyXc07l/WDTZdbPc5gMHD48GEMBtnH0WbEngPjXreoGFJUCmOmeGJUqTl0qHVmbWNdIwZ98z1IT4eIo/cw89Bu/jXjZrGxbB8UbnbEFUjaI/sX+GkQ3hUrmNAUW//zT8shpsDfjpwd/JjyIwAhZWeIwJ8Nkf5rOwaKORj79sG6V+6EnwaQeqiUk/9rTbLLHQb+TEiJOpdj2jT4179o9kf3QBGodTDSfyV9hpDJMOpZ8LGhjLVBD+UpLtuf+lT1X50OFi4U33/5Zdv5vtu2QW0thIQ0f75KHI+px5+1wJ+idE3us7RBVPwFeQRB3AUw+XPwirOZrY6k1/lv4BjRFqSbVX+K0iLZCfF9TEyLASo1qNx6ZqOk8wSMhLOPwsBbOn3IpJhJZN+Wza+X/sqIEUJ62d8frrhC9FZ1U58698/l/FdfJ/q1dbIAxqTudaiweYGvtrGWKe9O4fyvzqfWCFxYD+PftIe1Envi5gNLc2Hk4x2Pzf4V/rpSJBufIjjDZ2Xgz9YETxWZ7ZrOroZJnIY2AHwHAm18OJUfgn0PQ/E2h5p1StNQClk/QOUxFEVheNhwhoYOJTxWLHgcPkyrBeiWHDoEdW45BLqH4Ofux/yk+WKHoQF8BoAu1P7XIGkXt7ICwlT5NDS0DuRufnkzj7o/SvZ2MTFfswbc9P7MHDycodHxYtDMX2HRMccaLWmNSgsoYGhg9myx6c8/LftwJgUlMTZyLGcknsGaE6IJYGjZGfSPLhVB+PSvHG62pH3i4oSEmYcHrE89g68O/Iv/3KPm0kvh99/FGKMRNm0S34eFdXDCtM8h41t7mizpDDkr4Nj7be9vrBHBBUODw0ySSPoc/kNh0O3g3d9256zNgZ8Gwp57bXdOiSWrz4LtXe8ltGCB+Kw8cQK2bGnenpIC77wDaWnNMp9TpoBS9Jf4XXmrbWK2pB2q0kW/qabkXVPFnzWpT4ARI8S/HQX+amshp1RU/EX6B4tK3LgLZN9xR5F8K0z6UCQqdZMJE4TvzpvXLNkLCKnm/LVCok7iGNQ6kSijdu/0IRqVhogmFSBFgXvvhQ8/hFDTEk9jNWR8D+Ud9IeR2J4t18D30Z1u72Cq+GsZ+MutzGVDxgZ+TPkRd7W7CMRrvOxirsRFKNkBx96DxoqOx0q6jcbZBvQ1jPEXQsD1zjZD0hniLxZfbRE8Gebvk8EiRxI2Gy5sMPcYWnPFGhRFwWCAzzygpkZUgcXHNx9SWyukdmJixGTboyGG9RekEBhRjsrUq0jlBrN+c/z1nMpUZcCuuyBqIYe9R3PuV+fipnLjsscvI87gRd7Iq9mzpznLFiAwIZCBCwfiHeaN0SgCfwAzZjjlCiTtET4bFhwAYEwkxMYK3/zpJ7jgAhEAXLNGxffzt7Kj8ie+P/Q9nnX9mDFsAMMSD8LPH4oM6djznHwhkpYoCvz976IPyqZNS9m4cSm4QUUFvP666AGYlQXHjokFz3nz2jmZoRG2XicSbGKWOuwaJFY4+CwUboL+V1jfX7QF/pwBo56DQbc50jKJRNIeKi0M/heETHG2JX2XiqOg9ujyYd7ecNZZos/fl1/C+PGi398TT0BhISxb1lzsPnUqUJMNuX9CwlW2tV/SmupM2P0fULlD0DgGhwzmttNuY2Cw9bJLU+DvyBGorgZPT/FzYSGsXw+zZwtp13XroAZR8TcgOtgRVyKxMYoC11pTny/YAGsXwcQPod/fHG7XKUtNnpD8bJLl7QpGoxFFUSipLWbWB7OI94/n2/lPoVq3RHxujnzCDgZL2iR8Lrj5A52rZlqUvIikoCSGhQ4zbzNVZUd4R6A0lELZgabk/RDb2yuxL1k/i2Be3AXtjxv8bxhwI6g9HWPXKYoM/EkkbaHx6NZLiKQHqNQWPypNM2aVSkjk7Nol+vyZAn/19UKqLDVVTMhqa4X8TmwsqFRd0xNUq9UdD5J0HkWBE5+DZyweAZPZl78PN5Ubk+6axJE0LVsO0arP36Clgxi0VPSxOXQIMgtLOZRwC9tUY5livFH8PWT+CLowp0jSSayjKCLY98wzYqHrjDPgsceEr6pUkDXiV1BDeOUZ3HCrguI/GM6vtNo7rrtI/7UtOh3MnCm+TAuZmzfDSy9BXZ0Ys2iReO62iaKG0zdAQ5lDbJa0w/CHhQSP0WhddlUXBoPuhNCpjrcN6b+SPkJ9CaycC5FnwfAHbXNOXajLL172ev89u/uVIYsWwQ8/iMTDPXsgN1cEi7RaMUcxGiE8HBITAWUpXFjbaRk0SQ/wHw5nbANPIcHZP6A/z817rs3hISEQESESSffvh3HjoKwM/v1vcU937oQHH4RffwWNwZdATSTh3mGw7hwo3gGLjrd5blen1/nv8Y9Fu5bZK8Vaja3wGyJ6HQfJ+aVDWbdE9AJbmtfpQwqqCljw2QLSStPIvi2btNI0duftJrcyF5UuHCZ+BH6D7Wi06+BS/tvvEvHVSRIDE0kMTLTYZgr8RfpEihZAq8+A8W9B4tU2NVXiAPY9JKRfOwr8KQq42bgPjKQVMvBnY9QbzoeEM4TUi8S1qS2Eo2+KF7yI063vV1RCElT2J3IceauF9EPwaeZNRqOR5GTFHPg780yx/e23RdAPREUKwIBkPaqTAohk/iAmZgNvtioPolarGTZsWKvtkh7gEQkX1ILandDGWgAaDA0kXplIbFkgn18tFkpMwVoQVWK7dsHQoaLar8xzBycCPuTlbWu5dfJNYtCGCyB0Bsz82SmXJWnCaBTPT10YxCxhyhT49FNRDXbzzVBcDBoNNDbCgcbloIaLxp1BiClhz4ayHdJ/7URtIWz6G+rwOdxww+3s2yey4QG8vE6SSLKGosjkGVehxeepVfySYdTTjrHlJKT/SvoMag+oKwBDnbMtcRinuv/6+4vK9x9/hM8/h/x8sf2yy8S77J9/wuTJJ00j5ZzS/rh5i35wXWDECBH4W7ZM3Nc33hBBP4AdO4ScYEoKjNa8yfuPgZ8fIiLYiyWye6X/lh+Esv2igtaW/VR9EmDgTbY7n6RzJFwlPjfbSkyzQoBHALtzd1Onr+NE2QlOlJ4AIM4/DrR+0O/SDs7QN+iV/tsBORU5AELO1XcAjH4egic62SpJtxj1LBj1HY8raaoECBje/rg+hDMC9rLHn40xFm4SJckS10dfLXpmZP9iff/u/8A3QSKDV+I41i6B3fcAkFaaxug3R5PwcgKDRCGYuS/c2rUi81JR4L77jDz4IMxdmsPLHgHM+3geekOLD5qMb0XWiWL9kWc0GikvL8cos3Bth6Iya/brNDr8df4A5FXmERYmtPj1ejjQ9Lisr4dHZ6/igytXc8UVsGIFlHluB2BMRIvJ+4R3ZWKFK6AosOtuOPI6ICr7zj9f7DIF/R55BP71YCk17mkA3HXeLDGg8jgUbxcVSDZA+q+dcPOFgnVQlU5gIFx5ZfOupUtF8K9dbHiPJTbAaBDyqy6G9F9Jn0Gtg0Vptq3QS30XVs0XPctckD7hv8U7IfU90Nd36/AlS0Cthn37RODPz08oHyQkwDXXwBBT/kvxdsj+rVcHinoNRiM0VEJ9s+JAbmUu27K3UVJjfV4/dap4td29G267TSQ6+fiIewlC0hVEjzg/v6aDRj0N03+w44XYl17pv8MehPPKbBv0kziPhCth8N1dSojQqDQMCBoAwMGCg5woawr8+cXZxURXxeX8t3i7qILOXdnpQ/449gePrX2M3bmiwaq54s87Erz7ib6e/kPtYq7EzoROhbAZHY/beh2saq93SN/DGT4rA382xrAoQ5QjS1wfjwg4YwcM+Y/1/aHTIeFq0Po71KxTnjEvCckxIMgjiJ25OzleepzQODFRy8mBr76Cl1+GBlU52VOXsHR9DBEDsok6bROVDRXkVuaibln1N/ZVOHN3m/fSYDBw7NgxDAbbSQ9KEBk8uX8AEOYVBsDmJzbz1ug3GTpILED//LOo9Hv1VWDvXgLyDtPYKAKB1f5WAn/xF4r+chLnM3258C3Tj9MhOlrM3W6/XWS7Tx7jz+rLV7P5qs346rzFwCNvwK9jRQ8WGyD9106otXBeBYx9CYDTT4c5c2DYMFi4sINjG8rhtwmwsZ0+uhLHkbkMPtdC+pfW968+G7bd4libmpD+K5G0Q9UJyF8FimuK9PQJ/z32Pmy+UlSddIOQEJg1q/nnpUvB3d3KwEMvCtkyF0zA6HMY9fCVD2xpbua2+PPFjHt7HGtOrLF6yPDh8PjjMGmSSGbTauH++0U/uLimeIKBxvZ7G/cyeqX/qtzsUzV79G1YPkwm8PcSBoWIjPADBQeaK/784iD9G/guGnJWONM8h+By/ltfBhnfQeWxTh/yxrY3uHfVvaxOWw1AdmVTjz+fCHtYKHE0RmPH8uYDboZhDzjGHhfBGT7rmrOI3s7JMoMS10TlBoGj2t7fRZ1qiY3of5n5Wx93HyK8I8ipzCG79gixseNJTxdyKzVumewbdRZ51Xt4es7TRPpEsiljEwCToidZntPN+5QqH3cZtt8Cpbvh3GLCvMM4XHSYakM1NMCEIVWsXOPH5s2ikqi6GtQzb+CeO+sJiYO//oK7s7ZDJYyJ7Jpcj8RBhE6x+FGthqeeErK7UVHN26fHT7c8LmqBkFD2iHSAkZIe0WJxRVHgls7Ghox6GP4I+A6yj12SruGbLPqOuQe33mc0iP4qaq3j7ZJI+hoZ30Fjte3mD8MfEhUuEvuRcCVEzBXvJd3k3HNh9Wrw9ob589sYlHSdSCq1ZV8yiXVUGiEh2ELuM8o3CrKaK0qsMXSo+CotBYMBKlQneHnrN1x27f/xxP0BZCY+ysLV73Bn/Z3cPOFm2POgqDzr9ze7X5KkCUMj5P4OGp9W85AeYdSDoV70p5Y4jsItsPM2kfQdvajThw0KFvOLg4UHKa0tBZqkPtU68IwWfx8SxxI6HS5qbFNhyxrJQckAHCoUkl4N+gY0Ko3o8XfweUj7GKYtA68Yu5gssSP7H4c9D8D8PeDXznpA/IWOs+kURgb+bE3Ob+AxB3RWFlckrkdDBTRWiuo/iUsyIGgAOZU5pBSlMHToeNIy6smNeI/jsQ9S0phLmFcYM/vNBGBj5kYAJsa00ALX10H5YfBJBI2nMy7h1CX5NmgoBaPRXPHXsLSBm569CZVGxSMRoodGdtMc/O/Xahg3XXwsBYSXcdFTR4EWFX+Vx2DFZHHewXc6+mokJ2M0isouN19zgMjXV3y1S+g08SVxfcoOCbnP2PNF34zOog2AIf+2n12SruE7EKYvs75PUcGCAx1nZEokko7Z/xjUFdo2cVD2hLMvASPEVw+IjIRXXhE9q019q1sRMll8SRzDhLctfoz0FslmWeVZHR7q7w8rUldw0TcXUVxTzK2nZfLGG89z2W9b2JaWicq0sH3gCYiYJwN/jmbNQoicb9vAX9J14kviWBQFylOgrqhLhw0OGQyIwF99k0xznF8cRJ0lviSOpxvFL8nBIvB3uOgwAJ+e8ykfL/1YtOw59CzUl5pbx0h6Gd6J4jmtcnO2JRKk1KfNUW+8CEp3OdsMSWf5fQr83sYC9NrFsP9Jh5ojAfY+DN+Gm2UCTBruKUUpJMzYxLaJA9gReR0ljbkMDhnMX1f9xdjIsdQ11rExQwT+JsW0qPgr2w+/jICDz7X7a3VtztQl3Sb6bDEZVhSSApMYEjIEL28vVBrx0TMguppXXoGrroJr/q+BkSGZVBVUAbAjZwcgXuKDPIOaz+mT0KOsbIkN2fYP+Nq/y5M1eyD9105k/QhbrhHP0a7QsseqpHfgxOCC9F9Jn2HsqzDpM9udL+NbKNhou/PZgT7jvz1MfoiKgqCgjsdJnEOkjwj8maTk2uNo8VHmfzKf4ppiAL7Y/wUhoXp25G0BYELUBPH3suCgheR9b6TX+a9KAxM/kMllfYWgcXBOvqi87gLmir+Cg3hoPPB08xQVf6cYLuW/RiPkrYairZ0+ZGDwQEAEcE19z1SKCje1m/DxRcdAF2oPayX2Ju58kXTqk9j2mPpS+GkwHHjKYWadqsjAn60Z+xr4DXO2FZLOEn8JxFvpQaSvF73JSvc43qZTncAxQqJBXws0B/42Zmzk8p+Xkl9/ggjvCF4+42W2X7OdeP94ABZ+3tx0KiGgRcNv9yAYen+7feHUajXJycmo1VLew148Nvsx9t2wj6tGXwXAlle38ELMC5QfK2TRIhjbr5B3J77D9jdFX7/UklQUFEuZT+/+cPp6SLzKGZcgOZngydD//4Au6pT/dhpsvcFmZkj/tSMxS2D6T+A3uPPHVGfBN0GQ8pr97JJ0nRNfioSmhkrL7dm/wPGPRXW8E5D+K+lTBJ8GIRM7HtdZ/vo/2Ou6vU/6hP8WbYMvfeDwi/b9PT8lwwbZ99Zh7HkQtjRXcEX5Cg36zlT8JQYmctfku7hy5JX46/zJrsjmw90fUlRThFatZXjYcJEs490fvGLtdAH2p9f6b/zF4llrS3JWwJE3ZQ/OXsKAoAEkBSYxNW4qv136G5X/rmRY6DDxTrv/ydbvun0Ql/NfRYHV82HP/Z0+ZEjIEHQaHbmVuezL32dH4yQuSWMVYARDg7MtcSjO8Fkp9WljDPEXg4e/s82QdJbBd1nfrtbC+ZXy5c8ZRM6HyDPN+uCmwF9+VT5Xj76a7w99z8a/b8Rb621x2JSYKaxIXcGFQy9EaVm54BUneqS0g8FgoKSkhICAAFQqmQ9hM7J+hm03wNjXIcqy6UnE6AiSzkqiPKuc4ORgvMO8mfPUHOKmiWy9q0ZfxYVDL6SstswZlks6Q3f6oBqNooeGDV/wpP/aEZ/E9jP1rFFfLAKFsr+Ga1GRAlk/iWr6lj1vU14ViU5xFzjFLOm/kj6HQS/eYXtaRWs0wsSPQOPd8Vgn0Sf81z0Ygifav++wR7SsXHAk+auh7ACM/y/QouKvIhuj0chLm18iMTCRBQMWAEJpJNYvlmBP0a7lsVmPoSgKRoy8t+s9bltxGwCjwkfhrnEXScK1uSLBVOPl+OuzAb3af41GwNilfmLtkvoupH8BCX+3zfkknSf7ty5LZLtr3Em5KaX1jozvIPVtSLzGhga6Ji7pv2NfBY+oTg/30npxev/T+THlR17Z8gr7C/bTP6A/Hy35CE58IQY5aX4i6SFV6XDoRbEGGD7H+hjPKFE5f4phMHQxad4GuMgTou9glD1S+hYqGRt3OIrS/BJfsofkwEQGBQ9iWNgwHp75MFuv3toq6Adw77R7+eWSX3j1zK5LrhiNRjIyMqT/2ho3H9AGWV38ipkUw/lfn0//2f0B8In0YfJdk4k+Ldo8xlvrbc7QBaB0H+x7TPRslPROFAXO3NGq90pPkP5rZ4xGaKzu/Hj/YTB3I/S/zH42SbrOwFvg/CrLoB/AmJdh2jKn9WCQ/ivpU+y8Gz7XQHVmz8+lKBC9EMJn9fxcdqJP+K93PMxaYf/Fxdl/wJgX7fs7JM3M/BWW5pl/bBn4+ynlJ97a/hbVDeLdxmg0csm3lxD9fDR/HPsDwJxEeuHQCwEorS0FYHzUeHHCsv2wLE4sbPZSeq3/Hn0LvvQS1bq2YtgDMOt3ufbjDPY/Cjv+aZtzDXsQ5m0Fty70Je+luKT/JlwJkfO6dMiS5CUAfH3gazZmbGRD+gaxY98jLq14IOmA+lI4/ILLy9U7A2f4rAz82RjVyjai2RLXJONbWH02VB633F6dCVnLoTbfOXZJION7+GUkAyq2ceAfB/hk6SeAyPCyhqIonJF4hmU/OID1F8KGLlYlSWxD6FQ4cztEnsmhwkMMfX0oo94c1f3zFW+HPfeKJuAS51OdJWTI0r9ytiUSe7J8EPzpugvPkk7i5gNqK5+fPgkQeYbj7ZFI+iL+wyD2PFBsIOPjSot5EklvQ62zSDyM9Yvljol3cN+0+3h92+scLDzI5szNAPxx7A8OFR5Cq9aK/n0tmNVvFh8v+ZghIUMAmvdrA2DgraJaVOJYPGMhdJrtqv0A/Aa1XZUisS8jnoApX3fr0A92fcDAVwfy2NrHxAbPSAgaCyoXkb+UdMiSQUvYcc0O3jr7LQAifCLEjtPehwnvOM8wSc/wTYZFJ2DQHW2PKd0HR/4r1pQkdkUG/myNR4SzLZB0haoMyP0NavMst+euhDULoHCTc+ySQPAEGHCTeIHrCbV5re+vxOHoNDr2F+znQMEBc5bLxmc3svyG5QBsen4T/5vwP8qzytmQvoHJ707m0bWPWp4kehGcuVMEFCUugBGOvQ+Fmzt/SHkKHH5VyA1Kegfhp0PI5M6P3/Og+LuQuB7FO4Skkon6UqiXcsoSic3odylM+bLn764AmcvgqwCRCCexLweegQNP2e/8VRmw92HbVihJ2qfqBOStMveM99Z688zcZ1g4cCG/HRWfg9ePux6AV7a8AsAVI6/Ax91Splyj0nDJ8Es4M/FMxkeNZ0J0U+DPOx7GPO/SFbl9lsgzREVn0NjW+4xG8z3vErK9i/MInQJh07t82C9HfuGKZVeQUpRCbmWu2FhX3NQ3TOIUdtwB30V3yZ/8df6MihhFTkUO0FydTdDYrs0/Ja6FWit64Go82x6T+ydsvR4qUx1n1ymKrGW3MYaJHznbBElXGHAjDLy5tRRhyGSRYRI4xjl2SUQQfexLPT/PnFWdGubjI/tR2YWU10HjSViMkFCq19dTVleGv86ftNVpHF95nPmvzqe+sp7yrHI0Og1/Hf6LjRkbCfcOtzyX1h+0Ix1+CZI28IiE88pFJVFnKdwE228SvTe9+9vMFOm/dmTsK50fa2iEA09C2Czof4XdTJJ0ky3XQE0OLGnKrDz6Juz6F8zdDMHjnWaW9F+JxApu3hA42uX7wvUJ/037BBorYfDd9jl/xWEhWaYLtR6skNielNfg4DOwMNXiffO/2/6LESPzEubh6ebJgk8XsPyISEK8cfyNbZ7umbnP2N1kZ9An/NdEdTb8Nl7I9o5+rmvH/joWDLWw4JB9bJO0j0EPGLokO2/qxwkQ5x8nvlk5GxrKhd+fAric/2q8wTNGBF+1XZNbzakUgb8I7wgRwDc2Oq0NgcRGVJ0Ao77tNZ/Y88BvMPgPt75fYjNk4M/GqNWyrLxX0ZYMgE+C+JKcEqjVahIS5P22CwefBvdgPPpfga+7L+V15eRV5uGv82fJR0vQemtRVArT75/O9PtFtt/2nO0AjA4fbXmuhgowGkSgyZbyLpLuoai6FvQDiJwPs1eC3zCbmSH914VQaWBxuvBViesx9H7xDDUaRcKT3xCIu0hIXDkJ6b+SPkXpPjjyBsRfCiE9lAAMn+Py0nN9xn+n/wAaL/udP3ginLlbKgM5kuhF4BUvJDmbOFF6gmc3PQvAP8b9g/35+81Bv6mxUxkQNKDz5z/2PmR8B+NeB8+oDoe7Ir3af/c/CRhhyL/Fzw3lsOtuqMnqnp+Fz5ZVf84i+1dYc7ZIuu9Cf/Dk4GTz977uvuKb6CVgaLC1hS6JS/rv8AfFVxdpNDTyxPonAFFlTX0JfBMESTfAuNdsa6PEcfw2HnwGwOnrrO/3jLSNQkYvwxkxI7lyamMMeW38UUtcl/x1kP2Ls62QWGP7P2Hjpd0/vngnHHkTatqX+jQYDOTm5mIwGLr/uyTWmfY9TBE94MK8wgDIqxL3wyPAA7Vb6w8+U+BvTORJFbd7HoCv/UUPTolrUH5YSCl1Fl0IhM0EXXDHYzuJ9F87U7of/roS8lZ3brwuVCbOuCrRCyFmcbPKQdQCmPxp1wP4NkT6r6RPUZsLR16H0t3OtsQh9Bn/9YoF96COx3UXjRcEDBfvQBLHEDIZBtxgEfgb/VZzQuH8pPnM6T+HGfEzALh/+v1dO395CmT/DCgdDnVVerX/nvgMjr7d/HPhZkj7GMa8YtlTymjonPTn6OdsozQk6TpesSIxVBfWpcNayvIODR0qvhl2P4x4xJbWuSy92n9PQqNqrkdKCEgAjBB7gVA9kPRekm+Hfu0E8/V1p2Q/a2f4rKz4szWFmyDpLGdbIekKm68S/0ae2bxt/YVQth/O2uscmySC0j1Clqy7ZHwD+x8TmbYebb9MGo1GcnNzCQmRE3KbEzDS/G2YdxhHio+Ydfjrq+rJ252HT6QPaavT8Ar1Imx2GClFKQCMiTgp8Bd8GjT+XUh+SlyD7bdC3kq4oKa1ZLI1GiqF1rsNKzal/9qZxko49p6Q4gib0f7Y8iNClsUrrnN/DxLnYNC3rXjgYKT/SvoUIVNgSa5FsKHbHPkv1BbAsPt6fi470Wf8V18v5DjdQ8AjvOPxXaW+DDCCm5/8bHQiD814iJt+uYmXzngJddNn4A8X/kB6WTpDQod07WQjH4cRj9nBSsfRq/13+o+gDWz+OeJ0WJzZup/UhgtFRdk5BaB2d6yNks7hNximL+vWoTuu2cGhwkNMiplkY6NcH5f03/IjkPktRC4A/649U/ddv4/vD33PNWOuAbUbTPncTkZKHMbgu9rfv2YhlGyHcwodY4+LYHRCsFMG/myMMe4SZ5sg6Sqjnmn9kugeLOVYXIFZf/ZsgjzgRhH0803ueKzEPhj0UJcP2qDmir9KUfFXeLCQdye/y6zHZrHx2Y2EjwgnfnA8ADG+MYR4nfQiG3e++JK4DknXQ8xSod+udOKVYs3ZIqninHz72yaxDQGj4JwicA/seOzuf0Pmd3BeJWg87G+bpGvUFcHPw0XVbfKtsOMOGPIfsWAmkUh6jloHHjrbnOvYB1B9wqUDf32GgvWiN9SYl0Tvd1uz7xE49BwsPA7e8bY/v6Q1uStFT+kRT4hqd+CGcTewcOBCYv1izcN83H26HvQzIYO4zsMrtvU2zyhR3bflevCMhsH/EvLLXrGg0rZ/vq03iPfdxKvtY6/ELoyKGMWoiFHiB0OjCPSGzxbzU4njqUgRvcPdQ7oc+BsSOqT7z2JJ7yR8tn3VFiRmZODP1tgjS1BiX5omAxaMe9Xxdkha09MJlUc4RMkKXKdy4EnYcy+csZ0BQQMYGjoUL63ooxKYFMjpz55O/Ix4YqfGotaq+TLnS8CKzKfENYk+u2vjw2aKzE5J70GtBXUngn4AcReKBt0y6OeaaAPFV/wlUJUOxdtOmV4oEolDMBqEBLbao+cBnuk/ij43EvvjPxyG3AtBp9nn/METIOEqkVgqcQwqjZAQM+qbNykqi6Bfj8hZARpvCDn1Ko1cgsYaEWTwiBDBvqItEDEXND6QvVz0lRp6D5y1X/T/a29NwaAXvVljz5OBP2eRvxb2PQqjnwf/od07R2OVUHuSykDOI2QyzNsCPok9O0/RVkh9F5KutVCPkvQyDj4HGd/C7D9FYtzJdFQRKLEZMvBnYxROPY3aPoHRCPqa1pV/EudSlQ5FmyFkWrtSnVbR10N9sdCL7yCAqCgKgYGBKDJz0/YET4Skf4CbL4/PfpzHZz9u3qXz0zHpdssJs3GjkUifyNYynyB6/NXmwfj/2ttqib0Y1sUeKp1A+q8DqM6G4q2iB4fKre1xsec6ziZJ11EUmLtJ9JtSFIguE4EKp5ok/VfShzDUw/LBIrg+6eOenUsXbNN+uPagz/ivLti+faFizxNfEscROg0WHLDf+bdcIwK5Z2yz3++wM73af/NWwpoFMOFdkSCx83aYvhyi5sOZO5tlQA11om2IoaHt56mignNLnP4+dEqjrxPBv9K93Q/8af3gIr2o/DsFcEn/1fpD0Lien6dsPxz9r0jgl4G/3ktNDlQeg4YK64G/UxRn+KxidIbAaB+kvLwcPz8/yrJ24xs53NnmSLpCfQks6y8kBMe/CY3VsP8JMWGQ8lfOJfU92HylyHqOWtC1Yws2wu+TReZY8q32sU9iEwx6MdFSqZv7vjUaGi0aPQOwYpIIBi/JdKR5kvYo2AB/XQnDHoL4C51tjcRe7PoPHHgC5vdgQi6RSCSnAnseEAtVMUu6f46GcqjKEFnzsi+VROJ6ZHwPirrryhcS21BbAGsXw5w1UJsLWT9Cv8ubk7j3PADlByHmHCH/OO6/onpI4poYGkTijMbL2ZZIekpjtbiXPam8NDRAfSm4+ciAUV/l4HNQsgvGviqC9qcQ5thRWRm+vr4O+Z2qjodIuoJB69qZmRIruPlDyBTwGSh+rs2F/Y9C7gqnmiUBwmbApM8gYHTXj3XzhYSrIWhCh0MNBgPp6ekYDDLTz9H8dvtvPKJ5hEc0j7DjfzvM21sF/QBO3wALUx1onaRDNF5NFWCdyCGqyYMNl0Bm9xq4t4X0XwcQey5MeAd07ciZp30OP4+Aws2Os0vS65H+K+lzDH+oZ0E/EJUPPw+F4x/ZxiY70af89/Cr8EOCeFexNRsvgz0P2v68kvY59oH9fChmca8P+vVq/9WFwNwNQtLVM1r0dDMF/YxGIQNaukesAwy5R/Tva4vGKijZLaWVnYnKredBv/oSyFsNNbk2McnVcUn/NRrgKx+RFNwTVG7Cx2XQr+9SvE0kbLj5ONsSh+MMn5WBPxtjVMu+Nr0ORYEZP8Kg28TPHtFw1gEYYIfm7pKu4d1PVBF5Rnb9WP+hMOGtTvVeMBqNFBcXIwug7UBjFay/AA4+z8GCgwx5fQij32wO5Lp5uKFyU9F/Tn98+/m2fw8URWa+uxoBI+GsfRB/UcdjqzPgxKdQfsimJkj/dQCBoyHhyvZl51RuUJMN2gDH2SXp9Uj/lUis4NUPhvxHyKW7MH3KfxWV6M1YV2jb8xqNkPOzkMuWOJa9D8HBZ5xthcvSp/y3JYoC4XNh4sei1+qIRyF4fNvjS3bDLyOF0pDEedSXQsprkLeqe8cXb4c/Z0Lm97a0ymVxSf9VVBC9GOIu6Nl5Ko9D6X7Rf1PSe6nOEonBlWmt903+DBZniL+ZUwxn+Oyp978skXSEWgt+g8ArxtmWSCS9H7UHZH4HxdvRaXQcKDjAwcKD5g+8WY/O4r76+/jb73/jR88fiXgugkfXPmr9XHlroMyO/Tok9iVoLJxfDQNudLYlku7S3gQs9hxYnAk+SY6zRyKRSFyNrf+A1T2sBPIfAiMeE/9KHMOAG0Qik63/zxUFzimEqd/Z9rySjpn8KUz5xvbnLd0L34ZByuu2P7fENiT8n5h3dAbPKBj2MIRMtq9NkvbR18C2G+Ho/7p3vM8AGPMKhEy1rV2SrjH1m54H/vY9LFQP9NW2sUniHIq3w8aLhIqFNU7Baj9nYUVLTSI5BSk/DPsfh7gLIXiS0JV2DzwlMxBcCoMelsWJfouTP+38cUYjrJwDkWfCoDvsZ5+kYxQVnFcJai1hDeLlrbaxlor6CnzdLTWtd+XtIq8qD0NbzdVXnwFhs2DGcntbLekKxz8WL+aJ13Q8ViOr4nstGy4SvVMXn2h7jKzIlUgkpzo1WVDdznNScmqi1jrbglOP4NPsd26fJKlw0FvYcDGotDDxfev7veJg2H0ONUliBY8ImLkCgjtu02IVr1gYKJNLXQajUSS+dIfoJeAZCxpv29okcSxB42DKl61bL+Wtgbp8iDyrWaJZYldk4M/GKN19uEmci9EAxz8E70Qo3Qe77oJ5WzufKSaxDyo1+A0Br/iuHVdXJLIxfQd1ariiKISHh0v/tRdNix2ebp54a72prK8ktzIXX3dfakpqeG/qexgaDRy75RgAycHJ1s8z+gXwiHKU1ZLOcugFqC/qOPBXvEMEgv2H2zSpQvqvg/DqJxq26+ssA3xGI6xZCGHTZaKFpMtI/5X0OaZ93/Nz/DlbvP+Ofbnn57Ijfc5/0z4VfaIG/KNn52m54FmdJXqNBY4VPYskjqWuWNwLWwbp/IfB6ettdz4n0ef8ty2qTsheYb2FiNOdbUGvwWX912iEjRcL6daZv3TvHNELxZekd+MRAbHntd5+5HVI/wrOLQZOvcCfM3xWljPZGJVK/pf2SnwGwDlFIttr4M3gHgTe/Z1tlQRg1m8w8vGuHaMLhnPyYfTznRquUqkIDw+X/msvyg5BxrdgNBDuHQ5AbqVovG1oNFCwv4Ciw0UcKT8CQGJgovXzJF0H0T2Uz5LYngn/gxk/dzxuz33w2wQwNNr010v/dRAjH4fpy1pX9dXmQ+kuqEh1ilmS3o30X4nkJAx6qM0TSWwuTp/z38OvCImxnrD/SfguQgScAHL/hNXzoWBDz+2TdI2CTfBNEKS+42xLXJI+579tMXcDzP6z7f3HPoDfpwoFKIlzMRpFoLaqG1XzR96A5UPFusMpgMv6r6KIub5Rb/M5v6SPMOIxmPIVaP2dbYlTcIbPuthTovej18sGpL0SlVpIe4JY1FyY1vyzxHXoaiPUTsrq6PV6UlNTpf/ai8MvwrpzoK6QhIAEsalQTK48gz25fNXl/G3r38irzQMwj5H0EgJHgd/gjscNvEUE420sdyX918l4hMHC4zDySWdbIumFSP+V9DlK94qF5Iby7h2vUotec5M/sa1ddqDP+e/4N+D0jV2fb7TE2CgCt7X54ueQSTD+Laki4wx8B0L/K0T1rC3J+R32PdYrgvPt0ef8t7vUF0NFirOtkABUp8OyeDjwdNePNRrE8/cUkVV2af+d8gXMWgGqbgoMrlkIW66zrU0Sx6OvhW9C4K//s9zukwix5zjHJhfAGT4rA38SiTXcpJ60y5C1HHbcDo014iXgm2D4fQoUbW37mMxl4rguUFFR0UNDJW3S7zKY+BGoPRkcIgJEBwoOAKLUPX5GPOXRYoEsyCMIP51f63NUHoefkuHwqw4zW9JJjEaR2a6vbX9cxNyey2e1gfRfB2A0wO774NBL4ufafDjyX/G9SgNaK34rkXQC6b+SPkX61/DXFVCV7mxLHEKf8t+AkeCT0L2+RI2ijzVD74WLjeDXJFvvkwiJV4NntM3MlHQS90A47T3R892WZC+HPfeCvsa253UCfcp/26LsIKS+21yFezLJt8LSPBEoljgXz1hIvh0i53f92AH/gAWHTinVLpf135629KjOhJoc29gicR4qd3Dzg/AWEr7Zv0GDi/7d9mFk4E8ikbg2eSvh0PNQmysCsm4BULQNCjdbH6+vh81Xw07Za8plCJkE/S4FN29GhI1gaOhQQrws+5ykFguZwDZlPg31oGi63yRaYj9S3xFSSnmr2x5jcMFsREnXUFRw/H1I+0j8vP1W2Hq9eB5LJBKJRBB3IUxbBp4x3Ts+f734XK0vs61dko4xGqG2oOuVXIZGITGXt8Y+dklciyH3wBnbQRfubEsknSH7Z9j8dyg/NSQgezWKAqOfhaiznG2JpKfk/glrF4tef13lzB2ivYSkd6MoIhgff7H4uSoD1i6E9ec7165TkG7W3kokEomDGHQHDLhJZMpO/kxsqy0AXYj18WotzFkLtTJLyBW5fOTlXD7y8lbbPd08mRE/gxFhI6wf6DtQSF9JXA//4ZBwFejC2h6z7yE49h7MWXNKZWL2OeasBY8o8f3IJyFmMQSOcapJEolE4lL4DRJf3eXEp6JXUeR8QFZSO5Sy/fDzMBhyL4x4pPPHVaVBQxnk/g6h0+DgM6J3TeI1sGq+2DdX9vhzCsc/htS3YcrXbc8du4ouxHbnktif6EXgm9z2cznndyHNfApLz/UJMr4XMstJ1zrbEglAyW6hwlV2QCSBS05NTHKvdcWg9oCxr4JPknNtOgWRFX8n8dprrxEfH49Op2PChAls2bKlS8crshpFIrEtHhHgHW+pEW6abFUehz33N8vrmPBLhrCZnf4ViqIQExMj/ddelB+G72Pb1es/M+lMVl2+ihfPeNFxdklsQ/B4mPC26PXXFtogcA8FD9tLXUn/dSDe/Zp7Z3jFQOx5sgpX0iOk/0okJzH4bpj+Y6+oJupz/usVDwlXQ9C4rh3nkwjnFMCQ/4jPxJRX4OhbYp8utP3EKIl9qc0XC9DVmbY7Z00ONFTa7nxOos/5b1v4JIoKMm2A9f0HnoAtVzvWJknblO6Fn4fDkTe7dtyR12HnnfaxyQVxef9NvArOq+h60K+hHFLfg5I99rFL4njKD8OPibD/cSF9HjbD2RY5FWf4rAz8teCLL77gtttu44EHHmDHjh2MGDGCefPmkZ+f3+lzqFTyv1QisSmGRtEnZc8DopF6S8nAwy/Bvkea+/2lvgfZvwipni6gUqkICgqS/msv3PzFwoebr3mTwWigXl/f+XMUbxf9/WpybW+fxP4k3wJnbrdLw3XpvxJJ70X6r6TPkb8OvvTt+qKlCa84iFrQK5Iq+pz/unnDhLcgemHXj1VUoPEU38/8DWb8Ir6f+D5M+9ZmJkq6yMCb4dyS1slpO++GtM+6d84Vk+GPaT23zcn0Of/tiLbaDgx/DCZ+6FhbJG2jCxfykIaGrh03+gWY8ZNdTHJFXN5/3XzFZ2pLjEaROGFobPu4iiOw+UrIPnXuZZ/HJ0kUZQSNd7YlLoEzfFYxGru4Qt6HmTBhAuPGjePVV18FwGAwEBMTw0033cS//vUvi7F1dXXU1dWZfy4vLycmJoaCggICAkQ2kaIoqFQqDAYDLf+bTdv1esuXj7a2q1QqFEWxut1kZ2e2q9VqjEaj1e0n29jWdnlN8pocfk2le1D/Nlrs8IrDcPax5vGGRpS831FFL0Df2ICy7QaUgrUYztiDonbr9DXp9XpSU1NJSkpqlYEh75Ptr+mmX27igz0f8NK8l7hy1JWoVCqq6qrQaXRtXpNy4ElUe+/FOG8rBI5xuWtqSV+5T52+JsCw8W/gNxjjoLsdfk16vZ6jR4+SlJSEm5ubvE/ymuQ19aJramho4OjRoyQmJqJWq/vENfXF+ySvqQvXVHkY47abMSTeICTmunJNBj00lqO4B7rWNbVxn4xGIykpKSQkJKBWq61fU1vX6qLX1OW/PQwY9j+JMWI+BIzoG9fUF+9Ty+31Zai/C8LoFYeyKK3L12TcfS+4+WJMvsN1rqmFjZ29T+3Nf3vrNVndXleM6qcElPiLMY57o29cUwsb+8x9ktfUpWs6ef7rktdUtl8EcUMmi/G77kU58Bj6M/aA32Dr96m+FKVoIyrfAei9El3vmjrYfir87clr6tk1lZSUEBQURFlZGb6+vjgC2eOvifr6erZv386///1v8zaVSsWcOXPYtGlTq/FPPPEEDz30UKvtBw8exMfHB4DAwEBiY2PJzMykuLjYPCY8PJzw8HDS0tKoqKgwb4+JiSEoKIgjR45QW1tr3t6/f398fX05cOCAxR/UwIED0Wq17N2718KGYcOGUV9fz+HDh83b1Go1w4YNo6KigmPHjpm363Q6kpOTKSkpISMjw7zdx8eHhIQE8vPzyc1trrCR1ySvydHXpNZXEBZ4CUrYdEISplu5prGEA2kn0vEqV6gKuIWK/Qe7dE2mlyeDwcCBAwfkfbLzNZWVlFHdUM2ag2tYEr8Ebz9v/J7yI8g9iK9nfY2Pm0+ra9I0jEMX9xZR2ji0BoPLXVNfvE9duSZjxjKqitI41ji/1TV5VW3Hv3wFhYEXMWDs2Ta/JqPRSHFxMV5eXsTFxcn7JK9JXlMvuqb9+/dTXFxMXV0diqL0iWvqi/dJXlNXrmkweUM+FtdUsrdL1+Red4xBqedQnXAnnhOedqFrsn6foqKiKCoqora21hw46D33qY2/vervqE/9nEORr4Kitn5NxkYUYwPRcQMIMqai2nsfuTnp5IbeIK4pPhpfVTGphw/gW/Iz5d5TqfYcJv3JSdfUz/M4fh5GjlQNFtdk1KPr/xXRMXF4Q9evifOgAdi7t1ffJ9NiZ11dHUeOHHH6fbLb357RQIJuGD6+A/vONdEH71MPrykv+wRGRQuK0meuqb37ZJr/+vv7ExkZ6ZLXlHx0KaAna9RKEsI1KAceo1HlS8rRNOq1+nbu0zjCfcNJS011uWvq6n1qvqa+87cnr6ln19SygMxRyIq/JrKzs4mKimLjxo1MnDjRvP2uu+5izZo1bN682WK8rPiT1ySvqe9ck16vZ//+/QwbNkxW/Nnrmor+QnXiM/SD7ubtg8u5/ufrmZcwj58v/pmU4hQGvTYILzcvSu8qNZ/D5a+pL96n7l5TfTWo3a3arvx/e3ceH1V59///PTPZSWYCBJKwCgQREFHQstxVFBBB74rS24VaikrtrUUrVrFSFW2tim39qbV3lVKx6G29ta3b1w0VcaEisoiAIktAEUuCEDIDhCXJuX5/HBkYScgEJ3NyTV7Px2MeJOdcc+ZzHN85M/OZc52PfyP/qttVO2qx/G0HJnyfDuT3+OOP54w/9ol9smyfqqur9fHHH6tv376c8cc+sU+7Nsr3yZ1Sp/Pk7zy22e+TMUYrVqyI5rfOfapvX5vpPvmXTZH57Ak5Y1ZGr833zdp97/1Avq/elRn9ofyZrVX71UIpu4OU292t8Yu/y/feeDndfyz/hr/IGfCATM/Jzfv/vQaWN7vnqRH75J87QL79lar93obY5RbvU6LO+Kvv/a+t+3Sk5fXu0zNtZYrHyBnyv6mzT7Y/T5G1MqWPynS+IHomdUP75HumQGp9kpzTX2ue+9QEZ/wd+v63Oe6T7/MnJflkul6sQMX7Mu+cJ3PCXTLdL69zn9wHqpUvkJYyz1NDtbNPLW+fOOPPIpmZmcrMzDxseSAQiL7xOeDAk17X2GQv9/l8dS6vr8bGLmef2Kf6ln+rfapYJvnTpVDfJtsnn89Xb431bYfnqRHLd34qrfsfBYpHqV9hP0nS6m2r5ff7VVpRKkkqaVOitLTYw1K0lprdUiBb+vqNabPYpyM8ZlMvb3b7lJFT//h+06UelyqQ3fGIz9+32acDL8aONJ7niX2qbzn75O0+Haj10DG271O8y9mnFNwn48i/4hYp2EvqPrHh8Yc+ZqhEGvJoQmuvb3kinqfa2to685uoGhu7PCH/7w24T76T/6C6HjW67ewiydkrX1aB5A8oUDQsdmDbk6XeU+XvNFY6bor8We2kBv6+Nek+HWG5tc/TEZYftu0TfyultTq4vGqztH6WVD5PGvH2weWb/iGlh6TiM+utXTtLFVg6ReoxSep8nnf7lKDlR3r/a+s+HWn5Yfvk1ErtTpUv1Dsh+9os9qmBGq3Ypz1b5Fs9Q8psLRUMaLBGv98vdTpPyiuJ2V6z2qcmeJ4Off/bLPep+w8PLmz3H/KN2yqfcSR/oO7xkrTsGmnj49K5GxXIKmiy2utbnpJ5Yp+a1T5984s2yUDj72sFBQUKBAIqLy+PWV5eXq6ioqK4t1Pf/zgAvoVXB7r/XrRPCmQkfPN+v1/du3cnv02p8zipcITUqqt6790hSdoU3qSd+3ZqfcV6SVKPNj3qv/+8Ee7FoM/7PBnVorEia6Rdn0kdzjp8nc8nteraZA9NfgF7kV+kHJ9f+vReqWjUYY2/VJOS+fXX/eFNjJMfcG9OjbTjIym/n/u8H5BXIp3026arEY3zzdemi34sbZkrZbaTaiJSRmtp10Zp0RWSqZEu3Fn/tvaWSWVzpeI6Xu9aJiXzW5+Nj0vbFkon/0/0S4iS3Lyf/v+8qwt1KxgkjV4q5feP/z6DZzddPc2Qlfn1+aTlN0l7v5KG/LXuMcHjpIIh7t9lIAV5kVmL/ko0rYyMDA0cOFDz5s2LLnMcR/PmzYuZ+rMhXnRvgZR36rPSd2Y1SdNPcnMbDAbJb1PKaC3lHiP5fGqT3UZFue4XKlZvW63SHe4Zfz1aH6HxVzTSbR6ieVr5K+mt0VLtN+Ysr90rlb0h7Smv+34JQH4Be5FfpKRzPq7/Q636GEeaN1xa84cmKakppGR+a6qkzc9L25fUvX7vtoM/hz+WXjlRWjG97rF7yqV9FRJXVvGeU+s+F5LU8XtSn5uk7289+OFybjfpnFXSsBcPuU/N4dtp9x/uF1F7XtX0NTexlMxvfcrmSesekmqO0NRF8xHIktoMiO+LGC2UFfkNr5ZeKJFW3yute1gqf1uq+FD6akH9x8VeP5OGv8Zzj5TlRWZp/B3i5z//uWbNmqU5c+Zo9erVuuqqq7R7925ddtllcW/jm3O7AkiAzudJJT9uss3X1tZq5cqV5Lep7dvuvvEyRn3a9ZEkffLVJ9HGX0mbkvrv2/830sD7klEljkaPy93m/DdFPpXePNN9s91EyC9gL/KLlJRXImW2bdx99myRKldIu+2Z2SAl81u9U3rnPKn0L4ev2/WZ9GyhtOpO93ef320EHfPDw8d+eKP0bJH0z7aSs78pK0ZDnGrpnwXSIve6Ujp2snTi3YePy+koFQ5zv8Q2f7S08Ed1b8/nS4kPpVMyv/U56bfS97dJaXmxy3d/Li27Qdq6wJu6UD+nWtr2vlT174bH7imT3p8kbW45Z29akd+s9m4TV5KWTJbW/kE6/SXp3PWxZ94CLYgXmWWqz0NcdNFF+uqrrzR9+nSVlZXpxBNP1KuvvqrCwkKvSwPQxJr1i6ZU8dHN0vqZ0nmbNfyY4crLyFNRbtHBqT6PdMYfmreikXUvzyqSBj4oFQxu0ocnv4C9yC9Szt5tUnVYymvE65qcjtK4ryRnX8Njm5GUy29We2nwo1LrEw9fV7tH6jhWavP1JQjy+0ln1tMw2FPm/ltypRTIbJJSESd/utTlgtg87t0qlb8ltT5JCvaUti92m0AdxkhprSR9/aG0cWKncQ1/IlV96TZ80+q+vrVNUi6/9clqX/fyXRvcqZlzu0ntv5vcmnBkZfOkt8ZIA+6Xjrv2yGP3lkkbZrvPY6fvJaW85qDZ5zezrXsmtVMjFQx1/xYf6XhojLT4SqndqVK3Or5QA+Co0Pj7hquvvlpXX32112UAQOrpOl4K9ZECWbr5tJuji4cfM1xFuUU6tu2xdd9v33Zp6RSp01ipy38lp1YkRnaR1ItjKgCgBfnXxVLFUumCHY27n8938Nvx8IbPJ3W/tO51od7Sac/Et50hc6ShjyWsLHxLg/7s/vv501LpLKnTedKSq90vpwV7ShsedWenOL/MbfwNe7Hus/rWPSSt/aM09nMprUtSdwHfQs0et8mX1V7KandwecFQ6dyNUkbIu9pQt3ZDpX63S4VnNDw2/wTpvyoln/1n4qYkf5rU7uvLZ9Xslra8JuV0ktqeEjuuulJa/2fJ1NL4AxKIxh8AIDkKh7m3b3joPxuYBnLPFumz/5Vyu9P4a67+PffrD1AekDqe7XU1AAB4p+vF7oeWjVE6250itP1pTVMTGsc4knxHPx0Z05g1T1VfSF+9J51wp/QfT0ltT3aX97xKavddKbPA/f3Qpp8xB5/PnpPdD6xzOiW3bnw72xZKb45w36f0+tnB5YFM9xr0aH7Sg1K/2xoeZ4x7Vi7N2+Zp6zvu65uTfi9lFbjTab87Tir578Mbf+n50n9V1H19VQBHjWv8JZjfz39SwDZ+v1+9evUiv8lS38Wc65N/vHRhldTnxqapB99eIOvrKY++8dzOHyO9OapJH5r8AvYiv0hJJT+WTvh1/ONr90uLfyp9fFfT1dQEUja/y38pPZUt7a84uGzrO9Ibw6Ty+fFto+rf0t980vuXNU2NaJy9X0kLLpb8GdKFO90PnLte6H6pUHKnbT3mB7ENv8ha6bWh7hSCB4SOk/r8Inb6T0ulbH7r0rq/1Otaqf03voC6b7s7xatT7U1daJgxR35+PrlbWjzZvTZnC2JNfrctlDbOkZ5p5z6XWYXS4DnSsdccPtbnkzJax56VC6QYLzLLGX8AICkjI8PrElqG9ye5H56cu65x90vLbpp6kBiFw6SzPzp8eVquJKfJH578AvYiv0hZh54pdCS+gDQizoZSM5OS+c09xr12ce2eg8v2lEmVK+M/E+HAlHPZHRNeHo5Cekj68gUpPS+2aXcgo3VlNbtI2rnenXlEcqeKzOniTluXIlIyv3XJbCsNvP/w5esellbcIo35sO7resJb2xZJb39POuEOqed/H77eOO61OvdscaeHbGGsyG+3ie6XKNJyDv6N7f6jusfuKZP2bXNnP2DacyBhmvnXA+zjOE3/ASeAxHIcRytXriS/yZBZILXqLNVUxX+f3V9I2xc37j5oHk79u3TqP5v0IcgvYC/yi5T1/uXSa4PjG+sPuNe/OXANHEukbH5LfiKd/lLsdI5dL3SnICsaGd82sguli/e7H1jDe4EMaexnUn5/qfJjd9nLJ0jzhrvNg6dbuZk9VHpQOu8L6fhb3Mbgm2dJrw5MeulNJWXzeyTGxL6fLBgkHfdzKbuDdzWhfrndpZzO7nU36+LzS6e/Ig1//euZZ1oOa/KbXSQNfkQ6+cHY5U714V+k+ewJ6eV+UsWy5NUHJJkXmaXxBwBInpPukUa82bgX5xsfk+Z+R4qsabq68O04NdKaB6XNz3tdCQAA3vMF3A8ra/c2PDayjmva2KIx1+7zp3Otv+Zk1wZp6TXSl1+/Vs3vL4X6uo2g4lFS8LjD7xPIdP91qqVuP5K6TUhevUis/WHphR7SkkOmGCwaKQ24V8pq711dqF9WO2nMUqnbD+sf4w+4zSXYY/0s6akcadv7scsLhkjHT3fP+AOQMKkzTwEAIDUVjXCn1WnV1etKUB9fQFp2ndTxP6VOY91lXy2Uyue5U3y06uxtfQAAJNN3/hxf08epld441f2g68wFTV8XGlazW1r1G7cpdOAD59JH3bNPCocd+b5ovkJ9pTPmHryu39DHD6477bn677flNenDG6XTnpVyuzVpiWhCGSEp1Edq1cXrSpAIxkjvXyoVniF1v9TratAYud3ds22/mcV2Q90bgISi8QcASJ7qXdLaP0h5PaUuF8R3n4LB7g3Nl88njXxbyi4+uKzsNWnl7VKHc2j8AQBalnjP9HL2SSX/LWUVNm09iJ8/Q1r9W6nTeW7jz6mRPviJ1OFsGn82S89zz+xrLJ/fPVuwcgWNP9ud/mLs78tvknaWupcmQPNU9W9pxa1S4XCp2yUHl+/b7s4KFMii8WebwuFS+9PdszUBNDmfMcZ4XUQqiEQiCoVCqqysVCgU8rocAI1gjJHjOPL7/fIxJU/Tqt0vPZ0jdTxXOu2Zw9fv+sw9s4/nwX7VO93pWfOPb9ILdJNfwF7kFylt8/PS509LQx5LyQ+4Ujq/O9dL2R2ltGy38ffVAve1DF9ESx1bXpf+/bLU7rvStvekY6+uu7FnjFSzy20cppCUzm+83vpPt6F73iavK0F99oelf7aVelwhfeeh2HU1Ve502pltvKnNQymT3y2vS/n93Ola3x4rZRa41wQEUlQ4HFZ+fr7C4bCCwWBSHpNr/AGApP3793tdQssQyJBGL5EGP3pwWfl8af8O9+dlU6Q3hsVeeP3d/5LeOT+pZeIoOLUHn0fJ/YCk7clN2vQ7gPwC9iK/SFnbF0ub/k+KfFL3emOsv7ZfyuY3r8Rt+knudPOFp9P0SzXb3pfW3C99/jfp0/9Pqo7UPc7nS7mm3wEpm98j+fguaf7Z7s+nv0jTr7nLCEnnfXl400+S0nJaZNPvAOvzu2O5NH+UtHSK+/veMmnvVi8rAlISjb8EcxzH6xIANJLjOFqzZg35TZbWJ7ov4iVpT5k0b7i07Hr3A7D8/u50V2k5B8fX7nFvaN7eHSf9s737PBojhT9xv4XZxMgvYC/yi5TW61rp+9vcb7PXpWKp9Gyx9PlTya0rQVI6v061tG2RO3NBTZVkUnAfW7qeV0ljP3fPyD13oxTs7XVFSZXS+T2S8CdSeGXsl0zRvGXXMRV2ZI17DHWqk19PM5AS+W19onTyH6V+093fz1oknf7/PC0JaGpeZJZr/AEAkqt2n3utjJzO7nVUBj4ghY53v1F7wq8O/3Dl9Je8qRONUzzKbdo61e63pl/qK/WYJA36i9eVAQCQfFntjry+OuxeGzenU3LqQfx2bZReGywde40kn7R+pnRuqZTT0evKkChZBQd/zm3lXR1Iru/Mcmcj8fmkL55134+2PdnrqnAktfvdaXnTWknFZ7rLVv9eKv2LNO6r2CzDLsdO9roCIOXR+AMAJNeGR6XFV0mnvyJ1GC31+lnsep9f2vqutOGv0oDfSxmtPSkTjXToC/dqSX1/KbX9jmflAADguaovpfI3pS4XudOdH6pohHT2CvcseTQveT2l/ndJhcOlyo+kojOlrCKvq0IiOTVS1WZp90Ypr5eU08HripAMB6bwNY707velTufVfd15NCNGeu8HUsHQg42/rhdLud2lzLbeloZvzzju5z61e6Xis6S8Hl5XBKQUGn8AICkQCHhdQstRMFTqc5PUqov7Qs9Xx6zTlSulDbPdF397y6WCIXwb0yZZBVL/O5P2cOQXsBf5RUor/Yu08nYpt0RqN+TgcmPcM06kg/9aKGXz6/NJfae5PxcMkkp+4m09SLxdG6UXj3V/bvsdd5q5FiZl89uQsnnuVJH/8aQ7Wwmat0Cm+1wdOh1v0Qj31oKlTH7X/lFaeq3785DHafwBCeYzhq8YJkIkElEoFFI4HFYwGPS6HABo/pwa6bmOUtcfSAPvi123r0Kq2S3t3yG90l/qd7vU7zZPykScvnpPWvs/Uu/rpTYDvK4GAADvhT+VdiyTikdLmW0OLn/nfCm7g3t9G4sbfynPGMnZ504NiNRSvUtaeZu0Y7nUdbxU8mOvK0KyzB8tlb8lXRB2m0oAvLOnXFpxi9T2FKnDOUypjZTmRe+IM/4SjD4qYB9jjHbu3Km8vDz5+PAlefZXSPkn1H0NnMw27i2jtTTiLa5/Y4M9/5Y+/5vU6Vxp+S/caZNO+WOTPyz5BexFfpHyQse5t0PVVEl7yiR/ptVNv5TP7+7PpeePcX8+/WWpwxhPy0GCpedKA+71ugrPpHx+j+SEOyRfQPKne10JGqNmj1S1yf0ixhunS71vaLHXiEup/GYXSoNmeV0FkBRe9IzqmF8N34bjOF6XAKCRHMfRhg0byG8yLb1O+tfF0vDX3WvB1WV/WCqfJ+X3Y8oHG3Q8V7poj9R5nLRvm9vYTQLyC9iL/KLF2FchObXuz2k50qj3pMGPeFvTt5Ty+c0+5Etnuz/3rg6gCaR8fo+k7SnuzDPPFkuldv8dblFe7ueeLV+9U0oPSf6Mhu+Tolp0fgGLeZFZzvgDACTfvm3ut92dGslfz6Hoi39Ii34sDX1SOubi5NaHxgsc8uZrzIfu9FgAALR0a/4gLZ3iXkMsdLyUlu2e6ZfWyuvKcCT+gPQD455lIj5cTUmvDHSn4h3+Rou/XljL83WmM1p7Wwbid+xkyThSqI909nKvqwEAK3DGHwAg+YY85r7B3vhY/WOKR0tZRdJ746XKlcmrDUfHqZG2LZLCn7i/2z7tCAAAidB6gNTlQimyVnq+s/SvH0i1+7yuCvFKy6ZJm6p2LHP/dWq8rQPJt+x6t4nU6XyvK0G8jrvOvZa8j4+xASBenPEHAJKysrK8LqFlqdkprXtI6nKR1OPyusfkdJQG3i9telrKKkxqeTgKTrX02mD35yGPSd0mJO2hyS9gL/KLlNf+u+7NqZU2PyMd8wMpkOl1VQlBfmG1H7Ts2SladH67/UjaWyY5+9xrxsEeG/9Xqo5IPa9q0V80bdH5BRA3n/HiyoIpKBKJKBQKKRwOKxgMel0OADR/1TvdF+05Hb2uBImy+l7pwxvc6zKevcLragAAAAAAqeCjm6WP75LS86ULdnhdDQA0ihe9I86RTjAurgrYx3Ecbd++nfwmW3oeTb9U0/t6aexn0qDZSXtI8gvYi/wC9iK/gL3IL6yUHpJyukiD/uJ1JZ4iv4CdvMgsjb8E4wRKwD7GGH3xxRfkF0iEVl2lticn7eHIL2Av8gvYi/wC9iK/sFKfG6XzPpe6fN/rSjxFfgE7eZFZGn8AACAx3jxLeq6r11UAAAAAAAAALRaNPwAAkBj+NKlqk+TUeF0JAAAAAAAA0CKleV0AADQHeXl5XpcA2O/UZyVnr9sATCLyC9iL/AL2Ir+AvcgvYC/yCyAePsOkwAkRiUQUCoUUDocVDAa9LgcAAAAAAAAAAAAe8qJ3xFSfCeY4jtclAGgkx3FUVlZGfgELkV/AXuQXsBf5BexFfgF7kV/ATl5klsZfgnECJWAfY4zKysrIL2Ah8gvYi/wC9iK/gL3IL2Av8gvYyYvM0vgDAAAAAAAAAAAAUgCNPwAAAAAAAAAAACAF0PhLMJ/P53UJABrJ5/OpTZs25BewEPkF7EV+AXuRX8Be5BewF/kF7ORFZn2GSYETIhKJKBQKKRwOKxgMel0OAAAAAAAAAAAAPORF74gz/hLMcRyvSwDQSI7jaNOmTeQXsBD5BexFfgF7kV/AXuQXsBf5BezkRWZp/CUYJ1AC9jHGqKKigvwCFiK/gL3IL2Av8gvYi/wC9iK/gJ28yCyNPwAAAAAAAAAAACAF0PgDAAAAAAAAAAAAUgCNvwTz+XxelwCgkXw+n4qKisgvYCHyC9iL/AL2Ir+AvcgvYC/yC9jJi8z6DJMCJ0QkElEoFFI4HFYwGPS6HAAAAAAAAAAAAHjIi94RZ/wlWG1trdclAGik2tpalZaWkl/AQuQXsBf5BexFfgF7kV/AXuQXsJMXmaXxBwCSdu7c6XUJAI4S+QXsRX4Be5FfwF7kF7AX+QUQDxp/AAAAAAAAAAAAQAqg8QcAAAAAAAAAAACkABp/Cebz+bwuAUAj+Xw+de7cmfwCFiK/gL3IL2Av8gvYi/wC9iK/gJ28yKzPGGOS/qgpKBKJKBQKKRwOKxgMel0OAAAAAAAAAAAAPORF74gz/hKstrbW6xIANFJtba0+/fRT8gtYiPwC9iK/gL3IL2Av8gvYi/wCdvIiszT+AEDS3r17vS4BwFEiv4C9yC9gL/IL2Iv8AvYivwDiQeMPAAAAAAAAAAAASAE0/gAAAAAAAAAAAIAUQOMvwfx+/pMCtvH7/erevTv5BSxEfgF7kV/AXuQXsBf5BexFfgE7eZHZtKQ/Yorz+XxelwCgkXw+n4LBoNdlADgK5BewF/kF7EV+AXuRX8Be5Bewkxc9I74ekGC1tbVelwCgkWpra7Vy5UryC1iI/AL2Ir+AvcgvYC/yC9iL/AJ28iKzNP4AQDTtAZuRX8Be5BewF/kF7EV+AXuRXwDxoPEHAAAAAAAAAAAApAAafwAAAAAAAAAAAEAK8BljjNdFpIJIJKJQKKTKykqFQiGvywHQCMYY7d27V1lZWZ5cbBXA0SO/gL3IL2Av8gvYi/wC9iK/gJ3C4bDy8/MVDocVDAaT8pic8QcAkjIyMrwuAcBRIr+AvcgvYC/yC9iL/AL2Ir8A4kHjL8Ecx/G6BACN5DiOVq5cSX4BC5FfwF7kF7AX+QXsRX4Be5FfwE5eZJbGHwAAAAAAAAAAAJACaPwBAAAAAAAAAAAAKYDGHwAAAAAAAAAAAJACfMYY43URqSASiSgUCqmyslKhUMjrcgA0gjFGjuPI7/fL5/N5XQ6ARiC/gL3IL2Av8gvYi/wC9iK/gJ3C4bDy8/MVDocVDAaT8pic8QcAkvbv3+91CQCOEvkF7EV+AXuRX8Be5BewF/kFEA8afwnmOI7XJQBoJMdxtGbNGvILWIj8AvYiv4C9yC9gL/IL2Iv8AnbyIrM0/gAAAAAAAAAAAIAUQOMPAAAAAAAAAAAASAE0/gBAUiAQ8LoEAEeJ/AL2Ir+AvcgvYC/yC9iL/AKIh88YY7wuIhVEIhGFQiGFw2EFg0GvywEAAAAAAAAAAICHvOgdccZfgtFHBexjjFEkEiG/gIXIL2Av8gvYi/wC9iK/gL3IL2AnLzJL4y/BHMfxugQAjeQ4jjZs2EB+AQuRX8Be5BewF/kF7EV+AXuRX8BOXmSWxh8AAAAAAAAAAACQAmj8AQAAAAAAAAAAACmAxh8ASMrKyvK6BABHifwC9iK/gL3IL2Av8gvYi/wCiIfPcDXQhIhEIgqFQgqHwwoGg16XAwAAAAAAAAAAAA950TvijL8E4+KqgH0cx9H27dvJL2Ah8gvYi/wC9iK/gL3IL2Av8gvYyYvM0vhLME6gBOxjjNEXX3xBfgELkV/AXuQXsBf5BexFfgF7kV/ATl5klsYfAAAAAAAAAAAAkAJo/AEAAAAAAAAAAAApgMYfAEjKy8vzugQAR4n8AvYiv4C9yC9gL/IL2Iv8AoiHzzApcEJEIhGFQiGFw2EFg0GvywEAAAAAAAAAAICHvOgdccZfgjmO43UJABrJcRyVlZWRX8BC5BewF/kF7EV+AXuRX8Be5BewkxeZpfGXYJxACdjHGKOysjLyC1iI/AL2Ir+AvcgvYC/yC9iL/AJ28iKzNP4AAAAAAAAAAACAFEDjDwAAAAAAAAAAAEgBNP4SzOfzeV0CgEby+Xxq06YN+QUsRH4Be5FfwF7kF7AX+QXsRX4BO3mRWZ9hUuCEiEQiCoVCCofDCgaDXpcDAAAAAAAAAAAAD3nRO+KMvwRzHMfrEgA0kuM42rRpE/kFLER+AXuRX8Be5BewF/kF7EV+ATt5kVkafwnGCZSAfYwxqqioIL+AhcgvYC/yC9iL/AL2Ir+AvcgvYCcvMkvjDwAAAAAAAAAAAEgBaV4XkCoOdG0jkYgCgYDH1QBojNraWu3atYv8AhYiv4C9yC9gL/IL2Iv8AvYiv4CdIpGIpOSe+UfjL0G2b98uSTrmmGO8LQQAAAAAAAAAAADNxvbt2xUKhZLyWDT+EqRNmzaSpE2bNiXtyQOQGJFIRJ07d9YXX3yhYDDodTkAGoH8AvYiv4C9yC9gL/IL2Iv8AnYKh8Pq0qVLtIeUDDT+EsTvdy+XGAqF+MMLWCoYDJJfwFLkF7AX+QXsRX4Be5FfwF7kF7DTgR5SUh4raY8EAAAAAAAAAAAAoMnQ+AMAAAAAAAAAAABSAI2/BMnMzNRtt92mzMxMr0sB0EjkF7AX+QXsRX4Be5FfwF7kF7AX+QXs5EV2fcYYk7RHAwAAAAAAAAAAANAkOOMPAAAAAAAAAAAASAE0/gAAAAAAAAAAAIAUQOMPAAAAAAAAAAAASAE0/gAAAAAAAAAAAIAUQOPvELW1tbr11lvVrVs3ZWdnq0ePHrrjjjtkjImOuf3223XcccepVatWat26tUaOHKlFixbFbKeiokKXXHKJgsGg8vPzNWnSJO3atStmzIoVK3TqqacqKytLnTt31m9/+9uk7COQiuLJ7qGuvPJK+Xw+3X///THLyS6QfPHk99JLL5XP54u5jR49OmY75BdIvniPv6tXr9a5556rUCikVq1a6ZRTTtGmTZui6/fu3avJkyerbdu2ys3N1fe//32Vl5fHbGPTpk0655xzlJOTo/bt22vq1KmqqalJyn4CqSie/H7z2Hvg9rvf/S46huMvkHzx5HfXrl26+uqr1alTJ2VnZ6tPnz56+OGHY7bD8RdIvnjyW15erksvvVQdOnRQTk6ORo8erXXr1sVsh/wC3ti5c6emTJmirl27Kjs7W0OHDtXixYuj640xmj59uoqLi5Wdna2RI0celt+kvX42iLrzzjtN27ZtzYsvvmg2btxo/v73v5vc3FzzwAMPRMc88cQT5vXXXzelpaVm1apVZtKkSSYYDJqtW7dGx4wePdr079/fvP/+++bdd981JSUlZvz48dH14XDYFBYWmksuucSsWrXKPPnkkyY7O9vMnDkzqfsLpIp4snvAM888Y/r37286dOhg7rvvvph1ZBdIvnjyO3HiRDN69GizZcuW6K2ioiJmO+QXSL548rt+/XrTpk0bM3XqVLNs2TKzfv168/zzz5vy8vLomCuvvNJ07tzZzJs3zyxZssQMHjzYDB06NLq+pqbGHH/88WbkyJHmww8/NC+//LIpKCgw06ZNS+r+AqkknvweetzdsmWLmT17tvH5fKa0tDQ6huMvkHzx5PeKK64wPXr0MPPnzzcbN240M2fONIFAwDz//PPRMRx/geRrKL+O45jBgwebU0891XzwwQfm008/NT/5yU9Mly5dzK5du6LbIb+ANy688ELTp08f8/bbb5t169aZ2267zQSDQbN582ZjjDEzZswwoVDIPPfcc+ajjz4y5557runWrZvZs2dPdBvJev1M4+8Q55xzjrn88stjlo0bN85ccskl9d4nHA4bSeaNN94wxhjzySefGElm8eLF0TGvvPKK8fl85ssvvzTGGPOnP/3JtG7d2uzbty865he/+IXp1atXIncHaDHize7mzZtNx44dzapVq0zXrl1jGn9kF/BGPPmdOHGiGTt2bL3bIL+AN+LJ70UXXWR++MMf1ruNyspKk56ebv7+979Hl61evdpIMgsXLjTGGPPyyy8bv99vysrKomMeeughEwwGYzINIH5H89537NixZvjw4dHfOf4C3ognv3379jW//vWvY8YMGDDA3HzzzcYYjr+AVxrK75o1a4wks2rVquj62tpa065dOzNr1ixjDPkFvFJVVWUCgYB58cUXY5YfOL46jmOKiorM7373u+i6yspKk5mZaZ588kljTHJfPzPV5yGGDh2qefPmae3atZKkjz76SAsWLNCYMWPqHL9//379+c9/VigUUv/+/SVJCxcuVH5+vk4++eTouJEjR8rv90enBF24cKFOO+00ZWRkRMecddZZWrNmjXbs2NFUuwekrHiy6ziOJkyYoKlTp6pv376HbYPsAt6I99j71ltvqX379urVq5euuuoqbd++PbqO/ALeaCi/juPopZde0rHHHquzzjpL7du316BBg/Tcc89Ft7F06VJVV1dr5MiR0WXHHXecunTpooULF0py89uvXz8VFhZGx5x11lmKRCL6+OOPk7CnQOpp7Hvf8vJyvfTSS5o0aVJ0GcdfwBvx5Hfo0KF64YUX9OWXX8oYo/nz52vt2rUaNWqUJI6/gFcayu++ffskSVlZWdH7+P1+ZWZmasGCBZLIL+CVmpoa1dbWxuRTkrKzs7VgwQJt3LhRZWVlMdkMhUIaNGhQTDaT9fo57aj2MkXddNNNikQiOu644xQIBFRbW6s777xTl1xyScy4F198URdffLGqqqpUXFys119/XQUFBZKksrIytW/fPmZ8Wlqa2rRpo7KysuiYbt26xYw58Ie4rKxMrVu3bqpdBFJSPNm95557lJaWpp/97Gd1boPsAt6IJ7+jR4/WuHHj1K1bN5WWluqXv/ylxowZo4ULFyoQCJBfwCMN5Xfr1q3atWuXZsyYod/85je655579Oqrr2rcuHGaP3++hg0bprKyMmVkZCg/Pz9m24WFhTH5PfRDiwPrD6wD0Hjxvvc9YM6cOcrLy9O4ceOiyzj+At6IJ78PPvigfvKTn6hTp05KS0uT3+/XrFmzdNppp0kSx1/AIw3l90ADb9q0aZo5c6ZatWql++67T5s3b9aWLVskkV/AK3l5eRoyZIjuuOMO9e7dW4WFhXryySe1cOFClZSURLNVV/YOzWayXj/T+DvE008/rSeeeEJ/+9vf1LdvXy1fvlxTpkxRhw4dNHHixOi4M844Q8uXL9e2bds0a9YsXXjhhVq0aNFhTxqA5Ggou0uXLtUDDzygZcuWyefzeV0ugEPEc+y9+OKLo+P79eunE044QT169NBbb72lESNGeFU60OI1lF/HcSRJY8eO1XXXXSdJOvHEE/Xee+/p4Ycf1rBhw7wsH2jR4n3ve8Ds2bN1ySWXHPYNZwDJF09+H3zwQb3//vt64YUX1LVrV73zzjuaPHmyOnToEHMmAoDkaii/6enpeuaZZzRp0iS1adNGgUBAI0eO1JgxY2SM8bp8oMV7/PHHdfnll6tjx44KBAIaMGCAxo8fr6VLl3pd2mFo/B1i6tSpuummm6IfMPbr10+ff/657r777pg3P61atVJJSYlKSko0ePBg9ezZU4888oimTZumoqIibd26NWa7NTU1qqioUFFRkSSpqKhI5eXlMWMO/H5gDID4NZTdd999V1u3blWXLl2i96mtrdX111+v+++/X5999hnZBTwS77H3UN27d1dBQYHWr1+vESNGkF/AIw3lt6CgQGlpaerTp0/M/Xr37h2dqqioqEj79+9XZWVlzLeWy8vLY/L7wQcfxGyD/ALfTmOOv++++67WrFmjp556KmY5x1/AGw3ld8+ePfrlL3+pZ599Vuecc44k6YQTTtDy5cv1+9//XiNHjuT4C3gknuPvwIEDtXz5coXDYe3fv1/t2rXToEGDolMDkl/AOz169NDbb7+t3bt3KxKJqLi4WBdddJG6d+8ezVZ5ebmKi4uj9ykvL9eJJ54oKbmvn7nG3yGqqqrk98f+JwkEAtFvK9fHcZzoHMxDhgxRZWVlTJf3zTfflOM4GjRoUHTMO++8o+rq6uiY119/Xb169WKqE+AoNJTdCRMmaMWKFVq+fHn01qFDB02dOlVz586VRHYBrxzNsXfz5s3avn179IUU+QW80VB+MzIydMopp2jNmjUxY9auXauuXbtKcj/YSE9P17x586Lr16xZo02bNmnIkCGS3PyuXLky5g3S66+/rmAweFhTEUB8GnP8feSRRzRw4MDode0P4PgLeKOh/FZXV6u6uvqIYzj+At5ozPE3FAqpXbt2WrdunZYsWaKxY8dKIr9Ac9CqVSsVFxdrx44dmjt3rsaOHatu3bqpqKgoJpuRSESLFi2KyWbSXj8bRE2cONF07NjRvPjii2bjxo3mmWeeMQUFBebGG280xhiza9cuM23aNLNw4ULz2WefmSVLlpjLLrvMZGZmmlWrVkW3M3r0aHPSSSeZRYsWmQULFpiePXua8ePHR9dXVlaawsJCM2HCBLNq1Srzf//3fyYnJ8fMnDkz6fsMpIKGsluXrl27mvvuuy9mGdkFkq+h/O7cudPccMMNZuHChWbjxo3mjTfeMAMGDDA9e/Y0e/fujW6H/ALJF8/x95lnnjHp6enmz3/+s1m3bp158MEHTSAQMO+++250zJVXXmm6dOli3nzzTbNkyRIzZMgQM2TIkOj6mpoac/zxx5tRo0aZ5cuXm1dffdW0a9fOTJs2Lan7C6SSeF8/h8Nhk5OTYx566KE6t8PxF0i+ePI7bNgw07dvXzN//nyzYcMG8+ijj5qsrCzzpz/9KTqG4y+QfPHk9+mnnzbz5883paWl5rnnnjNdu3Y148aNi9kO+QW88eqrr5pXXnnFbNiwwbz22mumf//+ZtCgQWb//v3GGGNmzJhh8vPzzfPPP29WrFhhxo4da7p162b27NkT3UayXj/T+DtEJBIx1157renSpYvJysoy3bt3NzfffLPZt2+fMcaYPXv2mPPPP9906NDBZGRkmOLiYnPuueeaDz74IGY727dvN+PHjze5ubkmGAyayy67zOzcuTNmzEcffWS++93vmszMTNOxY0czY8aMpO0nkGoaym5d6mr8kV0g+RrKb1VVlRk1apRp166dSU9PN127djVXXHGFKSsri9kO+QWSL97j7yOPPGJKSkpMVlaW6d+/v3nuuedi1u/Zs8f89Kc/Na1btzY5OTnm/PPPN1u2bIkZ89lnn5kxY8aY7OxsU1BQYK6//npTXV3d5PsIpKp48ztz5kyTnZ1tKisr69wOx18g+eLJ75YtW8yll15qOnToYLKyskyvXr3MvffeaxzHiY7h+AskXzz5feCBB0ynTp1Menq66dKli7nlllsOOz6TX8AbTz31lOnevbvJyMgwRUVFZvLkyTGvkx3HMbfeeqspLCw0mZmZZsSIEWbNmjUx20jW62efMVwZFAAAAAAAAAAAALAd1/gDAAAAAAAAAAAAUgCNPwAAAAAAAAAAACAF0PgDAAAAAAAAAAAAUgCNPwAAAAAAAAAAACAF0PgDAAAAAAAAAAAAUgCNPwAAAJz+FvEAAAXpSURBVAAAAAAAACAF0PgDAAAAAAAAAAAAUgCNPwAAAAAAAAAAACAF0PgDAAAAgBR36aWX6rzzzvO6jKNme/0AAAAAkCxpXhcAAAAAADh6Pp/viOtvu+02PfDAAzLGJKmig9566y2dccYZ2rFjh/Lz85P++AAAAADQ0tD4AwAAAACLbdmyJfrzU089penTp2vNmjXRZbm5ucrNzfWiNAAAAABAkjHVJwAAAABYrKioKHoLhULy+Xwxy3Jzcw+bKvP000/XNddcoylTpqh169YqLCzUrFmztHv3bl122WXKy8tTSUmJXnnllZjHWrVqlcaMGaPc3FwVFhZqwoQJ2rZtW9y1/vWvf1V+fr7mzp2r3r17Kzc3V6NHj45pXtbW1urnP/+58vPz1bZtW914442Hna3oOI7uvvtudevWTdnZ2erfv7/+8Y9/SJKMMRo5cqTOOuus6P0qKirUqVMnTZ8+vbH/eQEAAADAKjT+AAAAAKAFmjNnjgoKCvTBBx/ommuu0VVXXaULLrhAQ4cO1bJlyzRq1ChNmDBBVVVVkqTKykoNHz5cJ510kpYsWaJXX31V5eXluvDCCxv1uFVVVfr973+vxx9/XO+88442bdqkG264Ibr+3nvv1V//+lfNnj1bCxYsUEVFhZ599tmYbdx999167LHH9PDDD+vjjz/Wddddpx/+8Id6++235fP5NGfOHC1evFh/+MMfJElXXnmlOnbsSOMPAAAAQMpjqk8AAAAAaIH69++vW265RZI0bdo0zZgxQwUFBbriiiskSdOnT9dDDz2kFStWaPDgwfrjH/+ok046SXfddVd0G7Nnz1bnzp21du1aHXvssXE9bnV1tR5++GH16NFDknT11Vfr17/+dXT9/fffr2nTpmncuHGSpIcfflhz586Nrt+3b5/uuusuvfHGGxoyZIgkqXv37lqwYIFmzpypYcOGqWPHjpo5c6Z+9KMfqaysTC+//LI+/PBDpaXxFhgAAABAauNdDwAAAAC0QCeccEL050AgoLZt26pfv37RZYWFhZKkrVu3SpI++ugjzZ8/v87rBZaWlsbd+MvJyYk2/SSpuLg4+hjhcFhbtmzRoEGDouvT0tJ08sknR6ftXL9+vaqqqnTmmWfGbHf//v066aSTor9fcMEFevbZZzVjxgw99NBD6tmzZ1z1AQAAAIDNaPwBAAAAQAuUnp4e87vP54tZ5vP5JLnX05OkXbt26Xvf+57uueeew7ZVXFz8rR73m9fwO5Jdu3ZJkl566SV17NgxZl1mZmb056qqKi1dulSBQEDr1q2Le/sAAAAAYDMafwAAAACABg0YMED//Oc/dcwxxzTZlJmhUEjFxcVatGiRTjvtNElSTU2Nli5dqgEDBkiS+vTpo8zMTG3atEnDhg2rd1vXX3+9/H6/XnnlFZ199tk655xzNHz48CapGwAAAACaC7/XBQAAAAAAmr/JkyeroqJC48eP1+LFi1VaWqq5c+fqsssuU21tbcIe59prr9WMGTP03HPP6dNPP9VPf/pTVVZWRtfn5eXphhtu0HXXXac5c+aotLRUy5Yt04MPPqg5c+ZIcs8GnD17tp544gmdeeaZmjp1qiZOnKgdO3YkrE4AAAAAaI5o/AEAAAAAGtShQwf961//Um1trUaNGqV+/fppypQpys/Pl9+fuLeW119/vSZMmKCJEydqyJAhysvL0/nnnx8z5o477tCtt96qu+++W71799bo0aP10ksvqVu3bvrqq680adIk3X777dGzBH/1q1+psLBQV155ZcLqBAAAAIDmyGcaczEFAAAAAAAAAAAAAM0SZ/wBAAAAAAAAAAAAKYDGHwAAAAAAAAAAAJACaPwBAAAAAAAAAAAAKYDGHwAAAAAAAAAAAJACaPwBAAAAAAAAAAAAKYDGHwAAAAAAAAAAAJACaPwBAAAAAAAAAAAAKYDGHwAAAAAAAAAAAJACaPwBAAAAAAAAAAAAKYDGHwAAAAAAAAAAAJACaPwBAAAAAAAAAAAAKeD/B6kcLRxacROcAAAAAElFTkSuQmCC\n" + }, + "metadata": {} + }, + { + "output_type": "display_data", + "data": { + "text/plain": [ + "
" + ], + "image/png": "iVBORw0KGgoAAAANSUhEUgAABv4AAAN5CAYAAADAfkzvAAAAOnRFWHRTb2Z0d2FyZQBNYXRwbG90bGliIHZlcnNpb24zLjEwLjAsIGh0dHBzOi8vbWF0cGxvdGxpYi5vcmcvlHJYcgAAAAlwSFlzAAAPYQAAD2EBqD+naQABAABJREFUeJzs3XdYFFfbBvB76IuwIIhioVlBxd6ISbBjLLH3AvbXErtRX401tqhRE2OPJZZYU0ywG7FrbCg2oiiWKKLSRPru+f7w23ldKYLusLrev+viinvm7MxzZoaHDQ/njCSEECAiIiIiIiIiIiIiIiKi95qZsQMgIiIiIiIiIiIiIiIiorfHwh8RERERERERERERERGRCWDhj4iIiIiIiIiIiIiIiMgEsPBHREREREREREREREREZAJY+CMiIiIiIiIiIiIiIiIyASz8EREREREREREREREREZkAFv6IiIiIiIiIiIiIiIiITAALf0REREREREREREREREQmgIU/IiIiIiIiIiIiIiIiIhPAwh8RERERvVeCgoLg6elp7DCM5kMfv5LWrl0LSZIQGRlp7FDeeZGRkZAkCWvXrjVaDM2aNUO/fv2Mdvx3TUhICCRJQkhIiLFDofdceno63NzcsGTJEmOHQkRERERvgIU/IiIiIjI6SZJy9fWu/kI7MjISvXr1QqlSpWBjYwNXV1d8+umnmDx5srFDM7hatWpBkiQsXbrU2KG884KCgvTuXzs7O5QsWRLt27fHjh07oNVq33jfu3btwpQpUwwXbDY2bdqEhQsXKn6cvDp+/Dj27duHsWPHym2enp6vzSFBQUHGC/odoStwnz17VvFjJSYmYvjw4ShRogSsra3h4+OTbe6Ii4tD//794eLiggIFCqB+/fo4f/58ln137tyJatWqwcbGBu7u7pg8eTIyMjLeap+vWrJkiSKF7fT0dEydOhUlS5aEtbU1SpYsia+//jrL+AHg/Pnz+Pzzz+Hk5ARbW1tUrFgR3333XaZ+J06cwMcffwxbW1u4urpi6NChSExMzNQvNTUVY8eORbFixaBSqVC7dm3s379fr4+lpSVGjhyJGTNmICUlxTADJyIiIqJ8Y2HsAIiIiIiI1q9fr/f6p59+wv79+zO1+/j4YOXKlW9VMDG0mzdvombNmlCpVOjduzc8PT3x8OFDnD9/HnPmzMHUqVMNejxjjv/GjRs4c+YMPD09sXHjRgwcONAocbxPrK2tsWrVKgBAcnIy7ty5gz/++APt27dHvXr18Pvvv0OtVud5v7t27cIPP/ygePFv06ZNuHz5MoYPH67X7uHhgeTkZFhaWip6/OzMnTsXDRs2ROnSpeW2hQsXZlnoAIDFixfj9OnTqFOnTn6F+MHTaDQICAjA2bNnMXjwYJQpUwZ79+7FoEGDEBsbi//+979yX61Wi+bNm+PixYsYM2YMChUqhCVLlqBevXo4d+4cypQpI/fdvXs3WrdujXr16uH7779HWFgYvv76a0RHR+sVFfOyz6wsWbIEhQoVMnixuHv37ti2bRt69+6NGjVq4NSpU/jqq69w9+5drFixQq/vvn370LJlS1StWhVfffUV7OzsEBERgfv37+v1Cw0NRcOGDeHj44Nvv/0W9+/fx7x583Djxg3s3r1br29QUBC2b9+O4cOHo0yZMli7di2aNWuGQ4cO4eOPP5b79erVC+PGjcOmTZvQu3dvg54DIiIiIlKYICIiIiJ6xwwePFi8Lx9VBw0aJCwsLERkZGSmbY8ePTLYcRITEw22rzc1adIkUbhwYbFjxw4hSZK4ffu2sUMyqDVr1ggABhtXYGCgKFCgQJbbZs2aJQCIjh07vtG+8+t7pHnz5sLDw0Px4+TFo0ePhIWFhVi1alWu+u/du1dIkiQ+//xzhSMzrkOHDgkA4tChQzn2093nZ86cUTSerVu3CgDixx9/1Gtv166dsLGx0cuPW7ZsEQDEtm3b5Lbo6Gjh6OgounTpovf+8uXLi8qVK4v09HS5bcKECUKSJHHt2rU32mdWKlSoIPz9/XM93tz4+++/BQDx1Vdf6bWPGjVKSJIkLl68KLfFx8eLIkWKiDZt2giNRpPjfj/77DNRtGhRER8fL7etXLlSABB79+6V206fPi0AiLlz58ptycnJolSpUsLPzy/Tflu0aCE++eSTPI+TiIiIiIyLS30SERER0Xvl1Wfc6Z41Nm/ePPzwww8oWbIkbG1t0aRJE9y7dw9CCEyfPh0lSpSASqVCq1atEBMTk2m/u3fvxieffIICBQrA3t4ezZs3x5UrV14bT0REBEqUKAEPD49M2woXLvxGxwkKCpJndjRr1gz29vbo1q1bluMHXsxsWbhwISpUqAAbGxsUKVIEAwYMQGxsrF6/s2fPIiAgAIUKFYJKpYKXl1eeZnJs2rQJ7du3R4sWLeDg4IBNmzZl6jNlyhRIkoSbN28iKCgIjo6OcHBwQK9evZCUlKTXNyMjA9OnT0epUqVgbW0NT09P/Pe//0VqaqpeP09PT7Ro0QIhISGoUaMGVCoVfH195aVff/nlF/j6+sLGxgbVq1fHhQsX9N5/6dIlBAUFoWTJkvJSrL1798bTp09zHG9gYCAKFSqE9PT0TNuaNGmCcuXK5ea0ZWncuHFo0qQJtm3bhn/++Udv2+vukaCgIPzwww8A9JfJ1cnt/aA7lr+/P+zt7aFWq1GzZk35utarVw/BwcG4c+eOfAzdvZfdM/7++usvOXZHR0e0atUK165d0+uTl3skK8HBwcjIyECjRo1e2zcqKgo9evRA8eLFsWbNGr1tub3/gBezvypUqABra2sUK1YMgwcPRlxcnF6fevXqoWLFirh06RL8/f1ha2uL0qVLY/v27QCAw4cPo3bt2lCpVChXrhwOHDiQ6Tj//vsvevfujSJFisDa2hoVKlTA6tWrM/W7f/8+WrdujQIFCqBw4cIYMWJElnHnli7n/Pvvv2jdujXs7Ozg4uKC0aNHQ6PR6PV9+PAhrl+/nuX3xcuOHj0KAOjcubNee+fOnZGSkoLff/9dbtu+fTuKFCmCtm3bym0uLi7o2LEjfv/9d3lsV69exdWrV9G/f39YWPxvAaNBgwZBCCGf67zsMyuenp64cuUKDh8+LN/79erVk7ffunULHTp0kJffrFOnDoKDg3M8H687J0IIbNmyRW7btGkTHj16hBkzZsDMzAzPnz/PcrZ3QkIC9u/fj+7du+vNHu7Zsyfs7OywdetWvXNibm6O/v37y202Njbo06cPTp48iXv37untu3Hjxjh27FiWPzOJiIiI6N3Fwh8RERERmYSNGzdiyZIl+OKLLzBq1CgcPnwYHTt2xMSJE7Fnzx6MHTsW/fv3xx9//IHRo0frvXf9+vVo3rw57OzsMGfOHHz11Ve4evUqPv74Y0RGRuZ4XA8PD9y7dw9//fXXa2PMy3EyMjIQEBCAwoULY968eWjXrl22+x0wYADGjBmDunXrYtGiRejVqxc2btyIgIAA+Zfz0dHRaNKkCSIjIzFu3Dh8//336NatG06dOvXauAHg9OnTuHnzJrp06QIrKyu0bdsWGzduzLZ/x44d8ezZM8yaNQsdO3bE2rVrMy172rdvX0yaNAnVqlXDggUL4O/vj1mzZmX6pTjwYknVrl27omXLlpg1axZiY2PRsmVLbNy4ESNGjED37t0xdepUREREoGPHjnq/IN+/fz9u3bqFXr164fvvv0fnzp2xefNmNGvWDEKIbMfQo0cPPH36FHv37tVrj4qKwl9//YXu3bvn6tzltH8hhN7ztXJzjwwYMACNGzeW++u+dHJzPwAvnvXWvHlzxMTEYPz48Zg9ezaqVKmCPXv2AAAmTJiAKlWqoFChQvIxcnre34EDBxAQEIDo6GhMmTIFI0eOxIkTJ1C3bt0sv49yc49k5cSJE3B2ds6y2P4yrVaL7t274+nTp9i0aROcnJz0tuf2/psyZQoGDx6MYsWKYf78+WjXrh2WL1+OJk2aZCp+xcbGokWLFqhduza++eYbWFtbo3PnztiyZQs6d+6MZs2aYfbs2Xj+/Dnat2+PZ8+eye999OgR6tSpgwMHDmDIkCFYtGgRSpcujT59+uid9+TkZDRs2BB79+7FkCFDMGHCBBw9ehRffvnla89dTnRLczo7O2PevHnw9/fH/PnzMy0/OX78ePj4+ODff//NcX+pqakwNzeHlZWVXrutrS0A4Ny5c3LbhQsXUK1aNZiZ6f+KolatWkhKSpKL47qifo0aNfT6FStWDCVKlNAr+ud2n1lZuHAhSpQoAW9vb/nenzBhAoAX1+mjjz6Sly3VPQfv888/x6+//vracwIAKpXqtefkwIEDUKvV+Pfff1GuXDnY2dlBrVZj4MCBes/dCwsLQ0ZGRqZzYmVlhSpVqmQ6J2XLls20vHCtWrUAvFgy9GXVq1eHEAInTpzIcVxERERE9I4x5nRDIiIiIqKs5LSMYWBgoN7Sg7dv3xYAhIuLi4iLi5Pbx48fLwBkWhKuS5cuwsrKSqSkpAghhHj27JlwdHQU/fr10ztOVFSUcHBwyNT+qsuXLwuVSiUAiCpVqohhw4aJ3377TTx//lyvX16OExgYKACIcePGvXb8R48eFQDExo0b9frt2bNHr/3XX399q+X9hgwZItzc3IRWqxVCCLFv3z4BQFy4cEGv3+TJkwUA0bt3b732Nm3aCGdnZ/l1aGioACD69u2r12/06NECgPjrr7/kNg8PDwFAnDhxQm7bu3evACBUKpW4c+eO3L58+fJMyx0mJSVlGs/PP/8sAIgjR47Iba8u9anRaESJEiVEp06d9N777bffCkmSxK1bt7I6VbKclvoUQogLFy4IAGLEiBFCiLzdI9l9j+T2foiLixP29vaidu3aIjk5Wa+v7hoLkf1Sn7rvuzVr1shtVapUEYULFxZPnz6V2y5evCjMzMxEz5495bbc3iPZ+fjjj0X16tVf22/atGkCgJg6dWqmbbm9/6Kjo4WVlZVo0qSJ3nKLixcvFgDE6tWr5TZ/f38BQGzatEluu379ugAgzMzMxKlTp+R23f378vnr06ePKFq0qHjy5IleTJ07dxYODg7yfbxw4UIBQGzdulXu8/z5c1G6dOk3XupTl3OmTZum17dq1aqZzrWu7+uWxJ0/f74AII4eParXPm7cOAFAtGjRQm4rUKBApvtBCCGCg4MFALFnzx4hhBBz584VAMTdu3cz9a1Zs6aoU6dOnveZneyW+hw+fHimcT179kx4eXkJT0/PHJfl3LFjhwAg1q9fr9e+bNkyAUBUrFhRbqtUqZKwtbUVtra24osvvhA7duwQX3zxhQAgOnfuLPfbtm1bplym06FDB+Hq6qo3pgYNGmTqd+XKFQFALFu2TK/9wYMHAoCYM2dOtmMiIiIioncPZ/wRERERkUno0KEDHBwc5Ne1a9cGAHTv3l1vSbjatWsjLS1Nnq2yf/9+xMXFoUuXLnjy5In8ZW5ujtq1a+PQoUM5HrdChQoIDQ1F9+7dERkZiUWLFqF169YoUqQIVq5cKfd7k+MMHDjwtePetm0bHBwc0LhxY739Vq9eHXZ2dvJ+HR0dAQB//vnna5foe1VGRga2bNmCTp06yUtKNmjQAIULF8521t9//vMfvdeffPIJnj59ioSEBADArl27AAAjR47U6zdq1CgAyLRsXvny5eHn5ye/1l3fBg0awN3dPVP7rVu35LaXZ9ekpKTgyZMnqFOnDgDg/Pnz2Y7bzMwM3bp1w86dO/VmZm3cuBEfffQRvLy8sn1vbtjZ2QGAvO+3vReB3N8P+/fvx7NnzzBu3DjY2Njo7ePlZUNz6+HDhwgNDUVQUJDezLpKlSqhcePG8vV+2evukew8ffoUBQsWzLHP0aNHMXXqVNSrVw8TJ07MtD2399+BAweQlpaG4cOH680c69evH9Rqdab71M7OTm/GYLly5eDo6AgfHx/53gQy36dCCOzYsQMtW7aEEELv2gUEBCA+Pl6+V3ft2oWiRYuiffv28v5sbW31lm98U1ldk5e/l4AXM0WFEJmWHH5V165d4eDggN69e2P//v2IjIzEihUrsGTJEgAvZi7qJCcnw9raOtM+dPemrq/uv9n1fZN95tWuXbtQq1YtfPzxx3KbnZ0d+vfvj8jISFy9ejXb9zZr1gweHh4YPXo0fvnlF9y5cwdbt27FhAkTYGFhoRdTYmIikpKS0LNnT3z33Xdo27YtvvvuOwwYMACbN2/GjRs39MahxDnRfZ89efLkteeFiIiIiN4dLPwRERERkUl4ufgDQC4Curm5Zdmue96Z7penDRo0gIuLi97Xvn37EB0d/dpjly1bFuvXr8eTJ09w6dIlzJw5ExYWFujfv7/8HK+8HsfCwgIlSpR47bFv3LiB+Ph4FC5cONN+ExMT5f36+/ujXbt2mDp1KgoVKoRWrVphzZo1uXou2L59+/D48WPUqlULN2/exM2bN3H79m3Ur18fP//8c5bPnXr1euh+gaw773fu3IGZmRlKly6t18/V1RWOjo64c+dOjvvL7fUFgJiYGAwbNgxFihSBSqWCi4uLXLSLj4/Pcew9e/ZEcnKyvIRfeHg4zp07hx49euT4vtxITEwEANjb2wMwzL2Y2/shIiICAFCxYsW3HgcA+Xpl9dxDHx8fPHnyBM+fP9drf909khORwxKtT58+RZcuXVCwYEFs3Lgx01KPunhzc/9lNy4rKyuULFky031aokSJTIVTBweH196njx8/RlxcHFasWJHpuvXq1QsA5Gt3584dlC5dOtNx3uaZk8CL4o+Li4teW8GCBXN1PbLi6uqKnTt3IjU1FU2aNIGXlxfGjBmD77//HsD/Ct/Ai+J8VrlIt6Slrniv+292fV8u8ud2n3l1586dbO9z3fbs2NjYIDg4GM7OzmjXrh08PT3Rs2dPTJo0CU5OTpnOCQB06dJFbx9du3YFAJw8eVKvnxLnRPd99iZ/DEBERERExmPx+i5ERERERO8+c3PzPLXrfqGpK1qtX78erq6umfq9PFswNzH4+vrC19cXfn5+qF+/PjZu3IhGjRrl+TjW1tZZFixepdVqc5x5p/tFviRJ2L59O06dOoU//vgDe/fuRe/evTF//nycOnVK7xfOr9Ltu2PHjlluP3z4MOrXr6/X9rrzrpPbXyi/6fUFXsR94sQJjBkzBlWqVIGdnR20Wi2aNm2aZdHyZeXLl0f16tWxYcMG9OzZExs2bICVlVW25yIvLl++DABy8ckQ92Ju74d3QW7vkVc5OztnW4wSQiAwMBAPHjzAH3/8gWLFiuW4L0MXNN42D3Xv3h2BgYFZ9q1UqZIBIsxedjG+jU8//RS3bt1CWFgYnj9/jsqVK+PBgwcAXvzBhE7RokXx8OHDTO/XtemuY9GiReX2V4upDx8+lJ9Vl5d95rcKFSrg8uXLuHr1KmJjY1G+fHmoVCqMGDEC/v7+cr9ixYrhypUrKFKkiN77CxcuDOB/ReOXz8mrHj58qDfOokWLZvlsxuzOie4YhQoVyvM4iYiIiMh4WPgjIiIiog9aqVKlALz4ZWqjRo0Mtt8aNWoA+N8vVJU6TqlSpXDgwAHUrVs3VzNY6tSpgzp16mDGjBnYtGkTunXrhs2bN6Nv375Z9n/+/Dl+//13dOrUSW9pQZ2hQ4di48aNmQp/r+Ph4QGtVosbN27IM2UA4NGjR4iLi4OHh0ee9ped2NhYHDx4EFOnTsWkSZPkdt3sutzo2bMnRo4ciYcPH2LTpk1o3rz5a5eazI3169dDkiQ0btwYQN7ukewKVrm9H3THunz5cqZZb7k5zqt01ys8PDzTtuvXr6NQoUIoUKBArvb1Ot7e3tixY0eW27799lsEBwdjxIgRaN68eY7x5ub+e3lcJUuWlPulpaXh9u3bBvtednFxgb29PTQazWv36eHhgcuXL0MIoXd9sjr37wJzc3NUqVJFfq2bBf3yOKtUqYKjR49Cq9Xq/cHD6dOnYWtrKxcJdfs5e/asXpHvwYMHuH//vt5yp7ndZ3ayu/c9PDyyvc91219HkiRUqFBBfr1r1y5otVq9c1K9enXs378f//77r94MQ13hVFfEr1ixIiwsLHD27Fm9P0hIS0tDaGioXluVKlVw6NAhJCQkQK1Wy+2nT5+Wt7/s9u3bAKD3PUJERERE7z4u9UlEREREH7SAgACo1WrMnDkzy2ffPX78OMf3Hz16NMv36Z4hpvuF7dseJzsdO3aERqPB9OnTM23LyMhAXFwcgBcFsFdnUul+yZvTcp+//vornj9/jsGDB6N9+/aZvlq0aIEdO3bkasnQlzVr1gwAsHDhQr32b7/9FgByLNrkhW4W06tjf/W4OenSpQskScKwYcNw69YtdO/e/a3jmj17Nvbt24dOnTqhTJkyAPJ2j+iKaLrrq5Pb+6FJkyawt7fHrFmz5GX+dF4+VwUKFHjtcqjAi5lEVapUwbp16/Riunz5Mvbt2ydfb0Pw8/NDbGxspmfPnTlzBuPHj0f16tUxe/bsHPeR2/uvUaNGsLKywnfffad3Xn788UfEx8cb9D5t164dduzYIc8EfdnL175Zs2Z48OABtm/fLrclJSVhxYoVBonldR4+fIjr16/n+VmhwItxzJkzB5UqVdIrcrVv3x6PHj3CL7/8Irc9efIE27ZtQ8uWLeXn0lWoUAHe3t5YsWIFNBqN3Hfp0qWQJEnvjxNyu8/sFChQINP3F/Di/P/999/yUpvAiz+QWLFiBTw9PVG+fPncnxC8eK7eV199haJFi+ot66kr2P344496/VetWgULCwvUq1cPwItlYxs1aoQNGzboPYt0/fr1SExMRIcOHeS29u3bQ6PR6N0rqampWLNmDWrXrp1pFuW5c+cgSZLe81WJiIiI6N3HGX9ERERE9EFTq9VYunQpevTogWrVqqFz585wcXHB3bt3ERwcjLp162Lx4sXZvn/OnDk4d+4c2rZtKy/Fd/78efz0009wcnLC8OHDDXKc7Pj7+2PAgAGYNWsWQkND0aRJE1haWuLGjRvYtm0bFi1ahPbt22PdunVYsmQJ2rRpg1KlSuHZs2dYuXIl1Gp1jkWZjRs3wtnZGR999FGW2z///HOsXLkSwcHBaNu2ba7jrly5MgIDA7FixQrExcXB398ff//9N9atW4fWrVvneQZhdtRqNT799FN88803SE9PR/HixbFv3z55JktuuLi4oGnTpti2bRscHR3zVOzJyMjAhg0bALx4jtadO3ewc+dOXLp0CfXr19f7BXxe7pHq1asDeDHjMiAgAObm5ujcuXOu7we1Wo0FCxagb9++qFmzJrp27YqCBQvi4sWLSEpKwrp16+TjbNmyBSNHjkTNmjVhZ2eHli1bZjnWuXPn4rPPPoOfnx/69OmD5ORkfP/993BwcMCUKVNyfc5ep3nz5rCwsMCBAwfkGV5JSUno1KkT0tPT0aJFC2zdujXL9xYpUgSNGzfO9f3n4uKC8ePHY+rUqWjatCk+//xzhIeHY8mSJahZs6ZBisA6s2fPxqFDh1C7dm3069cP5cuXR0xMDM6fP48DBw4gJiYGANCvXz8sXrwYPXv2xLlz51C0aFGsX78etra2BoslJ+PHj8e6detw+/ZteHp65tjX398ffn5+KF26NKKiorBixQokJibizz//1JuF1759e9SpUwe9evXC1atXUahQISxZsgQajQZTp07V2+fcuXPx+eefo0mTJujcuTMuX76MxYsXo2/fvnoz0/Kyz6xUr14dS5cuxddff43SpUujcOHCaNCgAcaNG4eff/4Zn332GYYOHQonJyf5fOzYseO1SzR37NgRxYoVQ/ny5ZGQkIDVq1fj1q1bCA4Olp/3CQBVq1ZF7969sXr1amRkZMDf3x8hISHYtm0bxo8fr7cs54wZM/DRRx/B398f/fv3x/379zF//nw0adIETZs2lfvVrl0bHTp0wPjx4xEdHY3SpUtj3bp1iIyMzFRgBID9+/ejbt26cHZ2fu35IiIiIqJ3iCAiIiIiescMHjxYZPdRNTAwUHh4eMivb9++LQCIuXPn6vU7dOiQACC2bdum175mzRoBQJw5cyZT/4CAAOHg4CBsbGxEqVKlRFBQkDh79myOsR4/flwMHjxYVKxYUTg4OAhLS0vh7u4ugoKCRERERKb+uTlOYGCgKFCgQK7Gr7NixQpRvXp1oVKphL29vfD19RVffvmlePDggRBCiPPnz4suXboId3d3YW1tLQoXLixatGiR4/gePXokLCwsRI8ePbLtk5SUJGxtbUWbNm2EEEJMnjxZABCPHz/W66c777dv35bb0tPTxdSpU4WXl5ewtLQUbm5uYvz48SIlJUXvvR4eHqJ58+aZjg1ADB48WK8tq/vh/v37ok2bNsLR0VE4ODiIDh06iAcPHggAYvLkyTnGqLN161YBQPTv3z/bc/GqwMBAAUD+srW1FZ6enqJdu3Zi+/btQqPRZPm+3NwjGRkZ4osvvhAuLi5CkqRM3y+vux90du7cKT766COhUqmEWq0WtWrVEj///LO8PTExUXTt2lU4OjoKAPK9pzvPa9as0dvfgQMHRN26deX9tWzZUly9elWvT17ukex8/vnnomHDhvJrXTyv+/L395ffk9v7TwghFi9eLLy9vYWlpaUoUqSIGDhwoIiNjdXr4+/vLypUqJDpvXm5fx89eiQGDx4s3NzchKWlpXB1dRUNGzYUK1as0Ot3584d8fnnnwtbW1tRqFAhMWzYMLFnzx4BQBw6dCjHc5dVDswu5+iu1ct093VurtOIESNEyZIlhbW1tXBxcRFdu3bNMi8KIURMTIzo06ePcHZ2Fra2tsLf3z9Tntb59ddfRZUqVYS1tbUoUaKEmDhxokhLS3urfb4qKipKNG/eXNjb22e6dyIiIkT79u2Fo6OjsLGxEbVq1RJ//vlnrvY7Z84c4e3tLWxsbETBggXF559/Li5cuJBl37S0NDFlyhTh4eEhLC0tRenSpcWCBQuy7Hv06FHx0UcfCRsbG+Hi4iIGDx4sEhISMvVLTk4Wo0ePFq6ursLa2lrUrFlT7NmzJ1O/uLg4YWVlJVatWpWrcRERERHRu0MS4jVPTiciIiIiIvrA/f7772jdujWOHDmCTz75xNjhfPCOHj2KevXq4fr16/JSqURkOAsXLsQ333yDiIiIXD0/loiIiIjeHSz8ERERERERvUaLFi1w7do13Lx5E5IkGTscAvDZZ5+hRIkSWLlypbFDITIp6enpKFWqFMaNG4dBgwYZOxwiIiIiyiM+44+IiIiIiCgbmzdvxqVLlxAcHIxFixax6PcO2b17t7FDIDJJlpaWuHv3rrHDICIiIqI3xBl/RERERERE2ZAkCXZ2dujUqROWLVsGCwv+7SQRERERERG9u/h/rURERERERNng30kSERERERHR+8TM2AEQERERERERERERERER0dvjjD8D0Wq1ePDgAezt7fncDyIiIiIiIiIiIiIiog+cEALPnj1DsWLFYGaWP3PxWPgzkAcPHsDNzc3YYRAREREREREREREREdE75N69eyhRokS+HIuFPwOxt7cHAERGRqJgwYJGjoaITIlGo8GVK1dQoUIFmJubGzscIjIhzC9EpBTmFyJSCvMLESmF+YWIlBAbGwtPT0+5hpQfWPgzEN3ynmq1Gmq12sjREJEp0Wg0sLOzg1qt5gdPIjIo5hciUgrzCxEphfmFiJTC/EJEStBoNACQr4+Iy58FRYmIiIiIiIiIiIiIiIhIUSz8GVh+Vm2J6MMgSRKcnJyYX4jI4JhfiEgpzC9EpBTmFyJSCvMLESnBGDlFEkKIfD+qCUpISICDgwPi4+O51CcREREREREREREREdEHzhi1Iz7jz8C0Wq2xQyAiE6PVanH//n2UKFECZmacqE1EhsP8QkRKYX4hIqUwvxB92DQaDdLT0xXZt1arRVRUFFxdXZlfiChPrKysss0bxqgZsfBnYJxASUSGJoRATEwMihcvbuxQiMjEML8QkVKYX4hIKcwvRB8mIQSioqIQFxen6DHS09ORkpLC5T6JKE/MzMzg5eUFKyurTNuMUTNi4Y+IiIiIiIiIiIiI3lm6ol/hwoVha2urSGFOCIGUlBTY2Niw8EdEuabVavHgwQM8fPgQ7u7u70T+YOGPiIiIiIiIiIiIiN5JGo1GLvo5OzsrdhwhBIQQLPwRUZ65uLjgwYMHyMjIgKWlpbHDARcrNjD+UCAiQ5MkCa6urswvRGRwzC9EpBTmFyJSCvML0YdH90w/W1tbxY/1LvzCnojeP7olPjUaTaZtxvjMwhl/BsYHvxKRoZmZmcHV1dXYYRCRCWJ+ISKlML8QkVKYX4g+XEr/8lySJBb+iOiN5JSfjFEzYpXKwLKq6BIRvQ2NRoOIiAjmFyIyOOYXIlIK8wsRKYX5hYiUonvGnxDC2KEQkQkxxmcWFv6IiN4Dz549M3YIRGSimF+ISCnML0SkFOYXIlKKVqs1dghERG+NhT8iIiIiIiIiIiIiondMZGQkJElCaGhort+zdu1aODo6Gj2OV9WrVw/Dhw83WEzvgilTpqBKlSrGDoMoExb+iIiIiIiIiIiIiIgUcO/ePfTu3RvFihWDlZUVPDw8MGzYMDx9+vS173Vzc8PDhw9RsWLFXB+vU6dO+Oeff94m5DzRFQVz+lq7di1++eUXTJ8+Pd/i0tFoNJg9eza8vb2hUqng5OSE2rVrY9WqVW+979GjR+PgwYMGiJLIsCyMHYCpUfohs0T04ZEkCW5ubswvRGRwzC9EpBTmFyJSCvMLESnJysrKoPu7desW/Pz8ULZsWfz888/w8vLClStXMGbMGOzevRunTp2Ck5NTlu9NS0uDlZUVXF1d83RMlUoFlUpliPBzRVec1Jk3bx727NmDAwcOyG0ODg75GtPLpk6diuXLl2Px4sWoUaMGEhIScPbsWcTGxr7xPoUQ0Gg0sLOzg52dnQGjJVNkjM8snPFnYGZmPKVEZFhmZmZwdnZmfiEig2N+ISKlML8QkVKYX4gIAIQAUlIM+5WaKiEjwwKpqVK2fYTIW5yDBw+GlZUV9u3bB39/f7i7u+Ozzz7DgQMH8O+//2LChAlyX09PT0yfPh09e/aEWq1G//79s1xic+fOnShTpgxsbGxQv359rFu3DpIkIS4uDkDmpT51y1GuX78enp6ecHBwQOfOnfWel7pnzx58/PHHcHR0hLOzM1q0aIGIiIhcjdHc3Byurq7yl52dHSwsLPTaVCpVpqU+PT098fXXX6Nnz56ws7ODh4cHdu7cicePH6NVq1aws7NDpUqVcPbsWb3jHTt2DJ988glUKhXc3NwwdOhQPH/+PNv4du7ciUGDBqFDhw7w8vJC5cqV0adPH4wePVruo9VqMWvWLHh5eUGlUqFy5crYvn27vD0kJASSJGH37t2oXr06rK2tcezYsSyX+ly1ahV8fHxgY2MDb29vLFmyRN6WlpaGIUOGoGjRorCxsYGHhwdmzZqVq/NM7y9jfGbhjD8D02g0xg6BiEyMRqPBjRs3UKZMGZibmxs7HCIyIcwvRKQU5hciUgrzCxEBQGoq0KGDofcqoNFoYW5uBiDrGTrbtgE2NrnbW0xMDPbu3YsZM2Zkmu3m6uqKbt26YcuWLViyZIk8I2jevHmYNGkSJk+enOU+b9++jfbt22PYsGHo27cvLly4oFfAyk5ERAR+++03/Pnnn4iNjUXHjh0xe/ZszJgxAwDw/PlzjBw5EpUqVUJiYiImTZqENm3aIDQ0VNGixYIFCzBz5kx89dVXWLBgAXr06IGPPvoIvXv3xty5czF27Fj07NkTV65cgSRJiIiIQNOmTfH1119j9erVePz4MYYMGYIhQ4ZgzZo1WR7D1dUVf/31FwYNGgQXF5cs+8yaNQsbNmzAsmXLUKZMGRw5cgTdu3eHi4sL/P395X7jxo3DvHnzULJkSRQsWBAhISF6+9m4cSMmTZqExYsXo2rVqrhw4QL69euHAgUKIDAwEN999x127tyJrVu3wt3dHffu3cO9e/cMdj7p3WSMmhELf0RE74GUlBRjh0BEJor5hYiUwvxCREphfiGi98GNGzcghICPj0+W2318fBAbG4vHjx+jcOHCAIAGDRpg1KhRcp/IyEi99yxfvhzlypXD3LlzAQDlypXD5cuX5QJedrRaLdauXQt7e3sAQI8ePXDw4EH5fe3atdPrv3r1ari4uODq1at5er5gXjVr1gwDBgwAAEyaNAlLly5FzZo10eH/q7pjx46Fn58fHj16BFdXV8yaNQvdunWTZw6WKVMG3333Hfz9/bF06VLYZFGV/fbbb9G+fXu4urqiQoUK+Oijj9CqVSt89tlnAIDU1FTMnDkTBw4cgJ+fHwCgZMmSOHbsGJYvX65X+Js2bRoaN26c7XgmT56M+fPno23btgAALy8vXL16FcuXL0dgYCDu3r2LMmXK4OOPP4YkSfDw8Hj7k0iUhXem8Dd79myMHz8ew4YNw8KFCwG8+CA3atQobN68GampqQgICMCSJUtQpEgR+X13797FwIEDcejQIdjZ2SEwMBCzZs2ChcX/hhYSEoKRI0fiypUrcHNzw8SJExEUFKR3/B9++AFz585FVFQUKleujO+//x61atXKj6ETERERERERERERUS5ZW7+YfWdIQgDJyalQqVTI7pFc1tZvst/crw9ao0aNHLeHh4ejZs2aem25+R22p6enXPQDgKJFiyI6Olp+fePGDUyaNAmnT5/GkydPoNVqAbz43buShb9KlSrJ/9b9zt/X1zdTW3R0NFxdXXHx4kVcunQJGzdulPsIIaDVanH79u0si6zly5fH5cuXce7cORw/fhxHjhxBy5YtERQUhFWrVuHmzZtISkrKVNBLS0tD1apV9dpyuj7Pnz9HREQE+vTpg379+sntGRkZcHBwAAAEBQWhcePGKFeuHJo2bYoWLVqgSZMmrz1PRHn1ThT+zpw5g+XLl+t9owPAiBEjEBwcjG3btsHBwQFDhgxB27Ztcfz4cQAvpkg2b94crq6uOHHiBB4+fIiePXvC0tISM2fOBPBi+nPz5s3xn//8Bxs3bsTBgwfRt29fFC1aFAEBAQCALVu2YOTIkVi2bBlq166NhQsXIiAgAOHh4fJfWxARERERERERERGR8UlS7pfczC0hXnzZ2CDbwl9elC5dGpIk4dq1a2jTpk2m7deuXUPBggX1lp8sUKDA2x84C5aWlnqvJUmSi3sA0LJlS3h4eGDlypUoVqwYtFotKlasiLS0NEXiySou3XKnWbXpYk1MTMSAAQMwdOjQTPtyd3fP9jhmZmaoWbMmatasieHDh2PDhg3o0aMHJkyYgMTERABAcHAwihcvrvc+61cqvTldH91+Vq5cidq1a+tt0y1NXa1aNdy+fRu7d+/GgQMH0LFjRzRq1EjveYJEhmD0wl9iYiK6deuGlStX4uuvv5bb4+Pj8eOPP2LTpk1o0KABAGDNmjXw8fHBqVOnUKdOHezbtw9Xr17FgQMHUKRIEVSpUgXTp0/H2LFjMWXKFFhZWWHZsmXw8vLC/PnzAbyYQn3s2DEsWLBALvx9++236NevH3r16gUAWLZsGYKDg7F69WqMGzcuy7hTU1ORmpoqv05ISADw4i8MdGu2SpIEMzMzaLVavb/s0LW/urZrdu1mZmaQJCnLdgB6STqndnNzc/kvIF5tfzXG7No5Jo6JY8r/MQkh9Kb+m8KYXtfOMXFMHFP+jEm3tMjLn1/e9zFlFTvHxDFxTPk/JiEEPD09s4zxfR1TTrFzTBwTx5R/YwKg9/nFFMZkiteJY+KYDDkmjUYDIYT8JUlSlrPoDNFuZWUFIG+z9LIiSRKcnJzQuHFjLFmyBMOHD///mYQvjhkVFYWNGzeiR48eeu/TjfFVuvayZcti9+7dejH+/fffeq9f999X254+fYrw8HCsWLECn3zyCQDg2LFjesd9uf+r8b16HrP7d3Yx6f6tK/K9eryX26pVq4arV6+iVKlSWV6/3F433czA58+fw8fHB9bW1rhz5w4+/fTTTGN6dczZ/btw4cIoVqwYIiIi0LVr1yz3AwD29vbo1KkTOnbsiHbt2uGzzz7D06dP4eTklO29mhdKfn8Ysj0v3rXYc7r3NBqNXk7U5cn8ZvTC3+DBg9G8eXM0atRIr/B37tw5pKeno1GjRnKbt7c33N3dcfLkSdSpUwcnT56Er6+v3tKfAQEBGDhwIK5cuYKqVavi5MmTevvQ9dGtA5yWloZz585h/Pjx8nYzMzM0atQIJ0+ezDbuWbNmYerUqZnar169Cjs7OwCAk5MT3N3dcf/+fcTExMh9XF1d4erqisjISDx79kxud3Nzg7OzM27cuKG3Xn3JkiWhVqtx9epVvZumXLlysLKyQlhYmF4Mvr6+SEtLQ3h4uNxmbm4OX19fPHv2DLdu3ZLbbWxs4O3tjdjYWL0Hidrb26NUqVKIjo5GVFSU3M4xcUwcE8fEMXFMHJNpjenOnTsmNyZTvE4cE8f0vo7J0dERMTExJjUmU7xOHBPH9D6N6d9//zW5MZnideKYOCZDjunu3bswMzNDSkoKJEmCjY0NMjIykJ6errcfa2trpKWl6cViaWkJS0tLpKam6hUWraysYGFhgZSUFL1f4FtbW0OSJCQlJenFbmNjA0mSkJycrNeuUqkghMj07FFbW1totVrMnTsXDRs2RJMmTTB58mR4e3vj0qVL+PLLL1GsWDFMnDgRqamp8rPp0tPT5WPoZokBLx6JlZycjMDAQCxYsABjx45F9+7dERoairVr1wL4X0FVd150+9GN7+XYXz53NjY2cHZ2xtKlS1GwYEFER0fLvytPS0tDcnKyPD6tVqu3H0mSoFKpoNFo5NmBGRkZ8jFfvk5arVaOMS0tDUIIeby66/TyMYH//fF7amoqkpOTMXToUNSvXx9DhgxBv379YGZmhuvXr+Ovv/7Ct99+m+V16tatGz755BP4+fmhYMGCiIyMxOTJk1GmTBl4e3tDkiQMGzYMI0eOREpKCurWrYvU1FQcOXIEtra26N69u94EoJfH9PJY09LSMGHCBIwePRq2trb47LPPoNVqcfLkScTExGDo0KH47rvvUKJECdSoUQNpaWnYvHkzihQpAmtra2i1Wpibm2e6x9703ns55qyuE/CiDqL099OHMibgxffaP//8o1fo8/X11Ys7v0jibcutb2Hz5s2YMWMGzpw5AxsbG9SrVw9VqlTBwoULsWnTJvTq1SvTSalVqxbq16+POXPmoH///rhz5w727t0rb09KSkKBAgWwa9cufPbZZyhbtix69eqlV9jbtWsXmjdvjqSkJMTGxqJ48eI4ceKE/PBOAPjyyy9x+PBhnD59OsvYs5rx5+bmhsePH6NgwYIA3r+/njHFvwjimDgmUxiTRqPBtWvXUL58eVhaWprEmF7XzjFxTBxT/oxJo9HgypUr8PHxkf+n8n0fU1axc0wcE8eU/2PSaDS4fv06KlSokOVfxL6PY8opdo6JY+KY8m9M6enpuHr1qvz5xRTGZIrXiWPimAw5puTkZNy5cwdeXl5ywUCJWTu6goNKpcrUN69e3vedO3cwZcoU7NmzBzExMXB1dUWrVq0wefJkODs7y/09PT0xbNgwecKK7r1eXl44f/48qlSpAgDYuXMnRo8ejXv37sHPzw8dO3bEoEGDkJSUBJVKhTVr1mDEiBGIjY0FAEyZMgW///47Lly4IO934cKFWLRoEW7fvg0AOHDgAIYNG4Zbt26hXLlyWLRoEerXr49ffvkFrVu3RmRkJEqWLKkXR3bnUXe80NBQvfb69eujcuXKWLRoEYQQ8PLy0huvJEmQJEk+JoAsj3vmzBlMnDgRJ0+ehBACpUqVQseOHfHf//43y2uxcuVKbN68GZcvX0Z8fDxcXV3RoEEDTJ48GZ6enhDixYy+7777DsuWLcOtW7fg6OiIatWqYfz48fj0008REhKCBg0aIDY2Vn5eX3Zj3bRpE+bNm4erV6+iQIEC8PX1xbBhw9CmTRusXLkSS5cuxY0bN2Bubo6aNWvim2++kZ8lmN29mhdKfX8Yuj0v3rXYs2pPSUlBZGQk3N3d5UI+8CLvxcbGwtnZGfHx8VCr1VkP0sCMVvi7d+8eatSogf3798vP9qv3HhX+XpWQkAAHBwfExMTIhT8iIkPQaDQICwuDr6+v3l97ERG9LeYXIlIK8wsRKYX5hejDk5KSgtu3b8uFP6UIIZCcnCwvyfm+mDFjBpYtW6Y325OI8ldOeSo2NhZOTk75Wvgzy5ejZOHcuXOIjo5GtWrVYGFhAQsLCxw+fBjfffcdLCwsUKRIEaSlpSEuLk7vfY8ePYKrqyuAF9PPHz16lGm7bltOfdRqNVQqFQoVKgRzc/Ms++j2QURERERERERERERkbEuWLMGZM2dw69YtrF+/HnPnzkVgYKCxwyKid4jRCn8NGzZEWFgYQkND5a8aNWqgW7du8r8tLS1x8OBB+T3h4eG4e/euPDPPz88PYWFhiI6Olvvs378farUa5cuXl/u8vA9dH90+rKysUL16db0+Wq0WBw8e1JsBSERERERERERERERkTDdu3ECrVq1Qvnx5TJ8+HaNGjcKUKVOMHRYRvUOM+oy/V7281CcADBw4ELt27cLatWuhVqvxxRdfAABOnDgB4MXyDlWqVEGxYsXwzTffICoqCj169EDfvn0xc+ZMAMDt27dRsWJFDB48GL1798Zff/2FoUOHIjg4GAEBAQCALVu2IDAwEMuXL0etWrWwcOFCbN26FdevX0eRIkVyFbtuqc+4uDi9dX6JiN6Wbo153Tr2RESGwvxCREphfiEipTC/EH148nOpTyGE/Kw5IqLcyilPxcfHw9HRMV+X+rTIl6O8oQULFsDMzAzt2rVDamoqAgICsGTJEnm7ubk5/vzzTwwcOBB+fn4oUKAAAgMDMW3aNLmPl5cXgoODMWLECCxatAglSpTAqlWr5KIfAHTq1AmPHz/GpEmTEBUVhSpVqmDPnj25LvoRESnNysrK2CEQkYlifiEipTC/EJFSmF+ISCks+BGRKXinZvy9z3Qz/mJiYlCwYEFjh0NEJoQPrycipTC/EJFSmF+ISCnML0Qfnvyc8ZecnAyVSsUCIBHlSU55KjY2Fk5OTvk6489oz/gjIiIiIiIiIiIiIiIiIsNh4Y+IiIiIiIiIiIiIiIjIBLDwR0RERERERERERERERGQCWPgzMDMznlIiMiwzMzP4+voyvxCRwTG/EJFSmF+ISCnML0SkJJVKZewQiMjEGOMzCz8lERG9B9LS0owdAhGZKOYXIlIK8wsRKYX5hYiUIoQwdghvLCgoCK1bt5Zf16tXD8OHD8/3OEJCQiBJEuLi4vL92ET0Agt/BqbVao0dAhGZGK1Wi/DwcOYXIjI45hciUgrzCxEphfmFiJSUkpJi0P0FBQVBkiRIkgQrKyuULl0a06ZNQ0ZGhkGPk5VffvkF06dPz1Xf/C7WeXp6QpIkbN68OdO2ChUqQJIkrF27Nl9iIVKaMT6zsPBHRERERERERERERKSApk2b4uHDh7hx4wZGjRqFKVOmYO7cuVn2NeSMZicnJ9jb2xtsf4bm5uaGNWvW6LWdOnUKUVFRKFCggJGiIjINLPwRERERERERERER0XvnedrzbL9SMlJy3Tc5Pfm1fd+UtbU1XF1d4eHhgYEDB6JRo0bYuXMngP8tzzljxgwUK1YM5cqVAwDcu3cPHTt2hKOjI5ycnNCqVStERkbK+9RoNBg5ciQcHR3h7OyML7/8MtMypa8u9ZmamoqxY8fCzc0N1tbWKF26NH788UdERkaifv36AICCBQtCkiQEBQUBeDFTadasWfDy8oJKpULlypWxfft2vePs2rULZcuWhUqlQv369fXizEm3bt1w+PBh3Lt3T25bvXo1unXrBgsLC72+cXFx6Nu3L1xcXKBWq9GgQQNcvHhR3h4REYFWrVqhSJEisLOzQ82aNXHgwAG9fXh6emLmzJno3bs37O3t4e7ujhUrVuQqVqL3jcXruxARkbGZm5sbOwQiMlHML0SkFOYXIlIK8wsR6djNsst2W7MyzRDcNVh+XXheYSSlJ2XZ19/DH4cCD8mvPRd54knSE70+YrJhnv+nUqnw9OlT+fXBgwehVquxf/9+AEB6ejoCAgLg5+eHo0ePwsLCAl9//TWaNm2KS5cuwcrKCvPnz8fatWuxevVq+Pj4YP78+fj111/RoEGDbI/bs2dPnDx5Et999x0qV66M27dv48mTJ3Bzc8OOHTvQrl07hIeHQ61WQ6VSAQBmzZqFDRs2YNmyZShTpgyOHDmC7t27w8XFBf7+/rh37x7atm2LwYMHo3///jh79ixGjRqVq/NQpEgRBAQEYN26dZg4cSKSkpKwZcsWHD58GD/99JNe3w4dOkClUmH37t1wcHDA8uXL0bBhQ/zzzz9wcnJCYmIimjVrhhkzZsDa2ho//fQTWrZsifDwcLi7u8v7mT9/PqZPn47//ve/2L59OwYOHAh/f3+54EpkKlj4MzB++CQiQzM3N4evr6+xwyAiE8T8QkRKYX4hIqUwvxCRUiRJgq2trWL7F0Lg4MGD2Lt3L7744gu5vUCBAli1ahWsrKwAABs2bIBWq8WqVasgSRIAYM2aNXB0dERISAiaNGmChQsXYvz48Wjbti0AYNmyZdi7d2+2x/7nn3+wdetW7N+/H40aNQIAlCxZUt7u5OQEAChcuDAcHR0BvJghOHPmTBw4cAB+fn7ye44dO4bly5fD398fS5cuRalSpTB//nwAQLly5RAWFoY5c+bk6pz07t0bo0aNwoQJE7B9+3aUKlUKVapU0etz7Ngx/P3334iOjoa1tTUAYN68efjtt9+wfft29O/fH5UrV0blypXl90yfPh2//vordu7ciSFDhsjtzZo1w6BBgwAAY8eOxYIFC3Do0CEW/khRxqgZsfBnYK9OqSYieltCCDx79gz29vbyBz4iIkNgfiEipTC/EJFSmF+I6GWJ4xOz3WZupv/L9ujR0dn2NZPMIISAVquFmZkZIodFGipE/Pnnn7Czs0N6ejq0Wi26du2KKVOmyNt9fX3loh8AXLx4ETdv3sz0fL6UlBREREQgPj4eDx8+RO3ateVtFhYWqFGjRra/mw4NDYW5uTn8/f1zHffNmzeRlJSExo0b67WnpaWhatWqAIBr167pxQFALhLmRvPmzTFgwAAcOXIEq1evRu/evTP1uXjxIhITE+Hs7KzXnpycjIiICABAYmIipkyZguDgYDx8+BAZGRlITk7G3bt39d5TqVIl+d+SJMHV1RXR0dnfF0SGYIyaEQt/BqbVao0dAhGZGK1Wi1u3bsHX15ezionIoJhfiEgpzC9EpBTmFyJ6WQGrAgbrK4RAamoqVCpVnvb7OvXr18fSpUthZWWFYsWKZXp+XYEC+sdKTExE9erVsXHjxkz7cnFxeaMYdEt35kVi4ouianBwMIoXL663TTfz7m1ZWFigR48emDx5Mk6fPo1ff/01yziKFi2KkJCQTNt0sxNHjx6N/fv3Y968eShdujRUKhXat2+PtLQ0vf6WlpZ6ryVJ4u/zSXHGuMdY+CMiIiIiIiIiIiIiUkCBAgVQunTpXPevVq0atmzZgsKFC0OtVmfZp2jRojh9+jQ+/fRTAEBGRgbOnTuHatWqZdnf19cXWq0Whw8flpf6fJluxqFGo5HbypcvD2tra9y9ezfbmYI+Pj7YuXOnXtupU6deP8iX9O7dG/PmzUOnTp1QsGDBTNurVauGqKgoWFhYwNPTM8t9HD9+HEFBQWjTpg2AF8XCyMjIPMVBZErMjB0AEREREREREREREREB3bp1Q6FChdCqVSscPXoUt2/fRkhICIYOHYr79+8DAIYNG4bZs2fjt99+w/Xr1zFo0CDExcVlu09PT08EBgaid+/e+O233+R9bt26FQDg4eEBSZLw559/4vHjx0hMTIS9vT1Gjx6NESNGYN26dYiIiMD58+fx/fffY926dQCA//znP7hx4wbGjBmD8PBwbNq0CWvXrs3TeH18fPDkyROsWbMmy+2NGjWCn58fWrdujX379iEyMhInTpzAhAkTcPbsWQBAmTJl8MsvvyA0NBQXL15E165dOZOPPmgs/BERvQdsbGyMHQIRmSjmFyJSCvMLESmF+YWIlPIuPDvU1tYWR44cgbu7O9q2bQsfHx/06dMHKSkp8gzAUaNGoUePHggMDISfnx/s7e3l2W7ZWbp0Kdq3b49BgwbB29sb/fr1w/PnzwEAxYsXx9SpUzFu3DgUKVIEQ4YMAQBMnz4dX331FWbNmgUfHx80bdoUwcHB8PLyAgC4u7tjx44d+O2331C5cmUsW7YMM2fOzPOYnZ2ds12OVJIk7Nq1C59++il69eqFsmXLonPnzrhz5w6KFCkCAPj2229RsGBBfPTRR2jZsiUCAgKynf1I9CGQhDGeLGiCEhIS4ODggPj4+GynYBMRERERERERERFR7qWkpOD27dvw8vJi4Z+I3kk55Slj1I4448/AOIWYiAxNq9Xi6dOnzC9EZHDML0SkFOYXIlIK8wsRKUUIgYyMDHCeDBEZkjE+s7DwZ2D8wUBEhiaEwL1795hfiMjgmF+ISCnML0SkFOYXIlJSWlqasUMgIhNjjM8sLPwRERERERERERERERERmQAW/oiIiIiIiIiIiIiIiIhMAAt/RETvAXt7e2OHQEQmivmFiJTC/EJESmF+ISKlmJnx1+VE9P6zMHYApsbc3NzYIRCRiTE3N0epUqWMHQYRmSDmFyJSCvMLESmF+YWIlCJJEmxsbIwdBhGZGGPUjPgnDAam1WqNHQIRmRitVouoqCjmFyIyOOYXIlIK8wsRKYX5hYiUIoRAeno6hBDGDoWITIgxPrOw8Gdg/MFARIYmhEBUVBTzCxEZHPMLESmF+YWIlML8QkRKSk9PN3YIRGRijPGZhYU/IiIiIiIiIiIiIiIiIhPAwh8REREREREREREREeVKSEgIJElCXFxcvh537dq1cHR0fKt9REZGQpIkhIaGZtvHWOMjMhQW/gxMkiRjh0BEJkaSJDg5OTG/EJHBMb8QkVKYX4hIKcwvRKQkc3Nzg+4vKCgIrVu3Nug+lSZJUo5fU6ZMMXaIRO8VY3xmscj3I5o4MzPWUonIsMzMzODu7m7sMIjIBDG/EJFSmF+ISCnML0SkFEmSYG1tbeww8lVaWhqsrKz02h4+fCj/e8uWLZg0aRLCw8PlNjs7O5w9e9YgxyL6EBijZsQqlYFptVpjh0BEJkar1eLu3bvML0RkcMwvRKQU5hciUgrzCxEpRQiB1NRUCCHy7ZiHDx9GrVq1YG1tjaJFi2LcuHHIyMgAAPz5559wdHSERqMBAISGhkKSJIwbN05+f9++fdG9e3f59bFjx/DJJ59ApVLBzc0NQ4cOxfPnz+Xtnp6emD59Onr27Am1Wo3+/ftnisnV1VX+cnBwgCRJem12dnZy33PnzqFGjRqwtbXFRx99pFcgnDJlCqpUqYJVq1bBy8sLNjY2AIC4uDj07dsXLi4uUKvVaNCgAS5evCi/7+LFi6hfvz7s7e2hVqtRvXr1TIXGvXv3wsfHB3Z2dmjatKlesVKr1WLatGkoUaIErK2tUaVKFezZsyfH67Br1y6ULVsWKpUK9evXR2RkZI79ifLCGJ9ZWPgzsPz8wUBEHwYhBGJiYphfiMjgmF+ISCnML0SkFOYXItITXAE4N+J/ry9//aItNebF62c3X7z+54f/9TkZBOz/+H+vH+x90efhXrnIhv0fv+in8/L7DeTff/9Fs2bNULNmTVy8eBFLly7Fjz/+iK+//hoA8Mknn+DZs2e4cOECgBdFwkKFCiEkJETex+HDh1GvXj0AQEREBJo2bYp27drh0qVL2LJlC44dO4YhQ4boHXfevHmoXLkyLly4gK+++uqtxjBhwgTMnz8fZ8+ehYWFBXr37q23/ebNm9ixYwd++eUX+Zl6HTp0QHR0NHbv3o1z586hWrVqaNiwIWJiXlyzbt26oUSJEjhz5gzOnTuHcePGwdLSUt5nUlIS5s2bh/Xr1+PIkSO4e/cuRo8eLW9ftGgR5s+fj3nz5uHSpUsICAjA559/jhs3bmQ5hnv37qFt27Zo2bIlQkND0bdvX73iKtHbMsZnFi71SURERERERERERESUj5YsWQI3NzcsXrwYkiTB29sbDx48wNixYzFp0iQ4ODigSpUqCAkJQY0aNRASEoIRI0Zg6tSpSExMRHx8PG7evAl/f38AwKxZs9CtWzcMHz4cAFCmTBl899138Pf3x9KlS+UZdw0aNMCoUaMMMoYZM2bIxx83bhyaN2+OlJQU+VhpaWn46aef4OLiAuDFjMS///4b0dHR8rKq8+bNw2+//Ybt27ejf//+uHv3LsaMGQNvb295HC9LT0/HsmXLUKpUKQDAkCFDMG3aNHn7vHnzMHbsWHTu3BkAMGfOHBw6dAgLFy7EDz9kLuAuXboUpUqVwvz58wEA5cqVQ1hYGObMmWOQc0RkDCz8EREREREREREREdH7p/kV/dcVJ7740rEvnbmP31r918UCgGJXACGA5OQXbY2P6fcpO9gg4b7s2rVr8PPzgyRJclvdunWRmJiI+/fvw93dHf7+/ggJCcGoUaNw9OhRzJo1C1u3bsWxY8cQExODYsWKyYWxixcv4tKlS9i4caO8PyEEtFotbt++DR8fHwBAjRo1DDaGSpUqyf8uWrQoACA6Olp+FquHh4dc9NPFmJiYCGdnZ739JCcnIyIiAgAwcuRI9O3bF+vXr0ejRo3QoUMHucgHALa2tnqvixYtiujoaABAQkICHjx4gLp16+rtv27dunrLib7s2rVrqF27tl6bn59f7k4A0TuKhT8DezlRExEZgm4tdeYXIjI05hciUgrzCxEphfmFiJT08pKS74J69eph9erVuHjxIiwtLeHt7Y169eohJCQEsbGx8mw7AEhMTMSAAQMwdOjQTPvRFeIAoECBAgaL7+XzpcvLLz/P7NVjJSYmomjRonrLleo4OjoCePFswK5duyI4OBi7d+/G5MmTsXnzZrRp0ybTMXXH5fLP9C4zxmcWFv4MzMyMj00kIsMyMzODq6urscMgIhPE/EJESmF+ISKlML8QkVIkScrXwp+Pjw927NgBIYRcGDh+/Djs7e1RokQJAP97zt+CBQvkIl+9evUwe/ZsxMbG6i3ZWa1aNVy9ehWlS5fOtzHkVbVq1RAVFQULCwt4enpm269s2bIoW7YsRowYgS5dumDNmjVy4S8narUaxYoVw/Hjx/WKosePH0etWrWyfI+Pjw927typ13bq1KncDYgoF4xRM2KVysDkB8ASERmIRqNBREQE8wsRGRzzCxEphfmFiJTC/EJEShFCICUlxeCzx+Lj4xEaGqr3de/ePQwaNAj37t3DF198gevXr+P333/H5MmTMXLkSLlQULBgQVSqVAkbN25EvXr1AACffvopzp8/j3/++UevuDV27FicOHECQ4YMQWhoKG7cuIHff/8dQ4YMMeh43kajRo3g5+eH1q1bY9++fYiMjMSJEycwYcIEnD17FsnJyRgyZAhCQkJw584dHD9+HGfOnJGXKc2NMWPGYM6cOdiyZQvCw8Mxbtw4hIaGYtiwYVn2/89//oMbN25gzJgxCA8Px6ZNm7B27VoDjZjIODUjzvgjInoPPHv2zNghEJGJYn4hIqUwvxCRUphfiEgpLy9TaSghISGoWrWqXlufPn2watUq7Nq1C2PGjEHlypXh5OSEPn36YOLEiXp9/f39ERoaKhf+nJycUL58eTx69AjlypWT+1WqVAmHDx/GhAkT8Mknn0AIgVKlSqFTp04GH9ObkiQJu3btwoQJE9CrVy88fvwYrq6u+PTTT1GkSBGYm5vj6dOn6NmzJx49eoRChQqhbdu2mDp1aq6PMXToUMTHx2PUqFGIjo5G+fLlsXPnTvlZiK9yd3fHjh07MGLECHz//feoVasWZs6cid69extq2ET5ThJcANcgEhIS4ODggJiYGBQsWNDY4RCRCdFoNAgLC4Ovry/Mzc2NHQ4RmRDmFyJSCvMLESmF+YXow5OSkoLbt2/Dy8sLNjY2ih1HCIHk5GSoVCo+R5SI8iSnPBUbGwsnJyfEx8dDrVbnSzxc6pOIiIiIiIiIiIiIiIjIBLDwZ2D8axAiMjRJkuDm5sb8QkQGx/xCREphfiEipTC/EJGSrKysjB0CEZkYY3xm4TP+DEz34FUiIkMxMzODs7OzscMgIhPE/EJESmF+ISKlML8QkVIkSYKFBX9dTkSGZYyaEatUBqbRaIwdAhGZGI1Gg+vXrzO/EJHBMb8QkVKYX4hIKcwvRKQU3TP+hBDGDoWITIgxPrOw8EdE9B5ISUkxdghEZKKYX4hIKcwvRKQU5hciUgqLfkRkClj4IyIiIiIiIiIiIiIiIjIBLPwRERERERERERERERERmQAW/gzMGA9qJCLTZmZmhpIlSzK/EJHBMb8QkVKYX4hIKcwvRKQka2trY4dARCbGGJ9ZLPL9iCZOkiRjh0BEJkaSJKjVamOHQUQmiPmFiJTC/EJESmF+ISKlSJIEc3NzY4dBRCbGGDUj/nmUgWk0GmOHQEQmRqPRICwsjPmFiAyO+YWIlML8QkRKYX4hIqUIIZCUlAQhhLFDeS/Uq1cPw4cPz/fjenp6YuHChW+1j6CgILRu3TrHPsYaH5keY3xmYeGPiOg9wP+pJSKlML8QkVKYX4hIKcwvRPS+yK54tHbtWjg6OuZ7PLkxZcoUSJKU4xcRvdtY+CMiIiIiIiIiIiIi+sBoNBpotVq9ttGjR+Phw4fyV4kSJTBt2jS9tjeVnp7+tiETUS6w8EdEREREREREREREZCS6pSenTp0KFxcXqNVq/Oc//0FaWprcp169ehgyZAiGDBkCBwcHFCpUCF999ZXe0qSpqakYPXo0ihcvjgIFCqB27doICQmRt+tmGu7cuRPly5eHtbU17t69qxeLnZ0dXF1d5S9zc3PY29vrtelotVp8+eWXcHJygqurK6ZMmaK3L0mSsHTpUnz++ecoUKAAZsyYAQD4/fffUa1aNdjY2KBkyZKYOnUqMjIyALxYcnXKlClwd3eHtbU1ihUrhqFDh+rtNykpCb1794a9vT3c3d2xYsUKve1hYWFo0KABVCoVnJ2d0b9/fyQmJmZ7/p8/f46ePXvCzs4ORYsWxfz583O4WkTvPhb+DMzMjKeUiAzLzMwM5cqVY34hIoNjfiEipTC/EJFSmF+I6GVLKizBnhF75NdHvj6CJRWWIDkmGQAQczMGSyoswd8//C33+S3oN6z+eLX8+ubem1hSYQlu7r0JGxsbAMDqj1fjt6Df5D4vv18pBw8exLVr1xASEoKff/4Zv/zyC6ZOnarXZ926dbCwsMDff/+NRYsW4dtvv8WqVavk7UOGDMHJkyexefNmXLp0CR06dEDTpk1x48YNuU9SUhLmzJmDVatW4cqVKyhcuPAbx7xu3ToUKFAAp0+fxjfffINp06Zh//79en2mTJmCNm3aICwsDL1798bRo0fRs2dPDBs2DFevXsXy5cuxdu1auSi4Y8cOLFiwAMuXL8eNGzfw22+/wdfXV2+f8+fPR40aNXDhwgUMGjQIAwcORHh4OIAXRbyAgAAULFgQZ86cwbZt23DgwAEMGTIk23GMGTMGhw8fxu+//459+/YhJCQE58+ff+PzQvQyY3xmscj3IxIRUZ5ZWVkZOwQiMlHML0SkFOYXIlIK8wsRKcWYz6+zsrLC6tWrYWtriwoVKmDatGkYM2YMpk+fLhcO3NzcsGDBAkiShHLlyiEsLAwLFixAv379cPfuXaxZswZ3795FsWLFALxYtnPPnj1Ys2YNZs6cCeDFcptLlixB5cqV3zrmSpUqYfLkyQCAMmXKYPHixTh48CAaN24s9+natSt69eolv+7duzfGjRuHwMBAAEDJkiUxffp0fPnll5g8eTLu3r0LV1dXNGrUCJaWlnB3d0etWrX0jtusWTMMGjQIADB27FgsWLAAhw4dQrly5bBp0yakpKTgp59+QoECBQAAixcvRsuWLTFnzhwUKVJEb1+JiYn48ccfsWHDBjRs2BDAi4JmiRIl3vr8EBkLC38G9uqayEREb0ur1SIsLAy+vr4wNzc3djhEZEKYX4hIKcwvRKQU5hcietmgK4P0Xn868VN8OvFT+bVTaadMfVqvba33unRAaZS+UhpCCCQnJ0OlUqH3sd56fWoN1i88KaFy5cqwtbWVX/v5+SExMRH37t2Dh4cHAKBOnTp6xUk/Pz/Mnz8fGo0GYWFh0Gg0KFu2rN5+U1NT4ezsLL+2srJCpUqVDBLzq/spWrQooqOj9dpq1Kih9/rixYs4fvy4PMMPePGswZSUFCQlJaFDhw5YuHAhSpYsiaZNm6JZs2Zo2bIlLCz+V8p4+biSJMHV1VU+7rVr11C5cmW56AcAdevWhVarRXh4eKbCX0REBNLS0lC7dm25zcnJCeXKlcvr6SDKkjFqRiz8EREREREREREREREZmFqtRnx8fKb2uLg4ODg4GPRYiYmJMDc3x7lz5zL9YYSdnZ38b5VKZbCZjZaWlnqvJUnKVOR4uQCni3Pq1Klo27Ztpv3Z2NjAzc0N4eHhOHDgAPbv349BgwZh7ty5OHz4sHy83ByX6EPGwh8RERERERERERERkYGVK1cO+/bty9R+/vz5TDPzLl68KM84BIBTp07Bzs4Obm5ucp/Tp0/rvefUqVMoU6YMzM3NUbVqVWg0GkRHR+OTTz5RYDSGUa1aNYSHh6N06dLZ9lGpVGjZsiVatmyJwYMHw9vbG2FhYahWrdpr9+/j44O1a9fi+fPnctHx+PHj8jNiX1WqVClYWlri9OnTcHd3BwDExsbin3/+gb+//xuOksi4WPgjIiIiIiIiIiIiIjKwgQMHYvHixRg6dCj69u0La2trBAcH4+eff8Yff/yh1zctLQ19+vTBxIkTERkZicmTJ2PIkCHy8/0A4O7duxg5ciQGDBiA8+fP4/vvv8f8+fMBAGXLlkW3bt3Qs2dPzJ8/H1WrVsXjx49x8OBBVKpUCc2bN8/XsWdn0qRJaNGiBdzd3dG+fXuYmZnh4sWLuHz5Mr7++musXbsWGo0GtWvXhq2tLTZs2ACVSiUvd/o63bp1w+TJkxEYGIgpU6bg8ePH+OKLL9CjR49My3wCL2ZD9unTB2PGjIGzszMKFy6MCRMm6J13ovcNC38GxoRARIZmZmYGX19f5hciMjjmFyJSCvMLESmF+YWIlKSbbWcoJUuWxJEjRzBhwgQ0atQIaWlp8Pb2xrZt29C0aVO9vg0bNkSZMmXw6aefIjU1FV26dMGUKVP0+vTs2RPJycmoVasWzM3NMWzYMPTv31/evmbNGnz99dcYNWoU/v33XxQqVAh16tRBixYtDDqutxEQEIA///wT06ZNw5w5c2BpaQlvb2/07dsXAODo6IjZs2dj5MiR0Gg08PX1xR9//KH3nMKc2NraYu/evRg2bBhq1qwJW1tbtGvXDt9++22275k7dy4SExPRsmVL2NvbY9SoUVku0Ur0JozxmUUSQoh8P6oJSkhIgIODgyLrMxPRh00IgZSUFNjY2BhsDXYiIoD5hYiUw/xCREphfiH68KSkpOD27dvw8vKCjY2NYscRQkAIAUmS8j2/BAUFIS4uDr/99lu2ferVq4cqVapg4cKF+RYXEeVOTnkqPj4ejo6OiI+Ph1qtzpd4+OdRBsaHiBKRoWm1WoSHhzO/EJHBMb8QkVKYX4hIKcwvRKSklJQUY4dARCbGGJ9ZWPgjIiIiIiIiIiIiIiIiMgF8xh8RERERERERERERkZGsXbv2tX1CQkIUj4OITANn/BERvQfMzc2NHQIRmSjmFyJSCvMLESmF+YWIiIgoe5zxZ2D88ElEhmZubg5fX19jh0FEJoj5hYiUwvxCREphfiEipUiSBFtbW2OHQUQmxhg1I874MzAhhLFDICITI4RAQkIC8wsRGRzzCxEphfmFiJTC/EJEShFCQKPRML8QkUEZI6ew8GdgWq3W2CEQkYnRarW4desW8wsRGRzzCxEphfmFiJTC/EJESkpNTTV2CERkYozxmYWFPyIiIiIiIiIiIiIiIiITwMIfERERERERERERERERkQlg4Y+I6D1gY2Nj7BCIyEQxvxCRUphfiEgpzC9E9CaioqLw9fTpqOnjg7LFi6Omjw++nj4dUVFRch9JkowY4fvP09MTCxcuNHYYiqpXrx6GDx8uv86PMR88eBA+Pj7QaDSKHoey17lzZ8yfP9/YYeQaC38GZm5ubuwQiMjEmJubw9vbm/mFiAyO+YWIlML8QkRKYX4hojexYP58eJQogWMzZ2L49etY+uABhl+/jqMzZ8KjRAks+PZbSJIElUpl0OJfUFAQJEmCJEmwtLREkSJF0LhxY6xevTrPz/2aMmUKqlSpYrDYdIxRrLt//z6srKxQsWLFfD2uEs6cOYP+/fsreowvv/wSEydOlH/2aTQazJ49G97e3lCpVHByckLt2rWxatUqReN4H6xdu1b+ntN95eYPhkJCQlCtWjVYW1ujdOnSWLt2rd72iRMnYsaMGYiPj89zTMb4zMLCn4Hx4dJEZGharRZPnz5lfiEig2N+ISKlML8QkVKYX4gorxbMn49ZEybgiEaDPSkp6AagIYBuAPampOCIRoNZ//0vvp0/HxkZGRBCGPT4TZs2xcOHDxEZGYndu3ejfv36GDZsGFq0aIGMjAyDHut9sXbtWnTs2BEJCQk4ffq0scN5Ky4uLrC1tVVs/8eOHUNERATatWsnt02dOhULFizA9OnTcfXqVRw6dAj9+/dHXFycYnGkpaW98Xvv3r1rwEheT61W4+HDh/LXnTt3cux/+/ZtNG/eHPXr10doaCiGDx+Ovn37Yu/evXKfihUrolSpUtiwYUOe4zHGZxYW/gzM0D8YiIiEELh37x7zCxEZHPMLESmF+YWIlML8QkR5ERUVhXFjx+KP1FTUzqZPbQB/pKZi/NixihQorK2t4erqiuLFi6NatWr473//i99//x27d+/Wm1UUFxeHvn37wsXFBWq1Gg0aNMDFixcBvCiUTZ06FRcvXpRnMenem9P7dP744w/UrFkTNjY2KFSoENq0aQPgxbKVd+7cwYgRI+T96hw7dgyffPIJVCoV3NzcMHToUDx//lzeHh0djZYtW0KlUsHLywsbN27M1fkQQmDNmjXo0aMHunbtih9//FFve2RkJCRJwi+//IL69evD1tYWlStXxsmTJ/X67dixAxUqVIC1tTU8PT0zLcPo6emJr7/+Gj179oSdnR08PDywc+dOPH78GK1atYKdnR0qVaqEs2fPyu95+vQpunTpguLFi8PW1ha+vr74+eefcxzPqzMmX3c9Ll68iPr168Pe3h5qtRrVq1fXi+FVmzdvRuPGjfVmre3cuRODBg1Chw4d4OXlhcqVK6NPnz4YPXq03Eer1eKbb75B6dKlYW1tDXd3d8yYMUPeHhYWhgYNGkClUsHZ2Rn9+/dHYmKivD0oKAitW7fGjBkzUKxYMZQrVw4AcO/ePXTs2BGOjo5wcnJCq1atEBkZmeM58vLyQqNGjbB+/XokJSXl2NcQJEmCq6ur/FWkSJEc+y9btgxeXl6YP38+fHx8MGTIELRv3x4LFizQ69eyZUts3rw5z/EY4zMLC39EREREREREREREZHJWrVyJ+paW2Rb9dGoDqGdlhZ/WrcuPsNCgQQNUrlwZv/zyi9zWoUMHREdHY/fu3Th37hyqVauGhg0bIiYmBp06dcKoUaNQoUIFeRZTp06dXvs+AAgODkabNm3QrFkzXLhwAQcPHkStWrUAAL/88gtKlCiBadOmyfsFgIiICDRt2hTt2rXDpUuXsGXLFhw7dgxDhgyR4w0KCsK9e/dw6NAhbN++HUuWLEF0dPRrx37o0CEkJSWhUaNG6N69OzZv3qxXUNSZMGECRo8ejdDQUJQtWxZdunSRZ0ieO3cOHTt2ROfOnREWFoYpU6bgq6++yrQ844IFC1C3bl1cuHABzZs3R48ePdCzZ090794d58+fR6lSpdCzZ0+5MJOSkoLq1asjODgYly9fRv/+/dGjRw/8/fffub20r70e3bp1Q4kSJXDmzBmcO3cO48aNg6WlZbb7O3r0KGrUqKHX5urqir/++guPHz/O9n3jx4/H7Nmz8dVXX+Hq1avYtGmTXAB7/vw5AgICULBgQZw5cwbbtm3DgQMH9K4v8OLZguHh4di/fz/+/PNPpKenIyAgAPb29jh69CiOHz8OOzs7NG3aNMcZgVevXkWtWrUwceJEFClSBL1798bhw4ezLIht3LgRdnZ2OX4dPXo022MBQGJiIjw8PODm5oZWrVrhypUrOfY/efIkGjVqpNcWEBCQqdhcq1Yt/P3330hNTc1xf+8EQQYRHx8vAIiYmBhjh0JEJiYjI0NcuHBBZGRkGDsUIjIxzC9EpBTmFyJSCvML0YcnOTlZXL16VSQnJ+f5vTW8vcUGQIhcfG0ARLVy5YRWqzVY7IGBgaJVq1ZZbuvUqZPw8fERQghx9OhRoVarRUpKil6fUqVKieXLlwshhJg8ebKoXLmy3vbcvM/Pz09069Yt2xg9PDzEggUL9Nr69Okj+vfvn+lYZmZmIjk5WYSHhwsA4u+//5a3X7t2TQDItK9Xde3aVQwfPlx+XblyZbFmzRr59e3btwUAsWrVKrntypUrAoC4du2avI/GjRvr7XfMmDGifPnyeuPq3r27/Prhw4cCgPjqq6/ktpMnTwoA4uHDh9nG27x5czFq1Cj5tb+/vxg2bJjecXRjzs31sLe3F2vXrs32eK9ycHAQP/30k17blStXhI+PjzAzMxO+vr5iwIABYteuXfL2hIQEYW1tLVauXJnlPlesWCEKFiwoEhMT5bbg4GBhZmYmoqKihBAv7t0iRYqI1NRUuc/69etFuVe+R1JTU4VKpRJ79+597Vi0Wq3466+/RFBQkLC3txdeXl5i8uTJ4tatW3qx37hxI8evpKSkbI9x4sQJsW7dOnHhwgUREhIiWrRoIdRqtbh371627ylTpoyYOXOmXltwcLAAoHesixcvCgAiMjIy0z5yylMxMTECgIiPj8/x/BiShbEKjkRElHv29vbGDoGITBTzCxEphfmFiJTC/EJEuRWfkADXXPYtAiAhIUHJcPQIIeSlNS9evIjExEQ4Ozvr9UlOTkZERES2+8jN+0JDQ9GvX788xXbx4kVcunRJb/lOIQS0Wi1u376Nf/75BxYWFqhevbq83dvbG46OjjnuNy4uDr/88guOHTsmt3Xv3h0//vgjgoKC9PpWqlRJ/nfRokUBvFhe1NvbG9euXUOrVq30+tetWxcLFy6ERqOBubl5pn3oZrv5+vpmaouOjoarqys0Gg1mzpyJrVu34t9//0VaWhpSU1Nz/Qy/3FyPkSNHom/fvli/fj0aNWqEDh06oFSpUtnuMzk5WW+ZTwAoX748Ll++jHPnzuH48eM4cuQIWrZsiaCgIKxatQrXrl1DamoqGjZsmOU+r127hsqVK6NAgQJyW926daHVahEeHq53rqysrPTGd/PmzUw/h1NSUnK8T3UkSUL9+vVRv359LFq0CAMHDsTUqVMRGhqK3377DcCLn/Fv83Pez88Pfn5+8uuPPvoIPj4+WL58OaZPn/7G+wUAlUoFAPmyXOnbYuHPwHRJhYjIUMzNzXP8AEBE9KaYX4hIKcwvRKQU5hciygsHtRpRDx7kqu8jAI6OjnrPuVPStWvX4OXlBeDF0oRFixZFSEhIpn45FdNy8z5dsSIvEhMTMWDAAAwdOjTTNnd3d/zzzz953icAbNq0CSkpKahd+3+Lr+oKiv/88w/Kli0rt7+8/KXummi12jwdL6t95LTfuXPnYtGiRVi4cCF8fX1RoEABDB8+PMdlLF+Wm+sxZcoUdO3aFcHBwdi9ezcmT56MzZs3y89dfFWhQoUQGxubqd3MzAw1a9ZEzZo1MXz4cGzYsAE9evTAhAkT3uiaZ+XlwiDwYnzVq1fP8nmOLi4uudrn+fPn8dNPP+Hnn3+GJElyIVRn48aNGDBgQI772L17Nz755JNcHc/S0hJVq1bFzZs3s+3j6uqKR48e6bU9evQIarVa71zqlmvN7Vh1jFEzYuHPwPKafIiIXker1SI6OhqFCxeGmRkfzUpEhsP8QkRKYX4hIqUwvxBRXrTq2hXrZ85Et5SU1/b9SaVCy06d9GbiKeWvv/5CWFgYRowYAQCoVq0aoqKiYGFhAU9PzyzfY2VlBY1Go9eWm/dVqlQJBw8eRK9evfK036tXr6J06dJZvsfb2xsZGRk4d+4catasCQAIDw9HXFxcNiN+4ccff8SoUaMyze4bNGgQVq9ejdmzZ+f4fh0fHx8cP35cr+348eMoW7bsWxVZjh8/jlatWqF79+4AIBcky5cvn6v35+Z6AEDZsmVRtmxZjBgxAl26dMGaNWuyLfxVrVoVV69efe2xdTE+f/4cZcqUgUqlwsGDB/WKajo+Pj5Yu3Ytnj9/Lhf3jh8/DjMzM5QrVy7H8W3ZsgWFCxeGWq1+bUw69+/fx4YNG7B+/XpERESgZcuW+PHHH9G0aVNYWOiXqD7//HO9wnBWihcvnutjazQahIWFoVmzZtn28fPzw65du/Ta9u/frzdzEAAuX76MEiVKoFChQrk+PmCcmhE/IRmYyOKBlEREb0MIgaioKOYXIjI45hciUgrzCxEphfmFiPKib79+OJSejtOv6XcaQEhaGrr36GHwGFJTUxEVFYV///0X58+fx8yZM9GqVSu0aNECPXv2BAA0atQIfn5+aN26Nfbt24fIyEicOHECEyZMwNmzZwEAnp6euH37NkJDQ/HkyROkpqbm6n2TJ0/Gzz//jMmTJ+PatWsICwvDnDlz5Pg8PT1x5MgR/Pvvv3jy5AkAYOzYsThx4gSGDBmC0NBQ3LhxA7///juGDBkCAChXrhyaNm2KAQMG4PTp0zh37hz69u2b40yz0NBQnD9/Hn379kXFihX1vrp06YJ169YhIyMjV+d01KhROHjwIKZPn45//vkH69atw+LFizF69Oi8X6CXlClTBvv378eJEydw7do1DBgwINNMsJy87nokJydjyJAhCAkJwZ07d3D8+HGcOXMGPj4+2e4zICBAb2lUAGjfvj0WLFiA06dP486dOwgJCcHgwYNRtmxZeHt7w8bGBmPHjsWXX36Jn376CRERETh16hR+/PFHAEC3bt1gY2ODwMBAXL58GYcOHcIXX3yBHj16yMt8ZqVbt24oVKgQWrVqhaNHj+L27dsICQnB0KFDcf/+/Wzf5+HhgV9//RWDBw/Gw4cPsW3bNrRo0SJT0Q94sdRn6dKlc/zK6T6bNm0a9u3bh1u3buH8+fPo3r077ty5o1cAHT9+vPy9BwD/+c9/cOvWLXz55Ze4fv06lixZgq1bt8qFeZ2jR4+iSZMm2R47O8b4zMLCHxERERERERERERGZHFdXV8yeMwctra2zLf6dBtDS2hqz5syBq2tunwiYe3v27EHRokXh6emJpk2b4tChQ/juu+/w+++/y7PTJEnCrl278Omnn6JXr14oW7YsOnfujDt37siFmHbt2qFp06aoX78+XFxc5KUSX/e+evXqYdu2bdi5cyeqVKmCBg0a4O+//5bjmzZtGiIjI1GqVCl5CcNKlSrh8OHD+Oeff/DJJ5+gatWqmDRpEooVKya/b82aNShWrBj8/f3Rtm1b9O/fH4ULF872PPz4448oX748vL29M21r06YNoqOjM826yk61atWwdetWbN68GRUrVsSkSZMwbdq0TDMJ82rixImoVq0aAgICUK9ePbi6uqJ169a5fv/rroe5uTmePn2Knj17omzZsujYsSM+++wzTJ06Ndt9duvWDVeuXEF4eLjcFhAQgD/++AMtW7ZE2bJlERgYCG9vb+zbt08upn311VcYNWoUJk2aBB8fH3Tq1AnR0dEAAFtbW+zduxcxMTGoWbMm2rdvj4YNG2Lx4sU5js/W1hZHjhyBu7s72rZtCx8fH/Tp0wcpKSk5zgC8cuUKTp8+jUGDBqFgwYK5Pp9vIjY2Fv369YOPjw+aNWuGhIQEnDhxQm/W5sOHD3H37l35tZeXF4KDg7F//35UrlwZ8+fPx6pVqxAQECD3SUlJwW+//Zbn52UaiyT4J1IGkZCQAAcHB8TExCh+8xLRh0U3Jd3X15fPESUig2J+ISKlML8QkVKYX4g+PCkpKbh9+za8vLxgY2PzRvtY8O23GPfll6hnaYmeKSkoAiAKwE82Njicno7Z33yD4SNGIDk5GSqVKt+e80eUG2PGjEFCQgKWL19u7FA+WEuXLsWvv/6Kffv2Zbk9pzwVGxsLJycnxMfH52mJ1LfBGX8Gxh8KRGRokiTBycmJ+YWIDI75hYiUwvxCREphfiGiNzFi5EjcuX8fn06YgIU+PhhUrBgW+fjAf8IE3Ll/HyNGjgQA/kEBvZMmTJgADw8Pozwrjl6wtLTE999//0bvNcZnFs74MxDdjL/8rNoSERERERERERERmTJDzPgjIlJSTnnKGLUjzvgzMFbdicjQtFot7t69y/xCRAbH/EJESmF+ISKlML8QkVKEEEhNTQXnyRCRIRnjMwsLfwbGHwxEZGhCCMTExDC/EJHBMb8QkVKYX4hIKcwvRB+u/Pi+12g0ih+DiExPTvnJGJ9ZWPgjIiIiIiIiIiIioneSpaUlACApKcnIkRARZS0tLQ3Au/OcUAtjB0BERERERERERERElBVzc3M4OjoiOjoaAGBrawtJkgx+HN1Sn5IkKbJ/IjJNWq0Wjx8/hq2tLSws3o2S27sRhQnhDwUiMjRJkuDq6sr8QkQGx/xCREphfiEipTC/EH2YXF1dAUAu/ilBCAGtVgszMzPmGCLKEzMzM7i7u2eZO4yRTyTBRdENIiEhAQ4ODoiPj4darTZ2OEREREREREREREQmRaPRID093dhhEBHpsbKygplZ1k/WM0btiDP+DIwPgCUiQ9NoNIiMjISnp+c7s040EZkG5hciUgrzCxEphfmF6MNmbm6u2Pc+8wsRKcEYNaOsS5BERPROefbsmbFDICITxfxCREphfiEipTC/EJFSmF+IyBSw8EdERERERERERERERERkAlj4IyIiIiIiIiIiIiIiIjIBLPwZmCRJxg6BiEyMJElwc3NjfiEig2N+ISKlML8QkVKYX4hIKcwvRKQEY+QUi3w/ookzM2MtlYgMy8zMDM7OzsYOg4hMEPMLESmF+YWIlML8QkRKYX4hIiUYo2bEKpWBaTQaY4dARCZGo9Hg+vXrzC9EZHDML0SkFOYXIlIK8wsRKYX5hYiUYIycwsIfEdF7ICUlxdghEJGJYn4hIqUwvxCRUphfiEgpzC9EZApY+CMiIiIiIiIiIiIiIiIyASz8EREREREREREREREREZkAFv4MzBgPaiQi02ZmZoaSJUsyvxCRwTG/EJFSmF+ISCnML0SkFOYXIlKCMXKKRb4f0cRJkmTsEIjIxEiSBLVabewwiMgEMb8QkVKYX4hIKcwvRKQU5hciUoIxakb88wUD02g0xg6BiEyMRqNBWFgY8wsRGRzzCxEphfmFiJTC/EJESmF+ISIlGCOnsPBHRPQe4IdOIlIK8wsRKYX5hYiUwvxCREphfiEiU8DCHxEREREREREREREREZEJYOGPiIiIiIiIiIiIiIiIyARIQghh7CBMQUJCAhwcHBAXFwcHBwdjh0NEJkQIgZSUFNjY2BjlYbBEZLqYX4hIKcwvRKQU5hciUgrzCxEpIT4+Ho6OjoiPj4darc6XY3LGHxHRe8DKysrYIRCRiWJ+ISKlML8QkVKYX4hIKcwvRGQKWPgzMK1Wa+wQiMjEaLVahIWFMb8QkcExvxCRUphfiEgpzC9EpBTmFyJSgjFyCgt/RERERERERERERERERCaAhT8iIiIiIiIiIiIiIiIiE8DCHxEREREREREREREREZEJkIQQwthBmIKEhAQ4ODggLi4ODg4Oxg6HiEyIEAJarRZmZmaQJMnY4RCRCWF+ISKlML8QkVKYX4hIKcwvRKSE+Ph4ODo6Ij4+Hmq1Ol+OyRl/RETvgbS0NGOHQEQmivmFiJTC/EJESmF+ISKlML8QkSlg4c/AtFqtsUMgIhOj1WoRHh7O/EJEBsf8QkRKYX4hIqUwvxCRUphfiEgJxsgpLPwRERERERERERERERERmQAW/oiIiIiIiIiIiIiIiIhMAAt/RETvAXNzc2OHQEQmivmFiJTC/EJESmF+ISKlML8QkSmQhBDC2EGYgoSEBDg4OCA+Ph5qtdrY4RAREREREREREREREZERGaN2xBl/BsY6KhEZmhACCQkJzC9EZHDML0SkFOYXIlIK8wsRKYX5hYiUYIycwsKfgWm1WmOHQEQmRqvV4tatW8wvRGRwzC9EpBTmFyJSCvMLESmF+YWIlGCMnMLCHxEREREREREREREREZEJYOGPiIiIiIiIiIiIiIiIyASw8EdE9B6wsbExdghEZKKYX4hIKcwvRKQU5hciUgrzCxGZAknwaaUGkZCQAAcHB8THx0OtVhs7HCIiIiIiIiIiIiIiIjIiY9SOOOPPwPjwVyIyNK1Wi6dPnzK/EJHBMb8QkVKYX4hIKcwvRKQU5hciUoIxcgoLfwbGCZREZGhCCNy7d4/5hYgMjvmFiJTC/EJESmF+ISKlML8QkRKMkVNY+CMiIiIiIiIiIiIiIiIyASz8EREREREREREREREREZkAFv6IiN4D9vb2xg6BiEwU8wsRKYX5hYiUwvxCREphfiEiUyAJLlpsEAkJCXBwcEB8fDzUarWxwyEiIiIiIiIiIiIiIiIjMkbtiDP+DEyr1Ro7BCIyMVqtFlFRUcwvRGRwzC9EpBTmFyJSCvMLESmF+YWIlGCMnMLCn4FxAiURGZoQAlFRUcwvRGRwzC9EpBTmFyJSCvMLESmF+YWIlGCMnMLCHxEREREREREREREREZEJYOGPiIiIiIiIiIiIiIiIyASw8GdgkiQZOwQiMjGSJMHJyYn5hYgMjvmFiJTC/EJESmF+ISKlML8QkRKMkVMkwUWLDSIhIQEODg6Ij4+HWq02djhERERERERERERERERkRMaoHXHGn4FptVpjh0BEJkar1eLu3bvML0RkcMwvRKQU5hciUgrzCxEphfmFiJRgjJzCwp+BcQIlERmaEAIxMTHML0RkcMwvRKQU5hciUgrzCxEphfmFiJRgjJzCwh8RERERERERERERERGRCWDhj4iIiIiIiIiIiIiIiMgEsPBnYJIkGTsEIjIxkiTB1dWV+YWIDI75hYiUwvxCREphfiEipTC/EJESjJFTJMFFiw0iISEBDg4OiI+Ph1qtNnY4REREREREREREREREZETGqB1xxp+BaTQaY4dARCZGo9EgIiKC+YWIDI75hYiUwvxCREphfiEipTC/EJESjJFTWPgjInoPPHv2zNghEJGJYn4hIqUwvxCRUphfiEgpzC9EZApY+CMiIiIiIiIiIiIiIiIyASz8EREREREREREREREREZkAFv4MTJIkY4dARCZGkiS4ubkxvxCRwTG/EJFSmF+ISCnML0SkFOYXIlKCMXKKRb4f0cSZmbGWSkSGZWZmBmdnZ2OHQUQmiPmFiJTC/EJESmF+ISKlML8QkRKMUTNilcrANBqNsUMgIhOj0Whw/fp15hciMjjmFyJSCvMLESmF+YWIlML8QkRKMEZOYeGPiOg9kJKSYuwQiMhEMb8QkVKYX4hIKcwvRKQU5hciMgUs/BERERERERERERERERGZABb+iIiIiIiIiIiIiIiIiEwAC38GZogHNWoztNjcajOu/XLNABER0fvOzMwMJUuWNMqDYInItDG/EJFSmF+ISCnML0SkFOYXIlKCMXIKs5iBSZL01vtIS0xD+M5w7Oi6wwAREdH7TpIkqNVqg+QXIqKXMb8QkVKYX4hIKcwvRKQU5hciUoIxcgoLfwam0Wjeeh82jjYYcX8EJiRPMEBERPS+02g0CAsLM0h+ISJ6GfMLESmF+YWIlML8QkRKYX4hIiUYI6dY5PsRKVfUxdXGDoGI3iH80ElESmF+ISKlML8QkVKYX4hIKcwvRGQKOOPvHRSxLwJb2mxB6NpQ/Bb0G5KeJBk7JCIiIiIiIqJMLm28hFsHbxk7DCIiIiIi+n8s/L2DYm/FInxnOP498y8urruIm3tuGjskIiIiIiIiIj3aDC1+7f4rDo4/aOxQiIiIiIjo/0lCCGHsIExBQkICHBwcEBcXBwcHh7fen9AKZKRmIPZWLApXKGyACInofSWEQEpKCmxsbPiAaSIyKOYXIlIK88uHIT05HedWnIO6hBrl25U3djj0gWB+ISKlML8QkRLi4+Ph6OiI+Ph4qNX584g3PuPvHSWZSbBUWbLoR0QAACsrK2OHQEQmivmFiJTC/GL6LFWWqDOsDoAXf7wqmfGXpJQ/mF+ISCnML0RkCrjUp4Fptdq33sf90/cReTgSAKBJ1+Dar9dwY9eNt94vEb2ftFotwsLCDJJfiIhexvxCREphfvmw7Bm+B8sqLwMXFKL8wPxCREphfiEiJRgjp7Dw9w76679/YXOrzQBezPz7LfA3HJt9zMhREREREREREf3PH/3/wI8f/QghBKwdrJEck2zskIiIiIiIPnhGLfwtXboUlSpVglqthlqthp+fH3bv3i1vT0lJweDBg+Hs7Aw7Ozu0a9cOjx490tvH3bt30bx5c9ja2qJw4cIYM2YMMjIy9PqEhISgWrVqsLa2RunSpbF27dpMsfzwww/w9PSEjY0Nateujb///luRMefGR2M+QtNFTQEAZuZmaPdzO7RY3sJo8RARERERERG9SggBbYYWTRc0Re9jvWHrbGvskIiIiIiIPnhGLfyVKFECs2fPxrlz53D27Fk0aNAArVq1wpUrVwAAI0aMwB9//IFt27bh8OHDePDgAdq2bSu/X6PRoHnz5khLS8OJEyewbt06rF27FpMmTZL73L59G82bN0f9+vURGhqK4cOHo2/fvti7d6/cZ8uWLRg5ciQmT56M8+fPo3LlyggICEB0dHT+nYyXlG5aGlUCq8ivyzYvCxcfF6PEQkRERERERJSVz1d+jn5/9+Oz/YiIiIiI3iGSeMcW4XdycsLcuXPRvn17uLi4YNOmTWjfvj0A4Pr16/Dx8cHJkydRp04d7N69Gy1atMCDBw9QpEgRAMCyZcswduxYPH78GFZWVhg7diyCg4Nx+fJl+RidO3dGXFwc9uzZAwCoXbs2atasicWLFwN4seaqm5sbvvjiC4wbNy7LOFNTU5Gamiq/TkhIgJubG54+fQoHBwcAgCRJMDMzg1ar1XvWga5do9Ho7TO7djMzM2QkZyDxcSLUJdR67bp4X+2fVbu5ufmLv8jMov3VGLNrN+SYJEnKsp1j4pg4Jv3YdccxNzeHubm5SYzpde0cE8fEMeXPmIQQyMjIkPuYwpiyip1j4pg4pvwfk+7flpaWEEKYxJhyip1jAqLOR+H4nOOoO74uXKu4msSYTPE6mcKYNBoNNBqNHJspjMkUrxPHxDG9j2PSvU+3f1MYky5GU7pOHBPH9L6NKT4+HgULFkR8fDzUajXyg0W+HCUXNBoNtm3bhufPn8PPzw/nzp1Deno6GjVqJPfx9vaGu7u7XPg7efIkfH195aIfAAQEBGDgwIG4cuUKqlatipMnT+rtQ9dn+PDhAIC0tDScO3cO48ePl7ebmZmhUaNGOHnyZLbxzpo1C1OnTs3UfvXqVdjZ2QF4UcR0d3fH/fv3ERMTI/dxdXWFq6srIiMj8ezZM7ndzc0NapUa33p+C7cWbig/tDwAwMPNA8s8l6Fg1YKo810duX+5cuVgZWWFsLAwvRh8fX2RlpaG8PBwuc3c3By+vr549uwZbt26Jbfb2NjA29sbsbGxuHfvntxub2+PUqVKITo6GlFRUXL7m4zJ2dkZN27cQEpKitxesmRJqNVqXL16Ve+bhGPimDimrMek0Wjg4eGBQoUKmcyYANO7ThwTx/S+jcnS0hJhYWEwNzc3mTGZ4nXimDim93VM5ubmqFixokmNyRSv05uOKXRXKG7uvYkSzUrAtqgtxAOBazuuQVVGBTdzt/dyTKZ4nUxxTP/++y8eP34sf34xhTGZ4nXimDim93VMxYsXR6FChUxqTKZ4nTgmjul9GtPLE8jyi9Fn/IWFhcHPzw8pKSmws7PDpk2b0KxZM2zatAm9evXKdFJq1aqF+vXrY86cOejfvz/u3Lmjt2xnUlISChQogF27duGzzz5D2bJl0atXL73C3q5du9C8eXMkJSUhNjYWxYsXx4kTJ+Dn5yf3+fLLL3H48GGcPn06y7izm/H3+PFjFCxYEMCbVZFTYlKwqcUmlGlRBh+P/xjAi0LkgbEHoHZTo8agGnL/D60yzjFxTB/qmDQaDa5cuYKKFSvC0tLSJMb0unaOiWPimPJnTBqNBmFhYahQoYL8y7P3fUxZxc4xcUwcU/6PSff5pVKlSpAkySTGlFPsH+KYDk8/jJBJIeh1oheK1yoOoRXQpmphrjLX28f7NCZTvE6mOKb09HRcvnxZ/vxiCmMyxevEMXFM7+OYXv788qr3dUy6GE3pOnFMHNP7NqbY2Fg4Ozt/WDP+ypUrh9DQUMTHx2P79u0IDAzE4cOHjR3Wa1lbW8Pa2jpTu7m5ud5fzQP/u+hZ9X2VbSFb9D3VN1N7428aZxtLVvvJrl2SpCzbs4sxr+15icVQ7RwTx2SoGPPanp9j0v2wyWuMeW3ndeKYDBVjXts5JuOMSZIkOc5Xt7+vY8prO8fEMRkqxry2fwhjkiT9JYRf9T6O6XUxfkhjqjOsDko1LgXXKq4v9mcOwDLLrnmOMa/tvE4f3piy+vzyvo9JqRjz2s4xcUyGijGv7e/KmHSfX0xpTIZs55g4JkPFmNf293lMurySn7IeZT6ysrJC6dKlUb16dcyaNQuVK1fGokWL4OrqirS0NMTFxen1f/ToEVxdXzwvwNXVFY8ePcq0Xbctpz5qtRoqlQqFChWCubl5ln10+yAiIiIiIiKi/7FWW6NEnRKwsPnf3xPH3o7F5S2XoUnT5PBOIiIiIiJSktELf6/SarVITU1F9erVYWlpiYMHD8rbwsPDcffuXXlJTj8/P4SFhSE6Olrus3//fqjVapQvX17u8/I+dH10+7CyskL16tX1+mi1Whw8eFBv6c/8EhUahWOzjyH2VqxeuzZDi03NN+GPAX/ke0xEZHzZ/RUJEdHbYn4hIqUwv5gurUaLJ9efQJOuX+A7v/I8dnTegac3nhopMvpQML8QkVKYX4jIFBh1qc/x48fjs88+g7u7O549e4ZNmzYhJCQEe/fuhYODA/r06YORI0fCyckJarUaX3zxBfz8/FCnTh0AQJMmTVC+fHn06NED33zzDaKiojBx4kQMHjxYXobzP//5DxYvXowvv/wSvXv3xl9//YWtW7ciODhYjmPkyJEIDAxEjRo1UKtWLSxcuBDPnz9Hr1698jymt/3hcPf4XRwcfxDFaxVHwZIF5XYzCzMkPkqEhcroq7MSUT4zN3/xEFsiIkNjfiEipTC/mLbYiFj84PMDPhrzkd5jKSp0rIDCFQvDvpi9EaMjU8f8QkRKYX4hIiUY4w8KjFpFio6ORs+ePfHw4UM4ODigUqVK2Lt3Lxo3fvE/DgsWLICZmRnatWuH1NRUBAQEYMmSJfL7zc3N8eeff2LgwIHw8/NDgQIFEBgYiGnTpsl9vLy8EBwcjBEjRmDRokUoUaIEVq1ahYCAALlPp06d8PjxY0yaNAlRUVGoUqUK9uzZgyJFiuR5TK8+SDKvfLv4okTtEnAu65xpW7+/+0Eyy//1YInIuIQQePbsGezt7Y2yJjQRmS7mFyLToknXID0pHTYONsYOhfnFxFmoLPDRlx+hVJNSeu2uVVzhWoWPzCBlMb8QkVKYX4hICW9bM3oTkjDGUU1QQkICHBwcEBMTg4IFC77+DUREuaTRaBAWFgZfX18uOUFEBsX8QmRazi47i78m/IUuf3aBo4ejUWddMb982LQaLczM37kni5CJYH4hIqUwvxCREmJjY+Hk5IT4+Hio1ep8OSY/ib9jkmOSkZ6UnuW2Zw+f4fR3p/Hg7IN8joqIiIiIiN51toVsUbBUQfze63esb7zeKH9ZSrTWfy1W1lxp7DCIiIiIiD5YLPy9Y3Z03YG5LnOz3Jb4MBF7hu3Bjd038jkqIiIiIiJ615VvXx59T/dFxS4VUaZFGWjTtcYOiUzUxmYbsXvY7iy3uVR0QeGKhfM5IiIiIiIi0jHqM/4os1IBpeBcLvPz/QCgkHch9PyrJ1x8XPI5KiIyNhsb4z+rh4hME/ML0fvv2cNnkCQJdq52kCQJ9SbXM3ZIAJhfTJU2Q4uEewmwsrPKcnvzH5rnc0T0IWJ+ISKlML8QkSngM/4MRPeMv/xcp5WIiIiIiGhV7VWIvhKNvqf66s200mq0kCQJkplkxOjIVAmt4L1FRERERPQaxqgdcalPA9NqlV1OR2gFkp4kKXoMInq3aLVaPH36VPH8QkQfHuYXItNQd2xdNPuhGVwq/G9lkH/+/AcL3Bbg7vG7RomJ+cX0ZVf0ex79HHtH7cXV7VfzOSL6UDC/EJFSmF+ISAnGyCks/BnY20yg1KRpsKXNFpxbcS7bPhsCNuC70t+98TGI6P0jhMC9e/feKr8QEWWF+YXo/fX0n6c4/s1xPL72GD5tfVAlsAok6X+FGAd3B1iqLJESl2KU+JhfTFfYz2G4uP5ittfWzMIMp749xWfTk2KYX4hIKcwvRKQEY+QUPuPvHZISl4LwP8JhV8wu2z5lmpeBU1knaDVamJmzbktERERE9CGKuhiFA2MPwNHLMctngBf2LYwvbn6hVwwkMoTjs48jOSYZlXtUznK7ykmFoRFD4eDhkM+RERERERERwMLfO6VA4QL4Ku0r/B97dx3e1Nk+cPybpO5CjXpL0ba4+zbYGIwpMybMfYP5xn57371zd1fmytgYM2A4w7U4VKi7e5P8/jgkbWnqSdO09+e6dqU9ec7Jk7XczTn3ee5bW6ttdsy4heO6cEZCCCGEEEKI7ijqrChu3HYj3lHeJp+XhJ+wlEuXXkppRmmLY5r7vRRCCCGEEEJYniwZ62ZUahV2jpKPFUI05u7ubu0pCCF6KIkvQtgmZ29ngkcH4+Lr0uyYI8uP8OfCP6mrquvCmdWT+NIz+UT7ED45vMUxVUVVpKxPobqkuotmJXobiS9CCEuR+CKE6Akk8WdmGo2mw/uWpJWQ9E8SlQWVzY7JO5zHT1f8xNEVRzv8OkII26LRaIiOju5UfBFCCFMkvghhu2oraqkpr2mxX0Ty2mS2vr6VsqyyLpyZQuJLz1SaWdqmvpH7vtzHZ1M/I21rWhfMSvQ2El+EEJYi8UUIYQnWiCmS+DMznU7X4X2P/X6Mz8/8nPRt6c2O0dZoSfg2gZz9OR1+HSGEbdHpdGRlZXUqvgghhCkSX4SwXRue2cCzbs+SfzS/2TETH5jIwpML8Qj16MKZKSS+9Exr/7OW572fpyy75WRy5JmRzHhpBj7RPl00M9GbSHwRQliKxBchhCVYI6ZITUkza+mO29aETQpj1luz8I/zb3aM3xA/Flcuxs5JfnRC9BZ6vZ6srCz8/PysPRUhRA8j8UUI2xU4LJCh1wxtsdSnW6BbF86oMYkvPVPE9AhUahVuAS3/bvkN8sNvkPzshWVIfBFCWIrEFyGEJXQmZ9RRkj3qRvwG++E3uOU/LGqNGrVGFmoKIYQQQgjRmw2+ZDCDLxnc4hhdnY7i1GLsXexbTdQI0RZxV8QRd0WctachhBBCCCGEaIFkkGxQ9v5sTvx9wtrTEEIIIYQQQnRj+UfzeSPqDba9uc3aUxG90A+X/sBn0z6z9jSEEEIIIYTodSTxZ2YqlarD+y5bsIwPRn7Q6ri/7/2b7y76zipLRIUQXU+lUuHj49Op+CKEEKZIfBHCdm18fiPLb1ne4hiPUA/G3TuOsMlhXTSrehJfep79X+/ns6mfkZPQtn7zGnsNGgeNhWcleiOJL0IIS5H4IoSwBGvEFCn1aWZqdcdzqRpHDXbOrf9Ixi4cy9Brh4IekL9DQvR4arWasLCuv2AnhLC+guMF+PTzsdjxJb4IYbtO/HWC7L3ZnPf+ec2OcXR35OyXz+7CWdWT+NLzlKSXkHswF3sX+zaNv+iriyw8I9FbSXwRQliKxBchhCV0JmfUUSq9LBszi5KSEjw9PSksLMTLy8va0xFC9CA6nY60tDRCQkKs8odCCGEdm1/ezKqHVrFg3QLCJlrm5FPiixC2S6fVUVdZh4Obg7WnYpLEFyGEpUh8EUJYisQXIYQlFBUV4e3tTXFxMR4eHl3ymhLBzEzyqEIIc9Pr9RQUFEh8EaKXiTk3hugZ0XiGeVrsNSS+CGG71Bp1m5J+qx5exVfnftUFM2pM4osozShl04ubSP031dpTET2MxBchhKVIfBFCWII1Yook/rqJuuo61j+9nuR1ya2OTduaxmsRr7Hr412Wn5gQQgghutyfC//k1xt+5dKfLsUz1JOashpqK2utPS0hRDeStjWNvMN5rY4rSioi90Auep1cwBKds/eLvSSuTmzz+Iq8ClY9uIrjfxy34KyEEEIIIYQQp5PEXzdRmlHKmsfWcHT50VbHOnk54ezjjJ2jtGgUQggheiJtjZbq4mpjH6Xt72znGZdnSN+WbuWZCSG6i69nf82K21a0Ou6S7y5hYcpCVGppDi46Z8VtK9j0/KY2j/cd4Mv1m65n3MJxFpyVEEIIIYQQ4nSSOTIzlapjJ9Tufd25eefNOHk5tTq2z4A+3LLrlg69jhDC9qhUKgIDAzscX4QQtmf2O7Mbfd9nYB/irozDK8LLrK8j8UUI23XGU2fg7Ots7Wk0S+JLz6LX67ni1yuwc2r7JQQ7RztCJ4RacFait5L4IoSwFIkvQghLsEZMUemlaLFZlJSU4Onp2aUNGoUQQgjRuxSnFpO0OomwyWH4RPtYezpCCBtQmlFK8rpkQsaF4B3pbe3piF5GV6ejOLVYfveEEEIIIUSvZY3ckZT6NDOtVtuh/apLqqkpr2nz+P3f7Gfjcxs79FpCCNui1Wo5ceJEh+OLEMK2ZOzIYNMLmyhOLW7yXNaeLH657hdObjhplteS+CJEz5e5O5OlVy4leU1yl76uxJeeRVen61CfyF9v+JU3ot6gurTaArMSvZXEFyGEpUh8EUJYgjViiiT+uolNL2ziWbdnyT2U26bx+z7fx7r/rUMWbArRO5SWllp7CkKILnLs92OsemgV5dnlTZ4LGRvCFcuvIHpmtNleT+KLELanLKuMd+Pf5d9X/211bN9RfZn3wzyizorqgpk1JvGl5zjw/QGecnqKoyta70nfUMzsGMbfNx5tjVxAFeYl8UUIYSkSX4QQPYH0+OsmAocFEn91PB7BbVvqOevNWZL0E0IIIXqgsfeMJfKMSPxj/Zs85+rvSv85/a0wKyFEd1JXXYe2RtumZIpbgBuDLxncBbMSPZmLnwv9zumHZ6hnu/YbcukQhlw6xEKzEkIIIYQQQpgiib9uYvAlg9t1Qu7TT/r6CCGEED2Rk6cTYZPCWhyjq9OhtpPCDUL0Vl7hXtx5+M527aPX663SVF70DNEzoomeYb7V5kIIIYQQQgjLkStGZlaWVdYlr6Or01GaWdquvoBCCNukUqkIDQ2Vi3VC9ALaGi15h/PQ1emaHfNGvzdYMn2JWV5P4osQvcM7se/w2ZTPuvQ1Jb4IgNqKWr676DvWP73e2lMRPYjEFyGEpUh8EUJYgjViiiT+zKymuGOJuKXzl7LphU1tHr//m/280vcVElclduj1hBC2Q61W4+vri1otIVuIni57fzZvD3q7xQukEdMjCB4XbJbXk/gihG0qTCpkx/s7KDhR0KbxwaODCRgWYOFZNSbxpWdZ/9R61v1vXbv3s3O2I+mfJPIO5llgVqK3kvgihLAUiS9CCEuwRkyRUp9m5jOg/SU4dVodh5Yeoq66rs37BMQFMPqO0e3usSCEsD1arZZjx44RExODRqOx9nSEEBbk5OXEhAcmEHVmVLNj5n4412yvJ/FFCNuUsSODFbeu4JLvLsEnuvXzj/M/Pb8LZtWYxJeeZe/ne9E4aJj6+NR27adSqXgw70EpTy3MSuKLEMJSJL4IISxBq229N7u5SeKvG1Br1Dxa8Sh1VW1P/AUOC+Tct8614KyEEN1JVVWVtacghOgCPtE+zHhhRpe+psQXIWxPxNQIrvr7KgLiunYVX3tJfOk5bt1zK9Ul1R3aV5J+whIkvgghLEXiixCiJ5BP4GaWuLJjpTdVKhX2zvZmno0QQggheprEVYksv2U5Jekl1p6KEMJKXP1diZ4RjVugW5vGJ69L5o+7/6AkTeKG6Bh7F/s2/76druB4Afu/3k9lQaWZZyWEEEIIIYQwRRJ/Zrbrw13t3qckrYTEVYntOhHS6/T8cOkPrHuy/X0WhBBCCNE9fTbtM1Y9sqrFMTkJOez6YBeFiYVdNCthS4pSilh+y3IqC+UCe0+m1+nR6/VtHp+9L5ttb24j/1i+BWcleqrq0moydmRQVdSxFRBHfzvK0vlLyTmQY+aZCSGEEEIIIUyRxJ+ZnfHUGe3e59jvx/hixhekb0tv8z4qtYqk1Ulk7shs9+sJIWyLWq0mKipKmksL0QuUpJVQnFzc4pjh1w/ngdwHCJsY1unXk/hi+06/EL/28bWkrEux0mxEV1nz+BqecniK/KNtS+TFXxXPorRFhE8Ot/DM6kl86Tkyd2by4egP2fv53g7tH3NuDPN+nIffID8zz0z0VhJfhBCWIvFFCGEJ1ogp0uPPzDpyMhM+JZxz3zmXgPj29ei4P+d+1Br5QyRET6dSqfDw8LD2NIQQXeDu43e3OsbRw9FsryfxxXbp9Xp2fbSLP+78gxu33kjgsEAAYmbHoHZQ4+TlZOUZCkvqM7APMbNj2vxzdvZ2xtnb2cKzakziS8/hGebJGc+cQdjkjt1w4tvfF9/+vmaelejNJL4IISxF4osQwhJUKlXXv6a+PTViRLNKSkrw9PQkOy2bPgF9pIG5EMJstFotBw8eZPDgwWg0GmtPRwhhZTqtjvwj+WgcNfhE+3TqWBJfbNc3531D5q5MfGJ8mPHCDILHBDd6vjCxkMxdmQy+ZLCVZii6E71OT1FyESqNCq9wry55TYkv4nR6vd4qFz1EzyPxRQhhKRJfhBCWUFhYiI+PD8XFxV12c4Fkp8zs1ZBXyUnomt4FuYdyOfTzIXRaXZe8nhDCerRarbWnIISwsKriKvZ+vrfVzxG1FbW8M+Qd1j+53iyvK/HF9hj6u/Ud3ZcFaxcYk36Gsp96vZ7vL/meZdcuk15/AgBtjZa3Br7FittWdO3rSnwRQG1lLS8FvMQv1/1i7amIHkTiixDCUiS+CCF6Akn8mdmgSwahcWzfHSFfnvMln5/1ebtfa+sbW/n+ou+pzJcLOkIIIYStK0ouYtm1yzj086EWxzm4OTDl8SkMunhQF81MdDcqtYorf7uSy5ZeZtxWcKKAF/1fZNMLm1CpVJz9ytlcueLKLi/vKLrGtre3sfLBlW0eb+dkx7T/TmPYdcMsNynRY/117198OvlTdHUdu+HU3tmegPgAvKO9zTwzIYQQQgghhCnS48/MzvvgPLy923dC4+Tp1GjVXmVtJXf9cReVdZWEeYQR5R3F5bGX4+7o3mi/odcMJWxSGPYu9maZuxBCCCGsxyvCi8t/ubzVPkgqlYrpT0zvolmJ7kylVqHX6/nxsh85ueEkUWdFGXtGR0yLsO7khEUdXnqYzN2ZzHhhRpv3mfzoZAvOSHRXep0elVopr1mUXMTqR1Yz/anp7SoVXV1STXlOeafaWVy98uoO7yuEEEIIIYRoH+nxZyaGHn9FRUV4enp26ljfJnzLFT9dAYCbgxsTQydy3/j7mBHd9hN7IUTPodfrqaqqwsnJSfqiCCHMSuKL7akurWbp/KWMum0UMbNiAPj56p+pLKzk0p8uxc6x/r4+vV5P5s5M7Jzs8I/1t9aUhQVUl1RTU16De5B764NPo6vTdUk/cokv1ldZWMknEz7h6pVX4xHiwZbXtvDXor+4etXVRJ0ZZe3pCdFhEl+EEJYi8UUIYQnFxcV4eXlJjz9b9vd9f7PxuY2dOsa+7H0AjA8Zz52j7+Sri76SpJ8QvZyDg4O1pyCE6EbWP7We94a+h7a28/0nJL7YlsydmRz/4zhFSUXGbRcsuYArf7uyUdIPoDS9lA9Hf8im5zd18SyFpTl6OLY76afX61kyfQlfzPjCQrNqSuKLdaVuSqXgeAFJ/yQBMG7hOG7de6tVkn6HfznM73f9Tl1VXZe/tuiZJL4IISxF4osQoieQxJ+ZHf3tKEd/O9rm8SVpJax7ch2ZuzKN2/bn7Adgftx8nj3rWXxdTJf8ytyVyathr7L93e2dm7QQolvT6XTs378fna5jfVWEELZh9ye7ec7zOZLWJLU6trq0mtqKWqqLqzv1mhJfbE/EtAjuz7mf+KvjjdsMZfxO5xHiwYyXZjD6jtFdNT3RRXIP5lKSVtKufVQqFR6hHrj3dacrir5IfLG+/nP6szBlIUMuG2LcFhAfQGVBJcd+P0ZRclGL+1fkVVBbWcvWN7eSvj29U3NJWZfC9re2U5ZV1qnjCAESX4QQliPxRQhhCdaIKZL4M7PrNl/Hdeuva/P43IO5rH18baMTqYScBABi/WPrx5XnUl5T3mhfR09H3ALdcHCVO1GEEEIIW+fi50Lw2GBcfF1aHTvj+RncdewuXPq0Plb0PM7ezji6O7Zp7IT7JhAyLsTCMxJd7dMpn7L0qqXt3u/Czy/koq8uktJVvYh7X/cmq4Ezd2fy9eyvW7xh9eBPB3k96nV2f7ybP+/+k0NLD3VqHhMfmsiitEV4hHZNaSMhhBBCCCF6M7vWh4j2cPZ2bvaua1NCJ4Zyy+5bcO+rlOqpqK0gvURJAsYFxAEw++vZ/H7sd7675DsuHXKpcV+faB9u2naTGWcvhBBCCGsZcN4ABpw3wNrTEN2YXqdn/zf7CRwWiP+Qtvfs0+v1lGWVdagfnOieJjwwAVd/V2tPQ3Rjy29ZTuT0SGIvj23yXEB8AOd9eB5hk8Oa3d/JywnvKG9izo3BP9YftyC3Ts3HLaBz+wshhBBCCCHaTlb8mVl5bjmJqxOpKqpq03gHVwcChwUaT9xd7F0oe7SMA7cfwMfZB4AQd+Uu7T1ZeywyZyGEEELYlvLccnZ9vIvM3ZmtDxY9Rll2GT9f9TPb32lfmfdPJ33Kp5M/7ZLyjqJrTHpoEsOvG96hfbe+sZVfb/rVzDMS3UlJWgn7v9zPiZUnTD7v6ufKiBtH0GdAnybP6fV6ipKLiDozilt234J3lDcR0yJMjm0PXZ2OghMFUupTCCGEEEKILiCJPzM7+stRvjjrCzJ2ZrRpfEVeBdWljfvzOGgcGOw32Pj9sMBhAOzO2t1k/50f7GTb29s6PmEhRLenVquJi4tDrZaQLURPlvBtAqsXr6auqq7VsaXppSy/cXm7+gqbIvHFtjh6ODLvx3kMWzCsXfvFzI5hwPkD0NZoLTMxYVNSN6dy4LsD1JTVWPR1JL5Yj0eIB4vSFnHGk2e0e9/MnZm8Hvk6m1/abNaSsPnH8nmz35tse0vOXUXnSXwRQliKxBchhCVYI6ZIFDOz8CnhzHprFr4xvm0av+L2FTzn8Rza2uYvxAwPUu7m3Z3ZNPG3/Z3tbHl1S8cmK4SwGTU1lr04J4SwviO/HGHjMxtRaVq/0OoT48MVv13B0GuGdvp1Jb7YDgdXBwZfPJjg0cHt2m/yo5M5++Wzm/T5ErapPKecTyd/yvZ361d+7tkD//0v5OS0vv+sN2bxYN6DOLi13id8+zvbeTf+3UarRduzclTii/U4ezsb20mY8uPlP/JmzJtNtts52zH0mqGETw0363w8Qz0Zf/94wqeY97ii95L4IoSwFIkvQoieQBJ/ZubT34cxd4zBM8yzTeOjzopi1O2j0NhrALh9xe1c98t1HMw9aBwTHxCPWqUmuzybrLKsRvtf/PXFXL3yavO9ASFEt6PT6Thy5Ag6nc7aUxFCWNCst2Zxx+E7jJ8JWuLg6kD/2f3xCvfq1GtKfLEtUqpTANSU15B/LJ+K3Arjtt9+g507YfXq1vd39XdF49B6nAHIPZhrbGFQnlvOByM/YOWDK9u0r8QX66itqGXnhzvJ3pfd4jj3YHd8+vmg1zWOK/5D/LlgyQXtvsGgNQ5uDsx8cSbRM6PNelzRO0l8EUJYisQXIYQlWCOmyG2/Vjby5pHGr/V6Pd8f+J78ynzuGnOXcbuLvQsDfAdwKO8QuzN3MytmlvE5v8F+XTpfIYQQQliGi68LLr4u7dqnrrpOVnH1In/c9QcHfzjI7Qdvb/fvyupHV5P2bxrXrrnWQrMTXcU70pv7s+5vtK2wUHk8frz1/fU6PSnrU7BztiNkbIjJ51VqZeXx9CenM/PlmahUKlz6uFBXXYfaTu4d7c4Kkwr57ebfmPzYZALiA5odd/bLZzfZpqvTyc9XCCGEEEKIHkA+1VvAZ1M/49sLvm33flllWeRX5qNWqRnUZ1Cj55rr86et0VKSXiI9W4QQQggbV5RSRHlueZvHfzr5U94e+LYFZyS6G89wT3wH+OLs7dzufUszSilMLGyxvLywXUVFymNbEn+o4KtZX7HuiXUmn97x3g7eG/oeuQdzcfZ2Nt5coFKpuD3hds569izzTFpYhGeYJ1f+fiVxV8a1e99Vj6zi/eHvU5ZdZoGZwaqHV/HFzC8scmwhhBBCCCFEPblF3AIcPRxxcHNAr9e32BC9urSaHy/7kcGXDGb49cPZn7MfgBifGJztG1/QmTtgLj7OPowLGddo+7+v/MvqR1Zz0/ab6Duqr/nfjBCiW9Bo2laSSwhhu5ZMX4KTpxO37L6lTeODxwYTODyw1c8brZH4YjsmPjCRiQ9M7NC+F3x2gXknI6ymOLWY1M2phI4PxTPME72+PvFXUKD85+PT/P4qlYpz3zm32dYEdVV1VBZUtrl1QUskvnQ9R3dHYmbFtDqu+GQxOz/cSfTMaMInK333NA4aUCnlYC2hJLWE/KP56LQ61Bq5B1l0jsQXIYSlSHwRQvQEkvgzM41Gw+W/Xt6mC3AVuRUk/ZNEwFClBEtCTgIAcQFN7868PPZyLo+9vMn2kHEhjLlrDE5eTp2cuRCiu9JoNMTFtf+ubSGEbRlx0wjsne3bPH7mSzM7/ZoSX4SwPWlb0vjp8p+4+NuL8QzzpKICamrqnz9xouXEH8Dw64Y3+9z4e8czbuE4Y7nPhrS1WjY8vQFHT0fGLxrf4mtIfLEOba0WtZ261fPRysJKNjy1AbWd2pj4O/PpMznjqTM6dTNJSy766iKLHFf0PhJfhBCWIvFFCGEJ1rihQBJ/Ztaeu+69o7xZXLkYXZ3S3NGw4i/WL7bNrxcxLYKIaRHtnqcQwnbo9XpKS0txd3e32IUYIYT1TX5kcpe/psQX21FVXMXqR1cz4LwB9DunX7v3L80oJemfJELGheDTr5WskOjWQseHMu+HeQSPDQbqV/sZHD8Oo0d37jVMJf0ANPYa9i7Zi5O3U6uJP4kv1rHithUc+O4Ai9IW4eTZ/M2hfQb24bb9t+EV6dVou/yshC2Q+CKEsBSJL0IIS9Dr9V3+mlJfw8x0OiWJt/eLvXx/8ffodS3/UFUqFRp7JeO7P1tJ/Jla8QdQXlPOv6n/klGaYcYZCyG6O51OR2JiojG+CCGEwerFq1k6f2mH95f4YjsKEwvZ8c4O0rend2j/7P3Z/Hz1zyT9k2TmmYmu5hHiweBLBuMZqpTiNJX4a822t7fxot+L5B7MNW7T6/V8e8G3bHtrW4v7XrP6Gm7cemOrryHxxTr8hvgRPiUcRw/HFsfZOdrhH+uPg6sDALs+2sXqR1dTU17T4n6dUZpZyr6v9lFwvMBiryF6B4kvQghLkfgihLAEa8QUSfxZSM7+HI79cYzCxMJmx+QezOXE3yeoKa9Br9dTo61BhYo4f9OJvyt+uoIJn0xg6aH6C3yVBZV8f8n3bH9nu9nfgxBCCCG6RlVRFV/M/IId7+9o1355B/M4uekk2hqthWYmuovAoYHcm34vo2/r2FKuoBFBXLr00g6tFhTdW+Gp0w0HJX/TpsSfi68LfQb2QVtbHzvKMstI3ZxK3uG8Fvf1jvI23rgoup/xi8Zz5Yor27RKoaq4itxDSvL3wPcH2PbWNuycLFcUKGd/Dj9f9TOJqxMt9hpCCCGEEEIIKfVpMZMensS0J6a12Ktn96e7+felf7nr+F34RPuw77Z9lNeU42zvbHJ8rH8sy48u50DOAeM2jaOGQ0sP4eLnYu63IIQQQoguUl1aTfrWdAKHBbZrvwu/uBB7V3spQ9MLqNQq3Pu6d3h/Vz9XBl04yIwzEtay7sl1bH19KzduuRGffj7GFX9xcbBrFxQUKP+11Ocv9vJYYi9v3F7Ava8796bfS01Z6yu+MndnUppRSv/Z/TvxToS1Lb1yKYmrEllcuZj5v8+nKKUItcZy9wYHjQji0p8upe+ovhZ7DSGEEEIIIYQk/izG2cd08q6h2Mtj8enng0ewh3Gbq4Nr8+P9lZPzhNwE4zYHVwf+r+b/UNvJ4k0hejInp+Z7tAghbJ9nqCcPFz/c7v0c3Bw6/doSX2xDwYkCVCoVXhFezfZfawudVmfRC/vC8twC3ZQSjaf+/RtW/AUGQkgIpKbCiRMtJ/6ao7HX4Ozd+nnMz1f9TFVxFYtSF7V444HEl65VXVrNn3f/Sf+5/duU6I+9Ipa+Y/qirdVi52iHT7Rl+3+69HFh0EVyA4IwD4kvQghLkfgihOgJ5KzfzDSa+rI3JWkl/Pvqv5RmlJoc23dkX0bdMqrN5VSMib+chEYNISXpJ0TPptFoGDhwYKP4IoQQBslrk9n65tYO7SvxxXasenAVb8a8iU7b8d4A7wx5h08nf2rGWQlrGHnTSBasXYBboBtQ3+PPywtiYpSvWyv3qdPq2PDsBvZ+sRcAvU7Pphc3kbY1rU1zOPPZMzn/0/OhhXbmEl+6XklqCXs+20PmzkzjtuJiaK6lSPxV8Uz7zzTyj+aTtiWtUelXS9LrWvjFEaINJL4IISxF4osQwhKsEVMkY2RmDRs1pv6byt/3/s3hXw63ut/zG59nyqdT+HLfl82OGeA7AI1KQ1FVERmlGcbtOQdyOLT0UOcmLoTotnQ6Hfn5+dJcWogerDSjlCO/HqE00/TNQi3Z+vpW/lr0F9Ul1e3eV+KL7Rhy2RCmPD6lU73VQiaE0He0lNjraRom/vqdauHYWuJPrVGz8dmN7F2iJP6KkotY9eAqEr5JaHnHUwbMHUD0jOgWV59KfOl6fQb14aHChxh/33gAkpLg6qvhrbda3m/LK1v4ePzHVBZUWnyO7w9/n4/GfmTx1xE9m8QXIYSlSHwRQliCNWKKJP7MrOFKvJhZMVz111UMmTfE5Ng3ot9g2YJlAOzM3MmGkxvIq8hr9tiOdo7E+Cq38Sbk1J+Ub3puE99f/H2b+nEIIWyPXq8nNTW1UXwRQvQsJzee5Nvzv+XkxpPt3nfK41O4ecfNHSr7KfHFdgy5dAjT/jOtU8eY++FcZr0+yzwTElaz+9PdrH96vfF7Q6lPb++2J/4Art90PRd/fTGg9Pe7buN1jLxlZLvm0lLskPjS9VQqFU5eTsZyrcePg14PBw6YHl9wooDPz/octb2ama/MxC3AzeJzDB4XLDcgiE6T+CKEsBSJL0IIS7BGTJHEnwU5uDkQPTMalz4uTZ7TaXV4hnvi4qc8l1iYCECUd1SLx2xY7tNgxM0juPibi1FpOt7vRQghRPeg1+vlJKMXCh4TzAVLLiB4THC79w0aHkTgsMBO9X0TQtiO/V/u59+X/jV+33DFX2QkqFRQUKD815KAuABc/ZX+4nZOdoRNDMNvkF+b5pC9P5vnvJ5j43MbO/AOhKUUHC8ge382ujrljuLiYmV7To7pcp92jnakb0vHM9yT8YvGd8kc57w7h9nvzO6S1xJCCCGEEKK3altzOdFher2ekrQS3IPcG/XiU2vUXPvPtcbv25r4uyb+GiaFTuKsqLOM28Inh5t51kIIIbpKxo4MTvx9gsmPTuave/9ix7s7uD/nfhzdHa09NdGFvCK88Irw6vD+tZW1lGWW4R3lbb5JiW4j/2g+P17+I+PvG0/8/PgOHydpTRIHfzzI5Ecm4xHiYcYZiq504RcXGkv76vX1iT9vb3ByguBgSEuDkyfBx6f549RV1VGSVoJHqAeV+ZW4Brii1rTtvlBXP1eCRgTh3te9k+9GmNPG5zay++PdPJj/IM4+zsbEX12dkgju06fxePdgdx7IfQCNg/QxEkIIIYQQoieRFX8Wtu5/63gt7DVyD+Y2O6awspDCKqVGT6RXZIvHO2/Aedwz7h6G+JsuHyqE6Jnc3eXCWk+16uFVbHp+E7o6HV4RXkRMi6Cuqs7a0xI25vMzPufjCR93aF+JL91fZUEllfmVnY4NOQk57HhnBwUnWlkKJro1977u9BmoZHAqKqC2Vtnu5aU89j1VRTEzs+XjbHpxE2/GvEneoTw+m/oZH4z8oM1zcAt049p/rmXYtcNanqvEly41+JLBTHtiGk7eTkD9ij+ArKym41UqFZtf3MxL/i+Rd7j5lhMtqaiARx6Bt99u2/hjvx/j97t+75J+gqJnk/gihLAUiS9CiJ5AVvyZmUbT+G7J8CnhjL5jNHbOjf9XZ+7O5MivR4ifH0+SSxIAAa4BuDq4tvs184/ls2T6EkbdNoopi6d0fPJCiG5Jo9EQHR1t7WkIC7noq4s4ufEkajs1Y+8ey9i7x1p7SsIK1jy+hr1L9nLDvzd0aAVN3FVxlGeXo9fp21XyU+KLbQgZF8LClIWNtun18OefEBoKsbFtO87Qa4YyZN4QY6l5YZtK0ktwcHXAycvJuNrPxQUcTrX5DApSHltL/IVPCWfCAxNw9HRk0MWDcPQ070pziS9dr985/eh3Tj/j96cn/kzFiurSanRaXYdXnf/wAyQkKP/dcIOy6rQlaVvS2P7WdkbfNhpnH+cOvaYQEl+EEJYi8UUIYQmn54y6giT+zEx3WvOEyOmRRE5vuoovdXMq6/67jvDJ4SQGKGU+o33a9odlX/Y+9mXv4+zos/Fz9cPF1wX3vu5y4iRED6XT6cjJycHf3x+1WhZq9zRuAW4MvniwtachrMzJywlXf1fsXew7tP+YO8Z0aD+JL7Zr1y545x3w94eP27jY08nTCTwtOy9hee8Pex+/IX4sWLuAQqVoCN4NqvwaEn8ZGS0fJ2JqBBFTIwA467mzWh5sws4PdlKYVMhZz5reV+KL9bW24g8g/qp4os6Kws6p/ZcGsrJg2bL675OTYeDAlvcZt3Aco24bhatf+294FcJA4osQwlIkvgghLOH0nFFXkAhmZnq9vk3jhl4zlFv33krf0X2p0dYQ6hFKP59+re8IXPPzNVz989X8m/YvAM4+zty07SZG3za6w/MWQnRfer2erKysNscXYRtyDuSw+9PdaGu0xm2FiYWsfGglJzeetOLMhDWMv3c8N22/CSevVpZKmJnEF9tw/K/j7FmyB72u/ue0bp3ymJur9O9qC51WR/6xfIqSi8w/SdFlRtw0gsHzlBtGDCv+DGU+oT7x11yix1yO/HKELa9uaTZ+SHzpWtUl1bw96G02vbjJuK1h4i872/R+AXEBRM/o2MqGTz9tHH+Sk1vfx9nHGfcgd9R2cilCdJzEFyGEpUh8EUJYgjViinza7gJrn1jLl+d82Wibo7sjAfEBOLo7Mj9+PicXneSz8z9r0/Fi/ZUaLQk5CeaeqhBCiC7y78v/8uv1v5K9v/5KXHluOZtf2Ezq5lQrzkzYotKMUr469yu2vb3N2lMRFrD9re38dvNvcKqKa00NbNmifK3XQ0EbW/bVVdbxVv+3WPvftRaZp+gaZz5zpnGVr2HFn6nEX2am8vvRHJ1Wxzdzv+EJ1RP8MO8Haspq2jWPOR/MYVHqonbtIyynqqgKVKCrVe4m1uuhpKT+eXMnghMSYPNmUKlg1ChlW1JS6/vVVdWRfyyfivwK805ICCGEEEIIYSSlPrtAUWIROQk51JTX4OCqNN8oSSvB0cMRR4/6XhoqVdt68sT5xwGNE38HfzxI+rZ0Zrwww4wzF0IIYSlnv3o2UWdF0XdkX+O2wKGB3Hn0zg71eBO2bdfHu9BWaxl9e8dW7zt6OJK8Jhn/WH8zz0x0B2c8fQaj7xht/Ky4YwdUVtY/n5OjlPxsjYObA5MenUTfUX1bHyxsgmHFn6eXlq1pOyiqKqKkqpwi90C8SidQUAC+vqb3VWvUZOxQ6oEm/ZOEvWv7Sg17BHt0YubC3DzDPLnj4B3G76uqlJsEDMyd+PvsM+XxnHNgyBAlLrUl8Ze2JY0l05cw681ZjLmzY2WqhRBCCCGEEC2TxJ/ZNU3ezf14bpNSJl/P+ZrqkmruSbyn3a9gasXf0d+OsnfJXqY8NqVRMlEIYftUKhU+Pj5tvjlA2AYnTyfiroxrtM3OyQ7fmGau0Ioebftb26kuqe5w4s/BzYFHyx9FpW5fnJD4YhsC4gMIiA8wfr9+fePn8/Lafqwznz7TTLMS1lCWXcavN/zKkMuGMPTqocYVf3/UPsLtH79YP3AAjDr2C5mZc5tN/AHcm34vKpUKbY223XGgrqqOouQiXP1dTfYal/hiXYYyn2o16HRKkriqCpzMUFG6oACOHFFW+115Zf1rJScrKw1b+pF7R3sz6ZFJBI0I6vxERK8l8UUIYSkSX4QQlmCNmCKlPs2srKzp/1JT/QtiL49l+A3DqdPVEfl6JFM/m0pxVXGTcaYYEn9H8o9Qq60FYPr/pnNP8j04uDl0YvZCiO5IrVYTFhYmjaV7kIr8CgqTChv19zMozSylMKnQCrMS1jTvh3lctuyyTh2jvUk/kPhiK3R19Y3AKyth+3bl64gI5TE3t+vnJKyjsqCSpNVJFJ5Q/k4YVvz1cfMm0C0QgEF9BgGwP+JmjpzMb/F4hhNQjYOm3XM5/tdx3h70NkdXHDX5vMSXrpW0JomdH+ykurQaqE/G+fqCm5vydU6OeV5r/37lMSpKKTMbHAx2dkp8ai0eeYZ6cuYzZxI6IdQ8kxG9ksQXIYSlSHwRQliCNWKKRDEzy8/XNdmm1+k5tPQQR5YfMW6b9PAkpiyeQmpxKslFyWxN24q7Y8ul3Wpq4NgxCPUIw83BjRptDScKTwBKaRevcK8OXfQTQnRvOp2OkydPotM1jS/CNu35bA9vRL1Bxs6MJs99NPYjfrz0RyvMSliTTz8fAuICWh/Ygrwjeez8cKfS56mNJL50f3qdnqecnmLp/KUAbN2qfCYMDoaxY5Ux7Vnxt+GZDXww8oNGyURhO/wG+bG4cjFTH58K1Cf+bh3yCJn3ZaL/j55dt+wiSDOYavtsXmxQ+tGUw8sO8+XZX1Jwoo2NIhsIiAtg0qOT8B9ius6sxJeutf/r/fx2y2/GHn+GxJ+nJwSc+vNirnKfe/cqj/HxyqOdHYSFKV+3pdynEJ0l8UUIYSkSX4QQlmCNmCKJPzMrL9c33aiC3275jY3PbGzyVGJhIgCR3pGoVc3/OMrK4P774d57YfduFRFeEQCcLD4JgE6rozi1mPKc8s6/CSFEt6LX6ykoKECvNxFfhE3qO6ov4+8fj1eEV5Pnxtw5hvhr4rt+UsKqynPKqauq69Qxjv52lN9u/o3s/dlt3kfiS/enrdEy+OLBBI1UyuIZynxOngx+fsrX7VnxV1lQSWVBJVXFbU8Qi+7HcLOfodSnl1f9c052TtwX/TkqvYbD5ZvJKW9+mVfK+hRO/H2CktSSds/BO8qbM58+s9mSjRJfutbkRycz/4/5OHkrtTwbJv4ClcWgZkv8GVb8xcdDRW0Fse/E8o//BUDriT+9Xs9X537FqodXmWcyoleS+CKEsBSJL0IIS7BGTJEef2ZWZeIaikql4sIvLsQtSKmxknswl7/u/Ysxd44hMUhJ/EV5RxnH6/Vw+LByAh8UpJRM+e9/60+iEhPh2TOfRa/XMyJohHLMA7m8N/Q9pjw+helPTLfkWxRCCNFJEVMjiJgaYfK5iQ9O7NrJCKvT1mh5KeAlYq+I5eKvLzZu1+v1FFcXk1WWRWVtJfEB8WjUzZfjG3jBQPoM7NPplYOie7FzsuOS7y4BlATfrl3K9ilT6hN+7Un8zXxpJjNfmmnmWYquUnyymNyDufQd3RdnHxeKikCnqsHVA6C+5P+kqJGM/P0nRvlOw9/Vs9njnfXcWQy9diiBQwMtPndhWd6R3nhHehu/b5j48z612RyJv9xc5ThqNQwZAv8kreFA7gHgAMHqEpKSPFrcX6VSkZOQg6O79KUXQgghhBDCUiTxZ2aVlaa39zunn/Hr0sxSUtanMOSyISQ6nUr8edUn/pYtg08+Ub6OiVEejx2rP1ZeHtzWf06j43tFeDHm7jGEjpdeCUIIIYQt0Wl1jLp9FH1H9TVuSypMYtKnk8gorS8H+8JZL/DAxAeaPY5PtA8+0T4Wnauwrp9/Bq1WWWUT2uAjn/T46z2O/XGMFbeu4Kq/ryJgfDR1dZDruZLQ989jZvRM/rzqTwD69oXAovMprFVuKmyul7zGQWNM+u3cCatWKeca+fkQHg533AF9+jQ/n5+u/Ak7RzvO//R8c79V0U7VpdU4uDkY+zaaWvGX3fYF4c3at095jIkBZ2eMrScAKhwTSU4e1uoxFp1c1PmJCCGEEEIIIZolpT7NrKqq+R57lYWV/HrTr3gEe/Bo+aMMvWYoiUVK4i/aJxpQ+rZ8+qkyXqVSEn7HjiknVeeco2w3dXHH0cORWa/PapRgFEL0DCqVisDAQOOFHGH7/rj7D3664ieTz+1ZsofPpn1GSXr7y64J22TvbM/st2cz/Lrhxm2rk1Ybk35OdkrZtiV7l7R6LL1eT015TZtfW+JL95e5O5MVt6/g6LoM/vpL2TZvnvJoSMiUlzd/85nBJ5/Au+9CWU45ez/f266SsKL7iJgawXkfnod/rL+xv1+deyJ69LjYuxjHBQQo5xLl5VBa2rbSMu+8Axs3KpVHcnNhxw5YuLA+0WNKUXIRRSlFJp+T+NJ19Ho9L/i8wPcXf2/cZqlSn4Yyn3FxyuPeLKXh332jH8WzchiZmaar4AhhThJfhBCWIvFFCGEJ1ogpkvgzs+pq0/9Ly3PL+XLml+z5ZA8aBw0qlQq1Rs2JAuUOySjvKBIT4aWXlLtyZ82CL76AW29Verg8+SSMHascKz8fssqy+DbhW348+GNXvTUhhJWo1WoCAwNRqyVk9xR5h/LI2mv66ltZVhk5CTlUFrRyFV/0aIak3w3DbyDzvkwcNY54OnlSUt1yQvjD0R/y0diP2vw6El+6v9wDuex4dwf//FxCTY2yymboUOU5Z2dwdVW+zstr/hhpacpqwd9/hxM7i1l27TKO/nbU8pMXZtdnYB9G3DgC9yD3+sSfW9PWAQ4O4OsLh4Ifpv97IXy0q+W4UFdXf3PhokXKuUdkpJI8euwxWLnS9H43bL6Ba/+51uRzEl+6jrZaS/zV8URMjzBuKzn158LTU0kEg5L460x7Eb0e9ip5PuJPtSPel6NkhsdHjMDLSxmTktLycdK2pJHwbULHJyJ6PYkvQghLkfgihLAEa8QUKfVpZuXlOpPbXf1cuWn7TdRW1mLnVP+/Pcg9iJDyEMLco3j6/5S7I4cNg5tvBjs7mD1b+Q8gOVl5zM2Ffdn7uOKnK4jzj+OSwUrfl3VPriP5n2Su+ecauTNFiB5Eq9WSnJxMREQEGk3z/b2E7bh65dXNPjfpoUlMemhSF85GWFvOgRzWPbGOETeNIHqGUgEg0C2QiaETiQ+Ix8vJi4z7MvBxbr2MZ+SZkdSW17b5tSW+dH+xl8cSNDGKux9Q+mHNm9e4bKOfn7KqKyencfnPhv79t/7rQo0vly27jIB46QVp6woLlccqZ6UReMPEHyi9wrVFFWRXpnOs4NjpuzeSn68kbOztYfp05XfspZeUVYCrV8OSJXDWWc2XDDVF4kvXsXOy4/xPGpdbbbjiz89P+dnV1EBRUX3Pv/bKylJuMrCzg0GDoE5XR0KOksCLD4hnbyTs3q2ctw4Y0PxxNr+4mUM/H2LIpUNQqeW8VbSfxBchhKVIfBFCWIJWq+3y15TEn5lVVLT8vL2zfaPvl1+xHIDUVOWCjZMTPPSQcjJ1OkM5p9JS8HcKASCtJM34fHFKMXlH8qgtr8XBzaHjb0II0e2UlpZaewpCCAspSSvh4A8HiT472rjt5pE3c/PIm43ftyXpBzDj+Rntfn2JL92b2k7N+l1ulFUrib1x4xo/7+enXGRvqc9fw8RfarYj064eaJG5Csv74+4/OP7HcW5LuI2iIuWEodQuEfSmE3+u2UobgIZ92Ewx/P4YEkSgrBq8807YvFlJIh0/Xt9/3CD/aD4nN52k/+z+uPq7NjmuxBfrMawI9fBQzi39/JTzzezsjif+DGU++/dXzlsra2t5avpTrElew3/W/ofdLln04x/jDavNGXfvOOKvjkev16NCEn+iYyS+CCEsReKLEKInkHXLZtZaf5Xm5OQoj4GB4OZmeoyrq3KCBeBcqyT+CqsKKa8pB+C8D8/jvoz7JOknhBDdmLZWy/5v9pO9z3R/reqSag4tPdRsKVDR8/Q7ux//V/d/DLt2WKtjCyoLKKwstPykRLdRmFjI5l/zQK/nwgubrrjy81Memyv1mZen9Is2MFyQL8sqQ6/rRM0/YRUObg44uDugcdBQUAB69BSilPqM9IpsNDYoCFyqlRsKjhccb/G4hnMRw++TgZ0dDD/VfnT79qb7nVh5gl+v/1V6RlpZyoYUfrnuF+NnC72+vtSnl5fyaI4+f4Z+j4Yyn872ztw34T6+u+Q7vk34lsPVa6i2y2k18Rc2MYyBFwxErZHLEUIIIYQQQliCfNI2s4KKjl2MM5xs+/s3P0alql/1V13igbuDOwDppemnnpe7JYUQorurzK9k6ZVL2fnBTpPPl+eU8/3F37P/6/1dPDNhTWqNGrVd/ccynb5p6fDFqxcT8FIAH+z8oNnjFKUU8csNv3Do50MWmafoev8s/gePz99GpdcRFdX0eUOiprkVf1u2KI8uLspjcjJsemETLwe93OwNCKL7OvOZM7ll1y2oVCry8qDGLpdqfTkqVIR7hTcaGxQErlWnVvwVnEDfQnO3ls5FRo1SHnfsaPpczLkxXLH8CgKHBXbo/QjzyN6XzZ7P9hj7A1dXK2U9QSn1CY37/HWEXl+/4i8urvFzrg6uxhWnpc4HOpVcFEIIIYQQQnSeJP7M7GTNgTaPfX/H+4S/Fs7i1YvblPiD+sRfbi6EeDQu91lVVEXCdwmySkSIHkalUhEaGirJ/R7Cwd2BeT/OY9iCYSafdw9258IvL2To1UO7dmLCagqTCklZn0JNmXKVtlZbi/PTzoS9GtZodV+4Vzh1ujp+PPRjs8dSa9Ts+WQPaVvSmh3TkMSX7m/QxYPIGjAVvVpjvIBvoNfrjZ8Nm1vxZyjzOWeO8piTA32GhTD0mqGo7eVUwJbl5YFeVcfZgddw/sDzcbJzavR8UBA410SAXkV5bTnZ5c0netuS+Dt2rL6voIF3pDf95/THxdelyX4SX7rO6NtH83DJw4ROUBp9Gsp8OjiAo9IeFF9fGj3XXhkZUFCg9IEceKpa8LrkdRzIOUCdro5Y/1gASp0TyM+Hurrmj7X387087/08yWuTOzYZ0etJfBFCWIrEFyGEJVgjpsjZvpll1bb9DvsThSc4WXyS8tryRn01WmK4uJOf3zTxV5pRyk+X/8TBHw62e95CiO5LrVbj6+uLWi0huydwcHVg8MWD6Tuqr8nn7Z3tiZ8fj39sK3eCiB5j3xf7+GzqZxQmKlfUs8qyqNHWkFWWhadTfabngoEXoFap2ZGxg+SiZJPH8gjx4N6Me9vc60/iS/cXPmswaTHTAKVX1+U/Xo7rM67YP2mP5n8aXkm6Bj1akyv+SkshIUH5esaM+s+RdcHhXLDkAvyHSJyxNVvf2Mr+b5RlV3l54FTbl1enLOHny35uMjYoCDR6R5xrwgBl1V9zWjoX8faGfsrCQXbtMr2/Ttt0lbLEl66jUqlwdHdE46ABlJ6MoKz2M1xjMCQAq6o69hp79yqPgwYpCUWAq3++mth3Y/k39V9j4q/CNQG9vuW+o67+rgSNCMLOyURjeyHaQOKLEMJSJL4IISzBGjFFopiZZevannQzXLQL9ww3y4o/r0gvLvn+EuLmxzW3uxDCBmm1Wg4fPoxWq7X2VIQQFtBvVj/Oef0cPEI9gPoS3kHuQahV9R/V/F39mRI+BYBlh5c1ezz3IPc2v7bEl+7PsDrHxQXs7fUsnryYK2KvoE5Xhx49y09+wZG+/1FWf51WyXH7dtDpICJC6e8VeaoFXFJSV74DYU7r/reOne/vRK+vX+VpWMl1Omdn8PEB77IJjPGf2uJdpq3dhGhY9Xd6nz9dnY7nvJ5j6fylTfaR+NJ1MndnNirda+jv13CVsKFXfHV1x17j9DKfhZWFpJakAhAfEG9M/FW6K3cbtFTus985/bhm9TWEjAvp2GREryfxxTJ2vLeD3277jaO/HbX2VISwGokvQghLsEZMkcSfmUUV3tzmsSnFKYBSuqu9K/7y8uC2Ubex/IrlXBV/FaCsEhkybwh+g1o5iBDC5lR19PZs0e3s+WwPL/q92GJ5q69mfcU7Q97pukkJqwoeHczYu8fi7O0MQEZpBgB93ZuuCj2v/3kArDi2otnjVZdUc3jZYXIPtbDcooGeEF9qymvIO9JMrUsbt+z8Twk+tOrUyh0VcQFx/Hfaf0lblMan538KwPG+T5Pm8ptxlY+Bob/fuHHKoyHxl5wMJ1ae4MMxH5K+Lb1r3ogwi2vXXMvsd2ZTUqKUUqyxz8XNs6bZ8cHBMCLpa54buJYJoRNMjmm4Oqu5mxBHj1Yed+9uXMJRbacmcnokAfEBJvfrCfHFFvx+++98Nesr4/eGGwYaJv6clT8xVFa2//gN+/vFxyuP+7L3AcpNrJ5OnsbEX6FDAnr0ZEsLUWFhEl/M78B3B9j53k6+u/A7ynPLrT0dIaxG4osQoieQxJ+ZOZUOavPYlCIl8RfiFkF+vrKtrSv+8vJgdPBo5vSfQ5hnWEemKoQQwgqcvJzwG+yHs49zs2N8Ynyk1Gcvll6iJGKC3YObPDenv9KobV3yOkqrS03un380n+8u/I79X++33CS7meU3LeftgW9TkV9h7amYla5OR3lmCXY1FXh51W8P8Qgh2COYBcMWcNeYuwDYE3k1O5MSG+2/LmUtR/r+l6HDlLsLG67409hrKDxRSHHqadlC0a0FxAXgN9jPeO6wt/+luD/vzE8HfzI5PvhUGElroe1ncTHU1CglIQ3nGqeLiVGSSBUVcPC0AieX/XwZkx+d3M53Isxp7MKxTP3PVOP3hpsAPDzqxxhKfXZkxd/Jk6eO6ZJLTIyytNiQ+BsaqPQk7u/bHyc7J3zsgqnVFLaY+KssqGTN42s4ukJWFQnRncz/cz5zP57L7PdmY+9ib+3pCCGEEKITJPFnZhUVqiZllkyprK0ku1w5G3LXhaPXg50djS7qmGJYEZjXzE3tH475kA/HfNj2CQshhOhSAy8YyIJ1C5pdHQEw641ZXPLdJV04K2FNv974Kx+Orv/b3dKKv/6+/YnxiaFWV8vKxJUmjxc4LJA5789h2IJhFplvd+Ti54KTlxOOHo7WnopZqe3UxH10DylD5+LpCZ/s/oRX/321Ua+2l2a+RJB2HBqdG0cy6rM7J7Iz+bvvdI71fYJU9QZAKfkJkJICIRPDeCD3AQZfPLgr35LoBJ1WR2VhJTqtznguUOGYiE6vI8g9yOQ+IacqKaanQ6221uQYw2o/b2/lfMQUlQpGjlS+3rq1o+9AWErsZbGMvHmk8XtDqc+G55aGUp8dWcSwT8nxsXZQPH1e9uJAzgH2ZitN/+L9lSWADhoHSh4u4d3YgzhofVos9VlXXcf6J9dz/M/j7Z+MEMJi7BztGH79cEbcMAIHVwdrT0cIIYQQnSCJPzNL7fMxu9ISWh13svgkAG4ObtSV+ABKUq+F1htAfQ+PsjLILynnm/3f8MbWN4zPBw4LJGBo8xeThRC2R61WExUVJc2lheipVKDS1H8AMPT4M5X4A3h40sO8O/vdZsv2qe3UjLx5JD7RPq2+dE+JL7Nen8VDhQ+hsddYeypmZ1i54+UF72x/h3v/vtd4wR2Ui+3XuyxjWsJBguumGLcv/ON+AHyrRnP2QGV7377g4KCs+MnJVaNSt/LBU3QrBccLeMHnBdb+Zy15eaBT1VCmVpK9Ud5RJvcJDoZyxxM8XRFG8CtNVxEDbe41PmmS8rh6dePkUcJ3Cay4fQV11XWNxveU+GKLTK34MyT+OlLqc98+qLHLo0yVRUl1CWGeYXy8+2OgfsUfgL3GnsBA5euWVvy5+rty+4Hbmf7E9PZPRggkvlhCVXEV6dvTqS5VlgVX5FVQWdCBgCGEjZP4IoSwBGvEFIliZnYw9D6WH/691XE12homhk5kXMg4cnOViy6tnWwDuLrW92fIyK3myqVXcs+f91BVp5x9n/fBecz9cG6H5y+E6H5UKhUeHh6oWrszQNiE3Z/uZv3T61sck7YljZUPrqQouahrJiWsau6Hc7lxy43G7/v79mdi6EQG9hlocvz1w6/n1lG3EugW2OJxaytrqSlrvvcX9Jz4UlNWw5bXt3Ds92PWnopZlaSVkPnHHhzKC3H30HIg9wAA8QHxjcZF+Qdgp3M3rgJbk7SG31K+Br2K8zXvolYpH/nVaggPV8YkJUHxyWLWP7WerD0tLM0R3Ya9iz3DbxxO8Jhg8vOh0uEkenQ42zkT4Gr6xr+QEHCsDaBMk0puRS5FVUVNxrQ18TdqFAQFQXk5/PNP/faUdSnseHcH1cWNa0j2lPjS3RWlFPHWwLfY/s524zZD4q9hjz9D4q+9pT71ekhIgFInJf5EekUaf49UqBjVd1Sj8QGnfhVbWvGn1qhbLXsuREskvphf6uZUPhrzEQnfJJCxI4OXg15uFFeE6C0kvgghLMEaMUUSfxaQkNP6ir+4gDg2Xr+RlVevbPPJtoGh90ZdiTfOdsrJkqEfkBCi59Fqtezfvx+tVmvtqQgz2Pf5Pra8sqXFMdn7stn84mbyjjRT11n0aI9NeYyN12/kgoEXdPgYqZtTec7zOXZ/srvFcT0hvuh1enZ+sJO/Fv7V4/oaZuzIoOTzX3ArTKXc6ThVdVW42Ls0Wd1lKAWfmann3e3vcsbnZwAQnnsbY0JHNhprKPeZnAylGaWs+b81HP7lsIXfiTAHz1BP5n44lwFzB5CXp5T5BIj0jmz2RNLfH5w1bjjWKtmYhmViDQylPls7F1Gp4LzzlK9//RVje4MznjqD+3Pux6WPS6PxPSG+2ILa8lrUGjU0+BVouFLYoKOlPpOSlGozVZ5KfI31jyXUM5RdN+/iz6v+JMIrwjh2d+ZuLv17HJsHTKO0VOkJ2ZyK/ArpMSo6TOKL+fn08+GMp88gbFIYAUMDGHjhQAKHtXyTmRA9kcQXIYQlWCOmSOLPAg7lt574a8iQ+DNctGmkphh2LoSqHOMmQ+IvP19FiIfSuCOtRCnzc+z3Y/y56E+qijvQvEEI0W3Jh86e4+JvLua6jde1OGbwvMHcdfwuIqZFdM2khNXodXo2vbiJxNWJ7dovozSDd7a/w1f7vjL5vN9gP6JnROMZ5mny+YZsPb6U55bz931/4x/rz4wXZ1h7OmYVMj6Emosvp9Q3gly10mQr1j/WuILPIDJSedydksjtv98OKCtxwnJv4ovqeZz3zXlNxiYlQd9RfVmwfgGTH5ls+TcjzCovDyockoDmy3yCssozKAhcqvoBcKKwaeKvxXOR05x1Fri4KD0Dd+1Stjn7OOPq52qydKytx5eucujnQ3w84WNjib328Bvsx+0Hbmf0baON21oq9dnexN/+U/dTqIOUc9xY/1gAhgcNZ2b0zEZj3Rzc2J65lWLXrejRtlju85OJn/DVLNN/w4RoC4kv5uUb48vkRyfjN9gPjb2Ged/Po/+c/taelhBWIfFFCNETSOLPAo4XH0Sra/mPhN5wiyyt3GWb+AkceR0OPGPcZEj85eXRJPF3ctNJtr62lbKsso6/ASGEEBbjFuiG36CWr646ezvjE+2DnaNdF81KWEtVURWrHlxFwrfKBVWdXodOr2t1v79P/M0dv9/BK1teMfm8k5cTV664koEXmC4X2pM4ejgy/8/5XPjFhbgHuVt7OmblFuBGof8Aap09SK9Vrr7H+cc1Gdevn7IaqyYrmv9OeBGNSsP0gm9xrAtgU+GP/H7sd2q0StlXw4q/lBSlH2T45HA0Dj2vN2JPdGLlCZbOX0rOgRzy8qDaXqmlGOxuunefQUgIuFZHA3C84HiT5w3nIm1J/Dk7w8xTuZ5fflEeq0uryTmQIzcedpBOq2Pp/KWUppeapU+pXt9yqc+6OuW/tjIk70oclb9TpmKQQZR3FE52TmjVVZQ7JrZY7nP49cOJvzq++QFCCCGEEEKIDpPEn5mpdE7U6KpN3k3b0PQl0wl/LZx/kv5p+WQ75na4IB2Gv2jcZBhnKvE3buE4FqYsxCfap9PvRQghhPkVJha2eke/XqenJK2E0szSLpqVsBYHNwdu2HIDE+6fACgX5R2fcmTw24Nb3G9Wv1moULErc5fxM0BvZe9sT7+z+xEQH0BxajHa2p5zh65er6eoSPk6uVJZ8WfqoruTU/1KvrNc7qfkwSpcky7FsTYQFzsXdHodKUUpAASeqtqVl6ckCPQ6PVl7s6SnqA3IPZjL/q/3U1lYRV4euFfGMa//NUwIndDifsHB4FKtrPg7mHuwyfPtbTswZ46SaN69G1JT4fCyw7wb+y4p61La9X6EQq1Rc0/iPVz4xYXYObX/hp+U9Slsf3e7MfFaXQ01p9q7mkr8Gca0VUUF6NGTXtt4xZ8pGrWGwX7K369S54QWV/xNfHAikx6a1PaJCCEs6suzv+Tnq382fp+9L5tPJ3/a48qoCyGEEL2FJP7MzL1qEAD7s1v+cHSi8AQni0/iYufa8oo/jSO49AW1vXFTwxV/oR6hQH3iz9XPFc8wT9R28qMVoqdQq9UMGDAAtVr+Xdu62spa3oh+gxW3rWhxXFVxFa+Gvsqa/1vTRTMT1qJx0BAyNoQ+A5Q/7ukl6dTp6lpd9RfgFsCkMOWC6df7vzY5Jnt/Nl/P/pqjK442e5yeEF90dTr0ej1rn1jLa2GvkXe45/TG/PWm5fT74Vk0tVUcK1YuuscHmF4hM/DU4s4jRyA70w69HtzdVET7NF7p5e2tJG3q6qCkBLL2ZvH+sPfZ8f4Oy78h0Snj7hnH4srFeA0JoaYGgoou4suLl3DN0Gta3C84uH7F3+G8xv0cKyuV/m3QthV/AAEBMHy48vXevRA4LJApj0/Bp1/jGw97QnzpKm6BbgSPDe7QqskD3x/g99t/p7a8FoDCQmW7vYMetX2NcZydnfIfKD/3tiovB52qhnMDbmF2zGwG9BnQ4nhDYrDUOaHFFX9CdIbEF/Orq6qjrrp+ObC9iz25B3OpyGuhWacQPZDEFyGEJVgjpkgUMzND4i8hp/k+fzXaGtJL0gHwVodTU6NcgDEk9IxSl8HB56EoQfn6VPlQw7jc3AYr/kqVxF9ddR2FSYVUFrTjbE4I0e05ODhYewrCDPQ6PRMemEDMuTEtjnPycmLM3WOInhndRTMT1lJXXUddVf1FlozSDAD6OPZl8WJY00Lu9+r4qwH4Yt8XjUqIG9RW1HJi5QmKTxa3OAdbjy/r/reOZ92exS3QjTF3jcHexb71nWyEW4QfZV4h6Owd2XvrXrbeuJUxwWNMjh1w6lr84cPKKiyA0FDo56Os9DIk/uzslOQfKDeRBcQHMOnRSQyY2/LFfNE92DnZUVisnMJ5ekJb/vmGhIBv6XQctX0YHzKeGm0N/137XyZ+MpGUjHIAXF2V3n0AVXVVzPthHnesuKPZYxqShGVlEBAXwPQnpuM3uGnm0NbjiyXp9Xp+vvpn9n25j/Kccp52eppVD69q93Em3D+Bq1dejau/K9CgNGf4V4z4YDh7s/ai1+t5afNLaF2Vc8b2rvjT6B25a+Dz/HblbzhoWv6ZxvrVJ/5aWvG3+5PdfHnOl1TkS1JBdIzEF/NasG4B876fZ/zeO9qbB/MfZOzdY604KyGsQ+KLEKInkMSfmYVm38p/w9dy99i7mx2TVpKGHj1Odk6oygMA5QKMnboWEpcodZcAEj+FfY8rPf42XAhliUB94i8/H2bFzGL5Fct57sznlGNvSeONqDfY9+U+y71JIUSX0ul07N+/H52u9b5fontzcHVgxgsziLuy+f44ACqVilmvz2LIpUO6aGbCWg58d4CnnZ/m6G/Kqrz0UuXGoLykvuzbB0uXNr/vvCHzcNQ4kpCTwN7svU2e7zuqL4srFzP6ttHNHqMnxBfvKG/Cp4QTd0Ucs96Y1aPKnUdfNZ5j467Gw1OFu5MrY4LH4OrganKsIfF34gQkJSlfh4Q0TfwB+Poqj3l5SpnBM58+k9DxoRZ7H12pqqiKnR/s5OCPTUta2rqcAzmkbUkjN0c5V3DtU0BVXesrxIKDwak2iLN25/DctNexV9vz2Z7P2Jy6md8P/wM0rjyyePVifjz4Y4slRN3clMeyFtqK94T4YklZe7I4tPQQSauTcOnjQuwVsQSPbrlfoyleEV5EnRVlrPiSmQkVDkls9Lydg7kHWX50OY/98xgPrHyATWHnU6cup6odCwvLldwwrqZDTxMNV/y1lPgrOFHAyY0nqSqS3pCi/SS+WJ5KpbL2FISwCokvQghLsEZMkcSfmXlUxhOhmoq3s3ezY5KLkgEI8wwjN1f5MOXvDyQ8DVsWQMp3ysBJ38OMjRB9I4z9GByVqzSGxF95OQQ4RjCn/xxjyRXvKG/G3z+ewOGBlnh7QgghhDAjj1AP4q6MwyvSC6hf8VdXoFz8TUtTSjKa4uXkxdwBcwH4fO/nTZ5Xa9SoNT3/o96wBcOY/8d8nLycWh9sYwz9/Rr26WpO377g7q709tqwQdkWFtYg8VdYn/hreBOZgV6vb1Tiy1blHcnjt1t+44d5P1BbWWvt6ZjV2sfX8vGEj8k7Vc32Z+/JOD/tzJqklstCu7kpv0MqVGRmqFGpVMyOmQ3AqhSl9LQh8bc2eS2vbnkVUGJMS8cEJfFXmlnKkjOWsO3tbR1/c71Q0PAgHil7hDnvz0GlVnHx1xcz/Prh7T5OdWl1o1XfmZl6dkdeTTWlTAydyCOTHuGmkTfh5+JHgeMuTgQ+365SnxUVUOp0iCp1bpvGx/rHEuYeiVvVALKy9ZhYkA7AGU+dwaNlj/aomzWEsFXp29PZ+sbWJv3FM3ZksPnlzWhrek7/ZCGEEKK36PlXg6ygopVqJSlFSuP7cM9wcnKUbf7+wMBFMPRZCL1I2ahxBN/R0GcsRF8PjspJkbNz/R2Xeae1sfEM9WTmizMJnxxupncjhBDCXJLXJvP17K9J3Zza6titb2zl4wkf94gL8aJ5kdMjueiri/Afolx135esrPhzrO2LnZ2S9EtLa37/q+Ovxl5tT1mN6WU3mbsy2f9Ny32He5JlC5bx56I/rT0Ns9n9wkr6pOzkmM8b3LHiDv5N/bfZsSpV/ao/wyobQ6lPR40jKurv3G/YLxqgLKuM1yNe71CZwe7GN8aXuCvjiJsf1+MuVA6/YThnv3o2BYXKz7JGrTRzaylBZxB8aiFZuhJimN1fSfxtLViBHj1+flBSXcKCZQvQo+emETcZx6xLXseyw8saHa9h4k+tUZO1J4uyzBaW/wmTVCoVGgdNp47xWvhrfDnzS+P3B7KOUei+CXuVI19c+AUatYYIrwhemvkSAFley9pV6rO8HHZHzmfSr/78dvS3VseHeoZy4u5ERicupbZGZew5eDpZTSRE93FsxTH+vOdPynPKG20/+NNBVt6/koITBVaamRBCCCE6ShJ/FnCgdAN3rLiD34/9bvL5lGIl8RfhFUHuqRsnh/hvgKQlyuo+jQOc/AFKjjbeUV+/JNTQm6WwEL4/8D0vbnqR/Ip8RMvq6mD/fqjtWTeACyFsREl6CYmrE6kuaf2KW2lGKYWJhZRnl7c6VvQM5eWwP0lZ8Td1eDD9+yvbk5Ob3+ecfueQcV8GH5z3gcnnNz67kaXzl/boBPKK21ew66NdAKRvSydzZ6aVZ2Qeer2e1B+24JV9lBMOP/POjnc4VnCsxX0GnNamLzQUpoZPpWJxBb9dWX/BvmGpTwDXAFc8wz1xC3Qz51uwCmcfZy766iIu+vIinDx71irQmHNjGHfPOOPPrUKvXIhtqdKIgSHxZ7iRYHrEdJztnCnQplHqvB9/f7jnz3tIKU4h0iuSl2e+DMDm1M3M+GIGD6x8oNHxGib+XP1deajgIc546ozOv8le5PAvh0nZkGL8fv/X+/n2/G/bVfpSr9cz6KJBRM2IMm5LylduLgpxjSLSO9K4fXbMbNCrKHXZz8miFu4oaXR8KKuoo8xZKZ07qM+gNu1nZ1ffB7K5cp8VeRUc/e0oRclFbTqmEMJyRt8+mmv+uQbf/r6Ntg+/bjgL1i3AK9zLOhMTQgghRIdJ4s/MVCrYW7Wcd3a8w3cHvjM5JsgtiImhExkaMNS44m+w6w+w8x6oKwVtNWy8FNbOrt9p7Wz4o770i5eX8lhcDA+ufJAHVz1ovBj0/cXf8/cDf1vi7dm85cvh0UeVRyFshVqtJi4uDrVaQrati58fz2NVjxF9dnSrY8989kzuz7ofz7A21PgTNmvzy5v57dbf0Ov1bNkCbqUjCayZxILzo4iIUMakpDS/v73Gnj4ufZp9fuw9Y7l82eXNrqyw9fhSV1XHjnd3kPSP0tTutn23cd3666w8K/MJe+1eUuLnUKZSlmlFekW2OH7gwPqvHR2VC+8atQa1qvHP9/RSnyqViuvWX8ekhyaZbe7Wom+urmAPkpcHWlUltXrlJhJvp9YTfyEhyqNhxZ+zvTMTgpREXY7n7yR6LOGzPZ+hQsWSC5bg7ugOwBC/Iej0Oo4XHDe2K4C29fiz9fhiaX/c+Qd/LfzL+H3+0XyO/X6M0ozSFvZqTKVSMfejuUx8cCKgJOrSSk8l/jxCGo31dfElhLEAbMxq28romhoo0ZxAp67Gxd6lUSKxNQGBerSqSrKyTD+ftSeLb877huN/Hjc9QIgWSHwxL1d/VyKnR2LvbN9ou29/X8KnhGPvYt/MnkL0PBJfhBCWYI2YYtflr9gLhNecy0b7F/nj2B/o9LomF1tuGXULt4y6BYB7vlK2FYY8TkTEXHCNAF0N9L8bwi6p38k1EjTOytmcSmVM/BUVQV/3vqQUp5BeopzJZ+7K7HGljczl5MnGj0LYipqaGpycetbKhd6sLeWtpARW73D89+Nk7s5kzntz2LgRYlPfYP4UGBMK+RHKmKSkth2rsrYSJzunRr87YZPCWt3PluOLxlHDw8UPG1c0qu16zgm6SqWiVOdKrROU65USEX6ufi3uExOj3ISm1yuJnubCyOkr/nqSPxf+yb4v9tFnYB98+vlw4ecXWntKZvPJxE/oM6gP+Q5zqbVT6ieqVWpjkq4loaHKY0KCkqxzcwPvvNnACspClvHOAeV37IlpTzA5fLJxP08nT8aGjGVz6mZWJ67mhhE3APVtBwyJv2O/HwMVxMyKafS6thxfLG3uJ3PRa+sT1RMfmsiU/5vSqd6spaVQgrKaL7pPaJPnB9rNIk27hS15fwA3tnq88nIodU4AlCTw6ee1zfnp4E+84XgLblGTyM5eZnKMf6w/5396PqETms5TiLaQ+GI+pRmluAW6oVI3/eCg0+qoKqrCxdfFCjMTwjokvggheoKec3Wkm9Drwad8Iu4O7uRW5LIzY2eLYw2lT3yC+kDgWcoVGo0jjHod/OtPuhn9Fkz+0XgFx/PUApCiIgj2UGr3pJcqib97ku7hiuVXmP299QRFRcpjgZSoFzZEp9Nx5MgRdDpd64NFt5a2NY3jfx1Hp239Z1lXXcehpYdIWd/Cci9h8676+yruSbyH8nLYs0fZNlFZuNGmFX8AWp2W8NfCcXnGhZzyHJNjmvuds/X4olKpcPRwxNVPyUKUZZVx4PsDFKUUWXdiZlBTVkPRsVzQllGuKwLAz6XlxJ+ra32CJ7TBtfQXN73IyA9G8tmez4DGiT/DArmsvVksv2U5aVvaVgKwu/KO8iYgPgC9Vk9lfqW1p2NWujodeq2evDyo1dT392tLMmboUOjbV2kT8O67kJsLxdtnY1/nxbDwaDZet5FHJz3Ko5MfbbLvWZFnAbAqqb4H5Okr/v646w/+efSfxvO18fhiadEzoul3Tj/j9/bO9u1O+mXsyODna34mbavy7zYzE3xLpzO04AkuHHx+k/FxTrNQ6xzQ6dp2c1FFBZS47AMg1j+2zfPycfahXJ9PqXMCGRmmx7gFujFswbAmpQWFaAuJL+ZTU17DK8GvsOzaZSaffz3ydb4575uunZQQViTxRQhhCdaIKZL4s4CaSntmRs8EaNLnLyEngaKqIgBKSpS7KJ3sygnySK6/8tIGDVf8Bbsrib+M0mbOqoSRIfHXXJN5IYSwpE3Pb+Lrc79u82q+7y/5nm1vbrPwrIQ1qTVqnLyc2LIFauv0hIbpjQmb8HDlMS9PWcXRHI1agwrld+pE4YlGz5Vll/GC7wv8de9fpna1eaWZpWTtzaK2Umnem749nR8v+9FY+tOWpf6biu7Nd/DKUGKAWqVuUy+3oUOVR0OPSIDMskx2Ze5if/Z+oD7xV1NTn7gpzShl1we7yD2Ya7b3YA3j7hnHgrULuGHLDVy54kprT8esbtx6I2e9dT6VlRhX/LWlzCeAgwPcdx+o1bB+PTz1FDhWhXGfLo/lC74iyD2Ip898Go1a02Tfs6KUxN/qxNXoTvUcNyT+qquVHtqz3pzFzJdnmuFd9kxbXt/CFzO+MJai1dZom5Sl1dZqSV6XTOautvcpzT2Uy74v9hn7AWdng0/ZRM5xfpy5A+Y2GR/tMpKZe/K50//HNh2/vByKXLcAMKrvqDbPy5AkrHBMJCVDehUL0Z1pq7WMvnM0UTOjTD4fe3lsoz6iQgghhLANkvizgIoKODfmXAB+P9448Xftsmvxe9GPFUdXGHtsTO7/Dw5/RMKJj5s/aG0Z7F0MiZ8DphN/hhV/mbszSfg2oVf0OGkvSfwJIaxp3MJxzP1krskyOqezc7Tjku8uYfLiya2OFbZJr9eTsj6FouQiNm2CQtd/+cDfgdEfjgbAxQX8/ZWxra366+ejrBo5XtC4V5JLHxcC4gPwjmxbcsDW7P96P+8Pe994oTx4TDAXfX0RUWfa/gUqzzBPSoZNpsBPqczv6+zbppVdV18NDz0Es2bVbzP+fhQqvx8ODuDhoTxn6PMXOT2SB/IeYOg1Q833Jqyop5ZLNvy8PB28uWboNSaTO83p3x+uOFUUJDFRebxsXtNE3+nGhozF1d6V3IpcY/LYUOoTlORxzLkxRJ7R9v5vvU1RUhFZe7Ioy1Iy7Zte2MTzXs83SrTrtXqWTFvCphc2tfm4Q68eyiOljxh7B2eeyhkGBpoe7+qixk7nRlVV245fWq6l8FTib0LohDbPy8/VD18nP1DpOZx/yOQYvU7PG9Fv8MsNv7T5uEII83P2cebcN89l6NWm//7PeGEG05+Y3sWzEkIIIURnSeLPAioq4Jx+5wCwPX27sexWSlEKuzJ3odPrGB082pj4s3MPgf53Qp/xzR9U4wSHXoKTPwD1ib/iYqXHH2Ds8bf1ta38dMVP1JbXmv/N2TC9vj7xV1ys3J0shDV0pAenRtP6hTnR/YVPCWfYtcMabcspz6FOZzogDZk3hMBhzVy9EzavtqKWz6Z+xspH17B7N9TaFaGlzriiBiDy1HX05OSWjxXtrVz0PT3xp9aouXbNtYxbOK7ZfW05voRNDGPaE9OMpeLcAtyIuyIOzzBP607MDPoM6MPJfmdQ4Kf8zejj0qdN+zk7w6RJYNegk7epxHCfU4cz9Pmzc7LDxdfF5vsk/v3A3+z9fC+VhZVsf3c7yeuSrT0ls6gsqGTrG1tJ3KhU+Ih0G8KSC5bwytmvtOs48+bBgAHK1zEx9StEW+KgcWBqxFQAViauBJSVg6f3+TPFluOLuRxZfoSDPx7koq8vwj1I6cfoEeJB0MggPEI8jOPsnOyY/e5sRt3W9pV1AA5uDtg5Kv/gs7Igz30NWu/DaHVNP28a2hVVVUFueeure0vLtAxJfYPhdbe1q9Qn1K/6y9YnmFy1rlKrcA92l75hosMkvgghLEXiixCiJ7DtM/tuSKVSUV0Nga59GR44nAivCJKLkgH45YhyN+PE0In4u/ob+x2o+wyHUW+C15DmD6y2g9kHYPJPQH3ir7CwaY+/kbeOZN4P81Dby4+3ofLyxsm+4mLrzUX0bstvWs5Xs76iprymTeM1Gg1xcXHy4bOHOvvLs3F52oV/kv5h08lNvLblNY7mHzU+r6vTodfJCu6eSKVScc7r56CKi6OuDjz9iwClZ5eBodxna4k/Q2Ln9FKfrbH1+BIyLoSpj0819vgzaEsfze6uulq5OO9bOpXUOwv4++q/O3wsQ2L4RMEJY2K5YZ8/UGJNTkIORclFnZm2VenqdPz70r8c/e0odZV1/H777yR8k2DtaZlFUUoRf97zJ6lrleSth0crOzRDo4GHH4Zzz4W77za2D2/VI5MeYfU1q7lzzJ3GbQ0Tf7/f+TvPuj9LXVX9h21bjy8dsWfJHj4a+5Gx/DCArlaHk6cTTp5Oxm3DFgzj2n+uxdHDsdH+o24dRcTUiDa/XtqWNHIS6nu7pmZVsGXAGdyfPIjSmqbZNkdH0Klqea5gNP4v+ZNanNri8WurHAjNv5YLHd7BTm3X4tjTxQcqib9S5wPGG15Pd93665jxwox2HVcI6J3xxVJ2frCTHy//kYq8CpPP5x3J44d5P3B42eEunpkQ1iHxRYjOOXIEXngBDpku+tBEVlYWTz35JKMHDaJ/cDCjBw3iqSefJCsry7IT7WLWiCmSGTI75eJsZSWsvmY1J+4+wZjgMQD8fPhnAC4ceCGAMfEXHNzGQ7v3A40DAJ6nbmQvLoZhgcP47Yrf+GGeshowdHwogy8ZbLzzUyhOT/QVFFhnHqJ30+v06PV6jv95nNWPrG7bPno9JSUlUr7Xxun1el4JeYVfb/q10baj+Uep1dUS4hHCUxueYtFfi1iTtAaAf1/9l6ecniJ7f7a1pi0syN7FnrF3j+VgrZK0C4tR/lA1TPy1ecWfj+kVfwCJqxJZfvNyY4m5hnpifPnuou94Lew1a0+j0za8uIX+Wz7HSVtJsI83IR4hHT5WuFc4bg5uVGurWXZ4GVC/4s9QOrK6tJp3495l/dPrOzlz61FpVNyXeR/nvHYObkFuXPXXVUz77zRrT8ssfPr5sGDdAtwnxAFg51JGZW1lh47Vpw/cdhtERLR9n0lhkzgj8gyc7OqTV4Y+f2Vl4BXhRejE0EZVDXpifGnNoZ8OKaWHG7zlQRcN4vYDt6Or07H9ne1mfb1fb/iV7y76zvh9Up6SYXOxc8XTsenKZ2dnUOvtUemU88S1yWtbPH75qfZ8rq4tDjMpxicGgArHE80m/oToqN4YXywlc3cmB747gL2LvcnnVSoVB3862OgmAyF6MokvQnScXg+vvw4bNig3G/7wg7KtOa++/DLhISFsfOYZFh4+zLsZGSw8fJgNzzxDeEgIr77Svuom3Zk1Yook/szMkLytqABvZ29jf5EDOQeMJ1YXDLwAgPR0cNSUc2bNcDjyVusH11ZB3jaoyMD7VKue6mpwwovZ/WcTHxBv5nfTs5ze189Q9lOIrqRSq7jw8wsZMHcA297ahra29bKfOp2OxMREdDrbX8HSm+lqdfj088HVv/7qWXppOhW1Fdip7Yj0iiTWT7k7PiFHWaHiG+PLgPMGoNbIn+ue7MAB5dGnbxEAXo5exucMK/5SUlr+wNxcjz+AnAM57PpwF3mH85o8Z+vx5YsZX7D85uWNtvkO8CVkfMeTZN1FQUopLkUZeHpp2rwqqzl2ajsWjl0IwMOrHqZWW9uk1KejuyNT/zuVgRcM7NyLWZFKpcIt0A33vu6oVCqiZ0bjFuhm7WmZhaO7I+FTwqlzU04C1tktxuUZFx775zGrzalh4m/C/RO46s+rGq1gs/X40hFT/zOVuR/PNXkBfctrW/j9zt8pTCzk52t+5tjvx5qM+feVf3nR/0UKk9rWkHzK/01hymNTAKipgczyNACC3UNM9rl0PPXj8dMNA+BwXssreFZkfkih61YcndvfIyE+IJ7BDufgVT7GeMPr6Q58f4CNz29s97GF6I3xxVLmvDuHxVWLm038+fTz4dHyR42xRoieTuKLEB23YwekpipVRXQ6+PxzePxx060BXn35ZZ5dvJj1Wi1/VlUxHzgTmA/8VVXFeq2WZx99tMck/6wRU+RKopm5uChX5SoaVEnQ6/Wc+fmZgNLrINI7Er1eWfHn65KOAwVQW9T6wXM2wN9j4eQPODqCg7L4r0kC68jyI7zo96KUYjjN6f+fZMWf6GpVxVXGOzzmfDCHh4seRmMv5SN6C42DhgVrF3Dm02ei18Px43Aw+wgAUd5R2Gvsjf1wEnKVxF//Of257OfL8I/1t9q8heUc/e0o78a/h+ZkMgB1dkVA4xV/ffuCvb1S8rGlShfR3tFMCpvE3AFzqdU27vE77Nph3J9zP+FTw838DqyvsqCS6pLqRtvOevYsLv3xUivNyHwib57BnnMeJiXkK2797VZWJa7q1PEenPggg/oM4rZRt6FH36TUp9pOzbT/TKP/7P6dnLn1VBVVkX8031huUq/XU5Je0ubS2t2ZtlaLTqszrsCq1SiJIVOruixlV+YuFv25iPd2vAc0TvwJRd+RfdHr9Wx7e5tx25rH13B0xVEmPjSR6zddT0l6Cfu+2EfuwaY99px9nPHt79uoZGpLYi+PZeg1SqPGnByodFBKd4Z7hZocb+jx51Gr/Ds/WnDU5DiArLIsPs69mU0Dx2PnUt6m+TQ0NWIqzwz+g35ZDzeb+Nv3xT7WPr623ccWQphXS9WiVGoV9s6mk4JCCCFEQ8uWKY8XXAD33KPkLvbsgQcfrD/vBKW858MPPcTy6mrGNnOsscDy6moefvDBHlf2s6tI4s/MnJ2Vx8oGlXfqdHVcPOhinO2ceWjiQwDk5kJtLeRU9kd1QQoMebT1g/uOgrj/gf8UVKr6Pn9FRbDi6Ape2PQCR/KO4OzjTJ9BfZq9Y6u3Oj3xd/oKQCEs7eerfubTyZ+irdHiFuDWpK+L6D3WrIFFi+DrlUrir7+vcgHOkPg7kHPAanMTXaeuqo6a8lr0auXjWLmuCGic+NNoICxM+bqlcp+uDq5suG4Dn57/Kfaaxn//nbyccPVzNbn6w9bdvPNmLvn2EmtPwyIMJcqzXP7m/Z3vdzouuDu6k3B7AovGL8JB49Ck1GdPcGT5Ed4a8BbH/1RWvu7+ZDevhrxK8ppk607MDHa+v5Mn7Z6kcE8KADVq5YOst7N3l81hW/o2Xtv6Gr8eUUpWN0z8Ze7KZNUjq8g70nRlcW9RWVCplPN8azsbntoAQEVeBeufXM+hpYfoO7IvoeNDCZ8czkOFDzHiphFNjjFswTCu33g9foP82v36mZlQaa+s+AvxNL3q2XCu6lZ1KvGX33zib3PqZgDcK2Pp49axBHPfvspjc6U+Z74yk1t23yLl1ISwEr1Oz8GfDpJ/tOUPA3mH8zjw/QH5tyqEEKJZJ07Avn3KNYy5c+Gss+Dll8HHR1kFeP/99dc0PvrwQ6bb2zeb9DMYC0xzcODjjz6y8Ox7Jkn8mZnhZKrhij97jT1vz36b8kfLuSr+KqC+v19g4KnyoKo2/CgcvCHu/8BnOFCf+Csuhte2vsZDqx5ia/pWwiaGcd3664ieGW2eN9VDSOJPWJtrgCvekd5oHDTUVdWRtTeL0ozSNu3r5OTU+iDRrRWcKGDtE2vJ2pPFrl3Ktv3pygW3Ab4DABjkNwgVKnIrcskpV/porF68mg3PbrDKnIVlDb5kMJesv4synzBcXKC/bwzTIqYR4xvTaJyhD1dSUsdfq+B4AVl7Td8l19PiS+q/qfx1718UnLDtpf0nVx7BPS+RajtlVZCfa/sTAadTN/i8eXqpT4Cfr/mZHy79odOvYy1+g/2Y+NBE4yrp4DHBjLxlJO593a08s87zivRi8CWDqbRT3kslpxJ/Tl2X+IvzV/oLGspRN0z85RzIYdNzm5qUFO5p8aUlf9z1By/4vsDZr53NjdtuRK/X4+TtxJ1H7mTyo5MB0Gl1FCYW4uTlhJNn5/7fJHybwBv93iBlg5IMzsqCKodTiT9304k/Q6lP5wrlc8ex/GPo9KbLDhkSfz5lEzvU4w+UXva16hJOZlaYLFftG+OL32C/HnljirC83hRfLKUsq4wfLvmBrW9sbXHclte28ONlP1KRW9HiOCF6CokvQrTfzz8rj5Mn159rRkTASy9BaKhyw+miRfDWW/DT519zdVVVm457TWUly77+2jKT7uEk8WdmLi7K/9IKE5+HGp7QGBJ/cwZ+AOkr2vciej3o9cbEX2EhBLsHA5BeIp3Tm2NI/Bn6I0qpT9HV5n40lwu/uBCArL1ZvD/sffZ+sbfV/TQaDQMHDkSjkbKgtiz3YC7r/ruO7H3ZJCYq29KrG6/4c7F3IdpHuWnDcGH14A8H2f/V/q6fsOgShhJ5bm5w/4T7WXPtGi6PvbzRmJhTecBDh1o/XnVdNYWVTe9sWTJ9Cb/e8GuT7bYcX/KP5rPjvR0UpRQ12p57MJctr24h/4htL2XL+OgPQg6tpkqtJFL6uPQxy3HzKvJ4ZsMzvHPwP4BSpcLwubUsq6zNN6R0R31H9uWs587Cp58PAAFxAcx5bw5BI4KsPLPO6z+7P/N+mEeFk/LeKvRdv+LPsCo9tSSVoqqiRom/AecN4M6jdxI9o/7GQ1uOLx0RNCqIAecPIGxiGJ6hnqhUKtQaNb79ffGJVn5uX53zFW9Ev2GyzCcoJV03PreR/V+37e++vbO9MYGYlVVf6jPUs+VSn/blEdip7aisqyStJM3k2E2pmwDwLpuAi0ubptPEzWvO468Rnpx0WWZydbG2Vkt5bjnamtZ7XgvRUG+LL5bi4O7AhV9eSNz8uBbHDVswjIu/vViqSoleQeKLEO2XmwsbTt2vfuGFjZ/z84MXXoARI6CuDv76C06mlhDYxmMHAMWGcjg2zBoxRRJ/ZubkpNwx2bDUpynp6aBCx8w+98Ghl9r+Aslfw89BkL8Nz1MVV4qKGiT+StOpq65j3f/WceB7KRXXkCHxFxmpPMqKP2FNvjG+TP3PVMInt95zS6fTkZ+fL82lbVzk9Ehu3XsrYWfGkHbqGlufwlnMjbqMkUEjjeOG+A0B6hN/C9Yt4JZdt3T5fIXlHf3tKAe/3AV6vfECuilDlF8JDh9WPig35/Utr+P8tDP3/n1vk+cmPTqJsXc3LaRhy/ElZUMKK25bQU5CTqPtgy8ZzMKUhUSdFWWlmZmH48VzSB94BmX6Uyv+XDq/4g+gvKacxf8s5qWtz+HiqizBMVyQv/rvq7l+4/VmeR1hGYYef+Xarl/x5+nkSaiHklBKyElolPhz8nLCN8a30UVhW44vHTF+0Xgu/Fy50lGWXUZhYiHFJ4upyK+/I3TwvMEAfDHjC5PHUNupWf/kevYuaf3GsNjLY7lt/20ExAcASuIvLO9G5vd9gnEh40zuY0j81VTZccngS1gwbIHJFX9VdVXszNgJgHcnVvz5uylxq9zxhMk+f1tf38pL/i+RuSuzYy8geq3eFl8sxdHdkfj58YSON32zgEHIuBBiL4vFwc2hi2YmhPVIfGldVlYWTz35JKMHDaJ/cDCjBw3iqSeflD5svdi//4JOB7GxEGXiNNzNDZ54Ap5/HkaPBnt7D9r625INeHp2XV9zS7FGTGm+g6/oEMPJlKkVfw0Z+hxs99rIhGHVbX8Bxz7gEgbaKuPKteJi6BulNFDIKM1Abadm7X/WMnjeYIZcOqSd76Dnapj427VLEn+iay2/ZTmeYZ5MWTwFAGcfZ6b9d1qb9tXr9aSmpuJlWOYrbJKDmwMB8QEcOoSx3FVkzt3cH3E3I/vWj/u/Kf/Ho5MfNSYA3YNsv0SdMG3729tJXJMMM0a0mPgLDwd3dygtVermDxhgepy/qz969BwvON7kudG3jTa5jy3Hl/5z+nPV31c1Wc3l5Nn5EnrdQXVYP4oz9ZRqzbviz1AytEZbg4dfCRXlnuTnK+VXbN3vd/1OWUYZl/50qXHbzg92svODnVy75loc3W23t+62t7eRezCXsppzARWltV2/4g8gLiCO1JJUEnISiHWfBCjJSJ1WR3l2ORpHDS6+yvIwW44vnVFbWcsrfV9hwPkDQK/0nlxcsRiNg4aRN4/E3sUendb0ib9KpeL6zdd36G9/ZiYEFp3PvSPP51QusAnDuWp1NXx/0Tc0V2FzR8YOanW1OGsDcKmJ7HDiL8pbufJTcSrxFx/f+PmgkUGMvHUkzj7OHXsB0Wv11vhibXq9Xkrzih5P4kvLXn35ZR5+6CGm29uzsKqKQCArI4PPn3mGJ594gudeeIFF9za9EVX0bDmn7sXt37/lcYMHw+OPQ23NlXzx8jPMb0O5z8+dnbngyivNMEvrskafXFnxZ2aGHn+trfjLyAA9atzDhkKfMW1/gaCZcM42CJhqLPVZVATBHvUr/tQaNbfuu5VZb85q9/xtXWYmHDtm+rnTV/wVFGCy10RHrbh9BXlH8lofKHodba2WI8uOkL5FSvH2ZlVFVVQWVHLieOPAYyj7aTCy70jGBI/B1UG5ylZdWk3aljQq8qSnRk8z46UZ9Htc+QDr5gahr4bi96Ifh/MONxqnUtWv+ktIaP54/Xz6AXCi4IRF5tvduAW4ET0j2phkMNBpdRQlF9l0yUpQVlHVaYrR6pVlnuZK/LnYu+DmoGSanXyVMzRDn7+MHRns/mQ3ddUtLC3txooSi8g91LiEYmVBJaUZpZSkllhpVuZxbMUx9ny6h7JyFXq0zIm8lPP6n4evs2+XziPWTyn3uT97f6MVf0VJRbwS/AqbX9zcpfPpLtY8voYfL/uR2spa7J3tmfTIJAZdPIj+c/sz4f4JaBzqS/vEXxXPsGuHNXuswKGBuPq3nmlb9+Q6Dv2s1IDW65USSwABzST9oD7xp9dDTU3z49anrAegT8UkVKg6nPiL9lZKv1Y4JhpvfG0ocnokc96dg2//rv09FkIoVj64ktfCX6Msu6zFcRV5FbwS8gp/Lvyzi2YmhOiOXn35ZZ5dvJj1Wi1/VlUxHzgTmA/8VVXFeq2WZx99lFdfecXKMxVdzfA5tE8bT1nvuPMm1tTWcnqH2SzgKWA00B+IBVZXVXHe3LnmmmqvIok/MzMk/lpa8VdXp5Ri8XHOIDiwbY0sTTFV6jOjVKmhEhAXgFtAC8sHeqCyMrjvPrj/fkyeWJ6e+Kurq++t1FnaWi073tvB0eVHlddKKSJjh4l6NqJX0thruDfjXs7/9PxG2/+69y8+GvuRlWYlutqa/6zhBd8XOL5bSUY49cmhwiGZxKSWl/sf/+M4H4//mBMre0cypzfxH+KPOlr5o+TmBrnlueRV5OFi37SZUqxyrb3FxJ+hP2RmWSblNeWNnjv2xzHejX+XpH+SzDP5Lrbh2Q3GC9wGtRW1Ju+aqyyo5PXI11n35Lqump7Z5R/Lx/GVZwk4obwHV3tXnO3NtyLG39UfADuPxom/vV/s5dcbfqUyv5U72LqpK1dcyR0H72i0bcIDE7gv4z78BpunVKq1XLb0Mu5JWkh5OajQ8MG5S/j1il9xd+zaVeFxAUofqOTiZGMyqKwMXPxcGHP3GEIn9IClox2QezCXtC1p2DkpBXXOeOoM4ufHM/y64Zz13FntOpa2VkvB8QJqK2ubHVNXVcfax9dy4DultUNFBZTWFpPrvopC9dFm93NssOi1qkpZ+Wuqx9994+9j7bXriMx4EKDDPf4Mf5fKHU+YPD8TQliXSx8XXPxcWl116+zjjGeop1QiEaIXy8rK4uGHHmJ5dTVNG0goxgLLq6t5+MEHpexnL2NoHdHWxF9gYCDPPf885zk6GpN/rwLhwEZgIfAu8AgwVaNh9IgRklDuAEn8mVlbVvxlZSl3WS4cdwPea/qArp3NzDP+gG234O2h3KbZcMVfZmkmWp2W6pJqik/afuPL5uj1cPJk4xV733+vlEHT6eCffxqPr6pSStqA0lTUcIeyIRnYWbo6HWe/ejY+MT5oa7W8N/Q9Vty+wjwHFz2CWqPGpU/jqya1FbXUVtSirW09Bri7y0mWrQudEMrwG4eTlH3qD8WwT/gnPpIvy65rMvbjXR9z5+93klmaSd/RfTnzuTMJHNrW1sfCVtRV1RlvQHF0raJaq/yh8nLyajLWkPg7eFD5O2eKj7OPsd9XYmHjpaRqOzV1lXUmLyTbQnz559F/2PHujkbbPpv2GW/2e7PJWGcfZ8bdO47omdFdNT2zU6lUVPmFgl0EO+cVsv+2/WY9viHxp3JXbs00nKiNuGEEV664Eicv2y+VaqDW9IzTHTsnOzQersZ//x1dgdVZcwfMJePeDH674rfGPf48nZj1+iwGzG1ci9gW4os5XPrjpdx55E6zlMDb9uY23ox5k4ztzd9EqHHQcOeRO5n+v+mA8m+42GU3WwfM4JKfmr8jWq0Gh1MtujanbMX5aWemfja1yThHO0fGBU3Bo1SpTNPZUp/VDhmczGh6glyUUsSPl//IgR+kN71ov94SXyxp4oMTuXnHzWjsNS2OU6lV3PDvDUx6eFIXzUwI65L40tRHH37IdHv7ZpN+BmOBaQ4OfPyR3OTem7R3xR/Aovvu45FnnmGKRsMgjYangfXAn9BoNenqujpZTdpBPeNMuBtxdVX+l7a04s9wt2Ni1SxU/W4FdcsfsprI+xeOf0Afh4OA0uMvwDWAFVeuYMfNO1CpVHx/8fe8G/duR96CTVi6FO64A158UbkAmp0Ny5fXP79mTeOkoKGfn4ODUuLG0B+xoMA887F3tmfcPeMYeP5ANPYaznruLCY/Otkq9XtF97Pr411k7Gx68WbOe3O4bf9trZ5oaTQaoqOj0WjaGStEtxJ7WSyz3pnLyQx7AFR+RwCwK+nXpOfoq1te5e3tb7Mnaw/ekd5MemiSza9WEY1pa7U87fw02e8uBUDlXASAWqU2lmFsKDJSWXFRUQFJLSzaM5T7PL3PX/SMaO46dhf9Zzcuum8L8UWv1zPylpEMvGBgo+0R0yOImRPTZLxao+bsl89m0IWDumqKZufTz4cT46+iIGQofX28iPSONOvxDYk/vbOy4s9wQ25AfAAx58Zg72Jv1tfrCrUVtez9fC85CTlNnktclcjOD3ZaYVbmk7k7k4yDRQCo7WrRqqyzKtPD0YMg9yBUKpUx8VdVpVTSOJ0txBdzMqz2Ayg4XsATqid4QvUEKRtS2nWcsMlhjL9/fIvlPlVqFb79fY0lMvPzocpeWbkX4hHS4vEN5T772Iej0+tILkqmuq5pz/nyUwvH1erGKwXbw9fZF3cHDwASi5Ka/J5oa7Qc+O4A2fuyO/YCotfqbfFFCNF1JL6Y9svXX3N1G/qxAVxTWcmyr7+28IxEd1FXV3/d3a+dl60W3Xsv23ft4oROxwro0atJrRFTJPFnZo6Oym24LSX+Mk5d/z9hdzeMeKn9LxJzB1yYgWvIMEBZ5abXaTg35lyGBQ5DrVIz6OJBjLxlZI9MPNXVwa+/Kl9v2ABvvQVLlijbY2OVC6O5uXCgwY2jxacWP3p5Kb2SDIm/0y+2m8uoW0cx8IKB0vhaUFVUxW83/8bGZzd2+Bg6nY6srCx0zS3zETbj5EklVrm5QXqlUorLrap/kz5/sf7K8q6EnBbqOgqbpq3WEjc/Dm3QqQu0TkUAeDp6olY1/XimViuNsKFt5T5PFLatNKwtxBeVSsXsd2cTe3ksel3955oZz89g1us9s59xXV19pQJLrOzyd1ESf7pTib+eUIKvKKWIZdcuI+Hbpv9A/n35X/64+482rbDvrpZMW8Lqe5S73Er8/8T1WRcmfWLdlRcNfzfLy+GX63/hr3v/Mm6zhfhiDhk7M9jz2R4qC+qTsS5+Dao8tPN0LHh0MDNfnEmfgc3fMl2eU05FXoXxXC8/H6ocUoG2J/7cVQF4OHqg0+sa/c14a9tb3PX7XWxOUVZZu7go508doVKpmB8/n6j826DWiZzT8vI+0T48VvMYZzx5RsdeQPRavSW+WFJddR1/3/83x/441qbxJ1ae4M+Ff1JV3PF2NULYAokvphWXlNDWGkQBQHFxz61CJxorKFAW39jZ1bcla49ff/mFMxwde/xqUmvEFEn8mZmzs3Ly1ZbEX3BwR18kAJyDcHdXLgQClJQ0HjLq1lHMeGFGj0w8bdumBBVnZ+UkdOVKJQGoUsFNN8HEicq4huU+DSU9DQk/Hx/l0VyJv61vbuWDkR9QcKLxEsLSzFLzvICwWfau9ly75lomPjixyXNl2WXseG+HydWADen1erKysnpkIr83WXHHCv55ZCUA0dFwJF9Z8edaNaDJCi5j4i9XuYC98sGVvBbxWqOkh7BtDm4OXPTlRVTEKmXUdA5FgOkynwYt9flLTFRWbZ0ZeSbXDL2GQX2arnbb+/leDv54sNE2W4kvf937Fy/4vtDmv6srH1zJN3O/sfCsLOfwX8kEH15Nkf0P3L/2Vn46+JNZj//wpIfZd+s+7pt8J6DcMFVdDYd+PsQzrs9w8KeDrRyh+/EI9uCyZZcRe3lsk+em/W8a12+8HpXadj8XT3l8CqFzhwGgclE+wLo6WKfe57cJ3zL3m7l8tveTRn3+Tm482ag8pa3El846tPQQv1z3C+U59b1VnTydeFz3OI/rHidscpjZX3Pj8xt50e9FipKKACXxV+nQvhV/1dUq+vsqq8CP5tf3Bfxy35e8tf0tErKVuyg72t/P4N3Z7zCbd3CtiWpyk4FKrWq18oUQpvSW+GJJpeml/PvyvySuSmx9MJD2bxpbX99qjDtC9FQSX0zz9PCgreussgHPjmSAhE0y9Iv39e3YzWK9ZTWpNWKKJP7MrC09/tLT4dyYtznHaRZUNOQiasMAAQAASURBVG2m3iaVmahyNxgz6UVFsC55Hc9vfJ6NJzu+ssgW/P678njeeXDPPfXbp02DqCiYrrS6YNMmqFHaIBoTf4b/X+Yu9VlXVUdFXgWO7vV1cFY9sopXgl+hJL2khT1FT6ex1xA+JZzgMU0z/WWZZay4bQXHVrTtLkth2078dYKc7Uq5r4CIfPIrlaZartUxra740zhqcA9yb7SaQPQMhh5/dXZFAHg6NX+CZEj8HTjQuJx1Tg7cdx888gjcMPxGllywhNn9ZzfZf/Ujq9n80mZzTb3LlGWVse3NbTh61P+NLThRwLJrl5H0j+m6pwXHC2y6dFzS2pMEHd9IneMOPtj1PptTzftzi/aJJi4gjjB/LwwtTDIywC3QjfAp4Tj7OJv19bqCo4cjA88fiH+sf5PngkcH03dUX5vu9zfhvgl4T4lTvnFSEn+Gnp5d7Wj+UZYfXc76lPWNEn93HrmT6zY07Vvb0424cQSXLbsMrwivRttVKpXxv/ZacccKvr/k+2afD5sYxoibRuAZpvzNaFjqM9QjtMVjGxJ/VVU0SfwVVxWzPWM7APHuyio8c6w69j/1z9LQT7ShtC1ppG/rAcuOhbAxnmGe3H3ibsbfO75N40fdOoqFKQvxj2v6d1YI0fOdf+WVfOHUtj7gnzs7c8GVV1p4RqK7MCT+2lvm00BWk1qO7Z79dlOGxF9rPf78XVPwrlkHDh08Yf/3WlgzEy9PpWRRYSF8f+B7Hl79MH8e/5OjK47y9eyvyTnQtM+JLUtPh717lTsIzj4bzjwT7r0Xxo6FBQuUMbGxSrCpqFBWB0J94s/LS3k0d6nPiQ9MZGHKwka9OMImhhF/VTx1VSaanoheoyyrrNlVWr79fbnqr6sYfsPwLp6VsIa7j99N4QXKBVFNgHKBLcApBDuda7OJv4O5B9HpdZzx5Bnc8O8NuPTp5G33ottI35bOrzf9Sk1KJgDebq5Mi5jGqKBRze4THa1csC0thZQGLaP27FFKQ+blQXJy86958bcXM/ejueZ5A12oIr8CvVbPuEXj8AhWekUVHCtg7+d7KUw0/Yf8sqWXsTB5YRfO0rwiLhvL/ml3UBiodEnv49KOLuntZKhAkZYGoeNDmf/HfCKnm7enYFdo7Q5KbY2WsqyyLpqNZRhuFNA5WjfxF+evJCD35+w39vkrK6NHVhppC+9IbwaeP7BRj7/OKkoqouBY83coDrpoEOd9cB5qO+V0Xin12d4VfzDAdwAAR/KUKgTrU9aj0+uI8YnBQ68kEM2R+HPyLKXCIcl4TtbQdxd+x1+L/mr6RA9TmFjI6sWr0WmldJzoHtR2aryjvI2frVrj6u+KZ5inTd9EI4TouBtvuok1tbVsbWXcVmBtTQ033HhjV0xLdAOGxF+fDp6yympSy5G/2Gbm4qKc8DaX+KuqUlaZfbbnBcrPLQG7Dp5JxdwGI17F17sWUHrY9XXvC0BGaQZlWWUkrk60+Qscp/vjD+Vx9Oj6O0enT4fHHqsv36lSKav/ANasUR4NCT5D4s/cpT5N6T+nPxd+fiE+0T6WexHR7b039D0+m/aZyefsXeyJnhnd6smWSqXCx8en115Q6yl0Okg6qZSzqnZTLrAN9FMuuGVkKH8fDCK8IlCr1FTVVZFT3rNu4BCK7P3Z7P5oNzUFyt/p6VGTWXPtGj6c+2Gz+9jZwaBTFTz37Knfvm9f/df790Odro6DuQcprylvtH/45PAmq6FsIb74D/HnsZrHmPzoZOO2fuf04+GSh4m7Ms6KM7OcahypdutDtYvy79/PtYO3TzYjvSSdZzY8w7MbnjUm/my9z9+ax9fwvPfzFBxvmizR6/W8HPQyS+cvtcLMOi//WD4fjvmQ1N/2AqC1P5X4c7ZO4q/hzSkubspNiGVlSlw7/MthYxLWFuKLOdRW1pr9mFeuuJJb997a5vENS30Ge7TcT8KQ+KusbLDir0C5IWl10moAzog8g/JTf0I6m/hbk7SGBwo82BYz2+S515nPnsmEByZ07kVswIZnNrDx2Y2kb7XxYNtN9Jb4YkmlmaWUZpS2uZWAXq+nOLWYwiQLXkQRohuQ+GJaYGAgzz3/POc5Ojab/NsKnOfoyHMvvEBgYFvXcAlb19nEX29ZTWqNmCKJPzNzc1P+l1ZWNi7DZWDo7+fhAe6enbgrNPRCiLkVdy/lH0ZRUf1JXnppOsOvG85jVY8RdWZUx1+jm6mqglWrlK/PPbflsYZynzt3KknR5lb8maPUp16vZ9OLm5otN2YYI3ofba2W2CtjGXjBwGbH6PX6Vhukq9VqwsLCUKslZNuq2spatn51HIoKcXKCSf1jeWTSI1w97Ap8fJS/Fw1Xatmp7QhyCwIgtTgVgG1vbWPH+zusMHthCcOvG8692Q+S76GsrDKsmmnNqFMLArcrldjQ65Vkn8H+/TD8/eEMeWeIyfKQNeU16OrqVxvYSnzRVmtZfvNytr29zbjN0d0Rexd7k+MLkwpJ+DahUc8tW5KfVIJdVRk1dspZlLlX/OVV5LH4n8W8tvU1Qk4tDkpLg6riKlY+tJJDSw+Z9fW6gle4F0EjgnD2bVqmVKVSMez6YUSfHW2FmXVeTVkNZVllVBYrNexr7Ky74i/KOwpnO2eq6qqocT0BKIm/zS9s5rsLvkNbrSQDbSW+dIZep+d5r+f58bIfzXrcli4MVORX8NnUz9jz2R7jtvx8GJj+DPcOe5oIr4gWj92w1OewwGHMj5vPJYMuobqumq/2fwUo/WINN7J2tsdfuFe4Mm/HRAoKm652G7ZgWIuflXuKc98+l/l/zCd0QsulWEXb9Ib4Ymnr/reOV4JfoTy3bZ+VdHU6Xo94nb/v+9vCMxPCuiS+NG/RfffxyDPPMEmlYbraia+AVcCXwBl2TkzRaHjkmWdYdO+9Vp6p6Eq5SpGaDpf67C2rSa0RUySKmZm7uw47O6XkVraJ1jIZGRDsfoSL49+Dis7f7dewx1+gm3I3RXZZNip1z7szZe9eKC9XVvqNGNHy2NBQ6NcPtFrYsEFJ/kF9ws+cpT6ri6tZ9eAq9n25r8lzWXuz+GDkBxz88WDnX0jYHI29hnNePafFvgnfzPmGV0NebTE5rNPpOHnyJDqdlAayVcUni/n7mq/wO7mL4GAYHTyKZ858hhtG3EDkqYp6x4833mfVNavIvj+bUX2VTM/W17ey9bXWPgoJW6FSq9A5OqPX2KFStX1FxejRyuOBA8rfxIyMxjexJCTAQF/lAuq+7MZ/lzY8s4Fn3Z4l92CucZstxJeKvAoydmawd8leUjcqifCcAzlk7s5s9i71xJWJ/HTFT2TtaWvRkO4l4X9LGbLuHarUys/Kz8W8K/78XZWVn3kVeQQFK0ma9HTQa/VsfmEzx/863tLu3dKIG0dwzeprcPY23Z9w5oszmfjgxC6elXkEDQ9i0clFaMYpAaBGbd0Vfxq1hsF+gwEoclTuPCgrgxE3jeDCLy40nofYQnzprLqqOoZcOoTgcS2vsmuvysJK9n6xl4ydGU2eK8ssIychx1jZpa5OOdcJy7uB/5v6KF5OXi0eu2Gpz4F9BvLlRV9yz7h7WJ+ynrwK5WaDaRHTzLbiL8wzDI1Kg05dTXpJ0/fTGyy/eTkHvj9Av7P7AVBbUUvmrkwrz8q29Yb4YmlRZ0Ux5q4xuPq17R+5xl7D5MWTGXTxIAvPTAjrkvjSsmsX3Mv0s9LI7LeYl2MGcZNfX+53G0RR7GKSU9Mk6dcLGVb8+fp2bP/esprUGjFFEn9mptHoCVduamzSswmUiyoj+/7BRaG3QUkn7qbW6+Cvscx0vQJQEn++zsq/sPzKfOqq6zjx9wkyd/ecEwrDhc3ISKWcZ2sMq/7WrKlf8WdIlBpKfZaXQ01N5+Zl72rPjVtvZML9TUvU1JTVUJ5T3mh1hRANRZ4VyZDLhqCrbf53RK/XU1BQICtHbZirnysDHjiPooCBTVZ2DVCqfXLkSOPtA/sMxN/V33jX/6VLL2XBugWWn6zoEvnH8knZlg16PS4ucM+fd+H/oj9vbH2jxf2CgiAkRLmxZffu+tV+gwYpF3PLyiDUMR6AfTmNE3+BwwKJvzq+UR8qW4gvKetTWDJtCXPen8PF31wMwD+L/+GjMR81u0/UWVFc8v0lBMQHdNU0zcp1zBByI8ZQzqnEn5lLfRpWEOr0Olx9lQ9YaWng6OnEXcfvYsYLM8z6esI8DImYwe7jOK//ecT4xFhtLoYSkeV2SsPRsjIInxJO/FXxaByUsta2EF86y97Fngu/uJDxi5q/yasjynPKWXbNMg7+0PTmQf9Yfx7Mf9BYHrOwUFn9bWdXf67TkoalPhvq59MPbydvFo5diJ+rn9lW/Nmp7QhyUU6Q08qbniCvengVr0e9jrZW27kX6qbKc8rZ/cluUtYr/1b0ej2fTPyEr2d/TV219ILvqN4QXyxt8MWDmfXGrHbdND79f9OJnx9vwVkJYX0SX1q2bRs4OgYy57zH2HX0IEfS0pk48yDBIY9RV2ebCRnROYbEX0dX/EH9atIpGg1nOzVeTTrTqWesJrVGTDFfB3JhFBUFJ05AUhJMOC0XlJ4OW05cz+DR4Uz0HdPxF1Gpwc4dtYNyd1Zxcf1FnPyKfOoq6/jy7C8Zdt0wzv/k/I6/TjdSWqo8erSt9zRTp8LHH8PRo6BRrj8YS326uICDg5L0KyyEgE5cF9TYawgeY/ou37CJYSxKXdTxgwubtuHZDeTsy2Hux3ObLUdn7gtFonty9nHGc/oIyg8p8WfjyY2EeYYR4hHCwIHKPTiHD7d8jIA420xgCNPW/N8aDnx3ANW5j+HmpiG3Ipfcitw2fRgcPVpJ0mzbpqz0ABg+XPnd2rkTnItPJf5OW/EXc24MMedaL1HQUf5x/pz92tlETIswbht+/XDCp4Y3e7HKO8ob7yjrrIYyB/sJo0nNraYa5cOPuUt92mvs8XH2oaCyAJV7Dmq1H1VVUFSsstnexOufWo+jpyNj7xpr8vnUzalsfHYjEx6cQPjk8C6eXefkJOSQuSuT0ux+gCsLohczw8q5WX9Xf9wd3LFzUIJQWc9qK2513pHezPtxHkEjgpodo9Yonx/y86HaLpvaoASOFYQak7LNabjir6EIrwgKHqpfQm74mXZ2xR9AhGckaeWJZFcnA1MaPae2V+Pk5URdZR0ae03nX6ybcfV35aHCh6gpU+42ValUjF041vi9EEIIYUu2neq8MG6c8ujgAIMHKz3o9+xRKrCJ3qO2tn6xTUd7/BksuvderrjySj7+6CNe++prkpKLsbPz5Lo7ruTzhTfa7Eo/a5IVfxYQdaqtnqkVfxkZUFnnASEXgn0bM1jNOXMVWeHK3e5FReDroqz4q6yrROuiZc77cxhxUys1MW1ISYny6O7etvGenvUlQbWnbiA1JP5UqvqvO1vus7q0mroquVtTNJW9N5vjfx3HzlnusRAY75zXOJcx+dPJhL8WTkl1Cf37KzEpK6u+LDHArsxdLPxzIa/8+wqg3B1UlFxESVqJFWYvzC32ilgG3H4GerUGNzcoqioCaLVEG8CYU/cN7dhRv+IvLg5iY5Wvq04qib+DuQep1daaeeZdzzfGl3H3jEPjqOHoiqPUVdcxYO6AHn3jRFkZqPUOfDagiGN3HWv596K6AHTtXyljKPdZWJ1rvAEqLQ1KM0opSi5q/6StbMe7O0j4JqHZ5ysLKjn+13FKUm0vhh5dcZRl1y6jMqMIME8iprNeOfsVSh4pYX7Eg4DyO7v/m/28GvoqKRtSrDy7rnPk1yMsv2U5xSeLWx/cDhoHDYMvHox3ZOMbGCoLK1n96GpyD9WXbM7Ph3z3dfwVcBY3/tp6zxNHR+Wx6rQW06f3FTTXij+AUK++AJTqsptUWznjyTO4ZdctOHo4dv6FuilHd0fcg+pPYoddO4wxd4zBzlHOEYR11JTX8PGEjxv1Tm6LhO8S+GjsR+QdybPQzIQQ3d0Jpb0zcXH124YOVR737Ony6Qgry89XHh0c2n69viWBgYEsfuwxth86yKXXpTN62kEuu/wxSfp1kCT+zEylUjWb+NProTIvlT4uqQSbqQ1Ewx5/7g7u/H3V3+y8eSeOdo6MvHkkoeN7zq0WhsRfW1f8QX25T1DK3zQssWco99mwN1JHbHh6A087P03BcdMHOvjjQQ78cKBzLyJs0iXfXsJ9mfc1uZDSUMGJApZdu4wjy480O0alUhEYGNjicUT3tmfJHg7f8SauhelUOCQD4O3kjZeTF66u9XfFNVz1l1SYxOtbX+fHgz8CkH8kn9cjX2frm9LnrycYeP5AAudNBmh34m/QIOXCf2mp8vffwUEpGWs4+Uo/EI67gzs12hqO5h817ldTXsOvN/3Kjvd2GLfZUnzZ8e4OvpnzDcUprV9gr8ir4EX/F/njnj+6YGbmpa3VUvzG5wQkbsXPw5N+Pv1Qq1r4yL7jDvjJF+oq2vU6hsRfTnkOISHKtvT/Z++8w5uq/j/+yu7ee9DBLKXsvZG9RAEXKrhAEReoCIL6UxEn4hYQFP0iigNlq+whe5e9S1u690raJvn9cZukoUmbtulIzet5+jS999xzz+m9Offc8/6MRPh+0Pf8dOdPNW1+gzH16FTu+eUes/tbjm7JfNV8YibFmC3TWIm+J5r7191Pnlww8pM5KisWStwMp14DVUa9tEl3T+rm1vn5IHeR4xrkqvdEs6Xxpabc3HeT48uO11mYyuL8YiNP8CtbrrDv3X3E74/XbxM8/lIBw/e6MhzL0mDeHurzdqyV4w8gyF2wLlDJkvVW4f8VYn+KNWtMUZRVZBdQash/YXypS5TZSjKvZFbbaKGksITchFwK06s357Bjx5awjy/mKSgwrKGW9+zTCX/nzglr33b+O+jCfPr4WJaWqzoElgW+uNVEUkQ3xJhiF/6sjLgkm/Bw4XN6uiE8JQifhzX7mO/GNSPIxfwiv8UUJROY8SYdA7aWeYmIGNp8KJ0DOyMVNz3rweqE+iwpEjwcevY0vNy6uxsPQrp6yl+jmuDf3p/o+6JxDTZt2rB97nb2vLWndiexY7NUZcmrVWs59cMpbh01/yQTi8UEBAQgFtuHbFtFLBGDVIpGIiVXIliFRHhG6Pe3aSP8Pl8u9WuouzCTjs8VFve8WnjR4/keRAwyHGfHttGFUauu8CeRQJcuhr+jokAmg+bNhRBuBfliWrkL4kb5cJ9SByknvz3J9R3X9dtsYXw5uvQoSzoswTfal7HLx6LVaPm81ecc+PiA2WMU7gp82vjg3syChFeNjOK8YjSJt3AoyKiQE1TPrS0GLz+vrhAyDqTVc8spL/zpDNISE6HztM50fLRjzRrfgLgGuuIWYn6SKBKJbHYBxzPSk9Z3tia/1AG1qIju6x1xWOBArqqc92LSX3B2AShTobD+3o7LC3+tx7bmiUNPENpbeH7ZwvhSW4a8P4SXUl7CI9zD6nVvfWUr77q+S36SIY5qu/vb8di/jxE1Pkq/LSMDimWCB6Alwp+5UJ+3Y02Pv/5h/WiTNx3P/D4Voq2kX0jnwMcHyLxaS2vMRkhOfA5rJ61lzzsV3wVLCkv4vMXnbHhiQ721J+1cGltf2Wp1D9WG4L8wvtQlbsFuvJz6MkPfr17c6E6PdmJW4iya9WlWRy2zY6fhsY8v5okvszvy8TGeH4SFCeut+fn85wx8/uuklQWhqG2YT1MECQEjmozw1xBjin0UszJqrRgnJ4MqXd7r79YtOJY0gq03n0VeRe4Fi9AU43z1/+gZ/CdqdcXcGhuf2siioEVNJiGtTqAz5Tqcn5LPwU8PotVq2fDkBn644wc0pRrkcujTRyijC+2pQ1dPbXOSxEyKYeLPE5E5ms7hNnb5WMZ91zTyLNqxnMwrmZz5+QwFaQWVlvNs7skrWa8w6M1BZsuo1WquXr2KWl031uR26p72D7XH6cXpFLn5kyMWHgyRnpH6/Trhr7zHX6ibsHB6K+8WpZpSxFIxIz4ZQYsRLeqt3Y2RI18f4cKfVSREtAG+v+N7rn6+GRCeRzkqYRHOEuEPDOE+weDpJ5VCdLTwuafDI7w18C3a+7fXlxNLxLyc/jIT10zUb7OF8UVTqqGkqISQniF0frwzYpm4SqMKiUzCo3sepfeLvSst1xhx9HIkY+ocTvZ049OrT/LtiW+NC9z8HXaNgjNvC39HvQi9vhc+azUWn2fBoAWcfuo0kztM1gt/CQnQ+8XeNhdGVaPWkHEpg6Is8y5M6mI1V/+5SvKp5HpsmXXQzeXz86FEKqgmJZoSXOTllOH2b8GQPbC1r+AFail5V6Gk+lZwVzKvMOrHUcw+fre+bbdjC+NLbRGJRDj7Oeu9HK1JYOdAYh6MMfImFIlFhPYOxdHTUb8tIwOKyzz+fJ18q6zXXKjP27Gmx9+dre9klPYrArPHV1gQTDqRxD8v/kPySdv7blaFo6cjE9dMpMvULhX2yZxkDPi/AcQ8FINGbfnYXRu8W3mjylGxoveKOvNSrS/+C+OLHTt2Ggb7+GIenfB3ex4/uRx96oD4eOz8h9CF+rQLf1XTEGOKXfizNjJBTYqMEC5meeEvMRFOJA9nd/5n1vF/dQqFEUf5+dJHgGBVsenSJt7f9z6nkk/h5OuET2sf1Kqm8bCqLMfftle28fcLfxP/bzwKVwVyFzmqPMGMdcwY4QW3Y0fjY8pbKNcl4QPCCeoaVLcnsdPouLTpEr8/8Dspp1MqLSeWiHHwcKiyvrzauqbaaXB0lvOZGsHbKtKjovB3+TKUlqUM9XfxRyaWodFqSMpLqs+mNlrybuWx+enNXFxvBa/5BqYgtQBVtrDqWl2PPxBy2OqmEu0N2p4+z19g0lReG/Aa0X7RRsc5ejpW8Hpq7ONL9xndefbSs3i3FMIcejX3YnrsdHrNsi1xqjrk50OO80nW3lzGX1f+Mt4ZPAZaPw+tnjbefu4D+M3b4lCPrX1aE+Mfg5vCTR/qMyHBCo1vAIoyivii9RfsmL/DbJmSohJWDV/FoU9tL1zymrvX8IHPBxSrtJRIBOHPw8HDOASs3AP8+kHYAxA81vI4SwcfhXVh1RKNATRaDVuubOFgynZAuGcL0wvZ9/4+buy6oS/X2MeX2hK3N67OcmK2u68d41eNxyPMA4DUs6kkHk5EqzG+tkKoT8s9/nTRUKoS/nTzFmvllDSXXz1ycCSP/ftYk4xoIHeRE31vNMHdTef56PFsD7o+2bVOhGNTiKVioiZEMeyjYUhkkno5Z13S1MeXuiTtfBrnfjtHYUb1QnZqtVrO/HyGS5suVV3Yjh0bxj6+mCauLI1zMxNOvzox0FbfJ+xUj+TkZBa8/TZznoli/9Zgvv4wigVvv01ysvUMuZqa8NcQ2IW/uuDm7zwUJuQYuW6IpkXSLSH8pLXy+yESgVcXnN2Et7fsbFhxYgVzts9hf/x+7nj7DqbsnILUoWmE/aws1OfIz0Zy96q7ada3GYPfHcxDfz+kt4Rt3hx++gkeecT4GJ2AWJvnuaZUw893/czx5ccrLacuUdebJaedxkHU3VGMXz2eoC5Vi77ZcdlcWHfBfo80Yc7+ehblnsMApJZWDPUZEiKIP8XFhueGWCQm2E14YOjCfV7aeIklHZYY5fb5TyGCXi/2ou09bRu6JbXm6TNPI39gPAAOTqX0DOlJx4COeDp6WnS8qys89RRMmGAQjsHwWZd0/Xay47K5tu2aTUYDKM4v5ovWX7D5mc0WlT/x7Ql2/d+uum1UHZCbkIvo0gXExcJCvo+TjyDKpOwUCkgU0OUTcLhtgV/hDR7RQqjHaqIT/lJT4d9FB/n+ju8pKSypRS/qF4lcQt+5fWk+rLnZMgpXBWOWjqHT453qsWXWwb+DP0G9wkEk0nv8eTrcNlbk3xBE325fQPPHLDcyDLsffPvCkRnVyhOpE5jyS/JQi5QolVCQoWT7nO1c+euKxfXYMhq1hu8Hfc/fM/+ul/Pt/3A/y3ssrxCmUQj1Webx52wdjz+t1roef1qtFrlbFvmKixU8/pz9nAUvRi9Hk8faMkWZVSRSLENTWrfvAFqtll3/t4vMK5k0H9qcdve3Q1OqqTIyiZ2my8V1F/n1nl/Jvp5dreNEIhGbZ2xm/4f766ZhduzYadSY8/gDbN6Q0I7lLF60iLCQEPYtXMibKRf4RXWLN5IusHfhQsJCQlj88cdWOY9O+EtJAbsDbs2wC391wcVP8Jf8C2iNFt56Fo/no6E9CAq04t2qVhEVcBJHaR45OeDtKFjDZxRZZu1tK2g0hpdPUx5/CjcF7R8UXB4kMgkisbDYocwR3mhlJqJw6jz+aiP8FaQWcHnTZVLPmF9kO/jJQRbIF5B0zO6x81/CvZk7MQ/EWOTNd3DxQdbctYbc+NwqyzZltBotv93/G6d/PF11YRvj2JJjSHYLnigpqjKPv3KhPkUiaN1a+Gwq3Gd8jjDDFsvEFGYUUlxQXA+tbny4Broy7KNhtBzZsqGbYhV0HufurlJ2TtnJiSdP4KawIJFtGaNGCUYt5df3I8r05LQ0OJt4g/UX15OnMjzo9r6zl/8N/Z/Fi5GNgatbr3Lsm2NIHaTIXeUkn0jmyFdH9M94c5z79ZxNLkxd2xlH2P41eKYJ183HyQcuLIbtd0BiJaJn88dh6D5wjzJfphzXs67zzp53+OTgJ7i7C4v7Wi3cOpdN8slkivNtZ5xx8HBg8MLBtBnXxmwZkVhEl2ldbDIv0aA3BzHgq3sBkDoLoo+7w235K7cPhJ3DDX+X5Fsm5LV6GvzvgCtLIGWXxW1yV7gjEwsT7OIybzOxpxtTj0yl5ws9La7HltGUahj52Ug6PtaxTurXarVseW4LSzosAaDbjG4M/WioUT5BrVYX6rP6Of4qE/5UKuH9C6yT4y+zKJM52V7simlDelbFsUVdotbnaG8q5N3K4wPvD9g2d1ul5TY+tZHFzRZX8OS0JvH749n95m4OfWbweF770Fo+8vvI5kN+2qkZUeOjmPDTBLxaelX72PE/jmfYomF10Co7duw0dm7eFH5X5vFnD/XZtFm8aBHvzpvHHrWav5RKHgQGAw8CfyuV7FGreffVV60i/vn4COv5paWGXIJ2qodd+LMyIpEI+v5G7h2JgIiEBMGDAyA1N4D0wlCCgq0YVuPaSp5v1Ym2vnvJygJvpzLhrzCDzCuZ7H5rN0knbF9wys83RCxyKZfORKvVsn/RfhIPJ1Y45seRP/J5i8/NejVYI8efa5Ar85TzGLxwsNkyPlE+RN8bjczJdA5AO00PrVZbLWGm3f3tGPfdOLMioUgkIjQ0tEJ4vqaGVqPl7JqzbHhiQ0M3xeqMXjKa9BEPAfB41MvM6TOHaF/jEIwm8/y5lwl/ZR5/kUMimRk/k+ZDzXu1NFVUeSo0ag37F+3nfa/3ybya2dBNqjGFGYUcX36cguuC0Uj551ptcXYGv7J13+GrBzHu53EcTzJ4pbe7vx2jvhqFRC7MRWxhfDn+zXE2TtsIIph2dBp+7f3YPGMzpcrSSo8bvWQ0My7MsDnvRq/2IVzveBfJgdmAILDg1Rnavy2EcrQS8bnxzN85n6+Pfo1IZIhIEfz4CF7JfAVnPyvF97ODulhNYXr1wqndjs4ATuIkGAm5K24T/po/ARGThc+JG+E3D0j4s/JKdd+NiIdh9FkIHmVxe0QikSBKA2JX4U1cWSIlqGsQLgEu+jKNfXypDVKFlG5Pd6P12NZ1Ur9IJCLrahZimfDKHtwtuELe0oIC4V2zRdJ83h6wkNbeVbfFklCfujCfYrHBQ7A2eDp6IhUJ70KJ2cYGk3m38lggX8C2VyoXyGyNUmUp7R9qX2XKB9cgVwI7BRrlKL206VKV6QKqQ7M+zXhk9yP0eaWPflvk0Ei6PNUFdbFtCn9NfXypa7xbedPu/nY4uFdtpHo7LUa0sCiqjR07top9fDFNYSGkpwufK/P4swt/TZfk5GTmvPIKG1Qqepgp0wPYoFIxZ/bsWof9FIkgIED4nGT70kaDjClNIwZkI0IsFoOjP14OQkjK3FwhBnKLFrBo3zcUF8PSyVY8oV9/jipfI6UgQvD48zJ4/GXHZbPrjV04+ToR2CnQiietf3T5/ZydQVrurs25mcPWl7bSbUa3CrkTwgaG4RriirpYjVRR8Va3Vo4/sUSM2Mm8ht5ieAtaDG9Ru5PYsSnyEvNY3Gwx/V7txx0L7qiyfEjPEEJ6hpjdLxaL8fb2tmYTGyViqZhJmyYZWbI3FbxbepPjCmTDpLZTCA+vWMaU8Pfh0A9ZPHwxvk5C6K76ygHTGNn5+k7O/3aebjO6EdAhAE2J7YbGzbqaxYapGyjtNQS8/awq/IHg9ZeaCsGy9iRyg9MppxkQPkDYd0cEEXcYwszawvgy4I0BdJ7aWX//95vbj7YT2+LsW7kwpcuLVZ+cXnWai+suMvGXiTWe2Et8PMkI8STfKxsAF7kL+A8SfqoiaStcWwndvgK5e6VFdZ5BqQXCInxwMFy6ZJsvVQmHEtj1+i56v9ybyCGRZsv9dOdPFKYX8vj+x+uxdfD3i39z5IsjzMmdg8K1+irK1le2kif3BjojchA8QSt4B7ebb/jsESPk+bs9HOzt7BoFEkfov1YIFVtNfJ19ScpPQuSSBpnCfN3Xoxh1sRpHT0ebGF8aO5M2Tap0f0ZZkJeokgeZP9CyOi0J9Vk+zKc11ijEIjFecn9SVQncyk0GDPNeB08H2t3fjsDOtv2+ejuekZ7c/b+7qyw34PUBRn9nXMpgyzNbGPn5SPzb+1utPWH9w4z+7vx4Zzo/3tlq9dc39vGlYdGUahCJRfpIS3bsNCXs44tpdCE8vbxMG67qxMCMDEEktEbEADuNi+XffMMgmYweVcTd7AEMlMtZsXw58+bPr7RsVQQFCWLyrVvQyfYyNhghFtf/et5/dwWxjlCr1aDVIEo/wKDoPYCQrykjQ7DGlEgMlvhWwT2KK05vkZAbRXZ2OY+/ogyCuwfz1Kmn9CEwbRldOM7bw3y6BLjw2L+P0e3pbhWO6ftKX+785k6Toh9YJ9Rn6tlUbuy+UaXXgZ3/FupiNdH3RFf7Zd2cV4parebChQvC+NKE0ZRqaDGyBb5tq85PY2sos5UUFghClbkJcKtWwuJaaqrBki7INQg/Zz8jASE3IZetr2zl+s7rpitqoriFuOHTxoc+r/Rhys4p+LTxaegm1RjvVt48+NeDZAUIIRlP52/D70M/xv08zir168J9uikFr9IL6RfMlrWF8cUv2k/v5Xpj9w2OLj1KUNegKhebSopKyLqeVW+56tIvpJN8KplLGy+RdTWrxvXoFty1MmGC4iKvhjKcex7iVkPWiSqL6oS/bGU2xepifR6v3IRczq89T96tWkyQ6pmC1ALiD8RXGcLWydupSsG4LtCUaHD2d66R6KfVajn48UFu7bgEgI8imLGtxtIj2JydLeAcBv3/gIAhlVUMlPsOaUrh1t+QutfitunuIZm7IB5nZMCnEZ/y87ifAdsYX2rDka+O8HX7r0k733Cxh3TCn9n1SU2JwbOzDF2oz5IS8/lSdOOQNRftfJ0Ek+2UAmPrb5mjjAk/TaDjIx2tdzIbJut6FqXKUhTuVnC1BBKPJHLlrytN7n21qY8vdc3yHstZ0WtFjY7ds2APb8vfJvOK7UbfsGOnMuzji2kqy+8Hwhqrh4fwObFiUDY7TYB1q1fzcGWWY+WYXFTEn6tX1/qcujx/t27VuqoGpyHGFLvwVyeIYPdoxjabA8CZM6A5MI272iwiIMDYY80auJcZdGdnl8vxV5iBwlWBf3t/FG7WeWloSHQef7cLf1KFlNDeoTUSCqwR6vPIl0f4fuD3RqFZbqdUWcrmZzZzdOnRmp/Ijk3hGenJxDUTib43uurCCIt6SzouYc3da8yWUVr4cLVlji8/zkKnhVzdepX0i+kN3RyrodVoed/rfUIP/EaB/BrHM/aQkl8xfJOTk8Hr76efzNdXqipl/wf7ufCneTGnKdL7xd48vPXhJhFyxcHDgRbDW5AtFvKqFIkySCtMI0eZY5X6dcKfJjMcgLicOP2+jEsZfNXuKw5+elC/rbGPL+XzHsXtiWPfwn0WCWsnV57ks8jPiD9Q9/Fm0i+m82XUl6hyVMzOmI1Xi+rnzNFxYP4mYrYtRqwR5hYuchfYPgT2WxAyImIyjE8D/4HC30XJkGA6fLKHgwcSkRDyNa0gTS8G5J2J45cJv5BwKKHGfahvWo9tzdzcuVU+d8d9N477191fT60yMGbJGF5KfgmAwvRCsq5XTxh+4eYLBD85GoBOzqNZ/8B65vabayiQewl2DIWbv1teqUgEgzZDv7JjSvNgz1g487bFVfg6+eIqd8XRVQUIuTfaP9SeVmNb6cs09vGlNmjUGkoKS2ok6FqLjAwolmSR472Nc2nnKha4uhzWNYPM41CUBBq1/rsO5r3+ynv8WYtAF0H4y1DVLuyTLVBSVMLKASs5ufKkReXPrDnDz3f9jLpETYvhLXj+xvM069OMq/9crXW46iNfHuHHkT9WEP4SDiXw+wO/k3DQdsb622nK40td4xvti290zYwtfdr4EDU+yu7tZ6dJYx9fKlJZfj8d9jx/TZuc3FwCLCzrD+Tk1H59oykJfw2BXfirC0Qi6PoFSb4LAPh3TxFOGb/SxvuA/oa1Jp1Kn+bdwQMqePyB4GWSl2Q7Ftvm0Hnlud0W1SgnPgeN2nS4N02phvVT17PnnT0m9+s8/oqKhEShNaH9w+0Z/slwXPzNW+NLFBKOLTvG5U2Xa3YSO00ekUiEezN3XAKtHO/PxnAJcCF8UDhr7lrDquGrbC4vlznUJWraTe5Erk8Et7x/YuQvA5i7fa7Jso8+KvzeuhUuXxYW45/f8jyPrntUX8aruRdPnnySEYtH1EfzGx2lylJ2v72bMz+faeim1BitVktJCaiEtXJUomxAEGKsgU74U6YIYb3KC39yV7nN5fNZ2nkpX7b9EoBOjwrxPXb9364qjwvqGkSvl3rhFuxWZdnaInWQ0nlaZzpM7lD7nL4uzpQ4uHKPZD3Xn7/O4MjBoFGBxoLcsXIPcCjnDZt/FY6/AEUVjQ3EIjG+zsKiX2pBqj78nzY0lPE/jq8yL5Wd6qMuVvNdv+9Y//h6i48RiUS4BrpSrBAs1kwKMap0SD8EynLXOeMI7BgmhH+t/ATCb7kn9Pweun5pcdtWjV9F7txcxgQ9AQjC3/CPh9Pn5T5VHNk06PFsD5678hxuIXU/xpgjIwNynY7zP9FQ7vn1nooFxHKQuYFIDH91hWPPIpMZLrvuOXQ7OsNIa4aiDvYQImHkaZMpvm042/n6Tra+UsW9akNkXcsi+VSyxe/h6RfSubLlCmlnBe9RqULK7rd3s2r4Kq5vr12Eh+7PdufOFXdWyCVekFrAmZ/PkHE5o1b127FNxn07jjuX31mjY9tObMu9v91bKyMnO3bs2AbJycksePttukVF8erzwRzZFcW+vW+bzd1mF/6aNu5ublhqvpUCuLtXnnrCEuzCX+2w5/irK8In0dofeh2DixcdeXBtGi7yLCY+aP1TOUjzcZLlkJOjJcYvhn8e+gd/F+HF6ovWX+DV0ovH9j1m/RPXI+ZCfS7vvhzPSE8e+7di/8RSMVc2X8Enyof+8/pX2F9+4aSgwOA5WR1Ce4US2suMn3sZIpGImfEzcfK2B7j+L6AuVvPjyB+JeShGv0BtCQ+sf6AOW2UbtLmrDW3uasPhLw5TnF+MplSDRCZp6GbVGqlCSt/372TlE6ByXAJAhEeEybJRUTBoEOzcCUuXwitvivjs8GcALBm9BIVUWJkP6GCpnVXT4M8pfyJxkDB26VjEMjG739xN67GtaXd/u4ZuWo3Y/9F+dr6+C6duj1HkEUihJhuwnvAXECCEcpMXlgl/2XFotVq9gPDspWetcp76InxQOFq1YAjg6OVI56mdCehU9XcguFswwd2CqyxnDTzCPBi7dCwAuYm5nPvtHNH3ROMa5FrFkRXxnTiQC2kD6ekC4R5l8fuGWh5+kaJkOPsOdHwPMg6DKgOKEsCxYvhpP2c/kvOTSS1INYT/c/IgZpJHtdvdkKSeTSXrWhbhA8Mr9b66sfsGN/fepMdzPeotIkbikUSOLjlK9xndCewcSPR90UgUlj/bSopKyLmZQ26aK6DAyVlDBdtN395wb65xSEexHNL2Quh4YGjFik+/Dk7NoMUThm3h1ZuLiEVCO3zLnEZSU6t1uB0rkJEBKpnwj9eFXjWi+ePCj1oFnh3Btz8ikfCMKCoSfkyRVha91MeKUbWD3YVxWylLJjvbOP3F5c2XKUwvZOj7Ju5VG8Qv2o9XMl+hVGWZdWnP53vS8ZGOfNvnWwa9NYhOj3Wi45SOlCpL8YupXZ6QoC5BBHWpaMjRYkQL5qvmI5Hb/lzbjh07duxYn8WLFjHnlVcYJJPxglJJAJDMLVb+upCwn97kvQ8+YOasWUbHhJSl8LULf02TcZMm8b+FC3nQAo/YHxwduWtS5bmqLUEn/KWkCE471o6i2NSx/7usTPlEjY4OWl6drQSpI3l5UnJyfAmug/Wnoo4/8PxXwgucu4M7Q5sbXpi6PNmlgnWfLWIq1Ke6RE37ye1xDTS/qPb0uadxcDfdf7FYEP8KCgRhsbrCn84byZKwc5V5BNppWmRezSTxSCJhA8KsVqdYLCYyMrJBEsE2BN2f6d7QTbA6hYXC7yJHwWo7wrOc8FecBbe2QNgDIBLxyCNw4ABcvAinDnjjIHVAWaokMS+RSM9IQAh9mHAwAXWxmvCB4fXbmQYg9WwqMkfBi0osEfPkiSdrJKg0FjzCPAjsHcEJHHF2hhxltrDdSsKfSCR4/RVcFOKw5BXnkaXMwsuxomW2LYwv5b1bZU4yxi4b24CtqcjW2Vvp92o//Xwrfn88f7/wNxK5hG7TK+Ygropae9rcWA2XvgC/gdDqOWj9gsG95zZ0QkFqQSq+ZdMlW4xsFPtjLPve3cczF5+pVPi7+s9V9i3cR9t72tab8Jd0PImT354kZlIMAAP/b2C1jk+NTWV5j+U4jB0C9GF59n3MWLCBL0d9yeOdHzcuXP46e8TAxByQyCtWqimB84vAr5+x8AeCQKRMBefKDdvKoxP+0tMhdnUs5349x9jlY3H0cmz040tt2L9oP75RvrQc1bLB2pCRAcVSQaUzKfzpkChgwEb9PeLoKIh+5jz+dMKfrxXTLvdt1oc2eU/jlDeggvA3ZccUpI5Na2lCJBbp5y5V4eDhQMblDKQOUv3Y5BHuwZB3K8nTaQEatQaRSGQyJKOtG9fZwvylsZJ3K4/9i/bT+s7WhA8Ir/bxqlwVO17bQVDXIDo83MH6DbRjp4Gxjy+C6PfuvHnsUavpcVtesgdVSg4BY199FcBI/HN0TObypW84uW81P3ydi7ubG+MmTeKJqVMJCPhvGS83RZ6YOpW333yTQ0Al2cY5BOwqLub7J56opJRleHsb1u5v3oTIyFpX2WA0xJjy3x3F6gi9CKRMh7V+cOJFuLIM19JThISYXXepFbrkqUplxcWaQW8NotesXtY/aT1jKtSnRCZh6PtD6flCT7PHmRP9dNQmz1/6hXQ+9PmQI18dqbJs3q084vbGGeUpstM08Y3y5ZXMV6r9vcu4lMG2udu4dbSi/7pIJMLNza1J5DarjB3zd3Dg4wP6v5tKqM/kU8nsenYtLhlxFMiuAegFPADi/4D9D0LczwB4ecEDZU4XP/wgItRNWHyNzzGYzWlKNawasYod83bUTycamGlHp/HInkf0f/vH+Nu0F3X0vdH0/mISxU4euLhAtjnhT1sulHXGEWGxvjzKVDj1GiRurHCOiAiQaJy41+Mj/nf3/1BIDCLHhXUX9Dn+mvL4osxWsnrMag5+crDqwtVk41MbWTN+DRmXMtj/4X5Wj1mtH7NajGjBfX/cR4fJNVsMu/XLPrwSTrO+5Flmb51NXu51uPg5ZJ20rIJWM6D3aggaCWJJpZPPT0d8ypnpZ7g76m59qE9Vag6Lghax8/WdNWp/QxB9XzR3fX9XlQYBXZ/qypMnnsQjzKN+GgZ0fbIrL6e9TLM+lSREqQRnP2f6vtqXkgDhWVAsykWlViEvL+hlnRJyOZYWGLaJxKZFPwCxDO6+Bd2+Mt6u1cK6cNhvmeffsVvHGPXjKD68JETeSEsT5jMXN1ykIKWgSY8vpapStr601eIcbnVFRgaopILHn6/TbSrdmQVwZAaoy14Qy10H3ffdnNBfF8LfyJYjGan5kqCse8i6Lc2lwk1h80KUDq1Wy/5F+0k6kVSt44K7BfPMhWeIGh9ltL1UWVrj1B3x++NZ6LLQ5H2qUWuI3x9PSmzFUNC2QFMeX+qarGtZHPz4IEnHq3eP6hDLxBz+7DBXtlyxcsvs2Gkc/NfHl+TkZOa88gobVCqz4k4PYINKxZzZs/VhPxcvWsToESEEXVnIe9kX+PrWLV64cIG9CxcSFhLC4o8/rrc+2KkbAgICeO/99xmrUHDITJlDwFiFgvc++MAqYq9IBC3LbOwuXap1dQ1KQ4wpduHPyqh1lhAKb/DqClJnOPwknP+ozs7pUHqdMa2XEOBylZwcWHV6Fe/ve5+UfNucxJtC5/F3e46/qlDlqri08RJp59NM7tcJf3k1eJcqVZbi29YXJ9+qF593v7Wblf1XUphRWP0T2bE5xFIxchczi21myE/O59/3/iVuT1yFfWq1mtjYWMP40kQ5tuwYF/68AMC2udv4NOJTNKWmc3jaElnXsri5KRaZMosCiSDeGYX6DBgC3j2F/Epl3Hmn4MWdlQU+ciFeRnyuQfiTyCWMWTKGYYuG1U8nGgHlJ0nFBcWkX0y36ftDZ3Di6grZqmzgNuGvOBsOTQWNWhD4rn0PN38rtz8L1vrD2QVw46cK9evy/LXOeJGH2j+Es9wQ3/rYkmNsn7sdsI3xZfMzmzm+/Hi1j5MoJFzbeq1O8hflJuSSfSMbr5ZeTD0ylTFLx+jvUYWrgjZ3tUHuXL3ngI6cDXvwTjzJjoIv+HD/h2hzL8Gx5yB5m2UVSBRCyEZp2fzk1t+CgYEJ2vq2JdovGhe5iz7Up1IrxyPcAycf2xHXAzoE0GFyhyqfve6h7gR0DEDqUL+eRU4+TvpzHvvmGD+N/YnifAtyNiJ4/Qx+ZzCFPoJwqEKYtLoqyomc176DPXeCKtP44MJbgmicc65ixXJ3cLnNZFYkguZPQLBlXrVFpUVsubKFY+lCPu3cXOg2qy+vFb+Gb1tfmxhfaopYKmba8WkMeGNAg7YjIwOKZWY8/lJ2CBEFxGUqn1oJG9vC0Wf13/f6DPUJBoPV7Gzj7bmJudz896bN5aA1RcalDLa+tJVT35+q9rFiqdjIO0+r0fKR/0ese2RdjdoilooJ6xeGezMToW208G2fb9m7oBqhpBsRTXl8qWuCewTz/PXna2ygJHOUMTNhJnd9f5d1G2bHTiPhvz6+LP/mGwbJZJV6dIEg/g2Uy1mxfLmRh+AOjZIHgcHAg8DfSiV71GreffVVu/jXBJj54ovMXbiQ/hIJg6UO/AhsA1YBwxwc6C+RMHfhwgphYGuDTvi7fNlqVTYIDTGmNK14Go0JkQgGbQF1MQSOFF6u6+pU2Sd5svN08lX/Izu7OW/seoNrWdfo26wvqRtSOf3Dae5ccadNh0UzleNv8zObKcosYvyP482q5lnXs/hp7E/0m9+PO96+o8J+XZ6/mnj8BXYK5NG9j1pUtu3Etvi08bHnUGjiqIvVHPz0IK1Gt8K3bfVMpIO6BjHj/Aw8Iz1N7v8vTDqfu/IcxQXCQqjUQYprkCsFqQU2PXYBRN0dRb+/57FnyTW0Ig0OUgcCXMpZPjk3g+EHjI6RSqFNGzh5EpxKDB5/58/De+9B69Zw550xBEfXY0caiJz4HG7uvUlY/zDcQgTrj52v7eTg4oM8f+P5evXcKUwv5PLmyzVeKNFx6PNDnD1UANyBszM0c2tGx4COhLiFGArlXYakv+HaCvDtD1eXgUO5HG0luRAwDFybQ5fPK5xDJ/xdv17x/EM/HIq6WK33UGvM44tWo+XIl0doO7EtnZ/oXK1jZY4y5inn1Yll3aSNhnwFQV0r5k4CyI7LRlOqwat5xRCrlSGbMY1rpw0WSQ4+XWDwLnAJr0FLgeMzBW/R0LsrLaYX/nDk8f2PV1rWVtFqtKjyVEhkEmROloXgqy2Xt1zGt62vfqxKP5/O1X+uosxRVstISDdXVWoFazg3RTlruIgpQv42h9uEn/wrgmjcYSG4tzVsT90DjsHC+HE7Hd62uE06oSm9KE0fOjI7T4pzuaY15vGlNoglYgI7BTZoG0pLIScHVD5mPP7u2A6qdIOnn1ghGAZIHPXf96pCffrVLr2cEVqtFplbFnkOyWRntzXad3DxQQ4sOlDvz/W6wCPcg0d2P2KRcWhViMQiOk/rXGlqi8oI7RXKQ38/ZHKfWCpmxKcj8GljZXW3Hmmq40tdI5FJ8Aj3qFUdbsHVtMi2Y8fG+C+PL+tWr+YFC2P/Ty4q4sMffuD8tWtCWFAz5XQegv1nz+aBSZPsYT9tnJmzZvHApEk8/NByXjy0Grk8B39/d+6aNIkfnnjC6te3VSvht617/DUEduGvrpHIIaCi4GRVfPuw5NI/nEjqQN9s8HHy4VrWNTKKMtDGabmx+4bNL56byvGXfiGdwvTCShf0fNr4MGbpGJr1Mx1eqTahPqtD5JBIIofYcCBiOxaReCSRbbO3UaosZUDb6lmAy5xkNv3ibQ0Ubgp9XpMBrw9g4BsDAci+kU3WtSwi7oio5OjGjUotpdAhAYBwj3Djcevg4+DWCtq+IoRZK9sXFSUIf+SWCX+58Ww8CJmZQg7AAwegZQstLz1XTFBE/eSqaghu7rvJ2gfXMvGXiUTfE82FC+DSPpK+c6VIFfU7jdn8zGbOrjlLQKcA/GP8qz7ADGfXnCXpbCb0vQMXF3hl6Pu8z/vGhZxCod088OoGri1hfCrIPQz7ncPgjr/NniMsTLiVkvOT+fXESYJ93Ogd2hsAv3ZWXMmta0TwctrLJvMTWXR4A4Xoyb6RzacRn9L92e6M/GxktY4tcPQh301ww5FL5MgdfMChFl5FXT4BiaPJXVcyr7DmzBrcFG4McXsWMC8ENGZ+u+83Uk6nMOP8jErLXf3nKj+O/JGx34yttpBcE5TZSlaPWk2HyR30nhFDPxzK8I+HW1xH7E+xnPzuJCq3kYAPhWoTwp9XJ+Hndry6wYBN4NvbePu/k4ToJKOq75FUHp3QlKvKxctXReJNBUk3Syi5nIhrkCsezT1qVX9jpjC9EKmjtMaevdYgs8zBs8Scx59IBA6+xn+PPAGAywZhU05OxXoLC4U8KmBdj7+CkgJezfGGdnB3Zh5gSGTaamwrXINdK83RaStIFVLC+lsv1/ewD+suukOP56ry57DTFMlPzkddrMY12BWxpGZBwPKS8si+kU1oL8vzwdqxY8c2yMnNxVLZxh9ISEoSPASrEEvLewjOmz+/ts2008AEBATQtdt8HBzn88or0Ldv3Z1L5/F386YQpt6h8qxedsphD/VZlxTEwda+cPP3uj2Pgx+p4qHkqPzIzgZvR28A0gvT6f1yb14teJWAjrZtTWEqx9/kbZOZemRqpcdJFVK6TOuCb5Tw0qsuMX4Q1SbU5+63d3N06dHqH2inyRLYOZDJ2yfT/sH2NTpema2sca4FW6coq4jkk8n60GflF+v3vruXHwb/QPLJ5IZqXq3IuJxB+skEXIoimODwJXP7zjXs1Grg5hpI2w/7J8Neg0dO2zJjeGWK8EJ9Ky+JE8J6Hd26gUJSivOXH/DLg+vrqysNQlj/MO759R6a9WlGVhbMnQtfb2vJHe8MxiXApeoKrEBJUQkHPj5Al2ldmLhmIn7RtRPOJm2cROjbwvPLxVwXHAOg5XRhMV8sMRb9bif7DFz8zGiTgwMEBUGS16/cu34kiw4s0u/TagWvJ1sIqSYSiXDyccLRy7RwVRXx++O5tNG6poG5ibkcX36czKuZZsu4h7nTc1ZPWoxsUa26S1WlFKTkoRYJq/EuchcheoS2FmFtA4eBXz+Tu65lXWP+zvksPbbUKOfXoc8P2dQcxznAGfewqqNreER40PGxjni1qJ4XZk0RSUSMXT6WDlMMXsLVXWTNu5VH4qFEivKFe6CgVBD+XOUWGPRJHSF4lPH4odVApw+g7Vyzh/HvJNh3b5XVezh4IBULBhhOvoL4dOtiLt8P+p4T356oun02zLa523jX5V0K0xsujH9GWRTjDkUvsPCOhXQM6GjYefFzSDGfpzOozFE5MbHivvR04beLCzjWbOg1iYvcBUex8NCLzzae04UPCKfXzF41HusbEwVpBRXeORuKv2b+xZGvq85Hb+e/xZ539vBJ2CcUpBRUXdgM2+ds59ve31octtqOHTu2g7ubG5auvKQAouJiHq6Gh+Cfq1fXuG12Ghe6tXSzaxpWwtsbvLxAo4Fr1+r2XE0Nu/BnZcTicv9SsRzS/oXrP9T5eT08tMglReTkgLeTIPxlFGYglohtPiGtVms61CdgcRJ4TamGxMOJfN7yc27uu6nfrhucquvxp9Vq2f/hfs6uOWtR+cwrmSzrsowjX9lfvJoyMkcZEXdEmA3XWRUbn9rIsi7LKCksMdouFotp3bq18fjSxLi+/TpLOy3l4oaLFfa1u68dE9dM1Id5tDUOfnKQa6+uwCXfl2GeTzO5w2TDTpEY7smF3qugNE8I36gpBYRwBiIRuNx4gItPpPJB57Xk5YGTE7z6KoybICXHrxWlvrZt2FEVbsFutJ3YFtcgVy5eNIQ2q4nBRk258OcF/nnxH1JOpxB9b3SNvc90OHg4oFIIIoXFk+TCRLj5K5SUdfz8R3B2ofD53Adw7HnIu2J0SEQEOKnCAYjLNuQPPfLVEd5ze4+b/95s9ONLqaqUlNiUGi+ub5+7nT8f+dOqbbp15BYbpm4g/t94s2VEIhHDFw2n5ciW1ao7+WQynis+JvjKGaBM+Du7AH6SQs6Fmjdaoxbye91GqJvBo1gf6lMJhz87zNGvbEf4G/npSB76y3Q4u/L4tPZh3IpxhA8Mr/tGIeR77Px4ZyOP9YK0Aq78fYW8W5YNYr1f7M3srDlkyfzQoiW/bAww8vjb3AF2VZKXL/86FJTNf0ViCJ8E4febL69KF37KwgGbQyQS4eMkuITJPQXhL1frxojPRhA1PqrRjy+1oVnfZnR8tCOO3g0nVOmEv65OE5nbby7NvcpCt2o1cHK24RlRnuRtEPsWocHCXCMhoWIRXZhP3+pFrbcIb4UwZ0nOs01jLkv4pus3rOi1wmr1Zcdls3rMao6vqF6uW3WJmiNfHuHq31fNltk0YxNfx3xd2yY2CE15fKlrIgdH0uOFHrXK5Rt9XzRDPxpqxVbZsdN4+K+PL+MmTeJ/FrpU/eDoiFShqJaHYI6pcAN2bBJTTjp1RVMI99kQY8p/cxSrLxwDYfQ56P1jnZ9qqncgr/Ufa+Txl1GUQUlhCVf+ukJKbEqdt6GuUCqFhV4QBpOizCJWDlhJ/AHzC27lOb78OAudF3LhzwuUFJRw6+gt1j64luL8Yv2Ca3UXkEUiEc9fe547l99pUXmpo5TC9EJKlaXVO5Edm0FdrCbtfJo+Z1ZNiL4vmjveuQNNaUXPDrm84UJJ1Qc+UT4MfGsggZ0r5suJuCOC6Huja/Vyag0SDyey8/Wd1b7GbSe2xXX8ENRyR5xMdUEkBpkr9P0FBu8AnfeEE4SHg0zjRtoNX44fF8Smjh2FHIBBQXC9091kx5j25GkqaDWG/7cumbOkRMmaO3/k3w//rZc2RI2PYuKaibR/SPDmzbmZw8FPD9a4vtQzqeQmCVbWLi7QbHEzWn/RmqS8ch6/x1+CTe0MQl/cz4IHTmbZwt+Vb+D6/4TPbWfDkD3gYhxSOjQUHIuFcGNxOQbhzz/Gnw5TOuDkLdyQjXl8yb6RzZL2Szj4Sc3+3/3m9WPsskoEkRoQ2ieUSZsmETHY+uGHnbydyGzRjSxv4Zq4yF3ArY2Qn8+hhqvwmcfhF2fBA+g2Qt0F4S9XlUuxWPAkUyrhvnX3c/+6SoQhOxZh6nkR/288P474kes7TSTgNENhme6tFZUwqsVoBoQNMBb+XCLB2Uy4tZzzsD4SLnwi/K2xYC466C/heWSB8aAu3KfYVcgzl5Ero8ezPQjuHgw07vGlNnSc0pFx345rUANLnfDn7W1i55C90PG9itsT1kHsG4QHCtcr3sTrVF0Kf35OwtJgaqGx8Be/P56lnZdy/o/z1j9pPdP2nra0ndi26oIWonBTcG3rNbKuZVXrOIlMwitZrzD669Fmy4glYiRyCRp1LbzKG5CmOr7UNW3uasOIxSOQyC0zojZFy1Et6f1i72rlqrVjx5b4L48vT0ydys6SEg5VUe4QsKu4mAB//2p5CLq7Vx2lw07jp7yTTl17/IEh3KduTciOZdiFPyuj0dw2aXaPAlndfwNSZHdyIb23sfBXmEFhRiE/jvyREytsN9yOLr+fXA4KhbD4nng4kYyLGRYd79nck8ghkbQe15rnbzyPWCYmdnUs8fvja5Xjz8nHyWLPLrdgN16Ie4Fes3pV/0R2bIKU0yl81fYr9r23r8Z1RN0dRb9X++nz3OnQaDTExsZWHF+aEH7Rfgx4bQA+rc0nkykvADUEsT/FsuftPaScqp4hRcSgCER9+5DrdJHrmt0k5paLq5V3BVJ2QUk+iGXCNlUmaASvz6goYdP583C8TO/pVJbGSRem69atGnbIRljecznLuiwD4EqZQ5tGIiPlUBw5cfVjLShVSI3E5x3zd/D3C3+TcMiEq0QVaEo1fB3zNfm/bgFA7lREfG48lzIu4SQrpwyL5YAWpGVziOAx0OsHYV4BMPI4DNwkfPZoJ4RyFBlP63x9DcJfemE6BcWC2BjWP4y7Vt6Ff3v/Rj++OHo6MuD/BtQ4x2fzYc2JGh9l1TY5+zrTclRL3IIrN21MPpnM1zFfVyvkoUekF9fajCItSBgPXOQugndWv9+FnGw1anAY+N8BzhXzHbvIXfBw8AAgTSUoAFoteLTwxSPco2bnawD2vb+P2J9iqyynLlHzx+Q/aiwkV5ff7v2NpZ2XGgmAgZ0DGbN0DCE9QiyqI25PHOfXCaatCqmcDZPWs+uRXTjLnQ2F+v8B3b4yXYFbG2j9AgSV5ZrcOQz+6l65N5/I8ldEP2c/3BRuOLgK6qRONIL/xvylIcnIgFJxASnOWzmdctqwQyQG767g1aXiQW1mwshTBIQJ863U1Ip5PetS+At0FYS/DNVtS4QiIeR9aZHtG0kO+2gYfedYL9GNo6cjc/PnMvidwdU+Vu4sxzXQfFjgkZ+NZNqxaTXO89aQ2McXO3bs1BX/9fElICCA995/n7EKhVnx7xAwVqHgvQ8+YMLkydXyELxr0iSrtdVOw1FcDCVlwcrsHn+W0RBjiu3N8OyY5GbAMn6MfUsQ/pwMHn8u/i6MWTaG9g/XLOdYY0An/OlEuhYjWvDMxWeM8qVURsSgCCZtmkRIjxDkznLaP9SeF26+QPNhzWvs8Zcdl03G5QybtY60Y30cPBzoObMnzfpUXFi1Uzu0Wi2fhH/CmvFrGqwNGZcz6PxEZ6Ydn4Z/B/9qH19YCDf8Puf16wP55vg3hh03VsP2QZBfpmiplbBrNOwaBZoSfZ6/VeeW80PpWFLcN9C5s7AtKAjkhdm4/LOWk6vO1LKHjZfAzoEEdg1EqzVYd2nFEjr+OpdRX4yq8/Pf2HWDnHhjgfGOBXfwyO5HapQ/V6PWMOD/BlDcXBCj1HLBgl8ikhh78HRcCKPPGjxu3FpDxMPgUJZfUOps7OGnUUPWKSg1hMT09gaZ2h25RrCqLO/1Zys4+zkz8I2BNRb+6gJLvfed/ZxR5igpVVm+kF1Qlm7HJ3cIl2Zc46cJP9WkicYovGHQZgi7z+RuXbjPNJVByM7PKiE3MbdWXuz1yZ639nD6f6erLCeWCsZfcXvq57vg6OOIS4CLkVeYezN3ukzrYnGewT1v72Hr9LVADRPZi0TQZTEEloVkc2kObq2q9ua79gOc/7jK6v9+6G9y5uQwPkrIUZuWBmsfXMuyrstq0FjbIOV0CquGr+LqP+ZDKNYHGRlQoLjM4oxhDF813LCjOMdkaF9AeG54tsfdU65/t7o9z59O+PMxb4tVY0I8hedmriaZ4nKpwUJ7hfL8teeJmRRj/ZM2ASxNb1Ge9IvpJJ9KNhlJxM5/m5/G/sT2edtrVUf6hXS+6f5NtUPQ2rFjxzaY+eKLzF24kP4SCcMdHPgR2AasAoY5ONBfImHuwoXMnDWr2h6Cjz/xRJ23307do1tHl0rR54qvS3Qef8nJ9Zv2xdaxC39NBA8P4Xd2NoxpNYatD2/lvSHvIZFL6DK1C0FdghqyebXCVH4/92buNQ6t4+DugHuocW6l6nr8HVh0gC9afWFxfhaAS5sucXSp7eTLsVM9vFp4Mfzj4YT1D6txHcpsJSsHrmTXm7us1zAbYd2j6/jpTtML3CKRiMDOgXi3rqG3ixX47b7f+HHEjwR0DKj22LNqxCpUP/5GiSQbAE+Hcp7CwWOhy2fg0kL4WyQBzw6Clb5Ypvf4u6E6Ror7RkoD9+NXpvu4uYHCRYp3YixX95ULEdnEGLNkDGOXjiU11XiCl55e9+HVSlWl/D7pd1b2X2kkgLg3cyesfxhShbTadUoVUga+MZCiyGgAlKJMADwdPS27tzQlUJggiHzqcm4al7+ELR0hzeB1rPPW0If7LMvzp8xWsu7RdZz4znajAdzOjh3wzjtQVGS8/cjXR/jQ90OST1kvn9TKASv5MurLKsu5Brky8+ZMuk3vZnHdsb+cJ/LYr3gUF9HSJ4JIz0g49RrEvlWbJleKLtznrfx4pGW39N/PbmRxyGKb8b6ZenQqIz8bWWU5kUjE3Ly53Pv7vfXQKhjz9Rge3PxgreroN78f3d8WwtU6mkonV5gAp+ZD2oGqK1OroMc3Ql7ZqrixCmLfMIQbNoNELAgSuvEmPR1kTrIK0QuaEjk3c4jbG4cqT1V14TokIwOKZULITj9nP8OOs+/AGifIv1bxIK1WyCdckktImdPp7Xn+dMKfnx9WJ8RDEP5UsmSys61ff0OTdDyJX+/51erGBdlx2RxbdqyCIVJl/PvBvyztuBRljhkRGEg9m8qBxQfIjsu2Qivt2AJarZak40lkXs6sVT0ShYT8pHyK84urLmzHjh2bZOasWcQlJNB/3jxeco3iPkUQi1pGMWDePOISEpg5axZQfQ/BgIDqG8/aaXyUD/NZH5HvnZ0hWMgkYNNef/VN9Ves7DRKAtQbeKHnb/x6+QOauTejmXvT8TrSDSbueQl8GvE7g98bTLv72tWqzlJlKRfXXyRf7QREVFv4azGyBVIHaZVhvspz+PPD3Nx3k65Pdq3eyez8Z5C7ykm/kF4jjzJbpyiziML0QrP771tr2lOlPtBqtbR/uD3qYjVajZa4fXF4RnrqDQiqQq1Soy7VUiIVPLt0YfUA8Ook/OgQy6Db1/o/fX3BywtciwQL+FJvQyg7kQj8I104MfwVRjxaEzcQ2+L2WO4pp5I58V0SHR7ugFhaN3ZMUoWUcd+NQ5mlrCDKaTVaUs+m4hnpidy5+jkglGXrcEqRcF8YCcIaNZxdCL69IGCIYfuBKZCwHqJegtPzYdgh8Oku7AsYAm1fASfD81/nreGgDCPH4bTe408kEXFy5UkQQfvJjTsiQNyeOHa+tpP+r/cncnCkyTIXL8Knn4JGA0eOQP/+hn1O3k74tvW1ah6u0L6hqIvVVquvPKln0/FKOkdhtwGGjTd+BLknxLxe84rj10LcGui+FOQecO4DEEkhapbe4y8+Jx4HB8EYKqB3JJ5Bjg0eYtlSfKMsj0koc5TVYUuqRpWrYlnXZbQe15phHw6rsnz4gHByPIBtkOG2A8d3RtMtqBt7Ht0jFMi7Igg9DgHCmGEKrQb+7lnm/bnFsoZ2+QwUXkIOWgvw9haeS8XF0P+jsbi7g1pdN9+ThqbVmFa8mv9qg38/MjJAJRWEP12uRQA8OkCzieBkIu9j1gn4qwu0X0BIyDzOn6+Y568uQ332CulJVP7TOOT2Iz3dIC5qNVpO/XAKZz8hlLKtknU9i3O/nyP6vmir1ptwMIGNT27k7lV30/5By57b7e5vh2ekpz6fryluHbnFP7P+wbulNx5hHlZqrZ3GjEgkYlbirFp79HtGeDIzfqaVWmV7FGUVIZaKOfPTGW7susH4H8c3aM5XO3bqioCAAObMnc/BQ/MBWL3a2ClDx8wXXwSRiP6zZzNQJmOyUok/kAz84ODA7pIS3ivzELTTNDDlpFPXtGwpRKq4fBm6mIhob6ciduHPyojFDeNE6aY5y+CIH9h06RlKS/31FtsghNtJPJzIs5efbZC21RZdqE8neSkKNwUyp9ov2BQXFLP2wbWEDowEhwjy8gQDWEvnai1HtqTlyOq9lA5eOFgQDrRa+6SwCbLrzV2knExh4i8TK4TjyVZms+3aNia2nVhpHWKJmJeSX6q4XSwmJiamwcaX+uD+dfc3dBPMIhKJ6DVTWEy9tv0a/xvyPwa9PYj+8/tXcaTAlJ1TePJJKJEsAgTPripOKPy+9Tei1J20bbuQyycF4S9VZBzKLigIrlxxaLJ5/lLPpnJ0yVE6TunI5cuC57qrqzDJzNtzgvWfHKb50Oa4hdRdUPkWw1uY3H5s2TE2Td/EAxseoNWYVhbXl3g4ka2ztyKS9wFFSwq1gsW1l2O5sH+qNIh9HVpONxb+3NpAQD54d4N2r4F7G8M+97bQ8T2jczk6gpMThKXOYM6d9zCseR8A5C5y5uTOQe4iCJaNeXxRZitJO59m1qK8qAgWLRJEPxAWwssTfW800fdadwF2+KLhVRcqI/1COqd+OEXMgzH4RVftPhM+uR8rLvWFyC3M2baH/mH9GTX6LKjNG0ZYRO4FuPkrtJkFp1+H5H+EeyZqFrP7zObZ7s8S5hHGsz8Iwl/I6A60bGlZSPWGRlOqoSizCIW7wiIv3NSzqSizlDTrW7dGcgcWHyA/OZ+uT3XFM8Iw7sucZEgdpNUyGNAZCmgVuShLlZRqynlieveA0edBUUlcRpEYXFtA3E9w4hXosMCQV9Yc5ceXSth5fScf7P+AaN9oPD0/IjNTEI7c3Zv2/EUkFiESN9x8Xqst8/jzElQ6I4+/iAeFH1M4NYPIR8EjhtAyXbC8x59GYxhHKwh/1XlZMsPgyMHcJRtMbJaQX1AX0lwkFrHp6U2EDwi3aeGv7YS2zFfNt3q94QPDuX/9/YT2MiHmmqH50OY0H9q80jItRrTg0X2PWvR8amw05fGlPrCvR9ScUlUpP435iVJVKT6tfTj7y1lGfTEKRy9Tbvl2bBH7+GJMYbnXEJPRJ8qYOWsWD0yaxIrly1n4zWqSU3JwcXbngSfG0kmrZfU33/D1okW4u7kxbtIknpg61e79Z8M0hPDXqhXs2mW7Hn8NMabYR7EmgrzdDO79LY/Lmd1IySzim2Pf8P6+99FqtTgHOOMR7mEzeVpuRzeYeHQM56lTT9F6bOta1+nk7cQ9v97DncvvBISX3NvDg1WH0lKYNw+WVZLOJLBzICE9Q+yT7CZK2tk04vbGVRD90gvTafdVO+777T4OJhyscf3Fxf/tMCo3991ky3Nb6jUUUVFmEVqN1iiXZ1j/MAa+OZCoCVHVqquwEH2oT73Hn6YU1gbAMTNWb7c2w7n36dw6Hrcyj78UZTzZymx9kaAgkKryub7hDLmJudVqky2QciqFI18cITsumytlaRB79hR+p4d25P719+PgWTfejke+PkL6xXSz+yOHRNLrxV54RHhUq96irCLSzqahLhBCxJkU/hTeMPKEINKUJ3ou9PsdAodB+7dAZkbwLPe89/EBv9zhDPR8WAgbibDYo3BV6J9HjXl8aX1na15OfZk240yLEN98A0nlIt2mm79kDULOzRz2vbuP69uvW1Q+Px8QiUh13sX7/77P9mvbQeoo3BO1odWzcG8BiOWQfgBaTIMhgsdYpGck0X7RuMhd9PkZVA0bwbBaZF7N5CP/j9j91m6Lym95Zgs/j/u5TtpS3gNMXazm4OKDFfJriaVipp+ezsD/G2hRnYuCFnHijT+FP+TCOO+qKPeGLXUURDqHKhKy9VktiIRxq6sW/fSdUMHV7+CWeS/BLGUWf135i/3x+/VC0fVDqex7bx+ZVzMb9fhSU86sOUPCwYSqC1aBRqvhTOoZNNrq52DLz4eSEjMef5Xh4AM9v4WQO02G+szOFt5rxGLw1OnVRSmwfTDEvlntdppC5+Wn8yzUcf+6+xnywZCKB9gYEpmkRjn5KsPF34XWY1vj5GPee688luahdwlwoVmfZjh42GbkiKY4vtQ1uQm5nP3lLLkJtX9vuLz5Mie+bTph4y1FIpcQMSSC8EHhjPpyFPMK59lFvyaIfXwxoBP+5HKMnExMERAQwLz589mw8xy9hybi0+xxFi9axKnPP+eFCxf4+tYtXrhwgb0LFxIWEsLij6vOJ22ncdIQwl94uPD79vzUdsxjF/6sjEZT/Rc3ayBWuKJwFhLWZWVpmLZxGnO2zyG/OJ/hi4bz8NaHbVZwqqvBpM1dbfAKc0VeZnBtabjP5JPJfNXuK87+ela/LT4eTp+GrVsrP1ZdrEZd0jTDHv3XueeXe3g59WWjbRqtBm9HbwZFDEKj1TDlzykUllTutZFyOoVDnx8yyseh0Wi4ePFig40vdU1BWgFHvjpC6tlUs2XSzqdx+PPDpJ1LM1vGWvw55U/WPb6Or2O+Zvdbu1kUuIgdr+0AhMWcAa8PsDisXOrZVA5+cpCStOyKoT5L88CttRDCzxRtZsKYi3ToFYK7wgNPsWDlfSb1jL5IUBC4ZsSR993vXN9hmbBgS0TfG83MhJk0H9ZCL/z1Kotkl0wgLUe3rlGYzarIuZnDlme2sOUZ84vdXi28GPbRsGpbybcY3oKX014mPVAIWe3u6EwH/w609Crn4SCWgWdHwUOnOsT/AX82g3SDkYFuIf72Bda082nE74+36fHlyBHhuSsSQR/BmbGC8FeQWsDut3ZzbbuJXFc1IOl4EuufWE/iEcveNkL7hPLkySfp9rRlef6Sj9/CJSMOrUyYlHjKHSH9kLDwXhtkroJA5NUJ7k6ATh+ZFBMdytZ+U07cYs3da2xiXJG7yOk6vSshPUMsKt/9ue4Med/64kJuQi5LOy0lbq8QUrfz452ZlTAL75Y1F221Gi2+bX2ReAoiv1YmTIrdFOVEf1WGcH9YIh4N/ReG7re8AepCOPoMnHvfbBGdp1laYZp+vEk8nsL2udtJOp5ks+OLObQaLeseXceuN3bVuq7fz/1OhyUduPOnO1GWms/DZgqdV57WURjcfZ3L/vlFSbBrDNz8vco6dB5/iYkGr2nds8LbGyQ67UrhA/nXoTirWm00hVarxcErgzyHsySnGN8XzYc2xz/GtsPd3zp6i5v/3qwzg9uCtAKLym19eSufRn7K1a1XqyyrKdVQUlhS26bVO7Y8f2lI4vfH89t9vxG/P77qwlVw4OMD/D3zbyu0yrYQiUQMenMQQz8YioOHAxK5dYV+Ow2PfXwxRif8OTtbfkxQEFy/uohrp+exR63mL6WSB4Fo4DqQqVQSqFbz5ksvMWrECJKTrZeP3U790BDCn+5cBZZNhxodDTGm2IW/poK6mHZBJwhwuUpxgRMKiWCynVGUUcWBjR9dqE/VnkOc+t8pq9atzFHiWSqIDZYKf0VZRZQqS43C++jbqDJytDDi9KrTLFAs4MpfV2rT5CaDVqu1+OW1oTjy1RF+mfCLxe28PeTT9mvbcXvPjaS8JIJdg7mUcYm52+ZWWsflzZf567m/SL/QyNxW6pC0s2lsnrG50gXm6HuimRk/k+bDKg9ZVFuK84u58OcFijKKcAt1QywT49vWt4KFtaZUY5GHXdzuOP6e+Tfi7AxKyzz+9Lnc5J4wZDfEvGb6YJdwcGuFn7+En3+GXpGC119siiHPX1AQ5HmFkdbvLiIGRVS7v40dsVSMW7Ab6blyCgsFK8MOHQShR60WPBPqIseSezN3Ht33KMMXVx3SUavVVrsNWq3Bo+re6Hs5+dRJPh35qaFASS6U5Jk+cP9kWC2CS19W3C/3FASeEsO96e0NalER2+K28P3J7/XbNz21qc68nqxJ6plUzv56FmV2xUXxXbuE32PGwMCBwufbQ32qclXsemMX17ZaR/hLOZ3CiRUnKEix7Lkgd5YT0CHA4jyUcd/toOWR1ailwqQkSFQM//SES5/XuM16sk5C2r8GT9Gcc6BWkl+cz4I9C3hm8zMoHIR7uSBLxcX1F8m6VvuF/rrGLdiN0V+NtjgiRNTdUXR+orPV25F+MZ2sa1nkJwnXzsnHCWc/0ysksatjOfL1kSrrFIlFTN42GZ977wBALRW+227ycsLfmQXwRwAUWuCBJpaAs+WhApF7woD10H+d2SI6T7PUglS98Fcc2pypR6YSOdR0Xk5bRqvVct/a++g7t2+t67qz9Z2MaTWGTZc3Me7ncVUah5VHL/w5Ce8x+lCfBXFCKN/K7ofT/wcHH8XPD2QywXMwtcz2ymR+v6IEGLABOldila9Rm38JKkeJpoTp133Y3a4d8emZFfbXVf7U+mLH/B2sGr6qTgxutzy3hY/8Pqo0J7aOgE4B+LTxqXJuWJBWwNuyt/nnpX+s1Uw7jZzQ3qFMXDOR0N7VeBaYYfDCwTy45UGbjSxVEwrSCvRCuUgkQqPWkHAwwWKDMDt2bBGdyOJkmdM5ANnZyVy88Ap/aVT0KNu2GAgD9gEvACuAL7Va8v/+2+79Z4M0hPCnuwcLa5kF47+EPcdfU6E4i1c6dGaj4wxycr7Ax8mHxLxEMgozcEp04uwvZ2l3Xzt829ZBlvY6Rieqpf62h2PnvOnwsPVyzixpv4TAfAUpvafrB62qiBgUwXNXnjPapjtWqxVenuUmHFC8WnjR7v52OPtWw0ymiaLVavlh8A8UpBQw/cz0RuuNmno2lfNrz+PWzI0Ri0eYLVdSWMKFPy8Q2DkQnzaGUFuXMi6RX5yPs9yZFXeuYMSPI/jiyBe8NuA1fJxMh+SKvjea4O7BNvldrSkBHQN4eOvDeLX0MlvGwcOhXsIQyV3kzM6YjSpXhYOHAyKxiP7zjHP5aTVaPgn7hMAugTyw/oFK64u+NxqXSD/mfOJL2/jF3Dsly+y1N4kqE4qzELk2J8Y/hj0395BXbBisgoKg1MGFOIcOKJrgLZN2Lg2Zk4zLNzwAaN5cGF+9vSEnPpclEV/TdWonhn00zCrnK1WWUqosxcHDwaI8OnF74vhj8h8M/XAo0fdYlkcu6XgSqRezEJe2QCOV60MrGnHhUyHH34jjgoeWDpEIbvyv7LOJKZz/QBh91miTry+USvL4IHEUokQRD8Q8gFwip9sz3VBmVc/DpCE49/s5dv/fbqbHTq8wBlwvsxXo3FnIJwYVPRvdm7nz5MknrZYHssOUDrS+szVSR8un0KXKUpJPJuPf3r/KPMXuw3pwRtOWUrEgCokdfKDj++DTq1btBmBL2b30gAYufCLcYyNPIHZpxWs7BQOE2Q5vA544tQnjtZLXGjSHma0ROTiS5649Z9E87/AXh8m+kU236ZZ5gurC0ZdKc6H4No8/376gUVFnD4GAwZXu1nma5apycfdSAQqyVE4EdXVCrbZtEccUYomYFiOq6Y1tBoVUwcyeM9l+bTv/XP2HMavHsPnBzThIq57v6IS/nvKpTBvcm14hZWOET0+4twi0lXhwZRyEzGOIxRAcDDduCNFLAgLMCH+xb8G1b2FiFsg9Ktan1cKvruA/GAZuqLTdcokcD7k32cUZxGUkA4Y50brH13Hy25PMU86zKF9nY6TnCz1pe0/bOqk7tE8oJYUllKpKqyzb4eEOtH+ofZXvWA4eDkTfG01gl0BrNdNOI8ctxK3S3Mfn0s4R6BJYdU5yILh7sDWbZhPsWbCH48uOM+P8DDzCPQBYOWAlkUMimbRpUsM2zo6dOkInslRH+Fv+zTf0l8joUSrMBRcD7wJ7QC8E6ngQOKRWM/bVVwEhV6Cdxk9DCH86r9OSEuFHZmH2gv8ydo+/poLCh3/z3uJgwl1kZ4O3kxBWKKMog4xLGex5aw9JJ5Iqr6ORohtMen/7OGOWjrFq3T1n9UTTuQtotRYLf6bILef4ozSzjhrSM4QJP02wOBRVU2bbnG2knU0jbGBYow4tM/rL0QR1C+LsmrOVWjJmXc9i7YNrOf3jaaPtlzMvA9DKqxXDWwwnyicKjVbD4cTDZuvyjPQk4o4IFK7GaoBE0nRDiDh4OBA5JBKPMI9Ky2XHZVeac81aiKViHL0czS54i8QiWo5uSUivqr/LTj5OeHUKQyNzonXu07w2YB6OsrIcEIkb4exCKM42X8HGNnDgYQDeGPAGOXNymN1ntn63i4th8rP385NNzqP4j8l/sHLASi4LXyValK21+vhAqdwJlxYB1c6xZw6NWsNv9/3GyoErKcqyLOmrW4gbErkETYnlIRtOfn+SPyf9iqREeFiYFP4820PEFHBuVnHfXYlwfwm0fNKi8/n4gLzUF6nWES1a4nOE0E7R90TTZVoXoHGPL9H3RjNxzUTcm7kbbS8uNuSliogQ+gmQlSXkp9IhkUsI6BCAk3c13lQRvDZuHbtVYbtIJMLRyxGZo+VvGYe/OMyKXisssgaXtW1JerPOlIiESYnUKQjazga/fpY33hw9VkCnDwUB2X8ARM8DuRdOMie8HYV5Y5FMuD9UJWKbEf1u7L7Br/f+yq2jFa+XKY4tO8YXrb+w2vPk2LJj5KcIXn6WGneNWTqGh/5+qMpyOfE57Ji/g5yzws1eKhbuC6Mcf80mQLevhFCudUXuJUjeYXKXh4MHEpEwhsjchf9pWppgFFWUWdSox5eaUKoqtap3y8Dwgfz10F+4yl3ZeWMnf5z/w6LjdMJfb++xzOk7hxj/GMNOsQQklYiH/dfBeMHF7/Y8fyaFv5Bx0HYuJG+F3XcKOYpvRySFzKMWtd3fOQCApLxkIyfBoC5BtLu/HWqV7QrGLUa0oPPj1vcoBmh3XzvuXH4nbsHmDVn+/eBfvUegJYaVEpmEiWsm1lmb65qmNr40Bh74/QG8PvDiryt/WVReXaK2eU/d6hDUNYg2d7fBPUyYl4olYkZ+PpIeL9wuZdixdezjiwGdx191Qn2uW72ax8rCmCcDc4ANVBT9dPQANqhUzJk92x7200ZoCOHPsdzrjt3rzzLswp+VabCHg1jCeclrnEoZIgh/ZQs46YXpRAyKYHrsdNqMa9MwbaslusEksK0Xfu2ql0upKno+3xPHAd1BJLI41GfsT7EVQo5aIvzZEVCXqDn82WGCugUx+svRdZKfy5rc/+f9vBD3QqUvz27Bbtzz6z0VrCd1wl9LbyF3V/fg7gAcSjhU6Tm1Gi2qXJX+b4lEQkxMTJOdfJYUlVi0iLasyzI2TK3ckry2nPjuBEnHqzaSGLtsLP3mVr0QX5heSF6uIApVsJCL/x1Ozau8gjYzIUywHnWUOSIWGT+2RSLB60+kLuXIu9vYPGNzk8oj2vWprvR6qZde+GtZlgbPxwe0Eilhr0+h+4zuVjmXWCImpFcI/jH+OLhb5l3qGenJs5eeJWZSTNWFy+j8eGeGLJlAqdwJuRwe+P1+Wn/Rmk2XNhkKhYyDXitN5mDDKQjElXhCZJ6AI88IC/UI/ysRIlzUgogYlxNnVLyxjy++Ub5E3xuNws1YIY2LE5xM3NzAyws8PIR8VFqtEAK2PMpsJfnJFj7kEbwy93+0X+8RWT6Ua+LhxGoLRpFDIxm0YFCli7U6dJ5dJQjtdZG7VOtcldL8MYh6Sfjs1x86LNCLy6HugoerTvhTKuHGrhskHLIgfGQDk3U1i3O/nqMww8K3P5EgCFtjofLW0VtsfHIjfz1n2QKpDv8Yf4tymWVeyWTvO3spuCyImkGOkfQP608LL+t4nFnMoSdg7wSToRzFIrHe60/rLIhJaala3nV9l01PbmrU40tN+OfFf3jf830KUk2H+/3tN7jnHnj3XTh2zJA773aWHl1KrxW9+PbEt/Rt1pd7o+8F4EqmZQY8OuHP+/bHRPI2IaRvZUgUwgQCS4W/O6HjQiE8cPI2yDH2LEckgntzYbxlRqZB7oLwl6dNNnr/6vZ0Nyb8NKHCeG/HMooyi9j2yrb/TM61xj5/aaz8dv9vfBz8sckw9bmqXH1KgQ7+VUdZuvDnBRYoFnD+j/NWb2djpcPDHZiweoLR2kCXaV1oPrRu01HYqV/s44sxuveT6nj85eTmElD2eTkwCPOin44ewEC5nBXLl1e7jXbqn4YQ/sRiQ056WxT+GmJMsQt/VqYh45t7eAi/c3LKefwVZuDg4YBfOz/kLo1bYDFHbi5IiouQKPPr5P/r4gJotRYLf/++/y+7/293hTbqUKkwSamqlI3TN1qU06UpI5FJeDntZUZ/PRqoKHI1FtTFamJ/ikWVp0Iiq3xwdvBwoO3EthUW8S5lCAvvrbxbAYLw5+3ojZbK7+OP/D/ilwm/6P/WarXk5uY22fwJ6x5Zx0KnhVUKVn3n9KXjIx3rrB3KHCUbntjA3oV7rVbn0s5L2Tx+BcWSDLI9dnEh/YJhZ6dFMPyI6bBZOqLnQutnKj1HUJAggoW/M5UJP02o8n61JTo/0Znuz/Tgxg3h7+Zl79S6BcnbwzrWlr5z+nLXD3dV29NJq9WyovcKdv3frirL+rf3p9modmglUhQKuJZ1jUsZl9BorZToufAmXP4S0oT7WOcJJ1cKq7u38gQBIXZ1LF+2/ZJbx2/Z5PiiuyciIoQ1Z5HIsACefpsut7TzUn4c9aPFdTv5OtHj+R44egkmhf+89A+HPhcMNtY+uJZfJ/5arbYGdAig/7z+eLUwH85YR9obX9Ls9EZUZcJfq+x/4e8ekFO3i2qhboLwVyAVVv+VSvj5rp/ZOX9nnZ7XGnR6rBOvlb5G5BDL8sl1mdqF6bHTLRLeqiKoaxB3r7qbkZ+PrNZxWo0WZbYSjbry731IzxCePvc02uh2AEwMnsXuR3YzucNkQ6H9k+HknGq3vVq0mQldPwMz45S3ozduCjccXIU38MwsEZ2ndSF8ULhNji+V4d3Km5AeIRVy/4Kgi65bJ3x/9u+H//s/mDHDtFHg9uvbOZhwkKQ8QSwLcRPG6PjceIvakZEBGlEJNyT/cDL5pOF/fOwFODCl8oOVqXDrbyhKJrQsqnV82WlNCn86Wj8H41PA8zZBIP8GlFhuXBHkKiwFqmQpVn+ONyT5Kfl8HPIx+97bV2fnOLnyJEs7LUWZU/GmkjpKeWDjA3SbYVkIYR373tvH5mc2W6uJ9UZTfz+qKzybe+Lfwd/kXPdQwiG0aInwiMBF7kJCbuXGPx7hHkSNj8I1qB5XfRuQqu41+73YdLCPL8bUJMefu5sbOr+9dcDDJsokAwuAbkCrst9uRUX8+sMPNW+snXqjIYQ/MHie2qLw1xBjil34szIac2ad9UBX6ct8NqID2dlavcdfRlEGWq2Wosyialm7NxZKSoSXZd+bx/ix4yISD1k/aXLJmj+I2rvM4lCfd//vbu7+391G23JyDJ/NefxJ5BKOf3Ocq39drWFLmw5yFznuoe4UZhTyRZsvGmVC+ZybOaydtJYTK06QeCSRG7tumC1ravAuUZdwPUtIQNXSS3BTmtp5Kmkvp/HWoLcqPXfbe9rSrJ8hxJ9Go+HatWsNOr7UJYFdA2lzV5sqBaveL/Wm02OdKi1TG2SOMh7e9jC9X+5dZdncxFx+f+B3Tq48WWm5qPFR+PZvQ5bLATZ4D+LhP8pNeRVe4N21Wm2cvXU2UV9GGYXfCQoSfqcVuxPcPZj85HyLvBZthbw8g5VhQJnZoE7MStkWy8bpG606gapJzlFllpLCtEIK0kx7gFQoX/accHCALGUWgHEulQOPwJkF1W4HAAFD4M6rEPkYYPhfSYqFeUFmUaawoaybJYUljXp8Wff4Oj70+7CCYYAuv19EhGGbOeGv46MdK81nczvuoe4MfX8ogZ0D0Wq0XN58mdhVsWjUGvrN62fRGFETtFotWrkcrUTKGy03cOLJE0S4+kHRLRBZWdBXpsKOYUI+SQzCX66oLNSnCkZ+NpJeL1kht2A9IJaIEUtMv9bUdZq59g+2x9mvevmbt7+6nfc93yf7enal5WSOMnyjfFFJhJUWB1POyKm7IcOyEIs1JvRuiHhYCCFpgtPTT5MzJ4ehrfvoHMno995ouj7dtVGPLzWhx3M9eOjvh0wuml+6JHgcOznB2LFCKOeEBDhzxricVqtl703BOKNfmBA9QPcdTC+0zKM4IwNU0mRePT+c7t+U83zv9KGQF7QyUnbCrhGQuscyj79/+ggRCuSeIDOxunPoCSHH38UvoKjq+UeAi074SyY11bA99UwqG5/ayM1/b1ZZR2OkpLAEt2A3ZM51l3CmuKCYvKQ8Mq9kVtgnc5TRanSraqeVuL79Oqf/d7rqgo2Mpv5+VFcMfmcwD25+0OS+/fH7AbiefR3fD3156Z+XKq0roGMA9/52L2H9wqzezsbIt72/Zf0T6ytsv7r1KosCF3Hut3MN0Co7dYF9fDFGJ7BUJ9TnuEmT+KFs4poDeu8/HYuBMGAf8ALwddnvbODM5css/vjjWrTYTn1QE+Fv/dT1/DHZsrD25tAJ0AWWLb00KhpiTLHNrNl2TOKkKKZQpCEjVcmMbjOY2HYirb1bA/Bx8MeEDwo3O8lrrOgGkiL3ALpM64J3KxNhz2qJ3FVGqdyJvKxSLPlKmLIQLy8amvP4E4lEvJT8Eg4eloWQa6okHEpA4arAJ8oHJ28nvFt64xZSdfiz+sbJ14mJaybi1dKL1aNW4xLowpPHTefU2vzMZs7/dp4Z52foPUSuZ19HrVXjJHMiyFVQZmQSyxYCRn812jqdsBH6vNynoZsACOJ8xKCIqgsiiNdnfzmLs3/ls98Rn4xg924o+XYVIORCAkCjhvwr4BxWeR6e3ItweJqQ7635YyTkJnAh/QInk08yosUIwCD8JZWttW16ehMX119kvmq+2YVwWyDhYAKbn9lMyycGAK3x9gZ5meO6TswqPHedY7EnGPzOYP13r6Z8f8f3yF3kPLD+gWof6+jlyLOXn63SewdgRa8VFBYBoY+jUBiEOC/HMm8wrRYS/gTfGuZ0kzqDi8HzycFBeEmTlxoiAQDEPBBDzAMxqNVqsmKzanauesAjzAO/aL8KhgGmhD9fXzh/3hACT8eA1wZYfL785HycfJ303x2RWMSDmx/ENdgVsURcI69jTamG7wd9T1D3IIYvGm62nEgkQvP4VOKPwXh36BgABHSE6Jerfc4qkThC+kG9544u1GeuNh53BHG6w9NVh/lqDKRfSKcgtYDgHsFIFcbzuI0b4bvv4JFHBCEGIC8pj/O/nye0dyiBnQNrfN4N0zYQ0jOkRgYpwT2C6fhYR6QOlc87lTlKSpWlFBU4AWKjvBZ67ooz64lndbRafYjI8ujCUIvFQiSNvDwhGoaLFSPV2gKHyiK5d+kC06YJCxI7dsDFi9C1nJ3P1ayrJOcnI5fI9WHg7293P/dE34ObwrI5cUYGqGSCaubr7GswWgmywPvUuwd0XwreXQmWCZc0Lw+mTzdEMdELf2qVYHygLFMEVZlwYxV4dQHfsvlb6HjIOQPHngXX5uBY+feqvPBX3uOvIK2AY0uP4RfjR7M+JnLcNnI8Izx54tATdXqOLtO60O3pbiaNlJTZyhq9Z977+71IHe3LQnZgf4Ig/E3uMJkfTv3ApsubUJYqcZD+t9cvQBD2pQ5Sk0Yfzn7OuDdzRyJvOlFX7Ngpj074q47H3xNTp/L2m29yCHAHvfcfCKLfu8AeKob/fBA4BIx99VUAZs6aVbNG26lTtNqaCX8nlp8A4O4f7q6ipHl096Etevw1BA26Ivjuu+/SrVs3XF1d8fPz46677uLixYtGZZRKJTNmzMDb2xsXFxcmTJhASkqKUZmbN28yevRonJyc8PPz4+WXX6a01Djp+K5du+jcuTMKhYIWLVqwcuXKCu358ssvCQ8Px8HBgR49enD48GGr97kukfb4lGe3xBKX4EhrzxiGRA4h1D0UkUhE92e702pMq4ZuYrXRLd5JWrdgzNIxtV7YNUXLF8ZwuefDFKiqfuHRlGooLiiusN3SHH9OPk6Ipba7EG8Ntjy7hZUDV+r/nrRpEgNet3xRtr5wcHcg+t5oAjsFMuzjYQx5b4jZsh5hHni38jZ62VZr1IxtNZbhzYebfDkvUZfUSbubMhfXX+Sbbt+QeNj6nr8A2TeyLfYcc3B3YE7uHEZ8MqLKsgUFUCLJBsDTocyrqygRNrapOjyb1BWyToFKWBnrEtgFgP+d/h+lGuE5pxP+bgkRHImZFMPghYPRlNq2haIyW0lBagGZGcI1CShnJqhbkLzVZjAvp72Mg2ftFyTkLvJa5xzNTcjl0GeHSDtvPnaZT5QPTs0E5VLhoCFbmQ2UuzdEIpiYBf2qF07SiMJbkG6Yw/j6gqxUEBb1Hn82woDXBzBlp3HYOq22eh5/1WHtQ2v5vOXnRvnfPCM99YJSfkr1oyeIpWLyk/NRZledBFhnPGTSs8uayFyFnFydPgQMYQazNIYcf7bCoc8OsXLASlQ5BssrrRZ+/hmWLoXiYiHsoo7c+Fy2PLuFq1trHoGhOL+Y498cJ253XNWFTRB1dxTjVoyr0vDp8OeHWRSwCFW88O7zxNGOBHwUwJHE28LGi+p4blmcAxtawdHKQ0+D4eX/1MoT/HL3L6iVTSfvbNLxJP6c8ieJR0zPQw4eFH73KFvFai3YX3LpknG5fTeFUJBdg7rqF9Sd5c4Wi34lJcK7R7FUEP78nMtyoFsqALuEQ4tp4BKJQgHt2wubdV5/3t7lFvckChh3HXosKzt5Dhx7Hq5+a6iv1dMw4ijcsVUQFauge3B37nB+Br+ckUYef6G9Q3kp9SW6PlW9aAj/JSQyidnIBEs6LOGHwdUPj6ZwUzSpMPF2zHPu93NsenoTuYm5FfapNWoOJgiD2PM9nifELYT84ny2Xt1aaZ2HvzjM2ofW1kl7GxMyJxlTdk5hzNIxFfYFdAjgiUNP0GZcmwZomR07dU9NQn0GBATw3vvvM1KioDPwv7LtycAcYAPmc/71ADaoVMyZPZvk5GQzpew0JEqlIapKdYS/N7Rv8Ib2jVqdW+d5aosefw1BgyoQu3fvZsaMGRw8eJCtW7dSUlLCsGHDKCh39WbOnMmGDRv49ddf2b17N7du3WL8+PH6/Wq1mtGjR1NcXMz+/fv5/vvvWblyJa+//rq+zPXr1xk9ejSDBg3i5MmTvPDCCzzxxBP8/bch8fWaNWuYNWsWb7zxBsePH6dDhw4MHz6c1PJvI40cT09wcxMWO+JuW4cY+sFQuj1dvXj/jQHdv9/Pr+7OobNGtiTUZ/KpZN51eZd97xvnbrBU+Mu+kU38fstydzRV+s7py+CFg2sUTq8+KS8AxTwQQ/Nh5hN295ndh0f3PmpkARjlG8X6B9az9j7jF6FvT3xL6OJQZm+dbba++P3x/D7pd5JOGMIlOdT5KnDDoC5R8+cjf3J6lXGIoaIi+PFHgxcbgEatIT85n6KsIqu3o7igmM9bfc4fD1kedqAqkejqP1f57b7fyDyfQolU8KjSe/yJZdD2FQgYVvlJHAMFEajtKwA81ukxvB29OZd2jqVHlwIG4S8zUxh/2k5sS5/ZfSp4vtgaLUa0YObNmahbCi/R/uWcrXUefxlKZ+QeTlYZTx5Y/wATfppQqzoyLmbw1/N/cWPnDbNlxn07jjavjANA5JCrz+1nFOpTJKrcE7QqDj0B2/rrF4F9fCAwayKvtPiRJzoL3giqXBVHlxwlbneczY0vaWnCRF8qRZ+fCgz3xe3C34nvTrB6zOoqhTetVktY/zBa39napNX2n1P+ZFHAohoJRs9efpZxK8ZVWiY/OR/tvn9xyI5nxY1XWbh3IcU31sDN36p9vuoyquUozj59ljdarwMEAXLH/B0sClpEcX5Fg6fGRLv72zHisxF64xutFlasEJ4hOuLLTb18onyYsnMK7R9qX+Nzyl3kzMmdw7CPqxjDa0lgl0C6Tu+KUiq8UWcVp5BSkIJcUvb8Kc6ChPVQUDMB0mJkbiD3AoXpyBvfnfiOEatGsOzYMtzKtKv0c6lc2XIFUVHjnu9Vh6QTSZz64RTKrIpjSVKScJ9JJILHH0CrMpvLS5eE+1LH3riyMJ/NaubZnVlmv6F2EIxMfJ3KrGGuroDffSDt32rV99ZbsGQJvPMOzJoFCxaYdOwUcImAgX9Bl0+NtzuFCKGmFVXnMu0f1p8XWn9OcOYkI+FPqpDi7Otss9EKru+4zv6P9lOYUbcm6Gnn0tgxfwd5SYaXV3WJmuYjmhM2sPohFwtSC4jfH48qr/HlXK8KW5u/WJPkk8kcX368Wsdc23aNo0uOmvQ2P5d2jlxVLi5yF9r7t2d8G2HN7ffzv1da560jtzj7y9lGP1ewFo19DcOO9fgvjy+3U5NQnwAzX3yRwXctZAVitiN48i0HBmFe9NPRAxgol7Ni+fLqNtdOPaBbP5fLhdD29Ynd4696NOis+q+//uKRRx4hOjqaDh06sHLlSm7evMmxY8cAyMnJYcWKFXz88cfccccddOnShe+++479+/dzsMyk8p9//uHcuXOsWrWKjh07MnLkSN5++22+/PJLiouFyceSJUuIiIhg0aJFREVF8cwzzzBx4kQWL16sb8vHH3/M1KlTefTRR2nbti1LlizBycmJb7/9tmLDK0EiaThrOVFRAvd1/ppm7mc4fimZb459w3cnvmuw9liD1FQQqUtx++1bDn9RNx6Yrq7gnXAKzfqNVZaVO8tp/3B7AjoaR6guL/yZC/UJ8M9L//Bt329t3hOnNkSNj6LzE531f+cl5bHusXWc/rFx5ZZY9+g6Pg7+mFKl4FWl1WopKay9l55CoiAhN4FDiYfMlilIK+DMT2dIPy+sXkskEtq0adOg40tdUZheyKnvT5Fw0Dh5/C+/CN4aq1cbtkXdHcXM+Jm0GN7C6u0QS8X0m9ePwK6Wh30rSC0gdnUsWddNh0lMO5/G2V/OUpRXSuntHn+OgdDxPQgeVflJRCKj1TdPR0/eHvQ2AK/vep3MokxcXAxWVk3RIE7n5F/e48/DQxB9UKu5ejCNvFsWJmmtY4K7BzN5+2Si76s8n5zOQESjEFZvnWROhjBKhbcg6R9Q1sJtrfljwv1V5hXq4wPuRR2JYRIdAoTwjSWFJWyavonzv59v1OPLoc8Ocegz4/FS5+0XGlp2H5ShF4RvC/WZcTGDa9uuVWk0IBKJGPD6ALOevH1e6aP3BK8LMq9k4rhvG87ZV1h59V3m7ZiH9Nx7cMK8oUitSPsXbvwECKFm2/q2xdNJGEyUSpC7yvEI97DKs68uCesfRo9ne+jF2vPnYZ2gXzKlzFk0J8eQj1nhqiB8YDhuwbULM65wVeDkXQ3T53Kknk3lj4f/4Nq2a5WWazmyJaO/Gk2BSLBSKygVJpx6z7Cs07BnHCSsq1E7LEYkguEHob3pHMVXs67y99W/iU2J1T+P/CcNZr5qPp0HdW6040t16fx4Z2Znziasf0VxRRcoJjraYFQYHi4siOTnG7zyAUN+v9uEv9lbZzN69WiuZF6ptB36iChut3n8ydzBrU2VoTYB2NIF9t0LCCFag4MFz79Bg9Dn/QMg7ypc+wEKy83TgoaDrKyTp+bB9iGCV6hWC6WWmV/rjDrTbnOQTzufRvqFWjz/GpCL6y+y9eWtdT5mJh1PYu87e43GD4lMwtilY6sV2lrHqf+d4ts+35IaazsGz9C034/McXPfTf2cN25vHLve2FWt40d/NZoXbrxg8tkV6BrIsjHLeK3/a0jFUsZHCcLf+ovrK41UM+KzEcwrnIfcpXZRMxo7F/68wO63dpv9fl/48wKbn92MVmO9vON2Go7/4vhSGTXx+NOxdNksvl6WyB1DhjNCJOJn4GELj51cVMSf5ReE7DQadMJfdcL6X1h3gRW9VvDbfb9x9pezNT63LQt/DTGmNCp3gJyyN3IvL8FS8NixY5SUlDBkiCHEXps2bWjWrBkHDhygZ8+eHDhwgJiYGPzLuQIMHz6c6dOnc/bsWTp16sSBAweM6tCVeeGFFwAoLi7m2LFjzJ07V79fLBYzZMgQDhw4YLKtKpUKVTmFJ7dM+SkpKUFd5u8qEokQi8VoNBoj7yHddl25qraLxWJEIpHJ7VAuOWTOJe4MfJqkxE85fq0bHxydRph7GJPbT+bEtyc4/+t5Jq6ZiNzNeFImkUgqtNHc9vruU0qKCKkyDzIyyInPqVBeIpGg1WorJMisTp+cnES4pl3D4VYsBRmD9Bbjptru2dKTcSvHGW0vLgalUly2Ni+ioEBN+WaW71PMQzGE9gmltKRUn+/NVNtr26d6v/eq2K7rk7pUbeQVJ5FIEMvEnPzuJBIHCdH3RzeaPrk1c8MnygeJQkKpqpTPmn9GSM8QJqyZYNSnEmUJ22ZvI/yOcMFLpKzt2UXZuCnc9FaBuu26UI3Hk46jKlGhkCkq9ClyWCSvFryKWCG0X6PRkJ2djaenJ1KptM6vU33ee05+TszNm0tJsWHs1Gph3z4xICIuToNabajH2n1KOp6ET1sfpAopfeb2ATUW9ynhcAJrH1zLyC9H0uXJLhXKd3umG52mdmLFtxKK0wWBx8PBo/rXKfMUopTtaCMeAbkHU7tM5aujX3Em9Qxv7nqTj4d9jKuriLw8Mbm5GhKPJbFlxha6P9ed6PuiG8X3qartpq7TjZ03yL+VT1JCNFqtBF9fjX5slUgkeHlpyb2Uys/9ltH/jf4MeH1AjfuUeTmTc7+do/W41vi3869xn2SuMpoNMOQlur1PJYUl7H17L6mOzdBqWyGWFtPRp6Px8+DW34gPP4a67x+IQu6sWZ+Cy8XMV6vx9ha+T2lphu+TwlPBQ/88hHuEO2lpaXh4eOj70pjGiCNfH0EsFtN1hiH029WrIkBERITxdfLwABCTlgbqcvkWB7w1gMHvDi77d9R8HuHV2ouJayai0WiM6rGkT4mHE0k6mkTHxzsic5CZ7Ktfez9ujX6c1LKFc6lYiqbLl4i0SjDRxtpeJ/HZ9yD5b0Rh96PRatFqtchkoNWKKSqCvm/2pffLvYXn921z28YwRpjqE8CxYyJATL9+Gu6+W8vmzSJSU0XExWlp3164Thq1BnWxWsjXU4M+5Sflk3srF+/W3sicZNXuU2FGIadXnSagcwBhg4xFJFN9KiwUoUVLkVp4w3WSOqFWqxG5tETccyUaz65oTdyT9XWd3OSCEJmtyibSVSvk/SgSUaouJTsjG++yOLy2PIfVbde9R6nV8PffGpo10xIVBfv3i9BqRfTsKdK3XSSCiAgRFy6IuHhRRGCgBlWpihi/GPKK8+gV0qusLqGNf1/5m9Opp5nRdQYtvFqY7VNqqhqtVgxOglDj4+QjtDFkAoSUea6r1ZX3SeGDVuphdN+YvB7JuxAfeQJN//VoFQZBUVQYj7goAa0qA/KuoBE7I14bAB4xiAZvq/Q6abVaRM6Z5DqkkJwaBRjmtit6riCoWxAPb324Vtepsj6Z7Wst771es3sRdU8ULgHCKlhdjXstR7fk8YOP49/J3+gcNe1Ts/7NGPLhEFyCXdBoNI1mLK+qT6WlpWRlZennL41ljKhNn6q693a/vRv3Zu6MXjKa4rxiWoxqQUlxid5L1pI+uQS76PeX3+6p8OSxjo/pj+kT2gc/Zz9SC1LZfm07QyOHmmy7zEWGWGK+7Y11LC+/3ZLrdP6P85z+4TS9ZvUy2adr269x5Isj9JzVE/dm7jbRp/JtbCrXyVp90mg05OTk4OXlVeGcttonXRtrcp0KCrRotSIUCuF9vDp9cneHqVMDePzxzSz++GPeePlljN0ozOOPoBOo1er/zL1nK33KyRHet1xdjdfrKutTXmIeSceTSDiYgCpfRZsJbWrUJyensneNPK3+3LZynW4vWx80GuFPo9Hwwgsv0KdPH9q1awdAcnIycrkcD2ElR4+/v78+zm9ycrKR6Kfbr9tXWZnc3FyKiorIyspCrVabLHPhwgWT7X333Xd58803K2w/d+4crmWmrl5eXjRr1oyEhAQydTFZEGIdBwQEcOPGDfLKxZcMDQ3F29uby5cvoywXLzIyMhI3NzfOnTtndJO0bt0auVxObGwsAGK1jLjCH9h7cwgO0ixwhrT8NGJjY7ly+Arx++NJi0sjE0NbHBwcaNOmDVlZWcSXi4Pk6upK8+bNSU1NNYqpXN99On3anwy1E51Wz6L3QKV+Owhf4JiYGPLy8rh2zWDxWN0+lZT4kBg1hBvRg7gcfxniq9enrCwJeXnNcHJyQiqVcfVqArGxBg8goz6Fg2O4I+cuniMmJobi4mKjvJbW6lN9XycdVfXpz2l/kro/lf4/9MctwI02bdqgFCkZuWskcnehvsbSJ6/xXniN90Kj0VCiKcEt2g21l5rY2FijPp3Zd4YjXxwhIysDcZRYf53arWhHfkk+y/sup2tIV32fipKKcJW5kleSx7bYbYzuPNpsn65evUpeXh5arZbMzEzat2+Pr69vnV+nhrj3rl24hvKm0KfERDkJCS2RSmVcuJDH6dPXDU5v50EilqBubfzArEmfbpy4wep+qwmfEE67l9rp+5ScnGxRnzyaezB+9Xi0oVqj//Ht997V674UaQXrdQ8HD86dO4dX6ve4FBzmZvDbtGjbvdLrlHnyB4JTFnM5ywOlW3diYmJY0HcBd629i2+Pf8tEn4moVNGAFykpuWTnXCX9ajrXzl3D+YZzo/g+1eQ6HV90nPi/4rn5QEvy8krIzb1FbKxKf52cnAq4rpHgOioadYiahISEGvfpyh9XOPbaMfIV+fQL61erPmk1WkqyS+h+R/cKfVJnqtn/4X7chivJKwpAnOXKd2O+088bUlNTycrwwiVwHrkpjrhS8z6Vv05OTi0pEavZcu1PSv5KYWjwUKFPfQVjhb179+Ll5aU3VGhMY8Ton0bj7uJu1KcDB/woLfUlIkJmdJ2ysyWo1W3JzBRz6lQs4nJxLSzp0/6n9uMc4kyPBT2s3qcDXx3g6qqrqJurad2jtdnvU65rCGnZQuQLR4kjZ5KcaN26E3KNxupjubN8PC4txhGIltTUVD7+92OO3bxJXukr+KW3AFwb9Rih69OP434kLy2P3kt6A7B3bxjgSVhYNrGxN1Eo/MnLc+L4cRXt2/uRkJDAD61/wKe7Dz0/7VmjPp1ZfYatL2+l33f98OroVe0+iVxFvFr4KoXFhUblb79Op949hUgkIjNzAhJng5dB3KU4kiRJwr0XOYWEmzfJjDPUUxfXSaG6inf2ejLdR9Gy20SjPhVkCGJ1tjIbuVxJXl4x506k45F1FbWrmmH3DSM7O9um57BXr1wlaWcSntGeeIZ7olS24dNPS1Eqi2jfvoDYWGfEYinduzsb9cnJyQul0peLFx1p0ULo02utX0PbSktxbjE4oe+Tu0hYLL6YfJFRrUaZ7dOpUwnk5XmQrxVCvHophPlqtfrU9Vfh+1R2jLmx3EPRhvD+60jXRHBLV79WS7trYxDLHInvsJ1M2ZNw5gwhjv1wcAzHBSq9ToVFhfTe0B11OzUDj12nuDicixeF69TiiRY4+DigVCptZg5rdJ2cwbPIEzeZZffeyZNOZGS0YMIEFdnZlvfJsYUjZ88ZrOUvfXEJFwcXOs3rRIouVIKlfZLl4TzEmbisOEJdGm4sr+51SkxM5MqVK/r5S0OPEfVx70lCJWSkZ5CdnU2/V/tx4cIFo/ugsj4lH0rm0tlL+Hb3RSwTV9mnwoJC+vn04/eC31m+fznh6nCTfXJ2cEaRriCnIAe1v+GcjXUsr+l1CpwcSMSkCGTOMuLj4yv0qe+cvoROCiUuOw5Rjsgm+tQUr5O1+qTVaikpKcHT07PJ9Alqfp0SEgrJy5OSkHALhaK0xn0aPGQIK0JDSS4fg78SUgCFQkFsbOx/5t6zlT7dvBkKeFNamkNsrCHlQGV96vZ0NxR9FWSezcQ51JnY2Nga9UkkKiIvr4QrV/KIjU23qetU/vj6QqS9XQ5tIKZPn86WLVvYt28fIWXxRVavXs2jjz5q5FkH0L17dwYNGsT777/PtGnTiIuLM8rXV1hYiLOzM5s3b2bkyJG0atWKRx991Mijb/PmzYwePZrCwkKysrIIDg5m//799OrVS19m9uzZ7N69m0OHKobkM+XxFxoaSlpaGp6eQii3hlCRr12DWbPEyNwy+aOVEPOqcG4hUqTI5DKbs2B4/nkRN26I+L//E/Jl1IUFQ0GBiAcfFKPVavn9d40+bJipth9bcoyk40mM+nwUYoXQ1mvXYOZMg8ffffdpeOCBuvNQaozXydI+7XpzF1f/usoj+x5BJBI1iT6pS9XkJeUhlohxCXBBIpFQoCrA9T1XtGhJmpmEr7OvUZ9Grh7J1mtb+WLkF8zoPsNkn9LOppGfmk/4wHDUajVnz56lXbt2yGQym7QIMnc9ClIKKEguwKOFhz5n3o8/ivj1V8GjR6vV8t13GsocwfmqzVdIHaRMOzmt1n0qSC9g1xu7aDmmpT6HozXvvUubL+ES6MLyDQGsv7Se6EFneGHUaNr7tUd07DlE11agGZ+FWOZgsu36PuVdh4Lr4NkVZC76Pn1y8BMmRk0kyDWI+fNFnDkj5sUXNfTrZ7vfp/LbU06nkHMzn9nLW6LVYnQfSCQSPvpIw65dIqZM0TB+fO36pMpVkXExA49ID1x8XWrVp9/u/Y2L6y4yr2geYqnYqLymVEN+Yj7rt0j4dYsrI0ZomT7dvJVajfuUcwXxgQfQhj2IttWzxMaKmbngErti2uCucCfj5Qx9eVW+igtXLxAdHa0PPdGYxghT2596SkRysogFC0S0a1feOhcmThSj0cC33xrul6LMIpKOJeHTxge3UDezfVrebTleLb0Yv3q81fuUfj6dvKQ8QnqGIHeWm+yrpljDffeJSFOcYF+77oS6hXD92WuIJVKT18Pa16nH8h4cTTpK18t/0tn5Tt5/NYfY1bFEDI4gsEug2etR2fb6eD79cs8vqHJVTNo8CaUSJk0So9GIWLZMg5+flpUrRfzxh4jRo7VMny5cp/WPr8erpRd9XulToz4lHk7kyt9X6DytM86+zlbvk+46Leu8DLFUwobAqSgV8WxvH4ZCoqBgboFRG+tlLE/eimT3SDSdP0XU+lmjPq09v5Z7f7+XPqF9eN51L99/DwNa3qLwk+W0mdGGiZ9ORCQS2fR8L/1SOl+1+Yoez/dg6KKh7NkjYdEi4/6Eh2v54gvjtu/bBx9+KCaipZKn5lyjjU8bs22fsXkGS48vZX6/+bx9x9tm+/TFFxr++ktEm9Fb8Wl3nIHhA+kR0gNt7Nsg90LbcrpFfarVWH5jFWK5K5qgO9FiiOZh6XVq9kkzbuXfou+5I/z8cVcCAmxvDnv79vzkfBy9HJE5yCrce3l5sHy5mF69oGdPoY3nz8O8ecIzy9UVXn5ZQ/v2lvcpbm8cJQUlNB/enOXdhBxIU49OtYnvkzWuU0lJCWfOnNHPX5pCn+ryOv087meubLnCi2kvonBVGPUpvSCdn8/+TO+Q3nQM6Kjv04H4A+yP38/4NuMJ8wgz2faSohLed3ufqAlRTPjZkCu7Kd979j41/T7p1l/alx+UbbxPujbW5Drdf7+WggIRX36pISSkdn16Z8EC9r/3Hn9ZIIAMd3Sk75w5vDpv3n/m3rOVPm3ZImLpUjE9e2qYM8cyjz9r9Wn9ei3LlkHfvlpeftm2PP6ysrLw9vYmJycHN11i9DqmUXj8PfPMM2zcuJE9e/boRT8Q1Nbi4mKys7ONvP5SUlIIKEv2ExAQwOHDxrnfdFZu5cuUt3zTlXFzc8PR0RGJRIJEIjFZRlfH7SgUChQmMljq6iqP7qKbKmvt7eFhWpwV+eTneiIWidFoNWSrsgl0NSzYmKrHXBuru93afUpLA7eM6yhPqdB2bIVEWrF8bfvk6goiESjyM7j4Vy7tx0WabeONnTe48McFxi4bqw+pUVCAwRsJKC4WY6pbEomEa9uvsfWlrQx+b7A+T5mpttvadbJku0gkYtD/DWLQ/w2q0MbchFySTyYTOTQSqUJaadvro095SXnse3cfUROiCB8QXmmfpDIpns08jbZfz7mOFi3uCnf8Xf31XjS6tvcM6cnWa1s5mnTUbJ/WP7aevKQ8ZiXM0p9LV66ur1N93nsX/rjAlme2MHnHZCIGRQBQlsJV356UFAm+vsLfY78Zi9xVbpU+Ofs4M/rL0bXuU2lhKTInmX5M0LF20loCOwei6jWFwJy7eLzlXehTg3X/Erp9gURkHPrWZJ/cIsHNeFwSiUTM7DXT0JeyRNtKpenxx1bGiPLbgzoFIQoAlgsJo318JEZjra+vYHCRmSkx6nNN+uTk6YRTTyeLy1e2veWolrgGuVJaVIrCTWFUXiKR4BnpidZZeG44Ooosanu1++TgCUVJiDRKkAjfH7naG4AcVQ5akRapWMrynsvJS8pj0LpBJucvjWGMyE3IRe4qx8FdEMiLigy5LCMiuO3/C15ekJ4ueP/pxo202DRWj1zNqC9H0e3pbmb79OTxJ+usT/7t/PFv519he3n+/eQAHTZs58CwTgCEKxyR/KKAqBeh04d1N5ZrtYjFYkLdQzmadBSlPAGVSkRufC47Xt3B8MXDCekeYnRIYxgjdNz76736zxcvCgKwnx8EBgp9DQ8Xvm8JCYZn8V3f3VWrtoT0CCGkR4iJ0pbVo9VoSTyciMxZhl+0X4Wyuus0/dR0sjLUbHxERIk4FwBXhauhrr33QE4s4tHnwcS1ter18O8HYy4idonUT3x15b2dhfElW5mNe4gIkQgKHbwZ8dkIlH5Koz6Z66ul2xvq3nPxc2HcynH4RgnGXEVFwvaWLcHREU6fhpEjjed7hSWFZLgd4krQfvbJ/+Cn72+w65FdtPNrZ/KczTyaAZCQl1BpG69dE55/d8UMo2/fYYZ2XvkSXFtBm2eq7lPav0iSt0Gb50FumMdW+L9rNaa3N58CydsRX/4SIicb1VFZ23XbA1wDuJV/C5UshbQ0CA62vTns7duXtFtCQKcApuyYUqH8pk2wezfs2QNPPSWhTx/48ENhvHJwEITB//s/CU89BSNHVt2n0qJSfh3/Kwp3Bc9efpZpx6ZRnF9coz7l3Mxhzd1riHkohl4ze1nU19pst+Z10tV1+1yrtm2s7vaGuPfyU/LZPnc74QPD6TC5Q5XlBy8cTLv72+HkUTFJ14HEAzz313O09W3L2acFD0KRSETvZr3p3ax3pW2UuEgY/dVoIUVGNf4Htnad8pPzkTpIcfBwMFleq9WSn5hPSWEJPm18rN52c9sbetyri+2NpU+i2+Y6Na1HV1dj6FNNtmu1UFQkzO1cXQ3v3TXt07Qnn+Sdt9/mENDDZEmBQ8Cu4mK+nzat3sZ4W75O5rbXVZ90eR/d3Myvf9/OsWXHCOgUQGDnQDIuZeDo5YiLv4vZ8ua2OzsL96NSWbu1lIa4TqLyC1r1hOle1hNarZZnnnmGP/74gx07dhAREWG0v0uXLshkMrZv367fdvHiRW7evKn3zOvVqxexsbGkphqSUW/duhU3Nzfatm2rL1O+Dl0ZXR1yuZwuXboYldFoNGzfvt3IA9AWkG3rxCcjuiNCjJtMMHXPKMpAlavi8ubLpJ1Pq6KGxkNBgZCs0//aAXY9/RvU0fdDeIBB5Im1bHl8bQULgfLc8+s9vJz+stECf1l6Rz23Oajedi4RyhxlnSd9tzWOLjnKT2N/IvNKZtWF64H0C+kc/vwwKaeNjQF2vLaDn8b+ZLQt71YeWdey0JQarDkuZ1wGoKV3S5MDe/fg7gAcTjxcYZ+OPnP6MPSDofq/daEAmxohPUO445078GktvCDFxws/UilERQllEhMN5cMHhhPUJahW59SUajj942k0ak3Vhavg3w/+5T3390i/kG60XavVMvqr0fSa1UufdLhCMuzqPPS1GlCaH791dRcVCb8vbbrE6VWnLa+/EaLVaNHZ4/j7V/x36USdlP/9w8qBK2t1roLUAkqKrDMud368M6M+H4XCraJxUGF6ITk3cygqEp4zu5Sf0urzVizYs8BQaNdo2Nq/do1QeMPdCRA9BwAfH5CWeuh3ZxUJ4aibD29Oy9EtG+34otVqWdxsMeseXaffduOG8NvLC0wZyZWlEiOt3NfFt60vY5aNIXxQeJ211RK0Wi3qYvNx/d2b+5IR3J5C51IAnOVOEDoe3KPrpkEJG+AXV4j/HYBAF8EyQSVLQamEgI4BTD0ytcKCYmPmdNmw16Fck0NDhd83b9Z/e8wighW9VrBz/s4qi5aohZdGuUxC32Z99XMIAByDwDmyes+TmiJ1BrdWIK5oN+rh4AFAljIL3XCSV+JA16e7EtLVtEBqazh6OtJxSkeCuwcD6J/tzZrBggWwejWMGgXKUiVLji5hzOoxeH/gzYT1d3AxeD45TscoKlHpx19ThLoJN+vNHPM3a2kpXL8ufG7R4radI05A7x8t61D6v3DmTcg5X3m5XaPgj2DT+xI3wsmXQVP2/EzeAfsfhrwrVZ4+wEUwrlXJkin3Gs+/H/zL0k5LKc4vtqQXjQatRkuHKR1oc1ebivu0sGuX4fPXX8OsWZCZCSEhsHw5DBwoiIBff238/DKH3FnO3avu5qG/HkIsEQQwnReXyfZVEuNJ6iClKLOIUmVp1SduZDTW+UtdcOjzQ6x7fB3KHMGYQuYo4+R3J4nfH2/R8X7t/IiZFGNy38nkkwDGz5dq0PWprnpj2abK93d8z4reKyot81X0V2yesbmeWmSnrvkvjS+VoVIZniEV1jNqQEBAAO+9/z5jFQoqxtYTOASMVSh474MPzDrj2BJ79gjGiU0JXSRMS53WijKL2PjkRo4uOcqNXTf4qu1XnPn5TI3OrbsPdeKjncppUI+/GTNmsHr1atatW4erq6s+5qq7uzuOjo64u7vz+OOPM2vWLLy8vHBzc+PZZ5+lV69e9OzZE4Bhw4bRtm1bHn74YT744AOSk5OZP38+M2bM0HvkPfXUU3zxxRfMnj2bxx57jB07dvDLL7+wadMmfVtmzZrFlClT6Nq1K927d+eTTz6hoKCARx99tFp9Mqf01hvBo0lIEN5EnfAmm3TSC9PxS/Nj9ejV9H+tP4PeGlRFJY0D3YJvbueBTH2iYwVvGmvi6wtJLfrSZ7wGrUaLSGJ6AUUkEuHo6Wi07XbhrzKP9Yg7Inj+2vO1ba5NknUtiyNfHSFqQhShvUKN9rW5qw0eER56a4+GJqx/GDPjZyJzkhltz43PJf1COqpclX5R/8DHBziw6ADPXX0Oz0jB2vlypiD8tfJuZbL+jgEd6desHzF+pl++ANpOaKv/LJFIaN68ea361FgJ6hJkJOT9+6/wu1MnCAwUwiCVF/4ASgpLKC4o1odXqy4nvjvBxmkbKUwrpOcLPWvadAD82/sTfW/FRXmRSKR/uS5YDxkue7lZ7EYHdVtkYincWAXu7cCrk2Un+qsrlOTBnZeNNn968FO2X99OZ9mHQGv9QuTeBXvJvpFN+4cqhiexFRY3W4zI1wdCJmNqvq8T/pSZheSV5qFRa2r8nPj5rp/JupbFS8kv1aLFVXPqf6f4Z9Y/OM+YAoSTo43ncuZlcpQ5hkJyLxDLzNZRExQKcHeVIiv1oESaTUZRBr7Ovgx6s3HPB7QaLT2e74F/jMFT7tYt4XezZqaP8fERXqwyMgzbXAJc6DK1S6Xnyk3I5fLmy4QPCse7pXdtm16B9AvpfN3+a/rO7Wv2/x58R2uud2pNodvPAKhkXtDvN6u3RY+DP/j0ArkHAJ6OwjOsRJKFSgVyFzlBXWtnaFEfHFh8AN+2vrQY3kIv/JWPzKQT/rKzhZdUV1c48tURkk8lM3bp2Bqdc9XwVbiGuDJuxbgaHS8SiRj28TC8mnuZLZOfnE/CwQS0QcGAK/7iKFY9ute4UNdPa3T+GlOcLQhFvsaGkTrhT61R6xcA8vKa1vzl9meMztDGyclgRAjw1Man+P7U9/pywa7BeOb3RRPXi+eHj6NfWLjZc4S6CzdrfI75hfybNwXxz8UFTudvJSnJhxj/GKRiKThV4/sa/jAEjRY8BCvDs7PwXDK5rwMEjwOHMq/VghvC/CZ8ErjerkoaE+BsEP7KC12qPBWqXBWqPBVyF7mFnWl4RGIRIz4ZYXLfpUuCp7qDA4weDb//Dqmpwt+vvgru7oIQeOuWUPb0aRg8uOpzthzZEoDrO69TUlhC82HNkcgqrkVs2waffSacq6eJaa+znzPPX7e999OmNL5YQvy+eC5tvMTYZcJzS+GmYHbm7AprE6bITczF2dcZidz0WtXVrKsAtPRqWWFfUUkRP5/5md1xu/lu3HeVeitotdoG8WaoD2ImxSASm++bSCSi/+v9cQ20i0VNgf/a+FIZOnFFLBbeKa3BzBdfBJGI/rNnM1AmY7JSiT+QDPzg4MDukhLeW7iQmbNmWeeEDcj164KHv0gEkybBfffVj71eXaMT/lwsXMKVOcl46J+HcPJ2wj3Mnd6ze+uN6aqLLtqVLQp/DaEZNajw9/XXXwMwcOBAo+3fffcdjzzyCACLFy9GLBYzYcIEVCoVw4cP56uvvtKXlUgkbNy4kenTp9OrVy+cnZ2ZMmUKb731lr5MREQEmzZtYubMmXz66aeEhISwfPlyhg8fri9z3333kZaWxuuvv05ycjIdO3bkr7/+wt/fOCxTVdwew7Xe6fAOcWVGlrISYeEqozADjzAPxi4fS3C3mn2xGgLdS6B7VBBtJ9TtwpOvL1wNbIusI4jNfA+1Wi3Xt1/Hq4UXHuEe+u26AU8kEixhGiBXp02QeSWTA4sO4N3Ku4LwF9Q1qFEtLoolYtxCKpqujPpyFDInmdELTcRgwVPZNcgwydd7/Jl4eQIIcQthz6N7LG6PRqMhNTUVPz8/s+7oTQWd8Nenj8F7VrfQD4KH5Sdhn9B52v+zd97xTZXtG/9mNt2L7l1m2XvvIaDIVAREREVcKO6Fvvo6kde9FVyIOFBUUBBk771bNh107520Wb8/Tk/SNEmb7pYf1+fDR3tyxpPk5DnPfV/3dd29bdp0OoKus7pSnF5M7wW96zlaaDehHe0mVJ/cKik1sL/jCGb8ayS9ezoBCgXsnwdtF8CA5Y5dKGy6kHCtgrXn1rIrcRdtfKcAHU2JyHFvj4MW0cG37ogaHUVCrvA7tEX8BVXYpl7qOpXXf67fArrjlI6UFVYj164FykvKWX/veoJ6BzH4SUt7pKBeQfR7qB+nfPwhEdQIyg+RcAFg8PcNMg4y90DWHoh5EqRy2rQBhc4HrTyfXLVZXd2S5xepTMqE9ywTqeK8YK/itE2Fu1J2tu3X7SHtWBp/3fcXU7+b2ijEn2uAK+0ntq/23OJ7Cy+fwPKFx1DKGjnp3aY/jN5s+tNbVUH8yfPQ6QRywViuRVuqxaVNA5T4NgL05Xo2P76ZrrO6EjikHZcq1sCViT9nZ2Gdl5UlKMo7d4b4bfGcXXuWSZ9NqjaRZw+aAg0qL1W9xj5wcfWFJ1f3XeWXGb8wYNk0oDvONed2Gx/Hn4TLX8H0TFD5mTZHekWifVGLXCrnagVnVVgIP07+kaKsIhbsXdDi5pfa4tPOn9KmUxtm/TkLwK6a/6OJH9E3uC/JhcnM7T6XLn5dWLNGwveHIS+++muIir+i8iK7CfSLFfU/4e1KGP+DYPNZ+Gwh7nIFlFwF5yBQOJCFcQl2jCjs+Yb916LnC/9ERMyC8JkOXd+k+JNbKv5Gvzqa0a+OrnlcrQii2m/gQLhhRiqby5ezJ/4gb41+h7Awwd5CIhGUyhcuwOnTjhF/Ivb9bx+XNl7ihbIXbL6+aZMQo/7+OwR1SuRI6hGmx0xv9QRNS16/NAZu+fkW1HlqiwIER0g/g87AtyO+RV+mZ3HCYptFclfyrgDQ1ts20fHwxocp0ZbwUL+H6BdibZke91scf9//N9NXT6ftuGuTLBn+Qs1uHEOeGtIEI7mOpsD/t/mlOlRe7zTkY+Oxxx9n9pw5fLViBe9+v5qExAIUCk/uf3wOKx9YcE0o/QCuCNMrRiP88INQ2P7kk+aCsdaK2ir+5Cq5xfNh3Fvjqtm7eogxkZj7ak1oDs6oWYm/6iwVRahUKj755BM++eQTu/tERESwYUP1kvqRI0dy/PjxavdZtGgRixYtqnafmuDIe2psiI6pPbPf4qtHy+gR2AOli5Le99Q/yd2UyMgAeVkxft4qGvtWNfUBygK9Vm+zWlKTr+H7cd/T655eTF4x2bRdVPz5+AgKg+qsPo0GIye+O4FbgBvtb7RNCl2riBgeweL4xai865coawrkXMhB5iTDM9zTIihWulonYdtPbG+quBUhVk3aC54cwYW/LrDx4Y3c9NlNRI2LIj09HT8/v5oPbGX4864/Kc4o5vYNt5OWJtj4yWTQvz9cFj5GC8WfW5AbXWd3tdtbyRE4uTsx4sUR9Rt4DTi6/Ci7Xt3FrWtupbDcFSTCs0FQRRhg+B+gCqruFJboajuZMzh0MLsSd5Gg248r95itx4bYkUO1IkxbOY2lS4G9tok/0f5TrRbmYU/Pul9r6DND635wFSicFZxfd94m8Ro5MpLIkZEcqahNKjUKBJyPs33VT51x9Vc4/4GgvHANx9MTFGU+wBVySgU53KVNlzi9+jQBtwXgN751zC/aCkc5pR1OTLT6rEz8GY1GPuv6GcH9gpn67VSbx4UNDmPuprn4dWmcz8HZ29lEGNjD0Y/2En4qj5JRk+gV1AsydsKRh6HDI+DR+GsG8T4slwv3ZVkZfBr5Pn6d/Zi/c36jX78ukMgk3L3vblSeKmJjhaA6JERYk1VGeLiwxktKEoi/yV9NZtrKaXW2kV9wYEH9B18DgvsGM/W7qZS0CYddgjrIAqUpcP5DCJ1qpcBrNITdKijEJJZJMKlEirRim5gAKC4GhYcSQ4GhRcRH9YFBb8C/qz+eEeYHjZhsqErIuju5s6i/ZTzZsaPw3wsXqr9OpFckBc8W4OFkP4siktt+EVmQAiq5CjelG+Qdh3/6QI/XocvzDr0vtEWgTm+4+UXueIGAPavP1oqkPUkceO8Ag58ebLFG1elgd4VQd+RIOJVxih/TXgYV/FngwWx+Mu3brRusWSMQf7VBzIwYosdF21RzlZSY77vYOAOdP+lCqa6ETXM3cUNbc3/IixsuUnC1gL739a3dxZsRRqPxmo2P7MHKeSilkLRjaUSNjrIZp4KQf+i/qD8SqcSuM8blXCHoivaOtnrNWeHMTR1u4pfYX/jt7G82iT9XP1d8O/jWqZDmOq6jJeL/4/xiD3bbljQAAgMDWfLCCyx54QXuuENw57h9ru3Yv7UiWWjbTHS08P/HjsHKlfDQQ807rvqitoo/bakWubO8QYqOWrPirzliov/fpQvXIrIP0qXwDtr5HMGQMJRhoWNo49Km5uNaIDIzIfTsFrSvvUVpTmmjXsvPDzAaSVzyJavGr7K5j0wh48ZPb6Tr7K4W20Xiz7/C5aY64g8JbHhwA4c/OVz/QbcyyFVyvCK9UHnaJv6+7PslP035yeZrTY2/7vuLL3p+YfOhlBmbyd8P/U1+Qr7d40dFjmJSh0l09e9qdx8QrFPEBHxVKFwVrYIkrS/KCsvQ5Asy2XPnhG0dOggVUMEVhejp6ULfExBsVKatnFbnnlMZpzMoK2oYZZeIlMMpfDX4Ky78bc7oKZwVuPq7ovRwplSfD4Cz3BknuRPInSF0iqC2qScGhQnJ3ktl+wDLqieDzoBea7+fWGtAhQO4zcW/QiHM3QpNEXs+Ot5iethKpBKezn6aGT/OsLuP+JwoNVQo/iqUVqgz4PR/Idtex4NaoP2DcMMBwc4RgTRQ6gRWTFT85ZzP4dTKU5SmNu4ztq4oTCnkx8k/cvpHcxZU/OwUdtxQRcVfZatPiUSCSxsXnDzt+9O4tHGh7Q1t8QhxsGyxEZC87RI+aXFmgif/JFz4GDRpjXfRUy/DRcGBQ1Se6mTCfanRQM+7e9L+ppZbqCSVSQkbFIZfZz9OnhS29bDxeKja50/lqbJS8Dcmtm+Hjz+G8kpty/564C9WDFhh9xjPcE96zOuBxNsLgHPunxL0ThCPb6qwPSo8C2eXCYRPUyF4PHR+WugjageVEwCjP5rKwA/qZ6ndEiCVSZn520zGv2t2i6lNIkzsxZeZaU6U2IJMKquW9AOz4s8rWHjm+bn4Cfexky90fhba1EJxsnkw7LBtTwkIxOChByD5T/v7VEXuccg9VuNufYP7cnu7hwnIn2xh9VmcXszxb46TGdu62MDcy7mc/f0sZQWWa8yTJ6GgQChO6tkTxkWPY1KHSQCsv7CeknJz1iomRih+y8ykVmRo73t6M+gx2+T/6dPmNXSByzFKdcL1/rrwl8V+Bz88yKbHNrV6kv5aRcapDM79ec4qhjm58iQ/Tf6JzDP2bxiZUsbARwcy4JEBNl8v1ZaSViysM2wRfwDTO00H4Lezv9m8RyKGR3D33ruJHmP7+NaO8pJyfrz5R458caTa/ZIPJvNF7y+I/SW2wceg1+rZ984+dGWtrxfndbRuiOSKSLY0FsQ2DqJzxLUC8f3ccAMsrnDVFtdyrRnietZR5eL6hetZ6rEUrVqo4j216hQftf+IrLja53DEe1GtNq9xrsM+rhN/1xrKsnBKXUXHwDiMRkhMNL/04+Qf+aLXF803tloiMxOKvcPwGdEVF9/GtZjy8wMkEsq9/PFpb1t5oXRT0u+BflYLWpH4M/WbqsbqUyKRMGvdLMYsrYV/yzWC0pxS8hPy0ZfbJiLcg9xx8W8ZVmI97uzBkGdtJ07yLudx5NMjXPrnEom7Enk74G12vrrTYp8XR7zI+tnr6RXUiy+/hLvvhrw8y/P8b+//cH3Dlee2PmfzOlGjorjv2H012ki2dsz8bSb37LsHEPzPQaiGggpSRyFUSzdUNfiaW9fweffPGzSx4ezjTObpTAqvmht+dp/bnYVHFuIc6otWlg+YeyBRl2sXJ8Ce2yDBkhwfFCokeq5qzlIuyzMlIs/8fIbXnF7j0sZLtb9WC0BxRjF/P/Q3RceF8dur+gsMBKeSXI68uI7Lmy/X6VrZ57L58eYfubix4VbgcpW1Sr3gagFf9P6CkytPmp4Txfoqir+iC3D6ZcjeX/9BeHSANgNAJpBdzs4QlfEYj4b9wPAIwa6o94LePJ3/NL59Gt7asiGgydNw6Z9L5F40W5OKxIm9HhP2rD7n75zPxA8m2r2Wvlzf6AnPgx8dZMPD9h0q+n98B6dGLybdbRNL9yxlr1MXmJYKvraTdQ2CS59Dwg8AjI4azflF5xmeLPTA1mgEG5YhT7dc6yqD3mBaV9jq7yeiajJBX64n5XAKORdsF99Uh9zLuRz+9DB5V/Jq3hlhyl++XLDbE5U/AEa90aHiDLGgQ6vIJr04nVJtxUTvNxwmXRCsFVsAHtn4CBNWTeBcTqwpIK/aB/taQlXi72jqUXp+3pPPj3xuta+rq/k5Jq516gKt1hzbOfsJCyN/14rKQ9cI6PkmBNTC0aDdvRB9t/11iTpNmCOyDzh+zq2j4dgTNe42KGwQ7477kNDcO8jJMQ8h91Iu6+5ex5V/rzh+zRaAnnf25MXyF4kaHWWxfccO0EvKGDi0DJlMIHfXzVpHlFcUpdpSNlw0PxNUKmhfUWdRW9WfPYiGR15e4FXal94lQtwRm2VJTIx+fTR3br+z1dvEX6s48d0Jfp76M0WplpUDHSZ14OYVN1u0IamMjNMZaAqq70MSX+FB7OnkadeB4sb2NwJwKfcSOeraPzdbO8qLyrn872WyYqtPUCvdlJRml1JeXF7tfrWBQW/g444f807gO/z75L8c/vT/XwH5dTQvxPVOYxN/oRVi+WuV+AsNNbvzpaXVLSXUklBb4i+gRwDtJrRD4SxU70oVUiQyCeq82vt1Vi66a412n02N68RfA6PZvfIDx8HMYpIV8yhWneOTA1+aKvrcg93xDK+HF1oTIzMTsiP60P+1KY1+LZG0S+k3lZu/uLlWx1ZV/NXU46/tuLYEdKtd78hrAUe/OMoHUR/YrUicvX42k5dPtvlaU6Pn/J52rf/ajm/L/afup899fWjTqQ0ubVwI7mO7R8qFC7B+vWAvVtVpOMg9CCNGzuecr3E8EokEHx+f5p9fGhliMkxcEEkkZtVfZbvPkswSfpj4A3uW7qnV+Q16AwMWD2Dg4wMb9LP0aevDU9lP0fd+wR6pMrldWir0y4JKfdxi34BffaEgzvGLSOWQ9AsUWGaC/Fz9aOcjkMP5rgdMCx/vKG86Te2Es09LaApVexSlFnHk0yPIMoQGj/ba7QYHg9ojgKAnZtN5Ruc6XavgagGXNl2iJKPhvCIKUwqJXRNLSab5nMXpxZRklqBVa02qtSJdlXvDpy9MPCn0SWoI6MtNvSGdnMC/cDz9necQ5S38yBQuCpzcnfD19W2R84t/V39eLH+R4S+a+6qIxJ89xZ9o9Vk5kewI/rjzD95wfaNRVbJX/r3C0c+PYjTYHliZVopB4USS8zqe2/ocm+N3Cf26ZPaVivXGDftg+DoAPJw86ODbAU+lkPir1sGghSD1SCqvOb3G7qX7TIRIly7W+4nEn6j4K0orYkX/FRz65FCtr5m8P5kND20g/WS6Q/unpZmD412V2vve/OXN3HfsPrvHrRy7kuX9lpvWlXq5sOA0KcJkSsGiUdWEzh7aYkEldvJFq5d2Je5i0+VNpBSlmOw+kw+nc3X1VYpSqpG5tQIc/uwwW57bYqG0qGr1+e2JbzmZcZKdiTttnMFc1HSlBj7riyNfcNPqm/j5zM9WryUkCMVQ7u5QJq9C/NUFHR+BrkvsN+1xbwfTMyDmKcfP2fNN4bwOQLxPdDrz5+nf1Z/Zf80mZnqM49dsIZDKpUjl5vSKRgMHDkCy77e8nB/Op4c/BYR1/cwuAmH/c6zl99ytm/DfhiL+RCX0/Pkgl4Nbwm0AHE45jN5gft4F9wkmdEBoq7JqvJbjo6PLj/JZ989MRWl9Fvbh5hU3W/UJDugWQO97euMWYO21ZtAb+HXmr3ze/XMMOvuSiAivCLbcsYWvJn9l97N0d3In2F0IyuLtNCs99cMp/n3mX4feX2uDW6AbL2heYML71SikAf8u/jyW9Bi97u7VYNfW5GmQO8vpdU8vbvrsJgY83IjFYNdhwrU8v9QWIvHX2L2mq67VrwVotUIcAIL7iNiqpLS0dRfHGY21J/6GPDWEW9fcavq7621dWXRuUZ1a1CgUwpoGzPdna0FzzCnXib8GRrM3fpU5gdyVyEjIcdvFB1fu48ujXwIw6fNJNfaXaUkQFT7+9YhnHYV4jZwcIfi0hcOfHebTrp9aSZFro/gT8f/RoiFkQAiDnhiEe0jr7mIrd5IT0C0AiUSCq78rD8Y+aNGvsbCskFx1LgaDke++Mx9XWX0L0NFXaPhyLvuc3Wsd/PAgp388jVQqJTw8vPnnlwaGtlTL4U8Pk3JYYPWqEn8g9GoCSE01b3P2cSblcAqFKbVbLUllUvo90K9RAia5k7DyMBqMfDPsGzY8vAGj0SgQf1UVf87B4NUNnGrRM8A5BGaWCr17qmBw2GAA8tz2mxJnIf1DmPnbTMKHts5efwHdApi+7wkyI/vj7W1f3RUUBHqFioKADniE1s2ise24trygeYHuc23IhOqIy5su8+vMX7m631yyGNIvhMeTH6fPwj6m50SwazjhnuH4OlckcuTO4N0dnBugsYG2GH5WwVHBU0QM1io/o3RlOjJOZOBh8GjR80vlBXJNij8fHyGg0umEPhEizq8/z+43dts+CAjsHUiHmzrY7PHbUJjyzRSeLXjWbmI189hVnEpyMciKhTFJdVB4HgyNaNnrFg1OlhX+4mer0VTYiE39qUbFQHNB5ami25xueHYwLxZtBaBiFXFurmBZ5BnmyejXR9P5ltoXDLQd35Y7/r3D4fn1fKX6nhMnBNs/R+AV5YVPOx/TvK6TCc88d2XFGyxJFOyBm7JkWO4CpcmgsybyxGdcnjrPROikHUnj2P+OkXOudStEzv9xnkMfHrLooVZZ8VemK2P1mdUAzO8x3+Y5HCH+jEbYERfLhosbeP/n4xZzGJj7+7VrB9mlFVafrhVriSvfwvaJgkNAQ0EiBZW/1RxRLdrfD2HTatzNaDRSpM9C7XEag0Rr+l2ovFR0uKlDqypWBbiy5QqpR1IttsXFgVpj5Grwh+SUZVKuN6uAbusiEHB/X/yb4vJi0/aGJP6ysoTCOakUIrqm0r8/uKm7oJS4UFReZBWD6Mp06DStJ0a9VuMjEGIWdY6asMGCT3Wbjm3ofU9vu+sHe44Fg58ezKAnBlkQ0lXhpnRjTPQYZnS2b1EPEOUlBGjx+baJvwvrLrBv2T60pdpqz9Oa0RzEuEsbF+4/cT/jlo2j7/19q/0ur6PhcC3PL7VFU1l9irb815LiLzVVWNu5uIC3t9CjXnSnSU2t/tiWDI3GHH409n1hD621z19zzCnXZ7EGhqElGMzmnaSDz0FTL5/WaMeg0UDA1tVEHf+9SYg/L6+KigGdjr8e3sSB960tbYwGIwatAYWrpcygKvFXU4X8H/P/4HXV63YtL69VRI+J5oa3b7BZkQiC3d62F7Y5XEHfWEjclciXfb+ss23gj6d/xHeZLxO+us1kOwY2iL82AvGXWZJJvibf5rl2/ncnR784isFgICkpqWXMLw2IotQiNjy0gbhf48jPF5KhEom52gtsK/6kcilPpj/JjR/dWKvrNWYgqtfq2btsLzte3oHcWY7cSWhcrFZXUvyJfdza3gVjd4CqFsSfRCKQQjYwOHQwTjIVOmlxq6t4sgepXEqB3g29QlVtc++gIOG/aWn1K6iQSCUNGkhHjY5i2qpphPQLsb6WRGJ6Tqy9aS+JjyaaFHiU5UB5QcMk8hVuEDUP2gh2sE5OoFakcLDgT7bHbwegKKWIL/t8ydalW1vk/FKcXszlzZcpzjAnRUXiT6m03PepzU/h9z8/zuWeMT2PKwdUZ348w7Yl2+zeJ1WrEBsDLr4uKFzsSBWB049/R1jsJnRS4f2OKNoJf3UCXSOWhJbnQ34sGHToDXpe2PYCB3wfQi8tRaOBrLNZXPjrAqXZLXNyadOpDdN/mE7oSEH5LJUKPbKqwtXVrAa9elX4zQ97fhgRwyJqfU1XP1eix0bj6udYpHuuUm7dYIC9e4X/Tz+ZzqGPD1kogytj8vLJzPhxhpn4kwpkm0nxd/BeWNfE/ZQkUpiaBH3et3pJVC7na/JNxJ+qZwdu/utmQgZaz4WtCTPXzmTRhUUWRQiVib/1F9aTq84lxD2EsdFjbZ6jJuIvNhYWLYKTu4TMV1LBVV55xbJYQ+wJ0769sH4E8HepCJRKEiFzl331ni2UpsK2cXDufduvq9Og4BzoG0f+G/lBJFs7dEetTLKqfDfoW94zqTqsW7CO9feut9h29ixke2whXxmHm9KNu3reZXqtZ2BP2vm0I9AtkMu5QsyRUZyBe0hynfr82cKJE8J/fTueo+MXYfzjJfRp81YLDhUHU8z9hK9sucLrqtc5tepU1dO0WFxL8ZG2VMvP0382xcE95vXgseTHUHmq0Kq1dp8TACvHrOTrIV9bbZfKpPS6q5fd3n61RZR3FBIkZBRn2Hx93P/G8WTGk8idre3uWzuKM4q5tOkSxenFNe6btCeJna/urFZlWR+Ul5SzcfFGdry8o1HOfx0CrqX5pb6oTU/j+kAk/tLTLXtit2aIJGZYmHl5Jua3WjPxJ8YmEol1TG4LWXFZ/D7vd5L2WMo5kw8ks2HRBqv+tY5AvB9bW/6rOeaU68RfA6NFNMTePZ3euoUo9BXEX6lA/KUdS2Pbi9vIvZxb3dEtAulpBmS6cpSS8kZ/wIAwYfn5gVEq4+xPJzn3u7UKq/9D/Vl0fhFeEV6mbWVl5oeSSFDWRPyF9A+h6+yu/y9Vf9WhIKmA3a/v5ure5i3xKc0ppTituM4VfVfyhIxO1hWBkejUSdhelfjzcPIgyE3Y53y2bbvPuZvmMuWbKRiNRnJzc1vG/NKAcAtyY97WefS6u5dJ7RccLPQ4EWFvYVRbkqYwpZBlvsvYu2xvPUZsH1K5lKNfHiXu1zju3H4nY94Q+niWloJnSR8GqV9jTrc59btI0SWhx5/eciU8r8c8YucW0CX5XQuP83+f+Zfdb9pXOLVklGaXkng0G4lea9fmE8zEn3LlV3zW9bM6XevqvqvEb4tv0N+XV6QX3W/vjnuwWXp09MujJssmMZFrpVo78Qz86gXa/IYZyKBvBfUFguIvx307XxRM5Y09bwCCddHIV0fi2dezRc4vibsTWTV+FQk7Ekzb7BF/ndp0Irs0m0f/edRmxeioV0Zx/8n7m7VSuqyojNQjqZRkWSfwjAYjgXeMIzu8F7oKxV+GZz/o8jzIbRfMNAhOvwIbuoI6BalEyv/2/Y/Tqk8pl2ej0cCo/47iRe2L+LStheKnGSCuvaoLPsX7ourzuLYoLymv1e9FVPyJ6wHR7jN+azwbH95Y47pcnC/KpVWsPsNvhZgnakf0NCJExV9l4k8tc0UaIrUqmmttULoq8QixVJVXtr4Siylu7XwrMqlt1bDoZnD1qnVCKzcXXn9dsLfyQLhRtc5XuXgR/vc/0FfUCl6uqElr1w6mdJzCW2PfMvXeottLcFuJ0OvP4TfmDdkHBYLPFi4th79joNC+O4UV4r+Hde0g70S1u0kkEgLdhMqeMkW6BfH3QdQHrByz0vFrtgCMf288I16y7K8YFwfx/h8AcFfPu/BUmVWMEomEXfN3ceWRK8T4xfDarteIeD+CPl93IbCDUO1WX9Wf2GYgPWQ5BqMBT08D7q5yglLv5YnuyxgSZu7f6hXlRZfbuuARVjf3hObAtRQfpR5J5eLfFzm79iwgxBViocGOl3fwdsDbXPrHdt9uz3BPi/yECEfJ8+9Pfs+3J74ltaj6LPSHEz5EvUTNwwMetjsOV3/Xa9Ia8eq+q/ww4QeubK259+i5P8+x4z87yIt3rA9wTTj942mOfH7EZBOvcFYQvzWehB0J18S931JxLc0v9UVTEX9eXuDmJtS/Vi76bs1IThb+KzqPgGXhcmuFGJu4uDgWhmTGZnLq+1MUpVk6hiQfTObwJ4e5uq/2OeDWSvw1x5xy7ZXjXAd0/Q85ySorxV/G6Qx2v7abkH4hLT6Bk5Ut5fzg+URHNd2Pws8P0tIk9PnqQcZNdayKWwxS5XJBug1CMG8wCFXnttDvwX70e7BfA4y4dWH7f7aTcSqDWX/YtpsNGxzGA2cewDvau4lHZomYaTHETIup84R8JV8ICMozo3F2hscfh4ULITtbkKFXlsJ3bNORtOI0zuecZ0CodTVmcF+B9dLrr011qNJVSdRoIRO2a62wrbLNJ5itPqsu/sqLyzn5/Uk8wz3pcFOHGq+lydMQPiwcn/aNM/dJJBLm/DUHr0gvJBKJyQ5MrQZPdU+6Snoyp8K+iePPgEcnQflXG1xaDmeXwaRz4NHRtNlZ4YxnBS9QWiosliUSiFsTh8pTxbDnhjXAO2xanF59mkuP/oProDsJDIy0u5+oBsz3iqD/sLp1dt7x0g6u7r/Kc0XP1en46lCQVIBnuCdGg5FNj20ibHAY7Sa0N5EUVsSfX0UvO4VXg49FpQKFXrj/xYIghYuCoc8N5XRDNRNqYAT3CWbSl5MslJP2iL9BYYOQIGFr/FaGBpwCupuCLQCfdvZ/+wa9gfX3ridqdFSDWr5WRfzWeH6e9jNTV06lxx09LF6TSCW4jhlAfhZoJQLxl99mOMTUbJlXLwSNB4U7yJyRSCR4q7zJKMlAK8ujrCzcwtqwJeLKlivE/hJL5O2DAd9qib+2bQUFzIULMH48nPj2BNuWbGPO33MI7Om4ve6q8asoSCzgsauP1bivRmO2sV6wAJ58UlB2ZWdDzIwYgnoH4d/F2toi93IuR788Spdbu6BWC2uBcqoQf+3udXjMDYrMPZB3DDosEhSAFfBy8gIgT5NH+4qah8JCCTq1DnWeGrc2jUhgNzLSjqXhFuSGe5DwxoxGy6THqUxBJdU3uK/dc/j6Cja0RUUCwddOEKliNMKHHwrbo6Nh8UNhjPsRlH5XUSrh0CF4800YPdpMWrdrB35+wxgWUc/nu9wZbi2wn7XxHw5dloBrLWzDpU6CJayu5mdyoFsgV/KuUCZPt7DAjRwdiUubJqj8bEDETLPsSajTwfFLKWTGbECChIf7W5MlQe5C9u/xTY/zyeFPACjTl5ERsgLOvsTp0zBmTN3GYzSa+/ulSIWit1ndZnLhApQcnct4N+hYqT2oT1sfbvnplrpd7P8ZTv94mn8e+Ye7991t1XOvrogYHsHCYwvx62ztBtJ5RmeK04pNtp9VMeWbKVbbjEYj7wa/S/tJ7ZnylfXrlfH67tc5n3Oef+/419THzxZM/ajtwGgwkp+Qj9FgrHbN1RoR1CuISV9OInRgaI379r2/L91md7NJxtYFhz46RO7FXFMveYlUwrwt83ANuDZJ1utoeWgqq0+JRCjSO3tWKJKqmhdqjais+BNxLSn+KhfsV0VmbCabH9+Me4g7k76YxLMFzyJVWCbJu83pRvuJ7euUoxPvx9ZG/DUHriv+rkVE34kx/DYT8ZerzsVgNNBhUgceOPMA0WOb2BaoDsiqaKPn51/zYsZgNJBdmo3BWD/JrGgNVqB3s6n2OvTJIeJ+i7PYJhJ/Hh6Wk15Nqr//j8iKy6q2kkPppsS/iz8K55ZRFV7XhbSo+HMti+bmm4WKnsr2YpUh9vmzp/gz6A0Upxe3OrsjR6HX6k0Ea0KCsC0y0nIfkfjLyqpSIS+BjQ9v5PhXxx26ln9Xf+7YfIdVYqYh0aZTG+Qqy3oaqwo5gx7OvQ0plnZQDiHiNhi8GlTWEjixf5teLzSRBrhn/z0sOLig9tdpAQjuF4xx6FDKXLyrtXtWqYSebikxY+n23M11utaQZ4dw06c3NXjwvO+dfbwf8T7pJwTbpvm75jP6jdFotUJCLt/lMANWdeCWXyol2qLnwYAVDafgSV4HO6dASRJOTlisC1oDvKO96XNvH4uCEHvEX2e/ztzSWfgst+uEXpiViT+jwYg6T22zf5E6V82Jb06QtLdxu8kH9AhgzJtj7JJMJmUXAvHnpmwCoiR4PHT/r9DHC/BxFgKvcnkuGg2UFZYRvy2+wSrXGxrpJ9I5tvwYxVlCBFod8RdTMf3HVSzlnDyccA92r3VPq8iRkXSc0rHmHRF6shkMwjqgY0fo0kXYvns3eEV4ETkyEicP64aVmacz2bdsH1lns0zBdbBzO7oHdMfftQk88KtD/Eqhd2i55T1hy+qzMLucvwf/zebHNzf1KBsM+nI9X/b50uI9VFbXq1RGTmcIxRPdArpVPdwEicRs9ymSwQCbNsHRo6BQwBNPQIcAITuUqU7hsccNSCRw8KBA/un14Olp7g9jgdRNkL6t9m+wuudNwEjo8ZqgDHQUETPhxlPgN6jGXSsr/ioTf1O+msK4t8Y5fs1mhq1iwfh4yJGfAomRLv5daO/b3saRAp4c/CRRXlHc0+seAPZplmNAx5kzdR/TlStCnKp0LudcgbBWHhAywBTv1tdG9P8zNPkaSrNL0eTXv/dteXG5qf2Hfxd/m2vRkP4hTFs5zeazwu55i8oJ6R9SY69Mg9Fg6tnX1rttLUZuDa1ay4dtP2T7i9vrdZ6WCK9IL/rc28eh4nmftj4E9Q5qsMKpW366hdnrZ1tscwt0M90r1xVp19HYaCrFH1x7ff6qI/5as+LPEeLPL8aPTtM6cWH9BWQKGU4eTla5Xlc/V3w7+NYpDyPej62tx19z4Drx18BoKVU3Xl6gqEjwGYwGCjQFuPi6CMRKNf1lWgou/XSYgCv77Vq8GYwGNlzcwL3r7iX4nWD8/ueHy+sulGrrTveLgVB6ooaEnQkWPYUAti3ZxuFPDltsK6pQKnt4CAG7+PVrqokDMk5n8Nf9f9VJztyaMfPXmTyV+VS1+2gKNGTGNl8kajQa2bN0D4m76u4DJhJ/LmXR9KsQdkZUuC5VtRcbHTWa+T3n0y/EtgJ08xObeSfoHYrTigkMDGwx80tD4cB7B3hN+RqpR1NNSbCqlV0eHkI1j9Eo+L2LULoqmfPXHKZ+N7XJxlsXqNVQpDpLnvIkBZoCQR0xLQ36flj7k/n0hsjZoPSyeum9I6+zs3MPrvp+a1qIuQW4tXi1jj2EDQpDM2QMWmdPE6lpD/W1y4geE02PeT1q3rGWiBwRSedbOiNVSJFIJQT3CSakX4ipMKRMkcGlvIskFtTTd7A6lCZD6t9QkoizMyh0FYq/Sr1/f5n+C7GvxLaa+aUq8bfx4kZGfTeK9efX88LwFwDYnbuGIlWcRdB4cuVJlvks48oWa5smlzYuPFvwrMmit7HgHeXN0GeHEtDNenGTcTqDtJc+xyf5lIn46574MeydbbVvo46xgrzRyvLQaIT+uyvHrCRuTVwNRzYPBj46kKdzn8a9vTAROEL8JScLCfGY6THce/jeGiv4q/aHHf3aaG782LEes6LNZ8cKnnB4hah3d4ULs9FgRK+1VvW3m9iORRcW0WFSB9Oa8tlO33Dy/pOCyqvgHPw7HJJ+c2gcDYqOj8DYnVYWtKLVp1qnxr1C8VdUpqDj7I516qXYUmA0GBn9xmg639rZtE1MgsnlUE4Jo6JG0cG3A53adKr2XFX7/KWlwYoVwv/feafQ4zjYPRipRIrWoKVt9wyWLYNJk8wK9wEDhHhje/x2jqUdo0xX8VA59hgcWVT7N1h4ES58Auqm77Ed6Gqb+GttOPzJYd4JeofUI+by/bg4KFYJE4BY6GcPkV6RXHj4Ap/c+AltXNqQqUkh0+svMjIsSebaQJx7fDqfpFxfjq+zL9He0aZ493xGAqtPr+ZizkXTMTtf3cnGxRvrdsFmgEQiaZb4qN8D/XjJ+JLNPs61xd7/7eW9sPfIPp9d53MUXC1g65KtxG8zVxQ4eTgxe/1sRr40stpjUwpTKNeXI5fKCfO0rSgUUVxezPw/5jPqu1HoDdbPLaWrkhEvjbCYK/+/QqfRUZRaVPOODsAz3NPmOkVbquWPO//g52k/N1o/wf/PaK75pSVCJFaakvirXLzZWmEwmF2r7Cn+WitvL8Ym1eVpJFIJfe/vyyOXH6n2XFq1lsubL9e6n7yo+GttxF9zzCnXib8GhtSev2NTIuk3PHe1o1ubw8j0QlCeo87BaDRSml1qRWi1RORvOYpfwmG7So9397/LTatvYsXxFWSUCA2mXZWuuCjq/jQSA6Gcw/F8N/I7Lm++bPH6/B3zGf/eeIttlRV/EonZtq064q8ks4SjXxw1Ne++DjN+nvYz3wz9ptmuX5RaxNbntnL6x7rZ3uWp88jX5APgYYiibUXhpD3ib2aXmXwz5Rumdppq83wRIyLo+2Bf5Eo5gYGBLWN+aUB4RXnR/qb2qHzdTAn6qsSfRGJeHFW1+2w3oR1O7jVXv17+9zKrb1ptUl41JUpL4Wzok7yn6clvZ38T3pDKH1xqtoqxCxvq5szSDIpcTlGsOm9KSJZml3J1/1XKS1pnd2xxHq2ukgwE4s8lP5VjL68j7VjLKZ0L7hvMrWtuxb+LP6XZpSZVkUj86ZWC6k5UWGE0wo5JcL4OpLA9tL0HbtOA/zALxV9xeTHlFb0iywvLkZRJWuT8cvDDg3zU4SOLhFhV4u+zI5+xI2EH2xO20z2gO9M6TcOIkYtBr5OZaf68/br40eueXrgFWavoJBIJTh5OOHvXwDI3InQaHYYyLRKjgWci17Ljzh346vOh8ELjXrg4HraOhcvCs9dbVUH8yfMoKxMsUm/85Ebajq+fEqCxIJVLcfZ2RmcUihyqI/48PMyB99mzjl9jef/lrJqwiuQDybWurK/a32/wYOG/Fy9Cwr5UXpG/wv5391sdJ3eS49veF2dvZ1Pi3yK4LsuEwjjQNgNb4tVVsICUWT5/7+97P9oXtXw39TuT4q+kRMas1bPoc2+fph9nA0GukjPsuWHETDc7BlT+Ttyd3Pj9tt85v+g8Slk1NyDmNY5I/H37rTBHde8OkydXXE8qJ9QjFG+VN5klmXTqBPfdB19+CatWwUMPCYVqE3+YSJ8v+5BWXPHc6/0u9Hyr9m8wa5dAGOYctn5t9y1QF+eAy9/ApS9r3E1U/Gmq9Pg7+/tZ1t+3nrLC1mGh4uTpZOpvJiIuDnyKhzLT77/M7DKzxnPIpXKc5E7c3fNuAFKDPwcEW9i6QCyWK/U6BED/kP5IJBJTvPtzwWPcvvZ2/jz/p+mYhG0JnFldD5lhE0MqlTZpfHTy+1OkHGrYxlOeYZ74dfGrlzVmeXE5e97YY5W7cARiwWqEZwRyafVdgJzlzqw+vZodCTtIKbL9OYx8eaTFXHmtYPebu/mo/UcUJDn2zH0/4n3W3Lqm3tfV5GsoTi+2ufaQO8sxGowYtAabBUTXUT809fzSkiGueRrb6hPM7ZOKGoY3b1ZkZgpxq0KBhaAlIEBICZWWYrH2aU2wGZtUQkFSAUl7kzDoDTWq1c/+dpZV41dx4e/axbwiEV3XAqnmQnPMKddnsQZGi+jFJXNGInfF072c3ld+5IcbthLkFoRBa+B/fv/jn0f+ae4R1ojCSbO52H+OXeLv17hfAZgRM4PNczdT8nwJpx+oX48i8Vo5qhDGvzeekP6WVXyBPQMJ7GFpzyVWp4oJDpH4q87qM3xoOE9lP0Wfha03CVIXXPj7AskHqi/d6XZ7NwYsHtBslhWufq7ce/heBi4eWKfjxeDJSRtATDsXFBXiWnvEX02ImRbDTZ/chIu/C5cvX24Z80sDosutXZj1xywKjB7o9cJi0pZ9lT3iD6A4vZiLGy5av1AJWXFZXN58GZlT06vf1GrQyvOBCjVEWY6QyNfXMZn1dxfYPtFqc5iHkM1WK5NMi59jXx3j68FfkxWbVbdrNSO2Pr8Vl9++B6PRIeJPUVZCwfbjZJzOqPHcBr2BjY9sJHFXIjkXcljWZhkHPjjQQCO3jXUL1vGG2xuUF5ebCE2jSrDJE4kWtAWQsQ3yGzDpJnOCikSOszMo9J5gFKrM8tTC9edumcuADwe0yPlFqpCicFZYKFcr90dMKkji74t/A3Bfn/sATKq/NJ+fKJflmOaNkH4hTF4xmeA+1v1ritOLST+ZbqXsagysvmk1a+eutdoe0i8E2aMPkxPWkxjvHoyIHIF83E6YeLRxB2Q0QO5R0Ai/HbPiT7D6dPZxpt+D/azWPy0FuZdzST+ZTplGWDdUR/yBtd3nye9PsvvN3Xb315frCeoVxOVNl/lq0FfE/hzL73f87lCC1WiEc+eE/xcVf15e5rWi1smNmGkxNhO+efF5pkI9m0UQ/sNhRjZE17JXbEPBoLd6jillSlPiWFwXFxQYrsn1i1hgU5MivSoqW31euQL79gnJn/vvt3TcPP3AaXKezqFHoFmNLpEINp9SKRSVF1FW8fn7uVQwOcETILQOltdBE2DUJvAfav1aSRKU1oHoOPcuxNVMQtqz+kw5mMKxL4+1imJVgB539GDBwQUmW0WjUZhjvEr78vKo/zhE/Im4r+99PDX4KaYqPwbqTvyJLghpsoOAYPMJ5njXs0j4+2DKQdMxt/1xG0+kP1G3CzYD9Hp9k80v+zcXsfau9aya8TsGvYHL/17m+3Hfk3W2fmvs3gt6c+e2O5HK6p6W82nnwwNnHmDkyyNN23a/uZtdr++q8djLecKzrK1PzcU9MqmMCC8hqI3Pi69h72sLcic5ChcFcufqyVERve/tTYfJNfegrwmnfjjFO0Hv2HSrkEgkTP56Mrf9cVuLaZVyLaEp55eWjqZU/Ilr3eqEFK0FYmF7SIiwdhOhVJpzXq21z19NBdrHvznON0O/IWlPzYuYtje0Zdz/xtXaHaS1Wn02x5xynfi7FhFyI9x4khTdaAIKJtHFZTSuSldkShn9H+lP2wkts2q7MjI1npS5tbFJ/GWXZnMoRahefH/C+4xrOw4XhQvB7sGkFaWxeONibv6x9oGvyeqz1IMBiwfSplLHc51GR1mRdaK+suIPzBNfdcSf3EmOi69LvRb4rRG/z/2dbUuq7z3S+57ejHx5ZLNZKsiUMoL7BtOmk63mKTXDw8mD4U4PE5Izl86iy0nWfnobF+PrnGyT+CvTlRGXFUdxefXJjaJroezJDirbfNr66sPDhf/aSn78OutXfpnxi83fp4iBiwfydM7Tdf5e64PSUsEyDyoInqQ18FdHyKw5GLcJz27gYR1IhnsKH5JGedWUkIweE824/42zqXBq6ShIKkCWmwUSiSlJbg/BwVDUJpKSB5+g5509azx39tlsDn10iIxTGei1evw6++Hi2ziRjNFoZNWEVZz/8zw97uyB0k1pej4Yqir+lF4wswT6fdJwAzDoIWMn5BxBpQIJMpQGgdipbPfZUueXfg/04/6T9+MdZe4vJfawVCjguxPfYTAaGB01mo5tBGald1BvpnScQif9beilaoesYmJ/ieWLnl+QcrhhK/ltoayoDG2JbYLRUZVrg8K9LdyaB12eBcBHJdyPWnleqwi6t7+4nS96fkGZWlBCK5WgN+h5YtMTrIm1rrYXn80i8Xfq+1Psfm03RoPtgiOZUsa076dx/6n7Gfj4QLRqLadWnSLvSs09D7OzIS8PZDJMDgAgkH8AelcPZv42k84zrG3R/pz/J592/hQQCkj0kjJu3BpMx487UlJeKcJtjvVS7lH4WVmtOtnU468QTi4/yS/Tfmm1fYiubLnCV4O/ImFHgmmbWGDj4iL0NHT0vYWECHOXWg0fVnx8w4ZZWkCBsJ6sbi2cVSKQDS4KF1yV9SzBdwmBoBts9/GbcAhG1cH6cdB3MHpLjbv1CurFreEPE5Q3w4L4G/LMEJ7KfsqhflotEenpkJ8vWMG2rWXYHe0dzbJxy+gZ3g6ofdFg5TEAjA4fz+3dbmdU1CjAHO8qsyqIv2Qz8afyVLW6+LQp1i9FRfDZShfiht2HesZcpDIp2lItSXuSyDmfU/MJ7KC2/WXtQaaQ4d/F36LP+PGvjhP7c2yNx4pFq9Fe0Q5dK8pLkC2LfQGtzrflCp91/6xO6sOWjEGPD+L+k/fj6ufYfDv6tdEMfWYoRqOR3Eu5HPrkUJ16Jft19qP3wt74d7VdDS9TyJAphOK4Y18d48R3J2p9jeuwj5YaHzU1rhN/dYMYg4baMHqqb6uS5kZNir+Y6TGMeHkE4UPCazyXq78rg58cjHd0LfpJY1agirmv67APx0pWrqNVwstLSJRXDqQmfmCtFGlpMBqNlGcVIsUZd3fr0u2c0hxGR42moKyAUA/LWVQlV/Hx4Y8xGA0k5ieaqtIcgVh1odEIk0dlKfv5def59bZfmb56Ot1mdzNtF4k/N3cDL23/Lxe9AnDPeACNpvpETFZcFhKppFmIiObCzctvrlVT8uZAUWoRzr7OyJ3qNjW2921P58QP8Ug2qwooiMU780OivcdyODWUggKhYltEv+X9OJ15mo23b2RCuwkW5ytMLmTzk5tpP6k9dK3jm2rB2PHfHUikEuKDRwDWNp8iqmvyPPTZoWgKNDX2smuue0+tBq0sH6hQ/Cl6QudnwbOONjhDf7K5WezLoVZeNS3EgvsGE9zXWt3UGjB91XTWzAaKHbP6NMgUpBU6Vu3qFeXFvG3z8Aj1wLe9L3ftajzFjEQiwb+rP26Bbtz4kdATTAxk9Moqij/hAJA0YNWuRALbxkDIJFTt/gCgZ/p7LH5YTpCbEHHEb4sncXci3bp1q+ZELQcicapUmpUK0ztNt9jnj1l/8NFHsPmEed4wGo2sX7ge9yB3Rr0yymL/0IGhjHxlJL4dfBtsnOfOwdKlMHMm3FipHZy9+y3teBq6Q2lIiGBN2griDrnxgL8PUtdI8BvUYOOqCU8PeZp2uYv4+1gAmgp7yu9Gf4fSVcns9U3bb9ARdJ3VFb8ufhZWn9sTtvPugXcBiA+JJ9Ir0rS/SPxduiTY79z48Y1C7+sa+LOAbgGMf0ewe+9yaxck0poJN9HmMyoKiwIGLy/IyBCIAbvva3ZXU68LtRp0snyyNGlka9JxVjhD2mbQlULolKYn/5yDIWQKuFkmilOLUnl80+MYMfLF2J8BKC6WUHC+kOR/rlJWUIbKqylZ7YaBJl9Dfny+hZVaZcXfkK+HkFaUxobbNzAwtHrHCLlccIG4dAkuXxa+ulmzaj+mzBKhH7a/a0UyuCQJNvaETk9A1yW1P6HRCGXZoPKr/bG24NPbod36BvflvwP78vRaKKi0jGtO2+XawmgwsnHxRqJGRxEzTVjbxcWBXqLBqdM2rhZ3JNo7utZFjRERYEBHbFIWEFS7MVXqi31X39sJCbnd9JqPj3DfuRf2RYKEq4VXySrJws/VD02+hoxTGfh28MUtsPUVjTUGjEb49FPIK5CBWxtyKqaB9hPb80z+M3WOGTPPZPL1kK+Z+PFEetxR/z7TRoOR5IPJYISwwWE8eOZBhxSztVH8gZn4EwnDqpCr5JQXlVdbkPn/Bdv/sx2dRkdw32A2LtqIy08uFoVsjiBqVBRRo+wEyJWgL9ezbck2XP1d6TGvx/W+dNfRoGhKq08x7m9t9o22IMagVYu7QChcPnXq2lX8BXQLsNnTvjrotXr05XqUrjXYt1RAJB1bm+KvOdC6Srquw3Fc/IJBQd+T53qIny5+YVLItQZoCsuJ+ed9ws9ssKn06NimI1vmbeHggoNWr3k7ezM4TGigsvFS7SpUnZzMFcp/PbSR98LeM1XweoR60HV2V4J6WwZeYhHQOeN6Xtn1CrvcH+J0+AOUqKuX764YsIJ/Frd8y9WGROdbOtP2huqDisv/XmblmJVC4NIM+GXGL3wQ+UGdjy8qMlf2xMQAuhJI2wj9PiPZcBNgrVpr79segPPZ563OZzQaif05lsxTmXUeU0vGqe9Pcfa3syQkCH/bI/5Exd/Vq9YNkNtNaEfX27raDbwvbrzI0S+PUl7cPH3uBMVfPlBhoddmIPR8s349/mxAtPrUKJIpLr02LEkcVT8FVjgQFmeUcPKnszX2NFS6KokaFYVv+4YjearDDW/fwNRvpwrkAub3pZVXUfwVX4HUf6C89hXBdiGRQr/PoMPDps8xIH0es7vOMVk6Hv38KCdePWFX8dScuPDXBQ58cACD3tzXUlT8KZVwKuMUAN0DulsdW7U5vEQiIX5rvIVqR0RI/xBGvDgC9yD3Bhm3Tgcffww5ObDbvoukBS7+fRHV5vVIdCl8dvFZFv/zCJJ9t8O5txtkTNUi5S9I+xeAEI8QIj3aozB4mEhWlafKYXurpkbHyR0ZvmS4qfejQmFpQbZkmyUJEhgo9A/R6YQ+e74dfPEIta2u0pfr+Xbktxz/5rhpm0FvQJ2nNv2eq0NVm08RouIvPx/2LtvLP49Zrwf73t+X4S8MB4Q5QysX5gVPlSdSiRRiX4eDdzeP4s85CIavhfAZFpv1Bj0/x/7M72d/x9VVmE+MRmj7cA+eK32uVZJ+IKxfn0h7grbjzGtYMSGldNFwPvs8eZo803O4JlRe6wwfbjshdCn3EpNWT2L8qvHWLwJZpYLiz2TzadSDdy9wrqMl785JsC5KWLeKSPkbLi0XCOa6QJMFmprXr2IxXOU+NzqNjszYTIrTW77VZ1FaEYc/Pkz8VvO8ExcHJaoL/Ka6if4r+tcpCe8ekM2hDhP4Rj+mRleQqigoEOYNiQQr9xy5HHx9QW5wx1clvJhcKDwo47fH8+2Ib7n877Wl1qoPdu6EPXtAoSlCXl5Kbo4wt8mUsjqTfgDqPDWe4Z716u1XGeXF5Xw38ju2/2c7IBBwXhFeNR63bOwyNs/dzPSY6TXuC4IiFewr/sKHhrM4frFNJXtrxqlVpzj6pePW67mXc7nw1wUu/HWB0IGhzNs6j+ixjqkq6wKZUsasP2Zx1+67rpN+zYgTJ+Cuu+CwjZa5rRU6nbm/elMo/kQy51pQ/Nkj/tLT09m751UO74jh6YdC6BcTw2uvvkq6WLHTClCd4q80p7TWeYXUI6ks81nGsRXHHD7muuLPcdSJ+NPpdGzZsoUvvvjCJH9OTU2luLjlL84bGy3mQRv7Gv09PyTF53s+S76fdefXAbDvnX2smrCqRSb4RJSVQXrUQAr92lab8JVKbN++E9sJqsYNFzfU+tqi/YlW6oR3tDflRcJTLmxwGDNWz7Cw/wRzkPpXzrumbUn+X/DM0VtQa+2XqYx4eQQ97+pZ6/Fd6ygrKCPtWBqlWc0ze3ec0pFeC3rV+fhNRy5SLssjNBTc3YGCs3B1LZRlEx4h3K9VLXs6+goZwXPZ56zO5xHiwQtlLzBm6RjCwsJazvzSQHjg1APM3XSHyeozMtL2foGBQqKirAyy7LTSKEwptEnuHf7kMBsWbWg2i7EidRkGmTAXeKm86n/CgrNw9DHItix8CHIPQooMo1RHaoHQq6u8uJwVA1aw9fmt9b9uE+P0z7GoMoQfS03En6urkDhsc/UEf8z+hdQj1ZfOaQo0FkRSU0MkU1yk3kR4Rph6HHH1D9gxEQriGvaC7e6FwDGmz9FoNJNnAEOfG8rkXye3yPnlxLcn2PToJgt1lfj5qY35XC0UIqqu/taS6MBgHVpZgYVS+IFTDzSqwlPEhg3muT6nigtYxqkMDn540KTmEtF9bneyJ9xOibvwrHBXuiEZ9ht0erzRx8uh++DUf0x/VrXZue3327j1l1sbfxz1gJiUcHISyHRPJ4FNWH16NYdTzBkYicSsyD97ViiwKUwupDC5sOopyb2US8bJDIpShFjHaDTymtNrfNLpE4fmkEuXhP92qOLOLBId+flw5d8rnPj6hN1nlNEoKserKIR7vgUDv61xDE0JU39IgxYtalNA7uMf2SyN7BsTYpKhSHUWvVGPt8qbYHfHFPZinz+JBGbbEdGq5Cr+vvg32+K3odVbWwNbKf7comDMVmh7T63ehwnhM6HtQsu+jefehaOLwVCH3qcFZ2GtP5x7v9rdjEYjalkahaozlGp0pt9x+sl0Puv6GSe/P1n7azcx3IPceSz5MYY9P8y0LS4OilUXAOjgW7c+XwFBeopUZylQnmXeb3fXah0r5g4lgae4mB+HwWg5X4nxrrdcKGhNLxYOCO4TzPj3xxPSz7LPfUuFRCJp1PioqAg+/1z4/wH5m+i5+X8U5etMa6j8xHwOfniQotTa2wFGDIvg/lP3EzqwYQoBnTycuOnzmxi7dCyZsZmkHUvDoKv5ORXmGca4tuNMhF5NiPKusPr8f9bj7+AHB9n9uoOVXIB3lDdj3hzDwqML8Qz3JGp0VK3bCmgKNKwYuMLhRHjowFBUnq2zwKYloi7zy44dgs37V1+BoflCzQZDeno6L78kEFT7/g1hVL/GJ6iuFatPo9FcfFqZ+HvvnXeICA0l+/c3eKf4HN8XpvLouXPsfuMNIkJDee/dd22fsIWhOuJv3d3reC/sPXRljttZe0V5ETowtFZuAyIR3dqIv+bIudQ6CktMFOygpkyZwkMPPURWRRb2rbfe4sknn2zwAbY2tJjAdtjvHFb+hEIvKBpySoXMU1ZsFkl7ktCW1iGIayIYFU4kdxlPXmg3lFVUvpklmabgxB5ubC94am2N34pGV7snhikQmjGa+Tvn12gPWFgI+S5HOFWwC7lUzo18hNTgRLo6CZ3B/kQ3+InBdJ11DXo32kFefB5vB7zNnrf2VLtf51s680zeM3SYVP9m2HXB0GeHMvrV0XU+/qF9E9ncywdlu4rAwLcvTM+A0Gn0jhASj1WJvy5+XQA4mWGd3JBIJciUMqRSKb6+vi1nfmkgKFwUyDxcTXbEtqreQeiPFFKRg7DV5+/8+vO8F/Yesb9Y97KYtnIat2+8HSf35rH6zNMIc69MIsPDyQP2zISD99b9hOW5cP59qx6Bcqkcf2kn3NSdyS0REtgypYySzJJmJbnqinV3/0nQJWG+qKnHHwh2n/kBHYh55marAo2q+G3WbyzzXdZgZLDRWH1f16oQ971J8hEJjyZwe/cKC67gCdD/S/Do1CDjqgoxkCpWneO32D85k3kGgJC+IfSa0QuZvHq73ObAmDfHcNcey+plMTGsIZ8xUWPoHdTbRDiI+P7k9wz9R8WJyHmkpIDYQ1vpZts6ZO3ta/llxi8NMub8fPjhB/Pf2dmWSuUrW6/wz+J/yDprWcXgFelFfpt2lDkJN4ir0h3CpoHfkAYZV7Xo9zn0XCqML+8K36cs4VLgW60i6F4zcw1r5661UPzN6DyD/GfzmddjHgBP/vukxe+9cp+/otQi3gt7j91vWif0/Dr78VTWUwx8TLBvlEgkDHl6iEMKDaMRrlQ4oVXt8VVZ8Tfjxxk8lfWUxT2eGZvJN8O/4ezvZykvF85VXqH4M93rbQZC6OQax9FoiFsGRy1JaVeFKzKJMI/kqfNwdxc+M32JMwnbEshPzG+GgdYf8dviOfPTGYtnqZhkyFWcBgTVsaOB/IABQnuBGTPMa5uqCHEPwVXhis6gM1nxVYbY489E/NUX0XdCn3fBqdK9Pfx3GPk3KD3tH2cPbm2h7QLhPq0BnZdHsqtrN8qUqaaCSu8ob4b/Zzhhgx1TUTYnJFIJHiEeuAcLinGDAVJSoMSpfsRftH8AY3LWIDEo+P3CGj49/KnDx4o9g+ICX6DLp134+NDHFq+L8a67RCD+0oqFAzzDPRm4eGCraUXR2PFRXJxgIRYUBOMeaEdm2wEYZQryKowZkvcn88/if7iyxbbtZU2QSCQNmgDsdVcvgvsGs+9/+/iyz5doChr+IR7lFYUESbU5litbr7DrtV2ttq+rLUxdOZWZa2c6vL9EKqHd+HYonAV3AKPBWGv706KUIvLj8ynJctzHzmg0cuCDA2x4uPYF8NdhibrMLyIflpICBw400sCaCCJBdfhdgaD6paxpCCoxXi0vb93kqU4Hoi5KfOa+9847vLlkCbv0ejaXabgdGAPcDmzSaNil1/Pm88+3CvKvOmemoL5BtLuxXa1U8S6+Ltzx7x10vc3xHLlYYNjarD6bI6db6ysuXryYvn37kpeXh3MlenfatGls3dr6VAUNDb2+hVis+fZF7tUWpa6C+FMLyefJX03m+eLn7Sa/WgLEScTJydrB6LPDnxH0ThBPbrZPMvcI6EGQWxCl2lJ2JzpemQXmSbmyqujy5st8PfRrru4TZAMbNsD8+bBwoSDfzvD6C4BZXWcxwnkRAy9s5cXoDbg7NYxl2DUBI/i098HZp/X07KgtdAYd2VqB1RvYqZKPk8ofjiziBvkYwGhFXPUL6QfAsbRjNqu600+kc2nzJc6dO9dy5pcGgFatJfVIKnkpQvZMIqme4Kmuz1/4kHCiRkcROsi6atbZx5noMY1nrVITtGoVnZKXsiDmWUGlXBwv9OKpK3z6wOQrEPOE1Uv/aXOGkbGx+BgE4kimlLE4fjHj3hpX9+s1E8Z/OYO0dkORSIREfk0ICgKNux/Svr1rrBQLHx5O19ldGyTZYjTC228LPZriHSx+NtknVl0oe3YW1HlODWxBGrsU1gYiLUtDqYQEv8+Ys24qq0+vBoR1y9mzZ9HpHK/Kayr4tve1aAqu15uDwLa+kWyZt4WjC62tl3xdfNEb9WhUieh0kFnhNqfOU3N+/XnyrljaqZbmlFKa0zDlgt9+K5ACItmj1Vpa2MVMj+HO7XcS0N2y74FBb0CjAb1UiBLdlE3YXyn0ZggQeq2mFaXxfeIbJLVZbrpXL268yJ6le1pkEq8ko4SSzBILxZ+I10a9hkquIqM4w6SQAkvizy3InQGLB1jYOFaGVC616DUx5o0x3H/yfqSy6kOotDShGlaphNAqjyaR+CsoAJc2LlY9aksyS8iKy6KssMxsDVxV8dfcSNsEV7622CSRSEzK9nxNPh4eQhLy6N8n+H7c91xYf6EZBlp/HPzgIGvnrrVQHouVzpnYtxu2Bz8/+OYbuPNO+/tIJBI6thEcIWxZwU9oN4G3xr7FlI5TKgayB069BCU2Fkm1gUEPZRUyZYUHBIyqfn97kClhwPIayWmJRIKvs/DMK5flmArBXP1dGfXfURbzf0tF3pU88hPyTW46hYXC2qBEVPz51L2YsW/AYDqlvAnA50c/d/i4jAwwYiRdLrhD9AvuZ/G6aP85SvEMf876k/FtbVvKtnTo9fpGjY/EYs2OHaH33T3RjBB6sYtK/uhx0czZMIdOU2tXsLXz1Z2snbuWssKG74NnNBgJ7hfM2GVja1SYxefF89L2l1gTu8bh8/cK6oV6iZojC4/Y3efMT2fY/uJ2K2eD1gy/GD+C+9S9b/oH0R+wavyq2l2zsx9PZjzJkKcdLwCTSCRc3nSZE9+coDDF2sngOhxHXeYXsegC4NdfrVuUtBZUJqg2aZqWoKocH7eGAkR7qDx2lUpQTz77zDOsLytjgJ1jBgDry8p49umnW7ztZ3WKvxEvjmDy8sYvTmytir/myOnWmvjbvXs3L7zwAsoqUqzIyEhSUlIabGDXUU8YjbRxz8JVLzStE4m/lmjlVRXJB5Jpe/gnvAoSrF4T+/Z1amN/gS2RSEyqv9rafYqBUGaqlj1v7SH2l1gKUwrJPJ1p6ufy99/Cgj8tTajk6Jr1MltnH+DF4S+iUoFP8RCc9dU3Mj258iSfdP6EnIs51e53rcA72pu799xNn3v7VLufQWfg3J/nSNpbD2Kkjjj+9XF+vPlH8hPy63R8Qm4yBokOqUHJ0B4VgUHWfoHkabeQgohXkUl0JCZaLgLb+7THS+VFmb6M05mnrc77z+J/WDtnLZrWvPKxgZwLOSzvt5zjXwpKSFtEf2WIff5sKf6cfZyZt2UefjF+pm2aAg3/PPZPszeXN5T40C79GZ4f8JqwYcJhGL2p7ieUqQRLLxtWx+Li51pohh08qiPFvhGoVI61sBL7/KWnC72BqsOw54Yx6bNJDTBKWLMGdu0SngVnzzp2jKO9CxsMSi9BfaFX4+QESp2g6MhVCz0G9y7dy5pua8iKteOj24zQqrUWVlXlldx8qysUiPCMAEDjJGTtxIKBjFMZ/DT5J86vs0ykz/1nLvN3zK/3eNPTQayBe+ABM8FT2e7TK8KLyJGRVnZMP978I53+fAudTCD+uqnksMYbzjZBj79KMNk1yvNM92rsT7FsfW4rOnXLI4fn75zPHZvvMFmvKRRQrq+wafcMY8edOzj9wGkC3Mzrsqgo4f4pKYHkZAkT3p9glbgtSi1i12u7yL2UW6dxiWq/yEhBtV4ZlRV/WrWWq/uvkhdvJqOjRkXxdPbT9JjXwxzMqiop/koS4WdngehpLgxdIzgaVIFI/OVp8kx9szVubkz8eCKRoyKbbnwNiBEvjWDmbzMtYijxe0kz1J74cxTVWcH3CurF00OeZkqnCuIvaxeceQXK6jGPa4thfXs4cDdc+MRMADYyfF0qiD+5mfhrTdi2ZBsfRH1gsrTKzxe2a1zrp/gDYe0blnMnEiScyTxTo+uNiLQ0UCuTKDZmIpfK6RnY0+J1k8NN4Qgmd5xMiIdZerp60mpWTagdQdGcaMz4SIw5xBjEt6IuK7fiseDi60L7ie1rdAiqisxTmSTuSkTp3vCF2L/P+51Nj26i94LeNe57NO0or+x6hXcPOJ68l0vlOMmrf79DnhrC/Sfvx9n72in4rW+bgE7TOhE12k4z+xpQU6FRVcz5ew7PFz+PR4hHna53HWbUZn7RaMxzg0Ih9JE+bZ3aafFoboJKoTDH/q05/SWOXaEQ4oAVy5czSqGw+5mKGACMVCr5asWKxh5ivVAd8VdXJO1NYs3MNWScso4vbKEy8ddaSfamQq2JP4PBYJOhTE5Oxt39usKpxeDce/RJ8Kebm/D0Ea0+C5MLufDXhVpZBjQ1Cq4W4Zl5EWeDJXWfXZrNoZRDgFDpWh0mtptImEcYPs61a5gtBkLZuTK2v7idU6tO0euuXjyd+zQB3QPQ6QTpPsCSJfCf/8D778PoDgPo4NvBwpO6uLyYqwX2K28NOgPq3GsgQ9+QkMDP037mwLtN742QezmXS5su1TkI2xsnZPncdFGEhlRMrbunwZ5bIXI2Hv0WY0BBSYk5KQACUd0/pD8AB5MPUhWDnhzEDe/eUKcxtWS4+rky5s0x+A8QgqCa7ByrU/xVRuKuRDJjMzn+9XEOvn+QuF8buF9aLSEmBxtyUURZDmRsFyrzK0G8RuWqp9g1scSusbZAbemoLTlmmrt/3cGb7m9SnNH4PYePHYNVlXJjhQ4W1paVgVZWwNLStgz+arDZFvrfYbCtEaru298PN+wFt2icnUFRxQnAu503AcMCkKsct+NoKnw9+Gs+7mS2KKtM/JUb7a9jwj2FLF2ZNB+ttNDUYyGwRyA3r7iZ9je1b5Txij3d2rcXFAJtKtzSsrMt9zMajJSXWPYkDewbSoF/e5Piz0nhCj69QRXYKGO1wOGH4Bd30Jeb1k1aWR5qjZDkGvbCMO49cq+VMq0lQVQnShRqnF93JuTdEIrKihgQOgCFrMJqy2hkW/w25HKzCi+jUmxZufDn5Pcn2f7idjJjzUrB2uByhTtjtA3BuUj85eUJRTBfD/6ak9/ZsPuWSExzoYvMg27+3Wjr3RaQQsAYoQikueDkAzLrB7dIHOdr8hHDwlKpK33u74N/lwaypWxiBPUOotMUS2JYfM4mlwtZvW7+3Rr8umKR4/kca8WfFdo/CDeeAY+Yul9Q4QZ+Q6E0CY4sgkvL634ugOQ/YfNgyD9T7W6i4k9bhfj79bZf+ePOP+o3hiZAp+mdGPbCMJOln7jGL66w+hSVm3VBRAQodW0IMAr9x7deccxdKS0N8l2FmLl7QHecFZaLUFsONyLkTvIWuR5oDojEn5+ygG9HfovXVeH3nlulHiQrLovcy44Xidy65lYePPNgoxRkx8yIYdCTgxzq73cx5yIgFKE2JHw7+BLQPQCp/NpoUWHQG3jL6y1+m/1bnc8x4b0JjH6tdu1EDn92mISdCbW+Vmso9L8WIfJfbm4woSJVucZxMW2LQXMTVBKJOadxLRB/Yi7jz9WrucPBNzRPreaP1asbaWQNA3u5mvPrzrNm5hqyz2dbH1TTOfM0xP0aR+YZx2Iv0epTr7fMEVyHNWr9NL7hhht4//33TX9LJBKKi4t56aWXuPHGGxtybNdRH/j2pST4Pso0QgWfmOC7tOkSP978I6lHUptzdNWizfAYjt34AsZOlsHr1itbMWKkm383Qj2qb4Q9tdNUEh9N5MURL9bq2mIglJAkZcDyBUz5ZiogVFpJpBKSk4WJxdVV6NHRr5+5ChDM5MXW3K/xWurFY5ses3mdHvN68PCFhwkd0DANvVs6Mk5lsOu1XeRcqL56WCqTMmP1DAY/PbiJRmbGmNfH8HzJ87VuvC3iyGXB589fGSVUKRkN0P1V6LgYEKp9xCRwRpUilvk95vP66NcZFjHM6rwdb+5It9sbPqHU3HAPdmfos0Px6i78gKr286yKyoo/exU9BVcL+G70d2x4aAMDHx3I7PWz6Tm/Z8MNupbQaqFAdokC5xOUS/MFwu7KSiiwrt6vFWLfhK2jodiyp8iZ8r/Y2bk73xXMN23b8Z8d7Pzvzvpdr4mRdjyNn3osxT/+oMPEn/jbKlL60v7G9pQX2V79pR5J5dfbfjVZN9cVmZnwv/8J96JYbeaoUkGjgTJ5OtmGK8RmxSKXViTYnINAVb1avL5wcgKl3lLx1/mWzgx4b4BDfcuaGu1ubGehxBIX9XKFkdD3Qgh7L4yE/ASr49yd3E2WiGplkqlgQOWlovc9vfFtb7ZTVeepOfL5EYerC6uDmCSMEASHNok/vVbPa6rXWHfPOotj+z4xgvhe09HJigDIV/jBmK0QNbfe46oRrhHQZjAYysxWkhIjReUCm+3b3pfgPsEtLolnNBg59tUxkvYmmRR/+SRiMBooKiuyskt9ecfLjFk5hg0XN+BZ0bZMJOz/euAvPun8CSWZAqE85OkhzNkwh46T65a0F4m/qv39wNLq06edD2OWjqH9jebE64lvTxC/XVhTiIF1F908Tj1wiqVjl4JrGIz8C6Ln12lsDYLyfMg+YKUKExV/RWVFJsVfaWnLJYwdgS2L29JSMKBjtN/tjIseRxf/Lg1+XXuKP51Bx+9nf+dUxikMxorkvtILvLqAvJ5VRoNXwri9MGhV/e8vvQaKLoI6rdrdKiv+KhfQFCYXUpRWVL8xNAG63NrFoj94fr5gW6qRCr+Ndj7t6nxuk9IsfywA+5P344grd0YG5LsKxYQDQqzTt2K8ezU3ix9P/8iPp380vTbzt5nM+mNWncd8rcBgwFQ05ONUTMapDJyNAuNfWcWfF5/Hp10/ZduSbbU6f21Vgo4iZloMY98ci6ufa437XsoVqpVqe49+dPAjRnw7glWn7CtDy0vKKUpt+b9fR2DQGeh1T686K/bqgvKScjY+vLFOxdAFSQVc+PvCNWW12hog2nwGBcHUqSCVwokTZgeI1oKWQFBVFlO0VlQlxgoKC3G0lDMAKGjhFgj2FH9ZcVnErYlDX157O8vocdE8V/gc3eY4lvus7AzV2vr8NTVqHcG/88477N27l86dO6PRaJgzZ47J5vOtt95qjDG2KjRHo0ab8B+ObODnpOYJnv6i4i9ieASTv55MQLfGTS7WB2VlgESCk8qyWmlnopC8HhM1psZzyKQyi2qnuKw4s6qiGoSHC4k6tRo++jWIV57IZ+OSPRQmC5FoZcuPovJCIt6PYMjXQ0y2UuLE7mPohN6oZ2fiTnNQ/v8YKYdT2P7idqu+SrbQdVbXZiNEZYq6J6eScoXVXrBbhV2ORCr064qcA9oi2DSA+T2eAqyJv9ndZvP8sOfp6m+7ma1UKiU6OrrlzC8NCFGlUZPiLzhYWECr1ZYBd2V4hnky6fNJTF4xGYlEQodJHZq16lGthsuBy9jdpRfLT30IhefhwJ2QtrF+Jw6dCr3fBaWnxWaF0kCRy2nS9WaF383Lb2byisb3WG9IyBQyPLuEoHVyr/G+ECESLCk+3Zj15yy7JFb2uWxif4lFnVc/tfUffwgNuzt0gFtvFbZVVvJWh7IyKFMIk0CAa6Vn8dBfhKRrQ0NbCHFvQcrfqFSVFH8V64KWPL+MeX0MN7xtVjyLxJ/O5SoFZQWkF6cT7G6750qEl8C+qZ0STck7EZXtmvIu5/H3A39z4e/69x8TCUYxWWuL+JMpZHSZ2YWQ/iEWx4pzYWjJZLbfuZ2XRjShjWPnpwULYoU7TnInnOUCm12sF8hho1FQKDqiIGhKaEu1rF+wnuMrjps+v3wEwizKO8pq/j+aJvSD3JW4y0RKiURDu/Ht6Dq7q+nekEgktJ/Yvk7PEKPRmvj75vg33P3n3Wh0GhPxV1wMUiclQ58ZarofDHoDGx7awJ439gCNpBpvCKT8DZsHQaZlYcnvt/2O9kUts7vNxsNDCMhVKn9WjVvFDzf+0EyDrTuMRiOvO7/O7/N+t9iuVoMUOY92fpvNd2xulJ6cndp0wl3pbtUzPKkgiem/TKf/8v6VBpTWcNaccheIuh2c66k2Dr8VZmRBUPV9hn1UwvO6qtXn3Xvv5o7Nd9RvDM2A/HyQGp2YpVzFW2PfwkVRt4JCMD9LAq4+wIF5pxlW9BEzZsC+ffaPKSsTFGmi4k90FakMkfhLL7/InLVzeH7b83UeY3OiMdcvaWlCAZ+TE3QbH8Izuc8QNkP4LCsr/ryjvBn58kgGPjrQtM1oMLJ32V6bRWb/PvMvFzdebPDx1gWX8upG/F3Ju8KuxF2cSD9h83Wjwcjb/m/z511/1neILQJyJzmTV0ym7/1963yO+O3xrL19LdnnHFPByFVyFhxYwPAXh9f6Wpf+ucSPk34k7Xj1RRfXUT1qO7+Iir+gIKF9UJ+KDjdnqhe9tzi0BIJKXPO25vYlVYk/Tw8PHDVFzQA8PT1r3K85IX43VYu0hz47lBfKX8Cvs5/1QTVA7iRH6ea4+5pE0jpb3TRHzqXWVwwNDeXkyZM8//zzPPbYY/Tq1YulS5dy/Phx/P1bp4VLQ6IlSeudnMBdEkTvy7/w/cS/MBqN+Lb3pdddvfAIbbme39lxmXhkXUYltyTqdiXuAmB4RO0WQP9e/pf+y/vz6D+P1rivSgWffAIzZ4JSrqf0330cemOrqWItIUHYLyICrhZcJakgibisOJQyYYISk9QB+r64KFzILs0mLsu21WDsmliOfnm0Vu+ltSJmegz3nbiPsMFhzT0UmyhMKeTwF0fJjbdcuJSXw8GDOFRdm1YkJPJDPG2Q6nI30Bbg5iqcqCrxVx3O/XmODyI/IOtgVouaX+qLgx8d5KvBX5GfKHzmNRE8crlA/kH1dp+9F/RuMcoltRo0CkFdHeoZDB4dYfifEHJz/U7sPxQ6PQYqy2dumIfw+8rH3AgxfGg4oQNbl7LYv6s/vd+7g7zgzrW2+iwpqX7h131ud5aol9B2nA0ZTi0g9myYNs18bUetPjWaSsSfWxMU4RgNcOJZuLoWZ2frHn9ZsVns+8++Fu0EIEIkd0rdhC+gU5tOpudvVYh2n2plkqkKF2DLc1t4w+UNNAVCRObbwZe5m+fSZWb9FTuJiRXXrtIPqKrV5/RV0xn0+CCLbTue2UibxKN4KfwZGTmSfs5yOPmiUDDQxBBVfyWGPAwGOPD+Ad50e5OUQy2rl7fMScacDXPo/0h/k+IvWy8Qf5FekVb7j4ocBUBiQaKV4q/T1E5M+WoKKQdT+GP+HyblX12QkyOcVyoV1otqrZq7193NNye+4YdTP+DmZu77Z6tg4I4tdzDyvyMBO1Y6OUfg+FNQ0IxW1r79oOcy8LT83bgp3UwqZoFclVBerkLlqbLqa9kaoC/T035ie/y6WCYwmoKQ7R7QnYJnC9g017Iv8OVcgVWO9o5GKvb73XMb/N258QZTF9joRWwLouJPW0Xx1xqQfjKdL/t8Sdxv5t9ifj7IDW6M8budp4c8Xa/zq1RCAtmlPJITm7uyapUEgwH+rIZPycgQ1KgFrkKMaUvx5+oqJMtU2iAA0orSTMrWrLgs9izdUyvryuaCRCLBw8PUOQfLAAEAAElEQVSjUeIj8XkeFmZWFPi2Ef6nagHiiP+MsFhrH/70MFue2WLlJFBwtYB9y/Zx9jcHm0I3Mupq9RntLXhYx+fH23xdIpXQb1E/2k9qHFv11ojC5EJOrz5NzkXHCjSkMinBfYMJ7mu7uK06RIyIYMo3U+qUeL8OM2o7v6RWhFFBwrRKVIVANCnJ9v4tFS2BoBLzQteS4m/KnDl872BiY6WzM1PnzGmkkTUMxPdnax0sU8hq3ZtURH5CPqd+OIVe65hiUCT+WpPirzlyunX6NuRyOXPnzmXZsmV8+umnLFiwAOcWV4raPLDV/7BZoMlGsm82UzqtJjjvVrq6jm41pEHS2sN0OLgKJ8wzfXZpNrFZgoplaPjQWp1PrVNToi3hk8OfsDOhZss7Fxe44w5Y2O8EvqlnyGw/GJ8uQt2LGARERMDVQoF9EJPtYJ7YdWVKhoQNAWBHwg6b19n/zn62v7i9Vu+ltcLZ25nAHoEO2Zr8df9fvOH2RpMqC+L+TmDD/X/xwZNXLWwkP/oIXnsNNjog0PLIGUVkxiKGR1bcn4cXwYbuoC0WIsZJ5zjv+h5gm/hLLkzm17hfTUGYCKWbEhc/F+Lj41vO/NIAKCsoIz8+H61RyH46ouyqbPfZGlBaCmUKgXEIcgsCJ18InQzudbd9qg4R3sJcVCrJpExXZtpuNLS+bse17fGnUpl93vd9epw1t66xac8GQhVtfXqVFRWZi0C6dcNEIDha8FhWBmVyYRLwd60gb0uS4NRLQkK/oaHwhBsOQs83cXKy7vGXF5/HwQ8Okn6yYZuzNwQ2LNrAvrfN0gaR3Cl0rrmv1ujI0czoMBuXskgKCwXbLhCq89ve0JayQuE34uThRNtxbfFpW7+CAZ3OHPRXVfzZUymL0JfrObfyMB7ZV8z3fN4xiH3NytK3UZB3Ak48b7IhFvu0aeW5lJVBQLcAetzZA5V3yyJuZAoZ7Se2J6hXkIkUztEnABDlZW3JJapAE/MTTYq/qr/bixsucnbtWbvzhyMQ1X7h4YKN9e6k3abXDiQfQCKxnDd2vrKTd4LeoayoDKlMStigMFOhlDgXrneeQcwnMWy5skX4vs6+LcwbzQWPDtD5KaGgxQ7c3QXF3OXLWdzy6y3M+HFGEw6wYSBXybnt99sY+oxl7KFWQ4HLMbTymp0s6gqJRGIzdrucJ9xgbX0qFbCEzYB29zXaWOqMjB1w6ctqdxkeMZwpAYvxLRpp8XtMP5nOsa+OUV7cchu3lBWUUZJZYmFpJZL5orK3vhCfJ3/9Zd4WFydYjttCWhogMXJTyS+8POJluz0G/fzAqYL4K9OXka/JB4T2DFuf20rasZavFtLr9Zw+fbpR4qPKLj/x2+I5v/48Pt7Cc6Fqjz8RyQeTKUwppPeC3ox6dRQ97+pp8bpnmCePJj7KsOetWzs0NUrKS0grFr7j2ir+oryF5+uVPPvrk3FvjWPAwzV1CWsdyD6fzdq5a7mype7rsc63dOb5kufpeLNj9uE5F3McTnxXRZuObeg5vyceIS230L81oLbzS2WrTzDb/reWvIWIlkBQXYs9/hbcey/btVoO1nDcQWBHeTn3LFjQmMOzQFoapNSyttOe1Wfi7kRSDte9UPTol0f5fe7vDqujxfyPWJDXGtAcOd1aE38rV66s9t91tBBIJJD4E+3bCNV+YhCSn5DPB1EfsPvN3faPbWZ4Du1OQrebcPI0zyIquYqVU1eyZNgS/FxrV700ueNk7usjBMP3rr8XtdYxHXCPSaHkdR5MZlAPriQISWKR+IuMFIgagDBPM/FXuTplZORIwD7xd+PHNzJv27xavZfWCk2BBk2+xqFEmlekF+FDw9GqtU0wMgGF/u242HcWp4qiOFohwkxLg50VPHFcDUX1Oh24JE6n69WPuK3PRGGj3BkkClCY7Z8CKsQ9toi/xzY9xq1rbmXt2bUW26PHRHPPwXto079NXd5ai8XwF4bzRNoT4Cp8Po4Qf2EVP7XqFH8tCZUVf/bsCOuMPbfBv5aJyCBPX6R6Yd4U56ff5/3OK/JXWpxFX3VI2JHApS+3oVAXOmz1CZXsPo9lkLAzwWZvi8ubL9e7l1tshZNqWJiQvLdHINhDWRmUV7X6LIiDM69A3vF6jc0mJBJo0x9U/jg7g0oXyMLQj1lx8wqMRiMRIyO44Z8b6DKr4XtU1RcnV57k8ubLpr9Ndo5Op4Dqib/FAxfz062r8S+ciNForgTss7APs9fPxjNMYF70Wn29SB4RqalCD2BnZ/O9aMvqEyDu1zh+m/2byXJWppQxae+zXO0ygRTPtXx2+DMuufWCm+LAr3bFTnVCwVmIexMKhJv7l1t+YdSZC/gUDUOjgeix0Uz9dip+MS23elwkhTPLK6w+bRF/nhXEX0GildWniElfTOL+k/fjFlB360axn0u0IIrghrY38N54ofBna/xWwEwK5OeDwlWBZ7gnmjwNJVklFvO1GMjmSS9yLvucYB8fOQcmx4N/8yePq+KfS/8w69dZvLf/PVMw3prsdxxFSamRg+0nMOgPH46lHWvSa4uKv7belYi/Touh+ytNOg6HcP4DOPwg6Mvs7nJj+xt5qtv7BOZPtXiOxv0ax/oF6ylIarl9biKGR/DY1cfoOsts1Z+fD5keG7lo3Eyeuv7EsEj8FTqfIrHP7VztuRCAXbts75+eDlKjgkFtbuKlkS+ZVaFV4OcHMqMKN5kXgIkEih4bzYKDC2g3vnGK1BoajZVAq0z87X5jN2vnrMW3mmKe5APJfD/2e07/cBq5Ss7wF4Yjd5Jb7ecZ7ol3tHejjLk2EAsIfJx9TMU+jkJUCF7Mudgg66eWjoKkAk7/cNquCvZy7mWSCqpndxTOChQuCoeuZzQYWTFgBd+P/b7WY72OhkVt5peqxJ84dycmQmv6mbQEgupa7PEXGBjI0rfe4mYnJ7uf7UHgZicnli5bRmBgPe3WHYRGA088AU89ZW6pURMMBvO+VTni9feur5fNc9dZXZnx0wyHHQpF4rE1Kf6aA7Um/hYvXmzx78EHH2T+/PksXLiQRx99tBGGeB11gtIHbitjW+EnZHn8y+oLXxCfF4/CRYGzjzNKV8e9c5sassgwsiP64uxmVmS4Kd24o8cdvDb6tTqd862xbxHkFsTF3Iu8uutVh44J7B5A4B3j0Hj4ExsrJC5EwiY8XLD6BNuKv7IyM/Fnr89fcN9g/Lv8/7DH3fLMFt7yfgt1Ts3Zn6HPDmXuP3Nxcm+cpufpJ9LZ984+i0DlYooLBYEd0Tm5snq1sDj77TfzIi3etpOJ+Zzpwr4qVaUK317/g4mVrFwLL9KFNwl1P2eT+BOteA6lHqr7m2uFsNfjLz4vngmrJrA93qyKbW2Kv8JincnSMcg9CGLfgDWekB9bw5EOQKYCmbNFJOHqKsFZK8xHYgAa1DuIzjM617lqtDmQuCuR1J92I9eqHVb8gZlk8Z44iEXnFuHq52q1z6+zfmXjw/XrsSj2auhakeerbBnoSGBnYfUpEn/+w+HG00L/xsaAXgPFCTg5gczgwhj3h5jdbTYSiURYFwQ4o3B2LCHRlHgi7QluW3ub6W8xyMhTCoq/7gHdqz1eLjdXAtqzkPv7gb95XfU6BVfrl1yunCQURTqVib/K90ZmbCZnfjpj6h8MoJMq0arcOev6GQ9ueJCDGWfBMwYUlv29GgXBN8KkCxAsFK509o/Bl/bIjKoWHXRf3X+V151f5/Cnh033RnqZfatPcVtqUSoqN+HhU/W+kEgkeEfZT4JqtTVXlVbt7wewoPcC5FI58fnxXM69bJo38vNh8BODWXBwAZ7hnqy9fS1vB7xtUmqLn78GgUDwVnkLPdjcIkFuPcc1KbaMhAP3WGyKz4vn59if2Z20u9KaWErygWR2vrqTkqzWFZlnnMrgn8f+seqVlKOPp1yRhVKqpItf4xVNrDi2gphPYliydYlpm0nxV5n4a6nosgTG7gJJ9Sr7qta7AN3mdGPOhjktuj2FiMrKzPx8OBv6DM/HjedA8oF6n7tjhUDIN6CM05LVXHL6BQM6U3FiVVTuM1UdRJtyD6nZ7hPApY0LIf1DHHJpuZZR+Zk+6tVRTPl2Cr6+wvesVlsXNIQMCCFkQAieEcLNrM5Ts3LMSrY8uwWA0pxSzq4922IUrJ39OnPp4Uv8NfuvmneugmjvaORSOSXaElKKbCs7ru6/yqoJq4jfXkMQ3QoQPTaaJeol9Jzf0+q1lMIUenzegwErBlCur/67TT2aSurRmm31dWU6+j/cn2632y9uqw7x2+N5P/J9Ytc0QLx5HQ5Bq4WsLOH/xbk3JESwfC8trdn5oyWhJRBU4vqxNReO2XIveuyJJ3jujTcYLpMxXqXiB2ALsAoY66RiuEzGc2+8wWOPP95k44yNFdyMioocv08rfy9VczWjXxvNiP+MqPN4AroH0PW2rjh7O+Yo2RoVf82BWhN/eXl5Fv+Ki4s5f/48Q4cO5ccff2yMMV5HXSCRgEyJlxdcDHqND6/cz6GUQ7j6u7Lw6EIGPNJyrRdqa/HmCDxVnnx606cALNu7jJPpJx06rktFLB8XZw4AvL0FhUd1Vp8aDfQL7ldjn7+ywjLUua34ieYgwoeG0+ueXijdm59w/vepf/n3yX9JPy5Exga9gXNHi02vX7wImzfD1q3mY1JTq684SkmBIlUcXiFZgJ3Mf9EFAtKfp4v/LrKyzLZzIvqHCA3jDyZbLrGMRiMH3z9I8sZkh99ja8D59eeJ+y3OLvH3wcEP2HR5E49uetRE0lZW/LWGyrm0wkyQGJEYZfi5+IFzKPj2B6VX/U8+6DsY/a+ZYUCoeHIuF9hRcX4a+OhAbl1za4skdexhwOIBRL79EBpX31o9B8QkVpHUE2cfYbFYmfA0Go3c9OlNDHpykK3DHYbY308k/kTlkMEAxcW2j6mMsjKQ6z0IVIUT6lHRE0buAl5dQdVIiqoDd8O6KNychYms8nym1+opTiqmON2BwTcxlK5Kiybf5eVgkJSTJxUsKbsFVJ8U0Rl0yLyEpFTlhPL2l7az9397AYgcFUnU6Kh6WyJVThKK8PExj7vyvTH4icEsUS8hoJtA/GafyyY7NgMMBsqlQkW5v0IB6nQwNAFpr/QEj/bCfVgBcU4uK4OcCzn8Mf8PLv1zqfHHUgsoXZXCdxfmYSL+evoOYXjEcDr4drDav41LG5zlwtxQKhfmSEeVuiLeegvmzbNWcVaGSPyJij8QCtgGhQpzz5YrWywUf5URNTqKbnO7IZEKc7v4W1WLxJ+zt3BflCQ2zb1RHfQaMFgqubxUXgDka/JN87dWK+Hqnqvs+M8Oci+1/L5hlZFxKoOD7x8kPyHftE2ng0ylQOh0D+iFk7zxCJJyfTnnss9xOvO0aZuV1aeuBLZPgIufN9o46gzfvuA3GKTWyicReoMejSKNYqcLFr9Hvxg/2k9s36IJqL3L9hK/zZLYyMs3UKKq6J3mW/8eZ4MGwYsvwtev9cZL5UWJvoBij6MkJJidaCojLQ2u+H9AvHI9pVr7mTD/itpTV0MF8VdsJreNBiNJe5NsOif8f4BOZ7Y9i4iAsEFhdJ7RWXBNqJjXqtp9SiQSbt94O11uFZIHKi8V+Yn5lBUJc+SF9Rf4ZcYvnF3bMvr7yaVy2vq0ZVBY7dfECpnC1OfvXPY5m/voy/Uk7kps0YpdRyGRSJCr5DYVnEqZkhJtCenF6VYxfFV8P+57Nj+xucbrKZwVjPrvKPos7FOn8cqd5Lj6udoc73U0DjIzrYvAFQoIrjD8aS1FyyIqE1Sj5ZYE1Q2qxieorgXFn0iOVc1lPPb44yQmJzN8yRLej4lhnkcwT7rF4D9tCYnJyU1K+gGcrJQWt2djXRXi9yKTCfd5ZXS+pTNdZta/IM5R9zexx9914q961K3jYhW0b9+epUuXsnjx4oY4XauGVNogH2nDIPco7bz2otAJmac8TeP1oWhIpC5dSfuDq0yTZGFZIe/uf5cjqfXrfTS101SmdZqG3qjn08OfOnSMSPydPWtWfYl+3Sbiz47Vp0Km4Lmhz/HxxI/NfZwqIetsFks9l7LvnX1Wr11r6D63O5NXTHZoAZp9Ppstz24h9UjNFXF1waQvJ7Hg4AICewnVSVePZdPmm3cIurCTUaOEfT75RAj6unQRErdGY/Wqv4RkDTu7dmGFt7/wO8s5DKf+Y9mbyW8ohrH72JsyG73euqKmd1BvpBIpKUUppBSaqyclEgm7Xt1F1uasljW/1BO7XtnF5sc32yX+Nl8WAqNTGac4ni7YH4aECDxXcbF1srQlIrlAuIfdJAHIpDKInieQdS4hjXI9Z2dwU8fgru6CxNh6iL6qUHmqoE0bjDJ5nRR/YkL+9I+n+ajdRxSlFgHCb6nrrK4O99ewheJi81wgEn8KhXnRaU9VVhkaDXROfpuN4xO5p3eFWkaTCeV5jcdoB98EnZ7ARSWwI7GF+1l3fh3ZpdmUZpSydcpW9r+9v3GuXUcY9AZSDqWQn5hv2lZeDnppKT2NCxgXPc6i8KYqskuzUb2mYnVgGAZJucV3c2b1GU5+J0Q63W/vzu0bbzcRLXWFGNSHVRqSUmlWslQmipRuSuQq8/Nwz9I9HL77c2S6MjQVxF/XzLXwexBomqDPkkEPpamgEQa5I2EHZwKWkOb1OxqNYNd98ruT9bbJbWgEdA9gzt9z6HhzRxPx93yvD9g5fycxfjFW+0skEub1mMeDfR/Ex1MglB35zYooKIBDhwQy9JztXCeFhebvOjoaHtn4CNN/ns6hlEPc0PYG+gX3w8PJw4L402v17H9vP2d+OsPQZ4cy8YOJpvOp1WCQaCkzCko5b5W3sL74MxJ0tRh8Y2D8ARi8ymKTaBmXr8nHyUl4ZiuVnnSf252FxxYS2KNprIsaCp1v7cyjSY/SdpxZXadWQ56rQPwNDhvYqNfv1KYTYE6uG41Ga6vP8nzI2gOFFxp1LHWGQQ86+xmZi7kX6fdTMHtjBlBcLKy/K6Ol9ilW56rZ8swW07MEhEd4RmkqBqkGuVRuU3lcW0gk0L8/eHnKGB01WtjWUVBp2bL7TMooIi7sMf57YTIFGvuki1gspSy3VPwBnPvjHN8M/YbTP562dWiLgVQqpWPHjg0eH6WlCfehSgVtfI0WLjG+QptkmwlSmUJmWktIJBIevvAwN31yEwBtb2jLxI8m0v7G+pPBLQGd2nQixD2E4nLbRWMRwyJ4vuR5et7Zs2kH1ggouFpA8sFkykusFX1+rn7M6joLMFt528PYpWMZ+GjjPjMAwgaHce/he+k4ue7xznXUbn4RbT4DAy1qcludW1FliARVSa8lPOEWw71+wXwQE8OIJY1PUF0LPf7EHJetXEZgYCBLXniBw3FxvPN5Cv1GxjFw0AtNZu9ZGccrdRnJc5AuqCzUsdGKut5Ye/ta3gl8x6H1n6j4a01Wn82R022wK8rlclJTGydRfx11xIG7GK64C4XeC8DUtPvI50c4/nUj9BFqIBidVOgVKhMRsDdpL09sfoJZv86q97nv6H4Hnk6eyKupPK2MyEghoVtaau73JhJ//q7+hHmEmXrGgKXVJ8ALw1/gof4P2ST+vKO86T63O0G9a/Bi+X+GwuRC9r61l5RDdW8KWx28o7wJ6R9isuVJSJKSFdYTp7ah3HOPkKgV47tbbzVX7FdH/F1IyQRAhkJIymUfgDOvQkmlVZ7SE6n/INy9Bdu2qnafbko3k13U4dTDFq/d8e8dTP5qch3fccvEhA8mMPmryTaJv6sFVzmbba6I/eb4N4Dw3Yh9ElvD48ZZH0in5KWMUD7W8CfXFkPcMrhq7gnp7Axdr37IiNgzTI6aDQh2LxsWbbBQK7R0FKUVUZIhJBLq0uNPtFpxcnfCoDdYEEf1RVycMD94haXwT8pqUzW9SO44oh6ySXYfXQy/+oChkSygom6H3m8jUwmqtpV59zLlpymcTD+Js48zg58aTNQY655ozQltiZYVA1aw879mPzOtFhR6L6bIPmPzHZst7NWqwtfZV3jWS4yoFckWBM+8rfOYvGIyxRkNp3K0pfgDc5KwMvGnL9eTcijF1Li825xuBM0djV7pjAYhm2hsMwTaPwQKrwYbo11oMuCPEKHPJLA7cTcnPd4g03MDGg0E9QriueLnGPzk4MYfSx0hEn/KGowFPp/0OZ/c9AkxwcIXVVws9GZ0BMeOmdcHaXb4WLG/X1AQKFU6Vp9eze/nfqe4vJglw5Zw6N5DzO4220T8FRSAVC5l+4vbOf6V9dpcrQatzByJe6m8IOgG6PQ4yJvABraWqKz4E+c4rVaKW5A7Qb2CHO5x1FIgd5LjGeZpoTxWqyHPTSD+BjUy8dfZrzMAl3IvkV2ajcFo4Nup37J0zFIzqeQSAjOLoffbjTqWOqEkEX5WwumX7O7i6yxMklp5PgZ0FAm1OmSczuAN1zfYs3RPU4y01nDycGLh0YUMfto8L5aUQIFcUEZHeEY6HG86ils73wrAGcVXGCRaduyATZvgs8+Ef7t3w8WiYyAxEuIWJtjM24FI/LXNWcSfs/5kZpeZptfaTWjHkGeGED022s7RLQfKmib9OkBUUoaHQ9rRVMFS+jMhNhOf6Y5YolUuKHIPdqf/ov64tHGp5oimw+u7Xue/O/5LfF7drDh/m/kbyY8nM7XTVJuvS6SSatdorQknvzvJVwO/IveibTnMmKgxgKDmrw59Fvah09RONV7vu1HfsfGR+rUluI6GgaPzS9X+fiLEvKEtdXZrQGBgIJ1iXqDfyDi27E/hcFwcS15ofILqWlb8VUVohflPcjOYe+XnQ0KC+W9HiT/xvYkErYj0k+m8F/4ex76qX+/rgJ4BtL+pvUPW2NcVf46h1qvRdevWWfxtNBpJS0vj448/ZsiQIQ02sNYKQ1X/vuZElyUkxRlR6ISeYSLxt+9/+3DydKLX3b2acXD2IZ8zkyu7YUzFJLkzUUj6DY8YXu9z39zxZnKfybXb6LwqpFKIiYGjR819ncQH+A/Tf7Dav7I1Vk2Qq+RM+36aQ+No7dj0xCaMBiMT3ptQ476hA0NZdH4R7iENn9Da9PgmLmR5c0Lel0VTkuk03I+k0jYk9pjC6NFC8v6mm+D33yEqCnr3FhL9R46Yrbts4UpmBijARxkgBDlR8yBwDLhGWO6oK6V9SA5paWFkZJgVQyL6BvfldOZpjqcdtwikAnsHcvr0abwMXshk1fdJaS0IGyzIYratEP6uTILoDDoW9FrAvuR9xGXFsfnKZoxGIxKJBC8voX9JbVQadYXBaGD/1f2UaksZGz221gGsiy6UdunPMFGcas9/BLoi6PJ8/QcnVcDJ5yFkMoRNFzZJhc+xrExYkHl6QubpTA5/cpguM7vgFelV/+s2Af648w+ydiXDuOfqZPUpEiwdJnXg4YsPm2xO9y7by6GPDzF301z8YupmqXnmDJTJs/ghIJxVaw08Nfgplo1bhoeHEPQ5okS1SfwFjAJVAMga19JM/DyVRmF+LS4vRu4ix2+OH+26tWvUa9cWUoWUsW+Nxb+ruXDGnkLYFiQSCeGe4VzMvYjaKYmCAnMC0y3QjZVjV6It0fLIlUfqbYek0wnFCEaMHNasRpvcloGhAiHg5yeQQZWThGWFZawYsILe9/bm5i9vpu0NbTma3xbDKi0ahMlNGTUbXNrUa1wOw8kHOjwM/kJPBh9nwSlCK8tDoxGIKaW8+a26qyJxdyLn/zxP3wf6Ul7ug15ailEmA2q+QdzdhSpVo1Hoa2Hqz1sNjlZq22uP+BOLhKKjYWfCTnLUOfg6+zI8YrjFM0S8Xl6ecK/O3zGfC39d4JdbfmHiRxNxDxJ+oxoNaGX5AHg4eQjq8fBbhH/NjZwjgtIs+i7BLhYz8ZenycPJSfx8Cykrc0Oq06Ev1+Pi2zIS346gIKkAg86AV6SXKYmfW6im0FkgacXfeWMh0C2QHgE9OJlxkg0XNzCvxzxu6Wznu3cwtmlSOPlDyBTw7Gp3F1ElCqCV51JQ4I+3t9BrLnxYOJ7hnk0x0lpDKpdaFW/m50Opk0D8tfdt+Gfq9JjpBLgGkFGSRq7fH0gzb+Xjj82vb9gA2YGC3eCA0P7VnktcM0lTB3JzB8uqfYWLgrFLxzb08BscBoOB06dP061btwaNjyoX8kjlUqJGR+EV4QWYLbwdsUQzGo0c/PAgxenFDH9hOErXlvMc/ezIZ6QUpTCh3QSivGtf+OUIqX11/1UKEgvoOsv+7781IHpsNDInGR5hZkt4o9HIQxse4uYON5uUuAdTDlJcXoyb0q3O19KV6dAUaCgrcCCZZAclmSWc+O4E4UPCTfH2ddQetZlf7BF/rVnxJ0JsVeBW99u61rgWiD9H21eJxJ/YyqYp6yVOnbL821GrT3ukpkFnwC3Qrd7PuiFPOc4ricRfa1L8NQdnVOtMx9SpUy3+lkgk+Pn5MXr0aN55552GGtd1NAQibqM8DxR6wfpFJP5mrp3Zoituq06SuxIFH5MREXVvEiqiLpWXnTtbJnsiI+3vW/UhpdFpOJVxijx1HuPbja/1ta8VXP7nMjj4EFO6KvHt4NvgY9CWajnxzQkyncPRBTqz7tu18NVkzl0WWJmYClewOXOEhc2QIaDRqcnw2IGBccTH2793ruamQwD4u1bI0ZSepiSYBXZP59F2u9l1pIT0dOuXO/oKlhxi/5bKUGeqif0llu6zu9fujbdw2ErkR3lHsXzycvQGPX9d+IuJ7SeaEqZiP7XGIP6ySrI4l32OS7mXOJRyiD/O/0F6sfBFvTziv7w08j+1Op+4KBIXJMR/B2U5DUP8yZzghgPgbplYcnERPlOx6qnnXT3pclsXU8+71oCY6TEk6oSmCLUh/iorq8SFs0j65VzIIfdSLi5tXHD1c63z2M6cASedHxKEbp6bLm9i2bhlpiS+I/dlUVkx27p155Z//Nm/cKfQH6rdwjqPySEUXYZjjxPBbGAWigrir6i8qHGvWw8onBUMedpy4V9eDiVOl0ARAtR8T0d4RQjEnzLR4ruRKqSMXTqW0pzSBumBkpoqqMayA3/kgS1zcVW4UvRcERKJxHRfikpUAGdfZ8YsHUNIP7Ptb2WCB8wESpNApoK+H5r+FBPxWnmeYDWpN5B+PB0nD6dGeT7XFamHU9n/zn5iZsRQXu5DYpvldFn9GAt6L+DLm7+0e1y+Jp+isiLc3MIoKhJ+tzURfwaDoPgTYesZDkLgDoLl6x/n/gAEq/nK68+isiKKFVlAtKlYILhvMCdXnuT8uvNMW2kuClOrwSjRE+XSjUCfJsy6OILUDYKSK2AUKHsA5vu2QFOAQmlAXPwVZmr4PPJtet7dkylfTWmmAdce/z79L7E/x/JC2QvIlELi72jqcYxSHc76AAvXj8bC5I6TOZlxknXn1zGvxzzrHUoSIT9W6CGsaqJiAUchd4bha6vfRSrHS+VFviafcnkOhYVCsYd7kDtz/5nbFKOsEwqSCnD1d7Wwbc7Pr3hGAe28G574U8qU3Nv7Xl7b/RpFnT4lxngroaFCoYFOJyTxjkiEgt+BoQOqPZePj7BO0umEAgSR0KoMg95AUWoRnmEtk3xtLFQm/oJ6B3H7httNr9WG+JNIJBz78hhZcVkceO8Ai+MXm4o6mhOl2lJSigRnnYboQ2kPu17dRfy2eLrM7FJvO/XmROjAUEIHhlps25Gwg8+OfMZ3J78j5fEUoryiiM+PZ1fiLm5sf6PN8xz/5ji7Xt3FrD9nmXo8V4XcSc59x+6r1uIuIT8BZ7kzAW62z1GcUcyWp7cw8r8jrxN/TYSaFH9JSU1P6jQEysvNjhrNQfyJuZTWiOqsPiujciubwkKzi1BTQLT5VCqF77m2Vp9VFX/BfYK599C9DTY+fbme3Eu5+HW2X7At5tlaM0ncFKh1tqNFKdquo0Z4eWFl9dmS+2to1Vq02/bjXhaKShVNqbbUZHvYEIq/ylBr1Tgrak4cin3+RIRVs34SJ3a9XgikEvITGLBiAG5KNwqfLbRSDMX9GseRz44w5dsp13RQ9cCZBzBoHZ87itOL0ZfrG7TKV+GiYNw/j/PaklIMMgXpHYfj0j4E7TPf4BfQhc6dhcpYlQpmVrjdTP1pNn+e/5OYgLdxSngCvV5oYlsZZWWQpRZ8O0O8KhbgmkxAAk5tLFd4YdOJz+iKVKIjM9N6+p3aaSrtfNrRLaCb1WvHXz5OzuEcIodH4hHiYfV6a4LRaOQNlzeImR5DWR9BrWZLwSOTypjSyTJB2FjE318X/mLKT1MwGC3vU4XBDR0a8i92gZG1O+elwjMUOOswKqMBDxi5EXQNWI7k29dqk941mZ1BNzLst0JSnkrAyd0JJ/fGVZE1NPre35cNmcDh2hF/otWnRiNUfYkBirZUyzfDv0GukrPo/KI6Ez0lJXBJyOdx5I4r9P4+ktMZp8lV5+LhIWSCHLH6LDJkUOoUz7m8dJSyJqz+Tt2Ah7twz8j0biATCAiAQ08cIr1bOhPer1mV3ZwoL4f9HUazoyiF0SkH6BfSr9r9xaS8WplkMWdIJBJiplv3gKsrkpLAiJ4LwS8DUKItIU+Th4+zj+m+rKz4k0gkDH1mKABX913l9zt+xzh2LNqKB4ynkyfyU/8BdSoM+rbBxukovFUi8ZcrEE96I8v7Lafr7K7MWD2jycdjD30W9qHTtE64B7lTXg5q73iMGKslTX8+8zOzfpvF8IjhxHjsNBF/NeHSJUwWhGBf8ZdS4VIeFgbL4oTCtcpJwDWxa5izdg4D/Efizb+mOcOgNzDoiUGMfHmkRXGeWg3umhi+H3wKk7HKgXsE1Xf/z2seeGMiai4EjLYoQBHVokaMlOgKkMu9hL+dnOh5d08ihjU+UVYfZMZm8lnXz5i/az4RwyLoNK0T3tHeJtIPwE/elu4Jy/Hx1zSJld3kjpN5dderbLq8iZ0JO8nX5NM7qLe5x3jqP3D4fhi1GYLGNfp4GgO+zr7ka/LRynOaxNGhIfDT1J9Q56p5NOFR0zYL4s+ncVT09/W9j7Xn1rKwzzQeftJodQ/+/u4hKIL+IdUr/mQy8PaGjIJ8vj26AU9ftbnvMMJafXnf5ejKdDwY++A1Y9voCERbvggb05VI/Dli9Qlw65pbid8eT9qRtBZB+gGmPqHeKm/TnF1bqLVqJv04iQs5Fzi/6DwuCmsl99BnhzLgkQGCa4ujFcCtBB8c/ACAO3vciZfKizFRY1hxfAVbr2y1S/zJVXJUnip0Gp3N1yvDHlEalxVHz8974iR3Yuf8nfQO6m21j09bHxYcXIBHaOvOF7Qm2CP+goJALhdi06ws8Lfu/tOiIar9JJJKhcxNgGuhx589O8yqUCqF+yIjQygebCriz2iEEyeE/x84UOgZXF+rz4aE0Wjk+xu+J+d8Dg/GPYizt+2LiWNozSRxU6AFeoJcR4Mhbhlhx9sSVBGr5qqFX7K+XE9JVolFo+qWgtLsUuQ7t+OZcREnJzibdRadQYefi1+DNEgH2B6/nagPorhp9U0O7d++vfDABqFhr0oFGy9uJOL9CBasW2Cxb2XyQqOBMA8hKC8uL6agzDojXJJVQvLBZAqvtpIot46QSCQWSZOa8EnnT/jjzj8afBwnYhVonT3RK11Ibj+KVb8oURZm42ootEno/nn+TwByvf5Fq7XtvZ2WBmWKCuLPs4L4O/wgrA1A0ARVQruFpAe/jcEot+rxB9CxTUdmdJ5BB98OVq91WtiJW9bcgltgC6v4rwOMeiNtx7fFv5u/leLvasFVDqUcQm+wbLpkMBrQ6DSNRvx1atOJ54Y+h7PcmXHR41jUbxF/z97AxNM5jIw9y6V1M0x9Ph3FH8XPsrtLL46W/SxsUPmBW2TDDrwgDooTTH96qtwpcjlNamkiaq0arVpL1tksSrNbl/G5uNCvTY8/JyfBvg8s+6kpXBRMeF/oKVlX0i+pIImRX48hzWMdQUHQKzqCzn6dMWJkV+Iuh3v86XRQKhV+/P4uAeZE2v75cHJJncbmENyiYVYZWQEvAiDTm60+AQovFpJ7yUF/jyZC3pU8lvdfzrEVZplVnjoPjdNVjBgcqlIP9xQ8dqoq/hoaiYmQ4vMTedKLeDh5kPdMnimZJhJ/le/JytCqtShcFOikSlTaUF6K2M6PM36E7P2QsbXxBl0VRx6Gg4LytLLVZ2kpyJQyxiwdQ7c51kUpzQmlmxLvKG9kTnLKy6FUmQBQ7XpRJEsS8xNNzxNHCHvR+UEsBsvJEXpOVoVI/Hn4FXI64zQAg8PMPcC6+ndFZ9BxNGsPBnQUFAjB97Hlx/gg8gNSDlv2N7ZpFZR/EvJO1jzoxoZbNPgPBblZRa2UKcl6Kgvdizq8nb1xchLWQWVlEqZ8NYWe83s202Adw8W/LwKY+op0va0rY94YY7GPsz6A8OwFDJItapIx9Q7qzaDQQdzV8y6W7VvG1J+n8sPpSu0GAkbBgK/Aq2X9Pk249CXsm2tukGkDvi6CkrhcnmPxe9z7v73sen1XY4+wTuh8S2d6L7BMuufnQ/u0F7hF9m2jOb2EeoRy5oEzPDLgESsyLrkwmeSiq0glUvoE96nxXL6+UCbP4Il9t/PopkctXpNIJMTMiCFmekytijdbOwwGcx/x8HDY/+5+tr+03fS6qOJ31BLNr7Mf/R/qz5RvWo7S+WKuMM/Vh5x2VjhzIv0EyYXJXMy5aHOfiOERtJvQDqmsdacc/37wb74e+rXFNtGR6u5edwOwsM9CfrnlF54b9pzd83Sb3Y37jt9n4fZQFSe+O8Gxr47ZzdPFtImhq39XisuLufGHG7mSd8VqH4WLgpD+IbgHtwyi+VqHwYApr1OV+JPLBUUXtM4+f6J9oqtr06oVrwWrz9q0p2iOPn+pqUJsKpdjKiysreKvaoH2xQ0X2ffOPspLau7NVxMkEgm9F/Smz319qnUrFD/f1nyvNAUcyoA9/vjjDp/w3XffrfNgrgVIpS1oYSNVInXywE/dmz6XfuW5WUICbN2CdZz6/hTPlzzf4iw/Xf1dyb11IZk5zqhUcLZiYdrBt0ODVRr6u/qTkJ9AZkkmWr0Whaz6z0CphA4dhF5vYuVfQn4CSQVJZJdaZvLkcqGCUq8XJntfN1fauLQhuzSbpIIkqyr03gt60/f+vtd0FaXRaOTKlit4RXg5bBE28NGBuPrX3Y6vKkpzSjn+9XGOn+kGeDByJOzYASfiPZCOeoRefWVWi5mUQiH5JpVIuU32E1cQ+jRVrf5MTRWCZoAA0eozeCK4hNnstxJQsYst4s8epFIpI+4YgVQqvSbuFalcyqw/ZgGw9b/CNvGhvfLkSl7Y/gJzus0x9dH8/uT3/GfHf3io30NEeTwJNDzx186nHa+Nfo3XRr9m2pafD59pwRUhMH7/faEiq1Mn6wprW8jXp4IEglyDKxodXQCVPyi9azzWIeSfhg3doeOj0Oc9ADydPZAY5BilOnLUOeiO6/hu5HdM+HACAx6u3vappWD1pNUY88PAe1itFH8g9KwpKhIWspVtmevbX+Sjgx9xJHcbbQIkLOw0GYCRESOJy4pjR8IORnpOBWomEMrKzIUCFhY96ZvBo1O9xlgtJBJAYvo8pVo3UApWn1KplMfiH2tZ6xeEPiclGSUWwUN8qdBs11cW7pAVpknx51R74u/wYeF+qs7e2zSuRC0Xgl8C4Jkhz1iMzR7xt/0/2zn1/SkeOvcQD5x+gGXLQJ4MfXxHMrE90H5itYnyBkf2QdAL5ZImq09ZnqmCUlQotiQUpxejK9PhEugJSCh1EhrsRXnZ71ck3hPJhcm4eegAuUP3hkj8jRkj9PzVaCAz05zQAaEyWpwDUjiIESNRXlEEupmdNjq26YhUIkWj11Auz0aqC6SoCEIGhDBg8QArRYjNqtoJR2oecFPBoANDOcjNpeBtKvWmdHaW4O7uYZMkbYkY8MgAOtzcoVqlhGilXa/q9yvfgioIgmsmh6QSKfvu2QfA4K8EErmtd1vzDh4dhH8tFdkHIeEH6PuJbRt8BMUfQLnMkvg78+MZNPkahi9pWNeXhsCw54dZbcvPB091T0b79qRTI7qu2luDbovfBgi9Jx3pM+bjA06XhUx1cXmxVX+y4S+0vM+9MqRSKd26dWvQ9YtaLSTyQXAZObnyJGUFZYz67yjATPw5qvhribiUK/ahrJ/NZ0ffjuxP3s/5nPP0COxhcx+j0YhBZ0CmaL096nVqnakYBAS3jDyNkCEXi3X7hfSr0YXCEex/ez/lJeX0vsdayQfCb3/7ndsZ/u1wTmWcYvyq8ey9ey/+rmYpmdFoRKfWQaWWB9dRezg6v2RnC4Wdcrl5zV8Z4eEC6ZeUBP3qf4s0KZqjvx9cG8RfbVRxYWFCnNGUxJ+o9uvcWRC3QP17/MX+HMvJlSfpe5+1K1Vd0H1uzS2OWqPirzlyLg5d8fjx4w79OyHePdfRMtDpUSQ3Hgf9UILyZxCtEp40UWOi6HN/n2q9w5sLcic5xe5BlLt4oVJhqiBrSP/5GL8YvFRelGpLOZnhWMX0gIp8ebeKYtqrhUITF1HRVxlVH1Si4iCpwLqrr0whuyaInOqgLdGy6oZV7H5jt8PHjPjPCPre3zAPDIBzf5xjy9NbKDp5BYkEFi6EcO8ievz7DiHnt9G5u3UNxI6EHQD0CuxF52gvAOLjrc+dmgq+RaMYqniYYREVSYC295iIGAtk7Sf66mQ6++22qxbYfHkzb+97m/g8y4uVVxis51zM4eq+qw6/95aOqtVQm69sBmBYuDmhkqfJIyE/gY2XNjZqj7+qEHtyeXvDoEFwOuAZ+v0QycaLmxw6vgihZDjEIxh0RfBXJzj2RMMN0LMrdH4Gwm8xbXJxlqDUC1mJnNIcvKO9GfbCMIL7BjfcdRsReq2exF2JGDMygdpZfULN6qq6oKisiOXHlgMQlfEYYyrEHyMihb6zOxN3mhR/Nd2XGg2Uy4X3FliZ+JuWCqO3NNygbSHnML7qdQBIdBU9/iqsPsX5pSXBL8aPRxMfZeDigaZtSRpBQRWuckzZ0i2gGzeGzcK/YGKt5ox9++CVV4R/jnBv/2Z/Q6nqMj5Kfx4Z8Agg/P4AC6vPyueSyCSovFWUFQiToM3qyaZcH4w/CDcJn69o9amTF1JcWrMlVXNh63Nb+SDyA0rytRgxonZA8RfkHoRCqkBv1GN0Ewp8aiLsCwvhgtAum969zQFyVbtPUe3n6wtOThKGhg9ldNRoi32kEqmJ5FB4Cg+Z/HwI6hXEhPcnENDdsmdPcTHE+3/IzZs68cbuN6ofaFOjNBV+UsKJZ+3uolQKbSI0Gji24hg/TPwBbWnLZQHlKjnuQe5c/Psi+Qn5fDfqOwu1T4GmgL9TVpLtvqPuFkfFV+DAXRD3Zq0PFXtAt/VpW8OeLQh93oPbyuySfgBTOk5hvOejuGtiLH6Ps9fPZuGRRu6B24AQx15Tz9CGgN6gZ/nR5bx/4H3Ttnk95hH7YCzvjbcRg9iAjw/IDe44SYRiy7QiOx7GLRgNvX4R4xKJBBQKuHvv3czfOd/0euUefy3QPMkhiMRffftQdmwj9KY/l33O5uu5l3J5y+stdr5SS8uUFoYp30zh/hP3m/4WczpeKi88nBy30zToDRz86CBxv8bZ3Wf2+tnc8tMtFtv0Bj1L9yw1uXR4qjz55/Z/iPCM4FLuJaI+iOLWNbey76pQIKLOVfOG6xtsesyxePU67MOR+UVcCwYEgK18friQCmyVir/mJv5aE5lTFbVxL2oOxZ/Y369HDyHXBUK8o9fbP0aE+L1ULYAb/cZo7t57NwpX64KDvUl7+ejgR1ZtdRyBXqtn0+ObSD9h3WD9WiCJmwIOKf62b99e807XAbTMHoheXoICQgxGet7Zk5539mzOIdmFTqNDW6wHoxInJwmL+i9iTPQY3JUNZ1UglUgZFDqIjZc2su/qPvoG10wwTZ0q2Du1r+AfTcSfpzXx5+QkyOLFwCHcM5xjacdsEn8ACTsTUOeoG7TfUEuCVC7lps9vwqdd3XoINAR63NGD80kuHDsYRceOgh3glLlubFknJCFibHz0IvE3KnIU0RW518uXrfdLTYWg/GncHj2NG2vip7UFyLP/IdhzJnFZw8jKguAqfMxru15jd9JuQj1CifIWVAsGg4Hz588T0z6Gz7p+RtvxbZm9bnYtPoGWhcLkQva9vY+OkztSVia8R6VSICHEoOWGtjeY9p/QTug7tjtxNy+0LQbcLHot1RerTq1CKVMyod0EiwBOJI/8/ODxx+HzZ9IpUSTy7/m93Nih+l5oOoOOEgSCJ9QzGJBAl+fB23YFZ50gkUDPpRabXFxAUeJLmSKD7NJsekT3YPSro+2coOVBppDxXOFz3HO3EbJqZ/UJZpJFJG0bAt+c+IaCsgJc1R3p5zWRqAox0YiIEQwMHcioyFG4/x97Zx0e1bV28d9o3JUkhJDg7u5SoEBLoRSoQJ3Krbvd3q9CWypUbuXWvUDR0hYrxd2KBwmECHG38fn+2DkjyUwyE8+9Xc/DQzJzzpkzmTP77P2ud63lZwZktRIIGo0TxR84VAg3KI49R2TOXqCEyMJruGduW/q26YvJZGLPsj20CWhDt+u6Ne451BOpOkFMxXm5Rvz1a9OPTyf9xD2rodhFEXlFBXz6qfg5J6f2TI7sbMguyYcAeHTQM3gqPen5cU9OZp8k7ZE0wkKEJKxq9uSYF8cw5sUx7Hp9F8Edg9FoulHofZANuYfwSe3NMGWZyIkN7uvaidcXNiRjsFcwi9rvZ/MvwWgCxHW5ct5K9OV65q6d2zTn4wI6TOmAV4gXZqUKo7wMg1Kwu47mZhLkMjmxAbEkFSSh9boMtKuVFD56VBR44+IEqdemDSQnVyf+pAV7dDRMiJ/AhPgJDo8X5hNGTnkOyoAczHmC+JMKQ7Ywm8U1UxGQwsXis+RX5INJD2lrwa8TBNXeCduo8AyDttdBkL3S48MDH7IzZSd39rsTT89xlJWVUlHhS+H5PC5tvURpVilB7RtI+d7AyD6VTfaJbFbOW8n418aTdy6PwLhAy/Pn8s7xQeoCPNtHM9+7jhUanziYfAQwiw+5PEV8rn41F+HzK/LJLhPzCjvF377bIHsnTD/ftM0CrkJVe2F84YCFRGXAp1vsG2haaqb1rtd3kbIzhRnfzsA7xFr5Ssq/yOWwTVxR9AKGOT9AA2Bj0kbu/vVuvJRezOo6yzLudQtz/T4eHAwyZPjL2pBjvkBGaYZds63ZbGbVjavwDvdmyntTGvw91BfS+qhnz54oqoaw1xG2TTgyGah91Kh9rHnMUoFUrxdFcb+GK1E0GRpK8dclRDhVnM076/B5vyg/2vRvQ0BsE4VWNRGkmo7kICAhKT+JH0/8iI/ah0eHVndMk8llbHl6C7EjYul2vePvaWBcoN09B+DpP57mrb1vsfbsWnbfvhu5TE4bvzZsvHkjM5bNIDE3kRWnVzC722xAWH32nt+bmCExDfBu/3fh6vgiqaTCwhw/L83vUhyXAls0mpv4a81kjjM7TEeQiL/UJurtLyy0OpkMGCByBWUyMSUtKrI2uDiDs/fmH+3vcN52JucMI74SzjFt/Npwfbfrq21TE3JO53Dww4NU5FUw45sZds+1xjzI5uCMWpav099oWJSlQOISOkQc5ErQcr47/VmLzPWzxZlVZ0hY/jqBmYl4egq7qWFth9EzomFzK4a3FUbGu1N3u7S9XA6dO1u7eFKLxKgc4199QlVN8ecv7vaXCx23+Wx6bBPrH1jv6qm3Oig9lQxYOID48fEu73P408N8M/YbKgrq3uaTmgp33AE//ABylYJkzy6YVB70r4y8GDtWhm7SdMw9etC5c/X9t13eBsBbe9/ixiORmDFx6VL17k6pw9+OwNt1A5x4qfpBIycim6PlrPZmQBSMq0LKW5AWZbZQeiqZ+ObEBlVDNgeK04rZ/95+Mo5m2Cn+DqQfwGAyEBcYR3yQ9XrpGNyR+KB49CY9ZzSiEaWhFH9ms5kXtr7AnBVzLGSvBIn4Cw0V3+t4pRg3dl2ufdzIKs0CmRmZWUF0UBio/KD3qxA7q2FO3BZmM1SICrSXF6gNlYq/itbrQ6TRiuKlNJ6ezjnNhgsbar2HNbTiz2Ay8P7+9wFon/0Q48fJLXXVCN8I9t6xl0XjFxEYKB50y+pTsgY2auDKeih2nJHSYOj6BCXdv0aGCd+CYdw/6H5L9ljix4n8ctsvjfv6bqI4rZi/vvmL/CSr70i6XhB/8b6uzwkklXBZmbDiqQ0//GBv4ZXouJHdgpUrIS77fhJMU3hs9D0o5UrUClEk3JmyE7XacfYkiA7GrS9s5fi3x9FoICvwN945dx/fHfsOdl4Hx5xnxTQ4ihIhdQ0YNSjkCnoGD8JH2wGtRkx8dKU6tCXapjsfF9D9hu5c9dZVGIwy9IpCAJRyJT6qmlnedoGiWFeuEvOy2u4nx4+L//tV9m1Iir/MKo2n0nwgppZaW5i3qA7JfMUkoLDQ8XYajbCc0yuEpViQZxDoCmHXbDj/cc0v0hSQq2DkSuFyYINdqbtYdmoZJ7JOoK6sl2s0MO7lcTxX8VyLJf3MZjOfD/6cA/8+wPTPp9N7fm8eTX/ULpsrrViQfZ66mLpbfcrkgtAP7gf6QlgbByf+r9bdRnxptdsN8LQppKuDwSuqZZJ+AIYyyNkLZTXLHRxl5WoKNWSfzMagaVnK46KUIlL3pOIZaF/1Ol22gxPt7mVZ1ouNfg5TOkxhZOxIKgwVPPnHk1wpueL2MSTbSi+TsPusqviTyWRkncgi90wD2ii0cNgWNC/vvEzG0Qy7uaftPd1VW7SWhg03b+DsP84ytePUeh1HUvydzXVM/Km8VSz4cwH976o9b7Il48gXR7iwwboul75rkquThLN5Z/nntn/yzt53HKpaZDIZ87fMZ9qn0xy+TmlWKWU5ZXaPff3X17y19y0AHh78MHKbJsHOoZ05fd9pDt11iKeHP235PFVeKmZ8M6PFZ+r+t0AaM5y5AEgxMamprU8l3FzEX2skc6qituvCFtK6ISfHKh5pTGzYINbEnTtDfLyocUtNLa7k/Dmz+izNLMWory4ZTC5Mtvz8+ZHP3T7fyN6RLNi6gOmfT6/23H8DSdwUqBPxd+jQIZ588knmzp3LzJkz7f79jRaE0iQ48iidg3dwJGEOb529G41Bw+Udl1kxZwUZR1qenYd/bAC5bfug9Qmuu4WOCxgeW0n8peyuExlaJ6vPYsdtPuMXjWfGNzNaPCnblChKLSLreBaagrqP4Hv3CmLt1w8v89niAouPtUT8qVTw5vdt+GB1W0thSkKxthitwXrXzdVkYfTIpaSkeqbDlStQ4nkKdVC2mOSbzZD5BxQcqX5ScgXIZE6LhlAz8Qcif6ZjrdLClo02/dvwSOoj9Lujnx3xJ73n7mHd7baXyWRMThAKu/15giRvKOLvVM4pkguT8VB4ML79eLvnJNWY1MHXJ1iMGycK9qM31mxVJi0KPfSR+Hg3co/Nruvhtx5g0gvFn0T8leehL9fz7YRv3bLabU6U55Vzdt1ZjPmi8ieNp9f8dA1TfpjCrOWzLBaKjlCV+Lt4EQ7VIw5r0c5FJBUkoTIE0zZ/PmPGON7O1uqzpqFcqwWl0R9fQzurKkmTBduuhguf1P1EXUGbicjbzcKMHIPBngTrdEcnh5Pp5kTG0QzW3rqWlF3i3mk2m8k0C+Kvg7/rxJ+nl5EKdQp6RVGtSuGLF2HdOvGzpOyULB5tUaYr45ENj/DHmQNs2mxGafJj1azf8VSKC1ayKt6VsguwjiG294/yvHIOfniQaZ9OY8IbE9BoQK8QVcRgr2CRh9X5IZffZ72R9JkgGzWCmK6amTBv3TwW/Lmg6c7HDWi1IENBu+J5zOw6s1YLdalLv1guiIjaCHvpc5O6tmsj/oIiSyjWOr9JhfmIC8LsbbX6dISyyvqfQVVJ/HkFgdIHhi+F+FtrPulmhGRlmleRZ1Fta7WgULdse3uT3sSIZ0bQ/+7+9LujH35R1aU8tsRfndcpJRcECWY2i8zf7s9B29qbgub2EGpbb1UVxrHf2zBxRx1PpglQnAibh4mcPyfQG/VoVFeoUKfYjdP739/Pxz0/Ju9cy2pkmvrRVJ7IfQK5wn5+l6UX89j4eloougKZTMZ7k99DhoylJ5fS7t123PfbfW4dQ+rq99BWEn+l1WsD9xy7h1s23VLv820tkNYlnp6iOfebMd9g1NkXMqW/W2vN+VMr1HQK6WTJ860rOodUEn95Z/+r6xjrH1jP/vf3W36/o98dFD1dxKfTP7Xbblz7cfh7+JNeks7e1L0OjxUzJIbAdoFc2HCBsmx7km/v23t5K/wtcs+KRcye1D0s/HUhAC+MeoE5PeZUO55MJqN/VH9em/AaPmoXrS3+RoOiNmVXmzai7qTTQVZW051XQ+BvxV/dUTXOpiYEBIiGErPZupZoLBgMsL5SczLdZukvWZS70tDiKL/QbDazJHYJy2cur7b9lI5T2HDTBkDEGtk64eWV59nVXZ2h7bC2DrNibW1h/4tvQ/WG29XIpUuXMmzYMM6cOcPq1avR6/WcOnWKP//8k4CAgNoP8DeaDkH9YOIu0s03g1l81AWaAkqulHBq+SkKkwub9/wcIGJALMm9r6XCPwINRTy68VE+OfRJg08mB0UPQiFTkF6SbiHxXIXJbLIs/J1ZfYL1RjUhfgLvTnqXe/rfU21bgISrEoifEN+iiyH1QereVP7d+d+cXHbS5X3GvjSWJ/OeJCi+7guSlBTAbCbu+C9cfvFLystM+PtDB5u1uEolgpirwt/Dn8sPXybtkTQifUWFLzBW3IVPnLBuV14OeUVatvfowdBVERRUFIiO6+vzYeQqxyeW+Se9ooRizNHkzxnx11AWNi0BCpUC/xh/PAM9HRJ/dhZWlZjSUVgMbU9fjxmzy+qd2vDruV8BGB8/vtqCyVbxB9C3bVdUhkC0ptrzQaXiiYc+SqgCsnfAnxMhqxFyLiLGQbs5oC/Bywt8tB1po+iBj9oHhVpBxuEMitOaIBSxAZB1LIul1yzFO02wLZ6eUK4vt+QarU5cTe9PerPzsmMiUyJYcnLE5/fUUyKnrS4LrcuFl3l5x8sA9Ej5gAG9fBxaXxRrizleJIquBoMYF5xBq4Wu6a9xa2Eyd/evzC1SBcLgzyF2tvsn6SakybFOUcDm89ss1rqRQyPpcl2XRn99dxA9KJp56+bRfpxg4PQmPSOMLxCTu4B4fwcybSeYtvRqtvRqR1bg2lobBj79VCisRoyA664Tj5110Mi+9uxa3t3/LjetuhG9Hrp0sWYAA4yIFcqcnSniOpVUFbaKP22Rlo2PbCTnVA6hXUIF8ae0If7iF0BUE1qrtZsLQ78TRATwZ8HXJEY9T5rmTNOdg5tYt3Adv9z5C3o9eOrbMDr3R5Zdv6zW/SbGT+S+AffRN0Ko52u7LiRiTlrmtBE1cqdWnyfNSwl8PZA71tor4SRcFX8VC/svJMGvm93xq0IquJjUNoo/pbcY70MH13zSTYXzH8Mee1LAQvyV51WOOTK0WtBX6EnanETWiZZZ+VKoFYx6bhS95wvr0uxT2ex7dx9FKVZmOL1EzAU99dF1J/6OPAa/dABzJZnQ+xVoO6PW3Z4b+RxvTXyL3be75lbSYuCbAH0WQ+REp5usOrOKSZuiOdr+FjviL25sHKNfHI1XcCN2g9YRVUk/gFyTmMd2Dmt84g+gb5u+3NFXjDMGkwEPhXv+6NKcRl7uWPEHjt9nS0JDr49si/iT3pnElA+moPSwXyxK9/TWqvhrKCQEJxDlF0XfyL6U6Bx3Vp395Sxrb1vb4hwD3MFNv9/E2JfH2j3m7+FvqRFI8FR6cm1noRBffqp68VuCtkTLspnL+Hn2z3aPtx3elr539iU4QXwxn9nyDDqjjlldZ/GvMf+q9Ty1Bi0rTq/gtZ2vse7udex5e48rb+9v1ABXxhdHJIgt5PLGiaJoCjQ38afXN0y9p6lhMgmiF1xT/IFV9XfFffG+W9izR9y7goJg+HDr49J8wNmaxBaO1IwmvYkB9wyg4zTHAoVJHSYxJm4MZsx8/dfXgIjxiVkSw4xlM1w697zzeayev5rzv1tdkqRzMJvF9fI3HMPtmdyiRYtYsmQJ69atQ61W895775GYmMgNN9xArKOAiv8xtKjivDoAwoaj9I1AZQwEoFBTSNdZXXle93yLzJSzlTZfLjnHkn1LeGn7Sw1OinmrvJnRZQYPDHoAhcy9z6xMV8aAqAG0C2hHtF90tedtu5sBekf25qEhDzE6brTTY5rNZvQV/50jldloRqFWOOzQcIaG+Lwlj2zvWVNI6zoRZHL69XPdCUkmkxHtH02Un/DwjOkq7sLbbXibpCTQKYVVl0qusu+cdJbXtet6hvs+D7hO/CkUCou//Nl1Z3k37l0u/nHRtTfSAlFRUEF+Uj4GjcGO+JPInYTg6sTf2LixqBVqLhcnU+4pSKGGyPlbd07Ie6Z3qq52qkr8tY2RE1QqrBF3p9RceEsI6ELX1MW0y7lbTEg02ZB3AAwNGE4oodP9MPAj8AjG2xu6pS3mIdUJ5veej1wp56mCp5j6Uf3sfJoKIZ1DuPqzaykOFVavHh5wqeASAD4qHzqFdCK9JJ1rll5Dhb66FbD0WeXlwWefiYmp2Vy9QO8K2gW24+fZK+hacj9R+fMYP776NvkV+QS/EczEH8Yg9xbF+ZrUQw6DvtUBwiovdIj7J+kOLn6L8pcIekTsotBnP1cvH8s/fv+H3fjSkuAb4UunaZ0IaCvYFrVCTX/t4/RJ/hofT3Ute1shqbvK1ZdqJHiMRjh1Svy8YIEg80CM81UXnd8f/x6AkCs3IUPGDTfY31skxd+JrBMUagodWtD6t/Vn1k+z6L1AkAwaDehsib+mRshAaH+zJY9rfeaXXIh6lSsG0bSTsiuFQ58cwmRoOVnWWX9lkXk0062uWoB5Pefx4dQPLfmxtRF/0ndaIv4kxV9WlrW71GSyjjMXdHswY65WFJRwV/+7+GTaJwyJEINKbcSfneKvpSF3HyR/L+wcKxHiLari+Zp8vLzk+Pv7o9crqMir4PurvufIZw4cEVoYCpML+bjHx2x8ZCMpu62dyVLjn1d9rD7bzYUeL4DcQedZDVDIFTw27DH6RPaxf+Lkq5DqpNmsJUAdCN2eEGOME0jXjF6ZZze3azeyHWP+NQb/mJaT9VecXsyRz4+Qd95e7qXRQImq0rmiTdMQfwCvjHuFQM9APBQePDj4Qbf2lQp9EVduY/XsX1g4YGG1bcqyyzj23TFyTre8inVjzF9sib/YEbGWRgBbSH+3hrKVb0r859B/uGnVTWxK2lTvY6kVatIfTWfHbTvsMtJtkXE0g7++/ouCiy74x7VQxI2JI6p/VO0bAjd0vwGAn0//7NDuE0STyTVfXMPwp4fbPd7l2i5c89k1yJVycspyLK4Rb1/1tp3FpzOcyzvH7J9n8+K2Fzn+43Eu/XHJpXP+G47h6vjiSpab1Jja2sYMaR7q08RiUtu/ZVNYXzY0bJWKrq5NpKbCxib+JGebKVPsBRCS1ac7ij/bz0mhVjDl/SkMWGiNJHpv33t8fuRzcsrE/EFqVPrqr6/ILc/lxlU3ojFo2HBhg2V+XRMUKgXHvztO6l6rcMf271tRvTTUItEcNRe3ib+kpCSmThUFRLVaTVlZGTKZjEceeYRPP/20lr3/+9HibA5MBgJ8ylAZAgFB/ClU7pEwTYnD/zlE3NHVeCl0XCgQTL5EhDQ0VtywgvenvE+0f3Xyrib4efix+/bdJD+cjEqhqva8NAC6epMqSi3idf/X2frPrW6dR2tB7IhY7j1xr1tEc0VBBWdWnyHnTN0WmSZTZee9TMYdr3Vg7qJetGkDV1/t4v42E3WJ+AtPEHfho0etRcANG6x5XeE+4WJSriuE9N+cZ5n0/4DsMJHb5CjjT1K7ZZVlWazCzGYzxcXFmM1mPPw98Az0bFHFV3dxculJPujwASm7U+wKtg8PeZglk5Ywrv24avv4qH2Y12MeC/svxNdHjF/1tfvMLc+1WLE4yrmQJufSZD0mBoJKK3P+UnfVeOy23p1IyHqC2Nw7BfEXez3MLoJox7kODYWq9nytDf7R/nS8vg9aX1EE9PAAX7UvTwx7gnsH3MuRu4/Q1r8tCUEJFuWFLaQubJ1OdLRJcMWv3hEGB1xLwtl/o1bJGOKAlwv2CqZDcAfMmKkIE9dETcSfrYVUk0MdhMy/EwqlCqVJWNiV6Eowm81sfGYji0MWU5zespWhUidfVXvmmtA+UCgGyz0u1jhmSItbEN/5yEhhu6LVG3hj8+eWnN7ssmxLwSwi6ybi4kQwui0ifCPoGNwRM2b2pO6xFAltF1MKlYKLWy7ySa9PKMspE4qoSqvPKIUR1raH02+6/kYbGP5qcY2U6kUV/ti3x/jt3t9aVNf+nfvv5O7Dd6PXg0mmQ6Fyry3YFYtes9lKzEk2OGFhooNbp7N+ptnZ4vpUqeBojhh8pAxNZ5COV5vVp8424+/KRlgVCSkrazx2k2HAv2GuTliQVsJW8efhYcZg0KPRmPGN9GXqx1PpdUuv5jrbGrHv3X18M+4bSq6UENAugISrEuh2fTdiR1gbWy1Wn/p6EH9x86DnP62/F5+D33tD4rvuH8tkhOPPw+WldTyZlgHpmtEp8tDprJ3yLRGZRzNZd9c6iw21hIICM2UegvjrFtF0xF+EbwSH7z7M4bsP0z6ovVv7+vuLwp9/RW9GhE+3y9eWkH8hnzXz13B2neMct+aE7fqooWBp0CrJdXrc1lrEB9Hw+OOJHzmT0zRq/sEPDubJ/CeJ7O24Eaalw2wyYzbZXwcL1izgnl/vcZirOTF+IgEeAWSUZjhtElV6KOk5rycdp3S0vEZV/HL2F0xmE/3a9LPkEteGnhE96d+mP3qTHt91vsxdO9el/f6GY7g6vtSm+AN7R5rWhOZS/CmVIHEjrdHuUzpnuVysC1yBMzeRhsT58yK7XqmEyZPtn6sL8VfTNW82m3lpx0vcte4ui7XnrK6zCPAIIKs0iwnfTrAj+1afWV3r6wbGBfJYxmOMe9laK5TLqzvutXQ0B2fkNvEXFBRESWUrXnR0NCdPik7gwsJCymvyt/ofgcnUgorxZhMsVTNMdqud4s+gMZCyK6Vap2JLQPreFELTj+PhJed8niD+Oga3rjwzR57Uh68cZsXpFcIKsgr82vgRMzTGYuvwv4RDVw4xZ8WcasqdwuRCls9czukVp+t03Oxs0GuM+FbkEBEhOlo+/RS6usA9phalEro4lOuXX4/JbLKoOsvk6XTqJEjFHTvETXH3bivxF+EbIQ5QdAq2T3Pefd3+JrzaXwU4vrEHeAYQ5i1mh0n5QgFnMpm4ePEiJpOJuNFx3PPXPXSY3HRFhYZGZJ9Ihj81HP/YIIyVTlceHjCq3SgeHvIwPcJ7ONzv6xlf88m0T4j2Eu+9vsTfr+d+xYyZPpF9qtn2mkzW7A5JrRMdDcGlI/Ev70OnwO7UBOl26OEhJiSNjjNvwYZB+HiU2r0+QNKmJM6vP+9kx5YHW1WcTCaUd4snLubNq97ER+3DyftOcujuQw6bQlQqazEdrIsGZzks6enwxhuV1sCVyCrNokgj2Dtp8hsa6rxjb2C0UDFU+In5UE3XZXG5hi092/ONagjl+soP6coG+CVBNAw0JmKmw8SdpGkGozRWEn/aEkwmE+WycsJ6hDXu67uJw58eZpHvIpK3JwOwN3UvGYZTmGR6l7snAUsh01Xiz8dHXDcyGXTqBKmhX/D8gbsY/Plg9EY9y04uw2g2EmEYhK+2E9dd51hJLqn+dl7e6dQWLH5CPAPvH4hnsDc6ndXqM8jDHzxCQNGE9nZZW2FNW7gsrDL9PcU1Ul6pUh70wCBu2XwLah83WNcmglYLKaFf8Em4ihtX3ujSPoWaQpI1xzBjQqdz3qxVXm5VfEpEoVIJ4eHiZ+k+bsn3i87lXL5QpQ+JcaziNZvNFFQUoPMUC97aFH8Bshhi/GOEKkrhCX4dhVK4JUDlB3L7qoakWM2ryEOtNlNeXk5FhRm5Us6AewYQPdC9hrumQll2GRmHM/AI8EAmk3HzxpuZ/fNsi+oYbKw+65PxVxVekaJpTLL+dAcyGVx9EnovaqCTaSTsvxM2DHL6tKT40ynzMGO2jNVFKUV8M+4bDn1Sj7DeBkb04GhuWn8TCRPt3Sku5+RhUIq5Q0JwdQKtMREfFE/38JrnpY4gk1mLfc7mSeE9w5n982x6znM9W7epYLs+aihoNKDSlKD85EN+v/93h9u01iK+2Wxmb5poeBzadmiDH9sRvIK88ApqeVa9riL7VDYvKV6yZKUbTAZ+OP4D/zn8H4fbeyg9uK6r8Iqvye5Twq/3/sqXw78kdU8qXwz7gqTNYu0/Om40L415ifsH3u/W+d7a51YAvjn3DQp1y2z0by1wdXxxRfHnyPmjNaC5iD+ZrHXn/NleE666jUVVioobk/jbvFn8P3Kk9d4vQfrdHatP22v+8o7L/Dz7Z9IPinnyxYKL5Ffko1ao6Rkh5g9eKi9+mfcLVx67wg8zf6Bfm37c0kvEBaw841pDo29k9YvRNuevNaA5OCOXy5ESwTdq1Cg2V14xs2fP5qGHHuKuu+5i3rx5jHfkg/U3mg8yOcQvoMxnOCqj+CYXagopyynjq5FfcfCjg818gtUx8LXrODL5GdTeSi4UiK7JjiGNR/zpjXq2XtpKYm5igx3T0U1qzoo5zP55NieyT1TbXq6Uc8umWxhwz4Bqz/03IG1/Gvs/2E9Zjn2A9ZGMI0z8biLLTy3nX9v+ZfdccEIw1313Hd2u71an10xNBb/cS3TZ8hHHvnLPUuqPi39QoCkgtTgVuUxuUfxdKbnCmDFim23bYNMmYQ0XEltJ/PlUEn++CTD4S4i8yulrSB09xcXWjn5b/DTrJ47cfYRuYXV7/y0dbYe2ZcLrE/COts443FHw+Fe6ydTX6vNYpsjpu67LddWeKygQ5J9CYZ0I+fhAgnIko04f5da4f9V47H2pBynyOorKp/IDzjsEaevA2Eht7Np80GTi61FCvu8uPvfoxtU/CInrxkc2sumx+lv6NAX++vovvu+/BJ+CNKcLKGd2QhKkxVVICEyaJH52pvj76ivYtQt+tonZeOqPp4h7L45lJ5dZ9qs6ObZFlxDhCVniKe4jNSn+skqzqfBIJoOjeCmlQohM2Cu6mc1TV3h6gsIoJs2lOrGia39De+b/OR//6JZjp+bbxpd2I9tZsp3uWncXS0N6kOO/yeXuScCigCj3uFTjZ+NocdulC1wJFkTYgKgBqBQqvj8hbD5j8m8Sx3cisJjZdSZ397ubwTGDHSr+AHrM6cHV/74arVasCnWVij+foO4w+RB0/oerb7P+UPqCVxTIxXUYUEn8VZjEQBvRM4L4CfEtqpB0ZtUZMo5kiAwQRSEgMnZqg96oJ2RxCIO/6oPJS9zDnZHC0uOenvb3KVu7T7Dm+xki9wHQOaSzhcyoim3J2wheHMzDR0TmmbPxSbomH/DeTeojqaLZIWI0TNwJkRNqfZ9NAn0J5OyBcmvXrsXqsyLfbReM5sT4ReN5qvCpGsntT6Z+woi8z/Gr6FY3xV/uPljfH9J/tT6m8ocZl6HrY+4fTyaHwO7g19KbweQgUziV1kqKP7Ncj1FeapnfyVVyMv/KpDSr1OF+zQGfMB86TO5QzX70VIZYt/qaYvBStR6iIzgYjPJylp76iff2vVfteQ8/D7pd342A2BbSbNDI0GgAsxnVyMF0mtbJ4Tatlfg7l3dOjMtKz+qWwXXE2sS1tHu3HTOXz3T4vNlsJud0TovNdq0NKi8VXWd1JaSzGKMySjIwmo2o5Cqndt43dLsBP7WfQ2coR1CoFeSdyyPvXJ4l7qRDcAdeGP0Ct/e93a3znddjnrBgPZbOjj93uLXv36gbHNkeVsX/EvGXvC2Z/Av1D0BtbWSOLVwhg6uiKRR/ZyqF3sMcGJI4W6s6giPFX975PE6vPI22SEz4D10RDVu9I3qjVljn1aPajSLQM5Du4d05dNchXhr7EgB70/ZaGq9rglFn5PTK05z77ZzlMek8WiNJ3FRwOVygV69eDBw4kBkzZjB79mwAnnvuOVQqFXv27GHWrFk8//zzjXaif6OOGPIVhYdAaRAWZIWaQrxDvbnqnauIGuCaV3lTQquVYVKq8fSEU3mNa/UJ8OD6B/nk8Cc8NPgh3p38rkv7PLvlWX448QOPDX3MYZZC1Yw/EIqVpIIki8z5fwnnfz/Pjpd20H5ce3zChBXU6ZzTTPh2AoWaQoa1Hcbzo+zHDg9/D3rdXHcrqNRU0Hv6ohjUz86iyRV8cfQLAK7pdA0grDd7hPcgwieCkf3g88/h3Dlrd3/brllw0Ubx5xUJCbc5f4GTr+J19j3iIo+RnNmGK1egYxVue3x8zU0Up34+Rd65PEY9N8qt99bSIH1HZDLILE9l3/m9dA3taukKcoQKfQXFfiepUEVRXFw/1cCSyUt4aMhDNgSMFdLEPDjYXrEXEyM6odLSqn9utnh2z0LOdD/K5LzfgKvh3L/h0jcwuwQUjaCY6bMI+ixClQiQRIHyDOfyBMl41dtXIVM0bE5qY0HppcQj2BeTQmkZS8/knCHAM4BI30i7jItibTHl+vJqC++ePeHSJbj3XsjMFI85msTm5MCBA+Lns5UOVhfyL/D98e8xmo3EB8WTWRmPURPx1zVMSIkLFGdoS83EX3a5KHz4ycKtWaZRk8S/xkZFFiR9Ro+wEVzMEsqAMn2Z0wyS5kbn6Z3pPL0zAFqDlrN54kPyL+9VJ8WfRp1OXpEGcLwKkwrNfn7Wx9q0LyL/jJg/vTPpHc7lneNA+gEUMgXBV+YA1oVSVUztNJWpnYSFcHKyeMyZokJaqAy4uJLHns9z2dapQREyECbtt/wa6C2K2jpKMBiEyk3q6G/o3OW6wKgzsnzWcnrM60HwXbPQK8QXL9AzsNZ9VQoV0X7RpBanYg4+D+ltKCqyqvhsUdXmU4JE/FVV/OX57ISimm0+w3xE1bhQJ6rG+fmikahq5IPUGNTUndZuIf8IbBkD/d+DzmJO3DeyL7lP5BLoGcjPlYIH6Rpf/+B6Tvx4gsezHkeuaAo5vHuo7doeHz+ez3Og1EjdiD9dAWhznHvLugujRpCv6iC3MwObFINrjgPxVnnjofBAa9SiU+ZRUiIGYr82fjyV/1RTnKHLMBlMyJXVr91gQw+GJu4gvksjZDk3IkJCwHRBx79O3Agn4O7+dzskLrUlWjz8mqZBqTmh1YLey5+A6ybT0UlEhC3xZza7ruhoKBRqCknMTXSqKncGSe03IGqAXSG2PvBQepBSlEKAh3Ni+LOBnxE7IpabN97cIK/ZlAjuEMwNK26w/C7VcmL8Y5zm7k1MmEj2E9kuNSJd/e+rLffC3gt6Qz1vDSHeIYxqN4quS7qy649djDrXumsFrQHS/OZvq08RYfTN2G+IGhDFXQfvqtdr/7co/lyFRPzl54v9GzoWpKICLlcmEXVy0NMi1TpciUdxdM33u6MffRb0sfx+8IoQGQ2Mcp7vLJPJiAuMY9UNqxjVbhQBnq41GK29dS0RvSLoNFW8kdZ8rTQVXF5xbd++ne7du/Paa6/RtWtXFixYwO7du3n66af55ZdfePvttwmqqTL2N5oNPj7QPvshxuetYEqHKai8VAx9ZCjtRjZDYakWZB5Kw6s4C09POJ/f+FafVyUIVda6c+tc9tq9WHCRlKIUDCbHWTKOPIZjAwT55Iz4u7zzMqtuXlXnTLuWjH539mPBtgUEtbeOD+/sfYcCTQGDowfz48wfWXpyKa/tfK3avgaNe3k9ElJTocI/koSHpxPaJdTl/U5mn2R36m6UcqWlw+6mXjdx4t4TvDzuZQIDoW9fsW1ZmbhBeoQIZsGi+KsNHsHg34WYSMF6udrV42lz9z/xwwm2vbgNo74OtlAtALvf3M33k7+nNF8QUx4esO3yVuasmMNDGx6qcd+bVt3EJ8ZBZAQvr7fVJ0BcYJyVtLWBNDEPrXL5xMSI/y+larhYcNHpcTPKRehwiKqSnOx4Hwz52i4HqTHg7Q0qQ2W+UoVgGTpM7lDNkqqlosecHoz6/i4q/CMtY+kNK24g+p1oS64awPv73yfszTBe3v5ytWPcdht89x0MHmwlZRxNYjdtstZeMzKEsueVHa9gNBuZ2nEqA6MHWghDZ+QOQJdQofjLNiVixlwj8ZdbIYg/f6WL40VDQpcPx1+ga/AWi9UnCNVfRXIF2/+1ndyzLbMV9WLBRQwmA0qjH576GLcUwiFeIXjKxYr1clGy0+0k4s92cXtJsR6zXI9vRRciVZ24VHCJNr5tGNP2KjwMESIbyQWRpHT9FBVZbSNtIc0X2hpHM6vbTHw1V4R9b3HzZSoFeYtrxKAooaICDn92mJdVL5O8LbnZzskOMpj10ywG3DsAvR70lYq/moqPthgeK/JaM4PWAM4Vf9L3OaDKYat25krE3wWjyGseGzfW6WtLdt4FmnxkciMmk2NrHVv7WQsyNsHJV4TKuyXAvwv0fRPCRloeUilUhHiHoJAr8PAwI5crLI0+PhE+RPSKQFfaskLcdKU69n+wn6zjNatSzGarlXadrD6jpsCMFGG9bIuSJDj2giBS3UHWdlgVDhe/rMPJtBzIZDI7u8/6Ojo0JtbcuoZFPouqrVG0Jb6ElI5kUJCLgeItBMHBoDQGoJYJJttRfvLGxzbyuv/rlOe2vFgXzwaujtrazTuDVMTXaBw7tzQmzuaepcdHPZj24zSySt1T0e1JFfmzQ2MazuazXYCoJ10uuuywliKTyRj/+nj63tm3wV6zOSHVcqTajiMo5UqXSD/ArgFGJpMhk8v46OBHLD+13OLK4S781H7sHr4brxtbj/K4pcKV8cUVkud/hfgLaBvA1I+nMuGN+rtStGYVV12IP19fa/Op1LhcEzIzM3nl5ZcZ2LUrnaKjGdi1K6+8/DKZTnY+f17MYUNDHdc1bIm/2srizlSucqXc0hhlIf6inRN/Eq7rep1Tl5SqUKgVXL/8emb+YFWZS9dKa1SHNhVcJv5GjhzJl19+SUZGBh988AHJycmMHj2aTp068cYbbzi9wP7XoKjartvcSFxC24y7CCkdSVjOLLdDv5saR5/5mbhjv2D2zCe/QhQ1GlPxNzFhImqFmosFFzmT61rItTTha+vf1uHzjmyNYv1rJv5KM0s58cMJck61stmACwhoG0Dc6DhU3la7i+NZxwF4YtgTANz96928uO1F9Ea9ZZuNj27k9YDX0ZW5XxxKTRF3q7aOPyKn+M8h4dd/TedraOPXxuE2o0dbf540CcbHj+XBQQ8yql1lR13iu7A2HoqcXE8d74WJO/AMjQPgSvVccNKK03hn7zu8s/cdQIwrXbp0sYwvExdP5IFzD7TIbnlXUHipkNTdqeiN4vw9PKx5hglBNRNUvSKEErTY63i9iD/ba80RJMVfVeIvOhpy/f7k7ksBXL/8eof7lunKKNSJA4R7VDZYhA6C+AWN1xZcng6XvsPHnITaIE66UFPotEGhJcN2smw2my0Eq6TcAtEQojPqWJ24uppiTSazTpyd2VYYDIL4k7YH2HJUqP0A/jXmX4CVMKyq9rFFQlACCpkCjbkErSrDJeIvyJb4y9ouxg1dofMdGwK+8TDlLw6XPITc7IFCJtQh5YZyfMp82PnKzhZ1Dzr32zk2P7mZ8txyirXiy642BiND5hbxJ5PJuKbNPSRkPI221Dnx7mhxu/HyWgAiCq/l3DmY1GESqY+k8lK/rwFxfdX0lS7Xl3Mk4wjlsiyUlWIcRyR0tQVi/hE4+gQUVrcHbzSY9HB6sSWfVrL6lIi/wLhAOk/vjGdAA7eg1hEKlYIec3vQbmQ7tFowuKH4A2GHBXDBYylmjE7vJ7Up/qTlj2T1Ob/bvdzY80bGtR+HM0iLWzNmvENFg4Yj+6eyMij2OsZzGV2ZuaxygXvldzj+AhhbSAHeKwK6Pg7Bjou63t4KfH190evF/X7Uc6NY8OeCFnMdScg9m8uGBzdw9hfnZPuF/At8efgb8ryEVLxOij9n0GTCqVcEsesOvNpAh4UQWHeXjCZBaTKceQcKTzrdZEHvBQyVPYzKEGRH/CVtTuLMKtfWaE2BiN4RJFyVgNLTXmEp3ftbWx90cDDIkBEgE41q6cXVib+o/lH0vKlnnRsyGwtV10cNgfLsMrpv/TfanQecbqNWW5tBmqqQbzKbKNOV0T6oPaHeoeRV5HHvb/e63LgMVsVfQxJ/EgFWrC2mUFPocJvBDwym+2z3MyhbAtL2pfH7P34n+1Q2IAhOqJn4s8XK0yt54PcHavyc8s7n8X+y/2Pzk5vRG/U8u+VZ5qyYw1+Zf9XpnFUKFUf6H0E5pQWrwFsBXB1fHNkeVoVUTygraz1Ell4PusoynCvEn8ko1uMD7hlA+3H1rzm3ZhVXXYg/sOb8OaoP2mLJ22/TLiaGXYsW8XBiIh9fucLDiYnsXLSIdjExLHnnnWr7nKt0xuzSxfExpbmLTmdtcHMEg0FcG2B/zaftTyNlt6h1G01GDl85DNSs+KsrOk7pSGBcoOX31natNAdn5HbV2MfHh9tuu43t27dz7tw5Zs+ezYcffkhsbCzXXHNNY5xjq0JzBDXWiOzt+GR9DZgpK7Oy9z9c/QPLrlvWnGfmEDG3jCEzfigB6iAyHstgz+178FE7KNRd+FQUp+ppleOr9rUUZ34992stWwskFyYDOCVRHQ08tSn+Ok3txNNFT9c5064lw6AxYDZZPyeT2cSpnFMA9AjvQWxALH5qP/QmPefyrF7NoV1C6Ti1I5pC90ZwsxnKtuyj+/aPCNC7rl4p15fz3fHvAFjYf6HT7YYOFaSChwdMngwzuszgvSnvcbXkB6PwAnVgrcqumm7sV0qu8Nimx3h779uAGFfy8vIs40tIpxCC4oOQyZvfbq0umPrRVJ4peQZDpdu0hwckFQjirzaiv3dEb0AUQ+tK/OmMOmKWxDDtx2nklTv23nNG/MXEgIc+AiM6LhVecrivtChUGgII9g6s20m6i4JjsHc+vmXbURmsVaf8inx+u/83Xg98HaOu5StEL2y8wLmvdiE3aPH0hJzyHMr15ciQWbqKAca1H4e/hz8ZpRnsS9vn9Hi2xJ/t7eLAAfFYQIAIuAb44MibGM1GpnWaxoAokbkqkTQ1Kf48lB68MOoFHkj4ELnJq2biTyOIvxBPG+IvbS0ceUTYtTUmFB4Q1BuTOhQZMm6NfoMPpnyAj8qHkCEhLDy+kPiJ8bUfp4mQvDWZPW/uQVeqo0wvWuqlbEJ3iD+AJ/q8Sdf015AVO+8GqWr1qTPq+P387wBEFs6w2MEq5AooE56QNV0XADf8fAP9P+3P6sRVNWYnlJaCRnWFS6Efs+7sOmhzFVy1FyKcq8YaHDIF/PU0JP8EwLye85iccoAuaYsoL4eEiQnMWT2HNv0cN8U0J/R60CsLAVy2iZmUMIlAz0BKZRnk+e2oVfEnKTvzK/JJzE20KP7S0mDzZuvn+tCo2/hh5g9E+zu3olbKlQR7iQvCK1RUjR3ZwJaVgVaVRbo20XKPpPuzMOUoeDaDatgNPPPHM8xdMZccwwV0Oh0VFQ1kbdlICOkUwi1/3EKPuT2cbvPnpT+587dbudDmFWSyOtowJX1hn+8nIbi/+FzdzfkL6gWDPoFQ9yz/mhylF+DoY5DtPHNq0fhFzAlYgo8u3u77uPX5rWx4aEMTnKRrGP7EcOasnlPt8fW5H3E59FPM3q0ry0y6N/kYhaWFI8Vfzxt7MvP7mdVyDZsbVddHDYGKPDHfUMprPmZTK3h+Pfcr7d5tx0cHP+KbGd+glCtZnbian07+5NL+eqMetUKNXCZnaNuGI/581D4WFbu0/vlvQtaJLA5+eJDiNDEoSbUc2zWJM1wquMTclXP598F/89aet5xu5+HvATLwDPJkW/I2irRFhPuE15mg/XbGt2if13L/oPvrtP/fEHB1fHGF5PH2tjYLtRbVn9QQKZNVcZ5wgLO/nOXzQZ+TfyEfXamOkiv1X9P+r2X8gdVNpCbib8nbb/Pac8+xw2hkg0bDTcB44CZgo0bDDqOR1559thr5J61jHdl8gqjHSddoTXaftuIWW+Lvjyf/YNkMwS9cyL9Amb4MH5WPxRmpNnx//HtGfDmCn064dk/TFGosWZKt7VppDs6oXnKRDh068Oyzz/L888/j5+fHb7/91lDn1WrhTtdVk2Dod+ivK6NCnUp64HJ+PbMZaBn5LI7gP7ovBdE98PKSEekb6XhiajLAsedEN3ye8248VzG9k7DbWXduXa3bVugryCgVvk7tA10n/qS8HmcTYpW3Skz6/guxfNZyXvOz2ngmFyZTri/HQ+FBQnACMpmMHuGi0HIy29oJ3P/u/sxZNQf/aPcWmfn5oNeZkRv1dOjvehD9spPLKNIWER8Uz4R4qz2B2Wym/6f9CX4jmJSiFDw94Z134L33RC5GNXRcCFOOgI+TTkBNNpx8la5BfwKOb+wS+XWl5AplujLMZjOpqamW8cVsNlOaWUpJRgv2QnIB0sTBlvhLCK5Z8dc7UhB/pV6nKCiuWbXnDH9e+pPssmyOZBwhyMtxa7Y0KZcW9hKio8FLJz7bQk2hRYlkC6k5wFsXZ50Q/d4Ltk2vtm2DIWQAjFyNOnYiCpkSlSEQgLzyPALaBhA9KLpVWMOeXXuWi//ZgtxowNMTi9ovxj8GD6V1jPRQeljG7lVnVjk9ntS9ptXaTwbXrxf/X3UV9Kis8x4r2gbAfQPus2wnTXxr6+B/ccyLzO9yH2pjUI3EX55OyIPCvG2K9l0egYm7mqaQrysk0Euw2pMDHuUfg/6Bn9qPnJIcwrqFtaj8npHPjeSBCw/gF+1HmU4i/sTKU6Wqac/qkEibmpoFpAWuRPylFKUQ7hNOkCqSwLJBXLhg3VYiaRzeA2zQPUx0uJ/KOWUprjoieEpKoMTrFDv97+O5P58TltChQ8DDNduTBoFMDpMOQD+xUIzyiyJGPhAvfdsWuZDK/CuTt9u8zaFPDqHTua/481B6cH1Xodq+EvyT0++t9Lik+Ju5bCY9PupBOgdRKkVX7Pvvi+cCAmovjkiQCqWqAHGzcaT4Ky0FvUIMQkGelYOQZzgE9QG5m1+CxsTOWbB1it1Da86uYdmpZRSYktFoKtBoxPylJKOE7S9t59JWx40zzQUPPw/ix8cT3ME5m59WLGSdnrpounSpo4D/yONw5s3qjys8W97n2pAIHggTdkK76oSZLaTx11bxN/618Vz71bWNeHJ1h9kMu3bBiy/C+or/40TcQnRe1Ymzakj/Hdb3hdwqjUuXl8G5DxvnZI06MFWfB0r3MbVWNCtI13lrQNX1UUPAEBzOqbH/IHpWzWS61BjYFEV8s9nMqztfJa8ij6zSLHpH9uafo/4JwIPrH0RjqL1JVqVQcfjuwxQ9XVQtG7u+sNQ5Ch3XOY59d4wPu35I9snsBn3dpkDv+b15LPMx4kbHAaIhEVxT/LUPas87V4k51VN/PGWxWq0K3whfntc+z4inR1ga0ad3mi4azeoAD6UHm+7fxHfjv6vT/n9DwNXxxZWMP7COGY7mey0R0rrI27v2+U7KrhTKcspQeilZPms5/+7873q/fmtTcdmivsSfsyigzMxMnn7qKdZptQx2cozBwDqtlqeffNLiymg2W4m/zp2dv74rOX/SmlCpxOJmAzDsyWFMfGuieI3QzuQ+kcuW+VtcHscScxPZnbqbZadcEyd92PVDVt64EnDsuNeS0RycUZ2Jvx07dnDrrbcSGRnJE088wcyZM9m9e3dDntvfaAio/FB7qikK2MmRhDm8tXcxADf+dqPDbsXmhkuDpFwJU/4CQxn8MQoM9bM7mtZpGiB87yV7UWeQiDs/tZ+lW7sqHHkM2yr+nH3R887ncX79eXdOvVUgdmQs3edY7T1CvUNZMXsF70x6B6Vc3C0cEX91RWoqZCUMI//mB/Hyr15AWZO4hjvW3sGVEnvG7eqOV7No3CKeGfGMXVi3TCYjrzyPAk2Bxf4mMlIQQNI5Z5VmVbMbdAp9CRx/nhjFRsDxjT3YK9hS5HOUI1dypYS327zNjlecd063ZCRtSuLSn5cs9hHuWH3GBcbho/TDJNdxubRu+Verz6wG4NrO1zoNZpeK81UVfxER4Cn3s6jqHKl4pYWvlzbOagXmm+CcDG4IeIZD2xnIfNtWy/kb8fQIbtl0C2ofN2VSzYARz4yg05t3YlB54eFhvf4dKayv6SxcBv689KfT43l6WsdkaRKbkQF//SUWMZMmiQmwQV5CvkwojiW1H1hVPK5Yd0m2TzWRSzKdH17aONr62XQK+7SFsOGgaILPZ01bpgbdBNgvpMxGM+W55WhLWs6M2SvIi+CEYBQqhVXxZxKsSk3ZO47g62eiQpVKmva0022qWn12CO7AuX+cY8X4v5Aht8tbcCX7EaB7eHXiz5Hir6QE9ArxRLBXMBg14l9TLwxCBojrsRK285milCI2PLyBpM1JTXtOTiCTywhKCMIz0BOdDoJKR9DD42qX7bdAqBoBMoJWUlDk2MLONuOvSFPE9svbMZqNLN73Mi++CNOmQc+e4l7hP+J7TmafdGlBF+YjiD+5nyiEOrP61CsriT+pSUWTA7oaugtaCEK8xD2oAnH+0mJcW6xl24vbSNrUMq4jCZoijZ07hSOkFFYSf/oY6mRyYzbDuE3QZ7Hj541ayDvonvo76QvYPQ+0jt0LWgzUARA+osZmBp1Rh8HrClplth3x135ce+IntAw1utlk5pe7fuHYd8cAWL0a3ngDDvxVhk4lvsuTBrlgcabyhYK/7IneslTYPRdy9jT82J+zG1YGQ14l0WioEPbOWO9jirJKxZ8Dq08QEQxb/7m1Yc+rBcJZdlFVNKXib8ulLRxIP4Cn0pOHhzwMwNMjnqaNbxvyKvLYlbLL5WP5ql0M63IDcYFxgPMGZ8mhpqVlu7oCpYcS3whfi7Xvz7N/pujpIm7seaNL+z8w+AFu6H4DZsysOL3C6XYKlQKZTMami8LueUqHKU63dQW6Eh2aglbImLRCuJILCtYxo7URf1JDTk2YuHgiD19+GP9ofzrP6Ey/u/rVm9z4X8v4A6sjmDPi7/PPPmOsSuWU9JMwGBijVvPF558D4porKAC5HBJqKLW5Q/xVfW+dpnaiz4I+lt9DvEMYHFPbmVpxQ/cbAFh/YT1FmtrXOYMfGkyvm4XN/d8Zf7XDLeLvypUrLFq0iE6dOjFmzBguXLjA+++/z5UrV/jss88YMqSF24z8L0KTDTl7CfcUBcX8ihq+xc2M4rRiLjz8IeEX97HLsIQnNj3h3NvcOxoGfgy9XwNz/VQssQGxdAjugMls4mjG0Rq3vVQgOpTbB7V3qpqUioa2i9bYgFiWTFrCtzO+xYzjm+AfT/7BT9N/ahV2fO5gxNMjuPZLa6euv4c/s7rN4r6BVlVNz/CeAJzIts8zOvjRQUsnh6tITRX/x7Zz/Pk8svERvvzrS4Z8PsSOaIzwjeCZkc9wZ787q+0T5SfuwlXJQp1RR8+PexL5dqSVNL74LZz7yPkJereFyYfx7PckIEgCR8HwnUKEDj8xN7Hac35Rfgy4r2H805sDGx7ewO//+N1SCDSrSywdlLUp/uQyOZ0rs2xSdMfcfm2jycias2sAmNl1ptPtpIV8VeJPLheTMi+d8+5WSfHnpWtn7f4btRoGNlIXty3MZnx8wFfTlY4Bzi3LWioC2gagiI0GuRxPT+uYa5vvJ6FPZB8AzuadrZF4r0q4nKz82nfvLojcdu3AS+nDmBPn+HTcKktB3mi0kni1ETwV+gqSNAfI8d9EUZHzml3f/FcZf+ISt3S9x/qgocxSgGt0dFjIFaYCkFxylh2Xd5BVmkXxhWLeiXyHfUuc26Y2NUqzSilKKcJsNtMrohcvDH2Ntrm3I5OBu9b4+3LXs6V3LPtjbnTaDVjV6hNE40f3OKHEzMqyfq5uK/6yT1m2dUT8FReDTmlD/J18BZZ5QXHdmhvqDEOFIJaAjJIMjvkt5mL4e1RUQEVBBfvf20/avpahBonoFcHtu26nx9we6HTQLW0xj7f5jX5t+rl8jNHtRnNn23cYefowpcWiqKfT2X9/bTP+bIua25K30al7OQsXwqJF8NaHeXySNZ+eH/cku6x2VcM1na7hnv73kBAi7iWOlKAOFX87roXfurr8HpsEI1fC2PV2D0k5hmVm8cak711Q+yAW/rWQEU+PaNJTrA0/Tf+JJbFLatzmVKogRELVMbi85M3eCclLxc8yGYQMhFAnhZCz78LGQVDkvEGhGvIPw+Wlwqq3pcNkqJGg/Ne2f7HwXDTn27xit4ZqSdAUajj6+VFSdoqmL8m1o0N/MTYEeATQNtSFTqHwUTBXLyxeJfi0hfF/Qq+XGj4PWl8Kfp3F+H7+E1juLQhGrPMbeWml4q/E8Rh/YcMFzq5t4ntSM8Bw4AjB6SeahfjTGXW8s/cdNidttjyWV57Hwl9FBMVd/e4iwlfMSVQKFZM6TAJg44WNtR7b6EDt2VDoG9mX4W2HO22K7nVTL+4/cz8xQ2Ia7RwaCyUZJeQm5tq5pvh7+DuOonGCmV3EenNrcs3EeUpRCom5ichlcsbHj6/bCSMs81bOWknQ960scLQVwmi05uDVpviTxozsViJ8dZR9XhOk2ujAewcy6Z1J9XaYa82KP1cbSKqiNsXf2h9/5BYX/yDzKypY8+OPgFXt1759zQS1NB+oifirK6lZG3qG96RbWDd0Rh1rEtfUuv2Ip0cw+MHBdufSGq+VpoLLxN+UKVNo164dH3zwAddddx1nzpxh165d3Hbbbfi46mvzN5oeyT/B5mH0DhKz0qLK0OXkbcnsXbK3RVm/GTQGzIgbxCHtT7y1963qaiddAST/KFR+ba+Dro+CyoU2lFrwydRPOHDnAYa1HVbjdiqFiuFthzMoapDTbSRLMdtFq9Shd13X65wqjAbcO4Brv7q21o7j/0Y4U/xlHMkgcXUiFfmut28kfrqD6MQtxERVJwLSitMspExqcSrDvxzOsC+GkVqUWuMxpayeqsSfVNxTyBTWxc75D+H0aziFQg3B/fAMCLF01Tiy++wWJvIepTxEP5tqtEwmY+qHU+k2q3VmQl711lVMeGOCpRBY7iE6/0O9Q/H3qN3atWe4sPvMNLtP/O1N20t2WTaBnoGMiRvjcBuDwTrhqWr1CSLnz0vr3L53RpcZXOu1mMjCa2tdBDQYTHr4ORD23IyvLwy6sI6fRp5gROwI0valseW5LRSltHyViLZES0WxWEHZWn06slZuH9gepVxJub7caYc6VCf+pOaA9pWHVCigcyc5vtqOtNdeZ9lPIvDkcuu47gx/Zf7FlFWDORZ3B3q984mnRPLbLaK2TYNVDWu75BT93uKS+kEAviu4n9Ffj2bLpS2EtAuh38J+tOnfcvLbNjy0gXfbvYvZaKZbWDfu6/U0Mfk3o1a7XxPtEiGI43L1JYqLHd9jbYm/41nHyS0XLbmhoeIa0Out15Crir+uYV2RISOnPAeFf47dvrYoLQW9LfEX1Bvibmpaq0+ArZPgN0FWZpRmsEP9FEmRi6mogLCuYTyS9gjDHqt5ntQckIou7mY/KuQKFnR6BG9dHMXFoingllvgLZsoHlvFX6+IXhj/aSTYK5gSXYmdzfD2y9sxI65VqTBbE54Y/gQfT/uYobFi0eqoeFxaCjplFeIv5jqIv829N9oMkBR/paY8FAql5X6vUCuI7B2JZ0ADVwzqibgxcXSd6ZxQNZvhYq4gRMb2j7GzN3KK3P2w50ahdgNBvhhrqEqEj4aeL4GXG/eDgR/BHC2oXLe2bzb83kM4tTiBdM3olHl2a6h97+1jkc8iMo46qYQ1ITyDPHmq4CnGvyaK8lJxL6i9tTHUKbJ3wp9XQWnl2rbS9YScvSK7HkSuq18C6ItFln1DIWoSTDkMbWdAQHdoN8+yfvbxEWNnRNFUvp2yjkXjFjk8xO27bmfhX84z0JsLfq7IUdyAYtcOIpP2uKzeaSjiT2/UM3fFXB7b9Bhf/vUlZrMZvVHP9T9fz8WCi8QFxvHP0f+022dSwiQ8lZ61Wn2azCai34lm6BdDq61lGwLPjnyWXbfv4uZeNzf4sZsbe97aw4ddP6xXZpm03jyWeaxGdymJ8B0cPdhl23JHOHzlMD+e+LHWhva/UTtqG19s13u1ESGt1eqzNuJv+8vb2b14d4PbF7a23DZbSHPeuir+cnOtaxtbFBUX4+oMMQIoqlzEnBOmRjXafIJV8edorSpB+jxsa1yFlwt5J/od9i7Zi86oY9L3k3j6j6ddsqGWIJPJmNt9LgBLTy11eT/bc2mN10pTwWXiT6VSsWLFCtLS0njjjTfoXNtV8z8Khbtt6I2N8FHQ9220CPVQsa4QgJNLT7Lp0U1oClsOLR7cIRj/Z+4jO34IOQZBBEhZZxakrIA9N4kMBAmuWizWgPHx4xkYPRAvVc1V+gnxE9h1+y4+u+Yzp9vYKv7cuf8lXJVA71t6W6wk/lvw+z9+Z++SvZbfPz/yOb+d+82S2QRW4u9iwUW7xycunsjTxU/jFVzz55J9Mhtdmbg7FhWaCcxMJDau+vBWpitjRpcZjGo3ilHtRlGsLWZv2l6e2PxEjceP8hV34aqB91IORrhPuJXQHfYDjPqlxuOhL4Xy9Brl/LbZUAqFgoSEhJY3vtQRHa/uSOfpnS2TonB1HGvmrOHdSe+6tP+cHrPpmvoW4dlz0bsplJJsPqd1moZKYbVY0mph82ZR5M3PF99dpdJq32gL25w/R4q/oW2HMownCC0ZJ6w+Nblw7AXI2u7eyboDuUrYRQZ0tWRMSSTTlUNX2LVoFwUXW67iW8LyWcu5/MDbgJgsX9vlWh4f+jij242utq1KoeLZEc+yZNKSGsfuqpPYtMpm9hibxmMp6PqsTTO7tH1goCB+akLnUDEn0qjThG2okwmzw0VU5ARRhGsiSIsQlUmcRLmhnB5DejD9k+l0muok8bsZ0Gl6J4Y8OgS5Uvzxpe+6u+QOQPugOAAMymJSch1/ONJno/bSM2fFHDp+0JEdl3egUEB4uHhOsvt0VfHnrfK2FIKL1Kfs9rVFcXEVq8/Y2TDse/B00HnQmIiZIQhHsDRhGBUlVFQIwsY/2h+Vd8vIILty+Ao7XtlBwaUCtFozJgxuZz+CdYxPT4dXXhGZfYcOWedvkuJP2k4uk/PQ4IcA+OqvryzH2XpJdPKPixvn1utLhaCq14XBIApKFsWfZPXZ7Qno/apbr9HoKDwFZ94WVoWVkEicMlMBPj4+aLXWQVRboiU/qWZr/abG2JfGMuV959Zq585BsVncPK4d46JqxSdW2Hx7Vg4gZ9+DZd7C4tERQodAzxfAp53j551BUYduiOZA7ByIdp7VJ6lE9co8O8vsgLYBtBvdDoW6+efAMpkMz0BPvEOEj7tUZMo3C+JPsjx0CGMFZG8XBKAEs0msaw8/Alqb78S+O2Dn9VBwvIHfARA+Eob/CMFCHS2TiSYWb117BvhNs8xnqsIr2KveCo6GRmOsj/KuvpnkXtMaXfF3LPMY7+17j0NXDqEz6rhx1Y2sThRrlGdGPINMJuOhDQ+xLXkbvmpf1s1bR6i3vQ3JjC4zyH8ynw+u/qDG17qQf4GssiyOZhy15Ms2JcxmM4c/O8zplW6omVsIEiYmMPL5kXgFe3Eu7xxX/3A1T21+yq1jRPhG0DW0K2bMbE92vhaUGqAnJUyq1zkr5UpiL8fCb2Ay1r9O9r8KV8YXifhTKKi1Iei/kfgzm80c/s9hzqw8Y7k/nP3lLEuvXUre+fpZkLdmFZcjcswV+Ppas8Id1QcD/P3JrP6wQ2QBAZWLF6m+0amWZb47Vp+2781QYcA/xh8PPw9OZJ1gU9ImPj38KR4K97I55vQQUWSbkzZbGnCdoeBiAV+P+ZqDHx9sdddKc9R0XWYYfvmllkL23wDAZGphN9fgvuKfSnR3legLMZvNDHlkCL1u6dXiOm41GjDKKig1icVPtayWmBmiWzZ2lvj92PNw4ROYkQaKlvFeJGWIXi/IBGkgOpt7lhPZJ+gY3JHekb2b7wSbGEe/PErcmDiGPjIUvVHPfb/dh96kJ/mhZItNRphPGNsWbKNrWFc764zaCD8QE44vR3xJRK8IbttxG1d8OmDoFEqsg5ifzqGdWT1HLKq0Bi3PbnmWEl0Ji8Y77m6V4Ezxt+OyyNizzQTDrwpZ7QhbxoEmg6ioVE6dcqz4s2RDZZ/CZDKRnZ1NeHg48koG4tTPp9i3ZB8zvp5BSKcmVoU0ECTiL9AzkGu7OC8GVcXkLmP4JGcMJpMg2GtT3Ugwm82sShQd1Nd1uc7uuT/+gE8+gfh4uP128VhoqONaWkwMBJeOxC+02P6zt0F5ZfSolxdQngKnXgGlF0RUJ7AaDGN+A6hG/PWY14P4ifEExLZ8RUDCVQmka8T17OEhcvykLD9H+L+x/1frMavaVjgi/rbIn+JCZAhHz90KhNtt70q+X7BXMOE+4WSXZVPqeY5z5/pbckAl5JWUsKFLDzz0kag8dwKVDFaP52p/gYbCqdcYbDjOF/yE0iS6WIu1xWRmZtqNLy0BvW7qRa+bhK1vcmEyR6/kolHGEOrhvjrSS+WFt7EN5YoMEjMv0b9r9TFTUpisSf+IxNxEwrzD6B0h7tWRkYL0y8wUFrESsVsb8QeiieNiwUVyZKeAMQ6Jv5ISa5abM6usJkHXRy0/+qnF9WFQlFBWbsJsllGQVIBCrWgRY0na3jS2vrCV2JGxFOqM/D4gjK1nfJlhKrDkB7sCf39ICf2cXL8tdMh8Gn96U14uCL/AQKvdr7+/GbNZFP4X9F7Ayztexlvljd6oR6VQWSy8xrYf69LrmswmijRFGL20QCR5eVQeXzwv3UNURn9i/GKI8KldRdhsyNsPRx8H/y6WjEiJxCnS56LVapDLPaDS1WP5rOWk7knlmZJnWhyR4AzL1xZjUIqLoUvVwb0qTHrRjOPVBsZvFR9q2i9CwRV7PfjENdyJZe8Epa9Y67V09Kr5fi0REjpljqXgCNB1Ztca1ZhNibLsMkozSwnuEIzKW2UpgOUaKxV/Vd0JTEYwlIA6UDT5zEgHTxvyRiaHCVvF56i2mWz0/BdEjofABrBsT/oKrvwK/d5xSioHB4v7m6P7k/WtmEjZmYJRbyRhYs22/E0FR+uj+qLMK5TyQNcz/vLyhN2fu3W8xNxElp9ezsMbH8ZL6UWFoQK1Qs2aOWvoFdELs9lM55DOqOQqfpz5o6VJ1haeStdqHwfTDwLQt01fu6bHhobBZHB4/5XJZPzx5B9E9olsdW41HSZ3oMNksbY/d+4c6y+sJ6ssy+3j3NTzJi4XXaZtQFun2yyZvITHhj3m1hzGEVQKFf2O9MPzmCfGV43IvVvO3L41wZXxxdb2sLbpTFPmgjYEHLrUVIFMJuPBCw9SkmFVxBalFHHut3MMfXwoIR3rXqNqzRl/Uo3L3Ux6mUyo/s6fF8Rfuyq37GtvvJHvFi3iJhf+KN96eTHjxhsxGODCBfGYq4o/d60+Q7uEcud+EZf02WEhkBkQNcDtOX6nkE70a9OPIxlHWHl6JQsHLOR41nHO551nVrdZdtt6BnmSeTST9uPb41k59Wotir/m4Iz+vgs0MBpa4txQCPYOBMCEkVJdKaGdQ4kdHtsiuiclZJ/MpmLHAcwGoUX2UnoR4FGluOQZBp0fAFUlu+YRAgE9RJZhPaAxaPj44Mc88PsDNWZFueKP7+Fh7fixtar58OCHzP55NktPOpYua0u0vBf/Hr/d/5tb597S8UT2E8z6SQzU5/PPozfp8VX7ViN1R8eNJtwnvNr+GUcyOLP6jNPjG3VGhj81nL6396WoCHJU0RRGd69WdK8KD6UHb096m0+nf1qtg7IqnGX8/XHxD0AoQS3Q5Nae1xU3D9rPt/h4OyL+hrUdxv4797P/zv2YzWYyMzPtxhddiY6CpAJKs0qr79yCYdQZeTPsTdY/tL5ekyLJecO2K7w2GEwGHh78MBPiJ1TrppS6qi5ehI8/Fj9XzfeTEBUFUQWzGZj+NbO7z7Z7TmvQsjZxLZc1xzBjFoq/gO4wLRHi73D9ZOsBHx9IDfmGu4514/FNj+Md4k1o51BUXi1DqVMThj0+DMX0q4GG8463tfrU6ayqrbaVa2+tQcuylHdIjHmK5PQKy4TWHeIPoEtoFwBKPc9w4kT155OyM6jwSKHE6zSBfnWQrTUE8g4Qrl8LmFEYxJeoRFtCRkYGK+euZNcbu5rnvGrBe/veY+aGgVyKeK9Oqi6AYJmw+zyfK2zWbAvLZnOlraIij/eO/QuAV8a9QoCnmINEVnKNmZliwSORMq40HczvPZ/FExYzLl5Y3DlSg5aUgM5W8XfmLThwT/UNmxB+HlZ7o8LyMjDDB50+YOOjtWcJNQV63tiThUcXEtU/yuJmYcbsdsEsIAAyglZxJWQpxphdls80Pd3eueFSxTFC3wzlplU30S6wHVmPZ7Fu3jryK/J5d9+7nMo5hQyZQ3WyIyw7uYzgxcHcv3UeMpkoHEvqQrBen/3z3ib10VTu6n+XOJkd10Hiu269x0ZH1BSYuAvCrLl9kuKvUJeHVqvFaBQqRoDuc7oz+KHBmI0tY82U+Vcmq25eRcruFKfbnDzmwaBzG3ht0Nf4qmuoghUcE3a5mX+K32Uy8bmdfgPKLsGg/wgSyBF0RfDHGDjphqJzz81w4C7Xt2/BkPJ1tcoct11TmgqnV5zmk96fWLJOpSJTltYJ8VeaBCuC4PiLguTzdDCx9GkH7W+2rxoHdoeO94h96ouiU5D+mz2x+NfT8Nezll+lcW9t0jIW717s2I7QDEtnLGXrC/Y5ZesfXM+GhzfU/zzrAEfro/qgPK8cfYmYBNY2Bw0KEut9s7lmWzRnuKH7DczsMhM/tR8VhgpUchUrZq9gSkehPJbJZDw05CHOP3Ce6Z2n13q8Io1zO/8D6QcAaowqqQ/K9eXEvRuH5yueds49tpj7y1ym/Wdao7x+UyGlSNwjqjWmu4DnRj3Hp9M/ddowKiHGP4ZI3/rZ/6vkKvYO3Uvh/xW2qFpfa4Mr44s7yi5bxV9LvL9VhatWn0pPJUHtrfeXAfcO4AX9C7Qb6aZ7QRW0NhWXLeqTg1dTzt+dd93FVr2e/bUcYz+wTafjjjvv5PJlUf/w8aHW+mjVeBRHqO2al2KVOoXUzUVofq/5XNP5GjoEd2DJ3iX0/7Q/1/98fTW1tFeQF08VPsXoF0a3OpK4OTijv4m//3aUXoT1/RkZ/AUyk6iWFVbm/Bn1RkyGlqNQTN6ejPnX9Sg0yYBQWVm6BAxlcOL/RLafLbo8AhO2CTudekAlV/Hopkf598F/V88VtEHEWxG0e7cdlwouOd1GJnOc8xfjL+QlzkLT1b5qfMJ8LPYx/y1Q+6otylLJwqJHeA+XO0A2PLSBtbeudZp9qPRQMvKZkfS5tQ9HjojHoqKqk0nl+nIuFVyq00DbPrA9PcJ7EB8Ub3lMY9CwM0XY9ViIP6MGVoXBvlryd7o8Ar1ftVh9OiL+/D38GRQ9yK74aos+t/Xh8azH6z2pamoYdUbCe4TjG+FrIf6OmX5ixekV5JW7bglhDrpAevBPHE8/5/I+KoWKh4Y8xOZbNlcLZbctuKZXOro6yvcDq+1fbq4o1toiuTCZGctm8IPHSKByUqTwAP/O4NXIio2kL+HoU/j6glFeTrr2DEkFSRj1RkoyStAWaxv39RsI0nVhVBaxJ3UPmaXOTS20Bi1HM47y56U/nW5jO4nNyBCLLR8foeYBYadrMBlQG4Pw1MZafPClSa+rxF/XUKFIKPVM5OTJ6s8n54kZvLexjX1X6P47m66QP+JnDsWXAzIr8acrQSaTcXbtWdL2Or4/NQc2PrqRNQvWAFCmF4UkpcnH7UYBCWFKUZC9WHCRTz+FefNgZ6XjWkUFmExwNvpFirSF9IroxR19rUS9LfEnKSI8PV1b6F/f7XqeGP4Ewzr0FO+lzHqNSygpgU5X/sWSwauYGD8RrmyAy+7lGzQIUtcIe7myy3gpvZBVLhMKykqQyWWMen4UXWe1DOWNV7AXkX0iUfuqKdWLgqevMtDt4/j7Q6xiIADRAw9asj/T0635fn5+sP/KbvIr8i3WM5IyMzE3kUc2PgLA4JjBFqVbbZBIjtzyHMsYY2v/JHVa28WoGysg/VcoaGG5PV5thNW02tqsN6/nPHKfyGXl9Sstj0nXfb87+jH+1fEWG9/mRt65PE78cILSTMeNVDodlBd7EF48iQdGLaj5YAovkUdebpMdLZPB0G/h6pP25EtVKH0ESaN1Qw7QexF0c892rtmQs1uML3kHHT4tNf/pVNkYTWZLg0VpZil/PP0HFzZeaKozdYqogVGM/tdoQjqL77lUZPrngH+zbcE2ZnSZYb+DTAHtF0BwzcV+pyhPF3l/9UG/t2BWjrVpFuDKekizZghK86QvUh/jqT+e4kJ+9b+1XCnnms+v4ep/X233eOruVJK3JtfvHFsI9r27j44r38CzJLvWgq1MZi3k10XBI5PJeGzYY+Q9mcfO23ZydOFRhwRfu8Ca13kpRSn0+KgH8e/HO21QPnhFfOcGRTcO8eet8qZQU4jRbHSYfQ7QbmS7VulQs+mJTfw49UfAhvjzr1/dyREashCsUqjIisyirG9Zi7nP/rfCHYJHGi+0Wvvmw5YK6Rx9fJxvc/z74+SetbdklCvkDeLm0JqJP4kcqwvxV1N9MDIyktffeIPpHh5Oyb/9wHQPD15fvJjIyEg7m8/aPhZ3FH+2a+DENYnseXsP+gq9pdYd7VcLy+gEDw15iLVz19ItrBsv7XgJg0l0DdrmqkuQrrPWfK00Ff6+E/y3Q64GXT7enjp6J3/B3YE/E+QVxIkfT/CK+hXO/ep60byx0XVmV8pnzyc7QrBldoPFhU/hxL8g+ftGeW2FXEG3MGE9cSLLgVQDQZjmVeSRUpRiKdg4g23OnwQL8VfsuLAqk8m4c/+djH3JNZuo1gCjzkj6wXRLILaF+AurbldyIf8CT21+iuf/fN7u8VH/HMX1y653SvzZ4vffxf/jHETsbEveRvz78Yz+2n2rxeGxwzlx7wk+nf6p5bHdKbvRGDRE+UVZCv6YDNDhHogY79Jxa1L81YbWYo9VFWpfNQu2LmDksyMtRcA1JU8z++fZJOYmunycw0HPcDT+RjYmN4wNtUT82XZCObPxk7p89WYthy5dsExIwNrl5GtohwyZUPzpi6EiS1wfjYn0XyDxbXy8zagNYnWRV55H2r403ol6h6NftbBisQOsf3A9uk3bALioO8DwL4cz/lvn36cjGUfo92k/5q+e73Qb24w/W5tP6St0NEP8XWKUfZEhI0lEzFomva5ayUqKvzKvRDIyqltmpRUIAtPH3Mb6oNkMF78WuT9NAbnSQpzJDOJGVaoTK7unSp5i7pq5TXMeLiDzaCape0XxXDpHhcmnThl/AG08RePGgfMXWbdOPCYRtKWlQmWSEvYfAN6d9C4KubVL2hHx54rNpy28va0NKVU7KUtKIKCiDzO6XCcyAceuhxnO1UeNhtIkYUdYkYVMJsNbLgrFRRpxDx/70lh6zuvZ9OflANpiLZpCDWazmVJjIQB+qkC3jyOXw/O3iaL8yYKDlgV3erp9vt+etD0ADG873G7/4bHDmdZpGm9f9TZr5651+XUlW8Oc8hyHOX8OO62V3jBXB4O/cPl1mgwmvR1B4av2JcQ7BLVKYRlrW+qCvPsN3Xm27Fk6TXPcnSx9Lmq1+B7XCP9OcM0liK9CEPolgHdUzfvKlYKg6f+uS+cNQPubRCZoa4A2X4wvJY4JPOk7YZJrMcpLLWsoXamO3W/sbhHkUvTAaMa8OAb/aDE2SsW9dsFRjI4bXd3Gzy8Bhn4NMbUrtqohdRWsiRENGfWFqkoT4YTtMNWatybNc/zMYq2aXmyfaS6h2/XdiBogrmOz2cyOV3Yw+b3J3HOseRXqDYXIvlHkxvRG6x3sUsG2LtZ9WoOWTw9/aml2VClUjIgdYYl4cBdRflGkFaeRX5HPoSuHqj2vN+o5kiE6YxuL+AMrQSmtg6rCbDajKdRg1NfuntSSUHS5iNxEQWzUR/EH4rPYn7bf4Xp38g+TmfLDFKd1KKco+Es0m9hAJRfN/vraHIj+Rr3hjuJPrbbmRbcGu8/aFH/lueWsnr+6mgpcW6Ll4h8Xyb9Qvyzn1kzmSDWuhlb8ATzy2GM8s2gRI2UKxso9+QH4A/geGKfyZJRCwTOLFvHIoyK+4UylcVqXLrW/tlQzKSmxunRUhSNS88QPJ9j8xGbkSrll/iDVvuuKNn5t+GbGN9zY80YA1p5dW61BIudMDvve3YesvNTu3P5GdfxN/DUwWlwx3jsGrr1EivdjxOTfQmfj9fiqfQlKCKLH3B74Rtai3W5C+LXxozyiPaU+wjddylUDoNODMPR7SLiz+o5p62DXXKEKrAd6hoti1vEsx0HqksovzDusZpsfHCv+2vqLxaAz4u+/ESVXSvh80OfsfWcvYK/4q4q88jwW71nMF0ftC1oJExPoMLmD04615dcvZ/ms5Vy8CImJImNh4sTq2+28LKQdHYJdyOBzAb0ievH59M95fuTz1u+9yhcGfQwJtSj+MjbDtmlEe/8FiOvEUefX1ktbuf+3+/n2+LcEBwdXG1/O/XquVQalS9BqwSTTkW8Uxf2EYNczQ9qqRfbWmYJjLm2fUpTCF0e+IKfM8UxbInnuuAM6VF4iVX3VJchkEBJqZlOfYIb80JHUImtnv7Tg9dTGAZWWpBc+h9WRTrvcGwwDP4YZqfj4gNogWIm8ijwCYgMY9MAgInq14IyoSpxecRqShOo61yD+t1XaVkXnUGFWn16STom2xOE2thl/jvL9jmYK4q9bkMhIknzw62r1qfMXC/qqqr+0IjGD95fZEH8yGdxQDsMap6mlGspSCNFvwktZgkwnCoGlulKCg4NRKFuWHdCCrQt44NwDgFXxpzDWnfhL8O9B25w7ibpkVcdIC++SEkgP+R6zzMDAqIHVctpsiT938v0knMk5w8+nl+MTUgjYE39ms3WuIFkYI1fZqzOaCp0fhHkGCBXFQW9lZQ6kxvF3qzmx6fFNvBH0BpoCjUXx56eqW/bgkFih+DuTc4aQNuJmbKv4CwiAPamC+BvWdpjdvkq5knXz1vHo0Ecd2pU7g9RAlleeR3CIcN+oqvgzY+LH4M6M/Gqk1XpPJhMEUUuCyQjLvB26HcjlMvz8xPlKhZDUPal8P/l7Lm117p7R1FB5q1B6OP675udDvu8u8tp+zemcU44PcPpNuPSd+EKrWs7aqkWhzSSYoxF29w7go/bhtj630b3oUcwyk2VcDGgXwAMXHmDkcyOb8GRdg12ec0MjeCAk3OVadrgzJC+Fi99WjyBQB9rZiErzJC+DWHunlzgm/gBMBhOFyYXknMph6wtbOfGTm0RFA0ImkzlcH9UV7SZ1IbnPDMwKpUvuAnUh/tZfWM/CXxcy8LOBDaLyUsqVFueZjUnVrbhPZJ9Aa9QS6BnYYGtgR4gLjAPgcqFjxd+u13bxRtAbZJ+sXzxLU2P28tk8mPQggCXbr65WnE9ufpIhXwzh3wf+bfd4kaaILRe3sOHCBqdOPw6RfwQ2DoKcvXYPqxQqxv45lnbXtyM/qX7ky/8yXBlf3LV0tLX7bOmojfhT+6m58dcbGfroULvHCy8V8t3E7+p9b5D+pq2RzHGHEK6K2og/gPvuf5SJU9LI6PAcbyZ05c6wKB737UpZn+e4nJZmIf0ATleWCru5EK/q52eNrLJ1w7KFo/c2+f3J3LnvThQqhWX+YFfLryOu6XwNn03/DE+lJ5eLLnMi2/6aStmZwsZHNlJ2XqgoWgtJ3Byc0d/EXwOjoYKlGxqSRFuyDooZHMOsn2YRM6R+THxDQleqQ1Nhpn3WQ2y/Jp03JrxhfVKuEF2tjrIOypIhfS0UOCbsXIVE/FUdUCRcKqzMbwhq7/B5WzjKH7NV/Dmb6F/84yKbn9qMrkzn6mm3aHj4ezD+9fF0vLojUDPxJ3U6ZpZmOrR8dNYhqC/ToyvTsX69+H3oUMeF+l2pIrtqZGzDFA7CfMK4o98d3DvwXvd31uZA5mY8jOmWxbajm/vRzKN8dOgjfjv/G7GxsdXGl81PbubPZ51bHLZEFKUUse1f20g/kI5WC1plFmbMqOQqInxcJ6YSvAXxd6HENeJv2cll3LnuTm5cdaPD56XJTVgYvPIKPP88jKzhUokIl+GpF99pW1sb6WeP8jigciwI6gUd7wMf54HuDQKvNuDVBh9fmZX4K88jsF0gU96fQvuxtY9dzY1HUh6h4JpbAMjSC+KvWmaODYK9gi3F9nN5jhXs0nesvNxK6tkSf1In9OBYQfxVVfy5Svz1a9OPdye9y4KId4HqxF9GpWVpoLJKwUChFhZvTYHkH2h7fhJRfucIKhnBq+Ne5aZeNxEbG0vm0UxSdjWDyswFSJkx9VH8TYyaTe/Ln+GjTaBTpbBHWniXlIixSG5WcVuf6uSFRPwVFlqtgN0h/q5Zeg1zVsxBF3oYsCf+ysvBaDJzOewTNqQuQ2PQQN4hKD7r5jtsAMhVdvOsf3ZbxvAz+/DVCFX7b/f/xoo5K5r+vBwgdmQsfe/oi9pXTblJMHT+HoF1OlakbyQx/jGYMVPoJcYDW+JP5n+F5MJk5DI5g6MHN8TpW7KFjWYj3iFisKmq+NMpc8iXn2NP6h781H6iqz9js7D/a0mQK4TCLcx60yzVlXLfb/dx46obCQr2QiaTWRbk2hItKTtTLG4QzY2MIxlcOezceiEvD9KCv2d70G0sP7W8+gZGHZx6FS78p3YfpdqQs1s0C7mC0ouwNh7Ovl+/12wqKNTiXw348tovGWd4G5UxwLKGUqgUBCcE4+FXR5/nBsSaBWtYNnMZIDherRbK1BdZfOQZfjzxY/Ud/noGDj1UtxfzaQuDP4WwYbVv6wynX4ejjwvLUVvoCiBrK1SIeYl0P1NViEJdTU2qnw/5nK/HfE1Y9zDuOngXfW/ry9l1Z8k757pdf0NBLpc7XB/VFVJzgkyGS3nCdSH+vj8uGr1mdZ3VYMU/Kbd8/YX11Z6Ty+Rc1+U6pnea3qjFxnYBolvSmdVn1IAo+tzWB5V3y88bdwapPuGqpXdVjGwn7pHbkrfZPb7u3DqMZiOdQjpZCFSXYNSCR7hVTV5p9XrvgHt55ZZX6DCpg9OGlr9RO1wZX1oT8VdaCu+8A3/95fr24Jz4U3oo6Xh1x2r15IDYAKZ+PNWpi4Kr+G9Q/NUlnkJyHsnJsTYXVcXhwyCXRzJ8xPMcPX+aE0npDBxzmvCI5/HystYZ8vIgO1vc0zp3rv21ZTJrFIqznD9H17xfGz+iB4n5g9SoWFerz6rwVnmLGAxgbaK9s0qnaZ2Y/+d8YkeI+09rIYmbgzNqmSxVK4bJ1HIy8yxIXko72QoKvQ9zqGK50+Joc+PHaT8S+cPbyFESGxQliDKTEY49L0LJnaHT/TDpEIQNdb6NC+gZUQvxV+AkuN0BJOLPVvEX5SdGcY1BQ16F48XRpT8vsWfxHorT6pnn0ELgFezFiKdG0H5ceyr0FZbMCEfEn6/a10KO2l6jJoOJxSGLWXVTdV9ngJvW38R1K29i2zbx+9VXV99GY9BYgs1HxI6o03u5dum1BL8RXK1Lzw75h4X6NHtXzQeLnSO6nqOn1mj32T1MkKGnsk+RkpJSbXyZ8sEUrv36WnfeRrMj/0I+2/9vO1cOXxHEn0p0T4b7hLu1IO0cJIi/dN0ZKvS13+VXJ64GYEbnGdWeMxqt39XAQNEoMXiwUI86Q1gYeGkrF7k23a2S4s9L1w6ZrHKyHDkBBn4oFNiNCX0pFJ/D37sMlY3irzkChOsKuVKOxiCKApkaMebWpPgD6BwiZrJn8xwTJV5e1om3RMa1reRgjSYjx7IEeTyhhyD+rlwRk1p3M/7CfcJ5aMhD3DBAdF6fqHIrySoT7H6w2kbxZygXnboVNbT1NSTaTKa4y3/ILW+LT9EAnh35LFd3uJqUlBTW3b2O1besbprzcAFJm5MsVp+2GX+uFOMcoWtX0cU4ejQ8IISElmJdaSl0TX+df2ivcEvvW6rt6+NjXfhKlimuWsCCdSwv9xFqIdvFVEkJmOQVnGh3LzevnYveqIft02D/HY4O1bjQl0DWdihNBmBg5FCCygZjrBDEdMGFAnLOtAx/ot639Oaaz69BoVZQbioEwF9dN8UfwMAoofpLNwtltq2ta66XUPv1iujlXjd+DVAr1AR4iPNV+ou/qW0hqLQUKtSi+B7hE4FKoRKWXluvgrQ1DXIODYrBn0OXhy2/quQqPj70MctOLaPcmI7ZbLYUQhKuSuDZsmfpdVOv5jnXKtj4yEa+n+RcdZ2fD5rKz8KhdZFCDdMvwMD/1P9kzn8MB+4CgwvVC5NeKLfkzU+IuYy8g5C1rcZNHK2hSjJKKEwubLTTchVlOWWUZYv7kVYryL9i779499DrvL/fAQF7ZT1kbWnis7TB2A0wclX1xtmsrbBlHGT+AViJP1lJpdVnDYq/njf2pPf83pj0JqIGRGE2mVl6zdJmcSAxmUwO10d1QX5SPiumfUNgxhk8PV3j8N0l/oo0Rfx67lcAbup1Ux3PtDqmdpqKDBn70vZZ7Cgl9Insw6o5q/j2um8b7PUcoTbiL+GqBK798lpCO4c26nk0NBLXJlrU6eV6UYUP8aob8Teq3ShAZItfKbEu/D88+CEAN/e82b0Dhg2FGakQ1EfM3zYNgdx9eKm8GHbnMG7+9Wb8Y5rBPeK/BK6ML+4qu+rSLNBQ2L4dtm6FpS5GiNdG/JmMjv8unoGeDLhnAG36tnH4vKuQ/qatkfirj+LP319Ez5jNsGGD422OVqa3DB0q7lV+ftbaRqKNk7C0Zm3f3vm5SHMaCbXl/FV9b2aTmaKUIotII/OxTPKfzG9QhfmMLjMYEDWgmp26X5Qf7ce2xz9MzINby7XSHJzR38RfA6NFFliPPUts6SKSIhfzi8ccNlzYQEVBBb/e8yvHf6ifSq4h0W50O4qiRVe5pTtCly+6aNN/db6jTA6Blb742jxha1IHSIq/C/kXHBIJFsWfC8SfI6tPD6WHRZnirJNy4P0D+ce5fxAU72KluRVBrVBzZOERfpz5o1M7rI7BQhloS/zJlXLixsYR3tO5hda2baKTvG1b6FGdU+TQlUPojDoifCLqfBPq36Y/BZoCHt/0OEv2LuG9fe9VJ9FLL0HKMtDUUsSXKywrypoCfCUV5Pn882TmZFYbX+LHx9N2aCOryBoY0YOjuffEvXS7vpsd8Rfh654NZVxgWzx0bTBhYH+6s3hjgZyyHPal7QPg2i7VidKiIjG5ksms393aEB4OXjqR8eBI8eeli8PPr/7N/27h4tfwa2dCZQcsij+DyUB+UT7LZy23WO62VJjNZi7vvIwhR7AiBTpxbbTxrXnhIFlsOsuIlMmsJI20iJEUfylFKeiMOryUXgxs35ng4MrYvYvuZ/xJ6N5dvGZ6uj3B42kKxlvbnkgvm2yQkguweZjrCo/6Irgv5oS7KdKGW4qWZrOZ/Px8hj81nHGvOghIbSasmb+GzY9vBuwVf3XpngRh4bt0KUyef5JH984mNeRrSkvFIkG6V4f7hjq18paaNKQFlTvXhZQDW6wWO9squ0pKQKcQF4pSrhSv3+OfQiXc1ChOhC1jIEWomqRFnbTIu3njzdx7vA4q90aGp7YtYUVT6Bnar87HGBg1EKVcSSnZqNWiIeRc5S0+RbYDqJ7vV19Y8qJ9qhN/ZWWgUVWxy/HrCAM/gfAxDXoejQEPpQc+KkEYV8jEBMeqpmlZsQiDHx7M+NecZ8nm5YFGXUtmiWeodS1SH3R5FMZuqq7QcgT/zjDlCHRcWP/XbSrsuxUOOD9fnVGH2S8dnSLfbg315fAvLUq75sRNv9/E7btuB6yd+BUeNTjCTDkKV+2p+wte2QDr+1Wz8nMZXpEQ7qDhMXgg9H8fQkTDg1SMVpRXWn06yfgDGProUHrP7036gXTMJjOhXUO55otr6Dqza93OsR6Q5i8NUX8pTi0m98QVFHqNy+odd4v4K8+sRGvU0i2sG70jetftRB0gyk9kTAIsPVm3Okh9ISnVnGX8tVb8du9vbH1eZJide+Ac5c+W07dN3zodK9Q7lKExolH99V2vA6JGsS9tHyq5irv73+36wbQ29t8ARaeg4IhQjf+NBoEr44u7ij9pzGgOxZ/kaJPufHi3Q23E30/Tf2Jx6OL6n5gTtGbFn3TOdVmzymQwuzK6edUq69zZFlmiREJ7m2lH98op6CkbR/rabD5TdqXwXvx7HP3yKOkH00lck+g28VeaWcq77d5l4yMbK89fRpBXEAp5w0WI3NbnNg7edZBb+9xa7TmzyYyhQEwY9XqxfmvpaA7O6G/i738BQ78mN/4zVMZAAAo1hWCGw/85zOXtjruymgNj/jWW5G5TOdn2AV45+BgZJRmgDoKpZ6D7064dZN/tsG8B6N1XzEX6RhLiFYLJbHKoHHHH6lO6QZZUcTFaMmkJK29YaemKqwr/aH9COoagULWsrKW6ImVXCp8P/pzzv59HIVfQJ7IP83rOc1r06RQiLAHO55+3e/yGFTcw+oXR1bY/tfwUB/59gA2/ifTZKVMcEy1Svt+I2BF1Ljg9N/I5pnWahtao5dFNj/Lwxof5+ODH9hvFXg9zdRAzo+aDmYzCriv3gIX4czQJi/aLxt/DH6PZyOUyx99Vk8GEQeskfbcFQu2jJrxHOD5hPharT8Atm0+AgAAZISVjAJGF6Aj5+bB5M6xLXI8ZM70jejss2kk2nwEB4KryXhB/4nts210rLXi9tXFWEvH4P2GPm12cdUHIIOj2DEr/aBRmL/x0HegZ3hMNGhLXJJL5V2bjn0M9YKgw8PWorwn8azsAJQYx4wz2qplhqU3xB/aqPaUSIiovt/ZB7Sl9ppQjC4+gkCtIqIyZPH5cTB6r7lsbMkoy+OXSj5g7/QLYT76v817CuBMXGRNik2/kGQF934Y2DoJJGwnSQsooq+BAyl8cunIIgK6zutLzxp5Ndh61YfJ7ky2ZTvcOuJfrQp/FR9OhzlafIBZgGy5sYPW5FVyIfhmTTE9aptbSxOFXg5hLumakBZg7Vp8SOZ0nE9doVcWfXimYwCDPIHGP6nQfxDm2JW5U+MaLYnCkuB4PF24iKeItUo2NnE9aB/zx9B+sf1DYmkUVXs/g879zX5/H63y8+wfdT/HTxSye+AbRlTybRPL2Cx1J97DuFju1hsIN3W7g3gH30i68UqFdxepTUplZ7HK8YwTJ0xAEU0Pj4jew6wa7vG3JDs3kKS54qRBiNpk5//t5UvekVjtMc6DrdV3pf1d/p8/n5ZkpVwvr6ardxuiLIelLYZ3YEAjuJ+4HtVhitlr0/Bf0ec3p03evu5vXtDGkhH1qt4YacO8A+tzap9FPzx1I17POR6wP4wLiqm8kk9Uvr1WuhooroKnD/K0iS1h5Oiow+bSFzg8I8hhQq8Vcx0tXu+IP4MC/D/DVyK/IOJqBh58HfW/v2+qUXFURNyaOGfufJi+md6MSfwA39rixwRsg5vUQc0tby1mtQculgktNUmTsGNKRYW2HMaDNAIfPm4wm1t6+ll1v1OKK08Iw/bPpjHlpjOV3L5UXynrk7L4y7hUAPj70MefzzlvUfjd0v8H1JtjSi7A6Cs68ZX0sZDDMyoOuj3Eg/QD/eOMfLJq76O+Mv0aGu8qu5rT6lIi/wkJr/JMzGAzW9Y4z4i96ULQlzqcqPh/8OSvnrazbiVZCGocNBvGvtUCyAYe65/+OHi1qTUVFsGlT9eel9YLtWlQi907biO8lxV9XJ305/m39Ce0cSmjXUJbPXM5v9/5GUIBgzpwRf1XJbplcxuCHBxM/oWaHpvqgpvvlT9f8xFd9P7DMdVojUdwU+Jv4+19A+ChUEf1RGQIBQfx5BnnyRM4TXP2hA1/EZoJeDyazmZSwT/no6DtojVqQKyGgC/g4JsqqodP9MOQrkLnvBSaTydgyfwt5T+bRJ7JPted7R/RmeNvhls79muBI8QdwY88bmdl1JkFejivJZrOZ0qxSSjNL3T39FgldmY6SjBKXiSlJ8VeV+HOGQx8fYsszW0hOEUTpqFGOt9t+WRAJ9cn3U8gV/DjzRzubUilM3Q5ylfhXG3bMgMMPWWT5KQ6itWQymcUiLqk4qdrzSZuTeNXrVU78WL/w5KaErlRHeV45JqMJrRZ0dVT8+ftDSMlYALYmOyb+vv0W3n8fPtwsrIKndZrmcDtpYiN5mruC8HDw1lW3tXl30rvc2+ENfLQdrSRC7n7IbAKrp9BB0GcRHqGCQJ+YeJ7j9x4nOjCaFwwvcN231zX+OdQDMoWMSUsmkRMpyKfCym7W2oi/2hR/YK/OatPGGlwNQpkiHUMi/g4JLgwfH9wimjYmbeSmVTdxLuQdwD7nT1pkSZm7AHhFQNdHIXSI6y9SH+QeQL2pM2Pbf0epZyJDvu7Ldctb5nXR/YbulgXlvQPvZVbgq3jr2teL+ANBIoZ5h1GmvsjuLsN5eMtdPJzUmcPxc5wubsGa8yfBHcVf51BRXM3UC+KvquKvQi0IkIYIQq8XPEJEMThYdLP/nvoDZ9o+QYpCjLGZf2VyculJjLrmb6e8sOECSRvFfVFXGYtcn2vD38MfL5VYoUvEn7Rwnxo3mxP3nmBqp6l1fwEHeHX8q3w09SOGxIvVem6utUZfVuaCyqwloeAvSPlZOG9UQrJDM1cSf5auZRksvXYpu99oHcqEiwXJGJTFqGRqS4OaBWm/CFveyw6y/+oKs9mS1VQj8g5C4pKms4puCMTOhrYznT4tOYJoldl2a6jhTwxn8IMNk69ZV2iKNBz+9DBZx8W8VSr4Vng6aQwtuywUe/UhhSPGwHUZ0LYO9+nEt2B1Gyitvn5whPBw8K/ozZt9fmXF7JqzXAPjAokdEUubflZHBme2b60JWq0M5HKXVRoS8VdW5jyLSYLGoLE0Kl7T+Zp6nKVjzOo6ixt73sjLY1+2EH0H0g8Q/348ff7Tp8Ffryp6RfRi9+27+eDqDxw+L1fISVyTSPLW5EY/l4ZEp6mdiB/fcAXtce3HMaXDFAwmA3etu4ufTvwEwD8G/cP1gxgqIGw4BNm4HMhkwvoZSMpPYv/O/eiX6VuERfJ/M9xVdkljRnZ2w59LeW45P0z5ga3/rF4XMRggNdlISOpR/HOSHDpN2UJS+8lkVdatNhjzrzFOawtqPzUqn/rledo2YLQmMkens87lXW0iqQql0qr6W7nS2owsQVpHhtr020jEX1KSIAw1GuFgZPtcVQS2C+SuQ3fRdmhbpn82nZs33UxQiKir1pbxJ5GavpG+TF4ymS4zuvD7+d+ZsXRGzdFI9UCxtpjtydvtHus0rRP97uqHCr3d+f0Ne/xN/DUwWpp9jQRfbwMqo8gSyS8vRCaT4R3q3WKUZYWXC1l9yyp88o5gkosKTpRflOik1eSA2cXFRJurRJe8sm7tFb0jezstMi8av4hdt++yBDPXBKngX+ym8LAsu4y3I99m+0vba9+4FaDDpA48kvIIXa/ryuakzby7710OpjtXDnQMEUXe1CL7LvBLWy/xy52/UHi50O7x65dfz/SfbwGZDKXSuU3jg4Mf5MFBDzK5w+R6vR8/Dz9+mfsLET4RhPuEW2xVLCg+Kyw2jA40+baQK2DQJzDwI+LixEOpqY67mSTiL9ucXW18CWofRMepHfGNqKFa3cJw6D+HeDP0TdIPpKPVQlT+XJYMXs09/e9x6zj+/hBRNJXRWSu43Xcld9wBX31lv01yMphkeo6XCeuBqR0dF20lxZ87xF9YGHhKVp82GX+zu8/mmpAnURkDrNfjuI1wnYu+Gg0AaYKu01kL4i313mQLpYeSfvcNoShM2PE+Nvhpnhv5HLEBsTXu169NPxZPWMwbE95wuo0tSSPZfKYXp1ezde5Q6QQsWfy5o/YDGBAlOp3TjIcxY7LL+avNMqVJoPBApvBAqZShNIkbVYmuhMjISLb9cxvvd3gfXZmuGU/QOaRFT32JPx+1D29d9RaYZRT5HOS3tO/E45qONSr+qhJ/7ij+JFVqjjYdg7zEbjFVXAzlklVcYHth3/R7H/tO7mZCgJf4g2hMJZjNcOTzI6yct5KKguZPT194dCH3nhC2o1qdWGHX99qQEF2Ffw0MFGOovGpOVgNBGp/0euu8sbTUxupTUvyd+wh+7QJFTZ+lVSt6LxKOBz7W8VpS/Cn9RNeDtBiXyWRc+/W1DH28ftncDYG8c3m8n/A+h/5zyOk2SWUiTKVDQHfUVZV40VNh8Jc1klluIWs7LPOEJBfsnzM3w5FHWxfxVwvCvEVVVKfKcXsN1dgouFjArwt/5ewvooFDIv7KVE6iINLWwbYpkH+k7i8qk9fdMz5sBHS4x3nz7K4bYMMgy6/h4aAyBtLBPNWSe+8MA+8fyPwt8y1zyz+e+YNF3oua/N4gk8mIjIxskDnuxS0XSflDfLauqjQ8Pa1zutoUPEcyjqAxaIj2i3aYdV9fhHiH8MPMH5jeebrl7yHl29eWlV0NNg0cDYlHUh7hpvUNl23YlLiQf4HpP03n4Q0P1/tYiycuRi6TU2Go4J1J7zC/93wGR7vR2BDYHcZvgcgq9vy6Qrj4NeHay5zoeYKdb+0kdkTNa6i/4RyujC9VSZDaIK0dCgoci7HdgdlsJj8pn6KUIgC8Q72RyWWk7aseKZSSAqGJu2h/7BdCLx8mo5Zpg7Rm9fau2y1o/h/zuebz+jU4KJXWRt3WRObYnmtd4ykAxo8X10teHmyx6R/XaKzNxLZr0fBw6NhR1BN/+AHOngWTSdSsQh0I8gsuFmDQGCzXd4fJHYjoGWFZkzhT/ElNLo6u+b8y/2Lt2bUczjjs5rutHalFqYQuDuWq769Ca7DWWgfcM4DJ705G7Svm5y31WklMhGXL4ODB5qnL/U38NTDkrvrENSUO3IvfbyoCEd/OvPJCAHJO55B9qhHaTeqAwuRCEpefQKERsqcw7zCxuL74DawKh+wd7h2wIguMzVe8dBRMDyLbb8XpFWy8sNHhfj5hPgy4dwBxY+Ia9wSbAStOr+CRjY/w+/nfnW4zvv140h5JY/ft9l3g+efzOfrFUXIT7VdVPmE+eHYQVXx/f+cTk6s7Xs17U96zqC7qg/ZB7Tn7j7Mk3p9YPQvq7HuweQToi1w40C0Q3FdYRnqJm3RNOX8FpoJq40twh2Dmrpnr1GahJSKiZwT97+mPf4w/Oh1469ozOW4Gg2Pc6+T29wdPfRR+qbNY+W0Y2dn2kyKzGTIyoNjrOAZFMZ6mUPpHDnJ4LIn4c4fkCQsDP20nYnIXMLfLrXbPScUqOyK6kQrGdqjIgD8n4Z3+seW7IE3O0valcXlHy7F2dgZbH/uHh/2DV8a9Ys3BcoI2fm14YvgTNRL7tp+tRPzd8csddPigA39e+tPynKT4kxZj7ub7dQntgrfKm3JjKWWe50hLE9/ttOI0Futj2dN5JD4+Niu91NWwvj/k1CMDyB0E9Yarj3Mo92aURnGjKtOVER4RjkKlwMPPo0WouUxGE+/EvMP6B9djNps5mnGU1PJzmDE2CLkzv/d8Xg5Oocflj+jpPYkYBhOXc79bij93xosgryBLQbvU8xz5+dZrrLQUytU2hWOjBgyloqO7ObBxMOy7DYBAb3GN6OUl6PXQ57Y+zF4xGw+/eqxkGwgymQyFWoHBAHs6TGB9Xz82Xl5Tr2N+dfQrBn42kJ2mNwHIDPiFS+EfoPKuxROpjjCZTeRX5FOgzbE0nkhdvGVloDT6Ee4ZY21+kMlBpgRFHVuIGxNKr2puB5LiT+5bjkwmsxvfe93Ui3YjXXTzaEToK/So/dQo1I4bIc1mSDUI4q9PhINcJ3UQJNwGnjXfp1yGVxthtetVc7YtAPG3w4SdFrvGVoErG2B1NKStdfi0pPjTKXMshUeAQ58c4pux3zRr00FwQjA3b7yZ7nPEvLyiAsyYKVUmAw4Uf5ETYODHEFjPLLfCE3D8X2Jt6w5iroVBHzt3IVH5C4VQ5c0ovDJK3RUlikKlsPvOBLUPIn5iPLrSpl17y+VyIiMjG6T+sv3/tnPqVWHT7k6xViqo5tXClQ1rO4ysx7NYecPKJin67by8k9d2CVvdIdFuuEokfQm/dBCK4jrAYDLYFWVtofZVt4pGRAnaYi2vB77Ohkc2kFKUwq/nfmXzxc31Pm6P8B4cvvsw++7Yx30D7+ObGd+4/nepiS3S5sK+22hfsBOtp5bSgFKUHnW3Jf1fhyvji7sZf1JtQK93nN3mDkx6Ex91/4g1t66xPDbv13ncsOIGQAgrpNrZ+TMGIpNEfe1K57G15vxJNUxnTfVms5l1C9dx7Ltj9XoPtUH6u1Y0f7+hy7BVgdZnuFOpYNYs8fOvv1ofl+413t725JtMBreLCGI2bBD/wLHNp9ls5j99/8MPU36o9rjpwkX8ci85Jf6ka0NaMx/48ADLZy2nPK+ctOIqEQUNiBj/GDyUHuiMOksEly2q5tK3NJw8Cd9/D7t3Nw9n1AJZqtYNY0tMkwzuC7E34KcQBZz8skIAfpz2I6tvWd2MJ2ZF3Og4bkl8loudxOQkyq8y+Cygu+hW9HWjU+3it8LaJOvP2retgrzyPB7b+BjXL78ek43KUGfUoTfqa9jTHs6Iv23J25j982wW73EchCuTy5j60VS639AC81vqgMxjmRz79hgV+RXkVYi7VKi38wwIH7UP0f7R1Sa/Peb14Mm8J0m4KsHyWHFaMUUpRRQWiglwQEAjvAEnCPAMcGzXGjsH+r0jCkGuwGxClr2d9nFi3EhOrr7JrX1uJfuxbJ7v8XzLHF/cRMJVCUz7eBoBbQMsE966dEMFBoKisuYg2WYUFVk7oEpKBOkVWN6fGRey6HduDb//5riwVxfFn1IJbXyj6JP8NbfEPQvAyeyTrE1cy4V8YatkmSyn/wqFJ50cqQEhU0DuHmTlqXh7w9mofzLg6y58fuRz1t21jnV3r2v8c6gH8s7l8fXgjwlNOYJSaf18GwK2BF7btiIXcmPSRrLLsokLjLM8FxJin/PmzjUBoJQr6RspisMlfocwm4VVRkZJBiXyVMrVl/HzsxnfjBrQ5QNNG/Ls6YlF8WfGzMmzJxn5wkgWHl2IV1AdAwkaECaDCf8YfzwCPKgwVNDv0378M6czRnl5g6m6OkXGEJdzL3eoN3CzZh+e+jYuK/78/cWCzB28fdXb/DRjJd7aeDQa68KkpMRG8RfUHryj4JoL0PMFN99RA0GuEuQSEOwjBjGDvISKCojqH0W3Wd1QedfPvqchcOnPS+ScyUGvB4OiEKOiFB/P+hGShZpCDl05xO7CpWT7b+Bs9Aucin2Q5ckfNtBZ2+O9fe8RsjiEBzc8WC33pbQUeqR+wJ/TUrmpV6VCouM9MPWke3PipoKuCLJ3Qbm1g0ki/gq0qZjN5hbZhRvZO5J7/rqHvrc5IPUQc4p2Vx5m8LlNPDTsfvsndUUNl+0nwb8TjPkVYlzolPeKhPARoHTiw9USoQ4Cr2jLGFMVUqNPVavP4rRisk5koS2qZ6W0HvDw9yDhqgRCOorruqICdMpsDLIKZMiquxMEdBHfWc96Zt/l7oeT/we5DdwgNPhzGLfJUpmUiL9dWetZvHsxZ3LOuHyo/nf358ZfbySgbRMuxhB1l6SkpAZZH417dRxxD0wHai/ipxSlsGjnIkq0JU7jPRwhzCfM7UZHd3Eq+xSxS2IZ9fUo8iryGBA1gIUDFrp+AN8EkQHpGQGlyW699q1rbsXzFU++Pfatw+eL04o5//v5FussURVmk5moAVEEtA0gr1zUMaT7Wn3RJ7KPG2SfjfPVkUfhj7Hi/lMVvgkw4mcuRs9DpVOhylKhLW6+MbO1w5Xxxd2MPw8P6/qhrqr2inzxogq1gvGLxtPvrn6YTWIdKZPJ8PD3oCynjPcT3ufP50Q99FKqkuPjH+bsxPvR+IXVavUpnZuzdZG2WMuRT4+QvC3Z4fPnfjvXIHme0ljcEuePzuAuGVwThlT2bKSlCfUeWNcJjpxnevSAYcNEf8Cuyj+/I5tPk8HEwH8MpPtc+5qzplDDoYe+Iyz5oEOrT7O5OimcfTKbxDWJqLxVlnzgxiD+ZDIZCUGiFnwh/4Ll8bLsMn6e/TNByaJJr6VeK7Z1z+ao6f5N/P0voMPdMGIZZrmQORRoxCJ12BPDGPQPxwqY5oBBpqLcS4SXW3JuIseJbkUfN2wKQodA21muky828FX78tmRz1h5ZqWdJeWK0yvwfNWTOSvmuHQc6SZZWmrfmCXltFS1svxvxdlfzrJmwRqK04rJLRd3Kcn6yR14+HngFexlN0Heu2Qv77Z7l8xTYiLujPj76uhXbL201Wn3YYMiYjR0ecS1jD+AM2/DljGMSBDdg5eqN68Q6BlIsFcwJU5WlAc/Psja2xx3Trd0aLWQFvItG9KWUlDhXvHM0xOefBJuuS+D2FteJqnjwwCWDjbJwiI0FO6dH0Zw6XC++w7eeUdMhGy7geqi+IPq3dFLTy5lxrIZ/Jb/NlA5ITJqYft0OLXIvYPXBZ7hcEMJ9FmEjw/olLkkFZ0lpSiF0S+OZtwr42o/RjPCqDMKywmzCYVXKfvT9nOx4KJL+yYXJrPi9AqLrVFV2BJ/0dFmntnyDAAL+y+0s0CSyayqv6r7uYr+bfoDoAkW1nG5uZBRKi5ID0OkfVZC3Dy49pLI6mgKmAxw9gMGtPkVuckLeeU0MKvQTSVBI0PpoeTOfXcy7uVxlOmsaiuFybvBiD+JaMnJqd696Gx7iYx2x+ZTwi29b2Fu75kEeYqBRlpQFRdDl/RFPNFuBVM6THH/wA2Nibtg8GcA+HuIyYxBUVJrflFTwmwy8+34b9n6wlZ0OtArCgEI8alf0XlE7AgAThUc4UCnKZR4H0dh9OH+IXfV95QdQiI5cspyqhF/UhNLs1oDu4P8g/DHSEj/xfLQy+NeJuvRLEaZHgDsu9t/vfdX3gx705JF1VKRnw9qYwjx5okMju1n/+Sl72BFMGQ5zhludOhLxJjemhA6GCYfEBapDuDM6nPcK+N4MvdJAuMCm+AkHcNkMNldrxoNqA1hPCK7xM7bdla3gW2oazvmGph0AKLcyBgtvwIbhwrnHBcRURm1vUPzAU/98RR70/a6eaLNA2frI3fRbmQ7vPqJvOfaCrZ3/HIHz/35HPPXzLcUP4tcMHtpCry04yVSi0WdYVbXWWy/dTuBnoGuHyBiNEw+Convwi/todz1qIIw7zCMZiMHrzhWC/719V/8OPXHag4+LRWegZ7M/2M+Qx8damlgrksdw20YKyvXpZeEC0Piu/bPaXOEYrcqZDKIvR6zRwTxF+OZ8PwEizXx36gbahtf3CV5ZDIrYVIX4i95WzJLYpdwZpVozBj66FB6zuuJTG5PIvuE+TD6xdH0u0vMWy5eBKPam55jQ5EZ9bUq/moj/jz8PXgy/0muevMqh8+f/PEkW57egkFTvzmKRKi2VDLHERqS+AsJEc3mRqN1fVAT8Qdw221Wi1RwrPhTqBSMf3U8AxYOsHvcK8iLif+ZRWaHERQWVp/GlJeLcwHrtTHt42k8r30elZeK9OLGzSZPCBZFmqR8a3axTCHj9IrTeBbZ5y+3NEhroIa4LuqCv4m//yHEqHrRK/kz7uskCtAD7x1I39sdd7g2NbJPZZNx+Ao6hWg/qVeXgH8nGPmzWFy6CQ+lB1d3vBqAVWdWWR6/WHARk9mEl4vZgdJAaDZjZ1UjDYJpxWlOix0H/n2Ar0Z9Ve8bZUtAj7k9mLt2LoHtA60T5lo65b46+hWzls9iTeIau8dzE3PJPJZp+T1+QjyDHhiEzlcczxHxpzVoue/3+xj37TiHkvBmR+z10O0pfKPEQvNyHZwY0/akcezbY62me3LP23tYMXcFJoMJrRZOtX2E+/6cx5WSWlrPHGDYMOg/uJz/2/FPzvp/hFFebpnISp1sbdrAxInQv7/Iu9u6Fd54Ax56yNo5JVkZuKvuCg8Ho0zL8dQk9qXtY+WZlQB46+MAaRyQwZCvoUPjFI6dwdcX1AZRSc4rz6Pb9d3odr2TZOcWgvAe4Uzb+AA57QZQ7n+MIV8M4arvHC8oquKzw58x++fZfHX0K4fP2xJ4l8w72J++H2+VN8+Per7atlLOH7hPBoM156/IRxB/OTlC8QfgqWvTvIV8mRwOP8jQiK+RIcNLKU6m3FBO1rEs9i7ZS3F6ywpWKtMLBkSJJzIU9cpLsIWkFM7Ntd6na1L8KRRWsr8uhLAEaV/JqqW0FPw0XZnUdpZY0JQkwYXP3e60bwz4qSXir5iKCjjx0wkWhy4maXNSLXs2LswmM1M/nkrfO/oK4k9ZCECQV2C9jjsweiCr56zmtj634aMXNpTdC58g2LsOA4ELkEiO7LJsO7s427mjXaNAykqR89cS4d8N+i2B0GGWh4K9ggnxDsFDJQpStsSfb6Qv4T3DMVQ071z33G/n2P/Bfqdzbul76rDA4t8J2s8X7iQNiZOvwslXat9ux7UiDuG/CLZWnyWlLYsU3vLsFhZ5L6IwuRAQxSUZctp4xTE8tkrzjskgSOFDD9T/hT3DIWQgVCUWa4ImC8ouVjoKOEFZimhKyxUNU9L9TVEm1uCSZZcrMJvN/Pn8n+x7b5/r59gC4Wph7o+LfwCwJnENHpUZpjUV8T89/Cmjvx7N0pNLG+I0a8RjQx8jxj+G50Y+x/LZy/FWebu2Y8kFyNwC+lKRRR85ATosBLPrY/TIdiMB2Jmy0+HzHad2ZNqn0/CPceIf2ILR0Io/OxSdht+6w6XvRU7fMi/hFuPVBioyQW9zcQ36GK4+XqOHoLepHFV4Hqcnnia0az0Vx3+jRkgkgzvF/PoQf3KVnKD4ICJ6RdS67egXRtNhcgeMRjN520+g1JYRsnMt/dcvIiPVUGNvSm1WnzKZDK8gL7yCHddGR784mrsO3YVcVT+64X9d8SeXW5tyMivLoNK81FFuHwiHmmuvFT97eUFcnHuvOfDWHpQHRmEw2NexwXpdqNX2jl1ypficLYo//4ZX/AEWxV9SgXUd6hXsxQv6F2CyiHxpqddKfZzOGgJ/E3//C8g7CAfupU9wDrG5d9LHtwV0lFfBjpd3sO3Gz9CqqhB/hx6Cg/fXsGctqEO35cyuMwFYlbgKs9mM2Wzm59M/A1gs3GqDUmntULFtFJKIvwpDhUV5WRWFlwvJOZ1DeW4Laq+vI0I6htD5ms54+HlYJ8y1dModzjjMqjOr2Jdmv3j8Zuw3/HbPb5bfO07pyJT3p1BSKia+jkibfWn70Bg0RPpG0jmkCTJQtk6GnbNc3963PfR5nYiEOMCx4g/gsyOfcd+e+yzEki0mvz+ZZ8ufRe3TQDKYRkb6/nROrziN0SxDb9KjV4qiRIRv7RNYR4gPiqetf1tMMj35PnssxF9mJlwO+4TlPqP5+fRy/vlPeO01mDlTfD8zMiCrUuRUF6tPEEWSY3G3ct/ZDgz9YiiJuYmEeocSVyKUwf7+iEJN/AKIGFun9+c2Mv+E9N/x8QGVQXzXJNK9NUCaFJk9xPjo0FLXAbqECvI8MS/R4fNRURAfD0OHwvHKIte0TtOI9I2stq2t4q8+xF+2/CgmDOTkQHqxVfFnR/xl74Ckr6zdvY0NmRwm7GBrgch98ZILYqfcUE7q7lQ2PbqJvHPNf71oCjXsfG0nyduTLYo/NYIBcddi0xkcKf5qIv7AuviqC/FXoi1hbeJa0iO+BKzNCdW6anP3wIG7oLBxczOc4sp6OPOOOCcbxV9FBXiHeBPRM6LZrT7lSjkD7hlAxykd0WrNGBRCahHgUX+buRldZvDltV9yvz6ZSUeKGGn+Z72P6QySzXBSQRIhIWK+mpYmxsEi5Vm29GzPLettVD4XPoFjzzba+dQL3lHQ5WEI6lXtKbVavDfbxfiYF8ew4M8FzX4tHf/2OBse3IBM4biIuj/lKGeinyEveH31J9tcBUO/EcRMQyJlOVz8uvbtIidA3E0N+9pNgaQvhOOFA4T7hDO36y3EZd9PabnB0llellPGmVVnyE+qgchqZIR0DqH9uPZ4hYgFnlTw9XbEqxhKIGQQeEU1zIubjIKg07p4fw7uCzOzoPPDzrfRZMGx5yBLBGRLzTCKCrEGlzr3XYFMJuPYN8c4/t1xl/dpSdCX61nku4grX4pQpJoKc1UbFbNUYr1akzDol7O/sOPyDi4XNn7W9qDoQaQ+ksor415B7k6+ePJP8OcEKDkvfo++GgZ9Aj6uZ7EObysI8MTcRHLKcqo936ZvG/rf1R/fiNYhZS+8XMifz/9J6t5UlxuY6wRdEZj0gBmKTkFAD2EjrfAU1u+9/mW/fU2fa0UGww/O4MEuuRyedpio/g00Bv0Nh5DmNa5afUL9iL/Y4bHcc+wegju4vhA58OMFYg+sIubiDnpOiSYvuicVJYYaX196zhnxV1FQIey3Sxw7aoV0CiGqfxRyRcMQfy1VxeUIDUn8gTVqoirxV5P7zA03wMiRsGCBIA+rYt97+/hx2o+UZVfPMFepwM/XDGZztZy/qoSw2Wzm5LKTZJ/KRm/Uk1UqimuNYfUJ0CFYdGfbEn8ymQy5Ut7iSeK/ib//MrTIwOLSS3DhE2L8hdRfYu43PLKBb8a6bgHSmOi9oDcd7ptI97SP+IfmCvcPqiT7cnZCTh38oTU5sGk4HHvG7V2ndJiCh8KDC/kXOJVzih2Xd3A86zjeKm/m957v8nEc5fx5Kj0tGXfOOiknvjGRJ3OfbJXdcM5gNptdnjB3DO4IwPn883aPj3phFIMeqG5NK5E2jiYmf14SvuZj48Y24XfT/ddp19ZAtN9ZO+WJLU7nnmZvzl72pFXP9/AK8mpVwd2zl8/mec3z6HQytErhkamQKQj2qpuERiaTMSZuDAB5/lvtrD7TQr4jySAW2nK58D2/7TaIrXQOlhSWdSX+wsLAS2e1If5/9s47PKpq7eK/6em9NxJ6Cb33pqA0QRR77/0q6rXfa/fae9drRUERREUQRaT33kIgkJDe+5RM+f7YU5OZZJJMMpPvup4nD2TmzJmd5Mw+e7/rXWuNSx7Hvlv2QZXYJLtaLHco9t4Le+4hMBCUZuKvtL6Udf9cx+vdXvdpJXH5qXIyluxGWVeBUSWKe+5eF32iBLGfUerc0kYuh9dfh0cegcMlIm9xYMxAp8e21+qzd2RvVlyyghdTMpAgo7QUcivFal3VEO+o4Mn6HHZcD8ZOVOzGTKReJubZudH/4KkpTzEwbSB9F/Tlus3XET8svvPG4gJ1JXWsf2Q9p9aesin+TOIX56kFs4X40+nEF7Rsq5icLP6Nb8OvqKC2gPlL57NGchcmjJw1O37n12dzKvYV9teaU9jjzoEpv0LU2Na/iSeQ9TnsWwxGPRNSJjC/5jcGZr+HWi0yWq/58xpSxrfCfr2DUVlfh0ki2IFW2Zm1gMREUBhDCAvtuLVD9/DuyCQy6hvqiekhiskHDoiCi1p5FrXqDDk1Z2wvGP4WTFndYePxNDLLMrnj1zvY7Pc64Kj48xWc8+I5XLvxWmQK56GyW/L/4FT8CxxTfdp5g5q6Bs7f3/JxAx6BEW91+HA8jlOfwrGXnD7lr/Dnq4Vf0D/3FaQmhXVNXHy4mGULl3FyzUmnr+sMDLthGJf/cjmqYHETUqshJ+oTVmsfZU/+HseDleEwbS0MaP0+1Clyf4DfRkPuyta9rrm9T2g6zNwFvW4HRJEyJAT8deJGd6bqTKve6rpN13H95utbN752QiKRkJyc3O49nl6jJ2VCCqYQ0TzSXMHWvjG1cHEhY2OnA2LePl56vEk2okavse5Hz+/le83XViTNgxFvQ3Cvps/ZZ8w1g8iASAZECwX05pz253t5G5WnK9n07CYK9hR0rNVn9FiYewLSrhLW/7MPif+DY3zI3sVw8qPmz+Ufj77n7Vw49X223dA17Hp9Fe7ML20hedpC/DXUN1BTUGMdl7tY/9h6frt6Cfm9JhMwfihj7xpB3cwLMSj8ms35a8nqM+v3LN4f9D6ZqzOdPm8ymdDWaDHo2pdlZvm9+uL60RU6ivizxNhYrD5dKf5ANCQ9+CDMduEQXnm6ktN/nEYV0nRTveWlLfRe9iyqurImOX+Nr4v60nqWX7qcnW/tpLS+FD+5Hwqpwhpl4GlYFX92Vp8A+XvyUeaKeBhfJYntrwtvcEZ/E38ehtQZpe5tJM6BC4vJZT4lIetYl/cdGr0GdZmamnzP+OK3F73O70XM/HFIkBHlF28lxzh/L5y3p/kXO4MqUvifuwiPbw7BqmDO7XEuIOw+39jxBgBXD7rabfUJOCf+wNHu0xkkUgn5+fDOOzZFUlfFiqtW8HzI81TXV6MziMpqSwvmXpFm4q/McSEx8vaRDLxcFOqPfHeE94e8T+6OXGumgjOrzz/PiNyVqamdpLaaugYmft+615hMBG6bzHPnnINcquPMmaaHDIkbAsCh4kNNntNr9eTuyKU0o2vkJYBQa2i1oFOICzwmMKZ1XamNYCH+ikNXk5snlAX7CvdSEbQVuUTBVYOvcji+m7l5NSdH+JRbFjFtUfzFVl5AoCGR+8fez4ZrNpAUkuTYDVX4O6zsBmdXtO2Hay2GvAAj3nIg/srV5SgCFATGBrZ7Ad6RyNuZx+HnfyGguhCDwqz483NvzrUoegtqC6jSOA9asayxDhcL4s9SnGiMuDjb/B3TBiGHTCpjft/59I5LQoKE0lLIqxKr9RBJvGPnXb/FMPkXkAU6P1lHwGQkyE+QabPC7ufxyY+T3i2d0KRQUsan4BfqJfN5O4Qmh3LT7psYcdsIq+LPQvx5SvGnUDh+5qXSljt2L74YbrwRzm9D7S4tLA25VI7OVI9GmUtOjnj8rGEXx5Lv54PjT4oH/OMh4TzPq4jcxcAnYMYOQEJMYAx9FecSoh7kUxup0uOlvN33bfZ8tIfyOvF5l5hk7luauQHLfaK5jXV7oZAprBmjupAMQkJEtt/OnaBROsnJCO0L0eOcnco3sHYs7LDZWpery/lgzwfsUq9AIpE4dOEWHynmz3/9SfGRYi8M1IawbmF0m+ha0ZJRtQ+AXsGNHD+qT8C6iZDbARnL/vGg6BqKmDZhzKcis84FZDKbxa1lfRaTHsOFX19Iz5k9Xb6us6FWQ37Et6yuec7p+tyjiJ0uiN5IN2Mscle1fG3K/SFyBChtG6jYWAjSCAeFoyVHWzXEsNQw5H6d24golUqJjIxsd/3FP8KfK9dciWyCaLhxh/i7edjNxAbFWov4m2s/J/3ddAa/P5iDRTbl4/rT61Hr1SQEJ7hsOPMJhA+G3nc4zj3l++Dnfu4pkM2YmCLsPjdmb2zynEFn4O2+b/PTLT+1d7SdgsTRidxx/A7SL0unRis2dx2i+GsJeathVU84/irkt9z8Ix/1DsguYu2Fazn87eFOGOD/T7gzv1jWxh2t+Mv4KYNXE17lyHdH3H8R0G1yNwLGDaYsaRCpYwSDlGAWgTZH/LVk9RndL5opT04hbnBT5xyAPR/u4YWQFzjz15lWjbcxLL9XX9qDtARPE3+WZtPWKP5awnmvn8fDtQ87vWeHp4Vj6t4DCTRR/DVWgioCFFy09CKGXDuE+OB46h6po+SBknbV9ZrDgJgB3DP6Hv4x5h8Oj6+5Zw26b0VMV1dQ/HmDM+o6MpEuAoPBB4uq8gCQBxAQBLt6zmNHroa7a8+w4IsF3h6ZA1xOktI2XKYSKZx/QGxq2oAL+17Izyd+5rXtr1GtFTPcXaNbl9PgivhLDklmf+F+zladdfo6Xa2OZU9mse1YKP7+8Vzfuc2THkVU/yhSSlMI8Atg2w3bKKsvI1DRfIHbovg7WX4So8no9Mahq9GhLlPjF+ZnJf4akzb1DfXWzdm0tGnO36xki9hEt+Ua8xQkEki7igMZJcgkerKzlaSnOx6SHi0eOFB4AJPJ5NAlUl9azydjPmHkHSOZ9faszhx5m5CzJQeFvwLi49HKBfHXVptPC2b3mo2fzJ/qgP3sLlmJybSArQbRBT+r28VN7BztFX/V1cIRWCJxTh43h5gYiKgbx5wTubz0lHhMrxeFWzDPAfUS8IsGeScROwmCkQgKArlRTEI1uhqm/GsKU/41pXPG0EakTU1jwAtXsv+PWBoUrVP8hfqFEh8UT0FtARllGYxKbKoOtuC92e9xoPAAY5OdK6okEnjoIbHAdqnsKt8Ha4bB9D8hdorTQ+ytJEO7xRKg6U6ELLnRwPuLr87Eb2O5NqqAteSgVot1S2ZmJr169QJzU7cr9UtnQe4nt9oT1Z0wE39G8RlSetDVODrapvgNCmpeHAHC+tWSm9BaKGQKeoT3IKMsg1q/DM6eTcFggAqT8HjuHpEmDjQaRL6Ot9DoerTfdNcW1rLrvV2kTk4lbVqaFwYnYGgwIJVLkUgkaHUmoqvOx8/P6NEOynPOEZu0iRM9dkqn6BPVh8zyTE5WnGDEiGmsXw8bNoBGKZrDHOxyGmrEvaSDNtTthxGwWexbGr2qGkpF97fW9vcpO1HGxqc2EtkrkpgB3iG5TSYTdcV1BEYHIpE6v3bOaAXxNyByiOMT9TlQeQj0TW2S2g1dhXBrCenjeu1gNMC2KwUh1PNGz4+hIxHSvPW+Vq9FGlZCgyaY2lqxMAuMDrQ2/3kLaxevJbx7OKPuEOsLjQbqlWL+TgtrNB+e/lpYJvZ/UOzD2wtVBAx+1v3jD/1b2IImtXDD0qtBUwBBogEhOhqCs0RTVG51LtXaakJU7tlXNNQ3UHK0hNCUUAJjOmfNa79+kcnaf990J+PvsUmPMbPHTGuTclCQiSNJizktew1MYDAZuP7H69l+43bqdHXc9auoHyzou8A3naGaQ1CqmONaYUc/sdtE3t/zvtOcP5lShipE5RMNZu5A4a8gqo/4O/9wyQ9o9B1UUc5eBga1iIZwBokEtKUw/lu33SD0aj1nt52l+4zuHhzo/xZaml9Mps5T/IUkhTDgkgEkj0tu+WA79Di3B1V/9UB7QDjalJ0oI+jPrQQbBpCX5/raaEnxF5MeQ0y667VbzIAYhlw7hMDo9t0LLC4v/8vEX2OrT4virz3EH+DShrX/Rf0Jye6PZkNT4q9xNIYyUMmARbZGaolEQqhf+2MXXCEuKI7Xz3u9yePj/zmetT81QEHXIP68wRn56s7xb3gSBh1UHiE6MA+FPgzAZb6cN6Cp1PBmjzfJWbKBvd0vZY1xMVq9VhQ4cleJzW9bYCH9jAYRlNwKzO0zl6iAKCo1lRhNRs7tfi79o1tXnHV1U79v7H38sOgHZvVyTtJoKjVUfbiUyLP7rZLuroqJD0/kil+vQCFTMCZpDLN7z25x05MalopMIkOtVzvkKJz45QTvDniX03+eZuj1Q7n37L1E9YmyEn+NO5K25GyhwdhAckiytaPeAdUnYN0E2POPdv6UZjTUwvHXoKSpHWeL6HUruWGPozUEOFX8DYgegBQpZeqyJtkSwfHBTH9husNN15fxwxU/8NPNP6HTgdas+IsNbB/xFxsUy71j7gPgUMzD7Mko5EzQNwDcN6EpYW+v+LMsaEJCnHugNwdLHkpdHdSbIzkttlQSidk2MG46nLdbZAF1IkTGXzjh0mQSgrtGvkNQXBD+6T3Qq4JokLVO8Qct231aMCpxFDcNv8lpvp8FgwbBjOb+ZMYG8a+Ljvr8mny+yX+Ko0n3U1IC/+z/IdMOn6KfvNG834Yc2nYjYTa5iGJgUV0hB4oOkFeZR/bGbJ5RPsPu93d3/pgawdBgQFutxag30j28Ow+Nf4he9UK560lv/Gg7J5KW8v08Acs1WqvKoLwciouhXiXWOD2jzIXj7dfAUn8weNHXxqABo576hnoOKD4kK/Y11GqR6bHxqY2c/rON6zIPIXZgLLcfvp1hNw4jQp7E6MzVLKhb49H3CAwUGRltsXVtDeb1nsedI+9kQPQARo4Uj2VkgEbhRPH3Qwz8NbdjB9QezNwBoz+2fmtRRmhN9RgkWofNeNq0NG47fBt95/ft7FFaoanU8ErcKy7VJ/UN9ZSaxP1kRGIjxV/cOXBRBaQs8vzATn0Ca4ZDRTN5aQ2VkP0tlO/y/Pt3NIx6qMtxuTdbuGwh38QnUxD+XZsykDoCJpOJXe/s4uRqm9VobZ0BtVJIt9PCGxF/OUvh6HMg9XD+tskkyLqWMPI9GPVBy8dtvRx+6m09Z2wsKAxhhEnFurE1qr+sP7L4aORHZPzU/BrM09B4oMpXsLeA3x/+HXWOyKVrrmAbogphevfpDI4bTI22hvkb0jgd9xoAt424jTC/MLIqsjhWcoybfrqJrIosuoV24+mpT7d7nB0Ggwa+j4B9Dzo+rgyHC7Kh9+1un2pSt0ksGrCIG4c5b0i4aedNnPviue0ZbadBV6ujpqDG6pbiJ/fDT94BpOWxl5vP742fCQvyoNslENiy1Xr18fdQHxuCYmkm4xb7sEtAF0Bz84tOZ9vGdbTiL2V8Chd9exEhia3LETGZIEs4INKjB2iqNGi27CWwMr9dir8WxzshhQv+ewFxQ1zvtd2BJUPXV8kcZ+hI4k+vx1r7bKsjibpCzd5P9lKW6TozONxcfmkp489X0GduHyImCaGEr14rlnH9nfH3NzoO6lxYnU664g3kRvEprdHWkLcrj33/3UeDusGrw9PWaFEGK6nR15AfsZTNurdRypRQcwI2XgDZ37T95Loq+HUI7Ly1VS+LCoiicHEhmkc1fDH/Cx6f9Hir39qV4m9K6hQW9FtAcqjzjp2guCAKx86nNGWYtbPjfwkKmcK6gba3+5QpZRj1RhrqHa9XV4q/DWc2AHBu93Odk42KEBj4pFhE569pfwFeUwh774O8VW16eWoqKGVqavOPN3nOT+5HtyDBVtnbx4Cwhp3wzwl0m+R++Lo3Me3ZaYz/53i0Wjvir52KP4CHJj5IoCGR2Ko5vLD+HYxSLRGa4Uzq3tQWyUL85eba7BIGJe2Goy9CvXMLXmfw97dlghWb3cosi/igoNYTiR7B4WfhuzCi/LMJ1gzgYb8c/rr2Lwr2FbDlpS1U5/pIFc0JTCaTdVGklQnFX2vslftGigLy8dKmnyGPI3IEXFgEw19z+rS6Qc1r+/9FVtwrnOZPq0d+kwy5NSPgl04m7Qc+wUGlUMR+W7GY4R8NZ3XuaoITgkm/LL1VgfEdhZxNObwQ+gJ7PtxD/+j+PH/O8/SpvBvwnNUnOG6aWsr38wQs16ghTBRGjx6FetUZAHpYFH/hQ0SRR+alnUHme4J4LN6IzqDjB90tHE2+j+o6LeHdw7nt8G2M+ccY74zNCSz5jN7aSLUXNw2/ibdmvcXEbhMZOlTYHIITxZ/RAN0uh7iuUTAFocS2uDY0yMscMlr8Qv2IGRCDMsjDxEgrYDKYGH7rcNKmOlevHio6hEliRNkQQ694J8UriaRjHCNiJsOgZ4TlpyuoIuESLQx71fPv39HI/xV+dG2BHhMoVARaebF1D2UymXg99XW+v7SVdvoegkQi4YGSB5j36TzrY6XafEzSBuQShaMyF2D8N3DePs9eH7VZsCIBjjzX8rFRo4VldEtIuQT6PwRG8eG02JtfJF3C4dsOMzx+uNvDix8Wz9RnplrV+l0J+Xvy2fLCFhpKxabS3YJtsCqY5BCxqRiY/zLvzn6X5YuWc/SOoxhNRlYcX4FcKmfpRUtbtZ7tdOiqxNpD5SSXybKHNrpXL0oKSWLpRUu5dUTrai++iINfH+TVhFfJ3pjd8sHVmZC9VDRt5f4kGhzcxdjPYPxS189LpK1yjtFqSlFoC1lxpB01tL/RIuwJhtasQVtL/JnaUaMqKRG1SJlMOB7FDY7j3HUPUNhjPHl5rl/X2NKxMf566i8+m/JZk7qcp2Eh/iwN1l0Bnib+Ys1lstparDERCkXbG1aLDhbx040/ceLnE06fNxlNaH5eR/SZXS1m/P35xJ+8kvAKFacreGvHW1zw7QV8f9RunVaXIxpK3GlYchOVmkp25u1skvNn+X37qjrUHUeBjsTfxN//AlTRkP4ENYHnIjfYrN8OfnWQVdevQlPhXVo8NDmUW/ffSuUUUWwMkyUIoiYgBcZ8Bglz2n5yZSiE9oPgnq0mdmRSGSq5iqsGX8XEbq33enJF/LWEOrWU3MjBqENiKSz0jiDEU9jy4ha2vrKVg0UHeW3ba/x26je3Xtc7sjcqmYriOlv2S49ze3Bnxp3EpMew/vH1FB0qQqez3VwbL0yemfYM22/YzuJxi52/iX+cyDIqWAsbzofiv9ryI9qdLxHO3Qw9bmr5WCdI7WbgzfMGc0XyhZiMTUPUe4f2BuBA0YF2DdPbGHTFIAZcPACtFhLKL+G8ypXcPsL9TlJXCFGFsFhxkr65L7Cu+HMARkvuckr6RkeLm65eD8fNHNHI+FWw/5+gscsbMpmE/YrOtULaUiQpEU3CTRfK+b/CsVc6xg7MGfzjIGIEAQFi4rDYjuZuy+X3B3+n7ITr7i5vY+MzGzl8ydP4VxcxImAhj058lHHJ7neqXj/0epYvWs7Nw292ecxvp37joz0fcaLM+WLXbdTngtz1irtHRA+uHXIdAPvSLufgKUFyNyGXoidAzJT2jaUNsHSmWtYE9fp6InpGsHDJQnqd36vTx9MYgbGBDLluCNH9bUWojiB4vKX40waLiefIEVA3torrdz9MWtnxg3GFkP6QegWoIglS2i7Y8roa5Co5MQNiCIj0XJZeW1B8pJidb++kKqeKBnPNwZOEsLcQGAj9zeYSFsVfYoiZUJDKYMwn0Pcf3hmcOyj8Q9zvzAtXqURqtWvWNSL+TCYT9aX11BV30r3RCQKiApjz3hyXFpJ7C/YDEFo/lKioRmuJjLehdEfHDCxyJKQ/Kmz2moNM2Xk24p5E2ADofTeEOFd7RgeIiVmnKLHuoSQSCdH9oglN6TgLqZagClYRFGubE4t0Yu6O809B1tieWR4ofk5PIqCbsOR0lv9anwfVZqWdQes+6ZB6KQx+BpRhgG1NG1gymQExA1DI3J9YQxJDmPTopHYrPLyBgZcP5M4Td1IfJdRUrtYZK46t4P7f7mdzzmbrY6su+YWpB0/RLX8xWq2Il4gLimNw3GA2XruR92e/z+gkN7MZvQX/WJi+Hvo/4Pz5sytgZQpUtb+x7uSak2z49wZMRt8vcET3j2bknSMJTglm7jdzuWblNdbc6SbYeTNsvRJ23Q4b50HRn+6/UWh/iJngmUEDmrRrST0hp3JfdwoP/A92kXcSLASDn1/LUQH2aC3xt+7BdXwy9hPqSlq/XjpqFm137y7WyTKljNT+ASCRUFDgvM5oMjW1dGyM2qJaSo6WuMx1rS2qZeU1Kzm0pH35t38Tf+I8FoHDEXPEY2Rk6645e8QOjOXSVZfSZ55z23WJVELFH/uIyD9ijcOwoHGdKzAmkLBuYQREBbA9bzurMlZxpvKM7QUn3oKi9dDgucbzJ/58gtEfj+ajvR9ZH9v/+X5O3PYaAZV5Pqv4s7f69Ab+Jv48DG8ENbYIRTAMehJd5LnIDeJTWq2tZtiNw7js58vwC/cNn/USrdCbRyrNRQ6/aOF1Hj6ofScev1RsajrZV98V8VelqeK7I9/x333/dfq6XIvgyKx+8RWbm7Zg70d7OfjFQTZmb+S+3+7jwz0fuvW6JRcuof7Rei5Jv6TJczmbc9j0zCaKDxVb1X5yuW1hYIFEImF00mjnFq26KlvnYvdrIf3x9udsyf0hejwE92jTyxMSZfyWdSu/nLiNokJH32epVMqY1DGE+YWhM+iavHb769t5NelVqvO6zsWi1UKALpW+XOCxDXG3RD8kSOh7+k3iKhYwObLp9QNiKkg2C24PmHnUQ4YHxaZXXwcaM4tXdRi2XAIHHnP5nhbioLHiz0r85SyDffeDqZO8vHvcANN/Rx6aCtisR/td2I8bd9xIwgjf7cSO7BWJ/8Be6BX+jA5ZwDPTnmkV8TcycSQX9ruQbmGu1a+f7vuUm3++mZXHV7ZvsOvPhZ96QM53wjbYCd6Z9Tbh+gFolYXcXxTH9t7nEti4RjviDRj5TvvG0lpkL2O47mqClOXIDKKIKQ+S+9T6JWZADBd8egGpU1IprivmZPlJavWVgGcz/uwVf51C/EWKTValXBRojxw1WRV/qWGpHT8AdxA7GcZ9BeGDkUvlKCWCJa5Ui8VMXUkd9aXe3YHnbM7h17t+pTSjlOXZ7/Pr0GDWKLtYzpkdqjRV7MrbRYOhwWr3qWqIJ0yaRHJI6/JcvIozX4v7nd0G32L32aAoc9iMGxuMvBT9Er/e/Wtnj9Jt7MsVFZZQ9RBHVwlNMey5C066t6btEGhKhLV8M41JPoug7uLeF+08qyo60Ez8yUsc9kBX/HqF1ywC60rqKNxfiK7OtgYv0QviLzEw1fFgXSVUHGhVLppbkMpgxhbocxdkvAV/TBdKYICak7B2lCAAc5YJ1XZ+6z9bFuKvuLj543wFUqmU7t27t3v9ogxUEtkrEo1RLDBcFWyXH1vOK9teYf3p9dbHokKCCDWKSInGe/6xyWO5YdgN7RqbT8A/HiQy4SLlBkwmExmlGfyU0dRG+diKY/z15F+oy31UlmGHbhO7MeutWZAIP5/4mS8OfIFK7qJqO+5LGL8EBjwCQ/4DYenuvYnBHG/jQShkClRaFYuWLGLPB3s8eu7/JbQ0v7SV4Gm11acJtNVaAqJa33R3+LD4N93ucpSWlxJUW4hWa3M+skd9PVh60F3tjWa/M5sHih9wmY9sMpg48MUBcne476TkDJb6Xp33esRaDXtC2FOwxA5Y/p5ttfkE8I/wF9aYPVw7/ExfejMnRl/RRPHXmBAedecobth2A6pgFXnVTiIKgrqDPEi4VLQVJkdBRI9wUWs9WW6zXlcFq1BFBSMx+a7iz97q0xs1F9+p8vw/gS+HNgcG4kD8xQ6Mpffs3ij8vdsmfeavM+x4awcVleLGEKVMbOEVrYSX/iauiL/iumIWfb+Iu369y6l0/+xZSDn0M0N/fQ6J0dCl7T6v23Qdl/18GWX1YlVhKQC1BHt7KHscXnoYbZWWm/fcTM/zelq7UEJDHf/MBmMLJMuRZ+CHOJEfGdwTBj3lvIO2NWioFXmabYRMBvu19/Fz5l2cyXH8TEokEh6Z+gjlD5bzxOQnmrxWGawkOD4YTaWPtriYoanS8GbPN9nw5IYO6XpJSgIJMmLK5zPi1A90S3S94rLYfWaa3WSDwoIEIfz7JGHVAkJ1nDAHMt+F/LVOz9O4SNIkDHvQ0zBjW7PqsI6AhWBaoprEgHcHUB1YTeKoRFQhvuuHl35pOkE3XUqDf0iH2SAcLhYr5vQYNzfkrpB6OUSMgM2LBPnnBAGKAC6V2p4rDfm9U+wkW0TFfhK0XxKiKkXaIK7LBhrQa/T8dMtP7P1kr5cH6Ig3d7xJr7d6cSBSWG57kvizV/x1xt9mUOwgll60lGf6i2JYVnEhRqkGiUlKSqg5t2XPPyDTjWymToK/TFwjVRqxmHk18VVW3dA2S2tPoc+8Plzz5zUkjEigSleJQVYL0s4PSvcETCYTya8lM+rjUZwsP2kl/kad/JnXU84yOG6weKA6A7ZdA4XrXZ/M2+j3AJy7FeS24lRkgJn4k1U4KP5kShmj7hpFjxlta5byBA4tOcTyy5e7tMC+f8AbTDuYxRDdnY7W3YpQmL4B+tzdMQNrqIF1E+HQk66PyV8N68ZD0YaOGYMXYW/16SvFvhM/n+CDoR9w5s8z1sfKTYL4SwlpZBVb+LuImXCxNvAIak9D+V7QCDcBDGrwTwBdOahiIHE2BDq3sG2C7TfApoWAbU1bXl/Jsxte5J5f72nVsFbftZoPh3ceIS6RSAgJCWl3/aUmv4aagho0arEvd7UG3Za7DYAxSTa7a4mkbZldPoXsZXD4GdC7aOqJGgPzskS2qRs4UXaCvu/0ZdH3i9DqHfOKJ/xzArcdug1VqO/uRxqjTC3qGKGqUOSN7XsttZyAJEi5WDQA93+weatmexT/Bd+FwMmPWz7WTajUucyP1rD+whWkX93O/c7/MFqaX9pK8NjPF+44e814eQa3Hb6tTfOcM+LvuwXf0OOgsNp2lvNnmcf8/Nq+5wqKC+Lhmoc573U3LKebgYX481Uyxxk6wtLRkvNnUfxFtCOVw52Yr8QBYZhkilZl/OVWN4ooAOh1G5yzQaxZ9i5uQuK1CF0lfCODJRIoFkr7HhFi33Cqwmb12e/Cfoz5+EbqwhO7hOLPG5zR38Sfh2Ew+GjhYdNCuhXditxotvrUerazqD049sMx1ty9huq6AgBi/M2TxcEn4Mc00b3YXhx4VGyiOxGuNgEW26a6hjqqtFVNXpebC9qACGqiuiM1NHRp4i8oLojQ5FDrgjkqoB3tKcC2V7ax6blNxA+Lxz/C3/q7DbVz/qnR1pDyego3rrrRtR1HUE+IHAX2XboGLRx/zdY921ocfhqW+okMjjYi1TycM2ccHzcYDGQczcDoxAIUYNgNw7hp103EDGgnednBMOgMqIJVyBQytFrIjfyCTNU3lKvLW36xG0hoJGaLb2bPlWKusRuNEKioJDnkGMRNE1knsVPEk8pQ6LcYIkeLTmsnsCzELAvnJguigCSxYe6sG3ztGTjwOBHGrQCUyQ5ztOQoFeoKGuoFuePLsCzW8kx7yCzLpMHQuuyA1Zmr+c/m/1BQU9DkOZ1BR0aZUFq1m/gb+C+Y8B2M/gTSrnR5WL+ofgw5/TmYpCSXXudILpmMsOsOOP1V+8bSWqQ/xt6edeTX9EKqF2uCs8VnMZqM7P1wL2fWn+nc8ThB9sZsVly1gsL9hdZ5XKoXbHZXJv6CVcEsGrCIyX0FmaPURzLh6G6u9ftRWKqZjMIWpcB5o0GnQF0EO24UhUAgUG5uGDMTf6PuHEWv2d61gw2ODyZ1Sir+4f5UaysBCJSFeXVMbYVEIqF3pLDyPlF2gsRE233FQSFcewZOf9GuNUaHI7SfUHFJbc1LSy9cys8TthJbcUGTzfj5b57P0OuHdvIgbSjYV8Dhbw6Di9tzRYWEAF0ayaFJjk/IVEIZGz64YwYmCxAKLm0za6OIETD0JYhwP4PNp7B3MWy62OlT9lafFtcCgKPfH+WPR/7ojNE1QfzQeKY8NYWYgbZ1du+Cx5h28DSLRzTKgQ/pK5xEIjswC3XI83BROQSYF74J58GsQxA2EBJmCrvoUOdWqk2gLhD25SYTAQG2e+Fjf/2TN3e+SaWm0u1hmQwmTCYTBl3n1EMMBgOHDh1qd/1l9Z2reS3ptWYLtsV1xWRViPl3VOIoh+e6PvH3LRz6t8Pc3QQy8+LLDaaid2RvogKi0Og1TbLpw7uHE5Meg0zhfF/lS9jz4R6WLVxGYb4oxlgaWawwaOC3sZD1RdMX69VQ74RVaQxVJKRdLT67HoJ/2Q6+SjLgP+YAMcN9uzbgy2hpfmmv4s9gcN/Csi1EQWUl5OWJEkR/O2Or8f8cj2KSUNw3R/y5yvcDOPTNIU7/edr1eKUSlEHKdhMclmiKrmj1aRm7J2DZF1jcztqj+Pt41Md8NPKjZo8JlGkIqMxHXaWzRm1A02tj/WPrOfj1QUwmE3k1jSIK7HHyfTj+auujlewjeBD3Hovi71T5KQcRjS9n/JlMjsSfNzijv4m//xXUZqEynCW59FqG5HzEjB4z2f3Bbp4NeJacLTleHdr4B8Zz9fqrKQoQXQJxAebJQhECygjRWdteaIpFV2Rn5WxhU/zYb1pBqEAsBFh2ZdOw6LNnoajHOM6MuwyDwq9LE3/lJ8upL6u3En9NFswuoG5Qc+UPV5L+bjrqBtvsPeudWUx7Zhp6rSAvqqrAhInV4bMY8O4Atp3dxsrjK8mvyWdj9kYCFC4sEXrdAlN/dSRjMl6HvfdB1idt+lkJGwgpF4Gfmx1+TpCaCneNupEpmqaFHPsbRHtCnr2JwOhAbtl3CxMfmYhWC0eTFvOD9HJrh1B7kdSoNmdZJDmDhfgDGJGwmnN1/SHvF1FMCUsXNlrackECztzussvVohzMNn+Umyj+tOXtUoK2GpoiOPIMIZotAMjMGW5nt5zlucDn2P3+7s4bSyux/7P91C1fA0Y9D58aQ++3e1NY27oJ8OE/HuahPx5ib0FT1VpmWSZ6o55gZbBn7PNkKuhxPQS6thaNjoaksquZfjCbgdkfOJJL+nqhJi3q5EKmPABlQAAgAa24PmobapEpZTxQ+gDzPp3XueNxgrITZRz86iC1RbXUNYj7tszoeeIvLEyoraFzrD4tSEkBE0akJiVh9cMZE2HJMpbAwlIY5UULQVMDnPoESkXzQKDcsWFs5qszGX6zd8kGQ4PBmg9U2yB2wYEy72V/tRcW4i+jLAOJBC64QGwMhwyxOyh+BiyqbbbRwCdg0Dnc8+KD4wnzC0SCFL1eFLp8BTNemsEj9Y8QHG/78GdkwO23w8qVNgusyMZLV01x67uWWwOpDC4sEHaYrhA2QOSBBqa4PsaXUX0cKvY7JREsij+dvMRB8Xd85XE2P7/ZrW51TyNuSByTH59MWLcwQAxbp5EToEulZ3Sjv0FYunASCenABgmZqmlDmUWJVHGwdXk6U36GmTus54uJAYUhjBg/sR8/WnLU7VPNfnc2t+y9BZmy80gdTxTQes/tzdjFY9Foxe/AmRvJjlyR6dk/uj9hfmEOz1nWD12W+BvzCZy3t3niD+DIC7Ay2bUy0AyJRGK1Nj9d6UgOmEwm1OVqtDVaZy/1KRQdKuLYimOU14smjCbORdUnoO4MaBt54zZUw4p42PuPlt8kYjiM/RyiPJgDGX8OF+TDPi00GDt/vvz/hObml7YSPEqljaRoac5Yc+8adrzZtjxhizqsWzfH5sZhNw4jcfYQwC5iyA4t5fsB/HTTT2x5YUuz7194oJCCvaIRV1OpYd+n+zi79ay7wwdsDXBdkfhrfB/R1eow6tu2dmxc02oP8ZcyMYVuk13XLgCOL9lL/80fEVBd6KD6s69zGQ1GNj23ieMrjlOhqUCjFz94QrC5Ien018KppD5XNEyfswlip7ZusCG94XKT+IoRIp608DQkSKjR1VBaXwqAtkZLzjdbCCnO9EnFn15vW+7+nfH3NzoW5+/DMPEXomqmk1R8I73D0glOEB3TyiAPVtHagJCkENKmplFhFAXe5HAzcdLvfjh/Dyg80IY/6kOYe0KErXcSmtsEpIUJ+5XGi2EQxB8ISX5E7kFOf/aXQ6ZEV4FRb+StXm+x+o7V1knZXatPP7kf67LWcaTkCAeKDlgfVwWr+PG6H1n/mLC6snS9jPa7iqyKLMZ9Oo6H/3gYgCsGXtG6LqM+98Lwt6B7G7MY0q6ECctE1l8bkZYGBqOcGnWgU7LoqY1P0ePNHnx+4HOHx7XVWra+spWTa082eY2vQq3Vo5OLqlpsYGzTAzTFsHY0VB1z+5yhoY4KicYKQHt0s1vvnK3uR1HEYog258mZjEJx/EMsVB5u9j0tKs3CQrHQa6L4+3WIsOPqLIQNgtlHkPS+GQC5mfjTh+sZdOUgIvu0w2O9g5G5OhPj1h3oZfUYTILcD/cPb9U5LIUGi7LPHvY2n+3qQMxeBhtm267NhlqX+RwWRZl/QxJSk8KR+JMHwIVFMOSlto+lLdBVEm7cSYiqBJNWDEitVyORSAiIDECuch7U3pkYev1QHql/hO7Tu1uJP3kHEH9Sqa2o31k2rCaTie9OfsrGQf1QK846vrdEAspw8GufOr5d8E+Aiypg6CtibEoLOew7ThG///N3npI9RdXZKmoaKgEIVoR5dUztgWXeOlEm8kIbeqxgz8Q0Xsu4w3aQRCLWsDLfyOV2irMrYKkKzi53eFiptJE79nafm/+zmW8v+LazRucUCn+FQ0bNzp1iHf7S0s3ctn4R2VEfNLVUWn8O/PK3fVq7MPknmJfp1A0hOTSZWQlXk1R2tUPz5PTnpnNX5l0+cY9qaLCR2B1lTd4mlO6EXwcLNwF30ShawWL3magcALSO+OuqGHrdUKY9fy4NZo7EWSHfavOZ2FTJaVnzN4736DJQhkP4oJaPkyrBLxbULSvZLPblZ6scC/05m3J4MfJF9n26r01D7UzMemsWTxieoFJZCThpYA4fBHMzofddjo8rQiD1iqZFbn2dsK7rYCiCe7KqDi575R7W3LSmw9/vfxWNFX9Fh4pYdtEyPpv8GZtf2Nzsa91RCZuMJg5+cZCMVU33tO7AQvwNdCImtTRL5zkxVmvSxOwEi75fxKQnJjl9TmfQYTKZWHbhMn68/kdAZHuuumEVh79tvq7SGBarz65I/NmvDUxGE2/2eJNPxn3Spgb+xi5WTRrSWoHZ785mxsszmj0mdWoqlcOmovMLthJ/Oh1W9V9wsFB1/uPMP5j56kxrvl+kfyR+cvMPXrYDTn8pMv4UIRAzoe2DtoOf3M+aI2ix+zQ2GDn4yu+EFxzzSeLPfkx/E39/o8Ph54c1o6KuDvrM7cMVq68gbnAzsphOQH1ZPSajiQmn13Lu/kIW9LvA82/iBR9dy81SoxEsvz26h4sQcItliAU6nS0rbORIiMrdj2bDDkyGrqfwMhlNTHh4An0u6GPL+HNT8SeRSBgeLxQFe/JtodTKYCWT/z2ZPnNFkayqCiRIGBUxi4X9RD6FRWZ+xaArnJ/896mw20lmhUwJfe60WTqWd/6GJDUV3t39Pot/3YhW37S6XaWtIqsii/2F+x0eNzQYWHf/Oo5+79ub88ozlWx/fTslR0soqSsBiQkJUucWsCVboWynWDS4CYnERvYFBjZfyI+MtC0msyqGUtf7ZaHc0tfDyhTI+hS6XQKh/SHjbTjxjtPzhIaKL5NJFAub2GOkLILkC93+GdoNuT+E9icgNBSJxKb4a4htYMGXC+h1vnct+prDvI/nob7tXhoUlQDIpXICFa1r1ugbJaytjpceb/Kcx/L96k6LDB+pAoo3wfdhkPW500Mbd+Q5WPdJpCJbtLNJnqINJB4ezbD4tfjXprN47GLOTzofgNLjpRQdKurc8TiBRCpB4a9AKpdarT5lhkDkcvB0HnbPnuLfZA+IQN3FFwe+oEZ5gg3pfcmKfZUqhfl6bagRag1dUxvwToNECsow673w3oHPMfrEWiJrxQb/twd+4+fbfvbe+IDYwbH0v6g/qhAVdQbxuwryhDuEl2Cv+AM4U3mGM5VnrG4JgLD6LNslrMV8FUHdodulgjw246/sv3jxyDNkx7wHOBJ/RQeKyPojq9NsARvj8LeHqcpx/KxZNucVgdvICf6OspA/mxZY4s8T64OORNVROPKca8eAtWNEJ3VXhZMcbwtiAmP4z9jP6Zv3nIPiLzQllIieEQ5EbWfhu0Xf8f0l31u/r6rVsTftCo4nPIZUYXdRm4zwywA48FinjxGAyBHQ5x5InOv+aww6OPUpnF0JQKy5Fy/aJIi/I8VH3D5VfVk9O97aQfampo42vo6WCnPbc7cDjvl+FnRpxZ9BIxrZ3Lm39L1XNGUH92zxUIuzRk6Vo7NUePdwht44lOj+0c5e5nOQSCSUa1wo/gAUwUKB2xgj3xH5VvY49V/4PlzkVR03K7r33Af7HvDomGVSGcfuOEaffn0Iiw3z6Ln/hg0WS0FLo8DxFcc5tvwY2RuzydvRfFSRO8SfRCrhvvz7mP/5/DaNz5LvN2CA4+MHvjjAsbvfw6+mpFnFX3NWnz3P60nK+KaOAwU1BaS+nsq8b+cx8bGJjH9wPEaDkTV3CwJ63P3jWvUz2BN/XcXwqvF1AUIdJ1VIiejRtnC+2EY98u0h/txBwvAEZFMnoQsIp9zsOm+5VmUy8XeRSCSEpoQSmhJKhaYCP7kf8cF2DOWIN2FhidhTgnDByvkeqprWaJzC2CDWubmrIOMtyPzA+pQl5+9kuRA9+IX5ccFPN5DfZ6pPWn1a9j5yufjyBrzfMvf/DFJPV6Q8hZKtSGqzkIZMp8h4iL9OhbAgogOzB9yEyWTi1YRXSTunB3XSy1ARS6LFivzI8yKYPPXS9r+RsQGyl4qOtsTZ7T+fBSYTGHVOF3yBgYKIMJnERGnfMeyK+MvLE8cHB0OfPvDRqCuJV5SgCuk6AdgWyJQypj83HYCyN8zEn5uKP4Dh8cP59eSv7CmwEX8hiSFM+dcU6/eVleLf+PBQvrrwK2b1msXi3xYzJXUKPSOcbEr0atBVQEMLRdWMN2HPPTDxB0he0PJgTUbYvEhYcfW8ueXjXSAsTJBIVVWQkwO9zByNVCqlT58+DGEIgIMKEsA/wp/rNl9HeFrr1FGdjeLDxay9dy0XfHYBJeau2mBpNDJn+Xmh/WDQ0xAxslXvkZQEmZmiM6o5vl8iEXZ7x81rj3DLr04eIDo0g3vDQHNuS6aZ9OvtvIM6NRUOHBDZjE2Iv2Evt2r8HoGmBIlJT0BAvFXxV6Pz/TZkVYgKtUJFg0wooSP8I1qtzGuW+CsRO6AB0QOaPNcq9P8n9HsAkIhsjsQ5EJTq9NDGxJ+j1Wcd1GWDf6LIk+wshA2kvvd/yPplKKqaAbx4zktotRqkUilLFyxFIpNw++HbO288TlCVU0VNfg2xg2IdrD4VLbhQtQX33gtXXOFo/9uRkEgkPD/9ecZ9Og6DrJ6jyYvJNUYDfaF0O/w5A0a+LyypvYXKI2DSQ/hgxqeMZ1k1SMw7hpyNOajLvburGnLNEIZcMwSAOkMlACGd+RnyMPpEOSr+LA1Mlo5WADLfg2MvwtyTENyj08foFsIHw/hvHB46UX6C77O/Jz5cR2rJ7Q7F9QVfLkAq886+qfJMJcsvW86wm4Yx90MbSWIpGKhSDoMJgtQDmtiIM/TFjh/g2R/g4OMQNc6WO2yBySQaTyRdeBuvLYfCdRCaLmxLG8Fyr7RX/Om1eupL6vEL90MZ2LluNepytQPheKo0h/zIJciMAfgpnrYdqKsQfxdv2etJpDD89Va+Rga774KosZA83+pUoKoeABI4UuI+8aet1rLm7jWMuXcM3SY2byXmCVj2R+2tv3x13ldEj+gGTEQioclaw2QyWQmsscljm7y+Syv+Kg7Ab2Ng8PMw4KHmj23Fmtyq+Kt2VPyFJIUw7yPvW8q7g7ydeeg1eqoNYnPnUMfY9yDU58H4r5s/SX2eKHzLA8VeIWocVB0Gg/lmk/+LIA89OvCf6bv1+GmgQgABAABJREFUCvp++zGkNK/s+Ruu0dL80ljZNfmJyfRb2I+w1LAW71Hu5oLKVXJCEpth4FygtlbUJaAp8Wc0GJEa9UgNDRQVCeGBvZtKSxl/JpPJ5f78yb+epKC2gJ9P/MyPT/yIVCLFoDMw+/3ZKIOUhKa0bq1uIc/0eqG096TrS0fBok60kJYAMoWM+3Lva/M5w8PFz25R3LXV6jPjpwwyVmUw8ZGJLdYNLbVrS73VXgkqkQh7TXWZmqD4ICZ1m4T6UTVafSMLZ5XdnFlzAjZfLO41oS3ca0BYwp/+AgJSRA6t3N+6N7552M3M7zOfkQmiTiiRSug2PomGD0CvEctkL+h+XMI+3w+8wxl14R3D32gVMl6HnO+oifqUHWHX8/LumUwM+Yp9n+yj/0X9SRzlJISzE2DQGRhy/RD8U2Jgq1hoBwUhPq2H/g1x53qG+EMKO2+GmEmeJf52XC+sjRbkNbERlUjExFhdLTYC9sSfxeqzMfFnsflMSjJ7OUulFBpj0elAoXB9k/V1fHfxdxTXFdM/un/LB5sxPMGs+LMj/hqjuhqyoz5gp9aPKXWzuHzg5Vw+8HLXEnq5P8zaD8YWusuTF0LJZoh206JRWwa5K0DV/u7F1FQTKeq3MByTQq87rY8rlUoGxQgblgOFBxwWXRKJxGnXla8heXwy122+jogeEZR+L3zhQ2VObD4BQvpA2jUi/yywm9uWv5YCXXM2nxZ06wa6on38c/wiQqqeg/CLxRPjvnQ8cOJyYVHQzHkOHBA5f+4EYnc4VnWHqHEEBq5FbhSbyeqqan68/keSxiR5PZ/LFcpPlaMvkaCTC0+JcL/WE9nNWX1+OOdD7h51t7VLrF2wqBWU4TBppcvDohtNCQ7EX/ke+H0yDH8D+tzd/jG5i+AeSAY8SI75WhUbKbGTGnPfGJ+4z+z7dB9/PfkXtx2+zar4kxuDOmTD5+fXeaSfBWOTxzIqdB47q1YB0D1CrAkITIWBT3o266Ut2DhPzHnn77MWNCwb2Ru23+AT14gFcZKBVNRKrHlUPofqTKjPgbjpLg/pFSG6fIrriimoKeDXk78C0C3UrnCeOBtUEcJmrQvBUijVK0QDmL3iz1ukH4B/pD8XLrmwSfe1ZXw1fkdADTfMTWeMN/ok066C6ImCjGkMiQTO3dT5Y/Ik6s7AlktFg5cT4k/up0GtKAF1BCZTIBIJHFl6hJXXrOTSHy+lz7w+nTrcq3+/2uH7zFLRoBTUkOo4H6oiYZZjc57PQyoTOX9Boil12DBxiZUf7w/9Wkf8hSaHcvUfVxPVr/OcDJTtXBjotXryduQhMVsy+Pk1LRhKJBIy78rkTOUZK6FlD8ua3xJB0aWgioL+D0PMZPeOL9og7NuGv9bs3ig51Lniryth3QPrKDlawlMlT/HIxEfQG+0snMr3Qu2p5k9QuhP+mCJ+vwMfF42CiXMcq9JzjotGQE9CGQGRo8W/f6NdaG5+sTQK2Vs6xgyIcX5wI7hD/GX+mklwQnCb3NmOHhWXWWKiaCy3x9DrhjLk2qFsuwxMdZCfb4susR+Tq1pG7rZcvpj+BTNfm8mIW0dYHzeajPyS+Yv1++K6YuKC4pApZQy6QtSw1BVqlEFKZAr3cmDtVXNqddcg/pwp/uxRV1LHsR+OMeKWEc4PcAKJRNSGc3LE/xv/Td1F7rZc9n28j/EPtlzjlHz9Fd1qgikvF258jbMfT645yfeLvmfhtwtJv0S4KankZmarfB/UZglRhKWxIXQAjP3S+brWGcIGwsIyMBmEy4bSVhe6bOBlTQ5XSPXItVr0qkB0Ou9ZajpDY+LPG/BReVrXhdHYgWHv7UHfxTBpFWEB4oNXpammKqeKrS9tpWBfgdeGJVfJmfPeHMLnDmBv2hVkdL8Ho8lMysw6BMNe88wbSWWiODv8Tc+cz4KocUI95iIHzDIxNu4AnNFjBj8s+oFXZrzi8LhFbp+UJG62/v4g1ap5d8A7rL1vrWfH3sEoP1XOlzO+5PDSw4xIGMGsXrMI9XO/y8di9Xmk+AjqBufqgqoqOJHwb14+dS2Z5ZnWx1ssSjpTmNkjIFHk9fm5t3jDLxou0cLQ9md1paZKOL/neyRU2659o9HIoUOH6BvZF7lUToWmgtxqR28GQ4OB2qLaxqfzKfiH+5MyPoWguCDKtCLTM0zRTBEz5zvYfl2r7D5nzIA5c2DRopaPTUmBIGUFEokUmaKZbMbQ/hDQuOXfBsti2V7xFxyMCDPevEg0B3Qmet8FSRcQFASqhjhi/BJRKpXs/+9+sv/yXful7xd9T/hPX9AgE54SEf6t37BalDPFdcWUq8sdnosOjGZq2lSnRRu3YTIJS6wK9wp7SqXjxsmB+AtIgvTHxea8k2FZeBrRczT/FCu3rMRoNDL8puEMu3FYp4+nMXrM6MG5L51LcHwwiwYs4rIetxOg6enQPdnV8eDwZ63/7xdrVqiH9IKBT0D4EO8MyoL+D4ncW+Bk7X5yoj4mT77JXKvyPum3/fXtrLlX2AYtUHzA+ONbGBDmZbLUGcp2w/Zr4a85YF8wbIRgVTAPjHuAV2e8yr1r7+VoyVFiA2O5JN3OTjJmklAbeyL3uqNgNMDOW4UljxmWBg5Lpq898VdTUMOJX05QW9j5axdVsIqBlw1s0vioVoMJA7laYZ2+YFy6o71w6Q7YfAmUbOvYAQZ2g9jJwob+/yNC+sC4b4Q1rBNcuGoKfwxOoTBgnfWaiRkYw6i7R7VaMdARyCoXxF+IKc3LI/EQYqeKaw5hez1lCgRpRLNmQU0B1Vr3PCylcilp09IIjvewgskFLPuj9tRf5Co5/6z4J2NenA+4zmyUSCSkhac5dSlxV73TGjQ0wK+/YrVZ6zAE94Ahz0G0m8XYigMiDqFsV7OHDY0bysvnvsy/p/y7yXNr71vLr/f82obBdi7GPTCOGa8IxZyf3I8gpd39d/rvMC/LxSvNiBgqCt+RoxzJPft1lETi+ft69DieU0zh0efX89e7f3n23P9DaGl+sSj+ZOXFLJmzhLJMsc7RVGo4uvwoJUdLXJ67pTnDZDKx6vpVrLxmZZvGbrH5THeRbiGR2JqlG9t9NiZ4GkPuLydtWhohyY7M4B9ZfzjUp/58+k/e6f8Omkrxi9r6ylZejHiRwv2Fbv8cUqmNQOsKOX9Go+26sCf+9n+2nze6v0Hu9lzW3ruWX279hdPrT7fq3Jacv4gIYbfZFkx7dhr3F99PePeWm6ulOg0yvc56D2psARvRI4Ix940hJt1JvTTrM9h8Eajt/taKYEi7snWuJaoIUWcNSxf12Wbw3dwvGfDXuwA+l/PXmPjzBmf0N/H3v4Ko0ZA0l/BA8SGv1dWQMj6Fu7PutnZgeBOni0rIj1zCiaAPxYJaIoGQ3qII5inEzxDn9CS6Xyc6EVx057si/rqFdWNBvwX0i+7n8LhF8ZecbOvsMCj8MMnkqIJ9qG3BDWgqNeRuz6Umr22+J0khSUQHRGMwGThYdNDpMfk1+WiVhUglUobEDWn5pPsfgrxfWj7OgrpsUeSvbWFhDyCVe2ThnpYGL2z+njdPNN1QqeQqq5VhY7vPby/4ljdS32hTYHBnwaAzYGgQxH6FTuSIRShdEH977xdK5XFLIHyo2+8RFga33CJUeC2hZ084WDSNV05kiA5MVzA2iHwlFx2ZlvfKysKaRxMSAqgLBHlZk+n0dR2GIc9B79sJDISBOe/w/bhcbhp9Ew9VPdTmnIDOwNAbh1LecxQNFsWff+sVf0HKIBKDxcIwo9Sm+vPY50JdADtugJM2n3lyV8Gfs0RGihPY23E4ZPwFdYdBT3W+uktbhvS3UVw26Dl0imKGfdGLSzZc4lNzR/K4ZMbdPw7/CH/+MeYfLO73DiGadJfdk10RUwekM/zkcoZkfUX3GO9mLTdBz5ugu1C4/JazkoOpN5EfsQSNBkqOlnByzUmvXi8ZP2Zw8EuxLrDY3vhkF3DkCJiwFEa8LTpWKw8L23knePHcF0kITmDpkaVIkPDVhV8RE+hm85GvQCqD059Dvq2ga1H86WTmgpjdZvzMn2f4Zs43nN3qaAXX0TCZTNSXOa8gabVQrzqN1qjGT+5ntea3ouYE5CwTlo4dP1Bz5mej9yrdAcdehvr8jh9DR0EeKBxdXGSFxQYJubxWXmJdV8UPjef8N84nbkjnzpfVedXsem8XpRml1sfOVImCXRipjgdnL4OTH7bsLOKL0JaBRvyMV1wB/pJQJh05wNY5tYSo3LexMBlN1BTU+NSawh3odIKMcUX8NQdX+/32YMkSePdd+Owzz53TI0i7EuafbVbFDkLxt3jcYub0brq/yv4rm6x1buytvYzec3oz+OrBrg9osdFYIRrPo0bDigQ4+C/xuK4KDj0Fx1+D/LXis+dhvLb9NWq/rGX/+/s9fu6/IWBZz5jyC8n8JZPS42L+LD9ZzncXfceR71yrpVtsFjDB3I/nMvWpqW0a2xHzWzsj/hrqG9j78V7i1OI+5or4c6X4ix8az+W/XE7v2Y511WptNckhyShlSj6a+xEhcnGC9we/z+fTPidheAJDrh2CIqB1uQ32OX++Dvt8OQerT6UMv1A/VCEqZrw8g+s2X0fMwNat8ePMS5+22nyCaGAJjA50y3Fj5Ac3kjX8YgrN3F1jJWj8sHhmvjKTmAExvLjlReZ9M49VGcLJhj53w5jPnK/xDNqWAxtNJpEHWG/OyjQ2iLqsuRan0WvYdnYba06usb6k74K+VKUMFM/7OPHnDfxN/P2PIcr8Sa3VV6MIUBCeFo4yyHvVkszVmSy/fDmnD4gbT6DEPJPp60Fd1GyHdJvQUOM5OwWDDsp3i2IOQEPTjuXWbgTsFX9gnuAlEnq9fEubb/zeQsLwBB6ufpjuN3fn1W2vsuzIsla9XiKRMDppNOkx6dTqnHeD5+j3AtArrB8BihakIPV5cPQ/kLvS/UFU7BdF/pKtzR9XnQnFm0WGYDvRrRvkVA/g6MkIp/fEQbGCqG9Mhvae05uhNw7FqPdR1TGw9eWtPKN8hoJ9BfQ3XcKIzB+ZF3+n84N1ZSBVQeplotunA9CvH9x/P9xzTwsHZn0Gq9KgeKPTp1NSxN7P/nMeFAREjoRLG6DPPzw04tbBQjLV1YnPkypEhVTuu7f9kbeNpCB1LCH1g7lvxGNc3P/iNp3nywVfsv+W/QyLF8o1g9HAqI9H8eC6B6nUVLZvkMowmPwz9LjR9pg6H4o3gKbY6Ussdp9+ft4LdHaAVAWaQgJVdciM4iIxmAzoDDo2/HsDHw7/0OfmkZZsU7oiwsNhatyFjPa/wraJO/aqsH91cS15AxanCL2sBrUaNj+/ma/P/xq9xsPrs1bgsp8v487j4t7h08QfCGVvjxtEFvS2q2HHjU7XiwDfHBb5eI9MfIRzup/j+OTmRbDRjcxhb2N+Hkz+yfqthfjTSMoxYXJQ/CWNTWLep/OIHxbfqUOsOFXBS1Ev8dfTTZUQajXU+IuKWf/o/k3VPWlXwSUaiD+34weaswx+HQy5Pzo+XrAG9j0A2lLnr+tKcJGFFx0obpw6RbFDzp83ULC3gNW3r3YgqM/WiH1rpLSR4u/EW6LJUOK7ay2nqDgAy6PgxNsAxMbC+edDiHoQy74OaLFGZ4+fb/2ZVxNetao8fB01BTUcX3mcCrP/ubPC3NTPp7Jg6QJOVzhXaHha8VdbC7+Y+1QtWeQdhu3Xw5bL3T9eFdmsC4o7uGHbDdxx1Hluui/ijl/u4OoVV1tzeDEZRQ6rC8enJtCWCtWf2U4XmQqOPAd774MN54l4EU+i9jSPh6o5es23DHvX+y4e/19h2ZvEzRjE/cX30/0c8feNSY9h/ufzGXSla3FF4zmjcaOERCqh9+zebbK1rquDkyfF/50Rf4YGAz/d9BOKY6KWlJfn+HxbY0sW9l/I6XtOU/JACTcOu5HZz87m9iO3kzIxhai+UaROSeWC/17gth2qBRYCrc5DJdyOhOWakMsds2IHXj6QW/bdQnT/aILigkgZn0JgdKDzk7iApT4c20bHf5PJxKnfTlF11j1P6mTh1kyO2a3ZwdWqEbbnbuenEz/ZFJ/BPaD7NU2bIw48CssCWt7n1mWLPMDDz4jvT38BP6ZCkVi3F9cVM+7TcVzw7QXWz87Ye8dSOeY8wJGA9QVYiEhvEn++UIL6G52BzPdg34MMDPkEyqDeUI1Rb6QmvwZFoIKASO/4ZxUdKuLwN4ep7z4QFBAsNweAFvwGmxYIH+C0Kz3zZnmr4a/ZMOa/0P3a9p+vLht+Gwv9HhRdb1svh4k/CDsmM1xtBL7+GjYX/Eb3sQeZ3/cCekX2wmi03XgtE62ls6PQfUW8zyGjLIPFvy2md2RvFg1ww3/RDj9eKkKBnUGrhVK5yP+z5AE2i4BEuCAbTK0oVkaOEUX+lhRnWZ8IUnHOiXarVAWJZCLQlEXVWT1hKY4LvtGJozlRdsIxYBwYefvIdr1vZyCqXxTpl6UTGB1IgC6euKpuDI50cfCY/9r+rykV5J+HCykSfS2Tg/8FfhcCzXidR4wQtncuNrp+fuKzWmB2TQ4KsrNgkHrhNnv0JShaT2DgakBiLZoVHSrCZDS1KSugM6DXi68w/Qj+PWmES4uRljA1zbFJ4seMH9mdv5tT5ad4YvIT7RukPKBpTmzPW6DXrS5fYiH+ghoLgrOXwcn3YfhbTjOOOgyKIJifw8+/gMxgmw9rdbVoqjSoK9ToNXqvNgVt+PcGTvx0gms3Xkt+Qz75Vf6YiMbfv9EcUJctNhFDXwF/u51Q2S5hwx3XiDxpC/Rq2HaVsEx31wrLDUgk8OKLwhbGOl9oS6DqKMjaIDnwJI48D9nfwLlbCDUrPfSyaurqYMh1Q+g2pZtX89mUgUqUgUo0eg0vEAuDgrhTehzoHHs5t3H8DXHf6nOX+H7YKyDzb5IJbcHyRcv57/7/cu2Qa5s+qa9zSZL4FBo16kQGiJu8SWJAL6tGo7HZNIanhROe1npld3th1BtJvyydhOFNw4A1GtDKi1BIlQyIdjEvyzpp5x47Taw9Glv/9r5TzG2edjHpbOy4SShEF9UJZYwdogPMxJ+szFrs01ZrWXntSrpN7saYezoveDF5XDJXrbvKIbcur04QQDGKRsTf2C9BndeyEsjXENJXNChE2PY7ixbBunWQmQk7d8JoN80J0qalIZVLMTb4VgORK+Ruz2XZhcsY+K8FwKAmir9ydTkbzmwA4OO5Hzs9h6cVf7/8YitcFhSIgndg62rE7qPuTOsbV3WVIusvdiooXVvvHik+QmZ5JsPjh1sz/0CoX7oCPhzxIdH9olkxZAUFtQXcO0ZYoKOrhE0LocdNMPrDlk8U3BOmrbN9L/ODmTvFHrF4I0R4OHu9Ppe7g+rI7FaHKq1rOUZ1JViK+X5+OJA4cj9580pRRI0wIWM92kN5LD8SgH+EP7PenmU7d5UGVYiqTfb6hw+LvUVionN1mCpExeWrL+dsbQS/f9FU8dccwQOQsSqDnC05jH9gPAFRjjVkmVTmoBCXSCRc+NWFrf4Z7GEh/nyNzHEGdxtVjXojulodfmHu7/emThX1T3fvxU3GVqbmq5lfMfTGocz7aF6LxweoS4k5fYqKuL5UVYU2UYL++cSfFB0s4pIVl1BSL2xtowOihTgGk/O1ckg/SJoPhhb+mMpwGPO5bZ0bMUI00ptrcbGBYs+vM+io1FRaXaL8/aGy0ncVf21xFPAUulg7mu9DKvXRX6lfPESNIzRYbMi1phoqzlTwerfX2fLiFq8Na8I/J/BI3SPkBwntdpjSfHcKTIZetws/X08htB+kXgEB7ch3socyFIb8BxJmQXBvsWgO6e9wiCV41UIIAJSVwbffwvcFL/Lg7w+w9axQkxUVCT9/pRJizI0wFuKv4GAJvz/0O0UHizwz9k5AVU4VGasyKMwRrGVjosoduCL9QOT7VQUK4m9UkpsL5sAUW7edO/CPFUX+gKbFIQckLRCF53Z2QIL4+3dPruODOb0wHXgcEPPKwIEDkUql3D36bnbdtItbRtzS7vfqbPRb0I+FSxYSkhTivuT9yAvwQ7QohnsalYfg+KsulXxWRAyF4a+KkGEXsLcWtS6Ua89A8SaX6o4OQ9VhKNlMaJCGwtBVPHRyLPetvY+lC5a2OSugM7Bk9tckHvsd8NzCaHf+bl7c8iIAt4+83TGfoy0w6Jo+1sKmzLLpakL8aYpEl31rmhE8CH9/kCJHKRUfQrVBzXmvncc9Wfd4lfQDYUOjLlcjkUro/kZ3Ltkdh1ZR1HQjdehJKPoTsCswmkyiKWf/I67fwGRyac3aBKVb4OxyqDrU2h+jRUgkjXIahjwPC0tA0co2W0/DoBFf+nqCVWbFn7SG+npR1B12wzCvFu6KDxdTkVVBjbYGraQarTKfID8flIOeeAtOfWT7PnYqRI1xOWfIpDJuHHYjcmcNI1N+gWm/ddBAPYjaMw75dwHKAKsjg05W5qD48xai+kaxcMlCes1q2qil0UC30pvZf3Edb57vJBe8ZIvIbuwM+EWLtUdj4k8VCdHjvd8g0F6EpkPiBaBvukYK8wsDQC+vtDYvyVQyMn7MoPhw5yqiAyID6H5Od0ISbfNysVao/2JVjYi/oFTxt+lqkKlg9MeQdIH1obAwGDWtgEMpd3DLnxe4fm0jpF+azux3ZxMY01FMlQ32+6O2ImFEAgu+XIB/H1EfaLz+3Jm3E4BeEb2sjQyNYSmEajQ2Fbo7cKak1GphldktzXKrOHXK/XO2GtPXw4wWnG0aI/sb0aBd9Gezh9279l4WLF3A+tPrHR6vzq0mc3Ummiofq842gjJQiUwlo7ReqKutf3+ZH4z7un2N5OGDRIZ8r1s9Uj9wQMRwppcns6RMQX15F/BH9FG0NL+o1aDQ1FC97TD1pU1/zw3qBpcOKiEhoNTUQFkZNXk1lBwtscahAPx3wn/5ZMwnbRr3vn3i3yFDnD8vkUjodX4v+owT13Nurm0uMplaVvxl/Z7F1he3YtDZxvtr5q80GERzWll9GetPr2fNX2vY9e4uyk/agko3PLmBFVetaNXP05WsPi1jbLxf3f7Gdg58YYvpeb3b63w96+tWnVulggsusNWHWwuZUsbs92cz8HLX9Sx7lOzLJeXIGgKrCjh7tmn2Y8mREnK35SKRSCipMxN/gdHCXW2pn1BFN0balTBxuVgrNQdlqIiciDI3eYUPhuGviXkTEX9kWScW1opa86l1p4havwy/6mKfJf4sdU9vcEY+ylL9DY8jeT5MW0tAjPjwGDEgCZUw5r4xdJvoRhhWB0IRoKBUI24IVnIoYjiMfKfpZrc9CEqDcV9B3DTPnM8vBvo/CLGTxeR1/j7wixK+7X9Mh0NPMsis8N+1y3ZD3WWObgvQCgIqq0J43Fu6bRISRJAt2Cb2iuxqtvxnC9kbsz0z9k5A1h9ZfHvBtxTvFRt0V5sld6A36jE0ysuoqoLKAIvirwUbC5NJZM7oKts2AFMLXatRo6HffSD3TOExPiWIJYee5LjGZr2ia2Enmbcrj2UXLesy18g+45fkRSxBLXGSa2BsECrl0h0iIyn1io6xTYocBXOOQ9o17T5Vaqrt/9aFcvYS+H2SyATqTIz5DBbV4Bfoj15eySntdo6UHGHCQxMYe5/nFEueRsnhYlT1ldT7nySr6gR1urZ7ephMJq5ZeQ0jPxrJjrwdqGQq7hp1V/sHue1qYYdlTwA21Ir5xQU5bQnjtjSCWNHnLrioTCxmOxtnf2Bk7PcA+MsEI+nKUtkbOPfFc7nn9D3o5XpMiJunzBDYlPhLXigKlX52uyB9rbDjay6388Tbwj7EHevv4N4w7DWhAP9fwaAnYe4J8I8lWGmz+vSVTfeXM77khyt/sF6zUoM//qpOUldbcybcaD0+5y+Y8J3jYyYTlG53n3juati3GNaNd1g37bl+D08oywjQpTlsxnV1Ot7o/gar71rthYE6h2V8wYFya1HBAbtuhy2XduqYMJkc7zl1Zz0fheAN9L0HJn4nOrsbwdK93SCrtCr+5Co5jzc87lanuifRoG5oYsP2Vo9sph3MIjXAThWqqxL2VV0s26459Ej1Izv6PQ5oV1mLa76GlvZHLSE0OVRY8pkXaY2Jv21nRSPDmCTXa4CAAFsTj7uqv1dfhQsvhOeeg+3bheMFwG+/icJ7XByMGiUe6xDiT1cJDeYKf2tVRQmzYeS7ItKgGSSHCJVfTlWOw+NHlh1hyewlnU7itxbX/nUtU9+ZSoNZbW+tU8kDIPVyiB7X9pObjCJSpHR7+wfaGPIA8iWBXPDFlWyd1kpS9284oLn5RaOBkOKTHH1yOaf/dLQB3vXeLp4Pep783c6zeENC4MzgCzg15y4uWXkJV/9+NTKFmERMRhMpE1PoNqVtNVoL8Te0BcOqiJAGZDLxc5SZyzFarW0ucqX4m/bsNO4+dTeBsaK5o1JTyawls0h5PQWdQcefZ/5k+hfT+WDZB6y+YzVfnPOF9R5auLeQk2tPYjS4rwjvSsSfRfFnn+8HsPm5zez5cI/1+/TL0ukxs0cnjkwoPUfcMoK0qWktHwz0mNEDrr2amohu5OQ0JYQXLV/E4sLFAI6KP79YSFkEQc4znFuErtKtDOu4ILH3t6xNavJrUJ46hlJT5XPq0L8z/v4fwmj0bVuL+MhA+p19mfFlHxIQHsDMV2bSe473rGJOrTtF8eFiytSikyomqB1ppd6GRYqsCIaqI6ApYdAgMfFXVEBGhnh6u3l9ZyH+TleKhcIhs5igp90caSH+cqXJ3Hbkdobf4mEriA5Et0ndmP/5fGpSxA6oLYo/gLnfzCX4+WD2FOxxeDyruAitMh9MEobEDWn+JLWnYMMsYQnXGhg0sDQQtl3bute1E6mpsPTI42w7Ox8Q80pGRobD/KLVa1E32O5q2iotx344Rlmm5wPCPYU9H+7hp1t+wqAzsEX1GPu6X0GJ3km4u6ZYFNfOfCXsrMZ9JboiPQ2pDEL6tKzoBNh6Fey+2+XT9oo/K/EXPxOGvQ6Bqe0ZZeth3sAHBoLMIFbtNdoaht04rEXrEW/i0p33kjX8Io50u52+7/Rh+bHlbT6XRCKhW6jtj3LtkGuJDWqjKb49wtIhZjLI7BRxmiIxv5z+wulLRo2Ca6+F669v/9t7DAceZWbCvwBQScTGrVpTTcmxEvb9dx91xb4RpFDXYBuH3OiE+EucDSPeEfdctbkoqQgWVsEDXdi6mkxibinZBBI3VGuBKdD3H6LZp6OVPjnfCatzH4JV8Wcm/vZ8uIc3e7zp1YLd+H+OZ/jNw6nRifWF3BjceZupb5UiZ6J8T4uHEpAo7jH2qDkpFKlHnnX/PQ0aOPGOg5LOZ5F2jSDKzfnXRqMRTaGGML8wJEgcFH+KAAX+Ef74h3eeWrPsRBmfjv+UEz87b8hpMYdj4L9g0FMdMzhnqD0Nq7rbrhejXmQOb7mk88bgBVhI1wZZhUPGn0Ta+RaayxYu48WIFx3IP71WQYAujSB/u7VAzlL4IVZkMHZF5HwP6yaKOcqMfqnhhKhFF+um7E1unUZXp2P55cvZ9mrHz1fO9kdthb1tnz2254mN+9gk141zEomtSO5Ozl9eHvz5pyiwb9sGzz4LF18Mt98OS5aIYxYuhN7m0oLHiT+jHjZdBL+Nt5F/rUFgCvS6TdzjmkFKqFBRnq0+6/B4jxk9mPvxXMK7d77Vc2tRphb7apVMZVWvewRGPfw6VKwHPA1jA4kyyO5zgqi5Xbi25mW0NL+o1VAd05NhT13QhEyJ7hdN3/l9kSqcl9ut9sB1MvzC/K33th+v/5Hf7v+NWW/P4tz/tD5LuLgY8vOFiMBZvp8FyxYu45Wo/xAXK+5rFgGCZf5SKFyvg1TBKsK7h1st/y3ZlxIkKGVKK+F/JFbkJetqdVbL0oXfLOT+ovtbFRfQlYg/V4q/6zZfx9wP51q/n/HyDKb8a0rnDawNCI4PJnVKGgalP9nZzi1gJRIJeqOecrUQ8UQHRgtRzISlVnVeExx4HA7+W/y/dIdw0jDoQGtWhp78CH5MFq5Z9thxI+y0OZ41Jv4GXj4Q02OPUx3Ty+cUf43XF97gjP4m/v5XUJcD+x8m2rSJHkWLic27CZWXLWJMJhNLFyxl7b1rqdQJ4i8u1EwOHXoatlzWstKqtTj1KfxxTuu97J3h0NOwdixoGxEtEilcWAgj30YuhxEjxMPbtokP/QGzyrux4m/vXvH4MDvxWkyM2Eyo9UqUCdHWTqCugIgeEQy+ejDlgWISjwpo28JTb9Sj0WvYk7+Hgwfh/vtF0LlcE8s5B/K5yvR7y/Z9ynAY8TaktjIvUqoS1lzNWDxiMoki4J5/tO7czcCiHjtzBqcqxZtW3UTQ80F8fchmEZA6JZXHdY8z7AbfDfHO+j2LvR/uRSqXokOsHiICnfhIKCNEFkLPDrYzLd/n/oa3Yp+w0HQBp8RfxHDR0d4o86jDUZ8LeT8T5l+M3EL86TwUPNKBsBSE9QoxZ0T4t+/39tCEh+gd2ZsARQD3j7u/vcMTSH9MWFTYwz9eWGSlOM8wlctFASetcYNd3mrI91KBcOT7rKkQFoQqqZg/6xrqyFqXxarrV1GaUeqdcZlx4pcTHPz6oFX1KUeFBJnjRsoSDF68AVYPhKxPbUoLfT1smAP7H256cokEztkEU9eKOb7EDbtzo168x44b2vVztYg998Chf3fse7iD2jNi41VzyprVYbH6lKlkqEJUTRQwnYkx94xhyLVDqNGaiT9DcIt5Gh6BfcaeqoU1TUMt1GY1tQcO6QVjv4Du17n/vtpS2H2nIBZ8HUnzxH2vUWabpYBkT/xJJBJu3n0zU59yzGXtSJSfLKf4cLGDpZYFBgNUyI6zsf9QHtlym/MTJF8olB6dhYBk0cxgud4MGuhzDyTObf51XQG6SrF2Pt3U7qpPZB/GqK4jpmqOVfEHwt3i5NqTTY7vSCSNTaLvgr4OWUtOc3yCekLPm5vfM/gyGqrMTTS2TvukJIioEdn167NasMU3Q+Gv4PiK4+Tvarlj3xew5h9reC3lNeorRGXOnvgzmozsyN0BNK/4A9va3x3i79dfxb/p6UL1FxEhSMCzZ6G2Vnw/bRr0MAtCPE78SWQQP0M0ssnbkY3bgvLYkuvXWPEXkx7DsBuGERzvY7m8dmhQN7D15a1krBOd25EBkbY5IOc7WJEIBeuaOUMLkClh4gqY9ocHRtsI1cf5PeQ44+duIelhD9uI/g0rNBpo8AumzyVDmmTdpU5JZdHyRU6zhAGOf7adyJx9mIwma3OLXquncF8hpcdL29zksn+/+LdPn+ZzQZPHJzPgkgEkxou1UGPiLyTEtRC4/GQ5Nfm2uoKF+OsdKToVkkLENZdFFo/oHuG2g7b1lCJA0ercwq5E/LlS/EX2iiS6f3TnD8gOq+9czXsD36Oh3v288OQkE7IGDTk5OGT8mYwmDi89TNGhIsrqbbVwt2o3eT8KVywQkR3rJohYH0stNWIYJF8snLnsUXEAKg5av7Xk/BXViSgsmUKGX4Cgt/5W/DVFJ/ni/A2vQ1sCR18gZEAIMAmdDtRqEz9e+i0JIxOY/Pjkzh+TCWa/OxtlqD+9PrmUVNm/WXyruVBQdRjKdnje3q8uG8q2Q31O0y7s1kJfK87TQhbP2LGwcaMg/vr0EQv72Fio14kq8MnS05SWQna2uMHa+3HL5SIfqqQEzp7UUqsvIXF0YpuCfr0FS6dcWxV/w+OHs+bkGnbn70Z9SignX34Zpk8Hv4Z4hoTGt3wSVST0vqP1by6RwJSfmz9GXysyNOWey7KwEATXpp6H6bccOO+gw/MhqhD0Rj0HCm1e4VK57/dxLPhyAXPenwMSaJCKlWVMqJPPj9xfKP0syF4KB5+Aid97rqBi1MO6cRA90b3cpFkHm52PEhJEd1xDg2tP/E5DwW+w4wZi4n5BbhSfuxptDeseXMfpP05z856bvTzAptBr9GSuOI5/dTQ6WQUA4X7t6wQOUASw88ad1DfUEx/sxjzRVsgDoEcbCKEDDwuryXmdW8QEIHYy5ebLeWLIlciTjpMQnED83HgiekYQMyCm88dkh20vb6PoUBFzZgi7TiVifrVupOpyhApmwGOQ/ij0vhtiz4GSzbD7Dhj+hlhHKF1sQGRKkfu7/yE4+h84ZyPETHR+7F/zwKiDQU+Le4nJ1HpbLHcx9guQKFo+rqNRsQ923gxjv6J7wlyuVC7jZEYE9aNgyDVDGHLNEG+PELDZ08qNQZ1D/EkVcLmbhGfxRvhrtmgKaDw/pF3VuvdVRoocJntL2y6ENblrWFP/HuqQ+Wg053t1LL1m9eLB8gcxGZv+HTUaqPY/SHXAfg6X+kh+nlQu1h8WKIJg2CveG48nIVVCxhvQ7XJIu8LhqaHxQ7k1/lO+34KD4u+3xb9ReqyUB0oe6LRhNt4jrzm5ho/LlqCLnIK/v52UP26a5+IkvIG0a6H79Q73t+BgSDZO5gxvscFN4k8ilfDPin8i9+saZaaA6ACC44PRIdSb9oW546XHqdJW4S/3Z2Bs8/sPy9q/JatPjQZ+F3HWXHQRDB8uXCFKSwXxV1AAAwaIzHcL8ZeXJwqZHrvPSSQirqQ965mD/4Kjz8P8XOGI4ASuFH9dAZoKDeseWEfcNXGQ1qiOIfWDgJT25zEnz2/f611BFU15ylVcO2Qs0Sku1rZ/o93Q1jYg1Rvx929dNd9kMrHlhU3E6UMpSxlKdbWYa+UqOTfvvRlNZdvlSi3l+1lgif6o/hx27rERf41z3JxhyewlSBVSbj98OwAZpYIc7xMpaqtxQXHIpXL0Rj3F6mISE2zKYL1Gz+k/TxMQGUDiqOYVwxZY5r2uRPzZz9W6Oh26Wh2B0YFWQvfYimMc/uYw571+HsEJndMAIfeXI1VIUQS4v8/ce/Ub9KoPJifyBgzmfrngYFBXqFl+6XKG3TyM3s+KJms/uZ/IKD/wqLi3DHnO+Umn/mZrZhvyHKRcBAVrbe5ecdPFV2PMdOQGGiv+DDoDiryz+FUHoFZ7t47RGH8Tf3+j8xA6AOacQOEfiyb0IDXGYrIKB5O9MRuFv3eKTBKphMFXD6a0FKQfQ4AphhTLZ3T8t6DO8/ybDngYBv7bM0W7of8RX85QdQwK/4Buixg+PAaFQizkl5uFImPGQGh2d9YDRep8tu9WA/707t30RhsbK4i/7U/9Rv5Pe1lcsJiguBYUbj6Av576i70f76X6n4LgaWvG34SUCQD8kvkLMRV6QE5REaww5wK7RbJ0ZKFWEQwzPWtnEx0tFgwny4bTXZFEsEGNTGZTew6OE3aNB4sdCcGzW8+CBJLHJnt0PJ6CXCVHrpJTp6vHJBFq3phQJ4sdo150o1r+ZhKpyMnzd8OS010YG2DgU6Kb3h200IQgk0FyMmRl2X2Gt1wuOqfP2dCuobYaMZNgzH8xlQ9EbhAr+BpdDbo6HdpqLUa90eeI4rqSOrbfv5yotNHopJ5R/AGE+oUS6hfa7vMAkPkBlG6FIS+CvwdsQ4e/IZRpXoKfnwmQcF7QPxky6Ah9Ivsgi5ERnuZ966VzXzoXTZWGMp1oHFGYiT/rRsqggaQFIl9VqoARb4jHz3wD9XmC8Jt7Stj52mPPfaKQPugZQf51vwEaaoSFqytIzVZuvW714E/oAvYND95E1DiYsgbCBxOiCmFMyMVU1vjGpltXp+OL6V/Qd0Ffque0Q/Fnvy6oOi6awTy5TghMhr6LhfLbnTE0B7m/cB/oCsj/VShth79mHfPe8r38VfcdvYLimhB/R78/SllmGRMf7rzipFQmBScGGlotVAWIqll6rJM5oXgTbL0Chr4E3f5/W212CuQBcEGOUM07gUWtYE/8TXhoAtpqrdPjOwv7Cvaxu+FLkoMUjsRfV0fj+6UZg8MmsgnIqDxEubrcrbVZZ5J+9vujtmDSo5OY9Ogk3n9ffG+v+KvV1TImaQzBymBR0GwG7ir+Nm2Cujqxv7e4/EgkYu8X3UgQEhYGkZEif+v0aejvidQDowFMepCp2nfPC+oOcTNEAyzOi6z2GX8mk8nauKyuUPPxqI/pM78PM16a0fYxdCD8I/25YfsNbCjfADsb1TGS5oovX4V/HBETvuDkOzvZ9/RvXPj1hV6xSf7/gObmF2XWcYbuWUHBhktIvqqpoGDvx3s5+etJLv7+YoemfYlEwh3H7mDxzTWgE3NGYqLtubban5tMNmexlvL9LEgyC0IbE3/N1deG3zLcoY6QUWYm/qLE70AmlZEQnEBOVQ5nq8+SGGJH/Gn1LJm1hEFXDWLBFwvcGqNlLeALe5CW4Mzq89TaUyxbuIwFXy4QebJAeWY5R787yrgHxnUa8deWubb/Rf04+avC4b4WEgIKuYKLll1EaEooSVFJ1D1Sh0ZvJqxzvgOZv2viz9+uiTF8iPjqYV5L6apA6aJu06gWt6DvArqHd2d04mgAGuobqHvnc2KTh6BWX9Dqn7Uj4cpKvDPhW5W//wdo7+KzwyDzE/ZCihAOpNzMjj7nsiFrKw9VPsRFSy/y6tDKzXa+4eEgMZmlxxIJBHSANYHMr+MIIHsU/Ql77oKK/fj727puTpgjRUaPhhH9I5DrxV11/d4zgOj6a4xI8zrTb1h/pj03zecK9q6gClURHB/MI9Me4ZfLf2Fmj5ltOs/0tOlE+EdQVFfEoeq/AGiQVfFb6giyYl8jKLQFubrJBD+mCF/otiDzA9jrIZtANyGRwODB8OXBZ7n3248prw5m4MCB1vllcKwg/g4UHnCwW1t64VLW3d8O25EORvGRYkqOlVBUaV49mCREhzpRSh5/BZb6QbnZ/zblYlhQINQ2RX/Bpouh6mj7BiP3h/4PQOql7h1fewbOfGuzF3SCQWYrc6ulo0TqedWyOwjuCd2vRRGWbLP61NYw+53Z3JV5l0/OIf4R/gx6+iJKkwailVQBEO7vfQLKAWU7IO8n5x2+v0+B9a3MYoidAomzPDGy1mPbNdwcLEcmaUCrlTrML76AhBEJdJ/e3ZrxJzc1Iv5CesPE7yDBjkQwmSDhPGG1HTaoaRHToIWi9YK8tdgQhvSCke8IO2hXmPg9TF5l+74up0V7qzbBi9aZTeAfCwkzrZsz+013RVYF217dRvER72T86TV6avJqUJepUUmCCasdQ7B6YOuIP6NeWHTvfxhqTpltXN0o4BeuF1l762c6t5G1R9hAGPay2NA2RtYX8H2EezazYC7U+naGuAP0NVZLfZlMRu8kYf+kk5c1Kdwc/Oogfz7+Z6dZx+7/bD9nNpxx+pxaDWXB6wGYaG46c4DJKJQt7bHGayuOvgTrJok16aaFtkzTro7AZNGM0Qgmkwm5fx1qRR41NbZro9esXqRf2kyjhodRW1jLiqtXOGRCltYLK2ylPso275hMIkri2MudNrYOQeHvooHGDn2SYghS98WEic05m906TWV2JUeXH+1wklYmk3ls/eKsMDcqcRTbbtjG2ivXtvh6dzL+TCb42WwkM2uWeyWJnj3Fvx6z+6w6Akv9xZzSHnS/RrjiBHV3eYjF6rNWV0uVtsr6uCpYhSJAgSrYi/KHFiBXyUkancSV51+J5lENPyz6wdtDajVyt+Vy+NvDGHRNra3/Rstobn7R60GrCKQyrg+Jw5w3gxYdLOL0n6epL23KWAVEBhCQKl5nmTP0TrYW+flw+LB7W4RTpwRx5+8PvXo1f2zp8VJW3bgK+VkROZRn1ltUmT+mzRF/Y+8by+i7R1u/b2z1CTbS/2yVo9rXL9SPeZ/OY/Q9o3EXXd3qMyQ5hBG3jSB2kO06GX33aB7TPUbiSPdUj97CrDdmYpxiczKQSCAoSFi2Drh4AEmjbfV6P7n55jn7WPNN7/p6sf/JWS72NxYYNLAiDja7aKyrzxW1uHrBUk9Onczdo+9mdJK4llShKiKuOI+ypME+l/HXWPHnjZqL71X/uji8mXnSLEwm0QWvLsBfJlamJVXezXza+8le3kh7g9Ob8ziUcgf7E+5C/dcCUdRvnIniSVQcgGOvtq/IZjLC8deaho5akDgXpv1u9SYea5fbHBQkOvbS0yUMz/qO2bl7OHtILJrt8/0siDA3Vjak9GDiwxObeIj7KsbcM4Ybd9zIsH7DmNVrFt3CurX8IidQyBRc1E+Q04elYjPa0O8rqgL3cDbqEyJCW+gqNdRDSH/wa6NCJ/8XOPGW6+uleBMcfRHURW07vwvceafowiothcceM3I2p9I6v/SL7odcKqdKW+WQm3DuS+cy/qHxHh2HJ/HDFT/w7bxvKa4SK1y5MQSVysmuNzBNdJHa25pZuoN0ZXB2udi4diYKf4Otl9nISCe49lr46CNbrifjvhL2bF6Cvz/IjcHIDcFEB0aj68h5tZ1QBioJGz+Aqgh/kIjrvL1Wnx7HmE9hToYgjRvDL651c4zJ5F2iJ2ww+ZILkEn1lNdXcTj3MGX1ZZzdepbnAp9j59s7vTc2O8QGxnLr8FvpqRX5iS7JHZMJfhsrCq9ShdiZNNTAyQ9tGSwyFZy/Dyb/3LTaZjS4t+7I+gx+7AZHXxDfG7Rw8mOxWWkvqg7Dt36+VTg2b8gO6r8nJ/JTymtrKT1eym+LfyNvZwe4MriBgMgA7j17L+e+eC4T4mYy4fg2Bp19F6WyFSfRFENAomgGkwdA92tFblpLyP5WZO2VbITKQ239EQR5FDbQ/caQ7CXwrQLyfmn7e3YWEs6HeaesTQ0mk4lAqWCOG2RlDuotgBkvz+DW/Z2gpkVkkvx0809se9W5S0NJTSWVgbsBmN7dic1Q7GQ4b7d3Gja0xSJeoPB3yF0prtv/D9CUinW0wZEg0hq0LNgdxB+Dkyivr3Lx4o5HZXYlB788SMmxEutjpWpB/CkMkTaSSFcBlftFI0FXxsF/wZ67HR5KSoKI2klESLpT3+Be5fXI0iN8d9F3Hd4gYjKZqK6ublf9ZdPzmzj41cFmrbjcidhwR/F34oRwBlEo4Bw3Bf4Wu8+TnnKFl8oheaHNVq0DEaAI4K3z32LZRctQymw3aalcyq0HbmXyE16ImnETRr2RBnUDJpMJlVzl2IyYvVTYxBt9lFDTlpH380hkF3/JvNx5XcZ219fQ3Pyi1UJNVHdOjbiE2D5hTl8/4eEJ3Jd7H4HRjk3OJ9ecpLao1mHO+OUXuPhi2GzXW2EywaOPwsMPw913w5YtzW8dLfl+gwaJuKDmoNfo2ffJPvSnxVq+tFSQVu5YfdrDaDJaiT+L1SfYcv6c2fwOvW6oy+xDZ+iKxJ/9fjVxZCKz353tQPzJ/eTIFJ1H/lTnVfPnv/6kYG9Bq1+bkmL7f2AgSFvaukhlzTfUVh8XuX6bL4KdN9ke15YJG+XYKc5fV7JV1OKKnTcgSSQSYueNpjYy1eeulcbrC29wRn8Tfx6G0ejDHbk/psCu2wmSi7tMaU012RuzOf3naa8MR6aU4RfmR61exdmoTziifBuj0QAYhQVXR+HUp7BvMVRntP0c2lLYex+caRpID4gO1rjpoAwDYNQoW51x5EhhC9inD8TXz0BSOAxNrYqgIOfdORbFX1lZ0+f+V3DZwMsAyA9ZjkGipSDhAwBSSm4mMrKFzZg8EKathcHPtu3NR30E85spcOavhv3/hIbKtp3fBUJD4emnYWSPXTwyeCAr3lqOwSDmF6VMSb+ofgAcLLLZfQ6+ajB95rYzu7IDMfru0YxdPJaSarGqVBpdrCq7LYIpP0GAk0Vhwiy4uFqoANuDzZfAhtnuHx93LkxY5ly9YYZMBnG+EMFUnws/phGR9yQKQxjn768m9948yg+Vs+fDPagrfCzx2AyNBhrkIt8vQBGASu6DncB+LoK5J3wriF53UbEflgVCxlseGVar0e8+9gb9gM7gz3fV9zHwk4G8t+s9/CP96Ta5G8GJXlC02OHtvm/z8ZiP6Rfdj/fmvMfQyicB80aq4gD8Nk50ClogkUBIX1EMN5pV4CYj7LwVsj51PM58X7Yifw0sVUG2o8oBgMrDorhjKeYmL4S0ayD1SvH9qU/EpiXrs/b/0BKFsPoMbFuTjEehqxQk5K7bAPig4AYOpt1AcX0BSWOSuH7L9fSe07v5c3QC7DfYrTJzCEiAGVsh/QlhMzj6I9vcrqtwXVkZ+C+Ytg4W1bWc/7v7bthyhfPnEs6Dc/6C6HHujdc/HhIv6BgnjA6G0WhEXy3a2HXysib5VxE9I4hJj+mU7GqTycSVa69k4qPObUU3nf0LJEZC9b2tRSufwcAnYV6WUDpfoml/tpSvION1+H0S1DiyGn5yP1RSwaqV11daH9/8n828mvQqVTmdQwYmjU7i4ZqHGXnbSOtjVsVfQ5Stq18VAQtLYYSX7umewpDnYeIPDnNgUhKk57zF5SWnuDTdPZeMPvP6sODLBYR379gGLqPRSFZWVrvqL5ueFcSf5X5iIXNNJhPqBvfXy5ZCeXMZf2vNwsFJkyAkQA112S2e10L8eUzxF9pfzCOJrdgDucLRF+HAY80ecueoO7l4wMUEKLpWs0L2xmyeC3iO3e/tbvrk6a9E9rw3XF3cgbGBmOo9nMpdyd4i1w2rf6N5NDe/WOYLudw1yRYcH9wkT634cDFfz/qaX277xUr8HTwIH38sFH+b7DQF+fmCkAM4cwZeeEEQgToXfYp7zX/qlvL9AGLSY7i/+H6mPzmRsDDxWG6urXHBleKvJr+Gz6Z8xr7/Clv0BkMDj096nGuHXEtaeJr1uOuHXs/Hcz/m/J7Oc531Wj0N9S24dpnRlTL+LGMMaGG602v0nN16ltKM0o4fFFB0oIiNT20kf09+q16XvSkbv1+W418tXCYs18Wej/bwUsxL5GzJ4dN9nzL3m7l8vv9zsYcq3iiaulwhpI+IQog/H1Ls1H0BiTD7CPR00QwYPR7GL4UYsYbX6rVsO7uNX07YmiIt14rax0pdjYk/b3BGf7d//K9AIoG+90NIX4JVG0EL5XXV/HyrKFzccfSOTh/S4KsGM/iqwXzyZR1GqRYjYJr4XceSfiByepIvbNaaokUoQuGcjSJHyBVMJmioAmUYoaHCunH/fphgdg9SKoV9x/Hj4vuhQ513UFgUf+XlsOyiZRj1Ri5d6aY9oRdxaMkhagpr2D52O5H+kVyafmmbC/kTUyZyfreLyP1rBrrYrRwrP4RK5sc9U69y28O8zWgpy6vvYki6QKjUPIyoKLjp7iQCNpSgq6klMxP6Cb6PwXGDOVR8iANFB5jbx4dzBuww9Hrxx9q6r4IRmT9ar+1WQebnNJun1TBqm3SYN4ugNPHl9vkNoqAVPhTiprV4uEchDwRlBHJ/UYUwmcSC48TPJ/jz8T9JHJXY5vyAjkLO5hz2XLqUhB6jOGfYY4wc7UNdtNpy2HU79L1XZMp5AlI5xEx2P2OyA2BZHEv1gSARVkxRfaK4YrULsqIT0W1SNxSBts2yQwdlQzXUnTHnythh+GvCAthi46kMhalrhbIqe6lQygx6qmmeVGA3SJzjqDC2oHgj7H8IwodBcA+R6Tr2M9vz3a8RRGPqlUJdWJMp8r/agtC+LZNJnQV5kGh2CBsIgL88kHpdNZXqWvwj/Eke573rtq6kjlNrT5EwMgG1TITDtzrfz4LGZFPtafhjKvS9D/rc3fT4gETx5Q6qjoDGQ04Acef4Tv5jSzBoRVNcYIp1zGFmsr1BXkZtZaPDGwxoq7SoQlUd3v0slUlJm+r6Pr4l/w8AUvQu7tlZnwklcZ+7OmB0LcBe4SdVuD6uqyFhtthLqZrmgIcowyjRFFKpqbQ+5hfqR2hyKEZD5xVNlEGO+9KyetGJ6WD1aUELOXA+j5hJTR5KSgKpSUl+PhiNbnT7A1F9o4jqG9UBA/Q8btl3CxKphJc+Ft9b/qZnKs/Q862eDI4dzO6bdyNtgeRpSfFnMMD27eL/MyecgTXni9/3qA+aHmw0WO3KLcRfTo5YyztTJHoNeaug+gQMfqbVLz3+43GKDhYx+XHfVP0FRAcw6KpBbJdt560Vb7FowCLm9J4jnhz1PmhLOic+pi3wj+MS2XzWn1zPU8o69P31f6v+PIzqEi09dy5H3a0v4MSyywxttZad7+wkomcEAy4eQFS/KOZ9PI+EEQn8vEscs2GD7fjMTNv/LRFB3bsLEcHKlXDoELzyCjz0kOPlV1UlLEFBiAxaglQutSoR09Jg3z5YvdpGKrpS/GkqNZQcKaGuSEQxqOQqHp7Y1Pp+Rg/XeXI5m3P4auZXzHh1BiNuGeHyOAu6uuJv7eK1aKu0zPt4nvWx+tJ6Ph3/KaPuGsX5bzonRz2J1Kmp3LL/FoLjW9fYW1dUR932w/gN64s6JM56XfiF+hHZOxL/CH/2nd7Hzyd+ZlDMICjdDhtmwagPoedNzk8qDxRRCM7grOnf+lyiEAeYUa4uZ9yn45BKpOge0yGTysh7bxX9N+ehHn5bq37OjkZzjgKdhb/vAP9LGPofAEL89oEWKuprmPbsBV4P+80tM1umSFQEKgI7fhEV2k98tQcylbXbwCV+HSwUB7PFXfj++0W3zuDBtkOie2exumIlMkMQ9wy72elp7Ik/g87gdneMt7H3470UHijkobsfAnC7S9QZZFIZT6V/x5PL4dTAa83nu4Sb5rvRSZrxliBgBzzatmtLVwX1ZwXpI3eSR+cXJb46CPHd43n+2zzWHKsnbo/ESvzN6y0WD6lhqdZjt7++na0vb+W6TdcRnuZjNol2UOjDiauaRy/nWfCw/xFhEdD/AefP154W3uDdLmua4+UuJq1s2+tMRvc6PBuqYN/90OOmzif+lOFw/h7kJnHJm0xisZx+aTqJoxI7vAO7LZD7yVEmRoM8jvmhT3OHE5c1r6HoT8hZKtSmroi/wt+F9UT/B5zPE40RNhCm/urZcbYGhb/TX72KSP8HKdMHgQK3Lbw6A3M/FM0MNdoaNHoNtZpgwE9spGImwgInHYvKcNENaI94c+5i4e+CjBj6YtPXhfZzPR+kXg4Rw1xbYskDoa/ZIjL3R7HhSX9CEIRdGVK5UF2b4S8PBB3UaOswmUwYG4wig0vV+duI0uOlrLhqBTNfn8lHMS/z+6CvGaa+H3DDqhMEcbzzVhEk35hMU0UJy16jk3Zqo164PfjFQm0WlO0Syr3GClILpv/RvCfTsVcEgeOMYOzq2HEDdLvU+vsNVYYCQvHX2Opz8wub2fDEBm49cKuDDVJHwNBgQCqTutz3hMhiCdD0pLvExQ3oxLugzvcO8Qdi7fPbGEj/F/S+3Ttj8DSix4ovJwj1E8RflbbS+tiIW0cw4taWC4WeQmlGKQ31DcQOjLXmI9tn/FmtPkt3CKI/7lznduBdCUYDGDXWtUxMjLCm1OmgoEhPUHgdmYdD0WhszaxdGZG9BOlsyQSyFOb2FuzFaM5WbYn0AxvxZ1H8GY1iDW7Zeh47Jp4LDobe6RFQaBLzJJgziP8U9xSTCbZfI9Y0w14lIqCCu8Y+TYg8hzOnl9Onbzvvuwf/BSZ9251w7DF+qVPS3h7ZldnsK9xHTGAM45JtKvcjS49w+JvDjH9gvE+SUrEDY1nwxQKuXnE1Xx78kkGxg2zEX2uagLwEhUzBqJ2jqHi9gppzany6LtAVUVmgJqQ0C2msCycYO2x+bjPJ45MZcPEApDKptRE6xM58LDgYamuhpAQqKiA83EYCpqfDFVeIGuLjj8PWrfDf/8L1dtHU27eLqaNnT4h1cylVk19D/p58LrmoJ/v3y/j9d6zqP1eKv+j+0TxQ4qI+4yaiB0STODqxiQWqK9jnjPs6LGO0J/5yt+WiLneUoAXGBjLjlRkkjHTf8rQ9UPgriBvceluq3nN7c9mRR1j8T9FwZrkuBiwawIBFAwAoOSKs0KMDo4X7zrDXILoDFwfmWlx0YDQSJBhNRkrrS4kNikUZIMcoV6KuNwG+05jhC8Sfj+rT/0ZHItxffGKrNNX0W9CPvhf09co4Nr+wmcPfHia/Smyg7g9XIdnzj855c5NJ5LMYnaTougO9uuXXJs6F+JnWby2qP3v4JZ7gaPJizsS87TTfDxyJv0t/vIyrf7+6bWPuZMz7eB7TVgjCI0gZ1FTt11AD+WvdzkYqLQWdrIJM1VIAbhl+i3sDOfWx+GoroXz6S1g9EMqcWH2AsFXs4Oy0ocOkSKUydtsN4eIBF/Plgi+5ctCV1seUQUpCEkMwaH1ILWWHTyd8yroH11kLfy5tELI+gbwfXZ8o833YdhXUtMOuty34qbfoYnIHimCYsQ36Le7YMTUDiUT8jg90u4EZS8eRHZhNjxk9UIX4UruwQMKIBGIevJaq2D6+1c0MkLIQ5p5s3l42fw0cfhI0Ja6P8SWU7yOx9i0i/PNBL3ZUtbpa9Fo96/65joNfH2zhBJ2Dd3a9Q8zLMeyMFp17bVJ2qQtg0DMw53jzmQPOoAyDqDHu2eqNfBcuyG476Ze/FvY9CPWts2HpDAQqggCo1dZRkVXBM6pn2PCvDV4ZS3S/aC5ZeQm9Z/emqK4QjTIXqbIVGYtlu4Wta9Wxps8pgmHmDuh3f9Pnak7Ainhh75W7QuRMODuHPZpbd5z8UHy5g5Mfw+57Onyt4RHIVDBlNaQ/bn0oJlh0+ViIP3s+NGFEAsNvGd4p96X9n+3nGb9nXEYcXBL/KNMOZzJYudD5CSZ+D5N/cv5cZ8CoF/mUtV08R85NRJgzteoMlRi8tKzd/NxmPhz2IQ1qW9OlPfFnvSedfB82XuC8aaArofa0sCE//LT1IakUEhMhO/p9+n4axYO//ounn4YXXxQqE2fQVGl4LeU1fr274xuc/Kzsa+uh1+ipya/BoDM0KdjuKxRWdsPiXat57GEpiFZVwbp1cNllwprPMt9tM0eLjhoFMr8QOP8gxE4VDx55HjacD9WZIpu+/qz4ksiRGOqYkvIpqzNvJ+u0Bwiy7G8hz0PzmCUrtxksO7KMBUsX8O6udx0en/rUVG4/cjtShW+XI2t0gskNUgbZHqzPF7UgX4XRwFBTKQ3pGaj+ofI5l5euBJfzS1gYe89/lIZJzXeqqkJUXPvXtVy68lL+evov6orrrM/Zk2t33QXJZjMNC+FnyfXs2VP8m54O//iH+P+KFbBmje31W7aIf8c36n9sDjve3MG3874lRl7OLHN5o7Ky6diaw+HiwxwrOYau0fpUo9fwR9YffHv42yav8Q/355r119DvQveEGF1R8Wdf47ph6w3cftixWUumkDH2vrF0m9g58Q6VZyrR1rTC6coMuUpOak+FdTvjTAlaUm8m/gKihVCi7z/aL7JxhbVjYY1o/pJL5UQFCPFFUZ1wWBn66CyOj78BtcZ3SD9o2ljkDfj2nbYLQibrvJDOVmP/Q7DlMiKCxExeq2smfbqDYTKa2PDvDRz86iBF1cIyZW6QAc582TkDOPIsrB4E5W30PT/6vMgEaq7oM/hZGPZKs6c5Z6gI9asPyMAv2PndLNxcp9RofM+vuDmEdw8Hsz1JuJ+TYmvh77DhPFHQAtG98X0E/DWv6bGIjMPi0J/Ro0ElUzEmaYx7A5mxFab+1oafwIyoMULB4Sxbx2SCVT1h4/y2n98NjO29h1vHvkBFfoHLTTbAsBuHceOOG33SXsdkMlGTX4O6XM2xsiPkRnxNhf8u5wfPyxIZI66QernoMvVvY5dU3s9w/HWh/GgNwodare9ahFQhrp0QL2UunvwYMj/A3x+qAvaxp3ibdVHkjUBhd6DRgFZeTIX0hIO1l9eQu0oUYUDYPDbXxd/nHph1yP1r8swSoWzVe2kX0/NmjvfLJ6tiKJIGUcyo19cjkUrY+uJWMn/ObOEEHYu1961l36f7qNOJDbLUKMhJkfF3EM58I3IEWkL5PliRAJnvCutBVzj6Iux1QvbUnXWfbAnsBoqglo9zheK/4NhLTS1MvYVjr1qze4KU4vdfp6vFP9yf9MvSiR3cseosVwiICqDvBX2J6BlhLcgFtkS21p62NWvFTYMLiyDtyuZf0xgyP5E7ET0BEuaIe1Cwk2BmENkWZ5YIZaArTP4Zpv/p3nvn/wKZb3cdi8eE860qWZlMxvRh0zlySybTD+ZYFegW9Dq/F3Pen0NYaliHDys4Ppie5/UkJNF5RcuyMfdTuSgYBKZAREf7yzeDkF6woACGvOC9MXgaJhOsmwDbb2jyVERAGAANskrrNVOWWcbmFzZTfLi4U4aXfnk605+fjirYVq3J+0cR5xzII0Db3Ub89bkHxn4hLKa7MvwTRUZ9sGOGa1ISKBtiqNVXseroGoxG8acrdvFnUAWrCI4PJiC6Y3PdZDIZffv2bXP9JX93Pq8mvsqu93ZZmxIthU0L8Tc0zr3PvKVQXlwMb74p5rmtW4U1n8kkiL/7xlzF/L6vmgdvZyEbkAhDX4HAZHM2/R8w9ivRPBKUxvKa7ewrnCGK8g3NhAi6g9lHxfk9AWODaKZpph6SHCrYjJyqHIfHI3pGEN0/GqnMN8uRp9ad4scbfkSfLdYOVuLPZIIfu8EWH45dMel5qGE91w/NQbJQgl9Y28nx/2U0N7/U1wMSCQFBLV+/8cPiObn2JBue2MD2N7ZbH09PF/uauXNh7FjobZ52T5wQ1sCWXM/edtPx5MlwpXn5+t//CmvhmhqREwgwzs3oaIB+C/sx96O5BMYEcvXVImLGAldWn6fXn+b4j8cx6EQ3zv2/3U//d/vzxYEvHI6r0lRxzpfncPnyy2kwtM+tzHKf1euhwceNz5xZfQJWxwBvYcmcJXw4zM1mw0YoO1JAiuQsYLvP7Xp3F9teFd0sJXV2ir+ORvgQiBhu/TYuSKgYC2tFBqGFp/e1mrlF8WcZnzc4I9+803ZheCOo0W1U7IeSrUxOnkq/sy+RUnUFq25axZs93uz8sUjgtkO3MfPVmZSpRefkQ9rhoiO/MxA/U2S4qNoSMoaQMSfNb5oV1Er0i+tOt9BuGNCx/vR6p8f4+dkk7mePVLP9je0U7Cto1/t2BurL6qmsqgQgROWkyBIxDIa8CElmok8iFTZbCbOc2mOVlopucYBPL/gUibsKPnkghPRu+ThXiBwBg54Uhf/GMGqh952QPL/t53cDwQ27uCT9WVJCDluDmwF0Bh1FtR7KEOpgSCQS7sm6h3kfz2NL8c/s734lu6XvOj9YHgh+rnxAgfDBwuPblcVaSzj9Jey9j1bfAicsdT+/y6BrXYagp5HxBmS8hr8/yI1i9Z6/Np+nlU9z+JvD3huXC+TtyqNy5QbKVR/z4Nk+3Lf2Pu8OqOYUbFoAu1wETDdGYDKEpbufUXt2BRx9wXuFfGUo8uB4DCYFaG2KP5lCxt2n7mbWu24qWzsAJpOJ7a9tJ/OXTOoaBPEnN4gx+vkBZ5fD1svdU8aFDhBfzZF+APm/wqlPmj6+eiD8ObPp467QUANHX4LcNnTTD3gE5p1qXZZoRyL3B6GWB4JVouBV11CHf4Q/C5csZOBlbjZBdCBq3SH+yvfB+nOEesICvxjX6k+TCfbcC7sbWXAGdYdR70HCTJHH2G2Ra5vvygOw9QqhBHaFkF7g5+YmedwSmJ/ru3lCzmAmzI1GI9WV1fSN6U6QzKzgqmvuhR2H3nN6c9mqy4js3dSaLqsii1qzqsupstighfo8YYPoTfjHdR0C2B1IJOJ3amr6ew33DwOgQV5hJWXKMsr44+E/yN/dOcronjN7MuEhR8sqk16FX0MCUuS2ayV8CKRd1Slj6lDIlDDlF2GFbIekJIiqmY4UGYX6DOqVZwDRlOkMEqmEG3fc2OH5bUajkbKysjbXXwKiAxh11yjihyVYLTqDzPzOvgIz8RfvHvFnXyiXyWzZfN98A6dPg7qqnMFxf5Ac6KTpuOdN0O8+m3pOKndoJNIFDAQkJGo+hpUpUHOyNT+mI6Qy9+89LUFbBmtHwvHXXB6SEirWX2erzzo8bjKaqC+rR1frmyrZogNF7P90Pw2V4r4QrDT/gU16sfdPnOPF0bUAqYIvVBP4pBoajD7OlPgwmptfyk9VEFJyCn+Je44Tfeb1YdHyRYx/wCbJS0yEb7+Fm81pP73MvWQnTohMT51OKMcSGvWULlokcv/q68X8snOnIApTU5se2xwSRyYy7MZhBEQFEBAAt9uJ0iyWn42x9eWtfL/oe2sT8YkyEUTYJ9Kx0Tk6MBqlTIkJE/k1Te/XdSV1LFu4jK0vb21xnPZrMl9X/TVWjqsr1BxfeZyqnKad+z/d8hPvD36/U8Y16MpBDLvJPfV6Yyy7cBnRu1YDNuJv9/u72fWuaOB3UPxtvRp+awX73FqMeg9Gf2T9tjHxp8kpJu7kFhpK3GgO7kQ0tvr0Bmf0N/HnYfiqkgKAqWtgfjYTuo+kR9H9BBfPwD/cn6C4oE4ft0QiIbJXJGE9IqnQCuIvPDi2+WK/JxE5Uqjxgnu27fWpl8PE5c0TD9UnYOuVzRYAJRIJs3qJAuvqzNUuj7PYfRacqGHtP9Zyaq3v2/y8O+BdDlx9AIBgVaOiXPk+UBcJG0T7guzcE9DrVqfFrdJSSCu+h5Vjyrl84OXuDaL2jNnStYMWvTI/EU7b03k+o6dgSlrI+2fXcbh4ktXu82zVWVTPqEh5PcX6+dVUatj68layfm9GZeADqNIKpV2wM/u8hmoo3wO6ypZP1Fa11Mh34dxN7VPntITcFbDUD3K+77j3aA7jv4FJqwgIALlBfP50YTp6nteToLgO/LnbiLwdeWjX/oWyXlT3nDYLdCaCe8DoT2HMf9073mgQBRC9mxXtMZ/AnAzvFXD19QSbThCoqMSoFR35tTrxuw/vHu51W6D78u5j9vuzrYo/mTEQPz/zrSH1cpjwvVDYtQSZUiizEmY3f9yE7+DCQsfHjHoxtzdn8doYxgY4+Dhkvuf+ayxQBAlyyVeK+hN/sDZjBfsJ4lVtqMXb/W37P9vPC6EvcHr9aeoaxDXrYMHVGBV7hfKu9jSoCyHjbdCWuz5eIoHyXSJvqaW1savnwwaKazS+GdLYoIHqDPeU53L/djeadSo2zofvBclnMpk4e/YsJpPJWlCvsROsFB0s4vtLv/fqusVkMjHt82lcezSCyoA9zq14KvbByiQ4/nKnj+//PWZug7GfNXl4SuoUetZcR5C6n5X4S5mQwk27bqLPPC+5KWBThsrl4guT0fuEcAcjKQkUhlDCa0VBryRkLSBiKLwJ+/mlLYjqE8X5b55P1LAU63QeHCyKiAW1BUiQMDh2cPMnMSMwEPr3F8X8//wHHntMXB+HD8Mnn0CtLoIPi84gG/l6q8dpmTsL63pBQELbnQHq86B4U/tVgxb4xcLg58W6zAUsxF9udS4Gu89J5q+ZvBT1EkeWHfHMWDyMMf8Yw0NVD5EXnwfYrTOkChj+miBrfRUSKdv9BlK+YyD66/RdomHcF9Hc/FL45zF67/gK/1r3Ih4kEgn9LuzXRH0ptavGW5R9mZk2u8+ePZuWxSQSuMEskl+9Gn40p6O0xubTHpafb+RIkRu4aJHIdnWGGS/P4MIlFyJXydHoNZypPCPGHunYZC+VSEkKEW5ZjUl/EHafZzacoeyEi+4R+3NJbUopXyf+Glt9Fh8qZumCpRxb0VQVLVPIkPvJMRk7vg4/4aEJjH+wbRfIlKemMPaBiYwZA5MmiceuXHMll/9yOSaTyWp/Hh0YLRpLJJ2X2Woh/ixCiNqTBSQd/x1JkW8JIxoTf97gjP4m/v4HEWp2IamrgynPnsv1W653Xz3lIWgqNdQW1lJZYaJb8e3MOFDERxOuE7ZanY2O+uCZDHDma6G0bAaze4mC5C+Zv7icBCzEnyEimmv+vIahN3jRashNDLxiIKqJYnazdslZcPAx+GMyLkNXnRB1ZWUgQUJafCsymjLfE5au7emMVBfAH9NEsdBb8IsmplciDUY/9u4VXV2RAaJjXWfQWQv2DeoG1j2wzuniwtvQ1erY/9l+CvYVWFUaTQhhgLKdwrs7+5vmT7jlcpG3ZGpDFVoVCdFtWPxUHobdd0GZC4tSe/gnQrfLRCHfGwhLh5DeQvFnJv60PbRctuoy0qb5iKLIDgOvGAi33UZxnJkUbjxneAPdr2lZKWZB4W+wPEpkp7gDRYhQ/HgLhX8Qu6sPo5N+RFXTlwu7LWReb6G+rsmvoeK09zrlJBIJwQnBBEYH2hR/xiBbXkJIH5G76C5xHzZAqGSag1+UyCazh1QOQ1+E3rc7f40zqCKEvfTE5e6/xoLaLFGU8xX4xVibm+4ZfQ/DTi0lunomajX8fNvPbHttm1eGFRAVQOKoRPwj/KnXi3tJiLN7iQU9boCLKoVFZ8km2HOXsBpvDpN/glkHHaste+6FnWYFsFEPKxKF8tQZ/GLENerMKcCC7KXwc18odO724ICKg95ZH7cVUeOEK4ZdFvYbO95gd/wt1PgdsZI4ILqhjyw9QmlGaYcP649H/mDzfzY3eTyrIovsqmwaTFqCNH2dK/6UEdD7Loga2+Hj/BsCNw+/mfO0nxJTfZ5VJeoX5kfCiAT8IzqnOeX9Ie+zdvFa6/eHig5xy5qrORn3H9t1UpMJywKE2vv/A/LXwuZFonnSjCRz2kFU5XkAlEUINbMrxR/AsR+O8ccjf/h2U7QZlutLqRRfFrVfn6g+BJqtrluCRCIy/d57D/r0EbZ5M2aI5yw2fKPGKF0rxZuBhfjLqJgssgHDh7T6HADkroTfJ0Hp9hYPdQsSCQx4CGKnuDwkNlDYguuNesrVNqY4slckw24eRniPVmYvdxKkcimqEBU1BicZf10A9429j39N/BehklCMDT7sSNZFIe/bkzMD5xCY0tRBoK1ITQWFAmprYeNG8VgvF1vFQYMEUWc0CkUxtI34+3rW13w8+mPr9wsWwFXNiNej+0fTf6Gwcj9ZfhITJkJVocQENmUKrcRfVdP1q1Qu5d7ce5n74Vy3xmlxP/Nl4s9ksjUGWdYHkb0jmf/FfHrObCo2mfX2LG7ccSMSqW+7eQy+ajAz7u3Po49CrDnlITghmKg+UdToalCZ98/RAdGiYfrcjR03mJJtIvO8RohgLPcXi+Kv96yeHBt/IxWhqV7Lhm4Mk6mp1ac38Dfx97+E6gw4uxKJtISakJ2UB25tNi+sI3HgiwO8Ev8Kx9acQYqchJAIorfMgn2LO28Q+b/Cj2lQ5EbRpTG2X9cyERTcGy6ugYGPN3vY1LSp+Mn9yKnK4WjJUafHWIi/KrWS1CmpBEa7twnxJma+MhPZ5cK/uIl6p9cdMOxV57ZVO28T3vmNumdLzTWhyNasrxJnw4BHXefwuAOJQpC3GiedIyfehY0Xgqbjs0Z6JBeTGpNNba2wgAhQBOBvzhyzdNoExgRyw7YbOtxapy2oKajhx+t+5Oh3R6k1KxxC/ZyougLTRD5mVAs2ARHDIHFu67teizdBXXbrXmOBtgROvA2lbhS8YybA+CVinN6AsQE0xQQEGK1Wn9Va7+W6tgT/cH80oTFoVW4oeDoap78W+X6tIZWDegh1mLtzTcXB5lVHHY2QPjT0foTsynSCaofy0IDHuGf0PQB8ee6XLJm9xGtDM+qNlGaUUl9abyX+ZIZA58V4T6GhRhTC1IUtH9sSIoY1nwfpCpsuFllXvgJNiVDMG3RM7zmJlJpFBGp7oFbDoa8PcWqNd5wHes/pzVXrriJuSBz15oJcqF8LjQLKUKH+DBsscpSSF7RwfHjT9UnpVvEFghQOGyw+987gztwRMQL6P+Rec8jvk8S6s6ug/4Mw/mvxezLju2PfcdT/Q2r9MhwUfykTUnhM9xij7hjV4cPa9+k+Tvx0osnj+wv3A5AkH4rcrC5ugpDeMOJNiJnUsYP8X0TFfkGaqZuusy3FPgtZbDKZ0NXqOsUe0KAzYNQbHYrmJ8pO8MOpLykK/cl2TzIZIW6G9xq9PA11rnCrqLF9VhITxb8x1YL4Kw/9A6NE1yzxl/FjBpuf3+yzVo4Ah5ceZtnCZRQcqwRsBFuEfwRXDrqS+X3mt+p8EonjreOii4Tqb0aPj7h68KOMHNK2dbhlXLW1CEUFtE1lGjNJ3APDBrVpHM3CBcGrkCmI8BeFDIslHIiC+NwP5pI6OdXzY/EAKrIqKNhbQJ1arEOtzarle2HjAuEK4MPouW0BC8es5t6T95I4KtHbw/l/B0NkDKXdhhMU47kcU7lcWHgCHBCmWfRsxpzsuutsisHkZAiLqeFg0cFWvWdwYjChyaFuNWjUldSh19gauqw2n1F9nIpIkkNEvqczxR+Awt+5w4nJZOLw0sOoy21BbZbmT18m/jQa2zRoWR8ExQUx+KrBRPVtfcOHp/D7w7/z9flfo6l0z5a2JRgaDFRkVdBQ30CIKoT/Y++sw5s82z58Rpum7i5QrLgz3J0NGGNjAjPmvncu3zt3d7d3LmzI2Nhwd/cChZZS96Zp08j3x90kTZu0aYmxcR4Hx8ZjuWmfPM99X7/r+l1Vj1SheUSDn9yeXYaLqTgER98S/wWmd5nOW5PeYnY34c4TkRSAJiwBg0JlEWG9TV2d9b6w6yjiITxXh3ke73PiSzj4PKcGLWBNp4tR6RI4/Oc6TFmnGXj7QPyCPXcnRnePpv8t/TGFCRUnItwoemcFttF6sy34RQm7htbaXRhqxc8ypRY63+74OKkMpC0Hr9UKNaNTR7Pq5CoOFB6gW3S3JseYxa6SEuGJX5VfRVCcD1TEtMCFnS4kNTSVSHWjl11CM/2j1IkQ1hd0pZasyOpqa+l8E+HPZISDL0HMGNGPT9IgnyF6xNkHaVSRcImDAH3FIchZDPJvzu4znKD7iek8MKw3t/26mu3bIT0dItWRZFdkU1RdRLuwdkhlUhIvSHT7WNpCUFwQly+5nLB2YWh+rAAJhPrbEf6C0kSvq5ZIv6/1gzCZYPM1YNTB9KzW90uKuECcpz4HFlA77oGMd4kIyLVU/FUWVbLs3mWkjEyhy/QuXh6gLTXlNehKDeiVXhb+TCbY8xBIlUJYdpbgTjDwQ+eOrauCP3pB6lwY8lXLx7uD4E7I+j7L8VIAE0ql9bvY94a+GA3eyw7WFGh4t8u7DLxzIJr+VqtPS5B19TQoPwDTM133oQVrYc00GPSJqBADOPGV6HPX9w0ITHX+WiaTeDcYda3Lym9/LRh9ZJUCcOglOPQKXJQJgamo1VBRISoj7sm6B7m/95cQUdLO1FapCVc7qBioKYDcvyB2rLDKDO4EwU72Dy1YD4Vrre+jiVts+7aOdmzPztqZwmK0ufdMaDfo/bxzY+n2CPi3onGLjxEUFESEf71LgbzYpuJPKpOCh3rc35V5F7rKpiJEgUYkbwWZxLvdmxm5/0oK1sHuByC8H/jHWDabTCaUAdXoZLVUVQnhQJOv4dW4Vxlw2wCmvOPeXrQypYxb99tWfJsT7ZT6SOt9EpIOo9rQ19VXSbkcUq6wSWBRqUQFm6moN0GSaCpNBZQGbKK42HGi4ainRjHi8REo1O61rw4Kavt6uOhwEYd+PUT8vDGAVWAblDiIQYmDznpsUVEwbhwMqf2RjpF7CQz5vzZdp7EAzuZrhQA1eXfr1jKhPcQfV3J6EWy7VVR6xI23e0iUOooSbQkFmgK6RnV17ee7iXXPr2PXJ7s4WnCUOnUdYar6eYYmC3IWie+JL6MIArnrRKl/K46eL+YqYbWLf8SdOsGRI7Z/d0RSEkyZAkuWwOjRMPOHmazIXMHn0z/nmt7XOPV5F318kdNjW/7QcvZ9vY97z9yLOkLNkSIx0MY2n5bxmYU/OxV/IN7xe77ag7HOSN/51kTpitMV/DLnF9qNbce85fOAc0P4M8cqpVJROd4SxRnFHFl0hE7TOhHZ2X3CYHVRNcUZxfiFtC3Wf/i3wyy7ZxkXfnIh7ce2p+xkGe90eoehDw5l3AvjABHPpqYQTnwGsePEfM4dJM2C2PEWJ59hycMYlmxNmpXLQSGpw6AzotX6Wd6d3qS2wdLxvPD3D0Im89DqtS0kXwIhXfFXi744elkFmX8d48T/NtJ9TnePCn/txrSj3Zh2/PknZMQ9S0FIHnvDb6BnjBsy0BwR3k/0lGstMj+4rAYM2paPrTwmMuYTZzQ7MX9/6vtEBUSJh6a9odZX/BUXwy+X/8LBnw/yqPZRZErfvN9qK2tZdP0iOkzuwJRrW7kw7/5ok03mjNKAADsBmcpjsOdhSJwubDl7PAHxk8XEXBVtbZTuDvq/LSoX3dyTSSaTQfpd1GSJZuzbtwsbhobCnxmDzoC2ROtzfdyUgUo6TRUTw2pDJcghTO1p8dokfl91la0X/UAEQeRJzh177CMo3g793mxb9c/ZEj0CMKKqkiE3hKCSBCKtlbL59c2YjCafE/5W/d8qgj/cimq+CMp6TfiTSGDcOqhugzDsNCbo8aTrgy+txLwwqdWZUIWFkqfJIyE4gQvuvsCr41KoFQx/dDiJFyQyPmw8aKKpPtTBKvyFdAWZi79TYb2g9wsQPsC6rfIo5CyBAa3s12eogT/6QMzY5sWhxjSXSOQNYieAPAgUQWQUZ5AXsZtaQyLV1YNRpXhPGTmy+AinN51m6ANDuV65jDUHoL2jWzZ/FWyaCxd8Iax7W8OJz8UCNuUKq/Db2A7WESFdxbzAVc+Qrg+45jqeongbHP8EOtyMLLwPaWlpROwTwl9dI+HPUGcgZ0sO/hH+RKVHuXVYCn+F3QzzfI2oNPM3CquqJvNMowH+Hir6ig751lpxcx7XkHSxqIAN7W6z+aeDP/G08TLCOwxHoxHWUX4hfvS6uhcJg7yTgNVQ+HNrFbo3cSAWjB4NK1dKuaXfnVRUGTi6N6XZir/QlFD3jK8BMpmMtLRmLJVbYNR/RzHy/0ayvt4BONANU89rroGff15G7AXHCXT2HdIIm4o/AHmwSGbRVwmBx5sow0WvP5Pe4SHPjnkWg8lgI/oZDUZ+nfsrcX3jGHJfCy4vXqDrrK6EtQ8jODQYmaLBMz9pBsypc1+rGBexpevrrNuyjpg//8fUOVMJ7xDu7SGdczT3fKn44Fs6F9YScItr3RgaWnsGBYnkgea44QYYNgzy/dew4n8rALjjjzsYmTKSdmGube2RNDgJk8GEOkK8I44UC+Gvc4T9nrtJISJmklWRZXe/RCJh/XPrMegM9Lm+j6VqMCQphN7X9iZ1VKrlWPP79lwQ/vz9rdP/3675jZOrT3JHxh22zxGg8EAhf9/3N4ExgW4V/i76+CKMemObW3spA5WoQlWWXoTKACWD7xts8/sBxLp590Oiqtxdwp8yRPxxQG1FLb0Wv0BRYm+02unuGUMrMVceKhTWCl1vaEbnhT8XYzT6sId2eF8I70tgfWarXlZJ6MT+3HhPd4ISvDNpLC2F3LCfOMIezlRO9azwdzYBGZlS/GmJgy/B8Y/h4gJQOX5zp4SmNHsZs/BXUgITJqWhjlajr9H7rvBXUcvBnw4SFG/nvtJkCzuzLvdAl7udup7Z5jPS3jsxIAXGbwR9JWyYA6V7RPXfsgEgC4Ch30PkWVpI5f4lhN5EOy8QN4t+IJ4rBdG3EJUUjeQjOHFC3AvmSsqGwt83U77hzLYzPFT+kNvH1VZqTMLqJizATsXf7odFH6YxK1oOsu55HPQa6Peacx8skdr/HbYGba7oxdVSj8C85cIqacB7Z/d5bSXlUki5FMlh6JT7f9zb7/+44TIjlcMqUYX5XjlD8tBkViwzoPUX1jletfoMTG1dhReIyuONVwkBqeuDzR+rCIIebcv6dhnaXNg0j6mdLuGbI5Po8nkqfjI/ah7zfsWZKlTFmGdE9v393E/PKnjnd+uijz4vuf5D1YlNf2+9nhGJJJJWvmfl/sLB4Gwspn2BuPGW7P2FOz5nZcT9JHAV1dWDKTpShL5GT2yvFnonuoHjy46z7d1tDLprUJNeGk2IGS2Emmb6Dzkk/X7ofJeYY5QdgJyFkHajtT9T3nI4vRi6P960Z5OzlXybrwWDTthi/pOoPi2SX6KGYwztRUFBAeEqMZFtXPGnr9Hz+fDP6X1Nb6Z/7r5FurZES+mJUiI6RTRJdDRX/KkMotqsifAnlYl+lxL5edHPHagT7DophKpCAaiTlVnuGYW/ghlfzPDIsEozS8lYmkHahDQiOgrhulgrlC4b4e/QK1BXAT2f8si4PELRFkBis36aN0/8gUfJzoZbv26+x59Rb0RToEERoEAV4p55p9FopKCggOjoaKTmiForkUgkluqdoCDQ1mnJLMukc0RnZC74vgcEwNVXS4G2zwnMwp95nPR7o/VxDJNRtDhJminOdxXRw2DyjmYPmdV1VpNtUpmUQwsO2VgH+hIdJnWgwyQHTlQSKfh2Wy7+PvE33/z0DXN+mENO+5zzwl8baO75YlD6o1dI3VLx1/D/W/qaS6XQrRvc9dXTAMilcqp0VVz929WsunpVi88wbYmWTa9vIq5PHOkXpzd7bN/5fW0q8+b1mkdaWBrj29uv9B2dOpoPpn7AwATHcbjpX0wnvEN4E1Fq+me288FzocefeWwN1ySBcYGEtQtrIvoBJA9P5roN13nEBlQqb3uHt/bj2nPTrpssfw+KD2LCy6KB7aIji/hox0eMbz+eu/rPhwlbQO1GhxKTydqyJzAVnUHHtpxtFGuLuajzRSiDlFR16EWVOsln7hVzxV/Daj9vaEbne/y5mHOhgXWQ0irGlMnlxPWJQ+7nWQ34x1k/svLxlWg0IggwVAXDD90LOa3IjncFVZmiQWheK/r8abKhcKOwa2uJdvNg8FetqjqrM9Q12dZQ+OtzbR+mvD3FoxWarSU4IZjH9Y8juUnC57s+t1gBAGCoBr/I5n8mpxfDqimgE00omxX+ZH4QNRjiJsCFx0STcZkf9H4ZBn109qIfwO4HYWcje7C6Ksj8WlQcuhmTyUReXh7BwSZLJtiOHfaFv87TO9Prml6WrBxfIWNpBs8FPMe+7/bRq/B5ep/4in7xdvrf6cpE0FDqhLBeuB6yf3Gun1LDicLZsP0O0W+poeWbPYZ+Dxfn2/Q48gbmBYlWKxbZIckh+AX53rOj26XdONN3GpGaWVyTfodD2xC3YjKJCi+9E9XcjZFI4fRCKNzg+nG5A5MJircSFXgamUFElGoNteiNeja+upGvxn6Foc43umI3zKD0OFJ525KEOt8pKs+dpSpTJMSc/K71n+UBzEK8Qaqhuhp+vepXvp/+vVfGMurJUdx+5HbUkeqW7w1VNKReLiojWktIFwjrKX7/Z5bAnkeFxayZ4u2iz0TVidZf20zVyZbPLz8Iyy6Ak975ebeJuEkwqwhSLrfMX6xWn4U2wp8yQMnE1yfS4yr3VkBnrsrk4wEfc2TxkSb7zMKfsq5RxV9VpkgmM9bBiEX/PIHWlzDWCYv/Bpit9epkZVbBw4Oc2XaGP27/g9wduZZtDSv+LK0HMr+Ck97ri+sW1lwIu/7jcLf5315djcM+OpmrMnkt4TX2f7ffDQMUmJ8vbY2/5O7MJXNlJpWV4vzAQNias5Vu73Wj+/vdWzjbSYo2w5k/Wl43NINZ+KutBb2ets1L6ipFkpPcdxxhHq54mMsWXObtYTikqLqIa367hnv+vMe6sWyfcBMw+G7vSoD+1bvon55N3hN5tBvj2sqvfwvNPV8qx83k+IA5Lhf+4uOtIldz/f0aUlxdzNHio8ilcv6e+zeBykA2Zm9k0+lNLZ4rlUtZ98w6Di041Oqxjmk3hsdHPu7QFjk9Kp2b+t9En7g+Dq+RNDiJgCirH2P2pmy+u+g7cneK9665L925ZPXZ8J4Y9/w4rl5l33FEHaEmaUgS/uHuW+CufmI1uz7b5bbrHyg4wO8Zv7M7fzfIA0TcVe3GtkMmIyxOs8RlK2orGPb5MKZ/Px2dQYdEIkE7cQZFyX0tvw9vY0/484ZmdF74+zeRtxx+S0J1Zgmy+mLP3OIyaitqPRrgMxlNZG/MpvBAIdXVJnTyIgKl4FdXJnrieBJjnQjc5P3l/DlZPwrLn7I9LR8bPQzazXXKhmNpxlJ6vt+TG5fc2GRfQ+HvHNCWASEyfLTvI65bdB0rMldYdwR3FpmBHW92fHJ1NuSvgDLRoNicUdqkvx+IoJmxPlvQr0E2W/t5wmPaFfR5uWn/ropDwkIs60fXfEYLBFVtQLpqHOP77QSE3eeIlBHM7TnXRiQZdMcgJr85GYnUt1IRVaEqUkakEBgbSHDxGBJL5tI+0o5V08D3YfpJ5xa1w34Ufb4kTrzKSnfDwlRRhXs2tLsa+r4OphaemRJps1W+bqdkJ6y/jATFGsA6US49UUrZqTLvjasZamuhXcEdvDjqLXrEeMEGs3ibCHbt+2/bzr+kFEYuavm4k9/C8tEioO8t1PEwu5y/8p5GZrQuuDQ6DaUnSsndlUtdddMkFE9QeKiQr8Z9xcFfDlJWU0aFphYTJqu4s/shkXThajZcCSvGiv/XV8ORd0QSwtlgcLKCsqYQKo42CXx7lbzl8PcIKFhLgELcI3ppFdXVMPCOgQx9sIWqZzehjlAT0SmCnKoc3pYnsy69n33hz46Q0GYSZ8Lgr217BqfNFz38GtvZaHNh2+3OJZSNXQkTWwjM1FWK+1DvRLKZryD3B78Im+q4hCDxvq9R5tgIfxKphAvuvoD2Y9u7dUhRXaMY89wYEgY0nXeMaz+Oq3tdTYhWuI5YhL+DL8EfvUVCkjMuH+dpGyYT/BQiquYbYK7408vLbO6ZZf9ZxvKHlrt9WO3GtGPeinmkjk61bLMKfxHWZMSJW2HcarePx6P0fqHZXtpaiimOXkCZervDqr+IThEMuH0AUV29OBdugTVPruHriV9TVSXWHIGBsCtPBEm7RLrIEv/Im7B6SsvrhmYICLAuiyzfhYL1sGoyVDRNZrCLMgQmbBBuBq4mewHscmxJfbLsJAsOLWB91nqb7b7qXASw8LqFLJixgC/3fMlXexv04z78BqwY41zLFy8ytHwFtydpqOxR6XPtP/4JmNfVrhb+JBLoVz+t7Nev+WPNRKgjOHbnMZbPXc6o1FF8Mf0LNly3wdL7rDmRwS/Yj1v238JFnzTf62/jKxv57sLv0BRoXC5a1FbWsvPTnWhLtOTtyiNjaQYAOz/ZySsxr5C7K/ecEP7sVfy1hKHOQE2569x2ltyyhHXPrwNAX6tn06ub2P/92Sff7Ph4Bzs+EpXde77aw3cXfkfZqTIKqwsB0ccVXanz6962IpVBt8cgeTYA4f7hyOuT7M1JfOafv68If+bkKG/3Dz9v9flvQh4I/vFIZCr8ZcFUGUooXb6HF27bwdy/59J+nHsX3WYkUgn/yf0PRoOR51+txiitYVk1aKceIcjPw5ajQR1h2mEIakV1Sew46P1i684xGlq0BzKZTOwr2IeJpi/TsPp+0jodFOfUsPq+JSQNTWLQHWffeNwdaEu1FOwrQJsvnrjBfnYsHZuj3TwO1FxNuCGAOKzCX5OKP6Mefu8O0aNa10uptdgTEAPTYNjPopePB5AZNFC6ix4dRQbUrl3w7f03c3P/ZgRUHyJpSBJX/nElej3U1jtznnUvjdYIazI/UYEbPfLsPjPxwpaPMZmgYI24RwKc7AnoanQlkPUjYSHjKVcH8Vnto2QuSiRtbhoJgxKY+9dc74zLAaufWkP8lhJO9pnp2cbHFUeFTdegj4S9Z9/X2p4w4GxguKagPnHE+7lXKhVITUpkyDGgp0pXxdR3pzL13aleG1NtRS25O3LRFGhIeSOFitoKRvkdxd+/o3iXHnwRki6Bdle1fLFWYbQG5vJWwI47RDJS+r3Nn2b3UgZY2l08A0Ytafn4yIEwq8C3Mnv0Gqg4DLoyApRC+DPIRMVfr3m9vDasytxKMEG5vJxKaTZKpdb+Irt4m0jS6v8OdLrt7D40uJP405DG9p6WAWZAxrsQlAaxY5q/rjMJLpGDYOZZCtCexmSE8kPCCj1A9MdJDBYZwFpFNpWVnh9SVHqUwx6CN/cXc6kbfoc8GizOE2cIC0plqIdG+S9FIoH210Kg7TrUIvzJKqnU6DGHLY4tPYZEJmHcCy5K7nOAOlLdpFKmYcWfpfeSTOXeDHdvkHZds7ufW/ccm5JfI7nwBoqL+5NgJ48vNCWUKW+3ss+7hxlw2wA6TuvIxnoxLSgI1uaKBMs+sY6rVFpFl3shbrLD3onOIJEIgUGjEcJfaCjiHZ2/Akp2iKRab5KzRPTF7fqgSPpoxOIji7nzzzu5pOslFjECIH9fPtWF1T5ZkaYt0aItEnEMm/YD7a8R7XO83VuxBX6Pnc+zG1+la3AdJpOpzf29ztOU2opalJs2ExiQSkBA8+162sIdd8CcOZDUihCCUqZkZKqIcTS21n1106tklmby9pS3kdpJlo7uFt3i9ctOlpG1PgtVmIofDvzAF7u/4MlRTzqs9jNzqPAQG7I30DWqK0OS7PfyPPzbYRbPX4y+Rs/A2wbS7bJu+If7YzQYSbwgEYPOcM71+AMhaK59Zi1p49McxtlfDHuRlOEpXPnHlS4Zw/5v95M0NAkeBrmfnHuy76G6qOUfWnZ5Niq5iqgA+/Pkza9vxmQ00e/GfpQcL+HYsmNIJBJb4W/brXDqB7hM63xP9LbQ80nL/0olUmICYsipzCG/Kp/E4ETUezbR7mAuWu3F7htDK7BX8ecNzgt/LsanX6qRF8DELQAEyIOoMpRQFiRn8vV9vJIJJJVJKakRio5CovROTyeJpPWT5bBe4o+z/DkAFCEwtvns1G7R3QA4UnQEvVFvyV4AUCrFYqSyEjQ6BYcWHEKhdn9vubZyZvsZvp7wNQFzAyDN1l6WM3+KYGLafFDY/53nFQfy8GNiYfPJJ81YfRq0ogePN3op+YVDctOeBe5AIpEgTb0U0/D7iJdICQmB8nI4dAh6NCqMytqQxebXNjPk/iEkXuB7gQhNtZHT4d+hMAQjV04CGtzHJiNkfABhvSHKyUbvRVuE/VoLQQpCusLgL9s67NZRVwYrRkPa9TDoE898ZmOiR8GcOsq3y9FL13Bc+ifyrM7MvX+u13q6NsepddmEFuSgUZ6g3BBAkCnK7uLE5eSvhOyfocONENFf9B5tK2UHoCa3ZeGwy91O9zd1K6cX0zkshEMMx0+qptpYgabOC35qjUgclMiDpaLfnuYpMR6ZUS2yPSVSUWXV2r57zjC0gc1mzCiR2BHZxuQaqUw8x1St7IHnS3PIxOmWnqiBx4Urgrniz5v8Nu83crbmMOrAKABkxkD7WZQyFaRcDmEuCt42xmQU7gQg+gCaiRoOM884Z1ddfVpUVkYNF0LhP4k/ekLCdCTDfiY8PJy06DQWjDrKx68lUJVqe+j307/HZDRx+eLLvTJUM+agjeV+ip8o/pzH/Qx4t8kms/AHUFxVAQhXjxt33ohc5f4Qhr5W9FJvuLZff916br6nmLzKELEmqSkCzUkI7uJwTfNPZEy7Mby2+TWKglY12+fP3UgkEsLDm/aHcpa0CeK5+1d9W9bAQNiVJSr+XCb8RQwQf86SwECr8AeIxJKL80EZ5twFTi8S6++Ot7hetOrxX+jxJCjt95EzB5PNFRlm/r7/b7I3ZPNw5cOuHY8LmPPbHFZmroSvGsUxooeLPz5OqTqN0hMJpD+WzqYXNzHkPifX1eex4Oj5UnG6goi9q6ntOBy12vXCn0rlvOi3+MhiJnWYhEJmPy6oM+h4acNLFFYXEu4fztNjnm5yTE15DWWZZUR0inAYX5zyzhQmvj4RiVzC02uf5mDhQYYmDW1R+Pts12e8sukVbh9wu0Phr+usrui1erpdJmKh6giRJJEwIIFr1lwjtp0Rx3p7DdIcja0+y7PK2fjSRmRKmUPhr9fVvQhNDXXZGGb/PNtyv57ecprEQYmoQu2Xmp0oPcGH2z9kScYSDhYeJEodRcYdGYSoQpocO+u7WZZWMaOfHM2oJ0YBULhWCH+R6kgIHCEKjdwp+tkhJlAIf3lVeQDIc08TfuYQ1VUz8IUka7Pw5196hgVXbmbA7QMISvd8LM77P4l/GG1tLO1prul0H+nZL1ET2p2LPrmI6O4tZ3q4isozlRz69RAVORWU6YSiMz4wCMnJr4W9lqcx1IrAS8lO91w/qFOTLFZ7JIcko1aoqTPWcaK0ac8Xs91naYWMhysebtL01peI6BTBpLcmcTpZZKnbVPyd+h523gN2KhvNZGdDmOoMgyPfZdPyUxbhr4nVpyIIej/fsuhztux/Bn4IgMrj1m1me1EPIJVKSU5JRSqV2lhAbN8uJnTF1dZVt7ZYy+HfDlOa6UOWcYheH38/8DcnDuSyu/1VbOt4EUgb/Qxr8mH7bZD5P+cvvP8p2HqTcz03XUFdlRDzdzrufYJEDv3ehmQv9q2QykEqx98f5EYxuajUVTLisRH0udZNgfCz4KIfr2LXhP+wqkcHUt6OpVBT6JkPjpsorFtdkbG552Fhu+RLVVvNsfEKxsQ8i0QiwV8mApZVuipKM0s5tOCQUxmC7qTOUIehvgJPZvQXGZQSiaiidWfjcBDvluRZZ1fFMfQ76Pe6c8fmLRdWWUbf6KvYGLPVp7nH35qn1/BRv4/Q13ruPWgmfVY6A+8YSGWtKBuTG4LsV/yF94Wh3zqfRNJajDphH727UcBSIhE9Be1UPTShdDdsvhYKVjd/zMlvfcsGtiUkUuj5LKReKeYvyckEqYLoGtsRmVFtY9sInul18eu8X/lmStMefQajgezybGr0NZbFubfteM4jUMgU+MvEs6espsy63V/hkUTbJTcu4WnF0za210qZkuqCOGRGNdHRiFYRywbAGQ/3qHc3GR/AbykObSSHpwxHioxq1TEO52Y5vMySm5fw+22/u2uUlufL2cZfzM8kpVrLoULR66q5vlROYzI614fcCcx9vyzPT6nCKvo5M3fI/J+wSndH4lRAipibOfheRgeIOFPjuf3AOwYy6c1JXul35AxVOvHD9kpy+lmilErQB1RT1reMsDQnxeHz2ODo+RKUHMaBETdTmNzP5VafrWHFiRVc9P1F9P2oL3UG++0ZlDIlL4x7AYBn1j3Dl7ubJkHv+GgHH/b5kPy9+c1+nkwhY8GhBRwsPEioKpQ7B93Z4hh7xggL9b0Fex0eo1Ar6HdjP44sOsLuL3bbbUFlfv75svDX2OozolMEd2TcwYBbHSd+TH13KkPvd13rBHN14a7Pd/HpBZ9yau0ph8fO+XkOL218iYOFovVIYXUhq0+utntsbK9YKs9UUpwhYo4SicS24i8gSiSVDPrYZf8Whxx5B/4cKGz4gdhAkWRrFv4C585gx5THqdH5hi5jtvpUSA1oS7QYag1e0Yx846fxD8JodM3kzi3oNcIiK2cpN/W5nbT8+6kr9nxF0OnNp/nx4h85teYUZTrx8Lg62Aib5oGx7Y2v20xtMawcLzz4W8Kgg18TYc+jzl9/6DfCRq4FpBIp6ZHpAJYHcEMa9vnzRKbr2RCaEsqgOwZxOkIIfzYWrr2ehXFrm802LCyEjuHbuKX/7eRuX+S44s9TqJNFb5+GC5oVY4TNqAcwGo1kZWVhzPkDcn6nf3+xfeGeNfg948fwz62Zhx2ndOQx3WP0uNwLPdKaIXtjNhtf3khetvhlSkwyVPJG0TVlGIxdBZ1udf7C3R4T58iaidSdXgx/9IXCjW0YeSPkAaLXUnMLeUUQdL4d4saf/ee1FaMBCtYTyj5khnrhr9YL/mpOUlMDBlktSMTi32OL7MB20P5q1/Rj7HgLDPyg5SBP1s9C6PE2F3zGdu0jmEwm5EaxStHoNGQszeDHWT9SsL+ghQu4h/Lscg79eoiirCLLNovwp9eKvq56N1QmVmXC4ddFbzZNtuuv3xwHX4aNc53rV+opdOVw6kco3Wv5PhqkGjQaYXOkLRWLF0/T/+b+jHlmDJX1ATm50YHw525kKuj2iLCDbEjRZqjOce4aEYNgxEJhA+eIrF9g45WgzWvzUL1Ct4cgeZZ1/mI0Wuy9Gwt/ly+63O3VfroqHbrKpn3E86rySH4jmaDng9DWiGe35X5aMx02Xe3WcZ2nnpwlsHameA434MJ2l5FUdC01GmsFbcmxEk6uPul2sSCuXxxdZnRB7m9dc2m1ouoK6tckoT2h13NNe32e68j8wS/SYb+eYL9g2vmJYObWglUOL5OzJYczW8+4ZYiAzfOlLXwx8gt+uPgHi/1wnmk/BpOBKHWUpS/pWVG8DX4MhGNnHwx19PwkfzUs6dJy3+gLPoPxG87KctQhJpN471Ues7s7Sm2/4q/T1E70ua6PTzpm7fnfHgrWivHarEn+6Ct6Qvs4M08+x5repRy+6zDpM9Pd9jnaUi0mo28Kt2eLo+eLzihHGxxDnX+I14Q/o8nIA8tFX83RqaMdVvwBXNfnOh4Z9ggANyy+gXWn1tnsTx2ZyqinRjl0gMvZmsP2D7dTXVTNn8f+FNfpe4PdyrDGWIS//L0tvrMzlmSw7N5lSGXWtVDR4SK+GvcVFRv2Ab4t/DWu+JMpZIR3CCcozjPVXYY6g+Vn3GV6F1JHpZK32/7aQVunZWe9rfVnF33GksuXcOLOE0zvYr+wRF+rZ9Xjq/hu2ncYDdbvg9n+3PyM9wi1RaA9Y0mIjA0Qwl++RgjXAaEKkEh8psefxeqzg2h7lDoq1SuakQ9FGP4Z+GrGEiAqlHY/BNm/EFL/nNblFLJg7q8c//t48+e6kPgB8cz8eiZJQ5KIqhrD+N0FJHT+EUb8JiwxPY06HgZ+CF2aqeAxoysFdZJYELmBrlGiX1xLwl/ZyTIO/HSA2kovCKWtwCw02FT8qRNatMgoLIS9+WN4dt0Cft09z7LIaVLxt/Zi2OqBHnft58HoP2wrN8P6iKCdBzCZTJSUlCDZcSfsfog+fYQGWXpG3BTmly6AVC61mTD5CoPuGMSdx+9EGykyTZWm4KYLPZlKWOyFtkK0jBoM0cNEhZsjaouEJZsDC5pWIZHAtEPOV/J4DSMsH05E/nPI64W/Kl0Vy+5dxg8X/+DlsTUla30WqgoR8JMgwV/hoUi+Kyus4icLe9cW+rmy4y7Y85jrPretJM+mWCb6QXQxTeT63tcTqY4kbXwaF39zMZFdvJNpkbU+SyQHbbJmKUpNKhGML90Fi9LgWMvJNK2mMgN23gsbr4CFySJgdzYYdLDtNiEmtkSvZ2HIN75l9anNhQ2XQdaPpISmcGfqJ3TPegetFia8PIG7TtyFX7D3GhaUapqp+Ks6AX8PE2KCO+n1LKRcav27yQSrJol7yBlUUZB4UfMVrO3mwvBfICD57MbqJczzF5PJxE8nPmRPynzOsB1Pr3kvW3AZ1667tsl2c6AgUh2FpH5ZbOnDUVsEtSWeGuK/m+psyFkEGtvqsXcnfkqvk59hKk+0FNOveXINX47+En2NeyuOB905iEt/vtQyVy2uLubKX+ZyMPE+AgLqBeLQ7tDt4X+eVW/7q2HyjmZbW/QJHQ3AnsqVDo+5cceN3LDtBpcPz0zD50tbUAQoUKgVFuEvs6a+v1+ci8QoiVQkjqrP/vntUPiTqaC2UPSsbg5FkFgzuQOTARa1F/NbO5gr/kq0Jeg96JhzNvx+8+8Ufi6qWWyEP2XoOWHrq0iagSrlEl4c96Jbrl+nraPidAWfDPyEZfcuc8tneBtHz5eyglpkdTX4KU3I3FBA6ww/7P+Bnbk7CVIG8fiIx1s8/ukxTzO762zqjHU8u+5Zm30JAxMY+fhIh5aTB346wO83/051cTVZ5eId3S2qm1Pj7BLZBblUTllNGacrmu9X7R/pz7gXxiGRWp+9UrmUnC05GMvFQ/pcEP7Ma5KqvCoqcyubFcb3frOXn2b/RE25/SSb1rD9g+08H/g8pzefxj/cn6tXXc2gO+3HKvfk78FgMhATEMM1va9haqeptAtz3Gs1b3ceOo2OGV/OsIkzauvEPzraLwDWzYLjn571v6NFej4hep8HivE2rvhT6qsJKjpJVYH325eA/R5/3tCMfLts6DyuRREEE7eCfxzl2hzK1GdQlEvY9/Ve4vvHkTbeM4uWkKQQel4psj9qteCnjyI6dhx4sx1ZhxudO84/BiZuat21a0vg4PMQ1g9S5zR7aHPCn1n0KimBAz8eYPmDy7l+0/U+2cdtz1d72PjKRtRD1ZTHltt642vzwC+q2eB4QQFo9UEcKJ+J2WFHpcI2q8pkBM0pMHlpAdHfiQpRF2Ps8xoyZRCBgZCeDjszRGC+WFuM0WREKpFiMpnIWpeFQq0gvr+b7fBagV+wH37BfpQezRB/J7jpQXWVoqKutVUvJqNY8IZ0sb8/7VoRxMBDQfWj78GxD2HYj63vIeoqpAro9yZGabrF6tOEiYIjBZQdLfPOmJph7d0LaK+VwSgIUAZ4pr+fUQ8/BUO7eaJSz1MM/spnKrvMi5OR+gf5v2kJyGQyiBL2JN4iaXASM7+eibynHDJAZlIJMdgfUMVA+v0Qfvb9cpoQMVBUo2tzRUXm2faGkyrg1HcQ7kTvyIj+4o8voU6EoT9AaHdCVaHMTLmeE2XWahdv8evcXwmMC6RkqggCyAx2evxVZEDpXtEH2JOYDND7Bed7Lpkx6h0nrwR3En/ONfY9Jexrx2+xbPo7azHZUb8TprkAjaY/QfVTw8yVmeTvzWfgHQM9nrhkrkCJVInAtETSYHE+YYNHx/Kvpv11kHZDk++BWezQ60XwRKWCnnN7knBBgserhM5UnuHX41+jjIhiivwVj362LzIsYQw/5z9Phn4lJpPJ7u+jYQDXF7lyqajc+m62+PuQ5At4Qv0E7cNabtHRBGOdeO83JGIAjP7zLEcpMH8XmryDIy+AGdnN9+3TnBLJSEEd3JNgJJULB5bAVLu7I9RiTmnCRHF1MTGBMQDs+nwX655Zx6ULLiW2Vyt7IruZy369jF9O/QJnGjkXjXUsdPsSQQPfQlFWw6EnVqMYrqDrrK4uvf773d8nolMEIckhRHT23prBG+x6fwt9lq3i9NQbgTiPf36tvpZHVooKvgeHPmjpodkcUomU2wfezk8HfyKjJKNVnzfkP0NIHZVKeIdwsv4Wwl9yiHPJDH5yP7pEdmF/wX725u8lKcRx88Kp705tsi0sLYyHKh7i8GEJPIDPVHHZo7HV59pn1rLt3W3cm3MvQfH2n8+FBws5+MtBxjw7BlXI2fnMB8YGkjQ0iaCElisMz1SeIUgZRL/4fk3e3ZW1lbbPPCBxUCI3bG2axFNwfwHaOi1+umLI/lUUyHiY6V2mEx8UT+/Y3gDoDx+j8+ZfqUq/FHBftbOzmIU/du1ixSMljH5qtFfGcV74+zchkVqaSz+7fD7ru35Kl6CneefrR4lL8k66itnz1l9lxOsFqGYf/uYqh9qCzA8OvQopl7Uo/PWM6Um3qG4kBjcV88wVf8XF0Gl6J4ITg33Ws92oN6Kv1fPmlDcxtDNYJvzotfBrHKReCUO+dni+2drzsstg0fen0RskqCMSbNcqEqnIRnVR74Rm0ZyCjPeFHVfMSPd/niPip2BOLevbF/YfEj9Xo8lIWU0Z4f7iJvnfhP/Rfmx7rvjdyaoDD1CVX4Wh1kBRZTkAKomdScmW6yHnd5hd1nTx3BxbrocTX8KsIvBzUNXnSqGl7ADkLBT3sb3ecMY6YQcqD3DdZ7aFzneiqAWZ0QQmKUiMjPt+HHFBnl+ktET6bWPYskRkAnrM5rOuAuImQFBH11zv5Pew91EY/HXzWdWxY13zeWfL6mlMV2bzM7vR+YgPPkBoaiihqaEcLjoMCOEP6hdSQWnQ5yX3fLAy1FqN3rCKq61IJDDtaMu93kxGUSnWUqWop1EE2vwczIk31dVQcKCAnC05dLqwEwFRnn3OnVp3ivC0cJRTAwnUdiHImNI0jhk/ES4pcf/8oGQHbJkPXe6DdleK+WPHVroQ/N4NZGqY5KDC1GT0mUSBVqGvFJa8emt5SlKImNtqldlUVmIR/vZ+s5fdn+2m97W9zzrwYQ9NgYb93++n3Zh2TXqam4W/8Hrhz8/Ptwpv/zXI7FcPK5UmjPJqTEYJGo0alQrSJqSRNsH9yapLbllCTI8YS38es7uGUh9BlDnOumW+WCOM+dvt4/Eoei0c+wACO0DihXYPGdl+CNLtSqqkp8ksy7QrlpVmlpK7M5f249q75bvtCvR6azxiUEovxnV3XOXoEKNBVJkrw0Xgs9OtENbbpeN0WPEHVtHPoBPvocbvjCNvweHXYOpBCHFTILSH46ojuVROhH8ExdpiCqsLLcKfTCnDL9gPo973WuWkTUjjLv1dXF1ztWeSEd2AvkbPlje3YDQYXSr81WnrSBiUQGi7UMY8PcbnBX5X498ulsKkPigj7SQxe4DHVj7GybKTxAfFc8/gFhILG9AuVFRIZZVnYTAakNWvO+q0dXwz+RuShiYx9tmma9TA2EA6Te0kksvLHQh/uX+JXrd9Xm4Sw+kZ09Mi/E3t1FTcaw6zKGVeg3g7+bA5Glt9po5OBSAgxvE6acRjIxj95GikcikVpysIiA5ApmzberDb7G50m+1cJebF6Rczo8sMymvKLdsqayuZ++tcVmauJOueLEJVoU5dy1/hD4pEuFwvEhndja5c3GvBnSC8HwMTBjIwYaBld2j3RDZ2m0zH8Bj3j8UJzMKfYd9BNv96kjHPjvHKOM7Nt5gP44se5Tboq6G2xGK9WCevpEYv9+i4/7jzD15Pep2a8hoOqz9mX/LthG1OhOVeFFRyfocfA0R2cnOc/FbYdhntN9C1izwALjomgsEtMKXjFPbfut/ShLchYfUaX0kJRHWNoscVPTwecHOWPtf14Y4jdzD7wtnM6T4Hpay+N4dRBx1vg9gJzZ5fWN/7u2fKfj69MImLOr9p29+vpgBy/xbBUk9MxmuLRH/Movrs78KNwsKtpX4KLkIikRAbGyu+pyYTGA1ER4PUpMTPJL7L5oCERCJhyrtTGHSXZ2xInWX5A8t5I+UNysoqAPCX2ZksRwyE5EtaJ/oBJEwXVUD2vpf5a2DvE873XHKG0l2iz6cjK8Aud8FFx0XFjJdRKkEmlSA3BKGWq6mu802PjKixPclNEVbPHhP+/MKFxXS6EzbPziDzA3kQ0EwAwyzy+AIBKWhlIngqV6opry1Ho9NwZvsZXo1/le0fbPfq8NQKNXO6zyGpaiaAZ/q4GWqgpqjl45xFFdmyilC2H35QwaHXXPe5bmBXxTJyQ3+holrLsT+Psej6RZQeL/X4OO4+eTfzVszjwqSrGXXgEBeUv2r/QKkczHMPd+EXLVwM9GfRPzV6JEQOcbz/j76w7IK2X99b9HkZLspA4hdmmb+Yk9pqFKdtgtdD7x/KteuvRaFu5bvfSQoOFPDnXX9ycvXJpvvMwp9SBAgs1X51lcJSuGSnW8Z0nkaYjFCwHoq22my+ZuHVLO0dyMmo9+0LHm7CaDCy48MdZK609hy0Cn+R1jVJXZUIQv3TkEiF9fWpbx0ekhijpt/xBYw/kEm7UPsVcgd/PshPl/xE0SEXvlcbDrPh+qiVGOoMbHh5A4d+P2HZFtDWZXVtESAFXQkc/1jcywB7/+uyd3uzwh+IwPvCZLE+bkz8FEi/z3suJMCbk97kh0t+sOmd2PPKnty06ybi+/mOQ01D/OR+xATGWCuqqs/AgRdE0o+PU7bzUU5su4gOKzow9rm2JRzm5eXxzNNPMyA9nU4JCQxIT+eZp5+muLyYWd/OYuyzYy2iX2VuJaUnPD8ndCeOni8hAzpxqtdFqCI8H4dbeHghr2wSFedvT34btcL5JoPxQfEsnLOQnTfutPk3yVVySo6VoC1uWk6nr9FTlVeFyWSiuq6a/vH9SQlJaVqkkL8Sjrxp13K4Z3R9n7+CvU6PtSHZG7PJXrIH8G2rz8YVf11ndWXKO1OadbJQ+CuQysX+hdcu5K0Ob1Fd5Jl/pFQiJczfWkQSqAzkeOlxKnWVfLP3m9ZfUCJ1/7oLQFcsWiqc+t7u7rC0cArbDaQmwAUtflyAWfiLuXUWt+y/BYlE4hXN6Lzw52KkUh//kf7RG1aMslgvGqkka+Npj76oA2ICCIoPQqpSkhu0lFPR73JSmeKxfmn2B5Usgi8t9Rg89rGYxEtaWRUY2P6ss/kbWn2aOeeaKStDYMA7omeeA4xGUdUIEJzYlerEW6gOGstFFzU4aN9TsGqC6AniCUK6wYUZ0OlO8feizZDxnsd6v0ilUmJjY5GeXiAE6jNLLQtAlVFEHxr2+et7fV+PZEO3hg6TO3DBvRdQVh8gDbAn/KXfB4O/bP3Fk2ZAnxeFFW9jsn+B/U+KgL6riJsIE7ZA3CTXXdMdbL4WyV+D8PeHibtLOHqNhsDTgRz48QD6Wt/qsVFbC3qpiGZ4TPhzNUkzYcpuiBrq+Ji85fCjGjJbTgRxOwPeZV/oAiQSCb/L7iXi5Qje2foOykAlYe3C8AvxTv+2re9u5dX4V1FmKvlu1nf0yfoMqF9InfweVk2BymPu+fAf/GFBlOt6P+pKRRBOm+v4GIlMBOV8sUfUrwmw4XIA7t48kx0dLqFEl0+XGV244vcrvGrv1LiXhoXK43DwJag66f5BBCTBzDPWKr+tN8Ef/cDQiv7LA95r3jo8YoCwiz1HscxfpFKSgoUFUI3SVviL7BJJ8tBkZAr3VL3G9Y3j6lVX0/mipkHv/CrR4y9ULir+LPdTdbb4fZ7+zS1jOk9jJGJev9e2asicqKqXlVky/ff/sJ/3ur1HzjYXJnQ1QiqT8kjVI1z4sbXarVgrFieKhsLfsO9h0lY7VzjHkfnB+A1CwHdAWBjEVkzFT5tKWZn9YzpN7cTF31zsNoeahs+X1qKr1LH8geUc+OmAuFZwHksyFrXYh8ou/jHCGnjMcph+EjrfLrZnvA+nW0gqdpIWhb+gjiAPBIOdoHHsWPG7dGeybOEm+HMgZP9md/eVPa/k0m6X2gSafZXyrHKeVj7NikdW2O6ozIA9D4sEYB/HcGYpiWWreOnoS/gFtX4+//qrr5KSmMj6557j7sOHef/MGe4+fJh1zz1HSmIir79mFbSri6t5Lf41Vj+x2oX/Au/j6PliFnjUzmtuLmNgwkCGJw/nP4P/w8XpF7fqXJlUxkWdL6JHTA+bKlaJRMK9p+9l2gfTmpxzat0pXo17lR0f7iBAGcDaa9dy8u6T+Mkb3FN6rUiG7/u63Yri2d1ms3zuct6a9Farxmtm06ubWHnnQiRGA3o91LWi/sKTNK74aw0mk4mUUSn0u6kf6sjWX8BkMrFo/iL2fLWn9R9ej0Qi4ca+ov3V57s/b/H4F9a/wKSvJ7Hg0ALQ5kP+KtcmzzrCPwGG/Qxp8wHQ6DSsPbWW5SeWi93183hfsYU1C3/+YSrC04QY6Q3N6LzVp4sxGFwULHIXKZcDEoKrhPAn12lYM/dTau4axKQ3PBPEHvHoCEY8OoKqKqhViCach9IeoFv3mR75fLuE9nDOg3/od6DNab0PkF4DJbtEYM+/ZZs9o8lInaHO5qVqtvosKREFIx/0fJ+ghCCu+vOq1o3FA5xYfoJTB09x6oJTxIbHMqmD8/dWSYkQ/2QyCAuXIh3xHnMHV0LGy2D6j1i0pFwKwV08Z6UoU4m+CGa63C2sW1uycHMRBoOBkydPkhoYhyx6BCgCLVmpSkMkyE7YCH++SPc53ek+pzs7PjhO78wvGT0wuuWTWovJCEhsv599X4XUq4T47ipUUeKPI46+J6r9Ei9yfIwnkMhAIketBo1GSnU1HPt0J1vf2tqs37ynMRqMbLr8XboFRkCn27go3UNWpMc+EiJBzyfFd9wTyIMgerRXPPDt4e8vFgt1GgWoQVOnIbJLJNdtuM5rY/IL9iM0NRSFWoHJ1MAS3B8oPQUFq0X1uDsY9pOoKHeV7Wb+alh3MVzwRX2fUTuEdoORC13zea4mtCcECGsgtSKAGoOWytoqwtNSLYsXT2LUG8n4I4Ow9mHU1DQSaszkLIHdD4qEHQf9hlxKw/eNVFlfaehC0XzQx667liepzoHcPzGEX8DJEhWpqamWDHGtMrtJ8NqoN4IEt/T4U4WoSB2VandfQbWo+AuWR1ME1n6R6iQYvcy+nfd5XI9EAv3faeKUYLaaqpOVWe4ZiUSCyWjCoHPvmluhVthUoTas+ItqZgr4jyGqmUpkxDotNBRKS0XCZpgdPSeqaxRRXd33w7Ksj1JTRY/iVqAMUjJ/y3xySvzhXSiL+Jvp389jePJw1l671vkLme2YJVJhudmw1970TJEA5ALM6z6Hwl9gO5Gk2jhGYTJ5xr9YpgLtaagrc/oUTaGGfd/sI2FgAklDfGNeDCCVS+kwqQP7lPv4belvXNrtUoanDIfwfjBxG6gTWr6IlzmY/hyTv55CV4Mf5dnlhCS1kNzegNdffZXnH3mUtQYDgxrFNq+sqWELMO3BhwC45957UUeoGfrgUBIG+v7PpTU4er4cfv0PEg9KCRg60eNjiguKY+XVoq+qJ1BHqul7Q9/mf7faHMhZDMHpdpML2oe1b1vf1HqG3D+EPjf0Y/vb4u/V1RDi/O3sMRpW/JmMJr4c8yWdpnViyH3Nv0tBzGtGPDrC8nejwUjOlhynn4vaEi27Pt0FEug1r3m76s2nNzP317lMSpvE21Pettk3q+ss7vzzTnbl7UKj0xCgdBxrXZe1jmXHlzG983Qh+m28XKyjky9xasxtRuYHybMsf80sy2TkFyOJVEdSeH8hfnI9Xdd8jCS7Hbzk/ST9mhrAZMKUX4C2JBj/cH+vaEY+Xp52HpfT80no+YQlg7JaXUbKDePpMqOLx4dSU2MV/hJCfKuhs0P8Y8Wkr7Xkr4blw+H0ohYPfXTFowQ/H8ybW2wzwM0LKr0eKiogvGO4WxdTZ8Oer/aw9q613LDoBv7zVwMbvayfYP1lUJXp8FyzzWdEBFiSIQ6+BLsfEOcDRI8Q2ZQeEt4AYe+hEb7mSKRi0u8psQCorKwUDdxH/wkxoy2ZnzGVk5jbcy4xAdZqt5WPr+S1hNfQVbkpOH4WBOrSSCyexwWRjV7EJbtg/RyrPU5ryfoZfkuC0t2226UKiBzo+kWvoRY0dipOTSbYdZ+oCPU2gz6BCRtsMp96zevFJT9c4rVqLnsYdAYkfgqUxhhm+b/DoyMebXqQySSsmBtaalXn2LUUcZqsnyHjXZC66GehK4fDb4rJryOiBsPopd7tFWomZwlpmv9DLq3FWCueZVU6D3qpOaDX3F5cv/F6QjqEUKmxpnX6+wNdH4TLqiHEdb1KbEi+RPTmcRURg6Df2xDZTM9HX2b0H9D7OQCC6itxtQYNei8VDNdW1PL9Rd+z5a0tPLf7dtZ07cGpgJ9sD+pwo6i6iPbQd6wqUzhBlOyA/m/DxC2tO//MMth6c/NVoeciFYdgy3wkBWvE/AWsVp+NKv62vb+NpxVPc2rtKbcMxVBncOiQMTJlJNf0voY0/75AA+FPESR6wHrRGu9fR9p14mfegDCVWPzUyUupv43odmk3bjt0G8lDkxtfwWXUlNWQsy0Hbak1ZbxJj7+6KjjyNhR71xbbbRj1Yu3TDKEROo7E/x93rboancE7aw7z86W1yBQyEgYmIIkQSSyVAbsA6BPbp3UXWjYI1jZIXq6rhNOLRdW5PMBltv8tVvyBda3TUBQ4+g4s6SJsxd1JeB9RAd/+Gru7M4oz+OXgL2zLsbZJqC6qZtk9y8j4I8O9Y2slQfFBXL7oclZ3Wc07297hUNEhsUMRCBH9nUri9jZSRTAaE0x9eirL7l7m9Hl5eXk89OCDLNbV4siLaxCwRF/HQw88QF6eiOWNe2Ec6Re7qX+kF7H3fCnbcZyg4lNeqfgD0TNTIWubNfq2nG28sP4FFh9ZbLM9Z2sO297bhqHOVpCI6xPHhR9dSFzfZu75wPYwM09890vbZufZHIkXJNJxUhp+aiG++qrdZ0MnEm2plsIDhZRmtj7xw2QysfjGxXw29DPqqp0rb1RHqHmo/CGnbH235WzjWMkxMsuaxmPjg+KJD4rHaDKyM7d5q/tDheK5mB6VLuLj/d+B8L5Ojdcl1LvzmBPESrWlmEwmAoLre1cafEPqqq0Fmb6Wo3d/wF/3/eW1cfjGT+M8HifIT2Sj1fhVEDxpiMNMWHew7vl17PnfHqqrTdQq8oiRQbdT7wtvem9y/FPYcqPj/XUVYhLfFguwiAHQ+0WIGtbioSq5Ck2dhoOFtv3j5HKIqdd2MjPhsgWXMfE1z2caOcPwR4aT9mkaerneYisLiEVH1o+A4ywls/AX3bAgrNsjMOgzSL7ULeN1iqU9YNM8KD8Mxz9znRVcGzFnfrY/9SRfzviKQYnW6bkyUElwUjA6je8If3/c+QcLr11osWpq0kej4jBk/SB8u9uCX5QI1NU2qHwsOyCCse74Xf09DP52kME1bi30btqn01uo1ZAR9yw3r5vCwbCDdLu0G8oAD3iwO4nCX0HqCzeT02WsNejamDO/w7pZcOA567Ydd4nvZdUJBye1wMglMGW/60RhvQZ23t1yr1hfIWcJCaVPo1ZUIqkTESWNToOhzsC659ZxaMEhrw7vt8O/EfKqko2dRyGRiH6V5xzqeJGkEtzJ8TFH3oFd99dXLPsugX7ioW2QVnFo8TGeVT/L7i93e3QMcn85M76cQa95vciqOk6lej8oNY0PErZmCg9ZBtfkw/6nRK/otlC2F459aE0sakjVCdh+R9sTYrxJWB8Y+TumeKt1lFn408sqya/v9wsQ0SmC7nO64x/unkaef9//N0/JnqIip6LJvuv6XMfn0z+nR8B4oIHw50v9WP/FNKz4q2j663MbWeuz+GTgJxxZeMSyrUnFX3UW7LgTTvtoxfbZsmU+/JYghCwHRIUrOB7zCkvPfEV2edNkuOrial5Pep1l9zovPHgKo95IXXUdlZXie16irBf+4loh/JmMQgRSNVi0Fm+FtRfB4ddFMmJrrJ+bwSnhD8Q8eWlPIdyCSFY11nm9Su3rvV9zyU+X8NmuzyzbQlNDuXbdtfS/2TftrCt14t63tCDQlYHeR1WHRgToCumrgt0TdpN+ifOC3Ccff8xohcKh6GdmEDBKqeTTTz6x2e6pSjRvEvvMbRwecp1Hhb9v9n7DvF/n8fvRNs4161l+YjkPr3iYnw7aJs3t/WYvS29biiZf4+BMeHj5wyS9nsSbmxvZ00ukwu54wxxYc6Hdc1ecWMFDyx+y2DG2FpPJhFol1km+Lvyp1UKIu7/wfqa8M6XV15FIJKTPTGf4o8Mx6AysfWYtH/T6AKPBuk7MXJnJK7GvUHTEGvfyC/YjIKplN7QduaJHab84+8UsAxMGArDtzDa7+wG0dVpOlp0EoGtUVwjuCJ1uc63DVnOsnQG/isC4OUHMYDKgqdOgVks4OPIWcntMaOYCnqO2FkxISL56pN2WA57ivPD3byPjQ1g3mxCFWFzrZRUebZYOsO7Zdez7eh9FlRUYpTUkySE4639Q0nxWgdspWCsacjta4Jz5Exa1c9hItFlU0dD1AWHp1QJdo0QlgyW7rAHp9fO2gweb7PIpIrtEQn9AYu3PAYiK08tqICDV4blm4c/SPwNEIC/tWs9YlTii812QMkcsqLZcD6W7vDOOjA9g+52WBaDRaPWONjPswWHM3zyfwBjf6ZWWtyuPnK05ZFRvJz9kCRXSk7YHpF4Ol1a3vW9e9AiYdhjixlu3bb0R/hoCtYVtHrdDUq+A1LlNA4MSicgGDevt+s9sLSU74NBrRAQWUabezqaiPzhectzbo7JLbS3UycrRKQqo0dvpxxg/BXo+LfpAmulyD3S5V0wy2xJYkSldawWoihaib/oDjo/Zca+oEPAFuj/Gye4HqdKFgk6sXjV1GiRSCSsfXcmBHw54ZVhHfz/KuufWoakQi0+pUYlaXf/4L9wg3sXnGs0FQrJ+hBOfu7f3Tls59hEcfBGwBr70Ug3SoABSR6USGOvZd4zCX0Gveb1IHpqMpn6uZpNcpM2F8kOeFWzC+grrr5jRov+wPQGvOTrcCBcXigSxxlQcEdUalUea7vN1/CIgYYpNsDlAGcDzcUeZvFODUWudG7Yf255Z380itpd73D9iesXQ9ZKu+AU7ru42z6Mswl/G+/CD6twUXc9VMj6ABbE2lUn2hD9NoYbtH2znzPbmq9HOhohOEYx9fqyNvdnroz5j3O5cUorni77rASkwdhW0c9y3/Jwmdjx0uh1Mjku8IyMl+OtE5eWp8qYVu6oQFUHxQQREe6g1QyvI3pjNcwHPkfm96NFYKRPj7xTRTKJOYyRSGLkIBn5o3RY5WFT666vgjz5Qts8l43Va+DMaABNU1/cq7HQbXHQclB7orVe4AQ48bzfhMipAuBSZ7ZVBvNOThyUTnGCn77sXyd+Xz7J7lyHNEPMyi/C37wn4McC+44uPkZr5NjuSYevELfS4vIfT5y389lvm1thZh9lhnlbLb99+C4gKp4/6f+STIr+rqa6WYJLJmyYxu5Glx5byv73/a7EKqyXahQn7/sbVXv1u6Mfcv+fiH2FNwDLoDHw65FO2vrPVcs7pitOYGifwa7KgIgM63iLW5nbm4EuOLuHFDS/yw/4fWj3m/L35PB/4PFEZGwDfFP5MJlurTzOSNsYuO03rxJhnxqAKVaGr0mHQGWxE2RMrTqDJ11i2FWcUk783X9jmt4BZ+Osfbz/hYmC8EP625jjuX3yk+AgmTIT7hxOl9oIDXVgfiBkDJiNqhRqFVFTAlmpLLfN4Jx9jbqemBowKPzrfOMorLotmfDDKcG7T1i+3xyjdCdk/0yUkkRnhj5FSeAvZT3zK77edXfZIa7hp101MeW8Kp8uENcABbRDMzIWON3tsDHbp8wpcWmXrzd+QwPZC/Ik4i6w0k6nFgJRZ+DtYeLBJ1pRZ+Dt0CPZ9u4+Vj69s+1jciFFvpKJWrNDN1aUWZH7NBjiL6hNXfK5/Ro//E/do/7dhyDdndx+0EolEQlJSkni+5C2HYx+glNUir+/SWlxeS3lNefMX8TLXrruWWw/cyibjm2zreCGbK35uepDcv+29kew9ewe8C/3fFRa9rqbLPcICr/HnGmp8Jxs0bwXs+g9xQSeQG8T3sHRJKS9HvUzmKsd2u56muriaoiWbKZO8yv0FMdz31322Bxz7BFZPg9QrbXsrRg2Fnk/B8lGwvpXVwLUlULS52Uz2ViOVQ/RwCHDgx28ywrH3IddHhCt1IvLwdIwmOQqTsLqq0lUhlUm5addNTHjVO5lyRxYeYeWjK6nRihm7zOhvXUTte0JklJ5LbL8LFkQ5rjwe8zdM3u3RITlN5ldw+A0AS58Hg6wKdVocVy69kg4TOzRzsnup1ovopyUgB3DiC/i9KxSu89xAZEoxH8hbDvv+C7WtrFpXhoAq0v68KHY8zMiB5NmuGasXkJgM1vkL0CG8IzKj2qNJh32u7cPsn2bjF2Q7vzCZTGSVZ6Gt01oyta09/hIhdoJtJc953IsyTKy1jFZrK7Pwp5eXUV4/za3KreL3W353qz1gRKcIhj00zKalQnmJEpU+lujgUDH/lgdAzCiR6f5PpN2VYs3TjGAUEQFqneiDeaqsqfAnlUuZv2U+wx5q2fGmLdisj1qJf4Q/Pef2xBQpfsfViAXoWQcx5WpR6Z82H7r/t/mK/1ZgFv5qaqDZ9kBdH4Qp+6yJbZ50qMn6GfY8AtVN74XoAPEsLdTYJmMa9UZqK11TFekqig4Vsfn1zUhzxXvZkmAU3h9SrjgnrD4r4i7kwSKoMzhnFWimvKICZ1fNMUB5/YNZFarCoDO4pU+vt7D3fNFV6ajcdQylttyjFX9mi9wBCXaSxFpBu9B64a/UNg4Q3T2a9uPao/C3WohW5FRQeqKUqjwxYcsqF4ltySGNbLYPvgRLOkHCRdDlbrsxmckdJwOwJGMJxlY6nATGBZI8LBl5uGjs54vCX22tNcSrVkPBgQIOLThETdnZq09jnxvLbYduIyjeGlMd++xY/mv6LykjxPt348sb+aDXBzb25PbQ6Kyucv3i7Vf8jUwdyaz0WYxrP87hdczXSI9MF9+Pvf8Hv3dv/RqorfT4Lwz7ESRSJBIJYf5inlJaU4q/PwQXZBB+ZJPXWlM0xJxY6NdgCeINzeif82T2EaRSH/+R9nsbLjfQLrYf16Y8TXLRfAw1dehrPPetiOgYQXhaOGcqhPCnMsWJwLwy1GNjsIsqSiziHBHRH/q90fZ+H0ffhV8ioKr5ipsO4R2QS+VU6ao4XXHaZl/X+rZGhw/DwZ8Pse7ZdT5pqfDJoE8oulgsoGyy8c8sg9I9zZ5rrvjzOeHPjDJEVHt5EKlUSkREhHi+9H8bLilDIvcjIACyIz4n8QMVV/16leX4itMVrH9xPTlbczw6TmfQGoQgHKZulN15ZtnZ95/Q5sG2W+Hkd+LvYb2hw/yzu2ZryfpZZIP6gvVTyhwYvwGNrCtyo/geVquqieoahVwl9/LgrJRnlVPx4zJicsTi1CaQD6DJhILVoAwX4tnKibDvabFPqgC/SBGYas1ComAN/DUYsn91zT/CjNEgbKHtIZHCrGIY9KlrP7OtGGoJlBegkNUh0YcCouIPILZ3LMGJ3snAHvXEKG7ceSM1fvXCn6mB8Nf1YRjwgVfG1WZU0RDaC+rK7O+X+bmsB5DLGfw/mLAJgACFmB/ppRqvLbrP7DjDGylvsOvzXVQbhGgfrGowx4geCZ3vhoiBnh2YsU7Yi476A0JadnZocm75IfsVBFK5sItV+FY1hNP8Go907VTr/AVr8Lph25yyU2UsuXkJGUs92+epvLaclDdSUD+nprJe+bMIf4nTYdRilwXtz+MEKZfBhI2iV1g9SSFJDA6bQVT5JEvFX1haGHOXz6X31b09OjxzYqLFkcRQ6/MWze4mIgL8zcKfnYo/d2OzPmol0d2imfnVTIyp7TFIaqkxiYdSpDqyhTMbsO8pEfS2R3gf4VLhoud3w+oijcbxcciU1sD7nsfgBz/Ri9YTdLgJJmwB/6ZzGrOgWqApsNn+WuJrfDvlW48Mz1k6X9SZu0/dzbGOx4AG65J2V8HQb8S72cfRxk7mpVIY+824ViX4hwQHk+fksflASIgQYyQSCbfsvYUJr/iGvZ4rsPd8Kc4opvbTb4g4vcdjwl9ZTRkZJWJ+5KhKy1nMFX9nKs9Qq7cV3E0mEwadNVEgrF0Y9+Xdx8j/ip7ZZuEvKbhRgmv8ZJFwoIpx+LkjU0YSpAwiryrPps+nMwREBXDVsqtQDewJOFH17AXMyWMSiRB4Dv50kB9n/Uh59tkn5kukjkUik8mEUW+k6yVdGfXkKNSRzd+Ue/L3YDQZiQuMIz4o3u4xQ5KG8POlPzO/r+MYmqW/X6TZRlgq3AFk7rHsbwmz3WepVgh/kaf3kHTwL6oqvNuaCYTwF3bmAFtu/ZL8vfmAdzQjH1epzj0MzaZg+QAypSWrOKg+VqKffzPTP53ukY/X1+qpPFOJQWcgXT2CCbuKuIGvhZ2Rizzw24xRL7z4y93U10gZAcHpwvqjGRQyBR3DRfZo4z5/ycli4l9TA+n3TOSOjDvcM9azJHV0Kvr+Qky2WH2aTLB+luhl1Aw+K/ytvxS+lbi2QshJDAYDhw8fFs8X/ziRTYoIoCkM4kVn7j0CUJlbyYqHVvhUVdfRJUfJ2ZZDjUlEbiICGyyETSZYfwnsuPvsPkTmD8c/gYx3IX+1e+3eKjJgzUVwspH1rzpRVKYF+UAGeEAyRA1BoQ60VPxV9q7kmjXXkDTYQVWaF4joGIHfLdeS0aUMsCP89XoWLikVontdheiJlbPQ+vsd9hMM/qJ1VokhXaHX88KSyZUs7QHLLnC8X652TwVqW8j4gNCVMXQM3wKlcczsNJvRqaMBqK2sRVPYXGTJfQTFBxHXJ44aoxD+pA0r/mLHQOo5VvHX/VEYu0JYHzZGr4GiraBrffN3jxDYzlI1cEv/WxhX/RERVcMpy9Xy1/1/cWSRZy0oJRIJATEBKNQKtPXCX6h/A+Evagj0ex1kjpqFuomcxbB8pLBWk7WyGaU2V1QpHn6t6b6qTKg8fu6KC3ETMYYPsM5fgD3axexJuZ4t2q8th9VW1LLjwx3kbHNPstKGlzbw9wN/N9luDkAHKYMw1IqHjMM+s+fxCp0iOvFK/1/pevoVS8WfMkBJ+7HtCUkOcdvnrn5iNZ8O/tSmEunx7ddzIOluAiPrn9f7noTvlVB5zG3j8CpVJ2H9ZXDKsTWbM8LfgZ8OsOLRFe4Yoe36qI1UVYFOLhafcqncUmXaIiYTHPsAsn9puq94G/wUKuyyXYRMZrWQczrwLQ+EwDSQeEioCukCkQPtvgctFX/VthV/3ed0p/14D/WFchK5Sk5IcghlkjLAzrrkHCAhOIFfL/uVvrV9KT7ifBXO9Cuu4Es/516EX/n7M+MKzyZDexJ7z5fghGBqJ0yjLLqjx4S/HWeENWNqaGrrEhPsEKWOQq1QY8Jk88wuOlLEs6pnWfvM2ibnyBQy6gx1nKkU9tpNKv4SpkLvF8Cghb+Gws77mlzDT+5nqfpbeKRtydH2Esd8BbPw5+8vxL/uc7oz8+uZhLV3jcVyxtIMVj+xGoBDvx5i0fxFHPzlIG+mvsmer/aQNiGNkf83ssVKMvO95Kjaz1mMJiNhqjCLUx09nxAtd+SeUsP3w/Y7xRoaqztEWU0ZcjkUpI/k4LD51NR6342xpgbktRrKj+RZCna8oRmdF/7+bWjzIH81xtpiijlCmXoHlVWeCyic2X6G1xJeY+u7W6mtlaA0RDAxejks6eIyD/w2Y6gWXvwHnrO/f/ko2PtE26+fOgcmbHCq95f5IXqg0LbHklQKXeqtgU+VhRCeFu6T9rITXplA2Y1lQMOKPxMM+FBk4zeD3R5/vkDkYLGA8oLwB1BjNqo2GaF0L5TsIDAQlHrxg2oo/EV1jWL+lvn0vb6vN4Zql59m/8SaJ9ZQi/j5RQQ2CNaajDDwI2GfeTYoQ+DC46BOhhWjocJNIj6IbM/cP5tW8MaMgiFfC2HJFzDoCPCvtVT8mRvV+xLKQCX6+GTKQ0VUz+4C22wBqwyFaQdh3DprRnPDZ6CzYm9wZ+j2kOstupIuhoRp9vdVHhO9bA06135mWwnrDR1vpdoYTVBZP94e8R2PDH8EgM+GfMbnwz73yrBqymqorahFqxerKJlRZdMv4R9F2T74axAc+9jbI7FPXaXo3WEyMrnjZAYpbiCwpguasjo2vbKJzJWeTS6J6xvHDVtvoPtl3ak1ichniNoHAnJRw6D7422rNFTFQLfH7Pe33f0QLO4gEtPORS74HFOPp63zF+CUbjfZUZ9xwrjasi2ySyT3F97P8IeHu2UYhxYcYt+3TdcYZuEvJjDG0gvEpsffrmb6tZ7H9RhqRX+wLFshJbg+T6yiQTG9yWTCUOe+wEltRS3lWeUo1ML2rFZfy/Liz8iMeZOIiPo5R3AX8b7/x9rBmkQP2mbcOMLDwb/WsdUnwOFfD7P+ufVucxeqaWMjn+N/H2fhdQupzCxGbgjmvg5f8OakN51fV0skcNEJYTfWmJCuYKwFrWv7UDrd589Mt4fgwiOOLejdgbEONE3vBXOPvxJtiY395KQ3JjHy/0Z6bHjOoCnUUHqiFE19aWWgMlC0cVg9DY57Z27cWgIz3mHGqee5f+/NzFvufB/S+TfcwOq6Ora0cNwWYLVOx/XzrVVBOo2ONU+tYfcXu9s0Zl+k8fMlIDqAyk790IbEeazH37Yz9Taf8Wdn8wkigc6e3WdgTCDtx7cnvEO4Zdu+b/dxbJlIbMmpzMGECT+Zn+W73AR5AOiKxbPPDtM7i0KTtgh/u7/YjXHhYsA3K/4a9/eL7BJJzyt7ogxoZTKgA/Z/v581T65BU6gha10Wuz7dRVTXKGR+slZdx0/uR5/YPgxNGtrscSaTiROlJzhQcMDu/mfHPkvxA8XcMchLRSg1eXD0bSgVQuZdg+7i7clv0yNG9DOVxkRRHZpArc77cldtLRS2G8hV+x50Wy9zZ/D9OvXzuJbTv8G2W9CP/J3Za6dCV+ixYQf7v1fSfU53t398QFQAA+8YSHz/eHaXiG35hiHQ7RFRneJNFMGiAiSsT9N9dVUi89pDVTxj241Fb9STGNzUKiM9HXbsgIN76hjRu5qA6ADkfr73Vb6uz3UMThxMelR9CbhEKvpFNENNjTWLx+cq/rrcI3o8tqaqyF38PQQiLiAgYLld4U8ZoCRhYIK3RmeXiz69CGVYAHVrReQmMrhBxZ9UBqmXu+aDApJE37eIQe4V3wJS4VKtGLuvUrwNlg2kh/IlS8WfNlfL6idX035se5KHefmZW49Rb6RGY8Igs9OzK2cpVGdB6lWgqN9ur99M0VbYdgv0eRFiHXvSu51ezzjed+RtOPoWTD8JASkeG5JDYkZCzEiqJEag0iaDstucbjZ2L57kh4t/IG93HtofhPBnU/G3MBViRsMF50bgBRBBsIMvgn8CpF1ru88/Xsw7YsZ4Z2wtset+OPYhXFwIqkhLdrNeFcidJ+7EP9w7iqzRZCSEZCp1lYSbbaN1pSLTOO06Ya/mSVTR4r3TFmR+0Otp+/uSLxFJCq2tIvRhkkPFvLbMaLWylylkLdoTnQ3zVsxDr20qOuRXCcud6IBozG2SLcJf1s8iUaOPAxu/87geiUz0B0ueDcmzLJuDg03opdWUlqsAMed6LuA52o9rz+WLXDR3bMTE1yYy8bWJlr8Xa0XFjMQkIzm6vtKw/Tzx559KQApcVtvs8ycszFrxZ7aBa8zY58Yy5tkxyJS+NV/O253H7s93U31lPxTGCC5ufzWDWxtXl6nsz+fkATDzDChcW5UaECASZH0x8G1h2SDxPp5umxgU4R+BBAkmTBRri4kN9BH3CztseWsL655Zx85tO1G0UxAXFCfEzLy/IbSnt4fnHHVlwlHA1DrBPTY2ltmXvcikHx7lT2Mtg+wcswW40M+PF557jthY6+9R7idn48sbSRiUQO9rep/N6H0as9WuKyr+avQ1+Mn8mk042H5mO3D2Np9m2oW140DhATLLrN9RVaiKK5ZYqzdNJhPL7l1GSHIIHSZ2sNp8hiQhbRgLM+jgrwuEVXfXB0XVlwMmd5iMTCLjYOFBjpUco0O46BNeXVfNLwd/4aeDPzGm3RjuvuDuJueeXH0S7fo9SCdOoKrKr8l+b9Ow4g/Ez8+VxRkjHhvByP8biTpSzcTXJjLk/iEExQVxx9E7KDtVxtsd32bwfwbT/+bm75Eb+93Ijf1ubPHzPtj+AbcuvZWpHaey5Ioldo+RSCTIzdXkJ74AqcpzrjyRQ2HGaYu97OU9bOeC/v5QWVpHVbkEb0te9nr8eQPfUwvO416ihkO/t1GEdEMmkWEwGQjasZGVx40eEf4iOkUw+S1R5v3UB++xL/kgW41zuLzXs27/bKfo9lDTbSaTmMDPOHX21oEnvoKKg6IcvhluGXALtwy4xe6+9Hod7cyvW3jj8RXcsO0G4vvb92j2BkaDkSU3LyFleAqXz2vdgtzcP0OtxmNZVK3CF0Q/iRT6vAz+iQTutFb8ldWUUWeoQyET2cn6Gj26Kp1bg2mtoccVPSgrA/0GIfxFBzey+nRl5WpQB+hyl+uuZw+JRASpGrP7YTAZfCNY6B8HKXPQFXVCbhCZFsYSI2ueWINMIfMZ4e/QgkMoXviZ1Gkq8vs3Ev6OfQBn/hDCX3PIlKIXoL0+WfZY2huihsKAd9s87laTdLGwe1T7js0qWDPIK6oMVNZWE+QXxIhHR3htPJ2mdSK2TyxnIs/QL2gadZndxULKZBIBNr9zrLpCIodDr4gKy8bCX0Cy/XmHrxA7DmRqkCrIKs8iW3GECv8Yqmt6EtbONfY1rSFvdx7H/z5Ot9nduFl/mN17IWZ8/c66CsB07lbH2SN5tvhzrpL1C5IzfyBVXmfZ1D5CPP+qpFbhz2QyUXigELlKbpNt7iqUAUq7Wdfmir/ogGhqhAZoTTIY8Svozr43y3lagVQOk3aAylYQ6Pt9NMV9ixi17yh6fUfkcki/OJ2obp7LEDQn1yn0EURH+57TiluQSFtMOggMhMja/ozel8H/3rU/twlNDXXD4M6ewfeKIOld/1FAubUFitMUrAdFkBCC7K1h/ONcMs6GtLrizxukXgW6EuHm0mDdLJPK+PjCjwn2C27gBgQ7P91J5opMZn41E6ncB9bZQMrwFAz3G0htl4o6on4dHZQGl2nB6COuHS1Q2/1JfpJ2p+qrHxgVM4ouF3Zx+twLBv+H3bslDD/yAKOVCubV1BAD5AGfImejzMQLzz3HPffea3OeVC5l/tb5hKe5/j3uK6x/cT2xP++mfMCVqNVnNw/elL2J0V+O5pb+t/D6pNcdHlepq0SCxCUVfwDPj32e58Y8R1p4WrPHXfH7FdSWC9VCKpEyLHlYU5tPXSnoypxqWRDmH8bI1JEcLT7KqbJTJIck88TqJ3hv23uU14r51l/H/+KW/rfgJ7dVSca/PB7jxMls/9bPp60+zWLwG8lvkDAwgUt/udRyTFlNGSq5CpW89Z7yEZ1s20UExVmfobUVtcj8ZM32AmwtfeJEIczWnK3OiZj7nhSJ2Z4S/uT+IHdc5BByej9Jy37h5JjZdOvpPRcukwl0OgjJO0LuOhnRszp4bSznhT8X441Gja0itBuEdkMCBCgCqdCVc6RnH2592vOByO0VSzgV/Qel0r7AMI9/vlOc+QN2/geGfgdhvc5enMhZKKouezzR5h40nToJn/9CZSKjbhzktYx7R+iqdOz6ZBcmo4le83pZd2R8IGxUR/wK4fZ9pX22v5+XkUqltG/f3vp86ShE4YAAUOjDLBmUJdoSYgJF5su76e/iH+7PjTtazurxFBoN6KVithasarDC3vt/cPJ/MG6Nb1RCOUvZPijZBe3mWp8Np38DqcI3hD91Igz9jqo1kFBSx5TEuTz5gImSaSUExbc2wuE+ghKCqO3YndLQ9UAj4e+CL6Bsj7XazxGhvWBWkXPivKEWpErADcG7qhNC/E26WGQ/NqS+ws5nKNsP+56kb+x17DvdgUG/d0L9lxrNI97p7Wdm8L3WvovKg9fyfXF9MF4iEc+Icw2JBCZsFN/Hc43kS8Qf4Jst7/FB9SMkxVyLVvsZZafKkMqkBCcGt3AR15G9KZvlDywnvl88Wm0o0ECoCUiBaW60d3YnO+4W38exy709EtdSvBnpiU9pN/oOy/ylQ7T4HlTLT1NXBwqRq8SHfT6k49SOzPnN9UGDvN15qMJUhKaE2my3CH/qaIvVpyUjVxEs/pzHs4Q3tagPUKop1oJeVkZVFYSGwsVfX+zWYez6bBfBScGkjReB0eJqUfGn1EdY1yjb74BADySaeZPS3aDNh/iJdndLJBAR4o+pqAPVFUBM02OMeiNVeVUoA5WoQl3bRLPJ+qg158qk+AX5UVUNlapDbCvLRFXSyVKF0iI77hQVVTNda+fZHOeE8Jd+r8Nd1/e9vsm2nK057P9uP5PfnmwV2bxM2oQ00ibYEUUkUs/3EG4jNfoa5v46l+s/uR6NQdMq4a+sDNqn3cujj11B5olPeOPbbykoKMdQqaRj4EBO7n+DuDj7wnZU+j8niGPv+SJTyDAiwyhTnnXF339X/5daQy0f7viQJ0c/SbCf/TnHsquWUaWrQuki94fu0faLPba9v43CA4VMeWcKEomE+H7WwoJhycNYd+26pif5x8D0E9a/a7Lh1HfCySSiafXZT7N/IkwVRnZFNiM+H8GWHGEqmxqaSnF1MRKJhOOlx6294+oJiAogpD7vU+PdZapdGlp9mkwm4vrFEd7JKoAfLzlOnw/7MChxEH/Pbdpz2hlqK2rZ9dkuQtuF0mFiB+QqOfoaPTs/2UmPK3rQ78bm+/YZjMLJR+aEW1Xv2N7IpXIKqwvJKs8iJdQan1t0ZBH/+es/TO88nVcmvCI2Dv/F8/3Izf2VgzqQW5nL0eKjhPmH0TOmJ4qYCIoTeiJpdUaPazFX+yUeXs66xyX0qhf+vKEZ+bhKde7hi/3WHBHsJ74IxdEqIvt4pvJj1+e7+HHWj1TmVlJalwfAjSELYN0lHvn8Fjn5HSxsD8WipB55ENQWCZtPV9D3NZiZ79Sk0WQyUV7TNNtYpYL27aEyMpXwKya5rGmsq/AL9uOBkgeQ3Crht8O/UaipV/Nk/uAXBcoIh+eaK/58rr+fl5FIJAQHBzd5vgQGmpAgQy0VE4uGdp89rupB5+mdPTpOR1TkVPBawmtseXktPbLeZ1DJ28QENIgO+IWDMtxSrn/OcPRd2Hw11DZoVD/1AIxf770x2cHfH6QmBTXVchT+CmJ6xPjM4hogeWgyhSNngf8ILkqZS0pIA/HXL1xYO7aEROJ8Ra7MDyZthQHvtG3AzV7bH7J+8n7PWmfQlUL2zySGHMNPIuyoquuqMZqMbP9gOz9c/AP6Wu9WTzXOoDxnCelqX0TYejMsH3n2bgIewCzI66VVVFfDe93eY/ENiz06hq6XdGX+lvnE949v0k/jnKamELSnbe8DvRb+7A+HXvXeuM6Wbo/ArGKCYnta5i8dokWGrl5eTm6JSASSSCSMfno0Pa7s4ZZhfDb0M36/+fcm2+31+LPcT2X7RF/083gWvRaqbYWUUFUoAHXyMso9UIRpMpn4/dbf2fKmtctVQZWYXyv1kWKNYjLBsY+E9d8/mV33w/rmq47D6pehpQ4KPjJXZfJ60uvs/95xr8C24mh95Awlx0s4syOXqgojOeHfcN2Kqbyx+Q3nL9DzKZHk58HYj1n488XAd1uZ+NpEHqt9zKfWJQA5FTnc9cddPL/uebGhZBfkrfSdPt0toKw4wNwg2DxmFWPfGtuqc8vKxH/bt4/l0cceY9vBg+w8kkOfCZkEDf2BgADH1axGg5HcXbnk7Tn335/2ni+9bxnMgZG3oPcLOCt3ql25u/j7hHh/aPVafjn4S7PHByoDXSb8OSJzRSbb39+OUW+k7GRZ2/ro1uTB7gdF8YQ2D9bNFt+desL9w5FIJDy79lm25GwhVBXKT7N/4vidx9l3yz5KHihpIvoB6Gv1mHLzUVaX+XTFn7+/uG/m/DaHcc9bW4/oDDoqdZUsP7HcYT/clvjxkh9Zds8yfpjxA3XVok+qzE/G0UVHOb7seIvnr8tah/o5NdO+ndbisSq5ip4xwtZ4a85Wm337C/ZzrOQY+Zp868bwvnaFXrfy5wCxjga+2fcNo74cxUsbROK9X2ocmX1m4tfBuy5LZuEvq/sUJr42wbLdG5rReeHPxRgM3umJ4zTlB+H37nDsIwL9xOzRIK2iqsozQafCg4Uc+vUQEqmEcoOYEMQoyqDyqEc+v0XkalFZYqiPKEUNEZl8STNcc/2AFFC1rGrlVuYS8FwAUS9HYbITEOxa/z48eNA1w3IlEokE/zB/Ht7yMDN/mMm+gvoAePurYfIOCEx1eG6BiMGcr/hrhMFgYN++fdbnS9kBWNiO3n6vAdBDdjHzes2zsQ4Y8/QY32mWboKQlBDw8yOxeB4DjLcToGwwW+5yD0zeec5kUVpIux5G/AbyBtVoEqlvVQlsv4vkqhcBMSk1mUxUF1ejLdV6eWC21NRAp9zHeWPkV5bGzFSfERV0zlr3VZ2AAy9ARTPvE3cLLP5xMLuiaa+/iqPwWwoc+9i9n98aoobC5QYOGm6jusyapVddV03e7jwO/3YYXZXnAxxLbl7Cqv+uEvdqQ3Gn+oywEina7PExnTWGWvHcri1utMMkRB9fTRrL/QvWXASleyzPbINUg0YjKjO7zvasfUpAVAAJAxM4oj3C92E92Z42yyrUFG4UgXgn7IZ8jqHfiL4oDe+DujJxb9QWOTzN51GGYZCHsG//fsv8JVgVhMIgEg2OFVjtPoc9NIxus7u5ZRgj/zuSXlf3arJ9SNIQru19LQMTBtpW/JmMsLQXbLvVLeM5TzOsuxgW21ZcWYQ/WSkVwi2ezW9uZtm9y9w2jKv+vIqR/7XOobOKxLPbzxBBaCjiu3pJGQz+ym1j8Am63AsDP2h27hQaCtkRX/DotmtYcWJFk/0RnSIYeMdAt1izNlkftYK1T63l4/4fYarTo5OL52ykuhWZpwnThOOHBzknKv4MOlg7A3Y92GTXocJD/HLwF/blW5PjlAFKn+v/uOLRFSyctZC3tr7FRzs/EhsPvw4rx4Kx1ruDcxJlzm98FQs1nU8gC5XxyxW/UHG6wqlzzcJfaKh1W0QEpKaKR8GuXXZOqkdXpeOjfh+x/jnfSoJtC/aeL+Z1iVQKyrPQ4dZlrUOCxJJU99Vez71LKmoreHH9i9y7zLY6d9oH03io4iGkcinfTP6GD3p9YNlndFTNVbRFtDIyW6OH9hAJ0F3uAkMNZP8MWT80Oe2VCa8wp/scdt64k0u6XoJUIiUlNMVhNVpZZhkb5n5AVNYOn3z+tZSomh6VzvDk4QAsOWq/Z15L9J3fl3439ePCTy60OL5JJBKu33w9V6++usXzDxcdRmfQOS06DYwfCMDG7I022w8VCXeV9Mj6/lMmI+irPZ/E2u1haCf6LIepRAZSWU0ZYE3i03o53GUW/mrj29FxknVu6w3N6Lzw929DIhdNfk1Gy4um+6a9fBj/tF2BydVMeHkC/6f/P1SRKqpMQuXZGPAjTNnr9s92isTp4mV18EVhbyKRip4TrsJkEsHpsgPNHhapjkSr11JnrLM0lG9IejqoKgvJfvl7Dv/muImuN6itrCVvTx61peJJ58i2wB7nrT4dY/OCUMeDPBCZnwjETjV+xJczvmzRq91bBCcGc/3G64m+ULQIP+erd8xEDBDPDHn9P0hXJrJBtfnNnuZRsn4kpGoRNYoz/Bk4h6t+uIqXI1/mj9v/8PbILBxZdITwDYtRaCtQNdR+D74Ii9JE9qAzVByFPQ9DbjPBwCNvwcrxzvcCbAv2bEn1laJ6UeZDN79EChIpgYEgNaqQ1FufVumqmPLOFP7P8H9eycA+/NthTq46yYXfXch/SlWcDv9aTOCrTsC+J85N4S9vBSztDplf224f+CFMdX0VhMuozoHcP0GbZ634k2morobRT42mz3V9PDqcuuo66qrrKNQUUqrch0Z1xPrMyF4AW28Sz+F/Av5xord07+e9PZK2o9dC2X4ktbbvxECjsPs8WewZi7yhDwy128f8yp5X8tn0z5jWaZptxZ/JAD2ebGrXfB73kzgdOtwERuucN8RPCMV1snJLxV/G7xns/GSnW4YgkUhIHZVKwgBr/5gzZUIYCpRFWPV5ub94r/+TiZ8MqVc0m5wSFgZFQStZXvRlk8oAgNCUUCa/NZmU4e6x8m9rAK3bZd0Y9OgYjDIFdUqxAI1SO7kArfNOyck5IfzJlFC6R8zZGvHutne55KdL+PHAj5Zt2hIt2ZuyqS6q9uQom6XkaAlle8uABu0HOt4MAz4QfR3PAaTtrubCM5Cnh/KscvZ/t5+cbTktnmcy2Rf+dFU6UrYvIOzMAXbscHy+KkTFxNcm0u/m5m0HzxUaP192f7GLiKxdBAScXc7enYPu5OgdR1l8+WIkSCjUFKKta6pSXPbzZQz9bChrT61t+4fZ4aEVD/H65teprLU+y9SRapQBSkxGE93mdLNJmOr7YV+SXk9iy+ktthc69Z1wP6qrfznLVCKxVBEsetrPKoZeTeexQX5BfDfrO9qFtXNqvCHJIfS+eyTlUWk+WfHXMFE1Z1sOf93/F0WHbRP3Lux0IQCLji5q02d0u7Qb0z6YRt/rbS3RA2MCnRLzDhUKwa5LhHO2v2Pbi0rhj3Z+xIlS8Tyv0lWx7pSwfLUIf9o8+DEAdt7j1HVdRtcHoH298OcvhL/SGpH8qfIzkrJnEad/8W7MoLYWMJnwU3rf2ee88PdvI7iTyCrueLNlIlMaISNudBdMBs/ckBKphJKaEowYwCQhJtDHVJ6C9cK6pcANmUomo6i43OnY/x5AIVMQHSCMrM9UNg2MdO4MUkMdypMZFGf4VnZ7zpYcPuz9IfHbhS94kDJIzCJ33genFzZ77nnhz0mUYTB1H6WRorzd3gLw0IJD/DDzB8qzPeCL5CQFFWXkh/xOibrBS1hXDjvuFYHxcxVzgKp0l8gGzfrJu+NpyLSDlPVbjVFSx8nAH1hwbAH9bupHu7HOTbQ9wenNpwnP3IlBUoBJVmNNQokbD+n3gV+0cxeKGgbj1kLafMfHaE9D+SHwc6OfsDYPMr8BTQMrj/B+MHkXtLvSfZ/bWgw6yF9NfOBBJEhQIEQ+jU6DVC71mnX5f3L/w7wV89DqteipRYJMBOPD+4okoZTLvTKusyJ6uKiciBradJ+zFrXeoP01MEcH8RMJUJgr/qosC1xPs+KRFTwX8Bwl2SUAyAxB1oq/znfA6GXg77jZu89SlQnHPwdNlrdH4lrK9iBb1puwMlubzUu0fzF5p4YeAVb7sSW3LOHbqd96eoQWzMKfSoXo09vj8fPCnzfoeDP0ex0aZP2HqITwp5eVWyr+Lvn+Eu7Jdk+QyWgwYjLaromvavcg43bnMrTmObGhthiKtlqrHP7J6KubfTaFhYFaJ0S9U+VtszDzBh2ndKTTdcNBIsHgVy/8BTi5AF09Gf7o4/EKh3NC+AO48BgMb7oWMgurZptlgGPLjvHZkM84ueakp0bXIrN/mk3XNcLRwCL8RQ2Bjjd5cVStQxLajWVaBRoTJExK4L78+0ifmU7x0WK+n/E9mgL7frFaLejqzT4aCn8VpyvQbN6Hf0U+x441/9kX3H0B7Ub7zjrTlex9fxOxJza6JIm5Q3gHRqWO4uBtB9l3yz78Fbbe9UaTkdUnV7MxeyMKqeLsP7CeYL9gwv1F0srJspOW7TqNjtyduWgKNYz67yiGPTjMsi+zLJPTFact72MLnW4X7kf+duxfpTKRHNOK9eTNS24m7a00DhfZFjYo1AqGPz6KqohUqqp8r0NCQ6vP05tPs+mVTVTmCoXyxfUvsj5rPRd2FsLfqsxVVNQ6V33rSg4Xi59pl0jnhL+L0y9mVOooquuq+WbvN5hMJm5cfCOnyk8RFxjHmHZjxIESGaTOhYiB7hp6i5gr/kq1Ii6uDpASfmY/FbuaJqB4kpoa8K/Ip+NPz7H5Te+KkD4cbTiPu7m8++VcoHuE/LQhdHvqUqRy998OmasyOb3lNHlVooJDpY+gnfF7a089XyBhCkzZB51uc/21pTLREyCtaXPrxsQHCeEsp6JpdlZEBBhj4tgx5TGS5wx2+TDPhtDUUEY9OYrjMcJrOtgvGGoK4PCrwu+7Gcw9/s4Lf87RsNdDjb6GKp11JViaWcqRRUeoyvX+6rD0RCnrnl/H4d1b2dZxGgskV1h3Vh2DI69D0SbvDbCt1FXBL1GwtV5oCuwA/d+BmFFeHZYNyjD8AxTIDSJDtcZQw6T3Jnm8Uqc5Bj88kp0T72LJ4M4kvO9vsWkgYRr0eVlkDzuDIlAILPJmmn71eRkuPNL8MWdL8RbYdBXk+njvn7oKWDGarjLR61CJeKBo6jRU5laSuTLTK5awEokEmUJmyXyVGlUiGC9XCwsZ/3OsFyiIDPG+r9r2Pzj2CRx+E4x13htXSzRYrJuDXwapqPj7854/WXDlAo8OJ35APL3m9aJSKu4NuTHQKvwFpEDcBOefF75E0WbYch0UN6iWKd0jKkSb2MOeQwS2x9jtcTRqW5vNGHU8MqPaJnhdXVBNRY7rAyEVORV8csEnbP+w6TojqzzL8pyxEf7O41OYK/4aCn/+4f6oQtzzyzq56iRPK55m1+dWPzudVoFKH0uUuj4RKW8F/DWoxXXNOY+hFpYNgLUzHb6rwsLAX5cKOBb+ltyyhEU3tK3KwZ2Yn0F1ilZYfZpMENYHIod43Kbb3FPM54U/B3Z95qTmwmprb/T4fvFMeHUCMT18a25nXlMHKc+NCj97KGT1YpE/BESLmyd/bz5HFh4hY2mGzbE1ZTW81/09tny4C6m+FpXK9n0Y2SWS6088Ql7aUHJzwejA+bEhBl3z1bj6Gj15u8+tXoB9n7+EE31mtVn4q66rprjadl7XJbKL3WTLnbk7KdAUEKAIoG9c3yb7z4Z2oUKYzSzLtGzLWp/FR/0+4sjCIzbHlteUW4SqpOBGPdOCOohKfUfCZNl+KHG+Ov9w0WFOlJ5gQ9aGJvvMcS+TCa8lIDqiofDX74Z+3Hn8ThIHJXK85DgPrXiIEZ+PIEARQMfwjtQZ6/jr+F8eH6NZTHVW+JNKpHxy4Sf8cMkPPDbiMd7b9h7f7f8OmUTGj7N/tIrA/jEw5CvhDuBJ9j0Fyy4Ak7FJxZ+/P+wddy/BN3k3Wbi2FkwSKbqEdoQkh7R8ghs5L/y5GKnUx3+kJhNkfAA5S7ix341M9nuWEG1vj00gF9+wmCU3LbEIf9HGKDoWzYNjH3pmAM4S3Nl9k/n0+yDl0hYPMwt/9ir+JBJISpaARMLp0012e5XwDuH0f6g/Z+LEuIP8gkAVBRdlQrdHHJ6n15+v+HOEVCqlc+fOts+X4m2kFN1FTMAJ1sn/i/+z/jy64lHL7kF3DOLxusdJGOj96ofCg4WsfGQlVUeF3Zda1sD+NbQXXHQC0m7w0ujOAkWgqOQK6ij+HpAkEgZCm1qKeY2qkwRUb0RutNpPNrT18AX0Jjk1KiNGmVhF2vR/bC0mo1hkNJeFLz+L6ztD5FAY8h3ET7FuO/quqOjxJRTB0O9tykMvJyAgELlR/FyqdFUc/vUwX439irxdnl2QGw1GMldlUpxRTI1eROJlRn8h7tRVQW2JjQXcOYfJZA2eHn0bDr0sLNh9FV05nFkGVScs30u9TFT8Fewr4MwOz1g1mul5ZU9mfDmDcsSKX24Isgam6nwwBdhZokfCyCUQNdy67fRvsGmusFs9V1FFI+n5JEl9LrOZv4TUr33NdmIAl/5yKTfvvtnlQ6irrqMyp5Lactu+TDqDjpQ3UlA/p6ZIU2wJ2qhUiGTEv0dAzu9NL3ge95K3UvQHK7GKbn3j+tJPPYOAms4W4U9ToCF3Vy76Wid7ALcCvxA/Ok/vTFi7MMs2TX1xjCXYG9oDer8g5oD/ZGR+kHwZpF6Oo7BRaCj419ZX/JXZF/7yduWRuz3X5cOzuz5ykl8u/4UNd4jeUzWyVlh9SiTQ/20Y8G6rP/NsOWcq/nSlcPyzJtbs5orKhsJfRKcIBt87mIhOER4dYnNkLM2geIsQZwKVgeI9/HM4HDiHrLcPv0F+cjV9/aCugWifPiudW/bfQu9retscXnS4iMIDhay9dxGxxzbYVPuZiU9RIPX3Q6+H4mZykipzK3kt8TWWP7S82SEuvX0pH/b50KeqPRti7/kii4tGGxJrEeFby8LDC4l+JZprfrumyb7qumqKqq32kIuOiGSJSR0m4Sf3a9sHOsBssZlZahX+ortFM/qZ0ax5co1NokZWuaj4jvCPaLpGN9Q0/0HrLobN1zg9rqFJwhllfXZT57UNz6yi+5p3wWS0vJN9hbr6r5hSCXKVnLD2YSjUCj7d9SkAEztMJCkkyWL3ufjoYo+Or0pXZfk9Oiv8AaSFp3Fpt0sxmox8vU+0q3hp/EsMSx7WwpkeoCYftDmgr7b0gm7Y48+gUFFT6x33IjM6HdQER1M94wrSZ6ZbtntDM/Jxleo8LkcigR13QYYQ2gIDIST/KEffWIqm0P1P0HEvjmPk/41kfPvxzD1TTNeM38hu95vo53AeG+IDHQt/AAnxJoILj3N0tWcDb85QqRPCggSJsAeTSCEwFQKSHZ5z8KB4aYaGQrSTzn7/JpSNO0hXZhBW8BadIrZi0oooWsOFlEwpQyL17svOTMqIFG7ccSOlHUU2mFreIHtSKofAdudmFQ/A6D+FoK3Jhsrj3h5NU/Y8jGL1UPykEqRGcQ8tv3c5f9zlO1nqBYdL8avKQmKUoJQpUZordlaOh92OkwXskvk/WNoD8hpl0pUfguUjPWMpq4qE1DmiF6eZgy/A0Xfc/9mtQaaEzrdjjByBVColsXYil3S9hBC/EFJGpDDlvSmEd/Rs/6K66jq+GvMV659bj1YvIvEyo78Itma8D79EQEkzzUV8maoTsKgd7H9G/H38Ohj1u8crBlpF5VFYPQmyfiIlJIVHB7xC55ynqaqCecvncfvh270yrNJqMcdQEojMXFiwMBlWTfLKeM4adTwkTLV9D6ZeBcMXQJBv9u5tDY3nL0UB69iTeh3fnXrF7Z8d0TGCe7LvYegDtja7hRoxX5JJZATKwyyasUqFCFpXHPZaH69/NTX5kLMEqq19eK/rcx2Pd/qVxJKrLD3+Nr+xmY/6fkT5KddbbSYMSOCyBZeROirVsu2TY/9lf9JdaPzrKyFC0qHrgxDc0eWf73P0+D+RtOqgiktU/FmtPk12EjCu33g9N+1yz1q/yfrISWrKaqgpr8GEgRqJUDGctvr0EueO8FcGW66Hk9/ZbLZn9emLLLxuIRVviyyDQGUgGLQQ2hP8fPv+sEGdQFVIL54Z/xqxgbGWzRKJhOhuItBiNBipqxaKReIFiTxc+TCRgztSExjZRPgrOlxE3s4zxESK5LvcZnT8wJhAwtqHERTffLXk6KdGkzYhjZQR7un/6QoaP1/KC2rAaGxzxd/SY0sxmozEBNjGPT7Z+Qmxr8Ty8PKHLdsWHhEtci7qfFHbPqwZUkLEz9wsBgEEJwYz7KFhBCcGoy22Or5kV4j3cVJIo2o/ox5+DISNVzn+oK4PQ/r9To9raLKYq9mr+DPqjUilIKur8blnoNkeV6mEslNllGeXYzKZ+GL3FwDc2PdGAC7tdim39r+Va3tf69HxHS0+CohncIS69UkWMqmMVVev4oOpH3DPBY1s1gs3wubroHS3C0baCga8CzOyQRFosfqs0ddQo69BrQalphTNgZOeHVMjGgrC3saH04zPTYzO1L17m9HLQBVNeU05GlUBfhWHKd62i+rCAQREubcSouusrta/aMOR14Sjj+0EvpPk5X7K9sOGy0VlUEfH2c3NVfwBJCZB5dZvOK3pDA/7Th+SXZ/tYtPHmwgfEI4+Xi+sCzTZwv/ZP85hkHN7vQtTv36+HQf1BkajkX379tGjRw9k5ihn/FSqxp5g/XcpSMO/AWyFvzptHTlbcgiICSAq3bsLFb9gP+L6xlH6dzXoIVDeoOKv6gTI/EEVe27/4nfeLYJV044IkdtXSL0SIgYS8LsRmSEIo7SYnDU5KCU+MAOpZ8NTy+m34iBLhjbopWE0QMURcV+0hugRYoER1Nl2e/lBUcVhdH11gEN05SAPFMGyceuEtaYPolYbqaysom/uO3w5uz4fLBqiu3s+A0OmkDHl3SlEdIpAu7/e6tOkEhV/yh7Q4UZbQfVcQp0EyghQhoq/K4IhrFezp3idwPYw8COIGERUQBT3DPoPe94HjVQU13n6kb3u+XWUZ5VTNkEIMn6S+qCSyQiJM4Rbw7mM0WANrgel/SNEP9ZeTJnGROjEXyzzlxrVSbIjP0dVOQ64D4Dcnbnk782n26XdUKhd18vGEea5dVxQHLpaax6sSgWox8Ms3w5M/2NJvlT0VmzU+zS4ftporvhLm5CGIkCBKtQz3qyrS74hP+Y4RtWlwDn+nGkreg0cehW6PWojAgrhTwSEq+uqKdYWN7HMdFciot31kZNc+ceV/PILmL6s43LVl/QbUUiEfwsBCZNJJJjEjhHCr4dp2OLBpwlIheG/QsQAm81mYbWh1aG2RMv/JvyPztM7M/LxkZ4cpUOmvj+V7zO+B2291WdQBxi32tvDah3Js4lNns1kB7u1JVq+mfINsb1jmfbBNACUgUraPXYFf74PnUJtj9/w8gZ2f7abuAcfICfPnzNnoGdP+9eWSCVcu9a+sFFbWUvBvgKShiQRFB/EVcuEaGQymVjx8ArGPDPGI+2HnKHx88VQZ2D7rBdpF98d9ZhZrb6ewWjgz2N/AjC101SbfZ0jOlOpq+TrfV/z/LjnqdJVsTd/L1KJlKkdp9q73FkRFyh68uVW2Sq4UpmU+Vvm2/S5NYtGZntQCwatqAhvrrdbWusErsGJg5EgIaMkg/yqfGICrQLp2OfGsqB8LIYsqPSxvCyzwKNQwMJrF5K3O48bcm6w/HwndRCJiYMSBzEocZDHxyeXyrmk6yWWfu1tQSVXcVN/Owk85QfgxOfiXghrutsTBPkF8fTopy0CoL8/JBxZiX7VfgzvPoZM2br5gavQ6SDszAH8i3OpKR9msaj3hmZ0Xvj7N1Lff+q1Vf/lqcKn6DDsdu66+nkiOrmgS20rsLHV+TfhFwn6qhYD0H3i+jCjywx6x/a2uz8pScIfPS8kqkOo68d4FmhLtGhOaHj1sVeRJdY/ZPc+BplfwexyEey0g1n469/f7u7zNEYZgn9kCCZAqa/vmaCxCn/VRdV8OfpLBt01iElveLcKok5bh0FnQFNvvxiobHAPbL0ZCjfApT6WuuUs+Wvg1HeQOBOCOvmW6AeiTx6gVIHcGEQdxQxcOpAhqUO8PDArSZO6s/yUHpPUZBX+pDKYkdX8ifYIbAd9Xmq6PXmWqKiReGjid/Bl2P0ATN4txB1fuy/M/D2cJEki8AFVVd4X3uUqOQNuFYGimt2NrD5jJkH8OVrRBaL/xeT6asXCjeAfCwHtfDvhwS8COlhtmM1BR6MRsrfloTldSueLOnssSHP8z+MUHixEOk6Fny6OQGl9UotEChd85pExuAWjAX4Jh9gJMPwn6zYHFTbnFNoc5Hpbi6qksBjIgjJ9vmXbvu/2semVTaSOSiU0NdRlH192soxT606ROjLVpr9GTqWwUI0Pirf091Mqwdc7NvzjcXDPBwWZMErqKC8XSUupo1JtKvJcyfYPt1OSUcKYZ8YgV4lQSaVeCBWR5kz5rTdDxSEYu9q3n+GuZP+zcPB5SJoFod0sm8PCQGZS4aeLpVaZR05FThPhr+xUGTlbc2g3uh3qSM/GG5qjqgqkJgWjwudyw2AnTqgtgvJ9XkvKML+Dq6u9k3zjNBIJJM1ostl8X5RoSzAYDcikMuT+crTFWvQ1HkzMa4H0mencU3MPV1RdYV2X/MNQhanwC/JDGaRk9ROrMegMjHpiFGVl4hncuOKvx+U9CO8QzuFIFRyAM200nNr+wXbWPLGGm3bdZGPvuu+bfWx4cQPJw5LpNK1TG/9V7sVkMBE8sg8nS+LbVPG3/cx2iqqLCPELYXCi7QNnWPIw+sT2YVfeLj7e8bHlvhuWPKxNFVotERdkX/j76dKfMBlMXPqLtS3RzlzRo69JTFIRBEO/cem4wvzD6Bbdjf0F+9mYvZGZ6TNt9vtq1XPDir/ul3en3Zh2ltZWYaow/BX+zZztfnrG9OSn2T+55+Jp14ukLZmH/41VmSKGGDMaqTqBx0Y8ZtmlUkFxYk8CuiTZdSHwFDodhBRkIDu9B4l0eMsnuJHzwt+/EZMJjDrLC6XGv5JqWSBSN98NpZmlfD78cwb/ZzCbL9jC9ugMrlQmELvuJRjyP0i80L0D8BX8Y2F6ZouHzegygxldZjjcn5QExUl90Eh8a/I/5L4hDLmvkagQPxVU0Q5Fv/x8yM4WAZc+fTwwyH8CJiMy7Sniw5SU1za1TgmMCWTy25OJ7+/96pjt72/nr//8hfx6CSRBgKKB9UfK5aKvka/cwK2l8qjoUZo4HXr7bu8Hf3/RDwtAY/CtVOGokekc2ygmxy5dYDd+MMo8mGUS1lvc2/ueEJVefV/3zSC+VIFcLl7+tbVQU2tErjBStLeIX674hSH3DaHPdd55KA9PHs6qrUUoDKH4uba1hffZch3otTD9pLdH4jQmk4mdBVsoDdUQUj6czW9t5dA3u3iw9EGPVd3MWzEPfY2e/UeVHP/iEdq1a/mccwKpDGLH21aALu4IASkwbpX3xuUCjOM3kblvHz0abEuNEslKVSbrnKX31b1JHZWKOsq1okD2pmx+m/cbs3+ebSP8mSv+Ggp/ludM8XbQZIo+re7uCXseWww6KFwvkiTDRDnJihMrmLR0Eur07kTl72rhAmfP4V8Pk7U+i/EvjwdAb9RTbSoDIDqwPgCr14i+oufq3LUtpF4uKrjUiTabVSrxZ9ihbXzyVhSpsU1f2BlLM1h661Lm/j2X9uPae2rEzbLz052U7A0HUi3B5BZRRcGMHDBUu3NoDjGP02QSVX9Oj9tb1JYI54t6C/9w/3Dem/IekepITIhgrMJfwV2Zd3lzlHYJVYVa+kZx8jvhUJN+n+h7eS5QvI3Mva9yOGQw/Ttf0cTGViKRcOWfVyKRSviwz4dggjHPjiF//VHStu0kYNwkINRyfPtx7Wk/rj0V9a1vm7P6BKjKr2L1E6tJHpZMzyutpYFxfeLoOLUjISkhNsenjBQtBpKHOW4L423kKjkBV1xE0WKa7fFXpatiwaEF/G/v/8guz2bBZQvoGtWVpRlLAZiQNgGFzNbZQCKRcPcFd3P1b1fz7rZ3+XDah8zoMoPx7ce75d8yqcMk9t2yj4SgBJvt+Xvzbar9ANIj0xmcOJhBCYOEw0bFEWF37QxH3hatGsb8DeqElo9H9PnbX7Cf9VnrbYS/itMVBBw6iqoymcpK3+oL1LDir98NovfvihOivYhZZDVTq69lQ/YGMooz7FfQnWtIpKAMafk4V1OwDjZfDSMWNrm3/P2hIrojFcng4vaYraKuDrK6TyZ29nCUgd512zqf1/hvZPlwWNTeEmCV1FVTdbKI2sraFk48O4x1RoITg/EL8mPh4UWcjH6PYkklxrD+YiJ9nlYRFyeEspqa5hss+wQpl0Kflx3u3lFfBNG1a/MTqfM0wFALi9ozr+fDKOvE96eousiS1SJTyhh4+0ASL0hs7ioeIapbFL2v6015gEjPCmwoAKddCz0e99LIXEDypXBxPsT5aCXS6UXwRx/SIzcy9PAG1o7T0bu2N8f/8p1+hDU1YJDW3xtm4a8qEzK/geqc1l8w9y9Y2M7aUy9vORx+XfRt8hRx42Hwl6LPSdFmMLr3/dpmxq5EOvQLAPakXo//CzLe3PwmUoVU2DR7mMKDhbzb9V12frqT7y7+icFHVuGvS0ahQFQbbJzr8TG5FL0Wtt8pAmE9n/L9oLFBB4s6wvY7kUgkjPhiOBs6jKNWXkDqxX24+NuLLRUxnkAql6IMVFocI/zNyaWFm2DztVDiflHAbQz/Gbpbs1WJHt68fdI5TPsYEbDRSgsx1NvdRHePptPUTigDXLswThmRwmW/XUbSYNveNBbhL9Aq/Fnup8yvYP2lwq75PJ7FWAMrx8KRNyyb1Ao1epMevayc8nIhemRtyOKzoZ+R8UeGy4dwxZIruO3QbZZ3YIm2ROwwSYgOrvexGvI/awX3v4XQHpA0026ALywM/OsSqa6wH2FLG5/Gxd9e7BULcXsYDUYWz19M9ZrtVCtPcaDudw4WHnTuZInEawkBcrk1QcHXKl6acOQd0Ze5eLNlk1wq55YBtzC722zk7s44byPVRdW8HP0yKx9bad148lvY/6RwbjhXKNtHu9wfeGvF3Q7vbalMzPVv2nkTVyy9AolEgia3ktCCo6iq7a+Z4uo1jJYq/pQBSnZ+tJOSjBKb7e3HtWf2j7OR+9n+/kOSQhhwywCPJZK1lep6zd9RxV9ORQ4xr8Rw9W9Xs/zEco4UH+GGxTdgNBlZekwIf1M6TrF77mXdLiMmIIacyhwqdZX8etmv3DrgVnf8Mwj3D6d7dHfC/G29GWf+byZT3rEd38PDH2bj9RuZ2GEi5CwWcyRtHpxeDJuvB00zDj0mA2ACvfPJEqNSR9E3rm+TSseSYyWYFv9OcFGmzz3/7PVyM1dTNuyxCSJRf+xXY7lt6W1U6TzzD8mrynNf5VvlMbH+8mQ7FYCY0TDiN8ta6VjJMdaeWkt+Vb5lPm+e33sLnQ6Mcj+UcRFeias05Lzw52Kk54JHTMxYSJhmCbCmZCqRvf8up9aecuvHRnSKYP7m+fSd39dS+rwnfzzSccsh8gK3frbPUbgJ9v5fi4Fok8lEqbYUo6mpD7BcDukHfiZ97YecPu2ugbaeE8tPsOrzVfx2+Dd25ToXhGvY3+88TZFKpfTo0cP2+SJTQfoDHKmahlIvhL86Yx3ltb4XqOowsQPTP51OQsBMumW9xaiY6d4ekutQhohqVl8N4JsMUFdBoH+1sPqsVbD26bV8PfFrDHUGb48OgJ2PLqDf2ly66q9kQvsJYmPBWth0FZS0IbAWNUz0UTMLzMc/h533ChHDk0gVoqfu6D9A7jvWVo2RyaTExwchNYmAhqZOQ3S3aG49cKvHq/2Meuu7Ttfg16VQIKpAcpZ4dDwuR6aC7AWgCIH2V3t7NC0jVYAi0BLkNM8bDVIN6k5J9Li8h0eFv5xtORQeLGwq1JTthRNfCBu2fwqDv4Q+L3p7FGeNtHQHPdVbkZqsAYGO8WaLViOni0scnOkaghOC6TK9C0HxQTbbzVafCcEJTSv+0ubD0O9F1dl5PIs8CPq/A+2tfYFCVEJoqpOVo9fXJwvpDJSeKKWmzPVRHalcSkiSVdwy9yNTGEIJDvRNscKj6LVNNpltAUsdLGvDO4TT4/IeBMa6tkTN7vrISa5adhUMG0phyB88mzmNh1c83PwJhlrY/QiU7GzjaF2DOUHW1wLfTQjvI77HipYrQQ7/dpi9X+/1wKBaxmQ0Edklkr01e3l85ePsydsDg7+ASTua9B71aZIvYXJlR9ZoRXygOSRSCcEJYs1UkdqTnZMfJXmk1VKhpryGN9u/ybrn11mEv7w8kYThCGWgkus2XseoJ0ZZtmlLmz47GqMt1ZK7q4VyQg/R+PlScqyEii9/JbjgmEPhLz4onnD/cDqGd+Sx4Y8RqAxkY/ZGnlv3HNvPiICXud9bY/zkftzc/2YA3tzypuv/QU6QMCCBtAnNWBlXnRCJtVXHxffhxGf14p4DutwN0w5BcEenxzCn+xx23LiDR4Y/YrM9tncsEfdcRWlcV597/pnXrHKZiS9GfsH6F9ZzabdLybwrkw+nfWhzbFJIEikhKRhMBjaf3mznaq7FYDSQ+kYqQc8HkVPRhoTqltj/NPzZ1/NJzgFJwnHLXwirt/x+CyO/GMnfJ/5GrYaQ/AxifnmfzJUtO+25C50OVJUFyHW2blve0IzOz17/jfR8EoDAQ78CkB9bREWf6whPC/fYEPI1QvgLksT6bLzcrRSuFQ/JuMkQZb+pgNFkJOSFEKp0VeTcm0N8UFPLRnW4ivJyNdnZ0Lu3m8fsJBte3EDmpkz+e/9/mdllJgsmvwCb5kKXeyHlsibH63SwZ4/4//P9/Ryj0+lQNWyIKZFAnxfJ+BFkJhgZdTGp8UEYjNbJ189zfqYss4z5W+Z7YcRNiaoZTLuCwQww92ku3CQEmR7/Pbd7d/kySTMhaSZ5h8Rfq6uhz/V9SB2T6tVhNURbWEmARsnFfM3TY+o3xo6F4b+0reJFroZJO61i7KCPocON4B/T/HnuQKYEmet7M7iM0wuh+gz+/jcgN4qAnKeyD+0R0zOG2w7eBkB5gxwGhQIhoLYQvPB5JBIRPFL5RtVDi0gkMNmawBOgCKCspgy9rAqNFxyDv7/oe8LSwvhhzma2dT5InPIZYJT4fqdeDtJzxILLHqd+gLwVMODdc6uqoCVO/g/p0XcwJUwFubDiCVQrUOrD0clLOJZbQMr/s3fW4VFdWx9+z2gyEncPIUBIcHeXtlCgSt2d3rbcun21W73trbtRd6pA8QLF3Z1AEoi7jX9/7MwkEw9MZpIy7/PwtJyzZ87anDP77L3XWr8VGsK+n/Yx/6r5zPpiFimzWikhdQaMjR+LhMSgqEHYVdIdG3mBvR0yk17cjCRBtzucDvmrhePALC/Bho2SEonEcYn8+9S/XX75yvxK8g/kE94rHLWfGE8KqoTjT2UOEc+IuRKOfAjBQyHkn5mV2yRLx4hMj+kHnA4HBkKRdiOPb3ubEcY4nhr3lNtMarA+agUyuYykyUmY14ExSwSMhGpaUB/KXydqHCp1ENT/dM09Y/z8oLAQyso8ZkLrCB0h/tRj66mtHC06yoDIASQGCufSmufWUHaqjN5Xen7c1YZpuW7VdYyfN54Vq1eQGpZKn4g+ouZxZ0LpRw46qmxgsrR+7lxcocQmc67xZygx4OPvg0wuIywM5HKxf1NQACHNxMfEDBGqQzabjeqiav4X9z8G3DKAKS9PabS9zWrj3d7votQouWP/HR7PkAHn8aU0sxTb9p2o06KdHH9rTqzBZDExKHoQOpWO9TesJ0ofhSRJBPkGMXfxXJ7860men/A8+ZX5DTLA6nLrwFt58q8nUcqUFFQWtEt9Pzsvr32ZY8XHeHjUw43uMwLkVeShVWnRKGs6nHgNhI0BfTcw5IlAHZ17dPd9AnwIGZyE6VDHC3ywO/5kFiN5+/IISQlBJVeREJDQaPuRcSM5vus4a06sYWKXie1qW3pxOgaLAUmSmn32Tpu4S0CfDHLPBjkH+ojs1aKqInyjwCZJ2CwWzAbP1Y81Gm30XP0+1QVJ8MhlHrMDvBl/LsdqbZiZ1VGxR25nR5ziVI/xhPRo38jWjHUZrHlhDfkn8ik2iJDAc6P/hh2Piii6s4mEK+Hcnc0uHGSSDD+1iL6ySxLVJ/6OaRwcelWHyvgb9/Q4lI+JTSs/tR9U5woJAHPju4S7domXZUgIxMe709LOg9Vq5cCBA42OL/bIzwe7/MinMz91miCqdCrHxoUn2fL+FuZfNR9ThZgVKe17moZ8UUens2/mdwJ8fSEz6HOeP3Q5WyK3MOCmAciVHaPmXMrz13Bg+LXOddw0MRB7gSOKq83YF4uGQvGchY85Yzv/kRx8G3Y8iMlUiNwiBpMKYwUWk4WNb23k8KLDHjErvzKfqDe1LO4TikJprQ0Q+ic4RHzDO26GcAtoVeIZscgqOPztFl6OepmTW1rQe3IhY54Yw+A5gzlSsZ0i/d+gqln5S5LI8O0stXcaI3c1HPkADAVQshc23QH5Gzxt1RljTbyBI3FvYlU413jWIJzf6Xl5AGjDtcSPjsc3yLfBd5wJKx5fwfP+z1N4xDmz8Jq+1/DxjI+ZlDSJGhOa3cD04jnsGX82yYJFVkFpaftd69jyY3wy8hMO/n7QcSy/UjiGlOZgMeeuzIItd0HGj+1nSEclZJhQVaiX6hMYCEZFHssL5vH7wYaZ+RaThVdiXmH+1fNdak5z66PmsEueGY1gUIgBoEXHX+gomLIJEq8+LVtdhd0hU1zsSStOn/9b+X9c/P3FLD9WK6U59bWpXPTtRR60qiH2IDi9Qi1qmjWS6dqhMVcRo7CilVrO+KtLURFoSk5RuqVWRtk/zp9btt3CiPtHIJdDeE0cZUt1/kAEU3x13lcseWAJXad2JaJv0+s6SSYx8uGRjHx4ZIM6c56g/vgSPyaekjsfIT+2v1NpmmdWPcP4z8bz2Y7PAKEkYHda3jnkTvqE92Fswlgu63UZL01uuvQNCFnIR0Y9wsGCg5QZ29e7/+6Wd3lr01scLmx6rffI8kfQP6fnf+v+Jw6og8T+pVInHH71AnUaUJUDh949rUxps9VMtdk5q1+rtSFZzB3O8WeX+tQEqLkv9z6mvTut2fYj40YCwmnc3uzJ2wNAt+BuyGXtsPcTfZ4oVeDutW3VKfg+ELaKIDCH46+6qKbGX1d2j51DwsTWZ5u6GmOVleykkWiH9XI67gmfkdfxdzZy4gfYcCP+MvFCtcjKKS9vPl3fFRxdepRlDy4j84TwUsmsKkZELoY9/wHpLEs+1USLWgktbFLZo2+acvzF1pQsychwqXVnRMzQGCoHCR1vvUoPYSPhglNOsj11sct8DhzYafdCPcfmO7k4SkSPNDYBOv/D87lqiedrYmX8ncHOL3aSxWYKdKupokYLKGY6XJAN0c1PjrycAaZyOPIxCbq1lGi38Hfp12w95VmZovoYjWCRqpEUdbQdXfFCMhSK+iI7Hm657dlKv5ewjlmIRmNFbs/4M4nBZOGcheyYt8Ot5hQcLGDjWxvJOZRDpbkSs7wUtapmqpq9HIp3u9UeL8DReXD4Q8BZ6tOs8CEwMRCZ3H1LiYG3DCRtdhpVlpoNOXWNbFzhVije5TY72oXeT8KF+eATLp7zQ29DRbqnrTpzAnpRphvRQO74WlZyztYKushEUEbciDiuWHgFCWMSXHp5/zh/ogdHo9Y3Pd/Orcn4C7Mn4i4cAIsbZqp4cRNrr4Y/ajdJtEotcklsVpnkJZSWgrnazK6vd5GxzrULoPA+4Ux8caJTfexp3aZxwdFsBhz5Tmz2aqJhwkpIusGl1+4U9H0ehn7UYMEWGAi+BhG9ebykYekQuVJOaEoo/vEtyz66g9xduTytehrb6r8x1jj+QjQteP5lcggeKALTPEhgTUmupmRVOxTbH4INNzkdsv872x3qIPYO6tdh9RSFhwv566m/kB8VY06YpRh+7yGyPTsTpxbxq+8Opmtbn/FnMAgp5bhdC1g19+cm27W2zh+A2k9NyfESJEnikh8uoc9VfZptP+i2QfS9pq9b55atRZIkKgwKbHKFI+PParOyMWsjAEOihzT4jEKmYNnVy1h85WLi/ONadZ1nxj9D9r3ZTWaLuYpInbiRp8qa9uBuy96G1WYl1r/m91mdJ/YWWktVFmy6DU4tapNtcxbMQfuslo+3fex0fPd1r5C88csOl/Fsz/irW+PvhTUv8ODSBzmQf6BBe7vjb33m+jZl5LaV/fn7mbNgDgD9ItxbvqPdUfoJJ7RWZJwG+AQAIuOvrghAlQdjNsw2OSe7jyVgZJrnjKih442oXtqfgk1w5CNifDTcPuBO0g5eT/Laz9j2xZ52vezAWwdy48YbqYoQvz61KYIfTrwL5+0Tk+mzDVMZVDS/YG3J8afKPk7EodWcPOrhyqX1KDWIcFy9uk5NlSa8egdq3oUdRaq0U1G6n1CV2AivqIBqczWVptYXT3YXMz6dwcOVD7Mq6F+s6zGaXcVrnRt4Pb7th7kMNtxAd/XXKCzi92j61cQbyW+QuyfXw8YJcpfsoJineNSg5t9/1kh3rZgKv3Q5sy9WB0H3u6Eys/0jWzorgb0heAgajcUp40+ulHPNimsY++RYt5qTtTGLhXMWkrMzBwCZ1bc2Q/ivabDtPrfa4wXY/wrsFbXmtErxjJjl5Sj7pnL939c3G73dXtgdf34+NSHXG2+Flee63Q6Xog4WfyRJZDvPOgVR53naKtdRbwyOCwpHbtVQUNC+l+1/Y3+uWnIV2rDa8Hyz1cyRwiNUmcR6xJ7xF2pP+AnsDQGeX6Sftaj8xfu75pmRJMmR9WeWl1BSAqYqEz9d/hPbPm5dLfHWEtI9hBH3jSCwS6DjmFxSYC0Nx9cUKxx/Co1QEfDr5tJrd2YCA0FjFI6/wqrCRiXDr1pyFeMdeu6eReGroOvUrhh1gRgVNVKf2mYy/kzlojyB1XOSYXY6VcZfwUY49afToWBfoUxT1/EHosZzR8jyytuXx8r/W4nvcZF97uMTAj0fhIj2leNzOfpkvrPEcNDU+ow/+zOV22MU09+b5siMPbLkCBvf3IihTCh0RdWoQrYm40+uknPjhhuZ/v70Nplv64DrtrKTZViOZyIzGxyOv0MFhyiqLsJH4UPv8MalaoM1wR1CtrQ+kfoax1954zfSZDGxM0fU3uwfWaNStuYS+KUNEl36bjBuMSS0LRBdq9RitBjZm7fX6XjEmO6UB8V1uIw/u+PPWFDG3h/2UpJRwqc7PuWFv19w1JSuS8/QngT6BFJhqmB79vZ2sWnzyc2M/HgkGaUZdA/uzjPjn2mX67DyPFh/fft8d3MotDBhGXQXjs1AXzFvKzYUI5eDWm4m5PgWDv151P221dCYQ9hTeB1/ZyNpj8JFxURFDOetaa+TXHQdmtJTFGW2b8EWbaiW6EHR5FnECltlDseqCAb/Hu163Q7Lb8ni5dkMUbrmHX+VO48Qc2A5FdllHqm30xgvR76Mz1MizMJP7QcnfoTMX5tsb39xB7mvxGSnRC5vxDk+fgl/ykWWwwfZt+L7H19eXf+q43TG2gzWPL+mVcW02xNJklD6KjHJxM0O8K1xCB/7Ek4u9KBlZwHqEBjzBxmaO5Fbxb97FVUotUqsZs9LU9tsNjLf/JkeNYlcdilBAvtAyNAzv8CA/8HElV7nclPYbGAxoNPZGtT4SxibQHA399Y06TKpC1cvvxpdP2GL3OpbO1ke8Bp0m+NWe7wAwz6D0UKerW7Gn7vnHIYyA+/0fodV/1mFwSaeUX9NTcZfj7mQ9rh7DXI15irYdj/sfgZkCiFzrNR52qozJ2cZvfcNQzr2qdNh+5yvsEaBs7KgkqUPLuXAbw0jo13NiZITdH2jK0EvBmGz2cgRcQYO+TKGfgKD32t3O7w0wcA3YOJfTu/tqV2nkiaficymprRUZJFc/MPFDLmzYYaFqzGZwFzj79FoEPL0Ns/PnzyCsQQ23Q6HP3A6HBgICqsetVVsvB0vbpj11140uj5qgeDkYC779TLK43piVLZC6jNnBSwZLiTrPEynyvgb8zvMcH4WHBl/VbWOv+WPLudp5dMUpxe707pGSRibwK07b2VfV1EcXeXfDfo+B2GjPWxZGwlI40NS2Gpofcaf3fEnT+lGygUpDmfV7m92s/DOhY51oz3jrzWOPxClR9rCj5f/yMuRL7fpM+1F3fFlz/d7iP3zIzSlOY6Mog1ZQpJ9QOQAlPLOVY6gpYy/ffn7MFqM+Kv9SQyoqeMXPa1t2e5KHUROAm3bMnpTw1IBGjj+hj87jZPdx3U4x599jlC0O4vvL/6e46uOO/5d7f/OdZFJMkbECWWJvzP+drk9ZYYypn4xlYKqAgZGDWTN9WuI8WunbHVDARiL2+e720DdGn8Avj5WEnb9zt6v3ateVJeq47l0X/spVVv3ecwGO17Hn4s5ncmn21HqRTSlJG6/PDaK7VMeIPHS9i1QXlVYhanKxLRu05g/tICBh+cT5XdUDBZnI8m3Q9zFzTaJ9osGmnb8DbljAJnn3YxBE9hh6vxFD46mPLJGhkulh52Pwvb7m2xv3zysq5XuxRm5XE6vXr0aHV90NfuCCpOIhs6ryHOcO7LkCMseWkZpZjsWRGkF2duzObnlJCZJ6DI4Nmu3/Rt2P+1By84CZEqIPhfJr5sj4+/UiFPcuv1WIvq4P1OnMcLvvoztA8R7wO5YoN+LMOIrD1p1lrD1HuQ/aOnbXYbGmEBP1VQGRQ0ChFPW3QWxdeE6EsclYg0Qmwsym09txl/Xm0QdAS/uJbCPI/vp2r7XclXES/hXDqD0eBGr/rOKU9taufNzhliMFiwGC+ZqMwabmDj4+dSMFwmzxfPRmak4Dvv+C1l/eNoSlyLXRCGLGIdM47zxcUKxlO0J1/F73puAkG78+4W/ObbsmEuvv/XDraz4vxVOx+xz6ih9FJIkNZT69NLh+PKCL/lX+Hy0hiRKS0Eml9Hzwp6E9w5v+cOtxGaz8UbyGyz8l3NA2tsbPmB37L8o1K3B1xc4+CZ8rRAZYLCaE/oAAQAASURBVGcbCg0c+RCylzgdtmehaU0iEyS9OL3BRw/+cZA/5/6Judp184rm1ketwWjEIfXZbMafXw8RXBJ1zmldx5XY/607heNP4dsg8K4xqc+wtDBSL0lFpvD81qRaryYsLYy8mufCsS7phMwdNpePz/+YYbHDWtXe7vizP2P2DMyxT4zl6uVX4xMgvF32jL/WSH2CcIp8/z0cbmXZ8KCuQUQPjsZisrTuA+1E/fElflQ8mT0nYdAEOurSb8gUjr/GZD47BFYLZP4C1fkNTjkcf01k/NlLg/SL7FebsZjyb7FGbws1QaZtoWdoT6C2Pp0d+75XR3L8Wa21jr/owVFc+M2FhA0Jo8RQAtRmVtbniTFPsPXmrdw5+E6X26RX63lv2ntM6jKJ5Vcvb1nK+kyYsh5G/9R+398c+1+DPc8DtRl/RdXi5eijU3JwyJV0u9FzQRumsip8ynIb6I16wmd0lhVWa386Ylp6A4wlUHEMmyaOHJMRs1851sIEysvb93H4evrXFB0t4t+n/o3KEoSvCW6JD4Q1/UWa7tlGr5aj01uS+vSP9Se8rz/ZO0Sdv+7dXWrhaTH7l9l88sUncKRG6nPoPDA3/na22aCyRplSo2m0iRfEuFJWVoZer3eWiijeTaJsDyr5+ShNoSCDvMpax1+/6/qRfG4yQUmeTaf847Y/KMkowXR9vYy/UT+dvVHTbsZPb3E4/tq7WHhbkCQJRc9unLLUc/x5cQ9BA7HFX4F03Ehw+WguUI7h3holrrdT30aSSdy++3a3mWO1WJFkkqOYu7yu1KcXz2A1g6USlH5cknoJ/pnwZhVUZh1lxacr0IRoiOzX+KLWlWiCNcw5MAejxYjlGaGd4u/7Dxov/HuIGn+qAE9b4lJs/qmUDfhGzF/qHC9VHCIz5FN8qouBOejCddx5+E40wa6dDO7+ejentp1i3JPjHMfqOv6qq3HUigkNBUylIiApfHyH2OQ/KynaDicXQMKVoK2tieRfUx6upKS2qc1mc5mEmqnChCZUg1ztvCGz8PDvpIf/SrA1FUkaKerJxM4C3/Yf9zocMiXMyAAfZy+5PQtNVR0P6u2N1vk7tvwY6/+3nsF3DiYwMbDB+dOhyfVRC2RtzGLX17sgbwAplpc495JTtRktjeGXLOqwdgDs/9adQurTVAq5a0CXCP4pQOOOv7TZaaTN7hjyysZyIxUVFVgtVpBBUOZ3kLMYhn7cuX7zxbuYmvkWJN8CQV1b9RG7M9lfKuHlyA/pc00fJjw3Af9Yf/xja+tz1s34s9laFlVZvhw++wxWrIC3327ZjnFPjWu5kRuoP76E9Ykku4vovH1tYs/4GxLTQR1/Jbth1Uzo+YCo0VqHlqQ+HY6/M60N90uCyPibtKbVH+kRIhThcityya/Md4wbp5btJWH7QTJSp2Kx+NARcm5MdRJqg+L9iOqexrEiEcSmlqvxVzde23ZA1ACX21JuLHfspVzY80IuSLmgQ8rMuoz0z8GQD6kP0ie8D0+Pe5rkoGQAdHqJ7NAkZM3E9LQ3luh4dky5n5H14pY94TPyfFjNPwyrtRNsYp9cCAv7QfYyYl6J4duAHujyNpC1qZVhO6dJ8rRkel8ttK/tTu+d1XMgfna7Xrcz0zO0J7N6zGJ0fOORCjarjehQA5LFTIZr69ufEXOHzuXtc99maMxQCBkMEY3XdDAaayNkdP+g/TtXY7VaOXr0aMPx5dg80kpmE+CTg6xavNXqOv784/yJHhSNUuPZnfMhdw9hzONjMNulPu0Zf6HDIWykBy07S1jQl16Z/VDUSH2aTplY/9p6srdne9gwgckEFrl4NnQqnZCL2HwnZC3wrGFnA4lXYh06j7wyMzabcwRl8nnJJE1Jcqs5q55exVPypyjYKxzBDqnPsiPwazIcfMut9ngB1l0D3/uDRTjb7Nn5FQHR3Lz1ZlIvTnWrOVWmKrS2COQWrajxV7Iffu8BRz52qx3tgjrIocbxT6Gp+UtssMjUKrXU1PNUyAhKCnJkFLiKC766gJs2OWeD2h1/0fpoR7afVlvzbFfliMzL3FUutcNLGyjYBDsegRJniS+93oYNC6U1IhafT/qcN5LfcNllVToVN6y9gckvTXY6nl8p3kd+ihrp69iZMOpH0CW47NqdCt/wBjv9dqesT7XI+Ktfvw1g+L3DuevYXU4OhDOlyfVRC2TvyGbDqxuwlZQQXXgZt/ebS7CmCWlzq7lDBSl2qoy/iuPw13lw/GvHoaZq/HUUNry+gdcjXmfN6DVsumkTPoYcyP0LpE4WhWYsgezFUHGi1R9xZPzF6AhICAAJPp/4OVmbnGuUhYWBTAYGQ+uew7VrxX8zMuhQ+1UtUX98qevgUanEfHRHjpAR7LAZf75RMOgdiGtYXqilBINt2aKGrqO+n6kc/jq/7fPt6GkQNqZNH9GpdCQEJACwL69WJrFo7ylCMnegMJR3mDJH9jpuUOsQtjtTI/WRbnO87c/fT+93enOipPY33+7XNpXBofegcEv7XqcpRv0Ek9cD0D2kO4+OfpRL0y4FavaWbTZKCz1Xm9c+ZtSv8ecJn9E/a3XppXUE9oXeTyMFpKJX61GalPTcsJT0Hza362VHPTSKSS9M4u1Nb/POidso0K1im+Xpzi/PdLrkb4Tlk+DkoiabDI0Zyk+X/sSDIx9s9PzBPw6Se/fzBJ3c3SGkPityK1h872KSjydz26Db6BaYJCQGmmpf88KWyXBIJnhpA/GXkxH/HWWGYGRVNY6/OlKfNpsNY4XRpbI6p0PapWkkXZECknjJBWn1op5RK4uNezlDwsZgDhrjyPizZdv48+4/Sf8r3bN2AUVHi8i9+1kGrBfR4zqVDiozhJRWwQYPW3f2oNGI32ZZWW0U2uSXJjPl5SlutSMkJYSUC1IICA2gt/9I/Cr7icmyzSwkxqQOENp5thE+FrpcDzYzOeU5HDdtokJ9mAqzmsh+kfgG+brFjPLscra8vwXDEQO3V57inG3laH1UYKkQz8U/zGH2j8FmIzLnNaTD7zsdTgwVY34FudgDX0uzSinJKKn/DWeELlzXQPUgq1RsYkbpoxrKfOoS4Pxj0OMel9rhpQ3EzICpmyF0hOPQ7X/czvRNKo5EvOxw/IX3CSdmSDvVrKlDYbVw/Pmr2lEqqzNhLIZTi6GydrNYqQS9HrpnPc3Bq6t5fExDVRt9pJ6AhIAOIefY+8re3H3ibkr8haOy/qacEye+h58iRJ2/DoA946+srDZ4tsOi6wqD33dyOvQI6cE7573Dy5Nra7jl7s7lj9v/IGOd571CEX0j6H9zf3ql9GJg1ECkfs/DpZWgdm/N6zMmbCS7xm1mgZTAkcIjrfqI3fEXFCLnhnU3kHpxKhlrMzi23FmCW6GorYnbktxnRQXsqFNia10rFJJLTpSw5P4lDa7raVY8uoyU1R8gWUyoVKCSq1h7/Vrem/Yecf5xLX+BJ/AJFWWVNt7SYE9uaMxQdt+2mzXXNZ6Jd0WvK7iy95W1Tk1joUggqReU0yKD3oI+/2mz6Y3JfY5+eCT7Zj2EQRvsUGvwNHbnjkwGyx5czH8j/ktGuhjLGqvvV5c/Dv7BtT9fyze7vzljOx5Y+gDHio/x1F9PnfF3tZrKLNh0K2TMd98166KNEzXRG0Gng+7r5rH+stfcbFQtxqw8Ak/uRTJUtdy4nfH8zMuL+/HvAWmPgn9PdCodBrWB3UP6oh830C2XX3h4IctK36XC56CjMO7ZiQ3y1kLl6XvsArsEEj2tLwZNIIWFLjTtNCnNKmXdy+vIXF/Tp/x18K0KDr3TaPu69f3+yVno7UZQP6wxF1Nl1mMrb5jxd2z5MZ7TPce2T7Z5ykIHRRU1qUQ2CX+NBo58AN/6it+Al/Zl4GtY+79BYPkIJm/L5+eHfufaVdeSdqnnZXVkChmyxHiKA4qBGsefX0+YmQXd/+VZ484GctcgbbyZCJ/9VKqP8obOj6AXPScNnHZpGpf8cAkTBk3g1T6r6XP8QxE96dcdzt0Bybd6zLazlq43wdCPQKHh/S3vc+WqwRyJeInycjBWGCnPcU+hjfwD+fx+y++kr0jHUFMqRK0GggbAeXugy7VuscNLG5EkQop+QMr80elwl3DhaTMoch2Zxu/3f5+frnBtnZDi48VU5lc6HTtZXiv12cDxJ1MK5189KUMvbsQnTPyulXrHIbkkx2IzY5aXOKQ+J/93Mhd8eYHLLrv3x72sfXktpkrnoLQig8hMCrJv/O95DrY/5LLrdjpyVsKKKXDqT6fDAQGgtPpRWdp4JKfVYqXkRAkVuZ5P01D6KtFH+1OuKCbH/3f2F29vurFMAZpo0HVxm33NodeLDWabDYcTvMOi8BVziIDa9UaoNpRbB97K+d3PdxwrO1nG5nc2k7sr1xNWOpF8bjLT35uOPkrvfKITblS8vO5lzvvqPH7c92PLjWlY4y+yfyRzDsxh5AMN1Xnqyn02x8aNzg7q9etbtsNQamDtS2tJX5necmM3YqgwozBWIFfJkSSQy+QMih7EzQNu7hhyiqZ6A4LNJgKtDQVCDtHkHFilU+lIDUt11Earz60Db+XzWZ+THCykE9HGwWwj9H2uPaxvwMTEiVzc82Inp6raT40mQAWS1GHq/Nkz/lQqEeAS1DWIPJvYj4vQNe6UsrM7dzfzdszjoWUPYTC3rQ5iXVamr+TXA78il+TcN/y+0/6eNqOJgXGLIfEq912zLqZSKD0EVhNWm5WdOTtZdXwVFqsFvR5KQxLQ9En2jG0A+/aRtPV7zHnFnrOhBq/j7yxHp9Jhk9nISgzAGhHVbtcxlBr4dta37Phsh0PaoZu6iqmqyZDxc7tdt0MTPAguKYeuNzbbzGazUVhViNFibHAuLDWMMa/MoDw4vkNIfoSlhnHXsbsonlTMimMrMEhKiJkpIv4aoa7jz0vz+DThJbf/21nLxCZVXkWeI2PHP86f3lf1JqirZ2v8fTT8I9bf+zc9T7xKStZzKBUy0MRB9HTQxnvUtrMFnQ4UqFFZgpGp9MSPikcX4Xl9Xf84f2RXXUFOfFeGBEwXsiMyOWiiOl90bWek/AiyYx8Tps1GZvXBJCuj1FCKzWZj28fb+OX6X7BZPVO7uO5CykvHQKsSLxyzrJzycngl+hV+vKx1G0pnSnjvcK5cfCXdpnfzPhudjPSUn7EO/87pWHSAmLOY5WVk5YhI2EFzBpF2mWsDUj4c/CHfXvCt07G6Nf4aOP6OfASGDhBJdzZjswlJsTqbmAE+AQCY5CXt5uzYMW8Hyx5chkxZuz1itVkpM4nnIahGopAT38Hxbxv7irODkCEw8E0IdXYG2DPRmlqPFhwo4NX4V9n45kaXmtPU+qg5ynPKydlfSInvBjYlT+dfS29ounHcxXDOtg6zXpHJaqVVO8Lav1W0IJUaPzqee3Pvpe+1fd1jTwukF6fz+IrH+WDLB5D5K+Su9rRJbcdURl9rNl2VYLK0TmHH/j4MrrP88o9rXJrX7vg70YKSqF3mc+pU4Ts9dAjy8pr/THC3YO48dCejHh7VCqvbl7rjy5DHp7Brwt2o1B1wC70qG34Ihq8k2FKjWFB+FH7wF7UpZxwTcvJniiSJAKm2cPw7WHeteK+3gXuG3cN3F3/HucnnOo6ZqkwEVJ1CWVXaYRx/9ow/pRKGzR3G9Wuu5+YRN5N+VzqvTHml2c/OGTyHSF0k6cXpvLWp9eUsHln2CF/s/IIKYwUr01dy7+J7AbhlwC10D+netg5YLY5yDm1GqYPISSJA1xPsfhp+7waVmdhsNvq824cxn46hsKoQnQ5OdRuL9rLzW/6edqIqIYUj/S7EP8E1dY3PhA44anVu5B2hwmhLVGbBkpFw4E1H8U+zvLxdB8/K/EoO/HqAvH15DsdfqFxGMOuhOqf9LtyRkWStih7r8VYPgl8MdhTYrY99oVVSAh6oE+qEXCXHP96fCxZdwPjPxlPgEyvqYEROarS93fGn0bjRyE6IXC6nR48eDceXkwsJ/iuMEbHfozCEcU7SeVzW6zLMVhFaF5wczKzPZpE0yb11uupjrjYjM8voknsXPYseEI997EwYPV9E0XppXzJ+Qrb5VkL8xW5ZaamIvPa0BKwdoxFSM1/h2V6/khaWBlWnoOywVwrWHcRfChcVETv4ekcNSKvNisFi4Piq42z/ZHuD7If2ZOuHW1kwZwFWs9VpIUXxHtj/GpSnu80WLzWcXARrr4KyI445o0VWQXU19L95IN2mdXOLGb6BviRNSuKw8jDzA0azM+5W4fg7+ScceldENXvpcMjlcpL6TEDu67zh5K/2R2YTntsj2WIXcMxjYxh02yCXXn/gbQMbOBNn9ZjF9X2vJzU01dnxl78BNtwI29wYLe2lIVWn4Hu9qPNXg7+P2Hw2y0soLxcZJFkbs1h09yLyD7imVtj5H53P9X9fj1xZO9cuqS7BinBahGprdsMnrYUpZ7EUuW8kdLsD/Jyj6AMDwSJV89TOaxk/b3yD7AW/GD+G3D2E2OGxLjOlyfVRC6x5bg3vp76BZBJ7EKG6UJfZ5A7sa397hlaHZsdj8J3WKaDi7xN/88PeHyioqZ+p8FGgDdUiV3l+H23tf9fy+1W/88zKZ3hj4xuw8WbY0QkzfCuOc3f1n1yuB1Mr1lM2G2TVlPKLaUZBObM0E7PVTFrNa3XVqqYlZ6urYWvN9tW550JPodzYYtafXCUnqGsQCh9Fi3a3J/XHF3vQmb2O2/1L7uejrR9RbuwAHihTKUSfJ/4/85caZ04lREwCfdMZT29veps7/riD/fn7nY7vyd3D3ry9zk7j8nTIXirknttC4WY4Ng8MLXh8W0He3jz8v3mf4KwdHcbx11gwokquIj4g3lGnsCm0Ki1Pj3sagGdWPUNRVcvRHKfKTvHsmme5ev7VvLXpLcbPG8+WU1vQq/T839j/a3sHTnwHv3aBvFbo8NbHZvXsBnT4BOj5ECi0yGVy/NR+ABRVF6GvSdr25HNSpQulKDoNTZBzgJInfEZex5+L8UShxjYjycSmqrHQsYkzdMkeCl5qY6HWNhDYJZDHTI8x9omxDsffvtwJLA8uha43t9t1Ozz5G+DovGab2ItgZ5Y2lAQ1lhtZdc/PhBzfgtns2YENoLqkmpzDOVAzR9Cr9M22tzv+dJ5PPOrQWK1WCgoKGo4vygDwS8Fg1SO3+fLZ1N/5ZMYnKOUdqwD5LVtvYfgLMwBvhoZHyF8Ph98jNPAkO+Nu5fYFV/K04ml+u/k3T1tG3t48rH8uQVNyqvbZOPA6/JYM5R2rtsM/ErkPVoUfBlMJckttBEa5sZxzXj+HB0sfRKl133hyeNFhNr21iTc3v8lVOyLYG/NvscDOXwtb74bSA26zxUsNZQch/QuoykKrFBl/FrmYbAx9dCLD5g5zixn2TPbs8mxyfFZTot0qpD7Tv4BNt7nFBi9tx2q1UnjqANbCnU7HJUlCh0izO55/5ptBTTH2ibENnIn/GvIvPprxEX0i+jg7/oIGwvAvoWfjdbW9uAl1MCReDcFDHIf81bWOPxDrnYKDBWx4bQN5e1zz/GhDtUQPdg5GK6gSjgm5RYeftkbCUuEr6iZ5cSIwEGQ2NX8Xf8eK9BVklDrXa1P7qZn6v6l0ndq4Cszp0OT6qAW6TOxCvzuHU6ERzqgwbRP3M2+tCAYo3n2mproUuxRjp8j408ZC2Fgw1xbkuv7X67n4+4sdtbusZit5e/MoOeHaGq+nQ8baDAoXFWKT2cQm8uAPIK1hzcoOjyaarzUT+LWidRl/xcVQWSli0iObKEu24NACEl9LZNAHg+jRt5iAACgshA1NxEFs2SKcIhERkJAAw2qmi62p81eZX0nOTs8mB9QfXw7N303AqX2oVJBbkctLa1/ipt9uwtpCRqtb8OsGo3+GWdkw/ZBQzwnoBWP/ENnZRz+Dop0NPvbFzi94e/Pb7Mnd43T8/1b+H6lvp/L6htdrD2b+DMsnQfGuttmW9ihcXArahDZ3y2qzkl6cTrW5GoCAhADkk8ZRFpzg8X1PO/ZAVZUK1r68ls3vbm7T56/tey1pYWkUVRfxn9Ut10JcmyHSaHuF96JnaE9kNTXOHxz5IGHa05Cpt5mhKguyfgVjG18qRz6Gb9XCIewJoqZC32cd8vx2dYiiqiJ0OvDLO0LZF79QfLzYI+bVfTbq4gmfkdfx52Jsnk65ag2+kXBBNvR63OGYKddLmPXtm4IqySRsChvF1cUAqMwh+PrSKTXTXcb+l2H9dWCpbrJJfICQFkkvTm9wTqaQseuLHQSWCZ0FTy8A9ny7h/e6vUdCegISEtqMb2HzXU1G4nulPluHzWYjIyOj4fgSOgxp0l8cKp8K1P572rFarPxy/S+sf60VgvrtTHZZHoW6v6nyPSQObLsfdre90LOX0yD1IbggD7OyKyfC3uP3U1/S9cKuRA/xfLZl/v58FBvWoi7PcURQEjYWUu4H3/aTn/ZSg7kSW+E2CrK2oVbJkVlFRFqFsQK1nxq1Xu3WuhUz583k39n/pthQTLE5B7O8XEyWo8+HiasgZLDbbPFSQ9db4JJKCB3lkPq0KsTLxp2L7q0fbOVp1dPkrhaeGoVFK56NXv8H45eJzXgvHQ6bzYa05U5ki/qA1Tkt4L6ATZyzpZIQ4wAA/nrqL7467yu32ufk+JPJIeHyBplMXtyMXA3D5kHilY5D9ow/q1I4BioqoPv53bn7xN10m37mWcfGciNFR4uwWpw3YxIDEnkpLJtRe7eItYrVLII2q7LP+Jqdmo23wPwop0j/wECQkAiQml63upom10ct0G1aNwY+OIkqXxGMHOIb0njD/PVC/tdU1vh5D9GpMv663gzjFjpJpYZoxL+3PRi8uqSat1PfZtV/VnnExLpc+tOlxK4VWal6tR5ipkPkZA9bdRqoAtmi6ct2Q+sy/uzZfmFhtRltdakyVXHHgjswW81sz97OkmMLmTJFnFuwoPHvtMt8Dh8utvvsjr/du1uuTzn/6vl8OORDj+6t1h9fdry6gqiDf6FSwYkSsfcWqY90ZBl1CHzDxVymLpUnYP01kNUw4DdSL7y8p8qdizXuzhXBDr3Ce9UejJgIg95pu6yj0k/U7D2N9WTq26kkvpbI5pPCmaYJ1uA/fTQVgbGUdZBhuW4m6IbXNrD9k+08vuJxHlz6YKveg3KZnBcnvgjAGxvfaOCErY/d8TcidgTTuk3j59k/8+CIB5k7bO7pdSDxKujzLOx9HkoPtu2zmmiInAo+zdcydBeBPuLlWFQtHH8+ZXmwbbvHgkoCl/1Ar2WvNnD8eWJc8zr+znLO6XoOF8XdwYFBw6iYPKvdrlOcXkz6ynSyc2oWSjaJrtosIiy/dbjJtFvpMRfGLqS5n2JiQCLQ+AJKrpbzUPlDVE+dCXje8RfWK4yUO1IoDBLZpLJTi+DgGyBrPM3LK/XpGuyO0/JyMTG3R0XJ5DJ2fbmLo0uOesw2U6WJTW9vYtXiBaztMZL14TU1LY99DiebWCl4cS2qQPAJQa9XILfoQIKh7w9l8B2ed6J0ndqVnKtu4LspQxm92IdKUyVETYF+LwjdeC/tS/Fu5IsHElj8B3q9DblFDCblxnIq8ys5ueUkxorT1P0/DVRaFbpwnWMMk1t9xWTZNxzCRoln2Yt7kauFU02SHCoRVoXw+B347QBfnfcVBQcL2t0Mv1g/ks9JxqgXz6PcqhPPhr4rRIxv9+t7OX1K9BOwpj4uoorrkBASgdzmS0HN45O/P5/M9ZkuqytaVVjFF1O+YMv7WxzHKk2VHC48TJWpCqOxdt4cU/E6VDZU1vDSMbBHcZsVYvPGaBQZZP6x/k7SnKfLsRXHeD3pdbZ/st3puFwmR1YVjs7QTcy1q3Nh8VDY8+wZX7NTow4Dvx5grZ0fRNTs+/lUCwfP8eLjDT629KGlfH3+124xsSWMRjAqahx/miYcfylzRQZN0AA3WtYyLdVT7OjUd/yp/dSMfmy026TDW6LcKuY4LSkXdXQUMiGV2ZqMv5ZkPp9f87xjL+qtc9/isl6XOer27dwJGc4JvhiNsGmT+P/hw8V/w8KgSxcRL9BUlqCd3lf2ZvRjo7GaOkA2XQ39npjJiV7nolRCXoXIND+tDCtXk/ETLB1Xm9FXsh823AxLx0LW70Lqc9R8EdhUj0hdjeOvrNbxV22u5lChCNROC6sjlR6QBsm3OrKrWo25Agq3nlbAjF0qc2/eXsexjiDhWJe6pSmuW30dF3x1AR9u/ZAX/n6BwqrW1Yye2nUqU7tOxWgx8tqG15pt+3fG3wAMjxU/rGndpvHcxOfwUbS93q0jOSPhcpi0RtzjthB1Doz5te2fcxWF22D5FMgUTu1AX/FyLK4uRqeD/Lj+ZF/7AHEj4jxinsE3gGpdaKPBFO7G6/g7WznxI5xcxG2DbuORfm8SVD4Sg6Hlj50ue77bw7xx8zi2Q0i3+diCGJfwNd1OnX/21vgDCBkqNrnlTesf2h1/x4obyt5JkoRKqyIwSETQeHoBEDsslq6PdKUoqEhEyQ3/EmZmNow8qsEr9XmGmMph11MMif4FgDmrL0DzrIavdtVGzN+Xfx+X/XqZpyyksqCSBXcsoHylWNypqLnZM47D2N89ZtdZhbkCivcQ6leAwiJmy2XGjhFwodQoKdcpMPgYMFmN+HqzdtyLNg5rr6eo0PRFpwOFVfw+K0wVbJ+3nQ8GfkDurly3mVNwsIDCw4VUmcRCRGb1FZNli6FBtpAXN2EqdWS41Jf6LMmq4OjSo5RmthC67QKSz0lm9i+zqYoVk1WFRSekPs2Vni9w7KVZiv2nYEt7HOTOmxLBNSXTCmv2RS786kLuL7gfSeaaLGNDmYGMtRkUHa2dHG/I3EDyG8n0f78/eTUKkeH+p/DZc5e3tl9HYtdTYuOyhih9FGMTxhJmFPKfRqOImC5OL6Y4vfiMLxcQH8DQe4Y2qoRQWSn+q9EggiD6vggx55/xNTs1fZ6GCctFYEgNqak1/1NS4/graej4KzpSRPa27AaZle7mz7l/svy27zEpxNgQVK8GqRO+4c2u0z1Bp5L6tFlh5+Nw8G3HIXuGpd3xJ1fKGffUOLpPb2MmUTuQsTaD0l1iTtNbXgU/BMHhDz1s1WlQlcP9Be/xWFDbMv6iGhFbOVJ4hBf+fgGA7y/+ntsH3Q5ASAgMqVFkXrjQ+TMbN0JVlWjTrY4/d+BA8d8DLSj397q8F6MeHtUh6j7a0fWMpTwoDpUK8irFBCJU0wFkn8vToWAjqGvGsewlcOQDyP1L1MxV6iF2JugSG3zU4firk/G3P38/VpuVQJ9Ax/kzomATLBoAJ35ouk1FBmx/qEEt925B4uE5WlQbxF70/vckbf6uwzj+6tb4C4gPwD/Rn5wKsb8doWtdJpwkSXw+63P+b8z/8fZ5bzfZrspUxdZTonDmiNgRZ2Z4/gb4OQYyfhYZ2aEjQNHJZNgs1aIcSNVJoE7GX5Wo8WdVqCgz+bhsXdFWslImcmjIFR2i1JHX8Xe2svkO2CNk9lQqCMjej3rNMiwmS7tcLmlyEue8cQ79B/Wn8P5Cpp/azF/HLyM74VOvnJvNJjY1m8Ae6XKsqPF6Vzk7c9CXidlaR5D8KDMIh4Kf2k8sCDVN319vxl/r0esbizq0wq7/o2+IcPwpbaKNPQoNEFJ9HnrZAWhCNFy97GrKxwkbfKSafshV3uwdd5GzEhakkRrwKwqr+Pff8eIOljywxLN2AdXF1ZiKc5Cb5WgUOiErufI82HKachVe2oZvBLaeDyMPH4lOB0HlIxkUNAlfhS/xo+IZ/+x49NHui3j+4dIf+GLqF1TVRCA6Mv623A3fKKG6/WqBeWmCgs0iwyXjJ+ID4nl01KMMtd4LQND4vjxS/QiJ4xtuJrQXpVVi4uDI+PstGZaMdNv1vbSdxucvsM/8B9sTrmVlaftsqgbEB/BQ2UNMfH6i41hWzXw5Wh/tcPxpA/yF+oa3tl/HIW81ZP7k+GtaWBorrlnB+Mr3gBrHn9XG60mvs/jfi8/4cuG9w5nyyhTCe4U7HV94aCE/G/7FqYCfRMafKhB63ickz7w44e8PsbHga2ja8XfRtxdxT8Y9yOSu24Jqanxpjvz9+RTuzcYkLwZqM0qdMJVD5q8iy7OD0amkPiUZHH5P1OOtIVgjoj7sjr+OxE9X/ITxRbGT76vSQ2DfzlnTU6ZA7RvGlG4zuCT1khabN5fxd9eiuzBYDEzsMpELUy4EwGw181f6XxR0fxmAZcuguk7lmqU1Jb/Gj3dWeLRnBud3vFvfKPbxxWazYagWQWb2Gn/QQTL+UubCRQWgqbl58bNh4mq4pNw5y6+RAMrGpD535Ygafr3CezmXe1h9MSwe3nb79N2g9zMi4aEpTnwvpCbr1Q+0O87sjjQAyWBAbjZ0GMefI+NPbqXkRAmn8k9htVmRkNr0fIRoQnhi7BOOTN3G2HxyMyariQhdhGOP+LQpPQD+KUI5BcBqOo0af5/A1n+Lz3qC0GFwSRkk3wLUBvEUVBWg04FkMWPJyqYko/0DVOtjszVd488TNP1UeTkt5PKOE5XSLEM/AaU/RouRMmsJurxd6I/vxVg2HN8g12dcRPSNIKJvbcSDojyQ9DIwxfY5u5/C6lz4NQkSr4FBbzbaJDGwVurTZrM1qLf005U/Ycm1wKA5Ho/8W/vyWrb+thXlCKWQx8jfKCYhTTj/vDX+WodcLicpKanhCYUeztnB2o/ERoUWsTCxR6GBWNxWFVUROyzWLbbWR+mrJHF8ImWnqqAY1JJOLKaLd4mJRmdcTHU2/FMg7XEMe3o7Mv5yl+ZSWFzIpBcmedS0ze9tpsfXywi/KRxzV5uYJZUdArk3GsBd2McXPz/ot/cL7jgXeoUD4RA92L11IAfeNhCzwcwWs5Dmk9l8RMZf0ACIu1RErXpxL37dRIZLyDCi9FE8Pf5pnlkLG4DKapnbyjTv/XEvJ9acoHSYWOkr7I6/6PNF7WovHRK5XE6SNh1W3AC9nxaSvTXkWPeSGTIPVbkFuJHCw4Xk7MohcXwiPv6nIVnUCk6WiajgKH2Uo75fYIgGoqa2y/W8nCZjfgOZusFhu1yS0Sjk7Ec9MoqQHk3INLqANSfWsEXxBgl60GovaLfrdDpKD0D61xA7CwL7OA6npcG6DU1Lfbq6ZnCT66MWuGLBFWzZAt/9L4Ge8vMZGDWwYaPCLbBqBvR7CVLudYG1rqNTZfwBTNnsNH+rL/UJwuEmySVmfdZ+pWdaw7inxzFvzzwASrRdYZx76866DHUw2vMP0NqcoKYy/k6VnWLxkcVISLxxzhuO33BxdTHj5o3Dho3LYy6lNDOGBQvgggugoAC2iqQkJkxw/r7QmmV/XgtxfHn78vj9lt/pd0M/+l7Tt5W9cC11xxdTlYnlY58lPqYfyn7nO4KsO0TGHzgrKviENtxfWToWKk7ADOfyL1F6ccPtcyOore+XFlpPvlGpA/Np1DPUREHaI8236XE3qAKAmr2AmucsXCf2uHLKax1/A1+7kmUvQM+OIV5Um/FXWcyr8W/Q9Y6uECqcws058ZrDYDZw35L76BbcjTmD5ziO78wRcq7DY4ef+fu0y9XiD4jM7O/9IXx844pcpnJYd6WQ+O77fO3xrF/Fn37/PTNbXMTMHjNJCEhgXMI4dDpQVxbR46/32PD2aCY/N86ttphMEHloFUZff5TKPk7nPOEzOptdLu2C1dpxdKibJeocAF79+0UeWPoA3cZdz9C8F1D7NVxktQf2iCCf9lnXdx7UIRA0EAJ7N9kkzj+O87ufT2JAIgaLoYF+84j7R7BxgxXSPb8AKDhQQPn6cv733P8I0ehg8RCIvwxGND5p9kp9tg6r1Upubi5hYWHIZHWiZCUJAnsjq/GRqC0NHX8L7ljAqa2neKDoAXea7MBmtYFUKy3pI9NDyR5YMlxsJvf0Smu1O7ou0PtJzBk4HH9R70Vxab9LPWwYRA2MIrNnLGX6MkJVEeKZnt7GwtJeTh9zJbblkynXD0erfR6QebRY+oCbRR2dl78XEcRyu9Rn1xvFHy/uRxPTYJy2v7PLymwcXXoMha+i3esnHFt2jM3vbMbcz4bM4osKndgXGPxOu17Xy5lhtVopycskoHg3ksG5FmR8iNjQKbPmYLXCnu/3sPzh5dy85WYi+5+5M7c8p5ysjVlE9o/EL1psVtV1/OXUBLhHhBnBpsRtXmwvLSNvfIGoUtuwAUajuFfjnjrzjRybzcanoz8lcWIiY/9vrNO5girxzKrMwSJI8fh3sP9lGPR2h6v75lbKj8LuJ8X7oZ7jT7NaOP7smft1qcyv5Oiyo0T0iXCJw7bJ9VErMBohvGQ6PSIhObiRBn7dYPD7ENrxMso7VcYfgNY5+LQxx1/JiRIkuefH4N5X9mZu6VwuKb2k1TJ9nR2zGbJryq/Vz/iL1EdS+UglGSUZjmB0EPdwSMwQ1meuJ3zkQkq/uYlvvxWOvhUrhO+mZ8+GjsSQmp99S44/hVpB3t48qgoajiPuou74YrPYCBqRwvGiCCJUkFEpIodCtR50/FlNQiUn6QaIb2FNHzK0eanPOjX+dufVOP7C6jn+hn5yZvY2hySD/PVCovTCfFCLQTlcW+P4q5PxZ1+DdJSMP7vjT6lRMuCeoRT2KIRTrZf5bIzv9nzHGxvfQCVXMTx2OP0j+wNwx+A7uCDlAipMFa4wvRZJJhJR9E0E0uz7L2T+AvH1sjWGfSayBD01f7bZ4OQfoAqC0OFM6zaNad2mOU5ZdXqyuo8jfFi8200zmSDi8BrKg+JRqZwdf57wGXkdfy7G1snqjOhUYuQs9S+mvDSE0wxKaJGvz/8ac5UZ7etaFh1aTIZ2Kh/1f43ALWaYurp9LtoZkGQwcUWzTVRyFb/M/qXJ872v7E1+FPA/zzv+pr8/nenvTxd/MVeCKk+k9zeBU90ML01is9nIzs4mNLSRyaWxmFB9NRCByiTO2+UnQGTQVOS5eHLQBo4uO8oXU77A50ozJIGvTA+aaBFBGzbWY3adjej1OKQ+K1WVKHw8PwXoMqELewYcpMyvjARlV0+bc/YhU0H5EaqtcXWcOeK/JzefZOGdCxl+33BSLkhxq1nx/vFESn1RmyI7hDyGl1oO5B8gT1WKVepDRYWSb677htjhsVy1+Kp2ve7EFyYy6uFRFFt0HP3+ObS6zjXfPlux2Wwct/TBb1ZOgwjX+FAhgWRQ5FJSAt2ndxf1UeL9XXLtrA1ZfDPjG2bOm0mfq8Wiu67jL08ETjMp6F/w/Zdw/jHwab/sMS9toPIklO4TwZEq8TxEvxJNtiaPccojGI2uU7EwlhkpzSql/FTDXUS7Y0JlDhFrFWNZTS2Zs7xaSugIOHeXqAtUh7Q0CCgfyrlbq/j23w2dtwUHC/hx9o9MemmSSxx/za6PmmH/L/vJz9ICsU3PMXwjoetNZ2xje2DP+CsvF5uL9kzYDouhQPxu/HqCTM7IuJG8e967JAcnO5pct/o6DxroTLRfNNF+0WKj+8QnQkpReRqZTp7EaqJ05zMct2kg6lx6hfdqsmluLlgsQo4uuBEnuEKmcHL62Tm367msz1zPMfkCkpJu4sgR+OIL2FWj1jixEUVku+OvqkoEgDel+hTYJZD78+9vqZftSt3xRaVTkXD/JSx+H7or4dnxz3LrgFsdUpkeoXS/yEwOaYX8Zt0srTr0COnB7tt2O/XjriF3MTR6KGMSxrjKUlg+WQQiD3634bn0r8A/FRKugODBYm1aQ2MZf+bMbELTM6jUpgGuV6prKw45x2A9U+6dwifbPoFfOaNn48reV/Ljvh/55cAvXPrDpWy9eSt6tdjDcckzZyqDrXMhZhZEnyuONRdIeewz8O8JI750Pq7Ue16NZ/WFEDEZxv7mdFiSQBPow6nk0ejTmvhsO2I0wt7Rt4JMRv0EP0/4jDy/6+fFM6y5FLKXokv+HwA2awW20jJMVb4ofV3/WCh9lWCD1cdX88G290jSBpBfGYtM11Br2kvbsS8AOlTkn0LTojSKPVLHK/V5BiwezhSFxGfsQW4Qm2h1a/z1vKinpywDwDfIl5QLUtjpv0z8Xa4TEcIdTDbnH43FACvPIdkyit7pHxEXK3FBiETm+kyiBkYhU3huA8tqBaNNOKa1Ko0o7p27EsJGN9hQ8tIOyBRYZ2SStWsXumLYFXc7yzK/omLD08xiFiUnSjCUNl2D1tV8NvEzogZG8dLzL6FfD5uKanTxdz4BNhP0+Y/bbPFSg7EYlk8Skpq9HqPve32pNlczXnmMiooEpr07zS11INV6NWq9mvwalSK1SoLKLNjxCMRdDNHntbsNXlxLpL7W8VdQAF3TwghLc129nIh+EcycN5PYEbVOInuNvyh9FJtq9pFs+hTQjAR1kMuu7eUMOfGt2JSauBrCRMaVwWzAKpkwyUscjr91/1vHnm/3cM3ya1BqTs/7ofZTc9fRuxrdiCmoFBl/SnvGX+wNIrvibEfpBwENd9KCgiA2WkFWloK9e2HwYOfzoamhXPjNhUQNbLr+uzuYf+V8fLvHkNMjmGxVAGbr4IaSbFYz7RYRfYbodKBQiEyt4uJa+cQOy45H4fC7cEEu+ITSLbgb3YKbDgz2FMYKI+/1fY+0y9JENnHWb3DkIyFD2NmwWfHb+xQ7S+GnoI38eMmPTTa1y3xGR7ctcefc5HN5fOXjLD22lNU3GHnsYRWLFolzajWMbCRZ1sdHPL/l5aLOX2faA6pbrytSH+lZpx9AQC+Yfgjkp+/8UivUpIalOh2bnDSZyUmTnRuaq2D/KxA6HMJPI9PekC+ysupjKoMNN4m+TFkP4c7Oxli/WK7odQUxfrWpqHnrDhG/ezmHQqPpUI6/Gn+lvV7imWT8SZLExzM+pu+7fTlceJjhHw/nnK7nMK3bNEbHjz5Tk0VZlSMfgiau1vHXHNMPiACOupQeEgFa0dNEMosnkCQY+qmjvmWVqYrDhYcxWU30j+yPXi/ekZ7IDjWZwKANQqXqGIIiHg1XW7VqFdOnTycqKgpJkvj555+dzttsNh5//HEiIyPx9fVl4sSJHDp0yKlNYWEhV1xxBX5+fgQEBHDDDTdQXu/O7ty5k1GjRuHj40NsbCwvvvhiA1u+//57evTogY+PD7169WLBggUu72+Hwi8FQkegV4oQ/277/eiz9BWOLG+ox+8KLvr2Ii777TLyq2ojJ9/Y/BmykZ1UN92VVByHtVfB8W+bbGKz2civzCe7PLvBudXPrmbVpW8hN1V7POPvyJIjrP5tNSvTV3Ki5ESL7b01/lxA4lXkay8DQFbZUOrT00QNiOKSHy6hx6BpdMt6iq7KUS1/yItrkamgZA8a6SRqcyjm0hDWPb+Oj4Z9RFWR5yRUANa8tI4hy08RlTedYTHDoWAjrLsa8v72qF1nI1qtDatkpMpWQqmhlJghMczNmkvfa/u6zYbCQ4WUZYmUQ0exdCWQ8QNk/OQ2O7zUQaYCQx5YRIq+XiWcfGZ5GeXlQnUgcVzDSHBXU3y8mKKjRRhq/NBqNcLxd2welOxt9+t7OX0kmwky50PeOqfjYVrh5DMq8sjJdb3sjX+sP32u7kNQUu1mU2M1/sxJd8G4hZ7buPDSkPAJMPAtJ2kyfx+R+WdWlDiktSpyKijNKKW6uPqML9lYvZz8GsefyhLsXavUx1gEVQ3XpWk1/kB71k9dfPx9SLs0zek36QlmfDID/xl92JR8Hh8zArO1XiCypVrUO9pyj2cMbAFJ6qBBv00RPQ16PQGypp3zmRsy2fHZDlEiwkNYjBaUGiVbs7fy8tqXyUi8E87dDUrXZKG7FZmKX+If5PFCMFlMzTat6/irz/W/XM9V869iX96+Buf6RfYjXBtOubGcQt0aRtQpKDh8OPg24ZOxZ/3l5zd+3k76X+ls/XBr843cROHhQjLeW4Cu4LhnlEgMhSII79iXYtzd8RhYLSJgSdEK51fBJth4GxTtPL3rV+fAzkchq5H6b63hnK0w8puGxxU6GL8E+r7Q6MfCdeF8ccEXPD+xNmOx92VpHBh6FWWqYMdcwJPYbbDuP8g3M77hav3VHL/7OE+Pe/qMvjfIN4hvLvoGH4UPu3N389Lal7jnz3v4bMdnZ250YD8RiNHt9tpjpxaL7LnSAw3by5TgGyFq++58XBzb94Kog1vo4d9owmWO+uGbT26m97u9mf3DbEAEGSRt/o5197l/D6G6woKyqgS1rPnx1114dIVTUVFBnz59eOuttxo9/+KLL/L666/z7rvvsmHDBrRaLVOmTKG6unZyf8UVV7Bnzx6WLFnC77//zqpVq7j55psd50tLS5k8eTLx8fFs2bKFl156iSeeeIL333/f0Wbt2rVcdtll3HDDDWzbto2ZM2cyc+ZMdu/e3eY+ubpodbvR+wkY8yu6mpThnPBcchIG4RPavlHbdSVTzvr6fnbkGkj/Ago3N9nkmVXPEPpSKI+veLzBOavZilwhIVnNlJYKqQZP8cetf7D47sWMmzeO1avvgkWDxUSjCexSn97FdPNIkkRQUFDj40vqQxREPgqArDyGad2mcW7X2sidrR9u5e20tyk4VNDws26kn2Y63U49RrLvMKETvmiQkHLy0v5IElyQQ3XvDwAh5ZgyK4VJL03yuNxn3t48QrKLGXBsPi9NflFEEo6aLzL+vLgF6eTvRNs24+cnobCKYCCX1w5oJXcfv5tZn88C6hRLVwFTN8PkdU1/0Ev7odDAjHTo+xyAQ2rGLCtzBO+4g1+v/5X3B7zPU5v+xcau51GoWQ/Bg+DiMuh2h/sM8dImJEkiKMAf+d8Xw6G3nc7ZHX82mZljp4o5vvo4r0S/wo7PdrSbPdf3vZ6b+t9ErC6RgpppUZjrkgy9uIrA3mJDSlO7Ex3gEwBQk/Enjk18fiJzs+aijzr99euhBYfY9dUuLKaGC6i8CrFu9bEGi3dR1u+Q3sjm5dnILwmwrqHEc1oaHI54kSeOjePXA782+lFXOXeaXR81Q8+LemLsJqQbFfjgo6i3KWEshvDxjdbE6ijYHX+eDvptFdHnQa//A1UAABarhRXHVvD9nu8dTqnN72zm52t+xlTpuU1S30Bfbt1xKx/1+Yh7l9zLkcoiCEjtGOkabUWSKNJ256gJTNbTc/zZbDZ+2vcTX+z8Aout4fgok2Sck3wOAAsOLeC668CmKsOgyG1U5tOOPUO1pTp/G1/fyO+3/N7o2OwO6o4vxenFFC/ehE95HkolPLb8MV5Z9wrlRjelElVmQP46qMqEvS/Anmfg1KLWf77ihMi6LW4YkTFv+zxu/+N2Xvz7Rbad2savB34lqzTLuZFvJEzdCt3mnGFHasj8Bfa+KH5bocNFpl9VNvzeQ2QIN0NUaiDloV2wKtUdos6fPVCVoiIOLTgEBojzj3PKUjxdhscO5/Cdh/l0xqfc1P8mNEqNo0bqGSFJ4BPqqKUICDnmzJ+h/Jhz24rjov6iuRKOfy2eP1M59HoKBr4JwQPP3B4XEawR/bHXZ9bpQGGspLrQ/YHu+QcK6LPsVcIONgxm94TPyKM7fueccw7nnHNOo+dsNhuvvvoqjz76KDNmzADgs88+Izw8nJ9//pnZs2ezb98+Fi1axKZNmxg4UDxwb7zxBueeey7//e9/iYqK4ssvv8RoNPLxxx+jUqlITU1l+/btvPLKKw4H4WuvvcbUqVO57777AHj66adZsmQJb775Ju++24gOcTO0tbC0p7HX+MuMzSAj7Vz82qHEUnVJNZvf2Uz86HiH48/PpuPy1Efh1GiInNzCN/zD8QmFCwualRiK9ReSNseKjzU4N+bxMYx6dAxLZooipiUlQmrFE0x+ZTJvbHwDAH+5XLxAmoigNpvB7sP3Ov6aRyaTERcX1+R5v5qyA9aSaH67zFnf2ma1YTVbMVd7Rlb30MJDHFpwiMruIwB/sXFirhSRY62JUPPiMvR6yNMvJTvwR5YFDuLme6/3tEmMefl8Pi08H7kcoX/uGwmxMz1t1lmFbMeDhEoy/KJuRG4Vg3G5sRxztZm9P+4lMDGQ2OGuq6fUGsbPG89W9QnStF+hVA4GuY/448Xj+KnFC8ee8bfh9Q2sfGIl16+5ntCe7ac31uuKXiSMT+CTgnvIDdiKpWKOWLjWKFd46ZjIZDJiE7uB9BXok53OqeQqNFIAlbZijuXmMKFLAP5x/ii1rilYtf7V9ax7ZR1X/nkloSni2XxszGOAyDSw2cDPp5jAA3dD7CyImeGS63ppH/zVNRl/dRx/rmDDaxs4vvo4abOdpSttNhtF1WLzyF8ZIvb+9z4vJLISZrvOgM5K8h3g21CyMy0Nyn32kqlcydbMSZzf/Xyn819P/5pT204xN3PuGZvQ0vqoOYqriwHQSAENT/pGNKgX1NEIDBT/7RQZf/WQJImJn0/EarNycu5JIvWRDJ4zmJQLU5Cr5C1/QTtTaigFINhSKpzANQ7LzoYaG2rp9DP+TpWfosRQgkySkRyU3PCDiDp/n27/lI1ZG9mU8Dsr+18BNomAuF1A42uH1mb8jXhwBIPvHOyxxIq640v8mHgiX/43W5eosCkqeWb1MwDc0M9N0s+BfUSgm9UE2CCovwh+ay1RU2FWNqgbOo1kkox3Nov6bh8Hf8yBggPcO+xeXpr8Um0juRqC+p2+/XlroXgnJN0EMjkcfAtyV0Hy7bXzeFUQIGsgXWqymMityMXfxx+dSockgVZjo6LMSkWF3GP7nnbs85Ggc4Zw11eDm298GkT7RXNN32u4pu81rvvSop0isFNfZ/M//jJRZ7F+Znb6l6KswuQN0P8VUAaIe6bUdYzAy3XXQM5ymJlBkK94GIqqirBYLeh0cjYNv5Zh13rALh81OYlDkEU2dAB7wmfUMYXLgWPHjpGdnc3EOuEi/v7+DBkyhHXr1jF79mzWrVtHQECAw+kHMHHiRGQyGRs2bGDWrFmsW7eO0aNHo6qTkz1lyhReeOEFioqKCAwMZN26dcyd6zz5nDJlSgPp0boYDAYMdr0hRGYhgMlkwlKTciVJEjKZDKvV6lQ3wH7cUi81q6njMpkMSZIaPQ5gtVpbdVwul2Oz2cTxnGVIJxeiDxE1USzyMmw2G5WVVvz9a9s3ZXtb+lSaUcqyh5Yx6rFR5AeJN3yEHKZ3+Q/WU9XYwia4pk/1jte30ZV9cvl9Uvg7UvUa61Ocn5h0pBenN/h+uVwOiPtWVCRRUGAlIMAzfep+fndyDbmwD44GjsYySMiXyqFBn0SEjuirWm11ylTssPepDu589qxWK1lZWcTExKBQKJxsl458QNyxP/BRfE1pqQaLpfZ7ZDIZ/W/qT98b+gI4PufOPp34+wSb3twEj2sp9Q3CIuuCpecj0PMR0d5q/cfcp5aOe7RPp5ajrs6lXHuS42HvsuhgMTcPut7jfaquBptNhkoliT5ZLE6RtWfdffJAn6z9XyM3rxC10oTcUuv4M1WbmH/lfPpc14eoIVHt3idztZkDPx8guEcw6cXplMiPYbNZUCiskL8JiyrUqe7j2XafPNknjn+HTaaEmJm1Up+yEsrLrfiG+BLWKwyLWdjcXn3qd30/MU99SYT3+so1WMpOIDfmYtX3wFZvo+BsvE8dsU/2+Utc3KUA2Op9//u9DvLFJxoY5ENYb7h2zbWOTb4z7ZNSo0QXrkOmcj4nk8koKxOOv/iQo0jH5mHVJiJFn9+qPtU9/k+5Tx2uT+VHkK2eBUk3Yu32L6A26MAkL8JgsGGxWKksqOTokqOEpYUR2SfytPo08b8TKTlegtVmRbLW9slqtbL4nHQef76IkEBRy8nS92UhcVlvrX9W3qdeQsZMVvOddhsDAyFYFUUmsDfjZIPvCesdhtxHjtlsdtrQP50+mc1mMjMziY6OdtjXUp9KMkr4aPBHlA0NgQGglQd2yvvk7y8BMgoKrA3Wfh3u2cvfiLTtPqzd74Ho6UiSRLBvMHmVeeSU5xCmCSO8XzgR/SM8+nsqySxh/4/7UZ9QQyD03HoNtqN9sI5f1rBPnWAsv3jfrfhFwgsWo+M6jd2nzEwZIBEZ6fws7cndA0BSYBIKqXb/oW6fJiVOYv316+kf2Z/jJcepMIs90W/3fMPcoXMbtT04WKz98vLEe7ipPkX0FzXSbNiwWCxuH/esVisnT54kNjYWZGBSa7DIJaok4SlVyVVoFdoG84u23qc29UmuEsfjLhcHLZbW9UnyAZUP2MSYXffZuzztcvbm7eX5v5/nQIGQeewZ0tN538hcgdViFtKcNeN2W/okHZ2H7Mj7WGMuxKYKgj4vgSoASa5BVvPvZUMO54iMxLr7QxM+m8DqE6v5+oKvuST1EoqPFdP9u7fI7jKMkpJxREV5dtwzGKzYbBIKhQ2r1cYDyx5AJsm4c9CdROlrg2M60hgh23ATUuVxrDNP1umTAmxi39apr2ETkPrIkPn3wCrXYTMUIx35DFvcJUhylcf7JPlEIumSsZpNDsefDRsFFQVotSHYbBLl5ZLb5+WKID8yUqcQE2NrMEbUb+sOOqzjLztbaMaHh4c7HQ8PD3ecy87OJqyePotCoSAoKMipTWJiYoPvsJ8LDAwkOzu72es0xnPPPceTTz7Z4PjevXvR68WmSFBQEHFxcWRmZlJYWOhoExERQUREBOnp6ZSVlTmOx8bGEhwczKFDh5zkTLt06YKfnx979+51eki6d++OSqViVz0R/V69emE0GjlwoFafVy6X06tXL8rKyjh69Chh+b8RlfsmJlMaV/W+ip0/BxK/4StWvBFMr8si0Ov1JCUlkZub6/TvcDp9CkwKZOJ3E5G0ErlbRUGNvKJuPL97HzN6ZGKoY/+Z9MmOj48PPXr0oKioiIyMDMdxV/bJ1fdJZinDr3wdJt8kkgdf2KBP1Hzd8eLjbN+5Hbkkd/QpwBrA9h+3IytJoKwshE2bslGrfT3Wp+KqYgBKckvYtWtXk/eptFQLJCOXG9m7t1Y3viPfJzvufvZsNhuFhYVotVpCQ0Od+hSdvZqQoiVolKWk51jZtu0oJqpQyBSkpaR5vE8B0wOYMmYK09bMpiD1OH2KF7BrV20x7H/SferIfTJtfwJFyTZkJhGhaPnbwvv/e5++D/ZF1bU2MMbdfTqwuJg88wK2pLzJvQtv4yHSCShdyt7kP1BpQ866++SJPslDxrJ332qs1v2YK8W0sMJYgVVhZfArg9FEa5ody13VJ61Jy/wr55NyfQplyaJP1eUWqivzYfFQygPP41jkM2ftffJkn8I334tF7sfBoiRUiPGi0lJAZmYx1p5W+r3WD2W0iBBt7z6VGkoAMJQZKNjyGmEnX6F06BLSS2ujmM/W+9QR+2Sz2SgpKSE2Npbi4uIGfUqOTsJSXc3+/YXs2pXl0j4lz05mwM0D2LVrFxm7MqgwVVBoLGRI6hBKS4MpKyvnYF4cO7uvQjJBT6v1rL1PHa1PSlMuKTax0WPvk7VS/N0kK6GiwsSuXXsp3l/MX1f+Rc85Pbn4jYtPr0+WbIgR16nfp8wDvpAXjSzQCChJLw2hrEwNebu896mZPoWoRRDR0bwsp+NyuZwJ/5lAaWmpU0mV0+1TVlYWR44coaioCEmSWtWnovQifGN9OWIpBkAjC2jQp1Tjp8h1MewyOusVdqT7pFIlAXoOHMhl167a7+mIz15u9klCCndw8sg2CgsTiIiIIFgjHH+bdm+C0Ib3yRO/py0LtrBu7jqCzwvm6KCjVCVcg0UWybE639+ZxvJj+qGsyF1FiST2ZBq7T9XVEidPdsPX1xfIYteuAsf3bMzfCEC8Jt7Jnvp9UqNmT8EeunTpwvV9r+fj7R/z+ZbPmaSd1GifSkt1lJWFkpurp7ra0Gyf7MpFWj+t28c9m82GyWQiJiaGbWu2cWKbDxUlkRzNEXXyQjWhDUpDtdfvSV/+Nzq/YMJ7zz7tPqkNxwEL0SkTGzx7lwRfwrGUY3y7TwTt+5T6OO5Xr169sBx4D9XOezkU/yEV2gFt7lOwZgaxg/pTenA+6TZ7VlwRQUFSi/dJbVYDsO3QNiZFTUIboqUyOp4iuYYdOw5jNld5dNwrLCynrMxGxvqDLCst4r2C9yg3l3NuxLkUWGp/Tx1pjAjTX0hUoqZBn0I4TExUBLnWrnV+T2qCgi4nTulH5okTWI99SULWwxzPzETd/XrP90maDSGz8Tl8hB49eqBT6ig3lbNuxzoKC/sgzzaTv6iEk2PCyS+uTTNu77G8oqIHNpuakpICdu3K4tEtj2K2mXl1xqv4m91fN1ay1XdbewhJkpg/fz4zZ84ERN29ESNGcPLkSSIjazeJL7nkEiRJ4ttvv+XZZ59l3rx5Tg8CQFhYGE8++SS33XYbkydPJjExkffee89xfu/evaSmprJ3715SUlJQqVTMmzePyy67zNHm7bff5sknnyQnJ6dRexvL+IuNjSUvL4/AGu2FDh0RZCgEcxn4RCJX+XLzhflE/fQWqbeN4oI3xjjau9IzXm2uRve8SOWesq2QgWmBPPlkB4pG8+R9Kt6N/M++WHv8G1n//zbok8VqQfu8FrPVzLE7jzmkPwH2fb+PHy/7EcuFF7G1uid33mll0iT396mqqIp3U99l58CdfD74c9ZOvJfB0YMh9oJG79Phw3DvvXKCg2189FErIoU6wn2qZ6O7nj2LxcKePXtIS0tDqVQ6226zgqRg1iwJq9VG7vThbMpez2+zf+O8budRnl3Ovp/3ETMkhvA+4R7rk/9/wqmwFfBC3C7uHbURmzoYoqb9o+5TS8c92qfsv7CZSpn6SjlLAy7nvMwLmbh4JOe9ex5dJnfxWJ/e6vsRGemZ/Pfep3lo5EM8E+GPdOpPrGP/BEl29t0nD/TJYrGwa9cu4uNTGTf3c3Ym3si5Xc/j98t/c2ufTJUmji09hn+8Pz0X96TEUMKYXft4/+l4uvEWFm1XiJ7eqj7BP+8+ebRPOStA5gMhQ7li/hV8u+dbep74H13z7+LHH61IUvv36cdLf8Qv2o+LQmdTYSnhX9I+XrmlAHn+aqyJ12GrJ190Vt6nDtgn+/ylb9nTULgZ6/RjTu0zMuTMmWNDr7fx6YcmNry6gdCeoaTMSnF5n347+BuzvpvFoKhBvN5rA888A1272nj5ZVub+lT3+D/lPnWGPr287mU+/Xsh6n3XctPgq7nlFiuGMgNHFx8lvE84od1D2x6pX27AarWi0qoa7dOqVfDyyzJ694Zn/wMWswlktVKEZ/N9ko5+jJTxPYz4FlQBTjbO/eBnXs+5iATFUA4/uKbd+mQymdi9ezepqanI5fI29emaVz/ly/Ib6aM5hy1z68h6WqqR/RwJ4eOxjvyxwb8BdIz79PvvEh9+KGPYMCsPPNDBM/4aOT5m3hjWnFjDNxd8w0U9L2L7J9tZfM9iLv/jcmJGOEujuatPVUVVHN92nKELh1LmV0b5Q+X4Knw77Vj+6/5fmfHtDAZGDmT9Deud+mq3/fBh+Pe/ZQQESHz2mXOf5iycw7tb3uXFwTcyd+QjoIltsU/Z5dlEvxKNDRtH7zxKnH9cA9t374ZHHpERFQXvvdd0n05uPcnHwz5m7JNjGX7/cLePe/b5S+/evVn38jqWPbCMvcOvJ/nm7fwn/Xz6RfRj042bzvg+taZPskV9wVKFdP7h0+6T7KcQCOwLE5Y3+uyZrCau++U6iqqL+OXSX1DIFLV9OvUntvRvsKU97lBfaXOfVk3HdmoR1nN2g193Jxsd7XOWI+WuhNRHkCk1WCwW/rXoX7y9+W0eGvEQz4x/BplMxkMPWdm1S+Kee6yMHevZce/FF62sWiUxPvsrqvak8+iDokZh4X2FDqWCxu5HS8c90SfZrwlIvhFYJ29o8vfEsS+RMn/EOuxrJIW6w/Up8bVE0ovTWX3tavK3DePPuUuIOLaBfx37F36xDe9He43lC14/yrIXtqCcPJrnPwkn6KUgSg2l7L19L+HycIKDgykpKcHPzw930GEz/iIiRGp3Tk6Ok+MvJyeHvn37Otrk5uY6fc5sNlNYWOj4fERERAPnnf3vLbWxn28MtVqNWq1ucFwulyOXO2uT2x/Yxtq6+7gkSeK4JhRHeBUgDw1my3mPc9HtEnU/1pTtbemTscKIJElofDUUPVDEFz/ls3K7gpiQHORSMMgaPoan1adW2uiKPrn8eFAvGPopsjDhdK3fJ7lcTpx/HEeLjpJRlkFCUILjXMLYBK5edjXz14YhbZAoK5Nj74o7+yTZJIKSgyj3ETJcaXk/Ii/4GRIubrRPdr+5Vuua++fR31MrbTyTPtlfNg1tEf+v00FZmYQCIXdWaixFkiRKTpSw8PaFTHxxIlH9nWtxuKNPRceKsBgtGKzlIIFe7YdsxwPg3xNiZzRri6uO/yPGiHq0uU8RYmzx91kAwKGUY/z+wQ+NtnWVja053vO2sfzw65sAaJVaZKkPQOoD1P3UWXWfPNAn2ebb6XNsHtaeRfiaYwgoH0oXvx5N2t5efZLr5aTMSgGgaoEowq2waVBrfCH+Xhrv0dlzn9p63KV9ipzg+PvMHjNJ8OvK6n2DsVolDFWw+dU1BCYG0vvK3u3Wp9xduZiqTFQFiTmGXu2HPLwHhI+gccvPwvvUTja29Xj9PkmShE3XFZnN3KD976c+YEfi30QXXoHROJYVj66g91W9SZmVcsZ92vvDXoqOFjH8vuFIkkRBlYi8DtWGUlkpIUnQJWQ/8jIz+Kc6yVe11Cc7/6T71JKNnu7T/SPvp0v2/cxbI2rqyOVyNAEa0i6prc3X1j4d+OUA86+cz+xfZ9N9enen9jtzdvLK/g84EZrGMO0tYCpG/kOQqGkz8I127WtdOux9qkyHvL/BWAjqQKf28UExkAPF5lMNvqfgYAGb3t5Ez4t6Ejcy7oxst2+41d9/aU2f7JKEeoWz7ci1cGEeGAo69H0KDhb/LSmR0ZiZHf3ZC9GIYJ3C6kLkcjl+UX7EDItBqVV67PekDdYSOCSQsr/LkEkyNEoNktR4fbnOMJanhqXy/ITnidJHOZ2va3tOjnj1RUc37NO+fKHIdF/Rh/Dbh3B5Q6dHfSL1kYyMG8nqE6v55eAv3D307gbtw8PFNQsKmu9TQGwAyecmE5AQ4Gjj7nHPfv/jhschGzsKkyKAKklkB4VqQ933exrwGpjLm23foi1pj4FPeJNzHblcztcXfd3odxA1FSlqaoPDberTkA+QTv2JPLBn0+1zlsC+lyDpOlB2RS6XE6ETe/J5lXmOdn5+MiQJqqvlTuOfJ8Y9k0nYEn3hMIJvCIdcUMvVBPgGNDp+dKQxokGf+j4PKv/a44YC+CMNetwNPR8Qx5OugqSrnNblHu1TwSbI+g2SbgBtPMG+omRIsaEYf385+XEDCBmajC5M16b7eqY2lmcUE5B7ELNtGBXmCkft2Bi/GMyV5ka/oz1paq3scRITE4mIiGDZsmWOY6WlpWzYsIFhw4YBMGzYMIqLi9myZYujzfLly7FarQwZMsTRZtWqVZhMtUVtlyxZQvfu3R2ZecOGDXO6jr2N/TptoanJQYfDUg3l6WAsodpcjcWnAKtkoU4So8tY9/I6ntU+S87OHAJ8AlBXdGVs/NfcGhYB2cta/oKzAUkGXa4BXUKTTRIDhGTtsWLnaGldhI7E8YkExwpZlaKidrOyWbShWq5fcz1bRonf44nuj8GQD5psL2r8gVbrDus6N5Ik5GsaHV+qsuHkn0QFieAFrUyMa0VV4kEITQnlysVXkjY7reFn3cCify3i3d7vYpbE4OLvo4fRP0PfFz1iz9lOgEZIUVeYyjxsiSBkeDeOdBOyCxqlxsPWnKX4dccYNAYZZrpYpzBy/zru7/tfAN5Oe5uPR37sVnMsVgtGi6iULrP6UqdEsxdPUhMNOTttNs9NeobQ6uEAVFbLWP2f1ez+endznz5j5hyYw4W/XIi1Rvtc7+OdPHQG7PMX+j4PY35tcH79qb/IDJlHqe9OisoU3LT5JiY8N6GRb2o7Oz7bwYrHVjjmTrkVIlg0TBvmmINOiXgUFvYFm6WJb/HiMY58AulfOR2yvw/qLOsBsFlPT8BIH6knbXYaIT1CGpzbnbub33Pf5GTQt2KtYjVD7AXg75n5dIej11NwaQXokxqcSggWQdultpMNMiSqCqvY8NoGMtZlnLEJza6PmiB7RzYrn1xJYt5Aep54lbHBlzdsJFeDJqrh8Q5EQID4b3GxJ61oAxk/O+37hPiK35w9ICP53GSuWnwVUQM89+9uMVkoqRRy4sO1GqSV58DJhR6z50xJSn+HBwJMXNXnqibbZGaK/0ZHNzxntpqRSRI50ZfCwDdbfd2Le4qg7x/2Nh5gandaG41QWtr092jDtMz+eTa9LuvV6mu7krrjS9zIOCxjxmPy0VNuywPEXMJtRIyHmPPP7DtS5kLiFa6x53TQREPS9c236XYHnLsbNLVBIeE6oViVU1GbqCPfsJaYvYupqGgXS9uEfT4SOqQL+ulinyVcF95x/QInvoc/ekHeuobnEq+A6Gm1fzcWiQxPhc599rWVwq2w+2koPQjATf1v4plxz5AclIxeD9X6UMrCklBqlG41K+y8QWw59zEUXeLIKhWlDPzV/ujVeo88Gx7N+CsvL+fw4cOOvx87dozt27c79FbvvvtunnnmGZKTk0lMTOSxxx4jKirKIQeakpLC1KlTuemmm3j33XcxmUzMmTOH2bNnExUlJg2XX345Tz75JDfccAMPPPAAu3fv5rXXXuN///uf47p33XUXY8aM4eWXX+a8887jm2++YfPmzbz//vtt7lNTHuMOR85fsHIqDP6AiB/upSSwhBmlq8nbnggDGnnznwFhvcLofWVv9FFiICwshKLSHqQrbyOhkcXCWY2pTEg3qhrq/s7sMZPU0FS6BXdrcM5ms+GvtwEyjzn+7Dw++nFyK3IJiT0XdOFNtrO/qL2Ov5aRyWRNZyDnrIC1l9Mr4gcOHL8QX6nG8VctHgS1n5qkSZ77naVdlkZQ/yDH3/18dBA6wmP2nLXsfxV2Pkqq30dQDOZSA9vnbScsNYyogZ5bZBuNYJFVAqBVaWHHI6BNhK43esymsw1Zz3/j0/PfgD1zuDYwI3pQNEqdeybK6X+l8+3Mbxn333GOY3KrLz7VO2HRDdDj35Aw2y22eKnH35dBxo9wqQEkkSml00FJCVRWSty85WZ0Ee2/KKw0VaKU1JhsBnQqLWy4GSqOwfgl7X5tL6dHs/MXxCIYwCwvIS9PYoALN32nvjaVqsIqx9/zKms26zRhVNSU+ThsvYbufYaAzL0bAl5awe4nQR0CCbWOGbvjr26g6iejPqGqsIrb99ze5kskjk8kcXxio+cKKoVDQmUOFmsVnxAY1bRSwlmHrPGod4Ck8AiwyVBbgygxlBDgE+A4F9E3gruP340u8szfGS2NL41xctNJ/nriLwKvuYIuBXcxov7e/ZFPIHS4Q4quo1ITv+7xdX+rWX8dBPWHCBHYYc/4y6/Mb+5TbmXjmxtZPHcxC+YvICZhDxx8AhKu9LRZp8+J78E3AtIebbKJvbxXbGzDc2uuX4PBbBCSj8383utzQcoFPLz8YZHZYjU7JCPtKJXCcV1cDPn54O/+cletov74YnfwlFpEEFGoJrSxj7keqwkkhSNTzyPseBSMxTCo9Q7g06JGRrQu4dqGjj/b4aOEHj9BWdkkwLMONqOIVUWlcg4w67BYqsFSCcpWvIP1XWHK+va36UyIvRDCRoE2AYBbBt7iOHWgJqu4rNQmqiPJ3PesGI2AJKFSQ0apGGjt5bo84TPyqONv8+bNjBtXu8Ezd+5cAK655ho+/fRT7r//fioqKrj55pspLi5m5MiRLFq0CB8fH8dnvvzyS+bMmcOECROQyWRceOGFvP76647z/v7+LF68mDvuuIMBAwYQEhLC448/zs033+xoM3z4cL766iseffRRHn74YZKTk/n5559JS2t7RF99bdcOiz4JUu6FgDR0Kh0lhhLS1v3FgfKjTL3hapdeKmVWCimzUvj7xN98/vvnpJcMQpl3A1lRo0nQu/RSnZv89bBkBPT+D6Q+2OD0nMFzGv1YcXoxrye9Tswlw4BJHlsAFBwsYM93e5g+czph/UKEA7MZKsVev9fx1wosFgvp6ekkJCQ0TC8PGQKD36ckvR8AaqtYCRZXFzua2GyiMLZc2fpJu6vodXkv/Ev84VWQWdX4qmQiaroRiV8v7YgmBkJHoivxh2Kg3Mwv1/7CiAdHeNTx99clbzFOE803l9Zk/O37H4SN9Tr+3Ejd8UWnE2OE3fE345MZzXzStai0KiIHRKIKUpFmTOPoiSpkVh9UskqozgVrdctf4qV9COwnNh5sZspNBk6WncTip4SSRMrLIT61fRe4FpOF/fP3E5QcxLsx1fw434zvLAWYSsVGhJcOi2N80Z1CfvI3SL7NSd3CXv/ELC8lNxeM5UbMBjOa4DPPAA9MDCQwMdDxd6eMv3RxLF99PvQ8wyh6L+3DiG9BUbtIWHR4EbO3XIWiW2/6m2ozhyIHRGIocb1kjT0TSWUOQeMVJGiIqRRy14g9hXpOsrBgFeduMeCjUuBfrzKKwkeBf5xrdvqbXR81QeolqUQPiebtLwOgQDghHFRmwYbrIfEaGPapS2xsL+wZf1VVwhHeSAWajsWwT4Ujv4aZPWaSGJhIvwixfi3NKmXTW5voMrFLk8749iY4OZjUS1IZ2nsogV3Ogb73OtQOOiOVkzeyr+AQ0qmt9I/s32gbe8ZfTEyjp1Er1GC1QPlRUAaAOqjxhnWI9oum4P4CVPKmJTtCQ2sdf0nNxCevfm41xnIjE/7jGiWAtlB3fFn9zGpkX5xASruCq5Lu5Z4pFxHsG+weQw69Ixxv45dCyODT/57DH8Kht2D0L6CNa7l9XU4tcs9822YDUwlYjeAj1haOjL/yWsdf7F2z+Ga+mpAKz2fV2R1/ay95DWmYEXrXOis7JIlXiT+NkTEfNt8BQz+FyMluNeu08QkRfxpBpwN93lFiFnzFthHn0v+GxsfB9qB4fzb6/EoUsgQyS8VAG+MnBlpP+Iw8uvM6duzYBvIPdZEkiaeeeoqnnnqqyTZBQUF89dVXTZ4H6N27N6tXr262zcUXX8zFF1/cvMH/JPRdod9LAOhUwtu/v19Xep3ffj+GbdnbeG/LeyRY8knjBoJanjecXQT0gcipYoOtDfgE+NB9Rnc0qaGwwXORfzm7cljx2AoCkwIJi8mFBb2hz38g9aFG23ulPttGWVkT0oy6LtC1CxZR2g+lxVnq02az8ZzuObpM7MLsXzyTLVNmELbLLTpCWA/fjBSSId3u8Ig9ZyVxF0HcRejzTYxbdIRxg3259Lcygru5adHSBMoALVUy4dDRKDVw/pEWgwa8uJj8degPvQchj2HSKVjaewxblqs4OeiIW82IGhjF1UtF4NEW4y4uvFAcl4cPhZnH3WqLl3r0vN/xv1/v+pSbf7+ZpKDzScn4hYoKMFWaKDlRQmBSYLsEmBhKDPxw6Q8MuHUAht7TkKEQm5wjv3H5tby4nrKyMiTzTtj3IkROatbx98GgD5DkErfvbnv2Vn2qiqpQ+ChQ+oqdfbvjL1QbSqFXdaLjEzLE6a8KmYIScz56RZ5jow1g6qsN6w61BqvZytfnf03KBSn0v7Hh+teeiaQ0B6PTIerZHf8Gkm8H/5TTuuY/ivJ0+Os86PUk9Hrc6VRgIMhQYDRCdTX4+jp/tDSrlMr8SiL6tC1brzGaXB81gdpPTXivcI7LNlOoNWKQ9QBqNiXUwTDmd1EHq4Oj0YBCAWazyL4P68AJJgDEOAeSDYkZwpCY2t94VWEVa55bg1wt95jjr9u0bnSbVk9ZqaPK9bWCfcWZDPx4FNH6aDLnZjY4b7FAllCgazTjD4BD78GmW8X/D34fut7Uqms35/QDCAmBQ4cgL6/57znwywFKM0s94viD2vGl5EQJstxsbJKMcF0YA6Jd8IOzmmDHwxB/OQQ1s/+nDoPgQSKI90ywVIOxRDjW2sqUTWCparndmWIsgh+DRb22IR8CEO8fz5W9ryTOr9ZZ6R+lxSan40h9Wq34hOkpVovfWYd2/DWHKlDIrEo1GWmHPwCLAbo3noDSIbBZRS1CSQ7qIEqqSzhechwfhQ/hum6YfHSUhiShDXNvxlHOj3/Tff1uVBc9SkZJTcafX1MDbfvjTbnw4nD8nUjWoemd7PLv/3Punyh8FeRPEgsoqSqES3r+h8SsndDjm049oXIpCl8Y+0eTp202GwVVBWSVZtEnoo/juE+AD5f+dKmQatjgOa3/pElJXLvpWg4pDqEorCAl9kLwa3ph7M34cy36mneZwuQs9SlJEj1m9iA01U1yFPX4ZsY3KKOUDA98goJ8BZJKD3EXg76hZK2X9idAr0Rr7AJG6DYt0tPm0OP5a3nt1y0kMJpIXST4et6msw2pZDdhhV9gKb8SP21PqqszyTOIBfuOz3eQuyuXSS9OcqtNdTd1Fd6ZaodCrxYvG4tCbIaUl8PKJ1ey9sW1zDk4h+Bk1wcTKLVKLvruIgISAjiyVhzz1n7sXNjiLoGoKaLGSx38fUTmj0leQm4uDL4s7bTrtdXn/f7v4xvky81bhMpL3Yy/E+UQpT/IOebpcOSBluvOeHE/NhuYK0TWnyQ5ycIaXZDgV3ayjOOrjhOW1vgGboOMv8ItcPBNiL3I6/gD0CWKjdmgQQ1O+fiIP9XVIiC1vuPv+4u/p+hoEfdm3+smY2upLKhEkiSWyu4hK2UNO0q/51wuEiflPhB9ntttOh0kSUgkFhR0EsdfCwR3C2bOgTlowz27MbArZxdLji5hiK+cEaHJQppU3tHTKRtHX5VBLxXkWE2Nns/JEU4Llarh8/P4isdZenQpr3dNZaBCJ7J//FPbbMOyo8swWoyck3yO0/HQmm2J/BaUXi/77TJUWs9P+GZ8NIM/1UCGC+efJxfBvv+CzKd5x1/CbNeUOug+5/QdOJIECjekvqsCoestEDrScShSH8nnsz53auarNKMpzqU8UwMEtL9dzWA0AjIZ47+4nuQeBq6ruA6Z1EHLf9lssOc/EDIUIiY2PB8+1lna88DrQha0Izv+KtLh1yRIuQ/6vci8HfO4a9FdXNzzYr6a9R3V+jAOD76MyDHuNUs3bgA7SuMZrJFTbixHLskdGX+ewLudcrZiKoW/L4eICQ7Hn0VW7rTZ5ir2/bQPXbiO/BHizS43hJAcsxF14Uqv068pKrMabI5kl2cT9UoUMklG9SPVKOXO9UjsWv8VFeIF5O5NMbWfGlMXE5PemIROpaPsoeYjML01/lxE5UlYNpahuiv5hcfxM3Xn/O7nMyS6Norygi8v8Jh5BYcKCJaCGaj6P46eArMO6P+dx+w5a6nIgENvEaOcCozFHiBts9k8WnzaaIS0jDcYnQjDIitFYWbfqNbpzntxCbbYS9hTHEeP0FEE6qqhGsw2IyaLiYO/HmTvD3uZ+PzEdtfFz1yfycHfD9Lv+n7YAsQLTZJAXrYbirZA1DkO2RcvbiZ7GWT+DCn3OzK0TLJSQLzLe07sgkwua7fC6UpfJakXp7L11FbeK70VY3RPVMrn4dC7YiMqbHS7XNeLC1EFgm9DKZ66GX95eTDmRdetzHte0hO1X+2G7RW9rmBw9GC6BnXlrwrwVZQhyeS1kc1eOhbrr4Nj8+CSclBo6ziJi53WqxlrM9j9zW6G3DWEoKTWy8n4x/nzYMmDmCob3xS3Z/w5avx1vVU4/VohdXdWoNSLrIwmyIn+gL3yr3l/62yeiLrZ6dyAWwZQVVDlkTnoon8tYtdXuzA+WApKCNLULKBtNqg47pSR3NGxO/5KSz1tSSvYNAeOfgwXFYLch0pTJesz11NhrGB69+ko1AqPq5Bs/XArSxct5eGUh1ncMxr2noCLSzut4y9uz4P8EAlDcxof4+z1/WJioH7JqY1ZG1mXuY5tfa9j4JgPTuv6vx74lRnfzCBUE8qe2/fw5a4vKTWU8vCohwkOlmOWlZOX13wWjja042wU2d87nx5+nqgyJdf0vcZRq/K0iJ4GE1aCf09XmNd+WIyQ+xfok9t/fJQkGPxuy+3y8+m55gOqTGOAse1rUwvYaz8qlUIa117HrUNiyIedj0HSTY07/uxUnIDsJTBuIRgK3Wff6aAOE87ikKEADgnegqoCFAoReFRVJQJV/fzcZ5ayawL58QkolfDS5Jd4fuLzmJoIwnAHXsefi/HkBmqbkBSQswy0cY7o7d7rT3Bw5Ttw4W0uvdRdR+/CVGVi6cKlgIicfH37L3x1byeph+huDr0Dm+fApLVOMjcRugjUcjUGi4GM0gy6BHZxnFv2yDJUOjUKxUjMZpH15+7IP0OZgYLcAiSrhF7Vciq1V+qz9UiSRGxsbOPji8IX5L4o1KL2aUTVeN6cPd7NFjbNHXuFnOeqGtUub5aGhzAWwN4XCAnXczByFUelbEKjEonsFcmVf3qmcL2x3EjWNxvQ58egVCZCwSZYNhYGvA7d7/SITWcjkjqAiKQhSAofgvUKqIm+rTBVMPX1qUx+ZbJb6qZnbcxi9X9WYxto45b02ylNTmL0iQVIpxbC9vth8gav489TFO0QmS7xlzve7yZZbcZf0vQkkiY1U6TFRWSWZrLP+jsB+nw06krYdDskXu11/HVgHPMXbCKoTZKDb628n8PxJxNSn65k0gvOmcr3jbjP8f/l5ZBeNIADyXvp16X+J710COwR/zaxXqzN+Cuj2mAFxE513r48Nr6xkaQpSW1y/AHI5DLU+sY39QsqRcaf0hIsMv7kKtB4riZyZ8OkPU6BagW7chpuave9pq9LrtHs+qgJEickovJTUakUzowgTYA4UboP/kiFtP+D3k+4xL72xr6JWXIayn1uR9cFwscJuUG5DyfLTjLhswlolVrKHxabAmUny5BkEroIzwT/HV1ylKofq7A+bmW1KpXxff8PFJ03ELE04Vr+e+ShJjeb6zr+6mKxWtievR2AlNDTz26ekjSFtLA0dufu5oLvLmBD5gaHLT+eWMKpJD298xc0+x2VBZUUHysmJCXE7Zl/dceXg78fRHFcAT5deHfvM1TurGBmj5ln5viTJAivCXYyFIBMJQIq6mKuhE23QcwsiJ15+tcCIdl44gcRvBJ1Tsvt7VSfghWTofvdMOB/Z2bDaWK0GMmtyCXAJwCdSkdItyCyuo9DGeYZWeC6mEygqixm/wc78bsimch+HVi9SBUI52wHuW/z7XY/A0c+gImrIWxk8209jVLn5CwO1tQ4/mrmcDqtjeAtS9j2QRBR/zfQbWbZAwXse59ymRy5TJTD8ITPyBvi6GJk9cNlOioKDVxaBYPedmT8mRRmpHbYlZdkEiqtqk7kZKio7ydzfR2YfwShoyBicoPoMkmSHOnBWaVZTud2f72bfT/sdWT9eaLO3+r/rGZRyiICiwKZrQc239lshIhX6rP1yGQygoODGx9fVIFw7g7yQ0UNJrtDtS6b3tnEwn8tbGcrmya/Mp9s224MihyCSr8UUZ/uKBDtpRa/FJh+CEP8nZwI+ZC9mnfwH+JP5EDPTU6rCqvI/Xo5/rkHxaTINwp6PgTBZ1C43EubkdnMBKvLkJmKCfRTIVlFTFi5sRx9pB7/WH+3TFD7XNOHO/bdgdRd4kjJQSpVR8VzEXcRjPkN/LzywB4j6QaYdRKCBzqCxYwIx5876mtkrs/kpdCXyPpazH0UVi1KtQrGL4Mec9vfAC+njWP+YsyHn2Ng99NO5yd2mci+G08y7OAKCgthw1ub+e7C77AY2zc40Bt81gnoeiMM+xSUwrthz/hDslFprlUVSb04lXsy76HrlK5t+vodn+/g5OaTTZ5ffNViZmbtJbhslHhOyg5D6aG29uKfzYqp8OewRk+F+Qon6anypv+Nz5Rm10dN0O/6fkx7ZxoGqRiAIN8AcUJSiqyBcDfrgZ0B/jU/iU7h+EuZK0qaqAIAHA6TClMF1WZR6/vVhFf57ebfPGUhM+fNxDLfgk1mI8+3i5CA7ixB/Y1QHX85H5SCydK44y+zpuxf/fp+Cw4tIKcihyDfQIYU/VkjSfkKLOgrnEetRK1Q89nMz1DIFKw5sQaT1cSsHrO4vNfl7CldR57/QnaX/t3sd2z7aBsfDPqA3N0ujgxqBXXHl4X/WkjglqVYZJVUmsXEN1R7BmVUTi0WAa82GxTvhl+7wp7nGrYrPwrHPoOibad/LTuSHDZcDwfeaNvnVIGivmO9Op3txtHPYNlEqK7VgR03bxyx/4vlz8N/AhAYruJU8mgKtZ7PrjMawac8n+0vr+DNz9/kvsX3caLkhKfNahyZAgL7tLym7v9fGPJxx3f6NULdjD8AvZ9E6PEtHF+036125D7/MUmbvm006cETPqNO4qXqPFgsnS+LbVTcKIbrruRg/z4E3d+0ZMfpYCw3cmz5McpOltVx/IUwJHYhFO9y6bX+MQSkibTqwL4NTtkdf5mlzgWab1x/I9esvMbh+PNEnb+ogVEEXxRMpaaS0WqTyA5oRj7JK/XZeiwWC/v37292fLFHfdrlXuyLKIAjfx5h8zubsdlcUzentZiqTOz8cidfzf+K+RG92BV/O/qKpXDoLTH59OI+5GrQd0UX6IfCKjbuu/w3yWPF0gG04Voin7iZ78+9lQcLwjkpaaHvs06Zzl7aH0vBVvg1Eeuh99DpQG4Vg3KFsQJDqYHCI4WYDeZ2t8PH34eQHiEYlSJETm7zRalE1BGKnubYLPLiAVT+ov6mTOnI+Ku21Wb8WS1Wvpr2FUseWNIul5er5YSmhmLSic0ruUWHUq2EiPFiAeulw+KYvygCodudol5SHTRKDd2jIvFVaLDZIH3dKfbN34ex/MxqD5iqTPxw6Q9s/WgrAJWmSg4XHqbUUCtROyL2e8JK3xcbb146PD4KH1QysYNSbq71dKj91PhF+yFTtH5bw1xt5tfrf+Wvp/5qsk2IJgRlcQoKq16sVTbdAYuaqcN0NuIb2aA0hZ0InXD85VWdanAuc30mHw75kP2/nNlGXGvWR41hspgwy2o27/U1i2e/ZJE1ED7ujGxyJ53K8VcPf7U/8pq1oD0zY9jcYXSf0d1jNil8FJT6iHeEPRu9M6OUCfn1ljL+6jv+3t78NgB39rkc5Z5nIOMHMBaBqQSqs9tkQ7/Ifjw2+jEAovXRfDD9A7oGdeXyFFFXd4P2ESyWpt/BCWMTmPDcBPRRLatJuZq648t5b5/HyZ4TMCjyAFDL1a1SuGoUmw22/AtWngtWkwjODRsFQf0btg1Ig4uKoftdp98ROzIFjP4FBrzWts8p/aDrTaL+mzuoyoSCjU7PWphWKL7kVOQAtfuHFRWen8IZjVAeFMtFS29mnm4e/133X8dcs8NRtB3MVS23U/pB0nXtbo7L2P4grBdjSoOMPx3sHns73R+/2L02+fhgUaoxy8oY+fFIZv8wG7NV7Kd4wmfklfo8m8ldBRYDtw+6nZBjt/P5SjC4oFh6XfL35/PZhM+Y8NwE8lXC8ac2B3FlzBDYNQtG/ejaC/7Dacrxpw0Tb7+AAPF3T2T89byoJ9u7b6f6p2reloYyc+aXoPRvsr032rptVFdXN33yyEdEVimAa8ivKMDnmWgMFgOmx0woZApmfDwDSe7+iMWKnArmXzkfxWUK6A4Ki46S7u+jH/lcp5ZO6bRUZqFXSCgsYqGSX1bWwgfaF4VagS0yjALSwSoWUV48gCaG3KCrCA4aiL4CFFYdZkqoMFWw7q11/PXEX9y681bCe4W3qxlVRVWYq81UVot0cLnVV0TJ2WydOuL6H4G5CipPgDrUsRlWbS3HhpWyMhkyuYzc3bntJsMU2S+Sa1deyyvrXoHF4hlVKa1gk7zPRiegurpabDgNfL3R85IEoaGQlQXd7zmXSz6fdsZZxsYyI3u+24NviC/cAFtObmH0p6NJDkpm3+0HqaqC6d1ex+/YPhh4c8tf6MX95G+Aw+8Kh3HNhuigiBHs3w8ms9WpadnJMipyK4joG9HYNzVAkktcsfAKFD5Nb4UYDKIuDNQ4WBKvct/GZ2dh6CdNnor1j4QSKDA1zPiT5BKlmaVUFzeztmklza6PGuGnK3/CJ94Hal5Xwdqm16odnU7l+CvZC0c/hfjZENQfSZII0YSQU5FDfmU+0X7RTHy+mZpTbiB3Ty5VR8WP/v78d2BtBgz/3KM2nQkBux9hcTRMzrJitVmR1QnIttlqHX9xcbWfOVJ4xJFVddWAO0Fxm5Cg1HeFPs4Z+63lkVGPkBiQyNCYoY5N+acnPsaXu+dRoP+LhXvXMK3XqEY/Gz04mujBjQcXuAP7+JI0pStFb4NRsRkQ2X5nNE8Z/iWUpwsJaYAxvzbdVuXCMSpqquu+q71IeQBSH3Y6FK4Va9CccuH40+kg8tAqQk5spST/dgJCPVNLxmYTUp82hZrQPiFkrclysrdDYTXB0nEQkAqT1njaGteSvwEq0oHajL8qcxVVpip0Ol9Mvv5UW5v5fDugvOZy0ldDCvv4O+Nv/NX+KGSec795M/7OZjbPgc2i/pZKBdqiTMr//JvKgkqXXcIvxo9z3jyHpMlJ7Ll9D0+HHsKvOoW/rZ+IIuleGqdwq5BTOPaF0+GmHH+lWaWc2nrKo1KfAGUG4UjQqPxFHYxmJkReqU8Xsud5AnOF5np1iR+GGhmOkmqxEvQN8sXH38ftetKaEA2Xzr+U6nFi0iy36lH5KEV9H+9mrfv5tQuqbbegsgnHX8aXWSy+b7HHzLGYLJQVFyFZxbOgP/65kI2qyvGYTWclvpGcjJgL4ePR6cCvsi9hxsEoZAriRsYx/P7h+Aa2UAvABaz8v5W8EvUKFTkiCl9mrcn4W3sFfKcDa/tnHXppgvx18HsPOPEtfmo/5gyaw9WJD2OTzI4s87uO3cVF317UrmZUGMWzIbdqCTEugG+UkP5Vu17TS/tSZariroV3sS78OmxYKCiWu2Suog3T8pj5Mab+T2xy5VYImbAwbZhj/vnaho+xjvz5jK/lpZ2ozBSOgtIDjkO/XricYQeXoyhPcIry/3DIh8y/en6rv1qulNNlYhfiRsY1er6gsoDbfruTQ5HPoFYjavwlXgmpD51mZ84+4oNExl+p9VQDxZHoQdHMzZrrslp/beHQgkNkbayRjTb74etTo0Cy4SaR1dmJ6FSOv4rjsO8lscdRg13u064K5Wm+v/h7gp4NQgLKVGGiBEEnRmkup5dfGE+OfbLBb7CwUAQ2yGQQWafqw3tb3sOGjaldp5IU0l04CfySz2jdLpfJuarPVSQHJzuOxQfGElt1PgBrjm467e92FxaLcPIYlGIuEao5A5lPSYKgARB3YcNz5nr7sIVbhNPclZhKxZ/WsmY2LOwHNjd5ThopB+Vw/NVk/CmVIMllmFUaCrNakcHWTtifC7mxivwS8V6RS3KHg5vD78OS0cLp5mmsJkh7VEha/9MYvxRmHANEtrbdwVZYVYheDzKTgdxtmZiq3Hcf7DX+im1i3z7W37OytN6Mv7OZ3s8A4iUuKYxoCw5g/HsNJSe6oAnWuOQSuggdg++ordckL/FHskJx4DXQgeueehyZEszlYHWWOorWi4inzDJnx9/ify9mz7d7SJr3GCCjsOnSeu3GyidWkr8tH/pDitIsNMl1XRpta7N5pT5dyshvsVp84H2wWZTolDrKTeUUVRcRrAmmsqCSkhMlBHcLdmthbJVORY+ZPShbVA4nQWHR41O5FcqDQJfgNju81NDj36Dvio/sdwAqVpWyYdsGJr802SPmHP/rOBUPfM6A8wawZdAWlNUnIW+NyAzx4hF0Ohh8+Hf8/KB3OBAOXSY0Po67mvgx8dhsNk76iOwAud3xF9BLLFC9z4Xn0CdB6iMQ2A+1Qs0b577Bnj3w4Pe1G47tGViStSmLQwsOUZEoJg4Kiw7JJxiip4Om8Y17Lx2Qnf8nNrHG/u44JJNkvL7xdZBBpPx/nDwg55iUT0S/iDMOOJDJZVCzf1TX8WdXnCgyJ6OITG7i0148TvQ0uLgMFLULBXutFJtNbLgpal4LIx8eiULd+ndERV4FmhBNk+NWVlkWn+x5E1VYGKN51Bur1hT5G+Hk79D1ZtDEOJ3qEiayLy2YKKgqcDh5PM0DhQ9wsjCXnvdVgmSprb9TuBlknUt5wu74K+2gqnJOhI6C6YeFPGwN9R1/Sx9aSvGxYi76pn2DiJpi6D1DeX3d69iA5YkPc03fazxih6tQjPqOCODxRs7Zs/0iI2vHUYCZPWZyouQEV/W+CqrzxJ6UKkAMuse/EaUjYi9wiX0Ryq4cBw7kHm2yTXl2Od9e8C0pF6Qw/N7hLrluWzGUGXg94XVigvqRMVpIfdqlJ9uMuUqMNaEjGpbE2fcy7HgEph8AbbwIeFw1U5RHmZF+Rn1wkLsKlo4Rcp/d/9W6z6gCQR3abAkfl2I1wakl4rkLFfc8XOfs+JMkqOg3kpNdRiJ5MGnb7tyJ3r+M77tuQXuvFn2Evja7tuoUlO4V+6J+npMxBkChgZR/e9aG9qKOs1iSJB4a+RBquRpfpS86HYSlb+TEn8vJnXyjWzKIjRVGTCs2oK+MocgiBlt7Ao+n8O6iuBhPFGo8bWJElM07m97h9i23kzRsNleO/C/B3YLb7ZJ2h1RQULtd4p9BQC84/3CDw4OjB3PXkLsYFDXI6XjqJamE9wmnLMAKyCgocJOddTi69Cjyw3KevfdZ7s5+Gf6+HKasb7RtdXWtHrfX8dcyMpmMLl26ND2+BPVHhdgQMRrBXx0oHH9VIvVz20fbWPrAUm5YfwMxQ9z/0ikziF02hVWHftMEIdk0YZnb7Tjr6fssABq5qGlTfIOMZ86/B5vN5vZsUBCBIeYBXckLzUMt0yD1exH6veh2O852ZBKk5j+IbM9Q9DFPAEKK2d0Kmz0v7EnPC3vy1sa3gDpSn94MC8+jjYc+zzgdqr/hWHS0iPS/0kk+JxldhGulnLM2ZPHXE39hfL6m/qNVhzVoGAxsfYaPF8/gNH8pPQCFm8BqcSzS1Qo1arkag8WAWV7KqZW5fHbHH1y15Cq6TDz9oIPq4mpyduUQ3C0YXbiOvEqxWReqCa1x/NkI8S8Dm859m1le2oZcLf7UQVUnds1orN2wHnSb87qoJT4a+hHaMC03rGu8tn1tXfpggoMRz+yS4SLYIO3RNl3rH03hJtj9NISNbeD4CwlSojaFI8lE1H19x9+hBYeoyKs4o6y/FtdHTRDgE0aXXFEzy/FMnbPN88Wi2kinyvhT6sSfOtifiYIqsXGRsz2HnF2eU/0YcNMA7ptxH1eUXUFqaKrH7HAHTdX3Gx47nOGxNQ62jbfC4fdgVjb4hsPWuaCNc5njr3t4FzYUwZ6TTTv+FL4KitOLMZS6uB5RK7CPL5ghrF80h3P9MCp2AkLq87TI/AXWXgaD3xMBE3XRJ4taf8YS0CLmJoPeAZMLS3P4pUD8ZeJarWXwO667fmtZNR1iZtU6/upJfYIIVi0pqU0m8ASmmgSy8qBYeg44TqWmkiRtkjhotUCPeyD10UazGN2KzSYSSv6pZVUqTkDxTggdCaoAnhr3lOOUTgeloUmok0Ab7p6N58r8SmTLl+PfZRiFZpGwE6OvnSN5wmfkdfy5GE9snp4pWpX4AZQGFGFQR6Ny4e9hzfNr2PnFTkZ9PYr/Z+8so9s6tjb8iCXLMjM7TuIwc9IwU5tym3JvmRlv4est8y3TLaWQQtokbdqkDTMzx4kdx8wg2eLvx1iyHbMtyXaqZy2vVuccnRk58pyZefd+93tp77HTnMisyL4MzbwNMt4T0Zxems3wmOEMjxle53jPC3sCsL3KKaE9hL/r11+PzWxDppTBUS3IGy547HhAy+UiVd9L40gkEvz8GikybrOCpQydzp+CAgm+8kAgnaJKIfzFj41nwnMT0EV6tjD2kcVHWHzdYuw3AH4i48/a8ynkvh3Q9/wfhFYhvgdFSr3LN+hbQlifMIrP70+aLQ1/aRssU7y0CYlUhqJ4G2gC8a36OthswgKocP8Z1j6zlpEPjGzTJnxL8FH4EKFOQGWOQOktBdohydPnkWctxiKNQq/XYrHAqdWnWPqvpVyx9Aq6z+7u0vb6zu9LwvgEAhICSP/XS+gNdlTn6Nr1XKPW/GX0d/VGE/ip/Mgz5GGWlVAaEMf5781scxBi5s5Mvp78NbM+mMWQW4fUyvjT68FHUcoHEwNg6/Uw4n9tasuLm7CZoXi/qBeuExtp/7fh36zo/yGJOfdhMj0uLDhr0JxAJpvVRvc53fEJadjdpsAgFlJKSzBB4YDVABWZUJnbpo90zhF3KYRPAG1inVOBgTB5bwYKuYxu9QT8bnhpA3mH8tok/DW5PjoLY6mR9E3pKKNDAaGa1VqHdrJ9HMdH7xTCn90OFcIKzyES3zjwRqYlTWNM3BgA5v8xv71656RPWB/6+AYIi76omRBSd9+l05CzlszszeRHzKRnSE8Usuove331/eoQNhbsFlBXZbeN+qb6/13A/HHD2fjBI/hXDKawsP7EALW/mgcy2ydLqeb4Mnvh1fxyI3Qp7c5XN01Aq2jlhmnQYOj5EESfX/dczFxnYobogNT1e6XqUBjdwS3ypQoY8ZXzuQ91M/4AfDVWwk7tIGWpjj59enm8m1At/JUl9sf3qT3YF9ur6/sV74Xlw2Dga9Dj3nbpn5PSo7B8CAx8Hbqdg1af6T+LwIQpG0Q2bQ18fcEQEIV5WBQB8Z7pjm+EL/qrbiI30wer+SegttVne2hG3hBHF2O1Wtu7C83n0MvwcxhhCF9ki7QMk96MxejaOjo2s40z5jN8tPMjjqq/xo5UpIs3Igx5AU5+Cem/tugtwVX7JO0h/EkkEiH6gbAOSLq+wWsdNktabadbY7ULVquV/fv3Nzy+bLkOfgokyF/U0vOViWKPjoy/mBExjH1yLP5xnvVCUPmpiBwcSZlaRKrJrL7Iet8HCVd6tB9eqtj7JKy7kKmaJ5iw/wTTFHeSfyTf5WN+SzCYRS0DjUwLGb9Bzpp268s/FavVyv7ktVhHLECphCPx97Gybzwfbf8MY6mRtPVplGW5MNq0AdY8s4afr/yZ6wdez4Ihp+iT/o7YkNv/HBx52+3te2kEczmsmgIHXwJg4lcTGfRFd4p9RVZ/aSkkTUni8iWXEz3c9RYqmkANYX3CUPoqsRiVyOwqdAXfwc57W1anxIvHqTV/aWDC568WcxOLrJRcwhh6+9A2z1eCuwcz/e3pxI4SC+2zrT4l2NleeL2wn/PSMTEVw5+D4fBrzkMWmxmTIh+TItdpsQVgNVn5ZOgnLLlxSZO3lcqkTH9rOmOfHNvgNY4MJIUlWGxGK3RwQToM+W9rP825iToU/HuBvK4tb0AASJBhsdSfkTHllSlcseSKOrXHWkKT66OzyDucxzczvmHLNxso1G7Cok0Xw5KpGE7/LOzYOhGOjL+KCmr9PXRM7LA4HnZU2wvO6DaDmwbfRM/Qnu3YL4Gp3MTHQz5m4ysboSwFDvwfFHT82nONcuwdog4+Rv8P+zuz3h04hL+YGom6n+36jBUpK6gwV9VMS7gShn9a/eyOmAgBfVzWvan9+3Gh30tEFF7CqlUuu63LqDm+OAQeX2kwQ6KGtP4769dNuNtomgiC1p+G0mOta8OVFOyAvf+GsrpOZG4lcT6EjHC+TAhI4Kp+V3Fln+o9JK1OSszhv0j7Zbdn+1YDx7irVFaLkg6REqkCoquE3P3PQkX7ZTNjLoXAgaBNaL8+uJOIqTDic/AVYnG+IZ+92XvJLMtEVyU5OPafPYFcJacyMAqTTwB5prpWn+2hGXmFv38yymDw64Fv1WQ9+rQPfu+8wN6v9rqsiTGPjuHOo3dSphKbhgpLELuypiOZsQPCx7msnXOS3Q/CkTfqHM7T57E7a7dzwxzg8KLDfDbqM6wZ2YCI/HNMUDxF6ppUtmzYwp7sPRRXFjd6raGq616bz+bT6AMifDwk3Yi/n7hGI6kS/qoy/tqLxAmJXPP3NYydPp0u2Q8QZO7vFXrbk5KDkL+ZaL9ItMYkCn85xHs936PgWDtECgApK1IIWrSfpNSp9PAfDNvvgN0Pt0tf/unUHF/s6mIqVKfJLsknaUoSj5c/Tv+r+7u9DxnbMkhZkQLUXkhx4kNRV8RL+yFVig0wg1i86JRiFaX0FXO70lLwj/MneU4y2lDXP9grSyox5BuwWuxYquIU1MV/wVGvINwZcI4vhkyxua5Pr3XeTyUi6i3SUvLyXOO25x/rz/C7hxPeT2zAnJ3xpzcH8GfR/xoNUvPSzigDoP/ztWzlHNaAJnlBLaFDppQhU8mQq11jZlSd8RfiDKr0Ug92OxgLoDK/zimlsnqdV1xc960xI2KIHRXb5sj3lmygBSQEMOfTOeyL3c+mnqM5EvlvcaLkEGy4GNI7l320VguyqpjbDl/nTyIVmU4x9WQ6VZF/NJ8ji49gNnh4EwNRk8mQb2DT4U18nn6A8ilbIf4yj/fDpfR6lHnZIsvPbK39Oz3b6tNgNnDT0puYtmAa5aZGdsgtBrC4zltxyhTx3xUrGn72n1h+wqX7ky3BarVSmFLI1hdX4VOcWctuusUYMoRLU2OUpcCa2fB7H/gtGYoPtKHBBjizBFZObJ6Yl7sGDv5H1KrzNHa780sR4xfD1/O+5rmJzzlP++okHB15LZE3t5+DnGMeErtpIaO2jCL9vnRemiSCJAnoC2MXgSoY9j8DWcvbrZ+EDIMp6yFqWvv1wZ0E9IYu14FG1BZ++K+HGfDRAL7Y84XTzUjy6y98Nfkrj3THYrRgLjeC3Y4NCzKJrN1r/HmFv38yXf8FU9Yh9xeWTMWBeRi69iOwS6DLmyqsEMX9FJZg/PxqFxH20gBjfoIh79Y5POSTIQz6eBD7cvY5j5nKTZSklaCwGZ2/2yIPaz4Lpi/g+7u+Z9qnA7EsGwTH3mvwWkfkp1f4cxFJN8LwT1FpxZMtSTmCuclznQ+Y/CP5/G/M/9pt0nxR0nX0OvMao3UW+HOI8Lf34nnG/gIXZlVPgBITOO+J89AE1Y3U9gS5B3LxPXiGIcc/5ZUhP4k6Bv2ea/qNXlyO1rAHMpcBVdmXQJHeg6FxwPxl83ko7yGgeiGlUADTd8F5P3m0L17OQqaES4phqHiu61RC+JNrq4U/B5ZK12cQr3pyFa+GvsqjPzzGrsT5lPjswjboXZGB43WP6Dzkbxab63kbah12CH82RSnyojze6fk+Oz7a4dKmL+p5ETcPupleob1quU546cBIFdD7cYic4jwUrBEqnEmeXyfD6YYNNzDr/VlN3va3237jt9t+a/SamjX+goIQG7anvu50GWFux1gAP4fAvifqPV0Q8SObu0/g5c3/qfe81Wx1yzOjIXzDfRl04yAKwoU3pkYSIE7ougobw6iZHuuLK5BIOpnd54CXoMu1zpeFFYWsPLmSNalrANjzxR4WXrCQ0gzPq5i+4b7cm3ovT3R5ght+v5MiZaTIaO3MBA9hpVENgNlWLfyVlVV/XxwZf8cKjmHHTpAmSARYVGTBysmQ+l31/bJXwg9aMRa6iK4DsygLWs+pvGwOHqz/mk2vbOKPu/9wWZstpfBYIfvfW49PaTbHA9/htU2vkV6S3vQbHTjEvnUXwLI+YLc1fK0qCHJWQfAQ6Pkg+Luh1qSpWATz6dOavrb7nTBzn7Ao9SSHXhXftbLjDV6i1YI+MBazj2cdrWpiNoPEakabeZy8vXnE+MUQ7XeW80n0bJi83n2OV/lbYOvNYp7iBaieKxYYCpwZf1aTFavJ2iaXgeZydPFRIj5/icCsQ3w56S+MTxqZmDjR7e02hld+8eKM3M4NzSBn9Dy6THLdvbe9tw1dlI7CECH8KS1BTOn2HRzJhuR7vT6PjdFARmSMXwynS06TUVo9uPe/pj/9rxEZGUFfQW6usPsMc50Ne6PY7Xamvz2d5w49h48EfKxlYlLRAF7hzz04HmxTtY9w5eW1zxWeKKSiqMKj/Tn+x3HSN6UTdeEIwAetqhyM+aJui5d2I1+2l8PR36MOieeOR25tt36MuG8E3x4fgv60TAg80Z1rw+VcIjLnLaTZORCbhVYhlOGSCj3mCjOnVp7CP96f8L7ur80pkUh4bu1zfHFkMb7Bt6NU3tC0HY4Xj+OYN8p8xOacQ/j7aNBH2G12bt3j2nElbkwcNouNf+c8TWbwLmIKrkHpMwikDdfo8tIBCRkOo76tU3/DIfypA0qwS2VYzHZo47p8w0sbOPDdAeb/MR9dlI67ht/lPLdbD0mBO5kd/F/Iv62WnZSXjk2wj0P4K2i1tWHOnhxoYvnptPq0Vgl/hTth8zUw4gvw9Uy9206BKkgEH4bXv6El9c2nQLGGvXl1g4pPrjzJ11O+ZvZHsxl8k2c3lYuqnGnUDuFPHdZpyxD4+4tg304h/J3F1jNbmfntTAZGDGTXLbvodXEvQnuFusU5oDkYLUbMNjO+EvDHJAQbqaxd+uIqHHX9TNbqAdOR7RcaCmqhC3Ik/wgAPUN6iizcikwo2FpbDNd1h7jLXGoXeNOyq1nbZSX9T33BihXX0qceJ9GJz0/0aIDA2cSPi2fqkjt4+V0tB/wvZMtfaYyJG1OrZleDOKzyk26EqBlC9JM0knujDISL8kDuxr+BhCsg8arG++FAphaZa57GJ7aqxmR1hqTJaiKnPIcgTRBapVbsI9rtlJwpx1KpcVnGf0swmcAuU5B94xM8+d+zsjkPvybqAg94GcLGuK8Tpccg5ROInApxF9c9n/otpC+Cga+Cb916vOcEhgz4awwkXgv9nnHOFQsqCpwB7ykDL+bVRZ7pjn+cP6VdBmDUBqFUgqwDPEe8GX8uRirtRL9SfTrsf44g/VEALLIyl/vDr3x0JTs+2FGd8WcNYlzMJ7D//7yiX3OwGusccmRxnSk9U+9b2qPOn0QiYcgtQzjZ4ySpFtg0+DvoU3/0J3iFv5YilUpJTk5ueHzJXQebribOT2SBlp1VjiukRwgPZj/IiHs8u7l18q+TrP/Peo5nHqVCkc7RsnFwfmr9kxIv7qf0OKT/gl66m5TIl9hR2b72iRKJBKNVAVIpKlW7duUfjVQqRTHwWWeGuVYpBubSynIqiyr5bs537Pp0l9v7kbo2Vdh9FqVwsnInJkUeSoUVSo6AsdDt7Xtpgqy/IFNEXDuEGmlV/VbHhmPC+ATixsS5vOk+l/Vh9gezKUY0pJL4Ii071LxoZS/tSq35i0+M2HDS1v6OfDjrQzLvz2S077UYtUEM+OQOhtw6pE3t2qw2THoTck3djSC9HqJ0x+mu+MobId3R2Tgf1lZbAzqiuM2yusJfZXEla55dw6GfDzV6yxs338j8P+Y3es1b095m6rHDxOZfK9ZVwcPgvF8gfEKrPsY5i0QqaoA1YIkY7hMJQI4+s865gIQAel7YE78Yv1Y33+T66Cy2vrOVd7q9g/GUWIj64HqXI0/jqPPXKYS/gy+ILLKqjAuHda9DaI8aHEX/q/ujDlB7vGvFqcVs+2wbAUUB3OoPfr93FYJ/Z2bH3aRHFeEvpVaJmJNVicvx8dWXOoS/HiE9xIGgwXBJKSRX12REGwtjvoeo6S7rYpdAEUhhUJ1i48bqcjA1iRkRQ8L4BJe12Vwc44vKV4UqOgSrUkOlVNRKdHx3G2XjFbD3cajMFlb5/f4P+tef/VwLd4p+ILLpmyP6ARTtAbP767zXIeFymPAn+FfXUhz3xTji3opjRcoKAHx9ISx1G7mPvMGZrfXvi7obR2klhVLC0xue5sEVD5JanCoOpi0UpSokEjHmFe6uY3WPIRN23t/6euU2qxDnG9tfK9oLGb+Bov0yI92OQidEc4UITHVm/NUQ/sxmqKz0THdiRsSQMfR8DP6RIrj9LNpDM+pEKpUXl1OZDfufIrBsP1Pj5xCdezEhmxaz82PXTHLsdjvXrb2OKa9OcQp/SksQayr/B5P+dkkb5zRbroeFarDUztKK0dUV/ioKK9j/7X5y9uW0i/DnoMwoJgaObICG8Ap/LUfZmKm8IRNSFxCuEXYIpaXi7+9sP39PM/bJsdxx+A6u3Dmelf3jMGpS27U//3hOL4T1FxKvFj5nvulSvpr8Fcd+b5/i4YUphZwpep+VvaP5fPcV8GMgHHmzXfryT0ceO9tZR0mnEjPkcpMeTbCG8z8/n35X9XN7H3695leW3bmMSouYlctsGnTKAvi9J+z7t9vb99IEu+6FXQ8A1c94u7K21ee0N6Yx8133Ze4azGLyoJH6wppZsO7CJt7hpSPQ6PwFiNRFEqmLJDpcRIBkZ7e9zbFPjOXuE3ejCdRgtBg5XnCckkqxM15eDutPX8ZvPnqIntP2xry4D3MpmIudLxvL+JMqpKz7v3Uc+K7pmkhq/8aFBTWBKEt7oLKEExiIqBsTe0Ed0dpL40T7RQGQb6wr/AUlBXHpT5fSbUa3NrXR1PhSE5lShsJHQUlVEIlWFiBO7H4IfokR2RmdDIfw1+Fr/IGoX1a8F6xinucQTxzWuu3JmS1nWHXrKqIyozhmVUHXW8Enuuk3dmR03dhj98cGnCo65Tx8vMo9sVuNP73D+YeBGsIfCMFC6t4sKofwZ9WdxGSCjAZicew2OzZrIxaZbkKpVFJZUkl5djlWux6LRCiToT5N2MCayyFjKZhLYMYu6PuM+zvbEgq2w4lPG7/GVAx/DIQdd3qkS00R6ycyLNNKRNCfVgv6gGgkw4bgE9w+DiBmMygqStBkpfDNpm94ffPrFFVU1VuaugmmbRX/n78Z/hwEKZ/VvsH+p+Hom6LuYmso3A6/RjVeP3Dgy3BhtsjQP1dR+Im/s55ineqYKxZWFKLRiJrDMlMFG17fyqlVpxq7k8swmaBIu5kLfxvDQyse8kibjeEV/lyMzeb5B1Kr8e8NM/agTr6LhRcsYcDpLwlO20PqmlSX3F4ikRA5KJKI/hE1avwFIfdP8LxPdGckeDjEXwn22uKNwzf6TFm18Fd6ppRF8xdx5Ncj7SL8FRwv4N3kd4lfF0+iHGIK1tSNaKmBV/hrGTabjf379zc8vsTOg0sNlAVeBMC2sp9RP69m+jfVEXnHlx3n0E+NR0G7Gk2QhqDkIMrtYjXaz/8wnPzSpUXBvbSA6LkwcgEKPxHiaTObyNyRiT6nff49Nr26ieE/5SGxFVNpM0DQIFB7bR09zdnji59aDMzlpnLkKjkDrhtA9FD3b35MfnkyYx4bQ0VVsIvUpkGqUEHvJyDyHC1G3pkY+BoMEsK8o8afXVE7489drHlmDYtvWIzeLIIWNDJfSL4Hut3i3oa9tJla44vdDosTYMOl9V4bHg7Y7aQv2s7RpUdd1oejBUfp/m53ur0jdjnFHFSCRucj6ld66biMXwqT1zpfhvqEEmrth79hMJXG2nNipVbJLbtv4cIFDQcErH56dbPWuYVVSeZabbUVnpcGOPEJrJwoNrrPIi5QCH8l1mxsjdW1aiVNro/OYsgtQ7h1763k6HIA8HEIf6owIfJ0wowIh/BXXNyu3WgeIz6rsjEUtcUdm7MGswGD2cCJ5Sd4M/ZNDi867PGuxY+LZ8RXIzgdd5qttgAY9kHnF/6S7+ID39mU2eBkUXV90vqEv5pWnwBkr4LMP53ZmU5OfgGrZzjF27biEP4qNaJ/9X2Pd322i+cUz5G21rMuD47xZfv721k963WUevGLk0vlTueLBlH4iozJIe+ITKSO5nR25C3YdlMTmWZ26PssxF7kqV5VY7PC3n/XEicTAhIASCsW3wNfX9AHxlAxcRZhfTxU3+gsTCYIyD2O9ucFKI+L+WS4b9VehlQBGpH1TvAw6PUYRJ9Vh3jo+zDrEMSfVaOnuUhkEDFN7J9k/Cbq7taHMqB19++k1KzxJ5FAQABIbFY2PvknB38UxUQztmVw8u+Tbqn5t+eLPURv/xWL/RTbsjeyPXN7rfPtoRl5hb9/MnIfCOwPqiBUKrBLZeya9iizPpvnkttbjBZM5SIc86t5X/GAPIWooouJCswCW/v5dHcaut0Ko78REQw1qM/qMyAxgEt/vpQ+l/dpF+HPZrYhU8qosFYwRgPRBx8V1gAN4NgkdKRee2kjMhXINc4C7+YKDSarqTriCPj7kb/5+1HPZtqWZZWRn1EdxXle+BLYch1Y6vHx8OJ+AvtB4nz8AsQYciruJI8WP8rAGwa2S3d6XdKLvSPBqDJSooqCSSs7bY2Vzk5E7odIf42CihzCteH4VvTA1+bZDY8+l/eh57yeVJiF8CezaZAo/YUtTsxcj/bFSz1EzYAoIcCOjBnJHUPvYGDQWKD6mZ53KI8lNy0hdW2qS5tOW5fG8d+PY6gKGvGR+0KPe6HrzS5tx4ubkUhE0OFZ9YHWp63n3j/vZaf9fyCRUPHrcnZ+1Db3kd2f73ZaPubqRRZPmFZsDJWXQ4RvCmGy7WB1cY0DL24l2CeYW+x7GXlsJRZz3W2M8H7hDdb5KTpVxLr/W8e+b/Y12c4Tax/kaNTTaIOLxYE9j8GPAaA/3Yben6Po04QlYmVOnVMJIeFgl2DFTIGh7sJ00+ub+O223zzRy1qUmsT6SCcPEAd6PSQyM2Sdz3fesfbrFFafZ6FT6lBIhQ9agaEApa8SXZQOmcrz9ZB0kTq047SU68qbFnU6Eecnn8/jYx5ndJyorVtRAWeqtpC6dhX/tdqsHCsQ7i/OjL9DL8GGS+oKVmUnIHdNwwHeeZvh1IJm9y8xQNQcK5OLLJz6hL/ALoEkn5+Myq99/j4jBkYQef5Q9L6iBE+wJljUQWwKibTOPl6Hoce9MPFvkDbyO1UGQt+n2mcNJpXBsf8Kq8wq4v1F4HJqSSpQvY9YXjfmxGOYTFAWFE/FzLFkhWUBVdmgpmLxt2AqFhdK5TDgBQgeWvsGUoWwM21tZm3wUBEgZSqCtXMga0Xt8weeh7QfWnfvzsapBU7nqCCNyG502EgHBoJFpWXIu9cw/pnxgAgqXTB9AWWZrreyTduQTsiZvZjkov1mWQO7Ga/w90/HVAKVeSiVYJOYsSrkmC2u+Vqc+OMEL+peZN83+8QEqqgLoQoLk0qjYOfdTd/AS73UJ/ypdCp6XtiT4O7B7SL8hfYK5frd17N90HZWVYBh5Hd1H2w1yK/SgkKbcEnw0kxsFsjfRrBcbHJZ9aJmRVFltfA37a1pXPDlBR7t1qIrF/G//v8TL+xSthbdB2OXiImkl3Yj1F/MlM2S8jqBnJ6ky6Qu7O9vwCq34qtsH4sOLwKL3B903cBuZULU+Yw/eJiZ9vcB+GjgRyy8cKHH+uLM+LOr6/XF99L+zOo+i3dnvsvMBFFPwmExZiw1svvT3WRur2vr1hauXXUtt6Tegh0xYGkV3qihTsv432HgK7UOHcg9wNtb32Z3+e8AZE+5mimvTmlTM6ufXM3m1zYDkKcXNXlCtWLSWV4O5ye/Sd/MYWDy1g/t0BRsh+Mf1HKKcDg71leX3m63k7U7i9wDdS0bAxICuOPIHYx+eHSjTVpsFr468TrHo/4Pv8Aq1xVtvHCr6aibuO1J32fgkhLQJdU5FRKkQGkRf3eZZXWfC2lr0tj75V7sNs9MRvd+vZcdH+7g4siH6Jb5NOGKrh5p1510KqtPQwac/tlZW1UikdSy+4wbHce/tv6L7rO6e7xrdrvdWbLkHt9yUZ+tPRdJriDzDy42buX5MfczJm4MACkp4mOFhIjNcAerr13N/+b+z5lVRZ8nYcT/6t6z9xNwqR78GrDo3fs4bL4aylOb1UVHxl8ZmVglFRQV1b0mcUIily26jKghUc26p6tJmppE3K0zKdcJhcmRqdoomcshd52be9YGgodCxKSOHewwfSeM/cX50vHddNTQcziHSTdv5LORn3nsOVITsxkqdaHoR4RSrisnWBOMQqYQ1p5/jYLTP9Z+g91enR1fcljUTjeVwP7n4OBL1dcd/xB2P9z8jkRMhcHvQGiN+Y1FDwf/Ayfr+Ts+F0n5FA4+DwiHvPtH3M+DIx8ERMYfEgnSLon4hos15Ix3ZjD3s7n4Rrh+TTnprdnsmv4Yeh+xxvAKf17an8UJsPFy+nzYk2WDlZgtqzizzTUbNr4RvvS/pj8hPaoKNxeAzS6lOPQOb3H05lC4C7beLBa9Nega1JV7ht/DvcPvrfdt7Vnj7z8T/sNVwx9BnXCpqIfRAA7hL6T9x8BzA7sFVgwnouAFACxlVcJfjYy/LpO6EDfas7VJel7Uk67Xi0W13OpLia0XxMxxe70ALw2Qtwl+iSHJsAwAu93A/h8Ok7G9gYIKbsZqBbNUbOb1l+bAvqegIqtd+vJPJz/oCmyT1oJPVJ0ISl2UDm2Ye32ZbRYbbye+zfIHlmMwi4xguVVLmGwrrJoCWX+5tX0vzWDHXaIOp9XoPOTINHBsOEYOiuShvIcY+cBIlzdvsFZnikf5FsCKUU3XJ/HSKXBkV5il4ouUqYgnoGvbIsMuX3w5M96ZAdTN+NPrYfOZeRTGveQNROronP4Jtt8OFdWFHxsT/krPlPLxoI9Z//z6OuckEgkhySEEd2t809ZRngK7hCjHzni3W4UrwT/MLqtZNDKnDwwEjSkWjSWCclPdtIx5X8/jsdLHkEg9Y4O37Z1trH9hPRMD/kVy5jOEqKqcDfY/C6nfeqQPrsYh/HWKjL+8DbDhYsjf5DzUUer8/XH3H+wetJsl05dweUQ8ZC7rePaMLSV3PRx+DYzVAS712XzKpDJGxIzg+oHXI5NWZVuGjYW4S+reU64RmWwN0f8FGPlVs0s3BGmCnHMAgyq1w1rWmky0LHtnz0Ow+Ro396qN2O2Nu7BtvwNWTwc32DQ3C11XUOicL+MDRMafw+rTIfzZyg2UZZW5JXOrKRzzkEpZ7Xkmuu4w4JXaQpy5XOy9b79dvD75BayZCfpTcPp7SF1QHWxgLoHDr0JlXsON28yi1nnqt+ATBcl31q5DLNfC3FRnmYZznqEfwOQNgBhXXp/2Oo+MeQSoDnIoKrRTcroEq8lKUFIQW97cwufnfe7yrpjNEmxyJSaleK51BOHPu/vqYqTSTqaldr0ZNJEoj4svfJ8te/jzpiP0PHlXm28dMyKGmBEx2O12bv/9Djaq/UgqfBzLwHeh/b/7HR9DBqR8AqGjamXPhWnDeGv6W3Uufz3qdaKHRTP+Q+ERXVAgnh2emLPmHswldXUqd869E/+4xusj2O2QV/UM8wp/zUMqldK3b9+GxxeZGga+hl3RBwBruXi6lRhLsNqszkm8zWrDZrEhV3lm6B925zDkWXL4GGQ2X5QaO9DJF1GdGbkv+MSi9RX2BzKrjF8u/4H+1/Yn+gvP17FYfMNipq3U8fFN0MueAQe+gYT51X74XjzC2eOLQ/grq1o/Xfm7++1XrWYrmmANCo2CAHUAPoQgt+nwkeWL4BdTPSHAXjyLT4zIeLGZsUhkFFYUYpCbgBin8CdTyvAJcX32btq6NJQ6JasnVfLia+X49csFfar3e9EJqDN/yfgNctYIC1+ZKJ7m3PSzlBKsAmOlnczTVuKTWj9XqZkZ4BT+fMKw20VQw77iSViTJ4HnHeW8tISkGyByaq1gwu8tl3Oo/0r65H/GbGpbkPnH+jPl1SnEjakd6GYsNZK+OZ2E8QlNzoEdlpQKawBhId6tkiaxGiH7b1CFQMjwWqcCAmDM4e3IZRJGxdZ9qzqgbQUUm1wfncW8r+dhKjexpapUmFKJ2NQ+8BxEz+mUdvOdSvgLGQWjvoXg6u/JI6MfwWA20COkB8YyI9vf305E/wi6TvdsNmZIcghxo+KY3m86Cs0cUWOss9PzAexJN5FusZFyajVj4sZw4oSw0ujall9vzhqwW0XG2NmEjhQ/zUQikfDMuGc4dkhDyp6QeoU/s8HMysdXEjEgggHXDWhtr1uMY3xZ+ehKTu2sINRvGvf7bueqac2YOAx+u2PPUctPwm+9oMd9MODF+q8x5kFFZuNCrzsxl4l+6rqDXOO0+iyqLKKksgRfXzH4pXWZwJtLJiOXe36PyWyGrtu+Q3UgHa6rUd9PlyQspGui8IWgQdXZ8Uk3gE8sBPSDcb+J/7dWCnE9aAgM+9g5T66X8lTIWAp+PaqPOWpvOt6nCRc//wT8ezZ4KiBA/Ddj4QbeumoV5z1xHuOfGU/UkChkStcvBDJ3ZaEplWCpEv4cNQcdtIdm5J3N/tMZ+DIAuk0/AZDSK5Dp5w92aRMVlgo+2PE+REDXrMedf3hemiByClxU0OxC4zEjYgjpEUKQ2NPHbBabGzpd4+9zBac3nOaPu/4gpEcI/rlPwqmvYM4JUNeN2C4vr46O8Qp/zcdkMqFWN/Lw7/kAGpsQehXW6gj2EmMJQZogTq06xYLpC5j1/iwG/WuQB3oscET4yq06bontC3+oYEbbavd4aSWB/WDaZtR2OzPS+lJZomXU63p6jWkfz12TwYLMIiLb9gdPY8y490Cb2C59+adjzt+DKvd3iLuUIomStb3nslmq5r/s8Ej7Co2Cm3eIem0Tmcgzz8DOcijRAnOLPdIHL03Q6xHxA6w7tYpJX02ie2BPunOI0tLqQKOiU0WUnC4hYVyCy5r+8ZIfCU4OJuHZ61FaVRTbg2Gea+1EvbiPWvOX7JVw9C1IvtsZmewQ/kqMJQwJB813n/PdiCIezXugVe3ZrDZsZpuz1lvNjD+zGSxVAe5a9yYye3EFfsnipwZmSRkmRT6FDUTCj3pwVJ1jx5cd5+crfmb2R7MZfHPj61xHTRiFJdi5puLwG4ANej7Y4o9wzmO3wtrZEH8lhHxT61RAAEiQYLWKYCK/s5xSTeUmsvdm4x/r32TgaEM0uT6qQUhyCBXmCg7s20S5OgilsgcggTnHW9V2R6BTWX1qY0F7Ra1D8/vNd/6/Id/AykdXMviWwR4X/obdOYxhdw6rPiA9B6JCVMGgDKLni74YzAaO3XmM48dFql/NjL+FBxZSWFHI1KSpJAVViRKLIiFyGoz8ou59N80HTRRMr+1KRep3kL9F2ITaLWCtAN8uTXbzvpH3sc4Mr1rqr/EnVUjZ+vZWel/a26PCH4jxJWNrBvqjZSiG+dPVZwgDmxOf2tHdzVQhQrjVNWDZCjDmh/a1uz32rrCOnbYNgoeiU+n418B/EaoNxWa3oauKM7TL5FRUeGbP82xMJrAq1Ji0whY8XNuEyFbDurTW/MY3ESrzYflQ6H4X9LwfqEdYr4lfN7i0rNqJJeN3WH8RjPhcCKbmkjrW+uc0Nouw71cEgExJrj6XzLJMEgISCAwMAMAYEYfKT8X659eji9Yx91P31K9cdc8Skk4ZWTfcW+PvnMVma6dU6DbiqxQh/qnd5MSdP9Al91z/wnqW3rzUGTkpscmZkrQc+ZbLxGDkpXFkalAF1TvxLDAUsCtrF9nl1dY3ly26jEkvTEKprH7wecrus8f5Pbho+UXkx+RTJA8RkXzy+v2SHdl+/v546zc1E5vNxtGjR5scX6RSsZEltSvRyMVsyGH36R/vT9fpXd3iY90QP1z0A/kf5DM75F5iCq8iwzy2tuWBl3ZBIpEQrxqA1tSNiCkDiB7m+Ww/gMkfXMTyeVb8KvoREpAE/r1ApmyXvvyTsdlsZB1dhWT/01C8F18fCWWagxQrRM3QA98fYOMrGz3aJ3NVWSXvM6JjUp2hJdJCLRYwVDlx/nn3n3w95WtsVtfNhye/MpkR943AWLW2VXXgkiRealNn/tLrYZh7SmwaVuH4PpUaSwkPh9LQJPyGJtd3u2aReyCX5zXPs/4FYfeYZ6iu8eewMH58zIVodl7c6ja8eBC7vZbVmE4uIqeLTQ0vcux2O3mHq4XBmBExTHppEt1nN107zGE5qLRU100n5VNI+YfUyWkpch8Y8SX0uLfuKXn1mrS+2l3Ze7P5fMznHPzxYKuabu76CMR3oqKogpMFJ3n69Gg2Jo8RGX8Sidh09e2cgWcO4U+vrw5q6KyoA9X8a9u/GPvvse3S/tYzW/ls12ccO/gBFO1tlz64FHMpEn0ayYEJABzITCGrqqJCzYy/97a/x+3LbmdrxlZxwGaFgD4iA6k+Br8NA16qezzzDzj+rhAcfo2DvU82u6uOxID6hD+ZQsZ96fdxwZcXNPt+rsAxvlyz+hqiXroTaOa6xFLR8etDKvxEzeWkGxq/rj3tbsMniBqyquoA5U/mfsILk14gUBOIXA6OmI/UTVksv385lSWVHu2i2QynBs6j210Pc+a+M7w29TVxYt08WNuIqGQqqlU6ARDZlT7RtS3oTUXi+9QQMjUoqx4C/j0hbJywR836A7KWg8XQ8HvPNY7+FxaFQ6EIWp717SwGfjSQ9WnrneNLiX88dxy5g/HPjqfvFX3d1pVeN40mO2k0UgkopIo6wl97aEZe4e+fzrH3YM1sApQi7NUiLa+3ZkJrSFmRwtHFRymqFDN9hTWIriGH4PQPjftJexHYrFC8H8pO1Dl109KbGPzxYBYdXlTvWz1d5883wpe0hDSGLxzO1O2/wcQVIk29Hhz1/ULbJ8no3GXnffB7X+cCe3TYdOYmz3XafAYlBXHFkiuatenhCux2OydXnsR8zMxVIW/SLetJNlS+D0P+65H2vdSDzSIKR5/+2fk9cdg5tgdGI/Q68yrTT+3lksSRYGyHwqReACjXDsc6dSdEzSA0QMwHrNIKTGYbe7/ay5pn1ri1fUO+gU2vbeLMljNAdVa4Pwcg/Vcwd4ZQ9nOc/K1iE6c8FZ1SDCBlpjLnotuRbTDopkHMfHcmdqvrNj0GXDsA40gjLx+7muORzxPhc1hEllfmuqwNLx5CEwm+CbXqgtUU/iIiIKv7OHwund3qJhQ+CvrO70t4PxF5PavbLG4dfCsDIgagF2Vl8VEZkFj0rW7Di4fI+gu+V4paOFX4KcQip8Tc8Jxh2Z3LeL/3+5SeEQNTQEIAYx4Zgy6q6ZQAR8Cq0lLtosKk1TB+Wes+wz+BLtfUKktRk/Kwv9mUPJ5H1t5Z51xIjxCmvDaFxInuF90sFRZeCXqFjXeIQCaFNUAIf+Zy0KeD1UUbIB7G11cEfkInsPu0GOCXKNh2i/NQVlkWK0+uZGfmTqQyKdFDo/GL9mvkJu5h/YvrWfzsYv619F/E7rsXdp8D2b3HP4QliUwMEJvO246nABARUTs76kj+EQB6hFRZBkplMPEv6P9c/feNu7h+m8+RX8Lck6I2W4/7If6yZnWz1FjKceMG8nWr6w0QAPCL8XNm8bcHZouUjMDv+bviNQ7kHmj84j2PwE9BUJHjmc65g10Pwqlvmr7OnYSMgL5PizljAzjKU6RvOcOWN7eQujrVI11z4FivqpUKov2iifGLEQfs1ob3u/c8Jr4fPwWKZ48DZQD0fAiSrhevzyyBn8PgzOL675PxOxTtq37t2wUmLofo2XDeIpi6WQTm/FMIGgxdbwWlmLhF+AqL+OzybGeNv+Ji0EXqGPfUONQBarJ2ZbHsrmXk7Hft32rM9D7kxw9mZsEKjE8amdZ1mkvv3xq8Vp//dIoPQPZfhKkvAqD7/mL+uvADuu+8AZVf20Kqr119LaZyE5sLNgOgtASxseQp5lz+GEjOAfsEd2O3wrJ+EH8FjK5dcNzxUDlTesZ5bPfnuylKKWLifyYSHAypqZ4T/qwmKyUVYrXh2BBsCIfw57X5dDF2G0ikhIWYycpS8ES3nxk/vv26I5FIeLT4Uex2OwsWiGNKbzJX+yKRwt7HIPYijmgLOBKdyrqbYznsp+TGTTd6vDtHvt+Lf44GSXJ3ERVnKoLzT3m8H17AKvODwL4gkxEWUD01zCs2MP3t6ViN7q11UpJewl8P/cX4/xvPxQcu5oxKQx/ZL4QZvoX1L8LsoyI61Uv7UbQbDj4P4RPQaUUdhTJTGTo/O5WVEkpKIDISkue2PlOrMU4VnWJN0QIC/UfRTesDm+6HKRtAHeaW9ry4CZsFKrJArhWuFoC/WkQr6816QsKsgIycNqzBg7sFc+GCC52vbxp8k/P/Dx8W/3334J98cm/r2/DiITSREDWzVoZogEIsIMosDS9ykucmI1PKOLPlDGe2nmHyi5ORypsX75xXI+PPKfz9U2rktAVHhstZGSJqPz2FsrXsza+bteAT7MOoB+pas7oDm9XGoJsGUdilEIw1hL+sP2HDJTDqO0i43CN9cSUSiRBxSkrEjzNLtSMi04jNaU2108gPB3/g3uX3clnvy/j+4u8xG8xYjBY0gfUHELuL3Z/uRooUroEV/jM5P9nz6yKXEzwMej6If1YRsI4DGcJxq6bNZ4GhwJkVnxzcgvmbIxO7pjOVRAJaUYfNUU6oOWw8vZGLl81EF9uXkEP7sFhEtnBNSjNK0efoiRzk2TrwNrONo0uOYkgP40zIF+wuXM70zBD6hPVp+E26rqLeaUefnx79LxjSYeCrtY8bC+H4e6LuaeL8+t/bTlhsFjJKM7DZbSQGJqLVir3F4LG9uXVvHGF9Pfs7NxbqiTy2A9vpJCCm+sS4JQ2/qbRqIhoxWdRPd6CNc1rgAyKYJnJq/d8ju11Y7gb0hSnr657v6N89dxA+Tvw4XlbZrubocwiomkKeHVhQcrqE7e9uJ2pIFOF9XTfPcwjCSqXYE5XQjpmzVXgz/v7pDHkHLqtEUpVCLbEZsRotWCrbnpEnkUhQ6VQUVhQCIuPP3x+QKtqvSGxnQqaEfs/VGy1Vn/B3+OfDbHptE+D5jL9ldy3j+IDjaAwarlXmwOHXGrzWYfXpzfhrGTJZE2L5kLdh5l5Cw4UHRXZ23Ut2fLSDxdc3EDXkJoori8mpOINCmcV5mjtE9o6X9kEihek7YfBb7JJ+yInIF6kIsBGUFNT0e93Atv8sJyJlk9h4SbgSut7U5Hu8uAeZVCKEV3MpWqUG7GKCmlNUTnC3YML6uHcBEdw9mBu33Ejy/GS2ZmwlQ7EGiV1OedDlMPLrWpu+XtqJuEtg1mEIGekM8LHZbWj9hY2Mu+oLVZZU8k63d0j/r4iKlVt9OcNcOO9n8Gu4kLuXjkOt+UvhTlgcBymfOQ8FqgM5dPshztx3hqgIKYGZhyj5ejEVRY3YG7USh9WnI0rcSwcnoA+MWwxR052H/FVikVNqyW/wbV2ndWX6m9M5uPAg29/bTmFKYbObvKbn7Yzff4Tumc+KKHG7DUqPgam4tZ/i3Ofgi/CDFvRpdU7F+Innd15FlluabnJ9VIVKp2LOx3OQzxSKglP4802E5HvEd62T4rAx6/B1/iQSEbDT9ynnIYcFmsNi972e77Fg2gKPd+3GzTdy+pHTAJwMHCuyZjo74eNh4KsEhYgyPilFIuOvpvDnyPaL849DW+UAhj5NODzkb63/vnmbRcZSysfVx4oPihq+Z9sSNmPc7BIo6gAaVKewY6/X7nPZ7cv4dPin2D1soWkuNvPjhT9iWLMdk1x8R5us15V8N0z4s31tMptDxm8iK/RsVEGijvbZgqCnsdth9XTYcZfz0Pvb3yfh7QQe/vthoLpWs0XpQ3i/cCQe/p1XZhYRfWwN23d/zAPLHyClMKXpN41cAFfYhDjYWH81kcKONWJi9TGbFTZcChlLYMQX0OuxNn+GcxWn8Fee48z4MxqhsoYbbJfJXbgv/T76XdXPZe2WnC5h+Zx3CU3d1qFKlnjVFxfT3Mlnh0EqB4mEfuH96GqbRUrfJIZ8eRfasLZVvLearKT8lULJ6RKn8Ke0BNMjaA0U7HBBx/8h9HkSYs6vczhaJyLlagp/sz+azV3HxYPRIfwVNn+d2yYiB0UimyDDpDQxlXQ49VWD13oz/lqOTCajb9++zRpfIkRWuzNa3lajLkr6xnT2fLkHU7n77WxM5SaO/XaMd399l4eyYslNuI+esvch37O1wrycRdAg8InBRy427vOvDGPe1/PapSsj3ryM36f/wM9R3dkTOBl6P94u/finI5PJ6Jvoi+yXUDjwPBKJBIVdzAHySvRYTVYM+QaX1mw7G6VWSczwGGTh1WOczKbB5t8PEq8ChXeXvt1RBYN/D5D7oFVqndGLKj/hF+ywGCvPLueDvh+w6t+rXNKszWxD4aPAZBfPLZlNS6U8CWIvdGaMeem41Jm/aBMg+V7xLHJcI5XRM7Qn0X7RRERI8CnJRHVwDxUFrRP+Dv9ymKU3L6U0Q+yCpxSmUFhRiN1uR68HudTIxJj3IG9TGz+dl/YgsEr4K7c1Hd144TcXct2a6whJbv6iw1Smw9eYTIQ6QWzamIrgt2Rh3+alfnziRG0hm7nOqfhgkaFTZMmqtSZxsPSWpbzf5/1WNduS9ZGD4spiAOSWKuEvaDAMfqtTC3+OOn8d3uqzHs4W/vrO7+s254DG0IZpKdKJdBCdqmlL4M5EUlASAFlGIUrUrO9Xx+YToOy4cHgo2lX/DX2iIbC/01IPgBMfw6rJYKjem2L3I/BzaJOlHOID4pEgwSorxyTPr1f463tVXyY+PxGbxXO1sWQyGQNGDOCCry5A0q8vJrn4HMGaRtJqO3ptv5qMWgDzMuo/pwyszt5sLyQS8X2qrLaAiPcXfUotTgWqhT+9HqxmK6lrU51zP09gCw/n4Hm38GePH3hjyxsUVBSIQKHDr9dbrgkQa9qWCJSGM0JsByg5ABlLoWA7xF4A0TPb/BnOGYwFsOFyOPEpUMPqU5+NWl1dD7Jm1p/SV4lfjB9SmetkMbPBDFIpFmkpi/zHcvlPdZ0E2kMz8gp/LsbTUShtpjIPMpdzY48ZXCP/jdiC61xS46/0TCkLpi5g6ztbqzP+LEFM8b2+lq+7l9ZRX8afX7Qf/rFi5u/pjL8htwzB/G8zVrmVt30vgnG/NXitI+PPK/w1H7vdTmlpaePjS9kJOPpfEoLFJOPr/HtQ/0fNqxuro7WmvzWdJyqeQOnrfs/NolNFfDfnO4x/isLF2YYIFsuyvZFJ7Y1FD5W5aKuElJKK9ivy598vnlMJeyiRH2+3PnipGl9MCuxJNzpr9CgQK6n80nJWP72aV0NfpSilgcIbLsBqtmKuMKM3iZpbMpsaCbIOFSn3j8dmEfVKzKVIJVJ8lWIMUfqKMcSRaaAJ1mA1WZEpXLOo8Qnx4da9t8I14rXM6uu1je5E1Jm/aMJh8Jv11wcCwsMhq9tYdk17FFVEYKvaTN+Uzq5PdmE1WTFbzXR9pyvBrwRTVFlEeTnolIXMibwT0r5v7cfy4ilsZlHDukaGaIRPNDpDP4Ls3Rp5o0CmlBE9LLrJ62riCJp02nxK5GLuGjm9wff840mcDxP+AL+6/yaJoeFgl2DD4hR3aiJXyVHpVFjNLbcUb9b6qIqMbRksmr+Igh1VNRytQefMs8Svygm9Uwh/qd/CvmecL88W/ia9MImxT471aJdsVht5h/OoKKggSQGXn3wCjr7j0T64hawVsGYWvaQiiKZMdhIkdpKEDojFZuGDHR8A0D+8f/X7QkYLh4fYS+q/rzYOJq+p7UqVdCMMflvYXDoIHCCCtJqo062Wq4nzFxaH5epD9Qp/vS/pzeiHR7tsbtkc7HY7RruRflf1wxIR03TGn80Cm66CzddCyRGP9bPVqMPqllEoOSyyADtKhvusAzDmB+fLhIAEANKKRXa5w72hvBxO/n2SL8d/ycGFBz3WPbNdgcE/nAwfUaokXBsOeRtFjdCSQ21vwFIBf0+Av0aJerSB/eH8NOj5QNvvfa4hkcHphaI0BRDuW53xB9WZ8WePLyWnS1xa4y+kRwjDvrid1OQQzsjWsy5tXZ1r2kMz8gp/LsZm81wUikvI2wBrpkPOapRKUJUXkLlkJ8VpxW26rTpAzawPZtFzXk/uGHoHDylTSM54jiOal6HPE67p+z+BnffBX3Unv1E6YZuSVV5tm2IsM1KYUojFaPG48Aei1g8gNnVq+lOfhdfqs+XYbDZOnjzZ+PhStAd23kOMj8iordBLMVqNFFVWb9ZrgjTIVZ4p7aqL0nH+5+dTPkT4akltfthV4d4MjfZm5ST4c3B1Lc69Rax8fKVL7J1bgt1ux2SyY5XqCZRCt7131drY8+I5bDYbJ9MLsQ35COIuBiBIkoS2sjsVBogdFcvgWwcj17hv7Di6+Cgv+LzA0R+PAiCzi2LkEScugaWej/z2Ug+FO+CXCDjxCQDXDbiO24fcTrCfEIkdG44yhYw7j97JuKfGNXSnVlFuEs8Suc2X0fIbhc2UtbKJd3lpb5o1fwHe2foO9/15H2cMx9EGKrEpVOTmtc6yacrLU3i44GH84/xrzYH8Vf7o9VBuCuSXkpXQ7bZW3d+LB5HI4dg7wpKsigHBoxh3aC+zLP9zS5Mf7X+do1FPIakKpEPpDwNegNj2cUfo7ESEKVBZhF14Vlldu88Z/53BjZtvbNWGfnPHF4CC4wXs/3Y/5TniWaKwVAl/e/8N6y/uXJk6Z9GpMv5O/wAH/+P8fdcU/torgF6fq+f9Xu8T/XM0csAq14FM3S59cSkV2ZCzmkilgsuiH6V3+tt06WrBR0yxSS9Jp6CiAH+VP/eNuK/6fXKNcHhQtyBKO7CfsLismcmUcAWMWSjsdJtgUKRwASjR7qxX+GsPao4vFWYjVpkYOxoU/qRy8bszl4Fvguc62lrMpWL/yFRj4Ej5DLbfBqUdU7iMDxAZf3mGPPQmvVP40+shbkwck16aRJcpXTzWH2NJBXZbLhabyJwJ04aJTLzJ6yF0TNsbkGug96Mwflm1+406TGRkeqmNwh8uM8LQ94DaNf4Ap93n2XX+FkxbwE+X/uTSrpjNOAMFgn3qZgi3h2bkmR1gLx2XoMEw7CMIHoZKBdridE6//xvZUy4jID6g1bfVBGkYcusQ52tFeRc0ZjCExEKsC/r9T8FcDMZcMTmuMZFyDCDlpnJMVhNKmZItb25hzdNruHXvrQQHi4HOU8LfmmfWwGGQ9IIEqUnY4tTzQLLbqyNpvRl/LiZsHExeiy+9AbDoA8Afiiqqn25Wk5XsPdmo/FUtsj1qDT7BPgy4bgDFvxZDIQRKJQTKDoMlDuRtsxL20gZiLwJjProcMTioj+rZ8PUGht4+FL8Yvybe7DoM+QaWj3qNKUPHcGjuMjTFu6BstMfa99I4d/lsZP1OCB8FyXMgeY57xTf/OH/6XdUPaZQUskBmFWOEXRMLinK3tu2lmfjEQLc7RLQp8N8Z/wXgp6q1krtqC5VmlHLox0MYVKKWoMympUIeC4EDQapyT6Ne3Mu228BmghHVwR5f7P2CXVm7mJo0lYjAWKxnikg/FkBioqbFt5dIJWiCxPsccyB/lT8yqYzycjDb1BSrJ4K/az6OFzcikcD5qaAIcB5yZGm5wqGmPhZnvE9O1Elm6qYBXZu83gsiM3P/s2KDP+nGWqdCQkBljsSoyCGzLJP+Ef0buIl76Te/H70u6sXK1JVsXOCDtXSM+C4V74OCrR2/HlcjOIS/Dl/jD2DQWzDoDedLh4hitBrRm/Uc+uQQaWvSuPiHiz1Wr0uulnPek+exOnc1R82wf9AXjIlzwaZ9e9PlGuhyDQqgf95s9PkwfEr16cTARA7cdoD9ufuJ1EVWnzAVi8AqVShIGxDkzyyBM7/AkCqbXqmy4WubweDIwfxy5BeKfeoX/k6tOsXKx1cy/O7h9L2yb6vbaSlpv6axeu5q9OPHQxJIkeKvPmvyYLeBpCqfZsj7IFV0jvHk9I+w9V8w7vdqy8i+T0PwMAge3r59c1ByBHLXQMw80IQToA7AX+VPibGE0yWn0WpFre/yclHHdcwjnv27tf29mmE7trO+hw5CQKPQABoIc2E/znqmemkAiQRk1Wn8XYO6ct+I+5xZog1l/A2/dzi4MObk9MbTpP+Sg90qBMcma4J6CG/G3z8dbRx0vZllOcd5uETDyhH3EvXIVcSOdq0654hA8/PcvvK5wYjPYfaROpOHAHUAD458kJcmvYTVJqxR4sbEMerhUagD1c6Mv5ISEXHgbg4uPEjYzjCeGnE3t2a/Cbsfrve6oiKwWMTHCfImfrkWdSiEjcUvNBiVChQWIbzWjHbX5+n5dPinbHtnm8e65cjSmBx0knFlvSDbNXWfvLSSXg/BwJfxU4uosaNDKrjz6J1owz0rxkqkEoJGJ5MTnk2aBUrmnoEBL3m0D15qI9l+K+x7Gqi2TvHUJlL0sGjmfT0PVV8h5EhtIhzZ2PsNYR/mpf3xiYGh70LE5FqHHfO6mt+VjG0ZrH56Nfo8fZubLThWwPL7lmPZJ7KS5VZf0rSPwaSVnWNjxUtdivbUqR3krxIbaSXGEkKKU+i1/mNO/n2yVbfP3pNN3mFhL+GYAwVqxJxIrwepxIrWp/Nm9/zj8ImpVefVncKf3W6nyJwNQFxQVdHs3HWwahrkrnd9g+cKEjkcfQvSFtY5FRwMGlMsKlMkemPdLO2yrDI2vrqR9M3pbu+mXC1nWo9pDNM/Q2jZZPFdGrcY5mW6vW134hD+OkqmVKP4JoBvF+fz20fhg1ousuvyDflk7czi8KLDHqlH70ATqGHicxN56tGnWHjxQnqG9PRY257AYoHdwv2OIUNqn9MqtYyIGVH74JE34ZfIxrO+CnfAyS+g/ARsvgp+8ofC3XWvy14Jf42p/1wNHBl/pT676v0e66J15B/Oxy/Ws5uJMpUMbbiWcrkIPvNXBiOVnLWFvvUmYfFpNQrhobPMTYOHQb/nQJdUfUyhg/hLO85nyF0tMhCL9zkPObL+UotTa1l9OrDb7B6rBWmJiCEjMQK9Vu+0lqQiC2wtt6724gKK9jj3GqP9onlj2hvcPfxuoOGMvyG3DKmVsNRWDv98mLT3lmG3N2EN7GG8GX9eAFDKlJippNQ/H0WPJLRttGHc9Pom9vxvD1csvYIvcr5gjTyX0RED6XPiQfD7j4hA8tJqpBIpr059tdaxxImJJE4UVgp2OygUQvQrKoKwMPf256YdN2E1WtHoLHDIV2SS1kN+VWmH4GBoh5qmnRq1uhmWIzYrEpuR8HAfUsrF081RxB5AF6lj0ouTiB3l/rTbgz8e5O+H/0Z2pQyUkFLalTTdE8T7n1uLqc5KoI+w+izQlBHcvZEi5W7CJ9iHiIens2vXFYBYeHppP9RqNZJTy6rseJ6tFRVXnFbMykdX0mNeD3pf2tut/bDarASqA7GXiw6cK/V3zkWMFiOlxlIUWg3gW8tiLH1zOuv+bx2JExLRjm/b33bkoEiuX389mlgNss8fY3+OCpU30a9TUWf+MnVTnU0lP5XYzCs1lhLWN5ytyRMJ8Wnd5HXR/EXYbXbuOHyHM+MvUF0t/I2MWcQl1suF5VzcRa1qw4sHMZwRGSgBfQDxXFjfczB/a05zZ9FWugS6ztar3FSOCbHBmxhSJfxV5kD+po5T86gjIpHAjD2gjqhzys8PRqQuxmqRMPaeum815Bv4++G/GfvUWGJHtnx90qz1EZB/NJ/K4koiB0ViMolFqPNZcvZGfiejU1l9Wo1gzANlEMh9kEgkvDL5FTQKDQHqAGa9P4u5n85FIvW88DAxcSIU7YW0L4X1fSOlSzoFFTlQuJNj2f0pMauwRRyk3NeP97ZtQm/W89Coh+rPqgweBt1uB03dv2cnyfdA8r2ihEf8FcJmz7+eNYJEKn6nZcchaGCDtxseM5wbot7iyKohFNWzLA1JDuGe1HvQBIpsfmOZEZXO/ZPBrvO6MuPhGey5z8SYQ9u57qaK2hfYrFCZLTIkZZ1schrQV/w4sJnBWCisJDuK8Bc1GyYkQdAg56GEgAT25ewjrSSNrlVLDH1VnOGx347xy9W/cMGXF5A81/2lIiq69WNn4DFsMpvTWpI/BoCuO0zxBgt5nJ33iNqKF+XVOeUQ/twdIDP8nuFk+XajNPNrAII1nt9nqw+v8OdiZJ1NzbBWwrJ+9NEJ2w2LtJzKSjs2qx2prPWTYIlUgs1qQ+mr5MsVX3IwcC/DzW+COrxW1KaXJig9KuowRs0ETWTT11fhyKjLyRF2n+4W/pRaJTj29vo/3+B1DuHPa/PZMmQyGT169Gj8IpsFFqoh9mLCw79HUVI3408ilTDmUc9YIMgUMjTBGsolIgTrRHEfMoIvIl7nkea9NETqt5D+Mzf1/Df7f7qAQGkE5dnlKLQKjyygalJuFJtrkTIpiqwVYlLvE+XRPnipMb50OyXsaYAl+sdY22sZ6vxHucIyjQPfHyAwKdBtwt+eL/dwev1ppr42lYy7C7n0UnFcfeL/ILCHiD710r7YLLDxcrEh1OthLvvpMhYfXczTgz8EbqmV8df70t4kTkh0SVCB2l9N3Bix+SY3gsIGXSpfgaMaSL6rzff34l7qnb/Us6FUU/jrMyCExd3Oo6CVS6qRD4x01ooqrBD+8gHqAEBEhVsrwylQn0+Ij7f2QKdg8zUiivti8W+pVIJRnkelNJ98Q75LhT9H7XS5VUdkSNXCJu4S8eOlcXT126JKJBAcJCE3t/41aXC3YG7YdAPB3Vr+vGjW+qiKja9sZM//9jDjyAxybDqski4o5DI4/Sv49xI/nZROlfF36kvYdgtMWA6RUwG4a3j7PstPrTrF+hfWM/6Z8cSFrofdD4gg5s4u/OVvhvXzyDUt4HTIGY7EPMr8X3pwrOAYNruNPmF9mNltZt33Rc8SP42hqvH3Gnexs0Z4HULHwkUFtSz46iNIE8RVXe/hjSUNf48dop8+T89XE7/i0kWXtmrcaC41xxebSUOAYQjDz9ZCpTIY/ztYDG7rh8coPgB/DhJZgH2ebO/eCLSx4qcGF/W8iF4hvRgQMQB7lbOZQ/jzj/MntFcoUrlngjlELTdh6RimDRNCcPz8zj92dFZ6PADm6giYnHJhMZ4YmEhAVVTz2Rl/qWtS+euhv5jw3AS6Tm+7vXtAfADKHgEY8xvO+GsPzahzhzd1QNqjUGObkKpArkWuFDNGbRnk3PkfVj6+sk23HXnfSO48cifaMC0FBrFQO5Y/Gvu07RB7YZu7/Y8hZ43w3i4+UOdUdnk2u7J2kVMuHjbZe7P5ZuY3HF16FKgW1/LqBjy4FLvdTsb2DHbt3UVqcSoma8PWHI6+eIW/lmGz2SgoKGh8fJHKRcRdyAgiIkBhrRL+Kooafo8b6XFBD27ecTMTZk6gj+lGtMauKBTt0hUvNSk5DOm/0DVQQ4BhMLqTFbwe+ToHvq87xriTgmMF5H2ygeRj45juE4Nk3Rzh4e/F4zjHF4ncuSFfKjlDmc8+svSZBCYG8qTxSSb+Z6Lb+pC2Lo3dn+1GIpU47dsk2JAdegZO17UO89IOSGSQsQQKdwKgU4koDqusDKidaaCL1BHeLxy5uu3xhVaz1WnZ4/huROnfh1NftfneXtxPvfOXshRI+0HUg67CafVZWUJoleuII1ispQy8YSCDbhTR4WdbfZaXw8G8saTFL4KQYa1rwItnSbwWelaXEFAqQWkVm70FBtcWM88uFzafKnMEOm+gWsswlwmB1lJR55Rj3Vdf7Xm5Wk7syFh8Qnxa3GSz1kdV9L2yL1NencIFSy7g54helKsP4SPNhw2XwLH3Wtx2R8JR4qOgQLj+dGgCB4psMU39gX6GAgOpa1Ipz/FcfWd9rp7MHZks3bOUJRVq7JM3QOAAj7XvNoIGwYjPWbVvBD5GYed4JP8INruN6wZcx4yuM9zfB6msSdHPQXMzclJWpJB3OI+Co64d/8/GZrOx4b0NbHl7i7N0Ti0nEluNejrylo9f7Y7VBH+NhT2PiddyX+h6C4SMbN9+nY3dXut3fU3/a3hx8ouMiBlRx+ozvF84N2y8gW4zu3mka9rffmDYln6snn2Gt6e/Lb7vg9+AHvd6pH0vZxEzFxKvdr6c/d1sBn08iPVp6xscXyRSCeU55RhLjS7pgqnchMlkxy6xIEdZr/DXHpqRN+PPxdg7/GzrLCQSmLGbipJ0+Oszyn0KkHXvSkgP1ykzjmhbP0WQd+O/pURNh/F/1Epvd3Drb7ey+OhiPpz1IbcMuQWzwUza2jS6z+kO0OaNk+ZiNVr5dNinnBh0guOXLmBFv+EoB/6nTh2gmn0JbaOV7D8Nu91Oenq6M1KlQUaJlPLwo2LTIkFyHiNj42tdsuWtLWx+fTM3brkRv2j3++Q/OOpB0r6DYYnP0yPzAbAs6ZyT43OFvk9Bv//DxyYEHqM2iP43DXXpmN8cik4WUbHiMCP7vcGE84JhxGoIGdH0G724HOf4IskAcwGETyBQKzIdSirKkUglyJTujUyb8/Ecpr0+DaVOSVnVOl4mlyCZfRikXr/PDoFEApdVOi3R/JTi+WGWilS/igoR+eqY55nKTRgKDATEB7Sp2d2f7eb3235H8pqEvxQ5BKlv50TiBnr3tLTpvl48Q73zl4wlsOt+mLwOws4Damf8SfVl9Nj4A+a83kDbngu9Q3tz6+BbGRgpLMYcm0O+XvORzkOXa2u9VCpBaakS/ipcu/GbVSYy/lTmyOrvSOEuqMiEiCmdz8rNkxx7B/Y+AdO2QfDQWqcqAneyKfl+HtkRxsYxP9Z5q9VsRZ+rb/G6pNnrI6DLpC50mdSFoudFMIDCGohMrYNR31XZnHdeHMKqyQRlZdW1dzskwUPrfD9OFp3kZNFJ4v3jMa8x8+PFP3LhtxfS94q+DdzEtfS5vA++031JfDsR9e9qKp6oK153SrRx5GivY/dx8NVWR2edF3ceH876sH6bT4DdD4OxAEZ85pp+VObByc9FvdaEKxu8zKzK5kzwCgqsUuCqBq/rdVEvel3UyyXBZY1ht9vZ9dEuLIUWTs80kBG+k71FI+lOlXvSlhtFANOYH0CucWtf3IJMCfpUcJRh8esGwz5s1y7VwVwKP4dBwvx6v4+O53RZmYf7BVhNVlTpx9GGy4jxP49Yf8/3wUvjRPiKFN3s8my6B4hjZ2f8xY+N577T97mszXeT38XoE0jfHh/w7LD3uWpYXZGvPTQjb8afF6A6crvSpxzb/AsZeH3DHtzNYes7Wzm86DBGixGDReReT4nbCodfrzcS0EsDaOOF+Keqa2PgiB7INwg1LXZkLI/rH2fobWIy7RDX3J3xBzDppUns772fUBloyw7WiuKuidfq0zNERIDWmMQFRev4et7Xtc4pfBRow7Uui2ppiBN/nmDTa5sw6U2YTBDik45P5U6RZeyl/ZAqQCIhrfQkabEvcSz+R4Y9M5P48+Kbfq8LSZyYSPSb91MY1QeTIh66XAe+rrPr8tJypPseh9XTAQjxEyspg6UMoxEyd2ZyesNp97Utk6IOUPPDwR+Y9+tkUsJfR6GQgF9yp9+QO6eoUQfJOW+0lyGtOlzT7vODfh/w/dzv29xkYJdA+lzeh+Uly9mveR+jIhupbwz4JrT53l7aiaiZMPp78fddhVP4M5Wi9ZWg0hdiMZgbukODlJ4p5YN+H7D9/e0ATEicwAezP+DmwTdjs0FhIYyL/5bY3HtEhpKXTodKBQqLezL+surL+Dv+PqydA9ZzwMrNnYRPhL7PivpQZ+Hvb6NQt47D5Vvqfet3s7/j3eR33b4ZVmGuoNJSCYDCEoTSRwsJl0PIcLe2624UCpy1mT2x7nc1r216jSlfT2HBvgVEDopk+n+nEzXYs9b/DoecKHWAyIQ6R9ixQ/x3aGJP4v3j6RnSk58v/RmVvJH1eP5myFntuk5YK2DfU6L2ViOcrNzJnsRr2R/wIlZrw9fJ1XK3i34OBr84mMuXXM5p9VIOxz7Mmsyl1SeNuWJd3RlFPwcXnIZhH7V3LxpGrhNBNwH9nIdsdhunS06zOX2zM8ihrKw62/n0xtP8cvUvFJ4odGvXZEoZKZc9TuqAC6ozQVP+B+svBkOGW9v20gCnvobFCZC/DcBZdzFHn1Mr48+dU43uc7sj7yEyrFUqCTJpxygF58348wKnf0ZXftL5stxcBrRtc37lYyuJGx1H4NSqvzC7hHFRi2H3D9DdW5OlxdhtdQqPOwqFOoS/s/GU8CdXyxnx0Aj2PreXvXoonp1KsE/9futeq083c+JjyN1AeOSXgITs7LqXDL55MINvHuz2rhz6+RC7P91N1GVRlJuDeG/HB3Sb/yFdveEm7YuxAIr3caYwh/3hj6Ez9KWs7BaPd0OmlGHz0WGTn2Wb4qXdsCXdjCxmLtjtBPuKsEWzrISCAlh83WKsJit3Hr3TLW1n7sxE4aPgeOFxNmSsJE7dBR+TESoKQRnkzbLoKBRsF3VMwsehU4pd8XJTGX5+YiFVWlptOTb4Ftc8Z5KmJpE0NYmH3n4IikFu0+BjOwmmYFB6w2s7JX7JtUQ/gOsGXMfc5LmEakORVPiyd+pDSKVicd5QUkJ9mPQmLJUWrKa6u4Z5eWCxwMDIv/BJ/wJG/KeNH8SLR0j9Vlgxjvgc/LqjUIDS4gh+dK3wN7/HTSx+bQYSZNUZf0k3Q+gYkHfkNKoOQMiIBp0bEkOiIBdKrNnY7DakZ61pky9IJrR3KDaLDZnCPZtkX078EnmoHHqBxC5DbtOdU/PP0FDxHM7Ph6Sk9u5NI5iKYcv1QiiuqtMbpAkChEtUYGIgw+/yrBCbsT2D47uOI7PIeDO4Ahaq4FJD5xZ0APK3Mjr/Mo4nPkPskOt48YLjAChkTVhwTVkv9p5chTYOpm0BVdUGkLkULHrQRNa6bEwX4XBVrj5CZr6e2HBtg7fMO5xH1q4s+s3v1+A1rkAbrSW8bzgVEvGsCdXW2MSa8Kdb2/Y4ux8BUwEM/7S9e1KNRALjl9Y6VFRRRPxbImC57JEKQI3NJhwddDooyyhj34J9dJ/bnaCuQW7tnsks4UjMczy/o5QHtLfSvfgApC+C4Z+4tV0vDSDTgDIYEMqeU/grz3EGx5hMYDCAtsbwcnTJUSoKKxhw3YA2d2H2B7M58Rawkg7ldujdgvUCJz9Htu/fDA6cSFjxTCQrt7HhpQ2tvp3dbueGDTcw5bUpTptPhTWQNSWvw9Qtzfb59gLoT8NCH9jzSJ1Tjow/h82NzWrj+LLjZGwXESaezPgrqay2j/BXN7wR57X6bD265hQbydsEqV8TESqybMvLobzcjs2Vk/dmMu7f47h+0/UkfZrEt/H+mOS559QCu9OSvxlWTiRKL2r6WWXlrLvrJ1Y/7cLIzmZgyDdw8OQSNnTvQ7lhsIjOKjvh0T54qUan00HULOh6M0gkBKoDALDISsjPh/OePI/x/zfebe3/eMmPLJq/CINZZFTIbD50C9oJv0R1+to75xTbb4ctwnbPkaFVYixxRtzWrPM35pExjHlkjMuaLjcJj0Y/qZ34vUmw+yGX3duLe2lw/lIj5DbcN5yeoT0J8QlxLsZtNjC20JwgJDmEu47dxYh7hQCRXZ5NgaEAq81KZqa4ZknWe3BBhqhn46XjYy6B8pNis5jaVp/5etcKf3aTD77G7gTak6rnrCHDhCtBB4na7ox0CQ8HuwQblnoDVofeNpRpb0xrlejXrPURYLPYqDSJbD+5JQAJEjTp74l5RlXt2s6MJ9f9bUKqEnbPJQedh5zCX6V7M3QaYscHOzh862HkFjnHJMEQfyXI1O3SF1dSpleSXxaC0aph6FAh+DUp+jmQuHibOHCAsPqszIcfA2Dvk3UuifaPRGOJBImNLaf2Nnq7Ta9t4perfkGfq3dtP2tgt9mRVciwGK1UysS4FeJbf3B7pyVvExz/SMzH8jZA7rr27lGTBGmC8FGIsjHZhjPOOaNjDdJtVjceyn+I3pf0dms/yjLLUJ85QaH6Rz7e/ya5+lxR3++yClAGurVtLw0QdzHM2OnM4ndafeqzUSrBp6ra0Nl1/ja8tIG/H/3bZd0wmWBL98m8lj7P5c4UrcUr/LkYmawTLgoGvgrTtvL+8L8ZduJ3VAdPsm/BvlbfTiKREDEggvC+4dXCnyVIWDN1cisNj6MMhLDxoOte59TZVp/Y4dtZ37L59c2A5xYA2Xuy+e687+h1sBfDtBrkmcvAXLcgt8UiLJbAm/HXUmQyGUlJSU2PL0PegcsqUfv64u8PG3uMJuhNFVvOVFvr2O12tr6zlT1f7HFrn/3j/AkeHIxNKkTH8yJXoi1f49Y2vTSDgH4w5D1swcMAsEjLydt+iqwdWR7txt6v9xL08m50xaUUWMtFFKisk0fWdlLqG18cARxmWTH5+dDnsj70uayP2/pw3uPnMeK+EehNYgEvs/lQSQh0uwOC3J+h7KWZ9H4c+r8ECKEGRBSlf1W8T02rT1ex/9v9LL1lKeYSYfsotfqhj38IIqe6vjEvLqfe+YvdLoI91l9U73tUKtAVpRGauh19G/f0Lv/pckJeDeGnQz85hb/QCB/wiWpZKqGX9qPbbXBhFgQPAYTw52NMxM/QnxBNhEubctQJaqaW5KUmdhusnAi7HqhzKjxUjsoiLEAzyzJd1mSz10fA9euup/tbYj2ttAqhSa7xA5944SzQyXGsrTu88CfXwOXmWrXEamb8Wc1WPuj3AX/c84fHujT09qEEPheISWlirbIXjP6mUz4fsrOz+c9zzzG0Z0+6R0czZPSVzH7nfFJt44iLa+ZN7HY4sxSKDzZ9bWtQh0DXWyB8Qr2nw21izr89o3ExfvDNg7l00aUotI0LmaZyE6by1lm3GouNLBq1iOX3rMAsE5v34b5Vf2hF++Dgi1CW0qp7dxhSPoHtt4KlHKZuhJn727tHdUn5DLZVuxNJJBKiddEAZJRmONcgDjFHqVXiE+zj9m6d/PskSZu/ISxbfAcd2WVel5qOQ821KlRbYp9d52/aG9O4YskVbW4v90Auv177K+VHj5Hvt5ItJb+irCfpqT00I6/w52JsNs9n1rQZ/54Q2B+lSkxwis6/jhs23tDq21kqLRhLjdjtdoZFD+OF8JMMPbGEcP8csLq3rtg5h0IHE5ZB15vqnHLYaTqEP6lcyvlfnM+wO8WGvkP4KyuDykr3ddGkN6HP0qMwK7jeXwbrzgdj3WjOwkIxl5TLqwddL83DZrORnZ3d9Pii0DknGxERYMeG2WYWEUhVSCQSNr60ke3vbXdnl6ksqaS4tNj5+s4h9+GX+m+3tumlGWjjoPvtKAOFNYpFVkbfLx/kyt8bLrbuDqKHRZM7zo8S/xIWm7rB9B3gE+3RPngROMeXY+/D0u5QepQQnxB0kggU1gBnprY7GfSvQfS/un+NjD8txdbuMPRdCB/n/g54aR6x80Q9JKBHSA8u73M5M7vNdC66a2b85ezP4eupX3Po58ZrujRF2ro0dn28C6NZzB+NpgjMvV8RUZ1eOjz1zl8kEgjoD7puzkMZpRk8s+YZXlj/AhIJRJzZSfyBZZQWtKzOX9buLHZ9usuZBVBUKVb3gZpAsqriW3pGH4by1DZ9Li/th1wOcYXXM/bQHu4f7Np55Ws7/o+jUU+Df3r1weXDYdU0l7ZzTiKRgj4dKusqTyEhoDKJmm0ZpXWFv/wj+Sy8cCGHfzncoiabvT6qwjEeKCyBSCQgTboapm0+J2oJO9b9npiztZmzsslqCn8yhQzsIJF6TniLGhJF5YRK7FI7gerOmanz5uuvEx8Tw4YXXuDeI0f4IDOTpzKOEHniBT77OIY333ijeTeyVsC6uXD4Ffd1dtgHkHhVvadi5cLuc1/erkZvETM8hp7zeqLUNm4ntOfLPbyoe5FTq0+1uJt27PS+vjeRo+IxycUfVpiuKuMvbwPsfRz0Lb9vhyL5Hpj4l8jEhY4pWmWvEuVsrNUbmtF+VcJfWUa9wYdlWWUc/PEglkqL27oVPjiGk/2mkhEtvgPhah2kftf5xeDOjKkIDr/hrFFas8YfUKvOX01iRsQQPazt+1AFxwrY+9VeKorFPEcuUeCrrOsu0h6akVf4czHuLkrtFuw2MBWhUohoGKPCF7V/6+0NTvx5gpf8X2L/N/tRypQo9YnoKntwqTIaNl7mql7/4znb6hNgwLUDiBsjQrp8fKq9i90Z/Rc3Oo4hG4awd8BeVtrCYcSXoA6vc51jIRIc3CmD6NoVu91OdnZ20+OLqQTyt0BlLuHhoLKIVWBN4Q/git+u4LJf3fu3+NWkr/hu8HcAyKxa3t7yOeaez7q1TS/Nx1GfyyatpFTfSAV1NxE3Oo5TE+SU+pfiI3d/VJ6XhnGMLyAX1kY2CzO7zeSDpCwGn/yBggLY/sF23k58m/yj7t1RMliqrT47ki++l7r0C+/Hdxd9x6NjHnXW9cut8aiRyqWkb0ynNL0Ui7H1C+/pb0/ntszbqFRXWbRZfb220Z2IBucv4xbDwJedL/MMeTy79lne2fYOAIYBozgy8joqTS1bqh5fdpylNy2lNEPsABVVVAl/6kBnxt90zVyxsemlc2AqgtTvoVhkIkgk1XVTzC3ThZtkYcp7HI/6P2TaGuHg2nhhUeelaeYeh1Ff1TkcGAhqsxD+TubWdZiw2+wcXXyUgmMts8Rq7vqoLKuMLW9vITQnlAcGP01MwXUolefWerTTZPyBqBecWZ3RV1P4A7ht/21Mf3O6R7vkeFbcYDsA+5/zaNtt5c3XX+fFJ55gndXKn5WVzAcmAfOB1bZK1tusvPj4480T/yQyGPEFdLnevZ1ugK4+IuPvcEnz7HfNFY0/BIK6BtHvqn4EJbU8q1cTpKHr3V1JmN0Lk/ysjL+E+TBtO1Q56HRaAgdAxGQRtJ/+K1Rkt3eP6jLkv3BJSbU4CfVm/NUMPtz92W5+uvQnsna7z9HINz6YM10jKPUvRS1Xo6vMhE1XQuo3bmvTSxOYy2H3A3BmCQBdg7py/4j7uXPonUDDGX8AxlIjx34/1qbme17Yk8fKH6MgSYh9/ooQJPVMNNpDM/IKf17g2LvwUxCfbJvIsoE+nDJ9T/bebOy21n0hfSN9GXDdAEJ7CdGhpATkUjMZmtsgcoYre/7P4PDrsL+uYJIYkMhDox7inuH3NPhWR/Rfbm6Dl7iEaL9oHhr1EEP73Axdrqm3GLa3vp8HyFkNK0ZC1goiIkBpFrY6efraq8DIgZH4Rfu5tSvdZ3cnao5Y5MttvuzMmok0cqJb2/TSDEwl8Htv/I9WR3LmHk3jyOIjHu+K0SpEnunqXDjypsfb91Ibe9KNMHMfBIiaCA4xJz8f5Go5miANVpPrReLSjFI+6PcB297dVm31adUyOOQHWH8xlHfyaNpziX1PC4tGU3Gtw9FVQZIZGdXHQnqE8Fj5Yyh8FLzX4z0M+eLv3WZpWZSjXCXH6mvFLrWDXUq3gIOot8yErBVt+CBeOhqOmpGlxqo6bnERlAfHU2FqmR1Ov6v6Mf+P+QR3EwOYI8MnQB3gFP6Kwu4RNsJeOgeGM7DpCkhf5Dykqtr/a2kNyMaw2CwUm8V8OVxbw0J0zA8w4jPXNfQPRC6HAEksKlMUpeV1nwEhPUJ40vikS+vC1iT3QC7L712O4pCCe/o/Q0LebSKA5NDLwkbuHKDT1PgD2PMIbJrvfHm28OdpPh7yMXHPxfH9Rd8zyn4asv5sl360huzsbB595BGWGo00VFBnOLDUaOTRhx+uCvRrBJkKulwL4eNd3NMa6E8LW+Ajb9c51SNgAAAZlYex2hpfc3w26jM+HfZpo9dED41GrpGTti6t1d01GM1Y5EJVCtFWLY6U/sJ+WuHe/RSPYLdDzipYP69j1vhTBYvfcw0BxSn8ldUv/PW6pBfnf3E+AfEBbuuWyQRGhcgkC9eGI/FNgNELIbZ+G3svHkATAVM2Qs8HAbFH/fq017lr+F1Awxl/AD9f8TML5y2k8ETbnkNKrZIyiWggQNlx6lt5hT8vwm4n6UaK7RJssgrit5zhowEfYSxt3WoqZngM539+PpGDIll8ZDHLTI+RrV1PTuw70O2Wpm/gpTanf4SUupOaSF0kr0x5hbuH3+089t3c7/h48MfO155YBOQeyMX4h5FnBj7Dw6MfbvC6gqogTsdmshc3ENgfBrwCgQOF8NdAxp/dZqckvYTKEvd5wI5/ZjwJjyQAILOKqBdvBk8HQKYGuw25VIlcIv5BSn7bwsILFmI2uDh0vhE2vrKR3l9XoDQqmalKgf3PeKxtL83DET2enw8Drx/IzTtvJrxv3WzutmI2mLFUWrCaxQJfigyZzYdI7WFI/xlsravN4cUNyDSgCgWbGCssNguZZZmERFYAcOZM9aUSiQSJRILCR4FULqU8u5wDCw/wXq/3KM+pWwe4IfKP5iNJk3Dohmwm7j9BkCYPad5aqHRzRJMX95K7DnbeCwahxvmrxM6NwWzAbDXj4wPYbZQVtey5FBAfQNfpXVH6KjFbzZSbxHfNTxmIY79T1fdO73qkM6FNhDE/QvzlzkMWTSar+yQzYmG8y5rJ1edix47ELiPMt+Ns1nQqivbA8Q/BUrc450zeZ8q+DCYH1i1fIZFKkMrdty0VPSya69ZdR/c53Z1isVIJHH4VTn7ptnY9iWPNX1gIVs8bebSMng/D0A+cL6N10bw8+WVemSyCEo8vO87mNzd7rDvB3YKJSY7hsj6XoZyXCRM6j/D36SefMEGhaFD0czAcGK9U8tmnjQtlHkEVAoW7wVh3HpcQHM2Io6t5RHsMqaTxMSFmZAzRI6Ibz56RwK5PdpG6NrXF3Uz5M4XtD20nf38BYw7tYHz68morWEMGWCpafM8OR85qWKiG0qMi0zN0VHv3qC7mcvFsqax2nYnSVVlHNyD8hSSHMODaAeii3Fewd/ltvzLij7VghzBtGKiCIP5SZwCtl3ZAqhDf4QbKxzSW8Tf2qbHMfG8mgV1ab/ecdyiPrF1ZlFUFkQWqOs7Gt1f4czH1pXJ2eMLHwfBPKVCKATQ9Hsb/33gksrZ/luUpy9mpeYkC3TrnoOylhYz9RWRhNAPfCF/8Yqsjjzwh/B37/Ri/XvMrxanFsGYW/DG43uu8wl/rkUgkBAUFNT2++CZCr4cgoHeV1afI+Ms11J5YH/7lMG/FvcXRJUfd1WUA54ZbslzBwov9kB19uYl3eHE7MhXMPgz9n+OlXisZc2g7tr7DmPf1PI/W0yhJL0GXBTapja+ZDpNWeaxtL7Vxji+GdDj6DhQfQG/Sc9v281jbqx85+e4V3oK7BXPXsbsYed9Ifr38V/4cZSG6cD4bSp6GK6yg6+7W9r20gN6PwvTtoBaTi4EfDST6jWjOSDYBkJ1d13qv7/y+3LrvVsL6hGG32jGVmShNLz37zg3y+62/89X4rwiQh+NjSuRA4Qy4TN9gfRgvHYsG5y9Fe+Do21Am5iH+6upFQomxBI0hn0HLnuf4Z+tb1J4jgMBxHwem0gCsVrHZH+LVdDoXCl9R09Mv2XnIR65Frz5Gpv40lRbXBLFllQlLMKU5HD9d1RaJuRT2PQXZf7ukjXOe0z/D9ttAXze7JjhIjAEFDbh5Zu3K4ujSlq1Lmrs+UvuriT8vnnyffA7nH8YsLRXC3/SdMOJ/LWqzoxIQADIZ2Gz1b2p2KKKmQ3x1yQl/tT8Pj36YGwfdCMCez/ew4v4V2KyeqYN00XcXMefjOeKFTAkK94kFrmbxt99ydWXzxsBrKir49dtvG78oexX8Gi/+lt2F3AcuLoD+z9c5FRwoI6RsPNKSxCb/rqe9Po25n8xt9Lplty9j1MOjmP3h7BZ3s+B4AZl/Z2IutxJgGEyceSoyaZULwd9jYcWIFt+zw6GOhIipEDJcZHp2RFvrnJXwx0DIXOY8NCp2FI+NeYwr+lzh3GOuL4vLnci1Kio1NpBAuG84NJGh6sVD2My1nGmyy7PZlbWL4sriRjP+YobHMPimwW3aD1vz9Bo+HvIxepsQqQPV9S842kMz8gp/LkYq7by/Un+1mOSkxRsZ++Q4VLrWFXf94+4/+OuRv4Bqy4a+agOxqRdDzhqX9PUfhSYSlPVHHpwpPcOurF2UGcsAmPPxHC7/tToi1hPCX6+LezHtu2noQ/WYVeGiFkY9FFZlTQe13GL9H49UKiUuLq5F40tNq8/c8trCX8SACIbeOZSgru75x7Db7fx8xc/of9RzRY8bCCqdyImi4aCJckt7XlrHyKjzCDAMoTK0G/2u6odcLfdY2zPfmclfd0lR2KKxabpDUP0BA17cj3N80afAzrshbz0quYoduRso89lPkaGUkuwKNr+5uVURsy3FZAIJErEpJ5GeW0V4zjEifIUdXhmZ+PgIt6Css8ppSCQSFBqRXdz3yr7ceexOooY0/1kw8F8DOe+J85xZGqrWTU29tBMNzl8SroLzUyH0PADkUjm+SuEOUFxZjDZcR0lYN6ShLYsW+3L8l7ydKOzDHDWbdEodeTni+ZYUU4BkxQgR5OCl06JV+CGxiX/TAkPL6sI1RHa5SAlVmyPQOfb9K3LgwHNe4a+5JF4FE1aAT2ydUw7BvSHhb+XjK/np0p9aVPumuesjk96EzWrjyVVPMnd5L9JDPxVzDG086Lo2u72OjFRaHVzbKew+G2HcM+P417Z/ebTNX4/8yi8HF1KcvkzYC3cSSkpLiWj6MgDCgZKaaVH1IZGK4C65tq1da7qdenBk5LhCxLGarRxdepTiU8VIZS3fox1570j+bfk3fr2EGFarvnT8laLOX2fHvweMXwpRs9q7Jw0T0Bf6PSecraoYGj2UFya9wIU9L3QKf6U1YgrLssp4K+EtVj3pvsDiwU/O4NTQh7g4LZP3Z74PKyfAsv5Nv9GLe1nWF/6qtg2f+91cBn88mLWpa53CX2PBMYZ8AyseWtGqsmf9r+vPlFenYMSI1KYkWFO/8NcemlHnVak6KDabZyKTXEplLmy6mslSMcmxSMswtSHA//jvx0nfkA5Avl6oPXFKM+q8n6HCfQVWz1lMJVC0F6x1o7kmfzWZwR8PZlfWrnrfGiZ0H7cuAIKSgvjW/1u6f9GdZwyRMHZRvdc5hD9vxl/LsdlsnD59uunxxW6H5SNh+50EB4PWlEhQ2RiSA2pPQoKSgpj5zkxiR9ZdmLsCS4WFA98fgEPw8qjP0Jx8j5e2/QWJV7ulPS8tJPU7OPkFvmKPlfLmu+65lAElTzFpfyq3JN8EVhcW6vHSIpzjS8BAmLQGYubV2oQ3y4rJSTey4v4VHF3s+izh3AO57PxkJ2WZIoDFMf+I026F3JZl+3hxM8UH4fBrUH4SqLbaySrLJKYqSPhME/tlLQ0q6ze/H75X+vLkpjs5FfYOcQFHIWOZyMTx0uFpcP6iChKb7tLqgBOHhVZxZTHaIBUpQy9HObRlmyixY2JJmpYEgEah4dbBt3J1v6ud9f1io8rBkA5G14hFXjzEbz1g5WTnS5VSgsIqgtcKKlzzb5lVLtaoKnNktfCnjYdZB6H7XS5p45zHLxkip9SbMWXVnWJT8jieTjuv3reOfGAkF3x1QYs225q7Pvrt5t/4j/I/FJcWA6CwBOGrLheZidZzx07cEfCbn9/4de1OyufwSxTkb3MeOpB7gL9P/k2BoYCw3mFED41ulVjTUoxlRlY+sZLX//s69/x6OQHrZ3WquuP+fn40UbXPSQ7g35QFV/h4mL5DZGW6k4oc8XvO31LrcGAgFPiuZY30SZYeXdroLUx6E7/f8Tvb3t1W73mZQsZjZY9x3hPnkbkzs8VdtNlsnMk4w6GCA5yIeIUM7W/VJ/s/B70aLnHT6fi9l3Du6oj4doE+T9YS/mpSX8afSqfCJ8QHdYDabd0ym0GKnABZJLH+sRDQD4IGua09L80k9hKInut86QhSzdHnOJ+R2dli27Q+Vj+1ms2vbUafW9eyvCm6z+rOqAdGkVzwIDN2VfJ/I/9b73XtoRl5hT8X05IotQ6D3Q6pC+gmEQun0MxKvp7weauL4N514i6uWi4smPKqhL+teVOwX2aBuEtc0+d/Ekf/C38MgJLDdU4F+wgVLd8gZvgnV55kzTNrMJaJTXRPFfouriwGIEAd0OA13oy/1mO32yksLGx6fJFIwKoHWyUyGSTIRjHq6Hoe6veGZzpahcJHwb/N/2b2h7Odtm+1ouS8tC+HX4ED/2Fj4SJORLxC6fFVvBn3ptutX2uS8lcK0pMnAOh6uDusmuKxtr3Uxjm+KPyF9bdGTJAdNbfMshIq5Tqu33A9ox50fe2HkytP8tvNv1GYUsj8RfN58eQ8DMpTjPV7BNZf6PL2vLSBwp2w+yEhAAJRvkL4yyzLJLqqnEJTwp/dZueL8V+w7K5ljV9Yg6MFR/nm+HtkBf7AqOhvYe0s0Ke36iN48SwNzl/sdtCfdorIUD2HLKooQluVbGAwtKy9KS9PcVp6xfjF8MHsD3hv1ntO4U8bGg/zMqDfM634NF7ajcBBENDH+VKpBKWl9hqorTgy/lTmCGdgFDIl+PdqsF6Ml3qw2+sV00KDlRTq1pFq2YzNXnfTK2lKEr0v6d0isae566PoEdH0ubwPhVaxGFVYA+kVtAoWJ0Dad81ur6PjyKrs8Bl/Sn/QJtRydLhq0VVM+XoKO7N2AqL+s9Xkfus8fY6eDS9sIOBQAGU2yO7+EETPcXu7ruL8K6/kK3XzxI2vNBouuPJKN/eomZgKYdf9cPqnWoeDgiDfbyUHgp5n6dHfG72FQqNg31f7OPHniQavkUgkLL1pKQvnLWxxF89sOcPxlcfZmbuJIzGPcET7cYvv0SnY9zSUHgHfzpX9nFacxsbTG1FqxUSxZjKr0lfJzTtudsu6FcRaZu9/1+KXewKFourg0HdhxOduac9LC+j/HAx4wfkyXBsOQE55DlFR4rFTXl47Q7QmQ24dwtV/XY3Kv/UWM2azcC/yUSnqPd8empFX+PMC6jC4rIL1oRcDILUZKTxRQGVJ62omSCQSlL5il7/QICbYgeogJDJZrcheL80kYiL0fUYUQj6LEB9xzBHtmrIihbXPrsWQJx6ADuGvoEB4/ruDRfMXkTA/AYlNwsTydZBeN+PPbq+2dvEKf25m5j4YLgp3N+Zjvfa5tXw50X0F7aVyKWaFmSJ9OckhG7g0+QkoP+W29ry0gOGfwpgf+e7EexyJeYRcxSl8I3yRKjw3JVj56EqCt4gFnSHsyk61yD6nsduc2ZeOmlsWWTGFJTLiRse5pUh6r4t6cdXyqwjvG84fx/9ge/mvWKWV7DM/BANfc3l7XtpA1HSYuhnCRMaGI+Mvs7z5GX8SqQRDvgFTafOyLL6Z+Q1pz4pANJnNl4OlF8LwzzpmHRIvLcAOS7vBjupMqm8u/IaDtx9kdNxofHwgOH0P+R/+7JI6Tw7hL8rrON45Gf0tDH7L+bKm8Ocqq8/7R97P/MJjdMv6d3XGn0UPlXlgs7ikjXOeihz4wQd2P1DnVNeIcLBLsUus5JTneLRbw+8azoXfXOgsQaKwBFFmj4fk+yBwoEf74k46TcZf7IUwdRMED3UeCtKIDYLCikL2frWXF7QvcGJ5w4KOq/CP8+e2w7exbug6im1gT75PZL11EqwWCysrK9naxHVbgTUmEzf+qwkL1ey/hbODqQlL0LbilwwT/xKZXDXw9wdfqygbcyK/8SQEiVTC3Sl3c8XSK+o9X3CsgJS/Uhhw/QDOe7z+TOPG+PuRv9l2/zYKK8UzRiupsq3KWQ1r50J+U7/1TkLqtyJbbcjb7d2Thlk1FbbfXuvQ8E+HM+bzMeTaROByaWnDWVyupqKogsPvr0FS+g2b/e/jcF7dBA0vHYNw3yrhT5+DUln9nGxovRreL5wuk7s4y1Q0F6vZyrs93mX1U6ud7kUdKfHBK/x5EbK3TE1iYBdCDGMoiIrmqj0Pkjwnuen3nkXB8QJO/HkCY6nYOCwyiplnv6AiyF13TtlpeIzQ0dD3adDWtWUM1tSOdh1+13Bu238bfjF+gBB+pFKwWNxX8DakZwhF3YpQyOwMLvit3mLQBkO1fZtX+PMcDp/8wkJ7nciSkrQSCo4VOLNDXYkh30Da+jRe/v1lhizSEdD1YWbGvwAVzTUj8eJWggZD0ED8fcTOVna4havX3US3Gd081oXJL09m3YRjrO85jBV+50OvhzzWtpcGMBXBdzLYcSdQnX1jlpWQnw82i42KwgqXN+sX40fS1CTUAWoMZhG0IrdpyZbOEoXmvXQc1GEQMgKUAQBE6iIBkfHXXOEP4PYDt3PBlxc0q8myjDJM+WICIbf6km/pD0k3iIwBL50XiRR6PwHx1XWp+4b3pVdoL3wUPmi1oC3OxLz7QLPtdkx6E0v+tYRDPx0CoNRYSoGhAKvN6hT+EkJOwqlvwJDh8o/kxXMolaBwCH8usvr0UfggK+6Gjym+WvhL+wEWhUHWcpe0cc6jCobwiSJL8izCQ+VoTOJBkVqcWuf8ma1neD3ydXZ8tMNt3SuqFIV9FJZAiuz9YfAbENjPbe15Gk85/biDmsJfcHIw/a7qh2+4bxPvajsypQxlgpJSX5H+EagJdHubruLN11/nvZdf5k5gDjQo/m0F5qhUvPTKK0RENFERMP1X4exQT4kZlyKRQsRk53zSeVgCkRoh/J0uPt3kbbRhWiQN1ALf/+1+FkxdQOyoWIbcOqTFXRz14Ch639eboiprcK20Kgi/PBWy/gRrCy0JOiqT18KUje3di8apzAVjYa1D0X4iE7/EJuZzdjuUlVWf3/vVXra8VdtK1lWo/dWM+vpW1oxbyh71W1Rk/gU77oZSz7kneWmA1O9hw2VOS3+H1afD1SG2aks9o5FlgN1ub3HGuanMhEwhw2axsT36erYnXcCJ0gMt77+b8Ap/LqahB0+Hp3Anl0R2YXrmerplPYmxlVrAwR8O8s2Mb8g/mo/BbEBvEaPvBTE/wN/jwFLWxB28tARnxl9VtKtfjB9hfcKQKWUAyGTuL/Q99smxrL11LWY7bOz3KfR/vs41DptPX9+OFfnQWZBIJERERDRvfMlZC0feBrudwEBY07s30zYoOZJ/pNZlcz6ew/1n7m9xvaXmkLY+jS/GfoFxixhIfswaygv7jzToze7Fw9jtYDHgrxJeahZZucfr/CVO6sKBntsp0W7HZG+5h7oX1+EcXxQ6YccdNBiotvq0VAl/nwz7hI+HuN7mxmYR2TxWmxVjVbahzOZTbZ3ipeNgt4vNoKoNIWfGXw3hLyPDtRG3t+69FemzYrkis/l65xCdjEbnL32farD2r1YLGT0mYnr4CXSRzcs01ufo2f3ZbjK2idX8u9veJeTVEG5cfBO5ueKaaMU62HwVFO1u1efx0k6c/hl23u/MvFMqwbcymUTNAHRK12WiO+ZCTqtPv+7Q7TbwTXJZG+c0UjmM/138zs4iOBg0pgQAjuTUdQDxCfEhICGgReuS5qyPrCYrP132E3u+2ENRhRD+lNYgfHya3UynodMIf6ZiOPgiZP7pPOQQ/goMBcQMj2He1/OIHuZ+i11TuYmMUxlIrVLm6BSo/xoF2avc3m5byc7O5tFHHmGp0chbwGPAWGAa8A3wN7AAmAyMBu549FHuu//+pm/c+zHh7KAKdlfXq7FUCBHNUjuoMM5PCH8Z+rQmLfEM+QZS/kqpN0Cox7wezHx/JkFJrYs67z6nO8NuHkZxlYCgk1X9TpKuh8sqIXRsq+7b4Sg7BgeeE/8WHZWZe2DM97UORevE+JCtz3A+s2vafe76ZBcbXtzglu5I5VJUceHkhlbVPDefgWPvCIcAL+1LyUE4/YMQi6lh9akXTgNNlaYoTi3mBZ8XWP3U6hY1qwnScNv+2zjvmUnk+/1NTuBiLNQfMN0empFX+HMxUmkn/ZWuuxC23YZSCVKrmROLD5KxveXRsD0u6MGsD2cR0iMEtVzNhz1SGX14K6es18Pgt0ER4Pq+n+tYKmDlRNj3VJ1Tzoy/CpHxZ7fZMRQYMOmrMysdiwDHpoc7KK4sxg4oA/uBb0Kd816bz7YhlUqJiIho3vhy6ivYdS+YSwkIALvEghULufraXwCJ1H0PnPC+4Ux7axpliULorzQFU2xNBvk5uMrujGy/HX7QEqEQO+hWSRmbX9vI4UWes6mwWsEqNRAohXE5r4sMDC/tgnN8katgzA/Q7VZAWGMEKsRkOT8fel7Yk54X9nR5+z9e8iMvaF9Ab6petMtsGubJusCOe1zenpc2ULwfFmrg0KsAJAQkcEWfK7iizxVERopIbYMBiooav03ugVy2vrMVfV7zRP9yk9iNl1t9uTT2GvitR5s+hhfP0ZL5y9rUtTyz5hl+O/YbPj5gVajRG5tfIiAgMYCHCx92Wno5NvnV9kCs1iqxKGkSnPczBLU8+t9LO5K9Ao6+CWaxq6dUQs+Ml3k1aTfz+813SRMPr3iM/SFPUSnPrhb+QkfD0PfB3zvmtBW1GvysiQAczqor/AUlBXHj5hvpe2XfZt+zOeOLPk/PwR8OkrYjDatdRPArLIFMCn4ENlzawk/RsXHU+OvwVp/WStj7OGQsdR6qmfHnSQ79fIifev1Et+PdCFH6ClESz9dfaimffvIJExQKhle9vg9IQ4h/bwG3A28DE4AxCilyeTOfpT7RwtnBE+V5TnwMSxKhcHutw0mhIiWn0qZv8vtwYvkJFkxdQNq6uragEf0jGHrbUA7+cJAP+39IwbGWZYc7xpcSk+iDTlaj7I5EClJZi+7XYclZDYdfEXUXOxEO4S+jLAP/KhOQmsLfnE/mcP36693StklvojS/BLOsKou8x/1wwRkI9s4t250+T8DlJvAXexZOq88qi/GmHGq0YVoSJiQQ1K11G9dGI5jk4iEc4V+3VBe0j2bUSVWqjovV6v4ixG6h7zPQ80FUKpBajGy6T0TGtZSw3mEMuWUIKp0KqUSKqiKeQP0wyjXjIPnuc+cB6UlkalFwt7Kucnd2xt/RpUd5NeTVWhv47o7+W/vcWhJWJ6CUQLDEUq+dq1f4axtWq5WUlJTmjS89H4BJa0DuQ2AgqMxhAHWEP3OFmYM/HiRtfeP++a0hqGsQI+4ZQUmomH1FyG0EarI9Z7zupXFCRkDiNWiV1Rl/u15dyb4F+zzWhbdiX2f6b4MIkkFU8Roo2e+xtr3UpqHx5fPzP2f5lGxiC66noEBkd099barL248aGkXy3GQqakT8KpFhkoSAXOPy9ry0AXUYJFwFAb0BkfH37UXf8sKkF1AowOEg1Zh9CsCpVaf48+4/yTvY+MTEVG5i12e7MBwRdkoymxYTQaBuwqrKS4eh0flL6rewfDiUiTpOa1LX8OzaZ/n92O/4+IDEasGcmtHszTqJRIImUIM6QA1U2/pJTMK6LTISJL6xor6Uxvsd6lT0fRbOT3MGkDoyf00uqiBht9t5d/vbHI96DrvcgMb76Gk9p76GzdeJmsFnEapIAOBEfqpLmmrO+sgv2o8njU8y5tkxPD3uacYp70Nm1xCm3A8F21zSj46CQ/grKXHd34ZbUIXCtO3Qpzqo2Sn8VRZiNphZ8q8lbrV9dRDcPZjIqyMpCC5gqzQSzj8JEZPc3m5bWfztt1xdWduOMwJ4AtgOHKv67xPATWYbv377bfNubCp21vp2O8FDocf9oA6vdTgqTI3KLI6llTS+TxE7KpaZ788kclBkg9dYzVbMBjNmg7nZXTOWGXkj9g0W3bWIUnOV8KeosoHN/BPyNjX7Xh2e8AmQeA0END/owuMUbIfjH9T6bjqsPjNKq4W/mqWNQnqEENzdPZmrB747wM6L3qLria5IkROoDROiuUztlva8tACZGqTVtkFdg7rywMgHuGuYqCnelPCn8FEwf9l8Bt04qEXN5uzLYdt720g9nINNKsbmCF0N4a/GnKg9NCMPhHJ46RQkXc/+nP18GRqJXBvI9RN+ZMis+hXqxrDb7bVSVx1RF35+ruroPxCJBOZl1ntqUOQgHh71MH3DxYM6uHswg24eRGCXan96dwt/O97fwfjg8cy5PoAua0aJzM7ku2td47D6DPaAa8S5SllZM21y/XtB1eQnIACUFvEFyDPU/gLYLDZ+uvQn+s7vS/x58S7saTWOLI1nuv/K2NDngY68Cv0H0eVa6HItsrX/Bwjhr/vL9zH9MvfX0gDxnAhKDqXYXEyKGfaN30W/8Lr1YLx4Duf4su9pwA79xHfDMWa7M3rckZ1zqkhkACjwwWpTs0m3jdkD3Neul1agiYBRXzd4OiYGsrLEYqpvI/sHPS7oQXi/cCIGNi6+lJwuYem/lsKlQC+Q23zZZn2JoZNb2X8v7UKD8xdzGRjOiLotuuq6osXGYnx8QG4yEPbHp2yNHcrMd2Y22U55djn6XD1B3YJQaBRO4c9uEHPiqCiXfBwv7cFZQq2rhb9SY6kz+CREHYFzKXvkTcjfCiO/BJnrrfHPSfI2wakvYeArIlikBtGaJPaZIrGb63cA2fPlHopTixn/9PhmN9ec9ZFMKSM4OJhnxj/D/60Tgsi+oGVETj23AhJ9fUVmZWWlmLd12DFPKquTGVMz40+mlLH7s90YS40MucW9GTSxI2M5v+/5+Kf6o5J3nr/xktJSmhu+Eg6U1EyFaoy/x4G1AuYca23Xmk/oKPFz9uFQ0BjjMSpySCtOY1BkwxvwgYmBDL1taJ3jdpud1yJeo/elvZn57swWf4/MBjO6CB1WqZUySzEAOnnV/tq2W4QV6oxdLbpnhyV8vPjpyKR+J7L+o2aCVuxb1cz4S6ra+yotrX6L1WyloqACTZDGWQbJVQR1DUI6NpbCoEJ8JaFISw6DXFuv85kXD2MuFQ412njwiSFKF8VrU19znnZYfWZng9mMy0qLnFx5khX3r6Dfp2K9IrUr8FXW2FvbeAXkb4a5Ka5psIV4hT8vThQyBQZpNgp1JSHjehPewlrXxjIjr0e8zrC7hzH5xcmsPrWaH4v/xOQ/him8BBvjYXQzo428NIuBkQMZGDnQ+Tq0ZyhzPppT6xp3C38377oZu9WOny4TjsVC4IA613iFPw9TFVESGChF2UDGn0qn4uIfLia0V6jLm1/15CqO/36cilvEJsr63KFoVFMZ0VlroJ6jOCYjFlkZJb7R+Md6pl2JRMKM767h5g/uAECr9vNuqHUU0heBzeQU/hzR48XFcHjJcQ5+u5eJz09sdb2MxqiwVCBBghKRieqt5dY5sNiElbSv0peYGD+2b284itKBf5w//nH+Td7bL9aPK5ddiTRCSuDm21i3xx+d68p5eWlvut0ifqpwCH9FFUVotWBW68jsNYmeF8U063b7vtnHXw/+xQ2bbiB2ZKzT6tNcWkP423YbnF4Ic0+BsunvoJcOgqUCKrNEppBCh1IJBb7reDj9X3y3oAt/XvVn0/dohOzybADkFj8CtDVEqYLtkP6jd/3aEvo/L0Q/Rd3BekLw1Vj+uprLGggMOfTDIU6uPMm4p8bVCiTOO5yH1WglYkDLM3WLU4spyyojvF84Sq3SuTHs7w+cY+sSiUTM286c6eDCH4BFD+Zy0IjMrhExI3h58sv0DOmJVC7lkaJHUPl5Zm0Q7hvOZX0ug9z1kPI/iL+8w5en8PfzIzuz/sDws8kB/P2b+byLntv6TrmIsDDon/o5MeEaZndv3vP/bEzlJsL7hqMN07bq/b7hvtyw5Qb279/PbTsWsHRVAYljBoiTQ94958aODk/Xf0H0TDEHqMKZ8VdWf8bfptc2serxVdy0/Saihrh2MEwYn0D+JZEUFhQSJx8AGy8frLcXAAEAAElEQVQHm9EzgrmXxincDSvH15uMAhAYCD4+ojRFdjbE1rP/tfervaSuTmXu/+Y2ux5fn8v7ENE/grWleXAGNPaQ2u/VdQWLoVY2oifxWn16Eex5lKSNs5AiNoErK1seAWcsNRI3Jg6/aJHetzZtLRt5hRz/35HK1d6N3bZQuLMqvb3loa1hVcGW7hL+dJE6/GL8RP2Loe9BWN1Cxw7hz2v16QGOfwTfySB3LQEBoLLUL/wB9L6kN2G9w+ocbys2qw1jqZEyiYjCXXp6Dhv0r7i8HS+tJH8rbLmBK6K683KvlXTPfJrsMxbKc8o91oWSEjtWqZ5gKQSUHYDKjl6Q5B/ClHUwYzcAy44vY/bPYzkS/wAAmQeLOLjwIKXppY3docUsu2sZOz/eSa/QXlifsnKX5TQhPul0Mb4KRXtd2pYXF7DtNmeNP4A5380h+o1oFh1e5IyibMrqE0Q0trGscTsplU5FtxndSBqYhErfFZUllIGaN0Xkr5dzjkCNEOiKK0XGHxIJmV3GED0qoVnvjx0Vy7hnxjldLxwZf8aSaqtPdEkiQE3umQx3Ly4iYwksSYIsIfAplSBBSq71OClFbY+ezirPAkBljqwdXDD6W7jMKOo5eWkeqqB6RT+oDgAtaMC9d8a7M7j7RN2Nuvd7vc9HAz9qVXf2LdjH/0b9j5Q9KRzOO0xOWQEqmZ4Y20IoPdqqe3ZkHAFb7lr3u4y/xsKK4c6XfcL68PDoh5mTLAKY1QFqt9ajd7DlrS38cNEPwgYydQFsvVHUIOzgnH/llXytbp6t4FdKKRdceWXzbtz/OfHjCYwFsPZ8OPpurcOhoaCr7EVldiLyZmySL7xwIR8P+bjWMZWfimtWXsO4p8ZRUVjBrs92kbG9GZPTeoiQ9Ce0bDKB6qqNrJg5ED27Vffy0kr8e0HE5FqCfK/QXjw+5nEeGPlAvTX+oodFM+T2IU77d1eTZxA14wJVYdD9Tki+1y3teGkhfskw6I1ae9LlpnK2ntnKkfwjSCTVWX/p6fXf4vTG0+z5Yg8VBRX1X1APukgdiRMTya6qRa2VnuWe2P95GL+0nnd6Bu8s1sU0VxHucNjtSGVq1BKwS6zsvONTPhn2SYtu4Rftx1XLr2LYncOA6uhJlSWCzJ4rYcTnLu/2P4bUb2H77VBRO7LLbreTVpzGzsydWG1W7HY7i65axLr/rHNe486MP5vVRsahDI6lHqPU2PBmsLfGX9uQSCTExsY2b3zRJUHsxaAMJDAQZ8ZfTnld4Q/AarJis9Stw9EWJr84mbtT7mZSt0kM1V6MxhTntfvtSBjS4eTnRNrLmJ48EV9jd4xf/8DbCW97pPmKwgq2v/I3yceGMkXtS/DGCyD7L4+07aUutcYXZaBzUVVUUcT60+sx6PYAEDp9EE+aniRhfILL2rbb7Gx/bzspK1KcfcGiJtbvEF3LHj7n6u+cE6T/CJm/O19G+oq6Kpllmc66CQ0tpBzY7XZeCX6FHy/5sfHrbNVBaA43tx6WZyHls5b320u70Oj8xWaF1O8hczlQI+OvskjU+Kt6i8HQvLZiR8Yy/unx+IYLUc+R8VdZJIS/0FCg54MwaZW35nhnw78P9HwIfLsCQvhTWISK5Khz3hZqrlnrZBVLvQZJLcJaCUV7QF/3QeAQpRoS/gITA/GL8aszXsx4ZwYXfHlBneubsz5KmprElFensM60jl7v92KV7nrCfU8Rk3q5WF+fY7jb6cdlxF8G8Q2LUYUnCjm98bTbu5H3/+ydd3hT1f/HX1lNmrbp3oUuNhTKXspGEEFUFBS34p440J/o1404cG/cAxeoiAgCCrL33hRK994rzfz9cZq0IelOoeh9PY8P9tx7z7m3Sc895zPen8N5HPnlCEuTlpIcehmMXA6q9p8NPuu221hrNLKtkfO2AevMcOusWWfjtpqHXA2Zf0CZowPeNk9UV0N5E2JStUFavEO9sVpdJy5U5FawbNYyji1tuqP/5KqTbH19K0FeQRhrSgOqVEA9Y0icBaxWsJjsP0bponhx7IvclHiTS8df3Ng4LnnvEgI6ud8Aue6ZdXT+wY9x+zKZ0+UzoV7R5W63jyPRAjzDoNtsBxW6Z9Y9w5BPh/D+jveBxuv8jXtpHP9X9n9og5qW+W21Wsk5kIPVaqWgrBy5RY2Pon6pu3PhM5Icf25GLj9Pf6V9X4ZJ+6mseZcpIn0JSWhdJlBOhYiC0BjDJKN/a4m7BUavAo2jLKPZaib2rVgGLBxAQVUBMpmME3+cIG1T7WbLtgEoKxOa/+6kIreCT3p+woMzHuTpr3rChqug/JTTeVLGX+uQy+UEBgY2bX4JGwcX/gT+ieh04GPsREDZhcT5ONdQ2/bONl70fJGs3VltcNfw/JjnuUHzEy/3f5ahPs+2yRgSLSByClxVAtEzCKtRTcrz60zfW/tiMbvXCeyK8pxyTn2+mZE75zOoajf0ewMC+rf5uBKucZhfKtKEtJnVajfCWzzELqq4TIlC5WZjuQweK3yMyR/VRs6Wl8PR/KEcjd4qvqsS7YspJ2HMX/YfI3yEfE5dx19eXsO1t2QyGT1n9KTjhR0bHGr9i+uZ5z2PV755hRWGJyhXn+BYh41CZknivKDB9YtMDttuEXVbqFPjT1+MTAaenhB+Yj2f9Hyj0exQV1zZ40pm9JyBIV+E9ga7X9lc4mzh11PIRwaI8gYeHuBR4/gr1hdjtphb1X1WmVgHawzheNdNBs3+S8o8by6lx2BFXzjlHPAbGAh7Ym5ggSmWjakbnY5brVYhzZnpWLdv0L2D6HNDH6fzm7I/ihwUybBHhlGsKhbXVAdQUBlJVf8foeOVzXy49o9tnst1He/ZfugxBxLn2X80WUzsytzF6pOrsVqtrLhvBd9c9E2b38aUj6eQtSSLq365iq+Stws5wfMgMCQsLIz5L7/MFLW6XuffNmCKWs38V14lLKwJMrnGctg4HU596c5brR+lF1xtgAHvODR7eIAyMJWjEU/yf6ufarSbKR9PYebymQ7G9KSVSax7dh0VeRX4dvRl5h8z6Xtr3wZ6ceTAogOsmbMGD08PVpYtICVoISqVVZREWBIEGcsb70TCfRTugh/Uou6uC1w5/tqS5L+T4UQyGmM4MYGRZ2dQiRaTECL0xffn7AdqHX/1KdR4Bnji4d30miO5B3P5sPeHrH1qLd25nIt3V/FoeB0J+qL9sPM+ERTFufEZnadeqvaL2dy6jce5RCFX4IHwaqtvGcDUT6c26/pdC3ex7pl19uyhzBIRPRki0xBS8CoU7HTvDf+X8OsJ4ePFAqkOSrnSbijJrxRSeY9kP8K1K661n6PVYo9ebazuTnNRapQE3xbMqbhT9PCQQdpiJ3kMq1Wq8ddazGYzR48ebfb8IpNBF/lEhh1bz51dn3M6HtwjmG6Xd3N7weNdH+/i2G8iqq642EqvkH8IVu536xgSrUChBpWOnIo8Fqd8SEbIZ+R0HMiA/01Crmj7ZYF/nD893r+H3JiBGNSdoduDoOvS5uNKuMZhftn/JPw5CMxV+GrELsqkELuovGwTqZtSKTje+uwKGzKZDI2fBm2glo2pG7nyxyvZJJ9HlUmHKmywiNqTaF94+DoYxeo6/nQ68PYW7/3G5D4nfziZEXOdpcHr4hfjR8cLOvJ9+vfs9HwJvUc6quBeQlpc4rygwfWLTAYX/ASJ8wHw19RKfVqtVrRasMoUePh7oS9uPHLtlxt+4ccrf7T//NpFr/HF5O+xlogCHkFBwP6nIal5iiYS7Q+R8SeiCa1Y7bKuLcUu9Xlmxt+GK2H3Q63q+z+HVwwkPAehY5wOBQVBtUcmpYrTnCpyDhTNO5THW7Fvsf3d2mx/W+Z31p4sTq1xvKY5+yPbd0Rl9qfK7I+my1XgV0+xwfMYW72iU86/3naNwWxgwMIBXPTNRZQbyul3ez/GvTyu3iwud1JkFt8Nf037z/Sry+yHH+b/5s3jApmC0XIN3wJrgG+AizQaRigU/N+8ecx+qIlzmKEIUn86e8EOMlm9tfK8g4pIiniRb49+2KKujy07xj/P/IPFaEGlVdH54s74x/o3+fqL376YG9bewN7svSytnMPhDg+hVsuEKoquu0OtOYmzgCYUwieCd4xDc1pJGhtTN2L2FJEOdR1/pRml/DT9J/Z/434b1M3rbyZ3xn0ABHschJUDIeXHRq6SOGv8OQS23Wb/MSFUvOsP5B7AarU2mvFnqjaRtSeL4pTiJg3n4eXBoPsH0fmSzpSWggwZgb51ypwVbIfj70Kl2ByfC5+R5PiTEBQfgmPvEqsSjr8SfVkjFziz/+v9bH9nO3Kl+FpllQnHXzdvE6qDcyB3ndtu9z+J1eqyxl+QVugh2KRuXDlxYmPFv8nJ7r0lT39PPO/y5Fi3Y/yqSoCrTaBzNMiVloJtbvPzc+/4/yX0TU3XrEiFHfdA5gqg9ndet9ixjbixcUxfPJ2wRPca11fPWc22t7dRaaykpBRm/lzIqYjFbh1DohWYDVC4h5ycbdzzx10cj3waEAWOzwZKtRKjbxAmtbezpJbEOcE+v3S8ChJfBmT4qoUBRC8rBiA/tYrPL/jcwSDXWgzlBrL3ZaMv1nOy8CRLjiwhQ7kBD0UVOq8qSVKnPVJ+CnJrMzXqOv7q1k3IckMieZ/r+3Ddyuso0Ij1jYfZE1/PQrAYW9+5xFmjwfVL5CV2OZ4w7zA23bKJPXeIOqNeXpDdaTgjvrkd3w6NG2QrciuoyKlwaLPJ3Xl5iQxCjr4OqdJ65LzDWAbrJsORBYBw/MlRoraK74Ut+LGlPDvqWf7nf5y47Icc1yX93xSSURJNx8MXEp6CkAucDgUGgme12JQm5TtvSgO7BjLovkFEj4y2t61/cT2vR77Ox/0+Zvndzlk2je2Pvhj1BT9f+zOFVSIKVWXyx8enXn/DeU+3mm14crL7lX7cStqvsP4yKD8NgKfSE7VCGEoLqwrpfnl3Bt07qE0l0Ux6E4d+PETVaVHH6brMN2B5rzYbry14cPZDXDw5naxOc3k1vjt3R0TwVvfujJw7l5T0dGb3Wwtbb25aZ14d4GqjQyZmm5O/DTJ+d2qODRBzQLExnwpDhdPxupSml/LPc/+QurFWGnbsvLHcsecOvMNqU7jNxqYb29U6NR0v7Eh+uXi3qMx+Quoz4mIYvwGCBjW5Lwk3oI2Ckb+JfWodrv/lei78/EL2l60Bzsj4s8Lhnw6TcyCnTW5pp8dHHOrwIJmVm0Gfc17UBv3vYK35T9A9qDtymZzCqkKyyrMcHH+uTA1Fp4r4uN/H7Pp4V5NG84/z5+K3LqbD0A6U1lS/8q27bYm/BaamQuioFj2NO5AcfxKCvI2w6z5GauPxLx+GeVcuqx5ZJQodN5Fr/7iWW7fcCgipjtxKYUWusg6Bi7ZC9NVtcuv/CSpS4Uct7J/rdMjm+LNtevOP5XN8+XGHCLm4OPGvux1/ICKzoUaiSa4Q0k11sNVx8PUFpVQmo+0xlsKJ98VCGvCvCW4rLLSelahJgOtWXsfo+aPxmufFa1oFBkUBvn7/0h32+Yg+B1b2IyprCQBmeTk++cn8c8+PZO9te++fscrIsZSdbO48lGrZAPijD5S3weQk0XwiJwv5JaWnPeNPby0RGRXVWi56/SJ6XOUsG9xSMndm8lHiR+z/dj+VRlHIS27WclWPlwj+Wwslh902loSb2DMH1owAq1B3qOv4A+zywY0FEpz66xRLrlnSpAzS8mpR5CVcU0bYxkDY57wWkjiPsVrAakWlUDGswzB6BPdAJpOhrSmt0dQaf9etvI6bNwgDp9FspKCygJw8YeizK05MPgaDP3HzA0i0OQoNZK+BshOAcPwBeFrdU+fPU+WJtqoznsYOeNUVV4m7UbwXJdyCtzf4mITj71iu87pPoVJw8dsX02lCJ3ubNlCLd5g3l352KRPfnNjivYw9488UwE19HoGfw6CqbUodnEuCgsR8Z7FAUtK5vpsGqDgtHD56sViQyWQEeIosXpuTtq0pOlXE4hmL0f0tatIU6xIhZORZGbu1ZGdn88Lzz9OvS3c2repPSda3XHH9NazftYsdhw8z98knhbynoRAMxU3vWK4U8+3ZYt8TsPk6p+aOIX4oTeJzSS1puNZjVWEV655ex8nVJ+1tGl8NYYlhyOTC/vBut3f5bNhnTbqlsqwyCo4XYLVYKTOKZAiV2df+3pFoP0TqRLRhsUVkUpWVibkPwCfSh6eMTzH+5fFuHdNisnD8jxMU6f8kOfQtDis94bJUiLvBreNItIIJ2xzW+p4qTzoHdAbgQM4BwsNF8E9lpevkCN+Ovox6bhSdJnZyPngGpmqTw7pkhf4ZdsRfxlHDmtqTZHIRWHGGet/ZRHL8SQgiJ8PYtYwLXsHwo5tQHzaxZcEWqoqqmtyFh7cHgV3EBqzMUEa1RUQ9BPvGQdBgEakh0TLUwRAyGny6Oh0K1NZseqvEpnfjSxv5bvJ3GMprswNtjj93y36kbUmj5P4SYk/F0kehh7wtTudIMp9nGV1XuDwbev6f+NHXwpreUVy82cNed7Muuz/ZzfeXfW+X0nEHUUOi0PUSi3U/hZVhYesIUJ92W/8SrUQdCL1fwBJ+MQAGWTnK6jIKNx1psqRBa0hakYT8ieVEpJZTbkkHUyXIpKiA9oYt489kNWKR6SksUTB09lCiL4xu5MpmjFGzsI4aEmV3/CnMXqSX94aY60DTulrDEm1A7I0iC8YqHCrh3uGAkMqzWC1NdvyVppdy8PuDFCbVb+D75/l/2LxgMxVGEe0tM/tjjb8dgoa0+jEk2gm7ZsN3Sqh2dtxotSA3VXPi660k/dk8C/a+nH0EvRrEpD+Fk8Fe308bITbfEucXchVcrYdBQvbNZoANMPciMSzRLVlBZTViN5ISgRvYcS+sGu7ULJNBqDoGgFOFTQv4Gnj3QG7fdTt9b+5L50mdm/1Z37TuJq749gqKqmqlPvWySPCOA49/Z/F5W9bf0aPn9j4apMu9Iruszvu8ruPvyM9H+CDhA9I2p7XZLXiHe3PFt1dwtJf4RWV3fQwGvtdm47mLNxYsIDoqio3z5vFI0lF+rM7ktbKjbJo/j+ioKN54/fXaky/aBCN+aVrHVTmQuwGqz47jFYAej8FgZ/ntkBDwNIj9RmOOv8Cugdyx5w6GPTLM3pZ3OA9DRR1b2Li4RutK29j7xV7e7fou6VvS7Y4/pakm4+/QfDj+fpP6kXAzh16C/c84NEX6CMdfoVE4/qzW2ne5TCazK9G5k4rcCr67ZBG99oj5Ktzv3/ke+bdRV+5TpaoNVHUl9+nh5cHIp0Y2yeax6ZVNvB3/NoUnxbyZat1Ijv9SKhV1NsL5W6Eys9XP0Bokx5+bOReFGt2CNhJCR6HRifQg87ALeSD5AbxDvRu5UKAv1pO9L9ueIejt4c17nVK44PB2ooJkYK5us1v/T6D0hNF/QKdZTofOzPhLvCmRSz+91OFFZ5P6PHXKvcppFbkVcAg8qzy52rAVNjoXSbc5/gKkd2KLkcvlxMXFNW1+kavAM1TUcQMCA+RYZAbMmMitcK70nnsol6SVSZRlNl/e1xUWswVTtYlyg8jQ6KlS8PSIK/Evb+KmQ6LtUWqh11xUkcLxZ8FEfkQn1C8+RbepbV87yy/Gj6KBweQH5bNIHw2XnpAMsecQh/kl7WdYOQgKd+Gj9sFT6UmgJhSTotyeve1O/OP8GfnUSCL6R9idOwqLliMVV8Kwr0Ej1dBod0RNga73i3cNQp5xZsJMHhz8IAazocmOv57Te/JE5RN0ntS53nN2fbSLQz8corpmDWlWxCEb/BF0uMItjyLR9jS6fvHtCR0uxybJ8+XeL3lm3TOcKjqFlxfILBZOf/gnh3481OA41WXV7Pp4lz1r3Wbk1+AH1NT3M1VC6QkwNSwbJtH+sTn+JhYtZc8dexjWYVjDFzSAxWph1m+zWG16GrO8stbxV7QXlsbCyaZliUjUwVwp/s5qMsPrEuUlNqVpZaddXnpy1Uk+7PMhyWudHYNWq5Xq0lqbQnP2R7YsMg9TAAdNs+Gizfa90r+N88LxJ1c66a06ZPzJRCaFSW9qs1vw9PckYWYCpwJFZLStzmx75o0FC3hp7lzWm82s1Ou5FhgLXAv8qdez3mzmpSeecHT+NZXsNULRIfcfN991A4Rf5CTfCCJYx+b4SylJabALpVpJWGIYah/x96wv1vN+z/dZce8K+zmT3p3EhNcnNOmWOl7QkcEPDCaifwQaP5H9qDL7CbnwY2/C6W+a1I+Em0ldDMlfODTZHH9Z5Zn2d3fdDK6MHRmkrG/4+9NcPLw9GLlgCod7igm2s/kUJH/rsiSTxDkiazUcfdOhKSFEOP7254iajza5z8Zq0jeGh7cHGl8Nvh1FwHSlVRhMIv2EjR6LGdaMhO21NQfPhc/oPPVStV/aUoe8zbFa8NMJx12pxRu/GL8mR0kkrUzio8SPOPLzEQDkMjmy0o74VQ7kwsDX4AcNFLm/sKoEBHk61viLGRVD31v6ovJU2c+JihIym1VVkONGmetuU7txetFpDvc8zE6/8ZA43+kcKeOv9chkMnQ6XdPmF6sVSo6IOkyIGn9qk8iayavIczp99LOjeaL8CXRROrfca+aOTF7UvMiut4UmdrJey5vbvkQd07TFtsTZw9ujTt0DVSU5uWdnSRDeL5zjo/3JDs92uAeJc4PD/GKqErJLpkrkMjmVcys5fHM2alMwhYXwyw2/8OnQT9vkPuwZfxYtOvdMRxJnAZVCxbdXfMurF72KRqlpsuNP5alyWKe44q4Dd3Hx9xfbfw7wluaL841G1y+dZsGFS+xO/ne2v8Oz/zzLkbwjaLVgVmmIeuomRj83usFxStNK+f2O3+37EJusn4dFGHKDgoCCHfB7FzjZNnOYRBuTtwkyRI03m+PP4AY7W25FLp/u+ZQdmheQWVXYpxmrWWSEKTxbP8h/jSGfwaS9TuUfAGL9heMvrzodo9m5pIjCQ0FVURX6Yj0lqSUsv2c5qZtExs/H/T7m6/Ff289tbH4pPl3Mlte3kHckj5sSb+Ii3/vxqu7sWHvnX0hdx19TAn5tspEDu3enS2QkA7t354Xnnye7LYt/m/WQsxaKa4M66jr+ul/enfuO30fsmNg2uwWr1YrFaqFYX4yfHKKPz4P039psvNayf/9+Hnv0UZZVVzO4nnMGA8uqq3l8zhzx+aUvbXpd24D+0O9Ne93dc4mD46+4cceNocJA3hFh55ApZIx6bhRx4+NaNHb0hdFMfHMiHl4eVCMCDVRmP+FYmrQfhn7dcAcSbcOoP2DSQYcmm9RnRlmGfV6vW+fvt1t/47dZ7v2bVuvURE7px+mYYwB0zF0OW67/9xaOPR9J/hJ2z3aouzi5y2TemPAG9w66F6itSZ9WT1L56jmr+SDhg0blxYfOHsode+5AoVJgNIJeIZJxOgbVOP6sJuj7KsTdYr/mXPiMJMefmzGbm140tl1RtA++U5CdNorVvSPYUvk+xaeLqS5rWqZeSEIIF8690CGF3pYdYPbpCTHXgmdYW9z5f4dTX8DWW51W8GNixzBn2BzGxI6p91KlEqJrMpXdLfc5OmY0t/e7HV38dRB7vdNx2/dAyvhrOWazmQMHDjRtfpHJYEUf2PUgIGr8eRiFQc1Vxp9ap3arDIJap6bXNb3wiBdWmUKjjl2FNyDzc19dMAk3sP4yFFtvwlMpDFpmSijel0LuIefvSFtgq9l1kVe5FE1/jnGYX2KvFXUKQi60H7fN3SYTmK1y5Cq526SBd3ywg89HfE5JWond8ae0eHFp7LOw8wG3jCHhZtJ+FnU587e6PGxz/OXm1tbZqI/MXZmkb3OhsVKDp78nsiCxOZJZVPQJ2wmbroH87S26dYmzT7PWL4C/p3DUFeuLRa01mQxrx2h0kQ1HA+g66Lh25bUkXCsiem0Zfwqj6C84GPCMELJigYNa9jAS55a9j8M2UUvenY6/tBJh9dGYwpBbVbUZfwH94eJdEHNN6weRsBMTFIq2OpYY1WB7nfi6RI+MZnbqbLpf3p2sPVnsfH+nvRZs9MhoIgdH2g1xjc0vWbuzWPXwKrL3ZHP/4PuZ6vEWXtWdGKl7FFJ+aLNnPNfExYm9f0lJwwG/JhNcM2MB0ZFCNvLBo0f5IDOTB48eZcM8F7KR7qS6AP4aA0kf2pvOdo2/xTMW83bc23x3+Xd8OOYptCnfQOGuszJ2c3ljwQL6JSYy3Gqt1+lnYzAwysODTz/5BPY/DXv/r2mD+HaDbg+Ad9s5W51I/hp+DhUSo3UQjj9hV0wuatzx99utv/F+j/cxVhlR+6gZ+dRIEmYm2I8f+fkIy25fhr5E30AvjpjNZk6kibqySrOveDdoQsAnvsl9SLgRz1BQOQYA2jL+UktS7Y6/0tLa4yOeGsGYF+q3kbaUkjIzRmUxAIYej8OIX+1KKBLtgB7/B+M3OZSS6RfejweHPMigSLEHiBAl6usNVDUbzCDDrmjYFEpLwaAUjr+ogJqsF4VaKOV0nFbb9znwGUlFdSQE6mDoMI30tCKqPbLwPJ7PW7FvMe37afSa0avRy0N6hjhMqqtPrubXitXIfUdgCr8CBkmyTK0mdz2c+hz6LQAPP3vzxZ0v5uLOtRHxyWuTWTZrGeNeGUePabXOlrg4OHkSkpNhWMsVcRxI25zG0KKhXH/x9fU6jyTHn3to1gsi4RnwElFufn7gUZPx58rxB5CxPYPK/MoGJdeaSnCPYKYtmsb6lPVwHBRm7399ZO15SXUhWIz4qH2oMlVhtRYTs/Z3Nrzal2lfXNqmQx/84SDdl+Wy/1JfpndKgd0PQfwtjV8o0WY0NL8oleDrKwxI/Z+bapeOdgeV+ZXkH81HrpA7ZPx1910CGfkw4C33DSbhHqxWIZlYJ4rSaDaSU5GDRqkhICAIpVIYFPPzRZ2W+vhu8nf4xfhx65ZbnY4Zq4wUnSxCF6Hjo17H+OaHSqIu2A8p30P0zLZ4Mok2osH1S9FeEfwRdxME9MNP4yea9UWEaMUpFeUWyrIq8Amvv/ia2kdNpwmdarutyfiT6UV/gYGArrNLZQqJ84SeTwoJSUBdo9B4RPMZXd6Zz5QuU1gwYUGLuk0vFcEH6mohOS7V+HMDlRmQ8RsEXwh+jnaEoCAZY34+xbBhEOzlfGndSPiul3ZldtpsPLyFp3fimxOdzm9ofokZHcPNG28msLMwwJWWglpRQQ/Za5BxHUTPaMnTtXs8PCA+Ho4dE1l/YfXEXs9+cAErF89lvcXMYL3j7/FavZ5twJQnnhDnPvSQe29SEwr934bAgfamad2n0S2oGyOjR2KsNLL/2/0EdAogdnTbOKL8Yv0wVhiZ0XuGkKXtf7+QIG1n2OQ9u1mtOBd9cc0NVVW8uWgRc2//yKXkrkus1rOfteQRAD5dnGq963QQX3IzEYVX8dKtkY120/2K7gR0DqAitwLvMG+Uasf+MnZksHvhboY9OgyNr6befo78fIS//u8vpiycQtTwKKZEXcrepdNQG8PRqQugtBC8okHh0bLnlWg5hmIoPwk+Xe0OwO7B3QHxHlfq8oEgB6nPnlf1dPtt7Pl8D2ue+oeISRFkRmaiCxkOCsnp167wa/xzt+1P85wF0QDX640z2f3pbtI2pzFhwQQ0fhpyCiuxyMX+ONgrqMm3ezaQMv4kBNoIuHAxJ7z6AJCrq2TIQ0PtC+WGcJX++k/KP+zSvEqu70pJ4tFd9H0Vrip1cPq5QqVVodapnVKIbcbakyfdd0ubX9vM95d+D8Yi+LUjHHjO6RxJ6vMc0PMJiLkaEBl/amON1Gel6zfbstuX8fsdv7v1Fmw1/m720/PCgI5ShkZ7Y/x6GLWcLy/7kr9u+IsQXRdSe0wgdHzvNh8691AeEadMKE1KFntc0PSi8xJtjz4fTn1ll+Z+4q8nGPnFSMpDVwG187m7GPnUSB7NfRSfCB8+nvIxH0SVE5vzIMuN22DSAfcOJuEeOk4TdTlDR9mb7l5+Nx3e6MAHOz5ALofQUNHemErYmHljGP7YcJfHcg/k8kHCB+xbuA+dsQu+VYmkKm6AGdUQcbHLayTOQypS4Pg7UCzkm/zUfkCdjD+g9MtfeT3idYxV9UfdWswWh/2ITf7eUiEWn8FSudDzn4gJNfUgsctx6k16ThSe4HTJ6RZ3m1YqMv48DcLxZ/vekbsejrwuMpMkmkf5KdhxN2Stcjpk2w82VDc4bXMafz/1N2aDGV2Uzl5nq7l4+nvScXhHVIEqDucdJqMkG4PZk21hqZD4Uov6PF9orM5fdnY2H33wGCstzZCNdCdyJXS9D4KG2Jsu6XIJc4bPYXDUYCwmC7/f/jt7P9trz7jI3JVpr/1orDQ2+E5oCuNfHs/M5TWBRDI5aIIatbOcbbKzs3n8scdYVl2NAWiqflYoUFJSIhQ8Qkc27aI/BwlVhbNJ5CUwfgMED3VolskgKiAYrSGWksLGnWw9p/dkzPNj2PL6Fl6PfJ2S1BKH4xc8fgFzCuYQEN9wJLpJb8JitqANEpFH/rIogkrHodP3wqdosZALz1nbzIeUcAunvoCVA6Boj73JT+PH/0b8jy+mfkGATkQElZTUc72bkMllmBUyjCojanxQSQqf7ROLESyONWKPFxzn2/3fsj9nv93xl9sKsauUf1LY+9lelBoRaJCSJ7L95FZVbTmbfU/ByoHnfC0pOf4kHAjWidSc/OAihvzvIsL7hTd6TerGVF6Pep0jvxyxt2WWisWhxhhGZMFTsGdO29zwfwl1IKicw1DNFjMpxSnszd4LQNTgKO7Ycwfdr+jucF5cjcx5snOt9BYz7JFhjPhoBOX6PKyaUJd1MGyGYinj79wgMv6ExSu7zPWbbfTzo5n80eRGNaybwuHFh1l2xzK8y7wZ6nsFisqeVBEGShdhvRLnnImdJjImdgwdwnzIjRuCtWNMm4858NHRfDdLQamvmSpdHIQ2XLtJ4ixSmQZbb4RMEQhwIPcA61PWY9EJmZ3ctGq2vL6FY8uOuX1ouUxOdbkXCqsnOl+Fy/edRPskzFuYorLLxdrPll3QWE3hvjf3pdtl3Vwe8wr1YuTTI4keEW2X7dHpEFHW7TAiX6KFhI2HyzLsmTd1pT61NRl/xo7xDLxnIObq+jN7Nr60kXnaeeQeFOuc3Erxr0IvdvZBQcDx92DtxaA/O5LWEm2Ht7cwDKuMwouUX5nf4r5sGX8aYxReXiC3WUcyfoc9D4Oxja2I/0b8E2HMalHq4wxsZW8KClwHDwOcWHGCDS9sYM9neyjLKnM4tnL2Sv64748m3UZ1aTUWk4WU4hR6vt+Tjzy6YEWOOqADaKOa9UjnG405/j7+aCEXyFTNk408i3j4eHDtimspTCrkq7FfUZ5TzvdTv2fJNUuoKqriu0u/4/up32OqNpG0Mon8oy2bAzJKM1hyeAl7kldAyWEHNYP2wCcLFzJaJT4nX6Cp7tccwLc5kjuWmpqmyvaz9rYF7NSXkeOKkJ4hRF8Yja6Dozy4xleDZ4AnMnnDXpqEmQncn3Q/wT3E4BUV4oWg0YAiqA90n9OkbCKJNiBkBPR50Wnufnb0s9yYeCOh/uK7W9fxt/WtrbwZ8yaFJ90XuZp4YyKx8++lT9Y+nvRbB997wPa73Na/hBtI/lp8LlkrHZpf2fQK1/1yHT8d+sk+v1RUQGWlcxcVuRXseH8HGdsz6h3msi8v47Gix+yOv+ziEuQWNVqCapNwrEbh9PPwd8ujtRTJ8edm5PLz+Fe6azZjDSLK36AoaHK0hLHSiDZQizZQa29LLxbLEk9zGOq8XyBjmdtv9z+HqQoK90ClYz2c1JJUYt6KYeinQxt03Ngy/vLzHbWvW0OHYR24LP8yfN/uzomB30KPRx2Om83Y0+0lx1/LkcvldO3atenzy877Yd0UQBhHfI3dCCgbQaRnJ5end53Slc6TOrssNGu1WKkubVqtTxCBALs/3k1iSCJ3BS7h5J7l/Fa9XVoktzdyN0LSx/aaoeE1MR5ZWW0/dFkZdM5+ksuOF/Ha+H93xPX5gMP84hMPI5ZCx+kAdtk9hVYsCAqL5ax6ZBUHvnFPNt6xZcccnIi2dUe053qRMSDR/jCWQtInkLfZ3mR3/FWItV9TM/4awi/aj1HPjKKsUxk/5D1JWuCXhHkliXHNTX8nSZxbGl2/KLVCdUQhIrXtUp9VRfbMq/L4Pkx6d1KDWT9+sX7Ejom1R+kPjRrK5JgZ6Cr74ONTIw1Zngy5/4Bckug6Lzn6JiwJhpIjKBRifasyCcefLcOzJdTN+POuWz6o64MwfiN4Ni4zJ3EGKh8IGydqMp1BYCBk+v/Aj+Fx3PjrTS4v7zerH7fvup0V963gt1t+cziWvjmdlH9EMFJj88vP1/3MPO95dsewyuyPl6qIQHUymN1QILId07Wr+Dc5GfQufFmLv1zErU10ct1QVcWvixa58e5q2DUbfg4XTieEWsyuzF3szNyJTCYjfkI80aOi8QrxQhuo5fKvL2fGzzPQ+Grwj/cnoFMAZZllLJq8iOV3LW/W0EWnilg5eyV//fEXV/50Jds33A3Le9qzz9sLSxct4vqaD3Aq8HUTr/vK05PLZs4UwS6/NGEOkytgzJ8w+OMW32uLMFXAwRch7VenQyEhcCr0DZ7edy1H8+vxYNdQVVjFtxd/i75Yz4xfZjjZNIyVRnIO5FCRW1FvH2fa0eRyObuNu0gNXojMN01kp/Z9+V8fNNBuCegnVK3qqUFp83PXtWGrtCq8gr2wGJsod9tEKsoVaIzhdPKLFntm/z5u7V+ilXh3guirRTmzOiSEiLqfB3IPoNHUSru7yvqryK3gj3v+4Pjy4/UOI5PJHPYmQeYELt5dxRx1HYm9xPkw9ZTIKq/hXPiMpJBZiVqSv6KLShhvZOZiVlz/HX2uiGXIA0MavKzThE4OdTUAsspEmHegJhTZxfvsNRkkWkHJIfhzoIh06fmEvTnUW2yq9CY9ZYYyfDx82Pb2NnzCfeg5vdbZotUK435WltgE9HHD+8litVBaLbyIvmrnqLLiYuFXkMuR6ry1Eg+PZhiqKtOhQhjNZTLoyXRCj03nujvqv8RqtWLSm1B51mqU7/92P79c9wu6Djpmp85u0tAXLbiIEU+NwNPf077wkj77dsjJTyD5S7Yq49mXn0Sl3wA6HMzh2N1JWK+6z6UT2F2kbs/GJ78Kz+4x8Es4+CXA2L/abDyJxrHPLyodRNXWeLTP6xrxx1xUruKWTbc0SQa8Kfw9929MehNdp3Tlqb+f4hdDOuFedzGodCTsvxaGfeOWcSTciKEEtt8mjOLBomBwuI+IHDgz468xx9+hnw6x+tHVXPHNFXS8oKPLcw7mHuRv04sEBI2km3w3rH4bLksHrWSMP19ocP1iMQmHnFIL2ki746+4ujbjz1Uk7pn0vrY3va+tlaq+c8CdDJTdyTOLIchmI+r3mvhP4vxEHQS+PQCxPtHpwKOwxvFX1QrHX4lw/GkMUfjUfbVpI8R/Ei3DagVDEagdIz/9/UEuU1CpTuZormvhQr9oP7zDvJn45kS8w70djl2/5np7zT9oeH6JHhmNd7g3ORXCLuFhCGNE9Pd02HM3+K8Wzsl/KUFBIui2sBCSkqBXLyEb+cnChSxdtIjTp483XzbS3Xj4gU8nMJWDhy9b07cy/uvx9ArpxYG7DiCTyRj30jgsJgtypdyh1t/kDyaDTBheL/3kUmJGxTRr6Ox92Wx7cxu+z4h17mlVJPS8XtRva0cUFRfbP6dZwPPANmgwU3MbsM5g4MtZsyC7AlTtfCO+/0mIuR46XObQHBwMWX5LOFy2iX3Zl9ItyLVCBIgM0dSNqfjHu86qydiRwZejvmTi2xMZfJ/r397mVzeTfySfSe9NQqUV9pBPjr5OUsx+4kpXAR1a9HgSbYvJYmJ7xnbWV+3Hyh3k5dXaMPrf1p/+t/V363j7vtpHwWoz0A8Pn0C44Ae39i/hBoKHOskHA/QOFfuEfTn7xGnBIiA9Lw9iYhzP9Y/35/rV1xPUvf5afafWnMI/zh//ODHvlJaCDBlBfs4qeOea8zg9rX1isbg3muCsMvkoB7uL7Au9qoCcLScpON6yjVRuTdR3mHeYiCCSJLtaj088JDwHIaMcmrUqrV1DOKc8B5lMxrr/rWPXx7ucurBl/Z1yQyKFodzAfN18xq0aR6IaAk5/BhWpDuds2SL+DQ2tI50j0WwsFgsHDhxo+vwy4me45JD9R/+aNXBRkevTjZVGXgt9jd9vd6zzd2SJkO+tT4rNFXKFHG2gFgsWioutDI5cSqL61X99ZO15R7cHYfSffLn/e+5cficnZX9gUagwabww6U2NXt4a9r71D122fi2irCImQfCFbTqeRMM0NL/YHH9Wj5qMv0LoMLSDPaumtUx6dxKT3p0EwG/Hf+OQ6gv0qjyyI99yKREm0Q7QhMKoFdDlXntTfVKfjTn+lBol2kCtS7WCrW9t5csxX1KSJb57SrM3FQHToN/rQo5K4ryg0fWLoVjUzDn4PACXdbuMjTdvZMFFC+wZf9U5xSyesZgD3zUv09gmDxZU/55d4nwi9joY9w/4ijWpTgceJvHhFlQWtFiufvX1q/l64AmCSyc4ZvwZikWNGImW8c9kWBptV5awIZdDqDoGgJQGajNaLVbixsfRY1oPh3a1T20d+8bml2EPD2PKR1PILMsU1xoiOF3cG3OXh8CnSwsf7PxAJoPuNVU/tm+HNxYsIDoqio3z5vHg0aNEWS1tJxvZVBKeFvXdPETfAZ7i3V5Y5SjLJ1c6GxFkcpn9e5B4UyJ+MX7NGrrrlK7cf+p+9INENl2WZyfo/Rxo2k9B2DcWLCAnO9v+OYUB84EpCOeeK7YBU9Rq5r/yCmFhYaKW5QXfNz7Y0bfg8CtOf69tjkILE3e7DMoJDgavapFccKLwRMPdqBQ8Xvq4fU9xJgGdAhj5zEgiB7kOGrNaraRtSiNjewZWi/gdWCwWCmuyhf09feGvcVL5onOJxQyrR8CuhxyaDWYDIz4fwauH70KvSm+V2khT2PL6FsrXrOZQhwfZU/1T2w4m4Vb6hvcF4HTxafIq8hqs86fyVBE3Lg5dpM75IMJ++vX4r1n39Dp7m0NpCoDiQ3DgOScVo3PhM5Iy/iRq0QQT4htDiKwHcnM8/X5+kksmN3xJcUoxG+dvpO/Nfe0vUqvVSqFBRNZ19AsUxdG9YsFLipJpFR7+kPCUy0OhXqGUG8rJLs+mc2Bnrl99vUvDbFwcbN7snjp/xkojwQODKfMoY5yXEtX+J0QBaS8RtV9VBd/XrDMvv7z140m0HD8/8W9+oRGrVemUzaXSqug4vCOB3RyzeK766SqKThU1K7sne182So2SN9Lf4CXTfH7r1oVuVUdB1rSMQYmzhH8iADpPkWlnUReR0X081UGgauMgpeCLB/CrZTfH/SZwWfgTjIoZ1bYDSjQdqxUW+0P4RXDBj/bsG5OiBBmiJo/FbKH4dDFeIV6ofdStGi56RG1kdaVRpPVYTH6YO90PUqJF+0ThARETHZpsjr+ssiysVithYeId09jmu+uUrnSd0tXlsYqcCrL3ZqNGfMeUFm9koSOg64hWPoBEu8LDF3r8n5DQAiJ8IojwEX/8OTWKrlV6OPTrIfw71V8f45/n/kGpUTJ8znCsViv5lfnk5gUAilrHX/oykWESIgWb/BvQ6cDDLNanRouRMkMZOrVrA01DeKo88TV3QmWulX0C4K/RIsN5qiQ73SLCLgJtB7AY7FK+NqJ1IhI1tyqLKmMVni4Wnt9c9A25h3KZUzDHYd9iMVlI+jMJhYeCmDExTbqVrDKhY68xRpBSORzFgOEtfKjzi7FjYdMmeOetBWQcnct6s5nBZiGrmYyQjWxKiJVdNrKNsTn+iqrqiVRtgOqyag4vPox/rH+Tsv/kSjn+sf4UnywGwF9zbusv1SU7O5tbbrqJLatWMRPHz8m2mx4BjAJuQGRkZgMLgS0yGS/Pm8fshx6iWZx4X3iLe5xlx5ZMBgF9XR4Sjr/OQOOOP9FV/Wo1ukgdo54e1eC1M36ZQUVehUNGcZVF1BgN0npDRQp4hjd6HxJthFwB+hwwFjs0a1VaEkIT2Ju9l2KvHXgWd6CqCjw9oTynnP3f7KfD0A50GOYeW/SVP1zJdfPfIzn0LZT647DjH1HuqJ1lC/+nMZbBnkchaCjE3Whv9tP40TWwK8cKjrEzcychIRcD9dcRtVqtVORW4B3q7fL4JR9eYs/2A/ir8FN2xP9GZ+OVXMv1kPk7HHha7Du849z3fC1AysGRqKUihd4qM08HHWJA8uIm1fhL35LOrg93UXSqdoFWpC/CZBURkl2DgDUj4YgkrdOW2OQ+bVImkYMiHSYhG3E1883Jk06Hmo1XiBeDFg1i69CtrDAFwPjNDvrWv/wiNLYjImD8+NaPJ9EMivaKorYmoWOv87XwV0Isk7dp7FGvZzLjlxmMmOtoUJUr5M2W9Fty9RJ+vOJHyg3lWGUWXkkey8lO20AuxZm0R/xtjh1lMSAcO8Y2DnBXdYtn+6CNJMtXt2hzL9GGyGSieLpvLwB8NSIKu1pWDIjvx76v9vFOp3dI/ssNESR1qDQIx5/CrK2NlJNov1hqM4NDvcQapMpURZmhzJ7xV1raNJlGV4ydN5bHCh+jSl0FgMLsLX0v/o3IVZA4z0Fi2IZN6rNC4ctj5XMZ++LYervZ8+keDv90WJxvrCDktRBuOqXCJC+vdfxtvx32Pu7uJ5A4W1RmiMjp3I2AcPwpLFrCPTrTN6yvPXikJZQJ266j4y9iMkTPaMUN/8fp9gAM+tDJ6QcQGRCAwiwMaaklqU7HAYJ7BoMVzAaz4wEZLJ6xmI0vbWxw+MKkQn6c9iNJK5Psex+NMfw/VX5gwAAICMjm4IHHWGaodpCHnAWspf7MMRs22chbZ81y/w1WZcGheSJInFrHX5WpiipjVbO6MlWZ+O3W39jx/o4mnZ93JI+yzDJ7duGN+nWw8epmjdkWvLFgAR0jI1n955+stFp5FufPaTaQgnD+vQncjcgE3CSTsXPvXkenX+pi2PlA48o7E3eJGt/nAmM5VKQ5NQcGgpe+xvFX0Ljjr6Wc/uc0ILJI6xr4LVYLemu5uBefILj0BAxrapVFiTZhyjEY8plT88CIgQBU+ou//xxhEqUyv5LVj6wm6c8kAMqyykhZn9KqWwjqGkRGUDEA/dQlcOI9MEllrdoVMgUkfQTZzqVkBkaK78qOzB0NZvwBLLlmCa9Hvu68DkEkTgy4YwDx4+PtbcnVu8nx/40CauoCdp0N4zcJB+Q5RnL8SdSy/U5YfYF9QZy/O5WTqxv2EPWc0ZMHTj9A50md7W1+Gj+e1KZx4eFdBAQEQP+3oYOU8uUWdtwNq50jlW1Gt5xy8ZazmC1U5DkXL46uCUTJzAR3ZBgX64sBMHsECB3lGknX4mLh+AO44QZQSj6fs8vpRbDlBhEVBQQGyAErViyklDS82LHJW6RuSiV1UyrJfyfz83U/k3uonjfiGQx9eChDHhpCuUEslPMqI/EIH9TyZ5FoG05+Bt+r6WkV35EKcxFeljJCT2zi4B/Omy93UlYGJnk5fnIYmPEppP3SpuNJNJORv0HC/wAh9alWqFGrRRRtSQmED4hi8IOD8Y1unfWsqrCKFzQvsOqRVQBU1Bht+wQcxXt9b0j7tVX9S7Qhv/eAP2vndS8PL25JvIVHhj6CxWrB07O2tqtt8+0KY6WRLW9s4cQf9Rt0SvTiXaKweBF84johtSTxr6W0upS3tr7FSxtesjv+kMkwmBteSN65/06u/k0YbfMqROiuwqoW3xubctugjyHhmba5cYm2pzpPRE7nrgNq55jngo+z+47d9szj5nAs/xi3LL2Fz468AQhDs50+zwuZPAm3ExQkQ1stsv6Si10HEV38zsXcf+p+lGrHv325Qs7lX1/OhNcnNDhG0akijvxyhNKMUjLLa6U+b+55B2y5qfUPcR4gk4GMhVwoUznVhGuRbKS7MRTBvrmQJdaBPh4+KGQKQASSNwevEC+u+e0aJr3nWurxTH668ie+GPWFfZxAayVUtu3+pzHeWLCAl+bO5WaLhbGIOn71fU5hwFxgByIjMFet5tXXXqN3796Oneb8DcffBmMjEf0qb9C5VmBoc9ZfBn/0cmqu6/g77gbH3zcTv+GP+/5waLOYLHw19iv+ee4fp/NLq0uxIuwiQT7/oYiB8xCb469c5+j484/z55bNtzDwLnF81UOr+GLkFxgrWxblbKo2UZFbQblJBAxs002GyzNFrVKJ9oPCE67IbdBJvD1ju31/UJ/jL25cHP1v79/k70upUZRJC/WpiThUeEDwMFBomnf/bYBkjncz8vO5kFnsjRA6Gl2SFZBR8t0Klv9azf0n76/3EplMhl+0n0ObXCbHXBSFb2UUuiCg631tetv/KUyVogC21SpW8zXYHX81GX9Lb17K/q/380TFE/bixCDqnCiVYDJBfj72KIeWkLw2mSM/HkHnqSMyyltEkimENML334NeD126wLBhLR9DQiCXy0lISGj6/BJ7AwRfAGrxAfv5gachmip1CinFKQzr4PpD2Th/I/u+3Med++7kn2f+IXVjKpPen8SBbw/Q48oehPRs/AvTb1Y/AMp+FMZaf7kcP58qoP0Vuf1P49URwifgUVPLori6mASfKgKPruHI7yPoO7XtpJlTn/+KS80D2XbzfqJyl0NQghQccg5paH6Z3nM6M3rNwGqFK5aId4c8NJiJb0x00VPzsJgtxI2Lwz9eZKdXGkWwir9WjsxcBZzlOiMSTSdsPMgcvy+fTv3U8ZQw4SjOyqqtL+yKVQ+tovf1vR0CyAAO/nAQrxAvCsvFu0Rl9UYhzwFr29YglXAvTVq/bJsFZj0M+4ZKYyUP/vkgMmQ8dsFjaDRy9HpI25mDh77U6XtiQ+OrQeMrNta5FWIHr7EEI0NWm/EXNcWdjyZxtvHpChfvA20UUFtDxVZTpSUcyjvE53s/x08/hFHK2YwZ44b7lBBUpMG+JyDiEohxzKQKDATtkRjKtAdILnLt+FOoFHj6u947dL9cFK+zWq31zi/xF8XzpP5JrBYr049Mx6O8M0WH+hLp/TkU/3eyMw5uW8Rsi97lsYZkI7/SaPjHaGR+S2Qjm4p3PEzcKUrCIOxKAZ4B5FXmUVhVaJd9bipdJje9buOAuweAFZZWiSy39fH/49re5662dHZ2No8/9hjrzWbuBR6sc6yhz+kzYD3wan2fU8Iz0HNuw7WRiw+JNZ2uq9Pa7qzQcZqQ+zzDvqXRQIhKOFQKqvIp1hfbSxC0hNL0UtQ6xwxkk95Ev1n98Ar1cjq/zCBSweUWDXHeh+DUQYicDGqpzvQ5o3AX5G2B+JtBWfuZ2bK48lQ7sWIhJ0d8j1WeKjoMFTaNwqRCel3Ti6hhUfZA9+aSviWdL0d/SfwoJadHQbBvsCT/2h6Ryeqt1zq161Ri/GIYFDmIGhXweqU+bXZNV+xauIutb2xl2nfTCOsjAmPKLaImaJhvIFRmQnUu+PV2mlfPhc9IcvxJ1FKzKH969RgO9z5CR83nTL8upt7TDRUGUtan0GFoBzR+jl7sAuHsdoyclGg9Q79w2Tyx00R8Nb6MiBZSjbFjY1FpVZiNZlTUOv7kcggNhYwMYYxrjePv9NrT5HyYw7XvXssjfqtgSRBML8VqhdWrxTk33uiwfpNoBQaDAY2midEifr3EfzX4+4O2OoZCn/WcLj5d/3UyUfOgJK2EUc+OouB4Ab2u7kWvGb0cHMhNobhSGGu/HfAm3pu+gUsONOt6iTYmbJz4L+lPQGTvBnYJ4MjwW+k7rm03NCaZEqOqmlNGODTiH3qG9Gn8Iok2xWF+Of4e6POg9zP2ehkyGQQEiIi4goLWvTtseAV7MfN3UTPGaDZitIhouvTq8XCpJK/WrhnwVqOnhIbCsWMNZ/yptCpu3ngzvh2dI6l/v+N3IvpHUHyzeJd4qbyRXfB9i29Z4tzR6PqlLEk4/sBu1LNipbS6FK3WD70etrzwN1kbk3hS/yQyuePC0lhlpOB4Ab4dffH09ySvUuzgldViorI7/iTOb5Se4F+bzeIOx196aToAnoYoxoyp810xlAhp2MhLIfbcOQPOa2QKOP0NeIY5Of6CgkBX2Q+FLs8uKd5czEYzhnIDaKh3flF4iOyxm/vejNeJm/m6Cn4p38ADE/87gUWlZaU0lKs3G7gG+BR4EUhDjp9/V+58aCZfzZrVNpl+NhRqCOjv0BTqHUpeZR7ppen0CnHOAmuMyvxKSlJLCO/XsDF+0D1CteCxk49xZY8rGRx1Zk7k2eWThQsZpVIx2GymBJw+s7qf05tACeAL9AI6hobW75zVNGHBfuB/kP4rXFXq4Ew5a3S+q95D4QE+qA1hmNR5pJaktsrxd/fBu53aPLw9mPzhZJfn25StVCY/YpRLYOs8uOSw5Pg7l6T9AodehLCx4Nvd3twzuCcapQa9qYQKdRI5ObVBAGaDGUOFgW8mfIOHjwd37r2zxcN7BniScFM/fjCI/Nvu6hIoPw3eMS3uU6KNKEsCYykEODrvov2iifYTMnieNdNjYaEodaNqnrkTrODhJRJfrFaoRDj+Iv2D4PS3sHcOjFkjvq/nGMnx52Ys7tBPPMeUmnOp9sgmNUJJt6nd6j0vfWs6iyYt4qIFFzH0oVrd2hUnVrJW9Rd+vqOINFfCHy/AwA+FFKREmzC121Smdptq/znxxkQSb0x0eW5ERK3jr08r7O3DHhlGr6t74R/vj/Lk61ApNs+lpWCokZHv0aPl/UvUYrFYOHbsGAkJCSgUimZfb8v4AxqU+hw6eyjD5wwXEZfxAc0uglyaUcqiSxbR//b+lMiFsXZ9wTBih/Ru5EqJc4VtA1VUVUR4lJKd/lEUNq+sRrMxTZ/JEuttWABPryjwkORTziVO80vqj1ByGHo/43BeYGCt42/DvA0c/eUos7bNcjLCt4QqU+2XLlCnbeBMifaKwWwgtyIXD4UHIV4h9jp/2dkNX9dxeEeX7Vf+cCUqrYpQ5Riylt1FXIgUUXs+0qT1y7h19v/VKDU1xhs9xfpivLz8KCyE6OmDGHRrT6wWq9Ock3c4j4UDFjLmxTFc+MSFdqlPlVFE+wYGAoW74a+x0OdF6OJs/JM4TzAUgcUImhC7429F9Vy+fedHHhv+GLP6Na8O2YFUIe3naezAlVfWOVCdJ96F3g2kK0s0jGe4cCSofJwOBQZC18xnCLc+w8yE5nddml7K253epv8d/Qm/Odzl/JKyIQWFh4KowSJD1OYg1un4T0Wl+up0ZGe6ru9uwyYbGQM87N2Vfhce5tE54OFxFm7QVAnV+UKJBLhn4D1UGivpGth82Umr1cpHfT/Cw9uDe47c06RrxsePZ7xpGKR8D7Jy8E9s9rju4PN33+U5vQiA8UVk852J7XOaW6ftW+BgQAOOKFMlVGUKB6CqnkLJcbdA4JBz4/RrhMBAuGD/Dh66I4TeoWfjC1lLtG8047J+pKTUm+IJ3aD/QOmdcK6JvUE4Ubwc9w4qhYq+YX3Zkr6FYq8dDo6/dzq/g9pXTeItiag8m+vZcSS0dyhD500h7a05AIxIfRkKf4CJ21vVr0QbsOUGKE+GK7LqPUWnE+85g0Go4YWfsdW0mC0svXkp/vH+jHp6lMOx/rf1p/9ttYErFRVgUIrsp45BQeA9Gno85rK+37nwGZ3HupQSbufUV/DnYBK9hdGtUF/Q4OlB3YK4+J2L6XyJo+zO6hNrORH8Gvm6Nfh4GcSCQy75mN1CRRocf18YZVuIzRjXyB6gUdQ6NcE9gkXthR5zYMDbgKjvB2IilWr7nSPSl8KPOkhdAojMC8/qGACSG8j4U3gokMlkVBZUYjGLF5LVaiVzZyZZe+p/adowVhgxlBswVZsoq6nL9FPurZDwdOueR8L9VGXD7ofpaUph8VWL+eryrwgNBZnZRPbxVoTPN4GSMjNmRSW+cvDVp4poLIn2w7BFMGk/IKJdL/3uUkZ9MYqAABEhX1gIFXkVVOZXUpnfcrmszF2ZrHp0FbmHcvHx8OGHxFLG782mV9BuSFooMi4k2iepP8G228BYbm96dNWjdHijA29ufROgyY4/i8ni8nvUaUInoi+MRmuKwq9yABHekSIbNfNPdz2FRDvFFpBSrC+21/nzSYyn97W9kSudt65eIV6Mem4U0SNFgJMt48/DGIyvb43xWqYQUmKa0LPxCBJtxW/xsElkj9kcfyXGApIKk+zZe81h+1Hh+OvRIcrR4OPTCa42QC9p/dpiZDKXTj+ofT/k5goJ8ebiE+lDl0u6ENzLtZQXwIr7VrDk6iVUGivZn7OfzOJ8tKoSenovgtLjzR/0PGXqzJl83UTFmK88PQmLm4nFAikNl4R3H/9Mhj9qI5HvHHAnDw19iFj/5jtYZDIZF869kGFzhjUo5Xfst2N8OvRT0rfWzBkVaUJyOvWnZo/pDt5YsIC83Fx7lt9URN2+pvCVpyeXzZxZ/wmZy2FZZ8hcWf85kZdAj0ebOGIbkL0G1k0RATpnEBgInsYoSota7/RL25LG7k92Y7XWfjf2fb2PX67/hfLscqfzdWodvvmjCSmdiEdALHS4rF3U6vpPo+sCoaNdOqnnjZ3HwqEbCC++3EFtpNfMXnS/ojsj5o5g8P2DWThwIcvuWNbiWygrA4NS1PgriLlFCiZrr3S+W8gcu2B/zn6eXvs0X+77wq5i5EruU66Qc2rNKdK3NL6+LC0Fg7JG6lMXCIEDIHE+KNtHQLPk+JOoxVQGVZlEaL0BiNyRymthr7l8EQLoInUMuncQQV0dNXRSC4WVRycPRdnpWrj0BAQObNt7/69Qdgx23gO5jgWITRYTp4tPszd7LwDZe7NZcs0STq877dSFbWPbmDGuMYpOFZGSmkJptaPRvqimFre/f+v6l2gF6mAIGgIefoBYNPuYhUEsuaDhnZzVauXVoFd5Xvm8iKyXyfhq7FesfnR1o8MGdgnk/qT7GTp7KN28hxJUOo5QrZSl0S4xlcPR1/Eu3s20HtMYET2CsDDotOM79K+967ApcidWixXL5g10OZbAeK2MwLWjIWN5m4wl0UK0kfZ6BSq5imXHl/FPyj94B4gafAUFMOH1CTyQ/ABeIS2PDs7em82W17ZQklqCTCbDVOGD2hRKH/8lQmLNWOyOp5FoC/K2wMlPwFBobwrzFiarrHIRJNJUx9+3k77lrThH6dC6809tloYFdt4HJxe28uYl2h1FeyH5GzBXA46OP6+aKaaygRgD3w6+jHxqpD171JbxpzaF1Eo3+veBsX+LWkIS5y+dauQ3qXX8yarEh1xQ2XDA6plkZEBajbPwYlcKF3KVkBeVaDllJ0Uw4hlrysBAUb/LbIbMLHOzu5XJZExfMp2+t/St95zRz41mzLwxHMg5QJ8P+7CgvB+RPscZZLwW0n5u9pjnK7Nuu421RiPbGjlvG7C2uhovr1J2rOvOuKGRDOzenReef57s1hoNGqLjVeLv2uqeDIgBdw6g7819G1SjsJgs5B/Lp9RUyrf7v2V3SQ6MXA4xZ1/W11bbryO1WX6zgLXQpM9sncHArbMayHT27QU9HhcOk/aKPg+y/4TKDKdDtnd4fn7rh9m9cDfLbluGocxgb8vYnsH+b/bbZYHPpLJSmMt12v9OXdB2j9Vil4evy6iYUUzofgEKi5acnNrXzpgXxnDB/10AiJI2MrmsxWo1S65Zwo6X/2bEoX1cW3yIwMRnIO6mFj6IRJsSex10vdfloR0ZO3hu/XN8te8ru+MvN9d1N/cn3c91K69z7uODHRz68ZD955yCWvWiII2f07rnXCM5/iRq6XIPXJZGpU88ACZFFQHdQ5EpnCdGq8WK2eB6oZ5eIlLJgjRtqAn/XyWgP4xZDVFXODSnFKcQ+1Yswz8bDkBVURUHvz9I3hHn0IWImjrZWfUkcCX9mcTiqxfX+/na+HL0l3ww+gOC5vtyYmlfOPkpIDn+2opmSXwGD4Mxq+x60nI5dPSJJ6DsAnr7XdjgpTKZjPGvjmf8q+Pti6JxL49j0H2DmnW/N4e+y4VJy3g05iWRvSPRvvCKhqkp0Ospe1NoKBSHd6corj8WU9tIEBirjARu+4eLNjzH25fsx9rraVH0WOKc4jC/GIqg9BhYTGhVWhQycUztWwwIx5/MDTJZCTMTuP/U/USPEEEJJTUJfqc97oLRf4K0hmi/JPwPriwEbZS9yeb4yy4Xpqu6GR0NKZp0vqQzva/r7RCdf3rtaV7UvsjeL/byc+qHJIW9jMkrBcb9A73+5/7nkWhTGl2/nPwctlwv5N4Af41YQBZVFdkz/nJ3pfNGxzfY/alzRsCZJIYlMiLgavwqBkr1/f5tJM6Hbg8CtY4/ebUoKJ9f1TzL8O7doFeJjL9+8Wc4/iozIXejlHneWo68Cusvs/9t25DJIDSyir97xRP3uYZyg+sg46ZQ3/zS9dKuJFyTYA9G0ZojyCqPJynsJ4ia6vKafyNhYWHMf/llpqjV9TqStgHjan6PnjvfYUH5UT4ryOTBo0fZMG8e0VFRvPH6621zg53vgr4vg0yYJU0WE/uy9/HzkdY5Z8tz6v9Odb+iO49kP0J6SDrX/XId1y27CyInge/Zr1HyycKFjFapuJraLL8wYD4whfqdf9uAKWo18195peE6jL7dIfGl+iVMkxbCb52gcFfLHsAdRM+AGdUQNcXpUFAQlGuOsjD/Bm5fdnurhhl03yCuXXEtCnXtnDHpnUk8Xvo4Gn/nTL6dmbs46fsN5T7r6bDVG7be2qrxJdxAVTb8oIE9c1weDq0RdaiqgvKaKSD572QWhC/g2G/HAJi1bRaTP3Bd17EhrBYrKetTKDqai8YURoxXDzRKKQP0fGRgpEhK2pW1i6BgsUmtz/Gn0rqWh133v3Vsf7dW4tVQ6cmk3VXcW1KBd/KnsLQjFB9w7423AkmIz820pP5WeyPIS2yg9g4s5I3H/oeXCxWN/KP5fNz/Y8a/Op5B9zo6BE6UCM93jHc3SF8G+hyIv/U/paffZnj4Q9g4p+ZQb/GWqzRWUm4oJ/rCaObq5woZzjOwZfxlZYlAhDM/lsKkQg79cIh+s/oRNy6u3lvpd3s/3jv0Hv4K6FyxFwr3QHyt1KefXwueT8IlCoWChIQWFMGoQ5eQGMp2beCOCY2fO+yRYQ4/D7hzQJPGyN6XTdauLLpM7kJxsReeqjK6aX+CwsCW3LJEWyJX2fXxfz7yM3kVeUyJn05e9ADygGoj1LPOaRUypZIjw2/FpNKgDQ9C5tvL/YNINAun+eXAc3DsTbgsDZk2Cl+NL4VVhXjoSoAoCgtFRtbB7w+i1Cjpfnn3+rpuEJWnCv9YYeA/UXCCzwrmUxreEYXv0xDeqfUPJtF21GST1+VMx19gYG3dhKwsiIx03dWQB4Y4tam8VMSMikEXpWP51jfJiTqGXjMEQka67REkzg5NWr/E3QSho+zfK1dSnwalBq8QL5f1Wba/u53DPx3mikVXoIvUcW3vaynddC1/FNU6oMlcCflbocu9oJG8gf8GtFpQKMDDKD5PW6ZnU0lJtaBXCadQlC7K8WDGUthxN4xZYw+ik2gBMdeJ+jYu5PGiIz0xlhZhtgrVml4hzVsPFhwvYPWc1SRcm4AioX77S2aZCEj2MERQbgjAHHmlKKL2H2L2ww+DTMaIOXMYpVJxg15PKCLD7CuNhrXV1eiANRYLgy2OmTTX6vXCyfTEE6Kvhx5q03utNFaS+FEiAMWPFeOraf6Htfm1zax5fA13H7yboG6u53uFh4JTpacAiPev397R1ixdtIgH9XrGAs8jHHqDgdk1x0cAo4AbwP6ZfQRsVyiYP29e6z8PmQIUnsLGdK6Q1Z+LEhgIFlk1+2Vfk3EkkI+nfNziYcL7ulYhUvuoXbb/dnQFx7s9g0fhjVg6zkQhKZide9RBED4RfHs6HbJarXx54BNOxh8nOvlpcnK88fGBnH056Iv0BHRqoBZmE5DJZTyU8RBLfzbD59A1cAusuBcSnnHptJY4x+Rvhe13QM8nRHBBHXoE98BT6UlpdSmmyONAt3odf5UFlaSsTyGsTxj+cbXz5A1/3+AQtGpTqAny1SJT+Yg51cu1ZPW58BlJGX9upq3k0c4KVTmQ/A2dlUJs36AssEfgn4lJbyJ6RDR+sX4O7QWVBRQaxSaqi39POP4O7J4tOf3cidUKxjKHJm8Pb7QqYR3JLs9GrpS7dPoBhISIDLDq6trsvLrEjo5l/KvjCezasLNmxNwRbBywkVwz7ByxBfq+CkgZf22B1WqltLS06fOLoQj2zRWO9xpsEVBtqdaStCKJ3279jeKUYkpLobQ6mB9kJuj/ZtsNKtFyyk9D+WnuW3Efdy6/k2x9sj2Cvq42vjupqpZT4R9FtXcQ3t5tM4ZE83CaX8LGiUWyQkic2YzwCq1YENgy/v588E82v7K5xeOWZZZRnFKM1WIlrTSNnebPyAr4CZ1P82W/JM4yxnIo2g/VtdJ6Zzr+5HKIFwISHG9mOaWowVFc+8e1xI2LQ28RErMB3lqwtKAYlMQ5pUnrl4C+0OFye82WeWPnsemWTUzpOsUu9WnyDeL2nbeTMNPZiViRV0HuoVyHde++feLfXjZfQuYKOPgsWJzloSTOI5IWwtqJYKpEJhNZf54GEcSUUtK8omQZ6XIm7inl64EnCPc+wyAcNBz6vQ66lgW2SNQQcgHE3eiy1l+HDqCtFkax5KLkFnV//PfjZB3IcppfjvxyhAURC0hamWR3/CkrheSN73/M6Wdj9kMPkZKezoi5c3mze3fujojgre7d6Xf//cjlcv4wmxlcz7WDgWXV1Tw+Z477ZT8Ld8O6yfYadDq1zv73eKzgWIu6jBoaRacJnez16uuy9c2t/PP8P5iqTZwsPAnALM8S+NFbBDGfZYqKiwnDdZbfbCAF4fx7EyEBejfgM3EiKenpTXP6Gctg7cVw+FXXx+NvgUsOgPe5c35iMYo6fy5+/0FBoK0WwYAFVQUUVbkwXjUDq8WKodxg//+jS49SeLLQ5bn5ZcUAGEwhKC74Bjrf2aqxJdyAXAkjf4POdzgdkslkzP17Lkf8X6NCc9xuy+h7S19u23EbwT1ENkthUiHrX1xP7sF6PD2NcLosnYMdHiBZ+SUYCsAq7U3aJUofMJaCucr5kFxJ/4j+AOSqRNaeqxp/IEqT/HjFjxxf7riZDU0IJaxPbba1zW+i0yG+n5P2g8q1setc+Iwkx5+bsTSkZ9TeKTsOW66nL3kEWXqiNoaT9PtRfpz2I6ZqxwktvF841/15HV0ucdQLP5h7EADP6hgig3yEwX/E0rP1BP8N/ugNq4Y6NduMbjnl4i2XvjWdrD3Oep5KJQTXZHGeKfdprDKi0qoY+vBQfDs0vDMymA1klYkOOvrH2etgSI4/92OxWDh16lTT5xdzNRyaB9mr7E22qPeMLCNVRucXYEPs+GAHb8e/Xe/C2Eava3ox84+Z+MT7cH++ij8T/VF4V4DCdSSdxDnmzwGw9WaH7IpwZR7xO35g/w9H22TIkmIL5R672dF1Aj/9NhrWjISK1DYZS6JpOM0vkZdAnxdBLYI/fNXiXSDT1Dr+rFa4YtEVTPmk5RGOq+es5q2YtzAbzFQYhHNHbtHSr2QiLHEhNSDRfshZCyv6ONTntK1BcityMVuE87ZzZ3EsKan+rlLWp7D46sVk73VtTKy2CK2eLl7p8L0KDs13wwNInC2avX5BSHUO6zCMIG1tgEhZWf3nj352NHPy56ANEgFwJ9ILSM8wI5OBPdkw4WmYdECSED7fKTsBueuhWlhodDrQVguD9eni0/a5pymkpYHCqmFIl04o5GdEX/v3hm6zQRvhtluXcCQyEjwNMYD47JpLQKcA/q/y/wi6IshpfpEr5eiidHgGetr3qqrqcG7p+zChG0OEXNx/kLCwMOY++SQ7Dh/meEYGOw4fxtvbm9EqVb1OPxuDgVEeHnz6ySfuvSlztdivVpy2N3UL6gbA0fyW7UU6Du/IzOUzCekZ4nRs31f72Pv5XhQeCk4WCcef2icegi8Q2URnkTcWLCAnO9te22828H8IR98E4FvgIBAN+AOZgF9ICH+sWNGwvGddFBoxZ5afcvftuw+LEf4eD0ffcDoUGAhKixdqg5iLTxSeaPEwhnID833n88c9fwBQnl3OD5f9wJbXt7g8P7+8GAAvpV+Lx5Q4u3QN6gpAubrW8ecZ4EnEgNp3eVFyEWufXMvpf043q+/MXZkcW3aM1PwTnA59m4/LNsDU0yJwTaL94dcTpibXW4NxYITI4E017QDqd/yF9w1n6hdTHfweJr0JfYnewYG3JXc1OzpdyhZZPUEWdTgXPiPJ8SdRi29PGPEr/QY8w0Oag/RIf4WCo3kcXXq0yRERtgWaT1UvAgMRWulhY9rwpv+DRE6GcGe9xlAvkdKVUyHecl+P/5q/Hv/LZRd15T7rkrohlbdi32LH+zswG80uI+UAMndm8vnYz4lLiiPaw4Pg8qNgKAZqHX+S1Oc5RB0Mk49CwrP2prAwONDxXp6o1PDejvea1Z3CQ4GHjwfGCmOD5/lF+9H54s4YNUYsmPDyKKaTZpfIJpZof3R7CGKudXD8BQZY8Ms9Rs6RgoavbSFpO7IZ9fMyOu+tpLD4GJQcBqSM8PaMv6eI4jCpxOReXQ2VlRA3Ns6lUaWpdJnchaGPDEWpUVJprARAafbC5DccIi5p/Y1LtB2+PcT7xb+2PmewVzAyZFisFvIrRT2nTjWKrScasNOUZZVx6IdDFJ2qjeLe/+1+1jy+BkOlAQPC8eejC4KOM0S9Gol/F+nL4Cd/SF3sdMgWqJaXB3u/3MumVzc12l3C5x1Y3l9FcOfT9oxB1AHg10tEi0ucv/R5EaZXiDrFCMefxhhBiLoDAyIGUKwvblI3paW10dlRUQ2fK9EKDMWiftjOB5wORUXVyfgrbn7Gn0wuQ65wbcrqOqUrt22/jciBkWSWi4w/jSGCQn0H8O16bmUN2xlLFy3ien3TMqFvqKri10WL3HsDQYNFfbc62VRdA4Xx/lh+yzL+6lJZUOnw86xts7h+1fXIZDK7488SPR1GrwSvDq66aBPeWLCAl+bOZSa1tf3AOcvvbuAtYCQwTKPhjvvua95AchXMqIBBH7g+vv8ZSPu1eX26G6UWBn0EnZxr+Gm1oNGAV7WIJDtR0HLHn4e3B50ndSa0T6j956mfT3WpJABQWFkMwLTQY7D9LgeVC4lzyKkvYPP1YHW2U3YJEM6Zuhl/ZxI1JIpZ22eReGNis4bdvXA331/6PUVFYo/jo5LeI+cziWGJAKTqRZmyvDwR2HwmngGeJN6Y6CDzmfRnEi/7vcz+b/bb25LLjpLjt4ws6zbYNRtSl7Tp/TcXyfEnUYs6QBS79o61y2CoLxjAnII5RPSvjZKoKqzih8t/cEp3Bbi9/+3cnJ9Nz7Q3CQ2xgqnS6RyJVpL4EvRb4NRsq/Nny/gb/9p4Bt7rWos8oubjPNPx5x3mzcB7B1KWWcYrAa+QusF1Jk55Tjl5u/PQ6DVcG+iH7K+RkC+ipaSMv3aAXAG6ruJvuoawMFCafbDKLKQUN08Oqd+t/bhz752E9g5t8Dybo7jcIAy1gzxUJOaNEbVSJNofPZ+ATrPw14g/1qKqIsJ7h7Br0pNYhw1vkyENqEmL9SUnNIfv6QTT8s7qRluiCeSsg78vgtwNAARpg1Ar1BipxKdGrcu2mTJUGKguq27RML2u7sVFr14EQEWN409h0WLu+QwM/aIVDyDR5vjEQ8L/wD/R3qSUK7lzwJ08OuxRe/aMLePv5Ekw15OI0/3y7jxpeJLuV9Q69I4tPcamlzdhkpmwyITihE9wb7jge7FOlfh3oQ6CgH6gEpuPQ7mHeGvrW/xy5BdCamILcnNh7+d72TTf2fF36MdDnFwtDLgVhgqqLVUgszKoV53sjYpUIYMucX4jVzmUj/D1BRkKPu2VypZbtxCobVpN6bQ0SAv8kqPdbuKv1OXOJ2y/C1YOcG0Fkmg6Kl9Ru9OFoy0iQigEAZzIP92i7rN2Z5H5V2aD59ikPtXGCNbnPIhs/AZJiaQOJaWlNDUPOhQoqa8OTEuRyZ1Kwtgz/gpapz6y9NalvN/jfYxVtYGrCpWCgE4BWK1Wu9RnvH98q8ZpLtnZ2Tz+2GMsq67mWWAttfKeIGQ/5wI7gOM1/44DthiN3DprlvtuxFQhJLDTnINuzjqdbhfSwGcgkwm5Ty+9WFC2NAvUxpU/XMnQh4RyllqnJvGmRDoO7+jy3JJq8V3vq8uApA/BKpUiaBfkb4XT37h0xHYJbNzxp/ZREzkwEg9vj2YNO/CegUz9fCp5ClHMbbyPGU59JTJWJdonuevhwLPgQg2iR3APAFLLTiKXg8nkugyWDbOxtg+fcB8Sb050CIAu1AuHcEetDxx7E7JXu+cZ3ITk+JNwic3xV27yROPrWJA792Aux5Ydo/h0sYsrZZRmheJVHU9EYB786OUyyk/C/ZyZ8TfgjgF0ndLV5bn1ZfyF9g5l0juT6HZZN8L7hderP9zlki5Eb4jmUM9D5HjGQb83wVcUUpEcf22DRqNp/KS6VGVDZYb9x7Aw0BpEhPTJgtNuvLNavh7/NW90eMPu+Dul96Yo7m1RK0Wi3VI34y8sXAYyeZvV+LP4B7JzhB8nupzA20Mq8tdecJhfjCViU1Ujpfb15V9TNbeKWf1mERkpTsnMhJQNKbzk/RJ7P9/b6vGLK2odf7Y6kxLnH+9f8j6vjH+FIK1wuERGgqcnGAzC0O4KhYcChcpRZu/STy7lvqT7qLLWylKH+HudeanEeUKj65fgoTD2LwgfD8DW9K08+OeDfLb3M3t94rw8mPTBZG7bcZvDpVVFVfx83c9se1OYTXPKhUKJ3KJhUGKd78xfY2DNKLc8j8Q5xFQBmX/WqAVgf1801xeRlgb5utUkeX/JgdwDLs6wiGwCqUZ965DJYOJO6P2M0yG1GiI8RcZfUn7LavxteH4DOx/f6aRQs/6F9ez7ShT6vK3fbVwdcz/e+q6SGo0LfHU6mip8mgP4tkWRxNyNog5rDa2V+rQR3jec2LGxVJdWYzaa+fXGXyk4IZwFORU5VBgrkCEjPvtnOPRSq8ZqDp8sXGiXV3VV2+9MtgFT1Grmv/JK0yU+65K7HtJ/c26Xa2DyMUh4pvl9nkUCA8G/QojRLjmy5KzVxyqtcfz9VHQ/TCs461KwEvWQOB+mV4LGuSyEXepTc6xBW4ap2kT+sfxmDRuaEEriTYkU1wSRTddmwdabkFwq7Zj0pXDgGQcpaRsJIQkcv/c4Jx84SVDNn3ZuPSKHqx5dxTztPPQlIjs+clAkUz+bSni/2vrQxQbxffL0ioSpqe1uXpW+pW5GoVA0flJ7xWyAn/wwbJjB/6X3ZHXvcAqLTZRnl7P70912qYToEdE8Xvw4fW7o49RFcbGQAZPJIDDAAjHXQaDrrDOJFpK3SaS3F+1zaL6408U8PvxxLux4YaNd1Of4sxE5MJKb/rmJ2NGx9fYRHxDPnQPuJKHz1dDtAfDqgMlUW4dFcvy5D4VCQbdu3Zo3v6zoCxun23/09IQgZQwApwqbl/FnrDKy7Z1tHP/dOcu3LlFDoogbH0dxlXD8ZVTrUPS4T2hsS7Q/DjwLf41zdPyFgbY4k8KtrZfXcUVZGZjk4vsxRFnR7mQQ/os4zS9RU2F6KXS4AgAPhQeyGuOnzfGXng6BXQLpdXUv/ONbNtn/OO1HVs8R0XCFZaLGn4dMi8fBR+DYO614Iok2x6wX9Tn3P93gaTJZ43KfFpOFtM1pDpLyap2agPgACivEXCG3qAkyrBNZOGUn3fEEEmeJlqxfbO+koqoiAgJAoRCRuPLgIAepHQCNn4abN9zMiP+NAOBQighYUJtC6NmzjtMm7iaIvaFVzyLRDqjKgnUT4dSXQK3jr1QE4DfZIJyWBmUa4TzsHuRCPnjQR3Dx7lbfrkTDdA7sjF/5YGLVLbMVDJk9hCu/v9KhRqPVamXDvA0c/O4gAPcOupdZUW+hNcQwrdP/QfI3brn3fwtTZ87k6yYGl37l6cllM2e6/yZ23Q877rb/aDPeJxUmYbKYWtztoHsHMW3RNLxDvSnPKmffV/s49KOQddOpdSyfuZyFUxaiPP01JH/VumdoBmfKq7qq7bcG+AYYAwwH/m/ePGY/9FDLBtz3BGx3ltEUCkFdwKdTy/p1J1tvhd9cZ14GBUF44XTUMi+CvYIp0rc8ez/3YC5LZi7h1JpTrHhgBe90fgd9sWup2zJTMQD+XsFCRUkmmc7bBR5+oPR0ecie8ac+Tk6utd6k/aU3L+W9bu9hrGxatp7VYsViEgEmpcZCAFYoR8DIZeLvSKJ90uUeuHgvaJ3VpdRKNZ0DO6OUK+1lBepz/AV1DaLL5C4Yygz1DlVqEkElIT7BQs3Ks/4gjXPhM5IKHbiZc1Go0W0oPCBoCAq/BNL0P4IH5JYWceKPdJbNWoZKq6Ln9J4YK42ofZwlMjJKM7j2pzvIjOjHhabnUPqEwbCvXQwk0SqqskR6e+QU8K91vk7tNpWp3WolsP5+6m8OfHOAe47cg1Lj+Kde1/FntQrjXFVRFV+M/IKB9wxkwB0DGryFlPUpBJUG8f6k95HJaw0rtohbhQK7JJxE67FYLBQVFeHv749c3sRFZ+e7nKR1ov1Exl96eQpWq9VuzG8MuULOyvtX0nNGT7pM7lLveWPnjQXgx90ialNl9pW+B+2Z8mQoOUCgr4iiLNYXExoKUUf/wqs4HYvlceRy90a75+5Mpf+mIvLVoVzZ9QRsuxU6TnPrGBLNoznzi60WUkYGeId6M+27ln92WbuzMBuEbIYt489T5SXkdIKGQ9dm1jGROHvI1VB6VNSGroPBbCCnPAeVQkWYt9jwdO4MBw4Ix9/48c5dmY1mPhv+GQkzE7ji2yuwWqxk7c4isEsgWksYFxzejlVZhaZik/hudHKjzJVEm9Ok+cVYDkdeEdKxHa4gwFPIlBdUFSCXizp/2dlCYlhtLCP/SD6xY0RgmkwmI2pwbZG2XYeF489XGYxHXRWnXk+2xeNJnG20UcIpFyD2KTbH38ain3jt7ccZ1mEYX1/e+N4zJdVCuUZkE9nkniTaiJx1kLUKejzqtC/pE9WF1GVbubyFpVujR0ZTVFTkFMp+3/H7MFXXOoxKS0EuM3FhwMuQPg1ir2vZgP9CZt12G88/+yzbgMENnLcNWGcw8KU7pSZt9H7O4ceOvh15edzLdAvq5rbsLrVOzdTPp9Jzuli3aFVaJnWeJA72vFxkE58lXMmrzgauAT5F1PYrAXyBXkDH0NCWO/0Aej0N5irndr14X6IOOvfZzZpg0HYUiQgKRwnGwEBQWXQ8F5jMnJucs7yag9lo5uB3BwnvF46nvydqnRq1r2vp3+t8FrJmSw4DxlZDyVHw7daqsSXchKkKSg6CJsRe79dGvH88cpkck7KUcksuRUWhBAQ4d9Htsm74xfphNphRaVWNDpm9L5tPBn/CRQsuotwsHM8Gny4QKdWkb9d4xzXptPBwOHQIUl1XuaLfrH70m9XP/vPap9dSkVPB5A8n29sqLDVSnzotlJ8CzwhQuA6qORc+Iylswc2crdTzNmP0ShQJT+Kj8gMgtzyfLpO7MO37aUT0j+DD3h+y6pFVLi/dn7Off7KWk+W/hJaoEEg0kcjJML0coqc3eJrZYCZiQITL76Tt86mogHIRUE9peinVJdUYK0TkS9aeLBbPWEzaZmd9ro3zN/LTVT+BDNh+p70Ohk3m08/v3K8f/01YrVbS0tKaN78k/M/JcN4pSCyOKkylFOuLm9yVwkPBTetvYtzL45p0fmqh0Fa4zd+EbEVvuySTRDtj6BdwRQ5X9rqaxVct5o4BdwgDa/wwTveeSnEblEMqO5VH3HELXhVe/Ok1DIZ+6f5BJJqF0/xiqhCSQMVC/mxX5i6mfj+Vu36/y57xl5Hhuq/ynHJ+v/N3qopcGBjO4IHkB7hm2TUAXBUxh7H70hlleU7IYwz7ttXPJdGGyGRwRQ4MfN+h+X9r/0fHNzsyf+N8e5utzl9SkuuuVJ4qJrw5gcSbEwEoSSth4cCF/P3U31RXqPGrHEisfASy7g/DFbng17stnkiijWja+sUCB5+HjN8BCPYShr28CmGUtMl95uRY+WLkFyy+ejHGKiMVeRWkbXbs+0CyCNcN82mdcVCinaLQiFpQAcIAY3P8VVcqOVV0iuMFDStT2DiSlYJFUYWHXE2svwt1k2PvCGeVROvJ3QCHX4Ly006H6gYTtQTb/FLXiCaTydBF6QiID6BYX8y+7H2kFeRjsSr4Tp8K/V5v2WD/UsLCwpj/8stMUavbTmqyMSIni/9qkMvkzBk+h0u7XopK0bhRvilo/DQk3pTo2sivDjir9cbrk1etr7ZfoCvPRXMIHw9Rlzq3H3kFfg6B8nagpJA4H8atdXL6AXYZPkNR69/roQmhPF7yOMMeGcaoZ0Zx+67b6w2EjjSOwi9jEpd43g0brmj12BJuouw4/DnInvlfF7VSzYabNzA9LQMPU0i9GVw9p/dk7Itj0fg1LdtZJpPRaUIn/GL8qbKKjL9wnzaQPZZwP6ZKh/JHdVl9cjXX/nwth33fAODIkYa7smV9Jq9JJmlF7cbWaAS9XDj++itPiuzlDBf1o2s4Fz4jyfEn4ZIAjSiOXlBVgFeIF71m9CKgcwC+HX3xDnNdl8lWI8GnqpdwLKX9Cttug4p6CrtItAyFBpTOtW5MFhPJRcnsydoDwNCHhnLpZ5ei8lRhtVopyyqzn+vhUbuIyqypiR6aEMqDKQ8y5MEhgHAcHvrpEFm7nfVARzw5gsHvDqa0ukZbBxnIZFJ9v3ZOx3AtHkZRhDalpHlyn9EXRuMX7Vfv8ex92Sy9ZSkZ2zPwMAcSVDqWADoKSTiZlFzenkkITWBaj2n0CumFUgnqHvEURfQgJ9f93ntTQl++u7mClOgU8rRdhKykRPuiOh/WT4VTXwBQbijnt2O/8ffpvx2kPq1WOPjDQb6Z8A2V+SJjb9XDq9j10S5SN9YTMlcPhgotnsZIwnRBwgCjkeponI/Ysvyyy2tNWjapz+RksTFyxZAHhhA3TkRlKjVKRj8/mq5Tutrl+3Q6hDFIEwxy9xgBJdoRSh+45DAkvgJAiJdYpxRUFWCymAgRP5KbK2PsS2O59JNLUaqV7P9mP58N/4xjvwlpaosFkjKFszA6OKS2/4oUWDsRUn48e88kcVawOf5U5WL+OFV0qtFrKishTS8C0roEdkUpP2ONajEK6cGTn7n1Xv+zdJol6oj5Osv+2xx/qelm9CbXcnsNkb0nm5VjV7L9re32torcCkpSS7CYLaxNXkviR4m8nj0FkOHhG3VWHTznC7Mffpj/mzePEQoFEzQaB6nJUXINF8gU3PJAK6Qmm8pZNIYuPbqU7w58R0bxaSg+CIY2iHash3YhrwoQNAzibxOZdu0Ym80qv6YkW25FLvtz9reoL7lSjlrnOsPvTGzla1I8Z0O3Nv7uSzQdrxjoMw/CXMiIAMM6DCMuOAIZsgbr/FlMFkpSm1YgOCwxjGuWXUPEqC4knP6IMYf2MzvzQdhyY/PvX+Ls8msH2Ox6Dj1dfJpFBxZxzPQnAEePitICrji+/Djvdn2X4pRibt54M/ccucd+rLQULLJqALxDekP3R8G3falJSI4/CUeOvwcHniNIKxx/xYYCzEKJC5lMxsw/ZjL62dEuLz2YK7T0dVUJwvGXvxlOfgLWpmknSzSDwj1QuMuhKbkombi34xjxhahz4h3qbZdk3f7Odr4Y8YVDdIEtYO/MOn826c6I/hHMyZ/DoHsHOQ0fNTSKy3Iuw+9lP07EPwwTdwCixiNIjr92waF5sM5RfiAsDMKKp5JguQ61ommLXhtWi5WK3Ip6I1Ry9uWw9/O9lOeU089rCkOOr6GsZAVMOS7qB0i0P8qShCG0usChuTa7og2GrJDTOe8VVo4y8Nzo5xq/QOLsow6BIV+KGr04Zt+Eh9dIQ1eJ+b4so4yUDSkUny4GYMLrE5j6+VQ6X9y5wSH0xXoO/XSIghPiu2eTifb3NUDR/lr5IYn2S/ZfcPp7hyZXjr/QUCH9bTLB6dONd+sd6s2IJ0cQNy6Onen7SAp7hXz/5VB6AooPnVXDoMRZQiYD3+52h3+gZyAyxFo0vzLf/k7KzYUe03rQ9dKuyOQi+nr4Y8OJv0jUBTp1CjzKutKx5BomdB9W278+T2QdVUqBiP8Ktt4MS2MA8LUF3BeLrL38yvw6QYmuSUuD8pr6fr1CXRlm5DB+E/Sa66Yb/o/jGS72AS4yeSIj4WCHB/g0zJN3tr3v4uKG0QZr8erohWdgbb2nXQt38Wb0m2TvySazTES3as0RaFUlhHmfBHN1y5/lX8zshx4iJT2dEXPn8mb37twdEcFb3bujvWguo8am06VrGzo+MpbD4kBI/9XelF2ezc9Hfmb58fqzJlrDy5teZubPM9l3ahn8kQCH5jd+kZuYddttrDUa682wtGGTV721tfKqRxaI32/pGfXbO1wOgz92+bd51qnMhIMvQO56p0OBwixJfr5w2Ea9HsVty25r8VAlqSXs/GgnKx5Ywel/Trs8p7S6lE36T8gP/o2SoFskmfn2hIcv9Pw/CB5a7ym2deOZdk4bFrOFHy7/gTej38RqEfsKk96Esaphu3VpKXiYA4lQxCLreBX492vwfIl2QOe7INJ1oHn3YKEznlx+BB8fMBjEXsIVcqUcfYmeolNFyGQyh+zxkhIYdegI008V0bnzdOj7itjXtCMkx5+EIynfw4n3CPYRb1iDosAecQ00WBPM5vjzsTn+El+GaQWgja73GokWsnYC7HrQoSnUW7zhyg3lVBorHY4VJhXiHe6Nvqg2mrJunT+Aza9t5tSa2plOrpTjGSA2UseXH+fkKiEDYbVaKawqpMIotPCjdLW1VepKfUq4F5/mFssrPiQMXXWKooeHQ++UjxmS9bX9RddUlt2xjNdCX6u3AHafG/owp2AOcePiHLM0JNovGb/DphmU5m7jh4M/8MPBHwAIqkylz+rXOPbdbrcPWZFZjGdJFj6eVpS/xcJ6STqlPeAwvyg9Ie4GCOgLQLBWOP6K9EUgN9o3UxkZMPCegTxR/gQRAyIA8ArxIvGmROTKhpeXBScKWDx9MUd+FpoaP6a/yqEOs7F4/gUr+sDhl938hBJu59CLsPNuhyZXjj+ZrDbr78QJ1139cd8fvNfjPaf2nTmbORr1GIc1n8Lu2bCyv6Qjfh7SpPWLPt+uEKKQKwisCUDMq8izZ/zVDUYpTCrEbDAzbv44VJ5i852cLIKbbvBaxF2D7qg9OXAAzKiQIvb/LWhCwTseLCb7OlNfoiNIKxzHyUXJDV6eng7Vqlywyuge5GItLFdA8DDwS3D3nf83sVpBnwtVztFkAQGglmuxyo0czmj4c3OFLkrHpMWT6H19rQR01OAoBt0/iIBOAXbHn7o6gkGRy7igsBNk/tHyZ/mXExYWxtwnn2TH4cMcz8hgx+HDPPnUk2g0YWzd2oZxN55hIiO0Tj2ktclrmfbjNOZvahuH3MkiYdeICuwmauCFu84eagvOuryqJgT8+7Suj7bGWAz7n3IpsWzL+CspgQFhQ7FiZXvGdrvtsbns/WIvy+9czva3t5O2yXVAUGpJKqs1d3Ci62yaa4KROLccyTvCRs3/kRQ2n+P1qH/LFXI6T+7M0IeHYjaaMVYZWTR5ET9d9RNmo9np/JUPrmTza5vtWaAeWm+44Hvo9kAbPomEW+jzAnR3vf63rQFTS1KJ6yY+3MP1VCjqNKETDyQ/QOSgSJJWJlGSVpstarN9Bnn7OatItBPa512dxygUinN9C61j+HcgUxC0cg4ARmUBJSWNZ3CZLCYO54m/ErvUp0wmJLsk3E+fF8HD8Xfr4+GDRqlBb9KTU57jULNi4lsTnZy2ERGgqioh46QafQmsfnQ1va7uZZfbsmExW1j5wEr0xXoeSH4Ak97EexHvMWboGA5eegDPpPfArw+Ej5ekPtsIhUJBfHx88y4a7lwjy7ZvyM0VkljyZoR+xI6ORa6UYzXXv+uzOYqLSyyAnF4Bf0BSupARkYy17Y+ISaCNJMWi4eollxDuHc6MXjMI6qglW+tPmaF5WaFNQbZtKz0PbUNV/SD49wWdVCj9XNPY/BLgGYAMGVasFFQVEBkZRna2MJ726lW7jCw4UYA2UMuqR1fhF+PHyKdG1tunf5w/076fRmiC8CJuKfuB9NBdlKkHQcKzEFR/FKdEOyHhOVEPsg7h3iKiKKvcMcS2c2fYs6f+On8KlQKVVoXZaOb3O36nPKucmX/MJL9cLCp8PQIgdjwEX+D+55BoU5q8fvlrlJBYnCIyEn648gc8lZ7EB8SjOCMLXV+s553O79D7ut5c/vXl9i5s0vU2SWInpHXIv4PEWkeAzSBrMkGMbxz5lfmcKjpFn7D6jdypqdAj/VVmJz7HDUNcRPdbTIBVkhV2F9X58HModLoTBn3gcEgmg0ivGI4Bx/Ob7/hzNb/EjYuz72Vt7yJFZQTppd0oDH2UABeSoxL1068fqFQiUDg1FaLbIp47oD+Md8z06hYk9gfH8o+5uqJVlFWXkVshin9Fhw6C6LPn9LMx++GHQSbjwkfnMBwVsyx6QoFs4CuNhn+MRubPc5O8auz14r+6GMvhn0sgeiZ0vsP1dWcT704wcRd4xzkd8vER30GjEVSGECZ2msjvx39n1clV9Arp1eyhul/RHV0HHUHdgvCPdW20KtYXAxAm09HjdH/Q3AjdZjd7LIk2YsfdULDDrjpWl/TSdJbkzMcrsCtHjz6O1ep6+TfgjgH2/7darPh28EXpqUSucDSQVeZXsufTPfS4sgeKC+Fw1KPk+3lTWj0bnVqKcj+fCdQGEuIVQm5FLj6xR2HHQI4cgcsuc32+2kdN1p4svr34W4Y/PpxxL40Dah1/vr7AoZcgbyNcuMQhmKUu58JnJDn+3Ezd4tLnJVqRvRXvH4+/qSdKs84h468+ThaepNpcjcKsRVsdWyP1uU04p3QNS35JtIBOzvIGMpmMUK9QUkpSyC7PdnD8ucrUDFUV0vvvdynYFQIP384de++wy3zWRa6Qc+PfN1JwokBIh1rB7xI/qiuq6eobBXsehdgbHRx/Usafe7FYLOTm5hISEoK8Od66MwgIAKUSDGYjR1KL6BkT0vhFNSTMTCBhZv3Rz8lrk/GL9sM/zp9HT/Umr082r3p2h53boNPtLb5niTZE1wV0XfCuiY63bXI69gti8fBbUbYywNQVBQFdODlwA7uSbuPJSU8yKNJZSlji7OJyfvmjj6ihMHIpCrmCIG0QeZV55FXkERUVxq5dIuMPIHVTKlUFVez8cCen151GqVYS1jesQcefNlBLrxm1m/UKcxHIwNc3GhKuacOnlXAbIc5OOFvGX2l1KZXGSrQqLSAcf0C9kbcTXp9g///qkmqqCquQyWQUVhUCwvlM9Aw33rzE2aLJ65e4m8BisP84JnaM/f9tGX/5+WA2g8ZPw5CHhlBwrACr1Wpf42ZkgEFRSFiEDoctbskRKE+GkAtBJYXu/5tQq8V/1dXQwSuenWxvtM5fWk2CR1wHT3RqT+cTslfDukkw5HPxvZRoHepA6Hw3hLheE8QHxPK3HtJKTze7a4vFwsb3NlKdUc34+c7OG1vGH+XhJBUOwNBzAEh22mah0UB8fDbLflvIRcMXYbWW4qvTMXXmTGbddlvrs9HqoUtgF2TIyKvMI7ci11771R3Y5ohAz0B8Nb6NnO1+srOz+WThQpYuWoROHcROk5kTOm80GiX+/v5cNnMmX82a1Wa/WwD0OVByCCrT226M5qDwgADXsokymZD7zM4W64ChUUP5/fjv7Mzc2aKhQnqFENKr4e9TUVUxAD5mL+TWCjBXtWgsibZCBlaLkG4+o3xN16CuAFRqTlJSZiIrS0lERCO9yWVM+WQKMrnMyW6qDdLyaN6j6Ev0rN9r4FTYa4RqwGN3CnR/GPykYJJ2TfEBYa/udDt0cFaa6hHcg9yKXKxBR4CBHKqpKlFfrKBftB9B3YMckmV+OfUV2zp/j9rrSig7DjnrQF5/AP258BlJUp9upr76V+cNhhIoPcEzI57kDtNBovPvsNdta4iMsgy8lN5463vi7SXH2xtYPxW23tTGNyxRF5vRLafCWU7l5KqT/DT9J6rLRG2D4VMCyLvoWgqDu7Jho5ywPmH27Isz8e3oS9xYMbmpdWosV1jYPmg7YbpouGgL9HgMQMr4ayOsVivZ2dnNm1/KT0H6MjAU25vkcjB1WMuKfp5c8Yv7ohutFivfXvwtf9wj5HOKTVkYVQXsUD4Co1a6bRyJtsHfU/zBVpmqqDZV26N5k5PdK+tjMECBbxzrRv3JssyfyK/Md1/nEi3G5fziFQPa2rQZW52/3Ipc++bJ5vhb+cBKlt+1nF7X9GLYo8N4KOMhbvy7ecXOK63CwRPuJ708zjvqfG90ah0apYhuzCmvXYd0q0nsTU3FLpNTH9OXTGfWNlFLpbhafC8CvaTvxflKk9cv3R+Bnk+4PGQLWjKboVB8JZiwYAIzf5/pYKDJyIANPfpyxR4PR4PgyU9EZkO1VDv0X0HZSdj/DBSIz9gm9xnvlcjgyMH2NU192Bx/HTvWc4I6CKKvBh8pcNUtyOQw8D2Inu7ycPfwGAByqpObbUexWq0c++0Ym1/ZjNlgJnNnJh/2+ZBjv4ksMZvjT1klFi5SYGrzeWPBAt56PYqIpHn8L+MoH2Rm8uDRo2yYN4/oqCjeeP119wx06ithnK3By8OLTgFCJ3xf9j73jFGDTeazU0AnSP4G/hrjXP+ujXhjwQKio6LYOG8eDx49yg9VOXxozKeHvpy0vDxmzprF3CefdK/TryIN9j8NuRtr23ziYVo+9H7WfeO0FlOFmN9dYJP7zM+HAREiU6uljj+AkrQSyjLL6p1z8suEjF9mZRCGCUfrXZ9InCMGvgcX73Jy+oEoQ6RRarDKTFSpT3P0aNO6lCvkyGQyDBUGDi921HtUapR4h3qTUSiMnf3VoEn+HKpzW/0oEm2MTAm5/0BFisvDNrnPAsVhVCohKVxfbUgQCmd3H7rbbhsHOFS8gzzfFZR5nBBBYzMqGlQZORc+I8nxJ+HIwefh9y5QlWGXBUxzLX3twJjYMfw1voTBx1cSGlrzPU94Frrc26a3+5/l5OfwWycodtQ2t9X5q2tws5F/LJ/Diw+TuVNsguRyGHdHPJldR7NsWfMM/JnhmRg9jET5xULQEHvxUsnx145IXQLrL4VSxxSL6IAIrDIzqeUnm/XSKU0vZcnMJRz47oDTMYvJwqT3JjHgrgEYzAYqEZY5r4ALIGyM0/kS7YTCPbAkBN3pL5EhFifF+mKioyEscze6Pf80af5vKrbscbOinI5KSMj8GvI2u28ACfcxcikMfN/+Y7A2GI1SQ7mhnKiasq7pNUHCY14Yw6WfXUrv63oz+tnRDsWu62Prm1t5Lew1svdlY7aYqZaJDXZ3zTFYPQKy17j9kSTczJEF8IOniKSsQSaTcdeAu5gzbI7dAQjC2BoVJdYZrmon5B/LZ8vrWyhMKnRoLzWKn0N9AmDNaNj9cJs8ikT7Y2fmTt7c+iarT65GLodgEXvgUOevLhYLZGZZqVbmYcVKoGdg7cHYG2DQR6Dt0PY3LtH2VKbDwWchbxNQ6/ibETWHrbO2ckvfW+q91GCAQxX/sKnrBXybUY/BO3CgKH0RPNzddy7hgr5xItrMQCV5lc13zvd8qCcPpDyAXCWnIrdCZIwrxJrWXuPPEMEdAx5EveMq9934f4A3Fizgpblz2WAx87dFz7XAWOBa4E+9nvVmMy898YR7nH8Zy8S6wlxbSz4xLBGAfTludvwVCudSfEC8yHwr2tuGBQxrsf0+15vNrNQ7/j5Xufv3WZfqXDj4HORtcD4ma0fm4I0zYHkPkcl1BoE1r/SCAugf3h+AE4Un7Go1zeXNjm/yeuTrWC2uP/ecUtGvyuyLh0eLhpA4R8hlcjoHiMCdcs1xjjXTp7/msTX8dNVPZGzPoCSthE2vbqI0QxgxsovFvuSrMj+4PFsqTXE+oOsK0yvqlertEdwDhUxBuaGELl1EW311/mycmRWaWSWyyDt4OUsVtxfa0Uwv0S4IGws9/g8UWuJqvrfJTZTcz82R42EOsDsM6XwHxEiSXW2C3AOUXg6LY4BQrxrHn4uMvz439OHR3EeJHR1L8t/JJP+dzNixVjw8xGfc2ARXl9SSVABidBEixb4GyfHXjoi4GIZ8Cd4xDs1dQmLAKkNvqbDXN2gqB787SPbebKd2hYeCfrf2o+ulXe19yixKovy9W3r3EmcDlQ503ZCrA+0a9cX6YhQKCC88RNipzU2OkmsKJSUQs/dXpv48lE4q6JDxPRS7dzMv0Tb8ed2fVD5RydRuU+31s3JyRF2lThM7ETs61r4ILkkt4fjy4xgqDPX25xngSUCnANQ6NcX6EpCJjXe4txlKjzrVjpNoh3hFQ+gYsR6pw+sTXufl8S8T7hPu0N6rRtn1gHPsCDn7c1j18Cr2fbWP7e9utxdMLzeLDXaYrz9UpoFeytj615K0ENZOtKsU/HHiD2b/OZufDv8EQOgZdf7OJD8f9OYKLAohx2XLUgbAv4+Q+JFqtv07CBwAk/ZDvHDw+dYo9ZWUNH7p1q1Q4rmXIp9NHC/Z34Y3KeHA0TdFVpXF7HSoa7watUFk5J0sON3srrXhWnwifJDJZHSe1JnZabPpPKkzVquV/438H1d3uhNPYxQd/Y8jK3HxApJwSXZ2No8/9hjLqqsZXM85g4Fl1dU8PmcO2dnO+8Nm0f8NuCLXQR6tT6io1bk3e2/r+j4DW8ZfvH+8kOq7shB827bm+Fn/fdZF1wMmHYTOd9W25W6E9N/AXP9a/azT8Uro9rCD7LeNusE/gdpAYv1ESZtdmbtaNNQNf93A1C+mOtVzs5FX4/jrobEgT/6s/UiiSgiMpXDsHcj80+Vhm9xnheZYs20ZA+4awKT3JhHUPYjDiw+zZs4ae+JEWpmwgepU4eAZWm8NN4l2hEzeYIDDLX1voXJuJR9M/oAePURbc+ziALlG4fiL8Y2DtF8hf2sLb7btkBx/bsZVLbXzioiLIXEeOwtTePBEDzZ3HdWo489qtdZI+Yif21KOXKKG2Gth0j6x+a3DxE4TeXz444yMdq6joPHVoA0S9XbWPL6G7y79DjUGRo8Wx3//venDX9L5Eu4acBcT1JXwgwZSl6DXg77GDyk5/tyLTCYjICCgefOLXy+IuwE0jhr2HSPUeBpE1Ltt49MUfCJ8mFs1l/EvNywRml0uJgK1KZS+qZ1h1bCm37PE2cUnHsavh9jr8NP4AVCkF977jvdM5uCoezlyxH0RsCUloDRWoan0YKMeki9cI+S0JM4pLueXlB/h4Iv2H9VKtf14QICo+WKxiHobFrOFt2LfYunNSwHY99U+vpv8HQXHCuods88Nfbhl4y34x/qTWRMxojB74dN1GkzLhaipbfCkEm6l45UwanmTjWUJNSViXTn+YkbFcMumW5DJZay4b4U986+qJns8IiAQLk2CYV+55dYlzh5NXr+UnYC8jWAU3ptgrbDy2TKAbHX+cuuJV8rIAINSnKtRavBSeYkDVutZyeSQOIsovcAvwV6v0ZbxZ1cVsJhdKlqYzbBoEZR7CotOj6AervtP/hq23ynKX0i4h7IkKNoHhkKnQ2FhEFE+mYjCGZQWNs+IKpPJ8PPxo+hUERW5FQ7tMpmM+wffz+zOH6Ay+/HF6T/gkiOtfpT/Cp8sXMholapeJ5WNwcAoDw8+/eST1g2ojQJNkIM8mi3jz92Ov1fGv8KS6UuY3GWyW/ttiLP++6yL0lPUIfPwq2079iZsuNx9Y7iDuJsgcZ5LZ4pNbcSmRDP3wrl8MfULeoX0cjq3KcSOiSXxxsR6j2eXifXEqIBy5DtuF7WCJdoPVivsuh9SFrk8bHu/l3ruJzm51kbZFEJ6hjDw7oGofdQMvGsgVy+9mviL4gE4XXEIgFG6KCdVLYl2TOlxOP4eGJ3rTWhVWjwUIoi1JY4/i9VCoUU4TDoHxcGW6+DAMw1ecy58RpLjz800WLj+PEIhU3Cq7AgV6mPk5kJ5ef3nrju9jpi3Yvgi8xGgxvGXux6W9xI1xiTOGld0v4KXxr3EyBjXBdQrCyrZ9/U+xr40lsu+uAy1Ts3kmjXv5s0iYropXN/net6/5H26Ro2B6GvAJ95eC1KtFkZhCfchl8vp2LGjW+aXmBjQVovFi624eVOQyWUoNUqXx9Y8vob3erxHRV6FXWZWbQzFEHSxyAiRaPfYauLYJFMSLvTHqPHh2DH3LUxKSuDEwKv57NZPMVhB7dcdPKQogXONy/kl5Ts48D+XBnOZDHvWX0YGVOZXUpZZhlwlru8yuQtTPpmCLkrXpPFT84Qh0MPiL8np/EvIrcjl+4PfYzQb7W22jL/kZOc1pVewFx2GdWDgPQO5fvX1RPSPQK+HxJOLGHx8FcPj+p7Fu5dwJ01evyS+DNPLRSYpEOIlPH02FYHGMv4yMqBalWe/1r6p1ufCj95woB3VMZJoPYYSUbeKWsdfSYmVnu/3RPOihrRSZ53ydevE96TKq8bxF1yP4y97DSR9JGWIupMBb8OVBaAJdjokk8GV6o/od+p71MW9m9WtXC5Hma/kvS7vsfPDnfz91N+kb3PMzLFlgvr60mDNHQlHli5axPVNtJbfUFXFr4tcG+CbjMUsjLPlp+1NQzsM5Ycrf+DnGT+3ru8z0Kl1XNH9CgZFDhKlMbL/cmv/rljy1Vdn9/d5JtUFUFXnBdrjcRj6LSjOj4W3rSZrSk2Zrlv73cqNiTfay9y4m6lhD9D35HeU6B+GUX9AQL82GUeihXj4wti1kDjf5eF+4eLzMnifwGqFEyeaP0RVURU5B3LoemlXVJ5iPZBhFI6/p/yOwWpJDvy8IeM32HkvFO5u8DRbTfqMjKapSABklWVhllUjsyiJD4qCIV9A9zkNXnMufEb/Di9VO8JicdakPq/I2wx/TyCiUkS1GFUFWLE2mPW37vQ6UktSya0SKdDh4YCpCizG9qUb/m/CYhRRC2m/Nuuy9K3p/HrDr+Tsz6HHlWLDGxMjDHIWC/zpOlu+fkIugOGLwD/RQeZT2le5F4vFQmpqavPml/yt8GtHOPWFQ3NsbK3j72hu0zP+QMixJf/tPBkoNUpkchmeAZ5klNZk/BnDsAz4CPq80KwxJM4yB1+A5K95YfQLLL5qsT26tnO8GY+KIjJPVlHhJtXF0lKwyKqxyswEKcDHVAwWk3s6l2gxLueXfguELFANa5PXcul3l/LEX6K4vc3xl54O3qHezE6bzSUfXAJAWGIY/W7th1eIV71j7v50N5teFfWZOij7Mm5fBlMK1oqMgJQfhISLRPumIhX2zIHsvx2arVYrvd7vxTVLrmFH5g57e0CA+N7UV+fPbDSj8lQRNy4OtU5NSQnoqnoTqR9PhNYq3mUlzdRekTjnNHn9csbC8UzHX9My/sRBW7YgAOYKCL5AZJNI/Hv4cxCsuxiodfyVlckwmA2YLCanwDaTCb77DqxYqfIR80j34O6u+x78KVyRA0ptm93+f45G7AGxQrWvyeVFbFgsFio1lQx9eChKTyUbXtjAyT/F3mZ/zn52Z+0mt6gST2Upg0O/htJmFnv6D1NSWkpTRZxCgZKmWknrQ58Nv3eFI6/ZmwI8A5jeczpdAru0ru+G2HU/7Jvbdv0javudSEo6u7/PM1kaC1tuqP05cADEtDPVlbIkWDsJkr9xOtShpkRvSUnTDfKtwccUS0ThDJRVw7GETQB1YOMXSZxdQkeBZ7jLQ2PjxnLq/lPM9hV1LZsr92m1WlkQtoClNy11UBDIswob+SmfydDziRbdtsQ5oMO0Gge+60DSlza8xJBPhrAm/VeiRfxhk7P+bOtNT0M0Af4qoYgT1nDiw7nwGUleGTfjSlrkvMJcCfmb8bWKiCSLzIhZXt6w4y9lHQA+BaOAmqjciAkw5RhEXtK29/tfRaaE3Q/ByYUOzVarlZTiFNanrHf5XYwZFcPUL6aSMDPBoX3iRPHv2rWNKyJVGCo4kHOA0mpHw6xU36/tsFqtFBYWNm9+UWhBEwYKT4dmX18IUogCngfSm+f4W3H/Cn666ien9lHPjOLug3cjV8jxIpig0nH4VQ7EWyrx1/45/DKc/pZLulzCtB7TCPMW29L8bSfpvfZt/LOPNLsodn0UZVcTlLqVsKwwHvYDnxU9oeK0ezqXaDEu5xfvOPDtbjfG51Xmsez4MjambgQcM/4AdFG6eutkuGLPp3vY+obQvy8vVaIxRhDt00k4/TZdDVVurG0i0TYYS+HIq5C3waFZJpMxOlZoiK85tcbhWH11/swGMy94vMB3l35n/x7aVAR8fUFWdhy23gyZf7j9MSTaliavX6pyION3ex0dm+Mvr0Jk8TUl40+vygJwrC/pHQdj/oT4W1v+EBLtj/hbIeY6oG7GH8T5i/XtmY6/NWvEd0cTmEO5uRAZMroGdnXdt1zpJJMv0UoMJZC+tN7gjdhYsGLmUHITpWdqsFqtVForGfvyWIY8OITbd99O4k2JADy//nn6f9yfpRkfEqk7zkTdDZDm3syxfzO+Oh1NXYnlAL62YpstxTMcejzW5rajOavn8My6Z0gvrckMHfoV9JnXZuO9sWABL82dSwyc3d/nmXS5p1ZG32IWQeTtDZkScv52uTfUaGrXAamizBpb0rbw9ra3KaoqcvutlNUoAsplpee/ffffitUq1owu6lR6e3gT6x9L9+5iH9tcx59MJmPiWxNRaVWYq8324YYeX8uIQ/uJS3wBus1u9SNInCW8Y0VJM5VrNaLjhcfZlrGNg7kH7XKf/8/eeYdHVXR//LO76b0XkhAIhBB6r9KbSFNUVLALttcGqKD4syIivoj9tWDFXhBEkd57bwFCCQnpvffs7u+PyaaQ3WQ32SSbMJ/nybPJvXPnnmGX2blzzvmec0aq+2YUZmJd5o5DcQjGTtvNMadIx5+kOr5jYEYuth0ewM5K6DWWWKUTZUARsLC0kANxYvPONXMUCkVl8V1JI6JQwOjN0P/TaodLNaW0f789I74ZUVEXpSo2jjb0uq8XTr7VPTKDBokFVXJy3ZPckYQj9Pi0B/0+7yei/cuj5HSbdNLxZyG494AbD0HwHTVO9fAYSJv0mbRDvySsIQbPG8yEFRNqbXOD91QGXdjMDYXzURx6GK7WdBRKLIgJB2HgVzUO+3T1wXrEYAqdvU1eLBsiMzqbDid38mLqDzw1+VcIe0Zurlkq6mKxEV++MXBtvS1drY04PbXuS/JL+CjsI9Y/adhJc/uvt3PvFhF5XE2Gq/3dcMOv4BBgnnFIGg/nTjD1MoQ/W+PUmPZjgJqOP12dvzNnqrdX2ajoekdXordHs3qW2JiNS8vikt8y4r2/BpcwGLEOAm82+zAkFkLaftg5BZJ3AODtKOac7OJsisuKKzL+0tJErbZriY8Hh5J2TAqayah2o5rIaEmz0eV56LoQqMwGjY2FEDfh+DubWulg0mrh99/F777D/wKgp19P7K2rB8ZVkHYI8kxMPZPUTsFV2HWzqB+shyzn/azvY8/n6vpLp6msVfj39se1rdh5O58mFq+upZ1JzO3AAZvVEGhhNc0smGkzZ7JKT+2OJGAx0B/oVP76qpUVY6ZMadgNFUoh29dmYrXD59POs2zvMr472fAav0VlRXx8+GNe2/ka6QXldaj9xtSZnVFfkpKSWLhgAeuKi7kTWGXkdd/Z23PzzJnmNabXW9DpcfF7xhEhgX3pc/Peo6E4BsMdhdDtJb2ndZk4OrnP+9bcx9MbnuZQ/CGzmnEm5Qy/Jywnw2kvd4c8ifJ3Z8t0lF7vnF0Ka4Ig66TBJjrpxshI08s993u0H3MOz6kodVNYCJoSW1wKu9PWW254tji0WihI0HtKt3aMyowyuc7fmMCpTDiRQf9Lf+FaehD+8ILLX5rDYrMiHX+S6pRH9ysUCjztRUp7iVW6wYy/A3EHKFGX4GPfBsfijnh7g5UVEPcXXFoJ2hYufWrJ+AwHx6Bqh2xUNrRxbgNATFaM0V3Z2sLQ8metbdtqb3smRezYtXVtC1d/h8TNQGXGn5ub0beVNBNjQkbT58oPdMw1LQI+bGoYPe/tWe1YTnwOWxdtJeGI+CLNKU8E9fXIFBmpaQfMYrOkkXDtAg5tiEyL5Jczv3Aw7iAAbu3c6PX8ePLdg8zm+MuzciVy4D0E3NgTh3a3Q98VBiOvJM3M2WXwp19FdL5uE16XfVNV6vPaBylrB2tsnG2wdbE12L1LoAveXUSf2+LXERE0l1j7deLz2PZ2sDIsEyqxEFQ2IptKz3s1NmQsINaIeSWVBf10GX+XL1NDQnjyp5Pp93g/QsaJh6/LqXGcD1zAHofnwdYDAiaDc8fGGYuk+fHoK4JQvAYD4GbnhpVSbLakFaTh4SGeLzQaSE+vfmlJCaSmgnfOOL6f/gPzBs+rPHnxMzj1inweacV07iweXxMSoK/XcADWRq6tlj2cnCzahHfR0sa5DTO71bKpvm0MHHq0CSy/jnAKgUHfQvAMvad7dfBDqywl3yqa9AzT/69uf2k7//X9L2XFQj6+TFPGhfQLANjlhZNf6k6++y3g2rn+Y7jOmD1nDttLSzlY5dgKIBjYAzwD/K/81b+sjBXLl7Pi3XfNbsfh+MMs2LKAlcdWNrivHdE7KCgtINAlkB6+PUTmWyNmXqz84gtGWVszEJgNbIdq/576OAjsKCnhodmzG80uFErwGwdOFramUihqrRejq/Ony/jr16YfIILSzcnmy5v5LftZrviuIFMRDn7jZc1XS8R7KIT+x+BewrYr23jh+G1cCnid7GxIaqCYjC5Q1cs5DbvtfSDyw4Z1KGlaDj0CawKEAsE1VFWL0Dn+Ll2C4uK6u9XtfTrZ2WFlYwWuXcHG8qSBpePPzChaQ3GzhI2QtAVPB/GBLbVK5+pVUR/hWnZE7wCgu9NIFCjw04mXX/gYjj0ja/w1JlqtkEMrzat2ONhNhEPFZBvv+AMYXR7stmeP2ETRf0stnx39DICJHSfC1Esi8xAp9dmYKBQK/Pz8TJtf1CVi8z5ubY1TIeK7zeRaGvpIOpHEniV7SDohVlNZOSIUX20TCLdlQPdXGn4TSeNRkg0Fcfwa8St3/nEnXx6vjFDSRclduGCe5+LcYltyvUPw62p5i6HrGb3zi+cACH0crEXkvC7jL6MwA7VGTVCQeDbPza2c+6v29/CRhxnz5hiD98y+mk1JvviiOZG5kyu+73FVudO8A5M0PgUJoibLNYS4h9DerT2lmlJ2x1RKgXp6ijrQ+ur82bnZMenjSfR+QNRfSMzKAMBR6dF49ksaHaPXL45B0OEBcBY1iJUKJX/f9TcHHjqAl4MXCkVlZte1cp9JSeIz5eBATZmdK9/Bxf/J55HWRsIG2DYBMk/g6CjqlQO0LZqMrcqWSxmXOJ0iNIV1a902beCJQY9w9ZmrPDHgCf39arXQYzF0bMRN9+sRK0cIuVcE9+iho08QCq0KjbKEI+eN353VzS9ZV7LIT8nnl1t+AeBK5hVK1CXYWdmhyA5GqVDLwFQT8fPzY+nbbzPF1paDCKffW8AuYAMwCxhT/roL2KVW89aLLzbM+Rf1DfzTDfIqpaZ0tcdPJp9ssDzaPxf+AeCmjjeJ76Srv8GvjhD/T4P6NcTaH3/kniJRPscPWApMwbDz7yAwxdaWpcuW4VexqWYmzi2HHZPEHOfZH0b+3WiZjg0i/TBE/6j31LUZfxWOv0TzOv503x3Ohd1I9Xoe7TApEWyR+AyH/h8JVRA9pOSn8GfkH2T7/AvQ4NIlP5/5hZPBD6H2Xw0lmVCWV/dFEsvBdxR0fBTUhTVO6Rx/V7Ku4O0NXl5CXeTChbq71TmEXVwQQYxjd0LQzbVe0xw+I/kUZGaUylbwT3rwQTj5Er6OvvT174uTtStlZfolvXT1/YLUIwEqHX9934MRfzeFtdcvMT/Dn/6Q+G+1w21dRTiUKRl/ICS4vLxEFP4hA4oJu2J2cTrlNA7WDjzY+0GxkWIjdllkxl/joVQq8fPzM21+USjgxALxObmGkBDQUEZEwhWyCnKN7jLyr0jeb/8+lzdX1gYMGRvC4xGP0/lm4SWauacrG3t5ku18BGzcZUaXpbP7VvinO252bgBkFWVVnDr37npCT/xGfr6Q0Goo2Zkasu2O8/q5OzjyZ18ReSVpdvTOL20mQP+PwakdAJ4OnihQoEVLemE6NjaVcp+GpMANUVZcxnvB77FuzjoAMouEg8fT0QP2zIA/Axs6JElTsWWEmEP0oMv6M1Tn77ff4NSpyqCCw58crpD5BEjJFZ8LFysPOPcu/OoM6ebd3JE0PvVav5QzoeMEBgYOxNZKZA/rHH+62qI6dH+7B6ai1l4TpThqI4zfZ/K9JRZOSSak7oFCUdexa1dxOPqCMzd2FIXLf4sQUvPR0eJc+/biVaVUGZb5VCig89Mi81xifgw4bqyUVrgphYrN4UvGRyXq5pcpn09hxKsjKlRJzqWJuhVhnmHkZCt5bsid9IzyExleEqOZO38+LyxZwjClkueBdcDAKueryn7eA/gUF/Pcs89y6tSp+t1QqwZ1ERRnVBzq7NUZG5UNOcU5RGdF13MkInj5n4vljr/Qm8RBWw/wGdFo8vLZOTlUdd/NBV4AhgMTgB+ALcD3CCfqUOCFJUuYO2/etV01nKxTQkpbXWT+vs3JuXdg3yxRcuAaqmb8abWVjr8DcQf46vhXXEy/WOMajVbDofhDlKqNl+o8lSw+vy6FPWnf3qN17O9eh/T2E4GEaaqTaFFzsebHwyS2X91MrPdXXLCJg2lXoOsLZrBS0mS0uwsG/A/sawZV6Bx/sdmxlGpKTJL7vGvzMA51nITSTY+zxADNMafIWczMqPUVnmhp9PsIer3F+lnrOfLwEfr5iSWevs29gQED6eHTg6wTIwHo1av8hGs4+I5sCmuvX9x7Q8eHwaFttcPBrvXL+FMoYFR5WZTt2/W3+ejwRwDc3f1u3BWlkLy9YnGui8D2lMk8ZketVnP58mXT5helNUw4BL3ernHKzw8OdLmBTeEh/HZ0q8EuMgszeWPnGzz979OAkPCzc7ODKs/tVrZWeHfxxsHLAYCMkiRKrTJo46yAzFN60+klFkTQdAh9BHd7kapb1fGXeSkD5yIh7WgOuU/rfTsZuXodR05sxTE/EjKONbxTSYMxZn6xUlrhYS8yr3Ryn7rMYX1rg6gtUWx7aRtlRTWlArRqLYPmDqLjRCExlFMqokZ8nNzBqQO492rAaCRNSuhjEPKg3lMVdf6uVHf8DRkiXs+dg0WL4PHHRWDZsZXHOP3j6YpM0LQ88blwtfUAh0Ah6WMrs/9aGkavX0oyYW17UTvaALos9BMnqh/XOf5+cu2FzRs2HE88XnnS2klKxLZGgu+AGXkV9cB0jr8zZ+DenvdyT497GN1eZLNcuQJatBT47KBMo0e+RtI07L4V/mpv8LS/fTsAIuKNd/zp5hcrRytGvjKSbneKyBJdfb9wr3CysyE2uwtqt0GgVNXf/uuUufPmMXf+fEZYWVVz+umT/XwBuEGrpV+vXvXL/OvwkFAT8uxXcchaZU1Xb/Ef/GSy4VpedXE+7TxXsq5go7JhTEi5IoX/eBj1b6OtO11dXLg2f3UuEINw/r0HPA68D3gC3UJDG8fpBzDoa7gjH1S2sOcOiGp4zcRGIewZGP4XUDMjJjBQ7Ffl5YmA8z7+fVCgICkviYf+eoiNlzfWuOa1Ha8xcOVA3j/4vlG3V2vURKRGAOBc0A2v2IfRRH7ckBFJGpMjT8EB/aVrQj1DcbJxopRC8uwijcreqo1LWeJzEWTXtWEdSSwOH0cfHKwd0KIlJivGaMdffkk+JzP3kOK2Hg9nR6GcePo1KE6v9brm8BlJx5+kJkG3gO+oitoatckCLhu3jM96n6QsJRRXVxg8GBGCUypTnxsd184w4DPwGljtcH0df1Dp+Dt6tDJCVkdsdix/nvsTQEjkJG+HraMhaTP5+ZV666GhJt9WYgS5ucZn5lXg2R8c29Y4rFCAj007AI5FX65xXocWLS/veJkPDn1AYWkhIWNDeOT4I3QY36GiTcalDIqyRfRgUVkRBRrh6OvrehH+7Qnxf5lut6Tp6PQ49FpakfGXWVSp2zjr31l0WC4KwTfU8VdWBrk2HqS2safAoYCntUOEY1piEdSYX/KuiI2BmF8qDnk7emNnZUd2sfg/Xtva4NLGS+x+cze5CTXnLWsHaya8O4Ge94jI/Dy1CB7xdfWAXm8J+SFJyyB8nsiO0cPYkLF8e/O3rJ+5vtrxfv3ggw9g4kSwsxNOv40bYcbvM7hjzR3YONoAQlYWwMPeXdSFGrVB1ImStDiMWr+oHMDGo0JeGGBf7D7eO/Aee67uAaB/f3H82LHq5QcSEkCLmjxtMlq0+Dr5ihMl2SIAqazAXEORWAoKZbVaUBUZf9Ewoe10vrvlO0a1Fw81V65AtsMxXr86ik4fdqrd+ZeyB/7tDfHye8jsOHUAtx4Gs/46eAinYFRGtEnd6ptfdBl/HVzDKSuDH8+8hmLEGpP6lVSybd06Hqgy6dYm+7kD2K3VNlz2swo9/cR68UTSiWrHfzj1A/0+70dESkSdffxw+gcARrYbiZONk1nsqotpM2eyys6uxnE/YBFwGLhQ/pptb8/t997beMbo5K4L4iH2D8g8Xnv75sJ7CAROEXWkr8HGRkg2g9h3crJx4uG+D9PGuQ1j2o/Bx9GnxjWv73odgOc2P2fU7S9lXKKorAiV2gGH4hAC879FkbC+7gslzUPmCUjXv5+gVCjp6SvmjmyH40RF6S9dZQxarZboAuEFusFNKWSJ63DuSCyQU6/Azmk1DisUCkI9Qmnv1p6soizCw8Xxc+dEbXFDXMkSmyDWZW74ubpD4gY4/apFPndIx5+kTvzbFqBRlBiU89qwQbyOHQtWVkBRCvzmLCIwJE1ORY0/E6U+AYKCICxMaBo/9RR89BFklKttbLy8EbVWzch2I+nu2x3c+0DfD8BzAOfPi+c4f38p9WlRlBWKKHo9tHMVzrvzyTUdf/kl+dz5+50k5CbgbOMMYFBeZdX4VXw97GsAkvNE2qdSY4OVU2/o/rrITJVYPPqkPhUKRUWGRUMdfzk5kBHYk73jnCl0KMTRxrHWAu4SC+Dqr5B1puLP448cp+DFAoYEiZSt2jL+Bj45kMcjHsclsG6p3wKtmKPayAKxrQpPB0/u7XkvAS41JbTatxeZfo+LuALOnQP3EHc6T+tc0SazuIoErKT1o7KFiUeh26KKQ79F/MbcjXP5+4JwwoSGijVmYSFEVNnnjY+HYqtUNKhRoKjc/EveLgKQrv7ahAORNAlaLST8Cymihqi7u/76oaWlIrgg3vN7AAYEDKgIbNVLWb5wGGtr2emR1I/ey2DEXwbXfl0DhOMvqeiKwVrzxjK792zeGfcOQ3xERqi9vXAaSOpHVcnKJGAhNWU/qzIQWFdczMLnnycpyfiajQBE/wRXvq92qJdvLwAOxldWx7uSeYWH/36Yo4lHeeLfJ+qs/+dm54aDtQNz+sypPHjqFVH7rpGYPWcO20tLDdb003EQ2FFSwkOzG7G2aEG8qGWoshXZ0t1fbrx7mQMDc/C1df4+nfwp8fPi2XLvFmZ0nQGIfQyAyxmVexzudsY9Y+jq+zkVdUWpUHGq40Y0A1bWZwSSpmDMNph02uDpPv59AChwPUZJSWWigqnE58ZTqMlBoVVxg9sJOPBAhdS4pAWRewkyj4k90ms49sgxop6Oon9Af9q1E+uGwkIx12i1cOAAJF7zlkdlik0Qh+IQUeOv6yKYeBLs/Rt/LCYiHX+Smpx7F351gexz3PH7HUw/5Eqa8zYhlVK+ptJqtWy6vInYxGKOlau1TZig60AL7e8FzwHNYf31xbl3RY2dKouj7j7dWTh0IXMHza1XlwsWiMxNrVZE4c+fD0VFMLvPbCIej2D5+PIFsksohD0JTu0rHrK76K/ZLmkuNg2Cf/vqPdXFTzj+rubWdPy9vvN1fon4hZt/vrmiZuSVrCtotVoOvH+AMz9XOgP6zOlDn9liUZWcLxx/NqW+KN3Cofv/gVs3sw5JYmbi1sKOyfhqcgAh76ojNyEX7anT2ORnEhsrpFXqS47oHiuHPGwVMFKVAdnnGmK5pDFxDIY7iqHnGxWH7KzsqhWj1tVLSkwUC+OquLZ1xbuLNyqbmrJa8Yfj+Xnaz0TviKasDIoV4jMX4OkOEW/BpS/MPx5J43BlFWweDvm1P0lrtVqKymrWldFl6Vy6BMXXlHPJLZeA9XX2gEufw8mXzGKypOXg7egNVMoLKxSVWX+HD1e2i4+HYmvxNO7j6FPp2HHpBN1eAc9BTWazpIlQKGDP7XBmccUhXf3Qs2fFnHMi6QSP/TmfEwGPE+MjpNru7nF37f22mQDToiBwamNZLjHA8A79aZszA9e8QRWb+vVlaNuhPDvkWdrb9MfDPoHH+z8OSdvMY+h1SFXJypXAKAw7/XQMBEba2PDlShOdJqdfhYg3qx0aHDQYEMoyao0arVbLo/88SkGpyKrYEb2jmlNQH88OeZaop6K4NbxKXeJLn4nst0bCz8+PpW+/zRRbW4POv4PAFFtbli5bhp9fzdpTZiNxE+ycDOmHQWUHNhYabJd9Fn4rfx7Qg67On745IjY7loErBxLyQQhqjZq1kWsBcLF1IXauccXqK+r7FXTH2VkLNm5gVzOTUGIh1BbIQ2Wdv2J3keFaX7nPs6lis9OxKJRUx4eEHK2TYelqiYUy6Cu4ORasatZ5VioqXWNKJRVZf6dPw/vvw5tvwtKl1a+p4fiz8wL3HnV+LpsD6fgzM4rWkMFg36ZCPtLeyp4ybRmZLrvJzRW1EwCOJR5jwvcT6PFlezRaNb16iUhLcZEfDP4W2tfxcCVpOPkxkHMeyutwAQS5BvHW2Le4r9d99erS2xtefBGWLQNXV0hLq8z26eLdpSJypirnyvfvdROkxLwoFAqCgoJMn1+C7xR1UPREQfYNEY6/lLKoaqc3XtrIf/f/F4AVE1bQ0UPUxrmSeQWFQsGOV3Zw9POjFe2HvTCMgU+J+SIpTzwW2pb5iS8/ieWTHwtJm3FD7LpnFWWhKQ8kSDqRxIY5qwkqEzIGp/blcfyr42jKTI+Ez86GgHOb6RiRRxsVPF2yC6K+Nt84JPVG7/yiUOqV2amKqyt4eIjp5VppaK1WS0F6AQXpNaUucuJyuLj+Ivmp+eTkQKlKZHYFeXrA+Xch+vsa10gslKJUyDkHJRkGm+yI3kHfz/vywpYXapzz9hZ1gdVqiIysPK7VQlDsswyM3MysHrMg5le4IGustERMWr9c+QHOr6j4U5e5l1KQUnFM5/g7dEh8ThITxfdLkU0CAG2c21T259oFerwqpPElrY+BX0LXFyv+1AUSRERAdnE2fT7rw5fn3iXG539oFKVMD5/OjR1vbCZjJWSfg+MLDNZ3nhh6I3fb/UJw2hy9EuL6qGt+ycqCNs4XGBnwP8g+o7eNpG6qSlauBe4x8rp7CwtZ8+OPpt1s0Ndww+/VDg0IGMD3t3zPhlkbUClV/HD6BzZd3oStypbnhjzH5ns2MyhQBHgUlhby+s7X+b9t/8fV7OpBSb5OvtU/K1Mu1riXOUlKSiK/oAA7FxeGAiNQ8AOwBfgeGG9nx3CViheWLGm82n46fIbBwK+EMlfmica9V0Ow9RE1F+3b6D2ty/jTl7nl5+RHZFokKfkpHIw/yJrzawB4Y9QbQmnGCBbesJDvhx8mJHke3h6FtHPLRVGaXY+BSJqEkky4+ns1hZqq9PbvjY3KBkcH4faor+NPJyfsVNQFa/cQIUdrZdxnSmJBqGyNbqpLaPnmG9i6VfweFSXWFToqHX8dcHVFZIGWVGlggObwGUnHn5lRKlvBP2m7O2H0ZnANZ1jbYQCU+gsplTfeEBPm6nOrAXDJHooCVZVsP0mT0mc5TE8Ge1+zdx0eDr16Qakqi+2n9HxLbpsA2ydSVla5YScz/hoHpVKJp6en6fNL1xdEzSw9Xy6DOgnHX75VNMmpQvA8Mi2SO36/A41Ww4O9HmRK2BTau4loJp3U5z2b72Hql/ojoXVSn7alvrQt+Rw2D4O8aNNsljQtnR6HO4txDZrEp5M+5fcZv1fI5bTp14bbf7+dwBHis3Lgza389dBfxO43LmqyKtnZ4H31GH5X1KRp4AfnScIpLWl2DM4vWaeFLFA5a86vYepPU1m+r1IWyVCdv+yYbN7xeofdS3bXuF/4LeG8VPISXW7tQnY2jDpzkclXz9DWLQgmHIZB35ptbJJGJnwe3JoqNmkMUFxWzPGk43x+7HPSCtKqnVMo0FtAPS8P7ItC8M4dS5/gULjhF5iof7NYYtmYtH65/EW1DK4Kx19+peOvVy9RViAxUWT6ffRRedsQkfHn72x58jqSRiL4DvAdUfGnbi65eBEclG7M7D4TAI/c4cz32MEfM/6oXeYTIHGzqN2jbqDWpKQmhfFwbpnBekwAnTqJ14i6S7YB+ueXqMwofov4jQvpF8jOhoiU4ayISYT2xrqrJNdSVbIyGzA2L80XyM420WniPQTcutY4PKvHLGytbMkozODN3SIj8OURL7Ns3DLGhowFRA3Afl/045Udr7B492KC3wtG8ZqCrVFb9d/L2hkc9DuYGsqK5csJDgxkz5IlvJWayk+AL1rmoeBm4BUfH0YsWkRMXFzjO/0AnDtChwfgzGuwz4L/L9h5wdjtwlY9VM34uzau2VplzcRQIe/7V+RfhHuF4+3gzbSwmjW9DOFg7UCAoh/ORV0J9zuN277hKK98Va+hSJqA/BiR/X9VvwO/h28Pcl/I5csRWwCxPqgPiXlijelc2BUX53oWCpQ0P1qNkJNO3Fzj1ImkEwz9aig3fi8CxHRrytJSsLauLGd1uoqyrG5/1KG4vUh62DYWNg2u04zm8Bm1Ai+VZaFWq5vbBLMyLFg4/hKVhwjvXkxhIbz8Mny1/08APFNuwdUVBlVV0Yn9Ew48VKf0k8QMGHiATchNYHfMbuJz4hvUfefOcMlvKc9GdWXZ3mXVT1o5gpUjUVFQUgLOzhAY2KDbSQygVqs5f/68WeeXdp5tUGlt0SrLOHA2lszCTKb+PJXs4myGBA3hk0mfANDeXTj+dMVrA/oH4N5eyIOc+v4U3435jtRzIuPUx9EHv7zxuOX3x0GVDrn1DKuSNB3lsgY2Khse6fcIN3e+GZVSyDM6+jjS5dYudBnsCkBmv3EMe2kYwcOCTb5NdjacGvMMJ+7IIFcDl5z7g4d+GVpJ02JwfjnxIuy+pUJKOjY7lnUX1lWTUzJU58/J34k+c/oQNDhI7z0VCgUKpYKsLLArbUM7h67YqGzAqZ34kbQaxncYT2+/3hSUFvDRoY9qnNfn+NNFUzo6lteOtvWUkjotFJPWL/0/gfH7Kv7U5/izt4fu3cXvK1bAqVOidleXgWJTpo1TlU3cbePg0CMNH4SkReDnJ7LQdQGJK6eu5D+lVxgcuYMJnUfU3QEIWeEDD1SsjSRmxGswTL4A7Q0r0vTspaHQOo5Dxwv1iZXUQN/8suHSBmb8PoP5m+aTnQ1alFg5+VmutGELoKpkpQ1gbNW+ZMDV1dX0G5bmQnG63lOutq5M7DiRHr49eG7IcxXHU/JTeGL9E5xNPYuvoy8j242sOHfbb7eRW5xbvaPidJH5VpZvun11sGL5ct5atIhdajUbioqYBdwO/Aoko2UrkJudjYODQ+PKe16LVgt9VkD3V5vunmamTRuxLiwqgtTUmuendJoCwN8X/uazKZ+R9GwSMdkxPPb3Y3x25DOj7qFbg2rtfEn1fwK1h5QLt1icw2DwKmg3S+9ppUKJjcqG0FDxd0yM+OyYyrJxy5gVk0dI8jx6xPWADbKkVctEAQcfrKYuosNaac2+2H0V+xydOoGLC9jZwauvwojyZeSpU5XX2FnZYaNxw74kWDj+2s0SJc/qoDl8RnJVK6lJaQ6c/D+4+gehHqH4OPpQrC5mwoOHCQ+HZPV5ktTnUGisCVNM4qmnyjdmdKQdgCgZGdMkaDWiRlfChmqHH//ncYZ/M7xC27y+uAbFcsX3fTSKMsK9rknnG74ahv1eIfPZubPBeu0SM1BUn1VKaQ7svg0iltY4pVQoGWb7HzomLCLijIpen/XiQvoFglyCWD1jNbZWIhVel/Gnc/xp1BrykvPQarTkJuYSfzgea3trACaHTqPf+Y10SnxZZBtOT5ab+JZOaR4kbIScSINNOoUKx8+FOAdGvT4agKLsIuIOxBl9m5wc0FjZMiP4fbIWZPHskGcbZrfErOidX8KeFNl35Y6/inpbBZVP2ro6f9c6/qxsrZjy+RS63FYzDTzlTApRW6IoKy6reLh2dQU0ahEwVJpb4xqJhVKaA1f/gMyTBpsoFAoW3rAQgC+Pf1njvM7xd+6ckPwEESgQ4/UZif4rySjMgJyL1STNJS0Lo9cvrl3AJaziT2+H6jX+dOjkPnWSTbNmwaCQrszsPpOhbYeKg1otFGdAiZToarUcfQZ+cwO1kCpXKCrlPvfvB1uVHZlX2qFAQbt2RvbZ4zUY+a9F1mdp8Vg5ivrwemrr6Lh7bw+29gwiuuyg0XKf184v51LFg2m4VzjZ2ULqM9D1kt6yBxLjmTt/Pi8sWcJ5hQJjq/Z9Z2/PzTNnmnaj7PPwm4uQftdDbkku/k7+/H3X31irrCuOrz63mpPJJ5kWNo3Tj51m+33bOf3YaRYNW8Sfd/yJs61z9Y4SNsC/vWvsoTSUpKQkFi5YwLriYoN1EAcC64qLWfj88yQlGetGbSAFCbCuo1ivtb217vbNSdQ3cFR/FqSVlXD+gf46fxM7TkSlUBGRGsGVzCsoFUoiUiL49Oin/HXhr1pvG5ESwWN/P8b6OFFyQOUSTLz7QyJoQWKZWNmL8lIuobU28/SsLE9x7TOrMWi1UJDliLXaDY3ncPC+oZ4GS5oVhQJu+A16vlHjlC7ZIasoi8zCTGxt4eOPYeVK6NGjMuiwasbfr7f/yrRzmXjnTBB7GV1fFHugFoh0/En0oICIxRC/DoVCUSH3eShxN6+8AjY9RbbfIJ8xfP+lKwOuDXjo/TbckggO+iP9JeZEAfvvh4gl1Y4Gu4qMnGu17U2hVF3KxxeeR6MswiN3OD3tJ+ltp4vSlzKfFoiVEyRtETWY9PD+pOV0TljMiZ1tCXPvgqe9J2vvXIuvU6V07Ih2Izj3n3PseWAPABvnbmS533JyE3MZ+txQFmYvxDVYRHPm5YlrFApwcmrcoUnMREEs7LgRrnzP8cTj/HzmZy6kV2ZqfjnkSzZN/wzP7ChKMvK4elXUb/tx0o/8NOUntBrjNlIyk4qxz07ExaoM1/jVOP4TCql7G2tUEnPgPx7a3VWxAapvE17n+IuJqXTa1MWhjw+xatwqSnJLOJdykYiguZxx+AiKU2FtMJywzAWzRA9FqbDnNoiuvYbPpNBJqBQq4nLiiM2uLhXcrh04OIgIXN0mTlYWnA9cyC63OSLba313kYUjad1oNaL+UFkhUJnxl1+aT35JZVaGzvEH0LEjTJsG08On88P0H7i/1/3ihEIBE4/CDT83lfWSpsa+DXj0A3VhxaExY8TrP//AsWMi6EihqKwLVSeuXaCNrAPYaBQmQ+5lg6cDXAJEM5srHD1qsFmtnEsTzzydvTqTnQ1z+jzDVKvu9etMUo258+Zx7MQJ9ioUHKyj7UFgR0kJD82ebdpNnDtA8Exw7633tJudG88NfY4g1+r7TJM7TWb9zPX8ecefFYFq3Xy6sXj04mrZf5UddYfur9UqVV4fVn7xBaOsrQ06/XQMBEba2PDlSmPdqA3EyhFUDi2jLlnCvxC5wqDksi57q6pShA53e3f6thGKMn9FCkdfVx8REaKr02aIg/EH+fTop+zMFiUH3NxksECLoawQNPolOL889iW9P+tNbIfXgPrV+SsqEpKPAIqBn0Jf/YEJkhZAwGS9qlMO1g74OYkMbF3tPje38uBkoFs3sZ6Mj4f08oT00lIoLAQFCpHxZ8FIx5+kJlZOMCkC+r4PUOH42311N46OcNlFRGzfP+gWVCoDfdj7yfSvpkChgMHfQp/qXz7BbuIJNyZbTyiUEeyM3knvz3rzy1mxYRIe9zYXLlR5P3MuQsQStFlnpePPklEoYXqK+IzooXt3oZVfVAS32f+PmGdi6O1f/UHLxdaFzl6dsbcWEbrBI4Lp91g/lCrx9aFQKCoK1GZkiQWXoyOoUrca1FuXWBAOgTDgMwi6hbf2vMVdf9zFhkuV0a9t+rfBv5cf7Q7/RujhHzl/Xrzn/f/Tn6ELh6Ip0xh1m+yz8XTd/Tnqk2fA2gkc24G1ha+QJNXQbaRUld3z9xcSGCUlkJBQvf3+d/fz/Y3fV9SM1NHz3p5M+nQSdm52XMyI5Irve5zgG1Fwu/M88Bvd2EORmAv7NjDkxzplTRxtHOnp1xOAfbH7qp1TKkVNYais65SRpabUKgsADzs3CH8O2t5uTssllkjUt7DaF5JFPSYnGyfW3bWOAw8dqFAhACHp2LWrmHuefBLDzyKS1k2X52HMFrBxqzjUty/ccANoNPDOO+JYQICQgzUKWduvcdk8BHYbzjbq6N4RgCzHoxyrR1lXrVbLiaQTgHD6ZGXB9uh7iHF5Re5LmIkePXrw9jvvMMXWtsL5lwQsBvoDnYBuwBiVihdfesl0KUulNQz9AdreZtJlgS6BDAseVvFMWifuPaD7y8LRaEbW/vgj9xiZ5X5vYSFrfqw9cMps2LiC5wCIfF+oNVgyfd+D6anis6CHnmI5yfHj+i8fESw0+Z7Z+AwAXb2F4y8mO4a8kjyDtz2fdh4At1KxKA23/YYO0XMgz3CwgsQCuPg/+M0ZMvRHi+SW5HIi6QSFTuIhw9Q6f6tOrmL4t0OI8foMGxux9pS0cAwoAFyrdFYVR0foUP51oZP7zCmfSpVKcFSlwo5JEPWd2c01B9LxZ2aao1Cj2VEoRMSjjXBvj24/mlvDb2V6+HQ0Wg1eDl4oUOgvlFuUBnHrxKukaQicCp79qh3SZfzFZJnu+Htn7zuM/HYkEakReDl48bDn97jnD+L8+SqN0g/ByUVkxZwlK0vILnTsWP8hSGpHqVQSEhJSv/lFZXi3Q6GAKUIKnyNb2uFgXXcUYJdbuzDpk0k4+jhy+JPDJByt3O0f+Xs4G3t5UOx5BM4uhYMmRnlKmh5rZ+j4MHj0wddRZHom5yVXnJ74/kSmfjkVnwcmkdhxWMU80P2u7gyZPwSVjXE7rvnWbsSHjWKf71rmnNjEmV6fi2hbSbNjcH5JOwRrgkTNIyoz/tIL09GUy38qlVRIqF0ry5V6LpWMSxmUFVaPwAwaHES/R/qhtFKSmpcJgKuNh6i/02c5BE037wAljYeVvcgKdetaZ9PBgUIqaX/c/hrnrq3zl5CRVXHO3d5TSLKEGK4LJbFcTFq/uPeEjo8KhzIiyGRyp8kMDByI1TXSi6+/Dl99VVlnNDE3kVJ1aWWDzJNw8VMoTDTXUCQthIcfFhs0+eVJokbLfGo18KsD7LmzsUyThD4GIYazt2/sKLItk93WcvactuI9NMS180tUZhTphenYqGzo6duT7GzYFXMXOYELzTYESaXs53CVinCVirbATuAZ4H/AC8BAhYIlixez4t3rKzMmOycHY12dvkB2dhPKUTu2A9dwyw+8tPcHOy+Dznqd4y8qqnLjvSqLhi3i7h5389edIuPP08GzIpPnbKqeNMFyItNF2QuHgs4AuNml4lgahVLWfLVsXMIhYKpBiW7dvmieSuyLmprxtz16O0eS91NoG0Mn/4tw6FFI2d0gkyXNyIGH4GcbvRmiIe7ioUKX8XctVeU+t0ZtZfCqbpwJehIXF1CUZkDyNqMCBZrDZyRnMTNjdJSRpVOSJTTWge6+3fl9xu/M7jMbpULJo/0eZcmYJdXkACtI2Qm7pkLCP01r7/WOpgzUldFlDcn4KyoroqNHR+7qdheRT0TyQD9RLLea4y9wCkw4zKkUkZnRsaMJ0bQSk1EoFLi4uNRvfilKgSurDNZwGzlSbJAkJmJQVufr41/z0NqH2Hu1UpoxJz6H9f9Zz4lvTlQcSytMotQqE3d7V+j+uii2LGkx6Ob05PzkasdVNir63d+NLP/w6vOACWQpPUgMHc4O5zWsPL6ymnNR0rwYnF9s3MHOX0gDAV4OXgBotBpRd60c3cb7tTUTRr46kicin8DaQX/ELkB6gejHzc69gaOQWDqTQifxUO+HGNN+TI1zVR1/Wi0kZYnPhZ3SuVoNH0nLw6T1i0cfGPA/8VoHNjbgXF6ySaPV0Pa9ttgutiUhtzwYKXEDHH5M1A2VtE5yIkVN+ozqaR/u7vBAFd+STpK6TjQl0PYO8KpLpE9Sb8Kfhc5PGzw9rsM4HK0dKbKJI8P2SEVUvSGunV8OxosctF5+vbC1skXnU9FJdUnMx9x583jxpZdIAXYDm4FZwJjy161lZexSq3nrxRdNd/4l74StoyF1X91t68uuW2CvifUHjcDVxQVjq/YlA65N+eHs/n8wenPT3a++qIsg+5zB2s4eHkK+WauFk3pKTLvaubLqllVMCZtScUyX9Xcm5YzB2+oy/qyzRa3hopCFKG9PR+Eio9stGt+RMHy1XvlGgLaubQFIKRH7oklJlVKNxrA3Vux/uecNpYN3JFz6DHIvNchkSTPi2hUCplTbO9ehy/gz5PjTBR2cOgWXMi5xMTuCQpurYo3hEgZ3FEL3V+o0oTl8RtLxZ2bUxha5sXQOzoZ/uuj1hN/f634W3mAgcs6zv5CN8x3ZuPZJKknaJiJUr3xfcUj3BZeYm0iJibI1/zfi/7jwxAV+mP4DHvYedBZBT1y9Whk9i7ULePbj0AkPQMp8NjZqtZrTp0/Xb37JOQ/774X4v/WetrODcePE7+vW6e/i30v/8tWJrziccJjsq9n8fufvRG2J4u5Nd9N3jlhk5ZfkU6AW8hm+Tr7gPVg4iCWWz+ZhsGNSZcZfFcdf3IE4frnlF9yLRNZEXBzk5kJOXA5fDv6Sff817qFct/lSpMljljOEJq02KLMgaVoMzi8uoXDjIVE0HbBWWePr6IunvafeOn+nTlV/S10CXCokgavy9bCv+WnqTwBkFYmMPy8HDyHRsmOKqEsqaTlsHQP/dKuz2cTQiaycurLaRoyO0FChHJCRIWonpOSKz4WLlYfY/Nk6Bq7+YXbTJY1Pg9YvwPYr23l3/7scSThisE1aQRpl5c8rusxk2t0NozYKBRNJ6yQvWtSkzzhc49T48dCrl/hdt1FTJyo7ITHYea65LJSYiJ2VHRNDJwKQ5P5nnXKf184vB+OE429gwEDKysBFe473buyNd34TySleRyQlJbFk8WLWq9UG69kNBNYVF7Pw+edJSjLWHQZoy4S6UH79SpYYRUmW+DEz02bOZJWRWoDf2dtz80zzOx9bPKl7xD7k1d8MNtHN74bkPq9F5/gzVOevRF3C5QyRqaPIEJtfLi4NW79ILANdQkRKQTKh4cLZs2tXzXbFZcXsiN5BXE4cXx3/ijJNGan5qVxIFymC7nmDidfeJMromChFLLEgwucJR7G1U41ToZ6hBLsG42zjrPfSLl2E2lFyMpxNEN9P9iVtq9f3MyJDuDnmFOn4k+gn6Fbo9hJoSutuWxXHtkI2ztHYKuqSBuPcAfwmCFmEcrwdvLG3skeLltjsWJO7rFq3zc0NfH3Fhm6FJnZhIvk5hRw4IP684YYGjkFSJ/X+gnDvDTf8BsGGpYsmTRJqGseOicy/a6nQu868glarJeKXCDIuZtBhXAd8ewhnUVKeeKBTqR3wdtH/ZSmxUGw9wcajMuOvSjZeXnIe59ecJ2nvJfzLp5gLF8DO3Y6s6CxK8uoOLNBowOnIdsL2fU1+Xg73u0BA3CpZb8WCMHZ+OfP4GVKfSyXcO7zi2KBBwmlz8SI1MkLzkvL49+l/ubi+sqCCvYc9dm5iUyS7RGR2eTm5Q2ESJG0Sr5KWg0s4uPVoUBe2tqJoOsC770JStvhcuNq4Q0mmcAoXp9TSg8SSMXr9otXCvrvh1KsVh749+S3zN81na9RWg5cl5oqFi7ejd2WGqEMA+I8XctaS1on3EFGTPviuGqcUCnjlFVi5EsLCmsE2iX6StsLm4ZCyx2CT6Z2F3Hei22qOHq07Rqzq/LLwhoX8MeMP7u91P+fPg71VFs62WTjam7ifIamTlV98wShra4NOPx0DgZE2Nny5cqXxnfuMhNtzhJR4YzF2O4xab/ZuZ8+Zw/bS0or6h4Y4COwoKeGh2bIsRg1cOkPXF2vN/u/dW7yeOGFcHGlXH+H4S8zTL/99OeMyaq0aJ2snrIuE3Lh74QYcswyvPSQWRNR3sGOy3r1rT3tPHKyFek3XIWJfdOvWmp+bvbF7GfXtKIJWBPHQXw+xO2Z3RV3yQNsu2Kg9cHVTgp23XFu2Uu7teS/Rz0Tzzvh39J63txfBqgARcUJRxL6krcj4y7sCCRvEc6sFIh1/Ev20uwt6vC7qt5iCqY5CScNxDIaR6yBgUsUhhULBG6Pe4H+T/oebnZvRXe2P3U9xWXGN47qsv4pN3R2TUK4Pp6QEgoJkfT+LxtpZRCU5BBhs4udXqVmtL3KuvXtloVvXIFdeKn6JYYuGoa2yYtItpG1L/XF1VcCatrBTTx1QieUxfA0MWaU34y9sahj3bb+Poc8PrdiYP3IEbBxtmJ84n5Gvjqyz+9xcUJUUYpufTpYmkweSoWjkxkYYiMTsXFkFEUsr/vRy8KohT+HmBqNGid///LP65VqtliP/O8K5P89VHLtz7Z3c8t0taLWQpxaLYz9XD/EddmcxtJvVKEORNBL9P4KhxmVTlKpLOZJwhGOJNdM4nnhCSDdevAjRycLx52HvITb3b88StaEkrRuFApI2Q/qBikO6DL6UfMOOX528ZxvnNpUHS/NkVnlrx9pZZHQa2ICzshKBi0aTfhgOPVJDOlRiRtTFkHMWivVL+AHcFHoTTw+YR7/4L0lJ1RIXZ3z3/s7+TA+fTh//Phw9CpHpg1mVfQVFB1kj1tys/fFH7imqKZWmj3sLC1nzowlZl0qVUVkTloifnx9L336bKba2Bp1/B4EptrYsXbYMPz9jKwJeRzgEQs83wWuQwSZdu4o5PjVVf9DytdzV7S4yns/gx1v1fw4vZQjpxhDXzihQ4OgI1udfIzBxqd72Egsj94JQjMm7UuOUQqGoUEPz7XQVa2uIialZm37T5U3V/v7z/J8VMp8hVkMBCHS9IMphlde6l7RAsk7D0WdEUGk9qJD7jKl0/Lm4AHFrYcdEyNKfVdzctMxvVIllUpwOvzqJeguSZmf+kPk82u9RPB08jWqfnJfMkK+G4PWOFznF1Ssl13D8Bd3K7nhRQGPcOJm40yIoyQaN4ah7nVMnQs93VTu3doBw/CmUClQ2KtY+sJalrktRl4g+dRH3tqV+ou6OZ39RQFzSYqia8adz6ioUCtqNbIdSpWTwYNFu3z7T9lNzciC2200cv2MSWqWWXJUrjj6DzW2+pDG48h1EvFlns1tuEa8HDlR/AHf2d+bRk48y5fMpRP4VSX5qfsW5oiIoVggHTxt3j8qL5BdKq+WDgx/Q/4v+vLHrjRrnfH1hwQIhoeKVM5aBkZtZOLBmO0kr5+Y4GLWh4k/d+iMi1fCDtC7wyN+pUvmCf3vBxv6NYaHEUtBqRQ0oc2WJZ52GS59DoRG7yJL60WYi3JoGQbcYbOJq58p7E5czrN1QFCg4caJ+t9LJhPapu2SopB5k5+RgrMvKF8jWaf4bS+YpOP9+4wSV516Gi/+DPP11nBrKXbNm0W/kSIYCw4EfgC3A98B4OzuGq1S8sGQJc+fNa5T7Xw/Y2UF4+TaDMXOEs60z7vaG64lPCZtCxvMZvNXnZ0AENWp6vk2c/6KGGytpfLosgBm54NJJ/2nvLnTx7oK1TRkDy9OUt16TzKlz/M3oOgOANefXsOeqyE5voxaOvyH2z8P6ussbSCyY/FiIfB8y9RQINYKpU6FDB8hTXSP16T8eBnwhav1ZINLxZ2aUylbyT5pxHLaNE55rYynNAb9x4BTSeHZJ9BP7pyhSXZxR7y42XBIbLWGeYbjYulQ7p6vhd+KEcAzFuS3iwx2voFTCyJH1vqXESJRKJWFhYfWfXyKWwO9uIhrKAF27ljeNqOnU0Ul9RmdFo9VqSTqRRFFWEWFTwlDZqIBKqU+7Un/x5TfsD+glo+RaBImb4fQb+Nt78Nnkz/jltl/QUtOz16sXODiIOlznz0P84Xj2vrO3TrlP3bO+0jUegL6uvlBWYO5RSOpJrfNLv0/gpsrifbnFuUz/ZTqdPuxUrX5sUBD07Suarb1m2eAd7k36hXR+mf4Lf95TmRKYlQU9o79h3IUz3NZtmiiUnrBRrCUkLYeUPXD8OfEgVQeDg4TDf1/svmoZ4zp69oSHHgLbMh988sYyNmyIqPEXu6ZRavFIGh+T1y9K62p/Dm0rNlv2xe5DbSB4SW/GX5uJ4H+j6QZLWhZrAuHgHPP01f4+mJ4KfqPN05+kJiYE9lRE1Z8y3Kbq/LL+4nre2PkGRxOOkpEBUVEwMGAtA31WyezfRsDVxQVjXe7JgKurq2k3iPkJjj0DOZEmWmYEafvg8OP13vitjRXLlxMcGIhm507eBzoCrwF3AI8BViNHEhMXJ51+dXHgQdgzo9Ymptb5qwt3e3ecyzqI391B6TcS/36zW8/+bmvG2rnG+rEqf8z4g4jHI5jQcQKjy7/id+6EMlEemuS8ZI4niQ/SsrHLcLR2JDYnlvTCdHwcffAuFGvRNKdZQhWvhWYkSwDfkTDtqkGFofvX3E/AuwF8d/I7veddXWHpsjKKbcW+ln1JMJ6eCAWKjrOFFGwdNMecIj+xEv1oyyDtoGlRj07tYeTf0OGBxrNLop+8KIhfJzZOyynTlHEg7gArjxmnqb/+ktC5vyn0phrn2rcXdfzUanh7aRm//iqO9+0rFkaSxsfGxqb+F7v3Lv9yM/zAHRYGKhWkp0PKNYpauqLIeSV5pBem8+9T/5J0PInpP0yvaOPv7E+7svG4FvQVGX+SlkP8X3D6ZWw1+Tzc92GmdZ6GUs+C1toa+pcnUOzbBxf/uciW57eQcbn2gIPsbPCKOYpTRhRKYKvrBdh/byMMRFJfDM4vLqHiu718s87Jxont0du5mHGRs6lnqzWdXj4dbNki5F2r4hXmxU0f38TEDyZWHMvOBtsyb4LsuorM9OifYMeNemVaJBZM5gk491/Iu1xn0z7+fbBWWpOSn8KVLP3v85Qp8PDD4sfJCYj9A3bfAvnRZjVb0nSYtH7JvSSCDtVCdr67T3dcbF3ILcnlVLJ+D4DO8Vct46/fh9Bzcb1tlrQAFAoIexoCp5inP6UK7LxAZWee/iT6SfgXYlfX2SzPdwv7wkby37ze3PHbnSzbu4zc4twa7XTzy89nfublHS+z7sK6CkfAXb2X43B+rlQSaASmzZzJKjvj/q98Z2/PzTNnmnaD9vfB6M1iDWpu/G8UfXvfYNZuVyxfzluLFrFLrWZDURH/Ab4CLgDpiKy/I9u385MpsqfXKwVxkH+11ia6On+nT4s9qrr4+vjXjF81nsW7DK8NsrLEq5ubeG3Q/oukack4ZtR3S58+wnmTnV2ZGb4lagsAvfx6EewWzI0dReDY7V1uJ2l+Eoos4RAu9r1d1J+UtFysHMAxCFS2ek+HeYaRkJvABwc/0BukCpBflk0336542HozZ6YvI0Y0psHmQTr+zIxG00r0fj36wYwcCH20uS2RGEPHR2BGPngNqDiUX5LPkC+HMGfdnFpro4BwEm68JGpu6XP8KRTw9NMQEgKzQh9jmHoytqp8xo417zAk+tFoNJw+fbr+80ubiTDke3DtbLCJrW1lrcZr5T7trOwqNtSuZl9lyLNDmPDehGptbutyGxNSN9IxaSGedlfh2HxI3Vs/eyVNS+f5MPEE2HrV2XSoCHhj3z7ocU9PHtz7IJ6htcsJZ6aV0e7033Q55UnWc6nkh8yBNpNqvUbSdNQ6v2jKxMN3iUjbVCgU9PEXulnX1mnr3l18RxQXw88/1+yq3yP98OxU+Vm59uGawCkw4HNwbNvAEUmalHYzYWoUeA2ps6mdlR192/QFRE1hfSgUoAz7h6Q2K4lMi4Sg6TB4FTg2wuafpNExef1y6QvYdbOYdwCVUsWQIPHZ2n11t95LBgYMZFb3WfQPkNKe1x29l0HHh83TV04kZJ+V2WGNzYkFcHxBnc1+iXmXDOedZNmd4Nezv7BgywJCPgjh3f3vUlhaCFSfXw7Gi4pqAwIGVGzmnnH4HAZ/32hDuZ6ZPWcO20tLDdax03EQ2FFSwkOzZ5t2A9fO4DcWrBzra6Jh7LxF30ZkZhhLUlISCxcsYF1xMQMNtBkIrCsuZuHzz5OUZCaJ4tbK6E0w4UCtTTp0ELWh8/OF868uvB292Ry1mbf3vk1mYWbFca1Wy/RfpvPMhmeIT88CwMutAH51JGPTrNazv9vaOf6sCCyuo/6eSlWpWKaT+9wUJWQ+J3QQ+1u3dBZy1H+e/xOFQkFOtggeMTVxWWKBaLUiaSYvWu/pOX3nYKuy5WjiUQ7E6Z+DPB08OfnoSdIXpjDjNhX29sDeWbC+h1EmNMecIh1/Ev3UJzLuyNMQ+aH5bZHUjbVTjagFVztXwr2F+PnBuNqX5ftj95NdnI2nvSf92+jfOLGzg0UvanG0L8TOKg8bB0cGDNDbVNJCqSr3eS37H9pPwYsF9PHvg3uIO5c3XObqnuqReDnlCn1uVhfh/LuQVYs+j8RycGoH7j1Bac2xxGP8dPonLqZf1Nu0Tx/hJE5JgXSNO0FDgrB2MCytAZCbqyBy0D24TByCs4MXjoM+l5nhLYWEf2FNULUIyj5++h1/CgXcfbf4/a+/hHO4Ng4d0hIROI8zXq+TX5IP7r2g4xywkWnkLQpbDxGRrzIuKnpwoJD73BtrODDk06OfMmfdHOHoce0C7e8GG/m0fV3Q9jYY9C3YVgYJDGs7DKBiY/9a7ut1H99P/56pYVPFgcyTsP8+SK1jEpJIqnJsPvwrC8I1On1WwKCv6my2+o4/eEC5jd5RP3KP3zt08uxEWkEa8zfNp/dnvSsyfQEyCzO5kC7KGfTzH1CR8RfatzO0kZK/jYGfnx9L336bKba2Bp1/B4EptrYsXbYMPz9jKwJWQauFotqDl02mJBNy9D/jNISVX3zBKGtrg04/HQOBkTY2fLnSOEUmiWGUShgmlgds3lx3+0mhk+jh24O8kjw+PFS5Z5mSn8Kf5//kg4MfkJ8tsljdXUvAZwQlNoGNYbqkMej6Igz9Va/jLyIlgl6f9qLf5/0AKpIX9u+Hs2e1FfX9xncYD8CkTpNwtXXFWmlNQnYyOTngZJNB+/M94fx7TTIcSWOhhXWdhJS0HrwcvJjZXWSof3DoA+O7tXEDW/MFk5gb6fiTGCZxM8T/bVxbTSlc/hySt9bdVtI4ZJ6E+PXVDg0MEMtPQ5slOn4/+zsAN3a8EZVSZbCdj68C95u+58OITdxxB1hZNdBmSdNxdB6cWFhrk9ocf8Fuwdhb2wOQejaVU9+fIi8pr+J8cWkpeeV/2gYOg6lXIPhOs5guaWS0GlE/q6yQN3e/yczVMysWwNdiawv9xJqZfftAXaqmIL32en05+SpyvULw7OZfazuJBeLaBTo9Wa1QdW9/oa1zreMPhBTsLSJIkvfeg/h4OHAAnn1W/OTni3NZWbBlVz5X/FawqeSVRh6EpFHRqEV9v8Jko5qPajcKgL8v/G1QQiWjUMgHe9h7mMdGScvBsz+E3CseoMu5t+e9HHv4GN/drL/eRg1yzsOV76Awoe62kpbN+RWwbYKYhxpKyH3Q4w0pC9nY+I0Bn2F1NrO3tufmnqMIyLiL9gnPEvF4BF9O/RJ/J38CXQLxdfStaHs44TAAHdw7kJXgRW4ueLnmEBaSZ6h7iRmYO38+LyxZwnCVigl2dvyAkLP8HhhjbcdwlYoXliypfz27TUNg01AzWgxc/Q3+7iQkpc3I2h9/5J6iIqPa3ltYyBop91k7eVFw+SsoiK+12bhx4nX//pplBq5FoVDw4g1CpvH9g++TVyLmh/Np5wFo796+wvHn6O6GZvjfpHre3YBBSJoUv7EQcBMoa25QOto4cjL5JKdTTqPRamjXDsaMEbEFH30Ma2f8w1tj3mJokJhv3OzcmB4+nfzSfE4et6GkBHzds7EiH9S173tILByFErq+BG0N1xB9csCTgNgjrxpkVCv9P4YxlusLkY4/iWGOzRXRj8agtIbbMqH/Z41rk8QwR56A/XdXk6jROf4MpSmDkAT99uS3ANzdo5bFjboEtFq6doXPV9owbZp5zJY0EUmb6nTkh4sEUeLjK2X49NF1RlcW5iwkdFJoxbHAd4PY0NOdEtdzOLvaiCwymbnTMoj7C353h9g/KjZSkvMNb+Lr5D737tHyjtc7rJ5Zu55+epoGtFp+znmCN9fcQv6WcZC83WzmSxoR5w7Q7wPwrpRx1El9nkg6gVrPZut994kggsJCeOIJePNNiIwUP998I9r88w8UaITMjrXSGgdrBzj0qIjAq0OiRWJhFCXD2rYQ8aZRzcd1GIeTjROxObEcSTiit41Ogsndzh123y4+F5LrlkCXQHr799YbmKbVaonNjqVUXVp5MPgOuD0HAiY3oZWSZiH7HKTth7I6dnyNoe3t0OW5hvcjqRutRkiJ10GPctWsiAhAY8WDvR/k5KMn+enWnyrmgx8u/8Bdq+8CoLfPAHaXKwLPGrQS1WpnmfnbyMydN4+YuDiGL1rEe+HhPOjRhmedwrEasYiYuLj6O/0A/MdDgJlLA7j1EiVSfMxblCk7Jwdjcxp9gezsbLPev9WRsgcOPgQZ+teJOjp0gPbtobQUdu2qu9vbutxGR4+OZBRm8NkRsW+pc/x19uqM7m2pKEMgaXmoazrgA5wDUCqUlKhLSM4TexwPPSSkO2OvKoje34eFNyzE1qpSQe2LKV9w5rEzbF8v9rP6j2qPYuolWeOvNdDjVVGqwgC9/XszNGgoZZqyinmiKs9teo5un3Tju5NGBiRaANLxZ2aUylb0T9r7v8JzbSwqO7D3rbudpHHo8gIM/LLapunAQOH4O5xwGI2BzVRHG0dOPnqSJaOXVOha6+XEQtgyDIrSzGq2pG6USiXdu3dv2Pwy4SDcVLsAvrMzBAeL38+erX7uQvoFHln3CI/+Lep+2jrbYm0vJB7LNGWkF6VQapVFz1APVMXxoiC33MBvGTiHQoeHwLFdpeMvz7Djr18/sLaGxCQFgZN70n6M4dpbkZFw8Y8z9Fm/mITD+zlyaQ0OqduhKNXsw5DUD1Pnl1CPUBytHSksKyQyPbLGeZUKnn9ePDSXlQmZaF1E7oYNcPQorF8PpVYiq8vd3h2FQgFWTmDtJiLxJC0HWy8IewZ8RxvV3M7Kjk9u+oS9D+6tqPd3LdUy/hwCxRwlaZGYvH7Ji4a17eH0G0Y1j8uJo+17bbF/076688/aWTyXSFo3Az4VNemrZIhKLJzYNfCLHcStqbNpcLDYmC0uFutJEHW6vB2FnJZSqcTa1ZrsYrFbf+nfiawp79ajXSdoe0c1xQJJ4+Dn58eil17i8Nmz/L09nv4jzxIU/FL95D2r0uM16PueWWyswGuAmDfMPGe4urhgbNW+ZMBVFgurHb/RMGIdeA2utZlCUfmMYYzcp0qpYuFQoYC0fP9ysoqyKh1/np0rAp99bE+jPPUC3QM1rWt/tzWjUcNqP9h7V41T1iprApwDAIjJjgHEvte0e2LRouXnn0VQ6tq14jU/X3xWYqJsOXdOqJxNnNiko5E0M7qsv8+OfkbZNYFK59LOEZEaQXFZsTig1YrnFiPVEptjTpGzmMQwbW4UKdPGkLARsoyoqitpPAJugqBboEpEdDefbjhYO5BTnFOxqNFHsFswLwx7QWy+GqIsD1BWq7siaTpKSkoa1oGVo1HyRYbkPovKivj82Of8dOanGvJsKfkpaNGi0KoY3MsLTr8Ca4OhNKdhNkuaBreuMHAl+NyAr1PdGX/29pUPWQfcJzJovkgBzMiAt9+GX38VDp+iIli+HEpsHFGFh5BoH8+afMialgJBtzb6sCTGU+v8cuRJ2H9/xZ8qpYrBQYMZEDCA3GL9WRYeHrB0KTz+OHz1FTz1FNxYXmZn8WJRD9TRq0pWF0Cf/8KNh8wxHElTorKBvisg6GajL7mn5z0MCRqCUo+TV6vVVnf89V0BI/8xl7WSZsCk9YuNq1ALsHapdvhIwhEeWPsAL217qdrxdRfWAdCvTT+sVeX1ZjNPQuaJhpgsaSmYK1BEXQzre8EZ4zKXJQ3AMRj8bzRKFUShqMz6O2WgbPg0/zmMvniY4WdOE5BxN66u0KsXhI2aDDf8LJ9bmxgnpyQuXniDv38JJ7RNAP3Dw1n8xhskJRnrFmtENKV1t6kn02bOZJWdccEm39nbc/NMw9kmEkTQV8BksPOps+nIkcIxc/kyREXV3fU9Pe8h2DWYxLxEfj/7O+fTxR5ZmFel48+DEyjOvUNZtvnrQUoaCaUKfEeBW3e9p9u6tgXgavZVQAQZPnm6P5F9J5GvTePTT2HlSvj0U3jxRSEdu7ZcEXj4cPDQHIDLX4vyKJKWzZnFQkbaQMkJgOnh03n/xvfZ9cAurK6Rj9V9hnSfKdQFcPplISVtoUjHn5nRaFpZhotWU3fdBK0WDj4Ie+UCxiKoEpFgpbSiXxtRkEuf3Kdu0jKKgZ/DuF2y9kUzoNFoiIyMbNj8UhAPSVugtPZ6F4Ycf529OmOjsiGnOIforOhq565mJAJgU+pDvz4q8UAfNhesZTRjS8MYqU+Au+8WkXIxMSJ7Kz8fXnkF9uyBVatg7lzh9EtMBJvwDty8cSDJfsk4WDvgZu9eLUBB0rzUOb9knYKMo9UObbp7EwdnH6zIKtdHQICIjnR2Fn/ff79wCJaVf0X1HCQcf7KOm6Qq6YXplJZvznk6yA3blo7J6xcbd5h4DDo/Xe1wan4q35z4hl8ifql2fPU5ITV9a3iVYJJjc2HLyIaYLWkpFKWICOv82Ib1U5orAhzNIRkqqR2P3jDiL1Hrzwh0jr9jx0SAWUFB5V6dWq3hs+UO2Gf1ZUjHbqz+Q8H338Mbb4CjYyPZLzHIiuXLCe8USJvLS1ied55PExN45vx5di9ZQnBgICvefde0DlN2wb67zRdcfvQZ+LsLFGeYp78qzJ4zh+2lpRyso91BYEdJCQ/Nnm12G1olRkgCOzvDoEHi9y1b6u7SRmXDmjvX8OHED5ndZzaRaSKduJ1TGLo4JbtOt6OeFElkVmDr299tzQz9CXq8rvdUsJuQtYrJEhl/z2x4huT8ZJQe0Qzu60T//jBihMgyj4qCF14Q+xqAKHEU84vY8y6VMr0tnsIEyI+GsnyDTaxV1jw18Ck6edYsN6HLGq1w/CntYOJJ6PayUbdvjjlFOv4khrnyA/zqDCk762iohQGfGZxkJU1EaR78GQCHHq52+PWRr7Pz/p3c0fWOasdPJJ0g5P0Qbvv1NkrUDcwmk1g+V76DbeMg51ytzbp0Ea9RUVC1/ICNyoau3sIreCLpRLVrDp8Xjj9HrT+BgUDb26Dvu9JJ3FJQl8C+e+D8+/g4isjK2qQ+QTxk3XsvOGbEsuOxX3jz8Tiio8Vi2dkZoqPhwAHxEZg3DzLLRHH20e7eKJK3ieh6SctgzHaYVH3TpdbscAM4OooMQN3vAaFC7rXCuXPxU7jyfYNMlTQTJxbCrltMuuRY4jEeWfcI7x14r9px3QZMW9e2ovbj6dcg5ldzWSppoQwOGowCBZcyLpGUJzJH0gvS2RG9AxCRuRV0egJ6LmkGKyVNTtoB2DkFkrc1rB87L5h6CXotNY9dErPRs6d4PX9e1BC+4w74z3+ErN/atXD5sh12dvD002BjU35R7mXxzBO3rtnsvt5YsXw5by1axC61mm3qImYBY4BZwMaiInap1bz14oumOf8K4iH6B8g1U9aVjYfIAG2EGvR+fn4sffttptjaGnT+HQSm2NqydNmyhsugtna0WljtC7umGdVcp0SzbZuoMV4Xvfx68cSAJyhVl5JbIgI+/Kw6A6JEga2DHTh1QKOSEQSthbYulRl/6yLXserUKpQKJd9O/4pXX7Lj5Zfh2WfhrbdEuYqYGNBooHt3CAlBrC2H/wX2bZp1HBIz0P8TuCUerJ1MvjS7KJucYqFqFuQaJA4qVeDeA5w7mNNKsyIdfxLDOAaB91BQ2tTeTqEUqfhBpm36SMyMtRO4dgPH6vW2RrQbwfDg4TjaVC5cItMiufH7G1Fr1ZRpyrBR1fIeq0tg61i5IdvS8b9RfMk5BNXazMsLQkPFenvbNfsovfx6ATUdf8cuCsefr4O/9PW1RJTWEPMjpO6uJvV5raTrtYwfD0E+xTjHRxJ/KgM7O3j9dfjf/+CGG0SbO++EKx/8RcRKkUL6qHMJbBtba4SVxMKoRUotKjOqYvFrDAMHwquvwpIlkFoknMG6mgtELIELHzbEUklzkX2uRlZoXUSkRPD5sc9ZeWxlteO9/Hqx6/5dfHLTJ0J14vSrcFU6/q4rYn4R80EV3Ozc6OErUn/WX1wPCJlPtVZNT9+edPCo8rAdNB06Pd5k5kqaEY++MOhr8bwqaTmcfRtOLjKqqZ+fkFlzcABdWZzYWPjgA/j6a3Hgvvs0VPOj5F+B1H1QkmlmwyX6SEpKYuGCBawrLsaQDsRAYF1xMQuff9542c+gW2BGvpjTzUHPN2Dc7kYLTJ07fz4vLFnCDQoVo5R2/ABsAb4HxtvZMVyl4oUlS5g7b16j3L9VoVCA3zgxxxtBr15CaSQ/X9QTNxZrlTUpz6aQ/GwyqiJRO9TdHciPEVlBkpZF9lk4OBuSayaudPLsRLhXOCqlikf+fgSA+YPnMyhwULV2QUHiOdXNTfx9i26L27kDBE4R+yaS64bvTn7Hbb/eRnyO2LfQKeZ52HvgZFPuOFQXifWG1nKzg63qbiK5bvEZDqM31d2uJEsWVbcURm80eOp08mm+PfktEztO5L4195Gcn0wvv158Pe3r2vvMjYSMw6LIsqTZUKkaKI3o0Vv8GMH48XDxooimvfnmyuejCsdf8olq7SPjk8AO2nv7iwNbR4PPKOj+fw2zWdI0KBRwex5Y2RNQWsjnkz/H18lX1G3E8MOxUgkPvtGeBSWLUFqreOXF8og4YMECePJJsNKWsPTeEyhGK2AoHLfrwaSe/ydlYC2MWueX/KuQfhC8h4F95c7a/I3zWXFgBe9OeJdnBj1j9L36lj/DPxf4HDO6zsDe2l4ckHXcWi7D15i8kTYlbArWSmsiUiOITIskzCsMAEcbR4YFDxONtFqYfEFKA7dwTF6/RP8AiRugywvVPlczus7gZPJJFm1bxPTw6fxx7g/gmmw/yfWFQwCE3N/wfrLPQdp+ESTnIKP5G534dSKLq2fdNRUVCnjuOfG7ViukPjdtEtl+aWkQFlbMxInXRO37jYXbc0BbR7kSiVlY+cUXjLK2ZqC69n/vgcBIGxu+XLmSRS+9VGtbAFTG1cyzBJKSklj5xRf8+f2POFl7cUSr5oq7E1bWVri7u3PzzJl8N3u2zPQzhSHGB50rlXDrrSIgYM0amDwZrI30zygUCnwcfTifLv52cwMOPIQy4zCqzntMtVrSnJTmweUvwTkMfEdUO/VA7wd4oPcD3L/mfhLzEgnzDOO1ka/p7SYoSHyWYmMr5aZRF7WoOUlSCwVxQkraawg4tau16f+O/I8DcQeY0GECc/rOqXD8BbsGVzZK2AC7b4HBq6D93Y1oeP2RGX9mpsGb8y2RDX3FRr/Eovns6Gcs37+csavGEp8bTxfvLmy6exPu9nXIXbh1h+kp0OnJpjFUUgOVSkX37t2bbH4ZNkzI5cTGQmRk5XF9GX+5uVCY2A7v7AmMDOsNZYViAyU/uklslZgJK+F8sbe2Z07fOUwNm4qylkwvHeHdVCx+S8Xbb0Pva/zKDg5g42jDi/kvkjNHZIUVuPeB0MfkRr4FUef8krQV9syAjCPVDod5haFFy4eHPkRdVy1gPTjZONHVpysh7uXeYrfuBguySyycekTPu9m5MSZE1Hhad8GAHJtCAS6h4BTSEOskzUi91i993oWbImocnj94PmGeYSTlJfHE+ifYdFkEJ1ar75d3BdZ1gov/a6jpkuuJpM1w8CHzSQpKaueGP2BqtMmXKRRCKvyWW2DlSli6VMFHH/lgZaVnflGqoDZFG4nZWPvjj9xTVGRU23sLC1nz44/GdazVQvoRyDzZAOvKOfI0nDKu/pKprFi+nODAQPYsWcK8C+f5vSSZT0vTCMvPIzY1lZmzZ7PopZek06+RGTVKKBdlZMDWraZff+yYeO3UCQi+A0XneU26/yIxA+69YHoqdHlO7+n1F9fz7clvUaDgq2lfVQaf6uvKvYrTD+APH9g51bz2SpqH9COwbxak1u3YnxQ6CYD1l4TaiEKhoKdvT7r5dKts5BAIHWaDa7hRt2+OOUU6/sxMXdJoLY4Ln8CxZw2f15SKqDrv4U1nk8Qw2efg8ONC3uQapnSawuROk1EqlIR5hrHlni14O3ob16/KFqydzWysxFi0Wi05OTkNm1+KUuGfrkY99Dg6Vko1bqqS9NvTtycKFKgUKgpLhYD+qVMQmH4Ptxdv4LmRjwsH0vREGPRl/W2VND1ZEXplMYwhxCeP9HX7UJfqd/5Y2Vrxzu3vkLMwh+eHPt8QKyWNQJ3zi+9IGPITuPepdnhW91m427kTlRnFv5f+bagRUJINmrKG9SNpHvJj4OpvUFh7bdBrGRk8EoCD8ZUVcd7c9SZfHvuS7KJsIQlckCDWmpIWSb3WL84dhcP3GoeyrZUtn03+DIAfTv/Axzd9zNIxS+ni3aWyUVl+eYkC+Yh7XaDVwl8dYf/9DesnaDqM+FvUaJE0Pva+FQFn9cXKCrp21VJUpGd+ifpGbOxJmoTsnByMdWn5AtlVi8jXxeYb4OSL9TGrEq0G4tYYtclrKlVrG24oMlNtQ4kg4V84+LDYwzACK6tKWcY//oA6ElCroVbDoUPi90GDgI5z0HZ7ueH7L5KmRWUjavYawN3OnVCPUOYOmsuQoCHG96spg8BpUla8teA1EG74HXxG1Nn0ptCbANh8eTPFZcXcFHoTJx49wXe3fFfZyLMfDPzCaGni5phT5FORmdFoLFfXtV4krIcLH4GhaH6lNQz4DHq82qRmSQxQmCiinDOO1Tg1oeME1t21jrTn0jjx6An8nf3r7i/zBEQsFf1Kmg2NRkNUVFTD5hdrFyF5ozAuwkRXJHv3btAFcbrauZK1MIv/BkVx3yx7nnwSVq0S53r1qr9pEgvg2FzYOQWAIwlH+PH0j1zKuGTcpSuPsfm5zVxcXzNSPvFYIglHEtCoNTjbOuO2ZQgceMCspksaRp3zi1N7aHdnDfkzRxtHZveZDcAHBz8w+b7zNs5j8a7FwsFTmgO/u8GhR0zuR2IBJO8QWaHpB0y6rH9AfwAOxx8GoKisiP/b/n/MXjebwrJCSNwEawJEzTdJi6Re6xeNGgqThGTTNYxoN4J5g+bx9bSveaj3Qyy4YQGKqg5Ct24w6QyEyrnkukChEJHWdkYGMhrCIRACJoFNHSooEvNQmiccc0UpDepG7/xSlg8HHoRz7zTQSImxuLq4YGTVPpIBV1cj5f4VCuizHDo+Wl/TyvtRwrQrMNS8a4lGq20oEWSegMtfiOAyIxk/HlxcICkJ9u41/lbnzwsVIycnCC9P2jHL/ouk6SmIg/i/9dZbGxw0mJOPnmTx6MWm9am0giGroMsCMxkpaVbs/aHtreAYVGfT3n698XfyJ780n10xu8xy++aYU6TjT1I7Az6HW9OkLFtLwWsw3JIAnR432MTd3h07KyP1qaN/gpMvyOLGrQGVLUw+D91fMap5167Qpo1w+u3eXXn81GEXvv4aCgshOhri40GtKK6UecyLEpu0hfLhpkXR6QkhrwYs3rWYWatnsSVqi1GX9pnThxl/zCD0ptAa53a8soMvh3wJWkRkvq0nWLmY03JJM/J4/8dRKpRsjtrMudRzRl9XWFrIigMr+L/t/4cWrdjcCXkQfIY1orWSRkOXFerR36TL+vqLyMiY7BhS81O5mH4RLVrc7NzwdfQFx2AxN7l1q6MnSasifi386Q/xf+k9vXzCcu7vdX91h5/k+mXsDujdQCePzCpuWlJ2wMb+YnPW3CisYdQG6DzX/H1L9DJt5kxW2Rm3t/CdvT03z5xpfOed/gOBU+ppWRUUyoYHCFxDRW3DOtpVrW0oMYHQx8Q+pJFZNAB2djC1XI3xt9/Eo6cxHCwXnujfH1QlSbB5OIqor000WGIRnH1bBDMXxOs9bW9tX6vEp+Q6wogJQqFQMLHjRADGfz+eojI9stYXPoF9d+sNWLQUpONPUjsObcDayfD53bfB6Teazh5J7VjZiwgGI2pzGUWP12HUphoSb5LWj0JRmfW3ahVs2wYXL8KKFeLYTTfBSy/B3Xdr2dLPlbFb3IjPiRf1wPbeKaL0JC2HwKnQUWRv+Tr6ApCcZ5xsn5OvE+HTw1FZ1wwQGfDkAMYuH8vkXybz8N+PUDhyE/R733x2SxqfskL4sw0cnF3jVDu3dkwNE0/YHx36yOgu43PFw5iDtQOutq4iI3nQlxByv1lMljQxjsF6s0LrwtXOlXCvcDp6dCQ+N57zaecB6OzVWTh1PPpAvw9FzQ7J9YNLFwh9vH61HRM3iwfwsnzz2yVpvWwcJOTwJU2DW0/ouQQ8+pm/b5UN+I8Hr0Hm71uil9lz5rC9tJSDdbQ7COwoKeGh2TXXk43KxU8h/bDZu2202oYSgY2bCBg1Mchn0iThAIyOhpNGlIfUauFAuWDFwIFASQbknIMi0+TrJRZC8J0ieaW2PWxTSd0rZGezzpivT0nzoVHDH16wz7gglMmdJlf8fjFdTy3otAMQ/UN5qQHLRDr+JLWjKYXMU5B7Wf+57AjIOd/0dkkMkx+jV+qzXqhswX+cyQsuifmxMzKSslbi/4Zz/zW6+fjx4O8PmZnC4Td/PqQrIzjTdzS/2I9i4EAYPzWLUm0x2cXZeDp4gt84IaXiIZ3FLRVfp3LHX75pDzyZVzK58M+Fasc6jO9AwL0B/HvpX7458Q22VrZms1NiPmqdX1R24BwKDvrlMJ4a8BQAayPXUqo2LmsiPkc4/gKcA2TWTmtCj6xOXRx75BgXn7xIL79e1Rx/ktaDyesX187Q/+P6bdxHfw9H/lOvz6KkhRK3VtSvbkjNFJ9h4DvafDZJascxCLq+YJaaijXml+KMhn0WJCbj5+fH0rffZoqtrUHn30Fgiq0tS5ctw8/P2IqAiOfWvzoYXeetBsXpcPjxRpF+bdTahhJRVy37LGSbttfo5FQZvLx6dd3t4+MhMVHUCOzTB3DtAremog1fYJ79F0nT4j0UOs4xr3R3xjEhO1ucbr4+Jc2HUgUeA8A5zKjm0zpPY8noJWy/bzvdfbvXbDDkO5hRIAKPLBTp+DMzKlUrk8QsiId/e8KFD2ueU1rDTaeh99tNb5fEMLtvh51TG95PSbZYaGnKGt6XpEGoVCo6d+7c8Pkl6ls4/rzR76mLC3z0Edx3Hzg6iufoQB8nohXb2Re7lxJ1CYl5ov6jm52bkJB1agfBM8DOp2G2SpqWqO9gXRhknqzI+DudcpqfTv9Ean7dD9tarZafpvzEn/f8SWlhacUxgLicOACGufugPLsUso2XhJQ0PnXOLwoFjN1pUCZ4ZLuRrLplFeefOI+1ytqoe+oy/gJcAsSB7HOw7x6RMSxpmWwcLLJmTKSq9Pj59HLHn2e54y/iLbGmkeuQFovZ1i/G0v01GL0FrJ2b5n6S5id2NZx5o2FZnn3fE9nFkhaF3vllx0SZvdkMzJ0/nxeWLGG4SsUEOzt+ALYA3wPj7OwYrlLxwpIlzJ03z7SOFSqwcoayekqoWbvCuD2NUpur0WobSgTaMtg6GrYMN3l+nzZNPL4cPy4y/66lrAyKi8Xvumy/nj3BvooCpMrKqmnXLxLzYs4AkE5PCNlZmUneehi1Hnq8alRTpULJC8NeYGS7kYYbWRkvH9scc4p0/JmZVlf81SEIur0MgdOqH9dNpEorURBdYjl0+g90WdjwfpK3wj/hIm1Z0qxoNBrS09MbPr/0eB1uPAoYn2FjYwO33QZffAFPPQUfLm6Lm50bpZpSzqaeJTFXOP78nfzFBTLKtmWitBIZvpoS/JxE/Oqeq3uYuXomZ1LqlrVQKBSMWzaO236+DStbKwAifong3cB3idwcCcBwZ0c4uUhEb0oshobOLwqFgrt73I2TjfGSKlUz/gDIvyoydfKu1MsGiQXg3qNB2RsaraaiTmRFxl/GMUjcIOYnSYuk3vPLvnvgxIum39CpHfiNMf06Scul+6sw6SyoZM2eFsWBB2HLyAZ1oXd+8RkObW5qmG2SejF33jxi4uIYvmgR74WHM8uxDc86hdPh3kXExMWZ7vQDUavxphPg1L5+RimtwHuISXXijKVRaxtKhOLIgC9g6E9g5WjSpb6+MHSo+H3NmurnoqPhP/+BO++EV1+FLeUl7QcMKG+QcQyu/oGmOMc8+y+SpmfzMNh+o/n6UyiE7KxKKhdJ9JCyC7JOG928OeYU+SRtZrStbdNbqYIer9U8vuc2cAiGPv81Xz05iXkIuc88/Th1gPDnwWuwefqT1ButVktsbCxubm4N68g1vN6XOjvrZDMU9PLrxY7oHeyL3SfqcwH+zuWOv+0ToCAWJsusrhZFu5niBxjjGMqwtsPIKsrCy8GrWkZObYTeFFrtb6WVEidfJ9Js0gCIcewOo38Ex3o+vEsaBaPml6u/Q/oh6PV2rdLPWq2W7OJs3Oxq6YsqGX86x5//eCGRIdcTLZcBn9XrMq1Wy8QfJrI3di95JSKiP9y7/Ltq2G+iDoOkxVLv9UvKDqMleKpRkgnWblKi/nqivg4BHfkxcGaxqAskncZNh6YU1MUiYLCe/1/1zi+9zS/pKDEePz8/Fr30Eoteeon33oOtW2HkKDBF3dNsaLWQdQrcujfK+nLqtGm8/sorHAQG1tJOV9vw26aubdgaCJxS70unT4c9e2DnTrjpJggOhv374cMPoaREtDl6tLJ9heMv6lu48AHaKTHExmY0fP9F0vTY+Zk3GCjzhOjPpR7rUollEv0TZByBPssb3tfu6eDaVSgkGUFz+Iyk409iOqV5IjpfYS036Voz7j3Fj6T1oNUKqRSFlUnp6NdyY4cb2RG9g8W7FjOnzxyAiiwxXMLBxsMc1kqaCTc7N3Y9sKve1yefTiY3IZcut3Why21dmL9xPlwGT5f2jRJxK2kC4v+GK98KBQADxdIPxh1k9rrZ+Dv5s+meTbV3V+74C3QpVwxQKBo0J0laLgqFgpT8FPJK8nj/xvfp5tONEPeQygZKKbF0XTI12vT3Xl0Mv3tA8EwYKtUqrhs0aihJFxkhJmaFAKKO/eWV4N5LOv6akiGrmtsCSSPTrp14vdIQMYfiDKE+5N5TZHOaQvJ22DYGwp81u0N4xfLlLFywgA5KJZPUav5Bv/OvorbhkiWm1TaUVCf7HFz6XGzSG7n/GBoK3brBmTMwf371c336wKxZQgr08GHo3Bm8vMpPdpwNXgPLy5ZkmHUYkiZi2G/m60urFWWUrJ1hUoT5+pU0LwnrhdpQj8UN34Potcy8NSUbAem1kdTNpZWiqHLuJfG3tRNMOAQDv2heuyT6Sd0P//YRGRoSSVXi18FvLnC1YYuhpwc9TUePjiTmJfL6rteBKlKf/d6HG35uqKWSpqY4HSI/hNS9NU6dSTnDm7veZP3F9XV2oy5R8/347/nrob9Ql4pMndicWAA6Oro3rAaPpPnouRimXgErB4NNfJ18OZt6ls1Rm9kRvaPWaLYvp35JxOMR3NHtDnEg/yqkHYSyAnNbLmkqsiLg6FxIP2Lypf3b9AeEBOzo9qOx0kl7xq6BrLqlhiWtkPo4fDXF0GEO+I4yvz0SyyXuT1jtC3Hr6ne970hRu6f9vWY1S9IMHHkaDj4syw5YCDrHX0xMAzopy4OjT0Hsn6Zf6xoOof+B0McbYEBNVixfzluLFrFLreacWs0iYDgwAarVNhxtZVX/2oaS6lz6DCLfg4R/TbrswQeFA9Ch/PFFoYA77oBXXoFOncTv//0vVEvGdOsuVHCUxtUtl1gwhckN70OrgZ5vmqeUksRy6L0Mbo4zj3xrhwch6JaG99OISMefpG6UNqBygNJsUBeJYwqFiHqQWB4qWyjJaNgGqrpEFEaPWGo+uyQNwtnZDP/fnEOh/X3gGNygbuys7Pj4po8r/g73Cqenr8wObdGUZIoH6/iaG2e/n/2dl7a/xHcnv6uzG5WNismfTWbG7zPYOG8jF9dfJLMoE4CpWWvgd3e5IWOB1Dm/OASK2lm1RNm2c2vHtDBRD3jUt6Po+klXlu5ZSqm6tEZbF1sXunh3wcfRRxyI+gY2DaoMMJK0PIoSxaZM+iGTLx0QIPSVDiccrjxYVgC7b4Gzb5vJQElzUa/1S06kcPxqyoy/xtoFBn4uovUl1w8unaHjo/WX/FQoRe0e+VzbtBQkiICzjGMN6qba/JJ+EDKPSalfC8HePomLF95gzQ/hhLYJoH94OIvfeIOkpCQTOmkDY7ZD+Py629a41h/6f9RwOeAqJCUlsXDBAtYVF1dk+M0FYhDOv/eAx4GlwC61msPHjkmnnzno/ipMvQz+ptVtCw2Fd9+Fn3+GH3+EH36Au+8GZW274FWeU82y/yJpHmJXw1/tIb7uwOVaUaqg/T3iR9J6sPcHh4DrRsFQoW11Remah5ycHFxdXcnOzsbFxaW5zWkcNGXwby8RSdvvw+a2RtKY5MfC9nHQdgb0eL25rZE0NQfniNqOHR6stdmCzQuY0HECo9uPFgdKMuH069BmoqjZJWk5qIshbb+o7ekYVO3Unqt7GPb1MLwcvEh+NhmlEQuktPNpfBz+MUOeH8K4t8eRVZSFQ9RX2BREQ78PGmkQkkZDXQJFSWDtCjauBpsl5iby5L9P8veFvylWFwPw860/V2b2GSJ1P6TuERv2Fi6VITFAWQEUp4lNOqVplQROJZ+i56cieOTwnMP0a9NPBJpd/UM4nX1HNIbFEkvmyJNw4SO4JRHspTyapBHJvSzmG5fOUlq4KUk/DBsHiEyKri+ar9+yglrVCSRNg04KcyjWPKQuwg9IAr6zs2NHaSlLly1rPIeYRg1xqyFwutn/Ty9+4w32LFnChqKiOttOsLdn+Isvsuill8xqg6SR+cMbfMdIBaOWTuZJOPAADPsdnELqbm+IskJZjqI1UlYIhfFC0te6Af6brDOw6xbouhA6PGTUJc3hO7o+3JtNiEajaW4TGo/idLDzBRvP5rZE0tg4BsHk89D9tea2RIKYV5KSkppmflEXQ8wvkHG0zqZvj3u70ukHwmEc+R6kHWg8+ySNg8pWyF1d4/QDGBgwECcbJ9IK0jiZdNKo7jzDPHnm6jMMenoQIOoG2nSZJ51+FohR80vSZlgbLKIna8Hf2Z/fZ/xO8rPJzOw+E4AtUVuqtUkvSOexvx/jrd1vVR70HgxdnpNOv5aMlQM4tjXZ6QfQxbtLxe//XPhH/KKyg/azpNOvhVPv9Uu7WTD4O7DSX1NUL7GrYd89kBdt2r0k1zdnXof13UBd2NyWXF+4doEx2yDEuI0yfeidX6TTr9mpKoW5TV3ELGAMMAvYWFTELrWat158kRXvvmtch1qNqPVnLDE/wZ4Z4pnUzKz98UfuMcLpB3BvYSFrfvzR7DZctxSlQsbxxr2HRi2Cn106Ne3+i8T8uPeEG482zOmn1YiswX0y26/VEb8O1oVCwoaG9aMpEesOhfHSwM0xp0jHn5lptQmUUd9C3BoYvQW6yagliydunSiA3FCkVIpFoNVqSUpKMs/8cvgJOPl/tbcZ+AUETBFSF1HfiGwfY3DtAtNiIPSxBpspaQa0Gr01+KxV1owIFpvvOifOz2d+5o2db6DWqPV2pVAocA1yxbmNlEixdIyaX1w6Q9gz4tUIXO1ceaDXA9zY8UYGBw2udi46K5pPj37Kh4ekckCroyBBRNiaiJXSivt63oezjTMP9q4901zSsqj3+sVrkJBVsjbB8ZdxDKK/B63+7yVJK+bAA3Dihfpd2/Z2EehoymdN0nCsHIWKkL1vvbuoNr/ErhZ14LRyk7450SeFeS0DgXXFxSx8/nnjZD+33wh/mbB5HzAZur1sdPaFKWTn5GBsDrovkJ2dbXYbrlv2zICtIxu3ZIRSBSP+gh6vm3f/RdI8KBQiq//q70KZylRKc8BnuNHPv5IWhFs3CH9elEJqCB594KaTEGJ8nejmmFOk409iHBc/hQsfismzHtHckiYm8n04Nr/+C6OobyFaRqi1ShL/hcRaIltUthB8B7S5UWT+HXgAThq5maK0Ehkfdt7msVXStPzVETYP13tqbMhYALZc2YJao2bFgRW8vONlfjrzU61dfnX8K8avGs93Rz+FnVPh8pdmN1vSBDh3gL4rRGaekYwNGcu/s/6t4ciJz40HIMAloPLg/vth2zhzWCppTvbcBtsn1OvSL6d+Sfrz6QS5lmcdx/wCa4IgaZsZDZS0anouhttzzVrPSdJCSN4Bafvqd23AZOj+slnNkRiJVgNFKebp6/RrcOQJQAatNicrv/iCUdbWBp1+OgYCI21s+HLlyro7bTMRgu+q26mrywq0cYMer4lXM+Pq4oKxFQqTAVdXw/L4EhPpMFs4dLUm1P6VSOL+hD23i9rRpmLjBjf8Ct0Wmd0sSTPj2gV6vw0evZvbkiZBOv4kxuE/DtrNbG4rJMbS+21RCLu+nFksfiStj5tOwYRDhs+XFVT+3vZ26PshdKsjQ1BHYbKQ2NLIBXmLpO2t4K9/017n+Nsds5tSTSkTOoh2r+18jbJa3u/9sfvZHLWZ5IyzwuGcfc78dktaFPE55Y4/5yqOP3VR9blH0jLp+DB0WVCvoCOVUoW1qopMitJW1Atsv38kfwAASbdJREFUSN0FScsl7RCsaQuXjNgUroq1ExhRh1bSypgcCWN3NrcVElPZOU3IiJsj+n3IjzDoG6lW08w0ihRm57kw4H+1z+1ZEfBPOJxfYaSl9WP0lCl8bWVcEPx39vbcPFPun5mN9rMgfD4oDUjqJW2Ds2837B4ZR+H4c5B9tmH9SCyHNpNgyE/g1KG5LZG0RlL3QuQHUJTW3JbUinwyMjOK1rrY7PG6eQtvSxoXj77g2a/+Dz8j/4ZBX5nXJkm9USgUeHh4mGd+sXI0/LnQaoWUyu7bxN9KFYQ9YXzEZOR7Qgc970rD7ZQ0Pb3fgV5L9J7q6t0VX0dfrJRWXEi/wPNDn8fLwYtLGZf4/tT3Brs8lyYcfQF+g+COYuipv39J82H0/LJ3psgkN5GE3AT2Xt1b8XdcThxwjePvhp9h/N5rL5W0NELuFxt05viuCroZJhwUaxlJi6Xe6xdrZ1FXXGVv/DVJ22RwyfWKyqZ+12nUsL4nnHrVnNZIjCXgJlHjT22co+haqs0vbl1FoLKkWWk2KUwbd3AIblg9rzpYsXw5K5YvZ2dZGQfraHsQ2FFSwkOzZzeaPZIqRCyFbWPgxMKGbcCnH4Fz/4WCePPuv0iaD9dwaHen6YpUmjIhMxz1bePYJWl+zi2Hv8PrvQYBIP5vOPq0SVKyzTGnSMefmVEq5T+pxEJoSAaFS5ioryKxCJRKJW3btjXP/JIfC4mb9H821AXgNxbcr0l515SKSJa6it/6DIfO88Dev+F2SiwKhULB7gd2k/58Oj18e+Bk48SCoQsAeGPXG5SqS/Vedz7tPADhXuHCGVDfzTlJo2H0/JJ+CLJOm9T3/tj9BLwbwO2/3V6hZ69X6lMikbRK6r1+cQ2HGw+LCH9j0Gph52Q4Ntd0IyUtn/yrELsGSrJMu64st/x5KbcxrJLURehj0P8jsDLBwV+FivmlNBPKCs1snKQ+NIoUZsZREXyWsttwG4c2MOEABE4z8u6msWL5ct5atIjdGg3LgClg0Pl3EJhia8vSZcvw8zPWDSqpk6I02DQETr9e81zs7+L1xqMNk3gNuQ+mXgbvIebdf5G0PHIvQeoeyItqbkskjYW6UNQFL4ivfx+d/iMUJxyDjL6kOeYUOYuZGY1GFpSWWADJO+EXe4j6xvRri9KgRBaitiQ0Gg1Xr141z/xyeaWowaRvEWPlCEO+r6ljXpwGJ18UEXC10WYi9FkupLYkLY+krbB3lkEN/FDP0GpSfI/1ewwfRx+iMqP47uR3Ndqn5qeSXpiOAgWd7WxE/6U5jWa+pH4YPb9MuQijN5nUd2//3thZ2ZGYl1jhBK5w/Oky/rRaiPxQ1GiStGwyT8LGgeapEXxlFZx7t+H9SJoVs65fakOrgX4fQ+h/Gvc+EsskdjXsvgWyTpl2nY0bTIkUa1dJi0M3v2iPL4RfHSxeaut6YNrMmayyszOqrdFSmCVZEPMT5JyHwiRQl4jj6hLYMaWyFnAjyTwnJSWxcMEC1hUXMxCYC7wADAcmAD8AW4DvgWHAcJWKF5YsYe68eY1iz3WLjZsI8ijLq3luwiGYFgMefUBpnBSrXlR2ImvUyrHp1i+SxiXnAqz2h4i3jGtfkgn5MeDaGW7LhPBnG9c+SfPRdRFMuQDODZCBdQgUyQ8q4773oHl8RtLxZ2a05tCol0gailN7CL5TvJrKiQXwuxvkXDS7WZL6odVqycjIMM/8EjAZ+n8qJLSMxd4fRm6AEX81/P4Sy6UgFmJ+hNzLRjV3tHFk4dCFgMj6KyqrLpOgk/kMdgvGPmEdbBsrFt8Si8Lo+aUeshR2VnYMDRoKwLYrYmOmosafLuOvLA+OPgVXpJRKi0dpK+aRUjMED136As4ubXg/kmalQeuXCx/D5S+Na6tUQYcHIHCK6feRtHz8J8Cgb8G5U3NbIjGFgjjYe1e9g0Uq5hevwdD+XrD1NLOBElOZPWcO20tLzSuF6TMc7igE+wBRE/LqL+K40hrS9wuptUZkxbvvMkyhYGCVY3OBGITz7z3gceB9IMnKirnz50unX2OgtIJb4qD3sprnFEpwbCsyfwvi6n+P3MsV2T9m3X+RNB/WLmJP1Jjvh5TdsNpPSMaCmGOsnRvXPknzYQ7JzdI8IRtvAs0xp0jHn0TSGnFsC0N/EhlYptL+bmg3C5w7mt8uSfPj2R9CH9Gvc77vHjirZzEN4HMDWDnU3vfOqXD8+YbbKGkegu+EGQWi5oqRPNrvUTp6dGRW91lotNWjl86lCsdfuFe4mIv6f9KwiCpJ85J9DmJ+NVkHf3T70QBsixaOvwOzD3D28bMMCiyXk1bZwZgd0Nn0+oESC8O1M9ySIOTbGsqQ72H05ob3I2m5nFsOFz5pbiskLQHXcAi5F+xNlNXLOg2XVoosIknTo7CCmJ8h80SDutG2vw8Gf2ueTTxJg/Dz82Pp228zxdbWfFKYSmuxVvQeDD4jRIYFiPd7wBeiRnkjsWL5cj555x0eKCurcc4PWAQcBi6Uv75aVsbWdesazR7JNWg1cPq1ylIEa4Nh753172/f3bCxv3lsk1gG9n4wfh90fLjutt5DYdA34De+0c2SWAhx6+BsA75Dto6EdZa/v9WAPGiJRNIq8R0lfiStG3VR9ZT0sgJI+Ae0NR9sKsi/KqJyOzxU03Go1YpFt9Ja/7USy8cEiQId9tb2nH38bDUJUB1FZUV42nsKx597T/EjablEfQ3n3oEpl0xy4I5pP4ZFLGLz5c08t+k55g2eR7h3eGUDpTX4jmgEgyUtGse24kdy/TJiLVgZKR1+5Qc4/YrY/Pce2rh2SVoPiRvh+HMwbp/pTkNJw7HzFQFn9azxJ7FM5s6fDwoFw59/npHW1txbVIQvkAR8ZW3HXk0pS+sjhWnjXlNyPugWc5ldg9defZUPFi/GE+HkMwZfIDtblkxpNLLPQvw/0G4mOARA2gE4/Sqoi6FXd+j4iMjwqi8h98myFNczCiW0u6u5rZA0JVFfQfw66PRE/dYi/jeaHBTdHCi0MnfZLOTk5ODq6kpmZiZubm7NbY5EIiIXsiNg8DfGtY/6BjwHiYh9iUWh0WhISUnBx8en4cVgtVohuagpgXHXFEnXasRi11BR7CurYP+9MOBz6DjHcP8y4rZlotVAxlFQ2pjVSVeqLtXrGJRYBkbPLxnHIfeCyN404aG6TFNGm+VtSC1IBSDmmRjaulZx6GjUYs5opNoskiYmaauow9OpAbXWtBooTARbL1DZms82SZNj1vVLbUT/BGffgsHfg3uPxruPxDLRamBdmFC1GGqCbGRBnAha8x7asM1iSbOg0WhIv7IPr6uLUXR6yiTFCknjk5SUxJcrV7Lmxx9JS8umsNCVgNCZ/P3PbPz9LdvR/torr/DG66+zF3gCeAaYZcR1PwDvhYdz+OzZxjTv+uXy13DwQRi+BgKnibk/7QDYtwGndma9VZOtXySNT+SHgALCnjDc5vz7Qi7eKaTJzJJYAFkRQkbYObTJ9iKysrJwd3cnOzsbF5emWXvKGczMyC8FicWQth+u/mqc5nD2OTg4Bw7cJxw3EotCqVTi5+dnnvlFoRALGsd2NT8bCqVhpx9AwFQY8Q+0v6/2/iUtFAVsHgonF9Xr6v2x+xn61VBOJZ+qdtxaZQ1bx8B2uSFjiRg9v3j0huA7TN4ctVJasf2+7Swfv5xnBj6Dv5N/9QaJ/8JPVnDlexMtl1gkFz8VmVcm1juoRnE6rAmEo0+bzy5Js9Cg9Yu6RDiAjfkstbsLbjolnX7XKwqlqOFj38a06xwCTQ5mkZiZnAsQu7pez59KpRJvhwIUyTugJN38tkkahJ+fH4teeonDZ89yPjaeIePO0ibgJUpLLdfpl5SUxE033sjS119nNDAQmAasMvL67+ztuXnmzMYz8HqnzU0wbg/4ijICKJTgPcTsTj8w8/6LpHm5+D+4WIt0fNYZOPYMnHq5yUySWAhuXcElrEkDkJtjTpFSn2ZGrW7ARodEYk4GfwtWjsZNYs4dYeKx8vbScWNpqNVqoqOjadeuHSqVquEdDvyi+t+aUrHQ6fBQ7bUdbVwNR9PmXYG0g0Kyz95ffxuJZaNQQJ8V4Bhcr8tXHFjBvth93LfmPjbfsxkvB6/Kk9auMnPHQjH7/KKHrj5d6erTVf9JWy8Imi42bSUtnw4PQue5oGzAZ0mhhE5PScnGVkCD5pcTCyFyBUyLkbKvkrq5VgLQGNTFcm3S3Jx7By6vhOmpYOdVd/sqqNVqogs60O62XFRyb96isbWFHj3g6FHx07aZp/SkpCRWfvEFa3/8kfTMTErUagoLCsgqKEAJhAG6MNfZwBuI2oQDa+nzILCjpIRvZ89uXOOvZ+x9xQ/Auf+C7xgRmKgjPwYOPSYkYA2pExkiL0ooG3V8DNrPapLnI0kTMfzP2kuauHaFMdtMDx6StA5Kc0SgoUuYidflwqFHIWCSkB82kubwGcklkkTSWrF2Nj5yQWkNbt1larsFk5ub23idJ2yAs0tFhqgx5FyAzJPVjyVvh313QcYx89snaTo6/QcCJtfr0mXjluHl4MWJpBN0/193Qt4P4d4/7xUnh6+GoT+Z0VCJOTFqfsmKgNX+DSuArQ+vQTDsd+nkaS20mSiir0F8V0QsNb0PW0/o9z4EzzCvbZJmod7rF5/houaGMbWDL3ws5MglElPY2B/WyyzRZqX9fUKitx51pqF8flEoZY3xFkDfvuL1n3/g0qXms2PF8uUEBwayZ8kSOp8/T1xyMk5paSgKCpgNjAFKqKzr5wcsBaYgnHv6OAhMsbVl6bJl+PlZbkZjq0BTJuQ9TyyEky9UP2ftAsnbID/a9H6LUiD3IpRVrlkadf9F0nS4hBkObNaVnPAdZbrjR9I6WN8Tdt9m+nXFqRDzI2SeMLtJ5kZm/EkkrRV1kVgU2XqBW7fa22aeFLrGVg5NY5uk+bn6u4iUG7leOHrGbBfO37ooyYZ/uoLfOBi1vvK431i44Vfw6Nd4NkssmnZu7dj9wG7GrRpHXE4cAEGuQc1slcRs2LiJ4BBbz+a2RNJSOPE8JPwLoY/WLiMtkegj6GbxYwwRb4JDW2h/T2NaJLFkUnZB/D8Q/pzxmWM+o0RtF0nz4XMDcEO9L3fJ3QFpueA7zFwWSRqJYcPgt98gORnmz4dbb4U77wQbm6azYcXy5by1aBG71Gr2qdW8BawBbgF2UVnP7z0gqcp1c8tfhwMjgXsB3/I2X5Vf+86SJcydN6/xB3G9s20MZJ+FG4/WDBiwcYcZefWb170GwfRkWfamNVJWAMVpYB9QXZEk/m9R4mTUBqlYdT3T6cn6XecUAncUgtbyVR9lxp9E0lopToOto0S9ndooK4ANfYS0geT6oTQHcs5DdkR5lNNI4zb0bVyh9zIh5VYVx7bQ9vZK+Q1Jy+T8ClgbAgUJ9bq8s1dn9jywh44eQjK2p29PKEyCM4sh/bA5LZU0NQ4BMH6vkHI0J9E/weEnoCTLvP1Kmp/wBTBmh5D6NYW4dbD7VlF/WCIxhjHba8qYS64v0g7AuWWQd9n4a/q9D32WN55NEuNRl9TrsqCExSiPP2tmYySNgZsbfPgh3HADaDTCCThnDqxdC8XFjX//pKQkFi5YwLriYoKBhcA64BgwCiHjmY3I8NNX128uEINw/r0HPA68D8RaWfHs889Lp19TEXSryBR27ao/Q6uhwRyy7E3r4+SLsDYYihKrH8+/KiQei2WN2Oua8Hnipz6o7ES5LAtHOv6u4eOPP6Zdu3bY2dkxcOBADh06ZNL1CvlFIbEU7AOgz3vQvg6HnqYUur8GwXc1iVkS01EoFAQFBZl3fmk3E6aniNR0U9PTO88F/3HVj8nouNaBwlo4d8vy691FsFsw+x7cx4oJK3hx2IuQEwmn/g/S9pvRUIm5aJT5xRSSt8HFjwE5h7Q6vAeLH1M/W7kXIPZPoVwgadE0aH7JvQS7b4fY1XW3dQkzTrVA0noJuR+mXAT33nU2lVgQmlL4u4soF2AiCqCw54dou79ifrskjYKrKyxYAC+8AF5ekJEBK1fCAw/Aa6/Bt9/C7t3iuLlZ+cUXjLK2ZiCwkkpn31pAlyvuisjimw1sp6a0px+wCDgMXAA+Aq5qtTwz95qAWEnjEfYU9PmvYQdf3hW4+Jlw6JhCwga4+kfFnkazPx9JzIfvKAh7BpTXpBd3ehymRNatjiaR6KMwCdIOioQKE2iOOUWh1crdWh2//PIL9957L59++ikDBw7kvffe47fffiMyMhIfH59ar83JycHV1ZXs7GxcXFyayGKJRCJpAAUJsDYI/G+Ekf+Yfr1WW7mhu3EwaIphoqzxJ7mGsnxR68veH+xl3YsWzcVPoSwPws0YXa8uEtl+dj7G16WVtBy0Wkg/JOTEbT2Mv05Xc0N+Jq5fss8KafEei6HbIsPtygrE94ytl4zUlxhP1hk4vxw6PCwCFCTNx67p4NgO+r7b3JZImpCyMti6tVL+81r8/SEwUPyuVIKDA7i4gJMTqNVQUiKWGL6+EBAA7u7iuFot+i4rq/xdrYaH7gjnhZjzzAL6IyQ9ZwGdgP8havstBvYAG4AVwFuIrMCBeuw/CNykUvHSsmUy28+SiPoODtwHN/wObW81/rotI8T3wm0y+6vVoymXZqwq+ym5fsmKgCOPi/Vg+1nGX3fxMzj8KIzaCP7jjb6sOXxHUtS+Cu+++y5z5szhgQceAODTTz/ln3/+4auvvmLhwoXV2hYXF1NcRZMgJ0d4eUtKSlCrxUSiUChQKpVoNBqq+ld1x3Xt6jquVCpRKBR6jwNoNBqjjqtUKrRard7j19po6LgcUwsck64fhaL1jKk1vk+1jEmtVnPp0iVCQ0OxtrY235hsfWHsARExp1YbP6aiFJSb+qNoexua3u+i1WpRuPUUUbsazXX7PskxGbBdYQeuPcTx8nu1+DEZcbyljEmtVnPhwgU6duyISqWqfUyXv0JbmICmU/XI5oaNyRpsvFFoQalAvk+tbUxxq1HsnYGm78doOz5i/Ji0lEddqy1vTK3xfWqkMenWL2FhYSgUCtPG5NQJ7e1FFWsUg2NK+BflntvQDPwGbbu7G31Mtdle55gs9H1qFWNSKtEWJqFRlwpp6rrGlHUKor5B7T9ZeAUscUyt8X3SN6ahv9VrTKXFhVy8FEXH0FBUKpVljcmA7XWNyaLfJzOPSaFQM3YsjBwJFy9CdLSCq1eVREZqiIpSkJAACQm6eI6a3x+6OI9r0xgMHU9MykEXfqiT9ITKLD8QmX5vIJx6tdX1+wbYAby0aBFPPf00mvLn39b4PrW4MfmMQjH8L5TeQ0wbU+/lqAsSKr4P1Go1ly9fplOnTlyLfJ9a+Jgyj8DmoWh7LUfb6YnWMabW+D412ZhAmRWBtiilYi/TqDF5D0fZYzEK51CTxlRWVkZTIx1/5ZSUlHD06FFeeOGFimNKpZKxY8eyf39NebK33nqL1157rcbxc+fO4ezsDICHhwdt27YlLi6OjCp6BX5+fvj5+REdHU1ubm7F8aCgIDw9Pbl48SJFRZXyRiEhIbi4uHD27NlqH6iwsDBsbGw4ffp0NRu6d+9OSUkJkZGRFcdUKhXdu3cnNzeXqKioiuN2dnZ07tyZzMxMYmNjK447OzvToUMHUlJSSEqqLG0sx9SyxpR55B1co97kcrsvKLQL0zum9rFzsXP0wHb0Hy1iTK3xfaprTFqtloyMDLy9vfH29jbzmKzFmBLOGj8mrYbOKh/s7NtUjsn2UTGmuLjr9n1qDWPy93LCN/9fkvNdSFH2NMuYrEpTUKuc0Srt5ftkgWNSqVQkJiZSXFxcIT1hcEwDV5KnduZylX+Dho7JrugyGqUtHm37yfepNY6p7VBcOs/nUo4/BVXsr21MZQk7uBobS75DL8scU2t8nxppTFqtluzsbMLCwkwfU3yCUWNq5+KCW8eHicl2JbuKnfJ9us7G1CUUxZo25LjcSEzgW0aMaSbJqj4kpWaizTxtmWNqje+TGceUf/glQmM/52LuN5TYdWgVY2qN75OxYxo8OIjJkz05f/4CmZklREXZkpurws/PH3t7R86du0pOjoL8fCUqlZa2bX1RKq04eTKZ1FRr8vKUqFTg5eWGQqEhNzcLlUqLSgVWVuDk6EJSsahhXtXZp6vnNwvhDFwKTEFk+s0F7gK+RNT1ywZsgCtKJXPmzOHmW27h9OnT19X71DLG1JO2tp7EXb1ac0xxb1KUco6Lbf57zZj6cTH1PEXldmq1WkpLSwEsZEyt8X1qmjFdPPwH/imfkOZ+K7nOw+geaA1+k4hOtyK3fLwtbUyt8X1qvjGVQsctUAx+KSkmjulJXJxcOHv6tNFjKm6KorbXIKU+y0lISCAgIIB9+/YxeHCl3Mfzzz/Pzp07OXiwusK3voy/oKAgUlNTcXd3B64Hz7gck8WPKXYNivPvoum5DDz767VduX0MWDmiGPlPyxhTa3yf6hiTWq0mIiKCbt26mTfjT75PckzXHi/NQrnaC03IQ2j7f2aWMSl3T4PkrWimZ6G0smn6MbXG98nMGX+nT5+ma9eudWf8NcKYlP+Eg8oe7cTj8n2SYxLH/+0DpVloJl9qPWPSc/x6GJNu/dKjRw/TM/40GrSp+0FbCt7DLGZMtdlu1Jgs8H1qNWM6Og+tW89qmZ8tfkyt8X26dkxaDarTL6JV2qPp9orRYyq78AWFZz/FdtwGVHYeljUmA7bXNSaLfp9a+JjeWrKEPUuWsKGoqJqkZxIQDOyiUtJzBbCQmpl+nwGHVCreWrqUp6vU9ZPvkwWOqTgVja13zeN7bkWbexnN+CPopB4VWg1KpQK1tlIuvOr65Vrk+9TCxpSyF+W2kWh7v4s29D+tY0yt8X1qSWMqv0apUpk8pszMTDw9PZtU6lM6/sox1fF3LTqd1oyMjArHn0TSYtBqK3UxJBaHbmO+e/fuFRvzFkXeFTj/HgTPAO+hzW2NpCFotZCyE5w6gGOQefq88DHkRUGf5ebpT2JWTJpftFrIjxGyvi6h5jHgwiegtIKOD5unP4llotVAUSrY+9bdNmEDFKdB+7vrbiuxaBq8flnXCZS2MOl03W0lElNI2gJOIeJH0vz80x0USrjppNGXWPzzkcSiSEpKIjgwkF1qNcFUd/atoGY9vyREpt8aIBXIVCgYOmECX339NX5+sma5RXPkSbjwEUxPBTuv6uc0amrUdkvZDVtHQf9PKp5H5PzSitBqAIXc75QYJnGzePZsd5dx7fNjYENf6PYqhD1RZ/OqZGZm4uHh0aSOP2WT3KUF4OXlhUqlIvmaysLJyckmfbHrPLsSSYtCfglaNEqlkpCQEMuaX1L2wJGnoCAess/BhQ8g53xzWyVpKAoF+I6sdPqVZMOxZ6Eopf59dvqPdPpZMCbNL2W58Fd7OP2y+Qzo9Lh0+rV2tFpYFwYHZxvXvs2N0unXSmjw+qXHG9CjZmmFahx/Dg49Wr/+JdcnmjLYcRMcebq5LZHoGL0FJh436RKLfD6SWCx+fn4sffttptjaEkOlpKeunt8LiHp+E4Af4P/bu/fwKMq7jeP3JiEJJNkNxyScj4IoIiAFLJ4RQm052FcsVVRebUs9VClSa1WktVXUtlK1FeyrFW2rVqu0RQERBaUgJ0EOSkQOBoFwSrJLCCHJ7vP+MbKwEJIAMzvJ8P1c116FncnMby3Xndn5zfM8WierOZiRlKSdiYma/Nvf6u3Zs2n61QctLra+W0S+nqEtfEia2U5a+yur6VeyVfrvaGnrK9b2xIZSq29L6Z2jhyBfPMSXEHu/c8NU6/4GY6Bw2LpfSSt/Uvv9y4uk9C5S8skP+nIjU1jj72vJycnq06eP5s+frxEjRkiyhmXOnz9ft99e+w6ujwYK6pqN06WyAqnHg8dv2zZTKtspdbhJSmoY78pQSz6fL25Pg9RacJ30+VNSy29JOYOlEdukpAy3q4IdTMR64im1hbT939KG30mmUuoz1e3K4ICTypcGfuncB6XGPWveFzjM55OaXyildah530iFlNDA+ZoQF6d9/dLu2pr32f2BVBE69XPAOzY9J22eIV30xvGjPI5mwlLfaVJqLUYgIz5qMxr8aOVF8q28S/62oyT/Vc7UBM8ZP2GC5PPp4p/9TJc2aKDBZWW6SFbDb6ystf5ek/RTn08HjFFWixb63zvu0N9vuYWGX33S9hrrdVjZLim1uTXDiCQlpkpfviI1aiu1/57U9ALp4pkxh6iT919w6grmS0lpUrP+Uv5rUskmqfdva/45nBl6/sb6Dlpbjc+Xhiw5pVO50TNiqs+jvPrqq7rxxhs1ffp0feMb39DUqVP1j3/8Qxs2bFBWVvUXo0z1iTpr/hXSvmXSNaHjR/a9P1Ta9b40aj832uqwcDisTz/9VN27d687U00cKpQqS6RGra2nqOAdi66V8v8hfa/cekKyeI0U6C4lZ578sTbPkLb+XbrgafumhoStXM2Xg7ukhd+ROt5ojQwFVtxp5c/Q1Sd/Ixh1TtzyJVwuJSY7d3zUD+t+LeVNlQZ9KAXOdrsanAxjvm7iB6XWw2ref/8mmdk9tbP5rcq6+JG68/0I9UJBQYGe+7//08y//137CgtVHg6rgc+nxMRENW7cWCO+/33dTLPPu4yx7mM0OPFDy3Xy/gtO3T8ypGYXSpfPta4Zy/dJDXPcrgpnIDem+mTE31GuvfZa7dmzR5MmTVJBQYHOP/98zZkzp8amH1Cn9XvOGqVxdNNv25vWaJ7+z0v7N9H0qweOXTDWdSlNrJdkrd8mn5TWnmljvSBnsJTS1Gr6NUi3RuqcqtJt0p4PrAxCnXVa+VJeLC26Rjp7ovVv52RUllgj0iuCp35+eEuj1pL/LOsaBZ5wWvmyYao16nzQwurXYqPpB0k65z7p3PvdrgKnwuf7ejpoU33j7+BO62ZtRidFLvqP9uxsKO7U4GRlZ2frvvvv1333kxeetfaXUtEn0sVvHL/N54tt+q1/WKo8KPV8KGa3Onf/Bafugj8eafQlJtP0w/GMsWa+OnYN0Kr2W36rlDNEajMiLqWdLoZpHOP222/Xl19+qUOHDmnp0qXq169fzT8E1GXp7Y80aCQrqFbcLi292fqF12Kga6WhnivdLhWvlVaOl/7TRRIDyD2h083W4uYr77QuasLlUijv+HnwI2EpXCYVr5PWPmQ1Co917v3SNSWM3PGS0OfS2+dJG56w/r7rfangXWn3wpM/VkYnaUS+dM4v7K0RdU/hKumjsdLepdXv132i1eThIRJIUlK61LDliaff2famtH2W9UUdqG1uLL9NmtVdqihxth6cnD5TpQEvWtebm2dIX/07dvuhfdLcftKBbdbfW1wsk5Aa9zIB1AOhPGnvEquht2qidb1wtNId1hp/ZXukrX+Ttr3uTp2Ij443SDlXSpUHpD1LrNmrgMPy/ym9miJt/0/N+x7YIn0xTdq9wPGy7MKIP8DrImGp5AtJCVJ6B8mXaK19UV7kdmWo794bZN1s6/Gg1KQ3U356SaTSauomB6wHBTb9Wbp6t7U+wmGrfybt/cgaEfjZb6Xm35SyLz/+WDU9NYX6pWG2tZ7W4RvtbUZK/1NsrZcBnEh5obT5Balpf6kZD9WhljrfYr1OZM0D1r+tEdvjVxPqrkhY2jHLWscne9CJ92vgt5rKDdLjVxtq1uqotfpWTbQa/t/ZeGS9xtDn1gwBO96Suoxzp0YA9UP/563vJmW7re+pHcda31kO2znHehD+ojel3I+tawl4X9Eaad6F0vmPWQ8bApLUqI2UPaR2S9ukd5Su3mXdL6snWOPPJofX+CsuLlYgEHC7HOCIki3Sv7+eHqnx+dLF/5LS2rpaEk6OMUZlZWVKTU11ZTHYE9o4XTJh6axb3a4EdgptlNY+KLW7Vsq6XNr1nrTnv1K3CUdG7hlzpPHX/y/W1EvN+sdOG1y6Q/pqpjUNQkYnVz4KauZqvoQ+t9agzb6CKVe8LnxICpdKydWsg71znrTtn9LZd0sZneNXGxzjeL4Ur5dKv5JaDrH/2Kh/jJFey5CaDZAun+d2NTgduz+UmvY9/qGiSFgyFVJiat39fgSgbindIUXKrZmwDjuQL+1ZJGVdVuV3EPLFY1bcKW17TbpysZT/mvWwcpM+bleFM1AwGFRmZiZr/AGwUaO2Uvd7rBv3lSXcXK2nkpPr4Po1XX7kdgVwgqmQvnxZ8p8ttR5+5HU0n0/q9bh1Mz8xpeqb9HsXSytuk/rPoPFXx51yvnxyn5SaJaVmS58/LV3415N7sGTXfGs62cvn87vJ6xJTrFd19nwofTFd6npnfGpCXJz29cvmF6wpu85/5PhtmedYL0Cyrk0GvGQ9uY36rcVFVb+fkCjpyEwSdfL7EYC6Yftb1sjhqtbhSmsrpX3fmvJx70fW997k2AEc5IuHJGdKaR2s2YsY6YfTsftDaya9Rq3drqTWmJfNZpEIa0ygjklIlM6fIl35oTRkeeyIHNQLkUhEa9eurZv5UrZbmvMNaeM0tyuBXTK6StcelHo8UPV2Ezmy3t/hG/kmIhWtlsqDR/bLvkK6dLY1nz7qrFPKl/1fWNNwrX/YWjOjotiaGrZ028mdvOW3pItnSpnnndzPoX7av8ka1XciPX5pTdmYcVb8aoKjbLl+yX/derDg2PXYQnnWOrPA0dqMlJpecOLtwQ3S6l9Y6xOjbiteL336uHWNWRGSPn1UCn4W3Vynvx8BcN+KO6QPR0oHd514n51zpHcGHLe2F/niMef9Uhr8X2sqcKAqnzxgXXNUx0SkBd+SFl9/yqdxI1No/AFnkgQG+cJGxWul/3SRCpdL5cVuVwO7JCQeP7XS4uulZV+P8Nw5V5rZWtr+9pHtW/4qze4l7Zh95L3kxlLLXEZyedHBHdZ6GT1+aa2h0fF/pf8ptNZ5PBlp7azRpIfX74G3rbpbWjDUevq6Kj6f1Kgl64Ii1gVPSSO3H78e2/u50py+7tSEui1SeeKc2btE+vQRqWRTfGvCydv6V2ta+aJV0p7F0uqfW9egAFAbFzxt/e/i71e9fflt0uLrpB6TpaasP31GWPZj6Z1vHlmrHjhs69+kL/9e/T6RSqnPVOmsO+JSkl3oAgAATk1SuvU67yGp60/crgZOCn1+5Am5SLmU3CS2oZd9hbUu19FTrh0ssKaBZF0E72nSVxq6Sgqcc3qjyI3h38eZpPMPpVbfsdaG1TH/bg7tk/YutdYKTWniSnmoo9I7HP9epFLq/COe3Mbx8l+3bvIOfF1qPez47e2/LzX9hpTGdKB1Xsf/lXIGS4FzJX9EuuI9KaOL21UBqC+yL5f6TpNSTvCAYWYPqe0oqfO4I+vYw5uCn0pfzZSCa601x32MgcIxrvyw+rXoJSkxWep0c3zqsRGNPwDAqUnvYD2FD+8b8tGRC+Sq1vxr1Mpa8++w8qD0Zo51YdTv/+JXJ+IjqaHU+PzY93YtlErzpQ5jan+cxd+3Rol+dy8j0s8ELYeeeNvuhdKH35X6PS91Ghu/mlA/lO2V8l+VWn3bGimckCSd83O3q0JdlN5BysmVGvir3p6YwrqQ9YW/i/U6LOsy92oBUP8kpkpdfnTi7V3GWS94X/E6a236b74itbvW7WpQFzVqVf324nWSL0kKdItPPTbyGXN4oR6cjlAopEAgoOLiYgUCgZp/AABqyRijSCSihIQE+era6Jg9S6QDW6RWw46fhgtnHmMkU2ktlL52stR8oNThOrerQjVsy5f3rrSm4hq1v/ZPUa77tbT3I+nSWad+XnjDgXxpx9vWDfv09m5XA5vYli/b35IWfls679fWqNHQBqndKPsKxZmj6BMprb2UzPf1esEYqehjSQnWA0dH5Uid/n4EoH5YcJWU2FC66PWYt8kXjzlUKB3YKqV34vc/qlZeJJVslvxnS0mNYrcZI837pjX1+PBtp7VMSTAYVGZmpoLBoPz+EzykZjPGtwJAPVBeXu52CVVbO9maG7/ygNuVwEmHCqXNL0obp0nLxlnTZRyraI30coK04ffWdCnfeIamXz1hS76c+6B06dvWhbEkVeyX1k+Rtr1Rzc/cT9PvTFJeJM3qJq0cf/y2tLbWU9c0/TzHlnzJGSxd+Hep6x3Sf0dJy35gjSwHTkblQWlOH+mjm9yuBLWV96Q05wJpTm8pb+pxm+vs9yMA9cOexVLR6io3kS8ektJEatRW2vTcCf//xhlu43TreiO4/vhtPp/U+/fS+Y+eVtPPLTT+bBaJsEgoAHtFIhHl5eXVzXzpNl7qOl5KbeF2JXDSwZ3SRzdKK38ifTFdqggdv09GJ+tp7ETWXKpPbMuXFgOlrEukhERr/S0Tltb/WvriWXsKRf3XINN6qrpBRuz7pdtp4niUbfmS0EBqP9qavrHfX6TL5/PENk7s8z9Ky6qY3s1UWOtSt78+/jXh1ORcKeUMsV5ZV8RsqtPfjwDUD5fNsR5cPAb54jHGSIUrpVUTpIL33K4GdVHWZVLP30ipJ1jvs1l/qetPTvs0bmQKC6oAAE5dy1zrBW/L6CINfF1qeoF14VzVHOhJadLQVfGvDXVHJCyFD0if/U7a/m9pwEtSy6tOvP/qX0jpHaXOt8SvRrjH56s6I1b/3BoZOmyzNVoYqE7zAW5XgLpu90IrU/o8aa3pd1gDv3TOve7VhZMX6G7dmAcAJzTr53YFiIeKkLQgV/J3ldr+j9vVoC5q1q/qPChaI4XLpKZ9Y6Ybr09o/AEAgOolJkttv+t2FajLKg9I/2wutfmutX5CpNxa+zMh8cQ/s+H3UvYgGn9nupxc64Y8TT8Aduj7jPXgydFNPwAAcGZq4Jc63iQ1v9haXgCorfW/kfL/IY3YJjVq7XY1p4SpPgGgHkhMrObmORAPJVukrS8zJZ8H2ZIvSWlS6xFSkz7SeZOlb62zmn4Hd0qf3Gf9+znW8C3SN5gK9IxSuNIa6Xkg/8h7Ha6T+v7RvZrgKK5fEHcpTatu+r17qbTkpnhXAweRLwCcQr54iM8n9fmD1GGM25WgrqosleZdJH1yf+z73X8m9Z5ab5t+Eo0/2/HLAYDdEhMT1aNHD/IF7lo1UVr8fWnX+25XAhvZmi/f/LvU7S7rz4enwiheJ61/WNry0vH7N8yRGrU8/fOi/ij6RPr0EalotTVtcNletyuCg7h+gSvC5dLeZVLwsyPvRcLW2rMm7F5dsBX5AsAp5IsHLblJ+keadT0AHCuxoXRgq1ReHPt+kz5StzvtO40LmULjz2bGGLdLAOAxxhiFQiHyBe7q+4zU/wWp9XC3K4GNbM2XygPSnL7Slr8eeS97kHTZO9K5xzw9F/xMKmRNyDNO6+HSVeulnCHS/s+lN5pLm/7idlVwCNcvcEV5kfROPylv6pH3EhKlKz+ULqziIRTUS+QLAKeQLx5UuV9KSq9+GQqcuXw+azrPvk8fea94rWQitp7GjUyh8WezSMTefxQAEIlEtHnzZvIF7kptLnW8sd4uaoyq2Zove5dKhStip/X0+aScKyXfMZecn/xCmtNHKtl6+udF/ZHSVAp0t6bhM2GpURup+UC3q4JDuH6BK1JbSD0fkdqNdrsSOIh8AeAU8sWDBrwkDV3tdhWo6w43+kp3SG/3lJb9yNbDu5EpSXE/IwAAALyn+UBpwItSu+8dv61ks/T5H6Wev5ESU6Uek63909vHu0rUFYHu0oj8mvcDgJPh80nn/Dz2vS/+LEXKpS638gATAABnmobZbleAum7vMmnb69KhPVLPh6Wz75ayLne7qtNG4w8AAACnLzH5xIumf/VvacPvpab9pHajpMY9rRcAAE4wxnpyOyHR+v0TKZfOus3tqgAAAFDXJDSQitdJ/q5Swxyp12NuV2QLGn8AUA+kpqa6XQIAj4pLvnS6xWr0pXeSdr0vtbiUURfAGYDrF7hi20xpxe1Shxuk8x+WBn0gHdzudlWwGfkCwCnkC3CGadJLuuxt68ExD/EZViu1RSgUUiAQUDAYlN/vd7scAACAumf5bdLGP0lXLpaaD3C7GgCAF4XLpQ+GS13GSa2Hu10NAAAAznBu9I4Y8WczFn8FYLdIJKKioiI1btxYCQkJbpcDwEPini/dfio1bCk16+/8uQC4iusXuCYxWbpstvXn/V9ICSlSo9aMNPcQ8gWAU8gXAE5wo2dEgtmMAZQA7GaM0bZt28gXALaLe75kdJLOvY+br8AZgOsX1Anrfi39q61UEXS7EtiIfAHgFPIFgBPcyBQafwAAAAAAwHvS2kvtr5OSM92uBAAAAIgbpvoEAAAAAADec95ktysAAAAA4o4RfwBQD2RkZLhdAgCPIl8AOIV8AeAU8gWAU8gXAF7gM0xabItQKKRAIKBgMCi/3+92OQAAAAAAAAAAAHCRG70jRvzZLBKJuF0CAI+JRCIqKCggXwDYjnwB4BTyBYBTyBcATiFfADjBjUyh8WczBlACsJsxRgUFBeQLANuRLwCcQr4AcAr5AsAp5AsAJ7iRKTT+AAAAAAAAAAAAAA+g8QcAAAAAAAAAAAB4AI0/m/l8PrdLAOAxPp9PTZo0IV8A2I58AeAU8gWAU8gXAE4hXwA4wY1M8RkmLbZFKBRSIBBQMBiU3+93uxwAAAAAAAAAAAC4yI3eESP+bBaJRNwuAYDHRCIR5efnky8AbEe+AHAK+QLAKeQLAKeQLwCc4Eam0PizGQMoAdjNGKPCwkLyBYDtyBcATiFfADiFfAHgFPIFgBPcyBQafwAAAAAAAAAAAIAH0PgDAAAAAAAAAAAAPIDGn818Pp/bJQDwGJ/Pp+zsbPIFgO3IFwBOIV8AOIV8AeAU8gWAE9zIFJ9h0mJbhEIhBQIBBYNB+f1+t8sBAAAAAAAAAACAi9zoHTHiz2bhcNjtEgB4TDgc1qZNm8gXALYjXwA4hXwB4BTyBYBTyBcATnAjU2j8AUA9sH//frdLAOBR5AsAp5AvAJxCvgBwCvkCwAto/AEAAAAAAAAAAAAeQOMPAAAAAAAAAAAA8AAafzbz+XxulwDAY3w+n9q0aUO+ALAd+QLAKeQLAKeQLwCcQr4AcIIbmZIU9zN6XEICvVQA9kpISFDTpk3dLgOAB5EvAJxCvgBwCvkCwCnkCwAnuNEzoktls3A47HYJADwmHA5rw4YN5AsA25EvAJxCvgBwCvkCwCnkCwAnuJEpNP4AoB4oKytzuwQAHkW+AHAK+QLAKeQLAKeQLwC8gMYfAAAAAAAAAAAA4AE0/gAAAAAAAAAAAAAPoPFnMzcWagTgbQkJCerYsSP5AsB25AsAp5AvAJxCvgBwCvkCwAluZEpS3M/ocT6fz+0SAHiMz+eT3+93uwwAHkS+AHAK+QLAKeQLAKeQLwCc4EbPiMcXbBYOh90uAYDHhMNhrV27lnwBYDvyBYBTyBcATiFfADiFfAHgBDcyhcYfANQDXHQCcAr5AsAp5AsAp5AvAJxCvgDwAhp/AAAAAAAAAAAAgAfQ+AMAAAAAAAAAAAA8wGeMMW4X4QWhUEiBQEDFxcUKBAJulwPAQ4wxKisrU2pqqiuLwQLwLvIFgFPIFwBOIV8AOIV8AeCEYDCozMxMBYNB+f3+uJyTEX8AUA8kJye7XQIAjyJfADiFfAHgFPIFgFPIFwBeQOPPZpFIxO0SAHhMJBLR2rVryRcAtiNfADiFfAHgFPIFgFPIFwBOcCNTaPwBAAAAAAAAAAAAHkDjDwAAAAAAAAAAAPAAGn8AAAAAAAAAAACAB/iMMcbtIrwgFAopEAiouLhYgUDA7XIAeIgxRpFIRAkJCfL5fG6XA8BDyBcATiFfADiFfAHgFPIFgBOCwaAyMzMVDAbl9/vjck5G/AFAPVBeXu52CQA8inwB4BTyBYBTyBcATiFfAHgBjT+bRSIRt0sA4DGRSER5eXnkCwDbkS8AnEK+AHAK+QLAKeQLACe4kSk0/gAAAAAAAAAAAAAPoPEHAAAAAAAAAAAAeACNPwCoBxITE90uAYBHkS8AnEK+AHAK+QLAKeQLAC/wGWOM20V4QSgUUiAQUDAYlN/vd7scAAAAAAAAAAAAuMiN3hEj/mxGHxWA3YwxCoVC5AsA25EvAJxCvgBwCvkCwCnkCwAnuJEpNP5sFolE3C4BgMdEIhFt3ryZfAFgO/IFgFPIFwBOIV8AOIV8AeAENzKFxh8AAAAAAAAAAADgATT+AAAAAAAAAAAAAA+g8QcA9UBqaqrbJQDwKPIFgFPIFwBOIV8AOIV8AeAFPsNqpbYIhUIKBAIKBoPy+/1ulwMAAAAAAAAAAAAXudE7YsSfzVj8FYDdIpGI9u3bR74AsB35AsAp5AsAp5AvAJxCvgBwghuZQuPPZgygBGA3Y4y2bdtGvgCwHfkCwCnkCwCnkC8AnEK+AHCCG5lC4w8AAAAAAAAAAADwABp/AAAAAAAAAAAAgAfQ+AOAeiAjI8PtEgB4FPkCwCnkCwCnkC8AnEK+APACn2HSYluEQiEFAgEFg0H5/X63ywEAAAAAAAAAAICL3OgdMeLPZpFIxO0SAHhMJBJRQUEB+QLAduQLAKeQLwCcQr4AcAr5AsAJbmQKjT+bMYASgN2MMSooKCBfANiOfAHgFPIFgFPIFwBOIV8AOMGNTKHxBwAAAAAAAAAAAHgAjT8AAAAAAAAAAADAA2j82czn87ldAgCP8fl8atKkCfkCwHbkCwCnkC8AnEK+AHAK+QLACW5kis8wabEtQqGQAoGAgsGg/H6/2+UAAAAAAAAAAADARW70jhjxZ7NIJOJ2CQA8JhKJKD8/n3wBYDvyBYBTyBcATiFfADiFfAHgBDcyhcafzRhACcBuxhgVFhaSLwBsR74AcAr5AsAp5AsAp5AvAJzgRqbQ+AMAAAAAAAAAAAA8IMntArzicNc2FAopMTHR5WoAeEk4HFZJSQn5AsB25AsAp5AvAJxCvgBwCvkCwAmhUEhSfEf+0fizyb59+yRJ7du3d7cQAAAAAAAAAAAA1Bn79u1TIBCIy7lo/NmkSZMmkqT8/Py4/Z8H4MwQCoXUpk0bbdu2TX6/3+1yAHgI+QLAKeQLAKeQLwCcQr4AcEIwGFTbtm2jPaR4oPFnk4QEa7nEQCDALwYAjvD7/eQLAEeQLwCcQr4AcAr5AsAp5AsAJxzuIcXlXHE7EwAAAAAAAAAAAADH0PgDAAAAAAAAAAAAPIDGn01SUlL04IMPKiUlxe1SAHgM+QLAKeQLAKeQLwCcQr4AcAr5AsAJbmSLzxhj4nY2AAAAAAAAAAAAAI5gxB8AAAAAAAAAAADgATT+AAAAAAAAAAAAAA+g8QcAAAAAAAAAAAB4AI0/AAAAAAAAAAAAwANo/B1l//79uuuuu9SuXTs1bNhQF154oZYvXx7dbozRpEmTlJOTo4YNG2rQoEHauHFjzDEKCwt13XXXye/3KzMzUzfffLNKSkpi9lmzZo0uuugipaamqk2bNnrsscfi8vkAuKemfHnjjTc0ePBgNW3aVD6fT6tXrz7uGGVlZbrtttvUtGlTpaen67vf/a527doVs09+fr6uuuoqNWrUSC1atNDEiRNVWVnp9McD4KLq8qWiokL33HOPevToobS0NLVs2VI33HCDduzYEXMMrl8AVKWm65fJkyerW7duSktLU+PGjTVo0CAtXbo05hjkC4Cq1JQvRxs3bpx8Pp+mTp0a8z75AqAqNeXLTTfdJJ/PF/PKzc2NOQb5AqAqtbl++eyzzzRs2DAFAgGlpaWpb9++ys/Pj26P1/1dGn9HueWWWzRv3jy99NJLWrt2rQYPHqxBgwZp+/btkqTHHntMTz75pKZNm6alS5cqLS1NQ4YMUVlZWfQY1113ndavX6958+Zp1qxZ+uCDD/TDH/4wuj0UCmnw4MFq166dVq5cqccff1yTJ0/Ws88+G/fPCyB+asqXAwcOaODAgXr00UdPeIzx48frP//5j1577TUtXLhQO3bs0NVXXx3dHg6HddVVV6m8vFyLFy/WjBkz9MILL2jSpEmOfz4A7qkuX0pLS/Xxxx/rgQce0Mcff6w33nhDeXl5GjZsWMwxuH4BUJWarl/OOussPf3001q7dq0WLVqk9u3ba/DgwdqzZ0/0GOQLgKrUlC+Hvfnmm/roo4/UsmXL445BvgCoSm3yJTc3Vzt37oy+Xn755ZhjkC8AqlJTvmzatEkDBw5Ut27dtGDBAq1Zs0YPPPCAUlNTo8eI2/1dA2OMMaWlpSYxMdHMmjUr5v3evXub++67z0QiEZOdnW0ef/zx6Lbi4mKTkpJiXn75ZWOMMZ9++qmRZJYvXx7dZ/bs2cbn85nt27cbY4z505/+ZBo3bmwOHToU3eeee+4xXbt2dfLjAXBRTflytC1bthhJZtWqVTHvFxcXmwYNGpjXXnst+t5nn31mJJklS5YYY4x5++23TUJCgikoKIju88wzzxi/3x+TOQC842Ty5bBly5YZSebLL780xnD9AqBqp5IvwWDQSDLvvvuuMYZ8AVC12ubLV199ZVq1amXWrVtn2rVrZ5544onoNvIFQFVqky833nijGT58+AmPQb4AqEpt8uXaa681119//QmPEc/7u4z4+1plZaXC4XBM91WSGjZsqEWLFmnLli0qKCjQoEGDotsCgYD69eunJUuWSJKWLFmizMxMXXDBBdF9Bg0apISEhOiUN0uWLNHFF1+s5OTk6D5DhgxRXl6eioqKnPyIAFxSU77UxsqVK1VRURGTQd26dVPbtm1jMqhHjx7KysqK7jNkyBCFQiGtX7/ehk8CoK45lXwJBoPy+XzKzMyUxPULgKqdbL6Ul5fr2WefVSAQUM+ePSWRLwCqVpt8iUQiGjNmjCZOnKhzzjnnuGOQLwCqUtvrlwULFqhFixbq2rWrfvzjH2vfvn3RbeQLgKrUlC+RSERvvfWWzjrrLA0ZMkQtWrRQv379NHPmzOi+8by/S+PvaxkZGRowYIAeeugh7dixQ+FwWH/961+1ZMkS7dy5UwUFBZIU8x/88N8PbysoKFCLFi1iticlJalJkyYx+1R1jMPbAHhPTflSGwUFBUpOTo7eqD/s2AwiX4Azy8nmS1lZme655x6NHj1afr9fEtcvAKpW23yZNWuW0tPTlZqaqieeeELz5s1Ts2bNJJEvAKpWm3x59NFHlZSUpJ/85CdVHoN8AVCV2uRLbm6uXnzxRc2fP1+PPvqoFi5cqKFDhyocDksiXwBUraZ82b17t0pKSjRlyhTl5ubqnXfe0ciRI3X11Vdr4cKFkuJ7f5fG31FeeuklGWPUqlUrpaSk6Mknn9To0aOVkMB/JgCnh3wB4JTa5ktFRYVGjRolY4yeeeYZl6oFUJ/UJl8uu+wyrV69WosXL1Zubq5GjRql3bt3u1g1gPqgunxZuXKl/vCHP+iFF16Qz+dzu1QA9UxN1y/f+973NGzYMPXo0UMjRozQrFmztHz5ci1YsMDdwgHUedXlSyQSkSQNHz5c48eP1/nnn6+f//zn+va3v61p06bFvVbuOB+lU6dOWrhwoUpKSrRt2zYtW7ZMFRUV6tixo7KzsyVJu3btivmZXbt2RbdlZ2cf9yW3srJShYWFMftUdYzD2wB4U3X5UhvZ2dkqLy9XcXFxzPvHZhD5Apx5apMvh5t+X375pebNmxcd7Sdx/QLgxGqTL2lpaercubP69++v5557TklJSXruueckkS8ATqy6fPnwww+1e/dutW3bVklJSUpKStKXX36pCRMmqH379pLIFwAndrL3Xzp27KhmzZrpiy++kES+ADix6vKlWbNmSkpKUvfu3WN+5uyzz1Z+fr6k+N7fpfFXhbS0NOXk5KioqEhz587V8OHD1aFDB2VnZ2v+/PnR/UKhkJYuXaoBAwZIkgYMGKDi4mKtXLkyus97772nSCSifv36Rff54IMPVFFREd1n3rx56tq1qxo3bhynTwjALVXlS2306dNHDRo0iMmgvLw85efnx2TQ2rVrYy5QD9/gP/aXDgDvOVG+HG76bdy4Ue+++66aNm0a83NcvwCoyclcv0QiER06dEgS+QKgZlXly5gxY7RmzRqtXr06+mrZsqUmTpyouXPnSiJfANSsttcvX331lfbt26ecnBxJ5AuAmlWVL8nJyerbt6/y8vJi9v3888/Vrl07SXG+v2sQNWfOHDN79myzefNm884775iePXuafv36mfLycmOMMVOmTDGZmZnmX//6l1mzZo0ZPny46dChgzl48GD0GLm5uaZXr15m6dKlZtGiRaZLly5m9OjR0e3FxcUmKyvLjBkzxqxbt8688sorplGjRmb69Olx/7wA4qemfNm3b59ZtWqVeeutt4wk88orr5hVq1aZnTt3Ro8xbtw407ZtW/Pee++ZFStWmAEDBpgBAwZEt1dWVppzzz3XDB482KxevdrMmTPHNG/e3Nx7771x/7wA4qe6fCkvLzfDhg0zrVu3NqtXrzY7d+6Mvg4dOhQ9BtcvAKpSXb6UlJSYe++91yxZssRs3brVrFixwowdO9akpKSYdevWRY9BvgCoSk3fj47Vrl0788QTT8S8R74AqEp1+bJ//35z9913myVLlpgtW7aYd9991/Tu3dt06dLFlJWVRY9BvgCoSk3XL2+88YZp0KCBefbZZ83GjRvNU089ZRITE82HH34YPUa87u/S+DvKq6++ajp27GiSk5NNdna2ue2220xxcXF0eyQSMQ888IDJysoyKSkp5oorrjB5eXkxx9i3b58ZPXq0SU9PN36/34wdO9bs378/Zp9PPvnEDBw40KSkpJhWrVqZKVOmxOXzAXBPTfnyl7/8xUg67vXggw9G9zl48KC59dZbTePGjU2jRo3MyJEjYxqDxhizdetWM3ToUNOwYUPTrFkzM2HCBFNRURGvjwnABdXly5YtW6rMFknm/fffjx6D6xcAVakuXw4ePGhGjhxpWrZsaZKTk01OTo4ZNmyYWbZsWcwxyBcAVanp+9Gxqmr8kS8AqlJdvpSWlprBgweb5s2bmwYNGph27dqZH/zgB6agoCDmGOQLgKrU5vrlueeeM507dzapqammZ8+eZubMmTHb43V/12eMMSc5khEAAAAAAAAAAABAHcMafwAAAAAAAAAAAIAH0PgDAAAAAAAAAAAAPIDGHwAAAAAAAAAAAOABNP4AAAAAAAAAAAAAD6DxBwAAAAAAAAAAAHgAjT8AAAAAAAAAAADAA2j8AQAAAAAAAAAAAB5A4w8AAAAAAAAAAADwABp/AAAAAOBxN910k0aMGOF2GaesvtcPAAAAAPGS5HYBAAAAAIBT5/P5qt3+4IMP6g9/+IOMMXGq6IgFCxbosssuU1FRkTIzM+N+fgAAAAA409D4AwAAAIB6bOfOndE/v/rqq5o0aZLy8vKi76Wnpys9Pd2N0gAAAAAAccZUnwAAAABQj2VnZ0dfgUBAPp8v5r309PTjpsq89NJLdccdd+iuu+5S48aNlZWVpT//+c86cOCAxo4dq4yMDHXu3FmzZ8+OOde6des0dOhQpaenKysrS2PGjNHevXtrXesLL7ygzMxMzZ07V2effbbS09OVm5sb07wMh8P66U9/qszMTDVt2lQ/+9nPjhutGIlE9Mgjj6hDhw5q2LChevbsqddff12SZIzRoEGDNGTIkOjPFRYWqnXr1po0adLJ/ucFAAAAgHqFxh8AAAAAnIFmzJihZs2aadmyZbrjjjv04x//WNdcc40uvPBCffzxxxo8eLDGjBmj0tJSSVJxcbEuv/xy9erVSytWrNCcOXO0a9cujRo16qTOW1paqt/+9rd66aWX9MEHHyg/P1933313dPvvfvc7vfDCC3r++ee1aNEiFRYW6s0334w5xiOPPKIXX3xR06ZN0/r16zV+/Hhdf/31WrhwoXw+n2bMmKHly5frySeflCSNGzdOrVq1ovEHAAAAwPOY6hMAAAAAzkA9e/bU/fffL0m69957NWXKFDVr1kw/+MEPJEmTJk3SM888ozVr1qh///56+umn1atXLz388MPRYzz//PNq06aNPv/8c5111lm1Om9FRYWmTZumTp06SZJuv/12/epXv4punzp1qu69915dffXVkqRp06Zp7ty50e2HDh3Sww8/rHfffVcDBgyQJHXs2FGLFi3S9OnTdckll6hVq1aaPn26brjhBhUUFOjtt9/WqlWrlJTEV2AAAAAA3sa3HgAAAAA4A5133nnRPycmJqpp06bq0aNH9L2srCxJ0u7duyVJn3zyid5///0q1wvctGlTrRt/jRo1ijb9JCknJyd6jmAwqJ07d6pfv37R7UlJSbrgggui03Z+8cUXKi0t1ZVXXhlz3PLycvXq1Sv692uuuUZvvvmmpkyZomeeeUZdunSpVX0AAAAAUJ/R+AMAAACAM1CDBg1i/u7z+WLe8/l8kqz19CSppKRE3/nOd/Too48ed6ycnJzTOu+xa/hVp6SkRJL01ltvqVWrVjHbUlJSon8uLS3VypUrlZiYqI0bN9b6+AAAAABQn9H4AwAAAADUqHfv3vrnP/+p9u3bOzZlZiAQUE5OjpYuXaqLL75YklRZWamVK1eqd+/ekqTu3bsrJSVF+fn5uuSSS054rAkTJighIUGzZ8/Wt771LV111VW6/PLLHakbAAAAAOqKBLcLAAAAAADUfbfddpsKCws1evRoLV++XJs2bdLcuXM1duxYhcNh285z5513asqUKZo5c6Y2bNigW2+9VcXFxdHtGRkZuvvuuzV+/HjNmDFDmzZt0scff6ynnnpKM2bMkGSNBnz++ef1t7/9TVdeeaUmTpyoG2+8UUVFRbbVCQAAAAB1EY0/AAAAAECNWrZsqf/+978Kh8MaPHiwevToobvuukuZmZlKSLDvq+WECRM0ZswY3XjjjRowYIAyMjI0cuTImH0eeughPfDAA3rkkUd09tlnKzc3V2+99ZY6dOigPXv26Oabb9bkyZOjowR/+ctfKisrS+PGjbOtTgAAAACoi3zmZBZTAAAAAAAAAAAAAFAnMeIPAAAAAAAAAAAA8AAafwAAAAAAAAAAAIAH0PgDAAAAAAAAAAAAPIDGHwAAAAAAAAAAAOABNP4AAAAAAAAAAAAAD6DxBwAAAAAAAAAAAHgAjT8AAAAAAAAAAADAA2j8AQAAAAAAAAAAAB5A4w8AAAAAAAAAAADwABp/AAAAAAAAAAAAgAfQ+AMAAAAAAAAAAAA84P8BauPGtHykQNAAAAAASUVORK5CYII=\n" + }, + "metadata": {} + }, + { + "output_type": "stream", + "name": "stdout", + "text": [ + "Plots have been generated and saved with the corrected original data.\n" + ] + } + ] + }, + { + "cell_type": "markdown", + "source": [ + "# Compare with RMSE (Original vs Finetuned on Test Data)" + ], + "metadata": { + "id": "WAyaVAEIt4Ey" + } + }, + { + "cell_type": "code", + "source": [ + "import json\n", + "import numpy as np\n", + "import pandas as pd\n", + "\n", + "def calculate_model_rmse(file_path, finetuning_cutoff_ts, outlier_threshold):\n", + " \"\"\"\n", + " Loads model output data, reconstructs the time series, and calculates RMSE\n", + " on a test set after filtering outliers.\n", + "\n", + " Args:\n", + " file_path (str): Path to the model's prediction data in JSONL format.\n", + " finetuning_cutoff_ts (int): Timestamp to split training and test data.\n", + " outlier_threshold (float): Outlier score at or above which to exclude points.\n", + "\n", + " Returns:\n", + " float: The calculated Root Mean Squared Error.\n", + " \"\"\"\n", + " all_window_data = []\n", + " with open(file_path, 'r') as f:\n", + " for line in f:\n", + " if line.strip():\n", + " all_window_data.append(json.loads(line))\n", + "\n", + " all_window_data.sort(key=lambda x: x['start_ts_micros'])\n", + "\n", + " # --- Reconstruct the full time series from the windows ---\n", + " timestamps = []\n", + " all_predicted_values = []\n", + " all_actual_values = []\n", + " all_anomalies = []\n", + "\n", + " current_ts = -1\n", + " if all_window_data:\n", + " # Initialize with the first window's start time\n", + " current_ts = all_window_data[0]['start_ts_micros'] // 1000000\n", + "\n", + " for window_data in all_window_data:\n", + " # Extend the series lists\n", + " all_predicted_values.extend(window_data['predicted_values'])\n", + " all_actual_values.extend(window_data.get('actual_horizon_values', []))\n", + " all_anomalies.extend(window_data.get('anomalies', []))\n", + "\n", + " # Reconstruct the timestamps for each predicted point\n", + " start_ts = window_data['start_ts_micros'] // 1000000\n", + " for _ in window_data['predicted_values']:\n", + " timestamps.append(start_ts)\n", + " start_ts += 1\n", + "\n", + " # Create a lookup for outlier scores\n", + " outlier_scores_map = {item['timestamp']: item['outlier_score'] for item in all_anomalies}\n", + "\n", + " # Ensure the actual values and predicted values align\n", + " min_len = min(len(timestamps), len(all_predicted_values), len(all_actual_values))\n", + "\n", + " # --- Create a DataFrame for easy filtering and calculation ---\n", + " df = pd.DataFrame({\n", + " 'timestamp': timestamps[:min_len],\n", + " 'actual': all_actual_values[:min_len],\n", + " 'predicted': all_predicted_values[:min_len]\n", + " })\n", + " df['outlier_score'] = df['timestamp'].map(outlier_scores_map).fillna(0.0)\n", + "\n", + " # 1. Isolate the test set\n", + " df_test = df[df['timestamp'] > finetuning_cutoff_ts].copy()\n", + " print(f\"\\n--- Analyzing: {file_path} ---\")\n", + " print(f\"Test set size (before filtering): {len(df_test)} points\")\n", + "\n", + " # 2. Filter out anomalies based on the threshold\n", + " df_filtered = df_test[df_test['outlier_score'] < outlier_threshold]\n", + " num_outliers = len(df_test) - len(df_filtered)\n", + " print(f\"Test set size (after filtering): {len(df_filtered)} points\")\n", + " print(f\"Removed {num_outliers} points with outlier_score >= {outlier_threshold}\")\n", + "\n", + " # 3. Calculate RMSE\n", + " y_true = df_filtered['actual']\n", + " y_pred = df_filtered['predicted']\n", + " rmse = np.sqrt(np.mean((y_true - y_pred)**2))\n", + "\n", + " return rmse\n", + "\n", + "# --- Configuration ---\n", + "FINETUNING_CUTOFF_TS = 8320\n", + "ORIGINAL_OUTLIER_THRESHOLD = 1.0\n", + "FINETUNED_OUTLIER_THRESHOLD = 5.0\n", + "\n", + "# --- Execution & Comparison ---\n", + "original_rmse = calculate_model_rmse(\n", + " file_path=\"plot_data_original.jsonl\",\n", + " finetuning_cutoff_ts=FINETUNING_CUTOFF_TS,\n", + " outlier_threshold=ORIGINAL_OUTLIER_THRESHOLD\n", + ")\n", + "\n", + "finetuned_rmse = calculate_model_rmse(\n", + " file_path=\"plot_data_finetuned.jsonl\",\n", + " finetuning_cutoff_ts=FINETUNING_CUTOFF_TS,\n", + " outlier_threshold=FINETUNED_OUTLIER_THRESHOLD\n", + ")\n", + "\n", + "print(\"\\n--- Final Results ---\")\n", + "print(f\"Original Model RMSE: {original_rmse:.2f}\")\n", + "print(f\"Fine-tuned Model RMSE: {finetuned_rmse:.2f}\")" + ], + "metadata": { + "colab": { + "base_uri": "https://localhost:8080/" + }, + "id": "zrMOmc90lQhJ", + "outputId": "d0679b35-b375-4068-aa4b-bd22a8c6166e" + }, + "execution_count": 22, + "outputs": [ + { + "output_type": "stream", + "name": "stdout", + "text": [ + "\n", + "--- Analyzing: plot_data_original.jsonl ---\n", + "Test set size (before filtering): 1407 points\n", + "Test set size (after filtering): 1369 points\n", + "Removed 38 points with outlier_score >= 1.0\n", + "\n", + "--- Analyzing: plot_data_finetuned.jsonl ---\n", + "Test set size (before filtering): 1407 points\n", + "Test set size (after filtering): 1384 points\n", + "Removed 23 points with outlier_score >= 5.0\n", + "\n", + "--- Final Results ---\n", + "Original Model RMSE: 7164.17\n", + "Fine-tuned Model RMSE: 3948.44\n" + ] + } + ] + } + ] +} \ No newline at end of file diff --git a/examples/notebooks/beam-ml/automatic_model_refresh.ipynb b/examples/notebooks/beam-ml/automatic_model_refresh.ipynb index 2f80846f313b..c29881ea72fd 100644 --- a/examples/notebooks/beam-ml/automatic_model_refresh.ipynb +++ b/examples/notebooks/beam-ml/automatic_model_refresh.ipynb @@ -98,7 +98,7 @@ } ], "source": [ - "!pip install apache_beam[gcp]>=2.46.0 tensorflow==2.15.0 tensorflow_hub==0.16.1 keras==2.15.0 Pillow==11.0.0 --quiet" + "!pip install apache_beam[interactive,gcp]>=2.46.0 tensorflow==2.15.0 tensorflow_hub==0.16.1 keras==2.15.0 Pillow==11.0.0 --quiet" ] }, { diff --git a/examples/notebooks/beam-ml/bigquery_vector_ingestion_and_search.ipynb b/examples/notebooks/beam-ml/bigquery_vector_ingestion_and_search.ipynb index 7608b83cb59c..b1becd294ff0 100644 --- a/examples/notebooks/beam-ml/bigquery_vector_ingestion_and_search.ipynb +++ b/examples/notebooks/beam-ml/bigquery_vector_ingestion_and_search.ipynb @@ -98,7 +98,7 @@ "cell_type": "code", "source": [ "# Apache Beam with GCP support\n", - "!pip install apache_beam[gcp]>=2.64.0 --quiet\n", + "!pip install apache_beam[interactive,gcp]>=2.64.0 --quiet\n", "# Huggingface sentence-transformers for embedding models\n", "!pip install sentence-transformers --quiet\n" ], diff --git a/examples/notebooks/beam-ml/cloudsql_mysql_product_catalog_embeddings.ipynb b/examples/notebooks/beam-ml/cloudsql_mysql_product_catalog_embeddings.ipynb new file mode 100644 index 000000000000..457d7d181b6b --- /dev/null +++ b/examples/notebooks/beam-ml/cloudsql_mysql_product_catalog_embeddings.ipynb @@ -0,0 +1,2785 @@ +{ + "cells": [ + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "cellView": "form", + "id": "8ZekaWhZH2SX" + }, + "outputs": [], + "source": [ + "# @title ###### Licensed to the Apache Software Foundation (ASF), Version 2.0 (the \"License\")\n", + "\n", + "# Licensed to the Apache Software Foundation (ASF) under one\n", + "# or more contributor license agreements. See the NOTICE file\n", + "# distributed with this work for additional information\n", + "# regarding copyright ownership. The ASF licenses this file\n", + "# to you under the Apache License, Version 2.0 (the\n", + "# \"License\"); you may not use this file except in compliance\n", + "# with the License. You may obtain a copy of the License at\n", + "#\n", + "# http://www.apache.org/licenses/LICENSE-2.0\n", + "#\n", + "# Unless required by applicable law or agreed to in writing,\n", + "# software distributed under the License is distributed on an\n", + "# \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY\n", + "# KIND, either express or implied. See the License for the\n", + "# specific language governing permissions and limitations\n", + "# under the License" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "K6-p-DVrIFTY" + }, + "source": [ + "# Vector Embedding Ingestion with Apache Beam and CloudSQL MySQL\n", + "\n", + "\n", + " \n", + " \n", + "
\n", + " Run in Google Colab\n", + " \n", + " View source on GitHub\n", + "
\n" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "WWwFCLRHZPm4" + }, + "source": [ + "# Introduction\n", + "\n", + "This Colab demonstrates how to generate embeddings from data and ingest them into [CloudSQL MySQL](https://cloud.google.com/sql/docs/mysql). We'll use Apache Beam and Dataflow for scalable data processing.\n", + "\n", + "The goal of this notebook is to make it easy for users to get started with generating embeddings at scale using Apache Beam and storing them in CloudSQL MySQL. We focus on building efficient ingestion pipelines that can handle various data sources and embedding models.\n", + "\n", + "## Example: Furniture Product Catalog\n", + "\n", + "We'll work with a sample e-commerce dataset representing a furniture product catalog. Each product has:\n", + "\n", + "* **Structured fields:** `id`, `name`, `category`, `price`\n", + "* **Detailed text descriptions:** Longer text describing the product's features.\n", + "* **Additional metadata:** `material`, `dimensions`\n", + "\n", + "## Pipeline Overview\n", + "We will build a pipeline to:\n", + "1. Read product data\n", + "2. Convert unstructured product data, to `Chunk`[1] type\n", + "2. Generate Embeddings: Use a pre-trained Hugging Face model (via MLTransform) to create vector embeddings\n", + "3. Write to CloudSQL MySQL: Store the embeddings in a CloudSQL MySQL vector database\n", + "\n", + "Here's a visualization of the data flow:\n", + "\n", + "| Stage | Data Representation | Notes |\n", + "| :------------------------ | :------------------------------------------------------- | :---------------------------------------------------------------------------------------------------------------------- |\n", + "| **1. Ingest Data** | `{`
` \"id\": \"desk-001\",`
` \"name\": \"Modern Desk\",`
` \"description\": \"Sleek...\",`
` \"category\": \"Desks\",`
` ...`
`}` | Supports:
- Reading from batch (e.g., files, databases)
- Streaming sources (e.g., Pub/Sub). |\n", + "| **2. Convert to Chunks** | `Chunk(`
  `id=\"desk-001\",`
  `content=Content(`
    `text=\"Modern Desk\"`
   `),`
  `metadata={...}`
`)` | - `Chunk` is the structured input for generating and ingesting embeddings.
- `chunk.content.text` is the field that is embedded.
- Converting to `Chunk` does not mean breaking data into smaller pieces,
   it's simply organizing your data in a standard format for the embedding pipeline.
- `Chunk` allows data to flow seamlessly throughout embedding pipelines. |\n", + "| **3. Generate Embeddings**| `Chunk(`
  `id=\"desk-001\",`
  `embedding=[-0.1, 0.6, ...],`
`...)` | Supports:
- Local Hugging Face models
- Remote Vertex AI models
- Custom embedding implementations. |\n", + "| **4. Write to CloudSQL MySQL** | **CloudSQL MySQL Table (Example Row):**
`id: desk-001`
`embedding: [-0.1, 0.6, ...]`
`name = \"Modern Desk\"`,
`Other fields ...` | Supports:
- Custom schemas
- Conflict resolution strategies for handling updates |\n", + "\n", + "\n", + "[1]: Chunk represents an embeddable unit of input. It specifies which fields should be embedded and which fields should be treated as metadata. Converting to Chunk does not necessarily mean breaking your text into smaller pieces - it's primarily about structuring your data for the embedding pipeline. For very long texts that exceed the embedding model's maximum input size, you can optionally [use Langchain TextSplitters](https://beam.apache.org/releases/pydoc/2.63.0/apache_beam.ml.rag.chunking.langchain.html) to break the text into smaller `Chunk`'s.\n", + "\n", + "## Execution Environments\n", + "\n", + "This notebook demonstrates two execution environments:\n", + "\n", + "1. **DirectRunner (Local Execution)**: All examples in this notebook run on DirectRunner by default, which executes the pipeline locally. This is ideal for development, testing, and processing small datasets.\n", + "\n", + "2. **DataflowRunner (Distributed Execution)**: The [Run on Dataflow](#scrollTo=Quick_Start_Run_on_Dataflow) section demonstrates how to execute the same pipeline on Google Cloud Dataflow for scalable, distributed processing. This is recommended for production workloads and large datasets.\n", + "\n", + "All examples in this notebook can be adapted to run on Dataflow by following the pattern shown in the \"Run on Dataflow\" section." + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "z2eAyRECIP3z" + }, + "source": [ + "# Connecting Apache Beam to CloudSQL MySQL\n", + "\n", + "Beam uses the [CloudSQL MySQL Java Connector](https://github.com/GoogleCloudPlatform/cloud-sql-jdbc-socket-factory/blob/main/docs/jdbc.md) to securely establish a connection to your database. Apache Beam supports any parameters that can be passed to the Java Connector e.g. IP types.\n", + "\n", + "# Setup and Prerequisites\n", + "\n", + "This example requires:\n", + "1. A CloudSQL MySQL instance with [cloudsql_vector](https://cloud.google.com/sql/docs/mysql/vector-search#requirements) flag enabled\n", + "2. Apache Beam 2.67.0 or later\n", + "\n" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "WhOOPUBa6PyW" + }, + "source": [ + "## Install Packages and Dependencies\n", + "\n", + "First, let's install the Python packages required for the embedding and ingestion pipeline:" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "gCWRw2YE11wN" + }, + "outputs": [], + "source": [ + "# Apache Beam with GCP support\n", + "!pip install apache_beam[interactive,gcp]>=2.67.0 --quiet\n", + "# Huggingface sentence-transformers for embedding models\n", + "!pip install sentence-transformers --quiet" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "2FlMPmA0IUuv" + }, + "outputs": [], + "source": [ + "!pip show apache-beam" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "4aqYZ_pG1oYb" + }, + "source": [ + "Next, let's install cloud-sql-python-connector to help set up our test database." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "eOYjnVDR87IE" + }, + "outputs": [], + "source": [ + "!pip install \"cloud-sql-python-connector[pymysql]>=1.0.0,<2.0.0\" sqlalchemy --quiet" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "VhgbpTKzI-zI" + }, + "source": [ + "## Database Setup\n", + "\n", + "To connect to CloudSQL MySQL, you'll need:\n", + "1. GCP project ID where the CloudSQL MySQL instance is located\n", + "2. The CloudSQL MySQL connection URI. This is the fully qualified connection name of the CloudSQL MySQL instance found in the google cloud console under CloudSQL > Instances > Instance > Connect to this Instance > Connection name.\n", + "3. Database name. This is the name of the mysql database within your CloudSQL MySQL instance. The default database name is mysql.\n", + "4. Database credentials\n", + "5. A CloudSQL MySQL instance with cloudsql_vector flag enabled\n", + "\n", + "Replace these placeholder values with your actual CloudSQL MySQL connection details:" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "oqKQT0c_JB5f" + }, + "outputs": [], + "source": [ + "PROJECT_ID = \"\" # @param {type:'string'}\n", + "\n", + "CONNECTION_NAME = \"\" # @param {type:'string'}\n", + "\n", + "DB_NAME = \"\" # @param {type:'string'}\n", + "\n", + "DB_USER = \"\" # @param {type:'string'}\n", + "\n", + "DB_PASSWORD = \"\" # @param {type:'string'}" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "doK840yZZNdl" + }, + "source": [ + "## Authenticate to Google Cloud\n", + "\n", + "To connect to the CloudSQL MySQL instance via the language conenctor, we authenticate with Google Cloud." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "CLM12rbiZHTN" + }, + "outputs": [], + "source": [ + "from google.colab import auth\n", + "auth.authenticate_user(project_id=PROJECT_ID)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "l_BBCKl7KKcb" + }, + "outputs": [], + "source": [ + " # @title SQLAlchemy + CloudSQL MySQL Connector helpers for creating tables and verifying data\n", + "\n", + "import sqlalchemy\n", + "from sqlalchemy import text\n", + "from sqlalchemy.exc import SQLAlchemyError\n", + "from google.cloud.sql.connector import Connector\n", + "\n", + "def get_db_engine(connection_name: str, user: str, password: str, db: str, **connect_kwargs) -> sqlalchemy.engine.Engine:\n", + " \"\"\"\n", + " Creates a SQLAlchemy engine configured for CloudSQL MySQL.\n", + "\n", + " To use this function, you may need to install necessary libraries:\n", + " 'pip install google-cloud-sql-connector[pymysql] sqlalchemy'\n", + "\n", + " Args:\n", + " connection_name: CloudSQL MySQL instance connection name (e.g., \"project:region:instance\").\n", + " user: The database user.\n", + " password: The database password.\n", + " db: The name of the database.\n", + " connect_kwargs: Additional keyword arguments for the connector (e.g., ip_type=\"PUBLIC\").\n", + "\n", + " Returns:\n", + " A SQLAlchemy engine instance.\n", + " \"\"\"\n", + " connector = Connector()\n", + "\n", + " def get_conn() -> sqlalchemy.engine.base.Connection:\n", + " \"\"\"Helper function to create a database connection.\"\"\"\n", + " conn = connector.connect(\n", + " connection_name,\n", + " \"pymysql\", # Use the PyMySQL driver for MySQL\n", + " user=user,\n", + " password=password,\n", + " db=db,\n", + " **connect_kwargs\n", + " )\n", + " return conn\n", + "\n", + " # Create the SQLAlchemy engine using the connection function\n", + " engine = sqlalchemy.create_engine(\n", + " \"mysql+pymysql://\", # Use the MySQL+PyMySQL dialect\n", + " creator=get_conn,\n", + " )\n", + "\n", + " # This hook ensures the connector is closed when the engine is disposed\n", + " engine.pool.dispose = lambda: connector.close()\n", + "\n", + " return engine\n", + "\n", + "def setup_db_table_sqlalchemy(connection_name: str,\n", + " database: str,\n", + " table_name: str,\n", + " table_schema: str,\n", + " user: str,\n", + " password: str,\n", + " **connect_kwargs):\n", + " \"\"\"\n", + " Sets up a CloudSQL MySQL table using SQLAlchemy.\n", + "\n", + " This function will drop the table if it already exists and then create it\n", + " based on the provided schema.\n", + "\n", + " Args:\n", + " connection_name: CloudSQL MySQL instance connection name.\n", + " database: The name of the database.\n", + " table_name: The name of the table to create.\n", + " table_schema: SQL string defining the table columns. For MySQL, use types like\n", + " 'INT AUTO_INCREMENT PRIMARY KEY'. For embeddings, consider using\n", + " 'JSON' or 'BLOB' to store the vector data.\n", + " Example: \"id INT AUTO_INCREMENT PRIMARY KEY, embedding JSON\"\n", + " user: The database user.\n", + " password: The database password.\n", + " connect_kwargs: Additional keyword arguments for the connector.\n", + " \"\"\"\n", + " engine = None\n", + " try:\n", + " engine = get_db_engine(connection_name, user, password, database, **connect_kwargs)\n", + "\n", + " with engine.connect() as connection:\n", + " # Use autocommit for DDL statements\n", + " with connection.execution_options(isolation_level=\"AUTOCOMMIT\"):\n", + " print(\"Connected to MySQL DB successfully via SQLAlchemy!\")\n", + "\n", + " # Use backticks for table names for MySQL compatibility\n", + " print(f\"Dropping table `{table_name}` if it exists...\")\n", + " connection.execute(text(f\"DROP TABLE IF EXISTS `{table_name}`;\"))\n", + "\n", + " print(f\"Creating table `{table_name}`...\")\n", + " create_sql = f\"\"\"\n", + " CREATE TABLE IF NOT EXISTS `{table_name}` (\n", + " {table_schema}\n", + " );\n", + " \"\"\"\n", + " connection.execute(text(create_sql))\n", + "\n", + " print(\"MySQL table setup completed successfully!\")\n", + "\n", + " except SQLAlchemyError as e:\n", + " print(f\"An SQLAlchemy error occurred during setup: {e}\")\n", + " except Exception as e:\n", + " print(f\"An unexpected error occurred during setup: {e}\")\n", + " finally:\n", + " if engine:\n", + " engine.dispose()\n", + "\n", + "def test_db_connection_sqlalchemy(connection_name: str,\n", + " database: str,\n", + " table_name: str,\n", + " user: str,\n", + " password: str,\n", + " **connect_kwargs):\n", + " \"\"\"\n", + " Tests the CloudSQL MySQL connection and verifies table existence.\n", + "\n", + " Args:\n", + " connection_name: CloudSQL MySQL instance connection name.\n", + " database: The name of the database.\n", + " table_name: The name of the table to check for.\n", + " user: The database user.\n", + " password: The database password.\n", + " connect_kwargs: Additional keyword arguments for the connector.\n", + " \"\"\"\n", + " engine = None\n", + " try:\n", + " engine = get_db_engine(connection_name, user, password, database, **connect_kwargs)\n", + "\n", + " with engine.connect() as connection:\n", + " print(\"Testing MySQL connection...\")\n", + " connection.execute(text(\"SELECT 1\"))\n", + " print(\"✓ Connection successful\")\n", + "\n", + " # Check if table exists using information_schema.\n", + " # In MySQL, schema is the database, which can be found with DATABASE().\n", + " table_exists_query = text(\"\"\"\n", + " SELECT EXISTS (\n", + " SELECT 1\n", + " FROM information_schema.tables\n", + " WHERE table_schema = DATABASE() AND table_name = :tname\n", + " );\n", + " \"\"\")\n", + " table_exists = connection.execute(table_exists_query, {\"tname\": table_name}).scalar()\n", + "\n", + " if table_exists:\n", + " print(f\"✓ Table `{table_name}` exists in database `{database}`.\")\n", + " else:\n", + " print(f\"✗ Table `{table_name}` does NOT exist in database `{database}`.\")\n", + "\n", + " except SQLAlchemyError as e:\n", + " print(f\"Connection test failed (SQLAlchemy error): {e}\")\n", + " except Exception as e:\n", + " print(f\"Connection test failed (Unexpected error): {e}\")\n", + " finally:\n", + " if engine:\n", + " engine.dispose()\n", + "\n", + "def verify_embeddings_sqlalchemy(connection_name: str,\n", + " database: str,\n", + " table_name: str,\n", + " user: str,\n", + " password: str,\n", + " embedding_column: str = \"embedding\",\n", + " **connect_kwargs):\n", + " \"\"\"\n", + " Connects to a CloudSQL MySQL table and prints all of its rows.\n", + "\n", + " Args:\n", + " connection_name: CloudSQL MySQL instance connection name.\n", + " database: The name of the database.\n", + " table_name: The name of the table to query.\n", + " user: The database user.\n", + " password: The database password.\n", + " connect_kwargs: Additional keyword arguments for the connector.\n", + " \"\"\"\n", + " engine = None\n", + " try:\n", + " engine = get_db_engine(connection_name, user, password, database, **connect_kwargs)\n", + "\n", + " with engine.connect() as connection:\n", + " # Use backticks for the table name for MySQL best practice\n", + " column_query = text(f\"\"\"\n", + " SELECT COLUMN_NAME\n", + " FROM INFORMATION_SCHEMA.COLUMNS\n", + " WHERE table_schema = :db_name\n", + " AND table_name = :t_name\n", + " AND COLUMN_NAME != '{embedding_column}'\n", + " \"\"\")\n", + "\n", + " column_result = connection.execute(\n", + " column_query,\n", + " {\"db_name\": database, \"t_name\": table_name}\n", + " )\n", + "\n", + " columns_to_select = [row[0] for row in column_result]\n", + "\n", + " if not columns_to_select:\n", + " print(f\"No columns to display in `{table_name}` (after excluding '{embedding_column}').\")\n", + " return\n", + "\n", + " # Construct the SELECT statement with the filtered columns, quoting them for safety\n", + " select_columns_str = \", \".join([f\"`{col}`\" for col in columns_to_select])\n", + " select_query = text(f\"SELECT {select_columns_str}, vector_to_string({embedding_column}) as {embedding_column} FROM `{table_name}`;\")\n", + "\n", + " # Execute the query to get the data\n", + " result = connection.execute(select_query)\n", + " rows = result.mappings().all()\n", + "\n", + " print(f\"\\nFound {len(rows)} rows in `{table_name}` (excluding '{embedding_column}' column):\")\n", + " print(\"-\" * 80)\n", + "\n", + " if not rows:\n", + " print(\"Table is empty.\")\n", + " else:\n", + " # result.keys() will have the correct column names from the executed query\n", + " columns = result.keys()\n", + " for row in rows:\n", + " for col in columns:\n", + " print(f\"{col}: {row[col]}\")\n", + " print(\"-\" * 80)\n", + " except SQLAlchemyError as e:\n", + " # Check specifically for ProgrammingError if the table might not exist\n", + " if isinstance(e, sqlalchemy.exc.ProgrammingError):\n", + " print(f\"Failed to query table `{table_name}`. Does it exist? Error: {e}\")\n", + " else:\n", + " print(f\"Failed to verify data (SQLAlchemy error): {e}\")\n", + " except Exception as e:\n", + " print(f\"Failed to verify data (Unexpected error): {e}\")\n", + " finally:\n", + " if engine:\n", + " engine.dispose()\n" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "70z2O4nbOuaM" + }, + "source": [ + "## Create Sample Product Catalog Data\n", + "\n", + "We'll create a typical e-commerce catalog where you might want to:\n", + "- Generate embeddings for product text\n", + "- Store vectors alongside product data\n", + "- Enable vector similarity features\n", + "\n", + "Example product:\n", + "```python\n", + "{\n", + " \"id\": \"desk-001\",\n", + " \"name\": \"Modern Minimalist Desk\",\n", + " \"description\": \"Sleek minimalist desk with clean lines and a spacious work surface. \"\n", + " \"Features cable management system and sturdy steel frame. \"\n", + " \"Perfect for contemporary home offices and workspaces.\",\n", + " \"category\": \"Desks\",\n", + " \"price\": 399.99,\n", + " \"material\": \"Engineered Wood, Steel\",\n", + " \"dimensions\": \"60W x 30D x 29H inches\"\n", + "}\n", + "```" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "7_J__S8JOwJ_" + }, + "outputs": [], + "source": [ + "#@title Create sample data\n", + "PRODUCTS_DATA = [\n", + " {\n", + " \"id\": \"desk-001\",\n", + " \"name\": \"Modern Minimalist Desk\",\n", + " \"description\": \"Sleek minimalist desk with clean lines and a spacious work surface. \"\n", + " \"Features cable management system and sturdy steel frame. \"\n", + " \"Perfect for contemporary home offices and workspaces.\",\n", + " \"category\": \"Desks\",\n", + " \"price\": 399.99,\n", + " \"material\": \"Engineered Wood, Steel\",\n", + " \"dimensions\": \"60W x 30D x 29H inches\"\n", + " },\n", + " {\n", + " \"id\": \"chair-001\",\n", + " \"name\": \"Ergonomic Mesh Office Chair\",\n", + " \"description\": \"Premium ergonomic office chair with breathable mesh back, \"\n", + " \"adjustable lumbar support, and 4D armrests. Features synchronized \"\n", + " \"tilt mechanism and memory foam seat cushion. Ideal for long work hours.\",\n", + " \"category\": \"Office Chairs\",\n", + " \"price\": 299.99,\n", + " \"material\": \"Mesh, Metal, Premium Foam\",\n", + " \"dimensions\": \"26W x 26D x 48H inches\"\n", + " },\n", + " {\n", + " \"id\": \"sofa-001\",\n", + " \"name\": \"Contemporary Sectional Sofa\",\n", + " \"description\": \"Modern L-shaped sectional with chaise lounge. Upholstered in premium \"\n", + " \"performance fabric. Features deep seats, plush cushions, and solid \"\n", + " \"wood legs. Perfect for modern living rooms.\",\n", + " \"category\": \"Sofas\",\n", + " \"price\": 1299.99,\n", + " \"material\": \"Performance Fabric, Solid Wood\",\n", + " \"dimensions\": \"112W x 65D x 34H inches\"\n", + " },\n", + " {\n", + " \"id\": \"table-001\",\n", + " \"name\": \"Rustic Dining Table\",\n", + " \"description\": \"Farmhouse-style dining table with solid wood construction. \"\n", + " \"Features distressed finish and trestle base. Seats 6-8 people \"\n", + " \"comfortably. Perfect for family gatherings.\",\n", + " \"category\": \"Dining Tables\",\n", + " \"price\": 899.99,\n", + " \"material\": \"Solid Pine Wood\",\n", + " \"dimensions\": \"72W x 42D x 30H inches\"\n", + " },\n", + " {\n", + " \"id\": \"bed-001\",\n", + " \"name\": \"Platform Storage Bed\",\n", + " \"description\": \"Modern queen platform bed with integrated storage drawers. \"\n", + " \"Features upholstered headboard and durable wood slat support. \"\n", + " \"No box spring needed. Perfect for maximizing bedroom space.\",\n", + " \"category\": \"Beds\",\n", + " \"price\": 799.99,\n", + " \"material\": \"Engineered Wood, Linen Fabric\",\n", + " \"dimensions\": \"65W x 86D x 48H inches\"\n", + " }\n", + "]\n", + "print(f\"\"\"✓ Created PRODUCTS_DATA with {len(PRODUCTS_DATA)} records\"\"\")" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "KUHPsWzQFKpL" + }, + "source": [ + "## Importing Pipeline Components\n", + "\n", + "We import the following for configuring our embedding ingestion pipeline:\n", + "- `apache_beam.ml.rag.types.Chunk`, the structured input for generating and ingesting embeddings\n", + "- `apache_beam.ml.rag.ingestion.cloudsql.CloudSQLMySQLVectorWriterConfig` for configuring write behavior like schema mapping and conflict resolution\n", + "- `apache_beam.ml.rag.ingestion.cloudsql.LanguageConnectorConfig` to connect using the [CloudSQL MySQL language connector](https://github.com/GoogleCloudPlatform/cloud-sql-jdbc-socket-factory/blob/main/docs/jdbc.md)\n", + "- `apache_beam.ml.rag.ingestion.base import VectorDatabaseWriteTransform` to perform the write step using CloudSQL MySQL configs" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "fFMjPZaelTi2" + }, + "outputs": [], + "source": [ + "# CloudSQL imports\n", + "from apache_beam.ml.rag.ingestion.cloudsql import CloudSQLMySQLVectorWriterConfig\n", + "from apache_beam.ml.rag.ingestion.cloudsql import LanguageConnectorConfig\n", + "\n", + "\n", + "from apache_beam.ml.rag.ingestion.base import VectorDatabaseWriteTransform\n", + "from apache_beam.ml.rag.types import Chunk, Content\n", + "from apache_beam.ml.rag.embeddings.huggingface import HuggingfaceTextEmbeddings\n", + "\n", + "# Apache Beam core\n", + "import apache_beam as beam\n", + "from apache_beam.options.pipeline_options import PipelineOptions\n", + "from apache_beam.ml.transforms.base import MLTransform\n", + "\n", + "# JDBC and MySQL utilities\n", + "from apache_beam.ml.rag.ingestion.jdbc_common import WriteConfig\n", + "from apache_beam.ml.rag.ingestion.mysql_common import ColumnSpecsBuilder, ConflictResolution" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "FjUzsUtXzFof" + }, + "source": [ + "# What's next?\n", + "\n", + "This colab covers several use cases that you can explore based on your needs after completing the Setup and Prerequisites:\n", + "\n", + "🔰 **New to vector embeddings?**\n", + "- [Start with Quick Start](#scrollTo=Quick_Start_Basic_Vector_Ingestion)\n", + "- Uses simple out-of-box schema\n", + "- Perfect for initial testing\n", + "\n", + "🚀 **Need to scale to large datasets?**\n", + "- [Go to Run on Dataflow](#scrollTo=Quick_Start_Run_on_Dataflow)\n", + "- Learn how to execute the same pipeline at scale\n", + "- Fully managed\n", + "- Process large datasets efficiently\n", + "\n", + "🎯 **Have a specific schema?**\n", + "- [Go to Custom Schema](#scrollTo=Custom_Schema_with_Column_Mapping)\n", + "- Learn to use different column names\n", + "- Map metadata to individual columns\n", + "\n", + "🔄 **Need to update embeddings?**\n", + "- [Check out Updating Embeddings](#scrollTo=Update_Embeddings_and_Metadata_with_Conflict_Resolution)\n", + "- Handle conflicts\n", + "- Selective field updates\n", + "\n", + "🔗 **Need to generate and Store Embeddings for Existing CloudSQL MySQL Data??**\n", + "- [See Database Integration](#scrollTo=Adding_Embeddings_to_Existing_Database_Records)\n", + "- Read data from your CloudSQL MySQL table.\n", + "- Generate embeddings for the relevant fields.\n", + "- Update your table (or a related table) with the generated embeddings.\n", + "\n", + "🤖 **Want to use Google's AI models?**\n", + "- [Try Vertex AI Embeddings](#scrollTo=Generate_Embeddings_with_VertexAI_Text_Embeddings)\n", + "- Use Google's powerful embedding models\n", + "- Seamlessly integrate with other Google Cloud services\n", + "\n", + "🔄 Need real-time embedding updates?\n", + "\n", + "- [Try Streaming Embeddings from PubSub](#scrollTo=Streaming_Embeddings_Updates_from_PubSub)\n", + "- Process continuous data streams\n", + "- Update embeddings in real-time as information changes" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "pLEi3Z4wKMOX" + }, + "source": [ + "
\n", + "# Quick Start: Basic Vector Ingestion\n", + "\n", + "This section shows the simplest way to generate embeddings and store them in CloudSQL MySQL." + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "LWqEgqjQOcbA" + }, + "source": [ + "## Create table with default schema\n", + "\n", + "Before running the pipeline, we need a table to store our embeddings:" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "93YnjdJkFWOi" + }, + "outputs": [], + "source": [ + "table_name = \"default_product_embeddings\"\n", + "table_schema = f\"\"\"\n", + " id VARCHAR(255) PRIMARY KEY,\n", + " embedding VECTOR(384) USING VARBINARY,\n", + " content text,\n", + " metadata JSON\n", + "\"\"\"\n", + "setup_db_table_sqlalchemy(CONNECTION_NAME, DB_NAME, table_name,table_schema, DB_USER, DB_PASSWORD)\n", + "test_db_connection_sqlalchemy(CONNECTION_NAME, DB_NAME, table_name, DB_USER, DB_PASSWORD)" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "DikTnoGbOioG" + }, + "source": [ + "## Configure Pipeline Components\n", + "\n", + "Now define the components that control the pipeline behavior:" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "M8rVyZ6o-Nep" + }, + "source": [ + "### Convert ingested product data to embeddable Chunks\n", + "- Our data is ingested as product dictionaries\n", + "- Embedding generation and ingestion processes `Chunks`\n", + "- We convert each product dictionary to a `Chunk` to configure what text to embed and what to treat as metadata" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "Rm_IX5U6mP_r" + }, + "outputs": [], + "source": [ + "from typing import Dict, Any\n", + "\n", + "# The create_chunk function converts our product dictionaries to Chunks.\n", + "# This doesn't split the text - it simply structures it in the format\n", + "# expected by the embedding pipeline components.\n", + "def create_chunk(product: Dict[str, Any]) -> Chunk:\n", + " \"\"\"Convert a product dictionary into a Chunk object.\n", + "\n", + " The pipeline components (MLTransform, VectorDatabaseWriteTransform)\n", + " work with Chunk objects. This function:\n", + " 1. Extracts text we want to embed\n", + " 2. Preserves product data as metadata\n", + " 3. Creates a Chunk in the expected format\n", + "\n", + " Args:\n", + " product: Dictionary containing product information\n", + "\n", + " Returns:\n", + " Chunk: A Chunk object ready for embedding\n", + " \"\"\"\n", + " return Chunk(\n", + " content=Content(\n", + " text=f\"{product['name']}: {product['description']}\"\n", + " ), # The text that will be embedded\n", + " id=product['id'], # Use product ID as chunk ID\n", + " metadata=product, # Store all product info in metadata\n", + " )" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "xJaI9m3D7Vw-" + }, + "source": [ + "### Generate embeddings with HuggingFace" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "0dlm1fjQh2dX" + }, + "source": [ + "We use a local pre-trained Hugging Face model to create vector embeddings from the product descriptions." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "E5LkHmjV7l2S" + }, + "outputs": [], + "source": [ + "huggingface_embedder = HuggingfaceTextEmbeddings(\n", + " model_name=\"sentence-transformers/all-MiniLM-L6-v2\"\n", + ")" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "vVv8hD5wQo3w" + }, + "source": [ + "### Write to CloudSQL MySQL\n", + "\n", + "The default CloudSQLMySQLVectorWriterConfig maps Chunk fields to database columns as:\n", + "\n", + "| Database Column | Chunk Field | Description |\n", + "|----------------|-------------|-------------|\n", + "| id | chunk.id | Unique identifier |\n", + "| embedding | chunk.embedding.dense_embedding | Vector representation |\n", + "| content | chunk.content.text | Text that was embedded |\n", + "| metadata | chunk.metadata | Additional data as JSONB |" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "moKsz_6xQt-E" + }, + "outputs": [], + "source": [ + "# Configure the language connector so we can connect securely\n", + "connector_config = LanguageConnectorConfig(\n", + " username=DB_USER,\n", + " password=DB_PASSWORD,\n", + " database_name=DB_NAME,\n", + " instance_name=CONNECTION_NAME\n", + ")\n", + "cloudsql_writer_config = CloudSQLMySQLVectorWriterConfig(\n", + " connection_config=connector_config,\n", + " table_name=table_name\n", + ")" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "Ww2BPxTNKmL2" + }, + "source": [ + "## Assemble and Run Pipeline\n", + "\n", + "Now we can create our pipeline that:\n", + "1. Takes our product data\n", + "2. Converts each product to a Chunk\n", + "3. Generates embeddings for each Chunk\n", + "4. Stores everything in CloudSQL MySQL" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "lyS3IpNBDgYw" + }, + "outputs": [], + "source": [ + "import tempfile\n", + "\n", + "# Executing on DirectRunner (local execution)\n", + "with beam.Pipeline() as p:\n", + " _ = (\n", + " p\n", + " | 'Create Products' >> beam.Create(PRODUCTS_DATA)\n", + " | 'Convert to Chunks' >> beam.Map(create_chunk)\n", + " | 'Generate Embeddings' >> MLTransform(write_artifact_location=tempfile.mkdtemp())\n", + " .with_transform(huggingface_embedder)\n", + " | 'Write to CloudSQL' >> VectorDatabaseWriteTransform(\n", + " cloudsql_writer_config\n", + " )\n", + " )" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "Qm97EAww6RvW" + }, + "source": [ + "## Verify Embeddings\n", + "Let's check what was written to our CloudSQL MySQL table:" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "-H3t2cIN6lO_" + }, + "outputs": [], + "source": [ + "verify_embeddings_sqlalchemy(connection_name=CONNECTION_NAME, database=DB_NAME, table_name=table_name, user=DB_USER, password=DB_PASSWORD)" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "lz5itufZ31KB" + }, + "source": [ + "## Quick Start Summary\n", + "\n", + "In this section, you learned how to:\n", + "- Convert product data to the Chunk format expected by embedding pipelines\n", + "- Generate embeddings using a HuggingFace model\n", + "- Configure and run a basic embedding ingestion pipeline\n", + "- Store embeddings and metadata in CloudSQL MySQL\n", + "\n", + "This basic pattern forms the foundation for all the advanced use cases covered in the following sections." + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "OqojLgpJKUGk" + }, + "source": [ + "# Quick Start: Run on Dataflow\n", + "\n", + "This section demonstrates how to launch the Quick Start embedding pipeline on Google Cloud Dataflow from the colab. While previous examples used DirectRunner for local execution, Dataflow provides a fully managed, distributed execution environment that is:\n", + "- Scalable: Automatically scales to handle large datasets\n", + "- Fault-tolerant: Handles worker failures and ensures exactly-once processing\n", + "- Fully managed: No need to provision or manage infrastructure\n", + "\n", + "For more in-depth documentation to package your pipeline into a python file and launch a DataFlow job from the command line see [Create Dataflow pipeline using Python](https://cloud.google.com/dataflow/docs/quickstarts/create-pipeline-python)." + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "zrMJSm-JUVGY" + }, + "source": [ + "## Create the CloudSQL MySQL table with default schema\n", + "\n", + "Before running the pipeline, we need a table to store our embeddings:" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "tgAvMT-yUixY" + }, + "outputs": [], + "source": [ + "table_name = \"default_dataflow_product_embeddings\"\n", + "table_schema = f\"\"\"\n", + " id VARCHAR(255) PRIMARY KEY,\n", + " embedding VECTOR(384) USING VARBINARY,\n", + " content text,\n", + " metadata JSON\n", + "\"\"\"\n", + "setup_db_table_sqlalchemy(CONNECTION_NAME, DB_NAME, table_name,table_schema, DB_USER, DB_PASSWORD)\n", + "test_db_connection_sqlalchemy(CONNECTION_NAME, DB_NAME, table_name, DB_USER, DB_PASSWORD)" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "mcZATJbaOec0" + }, + "source": [ + "## Save our Pipeline to a python file\n", + "\n", + "To launch our pipeline job on DataFlow, we\n", + "1. Add command line arguments for passing pipeline options like CloudSQL MySQL credentioals\n", + "2. Save our pipeline code to a local file `basic_ingestion_pipeline.py`" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "CzhIiBdqOknd" + }, + "outputs": [], + "source": [ + "file_content = \"\"\"\n", + "import apache_beam as beam\n", + "from apache_beam.options.pipeline_options import PipelineOptions\n", + "import argparse\n", + "import tempfile\n", + "\n", + "from apache_beam.ml.transforms.base import MLTransform\n", + "from apache_beam.ml.rag.types import Chunk, Content\n", + "from apache_beam.ml.rag.ingestion.base import VectorDatabaseWriteTransform\n", + "from apache_beam.ml.rag.ingestion.cloudsql import CloudSQLMySQLVectorWriterConfig, LanguageConnectorConfig\n", + "from apache_beam.ml.rag.embeddings.huggingface import HuggingfaceTextEmbeddings\n", + "from apache_beam.options.pipeline_options import SetupOptions\n", + "\n", + "PRODUCTS_DATA = [\n", + " {\n", + " \"id\": \"desk-001\",\n", + " \"name\": \"Modern Minimalist Desk\",\n", + " \"description\": \"Sleek minimalist desk with clean lines and a spacious work surface. \"\n", + " \"Features cable management system and sturdy steel frame. \"\n", + " \"Perfect for contemporary home offices and workspaces.\",\n", + " \"category\": \"Desks\",\n", + " \"price\": 399.99,\n", + " \"material\": \"Engineered Wood, Steel\",\n", + " \"dimensions\": \"60W x 30D x 29H inches\"\n", + " },\n", + " {\n", + " \"id\": \"chair-001\",\n", + " \"name\": \"Ergonomic Mesh Office Chair\",\n", + " \"description\": \"Premium ergonomic office chair with breathable mesh back, \"\n", + " \"adjustable lumbar support, and 4D armrests. Features synchronized \"\n", + " \"tilt mechanism and memory foam seat cushion. Ideal for long work hours.\",\n", + " \"category\": \"Office Chairs\",\n", + " \"price\": 299.99,\n", + " \"material\": \"Mesh, Metal, Premium Foam\",\n", + " \"dimensions\": \"26W x 26D x 48H inches\"\n", + " }\n", + "]\n", + "\n", + "def run(argv=None):\n", + " parser = argparse.ArgumentParser()\n", + " parser.add_argument(\n", + " '--connection_name',\n", + " required=True,\n", + " help='CloudSQL MySQL instance uri'\n", + " )\n", + " parser.add_argument(\n", + " '--cloudsql_database',\n", + " default='mysql',\n", + " help='CloudSQL MySQL database name'\n", + " )\n", + " parser.add_argument(\n", + " '--cloudsql_table',\n", + " required=True,\n", + " help='CloudSQL MySQL table name'\n", + " )\n", + " parser.add_argument(\n", + " '--cloudsql_username',\n", + " required=True,\n", + " help='CloudSQL MySQL user name'\n", + " )\n", + " parser.add_argument(\n", + " '--cloudsql_password',\n", + " required=True,\n", + " help='CloudSQL MySQL password'\n", + " )\n", + " known_args, pipeline_args = parser.parse_known_args(argv)\n", + "\n", + " pipeline_options = PipelineOptions(pipeline_args)\n", + " pipeline_options.view_as(SetupOptions).save_main_session = True\n", + "\n", + " with beam.Pipeline(options=pipeline_options) as p:\n", + " _ = (\n", + " p\n", + " | 'Create Products' >> beam.Create(PRODUCTS_DATA)\n", + " | 'Convert to Chunks' >> beam.Map(lambda product: Chunk(\n", + " content=Content(\n", + " text=f\"{product['name']}: {product['description']}\"\n", + " ), # The text that will be embedded\n", + " id=product['id'], # Use product ID as chunk ID\n", + " metadata=product, # Store all product info in metadata\n", + " )\n", + " )\n", + " | 'Generate Embeddings' >> MLTransform(write_artifact_location=tempfile.mkdtemp())\n", + " .with_transform(\n", + " HuggingfaceTextEmbeddings(model_name=\"sentence-transformers/all-MiniLM-L6-v2\")\n", + " )\n", + " | 'Write to CloudSQL MySQL' >> VectorDatabaseWriteTransform(\n", + " CloudSQLMySQLVectorWriterConfig(\n", + " connection_config=LanguageConnectorConfig(\n", + " username=known_args.cloudsql_username,\n", + " password=known_args.cloudsql_password,\n", + " database_name=known_args.cloudsql_database,\n", + " instance_name=known_args.connection_name\n", + " ),\n", + " table_name=known_args.cloudsql_table\n", + " )\n", + " )\n", + " )\n", + "\n", + "if __name__ == '__main__':\n", + " run()\n", + "\"\"\"\n", + "\n", + "with open(\"basic_ingestion_pipeline.py\", \"w\") as f:\n", + " f.write(file_content)" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "y_1IMXx7UuG4" + }, + "source": [ + "## Authenticate with Google Cloud\n", + "\n", + "To launch a pipeline on Google Cloud, authenticate this notebook. Replace `` with your Google Cloud project ID" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "WxrW-zlgRDLk" + }, + "outputs": [], + "source": [ + "PROJECT_ID = \"\" # @param {type:'string'}\n", + "import os\n", + "os.environ['PROJECT_ID'] = PROJECT_ID" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "GswFBa10Qxkx" + }, + "outputs": [], + "source": [ + "from google.colab import auth\n", + "auth.authenticate_user(project_id=PROJECT_ID)" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "7sELV2KeRG2c" + }, + "source": [ + "## Configure the Pipeline options\n", + "\n" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "nVDW0Q9iS_Pk" + }, + "source": [ + "To run the pipeline on DataFlow we need\n", + "- A gcs bucket for staging DataFlow files. Replace ``: the name of a valid Google Cloud Storage bucket.\n", + "- Optionally set the Google Cloud region that you want to run Dataflow in. Replace `` with the desired location.\n", + "- Optionally provide `NETWORK` and `SUBNETWORK` for dataflow workers to run on." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "qxFJflLiTMua" + }, + "outputs": [], + "source": [ + "import os\n", + "BUCKET_NAME = '' # @param {type:'string'}\n", + "REGION = 'us-central1' # @param {type:'string'}\n", + "\n", + "NETWORK = '' # @param {type:'string'}\n", + "SUBNETWORK = '' # @param {type:'string'}" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "WWjjqwV-aJFi" + }, + "source": [ + "## Provide additional Python dependencies to be installed on Worker VM's\n", + "\n", + "We are making use of the HuggingFace `sentence-transformers` package to generate embeddings. Since this package is not installed on Worker VM's by default, we create a requirements.txt file with the additional dependencies to be installed on worker VM's.\n", + "\n", + "See [Managing Python Pipeline Dependencies](https://beam.apache.org/documentation/sdks/python-pipeline-dependencies/) for more details.\n", + "\n" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "Hkxmk6aTJSKW" + }, + "outputs": [], + "source": [ + "!echo \"sentence-transformers\" > ./requirements.txt\n", + "!cat ./requirements.txt" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "NXgGZeOsY2O7" + }, + "source": [ + "## Run Pipeline on Dataflow\n", + "\n", + "We launch the pipeline via the command line, passing\n", + "- CloudSQL MySQL pipeline arguments defined in `basic_ingestion_pipeline.py`\n", + "- GCP Project ID\n", + "- Job Region\n", + "- The runner (DataflowRunner)\n", + "- Temp and Staging GCS locations for Pipeline artifacts\n", + "- Requirement file location for additional dependencies\n", + "- (Optional) The VPC network and Subnetwork that has access to the CloudSQL MySQL instance\n", + "\n", + "Once the job is launched, you can monitor its progress in the Google Cloud Console:\n", + "1. Go to https://console.cloud.google.com/dataflow/jobs\n", + "2. Select your project\n", + "3. Click on the job named \"cloudsql-dataflow-basic-embedding-ingest\"\n", + "4. View detailed execution graphs, logs, and metrics" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "fUeG_hEb5Qbb" + }, + "outputs": [], + "source": [ + "command_parts = [\n", + " \"python ./basic_ingestion_pipeline.py\",\n", + " f\"--project={PROJECT_ID}\",\n", + " f\"--cloudsql_username={DB_USER}\",\n", + " f\"--connection_name={CONNECTION_NAME}\",\n", + " f\"--cloudsql_password={DB_PASSWORD}\",\n", + " f\"--cloudsql_table=default_dataflow_product_embeddings\",\n", + " f\"--cloudsql_database={DB_NAME}\",\n", + " f\"--job_name=cloudsql-dataflow-basic-embedding-ingest\",\n", + " f\"--region={REGION}\",\n", + " \"--runner=DataflowRunner\",\n", + " f\"--temp_location=gs://{BUCKET_NAME}/temp\",\n", + " f\"--staging_location=gs://{BUCKET_NAME}/staging\",\n", + " \"--requirements_file=requirements.txt\",\n", + "]\n", + "\n", + "if NETWORK:\n", + " command_parts.append(f\"--network={NETWORK}\")\n", + "\n", + "if SUBNETWORK:\n", + " command_parts.append(f\"--subnetwork=regions/{REGION}/subnetworks/{SUBNETWORK}\")\n", + "\n", + "final_command = \" \".join(command_parts)\n", + "\n", + "print(\"Generated command:\\n\", final_command)\n", + "!{final_command}" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "Sp_M6tJbWXTw" + }, + "source": [ + "## Verify the Written Embeddings\n", + "\n", + "Let's check what was written to our CloudSQL MySQL table:" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "A11PeldtWXvP" + }, + "outputs": [], + "source": [ + "verify_embeddings_sqlalchemy(connection_name=CONNECTION_NAME, database=DB_NAME, table_name='default_dataflow_product_embeddings', user=DB_USER, password=DB_PASSWORD)\n" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "-2hTEi-jzYN6" + }, + "source": [ + "# Advanced Use Cases\n", + "\n", + "This section demonstrates more complex scenarios for using CloudSQL MySQL with Apache Beam for vector embeddings.\n", + "\n", + "🎯 **Have a specific schema?**\n", + "- [Go to Custom Schema](#scrollTo=Custom_Schema_with_Column_Mapping)\n", + "- Learn to use different column names and transform values\n", + "- Map metadata to individual columns\n", + "\n", + "🔄 **Need to update embeddings?**\n", + "- [Check out Updating Embeddings](#scrollTo=Update_Embeddings_and_Metadata_with_Conflict_Resolution)\n", + "- Handle conflicts\n", + "- Selective field updates\n", + "\n", + "🔗 **Need to generate and Store Embeddings for Existing CloudSQL MySQL Data??**\n", + "- [See Database Integration](#scrollTo=Adding_Embeddings_to_Existing_Database_Records)\n", + "- Read data from your CloudSQL MySQL table.\n", + "- Generate embeddings for the relevant fields.\n", + "- Update your table (or a related table) with the generated embeddings.\n", + "\n", + "🤖 **Want to use Google's AI models?**\n", + "- [Try Vertex AI Embeddings](#scrollTo=Generate_Embeddings_with_VertexAI_Text_Embeddings)\n", + "- Use Google's powerful embedding models\n", + "- Seamlessly integrate with other Google Cloud services\n", + "\n", + "🔄 Need real-time embedding updates?\n", + "\n", + "- [Try Streaming Embeddings from PubSub](#scrollTo=Streaming_Embeddings_Updates_from_PubSub)\n", + "- Process continuous data streams\n", + "- Update embeddings in real-time as information changes\n" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "qGaH_TqEzn8r" + }, + "source": [ + "## Custom Schema with Column Mapping \n", + "\n", + "In this example, we'll create a custom schema that:\n", + "- Uses different column names\n", + "- Maps metadata to individual columns\n", + "- Uses functions to transform values" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "R4d9W6ry_CN8" + }, + "source": [ + "### ColumnSpec and ColumnSpecsBuilder\n", + "\n", + "\n", + "ColumnSpec specifies how to map data to a database column. For example:\n", + "```python\n", + "from apache_beam.ml.rag.ingestion.mysql_common import ColumnSpecsBuilder\n", + "\n", + "ColumnSpec(\n", + " column_name=\"price\", # Database column\n", + " python_type=float, # Python Type for the value\n", + " value_fn=lambda c: c.metadata['price'], # Extract price from Chunk metadata to get actual value\n", + " placeholder=\"ROUND(?, 2)\" # Optional SQL cast or function\n", + ")\n", + "```\n", + "creates an INSERT statement like:\n", + "```sql\n", + "INSERT INTO table (price) VALUES (?::decimal)\n", + "```\n", + "where the `?` placeholder is poulated with the value from our ingested data.\n", + "\n", + "`ColumnSpecsBuilder` provides a builder and convenience methods to create these `ColumnSpecs`:\n", + "\n", + "1. Core Field Mapping\n", + " - `with_id_spec()` => Insert chunk.id as text in \"id\" column\n", + " - `with_embedding_spec()` => Insert chunk.embedding casted to `VECTOR` via `string_to_vector(?)` in \"embedding\" column\n", + " - `with_content_spec()` => Insert `chunk.content`.text as text in \"content\" column\n", + "\n", + " Note: All `with_id_spec`, `with_embedding_spec`, etc. methods allow overriding `column_name`, `python_type`, and `value_fn`.\n", + "\n", + "2. Metadata Extraction\n", + " - `add_metadata_field`: Creates a column from a `chunk.metadata` field\n", + " - Handles type conversion based on specified SQL type\n", + "\n", + "3. Custom Fields\n", + " - `add_custom_column_spec`: Grants complete control over mapping `Chunk` data to database rows using `ColumnSpec`\n", + "\n", + "Now, lets the table to store our embeddings:" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "6XpYLcCu80Dy" + }, + "source": [ + "### Create Custom Schema Table" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "6bUd6vprzh7O" + }, + "outputs": [], + "source": [ + "table_name = \"custom_product_embeddings\"\n", + "table_schema = \"\"\"\n", + " product_id VARCHAR(255) PRIMARY KEY,\n", + " vector_embedding VECTOR(384) USING VARBINARY,\n", + " product_name VARCHAR(255),\n", + " description TEXT,\n", + " price DECIMAL,\n", + " category VARCHAR(255),\n", + " display_text VARCHAR(255),\n", + " model_name VARCHAR(255),\n", + " created_at TIMESTAMP\n", + "\"\"\"\n", + "setup_db_table_sqlalchemy(CONNECTION_NAME, DB_NAME, table_name,table_schema, DB_USER, DB_PASSWORD)\n", + "test_db_connection_sqlalchemy(CONNECTION_NAME, DB_NAME, table_name, DB_USER, DB_PASSWORD)" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "ScCVCZFo-Fcv" + }, + "source": [ + "### Configure Pipeline Components" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "g9-f7tcf-0qC" + }, + "source": [ + "#### Write to custom schema using ColumnSpecsBuilder\n", + "\n", + "\n", + "We configure ConlumnSpecsBuilder to map data as:\n", + "\n", + "| Database Column | Chunk Field |\n", + "|-----------------|-------------------------------------------|\n", + "| `product_id` | `chunk.id` |\n", + "| `vector_embedding`| `chunk.embedding.dense_embedding` |\n", + "| `description` | `chunk.content.text` |\n", + "| `product_name` | `chunk.metadata['name']` |\n", + "| `price` | `chunk.metadata['price']` |\n", + "| `category` | `chunk.metadata['category']` |\n", + "| `display_text` | *Function that combines product name and price* |\n", + "| `model_name` | *Function that returns the model name: \"all-MiniLM-L6-v2\"* |\n", + "| `created_at` | *Function that returns the current timestamp cast to a SQL timestamp* |" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "TAq6ydMn-5Uu" + }, + "outputs": [], + "source": [ + "from apache_beam.ml.rag.ingestion.mysql_common import ColumnSpecsBuilder\n", + "from apache_beam.ml.rag.ingestion.mysql_common import ColumnSpec\n", + "from datetime import datetime\n", + "\n", + "column_specs = (\n", + " ColumnSpecsBuilder()\n", + " # Write chunk.id to a column named \"product_id\"\n", + " .with_id_spec(column_name='product_id')\n", + " # Write chunk.embedding.dense_embedding to a column named \"vector_embedding\"\n", + " .with_embedding_spec(column_name='vector_embedding')\n", + " # Write chunk.content.text to a column named \"description\"\n", + " .with_content_spec(column_name='description')\n", + " # Write chunk.metadata.['product_name'] to a column named \"product_name\"\n", + " .add_metadata_field(\n", + " field='name',\n", + " column_name='product_name',\n", + " python_type=str\n", + " )\n", + " # Write chunk.metadata.['price'] to a column named \"price\"\n", + " .add_metadata_field(\n", + " field='price',\n", + " column_name='price',\n", + " python_type=float\n", + " )\n", + " # Write chunk.metadata.['category'] to a column named \"category\"\n", + " .add_metadata_field(\n", + " field='category',\n", + " column_name='category',\n", + " python_type=str\n", + " )\n", + " # Write custom field using value_fn to column named \"display_text\" using\n", + " # ColumnSpec.text convenience method\n", + " .add_custom_column_spec(\n", + " ColumnSpec.text(\n", + " column_name='display_text',\n", + " value_fn=lambda chunk: \\\n", + " f\"{chunk.metadata['name']} - ${chunk.metadata['price']:.2f}\"\n", + " )\n", + " )\n", + " # Store model used to generate embedding using ColumnSpec constructor\n", + " .add_custom_column_spec(\n", + " ColumnSpec(\n", + " column_name='model_name',\n", + " python_type=str,\n", + " value_fn=lambda _: \"all-MiniLM-L6-v2\"\n", + " )\n", + " )\n", + " .add_custom_column_spec(\n", + " ColumnSpec(\n", + " column_name='created_at',\n", + " python_type=str,\n", + " value_fn=lambda _: datetime.now().isoformat()\n", + " )\n", + " )\n", + " .build()\n", + ")" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "MBfLVL6XX2mF" + }, + "source": [ + "### Assemble and Run Pipeline\n", + "\n", + "Now we can create our pipeline that will:\n", + "1. Take our product data\n", + "2. Convert each product to a Chunk\n", + "3. Generate embeddings for each Chunk\n", + "4. Store everything in CloudSQL MySQL with our custom schema configuration" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "4V-ILUlWVVX8" + }, + "outputs": [], + "source": [ + "import tempfile # For storing MLTransform artifacts\n", + "\n", + "# Executing on DirectRunner (local execution)\n", + "with beam.Pipeline() as p:\n", + " _ = (\n", + " p\n", + " | 'Create Products' >> beam.Create(PRODUCTS_DATA)\n", + " | 'Convert to Chunks' >> beam.Map(lambda product: Chunk(\n", + " content=Content(\n", + " text=f\"{product['name']}: {product['description']}\"\n", + " ), # The text that will be embedded\n", + " id=product['id'], # Use product ID as chunk ID\n", + " metadata=product, # Store all product info in metadata\n", + " )\n", + " )\n", + " | 'Generate Embeddings' >> MLTransform(write_artifact_location=tempfile.mkdtemp())\n", + " .with_transform(HuggingfaceTextEmbeddings(model_name=\"sentence-transformers/all-MiniLM-L6-v2\"))\n", + " | 'Write to CloudSQL MySQL' >> VectorDatabaseWriteTransform(\n", + " CloudSQLMySQLVectorWriterConfig(\n", + " connection_config=LanguageConnectorConfig(\n", + " username=DB_USER,\n", + " password=DB_PASSWORD,\n", + " database_name=DB_NAME,\n", + " instance_name=CONNECTION_NAME\n", + " ),\n", + " table_name=table_name,\n", + " column_specs=column_specs\n", + " )\n", + " )\n", + " )" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "LCpoJkBpYEsH" + }, + "source": [ + "### Verify the Written Embeddings\n", + "\n", + "Let's check what was written to our CloudSQL MySQL table:" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "B2UDOZL0VZ-p" + }, + "outputs": [], + "source": [ + "verify_embeddings_sqlalchemy(connection_name=CONNECTION_NAME, database=DB_NAME, table_name=table_name, user=DB_USER, password=DB_PASSWORD, embedding_column=\"vector_embedding\")" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "DQyJoyZic9GT" + }, + "source": [ + "## Update Embeddings and Metadata with Conflict Resolution \n", + "\n", + "This section demonstrates how to handle periodic updates to product descriptions and their embeddings using the default schema. We'll show how embeddings and metadata get updated when product descriptions change.\n" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "jwLHGKfNdbEG" + }, + "source": [ + "### Create table with desired schema\n", + "\n", + "Let's use the same default schema as in Quick Start:" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "0vK-b4xkXtgJ" + }, + "outputs": [], + "source": [ + "table_name = \"mutable_product_embeddings\"\n", + "table_schema = f\"\"\"\n", + " id VARCHAR(255) PRIMARY KEY,\n", + " embedding VECTOR(384) USING VARBINARY,\n", + " content text,\n", + " metadata JSON,\n", + " created_at TIMESTAMP DEFAULT CURRENT_TIMESTAMP\n", + "\"\"\"\n", + "setup_db_table_sqlalchemy(CONNECTION_NAME, DB_NAME, table_name,table_schema, DB_USER, DB_PASSWORD)\n", + "test_db_connection_sqlalchemy(CONNECTION_NAME, DB_NAME, table_name, DB_USER, DB_PASSWORD)" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "hhl2URWceSg_" + }, + "source": [ + "### Sample Data: Day 1 vs Day 2" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "t4z8tM_leZV8" + }, + "outputs": [], + "source": [ + "PRODUCTS_DATA_DAY1 = [\n", + " {\n", + " \"id\": \"desk-001\",\n", + " \"name\": \"Modern Minimalist Desk\",\n", + " \"description\": \"Sleek minimalist desk with clean lines and a spacious work surface. \"\n", + " \"Features cable management system and sturdy steel frame.\",\n", + " \"category\": \"Desks\",\n", + " \"price\": 399.99,\n", + " \"update_timestamp\": \"2024-02-18\"\n", + " }\n", + "]\n", + "\n", + "PRODUCTS_DATA_DAY2 = [\n", + " {\n", + " \"id\": \"desk-001\", # Same ID as Day 1\n", + " \"name\": \"Modern Minimalist Desk\",\n", + " \"description\": \"Updated: Sleek minimalist desk with built-in wireless charging. \"\n", + " \"Features cable management system, sturdy steel frame, and Qi charging pad. \"\n", + " \"Perfect for modern tech-enabled workspaces.\",\n", + " \"category\": \"Smart Desks\", # Category changed\n", + " \"price\": 449.99, # Price increased\n", + " \"update_timestamp\": \"2024-02-19\"\n", + " }\n", + "]" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "W_UTcRz9eskE" + }, + "source": [ + "### Configure Pipeline Components" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "PWvtwVmUedzw" + }, + "source": [ + "#### Writer with Conflict Resolution" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "Y2XEwxw6ee4b" + }, + "outputs": [], + "source": [ + "from apache_beam.ml.rag.ingestion.cloudsql import (\n", + " CloudSQLMySQLVectorWriterConfig,\n", + " LanguageConnectorConfig,\n", + ")\n", + "from apache_beam.ml.rag.ingestion.mysql_common import ConflictResolution\n", + "\n", + "# Define how to handle conflicts - update all fields when ID matches\n", + "conflict_resolution = ConflictResolution(\n", + " action=\"UPDATE\", # Update existing records\n", + " update_fields=[\"embedding\", \"content\", \"metadata\"]\n", + ")\n", + "\n", + "# Create writer config with conflict resolution\n", + "cloudsql_writer_config = CloudSQLMySQLVectorWriterConfig(\n", + " connection_config=LanguageConnectorConfig(\n", + " username=DB_USER,\n", + " password=DB_PASSWORD,\n", + " database_name=DB_NAME,\n", + " instance_name=CONNECTION_NAME\n", + " ),\n", + " table_name=table_name,\n", + " conflict_resolution=conflict_resolution,\n", + ")" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "tzo43G9NfCr5" + }, + "outputs": [], + "source": [ + "huggingface_embedder = HuggingfaceTextEmbeddings(\n", + " model_name=\"sentence-transformers/all-MiniLM-L6-v2\"\n", + ")" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "axMFW_DufKnO" + }, + "source": [ + "### Run Day 1 Pipeline\n", + "\n", + "First, let's ingest our initial product data:" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "eA3TpkHMfLUq" + }, + "outputs": [], + "source": [ + "# Executing on DirectRunner (local execution)\n", + "with beam.Pipeline() as p:\n", + " _ = (\n", + " p\n", + " | 'Create Day 1 Products' >> beam.Create(PRODUCTS_DATA_DAY1)\n", + " | 'Convert Day 1 to Chunks' >> beam.Map(lambda product: Chunk(\n", + " content=Content(\n", + " text=f\"{product['name']}: {product['description']}\"\n", + " ), # The text that will be embedded\n", + " id=product['id'], # Use product ID as chunk ID\n", + " metadata=product, # Store all product info in metadata\n", + " )\n", + " )\n", + " | 'Generate Day1 Embeddings' >> MLTransform(write_artifact_location=tempfile.mkdtemp())\n", + " .with_transform(HuggingfaceTextEmbeddings(model_name=\"sentence-transformers/all-MiniLM-L6-v2\"))\n", + " | 'Write Day 1 to CloudSQL MySQL' >> VectorDatabaseWriteTransform(\n", + " cloudsql_writer_config\n", + " )\n", + " )" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "hFjtKX9tZIrI" + }, + "source": [ + "#### Verify Initial Data" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "lxSyaIhbZG52" + }, + "outputs": [], + "source": [ + "print(\"\\nAfter Day 1 ingestion:\")\n", + "verify_embeddings_sqlalchemy(connection_name=CONNECTION_NAME, database=DB_NAME, table_name=table_name, user=DB_USER, password=DB_PASSWORD)" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "yvOlen9qfSQ4" + }, + "source": [ + "### Run Day 2 Pipeline\n", + "\n", + "Now let's process our updated product data:" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "r19qQs6ifVq1" + }, + "outputs": [], + "source": [ + "# Executing on DirectRunner (local execution)\n", + "with beam.Pipeline() as p:\n", + " _ = (\n", + " p\n", + " | 'Create Day 2 Products' >> beam.Create(PRODUCTS_DATA_DAY2)\n", + " | 'Convert Day 2 to Chunks' >> beam.Map(lambda product: Chunk(\n", + " content=Content(\n", + " text=f\"{product['name']}: {product['description']}\"\n", + " ), # The text that will be embedded\n", + " id=product['id'], # Use product ID as chunk ID\n", + " metadata=product, # Store all product info in metadata\n", + " )\n", + " )\n", + " | 'Generate Day 2 Embeddings' >> MLTransform(write_artifact_location=tempfile.mkdtemp())\n", + " .with_transform(HuggingfaceTextEmbeddings(model_name=\"sentence-transformers/all-MiniLM-L6-v2\"))\n", + " | 'Write Day 2 to CloudSQL MySQL' >> VectorDatabaseWriteTransform(\n", + " cloudsql_writer_config\n", + " )\n", + " )" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "QbcZOIdcZWA6" + }, + "source": [ + "#### Verify Updated Data" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "_VpqhPAQZD4K" + }, + "outputs": [], + "source": [ + "print(\"\\nAfter Day 2 ingestion:\")\n", + "verify_embeddings_sqlalchemy(connection_name=CONNECTION_NAME, database=DB_NAME, table_name=table_name, user=DB_USER, password=DB_PASSWORD)" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "D5hImiN0fZo5" + }, + "source": [ + "### What Changed?\n", + "\n", + "Key points to notice:\n", + "\n", + "1. The embedding vector changed because the product description was updated\n", + "2. The metadata JSON field contains the updated category, price, and timestamp\n", + "3. The content field reflects the new description\n", + "4. The original ID remained the same\n", + "\n", + "This pattern allows you to:\n", + "- Update embeddings when source text changes\n", + "- Maintain referential integrity with consistent IDs\n", + "- Track changes through the metadata field\n", + "- Handle conflicts gracefully using CloudSQL MySQL's conflict resolution\n" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "ndovzTB0mLdg" + }, + "source": [ + "## Adding Embeddings to Existing Database Records \n", + "\n", + "This section demonstrates how to:\n", + "1. Read existing product data from a database\n", + "2. Generate embeddings for that data\n", + "3. Write the embeddings back to the database" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "l3-wl9e1fjms" + }, + "outputs": [], + "source": [ + "table_name = \"existing_products\"\n", + "table_schema = \"\"\"\n", + " id VARCHAR(255) PRIMARY KEY,\n", + " title VARCHAR(255) NOT NULL,\n", + " description TEXT,\n", + " price DECIMAL,\n", + " embedding VECTOR(384) USING VARBINARY\n", + "\"\"\"" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "2cjjrbjUmaUN", + "cellView": "form" + }, + "outputs": [], + "source": [ + "#@title MySQL helper for inserting initial records\n", + "import sqlalchemy\n", + "from sqlalchemy import text\n", + "from sqlalchemy.exc import SQLAlchemyError\n", + "# The google.cloud.sql.connector and a driver like PyMySQL (`pip install pymysql`)\n", + "# are required for this to connect to Cloud SQL.\n", + "from google.cloud.sql.connector import Connector\n", + "\n", + "# Assume get_db_engine is defined elsewhere to connect using a MySQL dialect,\n", + "# e.g., 'mysql+pymysql'\n", + "# from your_utils import get_db_engine\n", + "\n", + "def setup_initial_data_sqlalchemy(connection_name: str,\n", + " database: str,\n", + " table_name: str,\n", + " table_schema: str,\n", + " user: str,\n", + " password: str,\n", + " **connect_kwargs):\n", + " \"\"\"Sets up a table and inserts sample data into a MySQL database using SQLAlchemy.\n", + "\n", + " This function will drop the specified table if it exists, recreate it based on the\n", + " provided schema, and insert a predefined set of sample products.\n", + "\n", + " Args:\n", + " connection_name: Cloud SQL MySQL instance connection name string.\n", + " database: Name of the database.\n", + " table_name: Name of the table to create and populate.\n", + " table_schema: A string containing MySQL-compatible column definitions\n", + " (e.g., \"id VARCHAR(255) PRIMARY KEY, title VARCHAR(255)\").\n", + " user: Database username.\n", + " password: Database password.\n", + " connect_kwargs: Additional keyword arguments for the Cloud SQL connector.\n", + " \"\"\"\n", + " engine = None\n", + " try:\n", + " # Assumes get_db_engine returns a SQLAlchemy engine configured for MySQL\n", + " engine = get_db_engine(connection_name, user, password, database, **connect_kwargs)\n", + "\n", + " with engine.connect() as connection:\n", + " print(\"✅ Connected to Cloud SQL MySQL successfully via SQLAlchemy!\")\n", + "\n", + " # DDL operations (DROP/CREATE) in MySQL cause an implicit commit,\n", + " # so they are run outside an explicit transaction block.\n", + " print(f\"Dropping table `{table_name}` if it exists...\")\n", + " connection.execute(text(f\"DROP TABLE IF EXISTS `{table_name}`;\"))\n", + "\n", + " print(f\"Creating table `{table_name}`...\")\n", + " # Note: Ensure the table_schema and sample data columns match.\n", + " create_sql = f\"CREATE TABLE `{table_name}` ({table_schema});\"\n", + " connection.execute(text(create_sql))\n", + " print(f\"Table `{table_name}` created.\")\n", + "\n", + " # Define the sample data to be inserted.\n", + " sample_products_dicts = [\n", + " {\n", + " \"id\": \"lamp-001\", \"title\": \"Artisan Table Lamp\",\n", + " \"description\": \"Hand-crafted ceramic...\", \"price\": 129.99\n", + " },\n", + " {\n", + " \"id\": \"mirror-001\", \"title\": \"Floating Wall Mirror\",\n", + " \"description\": \"Modern circular mirror...\", \"price\": 199.99\n", + " },\n", + " {\n", + " \"id\": \"vase-001\", \"title\": \"Contemporary Ceramic Vase\",\n", + " \"description\": \"Minimalist vase...\", \"price\": 79.99\n", + " }\n", + " ]\n", + "\n", + " # The INSERT statement uses named placeholders matching the dictionary keys.\n", + " insert_sql = text(f\"\"\"\n", + " INSERT INTO `{table_name}` (id, title, description, price)\n", + " VALUES (:id, :title, :description, :price)\n", + " \"\"\")\n", + "\n", + " print(f\"Inserting sample data into `{table_name}`...\")\n", + " # SQLAlchemy executes the insert for each dictionary in the list.\n", + " # This runs within a new transaction block started by the `connect()` context.\n", + " connection.execute(insert_sql, sample_products_dicts)\n", + "\n", + " # Explicitly commit the transaction that contains the INSERT statements.\n", + " connection.commit()\n", + " print(\"✓ Sample products inserted successfully.\")\n", + "\n", + " except SQLAlchemyError as e:\n", + " print(f\"❌ An SQLAlchemy error occurred during setup: {e}\")\n", + " except Exception as e:\n", + " print(f\"❌ An unexpected error occurred during setup: {e}\")\n", + " finally:\n", + " if engine:\n", + " # Dispose of the engine to close all connections in the pool.\n", + " engine.dispose()\n", + " print(\"Database engine pool disposed.\")" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "HjHOJ0sRmrwu" + }, + "outputs": [], + "source": [ + "setup_initial_data_sqlalchemy(CONNECTION_NAME, DB_NAME, table_name, table_schema, DB_USER, DB_PASSWORD)" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "MCN0mI08m0Ba" + }, + "source": [ + "### Read from Database and Generate Embeddings\n", + "\n", + "Now let's create a pipeline to read the existing data, generate embeddings, and write back:" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "Q2gY_kh1m08Z" + }, + "outputs": [], + "source": [ + "from apache_beam.io.jdbc import ReadFromJdbc\n", + "from apache_beam.io.jdbc import WriteToJdbc\n", + "from apache_beam.ml.rag.ingestion.mysql_common import ColumnSpecsBuilder\n", + "\n", + "# Configure database writer\n", + "cloudsql_writer_config = CloudSQLMySQLVectorWriterConfig(\n", + " connection_config=LanguageConnectorConfig(\n", + " username=DB_USER,\n", + " password=DB_PASSWORD,\n", + " database_name=DB_NAME,\n", + " instance_name=CONNECTION_NAME\n", + " ),\n", + " table_name=table_name,\n", + " column_specs=(\n", + " ColumnSpecsBuilder()\n", + " .with_id_spec()\n", + " .with_embedding_spec()\n", + " # Add a placeholder value for the title column, because it has a\n", + " # NOT NULL constraint. Insert with Conflict resolution statements in\n", + " # MySQL requires all NOT NULL fields to have a value, even if the\n", + " # value will not be updated (the original title is preserved).\n", + " .add_custom_column_spec(\n", + " ColumnSpec.text(\"title\", value_fn=lambda x: \"\")\n", + " )\n", + " .build()\n", + " ),\n", + " conflict_resolution=ConflictResolution(\n", + " action=\"UPDATE\",\n", + " update_fields=[\"embedding\"] # Update the embedding field\n", + " )\n", + ")\n", + "\n", + "# Create and run pipeline on DirectRunner (local execution)\n", + "with beam.Pipeline() as p:\n", + " # Read existing products\n", + " rows = (\n", + " p\n", + " | \"Read Products\" >> ReadFromJdbc(\n", + " table_name=table_name,\n", + " driver_class_name=\"com.mysql.cj.jdbc.Driver\",\n", + " jdbc_url=cloudsql_writer_config.connector_config.to_connection_config(\n", + " ).jdbc_url,\n", + " username=DB_USER,\n", + " password=DB_PASSWORD,\n", + " query=f\"SELECT id, title, description FROM {table_name}\",\n", + " classpath=cloudsql_writer_config.connector_config.additional_jdbc_args()['classpath']\n", + " )\n", + " )\n", + "\n", + " # Generate and write embeddings\n", + " _ = (\n", + " rows\n", + " | \"Convert to Chunks\" >> beam.Map(lambda row: Chunk(\n", + " id=row.id,\n", + " content=Content(text=f\"{row.title}: {row.description}\")\n", + " )\n", + " )\n", + " | \"Generate Embeddings\" >> MLTransform(\n", + " write_artifact_location=tempfile.mkdtemp()\n", + " ).with_transform(HuggingfaceTextEmbeddings(model_name=\"sentence-transformers/all-MiniLM-L6-v2\"))\n", + " | \"Write Back to CloudSQL MySQL\" >> VectorDatabaseWriteTransform(\n", + " cloudsql_writer_config\n", + " )\n", + " )\n", + "\n" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "bBNl5DK3Zh58" + }, + "source": [ + "### Verify Data" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "elU53NLtZlTf" + }, + "outputs": [], + "source": [ + "print(\"\\nAfter embedding generation:\")\n", + "verify_embeddings_sqlalchemy(connection_name=CONNECTION_NAME, database=DB_NAME, table_name=table_name, user=DB_USER, password=DB_PASSWORD)" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "ZFgpFarCp4Wo" + }, + "source": [ + "What Happened?\n", + "1. We started with a table containing product data but no embeddings\n", + "2. Read the existing records using ReadFromJdbc\n", + "3. Converted rows to Chunks, combining title and description for embedding\n", + "4. Generated embeddings using our model\n", + "5. Wrote back to the same table, updating only the embedding field\n", + "Preserved all other fields (price, etc.)\n", + "\n", + "This pattern is useful when:\n", + "\n", + "- You have an existing product database\n", + "- You want to add embeddings without disrupting current data\n", + "- You need to maintain existing schema and relationships\n" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "-L8mGusPd83L" + }, + "source": [ + "## Generate Embeddings with VertexAI Text Embeddings" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "dVB1qAARmOlc" + }, + "source": [ + "This section demonstrates how to use use the Vertex AI text-embeddings API to generate text embeddings that use Googles large generative artificial intelligence (AI) models.\n", + "\n", + "Vertex AI models are subject to [Rate Limits and Quotas](https://cloud.google.com/vertex-ai/generative-ai/docs/quotas#view-the-quotas-by-region-and-by-model) and Dataflow automatically retries throttled requests with exponential backoff.\n", + "\n", + "\n", + "For more information, see [Get text embeddings](https://cloud.google.com/vertex-ai/docs/generative-ai/embeddings/get-text-embeddings) in the Vertex AI documentation." + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "-eLuZ78Tqm4w" + }, + "source": [ + "### Authenticate with Google Cloud\n", + "To use the Vertex AI API, we authenticate with Google Cloud." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "84p608l4ql8p" + }, + "outputs": [], + "source": [ + "# Replace with a valid Google Cloud project ID.\n", + "PROJECT_ID = '' # @param {type:'string'}\n", + "\n", + "from google.colab import auth\n", + "auth.authenticate_user(project_id=PROJECT_ID)" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "9PZVv8S5oTHo" + }, + "source": [ + "### Create CloudSQL MySQL table with default schema\n", + "\n", + "First we create a table to store our embeddings:" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "cEuU4JkVkLBk" + }, + "outputs": [], + "source": [ + "table_name = \"vertex_product_embeddings\"\n", + "table_schema = f\"\"\"\n", + " id VARCHAR(255) PRIMARY KEY,\n", + " embedding VECTOR(768) USING VARBINARY,\n", + " content text,\n", + " metadata JSON\n", + "\"\"\"\n", + "setup_db_table_sqlalchemy(CONNECTION_NAME, DB_NAME, table_name,table_schema, DB_USER, DB_PASSWORD)\n", + "test_db_connection_sqlalchemy(CONNECTION_NAME, DB_NAME, table_name, DB_USER, DB_PASSWORD)" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "QZ7tSAfQpG_Z" + }, + "source": [ + "### Configure Embedding Handler\n", + "\n", + "Import the `VertexAITextEmbeddings` handler, and specify the desired `textembedding-gecko` model." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "Ipv7R6G9pqnx" + }, + "outputs": [], + "source": [ + "from apache_beam.ml.rag.embeddings.vertex_ai import VertexAITextEmbeddings\n", + "\n", + "vertexai_embedder = VertexAITextEmbeddings(model_name=\"text-embedding-005\")" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "D7VoYav9rQJU" + }, + "source": [ + "### Run the Pipeline" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "fi5SMGpZrPEm" + }, + "outputs": [], + "source": [ + "import tempfile\n", + "\n", + "# Executing on DirectRunner (local execution)\n", + "with beam.Pipeline() as p:\n", + " _ = (\n", + " p\n", + " | 'Create Products' >> beam.Create(PRODUCTS_DATA)\n", + " | 'Convert to Chunks' >> beam.Map(lambda product: Chunk(\n", + " content=Content(\n", + " text=f\"{product['name']}: {product['description']}\"\n", + " ), # The text that will be embedded\n", + " id=product['id'], # Use product ID as chunk ID\n", + " metadata=product, # Store all product info in metadata\n", + " )\n", + " )\n", + " | 'Generate Embeddings' >> MLTransform(write_artifact_location=tempfile.mkdtemp())\n", + " .with_transform(\n", + " vertexai_embedder\n", + " )\n", + " | 'Write to CloudSQL MySQL' >> VectorDatabaseWriteTransform(\n", + " CloudSQLMySQLVectorWriterConfig(\n", + " connection_config=LanguageConnectorConfig(\n", + " username=DB_USER,\n", + " password=DB_PASSWORD,\n", + " database_name=DB_NAME,\n", + " instance_name=CONNECTION_NAME\n", + " ),\n", + " table_name=table_name\n", + " )\n", + " )\n", + " )" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "9hVYw0rspp7Y" + }, + "source": [ + "### Verify Embeddings" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "xSEY1IILsMvi" + }, + "outputs": [], + "source": [ + "print(\"\\nAfter embedding generation:\")\n", + "verify_embeddings_sqlalchemy(connection_name=CONNECTION_NAME, database=DB_NAME, table_name=table_name, user=DB_USER, password=DB_PASSWORD)" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "yv4Rd1ZvsB_M" + }, + "source": [ + "## Streaming Embeddings Updates from PubSub\n", + "\n", + "This section demonstrates how to build a real-time embedding pipeline that continuously processes product updates and maintains fresh embeddings in CloudSQL MySQL. This approach is ideal data that changes frequently.\n", + "\n", + "This example runs on Dataflow because streaming with DirectRunner and writing via JDBC is not supported.\n", + "\n", + "### Authenticate with Google Cloud\n", + "To use the PubSub, we authenticate with Google Cloud.\n" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "VCqJmaznt1nS" + }, + "outputs": [], + "source": [ + "# Replace with a valid Google Cloud project ID.\n", + "PROJECT_ID = '' # @param {type:'string'}\n", + "\n", + "from google.colab import auth\n", + "auth.authenticate_user(project_id=PROJECT_ID)" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "2FsoFaugtsln" + }, + "source": [ + "### Setting Up PubSub Resources\n", + "\n", + "First, let's set up the necessary PubSub topics and subscriptions:" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "nqMe0Brlt7Bk" + }, + "outputs": [], + "source": [ + "from google.cloud import pubsub_v1\n", + "from google.api_core.exceptions import AlreadyExists\n", + "import json\n", + "\n", + "# Define pubsub topic\n", + "TOPIC = \"product-updates\" # @param {type:'string'}\n", + "\n", + "# Create publisher client and topic\n", + "publisher = pubsub_v1.PublisherClient()\n", + "topic_path = publisher.topic_path(PROJECT_ID, TOPIC)\n", + "try:\n", + " topic = publisher.create_topic(request={\"name\": topic_path})\n", + " print(f\"Created topic: {topic.name}\")\n", + "except AlreadyExists:\n", + " print(f\"Topic {topic_path} already exists.\")" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "07ZFeGbMuFj_" + }, + "source": [ + "### Create CloudSQL MySQL Table for Streaming Updates\n", + "\n", + "Next, create a table to store the embedded data." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "3Xc70uV_uJy5" + }, + "outputs": [], + "source": [ + "table_name = \"streaming_product_embeddings\"\n", + "table_schema = \"\"\"\n", + " id VARCHAR(255) PRIMARY KEY,\n", + " embedding VECTOR(384) USING VARBINARY,\n", + " content text,\n", + " metadata JSON,\n", + " created_at TIMESTAMP DEFAULT CURRENT_TIMESTAMP\n", + "\"\"\"" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "8HPhUfAuorBP" + }, + "outputs": [], + "source": [ + "setup_db_table_sqlalchemy(CONNECTION_NAME, DB_NAME, table_name,table_schema, DB_USER, DB_PASSWORD)\n", + "test_db_connection_sqlalchemy(CONNECTION_NAME, DB_NAME, table_name, DB_USER, DB_PASSWORD)" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "LSriDxtsn1wH" + }, + "source": [ + "### Configure the Pipeline options\n", + "To run the pipeline on DataFlow we need\n", + "- A gcs bucket for staging DataFlow files. Replace ``: the name of a valid Google Cloud Storage bucket. Don't include a gs:// prefix or trailing slashes\n", + "- Optionally set the Google Cloud region that you want to run Dataflow in. Replace `` with the desired location\n" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "kR0x7vzTrUlZ" + }, + "outputs": [], + "source": [ + "from apache_beam.options.pipeline_options import PipelineOptions, StandardOptions, SetupOptions, GoogleCloudOptions, WorkerOptions\n", + "\n", + "options = PipelineOptions()\n", + "options.view_as(StandardOptions).streaming = True\n", + "\n", + "# Provide required pipeline options for the Dataflow Runner.\n", + "options.view_as(StandardOptions).runner = \"DataflowRunner\"\n", + "\n", + "# Set the Google Cloud region that you want to run Dataflow in.\n", + "REGION = 'us-central1' # @param {type:'string'}\n", + "options.view_as(GoogleCloudOptions).region = REGION\n", + "\n", + "NETWORK = '' # @param {type:'string'}\n", + "if NETWORK:\n", + " options.view_as(WorkerOptions).network = NETWORK\n", + "\n", + "SUBNETWORK = '' # @param {type:'string'}\n", + "if SUBNETWORK:\n", + " options.view_as(WorkerOptions).subnetwork = f\"regions/{REGION}/subnetworks/{SUBNETWORK}\"\n", + "\n", + "options.view_as(GoogleCloudOptions).project = PROJECT_ID\n", + "\n", + "BUCKET_NAME = '' # @param {type:'string'}\n", + "dataflow_gcs_location = \"gs://%s/dataflow\" % BUCKET_NAME\n", + "\n", + "# The Dataflow staging location. This location is used to stage the Dataflow pipeline and the SDK binary.\n", + "options.view_as(GoogleCloudOptions).staging_location = '%s/staging' % dataflow_gcs_location\n", + "\n", + "# The Dataflow temp location. This location is used to store temporary files or intermediate results before outputting to the sink.\n", + "options.view_as(GoogleCloudOptions).temp_location = '%s/temp' % dataflow_gcs_location\n", + "\n", + "import random\n", + "options.view_as(GoogleCloudOptions).job_name = f\"cloudsql-streaming-embedding-ingest{random.randint(0,1000)}\"\n", + "\n", + "# options.view_as(SetupOptions).save_main_session = True\n", + "options.view_as(SetupOptions).requirements_file = \"./requirements.txt\"\n" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "gMKuccfHoDki" + }, + "source": [ + "### Provide additional Python dependencies to be installed on Worker VM's\n", + "\n", + "We are making use of the HuggingFace `sentence-transformers` package to generate embeddings. Since this package is not installed on Worker VM's by default, we create a requirements.txt file with the additional dependencies to be installed on worker VM's.\n", + "\n", + "See [Managing Python Pipeline Dependencies](https://beam.apache.org/documentation/sdks/python-pipeline-dependencies/) for more details.\n", + "\n" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "RTGoA0SmoEvm" + }, + "outputs": [], + "source": [ + "!echo \"sentence-transformers\" > ./requirements.txt\n", + "!cat ./requirements.txt" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "eU0Sn19nqzLM" + }, + "source": [ + "### Configure and Run Pipeline\n", + "\n", + "Our pipeline contains these key components:\n", + "\n", + "1. **Source**: Continuously reads messages from PubSub\n", + "2. **Windowing**: Groups messages into 10-second windows for batch processing\n", + "3. **Transformation**: Converts JSON messages to Chunk objects for embedding\n", + "4. **ML Processing**: Generates embeddings using HuggingFace models\n", + "5. **Sink**: Writes results to CloudSQL MySQL with conflict resolution" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "w2pmJn5fqXHx" + }, + "outputs": [], + "source": [ + "import apache_beam as beam\n", + "import tempfile\n", + "import json\n", + "\n", + "from apache_beam.ml.transforms.base import MLTransform\n", + "from apache_beam.ml.rag.types import Chunk, Content\n", + "from apache_beam.ml.rag.ingestion.base import VectorDatabaseWriteTransform\n", + "from apache_beam.ml.rag.ingestion.cloudsql import CloudSQLMySQLVectorWriterConfig\n", + "from apache_beam.ml.rag.ingestion.cloudsql import LanguageConnectorConfig\n", + "\n", + "from apache_beam.ml.rag.ingestion.mysql_common import ConflictResolution\n", + "\n", + "from apache_beam.ml.rag.embeddings.huggingface import HuggingfaceTextEmbeddings\n", + "from apache_beam.transforms.window import FixedWindows\n", + "\n", + "def parse_message(message):\n", + " #Parse a message containing product data.\n", + " product_json = json.loads(message.decode('utf-8'))\n", + " return Chunk(\n", + " content=Content(\n", + " text=f\"{product_json.get('name', '')}: {product_json.get('description', '')}\"\n", + " ),\n", + " id=product_json.get('id', ''),\n", + " metadata=product_json\n", + " )\n", + "\n", + "pipeline = beam.Pipeline(options=options)\n", + "# Streaming pipeline\n", + "_ = (\n", + " pipeline\n", + " | \"Read from PubSub\" >> beam.io.ReadFromPubSub(\n", + " topic=f\"projects/{PROJECT_ID}/topics/{TOPIC}\"\n", + " )\n", + " | \"Window\" >> beam.WindowInto(FixedWindows(10))\n", + " | \"Parse Messages\" >> beam.Map(parse_message)\n", + " | \"Generate Embeddings\" >> MLTransform(write_artifact_location=tempfile.mkdtemp())\n", + " .with_transform(HuggingfaceTextEmbeddings(model_name=\"sentence-transformers/all-MiniLM-L6-v2\"))\n", + " | \"Write to CloudSQL MySQL\" >> VectorDatabaseWriteTransform(\n", + " CloudSQLMySQLVectorWriterConfig(\n", + " connection_config=LanguageConnectorConfig(\n", + " username=DB_USER,\n", + " password=DB_PASSWORD,\n", + " database_name=DB_NAME,\n", + " instance_name=CONNECTION_NAME\n", + " ),\n", + " table_name=table_name,\n", + " conflict_resolution=ConflictResolution(\n", + " on_conflict_fields=\"id\",\n", + " action=\"UPDATE\",\n", + " update_fields=[\"embedding\", \"content\", \"metadata\"]\n", + " )\n", + " )\n", + " )\n", + ")\n", + "\n" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "r7nJdc09vs98" + }, + "source": [ + "### Create Publisher Subprocess\n", + "The publisher simulates real-time product updates by:\n", + "- Publishing sample product data to the PubSub topic every 5 seconds\n", + "- Modifying prices and descriptions to represent changes\n", + "- Adding timestamps to track update times\n", + "- Running for 25 minutes in the background while our pipeline processes the data" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "cellView": "form", + "id": "C9Bf0Nb0vY7r" + }, + "outputs": [], + "source": [ + "#@title Define PubSub publisher function\n", + "import threading\n", + "import time\n", + "import json\n", + "import logging\n", + "from google.cloud import pubsub_v1\n", + "import datetime\n", + "import os\n", + "import sys\n", + "log_file = os.path.join(os.getcwd(), \"publisher_log.txt\")\n", + "\n", + "print(f\"Log file will be created at: {log_file}\")\n", + "\n", + "def publisher_function(project_id, topic):\n", + " \"\"\"Function that publishes sample product updates to a PubSub topic.\n", + "\n", + " This function runs in a separate thread and continuously publishes\n", + " messages to simulate real-time product updates.\n", + " \"\"\"\n", + " time.sleep(300)\n", + " thread_id = threading.current_thread().ident\n", + "\n", + " process_log_file = os.path.join(os.getcwd(), f\"publisher_{thread_id}.log\")\n", + "\n", + " file_handler = logging.FileHandler(process_log_file)\n", + " file_handler.setFormatter(logging.Formatter('%(asctime)s - ThreadID:%(thread)d - %(levelname)s - %(message)s'))\n", + "\n", + " logger = logging.getLogger(f\"worker.{thread_id}\")\n", + " logger.setLevel(logging.INFO)\n", + " logger.addHandler(file_handler)\n", + "\n", + " logger.info(f\"Publisher thread started with ID: {thread_id}\")\n", + " file_handler.flush()\n", + "\n", + " publisher = pubsub_v1.PublisherClient()\n", + " topic_path = publisher.topic_path(project_id, topic)\n", + "\n", + " logger.info(\"Starting to publish messages...\")\n", + " file_handler.flush()\n", + " for i in range(300):\n", + " message_index = i % len(PRODUCTS_DATA)\n", + " message = PRODUCTS_DATA[message_index].copy()\n", + "\n", + "\n", + " dynamic_factor = 1.05 + (0.1 * ((i % 20) / 20))\n", + " message[\"price\"] = round(message[\"price\"] * dynamic_factor, 2)\n", + " message[\"description\"] = f\"PRICE UPDATE (factor: {dynamic_factor:.3f}): \" + message[\"description\"]\n", + "\n", + " message[\"published_at\"] = datetime.datetime.now().isoformat()\n", + "\n", + " data = json.dumps(message).encode('utf-8')\n", + " publish_future = publisher.publish(topic_path, data)\n", + "\n", + " try:\n", + " logger.info(f\"Publishing message {message}\")\n", + " file_handler.flush()\n", + " message_id = publish_future.result()\n", + " logger.info(f\"Published message {i+1}: {message['id']} (Message ID: {message_id})\")\n", + " file_handler.flush()\n", + " except Exception as e:\n", + " logger.error(f\"Error publishing message: {e}\")\n", + " file_handler.flush()\n", + "\n", + " time.sleep(5)\n", + "\n", + " logger.info(\"Finished publishing all messages.\")\n", + " file_handler.flush()" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "jnUSynmjEmVr" + }, + "source": [ + "#### Start publishing to PuBSub in background" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "ZnBBTwZHw7Ex" + }, + "outputs": [], + "source": [ + "# Launch publisher in a separate thread\n", + "print(\"Starting publisher thread in 5 minutes...\")\n", + "publisher_thread = threading.Thread(\n", + " target=publisher_function,\n", + " args=(PROJECT_ID, TOPIC),\n", + " daemon=True\n", + ")\n", + "publisher_thread.start()\n", + "print(f\"Publisher thread started with ID: {publisher_thread.ident}\")\n", + "print(f\"Publisher thread logging to file: publisher_{publisher_thread.ident}.log\")" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "vGToqM9GoKOV" + }, + "source": [ + "### Run Pipeline on Dataflow\n", + "\n", + "We launch the pipeline to run remotely on Dataflow. Once the job is launched, you can monitor its progress in the Google Cloud Console:\n", + "1. Go to https://console.cloud.google.com/dataflow/jobs\n", + "2. Select your project\n", + "3. Click on the job named \"cloudsql-streaming-embedding-ingest\"\n", + "4. View detailed execution graphs, logs, and metrics\n", + "\n", + "**Note**: This streaming pipeline runs indefinitely until manually stopped. Be sure to monitor usage and terminate the job in the [dataflow job console](https://console.cloud.google.com/dataflow/jobs) when finished testing to avoid unnecessary costs.\n", + "\n", + "### What to Expect\n", + "After running this pipeline, you should see:\n", + "- Continuous updates to product embeddings in the CloudSQL MySQL table\n", + "- Price and description changes reflected in the metadata\n", + "- New embeddings generated for updated product descriptions\n", + "- Timestamps showing when each record was last modified" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "NTibYI9rx46o" + }, + "outputs": [], + "source": [ + "# Run pipeline\n", + "pipeline.run().wait_until_finish()" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "vX9VxJ82CTum" + }, + "source": [ + "### Verify data" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "zSb1UoCSznkW" + }, + "outputs": [], + "source": [ + "# Verify the results\n", + "print(\"\\nAfter embedding generation:\")\n", + "verify_embeddings_sqlalchemy(connection_name=CONNECTION_NAME, database=DB_NAME, table_name=table_name, user=DB_USER, password=DB_PASSWORD)" + ] + } + ], + "metadata": { + "colab": { + "collapsed_sections": [ + "mcZATJbaOec0" + ], + "provenance": [] + }, + "kernelspec": { + "display_name": "Python 3", + "name": "python3" + }, + "language_info": { + "name": "python" + } + }, + "nbformat": 4, + "nbformat_minor": 0 +} diff --git a/examples/notebooks/beam-ml/cloudsql_postgres_product_catalog_embeddings.ipynb b/examples/notebooks/beam-ml/cloudsql_postgres_product_catalog_embeddings.ipynb index 6d3b622d06f4..eccfc405e694 100644 --- a/examples/notebooks/beam-ml/cloudsql_postgres_product_catalog_embeddings.ipynb +++ b/examples/notebooks/beam-ml/cloudsql_postgres_product_catalog_embeddings.ipynb @@ -135,7 +135,7 @@ "outputs": [], "source": [ "# Apache Beam with GCP support\n", - "!pip install apache_beam[gcp]>=2.66.0 --quiet\n", + "!pip install apache_beam[interactive,gcp]>=2.66.0 --quiet\n", "# Huggingface sentence-transformers for embedding models\n", "!pip install sentence-transformers --quiet" ] diff --git a/examples/notebooks/beam-ml/data_preprocessing/compute_and_apply_vocab.ipynb b/examples/notebooks/beam-ml/data_preprocessing/compute_and_apply_vocab.ipynb index ecd4ded6e70c..779ec99903f5 100644 --- a/examples/notebooks/beam-ml/data_preprocessing/compute_and_apply_vocab.ipynb +++ b/examples/notebooks/beam-ml/data_preprocessing/compute_and_apply_vocab.ipynb @@ -98,7 +98,7 @@ }, "outputs": [], "source": [ - "! pip install apache_beam>=2.53.0 --quiet\n", + "! pip install apache_beam[interactive]>=2.53.0 --quiet\n", "! pip install tensorflow-transform --quiet" ] }, diff --git a/examples/notebooks/beam-ml/data_preprocessing/huggingface_text_embeddings.ipynb b/examples/notebooks/beam-ml/data_preprocessing/huggingface_text_embeddings.ipynb index d2acaa2e9045..da445dd675b7 100644 --- a/examples/notebooks/beam-ml/data_preprocessing/huggingface_text_embeddings.ipynb +++ b/examples/notebooks/beam-ml/data_preprocessing/huggingface_text_embeddings.ipynb @@ -45,7 +45,7 @@ { "cell_type": "markdown", "source": [ - "# Generate text embeddings by using Hugging Face Hub models\n", + "# Generate text embeddings by using the EmbeddingGemma model from Hugging Face\n", "\n", "\n", "
\n", @@ -75,6 +75,8 @@ "\n", "This notebook uses Apache Beam's `MLTransform` to generate embeddings from text data.\n", "\n", + "Using a small, highly efficient open model like EmbeddingGemma at the core of your pipeline makes the entire process self-contained, which can simplify management by eliminating the need for external network calls to other services for the embedding step. Because it's an open model, it can be hosted entirely within Dataflow. This provides the confidence to securely process large-scale, private datasets. For more information about the model, see the [model card](https://huggingface.co/google/embeddinggemma-300m)\n", + "\n", "Hugging Face's [`SentenceTransformers`](https://huggingface.co/sentence-transformers) framework uses Python to generate sentence, text, and image embeddings.\n", "\n", "To generate text embeddings that use Hugging Face models and `MLTransform`, use the `SentenceTransformerEmbeddings` module to specify the model configuration.\n" @@ -97,7 +99,7 @@ { "cell_type": "code", "source": [ - "! pip install apache_beam>=2.53.0 --quiet\n", + "! pip install apache_beam[interactive]>=2.53.0 --quiet\n", "! pip install sentence-transformers --quiet" ], "metadata": { @@ -120,6 +122,28 @@ "execution_count": 29, "outputs": [] }, + { + "cell_type": "markdown", + "source": [ + "### Authenticate with HuggingFace\n", + "\n", + "To ensure that you can pull the correct model, authenticate with HuggingFace by following the prompts in the cell." + ], + "metadata": { + "id": "kXDM8C7d3nPW" + } + }, + { + "cell_type": "code", + "source": [ + "!hf auth login" + ], + "metadata": { + "id": "jVxSi2jS3M3c" + }, + "execution_count": 29, + "outputs": [] + }, { "cell_type": "markdown", "source": [ @@ -170,7 +194,7 @@ " {'x': \"Should I sign up for Medicare Part B if I have Veterans' Benefits?\"}\n", "]\n", "\n", - "text_embedding_model_name = 'sentence-transformers/sentence-t5-large'\n", + "text_embedding_model_name = 'google/embeddinggemma-300m'\n", "\n", "\n", "# helper function that returns a dict containing only first\n", @@ -191,7 +215,7 @@ "source": [ "\n", "### Generate text embeddings\n", - "This example uses the model `sentence-transformers/sentence-t5-large` to generate text embeddings. The model uses only the encoder from a `T5-large model`. The weights are stored in FP16. For more information about the model, see [Sentence-T5: Scalable Sentence Encoders from Pre-trained Text-to-Text Models](https://arxiv.org/abs/2108.08877)." + "This example uses the model `google/embeddinggemma-300m` to generate text embeddings. For more information about the model, see [the model card](https://huggingface.co/google/embeddinggemma-300m)." ], "metadata": { "id": "SApMmlRLRv_e" diff --git a/examples/notebooks/beam-ml/data_preprocessing/scale_data.ipynb b/examples/notebooks/beam-ml/data_preprocessing/scale_data.ipynb index ba367fbc8177..ddf5a0c5c7e4 100644 --- a/examples/notebooks/beam-ml/data_preprocessing/scale_data.ipynb +++ b/examples/notebooks/beam-ml/data_preprocessing/scale_data.ipynb @@ -104,7 +104,7 @@ { "cell_type": "code", "source": [ - "! pip install apache_beam>=2.53.0 --quiet\n", + "! pip install apache_beam[interactive]>=2.53.0 --quiet\n", "! pip install tensorflow-transform --quiet" ], "metadata": { diff --git a/examples/notebooks/beam-ml/data_preprocessing/vertex_ai_text_embeddings.ipynb b/examples/notebooks/beam-ml/data_preprocessing/vertex_ai_text_embeddings.ipynb index 4d816ef97fb0..2d8cca4e44a0 100644 --- a/examples/notebooks/beam-ml/data_preprocessing/vertex_ai_text_embeddings.ipynb +++ b/examples/notebooks/beam-ml/data_preprocessing/vertex_ai_text_embeddings.ipynb @@ -117,7 +117,7 @@ }, "outputs": [], "source": [ - "! pip install apache_beam[gcp]>=2.53.0 --quiet" + "! pip install apache_beam[interactive,gcp]>=2.53.0 --quiet" ] }, { diff --git a/examples/notebooks/beam-ml/dataflow_tpu_examples.ipynb b/examples/notebooks/beam-ml/dataflow_tpu_examples.ipynb new file mode 100644 index 000000000000..f48327b660dc --- /dev/null +++ b/examples/notebooks/beam-ml/dataflow_tpu_examples.ipynb @@ -0,0 +1,744 @@ +{ + "cells": [ + { + "cell_type": "code", + "source": [ + "# Licensed to the Apache Software Foundation (ASF) under one\n", + "# or more contributor license agreements. See the NOTICE file\n", + "# distributed with this work for additional information\n", + "# regarding copyright ownership. The ASF licenses this file\n", + "# to you under the Apache License, Version 2.0 (the\n", + "# \"License\"); you may not use this file except in compliance\n", + "# with the License. You may obtain a copy of the License at\n", + "#\n", + "# http://www.apache.org/licenses/LICENSE-2.0\n", + "#\n", + "# Unless required by applicable law or agreed to in writing,\n", + "# software distributed under the License is distributed on an\n", + "# \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY\n", + "# KIND, either express or implied. See the License for the\n", + "# specific language governing permissions and limitations\n", + "# under the License" + ], + "metadata": { + "id": "H-YbtpqChYYo" + }, + "execution_count": null, + "outputs": [] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "a5343d14" + }, + "source": [ + "# Running Dataflow on TPUs: Quickstart examples\n", + "\n", + "\n", + " \n", + " \n", + "
\n", + " Run in Google Colab\n", + " \n", + " View source on GitHub\n", + "
\n", + "
\n", + "
" + ] + }, + { + "cell_type": "markdown", + "source": [ + "This Colab notebook shows you how to set up two pipelines:\n", + "1. A pipeline that runs a trivial computation on a TPU.\n", + "2. A pipeline that runs inference using the [Gemma-3-27b-it model](https://huggingface.co/google/gemma-3-27b-it) on TPUs .\n", + "\n", + "Both pipelines use a custom Docker image. The Dataflow jobs will launch using a [Flex Template](https://cloud.google.com/dataflow/docs/guides/templates/using-flex-templates) to allow the same job to be reproduced in different Colab environments." + ], + "metadata": { + "id": "hAm4UpVHimSr" + } + }, + { + "cell_type": "markdown", + "metadata": { + "id": "8L0c_bikJt4d" + }, + "source": [ + "## Prerequisites" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "i5IAopB4ewpu" + }, + "source": [ + "First, you need to authenticate to your Google Cloud Project. After running the cell below, you might need to **click on the text prompts in the cell** and enter inputs as prompted.\n" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "OdZ5bkvwesGg" + }, + "outputs": [], + "source": [ + "from google.colab import auth\n", + "auth.authenticate_user()\n", + "!gcloud auth login" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "dEFtSATYJp6p" + }, + "source": [ + "Now, set environment variables to access pipeline resources, such as a\n", + "Cloud Storage bucket or a repository to host container images in Artifact Registry." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "collapsed": true, + "id": "cMJS0sYBfNkI" + }, + "outputs": [], + "source": [ + "import os\n", + "import datetime\n", + "\n", + "project_id = \"some-project\" # @param {type:\"string\"}\n", + "gcs_bucket = \"some-bucket\" # @param {type:\"string\"}\n", + "ar_repository = \"some-ar-repo\" # @param {type:\"string\"}\n", + "\n", + "# Use a region where you have TPU accelerator quota.\n", + "region = \"some-region1\" # @param {type:\"string\"}\n", + "!gcloud config set project {project_id}" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "vIrXayHQL-d6" + }, + "source": [ + "Enable the necessary APIs if your project hasn't enabled them yet. If you have the appropriate permissions, you can enable the APIs by running the following cell." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "_jKxVSK_MBFr" + }, + "outputs": [], + "source": [ + "!gcloud services enable \\\n", + " dataflow.googleapis.com \\\n", + " compute.googleapis.com \\\n", + " logging.googleapis.com \\\n", + " storage.googleapis.com \\\n", + " cloudresourcemanager.googleapis.com \\\n", + " artifactregistry.googleapis.com \\\n", + " cloudbuild.googleapis.com" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "lS3V0Sh5MbtT" + }, + "source": [ + "Now, you'll create a Cloud Storage bucket and Artifact Registry repository if you don't already have these resources." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "8Wrs8yUhMas7" + }, + "outputs": [], + "source": [ + "!gcloud storage buckets describe gs://{gcs_bucket} >/dev/null 2>&1 || gcloud storage buckets create gs://{gcs_bucket} --location={region}\n", + "!gcloud artifacts repositories describe {ar_repository} --location={region} >/dev/null 2>&1 || gcloud artifacts repositories create {ar_repository} --repository-format=docker --location={region}" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "Uv12ZxPVcTEc" + }, + "source": [ + "# Example 1: Minimal computation pipeline using TPU V5E\n", + "\n", + "First, create a simple pipeline you can run to verify that TPUs are accessible, your custom Docker image has the necessary dependencies to interact with the TPUs and your Dataflow pipeline launch configuration is valid.\n", + "\n", + "With this sample you use the PyTorch library to interact with a TPU device." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "31f4cabb" + }, + "outputs": [], + "source": [ + "%%writefile minimal_tpu_pipeline.py\n", + "from __future__ import annotations\n", + "import torch\n", + "import torch_xla\n", + "import argparse\n", + "import logging\n", + "import apache_beam as beam\n", + "from apache_beam.options.pipeline_options import PipelineOptions\n", + "\n", + "\n", + "class check_tpus(beam.DoFn):\n", + " \"\"\"Validates that a TPU is accessible.\"\"\"\n", + " def setup(self):\n", + " tpu_devices = torch_xla.xm.get_xla_supported_devices()\n", + " if not tpu_devices:\n", + " raise RuntimeError(\"No TPUs found on the worker.\")\n", + " logging.info(f\"Found TPU devices: {tpu_devices}\")\n", + " tpu = torch_xla.device()\n", + " t1 = torch.randn(3, 3, device=tpu)\n", + " t2 = torch.randn(3, 3, device=tpu)\n", + " result = t1 + t2\n", + " logging.info(f\"Result of a sample TPU computation: {result}\")\n", + "\n", + " def process(self, element):\n", + " yield element\n", + "\n", + "\n", + "def run(input_text: str, beam_args: list[str] | None = None) -> None:\n", + " beam_options = PipelineOptions(beam_args, save_main_session=True)\n", + " pipeline = beam.Pipeline(options=beam_options)\n", + " (\n", + " pipeline\n", + " | \"Create data\" >> beam.Create([input_text])\n", + " | \"Check TPU availability\" >> beam.ParDo(check_tpus())\n", + " | \"My transform\" >> beam.LogElements(level=logging.INFO)\n", + " )\n", + " pipeline.run()\n", + "\n", + "\n", + "if __name__ == \"__main__\":\n", + " logging.getLogger().setLevel(logging.INFO)\n", + "\n", + " parser = argparse.ArgumentParser()\n", + " parser.add_argument(\n", + " \"--input-text\",\n", + " default=\"Hello! This pipeline verified that TPUs are accessible.\",\n", + " help=\"Input text to display.\",\n", + " )\n", + " args, beam_args = parser.parse_known_args()\n", + "\n", + " run(args.input_text, beam_args)" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "4516f3e0" + }, + "source": [ + "## Create a Dockerfile for your TPU-compatible container image.\n", + "\n", + "In your Dockerfile you configure the environment variables to use with a `V5E` `1x1` TPU device.\n", + "\n", + "**You must use the region where you have V5E TPU quota to run this example.**\n", + "\n", + "To use a different TPU, adjust the configuration according to the [Dataflow documentation](https://cloud.google.com/dataflow/docs/tpu/use-tpus).\n", + "\n", + "This Dockerfile creates an image that serves both as a custom worker image for your Beam pipeline and also as a launcher image for your Flex template." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "_EY1_rmXdAM5" + }, + "outputs": [], + "source": [ + "%%writefile Dockerfile\n", + "\n", + "FROM python:3.11-slim\n", + "\n", + "COPY minimal_tpu_pipeline.py minimal_tpu_pipeline.py\n", + "\n", + "# Copy the Apache Beam worker dependencies from the Beam Python 3.10 SDK image.\n", + "COPY --from=apache/beam_python3.10_sdk:2.67.0 /opt/apache/beam /opt/apache/beam\n", + "\n", + "# Copy Template Launcher dependencies\n", + "COPY --from=gcr.io/dataflow-templates-base/python310-template-launcher-base /opt/google/dataflow/python_template_launcher /opt/google/dataflow/python_template_launcher\n", + "\n", + "# Install TPU software and Apache Beam SDK\n", + "RUN pip install --no-cache-dir torch~=2.8.0 torch_xla[tpu]~=2.8.0 apache-beam[gcp]==2.67.0 -f https://storage.googleapis.com/libtpu-releases/index.html\n", + "\n", + "# Configuration for v5e 1x1 accelerator type.\n", + "ENV TPU_CHIPS_PER_HOST_BOUNDS=1,1,1\n", + "ENV TPU_ACCELERATOR_TYPE=v5litepod-1\n", + "ENV TPU_SKIP_MDS_QUERY=1\n", + "ENV TPU_HOST_BOUNDS=1,1,1\n", + "ENV TPU_WORKER_HOSTNAMES=localhost\n", + "ENV TPU_WORKER_ID=0\n", + "\n", + "ENV FLEX_TEMPLATE_PYTHON_PY_FILE=minimal_tpu_pipeline.py\n", + "\n", + "# Set the entrypoint to Apache Beam SDK worker launcher.\n", + "ENTRYPOINT [ \"/opt/apache/beam/boot\"]" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "XBFIEqNmenRj" + }, + "source": [ + "## Push your Docker image to Artifact Registry." + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "F9XQBZfrfbM2" + }, + "source": [ + "Finally, build your Docker image, and push it in Artifact Registry. This process should take about 15 minutes or so." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "UaA-sBC1fabY" + }, + "outputs": [], + "source": [ + "container_tag = \"20250801\"\n", + "container_image = ''.join([\n", + " region, \"-docker.pkg.dev/\",\n", + " project_id, \"/\",\n", + " ar_repository, \"/\",\n", + " \"tpu-minimal-example\", \":\", container_tag\n", + "])\n", + "\n", + "!gcloud builds submit --tag {container_image}" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "1chqESwuSerP" + }, + "source": [ + "## Build the Dataflow Flex Template." + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "I3ukh2lwlmm3" + }, + "source": [ + "To create a reproducible environment for launching the pipeline, build a Flex Template.\n", + "\n", + "First, create a `metadata.json` file to change the default Dataflow worker disk size when launching the template.\n" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "GhlCMBnDl8-t" + }, + "outputs": [], + "source": [ + "%%writefile metadata.json\n", + "{\n", + " \"name\": \"Minimal TPU Example on Dataflow\",\n", + " \"description\": \"A Flex template launching a Dataflow Job doing a TPU computation \",\n", + " \"parameters\": [\n", + " {\n", + " \"name\": \"disk_size_gb\",\n", + " \"label\": \"disk_size_gb\",\n", + " \"helpText\": \"disk_size_gb for worker\",\n", + " \"isOptional\": true\n", + " }\n", + " ]\n", + "}" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "eQAX8rzJVtDS" + }, + "source": [ + "Run the following cell to build the Flex Template and save it Cloud Storage." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "CYLTC-jpSh6j" + }, + "outputs": [], + "source": [ + "!gcloud dataflow flex-template build gs://{gcs_bucket}/minimal_tpu_pipeline.json \\\n", + " --image {container_image} \\\n", + " --sdk-language \"PYTHON\" \\\n", + " --metadata-file metadata.json \\\n", + " --project {project_id}" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "cAhW0FdW5W7t" + }, + "source": [ + "## Submit your pipeline to Dataflow.\n", + "\n", + "Since you launch the pipeline as a Flex Template, make the following adjustments to the command line:\n", + "\n", + "* Use `--parameters` option to specify the container image and disk size.\n", + "* Use `--additional-experiments` option to specify the necessary Dataflow service options.\n", + "* To avoid using more than one process on a TPU simultaneously, limit process-level parallelism with the `no_use_multiple_sdk_containers` experiment." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "UVtBBPcWCzFu" + }, + "outputs": [], + "source": [ + "!gcloud dataflow flex-template run \"minimal-tpu-example-`date +%Y%m%d-%H%M%S`\" \\\n", + " --template-file-gcs-location gs://{gcs_bucket}/minimal_tpu_pipeline.json \\\n", + " --region {region} \\\n", + " --project {project_id} \\\n", + " --temp-location gs://{gcs_bucket}/tmp \\\n", + " --parameters sdk_container_image={container_image} \\\n", + " --worker-machine-type \"ct5lp-hightpu-1t\" \\\n", + " --parameters disk_size_gb=50 \\\n", + " --additional-experiments \"worker_accelerator=type:tpu-v5-lite-podslice;topology:1x1\" \\\n", + " --additional-experiments \"no_use_multiple_sdk_containers\"\n" + ] + }, + { + "cell_type": "markdown", + "source": [ + "Once the job is launched, use the following link to monitor its status: https://console.cloud.google.com/dataflow/jobs/\n", + "\n", + "Sample worker logs for the `Check TPU availability` step look like the following:\n", + "\n", + "```\n", + "Found TPU devices: ['xla:0']\n", + "Result of a sample TPU computation: tensor([[ 0.3355, -1.4628, -3.2610], [-1.4656, 0.3196, -2.8766], [ 0.8667, -1.5060, 0.7125]], device='xla:0')\n", + "```" + ], + "metadata": { + "id": "xRW_d_i_tVel" + } + }, + { + "cell_type": "markdown", + "metadata": { + "id": "DpUAUjDlcMOR" + }, + "source": [ + "# Example 2: Inference Pipeline with Gemma 3 27B using TPU V6E\n", + "\n", + "This example shows you how to perform inference on a TPU using Gemma 3 27b model.\n", + "\n", + "To fit this model in TPU memory, you need four V6E TPU chips connected in 2x2 topology.\n", + "\n", + "**You must use the region where you have V6E TPU quota to run this example.**\n", + "\n", + "The example uses [Apache Beam RunInference APIs](https://beam.apache.org/documentation/transforms/python/elementwise/runinference/) with the [VLLM Completions model handler](https://beam.apache.org/releases/pydoc/current/apache_beam.ml.inference.vllm_inference.html).\n", + "\n", + "The model is downloaded from HuggingFace at runtime, and running the example requires a [HuggingFace access token](https://huggingface.co/docs/hub/en/security-tokens).\n", + "\n", + "First, create a pipeline file." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "GGCqkzgXda97" + }, + "outputs": [], + "source": [ + "%%writefile gemma_tpu_pipeline.py\n", + "from __future__ import annotations\n", + "import argparse\n", + "import logging\n", + "import apache_beam as beam\n", + "from apache_beam.ml.inference.base import RunInference\n", + "from apache_beam.options.pipeline_options import PipelineOptions\n", + "from apache_beam.ml.inference.vllm_inference import VLLMCompletionsModelHandler\n", + "\n", + "\n", + "def run(input_text: str, beam_args: list[str] | None = None) -> None:\n", + " beam_options = PipelineOptions(beam_args, save_main_session=True)\n", + " pipeline = beam.Pipeline(options=beam_options)\n", + " (\n", + " pipeline\n", + " | \"Create data\" >> beam.Create([input_text])\n", + " | \"Run Inference\" >> RunInference(\n", + " model_handler=VLLMCompletionsModelHandler(\n", + " 'google/gemma-3-27b-it',\n", + " {\n", + " 'max-model-len': '4096',\n", + " 'no-enable-prefix-caching': None,\n", + " 'disable-log-requests': None,\n", + " 'tensor-parallel-size': '4',\n", + " 'limit-mm-per-prompt': '{\"image\": 0}'\n", + " })\n", + " )\n", + " | \"Log Output\" >> beam.LogElements(level=logging.INFO)\n", + " )\n", + " pipeline.run()\n", + "\n", + "\n", + "if __name__ == \"__main__\":\n", + " logging.getLogger().setLevel(logging.INFO)\n", + " parser = argparse.ArgumentParser()\n", + " parser.add_argument(\n", + " \"--input-text\",\n", + " default=\"What are TPUs?\",\n", + " help=\"Input text query.\",\n", + " )\n", + " args, beam_args = parser.parse_known_args()\n", + " run(args.input_text, beam_args)" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "emTd69gUonq-" + }, + "source": [ + "## Create a new Dockerfile for this pipeline with additional dependencies.\n", + "Note that this sample uses a different TPU device than the example 1, so the environment variables are different.\n", + "\n", + "**You must use your own HuggingFace Token in the Dockerfile.** For instructions on creating a token, see [User access tokens](https://huggingface.co/docs/hub/en/security-tokens)." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "6elKvBZ0_dc4" + }, + "outputs": [], + "source": [ + "%%writefile Dockerfile\n", + "# Use the official vLLM TPU base image, which has TPU dependencies.\n", + "# To use the latest version, use: vllm/vllm-tpu:nightly\n", + "FROM vllm/vllm-tpu:5964069367a7d54c3816ce3faba79e02110cde17\n", + "\n", + "# Copy your pipeline file.\n", + "COPY gemma_tpu_pipeline.py gemma_tpu_pipeline.py\n", + "\n", + "# You can use a more recent version of Apache Beam\n", + "COPY --from=apache/beam_python3.12_sdk:2.67.0 /opt/apache/beam /opt/apache/beam\n", + "RUN pip install --no-cache-dir apache-beam[gcp]==2.67.0\n", + "\n", + "# Copy Template Launcher dependencies\n", + "COPY --from=gcr.io/dataflow-templates-base/python310-template-launcher-base /opt/google/dataflow/python_template_launcher /opt/google/dataflow/python_template_launcher\n", + "\n", + "# Replace the Hugginface token here.\n", + "RUN python -c 'from huggingface_hub import HfFolder; HfFolder.save_token(\"YOUR HUGGINGFACE TOKEN\")'\n", + "\n", + "# TPU environment variables.\n", + "ENV TPU_SKIP_MDS_QUERY=1\n", + "\n", + "# Configuration for v6e 2x2 accelerator type.\n", + "ENV TPU_HOST_BOUNDS=1,1,1\n", + "ENV TPU_CHIPS_PER_HOST_BOUNDS=2,2,1\n", + "ENV TPU_ACCELERATOR_TYPE=v6e-4\n", + "ENV VLLM_USE_V1=1\n", + "\n", + "ENV FLEX_TEMPLATE_PYTHON_PY_FILE=gemma_tpu_pipeline.py\n", + "\n", + "# Set the entrypoint to Apache Beam SDK worker launcher.\n", + "ENTRYPOINT [ \"/opt/apache/beam/boot\"]" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "2V1PmAf1otG4" + }, + "source": [ + "Run the following cell to build the Docker image and push it to Artifact Registry. This process should take 15 min or so." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "collapsed": true, + "id": "He5WkUAE_pYp" + }, + "outputs": [], + "source": [ + "container_tag = \"20250801\"\n", + "container_image = ''.join([\n", + " region, \"-docker.pkg.dev/\",\n", + " project_id, \"/\",\n", + " ar_repository, \"/\",\n", + " \"tpu-run-inference-example\", \":\", container_tag\n", + "])\n", + "!gcloud builds submit --tag {container_image}" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "EyYuSgDudcVK" + }, + "source": [ + "## Build the Flex Template for this pipeline." + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "33V96JFAL_jk" + }, + "source": [ + "To create a reproducible environment for launching the pipeline, build a Flex Template.\n", + "\n", + "First, create a `metadata.json` file to change the default Dataflow worker disk size when launching the template." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "L8hylI64L_jl" + }, + "outputs": [], + "source": [ + "%%writefile metadata.json\n", + "{\n", + " \"name\": \"Gemma 3 27b Run Inference pipeline with VLLM\",\n", + " \"description\": \"A template for Dataflow RunInference pipline with VLLM in a TPU-enabled environment with VLLM\",\n", + " \"parameters\": [\n", + " {\n", + " \"name\": \"disk_size_gb\",\n", + " \"label\": \"disk_size_gb\",\n", + " \"helpText\": \"disk_size_gb for worker\",\n", + " \"isOptional\": true\n", + " }\n", + " ]\n", + "}" + ] + }, + { + "cell_type": "markdown", + "source": [ + "Run the following cell to build the Flex Template and save it in Cloud Storage." + ], + "metadata": { + "id": "yGRhrD1J2IIW" + } + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "Hvs2JWNydiBl" + }, + "outputs": [], + "source": [ + "!gcloud dataflow flex-template build gs://{gcs_bucket}/gemma_tpu_pipeline.json \\\n", + " --image {container_image} \\\n", + " --sdk-language \"PYTHON\" \\\n", + " --metadata-file metadata.json \\\n", + " --project {project_id}" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "id": "VWWaf4cmdi7z" + }, + "source": [ + "## Finally, submit the job to Dataflow.\n", + "\n", + "Since you launch the pipeline as a Flex Template, you are making the following adjustments to the command line:\n", + "\n", + "* Use the `--parameters` option to specify the container image and disk size\n", + "* Use the `--additional-experiments` option to specify the necessary Dataflow service options.\n", + "* The VLLMCompletionsModelHandler from Beam RunInference APIs only loads the model onto TPUs from a single process. Still, limit the intra-worker parallelism by reducing the value of\n", + "`--number_of_worker_harness_threads`, which achieves better performance.\n", + "\n", + "Once the job is launched, use the following link to monitor its status: https://console.cloud.google.com/dataflow/jobs/" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "id": "OUX0E0XzdlLW" + }, + "outputs": [], + "source": [ + "!gcloud dataflow flex-template run \"gemma-tpu-example-`date +%Y%m%d-%H%M%S`\" \\\n", + " --template-file-gcs-location gs://{gcs_bucket}/gemma_tpu_pipeline.json \\\n", + " --region {region} \\\n", + " --project {project_id} \\\n", + " --temp-location gs://{gcs_bucket}/tmp \\\n", + " --parameters number_of_worker_harness_threads=100 \\\n", + " --parameters sdk_container_image={container_image} \\\n", + " --parameters disk_size_gb=100 \\\n", + " --worker-machine-type \"ct6e-standard-4t\" \\\n", + " --additional-experiments \"worker_accelerator=type:tpu-v6e-slice;topology:2x2\"" + ] + }, + { + "cell_type": "markdown", + "source": [ + "Due to model loading and initialization time, the pipeline takes 25 min or so to complete.\n", + "\n", + "Sample worker logs for the `Run Inference` step look like the following:\n", + "\n", + "```\n", + "PredictionResult(example='What are TPUs?', inference=Completion(id='cmpl-57ebbddeb1c04dc0a8a74f2b60d10f67', choices=[CompletionChoice(finish_reason='length', index=0, logprobs=None, text='\\n\\nTensor Processing Units (TPUs) are custom-developed AI accelerator ASICs', stop_reason=None, prompt_logprobs=None)], created=1755614936, model='google/gemma-3-27b-it', object='text_completion', system_fingerprint=None, usage=CompletionUsage(completion_tokens=16, prompt_tokens=6, total_tokens=22, completion_tokens_details=None, prompt_tokens_details=None), service_tier=None, kv_transfer_params=None), model_id=None)\n", + "```" + ], + "metadata": { + "id": "1kpeVbdczt8u" + } + } + ], + "metadata": { + "colab": { + "provenance": [] + }, + "kernelspec": { + "display_name": "Python 3", + "name": "python3" + }, + "language_info": { + "name": "python" + } + }, + "nbformat": 4, + "nbformat_minor": 0 +} \ No newline at end of file diff --git a/examples/notebooks/beam-ml/gemma_2_sentiment_and_summarization.ipynb b/examples/notebooks/beam-ml/gemma_2_sentiment_and_summarization.ipynb index 160c09f563b0..ad56b674ee5d 100644 --- a/examples/notebooks/beam-ml/gemma_2_sentiment_and_summarization.ipynb +++ b/examples/notebooks/beam-ml/gemma_2_sentiment_and_summarization.ipynb @@ -206,7 +206,7 @@ }, "source": [ "```sh\n", - "apache_beam[gcp]==2.54.0\n", + "apache_beam[interactive,gcp]==2.54.0\n", "keras_nlp==0.14.3\n", "keras==3.4.1\n", "jax[cuda12]\n", @@ -293,7 +293,7 @@ }, "outputs": [{"output_type": "stream", "name": "stdout", "text": ["\n"]}], "source": [ - "%pip install apache_beam[gcp]==\"2.54.0\" keras_nlp==\"0.14.3\" keras==\"3.5.0\" jax[cuda12]" + "%pip install apache_beam[interactive,gcp]==\"2.54.0\" keras_nlp==\"0.14.3\" keras==\"3.5.0\" jax[cuda12]" ] }, { diff --git a/examples/notebooks/beam-ml/image_processing_tensorflow.ipynb b/examples/notebooks/beam-ml/image_processing_tensorflow.ipynb index 45fca09addbb..c2e41c1f0cf5 100644 --- a/examples/notebooks/beam-ml/image_processing_tensorflow.ipynb +++ b/examples/notebooks/beam-ml/image_processing_tensorflow.ipynb @@ -87,7 +87,6 @@ }, "outputs": [], "source": [ - "!pip install apache_beam --quiet\n", "!pip install apache-beam[interactive] --quiet" ] }, @@ -915,4 +914,4 @@ }, "nbformat": 4, "nbformat_minor": 0 -} \ No newline at end of file +} diff --git a/examples/notebooks/beam-ml/mltransform_basic.ipynb b/examples/notebooks/beam-ml/mltransform_basic.ipynb index b0af96d08593..470f100537e8 100644 --- a/examples/notebooks/beam-ml/mltransform_basic.ipynb +++ b/examples/notebooks/beam-ml/mltransform_basic.ipynb @@ -76,7 +76,7 @@ "cell_type": "code", "source": [ "!pip install tensorflow_transform --quiet\n", - "!pip install apache_beam>=2.50.0 --quiet" + "!pip install apache_beam[interactive]>=2.50.0 --quiet" ], "metadata": { "id": "MRWkC-n2DmjM" diff --git a/examples/notebooks/beam-ml/per_key_models.ipynb b/examples/notebooks/beam-ml/per_key_models.ipynb index 3e71c1d119a2..026a481dd2c4 100644 --- a/examples/notebooks/beam-ml/per_key_models.ipynb +++ b/examples/notebooks/beam-ml/per_key_models.ipynb @@ -107,7 +107,7 @@ } ], "source": [ - "!pip install apache_beam[gcp]>=2.51.0 --quiet\n", + "!pip install apache_beam[interactive,gcp]>=2.51.0 --quiet\n", "!pip install torch --quiet\n", "!pip install transformers --quiet\n", "\n", diff --git a/examples/notebooks/beam-ml/rag_usecase/beam_rag_notebook.ipynb b/examples/notebooks/beam-ml/rag_usecase/beam_rag_notebook.ipynb index e271074af555..4941f3f3ad63 100644 --- a/examples/notebooks/beam-ml/rag_usecase/beam_rag_notebook.ipynb +++ b/examples/notebooks/beam-ml/rag_usecase/beam_rag_notebook.ipynb @@ -108,7 +108,7 @@ "#installing dependencies\n", "!pip install pandas==1.4.4\n", "!pip install numpy==1.24.4\n", - "!pip install apache_beam==2.56.0\n", + "!pip install apache_beam[interactive]==2.56.0\n", "!pip install redis==5.0.1\n", "!pip install langchain==0.1.14 #used for chunking" ] diff --git a/examples/notebooks/beam-ml/rag_usecase/opensearch_rag_pipeline.ipynb b/examples/notebooks/beam-ml/rag_usecase/opensearch_rag_pipeline.ipynb index aae86e31aa44..f11044426720 100644 --- a/examples/notebooks/beam-ml/rag_usecase/opensearch_rag_pipeline.ipynb +++ b/examples/notebooks/beam-ml/rag_usecase/opensearch_rag_pipeline.ipynb @@ -46,7 +46,7 @@ "#installing dependencies\n", "!pip install pandas==1.4.4\n", "!pip install numpy==1.24.4\n", - "!pip install apache_beam==2.56.0\n", + "!pip install apache_beam[interactive]==2.56.0\n", "!pip install opensearch==2.1.0\n", "#used for chunking\n", "!pip install langchain==0.1.14 " diff --git a/examples/notebooks/beam-ml/run_inference_gemma.ipynb b/examples/notebooks/beam-ml/run_inference_gemma.ipynb index 489f01c4c9aa..12f5a03be109 100644 --- a/examples/notebooks/beam-ml/run_inference_gemma.ipynb +++ b/examples/notebooks/beam-ml/run_inference_gemma.ipynb @@ -130,7 +130,7 @@ ], "source": [ "!pip install -q -U protobuf\n", - "!pip install -q -U apache_beam[gcp]\n", + "!pip install -q -U apache_beam[interactive,gcp]\n", "!pip install -q -U keras_nlp>=0.8.0\n", "!pip install -q -U keras>3\n", "\n", diff --git a/examples/notebooks/beam-ml/run_inference_generative_ai.ipynb b/examples/notebooks/beam-ml/run_inference_generative_ai.ipynb index 40b283982b68..2ca2374abbf3 100644 --- a/examples/notebooks/beam-ml/run_inference_generative_ai.ipynb +++ b/examples/notebooks/beam-ml/run_inference_generative_ai.ipynb @@ -95,7 +95,7 @@ }, "outputs": [], "source": [ - "!pip install apache_beam[gcp]==2.48.0\n", + "!pip install apache_beam[interactive,gcp]==2.48.0\n", "!pip install torch\n", "!pip install transformers" ] diff --git a/examples/notebooks/beam-ml/run_inference_multi_model.ipynb b/examples/notebooks/beam-ml/run_inference_multi_model.ipynb index 7cd144223cae..d6c616a62c56 100644 --- a/examples/notebooks/beam-ml/run_inference_multi_model.ipynb +++ b/examples/notebooks/beam-ml/run_inference_multi_model.ipynb @@ -195,7 +195,7 @@ "!pip install ftfy==6.1.1 --quiet\n", "!pip install spacy==3.4.1 --quiet\n", "!pip install fairscale==0.4.4 --quiet\n", - "!pip install apache_beam[gcp]>=2.48.0\n", + "!pip install apache_beam[interactive,gcp]>=2.48.0\n", "\n", "# To use the newly installed versions, restart the runtime.\n", "exit()" diff --git a/examples/notebooks/beam-ml/run_inference_pytorch.ipynb b/examples/notebooks/beam-ml/run_inference_pytorch.ipynb index 93dd12dd20ab..a10d40e8f997 100644 --- a/examples/notebooks/beam-ml/run_inference_pytorch.ipynb +++ b/examples/notebooks/beam-ml/run_inference_pytorch.ipynb @@ -86,7 +86,7 @@ }, "outputs": [], "source": [ - "!pip install apache_beam[gcp,dataframe] --quiet" + "!pip install apache_beam[interactive,gcp,dataframe] --quiet" ] }, { diff --git a/examples/notebooks/beam-ml/run_inference_pytorch_tensorflow_sklearn.ipynb b/examples/notebooks/beam-ml/run_inference_pytorch_tensorflow_sklearn.ipynb index 115b70b11e94..4167cce47c4c 100644 --- a/examples/notebooks/beam-ml/run_inference_pytorch_tensorflow_sklearn.ipynb +++ b/examples/notebooks/beam-ml/run_inference_pytorch_tensorflow_sklearn.ipynb @@ -125,7 +125,7 @@ "outputs": [], "source": [ "!pip install --upgrade pip\n", - "!pip install apache_beam[gcp]>=2.40.0\n", + "!pip install apache_beam[interactive,gcp]>=2.40.0\n", "!pip install transformers\n", "!pip install google-api-core==1.32" ] @@ -406,7 +406,7 @@ "source": [ "!pip install --upgrade pip\n", "!pip install google-api-core==1.32\n", - "!pip install apache_beam[gcp]==2.41.0\n", + "!pip install apache_beam[interactive,gcp]==2.41.0\n", "!pip install tensorflow==2.8\n", "!pip install tfx_bsl\n", "!pip install tensorflow-text==2.8.1" @@ -649,7 +649,7 @@ "source": [ "!pip install --upgrade pip\n", "!pip install google-api-core==1.32\n", - "!pip install apache_beam[gcp]==2.41.0" + "!pip install apache_beam[interactive,gcp]==2.41.0" ] }, { diff --git a/examples/notebooks/beam-ml/run_inference_tensorflow.ipynb b/examples/notebooks/beam-ml/run_inference_tensorflow.ipynb index c15e9b21ecf9..ebeff1f77dbc 100644 --- a/examples/notebooks/beam-ml/run_inference_tensorflow.ipynb +++ b/examples/notebooks/beam-ml/run_inference_tensorflow.ipynb @@ -105,7 +105,7 @@ "outputs": [], "source": [ "!pip install protobuf --quiet\n", - "!pip install apache_beam==2.46.0 --quiet\n", + "!pip install apache_beam[interactive]==2.46.0 --quiet\n", "\n", "# To use the newly installed versions, restart the runtime.\n", "exit()" diff --git a/examples/notebooks/beam-ml/run_inference_tensorflow_with_tfx.ipynb b/examples/notebooks/beam-ml/run_inference_tensorflow_with_tfx.ipynb index 2c2f6460651b..42b300d943e4 100644 --- a/examples/notebooks/beam-ml/run_inference_tensorflow_with_tfx.ipynb +++ b/examples/notebooks/beam-ml/run_inference_tensorflow_with_tfx.ipynb @@ -100,7 +100,7 @@ "source": [ "!pip install tfx_bsl==1.10.0 --quiet\n", "!pip install protobuf --quiet\n", - "!pip install apache_beam --quiet" + "!pip install apache_beam[interactive] --quiet" ] }, { diff --git a/examples/notebooks/beam-ml/run_inference_vertex_ai.ipynb b/examples/notebooks/beam-ml/run_inference_vertex_ai.ipynb index 2ab45e0491a7..3c328348c7bf 100644 --- a/examples/notebooks/beam-ml/run_inference_vertex_ai.ipynb +++ b/examples/notebooks/beam-ml/run_inference_vertex_ai.ipynb @@ -109,7 +109,7 @@ "outputs": [], "source": [ "!pip install protobuf --quiet\n", - "!pip install apache_beam[gcp,interactive]==2.50.0 --quiet\n", + "!pip install apache_beam[interactive,gcp]==2.50.0 --quiet\n", "# Enforce shapely < 2.0.0 to avoid an issue with google.aiplatform\n", "!pip install shapely==1.7.1 --quiet\n", "\n", diff --git a/examples/notebooks/beam-ml/run_inference_with_tensorflow_hub.ipynb b/examples/notebooks/beam-ml/run_inference_with_tensorflow_hub.ipynb index b396851f9dcc..8ef185eaf0ff 100644 --- a/examples/notebooks/beam-ml/run_inference_with_tensorflow_hub.ipynb +++ b/examples/notebooks/beam-ml/run_inference_with_tensorflow_hub.ipynb @@ -95,7 +95,7 @@ }, "source": [ "!pip install tensorflow\n", - "!pip install apache_beam==2.46.0" + "!pip install apache_beam[interactive]==2.46.0" ], "execution_count": null, "outputs": [] diff --git a/examples/notebooks/beam-ml/speech_emotion_tensorflow.ipynb b/examples/notebooks/beam-ml/speech_emotion_tensorflow.ipynb index c2dfb06a6e67..3c47d7be0fb9 100644 --- a/examples/notebooks/beam-ml/speech_emotion_tensorflow.ipynb +++ b/examples/notebooks/beam-ml/speech_emotion_tensorflow.ipynb @@ -112,7 +112,7 @@ } ], "source": [ - "!pip install apache_beam --quiet" + "!pip install apache_beam[interactive] --quiet" ] }, { diff --git a/examples/notebooks/blog/unittests_in_beam.ipynb b/examples/notebooks/blog/unittests_in_beam.ipynb index da3f39d02959..2eacc69914f7 100644 --- a/examples/notebooks/blog/unittests_in_beam.ipynb +++ b/examples/notebooks/blog/unittests_in_beam.ipynb @@ -58,7 +58,7 @@ "cell_type": "code", "source": [ "# Install the Apache Beam library\n", - "!pip install apache_beam[gcp] --quiet" + "!pip install apache_beam[interactive,gcp] --quiet" ], "metadata": { "id": "5W2nuV7uzlPg" diff --git a/examples/yaml/README.md b/examples/yaml/README.md new file mode 100644 index 000000000000..121b0b03bcb7 --- /dev/null +++ b/examples/yaml/README.md @@ -0,0 +1,54 @@ + + +## Example YAML Pipelines + +A suite of YAML pipeline examples is currently located under the directory +[sdks/python/apache_beam/yaml/examples](../../sdks/python/apache_beam/yaml/examples). + +### [Aggregation](../../sdks/python/apache_beam/yaml/examples/transforms/aggregation) + +These examples leverage the built-in `Combine` transform for performing simple +aggregations including sum, mean, count, etc. + +### [Blueprints](../../sdks/python/apache_beam/yaml/examples/transforms/blueprint) + +These examples leverage DF or other existing templates and convert them to yaml +blueprints. + +### [Element-wise](../../sdks/python/apache_beam/yaml/examples/transforms/elementwise) + +These examples leverage the built-in mapping transforms including `MapToFields`, +`Filter` and `Explode`. + +### [IO](../../sdks/python/apache_beam/yaml/examples/transforms/io) + +These examples leverage the built-in IO transforms to read from and write to +various sources and sinks, including Iceberg, Kafka and Spanner. + +### [Jinja](../../sdks/python/apache_beam/yaml/examples/transforms/jinja) + +These examples use Jinja [templatization](https://beam.apache.org/documentation/sdks/yaml/#jinja-templatization) +to build off of different contexts and/or with different +configurations. + +### [ML](../../sdks/python/apache_beam/yaml/examples/transforms/ml) + +These examples include built-in ML-specific transforms such as `RunInference`, +`MLTransform` and `Enrichment`. diff --git a/gradle.properties b/gradle.properties index beb498d11943..61e25944ccf3 100644 --- a/gradle.properties +++ b/gradle.properties @@ -30,8 +30,8 @@ signing.gnupg.useLegacyGpg=true # buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy. # To build a custom Beam version make sure you change it in both places, see # https://github.com/apache/beam/issues/21302. -version=2.68.0-SNAPSHOT -sdk_version=2.68.0.dev +version=2.69.0-SNAPSHOT +sdk_version=2.69.0.dev javaVersion=1.8 diff --git a/infra/enforcement/README.md b/infra/enforcement/README.md new file mode 100644 index 000000000000..8136081fed75 --- /dev/null +++ b/infra/enforcement/README.md @@ -0,0 +1,224 @@ + + +# Infrastructure rules enforcement + +This module is used to check that the infrastructure rules are being used and provides automated notifications for compliance violations. + +The enforcement tools support multiple notification methods: +- **GitHub Issues**: Automatically create GitHub issues with detailed compliance reports +- **Email Notifications**: Send email alerts via SMTP for compliance violations +- **Console Output**: Print detailed reports to console for manual review + +## IAM Policies + +The enforcement is done by validating the IAM policies against the defined policies. +The tool monitors and enforces compliance for user permissions, service account roles, and group memberships across your GCP project. + +### Usage + +You can specify the action either through the configuration file (`config.yml`) or via command-line arguments: + +```bash +# Check compliance and report issues (default) +python iam.py --action check + +# Create/update GitHub issue and send email if compliance violations are found +python iam.py --action announce + +# Print announcement details for testing purposes (no actual issue created) +python iam.py --action print + +# Generate new compliance file based on current IAM policy +python iam.py --action generate +``` + +### Actions + +- **check**: Validates IAM policies against defined policies and reports any differences (default behavior) +- **announce**: Creates or updates a GitHub issue and sends an email notification when IAM policies differ from the defined ones. If no open issue exists, creates a new one; if an open issue exists, updates the issue body with current violations +- **print**: Prints announcement details for testing purposes without creating actual GitHub issues or sending emails +- **generate**: Updates the compliance file to match the current GCP IAM policy, creating a new baseline from existing permissions + +### Features + +The IAM Policy enforcement tool provides the following capabilities: + +- **Comprehensive Policy Export**: Automatically exports all IAM bindings and roles from the GCP project +- **Member Type Recognition**: Handles users, service accounts, and groups with proper parsing and identification +- **Permission Comparison**: Detailed comparison between expected and actual permissions for each user +- **Conditional Role Filtering**: Automatically excludes conditional roles (roles with conditions) from compliance checks +- **Sorted Output**: Provides consistent, sorted output for easy comparison and review +- **Detailed Reporting**: Comprehensive reporting of permission differences with clear before/after comparisons +- **GitHub Integration**: Automatic issue creation with detailed compliance violation reports +- **Email Notifications**: Optional email notifications for compliance issues via SMTP +- **Issue Management**: Smart issue handling - creates new issues when none exist, updates existing open issues with current violations +- **Testing Support**: Print action allows testing notification content without actually sending + +### Configuration + +The `config.yml` file supports the following parameters for IAM policies: + +- `project_id`: GCP project ID to check (default: `apache-beam-testing`) +- `users_file`: Path to the YAML file containing expected IAM policies (default: `../iam/users.yml`) +- `action`: Default action to perform (`check`, `announce`, `print`, or `generate`) +- `logging`: Logging configuration (level and format) + +### Environment Variables (for announce action) + +When using the `announce` action, the following environment variables are required: + +- `GITHUB_TOKEN`: GitHub personal access token for creating issues +- `GITHUB_REPOSITORY`: Repository in format `owner/repo` (default: `apache/beam`) +- `SMTP_SERVER`: SMTP server for email notifications +- `SMTP_PORT`: SMTP port (default: 587) +- `EMAIL_ADDRESS`: Email address for sending notifications +- `EMAIL_PASSWORD`: Email password for authentication +- `EMAIL_RECIPIENT`: Email address to receive notifications + +### IAM Policy File Format + +The IAM policy file should follow this YAML structure: + +```yaml +- username: john.doe + email: john.doe@example.com + permissions: + - role: roles/viewer + - role: roles/storage.objectViewer +- username: service-account-name + email: service-account-name@project-id.iam.gserviceaccount.com + permissions: + - role: roles/compute.instanceAdmin + - role: roles/iam.serviceAccountUser +``` + +Each user entry includes: +- `username`: The derived username (typically the part before @ in email addresses) +- `email`: The full email address of the user or service account +- `permissions`: List of IAM roles assigned to this member + - `role`: The full GCP IAM role name (e.g., `roles/viewer`, `roles/editor`) + +### Compliance Checking Process + +1. **Policy Extraction**: Retrieves current IAM policy from the GCP project +2. **Member Parsing**: Parses all IAM members and extracts usernames, emails, and types +3. **Role Processing**: Processes all roles while filtering out conditional bindings +4. **Comparison**: Compares current permissions with expected permissions from the policy file +5. **Reporting**: Generates detailed reports of any discrepancies found +6. **Notification**: Sends notifications via GitHub issues and/or email when using announce action + +The `print` action can be used for testing notification content without actually creating GitHub issues or sending emails. + +Command-line arguments take precedence over configuration file settings. + +## GitHub Actions Integration + +The enforcement tools are integrated with GitHub Actions to provide automated compliance monitoring. The workflow is configured to run weekly and automatically create GitHub issues and send email notifications for any policy violations. + +### Workflow Configuration + +The GitHub Actions workflow (`.github/workflows/beam_Infrastructure_PolicyEnforcer.yml`) runs: +- **Schedule**: Weekly on Mondays at 9:00 AM UTC +- **Manual trigger**: Can be triggered manually via `workflow_dispatch` +- **Actions**: Runs both IAM and Account Keys enforcement with the `announce` action + +**Note**: +- The email service is configured to use gmail +- The recipient email is set to `dev@beam.apache.org` for Apache Beam project notifications +- The `GITHUB_TOKEN` is automatically provided by GitHub Actions and doesn't need to be configured manually + +## Account Keys + +The enforcement is also done by validating service account keys and their access permissions against the defined policies. +The tool supports three different actions when discrepancies are found: + +### Usage + +You can specify the action either through the configuration file (`config.yml`) or via command-line arguments: + +```bash +# Check compliance and report issues (default) +python account_keys.py --action check + +# Create/update GitHub issue and send email if compliance violations are found +python account_keys.py --action announce + +# Print announcement details for testing purposes (no actual issue created) +python account_keys.py --action print + +# Generate new compliance file based on current service account keys policy +python account_keys.py --action generate +``` + +### Actions + +- **check**: Validates service account keys and their permissions against defined policies and reports any differences (default behavior) +- **announce**: Creates or updates a GitHub issue and sends an email notification when service account keys policies differ from the defined ones. If no open issue exists, creates a new one; if an open issue exists, updates the issue body with current violations +- **print**: Prints announcement details for testing purposes without creating actual GitHub issues or sending emails +- **generate**: Updates the compliance file to match the current GCP service account keys and Secret Manager permissions + +### Features + +The Account Keys enforcement tool provides the following capabilities: + +- **Service Account Discovery**: Automatically discovers all active (non-disabled) service accounts in the project +- **Secret Manager Integration**: Monitors secrets created by the beam-infra-secret-manager service +- **Permission Validation**: Ensures that Secret Manager permissions match the declared authorized users +- **Compliance Reporting**: Identifies missing service accounts, undeclared managed secrets, and permission mismatches +- **Automatic Remediation**: Can automatically update the compliance file to match current infrastructure state + +### Configuration + +The `config.yml` file supports the following parameters for account keys: + +- `project_id`: GCP project ID to check +- `service_account_keys_file`: Path to the YAML file containing expected service account keys policies (default: `../keys/keys.yaml`) +- `action`: Default action to perform (`check`, `announce`, `print`, or `generate`) +- `logging`: Logging configuration (level and format) + +### Environment Variables (for announce action) + +When using the `announce` action, the following environment variables are required: + +- `GITHUB_TOKEN`: GitHub personal access token for creating issues +- `GITHUB_REPOSITORY`: Repository in format `owner/repo` (default: `apache/beam`) +- `SMTP_SERVER`: SMTP server for email notifications +- `SMTP_PORT`: SMTP port (default: 587) +- `EMAIL_ADDRESS`: Email address for sending notifications +- `EMAIL_PASSWORD`: Email password for authentication +- `EMAIL_RECIPIENT`: Email address to receive notifications + +### Service Account Keys File Format + +The service account keys file should follow this YAML structure: + +```yaml +service_accounts: +- account_id: example-service-account + display_name: example-service-account@project-id.iam.gserviceaccount.com + authorized_users: + - email: user1@example.com + - email: user2@example.com +``` + +Each service account entry includes: +- `account_id`: The unique identifier for the service account (without the full email domain) +- `display_name`: The full service account email address or any custom display name +- `authorized_users`: List of users who should have access to the service account's secrets diff --git a/infra/enforcement/account_keys.py b/infra/enforcement/account_keys.py new file mode 100644 index 000000000000..4c3a8190d23f --- /dev/null +++ b/infra/enforcement/account_keys.py @@ -0,0 +1,523 @@ +# 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. + +import datetime +import logging +import sys +import yaml +import argparse +import os +from typing import List, Dict, TypedDict, Optional +from google.cloud import secretmanager +from google.cloud import iam_admin_v1 +from google.cloud.iam_admin_v1 import types +from sending import SendingClient + +SECRET_MANAGER_LABEL = "beam-infra-secret-manager" + +class AuthorizedUser(TypedDict): + email: str + +class ServiceAccount(TypedDict): + account_id: str + display_name: str + authorized_users: List[AuthorizedUser] + +class ServiceAccountsConfig(TypedDict): + service_accounts: List[ServiceAccount] + +CONFIG_FILE = "config.yml" + +class AccountKeysPolicyComplianceCheck: + def __init__(self, project_id: str, service_account_keys_file: str, logger: logging.Logger, sending_client: Optional[SendingClient] = None): + self.project_id = project_id + self.service_account_keys_file = service_account_keys_file + self.logger = logger + self.sending_client = sending_client + self.secret_client = secretmanager.SecretManagerServiceClient() + self.service_account_client = iam_admin_v1.IAMClient() + + def _normalize_account_email(self, account_id: str) -> str: + """ + Normalizes the account identifier to a full email format. + + Args: + account_id (str): The unique identifier or email of the service account. + + Returns: + str: The full service account email address. + """ + if "@" in account_id: + return account_id + else: + return f"{account_id}@{self.project_id}.iam.gserviceaccount.com" + + def _denormalize_account_email(self, email: str) -> str: + """ + Denormalizes the full service account email address to its unique identifier. + + Args: + email (str): The full service account email address. + + Returns: + str: The unique identifier for the service account. + """ + if email.endswith(f"@{self.project_id}.iam.gserviceaccount.com"): + return email.split("@")[0] + return email + + def _normalize_username(self, username: str) -> str: + """ + Normalizes the username to a consistent format. + + Args: + username (str): The username to normalize. + + Returns: + str: The normalized username. + """ + if not username.startswith("user:"): + return f"user:{username.strip().lower()}" + return username + + def _denormalize_username(self, username: str) -> str: + """ + Denormalizes the username from the consistent format. + + Args: + username (str): The normalized username. + + Returns: + str: The denormalized username. + """ + if username.startswith("user:"): + return username.split(":", 1)[1].strip().lower() + return username + + def _get_all_live_service_accounts(self) -> List[str]: + """ + Retrieves all service accounts that are currently active (not disabled) in the project. + + Returns: + List[str]: A list of email addresses for all live service accounts. + """ + request = types.ListServiceAccountsRequest() + request.name = f"projects/{self.project_id}" + + try: + accounts = self.service_account_client.list_service_accounts(request=request) + self.logger.debug(f"Retrieved {len(accounts.accounts)} service accounts for project {self.project_id}") + + if not accounts: + self.logger.warning(f"No service accounts found in project {self.project_id}.") + return [] + + return [self._normalize_account_email(account.email) for account in accounts.accounts if not account.disabled] + except Exception as e: + self.logger.error(f"Failed to retrieve service accounts for project {self.project_id}: {e}") + raise + + def _get_all_live_managed_secrets(self) -> List[str]: + """ + Retrieves the list of secrets from the Secret Manager that where created by the beam-secret-service + + Returns: + List[str]: A list of secret ids + """ + try: + secrets = list(self.secret_client.list_secrets(request={"parent": f"projects/{self.project_id}"})) + self.logger.debug(f"Retrieved {len(secrets)} secrets for project {self.project_id}") + + if not secrets: + self.logger.warning(f"No secrets found in project {self.project_id}.") + return [] + + return [secret.name.split("/")[-1] for secret in secrets if "created_by" in secret.labels and secret.labels["created_by"] == SECRET_MANAGER_LABEL] + except Exception as e: + self.logger.error(f"Failed to retrieve secrets for project {self.project_id}: {e}") + raise + + def _get_all_secret_authorized_users(self, secret_id: str) -> List[str]: + """ + Retrieves a list of all users who have access to the secrets in the Secret Manager. + + Args: + secret_id (str): The ID of the secret to check access for. + Returns: + List[str]: A list of email addresses for all users authorized to access the secrets. + """ + accessor_role = "roles/secretmanager.secretAccessor" + resource_name = self.secret_client.secret_path(self.project_id, secret_id) + + try: + policy = self.secret_client.get_iam_policy(request={"resource": resource_name}) + self.logger.debug(f"Retrieved IAM policy for secret '{secret_id}': {policy}") + + if not policy.bindings: + self.logger.warning(f"No IAM bindings found for secret '{secret_id}'.") + return [] + + authorized_users = [] + for binding in policy.bindings: + if binding.role == accessor_role: + for user in binding.members: + authorized_users.append(self._normalize_username(user)) + + return authorized_users + except Exception as e: + self.logger.error(f"Failed to get IAM policy for secret '{secret_id}': {e}") + raise + + def _read_service_account_keys(self) -> ServiceAccountsConfig: + """ + Reads the service account keys from a YAML file and returns a list of ServiceAccount objects. + + Returns: + List[ServiceAccount]: A list of service account declarations. + """ + try: + with open(self.service_account_keys_file, "r") as file: + keys = yaml.safe_load(file) + + if not keys or keys.get("service_accounts") is None: + return {"service_accounts": []} + + return keys + except FileNotFoundError: + self.logger.info(f"Service account keys file {self.service_account_keys_file} not found, starting with empty configuration") + return {"service_accounts": []} + except IOError as e: + error_msg = f"Failed to read service account keys from {self.service_account_keys_file}: {e}" + self.logger.error(error_msg) + raise + + def _to_yaml_file(self, data: List[ServiceAccount], output_file: str, header_info: str = "") -> None: + """ + Writes a list of dictionaries to a YAML file. + Include the apache license header on the files + + Args: + data: A list of dictionaries containing user permissions and details. + output_file: The file path where the YAML output will be written. + header_info: A string containing the header information to be included in the YAML file. + """ + + apache_license_header = """# 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. + """ + + # Prepare the header with the Apache license + header = f"{apache_license_header}\n# {header_info}\n# Generated on {datetime.datetime.now(datetime.timezone.utc).strftime('%Y-%m-%d %H:%M:%S')} UTC\n\n" + + try: + with open(output_file, "w") as file: + file.write(header) + yaml_data = {"service_accounts": data} + yaml.dump(yaml_data, file, sort_keys=False, default_flow_style=False, indent=2) + self.logger.info(f"Successfully wrote Service Account Keys policy data to {output_file}") + except IOError as e: + self.logger.error(f"Failed to write to {output_file}: {e}") + + + def check_compliance(self) -> List[str]: + """ + Checks the compliance of service account keys with the defined policies. + + Returns: + List[str]: A list of compliance issue messages. + """ + + service_account_data = self._read_service_account_keys() + file_service_accounts = service_account_data.get("service_accounts") + + if not file_service_accounts: + file_service_accounts = [] + self.logger.info(f"No service account keys found in the {self.service_account_keys_file}.") + + compliance_issues = [] + + # Check that all service accounts that exist are declared + for service_account in self._get_all_live_service_accounts(): + if self._denormalize_account_email(service_account) not in [account["account_id"] for account in file_service_accounts]: + msg = f"Service account '{service_account}' is not declared in the service account keys file." + compliance_issues.append(msg) + self.logger.warning(msg) + + managed_secrets = self._get_all_live_managed_secrets() + extracted_secrets = [f"{self._denormalize_account_email(account['account_id'])}-key" for account in file_service_accounts] + + # Check for managed secrets that are not declared + for secret in managed_secrets: + if secret not in extracted_secrets: + msg = f"Managed secret '{secret}' is not declared in the service account keys file." + compliance_issues.append(msg) + self.logger.warning(msg) + + # Check for each managed secret if it has the correct permissions + for account in file_service_accounts: + secret_name = f"{self._denormalize_account_email(account['account_id'])}-key" + if secret_name not in managed_secrets: + # Skip accounts that don't have managed secrets + continue + + authorized_users = [user["email"] for user in account["authorized_users"]] + actual_users = [self._denormalize_username(user) for user in self._get_all_secret_authorized_users(secret_name)] + + # Sort both lists for proper comparison + authorized_users.sort() + actual_users.sort() + + if authorized_users != actual_users: + msg = f"Managed secret '{account['account_id']}' does not have the correct permissions. Expected: {authorized_users}, Actual: {actual_users}" + compliance_issues.append(msg) + self.logger.warning(msg) + + return compliance_issues + + def create_announcement(self, recipient: str) -> None: + """ + Creates an announcement about compliance issues using the SendingClient. + + Args: + recipient (str): The email address of the announcement recipient. + """ + if not self.sending_client: + raise ValueError("SendingClient is required for creating announcements") + + diff = self.check_compliance() + + if not diff: + self.logger.info("No compliance issues found, no announcement will be created.") + return + + title = f"Account Keys Compliance Issue Detected" + body = f"Account keys for project {self.project_id} are not compliant with the defined policies on {self.service_account_keys_file}\n\n" + for issue in diff: + body += f"- {issue}\n" + + announcement = f"Dear team,\n\nThis is an automated notification about compliance issues detected in the Account Keys policy for project {self.project_id}.\n\n" + announcement += f"We found {len(diff)} compliance issue(s) that need your attention.\n" + announcement += f"\nPlease check the GitHub issue for detailed information and take appropriate action to resolve these compliance violations." + + self.sending_client.create_announcement(title, body, recipient, announcement) + + def print_announcement(self, recipient: str) -> None: + """ + Prints announcement details instead of sending them (for testing purposes). + Args: + recipient (str): The email address of the announcement recipient. + """ + if not self.sending_client: + raise ValueError("SendingClient is required for printing announcements") + + diff = self.check_compliance() + + if not diff: + self.logger.info("No compliance issues found, no announcement will be printed.") + return + + title = f"Account Keys Compliance Issue Detected" + body = f"Account keys for project {self.project_id} are not compliant with the defined policies on {self.service_account_keys_file}\n\n" + for issue in diff: + body += f"- {issue}\n" + + announcement = f"Dear team,\n\nThis is an automated notification about compliance issues detected in the Account Keys policy for project {self.project_id}.\n\n" + announcement += f"We found {len(diff)} compliance issue(s) that need your attention.\n" + announcement += f"\nPlease check the GitHub issue for detailed information and take appropriate action to resolve these compliance violations." + + self.sending_client.print_announcement(title, body, recipient, announcement) + + def generate_compliance(self) -> None: + """ + Modifies the service account keys file to match the current state of service accounts and secrets. + It will just add the non managed service accounts. + """ + + service_account_data = self._read_service_account_keys() + file_service_accounts = service_account_data.get("service_accounts", []) + + # Ensure file_service_accounts is a list + if file_service_accounts is None: + file_service_accounts = [] + + self.logger.info(f"Found {len(file_service_accounts)} existing service accounts in the keys file") + + # Check that all service accounts that exist are declared, if not, add them + for service_account in self._get_all_live_service_accounts(): + if self._denormalize_account_email(service_account) not in [account["account_id"] for account in file_service_accounts]: + self.logger.info(f"Service account '{service_account}' is not declared in the service account keys file, adding it") + file_service_accounts.append({ + "account_id": self._denormalize_account_email(service_account), + "display_name": service_account, + "authorized_users": [] + }) + + managed_secrets = self._get_all_live_managed_secrets() + extracted_secrets = [f"{self._denormalize_account_email(account['account_id'])}-key" for account in file_service_accounts] + + # Check for managed secrets that are not declared, if not, add them + for secret in managed_secrets: + if secret not in extracted_secrets: + self.logger.info(f"Managed secret '{secret}' is not declared in the service account keys file, adding it") + file_service_accounts.append({ + "account_id": secret.strip("-key"), + "display_name": self._normalize_account_email(secret.strip("-key")), + "authorized_users": [] + }) + + # Check for each managed secret if it has the correct permissions + for account in file_service_accounts: + secret_name = f"{self._denormalize_account_email(account['account_id'])}-key" + if secret_name not in managed_secrets: + continue + + authorized_users = sorted([user["email"] for user in account["authorized_users"]]) + + if not authorized_users: + self.logger.info(f"Managed secret '{account}' is new, skipping permission check") + continue + + actual_users_normalized = sorted(self._get_all_secret_authorized_users(secret_name)) + actual_users = sorted([self._denormalize_username(user) for user in actual_users_normalized]) + + if authorized_users != actual_users: + self.logger.info(f"Managed secret '{account}' does not have the correct permissions, updating it") + account["authorized_users"] = [{"email": user} for user in actual_users] + + # Remove duplicates based on account_id + seen_accounts = set() + deduplicated_accounts = [] + for account in file_service_accounts: + if account["account_id"] not in seen_accounts: + seen_accounts.add(account["account_id"]) + deduplicated_accounts.append(account) + else: + self.logger.info(f"Removing duplicate entry for account '{account['account_id']}'") + + self._to_yaml_file(deduplicated_accounts, self.service_account_keys_file, header_info="Service Account Keys") + +def config_process() -> Dict[str, str]: + with open(CONFIG_FILE, "r") as file: + config = yaml.safe_load(file) + + if not config: + raise ValueError("Configuration file is empty or invalid.") + + config_res = dict() + + config_res["project_id"] = config.get("project_id", "apache-beam-testing") + config_res["logging_level"] = config.get("logging", {}).get("level", "INFO") + config_res["logging_format"] = config.get("logging", {}).get("format", "[%(asctime)s] %(levelname)s: %(message)s") + config_res["service_account_keys_file"] = config.get("service_account_keys_file", "../keys/keys.yaml") + config_res["action"] = config.get("action", "check") + + # SendingClient configuration + config_res["github_token"] = os.getenv("GITHUB_TOKEN", "") + config_res["github_repo"] = os.getenv("GITHUB_REPOSITORY", "apache/beam") + config_res["smtp_server"] = os.getenv("SMTP_SERVER", "") + config_res["smtp_port"] = os.getenv("SMTP_PORT", 587) + config_res["email"] = os.getenv("EMAIL_ADDRESS", "") + config_res["password"] = os.getenv("EMAIL_PASSWORD", "") + config_res["recipient"] = os.getenv("EMAIL_RECIPIENT", "") + + return config_res + +def main(): + # Parse command line arguments + parser = argparse.ArgumentParser(description="Account Keys Compliance Checker") + parser.add_argument("--action", choices=["check", "announce", "print", "generate"], + help="Action to perform: check compliance, create announcement, print announcement, or generate new compliance") + args = parser.parse_args() + + config = config_process() + + # Command line argument takes precedence over config file + action = args.action if args.action else config.get("action", "check") + + logging.basicConfig(level=getattr(logging, config["logging_level"].upper(), logging.INFO), + format=config["logging_format"]) + logger = logging.getLogger("AccountKeysPolicyComplianceCheck") + + # Create SendingClient if needed for announcement actions + sending_client = None + if action in ["announce", "print"]: + try: + # Provide default values for testing, especially for print action + github_token = config["github_token"] or "dummy-token" + github_repo = config["github_repo"] or "dummy/repo" + smtp_server = config["smtp_server"] or "dummy-server" + smtp_port = int(config["smtp_port"]) if config["smtp_port"] else 587 + email = config["email"] or "dummy@example.com" + password = config["password"] or "dummy-password" + + sending_client = SendingClient( + logger=logger, + github_token=github_token, + github_repo=github_repo, + smtp_server=smtp_server, + smtp_port=smtp_port, + email=email, + password=password + ) + except Exception as e: + logger.error(f"Failed to initialize SendingClient: {e}") + return 1 + + logger.info(f"Starting Account Keys policy compliance check with action: {action}") + account_keys_checker = AccountKeysPolicyComplianceCheck(config["project_id"], config["service_account_keys_file"], logger, sending_client) + + try: + if action == "check": + compliance_issues = account_keys_checker.check_compliance() + if compliance_issues: + logger.warning("Account Keys policy compliance issues found:") + for issue in compliance_issues: + logger.warning(issue) + else: + logger.info("Account Keys policy is compliant.") + elif action == "announce": + logger.info("Creating announcement for compliance violations...") + recipient = config["recipient"] or "admin@example.com" + account_keys_checker.create_announcement(recipient) + elif action == "print": + logger.info("Printing announcement for compliance violations...") + recipient = config["recipient"] or "admin@example.com" + account_keys_checker.print_announcement(recipient) + elif action == "generate": + logger.info("Generating new compliance based on current Account Keys policy...") + account_keys_checker.generate_compliance() + else: + logger.error(f"Unknown action: {action}") + return 1 + except Exception as e: + logger.error(f"Error executing action '{action}': {e}") + return 1 + + return 0 + +if __name__ == "__main__": + sys.exit(main()) diff --git a/infra/enforcement/config.yml b/infra/enforcement/config.yml new file mode 100644 index 000000000000..ae01931567af --- /dev/null +++ b/infra/enforcement/config.yml @@ -0,0 +1,38 @@ +# 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. + +# Project ID +project_id: apache-beam-testing + +# Logging +logging: + level: DEBUG + format: "[%(asctime)s] %(levelname)s: %(message)s" + +# IAM + +# Working users file +users_file: ../iam/users.yml + +# Service Account Keys +service_account_keys_file: ../keys/keys.yaml + +# Action to perform when running the script +# Options: +# - check: Check compliance and report issues (default) +# - announce: Create/update GitHub issue and send email if compliance violations are found +# - print: Print announcement details for testing purposes +# - generate: Generate new compliance file based on current IAM policy +action: announce diff --git a/infra/enforcement/iam.py b/infra/enforcement/iam.py new file mode 100644 index 000000000000..92246aa7c62a --- /dev/null +++ b/infra/enforcement/iam.py @@ -0,0 +1,400 @@ +# 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. + +import argparse +import datetime +import logging +import os +import sys +import yaml +from google.api_core import exceptions +from google.cloud import resourcemanager_v3 +from typing import Optional, List, Dict, Tuple +from sending import SendingClient + +CONFIG_FILE = "config.yml" + +class IAMPolicyComplianceChecker: + def __init__(self, project_id: str, users_file: str, logger: logging.Logger, sending_client: Optional[SendingClient] = None): + self.project_id = project_id + self.users_file = users_file + self.client = resourcemanager_v3.ProjectsClient() + self.logger = logger + self.sending_client = sending_client + + def _parse_member(self, member: str) -> tuple[str, Optional[str], str]: + """Parses an IAM member string to extract type, email, and a derived username. + + Args: + member: The IAM member string + Returns: + A tuple containing: + - username: The derived username from the member string. + - email: The email address if available, otherwise None. + - member_type: The type of the member (e.g., user, serviceAccount, group). + """ + email = None + username = member + + # Split the member string to determine type and identifier + parts = member.split(':', 1) + member_type = parts[0] if len(parts) > 1 else "unknown" + identifier = parts[1] if len(parts) > 1 else member + + if member_type in ["user", "serviceAccount", "group"]: + email = identifier + if '@' in identifier: + username = identifier.split('@')[0] + else: + username = identifier + else: + username = identifier + member_type = "unknown" + email = None + + return username, email, member_type + + def _export_project_iam(self) -> List[Dict]: + """Exports the IAM policy for a given project to YAML format. + + Returns: + A list of dictionaries containing the IAM policy details. + """ + + try: + policy = self.client.get_iam_policy(resource=f"projects/{self.project_id}") + self.logger.debug(f"Retrieved IAM policy for project {self.project_id}") + except exceptions.NotFound as e: + self.logger.error(f"Project {self.project_id} not found: {e}") + raise + except exceptions.PermissionDenied as e: + self.logger.error(f"Permission denied for project {self.project_id}: {e}") + raise + except Exception as e: + self.logger.error(f"An error occurred while retrieving IAM policy for project {self.project_id}: {e}") + raise + + members_data = {} + + for binding in policy.bindings: + role = binding.role + + for member_str in binding.members: + if member_str not in members_data: + username, email_address, member_type = self._parse_member(member_str) + if member_type == "unknown": + self.logger.warning(f"Skipping member {member_str} with no email address") + continue # Skip if no email address is found, probably a malformed member + members_data[member_str] = { + "username": username, + "email": email_address, + "permissions": [] + } + + # Skip permissions that have a condition + if "withcond" in role: + continue + + permission_entry = {} + permission_entry["role"] = role + + members_data[member_str]["permissions"].append(permission_entry) + + output_list = [] + for data in members_data.values(): + data["permissions"] = sorted(data["permissions"], key=lambda p: p["role"]) + output_list.append({ + "username": data["username"], + "email": data["email"], + "permissions": data["permissions"] + }) + + output_list.sort(key=lambda x: x["username"]) + return output_list + + def _read_project_iam_file(self) -> List[Dict]: + """Reads the IAM policy from a YAML file. + + Returns: + A list of dictionaries containing the IAM policy details. + """ + try: + with open(self.users_file, "r") as file: + iam_policy = yaml.safe_load(file) + + + self.logger.debug(f"Retrieved IAM policy from file for project {self.project_id}") + return iam_policy + except FileNotFoundError: + self.logger.error(f"IAM policy file not found for project {self.project_id}") + return [] + except Exception as e: + self.logger.error(f"An error occurred while reading IAM policy file for project {self.project_id}: {e}") + return [] + + def _to_yaml_file(self, data: List[Dict], output_file: str, header_info: str = "") -> None: + """ + Writes a list of dictionaries to a YAML file. + Include the apache license header on the files + + Args: + data: A list of dictionaries containing user permissions and details. + output_file: The file path where the YAML output will be written. + header_info: A string containing the header information to be included in the YAML file. + """ + + apache_license_header = """# 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. + """ + + # Prepare the header with the Apache license + header = f"{apache_license_header}\n# {header_info}\n# Generated on {datetime.datetime.now(datetime.timezone.utc).strftime('%Y-%m-%d %H:%M:%S')} UTC\n\n" + + try: + with open(output_file, "w") as file: + file.write(header) + yaml.dump(data, file, sort_keys=False, default_flow_style=False, indent=2) + self.logger.info(f"Successfully wrote IAM policy data to {output_file}") + except IOError as e: + self.logger.error(f"Failed to write to {output_file}: {e}") + raise + + def check_compliance(self) -> List[str]: + """ + Checks the compliance of the IAM policy against the defined policies. + + Returns: + A list of strings describing any compliance issues found. + """ + current_users = {user['email']: user for user in self._export_project_iam()} + existing_users = {user['email']: user for user in self._read_project_iam_file()} + + if not existing_users: + error_msg = f"No IAM policy found in the {self.users_file}." + self.logger.info(error_msg) + raise RuntimeError(error_msg) + + differences = [] + + all_emails = set(current_users.keys()) | set(existing_users.keys()) + + for email in sorted(list(all_emails)): + current_user = current_users.get(email) + existing_user = existing_users.get(email) + + if current_user and not existing_user: + differences.append(f"User {email} not found in existing policy.") + elif not current_user and existing_user: + differences.append(f"User {email} found in policy file but not in GCP.") + elif current_user and existing_user: + if current_user["permissions"] != existing_user["permissions"]: + msg = f"\nPermissions for user {email} differ." + msg += f"\nIn GCP: {current_user['permissions']}" + msg += f"\nIn {self.users_file}: {existing_user['permissions']}" + self.logger.info(msg) + differences.append(msg) + + return differences + + def create_announcement(self, recipient: str) -> None: + """ + Creates an announcement about compliance issues using the SendingClient. + + Args: + recipient (str): The email address of the announcement recipient. + """ + if not self.sending_client: + raise ValueError("SendingClient is required for creating announcements") + + diff = self.check_compliance() + + if not diff: + self.logger.info("No compliance issues found, no announcement will be created.") + return + + title = f"IAM Policy Non-Compliance Detected" + body = f"IAM policy for project {self.project_id} is not compliant with the defined policies on {self.users_file}\n\n" + for issue in diff: + body += f"- {issue}\n" + + announcement = f"Dear team,\n\nThis is an automated notification about compliance issues detected in the IAM policy for project {self.project_id}.\n\n" + announcement += f"We found {len(diff)} compliance issue(s) that need your attention.\n" + announcement += f"\nPlease check the GitHub issue for detailed information and take appropriate action to resolve these compliance violations." + + self.sending_client.create_announcement(title, body, recipient, announcement) + + def print_announcement(self, recipient: str) -> None: + """ + Prints announcement details instead of sending them (for testing purposes). + + Args: + recipient (str): The email address of the announcement recipient. + """ + if not self.sending_client: + raise ValueError("SendingClient is required for printing announcements") + + diff = self.check_compliance() + + if not diff: + self.logger.info("No compliance issues found, no announcement will be printed.") + return + + title = f"IAM Policy Non-Compliance Detected" + body = f"IAM policy for project {self.project_id} is not compliant with the defined policies on {self.users_file}\n\n" + for issue in diff: + body += f"- {issue}\n" + + announcement = f"Dear team,\n\nThis is an automated notification about compliance issues detected in the IAM policy for project {self.project_id}.\n\n" + announcement += f"We found {len(diff)} compliance issue(s) that need your attention.\n" + announcement += f"\nPlease check the GitHub issue for detailed information and take appropriate action to resolve these compliance violations." + + self.sending_client.print_announcement(title, body, recipient, announcement) + + def generate_compliance(self) -> None: + """ + Modifies the users file to match the current IAM policy. + If no changes are needed, no file will be written. + """ + + try: + diff = self.check_compliance() + except RuntimeError: + self.logger.info("No existing IAM policy found.") + diff = ["No existing policy found"] + + if not diff or (len(diff) == 1 and "No existing policy found" not in diff[0]): + self.logger.info("No compliance issues found, no changes will be made.") + return + + current_policy = self._export_project_iam() + header_info = f"IAM policy for project {self.project_id}" + + self._to_yaml_file(current_policy, self.users_file, header_info) + self.logger.info(f"Generated new compliance file: {self.users_file}") + +def config_process() -> Dict[str, str]: + with open(CONFIG_FILE, "r") as file: + config = yaml.safe_load(file) + + if not config: + raise ValueError("Configuration file is empty or invalid.") + + config_res = dict() + + config_res["project_id"] = config.get("project_id", "apache-beam-testing") + config_res["logging_level"] = config.get("logging", {}).get("level", "INFO") + config_res["logging_format"] = config.get("logging", {}).get("format", "[%(asctime)s] %(levelname)s: %(message)s") + config_res["users_file"] = config.get("users_file", "../iam/users.yml") + config_res["action"] = config.get("action", "check") + + # SendingClient configuration + config_res["github_token"] = os.getenv("GITHUB_TOKEN", "") + config_res["github_repo"] = os.getenv("GITHUB_REPOSITORY", "apache/beam") + config_res["smtp_server"] = os.getenv("SMTP_SERVER", "") + config_res["smtp_port"] = os.getenv("SMTP_PORT", 587) + config_res["email"] = os.getenv("EMAIL_ADDRESS", "") + config_res["password"] = os.getenv("EMAIL_PASSWORD", "") + config_res["recipient"] = os.getenv("EMAIL_RECIPIENT", "") + + return config_res + +def main(): + # Parse command line arguments + parser = argparse.ArgumentParser(description="IAM Policy Compliance Checker") + parser.add_argument("--action", choices=["check", "announce", "print", "generate"], + help="Action to perform: check compliance, create announcement, print announcement, or generate new compliance") + args = parser.parse_args() + + config = config_process() + + # Command line argument takes precedence over config file + action = args.action if args.action else config.get("action", "check") + + logging.basicConfig(level=getattr(logging, config["logging_level"].upper(), logging.INFO), + format=config["logging_format"]) + logger = logging.getLogger("IAMPolicyComplianceChecker") + + # Create SendingClient if needed for announcement actions + sending_client = None + if action in ["announce", "print"]: + try: + # Provide default values for testing, especially for print action + github_token = config["github_token"] or "dummy-token" + github_repo = config["github_repo"] or "dummy/repo" + smtp_server = config["smtp_server"] or "dummy-server" + smtp_port = int(config["smtp_port"]) if config["smtp_port"] else 587 + email = config["email"] or "dummy@example.com" + password = config["password"] or "dummy-password" + + sending_client = SendingClient( + logger=logger, + github_token=github_token, + github_repo=github_repo, + smtp_server=smtp_server, + smtp_port=smtp_port, + email=email, + password=password + ) + except Exception as e: + logger.error(f"Failed to initialize SendingClient: {e}") + return 1 + + logger.info(f"Starting IAM policy compliance check with action: {action}") + iam_checker = IAMPolicyComplianceChecker(config["project_id"], config["users_file"], logger, sending_client) + + try: + if action == "check": + compliance_issues = iam_checker.check_compliance() + if compliance_issues: + logger.warning("IAM policy compliance issues found:") + for issue in compliance_issues: + logger.warning(issue) + else: + logger.info("IAM policy is compliant.") + elif action == "announce": + logger.info("Creating announcement for compliance violations...") + recipient = config["recipient"] or "admin@example.com" + iam_checker.create_announcement(recipient) + elif action == "print": + logger.info("Printing announcement for compliance violations...") + recipient = config["recipient"] or "admin@example.com" + iam_checker.print_announcement(recipient) + elif action == "generate": + logger.info("Generating new compliance based on current IAM policy...") + iam_checker.generate_compliance() + else: + logger.error(f"Unknown action: {action}") + return 1 + except Exception as e: + logger.error(f"Error executing action '{action}': {e}") + return 1 + + return 0 + +if __name__ == "__main__": + + sys.exit(main()) diff --git a/infra/enforcement/requirements.txt b/infra/enforcement/requirements.txt new file mode 100644 index 000000000000..1015266195cf --- /dev/null +++ b/infra/enforcement/requirements.txt @@ -0,0 +1,24 @@ +# +# 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. + +# This file is used to install the dependencies for the infrastructure + +PyYAML==6.0.2 +google-cloud-iam==2.19.0 +google-cloud-resource-manager==1.14.1 +google-cloud-secret-manager==2.24.0 +google-crc32c==1.7.1 +requests==2.32.4 diff --git a/infra/enforcement/sending.py b/infra/enforcement/sending.py new file mode 100644 index 000000000000..961674ca2f17 --- /dev/null +++ b/infra/enforcement/sending.py @@ -0,0 +1,179 @@ +# 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. + +import requests +import logging +import smtplib, ssl +from typing import List, Optional +from dataclasses import dataclass + +@dataclass +class GitHubIssue: + """ + Represents a GitHub issue. + """ + number: int + title: str + body: str + state: str + html_url: str + created_at: str + updated_at: str + +class SendingClient: + """ + Sends notifications about GitHub issues. + """ + def __init__(self, logger: logging.Logger, github_token: str, github_repo: str, + smtp_server: str, smtp_port: int, email: str, password: str): + + required_keys = [github_token, github_repo, smtp_server, smtp_port, email, password] + + if not all(required_keys): + raise ValueError("All parameters must be provided.") + + self.github_repo = github_repo + self.headers = { + "Authorization": f"Bearer {github_token}", + "X-GitHub-Api-Version": "2022-11-28", + "Accept": "application/vnd.github+json" + } + + self.smtp_server = smtp_server + self.smtp_port = smtp_port + self.email = email + self.password = password + + self.logger = logger + self.github_api_url = "https://api.github.com" + + def _make_github_request(self, method: str, endpoint: str, json: Optional[dict] = None) -> requests.Response: + """ + Makes a request to the GitHub API. + + Args: + method (str): The HTTP method to use (e.g., "GET", "POST", "PATCH"). + endpoint (str): The API endpoint to call. + json (Optional[dict]): The JSON payload to send with the request. + + Returns: + requests.Response: The response from the API. + """ + url = f"{self.github_api_url}/{endpoint}" + response = requests.request(method, url, headers=self.headers, json=json) + + if not response.ok: + self.logger.error(f"Failed GitHub API request to {endpoint}: {response.status_code} - {response.text}") + response.raise_for_status() + + return response + + def _send_email(self, title: str, body: str, recipient: str) -> None: + """ + Sends an email notification. + + Args: + title (str): The title of the email. + body (str): The body content of the email. + recipient (str): The email address of the recipient. + """ + message = f"Subject: {title}\n\n{body}" + context = ssl.create_default_context() + with smtplib.SMTP_SSL(self.smtp_server, self.smtp_port, context=context) as server: + server.login(self.email, self.password) + server.sendmail(self.email, recipient, message) + + def _get_open_issues(self, title: str) -> List[GitHubIssue]: + """ + Retrieves the number of open GitHub issues with a given title. + + Args: + title (str): The title of the GitHub issue. + """ + endpoint = f"search/issues/?q=is:issue+repo:{self.github_repo}+in:title+{title}+is:open" + response = self._make_github_request("GET", endpoint) + issues = response.json().get('items', []) + return [GitHubIssue(**issue) for issue in issues] + + def create_issue(self, title: str, body: str) -> GitHubIssue: + """ + Creates a GitHub issue in the specified repository. + + Args: + title (str): The title of the GitHub issue. + body (str): The body content of the GitHub issue. + """ + endpoint = f"repos/{self.github_repo}/issues" + payload = {"title": title, "body": body} + response = self._make_github_request("POST", endpoint, json=payload) + self.logger.info(f"Successfully created GitHub issue: {title}") + return GitHubIssue(**response.json()) + + def update_issue_body(self, issue_number: int, new_body: str) -> None: + """ + Updates the body of a GitHub issue in the specified repository. + + Args: + issue_number (int): The number of the GitHub issue to update. + new_body (str): The new body content for the GitHub issue. + """ + endpoint = f"repos/{self.github_repo}/issues/{issue_number}" + payload = {"body": new_body} + self._make_github_request("PATCH", endpoint, json=payload) + self.logger.info(f"Successfully updated body on GitHub issue: #{issue_number}") + + def create_announcement(self, title: str, body: str, recipient: str, announcement: str) -> None: + """ + This method sends an email with an announcement. The email will point to a GitHub issue. + + Creates a GitHub issue in the specified repository if it doesn't already exist. + If multiple open versions exist, the most recent one will be updated. + + Args: + title (str): The title of the GitHub issue. + body (str): The body content of the GitHub issue. + recipient (str): The email address of the recipient. + announcement (str): The announcement message to include in the email. + """ + open_issues = self._get_open_issues(title) + open_issues.sort(key=lambda x: x.updated_at, reverse=True) + if open_issues: + self.logger.info(f"Issue with title '{title}' already exists: #{open_issues[0].number}") + announcement += f"\n\nRelated GitHub Issue: {open_issues[0].html_url}" + + if open_issues[0].body != body: + self.logger.info(f"Updating body of issue #{open_issues[0].number}") + self.update_issue_body(open_issues[0].number, body) + else: + self.logger.info(f"No changes detected for issue #{open_issues[0].number}") + self._send_email(title, announcement, recipient) + else: + new_issue = self.create_issue(title, body) + announcement += f"\n\nRelated GitHub Issue: {new_issue.html_url}" + self._send_email(title, announcement, recipient) + + def print_announcement(self, title: str, body: str, recipient: str, announcement: str) -> None: + """ + This method prints the data instead of sending the email or creating an issue. + This is used for testing. + """ + self.logger.info("Printing announcement...") + print(f"Simulating email sending...") + print(f"Recipient: {recipient}") + print(f"Announcement: {announcement}") + + print("\nSimulating GitHub issue creation...") + print(f"Title: {title}") + print(f"Body: {body}") diff --git a/infra/iam/README.md b/infra/iam/README.md index 8c019cf48e1a..0322881aa856 100644 --- a/infra/iam/README.md +++ b/infra/iam/README.md @@ -61,3 +61,126 @@ This will update the IAM policies in the GCP project based on the changes made i - **config.auto.tfvars**: Contains the configuration variables for the Terraform project. - **users.tf**: Processes the `users.yml` file to associate users with their respective roles. - **users.yml**: A YAML file that contains the IAM policies and permissions for users and roles in the Beam project. + +### Migration and Automation + +- **migrate_roles.py**: Python script for migrating existing IAM policies to the new custom roles structure + +## Custom Roles + +The Beam project uses custom IAM roles to provide granular permissions for different levels of access to GCP resources. These roles follow a hierarchical structure where higher-level roles inherit permissions from lower-level roles. + +### Role Hierarchy + +The custom roles are structured in the following hierarchy: + +``` +beam_viewer < beam_writer < beam_infra_manager < beam_admin +``` + +### Available Roles + +#### beam_viewer +- **Description**: Read-only access to the Beam project resources +- **Permissions**: View-only access to all services used by Beam +- **Exclusions**: Secret management permissions, destructive actions +- **Use case**: For team members who need to monitor and observe project resources + +#### beam_writer +- **Description**: User access to resources in the Beam project +- **Permissions**: Inherits all `beam_viewer` permissions plus additional permissions for: + - BigQuery data access and querying + - Cloud SQL instance usage + - Container cluster viewing and development + - Datastore usage + - Network viewing +- **Exclusions**: Destructive actions, administrative operations +- **Use case**: For active contributors who need to work with project resources + +#### beam_infra_manager +- **Description**: Editor access to the Beam project infrastructure +- **Permissions**: Inherits all `beam_writer` permissions plus: + - Cloud Build editor access + - Service account token creation and usage + - Storage object creation and viewing + - General editor role (with exclusions) +- **Exclusions**: Destructive permissions, full administrative access +- **Use case**: For infrastructure maintainers who manage deployments and resources + +#### beam_admin +- **Description**: Full administrative access to the Beam project +- **Permissions**: Complete access including: + - All previous role permissions + - Administrative access to all services + - Secret management capabilities + - Destructive operations +- **Exclusions**: None +- **Use case**: For project administrators and senior maintainers + +### Managing Custom Roles + +Custom roles are defined and managed through configuration files in the `roles/` directory: + +- **roles_config.yaml**: Defines the roles, their hierarchy, services, and base permissions +- **generate_roles.py**: Python script that generates YAML role definitions from the configuration +- **roles.tf**: Terraform configuration that applies the custom roles to the GCP project + +To modify custom roles: + +1. Edit the `roles_config.yaml` file to update role definitions +2. Run `generate_roles.py` to regenerate the role YAML files +3. Apply changes through Terraform or via pull request + +For detailed information about custom roles management, see the [roles directory README](roles/README.md). + +### Migrating from Legacy Roles + +The `migrate_roles.py` script helps migrate existing GCP project IAM policies to the new custom roles structure. This is useful when transitioning from standard GCP roles to the custom Beam roles. + +#### Migration Rules + +The script applies the following hierarchical migration rules: + +- **Owner roles**: Left unchanged (highest privilege) +- **Admin/Secret roles**: Migrated to `beam_admin` (includes all lower roles) +- **Editor roles**: Migrated to `beam_infra_manager` (includes writer and viewer) +- **User roles**: Migrated to `beam_writer` (includes viewer) +- **Viewer roles**: Migrated to `beam_viewer` + +#### Using the Migration Script + +**Prerequisites:** +- Google Cloud SDK installed and authenticated +- Required Python dependencies (install with `pip install -r requirements.txt`) +- Appropriate GCP permissions to read IAM policies + +**Export and migrate IAM policies:** +```bash +python migrate_roles.py +``` + +This generates two files: +- `.original-roles.yaml`: Current IAM policy export +- `.migrated-roles.yaml`: Proposed migration to custom roles + +**Analyze permission differences for a specific user:** +```bash +python migrate_roles.py --difference +``` + +This generates: +- `.permission-differences.yaml`: Detailed comparison of permissions before and after migration + +**Example workflow:** +```bash +# Export current IAM policies and generate migration +python migrate_roles.py apache-beam-testing + +# Check permission differences for a specific user +python migrate_roles.py apache-beam-testing --difference user@example.com + +# Review the generated files before applying changes +# Then apply via Terraform or manual IAM policy updates +``` + +The migration script helps ensure a smooth transition to the custom roles while maintaining appropriate access levels for all users. diff --git a/infra/iam/generate.py b/infra/iam/generate.py deleted file mode 100644 index 71f6379710eb..000000000000 --- a/infra/iam/generate.py +++ /dev/null @@ -1,212 +0,0 @@ -# -# 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. -# -# THIS IS NOT SUPPOSED TO RUN AFTER THE MIGRATION. -# This script is used to export the IAM policy of a Google Cloud project to a YAML format. -# It retrieves the IAM policy bindings, parses the members, and formats the output in a structured -# YAML format, excluding service accounts and groups. The output includes usernames, emails, and -# their associated permissions, with optional conditions for roles that have conditions attached. -# You need to have the Google Cloud SDK installed and authenticated to run this script. - -import argparse -import datetime -import yaml -import logging -from typing import Optional, List, Dict -from google.cloud import resourcemanager_v3 -from google.api_core import exceptions - -# Configure logging -logging.basicConfig(level=logging.INFO, format='%(asctime)s - %(levelname)s - %(message)s') -logger = logging.getLogger(__name__) - -def parse_member(member: str) -> tuple[str, Optional[str], str]: - """Parses an IAM member string to extract type, email, and a derived username. - - Args: - member: The IAM member string - Returns: - A tuple containing: - - username: The derived username from the member string. - - email: The email address if available, otherwise None. - - member_type: The type of the member (e.g., user, serviceAccount, group). - """ - email = None - username = member - - # Split the member string to determine type and identifier - parts = member.split(':', 1) - member_type = parts[0] if len(parts) > 1 else "unknown" - identifier = parts[1] if len(parts) > 1 else member - - if member_type in ["user", "serviceAccount", "group"]: - email = identifier - if '@' in identifier: - username = identifier.split('@')[0] - else: - username = identifier - else: - username = identifier - email = None - - return username, email, member_type - -def export_project_iam(project_id: str) -> List[Dict]: - """Exports the IAM policy for a given project to YAML format. - - Args: - project_id: The ID of the Google Cloud project. - Returns: - A list of dictionaries containing the IAM policy details. - """ - - try: - client = resourcemanager_v3.ProjectsClient() - policy = client.get_iam_policy(resource=f"projects/{project_id}") - logger.info(f"Successfully retrieved IAM policy for project {project_id}") - except exceptions.NotFound as e: - logger.error(f"Project {project_id} not found: {e}") - raise - except exceptions.PermissionDenied as e: - logger.error(f"Permission denied for project {project_id}: {e}") - raise - except Exception as e: - logger.error(f"An error occurred while retrieving IAM policy for project {project_id}: {e}") - raise - - members_data = {} - - for binding in policy.bindings: - role = binding.role - - for member_str in binding.members: - if member_str not in members_data: - username, email_address, member_type = parse_member(member_str) - if member_type == "serviceAccount": - continue # Skip service accounts - if member_type == "group": - continue # Skip groups - if not email_address: - continue # Skip if no email address is found, probably a malformed member - members_data[member_str] = { - "username": username, - "email": email_address, - "permissions": [] - } - - # Skip permissions that have a condition - if "withcond" in role: - continue - - permission_entry = {} - permission_entry["role"] = role - - members_data[member_str]["permissions"].append(permission_entry) - - output_list = [] - for data in members_data.values(): - data["permissions"] = sorted(data["permissions"], key=lambda p: p["role"]) - output_list.append({ - "username": data["username"], - "email": data["email"], - "permissions": data["permissions"] - }) - - output_list.sort(key=lambda x: x["username"]) - return output_list - -def to_yaml_file(data: List[Dict], output_file: str, header_info: str = "") -> None: - """ - Writes a list of dictionaries to a YAML file. - Include the apache license header on the files - - Args: - data: A list of dictionaries containing user permissions and details. - output_file: The file path where the YAML output will be written. - header_info: A string containing the header information to be included in the YAML file. - """ - - apache_license_header = """# -# 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. -""" - - # Prepare the header with the Apache license - header = f"{apache_license_header}\n# {header_info}\n# Generated on {datetime.datetime.now(datetime.timezone.utc).strftime('%Y-%m-%d %H:%M:%S')} UTC\n\n" - - try: - with open(output_file, "w") as file: - file.write(header) - yaml.dump(data, file, sort_keys=False, default_flow_style=False, indent=2) - logger.info(f"Successfully wrote IAM policy data to {output_file}") - except IOError as e: - logger.error(f"Failed to write to {output_file}: {e}") - raise - -def main(): - """ - Main function to run the script. - - This function parses command-line arguments to either export IAM policies - or generate permission differences for a specified GCP project. - """ - parser = argparse.ArgumentParser( - description="Export IAM policies or generate permission differences for a GCP project." - ) - parser.add_argument( - "project_id", - help="The Google Cloud project ID." - ) - parser.add_argument( - "output_file", - help="Defaults to 'users.yml' if not specified. The file where the IAM policy will be saved in YAML format.", - nargs='?', - default="users.yml" - ) - parser.add_argument( - "--yes-i-know-what-i-am-doing", - action="store_true", - help="If set, the script will proceed" - ) - - args = parser.parse_args() - project_id = args.project_id - output_file = args.output_file - - if not args.yes_i_know_what_i_am_doing: - logger.error("You must use the --yes-i-know-what-i-am-doing flag to proceed.") - return - - # Export the IAM policy for the specified project - iam_data = export_project_iam(project_id) - - # Write the exported data to the specified output file in YAML format - to_yaml_file(iam_data, output_file, header_info=f"Exported IAM policy for project {project_id}") - -if __name__ == "__main__": - main() \ No newline at end of file diff --git a/infra/iam/main.tf b/infra/iam/main.tf index 28a1958a3135..42d1ceb62fc8 100644 --- a/infra/iam/main.tf +++ b/infra/iam/main.tf @@ -33,3 +33,8 @@ variable "project_id" { description = "The GCP project ID." type = string } + +module "beam_roles" { + source = "./roles" + project_id = var.project_id +} \ No newline at end of file diff --git a/infra/iam/migrate_roles.py b/infra/iam/migrate_roles.py new file mode 100644 index 000000000000..3abb9b7bcb0b --- /dev/null +++ b/infra/iam/migrate_roles.py @@ -0,0 +1,340 @@ +# +# 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. +# +# This script is used to export the IAM policy of a Google Cloud project to a YAML format. +# It retrieves the IAM policy bindings, parses the members, and formats the output in a structured +# YAML format, excluding service accounts and groups. The output includes usernames, emails, and +# their associated permissions, with optional conditions for roles that have conditions attached. +# You need to have the Google Cloud SDK installed and authenticated to run this script. + +import argparse +import os +import sys +import yaml +import roles.generate_roles as generate_roles +from generate import export_project_iam, to_yaml_file +from google.cloud.iam_admin_v1 import GetRoleRequest, IAMClient + +def migrate_permissions(data: list) -> list: + """ + Migrates permissions from the permissions to the new roles defined on beam_roles/ directory. + + The rules are: + - If the user has owner role, leave it as is, remove any other role as it is redundant. + - If the user has any admin or secret related role, it will be migrated to the beam_admin role. + - If the user has an editor role or any user role but not an admin or secret related role, it will be migrated to the beam_infra_manager role. + - If the user has a role that is not only viewer, it will be migrated to the beam_committer role. + - The users with just viewer roles will be migrated to the beam_viewer role. + + The rules are in a hierarchical order, meaning that if a user has a high role, it will also have the lower roles. + + Args: + data: A list of dictionaries containing user permissions and details. + Returns: + A list of dictionaries with migrated permissions. + """ + + migrated_data = [] + + for item in data: + username = item["username"] + email = item["email"] + permissions = item["permissions"] + + # Initialize the new roles + new_roles = { + "beam_owner": False, + "beam_admin": False, + "beam_infra_manager": False, + "beam_committer": False, + "beam_viewer": False + } + + for permission in permissions: + role = permission["role"] + + # If the role is 'roles/owner', it is considered an owner role. + if role == "roles/owner": + new_roles["beam_owner"] = True + # If it ends with 'admin' or containes 'secretmanager' in the role, it is considered an admin role. Case insensitive. + elif 'admin' in role.lower() or 'secretmanager' in role.lower(): + new_roles["beam_admin"] = True + new_roles["beam_infra_manager"] = True + new_roles["beam_committer"] = True + new_roles["beam_viewer"] = True + # If it is an editor role, it will be migrated to the beam_infra_manager. + elif role == "roles/editor": + new_roles["beam_infra_manager"] = True + new_roles["beam_committer"] = True + new_roles["beam_viewer"] = True + elif role != "roles/viewer": + # If it is a role that is not only viewer, it will be migrated to the beam_committer role. + new_roles["beam_committer"] = True + new_roles["beam_viewer"] = True + # If it is a viewer role, it will be migrated to the beam_viewer role. + else: + new_roles["beam_viewer"] = True + + # Create the migrated entry + migrated_entry = { + "username": username, + "email": email, + "permissions": [] + } + + if new_roles["beam_owner"]: + migrated_entry["permissions"].append({"role": "roles/owner"}) + else: + if new_roles["beam_admin"]: + migrated_entry["permissions"].append({"role": "projects/PROJECT-ID/roles/beam_admin"}) + if new_roles["beam_infra_manager"]: + migrated_entry["permissions"].append({"role": "projects/PROJECT-ID/roles/beam_infra_manager"}) + if new_roles["beam_committer"]: + migrated_entry["permissions"].append({"role": "projects/PROJECT-ID/roles/beam_committer"}) + if new_roles["beam_viewer"]: + migrated_entry["permissions"].append({"role": "projects/PROJECT-ID/roles/beam_viewer"}) + + migrated_data.append(migrated_entry) + + return migrated_data + +def get_gcp_role_permissions(role_id: str) -> list: + """ + Retrieves the permissions associated to a google cloud role. + Args: + project_id: The ID of the Google Cloud project. + role_id: The name of the role to retrieve permissions for. + Returns: + A list of permissions associated with the specified role. + """ + client = IAMClient() + + request = GetRoleRequest(name=role_id) + role = client.get_role(request=request) + + return list(role.included_permissions) + +def get_roles_from_file(file_path: str) -> list: + """ + Reads a YAML file containing roles and returns a list of dictionaries with user data. + + Args: + file_path: The path to the YAML file containing roles. + Returns: + A list of dictionaries with user data. + """ + with open(file_path, 'r') as file: + data = yaml.safe_load(file) + + roles = [] + for role in data: + email = role.get("email") + username = role.get("username") + permissions = role.get("permissions", []) + + roles.append({ + "email": email, + "username": username, + "permissions": permissions + }) + + return roles + +def permission_differences(project_id: str, user_email: str) -> list: + """ + Generates a list of differences between the original and migrated permissions for a user. + It gets the permission from the generated files, so it is expected that the files are already generated and up to date. + + Args: + project_id: The ID of the Google Cloud project. + user_email: The email of the user to compare permissions for. + Returns: + A list of dictionaries containing the differences in permissions for the specified user. + """ + + cache = {} + user_differences = {} + + original = get_roles_from_file(f"{project_id}.original-roles.yaml") + migrated = get_roles_from_file(f"{project_id}.migrated-roles.yaml") + + # Get the permissions on the beam_roles + beam_roles = generate_roles.get_roles() + for role_name, role_data in beam_roles.items(): + permissions = role_data["permissions"] + cache[role_name] = permissions + + # Get the permissions for the original roles + for user in original: + username = user["username"] + email = user["email"] + + # Skip if the user email does not match the specified user_email + if user_email and email != user_email: + continue + + original_roles = user["permissions"] + + original_permissions = [] + + for role in original_roles: + if '_withcond_' in role['role']: + # Skip roles with conditions, as they are not supported in the new roles + continue + if 'organizations/' in role['role']: + # Skip organization roles, as they are not supported in the new roles + continue + + if role['role'] not in cache: + permissions = get_gcp_role_permissions(role["role"]) + cache[role['role']] = sorted(permissions) + original_permissions.extend(cache[role['role']]) + + # Initialize the user differences entry + user_differences[username] = { + "email": email, + "original_roles": original_roles, + "original_permissions": sorted(original_permissions), + "migrated_roles": [], + "migrated_permissions": [], + "differences": [] + } + + # Get the permissions for the migrated roles + for user in migrated: + username = user["username"] + email = user["email"] + + # Skip if the user email does not match the specified user_email + if user_email and email != user_email: + continue + + migrated_roles = user["permissions"] + + migrated_permissions = [] + + for role in migrated_roles: + full_role_name = role["role"] + # Owner is a special case, it should not be migrated to any other role. + if "roles/owner" in full_role_name: + migrated_permissions.extend(get_gcp_role_permissions(full_role_name)) + else: + role_name = full_role_name.split('roles/')[1] + migrated_permissions.extend(cache[role_name]) + + user_differences[username]["migrated_roles"] = migrated_roles + user_differences[username]["migrated_permissions"] = sorted(migrated_permissions) + + # Compare original and migrated permissions + differences_list = [] + + for username, user_data in user_differences.items(): + original_permissions = user_data["original_permissions"] + migrated_permissions = user_data["migrated_permissions"] + + # Find differences in permissions + original_set = set(original_permissions) + migrated_set = set(migrated_permissions) + + added_permissions = migrated_set.difference(original_set) + removed_permissions = original_set.difference(migrated_set) + + if added_permissions or removed_permissions: + differences = { + "username": username, + "email": user_data["email"], + "added_permissions": sorted(list(added_permissions)), + "removed_permissions": sorted(list(removed_permissions)) + } + differences_list.append(differences) + + return differences_list + +def main(): + """ + Main function to run the script. + + This function parses command-line arguments to either export IAM policies + or generate permission differences for a specified GCP project. + """ + parser = argparse.ArgumentParser( + description="Export IAM policies or generate permission differences for a GCP project." + ) + parser.add_argument( + "project_id", + help="The Google Cloud project ID." + ) + parser.add_argument( + "--difference", + dest="user_email", + metavar="USER_EMAIL", + help="Generate permission differences for the specified user email." + ) + + args = parser.parse_args() + + project_id = args.project_id + user_email = args.user_email + + if user_email: + # If the iam policy has not been generated yet, it will generate the original IAM policy first. + if not os.path.exists(f"{project_id}.original-roles.yaml") or not os.path.exists(f"{project_id}.migrated-roles.yaml"): + print(f"Original IAM policy for project {project_id} not found. Generating original and migrated roles first.") + + print(f"Exporting IAM policy for project {project_id}...") + iam_data = export_project_iam(project_id) + + original_filename = f"{project_id}.original-roles.yaml" + original_header = f"Exported original IAM policy for project {project_id}" + to_yaml_file(iam_data, original_filename, header_info=original_header) + + print("Migrating permissions to new roles...") + migrated_data = migrate_permissions(iam_data) + migrated_filename = f"{project_id}.migrated-roles.yaml" + migrated_header = f"Migrated IAM policy for project {project_id} to new beam_roles" + to_yaml_file(migrated_data, migrated_filename, header_info=migrated_header) + + print(f"Generated {original_filename} and {migrated_filename}") + + print(f"Generating permission differences for {user_email} in project {project_id}...") + differences = permission_differences(project_id, user_email) + if differences: + output_filename = f"{project_id}.permission-differences.yaml" + header = f"Permission differences for user {user_email} in project {project_id}" + to_yaml_file(differences, output_filename, header_info=header) + print(f"Generated {output_filename}") + else: + print(f"No permission differences found for user {user_email} in project {project_id}.") + else: + print(f"Exporting IAM policy for project {project_id}...") + iam_data = export_project_iam(project_id) + + original_filename = f"{project_id}.original-roles.yaml" + original_header = f"Exported original IAM policy for project {project_id}" + to_yaml_file(iam_data, original_filename, header_info=original_header) + + print("Migrating permissions to new roles...") + migrated_data = migrate_permissions(iam_data) + migrated_filename = f"{project_id}.migrated-roles.yaml" + migrated_header = f"Migrated IAM policy for project {project_id} to new beam_roles" + to_yaml_file(migrated_data, migrated_filename, header_info=migrated_header) + + print(f"Generated {original_filename} and {migrated_filename}") + print(f"To generate permission differences, run: python {sys.argv[0]} {project_id} --difference ") + + +if __name__ == "__main__": + main() \ No newline at end of file diff --git a/infra/iam/roles/README.md b/infra/iam/roles/README.md new file mode 100644 index 000000000000..94b04b8f27b5 --- /dev/null +++ b/infra/iam/roles/README.md @@ -0,0 +1,75 @@ + + +# Beam custom roles + +This document describes the custom roles defined for the Beam project and their associated permissions. + + +## Roles + +The following files are used to define and manage roles: + +- `roles_config.yaml`: A YAML file that defines the roles and their associated services. +- `generate_roles.py`: A Python script that generates yaml files for the roles. +- `roles.tf`: A Terraform file that applies that generate the roles described over the custom roles created. + +### Defined roles + +The roles are defined in the `roles_config.yaml` file. Each role includes a name, description, and a list of services associated with it. + +The defined roles are: + +- `beam_viewer`: Read-only access to the Beam project. Excludes secret management permissions. +- `beam_writer`: User access to the the resources in the Beam project. +- `beam_infra_manager`: Editor access to the Beam project, excluding destructive permissions. +- `beam_admin`: Full access to the Beam project, including destructive capabilities and secret management. + +Roles are structured in a hierarchy, allowing for inheritance of permissions. Each role builds upon the previous one. The hierarchy is as follows: + +```plaintext +beam_viewer < beam_writer < beam_infra_manager < beam_admin +``` + +### Modifying Roles services + +Each role can have its associated base roles and services. The `roles_config.yaml` file defines the services associated with each role. For example, the `beam_viewer` role has read-only access to the project, while the `beam_infra_manager` role has editor access but excludes destructive permissions. + +To modify the services associated with a role, edit the `roles_config.yaml` file and update the relevant service and roles lists under each role. After making changes, re-run the `generate_roles.py` script to apply the updates. + +The `generate_roles.py` script, install the dependencies using: + +```bash +pip install -r requirements.txt +``` + +After modifying the `roles_config.yaml` file, run the script to generate the yaml files for the roles: + +```bash +python3 generate_roles.py +``` + +This will update the `beam_roles` directory with the new role definitions. You do not need any GCP permissions to run this script, as it only generates local files. + +To apply the changes to the GCP project, ensure you have a owner role in the GCP project, go to the main `infra/iam` directory and run the following Terraform commands: + +```bash +terraform plan +terraform apply +``` diff --git a/infra/iam/roles/beam_admin.role.yaml b/infra/iam/roles/beam_admin.role.yaml new file mode 100644 index 000000000000..4296196c495e --- /dev/null +++ b/infra/iam/roles/beam_admin.role.yaml @@ -0,0 +1,674 @@ +# 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. + +# This file is auto-generated by generate_roles.py. +# Do not edit manually. + +# This file was generated on 2025-08-11 14:34:54 UTC + +description: This is the beam_admin role +permissions: +- artifactregistry.attachments.delete +- artifactregistry.files.delete +- artifactregistry.packages.delete +- artifactregistry.repositories.createTagBinding +- artifactregistry.repositories.delete +- artifactregistry.repositories.deleteTagBinding +- artifactregistry.repositories.setIamPolicy +- artifactregistry.rules.delete +- artifactregistry.tags.delete +- artifactregistry.versions.delete +- biglake.catalogs.delete +- biglake.catalogs.setIamPolicy +- biglake.databases.delete +- biglake.locks.delete +- biglake.namespaces.delete +- biglake.namespaces.setIamPolicy +- biglake.tables.delete +- biglake.tables.setIamPolicy +- bigquery.capacityCommitments.create +- bigquery.capacityCommitments.delete +- bigquery.connections.delegate +- bigquery.connections.delete +- bigquery.connections.setIamPolicy +- bigquery.dataPolicies.delete +- bigquery.dataPolicies.setIamPolicy +- bigquery.datasets.createTagBinding +- bigquery.datasets.delete +- bigquery.datasets.deleteTagBinding +- bigquery.datasets.link +- bigquery.datasets.listSharedDatasetUsage +- bigquery.datasets.setIamPolicy +- bigquery.datasets.update +- bigquery.jobs.delete +- bigquery.jobs.listAll +- bigquery.jobs.update +- bigquery.models.delete +- bigquery.reservationAssignments.delete +- bigquery.reservationGroups.delete +- bigquery.reservations.delete +- bigquery.routines.delete +- bigquery.rowAccessPolicies.delete +- bigquery.rowAccessPolicies.setIamPolicy +- bigquery.savedqueries.delete +- bigquery.tables.create +- bigquery.tables.createTagBinding +- bigquery.tables.delete +- bigquery.tables.deleteSnapshot +- bigquery.tables.deleteTagBinding +- bigquery.tables.setCategory +- bigquery.tables.setIamPolicy +- bigquery.tables.update +- bigquery.tables.updateData +- bigquery.tables.updateTag +- bigquerymigration.workflows.delete +- cloudasset.feeds.list +- cloudasset.othercloudconnections.delete +- cloudasset.savedqueries.delete +- cloudbuild.connections.delete +- cloudbuild.connections.setIamPolicy +- cloudbuild.integrations.delete +- cloudbuild.repositories.delete +- cloudbuild.workerpools.delete +- cloudfunctions.functions.delete +- cloudfunctions.functions.setIamPolicy +- cloudkms.cryptoKeys.setIamPolicy +- cloudkms.ekmConfigs.setIamPolicy +- cloudkms.ekmConnections.setIamPolicy +- cloudkms.importJobs.setIamPolicy +- cloudkms.keyRings.setIamPolicy +- cloudsql.backupRuns.delete +- cloudsql.databases.delete +- cloudsql.instances.delete +- cloudsql.sslCerts.delete +- cloudsql.users.delete +- compute.addresses.createTagBinding +- compute.addresses.delete +- compute.addresses.deleteTagBinding +- compute.advice.calendarMode +- compute.autoscalers.delete +- compute.backendBuckets.createTagBinding +- compute.backendBuckets.delete +- compute.backendBuckets.deleteTagBinding +- compute.backendBuckets.setIamPolicy +- compute.backendServices.createTagBinding +- compute.backendServices.delete +- compute.backendServices.deleteTagBinding +- compute.backendServices.setIamPolicy +- compute.crossSiteNetworks.delete +- compute.disks.createTagBinding +- compute.disks.delete +- compute.disks.deleteTagBinding +- compute.disks.setIamPolicy +- compute.externalVpnGateways.createTagBinding +- compute.externalVpnGateways.delete +- compute.externalVpnGateways.deleteTagBinding +- compute.firewallPolicies.createTagBinding +- compute.firewallPolicies.delete +- compute.firewallPolicies.deleteTagBinding +- compute.firewallPolicies.setIamPolicy +- compute.firewalls.createTagBinding +- compute.firewalls.delete +- compute.firewalls.deleteTagBinding +- compute.forwardingRules.createTagBinding +- compute.forwardingRules.delete +- compute.forwardingRules.deleteTagBinding +- compute.futureReservations.cancel +- compute.futureReservations.delete +- compute.futureReservations.setIamPolicy +- compute.globalAddresses.createTagBinding +- compute.globalAddresses.delete +- compute.globalAddresses.deleteTagBinding +- compute.globalForwardingRules.createTagBinding +- compute.globalForwardingRules.delete +- compute.globalForwardingRules.deleteTagBinding +- compute.globalNetworkEndpointGroups.createTagBinding +- compute.globalNetworkEndpointGroups.delete +- compute.globalNetworkEndpointGroups.deleteTagBinding +- compute.globalOperations.delete +- compute.globalPublicDelegatedPrefixes.delete +- compute.healthChecks.createTagBinding +- compute.healthChecks.delete +- compute.healthChecks.deleteTagBinding +- compute.httpHealthChecks.createTagBinding +- compute.httpHealthChecks.delete +- compute.httpHealthChecks.deleteTagBinding +- compute.httpsHealthChecks.createTagBinding +- compute.httpsHealthChecks.delete +- compute.httpsHealthChecks.deleteTagBinding +- compute.images.createTagBinding +- compute.images.delete +- compute.images.deleteTagBinding +- compute.instanceGroupManagers.createTagBinding +- compute.instanceGroupManagers.delete +- compute.instanceGroupManagers.deleteTagBinding +- compute.instanceGroups.createTagBinding +- compute.instanceGroups.delete +- compute.instanceGroups.deleteTagBinding +- compute.instanceTemplates.delete +- compute.instanceTemplates.setIamPolicy +- compute.instances.createTagBinding +- compute.instances.delete +- compute.instances.deleteTagBinding +- compute.instances.setIamPolicy +- compute.instances.stop +- compute.instantSnapshots.delete +- compute.instantSnapshots.setIamPolicy +- compute.interconnectAttachmentGroups.delete +- compute.interconnectAttachments.createTagBinding +- compute.interconnectAttachments.deleteTagBinding +- compute.interconnectGroups.delete +- compute.interconnects.createTagBinding +- compute.interconnects.deleteTagBinding +- compute.interconnects.getMacsecConfig +- compute.licenseCodes.setIamPolicy +- compute.licenses.setIamPolicy +- compute.machineImages.delete +- compute.machineImages.setIamPolicy +- compute.multiMig.delete +- compute.networkAttachments.createTagBinding +- compute.networkAttachments.delete +- compute.networkAttachments.deleteTagBinding +- compute.networkAttachments.setIamPolicy +- compute.networkEdgeSecurityServices.createTagBinding +- compute.networkEdgeSecurityServices.delete +- compute.networkEdgeSecurityServices.deleteTagBinding +- compute.networkEndpointGroups.createTagBinding +- compute.networkEndpointGroups.delete +- compute.networkEndpointGroups.deleteTagBinding +- compute.networks.createTagBinding +- compute.networks.delete +- compute.networks.deleteTagBinding +- compute.nodeGroups.delete +- compute.nodeGroups.setIamPolicy +- compute.nodeTemplates.delete +- compute.nodeTemplates.setIamPolicy +- compute.organizations.disableXpnHost +- compute.organizations.disableXpnResource +- compute.organizations.enableXpnHost +- compute.organizations.enableXpnResource +- compute.packetMirrorings.createTagBinding +- compute.packetMirrorings.delete +- compute.packetMirrorings.deleteTagBinding +- compute.publicAdvertisedPrefixes.delete +- compute.publicDelegatedPrefixes.createTagBinding +- compute.publicDelegatedPrefixes.delete +- compute.publicDelegatedPrefixes.deleteTagBinding +- compute.regionBackendServices.createTagBinding +- compute.regionBackendServices.delete +- compute.regionBackendServices.deleteTagBinding +- compute.regionBackendServices.setIamPolicy +- compute.regionFirewallPolicies.createTagBinding +- compute.regionFirewallPolicies.delete +- compute.regionFirewallPolicies.deleteTagBinding +- compute.regionFirewallPolicies.setIamPolicy +- compute.regionHealthCheckServices.delete +- compute.regionHealthChecks.createTagBinding +- compute.regionHealthChecks.delete +- compute.regionHealthChecks.deleteTagBinding +- compute.regionNetworkEndpointGroups.createTagBinding +- compute.regionNetworkEndpointGroups.delete +- compute.regionNetworkEndpointGroups.deleteTagBinding +- compute.regionNotificationEndpoints.delete +- compute.regionOperations.delete +- compute.regionSecurityPolicies.createTagBinding +- compute.regionSecurityPolicies.delete +- compute.regionSecurityPolicies.deleteTagBinding +- compute.regionSslCertificates.createTagBinding +- compute.regionSslCertificates.delete +- compute.regionSslCertificates.deleteTagBinding +- compute.regionSslPolicies.createTagBinding +- compute.regionSslPolicies.delete +- compute.regionSslPolicies.deleteTagBinding +- compute.regionTargetHttpProxies.createTagBinding +- compute.regionTargetHttpProxies.delete +- compute.regionTargetHttpProxies.deleteTagBinding +- compute.regionTargetHttpsProxies.createTagBinding +- compute.regionTargetHttpsProxies.delete +- compute.regionTargetHttpsProxies.deleteTagBinding +- compute.regionTargetTcpProxies.createTagBinding +- compute.regionTargetTcpProxies.delete +- compute.regionTargetTcpProxies.deleteTagBinding +- compute.regionUrlMaps.createTagBinding +- compute.regionUrlMaps.delete +- compute.regionUrlMaps.deleteTagBinding +- compute.reservations.delete +- compute.resourcePolicies.delete +- compute.resourcePolicies.setIamPolicy +- compute.routers.createTagBinding +- compute.routers.delete +- compute.routers.deleteTagBinding +- compute.routes.createTagBinding +- compute.routes.delete +- compute.routes.deleteTagBinding +- compute.securityPolicies.createTagBinding +- compute.securityPolicies.deleteTagBinding +- compute.serviceAttachments.createTagBinding +- compute.serviceAttachments.delete +- compute.serviceAttachments.deleteTagBinding +- compute.serviceAttachments.setIamPolicy +- compute.snapshots.createTagBinding +- compute.snapshots.delete +- compute.snapshots.deleteTagBinding +- compute.snapshots.setIamPolicy +- compute.sslCertificates.createTagBinding +- compute.sslCertificates.delete +- compute.sslCertificates.deleteTagBinding +- compute.sslPolicies.createTagBinding +- compute.sslPolicies.deleteTagBinding +- compute.storagePools.delete +- compute.storagePools.setIamPolicy +- compute.subnetworks.createTagBinding +- compute.subnetworks.delete +- compute.subnetworks.deleteTagBinding +- compute.subnetworks.setIamPolicy +- compute.targetGrpcProxies.createTagBinding +- compute.targetGrpcProxies.delete +- compute.targetGrpcProxies.deleteTagBinding +- compute.targetHttpProxies.createTagBinding +- compute.targetHttpProxies.delete +- compute.targetHttpProxies.deleteTagBinding +- compute.targetHttpsProxies.createTagBinding +- compute.targetHttpsProxies.delete +- compute.targetHttpsProxies.deleteTagBinding +- compute.targetInstances.createTagBinding +- compute.targetInstances.delete +- compute.targetInstances.deleteTagBinding +- compute.targetPools.createTagBinding +- compute.targetPools.delete +- compute.targetPools.deleteTagBinding +- compute.targetSslProxies.createTagBinding +- compute.targetSslProxies.delete +- compute.targetSslProxies.deleteTagBinding +- compute.targetTcpProxies.createTagBinding +- compute.targetTcpProxies.delete +- compute.targetTcpProxies.deleteTagBinding +- compute.targetVpnGateways.createTagBinding +- compute.targetVpnGateways.delete +- compute.targetVpnGateways.deleteTagBinding +- compute.urlMaps.createTagBinding +- compute.urlMaps.deleteTagBinding +- compute.vpnGateways.createTagBinding +- compute.vpnGateways.delete +- compute.vpnGateways.deleteTagBinding +- compute.vpnTunnels.createTagBinding +- compute.vpnTunnels.delete +- compute.vpnTunnels.deleteTagBinding +- compute.wireGroups.delete +- compute.zoneOperations.delete +- container.apiServices.delete +- container.auditSinks.delete +- container.backendConfigs.delete +- container.certificateSigningRequests.approve +- container.certificateSigningRequests.delete +- container.clusterRoleBindings.create +- container.clusterRoleBindings.delete +- container.clusterRoleBindings.update +- container.clusterRoles.bind +- container.clusterRoles.create +- container.clusterRoles.delete +- container.clusterRoles.update +- container.clusters.createTagBinding +- container.clusters.delete +- container.clusters.deleteTagBinding +- container.configMaps.delete +- container.controllerRevisions.delete +- container.cronJobs.delete +- container.csiDrivers.delete +- container.csiNodeInfos.delete +- container.csiNodes.delete +- container.customResourceDefinitions.delete +- container.daemonSets.delete +- container.deployments.delete +- container.endpointSlices.delete +- container.endpoints.delete +- container.events.delete +- container.frontendConfigs.delete +- container.horizontalPodAutoscalers.delete +- container.hostServiceAgent.use +- container.ingresses.delete +- container.jobs.delete +- container.leases.delete +- container.limitRanges.delete +- container.managedCertificates.delete +- container.mutatingWebhookConfigurations.delete +- container.namespaces.delete +- container.networkPolicies.delete +- container.nodes.delete +- container.persistentVolumeClaims.delete +- container.persistentVolumes.delete +- container.podDisruptionBudgets.delete +- container.podSecurityPolicies.delete +- container.podTemplates.delete +- container.pods.delete +- container.priorityClasses.delete +- container.replicaSets.delete +- container.replicationControllers.delete +- container.resourceQuotas.delete +- container.roleBindings.create +- container.roleBindings.delete +- container.roleBindings.update +- container.roles.bind +- container.roles.create +- container.roles.delete +- container.roles.update +- container.runtimeClasses.delete +- container.secrets.delete +- container.serviceAccounts.delete +- container.services.delete +- container.statefulSets.delete +- container.storageClasses.delete +- container.storageStates.delete +- container.storageVersionMigrations.delete +- container.thirdPartyObjects.delete +- container.updateInfos.delete +- container.validatingWebhookConfigurations.delete +- container.volumeAttachments.delete +- container.volumeSnapshotClasses.delete +- container.volumeSnapshotContents.delete +- container.volumeSnapshots.delete +- containeranalysis.notes.delete +- containeranalysis.notes.setIamPolicy +- containeranalysis.occurrences.delete +- containeranalysis.occurrences.setIamPolicy +- dataflow.jobs.cancel +- dataflow.snapshots.delete +- dataform.commentThreads.delete +- dataform.comments.delete +- dataform.releaseConfigs.delete +- dataform.repositories.delete +- dataform.repositories.setIamPolicy +- dataform.workflowConfigs.delete +- dataform.workflowInvocations.cancel +- dataform.workflowInvocations.delete +- dataform.workspaces.delete +- dataform.workspaces.setIamPolicy +- dataplex.aspectTypes.delete +- dataplex.aspectTypes.setIamPolicy +- dataplex.assets.delete +- dataplex.assets.setIamPolicy +- dataplex.content.delete +- dataplex.content.setIamPolicy +- dataplex.dataAttributeBindings.delete +- dataplex.dataAttributeBindings.setIamPolicy +- dataplex.dataAttributes.delete +- dataplex.dataAttributes.setIamPolicy +- dataplex.dataTaxonomies.delete +- dataplex.dataTaxonomies.setIamPolicy +- dataplex.datascans.delete +- dataplex.datascans.setIamPolicy +- dataplex.entities.delete +- dataplex.entries.delete +- dataplex.entryGroups.delete +- dataplex.entryGroups.setIamPolicy +- dataplex.entryLinks.delete +- dataplex.entryTypes.delete +- dataplex.entryTypes.setIamPolicy +- dataplex.environments.delete +- dataplex.environments.setIamPolicy +- dataplex.glossaries.delete +- dataplex.glossaries.setIamPolicy +- dataplex.glossaryCategories.delete +- dataplex.glossaryTerms.delete +- dataplex.lakes.delete +- dataplex.lakes.setIamPolicy +- dataplex.metadataJobs.cancel +- dataplex.operations.cancel +- dataplex.operations.delete +- dataplex.partitions.delete +- dataplex.tasks.cancel +- dataplex.tasks.delete +- dataplex.tasks.setIamPolicy +- dataplex.zones.delete +- dataplex.zones.setIamPolicy +- dataproc.agents.delete +- dataproc.autoscalingPolicies.delete +- dataproc.autoscalingPolicies.setIamPolicy +- dataproc.batches.cancel +- dataproc.batches.delete +- dataproc.clusters.delete +- dataproc.clusters.setIamPolicy +- dataproc.clusters.stop +- dataproc.jobs.cancel +- dataproc.jobs.delete +- dataproc.jobs.setIamPolicy +- dataproc.operations.cancel +- dataproc.operations.delete +- dataproc.operations.setIamPolicy +- dataproc.sessionTemplates.delete +- dataproc.sessions.delete +- dataproc.sessions.terminate +- dataproc.workflowTemplates.delete +- dataproc.workflowTemplates.setIamPolicy +- dataprocrm.nodePools.delete +- dataprocrm.operations.cancel +- dataprocrm.operations.delete +- dataprocrm.workloads.cancel +- dataprocrm.workloads.delete +- datastore.backupSchedules.delete +- datastore.backups.delete +- datastore.backups.restoreDatabase +- datastore.databases.bulkDelete +- datastore.databases.clone +- datastore.databases.create +- datastore.databases.createTagBinding +- datastore.databases.delete +- datastore.databases.deleteTagBinding +- datastore.databases.export +- datastore.databases.import +- datastore.entities.delete +- datastore.indexes.delete +- datastore.locations.get +- datastore.locations.list +- datastore.operations.cancel +- datastore.operations.delete +- datastore.userCreds.delete +- dns.managedZones.delete +- dns.managedZones.setIamPolicy +- dns.policies.delete +- dns.resourceRecordSets.delete +- dns.responsePolicies.delete +- dns.responsePolicyRules.delete +- firebase.billingPlans.update +- firebase.clients.delete +- firebase.links.create +- firebase.links.delete +- firebase.links.update +- firebase.playLinks.update +- firebase.projects.delete +- firebaseabt.experiments.delete +- firebaseappcheck.appCheckTokens.verify +- firebaseappcheck.automations.delete +- firebaseauth.users.delete +- firebasedatabase.instances.delete +- firebasedataconnect.connectorRevisions.delete +- firebasedataconnect.connectors.delete +- firebasedataconnect.operations.cancel +- firebasedataconnect.operations.delete +- firebasedataconnect.schemaRevisions.delete +- firebasedataconnect.schemas.delete +- firebasedataconnect.services.delete +- firebasedynamiclinks.destinations.update +- firebasedynamiclinks.domains.delete +- firebaseextensions.configs.create +- firebaseextensions.configs.delete +- firebaseextensions.configs.update +- firebaseextensionspublisher.extensions.delete +- firebasehosting.sites.delete +- firebaseinappmessaging.campaigns.delete +- firebasemessagingcampaigns.campaigns.delete +- firebasemessagingcampaigns.campaigns.stop +- firebaseml.models.delete +- firebasenotifications.messages.delete +- firebaserules.releases.delete +- firebaserules.rulesets.delete +- firebasestorage.defaultBucket.delete +- iam.googleapis.com/workloadIdentityPoolProviderKeys.create +- iam.googleapis.com/workloadIdentityPoolProviderKeys.delete +- iam.googleapis.com/workloadIdentityPoolProviderKeys.undelete +- iam.googleapis.com/workloadIdentityPoolProviders.create +- iam.googleapis.com/workloadIdentityPoolProviders.delete +- iam.googleapis.com/workloadIdentityPoolProviders.undelete +- iam.googleapis.com/workloadIdentityPoolProviders.update +- iam.googleapis.com/workloadIdentityPools.create +- iam.googleapis.com/workloadIdentityPools.delete +- iam.googleapis.com/workloadIdentityPools.undelete +- iam.googleapis.com/workloadIdentityPools.update +- iam.roles.create +- iam.roles.delete +- iam.roles.undelete +- iam.roles.update +- iam.serviceAccountApiKeyBindings.delete +- iam.serviceAccountKeys.delete +- iam.serviceAccounts.createTagBinding +- iam.serviceAccounts.delete +- iam.serviceAccounts.deleteTagBinding +- iam.serviceAccounts.setIamPolicy +- iam.serviceAccounts.undelete +- iap.tunnel.getIamPolicy +- iap.tunnel.setIamPolicy +- iap.tunnelDestGroups.delete +- iap.tunnelDestGroups.getIamPolicy +- iap.tunnelDestGroups.setIamPolicy +- iap.tunnelInstances.getIamPolicy +- iap.tunnelInstances.setIamPolicy +- iap.tunnelLocations.getIamPolicy +- iap.tunnelLocations.setIamPolicy +- iap.tunnelZones.getIamPolicy +- iap.tunnelZones.setIamPolicy +- iap.web.getIamPolicy +- iap.web.setIamPolicy +- iap.webServiceVersions.getIamPolicy +- iap.webServiceVersions.setIamPolicy +- iap.webServices.getIamPolicy +- iap.webServices.setIamPolicy +- iap.webTypes.getIamPolicy +- iap.webTypes.setIamPolicy +- monitoring.alertPolicies.createTagBinding +- monitoring.alertPolicies.delete +- monitoring.alertPolicies.deleteTagBinding +- monitoring.dashboards.createTagBinding +- monitoring.dashboards.delete +- monitoring.dashboards.deleteTagBinding +- monitoring.groups.delete +- monitoring.metricDescriptors.delete +- monitoring.metricsScopes.link +- monitoring.services.delete +- monitoring.slos.delete +- monitoring.uptimeCheckConfigs.delete +- pubsub.schemas.delete +- pubsub.schemas.setIamPolicy +- pubsub.snapshots.delete +- pubsub.subscriptions.delete +- pubsub.subscriptions.getIamPolicy +- pubsub.subscriptions.setIamPolicy +- pubsub.topics.delete +- pubsub.topics.getIamPolicy +- pubsub.topics.setIamPolicy +- pubsublite.reservations.delete +- pubsublite.subscriptions.delete +- pubsublite.topics.delete +- redis.backupCollections.delete +- redis.backups.delete +- redis.clusters.delete +- redis.instances.createTagBinding +- redis.instances.delete +- redis.instances.deleteTagBinding +- redis.operations.cancel +- redis.operations.delete +- resourcemanager.projects.setIamPolicy +- resourcemanager.tagHolds.delete +- resourcemanager.tagKeys.delete +- resourcemanager.tagKeys.setIamPolicy +- resourcemanager.tagValueBindings.delete +- resourcemanager.tagValues.delete +- resourcemanager.tagValues.setIamPolicy +- secretmanager.secrets.createTagBinding +- secretmanager.secrets.delete +- secretmanager.secrets.deleteTagBinding +- secretmanager.secrets.setIamPolicy +- secretmanager.versions.access +- secretmanager.versions.destroy +- servicemanagement.services.delete +- servicemanagement.services.getIamPolicy +- servicemanagement.services.setIamPolicy +- spanner.backupOperations.cancel +- spanner.backupSchedules.delete +- spanner.backupSchedules.setIamPolicy +- spanner.backups.delete +- spanner.backups.setIamPolicy +- spanner.databaseOperations.cancel +- spanner.databases.setIamPolicy +- spanner.instanceConfigOperations.cancel +- spanner.instanceConfigOperations.delete +- spanner.instanceConfigs.delete +- spanner.instanceOperations.cancel +- spanner.instanceOperations.delete +- spanner.instancePartitionOperations.cancel +- spanner.instancePartitionOperations.delete +- spanner.instancePartitions.delete +- spanner.instances.createTagBinding +- spanner.instances.delete +- spanner.instances.deleteTagBinding +- spanner.instances.setIamPolicy +- spanner.sessions.delete +- storage.anywhereCaches.create +- storage.anywhereCaches.disable +- storage.anywhereCaches.get +- storage.anywhereCaches.list +- storage.anywhereCaches.pause +- storage.anywhereCaches.resume +- storage.anywhereCaches.update +- storage.bucketOperations.cancel +- storage.bucketOperations.get +- storage.bucketOperations.list +- storage.buckets.createTagBinding +- storage.buckets.delete +- storage.buckets.deleteTagBinding +- storage.buckets.enableObjectRetention +- storage.buckets.get +- storage.buckets.getIamPolicy +- storage.buckets.getIpFilter +- storage.buckets.getObjectInsights +- storage.buckets.relocate +- storage.buckets.restore +- storage.buckets.setIamPolicy +- storage.buckets.setIpFilter +- storage.buckets.update +- storage.folders.delete +- storage.hmacKeys.delete +- storage.intelligenceConfigs.update +- storage.managedFolders.delete +- storage.managedFolders.getIamPolicy +- storage.managedFolders.setIamPolicy +- storage.multipartUploads.list +- storage.objects.delete +- storage.objects.getIamPolicy +- storage.objects.move +- storage.objects.overrideUnlockedRetention +- storage.objects.restore +- storage.objects.setIamPolicy +- storage.objects.setRetention +- storage.objects.update +- storageinsights.datasetConfigs.delete +- storageinsights.operations.cancel +- storageinsights.operations.delete +- storageinsights.reportConfigs.delete +- storagetransfer.agentpools.delete +- storagetransfer.jobs.delete +- storagetransfer.operations.cancel +role_id: beam_admin +stage: GA +title: beam_admin diff --git a/infra/iam/roles/beam_infra_manager.role.yaml b/infra/iam/roles/beam_infra_manager.role.yaml new file mode 100644 index 000000000000..169bebd7fbc3 --- /dev/null +++ b/infra/iam/roles/beam_infra_manager.role.yaml @@ -0,0 +1,848 @@ +# 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. + +# This file is auto-generated by generate_roles.py. +# Do not edit manually. + +# This file was generated on 2025-08-11 14:34:54 UTC + +description: This is the beam_infra_manager role +permissions: +- artifactregistry.aptartifacts.create +- artifactregistry.attachments.create +- artifactregistry.files.update +- artifactregistry.files.upload +- artifactregistry.kfpartifacts.create +- artifactregistry.packages.update +- artifactregistry.projectsettings.update +- artifactregistry.repositories.create +- artifactregistry.repositories.createOnPush +- artifactregistry.repositories.deleteArtifacts +- artifactregistry.repositories.update +- artifactregistry.repositories.uploadArtifacts +- artifactregistry.rules.create +- artifactregistry.rules.update +- artifactregistry.tags.create +- artifactregistry.tags.update +- artifactregistry.versions.update +- artifactregistry.yumartifacts.create +- biglake.catalogs.create +- biglake.databases.create +- biglake.databases.update +- biglake.locks.check +- biglake.locks.create +- biglake.namespaces.create +- biglake.namespaces.update +- biglake.tables.create +- biglake.tables.lock +- biglake.tables.update +- biglake.tables.updateData +- bigquery.bireservations.update +- bigquery.capacityCommitments.update +- bigquery.config.update +- bigquery.connections.create +- bigquery.connections.update +- bigquery.connections.updateTag +- bigquery.dataPolicies.create +- bigquery.dataPolicies.update +- bigquery.datasets.updateTag +- bigquery.models.create +- bigquery.models.updateData +- bigquery.models.updateMetadata +- bigquery.models.updateTag +- bigquery.objectRefs.write +- bigquery.reservationAssignments.create +- bigquery.reservationGroups.create +- bigquery.reservations.create +- bigquery.reservations.update +- bigquery.routines.create +- bigquery.routines.update +- bigquery.routines.updateTag +- bigquery.rowAccessPolicies.create +- bigquery.rowAccessPolicies.update +- bigquery.savedqueries.create +- bigquery.savedqueries.update +- bigquery.tables.createIndex +- bigquery.tables.deleteIndex +- bigquery.tables.restoreSnapshot +- bigquery.tables.updateIndex +- bigquery.transfers.update +- bigquerymigration.workflows.create +- bigquerymigration.workflows.enableAiOutputTypes +- bigquerymigration.workflows.enableLineageOutputTypes +- bigquerymigration.workflows.enableOutputTypePermissions +- bigquerymigration.workflows.update +- cloudasset.othercloudconnections.create +- cloudasset.othercloudconnections.update +- cloudasset.othercloudconnections.verify +- cloudasset.savedqueries.create +- cloudasset.savedqueries.update +- cloudbuild.builds.approve +- cloudbuild.builds.create +- cloudbuild.builds.update +- cloudbuild.connections.create +- cloudbuild.connections.update +- cloudbuild.integrations.create +- cloudbuild.integrations.update +- cloudbuild.repositories.create +- cloudbuild.workerpools.create +- cloudbuild.workerpools.update +- cloudbuild.workerpools.use +- cloudfunctions.functions.call +- cloudfunctions.functions.create +- cloudfunctions.functions.generationUpgrade +- cloudfunctions.functions.invoke +- cloudfunctions.functions.sourceCodeSet +- cloudfunctions.functions.update +- cloudkms.cryptoKeyVersions.create +- cloudkms.cryptoKeyVersions.update +- cloudkms.cryptoKeys.create +- cloudkms.cryptoKeys.update +- cloudkms.ekmConfigs.update +- cloudkms.ekmConnections.create +- cloudkms.ekmConnections.update +- cloudkms.ekmConnections.use +- cloudkms.importJobs.create +- cloudkms.importJobs.useToImport +- cloudkms.kajPolicyConfigs.update +- cloudkms.keyRings.create +- cloudsql.backupRuns.create +- cloudsql.backupRuns.update +- cloudsql.databases.create +- cloudsql.databases.update +- cloudsql.instances.addServerCa +- cloudsql.instances.addServerCertificate +- cloudsql.instances.clone +- cloudsql.instances.connect +- cloudsql.instances.create +- cloudsql.instances.demoteMaster +- cloudsql.instances.executeSql +- cloudsql.instances.failover +- cloudsql.instances.import +- cloudsql.instances.migrate +- cloudsql.instances.performDiskShrink +- cloudsql.instances.promoteReplica +- cloudsql.instances.reencrypt +- cloudsql.instances.resetReplicaSize +- cloudsql.instances.resetSslConfig +- cloudsql.instances.restart +- cloudsql.instances.restoreBackup +- cloudsql.instances.rotateServerCa +- cloudsql.instances.rotateServerCertificate +- cloudsql.instances.startReplica +- cloudsql.instances.stopReplica +- cloudsql.instances.truncateLog +- cloudsql.instances.update +- cloudsql.instances.updateBackupDrConfig +- cloudsql.sslCerts.create +- cloudsql.users.create +- cloudsql.users.update +- compute.addresses.create +- compute.addresses.use +- compute.autoscalers.create +- compute.autoscalers.update +- compute.backendBuckets.addSignedUrlKey +- compute.backendBuckets.create +- compute.backendBuckets.deleteSignedUrlKey +- compute.backendBuckets.setSecurityPolicy +- compute.backendBuckets.update +- compute.backendBuckets.use +- compute.backendServices.addSignedUrlKey +- compute.backendServices.create +- compute.backendServices.deleteSignedUrlKey +- compute.backendServices.setSecurityPolicy +- compute.backendServices.update +- compute.backendServices.use +- compute.commitments.create +- compute.commitments.update +- compute.commitments.updateReservations +- compute.crossSiteNetworks.create +- compute.crossSiteNetworks.update +- compute.diskSettings.update +- compute.disks.addResourcePolicies +- compute.disks.create +- compute.disks.removeResourcePolicies +- compute.disks.resize +- compute.disks.setLabels +- compute.disks.startAsyncReplication +- compute.disks.stopAsyncReplication +- compute.disks.stopGroupAsyncReplication +- compute.disks.update +- compute.disks.use +- compute.externalVpnGateways.create +- compute.externalVpnGateways.setLabels +- compute.externalVpnGateways.use +- compute.firewallPolicies.cloneRules +- compute.firewallPolicies.create +- compute.firewallPolicies.update +- compute.firewallPolicies.use +- compute.firewalls.create +- compute.firewalls.update +- compute.forwardingRules.create +- compute.forwardingRules.pscCreate +- compute.forwardingRules.pscDelete +- compute.forwardingRules.pscSetLabels +- compute.forwardingRules.pscUpdate +- compute.forwardingRules.setTarget +- compute.forwardingRules.update +- compute.forwardingRules.use +- compute.futureReservations.create +- compute.futureReservations.update +- compute.globalAddresses.create +- compute.globalAddresses.createInternal +- compute.globalAddresses.deleteInternal +- compute.globalAddresses.use +- compute.globalForwardingRules.create +- compute.globalForwardingRules.pscCreate +- compute.globalForwardingRules.pscDelete +- compute.globalForwardingRules.pscSetLabels +- compute.globalForwardingRules.pscUpdate +- compute.globalForwardingRules.update +- compute.globalNetworkEndpointGroups.attachNetworkEndpoints +- compute.globalNetworkEndpointGroups.create +- compute.globalNetworkEndpointGroups.detachNetworkEndpoints +- compute.globalNetworkEndpointGroups.use +- compute.globalPublicDelegatedPrefixes.create +- compute.globalPublicDelegatedPrefixes.updatePolicy +- compute.healthChecks.create +- compute.healthChecks.update +- compute.healthChecks.use +- compute.httpHealthChecks.create +- compute.httpHealthChecks.update +- compute.httpsHealthChecks.create +- compute.httpsHealthChecks.update +- compute.images.create +- compute.images.deprecate +- compute.images.setLabels +- compute.images.update +- compute.instanceGroupManagers.create +- compute.instanceGroupManagers.update +- compute.instanceGroupManagers.use +- compute.instanceGroups.create +- compute.instanceGroups.update +- compute.instanceGroups.use +- compute.instanceSettings.update +- compute.instanceTemplates.create +- compute.instances.addAccessConfig +- compute.instances.addNetworkInterface +- compute.instances.addResourcePolicies +- compute.instances.attachDisk +- compute.instances.create +- compute.instances.deleteAccessConfig +- compute.instances.deleteNetworkInterface +- compute.instances.detachDisk +- compute.instances.osAdminLogin +- compute.instances.osLogin +- compute.instances.pscInterfaceCreate +- compute.instances.removeResourcePolicies +- compute.instances.reset +- compute.instances.resume +- compute.instances.sendDiagnosticInterrupt +- compute.instances.setDiskAutoDelete +- compute.instances.setLabels +- compute.instances.setMachineResources +- compute.instances.setMachineType +- compute.instances.setMetadata +- compute.instances.setMinCpuPlatform +- compute.instances.setName +- compute.instances.setScheduling +- compute.instances.setSecurityPolicy +- compute.instances.setServiceAccount +- compute.instances.setShieldedInstanceIntegrityPolicy +- compute.instances.setShieldedVmIntegrityPolicy +- compute.instances.setTags +- compute.instances.simulateMaintenanceEvent +- compute.instances.start +- compute.instances.startWithEncryptionKey +- compute.instances.suspend +- compute.instances.update +- compute.instances.updateAccessConfig +- compute.instances.updateDisplayDevice +- compute.instances.updateNetworkInterface +- compute.instances.updateSecurity +- compute.instances.updateShieldedInstanceConfig +- compute.instances.updateShieldedVmConfig +- compute.instances.use +- compute.instantSnapshots.create +- compute.instantSnapshots.export +- compute.instantSnapshots.setLabels +- compute.interconnectAttachmentGroups.create +- compute.interconnectAttachmentGroups.patch +- compute.interconnectGroups.create +- compute.interconnectGroups.patch +- compute.licenses.update +- compute.machineImages.create +- compute.machineImages.setLabels +- compute.multiMig.create +- compute.networkAttachments.create +- compute.networkAttachments.update +- compute.networkAttachments.use +- compute.networkEdgeSecurityServices.create +- compute.networkEdgeSecurityServices.update +- compute.networkEndpointGroups.attachNetworkEndpoints +- compute.networkEndpointGroups.create +- compute.networkEndpointGroups.detachNetworkEndpoints +- compute.networkEndpointGroups.use +- compute.networks.access +- compute.networks.create +- compute.networks.mirror +- compute.networks.setFirewallPolicy +- compute.networks.updatePeering +- compute.networks.updatePolicy +- compute.networks.use +- compute.networks.useExternalIp +- compute.nodeGroups.addNodes +- compute.nodeGroups.create +- compute.nodeGroups.deleteNodes +- compute.nodeGroups.performMaintenance +- compute.nodeGroups.setNodeTemplate +- compute.nodeGroups.simulateMaintenanceEvent +- compute.nodeGroups.update +- compute.nodeTemplates.create +- compute.organizations.setFirewallPolicy +- compute.organizations.setSecurityPolicy +- compute.packetMirrorings.create +- compute.packetMirrorings.update +- compute.previewFeatures.update +- compute.projects.setCloudArmorTier +- compute.projects.setCommonInstanceMetadata +- compute.projects.setManagedProtectionTier +- compute.projects.setUsageExportBucket +- compute.publicAdvertisedPrefixes.create +- compute.publicAdvertisedPrefixes.update +- compute.publicAdvertisedPrefixes.updatePolicy +- compute.publicDelegatedPrefixes.create +- compute.publicDelegatedPrefixes.update +- compute.publicDelegatedPrefixes.updatePolicy +- compute.publicDelegatedPrefixes.use +- compute.regionBackendServices.create +- compute.regionBackendServices.setSecurityPolicy +- compute.regionBackendServices.update +- compute.regionBackendServices.use +- compute.regionFirewallPolicies.cloneRules +- compute.regionFirewallPolicies.create +- compute.regionFirewallPolicies.update +- compute.regionFirewallPolicies.use +- compute.regionHealthCheckServices.create +- compute.regionHealthCheckServices.update +- compute.regionHealthCheckServices.use +- compute.regionHealthChecks.create +- compute.regionHealthChecks.update +- compute.regionHealthChecks.use +- compute.regionNetworkEndpointGroups.attachNetworkEndpoints +- compute.regionNetworkEndpointGroups.create +- compute.regionNetworkEndpointGroups.detachNetworkEndpoints +- compute.regionNetworkEndpointGroups.use +- compute.regionNotificationEndpoints.create +- compute.regionNotificationEndpoints.update +- compute.regionNotificationEndpoints.use +- compute.regionSecurityPolicies.create +- compute.regionSecurityPolicies.update +- compute.regionSecurityPolicies.use +- compute.regionSslCertificates.create +- compute.regionSslPolicies.create +- compute.regionSslPolicies.update +- compute.regionSslPolicies.use +- compute.regionTargetHttpProxies.create +- compute.regionTargetHttpProxies.setUrlMap +- compute.regionTargetHttpProxies.use +- compute.regionTargetHttpsProxies.create +- compute.regionTargetHttpsProxies.setSslCertificates +- compute.regionTargetHttpsProxies.setUrlMap +- compute.regionTargetHttpsProxies.update +- compute.regionTargetHttpsProxies.use +- compute.regionTargetTcpProxies.create +- compute.regionTargetTcpProxies.use +- compute.regionUrlMaps.create +- compute.regionUrlMaps.invalidateCache +- compute.regionUrlMaps.update +- compute.regionUrlMaps.use +- compute.reservationBlocks.performMaintenance +- compute.reservationSubBlocks.performMaintenance +- compute.reservations.create +- compute.reservations.performMaintenance +- compute.reservations.resize +- compute.reservations.update +- compute.resourcePolicies.create +- compute.resourcePolicies.update +- compute.resourcePolicies.use +- compute.routers.create +- compute.routers.deleteRoutePolicy +- compute.routers.update +- compute.routers.updateRoutePolicy +- compute.routers.use +- compute.routes.create +- compute.securityPolicies.setLabels +- compute.serviceAttachments.create +- compute.serviceAttachments.update +- compute.serviceAttachments.use +- compute.snapshotSettings.update +- compute.snapshots.create +- compute.snapshots.setLabels +- compute.sslCertificates.create +- compute.storagePools.create +- compute.storagePools.update +- compute.storagePools.use +- compute.subnetworks.create +- compute.subnetworks.expandIpCidrRange +- compute.subnetworks.mirror +- compute.subnetworks.setPrivateIpGoogleAccess +- compute.subnetworks.update +- compute.subnetworks.use +- compute.subnetworks.useExternalIp +- compute.subnetworks.usePeerMigration +- compute.targetGrpcProxies.create +- compute.targetGrpcProxies.update +- compute.targetGrpcProxies.use +- compute.targetHttpProxies.create +- compute.targetHttpProxies.setUrlMap +- compute.targetHttpProxies.update +- compute.targetHttpProxies.use +- compute.targetHttpsProxies.create +- compute.targetHttpsProxies.setCertificateMap +- compute.targetHttpsProxies.setQuicOverride +- compute.targetHttpsProxies.setSslCertificates +- compute.targetHttpsProxies.setUrlMap +- compute.targetHttpsProxies.update +- compute.targetHttpsProxies.use +- compute.targetInstances.create +- compute.targetInstances.setSecurityPolicy +- compute.targetInstances.use +- compute.targetPools.addHealthCheck +- compute.targetPools.addInstance +- compute.targetPools.create +- compute.targetPools.removeHealthCheck +- compute.targetPools.removeInstance +- compute.targetPools.setSecurityPolicy +- compute.targetPools.update +- compute.targetPools.use +- compute.targetSslProxies.create +- compute.targetSslProxies.setBackendService +- compute.targetSslProxies.setCertificateMap +- compute.targetSslProxies.setProxyHeader +- compute.targetSslProxies.setSslCertificates +- compute.targetSslProxies.setSslPolicy +- compute.targetSslProxies.update +- compute.targetSslProxies.use +- compute.targetTcpProxies.create +- compute.targetTcpProxies.update +- compute.targetTcpProxies.use +- compute.targetVpnGateways.create +- compute.targetVpnGateways.use +- compute.vpnGateways.create +- compute.vpnGateways.setLabels +- compute.vpnGateways.use +- compute.vpnTunnels.create +- compute.wireGroups.create +- compute.wireGroups.update +- container.clusters.create +- container.clusters.getCredentials +- container.clusters.update +- container.controllerRevisions.create +- container.controllerRevisions.update +- container.mutatingWebhookConfigurations.create +- container.mutatingWebhookConfigurations.update +- container.podSecurityPolicies.create +- container.podSecurityPolicies.update +- container.validatingWebhookConfigurations.create +- container.validatingWebhookConfigurations.update +- containeranalysis.notes.attachOccurrence +- containeranalysis.notes.create +- containeranalysis.notes.listOccurrences +- containeranalysis.notes.update +- containeranalysis.occurrences.create +- containeranalysis.occurrences.update +- containersecurity.clusterSummaries.list +- containersecurity.findings.list +- dataflow.jobs.create +- dataflow.jobs.snapshot +- dataflow.jobs.updateContents +- dataflow.shuffle.read +- dataflow.shuffle.write +- dataflow.streamingWorkItems.ImportState +- dataflow.streamingWorkItems.commitWork +- dataflow.streamingWorkItems.getData +- dataflow.streamingWorkItems.getWork +- dataflow.streamingWorkItems.getWorkerMetadata +- dataflow.workItems.lease +- dataflow.workItems.sendMessage +- dataflow.workItems.update +- dataform.commentThreads.create +- dataform.commentThreads.update +- dataform.comments.create +- dataform.comments.update +- dataform.compilationResults.create +- dataform.config.update +- dataform.releaseConfigs.create +- dataform.releaseConfigs.update +- dataform.repositories.commit +- dataform.repositories.update +- dataform.workflowConfigs.create +- dataform.workflowConfigs.update +- dataform.workflowInvocations.create +- dataform.workspaces.commit +- dataform.workspaces.create +- dataform.workspaces.installNpmPackages +- dataform.workspaces.makeDirectory +- dataform.workspaces.moveDirectory +- dataform.workspaces.moveFile +- dataform.workspaces.pull +- dataform.workspaces.push +- dataform.workspaces.removeDirectory +- dataform.workspaces.removeFile +- dataform.workspaces.reset +- dataform.workspaces.writeFile +- dataplex.aspectTypes.create +- dataplex.aspectTypes.update +- dataplex.aspectTypes.use +- dataplex.assets.create +- dataplex.assets.update +- dataplex.content.create +- dataplex.content.update +- dataplex.dataAttributeBindings.create +- dataplex.dataAttributeBindings.update +- dataplex.dataAttributes.bind +- dataplex.dataAttributes.create +- dataplex.dataAttributes.update +- dataplex.dataTaxonomies.configureDataAccess +- dataplex.dataTaxonomies.configureResourceAccess +- dataplex.dataTaxonomies.create +- dataplex.dataTaxonomies.update +- dataplex.datascans.create +- dataplex.datascans.run +- dataplex.datascans.update +- dataplex.entities.create +- dataplex.entities.update +- dataplex.entries.create +- dataplex.entries.link +- dataplex.entries.update +- dataplex.entryGroups.create +- dataplex.entryGroups.import +- dataplex.entryGroups.update +- dataplex.entryGroups.useContactsAspect +- dataplex.entryGroups.useDataQualityScorecardAspect +- dataplex.entryGroups.useDefinitionEntryLink +- dataplex.entryGroups.useGenericAspect +- dataplex.entryGroups.useGenericEntry +- dataplex.entryGroups.useOverviewAspect +- dataplex.entryGroups.useRelatedEntryLink +- dataplex.entryGroups.useSchemaAspect +- dataplex.entryGroups.useSynonymEntryLink +- dataplex.entryLinks.create +- dataplex.entryLinks.reference +- dataplex.entryTypes.create +- dataplex.entryTypes.update +- dataplex.entryTypes.use +- dataplex.environments.create +- dataplex.environments.execute +- dataplex.environments.update +- dataplex.glossaries.create +- dataplex.glossaries.import +- dataplex.glossaries.update +- dataplex.glossaryCategories.create +- dataplex.glossaryCategories.update +- dataplex.glossaryTerms.create +- dataplex.glossaryTerms.update +- dataplex.glossaryTerms.use +- dataplex.lakes.create +- dataplex.lakes.update +- dataplex.metadataJobs.create +- dataplex.partitions.create +- dataplex.partitions.update +- dataplex.tasks.create +- dataplex.tasks.run +- dataplex.tasks.update +- dataplex.zones.create +- dataplex.zones.update +- dataproc.agents.create +- dataproc.agents.update +- dataproc.autoscalingPolicies.create +- dataproc.autoscalingPolicies.update +- dataproc.batches.analyze +- dataproc.batches.create +- dataproc.batches.sparkApplicationWrite +- dataproc.clusters.create +- dataproc.clusters.start +- dataproc.clusters.update +- dataproc.clusters.use +- dataproc.jobs.create +- dataproc.jobs.update +- dataproc.nodeGroups.create +- dataproc.nodeGroups.update +- dataproc.sessionTemplates.create +- dataproc.sessionTemplates.update +- dataproc.sessions.create +- dataproc.sessions.sparkApplicationWrite +- dataproc.tasks.lease +- dataproc.tasks.reportStatus +- dataproc.workflowTemplates.create +- dataproc.workflowTemplates.instantiate +- dataproc.workflowTemplates.instantiateInline +- dataproc.workflowTemplates.update +- dataprocessing.datasources.update +- dataprocrm.nodePools.create +- dataprocrm.nodePools.deleteNodes +- dataprocrm.nodePools.resize +- dataprocrm.nodes.heartbeat +- dataprocrm.nodes.update +- dataprocrm.workloads.create +- datastore.backupSchedules.create +- datastore.backupSchedules.update +- datastore.databases.update +- datastore.indexes.create +- datastore.indexes.update +- datastore.userCreds.create +- datastore.userCreds.update +- dns.changes.create +- dns.gkeClusters.bindDNSResponsePolicy +- dns.gkeClusters.bindPrivateDNSZone +- dns.managedZones.create +- dns.managedZones.update +- dns.networks.bindDNSResponsePolicy +- dns.networks.bindPrivateDNSPolicy +- dns.networks.bindPrivateDNSZone +- dns.networks.targetWithPeeringZone +- dns.networks.useHealthSignals +- dns.policies.create +- dns.policies.update +- dns.resourceRecordSets.create +- dns.resourceRecordSets.update +- dns.responsePolicies.create +- dns.responsePolicies.update +- dns.responsePolicyRules.create +- dns.responsePolicyRules.update +- firebase.clients.create +- firebase.clients.undelete +- firebase.clients.update +- firebase.projects.update +- firebaseabt.experiments.create +- firebaseabt.experiments.update +- firebaseanalytics.resources.googleAnalyticsEdit +- firebaseappcheck.appAttestConfig.update +- firebaseappcheck.automations.create +- firebaseappcheck.automations.resume +- firebaseappcheck.automations.suspend +- firebaseappcheck.automations.update +- firebaseappcheck.debugTokens.update +- firebaseappcheck.deviceCheckConfig.update +- firebaseappcheck.playIntegrityConfig.update +- firebaseappcheck.recaptchaEnterpriseConfig.update +- firebaseappcheck.recaptchaV3Config.update +- firebaseappcheck.resourcePolicies.update +- firebaseappcheck.safetyNetConfig.update +- firebaseappcheck.services.update +- firebaseappdistro.groups.update +- firebaseappdistro.releases.update +- firebaseappdistro.testers.update +- firebaseauth.configs.create +- firebaseauth.configs.getHashConfig +- firebaseauth.configs.getSecret +- firebaseauth.configs.update +- firebaseauth.users.create +- firebaseauth.users.createSession +- firebaseauth.users.sendEmail +- firebaseauth.users.update +- firebasecrash.issues.update +- firebasecrashlytics.config.update +- firebasecrashlytics.issues.update +- firebasedatabase.instances.create +- firebasedatabase.instances.disable +- firebasedatabase.instances.reenable +- firebasedatabase.instances.undelete +- firebasedatabase.instances.update +- firebasedataconnect.connectors.create +- firebasedataconnect.connectors.update +- firebasedataconnect.schemas.create +- firebasedataconnect.schemas.update +- firebasedataconnect.services.create +- firebasedataconnect.services.executeGraphql +- firebasedataconnect.services.executeGraphqlRead +- firebasedataconnect.services.update +- firebasedynamiclinks.domains.create +- firebasedynamiclinks.domains.update +- firebasedynamiclinks.links.create +- firebasedynamiclinks.links.update +- firebaseextensionspublisher.extensions.create +- firebasehosting.sites.create +- firebasehosting.sites.update +- firebaseinappmessaging.campaigns.create +- firebaseinappmessaging.campaigns.update +- firebasemessagingcampaigns.campaigns.create +- firebasemessagingcampaigns.campaigns.start +- firebasemessagingcampaigns.campaigns.update +- firebaseml.models.create +- firebaseml.models.update +- firebaseml.modelversions.create +- firebaseml.modelversions.update +- firebasenotifications.messages.create +- firebasenotifications.messages.update +- firebaseperformance.config.update +- firebaserules.releases.create +- firebaserules.releases.update +- firebaserules.rulesets.create +- firebaserules.rulesets.get +- firebasestorage.buckets.addFirebase +- firebasestorage.buckets.removeFirebase +- firebasestorage.defaultBucket.create +- firebasevertexai.configs.update +- iam.serviceAccountApiKeyBindings.create +- iam.serviceAccountApiKeyBindings.undelete +- iam.serviceAccountKeys.create +- iam.serviceAccountKeys.disable +- iam.serviceAccountKeys.enable +- iam.serviceAccounts.actAs +- iam.serviceAccounts.create +- iam.serviceAccounts.disable +- iam.serviceAccounts.enable +- iam.serviceAccounts.getAccessToken +- iam.serviceAccounts.getOpenIdToken +- iam.serviceAccounts.implicitDelegation +- iam.serviceAccounts.signBlob +- iam.serviceAccounts.signJwt +- iam.serviceAccounts.update +- iap.tunnelDestGroups.create +- iap.tunnelDestGroups.update +- monitoring.alertPolicies.create +- monitoring.alertPolicies.update +- monitoring.dashboards.create +- monitoring.dashboards.update +- monitoring.groups.create +- monitoring.groups.update +- monitoring.metricDescriptors.create +- monitoring.services.create +- monitoring.services.update +- monitoring.slos.create +- monitoring.slos.update +- monitoring.snoozes.create +- monitoring.snoozes.update +- monitoring.timeSeries.create +- monitoring.uptimeCheckConfigs.create +- monitoring.uptimeCheckConfigs.update +- pubsub.schemas.commit +- pubsub.schemas.create +- pubsub.schemas.rollback +- pubsub.snapshots.create +- pubsub.subscriptions.consume +- pubsub.subscriptions.create +- pubsub.subscriptions.update +- pubsub.topics.attachSubscription +- pubsub.topics.create +- pubsub.topics.detachSubscription +- pubsub.topics.publish +- pubsub.topics.update +- pubsub.topics.updateTag +- pubsublite.reservations.attachTopic +- pubsublite.reservations.create +- pubsublite.reservations.update +- pubsublite.subscriptions.create +- pubsublite.subscriptions.seek +- pubsublite.subscriptions.setCursor +- pubsublite.subscriptions.update +- pubsublite.topics.create +- pubsublite.topics.publish +- pubsublite.topics.update +- redis.backupCollections.create +- redis.backups.create +- redis.clusters.backup +- redis.clusters.connect +- redis.clusters.create +- redis.clusters.update +- redis.instances.create +- redis.instances.export +- redis.instances.failover +- redis.instances.getAuthString +- redis.instances.import +- redis.instances.rescheduleMaintenance +- redis.instances.update +- redis.instances.updateAuth +- redis.instances.upgrade +- resourcemanager.hierarchyNodes.createTagBinding +- resourcemanager.hierarchyNodes.deleteTagBinding +- resourcemanager.projects.move +- resourcemanager.projects.update +- resourcemanager.tagHolds.create +- resourcemanager.tagKeys.create +- resourcemanager.tagKeys.update +- resourcemanager.tagValueBindings.create +- resourcemanager.tagValues.create +- resourcemanager.tagValues.update +- secretmanager.secrets.create +- secretmanager.secrets.update +- secretmanager.versions.add +- secretmanager.versions.disable +- secretmanager.versions.enable +- servicemanagement.services.bind +- servicemanagement.services.check +- servicemanagement.services.create +- servicemanagement.services.quota +- servicemanagement.services.report +- servicemanagement.services.update +- serviceusage.services.disable +- serviceusage.services.enable +- serviceusage.services.use +- spanner.backupSchedules.create +- spanner.backupSchedules.update +- spanner.backups.copy +- spanner.backups.create +- spanner.backups.restoreDatabase +- spanner.backups.update +- spanner.databases.adapt +- spanner.databases.addSplitPoints +- spanner.databases.beginOrRollbackReadWriteTransaction +- spanner.databases.beginPartitionedDmlTransaction +- spanner.databases.changequorum +- spanner.databases.create +- spanner.databases.createBackup +- spanner.databases.drop +- spanner.databases.update +- spanner.databases.updateDdl +- spanner.databases.useRoleBasedAccess +- spanner.databases.write +- spanner.instanceConfigs.create +- spanner.instanceConfigs.update +- spanner.instancePartitions.create +- spanner.instancePartitions.update +- spanner.instances.create +- spanner.instances.update +- storage.buckets.create +- storage.folders.create +- storage.folders.rename +- storage.hmacKeys.create +- storage.hmacKeys.update +- storage.managedFolders.create +- storage.managedFolders.get +- storage.managedFolders.list +- storage.multipartUploads.abort +- storage.multipartUploads.create +- storage.multipartUploads.listParts +- storage.objects.create +- storage.objects.get +- storage.objects.list +- storageinsights.datasetConfigs.create +- storageinsights.datasetConfigs.linkDataset +- storageinsights.datasetConfigs.unlinkDataset +- storageinsights.datasetConfigs.update +- storageinsights.reportConfigs.create +- storageinsights.reportConfigs.update +- storagetransfer.agentpools.create +- storagetransfer.agentpools.update +- storagetransfer.jobs.create +- storagetransfer.jobs.run +- storagetransfer.jobs.update +- storagetransfer.operations.pause +- storagetransfer.operations.resume +role_id: beam_infra_manager +stage: GA +title: beam_infra_manager diff --git a/infra/iam/roles/beam_viewer.role.yaml b/infra/iam/roles/beam_viewer.role.yaml new file mode 100644 index 000000000000..0525fda09560 --- /dev/null +++ b/infra/iam/roles/beam_viewer.role.yaml @@ -0,0 +1,1113 @@ +# 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. + +# This file is auto-generated by generate_roles.py. +# Do not edit manually. + +# This file was generated on 2025-08-11 14:34:54 UTC + +description: This is the beam_viewer role +permissions: +- artifactregistry.attachments.get +- artifactregistry.attachments.list +- artifactregistry.dockerimages.get +- artifactregistry.dockerimages.list +- artifactregistry.files.download +- artifactregistry.files.get +- artifactregistry.files.list +- artifactregistry.locations.get +- artifactregistry.locations.list +- artifactregistry.mavenartifacts.get +- artifactregistry.mavenartifacts.list +- artifactregistry.npmpackages.get +- artifactregistry.npmpackages.list +- artifactregistry.packages.get +- artifactregistry.packages.list +- artifactregistry.projectsettings.get +- artifactregistry.pythonpackages.get +- artifactregistry.pythonpackages.list +- artifactregistry.repositories.downloadArtifacts +- artifactregistry.repositories.get +- artifactregistry.repositories.getIamPolicy +- artifactregistry.repositories.list +- artifactregistry.repositories.listEffectiveTags +- artifactregistry.repositories.listTagBindings +- artifactregistry.repositories.readViaVirtualRepository +- artifactregistry.rules.get +- artifactregistry.rules.list +- artifactregistry.tags.get +- artifactregistry.tags.list +- artifactregistry.versions.get +- artifactregistry.versions.list +- biglake.catalogs.get +- biglake.catalogs.getIamPolicy +- biglake.catalogs.list +- biglake.databases.get +- biglake.databases.list +- biglake.locks.list +- biglake.namespaces.get +- biglake.namespaces.getIamPolicy +- biglake.namespaces.list +- biglake.tables.get +- biglake.tables.getData +- biglake.tables.getIamPolicy +- biglake.tables.list +- bigquery.bireservations.get +- bigquery.capacityCommitments.get +- bigquery.capacityCommitments.list +- bigquery.config.get +- bigquery.connections.get +- bigquery.connections.getIamPolicy +- bigquery.connections.list +- bigquery.connections.use +- bigquery.dataPolicies.get +- bigquery.dataPolicies.getIamPolicy +- bigquery.dataPolicies.list +- bigquery.datasets.get +- bigquery.datasets.getIamPolicy +- bigquery.datasets.listEffectiveTags +- bigquery.datasets.listTagBindings +- bigquery.jobs.create +- bigquery.jobs.get +- bigquery.jobs.list +- bigquery.jobs.listExecutionMetadata +- bigquery.models.export +- bigquery.models.getData +- bigquery.models.getMetadata +- bigquery.models.list +- bigquery.objectRefs.read +- bigquery.readsessions.create +- bigquery.readsessions.getData +- bigquery.readsessions.update +- bigquery.reservationAssignments.list +- bigquery.reservationAssignments.search +- bigquery.reservationGroups.get +- bigquery.reservationGroups.list +- bigquery.reservations.get +- bigquery.reservations.list +- bigquery.reservations.listFailoverDatasets +- bigquery.reservations.use +- bigquery.routines.get +- bigquery.routines.list +- bigquery.rowAccessPolicies.get +- bigquery.rowAccessPolicies.getIamPolicy +- bigquery.rowAccessPolicies.list +- bigquery.savedqueries.get +- bigquery.savedqueries.list +- bigquery.tables.createSnapshot +- bigquery.tables.getIamPolicy +- bigquery.tables.listEffectiveTags +- bigquery.tables.listTagBindings +- bigquery.tables.replicateData +- bigquery.transfers.get +- bigquerymigration.subtasks.get +- bigquerymigration.subtasks.list +- bigquerymigration.workflows.get +- bigquerymigration.workflows.list +- cloudasset.assets.analyzeIamPolicy +- cloudasset.assets.analyzeMove +- cloudasset.assets.analyzeOrgPolicy +- cloudasset.assets.exportAppengineApplications +- cloudasset.assets.exportAppengineServices +- cloudasset.assets.exportAppengineVersions +- cloudasset.assets.exportBigqueryDatasets +- cloudasset.assets.exportBigqueryModels +- cloudasset.assets.exportBigqueryTables +- cloudasset.assets.exportCloudDocumentAIEvaluation +- cloudasset.assets.exportCloudDocumentAIHumanReviewConfig +- cloudasset.assets.exportCloudDocumentAILabelerPool +- cloudasset.assets.exportCloudDocumentAIProcessor +- cloudasset.assets.exportCloudDocumentAIProcessorVersion +- cloudasset.assets.exportCloudbillingBillingAccounts +- cloudasset.assets.exportCloudkmsCryptoKeyVersions +- cloudasset.assets.exportCloudkmsCryptoKeys +- cloudasset.assets.exportCloudkmsKeyRings +- cloudasset.assets.exportCloudmemcacheInstances +- cloudasset.assets.exportCloudresourcemanagerFolders +- cloudasset.assets.exportCloudresourcemanagerOrganizations +- cloudasset.assets.exportCloudresourcemanagerProjects +- cloudasset.assets.exportCloudresourcemanagerTagBindings +- cloudasset.assets.exportCloudresourcemanagerTagKeys +- cloudasset.assets.exportCloudresourcemanagerTagValues +- cloudasset.assets.exportComputeAddress +- cloudasset.assets.exportComputeAutoscalers +- cloudasset.assets.exportComputeBackendBuckets +- cloudasset.assets.exportComputeBackendServices +- cloudasset.assets.exportComputeDisks +- cloudasset.assets.exportComputeFirewalls +- cloudasset.assets.exportComputeForwardingRules +- cloudasset.assets.exportComputeGlobalForwardingRules +- cloudasset.assets.exportComputeHealthChecks +- cloudasset.assets.exportComputeHttpHealthChecks +- cloudasset.assets.exportComputeHttpsHealthChecks +- cloudasset.assets.exportComputeImages +- cloudasset.assets.exportComputeInstanceGroupManagers +- cloudasset.assets.exportComputeInstanceGroups +- cloudasset.assets.exportComputeInstanceTemplates +- cloudasset.assets.exportComputeInstances +- cloudasset.assets.exportComputeInterconnect +- cloudasset.assets.exportComputeInterconnectAttachment +- cloudasset.assets.exportComputeLicenses +- cloudasset.assets.exportComputeNetworkEndpointGroups +- cloudasset.assets.exportComputeNetworks +- cloudasset.assets.exportComputeProjects +- cloudasset.assets.exportComputeRegionBackendServices +- cloudasset.assets.exportComputeRouters +- cloudasset.assets.exportComputeRoutes +- cloudasset.assets.exportComputeSecurityPolicy +- cloudasset.assets.exportComputeSnapshots +- cloudasset.assets.exportComputeSslCertificates +- cloudasset.assets.exportComputeSslPolicies +- cloudasset.assets.exportComputeSubnetworks +- cloudasset.assets.exportComputeTargetHttpProxies +- cloudasset.assets.exportComputeTargetHttpsProxies +- cloudasset.assets.exportComputeTargetInstances +- cloudasset.assets.exportComputeTargetPools +- cloudasset.assets.exportComputeTargetSslProxies +- cloudasset.assets.exportComputeTargetTcpProxies +- cloudasset.assets.exportComputeTargetVpnGateways +- cloudasset.assets.exportComputeUrlMaps +- cloudasset.assets.exportComputeVpnTunnels +- cloudasset.assets.exportContainerClusters +- cloudasset.assets.exportDataprocClusters +- cloudasset.assets.exportDataprocJobs +- cloudasset.assets.exportDnsManagedZones +- cloudasset.assets.exportDnsPolicies +- cloudasset.assets.exportIamRoles +- cloudasset.assets.exportIamServiceAccountKeys +- cloudasset.assets.exportIamServiceAccounts +- cloudasset.assets.exportOSConfigOSPolicyAssignmentReports +- cloudasset.assets.exportOSConfigOSPolicyAssignments +- cloudasset.assets.exportPubsubSnapshots +- cloudasset.assets.exportPubsubSubscriptions +- cloudasset.assets.exportPubsubTopics +- cloudasset.assets.exportServicemanagementServices +- cloudasset.assets.exportSpannerBackups +- cloudasset.assets.exportSpannerDatabases +- cloudasset.assets.exportSpannerInstances +- cloudasset.assets.exportSqladminBackupRuns +- cloudasset.assets.exportSqladminInstances +- cloudasset.assets.exportStorageBuckets +- cloudasset.assets.listCloudDocumentAIEvaluation +- cloudasset.assets.listCloudDocumentAIHumanReviewConfig +- cloudasset.assets.listCloudDocumentAILabelerPool +- cloudasset.assets.listCloudDocumentAIProcessor +- cloudasset.assets.listCloudDocumentAIProcessorVersion +- cloudasset.assets.listSqladminBackupRuns +- cloudasset.assets.searchAllIamPolicies +- cloudasset.assets.searchAllResources +- cloudasset.othercloudconnections.get +- cloudasset.othercloudconnections.list +- cloudasset.savedqueries.get +- cloudasset.savedqueries.list +- cloudbuild.builds.get +- cloudbuild.builds.list +- cloudbuild.connections.fetchLinkableRepositories +- cloudbuild.connections.get +- cloudbuild.connections.getIamPolicy +- cloudbuild.connections.list +- cloudbuild.integrations.get +- cloudbuild.integrations.list +- cloudbuild.locations.get +- cloudbuild.locations.list +- cloudbuild.operations.get +- cloudbuild.operations.list +- cloudbuild.repositories.fetchGitRefs +- cloudbuild.repositories.get +- cloudbuild.repositories.list +- cloudbuild.workerpools.get +- cloudbuild.workerpools.list +- cloudfunctions.functions.get +- cloudfunctions.functions.getIamPolicy +- cloudfunctions.functions.list +- cloudfunctions.functions.sourceCodeGet +- cloudfunctions.locations.list +- cloudfunctions.operations.get +- cloudfunctions.operations.list +- cloudsql.backupRuns.export +- cloudsql.backupRuns.get +- cloudsql.backupRuns.list +- cloudsql.databases.get +- cloudsql.databases.list +- cloudsql.instances.createBackupDrBackup +- cloudsql.instances.export +- cloudsql.instances.get +- cloudsql.instances.getDiskShrinkConfig +- cloudsql.instances.list +- cloudsql.instances.listEffectiveTags +- cloudsql.instances.listServerCas +- cloudsql.instances.listServerCertificates +- cloudsql.instances.listTagBindings +- cloudsql.schemas.view +- cloudsql.sslCerts.get +- cloudsql.sslCerts.list +- cloudsql.users.get +- cloudsql.users.list +- compute.acceleratorTypes.get +- compute.acceleratorTypes.list +- compute.addresses.get +- compute.addresses.list +- compute.addresses.listEffectiveTags +- compute.addresses.listTagBindings +- compute.autoscalers.get +- compute.autoscalers.list +- compute.backendBuckets.get +- compute.backendBuckets.getIamPolicy +- compute.backendBuckets.list +- compute.backendBuckets.listEffectiveTags +- compute.backendBuckets.listTagBindings +- compute.backendServices.get +- compute.backendServices.getIamPolicy +- compute.backendServices.list +- compute.backendServices.listEffectiveTags +- compute.backendServices.listTagBindings +- compute.commitments.get +- compute.commitments.list +- compute.crossSiteNetworks.get +- compute.crossSiteNetworks.list +- compute.diskSettings.get +- compute.diskTypes.get +- compute.diskTypes.list +- compute.disks.createSnapshot +- compute.disks.get +- compute.disks.getIamPolicy +- compute.disks.list +- compute.disks.listEffectiveTags +- compute.disks.listTagBindings +- compute.disks.useReadOnly +- compute.externalVpnGateways.get +- compute.externalVpnGateways.list +- compute.externalVpnGateways.listEffectiveTags +- compute.externalVpnGateways.listTagBindings +- compute.firewallPolicies.get +- compute.firewallPolicies.getIamPolicy +- compute.firewallPolicies.list +- compute.firewallPolicies.listEffectiveTags +- compute.firewallPolicies.listTagBindings +- compute.firewalls.get +- compute.firewalls.list +- compute.firewalls.listEffectiveTags +- compute.firewalls.listTagBindings +- compute.forwardingRules.get +- compute.forwardingRules.list +- compute.forwardingRules.listEffectiveTags +- compute.forwardingRules.listTagBindings +- compute.futureReservations.get +- compute.futureReservations.getIamPolicy +- compute.futureReservations.list +- compute.globalAddresses.get +- compute.globalAddresses.list +- compute.globalAddresses.listEffectiveTags +- compute.globalAddresses.listTagBindings +- compute.globalForwardingRules.get +- compute.globalForwardingRules.list +- compute.globalForwardingRules.listEffectiveTags +- compute.globalForwardingRules.listTagBindings +- compute.globalNetworkEndpointGroups.get +- compute.globalNetworkEndpointGroups.list +- compute.globalNetworkEndpointGroups.listEffectiveTags +- compute.globalNetworkEndpointGroups.listTagBindings +- compute.globalOperations.get +- compute.globalOperations.list +- compute.globalPublicDelegatedPrefixes.get +- compute.globalPublicDelegatedPrefixes.list +- compute.healthChecks.get +- compute.healthChecks.list +- compute.healthChecks.listEffectiveTags +- compute.healthChecks.listTagBindings +- compute.healthChecks.useReadOnly +- compute.httpHealthChecks.get +- compute.httpHealthChecks.list +- compute.httpHealthChecks.listEffectiveTags +- compute.httpHealthChecks.listTagBindings +- compute.httpHealthChecks.useReadOnly +- compute.httpsHealthChecks.get +- compute.httpsHealthChecks.list +- compute.httpsHealthChecks.listEffectiveTags +- compute.httpsHealthChecks.listTagBindings +- compute.httpsHealthChecks.useReadOnly +- compute.images.get +- compute.images.getFromFamily +- compute.images.getIamPolicy +- compute.images.list +- compute.images.listEffectiveTags +- compute.images.listTagBindings +- compute.images.useReadOnly +- compute.instanceGroupManagers.get +- compute.instanceGroupManagers.list +- compute.instanceGroupManagers.listEffectiveTags +- compute.instanceGroupManagers.listTagBindings +- compute.instanceGroups.get +- compute.instanceGroups.list +- compute.instanceGroups.listEffectiveTags +- compute.instanceGroups.listTagBindings +- compute.instanceSettings.get +- compute.instanceTemplates.get +- compute.instanceTemplates.getIamPolicy +- compute.instanceTemplates.list +- compute.instanceTemplates.useReadOnly +- compute.instances.get +- compute.instances.getEffectiveFirewalls +- compute.instances.getIamPolicy +- compute.instances.getScreenshot +- compute.instances.getSerialPortOutput +- compute.instances.getShieldedInstanceIdentity +- compute.instances.getShieldedVmIdentity +- compute.instances.list +- compute.instances.listEffectiveTags +- compute.instances.listReferrers +- compute.instances.listTagBindings +- compute.instances.useReadOnly +- compute.instantSnapshots.get +- compute.instantSnapshots.getIamPolicy +- compute.instantSnapshots.list +- compute.instantSnapshots.useReadOnly +- compute.interconnectAttachmentGroups.get +- compute.interconnectAttachmentGroups.list +- compute.interconnectAttachments.listEffectiveTags +- compute.interconnectAttachments.listTagBindings +- compute.interconnectGroups.get +- compute.interconnectGroups.list +- compute.interconnectRemoteLocations.get +- compute.interconnectRemoteLocations.list +- compute.interconnects.listEffectiveTags +- compute.interconnects.listTagBindings +- compute.licenseCodes.getIamPolicy +- compute.licenses.get +- compute.licenses.getIamPolicy +- compute.machineImages.get +- compute.machineImages.getIamPolicy +- compute.machineImages.list +- compute.machineImages.useReadOnly +- compute.machineTypes.get +- compute.machineTypes.list +- compute.multiMig.get +- compute.multiMig.list +- compute.multiMigMembers.get +- compute.multiMigMembers.list +- compute.networkAttachments.get +- compute.networkAttachments.getIamPolicy +- compute.networkAttachments.list +- compute.networkAttachments.listEffectiveTags +- compute.networkAttachments.listTagBindings +- compute.networkEdgeSecurityServices.get +- compute.networkEdgeSecurityServices.list +- compute.networkEdgeSecurityServices.listEffectiveTags +- compute.networkEdgeSecurityServices.listTagBindings +- compute.networkEndpointGroups.get +- compute.networkEndpointGroups.list +- compute.networkEndpointGroups.listEffectiveTags +- compute.networkEndpointGroups.listTagBindings +- compute.networkProfiles.get +- compute.networkProfiles.list +- compute.networks.get +- compute.networks.getEffectiveFirewalls +- compute.networks.getRegionEffectiveFirewalls +- compute.networks.list +- compute.networks.listEffectiveTags +- compute.networks.listPeeringRoutes +- compute.networks.listTagBindings +- compute.nodeGroups.get +- compute.nodeGroups.getIamPolicy +- compute.nodeGroups.list +- compute.nodeTemplates.get +- compute.nodeTemplates.getIamPolicy +- compute.nodeTemplates.list +- compute.nodeTypes.get +- compute.nodeTypes.list +- compute.organizations.listAssociations +- compute.packetMirrorings.get +- compute.packetMirrorings.list +- compute.packetMirrorings.listEffectiveTags +- compute.packetMirrorings.listTagBindings +- compute.previewFeatures.get +- compute.previewFeatures.list +- compute.projects.get +- compute.publicAdvertisedPrefixes.get +- compute.publicAdvertisedPrefixes.list +- compute.publicDelegatedPrefixes.get +- compute.publicDelegatedPrefixes.list +- compute.publicDelegatedPrefixes.listEffectiveTags +- compute.publicDelegatedPrefixes.listTagBindings +- compute.regionBackendServices.get +- compute.regionBackendServices.getIamPolicy +- compute.regionBackendServices.list +- compute.regionBackendServices.listEffectiveTags +- compute.regionBackendServices.listTagBindings +- compute.regionFirewallPolicies.get +- compute.regionFirewallPolicies.getIamPolicy +- compute.regionFirewallPolicies.list +- compute.regionFirewallPolicies.listEffectiveTags +- compute.regionFirewallPolicies.listTagBindings +- compute.regionHealthCheckServices.get +- compute.regionHealthCheckServices.list +- compute.regionHealthChecks.get +- compute.regionHealthChecks.list +- compute.regionHealthChecks.listEffectiveTags +- compute.regionHealthChecks.listTagBindings +- compute.regionHealthChecks.useReadOnly +- compute.regionNetworkEndpointGroups.get +- compute.regionNetworkEndpointGroups.list +- compute.regionNetworkEndpointGroups.listEffectiveTags +- compute.regionNetworkEndpointGroups.listTagBindings +- compute.regionNotificationEndpoints.get +- compute.regionNotificationEndpoints.list +- compute.regionOperations.get +- compute.regionOperations.list +- compute.regionSecurityPolicies.get +- compute.regionSecurityPolicies.list +- compute.regionSecurityPolicies.listEffectiveTags +- compute.regionSecurityPolicies.listTagBindings +- compute.regionSslCertificates.get +- compute.regionSslCertificates.list +- compute.regionSslCertificates.listEffectiveTags +- compute.regionSslCertificates.listTagBindings +- compute.regionSslPolicies.get +- compute.regionSslPolicies.list +- compute.regionSslPolicies.listAvailableFeatures +- compute.regionSslPolicies.listEffectiveTags +- compute.regionSslPolicies.listTagBindings +- compute.regionTargetHttpProxies.get +- compute.regionTargetHttpProxies.list +- compute.regionTargetHttpProxies.listEffectiveTags +- compute.regionTargetHttpProxies.listTagBindings +- compute.regionTargetHttpsProxies.get +- compute.regionTargetHttpsProxies.list +- compute.regionTargetHttpsProxies.listEffectiveTags +- compute.regionTargetHttpsProxies.listTagBindings +- compute.regionTargetTcpProxies.get +- compute.regionTargetTcpProxies.list +- compute.regionTargetTcpProxies.listEffectiveTags +- compute.regionTargetTcpProxies.listTagBindings +- compute.regionUrlMaps.get +- compute.regionUrlMaps.list +- compute.regionUrlMaps.listEffectiveTags +- compute.regionUrlMaps.listTagBindings +- compute.regionUrlMaps.validate +- compute.regions.get +- compute.regions.list +- compute.reservationBlocks.get +- compute.reservationBlocks.list +- compute.reservationSubBlocks.get +- compute.reservationSubBlocks.list +- compute.reservations.get +- compute.reservations.list +- compute.resourcePolicies.get +- compute.resourcePolicies.getIamPolicy +- compute.resourcePolicies.list +- compute.resourcePolicies.useReadOnly +- compute.routers.get +- compute.routers.getRoutePolicy +- compute.routers.list +- compute.routers.listBgpRoutes +- compute.routers.listEffectiveTags +- compute.routers.listRoutePolicies +- compute.routers.listTagBindings +- compute.routes.get +- compute.routes.list +- compute.routes.listEffectiveTags +- compute.routes.listTagBindings +- compute.securityPolicies.listEffectiveTags +- compute.securityPolicies.listTagBindings +- compute.serviceAttachments.get +- compute.serviceAttachments.getIamPolicy +- compute.serviceAttachments.list +- compute.serviceAttachments.listEffectiveTags +- compute.serviceAttachments.listTagBindings +- compute.snapshotSettings.get +- compute.snapshots.get +- compute.snapshots.getIamPolicy +- compute.snapshots.list +- compute.snapshots.listEffectiveTags +- compute.snapshots.listTagBindings +- compute.snapshots.useReadOnly +- compute.spotAssistants.get +- compute.sslCertificates.get +- compute.sslCertificates.list +- compute.sslCertificates.listEffectiveTags +- compute.sslCertificates.listTagBindings +- compute.sslPolicies.listEffectiveTags +- compute.sslPolicies.listTagBindings +- compute.storagePools.get +- compute.storagePools.getIamPolicy +- compute.storagePools.list +- compute.subnetworks.get +- compute.subnetworks.getIamPolicy +- compute.subnetworks.list +- compute.subnetworks.listEffectiveTags +- compute.subnetworks.listTagBindings +- compute.targetGrpcProxies.get +- compute.targetGrpcProxies.list +- compute.targetGrpcProxies.listEffectiveTags +- compute.targetGrpcProxies.listTagBindings +- compute.targetHttpProxies.get +- compute.targetHttpProxies.list +- compute.targetHttpProxies.listEffectiveTags +- compute.targetHttpProxies.listTagBindings +- compute.targetHttpsProxies.get +- compute.targetHttpsProxies.list +- compute.targetHttpsProxies.listEffectiveTags +- compute.targetHttpsProxies.listTagBindings +- compute.targetInstances.get +- compute.targetInstances.list +- compute.targetInstances.listEffectiveTags +- compute.targetInstances.listTagBindings +- compute.targetPools.get +- compute.targetPools.list +- compute.targetPools.listEffectiveTags +- compute.targetPools.listTagBindings +- compute.targetSslProxies.get +- compute.targetSslProxies.list +- compute.targetSslProxies.listEffectiveTags +- compute.targetSslProxies.listTagBindings +- compute.targetTcpProxies.get +- compute.targetTcpProxies.list +- compute.targetTcpProxies.listEffectiveTags +- compute.targetTcpProxies.listTagBindings +- compute.targetVpnGateways.get +- compute.targetVpnGateways.list +- compute.targetVpnGateways.listEffectiveTags +- compute.targetVpnGateways.listTagBindings +- compute.urlMaps.listEffectiveTags +- compute.urlMaps.listTagBindings +- compute.vpnGateways.get +- compute.vpnGateways.list +- compute.vpnGateways.listEffectiveTags +- compute.vpnGateways.listTagBindings +- compute.vpnTunnels.get +- compute.vpnTunnels.list +- compute.vpnTunnels.listEffectiveTags +- compute.vpnTunnels.listTagBindings +- compute.wireGroups.get +- compute.wireGroups.list +- compute.zoneOperations.get +- compute.zoneOperations.list +- compute.zones.get +- compute.zones.list +- container.apiServices.get +- container.apiServices.getStatus +- container.apiServices.list +- container.auditSinks.get +- container.auditSinks.list +- container.backendConfigs.get +- container.backendConfigs.list +- container.certificateSigningRequests.get +- container.certificateSigningRequests.getStatus +- container.certificateSigningRequests.list +- container.clusterRoleBindings.get +- container.clusterRoleBindings.list +- container.clusterRoles.get +- container.clusterRoles.list +- container.clusters.connect +- container.clusters.get +- container.clusters.list +- container.clusters.listEffectiveTags +- container.clusters.listTagBindings +- container.componentStatuses.get +- container.componentStatuses.list +- container.configMaps.get +- container.configMaps.list +- container.controllerRevisions.get +- container.controllerRevisions.list +- container.cronJobs.get +- container.cronJobs.getStatus +- container.cronJobs.list +- container.csiDrivers.get +- container.csiDrivers.list +- container.csiNodeInfos.get +- container.csiNodeInfos.list +- container.csiNodes.get +- container.csiNodes.list +- container.customResourceDefinitions.get +- container.customResourceDefinitions.getStatus +- container.customResourceDefinitions.list +- container.daemonSets.get +- container.daemonSets.getStatus +- container.daemonSets.list +- container.deployments.get +- container.deployments.getStatus +- container.deployments.list +- container.endpointSlices.get +- container.endpointSlices.list +- container.endpoints.get +- container.endpoints.list +- container.events.get +- container.events.list +- container.frontendConfigs.get +- container.frontendConfigs.list +- container.horizontalPodAutoscalers.get +- container.horizontalPodAutoscalers.getStatus +- container.horizontalPodAutoscalers.list +- container.ingresses.get +- container.ingresses.getStatus +- container.ingresses.list +- container.jobs.get +- container.jobs.getStatus +- container.jobs.list +- container.leases.get +- container.leases.list +- container.limitRanges.get +- container.limitRanges.list +- container.managedCertificates.get +- container.managedCertificates.list +- container.mutatingWebhookConfigurations.get +- container.mutatingWebhookConfigurations.list +- container.namespaces.get +- container.namespaces.getStatus +- container.namespaces.list +- container.networkPolicies.get +- container.networkPolicies.list +- container.nodes.get +- container.nodes.getStatus +- container.nodes.list +- container.operations.get +- container.operations.list +- container.persistentVolumeClaims.get +- container.persistentVolumeClaims.getStatus +- container.persistentVolumeClaims.list +- container.persistentVolumes.get +- container.persistentVolumes.getStatus +- container.persistentVolumes.list +- container.podDisruptionBudgets.get +- container.podDisruptionBudgets.getStatus +- container.podDisruptionBudgets.list +- container.podSecurityPolicies.get +- container.podSecurityPolicies.list +- container.podTemplates.get +- container.podTemplates.list +- container.pods.get +- container.pods.getLogs +- container.pods.getStatus +- container.pods.list +- container.priorityClasses.get +- container.priorityClasses.list +- container.replicaSets.get +- container.replicaSets.getScale +- container.replicaSets.getStatus +- container.replicaSets.list +- container.replicationControllers.get +- container.replicationControllers.getScale +- container.replicationControllers.getStatus +- container.replicationControllers.list +- container.resourceQuotas.get +- container.resourceQuotas.getStatus +- container.resourceQuotas.list +- container.roleBindings.get +- container.roleBindings.list +- container.roles.get +- container.roles.list +- container.runtimeClasses.get +- container.runtimeClasses.list +- container.selfSubjectAccessReviews.create +- container.selfSubjectRulesReviews.create +- container.serviceAccounts.get +- container.serviceAccounts.list +- container.services.get +- container.services.getStatus +- container.services.list +- container.statefulSets.get +- container.statefulSets.getScale +- container.statefulSets.getStatus +- container.statefulSets.list +- container.storageClasses.get +- container.storageClasses.list +- container.storageStates.get +- container.storageStates.getStatus +- container.storageStates.list +- container.storageVersionMigrations.get +- container.storageVersionMigrations.getStatus +- container.storageVersionMigrations.list +- container.thirdPartyObjects.get +- container.thirdPartyObjects.list +- container.tokenReviews.create +- container.updateInfos.get +- container.updateInfos.list +- container.validatingWebhookConfigurations.get +- container.validatingWebhookConfigurations.list +- container.volumeAttachments.get +- container.volumeAttachments.getStatus +- container.volumeAttachments.list +- container.volumeSnapshotClasses.get +- container.volumeSnapshotClasses.list +- container.volumeSnapshotContents.get +- container.volumeSnapshotContents.getStatus +- container.volumeSnapshotContents.list +- container.volumeSnapshots.get +- container.volumeSnapshots.getStatus +- container.volumeSnapshots.list +- containeranalysis.notes.get +- containeranalysis.notes.getIamPolicy +- containeranalysis.notes.list +- containeranalysis.occurrences.get +- containeranalysis.occurrences.getIamPolicy +- containeranalysis.occurrences.list +- containersecurity.locations.get +- containersecurity.locations.list +- dataflow.jobs.get +- dataflow.jobs.list +- dataflow.messages.list +- dataflow.metrics.get +- dataflow.snapshots.get +- dataflow.snapshots.list +- dataproc.agents.get +- dataproc.agents.list +- dataproc.autoscalingPolicies.get +- dataproc.autoscalingPolicies.getIamPolicy +- dataproc.autoscalingPolicies.list +- dataproc.autoscalingPolicies.use +- dataproc.batches.get +- dataproc.batches.list +- dataproc.batches.sparkApplicationRead +- dataproc.clusters.get +- dataproc.clusters.getIamPolicy +- dataproc.clusters.list +- dataproc.jobs.get +- dataproc.jobs.getIamPolicy +- dataproc.jobs.list +- dataproc.nodeGroups.get +- dataproc.operations.get +- dataproc.operations.getIamPolicy +- dataproc.operations.list +- dataproc.sessionTemplates.get +- dataproc.sessionTemplates.list +- dataproc.sessions.get +- dataproc.sessions.list +- dataproc.sessions.sparkApplicationRead +- dataproc.tasks.listInvalidatedLeases +- dataproc.workflowTemplates.get +- dataproc.workflowTemplates.getIamPolicy +- dataproc.workflowTemplates.list +- dataprocessing.datasources.get +- dataprocessing.datasources.list +- dataprocessing.featurecontrols.list +- dataprocessing.groupcontrols.get +- dataprocessing.groupcontrols.list +- dataprocrm.locations.get +- dataprocrm.locations.list +- dataprocrm.nodePools.get +- dataprocrm.nodePools.list +- dataprocrm.nodes.get +- dataprocrm.nodes.list +- dataprocrm.nodes.mintOAuthToken +- dataprocrm.operations.get +- dataprocrm.operations.list +- dataprocrm.workloads.get +- dataprocrm.workloads.list +- datastore.backupSchedules.get +- datastore.backupSchedules.list +- datastore.backups.get +- datastore.backups.list +- datastore.databases.get +- datastore.databases.getMetadata +- datastore.databases.list +- datastore.databases.listEffectiveTags +- datastore.databases.listTagBindings +- datastore.entities.get +- datastore.entities.list +- datastore.indexes.get +- datastore.indexes.list +- datastore.insights.get +- datastore.keyVisualizerScans.get +- datastore.keyVisualizerScans.list +- datastore.namespaces.get +- datastore.namespaces.list +- datastore.operations.get +- datastore.operations.list +- datastore.statistics.get +- datastore.statistics.list +- datastore.userCreds.get +- datastore.userCreds.list +- dns.changes.get +- dns.changes.list +- dns.dnsKeys.get +- dns.dnsKeys.list +- dns.managedZoneOperations.get +- dns.managedZoneOperations.list +- dns.managedZones.get +- dns.managedZones.getIamPolicy +- dns.managedZones.list +- dns.policies.get +- dns.policies.list +- dns.projects.get +- dns.resourceRecordSets.get +- dns.resourceRecordSets.list +- dns.responsePolicies.get +- dns.responsePolicies.list +- dns.responsePolicyRules.get +- dns.responsePolicyRules.list +- firebase.billingPlans.get +- firebase.clients.get +- firebase.clients.list +- firebase.links.list +- firebase.playLinks.get +- firebase.playLinks.list +- firebase.projects.get +- firebaseabt.experimentresults.get +- firebaseabt.experiments.get +- firebaseabt.experiments.list +- firebaseabt.projectmetadata.get +- firebaseanalytics.resources.googleAnalyticsReadAndAnalyze +- firebaseappcheck.appAttestConfig.get +- firebaseappcheck.automations.get +- firebaseappcheck.automations.list +- firebaseappcheck.debugTokens.get +- firebaseappcheck.deviceCheckConfig.get +- firebaseappcheck.playIntegrityConfig.get +- firebaseappcheck.recaptchaEnterpriseConfig.get +- firebaseappcheck.recaptchaV3Config.get +- firebaseappcheck.resourcePolicies.get +- firebaseappcheck.safetyNetConfig.get +- firebaseappcheck.services.get +- firebaseappdistro.groups.list +- firebaseappdistro.releases.list +- firebaseappdistro.testers.list +- firebaseauth.configs.get +- firebaseauth.users.get +- firebasecrash.reports.get +- firebasecrashlytics.config.get +- firebasecrashlytics.data.get +- firebasecrashlytics.issues.get +- firebasecrashlytics.issues.list +- firebasecrashlytics.sessions.get +- firebasedatabase.instances.get +- firebasedatabase.instances.list +- firebasedataconnect.connectorRevisions.get +- firebasedataconnect.connectorRevisions.list +- firebasedataconnect.connectors.get +- firebasedataconnect.connectors.list +- firebasedataconnect.locations.get +- firebasedataconnect.locations.list +- firebasedataconnect.operations.get +- firebasedataconnect.operations.list +- firebasedataconnect.schemaRevisions.get +- firebasedataconnect.schemaRevisions.list +- firebasedataconnect.schemas.get +- firebasedataconnect.schemas.list +- firebasedataconnect.services.get +- firebasedataconnect.services.list +- firebasedynamiclinks.destinations.list +- firebasedynamiclinks.domains.get +- firebasedynamiclinks.domains.list +- firebasedynamiclinks.links.get +- firebasedynamiclinks.links.list +- firebasedynamiclinks.stats.get +- firebaseextensions.configs.list +- firebaseextensionspublisher.extensions.get +- firebaseextensionspublisher.extensions.list +- firebasehosting.sites.get +- firebasehosting.sites.list +- firebaseinappmessaging.campaigns.get +- firebaseinappmessaging.campaigns.list +- firebasemessagingcampaigns.campaigns.get +- firebasemessagingcampaigns.campaigns.list +- firebaseml.models.get +- firebaseml.models.list +- firebaseml.modelversions.get +- firebaseml.modelversions.list +- firebasenotifications.messages.get +- firebasenotifications.messages.list +- firebaseperformance.data.get +- firebaserules.releases.get +- firebaserules.releases.getExecutable +- firebaserules.releases.list +- firebaserules.rulesets.list +- firebaserules.rulesets.test +- firebasestorage.buckets.get +- firebasestorage.buckets.list +- firebasestorage.defaultBucket.get +- firebasevertexai.configs.get +- iam.denypolicies.get +- iam.denypolicies.list +- iam.googleapis.com/oauthClientCredentials.get +- iam.googleapis.com/oauthClientCredentials.list +- iam.googleapis.com/oauthClients.get +- iam.googleapis.com/oauthClients.list +- iam.googleapis.com/workloadIdentityPoolProviderKeys.get +- iam.googleapis.com/workloadIdentityPoolProviderKeys.list +- iam.googleapis.com/workloadIdentityPoolProviders.get +- iam.googleapis.com/workloadIdentityPoolProviders.list +- iam.googleapis.com/workloadIdentityPools.get +- iam.googleapis.com/workloadIdentityPools.list +- iam.roles.get +- iam.roles.list +- iam.serviceAccountKeys.get +- iam.serviceAccountKeys.list +- iam.serviceAccounts.get +- iam.serviceAccounts.getIamPolicy +- iam.serviceAccounts.list +- iam.serviceAccounts.listEffectiveTags +- iam.serviceAccounts.listTagBindings +- iap.tunnelDestGroups.get +- iap.tunnelDestGroups.list +- monitoring.alertPolicies.get +- monitoring.alertPolicies.list +- monitoring.alertPolicies.listEffectiveTags +- monitoring.alertPolicies.listTagBindings +- monitoring.dashboards.get +- monitoring.dashboards.list +- monitoring.dashboards.listEffectiveTags +- monitoring.dashboards.listTagBindings +- monitoring.groups.get +- monitoring.groups.list +- monitoring.metricDescriptors.get +- monitoring.metricDescriptors.list +- monitoring.monitoredResourceDescriptors.get +- monitoring.monitoredResourceDescriptors.list +- monitoring.services.get +- monitoring.services.list +- monitoring.slos.get +- monitoring.slos.list +- monitoring.snoozes.get +- monitoring.snoozes.list +- monitoring.timeSeries.list +- monitoring.uptimeCheckConfigs.get +- monitoring.uptimeCheckConfigs.list +- pubsub.messageTransforms.validate +- pubsub.schemas.attach +- pubsub.schemas.get +- pubsub.schemas.getIamPolicy +- pubsub.schemas.list +- pubsub.schemas.listRevisions +- pubsub.schemas.validate +- pubsub.snapshots.list +- pubsub.subscriptions.get +- pubsub.subscriptions.list +- pubsub.topics.get +- pubsub.topics.list +- pubsublite.locations.openKafkaStream +- pubsublite.operations.get +- pubsublite.operations.list +- pubsublite.reservations.get +- pubsublite.reservations.list +- pubsublite.reservations.listTopics +- pubsublite.subscriptions.get +- pubsublite.subscriptions.getCursor +- pubsublite.subscriptions.list +- pubsublite.subscriptions.subscribe +- pubsublite.topics.computeHeadCursor +- pubsublite.topics.computeMessageStats +- pubsublite.topics.computeTimeCursor +- pubsublite.topics.get +- pubsublite.topics.getPartitions +- pubsublite.topics.list +- pubsublite.topics.listSubscriptions +- pubsublite.topics.subscribe +- redis.backupCollections.get +- redis.backupCollections.list +- redis.backups.export +- redis.backups.get +- redis.backups.list +- redis.clusters.get +- redis.clusters.list +- redis.instances.get +- redis.instances.list +- redis.instances.listEffectiveTags +- redis.instances.listTagBindings +- redis.locations.get +- redis.locations.list +- redis.operations.get +- redis.operations.list +- resourcemanager.hierarchyNodes.listEffectiveTags +- resourcemanager.hierarchyNodes.listTagBindings +- resourcemanager.projects.get +- resourcemanager.projects.getIamPolicy +- resourcemanager.tagHolds.list +- resourcemanager.tagKeys.get +- resourcemanager.tagKeys.getIamPolicy +- resourcemanager.tagKeys.list +- resourcemanager.tagValues.get +- resourcemanager.tagValues.getIamPolicy +- resourcemanager.tagValues.list +- secretmanager.locations.get +- secretmanager.locations.list +- secretmanager.secrets.get +- secretmanager.secrets.getIamPolicy +- secretmanager.secrets.list +- secretmanager.secrets.listEffectiveTags +- secretmanager.secrets.listTagBindings +- secretmanager.versions.get +- secretmanager.versions.list +- servicemanagement.services.get +- servicemanagement.services.list +- serviceusage.services.get +- serviceusage.services.list +- spanner.backupOperations.get +- spanner.backupOperations.list +- spanner.backupSchedules.get +- spanner.backupSchedules.getIamPolicy +- spanner.backupSchedules.list +- spanner.backups.get +- spanner.backups.getIamPolicy +- spanner.backups.list +- spanner.databaseOperations.get +- spanner.databaseOperations.list +- spanner.databaseRoles.list +- spanner.databases.beginReadOnlyTransaction +- spanner.databases.get +- spanner.databases.getDdl +- spanner.databases.getIamPolicy +- spanner.databases.list +- spanner.databases.partitionQuery +- spanner.databases.partitionRead +- spanner.databases.read +- spanner.databases.select +- spanner.databases.useDataBoost +- spanner.instanceConfigOperations.get +- spanner.instanceConfigOperations.list +- spanner.instanceConfigs.get +- spanner.instanceConfigs.list +- spanner.instanceOperations.get +- spanner.instanceOperations.list +- spanner.instancePartitionOperations.get +- spanner.instancePartitionOperations.list +- spanner.instancePartitions.get +- spanner.instancePartitions.list +- spanner.instances.get +- spanner.instances.getIamPolicy +- spanner.instances.list +- spanner.instances.listEffectiveTags +- spanner.instances.listTagBindings +- spanner.sessions.create +- spanner.sessions.get +- spanner.sessions.list +- storage.buckets.list +- storage.buckets.listEffectiveTags +- storage.buckets.listTagBindings +- storage.folders.get +- storage.folders.list +- storage.hmacKeys.get +- storage.hmacKeys.list +- storage.intelligenceConfigs.get +- storageinsights.datasetConfigs.get +- storageinsights.datasetConfigs.list +- storageinsights.locations.get +- storageinsights.locations.list +- storageinsights.operations.get +- storageinsights.operations.list +- storageinsights.reportConfigs.get +- storageinsights.reportConfigs.list +- storageinsights.reportDetails.get +- storageinsights.reportDetails.list +- storagetransfer.agentpools.get +- storagetransfer.agentpools.list +- storagetransfer.jobs.get +- storagetransfer.jobs.list +- storagetransfer.operations.get +- storagetransfer.operations.list +- storagetransfer.projects.getServiceAccount +- trafficdirector.networks.getConfigs +role_id: beam_viewer +stage: GA +title: beam_viewer diff --git a/infra/iam/roles/beam_writer.role.yaml b/infra/iam/roles/beam_writer.role.yaml new file mode 100644 index 000000000000..947757b0d6d9 --- /dev/null +++ b/infra/iam/roles/beam_writer.role.yaml @@ -0,0 +1,306 @@ +# 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. + +# This file is auto-generated by generate_roles.py. +# Do not edit manually. + +# This file was generated on 2025-08-11 15:53:17 UTC + +description: This is the beam_writer role +permissions: +- bigquery.datasets.create +- bigquery.tables.export +- bigquery.tables.get +- bigquery.tables.getData +- bigquery.tables.list +- bigquerymigration.translation.translate +- cloudkms.cryptoKeyVersions.get +- cloudkms.cryptoKeyVersions.list +- cloudkms.cryptoKeys.get +- cloudkms.cryptoKeys.getIamPolicy +- cloudkms.cryptoKeys.list +- cloudkms.ekmConfigs.get +- cloudkms.ekmConfigs.getIamPolicy +- cloudkms.ekmConnections.get +- cloudkms.ekmConnections.getIamPolicy +- cloudkms.ekmConnections.list +- cloudkms.ekmConnections.verifyConnectivity +- cloudkms.importJobs.get +- cloudkms.importJobs.getIamPolicy +- cloudkms.importJobs.list +- cloudkms.kajPolicyConfigs.get +- cloudkms.keyHandles.create +- cloudkms.keyHandles.get +- cloudkms.keyHandles.list +- cloudkms.keyRings.get +- cloudkms.keyRings.getIamPolicy +- cloudkms.keyRings.list +- cloudkms.keyRings.listEffectiveTags +- cloudkms.keyRings.listTagBindings +- cloudkms.locations.generateRandomBytes +- cloudkms.locations.get +- cloudkms.locations.list +- cloudkms.operations.get +- cloudkms.projects.showEffectiveAutokeyConfig +- cloudkms.projects.showEffectiveKajEnrollmentConfig +- cloudkms.projects.showEffectiveKajPolicyConfig +- cloudsql.instances.login +- container.apiServices.create +- container.apiServices.update +- container.apiServices.updateStatus +- container.auditSinks.create +- container.auditSinks.update +- container.backendConfigs.create +- container.backendConfigs.update +- container.bindings.create +- container.certificateSigningRequests.create +- container.certificateSigningRequests.update +- container.certificateSigningRequests.updateStatus +- container.configMaps.create +- container.configMaps.update +- container.cronJobs.create +- container.cronJobs.update +- container.cronJobs.updateStatus +- container.csiDrivers.create +- container.csiDrivers.update +- container.csiNodeInfos.create +- container.csiNodeInfos.update +- container.csiNodes.create +- container.csiNodes.update +- container.customResourceDefinitions.create +- container.customResourceDefinitions.update +- container.customResourceDefinitions.updateStatus +- container.daemonSets.create +- container.daemonSets.update +- container.daemonSets.updateStatus +- container.deployments.create +- container.deployments.getScale +- container.deployments.rollback +- container.deployments.update +- container.deployments.updateScale +- container.deployments.updateStatus +- container.endpointSlices.create +- container.endpointSlices.update +- container.endpoints.create +- container.endpoints.update +- container.events.create +- container.events.update +- container.frontendConfigs.create +- container.frontendConfigs.update +- container.horizontalPodAutoscalers.create +- container.horizontalPodAutoscalers.update +- container.horizontalPodAutoscalers.updateStatus +- container.ingresses.create +- container.ingresses.update +- container.ingresses.updateStatus +- container.jobs.create +- container.jobs.update +- container.jobs.updateStatus +- container.leases.create +- container.leases.update +- container.limitRanges.create +- container.limitRanges.update +- container.localSubjectAccessReviews.create +- container.managedCertificates.create +- container.managedCertificates.update +- container.namespaces.create +- container.namespaces.update +- container.namespaces.updateStatus +- container.networkPolicies.create +- container.networkPolicies.update +- container.nodes.create +- container.nodes.proxy +- container.nodes.update +- container.nodes.updateStatus +- container.persistentVolumeClaims.create +- container.persistentVolumeClaims.update +- container.persistentVolumeClaims.updateStatus +- container.persistentVolumes.create +- container.persistentVolumes.update +- container.persistentVolumes.updateStatus +- container.podDisruptionBudgets.create +- container.podDisruptionBudgets.update +- container.podDisruptionBudgets.updateStatus +- container.podTemplates.create +- container.podTemplates.update +- container.pods.attach +- container.pods.create +- container.pods.evict +- container.pods.exec +- container.pods.portForward +- container.pods.proxy +- container.pods.update +- container.pods.updateStatus +- container.priorityClasses.create +- container.priorityClasses.update +- container.replicaSets.create +- container.replicaSets.update +- container.replicaSets.updateScale +- container.replicaSets.updateStatus +- container.replicationControllers.create +- container.replicationControllers.update +- container.replicationControllers.updateScale +- container.replicationControllers.updateStatus +- container.resourceQuotas.create +- container.resourceQuotas.update +- container.resourceQuotas.updateStatus +- container.runtimeClasses.create +- container.runtimeClasses.update +- container.secrets.create +- container.secrets.get +- container.secrets.list +- container.secrets.update +- container.serviceAccounts.create +- container.serviceAccounts.createToken +- container.serviceAccounts.update +- container.services.create +- container.services.proxy +- container.services.update +- container.services.updateStatus +- container.statefulSets.create +- container.statefulSets.update +- container.statefulSets.updateScale +- container.statefulSets.updateStatus +- container.storageClasses.create +- container.storageClasses.update +- container.storageStates.create +- container.storageStates.update +- container.storageStates.updateStatus +- container.storageVersionMigrations.create +- container.storageVersionMigrations.update +- container.storageVersionMigrations.updateStatus +- container.subjectAccessReviews.create +- container.thirdPartyObjects.create +- container.thirdPartyObjects.update +- container.updateInfos.create +- container.updateInfos.update +- container.volumeAttachments.create +- container.volumeAttachments.update +- container.volumeAttachments.updateStatus +- container.volumeSnapshotClasses.create +- container.volumeSnapshotClasses.update +- container.volumeSnapshotContents.create +- container.volumeSnapshotContents.update +- container.volumeSnapshotContents.updateStatus +- container.volumeSnapshots.create +- container.volumeSnapshots.update +- container.volumeSnapshots.updateStatus +- dataform.commentThreads.get +- dataform.commentThreads.list +- dataform.comments.get +- dataform.comments.list +- dataform.compilationResults.get +- dataform.compilationResults.list +- dataform.compilationResults.query +- dataform.config.get +- dataform.locations.get +- dataform.locations.list +- dataform.releaseConfigs.get +- dataform.releaseConfigs.list +- dataform.repositories.computeAccessTokenStatus +- dataform.repositories.create +- dataform.repositories.fetchHistory +- dataform.repositories.fetchRemoteBranches +- dataform.repositories.get +- dataform.repositories.getIamPolicy +- dataform.repositories.list +- dataform.repositories.queryDirectoryContents +- dataform.repositories.readFile +- dataform.workflowConfigs.get +- dataform.workflowConfigs.list +- dataform.workflowInvocations.get +- dataform.workflowInvocations.list +- dataform.workflowInvocations.query +- dataform.workspaces.fetchFileDiff +- dataform.workspaces.fetchFileGitStatuses +- dataform.workspaces.fetchGitAheadBehind +- dataform.workspaces.get +- dataform.workspaces.getIamPolicy +- dataform.workspaces.list +- dataform.workspaces.queryDirectoryContents +- dataform.workspaces.readFile +- dataform.workspaces.searchFiles +- dataplex.aspectTypes.get +- dataplex.aspectTypes.getIamPolicy +- dataplex.aspectTypes.list +- dataplex.assetActions.list +- dataplex.assets.get +- dataplex.assets.getIamPolicy +- dataplex.assets.list +- dataplex.content.get +- dataplex.content.getIamPolicy +- dataplex.content.list +- dataplex.dataAttributeBindings.get +- dataplex.dataAttributeBindings.getIamPolicy +- dataplex.dataAttributeBindings.list +- dataplex.dataAttributes.get +- dataplex.dataAttributes.getIamPolicy +- dataplex.dataAttributes.list +- dataplex.dataTaxonomies.get +- dataplex.dataTaxonomies.getIamPolicy +- dataplex.dataTaxonomies.list +- dataplex.datascans.get +- dataplex.datascans.getData +- dataplex.datascans.getIamPolicy +- dataplex.datascans.list +- dataplex.entities.get +- dataplex.entities.list +- dataplex.entries.get +- dataplex.entries.list +- dataplex.entryGroups.export +- dataplex.entryGroups.get +- dataplex.entryGroups.getIamPolicy +- dataplex.entryGroups.list +- dataplex.entryLinks.get +- dataplex.entryTypes.get +- dataplex.entryTypes.getIamPolicy +- dataplex.entryTypes.list +- dataplex.environments.get +- dataplex.environments.getIamPolicy +- dataplex.environments.list +- dataplex.glossaries.get +- dataplex.glossaries.getIamPolicy +- dataplex.glossaries.list +- dataplex.glossaryCategories.get +- dataplex.glossaryCategories.list +- dataplex.glossaryTerms.get +- dataplex.glossaryTerms.list +- dataplex.lakeActions.list +- dataplex.lakes.get +- dataplex.lakes.getIamPolicy +- dataplex.lakes.list +- dataplex.locations.get +- dataplex.locations.list +- dataplex.metadataJobs.get +- dataplex.metadataJobs.list +- dataplex.operations.get +- dataplex.operations.list +- dataplex.partitions.get +- dataplex.partitions.list +- dataplex.projects.search +- dataplex.tasks.get +- dataplex.tasks.getIamPolicy +- dataplex.tasks.list +- dataplex.zoneActions.list +- dataplex.zones.get +- dataplex.zones.getIamPolicy +- dataplex.zones.list +- datastore.entities.allocateIds +- datastore.entities.create +- datastore.entities.update +- trafficdirector.networks.reportMetrics +role_id: beam_writer +stage: GA +title: beam_writer diff --git a/infra/iam/roles/generate_roles.py b/infra/iam/roles/generate_roles.py new file mode 100644 index 000000000000..2d2b4d294ef6 --- /dev/null +++ b/infra/iam/roles/generate_roles.py @@ -0,0 +1,277 @@ +# +# 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. + +# This script generates roles based on what Apache Beam uses in GCP. +# The roles are defined in a YAML file. + +import yaml +import datetime +import os +from google.cloud import iam_admin_v1 +from google.api_core import exceptions + +# Permissions cache to avoid repeated API calls. +permissions_cache = {} + +ASF_LICENSE_HEADER = """# 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. + +# This file is auto-generated by generate_roles.py. +# Do not edit manually. +\n""" + +def get_permission_stage(permission_name: str, project_id: str) -> str: + """ + Finds the support level of a specific IAM permission for a given project. This function caches the results to avoid repeated API calls. + + Args: + permission_name: The name of the permission to check, e.g., 'storage.buckets.create'. + project_id: The ID of the GCP project to check against. + Returns: + The support level of the permission as a string, or "" if the permission is not found. + """ + global permissions_cache + + try: + if f"{project_id}-stage" in permissions_cache: + return permissions_cache[f"{project_id}-stage"].get(permission_name, "") + else: + permissions_cache[f"{project_id}-stage"] = {} + + client = iam_admin_v1.IAMClient() + resource = f"//cloudresourcemanager.googleapis.com/projects/{project_id}" + + request = iam_admin_v1.QueryTestablePermissionsRequest( + full_resource_name=resource, + page_size=1000 + ) + + for permission in client.query_testable_permissions(request=request): + permissions_cache[f"{project_id}-stage"][permission.name] = permission.custom_roles_support_level + + return permissions_cache[f"{project_id}-stage"].get(permission_name, "") + + except exceptions.PermissionDenied as e: + print(f"Error: Permission denied. Ensure you have 'resourcemanager.projects.get' on project '{project_id}'.") + print(f"Details: {e}") + return "" + except exceptions.NotFound as e: + print(f"Error: Project '{project_id}' not found.") + print(f"Details: {e}") + return "" + except Exception as e: + print(f"An unexpected error occurred while fetching permissions: {e}") + return "" + +def get_role_permissions(role_name: str, project_id: str = "") -> list[str]: + """ + Gets the permissions included in a predefined or custom IAM role, filtered to only GA permissions. + + Args: + role_name: The full name of the role. + For predefined roles, e.g., 'roles/secretmanager.viewer'. + For custom roles, e.g., 'projects/your-project-id/roles/your-custom-role'. + + project_id: Optional, used for permission metadata lookup. + Returns: + A list of GA permissions associated with the role. + """ + + global permissions_cache + print(f"Fetching permissions for role: {role_name} in project: {project_id}") + + try: + if f"{project_id}-role" in permissions_cache and role_name in permissions_cache[f"{project_id}-role"]: + return permissions_cache[f"{project_id}-role"].get(role_name, []) + else: + if f"{project_id}-role" not in permissions_cache: + permissions_cache[f"{project_id}-role"] = {} + + client = iam_admin_v1.IAMClient() + request = iam_admin_v1.GetRoleRequest( + name=role_name, + ) + role = client.get_role(request=request) + all_perms = list(role.included_permissions) + ga_perms = [] + for perm in all_perms: + stage = get_permission_stage(perm, project_id) + if stage == iam_admin_v1.Permission.CustomRolesSupportLevel.SUPPORTED: + ga_perms.append(perm) + + permissions_cache[f"{project_id}-role"][role_name] = ga_perms + return ga_perms + except exceptions.NotFound: + print(f"Error: The role '{role_name}' was not found.") + return [] + except Exception as e: + print(f"An unexpected error occurred: {e}") + return [] + +def filter_permissions(permissions: list[str], allowed_prefixes: list[str] = [], denied_suffixes: list[str] = []) -> set[str]: + """ + Filters permissions based on the provided services. + + Args: + permissions: A list of permissions to filter. + allowed_prefixes: A list of strings that permissions must contain to be included. + denied_suffixes: A list of strings that permissions must not contain to be included. + Returns: + A list of permissions that match the specified services. + """ + + filtered_permissions = set() + + for perm in permissions: + if any(perm.startswith(prefix) for prefix in allowed_prefixes): + if not any(perm.endswith(suffix) for suffix in denied_suffixes): + filtered_permissions.add(perm) + + return filtered_permissions + +def generate_role(role_name: str , perms: set[str]) -> dict: + return { + "role_id": f"{role_name}", + "title": f"{role_name}", + "stage": "GA", + "description": f"This is the {role_name} role", + "permissions": sorted(list(perms)), + } + +def write_role_yaml(filename, role_data): + if not role_data.get("permissions"): + print(f"No permissions to write for {filename}. Skipping.") + return + with open(filename, "w") as f: + f.write(ASF_LICENSE_HEADER) + f.write(f"# This file was generated on {datetime.datetime.now(datetime.timezone.utc).strftime('%Y-%m-%d %H:%M:%S')} UTC\n\n") + yaml.dump(role_data, f, default_flow_style=False) + +def get_config(): + """ + Reads the roles configuration from the YAML file and returns it as a dictionary. + The configuration includes services, roles, and suffixes for filtering permissions. + """ + script_dir = os.path.dirname(os.path.abspath(__file__)) + config_path = os.path.join(script_dir, "roles_config.yaml") + with open(config_path, "r") as f: + config = yaml.safe_load(f) + + # Each role inherits permissions from the previous role. + # This means that the viewer role has all the permissions of the committer role, and so on. + # The roles are defined in the order of viewer, committer, infra_manager, and admin. + # The viewer role is the base role, so its file contains all its + + response = { + "project_id": config.get("project_id", "apache-beam-testing"), + "roles_prefix": config.get("roles_prefix", "beam"), + "role": {} + } + + # Add suffixes to the response + suffixes = {} + for suffix in config.get("suffixes", []): + suffixes[suffix["name"]] = suffix["values"] + + services = set() + roles = set() + + # Sort roles by hierarchy to ensure they are processed in the correct order. + config["roles"].sort(key=lambda x: int(x.get("hierarchy", 0))) + + for role in config["roles"]: + services.update(role.get("services", [])) + roles.update(role.get("roles", [])) + + response["role"][role["name"]] = { + "name": role["name"], + "description": role.get("description", f"This is the {role['name']} role"), + "services": services.copy(), + "roles": roles.copy(), + "except_suffixes": [], + } + + # If the role has except_suffixes, add them to the response + suffix_set = set() + for except_suffix in role.get("except_suffixes", []): + if except_suffix in suffixes: + suffix_set.update(suffixes[except_suffix]) + else: + raise ValueError(f"Unknown suffix '{except_suffix}' in role '{role['name']}'") + if suffix_set: + response["role"][role["name"]]["except_suffixes"] = list(suffix_set) + + return response + +def get_roles(): + """ + Generates the roles based on the predefined services and permissions. + This function creates roles for Beam Viewer, Committer, Infra Manager, and Admin. + It filters permissions based on the allowed and denied strings defined in the configuration. + """ + + config = get_config() + response = {} + + project_id = config["project_id"] + + permissions_added = set() + + for role in config["role"].values(): + print(f"Generating role: {config['roles_prefix']}_{role['name']} with services: {role['services']} and roles: {role['roles']}") + # Get the permissions for each base role. + role_permissions = set() + for role_name in role["roles"]: + role_permissions.update(get_role_permissions(role_name, project_id)) + role["permissions"] = filter_permissions( + permissions=list(role_permissions), + allowed_prefixes=list(role["services"]), + denied_suffixes=role.get("except_suffixes", []) + ) + # Remove already added permissions to avoid duplicates. + role["permissions"] = role["permissions"].difference(permissions_added) + permissions_added.update(role["permissions"]) + response[f"{config['roles_prefix']}_{role['name']}"] = generate_role(f"{config['roles_prefix']}_{role['name']}", role["permissions"]) + + return response + +def main(): + """ + Main function to generate the roles and write them to YAML files. + It creates a directory for the roles if it doesn't exist and writes each role to its respective file. + """ + + roles = get_roles() + + for role_name, role_data in roles.items(): + filename = f"{role_name}.role.yaml" + write_role_yaml(filename, role_data) + print(f"Generated {filename} with {len(role_data['permissions'])} permissions.") + +if __name__ == "__main__": + main() diff --git a/infra/iam/roles/roles.tf b/infra/iam/roles/roles.tf new file mode 100644 index 000000000000..d3348fa31b1e --- /dev/null +++ b/infra/iam/roles/roles.tf @@ -0,0 +1,45 @@ +# +# 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. +# + +# This Terraform configuration file is used to manage custom IAM roles +# in a Google Cloud Platform (GCP) project. It reads role definitions +# from YAML files located in the same directory and creates custom roles +# in the specified GCP project. + +locals { + role_files = fileset(path.module, "*.role.yaml") + roles_data = { + for f in local.role_files : + trimsuffix(f, ".role.yaml") => yamldecode(file("${path.module}/${f}")) + } +} + +variable "project_id" { + description = "The GCP project ID." + type = string +} + +resource "google_project_iam_custom_role" "custom_roles" { + for_each = local.roles_data + + project = var.project_id + role_id = each.value.role_id + title = each.value.title + description = lookup(each.value, "description", null) + permissions = each.value.permissions + stage = lookup(each.value, "stage", "GA") +} diff --git a/infra/iam/roles/roles_config.yaml b/infra/iam/roles/roles_config.yaml new file mode 100644 index 000000000000..1e94cdc2ccbd --- /dev/null +++ b/infra/iam/roles/roles_config.yaml @@ -0,0 +1,150 @@ +# +# 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. + +# Configuration for Apache Beam roles in GCP. +# This file defines the roles, their hierarchy, the services they can access and the roles they inherit from. + +project-id: "apache-beam-testing" # Default project ID for Apache Beam in GCP. +roles-prefix: "beam" # Prefix for the roles generated by this configuration. + +# Each custom role is defined here. +# name: The name of the role. +# hierarchy: The hierarchy level of the role, lower numbers indicate fewer permissions, +# the higher hierarchy level also gets the permissions of lower hierarchy levels. +# description: A brief description of the role. +# services: The list of services that the role can access. +# roles: The list of base roles that this role inherits permissions from. +# except_suffixes: A list of suffixes that indicate permissions that should not be included in the role. +# The suffixes are defined in the `suffixes` section below. +roles: + - name: viewer + hierarchy: 0 + description: "Viewer role for Apache Beam in GCP, it has read-only access to all services used by Beam." + services: + - artifactregistry + - biglake + - bigquery + - cloudasset + - cloudbuild + - cloudfunctions + - cloudsql + - compute + - container + - dataflow + - dataproc + - datastore + - dns + - firebase + - iam + - iap + - meshconfig + - monitoring + - pubsub + - redis + - resourcemanager + - secretmanager + - servicemanagement + - serviceusage + - spanner + - storage + - trafficdirector + roles: + - roles/viewer + except_suffixes: + - destructive + - name: writer + description: "Writer role for Apache Beam in GCP, it has additional permissions for managing resources." + hierarchy: 1 + services: + - cloudkms + - dataform + - dataplex + roles: + - roles/viewer + - roles/bigquery.user + - roles/bigquery.dataViewer + - roles/cloudsql.instanceUser + - roles/container.clusterViewer + - roles/container.developer + - roles/compute.networkViewer + - roles/datastore.user + - roles/trafficdirector.client + except_suffixes: + - destructive + - name: infra_manager + description: "Infrastructure Manager role for Apache Beam in GCP, it has permissions for managing infrastructure resources but not for destructive actions." + hierarchy: 2 + services: [] + roles: + - roles/cloudbuild.builds.editor + - roles/iam.serviceAccountTokenCreator + - roles/iam.serviceAccountUser + - roles/storage.objectCreator + - roles/storage.objectViewer + - roles/editor + except_suffixes: + - destructive + - name: admin + description: "Admin role for Apache Beam in GCP, it has permissions for managing all services used by Beam, it can perform destructive actions and access secrets." + hierarchy: 3 + services: + - secretmanager + roles: + - roles/editor + - roles/artifactregistry.admin + - roles/biglake.admin + - roles/bigquery.admin + - roles/cloudfunctions.admin + - roles/compute.admin + - roles/compute.instanceAdmin.v1 + - roles/compute.networkAdmin + - roles/container.admin + - roles/dataflow.admin + - roles/dataproc.admin + - roles/datastore.indexAdmin + - roles/dns.admin + - roles/firebase.admin + - roles/iam.roleAdmin + - roles/iam.securityAdmin + - roles/iam.serviceAccountAdmin + - roles/iam.workloadIdentityPoolAdmin + - roles/meshconfig.admin + - roles/monitoring.admin + - roles/pubsub.admin + - roles/redis.admin + - roles/resourcemanager.projectIamAdmin + - roles/secretmanager.admin + - roles/secretmanager.secretAccessor + - roles/secretmanager.viewer + - roles/servicemanagement.quotaAdmin + - roles/serviceusage.serviceUsageAdmin + - roles/spanner.admin + - roles/spanner.databaseAdmin + - roles/storage.admin + - roles/storage.objectAdmin + except_suffixes: [] + +suffixes: + - name: destructive + description: "Suffixes that indicate destructive actions in GCP." + values: + - ".delete" + - ".remove" + - ".destroy" + - ".purge" + - ".cancel" + - ".stop" + - ".terminate" diff --git a/infra/iam/roles/test_generate_roles.py b/infra/iam/roles/test_generate_roles.py new file mode 100644 index 000000000000..f5ebc5948e7c --- /dev/null +++ b/infra/iam/roles/test_generate_roles.py @@ -0,0 +1,82 @@ +# +# 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. + + # Tests for generate_roles.py + +import unittest +from unittest.mock import MagicMock +import sys +import types +import generate_roles + +# Patch yaml and google.cloud imports before importing the script +sys.modules['yaml'] = MagicMock() +sys.modules['google.cloud'] = types.SimpleNamespace(iam_admin_v1=MagicMock()) +sys.modules['google.api_core'] = types.SimpleNamespace(exceptions=MagicMock()) + +class TestGenerateRoles(unittest.TestCase): + def test_filter_permissions(self): + perms = [ + 'compute.instances.create', + 'compute.instances.delete', + 'storage.buckets.create', + 'storage.buckets.delete', + 'storage.objects.get', + 'bigquery.tables.get', + 'bigquery.tables.delete', + ] + allowed = ['storage', 'bigquery'] + denied = ['delete'] + filtered = generate_roles.filter_permissions(perms, allowed, denied) + self.assertIn('storage.buckets.create', filtered) + self.assertIn('storage.objects.get', filtered) + self.assertIn('bigquery.tables.get', filtered) + self.assertNotIn('storage.buckets.delete', filtered) + self.assertNotIn('bigquery.tables.delete', filtered) + self.assertNotIn('compute.instances.create', filtered) + self.assertNotIn('compute.instances.delete', filtered) + + def test_generate_role(self): + perms = {'a.b.c', 'd.e.f'} + role = generate_roles.generate_role('test_role', perms) + self.assertEqual(role['role_id'], 'test_role') + self.assertEqual(role['title'], 'test_role') + self.assertEqual(role['stage'], 'GA') + self.assertIn('a.b.c', role['permissions']) + self.assertIn('d.e.f', role['permissions']) + + def test_write_role_yaml(self): + import tempfile + import os + role_data = { + 'role_id': 'test_role', + 'title': 'test_role', + 'stage': 'GA', + 'description': 'desc', + 'permissions': ['a.b.c', 'd.e.f'], + } + with tempfile.TemporaryDirectory() as tmpdir: + filename = os.path.join(tmpdir, 'role.yaml') + generate_roles.ASF_LICENSE_HEADER = '' # Avoid header for test + generate_roles.write_role_yaml(filename, role_data) + with open(filename) as f: + content = f.read() + self.assertIn('role_id', content) + self.assertIn('a.b.c', content) + self.assertIn('d.e.f', content) + +if __name__ == '__main__': + unittest.main() diff --git a/infra/iam/users.tf b/infra/iam/users.tf index 32c26b8bcaa8..30d5bfddf8f8 100644 --- a/infra/iam/users.tf +++ b/infra/iam/users.tf @@ -46,7 +46,7 @@ resource "google_project_iam_member" "project_members" { } project = var.project_id role = each.value.role - member = "user:${each.value.email}" + member = can(regex(".*\\.gserviceaccount\\.com$", each.value.email)) ? "serviceAccount:${each.value.email}" : "user:${each.value.email}" dynamic "condition" { # Condition is only created if expiry_date is set diff --git a/infra/iam/users.yml b/infra/iam/users.yml index 06e9cea65e7d..d76eb5ae267d 100644 --- a/infra/iam/users.yml +++ b/infra/iam/users.yml @@ -544,4 +544,4 @@ - username: zhoufek email: zhoufek@google.com permissions: - - role: roles/editor + - role: roles/editor \ No newline at end of file diff --git a/infra/keys/README.md b/infra/keys/README.md new file mode 100644 index 000000000000..f7bf1d927c16 --- /dev/null +++ b/infra/keys/README.md @@ -0,0 +1,103 @@ + + +# Service Account Management + +This module is used to manage Google Cloud service accounts, including creating, retrieving, enabling, and deleting service accounts and their keys. It uses the Google Cloud IAM API to perform these operations. + +## User usage + +We use the `keys.py` script to manage service account keys. In order to use this script you will need to: + +1. Generate a change over `keys.yaml` file, where your email address is listed as an authorized user for the service accounts you want to manage. +2. Open a pull request with the changes to the `keys.yaml` file. The change will be reviewed and merged by the Infra team. +3. Once your changes are merged, install the required python packages: `pip install -r requirements.txt` and authenticate with Google Cloud using the `gcloud auth application-default login` command. +4. Run `keys.py --get-key ` to get the latest key for a service account. The key will be printed to the console, and you can use it to authenticate with Google Cloud services. + +> Remember this keys are rotated regularly, so you will need to run the command again to get the latest key after a rotation, the rotation days are defined in the `config.yaml` file. + +## Administrative usage + +This section is intended for developers who need to manage service accounts and their keys at a higher level. A regular user should not need to access this section. + +### Prerequisites + +- Google Cloud SDK installed and configured. +- Appropriate permissions to manage service accounts and secrets in your Google Cloud project. +- Required Python packages installed (see requirements.txt). + +### How it works + +This module provide a script `keys.py` that allows you to manage the service accounts and their keys. This script is run automatically by a GitHub Action to ensure that service account keys are rotated regularly and that the latest keys are available for authorized users. It is also run every time a PR is merged over the `keys.yaml` file to ensure that the service accounts, their keys and authorized users are up to date. + +### Automation with GitHub Actions + +A GitHub Actions workflow is set up to automate the execution of the `keys.py` script. This workflow is defined in `.github/workflows/beam_Infrastructure_ServiceAccountKeys.yml`. + +The workflow is triggered automatically on the following events: +- A push to the `main` branch that includes changes to the `infra/keys/keys.yaml` file. +- A manual trigger (`workflow_dispatch`) by a developer. + +When triggered, the workflow runs the `python keys.py --cron` command, which handles the creation and rotation of service account keys based on the configuration in `keys.yaml` and `config.yaml`. + +### Files + +#### config.yaml + +This file contains configuration settings for the service account management, including project ID, key rotation settings, and logging configuration. + +#### keys.yaml + +All the service accounts are managed through a configuration file in YAML format, `keys.yaml`. This file contains the necessary information about each service account, including its ID, display name, and authorized users. + +```yaml +service_accounts: + - account_id: my-service-account + display_name: My Service Account + authorized_users: + - email: user1@example.com + - email: user2@example.com +``` + +Where: + +- `account_id`: The unique identifier for the service account. The email address of the service account will be `@.iam.gserviceaccount.com`. +- `display_name`: A human-readable name for the service account. +- `authorized_users`: A list of users who will be granted access to the service account's keys. Each user is specified by their email address. This users will be able to retrieve the keys and act on behalf of the service account. + +Service accounts are created the first time the cron is run, or when the `keys.yaml` file is updated with a new service account. The script checks if the service account already exists in Google Cloud, and if not, it creates it. If the service account already exists but it is not managed by the Secret Manager, it creates a new key, storing it in the Secret Manager and ignore the rest. This ensures that the service account is always up to date with the latest keys and authorized users. + +### Rotation + +Service account keys should be rotated regularly to maintain security. To automate key rotation, you can set up a cron job that runs the command: + +```bash +python keys.py --cron +``` + +This will rotate keys for all service accounts defined in the `keys.yaml` file that have achieved the age threshold (e.g., 30 days). The age threshold can be adjusted in the `config.yaml` file. + +### Retrieval + +To retrieve the latest service account key, use the `--get-key` flag: + +```bash +python keys.py --get-key my-service-account +``` + diff --git a/infra/keys/config.yaml b/infra/keys/config.yaml new file mode 100644 index 000000000000..e28c3a589537 --- /dev/null +++ b/infra/keys/config.yaml @@ -0,0 +1,34 @@ +# 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. + +# This is the configuration file for the secrets rotation service. +# It defines the parameters for the secrets rotation process. + +# GENERAL CONFIGURATION + +# The project ID where the secrets rotation service will run +project_id: "apache-beam-testing" + +# Secret service configuration + +# Default secret rotation interval in days +rotation_interval: 7 +# Time the disabled secret versions will be kept before deletion +grace_period: 2 + +# LOGGING + +# Logging level for the secrets rotation service +logging_level: "DEBUG" # Options: DEBUG, INFO, WARNING, ERROR, CRITICAL diff --git a/infra/keys/keys.py b/infra/keys/keys.py new file mode 100644 index 000000000000..c06307ecb24f --- /dev/null +++ b/infra/keys/keys.py @@ -0,0 +1,383 @@ +# 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. + +import traceback +import yaml +import logging +import argparse +import sys +from typing import List, TypedDict +from google.api_core.exceptions import PermissionDenied +# Importing custom modules +from secret_manager import SecretManager +from service_account import ServiceAccountManager + + +# --- Configuration --- +CONFIG_FILE = 'config.yaml' +KEYS_FILE = 'keys.yaml' + +class ConfigDict(TypedDict): + project_id: str + rotation_interval: int + grace_period: int + logging_level: str + +class AuthorizedUser(TypedDict): + email: str + +class ServiceAccount(TypedDict): + account_id: str + display_name: str + authorized_users: List[AuthorizedUser] + +class ServiceAccountsConfig(TypedDict): + service_accounts: List[ServiceAccount] + +def load_config() -> ConfigDict: + """Loads the configuration from the YAML file.""" + with open(CONFIG_FILE, 'r') as f: + config = yaml.safe_load(f) + + if not config: + raise ValueError("Configuration file is empty or invalid.") + + required_keys = set(['project_id', 'rotation_interval', 'grace_period']) + missing_keys = required_keys - config.keys() + if missing_keys: + raise ValueError(f"Missing required configuration keys: {', '.join(missing_keys)}") + + if not isinstance(config['rotation_interval'], int) or config['rotation_interval'] <= 0: + raise ValueError("Configuration 'rotation_interval' must be a positive integer.") + if not isinstance(config['grace_period'], int) or config['grace_period'] < 0: + raise ValueError("Configuration 'grace_period' must be a non-negative integer.") + if 'logging_level' in config: + if not isinstance(config['logging_level'], str) or config['logging_level'].strip() not in logging._nameToLevel: + raise ValueError("Configuration 'logging_level' must be one of: " + ", ".join(logging._nameToLevel.keys())) + else: + config['logging_level'] = 'INFO' + + return config + +def load_service_accounts_config() -> ServiceAccountsConfig: + """Loads the service accounts configuration from the YAML file.""" + with open(KEYS_FILE, 'r') as f: + service_accounts_config = yaml.safe_load(f) + + if not service_accounts_config or 'service_accounts' not in service_accounts_config: + raise ValueError("Service accounts configuration file is empty or invalid.") + + if not isinstance(service_accounts_config['service_accounts'], list): + raise ValueError("Service accounts configuration must be a list of service accounts.") + + for account in service_accounts_config['service_accounts']: + if 'account_id' not in account or 'display_name' not in account: + raise ValueError("Each service account must have 'account_id' and 'display_name'.") + if 'authorized_users' not in account or not isinstance(account['authorized_users'], list): + raise ValueError("Each service account must have a list of 'authorized_users'.") + + return service_accounts_config + +def parse_arguments(): + """Parse command line arguments.""" + parser = argparse.ArgumentParser( + description="KeyService - GCP Service Account Key Management", + formatter_class=argparse.RawDescriptionHelpFormatter, + epilog=""" +Examples: + python keys.py --cron # Run key rotation for accounts that need it, ran only by cron job + python keys.py --cron-dry-run # Run a dry run of the key rotation cron job + python keys.py --get-key my-sa # Get the latest key for service account 'my-sa', ran by users + """ + ) + + group = parser.add_mutually_exclusive_group() + group.add_argument( + '--cron', + action='store_true', + help='Run the cron job to rotate keys that require rotation' + ) + group.add_argument( + '--cron-dry-run', + action='store_true', + help='Run a dry run of the cron job to see what would be rotated' + ) + group.add_argument( + '--get-key', + metavar='ACCOUNT_ID', + type=str, + help='Get the latest key for the specified service account ID' + ) + + return parser.parse_args() + +class KeyService: + """Service to manage GCP API keys rotation.""" + + # Configuration + project_id: str + service_accounts: List[ServiceAccount] + enable_logging: bool + + # Clients + secret_manager_client: SecretManager + service_account_manager: ServiceAccountManager + logger: logging.Logger + + def __init__(self, config: ConfigDict, service_accounts_config: ServiceAccountsConfig, enable_logging: bool = True) -> None: + """ + Initializes the KeyService with the provided configuration. + + Args: + config (ConfigDict): Configuration dictionary containing: + - project_id: GCP project ID + - rotation_interval: Interval in days for secret rotation + - max_versions_to_keep: Maximum number of secret versions to keep + - bucket_name: GCS bucket name for logging + - log_file_prefix: Prefix for log file names + - logging_level: Logging level (e.g., 'INFO', 'DEBUG') + service_accounts_config (ServiceAccountsConfig): Configuration for service accounts. + - service_accounts: List of service accounts to manage and their configuration + enable_logging (bool): Whether to enable logging. Defaults to True. + Raises: + ValueError: If any required configuration parameter is missing. + """ + + self.project_id = config['project_id'] + rotation_interval = config['rotation_interval'] + grace_period = config['grace_period'] + logging_level = config['logging_level'] + + self.service_accounts = service_accounts_config['service_accounts'] + self.enable_logging = enable_logging + + self.logger = logging.getLogger("KeyService") + if self.enable_logging: + self.logger.setLevel(logging_level) + handler = logging.StreamHandler(sys.stdout) + formatter = logging.Formatter('%(asctime)s - %(name)s - %(levelname)s - %(message)s') + handler.setFormatter(formatter) + self.logger.addHandler(handler) + else: + # Create a null logger that doesn't actually log anything + self.logger.setLevel(logging.CRITICAL + 1) # Set to a level higher than CRITICAL to disable all logging + + self.secret_manager_client = SecretManager(self.project_id, self.logger, rotation_interval, grace_period) + self.service_account_manager = ServiceAccountManager(self.project_id, self.logger) + + if self.enable_logging: + self.logger.info(f"Initialized KeyService for project: {self.project_id}") + + def _start_all_service_accounts(self) -> None: + """ + Reads the service accounts configuration and checks for service accounts. + + 1. If a service account exists and is managed, it checks if the secret exists and updates access if needed. + 2. If the service account exists but the secret does not, it creates the secret and clears the service account + keys as now keys will be managed by the Secret Manager. + 3. If neither the service account nor the secret exists, it creates and initializes both. + 4. If any other case is encountered, it logs an error and skips the account. + """ + + self.logger.debug("Creating service accounts if they do not exist") + for account in self.service_accounts: + account_id = account['account_id'] + authorized_users = [user['email'] for user in account.get('authorized_users', [])] + + try: + secret_name = f"{account_id}-key" + # If service account and secret exists and is managed, just check permissions + if self.service_account_manager._service_account_exists(account_id) and self.secret_manager_client._secret_is_managed(secret_name): + self.logger.debug(f"Service account {account_id} and secret {secret_name} already exist and are managed") + if self.secret_manager_client.is_different_user_access(secret_name, authorized_users): + self.logger.debug(f"Updating access policy for secret {secret_name}") + self.secret_manager_client.update_secret_access(secret_name, authorized_users) + + # If the service account exists but the secret does not, create the secret and a key and ignore the existing keys + elif self.service_account_manager._service_account_exists(account_id) and not self.secret_manager_client._secret_exists(secret_name): + self.logger.debug(f"Service account {account_id} exists but secret {secret_name} does not, creating secret and a new key") + self.secret_manager_client.create_secret(secret_name) + self.secret_manager_client.update_secret_access(secret_name, authorized_users) + + new_key = self.service_account_manager.create_service_account_key(account_id) + new_key_id = new_key.name.split('/')[-1] + self.secret_manager_client.add_secret_version(secret_name, new_key_id, new_key.private_key_data) + + # If neither secret nor service account exists, create and initialize both + elif not self.service_account_manager._service_account_exists(account_id) and not self.secret_manager_client._secret_exists(secret_name): + self.logger.debug(f"Service account {account_id} and secret {secret_name} do not exist, creating both") + display_name = account['display_name'] + + self.service_account_manager.create_service_account(account_id, display_name) + + secret_name = self.secret_manager_client.create_secret(secret_name) + self.secret_manager_client.update_secret_access(secret_name, authorized_users) + + new_key = self.service_account_manager.create_service_account_key(account_id) + new_key_id = new_key.name.split('/')[-1] + self.secret_manager_client.add_secret_version(secret_name, new_key_id, new_key.private_key_data) + + else: + # Any other case is not supported + self.logger.error(f"Unexpected state for service account {account_id}") + + except Exception as e: + self.logger.error(f"Error creating service account or secret for {account_id}: {e}") + + def cron(self, dry_run: bool = False) -> None: + """ + Cron job to rotate service account keys and secrets. + + This method should be called periodically based on the rotation interval. + It will: + + 1. Check each service account to see if its key is due for rotation. + 1.1. If the key is due for rotation, it will rotate the key and update the secret in Secret Manager. + 1.2. If the key is not due for rotation, it will log that no action is needed. + 2. Check for keys that have expired the grace period and delete them from both the service account and Secret Manager. + + Args: + dry_run (bool): If True, the method will only log the actions that would be taken. + """ + + if dry_run: + self.logger.info("Starting cron job DRY RUN for service account key rotation") + else: + self.logger.info("Starting cron job for service account key rotation") + + if not dry_run: + self._start_all_service_accounts() + + for account in self.service_accounts: + account_id = account['account_id'] + secret_name = f"{account_id}-key" + try: + if self.secret_manager_client._is_key_rotation_due(secret_name): + if dry_run: + self.logger.info(f"[DRY RUN] Service account key for {account_id} is due for rotation, would rotate key.") + else: + self.logger.info(f"Service account key for {account_id} is due for rotation, rotating key") + new_key = self.service_account_manager.create_service_account_key(account_id) + new_key_id = new_key.name.split('/')[-1] + self.secret_manager_client.add_secret_version(secret_name, new_key_id, new_key.private_key_data) + else: + self.logger.debug(f"Service account key for {account_id} is not due for rotation") + except Exception as e: + self.logger.error(f"Error during cron job for service account {account_id}: {e}") + + # Check for keys that have expired the grace period and delete them + + self.logger.info("Checking for keys that have expired the grace period") + keys_to_delete = self.secret_manager_client.cron() + for secret_id, key_ids in keys_to_delete: + try: + for key_id in key_ids: + if dry_run: + self.logger.info(f"[DRY RUN] Would delete expired key {key_id} for secret {secret_id}") + else: + self.logger.info(f"Deleting expired key {key_id} for secret {secret_id}") + self.service_account_manager.delete_service_account_key(secret_id, key_id) + except Exception as e: + self.logger.error(f"Error deleting expired keys for secret {secret_id}: {e}") + continue + + if dry_run: + self.logger.info("Cron job DRY RUN for service account key rotation completed") + else: + self.logger.info("Cron job for service account key rotation completed") + + def get_latest_service_account_key(self, account_id: str) -> str: + """ + Retrieves the latest service account key for a given service account. + + Args: + account_id (str): The ID of the service account to retrieve the key for. + + Returns: + str: The latest service account key. + """ + self.logger.info(f"Retrieving latest service account key for {account_id}") + try: + secret_name = f"{account_id}-key" + key_bytes = self.secret_manager_client.get_latest_secret_version(secret_name) + if not key_bytes: + self.logger.warning(f"No key found for service account {account_id}.") + raise ValueError(f"No key found for service account {account_id}.") + + self.logger.debug(f"Latest service account key for {account_id} retrieved successfully.") + return key_bytes[1].decode('utf-8') + except Exception as e: + self.logger.error(f"Error retrieving latest service account key for {account_id}: {e}") + return "" + +def main(): + """ + Main function to run the KeyService. + + Loads configuration, initializes the KeyService, and handles CLI arguments. + """ + args = parse_arguments() + + key_service = None + try: + config = load_config() + service_accounts_config = load_service_accounts_config() + + if args.cron or args.cron_dry_run: + is_dry_run = args.cron_dry_run + run_type = "dry run" if is_dry_run else "job" + print(f"Running cron {run_type} for key rotation...") + key_service = KeyService(config, service_accounts_config) + key_service.cron(dry_run=is_dry_run) + print(f"Cron {run_type} completed successfully.") + + elif args.get_key: + account_id = args.get_key + # If just a user getting the key, disable logging + key_service = KeyService(config, service_accounts_config, enable_logging=False) + print(f"Retrieving latest key for service account: {account_id}") + + # Validate that the account exists in configuration + account_ids = [account['account_id'] for account in service_accounts_config['service_accounts']] + if account_id not in account_ids: + print(f"Error: Service account '{account_id}' not found in configuration.") + print(f"Available accounts: {', '.join(account_ids)}") + sys.exit(1) + + try: + key = key_service.get_latest_service_account_key(account_id) + if key: + print(f"Latest key for {account_id}:") + print(key) + else: + print(f"No key found for service account: {account_id}") + sys.exit(1) + except PermissionDenied as e: + print(f"Permission denied when accessing the key for {account_id}: {e}") + sys.exit(1) + except Exception as e: + print(f"Error retrieving key for {account_id}: {e}") + sys.exit(1) + else: + print("You must specify either --cron to run the cron job or --get-key to retrieve a key.") + + except Exception as e: + print(f"An error occurred: {e}") + logging.error(f"An error occurred: {e}") + logging.error(f"Full traceback: {traceback.format_exc()}") + sys.exit(1) + +if __name__ == "__main__": + main() \ No newline at end of file diff --git a/infra/keys/keys.yaml b/infra/keys/keys.yaml new file mode 100644 index 000000000000..269a2841d91a --- /dev/null +++ b/infra/keys/keys.yaml @@ -0,0 +1,26 @@ +# 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. + +# Service Account Keys +# This file contains the service account for the project, the account id +# and the users authorized to use it +# service_accounts: +# - account_id: account_id +# display_name: account_@project_id.iam.gserviceaccount.com +# authorized_users: +# - email: "user1@google.com" +# - email: "user2@google.com" + +service_accounts: [] diff --git a/infra/keys/requirements.txt b/infra/keys/requirements.txt new file mode 100644 index 000000000000..98814332cf01 --- /dev/null +++ b/infra/keys/requirements.txt @@ -0,0 +1,23 @@ +# +# 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. + +# This file is used to install the dependencies for the infrastructure + +PyYAML==6.0.2 +google-cloud-iam==2.19.1 +google-cloud-secret-manager==2.24.0 +google-cloud-storage==3.2.0 +google-crc32c==1.7.1 diff --git a/infra/keys/secret_manager.py b/infra/keys/secret_manager.py new file mode 100644 index 000000000000..102b629bbfee --- /dev/null +++ b/infra/keys/secret_manager.py @@ -0,0 +1,787 @@ +# 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. + +import google_crc32c +import logging +import time +from datetime import datetime, timezone, timedelta +from google.cloud import secretmanager +from typing import List, Union, Tuple, Dict + +# What the "created_by" label is set to for secrets created by this service. +SECRET_MANAGER_LABEL = "beam-infra-secret-manager" + +class SecretManagerLoggerAdapter(logging.LoggerAdapter): + """Logger adapter that adds a prefix to all log messages.""" + + def process(self, msg, kwargs): + return f"[SecretManager] {msg}", kwargs + +class SecretManager: + """Service to manage GCP API keys rotation.""" + + project_id: str # The GCP project ID where secrets are managed + rotation_interval: int # The interval (in days) at which to rotate secrets + grace_period: int # The grace period (in days) before a secret is considered for rotation + max_retries: int # The maximum number of retries for API calls + client: secretmanager.SecretManagerServiceClient # GCP Secret Manager client + logger: Union[logging.Logger, logging.LoggerAdapter] # Logger for logging messages + + def __init__(self, project_id: str, logger: logging.Logger, rotation_interval: int = 30, grace_period: int = 7, max_retries: int = 3) -> None: + self.project_id = project_id + self.rotation_interval = rotation_interval + self.grace_period = grace_period + self.max_retries = max_retries + self.client = secretmanager.SecretManagerServiceClient() + self.logger = SecretManagerLoggerAdapter(logger, {}) + self.logger.info(f"Initialized SecretManager for project '{self.project_id}'") + + def _get_secret_ids(self) -> List[str]: + """ + Retrieves the list of secrets from the Secret Manager and populates the `secrets_ids` list. + This method filters secrets based on a specific label indicating they were created by this service. + + Returns: + List[str]: A list of secret IDs that were created by this service. + """ + self.logger.debug(f"Retrieving secrets with the label from project '{self.project_id}'") + secret_ids = [] + + try: + for secret in self.client.list_secrets(request={"parent": f"projects/{self.project_id}"}): + secret_id = secret.name.split("/")[-1] + if "created_by" in secret.labels and secret.labels["created_by"] == SECRET_MANAGER_LABEL: + secret_ids.append(secret_id) + except Exception as e: + self.logger.error(f"Error retrieving secrets: {e}") + + self.logger.debug(f"Found {len(secret_ids)} secrets created by {SECRET_MANAGER_LABEL} in project '{self.project_id}'") + return secret_ids + + def _secret_exists(self, secret_id: str) -> bool: + """ + Checks if a secret with the given ID exists. + + Args: + secret_id (str): The ID of the secret to check. + Returns: + bool: True if the secret exists, False otherwise. + """ + self.logger.debug(f"Checking if secret '{secret_id}' exists") + try: + name = self.client.secret_path(self.project_id, secret_id) + self.client.get_secret(request={"name": name}) + self.logger.debug(f"Secret '{secret_id}' exists") + return True + except Exception as e: + self.logger.debug(f"Secret '{secret_id}' does not exist: {e}") + return False + + def _secret_is_managed(self, secret_id: str) -> bool: + """ + Checks if a secret with the given ID exists and is managed by this service. + + Args: + secret_id (str): The ID of the secret to check. + Returns: + bool: True if the secret is managed by this service, False otherwise. + """ + self.logger.debug(f"Checking if secret '{secret_id}' exists and is managed by {SECRET_MANAGER_LABEL}") + if not self._secret_exists(secret_id): + self.logger.debug(f"Secret '{secret_id}' does not exist, cannot be managed") + return False + + name = self.client.secret_path(self.project_id, secret_id) + secret = self.client.get_secret(request={"name": name}) + + is_managed = "created_by" in secret.labels and secret.labels["created_by"] == SECRET_MANAGER_LABEL + self.logger.debug(f"Secret '{secret_id}' is managed by {SECRET_MANAGER_LABEL}: {is_managed}") + return is_managed + + def create_secret(self, secret_id: str) -> str: + """ + Create a new secret with the given name. A secret is a logical wrapper + around a collection of secret versions. Secret versions hold the actual + secret material. This method creates a new secret with automatic replication + and labels for tracking. + + Args: + secret_id (str): The ID to assign to the new secret. This ID must be unique within the project. + Returns: + str: The secret path of the newly created secret. + """ + if self._secret_is_managed(secret_id): + self.logger.debug(f"Secret '{secret_id}' already exists, returning existing secret path") + name = self.client.secret_path(self.project_id, secret_id) + return name + + self.logger.info(f"Creating new secret '{secret_id}' with rotation interval of {self.rotation_interval} days") + response = self.client.create_secret( + request={ + "parent": f"projects/{self.project_id}", + "secret_id": f"{secret_id}", + "secret": { + "replication": { + "automatic": {} + }, + "labels": { + "created_by": SECRET_MANAGER_LABEL, + "created_at": datetime.now(timezone.utc).strftime("%Y%m%d_%H%M%S"), + "rotation_interval_days": str(self.rotation_interval), + "grace_period_days": str(self.grace_period), + "last_version_created_at": datetime.now(timezone.utc).strftime("%Y%m%d_%H%M%S"), + } + } + } + ) + + # created_by : This label is used to identify secrets created by this service. + # created_at : This label stores the timestamp when the secret was created. + # rotation_interval_days : This label specifies the rotation interval for the secret. + # grace_period_days : This label specifies the grace period for the secret. + # last_version_created_at : This label stores the timestamp when the last version of the secret was created, this + # helps with the rotation and grace period calculations. + + # Wait for the secret to be created + self.logger.debug(f"Waiting for secret '{secret_id}' to be created") + delay = 1 + for _ in range(self.max_retries): + if self._secret_is_managed(secret_id): + self.logger.debug(f"Secret '{secret_id}' is now available") + break + self.logger.debug(f"Secret '{secret_id}' not found, retrying in {delay} seconds") + time.sleep(delay) + delay *= 2 + else: + error_msg = f"Could not verify creation of secret '{secret_id}' after {self.max_retries} retries." + self.logger.error(error_msg) + raise RuntimeError(error_msg) + + self.logger.info(f"Successfully created secret '{secret_id}'") + return response.name + + def get_secret(self, secret_id: str) -> secretmanager.Secret: + """ + Retrieves the specified secret by its ID. + + Args: + secret_id (str): The ID of the secret to retrieve. + Returns: + secretmanager.Secret: The requested secret. + """ + self.logger.info(f"Retrieving secret '{secret_id}'") + + if not self._secret_exists(secret_id): + error_msg = f"Secret {secret_id} does not exist. Please create it first." + self.logger.error(error_msg) + raise ValueError(error_msg) + if not self._secret_is_managed(secret_id): + error_msg = f"Secret {secret_id} is not managed by this service." + self.logger.error(error_msg) + raise ValueError(error_msg) + + name = self.client.secret_path(self.project_id, secret_id) + return self.client.get_secret(request={"name": name}) + + def delete_secret(self, secret_id: str) -> None: + """ + Deletes the specified secret and all its versions. + + Args: + secret_id (str): The ID of the secret to delete. + """ + if not self._secret_is_managed(secret_id): + self.logger.debug(f"Secret '{secret_id}' is not managed by this service, cannot delete") + return + + self.logger.info(f"Deleting secret '{secret_id}' and all its versions") + name = self.client.secret_path(self.project_id, secret_id) + self.client.delete_secret(request={"name": name}) + + # Wait for the secret to be deleted + self.logger.debug(f"Waiting for secret '{secret_id}' to be deleted") + delay = 1 + for _ in range(self.max_retries): + if not self._secret_exists(secret_id): + self.logger.debug(f"Secret '{secret_id}' is now deleted") + break + self.logger.debug(f"Secret '{secret_id}' still exists, retrying in {delay} seconds") + time.sleep(delay) + delay *= 2 + else: + error_msg = f"Could not verify deletion of secret '{secret_id}' after {self.max_retries} retries." + self.logger.error(error_msg) + raise RuntimeError(error_msg) + + self.logger.info(f"Successfully deleted secret '{secret_id}'") + + def is_different_user_access(self, secret_id: str, allowed_users: List[str]) -> bool: + """ + Checks if the current access policy of a secret allows only the specified users to read it. + This is used to determine if an update is needed. + + Args: + secret_id (str): The ID of the secret to check access for. + allowed_users (List[str]): A list of user emails to check against the current access policy. + Returns: + bool: True if the current access policy is different from the specified users, False otherwise. + """ + self.logger.debug(f"Checking if access for secret '{secret_id}' differs from allowed users: {allowed_users}") + + if not self._secret_is_managed(secret_id): + self.logger.debug(f"Secret '{secret_id}' is not managed by this service, cannot check access") + return True + + accessor_role = "roles/secretmanager.secretAccessor" + resource_name = self.client.secret_path(self.project_id, secret_id) + + try: + policy = self.client.get_iam_policy(request={"resource": resource_name}) + except Exception as e: + self.logger.error(f"Failed to get IAM policy for secret '{secret_id}': {e}") + return True + + current_members = set() + for binding in policy.bindings: + if binding.role == accessor_role: + current_members.update(binding.members) + + allowed_members = {f"user:{user_email}" for user_email in allowed_users} + + is_different = current_members != allowed_members + self.logger.debug(f"Current members: {current_members}") + self.logger.debug(f"Allowed members: {allowed_members}") + self.logger.debug(f"Access for secret '{secret_id}' differs: {is_different}") + return is_different + + def update_secret_access(self, secret_id: str, allowed_users: List[str]) -> None: + """ + Updates the access policy of a secret to allow only the specified users to read it. + Any existing users will be removed and replaced with the new list. + + Args: + secret_id (str): The ID of the secret to update access for. + allowed_users (List[str]): A list of user emails to grant read access to. + """ + self.logger.debug(f"Updating access for secret '{secret_id}' to allow users: {allowed_users}") + + if not self._secret_is_managed(secret_id): + error_msg = f"Secret {secret_id} is not managed by this service, cannot update access." + self.logger.error(error_msg) + raise ValueError(error_msg) + + accessor_role = "roles/secretmanager.secretAccessor" + resource_name = self.client.secret_path(self.project_id, secret_id) + policy = self.client.get_iam_policy(request={"resource": resource_name}) + + members = [f"user:{user_email}" for user_email in allowed_users] + + binding_found = False + for binding in policy.bindings: + if binding.role == accessor_role: + binding.members[:] = members + self.logger.debug(f"Replaced members for role '{accessor_role}' in secret '{secret_id}' with: {allowed_users}") + binding_found = True + break + + if not binding_found: + policy.bindings.add( + role=accessor_role, + members=members + ) + self.logger.debug(f"Created new binding for role '{accessor_role}' in secret '{secret_id}'") + + self.client.set_iam_policy( + request={ + "resource": resource_name, + "policy": policy + } + ) + + self.logger.info(f"Successfully updated access for secret '{secret_id}' to allow users: {allowed_users}") + + def _get_secret_versions(self, secret_id: str) -> List[secretmanager.SecretVersion]: + """ + Retrieves all versions of a secret. + + Args: + secret_id (str): The ID of the secret to list versions for. + Returns: + List[secretmanager.SecretVersion]: A list of secret versions. + """ + self.logger.debug(f"Retrieving versions for secret '{secret_id}'") + + if not self._secret_is_managed(secret_id): + self.logger.debug(f"Secret '{secret_id}' is not managed by this service, cannot retrieve versions") + return [] + + parent = self.client.secret_path(self.project_id, secret_id) + versions = list(self.client.list_secret_versions(request={"parent": parent})) + self.logger.debug(f"Found {len(versions)} versions for secret '{secret_id}'") + return versions + + def _secret_version_exists(self, secret_id: str, version_id: str) -> bool: + """ + Checks if a specific version of a secret exists. + + Args: + secret_id (str): The ID of the secret to check. + version_id (str): The ID of the version to check. + Returns: + bool: True if the version exists, False otherwise. + """ + self.logger.debug(f"Checking if version '{version_id}' exists for secret '{secret_id}'") + if not self._secret_is_managed(secret_id): + self.logger.debug(f"Secret '{secret_id}' is not managed by this service, cannot check version existence") + return False + + versions = self._get_secret_versions(secret_id) + exists = any(version.name.split("/")[-1] == version_id for version in versions) + self.logger.debug(f"Version '{version_id}' exists: {exists}") + return exists + + def _secret_version_is_enabled(self, secret_id: str, version_id: str) -> bool: + """ + Checks if a specific version of a secret is enabled. + + Args: + secret_id (str): The ID of the secret to check. + version_id (str): The ID of the version to check. + Returns: + bool: True if the version is enabled, False otherwise. + """ + self.logger.debug(f"Checking if version '{version_id}' of secret '{secret_id}' is enabled") + if not self._secret_is_managed(secret_id): + self.logger.debug(f"Secret '{secret_id}' is not managed by this service, version cannot be enabled") + return False + + versions = self._get_secret_versions(secret_id) + for version in versions: + if version.name.split("/")[-1] == version_id: + is_enabled = version.state == secretmanager.SecretVersion.State.ENABLED + self.logger.debug(f"Version '{version_id}' is enabled: {is_enabled}") + return is_enabled + self.logger.debug(f"Version '{version_id}' does not exist for secret '{secret_id}'") + return False + + def _secret_version_is_destroyed(self, secret_id: str, version_id: str) -> bool: + """ + Checks if a specific version of a secret is destroyed. + + Args: + secret_id (str): The ID of the secret to check. + version_id (str): The ID of the version to check. + Returns: + bool: True if the version is destroyed, False otherwise. + """ + self.logger.debug(f"Checking if version '{version_id}' of secret '{secret_id}' is destroyed") + if not self._secret_is_managed(secret_id): + self.logger.debug(f"Secret '{secret_id}' is not managed by this service, version cannot be destroyed") + return False + + versions = self._get_secret_versions(secret_id) + for version in versions: + if version.name.split("/")[-1] == version_id: + is_destroyed = version.state == secretmanager.SecretVersion.State.DESTROYED + self.logger.debug(f"Version '{version_id}' is destroyed: {is_destroyed}") + return is_destroyed + self.logger.debug(f"Version '{version_id}' does not exist for secret '{secret_id}'") + return False + + def _get_latest_secret_version_id(self, secret_id: str) -> str: + """ + Retrieves the latest enabled version of a secret. + + Args: + secret_id (str): The ID of the secret to retrieve the latest version for. + Returns: + str: The name of the latest secret version. + """ + self.logger.debug(f"Retrieving latest enabled version of secret '{secret_id}'") + if not self._secret_is_managed(secret_id): + error_msg = f"Secret {secret_id} does not exist or is not managed by this service, cannot retrieve latest version." + self.logger.error(error_msg) + raise ValueError(error_msg) + + for version in self._get_secret_versions(secret_id): + if version.state == secretmanager.SecretVersion.State.ENABLED: + version_id = version.name.split("/")[-1] + self.logger.debug(f"Found latest enabled version '{version_id}' for secret '{secret_id}'") + return version_id + error_msg = f"No enabled versions found for secret {secret_id}." + self.logger.error(error_msg) + raise ValueError(error_msg) + + def _is_key_rotation_due(self, secret_id: str) -> bool: + """ + Checks if the key rotation is due based on the last version created timestamp. + + Args: + secret_id (str): The ID of the secret to check. + Returns: + bool: True if the key rotation is due, False otherwise. + """ + self.logger.debug(f"Checking if key rotation is due for secret '{secret_id}'") + if not self._secret_is_managed(secret_id): + error_msg = f"Secret {secret_id} does not exist or is not managed by this service, cannot check rotation." + self.logger.error(error_msg) + raise ValueError(error_msg) + + secret = self.get_secret(secret_id) + last_version_created_at = secret.labels["last_version_created_at"] + last_version_date = datetime.strptime(last_version_created_at, "%Y%m%d_%H%M%S").replace(tzinfo=timezone.utc) + due_date = last_version_date + timedelta(days=self.rotation_interval) + + is_due = datetime.now(timezone.utc) >= due_date + self.logger.debug(f"Key rotation due for secret '{secret_id}': {is_due}") + return is_due + + def add_secret_version(self, secret_id: str, data_id: str, payload: Union[bytes, str]) -> str: + """ + Adds a new version to the specified secret with the given data ID and payload. + If the secret does not exist, it will be created first. All previous versions will be disabled. + + Args: + secret_id (str): The ID of the secret to which the version will be added. + data_id (str): The ID of the data to be stored in the new version. + payload (bytes): The secret data to be stored in the new version. + Returns: + str: The name of the newly created secret version. + """ + self.logger.info(f"Adding new version to secret '{secret_id}'") + + secret_path = self.create_secret(secret_id) + + if not isinstance(payload, (bytes, str)): + error_msg = "Payload must be a bytes object or a string that can be encoded to bytes." + self.logger.error(error_msg) + raise TypeError(error_msg) + + # Join data_id and payload to form the payload + if isinstance(payload, str): + payload = f"{data_id}:{payload}" + else: + payload = f"{data_id}:{payload.decode('utf-8')}" if isinstance(payload, bytes) else payload + + # Ensure payload is bytes + payload_bytes = payload.encode('utf-8') if isinstance(payload, str) else payload + + crc32c = google_crc32c.Checksum() + crc32c.update(payload_bytes) + + self.logger.debug(f"Creating secret version with CRC32C checksum") + response = self.client.add_secret_version( + request={ + "parent": secret_path, + "payload": { + "data": payload_bytes, + "data_crc32c": int(crc32c.hexdigest(), 16), + } + } + ) + + version_id = response.name.split("/")[-1] + + # Update the last version created timestamp + self.logger.debug(f"Updating last version created timestamp for secret '{secret_id}'") + secret_obj = self.get_secret(secret_id) + labels = dict(secret_obj.labels) + labels["last_version_created_at"] = datetime.now(timezone.utc).strftime("%Y%m%d_%H%M%S") + secret = {"name": secret_obj.name, "labels": labels} + update_mask = {"paths": ["labels"]} + self.client.update_secret(request={"secret": secret, "update_mask": update_mask}) + + # Wait for the new version to be available + self.logger.debug(f"Waiting for new version '{version_id}' of secret '{secret_id}' to be available") + delay = 1 + for _ in range(self.max_retries): + if self._secret_version_exists(secret_id, version_id): + self.logger.debug(f"Version '{version_id}' of secret '{secret_id}' is now available") + break + self.logger.debug(f"Version '{version_id}' of secret '{secret_id}' not found, retrying in {delay} seconds") + time.sleep(delay) + delay *= 2 + else: + error_msg = f"Could not verify creation of secret version '{version_id}' after {self.max_retries} retries." + self.logger.error(error_msg) + raise RuntimeError(error_msg) + + # Disable all the previous versions except the newly created one + for ver in self._get_secret_versions(secret_id): + if ver.name != response.name and ver.state == secretmanager.SecretVersion.State.ENABLED: + self.logger.debug(f"Disabling previous version '{ver.name}' of secret '{secret_id}'") + self.disable_secret_version(secret_id, ver.name.split("/")[-1]) + + self.logger.info(f"Successfully added version '{version_id}' to secret '{secret_id}'") + return response.name + + def get_latest_secret_version(self, secret_id: str) -> Tuple[str, bytes]: + """ + Retrieves the latest enabled version of a secret. + + Args: + secret_id (str): The ID of the secret from which to retrieve the version. + + Returns: + Tuple[str, bytes]: A tuple containing the data ID and the payload of the latest secret version. + """ + self.logger.info(f"Retrieving latest version of secret '{secret_id}'") + + if not self._secret_is_managed(secret_id): + error_msg = f"Secret {secret_id} does not exist or is not managed by this service, cannot retrieve latest version." + self.logger.error(error_msg) + raise ValueError(error_msg) + + version_id = self._get_latest_secret_version_id(secret_id) + name = f"projects/{self.project_id}/secrets/{secret_id}/versions/{version_id}" + + self.logger.debug(f"Accessing secret version '{version_id}' of secret '{secret_id}'") + response = self.client.access_secret_version(request={"name": name}) + + crc32c = google_crc32c.Checksum() + crc32c.update(response.payload.data) + + if int(crc32c.hexdigest(), 16) != response.payload.data_crc32c: + error_msg = "CRC32C checksum mismatch. The data may be corrupted." + self.logger.error(f"{error_msg} for secret '{secret_id}' version '{version_id}'") + raise ValueError(error_msg) + + self.logger.info(f"Successfully retrieved version '{version_id}' of secret '{secret_id}'") + + data_str = response.payload.data.decode('utf-8') + data_id, payload = data_str.split(":", 1) + return data_id, payload.encode('utf-8') + + def enable_secret_version(self, secret_id: str, version_id: str) -> None: + """ + Enables a specific version of a secret. + + Args: + secret_id (str): The ID of the secret from which to enable the version. + version_id (str): The version ID to enable. + """ + self.logger.info(f"Enabling version '{version_id}' of secret '{secret_id}'") + + if not self._secret_is_managed(secret_id): + error_msg = f"Secret {secret_id} does not exist or is not managed by this service, cannot enable version." + self.logger.error(error_msg) + raise ValueError(error_msg) + + self.logger.debug(f"Verifying version '{version_id}' exists for secret '{secret_id}'") + version_exists = any( + version.name.split("/")[-1] == version_id and version.state == secretmanager.SecretVersion.State.DISABLED + for version in self._get_secret_versions(secret_id) + ) + if not version_exists: + error_msg = f"Version {version_id} does not exist or is not disabled for secret {secret_id}." + self.logger.error(error_msg) + raise ValueError(error_msg) + + name = f"projects/{self.project_id}/secrets/{secret_id}/versions/{version_id}" + self.logger.debug(f"Enabling version '{version_id}' of secret '{secret_id}'") + response = self.client.enable_secret_version(request={"name": name}) + + if response.name.split("/")[-1] != version_id or response.state != secretmanager.SecretVersion.State.ENABLED: + error_msg = f"Failed to enable secret version {version_id} for secret {secret_id}." + self.logger.error(error_msg) + raise ValueError(error_msg) + + # Wait for the version to be enabled + self.logger.debug(f"Waiting for version '{version_id}' of secret '{secret_id}' to be enabled") + delay = 1 + for _ in range(self.max_retries): + if self._secret_version_is_enabled(secret_id, version_id): + self.logger.debug(f"Version '{version_id}' of secret '{secret_id}' is now enabled") + break + self.logger.debug(f"Version '{version_id}' of secret '{secret_id}' still disabled, retrying in {delay} seconds") + time.sleep(delay) + delay *= 2 + else: + error_msg = f"Could not verify enabling of version '{version_id}' of secret '{secret_id}' after {self.max_retries} retries." + self.logger.error(error_msg) + raise RuntimeError(error_msg) + + self.logger.info(f"Successfully enabled version '{version_id}' of secret '{secret_id}'") + + def disable_secret_version(self, secret_id: str, version_id: str) -> None: + """ + Disables a specific version of a secret. + + Args: + secret_id (str): The ID of the secret from which to delete the version. + version_id (str): The version ID to delete. + """ + self.logger.info(f"Disabling version '{version_id}' of secret '{secret_id}'") + + if not self._secret_is_managed(secret_id): + error_msg = f"Secret {secret_id} does not exist or is not managed by this service, cannot disable version." + self.logger.error(error_msg) + raise ValueError(error_msg) + + self.logger.debug(f"Verifying version '{version_id}' exists for secret '{secret_id}'") + + version_exists = any( + version.name.split("/")[-1] == version_id + for version in self._get_secret_versions(secret_id) + ) + if not version_exists: + error_msg = f"Version {version_id} does not exist for secret {secret_id}." + self.logger.error(error_msg) + raise ValueError(error_msg) + + name = f"projects/{self.project_id}/secrets/{secret_id}/versions/{version_id}" + self.logger.debug(f"Disabling version '{version_id}' of secret '{secret_id}'") + response = self.client.disable_secret_version(request={"name": name}) + + if response.name.split("/")[-1] != version_id or response.state != secretmanager.SecretVersion.State.DISABLED: + error_msg = f"Failed to disable secret version {version_id} for secret {secret_id}." + self.logger.error(error_msg) + raise ValueError(error_msg) + + # Wait for the version to be disabled + self.logger.debug(f"Waiting for version '{version_id}' of secret '{secret_id}' to be disabled") + delay = 1 + for _ in range(self.max_retries): + if not self._secret_version_is_enabled(secret_id, version_id): + self.logger.debug(f"Version '{version_id}' of secret '{secret_id}' is now disabled") + break + self.logger.debug(f"Version '{version_id}' of secret '{secret_id}' still enabled, retrying in {delay} seconds") + time.sleep(delay) + delay *= 2 + else: + error_msg = f"Could not verify disabling of version '{version_id}' of secret '{secret_id}' after {self.max_retries} retries." + self.logger.error(error_msg) + raise RuntimeError(error_msg) + + self.logger.info(f"Successfully disabled version '{version_id}' of secret '{secret_id}'") + + def destroy_secret_version(self, secret_id: str, version_id: str) -> str: + """ + Destroys a specific version of a secret. + + Args: + secret_id (str): The ID of the secret from which to delete the version. + version_id (str): The version ID to delete. + Returns: + str: The data ID of the destroyed version. + """ + self.logger.info(f"Destroying version '{version_id}' of secret '{secret_id}'") + + if not self._secret_is_managed(secret_id): + error_msg = f"Secret {secret_id} does not exist or is not managed by this service, cannot destroy version." + self.logger.error(error_msg) + raise ValueError(error_msg) + + self.logger.debug(f"Verifying version '{version_id}' exists for secret '{secret_id}'") + + version_exists = any( + version.name.split("/")[-1] == version_id + for version in self._get_secret_versions(secret_id) + ) + if not version_exists: + error_msg = f"Version {version_id} does not exist for secret {secret_id}." + self.logger.error(error_msg) + raise ValueError(error_msg) + + # Enable the version before destroying it to get the data ID + if not self._secret_version_is_enabled(secret_id, version_id): + self.logger.debug(f"Version '{version_id}' of secret '{secret_id}' is not enabled, enabling it before destruction") + self.enable_secret_version(secret_id, version_id) + + # Get the data ID from the specific version we're about to destroy + name = f"projects/{self.project_id}/secrets/{secret_id}/versions/{version_id}" + response = self.client.access_secret_version(request={"name": name}) + data_str = response.payload.data.decode('utf-8') + data_id, _ = data_str.split(":", 1) + self.logger.debug(f"Data ID for version '{version_id}' of secret '{secret_id}': {data_id}") + + # Now destroy the version + self.logger.debug(f"Destroying version '{version_id}' of secret '{secret_id}'") + response = self.client.destroy_secret_version(request={"name": name}) + + if response.name.split("/")[-1] != version_id or response.state != secretmanager.SecretVersion.State.DESTROYED: + error_msg = f"Failed to destroy secret version {version_id} for secret {secret_id}." + self.logger.error(error_msg) + raise ValueError(error_msg) + + # Wait for the version to be destroyed + self.logger.debug(f"Waiting for version '{version_id}' of secret '{secret_id}' to be destroyed") + delay = 1 + for _ in range(self.max_retries): + if self._secret_version_is_destroyed(secret_id, version_id): + self.logger.debug(f"Version '{version_id}' of secret '{secret_id}' is now destroyed") + break + self.logger.debug(f"Version '{version_id}' of secret '{secret_id}' still not destroyed, retrying in {delay} seconds") + time.sleep(delay) + delay *= 2 + else: + error_msg = f"Could not verify destruction of version '{version_id}' of secret '{secret_id}' after {self.max_retries} retries." + self.logger.error(error_msg) + raise RuntimeError(error_msg) + + self.logger.info(f"Successfully destroyed version '{version_id}' of secret '{secret_id}'") + return data_id + + def purge_disabled_secret_versions(self, secret_id: str) -> List[str]: + """ + Purges (destroys) all disabled versions of a secret that are older than the grace period. + To determine if a version is older than the grace period, it checks the creation time of each version, + if the latest version was created more than the grace period ago, it will purge the disabled versions. + + Args: + secret_id (str): The ID of the secret for which to purge disabled versions. + Returns: + List[str]: A list of data IDs of the destroyed versions. + """ + self.logger.info(f"Purging disabled versions of secret '{secret_id}'") + + if not self._secret_is_managed(secret_id): + error_msg = f"Secret {secret_id} does not exist or is not managed by this service, cannot purge versions." + self.logger.error(error_msg) + raise ValueError(error_msg) + + data_ids = [] + + for version in self._get_secret_versions(secret_id): + if version.state == secretmanager.SecretVersion.State.DISABLED: + version_id = version.name.split("/")[-1] + create_time = datetime.fromtimestamp(version.create_time.timestamp(), tz=timezone.utc) # type: ignore + if create_time < datetime.now(timezone.utc) - timedelta(days=self.grace_period): + self.logger.debug(f"Destroying disabled version '{version_id}' of secret '{secret_id}'") + data_ids.append(self.destroy_secret_version(secret_id, version_id)) + else: + self.logger.debug(f"Skipping version '{version_id}' of secret '{secret_id}' as it is within the grace period") + + return data_ids + + def cron(self) -> Dict[str, List[str]]: + """ + Performs periodic maintenance tasks: + - Purges disabled secret versions that are older than the grace period. + + Returns: + Dict[str, List[str]]: A dictionary with secret IDs as keys and lists of destroyed data IDs as values. + """ + self.logger.info("Starting periodic maintenance tasks (cron)") + destroyed_secret_ids = {} + + for secret_id in self._get_secret_ids(): + self.logger.debug(f"Processing secret '{secret_id}' for maintenance") + purged_data_ids = self.purge_disabled_secret_versions(secret_id) + if purged_data_ids: + self.logger.info(f"Purged disabled versions of secret '{secret_id}': {purged_data_ids}") + destroyed_secret_ids[secret_id] = purged_data_ids + + return destroyed_secret_ids \ No newline at end of file diff --git a/infra/keys/service_account.py b/infra/keys/service_account.py new file mode 100644 index 000000000000..a1036bf88a47 --- /dev/null +++ b/infra/keys/service_account.py @@ -0,0 +1,425 @@ +# 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. + +import logging +import json +import time +from typing import List,Optional +from google.cloud import iam_admin_v1 +from google.cloud.iam_admin_v1 import types +from google.oauth2 import service_account +from google.auth.transport.requests import Request +from google.api_core import exceptions + +class ServiceAccountManagerLoggerAdapter(logging.LoggerAdapter): + """Logger adapter that adds a prefix to all log messages.""" + + def process(self, msg, kwargs): + return f"[ServiceAccountManager] {msg}", kwargs + +class ServiceAccountManager: + def __init__(self, project_id: str, logger: logging.Logger, max_retries: int = 3) -> None: + self.project_id = project_id + self.client = iam_admin_v1.IAMClient() + self.logger = ServiceAccountManagerLoggerAdapter(logger, {}) + self.max_retries = max_retries + self.logger.info(f"Initialized ServiceAccountManager for project: {self.project_id}") + + def _normalize_account_email(self, account_id: str) -> str: + """ + Normalizes the account identifier to a full email format. + + Args: + account_id (str): The unique identifier or email of the service account. + + Returns: + str: The full service account email address. + """ + # Handle both account ID and full email formats + if "@" in account_id and account_id.endswith(".iam.gserviceaccount.com"): + # account_id is already a full email + return account_id + else: + # account_id is just the account name + return f"{account_id}@{self.project_id}.iam.gserviceaccount.com" + + def _get_service_accounts(self) -> List[iam_admin_v1.ServiceAccount]: + """ + Retrieves all service accounts in the specified project. + + Returns: + List[iam_admin_v1.ServiceAccount]: A list of service account objects. + """ + request = types.ListServiceAccountsRequest() + request.name = f"projects/{self.project_id}" + + accounts = self.client.list_service_accounts(request=request) + self.logger.debug(f"Listed service accounts: {[account.email for account in accounts.accounts]}") + return list(accounts.accounts) + + def _service_account_exists(self, account_id: str) -> bool: + """ + Checks if a service account with the given account_id exists in the project. + + Args: + account_id (str): The unique identifier or email of the service account. + + Returns: + bool: True if the service account exists, False otherwise. + """ + try: + self.get_service_account(account_id) + return True + except exceptions.NotFound: + return False + + def _service_account_is_enabled(self, account_id: str) -> bool: + """ + Checks if a service account is enabled. + + Args: + account_id (str): The unique identifier or email of the service account. + + Returns: + bool: True if the service account is enabled, False otherwise. + """ + try: + service_account = self.get_service_account(account_id) + return not service_account.disabled + except exceptions.NotFound: + self.logger.error(f"Service account {account_id} not found") + return False + + def create_service_account(self, account_id: str, display_name: Optional[str] = None) -> types.ServiceAccount: + """ + Creates a service account in the specified project. + If the service account already exists, returns the existing account (idempotent operation). + + Args: + account_id (str): The unique identifier for the service account. + display_name (Optional[str]): A human-readable name for the service account. + Returns: + types.ServiceAccount: The created or existing service account object. + """ + request = types.CreateServiceAccountRequest() + request.account_id = account_id + request.name = f"projects/{self.project_id}" + + service_account = types.ServiceAccount() + service_account.display_name = display_name or account_id + request.service_account = service_account + + try: + account = self.client.create_service_account(request=request) + + # Wait for the service account to be created + delay = 1 + for _ in range(self.max_retries): + if self._service_account_exists(account_id): + break + time.sleep(delay) + delay *= 2 + else: + self.logger.error(f"Service account {account_id} creation timed out after {self.max_retries} retries.") + raise exceptions.DeadlineExceeded(f"Service account {account_id} creation timed out.") + + self.logger.info(f"Created service account: {account.email}") + return account + except exceptions.Conflict: + existing_account = self.get_service_account(account_id) + self.logger.info(f"Service account already exists: {existing_account.email}") + return existing_account + + def get_service_account(self, account_id: str) -> types.ServiceAccount: + """ + Retrieves a service account by its unique identifier or email. + + Args: + account_id (str): The unique identifier or email of the service account. + + Returns: + types.ServiceAccount: The service account object. + """ + service_account_email = self._normalize_account_email(account_id) + + request = types.GetServiceAccountRequest() + request.name = f"projects/{self.project_id}/serviceAccounts/{service_account_email}" + + try: + service_account = self.client.get_service_account(request=request) + self.logger.info(f"Retrieved service account: {service_account.email}") + return service_account + except exceptions.NotFound: + self.logger.error(f"Service account {account_id} not found") + raise + + def enable_service_account(self, account_id: str) -> None: + """ + Enables a service account in the specified project. + + Args: + account_id (str): The unique identifier or email of the service account to enable. + """ + service_account_email = self._normalize_account_email(account_id) + request = types.EnableServiceAccountRequest() + request.name = f"projects/{self.project_id}/serviceAccounts/{service_account_email}" + + self.client.enable_service_account(request=request) + + # Wait for the service account to be enabled + delay = 1 + for _ in range(self.max_retries): + if self._service_account_is_enabled(account_id): + break + time.sleep(delay) + delay *= 2 + else: + self.logger.error(f"Service account {account_id} enabling timed out after {self.max_retries} retries.") + raise exceptions.DeadlineExceeded(f"Service account {account_id} enabling timed out.") + + self.logger.info(f"Enabled service account: {account_id}") + + def disable_service_account(self, account_id: str) -> None: + """ + Disables a service account in the specified project. + + Args: + account_id (str): The unique identifier or email of the service account to disable. + """ + service_account_email = self._normalize_account_email(account_id) + request = types.DisableServiceAccountRequest() + request.name = f"projects/{self.project_id}/serviceAccounts/{service_account_email}" + + self.client.disable_service_account(request=request) + + # Wait for the service account to be disabled + delay = 1 + for _ in range(self.max_retries): + if not self._service_account_is_enabled(account_id): + break + time.sleep(delay) + delay *= 2 + else: + self.logger.error(f"Service account {account_id} disabling timed out after {self.max_retries} retries.") + raise exceptions.DeadlineExceeded(f"Service account {account_id} disabling timed out.") + + self.logger.info(f"Disabled service account: {account_id}") + + def delete_service_account(self, account_id: str) -> None: + """ + Deletes a service account in the specified project. + + Args: + account_id (str): The unique identifier or email of the service account to delete. + """ + service_account_email = self._normalize_account_email(account_id) + request = types.DeleteServiceAccountRequest() + request.name = f"projects/{self.project_id}/serviceAccounts/{service_account_email}" + + self.client.delete_service_account(request=request) + + # Wait for the service account to be deleted + delay = 1 + for _ in range(self.max_retries): + if not self._service_account_exists(account_id): + break + time.sleep(delay) + delay *= 2 + else: + self.logger.error(f"Service account {account_id} deletion timed out after {self.max_retries} retries.") + raise exceptions.DeadlineExceeded(f"Service account {account_id} deletion timed out.") + + self.logger.info(f"Deleted service account: {account_id}") + + def _get_service_account_keys(self, account_id: str) -> List[iam_admin_v1.ServiceAccountKey]: + """ + Retrieves all keys for the specified service account. + + Args: + account_id (str): The unique identifier or email of the service account. + + Returns: + List[iam_admin_v1.ServiceAccountKey]: A list of service account key objects. + """ + service_account_email = self._normalize_account_email(account_id) + request = types.ListServiceAccountKeysRequest() + request.name = f"projects/{self.project_id}/serviceAccounts/{service_account_email}" + + response = self.client.list_service_account_keys(request=request) + self.logger.debug(f"Listed keys for service account: {account_id}") + return list(response.keys) + + def _service_account_key_exists(self, account_id: str, key_id: str) -> bool: + """ + Checks if a service account key exists for the specified service account. + + Args: + account_id (str): The unique identifier or email of the service account. + key_id (str): The ID of the service account key to check. + + Returns: + bool: True if the key exists, False otherwise. + """ + keys = self._get_service_account_keys(account_id) + return any(key.name.split('/')[-1] == key_id for key in keys) + + def create_service_account_key(self, account_id: str) -> types.ServiceAccountKey: + """ + Creates a key for the specified service account. + Remember the private key ID is only returned once. + If the service account is disabled, it will be enabled first. + Includes retry logic to handle service account propagation delays. + + Args: + account_id (str): The unique identifier or email of the service account. + + Returns: + types.ServiceAccountKey: The created service account key object. + str: The private key ID of the created key. + """ + service_account_email = self._normalize_account_email(account_id) + + # Retry logic for service account access and key creation + delay = 1 + for attempt in range(self.max_retries): + try: + # Check if service account exists and get its state + get_request = types.GetServiceAccountRequest() + get_request.name = f"projects/{self.project_id}/serviceAccounts/{service_account_email}" + + service_account = self.client.get_service_account(request=get_request) + if service_account.disabled: + self.logger.info(f"Service account {account_id} is disabled. Enabling it first.") + self.enable_service_account(account_id) + + # Create the key + request = types.CreateServiceAccountKeyRequest() + request.name = f"projects/{self.project_id}/serviceAccounts/{service_account_email}" + + key = self.client.create_service_account_key(request=request) + + # Wait for the key to be created and available + key_delay = 1 + for _ in range(self.max_retries): + if self._service_account_key_exists(account_id, key.name.split('/')[-1]): + break + time.sleep(key_delay) + key_delay *= 2 + else: + self.logger.error(f"Service account key creation for {account_id} timed out after {self.max_retries} retries.") + raise exceptions.DeadlineExceeded(f"Service account key creation for {account_id} timed out.") + + self.logger.info(f"Created service account key for {account_id}") + return key + + except exceptions.NotFound as e: + if attempt < self.max_retries - 1: + self.logger.warning(f"Service account {account_id} not found (attempt {attempt + 1}/{self.max_retries}), retrying in {delay}s. This may be due to propagation delay.") + time.sleep(delay) + delay *= 2 + else: + self.logger.error(f"Service account {account_id} not found after {self.max_retries} attempts") + raise + except Exception as e: + # For other exceptions, don't retry + self.logger.error(f"Error creating service account key for {account_id}: {e}") + raise + + # This should not be reached due to the raise in the except block + raise exceptions.NotFound(f"Service account {account_id} not found after {self.max_retries} attempts") + + def delete_service_account_key(self, account_id: str, key_id: str) -> None: + """ + Deletes a key for the specified service account. + + Args: + account_id (str): The unique identifier or email of the service account. + key_id (str): The ID of the key to delete. + + Raises: + exceptions.NotFound: If the key does not exist. + exceptions.FailedPrecondition: If the key cannot be deleted due to constraints. + """ + service_account_email = self._normalize_account_email(account_id) + request = types.DeleteServiceAccountKeyRequest() + request.name = f"projects/{self.project_id}/serviceAccounts/{service_account_email}/keys/{key_id}" + + try: + self.client.delete_service_account_key(request=request) + except exceptions.NotFound: + self.logger.warning(f"Service account key {key_id} not found for account: {account_id} (may have been already deleted)") + raise + except exceptions.FailedPrecondition as e: + self.logger.warning(f"Failed to delete service account key {key_id} for account: {account_id}. Error: {e}") + raise + except Exception as e: + self.logger.error(f"Unexpected error deleting service account key {key_id} for account: {account_id}. Error: {e}") + raise + + # Wait for the key to be deleted + delay = 1 + for _ in range(self.max_retries): + if not self._service_account_key_exists(account_id, key_id): + break + time.sleep(delay) + delay *= 2 + else: + self.logger.error(f"Service account key deletion for {account_id} timed out after {self.max_retries} retries.") + raise exceptions.DeadlineExceeded(f"Service account key deletion for {account_id} timed out.") + + self.logger.info(f"Deleted service account key: {key_id} for account: {account_id}") + + def test_service_account_key(self, key_data: bytes) -> bool: + """ + Tests if a service account key is valid by attempting to authenticate and make an API call. + Includes retry logic to handle key propagation delays. + + Args: + key_data (bytes): The private key data from the service account key. + + Returns: + bool: True if the key is valid and can authenticate, False otherwise. + """ + try: + key_info = json.loads(key_data.decode('utf-8')) + except json.JSONDecodeError as json_error: + self.logger.error(f"Invalid JSON in service account key: {json_error}") + return False + + delay = 1 + for attempt in range(self.max_retries): + try: + credentials = service_account.Credentials.from_service_account_info( + key_info, + scopes=['https://www.googleapis.com/auth/cloud-platform'] + ) + + request = Request() + credentials.refresh(request) + + self.logger.info(f"Service account key is valid and can authenticate") + return True + + except Exception as auth_error: + if attempt < self.max_retries - 1: # Don't log on the last attempt + delay *= 2 + self.logger.warning(f"Authentication attempt {attempt + 1} failed (will retry in {delay}s): {auth_error}") + time.sleep(delay) + else: + self.logger.error(f"Authentication failed with service account key after {self.max_retries} attempts: {auth_error}") + return False + + return False + \ No newline at end of file diff --git a/infra/keys/test_secret_manager.py b/infra/keys/test_secret_manager.py new file mode 100644 index 000000000000..4b301e10c58a --- /dev/null +++ b/infra/keys/test_secret_manager.py @@ -0,0 +1,839 @@ +# 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. + +import os +import logging +import unittest +import time +from unittest import mock +from datetime import datetime, timezone, timedelta +from secret_manager import SecretManager, SECRET_MANAGER_LABEL, SecretManagerLoggerAdapter +from google.cloud import secretmanager +from google.api_core import exceptions + +class TestSecretManagerLoggerAdapter(unittest.TestCase): + """Unit tests for SecretManagerLoggerAdapter class.""" + + def test_process_adds_prefix(self): + """Test that the logger adapter adds the correct prefix.""" + logger = logging.getLogger("test") + adapter = SecretManagerLoggerAdapter(logger, {}) + + msg, kwargs = adapter.process("test message", {"key": "value"}) + + self.assertEqual(msg, "[SecretManager] test message") + self.assertEqual(kwargs, {"key": "value"}) + +class TestSecretManager(unittest.TestCase): + """Unit tests for SecretManager class.""" + + def setUp(self): + """Set up test fixtures.""" + self.project_id = "test-project" + self.logger = logging.getLogger("test") + self.logger.setLevel(logging.CRITICAL) # Suppress logging during tests + + # Mock the SecretManagerServiceClient + with mock.patch('secret_manager.secretmanager.SecretManagerServiceClient'): + self.manager = SecretManager( + self.project_id, + self.logger, + rotation_interval=30, + grace_period=7, + max_retries=3 + ) + + self.test_secret_id = "test-secret" + self.test_data_id = "test-data" + self.test_payload = b"test-payload" + + def test_init(self): + """Test SecretManager initialization.""" + with mock.patch('secret_manager.secretmanager.SecretManagerServiceClient'): + manager = SecretManager("test-project", self.logger, 15, 3, 5) + + self.assertEqual(manager.project_id, "test-project") + self.assertEqual(manager.rotation_interval, 15) + self.assertEqual(manager.grace_period, 3) + self.assertEqual(manager.max_retries, 5) + self.assertIsInstance(manager.logger, SecretManagerLoggerAdapter) + + @mock.patch('secret_manager.secretmanager.SecretManagerServiceClient') + def test_get_secret_ids(self, mock_client): + """Test _get_secret_ids method.""" + # Mock response with secrets having the correct label + mock_secret1 = mock.Mock() + mock_secret1.name = "projects/test-project/secrets/secret1" + mock_secret1.labels = {"created_by": SECRET_MANAGER_LABEL} + + mock_secret2 = mock.Mock() + mock_secret2.name = "projects/test-project/secrets/secret2" + mock_secret2.labels = {"created_by": "other"} + + mock_secret3 = mock.Mock() + mock_secret3.name = "projects/test-project/secrets/secret3" + mock_secret3.labels = {"created_by": SECRET_MANAGER_LABEL} + + mock_client.return_value.list_secrets.return_value = [mock_secret1, mock_secret2, mock_secret3] + + manager = SecretManager(self.project_id, self.logger) + secret_ids = manager._get_secret_ids() + + self.assertEqual(secret_ids, ["secret1", "secret3"]) + mock_client.return_value.list_secrets.assert_called_once() + + @mock.patch('secret_manager.secretmanager.SecretManagerServiceClient') + def test_get_secret_ids_exception(self, mock_client): + """Test _get_secret_ids method with exception.""" + mock_client.return_value.list_secrets.side_effect = Exception("API Error") + + manager = SecretManager(self.project_id, self.logger) + secret_ids = manager._get_secret_ids() + + self.assertEqual(secret_ids, []) + + @mock.patch('secret_manager.secretmanager.SecretManagerServiceClient') + def test_secret_exists_true(self, mock_client): + """Test _secret_exists method when secret exists.""" + mock_client.return_value.get_secret.return_value = mock.Mock() + + manager = SecretManager(self.project_id, self.logger) + exists = manager._secret_exists(self.test_secret_id) + + self.assertTrue(exists) + mock_client.return_value.get_secret.assert_called_once() + + @mock.patch('secret_manager.secretmanager.SecretManagerServiceClient') + def test_secret_exists_false(self, mock_client): + """Test _secret_exists method when secret doesn't exist.""" + mock_client.return_value.get_secret.side_effect = exceptions.NotFound("Secret not found") + + manager = SecretManager(self.project_id, self.logger) + exists = manager._secret_exists(self.test_secret_id) + + self.assertFalse(exists) + + @mock.patch('secret_manager.secretmanager.SecretManagerServiceClient') + def test_secret_is_managed_true(self, mock_client): + """Test _secret_is_managed method when secret is managed.""" + mock_secret = mock.Mock() + mock_secret.labels = {"created_by": SECRET_MANAGER_LABEL} + mock_client.return_value.get_secret.return_value = mock_secret + + manager = SecretManager(self.project_id, self.logger) + is_managed = manager._secret_is_managed(self.test_secret_id) + + self.assertTrue(is_managed) + + @mock.patch('secret_manager.secretmanager.SecretManagerServiceClient') + def test_secret_is_managed_false(self, mock_client): + """Test _secret_is_managed method when secret is not managed.""" + mock_secret = mock.Mock() + mock_secret.labels = {"created_by": "other"} + mock_client.return_value.get_secret.return_value = mock_secret + + manager = SecretManager(self.project_id, self.logger) + is_managed = manager._secret_is_managed(self.test_secret_id) + + self.assertFalse(is_managed) + + @mock.patch('secret_manager.secretmanager.SecretManagerServiceClient') + def test_secret_is_managed_not_exists(self, mock_client): + """Test _secret_is_managed method when secret doesn't exist.""" + mock_client.return_value.get_secret.side_effect = exceptions.NotFound("Secret not found") + + manager = SecretManager(self.project_id, self.logger) + is_managed = manager._secret_is_managed(self.test_secret_id) + + self.assertFalse(is_managed) + + @mock.patch('time.sleep') + @mock.patch('secret_manager.secretmanager.SecretManagerServiceClient') + def test_create_secret_success(self, mock_client, mock_sleep): + """Test create_secret method success.""" + mock_response = mock.Mock() + mock_response.name = f"projects/{self.project_id}/secrets/{self.test_secret_id}" + mock_client.return_value.create_secret.return_value = mock_response + + # Mock the sequence of get_secret calls: first raises NotFound, then succeeds + call_count = [0] # Use list to make it mutable in nested function + + def get_secret_side_effect(*args, **kwargs): + call_count[0] += 1 + if call_count[0] == 1: + raise exceptions.NotFound("Not found") # _secret_is_managed returns False + else: + # For waiting loop - return a mock secret with proper labels + mock_secret = mock.Mock() + mock_secret.labels = {"created_by": SECRET_MANAGER_LABEL} + return mock_secret + + mock_client.return_value.get_secret.side_effect = get_secret_side_effect + + manager = SecretManager(self.project_id, self.logger) + result = manager.create_secret(self.test_secret_id) + + self.assertEqual(result, mock_response.name) + mock_client.return_value.create_secret.assert_called_once() + + @mock.patch('secret_manager.secretmanager.SecretManagerServiceClient') + def test_create_secret_already_managed(self, mock_client): + """Test create_secret method when secret already managed.""" + mock_secret = mock.Mock() + mock_secret.labels = {"created_by": SECRET_MANAGER_LABEL} + mock_secret.name = f"projects/{self.project_id}/secrets/{self.test_secret_id}" + mock_client.return_value.get_secret.return_value = mock_secret + + # Mock the secret_path method to return the expected path + expected_path = f"projects/{self.project_id}/secrets/{self.test_secret_id}" + mock_client.return_value.secret_path.return_value = expected_path + + manager = SecretManager(self.project_id, self.logger) + result = manager.create_secret(self.test_secret_id) + + self.assertEqual(result, expected_path) + mock_client.return_value.create_secret.assert_not_called() + + @mock.patch('secret_manager.secretmanager.SecretManagerServiceClient') + def test_get_secret_success(self, mock_client): + """Test get_secret method success.""" + mock_secret = mock.Mock() + mock_secret.labels = {"created_by": SECRET_MANAGER_LABEL} + mock_client.return_value.get_secret.return_value = mock_secret + + manager = SecretManager(self.project_id, self.logger) + result = manager.get_secret(self.test_secret_id) + + self.assertEqual(result, mock_secret) + + @mock.patch('secret_manager.secretmanager.SecretManagerServiceClient') + def test_get_secret_not_exists(self, mock_client): + """Test get_secret method when secret doesn't exist.""" + mock_client.return_value.get_secret.side_effect = exceptions.NotFound("Not found") + + manager = SecretManager(self.project_id, self.logger) + + with self.assertRaises(ValueError): + manager.get_secret(self.test_secret_id) + + @mock.patch('secret_manager.secretmanager.SecretManagerServiceClient') + def test_get_secret_not_managed(self, mock_client): + """Test get_secret method when secret is not managed.""" + mock_secret = mock.Mock() + mock_secret.labels = {"created_by": "other"} + mock_client.return_value.get_secret.return_value = mock_secret + + manager = SecretManager(self.project_id, self.logger) + + with self.assertRaises(ValueError): + manager.get_secret(self.test_secret_id) + + @mock.patch.object(SecretManager, '_secret_exists') + @mock.patch.object(SecretManager, '_secret_is_managed') + @mock.patch('secret_manager.secretmanager.SecretManagerServiceClient') + def test_delete_secret_success(self, mock_client, mock_is_managed, mock_exists): + """Test delete_secret method success.""" + # Mock that secret is managed + mock_is_managed.return_value = True + + # Mock that secret doesn't exist after deletion + mock_exists.return_value = False + + manager = SecretManager(self.project_id, self.logger) + manager.delete_secret(self.test_secret_id) + + mock_client.return_value.delete_secret.assert_called_once() + + @mock.patch('secret_manager.secretmanager.SecretManagerServiceClient') + def test_delete_secret_not_managed(self, mock_client): + """Test delete_secret method when secret is not managed.""" + mock_secret = mock.Mock() + mock_secret.labels = {"created_by": "other"} + mock_client.return_value.get_secret.return_value = mock_secret + + manager = SecretManager(self.project_id, self.logger) + + # The method should return early without raising exception when secret is not managed + manager.delete_secret(self.test_secret_id) + + # Verify that delete_secret was not called since the secret is not managed + mock_client.return_value.delete_secret.assert_not_called() + + @mock.patch('secret_manager.secretmanager.SecretManagerServiceClient') + def test_is_different_user_access_same(self, mock_client): + """Test is_different_user_access method when access is the same.""" + mock_secret = mock.Mock() + mock_secret.labels = {"created_by": SECRET_MANAGER_LABEL} + mock_client.return_value.get_secret.return_value = mock_secret + + mock_policy = mock.Mock() + mock_binding = mock.Mock() + mock_binding.role = "roles/secretmanager.secretAccessor" + mock_binding.members = ["user:test@example.com", "user:test2@example.com"] + mock_policy.bindings = [mock_binding] + mock_client.return_value.get_iam_policy.return_value = mock_policy + + manager = SecretManager(self.project_id, self.logger) + is_different = manager.is_different_user_access( + self.test_secret_id, + ["test@example.com", "test2@example.com"] + ) + + self.assertFalse(is_different) + + @mock.patch('secret_manager.secretmanager.SecretManagerServiceClient') + def test_is_different_user_access_different(self, mock_client): + """Test is_different_user_access method when access is different.""" + mock_secret = mock.Mock() + mock_secret.labels = {"created_by": SECRET_MANAGER_LABEL} + mock_client.return_value.get_secret.return_value = mock_secret + + mock_policy = mock.Mock() + mock_binding = mock.Mock() + mock_binding.role = "roles/secretmanager.secretAccessor" + mock_binding.members = ["user:different@example.com"] + mock_policy.bindings = [mock_binding] + mock_client.return_value.get_iam_policy.return_value = mock_policy + + manager = SecretManager(self.project_id, self.logger) + is_different = manager.is_different_user_access( + self.test_secret_id, + ["test@example.com"] + ) + + self.assertTrue(is_different) + + @mock.patch('secret_manager.secretmanager.SecretManagerServiceClient') + def test_update_secret_access_success(self, mock_client): + """Test update_secret_access method success.""" + mock_secret = mock.Mock() + mock_secret.labels = {"created_by": SECRET_MANAGER_LABEL} + mock_client.return_value.get_secret.return_value = mock_secret + + mock_policy = mock.Mock() + mock_binding = mock.Mock() + mock_binding.role = "roles/secretmanager.secretAccessor" + mock_binding.members = ["user:old@example.com"] + mock_policy.bindings = [mock_binding] + mock_client.return_value.get_iam_policy.return_value = mock_policy + + manager = SecretManager(self.project_id, self.logger) + manager.update_secret_access(self.test_secret_id, ["new@example.com"]) + + mock_client.return_value.set_iam_policy.assert_called_once() + self.assertEqual(mock_binding.members, ["user:new@example.com"]) + + @mock.patch('secret_manager.secretmanager.SecretManagerServiceClient') + def test_get_secret_versions_success(self, mock_client): + """Test _get_secret_versions method success.""" + mock_secret = mock.Mock() + mock_secret.labels = {"created_by": SECRET_MANAGER_LABEL} + mock_client.return_value.get_secret.return_value = mock_secret + + mock_versions = [mock.Mock(), mock.Mock()] + mock_client.return_value.list_secret_versions.return_value = mock_versions + + manager = SecretManager(self.project_id, self.logger) + versions = manager._get_secret_versions(self.test_secret_id) + + self.assertEqual(versions, mock_versions) + + @mock.patch('secret_manager.secretmanager.SecretManagerServiceClient') + def test_secret_version_exists_true(self, mock_client): + """Test _secret_version_exists method when version exists.""" + mock_secret = mock.Mock() + mock_secret.labels = {"created_by": SECRET_MANAGER_LABEL} + mock_client.return_value.get_secret.return_value = mock_secret + + mock_version = mock.Mock() + mock_version.name = f"projects/{self.project_id}/secrets/{self.test_secret_id}/versions/1" + mock_client.return_value.list_secret_versions.return_value = [mock_version] + + manager = SecretManager(self.project_id, self.logger) + exists = manager._secret_version_exists(self.test_secret_id, "1") + + self.assertTrue(exists) + + @mock.patch('secret_manager.secretmanager.SecretManagerServiceClient') + def test_secret_version_exists_false(self, mock_client): + """Test _secret_version_exists method when version doesn't exist.""" + mock_secret = mock.Mock() + mock_secret.labels = {"created_by": SECRET_MANAGER_LABEL} + mock_client.return_value.get_secret.return_value = mock_secret + + mock_version = mock.Mock() + mock_version.name = f"projects/{self.project_id}/secrets/{self.test_secret_id}/versions/2" + mock_client.return_value.list_secret_versions.return_value = [mock_version] + + manager = SecretManager(self.project_id, self.logger) + exists = manager._secret_version_exists(self.test_secret_id, "1") + + self.assertFalse(exists) + + @mock.patch('secret_manager.secretmanager.SecretManagerServiceClient') + def test_secret_version_is_enabled_true(self, mock_client): + """Test _secret_version_is_enabled method when version is enabled.""" + mock_secret = mock.Mock() + mock_secret.labels = {"created_by": SECRET_MANAGER_LABEL} + mock_client.return_value.get_secret.return_value = mock_secret + + mock_version = mock.Mock() + mock_version.name = f"projects/{self.project_id}/secrets/{self.test_secret_id}/versions/1" + mock_version.state = secretmanager.SecretVersion.State.ENABLED + mock_client.return_value.list_secret_versions.return_value = [mock_version] + + manager = SecretManager(self.project_id, self.logger) + is_enabled = manager._secret_version_is_enabled(self.test_secret_id, "1") + + self.assertTrue(is_enabled) + + @mock.patch('secret_manager.secretmanager.SecretManagerServiceClient') + def test_secret_version_is_enabled_false(self, mock_client): + """Test _secret_version_is_enabled method when version is not enabled.""" + mock_secret = mock.Mock() + mock_secret.labels = {"created_by": SECRET_MANAGER_LABEL} + mock_client.return_value.get_secret.return_value = mock_secret + + mock_version = mock.Mock() + mock_version.name = f"projects/{self.project_id}/secrets/{self.test_secret_id}/versions/1" + mock_version.state = secretmanager.SecretVersion.State.DISABLED + mock_client.return_value.list_secret_versions.return_value = [mock_version] + + manager = SecretManager(self.project_id, self.logger) + is_enabled = manager._secret_version_is_enabled(self.test_secret_id, "1") + + self.assertFalse(is_enabled) + + @mock.patch('secret_manager.secretmanager.SecretManagerServiceClient') + def test_get_latest_secret_version_id_success(self, mock_client): + """Test _get_latest_secret_version_id method success.""" + mock_secret = mock.Mock() + mock_secret.labels = {"created_by": SECRET_MANAGER_LABEL} + mock_client.return_value.get_secret.return_value = mock_secret + + mock_version1 = mock.Mock() + mock_version1.name = f"projects/{self.project_id}/secrets/{self.test_secret_id}/versions/1" + mock_version1.state = secretmanager.SecretVersion.State.ENABLED + mock_version1.create_time.timestamp.return_value = 1000 + + mock_version2 = mock.Mock() + mock_version2.name = f"projects/{self.project_id}/secrets/{self.test_secret_id}/versions/2" + mock_version2.state = secretmanager.SecretVersion.State.ENABLED + mock_version2.create_time.timestamp.return_value = 2000 + + # Return versions in reverse order (latest first) as Google API does + mock_client.return_value.list_secret_versions.return_value = [mock_version2, mock_version1] + + manager = SecretManager(self.project_id, self.logger) + latest_id = manager._get_latest_secret_version_id(self.test_secret_id) + + self.assertEqual(latest_id, "2") + + @mock.patch('secret_manager.secretmanager.SecretManagerServiceClient') + def test_get_latest_secret_version_id_no_enabled(self, mock_client): + """Test _get_latest_secret_version_id method when no enabled versions.""" + mock_secret = mock.Mock() + mock_secret.labels = {"created_by": SECRET_MANAGER_LABEL} + mock_client.return_value.get_secret.return_value = mock_secret + + mock_version = mock.Mock() + mock_version.name = f"projects/{self.project_id}/secrets/{self.test_secret_id}/versions/1" + mock_version.state = secretmanager.SecretVersion.State.DISABLED + mock_client.return_value.list_secret_versions.return_value = [mock_version] + + manager = SecretManager(self.project_id, self.logger) + + with self.assertRaises(ValueError): + manager._get_latest_secret_version_id(self.test_secret_id) + + @mock.patch('secret_manager.secretmanager.SecretManagerServiceClient') + def test_is_key_rotation_due_true(self, mock_client): + """Test _is_key_rotation_due method when rotation is due.""" + past_date = datetime.now(timezone.utc) - timedelta(days=40) + mock_secret = mock.Mock() + mock_secret.labels = { + "created_by": SECRET_MANAGER_LABEL, + "last_version_created_at": past_date.strftime("%Y%m%d_%H%M%S") + } + mock_client.return_value.get_secret.return_value = mock_secret + + manager = SecretManager(self.project_id, self.logger, rotation_interval=30) + is_due = manager._is_key_rotation_due(self.test_secret_id) + + self.assertTrue(is_due) + + @mock.patch('secret_manager.secretmanager.SecretManagerServiceClient') + def test_is_key_rotation_due_false(self, mock_client): + """Test _is_key_rotation_due method when rotation is not due.""" + recent_date = datetime.now(timezone.utc) - timedelta(days=10) + mock_secret = mock.Mock() + mock_secret.labels = { + "created_by": SECRET_MANAGER_LABEL, + "last_version_created_at": recent_date.strftime("%Y%m%d_%H%M%S") + } + mock_client.return_value.get_secret.return_value = mock_secret + + manager = SecretManager(self.project_id, self.logger, rotation_interval=30) + is_due = manager._is_key_rotation_due(self.test_secret_id) + + self.assertFalse(is_due) + + @mock.patch('time.sleep') + @mock.patch('google_crc32c.Checksum') + @mock.patch('secret_manager.secretmanager.SecretManagerServiceClient') + def test_add_secret_version_success(self, mock_client, mock_checksum, mock_sleep): + """Test add_secret_version method success.""" + # Mock checksum + mock_checksum_instance = mock.Mock() + mock_checksum_instance.hexdigest.return_value = "abcd1234" + mock_checksum.return_value = mock_checksum_instance + + # Mock create_secret behavior - secret already exists + mock_secret = mock.Mock() + mock_secret.name = f"projects/{self.project_id}/secrets/{self.test_secret_id}" + mock_secret.labels = {"created_by": SECRET_MANAGER_LABEL} + mock_client.return_value.get_secret.return_value = mock_secret + + # Mock add_secret_version + mock_response = mock.Mock() + mock_response.name = f"projects/{self.project_id}/secrets/{self.test_secret_id}/versions/1" + mock_client.return_value.add_secret_version.return_value = mock_response + + # Mock list_secret_versions for waiting and disabling + mock_version = mock.Mock() + mock_version.name = f"projects/{self.project_id}/secrets/{self.test_secret_id}/versions/1" + mock_version.state = secretmanager.SecretVersion.State.ENABLED + mock_client.return_value.list_secret_versions.return_value = [mock_version] + + manager = SecretManager(self.project_id, self.logger) + result = manager.add_secret_version(self.test_secret_id, self.test_data_id, self.test_payload) + + self.assertEqual(result, mock_response.name) + mock_client.return_value.add_secret_version.assert_called_once() + mock_client.return_value.update_secret.assert_called_once() + + @mock.patch('google_crc32c.Checksum') + @mock.patch('secret_manager.secretmanager.SecretManagerServiceClient') + def test_get_latest_secret_version_success(self, mock_client, mock_checksum): + """Test get_latest_secret_version method success.""" + mock_secret = mock.Mock() + mock_secret.labels = {"created_by": SECRET_MANAGER_LABEL} + mock_client.return_value.get_secret.return_value = mock_secret + + # Mock latest version + mock_version = mock.Mock() + mock_version.name = f"projects/{self.project_id}/secrets/{self.test_secret_id}/versions/1" + mock_version.state = secretmanager.SecretVersion.State.ENABLED + mock_version.create_time.timestamp.return_value = 1000 + mock_client.return_value.list_secret_versions.return_value = [mock_version] + + # Mock access_secret_version + mock_response = mock.Mock() + mock_response.payload.data = b"test-data:test-payload" + mock_response.payload.data_crc32c = int("abcd1234", 16) + mock_client.return_value.access_secret_version.return_value = mock_response + + # Mock checksum + mock_checksum_instance = mock.Mock() + mock_checksum_instance.hexdigest.return_value = "abcd1234" + mock_checksum.return_value = mock_checksum_instance + + manager = SecretManager(self.project_id, self.logger) + data_id, payload = manager.get_latest_secret_version(self.test_secret_id) + + self.assertEqual(data_id, "test-data") + self.assertEqual(payload, b"test-payload") + + @mock.patch('time.sleep') + @mock.patch('secret_manager.secretmanager.SecretManagerServiceClient') + def test_enable_secret_version_success(self, mock_client, mock_sleep): + """Test enable_secret_version method success.""" + mock_secret = mock.Mock() + mock_secret.labels = {"created_by": SECRET_MANAGER_LABEL} + mock_client.return_value.get_secret.return_value = mock_secret + + # Mock version exists and is not enabled initially + mock_disabled_version = mock.Mock() + mock_disabled_version.name = f"projects/{self.project_id}/secrets/{self.test_secret_id}/versions/1" + mock_disabled_version.state = secretmanager.SecretVersion.State.DISABLED + + # Mock version becomes enabled after the operation + mock_enabled_version = mock.Mock() + mock_enabled_version.name = f"projects/{self.project_id}/secrets/{self.test_secret_id}/versions/1" + mock_enabled_version.state = secretmanager.SecretVersion.State.ENABLED + + # First call returns disabled version, second call returns enabled version + mock_client.return_value.list_secret_versions.side_effect = [ + [mock_disabled_version], # Initial check + [mock_enabled_version] # After enabling + ] + + # Mock enable response + mock_response = mock.Mock() + mock_response.name = mock_disabled_version.name + mock_response.state = secretmanager.SecretVersion.State.ENABLED + mock_client.return_value.enable_secret_version.return_value = mock_response + + manager = SecretManager(self.project_id, self.logger) + manager.enable_secret_version(self.test_secret_id, "1") + + mock_client.return_value.enable_secret_version.assert_called_once() + + @mock.patch('time.sleep') + @mock.patch('secret_manager.secretmanager.SecretManagerServiceClient') + def test_disable_secret_version_success(self, mock_client, mock_sleep): + """Test disable_secret_version method success.""" + mock_secret = mock.Mock() + mock_secret.labels = {"created_by": SECRET_MANAGER_LABEL} + mock_client.return_value.get_secret.return_value = mock_secret + + # Mock version exists and is enabled initially + mock_enabled_version = mock.Mock() + mock_enabled_version.name = f"projects/{self.project_id}/secrets/{self.test_secret_id}/versions/1" + mock_enabled_version.state = secretmanager.SecretVersion.State.ENABLED + + # Mock version becomes disabled after the operation + mock_disabled_version = mock.Mock() + mock_disabled_version.name = f"projects/{self.project_id}/secrets/{self.test_secret_id}/versions/1" + mock_disabled_version.state = secretmanager.SecretVersion.State.DISABLED + + # First call returns enabled version, second call returns disabled version + mock_client.return_value.list_secret_versions.side_effect = [ + [mock_enabled_version], # Initial check + [mock_disabled_version] # After disabling + ] + + # Mock disable response + mock_response = mock.Mock() + mock_response.name = mock_enabled_version.name + mock_response.state = secretmanager.SecretVersion.State.DISABLED + mock_client.return_value.disable_secret_version.return_value = mock_response + + manager = SecretManager(self.project_id, self.logger) + manager.disable_secret_version(self.test_secret_id, "1") + + mock_client.return_value.disable_secret_version.assert_called_once() + + @mock.patch('time.sleep') + @mock.patch('secret_manager.secretmanager.SecretManagerServiceClient') + def test_destroy_secret_version_success(self, mock_client, mock_sleep): + """Test destroy_secret_version method success.""" + mock_secret = mock.Mock() + mock_secret.labels = {"created_by": SECRET_MANAGER_LABEL} + mock_client.return_value.get_secret.return_value = mock_secret + + # Mock version exists and is enabled initially + mock_enabled_version = mock.Mock() + mock_enabled_version.name = f"projects/{self.project_id}/secrets/{self.test_secret_id}/versions/1" + mock_enabled_version.state = secretmanager.SecretVersion.State.ENABLED + + # Mock version becomes destroyed after the operation + mock_destroyed_version = mock.Mock() + mock_destroyed_version.name = f"projects/{self.project_id}/secrets/{self.test_secret_id}/versions/1" + mock_destroyed_version.state = secretmanager.SecretVersion.State.DESTROYED + + # Multiple calls to list_secret_versions for different operations + mock_client.return_value.list_secret_versions.side_effect = [ + [mock_enabled_version], # Initial check in _secret_version_is_enabled + [mock_enabled_version], # Check in enable_secret_version before enabling + [mock_enabled_version], # After enabling check + [mock_destroyed_version] # After destroying check + ] + + # Mock access_secret_version for getting data_id + mock_access_response = mock.Mock() + mock_access_response.payload.data = b"test-data:test-payload" + mock_client.return_value.access_secret_version.return_value = mock_access_response + + # Mock destroy response + mock_destroy_response = mock.Mock() + mock_destroy_response.name = mock_enabled_version.name + mock_destroy_response.state = secretmanager.SecretVersion.State.DESTROYED + mock_client.return_value.destroy_secret_version.return_value = mock_destroy_response + + # Mock enable response (needed since version is already enabled) + mock_enable_response = mock.Mock() + mock_enable_response.name = mock_enabled_version.name + mock_enable_response.state = secretmanager.SecretVersion.State.ENABLED + mock_client.return_value.enable_secret_version.return_value = mock_enable_response + + manager = SecretManager(self.project_id, self.logger) + data_id = manager.destroy_secret_version(self.test_secret_id, "1") + + self.assertEqual(data_id, "test-data") + mock_client.return_value.destroy_secret_version.assert_called_once() + + @mock.patch.object(SecretManager, 'destroy_secret_version') + @mock.patch('secret_manager.secretmanager.SecretManagerServiceClient') + def test_purge_disabled_secret_versions_success(self, mock_client, mock_destroy): + """Test purge_disabled_secret_versions method success.""" + mock_secret = mock.Mock() + mock_secret.labels = {"created_by": SECRET_MANAGER_LABEL} + mock_client.return_value.get_secret.return_value = mock_secret + + # Mock old disabled version + old_time = datetime.now(timezone.utc) - timedelta(days=10) + mock_old_version = mock.Mock() + mock_old_version.name = f"projects/{self.project_id}/secrets/{self.test_secret_id}/versions/1" + mock_old_version.state = secretmanager.SecretVersion.State.DISABLED + mock_old_version.create_time.timestamp.return_value = old_time.timestamp() + + # Mock recent disabled version (within grace period) + recent_time = datetime.now(timezone.utc) - timedelta(days=2) + mock_recent_version = mock.Mock() + mock_recent_version.name = f"projects/{self.project_id}/secrets/{self.test_secret_id}/versions/2" + mock_recent_version.state = secretmanager.SecretVersion.State.DISABLED + mock_recent_version.create_time.timestamp.return_value = recent_time.timestamp() + + mock_client.return_value.list_secret_versions.return_value = [mock_old_version, mock_recent_version] + + # Mock destroy method to return data_id + mock_destroy.return_value = "old-data" + + manager = SecretManager(self.project_id, self.logger, grace_period=7) + data_ids = manager.purge_disabled_secret_versions(self.test_secret_id) + + self.assertEqual(data_ids, ["old-data"]) + mock_destroy.assert_called_once_with(self.test_secret_id, "1") + + @mock.patch.object(SecretManager, 'purge_disabled_secret_versions') + @mock.patch('secret_manager.secretmanager.SecretManagerServiceClient') + def test_cron_success(self, mock_client, mock_purge): + """Test cron method success.""" + # Mock _get_secret_ids + mock_secret1 = mock.Mock() + mock_secret1.name = f"projects/{self.project_id}/secrets/secret1" + mock_secret1.labels = {"created_by": SECRET_MANAGER_LABEL} + + mock_secret2 = mock.Mock() + mock_secret2.name = f"projects/{self.project_id}/secrets/secret2" + mock_secret2.labels = {"created_by": SECRET_MANAGER_LABEL} + + mock_client.return_value.list_secrets.return_value = [mock_secret1, mock_secret2] + + # Mock purge_disabled_secret_versions behavior + def mock_purge_side_effect(secret_id): + if secret_id == "secret1": + return ["purged-data"] + else: + return [] # secret2 has no versions to purge + + mock_purge.side_effect = mock_purge_side_effect + + manager = SecretManager(self.project_id, self.logger, grace_period=7) + result = manager.cron() + + self.assertIn("secret1", result) + self.assertEqual(result["secret1"], ["purged-data"]) + # secret2 should not be in result since it had no purged versions + self.assertNotIn("secret2", result) + + + + +# Integration tests (skipped unless environment variables are set) +@unittest.skipUnless( + 'GOOGLE_CLOUD_PROJECT' in os.environ, + "Skipping tests because environment variables are not set for Google Cloud project." +) +class TestSecretManagerIntegration(unittest.TestCase): + """Integration tests for SecretManager with real Google Cloud Secret Manager client.""" + + def setUp(self): + """Set up test fixtures.""" + self.project_id = os.environ['GOOGLE_CLOUD_PROJECT'] + # Create a logger for integration tests + self.logger = logging.getLogger(__name__) + self.manager = SecretManager(self.project_id, self.logger, rotation_interval=0, grace_period=0, max_retries=3) + self.test_secret_id = f"integration-test-secret-{int(time.time())}" + self.test_data_id = f"integration-test-data-{int(time.time())}" + self.test_payload = b"integration-test-payload" + self.test_allowed_users = ["pabloem@google.com"] + + def tearDown(self): + """Tear down test fixtures.""" + # Clean up any secrets created during tests + try: + if self.test_secret_id in self.manager._get_secret_ids(): + self.manager.delete_secret(self.test_secret_id) + except Exception as e: + self.logger.warning(f"Failed to clean up test secret: {e}") + + def test_full_secret_lifecycle(self): + """Test creating, adding versions, rotating, and deleting a secret.""" + # Test creating a secret + self.manager.create_secret(self.test_secret_id) + self.assertTrue(self.manager._secret_exists(self.test_secret_id)) + + # Test allowing users to access the secret + self.manager.update_secret_access(self.test_secret_id, self.test_allowed_users) + self.assertFalse(self.manager.is_different_user_access(self.test_secret_id, self.test_allowed_users)) + + # Add first version (creates the secret) + version1 = self.manager.add_secret_version(self.test_secret_id, self.test_data_id, self.test_payload) + self.assertIsNotNone(version1) + + # Verify secret exists + secret = self.manager.get_secret(self.test_secret_id) + self.assertEqual(secret.labels["created_by"], SECRET_MANAGER_LABEL) + + # Add second version + version2 = self.manager.add_secret_version(self.test_secret_id, f"{self.test_data_id}-v2", b"second-payload") + self.assertIsNotNone(version2) + + # List versions + versions = self.manager._get_secret_versions(self.test_secret_id) + self.assertGreaterEqual(len(versions), 2) + + # Get latest version + retrieved_payload = self.manager.get_latest_secret_version(self.test_secret_id) + self.assertEqual(retrieved_payload, (f"{self.test_data_id}-v2", b"second-payload")) + + # Rotate secret + latest_version = self.manager.add_secret_version(self.test_secret_id, f"{self.test_data_id}-rotated", b"rotated-payload") + + # Verify latest version has rotated payload + latest_payload = self.manager.get_latest_secret_version(self.test_secret_id) + self.assertEqual(latest_payload, (f"{self.test_data_id}-rotated", b"rotated-payload")) + + # Verify all the other versions are disabled + versions = self.manager._get_secret_versions(self.test_secret_id) + for version in versions: + if version.name != latest_version: + self.assertEqual(version.state, secretmanager.SecretVersion.State.DISABLED) + + # Try cron method (should be no-op since grace period is 0) + cron_result = self.manager.cron() + self.assertIn(self.test_secret_id, cron_result) + self.assertEqual(len(cron_result[self.test_secret_id]), len(versions) - 1) # All but the latest should be purged + self.assertNotIn(f"{self.test_data_id}-rotated", cron_result[self.test_secret_id]) # Latest id should not be purged + + # Try to get the latest version after cron + latest_payload_after_cron = self.manager.get_latest_secret_version(self.test_secret_id) + self.assertEqual(latest_payload_after_cron, (f"{self.test_data_id}-rotated", b"rotated-payload")) + + # Delete secret + self.manager.delete_secret(self.test_secret_id) + + # Verify secret is removed from secret_ids + self.assertNotIn(self.test_secret_id, self.manager._get_secret_ids()) + +if __name__ == '__main__': + # Configure logging to reduce noise during testing + logging.getLogger('google.cloud').setLevel(logging.WARNING) + logging.getLogger('google.auth').setLevel(logging.WARNING) + + # Run the tests + unittest.main() diff --git a/infra/keys/test_service_account.py b/infra/keys/test_service_account.py new file mode 100644 index 000000000000..370216653267 --- /dev/null +++ b/infra/keys/test_service_account.py @@ -0,0 +1,601 @@ +# 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. + +import os +import logging +import unittest +import time +from unittest import mock +from service_account import ServiceAccountManager +from google.cloud.iam_admin_v1 import types +from google.api_core import exceptions + +class TestServiceAccountManagerUnit(unittest.TestCase): + """Unit tests for ServiceAccountManager with mocked Google Cloud IAM client.""" + + def setUp(self): + """Set up test fixtures.""" + self.project_id = "test-project-123" + self.test_account_id = "test-service-account" + self.test_display_name = "Test Service Account" + + # Patch the IAM client + self.iam_client_patcher = mock.patch('service_account.iam_admin_v1.IAMClient') + self.mock_iam_client_class = self.iam_client_patcher.start() + self.mock_iam_client = self.mock_iam_client_class.return_value + + # Create a mock logger + self.mock_logger = mock.MagicMock() + + # Create the service account manager + self.manager = ServiceAccountManager(self.project_id, self.mock_logger) + + def tearDown(self): + """Tear down test fixtures.""" + self.iam_client_patcher.stop() + + def _create_mock_service_account(self, account_id: str, disabled: bool = False) -> types.ServiceAccount: + """Helper method to create a mock service account.""" + mock_account = types.ServiceAccount() + mock_account.name = f"projects/{self.project_id}/serviceAccounts/{account_id}@{self.project_id}.iam.gserviceaccount.com" + mock_account.email = f"{account_id}@{self.project_id}.iam.gserviceaccount.com" + mock_account.display_name = account_id + mock_account.disabled = disabled + mock_account.project_id = self.project_id + mock_account.unique_id = f"123456789{account_id}" + return mock_account + + def _create_mock_service_account_key(self, account_id: str, key_id: str = "test-key-id") -> types.ServiceAccountKey: + """Helper method to create a mock service account key.""" + mock_key = types.ServiceAccountKey() + mock_key.name = f"projects/{self.project_id}/serviceAccounts/{account_id}@{self.project_id}.iam.gserviceaccount.com/keys/{key_id}" + mock_key.private_key_data = b'{"type": "service_account", "project_id": "test-project"}' + return mock_key + + def test_init(self): + """Test ServiceAccountManager initialization.""" + self.assertEqual(self.manager.project_id, self.project_id) + self.mock_iam_client_class.assert_called_once() + + def test_create_service_account_success(self): + """Test successful service account creation.""" + expected_account = self._create_mock_service_account(self.test_account_id) + self.mock_iam_client.create_service_account.return_value = expected_account + + with mock.patch.object(self.manager, '_service_account_exists', return_value=True): + result = self.manager.create_service_account(self.test_account_id, self.test_display_name) + + self.assertEqual(result, expected_account) + self.mock_iam_client.create_service_account.assert_called_once() + + # Verify the request structure + call_args = self.mock_iam_client.create_service_account.call_args + request = call_args[1]['request'] + self.assertEqual(request.account_id, self.test_account_id) + self.assertEqual(request.name, f"projects/{self.project_id}") + self.assertEqual(request.service_account.display_name, self.test_display_name) + + def test_create_service_account_already_exists(self): + """Test service account creation when account already exists.""" + existing_account = self._create_mock_service_account(self.test_account_id) + + # Mock the conflict exception and then successful get + self.mock_iam_client.create_service_account.side_effect = exceptions.Conflict("Account already exists") + self.mock_iam_client.get_service_account.return_value = existing_account + + result = self.manager.create_service_account(self.test_account_id, self.test_display_name) + + self.assertEqual(result, existing_account) + self.mock_iam_client.create_service_account.assert_called_once() + self.mock_iam_client.get_service_account.assert_called_once() + + def test_enable_service_account(self): + """Test enabling a service account.""" + enabled_account = self._create_mock_service_account(self.test_account_id, disabled=False) + + with mock.patch.object(self.manager, '_service_account_is_enabled', return_value=True): + self.manager.enable_service_account(self.test_account_id) + + self.mock_iam_client.enable_service_account.assert_called_once() + + # Verify the request structure + call_args = self.mock_iam_client.enable_service_account.call_args + request = call_args[1]['request'] + expected_name = f"projects/{self.project_id}/serviceAccounts/{self.test_account_id}@{self.project_id}.iam.gserviceaccount.com" + self.assertEqual(request.name, expected_name) + + def test_disable_service_account(self): + """Test disabling a service account.""" + disabled_account = self._create_mock_service_account(self.test_account_id, disabled=True) + + with mock.patch.object(self.manager, '_service_account_is_enabled', return_value=False): + self.manager.disable_service_account(self.test_account_id) + + self.mock_iam_client.disable_service_account.assert_called_once() + + # Verify the request structure + call_args = self.mock_iam_client.disable_service_account.call_args + request = call_args[1]['request'] + expected_name = f"projects/{self.project_id}/serviceAccounts/{self.test_account_id}@{self.project_id}.iam.gserviceaccount.com" + self.assertEqual(request.name, expected_name) + + def test_delete_service_account(self): + """Test deleting a service account.""" + with mock.patch.object(self.manager, '_service_account_exists', return_value=False): + self.manager.delete_service_account(self.test_account_id) + + self.mock_iam_client.delete_service_account.assert_called_once() + + # Verify the request structure + call_args = self.mock_iam_client.delete_service_account.call_args + request = call_args[1]['request'] + expected_name = f"projects/{self.project_id}/serviceAccounts/{self.test_account_id}@{self.project_id}.iam.gserviceaccount.com" + self.assertEqual(request.name, expected_name) + + def test_list_service_accounts(self): + """Test listing all service accounts in the project.""" + mock_accounts = [ + self._create_mock_service_account("account1"), + self._create_mock_service_account("account2", disabled=True), + self._create_mock_service_account("account3"), + ] + + mock_response = mock.MagicMock() + mock_response.accounts = mock_accounts + # Make the mock response iterable so list(accounts) works + mock_response.__iter__ = lambda self: iter(mock_accounts) + self.mock_iam_client.list_service_accounts.return_value = mock_response + + result = self.manager._get_service_accounts() + + self.assertEqual(result, mock_accounts) + self.mock_iam_client.list_service_accounts.assert_called_once() + + # Verify the request structure + call_args = self.mock_iam_client.list_service_accounts.call_args + request = call_args[1]['request'] + self.assertEqual(request.name, f"projects/{self.project_id}") + + def test_create_service_account_key_enabled_account(self): + """Test creating a key for an enabled service account.""" + enabled_account = self._create_mock_service_account(self.test_account_id, disabled=False) + mock_key = self._create_mock_service_account_key(self.test_account_id) + + self.mock_iam_client.get_service_account.return_value = enabled_account + self.mock_iam_client.create_service_account_key.return_value = mock_key + + with mock.patch.object(self.manager, '_service_account_key_exists', return_value=True): + result = self.manager.create_service_account_key(self.test_account_id) + + self.assertEqual(result, mock_key) + self.mock_iam_client.get_service_account.assert_called_once() + self.mock_iam_client.create_service_account_key.assert_called_once() + + def test_create_service_account_key_disabled_account(self): + """Test creating a key for a disabled service account.""" + disabled_account = self._create_mock_service_account(self.test_account_id, disabled=True) + enabled_account = self._create_mock_service_account(self.test_account_id, disabled=False) + mock_key = self._create_mock_service_account_key(self.test_account_id) + + # First call returns disabled account, then we mock the enable flow + self.mock_iam_client.get_service_account.return_value = disabled_account + self.mock_iam_client.create_service_account_key.return_value = mock_key + + with mock.patch.object(self.manager, '_service_account_is_enabled', return_value=True), \ + mock.patch.object(self.manager, '_service_account_key_exists', return_value=True): + result = self.manager.create_service_account_key(self.test_account_id) + + self.assertEqual(result, mock_key) + # Should call get_service_account once to check if it's disabled + self.mock_iam_client.get_service_account.assert_called_once() + self.mock_iam_client.enable_service_account.assert_called_once() + self.mock_iam_client.create_service_account_key.assert_called_once() + + def test_create_service_account_key_not_found(self): + """Test creating a key for a non-existent service account.""" + self.mock_iam_client.get_service_account.side_effect = exceptions.NotFound("Account not found") + + with self.assertRaises(exceptions.NotFound): + self.manager.create_service_account_key(self.test_account_id) + + def test_delete_service_account_key(self): + """Test deleting a service account key.""" + key_id = "test-key-id" + + with mock.patch.object(self.manager, '_service_account_key_exists', return_value=False): + self.manager.delete_service_account_key(self.test_account_id, key_id) + + self.mock_iam_client.delete_service_account_key.assert_called_once() + + def test_list_service_account_keys(self): + """Test listing service account keys.""" + mock_keys = [ + self._create_mock_service_account_key(self.test_account_id, "key1"), + self._create_mock_service_account_key(self.test_account_id, "key2"), + ] + + mock_response = mock.MagicMock() + mock_response.keys = mock_keys + self.mock_iam_client.list_service_account_keys.return_value = mock_response + + result = self.manager._get_service_account_keys(self.test_account_id) + + self.assertEqual(result, mock_keys) + self.mock_iam_client.list_service_account_keys.assert_called_once() + + @mock.patch('service_account.service_account.Credentials.from_service_account_info') + @mock.patch('service_account.Request') + def test_test_service_account_key_valid(self, mock_request_class, mock_credentials_class): + """Test testing a valid service account key.""" + mock_credentials = mock.MagicMock() + mock_credentials_class.return_value = mock_credentials + + key_data = b'{"type": "service_account", "project_id": "test-project"}' + + result = self.manager.test_service_account_key(key_data) + + self.assertTrue(result) + mock_credentials_class.assert_called_once() + mock_credentials.refresh.assert_called_once() + + @mock.patch('service_account.service_account.Credentials.from_service_account_info') + def test_test_service_account_key_invalid_json(self, mock_credentials_class): + """Test testing an invalid JSON service account key.""" + key_data = b'invalid json' + + result = self.manager.test_service_account_key(key_data) + + self.assertFalse(result) + mock_credentials_class.assert_not_called() + + @mock.patch('service_account.service_account.Credentials.from_service_account_info') + def test_test_service_account_key_auth_error(self, mock_credentials_class): + """Test testing a service account key with authentication error.""" + mock_credentials = mock.MagicMock() + mock_credentials.refresh.side_effect = Exception("Authentication failed") + mock_credentials_class.return_value = mock_credentials + + key_data = b'{"type": "service_account", "project_id": "test-project"}' + + result = self.manager.test_service_account_key(key_data) + + self.assertFalse(result) + + def test_normalize_account_email_with_email(self): + """Test normalizing account email when input is already a full email.""" + full_email = f"{self.test_account_id}@{self.project_id}.iam.gserviceaccount.com" + result = self.manager._normalize_account_email(full_email) + self.assertEqual(result, full_email) + + def test_normalize_account_email_with_id(self): + """Test normalizing account email when input is just the account ID.""" + result = self.manager._normalize_account_email(self.test_account_id) + expected_email = f"{self.test_account_id}@{self.project_id}.iam.gserviceaccount.com" + self.assertEqual(result, expected_email) + + def test_service_account_exists_true(self): + """Test _service_account_exists when service account exists.""" + mock_account = self._create_mock_service_account(self.test_account_id) + self.mock_iam_client.get_service_account.return_value = mock_account + + result = self.manager._service_account_exists(self.test_account_id) + + self.assertTrue(result) + self.mock_iam_client.get_service_account.assert_called_once() + + def test_service_account_exists_false(self): + """Test _service_account_exists when service account does not exist.""" + self.mock_iam_client.get_service_account.side_effect = exceptions.NotFound("Not found") + + result = self.manager._service_account_exists(self.test_account_id) + + self.assertFalse(result) + self.mock_iam_client.get_service_account.assert_called_once() + + def test_service_account_is_enabled_true(self): + """Test _service_account_is_enabled when service account is enabled.""" + mock_account = self._create_mock_service_account(self.test_account_id, disabled=False) + self.mock_iam_client.get_service_account.return_value = mock_account + + result = self.manager._service_account_is_enabled(self.test_account_id) + + self.assertTrue(result) + self.mock_iam_client.get_service_account.assert_called_once() + + def test_service_account_is_enabled_false(self): + """Test _service_account_is_enabled when service account is disabled.""" + mock_account = self._create_mock_service_account(self.test_account_id, disabled=True) + self.mock_iam_client.get_service_account.return_value = mock_account + + result = self.manager._service_account_is_enabled(self.test_account_id) + + self.assertFalse(result) + self.mock_iam_client.get_service_account.assert_called_once() + + def test_service_account_is_enabled_not_found(self): + """Test _service_account_is_enabled when service account does not exist.""" + self.mock_iam_client.get_service_account.side_effect = exceptions.NotFound("Not found") + + result = self.manager._service_account_is_enabled(self.test_account_id) + + self.assertFalse(result) + self.mock_iam_client.get_service_account.assert_called_once() + + def test_get_service_account_success(self): + """Test successful retrieval of a service account.""" + mock_account = self._create_mock_service_account(self.test_account_id) + self.mock_iam_client.get_service_account.return_value = mock_account + + result = self.manager.get_service_account(self.test_account_id) + + self.assertEqual(result, mock_account) + self.mock_iam_client.get_service_account.assert_called_once() + + def test_get_service_account_not_found(self): + """Test retrieval of a non-existent service account.""" + self.mock_iam_client.get_service_account.side_effect = exceptions.NotFound("Not found") + + with self.assertRaises(exceptions.NotFound): + self.manager.get_service_account(self.test_account_id) + + self.mock_iam_client.get_service_account.assert_called_once() + + def test_service_account_key_exists_true(self): + """Test _service_account_key_exists when key exists.""" + key_id = "test-key-id" + mock_key = self._create_mock_service_account_key(self.test_account_id, key_id) + mock_response = mock.MagicMock() + mock_response.keys = [mock_key] + self.mock_iam_client.list_service_account_keys.return_value = mock_response + + result = self.manager._service_account_key_exists(self.test_account_id, key_id) + + self.assertTrue(result) + self.mock_iam_client.list_service_account_keys.assert_called_once() + + def test_service_account_key_exists_false(self): + """Test _service_account_key_exists when key does not exist.""" + key_id = "test-key-id" + other_key = self._create_mock_service_account_key(self.test_account_id, "other-key-id") + mock_response = mock.MagicMock() + mock_response.keys = [other_key] + self.mock_iam_client.list_service_account_keys.return_value = mock_response + + result = self.manager._service_account_key_exists(self.test_account_id, key_id) + + self.assertFalse(result) + self.mock_iam_client.list_service_account_keys.assert_called_once() + + def test_delete_service_account_key_not_found(self): + """Test deleting a non-existent service account key.""" + key_id = "non-existent-key" + self.mock_iam_client.delete_service_account_key.side_effect = exceptions.NotFound("Key not found") + + with self.assertRaises(exceptions.NotFound): + self.manager.delete_service_account_key(self.test_account_id, key_id) + + self.mock_iam_client.delete_service_account_key.assert_called_once() + + def test_delete_service_account_key_failed_precondition(self): + """Test deleting a service account key with failed precondition.""" + key_id = "test-key-id" + self.mock_iam_client.delete_service_account_key.side_effect = exceptions.FailedPrecondition("Cannot delete") + + with self.assertRaises(exceptions.FailedPrecondition): + self.manager.delete_service_account_key(self.test_account_id, key_id) + + self.mock_iam_client.delete_service_account_key.assert_called_once() + + def test_delete_service_account_key_unexpected_error(self): + """Test deleting a service account key with unexpected error.""" + key_id = "test-key-id" + self.mock_iam_client.delete_service_account_key.side_effect = Exception("Unexpected error") + + with self.assertRaises(Exception): + self.manager.delete_service_account_key(self.test_account_id, key_id) + + self.mock_iam_client.delete_service_account_key.assert_called_once() + + @mock.patch('service_account.time.sleep') + def test_test_service_account_key_retry_success(self, mock_sleep): + """Test service account key testing with retry logic success.""" + mock_credentials = mock.MagicMock() + + # First attempt fails, second succeeds + mock_credentials.refresh.side_effect = [Exception("Auth failed"), None] + + with mock.patch('service_account.service_account.Credentials.from_service_account_info', return_value=mock_credentials): + key_data = b'{"type": "service_account", "project_id": "test-project"}' + result = self.manager.test_service_account_key(key_data) + + self.assertTrue(result) + self.assertEqual(mock_credentials.refresh.call_count, 2) + mock_sleep.assert_called_once_with(2) # delay is doubled before sleep (1 * 2 = 2) + + @mock.patch('service_account.time.sleep') + def test_test_service_account_key_retry_exhausted(self, mock_sleep): + """Test service account key testing when all retries are exhausted.""" + mock_credentials = mock.MagicMock() + mock_credentials.refresh.side_effect = Exception("Auth failed") + + with mock.patch('service_account.service_account.Credentials.from_service_account_info', return_value=mock_credentials): + key_data = b'{"type": "service_account", "project_id": "test-project"}' + result = self.manager.test_service_account_key(key_data) + + self.assertFalse(result) + self.assertEqual(mock_credentials.refresh.call_count, 3) # max_retries + # Sleep is called with 2, then 4 (delay is doubled each time) + self.assertEqual(mock_sleep.call_count, 2) # 2 retry delays + mock_sleep.assert_any_call(2) # First retry delay (1 * 2) + mock_sleep.assert_any_call(4) # Second retry delay (2 * 2) + + def test_create_service_account_timeout(self): + """Test service account creation timeout scenario.""" + expected_account = self._create_mock_service_account(self.test_account_id) + self.mock_iam_client.create_service_account.return_value = expected_account + + # Mock the helper method to always return False (service account never exists) + with mock.patch.object(self.manager, '_service_account_exists', return_value=False): + with self.assertRaises(exceptions.DeadlineExceeded): + self.manager.create_service_account(self.test_account_id, self.test_display_name) + + def test_enable_service_account_timeout(self): + """Test service account enabling timeout scenario.""" + # Mock the helper method to always return False (service account never gets enabled) + with mock.patch.object(self.manager, '_service_account_is_enabled', return_value=False): + with self.assertRaises(exceptions.DeadlineExceeded): + self.manager.enable_service_account(self.test_account_id) + + def test_disable_service_account_timeout(self): + """Test service account disabling timeout scenario.""" + # Mock the helper method to always return True (service account never gets disabled) + with mock.patch.object(self.manager, '_service_account_is_enabled', return_value=True): + with self.assertRaises(exceptions.DeadlineExceeded): + self.manager.disable_service_account(self.test_account_id) + + def test_delete_service_account_timeout(self): + """Test service account deletion timeout scenario.""" + # Mock the helper method to always return True (service account never gets deleted) + with mock.patch.object(self.manager, '_service_account_exists', return_value=True): + with self.assertRaises(exceptions.DeadlineExceeded): + self.manager.delete_service_account(self.test_account_id) + + def test_create_service_account_key_timeout(self): + """Test service account key creation timeout scenario.""" + enabled_account = self._create_mock_service_account(self.test_account_id, disabled=False) + mock_key = self._create_mock_service_account_key(self.test_account_id) + + self.mock_iam_client.get_service_account.return_value = enabled_account + self.mock_iam_client.create_service_account_key.return_value = mock_key + + # Mock the helper method to always return False (key never gets created) + with mock.patch.object(self.manager, '_service_account_key_exists', return_value=False): + with self.assertRaises(exceptions.DeadlineExceeded): + self.manager.create_service_account_key(self.test_account_id) + + def test_delete_service_account_key_timeout(self): + """Test service account key deletion timeout scenario.""" + key_id = "test-key-id" + + # Mock the helper method to always return True (key never gets deleted) + with mock.patch.object(self.manager, '_service_account_key_exists', return_value=True): + with self.assertRaises(exceptions.DeadlineExceeded): + self.manager.delete_service_account_key(self.test_account_id, key_id) + +# Run these real tests just if the environment variables are set correctly +# export GOOGLE_CLOUD_PROJECT = "your-project-id" + +# Verify that the variables are set before running the tests +@unittest.skipUnless( + 'GOOGLE_CLOUD_PROJECT' in os.environ, + "Skipping tests because environment variables are not set for Google Cloud project." +) +class TestServiceAccountManagerIntegration(unittest.TestCase): + """Integration tests for ServiceAccountManager with real Google Cloud IAM client.""" + + def setUp(self): + """Set up test fixtures.""" + self.project_id = os.environ['GOOGLE_CLOUD_PROJECT'] + self.logger = logging.getLogger(__name__) + self.manager = ServiceAccountManager(self.project_id, self.logger, 5) + + def tearDown(self): + """Tear down test fixtures.""" + # Clean up any service accounts created during tests + try: + accounts = self.manager._get_service_accounts() + for account in accounts: + if account.email.startswith("test-account-"): + try: + self.manager.delete_service_account(account.email) + except Exception as e: + self.logger.warning(f"Failed to delete service account {account.email}: {e}") + except Exception as e: + self.logger.warning(f"Failed to list service accounts during tearDown: {e}") + + def test_full_service_account_lifecycle(self): + """Test creating and deleting a service account.""" + account_id = "test-account-" + str(os.getpid()) + display_name = "Test Account" + + # Create service account + account = self.manager.create_service_account(account_id, display_name) + service_account_email = account.email + self.assertEqual(account.display_name, display_name) + + # Wait until service account is created (with retries) + for i in range(5): + if service_account_email in [a.email for a in self.manager._get_service_accounts()]: + break + time.sleep(i ** 2) # Exponential backoff + # Verify service account exists + self.assertIn(service_account_email, [a.email for a in self.manager._get_service_accounts()]) + + # Create a key for the service account + key = self.manager.create_service_account_key(service_account_email) + self.assertIsNotNone(key.private_key_data) + + # Test the key (now includes retry logic for propagation delays) + key_valid = self.manager.test_service_account_key(key.private_key_data) + self.assertTrue(key_valid) + + # List keys for the service account - with delayed check + self.assertIn(key.name, [k.name for k in self.manager._get_service_account_keys(service_account_email)]) + + # Delete the service account key + self.manager.delete_service_account_key(service_account_email, key.name.split('/')[-1]) + + # Create a new key to ensure we have multiple keys + new_key = self.manager.create_service_account_key(service_account_email) + new_key_valid = self.manager.test_service_account_key(new_key.private_key_data) + self.assertTrue(new_key_valid) + + # Verify that we have 2 keys now + all_keys = self.manager._get_service_account_keys(service_account_email) + self.assertEqual(len(all_keys), 2) # 1 old key + 1 new key + + # Disable the service account + self.manager.disable_service_account(service_account_email) + + # Verify service account is disabled + account = self.manager.get_service_account(service_account_email) + self.assertTrue(account.disabled) + + # Enable the service account + self.manager.enable_service_account(service_account_email) + + # Verify service account is enabled + account = self.manager.get_service_account(service_account_email) + self.assertFalse(account.disabled) + + # Test again the key after enabling the service account + key_valid = self.manager.test_service_account_key(new_key.private_key_data) + self.assertTrue(key_valid) + + # Delete the service account + self.manager.delete_service_account(service_account_email) + + # Verify service account is deleted - using get_service_account with exception handling + with self.assertRaises(exceptions.NotFound): + self.manager.get_service_account(service_account_email) + +if __name__ == '__main__': + # Configure logging to reduce noise during testing + import logging + logging.getLogger('google.cloud').setLevel(logging.WARNING) + logging.getLogger('google.auth').setLevel(logging.WARNING) + + # Run the tests + unittest.main() diff --git a/infra/security/README.md b/infra/security/README.md new file mode 100644 index 000000000000..0e60c4b33043 --- /dev/null +++ b/infra/security/README.md @@ -0,0 +1,84 @@ + + +# GCP Security Analyzer + +This document describes the implementation of a security analyzer for Google Cloud Platform (GCP) resources. The analyzer is designed to enhance security monitoring within our GCP environment by capturing critical events and generating alerts for specific security-sensitive actions. + +## How It Works + +1. **Log Sinks**: The system uses [GCP Log Sinks](https://cloud.google.com/logging/docs/export/configure_export_v2) to capture specific security-related log entries. These sinks are configured to filter for events like IAM policy changes or service account key creation. +2. **Log Storage**: The filtered logs are routed to a dedicated Google Cloud Storage (GCS) bucket for persistence and analysis. +3. **Report Generation**: A scheduled job runs weekly, executing the `log_analyzer.py` script. +4. **Email Alerts**: The script analyzes the logs from the past week, compiles a summary of security events, and sends a report to a configured email address. + +## Configuration + +The behavior of the log analyzer is controlled by a `config.yml` file. Here’s an overview of the configuration options: + +- `project_id`: The GCP project ID where the resources are located. +- `bucket_name`: The name of the GCS bucket where logs will be stored. +- `logging`: Configures the logging level and format for the script. +- `sinks`: A list of log sinks to be created. Each sink has the following properties: + - `name`: A unique name for the sink. + - `description`: A brief description of what the sink monitors. + - `filter_methods`: A list of GCP API methods to include in the filter (e.g., `SetIamPolicy`). + - `excluded_principals`: A list of service accounts or user emails to exclude from monitoring, such as CI/CD service accounts. + +### Example Configuration (`config.yml`) + +```yaml +project_id: your-gcp-project-id +bucket_name: your-log-storage-bucket + +sinks: + - name: iam-policy-changes + description: Monitors changes to IAM policies. + filter_methods: + - "SetIamPolicy" + excluded_principals: + - "ci-cd-account@your-project.iam.gserviceaccount.com" +``` + +## Usage + +The `log_analyzer.py` script provides two main commands for managing the security analyzer. + +### Initializing Sinks + +To create or update the log sinks in GCP based on your `config.yml` file, run the following command: + +```bash +python log_analyzer.py --config config.yml initialize +``` + +This command ensures that the log sinks are correctly configured to capture the desired security events. + +### Generating Weekly Reports + +To generate and send the weekly security report, run this command: + +```bash +python log_analyzer.py --config config.yml generate-report +``` + +This is typically run as a scheduled job (GitHub Action) to automate the delivery of weekly security reports. + + + diff --git a/infra/security/config.yml b/infra/security/config.yml new file mode 100644 index 000000000000..e2c3659040cc --- /dev/null +++ b/infra/security/config.yml @@ -0,0 +1,43 @@ +# 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. + +project_id: apache-beam-testing + +# Logging +logging: + level: DEBUG + format: "[%(asctime)s] %(levelname)s: %(message)s" + +# gcloud storage bucket +bucket_name: "beam-sec-analytics-and-logging" + +# GCP Log sinks +sinks: + - name: iam-policy-changes + description: Monitors changes to IAM policies, excluding approved CI/CD service accounts. + filter_methods: + - "SetIamPolicy" + excluded_principals: + - beam-github-actions@apache-beam-testing.iam.gserviceaccount.com + - github-self-hosted-runners@apache-beam-testing.iam.gserviceaccount.com + + - name: sa-key-management + description: Monitors creation and deletion of service account keys. + filter_methods: + - "google.iam.admin.v1.IAM.CreateServiceAccountKey" + - "google.iam.admin.v1.IAM.DeleteServiceAccountKey" + excluded_principals: + - beam-github-actions@apache-beam-testing.iam.gserviceaccount.com + - github-self-hosted-runners@apache-beam-testing.iam.gserviceaccount.com diff --git a/infra/security/log_analyzer.py b/infra/security/log_analyzer.py new file mode 100644 index 000000000000..55ab4495e24f --- /dev/null +++ b/infra/security/log_analyzer.py @@ -0,0 +1,333 @@ +# 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. + +import json +import ssl +import yaml +import logging +import smtplib +import os +from dataclasses import dataclass +from datetime import datetime, timedelta, timezone +from google.cloud import logging_v2 +from google.cloud import storage +from typing import List, Dict, Any +import argparse + +REPORT_SUBJECT = "Weekly IAM Security Events Report" +REPORT_BODY_TEMPLATE = """ +Hello Team, + +Please find below the summary of IAM security events for the past week: + +{event_summary} + +Best Regards, +Automated GitHub Action +""" + +@dataclass +class SinkCls: + name: str + description: str + filter_methods: List[str] + excluded_principals: List[str] + +class LogAnalyzer(): + def __init__(self, project_id: str, gcp_bucket: str, logger: logging.Logger, sinks: List[SinkCls]): + self.project_id = project_id + self.bucket = gcp_bucket + self.logger = logger + self.sinks = sinks + + def _construct_filter(self, sink: SinkCls) -> str: + """ + Constructs a filter string for a given sink. + + Args: + sink (Sink): The sink object containing filter information. + + Returns: + str: The constructed filter string. + """ + + method_filters = [] + for method in sink.filter_methods: + method_filters.append(f'protoPayload.methodName="{method}"') + + exclusion_filters = [] + for principal in sink.excluded_principals: + exclusion_filters.append(f'protoPayload.authenticationInfo.principalEmail != "{principal}"') + + if method_filters and exclusion_filters: + filter_ = f"({' OR '.join(method_filters)}) AND ({' AND '.join(exclusion_filters)})" + elif method_filters: + filter_ = f"({' OR '.join(method_filters)})" + elif exclusion_filters: + filter_ = f"({' AND '.join(exclusion_filters)})" + else: + filter_ = "" + + return filter_ + + def _create_log_sink(self, sink: SinkCls) -> None: + """ + Creates a log sink in GCP if it doesn't already exist. + If it already exists, it updates the sink with the new filter in case the filter has changed. + + Args: + sink (Sink): The sink object to create. + """ + logging_client = logging_v2.Client(project=self.project_id) + filter_ = self._construct_filter(sink) + destination = "storage.googleapis.com/{bucket}".format(bucket=self.bucket) + + sink_client = logging_client.sink(sink.name, filter_=filter_, destination=destination) + + if sink_client.exists(): + self.logger.debug(f"Sink {sink.name} already exists.") + sink_client.reload() + if sink_client.filter_ != filter_: + sink_client.filter_ = filter_ + sink_client.update() + self.logger.info(f"Updated sink {sink.name}'s filter.") + else: + sink_client.create() + self.logger.info(f"Created sink {sink.name}.") + # Reload the sink to get the writer_identity, this may take a few moments + sink_client.reload() + + self._grant_bucket_permissions(sink_client) + + logging_client.close() + + def _grant_bucket_permissions(self, sink: logging_v2.Sink) -> None: + """ + Grants a log sink's writer identity permissions to write to the bucket. + """ + logging_client = logging_v2.Client(project=self.project_id) + storage_client = storage.Client(project=self.project_id) + + sink.reload() + writer_identity = sink.writer_identity + if not writer_identity: + self.logger.warning(f"Could not retrieve writer identity for sink {sink.name}. " + f"Manual permission granting might be required.") + return + + bucket = storage_client.get_bucket(self.bucket) + policy = bucket.get_iam_policy(requested_policy_version=3) + iam_role = "roles/storage.objectCreator" + + # Workaround for projects where the writer_identity is not a valid service account. + if writer_identity == "serviceAccount:cloud-logs@system.gserviceaccount.com": + member = "group:cloud-logs@google.com" + else: + member = f"serviceAccount:{writer_identity}" + + # Check if the policy is already configured + if any(member in b.get("members", []) and b.get("role") == iam_role for b in policy.bindings): + self.logger.debug(f"Sink {sink.name} already has the necessary permissions.") + return + + policy.bindings.append({ + "role": iam_role, + "members": {member} + }) + + bucket.set_iam_policy(policy) + self.logger.info(f"Granted {iam_role} to {member} on bucket {self.bucket} for sink {sink.name}.") + + def initialize_sinks(self) -> None: + for sink in self.sinks: + self._create_log_sink(sink) + self.logger.info(f"Initialized sink: {sink.name}") + + def get_event_logs(self, days: int = 7) -> List[Dict[str, Any]]: + """ + Reads and retrieves log events from the specified time range from the GCP Cloud Storage bucket. + + Args: + days (int): The number of days to look back for log analysis. + + Returns: + List[Dict[str, Any]]: A list of log entries that match the specified time range. + """ + found_events = [] + storage_client = storage.Client(project=self.project_id) + + now = datetime.now(timezone.utc) + end_time = now.replace(minute=0, second=0, microsecond=0) - timedelta(minutes=30) + start_time = end_time - timedelta(days=days) + + blobs = storage_client.list_blobs(self.bucket) + for blob in blobs: + if not (start_time <= blob.time_created < end_time): + continue + + self.logger.debug(f"Processing blob: {blob.name}") + content = blob.download_as_string().decode("utf-8") + + for num, line in enumerate(content.splitlines(), 1): + try: + log_entry = json.loads(line) + payload = log_entry.get("protoPayload") + if not payload: + self.logger.warning(f"Skipping log in blob {blob.name}, line {num}: no protoPayload found.") + continue + + event_details = { + "timestamp": log_entry.get("timestamp", "N/A"), + "principal": payload.get("authenticationInfo", {}).get("principalEmail", "N/A"), + "method": payload.get("methodName", "N/A"), + "resource": payload.get("resourceName", "N/A"), + "project_id": log_entry.get("resource", {}).get("labels", {}).get("project_id", "N/A"), + "file_name": blob.name + } + found_events.append(event_details) + except json.JSONDecodeError: + self.logger.warning(f"Skipping invalid JSON log in blob {blob.name}, line {num}.") + continue + + storage_client.close() + return found_events + + def create_weekly_email_report(self, dry_run: bool = False) -> None: + """ + Creates an email report based on the events found this week. + If `dry_run` is True, it will print the report to the console instead of sending it. + """ + events = self.get_event_logs(days=7) + if not events: + self.logger.info("No events found for the weekly report.") + return + + events.sort(key=lambda x: x['timestamp'], reverse=True) + event_summary = "\n".join( + f"Timestamp: {event['timestamp']}, Principal: {event['principal']}, Method: {event['method']}, Resource: {event['resource']}, Project ID: {event['project_id']}, File: {event['file_name']}" + for event in events + ) + + report_subject = REPORT_SUBJECT + report_body = REPORT_BODY_TEMPLATE.format(event_summary=event_summary) + + if dry_run: + self.logger.info("Dry run: printing email report to console.") + print(f"Subject: {report_subject}\n") + print(f"Body:\n{report_body}") + return + + self.send_email(report_subject, report_body) + + def send_email(self, subject: str, body: str) -> None: + """ + Sends an email with the specified subject and body. + If email configuration is not fully set, it prints the email instead. + + Args: + subject (str): The subject of the email. + body (str): The body of the email. + """ + smtp_server = os.getenv("SMTP_SERVER") + smtp_port_str = os.getenv("SMTP_PORT") + recipient = os.getenv("EMAIL_RECIPIENT") + email = os.getenv("EMAIL_ADDRESS") + password = os.getenv("EMAIL_PASSWORD") + + if not all([smtp_server, smtp_port_str, recipient, email, password]): + self.logger.warning("Email configuration is not fully set. Printing email instead.") + print(f"Subject: {subject}\n") + print(f"Body:\n{body}") + return + + assert smtp_server is not None + assert smtp_port_str is not None + assert recipient is not None + assert email is not None + assert password is not None + + message = f"Subject: {subject}\n\n{body}" + context = ssl.create_default_context() + + try: + smtp_port = int(smtp_port_str) + with smtplib.SMTP_SSL(smtp_server, smtp_port, context=context) as server: + server.login(email, password) + server.sendmail(email, recipient, message) + self.logger.info(f"Successfully sent email report to {recipient}") + except Exception as e: + self.logger.error(f"Failed to send email report: {e}") + +def load_config_from_yaml(config_path: str) -> Dict[str, Any]: + with open(config_path, 'r') as file: + config = yaml.safe_load(file) + + c = { + "project_id": config.get("project_id"), + "gcp_bucket": config.get("bucket_name"), + "sinks": [], + "logger": logging.getLogger(__name__) + } + + for sink_config in config.get("sinks", []): + sink = SinkCls( + name=sink_config["name"], + description=sink_config["description"], + filter_methods=sink_config.get("filter_methods", []), + excluded_principals=sink_config.get("excluded_principals", []) + ) + c["sinks"].append(sink) + + logging_config = config.get("logging", {}) + log_level = logging_config.get("level", "INFO") + log_format = logging_config.get("format", "[%(asctime)s] %(levelname)s: %(message)s") + + c["logger"].setLevel(log_level) + logging.basicConfig(level=log_level, format=log_format) + + return c + +def main(): + """ + Main entry point for the script. + """ + parser = argparse.ArgumentParser(description="GCP IAM Log Analyzer") + parser.add_argument("--config", required=True, help="Path to the configuration YAML file.") + + subparsers = parser.add_subparsers(dest="command", required=True) + + subparsers.add_parser("initialize", help="Initialize/update log sinks in GCP.") + report_parser = subparsers.add_parser("generate-report", help="Generate and send the weekly IAM security report.") + report_parser.add_argument("--dry-run", action="store_true", help="Do not send email, print report to console.") + + args = parser.parse_args() + + config = load_config_from_yaml(args.config) + log_analyzer = LogAnalyzer( + project_id=config["project_id"], + gcp_bucket=config["gcp_bucket"], + logger=config["logger"], + sinks=config["sinks"] + ) + + if args.command == "initialize": + log_analyzer.initialize_sinks() + log_analyzer.logger.info("Sinks initialized successfully.") + elif args.command == "generate-report": + log_analyzer.create_weekly_email_report(dry_run=args.dry_run) + log_analyzer.logger.info("Weekly report generation process completed.") + +if __name__ == "__main__": + main() diff --git a/infra/security/requirements.txt b/infra/security/requirements.txt new file mode 100644 index 000000000000..a4abb8bc5acf --- /dev/null +++ b/infra/security/requirements.txt @@ -0,0 +1,19 @@ +# +# 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. + +PyYAML==6.0.2 +google-cloud-storage==3.3.0 +google-cloud-logging==3.12.1 diff --git a/it/common/src/main/java/org/apache/beam/it/common/utils/ResourceManagerUtils.java b/it/common/src/main/java/org/apache/beam/it/common/utils/ResourceManagerUtils.java index 0492206643f0..e7911f9ba4c7 100644 --- a/it/common/src/main/java/org/apache/beam/it/common/utils/ResourceManagerUtils.java +++ b/it/common/src/main/java/org/apache/beam/it/common/utils/ResourceManagerUtils.java @@ -175,8 +175,7 @@ public static void cleanResources(boolean failOnCleanup, ResourceManager... mana throw new RuntimeException("Error cleaning up resources", bubbleException); } else if (bubbleException != null) { LOG.warn( - "Error cleaning up resources. This is not configured to fail the test: {}", - bubbleException.getMessage()); + "Error cleaning up resources. This is not configured to fail the test", bubbleException); } } diff --git a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/IOLoadTestBase.java b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/IOLoadTestBase.java index bbf9dd0519ec..14770a429731 100644 --- a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/IOLoadTestBase.java +++ b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/IOLoadTestBase.java @@ -121,7 +121,7 @@ protected void exportMetrics( try { metrics = getMetrics(launchInfo, metricsConfig); } catch (Exception e) { - LOG.warn("Unable to get metrics due to error: {}", e.getMessage()); + LOG.warn("Unable to get metrics due to error", e); return; } String testId = UUID.randomUUID().toString(); diff --git a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/artifacts/matchers/ArtifactsSubject.java b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/artifacts/matchers/ArtifactsSubject.java index 06ea14d0390b..ab5b49699484 100644 --- a/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/artifacts/matchers/ArtifactsSubject.java +++ b/it/google-cloud-platform/src/main/java/org/apache/beam/it/gcp/artifacts/matchers/ArtifactsSubject.java @@ -72,6 +72,7 @@ public void hasFiles() { * * @param expectedFiles Expected files */ + @SuppressWarnings("LenientFormatStringValidation") public void hasFiles(int expectedFiles) { check("there are %d files", expectedFiles).that(actual.size()).isEqualTo(expectedFiles); } diff --git a/it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/bigquery/BigQueryStreamingLT.java b/it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/bigquery/BigQueryStreamingLT.java index e89fe1dc8524..d68c0f07865e 100644 --- a/it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/bigquery/BigQueryStreamingLT.java +++ b/it/google-cloud-platform/src/test/java/org/apache/beam/it/gcp/bigquery/BigQueryStreamingLT.java @@ -132,7 +132,7 @@ public void setUpTest() { String expectedTable = TestProperties.getProperty("expectedTable", "", TestProperties.Type.PROPERTY); if (!Strings.isNullOrEmpty(expectedTable)) { - config.toBuilder().setExpectedTable(expectedTable).build(); + config = config.toBuilder().setExpectedTable(expectedTable).build(); } crashIntervalSeconds = @@ -396,7 +396,7 @@ public void runTest(BigQueryIO.Write.Method writeMethod) } catch (Exception e) { // Just log the error. Don't re-throw because we have accuracy checks that are more // important below - LOG.error("Encountered an error while exporting metrics to BigQuery:\n{}", e); + LOG.error("Encountered an error while exporting metrics to BigQuery:", e); } } // If we're not publishing metrics, just run the pipeline normally diff --git a/it/mongodb/build.gradle b/it/mongodb/build.gradle index 6be9b91f5b34..960e15af8394 100644 --- a/it/mongodb/build.gradle +++ b/it/mongodb/build.gradle @@ -35,6 +35,7 @@ dependencies { implementation library.java.testcontainers_mongodb implementation library.java.google_code_gson implementation library.java.mongo_java_driver + implementation library.java.mongo_bson implementation library.java.vendored_guava_32_1_2_jre testImplementation library.java.mockito_core diff --git a/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/external_transforms.proto b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/external_transforms.proto index add8a1999caf..02a5dd18e2c6 100644 --- a/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/external_transforms.proto +++ b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/external_transforms.proto @@ -76,6 +76,10 @@ message ManagedTransforms { "beam:schematransform:org.apache.beam:bigquery_write:v1"]; ICEBERG_CDC_READ = 6 [(org.apache.beam.model.pipeline.v1.beam_urn) = "beam:schematransform:org.apache.beam:iceberg_cdc_read:v1"]; + POSTGRES_READ = 7 [(org.apache.beam.model.pipeline.v1.beam_urn) = + "beam:schematransform:org.apache.beam:postgres_read:v1"]; + POSTGRES_WRITE = 8 [(org.apache.beam.model.pipeline.v1.beam_urn) = + "beam:schematransform:org.apache.beam:postgres_write:v1"]; } } diff --git a/playground/backend/containers/router/Dockerfile b/playground/backend/containers/router/Dockerfile index 863461013a45..1fcb98062201 100644 --- a/playground/backend/containers/router/Dockerfile +++ b/playground/backend/containers/router/Dockerfile @@ -44,16 +44,19 @@ RUN cd cmd/migration_tool &&\ go build -o /go/bin/migration_tool # Null image -FROM debian:stable-20221114-slim +FROM debian:bullseye-slim +ENV DEBIAN_FRONTEND=noninteractive # Install deps being used by sh files -RUN apt-get update \ - && apt-get install -y --no-install-recommends \ - ca-certificates \ - curl \ - && apt-get autoremove -yqq --purge \ - && apt-get clean \ - && rm -rf /var/lib/apt/lists/* +RUN set -eux; \ + # 1) use existing HTTP sources to bootstrap CAs + apt-get update; \ + apt-get install -y --no-install-recommends ca-certificates; \ + # 2) now it’s safe to use HTTPS + sed -ri 's|http://deb\.debian\.org|https://deb.debian.org|g' /etc/apt/sources.list; \ + apt-get update; \ + apt-get install -y --no-install-recommends curl; \ + rm -rf /var/lib/apt/lists/* # Set Environment ENV SERVER_IP=0.0.0.0 diff --git a/release/build.gradle.kts b/release/build.gradle.kts index 7ec49b86aac2..a13ad34b00fc 100644 --- a/release/build.gradle.kts +++ b/release/build.gradle.kts @@ -41,7 +41,9 @@ task("runJavaExamplesValidationTask") { dependsOn(":runners:spark:3:runQuickstartJavaSpark") dependsOn(":runners:flink:1.19:runQuickstartJavaFlinkLocal") dependsOn(":runners:direct-java:runMobileGamingJavaDirect") - dependsOn(":runners:google-cloud-dataflow-java:runMobileGamingJavaDataflow") - dependsOn(":runners:twister2:runQuickstartJavaTwister2") + if (project.hasProperty("ver") || !project.version.toString().endsWith("SNAPSHOT")) { + // only run one variant of MobileGaming on Dataflow for nightly + dependsOn(":runners:google-cloud-dataflow-java:runMobileGamingJavaDataflow") + } dependsOn(":runners:google-cloud-dataflow-java:runMobileGamingJavaDataflowBom") } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java index f8dbfd61e836..b16dad86df18 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java @@ -383,6 +383,16 @@ public PaneInfo pane() { return element.getPaneInfo(); } + @Override + public String currentRecordId() { + return element.getCurrentRecordId(); + } + + @Override + public Long currentRecordOffset() { + return element.getCurrentRecordOffset(); + } + @Override public PipelineOptions getPipelineOptions() { return pipelineOptions; @@ -411,6 +421,24 @@ public void outputWindowedValue( outputReceiver.output(mainOutputTag, WindowedValues.of(value, timestamp, windows, paneInfo)); } + @Override + public void outputWindowedValue( + OutputT value, + Instant timestamp, + Collection windows, + PaneInfo paneInfo, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { + noteOutput(); + if (watermarkEstimator instanceof TimestampObservingWatermarkEstimator) { + ((TimestampObservingWatermarkEstimator) watermarkEstimator).observeTimestamp(timestamp); + } + outputReceiver.output( + mainOutputTag, + WindowedValues.of( + value, timestamp, windows, paneInfo, currentRecordId, currentRecordOffset)); + } + @Override public void output(TupleTag tag, T value) { outputWithTimestamp(tag, value, element.getTimestamp()); @@ -429,11 +457,26 @@ public void outputWindowedValue( Instant timestamp, Collection windows, PaneInfo paneInfo) { + outputWindowedValue(tag, value, timestamp, windows, paneInfo, null, null); + } + + @Override + public void outputWindowedValue( + TupleTag tag, + T value, + Instant timestamp, + Collection windows, + PaneInfo paneInfo, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { noteOutput(); if (watermarkEstimator instanceof TimestampObservingWatermarkEstimator) { ((TimestampObservingWatermarkEstimator) watermarkEstimator).observeTimestamp(timestamp); } - outputReceiver.output(tag, WindowedValues.of(value, timestamp, windows, paneInfo)); + outputReceiver.output( + tag, + WindowedValues.of( + value, timestamp, windows, paneInfo, currentRecordId, currentRecordOffset)); } private void noteOutput() { diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.java index 543b2cb5a741..69e6225a33ea 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.java @@ -93,7 +93,7 @@ public void storeCurrentPaneInfo(ReduceFn.Context context, PaneInfo context.state().access(PANE_INFO_TAG).write(currentPane); } - private PaneInfo describePane( + private PaneInfo describePane( Object key, Instant windowMaxTimestamp, PaneInfo previousPane, boolean isFinal) { boolean isFirst = previousPane == null; Timing previousTiming = isFirst ? null : previousPane.getTiming(); diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java index 840245edf7ad..217c06c56fe5 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java @@ -334,6 +334,35 @@ public void output(OutputT output, Instant timestamp, BoundedWindow window) { public void output(TupleTag tag, T output, Instant timestamp, BoundedWindow window) { outputWindowedValue(tag, WindowedValues.of(output, timestamp, window, PaneInfo.NO_FIRING)); } + + @Override + public void output( + OutputT output, + Instant timestamp, + BoundedWindow window, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { + output(mainOutputTag, output, timestamp, window, currentRecordId, currentRecordOffset); + } + + @Override + public void output( + TupleTag tag, + T output, + Instant timestamp, + BoundedWindow window, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { + outputWindowedValue( + tag, + WindowedValues.of( + output, + timestamp, + Collections.singletonList(window), + PaneInfo.NO_FIRING, + currentRecordId, + currentRecordOffset)); + } } private final DoFnFinishBundleArgumentProvider.Context context = @@ -427,6 +456,24 @@ public void outputWindowedValue( outputWindowedValue(mainOutputTag, output, timestamp, windows, paneInfo); } + @Override + public void outputWindowedValue( + OutputT output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { + outputWindowedValue( + mainOutputTag, + output, + timestamp, + windows, + paneInfo, + currentRecordId, + currentRecordOffset); + } + @Override public void output(TupleTag tag, T output) { checkNotNull(tag, "Tag passed to output cannot be null"); @@ -451,11 +498,36 @@ public void outputWindowedValue( tag, WindowedValues.of(output, timestamp, windows, paneInfo)); } + @Override + public void outputWindowedValue( + TupleTag tag, + T output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { + SimpleDoFnRunner.this.outputWindowedValue( + tag, + WindowedValues.of( + output, timestamp, windows, paneInfo, currentRecordId, currentRecordOffset)); + } + @Override public Instant timestamp() { return elem.getTimestamp(); } + @Override + public String currentRecordId() { + return elem.getCurrentRecordId(); + } + + @Override + public Long currentRecordOffset() { + return elem.getCurrentRecordOffset(); + } + public Collection windows() { return elem.getWindows(); } @@ -867,6 +939,24 @@ public void outputWindowedValue( outputWindowedValue(mainOutputTag, output, timestamp, windows, paneInfo); } + @Override + public void outputWindowedValue( + OutputT output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { + outputWindowedValue( + mainOutputTag, + output, + timestamp, + windows, + paneInfo, + currentRecordId, + currentRecordOffset); + } + @Override public void output(TupleTag tag, T output) { checkTimestamp(timestamp(), timestamp); @@ -892,6 +982,22 @@ public void outputWindowedValue( tag, WindowedValues.of(output, timestamp, windows, paneInfo)); } + @Override + public void outputWindowedValue( + TupleTag tag, + T output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { + checkTimestamp(timestamp(), timestamp); + SimpleDoFnRunner.this.outputWindowedValue( + tag, + WindowedValues.of( + output, timestamp, windows, paneInfo, currentRecordId, currentRecordOffset)); + } + @Override public BundleFinalizer bundleFinalizer() { throw new UnsupportedOperationException( @@ -1096,6 +1202,24 @@ public void outputWindowedValue( outputWindowedValue(mainOutputTag, output, timestamp, windows, paneInfo); } + @Override + public void outputWindowedValue( + OutputT output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { + outputWindowedValue( + mainOutputTag, + output, + timestamp, + windows, + paneInfo, + currentRecordId, + currentRecordOffset); + } + @Override public void output(TupleTag tag, T output) { checkTimestamp(this.timestamp, timestamp); @@ -1121,6 +1245,22 @@ public void outputWindowedValue( tag, WindowedValues.of(output, timestamp, windows, paneInfo)); } + @Override + public void outputWindowedValue( + TupleTag tag, + T output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { + checkTimestamp(this.timestamp, timestamp); + SimpleDoFnRunner.this.outputWindowedValue( + tag, + WindowedValues.of( + output, timestamp, windows, paneInfo, currentRecordId, currentRecordOffset)); + } + @Override public BundleFinalizer bundleFinalizer() { throw new UnsupportedOperationException( diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDoViaKeyedWorkItems.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDoViaKeyedWorkItems.java index fafb02f9dd0b..6af54da0a08b 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDoViaKeyedWorkItems.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDoViaKeyedWorkItems.java @@ -42,7 +42,6 @@ import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; import org.apache.beam.sdk.transforms.splittabledofn.WatermarkEstimator; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.util.construction.PTransformReplacements; @@ -250,7 +249,7 @@ public static class ProcessFn watermarkHoldTag = StateTags.makeSystemTagInternal( - StateTags.watermarkStateInternal("hold", TimestampCombiner.LATEST)); + StateTags.watermarkStateInternal("hold", TimestampCombiner.LATEST)); /** * The state cell containing a copy of the element. Written during the first {@link @@ -663,6 +662,27 @@ public void output( throwUnsupportedOutput(); } + @Override + public void output( + OutputT output, + Instant timestamp, + BoundedWindow window, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { + throwUnsupportedOutput(); + } + + @Override + public void output( + TupleTag tag, + T output, + Instant timestamp, + BoundedWindow window, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { + throwUnsupportedOutput(); + } + @Override public PipelineOptions getPipelineOptions() { return baseContext.getPipelineOptions(); diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/StateMerging.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/StateMerging.java index e490e3188d81..f5a2ff679ed8 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/StateMerging.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/StateMerging.java @@ -76,8 +76,7 @@ public static void mergeBags( /** * Merge all bag state in {@code sources} (which may include {@code result}) into {@code result}. */ - public static void mergeBags( - Collection> sources, BagState result) { + public static void mergeBags(Collection> sources, BagState result) { if (sources.isEmpty()) { // Nothing to merge. return; @@ -117,8 +116,7 @@ public static void mergeSets( /** * Merge all set state in {@code sources} (which may include {@code result}) into {@code result}. */ - public static void mergeSets( - Collection> sources, SetState result) { + public static void mergeSets(Collection> sources, SetState result) { if (sources.isEmpty()) { // Nothing to merge. return; @@ -172,7 +170,7 @@ public static void mergeCo * Merge all value state from {@code sources} (which may include {@code result}) into {@code * result}. */ - public static void mergeCombiningValues( + public static void mergeCombiningValues( Collection> sources, CombiningState result) { if (sources.isEmpty()) { diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/StateTags.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/StateTags.java index 6ed7f8525fdc..ba5478be6c77 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/StateTags.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/StateTags.java @@ -35,7 +35,6 @@ import org.apache.beam.sdk.state.WatermarkHoldState; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.CombineWithContext.CombineFnWithContext; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Equivalence; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; @@ -223,7 +222,7 @@ public static StateTag> orderedList(String id, Coder } /** Create a state tag for holding the watermark. */ - public static StateTag watermarkStateInternal( + public static StateTag watermarkStateInternal( String id, TimestampCombiner timestampCombiner) { return new SimpleStateTag<>( new StructuredId(id), StateSpecs.watermarkStateInternal(timestampCombiner)); diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/TimerInternals.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/TimerInternals.java index 6c92d234b86f..254e6f5fcf5b 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/TimerInternals.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/TimerInternals.java @@ -222,12 +222,7 @@ public static TimerData of( */ public static TimerData of( StateNamespace namespace, Instant timestamp, Instant outputTimestamp, TimeDomain domain) { - String timerId = - new StringBuilder() - .append(domain.ordinal()) - .append(':') - .append(timestamp.getMillis()) - .toString(); + String timerId = String.valueOf(domain.ordinal()) + ':' + timestamp.getMillis(); return of(timerId, namespace, timestamp, outputTimestamp, domain); } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java index e5a5f90587c1..15ae8dfe5f1a 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java @@ -55,11 +55,10 @@ }) class WatermarkHold implements Serializable { /** Return tag for state containing the output watermark hold used for elements. */ - public static - StateTag watermarkHoldTagForTimestampCombiner( - TimestampCombiner timestampCombiner) { + public static StateTag watermarkHoldTagForTimestampCombiner( + TimestampCombiner timestampCombiner) { return StateTags.makeSystemTagInternal( - StateTags.watermarkStateInternal("hold", timestampCombiner)); + StateTags.watermarkStateInternal("hold", timestampCombiner)); } /** diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricsContainerImpl.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricsContainerImpl.java index 3fc078e83d7e..3532cd3be111 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricsContainerImpl.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricsContainerImpl.java @@ -51,7 +51,6 @@ import org.apache.beam.runners.core.metrics.MetricUpdates.MetricUpdate; import org.apache.beam.sdk.metrics.Distribution; import org.apache.beam.sdk.metrics.Histogram; -import org.apache.beam.sdk.metrics.Metric; import org.apache.beam.sdk.metrics.MetricKey; import org.apache.beam.sdk.metrics.MetricName; import org.apache.beam.sdk.metrics.MetricsContainer; @@ -608,7 +607,7 @@ public void commitUpdates() { }); } - private > + private > ImmutableList> extractCumulatives(MetricsMap cells) { ImmutableList.Builder> updates = ImmutableList.builder(); cells.forEach( @@ -619,7 +618,7 @@ ImmutableList> extractCumulatives(MetricsMap> + private > ImmutableList> extractHistogramCumulatives( MetricsMap, CellT> cells) { ImmutableList.Builder> updates = ImmutableList.builder(); diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/SimpleExecutionState.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/SimpleExecutionState.java index f1b7bd07c71c..821a7e06c526 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/SimpleExecutionState.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/SimpleExecutionState.java @@ -65,7 +65,7 @@ public class SimpleExecutionState extends ExecutionState { /** * @param stateName A state name to be used in lull logging when stuck in a state. - * @param urn A optional string urn for an execution time metric. + * @param urn An optional string urn for an execution time metric. * @param labelsMetadata arbitrary metadata to use for reporting purposes. */ public SimpleExecutionState( diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/ExecutableTriggerStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/ExecutableTriggerStateMachine.java index 006c34fe153c..2d64986f51a0 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/ExecutableTriggerStateMachine.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/ExecutableTriggerStateMachine.java @@ -23,7 +23,6 @@ import java.io.Serializable; import java.util.ArrayList; import java.util.List; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; /** * A wrapper around a trigger used during execution. While an actual trigger may appear multiple @@ -42,18 +41,17 @@ public class ExecutableTriggerStateMachine implements Serializable { private final List subTriggers = new ArrayList<>(); private final TriggerStateMachine trigger; - public static ExecutableTriggerStateMachine create( - TriggerStateMachine trigger) { + public static ExecutableTriggerStateMachine create(TriggerStateMachine trigger) { return create(trigger, 0); } - private static ExecutableTriggerStateMachine create( + private static ExecutableTriggerStateMachine create( TriggerStateMachine trigger, int nextUnusedIndex) { return new ExecutableTriggerStateMachine(trigger, nextUnusedIndex); } - public static ExecutableTriggerStateMachine createForOnceTrigger( + public static ExecutableTriggerStateMachine createForOnceTrigger( TriggerStateMachine trigger, int nextUnusedIndex) { return new ExecutableTriggerStateMachine(trigger, nextUnusedIndex); } diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MetricsContainerStepMapTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MetricsContainerStepMapTest.java index dcc3651e66fe..432f7c588191 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MetricsContainerStepMapTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MetricsContainerStepMapTest.java @@ -96,7 +96,7 @@ public class MetricsContainerStepMapTest { stringSet.add(FIRST_STRING, SECOND_STRING); boundedTrie.add(FIRST_STRING, SECOND_STRING); } catch (IOException e) { - LOG.error(e.getMessage(), e); + LOG.error("Suppressed Exception.", e); } } diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MetricsPusherTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MetricsPusherTest.java index 5c8902fd7ca5..1509b46dd4e6 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MetricsPusherTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MetricsPusherTest.java @@ -104,7 +104,7 @@ public void processElement(ProcessContext context) { counter.inc(); context.output(context.element()); } catch (Exception e) { - LOG.error(e.getMessage(), e); + LOG.error("Suppressed Exception.", e); } } } diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/ReshuffleTriggerStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/ReshuffleTriggerStateMachineTest.java index 89e100eccda2..191efc33c572 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/ReshuffleTriggerStateMachineTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/ReshuffleTriggerStateMachineTest.java @@ -21,7 +21,6 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.joda.time.Duration; @@ -35,7 +34,7 @@ public class ReshuffleTriggerStateMachineTest { /** Public so that other tests can instantiate {@link ReshuffleTriggerStateMachine}. */ - public static ReshuffleTriggerStateMachine forTest() { + public static ReshuffleTriggerStateMachine forTest() { return ReshuffleTriggerStateMachine.create(); } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java index 155f5566cc83..dc45de20002f 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java @@ -797,6 +797,8 @@ public Coder getOutputCoder() { } } + @SuppressWarnings( + "NullableOptional") // null value used to indicates no elements put to the queue yet private static class StaticQueue implements Serializable { static class StaticQueueSource extends UnboundedSource> { diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkDetachedRunnerResult.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkDetachedRunnerResult.java index 77d0e7d3434c..f7d82065b658 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkDetachedRunnerResult.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkDetachedRunnerResult.java @@ -98,7 +98,7 @@ public State waitUntilFinish(Duration duration) { return state; } try { - Thread.sleep(jobCheckIntervalInSecs * 1000); + Thread.sleep(jobCheckIntervalInSecs * 1000L); } catch (InterruptedException e) { throw new RuntimeException(e); } diff --git a/runners/google-cloud-dataflow-java/build.gradle b/runners/google-cloud-dataflow-java/build.gradle index daf480799b67..05cb8417106d 100644 --- a/runners/google-cloud-dataflow-java/build.gradle +++ b/runners/google-cloud-dataflow-java/build.gradle @@ -52,8 +52,8 @@ evaluationDependsOn(":sdks:java:container:java11") ext.dataflowLegacyEnvironmentMajorVersion = '8' ext.dataflowFnapiEnvironmentMajorVersion = '8' -ext.dataflowLegacyContainerVersion = 'beam-master-20250729' -ext.dataflowFnapiContainerVersion = 'beam-master-20250729' +ext.dataflowLegacyContainerVersion = 'beam-master-20250811' +ext.dataflowFnapiContainerVersion = 'beam-master-20250811' ext.dataflowContainerBaseRepository = 'gcr.io/cloud-dataflow/v1beta3' processResources { diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java index 537a2d855921..15627534411c 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java @@ -215,8 +215,7 @@ public PCollectionView> expand(PCollection> input) { return this.applyInternal(input); } - private PCollectionView> applyInternal( - PCollection> input) { + private PCollectionView> applyInternal(PCollection> input) { try { return BatchViewAsMultimap.applyForMapLike(runner, input, view, true /* unique keys */); } catch (NonDeterministicException e) { @@ -704,8 +703,7 @@ public PCollectionView>> expand(PCollection> input) return this.applyInternal(input); } - private PCollectionView>> applyInternal( - PCollection> input) { + private PCollectionView>> applyInternal(PCollection> input) { try { return applyForMapLike(runner, input, view, false /* unique keys not expected */); } catch (NonDeterministicException e) { @@ -1395,6 +1393,16 @@ public PaneInfo getPaneInfo() { return PaneInfo.NO_FIRING; } + @Override + public @Nullable String getCurrentRecordId() { + return null; + } + + @Override + public @Nullable Long getCurrentRecordOffset() { + return null; + } + @Override public Iterable> explodeWindows() { return Collections.emptyList(); diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java index 3838534c6aee..2be4b569ca9b 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java @@ -54,7 +54,8 @@ /** A DataflowPipelineJob represents a job submitted to Dataflow using {@link DataflowRunner}. */ @SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) + "nullness", // TODO(https://github.com/apache/beam/issues/20497) + "Slf4jDoNotLogMessageOfExceptionExplicitly", // intended, sent full stacktrace to LOG.debug }) public class DataflowPipelineJob implements PipelineResult { diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/RowCoderCloudObjectTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/RowCoderCloudObjectTranslator.java index 9e11ffe2a794..df062aad645a 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/RowCoderCloudObjectTranslator.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/RowCoderCloudObjectTranslator.java @@ -56,7 +56,6 @@ public RowCoder fromCloudObject(CloudObject cloudObject) { SchemaApi.Schema.Builder schemaBuilder = SchemaApi.Schema.newBuilder(); JsonFormat.parser().merge(Structs.getString(cloudObject, SCHEMA), schemaBuilder); Schema schema = SchemaTranslation.schemaFromProto(schemaBuilder.build()); - SchemaCoderCloudObjectTranslator.overrideEncodingPositions(schema); return RowCoder.of(schema); } catch (IOException e) { throw new RuntimeException(e); diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/SchemaCoderCloudObjectTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/SchemaCoderCloudObjectTranslator.java index fa58590ba798..029f6c3c61d7 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/SchemaCoderCloudObjectTranslator.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/SchemaCoderCloudObjectTranslator.java @@ -18,15 +18,11 @@ package org.apache.beam.runners.dataflow.util; import java.io.IOException; -import java.util.UUID; -import javax.annotation.Nullable; import org.apache.beam.model.pipeline.v1.SchemaApi; -import org.apache.beam.sdk.coders.RowCoder; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.SchemaCoder; import org.apache.beam.sdk.schemas.SchemaTranslation; import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.StringUtils; import org.apache.beam.sdk.util.construction.SdkComponents; @@ -102,50 +98,12 @@ public SchemaCoder fromCloudObject(CloudObject cloudObject) { SchemaApi.Schema.Builder schemaBuilder = SchemaApi.Schema.newBuilder(); JsonFormat.parser().merge(Structs.getString(cloudObject, SCHEMA), schemaBuilder); Schema schema = SchemaTranslation.schemaFromProto(schemaBuilder.build()); - overrideEncodingPositions(schema); return SchemaCoder.of(schema, typeDescriptor, toRowFunction, fromRowFunction); } catch (IOException e) { throw new RuntimeException(e); } } - static void overrideEncodingPositions(Schema schema) { - @Nullable UUID uuid = schema.getUUID(); - if (schema.isEncodingPositionsOverridden() && uuid != null) { - RowCoder.overrideEncodingPositions(uuid, schema.getEncodingPositions()); - } - schema.getFields().stream() - .map(Schema.Field::getType) - .forEach(SchemaCoderCloudObjectTranslator::overrideEncodingPositions); - } - - private static void overrideEncodingPositions(Schema.FieldType fieldType) { - switch (fieldType.getTypeName()) { - case ROW: - overrideEncodingPositions(Preconditions.checkArgumentNotNull(fieldType.getRowSchema())); - break; - case ARRAY: - case ITERABLE: - overrideEncodingPositions( - Preconditions.checkArgumentNotNull(fieldType.getCollectionElementType())); - break; - case MAP: - overrideEncodingPositions(Preconditions.checkArgumentNotNull(fieldType.getMapKeyType())); - overrideEncodingPositions(Preconditions.checkArgumentNotNull(fieldType.getMapValueType())); - break; - case LOGICAL_TYPE: - Schema.LogicalType logicalType = - Preconditions.checkArgumentNotNull(fieldType.getLogicalType()); - @Nullable Schema.FieldType argumentType = logicalType.getArgumentType(); - if (argumentType != null) { - overrideEncodingPositions(argumentType); - } - overrideEncodingPositions(logicalType.getBaseType()); - break; - default: - } - } - @Override public Class getSupportedClass() { return SchemaCoder.class; diff --git a/runners/google-cloud-dataflow-java/worker/build.gradle b/runners/google-cloud-dataflow-java/worker/build.gradle index fe7e3b93dd0e..4068c5f88e4f 100644 --- a/runners/google-cloud-dataflow-java/worker/build.gradle +++ b/runners/google-cloud-dataflow-java/worker/build.gradle @@ -131,7 +131,7 @@ applyJavaNature( dependencies { // We have to include jetty-server/jetty-servlet and all of its transitive dependencies // which includes several org.eclipse.jetty artifacts + servlet-api - include(dependency("org.eclipse.jetty:.*:9.4.54.v20240208")) + include(dependency("org.eclipse.jetty:.*:9.4.57.v20241219")) include(dependency("javax.servlet:javax.servlet-api:3.1.0")) } relocate("org.eclipse.jetty", getWorkerRelocatedPath("org.eclipse.jetty")) @@ -200,8 +200,8 @@ dependencies { compileOnly "org.conscrypt:conscrypt-openjdk-uber:2.5.1" implementation "javax.servlet:javax.servlet-api:3.1.0" - implementation "org.eclipse.jetty:jetty-server:9.4.54.v20240208" - implementation "org.eclipse.jetty:jetty-servlet:9.4.54.v20240208" + implementation "org.eclipse.jetty:jetty-server:9.4.57.v20241219" + implementation "org.eclipse.jetty:jetty-servlet:9.4.57.v20241219" implementation library.java.avro implementation library.java.jackson_annotations implementation library.java.jackson_core diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillSink.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillSink.java index b156ff45caf6..aac882cae36c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillSink.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillSink.java @@ -23,6 +23,7 @@ import com.google.auto.service.AutoService; import java.io.IOException; import java.io.InputStream; +import java.nio.charset.StandardCharsets; import java.util.Collection; import java.util.HashMap; import java.util.Map; @@ -43,6 +44,7 @@ import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.primitives.Longs; import org.checkerframework.checker.nullness.qual.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -52,6 +54,7 @@ "nullness" // TODO(https://github.com/apache/beam/issues/20497) }) class WindmillSink extends Sink> { + private WindmillStreamWriter writer; private final Coder valueCoder; private final Coder> windowsCoder; @@ -107,6 +110,7 @@ public Map factories() { } public static class Factory implements SinkFactory { + @Override public WindmillSink create( CloudObject spec, @@ -131,14 +135,21 @@ public SinkWriter> writer() { } class WindmillStreamWriter implements SinkWriter> { + private Map productionMap; private final String destinationName; private final ByteStringOutputStream stream; // Kept across encodes for buffer reuse. + // Builders are reused to reduce GC overhead. + private final Windmill.Message.Builder messageBuilder; + private final Windmill.OutputMessageBundle.Builder outputBuilder; + private WindmillStreamWriter(String destinationName) { this.destinationName = destinationName; productionMap = new HashMap<>(); stream = new ByteStringOutputStream(); + messageBuilder = Windmill.Message.newBuilder(); + outputBuilder = Windmill.OutputMessageBundle.newBuilder(); } private ByteString encode(Coder coder, EncodeT object) throws IOException { @@ -146,8 +157,13 @@ private ByteString encode(Coder coder, EncodeT object) throws stream.size() == 0, "Expected output stream to be empty but had %s", stream.toByteString()); - coder.encode(object, stream, Coder.Context.OUTER); - return stream.toByteStringAndReset(); + try { + coder.encode(object, stream, Coder.Context.OUTER); + return stream.toByteStringAndReset(); + } catch (Exception e) { + stream.toByteStringAndReset(); + throw e; + } } @Override @@ -208,28 +224,41 @@ public long add(WindowedValue data) throws IOException { productionMap.put(key, keyedOutput); } - Windmill.Message.Builder builder = - Windmill.Message.newBuilder() - .setTimestamp(WindmillTimeUtils.harnessToWindmillTimestamp(data.getTimestamp())) - .setData(value) - .setMetadata(metadata); - keyedOutput.addMessages(builder.build()); - + try { + messageBuilder + .setTimestamp(WindmillTimeUtils.harnessToWindmillTimestamp(data.getTimestamp())) + .setData(value) + .setMetadata(metadata); + keyedOutput.addMessages(messageBuilder.build()); + } finally { + messageBuilder.clear(); + } long offsetSize = 0; if (context.offsetBasedDeduplicationSupported()) { if (id.size() > 0) { throw new RuntimeException( "Unexpected record ID via ValueWithRecordIdCoder while offset-based deduplication enabled."); } - byte[] rawId = context.getCurrentRecordId(); - if (rawId.length == 0) { + byte[] rawId = null; + + if (data.getCurrentRecordId() != null) { + rawId = data.getCurrentRecordId().getBytes(StandardCharsets.UTF_8); + } else { + rawId = context.getCurrentRecordId(); + } + if (rawId == null || rawId.length == 0) { throw new RuntimeException( "Unexpected empty record ID while offset-based deduplication enabled."); } id = ByteString.copyFrom(rawId); - byte[] rawOffset = context.getCurrentRecordOffset(); - if (rawOffset.length == 0) { + byte[] rawOffset = null; + if (data.getCurrentRecordOffset() != null) { + rawOffset = Longs.toByteArray(data.getCurrentRecordOffset()); + } else { + rawOffset = context.getCurrentRecordOffset(); + } + if (rawOffset == null || rawOffset.length == 0) { throw new RuntimeException( "Unexpected empty record offset while offset-based deduplication enabled."); } @@ -245,14 +274,17 @@ public long add(WindowedValue data) throws IOException { @Override public void close() throws IOException { - Windmill.OutputMessageBundle.Builder outputBuilder = - Windmill.OutputMessageBundle.newBuilder().setDestinationStreamId(destinationName); + try { + outputBuilder.setDestinationStreamId(destinationName); - for (Windmill.KeyedMessageBundle.Builder keyedOutput : productionMap.values()) { - outputBuilder.addBundles(keyedOutput.build()); - } - if (outputBuilder.getBundlesCount() > 0) { - context.getOutputBuilder().addOutputMessages(outputBuilder.build()); + for (Windmill.KeyedMessageBundle.Builder keyedOutput : productionMap.values()) { + outputBuilder.addBundles(keyedOutput.build()); + } + if (outputBuilder.getBundlesCount() > 0) { + context.getOutputBuilder().addOutputMessages(outputBuilder.build()); + } + } finally { + outputBuilder.clear(); } productionMap.clear(); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillTimerInternals.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillTimerInternals.java index c8143cae864d..1dbc7b005345 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillTimerInternals.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillTimerInternals.java @@ -408,23 +408,25 @@ public static ByteString timerTag(WindmillNamespacePrefix prefix, TimerData time String tagString; if (useNewTimerTagEncoding(timerData)) { tagString = - new StringBuilder() - .append(prefix.byteString().toStringUtf8()) // this never ends with a slash - .append(timerData.getNamespace().stringKey()) // this must begin and end with a slash - .append('+') - .append(timerData.getTimerId()) // this is arbitrary; currently unescaped - .append('+') - .append(timerData.getTimerFamilyId()) - .toString(); + prefix.byteString().toStringUtf8() + + // this never ends with a slash + timerData.getNamespace().stringKey() + + // this must begin and end with a slash + '+' + + timerData.getTimerId() + + // this is arbitrary; currently unescaped + '+' + + timerData.getTimerFamilyId(); } else { // Timers without timerFamily would have timerFamily would be an empty string tagString = - new StringBuilder() - .append(prefix.byteString().toStringUtf8()) // this never ends with a slash - .append(timerData.getNamespace().stringKey()) // this must begin and end with a slash - .append('+') - .append(timerData.getTimerId()) // this is arbitrary; currently unescaped - .toString(); + prefix.byteString().toStringUtf8() + + // this never ends with a slash + timerData.getNamespace().stringKey() + + // this must begin and end with a slash + '+' + + timerData.getTimerId() // this is arbitrary; currently unescaped + ; } return ByteString.copyFromUtf8(tagString); } @@ -437,26 +439,30 @@ public static ByteString timerHoldTag(WindmillNamespacePrefix prefix, TimerData String tagString; if ("".equals(timerData.getTimerFamilyId())) { tagString = - new StringBuilder() - .append(prefix.byteString().toStringUtf8()) // this never ends with a slash - .append(TIMER_HOLD_PREFIX) // this never ends with a slash - .append(timerData.getNamespace().stringKey()) // this must begin and end with a slash - .append('+') - .append(timerData.getTimerId()) // this is arbitrary; currently unescaped - .toString(); + prefix.byteString().toStringUtf8() + + // this never ends with a slash + TIMER_HOLD_PREFIX + + // this never ends with a slash + timerData.getNamespace().stringKey() + + // this must begin and end with a slash + '+' + + timerData.getTimerId() // this is arbitrary; currently unescaped + ; } else { tagString = - new StringBuilder() - .append(prefix.byteString().toStringUtf8()) // this never ends with a slash - .append(TIMER_HOLD_PREFIX) // this never ends with a slash - .append(timerData.getNamespace().stringKey()) // this must begin and end with a slash - .append('+') - .append(timerData.getTimerId()) // this is arbitrary; currently unescaped - .append('+') - .append( - timerData.getTimerFamilyId()) // use to differentiate same timerId in different - // timerMap - .toString(); + prefix.byteString().toStringUtf8() + + // this never ends with a slash + TIMER_HOLD_PREFIX + + // this never ends with a slash + timerData.getNamespace().stringKey() + + // this must begin and end with a slash + '+' + + timerData.getTimerId() + + // this is arbitrary; currently unescaped + '+' + + timerData.getTimerFamilyId() // use to differentiate same timerId in different + // timerMap + ; } return ByteString.copyFromUtf8(tagString); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java index 6323e757561e..0181e647ac7b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java @@ -254,7 +254,8 @@ private static SourceOperationResponse performSplitTyped( // the sources into numBundlesLimit compressed serialized bundles. while (serializedSize > apiByteLimit || bundles.size() > numBundlesLimit) { // bundle size constrained by API limit, adds 5% allowance - int targetBundleSizeApiLimit = (int) (bundles.size() * apiByteLimit / serializedSize * 0.95); + int targetBundleSizeApiLimit = + (int) ((double) (bundles.size() * apiByteLimit) / serializedSize * 0.95); // bundle size constrained by numBundlesLimit int targetBundleSizeBundleLimit = Math.min(numBundlesLimit, bundles.size() - 1); int targetBundleSize = Math.min(targetBundleSizeApiLimit, targetBundleSizeBundleLimit); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingHandler.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingHandler.java index 572f9354ca93..864887f9bd36 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingHandler.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingHandler.java @@ -35,10 +35,13 @@ import java.text.SimpleDateFormat; import java.util.Date; import java.util.EnumMap; +import java.util.Map; import java.util.logging.ErrorManager; import java.util.logging.Handler; import java.util.logging.LogRecord; import java.util.logging.SimpleFormatter; +import javax.annotation.Nullable; +import javax.annotation.concurrent.GuardedBy; import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.runners.core.metrics.ExecutionStateTracker; import org.apache.beam.runners.core.metrics.ExecutionStateTracker.ExecutionState; @@ -47,6 +50,7 @@ import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.CountingOutputStream; +import org.slf4j.MDC; /** * Formats {@link LogRecord} into JSON format for Cloud Logging. Any exception is represented using @@ -83,6 +87,10 @@ public class DataflowWorkerLoggingHandler extends Handler { */ private static final int LOGGING_WRITER_BUFFER_SIZE = 262144; // 256kb + /** If true, add SLF4J MDC to custom_data of the log message. */ + @GuardedBy("this") + private boolean logCustomMdc = false; + /** * Formats the throwable as per {@link Throwable#printStackTrace()}. * @@ -123,6 +131,10 @@ public DataflowWorkerLoggingHandler(String filename, long sizeLimit) throws IOEx createOutputStream(); } + public synchronized void setLogMdc(boolean enabled) { + this.logCustomMdc = enabled; + } + @Override public synchronized void publish(LogRecord record) { DataflowExecutionState currrentDataflowState = null; @@ -171,6 +183,24 @@ public synchronized void publish(DataflowExecutionState currentExecutionState, L writeIfNotEmpty("work", DataflowWorkerLoggingMDC.getWorkId()); writeIfNotEmpty("logger", record.getLoggerName()); writeIfNotEmpty("exception", formatException(record.getThrown())); + if (logCustomMdc) { + @Nullable Map mdcMap = MDC.getCopyOfContextMap(); + if (mdcMap != null && !mdcMap.isEmpty()) { + generator.writeFieldName("custom_data"); + generator.writeStartObject(); + mdcMap.entrySet().stream() + .sorted(Map.Entry.comparingByKey()) + .forEach( + (entry) -> { + try { + generator.writeStringField(entry.getKey(), entry.getValue()); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + generator.writeEndObject(); + } + } generator.writeEndObject(); generator.writeRaw(System.lineSeparator()); } catch (IOException | RuntimeException e) { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingInitializer.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingInitializer.java index 0673ae790eaf..a56c62e92315 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingInitializer.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingInitializer.java @@ -247,6 +247,10 @@ public static synchronized void configure(DataflowWorkerLoggingOptions options) Charset.defaultCharset())); } + if (harnessOptions.getLogMdc()) { + loggingHandler.setLogMdc(true); + } + if (usedDeprecated) { LOG.warn( "Deprecated DataflowWorkerLoggingOptions are used for log level settings." diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/ValueInEmptyWindows.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/ValueInEmptyWindows.java index 42174629b3b8..1119617a068e 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/ValueInEmptyWindows.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/ValueInEmptyWindows.java @@ -49,6 +49,16 @@ public PaneInfo getPaneInfo() { return PaneInfo.NO_FIRING; } + @Override + public @Nullable String getCurrentRecordId() { + return null; + } + + @Override + public @Nullable Long getCurrentRecordOffset() { + return null; + } + @Override public Iterable> explodeWindows() { return Collections.emptyList(); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java index cf46e1f984dc..ed99ae1bbd6f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStream.java @@ -17,13 +17,22 @@ */ package org.apache.beam.runners.dataflow.worker.windmill.client; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; + import com.google.errorprone.annotations.CanIgnoreReturnValue; import java.io.PrintWriter; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collections; +import java.util.IdentityHashMap; +import java.util.List; import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; +import java.util.concurrent.Future; import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; @@ -37,9 +46,7 @@ import org.apache.beam.vendor.grpc.v1p69p0.com.google.api.client.util.Sleeper; import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.Status; import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.stub.StreamObserver; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.checkerframework.checker.nullness.qual.NonNull; -import org.joda.time.DateTime; import org.joda.time.Instant; import org.slf4j.Logger; @@ -75,11 +82,10 @@ public abstract class AbstractWindmillStream implements Win // shutdown. private static final Status OK_STATUS = Status.fromCode(Status.Code.OK); private static final String NEVER_RECEIVED_RESPONSE_LOG_STRING = "never received response"; - private static final String NOT_SHUTDOWN = "not shutdown"; protected final Sleeper sleeper; private final Logger logger; - private final ExecutorService executor; + private final ScheduledExecutorService executor; private final BackOff backoff; private final CountDownLatch finishLatch; private final Set> streamRegistry; @@ -89,6 +95,7 @@ public abstract class AbstractWindmillStream implements Win private final Function, TerminatingStreamObserver> physicalStreamFactory; protected final long physicalStreamDeadlineSeconds; + private final Duration halfClosePhysicalStreamAfter; private final ResettableThrowingStreamObserver requestObserver; private final StreamDebugMetrics debugMetrics; @@ -106,6 +113,17 @@ public abstract class AbstractWindmillStream implements Win @GuardedBy("this") protected @Nullable PhysicalStreamHandler currentPhysicalStream; + @GuardedBy("this") + @Nullable + Future halfCloseFuture = null; + + // Physical streams that have been half-closed and are waiting for responses or stream failure. + @GuardedBy("this") + protected final Set closingPhysicalStreams; + + private final Set closingPhysicalStreamsForDebug = + Collections.newSetFromMap(new ConcurrentHashMap()); + // Generally the same as currentPhysicalStream, set under synchronization of this but can be read // without. private final AtomicReference currentPhysicalStreamForDebug = @@ -114,25 +132,33 @@ public abstract class AbstractWindmillStream implements Win @GuardedBy("this") private boolean started; + // If halfClosePhysicalStream is non-zero, substreams created for the logical + // AbstractWindmillStream + // will be half-closed and a new physical stream will be created after this duraction. protected AbstractWindmillStream( Logger logger, - String debugStreamType, Function, StreamObserver> clientFactory, BackOff backoff, StreamObserverFactory streamObserverFactory, Set> streamRegistry, int logEveryNStreamFailures, - String backendWorkerToken) { + String backendWorkerToken, + Duration halfClosePhysicalStreamAfter, + ScheduledExecutorService executor) { + checkArgument(!halfClosePhysicalStreamAfter.isNegative()); this.backendWorkerToken = backendWorkerToken; this.physicalStreamFactory = (StreamObserver observer) -> streamObserverFactory.from(clientFactory, observer); this.physicalStreamDeadlineSeconds = streamObserverFactory.getDeadlineSeconds(); - this.executor = - Executors.newSingleThreadExecutor( - new ThreadFactoryBuilder() - .setDaemon(true) - .setNameFormat(createThreadName(debugStreamType, backendWorkerToken)) - .build()); + if (!halfClosePhysicalStreamAfter.isZero() + && halfClosePhysicalStreamAfter.compareTo(Duration.ofSeconds(physicalStreamDeadlineSeconds)) + >= 0) { + logger.debug("Not attempting to half-close cleanly as stream deadline is shorter."); + halfClosePhysicalStreamAfter = Duration.ZERO; + } + this.halfClosePhysicalStreamAfter = halfClosePhysicalStreamAfter; + this.closingPhysicalStreams = Collections.newSetFromMap(new IdentityHashMap<>()); + this.executor = executor; this.backoff = backoff; this.streamRegistry = streamRegistry; this.logEveryNStreamFailures = logEveryNStreamFailures; @@ -147,12 +173,6 @@ protected AbstractWindmillStream( this.debugMetrics = StreamDebugMetrics.create(); } - private static String createThreadName(String streamType, String backendWorkerToken) { - return !backendWorkerToken.isEmpty() - ? String.format("%s-%s-WindmillStream-thread", streamType, backendWorkerToken) - : String.format("%s-WindmillStream-thread", streamType); - } - /** Represents a physical grpc stream that is part of the logical windmill stream. */ protected abstract class PhysicalStreamHandler { @@ -178,11 +198,23 @@ protected abstract class PhysicalStreamHandler { public abstract void appendHtml(PrintWriter writer); private final StreamDebugMetrics streamDebugMetrics = StreamDebugMetrics.create(); + + @Override + public final boolean equals(@Nullable Object obj) { + return this == obj; + } + + @Override + public final int hashCode() { + return System.identityHashCode(this); + } } + /* Constructs and returns a new handler to be associated with a physical stream. */ protected abstract PhysicalStreamHandler newResponseHandler(); - protected abstract void onNewStream() throws WindmillStreamShutdownException; + protected abstract void onFlushPending(boolean isNewStream) + throws WindmillStreamShutdownException; /** Try to send a request to the server. Returns true if the request was successfully sent. */ @CanIgnoreReturnValue @@ -214,54 +246,68 @@ public final void start() { } if (shouldStartStream) { + // Add the stream to the registry after it has been fully constructed. + streamRegistry.add(this); startStream(); } } /** Starts the underlying stream. */ private void startStream() { - // Add the stream to the registry after it has been fully constructed. - streamRegistry.add(this); while (true) { @NonNull PhysicalStreamHandler streamHandler = newResponseHandler(); - try { - synchronized (this) { + synchronized (this) { + try { + checkState(currentPhysicalStream == null, "Overwriting existing physical stream"); + checkState(halfCloseFuture == null, "Unexpected half-close future"); + if (isShutdown) { + // No need to start the stream. shutdown() or onPhysicalStreamCompletion will be + // responsible for completing shutdown. + return; + } debugMetrics.recordStart(); streamHandler.streamDebugMetrics.recordStart(); currentPhysicalStream = streamHandler; currentPhysicalStreamForDebug.set(currentPhysicalStream); requestObserver.reset(physicalStreamFactory.apply(new ResponseObserver(streamHandler))); - onNewStream(); + onFlushPending(true); if (clientClosed) { - halfClose(); + // The logical stream is half-closed so after flushing the remaining requests close the + // physical stream. + streamHandler.streamDebugMetrics.recordHalfClose(); + requestObserver.onCompleted(); + } else if (!halfClosePhysicalStreamAfter.isZero()) { + halfCloseFuture = + executor.schedule( + () -> onHalfClosePhysicalStreamTimeout(streamHandler), + halfClosePhysicalStreamAfter.getSeconds(), + TimeUnit.SECONDS); } return; - } - } catch (WindmillStreamShutdownException e) { - // shutdown() is responsible for cleaning up pending requests. - logger.debug("Stream was shutdown while creating new stream.", e); - break; - } catch (Exception e) { - logger.error("Failed to create new stream, retrying: ", e); - try { - long sleep = backoff.nextBackOffMillis(); - debugMetrics.recordSleep(sleep); - sleeper.sleep(sleep); - } catch (InterruptedException ie) { - Thread.currentThread().interrupt(); - logger.info( - "Interrupted during {} creation backoff. The stream will not be created.", - getClass()); - // Shutdown the stream to clean up any dangling resources and pending requests. - shutdown(); + } catch (WindmillStreamShutdownException e) { + logger.debug("Stream was shutdown while creating new stream.", e); + clearCurrentPhysicalStream(true); break; + } catch (Exception e) { + logger.error("Failed to create new stream, retrying: ", e); + clearCurrentPhysicalStream(true); + debugMetrics.recordRestartReason("Failed to create new stream, retrying: " + e); } } + // Backoff outside the synchronized block. + try { + long sleep = backoff.nextBackOffMillis(); + debugMetrics.recordSleep(sleep); + sleeper.sleep(sleep); + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + logger.info( + "Interrupted during {} creation backoff. The stream will not be created.", getClass()); + // Shutdown the stream to clean up any dangling resources and pending requests. + shutdown(); + break; + } } - - // We were never able to start the stream, remove it from the stream registry. Otherwise, it is - // removed when closed. - streamRegistry.remove(this); } /** @@ -317,23 +363,6 @@ public final void maybeScheduleHealthCheck(Instant lastSendThreshold) { */ public final void appendSummaryHtml(PrintWriter writer) { appendSpecificHtml(writer); - - @Nullable PhysicalStreamHandler currentHandler = currentPhysicalStreamForDebug.get(); - if (currentHandler != null) { - writer.format("Physical stream: "); - currentHandler.appendHtml(writer); - StreamDebugMetrics.Snapshot summaryMetrics = - currentHandler.streamDebugMetrics.getSummaryMetrics(); - if (summaryMetrics.isClientClosed()) { - writer.write(" client closed"); - } - writer.format( - " current stream is %dms old, last send %dms, last response %dms\n", - summaryMetrics.streamAge(), - summaryMetrics.timeSinceLastSend(), - summaryMetrics.timeSinceLastResponse()); - } - StreamDebugMetrics.Snapshot summaryMetrics = debugMetrics.getSummaryMetrics(); summaryMetrics .restartMetrics() @@ -356,13 +385,44 @@ public final void appendSummaryHtml(PrintWriter writer) { } writer.format( - ", current stream is %dms old, last send %dms, last response %dms, closed: %s, " - + "shutdown time: %s", + ", stream is %dms old, last send %dms, last response %dms", summaryMetrics.streamAge(), summaryMetrics.timeSinceLastSend(), - summaryMetrics.timeSinceLastResponse(), - requestObserver.isClosed(), - summaryMetrics.shutdownTime().map(DateTime::toString).orElse(NOT_SHUTDOWN)); + summaryMetrics.timeSinceLastResponse()); + if (requestObserver.isClosed()) { + writer.append(", observer closed"); + } + summaryMetrics + .shutdownTime() + .ifPresent(dateTime -> writer.format(", shutdown at %s", dateTime)); + + @Nullable PhysicalStreamHandler currentHandler = currentPhysicalStreamForDebug.get(); + if (currentHandler != null) { + writer.format("
current physical stream: "); + appendPhysicalStream(writer, currentHandler); + } + + List closingStreamsSnapshot = + new ArrayList<>(closingPhysicalStreamsForDebug); + for (int i = 0; i < closingStreamsSnapshot.size(); ++i) { + writer.format("
closing physical stream #%d: ", i); + appendPhysicalStream(writer, closingStreamsSnapshot.get(i)); + } + } + + private void appendPhysicalStream( + PrintWriter writer, PhysicalStreamHandler physicalStreamHandler) { + physicalStreamHandler.appendHtml(writer); + StreamDebugMetrics.Snapshot summaryMetrics = + physicalStreamHandler.streamDebugMetrics.getSummaryMetrics(); + if (summaryMetrics.isClientClosed()) { + writer.write(" client closed"); + } + writer.format( + " started %dms ago, last send %dms, last response %dms\n", + summaryMetrics.streamAge(), + summaryMetrics.timeSinceLastSend(), + summaryMetrics.timeSinceLastResponse()); } /** @@ -375,7 +435,12 @@ public final void appendSummaryHtml(PrintWriter writer) { @Override public final synchronized void halfClose() { - // Synchronization of close and onCompleted necessary for correct retry logic in onNewStream. + if (clientClosed) { + logger.warn("Stream was previously closed."); + return; + } + // Synchronization of close and onCompleted necessary for correct retry logic in + // onPhysicalStreamCompleted. debugMetrics.recordHalfClose(); clientClosed = true; try { @@ -399,7 +464,7 @@ public final boolean awaitTermination(int time, TimeUnit unit) throws Interrupte @Override public final Instant startTime() { - return new Instant(debugMetrics.getStartTimeMs()); + return Instant.ofEpochMilli(debugMetrics.getStartTimeMs()); } @Override @@ -417,30 +482,22 @@ public final void shutdown() { isShutdown = true; debugMetrics.recordShutdown(); shutdownInternal(); + if (currentPhysicalStream == null && closingPhysicalStreams.isEmpty()) { + completeShutdown(); + } } } } - protected synchronized void shutdownInternal() {} - - /** Returns true if the stream was torn down and should not be restarted internally. */ - private synchronized boolean maybeTearDownStream(PhysicalStreamHandler doneStream) { - if (clientClosed && !doneStream.hasPendingRequests()) { - shutdown(); - } - - if (isShutdown) { - // Once we have background closing physicalStreams we will need to improve this to wait for - // all of the work of the logical stream to be complete. - streamRegistry.remove(AbstractWindmillStream.this); - finishLatch.countDown(); - executor.shutdownNow(); - return true; - } - - return false; + private void completeShutdown() { + logger.debug("Completing shutdown of stream after shutdown and all streams terminated."); + streamRegistry.remove(AbstractWindmillStream.this); + finishLatch.countDown(); + executor.shutdownNow(); } + protected synchronized void shutdownInternal() {} + private class ResponseObserver implements StreamObserver { private final PhysicalStreamHandler handler; @@ -467,22 +524,67 @@ public void onCompleted() { } } - @SuppressWarnings("nullness") - private void clearPhysicalStreamForDebug() { - currentPhysicalStreamForDebug.set(null); + @SuppressWarnings("ReferenceEquality") + private void onHalfClosePhysicalStreamTimeout(PhysicalStreamHandler handler) { + synchronized (this) { + if (currentPhysicalStream != handler || clientClosed || isShutdown) { + return; + } + handler.streamDebugMetrics.recordHalfClose(); + closingPhysicalStreams.add(handler); + closingPhysicalStreamsForDebug.add(handler); + clearCurrentPhysicalStream(false); + try { + requestObserver.onCompleted(); + } catch (Exception e) { + logger.debug( + "Exception while half-closing handler, onPhysicalStreamCompletion will be called for the stream", + e); + } + } + startStream(); } + @SuppressWarnings("ReferenceEquality") private void onPhysicalStreamCompletion(Status status, PhysicalStreamHandler handler) { synchronized (this) { - if (currentPhysicalStream == handler) { - clearPhysicalStreamForDebug(); - currentPhysicalStream = null; + final boolean wasActiveStream = currentPhysicalStream == handler; + if (wasActiveStream) { + clearCurrentPhysicalStream(true); + } else { + checkState(closingPhysicalStreams.remove(handler)); + closingPhysicalStreamsForDebug.remove(handler); } + boolean doneHandlerHadRequests = handler.hasPendingRequests(); + handler.onDone(status); + if (currentPhysicalStream == null && closingPhysicalStreams.isEmpty()) { + if (clientClosed && !doneHandlerHadRequests && !isShutdown) { + shutdown(); + } + if (isShutdown) { + completeShutdown(); + return; + } + } + if (currentPhysicalStream != null) { + if (!clientClosed) { + // Don't bother attempting to flush the requests if the active stream is closed. + try { + onFlushPending(false); + } catch (WindmillStreamShutdownException e) { + logger.debug( + "Requests will be flushed by onPhysicalStreamCompletion of the current stream.", e); + } + } + return; + } + if (clientClosed && !doneHandlerHadRequests) { + // We didn't have any leftover requests and are closing so we skip restarting a stream. + return; + } + // We're not shutting down and we don't have an active stream, create one. } - handler.onDone(status); - if (maybeTearDownStream(handler)) { - return; - } + // Backoff on errors.; if (!status.isOk()) { try { @@ -498,6 +600,16 @@ private void onPhysicalStreamCompletion(Status status, PhysicalStreamHandler han startStream(); } + @SuppressWarnings("nullness") + private synchronized void clearCurrentPhysicalStream(boolean cancelHalfCloseFuture) { + currentPhysicalStream = null; + if (halfCloseFuture != null && cancelHalfCloseFuture) { + halfCloseFuture.cancel(false); + } + halfCloseFuture = null; + currentPhysicalStreamForDebug.set(null); + } + private void recordStreamRestart(Status status) { int currentRestartCount = debugMetrics.incrementAndGetRestarts(); if (status.isOk()) { diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java index 51bc03e8e0e7..526b67890783 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/WindmillStream.java @@ -64,10 +64,6 @@ public interface WindmillStream { interface GetWorkStream extends WindmillStream { /** Adjusts the {@link GetWorkBudget} for the stream. */ void setBudget(GetWorkBudget newBudget); - - default void setBudget(long newItems, long newBytes) { - setBudget(GetWorkBudget.builder().setItems(newItems).setBytes(newBytes).build()); - } } /** Interface for streaming GetDataRequests to Windmill. */ diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GetWorkResponseChunkAssembler.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GetWorkResponseChunkAssembler.java index f978bad01e62..0ebb4726d3a1 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GetWorkResponseChunkAssembler.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GetWorkResponseChunkAssembler.java @@ -51,6 +51,7 @@ final class GetWorkResponseChunkAssembler { private final GetWorkTimingInfosTracker workTimingInfosTracker; private @Nullable ComputationMetadata metadata; + private final WorkItem.Builder workItemBuilder; // Reused to reduce GC overhead. private ByteString data; private long bufferedSize; @@ -59,6 +60,7 @@ final class GetWorkResponseChunkAssembler { data = ByteString.EMPTY; bufferedSize = 0; metadata = null; + workItemBuilder = WorkItem.newBuilder(); } /** @@ -94,15 +96,17 @@ List append(Windmill.StreamingGetWorkResponseChunk chunk) { */ private Optional flushToWorkItem() { try { + workItemBuilder.mergeFrom(data); return Optional.of( AssembledWorkItem.create( - WorkItem.parseFrom(data.newInput()), + workItemBuilder.build(), Preconditions.checkNotNull(metadata), workTimingInfosTracker.getLatencyAttributions(), bufferedSize)); } catch (IOException e) { LOG.error("Failed to parse work item from stream: ", e); } finally { + workItemBuilder.clear(); workTimingInfosTracker.reset(); data = ByteString.EMPTY; bufferedSize = 0; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcCommitWorkStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcCommitWorkStream.java index 7a7b1a5cd27e..d24676652fd8 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcCommitWorkStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcCommitWorkStream.java @@ -22,12 +22,14 @@ import com.google.auto.value.AutoValue; import java.io.PrintWriter; +import java.time.Duration; import java.util.HashMap; import java.util.Iterator; import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; import java.util.function.Function; @@ -74,6 +76,7 @@ private static class StreamAndRequest { private final AtomicLong idGenerator; private final JobHeader jobHeader; private final int streamingRpcBatchLimit; + private volatile boolean logMissingResponse = true; private GrpcCommitWorkStream( String backendWorkerToken, @@ -85,16 +88,19 @@ private GrpcCommitWorkStream( int logEveryNStreamFailures, JobHeader jobHeader, AtomicLong idGenerator, - int streamingRpcBatchLimit) { + int streamingRpcBatchLimit, + Duration halfClosePhysicalStreamAfter, + ScheduledExecutorService executor) { super( LOG, - "CommitWorkStream", startCommitWorkRpcFn, backoff, streamObserverFactory, streamRegistry, logEveryNStreamFailures, - backendWorkerToken); + backendWorkerToken, + halfClosePhysicalStreamAfter, + executor); this.idGenerator = idGenerator; this.jobHeader = jobHeader; this.streamingRpcBatchLimit = streamingRpcBatchLimit; @@ -110,7 +116,9 @@ static GrpcCommitWorkStream create( int logEveryNStreamFailures, JobHeader jobHeader, AtomicLong idGenerator, - int streamingRpcBatchLimit) { + int streamingRpcBatchLimit, + Duration halfClosePhysicalStreamAfter, + ScheduledExecutorService executor) { return new GrpcCommitWorkStream( backendWorkerToken, startCommitWorkRpcFn, @@ -120,25 +128,33 @@ static GrpcCommitWorkStream create( logEveryNStreamFailures, jobHeader, idGenerator, - streamingRpcBatchLimit); + streamingRpcBatchLimit, + halfClosePhysicalStreamAfter, + executor); } @Override public void appendSpecificHtml(PrintWriter writer) { - writer.format("CommitWorkStream: %d pending", pending.size()); + writer.format("CommitWorkStream: %d pending ", pending.size()); } @Override - protected synchronized void onNewStream() throws WindmillStreamShutdownException { - trySend(StreamingCommitWorkRequest.newBuilder().setHeader(jobHeader).build()); + @SuppressWarnings("ReferenceEquality") + protected synchronized void onFlushPending(boolean isNewStream) + throws WindmillStreamShutdownException { + if (isNewStream) { + trySend(StreamingCommitWorkRequest.newBuilder().setHeader(jobHeader).build()); + } // Flush all pending requests that are no longer on active streams. try (Batcher resendBatcher = new Batcher()) { for (Map.Entry entry : pending.entrySet()) { CommitWorkPhysicalStreamHandler requestHandler = entry.getValue().handler; checkState(requestHandler != currentPhysicalStream); - // When we have streams closing in the background we should avoid retrying the requests - // active on those streams. - + if (requestHandler != null && closingPhysicalStreams.contains(requestHandler)) { + LOG.debug( + "Not resending request that is active on background half-closing physical stream."); + continue; + } long id = entry.getKey(); PendingRequest request = entry.getValue().request; if (!resendBatcher.canAccept(request.getBytes())) { @@ -169,6 +185,7 @@ protected synchronized void sendHealthCheck() throws WindmillStreamShutdownExcep private class CommitWorkPhysicalStreamHandler extends PhysicalStreamHandler { @Override + @SuppressWarnings("ReferenceEquality") public void onResponse(StreamingCommitResponse response) { CommitCompletionFailureHandler failureHandler = new CommitCompletionFailureHandler(); for (int i = 0; i < response.getRequestIdCount(); ++i) { @@ -184,7 +201,9 @@ public void onResponse(StreamingCommitResponse response) { @Nullable StreamAndRequest entry = pending.remove(requestId); if (entry == null) { - LOG.error("Got unknown commit request ID: {}", requestId); + if (logMissingResponse) { + LOG.error("Got unknown commit request ID: {}", requestId); + } continue; } if (entry.handler != this) { @@ -206,6 +225,7 @@ public void onResponse(StreamingCommitResponse response) { } @Override + @SuppressWarnings("ReferenceEquality") public boolean hasPendingRequests() { return pending.entrySet().stream().anyMatch(e -> e.getValue().handler == this); } @@ -218,6 +238,7 @@ public void onDone(Status status) { } @Override + @SuppressWarnings("ReferenceEquality") public void appendHtml(PrintWriter writer) { writer.format( "CommitWorkStream: %d pending", @@ -232,6 +253,7 @@ protected PhysicalStreamHandler newResponseHandler() { @Override protected synchronized void shutdownInternal() { + logMissingResponse = false; Iterator pendingRequests = pending.values().iterator(); while (pendingRequests.hasNext()) { PendingRequest pendingRequest = pendingRequests.next().request; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java index 938ec1c693c7..2712bf1bd33d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java @@ -20,9 +20,11 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; import java.io.PrintWriter; +import java.time.Duration; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import javax.annotation.concurrent.GuardedBy; @@ -98,16 +100,19 @@ private GrpcDirectGetWorkStream( HeartbeatSender heartbeatSender, GetDataClient getDataClient, WorkCommitter workCommitter, - WorkItemScheduler workItemScheduler) { + WorkItemScheduler workItemScheduler, + Duration halfClosePhysicalStreamAfter, + ScheduledExecutorService executorService) { super( LOG, - "GetWorkStream", startGetWorkRpcFn, backoff, streamObserverFactory, streamRegistry, logEveryNStreamFailures, - backendWorkerToken); + backendWorkerToken, + halfClosePhysicalStreamAfter, + executorService); this.requestHeader = requestHeader; this.workItemScheduler = workItemScheduler; this.heartbeatSender = heartbeatSender; @@ -138,7 +143,9 @@ static GrpcDirectGetWorkStream create( HeartbeatSender heartbeatSender, GetDataClient getDataClient, WorkCommitter workCommitter, - WorkItemScheduler workItemScheduler) { + WorkItemScheduler workItemScheduler, + Duration halfClosePhysicalStreamAfter, + ScheduledExecutorService executor) { return new GrpcDirectGetWorkStream( backendWorkerToken, startGetWorkRpcFn, @@ -151,7 +158,9 @@ static GrpcDirectGetWorkStream create( heartbeatSender, getDataClient, workCommitter, - workItemScheduler); + workItemScheduler, + halfClosePhysicalStreamAfter, + executor); } private static Watermarks createWatermarks( @@ -230,7 +239,11 @@ protected PhysicalStreamHandler newResponseHandler() { } @Override - protected synchronized void onNewStream() throws WindmillStreamShutdownException { + protected synchronized void onFlushPending(boolean isNewStream) + throws WindmillStreamShutdownException { + if (!isNewStream) { + return; + } budgetTracker.reset(); GetWorkBudget initialGetWorkBudget = budgetTracker.computeBudgetExtension(); StreamingGetWorkRequest request = diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDispatcherClient.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDispatcherClient.java index 3603bacf461a..82e66c4b0d74 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDispatcherClient.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDispatcherClient.java @@ -21,6 +21,8 @@ import static org.apache.beam.runners.dataflow.worker.windmill.client.grpc.stubs.WindmillChannels.localhostChannel; import com.google.auto.value.AutoValue; +import java.time.Duration; +import java.time.Instant; import java.util.List; import java.util.Random; import java.util.Set; @@ -132,16 +134,15 @@ ImmutableSet getDispatcherEndpoints() { /** Will block the calling thread until the initial endpoints are present. */ public CloudWindmillMetadataServiceV1Alpha1Stub getWindmillMetadataServiceStubBlocking() { - boolean initialized = false; - long secondsWaited = 0; - while (!initialized) { - LOG.info( - "Blocking until Windmill Service endpoint has been set. " - + "Currently waited for [{}] seconds.", - secondsWaited); + Instant startTime = Instant.now(); + while (true) { try { - initialized = onInitializedEndpoints.await(10, TimeUnit.SECONDS); - secondsWaited += 10; + if (onInitializedEndpoints.await(10, TimeUnit.SECONDS)) { + break; + } + LOG.info( + "Blocking until Windmill Service endpoint has been set. " + "Currently waited for {}.", + Duration.between(startTime, Instant.now())); } catch (InterruptedException e) { LOG.error( "Interrupted while waiting for initial Windmill Service endpoints. " @@ -149,8 +150,10 @@ public CloudWindmillMetadataServiceV1Alpha1Stub getWindmillMetadataServiceStubBl e); } } - - LOG.info("Windmill Service endpoint initialized after {} seconds.", secondsWaited); + Duration elapsed = Duration.between(startTime, Instant.now()); + if (elapsed.getSeconds() >= 5) { + LOG.info("Windmill Service endpoint initialized after {}.", elapsed); + } ImmutableList windmillMetadataServiceStubs = dispatcherStubs.get().windmillMetadataServiceStubs(); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java index 7de074122a3c..6d6dcd569e85 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStream.java @@ -33,6 +33,7 @@ import java.util.concurrent.CancellationException; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedDeque; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; import java.util.function.Function; @@ -112,16 +113,19 @@ private GrpcGetDataStream( AtomicLong idGenerator, int streamingRpcBatchLimit, boolean sendKeyedGetDataRequests, - Consumer> processHeartbeatResponses) { + Consumer> processHeartbeatResponses, + java.time.Duration halfClosePhysicalStreamAfter, + ScheduledExecutorService executorService) { super( LOG, - "GetDataStream", startGetDataRpcFn, backoff, streamObserverFactory, streamRegistry, logEveryNStreamFailures, - backendWorkerToken); + backendWorkerToken, + halfClosePhysicalStreamAfter, + executorService); this.idGenerator = idGenerator; this.jobHeader = jobHeader; this.streamingRpcBatchLimit = streamingRpcBatchLimit; @@ -146,7 +150,9 @@ static GrpcGetDataStream create( AtomicLong idGenerator, int streamingRpcBatchLimit, boolean sendKeyedGetDataRequests, - Consumer> processHeartbeatResponses) { + Consumer> processHeartbeatResponses, + java.time.Duration halfClosePhysicalStreamAfter, + ScheduledExecutorService executor) { return new GrpcGetDataStream( backendWorkerToken, startGetDataRpcFn, @@ -158,7 +164,9 @@ static GrpcGetDataStream create( idGenerator, streamingRpcBatchLimit, sendKeyedGetDataRequests, - processHeartbeatResponses); + processHeartbeatResponses, + halfClosePhysicalStreamAfter, + executor); } private static WindmillStreamShutdownException shutdownExceptionFor(QueuedBatch batch) { @@ -189,7 +197,7 @@ public void sendBatch(QueuedBatch batch) throws WindmillStreamShutdownException } if (!trySend(batch.asGetDataRequest())) { - // The stream broke before this call went through; onNewStream will retry the fetch. + // The stream broke before this call went through; onFlushPending will retry the fetch. LOG.debug("GetData stream broke before call started."); } } @@ -260,8 +268,11 @@ protected PhysicalStreamHandler newResponseHandler() { } @Override - protected synchronized void onNewStream() throws WindmillStreamShutdownException { - trySend(StreamingGetDataRequest.newBuilder().setHeader(jobHeader).build()); + protected synchronized void onFlushPending(boolean isNewStream) + throws WindmillStreamShutdownException { + if (isNewStream) { + trySend(StreamingGetDataRequest.newBuilder().setHeader(jobHeader).build()); + } while (!batches.isEmpty()) { QueuedBatch batch = checkNotNull(batches.peekFirst()); verify(!batch.isEmpty()); @@ -392,6 +403,12 @@ protected synchronized void shutdownInternal() { } currentGetDataStream.pending.clear(); } + for (PhysicalStreamHandler handler : closingPhysicalStreams) { + for (AppendableInputStream ais : ((GetDataPhysicalStreamHandler) handler).pending.values()) { + ais.cancel(); + } + ((GetDataPhysicalStreamHandler) handler).pending.clear(); + } batches.forEach( batch -> { batch.markFinalized(); @@ -402,7 +419,12 @@ protected synchronized void shutdownInternal() { @Override public void appendSpecificHtml(PrintWriter writer) { - writer.format("GetDataStream: %d queued batches", batchesDebugSizeSupplier.get()); + int batches = batchesDebugSizeSupplier.get(); + if (batches > 0) { + writer.format("GetDataStream: %d queued batches ", batches); + } else { + writer.append("GetDataStream: no queued batches "); + } } private ResponseT issueRequest(QueuedRequest request, ParseFn parseFn) @@ -476,10 +498,11 @@ private void queueRequestAndWait(QueuedRequest request) prevBatch.waitForSendOrFailNotification(); } trySendBatch(batch); - } else { - // Wait for this batch to be sent before parsing the response. - batch.waitForSendOrFailNotification(); + // Since the above send may not succeed, we fall through to block on sending or failure. } + + // Wait for this batch to be sent before parsing the response. + batch.waitForSendOrFailNotification(); } private synchronized void trySendBatch(QueuedBatch batch) throws WindmillStreamShutdownException { @@ -494,8 +517,8 @@ private synchronized void trySendBatch(QueuedBatch batch) throws WindmillStreamS final @Nullable GetDataPhysicalStreamHandler currentGetDataPhysicalStream = (GetDataPhysicalStreamHandler) currentPhysicalStream; if (currentGetDataPhysicalStream == null) { - // Leave the batch finalized but in the batches queue. Finalized batches will be sent on the - // new stream in onNewStream. + // Leave the batch finalized but in the batches queue. Finalized batches will be sent on a + // new stream in onFlushPending. return; } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkStream.java index a1c758eac446..ae7ce85e13a8 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkStream.java @@ -18,8 +18,10 @@ package org.apache.beam.runners.dataflow.worker.windmill.client.grpc; import java.io.PrintWriter; +import java.time.Duration; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetWorkRequest; @@ -68,16 +70,19 @@ private GrpcGetWorkStream( Set> streamRegistry, int logEveryNStreamFailures, boolean requestBatchedGetWorkResponse, - WorkItemReceiver receiver) { + WorkItemReceiver receiver, + Duration halfClosePhysicalStreamAfter, + ScheduledExecutorService executor) { super( LOG, - "GetWorkStream", startGetWorkRpcFn, backoff, streamObserverFactory, streamRegistry, logEveryNStreamFailures, - backendWorkerToken); + backendWorkerToken, + halfClosePhysicalStreamAfter, + executor); this.request = request; this.receiver = receiver; this.inflightMessages = new AtomicLong(); @@ -97,7 +102,9 @@ public static GrpcGetWorkStream create( Set> streamRegistry, int logEveryNStreamFailures, boolean requestBatchedGetWorkResponse, - WorkItemReceiver receiver) { + WorkItemReceiver receiver, + Duration halfClosePhysicalStreamAfter, + ScheduledExecutorService executor) { return new GrpcGetWorkStream( backendWorkerToken, startGetWorkRpcFn, @@ -107,7 +114,9 @@ public static GrpcGetWorkStream create( streamRegistry, logEveryNStreamFailures, requestBatchedGetWorkResponse, - receiver); + receiver, + halfClosePhysicalStreamAfter, + executor); } private void sendRequestExtension(long moreItems, long moreBytes) { @@ -163,7 +172,11 @@ protected PhysicalStreamHandler newResponseHandler() { } @Override - protected synchronized void onNewStream() throws WindmillStreamShutdownException { + protected synchronized void onFlushPending(boolean isNewStream) + throws WindmillStreamShutdownException { + if (!isNewStream) { + return; + } inflightMessages.set(request.getMaxItems()); inflightBytes.set(request.getMaxBytes()); trySend( diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkerMetadataStream.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkerMetadataStream.java index 9b99b3bda909..4d1e8bb36d9d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkerMetadataStream.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetWorkerMetadataStream.java @@ -19,8 +19,11 @@ import com.google.errorprone.annotations.concurrent.GuardedBy; import java.io.PrintWriter; +import java.time.Duration; +import java.time.Instant; import java.util.Optional; import java.util.Set; +import java.util.concurrent.ScheduledExecutorService; import java.util.function.Consumer; import java.util.function.Function; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.JobHeader; @@ -50,6 +53,9 @@ public final class GrpcGetWorkerMetadataStream @GuardedBy("metadataLock") private WorkerMetadataResponse latestResponse; + @GuardedBy("metadataLock") + private Instant latestResponseReceived = Instant.EPOCH; + private GrpcGetWorkerMetadataStream( Function, StreamObserver> startGetWorkerMetadataRpcFn, @@ -58,16 +64,19 @@ private GrpcGetWorkerMetadataStream( Set> streamRegistry, int logEveryNStreamFailures, JobHeader jobHeader, - Consumer serverMappingConsumer) { + Consumer serverMappingConsumer, + Duration halfClosePhysicalStreamAfter, + ScheduledExecutorService executorService) { super( LOG, - "GetWorkerMetadataStream", startGetWorkerMetadataRpcFn, backoff, streamObserverFactory, streamRegistry, logEveryNStreamFailures, - ""); + "", + halfClosePhysicalStreamAfter, + executorService); this.workerMetadataRequest = WorkerMetadataRequest.newBuilder().setHeader(jobHeader).build(); this.serverMappingConsumer = serverMappingConsumer; this.latestResponse = WorkerMetadataResponse.getDefaultInstance(); @@ -82,7 +91,9 @@ public static GrpcGetWorkerMetadataStream create( Set> streamRegistry, int logEveryNStreamFailures, JobHeader jobHeader, - Consumer serverMappingUpdater) { + Consumer serverMappingUpdater, + Duration halfClosePhysicalStreamAfter, + ScheduledExecutorService executorService) { return new GrpcGetWorkerMetadataStream( startGetWorkerMetadataRpcFn, backoff, @@ -90,7 +101,9 @@ public static GrpcGetWorkerMetadataStream create( streamRegistry, logEveryNStreamFailures, jobHeader, - serverMappingUpdater); + serverMappingUpdater, + halfClosePhysicalStreamAfter, + executorService); } /** @@ -103,6 +116,7 @@ private Optional extractWindmillEndpointsFrom( synchronized (metadataLock) { if (response.getMetadataVersion() > latestResponse.getMetadataVersion()) { this.latestResponse = response; + this.latestResponseReceived = Instant.now(); return Optional.of(WindmillEndpoints.from(response)); } else { // If the currentMetadataVersion is greater than or equal to one in the response, the @@ -141,8 +155,10 @@ public void appendHtml(PrintWriter writer) {} } @Override - protected void onNewStream() throws WindmillStreamShutdownException { - trySend(workerMetadataRequest); + protected void onFlushPending(boolean isNewStream) throws WindmillStreamShutdownException { + if (isNewStream) { + trySend(workerMetadataRequest); + } } @Override @@ -154,8 +170,8 @@ protected void sendHealthCheck() throws WindmillStreamShutdownException { protected void appendSpecificHtml(PrintWriter writer) { synchronized (metadataLock) { writer.format( - "GetWorkerMetadataStream: job_header=[%s], current_metadata=[%s]", - workerMetadataRequest.getHeader(), latestResponse); + "GetWorkerMetadataStream: job_header=[%s], current_metadata=[%s] received_at=[%s]", + workerMetadataRequest.getHeader(), latestResponse, latestResponseReceived); } } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillStreamFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillStreamFactory.java index 1f261e59450a..244d2ad3fa14 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillStreamFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillStreamFactory.java @@ -27,6 +27,9 @@ import java.util.Timer; import java.util.TimerTask; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; @@ -59,6 +62,7 @@ import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Suppliers; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.joda.time.Duration; import org.joda.time.Instant; @@ -69,8 +73,10 @@ @ThreadSafe @Internal public class GrpcWindmillStreamFactory implements StatusDataProvider { - private static final long DEFAULT_STREAM_RPC_DEADLINE_SECONDS = 300; + private static final java.time.Duration + DEFAULT_DIRECT_STREAMING_RPC_PHYSICAL_STREAM_HALF_CLOSE_AFTER = + java.time.Duration.ofMinutes(3); private static final Duration MIN_BACKOFF = Duration.millis(1); private static final Duration DEFAULT_MAX_BACKOFF = Duration.standardSeconds(30); private static final int DEFAULT_LOG_EVERY_N_STREAM_FAILURES = 1; @@ -92,6 +98,8 @@ public class GrpcWindmillStreamFactory implements StatusDataProvider { private final boolean sendKeyedGetDataRequests; private final boolean requestBatchedGetWorkResponse; private final Consumer> processHeartbeatResponses; + private final java.time.Duration directStreamingRpcPhysicalStreamHalfCloseAfter; + private final Supplier executorServiceSupplier; private GrpcWindmillStreamFactory( JobHeader jobHeader, @@ -101,7 +109,9 @@ private GrpcWindmillStreamFactory( boolean sendKeyedGetDataRequests, boolean requestBatchedGetWorkResponse, Consumer> processHeartbeatResponses, - Supplier maxBackOffSupplier) { + Supplier maxBackOffSupplier, + java.time.Duration directStreamingRpcPhysicalStreamHalfCloseAfter, + Supplier executorServiceSupplier) { this.jobHeader = jobHeader; this.logEveryNStreamFailures = logEveryNStreamFailures; this.streamingRpcBatchLimit = streamingRpcBatchLimit; @@ -119,6 +129,9 @@ private GrpcWindmillStreamFactory( this.requestBatchedGetWorkResponse = requestBatchedGetWorkResponse; this.processHeartbeatResponses = processHeartbeatResponses; this.streamIdGenerator = new AtomicLong(); + this.directStreamingRpcPhysicalStreamHalfCloseAfter = + directStreamingRpcPhysicalStreamHalfCloseAfter; + this.executorServiceSupplier = executorServiceSupplier; } /** @implNote Used for {@link AutoBuilder} {@link Builder} class, do not call directly. */ @@ -131,7 +144,9 @@ static GrpcWindmillStreamFactory create( boolean requestBatchedGetWorkResponse, Consumer> processHeartbeatResponses, Supplier maxBackOffSupplier, - int healthCheckIntervalMillis) { + int healthCheckIntervalMillis, + java.time.Duration directStreamingRpcPhysicalStreamHalfCloseAfter, + Supplier scheduledExecutorServiceSupplier) { GrpcWindmillStreamFactory streamFactory = new GrpcWindmillStreamFactory( jobHeader, @@ -141,7 +156,9 @@ static GrpcWindmillStreamFactory create( sendKeyedGetDataRequests, requestBatchedGetWorkResponse, processHeartbeatResponses, - maxBackOffSupplier); + maxBackOffSupplier, + directStreamingRpcPhysicalStreamHalfCloseAfter, + scheduledExecutorServiceSupplier); if (healthCheckIntervalMillis >= 0) { // Health checks are run on background daemon thread, which will only be cleaned up on JVM @@ -169,6 +186,7 @@ public void run() { * Returns a new {@link Builder} for {@link GrpcWindmillStreamFactory} with default values set for * the given {@link JobHeader}. */ + @SuppressWarnings("nullness") public static GrpcWindmillStreamFactory.Builder of(JobHeader jobHeader) { return new AutoBuilder_GrpcWindmillStreamFactory_Builder() .setJobHeader(jobHeader) @@ -179,7 +197,10 @@ public static GrpcWindmillStreamFactory.Builder of(JobHeader jobHeader) { .setHealthCheckIntervalMillis(NO_HEALTH_CHECKS) .setSendKeyedGetDataRequests(true) .setRequestBatchedGetWorkResponse(false) - .setProcessHeartbeatResponses(ignored -> {}); + .setProcessHeartbeatResponses(ignored -> {}) + .setDirectStreamingRpcPhysicalStreamHalfCloseAfter( + DEFAULT_DIRECT_STREAMING_RPC_PHYSICAL_STREAM_HALF_CLOSE_AFTER) + .setScheduledExecutorServiceSupplier(() -> null); } private static > T withDefaultDeadline(T stub) { @@ -201,6 +222,41 @@ private static void printSummaryHtmlForWorker( writer.write("
"); } + private ScheduledExecutorService executorForDispatchedStreams(String debugStreamTypeName) { + ScheduledExecutorService result = executorServiceSupplier.get(); + if (result != null) { + return result; + } + return Executors.newSingleThreadScheduledExecutor( + new ThreadFactoryBuilder() + .setDaemon(true) + .setNameFormat(String.format("%s-WindmillStream-thread", debugStreamTypeName)) + .build()); + } + + private ScheduledExecutorService executorForDirectStreams( + String backendWorkerToken, String debugStreamTypeName) { + ScheduledExecutorService supplierResult = executorServiceSupplier.get(); + if (supplierResult != null) { + return supplierResult; + } + ScheduledThreadPoolExecutor result = + new ScheduledThreadPoolExecutor( + 0, + new ThreadFactoryBuilder() + .setDaemon(true) + .setNameFormat( + String.join( + "-", + debugStreamTypeName, + backendWorkerToken.substring(0, Math.min(10, backendWorkerToken.length())), + "WindmillStream", + "%d")) + .build()); + result.setKeepAliveTime(1, TimeUnit.MINUTES); + return result; + } + public GetWorkStream createGetWorkStream( CloudWindmillServiceV1Alpha1Stub stub, GetWorkRequest request, @@ -214,7 +270,9 @@ public GetWorkStream createGetWorkStream( streamRegistry, logEveryNStreamFailures, requestBatchedGetWorkResponse, - processWorkItem); + processWorkItem, + java.time.Duration.ZERO, + executorForDispatchedStreams("GetWork")); } public GetWorkStream createDirectGetWorkStream( @@ -226,7 +284,8 @@ public GetWorkStream createDirectGetWorkStream( WorkItemScheduler workItemScheduler) { return GrpcDirectGetWorkStream.create( connection.backendWorkerToken(), - responseObserver -> connection.currentStub().getWorkStream(responseObserver), + responseObserver -> + withDefaultDeadline(connection.currentStub()).getWorkStream(responseObserver), request, grpcBackOff.get(), newStreamObserverFactory(), @@ -236,7 +295,9 @@ public GetWorkStream createDirectGetWorkStream( heartbeatSender, getDataClient, workCommitter, - workItemScheduler); + workItemScheduler, + directStreamingRpcPhysicalStreamHalfCloseAfter, + executorForDirectStreams(connection.backendWorkerToken(), "GetWork")); } public GetDataStream createGetDataStream(CloudWindmillServiceV1Alpha1Stub stub) { @@ -251,13 +312,16 @@ public GetDataStream createGetDataStream(CloudWindmillServiceV1Alpha1Stub stub) streamIdGenerator, streamingRpcBatchLimit, sendKeyedGetDataRequests, - processHeartbeatResponses); + processHeartbeatResponses, + java.time.Duration.ZERO, + executorForDispatchedStreams("GetWorkerMetadata")); } public GetDataStream createDirectGetDataStream(WindmillConnection connection) { return GrpcGetDataStream.create( connection.backendWorkerToken(), - responseObserver -> connection.currentStub().getDataStream(responseObserver), + responseObserver -> + withDefaultDeadline(connection.currentStub()).getDataStream(responseObserver), grpcBackOff.get(), newStreamObserverFactory(), streamRegistry, @@ -266,7 +330,9 @@ public GetDataStream createDirectGetDataStream(WindmillConnection connection) { streamIdGenerator, streamingRpcBatchLimit, sendKeyedGetDataRequests, - processHeartbeatResponses); + processHeartbeatResponses, + directStreamingRpcPhysicalStreamHalfCloseAfter, + executorForDirectStreams(connection.backendWorkerToken(), "GetData")); } public CommitWorkStream createCommitWorkStream(CloudWindmillServiceV1Alpha1Stub stub) { @@ -279,20 +345,25 @@ public CommitWorkStream createCommitWorkStream(CloudWindmillServiceV1Alpha1Stub logEveryNStreamFailures, jobHeader, streamIdGenerator, - streamingRpcBatchLimit); + streamingRpcBatchLimit, + java.time.Duration.ZERO, + executorForDispatchedStreams("CommitWork")); } public CommitWorkStream createDirectCommitWorkStream(WindmillConnection connection) { return GrpcCommitWorkStream.create( connection.backendWorkerToken(), - responseObserver -> connection.currentStub().commitWorkStream(responseObserver), + responseObserver -> + withDefaultDeadline(connection.currentStub()).commitWorkStream(responseObserver), grpcBackOff.get(), newStreamObserverFactory(), streamRegistry, logEveryNStreamFailures, jobHeader, streamIdGenerator, - streamingRpcBatchLimit); + streamingRpcBatchLimit, + directStreamingRpcPhysicalStreamHalfCloseAfter, + executorForDirectStreams(connection.backendWorkerToken(), "CommitWork")); } public GetWorkerMetadataStream createGetWorkerMetadataStream( @@ -305,7 +376,9 @@ public GetWorkerMetadataStream createGetWorkerMetadataStream( streamRegistry, logEveryNStreamFailures, jobHeader, - onNewWindmillEndpoints); + onNewWindmillEndpoints, + directStreamingRpcPhysicalStreamHalfCloseAfter, + executorForDispatchedStreams("GetWorkerMetadataStream")); } private StreamObserverFactory newStreamObserverFactory() { @@ -351,6 +424,11 @@ Builder setProcessHeartbeatResponses( Builder setRequestBatchedGetWorkResponse(boolean enabled); + Builder setDirectStreamingRpcPhysicalStreamHalfCloseAfter(java.time.Duration timeout); + + Builder setScheduledExecutorServiceSupplier( + Supplier scheduledExecutorServiceSupplier); + GrpcWindmillStreamFactory build(); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/ChannelCache.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/ChannelCache.java index 11018bdb2c46..6de661e52cf6 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/ChannelCache.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/ChannelCache.java @@ -38,7 +38,6 @@ import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.RemovalListeners; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.MoreExecutors; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; -import org.checkerframework.checker.nullness.qual.MonotonicNonNull; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -58,8 +57,8 @@ public final class ChannelCache implements StatusDataProvider { private final LoadingCache channelCache; @GuardedBy("this") - @MonotonicNonNull - private UserWorkerGrpcFlowControlSettings currentFlowControlSettings = null; + private UserWorkerGrpcFlowControlSettings currentFlowControlSettings = + UserWorkerGrpcFlowControlSettings.getDefaultInstance(); private ChannelCache( WindmillChannelFactory channelFactory, @@ -78,7 +77,8 @@ public ManagedChannel load(WindmillServiceAddress key) { private UserWorkerGrpcFlowControlSettings resolveFlowControlSettings( WindmillServiceAddress.Kind addressType) { synchronized (ChannelCache.this) { - if (currentFlowControlSettings == null) { + if (currentFlowControlSettings.equals( + UserWorkerGrpcFlowControlSettings.getDefaultInstance())) { return addressType == AUTHENTICATED_GCP_SERVICE_ADDRESS ? WindmillChannels.DEFAULT_DIRECTPATH_FLOW_CONTROL_SETTINGS : WindmillChannels.DEFAULT_CLOUDPATH_FLOW_CONTROL_SETTINGS; @@ -132,9 +132,7 @@ public ManagedChannel get(WindmillServiceAddress windmillServiceAddress) { public synchronized void consumeFlowControlSettings( UserWorkerGrpcFlowControlSettings flowControlSettings) { - //noinspection PointlessNullCheck - if (currentFlowControlSettings == null - || !flowControlSettings.equals(currentFlowControlSettings)) { + if (!flowControlSettings.equals(currentFlowControlSettings)) { // Refreshing the cache will asynchronously terminate the old channels via the removalListener // and return a newly created one on the next Cache.load(address). This could be expensive so // only do it when we have received new flow control settings. diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/ToIterableFunction.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/ToIterableFunction.java index 3db058c79a03..7e164df2245b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/ToIterableFunction.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/ToIterableFunction.java @@ -64,13 +64,13 @@ public Iterable apply( valuesAndContPosition.getContinuationPosition()) .toBuilder(); if (stateTag.getSortedListRange() != null) { - continuationTBuilder.setSortedListRange(stateTag.getSortedListRange()).build(); + continuationTBuilder.setSortedListRange(stateTag.getSortedListRange()); } if (stateTag.getMultimapKey() != null) { - continuationTBuilder.setMultimapKey(stateTag.getMultimapKey()).build(); + continuationTBuilder.setMultimapKey(stateTag.getMultimapKey()); } if (stateTag.getOmitValues() != null) { - continuationTBuilder.setOmitValues(stateTag.getOmitValues()).build(); + continuationTBuilder.setOmitValues(stateTag.getOmitValues()); } return new PagingIterable<>( reader, valuesAndContPosition.getValues(), continuationTBuilder.build(), coder); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/StreamingWorkScheduler.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/StreamingWorkScheduler.java index e58a2759cd83..a4cd5d6d8a6b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/StreamingWorkScheduler.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/StreamingWorkScheduler.java @@ -425,7 +425,7 @@ private ExecuteWorkResult executeWork( // If processing failed due to a thrown exception, close the executionState. Do not // return/release the executionState back to computationState as that will lead to this // executionState instance being reused. - LOG.info("Invalidating executor after work item {} failed with Exception:", key, t); + LOG.debug("Invalidating executor after work item {} failed", workItem.getWorkToken(), t); computationWorkExecutor.invalidate(); // Re-throw the exception, it will be caught and handled by workFailureProcessor downstream. diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java index 59fd341fab4b..dd13d5b55930 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java @@ -253,7 +253,7 @@ public boolean awaitTermination(int time, TimeUnit unit) throws InterruptedExcep Windmill.GetWorkResponse response = workToOffer.get(null); if (response == null) { try { - sleepMillis(500); + sleepMillis(100); } catch (InterruptedException e) { halfClose(); Thread.currentThread().interrupt(); @@ -515,9 +515,9 @@ public void clearCommitsReceived() { public ConcurrentHashMap> waitForDroppedCommits( int droppedCommits) { LOG.debug("waitForDroppedCommits: {}", droppedCommits); - int maxTries = 10; + int maxTries = 100; while (maxTries-- > 0 && droppedStreamingCommits.size() < droppedCommits) { - Uninterruptibles.sleepUninterruptibly(1000, TimeUnit.MILLISECONDS); + Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS); } assertEquals(droppedCommits, droppedStreamingCommits.size()); return droppedStreamingCommits; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderTest.java index cc006d5b1651..8a962773821c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderTest.java @@ -215,7 +215,7 @@ private List writeShuffleEntries( return records; } - @SuppressWarnings("ReturnValueIgnored") + @SuppressWarnings({"ReturnValueIgnored"}) private List>>> runIterationOverGroupingShuffleReader( BatchModeExecutionContext context, TestShuffleReader shuffleReader, diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmSideInputReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmSideInputReaderTest.java index 8420977dc47d..06e089807299 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmSideInputReaderTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmSideInputReaderTest.java @@ -1727,14 +1727,14 @@ private static RandomAccessData encodeKeyPortion(IsmRecordCoder coder, IsmRec } /** Write input elements to a new temporary file and return the corresponding IsmSource. */ - private Source initInputFile( + private Source initInputFile( Iterable>> elements, IsmRecordCoder> coder) throws Exception { return initInputFile(elements, coder, tmpFolder.newFile().getPath()); } /** Write input elements to the given file and return the corresponding IsmSource. */ - private Source initInputFile( + private Source initInputFile( Iterable>> elements, IsmRecordCoder> coder, String tmpFilePath) @@ -1769,7 +1769,7 @@ private Source initInputFile( } /** Returns a new Source for the given ISM file using the specified coder. */ - private Source newIsmSource(IsmRecordCoder> coder, String tmpFilePath) { + private Source newIsmSource(IsmRecordCoder> coder, String tmpFilePath) { Source source = new Source(); source.setCodec( CloudObjects.asCloudObject( diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java index c1696d8a70ab..a60535dfbd69 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java @@ -1285,7 +1285,7 @@ public void testKeyCommitTooLargeException() throws Exception { int maxTries = 10; while (--maxTries > 0) { worker.reportPeriodicWorkerUpdatesForTest(); - Uninterruptibles.sleepUninterruptibly(1000, TimeUnit.MILLISECONDS); + Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS); } // We should see an exception reported for the large commit but not the small one. @@ -1489,9 +1489,9 @@ public void testExceptions() throws Exception { server.waitForEmptyWorkQueue(); // Wait until the worker has given up. - int maxTries = 10; + int maxTries = 100; while (maxTries-- > 0 && !worker.workExecutorIsEmpty()) { - Uninterruptibles.sleepUninterruptibly(1000, TimeUnit.MILLISECONDS); + Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS); } assertTrue(worker.workExecutorIsEmpty()); @@ -1499,7 +1499,7 @@ public void testExceptions() throws Exception { maxTries = 10; while (maxTries-- > 0) { worker.reportPeriodicWorkerUpdatesForTest(); - Uninterruptibles.sleepUninterruptibly(1000, TimeUnit.MILLISECONDS); + Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS); } // We should see our update only one time with the exceptions we are expecting. @@ -3520,7 +3520,7 @@ public void testActiveWorkFailure() throws Exception { // Release the blocked calls. BlockingFn.blocker().countDown(); Map commits = - server.waitForAndGetCommitsWithTimeout(2, Duration.standardSeconds((5))); + server.waitForAndGetCommitsWithTimeout(1, Duration.standardSeconds((5))); assertEquals(1, commits.size()); worker.stop(); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java index 9b92d7d48431..d18bc512723e 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java @@ -384,7 +384,7 @@ public void testMultipleSideInputs() throws Exception { assertThat(sideInputFetcher.elementBag(createWindow(0)).read(), Matchers.emptyIterable()); } - private StreamingSideInputDoFnRunner createRunner( + private StreamingSideInputDoFnRunner createRunner( WindowedValueMultiReceiver outputManager, List> views, StreamingSideInputFetcher sideInputFetcher) @@ -392,7 +392,7 @@ private StreamingSideInputDoFnRunner return createRunner(WINDOW_FN, outputManager, views, sideInputFetcher); } - private StreamingSideInputDoFnRunner createRunner( + private StreamingSideInputDoFnRunner createRunner( WindowFn windowFn, WindowedValueMultiReceiver outputManager, List> views, @@ -415,7 +415,7 @@ private StreamingSideInputDoFnRunner return new StreamingSideInputDoFnRunner<>(simpleDoFnRunner, sideInputFetcher); } - private StreamingSideInputFetcher createFetcher( + private StreamingSideInputFetcher createFetcher( List> views) throws Exception { @SuppressWarnings({"unchecked", "rawtypes"}) Iterable> typedViews = (Iterable) views; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcherTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcherTest.java index 0f5cd1a0d233..cf616ee6ac0d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcherTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcherTest.java @@ -181,7 +181,7 @@ public void testStoreIfBlocked() throws Exception { assertThat(restTimers, Matchers.contains(timer2)); } - private StreamingSideInputFetcher createFetcher( + private StreamingSideInputFetcher createFetcher( List> views) throws Exception { @SuppressWarnings({"unchecked", "rawtypes"}) Iterable> typedViews = (Iterable) views; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingHandlerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingHandlerTest.java index c69b031bf74b..3191228687c3 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingHandlerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingHandlerTest.java @@ -108,6 +108,17 @@ private static String createJson(LogRecord record, Formatter formatter) throws I return new String(output.toByteArray(), StandardCharsets.UTF_8); } + private static String createJsonWithCustomMdc(LogRecord record) throws IOException { + ByteArrayOutputStream output = new ByteArrayOutputStream(); + FixedOutputStreamFactory factory = new FixedOutputStreamFactory(output); + DataflowWorkerLoggingHandler handler = new DataflowWorkerLoggingHandler(factory, 0); + handler.setLogMdc(true); + // Format the record as JSON. + handler.publish(record); + // Decode the binary output as UTF-8 and return the generated string. + return new String(output.toByteArray(), StandardCharsets.UTF_8); + } + /** * Encodes a {@link org.apache.beam.model.fnexecution.v1.BeamFnApi.LogEntry} into a Json string. */ @@ -233,14 +244,14 @@ public synchronized String formatMessage(LogRecord record) { return MDC.get("testMdcKey") + ":" + super.formatMessage(record); } }; - MDC.put("testMdcKey", "testMdcValue"); - - assertEquals( - "{\"timestamp\":{\"seconds\":0,\"nanos\":1000000},\"severity\":\"INFO\"," - + "\"message\":\"testMdcValue:test.message\",\"thread\":\"2\",\"job\":\"testJobId\"," - + "\"worker\":\"testWorkerId\",\"work\":\"testWorkId\",\"logger\":\"LoggerName\"}" - + System.lineSeparator(), - createJson(createLogRecord("test.message", null /* throwable */), customFormatter)); + try (MDC.MDCCloseable ignored = MDC.putCloseable("testMdcKey", "testMdcValue")) { + assertEquals( + "{\"timestamp\":{\"seconds\":0,\"nanos\":1000000},\"severity\":\"INFO\"," + + "\"message\":\"testMdcValue:test.message\",\"thread\":\"2\",\"job\":\"testJobId\"," + + "\"worker\":\"testWorkerId\",\"work\":\"testWorkId\",\"logger\":\"LoggerName\"}" + + System.lineSeparator(), + createJson(createLogRecord("test.message", null /* throwable */), customFormatter)); + } } @Test @@ -299,6 +310,40 @@ public void testWithException() throws IOException { createJson(createLogRecord(null /* message */, createThrowable()))); } + @Test + public void testWithCustomDataEnabledNoMdc() throws IOException { + assertEquals( + "{\"timestamp\":{\"seconds\":0,\"nanos\":1000000},\"severity\":\"INFO\"," + + "\"message\":\"test.message\",\"thread\":\"2\",\"logger\":\"LoggerName\"}" + + System.lineSeparator(), + createJsonWithCustomMdc(createLogRecord("test.message", null))); + } + + @Test + public void testWithCustomDataDisabledWithMdc() throws IOException { + MDC.clear(); + try (MDC.MDCCloseable closeable = MDC.putCloseable("key1", "cool value")) { + assertEquals( + "{\"timestamp\":{\"seconds\":0,\"nanos\":1000000},\"severity\":\"INFO\"," + + "\"message\":\"test.message\",\"thread\":\"2\",\"logger\":\"LoggerName\"}" + + System.lineSeparator(), + createJson(createLogRecord("test.message", null))); + } + } + + @Test + public void testWithCustomDataEnabledWithMdc() throws IOException { + try (MDC.MDCCloseable ignored = MDC.putCloseable("key1", "cool value"); + MDC.MDCCloseable ignored2 = MDC.putCloseable("key2", "another")) { + assertEquals( + "{\"timestamp\":{\"seconds\":0,\"nanos\":1000000},\"severity\":\"INFO\"," + + "\"message\":\"test.message\",\"thread\":\"2\",\"logger\":\"LoggerName\"," + + "\"custom_data\":{\"key1\":\"cool value\",\"key2\":\"another\"}}" + + System.lineSeparator(), + createJsonWithCustomMdc(createLogRecord("test.message", null))); + } + } + @Test public void testWithoutExceptionOrMessage() throws IOException { DataflowWorkerLoggingMDC.setJobId("testJobId"); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/GroupAlsoByWindowProperties.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/GroupAlsoByWindowProperties.java index bca4efa518f2..06206de92e49 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/GroupAlsoByWindowProperties.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/GroupAlsoByWindowProperties.java @@ -588,11 +588,10 @@ public StateInternals load(K key) throws Exception { private static final PipelineOptions OPTIONS = PipelineOptionsFactory.create(); - private static - List>> processElement( - BatchGroupAlsoByWindowFn fn, - KV>> element) - throws Exception { + private static List>> processElement( + BatchGroupAlsoByWindowFn fn, + KV>> element) + throws Exception { TestOutput output = new TestOutput<>(); fn.processElement( element, OPTIONS, null /* timerInternals */, NullSideInputReader.empty(), output); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStreamTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStreamTest.java index 92c081591c73..80c39e770c3e 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStreamTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/AbstractWindmillStreamTest.java @@ -21,12 +21,14 @@ import static org.junit.Assert.assertThrows; import java.io.PrintWriter; +import java.time.temporal.ChronoUnit; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; @@ -59,7 +61,12 @@ public void setUp() { private TestStream newStream( Function, StreamObserver> clientFactory) { - return new TestStream(clientFactory, streamRegistry, streamObserverFactory); + return new TestStream( + clientFactory, + streamRegistry, + streamObserverFactory, + Duration.ZERO, + Executors.newScheduledThreadPool(0)); } @Test @@ -140,21 +147,25 @@ private static class TestStream extends AbstractWindmillStream private static final Logger LOG = LoggerFactory.getLogger(AbstractWindmillStreamTest.class); private final AtomicInteger numStarts = new AtomicInteger(); + private final AtomicInteger numFlushPending = new AtomicInteger(); private final AtomicInteger numHealthChecks = new AtomicInteger(); private TestStream( Function, StreamObserver> clientFactory, Set> streamRegistry, - StreamObserverFactory streamObserverFactory) { + StreamObserverFactory streamObserverFactory, + Duration halfCloseAfterTimeout, + ScheduledExecutorService executorService) { super( LoggerFactory.getLogger(AbstractWindmillStreamTest.class), - "Test", clientFactory, FluentBackoff.DEFAULT.backoff(), streamObserverFactory, streamRegistry, 1, - "Test"); + "Test", + java.time.Duration.of(halfCloseAfterTimeout.getMillis(), ChronoUnit.MILLIS), + executorService); } @Override @@ -178,8 +189,11 @@ public void appendHtml(PrintWriter writer) {} } @Override - protected void onNewStream() { - numStarts.incrementAndGet(); + protected void onFlushPending(boolean isNewStream) { + if (isNewStream) { + numStarts.incrementAndGet(); + } + numFlushPending.incrementAndGet(); } private void testSend() throws WindmillStreamShutdownException { diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/TriggeredScheduledExecutorService.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/TriggeredScheduledExecutorService.java new file mode 100644 index 000000000000..a43da93b3680 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/TriggeredScheduledExecutorService.java @@ -0,0 +1,140 @@ +/* + * 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. + */ +package org.apache.beam.runners.dataflow.worker.windmill.client; + +import java.time.Duration; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Delayed; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import javax.annotation.Nullable; + +public class TriggeredScheduledExecutorService extends ThreadPoolExecutor + implements ScheduledExecutorService { + private final BlockingQueue futures = new LinkedBlockingQueue<>(); + + public TriggeredScheduledExecutorService() { + super(0, 100, 30, TimeUnit.SECONDS, new LinkedBlockingQueue<>()); + } + + public boolean unblockNextFuture() throws InterruptedException { + @Nullable FakeScheduledFuture f = futures.take(); + if (f == null) { + return false; + } + f.triggerRun(); + return true; + } + + @Override + public ScheduledFuture schedule(Runnable runnable, long l, TimeUnit timeUnit) { + FakeScheduledFuture f = + new FakeScheduledFuture(runnable, Duration.ofMillis(timeUnit.toMillis(l))); + try { + futures.put(f); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + return f; + } + + @Override + public ScheduledFuture schedule(Callable callable, long l, TimeUnit timeUnit) { + throw new UnsupportedOperationException("not supported yet"); + } + + @Override + public ScheduledFuture scheduleAtFixedRate( + Runnable runnable, long l, long l1, TimeUnit timeUnit) { + throw new UnsupportedOperationException("not supported yet"); + } + + @Override + public ScheduledFuture scheduleWithFixedDelay( + Runnable runnable, long l, long l1, TimeUnit timeUnit) { + throw new UnsupportedOperationException("not supported yet"); + } + + private class FakeScheduledFuture implements ScheduledFuture { + private final Runnable r; + private final Duration delay; + private transient boolean cancelled; + private final CompletableFuture delegateFuture = new CompletableFuture<>(); + + private FakeScheduledFuture(Runnable r, Duration delay) { + this.r = r; + this.delay = delay; + } + + void triggerRun() { + TriggeredScheduledExecutorService.this.execute( + () -> { + try { + r.run(); + delegateFuture.complete(null); + } catch (RuntimeException e) { + delegateFuture.completeExceptionally(e); + } + }); + } + + @Override + public long getDelay(TimeUnit timeUnit) { + return timeUnit.convert(delay.toMillis(), TimeUnit.MILLISECONDS); + } + + @Override + public int compareTo(Delayed delayed) { + return 0; + } + + @Override + public boolean cancel(boolean b) { + cancelled = true; + return true; + } + + @Override + public boolean isCancelled() { + return cancelled; + } + + @Override + public boolean isDone() { + return delegateFuture.isDone(); + } + + @Override + public Void get() throws InterruptedException, ExecutionException { + return delegateFuture.get(); + } + + @Override + public Void get(long l, TimeUnit timeUnit) + throws InterruptedException, ExecutionException, TimeoutException { + return delegateFuture.get(l, timeUnit); + } + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/FakeWindmillGrpcService.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/FakeWindmillGrpcService.java index 85c3c71663f1..19f8c1578b46 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/FakeWindmillGrpcService.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/FakeWindmillGrpcService.java @@ -19,6 +19,7 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; import java.util.concurrent.LinkedBlockingQueue; import javax.annotation.concurrent.GuardedBy; import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc; @@ -32,12 +33,31 @@ class FakeWindmillGrpcService private final ErrorCollector errorCollector; @GuardedBy("this") - private boolean failOnNewStreams = false; + private boolean noMoreStreamsExpected = false; + + @GuardedBy("this") + private int failedStreamConnectsRemaining = 0; public FakeWindmillGrpcService(ErrorCollector errorCollector) { this.errorCollector = errorCollector; } + @SuppressWarnings("BusyWait") + public void waitForFailedConnectAttempts() throws InterruptedException { + while (true) { + Thread.sleep(2); + synchronized (this) { + if (failedStreamConnectsRemaining <= 0) { + break; + } + } + } + } + + public synchronized void setFailedStreamConnectsRemaining(int failedStreamConnectsRemaining) { + this.failedStreamConnectsRemaining = failedStreamConnectsRemaining; + } + public static class StreamInfo { public StreamInfo(StreamObserver responseObserver) { this.responseObserver = responseObserver; @@ -63,6 +83,17 @@ public StreamInfoObserver( @Override public void onNext(RequestT request) { + if (streamInfo.onDone.isDone()) { + try { + if (streamInfo.onDone.get() == null) { + throw new IllegalStateException("Stream already half-closed."); + } else { + throw new IllegalStateException("Stream already closed with error."); + } + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException(e); + } + } errorCollector.checkThat(streamInfo.requests.add(request), Matchers.is(true)); } @@ -89,7 +120,11 @@ public StreamObserver commitWorkStream( StreamObserver responseObserver) { CommitStreamInfo info = new CommitStreamInfo(responseObserver); synchronized (this) { - errorCollector.checkThat(failOnNewStreams, Matchers.is(false)); + errorCollector.checkThat(noMoreStreamsExpected, Matchers.is(false)); + if (failedStreamConnectsRemaining-- > 0) { + throw new RuntimeException( + "Injected connection error, remaining failures: " + failedStreamConnectsRemaining); + } errorCollector.checkThat(commitStreams.offer(info), Matchers.is(true)); } return new StreamInfoObserver<>(info, errorCollector); @@ -100,7 +135,7 @@ public CommitStreamInfo waitForConnectedCommitStream() throws InterruptedExcepti } public synchronized void expectNoMoreStreams() { - failOnNewStreams = true; + noMoreStreamsExpected = true; errorCollector.checkThat(commitStreams.isEmpty(), Matchers.is(true)); errorCollector.checkThat(getDataStreams.isEmpty(), Matchers.is(true)); } @@ -117,7 +152,11 @@ public StreamObserver getDataStream( StreamObserver responseObserver) { GetDataStreamInfo info = new GetDataStreamInfo(responseObserver); synchronized (this) { - errorCollector.checkThat(failOnNewStreams, Matchers.is(false)); + errorCollector.checkThat(noMoreStreamsExpected, Matchers.is(false)); + if (failedStreamConnectsRemaining-- > 0) { + throw new RuntimeException( + "Injected connection error, remaining failures: " + failedStreamConnectsRemaining); + } errorCollector.checkThat(getDataStreams.offer(info), Matchers.is(true)); } return new StreamInfoObserver<>(info, errorCollector); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcCommitWorkStreamTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcCommitWorkStreamTest.java index 195e13e84e26..e9fd55fa5668 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcCommitWorkStreamTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcCommitWorkStreamTest.java @@ -21,21 +21,28 @@ import static org.hamcrest.Matchers.*; import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import java.io.IOException; +import java.time.Duration; import java.util.HashMap; import java.util.HashSet; import java.util.Map; import java.util.Set; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Supplier; import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillConnection; +import org.apache.beam.runners.dataflow.worker.windmill.client.TriggeredScheduledExecutorService; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream; import org.apache.beam.runners.dataflow.worker.windmill.client.grpc.observers.StreamObserverCancelledException; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; @@ -120,6 +127,32 @@ private GrpcCommitWorkStream createCommitWorkStream() { return commitWorkStream; } + private GrpcCommitWorkStream createCommitWorkStreamWithPhysicalStreamHandover( + ScheduledExecutorService executor) { + GrpcCommitWorkStream commitWorkStream = + (GrpcCommitWorkStream) + GrpcWindmillStreamFactory.of(TEST_JOB_HEADER) + .setDirectStreamingRpcPhysicalStreamHalfCloseAfter(Duration.ofMinutes(1)) + .setScheduledExecutorServiceSupplier( + new Supplier() { + private final AtomicBoolean vended = new AtomicBoolean(); + + @Override + public ScheduledExecutorService get() { + assertFalse(vended.getAndSet(true)); + return executor; + } + }) + .build() + .createDirectCommitWorkStream( + WindmillConnection.builder() + .setStubSupplier( + () -> CloudWindmillServiceV1Alpha1Grpc.newStub(inProcessChannel)) + .build()); + commitWorkStream.start(); + return commitWorkStream; + } + @Test public void testShutdown_abortsActiveCommits() throws InterruptedException, ExecutionException { int numCommits = 5; @@ -459,6 +492,647 @@ public void testSend_notCalledAfterShutdown_Multichunk() assertThat(streamInfo.requests).isEmpty(); } + private Windmill.WorkItemCommitRequest createTestCommit(int id) { + return Windmill.WorkItemCommitRequest.newBuilder() + .setKey(ByteString.EMPTY) + .setShardingKey(id) + .setWorkToken(id * 100L) + .setCacheToken(id * 1000L) + .build(); + } + + @Test + public void testCommitWorkItem_multiplePhysicalStreams() throws Exception { + // A special executor that allows triggering scheduled futures (of which the handover is the + // only such future). + TriggeredScheduledExecutorService triggeredExecutor = new TriggeredScheduledExecutorService(); + GrpcCommitWorkStream commitWorkStream = + createCommitWorkStreamWithPhysicalStreamHandover(triggeredExecutor); + FakeWindmillGrpcService.CommitStreamInfo streamInfo = waitForConnectionAndConsumeHeader(); + + // Send a request where the response is captured in a future. + Windmill.WorkItemCommitRequest workItemCommitRequest = createTestCommit(1); + CompletableFuture commitStatusFuture = new CompletableFuture<>(); + try (WindmillStream.CommitWorkStream.RequestBatcher batcher = commitWorkStream.batcher()) { + assertTrue( + batcher.commitWorkItem( + COMPUTATION_ID, workItemCommitRequest, commitStatusFuture::complete)); + } + + Windmill.StreamingCommitWorkRequest request = streamInfo.requests.take(); + assertThat(request.getCommitChunkList()).hasSize(1); + Windmill.WorkItemCommitRequest parsedRequest = + Windmill.WorkItemCommitRequest.parseFrom( + request.getCommitChunk(0).getSerializedWorkItemCommit()); + assertThat(parsedRequest).isEqualTo(workItemCommitRequest); + + // Trigger a new stream to be created by forcing the scheduled halfCloseFuture scheduled within + // AbstractWindmillStream to run. + assertTrue(triggeredExecutor.unblockNextFuture()); + FakeWindmillGrpcService.CommitStreamInfo streamInfo2 = waitForConnectionAndConsumeHeader(); + fakeService.expectNoMoreStreams(); + + // Previous stream client should be half-closed. + assertNull(streamInfo.onDone.get()); + + Windmill.WorkItemCommitRequest workItemCommitRequest2 = createTestCommit(2); + CompletableFuture commitStatusFuture2 = new CompletableFuture<>(); + try (WindmillStream.CommitWorkStream.RequestBatcher batcher = commitWorkStream.batcher()) { + assertTrue( + batcher.commitWorkItem( + COMPUTATION_ID, workItemCommitRequest2, commitStatusFuture2::complete)); + } + Windmill.StreamingCommitWorkRequest request2 = streamInfo2.requests.take(); + assertThat(request2.getCommitChunkList()).hasSize(1); + Windmill.WorkItemCommitRequest parsedRequest2 = + Windmill.WorkItemCommitRequest.parseFrom( + request2.getCommitChunk(0).getSerializedWorkItemCommit()); + assertThat(parsedRequest2).isEqualTo(workItemCommitRequest2); + + streamInfo2.responseObserver.onNext( + Windmill.StreamingCommitResponse.newBuilder().addRequestId(2).build()); + + streamInfo.responseObserver.onNext( + Windmill.StreamingCommitResponse.newBuilder().addRequestId(1).build()); + assertThat(commitStatusFuture.get()).isEqualTo(Windmill.CommitStatus.OK); + assertThat(commitStatusFuture2.get()).isEqualTo(Windmill.CommitStatus.OK); + + // Complete server-side half-close of first stream. No new + // stream should be created since the current stream is active. + streamInfo.responseObserver.onCompleted(); + + // Close the stream, the open stream should be client half-closed + // but logical remains not terminated. + commitWorkStream.halfClose(); + assertNull(streamInfo2.onDone.get()); + assertFalse(commitWorkStream.awaitTermination(10, TimeUnit.MILLISECONDS)); + + // Complete half-closing from the server and verify shutdown completes. + streamInfo2.responseObserver.onCompleted(); + + assertTrue(commitWorkStream.awaitTermination(10, TimeUnit.SECONDS)); + } + + @Test + public void testCommitWorkItem_multiplePhysicalStreams_oldStreamFails() throws Exception { + TriggeredScheduledExecutorService triggeredExecutor = new TriggeredScheduledExecutorService(); + GrpcCommitWorkStream commitWorkStream = + createCommitWorkStreamWithPhysicalStreamHandover(triggeredExecutor); + commitWorkStream.start(); + FakeWindmillGrpcService.CommitStreamInfo streamInfo = waitForConnectionAndConsumeHeader(); + + Windmill.WorkItemCommitRequest workItemCommitRequest = createTestCommit(1); + CompletableFuture commitStatusFuture = new CompletableFuture<>(); + try (WindmillStream.CommitWorkStream.RequestBatcher batcher = commitWorkStream.batcher()) { + assertTrue( + batcher.commitWorkItem( + COMPUTATION_ID, workItemCommitRequest, commitStatusFuture::complete)); + } + + Windmill.StreamingCommitWorkRequest request = streamInfo.requests.take(); + assertThat(request.getCommitChunkList()).hasSize(1); + Windmill.WorkItemCommitRequest parsedRequest = + Windmill.WorkItemCommitRequest.parseFrom( + request.getCommitChunk(0).getSerializedWorkItemCommit()); + assertThat(parsedRequest).isEqualTo(workItemCommitRequest); + + // A new stream should be created due to handover. + assertTrue(triggeredExecutor.unblockNextFuture()); + FakeWindmillGrpcService.CommitStreamInfo streamInfo2 = waitForConnectionAndConsumeHeader(); + fakeService.expectNoMoreStreams(); + + // Previous stream client should be half-closed. + assertNull(streamInfo.onDone.get()); + + Windmill.WorkItemCommitRequest workItemCommitRequest2 = createTestCommit(2); + CompletableFuture commitStatusFuture2 = new CompletableFuture<>(); + try (WindmillStream.CommitWorkStream.RequestBatcher batcher = commitWorkStream.batcher()) { + assertTrue( + batcher.commitWorkItem( + COMPUTATION_ID, workItemCommitRequest2, commitStatusFuture2::complete)); + } + Windmill.StreamingCommitWorkRequest request2 = streamInfo2.requests.take(); + assertThat(request2.getCommitChunkList()).hasSize(1); + Windmill.WorkItemCommitRequest parsedRequest2 = + Windmill.WorkItemCommitRequest.parseFrom( + request2.getCommitChunk(0).getSerializedWorkItemCommit()); + assertThat(parsedRequest2).isEqualTo(workItemCommitRequest2); + + streamInfo2.responseObserver.onNext( + Windmill.StreamingCommitResponse.newBuilder().addRequestId(2).build()); + assertThat(commitStatusFuture2.get()).isEqualTo(Windmill.CommitStatus.OK); + + // Complete first stream with an error. No new + // stream should be created since the current stream is active. The request should have an + // error and the request should be retried on the new stream. + streamInfo.responseObserver.onError(new RuntimeException("test error")); + Windmill.StreamingCommitWorkRequest request3 = streamInfo2.requests.take(); + assertThat(request3.getCommitChunkList()).hasSize(1); + Windmill.WorkItemCommitRequest parsedRequest3 = + Windmill.WorkItemCommitRequest.parseFrom( + request3.getCommitChunk(0).getSerializedWorkItemCommit()); + assertThat(parsedRequest3).isEqualTo(workItemCommitRequest); + + // Close the stream, the open stream should be client half-closed + // but logical remains not terminated. + commitWorkStream.halfClose(); + assertNull(streamInfo2.onDone.get()); + assertFalse(commitWorkStream.awaitTermination(10, TimeUnit.MILLISECONDS)); + + streamInfo2.responseObserver.onNext( + Windmill.StreamingCommitResponse.newBuilder().addRequestId(1).build()); + assertThat(commitStatusFuture.get()).isEqualTo(Windmill.CommitStatus.OK); + + // Complete half-closing from the server and verify shutdown completes. + streamInfo2.responseObserver.onCompleted(); + + assertTrue(commitWorkStream.awaitTermination(10, TimeUnit.SECONDS)); + } + + @Test + public void testCommitWorkItem_multiplePhysicalStreams_newStreamFailsWhileEmpty() + throws Exception { + TriggeredScheduledExecutorService triggeredExecutor = new TriggeredScheduledExecutorService(); + GrpcCommitWorkStream commitWorkStream = + createCommitWorkStreamWithPhysicalStreamHandover(triggeredExecutor); + commitWorkStream.start(); + FakeWindmillGrpcService.CommitStreamInfo streamInfo = waitForConnectionAndConsumeHeader(); + + Windmill.WorkItemCommitRequest workItemCommitRequest = createTestCommit(1); + CompletableFuture commitStatusFuture = new CompletableFuture<>(); + try (WindmillStream.CommitWorkStream.RequestBatcher batcher = commitWorkStream.batcher()) { + assertTrue( + batcher.commitWorkItem( + COMPUTATION_ID, workItemCommitRequest, commitStatusFuture::complete)); + } + + Windmill.StreamingCommitWorkRequest request = streamInfo.requests.take(); + assertThat(request.getCommitChunkList()).hasSize(1); + Windmill.WorkItemCommitRequest parsedRequest = + Windmill.WorkItemCommitRequest.parseFrom( + request.getCommitChunk(0).getSerializedWorkItemCommit()); + assertThat(parsedRequest).isEqualTo(workItemCommitRequest); + + // A new stream should be created due to handover. + assertTrue(triggeredExecutor.unblockNextFuture()); + + FakeWindmillGrpcService.CommitStreamInfo streamInfo2 = waitForConnectionAndConsumeHeader(); + assertNull(streamInfo.onDone.get()); + + // Before stream 1 is finished simulate stream 2 failing. + streamInfo2.responseObserver.onError(new IOException("stream 2 failed")); + // A new stream should be created and handle new requests. + FakeWindmillGrpcService.CommitStreamInfo streamInfo3 = waitForConnectionAndConsumeHeader(); + assertNull(streamInfo.onDone.get()); + + Windmill.WorkItemCommitRequest workItemCommitRequest2 = createTestCommit(2); + CompletableFuture commitStatusFuture2 = new CompletableFuture<>(); + try (WindmillStream.CommitWorkStream.RequestBatcher batcher = commitWorkStream.batcher()) { + assertTrue( + batcher.commitWorkItem( + COMPUTATION_ID, workItemCommitRequest2, commitStatusFuture2::complete)); + } + Windmill.StreamingCommitWorkRequest request2 = streamInfo3.requests.take(); + assertThat(request2.getCommitChunkList()).hasSize(1); + Windmill.WorkItemCommitRequest parsedRequest2 = + Windmill.WorkItemCommitRequest.parseFrom( + request2.getCommitChunk(0).getSerializedWorkItemCommit()); + assertThat(parsedRequest2).isEqualTo(workItemCommitRequest2); + + streamInfo3.responseObserver.onNext( + Windmill.StreamingCommitResponse.newBuilder().addRequestId(2).build()); + + streamInfo.responseObserver.onNext( + Windmill.StreamingCommitResponse.newBuilder().addRequestId(1).build()); + assertThat(commitStatusFuture.get()).isEqualTo(Windmill.CommitStatus.OK); + assertThat(commitStatusFuture2.get()).isEqualTo(Windmill.CommitStatus.OK); + + // Close the stream. + commitWorkStream.halfClose(); + assertNull(streamInfo.onDone.get()); + fakeService.expectNoMoreStreams(); + streamInfo.responseObserver.onCompleted(); + streamInfo3.responseObserver.onCompleted(); + + assertTrue(commitWorkStream.awaitTermination(10, TimeUnit.SECONDS)); + } + + @Test + public void testCommitWorkItem_multiplePhysicalStreams_newStreamFailsWithRequests() + throws Exception { + TriggeredScheduledExecutorService triggeredExecutor = new TriggeredScheduledExecutorService(); + GrpcCommitWorkStream commitWorkStream = + createCommitWorkStreamWithPhysicalStreamHandover(triggeredExecutor); + commitWorkStream.start(); + FakeWindmillGrpcService.CommitStreamInfo streamInfo = waitForConnectionAndConsumeHeader(); + + Windmill.WorkItemCommitRequest workItemCommitRequest = createTestCommit(1); + CompletableFuture commitStatusFuture = new CompletableFuture<>(); + try (WindmillStream.CommitWorkStream.RequestBatcher batcher = commitWorkStream.batcher()) { + assertTrue( + batcher.commitWorkItem( + COMPUTATION_ID, workItemCommitRequest, commitStatusFuture::complete)); + } + + Windmill.StreamingCommitWorkRequest request = streamInfo.requests.take(); + assertThat(request.getCommitChunkList()).hasSize(1); + Windmill.WorkItemCommitRequest parsedRequest = + Windmill.WorkItemCommitRequest.parseFrom( + request.getCommitChunk(0).getSerializedWorkItemCommit()); + assertThat(parsedRequest).isEqualTo(workItemCommitRequest); + + // A new stream should be created due to handover. + assertTrue(triggeredExecutor.unblockNextFuture()); + + FakeWindmillGrpcService.CommitStreamInfo streamInfo2 = waitForConnectionAndConsumeHeader(); + assertNull(streamInfo.onDone.get()); + + Windmill.WorkItemCommitRequest workItemCommitRequest2 = createTestCommit(2); + CompletableFuture commitStatusFuture2 = new CompletableFuture<>(); + try (WindmillStream.CommitWorkStream.RequestBatcher batcher = commitWorkStream.batcher()) { + assertTrue( + batcher.commitWorkItem( + COMPUTATION_ID, workItemCommitRequest2, commitStatusFuture2::complete)); + } + Windmill.StreamingCommitWorkRequest request2 = streamInfo2.requests.take(); + assertThat(request2.getCommitChunkList()).hasSize(1); + Windmill.WorkItemCommitRequest parsedRequest2 = + Windmill.WorkItemCommitRequest.parseFrom( + request2.getCommitChunk(0).getSerializedWorkItemCommit()); + assertThat(parsedRequest2).isEqualTo(workItemCommitRequest2); + + // Before stream 1 is finished simulate stream 2 failing. + streamInfo2.responseObserver.onError(new IOException("stream 2 failed")); + // A new stream should be created and receive the pending requests from stream2 but not the + // request from stream1. + FakeWindmillGrpcService.CommitStreamInfo streamInfo3 = waitForConnectionAndConsumeHeader(); + assertNull(streamInfo.onDone.get()); + Windmill.StreamingCommitWorkRequest request3 = streamInfo3.requests.take(); + assertThat(request3.getCommitChunkList()).hasSize(1); + Windmill.WorkItemCommitRequest parsedRequest3 = + Windmill.WorkItemCommitRequest.parseFrom( + request3.getCommitChunk(0).getSerializedWorkItemCommit()); + assertThat(parsedRequest3).isEqualTo(workItemCommitRequest2); + + streamInfo3.responseObserver.onNext( + Windmill.StreamingCommitResponse.newBuilder().addRequestId(2).build()); + + streamInfo.responseObserver.onNext( + Windmill.StreamingCommitResponse.newBuilder().addRequestId(1).build()); + assertThat(commitStatusFuture.get()).isEqualTo(Windmill.CommitStatus.OK); + assertThat(commitStatusFuture2.get()).isEqualTo(Windmill.CommitStatus.OK); + + // Close the stream. + commitWorkStream.halfClose(); + assertNull(streamInfo.onDone.get()); + fakeService.expectNoMoreStreams(); + streamInfo.responseObserver.onCompleted(); + streamInfo3.responseObserver.onCompleted(); + + assertTrue(commitWorkStream.awaitTermination(10, TimeUnit.SECONDS)); + } + + @Test + public void testCommitWorkItem_multiplePhysicalStreams_multipleHandovers() throws Exception { + TriggeredScheduledExecutorService triggeredExecutor = new TriggeredScheduledExecutorService(); + GrpcCommitWorkStream commitWorkStream = + createCommitWorkStreamWithPhysicalStreamHandover(triggeredExecutor); + commitWorkStream.start(); + FakeWindmillGrpcService.CommitStreamInfo streamInfo1 = waitForConnectionAndConsumeHeader(); + + // Commit request 1 on stream 1 + Windmill.WorkItemCommitRequest workItemCommitRequest1 = createTestCommit(1); + CompletableFuture commitStatusFuture1 = new CompletableFuture<>(); + try (WindmillStream.CommitWorkStream.RequestBatcher batcher = commitWorkStream.batcher()) { + assertTrue( + batcher.commitWorkItem( + COMPUTATION_ID, workItemCommitRequest1, commitStatusFuture1::complete)); + } + + Windmill.StreamingCommitWorkRequest request1 = streamInfo1.requests.take(); + assertThat(request1.getCommitChunkList()).hasSize(1); + Windmill.WorkItemCommitRequest parsedRequest1 = + Windmill.WorkItemCommitRequest.parseFrom( + request1.getCommitChunk(0).getSerializedWorkItemCommit()); + assertThat(parsedRequest1).isEqualTo(workItemCommitRequest1); + + // Trigger handover 1 + assertTrue(triggeredExecutor.unblockNextFuture()); + FakeWindmillGrpcService.CommitStreamInfo streamInfo2 = waitForConnectionAndConsumeHeader(); + assertNull(streamInfo1.onDone.get()); + + // Commit request 2 on stream 2 + Windmill.WorkItemCommitRequest workItemCommitRequest2 = createTestCommit(2); + CompletableFuture commitStatusFuture2 = new CompletableFuture<>(); + try (WindmillStream.CommitWorkStream.RequestBatcher batcher = commitWorkStream.batcher()) { + assertTrue( + batcher.commitWorkItem( + COMPUTATION_ID, workItemCommitRequest2, commitStatusFuture2::complete)); + } + + Windmill.StreamingCommitWorkRequest request2 = streamInfo2.requests.take(); + assertThat(request2.getCommitChunkList()).hasSize(1); + Windmill.WorkItemCommitRequest parsedRequest2 = + Windmill.WorkItemCommitRequest.parseFrom( + request2.getCommitChunk(0).getSerializedWorkItemCommit()); + assertThat(parsedRequest2).isEqualTo(workItemCommitRequest2); + + // Trigger handover 2 before streamInfo2 completes + assertTrue(triggeredExecutor.unblockNextFuture()); + FakeWindmillGrpcService.CommitStreamInfo streamInfo3 = waitForConnectionAndConsumeHeader(); + assertNull(streamInfo2.onDone.get()); + + // Commit request 3 on stream 3 + Windmill.WorkItemCommitRequest workItemCommitRequest3 = createTestCommit(3); + CompletableFuture commitStatusFuture3 = new CompletableFuture<>(); + try (WindmillStream.CommitWorkStream.RequestBatcher batcher = commitWorkStream.batcher()) { + assertTrue( + batcher.commitWorkItem( + COMPUTATION_ID, workItemCommitRequest3, commitStatusFuture3::complete)); + } + + Windmill.StreamingCommitWorkRequest request3 = streamInfo3.requests.take(); + assertThat(request3.getCommitChunkList()).hasSize(1); + Windmill.WorkItemCommitRequest parsedRequest3 = + Windmill.WorkItemCommitRequest.parseFrom( + request3.getCommitChunk(0).getSerializedWorkItemCommit()); + assertThat(parsedRequest3).isEqualTo(workItemCommitRequest3); + + // Respond to all requests + streamInfo1.responseObserver.onNext( + Windmill.StreamingCommitResponse.newBuilder().addRequestId(1).build()); + streamInfo2.responseObserver.onNext( + Windmill.StreamingCommitResponse.newBuilder().addRequestId(2).build()); + streamInfo3.responseObserver.onNext( + Windmill.StreamingCommitResponse.newBuilder().addRequestId(3).build()); + + assertThat(commitStatusFuture1.get()).isEqualTo(Windmill.CommitStatus.OK); + assertThat(commitStatusFuture2.get()).isEqualTo(Windmill.CommitStatus.OK); + assertThat(commitStatusFuture3.get()).isEqualTo(Windmill.CommitStatus.OK); + + // Close the stream + commitWorkStream.halfClose(); + assertNull(streamInfo3.onDone.get()); + + // Verify no more streams + fakeService.expectNoMoreStreams(); + streamInfo1.responseObserver.onCompleted(); + streamInfo2.responseObserver.onCompleted(); + streamInfo3.responseObserver.onCompleted(); + + assertTrue(commitWorkStream.awaitTermination(10, TimeUnit.SECONDS)); + } + + @Test + public void testCommitWorkItem_multiplePhysicalStreams_oldStreamFailsWhileNewStreamInBackoff() + throws Exception { + TriggeredScheduledExecutorService triggeredExecutor = new TriggeredScheduledExecutorService(); + GrpcCommitWorkStream commitWorkStream = + createCommitWorkStreamWithPhysicalStreamHandover(triggeredExecutor); + commitWorkStream.start(); + FakeWindmillGrpcService.CommitStreamInfo streamInfo1 = waitForConnectionAndConsumeHeader(); + + // Commit request 1 on stream 1 + Windmill.WorkItemCommitRequest workItemCommitRequest1 = createTestCommit(1); + CompletableFuture commitStatusFuture1 = new CompletableFuture<>(); + try (WindmillStream.CommitWorkStream.RequestBatcher batcher = commitWorkStream.batcher()) { + assertTrue( + batcher.commitWorkItem( + COMPUTATION_ID, workItemCommitRequest1, commitStatusFuture1::complete)); + } + + Windmill.StreamingCommitWorkRequest request1 = streamInfo1.requests.take(); + assertThat(request1.getCommitChunkList()).hasSize(1); + Windmill.WorkItemCommitRequest parsedRequest1 = + Windmill.WorkItemCommitRequest.parseFrom( + request1.getCommitChunk(0).getSerializedWorkItemCommit()); + assertThat(parsedRequest1).isEqualTo(workItemCommitRequest1); + + // Trigger handover but fail new connections + assertTrue(triggeredExecutor.unblockNextFuture()); + fakeService.setFailedStreamConnectsRemaining(1); + fakeService.waitForFailedConnectAttempts(); + assertNull(streamInfo1.onDone.get()); + + // Fail first stream + streamInfo1.responseObserver.onError(new RuntimeException("test error")); + + FakeWindmillGrpcService.CommitStreamInfo streamInfo2 = waitForConnectionAndConsumeHeader(); + fakeService.expectNoMoreStreams(); + + Windmill.StreamingCommitWorkRequest request2 = streamInfo2.requests.take(); + assertThat(request2.getCommitChunkList()).hasSize(1); + Windmill.WorkItemCommitRequest parsedRequest2 = + Windmill.WorkItemCommitRequest.parseFrom( + request2.getCommitChunk(0).getSerializedWorkItemCommit()); + assertThat(parsedRequest2).isEqualTo(workItemCommitRequest1); + + // Respond to the request + streamInfo2.responseObserver.onNext( + Windmill.StreamingCommitResponse.newBuilder().addRequestId(1).build()); + assertThat(commitStatusFuture1.get()).isEqualTo(Windmill.CommitStatus.OK); + + // Close the stream + commitWorkStream.halfClose(); + assertNull(streamInfo2.onDone.get()); + + streamInfo2.responseObserver.onCompleted(); + + assertTrue(commitWorkStream.awaitTermination(10, TimeUnit.SECONDS)); + } + + @Test + public void testCommitWorkItem_multiplePhysicalStreams_multipleHandovers_shutdown() + throws Exception { + TriggeredScheduledExecutorService triggeredExecutor = new TriggeredScheduledExecutorService(); + GrpcCommitWorkStream commitWorkStream = + createCommitWorkStreamWithPhysicalStreamHandover(triggeredExecutor); + commitWorkStream.start(); + FakeWindmillGrpcService.CommitStreamInfo streamInfo1 = waitForConnectionAndConsumeHeader(); + + // Commit request 1 on stream 1 + Windmill.WorkItemCommitRequest workItemCommitRequest1 = createTestCommit(1); + CompletableFuture commitStatusFuture1 = new CompletableFuture<>(); + try (WindmillStream.CommitWorkStream.RequestBatcher batcher = commitWorkStream.batcher()) { + assertTrue( + batcher.commitWorkItem( + COMPUTATION_ID, workItemCommitRequest1, commitStatusFuture1::complete)); + } + + Windmill.StreamingCommitWorkRequest request1 = streamInfo1.requests.take(); + assertThat(request1.getCommitChunkList()).hasSize(1); + Windmill.WorkItemCommitRequest parsedRequest1 = + Windmill.WorkItemCommitRequest.parseFrom( + request1.getCommitChunk(0).getSerializedWorkItemCommit()); + assertThat(parsedRequest1).isEqualTo(workItemCommitRequest1); + + // Trigger handover 1 + assertTrue(triggeredExecutor.unblockNextFuture()); + FakeWindmillGrpcService.CommitStreamInfo streamInfo2 = waitForConnectionAndConsumeHeader(); + assertNull(streamInfo1.onDone.get()); + + // Commit request 2 on stream 2 + Windmill.WorkItemCommitRequest workItemCommitRequest2 = createTestCommit(2); + CompletableFuture commitStatusFuture2 = new CompletableFuture<>(); + try (WindmillStream.CommitWorkStream.RequestBatcher batcher = commitWorkStream.batcher()) { + assertTrue( + batcher.commitWorkItem( + COMPUTATION_ID, workItemCommitRequest2, commitStatusFuture2::complete)); + } + + Windmill.StreamingCommitWorkRequest request2 = streamInfo2.requests.take(); + assertThat(request2.getCommitChunkList()).hasSize(1); + Windmill.WorkItemCommitRequest parsedRequest2 = + Windmill.WorkItemCommitRequest.parseFrom( + request2.getCommitChunk(0).getSerializedWorkItemCommit()); + assertThat(parsedRequest2).isEqualTo(workItemCommitRequest2); + + // Trigger handover 2 before streamInfo2 completes + assertTrue(triggeredExecutor.unblockNextFuture()); + FakeWindmillGrpcService.CommitStreamInfo streamInfo3 = waitForConnectionAndConsumeHeader(); + assertNull(streamInfo2.onDone.get()); + + // Commit request 3 on stream 3 + Windmill.WorkItemCommitRequest workItemCommitRequest3 = createTestCommit(3); + CompletableFuture commitStatusFuture3 = new CompletableFuture<>(); + try (WindmillStream.CommitWorkStream.RequestBatcher batcher = commitWorkStream.batcher()) { + assertTrue( + batcher.commitWorkItem( + COMPUTATION_ID, workItemCommitRequest3, commitStatusFuture3::complete)); + } + + Windmill.StreamingCommitWorkRequest request3 = streamInfo3.requests.take(); + assertThat(request3.getCommitChunkList()).hasSize(1); + Windmill.WorkItemCommitRequest parsedRequest3 = + Windmill.WorkItemCommitRequest.parseFrom( + request3.getCommitChunk(0).getSerializedWorkItemCommit()); + assertThat(parsedRequest3).isEqualTo(workItemCommitRequest3); + + // Shutdown while there are active streams and verify it isn't completed until all the streams + // are done. + fakeService.expectNoMoreStreams(); + assertFalse(commitWorkStream.awaitTermination(0, TimeUnit.SECONDS)); + commitWorkStream.shutdown(); + assertThat(commitStatusFuture1.isDone()).isTrue(); + assertThat(commitStatusFuture2.isDone()).isTrue(); + assertThat(commitStatusFuture3.isDone()).isTrue(); + assertFalse(commitWorkStream.awaitTermination(10, TimeUnit.MILLISECONDS)); + + assertFalse(commitWorkStream.awaitTermination(0, TimeUnit.MILLISECONDS)); + streamInfo3.responseObserver.onCompleted(); + assertFalse(commitWorkStream.awaitTermination(0, TimeUnit.MILLISECONDS)); + streamInfo1.responseObserver.onCompleted(); + assertFalse(commitWorkStream.awaitTermination(0, TimeUnit.MILLISECONDS)); + streamInfo2.responseObserver.onError(new RuntimeException("test")); + assertTrue(commitWorkStream.awaitTermination(10, TimeUnit.SECONDS)); + } + + @Test + public void testCommitWorkItem_multiplePhysicalStreams_multipleHandovers_halfClose() + throws Exception { + TriggeredScheduledExecutorService triggeredExecutor = new TriggeredScheduledExecutorService(); + GrpcCommitWorkStream commitWorkStream = + createCommitWorkStreamWithPhysicalStreamHandover(triggeredExecutor); + commitWorkStream.start(); + FakeWindmillGrpcService.CommitStreamInfo streamInfo1 = waitForConnectionAndConsumeHeader(); + + // Commit request 1 on stream 1 + Windmill.WorkItemCommitRequest workItemCommitRequest1 = createTestCommit(1); + CompletableFuture commitStatusFuture1 = new CompletableFuture<>(); + try (WindmillStream.CommitWorkStream.RequestBatcher batcher = commitWorkStream.batcher()) { + assertTrue( + batcher.commitWorkItem( + COMPUTATION_ID, workItemCommitRequest1, commitStatusFuture1::complete)); + } + + Windmill.StreamingCommitWorkRequest request1 = streamInfo1.requests.take(); + assertThat(request1.getCommitChunkList()).hasSize(1); + Windmill.WorkItemCommitRequest parsedRequest1 = + Windmill.WorkItemCommitRequest.parseFrom( + request1.getCommitChunk(0).getSerializedWorkItemCommit()); + assertThat(parsedRequest1).isEqualTo(workItemCommitRequest1); + + // Trigger handover 1 + assertTrue(triggeredExecutor.unblockNextFuture()); + FakeWindmillGrpcService.CommitStreamInfo streamInfo2 = waitForConnectionAndConsumeHeader(); + assertNull(streamInfo1.onDone.get()); + + // Commit request 2 on stream 2 + Windmill.WorkItemCommitRequest workItemCommitRequest2 = createTestCommit(2); + CompletableFuture commitStatusFuture2 = new CompletableFuture<>(); + try (WindmillStream.CommitWorkStream.RequestBatcher batcher = commitWorkStream.batcher()) { + assertTrue( + batcher.commitWorkItem( + COMPUTATION_ID, workItemCommitRequest2, commitStatusFuture2::complete)); + } + + Windmill.StreamingCommitWorkRequest request2 = streamInfo2.requests.take(); + assertThat(request2.getCommitChunkList()).hasSize(1); + Windmill.WorkItemCommitRequest parsedRequest2 = + Windmill.WorkItemCommitRequest.parseFrom( + request2.getCommitChunk(0).getSerializedWorkItemCommit()); + assertThat(parsedRequest2).isEqualTo(workItemCommitRequest2); + + // Trigger handover 2 before streamInfo2 completes + assertTrue(triggeredExecutor.unblockNextFuture()); + FakeWindmillGrpcService.CommitStreamInfo streamInfo3 = waitForConnectionAndConsumeHeader(); + assertNull(streamInfo2.onDone.get()); + + // Commit request 3 on stream 3 + Windmill.WorkItemCommitRequest workItemCommitRequest3 = createTestCommit(3); + CompletableFuture commitStatusFuture3 = new CompletableFuture<>(); + try (WindmillStream.CommitWorkStream.RequestBatcher batcher = commitWorkStream.batcher()) { + assertTrue( + batcher.commitWorkItem( + COMPUTATION_ID, workItemCommitRequest3, commitStatusFuture3::complete)); + } + + Windmill.StreamingCommitWorkRequest request3 = streamInfo3.requests.take(); + assertThat(request3.getCommitChunkList()).hasSize(1); + Windmill.WorkItemCommitRequest parsedRequest3 = + Windmill.WorkItemCommitRequest.parseFrom( + request3.getCommitChunk(0).getSerializedWorkItemCommit()); + assertThat(parsedRequest3).isEqualTo(workItemCommitRequest3); + + // Shutdown while there are active streams and verify it isn't completed until all the streams + // are done. + fakeService.expectNoMoreStreams(); + assertFalse(commitWorkStream.awaitTermination(0, TimeUnit.SECONDS)); + commitWorkStream.halfClose(); + + assertFalse(commitWorkStream.awaitTermination(10, TimeUnit.MILLISECONDS)); + assertThat(streamInfo3.onDone.get()).isNull(); + + assertThat(commitStatusFuture1.isDone()).isFalse(); + assertThat(commitStatusFuture2.isDone()).isFalse(); + assertThat(commitStatusFuture3.isDone()).isFalse(); + + streamInfo3.responseObserver.onNext( + Windmill.StreamingCommitResponse.newBuilder().addRequestId(3).build()); + streamInfo3.responseObserver.onCompleted(); + assertThat(commitStatusFuture3.get()).isEqualTo(Windmill.CommitStatus.OK); + assertFalse(commitWorkStream.awaitTermination(0, TimeUnit.MILLISECONDS)); + + streamInfo1.responseObserver.onNext( + Windmill.StreamingCommitResponse.newBuilder() + .addRequestId(1) + .addStatus(Windmill.CommitStatus.ABORTED) + .build()); + streamInfo1.responseObserver.onCompleted(); + assertThat(commitStatusFuture1.get()).isEqualTo(Windmill.CommitStatus.ABORTED); + assertFalse(commitWorkStream.awaitTermination(0, TimeUnit.MILLISECONDS)); + + streamInfo2.responseObserver.onNext( + Windmill.StreamingCommitResponse.newBuilder() + .addRequestId(2) + .addStatus(Windmill.CommitStatus.ALREADY_IN_COMMIT) + .build()); + streamInfo2.responseObserver.onCompleted(); + assertThat(commitStatusFuture2.get()).isEqualTo(Windmill.CommitStatus.ALREADY_IN_COMMIT); + + assertTrue(commitWorkStream.awaitTermination(10, TimeUnit.SECONDS)); + } + private FakeWindmillGrpcService.CommitStreamInfo waitForConnectionAndConsumeHeader() { try { FakeWindmillGrpcService.CommitStreamInfo info = fakeService.waitForConnectedCommitStream(); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStreamTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStreamTest.java index 1014242317de..419000178381 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStreamTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStreamTest.java @@ -392,7 +392,9 @@ public void testConsumedWorkItems() throws InterruptedException { @Test public void testConsumedWorkItems_itemsSplitAcrossResponses() throws InterruptedException { - int expectedRequests = 3; + // We send all the responses on the first request. We don't care if there are additional + // requests. + int expectedRequests = 1; CountDownLatch waitForRequests = new CountDownLatch(expectedRequests); TestGetWorkRequestObserver requestObserver = new TestGetWorkRequestObserver(waitForRequests); GetWorkStreamTestStub testStub = new GetWorkStreamTestStub(requestObserver); @@ -426,9 +428,9 @@ public void testConsumedWorkItems_itemsSplitAcrossResponses() throws Interrupted Windmill.WorkItem workItem3 = Windmill.WorkItem.newBuilder() .setKey(ByteString.copyFromUtf8("somewhat_long_key3")) - .setWorkToken(2L) - .setShardingKey(2L) - .setCacheToken(2L) + .setWorkToken(3L) + .setShardingKey(3L) + .setCacheToken(3L) .build(); List chunks1 = new ArrayList<>(); @@ -444,12 +446,12 @@ public void testConsumedWorkItems_itemsSplitAcrossResponses() throws Interrupted chunks3.add(workItem3.toByteString()); + assertTrue(waitForRequests.await(5, TimeUnit.SECONDS)); + testStub.injectResponse(createResponse(chunks1, bytes.size() - third)); testStub.injectResponse(createResponse(chunks2, bytes.size() - 2 * third)); testStub.injectResponse(createResponse(chunks3, 0)); - assertTrue(waitForRequests.await(5, TimeUnit.SECONDS)); - assertThat(scheduledWorkItems).containsExactly(workItem1, workItem2, workItem3); } @@ -458,6 +460,7 @@ private static class GetWorkStreamTestStub private final TestGetWorkRequestObserver requestObserver; private @Nullable StreamObserver responseObserver; + private final CountDownLatch waitForStream = new CountDownLatch(1); private GetWorkStreamTestStub(TestGetWorkRequestObserver requestObserver) { this.requestObserver = requestObserver; @@ -466,15 +469,17 @@ private GetWorkStreamTestStub(TestGetWorkRequestObserver requestObserver) { @Override public StreamObserver getWorkStream( StreamObserver responseObserver) { - if (this.responseObserver == null) { - this.responseObserver = responseObserver; - requestObserver.responseObserver = this.responseObserver; - } + assertThat(this.responseObserver).isNull(); + this.responseObserver = responseObserver; + requestObserver.responseObserver = this.responseObserver; + waitForStream.countDown(); return requestObserver; } - private void injectResponse(Windmill.StreamingGetWorkResponseChunk responseChunk) { + private void injectResponse(Windmill.StreamingGetWorkResponseChunk responseChunk) + throws InterruptedException { + waitForStream.await(); checkNotNull(responseObserver).onNext(responseChunk); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStreamTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStreamTest.java index e954f2cc7105..4f584022c8a5 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStreamTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcGetDataStreamTest.java @@ -19,26 +19,42 @@ import static com.google.common.truth.Truth.assertThat; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import java.io.IOException; +import java.time.Duration; import java.util.List; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Supplier; +import java.util.logging.Level; +import java.util.logging.Logger; import java.util.stream.Collectors; import java.util.stream.IntStream; +import javax.annotation.Nullable; import org.apache.beam.runners.dataflow.worker.windmill.CloudWindmillServiceV1Alpha1Grpc; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.runners.dataflow.worker.windmill.WindmillConnection; +import org.apache.beam.runners.dataflow.worker.windmill.client.TriggeredScheduledExecutorService; import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStreamShutdownException; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.CallOptions; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.Channel; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.ClientCall; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.ClientInterceptor; import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.ManagedChannel; +import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.MethodDescriptor; import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.Server; import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.inprocess.InProcessChannelBuilder; import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.inprocess.InProcessServerBuilder; @@ -86,6 +102,8 @@ public void setUp() throws IOException { inProcessChannel = grpcCleanup.register( InProcessChannelBuilder.forName(FAKE_SERVER_NAME).directExecutor().build()); + Logger.getLogger(GrpcGetDataStream.class.getName()).setLevel(Level.ALL); + Logger.getLogger(AbstractMethodError.class.getName()).setLevel(Level.ALL); } @After @@ -105,20 +123,39 @@ private GrpcGetDataStream createGetDataStream() { return getDataStream; } + private GrpcGetDataStream createGetDataStreamWithPhysicalStreamHandover( + Duration handover, @Nullable ScheduledExecutorService executor) { + GrpcGetDataStream getDataStream = + (GrpcGetDataStream) + GrpcWindmillStreamFactory.of(TEST_JOB_HEADER) + .setDirectStreamingRpcPhysicalStreamHalfCloseAfter(handover) + .setScheduledExecutorServiceSupplier( + new Supplier() { + private final AtomicBoolean vended = new AtomicBoolean(); + + @Override + public ScheduledExecutorService get() { + assertFalse(vended.getAndSet(true)); + return executor; + } + }) + .build() + .createDirectGetDataStream( + WindmillConnection.builder() + .setStubSupplier( + () -> CloudWindmillServiceV1Alpha1Grpc.newStub(inProcessChannel)) + .build()); + getDataStream.start(); + return getDataStream; + } + @Test public void testRequestKeyedData() throws InterruptedException { GrpcGetDataStream getDataStream = createGetDataStream(); FakeWindmillGrpcService.GetDataStreamInfo streamInfo = waitForConnectionAndConsumeHeader(); // These will block until they are successfully sent. - Windmill.KeyedGetDataRequest keyedGetDataRequest = - Windmill.KeyedGetDataRequest.newBuilder() - .setKey(ByteString.EMPTY) - .setShardingKey(1) - .setCacheToken(1) - .setWorkToken(1) - .build(); - + Windmill.KeyedGetDataRequest keyedGetDataRequest = createTestRequest(1); CompletableFuture sendFuture = CompletableFuture.supplyAsync( () -> { @@ -133,12 +170,7 @@ public void testRequestKeyedData() throws InterruptedException { assertThat(request.getRequestIdList()).containsExactly(1L); assertEquals(keyedGetDataRequest, request.getStateRequest(0).getRequests(0)); - Windmill.KeyedGetDataResponse keyedGetDataResponse = - Windmill.KeyedGetDataResponse.newBuilder() - .setShardingKey(1) - .setKey(ByteString.EMPTY) - .build(); - + Windmill.KeyedGetDataResponse keyedGetDataResponse = createTestResponse(1); streamInfo.responseObserver.onNext( Windmill.StreamingGetDataResponse.newBuilder() .addRequestId(1) @@ -171,14 +203,7 @@ public void testRequestKeyedData_sendOnShutdownStreamThrowsWindmillStreamShutdow } } try { - getDataStream.requestKeyedData( - "computationId", - Windmill.KeyedGetDataRequest.newBuilder() - .setKey(ByteString.EMPTY) - .setShardingKey(i) - .setCacheToken(i) - .setWorkToken(i) - .build()); + getDataStream.requestKeyedData("computationId", createTestRequest(i)); } catch (WindmillStreamShutdownException e) { throw new RuntimeException(e); } @@ -290,14 +315,766 @@ public void testRequestKeyedData_reconnectOnStreamErrorAfterHalfClose() getDataStream.halfClose(); assertNull(streamInfo.onDone.get()); - // Simulate an error on the grpc stream, this should trigger an error on all - // existing requests but no new connection since we half-closed and nothing left after - // responding with errors. - fakeService.expectNoMoreStreams(); + // Simulate an error on the grpc stream, this should trigger retrying the requests on a new + // stream + // which is half-closed. streamInfo.responseObserver.onError(new IOException("test error")); - assertThrows(RuntimeException.class, sendFuture::join); + FakeWindmillGrpcService.GetDataStreamInfo streamInfo2 = waitForConnectionAndConsumeHeader(); + Windmill.StreamingGetDataRequest request2 = streamInfo2.requests.take(); + assertThat(request2.getRequestIdList()).containsExactly(1L); + assertEquals(keyedGetDataRequest, request2.getStateRequest(0).getRequests(0)); + assertNull(streamInfo2.onDone.get()); + Windmill.KeyedGetDataResponse keyedGetDataResponse = createTestResponse(1); + streamInfo2.responseObserver.onNext( + Windmill.StreamingGetDataResponse.newBuilder() + .addRequestId(1) + .addSerializedResponse(keyedGetDataResponse.toByteString()) + .build()); + assertThat(sendFuture.join()).isEqualTo(keyedGetDataResponse); + assertFalse(getDataStream.awaitTermination(0, TimeUnit.MILLISECONDS)); + + // Sending an error this time shouldn't result in a new stream since there were no requests. + fakeService.expectNoMoreStreams(); + streamInfo2.responseObserver.onError(new IOException("test error")); + + getDataStream.awaitTermination(60, TimeUnit.MINUTES); + } + + private Windmill.KeyedGetDataRequest createTestRequest(long id) { + return Windmill.KeyedGetDataRequest.newBuilder() + .setKey(ByteString.EMPTY) + .setShardingKey(id) + .setCacheToken(id * 100) + .setWorkToken(id * 1000) + .build(); + } + + private Windmill.KeyedGetDataResponse createTestResponse(long id) { + return Windmill.KeyedGetDataResponse.newBuilder() + .setShardingKey(id) + .setKey(ByteString.EMPTY) + .build(); + } + + @Test + public void testRequestKeyedData_multiplePhysicalStreams() + throws InterruptedException, ExecutionException { + TriggeredScheduledExecutorService triggeredExecutor = new TriggeredScheduledExecutorService(); + GrpcGetDataStream getDataStream = + createGetDataStreamWithPhysicalStreamHandover(Duration.ofSeconds(60), triggeredExecutor); + FakeWindmillGrpcService.GetDataStreamInfo streamInfo = waitForConnectionAndConsumeHeader(); + + // These will block until they are successfully sent. + Windmill.KeyedGetDataRequest keyedGetDataRequest = createTestRequest(1); + + CompletableFuture sendFuture = + CompletableFuture.supplyAsync( + () -> { + try { + return getDataStream.requestKeyedData("computationId", keyedGetDataRequest); + } catch (WindmillStreamShutdownException e) { + throw new RuntimeException(e); + } + }); + + Windmill.StreamingGetDataRequest request = streamInfo.requests.take(); + assertThat(request.getRequestIdList()).containsExactly(1L); + assertEquals(keyedGetDataRequest, request.getStateRequest(0).getRequests(0)); + + // A new stream should be created due to handover. + assertTrue(triggeredExecutor.unblockNextFuture()); + FakeWindmillGrpcService.GetDataStreamInfo streamInfo2 = waitForConnectionAndConsumeHeader(); + fakeService.expectNoMoreStreams(); + + // Previous stream client should be half-closed. + assertNull(streamInfo.onDone.get()); + + Windmill.KeyedGetDataRequest keyedGetDataRequest2 = createTestRequest(2); + CompletableFuture sendFuture2 = + CompletableFuture.supplyAsync( + () -> { + try { + return getDataStream.requestKeyedData("computationId", keyedGetDataRequest2); + } catch (WindmillStreamShutdownException e) { + throw new RuntimeException(e); + } + }); + Windmill.StreamingGetDataRequest request2 = streamInfo2.requests.take(); + assertThat(request2.getRequestIdList()).containsExactly(2L); + assertEquals(keyedGetDataRequest2, request2.getStateRequest(0).getRequests(0)); + + Windmill.KeyedGetDataResponse keyedGetDataResponse2 = createTestResponse(2); + streamInfo2.responseObserver.onNext( + Windmill.StreamingGetDataResponse.newBuilder() + .addRequestId(2) + .addSerializedResponse(keyedGetDataResponse2.toByteString()) + .build()); + + Windmill.KeyedGetDataResponse keyedGetDataResponse = createTestResponse(1); + streamInfo.responseObserver.onNext( + Windmill.StreamingGetDataResponse.newBuilder() + .addRequestId(1) + .addSerializedResponse(keyedGetDataResponse.toByteString()) + .build()); + assertThat(sendFuture.join()).isEqualTo(keyedGetDataResponse); + assertThat(sendFuture2.join()).isEqualTo(keyedGetDataResponse2); + + // Complete server-side half-close of first stream. No new + // stream should be created since the current stream is active. + streamInfo.responseObserver.onCompleted(); + + // Close the stream, the open stream should be client half-closed + // but logical remains not terminated. + getDataStream.halfClose(); + assertNull(streamInfo2.onDone.get()); + assertFalse(getDataStream.awaitTermination(10, TimeUnit.MILLISECONDS)); + + // Complete half-closing from the server and verify shutdown completes. + streamInfo2.responseObserver.onCompleted(); + + assertTrue(getDataStream.awaitTermination(10, TimeUnit.SECONDS)); + } + + @Test + public void testRequestKeyedData_multiplePhysicalStreams_oldStreamFails() + throws InterruptedException, ExecutionException { + TriggeredScheduledExecutorService triggeredExecutor = new TriggeredScheduledExecutorService(); + GrpcGetDataStream getDataStream = + createGetDataStreamWithPhysicalStreamHandover(Duration.ofSeconds(60), triggeredExecutor); + FakeWindmillGrpcService.GetDataStreamInfo streamInfo = waitForConnectionAndConsumeHeader(); + + // These will block until they are successfully sent. + Windmill.KeyedGetDataRequest keyedGetDataRequest = createTestRequest(1); + CompletableFuture sendFuture = + CompletableFuture.supplyAsync( + () -> { + try { + return getDataStream.requestKeyedData("computationId", keyedGetDataRequest); + } catch (WindmillStreamShutdownException e) { + throw new RuntimeException(e); + } + }); + + Windmill.StreamingGetDataRequest request = streamInfo.requests.take(); + assertThat(request.getRequestIdList()).containsExactly(1L); + assertEquals(keyedGetDataRequest, request.getStateRequest(0).getRequests(0)); + + // A new stream should be created due to handover. + assertTrue(triggeredExecutor.unblockNextFuture()); + FakeWindmillGrpcService.GetDataStreamInfo streamInfo2 = waitForConnectionAndConsumeHeader(); + fakeService.expectNoMoreStreams(); + + // Previous stream client should be half-closed. + assertNull(streamInfo.onDone.get()); + + Windmill.KeyedGetDataRequest keyedGetDataRequest2 = createTestRequest(2); + CompletableFuture sendFuture2 = + CompletableFuture.supplyAsync( + () -> { + try { + return getDataStream.requestKeyedData("computationId", keyedGetDataRequest2); + } catch (WindmillStreamShutdownException e) { + throw new RuntimeException(e); + } + }); + Windmill.StreamingGetDataRequest request2 = streamInfo2.requests.take(); + assertThat(request2.getRequestIdList()).containsExactly(2L); + assertEquals(keyedGetDataRequest2, request2.getStateRequest(0).getRequests(0)); + + Windmill.KeyedGetDataResponse keyedGetDataResponse2 = createTestResponse(2); + streamInfo2.responseObserver.onNext( + Windmill.StreamingGetDataResponse.newBuilder() + .addRequestId(2) + .addSerializedResponse(keyedGetDataResponse2.toByteString()) + .build()); + assertThat(sendFuture2.join()).isEqualTo(keyedGetDataResponse2); + + // Complete first stream with an error. No new + // stream should be created since the current stream is active. The request should have an + // error and the request should be retried on the new stream. + streamInfo.responseObserver.onError(new RuntimeException("test error")); + Windmill.StreamingGetDataRequest request3 = streamInfo2.requests.take(); + assertThat(request3.getRequestIdList()).containsExactly(1L); + assertEquals(keyedGetDataRequest, request3.getStateRequest(0).getRequests(0)); + + // Close the stream, the open stream should be client half-closed + // but logical remains not terminated. + getDataStream.halfClose(); + assertNull(streamInfo2.onDone.get()); + assertFalse(getDataStream.awaitTermination(10, TimeUnit.MILLISECONDS)); + + Windmill.KeyedGetDataResponse keyedGetDataResponse = createTestResponse(1); + streamInfo2.responseObserver.onNext( + Windmill.StreamingGetDataResponse.newBuilder() + .addRequestId(1) + .addSerializedResponse(keyedGetDataResponse.toByteString()) + .build()); + assertThat(sendFuture.join()).isEqualTo(keyedGetDataResponse); + + // Complete half-closing from the server and verify shutdown completes. + streamInfo2.responseObserver.onCompleted(); + + assertTrue(getDataStream.awaitTermination(10, TimeUnit.SECONDS)); + } + + @Test + public void testRequestKeyedData_multiplePhysicalStreams_newStreamFailsWhileEmpty() + throws InterruptedException, ExecutionException { + TriggeredScheduledExecutorService triggeredExecutor = new TriggeredScheduledExecutorService(); + GrpcGetDataStream getDataStream = + createGetDataStreamWithPhysicalStreamHandover(Duration.ofSeconds(60), triggeredExecutor); + FakeWindmillGrpcService.GetDataStreamInfo streamInfo = waitForConnectionAndConsumeHeader(); + + // These will block until they are successfully sent. + Windmill.KeyedGetDataRequest keyedGetDataRequest = createTestRequest(1); + CompletableFuture sendFuture = + CompletableFuture.supplyAsync( + () -> { + try { + return getDataStream.requestKeyedData("computationId", keyedGetDataRequest); + } catch (WindmillStreamShutdownException e) { + throw new RuntimeException(e); + } + }); + + Windmill.StreamingGetDataRequest request = streamInfo.requests.take(); + assertThat(request.getRequestIdList()).containsExactly(1L); + assertEquals(keyedGetDataRequest, request.getStateRequest(0).getRequests(0)); + + // A new stream should be created due to handover. + assertTrue(triggeredExecutor.unblockNextFuture()); + + FakeWindmillGrpcService.GetDataStreamInfo streamInfo2 = waitForConnectionAndConsumeHeader(); + assertNull(streamInfo.onDone.get()); + + // Before stream 1 is finished simulate stream 2 failing. + streamInfo2.responseObserver.onError(new IOException("stream 2 failed")); + // A new stream should be created and handle new requests. + FakeWindmillGrpcService.GetDataStreamInfo streamInfo3 = waitForConnectionAndConsumeHeader(); + assertNull(streamInfo.onDone.get()); + + Windmill.KeyedGetDataRequest keyedGetDataRequest2 = createTestRequest(2); + CompletableFuture sendFuture2 = + CompletableFuture.supplyAsync( + () -> { + try { + return getDataStream.requestKeyedData("computationId", keyedGetDataRequest2); + } catch (WindmillStreamShutdownException e) { + throw new RuntimeException(e); + } + }); + Windmill.StreamingGetDataRequest request2 = streamInfo3.requests.take(); + assertThat(request2.getRequestIdList()).containsExactly(2L); + assertEquals(keyedGetDataRequest2, request2.getStateRequest(0).getRequests(0)); + + Windmill.KeyedGetDataResponse keyedGetDataResponse2 = createTestResponse(2); + streamInfo3.responseObserver.onNext( + Windmill.StreamingGetDataResponse.newBuilder() + .addRequestId(2) + .addSerializedResponse(keyedGetDataResponse2.toByteString()) + .build()); + + Windmill.KeyedGetDataResponse keyedGetDataResponse = createTestResponse(1); + streamInfo.responseObserver.onNext( + Windmill.StreamingGetDataResponse.newBuilder() + .addRequestId(1) + .addSerializedResponse(keyedGetDataResponse.toByteString()) + .build()); + assertThat(sendFuture.join()).isEqualTo(keyedGetDataResponse); + assertThat(sendFuture2.join()).isEqualTo(keyedGetDataResponse2); + + // Close the stream. + getDataStream.halfClose(); + assertNull(streamInfo.onDone.get()); + fakeService.expectNoMoreStreams(); + streamInfo.responseObserver.onCompleted(); + streamInfo3.responseObserver.onCompleted(); + + assertTrue(getDataStream.awaitTermination(10, TimeUnit.SECONDS)); + } + + @Test + public void testRequestKeyedData_multiplePhysicalStreams_newStreamFailsWithRequests() + throws InterruptedException, ExecutionException { + TriggeredScheduledExecutorService triggeredExecutor = new TriggeredScheduledExecutorService(); + GrpcGetDataStream getDataStream = + createGetDataStreamWithPhysicalStreamHandover(Duration.ofSeconds(60), triggeredExecutor); + FakeWindmillGrpcService.GetDataStreamInfo streamInfo = waitForConnectionAndConsumeHeader(); + + // These will block until they are successfully sent. + Windmill.KeyedGetDataRequest keyedGetDataRequest = createTestRequest(1); + CompletableFuture sendFuture = + CompletableFuture.supplyAsync( + () -> { + try { + return getDataStream.requestKeyedData("computationId", keyedGetDataRequest); + } catch (WindmillStreamShutdownException e) { + throw new RuntimeException(e); + } + }); + + Windmill.StreamingGetDataRequest request = streamInfo.requests.take(); + assertThat(request.getRequestIdList()).containsExactly(1L); + assertEquals(keyedGetDataRequest, request.getStateRequest(0).getRequests(0)); + + // A new stream should be created due to handover. + assertTrue(triggeredExecutor.unblockNextFuture()); + + FakeWindmillGrpcService.GetDataStreamInfo streamInfo2 = waitForConnectionAndConsumeHeader(); + assertNull(streamInfo.onDone.get()); + + Windmill.KeyedGetDataRequest keyedGetDataRequest2 = createTestRequest(2); + CompletableFuture sendFuture2 = + CompletableFuture.supplyAsync( + () -> { + try { + return getDataStream.requestKeyedData("computationId", keyedGetDataRequest2); + } catch (WindmillStreamShutdownException e) { + throw new RuntimeException(e); + } + }); + Windmill.StreamingGetDataRequest request2 = streamInfo2.requests.take(); + assertThat(request2.getRequestIdList()).containsExactly(2L); + assertEquals(keyedGetDataRequest2, request2.getStateRequest(0).getRequests(0)); + + // Before stream 1 is finished simulate stream 2 failing. + streamInfo2.responseObserver.onError(new IOException("stream 2 failed")); + // A new stream should be created and receive the pending requests from stream2 but not the + // request from stream1. + FakeWindmillGrpcService.GetDataStreamInfo streamInfo3 = waitForConnectionAndConsumeHeader(); + assertNull(streamInfo.onDone.get()); + Windmill.StreamingGetDataRequest request3 = streamInfo3.requests.take(); + assertThat(request3.getRequestIdList()).containsExactly(2L); + assertEquals(keyedGetDataRequest2, request3.getStateRequest(0).getRequests(0)); + + Windmill.KeyedGetDataResponse keyedGetDataResponse2 = createTestResponse(2); + streamInfo3.responseObserver.onNext( + Windmill.StreamingGetDataResponse.newBuilder() + .addRequestId(2) + .addSerializedResponse(keyedGetDataResponse2.toByteString()) + .build()); + + Windmill.KeyedGetDataResponse keyedGetDataResponse = createTestResponse(1); + streamInfo.responseObserver.onNext( + Windmill.StreamingGetDataResponse.newBuilder() + .addRequestId(1) + .addSerializedResponse(keyedGetDataResponse.toByteString()) + .build()); + assertThat(sendFuture.join()).isEqualTo(keyedGetDataResponse); + assertThat(sendFuture2.join()).isEqualTo(keyedGetDataResponse2); + + // Close the stream. + getDataStream.halfClose(); + assertNull(streamInfo.onDone.get()); + fakeService.expectNoMoreStreams(); + streamInfo.responseObserver.onCompleted(); + streamInfo3.responseObserver.onCompleted(); + + assertTrue(getDataStream.awaitTermination(10, TimeUnit.SECONDS)); + } + + @Test + public void testRequestKeyedData_multiplePhysicalStreams_multipleHandovers_allResponsesReceived() + throws InterruptedException, ExecutionException { + TriggeredScheduledExecutorService triggeredExecutor = new TriggeredScheduledExecutorService(); + GrpcGetDataStream getDataStream = + createGetDataStreamWithPhysicalStreamHandover(Duration.ofSeconds(60), triggeredExecutor); + FakeWindmillGrpcService.GetDataStreamInfo streamInfo = waitForConnectionAndConsumeHeader(); + + // Request 1, Stream 1 + Windmill.KeyedGetDataRequest keyedGetDataRequest1 = createTestRequest(1); + CompletableFuture sendFuture1 = + CompletableFuture.supplyAsync( + () -> { + try { + return getDataStream.requestKeyedData("computationId", keyedGetDataRequest1); + } catch (WindmillStreamShutdownException e) { + throw new RuntimeException(e); + } + }); + Windmill.StreamingGetDataRequest request1 = streamInfo.requests.take(); + assertThat(request1.getRequestIdList()).containsExactly(1L); + assertEquals(keyedGetDataRequest1, request1.getStateRequest(0).getRequests(0)); + + // Trigger handover 1 + assertTrue(triggeredExecutor.unblockNextFuture()); + FakeWindmillGrpcService.GetDataStreamInfo streamInfo2 = waitForConnectionAndConsumeHeader(); + assertNull(streamInfo.onDone.get()); + + // Request 2, Stream 2 + Windmill.KeyedGetDataRequest keyedGetDataRequest2 = createTestRequest(2); + CompletableFuture sendFuture2 = + CompletableFuture.supplyAsync( + () -> { + try { + return getDataStream.requestKeyedData("computationId", keyedGetDataRequest2); + } catch (WindmillStreamShutdownException e) { + throw new RuntimeException(e); + } + }); + Windmill.StreamingGetDataRequest request2 = streamInfo2.requests.take(); + assertThat(request2.getRequestIdList()).containsExactly(2L); + assertEquals(keyedGetDataRequest2, request2.getStateRequest(0).getRequests(0)); + + // Trigger handover 2 before streamInfo2 completes + assertTrue(triggeredExecutor.unblockNextFuture()); + FakeWindmillGrpcService.GetDataStreamInfo streamInfo3 = waitForConnectionAndConsumeHeader(); + assertNull(streamInfo2.onDone.get()); + + // Request 3, Stream 3 + Windmill.KeyedGetDataRequest keyedGetDataRequest3 = createTestRequest(3); + CompletableFuture sendFuture3 = + CompletableFuture.supplyAsync( + () -> { + try { + return getDataStream.requestKeyedData("computationId", keyedGetDataRequest3); + } catch (WindmillStreamShutdownException e) { + throw new RuntimeException(e); + } + }); + Windmill.StreamingGetDataRequest request3 = streamInfo3.requests.take(); + assertThat(request3.getRequestIdList()).containsExactly(3L); + assertEquals(keyedGetDataRequest3, request3.getStateRequest(0).getRequests(0)); + + // Respond to all requests + Windmill.KeyedGetDataResponse keyedGetDataResponse1 = createTestResponse(1); + streamInfo.responseObserver.onNext( + Windmill.StreamingGetDataResponse.newBuilder() + .addRequestId(1) + .addSerializedResponse(keyedGetDataResponse1.toByteString()) + .build()); + + Windmill.KeyedGetDataResponse keyedGetDataResponse2 = createTestResponse(2); + streamInfo2.responseObserver.onNext( + Windmill.StreamingGetDataResponse.newBuilder() + .addRequestId(2) + .addSerializedResponse(keyedGetDataResponse2.toByteString()) + .build()); + streamInfo2.responseObserver.onCompleted(); + + Windmill.KeyedGetDataResponse keyedGetDataResponse3 = createTestResponse(3); + streamInfo3.responseObserver.onNext( + Windmill.StreamingGetDataResponse.newBuilder() + .addRequestId(3) + .addSerializedResponse(keyedGetDataResponse3.toByteString()) + .build()); + + assertThat(sendFuture1.join()).isEqualTo(keyedGetDataResponse1); + assertThat(sendFuture2.join()).isEqualTo(keyedGetDataResponse2); + assertThat(sendFuture3.join()).isEqualTo(keyedGetDataResponse3); + + // Close the stream. + getDataStream.halfClose(); + assertNull(streamInfo3.onDone.get()); + + fakeService.expectNoMoreStreams(); + streamInfo.responseObserver.onCompleted(); + streamInfo3.responseObserver.onCompleted(); + + assertTrue(getDataStream.awaitTermination(10, TimeUnit.SECONDS)); + } + + @Test + public void testRequestKeyedData_multiplePhysicalStreams_oldStreamFailsWhileNewStreamInBackoff() + throws InterruptedException, ExecutionException { + TriggeredScheduledExecutorService triggeredExecutor = new TriggeredScheduledExecutorService(); + GrpcGetDataStream getDataStream = + createGetDataStreamWithPhysicalStreamHandover(Duration.ofSeconds(60), triggeredExecutor); + FakeWindmillGrpcService.GetDataStreamInfo streamInfo = waitForConnectionAndConsumeHeader(); + + Windmill.KeyedGetDataRequest keyedGetDataRequest = createTestRequest(1); + CompletableFuture sendFuture = + CompletableFuture.supplyAsync( + () -> { + try { + return getDataStream.requestKeyedData("computationId", keyedGetDataRequest); + } catch (WindmillStreamShutdownException e) { + throw new RuntimeException(e); + } + }); + + Windmill.StreamingGetDataRequest request = streamInfo.requests.take(); + assertThat(request.getRequestIdList()).containsExactly(1L); + assertEquals(keyedGetDataRequest, request.getStateRequest(0).getRequests(0)); + + // A new stream should be created due to handover. However we configure the server to have + // errors. + assertTrue(triggeredExecutor.unblockNextFuture()); + fakeService.setFailedStreamConnectsRemaining(1); + fakeService.waitForFailedConnectAttempts(); + // Previous stream client should be half-closed. + assertNull(streamInfo.onDone.get()); + // Complete first stream with an error. No new + // stream should be created since the current stream is being created or created. The request + // should have an + // error and the request should be retried on the new stream. + streamInfo.responseObserver.onError(new RuntimeException("test error")); + + FakeWindmillGrpcService.GetDataStreamInfo streamInfo2 = waitForConnectionAndConsumeHeader(); + fakeService.expectNoMoreStreams(); + + Windmill.StreamingGetDataRequest request2 = streamInfo2.requests.take(); + assertThat(request2.getRequestIdList()).containsExactly(1L); + assertEquals(keyedGetDataRequest, request2.getStateRequest(0).getRequests(0)); + + // Close the stream, the open stream should be client half-closed + // but logical remains not terminated. + getDataStream.halfClose(); + assertNull(streamInfo2.onDone.get()); + assertFalse(getDataStream.awaitTermination(10, TimeUnit.MILLISECONDS)); + + Windmill.KeyedGetDataResponse keyedGetDataResponse = createTestResponse(1); + streamInfo2.responseObserver.onNext( + Windmill.StreamingGetDataResponse.newBuilder() + .addRequestId(1) + .addSerializedResponse(keyedGetDataResponse.toByteString()) + .build()); + assertThat(sendFuture.join()).isEqualTo(keyedGetDataResponse); + + // Complete half-closing from the server and verify shutdown completes. + streamInfo2.responseObserver.onCompleted(); + + assertTrue(getDataStream.awaitTermination(10, TimeUnit.SECONDS)); + } + + @Test + public void testRequestKeyedData_multiplePhysicalStreams_multipleHandovers_shutdown() + throws InterruptedException, ExecutionException { + TriggeredScheduledExecutorService triggeredExecutor = new TriggeredScheduledExecutorService(); + GrpcGetDataStream getDataStream = + createGetDataStreamWithPhysicalStreamHandover(Duration.ofSeconds(60), triggeredExecutor); + FakeWindmillGrpcService.GetDataStreamInfo streamInfo = waitForConnectionAndConsumeHeader(); + + // Request 1, Stream 1 + Windmill.KeyedGetDataRequest keyedGetDataRequest1 = createTestRequest(1); + CompletableFuture sendFuture1 = + CompletableFuture.supplyAsync( + () -> { + try { + return getDataStream.requestKeyedData("computationId", keyedGetDataRequest1); + } catch (WindmillStreamShutdownException e) { + throw new RuntimeException(e); + } + }); + Windmill.StreamingGetDataRequest request1 = streamInfo.requests.take(); + assertThat(request1.getRequestIdList()).containsExactly(1L); + assertEquals(keyedGetDataRequest1, request1.getStateRequest(0).getRequests(0)); + + // Trigger handover 1 + assertTrue(triggeredExecutor.unblockNextFuture()); + FakeWindmillGrpcService.GetDataStreamInfo streamInfo2 = waitForConnectionAndConsumeHeader(); + assertNull(streamInfo.onDone.get()); + + // Request 2, Stream 2 + Windmill.KeyedGetDataRequest keyedGetDataRequest2 = createTestRequest(2); + CompletableFuture sendFuture2 = + CompletableFuture.supplyAsync( + () -> { + try { + return getDataStream.requestKeyedData("computationId", keyedGetDataRequest2); + } catch (WindmillStreamShutdownException e) { + throw new RuntimeException(e); + } + }); + Windmill.StreamingGetDataRequest request2 = streamInfo2.requests.take(); + assertThat(request2.getRequestIdList()).containsExactly(2L); + assertEquals(keyedGetDataRequest2, request2.getStateRequest(0).getRequests(0)); + + // Trigger handover 2 before streamInfo2 completes + assertTrue(triggeredExecutor.unblockNextFuture()); + FakeWindmillGrpcService.GetDataStreamInfo streamInfo3 = waitForConnectionAndConsumeHeader(); + assertNull(streamInfo2.onDone.get()); + + // Request 3, Stream 3 + Windmill.KeyedGetDataRequest keyedGetDataRequest3 = createTestRequest(3); + CompletableFuture sendFuture3 = + CompletableFuture.supplyAsync( + () -> { + try { + return getDataStream.requestKeyedData("computationId", keyedGetDataRequest3); + } catch (WindmillStreamShutdownException e) { + throw new RuntimeException(e); + } + }); + Windmill.StreamingGetDataRequest request3 = streamInfo3.requests.take(); + assertThat(request3.getRequestIdList()).containsExactly(3L); + assertEquals(keyedGetDataRequest3, request3.getStateRequest(0).getRequests(0)); + + // Shutdown while there are active streams and verify it isn't completed until all the streams + // are done. + fakeService.expectNoMoreStreams(); + assertFalse(getDataStream.awaitTermination(0, TimeUnit.SECONDS)); + getDataStream.shutdown(); + assertThrows("WindmillStreamShutdownException", CompletionException.class, sendFuture1::join); + assertThrows("WindmillStreamShutdownException", CompletionException.class, sendFuture2::join); + assertThrows("WindmillStreamShutdownException", CompletionException.class, sendFuture3::join); + assertFalse(getDataStream.awaitTermination(10, TimeUnit.MILLISECONDS)); + + assertFalse(getDataStream.awaitTermination(0, TimeUnit.MILLISECONDS)); + streamInfo3.responseObserver.onCompleted(); + assertFalse(getDataStream.awaitTermination(0, TimeUnit.MILLISECONDS)); + streamInfo.responseObserver.onCompleted(); + assertFalse(getDataStream.awaitTermination(0, TimeUnit.MILLISECONDS)); + streamInfo2.responseObserver.onError(new RuntimeException("test")); + assertTrue(getDataStream.awaitTermination(10, TimeUnit.SECONDS)); + } + + @Test + public void testRequestKeyedData_multiplePhysicalStreams_multipleHandovers_halfClose() + throws InterruptedException, ExecutionException { + TriggeredScheduledExecutorService triggeredExecutor = new TriggeredScheduledExecutorService(); + GrpcGetDataStream getDataStream = + createGetDataStreamWithPhysicalStreamHandover(Duration.ofSeconds(60), triggeredExecutor); + FakeWindmillGrpcService.GetDataStreamInfo streamInfo = waitForConnectionAndConsumeHeader(); + + // Request 1, Stream 1 + Windmill.KeyedGetDataRequest keyedGetDataRequest1 = createTestRequest(1); + CompletableFuture sendFuture1 = + CompletableFuture.supplyAsync( + () -> { + try { + return getDataStream.requestKeyedData("computationId", keyedGetDataRequest1); + } catch (WindmillStreamShutdownException e) { + throw new RuntimeException(e); + } + }); + Windmill.StreamingGetDataRequest request1 = streamInfo.requests.take(); + assertThat(request1.getRequestIdList()).containsExactly(1L); + assertEquals(keyedGetDataRequest1, request1.getStateRequest(0).getRequests(0)); + + // Trigger handover 1 + assertTrue(triggeredExecutor.unblockNextFuture()); + FakeWindmillGrpcService.GetDataStreamInfo streamInfo2 = waitForConnectionAndConsumeHeader(); + assertNull(streamInfo.onDone.get()); + + // Request 2, Stream 2 + Windmill.KeyedGetDataRequest keyedGetDataRequest2 = createTestRequest(2); + CompletableFuture sendFuture2 = + CompletableFuture.supplyAsync( + () -> { + try { + return getDataStream.requestKeyedData("computationId", keyedGetDataRequest2); + } catch (WindmillStreamShutdownException e) { + throw new RuntimeException(e); + } + }); + Windmill.StreamingGetDataRequest request2 = streamInfo2.requests.take(); + assertThat(request2.getRequestIdList()).containsExactly(2L); + assertEquals(keyedGetDataRequest2, request2.getStateRequest(0).getRequests(0)); + + // Trigger handover 2 before streamInfo2 completes + assertTrue(triggeredExecutor.unblockNextFuture()); + FakeWindmillGrpcService.GetDataStreamInfo streamInfo3 = waitForConnectionAndConsumeHeader(); + assertNull(streamInfo2.onDone.get()); + + // Request 3, Stream 3 + Windmill.KeyedGetDataRequest keyedGetDataRequest3 = createTestRequest(3); + CompletableFuture sendFuture3 = + CompletableFuture.supplyAsync( + () -> { + try { + return getDataStream.requestKeyedData("computationId", keyedGetDataRequest3); + } catch (WindmillStreamShutdownException e) { + throw new RuntimeException(e); + } + }); + Windmill.StreamingGetDataRequest request3 = streamInfo3.requests.take(); + assertThat(request3.getRequestIdList()).containsExactly(3L); + assertEquals(keyedGetDataRequest3, request3.getStateRequest(0).getRequests(0)); + + // Half-close while there are active streams and verify it isn't completed until all the streams + // are done. Streams with requests should have requests resent. + fakeService.expectNoMoreStreams(); + assertFalse(getDataStream.awaitTermination(0, TimeUnit.SECONDS)); + getDataStream.halfClose(); + assertNull(streamInfo.onDone.get()); + assertFalse(getDataStream.awaitTermination(10, TimeUnit.MILLISECONDS)); + + Windmill.KeyedGetDataResponse keyedGetDataResponse3 = createTestResponse(3); + streamInfo3.responseObserver.onNext( + Windmill.StreamingGetDataResponse.newBuilder() + .addRequestId(3) + .addSerializedResponse(keyedGetDataResponse3.toByteString()) + .build()); + assertThat(sendFuture3.join()).isEqualTo(keyedGetDataResponse3); + + assertFalse(getDataStream.awaitTermination(0, TimeUnit.MILLISECONDS)); + Windmill.KeyedGetDataResponse keyedGetDataResponse = createTestResponse(1); + streamInfo.responseObserver.onNext( + Windmill.StreamingGetDataResponse.newBuilder() + .addRequestId(1) + .addSerializedResponse(keyedGetDataResponse.toByteString()) + .build()); + assertThat(sendFuture1.join()).isEqualTo(keyedGetDataResponse); + + streamInfo.responseObserver.onCompleted(); + assertFalse(getDataStream.awaitTermination(0, TimeUnit.MILLISECONDS)); + + Windmill.KeyedGetDataResponse keyedGetDataResponse2 = createTestResponse(2); + streamInfo2.responseObserver.onNext( + Windmill.StreamingGetDataResponse.newBuilder() + .addRequestId(2) + .addSerializedResponse(keyedGetDataResponse2.toByteString()) + .build()); + assertThat(sendFuture2.join()).isEqualTo(keyedGetDataResponse2); + streamInfo2.responseObserver.onCompleted(); + streamInfo3.responseObserver.onCompleted(); + assertTrue(getDataStream.awaitTermination(10, TimeUnit.SECONDS)); + } + + @Test + public void testRequestKeyedData_raceShutdownDuringTrySendBatch() throws Exception { + AtomicBoolean connectedOnce = new AtomicBoolean(false); + CountDownLatch failedConnects = new CountDownLatch(2); + GrpcGetDataStream getDataStream = + (GrpcGetDataStream) + GrpcWindmillStreamFactory.of(TEST_JOB_HEADER) + .setSendKeyedGetDataRequests(false) + .build() + .createGetDataStream( + CloudWindmillServiceV1Alpha1Grpc.newStub(inProcessChannel) + .withInterceptors( + new ClientInterceptor() { + @Override + public ClientCall interceptCall( + MethodDescriptor methodDescriptor, + CallOptions callOptions, + Channel channel) { + if (connectedOnce.getAndSet(true)) { + failedConnects.countDown(); + throw new RuntimeException("test error"); + } + return channel.newCall(methodDescriptor, callOptions); + } + })); + getDataStream.start(); + // Wait for the first stream to succeed and cause it to fail, the rest should fail. + FakeWindmillGrpcService.GetDataStreamInfo streamInfo = waitForConnectionAndConsumeHeader(); + streamInfo.responseObserver.onError(new RuntimeException("fake error")); + + failedConnects.await(); + + // Send while we're in this state. + // Create a request + Windmill.KeyedGetDataRequest keyedGetDataRequest = createTestRequest(1); + CompletableFuture sendFuture = + CompletableFuture.supplyAsync( + () -> { + try { + return getDataStream.requestKeyedData("computationId", keyedGetDataRequest); + } catch (WindmillStreamShutdownException e) { + throw new RuntimeException(e); + } + }); + + // The shutdown should work if it occurs either before or after the above request is sent. + Thread.sleep(100); getDataStream.shutdown(); + + // The request should complete with an exception, it may or may not get there. + assertThrows(CompletionException.class, sendFuture::join); + assertTrue(sendFuture.isCompletedExceptionally()); } private FakeWindmillGrpcService.GetDataStreamInfo waitForConnectionAndConsumeHeader() { diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/ChannelCacheTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/ChannelCacheTest.java index 311bed75ccc7..dd039782d1fc 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/ChannelCacheTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/stubs/ChannelCacheTest.java @@ -194,6 +194,139 @@ public void testConsumeFlowControlSettings() throws InterruptedException { assertThat(consumedFlowControlSettings.get()).isEqualTo(flowControlSettings); } + @Test + public void testConsumeFlowControlSettings_UsesDefaultOverridesForDirect() + throws InterruptedException { + String channelName = "channel"; + AtomicReference notifyWhenChannelClosed = + new AtomicReference<>(new CountDownLatch(1)); + AtomicInteger newChannelsCreated = new AtomicInteger(); + AtomicReference consumedFlowControlSettings = + new AtomicReference<>(); + cache = + ChannelCache.forTesting( + (newFlowControlSettings, ignoredServiceAddress) -> { + ManagedChannel channel = newChannel(channelName); + newChannelsCreated.incrementAndGet(); + consumedFlowControlSettings.set(newFlowControlSettings); + return channel; + }, + () -> notifyWhenChannelClosed.get().countDown()); + WindmillServiceAddress someAddress = mock(WindmillServiceAddress.class); + when(someAddress.getKind()) + .thenReturn(WindmillServiceAddress.Kind.AUTHENTICATED_GCP_SERVICE_ADDRESS); + + UserWorkerGrpcFlowControlSettings emptyFlowControlSettings = + UserWorkerGrpcFlowControlSettings.newBuilder().build(); + + // Load the cache w/ this first get. + ManagedChannel cachedChannel = cache.get(someAddress); + // Verify that the appropriate default was used. + assertThat(consumedFlowControlSettings.get()) + .isEqualTo(WindmillChannels.DEFAULT_DIRECTPATH_FLOW_CONTROL_SETTINGS); + + // Load empty flow control settings. + cache.consumeFlowControlSettings(emptyFlowControlSettings); + // This get shouldn't reload the cache, since the same default flow control settings + // should be used. + assertThat(consumedFlowControlSettings.get()) + .isEqualTo(WindmillChannels.DEFAULT_DIRECTPATH_FLOW_CONTROL_SETTINGS); + assertThat(cachedChannel).isSameInstanceAs(cache.get(someAddress)); + + // This get should reload the cache, since flow control settings have changed + UserWorkerGrpcFlowControlSettings flowControlSettingsModified = + UserWorkerGrpcFlowControlSettings.newBuilder().setEnableAutoFlowControl(true).build(); + cache.consumeFlowControlSettings(flowControlSettingsModified); + ManagedChannel reloadedChannel = cache.get(someAddress); + notifyWhenChannelClosed.get().await(); + assertThat(cachedChannel).isNotSameInstanceAs(reloadedChannel); + assertTrue(cachedChannel.isShutdown()); + assertFalse(reloadedChannel.isShutdown()); + assertThat(newChannelsCreated.get()).isEqualTo(2); + assertThat(cache.get(someAddress)).isSameInstanceAs(reloadedChannel); + assertThat(consumedFlowControlSettings.get()).isEqualTo(flowControlSettingsModified); + + // Change back to empty settings and verify the default is used again. + notifyWhenChannelClosed.set(new CountDownLatch(1)); + cache.consumeFlowControlSettings(emptyFlowControlSettings); + ManagedChannel reloadedChannel2 = cache.get(someAddress); + notifyWhenChannelClosed.get().await(); + assertThat(reloadedChannel2).isNotSameInstanceAs(reloadedChannel); + assertThat(reloadedChannel2).isNotSameInstanceAs(cachedChannel); + assertTrue(reloadedChannel.isShutdown()); + assertFalse(reloadedChannel2.isShutdown()); + assertThat(newChannelsCreated.get()).isEqualTo(3); + assertThat(cache.get(someAddress)).isSameInstanceAs(reloadedChannel2); + assertThat(consumedFlowControlSettings.get()) + .isEqualTo(WindmillChannels.DEFAULT_DIRECTPATH_FLOW_CONTROL_SETTINGS); + } + + @Test + public void testConsumeFlowControlSettings_UsesDefaultOverridesForCloudPath() + throws InterruptedException { + String channelName = "channel"; + AtomicReference notifyWhenChannelClosed = + new AtomicReference<>(new CountDownLatch(1)); + AtomicInteger newChannelsCreated = new AtomicInteger(); + AtomicReference consumedFlowControlSettings = + new AtomicReference<>(); + cache = + ChannelCache.forTesting( + (newFlowControlSettings, ignoredServiceAddress) -> { + ManagedChannel channel = newChannel(channelName); + newChannelsCreated.incrementAndGet(); + consumedFlowControlSettings.set(newFlowControlSettings); + return channel; + }, + () -> notifyWhenChannelClosed.get().countDown()); + WindmillServiceAddress someAddress = mock(WindmillServiceAddress.class); + when(someAddress.getKind()).thenReturn(WindmillServiceAddress.Kind.GCP_SERVICE_ADDRESS); + + UserWorkerGrpcFlowControlSettings emptyFlowControlSettings = + UserWorkerGrpcFlowControlSettings.newBuilder().build(); + + // Load the cache w/ this first get. + ManagedChannel cachedChannel = cache.get(someAddress); + // Verify that the appropriate default was used. + assertThat(consumedFlowControlSettings.get()) + .isEqualTo(WindmillChannels.DEFAULT_CLOUDPATH_FLOW_CONTROL_SETTINGS); + + // Load empty flow control settings. + cache.consumeFlowControlSettings(emptyFlowControlSettings); + // This get shouldn't reload the cache, since the same default flow control settings + // should be used. + assertThat(consumedFlowControlSettings.get()) + .isEqualTo(WindmillChannels.DEFAULT_CLOUDPATH_FLOW_CONTROL_SETTINGS); + assertThat(cachedChannel).isSameInstanceAs(cache.get(someAddress)); + + // This get should reload the cache, since flow control settings have changed + UserWorkerGrpcFlowControlSettings flowControlSettingsModified = + UserWorkerGrpcFlowControlSettings.newBuilder().setEnableAutoFlowControl(true).build(); + cache.consumeFlowControlSettings(flowControlSettingsModified); + ManagedChannel reloadedChannel = cache.get(someAddress); + notifyWhenChannelClosed.get().await(); + assertThat(cachedChannel).isNotSameInstanceAs(reloadedChannel); + assertTrue(cachedChannel.isShutdown()); + assertFalse(reloadedChannel.isShutdown()); + assertThat(newChannelsCreated.get()).isEqualTo(2); + assertThat(cache.get(someAddress)).isSameInstanceAs(reloadedChannel); + assertThat(consumedFlowControlSettings.get()).isEqualTo(flowControlSettingsModified); + + // Change back to empty settings and verify the default is used again. + notifyWhenChannelClosed.set(new CountDownLatch(1)); + cache.consumeFlowControlSettings(emptyFlowControlSettings); + ManagedChannel reloadedChannel2 = cache.get(someAddress); + notifyWhenChannelClosed.get().await(); + assertThat(reloadedChannel2).isNotSameInstanceAs(reloadedChannel); + assertThat(reloadedChannel2).isNotSameInstanceAs(cachedChannel); + assertTrue(reloadedChannel.isShutdown()); + assertFalse(reloadedChannel2.isShutdown()); + assertThat(newChannelsCreated.get()).isEqualTo(3); + assertThat(cache.get(someAddress)).isSameInstanceAs(reloadedChannel2); + assertThat(consumedFlowControlSettings.get()) + .isEqualTo(WindmillChannels.DEFAULT_CLOUDPATH_FLOW_CONTROL_SETTINGS); + } + @Test public void testConsumeFlowControlSettings_sameFlowControlSettings() { String channelName = "channel"; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternalsTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternalsTest.java index 9fe424fe9894..cb4f7a1298f2 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternalsTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternalsTest.java @@ -272,16 +272,12 @@ public void tearDown() throws Exception { } private void waitAndSet(final SettableFuture future, final T value, final long millis) { - new Thread( - () -> { - try { - sleepMillis(millis); - } catch (InterruptedException e) { - throw new RuntimeException("Interrupted before setting", e); - } - future.set(value); - }) - .run(); + try { + sleepMillis(millis); + } catch (InterruptedException e) { + throw new RuntimeException("Interrupted before setting", e); + } + future.set(value); } private WeightedList weightedList(String... elems) { diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/DockerCommand.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/DockerCommand.java index 79484fea9fa7..ff69ee3c4171 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/DockerCommand.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/DockerCommand.java @@ -97,7 +97,7 @@ public String runImage(String imageTag, List dockerOpts, List ar if (LOG.isDebugEnabled()) { LOG.debug("Unable to pull docker image {}", imageTag, e); } else { - LOG.warn("Unable to pull docker image {}, cause: {}", imageTag, e.getMessage()); + LOG.warn("Unable to pull docker image {}", imageTag, e); } } // TODO: Validate args? diff --git a/runners/java-job-service/src/test/java/org/apache/beam/runners/jobsubmission/PortablePipelineJarCreatorTest.java b/runners/java-job-service/src/test/java/org/apache/beam/runners/jobsubmission/PortablePipelineJarCreatorTest.java index e9650efad0f4..9296fcea3597 100644 --- a/runners/java-job-service/src/test/java/org/apache/beam/runners/jobsubmission/PortablePipelineJarCreatorTest.java +++ b/runners/java-job-service/src/test/java/org/apache/beam/runners/jobsubmission/PortablePipelineJarCreatorTest.java @@ -143,6 +143,7 @@ public void testCreateManifest_withoutMainMethod() { assertNull(manifest.getMainAttributes().getValue(Name.MAIN_CLASS)); } + @SuppressWarnings("IncorrectMainMethod") // intended private static class EvilPipelineRunner { public static int main(String[] args) { return 0; diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/Utils.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/Utils.java index cb1cd69d33c6..06e07d0c6cfc 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/Utils.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/Utils.java @@ -110,10 +110,8 @@ static Map.Entry, PCollection> getOutput( return Iterables.getOnlyElement(getOutputs(appliedTransform).entrySet()); } - static boolean isBounded(AppliedPTransform appliedTransform) { - return ((PCollection) getOutput(appliedTransform).getValue()) - .isBounded() - .equals(PCollection.IsBounded.BOUNDED); + static boolean isBounded(AppliedPTransform appliedTransform) { + return getOutput(appliedTransform).getValue().isBounded().equals(PCollection.IsBounded.BOUNDED); } static boolean isKeyedValueCoder(Coder coder) { diff --git a/runners/jet/src/test/java/org/apache/beam/runners/jet/TestStreamP.java b/runners/jet/src/test/java/org/apache/beam/runners/jet/TestStreamP.java index 291ca91b5c18..aee37d7a5ce8 100644 --- a/runners/jet/src/test/java/org/apache/beam/runners/jet/TestStreamP.java +++ b/runners/jet/src/test/java/org/apache/beam/runners/jet/TestStreamP.java @@ -82,7 +82,7 @@ private TestStreamP(byte[] payload, TestStream.TestStreamCoder payloadCoder, Cod })); } - public static ProcessorMetaSupplier supplier( + public static ProcessorMetaSupplier supplier( byte[] payload, TestStream.TestStreamCoder payloadCoder, Coder outputCoder) { return ProcessorMetaSupplier.forceTotalParallelismOne( ProcessorSupplier.of( diff --git a/runners/local-java/src/main/java/org/apache/beam/runners/local/StructuralKey.java b/runners/local-java/src/main/java/org/apache/beam/runners/local/StructuralKey.java index 5d8578440152..ecb652d2ffe2 100644 --- a/runners/local-java/src/main/java/org/apache/beam/runners/local/StructuralKey.java +++ b/runners/local-java/src/main/java/org/apache/beam/runners/local/StructuralKey.java @@ -20,6 +20,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.util.CoderUtils; +import org.checkerframework.checker.nullness.qual.MonotonicNonNull; import org.checkerframework.checker.nullness.qual.Nullable; /** @@ -57,19 +58,34 @@ public static StructuralKey of(K key, Coder coder) { private static class CoderStructuralKey extends StructuralKey { private final Coder coder; - private final Object structuralValue; - private final byte[] encoded; + private final K key; - private CoderStructuralKey(Coder coder, K key) throws Exception { + private byte @MonotonicNonNull [] encoded; + private @MonotonicNonNull Object structuralValue; + + private CoderStructuralKey(Coder coder, K key) { this.coder = coder; - this.structuralValue = coder.structuralValue(key); - this.encoded = CoderUtils.encodeToByteArray(coder, key); + this.key = key; + } + + private byte[] getEncoded() throws CoderException { + if (encoded == null) { + this.encoded = CoderUtils.encodeToByteArray(coder, this.key); + } + return encoded; + } + + private Object getStructuralValue() { + if (structuralValue == null) { + this.structuralValue = coder.structuralValue(this.key); + } + return structuralValue; } @Override public K getKey() { try { - return CoderUtils.decodeFromByteArray(coder, encoded); + return CoderUtils.decodeFromByteArray(coder, getEncoded()); } catch (CoderException e) { throw new IllegalArgumentException( "Could not decode Key with coder of type " + coder.getClass().getSimpleName(), e); @@ -83,14 +99,14 @@ public boolean equals(@Nullable Object other) { } if (other instanceof CoderStructuralKey) { CoderStructuralKey that = (CoderStructuralKey) other; - return structuralValue.equals(that.structuralValue); + return getStructuralValue().equals(that.getStructuralValue()); } return false; } @Override public int hashCode() { - return structuralValue.hashCode(); + return getStructuralValue().hashCode(); } } } diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineResult.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineResult.java index a75526dc0b1d..ffcc949e4611 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineResult.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineResult.java @@ -108,6 +108,7 @@ public MetricResults metrics() { return asAttemptedOnlyMetricResults(executionContext.getMetricsContainer().getContainers()); } + @SuppressWarnings("Slf4jDoNotLogMessageOfExceptionExplicitly") private StateInfo getStateInfo() { final ApplicationStatus status = runner.status(); switch (status.getStatusCode()) { diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaRunner.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaRunner.java index 111fd684ff63..bc1ada6941b9 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaRunner.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaRunner.java @@ -59,10 +59,13 @@ /** * A {@link PipelineRunner} that executes the operations in the {@link Pipeline} into an equivalent * Samza plan. + * + * @deprecated The support for Samza is scheduled for removal in Beam 3.0. */ @SuppressWarnings({ "nullness" // TODO(https://github.com/apache/beam/issues/20497) }) +@Deprecated public class SamzaRunner extends PipelineRunner { private static final Logger LOG = LoggerFactory.getLogger(SamzaRunner.class); private static final String BEAM_DOT_GRAPH = "beamDotGraph"; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpMessage.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpMessage.java index 9ee7ffd48f2f..217785f19b21 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpMessage.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpMessage.java @@ -59,7 +59,7 @@ public static OpMessage ofSideInput( return new OpMessage<>(Type.SIDE_INPUT, null, viewId, elements, null); } - public static OpMessage ofSideInputWatermark(Instant watermark) { + public static OpMessage ofSideInputWatermark(Instant watermark) { return new OpMessage<>(Type.SIDE_INPUT_WATERMARK, null, null, null, watermark); } diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TranslationContext.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TranslationContext.java index 82725d1ce2e5..c5e984fbde07 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TranslationContext.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TranslationContext.java @@ -224,7 +224,7 @@ public void attachTransformMetricOp( private List getPValueForTransform( SamzaMetricOpFactory.OpType opType, @NonNull PTransform transform, - @NonNull TransformHierarchy.Node node) { + TransformHierarchy.@NonNull Node node) { switch (opType) { case INPUT: { @@ -250,7 +250,7 @@ private List getPValueForTrans // Transforms that read or write to/from external sources are not supported private static boolean isIOTransform( - @NonNull TransformHierarchy.Node node, SamzaMetricOpFactory.OpType opType) { + TransformHierarchy.@NonNull Node node, SamzaMetricOpFactory.OpType opType) { switch (opType) { case INPUT: return node.getInputs().size() == 0; diff --git a/runners/samza/src/test/java/org/apache/beam/runners/samza/adapter/BoundedSourceSystemTest.java b/runners/samza/src/test/java/org/apache/beam/runners/samza/adapter/BoundedSourceSystemTest.java index 4325c29b9b3c..a6bc9940a745 100644 --- a/runners/samza/src/test/java/org/apache/beam/runners/samza/adapter/BoundedSourceSystemTest.java +++ b/runners/samza/src/test/java/org/apache/beam/runners/samza/adapter/BoundedSourceSystemTest.java @@ -291,8 +291,7 @@ private static List pollOnce( return pollResult.get(ssp); } - private static BoundedSourceSystem.Consumer createConsumer( - BoundedSource source) { + private static BoundedSourceSystem.Consumer createConsumer(BoundedSource source) { return createConsumer(source, 1); } diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/EvaluationContext.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/EvaluationContext.java index 32cbe5b0acab..55c4bbaedd3c 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/EvaluationContext.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/EvaluationContext.java @@ -38,6 +38,7 @@ * pipeline. For example, this is necessary to materialize side-inputs. The {@link * EvaluationContext} won't re-evaluate such datasets. */ +@SuppressWarnings("Slf4jDoNotLogMessageOfExceptionExplicitly") @Internal public final class EvaluationContext { private static final Logger LOG = LoggerFactory.getLogger(EvaluationContext.class); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupNonMergingWindowsFunctions.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupNonMergingWindowsFunctions.java index 3b46bee0e8cf..3faf00834fd3 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupNonMergingWindowsFunctions.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupNonMergingWindowsFunctions.java @@ -273,12 +273,11 @@ private WindowedValue> decodeItem(Tuple2 item) { *

This implementation uses {@link JavaPairRDD#combineByKey} for better performance compared to * {@link JavaPairRDD#groupByKey}, as it allows for local aggregation before shuffle operations. */ - static - JavaRDD>>> groupByKeyInGlobalWindow( - JavaRDD>> rdd, - Coder keyCoder, - Coder valueCoder, - Partitioner partitioner) { + static JavaRDD>>> groupByKeyInGlobalWindow( + JavaRDD>> rdd, + Coder keyCoder, + Coder valueCoder, + Partitioner partitioner) { final JavaPairRDD rawKeyValues = rdd.mapPartitionsToPair( (Iterator>> iter) -> diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java index 1345e99bedca..5362beba09dc 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java @@ -754,7 +754,7 @@ public String toNativeString() { }; } - private static TransformEvaluator> window() { + private static TransformEvaluator> window() { return new TransformEvaluator>() { @Override public void evaluate(Window.Assign transform, EvaluationContext context) { diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java index 0963a3c7a750..9534b352f200 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java @@ -325,7 +325,7 @@ public String toNativeString() { }; } - private static TransformEvaluator> window() { + private static TransformEvaluator> window() { return new TransformEvaluator>() { @Override public void evaluate(final Window.Assign transform, EvaluationContext context) { diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SideInputBroadcast.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SideInputBroadcast.java index cf6815c44ec9..a0e0dcaa29bc 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SideInputBroadcast.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SideInputBroadcast.java @@ -75,6 +75,7 @@ public void unpersist() { this.bcast.unpersist(); } + @SuppressWarnings("Slf4jDoNotLogMessageOfExceptionExplicitly") private T deserialize() { T val; try { diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/TimerUtils.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/TimerUtils.java index 30a6577d3e97..d03914a256ca 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/TimerUtils.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/TimerUtils.java @@ -110,6 +110,16 @@ public PaneInfo getPaneInfo() { return PaneInfo.NO_FIRING; } + @Override + public @Nullable String getCurrentRecordId() { + return null; + } + + @Override + public @Nullable Long getCurrentRecordOffset() { + return null; + } + @Override public Iterable> explodeWindows() { return Collections.emptyList(); diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/EmbeddedKafkaCluster.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/EmbeddedKafkaCluster.java index 7794d5b4318e..df5646fed590 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/EmbeddedKafkaCluster.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/EmbeddedKafkaCluster.java @@ -20,6 +20,7 @@ import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; +import java.net.InetAddress; import java.net.InetSocketAddress; import java.net.ServerSocket; import java.util.ArrayList; @@ -145,6 +146,7 @@ public String getZkConnection() { return zkConnection; } + @SuppressWarnings("Slf4jDoNotLogMessageOfExceptionExplicitly") public void shutdown() { for (KafkaServerStartable broker : brokers) { try { @@ -201,7 +203,8 @@ public void startup() throws IOException { this.port = TestUtils.getAvailablePort(); } this.factory = - NIOServerCnxnFactory.createFactory(new InetSocketAddress("127.0.0.1", port), 1024); + NIOServerCnxnFactory.createFactory( + new InetSocketAddress(InetAddress.getLoopbackAddress(), port), 1024); this.snapshotDir = TestUtils.constructTempDir("embedded-zk/snapshot"); this.logDir = TestUtils.constructTempDir("embedded-zk/log"); diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2Runner.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2Runner.java index c101358cd9f7..e2f236fa0eb6 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2Runner.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2Runner.java @@ -61,11 +61,14 @@ * A {@link PipelineRunner} that executes the operations in the pipeline by first translating them * to a Twister2 Plan and then executing them either locally or on a Twister2 cluster, depending on * the configuration. + * + * @deprecated The support for twister2 is scheduled for removal in Beam 3.0. */ @SuppressWarnings({ "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) "nullness" // TODO(https://github.com/apache/beam/issues/20497) }) +@Deprecated public class Twister2Runner extends PipelineRunner { private static final Logger LOG = Logger.getLogger(Twister2Runner.class.getName()); diff --git a/sdks/go.mod b/sdks/go.mod index 76c94686e8e1..62627fd5d2a2 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -25,20 +25,20 @@ go 1.23.0 toolchain go1.24.4 require ( - cloud.google.com/go/bigquery v1.69.0 - cloud.google.com/go/bigtable v1.38.0 + cloud.google.com/go/bigquery v1.70.0 + cloud.google.com/go/bigtable v1.39.0 cloud.google.com/go/datastore v1.20.0 cloud.google.com/go/profiler v0.4.3 - cloud.google.com/go/pubsub v1.49.0 - cloud.google.com/go/spanner v1.83.0 - cloud.google.com/go/storage v1.55.0 - github.com/aws/aws-sdk-go-v2 v1.36.6 - github.com/aws/aws-sdk-go-v2/config v1.29.18 - github.com/aws/aws-sdk-go-v2/credentials v1.17.71 - github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.17.84 - github.com/aws/aws-sdk-go-v2/service/s3 v1.84.1 - github.com/aws/smithy-go v1.22.5 - github.com/docker/go-connections v0.5.0 + cloud.google.com/go/pubsub v1.50.0 + cloud.google.com/go/spanner v1.85.0 + cloud.google.com/go/storage v1.56.1 + github.com/aws/aws-sdk-go-v2 v1.38.3 + github.com/aws/aws-sdk-go-v2/config v1.31.6 + github.com/aws/aws-sdk-go-v2/credentials v1.18.10 + github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.19.2 + github.com/aws/aws-sdk-go-v2/service/s3 v1.87.3 + github.com/aws/smithy-go v1.23.0 + github.com/docker/go-connections v0.6.0 github.com/dustin/go-humanize v1.0.1 github.com/go-sql-driver/mysql v1.9.3 github.com/google/go-cmp v0.7.0 @@ -47,23 +47,23 @@ require ( github.com/lib/pq v1.10.9 github.com/linkedin/goavro/v2 v2.14.0 github.com/nats-io/nats-server/v2 v2.11.6 - github.com/nats-io/nats.go v1.43.0 + github.com/nats-io/nats.go v1.45.0 github.com/proullon/ramsql v0.1.4 - github.com/spf13/cobra v1.9.1 + github.com/spf13/cobra v1.10.1 github.com/testcontainers/testcontainers-go v0.38.0 github.com/tetratelabs/wazero v1.9.0 github.com/xitongsys/parquet-go v1.6.2 github.com/xitongsys/parquet-go-source v0.0.0-20241021075129-b732d2ac9c9b go.mongodb.org/mongo-driver v1.17.4 - golang.org/x/net v0.42.0 + golang.org/x/net v0.43.0 golang.org/x/oauth2 v0.30.0 golang.org/x/sync v0.16.0 - golang.org/x/sys v0.34.0 - golang.org/x/text v0.27.0 - google.golang.org/api v0.243.0 + golang.org/x/sys v0.35.0 + golang.org/x/text v0.28.0 + google.golang.org/api v0.248.0 google.golang.org/genproto v0.0.0-20250603155806-513f23925822 - google.golang.org/grpc v1.73.0 - google.golang.org/protobuf v1.36.6 + google.golang.org/grpc v1.75.0 + google.golang.org/protobuf v1.36.8 gopkg.in/yaml.v2 v2.4.0 gopkg.in/yaml.v3 v3.0.1 ) @@ -76,27 +76,28 @@ require ( ) require ( - cel.dev/expr v0.23.1 // indirect - cloud.google.com/go/auth v0.16.3 // indirect + cel.dev/expr v0.24.0 // indirect + cloud.google.com/go/auth v0.16.5 // indirect cloud.google.com/go/auth/oauth2adapt v0.2.8 // indirect cloud.google.com/go/monitoring v1.24.2 // indirect + cloud.google.com/go/pubsub/v2 v2.0.0 // indirect dario.cat/mergo v1.0.1 // indirect filippo.io/edwards25519 v1.1.0 // indirect github.com/GoogleCloudPlatform/grpc-gcp-go/grpcgcp v1.5.3 // indirect - github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.27.0 // indirect - github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/metric v0.51.0 // indirect - github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/resourcemapping v0.51.0 // indirect + github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.29.0 // indirect + github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/metric v0.53.0 // indirect + github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/resourcemapping v0.53.0 // indirect github.com/apache/arrow/go/v15 v15.0.2 // indirect github.com/containerd/errdefs v1.0.0 // indirect github.com/containerd/errdefs/pkg v0.3.0 // indirect github.com/containerd/log v0.1.0 // indirect github.com/containerd/platforms v0.2.1 // indirect - github.com/davecgh/go-spew v1.1.1 // indirect + github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect github.com/distribution/reference v0.6.0 // indirect github.com/ebitengine/purego v0.8.4 // indirect github.com/envoyproxy/go-control-plane/envoy v1.32.4 // indirect - github.com/go-jose/go-jose/v4 v4.0.5 // indirect - github.com/go-logr/logr v1.4.2 // indirect + github.com/go-jose/go-jose/v4 v4.1.1 // indirect + github.com/go-logr/logr v1.4.3 // indirect github.com/go-logr/stdr v1.2.2 // indirect github.com/go-ole/go-ole v1.3.0 // indirect github.com/google/go-tpm v0.9.5 // indirect @@ -110,7 +111,7 @@ require ( github.com/nats-io/nkeys v0.4.11 // indirect github.com/nats-io/nuid v1.0.1 // indirect github.com/planetscale/vtprotobuf v0.6.1-0.20240319094008-0393e58bdf10 // indirect - github.com/pmezard/go-difflib v1.0.0 // indirect + github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect github.com/power-devops/perfstat v0.0.0-20240221224432-82ca36839d55 // indirect github.com/shirou/gopsutil/v4 v4.25.5 // indirect github.com/spiffe/go-spiffe/v2 v2.5.0 // indirect @@ -119,24 +120,24 @@ require ( github.com/tklauser/numcpus v0.9.0 // indirect github.com/yusufpapurcu/wmi v1.2.4 // indirect github.com/zeebo/errs v1.4.0 // indirect - go.einride.tech/aip v0.68.1 // indirect + go.einride.tech/aip v0.73.0 // indirect go.opentelemetry.io/auto/sdk v1.1.0 // indirect go.opentelemetry.io/contrib/detectors/gcp v1.36.0 // indirect go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.61.0 // indirect go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.61.0 // indirect - go.opentelemetry.io/otel v1.36.0 // indirect + go.opentelemetry.io/otel v1.37.0 // indirect go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracehttp v1.33.0 // indirect - go.opentelemetry.io/otel/metric v1.36.0 // indirect - go.opentelemetry.io/otel/sdk v1.36.0 // indirect - go.opentelemetry.io/otel/sdk/metric v1.36.0 // indirect - go.opentelemetry.io/otel/trace v1.36.0 // indirect + go.opentelemetry.io/otel/metric v1.37.0 // indirect + go.opentelemetry.io/otel/sdk v1.37.0 // indirect + go.opentelemetry.io/otel/sdk/metric v1.37.0 // indirect + go.opentelemetry.io/otel/trace v1.37.0 // indirect go.shabbyrobe.org/gocovmerge v0.0.0-20230507111327-fa4f82cfbf4d // indirect golang.org/x/time v0.12.0 // indirect ) require ( - cloud.google.com/go v0.121.2 // indirect - cloud.google.com/go/compute/metadata v0.7.0 // indirect + cloud.google.com/go v0.121.6 // indirect + cloud.google.com/go/compute/metadata v0.8.0 // indirect cloud.google.com/go/iam v1.5.2 // indirect cloud.google.com/go/longrunning v0.6.7 // indirect github.com/Azure/go-ansiterm v0.0.0-20250102033503-faa5f7b0171c // indirect @@ -144,24 +145,24 @@ require ( github.com/apache/arrow/go/arrow v0.0.0-20211112161151-bc219186db40 // indirect github.com/apache/thrift v0.21.0 // indirect github.com/aws/aws-sdk-go v1.55.5 // indirect - github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.6.11 // indirect - github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.33 // indirect - github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.37 // indirect - github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.37 // indirect + github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.7.1 // indirect + github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.18.6 // indirect + github.com/aws/aws-sdk-go-v2/internal/configsources v1.4.6 // indirect + github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.7.6 // indirect github.com/aws/aws-sdk-go-v2/internal/ini v1.8.3 // indirect - github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.37 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.12.4 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.7.5 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.12.18 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.18.18 // indirect - github.com/aws/aws-sdk-go-v2/service/sso v1.25.6 // indirect - github.com/aws/aws-sdk-go-v2/service/ssooidc v1.30.4 // indirect - github.com/aws/aws-sdk-go-v2/service/sts v1.34.1 // indirect + github.com/aws/aws-sdk-go-v2/internal/v4a v1.4.6 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.13.1 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.8.6 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.13.6 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.19.6 // indirect + github.com/aws/aws-sdk-go-v2/service/sso v1.29.1 // indirect + github.com/aws/aws-sdk-go-v2/service/ssooidc v1.34.2 // indirect + github.com/aws/aws-sdk-go-v2/service/sts v1.38.2 // indirect github.com/cenkalti/backoff/v4 v4.3.0 // indirect github.com/cespare/xxhash/v2 v2.3.0 // indirect - github.com/cncf/xds/go v0.0.0-20250326154945-ae57f3c0d45f // indirect + github.com/cncf/xds/go v0.0.0-20250501225837-2ac532fd4443 // indirect github.com/cpuguy83/dockercfg v0.3.2 // indirect - github.com/docker/docker v28.3.2+incompatible // but required to resolve issue docker has with go1.20 + github.com/docker/docker v28.3.3+incompatible // but required to resolve issue docker has with go1.20 github.com/docker/go-units v0.5.0 // indirect github.com/envoyproxy/protoc-gen-validate v1.2.1 // indirect github.com/felixge/httpsnoop v1.0.4 // indirect @@ -193,17 +194,17 @@ require ( github.com/pkg/xattr v0.4.10 // indirect github.com/ryszard/goskiplist v0.0.0-20150312221310-2dfbae5fcf46 // indirect github.com/sirupsen/logrus v1.9.3 // indirect - github.com/spf13/pflag v1.0.6 // indirect + github.com/spf13/pflag v1.0.9 // indirect github.com/xdg-go/pbkdf2 v1.0.0 // indirect github.com/xdg-go/scram v1.1.2 // indirect github.com/xdg-go/stringprep v1.0.4 // indirect github.com/youmark/pkcs8 v0.0.0-20240726163527-a2c0da244d78 // indirect github.com/zeebo/xxh3 v1.0.2 // indirect go.opencensus.io v0.24.0 // indirect - golang.org/x/crypto v0.40.0 // indirect - golang.org/x/mod v0.25.0 // indirect - golang.org/x/tools v0.34.0 // indirect + golang.org/x/crypto v0.41.0 // indirect + golang.org/x/mod v0.26.0 // indirect + golang.org/x/tools v0.35.0 // indirect golang.org/x/xerrors v0.0.0-20240903120638-7835f813f4da // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20250603155806-513f23925822 // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20250715232539-7130f93afb79 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20250818200422-3122310a409c // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20250818200422-3122310a409c // indirect ) diff --git a/sdks/go.sum b/sdks/go.sum index ee1dfcbbcc86..12516b0b7999 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -1,5 +1,5 @@ -cel.dev/expr v0.23.1 h1:K4KOtPCJQjVggkARsjG9RWXP6O4R73aHeJMa/dmCQQg= -cel.dev/expr v0.23.1/go.mod h1:hLPLo1W4QUmuYdA72RBX06QTs6MXw941piREPl3Yfiw= +cel.dev/expr v0.24.0 h1:56OvJKSH3hDGL0ml5uSxZmz3/3Pq4tJ+fb1unVLAFcY= +cel.dev/expr v0.24.0/go.mod h1:hLPLo1W4QUmuYdA72RBX06QTs6MXw941piREPl3Yfiw= cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSRU= @@ -40,8 +40,8 @@ cloud.google.com/go v0.104.0/go.mod h1:OO6xxXdJyvuJPcEPBLN9BJPD+jep5G1+2U5B5gkRY cloud.google.com/go v0.105.0/go.mod h1:PrLgOJNe5nfE9UMxKxgXj4mD3voiP+YQ6gdt6KMFOKM= cloud.google.com/go v0.107.0/go.mod h1:wpc2eNrD7hXUTy8EKS10jkxpZBjASrORK7goS+3YX2I= cloud.google.com/go v0.110.0/go.mod h1:SJnCLqQ0FCFGSZMUNUf84MV3Aia54kn7pi8st7tMzaY= -cloud.google.com/go v0.121.2 h1:v2qQpN6Dx9x2NmwrqlesOt3Ys4ol5/lFZ6Mg1B7OJCg= -cloud.google.com/go v0.121.2/go.mod h1:nRFlrHq39MNVWu+zESP2PosMWA0ryJw8KUBZ2iZpxbw= +cloud.google.com/go v0.121.6 h1:waZiuajrI28iAf40cWgycWNgaXPO06dupuS+sgibK6c= +cloud.google.com/go v0.121.6/go.mod h1:coChdst4Ea5vUpiALcYKXEpR1S9ZgXbhEzzMcMR66vI= cloud.google.com/go/accessapproval v1.4.0/go.mod h1:zybIuC3KpDOvotz59lFe5qxRZx6C75OtwbisN56xYB4= cloud.google.com/go/accessapproval v1.5.0/go.mod h1:HFy3tuiGvMdcd/u+Cu5b9NkO1pEICJ46IR82PoUdplw= cloud.google.com/go/accessapproval v1.6.0/go.mod h1:R0EiYnwV5fsRFiKZkPHr6mwyk2wxUJ30nL4j2pcFY2E= @@ -103,8 +103,8 @@ cloud.google.com/go/assuredworkloads v1.7.0/go.mod h1:z/736/oNmtGAyU47reJgGN+KVo cloud.google.com/go/assuredworkloads v1.8.0/go.mod h1:AsX2cqyNCOvEQC8RMPnoc0yEarXQk6WEKkxYfL6kGIo= cloud.google.com/go/assuredworkloads v1.9.0/go.mod h1:kFuI1P78bplYtT77Tb1hi0FMxM0vVpRC7VVoJC3ZoT0= cloud.google.com/go/assuredworkloads v1.10.0/go.mod h1:kwdUQuXcedVdsIaKgKTp9t0UJkE5+PAVNhdQm4ZVq2E= -cloud.google.com/go/auth v0.16.3 h1:kabzoQ9/bobUmnseYnBO6qQG7q4a/CffFRlJSxv2wCc= -cloud.google.com/go/auth v0.16.3/go.mod h1:NucRGjaXfzP1ltpcQ7On/VTZ0H4kWB5Jy+Y9Dnm76fA= +cloud.google.com/go/auth v0.16.5 h1:mFWNQ2FEVWAliEQWpAdH80omXFokmrnbDhUS9cBywsI= +cloud.google.com/go/auth v0.16.5/go.mod h1:utzRfHMP+Vv0mpOkTRQoWD2q3BatTOoWbA7gCc2dUhQ= cloud.google.com/go/auth/oauth2adapt v0.2.8 h1:keo8NaayQZ6wimpNSmW5OPc283g65QNIiLpZnkHRbnc= cloud.google.com/go/auth/oauth2adapt v0.2.8/go.mod h1:XQ9y31RkqZCcwJWNSx2Xvric3RrU88hAYYbjDWYDL+c= cloud.google.com/go/automl v1.5.0/go.mod h1:34EjfoFGMZ5sgJ9EoLsRtdPSNZLcfflJR39VbVNS2M0= @@ -135,10 +135,10 @@ cloud.google.com/go/bigquery v1.47.0/go.mod h1:sA9XOgy0A8vQK9+MWhEQTY6Tix87M/Zur cloud.google.com/go/bigquery v1.48.0/go.mod h1:QAwSz+ipNgfL5jxiaK7weyOhzdoAy1zFm0Nf1fysJac= cloud.google.com/go/bigquery v1.49.0/go.mod h1:Sv8hMmTFFYBlt/ftw2uN6dFdQPzBlREY9yBh7Oy7/4Q= cloud.google.com/go/bigquery v1.50.0/go.mod h1:YrleYEh2pSEbgTBZYMJ5SuSr0ML3ypjRB1zgf7pvQLU= -cloud.google.com/go/bigquery v1.69.0 h1:rZvHnjSUs5sHK3F9awiuFk2PeOaB8suqNuim21GbaTc= -cloud.google.com/go/bigquery v1.69.0/go.mod h1:TdGLquA3h/mGg+McX+GsqG9afAzTAcldMjqhdjHTLew= -cloud.google.com/go/bigtable v1.38.0 h1:L/PnUXRtAzFfa7qMULJHt4cXa/O2dqPJEkzYNGA4hfo= -cloud.google.com/go/bigtable v1.38.0/go.mod h1:o/lntJarF3Y5C0XYLMJLjLYwxaRbcrtM0BiV57ymXbI= +cloud.google.com/go/bigquery v1.70.0 h1:V1OIhhOSionCOXWMmypXOvZu/ogkzosa7s1ArWJO/Yg= +cloud.google.com/go/bigquery v1.70.0/go.mod h1:6lEAkgTJN+H2JcaX1eKiuEHTKyqBaJq5U3SpLGbSvwI= +cloud.google.com/go/bigtable v1.39.0 h1:NF0aaSend+Z5CKND2vWY9fgDwaeZ4bDgzUdgw8rk75Y= +cloud.google.com/go/bigtable v1.39.0/go.mod h1:zgL2Vxux9Bx+TcARDJDUxVyE+BCUfP2u4Zm9qeHF+g0= cloud.google.com/go/billing v1.4.0/go.mod h1:g9IdKBEFlItS8bTtlrZdVLWSSdSyFUZKXNS02zKMOZY= cloud.google.com/go/billing v1.5.0/go.mod h1:mztb1tBc3QekhjSgmpf/CV4LzWXLzCArwpLmP2Gm88s= cloud.google.com/go/billing v1.6.0/go.mod h1:WoXzguj+BeHXPbKfNWkqVtDdzORazmCjraY+vrxcyvI= @@ -191,8 +191,8 @@ cloud.google.com/go/compute/metadata v0.1.0/go.mod h1:Z1VN+bulIf6bt4P/C37K4DyZYZ cloud.google.com/go/compute/metadata v0.2.0/go.mod h1:zFmK7XCadkQkj6TtorcaGlCW1hT1fIilQDwofLpJ20k= cloud.google.com/go/compute/metadata v0.2.1/go.mod h1:jgHgmJd2RKBGzXqF5LR2EZMGxBkeanZ9wwa75XHJgOM= cloud.google.com/go/compute/metadata v0.2.3/go.mod h1:VAV5nSsACxMJvgaAuX6Pk2AawlZn8kiOGuCv6gTkwuA= -cloud.google.com/go/compute/metadata v0.7.0 h1:PBWF+iiAerVNe8UCHxdOt6eHLVc3ydFeOCw78U8ytSU= -cloud.google.com/go/compute/metadata v0.7.0/go.mod h1:j5MvL9PprKL39t166CoB1uVHfQMs4tFQZZcKwksXUjo= +cloud.google.com/go/compute/metadata v0.8.0 h1:HxMRIbao8w17ZX6wBnjhcDkW6lTFpgcaobyVfZWqRLA= +cloud.google.com/go/compute/metadata v0.8.0/go.mod h1:sYOGTp851OV9bOFJ9CH7elVvyzopvWQFNNghtDQ/Biw= cloud.google.com/go/contactcenterinsights v1.3.0/go.mod h1:Eu2oemoePuEFc/xKFPjbTuPSj0fYJcPls9TFlPNnHHY= cloud.google.com/go/contactcenterinsights v1.4.0/go.mod h1:L2YzkGbPsv+vMQMCADxJoT9YiTTnSEd6fEvCeHTYVck= cloud.google.com/go/contactcenterinsights v1.6.0/go.mod h1:IIDlT6CLcDoyv79kDv8iWxMSTZhLxSCofVV5W6YFM/w= @@ -460,8 +460,10 @@ cloud.google.com/go/pubsub v1.26.0/go.mod h1:QgBH3U/jdJy/ftjPhTkyXNj543Tin1pRYcd cloud.google.com/go/pubsub v1.27.1/go.mod h1:hQN39ymbV9geqBnfQq6Xf63yNhUAhv9CZhzp5O6qsW0= cloud.google.com/go/pubsub v1.28.0/go.mod h1:vuXFpwaVoIPQMGXqRyUQigu/AX1S3IWugR9xznmcXX8= cloud.google.com/go/pubsub v1.30.0/go.mod h1:qWi1OPS0B+b5L+Sg6Gmc9zD1Y+HaM0MdUr7LsupY1P4= -cloud.google.com/go/pubsub v1.49.0 h1:5054IkbslnrMCgA2MAEPcsN3Ky+AyMpEZcii/DoySPo= -cloud.google.com/go/pubsub v1.49.0/go.mod h1:K1FswTWP+C1tI/nfi3HQecoVeFvL4HUOB1tdaNXKhUY= +cloud.google.com/go/pubsub v1.50.0 h1:hnYpOIxVlgVD1Z8LN7est4DQZK3K6tvZNurZjIVjUe0= +cloud.google.com/go/pubsub v1.50.0/go.mod h1:Di2Y+nqXBpIS+dXUEJPQzLh8PbIQZMLE9IVUFhf2zmM= +cloud.google.com/go/pubsub/v2 v2.0.0 h1:0qS6mRJ41gD1lNmM/vdm6bR7DQu6coQcVwD+VPf0Bz0= +cloud.google.com/go/pubsub/v2 v2.0.0/go.mod h1:0aztFxNzVQIRSZ8vUr79uH2bS3jwLebwK6q1sgEub+E= cloud.google.com/go/pubsublite v1.5.0/go.mod h1:xapqNQ1CuLfGi23Yda/9l4bBCKz/wC3KIJ5gKcxveZg= cloud.google.com/go/pubsublite v1.6.0/go.mod h1:1eFCS0U11xlOuMFV/0iBqw3zP12kddMeCbj/F3FSj9k= cloud.google.com/go/pubsublite v1.7.0/go.mod h1:8hVMwRXfDfvGm3fahVbtDbiLePT3gpoiJYJY+vxWxVM= @@ -552,8 +554,8 @@ cloud.google.com/go/shell v1.6.0/go.mod h1:oHO8QACS90luWgxP3N9iZVuEiSF84zNyLytb+ cloud.google.com/go/spanner v1.41.0/go.mod h1:MLYDBJR/dY4Wt7ZaMIQ7rXOTLjYrmxLE/5ve9vFfWos= cloud.google.com/go/spanner v1.44.0/go.mod h1:G8XIgYdOK+Fbcpbs7p2fiprDw4CaZX63whnSMLVBxjk= cloud.google.com/go/spanner v1.45.0/go.mod h1:FIws5LowYz8YAE1J8fOS7DJup8ff7xJeetWEo5REA2M= -cloud.google.com/go/spanner v1.83.0 h1:AH3QIoSIa01l3WbeTppkwCEYFNK1AER6drcYhPmwhxY= -cloud.google.com/go/spanner v1.83.0/go.mod h1:QSWcjxszT0WRHNd8zyGI0WctrYA1N7j0yTFsWyol9Yw= +cloud.google.com/go/spanner v1.85.0 h1:VVO3yW+0+Yx9tg4SQaZvJHGAnU6qCnGXQ3NX4E3+src= +cloud.google.com/go/spanner v1.85.0/go.mod h1:9zhmtOEoYV06nE4Orbin0dc/ugHzZW9yXuvaM61rpxs= cloud.google.com/go/speech v1.6.0/go.mod h1:79tcr4FHCimOp56lwC01xnt/WPJZc4v3gzyT7FoBkCM= cloud.google.com/go/speech v1.7.0/go.mod h1:KptqL+BAQIhMsj1kOP2la5DSEEerPDuOP/2mmkhHhZQ= cloud.google.com/go/speech v1.8.0/go.mod h1:9bYIl1/tjsAnMgKGHKmBZzXKEkGgtU+MpdDPTE9f7y0= @@ -573,8 +575,8 @@ cloud.google.com/go/storage v1.23.0/go.mod h1:vOEEDNFnciUMhBeT6hsJIn3ieU5cFRmzeL cloud.google.com/go/storage v1.27.0/go.mod h1:x9DOL8TK/ygDUMieqwfhdpQryTeEkhGKMi80i/iqR2s= cloud.google.com/go/storage v1.28.1/go.mod h1:Qnisd4CqDdo6BGs2AD5LLnEsmSQ80wQ5ogcBBKhU86Y= cloud.google.com/go/storage v1.29.0/go.mod h1:4puEjyTKnku6gfKoTfNOU/W+a9JyuVNxjpS5GBrB8h4= -cloud.google.com/go/storage v1.55.0 h1:NESjdAToN9u1tmhVqhXCaCwYBuvEhZLLv0gBr+2znf0= -cloud.google.com/go/storage v1.55.0/go.mod h1:ztSmTTwzsdXe5syLVS0YsbFxXuvEmEyZj7v7zChEmuY= +cloud.google.com/go/storage v1.56.1 h1:n6gy+yLnHn0hTwBFzNn8zJ1kqWfR91wzdM8hjRF4wP0= +cloud.google.com/go/storage v1.56.1/go.mod h1:C9xuCZgFl3buo2HZU/1FncgvvOgTAs/rnh4gF4lMg0s= cloud.google.com/go/storagetransfer v1.5.0/go.mod h1:dxNzUopWy7RQevYFHewchb29POFv3/AaBgnhqzqiK0w= cloud.google.com/go/storagetransfer v1.6.0/go.mod h1:y77xm4CQV/ZhFZH75PLEXY0ROiS7Gh6pSKrM8dJyg6I= cloud.google.com/go/storagetransfer v1.7.0/go.mod h1:8Giuj1QNb1kfLAiWM1bN6dHzfdlDAVC9rv9abHot2W4= @@ -703,14 +705,14 @@ github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym github.com/GoogleCloudPlatform/cloudsql-proxy v1.29.0/go.mod h1:spvB9eLJH9dutlbPSRmHvSXXHOwGRyeXh1jVdquA2G8= github.com/GoogleCloudPlatform/grpc-gcp-go/grpcgcp v1.5.3 h1:2afWGsMzkIcN8Qm4mgPJKZWyroE5QBszMiDMYEBrnfw= github.com/GoogleCloudPlatform/grpc-gcp-go/grpcgcp v1.5.3/go.mod h1:dppbR7CwXD4pgtV9t3wD1812RaLDcBjtblcDF5f1vI0= -github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.27.0 h1:ErKg/3iS1AKcTkf3yixlZ54f9U1rljCkQyEXWUnIUxc= -github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.27.0/go.mod h1:yAZHSGnqScoU556rBOVkwLze6WP5N+U11RHuWaGVxwY= -github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/metric v0.51.0 h1:fYE9p3esPxA/C0rQ0AHhP0drtPXDRhaWiwg1DPqO7IU= -github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/metric v0.51.0/go.mod h1:BnBReJLvVYx2CS/UHOgVz2BXKXD9wsQPxZug20nZhd0= -github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/cloudmock v0.51.0 h1:OqVGm6Ei3x5+yZmSJG1Mh2NwHvpVmZ08CB5qJhT9Nuk= -github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/cloudmock v0.51.0/go.mod h1:SZiPHWGOOk3bl8tkevxkoiwPgsIl6CwrWcbwjfHZpdM= -github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/resourcemapping v0.51.0 h1:6/0iUd0xrnX7qt+mLNRwg5c0PGv8wpE8K90ryANQwMI= -github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/resourcemapping v0.51.0/go.mod h1:otE2jQekW/PqXk1Awf5lmfokJx4uwuqcj1ab5SpGeW0= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.29.0 h1:UQUsRi8WTzhZntp5313l+CHIAT95ojUI2lpP/ExlZa4= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.29.0/go.mod h1:Cz6ft6Dkn3Et6l2v2a9/RpN7epQ1GtDlO6lj8bEcOvw= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/metric v0.53.0 h1:owcC2UnmsZycprQ5RfRgjydWhuoxg71LUfyiQdijZuM= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/metric v0.53.0/go.mod h1:ZPpqegjbE99EPKsu3iUWV22A04wzGPcAY/ziSIQEEgs= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/cloudmock v0.53.0 h1:4LP6hvB4I5ouTbGgWtixJhgED6xdf67twf9PoY96Tbg= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/cloudmock v0.53.0/go.mod h1:jUZ5LYlw40WMd07qxcQJD5M40aUxrfwqQX1g7zxYnrQ= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/resourcemapping v0.53.0 h1:Ron4zCA/yk6U7WOBXhTJcDpsUBG9npumK6xw2auFltQ= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/resourcemapping v0.53.0/go.mod h1:cSgYe11MCNYunTnRXrKiR/tHc0eoKjICUuWpNZoVCOo= github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c/go.mod h1:X0CRv0ky0k6m906ixxpzmDRLvX58TFUKS2eePweuyxk= github.com/Masterminds/semver/v3 v3.1.1/go.mod h1:VPu/7SZ7ePZ3QOrcuXROw5FAcLl4a0cBrbBpGY/8hQs= github.com/Microsoft/go-winio v0.6.2 h1:F2VQgta7ecxGYO8k3ZZz3RS8fVIXVxONVUPlNERoyfY= @@ -747,83 +749,83 @@ github.com/aws/aws-sdk-go v1.55.5 h1:KKUZBfBoyqy5d3swXyiC7Q76ic40rYcbqH7qjh59kzU github.com/aws/aws-sdk-go v1.55.5/go.mod h1:eRwEWoyTWFMVYVQzKMNHWP5/RV4xIUGMQfXQHfHkpNU= github.com/aws/aws-sdk-go-v2 v1.16.2/go.mod h1:ytwTPBG6fXTZLxxeeCCWj2/EMYp/xDUgX+OET6TLNNU= github.com/aws/aws-sdk-go-v2 v1.23.0/go.mod h1:i1XDttT4rnf6vxc9AuskLc6s7XBee8rlLilKlc03uAA= -github.com/aws/aws-sdk-go-v2 v1.36.6 h1:zJqGjVbRdTPojeCGWn5IR5pbJwSQSBh5RWFTQcEQGdU= -github.com/aws/aws-sdk-go-v2 v1.36.6/go.mod h1:EYrzvCCN9CMUTa5+6lf6MM4tq3Zjp8UhSGR/cBsjai0= +github.com/aws/aws-sdk-go-v2 v1.38.3 h1:B6cV4oxnMs45fql4yRH+/Po/YU+597zgWqvDpYMturk= +github.com/aws/aws-sdk-go-v2 v1.38.3/go.mod h1:sDioUELIUO9Znk23YVmIk86/9DOpkbyyVb1i/gUNFXY= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.1/go.mod h1:n8Bs1ElDD2wJ9kCRTczA83gYbBmjSwZp3umc6zF4EeM= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.1/go.mod h1:t8PYl/6LzdAqsU4/9tz28V/kU+asFePvpOMkdul0gEQ= -github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.6.11 h1:12SpdwU8Djs+YGklkinSSlcrPyj3H4VifVsKf78KbwA= -github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.6.11/go.mod h1:dd+Lkp6YmMryke+qxW/VnKyhMBDTYP41Q2Bb+6gNZgY= +github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.7.1 h1:i8p8P4diljCr60PpJp6qZXNlgX4m2yQFpYk+9ZT+J4E= +github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.7.1/go.mod h1:ddqbooRZYNoJ2dsTwOty16rM+/Aqmk/GOXrK8cg7V00= github.com/aws/aws-sdk-go-v2/config v1.15.3/go.mod h1:9YL3v07Xc/ohTsxFXzan9ZpFpdTOFl4X65BAKYaz8jg= github.com/aws/aws-sdk-go-v2/config v1.25.3/go.mod h1:tAByZy03nH5jcq0vZmkcVoo6tRzRHEwSFx3QW4NmDw8= -github.com/aws/aws-sdk-go-v2/config v1.29.18 h1:x4T1GRPnqKV8HMJOMtNktbpQMl3bIsfx8KbqmveUO2I= -github.com/aws/aws-sdk-go-v2/config v1.29.18/go.mod h1:bvz8oXugIsH8K7HLhBv06vDqnFv3NsGDt2Znpk7zmOU= +github.com/aws/aws-sdk-go-v2/config v1.31.6 h1:a1t8fXY4GT4xjyJExz4knbuoxSCacB5hT/WgtfPyLjo= +github.com/aws/aws-sdk-go-v2/config v1.31.6/go.mod h1:5ByscNi7R+ztvOGzeUaIu49vkMk2soq5NaH5PYe33MQ= github.com/aws/aws-sdk-go-v2/credentials v1.11.2/go.mod h1:j8YsY9TXTm31k4eFhspiQicfXPLZ0gYXA50i4gxPE8g= github.com/aws/aws-sdk-go-v2/credentials v1.16.2/go.mod h1:sDdvGhXrSVT5yzBDR7qXz+rhbpiMpUYfF3vJ01QSdrc= -github.com/aws/aws-sdk-go-v2/credentials v1.17.71 h1:r2w4mQWnrTMJjOyIsZtGp3R3XGY3nqHn8C26C2lQWgA= -github.com/aws/aws-sdk-go-v2/credentials v1.17.71/go.mod h1:E7VF3acIup4GB5ckzbKFrCK0vTvEQxOxgdq4U3vcMCY= +github.com/aws/aws-sdk-go-v2/credentials v1.18.10 h1:xdJnXCouCx8Y0NncgoptztUocIYLKeQxrCgN6x9sdhg= +github.com/aws/aws-sdk-go-v2/credentials v1.18.10/go.mod h1:7tQk08ntj914F/5i9jC4+2HQTAuJirq7m1vZVIhEkWs= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.12.3/go.mod h1:uk1vhHHERfSVCUnqSqz8O48LBYDSC+k6brng09jcMOk= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.4/go.mod h1:t4i+yGHMCcUNIX1x7YVYa6bH/Do7civ5I6cG/6PMfyA= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.33 h1:D9ixiWSG4lyUBL2DDNK924Px9V/NBVpML90MHqyTADY= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.16.33/go.mod h1:caS/m4DI+cij2paz3rtProRBI4s/+TCiWoaWZuQ9010= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.18.6 h1:wbjnrrMnKew78/juW7I2BtKQwa1qlf6EjQgS69uYY14= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.18.6/go.mod h1:AtiqqNrDioJXuUgz3+3T0mBWN7Hro2n9wll2zRUc0ww= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.3/go.mod h1:0dHuD2HZZSiwfJSy1FO5bX1hQ1TxVV1QXXjpn3XUE44= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.14.0/go.mod h1:UcgIwJ9KHquYxs6Q5skC9qXjhYMK+JASDYcXQ4X7JZE= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.17.84 h1:cTXRdLkpBanlDwISl+5chq5ui1d1YWg4PWMR9c3kXyw= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.17.84/go.mod h1:kwSy5X7tfIHN39uucmjQVs2LvDdXEjQucgQQEqCggEo= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.19.2 h1:eZAl6tdv3HrIHAxbpnDQByEOD84bmxyhLmgvUYJ8ggo= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.19.2/go.mod h1:vV+YS0SWfpwbIGOUWbB5NWklaYKscfYrQRb9ggHptxs= github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.9/go.mod h1:AnVH5pvai0pAF4lXRq0bmhbes1u9R8wTE+g+183bZNM= github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.3/go.mod h1:7sGSz1JCKHWWBHq98m6sMtWQikmYPpxjqOydDemiVoM= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.37 h1:osMWfm/sC/L4tvEdQ65Gri5ZZDCUpuYJZbTTDrsn4I0= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.37/go.mod h1:ZV2/1fbjOPr4G4v38G3Ww5TBT4+hmsK45s/rxu1fGy0= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.4.6 h1:uF68eJA6+S9iVr9WgX1NaRGyQ/6MdIyc4JNUo6TN1FA= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.4.6/go.mod h1:qlPeVZCGPiobx8wb1ft0GHT5l+dc6ldnwInDFaMvC7Y= github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.3/go.mod h1:ssOhaLpRlh88H3UmEcsBoVKq309quMvm3Ds8e9d4eJM= github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.3/go.mod h1:ify42Rb7nKeDDPkFjKn7q1bPscVPu/+gmHH8d2c+anU= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.37 h1:v+X21AvTb2wZ+ycg1gx+orkB/9U6L7AOp93R7qYxsxM= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.37/go.mod h1:G0uM1kyssELxmJ2VZEfG0q2npObR3BAkF3c1VsfVnfs= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.7.6 h1:pa1DEC6JoI0zduhZePp3zmhWvk/xxm4NB8Hy/Tlsgos= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.7.6/go.mod h1:gxEjPebnhWGJoaDdtDkA0JX46VRg1wcTHYe63OfX5pE= github.com/aws/aws-sdk-go-v2/internal/ini v1.3.10/go.mod h1:8DcYQcz0+ZJaSxANlHIsbbi6S+zMwjwdDqwW3r9AzaE= github.com/aws/aws-sdk-go-v2/internal/ini v1.7.1/go.mod h1:6fQQgfuGmw8Al/3M2IgIllycxV7ZW7WCdVSqfBeUiCY= github.com/aws/aws-sdk-go-v2/internal/ini v1.8.3 h1:bIqFDwgGXXN1Kpp99pDOdKMTTb5d2KyU5X/BZxjOkRo= github.com/aws/aws-sdk-go-v2/internal/ini v1.8.3/go.mod h1:H5O/EsxDWyU+LP/V8i5sm8cxoZgc2fdNR9bxlOFrQTo= github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.3/go.mod h1:5yzAuE9i2RkVAttBl8yxZgQr5OCq4D5yDnG7j9x2L0U= -github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.37 h1:XTZZ0I3SZUHAtBLBU6395ad+VOblE0DwQP6MuaNeics= -github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.37/go.mod h1:Pi6ksbniAWVwu2S8pEzcYPyhUkAcLaufxN7PfAUQjBk= +github.com/aws/aws-sdk-go-v2/internal/v4a v1.4.6 h1:R0tNFJqfjHL3900cqhXuwQ+1K4G0xc9Yf8EDbFXCKEw= +github.com/aws/aws-sdk-go-v2/internal/v4a v1.4.6/go.mod h1:y/7sDdu+aJvPtGXr4xYosdpq9a6T9Z0jkXfugmti0rI= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.9.1/go.mod h1:GeUru+8VzrTXV/83XyMJ80KpH8xO89VPoUileyNQ+tc= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.1/go.mod h1:l9ymW25HOqymeU2m1gbUQ3rUIsTwKs8gYHXkqDQUhiI= -github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.12.4 h1:CXV68E2dNqhuynZJPB80bhPQwAKqBWVer887figW6Jc= -github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.12.4/go.mod h1:/xFi9KtvBXP97ppCz1TAEvU1Uf66qvid89rbem3wCzQ= +github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.13.1 h1:oegbebPEMA/1Jny7kvwejowCaHz1FWZAQ94WXFNCyTM= +github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.13.1/go.mod h1:kemo5Myr9ac0U9JfSjMo9yHLtw+pECEHsFtJ9tqCEI8= github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.3/go.mod h1:Seb8KNmD6kVTjwRjVEgOT5hPin6sq+v4C2ycJQDwuH8= github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.3/go.mod h1:R+/S1O4TYpcktbVwddeOYg+uwUfLhADP2S/x4QwsCTM= -github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.7.5 h1:M5/B8JUaCI8+9QD+u3S/f4YHpvqE9RpSkV3rf0Iks2w= -github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.7.5/go.mod h1:Bktzci1bwdbpuLiu3AOksiNPMl/LLKmX1TWmqp2xbvs= +github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.8.6 h1:hncKj/4gR+TPauZgTAsxOxNcvBayhUlYZ6LO/BYiQ30= +github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.8.6/go.mod h1:OiIh45tp6HdJDDJGnja0mw8ihQGz3VGrUflLqSL0SmM= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.3/go.mod h1:wlY6SVjuwvh3TVRpTqdy4I1JpBFLX4UGeKZdWntaocw= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.3/go.mod h1:Owv1I59vaghv1Ax8zz8ELY8DN7/Y0rGS+WWAmjgi950= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.12.18 h1:vvbXsA2TVO80/KT7ZqCbx934dt6PY+vQ8hZpUZ/cpYg= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.12.18/go.mod h1:m2JJHledjBGNMsLOF1g9gbAxprzq3KjC8e4lxtn+eWg= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.13.6 h1:LHS1YAIJXJ4K9zS+1d/xa9JAA9sL2QyXIQCQFQW/X08= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.13.6/go.mod h1:c9PCiTEuh0wQID5/KqA32J+HAgZxN9tOGXKCiYJjTZI= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.13.3/go.mod h1:Bm/v2IaN6rZ+Op7zX+bOUMdL4fsrYZiD0dsjLhNKwZc= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.3/go.mod h1:KZgs2ny8HsxRIRbDwgvJcHHBZPOzQr/+NtGwnP+w2ec= -github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.18.18 h1:OS2e0SKqsU2LiJPqL8u9x41tKc6MMEHrWjLVLn3oysg= -github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.18.18/go.mod h1:+Yrk+MDGzlNGxCXieljNeWpoZTCQUQVL+Jk9hGGJ8qM= +github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.19.6 h1:nEXUSAwyUfLTgnc9cxlDWy637qsq4UWwp3sNAfl0Z3Y= +github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.19.6/go.mod h1:HGzIULx4Ge3Do2V0FaiYKcyKzOqwrhUZgCI77NisswQ= github.com/aws/aws-sdk-go-v2/service/kms v1.16.3/go.mod h1:QuiHPBqlOFCi4LqdSskYYAWpQlx3PKmohy+rE2F+o5g= github.com/aws/aws-sdk-go-v2/service/s3 v1.26.3/go.mod h1:g1qvDuRsJY+XghsV6zg00Z4KJ7DtFFCx8fJD2a491Ak= github.com/aws/aws-sdk-go-v2/service/s3 v1.43.0/go.mod h1:NXRKkiRF+erX2hnybnVU660cYT5/KChRD4iUgJ97cI8= -github.com/aws/aws-sdk-go-v2/service/s3 v1.84.1 h1:RkHXU9jP0DptGy7qKI8CBGsUJruWz0v5IgwBa2DwWcU= -github.com/aws/aws-sdk-go-v2/service/s3 v1.84.1/go.mod h1:3xAOf7tdKF+qbb+XpU+EPhNXAdun3Lu1RcDrj8KC24I= +github.com/aws/aws-sdk-go-v2/service/s3 v1.87.3 h1:ETkfWcXP2KNPLecaDa++5bsQhCRa5M5sLUJa5DWYIIg= +github.com/aws/aws-sdk-go-v2/service/s3 v1.87.3/go.mod h1:+/3ZTqoYb3Ur7DObD00tarKMLMuKg8iqz5CHEanqTnw= github.com/aws/aws-sdk-go-v2/service/secretsmanager v1.15.4/go.mod h1:PJc8s+lxyU8rrre0/4a0pn2wgwiDvOEzoOjcJUBr67o= github.com/aws/aws-sdk-go-v2/service/sns v1.17.4/go.mod h1:kElt+uCcXxcqFyc+bQqZPFD9DME/eC6oHBXvFzQ9Bcw= github.com/aws/aws-sdk-go-v2/service/sqs v1.18.3/go.mod h1:skmQo0UPvsjsuYYSYMVmrPc1HWCbHUJyrCEp+ZaLzqM= github.com/aws/aws-sdk-go-v2/service/ssm v1.24.1/go.mod h1:NR/xoKjdbRJ+qx0pMR4mI+N/H1I1ynHwXnO6FowXJc0= github.com/aws/aws-sdk-go-v2/service/sso v1.11.3/go.mod h1:7UQ/e69kU7LDPtY40OyoHYgRmgfGM4mgsLYtcObdveU= github.com/aws/aws-sdk-go-v2/service/sso v1.17.2/go.mod h1:/pE21vno3q1h4bbhUOEi+6Zu/aT26UK2WKkDXd+TssQ= -github.com/aws/aws-sdk-go-v2/service/sso v1.25.6 h1:rGtWqkQbPk7Bkwuv3NzpE/scwwL9sC1Ul3tn9x83DUI= -github.com/aws/aws-sdk-go-v2/service/sso v1.25.6/go.mod h1:u4ku9OLv4TO4bCPdxf4fA1upaMaJmP9ZijGk3AAOC6Q= +github.com/aws/aws-sdk-go-v2/service/sso v1.29.1 h1:8OLZnVJPvjnrxEwHFg9hVUof/P4sibH+Ea4KKuqAGSg= +github.com/aws/aws-sdk-go-v2/service/sso v1.29.1/go.mod h1:27M3BpVi0C02UiQh1w9nsBEit6pLhlaH3NHna6WUbDE= github.com/aws/aws-sdk-go-v2/service/ssooidc v1.20.0/go.mod h1:dWqm5G767qwKPuayKfzm4rjzFmVjiBFbOJrpSPnAMDs= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.30.4 h1:OV/pxyXh+eMA0TExHEC4jyWdumLxNbzz1P0zJoezkJc= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.30.4/go.mod h1:8Mm5VGYwtm+r305FfPSuc+aFkrypeylGYhFim6XEPoc= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.34.2 h1:gKWSTnqudpo8dAxqBqZnDoDWCiEh/40FziUjr/mo6uA= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.34.2/go.mod h1:x7+rkNmRoEN1U13A6JE2fXne9EWyJy54o3n6d4mGaXQ= github.com/aws/aws-sdk-go-v2/service/sts v1.16.3/go.mod h1:bfBj0iVmsUyUg4weDB4NxktD9rDGeKSVWnjTnwbx9b8= github.com/aws/aws-sdk-go-v2/service/sts v1.25.3/go.mod h1:4EqRHDCKP78hq3zOnmFXu5k0j4bXbRFfCh/zQ6KnEfQ= -github.com/aws/aws-sdk-go-v2/service/sts v1.34.1 h1:aUrLQwJfZtwv3/ZNG2xRtEen+NqI3iesuacjP51Mv1s= -github.com/aws/aws-sdk-go-v2/service/sts v1.34.1/go.mod h1:3wFBZKoWnX3r+Sm7in79i54fBmNfwhdNdQuscCw7QIk= +github.com/aws/aws-sdk-go-v2/service/sts v1.38.2 h1:YZPjhyaGzhDQEvsffDEcpycq49nl7fiGcfJTIo8BszI= +github.com/aws/aws-sdk-go-v2/service/sts v1.38.2/go.mod h1:2dIN8qhQfv37BdUYGgEC8Q3tteM3zFxTI1MLO2O3J3c= github.com/aws/smithy-go v1.11.2/go.mod h1:3xHYmszWVx2c0kIwQeEVf9uSm4fYZt67FBJnwub1bgM= github.com/aws/smithy-go v1.17.0/go.mod h1:NukqUGpCZIILqqiV0NIjeFh24kd/FAa4beRb6nbIUPE= -github.com/aws/smithy-go v1.22.5 h1:P9ATCXPMb2mPjYBgueqJNCA5S9UfktsW0tTxi+a7eqw= -github.com/aws/smithy-go v1.22.5/go.mod h1:t1ufH5HMublsJYulve2RKmHDC15xu1f26kHCp/HgceI= +github.com/aws/smithy-go v1.23.0 h1:8n6I3gXzWJB2DxBDnfxgBaSX6oe0d/t10qGz7OKqMCE= +github.com/aws/smithy-go v1.23.0/go.mod h1:t1ufH5HMublsJYulve2RKmHDC15xu1f26kHCp/HgceI= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/bobg/gcsobj v0.1.2/go.mod h1:vS49EQ1A1Ib8FgrL58C8xXYZyOCR2TgzAdopy6/ipa8= github.com/boombuler/barcode v1.0.0/go.mod h1:paBWMcWSl3LHKBqUq+rly7CNSldXjb2rDl3JlRe0mD8= @@ -857,8 +859,8 @@ github.com/cncf/xds/go v0.0.0-20211011173535-cb28da3451f1/go.mod h1:eXthEFrGJvWH github.com/cncf/xds/go v0.0.0-20220314180256-7f1daf1720fc/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/cncf/xds/go v0.0.0-20230105202645-06c439db220b/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/cncf/xds/go v0.0.0-20230607035331-e9ce68804cb4/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= -github.com/cncf/xds/go v0.0.0-20250326154945-ae57f3c0d45f h1:C5bqEmzEPLsHm9Mv73lSE9e9bKV23aB1vxOsmZrkl3k= -github.com/cncf/xds/go v0.0.0-20250326154945-ae57f3c0d45f/go.mod h1:W+zGtBO5Y1IgJhy4+A9GOqVhqLpfZi+vwmdNXUehLA8= +github.com/cncf/xds/go v0.0.0-20250501225837-2ac532fd4443 h1:aQ3y1lwWyqYPiWZThqv1aFbZMiM9vblcSArJRf2Irls= +github.com/cncf/xds/go v0.0.0-20250501225837-2ac532fd4443/go.mod h1:W+zGtBO5Y1IgJhy4+A9GOqVhqLpfZi+vwmdNXUehLA8= github.com/cockroachdb/apd v1.1.0/go.mod h1:8Sl8LxpKi29FqWXR16WEFZRNSz3SoPzUzeMeY4+DwBQ= github.com/colinmarc/hdfs/v2 v2.1.1/go.mod h1:M3x+k8UKKmxtFu++uAZ0OtDU8jR3jnaZIAc6yK4Ue0c= github.com/containerd/errdefs v1.0.0 h1:tg5yIfIlQIrxYtu9ajqY42W3lpS19XqdxRQeEwYG8PI= @@ -880,8 +882,9 @@ github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ3 github.com/creack/pty v1.1.18 h1:n56/Zwd5o6whRC5PMGretI4IdRLlmBXYNjScPaBgsbY= github.com/creack/pty v1.1.18/go.mod h1:MOBLtS5ELjhRRrroQr9kyvTxUAFNvYEK993ew/Vr4O4= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc h1:U9qPSI2PIWSS1VwoXQT9A3Wy9MM3WgvqSxFWenqJduM= +github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/denisenkom/go-mssqldb v0.12.0/go.mod h1:iiK0YP1ZeepvmBQk/QpLEhhTNJgfzrpArPY/aFvc9yU= github.com/devigned/tab v0.1.1/go.mod h1:XG9mPq0dFghrYvoBF3xdRrJzSTX1b7IQrvaL9mzjeJY= github.com/dimchansky/utfbom v1.1.0/go.mod h1:rO41eb7gLfo8SF1jd9F8HplJm1Fewwi4mQvIirEdv+8= @@ -890,10 +893,10 @@ github.com/distribution/reference v0.6.0 h1:0IXCQ5g4/QMHHkarYzh5l+u8T3t73zM5Qvfr github.com/distribution/reference v0.6.0/go.mod h1:BbU0aIcezP1/5jX/8MP0YiH4SdvB5Y4f/wlDRiLyi3E= github.com/dnaeon/go-vcr v1.1.0/go.mod h1:M7tiix8f0r6mKKJ3Yq/kqU1OYf3MnfmBWVbPx/yU9ko= github.com/dnaeon/go-vcr v1.2.0/go.mod h1:R4UdLID7HZT3taECzJs4YgbbH6PIGXB6W/sc5OLb6RQ= -github.com/docker/docker v28.3.2+incompatible h1:wn66NJ6pWB1vBZIilP8G3qQPqHy5XymfYn5vsqeA5oA= -github.com/docker/docker v28.3.2+incompatible/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk= -github.com/docker/go-connections v0.5.0 h1:USnMq7hx7gwdVZq1L49hLXaFtUdTADjXGp+uj1Br63c= -github.com/docker/go-connections v0.5.0/go.mod h1:ov60Kzw0kKElRwhNs9UlUHAE/F9Fe6GLaXnqyDdmEXc= +github.com/docker/docker v28.3.3+incompatible h1:Dypm25kh4rmk49v1eiVbsAtpAsYURjYkaKubwuBdxEI= +github.com/docker/docker v28.3.3+incompatible/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk= +github.com/docker/go-connections v0.6.0 h1:LlMG9azAe1TqfR7sO+NJttz1gy6KO7VJBh+pMmjSD94= +github.com/docker/go-connections v0.6.0/go.mod h1:AahvXYshr6JgfUJGdDCs2b5EZG/vmaMAntpSFH5BFKE= github.com/docker/go-units v0.5.0 h1:69rxXcBk27SvSaaxTtLh/8llcHD8vYHT7WSdRZ/jvr4= github.com/docker/go-units v0.5.0/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk= github.com/docopt/docopt-go v0.0.0-20180111231733-ee0de3bc6815/go.mod h1:WwZ+bS3ebgob9U8Nd0kOddGdZWjyMGR8Wziv+TBNwSE= @@ -951,15 +954,15 @@ github.com/go-gorp/gorp v2.2.0+incompatible/go.mod h1:7IfkAQnO7jfT/9IQ3R9wL1dFhu github.com/go-ini/ini v1.25.4/go.mod h1:ByCAeIL28uOIIG0E3PJtZPDL8WnHpFKFOtgjp+3Ies8= github.com/go-ini/ini v1.67.0 h1:z6ZrTEZqSWOTyH2FlglNbNgARyHG8oLW9gMELqKr06A= github.com/go-ini/ini v1.67.0/go.mod h1:ByCAeIL28uOIIG0E3PJtZPDL8WnHpFKFOtgjp+3Ies8= -github.com/go-jose/go-jose/v4 v4.0.5 h1:M6T8+mKZl/+fNNuFHvGIzDz7BTLQPIounk/b9dw3AaE= -github.com/go-jose/go-jose/v4 v4.0.5/go.mod h1:s3P1lRrkT8igV8D9OjyL4WRyHvjB6a4JSllnOrmmBOA= +github.com/go-jose/go-jose/v4 v4.1.1 h1:JYhSgy4mXXzAdF3nUx3ygx347LRXJRrpgyU3adRmkAI= +github.com/go-jose/go-jose/v4 v4.1.1/go.mod h1:BdsZGqgdO3b6tTc6LSE56wcDbMMLuPsw5d4ZD5f94kA= github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= github.com/go-latex/latex v0.0.0-20210118124228-b3d85cf34e07/go.mod h1:CO1AlKB2CSIqUrmQPqA0gdRIlnLEY0gK5JGjh37zN5U= github.com/go-latex/latex v0.0.0-20210823091927-c0d11ff05a81/go.mod h1:SX0U8uGpxhq9o2S/CELCSUxEWWAuoCUcVCQWv7G2OCk= github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= -github.com/go-logr/logr v1.4.2 h1:6pFjapn8bFcIbiKo3XT4j/BhANplGihG6tvd+8rYgrY= -github.com/go-logr/logr v1.4.2/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY= +github.com/go-logr/logr v1.4.3 h1:CjnDlHq8ikf6E492q6eKboGOC0T8CDaOvkHCIg8idEI= +github.com/go-logr/logr v1.4.3/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY= github.com/go-logr/stdr v1.2.2 h1:hSWxHoqTgW2S2qGc0LTAI563KZ5YKYRhT3MFKZMbjag= github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre4VKE= github.com/go-ole/go-ole v1.2.6/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= @@ -1325,8 +1328,8 @@ github.com/nats-io/jwt/v2 v2.7.4 h1:jXFuDDxs/GQjGDZGhNgH4tXzSUK6WQi2rsj4xmsNOtI= github.com/nats-io/jwt/v2 v2.7.4/go.mod h1:me11pOkwObtcBNR8AiMrUbtVOUGkqYjMQZ6jnSdVUIA= github.com/nats-io/nats-server/v2 v2.11.6 h1:4VXRjbTUFKEB+7UoaKL3F5Y83xC7MxPoIONOnGgpkHw= github.com/nats-io/nats-server/v2 v2.11.6/go.mod h1:2xoztlcb4lDL5Blh1/BiukkKELXvKQ5Vy29FPVRBUYs= -github.com/nats-io/nats.go v1.43.0 h1:uRFZ2FEoRvP64+UUhaTokyS18XBCR/xM2vQZKO4i8ug= -github.com/nats-io/nats.go v1.43.0/go.mod h1:iRWIPokVIFbVijxuMQq4y9ttaBTMe0SFdlZfMDd+33g= +github.com/nats-io/nats.go v1.45.0 h1:/wGPbnYXDM0pLKFjZTX+2JOw9TQPoIgTFrUaH97giwA= +github.com/nats-io/nats.go v1.45.0/go.mod h1:iRWIPokVIFbVijxuMQq4y9ttaBTMe0SFdlZfMDd+33g= github.com/nats-io/nkeys v0.4.11 h1:q44qGV008kYd9W1b1nEBkNzvnWxtRSQ7A8BoqRrcfa0= github.com/nats-io/nkeys v0.4.11/go.mod h1:szDimtgmfOi9n25JpfIdGw12tZFYXqhGxjhVxsatHVE= github.com/nats-io/nuid v1.0.1 h1:5iA8DT8V7q8WK2EScv2padNa/rTESc1KdnPw4TC2paw= @@ -1357,8 +1360,9 @@ github.com/pkg/xattr v0.4.10 h1:Qe0mtiNFHQZ296vRgUjRCoPHPqH7VdTOrZx3g0T+pGA= github.com/pkg/xattr v0.4.10/go.mod h1:di8WF84zAKk8jzR1UBTEWh9AUlIZZ7M/JNt8e9B6ktU= github.com/planetscale/vtprotobuf v0.6.1-0.20240319094008-0393e58bdf10 h1:GFCKgmp0tecUJ0sJuv4pzYCqS9+RGSn52M3FUwPs+uo= github.com/planetscale/vtprotobuf v0.6.1-0.20240319094008-0393e58bdf10/go.mod h1:t/avpk3KcrXxUnYOhZhMXJlSEyie6gQbtLq5NM3loB8= -github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= +github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 h1:Jamvg5psRIccs7FGNTlIRMkT8wgtp5eCXdBlqhYGL6U= +github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/power-devops/perfstat v0.0.0-20240221224432-82ca36839d55 h1:o4JXh1EVt9k/+g42oCprj/FisM4qX9L3sZB3upGN2ZU= github.com/power-devops/perfstat v0.0.0-20240221224432-82ca36839d55/go.mod h1:OmDBASR4679mdNQnz2pUhc2G8CO2JrUAVFDRBDP/hJE= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= @@ -1400,10 +1404,10 @@ github.com/spf13/afero v1.2.2/go.mod h1:9ZxEEn6pIJ8Rxe320qSDBk6AsU0r9pR7Q4OcevTd github.com/spf13/afero v1.3.3/go.mod h1:5KUK8ByomD5Ti5Artl0RtHeI5pTF7MIDuXL3yY520V4= github.com/spf13/afero v1.6.0/go.mod h1:Ai8FlHk4v/PARR026UzYexafAt9roJ7LcLMAmO6Z93I= github.com/spf13/afero v1.9.2/go.mod h1:iUV7ddyEEZPO5gA3zD4fJt6iStLlL+Lg4m2cihcDf8Y= -github.com/spf13/cobra v1.9.1 h1:CXSaggrXdbHK9CF+8ywj8Amf7PBRmPCOJugH954Nnlo= -github.com/spf13/cobra v1.9.1/go.mod h1:nDyEzZ8ogv936Cinf6g1RU9MRY64Ir93oCnqb9wxYW0= -github.com/spf13/pflag v1.0.6 h1:jFzHGLGAlb3ruxLB8MhbI6A8+AQX/2eW4qeyNZXNp2o= -github.com/spf13/pflag v1.0.6/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= +github.com/spf13/cobra v1.10.1 h1:lJeBwCfmrnXthfAupyUTzJ/J4Nc1RsHC/mSRU2dll/s= +github.com/spf13/cobra v1.10.1/go.mod h1:7SmJGaTHFVBY0jW4NXGluQoLvhqFQM+6XSKD+P4XaB0= +github.com/spf13/pflag v1.0.9 h1:9exaQaMOCwffKiiiYk6/BndUBv+iRViNW+4lEMi0PvY= +github.com/spf13/pflag v1.0.9/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= github.com/spiffe/go-spiffe/v2 v2.5.0 h1:N2I01KCUkv1FAjZXJMwh95KK1ZIQLYbPfhaxw8WS0hE= github.com/spiffe/go-spiffe/v2 v2.5.0/go.mod h1:P+NxobPc6wXhVtINNtFjNWGBTreew1GBUCwT2wPmb7g= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= @@ -1468,8 +1472,8 @@ github.com/zeebo/errs v1.4.0/go.mod h1:sgbWHsvVuTPHcqJJGQ1WhI5KbWlHYz+2+2C/LSEtC github.com/zeebo/xxh3 v1.0.2 h1:xZmwmqxHZA8AI603jOQ0tMqmBr9lPeFwGg6d+xy9DC0= github.com/zeebo/xxh3 v1.0.2/go.mod h1:5NWz9Sef7zIDm2JHfFlcQvNekmcEl9ekUZQQKCYaDcA= github.com/zenazn/goji v0.9.0/go.mod h1:7S9M489iMyHBNxwZnk9/EHS098H4/F6TATF2mIxtB1Q= -go.einride.tech/aip v0.68.1 h1:16/AfSxcQISGN5z9C5lM+0mLYXihrHbQ1onvYTr93aQ= -go.einride.tech/aip v0.68.1/go.mod h1:XaFtaj4HuA3Zwk9xoBtTWgNubZ0ZZXv9BZJCkuKuWbg= +go.einride.tech/aip v0.73.0 h1:bPo4oqBo2ZQeBKo4ZzLb1kxYXTY1ysJhpvQyfuGzvps= +go.einride.tech/aip v0.73.0/go.mod h1:Mj7rFbmXEgw0dq1dqJ7JGMvYCZZVxmGOR3S4ZcV5LvQ= go.etcd.io/bbolt v1.3.5/go.mod h1:G5EMThwa9y8QZGBClrRx5EY+Yw9kAhnjy3bSjsnlVTQ= go.mongodb.org/mongo-driver v1.17.4 h1:jUorfmVzljjr0FLzYQsGP8cgN/qzzxlY9Vh0C9KFXVw= go.mongodb.org/mongo-driver v1.17.4/go.mod h1:Hy04i7O2kC4RS06ZrhPRqj/u4DTYkFDAAccj+rVKqgQ= @@ -1491,22 +1495,22 @@ go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.6 go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.61.0/go.mod h1:snMWehoOh2wsEwnvvwtDyFCxVeDAODenXHtn5vzrKjo= go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.61.0 h1:F7Jx+6hwnZ41NSFTO5q4LYDtJRXBf2PD0rNBkeB/lus= go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.61.0/go.mod h1:UHB22Z8QsdRDrnAtX4PntOl36ajSxcdUMt1sF7Y6E7Q= -go.opentelemetry.io/otel v1.36.0 h1:UumtzIklRBY6cI/lllNZlALOF5nNIzJVb16APdvgTXg= -go.opentelemetry.io/otel v1.36.0/go.mod h1:/TcFMXYjyRNh8khOAO9ybYkqaDBb/70aVwkNML4pP8E= +go.opentelemetry.io/otel v1.37.0 h1:9zhNfelUvx0KBfu/gb+ZgeAfAgtWrfHJZcAqFC228wQ= +go.opentelemetry.io/otel v1.37.0/go.mod h1:ehE/umFRLnuLa/vSccNq9oS1ErUlkkK71gMcN34UG8I= go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.33.0 h1:Vh5HayB/0HHfOQA7Ctx69E/Y/DcQSMPpKANYVMQ7fBA= go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.33.0/go.mod h1:cpgtDBaqD/6ok/UG0jT15/uKjAY8mRA53diogHBg3UI= go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracehttp v1.33.0 h1:wpMfgF8E1rkrT1Z6meFh1NDtownE9Ii3n3X2GJYjsaU= go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracehttp v1.33.0/go.mod h1:wAy0T/dUbs468uOlkT31xjvqQgEVXv58BRFWEgn5v/0= go.opentelemetry.io/otel/exporters/stdout/stdoutmetric v1.36.0 h1:rixTyDGXFxRy1xzhKrotaHy3/KXdPhlWARrCgK+eqUY= go.opentelemetry.io/otel/exporters/stdout/stdoutmetric v1.36.0/go.mod h1:dowW6UsM9MKbJq5JTz2AMVp3/5iW5I/TStsk8S+CfHw= -go.opentelemetry.io/otel/metric v1.36.0 h1:MoWPKVhQvJ+eeXWHFBOPoBOi20jh6Iq2CcCREuTYufE= -go.opentelemetry.io/otel/metric v1.36.0/go.mod h1:zC7Ks+yeyJt4xig9DEw9kuUFe5C3zLbVjV2PzT6qzbs= -go.opentelemetry.io/otel/sdk v1.36.0 h1:b6SYIuLRs88ztox4EyrvRti80uXIFy+Sqzoh9kFULbs= -go.opentelemetry.io/otel/sdk v1.36.0/go.mod h1:+lC+mTgD+MUWfjJubi2vvXWcVxyr9rmlshZni72pXeY= -go.opentelemetry.io/otel/sdk/metric v1.36.0 h1:r0ntwwGosWGaa0CrSt8cuNuTcccMXERFwHX4dThiPis= -go.opentelemetry.io/otel/sdk/metric v1.36.0/go.mod h1:qTNOhFDfKRwX0yXOqJYegL5WRaW376QbB7P4Pb0qva4= -go.opentelemetry.io/otel/trace v1.36.0 h1:ahxWNuqZjpdiFAyrIoQ4GIiAIhxAunQR6MUoKrsNd4w= -go.opentelemetry.io/otel/trace v1.36.0/go.mod h1:gQ+OnDZzrybY4k4seLzPAWNwVBBVlF2szhehOBB/tGA= +go.opentelemetry.io/otel/metric v1.37.0 h1:mvwbQS5m0tbmqML4NqK+e3aDiO02vsf/WgbsdpcPoZE= +go.opentelemetry.io/otel/metric v1.37.0/go.mod h1:04wGrZurHYKOc+RKeye86GwKiTb9FKm1WHtO+4EVr2E= +go.opentelemetry.io/otel/sdk v1.37.0 h1:ItB0QUqnjesGRvNcmAcU0LyvkVyGJ2xftD29bWdDvKI= +go.opentelemetry.io/otel/sdk v1.37.0/go.mod h1:VredYzxUvuo2q3WRcDnKDjbdvmO0sCzOvVAiY+yUkAg= +go.opentelemetry.io/otel/sdk/metric v1.37.0 h1:90lI228XrB9jCMuSdA0673aubgRobVZFhbjxHHspCPc= +go.opentelemetry.io/otel/sdk/metric v1.37.0/go.mod h1:cNen4ZWfiD37l5NhS+Keb5RXVWZWpRE+9WyVCpbo5ps= +go.opentelemetry.io/otel/trace v1.37.0 h1:HLdcFNbRQBE2imdSEgm/kwqmQj1Or1l/7bW6mxVK7z4= +go.opentelemetry.io/otel/trace v1.37.0/go.mod h1:TlgrlQ+PtQO5XFerSPUYG0JSgGyryXewPGyayAWSBS0= go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI= go.opentelemetry.io/proto/otlp v0.15.0/go.mod h1:H7XAot3MsfNsj7EXtrA2q5xSNQ10UqI405h3+duxN4U= go.opentelemetry.io/proto/otlp v0.19.0/go.mod h1:H7XAot3MsfNsj7EXtrA2q5xSNQ10UqI405h3+duxN4U= @@ -1556,8 +1560,8 @@ golang.org/x/crypto v0.0.0-20220511200225-c6db032c6c88/go.mod h1:IxCIyHEi3zRg3s0 golang.org/x/crypto v0.0.0-20220722155217-630584e8d5aa/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.7.0/go.mod h1:pYwdfH91IfpZVANVyUOhSIPZaFoJGxTFbZhFTx+dXZU= golang.org/x/crypto v0.9.0/go.mod h1:yrmDGqONDYtNj3tH8X9dzUun2m2lzPa9ngI6/RUPGR0= -golang.org/x/crypto v0.40.0 h1:r4x+VvoG5Fm+eJcxMaY8CQM7Lb0l1lsmjGBQ6s8BfKM= -golang.org/x/crypto v0.40.0/go.mod h1:Qr1vMER5WyS2dfPHAlsOj01wgLbsyWtFn/aY+5+ZdxY= +golang.org/x/crypto v0.41.0 h1:WKYxWedPGCTVVl5+WHSSrOBT0O8lx32+zxmHxijgXp4= +golang.org/x/crypto v0.41.0/go.mod h1:pO5AFd7FA68rFak7rOAGVuygIISepHftHnr8dr6+sUc= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -1618,8 +1622,8 @@ golang.org/x/mod v0.7.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.8.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.9.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.10.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= -golang.org/x/mod v0.25.0 h1:n7a+ZbQKQA/Ysbyb0/6IbB1H/X41mKgbhfv7AfG/44w= -golang.org/x/mod v0.25.0/go.mod h1:IXM97Txy2VM4PJ3gI61r1YEk/gAj6zAHN3AdZt6S9Ww= +golang.org/x/mod v0.26.0 h1:EGMPT//Ezu+ylkCijjPc+f4Aih7sZvaAr+O3EHBxvZg= +golang.org/x/mod v0.26.0/go.mod h1:/j6NAhSk8iQ723BGAUyoAcn7SlD7s15Dp9Nd/SfeaFQ= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -1688,8 +1692,8 @@ golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.8.0/go.mod h1:QVkue5JL9kW//ek3r6jTKnTFis1tRmNAW2P1shuFdJc= golang.org/x/net v0.9.0/go.mod h1:d48xBJpPfHeWQsugry2m+kC02ZBRGRgulfHnEXEuWns= golang.org/x/net v0.10.0/go.mod h1:0qNGK6F8kojg2nk9dLZ2mShWaEBan6FAoqfSigmmuDg= -golang.org/x/net v0.42.0 h1:jzkYrhi3YQWD6MLBJcsklgQsoAcw89EcZbJw8Z614hs= -golang.org/x/net v0.42.0/go.mod h1:FF1RA5d3u7nAYA4z2TkclSCKh68eSXtiFwcWQpPXdt8= +golang.org/x/net v0.43.0 h1:lat02VYK2j4aLzMzecihNvTlJNQUq316m2Mr9rnM6YE= +golang.org/x/net v0.43.0/go.mod h1:vhO1fvI4dGsIjh73sWfUVjj3N7CA9WkKJNQm2svM6Jg= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -1842,8 +1846,8 @@ golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.7.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.21.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/sys v0.34.0 h1:H5Y5sJ2L2JRdyv7ROF1he/lPdvFsd0mJHFw2ThKHxLA= -golang.org/x/sys v0.34.0/go.mod h1:BJP2sWEmIv4KK5OTEluFJCKSidICx8ciO85XgH3Ak8k= +golang.org/x/sys v0.35.0 h1:vz1N37gP5bs89s7He8XuIYXpyY0+QlsKmzipCbUtyxI= +golang.org/x/sys v0.35.0/go.mod h1:BJP2sWEmIv4KK5OTEluFJCKSidICx8ciO85XgH3Ak8k= golang.org/x/term v0.0.0-20201117132131-f5c789dd3221/go.mod h1:Nr5EML6q2oocZ2LXRh80K7BxOlk5/8JxuGnuhpl+muw= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= @@ -1855,8 +1859,8 @@ golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= golang.org/x/term v0.6.0/go.mod h1:m6U89DPEgQRMq3DNkDClhWw02AUbt2daBVO4cn4Hv9U= golang.org/x/term v0.7.0/go.mod h1:P32HKFT3hSsZrRxla30E9HqToFYAQPCMs/zFMBUFqPY= golang.org/x/term v0.8.0/go.mod h1:xPskH00ivmX89bAKVGSKKtLOWNx2+17Eiy94tnKShWo= -golang.org/x/term v0.33.0 h1:NuFncQrRcaRvVmgRkvM3j/F00gWIAlcmlB8ACEKmGIg= -golang.org/x/term v0.33.0/go.mod h1:s18+ql9tYWp1IfpV9DmCtQDDSRBUjKaw9M1eAv5UeF0= +golang.org/x/term v0.34.0 h1:O/2T7POpk0ZZ7MAzMeWFSg6S5IpWd/RXDlM9hgM3DR4= +golang.org/x/term v0.34.0/go.mod h1:5jC53AEywhIVebHgPVeg0mj8OD3VO9OzclacVrqpaAw= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -1873,8 +1877,8 @@ golang.org/x/text v0.6.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.8.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= golang.org/x/text v0.9.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= -golang.org/x/text v0.27.0 h1:4fGWRpyh641NLlecmyl4LOe6yDdfaYNrGb2zdfo4JV4= -golang.org/x/text v0.27.0/go.mod h1:1D28KMCvyooCX9hBiosv5Tz/+YLxj0j7XhWjpSUF7CU= +golang.org/x/text v0.28.0 h1:rhazDwis8INMIwQ4tpjLDzUhx6RlXqZNPEM0huQojng= +golang.org/x/text v0.28.0/go.mod h1:U8nCwOR8jO/marOQ0QbDiOngZVEBB7MAiitBuMjXiNU= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -1959,8 +1963,8 @@ golang.org/x/tools v0.3.0/go.mod h1:/rWhSS2+zyEVwoJf8YAX6L2f0ntZ7Kn/mGgAWcipA5k= golang.org/x/tools v0.6.0/go.mod h1:Xwgl3UAJ/d3gWutnCtw505GrjyAbvKui8lOU390QaIU= golang.org/x/tools v0.7.0/go.mod h1:4pg6aUX35JBAogB10C9AtvVL+qowtN4pT3CGSQex14s= golang.org/x/tools v0.8.0/go.mod h1:JxBZ99ISMI5ViVkT1tr6tdNmXeTrcpVSD3vZ1RsRdN4= -golang.org/x/tools v0.34.0 h1:qIpSLOxeCYGg9TrcJokLBG4KFA6d795g0xkBkiESGlo= -golang.org/x/tools v0.34.0/go.mod h1:pAP9OwEaY1CAW3HOmg3hLZC5Z0CCmzjAF2UQMSqNARg= +golang.org/x/tools v0.35.0 h1:mBffYraMEf7aa0sB+NuKnuCy8qI/9Bughn8dC2Gu5r0= +golang.org/x/tools v0.35.0/go.mod h1:NKdj5HkL/73byiZSJjqJgKn3ep7KjFkBOkR/Hps3VPw= golang.org/x/xerrors v0.0.0-20190410155217-1f06c39b4373/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20190513163551-3ee3066db522/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -1977,8 +1981,8 @@ gonum.org/v1/gonum v0.0.0-20180816165407-929014505bf4/go.mod h1:Y+Yx5eoAFn32cQvJ gonum.org/v1/gonum v0.8.2/go.mod h1:oe/vMfY3deqTw+1EZJhuvEW2iwGF1bW9wwu7XCu0+v0= gonum.org/v1/gonum v0.9.3/go.mod h1:TZumC3NeyVQskjXqmyWt4S3bINhy7B4eYwW69EbyX+0= gonum.org/v1/gonum v0.11.0/go.mod h1:fSG4YDCxxUZQJ7rKsQrj0gMOg00Il0Z96/qMA4bVQhA= -gonum.org/v1/gonum v0.12.0 h1:xKuo6hzt+gMav00meVPUlXwSdoEJP46BR+wdxQEFK2o= -gonum.org/v1/gonum v0.12.0/go.mod h1:73TDxJfAAHeA8Mk9mf8NlIppyhQNo5GLTcYeqgo2lvY= +gonum.org/v1/gonum v0.16.0 h1:5+ul4Swaf3ESvrOnidPp4GZbzf0mxVQpDCYUQE7OJfk= +gonum.org/v1/gonum v0.16.0/go.mod h1:fef3am4MQ93R2HHpKnLk4/Tbh/s0+wqD5nfa6Pnwy4E= gonum.org/v1/netlib v0.0.0-20190313105609-8cb42192e0e0/go.mod h1:wa6Ws7BG/ESfp6dHfk7C6KdzKA7wR7u/rKwOGE66zvw= gonum.org/v1/plot v0.0.0-20190515093506-e2840ee46a6b/go.mod h1:Wt8AAjI+ypCyYX3nZBvf6cAIx93T+c/OS2HFAYskSZc= gonum.org/v1/plot v0.9.0/go.mod h1:3Pcqqmp6RHvJI72kgb8fThyUnav364FOsdDo2aGW5lY= @@ -2049,8 +2053,8 @@ google.golang.org/api v0.108.0/go.mod h1:2Ts0XTHNVWxypznxWOYUeI4g3WdP9Pk2Qk58+a/ google.golang.org/api v0.110.0/go.mod h1:7FC4Vvx1Mooxh8C5HWjzZHcavuS2f6pmJpZx60ca7iI= google.golang.org/api v0.111.0/go.mod h1:qtFHvU9mhgTJegR31csQ+rwxyUTHOKFqCKWp1J0fdw0= google.golang.org/api v0.114.0/go.mod h1:ifYI2ZsFK6/uGddGfAD5BMxlnkBqCmqHSDUVi45N5Yg= -google.golang.org/api v0.243.0 h1:sw+ESIJ4BVnlJcWu9S+p2Z6Qq1PjG77T8IJ1xtp4jZQ= -google.golang.org/api v0.243.0/go.mod h1:GE4QtYfaybx1KmeHMdBnNnyLzBZCVihGBXAmJu/uUr8= +google.golang.org/api v0.248.0 h1:hUotakSkcwGdYUqzCRc5yGYsg4wXxpkKlW5ryVqvC1Y= +google.golang.org/api v0.248.0/go.mod h1:yAFUAF56Li7IuIQbTFoLwXTCI6XCFKueOlS7S9e4F9k= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= @@ -2211,10 +2215,10 @@ google.golang.org/genproto v0.0.0-20230331144136-dcfb400f0633/go.mod h1:UUQDJDOl google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1/go.mod h1:nKE/iIaLqn2bQwXBg8f1g2Ylh6r5MN5CmZvuzZCgsCU= google.golang.org/genproto v0.0.0-20250603155806-513f23925822 h1:rHWScKit0gvAPuOnu87KpaYtjK5zBMLcULh7gxkCXu4= google.golang.org/genproto v0.0.0-20250603155806-513f23925822/go.mod h1:HubltRL7rMh0LfnQPkMH4NPDFEWp0jw3vixw7jEM53s= -google.golang.org/genproto/googleapis/api v0.0.0-20250603155806-513f23925822 h1:oWVWY3NzT7KJppx2UKhKmzPq4SRe0LdCijVRwvGeikY= -google.golang.org/genproto/googleapis/api v0.0.0-20250603155806-513f23925822/go.mod h1:h3c4v36UTKzUiuaOKQ6gr3S+0hovBtUrXzTG/i3+XEc= -google.golang.org/genproto/googleapis/rpc v0.0.0-20250715232539-7130f93afb79 h1:1ZwqphdOdWYXsUHgMpU/101nCtf/kSp9hOrcvFsnl10= -google.golang.org/genproto/googleapis/rpc v0.0.0-20250715232539-7130f93afb79/go.mod h1:qQ0YXyHHx3XkvlzUtpXDkS29lDSafHMZBAZDc03LQ3A= +google.golang.org/genproto/googleapis/api v0.0.0-20250818200422-3122310a409c h1:AtEkQdl5b6zsybXcbz00j1LwNodDuH6hVifIaNqk7NQ= +google.golang.org/genproto/googleapis/api v0.0.0-20250818200422-3122310a409c/go.mod h1:ea2MjsO70ssTfCjiwHgI0ZFqcw45Ksuk2ckf9G468GA= +google.golang.org/genproto/googleapis/rpc v0.0.0-20250818200422-3122310a409c h1:qXWI/sQtv5UKboZ/zUk7h+mrf/lXORyI+n9DKDAusdg= +google.golang.org/genproto/googleapis/rpc v0.0.0-20250818200422-3122310a409c/go.mod h1:gw1tLEfykwDz2ET4a12jcXt4couGAm7IwsVaTy0Sflo= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= @@ -2257,8 +2261,8 @@ google.golang.org/grpc v1.52.3/go.mod h1:pu6fVzoFb+NBYNAvQL08ic+lvB2IojljRYuun5v google.golang.org/grpc v1.53.0/go.mod h1:OnIrk0ipVdj4N5d9IUoFUx72/VlD7+jUsHwZgwSMQpw= google.golang.org/grpc v1.54.0/go.mod h1:PUSEXI6iWghWaB6lXM4knEgpJNu2qUcKfDtNci3EC2g= google.golang.org/grpc v1.56.3/go.mod h1:I9bI3vqKfayGqPUAwGdOSu7kt6oIJLixfffKrpXqQ9s= -google.golang.org/grpc v1.73.0 h1:VIWSmpI2MegBtTuFt5/JWy2oXxtjJ/e89Z70ImfD2ok= -google.golang.org/grpc v1.73.0/go.mod h1:50sbHOUqWoCQGI8V2HQLJM0B+LMlIUjNSZmow7EVBQc= +google.golang.org/grpc v1.75.0 h1:+TW+dqTd2Biwe6KKfhE5JpiYIBWq865PhKGSXiivqt4= +google.golang.org/grpc v1.75.0/go.mod h1:JtPAzKiq4v1xcAB2hydNlWI2RnF85XXcV0mhKXr2ecQ= google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.1.0/go.mod h1:6Kw0yEErY5E/yWrBtf03jp27GLLJujG4z/JK95pnjjw= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= @@ -2277,8 +2281,8 @@ google.golang.org/protobuf v1.28.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqw google.golang.org/protobuf v1.28.1/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= google.golang.org/protobuf v1.29.1/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= google.golang.org/protobuf v1.30.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= -google.golang.org/protobuf v1.36.6 h1:z1NpPI8ku2WgiWnf+t9wTPsn6eP1L7ksHUlkfLvd9xY= -google.golang.org/protobuf v1.36.6/go.mod h1:jduwjTPXsFjZGTmRluh+L6NjiWu7pchiJ2/5YcXBHnY= +google.golang.org/protobuf v1.36.8 h1:xHScyCOEuuwZEc6UtSOvPbAT4zRh0xcNRYekJwfqyMc= +google.golang.org/protobuf v1.36.8/go.mod h1:fuxRtAxBytpl4zzqUh6/eyUujkJdNiuEkXntxiD/uRU= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= diff --git a/sdks/go/pkg/beam/core/core.go b/sdks/go/pkg/beam/core/core.go index 843cfea07743..0856d430804f 100644 --- a/sdks/go/pkg/beam/core/core.go +++ b/sdks/go/pkg/beam/core/core.go @@ -27,7 +27,7 @@ const ( // SdkName is the human readable name of the SDK for UserAgents. SdkName = "Apache Beam SDK for Go" // SdkVersion is the current version of the SDK. - SdkVersion = "2.68.0.dev" + SdkVersion = "2.69.0.dev" // DefaultDockerImage represents the associated image for this release. DefaultDockerImage = "apache/beam_go_sdk:" + SdkVersion diff --git a/sdks/go/pkg/beam/core/metrics/sampler.go b/sdks/go/pkg/beam/core/metrics/sampler.go index 3b768d579d36..264eaeae487b 100644 --- a/sdks/go/pkg/beam/core/metrics/sampler.go +++ b/sdks/go/pkg/beam/core/metrics/sampler.go @@ -63,11 +63,11 @@ func (s *StateSampler) Sample(ctx context.Context, t time.Duration) error { } if s.millisSinceLastTransition > s.nextLogTime { - log.Infof(ctx, "Operation ongoing in transform %v for at least %v ms without outputting or completing in state %v", ps.pid, s.millisSinceLastTransition, getState(ps.state)) + log.Infof(ctx, "Operation ongoing in transform %v for at least %v without outputting or completing in state %v", ps.pid, s.millisSinceLastTransition, getState(ps.state)) s.nextLogTime += s.logInterval } if s.restartLullTimeout > 0 && s.millisSinceLastTransition > s.restartLullTimeout { - return errors.Errorf("Operation ongoing in transform %v for at least %v ms without outputting or completing in state %v, the SDK harness will be terminated and restarted", ps.pid, s.millisSinceLastTransition, getState(ps.state)) + return errors.Errorf("Processing of an element in transform %v has exceeded the specified timeout of %v without outputting or completing in state %v, SDK harness will be terminated", ps.pid, s.restartLullTimeout, getState(ps.state)) } } return nil diff --git a/sdks/go/pkg/beam/core/metrics/sampler_test.go b/sdks/go/pkg/beam/core/metrics/sampler_test.go index 492d2f6748fe..ec50bc22d1e8 100644 --- a/sdks/go/pkg/beam/core/metrics/sampler_test.go +++ b/sdks/go/pkg/beam/core/metrics/sampler_test.go @@ -176,8 +176,8 @@ func TestSamplerWithRestartLullTimeout(t *testing.T) { t.Errorf("s.sample(bctx, interval) = %v, want %v", got, want) } err := s.Sample(bctx, interval) - if err == nil || !strings.Contains(err.Error(), "the SDK harness will be terminated and restarted") { - t.Errorf("s.sample(bctx, interval) = %v, want %v", err, "the SDK harness will be terminated and restarted") + if err == nil || !strings.Contains(err.Error(), "SDK harness will be terminated") { + t.Errorf("s.sample(bctx, interval) = %v, want %v", err, "SDK harness will be terminated") } } diff --git a/sdks/go/pkg/beam/core/runtime/exec/userstate.go b/sdks/go/pkg/beam/core/runtime/exec/userstate.go index f83aee4bf741..ea723b18e3a7 100644 --- a/sdks/go/pkg/beam/core/runtime/exec/userstate.go +++ b/sdks/go/pkg/beam/core/runtime/exec/userstate.go @@ -35,17 +35,18 @@ type stateProvider struct { elementKey []byte window []byte - transactionsByKey map[string][]state.Transaction - initialValueByKey map[string]any - initialBagByKey map[string][]any - initialMapValuesByKey map[string]map[string]any - initialMapKeysByKey map[string][]any - readersByKey map[string]io.ReadCloser - appendersByKey map[string]io.Writer - clearersByKey map[string]io.Writer - codersByKey map[string]*coder.Coder - keyCodersByID map[string]*coder.Coder - combineFnsByKey map[string]*graph.CombineFn + transactionsByKey map[string][]state.Transaction + initialValueByKey map[string]any + initialBagByKey map[string][]any + blindBagWriteCountsByKey map[string]int // Tracks blind writes to bags before a read. + initialMapValuesByKey map[string]map[string]any + initialMapKeysByKey map[string][]any + readersByKey map[string]io.ReadCloser + appendersByKey map[string]io.Writer + clearersByKey map[string]io.Writer + codersByKey map[string]*coder.Coder + keyCodersByID map[string]*coder.Coder + combineFnsByKey map[string]*graph.CombineFn } // ReadValueState reads a value state from the State API @@ -148,6 +149,12 @@ func (s *stateProvider) ReadBagState(userStateID string) ([]any, []state.Transac if !ok { transactions = []state.Transaction{} } + // If there were blind writes before this read, trim the transactions. + // These don't need to be reset, unless a clear happens. + if s.blindBagWriteCountsByKey[userStateID] > 0 { + // Trim blind writes from the transaction queue, to avoid re-applying them. + transactions = transactions[s.blindBagWriteCountsByKey[userStateID]:] + } return initialValue, transactions, nil } @@ -165,12 +172,17 @@ func (s *stateProvider) ClearBagState(val state.Transaction) error { // Any transactions before a clear don't matter s.transactionsByKey[val.Key] = []state.Transaction{val} + s.blindBagWriteCountsByKey[val.Key] = 1 // To account for the clear. return nil } // WriteBagState writes a bag state to the State API func (s *stateProvider) WriteBagState(val state.Transaction) error { + _, ok := s.initialBagByKey[val.Key] + if !ok { + s.blindBagWriteCountsByKey[val.Key]++ + } ap, err := s.getBagAppender(val.Key) if err != nil { return err @@ -510,22 +522,23 @@ func (s *userStateAdapter) NewStateProvider(ctx context.Context, reader StateRea return stateProvider{}, err } sp := stateProvider{ - ctx: ctx, - sr: reader, - SID: s.sid, - elementKey: elementKey, - window: win, - transactionsByKey: make(map[string][]state.Transaction), - initialValueByKey: make(map[string]any), - initialBagByKey: make(map[string][]any), - initialMapValuesByKey: make(map[string]map[string]any), - initialMapKeysByKey: make(map[string][]any), - readersByKey: make(map[string]io.ReadCloser), - appendersByKey: make(map[string]io.Writer), - clearersByKey: make(map[string]io.Writer), - combineFnsByKey: s.stateIDToCombineFn, - codersByKey: s.stateIDToCoder, - keyCodersByID: s.stateIDToKeyCoder, + ctx: ctx, + sr: reader, + SID: s.sid, + elementKey: elementKey, + window: win, + transactionsByKey: make(map[string][]state.Transaction), + initialValueByKey: make(map[string]any), + initialBagByKey: make(map[string][]any), + blindBagWriteCountsByKey: make(map[string]int), + initialMapValuesByKey: make(map[string]map[string]any), + initialMapKeysByKey: make(map[string][]any), + readersByKey: make(map[string]io.ReadCloser), + appendersByKey: make(map[string]io.Writer), + clearersByKey: make(map[string]io.Writer), + combineFnsByKey: s.stateIDToCombineFn, + codersByKey: s.stateIDToCoder, + keyCodersByID: s.stateIDToKeyCoder, } return sp, nil diff --git a/sdks/go/pkg/beam/core/runtime/harness/harness.go b/sdks/go/pkg/beam/core/runtime/harness/harness.go index ea2d7dba76f4..cc1e53d02d21 100644 --- a/sdks/go/pkg/beam/core/runtime/harness/harness.go +++ b/sdks/go/pkg/beam/core/runtime/harness/harness.go @@ -704,7 +704,7 @@ func (c *control) handleInstruction(ctx context.Context, req *fnpb.InstructionRe func parseTimeoutDurationFlag(ctx context.Context, elementProcessingTimeout string) time.Duration { userSpecifiedTimeout, err := time.ParseDuration(elementProcessingTimeout) if err != nil { - log.Errorf(ctx, "Failed to parse element_processing_timeout: %v, there will be no timeout for processing an element in a PTransform operation", err) + log.Warnf(ctx, "Failed to parse element_processing_timeout: %v, there will be no timeout for processing an element in a PTransform operation", err) return 0 * time.Minute } return userSpecifiedTimeout diff --git a/sdks/go/pkg/beam/options/jobopts/options.go b/sdks/go/pkg/beam/options/jobopts/options.go index 04348b1d1ae2..327f3895b117 100644 --- a/sdks/go/pkg/beam/options/jobopts/options.go +++ b/sdks/go/pkg/beam/options/jobopts/options.go @@ -91,18 +91,20 @@ var ( // executing them and fails early if the pipelines don't pass. Strict = flag.Bool("beam_strict", false, "Apply additional validation to pipelines.") - // Flag to retain docker containers created by the runner. If false, then + // RetrainDockerContainers flag to retain docker containers created by the runner. If false, then // containers are deleted once the job ends, even if it failed. RetainDockerContainers = flag.Bool("retain_docker_containers", false, "Retain Docker containers created by the runner.") - // Flag to set the degree of parallelism. If not set, the configured Flink default is used, or 1 if none can be found. + // Parallelisn flag to set the degree of parallelism. If not set, the configured Flink default is used, or 1 if none can be found. Parallelism = flag.Int("parallelism", -1, "The degree of parallelism to be used when distributing operations onto Flink workers.") // ResourceHints flag takes whole pipeline hints for resources. ResourceHints stringSlice - // Flag to set the timeout for processing an element in a PTransform operation. If set to -1, there is no timeout. - ElementProcessingTimeout = flag.Duration("element_processing_timeout", -1, "The timeout for processing an element in a PTransform operation. If set to -1, there is no timeout.") + // ElementProcessingTimeout flag to set the timeout for processing an element in a PTransform operation. If set to -1, there is no timeout. + ElementProcessingTimeout = flag.Duration("element_processing_timeout", -1, + "The time limit (in minutes) for any PTransform to finish processing a single element. If exceeded, "+ + "the SDK worker process self-terminates and processing may be restarted by a runner. There is no time limit if the value is set to -1.") ) type missingFlagError error diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go b/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go index 2ddd7bbc5c1f..d489bcc18c21 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/elementmanager.go @@ -89,6 +89,14 @@ type PColInfo struct { KeyDec func(io.Reader) []byte } +func (info PColInfo) LogValue() slog.Value { + return slog.GroupValue( + slog.String("GlobalID", info.GlobalID), + slog.String("WindowCoder", info.WindowCoder.String()), + // Do not attempt to log functions, or it will result in JSON marshaling error. + ) +} + // WinCoderType indicates what kind of coder // the window is using. There are only 3 // valid single window encodings. @@ -110,6 +118,19 @@ const ( WinCustom ) +func (wct WinCoderType) String() string { + switch wct { + case WinGlobal: + return "WinGlobal" + case WinInterval: + return "WinInterval" + case WinCustom: + return "WinCustom" + default: + return fmt.Sprintf("Unknown(%d)", wct) + } +} + // ToData recodes the elements with their approprate windowed value header. func (es elements) ToData(info PColInfo) [][]byte { var ret [][]byte @@ -338,7 +359,7 @@ func (rb RunBundle) LogValue() slog.Value { return slog.GroupValue( slog.String("ID", rb.BundleID), slog.String("stage", rb.StageID), - slog.Time("watermark", rb.Watermark.ToTime())) + slog.Any("watermark", rb.Watermark)) } // Bundles is the core execution loop. It produces a sequences of bundles able to be executed. @@ -871,70 +892,75 @@ func (em *ElementManager) PersistBundle(rb RunBundle, col2Coders map[string]PCol // Clear out the inprogress elements associated with the completed bundle. // Must be done after adding the new pending elements to avoid an incorrect // watermark advancement. - stage.mu.Lock() - completed := stage.inprogress[rb.BundleID] - em.addPending(-len(completed.es)) - delete(stage.inprogress, rb.BundleID) - for k := range stage.inprogressKeysByBundle[rb.BundleID] { - delete(stage.inprogressKeys, k) - } - delete(stage.inprogressKeysByBundle, rb.BundleID) - - // Adjust holds as needed. - for h, c := range newHolds { - if c > 0 { - stage.watermarkHolds.Add(h, c) - } else if c < 0 { - stage.watermarkHolds.Drop(h, -c) - } - } - for hold, v := range stage.inprogressHoldsByBundle[rb.BundleID] { - stage.watermarkHolds.Drop(hold, v) - } - delete(stage.inprogressHoldsByBundle, rb.BundleID) - - // Clean up OnWindowExpiration bundle accounting, so window state - // may be garbage collected. - if stage.expiryWindowsByBundles != nil { - win, ok := stage.expiryWindowsByBundles[rb.BundleID] - if ok { - stage.inProgressExpiredWindows[win] -= 1 - if stage.inProgressExpiredWindows[win] == 0 { - delete(stage.inProgressExpiredWindows, win) + func() { + stage.mu.Lock() + // Defer unlocking the mutex within an anonymous function to ensure it's released + // even if a panic occurs during `em.addPending`. This prevents potential deadlocks + // if the waitgroup unexpectedly drops below zero due to a runner bug. + defer stage.mu.Unlock() + completed := stage.inprogress[rb.BundleID] + em.addPending(-len(completed.es)) + delete(stage.inprogress, rb.BundleID) + for k := range stage.inprogressKeysByBundle[rb.BundleID] { + delete(stage.inprogressKeys, k) + } + delete(stage.inprogressKeysByBundle, rb.BundleID) + + // Adjust holds as needed. + for h, c := range newHolds { + if c > 0 { + stage.watermarkHolds.Add(h, c) + } else if c < 0 { + stage.watermarkHolds.Drop(h, -c) } - delete(stage.expiryWindowsByBundles, rb.BundleID) } - } + for hold, v := range stage.inprogressHoldsByBundle[rb.BundleID] { + stage.watermarkHolds.Drop(hold, v) + } + delete(stage.inprogressHoldsByBundle, rb.BundleID) - // If there are estimated output watermarks, set the estimated - // output watermark for the stage. - if len(residuals.MinOutputWatermarks) > 0 { - estimate := mtime.MaxTimestamp - for _, t := range residuals.MinOutputWatermarks { - estimate = mtime.Min(estimate, t) + // Clean up OnWindowExpiration bundle accounting, so window state + // may be garbage collected. + if stage.expiryWindowsByBundles != nil { + win, ok := stage.expiryWindowsByBundles[rb.BundleID] + if ok { + stage.inProgressExpiredWindows[win] -= 1 + if stage.inProgressExpiredWindows[win] == 0 { + delete(stage.inProgressExpiredWindows, win) + } + delete(stage.expiryWindowsByBundles, rb.BundleID) + } } - stage.estimatedOutput = estimate - } - // Handle persisting. - for link, winMap := range d.state { - linkMap, ok := stage.state[link] - if !ok { - linkMap = map[typex.Window]map[string]StateData{} - stage.state[link] = linkMap + // If there are estimated output watermarks, set the estimated + // output watermark for the stage. + if len(residuals.MinOutputWatermarks) > 0 { + estimate := mtime.MaxTimestamp + for _, t := range residuals.MinOutputWatermarks { + estimate = mtime.Min(estimate, t) + } + stage.estimatedOutput = estimate } - for w, keyMap := range winMap { - wlinkMap, ok := linkMap[w] + + // Handle persisting. + for link, winMap := range d.state { + linkMap, ok := stage.state[link] if !ok { - wlinkMap = map[string]StateData{} - linkMap[w] = wlinkMap + linkMap = map[typex.Window]map[string]StateData{} + stage.state[link] = linkMap } - for key, data := range keyMap { - wlinkMap[key] = data + for w, keyMap := range winMap { + wlinkMap, ok := linkMap[w] + if !ok { + wlinkMap = map[string]StateData{} + linkMap[w] = wlinkMap + } + for key, data := range keyMap { + wlinkMap[key] = data + } } } - } - stage.mu.Unlock() + }() em.markChangedAndClearBundle(stage.ID, rb.BundleID, ptRefreshes) } @@ -1011,11 +1037,16 @@ func (em *ElementManager) triageTimers(d TentativeData, inputInfo PColInfo, stag // FailBundle clears the extant data allowing the execution to shut down. func (em *ElementManager) FailBundle(rb RunBundle) { stage := em.stages[rb.StageID] - stage.mu.Lock() - completed := stage.inprogress[rb.BundleID] - em.addPending(-len(completed.es)) - delete(stage.inprogress, rb.BundleID) - stage.mu.Unlock() + func() { + stage.mu.Lock() + // Defer unlocking the mutex within an anonymous function to ensure it's released + // even if a panic occurs during `em.addPending`. This prevents potential deadlocks + // if the waitgroup unexpectedly drops below zero due to a runner bug. + defer stage.mu.Unlock() + completed := stage.inprogress[rb.BundleID] + em.addPending(-len(completed.es)) + delete(stage.inprogress, rb.BundleID) + }() em.markChangedAndClearBundle(rb.StageID, rb.BundleID, nil) } @@ -1132,6 +1163,7 @@ type stageState struct { input mtime.Time // input watermark for the parallel input. output mtime.Time // Output watermark for the whole stage estimatedOutput mtime.Time // Estimated watermark output from DoFns + previousInput mtime.Time // input watermark before the latest watermark refresh pending elementHeap // pending input elements for this stage that are to be processesd inprogress map[string]elements // inprogress elements by active bundles, keyed by bundle @@ -1993,6 +2025,8 @@ func (ss *stageState) updateWatermarks(em *ElementManager) set[string] { newIn = minPending } + ss.previousInput = ss.input + // If bigger, advance the input watermark. if newIn > ss.input { ss.input = newIn @@ -2150,11 +2184,13 @@ func (ss *stageState) bundleReady(em *ElementManager, emNow mtime.Time) (mtime.T ptimeEventsReady := ss.processingTimeTimers.Peek() <= emNow || emNow == mtime.MaxTimestamp injectedReady := len(ss.bundlesToInject) > 0 - // If the upstream watermark and the input watermark are the same, - // then we can't yet process this stage. + // If the upstream watermark does not change, we can't yet process this stage. + // To check whether upstream water is unchanged, we evaluate if the input watermark, and + // the input watermark before the latest refresh are the same. inputW := ss.input _, upstreamW := ss.UpstreamWatermark() - if inputW == upstreamW { + previousInputW := ss.previousInput + if inputW == upstreamW && previousInputW == inputW { slog.Debug("bundleReady: unchanged upstream watermark", slog.String("stage", ss.ID), slog.Group("watermark", diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/strategy.go b/sdks/go/pkg/beam/runners/prism/internal/engine/strategy.go index 5446d3edd3c0..5ccc4a513667 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/strategy.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/strategy.go @@ -302,15 +302,23 @@ func (t *TriggerAfterEach) onFire(state *StateData) { if !t.shouldFire(state) { return } - for _, sub := range t.SubTriggers { + for i, sub := range t.SubTriggers { if state.getTriggerState(sub).finished { continue } sub.onFire(state) + // If the sub-trigger didn't finish, we return, waiting for it to finish on a subsequent call. if !state.getTriggerState(sub).finished { return } + + // If the sub-trigger finished, we check if it's the last one. + // If it's not the last one, we return, waiting for the next onFire call to advance to the next sub-trigger. + if i < len(t.SubTriggers)-1 { + return + } } + // clear and reset when all sub-triggers have fired. triggerClearAndFinish(t, state) } diff --git a/sdks/go/pkg/beam/runners/prism/internal/engine/strategy_test.go b/sdks/go/pkg/beam/runners/prism/internal/engine/strategy_test.go index 4934665833ed..86393d1c1938 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/engine/strategy_test.go +++ b/sdks/go/pkg/beam/runners/prism/internal/engine/strategy_test.go @@ -122,6 +122,25 @@ func TestTriggers_isReady(t *testing.T) { {triggerInput{newElementCount: 1}, false}, {triggerInput{newElementCount: 1}, false}, }, + }, { + name: "afterEach_2_Always_1", + trig: &TriggerAfterEach{ + SubTriggers: []Trigger{ + &TriggerElementCount{2}, + &TriggerAfterAny{SubTriggers: []Trigger{&TriggerAlways{}}}, + &TriggerElementCount{1}, + }, + }, + inputs: []io{ + {triggerInput{newElementCount: 1}, false}, + {triggerInput{newElementCount: 1}, true}, // first is ready + {triggerInput{newElementCount: 1}, true}, // second is ready + {triggerInput{newElementCount: 1}, true}, // third is ready + {triggerInput{newElementCount: 1}, false}, // never resets after this. + {triggerInput{newElementCount: 1}, false}, + {triggerInput{newElementCount: 1}, false}, + {triggerInput{newElementCount: 1}, false}, + }, }, { name: "afterAny_2_3_4", trig: &TriggerAfterAny{ diff --git a/sdks/go/pkg/beam/runners/prism/internal/environments.go b/sdks/go/pkg/beam/runners/prism/internal/environments.go index 3239c76dfe1f..1f852e0862f1 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/environments.go +++ b/sdks/go/pkg/beam/runners/prism/internal/environments.go @@ -24,6 +24,7 @@ import ( "os" "os/exec" "slices" + "strconv" "time" fnpb "github.com/apache/beam/sdks/v2/go/pkg/beam/model/fnexecution_v1" @@ -79,7 +80,7 @@ func runEnvironment(ctx context.Context, j *jobservices.Job, env string, wk *wor logger.Error("unmarshaling docker environment payload", "error", err) return err } - return dockerEnvironment(ctx, logger, dp, wk, j.ArtifactEndpoint()) + return dockerEnvironment(ctx, logger, dp, wk, wk.ArtifactEndpoint) case urns.EnvProcess: pp := &pipepb.ProcessPayload{} if err := (proto.UnmarshalOptions{}).Unmarshal(e.GetPayload(), pp); err != nil { @@ -87,7 +88,7 @@ func runEnvironment(ctx context.Context, j *jobservices.Job, env string, wk *wor return err } go func() { - processEnvironment(ctx, pp, wk) + processEnvironment(ctx, logger, pp, wk) logger.Debug("environment stopped", slog.String("job", j.String())) }() return nil @@ -207,17 +208,18 @@ func dockerEnvironment(ctx context.Context, logger *slog.Logger, dp *pipepb.Dock } logger.Debug("creating container", "envs", envs, "mounts", mounts) + cmd := []string{ + fmt.Sprintf("--id=%v", wk.ID), + fmt.Sprintf("--control_endpoint=%v", wk.Endpoint()), + fmt.Sprintf("--artifact_endpoint=%v", artifactEndpoint), + fmt.Sprintf("--provision_endpoint=%v", wk.Endpoint()), + fmt.Sprintf("--logging_endpoint=%v", wk.Endpoint()), + } ccr, err := cli.ContainerCreate(ctx, &container.Config{ Image: dp.GetContainerImage(), - Cmd: []string{ - fmt.Sprintf("--id=%v", wk.ID), - fmt.Sprintf("--control_endpoint=%v", wk.Endpoint()), - fmt.Sprintf("--artifact_endpoint=%v", artifactEndpoint), - fmt.Sprintf("--provision_endpoint=%v", wk.Endpoint()), - fmt.Sprintf("--logging_endpoint=%v", wk.Endpoint()), - }, - Env: envs, - Tty: false, + Cmd: cmd, + Env: envs, + Tty: false, }, &container.HostConfig{ NetworkMode: "host", Mounts: mounts, @@ -236,6 +238,7 @@ func dockerEnvironment(ctx context.Context, logger *slog.Logger, dp *pipepb.Dock } logger.Debug("container started") + logger.Debug("container start command", "cmd", cmd) // Start goroutine to wait on container state. go func() { @@ -257,7 +260,12 @@ func dockerEnvironment(ctx context.Context, logger *slog.Logger, dp *pipepb.Dock defer rc.Close() var buf bytes.Buffer stdcopy.StdCopy(&buf, &buf, rc) - logger.Info("container being killed", slog.Any("cause", context.Cause(ctx)), slog.Any("containerLog", buf)) + logger.Info("container being killed", slog.Any("cause", context.Cause(ctx))) + msgs, err := strconv.Unquote(buf.String()) + if err != nil { + msgs = buf.String() + } + logger.Debug("container log", "log", msgs) } // Can't use command context, since it's already canceled here. if err := cli.ContainerKill(bgctx, containerID, ""); err != nil { @@ -273,6 +281,7 @@ func dockerEnvironment(ctx context.Context, logger *slog.Logger, dp *pipepb.Dock rc, err := cli.ContainerLogs(bgctx, containerID, container.LogsOptions{Details: true, ShowStdout: true, ShowStderr: true}) if err != nil { logger.Error("docker container logs error", "error", err) + return } defer rc.Close() var buf bytes.Buffer @@ -284,8 +293,11 @@ func dockerEnvironment(ctx context.Context, logger *slog.Logger, dp *pipepb.Dock return nil } -func processEnvironment(ctx context.Context, pp *pipepb.ProcessPayload, wk *worker.W) { - cmd := exec.CommandContext(ctx, pp.GetCommand(), "--id="+wk.ID, "--provision_endpoint="+wk.Endpoint()) +func processEnvironment(ctx context.Context, logger *slog.Logger, pp *pipepb.ProcessPayload, wk *worker.W) { + defer wk.Stop() + + cmd := exec.CommandContext(ctx, pp.GetCommand(), "--id='"+wk.ID+"'", "--provision_endpoint="+wk.Endpoint()) + logger.Debug("starting process", "cmd", cmd.String()) cmd.WaitDelay = time.Millisecond * 100 cmd.Stderr = os.Stderr @@ -296,9 +308,12 @@ func processEnvironment(ctx context.Context, pp *pipepb.ProcessPayload, wk *work cmd.Env = append(cmd.Environ(), fmt.Sprintf("%v=%v", k, v)) } if err := cmd.Start(); err != nil { + logger.Error("process failed to start", "error", err) return } // Job processing happens here, but orchestrated by other goroutines // This call blocks until the context is cancelled, or the command exits. - cmd.Wait() + if err := cmd.Wait(); err != nil { + logger.Error("process failed while running", "error", err) + } } diff --git a/sdks/go/pkg/beam/runners/prism/internal/execute.go b/sdks/go/pkg/beam/runners/prism/internal/execute.go index ab041da314dc..772c3a9ebb8b 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/execute.go +++ b/sdks/go/pkg/beam/runners/prism/internal/execute.go @@ -22,6 +22,7 @@ import ( "fmt" "io" "log/slog" + "runtime/debug" "sort" "sync/atomic" "time" @@ -76,6 +77,14 @@ func RunPipeline(j *jobservices.Job) { // any related job resources. defer func() { j.CancelFn(fmt.Errorf("runPipeline returned, cleaning up")) + j.WaitForCleanUp() + }() + + // Add this defer function to capture and log panics. + defer func() { + if e := recover(); e != nil { + j.Failed(fmt.Errorf("pipeline panicked: %v\nStacktrace: %s", e, string(debug.Stack()))) + } }() j.SendMsg("running " + j.String()) @@ -95,7 +104,7 @@ func RunPipeline(j *jobservices.Job) { j.SendMsg("pipeline completed " + j.String()) j.SendMsg("terminating " + j.String()) - j.Done() + j.PendingDone() } type transformExecuter interface { @@ -359,7 +368,11 @@ func executePipeline(ctx context.Context, wks map[string]*worker.W, j *jobservic case rb, ok := <-bundles: if !ok { err := eg.Wait() - j.Logger.Debug("pipeline done!", slog.String("job", j.String()), slog.Any("error", err), slog.Any("topo", topo)) + var topoAttrs []any + for _, s := range topo { + topoAttrs = append(topoAttrs, slog.Any(s.ID, s)) + } + j.Logger.Debug("pipeline done!", slog.String("job", j.String()), slog.Any("error", err), slog.Group("topo", topoAttrs...)) return err } eg.Go(func() error { diff --git a/sdks/go/pkg/beam/runners/prism/internal/handlecombine.go b/sdks/go/pkg/beam/runners/prism/internal/handlecombine.go index 6b336043b8c9..d65ef63cccc9 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/handlecombine.go +++ b/sdks/go/pkg/beam/runners/prism/internal/handlecombine.go @@ -64,43 +64,52 @@ func (h *combine) PrepareTransform(tid string, t *pipepb.PTransform, comps *pipe combineInput := comps.GetPcollections()[onlyInput] ws := comps.GetWindowingStrategies()[combineInput.GetWindowingStrategyId()] - var hasElementCount func(tpb *pipepb.Trigger) bool + var hasTriggerType func(tpb *pipepb.Trigger, targetTriggerType reflect.Type) bool - hasElementCount = func(tpb *pipepb.Trigger) bool { - elCount := false + hasTriggerType = func(tpb *pipepb.Trigger, targetTriggerType reflect.Type) bool { + if tpb == nil { + return false + } switch at := tpb.GetTrigger().(type) { - case *pipepb.Trigger_ElementCount_: - return true case *pipepb.Trigger_AfterAll_: for _, st := range at.AfterAll.GetSubtriggers() { - elCount = elCount || hasElementCount(st) + if hasTriggerType(st, targetTriggerType) { + return true + } } - return elCount + return false case *pipepb.Trigger_AfterAny_: for _, st := range at.AfterAny.GetSubtriggers() { - elCount = elCount || hasElementCount(st) + if hasTriggerType(st, targetTriggerType) { + return true + } } - return elCount + return false case *pipepb.Trigger_AfterEach_: for _, st := range at.AfterEach.GetSubtriggers() { - elCount = elCount || hasElementCount(st) + if hasTriggerType(st, targetTriggerType) { + return true + } } - return elCount + return false case *pipepb.Trigger_AfterEndOfWindow_: - return hasElementCount(at.AfterEndOfWindow.GetEarlyFirings()) || - hasElementCount(at.AfterEndOfWindow.GetLateFirings()) + return hasTriggerType(at.AfterEndOfWindow.GetEarlyFirings(), targetTriggerType) || + hasTriggerType(at.AfterEndOfWindow.GetLateFirings(), targetTriggerType) case *pipepb.Trigger_OrFinally_: - return hasElementCount(at.OrFinally.GetMain()) || - hasElementCount(at.OrFinally.GetFinally()) + return hasTriggerType(at.OrFinally.GetMain(), targetTriggerType) || + hasTriggerType(at.OrFinally.GetFinally(), targetTriggerType) case *pipepb.Trigger_Repeat_: - return hasElementCount(at.Repeat.GetSubtrigger()) + return hasTriggerType(at.Repeat.GetSubtrigger(), targetTriggerType) default: - return false + return reflect.TypeOf(at) == targetTriggerType } } // If we aren't lifting, the "default impl" for combines should be sufficient. - if !h.config.EnableLifting || hasElementCount(ws.GetTrigger()) { + // Disable lifting if there is any TriggerElementCount or TriggerAlways. + if (!h.config.EnableLifting || + hasTriggerType(ws.GetTrigger(), reflect.TypeOf(&pipepb.Trigger_ElementCount_{})) || + hasTriggerType(ws.GetTrigger(), reflect.TypeOf(&pipepb.Trigger_Always_{}))) { return prepareResult{} // Strip the composite layer when lifting is disabled. } diff --git a/sdks/go/pkg/beam/runners/prism/internal/handlecombine_test.go b/sdks/go/pkg/beam/runners/prism/internal/handlecombine_test.go index 7b38daa295ef..26be37e77d17 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/handlecombine_test.go +++ b/sdks/go/pkg/beam/runners/prism/internal/handlecombine_test.go @@ -25,10 +25,14 @@ import ( "google.golang.org/protobuf/testing/protocmp" ) -func TestHandleCombine(t *testing.T) { - undertest := "UnderTest" +func makeWindowingStrategy(trigger *pipepb.Trigger) *pipepb.WindowingStrategy { + return &pipepb.WindowingStrategy{ + Trigger: trigger, + } +} - combineTransform := &pipepb.PTransform{ +func makeCombineTransform(inputPCollectionID string) *pipepb.PTransform { + return &pipepb.PTransform{ UniqueName: "COMBINE", Spec: &pipepb.FunctionSpec{ Urn: urns.TransformCombinePerKey, @@ -41,7 +45,7 @@ func TestHandleCombine(t *testing.T) { }), }, Inputs: map[string]string{ - "input": "combineIn", + "input": inputPCollectionID, }, Outputs: map[string]string{ "input": "combineOut", @@ -51,6 +55,15 @@ func TestHandleCombine(t *testing.T) { "combine_values", }, } +} + +func TestHandleCombine(t *testing.T) { + undertest := "UnderTest" + + combineTransform := makeCombineTransform("combineIn") + combineTransformWithTriggerElementCount := makeCombineTransform("combineInWithTriggerElementCount") + combineTransformWithTriggerAlways := makeCombineTransform("combineInWithTriggerAlways") + combineValuesTransform := &pipepb.PTransform{ UniqueName: "combine_values", Subtransforms: []string{ @@ -64,6 +77,14 @@ func TestHandleCombine(t *testing.T) { "combineOut": { CoderId: "outputCoder", }, + "combineInWithTriggerElementCount": { + CoderId: "inputCoder", + WindowingStrategyId: "wsElementCount", + }, + "combineInWithTriggerAlways": { + CoderId: "inputCoder", + WindowingStrategyId: "wsAlways", + }, } baseCoderMap := map[string]*pipepb.Coder{ "int": { @@ -84,7 +105,20 @@ func TestHandleCombine(t *testing.T) { ComponentCoderIds: []string{"int", "string"}, }, } - + baseWindowingStrategyMap := map[string]*pipepb.WindowingStrategy{ + "wsElementCount": makeWindowingStrategy(&pipepb.Trigger{ + Trigger: &pipepb.Trigger_ElementCount_{ + ElementCount: &pipepb.Trigger_ElementCount{ + ElementCount: 10, + }, + }, + }), + "wsAlways": makeWindowingStrategy(&pipepb.Trigger{ + Trigger: &pipepb.Trigger_Always_{ + Always: &pipepb.Trigger_Always{}, + }, + }), + } tests := []struct { name string lifted bool @@ -188,6 +222,32 @@ func TestHandleCombine(t *testing.T) { }, }, }, + }, { + name: "noLift_triggerElementCount", + lifted: true, // Lifting is enabled, but should be disabled in the present of the trigger + comps: &pipepb.Components{ + Transforms: map[string]*pipepb.PTransform{ + undertest: combineTransformWithTriggerElementCount, + "combine_values": combineValuesTransform, + }, + Pcollections: basePCollectionMap, + Coders: baseCoderMap, + WindowingStrategies: baseWindowingStrategyMap, + }, + want: prepareResult{}, + }, { + name: "noLift_triggerAlways", + lifted: true, // Lifting is enabled, but should be disabled in the present of the trigger + comps: &pipepb.Components{ + Transforms: map[string]*pipepb.PTransform{ + undertest: combineTransformWithTriggerAlways, + "combine_values": combineValuesTransform, + }, + Pcollections: basePCollectionMap, + Coders: baseCoderMap, + WindowingStrategies: baseWindowingStrategyMap, + }, + want: prepareResult{}, }, } for _, test := range tests { diff --git a/sdks/go/pkg/beam/runners/prism/internal/jobservices/job.go b/sdks/go/pkg/beam/runners/prism/internal/jobservices/job.go index f186b11fd1d8..c0ba7d2ee5ec 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/jobservices/job.go +++ b/sdks/go/pkg/beam/runners/prism/internal/jobservices/job.go @@ -94,6 +94,8 @@ type Job struct { // Logger for this job. Logger *slog.Logger + pendingDone atomic.Bool // indicate the job is done but waiting for clean-up + metrics metricsStore mw *worker.MultiplexW } @@ -194,6 +196,20 @@ func (j *Job) Canceled() { j.sendState(jobpb.JobState_CANCELLED) } +// PendingDone indicates that the job is completed and is waiting for clean-up. +func (j *Job) PendingDone() { + j.pendingDone.Store(true) +} + +// WaitForCleanUp waits until all environments relevant to the job are cleaned up. +func (j *Job) WaitForCleanUp() { + j.mw.WaitForCleanUp(j.String()) + if j.pendingDone.Load() { + // If there is a pending done, only mark it as done after clean-up + j.Done() + } +} + // Failed indicates that the job completed unsuccessfully. func (j *Job) Failed(err error) { slog.Error("job failed", slog.Any("job", j), slog.Any("error", err)) @@ -204,11 +220,11 @@ func (j *Job) Failed(err error) { // MakeWorker instantiates a worker.W populating environment and pipeline data from the Job. func (j *Job) MakeWorker(env string) *worker.W { - wk := j.mw.MakeWorker(j.String()+"_"+env, env) + wk := j.mw.MakeWorker(j.String(), env) wk.EnvPb = j.Pipeline.GetComponents().GetEnvironments()[env] wk.PipelineOptions = j.PipelineOptions() wk.JobKey = j.JobKey() - wk.ArtifactEndpoint = j.ArtifactEndpoint() + wk.ResolveEndpoints(j.ArtifactEndpoint()) return wk } diff --git a/sdks/go/pkg/beam/runners/prism/internal/stage.go b/sdks/go/pkg/beam/runners/prism/internal/stage.go index 97300cb1122f..101d7a8dc0fa 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/stage.go +++ b/sdks/go/pkg/beam/runners/prism/internal/stage.go @@ -108,6 +108,19 @@ func clampTick(dur time.Duration) time.Duration { } } +func (s *stage) LogValue() slog.Value { + var outAttrs []any + for k, v := range s.OutputsToCoders { + outAttrs = append(outAttrs, slog.Any(k, v)) + } + return slog.GroupValue( + slog.String("ID", s.ID), + slog.Any("transforms", s.transforms), + slog.Any("inputInfo", s.inputInfo), + slog.Group("outputInfo", outAttrs...), + ) +} + func (s *stage) Execute(ctx context.Context, j *jobservices.Job, wk *worker.W, comps *pipepb.Components, em *engine.ElementManager, rb engine.RunBundle) (err error) { if s.baseProgTick.Load() == nil { s.baseProgTick.Store(minimumProgTick) diff --git a/sdks/go/pkg/beam/runners/prism/internal/unimplemented_test.go b/sdks/go/pkg/beam/runners/prism/internal/unimplemented_test.go index 185940eada14..7a742c22d0fb 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/unimplemented_test.go +++ b/sdks/go/pkg/beam/runners/prism/internal/unimplemented_test.go @@ -49,7 +49,6 @@ func TestUnimplemented(t *testing.T) { // See https://github.com/apache/beam/issues/31153. {pipeline: primitives.TriggerElementCount}, {pipeline: primitives.TriggerOrFinally}, - {pipeline: primitives.TriggerAlways}, // Currently unimplemented triggers. // https://github.com/apache/beam/issues/31438 @@ -87,6 +86,7 @@ func TestImplemented(t *testing.T) { {pipeline: primitives.ParDoProcessElementBundleFinalizer}, {pipeline: primitives.TriggerNever}, + {pipeline: primitives.TriggerAlways}, {pipeline: primitives.Panes}, {pipeline: primitives.TriggerAfterAll}, {pipeline: primitives.TriggerAfterAny}, diff --git a/sdks/go/pkg/beam/runners/prism/internal/worker/worker.go b/sdks/go/pkg/beam/runners/prism/internal/worker/worker.go index b4133b0332a6..c962aa4bff6f 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/worker/worker.go +++ b/sdks/go/pkg/beam/runners/prism/internal/worker/worker.go @@ -24,8 +24,12 @@ import ( "io" "log/slog" "net" + "os" + "runtime" + "strings" "sync" "sync/atomic" + "time" "github.com/apache/beam/sdks/v2/go/pkg/beam/core" "github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/coder" @@ -58,9 +62,9 @@ type W struct { ID, Env string - JobKey, ArtifactEndpoint string - EnvPb *pipepb.Environment - PipelineOptions *structpb.Struct + JobKey, ArtifactEndpoint, endpoint string + EnvPb *pipepb.Environment + PipelineOptions *structpb.Struct // These are the ID sources inst uint64 @@ -73,14 +77,39 @@ type W struct { mu sync.Mutex activeInstructions map[string]controlResponder // Active instructions keyed by InstructionID Descriptors map[string]*fnpb.ProcessBundleDescriptor // Stages keyed by PBDID + wg *sync.WaitGroup } type controlResponder interface { Respond(*fnpb.InstructionResponse) } +// resolveEndpoint checks if the worker is running inside a docker container on mac or Windows and +// if the endpoint is a "localhost" endpoint. If so, overrides it with "host.docker.internal". +// Reference: https://docs.docker.com/desktop/features/networking/#networking-mode-and-dns-behaviour-for-mac-and-windows +func (wk *W) resolveEndpoint(endpoint string) string { + // The presence of an external environment does not guarantee execution within + // Docker, as Python's LOOPBACK also runs in an external environment. + // A specific check for the "BEAM_WORKER_POOL_IN_DOCKER_VM" environment variable is required to confirm + // if the worker is running inside a Docker container. + // Python LOOPBACK mode: https://github.com/apache/beam/blob/0589b14812ec52bff9d20d3bfcd96da393b9ebdb/sdks/python/apache_beam/runners/portability/portable_runner.py#L397 + // External Environment: https://beam.apache.org/documentation/runtime/sdk-harness-config/ + + workerInDocker := wk.EnvPb.GetUrn() == urns.EnvDocker || + (wk.EnvPb.GetUrn() == urns.EnvExternal && (os.Getenv("BEAM_WORKER_POOL_IN_DOCKER_VM") == "1")) + if runtime.GOOS != "linux" && workerInDocker && strings.HasPrefix(endpoint, "localhost:") { + return "host.docker.internal:" + strings.TrimPrefix(endpoint, "localhost:") + } + return endpoint +} + +func (wk *W) ResolveEndpoints(artifactEndpoint string) { + wk.ArtifactEndpoint = wk.resolveEndpoint(artifactEndpoint) + wk.endpoint = wk.resolveEndpoint(wk.parentPool.endpoint) +} + func (wk *W) Endpoint() string { - return wk.parentPool.endpoint + return wk.endpoint } func (wk *W) String() string { @@ -115,6 +144,7 @@ func (wk *W) shutdown() { func (wk *W) Stop() { wk.shutdown() wk.parentPool.delete(wk) + wk.wg.Done() slog.Debug("stopped", "worker", wk) } @@ -129,6 +159,14 @@ func (wk *W) GetProvisionInfo(_ context.Context, _ *fnpb.GetProvisionInfoRequest endpoint := &pipepb.ApiServiceDescriptor{ Url: wk.Endpoint(), } + + var rt string + if len(wk.EnvPb.GetDependencies()) > 0 { + rt = wk.JobKey + } else { + rt = "__no_artifacts_staged__" + } + resp := &fnpb.GetProvisionInfoResponse{ Info: &fnpb.ProvisionInfo{ // TODO: Include runner capabilities with the per job configuration. @@ -141,7 +179,7 @@ func (wk *W) GetProvisionInfo(_ context.Context, _ *fnpb.GetProvisionInfoRequest Url: wk.ArtifactEndpoint, }, - RetrievalToken: wk.JobKey, + RetrievalToken: rt, Dependencies: wk.EnvPb.GetDependencies(), PipelineOptions: wk.PipelineOptions, @@ -674,6 +712,7 @@ type MultiplexW struct { endpoint string logger *slog.Logger pool map[string]*W + wg map[string]*sync.WaitGroup } // NewMultiplexW instantiates a new FnAPI server for multiplexing FnAPI requests to a W. @@ -683,6 +722,7 @@ func NewMultiplexW(lis net.Listener, g *grpc.Server, logger *slog.Logger) *Multi endpoint: "localhost:" + p, logger: logger, pool: make(map[string]*W), + wg: make(map[string]*sync.WaitGroup), } fnpb.RegisterBeamFnControlServer(g, mw) @@ -700,8 +740,12 @@ func NewMultiplexW(lis net.Listener, g *grpc.Server, logger *slog.Logger) *Multi func (mw *MultiplexW) MakeWorker(id, env string) *W { mw.mu.Lock() defer mw.mu.Unlock() + workerId := id + "_" + env + if _, ok := mw.wg[id]; !ok { + mw.wg[id] = &sync.WaitGroup{} + } w := &W{ - ID: id, + ID: workerId, Env: env, InstReqs: make(chan *fnpb.InstructionRequest, 10), @@ -711,8 +755,11 @@ func (mw *MultiplexW) MakeWorker(id, env string) *W { activeInstructions: make(map[string]controlResponder), Descriptors: make(map[string]*fnpb.ProcessBundleDescriptor), parentPool: mw, + wg: mw.wg[id], } - mw.pool[id] = w + mw.pool[workerId] = w + + mw.wg[id].Add(1) return w } @@ -774,6 +821,32 @@ func (mw *MultiplexW) delete(w *W) { delete(mw.pool, w.ID) } +// WaitForCleanUp waits until all resources relevant to the job are cleaned up. +func (mw *MultiplexW) WaitForCleanUp(id string) { + mw.mu.Lock() + wg := mw.wg[id] + mw.mu.Unlock() + if wg == nil { + return + } + + const cleanUpTimeout = 60 * time.Second + c := make(chan struct{}) + go func() { + defer close(c) + wg.Wait() + }() + + select { + case <-c: // Waitgroup finishes successfully + slog.Debug("Finished cleaning up job " + id) + return + case <-time.After(cleanUpTimeout): // Timeout + slog.Warn("Timeout when cleaning up job " + id) + return + } +} + func handleUnary[Request any, Response any, Method func(*W, context.Context, *Request) (*Response, error)](mw *MultiplexW, ctx context.Context, req *Request, m Method) (*Response, error) { w, err := mw.workerFromMetadataCtx(ctx) if err != nil { diff --git a/sdks/go/pkg/beam/runners/prism/internal/worker/worker_test.go b/sdks/go/pkg/beam/runners/prism/internal/worker/worker_test.go index a0cf577fbdba..76a05563ec38 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/worker/worker_test.go +++ b/sdks/go/pkg/beam/runners/prism/internal/worker/worker_test.go @@ -44,7 +44,7 @@ func TestMultiplexW_MakeWorker(t *testing.T) { if w.parentPool == nil { t.Errorf("MakeWorker instantiated W with a nil reference to MultiplexW") } - if got, want := w.ID, "test"; got != want { + if got, want := w.ID, "test_testEnv"; got != want { t.Errorf("MakeWorker(%q) = %v, want %v", want, got, want) } got, ok := w.parentPool.pool[w.ID] @@ -77,8 +77,8 @@ func TestMultiplexW_workerFromMetadataCtx(t *testing.T) { }, { name: "matched worker_id", - ctx: metadata.NewIncomingContext(context.Background(), metadata.Pairs("worker_id", "test")), - want: &W{ID: "test"}, + ctx: metadata.NewIncomingContext(context.Background(), metadata.Pairs("worker_id", "test_testEnv")), + want: &W{ID: "test_testEnv"}, }, } { t.Run(tt.name, func(t *testing.T) { @@ -525,6 +525,7 @@ func TestWorker_State_MultimapSideInput(t *testing.T) { func newWorker() *W { mw := &MultiplexW{ pool: map[string]*W{}, + wg: map[string]*sync.WaitGroup{}, } return mw.MakeWorker("test", "testEnv") } diff --git a/sdks/go/test/integration/expansions.go b/sdks/go/test/integration/expansions.go index 7e8c1164f506..633f88d02930 100644 --- a/sdks/go/test/integration/expansions.go +++ b/sdks/go/test/integration/expansions.go @@ -17,6 +17,7 @@ package integration import ( "fmt" + "net" "strconv" "time" @@ -57,6 +58,7 @@ type ExpansionServices struct { // Callback for running jars, stored this way for testing purposes. run func(time.Duration, string, ...string) (jars.Process, error) waitTime time.Duration // Time to sleep after running jar. Tests can adjust this. + testMode bool // Skip connectivity checks when in test mode } // NewExpansionServices creates and initializes an ExpansionServices instance. @@ -67,6 +69,7 @@ func NewExpansionServices() *ExpansionServices { procs: make([]jars.Process, 0), run: jars.Run, waitTime: 3 * time.Second, + testMode: false, } } @@ -100,9 +103,33 @@ func (es *ExpansionServices) GetAddr(label string) (string, error) { if err != nil { return "", fmt.Errorf("cannot run jar for expansion service labeled \"%s\": %w", label, err) } - time.Sleep(es.waitTime) // Wait a bit for the jar to start. - es.procs = append(es.procs, proc) + addr := "localhost:" + portStr + + // Use different wait strategies for test mode vs production + if es.testMode { + // In test mode, use simple wait time for compatibility with mock processes + time.Sleep(es.waitTime) + } else { + // In production, wait for the jar to start with improved retry logic + maxRetries := 30 + retryDelay := time.Second + + for i := 0; i < maxRetries; i++ { + time.Sleep(retryDelay) + // Try to connect to the expansion service to verify it's ready + conn, err := net.DialTimeout("tcp", addr, 2*time.Second) + if err == nil { + conn.Close() + break + } + if i == maxRetries-1 { + return "", fmt.Errorf("expansion service labeled \"%s\" failed to start after %d retries: %w", label, maxRetries, err) + } + } + } + + es.procs = append(es.procs, proc) es.addrs[label] = addr return addr, nil } diff --git a/sdks/go/test/integration/expansions_test.go b/sdks/go/test/integration/expansions_test.go index 99878d0623fd..3afa2470157c 100644 --- a/sdks/go/test/integration/expansions_test.go +++ b/sdks/go/test/integration/expansions_test.go @@ -63,6 +63,7 @@ func TestExpansionServices_GetAddr_Addresses(t *testing.T) { procs: make([]jars.Process, 0), run: failRun, waitTime: 0, + testMode: true, } // Ensure we get the same map we put in, and that addresses take priority over jars if @@ -97,6 +98,7 @@ func TestExpansionServices_GetAddr_Jars(t *testing.T) { procs: make([]jars.Process, 0), run: succeedRun, waitTime: 0, + testMode: true, } // Call GetAddr on each jar twice, checking that the addresses remain consistent. @@ -151,6 +153,7 @@ func TestExpansionServices_Shutdown(t *testing.T) { procs: make([]jars.Process, 0), run: succeedRun, waitTime: 0, + testMode: true, } // Call getAddr on each label to run jars. for label := range addrsMap { diff --git a/sdks/go/test/integration/integration.go b/sdks/go/test/integration/integration.go index 88db6a5b6c3b..8d951fe8ce96 100644 --- a/sdks/go/test/integration/integration.go +++ b/sdks/go/test/integration/integration.go @@ -98,6 +98,7 @@ var directFilters = []string{ "TestValueStateClear", "TestBagState", "TestBagStateClear", + "TestBagStateBlindWrite", "TestCombiningState", "TestMapState", "TestMapStateClear", @@ -240,6 +241,7 @@ var samzaFilters = []string{ // Samza does not support state. "TestTimers.*", + "TestBagStateBlindWrite", // no support for BundleFinalizer "TestParDoBundleFinalizer.*", diff --git a/sdks/go/test/integration/primitives/state.go b/sdks/go/test/integration/primitives/state.go index acf1bf8fa665..6b672acc27bd 100644 --- a/sdks/go/test/integration/primitives/state.go +++ b/sdks/go/test/integration/primitives/state.go @@ -34,6 +34,7 @@ func init() { register.DoFn3x1[state.Provider, string, int, string](&valueStateClearFn{}) register.DoFn3x1[state.Provider, string, int, string](&bagStateFn{}) register.DoFn3x1[state.Provider, string, int, string](&bagStateClearFn{}) + register.DoFn3x1[state.Provider, string, int, string](&bagStateBlindWriteFn{}) register.DoFn3x1[state.Provider, string, int, string](&combiningStateFn{}) register.DoFn3x1[state.Provider, string, int, string](&mapStateFn{}) register.DoFn3x1[state.Provider, string, int, string](&mapStateClearFn{}) @@ -211,6 +212,45 @@ func BagStateParDoClear(s beam.Scope) { passert.Equals(s, counts, "apple: 0", "pear: 0", "apple: 1", "apple: 2", "pear: 1", "apple: 3", "apple: 0", "pear: 2", "pear: 3", "pear: 0", "apple: 1", "pear: 1") } +type bagStateBlindWriteFn struct { + State1 state.Bag[int] +} + +func (f *bagStateBlindWriteFn) ProcessElement(s state.Provider, w string, c int) string { + err := f.State1.Add(s, 1) + if err != nil { + panic(err) + } + i, ok, err := f.State1.Read(s) + if err != nil { + panic(err) + } + if !ok { + i = []int{} + } + sum := 0 + for _, val := range i { + sum += val + } + + // Bonus "non-blind" write + err = f.State1.Add(s, 1) + if err != nil { + panic(err) + } + + return fmt.Sprintf("%s: %v", w, sum) +} + +// BagStateBlindWriteParDo tests a DoFn that uses bag state, but performs a +// blind write to the state before reading. +func BagStateBlindWriteParDo(s beam.Scope) { + in := beam.Create(s, "apple", "pear", "peach", "apple", "apple", "pear") + keyed := beam.ParDo(s, pairWithOne, in) + counts := beam.ParDo(s, &bagStateBlindWriteFn{}, keyed) + passert.Equals(s, counts, "apple: 1", "pear: 1", "peach: 1", "apple: 3", "apple: 5", "pear: 3") +} + type combiningStateFn struct { State0 state.Combining[int, int, int] State1 state.Combining[int, int, int] diff --git a/sdks/go/test/integration/primitives/state_test.go b/sdks/go/test/integration/primitives/state_test.go index 79cb8c1839fc..1d1d4860e8f9 100644 --- a/sdks/go/test/integration/primitives/state_test.go +++ b/sdks/go/test/integration/primitives/state_test.go @@ -47,6 +47,11 @@ func TestBagStateClear(t *testing.T) { ptest.BuildAndRun(t, BagStateParDoClear) } +func TestBagStateBlindWrite(t *testing.T) { + integration.CheckFilters(t) + ptest.BuildAndRun(t, BagStateBlindWriteParDo) +} + func TestCombiningState(t *testing.T) { integration.CheckFilters(t) ptest.BuildAndRun(t, CombiningStateParDo) diff --git a/sdks/go/test/run_validatesrunner_tests.sh b/sdks/go/test/run_validatesrunner_tests.sh index 2e1b5fa3f396..be7a795f01a5 100755 --- a/sdks/go/test/run_validatesrunner_tests.sh +++ b/sdks/go/test/run_validatesrunner_tests.sh @@ -351,6 +351,33 @@ fi if [[ "$RUNNER" == "dataflow" ]]; then # Verify docker and gcloud commands exist command -v docker + # Check if Docker daemon is running + if ! docker info >/dev/null 2>&1; then + echo "Warning: Docker daemon is not running. Starting Docker..." + # Try to start Docker daemon (this may require sudo on some systems) + if command -v systemctl >/dev/null 2>&1; then + sudo systemctl start docker || echo "Failed to start Docker daemon via systemctl" + elif command -v service >/dev/null 2>&1; then + sudo service docker start || echo "Failed to start Docker daemon via service" + else + echo "Please start Docker daemon manually" + exit 1 + fi + # Wait for Docker daemon to be ready + for i in {1..30}; do + if docker info >/dev/null 2>&1; then + echo "Docker daemon is now running" + break + fi + echo "Waiting for Docker daemon to start... ($i/30)" + sleep 2 + done + # Final check + if ! docker info >/dev/null 2>&1; then + echo "Error: Docker daemon failed to start. Please start it manually." + exit 1 + fi + fi docker -v command -v gcloud gcloud --version diff --git a/sdks/java/container/Dockerfile b/sdks/java/container/Dockerfile index 9c266ea132b8..c43eb0cb8c02 100644 --- a/sdks/java/container/Dockerfile +++ b/sdks/java/container/Dockerfile @@ -34,7 +34,7 @@ ADD target/beam-sdks-java-harness.jar /opt/apache/beam/jars/ # Required to use jamm as a javaagent to get accurate object size measuring # COPY fails if file is not found, so use a wildcard for open-module-agent.jar # since it is only included in Java 9+ containers -COPY target/jamm.jar target/open-module-agent*.jar /opt/apache/beam/jars/ +COPY target/jamm.jar target/open-module-agent.jar /opt/apache/beam/jars/ COPY target/${TARGETOS}_${TARGETARCH}/boot /opt/apache/beam/ diff --git a/sdks/java/container/boot.go b/sdks/java/container/boot.go index 1f574d251cb3..20283740ca0f 100644 --- a/sdks/java/container/boot.go +++ b/sdks/java/container/boot.go @@ -227,9 +227,9 @@ func main() { if pipelineOptions, ok := info.GetPipelineOptions().GetFields()["options"]; ok { if heapDumpOption, ok := pipelineOptions.GetStructValue().GetFields()["enableHeapDumps"]; ok { if heapDumpOption.GetBoolValue() { - args = append(args, "-XX:+HeapDumpOnOutOfMemoryError", - "-Dbeam.fn.heap_dump_dir="+filepath.Join(dir, "heapdumps"), - "-XX:HeapDumpPath="+filepath.Join(dir, "heapdumps", "heap_dump.hprof")) + args = append(args, "-XX:+HeapDumpOnOutOfMemoryError", + "-Dbeam.fn.heap_dump_dir="+filepath.Join(dir, "heapdumps"), + "-XX:HeapDumpPath="+filepath.Join(dir, "heapdumps", "heap_dump.hprof")) } } } @@ -237,9 +237,10 @@ func main() { // Apply meta options const metaDir = "/opt/apache/beam/options" - // Note: Error is unchecked, so parsing errors won't abort container. - // TODO: verify if it's intentional or not. - metaOptions, _ := LoadMetaOptions(ctx, logger, metaDir) + metaOptions, err := LoadMetaOptions(ctx, logger, metaDir) + if err != nil { + logger.Errorf(ctx, "LoadMetaOptions failed: %v", err) + } javaOptions := BuildOptions(ctx, logger, metaOptions) // (1) Add custom jvm arguments: "-server -Xmx1324 -XXfoo .." diff --git a/sdks/java/container/common.gradle b/sdks/java/container/common.gradle index acb6b79b3462..c81a33827bef 100644 --- a/sdks/java/container/common.gradle +++ b/sdks/java/container/common.gradle @@ -52,9 +52,7 @@ task copyDockerfileDependencies(type: Copy) { rename 'jcl-over-slf4j.*', 'jcl-over-slf4j.jar' rename 'log4j-over-slf4j.*', 'log4j-over-slf4j.jar' rename 'log4j-to-slf4j.*', 'log4j-to-slf4j.jar' - if (imageJavaVersion == "11" || imageJavaVersion == "17") { - rename 'beam-sdks-java-container-agent.*.jar', 'open-module-agent.jar' - } + rename 'beam-sdks-java-container-agent.*.jar', 'open-module-agent.jar' rename 'beam-sdks-java-harness-.*.jar', 'beam-sdks-java-harness.jar' rename 'jamm.*.jar', 'jamm.jar' @@ -84,9 +82,7 @@ task copyGolangLicenses(type: Copy) { } task copyJdkOptions(type: Copy) { - if (["11", "17", "21"].contains(imageJavaVersion)) { - from "option-jamm.json" - } + from "option-jamm.json" from "java${imageJavaVersion}-security.properties" from "option-java${imageJavaVersion}-security.json" into "build/target/options" @@ -97,33 +93,6 @@ task skipPullLicenses(type: Exec) { args "-c", "mkdir -p build/target/go-licenses build/target/options build/target/third_party_licenses && touch build/target/go-licenses/skip && touch build/target/third_party_licenses/skip" } -// Java11+ container depends on the java agent project. To compile it, need a compatible JDK version: -// lower bound 11 and upper bound imageJavaVersion -task validateJavaHome { - def requiredForVer = ["11", "17", "21"] - if (requiredForVer.contains(imageJavaVersion)) { - doFirst { - if (JavaVersion.VERSION_1_8.compareTo(JavaVersion.current()) < 0) { - return - } - boolean propertyFound = false - // enable to build agent with compatible java versions (11-requiredForVer) - for (def checkVer : requiredForVer) { - if (project.hasProperty("java${checkVer}Home")) { - propertyFound = true - } - if (checkVer == imageJavaVersion) { - // cannot build agent with a higher version than the docker java ver - break - } - } - if (!propertyFound) { - throw new GradleException("System Java needs to have version 11+ or java${imageJavaVersion}Home required for imageJavaVersion=${imageJavaVersion}. Re-run with -Pjava${imageJavaVersion}Home") - } - } - } -} - def pushContainers = project.rootProject.hasProperty(["isRelease"]) || project.rootProject.hasProperty("push-containers") docker { @@ -162,4 +131,3 @@ if (project.rootProject.hasProperty("docker-pull-licenses") || dockerPrepare.dependsOn copySdkHarnessLauncher dockerPrepare.dependsOn copyDockerfileDependencies dockerPrepare.dependsOn copyJdkOptions -dockerPrepare.dependsOn validateJavaHome diff --git a/sdks/java/container/license_scripts/dep_urls_java.yaml b/sdks/java/container/license_scripts/dep_urls_java.yaml index 4f9f50725def..93f5f6fa211f 100644 --- a/sdks/java/container/license_scripts/dep_urls_java.yaml +++ b/sdks/java/container/license_scripts/dep_urls_java.yaml @@ -46,7 +46,7 @@ jaxen: '1.1.6': type: "3-Clause BSD" libraries-bom: - '26.62.0': + '26.65.0': license: "https://raw.githubusercontent.com/GoogleCloudPlatform/cloud-opensource-java/master/LICENSE" type: "Apache License 2.0" paranamer: diff --git a/sdks/java/core/build.gradle b/sdks/java/core/build.gradle index a1f2916f1958..e849ae597791 100644 --- a/sdks/java/core/build.gradle +++ b/sdks/java/core/build.gradle @@ -130,3 +130,11 @@ project.tasks.compileTestJava { // TODO: fix other places with warnings in tests and delete this option options.compilerArgs += ['-Xlint:-rawtypes'] } + +// Configure test task to use JUnit 4. JUnit 5 support is provided in module +// sdks/java/testing/junit, which configures useJUnitPlatform(). Submodules that +// need to run both JUnit 4 and 5 via the JUnit Platform must also add the +// Vintage engine explicitly. +test { + useJUnit() +} diff --git a/sdks/java/core/jmh/src/main/java/org/apache/beam/sdk/jmh/schemas/RowBundles.java b/sdks/java/core/jmh/src/main/java/org/apache/beam/sdk/jmh/schemas/RowBundles.java index a1a8ca7f3af2..572bc3985d2b 100644 --- a/sdks/java/core/jmh/src/main/java/org/apache/beam/sdk/jmh/schemas/RowBundles.java +++ b/sdks/java/core/jmh/src/main/java/org/apache/beam/sdk/jmh/schemas/RowBundles.java @@ -28,6 +28,7 @@ import org.openjdk.jmh.annotations.State; import org.openjdk.jmh.infra.Blackhole; +@SuppressWarnings("SameNameButDifferent") public interface RowBundles { @State(Scope.Benchmark) class IntBundle extends RowBundle { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexer.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexer.java index a6044b931e68..8fec8b455cce 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexer.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexer.java @@ -208,6 +208,7 @@ public void close() throws Exception { * it is ready to consume that data. */ private final class InboundObserver implements StreamObserver { + @SuppressWarnings("LabelledBreakTarget") @Override public void onNext(BeamFnApi.Elements value) { // Have a fast path to handle the common case and provide a short circuit to exit if we detect diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataInboundObserver.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataInboundObserver.java index ee4a36f0171a..54fe42adefee 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataInboundObserver.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataInboundObserver.java @@ -68,6 +68,7 @@ private static class EndpointStatus { transformIdToTimerFamilyIdToTimerEndpoint; private final CancellableQueue queue; // We use a custom exception for closing to avoid the expense of stack trace generation. + @SuppressWarnings("StaticAssignmentOfThrowable") protected static class CloseException extends Exception { private CloseException() { super( diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/server/GrpcContextHeaderAccessorProvider.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/server/GrpcContextHeaderAccessorProvider.java index a0ac89313b12..6288ceba4cd1 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/server/GrpcContextHeaderAccessorProvider.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/server/GrpcContextHeaderAccessorProvider.java @@ -66,7 +66,7 @@ public static HeaderAccessor getHeaderAccessor() { private static class GrpcHeaderAccessor implements HeaderAccessor { @Override - /** This method should be called from the request method. */ + // This method should be called from the request method. public String getSdkWorkerId() { return Preconditions.checkNotNull( SDK_WORKER_CONTEXT_KEY.get(), "No worker_id header provided in client headers."); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java index d5c235b696ca..cfa06f3cf0d5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java @@ -373,6 +373,7 @@ public static Match match() { public static MatchAll matchAll() { return new AutoValue_FileIO_MatchAll.Builder() .setConfiguration(MatchConfiguration.create(EmptyMatchTreatment.ALLOW_IF_WILDCARD)) + .setOutputParallelization(true) .build(); } @@ -677,12 +678,18 @@ abstract static class Builder { abstract Builder setConfiguration(MatchConfiguration configuration); abstract MatchAll build(); + + abstract Builder setOutputParallelization(boolean b); } /** Like {@link Match#withConfiguration}. */ public MatchAll withConfiguration(MatchConfiguration configuration) { return toBuilder().setConfiguration(configuration).build(); } + /** Like {@link Match#withOutputParallelization}. */ + public MatchAll withOutputParallelization(boolean outputParallelization) { + return toBuilder().setOutputParallelization(outputParallelization).build(); + } /** Like {@link Match#withEmptyMatchTreatment}. */ public MatchAll withEmptyMatchTreatment(EmptyMatchTreatment treatment) { @@ -723,8 +730,15 @@ public PCollection expand(PCollection input) { res = input.apply(createWatchTransform(new ExtractFilenameFn())).apply(Values.create()); } } - return res.apply(Reshuffle.viaRandomKey()); + // Apply Reshuffle conditionally based on the flag + if (getOutputParallelization()) { + return res.apply(Reshuffle.viaRandomKey()); + } else { + return res; + } } + /** Returns whether to avoid the reshuffle operation. */ + public abstract boolean getOutputParallelization(); @Override public void populateDisplayData(DisplayData.Builder builder) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java index dbdbf6b2f72a..8b0e4ee433fa 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java @@ -401,6 +401,7 @@ private boolean tryClaimOrThrow(TimestampedValue[] position) throws IOExcepti return true; } + @SuppressWarnings("Finalize") @Override protected void finalize() throws Throwable { if (currentReader != null) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java index 8404f15842ee..b0b5051f3210 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java @@ -689,7 +689,7 @@ private class WriteUnshardedTempFilesFn extends DoFn parseCommandLine( if (strictParsing) { throw e; } else { - LOG.warn( - "Strict parsing is disabled, ignoring option '{}' because {}", arg, e.getMessage()); + LOG.warn("Strict parsing is disabled, ignoring option '{}'", arg, e); } } } @@ -1954,10 +1953,10 @@ private static Map parseObjects( throw e; } else { LOG.warn( - "Strict parsing is disabled, ignoring option '{}' with value '{}' because {}", + "Strict parsing is disabled, ignoring option '{}' with value '{}'", entry.getKey(), entry.getValue(), - e.getMessage()); + e); } } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/SdkHarnessOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/SdkHarnessOptions.java index b722947ef817..ad5b1451075c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/SdkHarnessOptions.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/SdkHarnessOptions.java @@ -433,8 +433,9 @@ public Duration create(PipelineOptions options) { * signaling the runner harness to restart the SDK worker. */ @Description( - "The time limit (minute) that an SDK worker allows for a PTransform operation " - + "before signaling the runner harness to restart the SDK worker. There is no time limit if the value is set to 0.") + "The time limit (in minutes) for any PTransform to finish processing a single element." + + " If exceeded, the SDK worker process self-terminates and processing may be restarted by a runner." + + " There is no time limit if the value is set to 0.") @NonNegative int getElementProcessingTimeoutMinutes(); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaTranslation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaTranslation.java index 205d57319f8f..8ad5bb5ff97f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaTranslation.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaTranslation.java @@ -38,6 +38,7 @@ import org.apache.beam.model.pipeline.v1.SchemaApi.LogicalTypeValue; import org.apache.beam.model.pipeline.v1.SchemaApi.MapTypeEntry; import org.apache.beam.model.pipeline.v1.SchemaApi.MapTypeValue; +import org.apache.beam.sdk.coders.RowCoder; import org.apache.beam.sdk.schemas.Schema.Field; import org.apache.beam.sdk.schemas.Schema.FieldType; import org.apache.beam.sdk.schemas.Schema.LogicalType; @@ -326,6 +327,7 @@ public static Schema schemaFromProto(SchemaApi.Schema protoSchema) { if (!protoSchema.getId().isEmpty()) { schema.setUUID(UUID.fromString(protoSchema.getId())); } + overrideEncodingPositions(schema); return schema; } @@ -504,6 +506,50 @@ private static FieldType fieldTypeFromProtoWithoutNullable(SchemaApi.FieldType p } } + private static void overrideEncodingPositions(Schema schema) { + @javax.annotation.Nullable UUID uuid = schema.getUUID(); + if (schema.isEncodingPositionsOverridden() && uuid != null) { + RowCoder.overrideEncodingPositions(uuid, schema.getEncodingPositions()); + } + schema.getFields().stream() + .map(Schema.Field::getType) + .forEach(SchemaTranslation::overrideEncodingPositions); + } + + private static void overrideEncodingPositions(Schema.FieldType fieldType) { + switch (fieldType.getTypeName()) { + case ROW: + overrideEncodingPositions( + org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull(fieldType.getRowSchema())); + break; + case ARRAY: + case ITERABLE: + overrideEncodingPositions( + org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull( + fieldType.getCollectionElementType())); + break; + case MAP: + overrideEncodingPositions( + org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull(fieldType.getMapKeyType())); + overrideEncodingPositions( + org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull( + fieldType.getMapValueType())); + break; + case LOGICAL_TYPE: + Schema.LogicalType logicalType = + (Schema.LogicalType) + org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull( + fieldType.getLogicalType()); + @javax.annotation.Nullable Schema.FieldType argumentType = logicalType.getArgumentType(); + if (argumentType != null) { + overrideEncodingPositions(argumentType); + } + overrideEncodingPositions(logicalType.getBaseType()); + break; + default: + } + } + public static SchemaApi.Row rowToProto(Row row) { SchemaApi.Row.Builder builder = SchemaApi.Row.newBuilder(); for (int i = 0; i < row.getFieldCount(); ++i) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/OneOfType.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/OneOfType.java index 31b6c8db2fed..5c2e376e4bf4 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/OneOfType.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/OneOfType.java @@ -155,12 +155,12 @@ public Value toInputType(Row base) { for (int i = 0; i < base.getFieldCount(); ++i) { Object value = base.getValue(i); if (value != null) { - checkArgument(caseType == null, "More than one field set in union " + this); + checkArgument(caseType == null, "More than one field set in union %s", this); caseType = enumerationType.valueOf(oneOfSchema.getField(i).getName()); oneOfValue = value; } } - checkNotNull(oneOfValue, "No value set in union" + this); + checkNotNull(oneOfValue, "No value set in union %s", this); return createValue(caseType, oneOfValue); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Select.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Select.java index 84ae7c42cb64..86af822a6a4b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Select.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Select.java @@ -131,6 +131,7 @@ private static class SelectDoFn extends DoFn { // TODO: This should be the same as resolved so that Beam knows which fields // are being accessed. Currently Beam only supports wildcard descriptors. // Once https://github.com/apache/beam/issues/18903 is fixed, fix this. + @SuppressWarnings("unused") @FieldAccess("selectFields") final FieldAccessDescriptor fieldAccess = FieldAccessDescriptor.withAllFields(); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java index 328bf19c466c..782471407a2a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java @@ -82,7 +82,11 @@ * * *

Use {@link PAssert} for tests, as it integrates with this test harness in both direct and - * remote execution modes. For example: + * remote execution modes. + * + *

JUnit 4 Usage

+ * + * For JUnit 4 tests, use this class as a TestRule: * *

  * {@literal @Rule}
@@ -97,6 +101,25 @@
  *  }
  * 
* + *

JUnit5 Usage

+ * + * For JUnit5 tests, use {@link TestPipelineExtension} from the module + * sdks/java/testing/junit (artifact org.apache.beam:beam-sdks-java-testing-junit + * ): + * + *

+ * {@literal @ExtendWith}(TestPipelineExtension.class)
+ * class MyPipelineTest {
+ *   {@literal @Test}
+ *   {@literal @Category}(NeedsRunner.class)
+ *   void myPipelineTest(TestPipeline pipeline) {
+ *     final PCollection<String> pCollection = pipeline.apply(...)
+ *     PAssert.that(pCollection).containsInAnyOrder(...);
+ *     pipeline.run();
+ *   }
+ * }
+ * 
+ * *

For pipeline runners, it is required that they must throw an {@link AssertionError} containing * the message from the {@link PAssert} that failed. * @@ -108,7 +131,7 @@ public class TestPipeline extends Pipeline implements TestRule { private final PipelineOptions options; - private static class PipelineRunEnforcement { + static class PipelineRunEnforcement { @SuppressWarnings("WeakerAccess") protected boolean enableAutoRunIfMissing; @@ -117,7 +140,7 @@ private static class PipelineRunEnforcement { protected boolean runAttempted; - private PipelineRunEnforcement(final Pipeline pipeline) { + PipelineRunEnforcement(final Pipeline pipeline) { this.pipeline = pipeline; } @@ -138,7 +161,7 @@ protected void afterUserCodeFinished() { } } - private static class PipelineAbandonedNodeEnforcement extends PipelineRunEnforcement { + static class PipelineAbandonedNodeEnforcement extends PipelineRunEnforcement { // Null until the pipeline has been run private @MonotonicNonNull List runVisitedNodes; @@ -164,7 +187,7 @@ public void visitPrimitiveTransform(final TransformHierarchy.Node node) { } } - private PipelineAbandonedNodeEnforcement(final TestPipeline pipeline) { + PipelineAbandonedNodeEnforcement(final TestPipeline pipeline) { super(pipeline); runVisitedNodes = null; } @@ -574,7 +597,7 @@ public static void verifyPAssertsSucceeded(Pipeline pipeline, PipelineResult pip } } - private static class IsEmptyVisitor extends PipelineVisitor.Defaults { + static class IsEmptyVisitor extends PipelineVisitor.Defaults { private boolean empty = true; public boolean isEmpty() { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestStream.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestStream.java index d63753fc8b34..f26cdd87200c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestStream.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestStream.java @@ -65,7 +65,7 @@ * the {@link Pipeline} before advancing the state of the {@link TestStream}. */ @SuppressWarnings({ - "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) + "rawtypes", // TODO(https://github.com/apache/beam/issues/20447), }) public final class TestStream extends PTransform> { private final List> events; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java index 0961c8512523..10904b2aa393 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java @@ -122,6 +122,12 @@ public abstract class FinishBundleContext { */ public abstract void output(OutputT output, Instant timestamp, BoundedWindow window); + public abstract void output( + OutputT output, + Instant timestamp, + BoundedWindow window, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset); /** * Adds the given element to the output {@code PCollection} with the given tag at the given * timestamp in the given window. @@ -133,6 +139,14 @@ public abstract class FinishBundleContext { */ public abstract void output( TupleTag tag, T output, Instant timestamp, BoundedWindow window); + + public abstract void output( + TupleTag tag, + T output, + Instant timestamp, + BoundedWindow window, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset); } /** @@ -211,6 +225,14 @@ public abstract void outputWindowedValue( Collection windows, PaneInfo paneInfo); + public abstract void outputWindowedValue( + OutputT output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset); + /** * Adds the given element to the output {@code PCollection} with the given tag. * @@ -283,6 +305,15 @@ public abstract void outputWindowedValue( Instant timestamp, Collection windows, PaneInfo paneInfo); + + public abstract void outputWindowedValue( + TupleTag tag, + T output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset); } /** Information accessible when running a {@link DoFn.ProcessElement} method. */ @@ -323,6 +354,12 @@ public abstract class ProcessContext extends WindowedContext { */ @Pure public abstract PaneInfo pane(); + + @Pure + public abstract String currentRecordId(); + + @Pure + public abstract Long currentRecordOffset(); } /** Information accessible when running a {@link DoFn.OnTimer} method. */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnOutputReceivers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnOutputReceivers.java index 1a73d8e52697..d1d5fb3c6ce5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnOutputReceivers.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnOutputReceivers.java @@ -152,7 +152,7 @@ public static OutputReceiver windowedReceiver( } /** Returns a {@link MultiOutputReceiver} that delegates to a {@link DoFn.WindowedContext}. */ - public static MultiOutputReceiver windowedMultiReceiver( + public static MultiOutputReceiver windowedMultiReceiver( DoFn.WindowedContext context, @Nullable Map, Coder> outputCoders) { return new WindowedContextMultiOutputReceiver(context, outputCoders); } @@ -162,7 +162,7 @@ public static MultiOutputReceiver windowedMultiReceiver( * *

This exists for backwards-compatibility with the Dataflow runner, and will be removed. */ - public static MultiOutputReceiver windowedMultiReceiver(DoFn.WindowedContext context) { + public static MultiOutputReceiver windowedMultiReceiver(DoFn.WindowedContext context) { return new WindowedContextMultiOutputReceiver(context); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnSchemaInformation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnSchemaInformation.java index 7576eb71b3a4..8dc302dd1d54 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnSchemaInformation.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnSchemaInformation.java @@ -224,7 +224,7 @@ private UnboxingConversionFunction( this.rowSelector = new RowSelectorContainer(inputSchema, selectDescriptor, true); } - public static UnboxingConversionFunction of( + public static UnboxingConversionFunction of( Schema inputSchema, SerializableFunction toRowFunction, FieldAccessDescriptor selectDescriptor, diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java index fb1947ad5ba3..f4670a4d0e94 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java @@ -213,7 +213,7 @@ public void processWindowedElement(InputT element, Instant timestamp, final Boun try { final DoFn.ProcessContext processContext = createProcessContext( - ValueInSingleWindow.of(element, timestamp, window, PaneInfo.NO_FIRING)); + ValueInSingleWindow.of(element, timestamp, window, PaneInfo.NO_FIRING, null, null)); fnInvoker.invokeProcessElement( new DoFnInvoker.BaseArgumentProvider() { @@ -478,7 +478,38 @@ public void output(OutputT output, Instant timestamp, BoundedWindow window) { @Override public void output(TupleTag tag, T output, Instant timestamp, BoundedWindow window) { getMutableOutput(tag) - .add(ValueInSingleWindow.of(output, timestamp, window, PaneInfo.NO_FIRING)); + .add( + ValueInSingleWindow.of( + output, timestamp, window, PaneInfo.NO_FIRING, null, null)); + } + + @Override + public void output( + OutputT output, + Instant timestamp, + BoundedWindow window, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { + output(mainOutputTag, output, timestamp, window, currentRecordId, currentRecordOffset); + } + + @Override + public void output( + TupleTag tag, + T output, + Instant timestamp, + BoundedWindow window, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { + getMutableOutput(tag) + .add( + ValueInSingleWindow.of( + output, + timestamp, + window, + PaneInfo.NO_FIRING, + currentRecordId, + currentRecordOffset)); } }; } @@ -567,6 +598,16 @@ public PaneInfo pane() { return element.getPaneInfo(); } + @Override + public String currentRecordId() { + return element.getCurrentRecordId(); + } + + @Override + public Long currentRecordOffset() { + return element.getCurrentRecordOffset(); + } + @Override public PipelineOptions getPipelineOptions() { return options; @@ -591,6 +632,24 @@ public void outputWindowedValue( outputWindowedValue(mainOutputTag, output, timestamp, windows, paneInfo); } + @Override + public void outputWindowedValue( + OutputT output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { + outputWindowedValue( + mainOutputTag, + output, + timestamp, + windows, + paneInfo, + currentRecordId, + currentRecordOffset); + } + @Override public void output(TupleTag tag, T output) { outputWithTimestamp(tag, output, element.getTimestamp()); @@ -601,7 +660,7 @@ public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp getMutableOutput(tag) .add( ValueInSingleWindow.of( - output, timestamp, element.getWindow(), element.getPaneInfo())); + output, timestamp, element.getWindow(), element.getPaneInfo(), null, null)); } @Override @@ -612,7 +671,25 @@ public void outputWindowedValue( Collection windows, PaneInfo paneInfo) { for (BoundedWindow w : windows) { - getMutableOutput(tag).add(ValueInSingleWindow.of(output, timestamp, w, paneInfo)); + getMutableOutput(tag) + .add(ValueInSingleWindow.of(output, timestamp, w, paneInfo, null, null)); + } + } + + @Override + public void outputWindowedValue( + TupleTag tag, + T output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { + for (BoundedWindow w : windows) { + getMutableOutput(tag) + .add( + ValueInSingleWindow.of( + output, timestamp, w, paneInfo, currentRecordId, currentRecordOffset)); } } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/MapElements.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/MapElements.java index 6b123d3bd106..6434498d4bcd 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/MapElements.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/MapElements.java @@ -158,6 +158,7 @@ public void processElement( } /** A DoFn implementation that handles a trivial map call. */ + @SuppressWarnings("unused") // for outer private abstract class MapDoFn extends DoFn { /** Holds {@link MapDoFn#outer instance} of enclosing class, used by runner implementations. */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Reify.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Reify.java index 92f1b73900b2..797af9538c53 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Reify.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Reify.java @@ -272,7 +272,7 @@ public static PTransform, PCollection>> viewAsVal * Returns a {@link PCollection} consisting of a single element, containing the value of the given * view in the global window. */ - public static PTransform> viewInGlobalWindow( + public static PTransform> viewInGlobalWindow( PCollectionView view, Coder coder) { return new ReifyViewInGlobalWindow<>(view, coder); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Wait.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Wait.java index 454fc4dbcd21..3177de818fec 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Wait.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Wait.java @@ -135,7 +135,7 @@ public void startBundle() { } @ProcessElement - public void process(ProcessContext c, BoundedWindow w) { + public void process(ProcessContext unused, BoundedWindow w) { windows.add(w); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java index 2e26d13da547..f1a002d6277d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java @@ -369,7 +369,7 @@ public CoGbkResult and(TupleTag tag, List data) { } /** Returns an empty {@link CoGbkResult}. */ - public static CoGbkResult empty() { + public static CoGbkResult empty() { return new CoGbkResult( new CoGbkResultSchema(TupleTagList.empty()), new ArrayList>()); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java index 3e41d9d287b9..310736c014cc 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java @@ -410,7 +410,7 @@ public boolean hasParameter(Class type) { * Returns the specified {@link Parameter} if it is known in this context. Throws {@link * IllegalStateException} if there is more than one instance of the parameter. */ - public @Nullable Optional findParameter(Class type) { + public Optional findParameter(Class type) { List parameters = findParameters(type); switch (parameters.size()) { case 0: diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/GrowableOffsetRangeTracker.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/GrowableOffsetRangeTracker.java index 33cd7aeb2d42..97b0d9b8e787 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/GrowableOffsetRangeTracker.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/GrowableOffsetRangeTracker.java @@ -23,6 +23,7 @@ import java.math.MathContext; import org.apache.beam.sdk.io.range.OffsetRange; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Suppliers; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.primitives.UnsignedLong; /** * An {@link OffsetRangeTracker} for tracking a growable offset range. {@code Long.MAX_VALUE} is @@ -68,6 +69,7 @@ public GrowableOffsetRangeTracker(long start, RangeEndEstimator rangeEndEstimato this.rangeEndEstimator = checkNotNull(rangeEndEstimator); } + // TODO(sjvanrossum): Use UnsignedLong instead of BigDecimal for splitting ranges @Override public SplitResult trySplit(double fractionOfRemainder) { // If current tracking range is no longer growable, split it as a normal range. @@ -115,30 +117,12 @@ public Progress getProgress() { return super.getProgress(); } - // Convert to BigDecimal in computation to prevent overflow, which may result in lost of - // precision. - BigDecimal estimateRangeEnd = BigDecimal.valueOf(rangeEndEstimator.estimate()); - - if (lastAttemptedOffset == null) { - return Progress.from( - 0, - estimateRangeEnd - .subtract(BigDecimal.valueOf(range.getFrom()), MathContext.DECIMAL128) - .max(BigDecimal.ZERO) - .doubleValue()); - } + final long completedEnd = lastAttemptedOffset == null ? range.getFrom() : lastAttemptedOffset; + final long remainingEnd = Math.max(completedEnd, rangeEndEstimator.estimate()); - BigDecimal workRemaining = - estimateRangeEnd - .subtract(BigDecimal.valueOf(lastAttemptedOffset), MathContext.DECIMAL128) - .max(BigDecimal.ZERO); - BigDecimal totalWork = - estimateRangeEnd - .max(BigDecimal.valueOf(lastAttemptedOffset)) - .subtract(BigDecimal.valueOf(range.getFrom()), MathContext.DECIMAL128); return Progress.from( - totalWork.subtract(workRemaining, MathContext.DECIMAL128).doubleValue(), - workRemaining.doubleValue()); + UnsignedLong.fromLongBits(completedEnd - range.getFrom()).doubleValue(), + UnsignedLong.fromLongBits(remainingEnd - completedEnd).doubleValue()); } @Override diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineFnUtil.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineFnUtil.java index 20d6325b79b9..f6105445c16b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineFnUtil.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineFnUtil.java @@ -42,7 +42,7 @@ public class CombineFnUtil { * *

The returned {@link CombineFn} cannot be serialized. */ - public static CombineFn bindContext( + public static CombineFn bindContext( CombineFnWithContext combineFn, StateContext stateContext) { Context context = CombineContextFactory.createFromStateContext(stateContext); return new NonSerializableBoundedCombineFn<>(combineFn, context); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJsonUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJsonUtils.java index 408143fb1ebe..c83048ca8def 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJsonUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJsonUtils.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.util; +import com.fasterxml.jackson.core.JsonFactory; import com.fasterxml.jackson.core.JsonParseException; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.JsonMappingException; @@ -34,9 +35,14 @@ @Internal public class RowJsonUtils { + // The maximum string length for the JSON parser, set to 100 MB. + public static final int MAX_STRING_LENGTH = 100 * 1024 * 1024; + // private static int defaultBufferLimit; + private static final boolean STREAM_READ_CONSTRAINTS_AVAILABLE = streamReadConstraintsAvailable(); + /** * Increase the default jackson-databind stream read constraint. * @@ -63,14 +69,52 @@ public static void increaseDefaultStreamReadConstraints(int newLimit) { } static { - increaseDefaultStreamReadConstraints(100 * 1024 * 1024); + increaseDefaultStreamReadConstraints(MAX_STRING_LENGTH); + } + + private static boolean streamReadConstraintsAvailable() { + try { + Class.forName("com.fasterxml.jackson.core.StreamReadConstraints"); + return true; + } catch (ClassNotFoundException e) { + return false; + } + } + + private static class StreamReadConstraintsHelper { + static void setStreamReadConstraints(JsonFactory jsonFactory, int sizeLimit) { + com.fasterxml.jackson.core.StreamReadConstraints streamReadConstraints = + com.fasterxml.jackson.core.StreamReadConstraints.builder() + .maxStringLength(sizeLimit) + .build(); + jsonFactory.setStreamReadConstraints(streamReadConstraints); + } + } + + /** + * Creates a thread-safe JsonFactory with custom stream read constraints. + * + *

This method encapsulates the logic to increase the default jackson-databind stream read + * constraint to 100MB. This functionality was introduced in Jackson 2.15 causing string > 20MB + * (5MB in <2.15.0) parsing failure. This has caused regressions in its dependencies including + * Beam. Here we create a streamReadConstraints minimum size limit set to 100MB and exposing the + * factory to higher limits. If needed, call this method during pipeline run time, e.g. in + * DoFn.setup. This avoids a data race caused by modifying the global default settings. + */ + public static JsonFactory createJsonFactory(int sizeLimit) { + sizeLimit = Math.max(sizeLimit, MAX_STRING_LENGTH); + JsonFactory jsonFactory = new JsonFactory(); + if (STREAM_READ_CONSTRAINTS_AVAILABLE) { + StreamReadConstraintsHelper.setStreamReadConstraints(jsonFactory, sizeLimit); + } + return jsonFactory; } public static ObjectMapper newObjectMapperWith(RowJson.RowJsonDeserializer deserializer) { SimpleModule module = new SimpleModule("rowDeserializationModule"); module.addDeserializer(Row.class, deserializer); - ObjectMapper objectMapper = new ObjectMapper(); + ObjectMapper objectMapper = new ObjectMapper(createJsonFactory(MAX_STRING_LENGTH)); objectMapper.registerModule(module); return objectMapper; @@ -80,7 +124,7 @@ public static ObjectMapper newObjectMapperWith(RowJson.RowJsonSerializer seriali SimpleModule module = new SimpleModule("rowSerializationModule"); module.addSerializer(Row.class, serializer); - ObjectMapper objectMapper = new ObjectMapper(); + ObjectMapper objectMapper = new ObjectMapper(createJsonFactory(MAX_STRING_LENGTH)); objectMapper.registerModule(module); return objectMapper; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/GroupIntoBatchesTranslation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/GroupIntoBatchesTranslation.java index 7129854d44cc..e079cc3a91a1 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/GroupIntoBatchesTranslation.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/GroupIntoBatchesTranslation.java @@ -78,7 +78,7 @@ public FunctionSpec translate( } } - private static GroupIntoBatchesPayload getPayloadFromParameters( + private static GroupIntoBatchesPayload getPayloadFromParameters( GroupIntoBatches.BatchingParams params) { return RunnerApi.GroupIntoBatchesPayload.newBuilder() .setBatchSize(params.getBatchSize()) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ParDoTranslation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ParDoTranslation.java index 7c8cce8da3b4..d3ff5d1cc712 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ParDoTranslation.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ParDoTranslation.java @@ -800,7 +800,7 @@ public static FunctionSpec translateViewFn(ViewFn viewFn, SdkComponents co .build(); } - private static ParDoPayload getParDoPayload(AppliedPTransform transform) + private static ParDoPayload getParDoPayload(AppliedPTransform transform) throws IOException { SdkComponents components = SdkComponents.create(transform.getPipeline().getOptions()); RunnerApi.PTransform parDoPTransform = diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/SplittableParDo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/SplittableParDo.java index 3873d154a884..8dd19528db4e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/SplittableParDo.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/SplittableParDo.java @@ -242,7 +242,7 @@ public Map, PValue> getAdditionalInputs() { */ private static class ExplodeWindowsFn extends DoFn { @ProcessElement - public void process(ProcessContext c, BoundedWindow window) { + public void process(ProcessContext c, BoundedWindow unused) { c.output(c.element()); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/SplittableParDoNaiveBounded.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/SplittableParDoNaiveBounded.java index edae34fbecf9..d462d422446c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/SplittableParDoNaiveBounded.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/SplittableParDoNaiveBounded.java @@ -397,6 +397,29 @@ public void output( "Output from FinishBundle for SDF is not supported in naive implementation"); } + @Override + public void output( + TupleTag tag, + T output, + Instant timestamp, + BoundedWindow window, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { + throw new UnsupportedOperationException( + "Output from FinishBundle for SDF is not supported in naive implementation"); + } + + @Override + public void output( + @Nullable OutputT output, + Instant timestamp, + BoundedWindow window, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { + throw new UnsupportedOperationException( + "Output from FinishBundle for SDF is not supported in naive implementation"); + } + @Override public void output( TupleTag tag, T output, Instant timestamp, BoundedWindow window) { @@ -617,6 +640,18 @@ public void outputWindowedValue( outerContext.outputWindowedValue(output, timestamp, windows, paneInfo); } + @Override + public void outputWindowedValue( + OutputT output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { + outerContext.outputWindowedValue( + output, timestamp, windows, paneInfo, currentRecordId, currentRecordOffset); + } + @Override public void output(TupleTag tag, T output) { outerContext.output(tag, output); @@ -637,6 +672,19 @@ public void outputWindowedValue( outerContext.outputWindowedValue(tag, output, timestamp, windows, paneInfo); } + @Override + public void outputWindowedValue( + TupleTag tag, + T output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { + outerContext.outputWindowedValue( + tag, output, timestamp, windows, paneInfo, currentRecordId, currentRecordOffset); + } + @Override public InputT element() { return element; @@ -657,6 +705,16 @@ public PaneInfo pane() { return outerContext.pane(); } + @Override + public String currentRecordId() { + return outerContext.currentRecordId(); + } + + @Override + public Long currentRecordOffset() { + return outerContext.currentRecordOffset(); + } + @Override public Object watermarkEstimatorState() { throw new UnsupportedOperationException( diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/TransformUpgrader.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/TransformUpgrader.java index 28359b443afd..4268c6c70671 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/TransformUpgrader.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/TransformUpgrader.java @@ -50,8 +50,6 @@ import org.apache.beam.sdk.transformservice.launcher.TransformServiceLauncher; import org.apache.beam.sdk.util.ReleaseInfo; import org.apache.beam.sdk.util.construction.PTransformTranslation.TransformPayloadTranslator; -import org.apache.beam.sdk.values.PInput; -import org.apache.beam.sdk.values.POutput; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.InvalidProtocolBufferException; import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.ManagedChannelBuilder; @@ -188,16 +186,12 @@ public RunnerApi.Pipeline upgradeTransformsViaTransformService( return pipeline; } - private < - InputT extends PInput, - OutputT extends POutput, - TransformT extends org.apache.beam.sdk.transforms.PTransform> - RunnerApi.Pipeline updateTransformViaTransformService( - RunnerApi.Pipeline runnerAPIpipeline, - String transformId, - Endpoints.ApiServiceDescriptor transformServiceEndpoint, - PipelineOptions options) - throws IOException { + private RunnerApi.Pipeline updateTransformViaTransformService( + RunnerApi.Pipeline runnerAPIpipeline, + String transformId, + Endpoints.ApiServiceDescriptor transformServiceEndpoint, + PipelineOptions options) + throws IOException { RunnerApi.PTransform transformToUpgrade = runnerAPIpipeline.getComponents().getTransformsMap().get(transformId); if (transformToUpgrade == null) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/UnconsumedReads.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/UnconsumedReads.java index fafd385708b1..e0c049b98265 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/UnconsumedReads.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/UnconsumedReads.java @@ -69,6 +69,6 @@ private static void consume(PCollection unconsumedPCollection, int uniq) private static class NoOpDoFn extends DoFn { @ProcessElement - public void doNothing(ProcessContext context) {} + public void doNothing(ProcessContext unused) {} } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java index cf7ad3de7b7b..880e11382a10 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java @@ -48,7 +48,6 @@ import org.apache.beam.sdk.values.RowUtils.RowFieldMatcher; import org.apache.beam.sdk.values.RowUtils.RowPosition; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; -import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.DateTime; import org.joda.time.ReadableDateTime; @@ -838,7 +837,7 @@ public int nextFieldId() { } @Internal - public <@NonNull T> Row withFieldValueGetters( + public Row withFieldValueGetters( Factory>> fieldValueGetterFactory, T getterTarget) { checkState(getterTarget != null, "getters require withGetterTarget."); return new RowWithGetters<>(schema, fieldValueGetterFactory, getterTarget); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ShardedKey.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ShardedKey.java index b307196bc548..544a5a960828 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ShardedKey.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ShardedKey.java @@ -21,7 +21,13 @@ import java.util.Objects; import org.checkerframework.checker.nullness.qual.Nullable; -/** A key and a shard number. */ +/** + * A key and a shard number. + * + * @deprecated + *

Use {@link org.apache.beam.sdk.util.ShardedKey} instead. + */ +@Deprecated public class ShardedKey implements Serializable { private static final long serialVersionUID = 1L; private final K key; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ValueInSingleWindow.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ValueInSingleWindow.java index 74717fc606b2..7dc5fef52ecb 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ValueInSingleWindow.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ValueInSingleWindow.java @@ -60,9 +60,24 @@ public T getValue() { /** Returns the pane of this {@code ValueInSingleWindow} in its window. */ public abstract PaneInfo getPaneInfo(); + public abstract @Nullable String getCurrentRecordId(); + + public abstract @Nullable Long getCurrentRecordOffset(); + + public static ValueInSingleWindow of( + T value, + Instant timestamp, + BoundedWindow window, + PaneInfo paneInfo, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { + return new AutoValue_ValueInSingleWindow<>( + value, timestamp, window, paneInfo, currentRecordId, currentRecordOffset); + } + public static ValueInSingleWindow of( T value, Instant timestamp, BoundedWindow window, PaneInfo paneInfo) { - return new AutoValue_ValueInSingleWindow<>(value, timestamp, window, paneInfo); + return of(value, timestamp, window, paneInfo, null, null); } /** A coder for {@link ValueInSingleWindow}. */ @@ -110,7 +125,7 @@ public ValueInSingleWindow decode(InputStream inStream, Context context) thro BoundedWindow window = windowCoder.decode(inStream); PaneInfo paneInfo = PaneInfo.PaneInfoCoder.INSTANCE.decode(inStream); T value = valueCoder.decode(inStream, context); - return new AutoValue_ValueInSingleWindow<>(value, timestamp, window, paneInfo); + return new AutoValue_ValueInSingleWindow<>(value, timestamp, window, paneInfo, null, null); } @Override diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/WindowedValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/WindowedValue.java index d2c4d1f07da7..0512be524b91 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/WindowedValue.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/WindowedValue.java @@ -20,6 +20,7 @@ import java.util.Collection; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.checkerframework.checker.nullness.qual.Nullable; import org.checkerframework.dataflow.qual.Pure; import org.joda.time.Instant; @@ -45,6 +46,12 @@ public interface WindowedValue { @Pure PaneInfo getPaneInfo(); + @Nullable + String getCurrentRecordId(); + + @Nullable + Long getCurrentRecordOffset(); + /** * A representation of each of the actual values represented by this compressed {@link * WindowedValue}, one per window. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/WindowedValues.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/WindowedValues.java index 9616fd845fa7..4bbab33a8936 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/WindowedValues.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/WindowedValues.java @@ -61,16 +61,27 @@ public class WindowedValues { private WindowedValues() {} // non-instantiable utility class - /** Returns a {@code WindowedValue} with the given value, timestamp, and windows. */ public static WindowedValue of( T value, Instant timestamp, Collection windows, PaneInfo paneInfo) { + return of(value, timestamp, windows, paneInfo, null, null); + } + + /** Returns a {@code WindowedValue} with the given value, timestamp, and windows. */ + public static WindowedValue of( + T value, + Instant timestamp, + Collection windows, + PaneInfo paneInfo, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { checkArgument(paneInfo != null, "WindowedValue requires PaneInfo, but it was null"); checkArgument(windows.size() > 0, "WindowedValue requires windows, but there were none"); if (windows.size() == 1) { return of(value, timestamp, windows.iterator().next(), paneInfo); } else { - return new TimestampedValueInMultipleWindows<>(value, timestamp, windows, paneInfo); + return new TimestampedValueInMultipleWindows<>( + value, timestamp, windows, paneInfo, currentRecordId, currentRecordOffset); } } @@ -81,7 +92,8 @@ static WindowedValue createWithoutValidation( if (windows.size() == 1) { return of(value, timestamp, windows.iterator().next(), paneInfo); } else { - return new TimestampedValueInMultipleWindows<>(value, timestamp, windows, paneInfo); + return new TimestampedValueInMultipleWindows<>( + value, timestamp, windows, paneInfo, null, null); } } @@ -94,9 +106,9 @@ public static WindowedValue of( if (isGlobal && BoundedWindow.TIMESTAMP_MIN_VALUE.equals(timestamp)) { return valueInGlobalWindow(value, paneInfo); } else if (isGlobal) { - return new TimestampedValueInGlobalWindow<>(value, timestamp, paneInfo); + return new TimestampedValueInGlobalWindow<>(value, timestamp, paneInfo, null, null); } else { - return new TimestampedValueInSingleWindow<>(value, timestamp, window, paneInfo); + return new TimestampedValueInSingleWindow<>(value, timestamp, window, paneInfo, null, null); } } @@ -105,7 +117,7 @@ public static WindowedValue of( * default timestamp and pane. */ public static WindowedValue valueInGlobalWindow(T value) { - return new ValueInGlobalWindow<>(value, PaneInfo.NO_FIRING); + return new ValueInGlobalWindow<>(value, PaneInfo.NO_FIRING, null, null); } /** @@ -113,7 +125,7 @@ public static WindowedValue valueInGlobalWindow(T value) { * default timestamp and the specified pane. */ public static WindowedValue valueInGlobalWindow(T value, PaneInfo paneInfo) { - return new ValueInGlobalWindow<>(value, paneInfo); + return new ValueInGlobalWindow<>(value, paneInfo, null, null); } /** @@ -124,7 +136,7 @@ public static WindowedValue timestampedValueInGlobalWindow(T value, Insta if (BoundedWindow.TIMESTAMP_MIN_VALUE.equals(timestamp)) { return valueInGlobalWindow(value); } else { - return new TimestampedValueInGlobalWindow<>(value, timestamp, PaneInfo.NO_FIRING); + return new TimestampedValueInGlobalWindow<>(value, timestamp, PaneInfo.NO_FIRING, null, null); } } @@ -137,7 +149,7 @@ public static WindowedValue timestampedValueInGlobalWindow( if (paneInfo.equals(PaneInfo.NO_FIRING)) { return timestampedValueInGlobalWindow(value, timestamp); } else { - return new TimestampedValueInGlobalWindow<>(value, timestamp, paneInfo); + return new TimestampedValueInGlobalWindow<>(value, timestamp, paneInfo, null, null); } } @@ -151,7 +163,9 @@ public static WindowedValue withValue( newValue, windowedValue.getTimestamp(), windowedValue.getWindows(), - windowedValue.getPaneInfo()); + windowedValue.getPaneInfo(), + windowedValue.getCurrentRecordId(), + windowedValue.getCurrentRecordOffset()); } public static boolean equals( @@ -200,10 +214,28 @@ private abstract static class SimpleWindowedValue implements WindowedValue private final T value; private final PaneInfo paneInfo; + private final @Nullable String currentRecordId; + private final @Nullable Long currentRecordOffset; + + @Override + public @Nullable String getCurrentRecordId() { + return currentRecordId; + } - protected SimpleWindowedValue(T value, PaneInfo paneInfo) { + @Override + public @Nullable Long getCurrentRecordOffset() { + return currentRecordOffset; + } + + protected SimpleWindowedValue( + T value, + PaneInfo paneInfo, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { this.value = value; this.paneInfo = checkNotNull(paneInfo); + this.currentRecordId = currentRecordId; + this.currentRecordOffset = currentRecordOffset; } @Override @@ -232,8 +264,13 @@ public Iterable> explodeWindows() { /** The abstract superclass of WindowedValue representations where timestamp == MIN. */ private abstract static class MinTimestampWindowedValue extends SimpleWindowedValue { - public MinTimestampWindowedValue(T value, PaneInfo paneInfo) { - super(value, paneInfo); + + public MinTimestampWindowedValue( + T value, + PaneInfo pane, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { + super(value, pane, currentRecordId, currentRecordOffset); } @Override @@ -246,8 +283,12 @@ public Instant getTimestamp() { private static class ValueInGlobalWindow extends MinTimestampWindowedValue implements SingleWindowedValue { - public ValueInGlobalWindow(T value, PaneInfo paneInfo) { - super(value, paneInfo); + public ValueInGlobalWindow( + T value, + PaneInfo paneInfo, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { + super(value, paneInfo, currentRecordId, currentRecordOffset); } @Override @@ -262,7 +303,8 @@ public BoundedWindow getWindow() { @Override public WindowedValue withValue(NewT newValue) { - return new ValueInGlobalWindow<>(newValue, getPaneInfo()); + return new ValueInGlobalWindow<>( + newValue, getPaneInfo(), getCurrentRecordId(), getCurrentRecordOffset()); } @Override @@ -294,8 +336,13 @@ public String toString() { private abstract static class TimestampedWindowedValue extends SimpleWindowedValue { private final Instant timestamp; - public TimestampedWindowedValue(T value, Instant timestamp, PaneInfo paneInfo) { - super(value, paneInfo); + public TimestampedWindowedValue( + T value, + Instant timestamp, + PaneInfo paneInfo, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { + super(value, paneInfo, currentRecordId, currentRecordOffset); this.timestamp = checkNotNull(timestamp); } @@ -312,8 +359,13 @@ public Instant getTimestamp() { private static class TimestampedValueInGlobalWindow extends TimestampedWindowedValue implements SingleWindowedValue { - public TimestampedValueInGlobalWindow(T value, Instant timestamp, PaneInfo paneInfo) { - super(value, timestamp, paneInfo); + public TimestampedValueInGlobalWindow( + T value, + Instant timestamp, + PaneInfo paneInfo, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { + super(value, timestamp, paneInfo, currentRecordId, currentRecordOffset); } @Override @@ -328,7 +380,8 @@ public BoundedWindow getWindow() { @Override public WindowedValue withValue(NewT newValue) { - return new TimestampedValueInGlobalWindow<>(newValue, getTimestamp(), getPaneInfo()); + return new TimestampedValueInGlobalWindow<>( + newValue, getTimestamp(), getPaneInfo(), getCurrentRecordId(), getCurrentRecordOffset()); } @Override @@ -372,14 +425,25 @@ private static class TimestampedValueInSingleWindow extends TimestampedWindow private final BoundedWindow window; public TimestampedValueInSingleWindow( - T value, Instant timestamp, BoundedWindow window, PaneInfo paneInfo) { - super(value, timestamp, paneInfo); + T value, + Instant timestamp, + BoundedWindow window, + PaneInfo paneInfo, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { + super(value, timestamp, paneInfo, currentRecordId, currentRecordOffset); this.window = checkNotNull(window); } @Override public WindowedValue withValue(NewT newValue) { - return new TimestampedValueInSingleWindow<>(newValue, getTimestamp(), window, getPaneInfo()); + return new TimestampedValueInSingleWindow<>( + newValue, + getTimestamp(), + window, + getPaneInfo(), + getCurrentRecordId(), + getCurrentRecordOffset()); } @Override @@ -433,8 +497,10 @@ public TimestampedValueInMultipleWindows( T value, Instant timestamp, Collection windows, - PaneInfo paneInfo) { - super(value, timestamp, paneInfo); + PaneInfo paneInfo, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { + super(value, timestamp, paneInfo, currentRecordId, currentRecordOffset); this.windows = checkNotNull(windows); } @@ -446,7 +512,12 @@ public Collection getWindows() { @Override public WindowedValue withValue(NewT newValue) { return new TimestampedValueInMultipleWindows<>( - newValue, getTimestamp(), getWindows(), getPaneInfo()); + newValue, + getTimestamp(), + getWindows(), + getPaneInfo(), + getCurrentRecordId(), + getCurrentRecordOffset()); } @Override diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ZstdCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ZstdCoderTest.java index 7dc8bdac8b44..1c07555666a3 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ZstdCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ZstdCoderTest.java @@ -107,6 +107,7 @@ public void testStructuralValueConsistentWithEquals() throws Exception { } @Test + @SuppressWarnings("JUnitIncompatibleType") // intended public void testCoderEquals() throws Exception { // True if coder, dict and level are equal. assertEquals( diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingSourceTest.java index 9db9c8979b5b..70a09083619d 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingSourceTest.java @@ -212,7 +212,7 @@ public void testUnboundedSourceWithRate() { Instant started = Instant.now(); p.run(); Instant finished = Instant.now(); - Duration expectedDuration = period.multipliedBy((int) numElements); + Duration expectedDuration = period.multipliedBy(numElements); assertThat(started.plus(expectedDuration).isBefore(finished), is(true)); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileSystemsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileSystemsTest.java index d3fcfb291fca..34567309c7d0 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileSystemsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileSystemsTest.java @@ -311,6 +311,7 @@ public void testValidMatchNewResourceForLocalFileSystem() { assertEquals("file", FileSystems.matchNewResource("c:\\tmp\\f1", false).getScheme()); } + @SuppressWarnings("JUnitIncompatibleType") @Test(expected = IllegalArgumentException.class) public void testInvalidSchemaMatchNewResource() { assertEquals("file", FileSystems.matchNewResource("invalidschema://tmp/f1", false)); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOWriteTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOWriteTest.java index eba0f793265d..bb60c7aef1d4 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOWriteTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOWriteTest.java @@ -731,11 +731,20 @@ public void testWriteUnboundedWithCustomBatchParameters() throws Exception { input.apply(write); p.run(); + // On some environments/runners, the exact shard filenames may not be materialized + // deterministically by the time we assert. Verify shard count via a glob, then + // validate contents using pattern matching. + String pattern = baseFilename.toString() + "*"; + List matches = FileSystems.match(Collections.singletonList(pattern)); + List found = new ArrayList<>(Iterables.getOnlyElement(matches).metadata()); + assertEquals(3, found.size()); + + // Now assert file contents irrespective of exact shard indices. assertOutputFiles( LINES2_ARRAY, null, null, - 3, + 0, // match all files by prefix baseFilename, DefaultFilenamePolicy.DEFAULT_UNWINDOWED_SHARD_TEMPLATE, false); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java index 291bb5297880..5a112d5084dd 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java @@ -1673,7 +1673,7 @@ public void testUsingArgumentWithMisspelledPropertyGivesMultipleSuggestions() { public void testUsingArgumentWithUnknownPropertyIsIgnoredWithoutStrictParsing() { String[] args = new String[] {"--unknownProperty=value"}; PipelineOptionsFactory.fromArgs(args).withoutStrictParsing().create(); - expectedLogs.verifyWarn("missing a property named 'unknownProperty'"); + expectedLogs.verifyWarn("Strict parsing is disabled, ignoring option 'unknownProperty'"); } @Test diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/AutoValueSchemaTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/AutoValueSchemaTest.java index d0ee623dea7c..d7a5c3862243 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/AutoValueSchemaTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/AutoValueSchemaTest.java @@ -84,7 +84,7 @@ private Row createSimpleRow(String name) { BYTE_ARRAY, BYTE_ARRAY, BigDecimal.ONE, - new StringBuilder(name).append("builder").toString()) + name + "builder") .build(); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaBeanSchemaTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaBeanSchemaTest.java index 5313feb5c6c0..736cc250a827 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaBeanSchemaTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaBeanSchemaTest.java @@ -127,7 +127,7 @@ private Row createSimpleRow(String name) { BYTE_ARRAY, BYTE_ARRAY, BigDecimal.ONE, - new StringBuilder(name).append("builder").toString()) + name + "builder") .build(); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaFieldSchemaTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaFieldSchemaTest.java index 11bef79b26f7..66794d5a512e 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaFieldSchemaTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaFieldSchemaTest.java @@ -158,7 +158,7 @@ private Row createSimpleRow(String name) { BYTE_ARRAY, BYTE_BUFFER.array(), BigDecimal.ONE, - new StringBuilder(name).append("builder").toString()) + name + "builder") .build(); } @@ -176,7 +176,7 @@ private Row createAnnotatedRow(String name) { BYTE_ARRAY, BYTE_BUFFER.array(), BigDecimal.ONE, - new StringBuilder(name).append("builder").toString()) + name + "builder") .build(); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/GroupTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/GroupTest.java index 357ef024bea9..9ecaafbff27f 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/GroupTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/GroupTest.java @@ -671,7 +671,7 @@ public void process(@Element Row value) { pipeline.run(); } - private static Void containsKIterableVs(List expectedKvs, Iterable actualKvs) { + private static Void containsKIterableVs(List expectedKvs, Iterable actualKvs) { List> matchers = new ArrayList<>(); for (Row expected : expectedKvs) { List fieldMatchers = Lists.newArrayList(); @@ -687,7 +687,7 @@ private static Void containsKIterableVs(List expectedKvs, Iterable return null; } - private static Void containsKvRows(List expectedKvs, Iterable actualKvs) { + private static Void containsKvRows(List expectedKvs, Iterable actualKvs) { List> matchers = new ArrayList<>(); for (Row expected : expectedKvs) { matchers.add(new KvRowMatcher(equalTo(expected.getRow(0)), equalTo(expected.getRow(1)))); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/TestJavaBeans.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/TestJavaBeans.java index b5ad6f989d9e..f8affb08ac95 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/TestJavaBeans.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/TestJavaBeans.java @@ -275,7 +275,7 @@ public boolean equals(@Nullable Object o) { && Arrays.equals(bytes, that.bytes) && Objects.equals(byteBuffer, that.byteBuffer) && Objects.equals(bigDecimal, that.bigDecimal) - && Objects.equals(stringBuilder, that.stringBuilder); + && Objects.equals(stringBuilder.toString(), that.stringBuilder.toString()); } @Override @@ -462,7 +462,7 @@ public boolean equals(@Nullable Object o) { && Arrays.equals(bytes, that.bytes) && Objects.equals(byteBuffer, that.byteBuffer) && Objects.equals(bigDecimal, that.bigDecimal) - && Objects.equals(stringBuilder, that.stringBuilder); + && Objects.equals(stringBuilder.toString(), that.stringBuilder.toString()); } @Override diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupIntoBatchesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupIntoBatchesTest.java index bc2aab2ba0ef..832eb03f05d1 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupIntoBatchesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupIntoBatchesTest.java @@ -82,9 +82,9 @@ public class GroupIntoBatchesTest implements Serializable { private static final Logger LOG = LoggerFactory.getLogger(GroupIntoBatchesTest.class); @Rule public transient TestPipeline pipeline = TestPipeline.create(); @Rule public transient Timeout globalTimeout = Timeout.seconds(1200); - private transient ArrayList> data = createTestData(EVEN_NUM_ELEMENTS); + private transient List> data = createTestData(EVEN_NUM_ELEMENTS); - private static ArrayList> createTestData(long numElements) { + private static List> createTestData(long numElements) { String[] scientists = { "Einstein", "Darwin", diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/CombineTranslationTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/CombineTranslationTest.java index 468bce71475c..79cd79dcf3aa 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/CombineTranslationTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/CombineTranslationTest.java @@ -209,7 +209,7 @@ public Coder getAccumulatorCoder(CoderRegistry registry, Coder in @Override public Void extractOutput(Void accumulator) { - return accumulator; + return null; } @Override @@ -219,7 +219,7 @@ public Void mergeAccumulators(Iterable accumulators) { @Override public Void addInput(Void accumulator, Integer input) { - return accumulator; + return null; } @Override diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/EncodableThrowableTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/EncodableThrowableTest.java index 36eb7eb585a8..b9116dc2e352 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/EncodableThrowableTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/EncodableThrowableTest.java @@ -37,14 +37,13 @@ public void testEquals() { EncodableThrowable comparable1 = EncodableThrowable.forThrowable(exception); EncodableThrowable comparable2 = EncodableThrowable.forThrowable(exception); - - assertEquals(comparable1, comparable1); assertEquals(comparable1, comparable2); } @Test + @SuppressWarnings("JUnitIncompatibleType") public void testEqualsNonComparable() { - assertNotEquals(EncodableThrowable.forThrowable(new Exception()), new Throwable()); + assertNotEquals(new Throwable(), EncodableThrowable.forThrowable(new Exception())); } @Test diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorsTest.java index e33186c2a3ff..74a61a19a57e 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorsTest.java @@ -57,6 +57,7 @@ public void testTypeDescriptorsKV() throws Exception { } @Test + @SuppressWarnings("JUnitIncompatibleType") public void testTypeDescriptorsLists() throws Exception { TypeDescriptor> descriptor = lists(strings()); assertEquals(descriptor, new TypeDescriptor>() {}); diff --git a/sdks/java/expansion-service/container/Dockerfile b/sdks/java/expansion-service/container/Dockerfile index 1b83ec68b994..2688a3176713 100644 --- a/sdks/java/expansion-service/container/Dockerfile +++ b/sdks/java/expansion-service/container/Dockerfile @@ -24,6 +24,8 @@ ARG TARGETARCH WORKDIR /opt/apache/beam # Copy dependencies generated by the Gradle build. +# TODO(https://github.com/apache/beam/issues/34098) remove when Beam moved to avro 1.12 +COPY target/avro.jar jars/ COPY target/beam-sdks-java-io-expansion-service.jar jars/ COPY target/beam-sdks-java-io-google-cloud-platform-expansion-service.jar jars/ COPY target/beam-sdks-java-extensions-schemaio-expansion-service.jar jars/ diff --git a/sdks/java/expansion-service/container/build.gradle b/sdks/java/expansion-service/container/build.gradle index cf81d462f08b..080eb68c3a2e 100644 --- a/sdks/java/expansion-service/container/build.gradle +++ b/sdks/java/expansion-service/container/build.gradle @@ -36,6 +36,8 @@ configurations { } dependencies { + // TODO(https://github.com/apache/beam/issues/34098) remove when Beam moved to avro 1.12 + dockerDependency "org.apache.avro:avro:1.12.0" dockerDependency project(path: ":sdks:java:extensions:schemaio-expansion-service", configuration: "shadow") dockerDependency project(path: ":sdks:java:io:expansion-service", configuration: "shadow") dockerDependency project(path: ":sdks:java:io:google-cloud-platform:expansion-service", configuration: "shadow") @@ -48,6 +50,8 @@ goBuild { task copyDockerfileDependencies(type: Copy) { from configurations.dockerDependency + // TODO(https://github.com/apache/beam/issues/34098) remove when Beam moved to avro 1.12 + rename 'avro-.*.jar', 'avro.jar' rename 'beam-sdks-java-extensions-schemaio-expansion-service-.*.jar', 'beam-sdks-java-extensions-schemaio-expansion-service.jar' rename 'beam-sdks-java-io-expansion-service-.*.jar', 'beam-sdks-java-io-expansion-service.jar' rename 'beam-sdks-java-io-google-cloud-platform-expansion-service-.*.jar', 'beam-sdks-java-io-google-cloud-platform-expansion-service.jar' diff --git a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java index 2bd45067918c..337868c71638 100644 --- a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java +++ b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java @@ -396,7 +396,7 @@ private static Class getConfigClass( return configurationClass; } - static Row decodeConfigObjectRow(SchemaApi.Schema schema, ByteString payload) { + static Row decodeConfigObjectRow(SchemaApi.Schema schema, ByteString payload) { Schema payloadSchema = SchemaTranslation.schemaFromProto(schema); if (payloadSchema.getFieldCount() == 0) { diff --git a/sdks/java/extensions/arrow/src/main/java/org/apache/beam/sdk/extensions/arrow/ArrowConversion.java b/sdks/java/extensions/arrow/src/main/java/org/apache/beam/sdk/extensions/arrow/ArrowConversion.java index 78ba610ad4d1..e0dcedc47faf 100644 --- a/sdks/java/extensions/arrow/src/main/java/org/apache/beam/sdk/extensions/arrow/ArrowConversion.java +++ b/sdks/java/extensions/arrow/src/main/java/org/apache/beam/sdk/extensions/arrow/ArrowConversion.java @@ -154,11 +154,23 @@ public FieldType visit(ArrowType.Utf8 type) { return FieldType.STRING; } + @Override + public FieldType visit(ArrowType.Utf8View type) { + throw new IllegalArgumentException( + "Type \'" + type.toString() + "\' not supported."); + } + @Override public FieldType visit(ArrowType.Binary type) { return FieldType.BYTES; } + @Override + public FieldType visit(ArrowType.BinaryView type) { + throw new IllegalArgumentException( + "Type \'" + type.toString() + "\' not supported."); + } + @Override public FieldType visit(ArrowType.FixedSizeBinary type) { return FieldType.logicalType(FixedBytes.of(type.getByteWidth())); @@ -213,6 +225,12 @@ public FieldType visit(ArrowType.Duration type) { "Type \'" + type.toString() + "\' not supported."); } + @Override + public FieldType visit(ArrowType.ListView type) { + throw new IllegalArgumentException( + "Type \'" + type.toString() + "\' not supported."); + } + @Override public FieldType visit(ArrowType.LargeBinary type) { throw new IllegalArgumentException( @@ -376,6 +394,11 @@ public Optional> visit(ArrowType.Duration type) { throw new IllegalArgumentException("Type \'" + type.toString() + "\' not supported."); } + @Override + public Optional> visit(ArrowType.ListView listView) { + return Optional.empty(); + } + @Override public Optional> visit(ArrowType.Int type) { return Optional.empty(); @@ -391,11 +414,21 @@ public Optional> visit(ArrowType.Utf8 type) { return Optional.of((Object text) -> ((Text) text).toString()); } + @Override + public Optional> visit(ArrowType.Utf8View utf8View) { + return Optional.empty(); + } + @Override public Optional> visit(ArrowType.Binary type) { return Optional.empty(); } + @Override + public Optional> visit(ArrowType.BinaryView binaryView) { + return Optional.empty(); + } + @Override public Optional> visit(ArrowType.FixedSizeBinary type) { return Optional.empty(); diff --git a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoByteBuddyUtils.java b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoByteBuddyUtils.java index 9fe6162ec936..6f5a5c3b6d32 100644 --- a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoByteBuddyUtils.java +++ b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoByteBuddyUtils.java @@ -493,7 +493,7 @@ public TypeConversion createSetterConversions(StackManipulati static FieldValueGetter<@NonNull ProtoT, OneOfType.Value> createOneOfGetter( FieldValueTypeInformation typeInformation, - TreeMap> getterMethodMap, + Map> getterMethodMap, Class protoClass, OneOfType oneOfType, Method getCaseMethod) { @@ -555,7 +555,7 @@ public TypeConversion createSetterConversions(StackManipulati static FieldValueSetter createOneOfSetter( String name, - TreeMap> setterMethodMap, + Map> setterMethodMap, Class protoBuilderClass) { Set indices = setterMethodMap.keySet(); boolean contiguous = isContiguous(indices); diff --git a/sdks/java/extensions/sketching/src/test/java/org/apache/beam/sdk/extensions/sketching/ApproximateDistinctTest.java b/sdks/java/extensions/sketching/src/test/java/org/apache/beam/sdk/extensions/sketching/ApproximateDistinctTest.java index 0cb3e0e5116d..312aae7afdb1 100644 --- a/sdks/java/extensions/sketching/src/test/java/org/apache/beam/sdk/extensions/sketching/ApproximateDistinctTest.java +++ b/sdks/java/extensions/sketching/src/test/java/org/apache/beam/sdk/extensions/sketching/ApproximateDistinctTest.java @@ -188,7 +188,7 @@ private static class VerifyAccuracy implements SerializableFunction input) { for (Long estimate : input) { - boolean isAccurate = Math.abs(estimate - expectedCard) / expectedCard < expectedError; + boolean isAccurate = Math.abs(0.0 + estimate - expectedCard) / expectedCard < expectedError; Assert.assertTrue( "not accurate enough : \nExpected Cardinality : " + expectedCard diff --git a/sdks/java/extensions/sketching/src/test/java/org/apache/beam/sdk/extensions/sketching/TDigestQuantilesTest.java b/sdks/java/extensions/sketching/src/test/java/org/apache/beam/sdk/extensions/sketching/TDigestQuantilesTest.java index 9ee901317038..943cd7a52f6e 100644 --- a/sdks/java/extensions/sketching/src/test/java/org/apache/beam/sdk/extensions/sketching/TDigestQuantilesTest.java +++ b/sdks/java/extensions/sketching/src/test/java/org/apache/beam/sdk/extensions/sketching/TDigestQuantilesTest.java @@ -122,7 +122,7 @@ public void testMergeAccum() { Assert.assertEquals(3000, res.size()); } - private boolean encodeDecodeEquals(MergingDigest tDigest) throws IOException { + private boolean encodeDecodeEquals(MergingDigest tDigest) throws IOException { MergingDigest decoded = CoderUtils.clone(new MergingDigestCoder(), tDigest); boolean equal = true; diff --git a/sdks/java/extensions/sql/build.gradle b/sdks/java/extensions/sql/build.gradle index af8b6cba1742..afbc87f8eeba 100644 --- a/sdks/java/extensions/sql/build.gradle +++ b/sdks/java/extensions/sql/build.gradle @@ -41,6 +41,8 @@ applyJavaNature( ], // javacc generated code produces lint warnings disableLintWarnings: ['dep-ann', 'rawtypes'], + // Disable SpotBugs due to ASM bytecode analysis issue with BeamCalcRel class + enableSpotbugs: false, ) description = "Apache Beam :: SDKs :: Java :: Extensions :: SQL" @@ -74,10 +76,6 @@ dependencies { fmppTask "org.freemarker:freemarker:2.3.31" fmppTemplates library.java.vendored_calcite_1_40_0 implementation project(path: ":sdks:java:core", configuration: "shadow") - implementation project(":sdks:java:managed") - implementation project(":sdks:java:io:iceberg") - runtimeOnly project(":sdks:java:io:iceberg:bqms") - runtimeOnly project(":sdks:java:io:iceberg:hive") implementation project(":sdks:java:extensions:avro") implementation project(":sdks:java:extensions:join-library") permitUnusedDeclared project(":sdks:java:extensions:join-library") // BEAM-11761 @@ -92,6 +90,9 @@ dependencies { implementation "org.codehaus.janino:commons-compiler:3.0.11" implementation library.java.jackson_core implementation library.java.mongo_java_driver + permitUnusedDeclared library.java.mongo_java_driver + implementation library.java.mongo_bson + implementation library.java.mongodb_driver_core implementation library.java.slf4j_api implementation library.java.joda_time implementation library.java.vendored_guava_32_1_2_jre @@ -115,8 +116,6 @@ dependencies { permitUnusedDeclared library.java.hadoop_client provided library.java.kafka_clients - testImplementation "org.apache.iceberg:iceberg-api:1.6.1" - testImplementation "org.apache.iceberg:iceberg-core:1.6.1" testImplementation library.java.vendored_calcite_1_40_0 testImplementation library.java.vendored_guava_32_1_2_jre testImplementation library.java.junit @@ -131,6 +130,7 @@ dependencies { testImplementation library.java.kafka_clients testImplementation project(":sdks:java:io:kafka") testImplementation project(path: ":sdks:java:io:mongodb", configuration: "testRuntimeMigration") + testImplementation library.java.mongo_java_driver testImplementation project(path: ":sdks:java:io:thrift", configuration: "testRuntimeMigration") testImplementation project(path: ":sdks:java:extensions:protobuf", configuration: "testRuntimeMigration") testCompileOnly project(":sdks:java:extensions:sql:udf-test-provider") diff --git a/sdks/java/extensions/sql/hcatalog/build.gradle b/sdks/java/extensions/sql/hcatalog/build.gradle index e8abf21b7c3e..0a267a6f424e 100644 --- a/sdks/java/extensions/sql/hcatalog/build.gradle +++ b/sdks/java/extensions/sql/hcatalog/build.gradle @@ -26,7 +26,7 @@ applyJavaNature( ) def hive_version = "3.1.3" -def netty_version = "4.1.51.Final" +def netty_version = "4.1.110.Final" /* * We need to rely on manually specifying these evaluationDependsOn to ensure that diff --git a/sdks/java/extensions/sql/iceberg/build.gradle b/sdks/java/extensions/sql/iceberg/build.gradle new file mode 100644 index 000000000000..d5f9e74c53bd --- /dev/null +++ b/sdks/java/extensions/sql/iceberg/build.gradle @@ -0,0 +1,81 @@ +import groovy.json.JsonOutput + +/* + * 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. + */ + +plugins { id 'org.apache.beam.module' } + +applyJavaNature( + automaticModuleName: 'org.apache.beam.sdk.extensions.sql.meta.provider.hcatalog', + // iceberg requires Java11+ + requireJavaVersion: JavaVersion.VERSION_11, +) + +dependencies { + implementation project(":sdks:java:extensions:sql") + implementation project(":sdks:java:core") + implementation project(":sdks:java:managed") + implementation project(":sdks:java:io:iceberg") + runtimeOnly project(":sdks:java:io:iceberg:bqms") + runtimeOnly project(":sdks:java:io:iceberg:hive") + // TODO(https://github.com/apache/beam/issues/21156): Determine how to build without this dependency + provided "org.immutables:value:2.8.8" + permitUnusedDeclared "org.immutables:value:2.8.8" + implementation library.java.slf4j_api + implementation library.java.vendored_guava_32_1_2_jre + implementation library.java.vendored_calcite_1_40_0 + implementation library.java.jackson_databind + + testImplementation library.java.joda_time + testImplementation library.java.junit + testImplementation library.java.google_api_services_bigquery + testImplementation "org.apache.iceberg:iceberg-api:1.9.2" + testImplementation "org.apache.iceberg:iceberg-core:1.9.2" + testImplementation project(":sdks:java:io:google-cloud-platform") + testImplementation project(":sdks:java:extensions:google-cloud-platform-core") +} + +task integrationTest(type: Test) { + def gcpProject = project.findProperty('gcpProject') ?: 'apache-beam-testing' + def gcsTempRoot = project.findProperty('gcsTempRoot') ?: 'gs://temp-storage-for-end-to-end-tests/' + + // Disable Gradle cache (it should not be used because the IT's won't run). + outputs.upToDateWhen { false } + + def pipelineOptions = [ + "--project=${gcpProject}", + "--tempLocation=${gcsTempRoot}", + "--blockOnRun=false"] + + systemProperty "beamTestPipelineOptions", JsonOutput.toJson(pipelineOptions) + + include '**/*IT.class' + + maxParallelForks 4 + classpath = project(":sdks:java:extensions:sql:iceberg") + .sourceSets + .test + .runtimeClasspath + testClassesDirs = files(project(":sdks:java:extensions:sql:iceberg").sourceSets.test.output.classesDirs) + useJUnit { } +} + +configurations.all { + // iceberg-core needs avro:1.12.0 + resolutionStrategy.force 'org.apache.avro:avro:1.12.0' +} diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergCatalog.java b/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergCatalog.java similarity index 100% rename from sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergCatalog.java rename to sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergCatalog.java diff --git a/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergCatalogRegistrar.java b/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergCatalogRegistrar.java new file mode 100644 index 000000000000..03c524f7b0fc --- /dev/null +++ b/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergCatalogRegistrar.java @@ -0,0 +1,31 @@ +/* + * 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. + */ +package org.apache.beam.sdk.extensions.sql.meta.provider.iceberg; + +import com.google.auto.service.AutoService; +import org.apache.beam.sdk.extensions.sql.meta.catalog.Catalog; +import org.apache.beam.sdk.extensions.sql.meta.catalog.CatalogRegistrar; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; + +@AutoService(CatalogRegistrar.class) +public class IcebergCatalogRegistrar implements CatalogRegistrar { + @Override + public Iterable> getCatalogs() { + return ImmutableList.>builder().add(IcebergCatalog.class).build(); + } +} diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergFilter.java b/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergFilter.java similarity index 100% rename from sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergFilter.java rename to sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergFilter.java diff --git a/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastore.java b/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastore.java new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTable.java b/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTable.java similarity index 100% rename from sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTable.java rename to sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTable.java diff --git a/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProvider.java b/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProvider.java new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/package-info.java b/sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/package-info.java similarity index 100% rename from sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/package-info.java rename to sdks/java/extensions/sql/iceberg/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/package-info.java diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/BeamSqlCliIcebergTest.java b/sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/BeamSqlCliIcebergTest.java similarity index 100% rename from sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/BeamSqlCliIcebergTest.java rename to sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/BeamSqlCliIcebergTest.java diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergFilterTest.java b/sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergFilterTest.java similarity index 100% rename from sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergFilterTest.java rename to sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergFilterTest.java diff --git a/sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastoreTest.java b/sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastoreTest.java new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java b/sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java similarity index 98% rename from sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java rename to sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java index c0e8c6c7d726..417db09a2210 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java +++ b/sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java @@ -19,7 +19,6 @@ import static java.lang.String.format; import static java.util.Arrays.asList; -import static org.apache.beam.sdk.extensions.sql.utils.DateTimeUtils.parseTimestampWithUTCTimeZone; import static org.apache.beam.sdk.schemas.Schema.FieldType.BOOLEAN; import static org.apache.beam.sdk.schemas.Schema.FieldType.DOUBLE; import static org.apache.beam.sdk.schemas.Schema.FieldType.FLOAT; @@ -65,6 +64,7 @@ import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.TableIdentifier; import org.joda.time.Duration; +import org.joda.time.format.DateTimeFormat; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Rule; @@ -236,7 +236,9 @@ public void runSqlWriteAndRead(boolean withPartitionFields) (float) 1.0, 1.0, true, - parseTimestampWithUTCTimeZone("2018-05-28 20:17:40.123"), + DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSS") + .withZoneUTC() + .parseDateTime("2018-05-28 20:17:40.123"), "varchar", "char", asList("123", "456"), diff --git a/sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProviderTest.java b/sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProviderTest.java new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/PubsubToIcebergIT.java b/sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/PubsubToIcebergIT.java similarity index 98% rename from sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/PubsubToIcebergIT.java rename to sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/PubsubToIcebergIT.java index 96aeda2111f6..900fdae743a1 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/PubsubToIcebergIT.java +++ b/sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/PubsubToIcebergIT.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.extensions.sql; +package org.apache.beam.sdk.extensions.sql.meta.provider.iceberg; import static java.lang.String.format; import static java.nio.charset.StandardCharsets.UTF_8; @@ -33,6 +33,7 @@ import java.util.UUID; import java.util.stream.Collectors; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; +import org.apache.beam.sdk.extensions.sql.SqlTransform; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils; import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage; import org.apache.beam.sdk.io.gcp.pubsub.TestPubsub; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/CovarianceFn.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/CovarianceFn.java index aeee7542fcb3..cd90511e6e74 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/CovarianceFn.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/CovarianceFn.java @@ -58,7 +58,7 @@ public class CovarianceFn private boolean isSample; // flag to determine return value should be Covariance Pop or Sample private SerializableFunction decimalConverter; - public static CovarianceFn newPopulation(Schema.TypeName typeName) { + public static CovarianceFn newPopulation(Schema.TypeName typeName) { return newPopulation(BigDecimalConverter.forSqlType(typeName)); } @@ -68,7 +68,7 @@ public static CovarianceFn newPopulation( return new CovarianceFn<>(POP, decimalConverter); } - public static CovarianceFn newSample(Schema.TypeName typeName) { + public static CovarianceFn newSample(Schema.TypeName typeName) { return newSample(BigDecimalConverter.forSqlType(typeName)); } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/VarianceFn.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/VarianceFn.java index f96e7ce750a1..b7c0459fd853 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/VarianceFn.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/VarianceFn.java @@ -78,7 +78,7 @@ public class VarianceFn extends Combine.CombineFn decimalConverter; - public static VarianceFn newPopulation(Schema.TypeName typeName) { + public static VarianceFn newPopulation(Schema.TypeName typeName) { return newPopulation(BigDecimalConverter.forSqlType(typeName)); } @@ -88,7 +88,7 @@ public static VarianceFn newPopulation( return new VarianceFn<>(POP, decimalConverter); } - public static VarianceFn newSample(Schema.TypeName typeName) { + public static VarianceFn newSample(Schema.TypeName typeName) { return newSample(BigDecimalConverter.forSqlType(typeName)); } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalogRegistrar.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalogRegistrar.java index 2d94e19c1689..afffa24e6cd7 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalogRegistrar.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalogRegistrar.java @@ -18,16 +18,12 @@ package org.apache.beam.sdk.extensions.sql.meta.catalog; import com.google.auto.service.AutoService; -import org.apache.beam.sdk.extensions.sql.meta.provider.iceberg.IcebergCatalog; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @AutoService(CatalogRegistrar.class) public class InMemoryCatalogRegistrar implements CatalogRegistrar { @Override public Iterable> getCatalogs() { - return ImmutableList.>builder() - .add(InMemoryCatalog.class) - .add(IcebergCatalog.class) - .build(); + return ImmutableList.>builder().add(InMemoryCatalog.class).build(); } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastore.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastore.java deleted file mode 100644 index b73aa25c7a2b..000000000000 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastore.java +++ /dev/null @@ -1,154 +0,0 @@ -/* - * 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. - */ -package org.apache.beam.sdk.extensions.sql.meta.provider.iceberg; - -import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; - -import java.util.HashMap; -import java.util.Map; -import org.apache.beam.sdk.extensions.sql.TableUtils; -import org.apache.beam.sdk.extensions.sql.impl.TableName; -import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable; -import org.apache.beam.sdk.extensions.sql.meta.Table; -import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; -import org.apache.beam.sdk.extensions.sql.meta.store.InMemoryMetaStore; -import org.apache.beam.sdk.io.iceberg.IcebergCatalogConfig; -import org.apache.beam.sdk.io.iceberg.IcebergCatalogConfig.IcebergTableInfo; -import org.apache.beam.sdk.io.iceberg.TableAlreadyExistsException; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class IcebergMetastore extends InMemoryMetaStore { - private static final Logger LOG = LoggerFactory.getLogger(IcebergMetastore.class); - @VisibleForTesting final IcebergCatalogConfig catalogConfig; - private final Map cachedTables = new HashMap<>(); - private final String database; - - public IcebergMetastore(String db, IcebergCatalogConfig catalogConfig) { - this.database = db; - this.catalogConfig = catalogConfig; - } - - @Override - public String getTableType() { - return "iceberg"; - } - - @Override - public void createTable(Table table) { - if (!table.getType().equals("iceberg")) { - getProvider(table.getType()).createTable(table); - } else { - String identifier = getIdentifier(table); - try { - catalogConfig.createTable(identifier, table.getSchema(), table.getPartitionFields()); - } catch (TableAlreadyExistsException e) { - LOG.info( - "Iceberg table '{}' already exists at location '{}'.", table.getName(), identifier); - } - } - cachedTables.put(table.getName(), table); - } - - @Override - public void dropTable(String tableName) { - String identifier = getIdentifier(tableName); - if (catalogConfig.dropTable(identifier)) { - LOG.info("Dropped table '{}' (path: '{}').", tableName, identifier); - } else { - LOG.info( - "Ignoring DROP TABLE call for '{}' (path: '{}') because it does not exist.", - tableName, - identifier); - } - cachedTables.remove(tableName); - } - - @Override - public Map getTables() { - for (String id : catalogConfig.listTables(database)) { - String name = TableName.create(id).getTableName(); - @Nullable Table cachedTable = cachedTables.get(name); - if (cachedTable == null) { - Table table = checkStateNotNull(loadTable(id)); - cachedTables.put(name, table); - } - } - return ImmutableMap.copyOf(cachedTables); - } - - @Override - public @Nullable Table getTable(String name) { - if (cachedTables.containsKey(name)) { - return cachedTables.get(name); - } - @Nullable Table table = loadTable(getIdentifier(name)); - if (table != null) { - cachedTables.put(name, table); - } - return table; - } - - private String getIdentifier(String name) { - return database + "." + name; - } - - private String getIdentifier(Table table) { - checkArgument( - table.getLocation() == null, "Cannot create Iceberg tables using LOCATION property."); - return getIdentifier(table.getName()); - } - - private @Nullable Table loadTable(String identifier) { - @Nullable IcebergTableInfo tableInfo = catalogConfig.loadTable(identifier); - if (tableInfo == null) { - return null; - } - return Table.builder() - .type(getTableType()) - .name(identifier) - .schema(tableInfo.getSchema()) - .properties(TableUtils.parseProperties(tableInfo.getProperties())) - .build(); - } - - @Override - public BeamSqlTable buildBeamSqlTable(Table table) { - if (table.getType().equals("iceberg")) { - return new IcebergTable(getIdentifier(table), table, catalogConfig); - } - return getProvider(table.getType()).buildBeamSqlTable(table); - } - - @Override - public boolean supportsPartitioning(Table table) { - if (table.getType().equals("iceberg")) { - return true; - } - return getProvider(table.getType()).supportsPartitioning(table); - } - - @Override - public void registerProvider(TableProvider provider) { - super.registerProvider(provider); - } -} diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbTable.java index 34f56082324a..35ea74996f31 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbTable.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbTable.java @@ -21,6 +21,8 @@ import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind.COMPARISON; import static org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.SqlKind.OR; +import com.mongodb.BasicDBObject; +import com.mongodb.MongoClientSettings; import com.mongodb.client.model.Filters; import java.io.Serializable; import java.util.ArrayList; @@ -64,6 +66,7 @@ import org.apache.beam.vendor.calcite.v1_40_0.org.apache.calcite.sql.type.SqlTypeName; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.bson.BsonDocument; import org.bson.Document; import org.bson.conversions.Bson; import org.bson.json.JsonMode; @@ -178,7 +181,21 @@ private Bson constructPredicate(List supported) { if (cnf.size() == 1) { return cnf.get(0); } - return Filters.and(cnf); + // Convert all filters to BsonDocument and merge them into a single Document + // This avoids wrapping in $and which changed behavior in MongoDB driver 5.x + Document compositeFilter = new Document(); + for (Bson filter : cnf) { + // Convert any Bson filter to BsonDocument first + BsonDocument bsonDoc = + filter.toBsonDocument(BasicDBObject.class, MongoClientSettings.getDefaultCodecRegistry()); + // Convert BsonDocument to Document for easier manipulation + Document doc = Document.parse(bsonDoc.toJson()); + // Merge all top-level conditions into the composite filter + for (String key : doc.keySet()) { + compositeFilter.append(key, doc.get(key)); + } + } + return compositeFilter; } /** diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastoreTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastoreTest.java deleted file mode 100644 index a7baf1191d15..000000000000 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergMetastoreTest.java +++ /dev/null @@ -1,97 +0,0 @@ -/* - * 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. - */ -package org.apache.beam.sdk.extensions.sql.meta.provider.iceberg; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; - -import java.io.File; -import java.io.IOException; -import java.util.UUID; -import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable; -import org.apache.beam.sdk.extensions.sql.meta.Table; -import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -/** UnitTest for {@link IcebergMetastore}. */ -public class IcebergMetastoreTest { - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - private IcebergCatalog catalog; - - @Before - public void setup() throws IOException { - File warehouseFile = TEMPORARY_FOLDER.newFolder(); - assertTrue(warehouseFile.delete()); - String warehouse = "file:" + warehouseFile + "/" + UUID.randomUUID(); - catalog = - new IcebergCatalog( - "test_catalog", ImmutableMap.of("type", "hadoop", "warehouse", warehouse)); - } - - private IcebergMetastore metastore() { - return catalog.metaStore(catalog.currentDatabase()); - } - - @Test - public void testGetTableType() { - assertEquals("iceberg", metastore().getTableType()); - } - - @Test - public void testBuildBeamSqlTable() { - Table table = Table.builder().name("my_table").schema(Schema.of()).type("iceberg").build(); - BeamSqlTable sqlTable = metastore().buildBeamSqlTable(table); - - assertNotNull(sqlTable); - assertTrue(sqlTable instanceof IcebergTable); - - IcebergTable icebergTable = (IcebergTable) sqlTable; - assertEquals(catalog.currentDatabase() + ".my_table", icebergTable.tableIdentifier); - assertEquals(catalog.catalogConfig, icebergTable.catalogConfig); - } - - @Test - public void testCreateTable() { - Table table = Table.builder().name("my_table").schema(Schema.of()).type("iceberg").build(); - metastore().createTable(table); - - assertNotNull(catalog.catalogConfig.loadTable(catalog.currentDatabase() + ".my_table")); - } - - @Test - public void testGetTables() { - Table table1 = Table.builder().name("my_table_1").schema(Schema.of()).type("iceberg").build(); - Table table2 = Table.builder().name("my_table_2").schema(Schema.of()).type("iceberg").build(); - metastore().createTable(table1); - metastore().createTable(table2); - - assertEquals(ImmutableSet.of("my_table_1", "my_table_2"), metastore().getTables().keySet()); - } - - @Test - public void testSupportsPartitioning() { - Table table = Table.builder().name("my_table_1").schema(Schema.of()).type("iceberg").build(); - assertTrue(metastore().supportsPartitioning(table)); - } -} diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbReadWriteIT.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbReadWriteIT.java index 76be08fe9a6e..804639cacfc3 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbReadWriteIT.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbReadWriteIT.java @@ -31,7 +31,8 @@ import static org.hamcrest.core.IsInstanceOf.instanceOf; import com.mongodb.BasicDBObject; -import com.mongodb.MongoClient; +import com.mongodb.client.MongoClient; +import com.mongodb.client.MongoClients; import com.mongodb.client.MongoCollection; import com.mongodb.client.MongoDatabase; import com.mongodb.client.model.Filters; @@ -128,14 +129,14 @@ public static void setUp() throws Exception { .build(); mongodExecutable = mongodStarter.prepare(mongodConfig); mongodProcess = mongodExecutable.start(); - client = new MongoClient(hostname, port); + client = MongoClients.create("mongodb://" + hostname + ":" + port); mongoSqlUrl = String.format("mongodb://%s:%d/%s/%s", hostname, port, database, collection); } @AfterClass public static void tearDown() throws Exception { - client.dropDatabase(database); + client.getDatabase(database).drop(); client.close(); mongodProcess.stop(); mongodExecutable.stop(); diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java index cc7c971e10bc..6fcaf42d568c 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java @@ -1180,18 +1180,18 @@ private HandlesSplits.SplitResult trySplitForElementAndRestriction( splitResult.getWindowSplit(), PTransformTranslation.SPLITTABLE_PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS_URN + "/GetSize"); + Coder fullInputCoder = WindowedValues.getFullCoder(inputCoder, windowCoder); + return constructSplitResult( + windowedSplitResult, + null, + fullInputCoder, + initialWatermark, + watermarkAndState, + pTransformId, + mainInputId, + pTransform.getOutputsMap().keySet(), + resumeDelay); } - Coder fullInputCoder = WindowedValues.getFullCoder(inputCoder, windowCoder); - return constructSplitResult( - windowedSplitResult, - null, - fullInputCoder, - initialWatermark, - watermarkAndState, - pTransformId, - mainInputId, - pTransform.getOutputsMap().keySet(), - resumeDelay); } private void processTimer( @@ -1667,6 +1667,48 @@ public void output(TupleTag tag, T output, Instant timestamp, BoundedWind } outputTo(consumer, WindowedValues.of(output, timestamp, window, PaneInfo.NO_FIRING)); } + + @Override + public void output( + OutputT output, + Instant timestamp, + BoundedWindow window, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { + outputTo( + mainOutputConsumer, + WindowedValues.of( + output, + timestamp, + Collections.singletonList(window), + PaneInfo.NO_FIRING, + currentRecordId, + currentRecordOffset)); + } + + @Override + public void output( + TupleTag tag, + T output, + Instant timestamp, + BoundedWindow window, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { + FnDataReceiver> consumer = + (FnDataReceiver) localNameToConsumer.get(tag.getId()); + if (consumer == null) { + throw new IllegalArgumentException(String.format("Unknown output tag %s", tag)); + } + outputTo( + consumer, + WindowedValues.of( + output, + timestamp, + Collections.singletonList(window), + PaneInfo.NO_FIRING, + currentRecordId, + currentRecordOffset)); + } } private final FinishBundleArgumentProvider.Context context = @@ -1758,6 +1800,22 @@ public void outputWindowedValue( outputTo(mainOutputConsumer, WindowedValues.of(output, timestamp, windows, paneInfo)); } + @Override + public void outputWindowedValue( + OutputT output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { + // TODO(https://github.com/apache/beam/issues/29637): Check that timestamp is valid once all + // runners can provide proper timestamps. + outputTo( + mainOutputConsumer, + WindowedValues.of( + output, timestamp, windows, paneInfo, currentRecordId, currentRecordOffset)); + } + @Override public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp) { // TODO(https://github.com/apache/beam/issues/29637): Check that timestamp is valid once all @@ -1789,6 +1847,26 @@ public void outputWindowedValue( outputTo(consumer, WindowedValues.of(output, timestamp, windows, paneInfo)); } + @Override + public void outputWindowedValue( + TupleTag tag, + T output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { + FnDataReceiver> consumer = + (FnDataReceiver) localNameToConsumer.get(tag.getId()); + if (consumer == null) { + throw new IllegalArgumentException(String.format("Unknown output tag %s", tag)); + } + outputTo( + consumer, + WindowedValues.of( + output, timestamp, windows, paneInfo, currentRecordId, currentRecordOffset)); + } + @Override public State state(String stateId, boolean alwaysFetched) { StateDeclaration stateDeclaration = doFnSignature.stateDeclarations().get(stateId); @@ -1886,6 +1964,21 @@ public void outputWindowedValue( outputTo(mainOutputConsumer, WindowedValues.of(output, timestamp, windows, paneInfo)); } + @Override + public void outputWindowedValue( + OutputT output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { + checkTimestamp(timestamp); + outputTo( + mainOutputConsumer, + WindowedValues.of( + output, timestamp, windows, paneInfo, currentRecordId, currentRecordOffset)); + } + @Override public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp) { checkTimestamp(timestamp); @@ -1915,6 +2008,27 @@ public void outputWindowedValue( } outputTo(consumer, WindowedValues.of(output, timestamp, windows, paneInfo)); } + + @Override + public void outputWindowedValue( + TupleTag tag, + T output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { + checkTimestamp(timestamp); + FnDataReceiver> consumer = + (FnDataReceiver) localNameToConsumer.get(tag.getId()); + if (consumer == null) { + throw new IllegalArgumentException(String.format("Unknown output tag %s", tag)); + } + outputTo( + consumer, + WindowedValues.of( + output, timestamp, windows, paneInfo, currentRecordId, currentRecordOffset)); + } } /** Provides base arguments for a {@link DoFnInvoker} for a non-window observing method. */ @@ -2205,6 +2319,16 @@ public Instant timestamp() { return currentElement.getTimestamp(); } + @Override + public String currentRecordId() { + return currentElement.getCurrentRecordId(); + } + + @Override + public Long currentRecordOffset() { + return currentElement.getCurrentRecordOffset(); + } + @Override public PaneInfo pane() { return currentElement.getPaneInfo(); @@ -2271,6 +2395,21 @@ public void outputWindowedValue( outputTo(mainOutputConsumer, WindowedValues.of(output, timestamp, windows, paneInfo)); } + @Override + public void outputWindowedValue( + OutputT output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { + checkOnWindowExpirationTimestamp(timestamp); + outputTo( + mainOutputConsumer, + WindowedValues.of( + output, timestamp, windows, paneInfo, currentRecordId, currentRecordOffset)); + } + @Override public void output(TupleTag tag, T output) { FnDataReceiver> consumer = @@ -2307,10 +2446,25 @@ public void outputWindowedValue( Instant timestamp, Collection windows, PaneInfo paneInfo) { + outputWindowedValue(tag, output, timestamp, windows, paneInfo, null, null); + } + + @Override + public void outputWindowedValue( + TupleTag tag, + T output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { checkOnWindowExpirationTimestamp(timestamp); FnDataReceiver> consumer = (FnDataReceiver) localNameToConsumer.get(tag.getId()); - outputTo(consumer, WindowedValues.of(output, timestamp, windows, paneInfo)); + outputTo( + consumer, + WindowedValues.of( + output, timestamp, windows, paneInfo, currentRecordId, currentRecordOffset)); } @SuppressWarnings( @@ -2574,6 +2728,21 @@ public void outputWindowedValue( outputTo(mainOutputConsumer, WindowedValues.of(output, timestamp, windows, paneInfo)); } + @Override + public void outputWindowedValue( + OutputT output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) { + checkTimerTimestamp(timestamp); + outputTo( + mainOutputConsumer, + WindowedValues.of( + output, timestamp, windows, paneInfo, currentRecordId, currentRecordOffset)); + } + @Override public void output(TupleTag tag, T output) { checkTimerTimestamp(currentTimer.getHoldTimestamp()); @@ -2612,6 +2781,16 @@ public void outputWindowedValue( Collection windows, PaneInfo paneInfo) {} + @Override + public void outputWindowedValue( + TupleTag tag, + T output, + Instant timestamp, + Collection windows, + PaneInfo paneInfo, + @Nullable String currentRecordId, + @Nullable Long currentRecordOffset) {} + @Override public TimeDomain timeDomain() { return currentTimeDomain; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java index e0b63527bec5..034695237d83 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java @@ -29,7 +29,6 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutorService; import java.util.function.Function; import javax.annotation.Nullable; import org.apache.beam.fn.harness.control.BeamFnControlClient; @@ -64,6 +63,7 @@ import org.apache.beam.sdk.options.ExperimentalOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.SdkHarnessOptions; +import org.apache.beam.sdk.util.UnboundedScheduledExecutorService; import org.apache.beam.sdk.util.construction.CoderTranslation; import org.apache.beam.sdk.util.construction.PipelineOptionsTranslation; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.TextFormat; @@ -276,8 +276,8 @@ public static void main( IdGenerator idGenerator = IdGenerators.decrementingLongs(); ShortIdMap metricsShortIds = new ShortIdMap(); - ExecutorService executorService = - options.as(ExecutorOptions.class).getScheduledExecutorService(); + UnboundedScheduledExecutorService executorService = new UnboundedScheduledExecutorService(); + options.as(ExecutorOptions.class).setScheduledExecutorService(executorService); CompletableFuture samplerTerminationFuture = new CompletableFuture<>(); ExecutionStateSampler executionStateSampler = new ExecutionStateSampler( diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/SplittableTruncateSizedRestrictionsDoFnRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/SplittableTruncateSizedRestrictionsDoFnRunner.java index f1001118d339..f7e2efdbcf35 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/SplittableTruncateSizedRestrictionsDoFnRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/SplittableTruncateSizedRestrictionsDoFnRunner.java @@ -677,7 +677,7 @@ private static WindowedSplitResult computeWindowSplit } @VisibleForTesting - static HandlesSplits.SplitResult constructSplitResult( + static HandlesSplits.SplitResult constructSplitResult( @Nullable WindowedSplitResult windowedSplitResult, HandlesSplits.@Nullable SplitResult downstreamElementSplit, Coder> fullInputCoder, diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ExecutionStateSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ExecutionStateSampler.java index 9a2d0e3d3539..fdc273b64b3f 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ExecutionStateSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ExecutionStateSampler.java @@ -400,16 +400,16 @@ private Optional takeSample(long currentTimeMillis, long millisSinceLast if (thread == null) { timeoutMessage = String.format( - "Operation ongoing in bundle %s for at least %s without outputting " - + "or completing (stack trace unable to be generated). The SDK worker will restart.", + "Processing of an element in bundle %s has exceeded the specified timeout of %s " + + "(stack trace unable to be generated). The SDK worker will be terminated.", processBundleId.get(), DURATION_FORMATTER.print( Duration.millis(userSpecifiedLullTimeMsForRestart).toPeriod())); } else if (currentExecutionState == null) { timeoutMessage = String.format( - "Operation ongoing in bundle %s for at least %s without outputting " - + "or completing:%n at %s. The SDK worker will restart.", + "Processing of an element in bundle %s has exceeded the specified timeout of %s " + + "without outputting or completing:%n at %s. The SDK worker will be terminated.", processBundleId.get(), DURATION_FORMATTER.print( Duration.millis(userSpecifiedLullTimeMsForRestart).toPeriod()), @@ -417,8 +417,9 @@ private Optional takeSample(long currentTimeMillis, long millisSinceLast } else { timeoutMessage = String.format( - "Operation ongoing in bundle %s for PTransform{id=%s, name=%s, state=%s} " - + "for at least %s without outputting or completing:%n at %s. The SDK worker will restart.", + "Processing of an element in bundle %s for PTransform{id=%s, name=%s, state=%s} " + + "has exceeded the specified timeout of %s without outputting or completing:%n at %s. " + + "The SDK worker will be terminated.", processBundleId.get(), currentExecutionState.ptransformId, currentExecutionState.ptransformUniqueName, diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java index df9cf428ff1b..fe422939e535 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java @@ -589,10 +589,10 @@ public BeamFnApi.InstructionResponse.Builder processBundle(InstructionRequest re return BeamFnApi.InstructionResponse.newBuilder().setProcessBundle(response); } catch (Exception e) { LOG.debug( - "Error processing bundle {} with bundleProcessor for {} after exception: {}", + "Error processing bundle {} with bundleProcessor for {} after exception", request.getInstructionId(), request.getProcessBundle().getProcessBundleDescriptorId(), - e.getMessage()); + e); if (bundleProcessor != null) { // Make sure we clean up from the active set of bundle processors. bundleProcessorCache.discard(bundleProcessor); diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ExecutionStateSamplerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ExecutionStateSamplerTest.java index 0753f7a00fc7..8b9678733f85 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ExecutionStateSamplerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ExecutionStateSamplerTest.java @@ -809,8 +809,12 @@ public Long answer(InvocationOnMock invocation) throws Throwable { // and unblock the state transition once a certain number of samples // have been taken. waitTillActive.await(); - waitForSamples.countDown(); - currentTime += Duration.standardMinutes(1).getMillis(); + // Freeze time after the desired number of samples to avoid races where + // the sampling loop spins and exceeds the timeout before we deactivate. + if (waitForSamples.getCount() > 0) { + waitForSamples.countDown(); + currentTime += Duration.standardMinutes(1).getMillis(); + } return currentTime; } } diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/HarnessMonitoringInfosInstructionHandlerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/HarnessMonitoringInfosInstructionHandlerTest.java index ac69ed29a565..9e69cb2ec700 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/HarnessMonitoringInfosInstructionHandlerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/HarnessMonitoringInfosInstructionHandlerTest.java @@ -30,7 +30,10 @@ import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.MetricsEnvironment; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +@RunWith(JUnit4.class) public class HarnessMonitoringInfosInstructionHandlerTest { @Test diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java index 8a35351fdb25..a7a62571e38e 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java @@ -236,6 +236,7 @@ public void finishBundle(FinishBundleContext context) { } } + @SuppressWarnings("ExtendsAutoValue") private static class TestBundleProcessor extends BundleProcessor { static int resetCnt = 0; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClientTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClientTest.java index e440ba818273..249e720d1e42 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClientTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClientTest.java @@ -220,8 +220,9 @@ public synchronized String formatMessage(LogRecord record) { } }); } - MDC.put("testMdcKey", "testMdcValue"); - configuredLogger.log(TEST_RECORD); + try (MDC.MDCCloseable ignored = MDC.putCloseable("testMdcKey", "testMdcValue")) { + configuredLogger.log(TEST_RECORD); + } client.close(); diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/MultimapUserStateTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/MultimapUserStateTest.java index 17550793a8b2..48c9ce43bdf0 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/MultimapUserStateTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/MultimapUserStateTest.java @@ -21,6 +21,7 @@ import static java.util.Collections.singletonList; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.emptyIterable; +import static org.hamcrest.collection.ArrayMatching.arrayContainingInAnyOrder; import static org.hamcrest.core.Is.is; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; @@ -167,7 +168,9 @@ public void testKeys() throws Exception { userState.put(A3, "V1"); userState.put(A1, "V3"); assertArrayEquals(new byte[][] {A1, A2}, Iterables.toArray(initKeys, byte[].class)); - assertArrayEquals(new byte[][] {A1, A2, A3}, Iterables.toArray(userState.keys(), byte[].class)); + assertThat( + Iterables.toArray(userState.keys(), byte[].class), + is(arrayContainingInAnyOrder(A1, A2, A3))); userState.clear(); assertArrayEquals(new byte[][] {A1, A2}, Iterables.toArray(initKeys, byte[].class)); @@ -822,8 +825,9 @@ public void testKeysCached() throws Exception { userState.put(A2, "V1"); userState.put(A3, "V1"); - assertArrayEquals( - new byte[][] {A1, A2, A3}, Iterables.toArray(userState.keys(), byte[].class)); + assertThat( + Iterables.toArray(userState.keys(), byte[].class), + is(arrayContainingInAnyOrder(A1, A2, A3))); userState.asyncClose(); } @@ -841,8 +845,9 @@ public void testKeysCached() throws Exception { ByteArrayCoder.of(), StringUtf8Coder.of()); - assertArrayEquals( - new byte[][] {A1, A2, A3}, Iterables.toArray(userState.keys(), byte[].class)); + assertThat( + Iterables.toArray(userState.keys(), byte[].class), + is(arrayContainingInAnyOrder(A1, A2, A3))); userState.asyncClose(); } } diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisIO.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisIO.java index 7302a5a47600..835bde170d33 100644 --- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisIO.java +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisIO.java @@ -1244,7 +1244,7 @@ public void refreshPeriodically( private void refresh( KinesisAsyncClient client, Supplier nextRefreshFn, - TreeSet bounds, + NavigableSet bounds, @Nullable String nextToken) { ListShardsRequest.Builder reqBuilder = ListShardsRequest.builder().shardFilter(f -> f.type(AT_LATEST)); diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/StaticSupplier.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/StaticSupplier.java index bd56c241429c..5eab91b24f76 100644 --- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/StaticSupplier.java +++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/StaticSupplier.java @@ -45,6 +45,7 @@ public V get() { } @Override + @SuppressWarnings("Finalize") protected void finalize() { if (cleanup) { objects.remove(id); diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapper.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapper.java index 12ba03df379b..c61240c289a5 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapper.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapper.java @@ -67,7 +67,7 @@ public ValidationException getOrThrowException() throws ValidationException { } @Override - public ArrayList getValidationFailures() { + public List getValidationFailures() { return this.failuresCollection; } } diff --git a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapperTest.java b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapperTest.java index 5031cb7e0af7..a0b5edf6a6aa 100644 --- a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapperTest.java +++ b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapperTest.java @@ -23,7 +23,7 @@ import io.cdap.cdap.etl.api.validation.CauseAttributes; import io.cdap.cdap.etl.api.validation.ValidationException; import io.cdap.cdap.etl.api.validation.ValidationFailure; -import java.util.ArrayList; +import java.util.List; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -64,7 +64,7 @@ public void getOrThrowException() { assertEquals(expectedMessage, e.getMessage()); // A case when return ValidationException with empty collector - ArrayList exceptionCollector = + List exceptionCollector = emptyFailureCollectorWrapper.getValidationFailures(); assertEquals(0, exceptionCollector.size()); } @@ -81,9 +81,8 @@ public void getValidationFailures() { failureCollectorWrapper.addFailure(error.getMessage(), null); /** act */ - ArrayList exceptionCollector = - failureCollectorWrapper.getValidationFailures(); - ArrayList emptyExceptionCollector = + List exceptionCollector = failureCollectorWrapper.getValidationFailures(); + List emptyExceptionCollector = emptyFailureCollectorWrapper.getValidationFailures(); /** assert */ diff --git a/sdks/java/io/common/src/main/java/org/apache/beam/sdk/io/common/DatabaseTestHelper.java b/sdks/java/io/common/src/main/java/org/apache/beam/sdk/io/common/DatabaseTestHelper.java index 319e932265db..bbd7c6eaa38c 100644 --- a/sdks/java/io/common/src/main/java/org/apache/beam/sdk/io/common/DatabaseTestHelper.java +++ b/sdks/java/io/common/src/main/java/org/apache/beam/sdk/io/common/DatabaseTestHelper.java @@ -174,7 +174,7 @@ public static void createTableWithStatement(DataSource dataSource, String stmt) } } - public static ArrayList> getTestDataToWrite(long rowsToAdd) { + public static List> getTestDataToWrite(long rowsToAdd) { ArrayList> data = new ArrayList<>(); for (int i = 0; i < rowsToAdd; i++) { KV kv = KV.of(i, TestRow.getNameForSeed(i)); diff --git a/sdks/java/io/common/src/main/java/org/apache/beam/sdk/io/common/IOITHelper.java b/sdks/java/io/common/src/main/java/org/apache/beam/sdk/io/common/IOITHelper.java index d14eacb8230b..57dca46af0b5 100644 --- a/sdks/java/io/common/src/main/java/org/apache/beam/sdk/io/common/IOITHelper.java +++ b/sdks/java/io/common/src/main/java/org/apache/beam/sdk/io/common/IOITHelper.java @@ -86,7 +86,7 @@ public static void executeWithRetry(int maxAttempts, long minDelay, RetryFunctio function.run(); return; } catch (Exception e) { - LOG.warn("Attempt #{} of {} failed: {}.", attempts, maxAttempts, e.getMessage()); + LOG.warn("Attempt #{} of {} failed", attempts, maxAttempts, e); if (attempts == maxAttempts) { throw e; } else { diff --git a/sdks/java/io/debezium/src/main/java/org/apache/beam/io/debezium/DebeziumReadSchemaTransformProvider.java b/sdks/java/io/debezium/src/main/java/org/apache/beam/io/debezium/DebeziumReadSchemaTransformProvider.java index a0838174759c..d5f3f98f3b5e 100644 --- a/sdks/java/io/debezium/src/main/java/org/apache/beam/io/debezium/DebeziumReadSchemaTransformProvider.java +++ b/sdks/java/io/debezium/src/main/java/org/apache/beam/io/debezium/DebeziumReadSchemaTransformProvider.java @@ -97,8 +97,7 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { + ". Unable to select a JDBC driver for it. Supported Databases are: " + String.join(", ", connectors)); } - Class connectorClass = - Objects.requireNonNull(Connectors.valueOf(configuration.getDatabase())).getConnector(); + Class connectorClass = Connectors.valueOf(configuration.getDatabase()).getConnector(); DebeziumIO.ConnectorConfiguration connectorConfiguration = DebeziumIO.ConnectorConfiguration.create() .withUsername(configuration.getUsername()) diff --git a/sdks/java/io/expansion-service/build.gradle b/sdks/java/io/expansion-service/build.gradle index c139315d925f..08c3f2b051dc 100644 --- a/sdks/java/io/expansion-service/build.gradle +++ b/sdks/java/io/expansion-service/build.gradle @@ -25,6 +25,8 @@ applyJavaNature( exportJavadoc: false, validateShadowJar: false, shadowClosure: {}, + // iceberg requires Java11+ + requireJavaVersion: JavaVersion.VERSION_11 ) // We don't want to use the latest version for the entire beam sdk since beam Java users can override it themselves. @@ -33,9 +35,8 @@ applyJavaNature( configurations.runtimeClasspath { // Pin kafka-clients version due to <3.4.0 missing auth callback classes. resolutionStrategy.force 'org.apache.kafka:kafka-clients:3.9.0' - // Pin avro to 1.11.4 due to https://github.com/apache/beam/issues/34968 - // cannot upgrade this to the latest version due to https://github.com/apache/beam/issues/34993 - resolutionStrategy.force 'org.apache.avro:avro:1.11.4' + // iceberg needs avro:1.12.0 + resolutionStrategy.force 'org.apache.avro:avro:1.12.0' // force parquet-avro:1.15.2 to fix CVE-2025-46762 resolutionStrategy.force 'org.apache.parquet:parquet-avro:1.15.2' @@ -66,18 +67,17 @@ dependencies { permitUnusedDeclared project(":sdks:java:expansion-service") // BEAM-11761 implementation project(":sdks:java:managed") permitUnusedDeclared project(":sdks:java:managed") // BEAM-11761 - implementation project(":sdks:java:io:iceberg") - permitUnusedDeclared project(":sdks:java:io:iceberg") // BEAM-11761 implementation project(":sdks:java:io:kafka") permitUnusedDeclared project(":sdks:java:io:kafka") // BEAM-11761 implementation project(":sdks:java:io:kafka:upgrade") permitUnusedDeclared project(":sdks:java:io:kafka:upgrade") // BEAM-11761 - // **** IcebergIO catalogs **** - // HiveCatalog - runtimeOnly project(path: ":sdks:java:io:iceberg:hive") - // BigQueryMetastoreCatalog (Java 11+) - runtimeOnly project(path: ":sdks:java:io:iceberg:bqms", configuration: "shadow") + if (JavaVersion.current().compareTo(JavaVersion.VERSION_11) >= 0 && project.findProperty('testJavaVersion') != '8') { + // iceberg ended support for Java 8 in 1.7.0 + runtimeOnly project(":sdks:java:io:iceberg") + runtimeOnly project(":sdks:java:io:iceberg:hive") + runtimeOnly project(path: ":sdks:java:io:iceberg:bqms", configuration: "shadow") + } runtimeOnly library.java.kafka_clients runtimeOnly library.java.slf4j_jdk14 diff --git a/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/text/TextIOIT.java b/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/text/TextIOIT.java index e50a8aba4162..d0ea19ffdf85 100644 --- a/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/text/TextIOIT.java +++ b/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/text/TextIOIT.java @@ -136,7 +136,7 @@ public void writeThenReadAll() { PCollection consolidatedHashcode = testFilenames - .apply("Match all files", FileIO.matchAll()) + .apply("Match all files", FileIO.matchAll().withOutputParallelization(false)) .apply( "Read matches", FileIO.readMatches().withDirectoryTreatment(DirectoryTreatment.PROHIBIT)) diff --git a/sdks/java/io/file-schema-transform/src/main/java/org/apache/beam/sdk/io/fileschematransform/FileWriteSchemaTransformFormatProviders.java b/sdks/java/io/file-schema-transform/src/main/java/org/apache/beam/sdk/io/fileschematransform/FileWriteSchemaTransformFormatProviders.java index 85c48ea498a3..b02c1bcb7e1a 100644 --- a/sdks/java/io/file-schema-transform/src/main/java/org/apache/beam/sdk/io/fileschematransform/FileWriteSchemaTransformFormatProviders.java +++ b/sdks/java/io/file-schema-transform/src/main/java/org/apache/beam/sdk/io/fileschematransform/FileWriteSchemaTransformFormatProviders.java @@ -80,7 +80,7 @@ static MapElements mapRowsToGenericRecords(Schema beamSchema // mapFn: the mapping function for mapping from Beam row to other data types. // outputTag: TupleTag for output. Used to direct output to correct output source, or in the // case of error, a DLQ. - static class BeamRowMapperWithDlq extends DoFn { + static class BeamRowMapperWithDlq extends DoFn { private SerializableFunction mapFn; private Counter errorCounter; private TupleTag outputTag; diff --git a/sdks/java/io/file-schema-transform/src/main/java/org/apache/beam/sdk/io/fileschematransform/XmlRowAdapter.java b/sdks/java/io/file-schema-transform/src/main/java/org/apache/beam/sdk/io/fileschematransform/XmlRowAdapter.java index 0b5d859dadf1..f57ebc5f8d5e 100644 --- a/sdks/java/io/file-schema-transform/src/main/java/org/apache/beam/sdk/io/fileschematransform/XmlRowAdapter.java +++ b/sdks/java/io/file-schema-transform/src/main/java/org/apache/beam/sdk/io/fileschematransform/XmlRowAdapter.java @@ -38,6 +38,7 @@ * XmlRowAdapter} exposes the String key and Object value pairs of the {@link Row} to the {@link * javax.xml.bind.Marshaller}. */ +// return value used for assignment @XmlRootElement(name = "row") @XmlAccessorType(XmlAccessType.PROPERTY) class XmlRowAdapter implements Serializable { diff --git a/sdks/java/io/google-ads/src/test/java/org/apache/beam/sdk/io/googleads/GoogleAdsIOTest.java b/sdks/java/io/google-ads/src/test/java/org/apache/beam/sdk/io/googleads/GoogleAdsIOTest.java index 4804918bed6c..29ac09951801 100644 --- a/sdks/java/io/google-ads/src/test/java/org/apache/beam/sdk/io/googleads/GoogleAdsIOTest.java +++ b/sdks/java/io/google-ads/src/test/java/org/apache/beam/sdk/io/googleads/GoogleAdsIOTest.java @@ -290,6 +290,7 @@ public static class ExecutionTests { @Rule public final transient TestPipeline pipeline = TestPipeline.create(); @Before + @SuppressWarnings("LockOnNonEnclosingClassLiteral") // valid use public void init() { GoogleAdsOptions options = pipeline.getOptions().as(GoogleAdsOptions.class); options.setGoogleAdsCredentialFactoryClass(NoopCredentialFactory.class); diff --git a/sdks/java/io/google-cloud-platform/build.gradle b/sdks/java/io/google-cloud-platform/build.gradle index b5b27003b944..0381193993f2 100644 --- a/sdks/java/io/google-cloud-platform/build.gradle +++ b/sdks/java/io/google-cloud-platform/build.gradle @@ -31,7 +31,17 @@ description = "Apache Beam :: SDKs :: Java :: IO :: Google Cloud Platform" ext.summary = "IO library to read and write Google Cloud Platform systems from Beam." dependencies { - implementation enforcedPlatform(library.java.google_cloud_platform_libraries_bom) + implementation(enforcedPlatform(library.java.google_cloud_platform_libraries_bom)) { + // TODO(https://github.com/apache/beam/issues/35868) remove exclude after upstream and/or tests fixed + exclude group: "com.google.cloud", module: "google-cloud-spanner" + exclude group: "com.google.api.grpc", module: "proto-google-cloud-spanner-v1" + exclude group: "com.google.api.grpc", module: "proto-google-cloud-spanner-admin-instance-v1" + exclude group: "com.google.api.grpc", module: "proto-google-cloud-spanner-admin-database-v1" + exclude group: "com.google.api.grpc", module: "grpc-google-cloud-spanner-v1" + exclude group: "com.google.api.grpc", module: "grpc-google-cloud-spanner-admin-instance-v1" + exclude group: "com.google.api.grpc", module: "grpc-google-cloud-spanner-admin-database-v1" + } + implementation(enforcedPlatform(library.java.google_cloud_spanner_bom)) implementation project(path: ":model:pipeline", configuration: "shadow") implementation project(":runners:core-java") implementation project(path: ":sdks:java:core", configuration: "shadow") diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java index 129c8314fc80..d468ffbea43c 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java @@ -25,6 +25,7 @@ import com.google.api.client.util.Sleeper; import com.google.api.services.bigquery.model.Clustering; import com.google.api.services.bigquery.model.Dataset; +import com.google.api.services.bigquery.model.ErrorProto; import com.google.api.services.bigquery.model.Job; import com.google.api.services.bigquery.model.JobReference; import com.google.api.services.bigquery.model.JobStatus; @@ -205,6 +206,7 @@ static class PendingJob implements Serializable { void runJob() throws IOException { ++currentAttempt; if (!shouldRetry()) { + logBigQueryError(lastJobAttempted); throw new RuntimeException( String.format( "Failed to create job with prefix %s, " @@ -281,6 +283,21 @@ boolean pollJob() throws IOException { boolean shouldRetry() { return currentAttempt < maxRetries + 1; } + + void logBigQueryError(@Nullable Job job) { + if (job == null || !parseStatus(job).equals(Status.FAILED)) { + return; + } + + List jobErrors = job.getStatus().getErrors(); + String finalError = job.getStatus().getErrorResult().getMessage(); + String causativeError = + jobErrors != null && !jobErrors.isEmpty() + ? String.format(" due to: %s", jobErrors.get(jobErrors.size() - 1).getMessage()) + : ""; + + LOG.error(String.format("BigQuery Error : %s %s", finalError, causativeError)); + } } static class RetryJobId implements Serializable { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java index f986e802f1ca..d5e927b4b44b 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java @@ -551,7 +551,8 @@ * using {@link Write#withPrimaryKey}. */ @SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20506) + "nullness", // TODO(https://github.com/apache/beam/issues/20506), + "SameNameButDifferent" }) public class BigQueryIO { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySourceBase.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySourceBase.java index b7b83dccaece..d2aed44d9f48 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySourceBase.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySourceBase.java @@ -221,8 +221,7 @@ private List executeExtract( // The error messages thrown in this case are generic and misleading, so leave this breadcrumb // in case it's the root cause. LOG.warn( - "Error extracting table: {} " - + "Note that external tables cannot be exported: " + "Error extracting table. Note that external tables cannot be exported: " + "https://cloud.google.com/bigquery/docs/external-tables#external_table_limitations", exn); throw exn; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java index d905c4bf93ca..a441803cc4fa 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWritesShardedRecords.java @@ -182,11 +182,11 @@ public String toString() { } }; - private static final Cache, AppendClientInfo> APPEND_CLIENTS = + private static final Cache>, AppendClientInfo> APPEND_CLIENTS = CacheBuilder.newBuilder() .expireAfterAccess(5, TimeUnit.MINUTES) .removalListener( - (RemovalNotification, AppendClientInfo> removal) -> { + (RemovalNotification>, AppendClientInfo> removal) -> { final @Nullable AppendClientInfo appendClientInfo = removal.getValue(); if (appendClientInfo != null) { appendClientInfo.close(); @@ -580,14 +580,18 @@ public void process( }; AtomicReference appendClientInfo = - new AtomicReference<>(APPEND_CLIENTS.get(element.getKey(), getAppendClientInfo)); + new AtomicReference<>( + APPEND_CLIENTS.get( + messageConverters.getAppendClientKey(element.getKey()), getAppendClientInfo)); String currentStream = getOrCreateStream.get(); if (!currentStream.equals(appendClientInfo.get().getStreamName())) { // Cached append client is inconsistent with persisted state. Throw away cached item and // force it to be // recreated. - APPEND_CLIENTS.invalidate(element.getKey()); - appendClientInfo.set(APPEND_CLIENTS.get(element.getKey(), getAppendClientInfo)); + APPEND_CLIENTS.invalidate(messageConverters.getAppendClientKey(element.getKey())); + appendClientInfo.set( + APPEND_CLIENTS.get( + messageConverters.getAppendClientKey(element.getKey()), getAppendClientInfo)); } TableSchema updatedSchemaValue = updatedSchema.read(); @@ -596,8 +600,9 @@ public void process( appendClientInfo.set( AppendClientInfo.of( updatedSchemaValue, appendClientInfo.get().getCloseAppendClient(), false)); - APPEND_CLIENTS.invalidate(element.getKey()); - APPEND_CLIENTS.put(element.getKey(), appendClientInfo.get()); + APPEND_CLIENTS.invalidate(messageConverters.getAppendClientKey(element.getKey())); + APPEND_CLIENTS.put( + messageConverters.getAppendClientKey(element.getKey()), appendClientInfo.get()); } } @@ -664,9 +669,10 @@ public void process( Consumer> clearClients = contexts -> { - APPEND_CLIENTS.invalidate(element.getKey()); + APPEND_CLIENTS.invalidate(messageConverters.getAppendClientKey(element.getKey())); appendClientInfo.set(appendClientInfo.get().withNoAppendClient()); - APPEND_CLIENTS.put(element.getKey(), appendClientInfo.get()); + APPEND_CLIENTS.put( + messageConverters.getAppendClientKey(element.getKey()), appendClientInfo.get()); for (AppendRowsContext context : contexts) { if (context.client != null) { // Unpin in a different thread, as it may execute a blocking close. @@ -960,8 +966,9 @@ public void process( appendClientInfo.set( AppendClientInfo.of( newSchema.get(), appendClientInfo.get().getCloseAppendClient(), false)); - APPEND_CLIENTS.invalidate(element.getKey()); - APPEND_CLIENTS.put(element.getKey(), appendClientInfo.get()); + APPEND_CLIENTS.invalidate(messageConverters.getAppendClientKey(element.getKey())); + APPEND_CLIENTS.put( + messageConverters.getAppendClientKey(element.getKey()), appendClientInfo.get()); LOG.debug( "Fetched updated schema for table {}:\n\t{}", tableId, updatedSchemaReturned); updatedSchema.write(newSchema.get()); @@ -993,7 +1000,7 @@ private void finalizeStream( streamName.clear(); streamOffset.clear(); // Make sure that the stream object is closed. - APPEND_CLIENTS.invalidate(key); + APPEND_CLIENTS.invalidate(messageConverters.getAppendClientKey(key)); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowJsonCoder.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowJsonCoder.java index 8cf3eeb479c0..f8e877fe98e6 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowJsonCoder.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableRowJsonCoder.java @@ -75,10 +75,8 @@ public long getEncodedElementByteSize(TableRow value) throws Exception { private static final TypeDescriptor TYPE_DESCRIPTOR; static { - RowJsonUtils.increaseDefaultStreamReadConstraints(100 * 1024 * 1024); - MAPPER = - new ObjectMapper() + new ObjectMapper(RowJsonUtils.createJsonFactory(RowJsonUtils.MAX_STRING_LENGTH)) .registerModule(new JavaTimeModule()) .registerModule(new JodaModule()) .disable(SerializationFeature.WRITE_DATES_AS_TIMESTAMPS) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TwoLevelMessageConverterCache.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TwoLevelMessageConverterCache.java index 5f90e1dd3950..0ce7c7573c9c 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TwoLevelMessageConverterCache.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TwoLevelMessageConverterCache.java @@ -20,6 +20,7 @@ import java.io.Serializable; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; import org.apache.beam.sdk.io.gcp.bigquery.StorageApiDynamicDestinations.MessageConverter; +import org.apache.beam.sdk.util.ShardedKey; import org.apache.beam.sdk.values.KV; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.Cache; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheBuilder; @@ -71,4 +72,8 @@ public MessageConverter get( KV.of(operationName, destination), () -> dynamicDestinations.getMessageConverter(destination, datasetService))); } + + public KV> getAppendClientKey(ShardedKey shardedKey) { + return KV.of(operationName, shardedKey); + } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadSchemaTransformProvider.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadSchemaTransformProvider.java index f48a23559141..2ed75d7bc7e0 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadSchemaTransformProvider.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadSchemaTransformProvider.java @@ -24,7 +24,8 @@ import com.google.bigtable.v2.Cell; import com.google.bigtable.v2.Column; import com.google.bigtable.v2.Family; -import java.nio.ByteBuffer; +import com.google.protobuf.ByteString; +import java.io.Serializable; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -37,11 +38,12 @@ import org.apache.beam.sdk.schemas.transforms.SchemaTransform; import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; -import org.apache.beam.sdk.transforms.MapElements; -import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionRowTuple; import org.apache.beam.sdk.values.Row; +import org.checkerframework.checker.nullness.qual.Nullable; /** * An implementation of {@link TypedSchemaTransformProvider} for Bigtable Read jobs configured via @@ -69,6 +71,13 @@ public class BigtableReadSchemaTransformProvider Schema.FieldType.STRING, Schema.FieldType.array(Schema.FieldType.row(CELL_SCHEMA)))) .build(); + public static final Schema FLATTENED_ROW_SCHEMA = + Schema.builder() + .addByteArrayField("key") + .addStringField("family_name") + .addByteArrayField("column_qualifier") + .addArrayField("cells", Schema.FieldType.row(CELL_SCHEMA)) + .build(); @Override protected SchemaTransform from(BigtableReadSchemaTransformConfiguration configuration) { @@ -88,7 +97,7 @@ public List outputCollectionNames() { /** Configuration for reading from Bigtable. */ @DefaultSchema(AutoValueSchema.class) @AutoValue - public abstract static class BigtableReadSchemaTransformConfiguration { + public abstract static class BigtableReadSchemaTransformConfiguration implements Serializable { /** Instantiates a {@link BigtableReadSchemaTransformConfiguration.Builder} instance. */ public void validate() { String emptyStringMessage = @@ -100,7 +109,8 @@ public void validate() { public static Builder builder() { return new AutoValue_BigtableReadSchemaTransformProvider_BigtableReadSchemaTransformConfiguration - .Builder(); + .Builder() + .setFlatten(true); } public abstract String getTableId(); @@ -109,6 +119,8 @@ public static Builder builder() { public abstract String getProjectId(); + public abstract @Nullable Boolean getFlatten(); + /** Builder for the {@link BigtableReadSchemaTransformConfiguration}. */ @AutoValue.Builder public abstract static class Builder { @@ -118,6 +130,8 @@ public abstract static class Builder { public abstract Builder setProjectId(String projectId); + public abstract Builder setFlatten(Boolean flatten); + /** Builds a {@link BigtableReadSchemaTransformConfiguration} instance. */ public abstract BigtableReadSchemaTransformConfiguration build(); } @@ -152,45 +166,97 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { .withInstanceId(configuration.getInstanceId()) .withProjectId(configuration.getProjectId())); + Schema outputSchema = + Boolean.FALSE.equals(configuration.getFlatten()) ? ROW_SCHEMA : FLATTENED_ROW_SCHEMA; + PCollection beamRows = - bigtableRows.apply(MapElements.via(new BigtableRowToBeamRow())).setRowSchema(ROW_SCHEMA); + bigtableRows + .apply("ConvertToBeamRows", ParDo.of(new BigtableRowConverterDoFn(configuration))) + .setRowSchema(outputSchema); return PCollectionRowTuple.of(OUTPUT_TAG, beamRows); } } - public static class BigtableRowToBeamRow extends SimpleFunction { - @Override - public Row apply(com.google.bigtable.v2.Row bigtableRow) { - // The collection of families is represented as a Map of column families. - // Each column family is represented as a Map of columns. - // Each column is represented as a List of cells - // Each cell is represented as a Beam Row consisting of value and timestamp_micros - Map>> families = new HashMap<>(); - - for (Family fam : bigtableRow.getFamiliesList()) { - // Map of column qualifier to list of cells - Map> columns = new HashMap<>(); - for (Column col : fam.getColumnsList()) { - List cells = new ArrayList<>(); - for (Cell cell : col.getCellsList()) { - Row cellRow = - Row.withSchema(CELL_SCHEMA) - .withFieldValue("value", ByteBuffer.wrap(cell.getValue().toByteArray())) - .withFieldValue("timestamp_micros", cell.getTimestampMicros()) + /** + * A {@link DoFn} that converts a Bigtable {@link com.google.bigtable.v2.Row} to a Beam {@link + * Row}. It supports both a nested representation and a flattened representation where each column + * becomes a separate output element. + */ + private static class BigtableRowConverterDoFn extends DoFn { + private final BigtableReadSchemaTransformConfiguration configuration; + + BigtableRowConverterDoFn(BigtableReadSchemaTransformConfiguration configuration) { + this.configuration = configuration; + } + + private List convertCells(List bigtableCells) { + List beamCells = new ArrayList<>(); + for (Cell cell : bigtableCells) { + Row cellRow = + Row.withSchema(CELL_SCHEMA) + .withFieldValue("value", cell.getValue().toByteArray()) + .withFieldValue("timestamp_micros", cell.getTimestampMicros()) + .build(); + beamCells.add(cellRow); + } + return beamCells; + } + + @ProcessElement + public void processElement( + @Element com.google.bigtable.v2.Row bigtableRow, OutputReceiver out) { + // The builder defaults flatten to true. We check for an explicit false setting to disable it. + + if (Boolean.FALSE.equals(configuration.getFlatten())) { + // Non-flattening logic (original behavior): one output row per Bigtable row. + Map>> families = new HashMap<>(); + for (Family fam : bigtableRow.getFamiliesList()) { + Map> columns = new HashMap<>(); + for (Column col : fam.getColumnsList()) { + + List bigTableCells = col.getCellsList(); + + List cells = convertCells(bigTableCells); + + columns.put(col.getQualifier().toStringUtf8(), cells); + } + families.put(fam.getName(), columns); + } + Row beamRow = + Row.withSchema(ROW_SCHEMA) + .withFieldValue("key", bigtableRow.getKey().toByteArray()) + .withFieldValue("column_families", families) + .build(); + out.output(beamRow); + } else { + // Flattening logic (new behavior): one output row per column qualifier. + byte[] key = bigtableRow.getKey().toByteArray(); + for (Family fam : bigtableRow.getFamiliesList()) { + String familyName = fam.getName(); + for (Column col : fam.getColumnsList()) { + ByteString qualifierName = col.getQualifier(); + List cells = new ArrayList<>(); + for (Cell cell : col.getCellsList()) { + Row cellRow = + Row.withSchema(CELL_SCHEMA) + .withFieldValue("value", cell.getValue().toByteArray()) + .withFieldValue("timestamp_micros", cell.getTimestampMicros()) + .build(); + cells.add(cellRow); + } + + Row flattenedRow = + Row.withSchema(FLATTENED_ROW_SCHEMA) + .withFieldValue("key", key) + .withFieldValue("family_name", familyName) + .withFieldValue("column_qualifier", qualifierName.toByteArray()) + .withFieldValue("cells", cells) .build(); - cells.add(cellRow); + out.output(flattenedRow); } - columns.put(col.getQualifier().toStringUtf8(), cells); } - families.put(fam.getName(), columns); } - Row beamRow = - Row.withSchema(ROW_SCHEMA) - .withFieldValue("key", ByteBuffer.wrap(bigtableRow.getKey().toByteArray())) - .withFieldValue("column_families", families) - .build(); - return beamRow; } } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteSchemaTransformProvider.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteSchemaTransformProvider.java index 480d4199c653..455591543898 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteSchemaTransformProvider.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteSchemaTransformProvider.java @@ -168,7 +168,7 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { validateField(inputSchema, "column_qualifier", Schema.TypeName.BYTES); } if (inputSchema.hasField("family_name")) { - validateField(inputSchema, "family_name", Schema.TypeName.BYTES); + validateField(inputSchema, "family_name", Schema.TypeName.STRING); } if (inputSchema.hasField("timestamp_micros")) { validateField(inputSchema, "timestamp_micros", Schema.TypeName.INT64); @@ -189,7 +189,7 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { + "\"type\": String\n" + "\"value\": ByteString\n" + "\"column_qualifier\": ByteString\n" - + "\"family_name\": ByteString\n" + + "\"family_name\": String\n" + "\"timestamp_micros\": Long\n" + "\"start_timestamp_micros\": Long\n" + "\"end_timestamp_micros\": Long\n" @@ -259,11 +259,10 @@ public PCollection>> changeMutationInput( Preconditions.checkStateNotNull( input.getBytes("column_qualifier"), "Encountered SetCell mutation with null 'column_qualifier' property. "))) - .setFamilyNameBytes( - ByteString.copyFrom( - Preconditions.checkStateNotNull( - input.getBytes("family_name"), - "Encountered SetCell mutation with null 'family_name' property."))); + .setFamilyName( + Preconditions.checkStateNotNull( + input.getString("family_name"), + "Encountered SetCell mutation with null 'family_name' property.")); // Use timestamp if provided, else default to -1 (current // Bigtable // server time) @@ -284,11 +283,10 @@ public PCollection>> changeMutationInput( Preconditions.checkStateNotNull( input.getBytes("column_qualifier"), "Encountered DeleteFromColumn mutation with null 'column_qualifier' property."))) - .setFamilyNameBytes( - ByteString.copyFrom( - Preconditions.checkStateNotNull( - input.getBytes("family_name"), - "Encountered DeleteFromColumn mutation with null 'family_name' property."))); + .setFamilyName( + Preconditions.checkStateNotNull( + input.getString("family_name"), + "Encountered DeleteFromColumn mutation with null 'family_name' property.")); // if start or end timestamp provided // Timestamp Range (optional, assuming Long type in Row schema) @@ -322,11 +320,10 @@ public PCollection>> changeMutationInput( Mutation.newBuilder() .setDeleteFromFamily( Mutation.DeleteFromFamily.newBuilder() - .setFamilyNameBytes( - ByteString.copyFrom( - Preconditions.checkStateNotNull( - input.getBytes("family_name"), - "Encountered DeleteFromFamily mutation with null 'family_name' property."))) + .setFamilyName( + Preconditions.checkStateNotNull( + input.getString("family_name"), + "Encountered DeleteFromFamily mutation with null 'family_name' property.")) .build()) .build(); break; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java index 471675a2c988..c9507475648d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java @@ -614,7 +614,7 @@ static Query translateGqlQueryWithLimitCheck( // limit, so we just check for INVALID_ARGUMENT and assume that that the query might have // a limit already set. if (e.getCode() == Code.INVALID_ARGUMENT) { - LOG.warn("Failed to translate Gql query '{}': {}", gqlQueryWithZeroLimit, e.getMessage()); + LOG.warn("Failed to translate Gql query '{}'", gqlQueryWithZeroLimit, e); LOG.warn("User query might have a limit already set, so trying without zero limit"); // Retry without the zero limit. return translateGqlQuery(gql, datastore, projectId, databaseId, namespace, readTime); @@ -2440,10 +2440,10 @@ private synchronized void flushBatch(ContextAdapter context) // Only log the code and message for potentially-transient errors. The entire exception // will be propagated upon the last retry. LOG.error( - "Error writing batch of {} mutations to Datastore ({}): {}", + "Error writing batch of {} mutations to Datastore ({})", mutations.size(), exception.getCode(), - exception.getMessage()); + exception); rpcErrors.inc(); if (NON_RETRYABLE_ERRORS.contains(exception.getCode())) { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java index 0206d48b813c..0fbd2a1b45a7 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java @@ -1531,7 +1531,6 @@ private void parseResponse(ProcessContext context, HttpBody resp) SUCCESSFUL_BUNDLES, FhirBundleResponse.of(context.element(), bundle.toString())); } EXECUTE_BUNDLE_SUCCESS.inc(); - return; } // parseBundleStatus parses out the status code from a Bundle.entry.response.status string, diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PreparePubsubWriteDoFn.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PreparePubsubWriteDoFn.java index fb096e382994..9171bdf28494 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PreparePubsubWriteDoFn.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PreparePubsubWriteDoFn.java @@ -59,7 +59,7 @@ public class PreparePubsubWriteDoFn extends DoFn private final TupleTag outputTag; - static int validatePubsubMessageSize(PubsubMessage message, int maxPublishBatchSize) + static int validatePubsubMessage(PubsubMessage message, int maxPublishBatchSize) throws SizeLimitExceededException { int payloadSize = message.getPayload().length; if (payloadSize > PUBSUB_MESSAGE_DATA_MAX_BYTES) { @@ -86,7 +86,12 @@ static int validatePubsubMessageSize(PubsubMessage message, int maxPublishBatchS totalSize += orderingKeySize; } - @Nullable Map attributes = message.getAttributeMap(); + final @Nullable Map attributes = message.getAttributeMap(); + if (payloadSize == 0 && (attributes == null || attributes.isEmpty())) { + throw new IllegalArgumentException( + "Pubsub message must contain a non-empty payload or at least one attribute."); + } + if (attributes != null) { if (attributes.size() > PUBSUB_MESSAGE_MAX_ATTRIBUTES) { throw new SizeLimitExceededException( @@ -212,7 +217,7 @@ public void process( message = message.withOrderingKey(null); } try { - validatePubsubMessageSize(message, maxPublishBatchSize); + validatePubsubMessage(message, maxPublishBatchSize); } catch (SizeLimitExceededException e) { badRecordRouter.route( o, diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java index 3c08bcbf2819..d62d294ed2a7 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java @@ -1559,9 +1559,8 @@ public Write withPubsubRootUrl(String pubsubRootUrl) { /** * Writes any serialization failures out to the Error Handler. See {@link ErrorHandler} for - * details on how to configure an Error Handler. Error Handlers are not well supported when - * writing to topics with schemas, and it is not recommended to configure an error handler if - * the target topic has a schema. + * details on how to configure an Error Handler. Schema errors are not handled by Error + * Handlers, and will be handled using the default behavior of the runner. */ public Write withErrorHandler(ErrorHandler badRecordErrorHandler) { return toBuilder() @@ -1738,7 +1737,7 @@ public void processElement(@Element PubsubMessage message, @Timestamp Instant ti // TODO(sjvanrossum): https://github.com/apache/beam/issues/31800 // - Size validation makes no distinction between JSON and Protobuf encoding // - Accounting for HTTP to gRPC transcoding is non-trivial - PreparePubsubWriteDoFn.validatePubsubMessageSize(message, maxPublishBatchByteSize); + PreparePubsubWriteDoFn.validatePubsubMessage(message, maxPublishBatchByteSize); // NOTE: The record id is always null since it will be assigned by Pub/Sub. final OutgoingMessage msg = OutgoingMessage.of(message, timestamp.getMillis(), null, message.getTopic()); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubTestClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubTestClient.java index 7bf342eee8c6..22fcaae20cad 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubTestClient.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubTestClient.java @@ -535,7 +535,7 @@ public List pull( incomingMessageWithRequestTime.ackId(), incomingMessageWithRequestTime); STATE.ackDeadline.put( incomingMessageWithRequestTime.ackId(), - requestTimeMsSinceEpoch + STATE.ackTimeoutSec * 1000); + requestTimeMsSinceEpoch + STATE.ackTimeoutSec * 1000L); if (incomingMessages.size() >= batchSize) { break; } @@ -588,7 +588,7 @@ public void modifyAckDeadline( STATE.pendingAckIncomingMessages.containsKey(ackId), "No message with ACK id %s is waiting for an ACK", ackId); - STATE.ackDeadline.put(ackId, STATE.clock.currentTimeMillis() + deadlineSeconds * 1000); + STATE.ackDeadline.put(ackId, STATE.clock.currentTimeMillis() + deadlineSeconds * 1000L); } else { checkState( STATE.ackDeadline.remove(ackId) != null, diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/CloserReference.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/CloserReference.java index 089f0f2242f1..9853e24e8ff3 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/CloserReference.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/CloserReference.java @@ -60,7 +60,7 @@ public void run() { } } - @SuppressWarnings("deprecation") + @SuppressWarnings({"deprecation", "Finalize"}) @Override protected void finalize() { SystemExecutors.getFuturesExecutor().execute(new Closer(object)); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/MemoryLimiterImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/MemoryLimiterImpl.java index 3f86e880f8de..753c45e0c1e3 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/MemoryLimiterImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/MemoryLimiterImpl.java @@ -82,6 +82,7 @@ public void close() { } @Override + @SuppressWarnings("Finalize") public void finalize() { if (!released) { LOG.error("Failed to release memory block- likely SDF implementation error."); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerAccessor.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerAccessor.java index d81ff4f459c2..96ce735cad4a 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerAccessor.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerAccessor.java @@ -61,6 +61,9 @@ public class SpannerAccessor implements AutoCloseable { */ private static final String USER_AGENT_PREFIX = "Apache_Beam_Java"; + /** Instance ID to use when connecting to an experimental host. */ + public static final String EXPERIMENTAL_HOST_INSTANCE_ID = "default"; + // Only create one SpannerAccessor for each different SpannerConfig. private static final ConcurrentHashMap spannerAccessors = new ConcurrentHashMap<>(); @@ -220,6 +223,24 @@ static SpannerOptions buildSpannerOptions(SpannerConfig spannerConfig) { builder.setServiceFactory(serviceFactory); } builder.setHost(spannerConfig.getHostValue()); + + ValueProvider experimentalHost = spannerConfig.getExperimentalHost(); + if (experimentalHost != null && !Strings.isNullOrEmpty(experimentalHost.get())) { + builder.setExperimentalHost(experimentalHost.get()); + ValueProvider plainText = spannerConfig.getPlainText(); + ValueProvider instanceId = spannerConfig.getInstanceId(); + if (Strings.isNullOrEmpty(instanceId.get()) + || !instanceId.get().equals(EXPERIMENTAL_HOST_INSTANCE_ID)) { + throw new IllegalArgumentException( + "Experimental host can only be used with instance id: " + + EXPERIMENTAL_HOST_INSTANCE_ID); + } + if (plainText != null && Boolean.TRUE.equals(plainText.get())) { + builder.setChannelConfigurator(b -> b.usePlaintext()); + builder.setCredentials(NoCredentials.getInstance()); + } + } + ValueProvider emulatorHost = spannerConfig.getEmulatorHost(); if (emulatorHost != null) { builder.setEmulatorHost(emulatorHost.get()); @@ -269,7 +290,7 @@ private static SpannerAccessor createAndConnect(SpannerConfig spannerConfig) { // fetch instanceConfigId is fail-free. // Do not emit warning when serviceFactory is overridden (e.g. in tests). if (spannerConfig.getServiceFactory() == null) { - LOG.warn("unable to get Spanner instanceConfigId for {}: {}", instanceId, e.getMessage()); + LOG.warn("unable to get Spanner instanceConfigId for {}", instanceId, e); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.java index 5141251f6d94..f52b8378cb6a 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.java @@ -48,6 +48,9 @@ public abstract class SpannerConfig implements Serializable { private static final Duration DEFAULT_COMMIT_DEADLINE = Duration.standardSeconds(15); // Total allowable backoff time. private static final Duration DEFAULT_MAX_CUMULATIVE_BACKOFF = Duration.standardMinutes(15); + // Instance id of experimental hosts + private static final ValueProvider EXPERIMENTAL_HOST_INSTANCE_ID = + ValueProvider.StaticValueProvider.of("default"); // A default priority for batch traffic. static final RpcPriority DEFAULT_RPC_PRIORITY = RpcPriority.MEDIUM; @@ -68,6 +71,8 @@ public String getHostValue() { public abstract @Nullable ValueProvider getEmulatorHost(); + public abstract @Nullable ValueProvider getExperimentalHost(); + public abstract @Nullable ValueProvider getIsLocalChannelProvider(); public abstract @Nullable ValueProvider getCommitDeadline(); @@ -90,6 +95,8 @@ public String getHostValue() { public abstract @Nullable ValueProvider getPartitionReadTimeout(); + public abstract @Nullable ValueProvider getPlainText(); + @VisibleForTesting abstract @Nullable ServiceFactory getServiceFactory(); @@ -149,6 +156,8 @@ public abstract static class Builder { abstract Builder setEmulatorHost(ValueProvider emulatorHost); + abstract Builder setExperimentalHost(ValueProvider experimentalHost); + abstract Builder setIsLocalChannelProvider(ValueProvider isLocalChannelProvider); abstract Builder setCommitDeadline(ValueProvider commitDeadline); @@ -178,6 +187,8 @@ abstract Builder setExecuteStreamingSqlRetrySettings( abstract Builder setCredentials(ValueProvider credentials); + abstract Builder setPlainText(ValueProvider plainText); + public abstract SpannerConfig build(); } @@ -345,4 +356,37 @@ public SpannerConfig withCredentials(Credentials credentials) { public SpannerConfig withCredentials(ValueProvider credentials) { return toBuilder().setCredentials(credentials).build(); } + + /** Specifies the experimental host to set on SpannerOptions (setExperimentalHost). */ + public SpannerConfig withExperimentalHost(ValueProvider experimentalHost) { + return toBuilder() + .setInstanceId(EXPERIMENTAL_HOST_INSTANCE_ID) + .setExperimentalHost(experimentalHost) + .build(); + } + + /** Specifies the experimental host to set on SpannerOptions (setExperimentalHost). */ + public SpannerConfig withExperimentalHost(String experimentalHost) { + return withExperimentalHost(ValueProvider.StaticValueProvider.of(experimentalHost)); + } + + /** + * Specifies whether to use plaintext channel. + * + *

Note: This parameter is only valid when using an experimental host (set via {@code + * withExperimentalHost}). + */ + public SpannerConfig withUsingPlainTextChannel(ValueProvider plainText) { + return toBuilder().setPlainText(plainText).build(); + } + + /** + * Specifies whether to use plaintext channel. + * + *

Note: This parameter is only valid when using an experimental host (set via {@code + * withExperimentalHost}). + */ + public SpannerConfig withUsingPlainTextChannel(boolean plainText) { + return withUsingPlainTextChannel(ValueProvider.StaticValueProvider.of(plainText)); + } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java index d3b2632bad0e..8159118771e4 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java @@ -52,6 +52,8 @@ import com.google.cloud.spanner.Statement; import com.google.cloud.spanner.Struct; import com.google.cloud.spanner.TimestampBound; +import com.google.gson.Gson; +import com.google.gson.GsonBuilder; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -610,6 +612,37 @@ public ReadAll withEmulatorHost(String emulatorHost) { return withEmulatorHost(ValueProvider.StaticValueProvider.of(emulatorHost)); } + /** Specifies the SpannerOptions experimental host (setExperimentalHost). */ + public ReadAll withExperimentalHost(ValueProvider experimentalHost) { + SpannerConfig config = getSpannerConfig(); + return withSpannerConfig(config.withExperimentalHost(experimentalHost)); + } + + public ReadAll withExperimentalHost(String experimentalHost) { + return withExperimentalHost(ValueProvider.StaticValueProvider.of(experimentalHost)); + } + + /** + * Specifies whether to use plaintext channel. + * + *

Note: This parameter is only valid when using an experimental host (set via {@code + * withExperimentalHost}). + */ + public ReadAll withUsingPlainTextChannel(ValueProvider plainText) { + SpannerConfig config = getSpannerConfig(); + return withSpannerConfig(config.withUsingPlainTextChannel(plainText)); + } + + /** + * Specifies whether to use plaintext channel. + * + *

Note: This parameter is only valid when using an experimental host (set via {@code + * withExperimentalHost}). + */ + public ReadAll withUsingPlainTextChannel(boolean plainText) { + return withUsingPlainTextChannel(ValueProvider.StaticValueProvider.of(plainText)); + } + /** Specifies the Cloud Spanner database. */ public ReadAll withDatabaseId(ValueProvider databaseId) { SpannerConfig config = getSpannerConfig(); @@ -839,6 +872,37 @@ public Read withEmulatorHost(String emulatorHost) { return withEmulatorHost(ValueProvider.StaticValueProvider.of(emulatorHost)); } + /** Specifies the SpannerOptions experimental host (setExperimentalHost). */ + public Read withExperimentalHost(ValueProvider experimentalHost) { + SpannerConfig config = getSpannerConfig(); + return withSpannerConfig(config.withExperimentalHost(experimentalHost)); + } + + public Read withExperimentalHost(String experimentalHost) { + return withExperimentalHost(ValueProvider.StaticValueProvider.of(experimentalHost)); + } + + /** + * Specifies whether to use plaintext channel. + * + *

Note: This parameter is only valid when using an experimental host (set via {@code + * withExperimentalHost}). + */ + public Read withUsingPlainTextChannel(ValueProvider plainText) { + SpannerConfig config = getSpannerConfig(); + return withSpannerConfig(config.withUsingPlainTextChannel(plainText)); + } + + /** + * Specifies whether to use plaintext channel. + * + *

Note: This parameter is only valid when using an experimental host (set via {@code + * withExperimentalHost}). + */ + public Read withUsingPlainTextChannel(boolean plainText) { + return withUsingPlainTextChannel(ValueProvider.StaticValueProvider.of(plainText)); + } + /** If true the uses Cloud Spanner batch API. */ public Read withBatching(boolean batching) { return toBuilder().setBatching(batching).build(); @@ -1015,6 +1079,32 @@ public PCollection expand(PBegin input) { } } + static class ChangeStreamRead extends PTransform> { + + ReadChangeStream readChangeStream; + + public ChangeStreamRead(ReadChangeStream readChangeStream) { + this.readChangeStream = readChangeStream; + } + + @Override + public PCollection expand(PBegin input) { + return input + .apply(readChangeStream) + .apply("DataChangeRecordToStringJSON", ParDo.of(new DataChangeRecordToJsonFn())); + } + } + + private static class DataChangeRecordToJsonFn extends DoFn { + private static Gson gson = new GsonBuilder().disableHtmlEscaping().create(); + + @ProcessElement + public void process(@Element DataChangeRecord input, OutputReceiver receiver) { + String modJsonString = gson.toJson(input, DataChangeRecord.class); + receiver.output(modJsonString); + } + } + /** * A {@link PTransform} that create a transaction. If applied to a {@link PCollection}, it will * create a transaction after the {@link PCollection} is closed. @@ -1109,6 +1199,37 @@ public CreateTransaction withEmulatorHost(String emulatorHost) { return withEmulatorHost(ValueProvider.StaticValueProvider.of(emulatorHost)); } + /** Specifies the SpannerOptions experimental host (setExperimentalHost). */ + public CreateTransaction withExperimentalHost(ValueProvider experimentalHost) { + SpannerConfig config = getSpannerConfig(); + return withSpannerConfig(config.withExperimentalHost(experimentalHost)); + } + + public CreateTransaction withExperimentalHost(String experimentalHost) { + return withExperimentalHost(ValueProvider.StaticValueProvider.of(experimentalHost)); + } + + /** + * Specifies whether to use plaintext channel. + * + *

Note: This parameter is only valid when using an experimental host (set via {@code + * withExperimentalHost}). + */ + public CreateTransaction withUsingPlainTextChannel(ValueProvider plainText) { + SpannerConfig config = getSpannerConfig(); + return withSpannerConfig(config.withUsingPlainTextChannel(plainText)); + } + + /** + * Specifies whether to use plaintext channel. + * + *

Note: This parameter is only valid when using an experimental host (set via {@code + * withExperimentalHost}). + */ + public CreateTransaction withUsingPlainTextChannel(boolean plainText) { + return withUsingPlainTextChannel(ValueProvider.StaticValueProvider.of(plainText)); + } + @VisibleForTesting CreateTransaction withServiceFactory(ServiceFactory serviceFactory) { SpannerConfig config = getSpannerConfig(); @@ -1246,6 +1367,37 @@ public Write withEmulatorHost(String emulatorHost) { return withEmulatorHost(ValueProvider.StaticValueProvider.of(emulatorHost)); } + /** Specifies the SpannerOptions experimental host (setExperimentalHost). */ + public Write withExperimentalHost(ValueProvider experimentalHost) { + SpannerConfig config = getSpannerConfig(); + return withSpannerConfig(config.withExperimentalHost(experimentalHost)); + } + + public Write withExperimentalHost(String experimentalHost) { + return withExperimentalHost(ValueProvider.StaticValueProvider.of(experimentalHost)); + } + + /** + * Specifies whether to use plaintext channel. + * + *

Note: This parameter is only valid when using an experimental host (set via {@code + * withExperimentalHost}). + */ + public Write withUsingPlainTextChannel(ValueProvider plainText) { + SpannerConfig config = getSpannerConfig(); + return withSpannerConfig(config.withUsingPlainTextChannel(plainText)); + } + + /** + * Specifies whether to use plaintext channel. + * + *

Note: This parameter is only valid when using an experimental host (set via {@code + * withExperimentalHost}). + */ + public Write withUsingPlainTextChannel(boolean plainText) { + return withUsingPlainTextChannel(ValueProvider.StaticValueProvider.of(plainText)); + } + public Write withDialectView(PCollectionView dialect) { return toBuilder().setDialectView(dialect).build(); } @@ -1598,6 +1750,10 @@ public abstract static class ReadChangeStream abstract @Nullable Duration getWatermarkRefreshRate(); + abstract @Nullable ValueProvider getExperimentalHost(); + + abstract @Nullable ValueProvider getPlainText(); + abstract Builder toBuilder(); @AutoValue.Builder @@ -1623,6 +1779,10 @@ abstract static class Builder { abstract Builder setWatermarkRefreshRate(Duration refreshRate); + abstract Builder setExperimentalHost(ValueProvider experimentalHost); + + abstract Builder setPlainText(ValueProvider plainText); + abstract ReadChangeStream build(); } @@ -1713,6 +1873,38 @@ public ReadChangeStream withWatermarkRefreshRate(Duration refreshRate) { return toBuilder().setWatermarkRefreshRate(refreshRate).build(); } + /** Specifies the experimental host to set on SpannerOptions (setExperimentalHost). */ + public ReadChangeStream withExperimentalHost(ValueProvider experimentalHost) { + SpannerConfig config = getSpannerConfig(); + return withSpannerConfig(config.withExperimentalHost(experimentalHost)); + } + + /** Specifies the experimental host to set on SpannerOptions (setExperimentalHost). */ + public ReadChangeStream withExperimentalHost(String experimentalHost) { + return withExperimentalHost(ValueProvider.StaticValueProvider.of(experimentalHost)); + } + + /** + * Specifies whether to use plaintext channel. + * + *

Note: This parameter is only valid when using an experimental host (set via {@code + * withExperimentalHost}). + */ + public ReadChangeStream withUsingPlainTextChannel(ValueProvider plainText) { + SpannerConfig config = getSpannerConfig(); + return withSpannerConfig(config.withUsingPlainTextChannel(plainText)); + } + + /** + * Specifies whether to use plaintext channel. + * + *

Note: This parameter is only valid when using an experimental host (set via {@code + * withExperimentalHost}). + */ + public ReadChangeStream withUsingPlainTextChannel(boolean plainText) { + return withUsingPlainTextChannel(ValueProvider.StaticValueProvider.of(plainText)); + } + @Override public PCollection expand(PBegin input) { checkArgument( @@ -2433,11 +2625,10 @@ private void writeMutations(Iterable mutationIterable) } LOG.info( "DEADLINE_EXCEEDED writing batch of {} mutations to Cloud Spanner, " - + "retrying after backoff of {}ms\n" - + "({})", + + "retrying after backoff of {}ms", mutations.size(), sleepTimeMsecs, - exception.getMessage()); + exception); spannerWriteRetries.inc(); try { sleeper.sleep(sleepTimeMsecs); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerTransformRegistrar.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerTransformRegistrar.java index 72b51beadb57..70908f982721 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerTransformRegistrar.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerTransformRegistrar.java @@ -23,6 +23,7 @@ import com.google.auto.service.AutoService; import com.google.cloud.Timestamp; import com.google.cloud.spanner.Mutation; +import com.google.cloud.spanner.Options.RpcPriority; import com.google.cloud.spanner.TimestampBound; import java.util.Map; import java.util.concurrent.TimeUnit; @@ -43,8 +44,8 @@ import org.joda.time.Duration; /** - * Exposes {@link SpannerIO.WriteRows} and {@link SpannerIO.ReadRows} as an external transform for - * cross-language usage. + * Exposes {@link SpannerIO.WriteRows}, {@link SpannerIO.ReadRows} and {@link + * SpannerIO.ChangeStreamRead} as an external transform for cross-language usage. */ @AutoService(ExternalTransformRegistrar.class) public class SpannerTransformRegistrar implements ExternalTransformRegistrar { @@ -55,6 +56,8 @@ public class SpannerTransformRegistrar implements ExternalTransformRegistrar { "beam:transform:org.apache.beam:spanner_insert_or_update:v1"; public static final String DELETE_URN = "beam:transform:org.apache.beam:spanner_delete:v1"; public static final String READ_URN = "beam:transform:org.apache.beam:spanner_read:v1"; + public static final String READ_CHANGE_STREAM_URN = + "beam:transform:org.apache.beam:spanner_change_stream_reader:v1"; @Override @NonNull @@ -66,6 +69,7 @@ public class SpannerTransformRegistrar implements ExternalTransformRegistrar { .put(INSERT_OR_UPDATE_URN, new InsertOrUpdateBuilder()) .put(DELETE_URN, new DeleteBuilder()) .put(READ_URN, new ReadBuilder()) + .put(READ_CHANGE_STREAM_URN, new ChangeStreamReaderBuilder()) .build(); } @@ -75,6 +79,8 @@ public abstract static class CrossLanguageConfiguration { String projectId = ""; @Nullable String host; @Nullable String emulatorHost; + @Nullable String experimentalHost; + @Nullable Boolean plainText; public void setInstanceId(String instanceId) { this.instanceId = instanceId; @@ -96,6 +102,14 @@ public void setEmulatorHost(@Nullable String emulatorHost) { this.emulatorHost = emulatorHost; } + public void setExperimentalHost(@Nullable String experimentalHost) { + this.experimentalHost = experimentalHost; + } + + public void setPlainText(@Nullable Boolean plainText) { + this.plainText = plainText; + } + void checkMandatoryFields() { if (projectId.isEmpty()) { throw new IllegalArgumentException("projectId can't be empty"); @@ -229,6 +243,12 @@ public PTransform> buildExternal( if (configuration.emulatorHost != null) { readTransform = readTransform.withEmulatorHost(configuration.emulatorHost); } + if (configuration.experimentalHost != null) { + readTransform = readTransform.withExperimentalHost(configuration.experimentalHost); + } + if (configuration.plainText != null) { + readTransform = readTransform.withUsingPlainTextChannel(configuration.plainText); + } @Nullable TimestampBound timestampBound = configuration.getTimestampBound(); if (timestampBound != null) { readTransform = readTransform.withTimestampBound(timestampBound); @@ -367,6 +387,12 @@ public PTransform, PDone> buildExternal( if (configuration.emulatorHost != null) { writeTransform = writeTransform.withEmulatorHost(configuration.emulatorHost); } + if (configuration.experimentalHost != null) { + writeTransform = writeTransform.withExperimentalHost(configuration.experimentalHost); + } + if (configuration.plainText != null) { + writeTransform = writeTransform.withUsingPlainTextChannel(configuration.plainText); + } if (configuration.commitDeadline != null) { writeTransform = writeTransform.withCommitDeadline(configuration.commitDeadline); } @@ -382,4 +408,113 @@ public PTransform, PDone> buildExternal( return SpannerIO.WriteRows.of(writeTransform, operation, configuration.table); } } + + public static class ChangeStreamReaderBuilder + implements ExternalTransformBuilder< + ChangeStreamReaderBuilder.Configuration, PBegin, PCollection> { + + public static class Configuration extends CrossLanguageConfiguration { + private String changeStreamName = ""; + private String metadataDatabase = ""; + private String metadataInstance = ""; + private @Nullable Timestamp inclusiveStartAt; + private @Nullable Timestamp inclusiveEndAt; + private @Nullable String metadataTable; + private @Nullable RpcPriority rpcPriority; + private @Nullable Duration watermarkRefreshRate; + + public void setChangeStreamName(String changeStreamName) { + this.changeStreamName = changeStreamName; + } + + public void setInclusiveStartAt(@Nullable String inclusiveStartAtString) { + if (inclusiveStartAtString != null) { + this.inclusiveStartAt = Timestamp.parseTimestamp(inclusiveStartAtString); + } + } + + public void setInclusiveEndAt(@Nullable String inclusiveEndAtString) { + if (inclusiveEndAtString != null) { + this.inclusiveEndAt = Timestamp.parseTimestamp(inclusiveEndAtString); + } + } + + public void setMetadataDatabase(String metadataDatabase) { + this.metadataDatabase = metadataDatabase; + } + + public void setMetadataInstance(String metadataInstance) { + this.metadataInstance = metadataInstance; + } + + public void setMetadataTable(@Nullable String metadataTable) { + this.metadataTable = metadataTable; + } + + public void setRpcPriority(@Nullable String rpcPriorityString) { + if (rpcPriorityString != null) { + this.rpcPriority = RpcPriority.valueOf(rpcPriorityString); + } + } + + public void setWatermarkRefreshRate(@Nullable String watermarkRefreshRateString) { + if (watermarkRefreshRateString != null) { + this.watermarkRefreshRate = Duration.parse(watermarkRefreshRateString); + } + } + } + + @Override + @NonNull + public PTransform> buildExternal( + ChangeStreamReaderBuilder.Configuration configuration) { + + configuration.checkMandatoryFields(); + + if (configuration.changeStreamName.isEmpty()) { + throw new IllegalArgumentException("ChangeStreamName can't be empty"); + } + + if (configuration.metadataInstance.isEmpty()) { + throw new IllegalArgumentException("MetadataInstance can't be empty"); + } + + if (configuration.metadataDatabase.isEmpty()) { + throw new IllegalArgumentException("MetadataDatabase can't be empty"); + } + + SpannerIO.ReadChangeStream readChangeStream = + SpannerIO.readChangeStream() + .withProjectId(configuration.projectId) + .withInstanceId(configuration.instanceId) + .withDatabaseId(configuration.databaseId) + .withChangeStreamName(configuration.changeStreamName) + .withMetadataInstance(configuration.metadataInstance) + .withMetadataDatabase(configuration.metadataDatabase); + + if (configuration.inclusiveStartAt != null) { + readChangeStream = readChangeStream.withInclusiveStartAt(configuration.inclusiveStartAt); + } + + if (configuration.inclusiveEndAt != null) { + readChangeStream = readChangeStream.withInclusiveEndAt(configuration.inclusiveEndAt); + } + + if (configuration.metadataTable != null) { + readChangeStream = readChangeStream.withMetadataTable(configuration.metadataTable); + } + + if (configuration.rpcPriority != null) { + + readChangeStream = readChangeStream.withRpcPriority(configuration.rpcPriority); + } + + if (configuration.watermarkRefreshRate != null) { + readChangeStream = + readChangeStream.withWatermarkRefreshRate(configuration.watermarkRefreshRate); + } + + return new SpannerIO.ChangeStreamRead(readChangeStream); + } + } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/MetadataSpannerConfigFactory.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/MetadataSpannerConfigFactory.java index 7132d4deb030..959582e9c35f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/MetadataSpannerConfigFactory.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/MetadataSpannerConfigFactory.java @@ -77,6 +77,16 @@ public static SpannerConfig create( config = config.withEmulatorHost(StaticValueProvider.of(emulatorHost.get())); } + ValueProvider experimentalHost = primaryConfig.getExperimentalHost(); + if (experimentalHost != null && experimentalHost.get() != null) { + config = config.withExperimentalHost(experimentalHost.get()); + } + + ValueProvider plainText = primaryConfig.getPlainText(); + if (plainText != null && plainText.get() != null) { + config = config.withUsingPlainTextChannel(plainText.get()); + } + ValueProvider isLocalChannelProvider = primaryConfig.getIsLocalChannelProvider(); if (isLocalChannelProvider != null) { config = diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/DetectNewPartitionsAction.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/DetectNewPartitionsAction.java index 40160de7b958..080372d04593 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/DetectNewPartitionsAction.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/DetectNewPartitionsAction.java @@ -149,7 +149,7 @@ private ProcessContinuation schedulePartitions( RestrictionTracker tracker, OutputReceiver receiver, Timestamp minWatermark, - TreeMap> batches) { + Map> batches) { List batchPartitionsDifferentCreatedAt = new ArrayList<>(); int numTimestampsHandledSofar = 0; for (Map.Entry> batch : batches.entrySet()) { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/QueryChangeStreamAction.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/QueryChangeStreamAction.java index 344300b9322d..3176abd9f247 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/QueryChangeStreamAction.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/QueryChangeStreamAction.java @@ -325,7 +325,7 @@ private BundleFinalizer.Callback updateWatermarkCallback( if (e.getErrorCode() == ErrorCode.NOT_FOUND) { LOG.debug("[{}] Unable to update the current watermark, partition NOT FOUND", token); } else { - LOG.error("[{}] Error updating the current watermark: {}", token, e.getMessage(), e); + LOG.error("[{}] Error updating the current watermark", token, e); } } }; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeDatasetService.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeDatasetService.java index 3d100413cb2d..77fc7cab0245 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeDatasetService.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeDatasetService.java @@ -84,7 +84,8 @@ /** A fake dataset service that can be serialized, for use in testReadFromTable. */ @Internal @SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) + "nullness", // TODO(https://github.com/apache/beam/issues/20497) + "LockOnNonEnclosingClassLiteral" }) public class FakeDatasetService implements DatasetService, WriteStreamService, Serializable { // Table information must be static, as each ParDo will get a separate instance of diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadSchemaTransformProviderIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadSchemaTransformProviderIT.java index 81d3103f38bf..99db2641fa4f 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadSchemaTransformProviderIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadSchemaTransformProviderIT.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.io.gcp.bigtable; import static org.apache.beam.sdk.io.gcp.bigtable.BigtableReadSchemaTransformProvider.CELL_SCHEMA; +import static org.apache.beam.sdk.io.gcp.bigtable.BigtableReadSchemaTransformProvider.FLATTENED_ROW_SCHEMA; import static org.apache.beam.sdk.io.gcp.bigtable.BigtableReadSchemaTransformProvider.ROW_SCHEMA; import static org.junit.Assert.assertThrows; @@ -28,7 +29,6 @@ import com.google.cloud.bigtable.data.v2.BigtableDataClient; import com.google.cloud.bigtable.data.v2.BigtableDataSettings; import com.google.cloud.bigtable.data.v2.models.RowMutation; -import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; @@ -130,101 +130,200 @@ public void tearDown() { tableAdminClient.deleteTable(tableId); LOG.info("Table {} deleted successfully.", tableId); } catch (NotFoundException e) { - LOG.warn("Failed to delete a non-existent table [{}]: \n{}", tableId, e.getMessage()); + LOG.warn("Failed to delete a non-existent table [{}]", tableId, e); } dataClient.close(); tableAdminClient.close(); } - public List writeToTable(int numRows) { + @Test + public void testRead() { + int numRows = 20; List expectedRows = new ArrayList<>(); + for (int i = 1; i <= numRows; i++) { + String key = "key" + i; + byte[] keyBytes = key.getBytes(StandardCharsets.UTF_8); + String valueA = "value a" + i; + byte[] valueABytes = valueA.getBytes(StandardCharsets.UTF_8); + String valueB = "value b" + i; + byte[] valueBBytes = valueB.getBytes(StandardCharsets.UTF_8); + String valueC = "value c" + i; + byte[] valueCBytes = valueC.getBytes(StandardCharsets.UTF_8); + String valueD = "value d" + i; + byte[] valueDBytes = valueD.getBytes(StandardCharsets.UTF_8); + long timestamp = 1000L * i; - try { - for (int i = 1; i <= numRows; i++) { - String key = "key" + i; - String valueA = "value a" + i; - String valueB = "value b" + i; - String valueC = "value c" + i; - String valueD = "value d" + i; - long timestamp = 1000L * i; - - RowMutation rowMutation = - RowMutation.create(tableId, key) - .setCell(COLUMN_FAMILY_NAME_1, "a", timestamp, valueA) - .setCell(COLUMN_FAMILY_NAME_1, "b", timestamp, valueB) - .setCell(COLUMN_FAMILY_NAME_2, "c", timestamp, valueC) - .setCell(COLUMN_FAMILY_NAME_2, "d", timestamp, valueD); - dataClient.mutateRow(rowMutation); - - // Set up expected Beam Row - Map> columns1 = new HashMap<>(); - columns1.put( - "a", - Arrays.asList( - Row.withSchema(CELL_SCHEMA) - .withFieldValue( - "value", ByteBuffer.wrap(valueA.getBytes(StandardCharsets.UTF_8))) - .withFieldValue("timestamp_micros", timestamp) - .build())); - columns1.put( - "b", - Arrays.asList( - Row.withSchema(CELL_SCHEMA) - .withFieldValue( - "value", ByteBuffer.wrap(valueB.getBytes(StandardCharsets.UTF_8))) - .withFieldValue("timestamp_micros", timestamp) - .build())); - - Map> columns2 = new HashMap<>(); - columns2.put( - "c", - Arrays.asList( - Row.withSchema(CELL_SCHEMA) - .withFieldValue( - "value", ByteBuffer.wrap(valueC.getBytes(StandardCharsets.UTF_8))) - .withFieldValue("timestamp_micros", timestamp) - .build())); - columns2.put( - "d", - Arrays.asList( - Row.withSchema(CELL_SCHEMA) - .withFieldValue( - "value", ByteBuffer.wrap(valueD.getBytes(StandardCharsets.UTF_8))) - .withFieldValue("timestamp_micros", timestamp) - .build())); - - Map>> families = new HashMap<>(); - families.put(COLUMN_FAMILY_NAME_1, columns1); - families.put(COLUMN_FAMILY_NAME_2, columns2); - - Row expectedRow = - Row.withSchema(ROW_SCHEMA) - .withFieldValue("key", ByteBuffer.wrap(key.getBytes(StandardCharsets.UTF_8))) - .withFieldValue("column_families", families) - .build(); - - expectedRows.add(expectedRow); - } - LOG.info("Finished writing {} rows to table {}", numRows, tableId); - } catch (NotFoundException e) { - throw new RuntimeException("Failed to write to table", e); + RowMutation rowMutation = + RowMutation.create(tableId, key) + .setCell(COLUMN_FAMILY_NAME_1, "a", timestamp, valueA) + .setCell(COLUMN_FAMILY_NAME_1, "b", timestamp, valueB) + .setCell(COLUMN_FAMILY_NAME_2, "c", timestamp, valueC) + .setCell(COLUMN_FAMILY_NAME_2, "d", timestamp, valueD); + dataClient.mutateRow(rowMutation); + + // Set up expected Beam Row + Map> columns1 = new HashMap<>(); + columns1.put( + "a", + Arrays.asList( + Row.withSchema(CELL_SCHEMA) + .withFieldValue("value", valueABytes) + .withFieldValue("timestamp_micros", timestamp) + .build())); + columns1.put( + "b", + Arrays.asList( + Row.withSchema(CELL_SCHEMA) + .withFieldValue("value", valueBBytes) + .withFieldValue("timestamp_micros", timestamp) + .build())); + + Map> columns2 = new HashMap<>(); + columns2.put( + "c", + Arrays.asList( + Row.withSchema(CELL_SCHEMA) + .withFieldValue("value", valueCBytes) + .withFieldValue("timestamp_micros", timestamp) + .build())); + columns2.put( + "d", + Arrays.asList( + Row.withSchema(CELL_SCHEMA) + .withFieldValue("value", valueDBytes) + .withFieldValue("timestamp_micros", timestamp) + .build())); + + Map>> families = new HashMap<>(); + families.put(COLUMN_FAMILY_NAME_1, columns1); + families.put(COLUMN_FAMILY_NAME_2, columns2); + + Row expectedRow = + Row.withSchema(ROW_SCHEMA) + .withFieldValue("key", keyBytes) + .withFieldValue("column_families", families) + .build(); + + expectedRows.add(expectedRow); } - return expectedRows; + LOG.info("Finished writing {} rows to table {}", numRows, tableId); + + BigtableReadSchemaTransformConfiguration config = + BigtableReadSchemaTransformConfiguration.builder() + .setTableId(tableId) + .setInstanceId(instanceId) + .setProjectId(projectId) + .setFlatten(false) + .build(); + + SchemaTransform transform = new BigtableReadSchemaTransformProvider().from(config); + + PCollection rows = PCollectionRowTuple.empty(p).apply(transform).get("output"); + + PAssert.that(rows).containsInAnyOrder(expectedRows); + p.run().waitUntilFinish(); } @Test - public void testRead() { - List expectedRows = writeToTable(20); + public void testReadFlatten() { + int numRows = 20; + List expectedRows = new ArrayList<>(); + for (int i = 1; i <= numRows; i++) { + String key = "key" + i; + byte[] keyBytes = key.getBytes(StandardCharsets.UTF_8); + String valueA = "value a" + i; + byte[] valueABytes = valueA.getBytes(StandardCharsets.UTF_8); + String valueB = "value b" + i; + byte[] valueBBytes = valueB.getBytes(StandardCharsets.UTF_8); + String valueC = "value c" + i; + byte[] valueCBytes = valueC.getBytes(StandardCharsets.UTF_8); + String valueD = "value d" + i; + byte[] valueDBytes = valueD.getBytes(StandardCharsets.UTF_8); + long timestamp = 1000L * i; + // Write a row with four distinct columns to Bigtable + RowMutation rowMutation = + RowMutation.create(tableId, key) + .setCell(COLUMN_FAMILY_NAME_1, "a", timestamp, valueA) + .setCell(COLUMN_FAMILY_NAME_1, "b", timestamp, valueB) + .setCell(COLUMN_FAMILY_NAME_2, "c", timestamp, valueC) + .setCell(COLUMN_FAMILY_NAME_2, "d", timestamp, valueD); + dataClient.mutateRow(rowMutation); + + // For each Bigtable row, we expect four flattened Beam Rows as output. + // Each Row corresponds to one column. + expectedRows.add( + Row.withSchema(FLATTENED_ROW_SCHEMA) + .withFieldValue("key", keyBytes) + .withFieldValue("family_name", COLUMN_FAMILY_NAME_1) + .withFieldValue("column_qualifier", "a".getBytes(StandardCharsets.UTF_8)) + .withFieldValue( + "cells", + Arrays.asList( + Row.withSchema(CELL_SCHEMA) + .withFieldValue("value", valueABytes) + .withFieldValue("timestamp_micros", timestamp) + .build())) + .build()); + + expectedRows.add( + Row.withSchema(FLATTENED_ROW_SCHEMA) + .withFieldValue("key", keyBytes) + .withFieldValue("family_name", COLUMN_FAMILY_NAME_1) + .withFieldValue("column_qualifier", "b".getBytes(StandardCharsets.UTF_8)) + .withFieldValue( + "cells", + Arrays.asList( + Row.withSchema(CELL_SCHEMA) + .withFieldValue("value", valueBBytes) + .withFieldValue("timestamp_micros", timestamp) + .build())) + .build()); + + expectedRows.add( + Row.withSchema(FLATTENED_ROW_SCHEMA) + .withFieldValue("key", keyBytes) + .withFieldValue("family_name", COLUMN_FAMILY_NAME_2) + .withFieldValue("column_qualifier", "c".getBytes(StandardCharsets.UTF_8)) + .withFieldValue( + "cells", + Arrays.asList( + Row.withSchema(CELL_SCHEMA) + .withFieldValue("value", valueCBytes) + .withFieldValue("timestamp_micros", timestamp) + .build())) + .build()); + + expectedRows.add( + Row.withSchema(FLATTENED_ROW_SCHEMA) + .withFieldValue("key", keyBytes) + .withFieldValue("family_name", COLUMN_FAMILY_NAME_2) + .withFieldValue("column_qualifier", "d".getBytes(StandardCharsets.UTF_8)) + .withFieldValue( + "cells", + Arrays.asList( + Row.withSchema(CELL_SCHEMA) + .withFieldValue("value", valueDBytes) + .withFieldValue("timestamp_micros", timestamp) + .build())) + .build()); + } + LOG.info("Finished writing {} rows to table {} with Flatten state true", numRows, tableId); + + // Configure the transform to use flatten mode (the default). BigtableReadSchemaTransformConfiguration config = BigtableReadSchemaTransformConfiguration.builder() .setTableId(tableId) .setInstanceId(instanceId) .setProjectId(projectId) + .setFlatten(true) .build(); + SchemaTransform transform = new BigtableReadSchemaTransformProvider().from(config); PCollection rows = PCollectionRowTuple.empty(p).apply(transform).get("output"); + + // Assert that the actual rows match the expected flattened rows. PAssert.that(rows).containsInAnyOrder(expectedRows); p.run().waitUntilFinish(); } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableSimpleWriteSchemaTransformProviderIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableSimpleWriteSchemaTransformProviderIT.java index 7a5dcdc3e999..eceb1ddff4be 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableSimpleWriteSchemaTransformProviderIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableSimpleWriteSchemaTransformProviderIT.java @@ -156,7 +156,7 @@ public void testSetMutationsExistingColumn() { .addStringField("type") .addByteArrayField("value") .addByteArrayField("column_qualifier") - .addByteArrayField("family_name") + .addStringField("family_name") .addField("timestamp_micros", FieldType.INT64) // Changed to INT64 .build(); @@ -166,7 +166,7 @@ public void testSetMutationsExistingColumn() { .withFieldValue("type", "SetCell") .withFieldValue("value", "new-val-1-a".getBytes(StandardCharsets.UTF_8)) .withFieldValue("column_qualifier", "col_a".getBytes(StandardCharsets.UTF_8)) - .withFieldValue("family_name", COLUMN_FAMILY_NAME_1.getBytes(StandardCharsets.UTF_8)) + .withFieldValue("family_name", COLUMN_FAMILY_NAME_1) .withFieldValue("timestamp_micros", 2000L) .build(); Row mutationRow2 = @@ -175,7 +175,7 @@ public void testSetMutationsExistingColumn() { .withFieldValue("type", "SetCell") .withFieldValue("value", "new-val-1-c".getBytes(StandardCharsets.UTF_8)) .withFieldValue("column_qualifier", "col_c".getBytes(StandardCharsets.UTF_8)) - .withFieldValue("family_name", COLUMN_FAMILY_NAME_2.getBytes(StandardCharsets.UTF_8)) + .withFieldValue("family_name", COLUMN_FAMILY_NAME_2) .withFieldValue("timestamp_micros", 2000L) .build(); @@ -225,7 +225,7 @@ public void testSetMutationNewColumn() { .addStringField("type") .addByteArrayField("value") .addByteArrayField("column_qualifier") - .addByteArrayField("family_name") + .addStringField("family_name") .addField("timestamp_micros", FieldType.INT64) .build(); Row mutationRow = @@ -234,7 +234,7 @@ public void testSetMutationNewColumn() { .withFieldValue("type", "SetCell") .withFieldValue("value", "new-val-1".getBytes(StandardCharsets.UTF_8)) .withFieldValue("column_qualifier", "new_col".getBytes(StandardCharsets.UTF_8)) - .withFieldValue("family_name", COLUMN_FAMILY_NAME_1.getBytes(StandardCharsets.UTF_8)) + .withFieldValue("family_name", COLUMN_FAMILY_NAME_1) .withFieldValue("timestamp_micros", 999_000L) .build(); @@ -276,14 +276,14 @@ public void testDeleteCellsFromColumn() { .addByteArrayField("key") .addStringField("type") .addByteArrayField("column_qualifier") - .addByteArrayField("family_name") + .addStringField("family_name") .build(); Row mutationRow = Row.withSchema(testSchema) .withFieldValue("key", "key-1".getBytes(StandardCharsets.UTF_8)) .withFieldValue("type", "DeleteFromColumn") .withFieldValue("column_qualifier", "col_a".getBytes(StandardCharsets.UTF_8)) - .withFieldValue("family_name", COLUMN_FAMILY_NAME_1.getBytes(StandardCharsets.UTF_8)) + .withFieldValue("family_name", COLUMN_FAMILY_NAME_1) .build(); PCollection inputPCollection = p.apply(Create.of(Arrays.asList(mutationRow))); @@ -325,7 +325,7 @@ public void testDeleteCellsFromColumnWithTimestampRange() { .addByteArrayField("key") .addStringField("type") .addByteArrayField("column_qualifier") - .addByteArrayField("family_name") + .addStringField("family_name") .addField("start_timestamp_micros", FieldType.INT64) .addField("end_timestamp_micros", FieldType.INT64) .build(); @@ -334,7 +334,7 @@ public void testDeleteCellsFromColumnWithTimestampRange() { .withFieldValue("key", "key-1".getBytes(StandardCharsets.UTF_8)) .withFieldValue("type", "DeleteFromColumn") .withFieldValue("column_qualifier", "col".getBytes(StandardCharsets.UTF_8)) - .withFieldValue("family_name", COLUMN_FAMILY_NAME_1.getBytes(StandardCharsets.UTF_8)) + .withFieldValue("family_name", COLUMN_FAMILY_NAME_1) .withFieldValue("start_timestamp_micros", 99_990_000L) .withFieldValue("end_timestamp_micros", 100_000_000L) .build(); @@ -373,13 +373,13 @@ public void testDeleteColumnFamily() { Schema.builder() .addByteArrayField("key") .addStringField("type") - .addByteArrayField("family_name") + .addStringField("family_name") .build(); Row mutationRow = Row.withSchema(testSchema) .withFieldValue("key", "key-1".getBytes(StandardCharsets.UTF_8)) .withFieldValue("type", "DeleteFromFamily") - .withFieldValue("family_name", COLUMN_FAMILY_NAME_1.getBytes(StandardCharsets.UTF_8)) + .withFieldValue("family_name", COLUMN_FAMILY_NAME_1) .build(); PCollection inputPCollection = p.apply(Create.of(Arrays.asList(mutationRow))); @@ -484,7 +484,7 @@ public void testAllMutations() { "column_qualifier", FieldType.BYTES) // Used by SetCell, DeleteFromColumn .addNullableField( "family_name", - FieldType.BYTES) // Used by SetCell, DeleteFromColumn, DeleteFromFamily + FieldType.STRING) // Used by SetCell, DeleteFromColumn, DeleteFromFamily .addNullableField("timestamp_micros", FieldType.INT64) // Optional for SetCell .addNullableField( "start_timestamp_micros", FieldType.INT64) // Used by DeleteFromColumn with range @@ -503,7 +503,7 @@ public void testAllMutations() { .withFieldValue("type", "SetCell") .withFieldValue("value", "updated_val_1".getBytes(StandardCharsets.UTF_8)) .withFieldValue("column_qualifier", "col_initial_1".getBytes(StandardCharsets.UTF_8)) - .withFieldValue("family_name", COLUMN_FAMILY_NAME_1.getBytes(StandardCharsets.UTF_8)) + .withFieldValue("family_name", COLUMN_FAMILY_NAME_1) .withFieldValue("timestamp_micros", 3000L) .build()); // Add new cell to "row-setcell" @@ -513,7 +513,7 @@ public void testAllMutations() { .withFieldValue("type", "SetCell") .withFieldValue("value", "new_col_val".getBytes(StandardCharsets.UTF_8)) .withFieldValue("column_qualifier", "new_col_A".getBytes(StandardCharsets.UTF_8)) - .withFieldValue("family_name", COLUMN_FAMILY_NAME_1.getBytes(StandardCharsets.UTF_8)) + .withFieldValue("family_name", COLUMN_FAMILY_NAME_1) .withFieldValue("timestamp_micros", 4000L) .build()); @@ -524,7 +524,7 @@ public void testAllMutations() { .withFieldValue("key", "row-delete-col".getBytes(StandardCharsets.UTF_8)) .withFieldValue("type", "DeleteFromColumn") .withFieldValue("column_qualifier", "col_to_delete_A".getBytes(StandardCharsets.UTF_8)) - .withFieldValue("family_name", COLUMN_FAMILY_NAME_1.getBytes(StandardCharsets.UTF_8)) + .withFieldValue("family_name", COLUMN_FAMILY_NAME_1) .build()); // 3. DeleteFromColumn with Timestamp Range @@ -534,7 +534,7 @@ public void testAllMutations() { .withFieldValue("key", "row-delete-col-ts".getBytes(StandardCharsets.UTF_8)) .withFieldValue("type", "DeleteFromColumn") .withFieldValue("column_qualifier", "ts_col".getBytes(StandardCharsets.UTF_8)) - .withFieldValue("family_name", COLUMN_FAMILY_NAME_1.getBytes(StandardCharsets.UTF_8)) + .withFieldValue("family_name", COLUMN_FAMILY_NAME_1) .withFieldValue("start_timestamp_micros", 999L) // Inclusive .withFieldValue("end_timestamp_micros", 1001L) // Exclusive .build()); @@ -545,7 +545,7 @@ public void testAllMutations() { Row.withSchema(uberSchema) .withFieldValue("key", "row-delete-family".getBytes(StandardCharsets.UTF_8)) .withFieldValue("type", "DeleteFromFamily") - .withFieldValue("family_name", COLUMN_FAMILY_NAME_1.getBytes(StandardCharsets.UTF_8)) + .withFieldValue("family_name", COLUMN_FAMILY_NAME_1) .build()); // 5. DeleteFromRow diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java index dbe689c05759..70bf061e9412 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java @@ -316,10 +316,7 @@ private void flushBatch() throws DatastoreException, InterruptedException { // Break if the commit threw no exception. break; } catch (DatastoreException exception) { - LOG.error( - "Error writing to the Datastore ({}): {}", - exception.getCode(), - exception.getMessage()); + LOG.error("Error writing to the Datastore ({})", exception.getCode(), exception); if (!BackOffUtils.next(sleeper, backoff)) { LOG.error("Aborting after {} retries.", MAX_RETRIES); throw exception; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PreparePubsubWriteDoFnTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PreparePubsubWriteDoFnTest.java index a125a7b67e69..caae41aaab65 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PreparePubsubWriteDoFnTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PreparePubsubWriteDoFnTest.java @@ -34,32 +34,30 @@ @RunWith(JUnit4.class) public class PreparePubsubWriteDoFnTest implements Serializable { @Test - public void testValidatePubsubMessageSizeOnlyPayload() throws SizeLimitExceededException { + public void testValidatePubsubMessageOnlyPayload() throws SizeLimitExceededException { byte[] data = new byte[1024]; PubsubMessage message = new PubsubMessage(data, null); int messageSize = - PreparePubsubWriteDoFn.validatePubsubMessageSize(message, PUBSUB_MESSAGE_MAX_TOTAL_SIZE); + PreparePubsubWriteDoFn.validatePubsubMessage(message, PUBSUB_MESSAGE_MAX_TOTAL_SIZE); assertEquals(data.length, messageSize); } @Test - public void testValidatePubsubMessageSizePayloadAndOrderingKey() - throws SizeLimitExceededException { + public void testValidatePubsubMessagePayloadAndOrderingKey() throws SizeLimitExceededException { byte[] data = new byte[1024]; String orderingKey = "key"; PubsubMessage message = new PubsubMessage(data, null, null, orderingKey); int messageSize = - PreparePubsubWriteDoFn.validatePubsubMessageSize(message, PUBSUB_MESSAGE_MAX_TOTAL_SIZE); + PreparePubsubWriteDoFn.validatePubsubMessage(message, PUBSUB_MESSAGE_MAX_TOTAL_SIZE); assertEquals(data.length + orderingKey.getBytes(StandardCharsets.UTF_8).length, messageSize); } @Test - public void testValidatePubsubMessageSizePayloadAndAttributes() - throws SizeLimitExceededException { + public void testValidatePubsubMessagePayloadAndAttributes() throws SizeLimitExceededException { byte[] data = new byte[1024]; String attributeKey = "key"; String attributeValue = "value"; @@ -67,7 +65,7 @@ public void testValidatePubsubMessageSizePayloadAndAttributes() PubsubMessage message = new PubsubMessage(data, attributes); int messageSize = - PreparePubsubWriteDoFn.validatePubsubMessageSize(message, PUBSUB_MESSAGE_MAX_TOTAL_SIZE); + PreparePubsubWriteDoFn.validatePubsubMessage(message, PUBSUB_MESSAGE_MAX_TOTAL_SIZE); assertEquals( data.length @@ -78,32 +76,28 @@ public void testValidatePubsubMessageSizePayloadAndAttributes() } @Test - public void testValidatePubsubMessageSizePayloadTooLarge() { + public void testValidatePubsubMessagePayloadTooLarge() { byte[] data = new byte[(10 << 20) + 1]; PubsubMessage message = new PubsubMessage(data, null); assertThrows( SizeLimitExceededException.class, - () -> - PreparePubsubWriteDoFn.validatePubsubMessageSize( - message, PUBSUB_MESSAGE_MAX_TOTAL_SIZE)); + () -> PreparePubsubWriteDoFn.validatePubsubMessage(message, PUBSUB_MESSAGE_MAX_TOTAL_SIZE)); } @Test - public void testValidatePubsubMessageSizePayloadPlusOrderingKeyTooLarge() { + public void testValidatePubsubMessagePayloadPlusOrderingKeyTooLarge() { byte[] data = new byte[(10 << 20)]; String orderingKey = "key"; PubsubMessage message = new PubsubMessage(data, null, null, orderingKey); assertThrows( SizeLimitExceededException.class, - () -> - PreparePubsubWriteDoFn.validatePubsubMessageSize( - message, PUBSUB_MESSAGE_MAX_TOTAL_SIZE)); + () -> PreparePubsubWriteDoFn.validatePubsubMessage(message, PUBSUB_MESSAGE_MAX_TOTAL_SIZE)); } @Test - public void testValidatePubsubMessageSizePayloadPlusAttributesTooLarge() { + public void testValidatePubsubMessagePayloadPlusAttributesTooLarge() { byte[] data = new byte[(10 << 20)]; String attributeKey = "key"; String attributeValue = "value"; @@ -112,13 +106,11 @@ public void testValidatePubsubMessageSizePayloadPlusAttributesTooLarge() { assertThrows( SizeLimitExceededException.class, - () -> - PreparePubsubWriteDoFn.validatePubsubMessageSize( - message, PUBSUB_MESSAGE_MAX_TOTAL_SIZE)); + () -> PreparePubsubWriteDoFn.validatePubsubMessage(message, PUBSUB_MESSAGE_MAX_TOTAL_SIZE)); } @Test - public void testValidatePubsubMessageSizeAttributeKeyTooLarge() { + public void testValidatePubsubMessageAttributeKeyTooLarge() { byte[] data = new byte[1024]; String attributeKey = RandomStringUtils.randomAscii(257); String attributeValue = "value"; @@ -127,13 +119,11 @@ public void testValidatePubsubMessageSizeAttributeKeyTooLarge() { assertThrows( SizeLimitExceededException.class, - () -> - PreparePubsubWriteDoFn.validatePubsubMessageSize( - message, PUBSUB_MESSAGE_MAX_TOTAL_SIZE)); + () -> PreparePubsubWriteDoFn.validatePubsubMessage(message, PUBSUB_MESSAGE_MAX_TOTAL_SIZE)); } @Test - public void testValidatePubsubMessageSizeAttributeValueTooLarge() { + public void testValidatePubsubMessageAttributeValueTooLarge() { byte[] data = new byte[1024]; String attributeKey = "key"; String attributeValue = RandomStringUtils.randomAscii(1025); @@ -142,33 +132,45 @@ public void testValidatePubsubMessageSizeAttributeValueTooLarge() { assertThrows( SizeLimitExceededException.class, - () -> - PreparePubsubWriteDoFn.validatePubsubMessageSize( - message, PUBSUB_MESSAGE_MAX_TOTAL_SIZE)); + () -> PreparePubsubWriteDoFn.validatePubsubMessage(message, PUBSUB_MESSAGE_MAX_TOTAL_SIZE)); } @Test - public void testValidatePubsubMessageSizeOrderingKeyTooLarge() { + public void testValidatePubsubMessageOrderingKeyTooLarge() { byte[] data = new byte[1024]; String orderingKey = RandomStringUtils.randomAscii(1025); PubsubMessage message = new PubsubMessage(data, null, null, orderingKey); assertThrows( SizeLimitExceededException.class, - () -> - PreparePubsubWriteDoFn.validatePubsubMessageSize( - message, PUBSUB_MESSAGE_MAX_TOTAL_SIZE)); + () -> PreparePubsubWriteDoFn.validatePubsubMessage(message, PUBSUB_MESSAGE_MAX_TOTAL_SIZE)); } @Test - public void testValidatePubsubMessagePayloadTooLarge() { - byte[] data = new byte[(10 << 20) + 1]; + public void testValidatePubsubMessageEmptyMessageRejectedNullMap() { + byte[] data = new byte[0]; PubsubMessage message = new PubsubMessage(data, null); + assertThrows( + "non-empty payload or at least one attribute", + IllegalArgumentException.class, + () -> PreparePubsubWriteDoFn.validatePubsubMessage(message, PUBSUB_MESSAGE_MAX_TOTAL_SIZE)); + } + @Test + public void testValidatePubsubMessageEmptyMessageRejectedEmptyMap() { + byte[] data = new byte[0]; + PubsubMessage message = new PubsubMessage(data, ImmutableMap.of()); assertThrows( - SizeLimitExceededException.class, - () -> - PreparePubsubWriteDoFn.validatePubsubMessageSize( - message, PUBSUB_MESSAGE_MAX_TOTAL_SIZE)); + "non-empty payload or at least one attribute", + IllegalArgumentException.class, + () -> PreparePubsubWriteDoFn.validatePubsubMessage(message, PUBSUB_MESSAGE_MAX_TOTAL_SIZE)); + } + + @Test + public void testValidatePubsubMessageEmptyDataButAttributesAllowed() + throws SizeLimitExceededException { + byte[] data = new byte[0]; + PubsubMessage message = new PubsubMessage(data, ImmutableMap.of("key", "value")); + PreparePubsubWriteDoFn.validatePubsubMessage(message, PUBSUB_MESSAGE_MAX_TOTAL_SIZE); } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java index bec157ae83cc..3d9c65aa1376 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java @@ -125,15 +125,10 @@ public void testTopicValidationSuccess() throws Exception { PubsubIO.readStrings().fromTopic("projects/my-project/topics/AbC-1234-_.~%+-_.~%+-_.~%+-abc"); PubsubIO.readStrings() .fromTopic( - new StringBuilder() - .append("projects/my-project/topics/A-really-long-one-") - .append( - "111111111111111111111111111111111111111111111111111111111111111111111111111111111") - .append( - "111111111111111111111111111111111111111111111111111111111111111111111111111111111") - .append( - "11111111111111111111111111111111111111111111111111111111111111111111111111") - .toString()); + "projects/my-project/topics/A-really-long-one-" + + "111111111111111111111111111111111111111111111111111111111111111111111111111111111" + + "111111111111111111111111111111111111111111111111111111111111111111111111111111111" + + "11111111111111111111111111111111111111111111111111111111111111111111111111"); } @Test @@ -147,15 +142,10 @@ public void testTopicValidationTooLong() throws Exception { thrown.expect(IllegalArgumentException.class); PubsubIO.readStrings() .fromTopic( - new StringBuilder() - .append("projects/my-project/topics/A-really-long-one-") - .append( - "111111111111111111111111111111111111111111111111111111111111111111111111111111111") - .append( - "111111111111111111111111111111111111111111111111111111111111111111111111111111111") - .append( - "1111111111111111111111111111111111111111111111111111111111111111111111111111") - .toString()); + "projects/my-project/topics/A-really-long-one-" + + "111111111111111111111111111111111111111111111111111111111111111111111111111111111" + + "111111111111111111111111111111111111111111111111111111111111111111111111111111111" + + "1111111111111111111111111111111111111111111111111111111111111111111111111111"); } @Test @@ -1008,10 +998,8 @@ public void testWriteTopicValidationSuccess() throws Exception { PubsubIO.writeStrings().to("projects/my-project/topics/AbC-1234-_.~%+-_.~%+-_.~%+-abc"); PubsubIO.writeStrings() .to( - new StringBuilder() - .append("projects/my-project/topics/A-really-long-one-") - .append(RandomStringUtils.randomAlphanumeric(100)) - .toString()); + "projects/my-project/topics/A-really-long-one-" + + RandomStringUtils.randomAlphanumeric(100)); } @Test @@ -1025,10 +1013,8 @@ public void testWriteValidationTooLong() throws Exception { thrown.expect(IllegalArgumentException.class); PubsubIO.writeStrings() .to( - new StringBuilder() - .append("projects/my-project/topics/A-really-long-one-") - .append(RandomStringUtils.randomAlphanumeric(1000)) - .toString()); + "projects/my-project/topics/A-really-long-one-" + + RandomStringUtils.randomAlphanumeric(1000)); } @Test diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOReadTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOReadTest.java index d82c50fd79a3..7abc6f9573a7 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOReadTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOReadTest.java @@ -827,7 +827,7 @@ private void checkMessage(String substring, @Nullable String message) { } } - private long getRequestMetricCount(HashMap baseLabels) { + private long getRequestMetricCount(Map baseLabels) { MonitoringInfoMetricName name = MonitoringInfoMetricName.named(MonitoringInfoConstants.Urns.API_REQUEST_COUNT, baseLabels); MetricsContainerImpl container = diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerReadIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerReadIT.java index 38fc1887a887..d0164717e158 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerReadIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerReadIT.java @@ -20,7 +20,6 @@ import static org.junit.Assert.assertEquals; import com.google.api.gax.longrunning.OperationFuture; -import com.google.cloud.spanner.BatchClient; import com.google.cloud.spanner.Database; import com.google.cloud.spanner.DatabaseAdminClient; import com.google.cloud.spanner.DatabaseClient; @@ -45,12 +44,8 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.TestPipelineOptions; import org.apache.beam.sdk.transforms.Count; -import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.MapElements; -import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.transforms.SimpleFunction; -import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TypeDescriptor; @@ -70,7 +65,6 @@ public class SpannerReadIT { private static final int MAX_DB_NAME_LENGTH = 30; - private static final int CLEANUP_PROPAGATION_DELAY_MS = 5000; @Rule public final transient TestPipeline p = TestPipeline.create(); @Rule public transient ExpectedException thrown = ExpectedException.none(); @@ -275,55 +269,6 @@ public void testReadFailsBadTable() throws Exception { p.run().waitUntilFinish(); } - private static class CloseTransactionFn extends SimpleFunction { - private final SpannerConfig spannerConfig; - - private CloseTransactionFn(SpannerConfig spannerConfig) { - this.spannerConfig = spannerConfig; - } - - @Override - public Transaction apply(Transaction tx) { - BatchClient batchClient = SpannerAccessor.getOrCreate(spannerConfig).getBatchClient(); - batchClient.batchReadOnlyTransaction(tx.transactionId()).cleanup(); - try { - // Wait for cleanup to propagate. - Thread.sleep(CLEANUP_PROPAGATION_DELAY_MS); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - } - return tx; - } - } - - @Test - public void testReadFailsBadSession() throws Exception { - - thrown.expect(new SpannerWriteIT.StackTraceContainsString("SpannerException")); - thrown.expect(new SpannerWriteIT.StackTraceContainsString("NOT_FOUND: Session not found")); - - SpannerConfig spannerConfig = createSpannerConfig(); - - // This creates a transaction then closes the session. - // The (closed) transaction is then passed to SpannerIO.read() and should - // raise SessionNotFound errors. - PCollectionView tx = - p.apply("Transaction seed", Create.of(1)) - .apply( - "Create transaction", - ParDo.of(new CreateTransactionFn(spannerConfig, TimestampBound.strong()))) - .apply("Close Transaction", MapElements.via(new CloseTransactionFn(spannerConfig))) - .apply("As PCollectionView", View.asSingleton()); - p.apply( - "read db", - SpannerIO.read() - .withSpannerConfig(spannerConfig) - .withTable(options.getTable()) - .withColumns("Key", "Value") - .withTransaction(tx)); - p.run().waitUntilFinish(); - } - @Test public void testQuery() throws Exception { SpannerConfig spannerConfig = createSpannerConfig(); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerTransformRegistrarTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerTransformRegistrarTest.java index 3b38e7e528a3..666cda91f731 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerTransformRegistrarTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerTransformRegistrarTest.java @@ -26,6 +26,7 @@ import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.stream.Stream; +import org.apache.beam.sdk.io.gcp.spanner.SpannerTransformRegistrar.ChangeStreamReaderBuilder; import org.apache.beam.sdk.io.gcp.spanner.SpannerTransformRegistrar.InsertBuilder; import org.apache.beam.sdk.io.gcp.spanner.SpannerTransformRegistrar.ReadBuilder; import org.apache.beam.sdk.schemas.Schema; @@ -48,22 +49,29 @@ public class SpannerTransformRegistrarTest { public static final String SPANNER_PROJECT = "spanner-project"; public static final String SPANNER_TABLE = "spanner-table"; public static final String SPANNER_SQL_QUERY = "SELECT * from spanner_table;"; + public static final String SPANNER_CHANGE_STREAM_NAME = "spanner-change-stream-name"; + public static final String SPANNER_CHANGE_STREAM_METADATA_INSTANCE = + "spanner-change-stream-instance"; + public static final String SPANNER_CHANGE_STREAM_METADATA_DATABASE = + "spanner-change-stream-database"; private SpannerTransformRegistrar spannerTransformRegistrar; private ReadBuilder readBuilder; private InsertBuilder writeBuilder; + private ChangeStreamReaderBuilder changeStreamReaderBuilder; @Before public void setup() { spannerTransformRegistrar = new SpannerTransformRegistrar(); readBuilder = new ReadBuilder(); writeBuilder = new InsertBuilder(); + changeStreamReaderBuilder = new ChangeStreamReaderBuilder(); } @Test public void testKnownBuilderInstances() { Map> builderInstancesMap = spannerTransformRegistrar.knownBuilderInstances(); - assertEquals(6, builderInstancesMap.size()); + assertEquals(7, builderInstancesMap.size()); assertThat(builderInstancesMap, IsMapContaining.hasKey(SpannerTransformRegistrar.INSERT_URN)); assertThat(builderInstancesMap, IsMapContaining.hasKey(SpannerTransformRegistrar.UPDATE_URN)); assertThat(builderInstancesMap, IsMapContaining.hasKey(SpannerTransformRegistrar.REPLACE_URN)); @@ -72,6 +80,9 @@ public void testKnownBuilderInstances() { IsMapContaining.hasKey(SpannerTransformRegistrar.INSERT_OR_UPDATE_URN)); assertThat(builderInstancesMap, IsMapContaining.hasKey(SpannerTransformRegistrar.DELETE_URN)); assertThat(builderInstancesMap, IsMapContaining.hasKey(SpannerTransformRegistrar.READ_URN)); + assertThat( + builderInstancesMap, + IsMapContaining.hasKey(SpannerTransformRegistrar.READ_CHANGE_STREAM_URN)); } @Test(expected = IllegalArgumentException.class) @@ -207,4 +218,136 @@ private InsertBuilder.Configuration getBasicWriteConfiguration() { configuration.setMaxCumulativeBackoff(100L); return configuration; } + + @Test(expected = IllegalArgumentException.class) + public void testChangeStreamReaderBuilderBuildExternalWithMissingMandatoryFields() { + changeStreamReaderBuilder.buildExternal(new ChangeStreamReaderBuilder.Configuration()); + } + + @Test(expected = IllegalArgumentException.class) + public void testChangeStreamReaderBuilderBuildExternalWithMissingDatabaseId() { + ChangeStreamReaderBuilder.Configuration configuration = + new ChangeStreamReaderBuilder.Configuration(); + configuration.setProjectId(SPANNER_PROJECT); + configuration.setInstanceId(SPANNER_INSTANCE); + configuration.setChangeStreamName(SPANNER_CHANGE_STREAM_NAME); + configuration.setMetadataInstance(SPANNER_CHANGE_STREAM_METADATA_INSTANCE); + configuration.setMetadataDatabase(SPANNER_CHANGE_STREAM_METADATA_DATABASE); + changeStreamReaderBuilder.buildExternal(configuration); + } + + @Test(expected = IllegalArgumentException.class) + public void testChangeStreamReaderBuilderBuildExternalWithMissingInstanceId() { + ChangeStreamReaderBuilder.Configuration configuration = + new ChangeStreamReaderBuilder.Configuration(); + configuration.setProjectId(SPANNER_PROJECT); + configuration.setDatabaseId(SPANNER_DATABASE); + configuration.setChangeStreamName(SPANNER_CHANGE_STREAM_NAME); + configuration.setMetadataInstance(SPANNER_CHANGE_STREAM_METADATA_INSTANCE); + configuration.setMetadataDatabase(SPANNER_CHANGE_STREAM_METADATA_DATABASE); + changeStreamReaderBuilder.buildExternal(configuration); + } + + @Test(expected = IllegalArgumentException.class) + public void testChangeStreamReaderBuilderBuildExternalWithMissingChangeStreamName() { + ChangeStreamReaderBuilder.Configuration configuration = + new ChangeStreamReaderBuilder.Configuration(); + configuration.setProjectId(SPANNER_PROJECT); + configuration.setDatabaseId(SPANNER_DATABASE); + configuration.setInstanceId(SPANNER_INSTANCE); + configuration.setMetadataInstance(SPANNER_CHANGE_STREAM_METADATA_INSTANCE); + configuration.setMetadataDatabase(SPANNER_CHANGE_STREAM_METADATA_DATABASE); + changeStreamReaderBuilder.buildExternal(configuration); + } + + @Test(expected = IllegalArgumentException.class) + public void testChangeStreamReaderBuilderBuildExternalWithMissingMetadataInstance() { + ChangeStreamReaderBuilder.Configuration configuration = + new ChangeStreamReaderBuilder.Configuration(); + configuration.setProjectId(SPANNER_PROJECT); + configuration.setDatabaseId(SPANNER_DATABASE); + configuration.setInstanceId(SPANNER_INSTANCE); + configuration.setChangeStreamName(SPANNER_CHANGE_STREAM_NAME); + configuration.setMetadataDatabase(SPANNER_CHANGE_STREAM_METADATA_DATABASE); + changeStreamReaderBuilder.buildExternal(configuration); + } + + @Test(expected = IllegalArgumentException.class) + public void testChangeStreamReaderBuilderBuildExternalWithMissingMetadataDatabase() { + ChangeStreamReaderBuilder.Configuration configuration = + new ChangeStreamReaderBuilder.Configuration(); + configuration.setProjectId(SPANNER_PROJECT); + configuration.setDatabaseId(SPANNER_DATABASE); + configuration.setInstanceId(SPANNER_INSTANCE); + configuration.setChangeStreamName(SPANNER_CHANGE_STREAM_NAME); + configuration.setMetadataInstance(SPANNER_CHANGE_STREAM_METADATA_INSTANCE); + changeStreamReaderBuilder.buildExternal(configuration); + } + + @Test + public void testChangeStreamReaderBuilderBuildExternalWithRequiredFields() { + ChangeStreamReaderBuilder.Configuration configuration = + new ChangeStreamReaderBuilder.Configuration(); + + configuration.setProjectId(SPANNER_PROJECT); + configuration.setDatabaseId(SPANNER_DATABASE); + configuration.setInstanceId(SPANNER_INSTANCE); + configuration.setChangeStreamName(SPANNER_CHANGE_STREAM_NAME); + configuration.setMetadataInstance(SPANNER_CHANGE_STREAM_METADATA_INSTANCE); + configuration.setMetadataDatabase(SPANNER_CHANGE_STREAM_METADATA_DATABASE); + + PTransform> changeStreamReaderTransform = + changeStreamReaderBuilder.buildExternal(configuration); + assertNotNull(changeStreamReaderTransform); + } + + @Test + public void testChangeStreamReaderBuilderBuildExternalWithAllFields() { + String startAt = "2023-01-01T00:00:00Z"; + String endAt = "2023-01-02T00:00:00Z"; + String metadataTable = "meta-table"; + String rpcPriority = "HIGH"; + String refreshRate = "PT30S"; + + ChangeStreamReaderBuilder.Configuration configuration = + new ChangeStreamReaderBuilder.Configuration(); + + configuration.setProjectId(SPANNER_PROJECT); + configuration.setDatabaseId(SPANNER_DATABASE); + configuration.setInstanceId(SPANNER_INSTANCE); + configuration.setChangeStreamName(SPANNER_CHANGE_STREAM_NAME); + configuration.setMetadataInstance(SPANNER_CHANGE_STREAM_METADATA_INSTANCE); + configuration.setMetadataDatabase(SPANNER_CHANGE_STREAM_METADATA_DATABASE); + configuration.setInclusiveStartAt(startAt); + configuration.setInclusiveEndAt(endAt); + configuration.setMetadataTable(metadataTable); + configuration.setRpcPriority(rpcPriority); + configuration.setWatermarkRefreshRate(refreshRate); + + PTransform> changeStreamReaderTransform = + changeStreamReaderBuilder.buildExternal(configuration); + assertNotNull(changeStreamReaderTransform); + } + + @Test + public void testChangeStreamReaderBuilderBuildExternalWithNullOptionalValues() { + ChangeStreamReaderBuilder.Configuration configuration = + new ChangeStreamReaderBuilder.Configuration(); + + configuration.setProjectId(SPANNER_PROJECT); + configuration.setDatabaseId(SPANNER_DATABASE); + configuration.setInstanceId(SPANNER_INSTANCE); + configuration.setChangeStreamName(SPANNER_CHANGE_STREAM_NAME); + configuration.setMetadataInstance(SPANNER_CHANGE_STREAM_METADATA_INSTANCE); + configuration.setMetadataDatabase(SPANNER_CHANGE_STREAM_METADATA_DATABASE); + configuration.setInclusiveStartAt(null); + configuration.setInclusiveEndAt(null); + configuration.setMetadataTable(null); + configuration.setRpcPriority(null); + configuration.setWatermarkRefreshRate(null); + + PTransform> changeStreamReaderTransform = + changeStreamReaderBuilder.buildExternal(configuration); + assertNotNull(changeStreamReaderTransform); + } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/IntegrationTestEnv.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/IntegrationTestEnv.java index 6d3f12ac8e53..dc10e68c1e9d 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/IntegrationTestEnv.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/IntegrationTestEnv.java @@ -281,7 +281,6 @@ void createRoleAndGrantPrivileges(String table, String changeStream) + DATABASE_ROLE), null) .get(TIMEOUT_MINUTES, TimeUnit.MINUTES); - return; } String getProjectId() { diff --git a/sdks/java/io/hbase/build.gradle b/sdks/java/io/hbase/build.gradle index 07014f2d5e3b..a361a593b4fe 100644 --- a/sdks/java/io/hbase/build.gradle +++ b/sdks/java/io/hbase/build.gradle @@ -34,7 +34,7 @@ test { jvmArgs "-Dtest.build.data.basedirectory=build/test-data" } -def hbase_version = "2.6.1-hadoop3" +def hbase_version = "2.6.3-hadoop3" dependencies { implementation library.java.vendored_guava_32_1_2_jre diff --git a/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseRowMutationsCoder.java b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseRowMutationsCoder.java index c7ecad045a96..6d66cee21109 100644 --- a/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseRowMutationsCoder.java +++ b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseRowMutationsCoder.java @@ -95,9 +95,7 @@ public List> getCoderArguments() { * @throws @UnknownKeyFor@NonNull@Initialized NonDeterministicException */ @Override - public void verifyDeterministic() { - return; - } + public void verifyDeterministic() {} private static MutationType getType(Mutation mutation) { if (mutation instanceof Put) { diff --git a/sdks/java/io/iceberg/build.gradle b/sdks/java/io/iceberg/build.gradle index ba1d27b0e3e6..33a0203d46b2 100644 --- a/sdks/java/io/iceberg/build.gradle +++ b/sdks/java/io/iceberg/build.gradle @@ -23,6 +23,8 @@ import java.util.stream.Collectors plugins { id 'org.apache.beam.module' } applyJavaNature( automaticModuleName: 'org.apache.beam.sdk.io.iceberg', + // iceberg ended support for Java 8 in 1.7.0 + requireJavaVersion: JavaVersion.VERSION_11, ) description = "Apache Beam :: SDKs :: Java :: IO :: Iceberg" @@ -37,12 +39,9 @@ def hadoopVersions = [ hadoopVersions.each {kv -> configurations.create("hadoopVersion$kv.key")} -// we cannot upgrade this since the newer iceberg requires Java 11 -// many other modules like examples/expansion use Java 8 and have the iceberg dependency -// def iceberg_version = "1.9.0" -def iceberg_version = "1.6.1" +def iceberg_version = "1.9.2" def parquet_version = "1.15.2" -def orc_version = "1.9.2" +def orc_version = "1.9.6" def hive_version = "3.1.3" dependencies { @@ -107,6 +106,11 @@ dependencies { } } +configurations.all { + // iceberg-core needs avro:1.12.0 + resolutionStrategy.force 'org.apache.avro:avro:1.12.0' +} + hadoopVersions.each {kv -> configurations."hadoopVersion$kv.key" { resolutionStrategy { diff --git a/sdks/java/io/iceberg/hive/build.gradle b/sdks/java/io/iceberg/hive/build.gradle index 480707b128b1..723036fb1183 100644 --- a/sdks/java/io/iceberg/hive/build.gradle +++ b/sdks/java/io/iceberg/hive/build.gradle @@ -26,7 +26,7 @@ description = "Apache Beam :: SDKs :: Java :: IO :: Iceberg :: Hive" ext.summary = "Runtime dependencies needed for Hive catalog integration." def hive_version = "3.1.3" -def hbase_version = "2.6.1-hadoop3" +def hbase_version = "2.6.3-hadoop3" def hadoop_version = "3.4.1" def iceberg_version = "1.6.1" def avatica_version = "1.25.0" diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/RecordWriterManagerTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/RecordWriterManagerTest.java index b240442deb6d..36b74967f0b2 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/RecordWriterManagerTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/RecordWriterManagerTest.java @@ -34,8 +34,8 @@ import java.nio.ByteBuffer; import java.time.LocalDate; import java.time.LocalDateTime; +import java.time.LocalTime; import java.util.ArrayList; -import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -59,11 +59,15 @@ import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.transforms.Transform; import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.DateTimeUtil; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; +import org.joda.time.ReadableDateTime; import org.junit.Before; import org.junit.ClassRule; import org.junit.Rule; @@ -532,13 +536,35 @@ public void testIdentityPartitioning() throws IOException { assertEquals(1, dataFile.getRecordCount()); // build this string: bool=true/int=1/long=1/float=1.0/double=1.0/str=str List expectedPartitions = new ArrayList<>(); - List dateTypes = Arrays.asList("date", "time", "datetime", "datetime_tz"); - for (Schema.Field field : primitiveTypeSchema.getFields()) { - Object val = checkStateNotNull(row.getValue(field.getName())); - if (dateTypes.contains(field.getName())) { - val = URLEncoder.encode(val.toString(), UTF_8.toString()); + + for (PartitionField field : spec.fields()) { + String name = field.name(); + Type type = spec.schema().findType(name); + Transform transform = (Transform) field.transform(); + String val; + switch (name) { + case "date": + LocalDate localDate = checkStateNotNull(row.getValue(name)); + Integer day = Integer.parseInt(String.valueOf(localDate.toEpochDay())); + val = transform.toHumanString(type, day); + break; + case "time": + LocalTime localTime = checkStateNotNull(row.getValue(name)); + val = transform.toHumanString(type, localTime.toNanoOfDay() / 1000); + break; + case "datetime": + LocalDateTime ldt = checkStateNotNull(row.getValue(name)); + val = transform.toHumanString(type, DateTimeUtil.microsFromTimestamp(ldt)); + break; + case "datetime_tz": + ReadableDateTime dt = checkStateNotNull(row.getDateTime(name)); + val = transform.toHumanString(type, dt.getMillis() * 1000); + break; + default: + val = transform.toHumanString(type, checkStateNotNull(row.getValue(name))); + break; } - expectedPartitions.add(field.getName() + "=" + val); + expectedPartitions.add(name + "=" + URLEncoder.encode(val, UTF_8.toString())); } String expectedPartitionPath = String.join("/", expectedPartitions); assertEquals(expectedPartitionPath, dataFile.getPartitionPath()); diff --git a/sdks/java/io/jdbc/build.gradle b/sdks/java/io/jdbc/build.gradle index 8c5fa685fdad..87a231a5a42b 100644 --- a/sdks/java/io/jdbc/build.gradle +++ b/sdks/java/io/jdbc/build.gradle @@ -29,6 +29,7 @@ ext.summary = "IO to read and write on JDBC datasource." dependencies { implementation library.java.vendored_guava_32_1_2_jre implementation project(path: ":sdks:java:core", configuration: "shadow") + implementation project(path: ":model:pipeline", configuration: "shadow") implementation library.java.dbcp2 implementation library.java.joda_time implementation "org.apache.commons:commons-pool2:2.11.1" @@ -39,8 +40,10 @@ dependencies { testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") testImplementation project(path: ":sdks:java:extensions:avro", configuration: "testRuntimeMigration") testImplementation project(path: ":sdks:java:io:common") + testImplementation project(path: ":sdks:java:managed") testImplementation project(path: ":sdks:java:testing:test-utils") testImplementation library.java.junit + testImplementation library.java.mockito_inline testImplementation library.java.slf4j_api testImplementation library.java.postgres diff --git a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java index f075d5b7b6cc..e6db4d82712b 100644 --- a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java +++ b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java @@ -355,6 +355,10 @@ public static ReadRows readRows() { * Like {@link #read}, but executes multiple instances of the query substituting each element of a * {@link PCollection} as query parameters. * + *

The substitution is configured via {@link ReadAll#withParameterSetter}. Substitutions + * allowed by the JDBC API's {@link PreparedStatement} are supported. In particular, this does not + * support parameterizing the table name to read from a different table for each input element. + * * @param Type of the data representing query parameters. * @param Type of the data to be read. */ @@ -1175,6 +1179,18 @@ public ReadAll withQuery(ValueProvider query) { return toBuilder().setQuery(query).build(); } + /** + * Sets the {@link PreparedStatementSetter} to set the parameters of the query for each input + * element. + * + *

For example, + * + *

{@code
+     * JdbcIO.readAll()
+     *     .withQuery("select * from table where field = ?")
+     *     .withParameterSetter((element, preparedStatement) -> preparedStatement.setString(1, element))
+     * }
+ */ public ReadAll withParameterSetter( PreparedStatementSetter parameterSetter) { checkArgumentNotNull( @@ -1709,6 +1725,15 @@ private Connection getConnection() throws SQLException { try { connection = validSource.getConnection(); this.connection = connection; + + // PostgreSQL requires autocommit to be disabled to enable cursor streaming + // see https://jdbc.postgresql.org/documentation/head/query.html#query-with-cursor + // This option is configurable as Informix will error + // if calling setAutoCommit on a non-logged database + if (disableAutoCommit) { + LOG.info("Autocommit has been disabled"); + connection.setAutoCommit(false); + } } finally { connectionLock.unlock(); } @@ -1739,14 +1764,6 @@ private Connection getConnection() throws SQLException { public void processElement(ProcessContext context) throws Exception { // Only acquire the connection if we need to perform a read. Connection connection = getConnection(); - // PostgreSQL requires autocommit to be disabled to enable cursor streaming - // see https://jdbc.postgresql.org/documentation/head/query.html#query-with-cursor - // This option is configurable as Informix will error - // if calling setAutoCommit on a non-logged database - if (disableAutoCommit) { - LOG.info("Autocommit has been disabled"); - connection.setAutoCommit(false); - } try (PreparedStatement statement = connection.prepareStatement( query.get(), ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY)) { @@ -1829,8 +1846,8 @@ public static RetryConfiguration create( } /** - * An interface used by the JdbcIO Write to set the parameters of the {@link PreparedStatement} - * used to setParameters into the database. + * An interface used by the JdbcIO {@link ReadAll} and {@link Write} to set the parameters of the + * {@link PreparedStatement} used to setParameters into the database. */ @FunctionalInterface public interface PreparedStatementSetter extends Serializable { @@ -2872,6 +2889,8 @@ private void cleanUpStatementAndConnection() throws Exception { } } + @SuppressWarnings( + "Slf4jDoNotLogMessageOfExceptionExplicitly") // for tests checking error message private void executeBatch(ProcessContext context, Iterable records) throws SQLException, InterruptedException { Long startTimeNs = System.nanoTime(); diff --git a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcReadSchemaTransformProvider.java b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcReadSchemaTransformProvider.java index 6777be50ab50..da75c9baaa45 100644 --- a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcReadSchemaTransformProvider.java +++ b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcReadSchemaTransformProvider.java @@ -27,6 +27,8 @@ import java.util.Objects; import javax.annotation.Nullable; import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.NoSuchSchemaException; +import org.apache.beam.sdk.schemas.SchemaRegistry; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription; import org.apache.beam.sdk.schemas.transforms.SchemaTransform; @@ -265,6 +267,20 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { } return PCollectionRowTuple.of("output", input.getPipeline().apply(readRows)); } + + public Row getConfigurationRow() { + try { + // To stay consistent with our SchemaTransform configuration naming conventions, + // we sort lexicographically + return SchemaRegistry.createDefault() + .getToRowFunction(JdbcReadSchemaTransformConfiguration.class) + .apply(config) + .sorted() + .toSnakeCase(); + } catch (NoSuchSchemaException e) { + throw new RuntimeException(e); + } + } } @Override @@ -401,6 +417,8 @@ public static Builder builder() { .Builder(); } + public abstract Builder toBuilder(); + @AutoValue.Builder public abstract static class Builder { public abstract Builder setDriverClassName(String value); diff --git a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcWriteSchemaTransformProvider.java b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcWriteSchemaTransformProvider.java index 6f10df56aab5..4dbb9b396f09 100644 --- a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcWriteSchemaTransformProvider.java +++ b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcWriteSchemaTransformProvider.java @@ -27,7 +27,9 @@ import java.util.Objects; import javax.annotation.Nullable; import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.NoSuchSchemaException; import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.SchemaRegistry; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription; import org.apache.beam.sdk.schemas.transforms.SchemaTransform; @@ -265,6 +267,20 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { .setRowSchema(Schema.of()); return PCollectionRowTuple.of("post_write", postWrite); } + + public Row getConfigurationRow() { + try { + // To stay consistent with our SchemaTransform configuration naming conventions, + // we sort lexicographically + return SchemaRegistry.createDefault() + .getToRowFunction(JdbcWriteSchemaTransformConfiguration.class) + .apply(config) + .sorted() + .toSnakeCase(); + } catch (NoSuchSchemaException e) { + throw new RuntimeException(e); + } + } } @Override @@ -382,6 +398,8 @@ public static Builder builder() { .Builder(); } + public abstract Builder toBuilder(); + @AutoValue.Builder public abstract static class Builder { public abstract Builder setDriverClassName(String value); diff --git a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/providers/PostgresSchemaTransformTranslation.java b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/providers/PostgresSchemaTransformTranslation.java new file mode 100644 index 000000000000..288b29642c5a --- /dev/null +++ b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/providers/PostgresSchemaTransformTranslation.java @@ -0,0 +1,93 @@ +/* + * 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. + */ +package org.apache.beam.sdk.io.jdbc.providers; + +import static org.apache.beam.sdk.io.jdbc.providers.ReadFromPostgresSchemaTransformProvider.PostgresReadSchemaTransform; +import static org.apache.beam.sdk.io.jdbc.providers.WriteToPostgresSchemaTransformProvider.PostgresWriteSchemaTransform; +import static org.apache.beam.sdk.schemas.transforms.SchemaTransformTranslation.SchemaTransformPayloadTranslator; + +import com.google.auto.service.AutoService; +import java.util.Map; +import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.util.construction.TransformPayloadTranslatorRegistrar; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; + +public class PostgresSchemaTransformTranslation { + static class PostgresReadSchemaTransformTranslator + extends SchemaTransformPayloadTranslator { + @Override + public SchemaTransformProvider provider() { + return new ReadFromPostgresSchemaTransformProvider(); + } + + @Override + public Row toConfigRow(PostgresReadSchemaTransform transform) { + return transform.getConfigurationRow(); + } + } + + @AutoService(TransformPayloadTranslatorRegistrar.class) + public static class ReadRegistrar implements TransformPayloadTranslatorRegistrar { + @Override + @SuppressWarnings({ + "rawtypes", + }) + public Map< + ? extends Class, + ? extends PTransformTranslation.TransformPayloadTranslator> + getTransformPayloadTranslators() { + return ImmutableMap + ., PTransformTranslation.TransformPayloadTranslator>builder() + .put(PostgresReadSchemaTransform.class, new PostgresReadSchemaTransformTranslator()) + .build(); + } + } + + static class PostgresWriteSchemaTransformTranslator + extends SchemaTransformPayloadTranslator { + @Override + public SchemaTransformProvider provider() { + return new WriteToPostgresSchemaTransformProvider(); + } + + @Override + public Row toConfigRow(PostgresWriteSchemaTransform transform) { + return transform.getConfigurationRow(); + } + } + + @AutoService(TransformPayloadTranslatorRegistrar.class) + public static class WriteRegistrar implements TransformPayloadTranslatorRegistrar { + @Override + @SuppressWarnings({ + "rawtypes", + }) + public Map< + ? extends Class, + ? extends PTransformTranslation.TransformPayloadTranslator> + getTransformPayloadTranslators() { + return ImmutableMap + ., PTransformTranslation.TransformPayloadTranslator>builder() + .put(PostgresWriteSchemaTransform.class, new PostgresWriteSchemaTransformTranslator()) + .build(); + } + } +} diff --git a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/providers/ReadFromPostgresSchemaTransformProvider.java b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/providers/ReadFromPostgresSchemaTransformProvider.java index 62ff14c23e0a..834e7a0a4927 100644 --- a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/providers/ReadFromPostgresSchemaTransformProvider.java +++ b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/providers/ReadFromPostgresSchemaTransformProvider.java @@ -18,20 +18,30 @@ package org.apache.beam.sdk.io.jdbc.providers; import static org.apache.beam.sdk.io.jdbc.JdbcUtil.POSTGRES; +import static org.apache.beam.sdk.util.construction.BeamUrns.getUrn; import com.google.auto.service.AutoService; +import java.util.Collections; +import java.util.List; +import org.apache.beam.model.pipeline.v1.ExternalTransforms; import org.apache.beam.sdk.io.jdbc.JdbcReadSchemaTransformProvider; +import org.apache.beam.sdk.schemas.transforms.SchemaTransform; import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; import org.checkerframework.checker.initialization.qual.Initialized; import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.UnknownKeyFor; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; @AutoService(SchemaTransformProvider.class) public class ReadFromPostgresSchemaTransformProvider extends JdbcReadSchemaTransformProvider { + private static final Logger LOG = + LoggerFactory.getLogger(ReadFromPostgresSchemaTransformProvider.class); + @Override public @UnknownKeyFor @NonNull @Initialized String identifier() { - return "beam:schematransform:org.apache.beam:postgres_read:v1"; + return getUrn(ExternalTransforms.ManagedTransforms.Urns.POSTGRES_READ); } @Override @@ -43,4 +53,40 @@ public String description() { protected String jdbcType() { return POSTGRES; } + + @Override + public @UnknownKeyFor @NonNull @Initialized SchemaTransform from( + JdbcReadSchemaTransformConfiguration configuration) { + String jdbcType = configuration.getJdbcType(); + if (jdbcType != null && !jdbcType.isEmpty() && !jdbcType.equals(jdbcType())) { + throw new IllegalArgumentException( + String.format("Wrong JDBC type. Expected '%s' but got '%s'", jdbcType(), jdbcType)); + } + + List<@org.checkerframework.checker.nullness.qual.Nullable String> connectionInitSql = + configuration.getConnectionInitSql(); + if (connectionInitSql != null && !connectionInitSql.isEmpty()) { + LOG.warn("Postgres does not support connectionInitSql, ignoring."); + } + + Boolean disableAutoCommit = configuration.getDisableAutoCommit(); + if (disableAutoCommit != null && !disableAutoCommit) { + LOG.warn("Postgres reads require disableAutoCommit to be true, overriding to true."); + } + + // Override "connectionInitSql" and "disableAutoCommit" for postgres + configuration = + configuration + .toBuilder() + .setConnectionInitSql(Collections.emptyList()) + .setDisableAutoCommit(true) + .build(); + return new PostgresReadSchemaTransform(configuration); + } + + public static class PostgresReadSchemaTransform extends JdbcReadSchemaTransform { + public PostgresReadSchemaTransform(JdbcReadSchemaTransformConfiguration config) { + super(config, POSTGRES); + } + } } diff --git a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/providers/WriteToPostgresSchemaTransformProvider.java b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/providers/WriteToPostgresSchemaTransformProvider.java index c50b84311630..97074742dbed 100644 --- a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/providers/WriteToPostgresSchemaTransformProvider.java +++ b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/providers/WriteToPostgresSchemaTransformProvider.java @@ -18,20 +18,30 @@ package org.apache.beam.sdk.io.jdbc.providers; import static org.apache.beam.sdk.io.jdbc.JdbcUtil.POSTGRES; +import static org.apache.beam.sdk.util.construction.BeamUrns.getUrn; import com.google.auto.service.AutoService; +import java.util.Collections; +import java.util.List; +import org.apache.beam.model.pipeline.v1.ExternalTransforms; import org.apache.beam.sdk.io.jdbc.JdbcWriteSchemaTransformProvider; +import org.apache.beam.sdk.schemas.transforms.SchemaTransform; import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; import org.checkerframework.checker.initialization.qual.Initialized; import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.UnknownKeyFor; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; @AutoService(SchemaTransformProvider.class) public class WriteToPostgresSchemaTransformProvider extends JdbcWriteSchemaTransformProvider { + private static final Logger LOG = + LoggerFactory.getLogger(WriteToPostgresSchemaTransformProvider.class); + @Override public @UnknownKeyFor @NonNull @Initialized String identifier() { - return "beam:schematransform:org.apache.beam:postgres_write:v1"; + return getUrn(ExternalTransforms.ManagedTransforms.Urns.POSTGRES_WRITE); } @Override @@ -43,4 +53,30 @@ public String description() { protected String jdbcType() { return POSTGRES; } + + @Override + public @UnknownKeyFor @NonNull @Initialized SchemaTransform from( + JdbcWriteSchemaTransformConfiguration configuration) { + String jdbcType = configuration.getJdbcType(); + if (jdbcType != null && !jdbcType.isEmpty() && !jdbcType.equals(jdbcType())) { + throw new IllegalArgumentException( + String.format("Wrong JDBC type. Expected '%s' but got '%s'", jdbcType(), jdbcType)); + } + + List<@org.checkerframework.checker.nullness.qual.Nullable String> connectionInitSql = + configuration.getConnectionInitSql(); + if (connectionInitSql != null && !connectionInitSql.isEmpty()) { + LOG.warn("Postgres does not support connectionInitSql, ignoring."); + } + + // Override "connectionInitSql" for postgres + configuration = configuration.toBuilder().setConnectionInitSql(Collections.emptyList()).build(); + return new PostgresWriteSchemaTransform(configuration); + } + + public static class PostgresWriteSchemaTransform extends JdbcWriteSchemaTransform { + public PostgresWriteSchemaTransform(JdbcWriteSchemaTransformConfiguration config) { + super(config, POSTGRES); + } + } } diff --git a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOPostgresIT.java b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOPostgresIT.java new file mode 100644 index 000000000000..d58783096929 --- /dev/null +++ b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOPostgresIT.java @@ -0,0 +1,178 @@ +/* + * 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. + */ +package org.apache.beam.sdk.io.jdbc; + +import static org.apache.beam.sdk.io.common.IOITHelper.readIOTestPipelineOptions; + +import java.sql.SQLException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import org.apache.beam.sdk.io.common.DatabaseTestHelper; +import org.apache.beam.sdk.io.common.PostgresIOTestPipelineOptions; +import org.apache.beam.sdk.io.jdbc.providers.ReadFromPostgresSchemaTransformProvider; +import org.apache.beam.sdk.io.jdbc.providers.WriteToPostgresSchemaTransformProvider; +import org.apache.beam.sdk.managed.Managed; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionRowTuple; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.postgresql.ds.PGSimpleDataSource; + +/** + * A test of {@link org.apache.beam.sdk.io.jdbc.JdbcIO} on an independent Postgres instance. + * + *

Similar to JdbcIOIT, this test requires a running instance of Postgres. Pass in connection + * information using PipelineOptions: + * + *

+ *  ./gradlew integrationTest -p sdks/java/io/jdbc -DintegrationTestPipelineOptions='[
+ *  "--postgresServerName=1.2.3.4",
+ *  "--postgresUsername=postgres",
+ *  "--postgresDatabaseName=myfancydb",
+ *  "--postgresPassword=mypass",
+ *  "--postgresSsl=false" ]'
+ *  --tests org.apache.beam.sdk.io.jdbc.JdbcIOPostgresIT
+ *  -DintegrationTestRunner=direct
+ * 
+ */ +@RunWith(JUnit4.class) +public class JdbcIOPostgresIT { + private static final Schema INPUT_SCHEMA = + Schema.of( + Schema.Field.of("id", Schema.FieldType.INT32), + Schema.Field.of("name", Schema.FieldType.STRING)); + + private static final List ROWS = + Arrays.asList( + Row.withSchema(INPUT_SCHEMA) + .withFieldValue("id", 1) + .withFieldValue("name", "foo") + .build(), + Row.withSchema(INPUT_SCHEMA) + .withFieldValue("id", 2) + .withFieldValue("name", "bar") + .build(), + Row.withSchema(INPUT_SCHEMA) + .withFieldValue("id", 3) + .withFieldValue("name", "baz") + .build()); + + private static PGSimpleDataSource dataSource; + private static String jdbcUrl; + + @Rule public TestPipeline writePipeline = TestPipeline.create(); + @Rule public TestPipeline readPipeline = TestPipeline.create(); + + @BeforeClass + public static void setup() { + PostgresIOTestPipelineOptions options; + try { + options = readIOTestPipelineOptions(PostgresIOTestPipelineOptions.class); + } catch (IllegalArgumentException e) { + options = null; + } + org.junit.Assume.assumeNotNull(options); + dataSource = DatabaseTestHelper.getPostgresDataSource(options); + jdbcUrl = DatabaseTestHelper.getPostgresDBUrl(options); + } + + @Test + public void testWriteThenRead() throws SQLException { + String tableName = DatabaseTestHelper.getTestTableName("JdbcIOPostgresIT"); + DatabaseTestHelper.createTable(dataSource, tableName); + + JdbcWriteSchemaTransformProvider.JdbcWriteSchemaTransformConfiguration writeConfig = + JdbcWriteSchemaTransformProvider.JdbcWriteSchemaTransformConfiguration.builder() + .setJdbcUrl(jdbcUrl) + .setUsername(dataSource.getUser()) + .setPassword(dataSource.getPassword()) + .setLocation(tableName) + .build(); + + JdbcReadSchemaTransformProvider.JdbcReadSchemaTransformConfiguration readConfig = + JdbcReadSchemaTransformProvider.JdbcReadSchemaTransformConfiguration.builder() + .setJdbcUrl(jdbcUrl) + .setUsername(dataSource.getUser()) + .setPassword(dataSource.getPassword()) + .setLocation(tableName) + .build(); + + try { + PCollection input = writePipeline.apply(Create.of(ROWS)).setRowSchema(INPUT_SCHEMA); + PCollectionRowTuple inputTuple = PCollectionRowTuple.of("input", input); + inputTuple.apply( + new WriteToPostgresSchemaTransformProvider.PostgresWriteSchemaTransform(writeConfig)); + writePipeline.run().waitUntilFinish(); + + PCollectionRowTuple pbeginTuple = PCollectionRowTuple.empty(readPipeline); + PCollectionRowTuple outputTuple = + pbeginTuple.apply( + new ReadFromPostgresSchemaTransformProvider.PostgresReadSchemaTransform(readConfig)); + PCollection output = outputTuple.get("output"); + PAssert.that(output).containsInAnyOrder(ROWS); + readPipeline.run().waitUntilFinish(); + } finally { + DatabaseTestHelper.deleteTable(dataSource, tableName); + } + } + + @Test + public void testManagedWriteThenManagedRead() throws SQLException { + String tableName = DatabaseTestHelper.getTestTableName("ManagedJdbcIOPostgresIT"); + DatabaseTestHelper.createTable(dataSource, tableName); + + Map writeConfig = + ImmutableMap.builder() + .put("jdbc_url", jdbcUrl) + .put("username", dataSource.getUser()) + .put("password", dataSource.getPassword()) + .put("location", tableName) + .build(); + + Map readConfig = + ImmutableMap.builder() + .put("jdbc_url", jdbcUrl) + .put("username", dataSource.getUser()) + .put("password", dataSource.getPassword()) + .put("location", tableName) + .build(); + + try { + PCollection input = writePipeline.apply(Create.of(ROWS)).setRowSchema(INPUT_SCHEMA); + input.apply(Managed.write(Managed.POSTGRES).withConfig(writeConfig)); + writePipeline.run().waitUntilFinish(); + + PCollectionRowTuple output = + readPipeline.apply(Managed.read(Managed.POSTGRES).withConfig(readConfig)); + PAssert.that(output.get("output")).containsInAnyOrder(ROWS); + readPipeline.run().waitUntilFinish(); + } finally { + DatabaseTestHelper.deleteTable(dataSource, tableName); + } + } +} diff --git a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/providers/PostgresSchemaTransformTranslationTest.java b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/providers/PostgresSchemaTransformTranslationTest.java new file mode 100644 index 000000000000..503baaefc334 --- /dev/null +++ b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/providers/PostgresSchemaTransformTranslationTest.java @@ -0,0 +1,233 @@ +/* + * 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. + */ +package org.apache.beam.sdk.io.jdbc.providers; + +import static org.apache.beam.model.pipeline.v1.ExternalTransforms.ExpansionMethods.Enum.SCHEMA_TRANSFORM; +import static org.apache.beam.sdk.io.jdbc.providers.PostgresSchemaTransformTranslation.PostgresReadSchemaTransformTranslator; +import static org.apache.beam.sdk.io.jdbc.providers.PostgresSchemaTransformTranslation.PostgresWriteSchemaTransformTranslator; +import static org.apache.beam.sdk.io.jdbc.providers.ReadFromPostgresSchemaTransformProvider.PostgresReadSchemaTransform; +import static org.apache.beam.sdk.io.jdbc.providers.WriteToPostgresSchemaTransformProvider.PostgresWriteSchemaTransform; +import static org.junit.Assert.assertEquals; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.beam.model.pipeline.v1.ExternalTransforms.SchemaTransformPayload; +import org.apache.beam.model.pipeline.v1.RunnerApi; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.RowCoder; +import org.apache.beam.sdk.io.jdbc.JdbcIO; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.SchemaTranslation; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.util.construction.BeamUrns; +import org.apache.beam.sdk.util.construction.PipelineTranslation; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionRowTuple; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.rules.TemporaryFolder; +import org.mockito.MockedStatic; +import org.mockito.Mockito; + +public class PostgresSchemaTransformTranslationTest { + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule public transient ExpectedException thrown = ExpectedException.none(); + + static final WriteToPostgresSchemaTransformProvider WRITE_PROVIDER = + new WriteToPostgresSchemaTransformProvider(); + static final ReadFromPostgresSchemaTransformProvider READ_PROVIDER = + new ReadFromPostgresSchemaTransformProvider(); + + static final Row READ_CONFIG = + Row.withSchema(READ_PROVIDER.configurationSchema()) + .withFieldValue("jdbc_url", "jdbc:postgresql://host:port/database") + .withFieldValue("location", "test_table") + .withFieldValue("connection_properties", "some_property") + .withFieldValue("connection_init_sql", ImmutableList.builder().build()) + .withFieldValue("driver_class_name", null) + .withFieldValue("driver_jars", null) + .withFieldValue("disable_auto_commit", true) + .withFieldValue("fetch_size", 10) + .withFieldValue("num_partitions", 5) + .withFieldValue("output_parallelization", true) + .withFieldValue("partition_column", "col") + .withFieldValue("read_query", null) + .withFieldValue("username", "my_user") + .withFieldValue("password", "my_pass") + .build(); + + static final Row WRITE_CONFIG = + Row.withSchema(WRITE_PROVIDER.configurationSchema()) + .withFieldValue("jdbc_url", "jdbc:postgresql://host:port/database") + .withFieldValue("location", "test_table") + .withFieldValue("autosharding", true) + .withFieldValue("connection_init_sql", ImmutableList.builder().build()) + .withFieldValue("connection_properties", "some_property") + .withFieldValue("driver_class_name", null) + .withFieldValue("driver_jars", null) + .withFieldValue("batch_size", 100L) + .withFieldValue("username", "my_user") + .withFieldValue("password", "my_pass") + .withFieldValue("write_statement", null) + .build(); + + @Test + public void testRecreateWriteTransformFromRow() { + PostgresWriteSchemaTransform writeTransform = + (PostgresWriteSchemaTransform) WRITE_PROVIDER.from(WRITE_CONFIG); + + PostgresWriteSchemaTransformTranslator translator = + new PostgresWriteSchemaTransformTranslator(); + Row translatedRow = translator.toConfigRow(writeTransform); + + PostgresWriteSchemaTransform writeTransformFromRow = + translator.fromConfigRow(translatedRow, PipelineOptionsFactory.create()); + + assertEquals(WRITE_CONFIG, writeTransformFromRow.getConfigurationRow()); + } + + @Test + public void testWriteTransformProtoTranslation() + throws InvalidProtocolBufferException, IOException { + // First build a pipeline + Pipeline p = Pipeline.create(); + Schema inputSchema = Schema.builder().addStringField("name").build(); + PCollection input = + p.apply( + Create.of( + Collections.singletonList( + Row.withSchema(inputSchema).addValue("test").build()))) + .setRowSchema(inputSchema); + + PostgresWriteSchemaTransform writeTransform = + (PostgresWriteSchemaTransform) WRITE_PROVIDER.from(WRITE_CONFIG); + PCollectionRowTuple.of("input", input).apply(writeTransform); + + // Then translate the pipeline to a proto and extract PostgresWriteSchemaTransform proto + RunnerApi.Pipeline pipelineProto = PipelineTranslation.toProto(p); + List writeTransformProto = + pipelineProto.getComponents().getTransformsMap().values().stream() + .filter( + tr -> { + RunnerApi.FunctionSpec spec = tr.getSpec(); + try { + return spec.getUrn().equals(BeamUrns.getUrn(SCHEMA_TRANSFORM)) + && SchemaTransformPayload.parseFrom(spec.getPayload()) + .getIdentifier() + .equals(WRITE_PROVIDER.identifier()); + } catch (InvalidProtocolBufferException e) { + throw new RuntimeException(e); + } + }) + .collect(Collectors.toList()); + assertEquals(1, writeTransformProto.size()); + RunnerApi.FunctionSpec spec = writeTransformProto.get(0).getSpec(); + + // Check that the proto contains correct values + SchemaTransformPayload payload = SchemaTransformPayload.parseFrom(spec.getPayload()); + Schema schemaFromSpec = SchemaTranslation.schemaFromProto(payload.getConfigurationSchema()); + assertEquals(WRITE_PROVIDER.configurationSchema(), schemaFromSpec); + Row rowFromSpec = RowCoder.of(schemaFromSpec).decode(payload.getConfigurationRow().newInput()); + + assertEquals(WRITE_CONFIG, rowFromSpec); + + // Use the information in the proto to recreate the PostgresWriteSchemaTransform + PostgresWriteSchemaTransformTranslator translator = + new PostgresWriteSchemaTransformTranslator(); + PostgresWriteSchemaTransform writeTransformFromSpec = + translator.fromConfigRow(rowFromSpec, PipelineOptionsFactory.create()); + + assertEquals(WRITE_CONFIG, writeTransformFromSpec.getConfigurationRow()); + } + + @Test + public void testReCreateReadTransformFromRow() { + // setting a subset of fields here. + PostgresReadSchemaTransform readTransform = + (PostgresReadSchemaTransform) READ_PROVIDER.from(READ_CONFIG); + + PostgresReadSchemaTransformTranslator translator = new PostgresReadSchemaTransformTranslator(); + Row row = translator.toConfigRow(readTransform); + + PostgresReadSchemaTransform readTransformFromRow = + translator.fromConfigRow(row, PipelineOptionsFactory.create()); + + assertEquals(READ_CONFIG, readTransformFromRow.getConfigurationRow()); + } + + @Test + public void testReadTransformProtoTranslation() + throws InvalidProtocolBufferException, IOException { + // First build a pipeline + Pipeline p = Pipeline.create(); + + PostgresReadSchemaTransform readTransform = + (PostgresReadSchemaTransform) READ_PROVIDER.from(READ_CONFIG); + + // Mock inferBeamSchema since it requires database connection. + Schema expectedSchema = Schema.builder().addStringField("name").build(); + try (MockedStatic mock = Mockito.mockStatic(JdbcIO.ReadRows.class)) { + mock.when(() -> JdbcIO.ReadRows.inferBeamSchema(Mockito.any(), Mockito.any())) + .thenReturn(expectedSchema); + PCollectionRowTuple.empty(p).apply(readTransform); + } + + // Then translate the pipeline to a proto and extract PostgresReadSchemaTransform proto + RunnerApi.Pipeline pipelineProto = PipelineTranslation.toProto(p); + List readTransformProto = + pipelineProto.getComponents().getTransformsMap().values().stream() + .filter( + tr -> { + RunnerApi.FunctionSpec spec = tr.getSpec(); + try { + return spec.getUrn().equals(BeamUrns.getUrn(SCHEMA_TRANSFORM)) + && SchemaTransformPayload.parseFrom(spec.getPayload()) + .getIdentifier() + .equals(READ_PROVIDER.identifier()); + } catch (InvalidProtocolBufferException e) { + throw new RuntimeException(e); + } + }) + .collect(Collectors.toList()); + assertEquals(1, readTransformProto.size()); + RunnerApi.FunctionSpec spec = readTransformProto.get(0).getSpec(); + + // Check that the proto contains correct values + SchemaTransformPayload payload = SchemaTransformPayload.parseFrom(spec.getPayload()); + Schema schemaFromSpec = SchemaTranslation.schemaFromProto(payload.getConfigurationSchema()); + assertEquals(READ_PROVIDER.configurationSchema(), schemaFromSpec); + Row rowFromSpec = RowCoder.of(schemaFromSpec).decode(payload.getConfigurationRow().newInput()); + assertEquals(READ_CONFIG, rowFromSpec); + + // Use the information in the proto to recreate the PostgresReadSchemaTransform + PostgresReadSchemaTransformTranslator translator = new PostgresReadSchemaTransformTranslator(); + PostgresReadSchemaTransform readTransformFromSpec = + translator.fromConfigRow(rowFromSpec, PipelineOptionsFactory.create()); + + assertEquals(READ_CONFIG, readTransformFromSpec.getConfigurationRow()); + } +} diff --git a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java index 77b2e3f617c4..2a7cd62d33d2 100644 --- a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java +++ b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java @@ -847,6 +847,7 @@ private void closeAutoscaler() { } @Override + @SuppressWarnings("Finalize") protected void finalize() { doClose(); } diff --git a/sdks/java/io/kafka/build.gradle b/sdks/java/io/kafka/build.gradle index 6e9b5aec0932..ba25078b64e3 100644 --- a/sdks/java/io/kafka/build.gradle +++ b/sdks/java/io/kafka/build.gradle @@ -74,6 +74,9 @@ dependencies { implementation (group: 'com.google.cloud.hosted.kafka', name: 'managed-kafka-auth-login-handler', version: '1.0.5') { // "kafka-clients" has to be provided since user can use its own version. exclude group: 'org.apache.kafka', module: 'kafka-clients' + // "kafka-schema-registry-client must be excluded per the Google Cloud documentation: + // https://cloud.google.com/managed-service-for-apache-kafka/docs/quickstart-avro#configure_and_run_the_producer + exclude group: "io.confluent", module: "kafka-schema-registry-client" } implementation ("io.confluent:kafka-avro-serializer:${confluentVersion}") { // zookeeper depends on "spotbugs-annotations:3.1.9" which clashes with current diff --git a/sdks/java/io/kafka/jmh/src/main/java/org/apache/beam/sdk/io/kafka/jmh/KafkaIOUtilsBenchmark.java b/sdks/java/io/kafka/jmh/src/main/java/org/apache/beam/sdk/io/kafka/jmh/KafkaIOUtilsBenchmark.java index 8523e2094895..36fb389053f7 100644 --- a/sdks/java/io/kafka/jmh/src/main/java/org/apache/beam/sdk/io/kafka/jmh/KafkaIOUtilsBenchmark.java +++ b/sdks/java/io/kafka/jmh/src/main/java/org/apache/beam/sdk/io/kafka/jmh/KafkaIOUtilsBenchmark.java @@ -33,6 +33,7 @@ import org.openjdk.jmh.infra.IterationParams; import org.openjdk.jmh.infra.ThreadParams; +@SuppressWarnings("SameNameButDifferent") // for MovingArg @BenchmarkMode(Mode.AverageTime) @OutputTimeUnit(TimeUnit.NANOSECONDS) @Threads(Threads.MAX) diff --git a/sdks/java/io/kafka/kafka-integration-test.gradle b/sdks/java/io/kafka/kafka-integration-test.gradle index 3bbab72ff77c..14d90349dedd 100644 --- a/sdks/java/io/kafka/kafka-integration-test.gradle +++ b/sdks/java/io/kafka/kafka-integration-test.gradle @@ -33,6 +33,7 @@ dependencies { // instead, rely on io/kafka/build.gradle's custom configurations with forced kafka-client resolutionStrategy testImplementation 'org.junit.jupiter:junit-jupiter-api:5.8.1' testRuntimeOnly 'org.junit.jupiter:junit-jupiter-engine:5.8.1' + testImplementation library.java.avro } configurations.create("kafkaVersion$undelimited") diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaCommitOffset.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaCommitOffset.java index fa692d3aaf42..ac6650c354d4 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaCommitOffset.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaCommitOffset.java @@ -71,6 +71,8 @@ static class CommitOffsetDoFn extends DoFn, Void consumerFactoryFn = readSourceDescriptors.getConsumerFactoryFn(); } + @SuppressWarnings( + "Slf4jDoNotLogMessageOfExceptionExplicitly") // for tests checking error message @RequiresStableInput @ProcessElement public void processElement(@Element KV element) { diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java index bbe20b1ed63a..045a74a8507e 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java @@ -92,6 +92,7 @@ import org.apache.beam.sdk.transforms.splittabledofn.WatermarkEstimators.Manual; import org.apache.beam.sdk.transforms.splittabledofn.WatermarkEstimators.MonotonicallyIncreasing; import org.apache.beam.sdk.transforms.splittabledofn.WatermarkEstimators.WallTime; +import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.util.construction.PTransformMatchers; import org.apache.beam.sdk.util.construction.ReplacementOutputs; @@ -109,6 +110,7 @@ import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Joiner; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerConfig; @@ -653,6 +655,14 @@ public static WriteRecords writeRecords() { ///////////////////////// Read Support \\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\ + /** + * Default number of keys to redistribute Kafka inputs into. + * + *

This value is used when {@link Read#withRedistribute()} is used without {@link + * Read#withRedistributeNumKeys(int redistributeNumKeys)}. + */ + private static final int DEFAULT_REDISTRIBUTE_NUM_KEYS = 32768; + /** * A {@link PTransform} to read from Kafka topics. See {@link KafkaIO} for more information on * usage and configuration. @@ -1093,19 +1103,60 @@ public Read withTopicPartitions(List topicPartitions) { /** * Sets redistribute transform that hints to the runner to try to redistribute the work evenly. + * + * @return an updated {@link Read} transform. */ public Read withRedistribute() { - return toBuilder().setRedistributed(true).build(); + Builder builder = toBuilder().setRedistributed(true); + if (getRedistributeNumKeys() == 0) { + builder = builder.setRedistributeNumKeys(DEFAULT_REDISTRIBUTE_NUM_KEYS); + } + return builder.build(); } + /** + * Hints to the runner that it can relax exactly-once processing guarantees, allowing duplicates + * in at-least-once processing mode of Kafka inputs. + * + *

Must be used with {@link KafkaIO#withRedistribute()}. + * + *

Not compatible with {@link KafkaIO#withOffsetDeduplication()}. + * + * @param allowDuplicates specifies whether to allow duplicates. + * @return an updated {@link Read} transform. + */ public Read withAllowDuplicates(Boolean allowDuplicates) { return toBuilder().setAllowDuplicates(allowDuplicates).build(); } + /** + * Redistributes Kafka messages into a distinct number of keys for processing in subsequent + * steps. + * + *

If unset, defaults to {@link KafkaIO#DEFAULT_REDISTRIBUTE_NUM_KEYS}. + * + *

Use zero to disable bucketing into a distinct number of keys. + * + *

Must be used with {@link Read#withRedistribute()}. + * + * @param redistributeNumKeys specifies the total number of keys for redistributing inputs. + * @return an updated {@link Read} transform. + */ public Read withRedistributeNumKeys(int redistributeNumKeys) { return toBuilder().setRedistributeNumKeys(redistributeNumKeys).build(); } + /** + * Hints to the runner to optimize the redistribute by minimizing the amount of data required + * for persistence as part of the redistribute operation. + * + *

Must be used with {@link KafkaIO#withRedistribute()}. + * + *

Not compatible with {@link KafkaIO#withAllowDuplicates()}. + * + * @param offsetDeduplication specifies whether to enable offset-based deduplication. + * @return an updated {@link Read} transform. + */ public Read withOffsetDeduplication(Boolean offsetDeduplication) { return toBuilder().setOffsetDeduplication(offsetDeduplication).build(); } @@ -1619,10 +1670,14 @@ private void checkRedistributeConfiguration() { isRedistributed(), "withRedistributeNumKeys is ignored if withRedistribute() is not enabled on the transform."); } - if (getOffsetDeduplication() != null && getOffsetDeduplication()) { + if (getOffsetDeduplication() != null && getOffsetDeduplication() && isRedistributed()) { checkState( - isRedistributed() && !isAllowDuplicates(), - "withOffsetDeduplication should only be used with withRedistribute and withAllowDuplicates(false)."); + !isAllowDuplicates(), + "withOffsetDeduplication and withRedistribute can only be used when withAllowDuplicates is set to false."); + } + if (getOffsetDeduplication() != null && getOffsetDeduplication() && !isRedistributed()) { + LOG.warn( + "Offsets used for deduplication are available in WindowedValue's metadata. Combining, aggregating, mutating them may risk with data loss."); } } @@ -1790,13 +1845,18 @@ public PCollection> expand(PBegin input) { .withMaxReadTime(kafkaRead.getMaxReadTime()) .withMaxNumRecords(kafkaRead.getMaxNumRecords()); } - + PCollection> output = input.getPipeline().apply(transform); + if (kafkaRead.getOffsetDeduplication() != null && kafkaRead.getOffsetDeduplication()) { + output = + output.apply( + "Insert Offset for offset deduplication", + ParDo.of(new OffsetDeduplicationIdExtractor<>())); + } if (kafkaRead.isRedistributed()) { if (kafkaRead.isCommitOffsetsInFinalizeEnabled() && kafkaRead.isAllowDuplicates()) { LOG.warn( "Offsets committed due to usage of commitOffsetsInFinalize() and may not capture all work processed due to use of withRedistribute() with duplicates enabled"); } - PCollection> output = input.getPipeline().apply(transform); if (kafkaRead.getRedistributeNumKeys() == 0) { return output.apply( @@ -1811,7 +1871,7 @@ public PCollection> expand(PBegin input) { .withNumBuckets((int) kafkaRead.getRedistributeNumKeys())); } } - return input.getPipeline().apply(transform); + return output; } } @@ -1920,6 +1980,29 @@ public PCollection> expand(PBegin input) { } } + static class OffsetDeduplicationIdExtractor + extends DoFn, KafkaRecord> { + + @ProcessElement + public void processElement(ProcessContext pc) { + KafkaRecord element = pc.element(); + Long offset = null; + String uniqueId = null; + if (element != null) { + offset = element.getOffset(); + uniqueId = + (String.format("%s-%d-%d", element.getTopic(), element.getPartition(), offset)); + } + pc.outputWindowedValue( + element, + pc.timestamp(), + Lists.newArrayList(GlobalWindow.INSTANCE), + pc.pane(), + uniqueId, + offset); + } + } + /** * A DoFn which generates {@link KafkaSourceDescriptor} based on the configuration of {@link * Read}. @@ -2597,13 +2680,30 @@ public ReadSourceDescriptors withProcessingTime() { /** Enable Redistribute. */ public ReadSourceDescriptors withRedistribute() { - return toBuilder().setRedistribute(true).build(); + Builder builder = toBuilder().setRedistribute(true); + if (getRedistributeNumKeys() == 0) { + builder = builder.setRedistributeNumKeys(DEFAULT_REDISTRIBUTE_NUM_KEYS); + } + return builder.build(); } public ReadSourceDescriptors withAllowDuplicates() { return toBuilder().setAllowDuplicates(true).build(); } + /** + * Redistributes Kafka messages into a distinct number of keys for processing in subsequent + * steps. + * + *

If unset, defaults to {@link KafkaIO#DEFAULT_REDISTRIBUTE_NUM_KEYS}. + * + *

Use zero to disable bucketing into a distinct number of keys. + * + *

Must be used with {@link ReadSourceDescriptors#withRedistribute()}. + * + * @param redistributeNumKeys specifies the total number of keys for redistributing inputs. + * @return an updated {@link Read} transform. + */ public ReadSourceDescriptors withRedistributeNumKeys(int redistributeNumKeys) { return toBuilder().setRedistributeNumKeys(redistributeNumKeys).build(); } diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIOUtils.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIOUtils.java index 1352d6bd864b..91aa85577959 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIOUtils.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIOUtils.java @@ -168,18 +168,23 @@ private void setAvg(final double value) { AVG.lazySet(this, Double.doubleToRawLongBits(value)); } - private long incrementAndGetNumUpdates() { - final long nextNumUpdates = Math.min(MOVING_AVG_WINDOW, numUpdates + 1); - numUpdates = nextNumUpdates; - return nextNumUpdates; + public void update(final double quantity) { + final double prevAvg = getAvg(); // volatile load (acquire) + + final long nextNumUpdates = numUpdates + 1; // normal load + final double nextAvg = prevAvg + (quantity - prevAvg) / nextNumUpdates; + + numUpdates = Math.min(MOVING_AVG_WINDOW, nextNumUpdates); // normal store + setAvg(nextAvg); // ordered store (release) } - public void update(final double quantity) { + public void update(final double sum, final long count) { final double prevAvg = getAvg(); // volatile load (acquire) - final long nextNumUpdates = incrementAndGetNumUpdates(); // normal load/store - final double nextAvg = prevAvg + (quantity - prevAvg) / nextNumUpdates; // normal load/store + final long nextNumUpdates = numUpdates + count; // normal load + final double nextAvg = prevAvg + (sum / count - prevAvg) * ((double) count / nextNumUpdates); + numUpdates = Math.min(MOVING_AVG_WINDOW, nextNumUpdates); // normal store setAvg(nextAvg); // ordered store (release) } diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProvider.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProvider.java index d6f46b11cb7d..e2a4f394ccdb 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProvider.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProvider.java @@ -21,6 +21,7 @@ import com.google.auto.service.AutoService; import com.google.auto.value.AutoValue; +import io.confluent.kafka.serializers.KafkaAvroSerializer; import java.io.Serializable; import java.util.Collections; import java.util.HashMap; @@ -28,7 +29,11 @@ import java.util.Map; import java.util.Set; import javax.annotation.Nullable; +import org.apache.avro.generic.GenericRecord; import org.apache.beam.model.pipeline.v1.ExternalTransforms; +import org.apache.beam.sdk.coders.ByteArrayCoder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils; import org.apache.beam.sdk.extensions.protobuf.ProtoByteUtils; import org.apache.beam.sdk.metrics.Counter; @@ -74,6 +79,8 @@ public class KafkaWriteSchemaTransformProvider public static final TupleTag ERROR_TAG = new TupleTag() {}; public static final TupleTag> OUTPUT_TAG = new TupleTag>() {}; + public static final TupleTag> RECORD_OUTPUT_TAG = + new TupleTag>() {}; private static final Logger LOG = LoggerFactory.getLogger(KafkaWriteSchemaTransformProvider.class); @@ -118,29 +125,32 @@ Row getConfigurationRow() { } } - public static class ErrorCounterFn extends DoFn> { - private final SerializableFunction toBytesFn; + public abstract static class BaseKafkaWriterFn extends DoFn> { + private final SerializableFunction conversionFn; private final Counter errorCounter; private Long errorsInBundle = 0L; private final boolean handleErrors; private final Schema errorSchema; + private final TupleTag> successTag; - public ErrorCounterFn( + public BaseKafkaWriterFn( String name, - SerializableFunction toBytesFn, + SerializableFunction conversionFn, Schema errorSchema, - boolean handleErrors) { - this.toBytesFn = toBytesFn; + boolean handleErrors, + TupleTag> successTag) { + this.conversionFn = conversionFn; this.errorCounter = Metrics.counter(KafkaWriteSchemaTransformProvider.class, name); this.handleErrors = handleErrors; this.errorSchema = errorSchema; + this.successTag = successTag; } @ProcessElement public void process(@DoFn.Element Row row, MultiOutputReceiver receiver) { - KV output = null; + KV output = null; try { - output = KV.of(new byte[1], toBytesFn.apply(row)); + output = KV.of(new byte[1], conversionFn.apply(row)); } catch (Exception e) { if (!handleErrors) { throw new RuntimeException(e); @@ -150,7 +160,7 @@ public void process(@DoFn.Element Row row, MultiOutputReceiver receiver) { receiver.get(ERROR_TAG).output(ErrorHandling.errorRecord(errorSchema, row, e)); } if (output != null) { - receiver.get(OUTPUT_TAG).output(output); + receiver.get(successTag).output(output); } } @@ -161,13 +171,35 @@ public void finish() { } } + public static class ErrorCounterFn extends BaseKafkaWriterFn { + public ErrorCounterFn( + String name, + SerializableFunction toBytesFn, + Schema errorSchema, + boolean handleErrors) { + super(name, toBytesFn, errorSchema, handleErrors, OUTPUT_TAG); + } + } + + public static class GenericRecordErrorCounterFn extends BaseKafkaWriterFn { + public GenericRecordErrorCounterFn( + String name, + SerializableFunction toGenericRecordsFn, + Schema errorSchema, + boolean handleErrors) { + super(name, toGenericRecordsFn, errorSchema, handleErrors, RECORD_OUTPUT_TAG); + } + } + @SuppressWarnings({ "nullness" // TODO(https://github.com/apache/beam/issues/20497) }) @Override public PCollectionRowTuple expand(PCollectionRowTuple input) { Schema inputSchema = input.get("input").getSchema(); + org.apache.avro.Schema avroSchema = AvroUtils.toAvroSchema(inputSchema); final SerializableFunction toBytesFn; + SerializableFunction toGenericRecordsFn = null; if (configuration.getFormat().equals("RAW")) { int numFields = inputSchema.getFields().size(); if (numFields != 1) { @@ -198,36 +230,70 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { throw new IllegalArgumentException( "At least a descriptorPath or a proto Schema is required."); } - } else { - toBytesFn = AvroUtils.getRowToAvroBytesFunction(inputSchema); + if (configuration.getProducerConfigUpdates() != null + && configuration.getProducerConfigUpdates().containsKey("schema.registry.url")) { + toGenericRecordsFn = AvroUtils.getRowToGenericRecordFunction(avroSchema); + toBytesFn = null; + } else { + toBytesFn = AvroUtils.getRowToAvroBytesFunction(inputSchema); + } } boolean handleErrors = ErrorHandling.hasOutput(configuration.getErrorHandling()); final Map configOverrides = configuration.getProducerConfigUpdates(); Schema errorSchema = ErrorHandling.errorSchema(inputSchema); - PCollectionTuple outputTuple = - input - .get("input") - .apply( - "Map rows to Kafka messages", - ParDo.of( - new ErrorCounterFn( - "Kafka-write-error-counter", toBytesFn, errorSchema, handleErrors)) - .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); - - outputTuple - .get(OUTPUT_TAG) - .apply( - KafkaIO.write() - .withTopic(configuration.getTopic()) - .withBootstrapServers(configuration.getBootstrapServers()) - .withProducerConfigUpdates( - configOverrides == null - ? new HashMap<>() - : new HashMap(configOverrides)) - .withKeySerializer(ByteArraySerializer.class) - .withValueSerializer(ByteArraySerializer.class)); + PCollectionTuple outputTuple; + if (toGenericRecordsFn != null) { + LOG.info("Convert to GenericRecord with schema {}", avroSchema); + outputTuple = + input + .get("input") + .apply( + "Map rows to Kafka messages", + ParDo.of( + new GenericRecordErrorCounterFn( + "Kafka-write-error-counter", + toGenericRecordsFn, + errorSchema, + handleErrors)) + .withOutputTags(RECORD_OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); + HashMap producerConfig = new HashMap<>(configOverrides); + outputTuple + .get(RECORD_OUTPUT_TAG) + .setCoder(KvCoder.of(ByteArrayCoder.of(), AvroCoder.of(avroSchema))) + .apply( + "Map Rows to GenericRecords", + KafkaIO.write() + .withTopic(configuration.getTopic()) + .withBootstrapServers(configuration.getBootstrapServers()) + .withProducerConfigUpdates(producerConfig) + .withKeySerializer(ByteArraySerializer.class) + .withValueSerializer((Class) KafkaAvroSerializer.class)); + } else { + outputTuple = + input + .get("input") + .apply( + "Map rows to Kafka messages", + ParDo.of( + new ErrorCounterFn( + "Kafka-write-error-counter", toBytesFn, errorSchema, handleErrors)) + .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); + + outputTuple + .get(OUTPUT_TAG) + .apply( + KafkaIO.write() + .withTopic(configuration.getTopic()) + .withBootstrapServers(configuration.getBootstrapServers()) + .withProducerConfigUpdates( + configOverrides == null + ? new HashMap<>() + : new HashMap(configOverrides)) + .withKeySerializer(ByteArraySerializer.class) + .withValueSerializer(ByteArraySerializer.class)); + } // TODO: include output from KafkaIO Write once updated from PDone PCollection errorOutput = diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriter.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriter.java index f483c69d33bf..cad0f8a68d8c 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriter.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriter.java @@ -202,7 +202,7 @@ public void onCompletion(RecordMetadata metadata, Exception exception) { } numSendFailures++; // don't log exception stacktrace here, exception will be propagated up. - LOG.warn("send failed : '{}'", exception.getMessage()); + LOG.warn("send failed", exception); } } } diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFn.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFn.java index 70015847e19d..eab5ae083187 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFn.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFn.java @@ -78,7 +78,10 @@ import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.serialization.Deserializer; +import org.checkerframework.checker.nullness.qual.EnsuresNonNull; +import org.checkerframework.checker.nullness.qual.MonotonicNonNull; import org.checkerframework.checker.nullness.qual.Nullable; +import org.checkerframework.checker.nullness.qual.RequiresNonNull; import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -316,6 +319,16 @@ public Consumer load( private final SerializableSupplier>> pollConsumerCacheSupplier; + private transient @MonotonicNonNull LoadingCache + avgRecordSizeCache; + + private transient @MonotonicNonNull LoadingCache< + KafkaSourceDescriptor, KafkaLatestOffsetEstimator> + latestOffsetEstimatorCache; + + private transient @MonotonicNonNull LoadingCache> + pollConsumerCache; + // Valid between bundle start and bundle finish. private transient @Nullable Deserializer keyDeserializerInstance = null; private transient @Nullable Deserializer valueDeserializerInstance = null; @@ -433,9 +446,12 @@ private void refresh() { } @GetInitialRestriction + @RequiresNonNull({"pollConsumerCache"}) public OffsetRange initialRestriction(@Element KafkaSourceDescriptor kafkaSourceDescriptor) { - final Consumer consumer = - pollConsumerCacheSupplier.get().getUnchecked(kafkaSourceDescriptor); + final LoadingCache> pollConsumerCache = + this.pollConsumerCache; + + final Consumer consumer = pollConsumerCache.getUnchecked(kafkaSourceDescriptor); final long startOffset; final long stopOffset; @@ -513,12 +529,16 @@ public WatermarkEstimator newWatermarkEstimator( } @GetSize + @RequiresNonNull({"avgRecordSizeCache", "latestOffsetEstimatorCache"}) public double getSize( @Element KafkaSourceDescriptor kafkaSourceDescriptor, @Restriction OffsetRange offsetRange) { + final LoadingCache avgRecordSizeCache = + this.avgRecordSizeCache; + // If present, estimates the record size to offset gap ratio. Compacted topics may hold less // records than the estimated offset range due to record deletion within a partition. final @Nullable MovingAvg avgRecordSize = - avgRecordSizeCacheSupplier.get().getIfPresent(kafkaSourceDescriptor); + avgRecordSizeCache.getIfPresent(kafkaSourceDescriptor); // The tracker estimates the offset range by subtracting the last claimed position from the // currently observed end offset for the partition belonging to this split. final double estimatedOffsetRange = @@ -533,8 +553,12 @@ public double getSize( } @NewTracker + @RequiresNonNull({"latestOffsetEstimatorCache"}) public OffsetRangeTracker restrictionTracker( @Element KafkaSourceDescriptor kafkaSourceDescriptor, @Restriction OffsetRange restriction) { + final LoadingCache + latestOffsetEstimatorCache = this.latestOffsetEstimatorCache; + if (restriction.getTo() < Long.MAX_VALUE) { return new OffsetRangeTracker(restriction); } @@ -543,22 +567,28 @@ public OffsetRangeTracker restrictionTracker( // so we want to minimize the amount of connections that we start and track with Kafka. Another // point is that it has a memoized backlog, and this should make that more reusable estimations. return new GrowableOffsetRangeTracker( - restriction.getFrom(), - latestOffsetEstimatorCacheSupplier.get().getUnchecked(kafkaSourceDescriptor)); + restriction.getFrom(), latestOffsetEstimatorCache.getUnchecked(kafkaSourceDescriptor)); } @ProcessElement + @RequiresNonNull({"avgRecordSizeCache", "latestOffsetEstimatorCache", "pollConsumerCache"}) public ProcessContinuation processElement( @Element KafkaSourceDescriptor kafkaSourceDescriptor, RestrictionTracker tracker, WatermarkEstimator watermarkEstimator, MultiOutputReceiver receiver) throws Exception { - final MovingAvg avgRecordSize = avgRecordSizeCacheSupplier.get().get(kafkaSourceDescriptor); + final LoadingCache avgRecordSizeCache = + this.avgRecordSizeCache; + final LoadingCache + latestOffsetEstimatorCache = this.latestOffsetEstimatorCache; + final LoadingCache> pollConsumerCache = + this.pollConsumerCache; + + final MovingAvg avgRecordSize = avgRecordSizeCache.get(kafkaSourceDescriptor); final KafkaLatestOffsetEstimator latestOffsetEstimator = - latestOffsetEstimatorCacheSupplier.get().get(kafkaSourceDescriptor); - final Consumer consumer = - pollConsumerCacheSupplier.get().get(kafkaSourceDescriptor); + latestOffsetEstimatorCache.get(kafkaSourceDescriptor); + final Consumer consumer = pollConsumerCache.get(kafkaSourceDescriptor); final Deserializer keyDeserializerInstance = Preconditions.checkStateNotNull(this.keyDeserializerInstance); final Deserializer valueDeserializerInstance = @@ -588,6 +618,7 @@ public ProcessContinuation processElement( topicPartition, Optional.ofNullable(watermarkEstimator.currentWatermark())); } + Duration remainingTimeout = this.consumerPollingTimeout; long expectedOffset = tracker.currentRestriction().getFrom(); consumer.resume(Collections.singleton(topicPartition)); consumer.seek(topicPartition, expectedOffset); @@ -595,16 +626,21 @@ public ProcessContinuation processElement( final KafkaMetrics kafkaMetrics = KafkaSinkMetrics.kafkaMetrics(); try { - while (true) { + while (Duration.ZERO.compareTo(remainingTimeout) < 0) { // TODO: Remove this timer and use the existing fetch-latency-avg metric. // A consumer will often have prefetches waiting to be returned immediately in which case // this timer may contribute more latency than it measures. // See https://shipilev.net/blog/2014/nanotrusting-nanotime/ for more information. pollTimer.reset().start(); // Fetch the next records. - final ConsumerRecords rawRecords = - consumer.poll(this.consumerPollingTimeout); - kafkaMetrics.updateSuccessfulRpcMetrics(topicPartition.topic(), pollTimer.elapsed()); + final ConsumerRecords rawRecords = consumer.poll(remainingTimeout); + final Duration elapsed = pollTimer.elapsed(); + try { + remainingTimeout = remainingTimeout.minus(elapsed); + } catch (ArithmeticException e) { + remainingTimeout = Duration.ZERO; + } + kafkaMetrics.updateSuccessfulRpcMetrics(topicPartition.topic(), elapsed); // No progress when the polling timeout expired. // Self-checkpoint and move to process the next element. @@ -624,57 +660,70 @@ public ProcessContinuation processElement( // Visible progress within the consumer polling timeout. // Partially or fully claim and process records in this batch. - for (ConsumerRecord rawRecord : rawRecords) { - if (!tracker.tryClaim(rawRecord.offset())) { - consumer.seek(topicPartition, rawRecord.offset()); - consumer.pause(Collections.singleton(topicPartition)); + long rawSizesSum = 0L; + long rawSizesCount = 0L; + long rawSizesMin = Long.MAX_VALUE; + long rawSizesMax = Long.MIN_VALUE; + try { + for (ConsumerRecord rawRecord : rawRecords) { + if (!tracker.tryClaim(rawRecord.offset())) { + consumer.seek(topicPartition, rawRecord.offset()); + consumer.pause(Collections.singleton(topicPartition)); - return ProcessContinuation.stop(); - } - expectedOffset = rawRecord.offset() + 1; - try { - KafkaRecord kafkaRecord = - new KafkaRecord<>( - rawRecord.topic(), - rawRecord.partition(), - rawRecord.offset(), - ConsumerSpEL.getRecordTimestamp(rawRecord), - ConsumerSpEL.getRecordTimestampType(rawRecord), - ConsumerSpEL.hasHeaders() ? rawRecord.headers() : null, - ConsumerSpEL.deserializeKey(keyDeserializerInstance, rawRecord), - ConsumerSpEL.deserializeValue(valueDeserializerInstance, rawRecord)); - int recordSize = - (rawRecord.key() == null ? 0 : rawRecord.key().length) - + (rawRecord.value() == null ? 0 : rawRecord.value().length); - avgRecordSize.update(recordSize); - rawSizes.update(recordSize); - Instant outputTimestamp; - // The outputTimestamp and watermark will be computed by timestampPolicy, where the - // WatermarkEstimator should be a manual one. - if (timestampPolicy != null) { - TimestampPolicyContext context = - updateWatermarkManually(timestampPolicy, watermarkEstimator, tracker); - outputTimestamp = timestampPolicy.getTimestampForRecord(context, kafkaRecord); - } else { - Preconditions.checkStateNotNull(this.extractOutputTimestampFn); - outputTimestamp = extractOutputTimestampFn.apply(kafkaRecord); + return ProcessContinuation.stop(); } - receiver - .get(recordTag) - .outputWithTimestamp(KV.of(kafkaSourceDescriptor, kafkaRecord), outputTimestamp); - } catch (SerializationException e) { - // This exception should only occur during the key and value deserialization when - // creating the Kafka Record - badRecordRouter.route( - receiver, - rawRecord, - null, - e, - "Failure deserializing Key or Value of Kakfa record reading from Kafka"); - if (timestampPolicy != null) { - updateWatermarkManually(timestampPolicy, watermarkEstimator, tracker); + expectedOffset = rawRecord.offset() + 1; + try { + KafkaRecord kafkaRecord = + new KafkaRecord<>( + rawRecord.topic(), + rawRecord.partition(), + rawRecord.offset(), + ConsumerSpEL.getRecordTimestamp(rawRecord), + ConsumerSpEL.getRecordTimestampType(rawRecord), + ConsumerSpEL.hasHeaders() ? rawRecord.headers() : null, + ConsumerSpEL.deserializeKey(keyDeserializerInstance, rawRecord), + ConsumerSpEL.deserializeValue(valueDeserializerInstance, rawRecord)); + int recordSize = + (rawRecord.key() == null ? 0 : rawRecord.key().length) + + (rawRecord.value() == null ? 0 : rawRecord.value().length); + rawSizesSum = rawSizesSum + recordSize; + rawSizesCount = rawSizesCount + 1L; + rawSizesMin = Math.min(rawSizesMin, recordSize); + rawSizesMax = Math.max(rawSizesMax, recordSize); + Instant outputTimestamp; + // The outputTimestamp and watermark will be computed by timestampPolicy, where the + // WatermarkEstimator should be a manual one. + if (timestampPolicy != null) { + TimestampPolicyContext context = + updateWatermarkManually(timestampPolicy, watermarkEstimator, tracker); + outputTimestamp = timestampPolicy.getTimestampForRecord(context, kafkaRecord); + } else { + Preconditions.checkStateNotNull(this.extractOutputTimestampFn); + outputTimestamp = extractOutputTimestampFn.apply(kafkaRecord); + } + receiver + .get(recordTag) + .outputWithTimestamp(KV.of(kafkaSourceDescriptor, kafkaRecord), outputTimestamp); + } catch (SerializationException e) { + // This exception should only occur during the key and value deserialization when + // creating the Kafka Record + badRecordRouter.route( + receiver, + rawRecord, + null, + e, + "Failure deserializing Key or Value of Kakfa record reading from Kafka"); + if (timestampPolicy != null) { + updateWatermarkManually(timestampPolicy, watermarkEstimator, tracker); + } } } + } finally { + if (rawSizesCount > 0L) { + avgRecordSize.update(rawSizesSum, rawSizesCount); + rawSizes.update(rawSizesSum, rawSizesCount, rawSizesMin, rawSizesMax); + } } // Non-visible progress within the consumer polling timeout. @@ -703,6 +752,12 @@ public ProcessContinuation processElement( kafkaSourceDescriptor.getPartition(), estimatedBacklogBytes); } + + if (timestampPolicy != null) { + updateWatermarkManually(timestampPolicy, watermarkEstimator, tracker); + } + + return ProcessContinuation.resume(); } finally { kafkaMetrics.flushBufferedMetrics(); } @@ -734,7 +789,12 @@ public Coder restrictionCoder() { } @Setup + @EnsuresNonNull({"avgRecordSizeCache", "latestOffsetEstimatorCache", "pollConsumerCache"}) public void setup() throws Exception { + avgRecordSizeCache = avgRecordSizeCacheSupplier.get(); + latestOffsetEstimatorCache = latestOffsetEstimatorCacheSupplier.get(); + pollConsumerCache = pollConsumerCacheSupplier.get(); + keyDeserializerInstance = keyDeserializerProvider.getDeserializer(consumerConfig, true); valueDeserializerInstance = valueDeserializerProvider.getDeserializer(consumerConfig, false); if (checkStopReadingFn != null) { @@ -743,7 +803,15 @@ public void setup() throws Exception { } @Teardown + @RequiresNonNull({"avgRecordSizeCache", "latestOffsetEstimatorCache", "pollConsumerCache"}) public void teardown() throws Exception { + final LoadingCache avgRecordSizeCache = + this.avgRecordSizeCache; + final LoadingCache + latestOffsetEstimatorCache = this.latestOffsetEstimatorCache; + final LoadingCache> pollConsumerCache = + this.pollConsumerCache; + try { if (valueDeserializerInstance != null) { Closeables.close(valueDeserializerInstance, true); @@ -761,9 +829,9 @@ public void teardown() throws Exception { } // Allow the cache to perform clean up tasks when this instance is about to be deleted. - avgRecordSizeCacheSupplier.get().cleanUp(); - latestOffsetEstimatorCacheSupplier.get().cleanUp(); - pollConsumerCacheSupplier.get().cleanUp(); + avgRecordSizeCache.cleanUp(); + latestOffsetEstimatorCache.cleanUp(); + pollConsumerCache.cleanUp(); } private static Instant ensureTimestampWithinBounds(Instant timestamp) { diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java index 0633887122ba..0e8cbd2183ca 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java @@ -879,9 +879,8 @@ public void runReadWriteKafkaViaManagedSchemaTransforms( numb -> Row.withSchema(beamSchema) .withFieldValue("name", numb.toString()) - .withFieldValue( - "userId", Long.valueOf(numb.hashCode())) // User ID - .withFieldValue("age", Long.valueOf(numb.intValue())) // Age + .withFieldValue("userId", (long) numb.hashCode()) // User ID + .withFieldValue("age", (long) numb.intValue()) // Age .withFieldValue("ageIsEven", numb % 2 == 0) // ageIsEven .withFieldValue("temperature", new Random(numb).nextDouble()) .withFieldValue( diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java index fcc7b16e4672..83c2e1b38826 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java @@ -30,6 +30,7 @@ import static org.hamcrest.Matchers.matchesPattern; import static org.hamcrest.Matchers.not; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; @@ -168,6 +169,7 @@ * Tests of {@link KafkaIO}. Run with 'mvn test -Dkafka.clients.version=0.10.1.1', to test with a * specific Kafka version. */ +@SuppressWarnings("UnnecessaryLongToIntConversion") // for assert @RunWith(JUnit4.class) public class KafkaIOTest { @@ -633,7 +635,7 @@ public Long deserialize(String topic, Headers headers, byte[] data) { } @Test - public void testDeserializationWithHeaders() throws Exception { + public void testDeserializationWithHeaders() { // To assert that we continue to prefer the Deserializer API with headers in Kafka API 2.1.0 // onwards int numElements = 1000; @@ -791,6 +793,53 @@ public void testNumKeysIgnoredWithRedistributeNotEnabled() { p.run(); } + @Test + public void testDefaultRedistributeNumKeys() { + int numElements = 1000; + // Redistribute is not used and does not modify the read transform further. + KafkaIO.Read read = + mkKafkaReadTransform( + numElements, + numElements, + new ValueAsTimestampFn(), + false, /*redistribute*/ + false, /*allowDuplicates*/ + null, /*numKeys*/ + null, /*offsetDeduplication*/ + null /*topics*/); + assertFalse(read.isRedistributed()); + assertEquals(0, read.getRedistributeNumKeys()); + + // Redistribute is used and defaulted the number of keys due to no user setting. + read = + mkKafkaReadTransform( + numElements, + numElements, + new ValueAsTimestampFn(), + true, /*redistribute*/ + false, /*allowDuplicates*/ + null, /*numKeys*/ + null, /*offsetDeduplication*/ + null /*topics*/); + assertTrue(read.isRedistributed()); + // Default is defined by DEFAULT_REDISTRIBUTE_NUM_KEYS in KafkaIO. + assertEquals(32768, read.getRedistributeNumKeys()); + + // Redistribute is set with user-specified the number of keys. + read = + mkKafkaReadTransform( + numElements, + numElements, + new ValueAsTimestampFn(), + true, /*redistribute*/ + false, /*allowDuplicates*/ + 10, /*numKeys*/ + null, /*offsetDeduplication*/ + null /*topics*/); + assertTrue(read.isRedistributed()); + assertEquals(10, read.getRedistributeNumKeys()); + } + @Test public void testDisableRedistributeKafkaOffsetLegacy() { thrown.expect(Exception.class); @@ -1062,7 +1111,7 @@ public void testUnboundedSourceWithWrongTopic() { private static class ElementValueDiff extends DoFn { @ProcessElement - public void processElement(ProcessContext c) throws Exception { + public void processElement(ProcessContext c) { c.output(c.element() - c.timestamp().getMillis()); } } @@ -1604,7 +1653,7 @@ public void testUnboundedReaderLogsCommitFailure() throws Exception { } @Test - public void testSink() throws Exception { + public void testSink() { // Simply read from kafka source and write to kafka sink. Then verify the records // are correctly published to mock kafka producer. @@ -1660,7 +1709,7 @@ public void close() { } @Test - public void testSinkWithSerializationErrors() throws Exception { + public void testSinkWithSerializationErrors() { // Attempt to write 10 elements to Kafka, but they will all fail to serialize, and be sent to // the DLQ @@ -1701,7 +1750,7 @@ public void testSinkWithSerializationErrors() throws Exception { } @Test - public void testValuesSink() throws Exception { + public void testValuesSink() { // similar to testSink(), but use values()' interface. int numElements = 1000; @@ -1732,7 +1781,7 @@ public void testValuesSink() throws Exception { } @Test - public void testRecordsSink() throws Exception { + public void testRecordsSink() { // Simply read from kafka source and write to kafka sink using ProducerRecord transform. Then // verify the records are correctly published to mock kafka producer. @@ -1766,7 +1815,7 @@ public void testRecordsSink() throws Exception { } @Test - public void testSinkToMultipleTopics() throws Exception { + public void testSinkToMultipleTopics() { // Set different output topic names int numElements = 1000; @@ -1811,7 +1860,7 @@ public void testSinkToMultipleTopics() throws Exception { } @Test - public void testKafkaWriteHeaders() throws Exception { + public void testKafkaWriteHeaders() { // Set different output topic names int numElements = 1; SimpleEntry header = new SimpleEntry<>("header_key", "header_value"); @@ -1855,7 +1904,7 @@ public void testKafkaWriteHeaders() throws Exception { } @Test - public void testSinkProducerRecordsWithCustomTS() throws Exception { + public void testSinkProducerRecordsWithCustomTS() { int numElements = 1000; try (MockProducerWrapper producerWrapper = new MockProducerWrapper(new LongSerializer())) { @@ -1894,7 +1943,7 @@ public void testSinkProducerRecordsWithCustomTS() throws Exception { } @Test - public void testSinkProducerRecordsWithCustomPartition() throws Exception { + public void testSinkProducerRecordsWithCustomPartition() { int numElements = 1000; try (MockProducerWrapper producerWrapper = new MockProducerWrapper(new LongSerializer())) { @@ -2342,7 +2391,7 @@ public void testSinkDisplayData() { } @Test - public void testSinkMetrics() throws Exception { + public void testSinkMetrics() { // Simply read from kafka source and write to kafka sink. Then verify the metrics are reported. int numElements = 1000; diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProviderTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProviderTest.java index dffa6ece9d1b..b63a9334239c 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProviderTest.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaWriteSchemaTransformProviderTest.java @@ -24,9 +24,16 @@ import java.util.Collections; import java.util.List; import java.util.Objects; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.ByteArrayCoder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; +import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils; import org.apache.beam.sdk.extensions.protobuf.ProtoByteUtils; import org.apache.beam.sdk.io.kafka.KafkaWriteSchemaTransformProvider.KafkaWriteSchemaTransform.ErrorCounterFn; +import org.apache.beam.sdk.io.kafka.KafkaWriteSchemaTransformProvider.KafkaWriteSchemaTransform.GenericRecordErrorCounterFn; import org.apache.beam.sdk.managed.Managed; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.transforms.providers.ErrorHandling; @@ -53,6 +60,8 @@ public class KafkaWriteSchemaTransformProviderTest { private static final TupleTag> OUTPUT_TAG = KafkaWriteSchemaTransformProvider.OUTPUT_TAG; + private static final TupleTag> RECORD_OUTPUT_TAG = + KafkaWriteSchemaTransformProvider.RECORD_OUTPUT_TAG; private static final TupleTag ERROR_TAG = KafkaWriteSchemaTransformProvider.ERROR_TAG; private static final Schema BEAMSCHEMA = @@ -126,7 +135,8 @@ public class KafkaWriteSchemaTransformProviderTest { getClass().getResource("/proto_byte/file_descriptor/proto_byte_utils.pb")) .getPath(), "MyMessage"); - + final SerializableFunction recordValueMapper = + AvroUtils.getRowToGenericRecordFunction(AvroUtils.toAvroSchema(BEAMSCHEMA)); @Rule public transient TestPipeline p = TestPipeline.create(); @Test @@ -198,6 +208,38 @@ public void testKafkaErrorFnProtoSuccess() { + " bool active = 3;\n" + "}"; + @Test + public void testKafkaRecordErrorFnSuccess() throws Exception { + org.apache.avro.Schema avroSchema = AvroUtils.toAvroSchema(BEAMSCHEMA); + + GenericRecord record1 = new GenericData.Record(avroSchema); + GenericRecord record2 = new GenericData.Record(avroSchema); + GenericRecord record3 = new GenericData.Record(avroSchema); + record1.put("name", "a"); + record2.put("name", "b"); + record3.put("name", "c"); + + List> msg = + Arrays.asList( + KV.of(new byte[1], record1), KV.of(new byte[1], record2), KV.of(new byte[1], record3)); + + PCollection input = p.apply(Create.of(ROWS)); + Schema errorSchema = ErrorHandling.errorSchema(BEAMSCHEMA); + PCollectionTuple output = + input.apply( + ParDo.of( + new GenericRecordErrorCounterFn( + "Kafka-write-error-counter", recordValueMapper, errorSchema, true)) + .withOutputTags(RECORD_OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); + + output.get(ERROR_TAG).setRowSchema(errorSchema); + output + .get(RECORD_OUTPUT_TAG) + .setCoder(KvCoder.of(ByteArrayCoder.of(), AvroCoder.of(avroSchema))); + PAssert.that(output.get(RECORD_OUTPUT_TAG)).containsInAnyOrder(msg); + p.run().waitUntilFinish(); + } + @Test public void testBuildTransformWithManaged() { List configs = diff --git a/sdks/java/io/mongodb/build.gradle b/sdks/java/io/mongodb/build.gradle index b9e90082f0dc..56d29750dead 100644 --- a/sdks/java/io/mongodb/build.gradle +++ b/sdks/java/io/mongodb/build.gradle @@ -28,13 +28,14 @@ dependencies { implementation project(path: ":sdks:java:core", configuration: "shadow") implementation library.java.joda_time implementation library.java.mongo_java_driver + implementation library.java.mongo_bson + implementation library.java.mongodb_driver_core implementation library.java.slf4j_api implementation library.java.vendored_guava_32_1_2_jre testImplementation library.java.junit testImplementation project(path: ":sdks:java:io:common") testImplementation project(path: ":sdks:java:testing:test-utils") - testImplementation "de.flapdoodle.embed:de.flapdoodle.embed.mongo:3.0.0" - testImplementation "de.flapdoodle.embed:de.flapdoodle.embed.process:3.0.0" + testImplementation "de.flapdoodle.embed:de.flapdoodle.embed.mongo:3.5.4" testRuntimeOnly library.java.slf4j_jdk14 testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") } diff --git a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/FindQuery.java b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/FindQuery.java index 2131656d458a..d89db9dea54b 100644 --- a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/FindQuery.java +++ b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/FindQuery.java @@ -21,7 +21,7 @@ import com.google.auto.value.AutoValue; import com.mongodb.BasicDBObject; -import com.mongodb.MongoClient; +import com.mongodb.MongoClientSettings; import com.mongodb.client.MongoCollection; import com.mongodb.client.MongoCursor; import com.mongodb.client.model.Projections; @@ -79,7 +79,8 @@ private FindQuery withFilters(BsonDocument filters) { /** Convert the Bson filters into a BsonDocument via default encoding. */ static BsonDocument bson2BsonDocument(Bson filters) { - return filters.toBsonDocument(BasicDBObject.class, MongoClient.getDefaultCodecRegistry()); + return filters.toBsonDocument( + BasicDBObject.class, MongoClientSettings.getDefaultCodecRegistry()); } /** Sets the filters to find. */ diff --git a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbGridFSIO.java b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbGridFSIO.java index 07cc238c7e6b..71f8b291e0d5 100644 --- a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbGridFSIO.java +++ b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbGridFSIO.java @@ -21,15 +21,18 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; import com.google.auto.value.AutoValue; -import com.mongodb.DB; -import com.mongodb.DBCursor; -import com.mongodb.DBObject; -import com.mongodb.MongoClient; -import com.mongodb.MongoClientURI; -import com.mongodb.gridfs.GridFS; -import com.mongodb.gridfs.GridFSDBFile; -import com.mongodb.gridfs.GridFSInputFile; -import com.mongodb.util.JSON; +import com.mongodb.ConnectionString; +import com.mongodb.MongoClientSettings; +import com.mongodb.client.MongoClient; +import com.mongodb.client.MongoClients; +import com.mongodb.client.MongoCursor; +import com.mongodb.client.MongoDatabase; +import com.mongodb.client.gridfs.GridFSBucket; +import com.mongodb.client.gridfs.GridFSBuckets; +import com.mongodb.client.gridfs.GridFSDownloadStream; +import com.mongodb.client.gridfs.GridFSUploadStream; +import com.mongodb.client.gridfs.model.GridFSFile; +import com.mongodb.client.gridfs.model.GridFSUploadOptions; import java.io.BufferedReader; import java.io.IOException; import java.io.InputStreamReader; @@ -53,6 +56,7 @@ import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; +import org.bson.Document; import org.bson.types.ObjectId; import org.checkerframework.checker.nullness.qual.Nullable; import org.checkerframework.dataflow.qual.Pure; @@ -117,16 +121,18 @@ public class MongoDbGridFSIO { /** Callback for the parser to use to submit data. */ public interface ParserCallback extends Serializable { - /** Output the object. The default timestamp will be the GridFSDBFile creation timestamp. */ + /** Output the object. The default timestamp will be the GridFSFile creation timestamp. */ void output(T output); /** Output the object using the specified timestamp. */ void output(T output, Instant timestamp); } - /** Interface for the parser that is used to parse the GridFSDBFile into the appropriate types. */ + /** Interface for the parser that is used to parse the GridFSFile into the appropriate types. */ public interface Parser extends Serializable { - void parse(GridFSDBFile input, ParserCallback callback) throws IOException; + void parse( + GridFSFile gridFSFile, GridFSDownloadStream downloadStream, ParserCallback callback) + throws IOException; } /** @@ -134,11 +140,10 @@ public interface Parser extends Serializable { * file into Strings. It uses the timestamp of the file for the event timestamp. */ private static final Parser TEXT_PARSER = - (input, callback) -> { - final Instant time = new Instant(input.getUploadDate().getTime()); + (gridFSFile, downloadStream, callback) -> { + final Instant time = new Instant(gridFSFile.getUploadDate().getTime()); try (BufferedReader reader = - new BufferedReader( - new InputStreamReader(input.getInputStream(), StandardCharsets.UTF_8))) { + new BufferedReader(new InputStreamReader(downloadStream, StandardCharsets.UTF_8))) { for (String line = reader.readLine(); line != null; line = reader.readLine()) { callback.output(line, time); } @@ -197,12 +202,20 @@ static ConnectionConfiguration create( } MongoClient setupMongo() { - return uri() == null ? new MongoClient() : new MongoClient(new MongoClientURI(uri())); + if (uri() == null) { + return MongoClients.create(); + } + MongoClientSettings settings = + MongoClientSettings.builder() + .applyConnectionString(new ConnectionString(Preconditions.checkStateNotNull(uri()))) + .build(); + return MongoClients.create(settings); } - GridFS setupGridFS(MongoClient mongo) { - DB db = database() == null ? mongo.getDB("gridfs") : mongo.getDB(database()); - return bucket() == null ? new GridFS(db) : new GridFS(db, bucket()); + GridFSBucket setupGridFS(MongoClient mongo) { + MongoDatabase db = + database() == null ? mongo.getDatabase("gridfs") : mongo.getDatabase(database()); + return bucket() == null ? GridFSBuckets.create(db) : GridFSBuckets.create(db, bucket()); } } @@ -313,12 +326,12 @@ public PCollection expand(PBegin input) { ParDo.of( new DoFn() { @Nullable MongoClient mongo; - @Nullable GridFS gridfs; + @Nullable GridFSBucket gridFSBucket; @Setup public void setup() { mongo = source.spec.connectionConfiguration().setupMongo(); - gridfs = source.spec.connectionConfiguration().setupGridFS(mongo); + gridFSBucket = source.spec.connectionConfiguration().setupGridFS(mongo); } @Teardown @@ -331,12 +344,18 @@ public void teardown() { @ProcessElement public void processElement(final ProcessContext c) throws IOException { - Preconditions.checkStateNotNull(gridfs); + GridFSBucket bucket = Preconditions.checkStateNotNull(gridFSBucket); ObjectId oid = c.element(); - GridFSDBFile file = gridfs.find(oid); + GridFSDownloadStream downloadStream = bucket.openDownloadStream(oid); + GridFSFile gridFSFile = + bucket.find(com.mongodb.client.model.Filters.eq("_id", oid)).first(); + if (gridFSFile == null) { + return; // Skip if file not found + } Parser parser = Preconditions.checkStateNotNull(parser()); parser.parse( - file, + gridFSFile, + downloadStream, new ParserCallback() { @Override public void output(T output, Instant timestamp) { @@ -378,12 +397,12 @@ protected static class BoundedGridFSSource extends BoundedSource { this.objectIds = objectIds; } - private DBCursor createCursor(GridFS gridfs) { + private MongoCursor createCursor(GridFSBucket gridFSBucket) { if (spec.filter() != null) { - DBObject query = (DBObject) JSON.parse(spec.filter()); - return gridfs.getFileList(query); + Document query = Document.parse(spec.filter()); + return gridFSBucket.find(query).iterator(); } - return gridfs.getFileList(); + return gridFSBucket.find().iterator(); } @Override @@ -391,20 +410,20 @@ public List> split( long desiredBundleSizeBytes, PipelineOptions options) throws Exception { MongoClient mongo = spec.connectionConfiguration().setupMongo(); try { - GridFS gridfs = spec.connectionConfiguration().setupGridFS(mongo); - DBCursor cursor = createCursor(gridfs); + GridFSBucket gridFSBucket = spec.connectionConfiguration().setupGridFS(mongo); + MongoCursor cursor = createCursor(gridFSBucket); long size = 0; List list = new ArrayList<>(); List objects = new ArrayList<>(); while (cursor.hasNext()) { - GridFSDBFile file = (GridFSDBFile) cursor.next(); + GridFSFile file = cursor.next(); long len = file.getLength(); if ((size + len) > desiredBundleSizeBytes && !objects.isEmpty()) { list.add(new BoundedGridFSSource(spec, objects)); size = 0; objects = new ArrayList<>(); } - objects.add((ObjectId) file.getId()); + objects.add(file.getObjectId()); size += len; } if (!objects.isEmpty() || list.isEmpty()) { @@ -419,10 +438,11 @@ public List> split( @Override public long getEstimatedSizeBytes(PipelineOptions options) throws Exception { try (MongoClient mongo = spec.connectionConfiguration().setupMongo(); - DBCursor cursor = createCursor(spec.connectionConfiguration().setupGridFS(mongo))) { + MongoCursor cursor = + createCursor(spec.connectionConfiguration().setupGridFS(mongo))) { long size = 0; while (cursor.hasNext()) { - GridFSDBFile file = (GridFSDBFile) cursor.next(); + GridFSFile file = cursor.next(); size += file.getLength(); } return size; @@ -456,7 +476,7 @@ static class GridFSReader extends BoundedSource.BoundedReader { final @Nullable List objects; @Nullable MongoClient mongo; - @Nullable DBCursor cursor; + @Nullable MongoCursor cursor; @Nullable Iterator iterator; @Nullable ObjectId current; @@ -474,8 +494,8 @@ public BoundedSource getCurrentSource() { public boolean start() throws IOException { if (objects == null) { mongo = source.spec.connectionConfiguration().setupMongo(); - GridFS gridfs = source.spec.connectionConfiguration().setupGridFS(mongo); - cursor = source.createCursor(gridfs); + GridFSBucket gridFSBucket = source.spec.connectionConfiguration().setupGridFS(mongo); + cursor = source.createCursor(gridFSBucket); } else { iterator = objects.iterator(); } @@ -488,8 +508,8 @@ public boolean advance() throws IOException { current = iterator.next(); return true; } else if (cursor != null && cursor.hasNext()) { - GridFSDBFile file = (GridFSDBFile) cursor.next(); - current = (ObjectId) file.getId(); + GridFSFile file = cursor.next(); + current = file.getObjectId(); return true; } current = null; @@ -628,9 +648,9 @@ private static class GridFsWriteFn extends DoFn { private final Write spec; private transient @Nullable MongoClient mongo; - private transient @Nullable GridFS gridfs; + private transient @Nullable GridFSBucket gridFSBucket; - private transient @Nullable GridFSInputFile gridFsFile; + private transient @Nullable GridFSUploadStream gridFsUploadStream; private transient @Nullable OutputStream outputStream; public GridFsWriteFn(Write spec) { @@ -640,20 +660,22 @@ public GridFsWriteFn(Write spec) { @Setup public void setup() throws Exception { mongo = spec.connectionConfiguration().setupMongo(); - gridfs = spec.connectionConfiguration().setupGridFS(mongo); + gridFSBucket = spec.connectionConfiguration().setupGridFS(mongo); } @StartBundle public void startBundle() { - GridFS gridfs = Preconditions.checkStateNotNull(this.gridfs); + GridFSBucket gridFSBucket = Preconditions.checkStateNotNull(this.gridFSBucket); String filename = Preconditions.checkStateNotNull(spec.filename()); - GridFSInputFile gridFsFile = gridfs.createFile(filename); + if (spec.chunkSize() != null) { - gridFsFile.setChunkSize(spec.chunkSize()); + gridFsUploadStream = + gridFSBucket.openUploadStream( + filename, new GridFSUploadOptions().chunkSizeBytes(spec.chunkSize().intValue())); + } else { + gridFsUploadStream = gridFSBucket.openUploadStream(filename); } - outputStream = gridFsFile.getOutputStream(); - - this.gridFsFile = gridFsFile; + outputStream = gridFsUploadStream; } @ProcessElement @@ -665,35 +687,20 @@ public void processElement(ProcessContext context) throws Exception { @FinishBundle public void finishBundle() throws Exception { - if (outputStream != null) { - OutputStream outputStream = this.outputStream; - outputStream.flush(); - outputStream.close(); - this.outputStream = null; - } - if (gridFsFile != null) { - gridFsFile = null; + GridFSUploadStream uploadStream = gridFsUploadStream; + if (uploadStream != null) { + uploadStream.flush(); + uploadStream.close(); + gridFsUploadStream = null; + outputStream = null; } } @Teardown public void teardown() throws Exception { - try { - if (outputStream != null) { - OutputStream outputStream = this.outputStream; - outputStream.flush(); - outputStream.close(); - this.outputStream = null; - } - if (gridFsFile != null) { - gridFsFile = null; - } - } finally { - if (mongo != null) { - mongo.close(); - mongo = null; - gridfs = null; - } + if (mongo != null) { + mongo.close(); + mongo = null; } } } diff --git a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java index 905c7418e26c..1283e873f2b6 100644 --- a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java +++ b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java @@ -22,12 +22,14 @@ import com.google.auto.value.AutoValue; import com.mongodb.BasicDBObject; +import com.mongodb.ConnectionString; import com.mongodb.MongoBulkWriteException; -import com.mongodb.MongoClient; -import com.mongodb.MongoClientOptions; -import com.mongodb.MongoClientURI; +import com.mongodb.MongoClientSettings; +import com.mongodb.MongoClientSettings.Builder; import com.mongodb.MongoCommandException; import com.mongodb.client.AggregateIterable; +import com.mongodb.client.MongoClient; +import com.mongodb.client.MongoClients; import com.mongodb.client.MongoCollection; import com.mongodb.client.MongoCursor; import com.mongodb.client.MongoDatabase; @@ -46,6 +48,7 @@ import java.util.Map; import java.util.NoSuchElementException; import java.util.Optional; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import javax.net.ssl.SSLContext; import org.apache.beam.sdk.coders.Coder; @@ -64,6 +67,7 @@ import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.bson.BsonDocument; import org.bson.BsonInt32; +import org.bson.BsonObjectId; import org.bson.BsonString; import org.bson.Document; import org.bson.conversions.Bson; @@ -362,22 +366,25 @@ public void populateDisplayData(DisplayData.Builder builder) { } } - private static MongoClientOptions.Builder getOptions( + private static MongoClientSettings.Builder getOptions( int maxConnectionIdleTime, boolean sslEnabled, boolean sslInvalidHostNameAllowed, boolean ignoreSSLCertificate) { - MongoClientOptions.Builder optionsBuilder = new MongoClientOptions.Builder(); - optionsBuilder.maxConnectionIdleTime(maxConnectionIdleTime); + MongoClientSettings.Builder settingsBuilder = MongoClientSettings.builder(); + settingsBuilder.applyToConnectionPoolSettings( + builder -> builder.maxConnectionIdleTime(maxConnectionIdleTime, TimeUnit.MILLISECONDS)); if (sslEnabled) { - optionsBuilder.sslEnabled(sslEnabled).sslInvalidHostNameAllowed(sslInvalidHostNameAllowed); - if (ignoreSSLCertificate) { - SSLContext sslContext = SSLUtils.ignoreSSLCertificate(); - optionsBuilder.sslContext(sslContext); - optionsBuilder.socketFactory(sslContext.getSocketFactory()); - } + settingsBuilder.applyToSslSettings( + builder -> { + builder.enabled(sslEnabled).invalidHostNameAllowed(sslInvalidHostNameAllowed); + if (ignoreSSLCertificate) { + SSLContext sslContext = SSLUtils.ignoreSSLCertificate(); + builder.context(sslContext); + } + }); } - return optionsBuilder; + return settingsBuilder; } /** A MongoDB {@link BoundedSource} reading {@link Document} from a given instance. */ @@ -414,15 +421,15 @@ long getDocumentCount() { String uri = Preconditions.checkStateNotNull(spec.uri()); String database = Preconditions.checkStateNotNull(spec.database()); String collection = Preconditions.checkStateNotNull(spec.collection()); - try (MongoClient mongoClient = - new MongoClient( - new MongoClientURI( - uri, - getOptions( - spec.maxConnectionIdleTime(), - spec.sslEnabled(), - spec.sslInvalidHostNameAllowed(), - spec.ignoreSSLCertificate())))) { + MongoClientSettings settings = + getOptions( + spec.maxConnectionIdleTime(), + spec.sslEnabled(), + spec.sslInvalidHostNameAllowed(), + spec.ignoreSSLCertificate()) + .applyConnectionString(new ConnectionString(uri)) + .build(); + try (MongoClient mongoClient = MongoClients.create(settings)) { return getDocumentCount(mongoClient, database, collection); } catch (Exception e) { return -1; @@ -446,15 +453,15 @@ public long getEstimatedSizeBytes(PipelineOptions pipelineOptions) { String uri = Preconditions.checkStateNotNull(spec.uri()); String database = Preconditions.checkStateNotNull(spec.database()); String collection = Preconditions.checkStateNotNull(spec.collection()); - try (MongoClient mongoClient = - new MongoClient( - new MongoClientURI( - uri, - getOptions( - spec.maxConnectionIdleTime(), - spec.sslEnabled(), - spec.sslInvalidHostNameAllowed(), - spec.ignoreSSLCertificate())))) { + MongoClientSettings settings = + getOptions( + spec.maxConnectionIdleTime(), + spec.sslEnabled(), + spec.sslInvalidHostNameAllowed(), + spec.ignoreSSLCertificate()) + .applyConnectionString(new ConnectionString(uri)) + .build(); + try (MongoClient mongoClient = MongoClients.create(settings)) { try { return getEstimatedSizeBytes(mongoClient, database, collection); } catch (MongoCommandException exception) { @@ -483,15 +490,15 @@ public List> split( String uri = Preconditions.checkStateNotNull(spec.uri()); String database = Preconditions.checkStateNotNull(spec.database()); String collection = Preconditions.checkStateNotNull(spec.collection()); - try (MongoClient mongoClient = - new MongoClient( - new MongoClientURI( - uri, - getOptions( - spec.maxConnectionIdleTime(), - spec.sslEnabled(), - spec.sslInvalidHostNameAllowed(), - spec.ignoreSSLCertificate())))) { + MongoClientSettings settings = + getOptions( + spec.maxConnectionIdleTime(), + spec.sslEnabled(), + spec.sslInvalidHostNameAllowed(), + spec.ignoreSSLCertificate()) + .applyConnectionString(new ConnectionString(uri)) + .build(); + try (MongoClient mongoClient = MongoClients.create(settings)) { MongoDatabase mongoDatabase = mongoClient.getDatabase(database); List splitKeys; @@ -671,26 +678,39 @@ static List splitKeysToMatch(List splitKeys) { if (i == 0) { aggregates.add(Aggregates.match(Filters.lte("_id", splitKey))); if (splitKeys.size() == 1) { - aggregates.add(Aggregates.match(Filters.and(Filters.gt("_id", splitKey)))); + aggregates.add(Aggregates.match(Filters.gt("_id", splitKey))); } } else if (i == splitKeys.size() - 1) { // this is the last split in the list, the filters define // the range from the previous split to the current split and also // the current split to the end - aggregates.add( - Aggregates.match( - Filters.and(Filters.gt("_id", lowestBound), Filters.lte("_id", splitKey)))); - aggregates.add(Aggregates.match(Filters.and(Filters.gt("_id", splitKey)))); + // Create a custom BSON document with multiple conditions on the same field + BsonDocument rangeFilter = + new BsonDocument( + "_id", + new BsonDocument( + "$gt", new BsonObjectId(Preconditions.checkStateNotNull(lowestBound))) + .append("$lte", new BsonObjectId(splitKey))); + aggregates.add(Aggregates.match(rangeFilter)); + aggregates.add(Aggregates.match(Filters.gt("_id", splitKey))); } else { - aggregates.add( - Aggregates.match( - Filters.and(Filters.gt("_id", lowestBound), Filters.lte("_id", splitKey)))); + // Create a custom BSON document with multiple conditions on the same field + BsonDocument rangeFilter = + new BsonDocument( + "_id", + new BsonDocument( + "$gt", new BsonObjectId(Preconditions.checkStateNotNull(lowestBound))) + .append("$lte", new BsonObjectId(splitKey))); + aggregates.add(Aggregates.match(rangeFilter)); } lowestBound = splitKey; } return aggregates.stream() - .map(s -> s.toBsonDocument(BasicDBObject.class, MongoClient.getDefaultCodecRegistry())) + .map( + s -> + s.toBsonDocument( + BasicDBObject.class, MongoClientSettings.getDefaultCodecRegistry())) .collect(Collectors.toList()); } @@ -786,14 +806,15 @@ public void close() { private MongoClient createClient(Read spec) { String uri = Preconditions.checkStateNotNull(spec.uri(), "withUri() is required"); - return new MongoClient( - new MongoClientURI( - uri, - getOptions( + MongoClientSettings settings = + getOptions( spec.maxConnectionIdleTime(), spec.sslEnabled(), spec.sslInvalidHostNameAllowed(), - spec.ignoreSSLCertificate()))); + spec.ignoreSSLCertificate()) + .applyConnectionString(new ConnectionString(uri)) + .build(); + return MongoClients.create(settings); } } @@ -985,15 +1006,15 @@ static class WriteFn extends DoFn { @Setup public void createMongoClient() { String uri = Preconditions.checkStateNotNull(spec.uri()); - client = - new MongoClient( - new MongoClientURI( - uri, - getOptions( - spec.maxConnectionIdleTime(), - spec.sslEnabled(), - spec.sslInvalidHostNameAllowed(), - spec.ignoreSSLCertificate()))); + MongoClientSettings settings = + getOptions( + spec.maxConnectionIdleTime(), + spec.sslEnabled(), + spec.sslInvalidHostNameAllowed(), + spec.ignoreSSLCertificate()) + .applyConnectionString(new ConnectionString(uri)) + .build(); + client = MongoClients.create(settings); } @StartBundle diff --git a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/FindQueryTest.java b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/FindQueryTest.java index df66179f3904..da90f92dc190 100644 --- a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/FindQueryTest.java +++ b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/FindQueryTest.java @@ -21,7 +21,7 @@ import com.google.auto.value.AutoValue; import com.mongodb.BasicDBObject; -import com.mongodb.MongoClient; +import com.mongodb.MongoClientSettings; import com.mongodb.client.MongoCollection; import com.mongodb.client.MongoCursor; import com.mongodb.client.model.Projections; @@ -79,7 +79,8 @@ private FindQueryTest withFilters(BsonDocument filters) { /** Convert the Bson filters into a BsonDocument via default encoding. */ static BsonDocument bson2BsonDocument(Bson filters) { - return filters.toBsonDocument(BasicDBObject.class, MongoClient.getDefaultCodecRegistry()); + return filters.toBsonDocument( + BasicDBObject.class, MongoClientSettings.getDefaultCodecRegistry()); } /** Sets the filters to find. */ diff --git a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBGridFSIOTest.java b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBGridFSIOTest.java index 09343606f228..d13185a08fb6 100644 --- a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBGridFSIOTest.java +++ b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBGridFSIOTest.java @@ -20,11 +20,13 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import com.mongodb.DB; -import com.mongodb.MongoClient; -import com.mongodb.gridfs.GridFS; -import com.mongodb.gridfs.GridFSDBFile; -import com.mongodb.gridfs.GridFSInputFile; +import com.mongodb.client.MongoClient; +import com.mongodb.client.MongoClients; +import com.mongodb.client.MongoDatabase; +import com.mongodb.client.gridfs.GridFSBucket; +import com.mongodb.client.gridfs.GridFSBuckets; +import com.mongodb.client.gridfs.GridFSUploadStream; +import com.mongodb.client.gridfs.model.GridFSFile; import de.flapdoodle.embed.mongo.MongodExecutable; import de.flapdoodle.embed.mongo.MongodProcess; import de.flapdoodle.embed.mongo.MongodStarter; @@ -35,12 +37,10 @@ import de.flapdoodle.embed.mongo.distribution.Version; import de.flapdoodle.embed.process.runtime.Network; import java.io.BufferedReader; -import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.DataInputStream; import java.io.InputStream; import java.io.InputStreamReader; -import java.io.OutputStream; import java.io.OutputStreamWriter; import java.nio.charset.StandardCharsets; import java.util.ArrayList; @@ -117,9 +117,9 @@ public static void start() throws Exception { LOG.info("Insert test data"); - MongoClient client = new MongoClient("localhost", port); - DB database = client.getDB(DATABASE); - GridFS gridfs = new GridFS(database); + MongoClient client = MongoClients.create("mongodb://localhost:" + port); + MongoDatabase database = client.getDatabase(DATABASE); + GridFSBucket gridfs = GridFSBuckets.create(database); ByteArrayOutputStream out = new ByteArrayOutputStream(); for (int x = 0; x < 100; x++) { @@ -129,10 +129,12 @@ public static void start() throws Exception { .getBytes(StandardCharsets.UTF_8)); } for (int x = 0; x < 5; x++) { - gridfs.createFile(new ByteArrayInputStream(out.toByteArray()), "file" + x).save(); + try (GridFSUploadStream uploadStream = gridfs.openUploadStream("file" + x)) { + uploadStream.write(out.toByteArray()); + } } - gridfs = new GridFS(database, "mapBucket"); + GridFSBucket mapBucketGridfs = GridFSBuckets.create(database, "mapBucket"); long now = System.currentTimeMillis(); Random random = new Random(); String[] scientists = { @@ -148,26 +150,25 @@ public static void start() throws Exception { "Maxwell" }; for (int x = 0; x < 10; x++) { - GridFSInputFile file = gridfs.createFile("file_" + x); - OutputStream outf = file.getOutputStream(); - OutputStreamWriter writer = new OutputStreamWriter(outf, StandardCharsets.UTF_8); - for (int y = 0; y < 5000; y++) { - long time = now - random.nextInt(3600000); - String name = scientists[y % scientists.length]; - writer.write(time + "\t"); - writer.write(name + "\t"); - writer.write(Integer.toString(random.nextInt(100))); - writer.write("\n"); - } - for (int y = 0; y < scientists.length; y++) { - String name = scientists[y % scientists.length]; - writer.write(now + "\t"); - writer.write(name + "\t"); - writer.write("101"); - writer.write("\n"); + try (GridFSUploadStream uploadStream = mapBucketGridfs.openUploadStream("file_" + x)) { + OutputStreamWriter writer = new OutputStreamWriter(uploadStream, StandardCharsets.UTF_8); + for (int y = 0; y < 5000; y++) { + long time = now - random.nextInt(3600000); + String name = scientists[y % scientists.length]; + writer.write(time + "\t"); + writer.write(name + "\t"); + writer.write(Integer.toString(random.nextInt(100))); + writer.write("\n"); + } + for (int y = 0; y < scientists.length; y++) { + String name = scientists[y % scientists.length]; + writer.write(now + "\t"); + writer.write(name + "\t"); + writer.write("101"); + writer.write("\n"); + } + writer.flush(); } - writer.flush(); - writer.close(); } client.close(); } @@ -208,11 +209,10 @@ public void testReadWithParser() { .withDatabase(DATABASE) .withBucket("mapBucket") .>withParser( - (input, callback) -> { + (gridFSFile, downloadStream, callback) -> { try (final BufferedReader reader = new BufferedReader( - new InputStreamReader( - input.getInputStream(), StandardCharsets.UTF_8))) { + new InputStreamReader(downloadStream, StandardCharsets.UTF_8))) { String line = reader.readLine(); while (line != null) { try (Scanner scanner = new Scanner(line.trim())) { @@ -311,19 +311,20 @@ public void testWriteMessage() throws Exception { MongoClient client = null; try { StringBuilder results = new StringBuilder(); - client = new MongoClient("localhost", port); - DB database = client.getDB(DATABASE); - GridFS gridfs = new GridFS(database, "WriteTest"); - List files = gridfs.find("WriteTestData"); - assertTrue(files.size() > 0); - for (GridFSDBFile file : files) { - assertEquals(100, file.getChunkSize()); - int l = (int) file.getLength(); - try (InputStream ins = file.getInputStream()) { - DataInputStream dis = new DataInputStream(ins); - byte[] b = new byte[l]; - dis.readFully(b); - results.append(new String(b, StandardCharsets.UTF_8)); + client = MongoClients.create("mongodb://localhost:" + port); + MongoDatabase database = client.getDatabase(DATABASE); + GridFSBucket gridfs = GridFSBuckets.create(database, "WriteTest"); + + for (GridFSFile file : gridfs.find()) { + if (file.getFilename().equals("WriteTestData")) { + assertEquals(100, file.getChunkSize()); + int l = (int) file.getLength(); + try (InputStream ins = gridfs.openDownloadStream(file.getObjectId())) { + DataInputStream dis = new DataInputStream(ins); + byte[] b = new byte[l]; + dis.readFully(b); + results.append(new String(b, StandardCharsets.UTF_8)); + } } } String dataString = results.toString(); @@ -331,16 +332,17 @@ public void testWriteMessage() throws Exception { assertTrue(dataString.contains("Message " + x)); } - files = gridfs.find("WriteTestIntData"); boolean[] intResults = new boolean[100]; - for (GridFSDBFile file : files) { - int l = (int) file.getLength(); - try (InputStream ins = file.getInputStream()) { - DataInputStream dis = new DataInputStream(ins); - byte[] b = new byte[l]; - dis.readFully(b); - for (byte aB : b) { - intResults[aB] = true; + for (GridFSFile file : gridfs.find()) { + if (file.getFilename().equals("WriteTestIntData")) { + int l = (int) file.getLength(); + try (InputStream ins = gridfs.openDownloadStream(file.getObjectId())) { + DataInputStream dis = new DataInputStream(ins); + byte[] b = new byte[l]; + dis.readFully(b); + for (byte aB : b) { + intResults[aB] = true; + } } } } diff --git a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java index 4dda988e355c..cc85db937975 100644 --- a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java +++ b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java @@ -21,7 +21,8 @@ import static org.hamcrest.Matchers.greaterThan; import static org.junit.Assert.assertEquals; -import com.mongodb.MongoClient; +import com.mongodb.client.MongoClient; +import com.mongodb.client.MongoClients; import com.mongodb.client.MongoCollection; import com.mongodb.client.MongoDatabase; import com.mongodb.client.model.Filters; @@ -107,7 +108,7 @@ public static void beforeClass() throws Exception { .build(); mongodExecutable = mongodStarter.prepare(mongodConfig); mongodProcess = mongodExecutable.start(); - client = new MongoClient("localhost", port); + client = MongoClients.create("mongodb://localhost:" + port); database = client.getDatabase(DATABASE_NAME); LOG.info("Insert test data"); diff --git a/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttIO.java b/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttIO.java index efc51362d06a..78876eb6534d 100644 --- a/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttIO.java +++ b/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttIO.java @@ -422,20 +422,16 @@ public void populateDisplayData(DisplayData.Builder builder) { static class MqttCheckpointMark implements UnboundedSource.CheckpointMark, Serializable { @VisibleForTesting String clientId; - @VisibleForTesting Instant oldestMessageTimestamp = Instant.now(); @VisibleForTesting transient List messages = new ArrayList<>(); - public MqttCheckpointMark() {} - - public MqttCheckpointMark(String id) { - clientId = id; + public MqttCheckpointMark(String id, List messages) { + this.clientId = id; + this.messages = messages; } - public void add(Message message, Instant timestamp) { - if (timestamp.isBefore(oldestMessageTimestamp)) { - oldestMessageTimestamp = timestamp; - } - messages.add(message); + @VisibleForTesting + MqttCheckpointMark(String id) { + this.clientId = id; } @Override @@ -448,7 +444,6 @@ public void finalizeCheckpoint() { LOG.warn("Can't ack message for client ID {}", clientId, e); } } - oldestMessageTimestamp = Instant.now(); messages.clear(); } @@ -464,7 +459,6 @@ public boolean equals(@Nullable Object other) { if (other instanceof MqttCheckpointMark) { MqttCheckpointMark that = (MqttCheckpointMark) other; return Objects.equals(this.clientId, that.clientId) - && Objects.equals(this.oldestMessageTimestamp, that.oldestMessageTimestamp) && Objects.deepEquals(this.messages, that.messages); } else { return false; @@ -473,7 +467,38 @@ public boolean equals(@Nullable Object other) { @Override public int hashCode() { - return Objects.hash(clientId, oldestMessageTimestamp, messages); + return Objects.hash(clientId, messages); + } + + static class Preparer { + @VisibleForTesting String clientId; + @VisibleForTesting Instant oldestMessageTimestamp = Instant.now(); + @VisibleForTesting transient List messages = new ArrayList<>(); + + public Preparer(MqttCheckpointMark checkpointMark) { + clientId = checkpointMark.clientId; + messages = checkpointMark.messages; + } + + public Preparer(String id) { + clientId = id; + } + + public Preparer() {} + + public void add(Message message, Instant timestamp) { + if (timestamp.isBefore(oldestMessageTimestamp)) { + oldestMessageTimestamp = timestamp; + } + messages.add(message); + } + + MqttCheckpointMark newCheckpoint() { + List currentMessages = messages; + messages = new ArrayList<>(); + oldestMessageTimestamp = Instant.now(); + return new MqttCheckpointMark(clientId, currentMessages); + } } } @@ -489,16 +514,20 @@ public UnboundedMqttSource(Read spec) { @Override @SuppressWarnings("unchecked") public UnboundedReader createReader( - PipelineOptions options, MqttCheckpointMark checkpointMark) { + PipelineOptions options, @Nullable MqttCheckpointMark checkpointMark) { final UnboundedMqttReader unboundedMqttReader; + MqttCheckpointMark.Preparer preparer = + checkpointMark == null + ? new MqttCheckpointMark.Preparer() + : new MqttCheckpointMark.Preparer(checkpointMark); if (spec.withMetadata()) { unboundedMqttReader = new UnboundedMqttReader<>( this, - checkpointMark, + preparer, message -> (T) MqttRecord.of(message.getTopic(), message.getPayload())); } else { - unboundedMqttReader = new UnboundedMqttReader<>(this, checkpointMark); + unboundedMqttReader = new UnboundedMqttReader<>(this, preparer); } return unboundedMqttReader; @@ -538,25 +567,26 @@ static class UnboundedMqttReader extends UnboundedSource.UnboundedReader { private BlockingConnection connection; private T current; private Instant currentTimestamp; - private MqttCheckpointMark checkpointMark; + private final MqttCheckpointMark.Preparer checkpointPreparer; private SerializableFunction extractFn; - public UnboundedMqttReader(UnboundedMqttSource source, MqttCheckpointMark checkpointMark) { + public UnboundedMqttReader( + UnboundedMqttSource source, MqttCheckpointMark.Preparer checkpointPreparer) { this.source = source; this.current = null; - if (checkpointMark != null) { - this.checkpointMark = checkpointMark; + if (checkpointPreparer != null) { + this.checkpointPreparer = checkpointPreparer; } else { - this.checkpointMark = new MqttCheckpointMark(); + this.checkpointPreparer = new MqttCheckpointMark.Preparer(); } this.extractFn = message -> (T) message.getPayload(); } public UnboundedMqttReader( UnboundedMqttSource source, - MqttCheckpointMark checkpointMark, + MqttCheckpointMark.Preparer checkpointPreparer, SerializableFunction extractFn) { - this(source, checkpointMark); + this(source, checkpointPreparer); this.extractFn = extractFn; } @@ -567,7 +597,7 @@ public boolean start() throws IOException { try { client = spec.connectionConfiguration().createClient(); LOG.debug("Reader client ID is {}", client.getClientId()); - checkpointMark.clientId = client.getClientId().toString(); + checkpointPreparer.clientId = client.getClientId().toString(); connection = createConnection(client); connection.subscribe( new Topic[] {new Topic(spec.connectionConfiguration().getTopic(), QoS.AT_LEAST_ONCE)}); @@ -587,7 +617,7 @@ public boolean advance() throws IOException { } current = this.extractFn.apply(message); currentTimestamp = Instant.now(); - checkpointMark.add(message, currentTimestamp); + checkpointPreparer.add(message, currentTimestamp); } catch (Exception e) { throw new IOException(e); } @@ -608,12 +638,12 @@ public void close() throws IOException { @Override public Instant getWatermark() { - return checkpointMark.oldestMessageTimestamp; + return checkpointPreparer.oldestMessageTimestamp; } @Override public UnboundedSource.CheckpointMark getCheckpointMark() { - return checkpointMark; + return checkpointPreparer.newCheckpoint(); } @Override diff --git a/sdks/java/io/mqtt/src/test/java/org/apache/beam/sdk/io/mqtt/MqttIOTest.java b/sdks/java/io/mqtt/src/test/java/org/apache/beam/sdk/io/mqtt/MqttIOTest.java index f0b4fab39535..754c88f0c6a4 100644 --- a/sdks/java/io/mqtt/src/test/java/org/apache/beam/sdk/io/mqtt/MqttIOTest.java +++ b/sdks/java/io/mqtt/src/test/java/org/apache/beam/sdk/io/mqtt/MqttIOTest.java @@ -27,6 +27,7 @@ import java.io.ObjectOutputStream; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.List; import java.util.Map; @@ -50,11 +51,13 @@ import org.apache.beam.sdk.values.PCollection; import org.fusesource.hawtbuf.Buffer; import org.fusesource.mqtt.client.BlockingConnection; +import org.fusesource.mqtt.client.Callback; import org.fusesource.mqtt.client.MQTT; import org.fusesource.mqtt.client.Message; import org.fusesource.mqtt.client.QoS; import org.fusesource.mqtt.client.Topic; import org.joda.time.Duration; +import org.joda.time.Instant; import org.junit.After; import org.junit.Before; import org.junit.Ignore; @@ -286,6 +289,61 @@ public void testReceiveWithTimeoutAndNoData() throws Exception { pipeline.run(); } + private static class FakeMessage extends Message { + + private int ackCount; + + public FakeMessage() { + super(null, null, null, null); + this.ackCount = 0; + } + + @Override + public void ack() { + ++ackCount; + } + + @Override + public void ack(final Callback unused) { + ++ackCount; + } + + public int getAckCount() { + return ackCount; + } + } + + @Test + public void testReadCheckpoint() { + MqttIO.MqttCheckpointMark.Preparer preparer = new MqttIO.MqttCheckpointMark.Preparer("id"); + ArrayList messages = new ArrayList<>(); + for (int i = 0; i < 5; ++i) { + messages.add(new FakeMessage()); + } + preparer.add(messages.get(0), Instant.ofEpochMilli(20)); + preparer.add(messages.get(1), Instant.ofEpochMilli(10)); + preparer.add(messages.get(2), Instant.ofEpochMilli(30)); + assertEquals(Instant.ofEpochMilli(10), preparer.oldestMessageTimestamp); + MqttIO.MqttCheckpointMark checkpointA = preparer.newCheckpoint(); + preparer.add(messages.get(3), Instant.ofEpochMilli(40)); + preparer.add(messages.get(4), Instant.ofEpochMilli(50)); + MqttIO.MqttCheckpointMark checkpointB = preparer.newCheckpoint(); + assertTrue( + Arrays.stream(messages.toArray()).allMatch((m -> ((FakeMessage) m).getAckCount() == 0))); + checkpointA.finalizeCheckpoint(); + // only messages in finalized checkpoint acked + assertTrue( + Arrays.stream(messages.subList(0, 3).toArray()) + .allMatch((m -> ((FakeMessage) m).getAckCount() == 1))); + assertTrue( + Arrays.stream(messages.subList(3, 5).toArray()) + .allMatch((m -> ((FakeMessage) m).getAckCount() == 0))); + checkpointB.finalizeCheckpoint(); + // all messaged acked once + assertTrue( + Arrays.stream(messages.toArray()).allMatch((m -> ((FakeMessage) m).getAckCount() == 1))); + } + @Test public void testWrite() throws Exception { final int numberOfTestMessages = 200; @@ -560,7 +618,6 @@ public void testReadObject() throws Exception { // the number of messages of the decoded checkpoint should be zero assertEquals(0, cp2.messages.size()); assertEquals(cp1.clientId, cp2.clientId); - assertEquals(cp1.oldestMessageTimestamp, cp2.oldestMessageTimestamp); } /** diff --git a/sdks/java/io/pulsar/build.gradle b/sdks/java/io/pulsar/build.gradle index 7ffe3f22cca4..a6428e75c89d 100644 --- a/sdks/java/io/pulsar/build.gradle +++ b/sdks/java/io/pulsar/build.gradle @@ -18,11 +18,12 @@ plugins { id 'org.apache.beam.module' } applyJavaNature(automaticModuleName: 'org.apache.beam.sdk.io.pulsar') +enableJavaPerformanceTesting() description = "Apache Beam :: SDKs :: Java :: IO :: Pulsar" ext.summary = "IO to read and write to Pulsar" -def pulsar_version = '2.8.2' +def pulsar_version = '2.11.4' dependencies { @@ -30,19 +31,19 @@ dependencies { implementation library.java.slf4j_api implementation library.java.joda_time - implementation "org.apache.pulsar:pulsar-client:$pulsar_version" - implementation "org.apache.pulsar:pulsar-client-admin:$pulsar_version" - permitUnusedDeclared "org.apache.pulsar:pulsar-client:$pulsar_version" - permitUnusedDeclared "org.apache.pulsar:pulsar-client-admin:$pulsar_version" - permitUsedUndeclared "org.apache.pulsar:pulsar-client-api:$pulsar_version" - permitUsedUndeclared "org.apache.pulsar:pulsar-client-admin-api:$pulsar_version" + implementation "org.apache.pulsar:pulsar-client-api:$pulsar_version" + implementation "org.apache.pulsar:pulsar-client-admin-api:$pulsar_version" + runtimeOnly "org.apache.pulsar:pulsar-client:$pulsar_version" + runtimeOnly("org.apache.pulsar:pulsar-client-admin:$pulsar_version") { + // To prevent a StackOverflow within Pulsar admin client because JUL -> SLF4J -> JUL + exclude group: "org.slf4j", module: "jul-to-slf4j" + } implementation project(path: ":sdks:java:core", configuration: "shadow") - testImplementation library.java.jupiter_api - testRuntimeOnly library.java.jupiter_engine + testImplementation library.java.junit + testRuntimeOnly library.java.slf4j_jdk14 testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") testImplementation "org.testcontainers:pulsar:1.15.3" testImplementation "org.assertj:assertj-core:2.9.1" - } diff --git a/sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/ReadFromPulsarDoFn.java b/sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/NaiveReadFromPulsarDoFn.java similarity index 51% rename from sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/ReadFromPulsarDoFn.java rename to sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/NaiveReadFromPulsarDoFn.java index 3d255ac9baee..a80f02590827 100644 --- a/sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/ReadFromPulsarDoFn.java +++ b/sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/NaiveReadFromPulsarDoFn.java @@ -17,11 +17,13 @@ */ package org.apache.beam.sdk.io.pulsar; -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import java.io.IOException; +import java.time.Duration; +import java.util.ArrayList; import java.util.concurrent.TimeUnit; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.range.OffsetRange; +import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.splittabledofn.GrowableOffsetRangeTracker; @@ -30,6 +32,9 @@ import org.apache.beam.sdk.transforms.splittabledofn.WatermarkEstimator; import org.apache.beam.sdk.transforms.splittabledofn.WatermarkEstimators; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Stopwatch; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Suppliers; import org.apache.pulsar.client.admin.PulsarAdmin; @@ -40,68 +45,73 @@ import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Reader; import org.apache.pulsar.client.api.ReaderBuilder; +import org.checkerframework.checker.nullness.qual.MonotonicNonNull; +import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** - * Transform for reading from Apache Pulsar. Support is currently incomplete, and there may be bugs; - * see https://github.com/apache/beam/issues/31078 for more info, and comment in that issue if you - * run into issues with this IO. + * DoFn for reading from Apache Pulsar based on Pulsar {@link Reader} from the start message id. It + * does not support split or acknowledge message get read. */ @DoFn.UnboundedPerElement -@SuppressWarnings({"rawtypes", "nullness"}) -@SuppressFBWarnings(value = "CT_CONSTRUCTOR_THROW", justification = "Initialization is safe.") -public class ReadFromPulsarDoFn extends DoFn { +@SuppressWarnings("nullness") +public class NaiveReadFromPulsarDoFn extends DoFn { - private static final Logger LOG = LoggerFactory.getLogger(ReadFromPulsarDoFn.class); - private SerializableFunction pulsarClientSerializableFunction; - private PulsarClient client; - private PulsarAdmin admin; - private String clientUrl; - private String adminUrl; + private static final Logger LOG = LoggerFactory.getLogger(NaiveReadFromPulsarDoFn.class); + private final SerializableFunction clientFn; + private final SerializableFunction adminFn; + private final SerializableFunction, T> outputFn; + private final java.time.Duration pollingTimeout; + private transient @MonotonicNonNull PulsarClient client; + private transient @MonotonicNonNull PulsarAdmin admin; + private @MonotonicNonNull String clientUrl; + private @Nullable final String adminUrl; private final SerializableFunction, Instant> extractOutputTimestampFn; - public ReadFromPulsarDoFn(PulsarIO.Read transform) { - this.extractOutputTimestampFn = transform.getExtractOutputTimestampFn(); + public NaiveReadFromPulsarDoFn(PulsarIO.Read transform) { + this.extractOutputTimestampFn = + transform.getTimestampType() == PulsarIO.ReadTimestampType.PUBLISH_TIME + ? record -> new Instant(record.getPublishTime()) + : ignored -> Instant.now(); + this.pollingTimeout = Duration.ofSeconds(transform.getConsumerPollingTimeout()); + this.outputFn = transform.getOutputFn(); this.clientUrl = transform.getClientUrl(); this.adminUrl = transform.getAdminUrl(); - this.pulsarClientSerializableFunction = transform.getPulsarClient(); + this.clientFn = + MoreObjects.firstNonNull( + transform.getPulsarClient(), PulsarIOUtils.PULSAR_CLIENT_SERIALIZABLE_FUNCTION); + this.adminFn = + MoreObjects.firstNonNull( + transform.getPulsarAdmin(), PulsarIOUtils.PULSAR_ADMIN_SERIALIZABLE_FUNCTION); + admin = null; } - // Open connection to Pulsar clients + /** Open connection to Pulsar clients. */ @Setup public void initPulsarClients() throws Exception { - if (this.clientUrl == null) { - this.clientUrl = PulsarIOUtils.SERVICE_URL; - } - if (this.adminUrl == null) { - this.adminUrl = PulsarIOUtils.SERVICE_HTTP_URL; - } - - if (this.client == null) { - this.client = pulsarClientSerializableFunction.apply(this.clientUrl); - if (this.client == null) { - this.client = PulsarClient.builder().serviceUrl(clientUrl).build(); + if (client == null) { + if (clientUrl == null) { + clientUrl = PulsarIOUtils.LOCAL_SERVICE_URL; } + client = clientFn.apply(clientUrl); } - if (this.admin == null) { - this.admin = - PulsarAdmin.builder() - .serviceHttpUrl(adminUrl) - .tlsTrustCertsFilePath(null) - .allowTlsInsecureConnection(false) - .build(); + // admin is optional + if (this.admin == null && !Strings.isNullOrEmpty(adminUrl)) { + admin = adminFn.apply(adminUrl); } } - // Close connection to Pulsar clients + /** Close connection to Pulsar clients. */ @Teardown public void teardown() throws Exception { this.client.close(); - this.admin.close(); + if (this.admin != null) { + this.admin.close(); + } } @GetInitialRestriction @@ -152,31 +162,60 @@ public Coder getRestrictionCoder() { public ProcessContinuation processElement( @Element PulsarSourceDescriptor pulsarSourceDescriptor, RestrictionTracker tracker, - WatermarkEstimator watermarkEstimator, - OutputReceiver output) + WatermarkEstimator watermarkEstimator, + OutputReceiver output) throws IOException { long startTimestamp = tracker.currentRestriction().getFrom(); String topicDescriptor = pulsarSourceDescriptor.getTopic(); try (Reader reader = newReader(this.client, topicDescriptor)) { if (startTimestamp > 0) { + // reader.seek moves the cursor at the first occurrence of the message published after the + // assigned timestamp. + // i.e. all messages should be captured within the rangeTracker is after cursor reader.seek(startTimestamp); } - while (true) { - if (reader.hasReachedEndOfTopic()) { - reader.close(); - return ProcessContinuation.stop(); + if (reader.hasReachedEndOfTopic()) { + // topic has terminated + tracker.tryClaim(Long.MAX_VALUE); + reader.close(); + return ProcessContinuation.stop(); + } + boolean claimed = false; + ArrayList> maybeLateMessages = new ArrayList<>(); + final Stopwatch pollTimer = Stopwatch.createUnstarted(); + Duration remainingTimeout = pollingTimeout; + while (Duration.ZERO.compareTo(remainingTimeout) < 0) { + pollTimer.reset().start(); + Message message = + reader.readNext((int) remainingTimeout.toMillis(), TimeUnit.MILLISECONDS); + final Duration elapsed = pollTimer.elapsed(); + try { + remainingTimeout = remainingTimeout.minus(elapsed); + } catch (ArithmeticException e) { + remainingTimeout = Duration.ZERO; } - Message message = reader.readNext(); + // No progress when the polling timeout expired. + // Self-checkpoint and move to process the next element. if (message == null) { return ProcessContinuation.resume(); - } - Long currentTimestamp = message.getPublishTime(); - // if tracker.tryclaim() return true, sdf must execute work otherwise - // doFn must exit processElement() without doing any work associated - // or claiming more work - if (!tracker.tryClaim(currentTimestamp)) { + } // Trying to claim offset -1 before start of the range [0, 9223372036854775807) + long currentTimestamp = message.getPublishTime(); + if (currentTimestamp < startTimestamp) { + // This should not happen per pulsar spec (see comments around read.seek). If it + // does happen, this prevents tryClaim crash (IllegalArgumentException: Trying to + // claim offset before start of the range) + LOG.warn( + "Received late message of publish time {} before startTimestamp {}", + currentTimestamp, + startTimestamp); + } else if (!tracker.tryClaim(currentTimestamp)) { + // if tracker.tryclaim() return true, sdf must execute work otherwise + // doFn must exit processElement() without doing any work associated + // or claiming more work reader.close(); return ProcessContinuation.stop(); + } else { + claimed = true; } if (pulsarSourceDescriptor.getEndMessageId() != null) { MessageId currentMsgId = message.getMessageId(); @@ -186,12 +225,35 @@ public ProcessContinuation processElement( return ProcessContinuation.stop(); } } - PulsarMessage pulsarMessage = - new PulsarMessage(message.getTopicName(), message.getPublishTime(), message); - Instant outputTimestamp = extractOutputTimestampFn.apply(message); - output.outputWithTimestamp(pulsarMessage, outputTimestamp); + if (claimed) { + if (!maybeLateMessages.isEmpty()) { + for (Message lateMessage : maybeLateMessages) { + publishMessage(lateMessage, output); + } + maybeLateMessages.clear(); + } + publishMessage(message, output); + } else { + maybeLateMessages.add(message); + } } } + return ProcessContinuation.resume(); + } + + private void publishMessage(Message message, OutputReceiver output) { + T messageT = outputFn.apply(message); + Instant outputTimestamp = extractOutputTimestampFn.apply(message); + output.outputWithTimestamp(messageT, outputTimestamp); + } + + @SplitRestriction + public void splitRestriction( + @Restriction OffsetRange restriction, + OutputReceiver receiver, + PipelineOptions unused) { + // read based on Reader does not support split + receiver.output(restriction); } @GetInitialWatermarkEstimatorState @@ -221,28 +283,34 @@ public OffsetRangeTracker restrictionTracker( private static class PulsarLatestOffsetEstimator implements GrowableOffsetRangeTracker.RangeEndEstimator { - private final Supplier memoizedBacklog; + private final @Nullable Supplier> memoizedBacklog; - private PulsarLatestOffsetEstimator(PulsarAdmin admin, String topic) { - this.memoizedBacklog = - Suppliers.memoizeWithExpiration( - () -> { - try { - Message lastMsg = admin.topics().examineMessage(topic, "latest", 1); - return lastMsg; - } catch (PulsarAdminException e) { - LOG.error(e.getMessage()); - throw new RuntimeException(e); - } - }, - 1, - TimeUnit.SECONDS); + private PulsarLatestOffsetEstimator(@Nullable PulsarAdmin admin, String topic) { + if (admin != null) { + this.memoizedBacklog = + Suppliers.memoizeWithExpiration( + () -> { + try { + return admin.topics().examineMessage(topic, "latest", 1); + } catch (PulsarAdminException e) { + throw new RuntimeException(e); + } + }, + 1, + TimeUnit.SECONDS); + } else { + memoizedBacklog = null; + } } @Override public long estimate() { - Message msg = memoizedBacklog.get(); - return msg.getPublishTime(); + if (memoizedBacklog != null) { + Message msg = memoizedBacklog.get(); + return msg.getPublishTime(); + } else { + return Long.MIN_VALUE; + } } } diff --git a/sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/PulsarIO.java b/sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/PulsarIO.java index aaff08a96d36..34535e7cb44f 100644 --- a/sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/PulsarIO.java +++ b/sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/PulsarIO.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.io.pulsar; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; + import com.google.auto.value.AutoValue; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.PTransform; @@ -25,16 +27,17 @@ import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.PulsarClient; import org.checkerframework.checker.nullness.qual.Nullable; -import org.joda.time.Instant; /** - * Class for reading and writing from Apache Pulsar. Support is currently incomplete, and there may - * be bugs; see https://github.com/apache/beam/issues/31078 for more info, and comment in that issue - * if you run into issues with this IO. + * IO connector for reading and writing from Apache Pulsar. Support is currently experimental, and + * there may be bugs or performance issues; see https://github.com/apache/beam/issues/31078 for more + * info, and comment in that issue if you run into issues with this IO. */ @SuppressWarnings({"rawtypes", "nullness"}) public class PulsarIO { @@ -43,19 +46,41 @@ public class PulsarIO { private PulsarIO() {} /** - * Read from Apache Pulsar. Support is currently incomplete, and there may be bugs; see + * Read from Apache Pulsar. + * + *

Support is currently experimental, and there may be bugs or performance issues; see * https://github.com/apache/beam/issues/31078 for more info, and comment in that issue if you run * into issues with this IO. + * + * @param fn a mapping function converting {@link Message} that returned by Pulsar client to a + * custom type understood by Beam. */ - public static Read read() { + public static Read read(SerializableFunction fn) { return new AutoValue_PulsarIO_Read.Builder() - .setPulsarClient(PulsarIOUtils.PULSAR_CLIENT_SERIALIZABLE_FUNCTION) + .setOutputFn(fn) + .setConsumerPollingTimeout(PulsarIOUtils.DEFAULT_CONSUMER_POLLING_TIMEOUT) + .setTimestampType(ReadTimestampType.PUBLISH_TIME) .build(); } + /** + * The same as {@link PulsarIO#read(SerializableFunction)}, but returns {@link + * PCollection}. + */ + public static Read read() { + return new AutoValue_PulsarIO_Read.Builder() + .setOutputFn(PULSAR_MESSAGE_SERIALIZABLE_FUNCTION) + .setConsumerPollingTimeout(PulsarIOUtils.DEFAULT_CONSUMER_POLLING_TIMEOUT) + .setTimestampType(ReadTimestampType.PUBLISH_TIME) + .build(); + } + + private static final SerializableFunction, PulsarMessage> + PULSAR_MESSAGE_SERIALIZABLE_FUNCTION = PulsarMessage::create; + @AutoValue @SuppressWarnings({"rawtypes"}) - public abstract static class Read extends PTransform> { + public abstract static class Read extends PTransform> { abstract @Nullable String getClientUrl(); @@ -69,107 +94,152 @@ public abstract static class Read extends PTransform, Instant> getExtractOutputTimestampFn(); + abstract ReadTimestampType getTimestampType(); - abstract SerializableFunction getPulsarClient(); + abstract long getConsumerPollingTimeout(); - abstract Builder builder(); + abstract @Nullable SerializableFunction getPulsarClient(); + + abstract @Nullable SerializableFunction getPulsarAdmin(); + + abstract SerializableFunction, T> getOutputFn(); + + abstract Builder builder(); @AutoValue.Builder - abstract static class Builder { - abstract Builder setClientUrl(String url); + abstract static class Builder { + abstract Builder setClientUrl(String url); - abstract Builder setAdminUrl(String url); + abstract Builder setAdminUrl(String url); - abstract Builder setTopic(String topic); + abstract Builder setTopic(String topic); - abstract Builder setStartTimestamp(Long timestamp); + abstract Builder setStartTimestamp(Long timestamp); - abstract Builder setEndTimestamp(Long timestamp); + abstract Builder setEndTimestamp(Long timestamp); - abstract Builder setEndMessageId(MessageId msgId); + abstract Builder setEndMessageId(MessageId msgId); - abstract Builder setExtractOutputTimestampFn( - SerializableFunction, Instant> fn); + abstract Builder setTimestampType(ReadTimestampType timestampType); - abstract Builder setPulsarClient(SerializableFunction fn); + abstract Builder setConsumerPollingTimeout(long timeOutMs); + + abstract Builder setPulsarClient(SerializableFunction fn); + + abstract Builder setPulsarAdmin(SerializableFunction fn); - abstract Read build(); + @SuppressWarnings("getvsset") // outputFn determines generic type + abstract Builder setOutputFn(SerializableFunction, T> fn); + + abstract Read build(); } - public Read withAdminUrl(String url) { + /** + * Configure Pulsar admin url. + * + *

Admin client is used to approximate backlogs. This setting is optional. + * + * @param url admin url. For example, {@code "http://localhost:8080"}. + */ + public Read withAdminUrl(String url) { return builder().setAdminUrl(url).build(); } - public Read withClientUrl(String url) { + /** + * Configure Pulsar client url. {@code "pulsar://localhost:6650"}. + * + * @param url client url. For example, + */ + public Read withClientUrl(String url) { return builder().setClientUrl(url).build(); } - public Read withTopic(String topic) { + public Read withTopic(String topic) { return builder().setTopic(topic).build(); } - public Read withStartTimestamp(Long timestamp) { + public Read withStartTimestamp(Long timestamp) { return builder().setStartTimestamp(timestamp).build(); } - public Read withEndTimestamp(Long timestamp) { + public Read withEndTimestamp(Long timestamp) { return builder().setEndTimestamp(timestamp).build(); } - public Read withEndMessageId(MessageId msgId) { + public Read withEndMessageId(MessageId msgId) { return builder().setEndMessageId(msgId).build(); } - public Read withExtractOutputTimestampFn(SerializableFunction, Instant> fn) { - return builder().setExtractOutputTimestampFn(fn).build(); + /** Set elements timestamped by {@link Message#getPublishTime()}. It is the default. */ + public Read withPublishTime() { + return builder().setTimestampType(ReadTimestampType.PUBLISH_TIME).build(); } - public Read withPublishTime() { - return withExtractOutputTimestampFn(ExtractOutputTimestampFn.usePublishTime()); + /** Set elements timestamped to the moment it get processed. */ + public Read withProcessingTime() { + return builder().setTimestampType(ReadTimestampType.PROCESSING_TIME).build(); } - public Read withProcessingTime() { - return withExtractOutputTimestampFn(ExtractOutputTimestampFn.useProcessingTime()); + /** + * Sets the timeout time in seconds for Pulsar consumer polling request. A lower timeout + * optimizes for latency. Increase the timeout if the consumer is not fetching any records. The + * default is 2 seconds. + */ + public Read withConsumerPollingTimeout(long duration) { + checkState(duration > 0, "Consumer polling timeout must be greater than 0."); + return builder().setConsumerPollingTimeout(duration).build(); } - public Read withPulsarClient(SerializableFunction pulsarClientFn) { + public Read withPulsarClient(SerializableFunction pulsarClientFn) { return builder().setPulsarClient(pulsarClientFn).build(); } + public Read withPulsarAdmin(SerializableFunction pulsarAdminFn) { + return builder().setPulsarAdmin(pulsarAdminFn).build(); + } + + @SuppressWarnings("unchecked") // for PulsarMessage @Override - public PCollection expand(PBegin input) { - return input - .apply( - Create.of( - PulsarSourceDescriptor.of( - getTopic(), - getStartTimestamp(), - getEndTimestamp(), - getEndMessageId(), - getClientUrl(), - getAdminUrl()))) - .apply(ParDo.of(new ReadFromPulsarDoFn(this))) - .setCoder(PulsarMessageCoder.of()); + public PCollection expand(PBegin input) { + PCollection pcoll = + input + .apply( + Create.of( + PulsarSourceDescriptor.of( + getTopic(), getStartTimestamp(), getEndTimestamp(), getEndMessageId()))) + .apply(ParDo.of(new NaiveReadFromPulsarDoFn<>(this))); + if (getOutputFn().equals(PULSAR_MESSAGE_SERIALIZABLE_FUNCTION)) { + // register coder for default implementation of read + return pcoll.setTypeDescriptor((TypeDescriptor) TypeDescriptor.of(PulsarMessage.class)); + } + return pcoll; } } + enum ReadTimestampType { + PROCESSING_TIME, + PUBLISH_TIME, + } + /** - * Write to Apache Pulsar. Support is currently incomplete, and there may be bugs; see - * https://github.com/apache/beam/issues/31078 for more info, and comment in that issue if you run - * into issues with this IO. + * Write to Apache Pulsar. Support is currently experimental, and there may be bugs or performance + * issues; see https://github.com/apache/beam/issues/31078 for more info, and comment in that + * issue if you run into issues with this IO. */ public static Write write() { - return new AutoValue_PulsarIO_Write.Builder().build(); + return new AutoValue_PulsarIO_Write.Builder() + .setPulsarClient(PulsarIOUtils.PULSAR_CLIENT_SERIALIZABLE_FUNCTION) + .build(); } @AutoValue - @SuppressWarnings({"rawtypes"}) public abstract static class Write extends PTransform, PDone> { abstract @Nullable String getTopic(); - abstract String getClientUrl(); + abstract @Nullable String getClientUrl(); + + abstract SerializableFunction getPulsarClient(); abstract Builder builder(); @@ -179,6 +249,8 @@ abstract static class Builder { abstract Builder setClientUrl(String clientUrl); + abstract Builder setPulsarClient(SerializableFunction fn); + abstract Write build(); } @@ -190,20 +262,14 @@ public Write withClientUrl(String clientUrl) { return builder().setClientUrl(clientUrl).build(); } + public Write withPulsarClient(SerializableFunction pulsarClientFn) { + return builder().setPulsarClient(pulsarClientFn).build(); + } + @Override public PDone expand(PCollection input) { input.apply(ParDo.of(new WriteToPulsarDoFn(this))); return PDone.in(input.getPipeline()); } } - - static class ExtractOutputTimestampFn { - public static SerializableFunction, Instant> useProcessingTime() { - return record -> Instant.now(); - } - - public static SerializableFunction, Instant> usePublishTime() { - return record -> new Instant(record.getPublishTime()); - } - } } diff --git a/sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/PulsarIOUtils.java b/sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/PulsarIOUtils.java index 53bc8e448768..8c4a3af282e1 100644 --- a/sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/PulsarIOUtils.java +++ b/sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/PulsarIOUtils.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.io.pulsar; import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.slf4j.Logger; @@ -26,19 +27,27 @@ final class PulsarIOUtils { private static final Logger LOG = LoggerFactory.getLogger(PulsarIOUtils.class); - public static final String SERVICE_HTTP_URL = "http://localhost:8080"; - public static final String SERVICE_URL = "pulsar://localhost:6650"; + static final String LOCAL_SERVICE_URL = "pulsar://localhost:6650"; + static final long DEFAULT_CONSUMER_POLLING_TIMEOUT = 2L; static final SerializableFunction PULSAR_CLIENT_SERIALIZABLE_FUNCTION = - new SerializableFunction() { - @Override - public PulsarClient apply(String input) { - try { - return PulsarClient.builder().serviceUrl(input).build(); - } catch (PulsarClientException e) { - LOG.error(e.getMessage()); - throw new RuntimeException(e); - } + input -> { + try { + return PulsarClient.builder().serviceUrl(input).build(); + } catch (PulsarClientException e) { + throw new RuntimeException(e); + } + }; + + static final SerializableFunction PULSAR_ADMIN_SERIALIZABLE_FUNCTION = + input -> { + try { + return PulsarAdmin.builder() + .serviceHttpUrl(input) + .allowTlsInsecureConnection(false) + .build(); + } catch (PulsarClientException e) { + throw new RuntimeException(e); } }; } diff --git a/sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/PulsarMessage.java b/sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/PulsarMessage.java index 34fa989177eb..739d34c98604 100644 --- a/sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/PulsarMessage.java +++ b/sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/PulsarMessage.java @@ -17,40 +17,52 @@ */ package org.apache.beam.sdk.io.pulsar; +import com.google.auto.value.AutoValue; +import java.util.Map; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.pulsar.client.api.Message; +import org.checkerframework.checker.nullness.qual.Nullable; + /** * Class representing a Pulsar Message record. Each PulsarMessage contains a single message basic * message data and Message record to access directly. */ -@SuppressWarnings("initialization.fields.uninitialized") -public class PulsarMessage { - private String topic; - private Long publishTimestamp; - private Object messageRecord; - - public PulsarMessage(String topic, Long publishTimestamp, Object messageRecord) { - this.topic = topic; - this.publishTimestamp = publishTimestamp; - this.messageRecord = messageRecord; - } +@DefaultSchema(AutoValueSchema.class) +@AutoValue +public abstract class PulsarMessage { + abstract @Nullable String getTopic(); - public PulsarMessage(String topic, Long publishTimestamp) { - this.topic = topic; - this.publishTimestamp = publishTimestamp; - } + abstract long getPublishTimestamp(); - public String getTopic() { - return topic; - } + abstract @Nullable String getKey(); - public Long getPublishTimestamp() { - return publishTimestamp; - } + @SuppressWarnings("mutable") + abstract byte[] getValue(); + + abstract @Nullable Map getProperties(); + + @SuppressWarnings("mutable") + abstract byte[] getMessageId(); - public void setMessageRecord(Object messageRecord) { - this.messageRecord = messageRecord; + public static PulsarMessage create( + @Nullable String topicName, + long publishTimestamp, + @Nullable String key, + byte[] value, + @Nullable Map properties, + byte[] messageId) { + return new AutoValue_PulsarMessage( + topicName, publishTimestamp, key, value, properties, messageId); } - public Object getMessageRecord() { - return messageRecord; + public static PulsarMessage create(Message message) { + return create( + message.getTopicName(), + message.getPublishTime(), + message.getKey(), + message.getValue(), + message.getProperties(), + message.getMessageId().toByteArray()); } } diff --git a/sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/PulsarMessageCoder.java b/sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/PulsarMessageCoder.java deleted file mode 100644 index 2f3bed5fa085..000000000000 --- a/sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/PulsarMessageCoder.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * 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. - */ -package org.apache.beam.sdk.io.pulsar; - -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.CustomCoder; -import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.coders.VarLongCoder; - -public class PulsarMessageCoder extends CustomCoder { - - private static final StringUtf8Coder stringCoder = StringUtf8Coder.of(); - private static final VarLongCoder longCoder = VarLongCoder.of(); - - public static PulsarMessageCoder of() { - return new PulsarMessageCoder(); - } - - public PulsarMessageCoder() {} - - @Override - public void encode(PulsarMessage value, OutputStream outStream) - throws CoderException, IOException { - stringCoder.encode(value.getTopic(), outStream); - longCoder.encode(value.getPublishTimestamp(), outStream); - } - - @Override - public PulsarMessage decode(InputStream inStream) throws CoderException, IOException { - return new PulsarMessage(stringCoder.decode(inStream), longCoder.decode(inStream)); - } -} diff --git a/sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/PulsarSourceDescriptor.java b/sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/PulsarSourceDescriptor.java index 427d37d1d72a..66617f9863aa 100644 --- a/sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/PulsarSourceDescriptor.java +++ b/sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/PulsarSourceDescriptor.java @@ -44,20 +44,9 @@ public abstract class PulsarSourceDescriptor implements Serializable { @Nullable abstract MessageId getEndMessageId(); - @SchemaFieldName("client_url") - abstract String getClientUrl(); - - @SchemaFieldName("admin_url") - abstract String getAdminUrl(); - public static PulsarSourceDescriptor of( - String topic, - Long startOffsetTimestamp, - Long endOffsetTimestamp, - MessageId endMessageId, - String clientUrl, - String adminUrl) { + String topic, Long startOffsetTimestamp, Long endOffsetTimestamp, MessageId endMessageId) { return new AutoValue_PulsarSourceDescriptor( - topic, startOffsetTimestamp, endOffsetTimestamp, endMessageId, clientUrl, adminUrl); + topic, startOffsetTimestamp, endOffsetTimestamp, endMessageId); } } diff --git a/sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/WriteToPulsarDoFn.java b/sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/WriteToPulsarDoFn.java index 375e8ce92a3a..7d64b6e49b19 100644 --- a/sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/WriteToPulsarDoFn.java +++ b/sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/WriteToPulsarDoFn.java @@ -18,33 +18,39 @@ package org.apache.beam.sdk.io.pulsar; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.pulsar.client.api.CompressionType; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; -/** - * Transform for writing to Apache Pulsar. Support is currently incomplete, and there may be bugs; - * see https://github.com/apache/beam/issues/31078 for more info, and comment in that issue if you - * run into issues with this IO. - */ -@DoFn.UnboundedPerElement -@SuppressWarnings({"rawtypes", "nullness"}) +/** DoFn for writing to Apache Pulsar. */ +@SuppressWarnings({"nullness"}) public class WriteToPulsarDoFn extends DoFn { - - private Producer producer; - private PulsarClient client; + private final SerializableFunction clientFn; + private transient Producer producer; + private transient PulsarClient client; private String clientUrl; private String topic; WriteToPulsarDoFn(PulsarIO.Write transform) { this.clientUrl = transform.getClientUrl(); this.topic = transform.getTopic(); + this.clientFn = transform.getPulsarClient(); } @Setup - public void setup() throws PulsarClientException { - client = PulsarClient.builder().serviceUrl(clientUrl).build(); + public void setup() { + if (client == null) { + if (clientUrl == null) { + clientUrl = PulsarIOUtils.LOCAL_SERVICE_URL; + } + client = clientFn.apply(clientUrl); + } + } + + @StartBundle + public void startBundle() throws PulsarClientException { producer = client.newProducer().topic(topic).compressionType(CompressionType.LZ4).create(); } @@ -53,9 +59,13 @@ public void processElement(@Element byte[] messageToSend) throws Exception { producer.send(messageToSend); } + @FinishBundle + public void finishBundle() throws PulsarClientException { + producer.close(); + } + @Teardown public void teardown() throws PulsarClientException { - producer.close(); client.close(); } } diff --git a/sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/package-info.java b/sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/package-info.java index ffa15257fe5a..3ec49fa1f73e 100644 --- a/sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/package-info.java +++ b/sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/package-info.java @@ -16,8 +16,8 @@ * limitations under the License. */ /** - * Transforms for reading and writing from Apache Pulsar. Support is currently incomplete, and there - * may be bugs; see https://github.com/apache/beam/issues/31078 for more info, and comment in that - * issue if you run into issues with this IO. + * Transforms for reading and writing from Apache Pulsar. Support is currently experimental, and + * there may be bugs and performance issues; see https://github.com/apache/beam/issues/31078 for + * more info, and comment in that issue if you run into issues with this IO. */ package org.apache.beam.sdk.io.pulsar; diff --git a/sdks/java/io/pulsar/src/test/java/org/apache/beam/sdk/io/pulsar/FakeMessage.java b/sdks/java/io/pulsar/src/test/java/org/apache/beam/sdk/io/pulsar/FakeMessage.java index 9cdc4af37435..b02ef98a2f85 100644 --- a/sdks/java/io/pulsar/src/test/java/org/apache/beam/sdk/io/pulsar/FakeMessage.java +++ b/sdks/java/io/pulsar/src/test/java/org/apache/beam/sdk/io/pulsar/FakeMessage.java @@ -68,12 +68,13 @@ public int size() { @Override public byte[] getValue() { - return null; + return new byte[0]; } @Override public MessageId getMessageId() { - return DefaultImplementation.newMessageId(this.ledgerId, this.entryId, this.partitionIndex); + return DefaultImplementation.getDefaultImplementation() + .newMessageId(this.ledgerId, this.entryId, this.partitionIndex); } @Override @@ -158,4 +159,24 @@ public String getReplicatedFrom() { @Override public void release() {} + + @Override + public boolean hasBrokerPublishTime() { + return false; + } + + @Override + public Optional getBrokerPublishTime() { + return Optional.empty(); + } + + @Override + public boolean hasIndex() { + return false; + } + + @Override + public Optional getIndex() { + return Optional.empty(); + } } diff --git a/sdks/java/io/pulsar/src/test/java/org/apache/beam/sdk/io/pulsar/FakePulsarClient.java b/sdks/java/io/pulsar/src/test/java/org/apache/beam/sdk/io/pulsar/FakePulsarClient.java index 4639d8420be9..debded32494b 100644 --- a/sdks/java/io/pulsar/src/test/java/org/apache/beam/sdk/io/pulsar/FakePulsarClient.java +++ b/sdks/java/io/pulsar/src/test/java/org/apache/beam/sdk/io/pulsar/FakePulsarClient.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.io.pulsar; +import java.time.Instant; import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; @@ -31,11 +32,13 @@ import org.apache.pulsar.client.api.Range; import org.apache.pulsar.client.api.Reader; import org.apache.pulsar.client.api.ReaderBuilder; +import org.apache.pulsar.client.api.ReaderInterceptor; import org.apache.pulsar.client.api.ReaderListener; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.TableViewBuilder; import org.apache.pulsar.client.api.transaction.TransactionBuilder; -@SuppressWarnings({"rawtypes"}) +@SuppressWarnings("rawtypes") public class FakePulsarClient implements PulsarClient { private MockReaderBuilder readerBuilder; @@ -86,6 +89,11 @@ public ReaderBuilder newReader(Schema schema) { return null; } + @Override + public TableViewBuilder newTableViewBuilder(Schema schema) { + return null; + } + @Override public void updateServiceUrl(String serviceUrl) throws PulsarClientException {} @@ -134,7 +142,8 @@ public Reader create() throws PulsarClientException { if (this.reader != null) { return this.reader; } - this.reader = new FakePulsarReader(this.topic, this.numberOfMessages); + this.reader = + new FakePulsarReader(this.topic, this.numberOfMessages, Instant.now().toEpochMilli()); return this.reader; } @@ -145,7 +154,7 @@ public CompletableFuture> createAsync() { @Override public ReaderBuilder clone() { - return null; + return this; } @Override @@ -162,77 +171,114 @@ public ReaderBuilder startMessageId(MessageId startMessageId) { @Override public ReaderBuilder startMessageFromRollbackDuration( long rollbackDuration, TimeUnit timeunit) { - return null; + return this; } @Override public ReaderBuilder startMessageIdInclusive() { - return null; + return this; } @Override public ReaderBuilder readerListener(ReaderListener readerListener) { - return null; + return this; } @Override public ReaderBuilder cryptoKeyReader(CryptoKeyReader cryptoKeyReader) { - return null; + return this; } @Override public ReaderBuilder defaultCryptoKeyReader(String privateKey) { - return null; + return this; } @Override public ReaderBuilder cryptoFailureAction(ConsumerCryptoFailureAction action) { - return null; + return this; } @Override public ReaderBuilder receiverQueueSize(int receiverQueueSize) { - return null; + return this; } @Override public ReaderBuilder readerName(String readerName) { - return null; + return this; } @Override public ReaderBuilder subscriptionRolePrefix(String subscriptionRolePrefix) { - return null; + return this; } @Override public ReaderBuilder subscriptionName(String subscriptionName) { - return null; + return this; } @Override public ReaderBuilder readCompacted(boolean readCompacted) { - return null; + return this; } @Override public ReaderBuilder keyHashRange(Range... ranges) { - return null; + return this; + } + + @Override + public ReaderBuilder poolMessages(boolean poolMessages) { + return this; + } + + @Override + public ReaderBuilder autoUpdatePartitions(boolean autoUpdate) { + return this; + } + + @Override + public ReaderBuilder autoUpdatePartitionsInterval(int interval, TimeUnit unit) { + return this; + } + + @Override + public ReaderBuilder intercept(ReaderInterceptor... interceptors) { + return this; + } + + @Override + public ReaderBuilder maxPendingChunkedMessage(int maxPendingChunkedMessage) { + return this; + } + + @Override + public ReaderBuilder autoAckOldestChunkedMessageOnQueueFull( + boolean autoAckOldestChunkedMessageOnQueueFull) { + return this; } @Override public ReaderBuilder defaultCryptoKeyReader(Map privateKeys) { - return null; + return this; } @Override public ReaderBuilder topics(List topicNames) { - return null; + return this; } @Override public ReaderBuilder loadConf(Map config) { - return null; + return this; + } + + @Override + public ReaderBuilder expireTimeOfIncompleteChunkedMessage( + long duration, TimeUnit unit) { + return this; } } } diff --git a/sdks/java/io/pulsar/src/test/java/org/apache/beam/sdk/io/pulsar/FakePulsarReader.java b/sdks/java/io/pulsar/src/test/java/org/apache/beam/sdk/io/pulsar/FakePulsarReader.java index 834fd0427532..6d937e77ce12 100644 --- a/sdks/java/io/pulsar/src/test/java/org/apache/beam/sdk/io/pulsar/FakePulsarReader.java +++ b/sdks/java/io/pulsar/src/test/java/org/apache/beam/sdk/io/pulsar/FakePulsarReader.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.io.pulsar; import java.io.IOException; +import java.io.Serializable; import java.util.ArrayList; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -30,17 +31,18 @@ import org.joda.time.Duration; import org.joda.time.Instant; -public class FakePulsarReader implements Reader { +public class FakePulsarReader implements Reader, Serializable { private String topic; private List fakeMessages = new ArrayList<>(); private int currentMsg; - private long startTimestamp; + private final long startTimestamp; private long endTimestamp; private boolean reachedEndOfTopic; private int numberOfMessages; - public FakePulsarReader(String topic, int numberOfMessages) { + public FakePulsarReader(String topic, int numberOfMessages, long startTimestamp) { + this.startTimestamp = startTimestamp; this.numberOfMessages = numberOfMessages; this.setMock(topic, numberOfMessages); } @@ -52,10 +54,9 @@ public void setReachedEndOfTopic(boolean hasReachedEnd) { public void setMock(String topic, int numberOfMessages) { this.topic = topic; for (int i = 0; i < numberOfMessages; i++) { - long timestamp = Instant.now().plus(Duration.standardSeconds(i)).getMillis(); - if (i == 0) { - startTimestamp = timestamp; - } else if (i == 99) { + long timestamp = + Instant.ofEpochMilli(startTimestamp).plus(Duration.standardSeconds(i)).getMillis(); + if (i == numberOfMessages - 1) { endTimestamp = timestamp; } fakeMessages.add(new FakeMessage(topic, timestamp, Long.valueOf(i), Long.valueOf(i), i)); @@ -89,20 +90,23 @@ public String getTopic() { @Override public Message readNext() throws PulsarClientException { - if (currentMsg == 0 && fakeMessages.isEmpty()) { + if (fakeMessages.isEmpty()) { return null; } - Message msg = fakeMessages.get(currentMsg); - if (currentMsg <= fakeMessages.size() - 1) { + if (currentMsg < fakeMessages.size()) { + Message msg = fakeMessages.get(currentMsg); currentMsg++; + return msg; + } else { + reachedEndOfTopic = true; + return null; } - return msg; } @Override public Message readNext(int timeout, TimeUnit unit) throws PulsarClientException { - return null; + return readNext(); } @Override @@ -141,11 +145,12 @@ public void seek(MessageId messageId) throws PulsarClientException {} @Override public void seek(long timestamp) throws PulsarClientException { for (int i = 0; i < fakeMessages.size(); i++) { - if (timestamp == fakeMessages.get(i).getPublishTime()) { + if (timestamp <= fakeMessages.get(i).getPublishTime()) { currentMsg = i; - break; + return; } } + currentMsg = fakeMessages.size(); } @Override diff --git a/sdks/java/io/pulsar/src/test/java/org/apache/beam/sdk/io/pulsar/PulsarIOIT.java b/sdks/java/io/pulsar/src/test/java/org/apache/beam/sdk/io/pulsar/PulsarIOIT.java new file mode 100644 index 000000000000..d3b8cea7d899 --- /dev/null +++ b/sdks/java/io/pulsar/src/test/java/org/apache/beam/sdk/io/pulsar/PulsarIOIT.java @@ -0,0 +1,227 @@ +/* + * 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. + */ +package org.apache.beam.sdk.io.pulsar; + +import static org.junit.Assert.assertEquals; + +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.List; +import java.util.concurrent.TimeUnit; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.MetricNameFilter; +import org.apache.beam.sdk.metrics.MetricQueryResults; +import org.apache.beam.sdk.metrics.MetricResult; +import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.metrics.MetricsFilter; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.commons.lang3.RandomStringUtils; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.Timeout; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.PulsarContainer; +import org.testcontainers.utility.DockerImageName; + +@RunWith(JUnit4.class) +public class PulsarIOIT { + @Rule public Timeout globalTimeout = Timeout.seconds(60); + protected static PulsarContainer pulsarContainer; + protected static PulsarClient client; + + private long endExpectedTime = 0; + private long startTime = 0; + + private static final Logger LOG = LoggerFactory.getLogger(PulsarIOIT.class); + + @Rule public final transient TestPipeline testPipeline = TestPipeline.create(); + + public List> receiveMessages(String topic) throws PulsarClientException { + if (client == null) { + initClient(); + } + List> messages = new ArrayList<>(); + try (Consumer consumer = + client.newConsumer().topic(topic).subscriptionName("receiveMockMessageFn").subscribe()) { + consumer.seek(MessageId.earliest); + LOG.warn("started receiveMessages"); + while (!consumer.hasReachedEndOfTopic()) { + Message msg = consumer.receive(5, TimeUnit.SECONDS); + if (msg == null) { + LOG.warn("null message"); + break; + } + messages.add(msg); + consumer.acknowledge(msg); + } + } + messages.sort(Comparator.comparing(s -> new String(s.getValue(), StandardCharsets.UTF_8))); + return messages; + } + + public List produceMessages(String topic) throws PulsarClientException { + client = initClient(); + Producer producer = client.newProducer().topic(topic).create(); + Consumer consumer = + client.newConsumer().topic(topic).subscriptionName("produceMockMessageFn").subscribe(); + int numElements = 101; + List inputs = new ArrayList<>(); + for (int i = 0; i < numElements; i++) { + String msg = ("PULSAR_TEST_READFROMSIMPLETOPIC_" + i); + producer.send(msg.getBytes(StandardCharsets.UTF_8)); + Message message = consumer.receive(5, TimeUnit.SECONDS); + if (i == 100) { + endExpectedTime = message.getPublishTime(); + } else { + inputs.add(PulsarMessage.create(message)); + if (i == 0) { + startTime = message.getPublishTime(); + } + } + } + consumer.close(); + producer.close(); + client.close(); + return inputs; + } + + private static PulsarClient initClient() throws PulsarClientException { + return PulsarClient.builder().serviceUrl(pulsarContainer.getPulsarBrokerUrl()).build(); + } + + private static void setupPulsarContainer() { + pulsarContainer = new PulsarContainer(DockerImageName.parse("apachepulsar/pulsar:2.11.4")); + pulsarContainer.withCommand("bin/pulsar", "standalone"); + try { + pulsarContainer.start(); + } catch (IllegalStateException unused) { + pulsarContainer = new PulsarContainerLocalProxy(); + } + } + + static class PulsarContainerLocalProxy extends PulsarContainer { + @Override + public String getPulsarBrokerUrl() { + return "pulsar://localhost:6650"; + } + + @Override + public String getHttpServiceUrl() { + return "http://localhost:8080"; + } + } + + @BeforeClass + public static void setup() throws PulsarClientException { + setupPulsarContainer(); + client = initClient(); + } + + @AfterClass + public static void afterClass() { + if (pulsarContainer != null && pulsarContainer.isRunning()) { + pulsarContainer.stop(); + } + } + + @Test + public void testReadFromSimpleTopic() throws PulsarClientException { + String topic = "PULSARIOIT_READ" + RandomStringUtils.randomAlphanumeric(4); + List inputsMock = produceMessages(topic); + PulsarIO.Read reader = + PulsarIO.read() + .withClientUrl(pulsarContainer.getPulsarBrokerUrl()) + .withAdminUrl(pulsarContainer.getHttpServiceUrl()) + .withTopic(topic) + .withStartTimestamp(startTime) + .withEndTimestamp(endExpectedTime) + .withPublishTime(); + testPipeline.apply(reader).apply(ParDo.of(new PulsarRecordsMetric())); + + PipelineResult pipelineResult = testPipeline.run(); + MetricQueryResults metrics = + pipelineResult + .metrics() + .queryMetrics( + MetricsFilter.builder() + .addNameFilter( + MetricNameFilter.named(PulsarIOIT.class.getName(), "PulsarRecordsCounter")) + .build()); + long recordsCount = 0; + for (MetricResult metric : metrics.getCounters()) { + if (metric + .getName() + .toString() + .equals("org.apache.beam.sdk.io.pulsar.PulsarIOIT:PulsarRecordsCounter")) { + recordsCount = metric.getAttempted(); + break; + } + } + assertEquals(inputsMock.size(), (int) recordsCount); + } + + @Test + public void testWriteToTopic() throws PulsarClientException { + String topic = "PULSARIOIT_WRITE_" + RandomStringUtils.randomAlphanumeric(4); + PulsarIO.Write writer = + PulsarIO.write().withClientUrl(pulsarContainer.getPulsarBrokerUrl()).withTopic(topic); + int numberOfMessages = 10; + List messages = new ArrayList<>(); + for (int i = 0; i < numberOfMessages; i++) { + messages.add(("PULSAR_WRITER_TEST_" + i).getBytes(StandardCharsets.UTF_8)); + } + testPipeline.apply(Create.of(messages)).apply(writer); + + testPipeline.run(); + + List> receiveMsgs = receiveMessages(topic); + assertEquals(numberOfMessages, receiveMsgs.size()); + for (int i = 0; i < numberOfMessages; i++) { + assertEquals( + new String(receiveMsgs.get(i).getValue(), StandardCharsets.UTF_8), + "PULSAR_WRITER_TEST_" + i); + } + } + + public static class PulsarRecordsMetric extends DoFn { + private final Counter counter = + Metrics.counter(PulsarIOIT.class.getName(), "PulsarRecordsCounter"); + + @ProcessElement + public void processElement(ProcessContext context) { + counter.inc(); + context.output(context.element()); + } + } +} diff --git a/sdks/java/io/pulsar/src/test/java/org/apache/beam/sdk/io/pulsar/PulsarIOTest.java b/sdks/java/io/pulsar/src/test/java/org/apache/beam/sdk/io/pulsar/PulsarIOTest.java index eeb6a5d7652c..52ee3044d60c 100644 --- a/sdks/java/io/pulsar/src/test/java/org/apache/beam/sdk/io/pulsar/PulsarIOTest.java +++ b/sdks/java/io/pulsar/src/test/java/org/apache/beam/sdk/io/pulsar/PulsarIOTest.java @@ -17,225 +17,74 @@ */ package org.apache.beam.sdk.io.pulsar; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertTrue; - -import java.nio.charset.StandardCharsets; +import java.io.Serializable; +import java.time.Instant; import java.util.ArrayList; import java.util.List; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -import org.apache.beam.sdk.PipelineResult; -import org.apache.beam.sdk.metrics.Counter; -import org.apache.beam.sdk.metrics.MetricNameFilter; -import org.apache.beam.sdk.metrics.MetricQueryResults; -import org.apache.beam.sdk.metrics.MetricResult; -import org.apache.beam.sdk.metrics.Metrics; -import org.apache.beam.sdk.metrics.MetricsFilter; +import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.pulsar.client.api.Consumer; -import org.apache.pulsar.client.api.Message; -import org.apache.pulsar.client.api.Producer; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.pulsar.client.api.PulsarClient; -import org.apache.pulsar.client.api.PulsarClientException; -import org.junit.AfterClass; -import org.junit.BeforeClass; +import org.junit.Assert; import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.testcontainers.containers.PulsarContainer; -import org.testcontainers.utility.DockerImageName; +// TODO(https://github.com/apache/beam/issues/31078) exceptions are currently suppressed +@SuppressWarnings("Slf4jDoNotLogMessageOfExceptionExplicitly") @RunWith(JUnit4.class) -public class PulsarIOTest { - - private static final String TOPIC = "PULSAR_IO_TEST"; - protected static PulsarContainer pulsarContainer; - protected static PulsarClient client; - - private long endExpectedTime = 0; - private long startTime = 0; - +public class PulsarIOTest implements Serializable { + @Rule public final transient TestPipeline pipeline = TestPipeline.create(); private static final Logger LOG = LoggerFactory.getLogger(PulsarIOTest.class); - @Rule public final transient TestPipeline testPipeline = TestPipeline.create(); - - public List> receiveMessages() throws PulsarClientException { - if (client == null) { - initClient(); - } - List> messages = new ArrayList<>(); - Consumer consumer = - client.newConsumer().topic(TOPIC).subscriptionName("receiveMockMessageFn").subscribe(); - while (consumer.hasReachedEndOfTopic()) { - Message msg = consumer.receive(); - messages.add(msg); - try { - consumer.acknowledge(msg); - } catch (Exception e) { - consumer.negativeAcknowledge(msg); - } - } - return messages; - } - - public List produceMessages() throws PulsarClientException { - client = initClient(); - Producer producer = client.newProducer().topic(TOPIC).create(); - Consumer consumer = - client.newConsumer().topic(TOPIC).subscriptionName("produceMockMessageFn").subscribe(); - int numElements = 101; - List inputs = new ArrayList<>(); - for (int i = 0; i < numElements; i++) { - String msg = ("PULSAR_TEST_READFROMSIMPLETOPIC_" + i); - producer.send(msg.getBytes(StandardCharsets.UTF_8)); - CompletableFuture> future = consumer.receiveAsync(); - Message message = null; - try { - message = future.get(5, TimeUnit.SECONDS); - if (i >= 100) { - endExpectedTime = message.getPublishTime(); - } else { - inputs.add(new PulsarMessage(message.getTopicName(), message.getPublishTime(), message)); - if (i == 0) { - startTime = message.getPublishTime(); - } - } - } catch (InterruptedException e) { - LOG.error(e.getMessage()); - } catch (ExecutionException e) { - LOG.error(e.getMessage()); - } catch (TimeoutException e) { - LOG.error(e.getMessage()); - } - } - consumer.close(); - producer.close(); - client.close(); - return inputs; - } - - private static PulsarClient initClient() throws PulsarClientException { - return PulsarClient.builder().serviceUrl(pulsarContainer.getPulsarBrokerUrl()).build(); - } - - private static void setupPulsarContainer() { - pulsarContainer = new PulsarContainer(DockerImageName.parse("apachepulsar/pulsar:2.9.0")); - pulsarContainer.withCommand("bin/pulsar", "standalone"); - pulsarContainer.start(); - } - - @BeforeClass - public static void setup() throws PulsarClientException { - setupPulsarContainer(); - client = initClient(); - } - - @AfterClass - public static void afterClass() { - if (pulsarContainer != null) { - pulsarContainer.stop(); - } - } + private static final String TEST_TOPIC = "TEST_TOPIC"; + // In order to pin fake readers having same set of messages + private static final long START_TIMESTAMP = Instant.now().toEpochMilli(); - @Test - @SuppressWarnings({"rawtypes"}) - public void testPulsarFunctionality() throws Exception { - try (Consumer consumer = - client.newConsumer().topic(TOPIC).subscriptionName("PulsarIO_IT").subscribe(); - Producer producer = client.newProducer().topic(TOPIC).create(); ) { - String messageTxt = "testing pulsar functionality"; - producer.send(messageTxt.getBytes(StandardCharsets.UTF_8)); - CompletableFuture future = consumer.receiveAsync(); - Message message = future.get(5, TimeUnit.SECONDS); - assertEquals(messageTxt, new String(message.getData(), StandardCharsets.UTF_8)); - client.close(); - } + /** Create a fake client. */ + static PulsarClient newFakeClient() { + return new FakePulsarClient(new FakePulsarReader(TEST_TOPIC, 10, START_TIMESTAMP)); } @Test - public void testReadFromSimpleTopic() { - try { - List inputsMock = produceMessages(); - PulsarIO.Read reader = - PulsarIO.read() - .withClientUrl(pulsarContainer.getPulsarBrokerUrl()) - .withAdminUrl(pulsarContainer.getHttpServiceUrl()) - .withTopic(TOPIC) - .withStartTimestamp(startTime) - .withEndTimestamp(endExpectedTime) - .withPublishTime(); - testPipeline.apply(reader).apply(ParDo.of(new PulsarRecordsMetric())); - - PipelineResult pipelineResult = testPipeline.run(); - MetricQueryResults metrics = - pipelineResult - .metrics() - .queryMetrics( - MetricsFilter.builder() - .addNameFilter( - MetricNameFilter.named( - PulsarIOTest.class.getName(), "PulsarRecordsCounter")) - .build()); - long recordsCount = 0; - for (MetricResult metric : metrics.getCounters()) { - if (metric - .getName() - .toString() - .equals("org.apache.beam.sdk.io.pulsar.PulsarIOTest:PulsarRecordsCounter")) { - recordsCount = metric.getAttempted(); - break; - } - } - assertEquals(inputsMock.size(), (int) recordsCount); - - } catch (PulsarClientException e) { - LOG.error(e.getMessage()); - } + public void testRead() { + + PCollection pcoll = + pipeline + .apply( + PulsarIO.read() + .withTopic(TEST_TOPIC) + .withPulsarClient((ignored -> newFakeClient()))) + .apply( + MapElements.into(TypeDescriptor.of(Integer.class)) + .via(m -> (int) m.getMessageId()[1])); + PAssert.that(pcoll) + .satisfies( + iterable -> { + List result = new ArrayList(); + iterable.forEach(result::add); + Assert.assertArrayEquals( + result.toArray(), new Integer[] {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}); + return null; + }); + pipeline.run(); } @Test - public void testWriteFromTopic() { - try { - PulsarIO.Write writer = - PulsarIO.write().withClientUrl(pulsarContainer.getPulsarBrokerUrl()).withTopic(TOPIC); - int numberOfMessages = 100; - List messages = new ArrayList<>(); - for (int i = 0; i < numberOfMessages; i++) { - messages.add(("PULSAR_WRITER_TEST_" + i).getBytes(StandardCharsets.UTF_8)); - } - testPipeline.apply(Create.of(messages)).apply(writer); - - testPipeline.run(); - - List> receiveMsgs = receiveMessages(); - assertEquals(numberOfMessages, receiveMessages().size()); - for (int i = 0; i < numberOfMessages; i++) { - assertTrue( - new String(receiveMsgs.get(i).getValue(), StandardCharsets.UTF_8) - .equals("PULSAR_WRITER_TEST_" + i)); - } - } catch (Exception e) { - LOG.error(e.getMessage()); - } - } - - public static class PulsarRecordsMetric extends DoFn { - private final Counter counter = - Metrics.counter(PulsarIOTest.class.getName(), "PulsarRecordsCounter"); - - @ProcessElement - public void processElement(ProcessContext context) { - counter.inc(); - context.output(context.element()); - } + public void testExpandReadFailUnserializableType() { + pipeline.apply( + PulsarIO.read(t -> t).withTopic(TEST_TOPIC).withPulsarClient((ignored -> newFakeClient()))); + IllegalStateException exception = + Assert.assertThrows(IllegalStateException.class, pipeline::run); + String errorMsg = exception.getMessage(); + Assert.assertTrue( + "Actual message: " + errorMsg, + exception.getMessage().contains("Unable to return a default Coder for PulsarIO.Read")); + pipeline.enableAbandonedNodeEnforcement(false); } } diff --git a/sdks/java/io/pulsar/src/test/java/org/apache/beam/sdk/io/pulsar/ReadFromPulsarDoFnTest.java b/sdks/java/io/pulsar/src/test/java/org/apache/beam/sdk/io/pulsar/ReadFromPulsarDoFnTest.java index 273a1915d2bb..adfcbc98c56c 100644 --- a/sdks/java/io/pulsar/src/test/java/org/apache/beam/sdk/io/pulsar/ReadFromPulsarDoFnTest.java +++ b/sdks/java/io/pulsar/src/test/java/org/apache/beam/sdk/io/pulsar/ReadFromPulsarDoFnTest.java @@ -46,23 +46,19 @@ public class ReadFromPulsarDoFnTest { public static final String TOPIC = "PULSARIO_READFROMPULSAR_TEST"; public static final int NUMBEROFMESSAGES = 100; - private final ReadFromPulsarDoFn dofnInstance = new ReadFromPulsarDoFn(readSourceDescriptor()); - public FakePulsarReader fakePulsarReader = new FakePulsarReader(TOPIC, NUMBEROFMESSAGES); + private final NaiveReadFromPulsarDoFn dofnInstance = + new NaiveReadFromPulsarDoFn<>(readSourceDescriptor()); + public FakePulsarReader fakePulsarReader = + new FakePulsarReader(TOPIC, NUMBEROFMESSAGES, Instant.now().getMillis()); private FakePulsarClient fakePulsarClient = new FakePulsarClient(fakePulsarReader); - private PulsarIO.Read readSourceDescriptor() { + private PulsarIO.Read readSourceDescriptor() { return PulsarIO.read() .withClientUrl(SERVICE_URL) .withTopic(TOPIC) .withAdminUrl(ADMIN_URL) .withPublishTime() - .withPulsarClient( - new SerializableFunction() { - @Override - public PulsarClient apply(String input) { - return fakePulsarClient; - } - }); + .withPulsarClient((SerializableFunction) ignored -> fakePulsarClient); } @Before @@ -76,8 +72,7 @@ public void testInitialRestrictionWhenHasStartOffset() throws Exception { long expectedStartOffset = 0; OffsetRange result = dofnInstance.getInitialRestriction( - PulsarSourceDescriptor.of( - TOPIC, expectedStartOffset, null, null, SERVICE_URL, ADMIN_URL)); + PulsarSourceDescriptor.of(TOPIC, expectedStartOffset, null, null)); assertEquals(new OffsetRange(expectedStartOffset, Long.MAX_VALUE), result); } @@ -86,8 +81,7 @@ public void testInitialRestrictionWithConsumerPosition() throws Exception { long expectedStartOffset = Instant.now().getMillis(); OffsetRange result = dofnInstance.getInitialRestriction( - PulsarSourceDescriptor.of( - TOPIC, expectedStartOffset, null, null, SERVICE_URL, ADMIN_URL)); + PulsarSourceDescriptor.of(TOPIC, expectedStartOffset, null, null)); assertEquals(new OffsetRange(expectedStartOffset, Long.MAX_VALUE), result); } @@ -97,7 +91,7 @@ public void testInitialRestrictionWithConsumerEndPosition() throws Exception { long endOffset = fakePulsarReader.getEndTimestamp(); OffsetRange result = dofnInstance.getInitialRestriction( - PulsarSourceDescriptor.of(TOPIC, startOffset, endOffset, null, SERVICE_URL, ADMIN_URL)); + PulsarSourceDescriptor.of(TOPIC, startOffset, endOffset, null)); assertEquals(new OffsetRange(startOffset, endOffset), result); } @@ -108,9 +102,9 @@ public void testProcessElement() throws Exception { long endOffset = fakePulsarReader.getEndTimestamp(); OffsetRangeTracker tracker = new OffsetRangeTracker(new OffsetRange(startOffset, endOffset)); PulsarSourceDescriptor descriptor = - PulsarSourceDescriptor.of(TOPIC, startOffset, endOffset, null, SERVICE_URL, ADMIN_URL); + PulsarSourceDescriptor.of(TOPIC, startOffset, endOffset, null); DoFn.ProcessContinuation result = - dofnInstance.processElement(descriptor, tracker, null, (DoFn.OutputReceiver) receiver); + dofnInstance.processElement(descriptor, tracker, null, receiver); int expectedResultWithoutCountingLastOffset = NUMBEROFMESSAGES - 1; assertEquals(DoFn.ProcessContinuation.stop(), result); assertEquals(expectedResultWithoutCountingLastOffset, receiver.getOutputs().size()); @@ -120,13 +114,11 @@ public void testProcessElement() throws Exception { public void testProcessElementWhenEndMessageIdIsDefined() throws Exception { MockOutputReceiver receiver = new MockOutputReceiver(); OffsetRangeTracker tracker = new OffsetRangeTracker(new OffsetRange(0L, Long.MAX_VALUE)); - MessageId endMessageId = DefaultImplementation.newMessageId(50L, 50L, 50); + MessageId endMessageId = + DefaultImplementation.getDefaultImplementation().newMessageId(50L, 50L, 50); DoFn.ProcessContinuation result = dofnInstance.processElement( - PulsarSourceDescriptor.of(TOPIC, null, null, endMessageId, SERVICE_URL, ADMIN_URL), - tracker, - null, - (DoFn.OutputReceiver) receiver); + PulsarSourceDescriptor.of(TOPIC, null, null, endMessageId), tracker, null, receiver); assertEquals(DoFn.ProcessContinuation.stop(), result); assertEquals(50, receiver.getOutputs().size()); } @@ -138,10 +130,7 @@ public void testProcessElementWithEmptyRecords() throws Exception { OffsetRangeTracker tracker = new OffsetRangeTracker(new OffsetRange(0L, Long.MAX_VALUE)); DoFn.ProcessContinuation result = dofnInstance.processElement( - PulsarSourceDescriptor.of(TOPIC, null, null, null, SERVICE_URL, ADMIN_URL), - tracker, - null, - (DoFn.OutputReceiver) receiver); + PulsarSourceDescriptor.of(TOPIC, null, null, null), tracker, null, receiver); assertEquals(DoFn.ProcessContinuation.resume(), result); assertTrue(receiver.getOutputs().isEmpty()); } @@ -153,10 +142,7 @@ public void testProcessElementWhenHasReachedEndTopic() throws Exception { OffsetRangeTracker tracker = new OffsetRangeTracker(new OffsetRange(0L, Long.MAX_VALUE)); DoFn.ProcessContinuation result = dofnInstance.processElement( - PulsarSourceDescriptor.of(TOPIC, null, null, null, SERVICE_URL, ADMIN_URL), - tracker, - null, - (DoFn.OutputReceiver) receiver); + PulsarSourceDescriptor.of(TOPIC, null, null, null), tracker, null, receiver); assertEquals(DoFn.ProcessContinuation.stop(), result); } diff --git a/sdks/java/io/sparkreceiver/3/src/test/java/org/apache/beam/sdk/io/sparkreceiver/RabbitMqReceiverWithOffset.java b/sdks/java/io/sparkreceiver/3/src/test/java/org/apache/beam/sdk/io/sparkreceiver/RabbitMqReceiverWithOffset.java index da8f1dde841e..730001ffe459 100644 --- a/sdks/java/io/sparkreceiver/3/src/test/java/org/apache/beam/sdk/io/sparkreceiver/RabbitMqReceiverWithOffset.java +++ b/sdks/java/io/sparkreceiver/3/src/test/java/org/apache/beam/sdk/io/sparkreceiver/RabbitMqReceiverWithOffset.java @@ -177,7 +177,7 @@ public void handleDelivery( messageConsumer.accept(sMessage); } } catch (Exception e) { - LOG.error("Can't read from RabbitMQ: {}", e.getMessage()); + LOG.error("Can't read from RabbitMQ.", e); } } } diff --git a/sdks/java/io/sparkreceiver/3/src/test/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIOIT.java b/sdks/java/io/sparkreceiver/3/src/test/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIOIT.java index b7af2054236e..32258934d0d9 100644 --- a/sdks/java/io/sparkreceiver/3/src/test/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIOIT.java +++ b/sdks/java/io/sparkreceiver/3/src/test/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIOIT.java @@ -315,7 +315,7 @@ public void testSparkReceiverIOReadsInStreamingWithOffset() throws IOException { try { writeToRabbitMq(messages); } catch (Exception e) { - LOG.error("Can not write to rabbit {}", e.getMessage()); + LOG.error("Can not write to rabbit.", e); fail(); } LOG.info(sourceOptions.numRecords + " records were successfully written to RabbitMQ"); diff --git a/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/SplunkEventWriter.java b/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/SplunkEventWriter.java index 615d4e932f4d..a86fdff608d2 100644 --- a/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/SplunkEventWriter.java +++ b/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/SplunkEventWriter.java @@ -219,7 +219,7 @@ public void setup() { | KeyManagementException | IOException | CertificateException e) { - LOG.error("Error creating HttpEventPublisher: {}", e.getMessage()); + LOG.error("Error creating HttpEventPublisher.", e); throw new RuntimeException(e); } } @@ -273,7 +273,7 @@ public void tearDown() { LOG.info("Successfully closed HttpEventPublisher"); } catch (IOException e) { - LOG.warn("Received exception while closing HttpEventPublisher: {}", e.getMessage()); + LOG.warn("Received exception while closing HttpEventPublisher.", e); } } } @@ -347,7 +347,7 @@ private void flush( flushWriteFailures(events, e.getStatusMessage(), e.getStatusCode(), receiver); } catch (IOException ioe) { - LOG.error("Error writing to Splunk: {}", ioe.getMessage()); + LOG.error("Error writing to Splunk.", ioe); UNSUCCESSFUL_WRITE_LATENCY_MS.update(nanosToMillis(System.nanoTime() - startTime)); FAILED_WRITES.inc(countState.read()); INVALID_REQUESTS.inc(); diff --git a/sdks/java/io/synthetic/src/test/java/org/apache/beam/sdk/io/synthetic/BundleSplitterTest.java b/sdks/java/io/synthetic/src/test/java/org/apache/beam/sdk/io/synthetic/BundleSplitterTest.java index f37ac4614d83..15f3ff4f5dd9 100644 --- a/sdks/java/io/synthetic/src/test/java/org/apache/beam/sdk/io/synthetic/BundleSplitterTest.java +++ b/sdks/java/io/synthetic/src/test/java/org/apache/beam/sdk/io/synthetic/BundleSplitterTest.java @@ -69,7 +69,7 @@ public void bundlesShouldBeEvenForConstDistribution() { bundleSizes.stream() .map(range -> range.getTo() - range.getFrom()) - .forEach(size -> assertEquals(expectedBundleSize, size.intValue())); + .forEach(size -> assertEquals(expectedBundleSize, size.longValue())); } @Test @@ -83,7 +83,7 @@ public void bundleSizesShouldBeProportionalToTheOneSuggestedInBundleSizeDistribu bundleSizes.stream() .map(range -> range.getTo() - range.getFrom()) - .forEach(size -> assertEquals(expectedBundleSize, size.intValue())); + .forEach(size -> assertEquals(expectedBundleSize, size.longValue())); } @Test diff --git a/sdks/java/javadoc/overview.html b/sdks/java/javadoc/overview.html index 66d4ab613781..8c0d15de121e 100644 --- a/sdks/java/javadoc/overview.html +++ b/sdks/java/javadoc/overview.html @@ -37,9 +37,5 @@

  • minor version for new functionality added in a backward-compatible manner
  • incremental version for forward-compatible bug fixes
  • - -

    Please note that APIs marked - {@link org.apache.beam.sdk.annotations.Experimental @Experimental} - may change at any point and are not guaranteed to remain compatible across versions.

    diff --git a/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/Managed.java b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/Managed.java index 06aed06c71c4..cda84629a7d7 100644 --- a/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/Managed.java +++ b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/Managed.java @@ -96,6 +96,7 @@ public class Managed { public static final String ICEBERG_CDC = "iceberg_cdc"; public static final String KAFKA = "kafka"; public static final String BIGQUERY = "bigquery"; + public static final String POSTGRES = "postgres"; // Supported SchemaTransforms public static final Map READ_TRANSFORMS = @@ -104,12 +105,14 @@ public class Managed { .put(ICEBERG_CDC, getUrn(ExternalTransforms.ManagedTransforms.Urns.ICEBERG_CDC_READ)) .put(KAFKA, getUrn(ExternalTransforms.ManagedTransforms.Urns.KAFKA_READ)) .put(BIGQUERY, getUrn(ExternalTransforms.ManagedTransforms.Urns.BIGQUERY_READ)) + .put(POSTGRES, getUrn(ExternalTransforms.ManagedTransforms.Urns.POSTGRES_READ)) .build(); public static final Map WRITE_TRANSFORMS = ImmutableMap.builder() .put(ICEBERG, getUrn(ExternalTransforms.ManagedTransforms.Urns.ICEBERG_WRITE)) .put(KAFKA, getUrn(ExternalTransforms.ManagedTransforms.Urns.KAFKA_WRITE)) .put(BIGQUERY, getUrn(ExternalTransforms.ManagedTransforms.Urns.BIGQUERY_WRITE)) + .put(POSTGRES, getUrn(ExternalTransforms.ManagedTransforms.Urns.POSTGRES_WRITE)) .build(); /** diff --git a/sdks/java/testing/junit/build.gradle b/sdks/java/testing/junit/build.gradle new file mode 100644 index 000000000000..977dbd2cd344 --- /dev/null +++ b/sdks/java/testing/junit/build.gradle @@ -0,0 +1,50 @@ +/* + * 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. + */ + +plugins { id 'org.apache.beam.module' } + +applyJavaNature( + exportJavadoc: false, + automaticModuleName: 'org.apache.beam.sdk.testing.junit', + archivesBaseName: 'beam-sdks-java-testing-junit' +) + +description = "Apache Beam :: SDKs :: Java :: Testing :: JUnit" + +dependencies { + implementation enforcedPlatform(library.java.google_cloud_platform_libraries_bom) + implementation project(path: ":sdks:java:core", configuration: "shadow") + implementation library.java.vendored_guava_32_1_2_jre + // Needed to resolve TestPipeline's JUnit 4 TestRule type and @Category at compile time, + // but should not leak to consumers at runtime. + provided library.java.junit + + // JUnit 5 API needed to compile the extension; not packaged for consumers of core. + provided library.java.jupiter_api + + testImplementation project(path: ":sdks:java:core", configuration: "shadow") + testImplementation library.java.jupiter_api + testImplementation library.java.junit + testRuntimeOnly library.java.jupiter_engine + testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") +} + +// This module runs JUnit 5 tests using the JUnit Platform. +test { + useJUnitPlatform() +} diff --git a/sdks/java/testing/junit/src/main/java/org/apache/beam/sdk/testing/TestPipelineExtension.java b/sdks/java/testing/junit/src/main/java/org/apache/beam/sdk/testing/TestPipelineExtension.java new file mode 100644 index 000000000000..ea0e1f3eac9b --- /dev/null +++ b/sdks/java/testing/junit/src/main/java/org/apache/beam/sdk/testing/TestPipelineExtension.java @@ -0,0 +1,213 @@ +/* + * 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. + */ +package org.apache.beam.sdk.testing; + +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; + +import java.lang.annotation.Annotation; +import java.lang.reflect.Method; +import java.util.Arrays; +import java.util.Optional; +import org.apache.beam.sdk.options.ApplicationNameOptions; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.testing.TestPipeline.PipelineAbandonedNodeEnforcement; +import org.apache.beam.sdk.testing.TestPipeline.PipelineRunEnforcement; +import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.extension.AfterEachCallback; +import org.junit.jupiter.api.extension.BeforeEachCallback; +import org.junit.jupiter.api.extension.ExtensionContext; +import org.junit.jupiter.api.extension.ParameterContext; +import org.junit.jupiter.api.extension.ParameterResolver; + +/** + * JUnit 5 extension for {@link TestPipeline} that provides the same functionality as the JUnit 4 + * {@link org.junit.rules.TestRule} implementation. + * + *

    Use this extension to test pipelines in JUnit 5: + * + *

    
    + * {@literal @}ExtendWith(TestPipelineExtension.class)
    + * class MyPipelineTest {
    + *   {@literal @}Test
    + *   {@literal @}Category(NeedsRunner.class)
    + *   void myPipelineTest(TestPipeline pipeline) {
    + *     final PCollection<String> pCollection = pipeline.apply(...)
    + *     PAssert.that(pCollection).containsInAnyOrder(...);
    + *     pipeline.run();
    + *   }
    + * }
    + * 
    + * + *

    You can also create the extension yourself for more control: + * + *

    
    + * class MyPipelineTest {
    + *   {@literal @}RegisterExtension
    + *   final TestPipelineExtension pipeline = TestPipelineExtension.create();
    + *
    + *   {@literal @}Test
    + *   void testUsingPipeline() {
    + *     pipeline.apply(...);
    + *     pipeline.run();
    + *   }
    + * }
    + * 
    + */ +public class TestPipelineExtension + implements BeforeEachCallback, AfterEachCallback, ParameterResolver { + + private static final ExtensionContext.Namespace NAMESPACE = + ExtensionContext.Namespace.create(TestPipelineExtension.class); + private static final String PIPELINE_KEY = "testPipeline"; + private static final String ENFORCEMENT_KEY = "enforcement"; + + /** Creates a new TestPipelineExtension with default options. */ + public static TestPipelineExtension create() { + return new TestPipelineExtension(); + } + + /** Creates a new TestPipelineExtension with custom options. */ + public static TestPipelineExtension fromOptions(PipelineOptions options) { + return new TestPipelineExtension(options); + } + + private TestPipeline testPipeline; + + /** Creates a TestPipelineExtension with default options. */ + public TestPipelineExtension() { + this.testPipeline = TestPipeline.create(); + } + + /** Creates a TestPipelineExtension with custom options. */ + public TestPipelineExtension(PipelineOptions options) { + this.testPipeline = TestPipeline.fromOptions(options); + } + + @Override + public boolean supportsParameter( + ParameterContext parameterContext, ExtensionContext extensionContext) { + return parameterContext.getParameter().getType() == TestPipeline.class; + } + + @Override + public Object resolveParameter( + ParameterContext parameterContext, ExtensionContext extensionContext) { + if (this.testPipeline == null) { + return getOrCreateTestPipeline(extensionContext); + } else { + return this.testPipeline; + } + } + + @Override + public void beforeEach(ExtensionContext context) throws Exception { + TestPipeline pipeline; + + if (this.testPipeline != null) { + pipeline = this.testPipeline; + } else { + pipeline = getOrCreateTestPipeline(context); + } + + // Set application name based on test method + String appName = getAppName(context); + pipeline.getOptions().as(ApplicationNameOptions.class).setAppName(appName); + + // Set up enforcement based on annotations + setDeducedEnforcementLevel(context, pipeline); + } + + @Override + public void afterEach(ExtensionContext context) throws Exception { + Optional enforcement = getEnforcement(context); + if (enforcement.isPresent()) { + enforcement.get().afterUserCodeFinished(); + } + } + + private TestPipeline getOrCreateTestPipeline(ExtensionContext context) { + return context + .getStore(NAMESPACE) + .getOrComputeIfAbsent(PIPELINE_KEY, key -> TestPipeline.create(), TestPipeline.class); + } + + private Optional getEnforcement(ExtensionContext context) { + return Optional.ofNullable( + context.getStore(NAMESPACE).get(ENFORCEMENT_KEY, PipelineRunEnforcement.class)); + } + + private void setEnforcement(ExtensionContext context, PipelineRunEnforcement enforcement) { + context.getStore(NAMESPACE).put(ENFORCEMENT_KEY, enforcement); + } + + private String getAppName(ExtensionContext context) { + String className = context.getTestClass().map(Class::getSimpleName).orElse("UnknownClass"); + String methodName = context.getTestMethod().map(Method::getName).orElse("unknownMethod"); + return className + "-" + methodName; + } + + private void setDeducedEnforcementLevel(ExtensionContext context, TestPipeline pipeline) { + // If enforcement level has not been set, do auto-inference + if (!getEnforcement(context).isPresent()) { + boolean annotatedWithNeedsRunner = hasNeedsRunnerAnnotation(context); + + PipelineOptions options = pipeline.getOptions(); + boolean crashingRunner = CrashingRunner.class.isAssignableFrom(options.getRunner()); + + checkState( + !(annotatedWithNeedsRunner && crashingRunner), + "The test was annotated with a [@%s] / [@%s] while the runner " + + "was set to [%s]. Please re-check your configuration.", + NeedsRunner.class.getSimpleName(), + ValidatesRunner.class.getSimpleName(), + CrashingRunner.class.getSimpleName()); + + if (annotatedWithNeedsRunner || !crashingRunner) { + setEnforcement(context, new PipelineAbandonedNodeEnforcement(pipeline)); + } + } + } + + private boolean hasNeedsRunnerAnnotation(ExtensionContext context) { + // Check method annotations + Method testMethod = context.getTestMethod().orElse(null); + if (testMethod != null) { + if (hasNeedsRunnerCategory(testMethod.getAnnotations())) { + return true; + } + } + + // Check class annotations + Class testClass = context.getTestClass().orElse(null); + if (testClass != null) { + if (hasNeedsRunnerCategory(testClass.getAnnotations())) { + return true; + } + } + + return false; + } + + private boolean hasNeedsRunnerCategory(Annotation[] annotations) { + return Arrays.stream(annotations) + .filter(annotation -> annotation instanceof Category) + .map(annotation -> (Category) annotation) + .flatMap(category -> Arrays.stream(category.value())) + .anyMatch(categoryClass -> NeedsRunner.class.isAssignableFrom(categoryClass)); + } +} diff --git a/sdks/java/testing/junit/src/main/java/org/apache/beam/sdk/testing/package-info.java b/sdks/java/testing/junit/src/main/java/org/apache/beam/sdk/testing/package-info.java new file mode 100644 index 000000000000..2909111bfec8 --- /dev/null +++ b/sdks/java/testing/junit/src/main/java/org/apache/beam/sdk/testing/package-info.java @@ -0,0 +1,19 @@ +/* + * 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. + */ +/** JUnit 5 testing support for Apache Beam Java SDK. */ +package org.apache.beam.sdk.testing; diff --git a/sdks/java/testing/junit/src/test/java/org/apache/beam/sdk/testing/TestPipelineExtensionAdvancedTest.java b/sdks/java/testing/junit/src/test/java/org/apache/beam/sdk/testing/TestPipelineExtensionAdvancedTest.java new file mode 100644 index 000000000000..b792204a945e --- /dev/null +++ b/sdks/java/testing/junit/src/test/java/org/apache/beam/sdk/testing/TestPipelineExtensionAdvancedTest.java @@ -0,0 +1,88 @@ +/* + * 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. + */ +package org.apache.beam.sdk.testing; + +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import org.apache.beam.sdk.options.ApplicationNameOptions; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; +import org.junit.experimental.categories.Category; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; + +/** Advanced tests for {@link TestPipelineExtension} demonstrating comprehensive functionality. */ +@ExtendWith(TestPipelineExtension.class) +public class TestPipelineExtensionAdvancedTest { + + @Test + public void testApplicationNameIsSet(TestPipeline pipeline) { + String appName = pipeline.getOptions().as(ApplicationNameOptions.class).getAppName(); + assertNotNull(appName); + assertTrue(appName.contains("TestPipelineExtensionAdvancedTest")); + assertTrue(appName.contains("testApplicationNameIsSet")); + } + + @Test + public void testMultipleTransforms(TestPipeline pipeline) { + PCollection input = pipeline.apply("Create", Create.of("a", "b", "c")); + + PCollection output = + input.apply( + "Transform", + ParDo.of( + new DoFn() { + @ProcessElement + public void processElement(ProcessContext c) { + c.output(c.element().toUpperCase()); + } + })); + + PAssert.that(output).containsInAnyOrder("A", "B", "C"); + pipeline.run(); + } + + @Test + @Category(ValidatesRunner.class) + public void testWithValidatesRunnerCategory(TestPipeline pipeline) { + // This test demonstrates that @Category annotations work with JUnit 5 + PCollection numbers = pipeline.apply("Create", Create.of(1, 2, 3, 4, 5)); + PAssert.that(numbers).containsInAnyOrder(1, 2, 3, 4, 5); + pipeline.run(); + } + + @Test + public void testPipelineInstancesAreIsolated(TestPipeline pipeline1) { + // Each test method gets its own pipeline instance + assertNotNull(pipeline1); + pipeline1.apply("Create", Create.of("test")); + // Don't run the pipeline - test should still pass due to auto-run functionality + } + + @Test + public void testAnotherPipelineInstance(TestPipeline pipeline2) { + // This should be a different instance from the previous test + assertNotNull(pipeline2); + PCollection data = pipeline2.apply("Create", Create.of("different", "data")); + PAssert.that(data).containsInAnyOrder("different", "data"); + pipeline2.run(); + } +} diff --git a/sdks/java/testing/junit/src/test/java/org/apache/beam/sdk/testing/TestPipelineExtensionTest.java b/sdks/java/testing/junit/src/test/java/org/apache/beam/sdk/testing/TestPipelineExtensionTest.java new file mode 100644 index 000000000000..bc6d5741bac0 --- /dev/null +++ b/sdks/java/testing/junit/src/test/java/org/apache/beam/sdk/testing/TestPipelineExtensionTest.java @@ -0,0 +1,56 @@ +/* + * 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. + */ +package org.apache.beam.sdk.testing; + +import static org.junit.jupiter.api.Assertions.assertNotNull; + +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.values.PCollection; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; + +/** Tests for {@link TestPipelineExtension} to demonstrate JUnit 5 integration. */ +@ExtendWith(TestPipelineExtension.class) +public class TestPipelineExtensionTest { + + @Test + public void testPipelineInjection(TestPipeline pipeline) { + // Verify that the pipeline is injected and not null + assertNotNull(pipeline); + assertNotNull(pipeline.getOptions()); + } + + @Test + public void testBasicPipelineExecution(TestPipeline pipeline) { + // Create a simple pipeline + PCollection input = pipeline.apply("Create", Create.of("hello", "world")); + + // Use PAssert to verify the output + PAssert.that(input).containsInAnyOrder("hello", "world"); + + // Run the pipeline + pipeline.run(); + } + + @Test + public void testEmptyPipeline(TestPipeline pipeline) { + // Test that an empty pipeline doesn't cause issues + assertNotNull(pipeline); + // The extension should handle empty pipelines gracefully + } +} diff --git a/sdks/java/testing/test-utils/src/main/java/org/apache/beam/sdk/testutils/publishing/InfluxDBPublisher.java b/sdks/java/testing/test-utils/src/main/java/org/apache/beam/sdk/testutils/publishing/InfluxDBPublisher.java index 30e72fd53dad..d7034620ed45 100644 --- a/sdks/java/testing/test-utils/src/main/java/org/apache/beam/sdk/testutils/publishing/InfluxDBPublisher.java +++ b/sdks/java/testing/test-utils/src/main/java/org/apache/beam/sdk/testutils/publishing/InfluxDBPublisher.java @@ -158,7 +158,7 @@ private static void publishWithCheck(final InfluxDBSettings settings, final Stri postRequest.setEntity(new GzipCompressingEntity(new ByteArrayEntity(data.getBytes(UTF_8)))); executeWithVerification(postRequest, builder); } catch (Exception exception) { - LOG.warn("Unable to publish metrics due to error: {}", exception.getMessage()); + LOG.warn("Unable to publish metrics due to error", exception); } } else { LOG.warn("Missing setting InfluxDB database. Metrics won't be published."); diff --git a/sdks/python/apache_beam/coders/coder_impl.pxd b/sdks/python/apache_beam/coders/coder_impl.pxd index 27cffe7b62df..6238167bc2d7 100644 --- a/sdks/python/apache_beam/coders/coder_impl.pxd +++ b/sdks/python/apache_beam/coders/coder_impl.pxd @@ -81,6 +81,7 @@ cdef class FastPrimitivesCoderImpl(StreamCoderImpl): cdef CoderImpl iterable_coder_impl cdef object requires_deterministic_step_label cdef bint warn_deterministic_fallback + cdef bint force_use_dill @cython.locals(dict_value=dict, int_value=libc.stdint.int64_t, unicode_value=unicode) @@ -88,9 +89,12 @@ cdef class FastPrimitivesCoderImpl(StreamCoderImpl): @cython.locals(t=int) cpdef decode_from_stream(self, InputStream stream, bint nested) cdef encode_special_deterministic(self, value, OutputStream stream) + cdef encode_type_2_67_0(self, t, OutputStream stream) cdef encode_type(self, t, OutputStream stream) cdef decode_type(self, InputStream stream) +cdef dict _pickled_types + cdef dict _unpickled_types diff --git a/sdks/python/apache_beam/coders/coder_impl.py b/sdks/python/apache_beam/coders/coder_impl.py index 807d083d8a38..c2241268b8ba 100644 --- a/sdks/python/apache_beam/coders/coder_impl.py +++ b/sdks/python/apache_beam/coders/coder_impl.py @@ -50,7 +50,6 @@ from typing import Tuple from typing import Type -import dill import numpy as np from fastavro import parse_schema from fastavro import schemaless_reader @@ -58,6 +57,7 @@ from apache_beam.coders import observable from apache_beam.coders.avro_record import AvroRecord +from apache_beam.internal import cloudpickle_pickler from apache_beam.typehints.schemas import named_tuple_from_schema from apache_beam.utils import proto_utils from apache_beam.utils import windowed_value @@ -71,6 +71,11 @@ except ImportError: dataclasses = None # type: ignore +try: + import dill +except ImportError: + dill = None + if TYPE_CHECKING: import proto from apache_beam.transforms import userstate @@ -354,14 +359,30 @@ def decode(self, value): _ITERABLE_LIKE_TYPES = set() # type: Set[Type] +def _verify_dill_compat(): + base_error = ( + "This pipeline runs with the pipeline option " + "--update_compatibility_version=2.67.0 or earlier. " + "When running with this option on SDKs 2.68.0 or " + "later, you must ensure dill==0.3.1.1 is installed.") + if not dill: + raise RuntimeError(base_error + ". Dill is not installed.") + if dill.__version__ != "0.3.1.1": + raise RuntimeError(base_error + f". Found dill version '{dill.__version__}") + + class FastPrimitivesCoderImpl(StreamCoderImpl): """For internal use only; no backwards-compatibility guarantees.""" def __init__( - self, fallback_coder_impl, requires_deterministic_step_label=None): + self, + fallback_coder_impl, + requires_deterministic_step_label=None, + force_use_dill=False): self.fallback_coder_impl = fallback_coder_impl self.iterable_coder_impl = IterableCoderImpl(self) self.requires_deterministic_step_label = requires_deterministic_step_label self.warn_deterministic_fallback = True + self.force_use_dill = force_use_dill @staticmethod def register_iterable_like_type(t): @@ -525,10 +546,27 @@ def _deterministic_encoding_error_msg(self, value): "please provide a type hint for the input of '%s'" % (value, type(value), self.requires_deterministic_step_label)) + def encode_type_2_67_0(self, t, stream): + """ + Encode special type with <=2.67.0 compatibility. + """ + if t not in _pickled_types: + _verify_dill_compat() + _pickled_types[t] = dill.dumps(t) + stream.write(_pickled_types[t], True) + def encode_type(self, t, stream): - stream.write(dill.dumps(t), True) + if self.force_use_dill: + return self.encode_type_2_67_0(t, stream) + + if t not in _pickled_types: + _pickled_types[t] = cloudpickle_pickler.dumps( + t, config=cloudpickle_pickler.NO_DYNAMIC_CLASS_TRACKING_CONFIG) + stream.write(_pickled_types[t], True) def decode_type(self, stream): + if self.force_use_dill: + return _unpickle_type_2_67_0(stream.read_all(True)) return _unpickle_type(stream.read_all(True)) def decode_from_stream(self, stream, nested): @@ -586,22 +624,39 @@ def decode_from_stream(self, stream, nested): raise ValueError('Unknown type tag %x' % t) +_pickled_types = {} # type: Dict[type, bytes] _unpickled_types = {} # type: Dict[bytes, type] -def _unpickle_type(bs): +def _unpickle_type_2_67_0(bs): + """ + Decode special type with <=2.67.0 compatibility. + """ t = _unpickled_types.get(bs, None) if t is None: + _verify_dill_compat() t = _unpickled_types[bs] = dill.loads(bs) # Fix unpicklable anonymous named tuples for Python 3.6. if t.__base__ is tuple and hasattr(t, '_fields'): try: pickle.loads(pickle.dumps(t)) except pickle.PicklingError: - t.__reduce__ = lambda self: (_unpickle_named_tuple, (bs, tuple(self))) + t.__reduce__ = lambda self: ( + _unpickle_named_tuple_2_67_0, (bs, tuple(self))) return t +def _unpickle_named_tuple_2_67_0(bs, items): + return _unpickle_type_2_67_0(bs)(*items) + + +def _unpickle_type(bs): + if not _unpickled_types.get(bs, None): + _unpickled_types[bs] = cloudpickle_pickler.loads(bs) + + return _unpickled_types[bs] + + def _unpickle_named_tuple(bs, items): return _unpickle_type(bs)(*items) @@ -837,6 +892,7 @@ def decode_from_stream(self, in_, nested): if IntervalWindow is None: from apache_beam.transforms.window import IntervalWindow # instantiating with None is not part of the public interface + # pylint: disable=too-many-function-args typed_value = IntervalWindow(None, None) # type: ignore[arg-type] typed_value._end_micros = ( 1000 * self._to_normal_time(in_.read_bigendian_uint64())) diff --git a/sdks/python/apache_beam/coders/coders.py b/sdks/python/apache_beam/coders/coders.py index 2691857bf0a6..fe5728c0f16e 100644 --- a/sdks/python/apache_beam/coders/coders.py +++ b/sdks/python/apache_beam/coders/coders.py @@ -85,9 +85,7 @@ # occurs. from apache_beam.internal.dill_pickler import dill except ImportError: - # We fall back to using the stock dill library in tests that don't use the - # full Python SDK. - import dill + dill = None __all__ = [ 'Coder', @@ -900,6 +898,13 @@ def to_type_hint(self): class DillCoder(_PickleCoderBase): """Coder using dill's pickle functionality.""" + def __init__(self): + if not dill: + raise RuntimeError( + "This pipeline contains a DillCoder which requires " + "the dill package. Install the dill package with the dill extra " + "e.g. apache-beam[dill]") + def _create_impl(self): return coder_impl.CallbackCoderImpl(maybe_dill_dumps, maybe_dill_loads) @@ -911,6 +916,44 @@ def _create_impl(self): cloudpickle_pickler.dumps, cloudpickle_pickler.loads) +class DeterministicFastPrimitivesCoderV2(FastCoder): + """Throws runtime errors when encoding non-deterministic values.""" + def __init__(self, coder, step_label): + self._underlying_coder = coder + self._step_label = step_label + + def _create_impl(self): + + return coder_impl.FastPrimitivesCoderImpl( + self._underlying_coder.get_impl(), + requires_deterministic_step_label=self._step_label, + force_use_dill=False) + + def is_deterministic(self): + # type: () -> bool + return True + + def is_kv_coder(self): + # type: () -> bool + return True + + def key_coder(self): + return self + + def value_coder(self): + return self + + def to_type_hint(self): + return Any + + def to_runner_api_parameter(self, context): + # type: (Optional[PipelineContext]) -> Tuple[str, Any, Sequence[Coder]] + return ( + python_urns.PICKLED_CODER, + google.protobuf.wrappers_pb2.BytesValue(value=serialize_coder(self)), + ()) + + class DeterministicFastPrimitivesCoder(FastCoder): """Throws runtime errors when encoding non-deterministic values.""" def __init__(self, coder, step_label): @@ -920,7 +963,8 @@ def __init__(self, coder, step_label): def _create_impl(self): return coder_impl.FastPrimitivesCoderImpl( self._underlying_coder.get_impl(), - requires_deterministic_step_label=self._step_label) + requires_deterministic_step_label=self._step_label, + force_use_dill=True) def is_deterministic(self): # type: () -> bool @@ -940,6 +984,34 @@ def to_type_hint(self): return Any +def _should_force_use_dill(): + from apache_beam.coders import typecoders + from apache_beam.transforms.util import is_v1_prior_to_v2 + update_compat_version = typecoders.registry.update_compatibility_version + + if not update_compat_version: + return False + + if not is_v1_prior_to_v2(v1=update_compat_version, v2="2.68.0"): + return False + + try: + import dill + assert dill.__version__ == "0.3.1.1" + except Exception as e: + raise RuntimeError("This pipeline runs with the pipeline option " \ + "--update_compatibility_version=2.67.0 or earlier. When running with " \ + "this option on SDKs 2.68.0 or later, you must ensure dill==0.3.1.1 " \ + f"is installed. Error {e}") + return True + + +def _update_compatible_deterministic_fast_primitives_coder(coder, step_label): + if _should_force_use_dill(): + return DeterministicFastPrimitivesCoder(coder, step_label) + return DeterministicFastPrimitivesCoderV2(coder, step_label) + + class FastPrimitivesCoder(FastCoder): """Encodes simple primitives (e.g. str, int) efficiently. @@ -960,7 +1032,8 @@ def as_deterministic_coder(self, step_label, error_message=None): if self.is_deterministic(): return self else: - return DeterministicFastPrimitivesCoder(self, step_label) + return _update_compatible_deterministic_fast_primitives_coder( + self, step_label) def to_type_hint(self): return Any diff --git a/sdks/python/apache_beam/coders/coders_test_common.py b/sdks/python/apache_beam/coders/coders_test_common.py index dbd0a301bb0d..1ae9a32790ac 100644 --- a/sdks/python/apache_beam/coders/coders_test_common.py +++ b/sdks/python/apache_beam/coders/coders_test_common.py @@ -34,6 +34,8 @@ from typing import NamedTuple import pytest +from parameterized import param +from parameterized import parameterized from apache_beam.coders import proto2_coder_test_messages_pb2 as test_message from apache_beam.coders import coders @@ -57,7 +59,13 @@ except ImportError: dataclasses = None # type: ignore +try: + import dill +except ImportError: + dill = None + MyNamedTuple = collections.namedtuple('A', ['x', 'y']) # type: ignore[name-match] +AnotherNamedTuple = collections.namedtuple('AnotherNamedTuple', ['x', 'y']) MyTypedNamedTuple = NamedTuple('MyTypedNamedTuple', [('f1', int), ('f2', str)]) @@ -113,6 +121,7 @@ class UnFrozenDataClass: # These tests need to all be run in the same process due to the asserts # in tearDownClass. @pytest.mark.no_xdist +@pytest.mark.uses_dill class CodersTest(unittest.TestCase): # These class methods ensure that we test each defined coder in both @@ -170,11 +179,17 @@ def tearDownClass(cls): coders.BigIntegerCoder, # tested in DecimalCoder coders.TimestampPrefixingOpaqueWindowCoder, ]) + if not dill: + standard -= set( + [coders.DillCoder, coders.DeterministicFastPrimitivesCoder]) cls.seen_nested -= set( [coders.ProtoCoder, coders.ProtoPlusCoder, CustomCoder]) assert not standard - cls.seen, str(standard - cls.seen) assert not cls.seen_nested - standard, str(cls.seen_nested - standard) + def tearDown(self): + typecoders.registry.update_compatibility_version = None + @classmethod def _observe(cls, coder): cls.seen.add(type(coder)) @@ -230,9 +245,20 @@ def test_memoizing_pickle_coder(self): coder = coders._MemoizingPickleCoder() self.check_coder(coder, *self.test_values) - def test_deterministic_coder(self): + @parameterized.expand([ + param(compat_version=None), + param(compat_version="2.67.0"), + ]) + def test_deterministic_coder(self, compat_version): + + typecoders.registry.update_compatibility_version = compat_version coder = coders.FastPrimitivesCoder() - deterministic_coder = coders.DeterministicFastPrimitivesCoder(coder, 'step') + if not dill and compat_version: + with self.assertRaises(RuntimeError): + coder.as_deterministic_coder(step_label="step") + self.skipTest('Dill not installed') + deterministic_coder = coder.as_deterministic_coder(step_label="step") + self.check_coder(deterministic_coder, *self.test_values_deterministic) for v in self.test_values_deterministic: self.check_coder(coders.TupleCoder((deterministic_coder, )), (v, )) @@ -254,8 +280,16 @@ def test_deterministic_coder(self): self.check_coder(deterministic_coder, test_message.MessageA(field1='value')) + # Skip this test during cloudpickle. Dill monkey patches the __reduce__ + # method for anonymous named tuples (MyNamedTuple) which is not pickleable. + # Since the test is parameterized the type gets colbbered. + if compat_version: + self.check_coder( + deterministic_coder, [MyNamedTuple(1, 2), MyTypedNamedTuple(1, 'a')]) + self.check_coder( - deterministic_coder, [MyNamedTuple(1, 2), MyTypedNamedTuple(1, 'a')]) + deterministic_coder, + [AnotherNamedTuple(1, 2), MyTypedNamedTuple(1, 'a')]) if dataclasses is not None: self.check_coder(deterministic_coder, FrozenDataClass(1, 2)) @@ -265,9 +299,10 @@ def test_deterministic_coder(self): with self.assertRaises(TypeError): self.check_coder( deterministic_coder, FrozenDataClass(UnFrozenDataClass(1, 2), 3)) - with self.assertRaises(TypeError): - self.check_coder( - deterministic_coder, MyNamedTuple(UnFrozenDataClass(1, 2), 3)) + with self.assertRaises(TypeError): + self.check_coder( + deterministic_coder, + AnotherNamedTuple(UnFrozenDataClass(1, 2), 3)) self.check_coder(deterministic_coder, list(MyEnum)) self.check_coder(deterministic_coder, list(MyIntEnum)) @@ -286,7 +321,40 @@ def test_deterministic_coder(self): 1: 'x', 'y': 2 })) + @parameterized.expand([ + param(compat_version=None), + param(compat_version="2.67.0"), + ]) + def test_deterministic_map_coder_is_update_compatible(self, compat_version): + typecoders.registry.update_compatibility_version = compat_version + values = [{ + MyTypedNamedTuple(i, 'a'): MyTypedNamedTuple('a', i) + for i in range(10) + }] + + coder = coders.MapCoder( + coders.FastPrimitivesCoder(), coders.FastPrimitivesCoder()) + + if not dill and compat_version: + with self.assertRaises(RuntimeError): + coder.as_deterministic_coder(step_label="step") + self.skipTest('Dill not installed') + + deterministic_coder = coder.as_deterministic_coder(step_label="step") + + assert isinstance( + deterministic_coder._key_coder, + coders.DeterministicFastPrimitivesCoderV2 + if not compat_version else coders.DeterministicFastPrimitivesCoder) + + self.check_coder(deterministic_coder, *values) + def test_dill_coder(self): + if not dill: + with self.assertRaises(RuntimeError): + coders.DillCoder() + self.skipTest('Dill not installed') + cell_value = (lambda x: lambda: x)(0).__closure__[0] self.check_coder(coders.DillCoder(), 'a', 1, cell_value) self.check_coder( @@ -610,15 +678,23 @@ def test_param_windowed_value_coder(self): 1, (window.IntervalWindow(11, 21), ), PaneInfo(True, False, 1, 2, 3)))) - def test_cross_process_encoding_of_special_types_is_deterministic(self): + @parameterized.expand([ + param(compat_version=None), + param(compat_version="2.67.0"), + ]) + def test_cross_process_encoding_of_special_types_is_deterministic( + self, compat_version): """Test cross-process determinism for all special deterministic types""" + if compat_version: + pytest.importorskip("dill") if sys.executable is None: self.skipTest('No Python interpreter found') + typecoders.registry.update_compatibility_version = compat_version # pylint: disable=line-too-long script = textwrap.dedent( - '''\ + f'''\ import pickle import sys import collections @@ -626,13 +702,19 @@ def test_cross_process_encoding_of_special_types_is_deterministic(self): import logging from apache_beam.coders import coders - from apache_beam.coders import proto2_coder_test_messages_pb2 as test_message - from typing import NamedTuple + from apache_beam.coders import typecoders + from apache_beam.coders.coders_test_common import MyNamedTuple + from apache_beam.coders.coders_test_common import MyTypedNamedTuple + from apache_beam.coders.coders_test_common import MyEnum + from apache_beam.coders.coders_test_common import MyIntEnum + from apache_beam.coders.coders_test_common import MyIntFlag + from apache_beam.coders.coders_test_common import MyFlag + from apache_beam.coders.coders_test_common import DefinesGetState + from apache_beam.coders.coders_test_common import DefinesGetAndSetState + from apache_beam.coders.coders_test_common import FrozenDataClass + - try: - import dataclasses - except ImportError: - dataclasses = None + from apache_beam.coders import proto2_coder_test_messages_pb2 as test_message logging.basicConfig( level=logging.INFO, @@ -640,38 +722,6 @@ def test_cross_process_encoding_of_special_types_is_deterministic(self): stream=sys.stderr, force=True ) - - # Define all the special types that encode_special_deterministic handles - MyNamedTuple = collections.namedtuple('A', ['x', 'y']) - MyTypedNamedTuple = NamedTuple('MyTypedNamedTuple', [('f1', int), ('f2', str)]) - - class MyEnum(enum.Enum): - E1 = 5 - E2 = enum.auto() - E3 = 'abc' - - MyIntEnum = enum.IntEnum('MyIntEnum', 'I1 I2 I3') - MyIntFlag = enum.IntFlag('MyIntFlag', 'F1 F2 F3') - MyFlag = enum.Flag('MyFlag', 'F1 F2 F3') - - if dataclasses is not None: - @dataclasses.dataclass(frozen=True) - class FrozenDataClass: - a: int - b: int - - class DefinesGetAndSetState: - def __init__(self, value): - self.value = value - - def __getstate__(self): - return self.value - - def __setstate__(self, value): - self.value = value - - def __eq__(self, other): - return type(other) is type(self) and other.value == self.value # Test cases for all special deterministic types # NOTE: When this script run in a subprocess the module is considered @@ -683,26 +733,28 @@ def __eq__(self, other): ("named_tuple_simple", MyNamedTuple(1, 2)), ("typed_named_tuple", MyTypedNamedTuple(1, 'a')), ("named_tuple_list", [MyNamedTuple(1, 2), MyTypedNamedTuple(1, 'a')]), - # ("enum_single", MyEnum.E1), - # ("enum_list", list(MyEnum)), - # ("int_enum_list", list(MyIntEnum)), - # ("int_flag_list", list(MyIntFlag)), - # ("flag_list", list(MyFlag)), + ("enum_single", MyEnum.E1), + ("enum_list", list(MyEnum)), + ("int_enum_list", list(MyIntEnum)), + ("int_flag_list", list(MyIntFlag)), + ("flag_list", list(MyFlag)), ("getstate_setstate_simple", DefinesGetAndSetState(1)), ("getstate_setstate_complex", DefinesGetAndSetState((1, 2, 3))), ("getstate_setstate_list", [DefinesGetAndSetState(1), DefinesGetAndSetState((1, 2, 3))]), ] - if dataclasses is not None: - test_cases.extend([ - ("frozen_dataclass", FrozenDataClass(1, 2)), - ("frozen_dataclass_list", [FrozenDataClass(1, 2), FrozenDataClass(3, 4)]), - ]) + + test_cases.extend([ + ("frozen_dataclass", FrozenDataClass(1, 2)), + ("frozen_dataclass_list", [FrozenDataClass(1, 2), FrozenDataClass(3, 4)]), + ]) + compat_version = {'"'+ compat_version +'"' if compat_version else None} + typecoders.registry.update_compatibility_version = compat_version coder = coders.FastPrimitivesCoder() - deterministic_coder = coders.DeterministicFastPrimitivesCoder(coder, 'step') + deterministic_coder = coder.as_deterministic_coder("step") - results = {} + results = dict() for test_name, value in test_cases: try: encoded = deterministic_coder.encode(value) @@ -730,7 +782,7 @@ def run_subprocess(): results2 = run_subprocess() coder = coders.FastPrimitivesCoder() - deterministic_coder = coders.DeterministicFastPrimitivesCoder(coder, 'step') + deterministic_coder = coder.as_deterministic_coder("step") for test_name in results1: data1 = results1[test_name] @@ -861,7 +913,7 @@ def test_map_coder(self): { i: str(i) for i in range(5000) - } + }, ] map_coder = coders.MapCoder(coders.VarIntCoder(), coders.StrUtf8Coder()) self.check_coder(map_coder, *values) diff --git a/sdks/python/apache_beam/coders/typecoders.py b/sdks/python/apache_beam/coders/typecoders.py index 19300c675596..779c65dc772c 100644 --- a/sdks/python/apache_beam/coders/typecoders.py +++ b/sdks/python/apache_beam/coders/typecoders.py @@ -84,6 +84,7 @@ def __init__(self, fallback_coder=None): self._coders: Dict[Any, Type[coders.Coder]] = {} self.custom_types: List[Any] = [] self.register_standard_coders(fallback_coder) + self.update_compatibility_version = None def register_standard_coders(self, fallback_coder): """Register coders for all basic and composite types.""" diff --git a/sdks/python/apache_beam/dataframe/io.py b/sdks/python/apache_beam/dataframe/io.py index a804e4b4f2d2..752df1e68b7c 100644 --- a/sdks/python/apache_beam/dataframe/io.py +++ b/sdks/python/apache_beam/dataframe/io.py @@ -93,6 +93,7 @@ def read_csv(path, *args, splittable=False, binary=True, **kwargs): newlines may result in partial records and data corruption.""" if 'nrows' in kwargs: raise ValueError('nrows not yet supported') + filename_column = kwargs.pop('filename_column', None) return _ReadFromPandas( pd.read_csv, path, @@ -100,7 +101,8 @@ def read_csv(path, *args, splittable=False, binary=True, **kwargs): kwargs, incremental=True, binary=binary, - splitter=_TextFileSplitter(args, kwargs) if splittable else None) + splitter=_TextFileSplitter(args, kwargs) if splittable else None, + filename_column=filename_column) def _as_pc(df, label=None): @@ -254,7 +256,8 @@ def __init__( kwargs, binary=True, incremental=False, - splitter=False): + splitter=False, + filename_column=None): if 'compression' in kwargs: raise NotImplementedError('compression') if not isinstance(path, str): @@ -266,6 +269,7 @@ def __init__( self.binary = binary self.incremental = incremental self.splitter = splitter + self.filename_column = filename_column def expand(self, root): paths_pcoll = root | beam.Create([self.path]) @@ -285,6 +289,8 @@ def expand(self, root): sample = next(stream) else: sample = self.reader(handle, *self.args, **self.kwargs) + if self.filename_column: + sample[self.filename_column] = '' matches_pcoll = paths_pcoll | fileio.MatchAll() indices_pcoll = ( @@ -309,7 +315,8 @@ def expand(self, root): self.kwargs, self.binary, self.incremental, - self.splitter), + self.splitter, + self.filename_column), path_indices=beam.pvalue.AsSingleton(indices_pcoll))) from apache_beam.dataframe import convert return convert.to_dataframe(pcoll, proxy=sample[:0]) @@ -580,7 +587,15 @@ def flush(self): class _ReadFromPandasDoFn(beam.DoFn, beam.RestrictionProvider): - def __init__(self, reader, args, kwargs, binary, incremental, splitter): + def __init__( + self, + reader, + args, + kwargs, + binary, + incremental, + splitter, + filename_column=None): # avoid pickling issues if reader.__module__.startswith('pandas.'): reader = reader.__name__ @@ -590,6 +605,7 @@ def __init__(self, reader, args, kwargs, binary, incremental, splitter): self.binary = binary self.incremental = incremental self.splitter = splitter + self.filename_column = filename_column def initial_restriction(self, readable_file): return beam.io.restriction_trackers.OffsetRange( @@ -642,6 +658,8 @@ def process( else: frames = [reader(handle, *self.args, **self.kwargs)] for df in frames: + if self.filename_column: + df[self.filename_column] = readable_file.metadata.path yield _shift_range_index(start_index, df) if not self.incremental: # Satisfy the SDF contract by claiming the whole range. @@ -769,10 +787,15 @@ def __init__( *args, include_indexes=False, objects_as_strings=True, + filename_column=None, **kwargs): + if format == 'csv': + kwargs['filename_column'] = filename_column + self._reader = globals()['read_%s' % format](*args, **kwargs) self._reader = globals()['read_%s' % format](*args, **kwargs) self._include_indexes = include_indexes self._objects_as_strings = objects_as_strings + self._filename_column = filename_column def expand(self, p): from apache_beam.dataframe import convert # avoid circular import diff --git a/sdks/python/apache_beam/examples/complete/juliaset/juliaset/juliaset_test_it.py b/sdks/python/apache_beam/examples/complete/juliaset/juliaset/juliaset_test_it.py index a2a3262a1fb6..148343ea9ae6 100644 --- a/sdks/python/apache_beam/examples/complete/juliaset/juliaset/juliaset_test_it.py +++ b/sdks/python/apache_beam/examples/complete/juliaset/juliaset/juliaset_test_it.py @@ -38,7 +38,7 @@ class JuliaSetTestIT(unittest.TestCase): GRID_SIZE = 1000 - def test_run_example_with_setup_file(self): + def test_run_example_with_requirements_file(self): pipeline = TestPipeline(is_integration_test=True) coordinate_output = FileSystems.join( pipeline.get_option('output'), @@ -47,8 +47,8 @@ def test_run_example_with_setup_file(self): extra_args = { 'coordinate_output': coordinate_output, 'grid_size': self.GRID_SIZE, - 'setup_file': os.path.normpath( - os.path.join(os.path.dirname(__file__), '..', 'setup.py')), + 'requirements_file': os.path.normpath( + os.path.join(os.path.dirname(__file__), '..', 'requirements.txt')), 'on_success_matcher': all_of(PipelineStateMatcher(PipelineState.DONE)), } args = pipeline.get_full_options_as_args(**extra_args) diff --git a/sdks/python/apache_beam/examples/complete/juliaset/juliaset_main.py b/sdks/python/apache_beam/examples/complete/juliaset/juliaset_main.py index fb64c2702fd2..589c21687dcd 100644 --- a/sdks/python/apache_beam/examples/complete/juliaset/juliaset_main.py +++ b/sdks/python/apache_beam/examples/complete/juliaset/juliaset_main.py @@ -21,17 +21,12 @@ workflow. It is organized in this way so that it can be packaged as a Python package and later installed in the VM workers executing the job. The root directory for the example contains just a "driver" script to launch the job -and the setup.py file needed to create a package. +and the requirements.txt file needed to create a package. The advantages for organizing the code is that large projects will naturally evolve beyond just one module and you will have to make sure the additional modules are present in the worker. -In Python Dataflow, using the --setup_file option when submitting a job, will -trigger creating a source distribution (as if running python setup.py sdist) and -then staging the resulting tarball in the staging area. The workers, upon -startup, will install the tarball. - Below is a complete command line for running the juliaset workflow remotely as an example: @@ -40,7 +35,7 @@ --project YOUR-PROJECT \ --region GCE-REGION \ --runner DataflowRunner \ - --setup_file ./setup.py \ + --requirements_file ./requirements.txt \ --staging_location gs://YOUR-BUCKET/juliaset/staging \ --temp_location gs://YOUR-BUCKET/juliaset/temp \ --coordinate_output gs://YOUR-BUCKET/juliaset/out \ diff --git a/sdks/python/apache_beam/examples/complete/juliaset/requirements.txt b/sdks/python/apache_beam/examples/complete/juliaset/requirements.txt new file mode 100644 index 000000000000..7d514bd30998 --- /dev/null +++ b/sdks/python/apache_beam/examples/complete/juliaset/requirements.txt @@ -0,0 +1,17 @@ +# 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. + +numpy diff --git a/sdks/python/apache_beam/examples/complete/juliaset/setup.py b/sdks/python/apache_beam/examples/complete/juliaset/setup.py deleted file mode 100644 index c3a9fe043765..000000000000 --- a/sdks/python/apache_beam/examples/complete/juliaset/setup.py +++ /dev/null @@ -1,128 +0,0 @@ -# -# 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. -# - -"""Setup.py module for the workflow's worker utilities. - -All the workflow related code is gathered in a package that will be built as a -source distribution, staged in the staging area for the workflow being run and -then installed in the workers when they start running. - -This behavior is triggered by specifying the --setup_file command line option -when running the workflow for remote execution. -""" - -# pytype: skip-file - -import subprocess - -import setuptools - -# It is recommended to import setuptools prior to importing distutils to avoid -# using legacy behavior from distutils. -# https://setuptools.readthedocs.io/en/latest/history.html#v48-0-0 -from distutils.command.build import build as _build # isort:skip - - -# This class handles the pip install mechanism. -class build(_build): # pylint: disable=invalid-name - """A build command class that will be invoked during package install. - - The package built using the current setup.py will be staged and later - installed in the worker using `pip install package'. This class will be - instantiated during install for this specific scenario and will trigger - running the custom commands specified. - """ - sub_commands = _build.sub_commands + [('CustomCommands', None)] - - -# Some custom command to run during setup. The command is not essential for this -# workflow. It is used here as an example. Each command will spawn a child -# process. Typically, these commands will include steps to install non-Python -# packages. For instance, to install a C++-based library libjpeg62 the following -# two commands will have to be added: -# -# ['apt-get', 'update'], -# ['apt-get', '--assume-yes', 'install', 'libjpeg62'], -# -# First, note that there is no need to use the sudo command because the setup -# script runs with appropriate access. -# Second, if apt-get tool is used then the first command needs to be 'apt-get -# update' so the tool refreshes itself and initializes links to download -# repositories. Without this initial step the other apt-get install commands -# will fail with package not found errors. Note also --assume-yes option which -# shortcuts the interactive confirmation. -# -# Note that in this example custom commands will run after installing required -# packages. If you have a PyPI package that depends on one of the custom -# commands, move installation of the dependent package to the list of custom -# commands, e.g.: -# -# ['pip', 'install', 'my_package'], -# -# TODO(https://github.com/apache/beam/issues/18568): Output from the custom -# commands are missing from the logs. The output of custom commands (including -# failures) will be logged in the worker-startup log. -CUSTOM_COMMANDS = [['echo', 'Custom command worked!']] - - -class CustomCommands(setuptools.Command): - """A setuptools Command class able to run arbitrary commands.""" - def initialize_options(self): - pass - - def finalize_options(self): - pass - - def RunCustomCommand(self, command_list): - print('Running command: %s' % command_list) - p = subprocess.Popen( - command_list, - stdin=subprocess.PIPE, - stdout=subprocess.PIPE, - stderr=subprocess.STDOUT) - # Can use communicate(input='y\n'.encode()) if the command run requires - # some confirmation. - stdout_data, _ = p.communicate() - print('Command output: %s' % stdout_data) - if p.returncode != 0: - raise RuntimeError( - 'Command %s failed: exit code: %s' % (command_list, p.returncode)) - - def run(self): - for command in CUSTOM_COMMANDS: - self.RunCustomCommand(command) - - -# Configure the required packages and scripts to install. -# Note that the Python Dataflow containers come with numpy already installed -# so this dependency will not trigger anything to be installed unless a version -# restriction is specified. -REQUIRED_PACKAGES = [ - 'numpy', -] - -setuptools.setup( - name='juliaset', - version='0.0.1', - description='Julia set workflow package.', - install_requires=REQUIRED_PACKAGES, - packages=setuptools.find_packages(), - cmdclass={ - # Command class instantiated and run during pip install scenarios. - 'build': build, - 'CustomCommands': CustomCommands, - }) diff --git a/sdks/python/apache_beam/examples/inference/vllm_gemma_batch.py b/sdks/python/apache_beam/examples/inference/vllm_gemma_batch.py new file mode 100644 index 000000000000..f6e33e5be786 --- /dev/null +++ b/sdks/python/apache_beam/examples/inference/vllm_gemma_batch.py @@ -0,0 +1,130 @@ +# +# 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. +# + +from __future__ import annotations + +import logging +import os +import tempfile + +import apache_beam as beam +from apache_beam.io.filesystems import FileSystems +from apache_beam.ml.inference.base import RunInference +from apache_beam.ml.inference.vllm_inference import VLLMCompletionsModelHandler +from apache_beam.ml.inference.vllm_inference import _VLLMModelServer +from apache_beam.options.pipeline_options import PipelineOptions +from apache_beam.options.pipeline_options import SetupOptions + + +class GemmaVLLMOptions(PipelineOptions): + """Custom pipeline options for the Gemma vLLM batch inference job.""" + @classmethod + def _add_argparse_args(cls, parser): + parser.add_argument( + "--input", + dest="input_file", + required=True, + help="Input file gs://path containing prompts.", + ) + parser.add_argument( + "--output_table", + required=True, + help="BigQuery table to write to in the form project:dataset.table.", + ) + parser.add_argument( + "--model_gcs_path", + required=True, + help="GCS path to the directory containing model files.", + ) + + +class FormatOutput(beam.DoFn): + def process(self, element): + prompt = element.example + comp = element.inference + + if hasattr(comp, 'choices'): + completion = comp.choices[0].text + # fallback to a single .text field + elif hasattr(comp, 'text'): + completion = comp.text + # final fallback + else: + completion = str(comp) + + yield {'prompt': prompt, 'completion': completion} + + +class GcsVLLMCompletionsModelHandler(VLLMCompletionsModelHandler): + def __init__(self, model_name, vllm_server_kwargs=None): + super().__init__(model_name, vllm_server_kwargs) + self._local_model_dir = None + + def _download_gcs_directory(self, gcs_path: str, local_path: str): + logging.info("Downloading model from %s to %s…", gcs_path, local_path) + matches = FileSystems.match([os.path.join(gcs_path, "**")])[0].metadata_list + for md in matches: + rel = os.path.relpath(md.path, gcs_path) + dst = os.path.join(local_path, rel) + os.makedirs(os.path.dirname(dst), exist_ok=True) + with FileSystems.open(md.path) as src, open(dst, "wb") as dstf: + dstf.write(src.read()) + logging.info("Download complete.") + + def load_model(self) -> _VLLMModelServer: + uri = self._model_name + if uri.startswith("gs://"): + self._local_model_dir = tempfile.mkdtemp(prefix="vllm_model_") + self._download_gcs_directory(uri, self._local_model_dir) + logging.info("Loading vLLM from local dir %s", self._local_model_dir) + return _VLLMModelServer(self._local_model_dir, self._vllm_server_kwargs) + else: + logging.info("Loading vLLM from HF hub: %s", uri) + return super().load_model() + + +def run(argv=None, save_main_session=True, test_pipeline=None): + # Build pipeline options + opts = PipelineOptions(argv) + + gem = opts.view_as(GemmaVLLMOptions) + opts.view_as(SetupOptions).save_main_session = save_main_session + + logging.info("Pipeline starting with model path: %s", gem.model_gcs_path) + handler = GcsVLLMCompletionsModelHandler( + model_name=gem.model_gcs_path, + vllm_server_kwargs={"served-model-name": gem.model_gcs_path}) + + with (test_pipeline or beam.Pipeline(options=opts)) as p: + _ = ( + p + | "Read" >> beam.io.ReadFromText(gem.input_file) + | "InferBatch" >> RunInference(handler, inference_batch_size=32) + | "FormatForBQ" >> beam.ParDo(FormatOutput()) + | "WriteToBQ" >> beam.io.WriteToBigQuery( + gem.output_table, + schema="prompt:STRING,completion:STRING", + write_disposition=beam.io.BigQueryDisposition.WRITE_APPEND, + create_disposition=beam.io.BigQueryDisposition.CREATE_IF_NEEDED, + method=beam.io.WriteToBigQuery.Method.FILE_LOADS, + )) + return p.result + + +if __name__ == "__main__": + logging.getLogger().setLevel(logging.INFO) + run() diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py index acee633b6f67..d71faa6d8477 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment.py @@ -116,3 +116,214 @@ def enrichment_with_vertex_ai_legacy(): | "Enrich W/ Vertex AI" >> Enrichment(vertex_ai_handler) | "Print" >> beam.Map(print)) # [END enrichment_with_vertex_ai_legacy] + + +def enrichment_with_google_cloudsql_pg(): + # [START enrichment_with_google_cloudsql_pg] + import apache_beam as beam + from apache_beam.transforms.enrichment import Enrichment + from apache_beam.transforms.enrichment_handlers.cloudsql import ( + CloudSQLEnrichmentHandler, + DatabaseTypeAdapter, + TableFieldsQueryConfig, + CloudSQLConnectionConfig) + import os + + database_adapter = DatabaseTypeAdapter.POSTGRESQL + database_uri = os.environ.get("GOOGLE_CLOUD_SQL_DB_URI") + database_user = os.environ.get("GOOGLE_CLOUD_SQL_DB_USER") + database_password = os.environ.get("GOOGLE_CLOUD_SQL_DB_PASSWORD") + database_id = os.environ.get("GOOGLE_CLOUD_SQL_DB_ID") + table_id = os.environ.get("GOOGLE_CLOUD_SQL_DB_TABLE_ID") + where_clause_template = "product_id = :pid" + where_clause_fields = ["product_id"] + + data = [ + beam.Row(product_id=1, name='A'), + beam.Row(product_id=2, name='B'), + beam.Row(product_id=3, name='C'), + ] + + connection_config = CloudSQLConnectionConfig( + db_adapter=database_adapter, + instance_connection_uri=database_uri, + user=database_user, + password=database_password, + db_id=database_id) + + query_config = TableFieldsQueryConfig( + table_id=table_id, + where_clause_template=where_clause_template, + where_clause_fields=where_clause_fields) + + cloudsql_handler = CloudSQLEnrichmentHandler( + connection_config=connection_config, + table_id=table_id, + query_config=query_config) + with beam.Pipeline() as p: + _ = ( + p + | "Create" >> beam.Create(data) + | + "Enrich W/ Google CloudSQL PostgreSQL" >> Enrichment(cloudsql_handler) + | "Print" >> beam.Map(print)) + # [END enrichment_with_google_cloudsql_pg] + + +def enrichment_with_external_pg(): + # [START enrichment_with_external_pg] + import apache_beam as beam + from apache_beam.transforms.enrichment import Enrichment + from apache_beam.transforms.enrichment_handlers.cloudsql import ( + CloudSQLEnrichmentHandler, + DatabaseTypeAdapter, + TableFieldsQueryConfig, + ExternalSQLDBConnectionConfig) + import os + + database_adapter = DatabaseTypeAdapter.POSTGRESQL + database_host = os.environ.get("EXTERNAL_SQL_DB_HOST") + database_port = int(os.environ.get("EXTERNAL_SQL_DB_PORT")) + database_user = os.environ.get("EXTERNAL_SQL_DB_USER") + database_password = os.environ.get("EXTERNAL_SQL_DB_PASSWORD") + database_id = os.environ.get("EXTERNAL_SQL_DB_ID") + table_id = os.environ.get("EXTERNAL_SQL_DB_TABLE_ID") + where_clause_template = "product_id = :pid" + where_clause_fields = ["product_id"] + + data = [ + beam.Row(product_id=1, name='A'), + beam.Row(product_id=2, name='B'), + beam.Row(product_id=3, name='C'), + ] + + connection_config = ExternalSQLDBConnectionConfig( + db_adapter=database_adapter, + host=database_host, + port=database_port, + user=database_user, + password=database_password, + db_id=database_id) + + query_config = TableFieldsQueryConfig( + table_id=table_id, + where_clause_template=where_clause_template, + where_clause_fields=where_clause_fields) + + cloudsql_handler = CloudSQLEnrichmentHandler( + connection_config=connection_config, + table_id=table_id, + query_config=query_config) + with beam.Pipeline() as p: + _ = ( + p + | "Create" >> beam.Create(data) + | "Enrich W/ Unmanaged PostgreSQL" >> Enrichment(cloudsql_handler) + | "Print" >> beam.Map(print)) + # [END enrichment_with_external_pg] + + +def enrichment_with_external_mysql(): + # [START enrichment_with_external_mysql] + import apache_beam as beam + from apache_beam.transforms.enrichment import Enrichment + from apache_beam.transforms.enrichment_handlers.cloudsql import ( + CloudSQLEnrichmentHandler, + DatabaseTypeAdapter, + TableFieldsQueryConfig, + ExternalSQLDBConnectionConfig) + import os + + database_adapter = DatabaseTypeAdapter.MYSQL + database_host = os.environ.get("EXTERNAL_SQL_DB_HOST") + database_port = int(os.environ.get("EXTERNAL_SQL_DB_PORT")) + database_user = os.environ.get("EXTERNAL_SQL_DB_USER") + database_password = os.environ.get("EXTERNAL_SQL_DB_PASSWORD") + database_id = os.environ.get("EXTERNAL_SQL_DB_ID") + table_id = os.environ.get("EXTERNAL_SQL_DB_TABLE_ID") + where_clause_template = "product_id = :pid" + where_clause_fields = ["product_id"] + + data = [ + beam.Row(product_id=1, name='A'), + beam.Row(product_id=2, name='B'), + beam.Row(product_id=3, name='C'), + ] + + connection_config = ExternalSQLDBConnectionConfig( + db_adapter=database_adapter, + host=database_host, + port=database_port, + user=database_user, + password=database_password, + db_id=database_id) + + query_config = TableFieldsQueryConfig( + table_id=table_id, + where_clause_template=where_clause_template, + where_clause_fields=where_clause_fields) + + cloudsql_handler = CloudSQLEnrichmentHandler( + connection_config=connection_config, + table_id=table_id, + query_config=query_config) + with beam.Pipeline() as p: + _ = ( + p + | "Create" >> beam.Create(data) + | "Enrich W/ Unmanaged MySQL" >> Enrichment(cloudsql_handler) + | "Print" >> beam.Map(print)) + # [END enrichment_with_external_mysql] + + +def enrichment_with_external_sqlserver(): + # [START enrichment_with_external_sqlserver] + import apache_beam as beam + from apache_beam.transforms.enrichment import Enrichment + from apache_beam.transforms.enrichment_handlers.cloudsql import ( + CloudSQLEnrichmentHandler, + DatabaseTypeAdapter, + TableFieldsQueryConfig, + ExternalSQLDBConnectionConfig) + import os + + database_adapter = DatabaseTypeAdapter.SQLSERVER + database_host = os.environ.get("EXTERNAL_SQL_DB_HOST") + database_port = int(os.environ.get("EXTERNAL_SQL_DB_PORT")) + database_user = os.environ.get("EXTERNAL_SQL_DB_USER") + database_password = os.environ.get("EXTERNAL_SQL_DB_PASSWORD") + database_id = os.environ.get("EXTERNAL_SQL_DB_ID") + table_id = os.environ.get("EXTERNAL_SQL_DB_TABLE_ID") + where_clause_template = "product_id = :pid" + where_clause_fields = ["product_id"] + + data = [ + beam.Row(product_id=1, name='A'), + beam.Row(product_id=2, name='B'), + beam.Row(product_id=3, name='C'), + ] + + connection_config = ExternalSQLDBConnectionConfig( + db_adapter=database_adapter, + host=database_host, + port=database_port, + user=database_user, + password=database_password, + db_id=database_id) + + query_config = TableFieldsQueryConfig( + table_id=table_id, + where_clause_template=where_clause_template, + where_clause_fields=where_clause_fields) + + cloudsql_handler = CloudSQLEnrichmentHandler( + connection_config=connection_config, + table_id=table_id, + query_config=query_config) + with beam.Pipeline() as p: + _ = ( + p + | "Create" >> beam.Create(data) + | "Enrich W/ Unmanaged SQL Server" >> Enrichment(cloudsql_handler) + | "Print" >> beam.Map(print)) + # [END enrichment_with_external_sqlserver] diff --git a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py index afa2bca7ec68..5a64d2667f2a 100644 --- a/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py +++ b/sdks/python/apache_beam/examples/snippets/transforms/elementwise/enrichment_test.py @@ -18,19 +18,42 @@ # pytype: skip-file # pylint: disable=line-too-long +import os import unittest +import uuid +from collections.abc import Callable +from contextlib import contextmanager +from dataclasses import dataclass from io import StringIO +from typing import Optional import mock +import pytest +from sqlalchemy.engine import Connection as DBAPIConnection # pylint: disable=unused-import try: - from apache_beam.examples.snippets.transforms.elementwise.enrichment import enrichment_with_bigtable, \ - enrichment_with_vertex_ai_legacy - from apache_beam.examples.snippets.transforms.elementwise.enrichment import enrichment_with_vertex_ai + from sqlalchemy import ( + Column, Integer, VARCHAR, Engine, MetaData, create_engine) + from apache_beam.examples.snippets.transforms.elementwise.enrichment import ( + enrichment_with_bigtable, enrichment_with_vertex_ai_legacy) + from apache_beam.examples.snippets.transforms.elementwise.enrichment import ( + enrichment_with_vertex_ai, + enrichment_with_google_cloudsql_pg, + enrichment_with_external_pg, + enrichment_with_external_mysql, + enrichment_with_external_sqlserver) + from apache_beam.transforms.enrichment_handlers.cloudsql import ( + DatabaseTypeAdapter) + from apache_beam.transforms.enrichment_handlers.cloudsql_it_test import ( + SQLEnrichmentTestHelper, + SQLDBContainerInfo, + ConnectionConfig, + CloudSQLConnectionConfig, + ExternalSQLDBConnectionConfig) from apache_beam.io.requestresponse import RequestResponseIO -except ImportError: - raise unittest.SkipTest('RequestResponseIO dependencies are not installed') +except ImportError as e: + raise unittest.SkipTest(f'RequestResponseIO dependencies not installed: {e}') def validate_enrichment_with_bigtable(): @@ -60,52 +83,232 @@ def validate_enrichment_with_vertex_ai_legacy(): return expected -def std_out_to_dict(stdout_lines, row_key): - output_dict = {} - for stdout_line in stdout_lines: - # parse the stdout in a dictionary format so that it can be - # evaluated/compared as one. This allows us to compare without - # considering the order of the stdout or the order that the fields of the - # row are arranged in. - fmtd = '{\"' + stdout_line[4:-1].replace('=', '\": ').replace( - ', ', ', \"').replace('\"\'', '\'') + "}" - stdout_dict = eval(fmtd) # pylint: disable=eval-used - output_dict[stdout_dict[row_key]] = stdout_dict - return output_dict +def validate_enrichment_with_google_cloudsql_pg(): + expected = '''[START enrichment_with_google_cloudsql_pg] +Row(product_id=1, name='A', quantity=2, region_id=3) +Row(product_id=2, name='B', quantity=3, region_id=1) +Row(product_id=3, name='C', quantity=10, region_id=4) + [END enrichment_with_google_cloudsql_pg]'''.splitlines()[1:-1] + return expected + + +def validate_enrichment_with_external_pg(): + expected = '''[START enrichment_with_external_pg] +Row(product_id=1, name='A', quantity=2, region_id=3) +Row(product_id=2, name='B', quantity=3, region_id=1) +Row(product_id=3, name='C', quantity=10, region_id=4) + [END enrichment_with_external_pg]'''.splitlines()[1:-1] + return expected + + +def validate_enrichment_with_external_mysql(): + expected = '''[START enrichment_with_external_mysql] +Row(product_id=1, name='A', quantity=2, region_id=3) +Row(product_id=2, name='B', quantity=3, region_id=1) +Row(product_id=3, name='C', quantity=10, region_id=4) + [END enrichment_with_external_mysql]'''.splitlines()[1:-1] + return expected + + +def validate_enrichment_with_external_sqlserver(): + expected = '''[START enrichment_with_external_sqlserver] +Row(product_id=1, name='A', quantity=2, region_id=3) +Row(product_id=2, name='B', quantity=3, region_id=1) +Row(product_id=3, name='C', quantity=10, region_id=4) + [END enrichment_with_external_sqlserver]'''.splitlines()[1:-1] + return expected @mock.patch('sys.stdout', new_callable=StringIO) +@pytest.mark.uses_testcontainer class EnrichmentTest(unittest.TestCase): def test_enrichment_with_bigtable(self, mock_stdout): enrichment_with_bigtable() output = mock_stdout.getvalue().splitlines() expected = validate_enrichment_with_bigtable() - - self.assertEqual(len(output), len(expected)) - self.assertEqual( - std_out_to_dict(output, 'sale_id'), - std_out_to_dict(expected, 'sale_id')) + self.assertEqual(output, expected) def test_enrichment_with_vertex_ai(self, mock_stdout): enrichment_with_vertex_ai() output = mock_stdout.getvalue().splitlines() expected = validate_enrichment_with_vertex_ai() - self.assertEqual(len(output), len(expected)) - self.assertEqual( - std_out_to_dict(output, 'user_id'), - std_out_to_dict(expected, 'user_id')) + for i in range(len(expected)): + self.assertEqual(set(output[i].split(',')), set(expected[i].split(','))) def test_enrichment_with_vertex_ai_legacy(self, mock_stdout): enrichment_with_vertex_ai_legacy() output = mock_stdout.getvalue().splitlines() expected = validate_enrichment_with_vertex_ai_legacy() self.maxDiff = None + self.assertEqual(sorted(output), sorted(expected)) + + @unittest.skipUnless( + os.environ.get('ALLOYDB_PASSWORD'), + "ALLOYDB_PASSWORD environment var is not provided") + def test_enrichment_with_google_cloudsql_pg(self, mock_stdout): + db_adapter = DatabaseTypeAdapter.POSTGRESQL + with EnrichmentTestHelpers.sql_test_context(True, db_adapter): + try: + enrichment_with_google_cloudsql_pg() + output = mock_stdout.getvalue().splitlines() + expected = validate_enrichment_with_google_cloudsql_pg() + self.assertEqual(output, expected) + except Exception as e: + self.fail(f"Test failed with unexpected error: {e}") + + def test_enrichment_with_external_pg(self, mock_stdout): + db_adapter = DatabaseTypeAdapter.POSTGRESQL + with EnrichmentTestHelpers.sql_test_context(False, db_adapter): + try: + enrichment_with_external_pg() + output = mock_stdout.getvalue().splitlines() + expected = validate_enrichment_with_external_pg() + self.assertEqual(output, expected) + except Exception as e: + self.fail(f"Test failed with unexpected error: {e}") + + def test_enrichment_with_external_mysql(self, mock_stdout): + db_adapter = DatabaseTypeAdapter.MYSQL + with EnrichmentTestHelpers.sql_test_context(False, db_adapter): + try: + enrichment_with_external_mysql() + output = mock_stdout.getvalue().splitlines() + expected = validate_enrichment_with_external_mysql() + self.assertEqual(output, expected) + except Exception as e: + self.fail(f"Test failed with unexpected error: {e}") + + def test_enrichment_with_external_sqlserver(self, mock_stdout): + db_adapter = DatabaseTypeAdapter.SQLSERVER + with EnrichmentTestHelpers.sql_test_context(False, db_adapter): + try: + enrichment_with_external_sqlserver() + output = mock_stdout.getvalue().splitlines() + expected = validate_enrichment_with_external_sqlserver() + self.assertEqual(output, expected) + except Exception as e: + self.fail(f"Test failed with unexpected error: {e}") + + +@dataclass +class CloudSQLEnrichmentTestDataConstruct: + client_handler: Callable[[], DBAPIConnection] + engine: Engine + metadata: MetaData + db: SQLDBContainerInfo = None + + +class EnrichmentTestHelpers: + @contextmanager + def sql_test_context(is_cloudsql: bool, db_adapter: DatabaseTypeAdapter): + result: Optional[CloudSQLEnrichmentTestDataConstruct] = None + try: + result = EnrichmentTestHelpers.pre_sql_enrichment_test( + is_cloudsql, db_adapter) + yield + finally: + if result: + EnrichmentTestHelpers.post_sql_enrichment_test(result) + + @staticmethod + def pre_sql_enrichment_test( + is_cloudsql: bool, + db_adapter: DatabaseTypeAdapter) -> CloudSQLEnrichmentTestDataConstruct: + unique_suffix = str(uuid.uuid4())[:8] + table_id = f"products_{unique_suffix}" + columns = [ + Column("product_id", Integer, primary_key=True), + Column("name", VARCHAR(255), nullable=False), + Column("quantity", Integer, nullable=False), + Column("region_id", Integer, nullable=False), + ] + table_data = [ + { + "product_id": 1, "name": "A", 'quantity': 2, 'region_id': 3 + }, + { + "product_id": 2, "name": "B", 'quantity': 3, 'region_id': 1 + }, + { + "product_id": 3, "name": "C", 'quantity': 10, 'region_id': 4 + }, + ] + metadata = MetaData() + + connection_config: ConnectionConfig + db = None + if is_cloudsql: + gcp_project_id = "apache-beam-testing" + region = "us-central1" + instance_name = "beam-integration-tests" + instance_connection_uri = f"{gcp_project_id}:{region}:{instance_name}" + db_id = "postgres" + user = "postgres" + password = os.getenv("ALLOYDB_PASSWORD") + os.environ['GOOGLE_CLOUD_SQL_DB_URI'] = instance_connection_uri + os.environ['GOOGLE_CLOUD_SQL_DB_ID'] = db_id + os.environ['GOOGLE_CLOUD_SQL_DB_USER'] = user + os.environ['GOOGLE_CLOUD_SQL_DB_PASSWORD'] = password + os.environ['GOOGLE_CLOUD_SQL_DB_TABLE_ID'] = table_id + connection_config = CloudSQLConnectionConfig( + db_adapter=db_adapter, + instance_connection_uri=instance_connection_uri, + user=user, + password=password, + db_id=db_id) + else: + db = SQLEnrichmentTestHelper.start_sql_db_container(db_adapter) + os.environ['EXTERNAL_SQL_DB_HOST'] = db.host + os.environ['EXTERNAL_SQL_DB_PORT'] = str(db.port) + os.environ['EXTERNAL_SQL_DB_ID'] = db.id + os.environ['EXTERNAL_SQL_DB_USER'] = db.user + os.environ['EXTERNAL_SQL_DB_PASSWORD'] = db.password + os.environ['EXTERNAL_SQL_DB_TABLE_ID'] = table_id + connection_config = ExternalSQLDBConnectionConfig( + db_adapter=db_adapter, + host=db.host, + port=db.port, + user=db.user, + password=db.password, + db_id=db.id) + + conenctor = connection_config.get_connector_handler() + engine = create_engine( + url=connection_config.get_db_url(), creator=conenctor) + + SQLEnrichmentTestHelper.create_table( + table_id=table_id, + engine=engine, + columns=columns, + table_data=table_data, + metadata=metadata) + + result = CloudSQLEnrichmentTestDataConstruct( + db=db, client_handler=conenctor, engine=engine, metadata=metadata) + return result + + @staticmethod + def post_sql_enrichment_test(res: CloudSQLEnrichmentTestDataConstruct): + # Clean up the data inserted previously. + res.metadata.drop_all(res.engine) + res.engine.dispose(close=True) - self.assertEqual(len(output), len(expected)) - self.assertEqual( - std_out_to_dict(output, 'entity_id'), - std_out_to_dict(expected, 'entity_id')) + # Check if the test used a container-based external SQL database. + if res.db: + SQLEnrichmentTestHelper.stop_sql_db_container(res.db) + os.environ.pop('EXTERNAL_SQL_DB_HOST', None) + os.environ.pop('EXTERNAL_SQL_DB_PORT', None) + os.environ.pop('EXTERNAL_SQL_DB_ID', None) + os.environ.pop('EXTERNAL_SQL_DB_USER', None) + os.environ.pop('EXTERNAL_SQL_DB_PASSWORD', None) + os.environ.pop('EXTERNAL_SQL_DB_TABLE_ID', None) + else: + os.environ.pop('GOOGLE_CLOUD_SQL_DB_URI', None) + os.environ.pop('GOOGLE_CLOUD_SQL_DB_ID', None) + os.environ.pop('GOOGLE_CLOUD_SQL_DB_USER', None) + os.environ.pop('GOOGLE_CLOUD_SQL_DB_PASSWORD', None) + os.environ.pop('GOOGLE_CLOUD_SQL_DB_TABLE_ID', None) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/internal/cloudpickle_pickler.py b/sdks/python/apache_beam/internal/cloudpickle_pickler.py index 63038e770f27..e55818bfb226 100644 --- a/sdks/python/apache_beam/internal/cloudpickle_pickler.py +++ b/sdks/python/apache_beam/internal/cloudpickle_pickler.py @@ -39,6 +39,8 @@ DEFAULT_CONFIG = cloudpickle.CloudPickleConfig( skip_reset_dynamic_type_state=True) +NO_DYNAMIC_CLASS_TRACKING_CONFIG = cloudpickle.CloudPickleConfig( + id_generator=None, skip_reset_dynamic_type_state=True) try: from absl import flags diff --git a/sdks/python/apache_beam/internal/code_object_pickler.py b/sdks/python/apache_beam/internal/code_object_pickler.py index c3658120b4ef..b6ea015cc06f 100644 --- a/sdks/python/apache_beam/internal/code_object_pickler.py +++ b/sdks/python/apache_beam/internal/code_object_pickler.py @@ -15,7 +15,468 @@ # limitations under the License. # +"""Customizations to how Python code objects are pickled. + +This module provides helper functions to improve pickling code objects, +especially lambdas, in a consistent way by using code object identifiers. These +helper functions will be used to patch pickler implementations used by Beam +(e.g. Cloudpickle). + +A code object identifier is a unique identifier for a code object that provides +a unique reference to the code object in the context where the code is defined +and is invariant to small changes in the surrounding code. + +The code object identifiers consists of a sequence of the following parts +separated by periods: +- Module names - The name of the module the code object is in +- Class names - The name of a class containing the code object. There can be + multiple of these in the same identifier in the case of nested + classes. +- Function names - The name of the function containing the code object. + There can be multiple of these in the case of nested functions. +- __code__ - Attribute indicating that we are entering the code object of a + function/method. +- __co_consts__[] - The name of the local variable containing the + code object. In the case of lambdas, the name is created by using the + signature of the lambda and hashing the bytecode, as shown below. + +Examples: +- __main__.top_level_function.__code__ +- __main__.ClassWithNestedFunction.process.__code__.co_consts[nested_function] +- __main__.ClassWithNestedLambda.process.__code__.co_consts[ + get_lambda_from_dictionary].co_consts[, ('x',)] +- __main__.ClassWithNestedLambda.process.__code__.co_consts[ + , ('x',), 1234567890] +""" + +import collections +import hashlib +import inspect +import re +import sys +import types +from typing import Optional +from typing import Union + def get_normalized_path(path): """Returns a normalized path. This function is intended to be overridden.""" return path + + +def get_code_object_identifier(callable: types.FunctionType): + """Returns the code object identifier for a given callable. + + Args: + callable: The callable object to search for. + + Returns: + The code object identifier. + Examples: + - __main__.top_level_function.__code__ + - __main__.ClassWithNestedFunction.process.__code__.co_consts[ + nested_function] + - __main__.ClassWithNestedLambda.process.__code__.co_consts[ + get_lambda_from_dictionary].co_consts[, ('x',)] + - __main__.ClassWithNestedLambda.process.__code__.co_consts[ + , ('x',), 1234567890] + """ + if not hasattr(callable, '__module__') or not hasattr(callable, + '__qualname__'): + return None + code_path: str = _extend_path( + callable.__module__, + _search( + callable, + sys.modules[callable.__module__], + callable.__qualname__.split('.'), + ), + ) + return code_path + + +def _extend_path(prefix: str, current_path: Optional[str]): + """Extends the path to the code object. + + Args: + prefix: The prefix of the path. + suffix: The rest of the path. + + Returns: + The extended path. + """ + if current_path is None: + return None + if not current_path: + return prefix + return prefix + '.' + current_path + + +def _search( + callable: types.FunctionType, + node: Union[types.ModuleType, types.FunctionType, types.CodeType], + qual_name_parts: list[str]): + """Searches an object to create a code object identifier. + + Recursively searches the tree of objects starting from node to find the + callable's code object. It navigates through the attributes by using + the first element of qual_name_parts to indicate what object it is + currently at, then recursively passes through the rest of the list until + the callable is found. Special components like '' and '' + direct the search within nested code objects. + + + Example of qual_name_parts: ['MyClass', 'process', '', ''] + + Args: + callable: The callable object to search for. + node: The object to search within. + qual_name_parts: A list of strings representing the qualified name of the + callable object. + + Returns: + The code object identifier, or None if not found. + """ + if node is None: + return None + if not qual_name_parts: + if (hasattr(node, '__code__') and hasattr(callable, '__code__') and + node.__code__ == callable.__code__): + return '__code__' + else: + return None + if inspect.ismodule(node) or inspect.isclass(node): + return _search_module_or_class(callable, node, qual_name_parts) + elif inspect.isfunction(node): + return _search_function(callable, node, qual_name_parts) + elif inspect.iscode(node): + return _search_code(callable, node, qual_name_parts) + + +def _search_module_or_class( + callable: types.FunctionType, + node: types.ModuleType, + qual_name_parts: list[str]): + """Searches a module or class to create a code object identifier. + + Args: + callable: The callable object to search for. + node: The module or class to search within. + qual_name_parts: The list of qual name parts. + + Returns: + The code object identifier, or None if not found. + """ + # Functions/methods have a name that is unique within a given module or class + # so the traversal can directly lookup function object identified by the name. + # Lambdas don't have a name so we need to search all the attributes of the + # node. + first_part = qual_name_parts[0] + rest = qual_name_parts[1:] + if first_part == '': + for name in dir(node): + value = getattr(node, name) + if (hasattr(callable, '__code__') and + isinstance(value, type(callable)) and + value.__code__ == callable.__code__): + return name + '.__code__' + elif (isinstance(value, types.FunctionType) and + value.__defaults__ is not None): + # Python functions can have other functions as default parameters which + # might contain the code object so we have to search them. + for i, default_param_value in enumerate(value.__defaults__): + path = _search(callable, default_param_value, rest) + if path is not None: + return _extend_path(name, _extend_path(f'__defaults__[{i}]', path)) + else: + return _extend_path( + first_part, _search(callable, getattr(node, first_part), rest)) + + +def _search_function( + callable: types.FunctionType, + node: types.FunctionType, + qual_name_parts: list[str]): + """Searches a function to create a code object identifier. + + Args: + callable: The callable object to search for. + node: The function to search within. + qual_name_parts: The list of qual name parts. + + Returns: + The code object identifier, or None if not found. + """ + first_part = qual_name_parts[0] + if (node.__code__ == callable.__code__): + if len(qual_name_parts) > 1: + raise ValueError('Qual name parts too long') + return '__code__' + # If first part is '' then the code object is in a local variable + # so we should add __code__ to the path to indicate that we are entering + # the code object of the function. + if first_part == '': + return _extend_path( + '__code__', _search(callable, node.__code__, qual_name_parts)) + + +def _search_code( + callable: types.FunctionType, + node: types.CodeType, + qual_name_parts: list[str]): + """Searches a code object to create a code object identifier. + + Args: + callable: The callable to search for. + node: The code object to search within. + qual_name_parts: The list of qual name parts. + + Returns: + The code object identifier, or None if not found. + + Raises: + ValueError: If the qual name parts are too long. + """ + first_part = qual_name_parts[0] + rest = qual_name_parts[1:] + if hasattr(callable, '__code__') and node == callable.__code__: + if len(qual_name_parts) > 1: + raise ValueError('Qual name parts too long') + return '' + elif first_part == '': + code_objects_by_name = collections.defaultdict(list) + for co_const in node.co_consts: + if inspect.iscode(co_const): + code_objects_by_name[co_const.co_name].append(co_const) + num_lambdas = len(code_objects_by_name.get('', [])) + # If there is only one lambda, we can use the default path + # 'co_consts[]'. This is the most common case and it is + # faster than calculating the signature and the hash. + if num_lambdas == 1: + path = _search(callable, code_objects_by_name[''][0], rest) + if path is not None: + return _extend_path('co_consts[]', path) + else: + return _search_lambda(callable, code_objects_by_name, rest) + elif node.co_name == first_part: + return _search(callable, node, rest) + + +def _search_lambda( + callable: types.FunctionType, + code_objects_by_name: dict[str, list[types.CodeType]], + qual_name_parts: list[str]): + """Searches a lambda to create a code object identifier. + + Args: + callable: The callable to search for. + code_objects_by_name: The code objects to search within, keyed by name. + qual_name_parts: The rest of the qual_name_parts. + + Returns: + The code object identifier, or None if not found. + """ + # There are multiple lambdas in the code object, so we need to calculate + # the signature and the hash to identify the correct lambda. + lambda_code_objects_by_name = collections.defaultdict(list) + name = qual_name_parts[0] + code_objects = code_objects_by_name[name] + if name == '': + for code_object in code_objects: + lambda_name = f', {_signature(code_object)}' + lambda_code_objects_by_name[lambda_name].append(code_object) + # Check if there are any lambdas with the same signature. + # If there are, we need to calculate the hash to identify the correct + # lambda. + for lambda_name, lambda_objects in lambda_code_objects_by_name.items(): + if len(lambda_objects) > 1: + for lambda_object in lambda_objects: + path = _search(callable, lambda_object, qual_name_parts) + if path is not None: + return _extend_path( + f'co_consts[{lambda_name},' + f' {_create_bytecode_hash(lambda_object)}]', + path, + ) + else: + # If there is only one lambda with this signature, we can + # use the signature to identify the correct lambda. + path = _search(callable, code_objects[0], qual_name_parts) + if path is not None: + return _extend_path(f'co_consts[{lambda_name}]', path) + else: + # For non lambda objects, we can use the name to identify the object. + path = _search(callable, code_objects[0], qual_name_parts) + if path is not None: + return _extend_path(f'co_consts[{name}]', path) + + +# Matches a path like: co_consts[my_function] +_SINGLE_NAME_PATTERN = re.compile(r'co_consts\[([a-zA-Z0-9\<\>_-]+)]') +# Matches a path like: co_consts[, ('x',)] +_LAMBDA_WITH_ARGS_PATTERN = re.compile( + r"co_consts\[(<[^>]+>),\s*(\('[^']*'\s*,\s*\))\]") +# Matches a path like: co_consts[, ('x',), 1234567890] +_LAMBDA_WITH_HASH_PATTERN = re.compile( + r"co_consts\[(<[^>]+>),\s*(\('[^']*'\s*,\s*\)),\s*(.+)\]") +# Matches a path like: __defaults__[0] +_DEFAULT_PATTERN = re.compile(r'(__defaults__)\[(\d+)\]') +# Matches an argument like: 'x' +_ARGUMENT_PATTERN = re.compile(r"'([^']*)'") + + +def _get_code_object_from_single_name_pattern( + obj: types.ModuleType, name_result: re.Match[str], path: str): + """Returns the code object from a name pattern. + + Args: + obj: The object to search within. + name_result: The result of the name pattern search. + path: The path to the code object. + + Returns: + The code object. + + Raises: + ValueError: If the pattern is invalid. + AttributeError: If the code object is not found. + """ + if len(name_result.groups()) > 1: + raise ValueError(f'Invalid pattern for single name: {name_result.group(0)}') + # Groups are indexed starting at 1, group(0) is the entire match. + name = name_result.group(1) + for co_const in obj.co_consts: + if inspect.iscode(co_const) and co_const.co_name == name: + return co_const + raise AttributeError(f'Could not find code object with path: {path}') + + +def _get_code_object_from_lambda_with_args_pattern( + obj: types.ModuleType, lambda_with_args_result: re.Match[str], path: str): + """Returns the code object from a lambda with args pattern. + + Args: + obj: The object to search within. + lambda_with_args_result: The result of the lambda with args pattern search. + path: The path to the code object. + + Returns: + The code object. + + Raises: + AttributeError: If the code object is not found. + """ + name = lambda_with_args_result.group(1) + code_objects = collections.defaultdict(list) + for co_const in obj.co_consts: + if inspect.iscode(co_const) and co_const.co_name == name: + code_objects[co_const.co_name].append(co_const) + for name, objects in code_objects.items(): + for obj_ in objects: + args = tuple( + re.findall(_ARGUMENT_PATTERN, lambda_with_args_result.group(2))) + if obj_.co_varnames == args: + return obj_ + raise AttributeError(f'Could not find code object with path: {path}') + + +def _get_code_object_from_lambda_with_hash_pattern( + obj: types.ModuleType, lambda_with_hash_result: re.Match[str], path: str): + """Returns the code object from a lambda with hash pattern. + + Args: + obj: The object to search within. + lambda_with_hash_result: The result of the lambda with hash pattern search. + path: The path to the code object. + + Returns: + The code object. + + Raises: + AttributeError: If the code object is not found. + """ + name = lambda_with_hash_result.group(1) + code_objects = collections.defaultdict(list) + for co_const in obj.co_consts: + if inspect.iscode(co_const) and co_const.co_name == name: + code_objects[co_const.co_name].append(co_const) + for name, objects in code_objects.items(): + for obj_ in objects: + args = tuple( + re.findall(_ARGUMENT_PATTERN, lambda_with_hash_result.group(2))) + if obj_.co_varnames == args: + hash_value = lambda_with_hash_result.group(3) + if hash_value == str(_create_bytecode_hash(obj_)): + return obj_ + raise AttributeError(f'Could not find code object with path: {path}') + + +def get_code_from_identifier(code_object_identifier: str): + """Returns the code object corresponding to the code object identifier. + + Args: + code_object_identifier: A string representing the code object identifier. + + Returns: + The code object. + + Raises: + ValueError: If the path is empty or invalid. + AttributeError: If the attribute is not found. + """ + if not code_object_identifier: + raise ValueError('Path must not be empty.') + parts = code_object_identifier.split('.') + obj = sys.modules[parts[0]] + for part in parts[1:]: + if name_result := _SINGLE_NAME_PATTERN.fullmatch(part): + obj = _get_code_object_from_single_name_pattern( + obj, name_result, code_object_identifier) + elif lambda_with_args_result := _LAMBDA_WITH_ARGS_PATTERN.fullmatch(part): + obj = _get_code_object_from_lambda_with_args_pattern( + obj, lambda_with_args_result, code_object_identifier) + elif lambda_with_hash_result := _LAMBDA_WITH_HASH_PATTERN.fullmatch(part): + obj = _get_code_object_from_lambda_with_hash_pattern( + obj, lambda_with_hash_result, code_object_identifier) + elif default_result := _DEFAULT_PATTERN.fullmatch(part): + index = int(default_result.group(2)) + if index >= len(obj.__defaults__): + raise ValueError( + f'Index {index} is out of bounds for obj.__defaults__' + f' {len(obj.__defaults__)} in path {code_object_identifier}') + obj = getattr(obj, '__defaults__')[index] + else: + obj = getattr(obj, part) + return obj + + +def _signature(obj: types.CodeType): + """Returns the signature of a code object. + + The signature is the names of the arguments of the code object. This is used + to unique identify lambdas. + + Args: + obj: A code object, function, method, or cell. + + Returns: + A tuple of the names of the arguments of the code object. + """ + arg_count = ( + obj.co_argcount + obj.co_kwonlyargcount + + (obj.co_flags & 4 == 4) # PyCF_VARARGS + + (obj.co_flags & 8 == 8) # PyCF_VARKEYWORDS + ) + return obj.co_varnames[:arg_count] + + +def _create_bytecode_hash(code_object: types.CodeType): + """Returns the hash of a code object. + + Args: + code_object: A code object. + + Returns: + The hash of the code object. + """ + return hashlib.md5(code_object.co_code).hexdigest() diff --git a/sdks/python/apache_beam/internal/code_object_pickler_test.py b/sdks/python/apache_beam/internal/code_object_pickler_test.py new file mode 100644 index 000000000000..de01f16fd0a7 --- /dev/null +++ b/sdks/python/apache_beam/internal/code_object_pickler_test.py @@ -0,0 +1,565 @@ +# +# 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. +# + +"""Tests for generating stable identifiers to use for Pickle serialization.""" + +import hashlib +import unittest + +from parameterized import parameterized + +# pylint: disable=unused-import +from apache_beam.internal import code_object_pickler +from apache_beam.internal.test_data import module_1 +from apache_beam.internal.test_data import module_1_class_added +from apache_beam.internal.test_data import module_1_function_added +from apache_beam.internal.test_data import module_1_global_variable_added +from apache_beam.internal.test_data import module_1_lambda_variable_added +from apache_beam.internal.test_data import module_1_local_variable_added +from apache_beam.internal.test_data import module_1_local_variable_removed +from apache_beam.internal.test_data import module_1_nested_function_2_added +from apache_beam.internal.test_data import module_1_nested_function_added +from apache_beam.internal.test_data import module_2 +from apache_beam.internal.test_data import module_2_modified +from apache_beam.internal.test_data import module_3 +from apache_beam.internal.test_data import module_3_modified +from apache_beam.internal.test_data import module_with_default_argument + + +def top_level_function(): + return 1 + + +top_level_lambda = lambda x: 1 + + +def get_nested_function(): + def nested_function(): + return 1 + + return nested_function + + +def get_lambda_from_dictionary(): + d = {"a": lambda x: 1, "b": lambda y: 2} + return d["a"] + + +def get_lambda_from_dictionary_same_args(): + d = {"a": lambda x: 1, "b": lambda x: x + 1} + return d["a"] + + +def function_with_lambda_default_argument(fn=lambda x: 1): + return fn + + +def function_with_function_default_argument(fn=top_level_function): + return fn + + +def function_decorator(f): + return lambda x: f(f(x)) + + +@function_decorator +def add_one(x): + return x + 1 + + +class ClassWithFunction: + def process(self): + return 1 + + +class ClassWithStaticMethod: + @staticmethod + def static_method(): + return 1 + + +class ClassWithClassMethod: + @classmethod + def class_method(cls): + return 1 + + +class ClassWithNestedFunction: + def process(self): + def nested_function(): + return 1 + + return nested_function + + +class ClassWithLambda: + def process(self): + return lambda: 1 + + +class ClassWithNestedClass: + class InnerClass: + def process(self): + return 1 + + +class ClassWithNestedLambda: + def process(self): + def get_lambda_from_dictionary(): + d = {"a": lambda x: 1, "b": lambda y: 2} + return d["a"] + + return get_lambda_from_dictionary() + + +prefix = __name__ + +test_cases = [ + (top_level_function, f"{prefix}.top_level_function" + ".__code__"), + (top_level_lambda, f"{prefix}.top_level_lambda" + ".__code__"), + ( + get_nested_function(), ( + f"{prefix}.get_nested_function" + ".__code__.co_consts[nested_function]")), + ( + get_lambda_from_dictionary(), + ( + f"{prefix}" + ".get_lambda_from_dictionary.__code__.co_consts[, ('x',)]") + ), + ( + get_lambda_from_dictionary_same_args(), + ( + f"{prefix}" + ".get_lambda_from_dictionary_same_args.__code__.co_consts" + "[, ('x',), " + hashlib.md5( + get_lambda_from_dictionary_same_args().__code__.co_code). + hexdigest() + "]")), + ( + function_with_lambda_default_argument(), + ( + f"{prefix}" + ".function_with_lambda_default_argument.__defaults__[0].__code__")), + ( + function_with_function_default_argument(), + f"{prefix}.top_level_function" + ".__code__"), + (add_one, f"{prefix}.function_decorator" + ".__code__.co_consts[]"), + ( + ClassWithFunction.process, + f"{prefix}.ClassWithFunction" + ".process.__code__"), + ( + ClassWithStaticMethod.static_method, + f"{prefix}.ClassWithStaticMethod" + ".static_method.__code__"), + ( + ClassWithClassMethod.class_method, + f"{prefix}.ClassWithClassMethod" + ".class_method.__code__"), + ( + ClassWithNestedFunction().process(), + ( + f"{prefix}.ClassWithNestedFunction.process.__code__.co_consts" + "[nested_function]")), + ( + ClassWithLambda().process(), + f"{prefix}.ClassWithLambda.process.__code__.co_consts[]"), + ( + ClassWithNestedClass.InnerClass().process, + f"{prefix}.ClassWithNestedClass.InnerClass.process.__code__"), + ( + ClassWithNestedLambda().process(), + ( + f"{prefix}" + ".ClassWithNestedLambda.process.__code__.co_consts" + "[get_lambda_from_dictionary].co_consts[, ('x',)]")), + ( + ClassWithNestedLambda.process, + f"{prefix}.ClassWithNestedLambda.process.__code__"), +] + + +class CodeObjectIdentifierGenerationTest(unittest.TestCase): + @parameterized.expand(test_cases) + def test_get_code_object_identifier(self, callable, expected_path): + actual = code_object_pickler.get_code_object_identifier(callable) + self.assertEqual(actual, expected_path) + + @parameterized.expand(test_cases) + def test_get_code_from_identifier(self, expected_callable, path): + actual = code_object_pickler.get_code_from_identifier(path) + self.assertEqual(actual, expected_callable.__code__) + + @parameterized.expand(test_cases) + def test_roundtrip(self, callable, unused_path): + path = code_object_pickler.get_code_object_identifier(callable) + actual = code_object_pickler.get_code_from_identifier(path) + self.assertEqual(actual, callable.__code__) + + +class GetCodeFromCodeObjectIdentifierTest(unittest.TestCase): + def test_empty_path_raises_exception(self): + with self.assertRaisesRegex(ValueError, "Path must not be empty"): + code_object_pickler.get_code_from_identifier("") + + def test_invalid_default_index_raises_exception(self): + with self.assertRaisesRegex(ValueError, "out of bounds"): + code_object_pickler.get_code_from_identifier( + "apache_beam.internal.test_data.module_with_default_argument." + "function_with_lambda_default_argument.__defaults__[1]") + + def test_invalid_single_name_path_raises_exception(self): + with self.assertRaisesRegex(AttributeError, + "Could not find code object with path"): + code_object_pickler.get_code_from_identifier( + "apache_beam.internal.test_data.module_3." + "my_function.__code__.co_consts[something]") + + def test_invalid_lambda_with_args_path_raises_exception(self): + with self.assertRaisesRegex(AttributeError, + "Could not find code object with path"): + code_object_pickler.get_code_from_identifier( + "apache_beam.internal.test_data.module_3." + "my_function.__code__.co_consts[, ('x',)]") + + def test_invalid_lambda_with_hash_path_raises_exception(self): + with self.assertRaisesRegex(AttributeError, + "Could not find code object with path"): + code_object_pickler.get_code_from_identifier( + "apache_beam.internal.test_data.module_3." + "my_function.__code__.co_consts[, ('',), 1234567890]") + + def test_adding_local_variable_in_class_preserves_object(self): + self.assertEqual( + code_object_pickler.get_code_from_identifier( + code_object_pickler.get_code_object_identifier( + module_2.AddLocalVariable.my_method(self)).replace( + "module_2", "module_2_modified")), + module_2_modified.AddLocalVariable.my_method(self).__code__, + ) + + def test_removing_local_variable_in_class_preserves_object(self): + self.assertEqual( + code_object_pickler.get_code_from_identifier( + code_object_pickler.get_code_object_identifier( + module_2.RemoveLocalVariable.my_method(self)).replace( + "module_2", "module_2_modified")), + module_2_modified.RemoveLocalVariable.my_method(self).__code__, + ) + + def test_adding_lambda_variable_in_class_preserves_object(self): + self.assertEqual( + code_object_pickler.get_code_from_identifier( + code_object_pickler.get_code_object_identifier( + module_2.AddLambdaVariable.my_method(self)).replace( + "module_2", "module_2_modified")), + module_2_modified.AddLambdaVariable.my_method(self).__code__, + ) + + def test_removing_lambda_variable_in_class_changes_object(self): + with self.assertRaisesRegex(AttributeError, "object has no attribute"): + code_object_pickler.get_code_from_identifier( + code_object_pickler.get_code_object_identifier( + module_2.RemoveLambdaVariable.my_method(self)).replace( + "module_2", "module_2_modified")) + + def test_adding_nested_function_in_class_preserves_object(self): + self.assertEqual( + code_object_pickler.get_code_from_identifier( + code_object_pickler.get_code_object_identifier( + module_2.ClassWithNestedFunction.my_method(self)).replace( + "module_2", "module_2_modified")), + module_2_modified.ClassWithNestedFunction.my_method(self).__code__, + ) + + def test_adding_nested_function_2_in_class_preserves_object(self): + self.assertEqual( + code_object_pickler.get_code_from_identifier( + code_object_pickler.get_code_object_identifier( + module_2.ClassWithNestedFunction2.my_method(self)).replace( + "module_2", "module_2_modified")), + module_2_modified.ClassWithNestedFunction2.my_method(self).__code__, + ) + + def test_adding_new_function_in_class_preserves_object(self): + self.assertEqual( + code_object_pickler.get_code_from_identifier( + code_object_pickler.get_code_object_identifier( + module_2.ClassWithTwoMethods.my_method(self)).replace( + "module_2", "module_2_modified")), + module_2_modified.ClassWithTwoMethods.my_method(self).__code__, + ) + + def test_removing_method_in_class_preserves_object(self): + self.assertEqual( + code_object_pickler.get_code_from_identifier( + code_object_pickler.get_code_object_identifier( + module_2.RemoveMethod.my_method(self)).replace( + "module_2", "module_2_modified")), + module_2_modified.RemoveMethod.my_method(self).__code__, + ) + + def test_adding_global_variable_preserves_object(self): + self.assertEqual( + code_object_pickler.get_code_from_identifier( + code_object_pickler.get_code_object_identifier( + module_1.my_function()).replace( + "module_1", + "module_1_global_variable_added", + )), + module_1_global_variable_added.my_function().__code__, + ) + + def test_adding_top_level_function_preserves_object(self): + self.assertEqual( + code_object_pickler.get_code_from_identifier( + code_object_pickler.get_code_object_identifier( + module_1.my_function()).replace( + "module_1", "module_1_function_added")), + module_1_function_added.my_function().__code__, + ) + + def test_adding_local_variable_in_function_preserves_object(self): + self.assertEqual( + code_object_pickler.get_code_from_identifier( + code_object_pickler.get_code_object_identifier( + module_1.my_function()).replace( + "module_1", "module_1_local_variable_added")), + module_1_local_variable_added.my_function().__code__, + ) + + def test_removing_local_variable_in_function_preserves_object(self): + self.assertEqual( + code_object_pickler.get_code_from_identifier( + code_object_pickler.get_code_object_identifier( + module_1.my_function()).replace( + "module_1", "module_1_local_variable_removed")), + module_1_local_variable_removed.my_function().__code__, + ) + + def test_adding_nested_function_in_function_preserves_object(self): + self.assertEqual( + code_object_pickler.get_code_from_identifier( + code_object_pickler.get_code_object_identifier( + module_1.my_function()).replace( + "module_1", "module_1_nested_function_added")), + module_1_nested_function_added.my_function().__code__, + ) + + def test_adding_nested_function_2_in_function_preserves_object(self): + self.assertEqual( + code_object_pickler.get_code_from_identifier( + code_object_pickler.get_code_object_identifier( + module_1.my_function()).replace( + "module_1", "module_1_nested_function_2_added")), + module_1_nested_function_2_added.my_function().__code__, + ) + + def test_adding_class_preserves_object(self): + self.assertEqual( + code_object_pickler.get_code_from_identifier( + code_object_pickler.get_code_object_identifier( + module_1.my_function()).replace( + "module_1", "module_1_class_added")), + module_1_class_added.my_function().__code__, + ) + + def test_adding_lambda_variable_in_function_preserves_object(self): + self.assertEqual( + code_object_pickler.get_code_from_identifier( + code_object_pickler.get_code_object_identifier( + module_1.my_function()).replace( + "module_1", "module_1_lambda_variable_added")), + module_1_lambda_variable_added.my_function().__code__, + ) + + def test_removing_lambda_variable_in_function_raises_exception(self): + with self.assertRaisesRegex(AttributeError, "object has no attribute"): + code_object_pickler.get_code_from_identifier( + code_object_pickler.get_code_object_identifier( + module_3.my_function()).replace("module_3", "module_3_modified")) + + +class CodePathStabilityTest(unittest.TestCase): + def test_adding_local_variable_in_class_preserves_path(self): + self.assertEqual( + code_object_pickler.get_code_object_identifier( + module_2.AddLocalVariable.my_method(self)).replace( + "module_2", "module_name"), + code_object_pickler.get_code_object_identifier( + module_2_modified.AddLocalVariable.my_method(self)).replace( + "module_2_modified", "module_name"), + ) + + def test_removing_local_variable_in_class_preserves_path(self): + self.assertEqual( + code_object_pickler.get_code_object_identifier( + module_2.RemoveLocalVariable.my_method(self)).replace( + "module_2", "module_name"), + code_object_pickler.get_code_object_identifier( + module_2_modified.RemoveLocalVariable.my_method(self)).replace( + "module_2_modified", "module_name"), + ) + + def test_adding_lambda_variable_in_class_changes_path(self): + self.assertNotEqual( + code_object_pickler.get_code_object_identifier( + module_2.AddLambdaVariable.my_method(self)).replace( + "module_2", "module_name"), + code_object_pickler.get_code_object_identifier( + module_2_modified.AddLambdaVariable.my_method(self)).replace( + "module_2_modified", "module_name"), + ) + + def test_removing_lambda_variable_in_class_changes_path(self): + self.assertNotEqual( + code_object_pickler.get_code_object_identifier( + module_2.RemoveLambdaVariable.my_method(self)).replace( + "module_2", "module_name"), + code_object_pickler.get_code_object_identifier( + module_2_modified.RemoveLambdaVariable.my_method(self)).replace( + "module_2_modified", "module_name"), + ) + + def test_adding_nested_function_in_class_preserves_path(self): + self.assertEqual( + code_object_pickler.get_code_object_identifier( + module_2.ClassWithNestedFunction.my_method(self)).replace( + "module_2", "module_name"), + code_object_pickler.get_code_object_identifier( + module_2_modified.ClassWithNestedFunction.my_method(self)).replace( + "module_2_modified", "module_name"), + ) + + def test_adding_nested_function_2_in_class_preserves_path(self): + self.assertEqual( + code_object_pickler.get_code_object_identifier( + module_2.ClassWithNestedFunction2.my_method(self)).replace( + "module_2", "module_name"), + code_object_pickler.get_code_object_identifier( + module_2_modified.ClassWithNestedFunction2.my_method(self)).replace( + "module_2_modified", "module_name"), + ) + + def test_adding_new_function_in_class_preserves_path(self): + self.assertEqual( + code_object_pickler.get_code_object_identifier( + module_2.ClassWithTwoMethods.my_method(self)).replace( + "module_2", "module_name"), + code_object_pickler.get_code_object_identifier( + module_2_modified.ClassWithTwoMethods.my_method(self)).replace( + "module_2_modified", "module_name"), + ) + + def test_removing_function_in_class_preserves_path(self): + self.assertEqual( + code_object_pickler.get_code_object_identifier( + module_2.RemoveMethod.my_method(self)).replace( + "module_2", "module_name"), + code_object_pickler.get_code_object_identifier( + module_2_modified.RemoveMethod.my_method(self)).replace( + "module_2_modified", "module_name"), + ) + + def test_adding_global_variable_preserves_path(self): + self.assertEqual( + code_object_pickler.get_code_object_identifier( + module_1.my_function()).replace("module_1", "module_name"), + code_object_pickler.get_code_object_identifier( + module_1_global_variable_added.my_function()).replace( + "module_1_global_variable_added", "module_name"), + ) + + def test_adding_top_level_function_preserves_path(self): + self.assertEqual( + code_object_pickler.get_code_object_identifier( + module_1.my_function()).replace("module_1", "module_name"), + code_object_pickler.get_code_object_identifier( + module_1_function_added.my_function()).replace( + "module_1_function_added", "module_name"), + ) + + def test_adding_local_variable_in_function_preserves_path(self): + self.assertEqual( + code_object_pickler.get_code_object_identifier( + module_1.my_function()).replace("module_1", "module_name"), + code_object_pickler.get_code_object_identifier( + module_1_local_variable_added.my_function()).replace( + "module_1_local_variable_added", "module_name"), + ) + + def test_removing_local_variable_in_function_preserves_path(self): + self.assertEqual( + code_object_pickler.get_code_object_identifier( + module_1.my_function()).replace("module_1", "module_name"), + code_object_pickler.get_code_object_identifier( + module_1_local_variable_removed.my_function()).replace( + "module_1_local_variable_removed", "module_name"), + ) + + def test_adding_nested_function_in_function_preserves_path(self): + self.assertEqual( + code_object_pickler.get_code_object_identifier( + module_1.my_function()).replace("module_1", "module_name"), + code_object_pickler.get_code_object_identifier( + module_1_nested_function_added.my_function()).replace( + "module_1_nested_function_added", "module_name"), + ) + + def test_adding_nested_function_2_in_function_preserves_path(self): + self.assertEqual( + code_object_pickler.get_code_object_identifier( + module_1.my_function()).replace("module_1", "module_name"), + code_object_pickler.get_code_object_identifier( + module_1_nested_function_2_added.my_function()).replace( + "module_1_nested_function_2_added", "module_name"), + ) + + def test_adding_class_preserves_path(self): + self.assertEqual( + code_object_pickler.get_code_object_identifier( + module_1.my_function()).replace("module_1", "module_name"), + code_object_pickler.get_code_object_identifier( + module_1_class_added.my_function()).replace( + "module_1_class_added", "module_name"), + ) + + def test_adding_lambda_variable_in_function_changes_path(self): + self.assertNotEqual( + code_object_pickler.get_code_object_identifier( + module_1.my_function()).replace("module_1", "module_name"), + code_object_pickler.get_code_object_identifier( + module_1_lambda_variable_added.my_function()).replace( + "module_1_lambda_variable_added", "module_name"), + ) + + def test_removing_lambda_variable_in_function_changes_path(self): + self.assertNotEqual( + code_object_pickler.get_code_object_identifier( + module_3.my_function()).replace("module_3", "module_name"), + code_object_pickler.get_code_object_identifier( + module_3_modified.my_function()).replace( + "module_3_modified", "module_name"), + ) + + +if __name__ == "__main__": + unittest.main() diff --git a/sdks/python/apache_beam/internal/pickler.py b/sdks/python/apache_beam/internal/pickler.py index 256f88c5453f..6f8dba463bc3 100644 --- a/sdks/python/apache_beam/internal/pickler.py +++ b/sdks/python/apache_beam/internal/pickler.py @@ -29,10 +29,15 @@ """ from apache_beam.internal import cloudpickle_pickler -from apache_beam.internal import dill_pickler + +try: + from apache_beam.internal import dill_pickler +except ImportError: + dill_pickler = None # type: ignore[assignment] USE_CLOUDPICKLE = 'cloudpickle' USE_DILL = 'dill' +USE_DILL_UNSAFE = 'dill_unsafe' DEFAULT_PICKLE_LIB = USE_CLOUDPICKLE desired_pickle_lib = cloudpickle_pickler @@ -74,14 +79,29 @@ def load_session(file_path): def set_library(selected_library=DEFAULT_PICKLE_LIB): """ Sets pickle library that will be used. """ global desired_pickle_lib - # If switching to or from dill, update the pickler hook overrides. - if (selected_library == USE_DILL) != (desired_pickle_lib == dill_pickler): - dill_pickler.override_pickler_hooks(selected_library == USE_DILL) if selected_library == 'default': selected_library = DEFAULT_PICKLE_LIB - if selected_library == USE_DILL: + if selected_library == USE_DILL and not dill_pickler: + raise ImportError( + "Pipeline option pickle_library=dill is set, but dill is not " + "installed. Install apache-beam with the dill extras package " + "e.g. apache-beam[dill].") + if selected_library == USE_DILL_UNSAFE and not dill_pickler: + raise ImportError( + "Pipeline option pickle_library=dill_unsafe is set, but dill is not " + "installed. Install dill in job submission and runtime environments.") + + is_currently_dill = (desired_pickle_lib == dill_pickler) + dill_is_requested = ( + selected_library == USE_DILL or selected_library == USE_DILL_UNSAFE) + + # If switching to or from dill, update the pickler hook overrides. + if is_currently_dill != dill_is_requested: + dill_pickler.override_pickler_hooks(selected_library == USE_DILL) + + if dill_is_requested: desired_pickle_lib = dill_pickler elif selected_library == USE_CLOUDPICKLE: desired_pickle_lib = cloudpickle_pickler diff --git a/sdks/python/apache_beam/internal/pickler_test.py b/sdks/python/apache_beam/internal/pickler_test.py index 7048f680de87..a0135b221e8c 100644 --- a/sdks/python/apache_beam/internal/pickler_test.py +++ b/sdks/python/apache_beam/internal/pickler_test.py @@ -25,6 +25,7 @@ import types import unittest +import pytest from parameterized import param from parameterized import parameterized @@ -34,6 +35,12 @@ from apache_beam.internal.pickler import loads +def maybe_skip_if_no_dill(pickle_library): + if pickle_library == 'dill': + pytest.importorskip("dill") + + +@pytest.mark.uses_dill class PicklerTest(unittest.TestCase): NO_MAPPINGPROXYTYPE = not hasattr(types, "MappingProxyType") @@ -43,6 +50,7 @@ class PicklerTest(unittest.TestCase): param(pickle_lib='cloudpickle'), ]) def test_basics(self, pickle_lib): + maybe_skip_if_no_dill(pickle_lib) pickler.set_library(pickle_lib) self.assertEqual([1, 'a', ('z', )], loads(dumps([1, 'a', ('z', )]))) @@ -55,6 +63,7 @@ def test_basics(self, pickle_lib): ]) def test_lambda_with_globals(self, pickle_lib): """Tests that the globals of a function are preserved.""" + maybe_skip_if_no_dill(pickle_lib) pickler.set_library(pickle_lib) # The point of the test is that the lambda being called after unpickling @@ -68,6 +77,7 @@ def test_lambda_with_globals(self, pickle_lib): param(pickle_lib='cloudpickle'), ]) def test_lambda_with_main_globals(self, pickle_lib): + maybe_skip_if_no_dill(pickle_lib) pickler.set_library(pickle_lib) self.assertEqual(unittest, loads(dumps(lambda: unittest))()) @@ -77,6 +87,7 @@ def test_lambda_with_main_globals(self, pickle_lib): ]) def test_lambda_with_closure(self, pickle_lib): """Tests that the closure of a function is preserved.""" + maybe_skip_if_no_dill(pickle_lib) pickler.set_library(pickle_lib) self.assertEqual( 'closure: abc', @@ -88,6 +99,7 @@ def test_lambda_with_closure(self, pickle_lib): ]) def test_class(self, pickle_lib): """Tests that a class object is pickled correctly.""" + maybe_skip_if_no_dill(pickle_lib) pickler.set_library(pickle_lib) self.assertEqual(['abc', 'def'], loads(dumps(module_test.Xyz))().foo('abc def')) @@ -98,6 +110,7 @@ def test_class(self, pickle_lib): ]) def test_object(self, pickle_lib): """Tests that a class instance is pickled correctly.""" + maybe_skip_if_no_dill(pickle_lib) pickler.set_library(pickle_lib) self.assertEqual(['abc', 'def'], loads(dumps(module_test.XYZ_OBJECT)).foo('abc def')) @@ -108,6 +121,7 @@ def test_object(self, pickle_lib): ]) def test_nested_class(self, pickle_lib): """Tests that a nested class object is pickled correctly.""" + maybe_skip_if_no_dill(pickle_lib) pickler.set_library(pickle_lib) self.assertEqual( 'X:abc', loads(dumps(module_test.TopClass.NestedClass('abc'))).datum) @@ -121,6 +135,7 @@ def test_nested_class(self, pickle_lib): ]) def test_dynamic_class(self, pickle_lib): """Tests that a nested class object is pickled correctly.""" + maybe_skip_if_no_dill(pickle_lib) pickler.set_library(pickle_lib) self.assertEqual( 'Z:abc', loads(dumps(module_test.create_class('abc'))).get()) @@ -130,6 +145,7 @@ def test_dynamic_class(self, pickle_lib): param(pickle_lib='cloudpickle'), ]) def test_generators(self, pickle_lib): + maybe_skip_if_no_dill(pickle_lib) pickler.set_library(pickle_lib) with self.assertRaises(TypeError): dumps((_ for _ in range(10))) @@ -139,6 +155,7 @@ def test_generators(self, pickle_lib): param(pickle_lib='cloudpickle'), ]) def test_recursive_class(self, pickle_lib): + maybe_skip_if_no_dill(pickle_lib) pickler.set_library(pickle_lib) self.assertEqual( 'RecursiveClass:abc', @@ -149,6 +166,7 @@ def test_recursive_class(self, pickle_lib): param(pickle_lib='cloudpickle'), ]) def test_pickle_rlock(self, pickle_lib): + maybe_skip_if_no_dill(pickle_lib) pickler.set_library(pickle_lib) rlock_instance = threading.RLock() rlock_type = type(rlock_instance) @@ -160,6 +178,7 @@ def test_pickle_rlock(self, pickle_lib): param(pickle_lib='cloudpickle'), ]) def test_save_paths(self, pickle_lib): + maybe_skip_if_no_dill(pickle_lib) pickler.set_library(pickle_lib) f = loads(dumps(lambda x: x)) co_filename = f.__code__.co_filename @@ -171,6 +190,7 @@ def test_save_paths(self, pickle_lib): param(pickle_lib='cloudpickle'), ]) def test_dump_and_load_mapping_proxy(self, pickle_lib): + maybe_skip_if_no_dill(pickle_lib) pickler.set_library(pickle_lib) self.assertEqual( 'def', loads(dumps(types.MappingProxyType({'abc': 'def'})))['abc']) @@ -184,6 +204,7 @@ def test_dump_and_load_mapping_proxy(self, pickle_lib): param(pickle_lib='cloudpickle'), ]) def test_dataclass(self, pickle_lib): + maybe_skip_if_no_dill(pickle_lib) exec( ''' from apache_beam.internal.module_test import DataClass @@ -195,6 +216,7 @@ def test_dataclass(self, pickle_lib): param(pickle_lib='cloudpickle'), ]) def test_class_states_not_changed_at_subsequent_loading(self, pickle_lib): + maybe_skip_if_no_dill(pickle_lib) pickler.set_library(pickle_lib) class Local: @@ -255,6 +277,7 @@ def maybe_get_sets_with_different_iteration_orders(self): return set1, set2 def test_best_effort_determinism(self): + maybe_skip_if_no_dill('dill') pickler.set_library('dill') set1, set2 = self.maybe_get_sets_with_different_iteration_orders() self.assertEqual( @@ -267,6 +290,7 @@ def test_best_effort_determinism(self): self.skipTest('Set iteration orders matched. Test results inconclusive.') def test_disable_best_effort_determinism(self): + maybe_skip_if_no_dill('dill') pickler.set_library('dill') set1, set2 = self.maybe_get_sets_with_different_iteration_orders() # The test relies on the sets having different iteration orders for the diff --git a/sdks/python/apache_beam/internal/test_data/__init__.py b/sdks/python/apache_beam/internal/test_data/__init__.py new file mode 100644 index 000000000000..7f27bba88cf5 --- /dev/null +++ b/sdks/python/apache_beam/internal/test_data/__init__.py @@ -0,0 +1,20 @@ +# +# 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. +# + +"""Test data to validate that code identifiers are invariant to small + modifications. +""" diff --git a/sdks/python/apache_beam/internal/test_data/module_1.py b/sdks/python/apache_beam/internal/test_data/module_1.py new file mode 100644 index 000000000000..9dd7ebf90881 --- /dev/null +++ b/sdks/python/apache_beam/internal/test_data/module_1.py @@ -0,0 +1,27 @@ +# +# 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. +# + +"""Module for testing code path generation with functions. +Counterpart to after_module_with_functions and is used as a test case +for various code changes. +""" + + +def my_function(): + a = 1 # pylint: disable=unused-variable + b = lambda: 2 + return b diff --git a/sdks/python/apache_beam/internal/test_data/module_1_class_added.py b/sdks/python/apache_beam/internal/test_data/module_1_class_added.py new file mode 100644 index 000000000000..0a4a7f73974c --- /dev/null +++ b/sdks/python/apache_beam/internal/test_data/module_1_class_added.py @@ -0,0 +1,34 @@ +# +# 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. +# + +"""Module for testing code path generation with functions. +Counterpart to before_module_with_functions and is used as a test case +for adding a class. +""" + + +class MyClass: + def another_function(self): + a = 1 # pylint: disable=unused-variable + b = lambda: 2 + return b + + +def my_function(): + a = 1 # pylint: disable=unused-variable + b = lambda: 2 + return b diff --git a/sdks/python/apache_beam/internal/test_data/module_1_function_added.py b/sdks/python/apache_beam/internal/test_data/module_1_function_added.py new file mode 100644 index 000000000000..063125a2500d --- /dev/null +++ b/sdks/python/apache_beam/internal/test_data/module_1_function_added.py @@ -0,0 +1,33 @@ +# +# 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. +# + +"""Module for testing code path generation with functions. +Counterpart to before_module_with_functions and is used as a test case +for adding a function. +""" + + +def another_function(): + a = 1 # pylint: disable=unused-variable + b = lambda: 2 + return b + + +def my_function(): + a = 1 # pylint: disable=unused-variable + b = lambda: 2 + return b diff --git a/sdks/python/apache_beam/internal/test_data/module_1_global_variable_added.py b/sdks/python/apache_beam/internal/test_data/module_1_global_variable_added.py new file mode 100644 index 000000000000..c4d20f27c837 --- /dev/null +++ b/sdks/python/apache_beam/internal/test_data/module_1_global_variable_added.py @@ -0,0 +1,29 @@ +# +# 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. +# + +"""Module for testing code path generation with functions. +Counterpart to before_module_with_functions and is used as a test case +for adding a global variable. +""" + +GLOBAL_VARIABLE = lambda: 3 + + +def my_function(): + a = 1 # pylint: disable=unused-variable + b = lambda: 2 + return b diff --git a/sdks/python/apache_beam/internal/test_data/module_1_lambda_variable_added.py b/sdks/python/apache_beam/internal/test_data/module_1_lambda_variable_added.py new file mode 100644 index 000000000000..7e1ddc48790f --- /dev/null +++ b/sdks/python/apache_beam/internal/test_data/module_1_lambda_variable_added.py @@ -0,0 +1,28 @@ +# +# 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. +# + +"""Module for testing code path generation with functions. +Counterpart to before_module_with_functions and is used as a test case +for adding a lambda variable. +""" + + +def my_function(): + a = 1 # pylint: disable=unused-variable + b = lambda: 2 + new_lambda_variable = lambda: 4 # pylint: disable=unused-variable + return b diff --git a/sdks/python/apache_beam/internal/test_data/module_1_local_variable_added.py b/sdks/python/apache_beam/internal/test_data/module_1_local_variable_added.py new file mode 100644 index 000000000000..a808e7ca4e9b --- /dev/null +++ b/sdks/python/apache_beam/internal/test_data/module_1_local_variable_added.py @@ -0,0 +1,28 @@ +# +# 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. +# + +"""Module for testing code path generation with functions. +Counterpart to before_module_with_functions and is used as a test case +for adding a variable. +""" + + +def my_function(): + a = 1 # pylint: disable=unused-variable + b = lambda: 2 + new_local_variable = 3 # pylint: disable=unused-variable + return b diff --git a/sdks/python/apache_beam/internal/test_data/module_1_local_variable_removed.py b/sdks/python/apache_beam/internal/test_data/module_1_local_variable_removed.py new file mode 100644 index 000000000000..072b93d48447 --- /dev/null +++ b/sdks/python/apache_beam/internal/test_data/module_1_local_variable_removed.py @@ -0,0 +1,26 @@ +# +# 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. +# + +"""Module for testing code path generation with functions. +Counterpart to before_module_with_functions and is used as a test case +for removing a variable. +""" + + +def my_function(): + b = lambda: 2 + return b diff --git a/sdks/python/apache_beam/internal/test_data/module_1_nested_function_2_added.py b/sdks/python/apache_beam/internal/test_data/module_1_nested_function_2_added.py new file mode 100644 index 000000000000..191e2e92065a --- /dev/null +++ b/sdks/python/apache_beam/internal/test_data/module_1_nested_function_2_added.py @@ -0,0 +1,32 @@ +# +# 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. +# + +"""Module for testing code path generation with functions. +Counterpart to before_module_with_functions and is used as a test case +for adding a nested function. +""" + + +def my_function(): + a = 1 # pylint: disable=unused-variable + b = lambda: 2 + + def nested_function(): # pylint: disable=unused-variable + c = 3 + return c + + return b diff --git a/sdks/python/apache_beam/internal/test_data/module_1_nested_function_added.py b/sdks/python/apache_beam/internal/test_data/module_1_nested_function_added.py new file mode 100644 index 000000000000..358bf12b3d05 --- /dev/null +++ b/sdks/python/apache_beam/internal/test_data/module_1_nested_function_added.py @@ -0,0 +1,31 @@ +# +# 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. +# + +"""Module for testing code path generation with functions. +Counterpart to before_module_with_functions and is used as a test case +for adding a nested function. +""" + + +def my_function(): + def nested_function(): # pylint: disable=unused-variable + c = 3 + return c + + a = 1 # pylint: disable=unused-variable + b = lambda: 2 + return b diff --git a/sdks/python/apache_beam/internal/test_data/module_2.py b/sdks/python/apache_beam/internal/test_data/module_2.py new file mode 100644 index 000000000000..e59deca69f0b --- /dev/null +++ b/sdks/python/apache_beam/internal/test_data/module_2.py @@ -0,0 +1,82 @@ +# +# 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. +# + +"""Module for testing code path generation with classes. +Counterpart to after_module_with_classes and is used as a test case +for various code changes. +""" + + +class AddLocalVariable: + def my_method(self): + a = 1 # pylint: disable=unused-variable + b = lambda: 2 + return b + + +class RemoveLocalVariable: + def my_method(self): + a = 1 # pylint: disable=unused-variable + b = lambda: 2 + return b + + +class AddLambdaVariable: + def my_method(self): + a = 1 # pylint: disable=unused-variable + b = lambda: 2 + return b + + +class RemoveLambdaVariable: + def my_method(self): + a = lambda: 1 # pylint: disable=unused-variable + b = lambda: 2 + return b + + +class ClassWithNestedFunction: + def my_method(self): + a = 1 # pylint: disable=unused-variable + b = lambda: 2 + return b + + +class ClassWithNestedFunction2: + def my_method(self): + a = 1 # pylint: disable=unused-variable + b = lambda: 2 + return b + + +class ClassWithTwoMethods: + def my_method(self): + a = 1 # pylint: disable=unused-variable + b = lambda: 2 + return b + + +class RemoveMethod: + def another_method(self): + a = 1 # pylint: disable=unused-variable + b = lambda: 2 + return b + + def my_method(self): + a = 1 # pylint: disable=unused-variable + b = lambda: 2 + return b diff --git a/sdks/python/apache_beam/internal/test_data/module_2_modified.py b/sdks/python/apache_beam/internal/test_data/module_2_modified.py new file mode 100644 index 000000000000..56e57f8cb87b --- /dev/null +++ b/sdks/python/apache_beam/internal/test_data/module_2_modified.py @@ -0,0 +1,92 @@ +# +# 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. +# + +"""Module for testing code path generation with classes. +Counterpart to before_module_with_classes and is used as a test case +for various code changes. +""" + + +class AddLocalVariable: + def my_method(self): + a = 1 # pylint: disable=unused-variable + b = lambda: 2 + new_local_variable = 3 # pylint: disable=unused-variable + return b + + +class RemoveLocalVariable: + def my_method(self): + b = lambda: 2 + return b + + +class AddLambdaVariable: + def my_method(self): + a = 1 # pylint: disable=unused-variable + b = lambda: 2 + c = lambda: 3 # pylint: disable=unused-variable + return b + + +class RemoveLambdaVariable: + def my_method(self): + b = lambda: 2 + return b + + +class ClassWithNestedFunction: + def my_method(self): + def nested_function(): # pylint: disable=unused-variable + c = 3 + return c + + a = 1 # pylint: disable=unused-variable + b = lambda: 2 + return b + + +class ClassWithNestedFunction2: + def my_method(self): + a = 1 # pylint: disable=unused-variable + b = lambda: 2 + + def nested_function(): # pylint: disable=unused-variable + c = 3 + return c + + return b + + +class ClassWithTwoMethods: + def another_method(self): + a = 1 # pylint: disable=unused-variable + b = lambda: 2 + return b + + def my_method(self): + a = 1 # pylint: disable=unused-variable + b = lambda: 2 + return b + + +class RemoveMethod: + def my_method(self): + a = 1 # pylint: disable=unused-variable + + b = lambda: 2 + return b diff --git a/sdks/python/apache_beam/internal/test_data/module_3.py b/sdks/python/apache_beam/internal/test_data/module_3.py new file mode 100644 index 000000000000..98f9c29ceb52 --- /dev/null +++ b/sdks/python/apache_beam/internal/test_data/module_3.py @@ -0,0 +1,26 @@ +# +# 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. +# + +"""Module for testing code path generation with functions. +Used as a test case for various code changes. +""" + + +def my_function(): + a = lambda: 1 # pylint: disable=unused-variable + b = lambda: 2 + return b diff --git a/sdks/python/apache_beam/internal/test_data/module_3_modified.py b/sdks/python/apache_beam/internal/test_data/module_3_modified.py new file mode 100644 index 000000000000..326dedb93f0d --- /dev/null +++ b/sdks/python/apache_beam/internal/test_data/module_3_modified.py @@ -0,0 +1,26 @@ +# +# 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. +# + +"""Module for testing code path generation with functions. +Counterpart to before_module_with_functions and is used as a test case +for removing a lambda variable. +""" + + +def my_function(): + b = lambda: 2 + return b diff --git a/sdks/python/apache_beam/internal/test_data/module_with_default_argument.py b/sdks/python/apache_beam/internal/test_data/module_with_default_argument.py new file mode 100644 index 000000000000..a1758a8be740 --- /dev/null +++ b/sdks/python/apache_beam/internal/test_data/module_with_default_argument.py @@ -0,0 +1,24 @@ +# +# 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. +# + +"""Module for testing code path generation with functions. +Used as a test case for default arguments. +""" + + +def function_with_lambda_default_argument(fn=lambda x: 1): + return fn diff --git a/sdks/python/apache_beam/io/avroio.py b/sdks/python/apache_beam/io/avroio.py index 553b6c741f3d..da904bf6fb55 100644 --- a/sdks/python/apache_beam/io/avroio.py +++ b/sdks/python/apache_beam/io/avroio.py @@ -354,8 +354,7 @@ def split_points_unclaimed(stop_position): while range_tracker.try_claim(next_block_start): block = next(blocks) next_block_start = block.offset + block.size - for record in block: - yield record + yield from block _create_avro_source = _FastAvroSource @@ -375,7 +374,8 @@ def __init__( num_shards=0, shard_name_template=None, mime_type='application/x-avro', - use_fastavro=True): + use_fastavro=True, + triggering_frequency=None): """Initialize a WriteToAvro transform. Args: @@ -393,17 +393,30 @@ def __init__( Constraining the number of shards is likely to reduce the performance of a pipeline. Setting this value is not recommended unless you require a specific number of output files. + In streaming if not set, the service will write a file per bundle. shard_name_template: A template string containing placeholders for - the shard number and shard count. When constructing a filename for a - particular shard number, the upper-case letters 'S' and 'N' are - replaced with the 0-padded shard number and shard count respectively. - This argument can be '' in which case it behaves as if num_shards was - set to 1 and only one file will be generated. The default pattern used - is '-SSSSS-of-NNNNN' if None is passed as the shard_name_template. + the shard number and shard count. Currently only ``''``, + ``'-SSSSS-of-NNNNN'``, ``'-W-SSSSS-of-NNNNN'`` and + ``'-V-SSSSS-of-NNNNN'`` are patterns accepted by the service. + When constructing a filename for a particular shard number, the + upper-case letters ``S`` and ``N`` are replaced with the ``0``-padded + shard number and shard count respectively. This argument can be ``''`` + in which case it behaves as if num_shards was set to 1 and only one file + will be generated. The default pattern used is ``'-SSSSS-of-NNNNN'`` for + bounded PCollections and for ``'-W-SSSSS-of-NNNNN'`` unbounded + PCollections. + W is used for windowed shard naming and is replaced with + ``[window.start, window.end)`` + V is used for windowed shard naming and is replaced with + ``[window.start.to_utc_datetime().strftime("%Y-%m-%dT%H-%M-%S"), + window.end.to_utc_datetime().strftime("%Y-%m-%dT%H-%M-%S")`` mime_type: The MIME type to use for the produced files, if the filesystem supports specifying MIME types. use_fastavro (bool): This flag is left for API backwards compatibility and no longer has an effect. Do not use. + triggering_frequency: (int) Every triggering_frequency duration, a window + will be triggered and all bundles in the window will be written. + If set it overrides user windowing. Mandatory for GlobalWindow. Returns: A WriteToAvro transform usable for writing. @@ -411,7 +424,7 @@ def __init__( self._schema = schema self._sink_provider = lambda avro_schema: _create_avro_sink( file_path_prefix, avro_schema, codec, file_name_suffix, num_shards, - shard_name_template, mime_type) + shard_name_template, mime_type, triggering_frequency) def expand(self, pcoll): if self._schema: @@ -428,6 +441,15 @@ def expand(self, pcoll): records = pcoll | beam.Map( beam_row_to_avro_dict(avro_schema, beam_schema)) self._sink = self._sink_provider(avro_schema) + if (not pcoll.is_bounded and self._sink.shard_name_template + == filebasedsink.DEFAULT_SHARD_NAME_TEMPLATE): + self._sink.shard_name_template = ( + filebasedsink.DEFAULT_WINDOW_SHARD_NAME_TEMPLATE) + self._sink.shard_name_format = self._sink._template_to_format( + self._sink.shard_name_template) + self._sink.shard_name_glob_format = self._sink._template_to_glob_format( + self._sink.shard_name_template) + return records | beam.io.iobase.Write(self._sink) def display_data(self): @@ -441,7 +463,8 @@ def _create_avro_sink( file_name_suffix, num_shards, shard_name_template, - mime_type): + mime_type, + triggering_frequency=60): if "class 'avro.schema" in str(type(schema)): raise ValueError( 'You are using Avro IO with fastavro (default with Beam on ' @@ -454,7 +477,8 @@ def _create_avro_sink( file_name_suffix, num_shards, shard_name_template, - mime_type) + mime_type, + triggering_frequency) class _BaseAvroSink(filebasedsink.FileBasedSink): @@ -467,7 +491,8 @@ def __init__( file_name_suffix, num_shards, shard_name_template, - mime_type): + mime_type, + triggering_frequency): super().__init__( file_path_prefix, file_name_suffix=file_name_suffix, @@ -477,7 +502,8 @@ def __init__( mime_type=mime_type, # Compression happens at the block level using the supplied codec, and # not at the file level. - compression_type=CompressionTypes.UNCOMPRESSED) + compression_type=CompressionTypes.UNCOMPRESSED, + triggering_frequency=triggering_frequency) self._schema = schema self._codec = codec @@ -498,7 +524,8 @@ def __init__( file_name_suffix, num_shards, shard_name_template, - mime_type): + mime_type, + triggering_frequency): super().__init__( file_path_prefix, schema, @@ -506,7 +533,8 @@ def __init__( file_name_suffix, num_shards, shard_name_template, - mime_type) + mime_type, + triggering_frequency) self.file_handle = None def open(self, temp_path): diff --git a/sdks/python/apache_beam/io/avroio_test.py b/sdks/python/apache_beam/io/avroio_test.py index 6dd9e620c665..6669b6fb8abf 100644 --- a/sdks/python/apache_beam/io/avroio_test.py +++ b/sdks/python/apache_beam/io/avroio_test.py @@ -16,11 +16,15 @@ # # pytype: skip-file +import glob import json import logging import math import os +import pytz import pytest +import re +import shutil import tempfile import unittest from typing import List, Any @@ -47,14 +51,17 @@ from apache_beam.io.filesystems import FileSystems from apache_beam.options.pipeline_options import StandardOptions from apache_beam.testing.test_pipeline import TestPipeline +from apache_beam.testing.test_stream import TestStream from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to from apache_beam.transforms.display import DisplayData from apache_beam.transforms.display_test import DisplayDataItemMatcher from apache_beam.transforms.sql import SqlTransform from apache_beam.transforms.userstate import CombiningValueStateSpec +from apache_beam.transforms.util import LogElements from apache_beam.utils.timestamp import Timestamp from apache_beam.typehints import schemas +from datetime import datetime # Import snappy optionally; some tests will be skipped when import fails. try: @@ -673,6 +680,273 @@ def _write_data( return f.name +class GenerateEvent(beam.PTransform): + @staticmethod + def sample_data(): + return GenerateEvent() + + def expand(self, input): + elemlist = [{'age': 10}, {'age': 20}, {'age': 30}] + elem = elemlist + return ( + input + | TestStream().add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 1, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 2, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 3, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 4, 0, + tzinfo=pytz.UTC).timestamp()). + advance_watermark_to( + datetime(2021, 3, 1, 0, 0, 5, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 5, 0, + tzinfo=pytz.UTC).timestamp()). + add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 6, + 0, tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 7, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 8, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 9, 0, + tzinfo=pytz.UTC).timestamp()). + advance_watermark_to( + datetime(2021, 3, 1, 0, 0, 10, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 10, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 11, 0, + tzinfo=pytz.UTC).timestamp()). + add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 12, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 13, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 14, 0, + tzinfo=pytz.UTC).timestamp()). + advance_watermark_to( + datetime(2021, 3, 1, 0, 0, 15, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 15, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 16, 0, + tzinfo=pytz.UTC).timestamp()). + add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 17, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 18, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 19, 0, + tzinfo=pytz.UTC).timestamp()). + advance_watermark_to( + datetime(2021, 3, 1, 0, 0, 20, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 20, 0, + tzinfo=pytz.UTC).timestamp()).advance_watermark_to( + datetime( + 2021, 3, 1, 0, 0, 25, 0, tzinfo=pytz.UTC). + timestamp()).advance_watermark_to_infinity()) + + +class WriteStreamingTest(unittest.TestCase): + def setUp(self): + super().setUp() + self.tempdir = tempfile.mkdtemp() + + def tearDown(self): + if os.path.exists(self.tempdir): + shutil.rmtree(self.tempdir) + + def test_write_streaming_2_shards_default_shard_name_template( + self, num_shards=2): + with TestPipeline() as p: + output = ( + p + | GenerateEvent.sample_data() + | 'User windowing' >> beam.transforms.core.WindowInto( + beam.transforms.window.FixedWindows(60), + trigger=beam.transforms.trigger.AfterWatermark(), + accumulation_mode=beam.transforms.trigger.AccumulationMode. + DISCARDING, + allowed_lateness=beam.utils.timestamp.Duration(seconds=0))) + #AvroIO + avroschema = { + 'name': 'dummy', # your supposed to be file name with .avro extension + 'type': 'record', # type of avro serilazation, there are more (see + # above docs) + 'fields': [ # this defines actual keys & their types + {'name': 'age', 'type': 'int'}, + ], + } + output2 = output | 'WriteToAvro' >> beam.io.WriteToAvro( + file_path_prefix=self.tempdir + "/ouput_WriteToAvro", + file_name_suffix=".avro", + num_shards=num_shards, + schema=avroschema) + _ = output2 | 'LogElements after WriteToAvro' >> LogElements( + prefix='after WriteToAvro ', with_window=True, level=logging.INFO) + + # Regex to match the expected windowed file pattern + # Example: + # ouput_WriteToAvro-[1614556800.0, 1614556805.0)-00000-of-00002.avro + # It captures: window_interval, shard_num, total_shards + pattern_string = ( + r'.*-\[(?P[\d\.]+), ' + r'(?P[\d\.]+|Infinity)\)-' + r'(?P\d{5})-of-(?P\d{5})\.avro$') + pattern = re.compile(pattern_string) + file_names = [] + for file_name in glob.glob(self.tempdir + '/ouput_WriteToAvro*'): + match = pattern.match(file_name) + self.assertIsNotNone( + match, f"File name {file_name} did not match expected pattern.") + if match: + file_names.append(file_name) + print("Found files matching expected pattern:", file_names) + self.assertEqual( + len(file_names), + num_shards, + "expected %d files, but got: %d" % (num_shards, len(file_names))) + + def test_write_streaming_2_shards_custom_shard_name_template( + self, num_shards=2, shard_name_template='-V-SSSSS-of-NNNNN'): + with TestPipeline() as p: + output = (p | GenerateEvent.sample_data()) + #AvroIO + avroschema = { + 'name': 'dummy', # your supposed to be file name with .avro extension + 'type': 'record', # type of avro serilazation + 'fields': [ # this defines actual keys & their types + {'name': 'age', 'type': 'int'}, + ], + } + output2 = output | 'WriteToAvro' >> beam.io.WriteToAvro( + file_path_prefix=self.tempdir + "/ouput_WriteToAvro", + file_name_suffix=".avro", + shard_name_template=shard_name_template, + num_shards=num_shards, + triggering_frequency=60, + schema=avroschema) + _ = output2 | 'LogElements after WriteToAvro' >> LogElements( + prefix='after WriteToAvro ', with_window=True, level=logging.INFO) + + # Regex to match the expected windowed file pattern + # Example: + # ouput_WriteToAvro-[2021-03-01T00-00-00, 2021-03-01T00-01-00)- + # 00000-of-00002.avro + # It captures: window_interval, shard_num, total_shards + pattern_string = ( + r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}-\d{2}-\d{2}), ' + r'(?P\d{4}-\d{2}-\d{2}T\d{2}-\d{2}-\d{2}|Infinity)\)-' + r'(?P\d{5})-of-(?P\d{5})\.avro$') + pattern = re.compile(pattern_string) + file_names = [] + for file_name in glob.glob(self.tempdir + '/ouput_WriteToAvro*'): + match = pattern.match(file_name) + self.assertIsNotNone( + match, f"File name {file_name} did not match expected pattern.") + if match: + file_names.append(file_name) + print("Found files matching expected pattern:", file_names) + self.assertEqual( + len(file_names), + num_shards, + "expected %d files, but got: %d" % (num_shards, len(file_names))) + + def test_write_streaming_2_shards_custom_shard_name_template_5s_window( + self, + num_shards=2, + shard_name_template='-V-SSSSS-of-NNNNN', + triggering_frequency=5): + with TestPipeline() as p: + output = (p | GenerateEvent.sample_data()) + #AvroIO + avroschema = { + 'name': 'dummy', # your supposed to be file name with .avro extension + 'type': 'record', # type of avro serilazation + 'fields': [ # this defines actual keys & their types + {'name': 'age', 'type': 'int'}, + ], + } + output2 = output | 'WriteToAvro' >> beam.io.WriteToAvro( + file_path_prefix=self.tempdir + "/ouput_WriteToAvro", + file_name_suffix=".txt", + shard_name_template=shard_name_template, + num_shards=num_shards, + triggering_frequency=triggering_frequency, + schema=avroschema) + _ = output2 | 'LogElements after WriteToAvro' >> LogElements( + prefix='after WriteToAvro ', with_window=True, level=logging.INFO) + + # Regex to match the expected windowed file pattern + # Example: + # ouput_WriteToAvro-[2021-03-01T00-00-00, 2021-03-01T00-01-00)- + # 00000-of-00002.avro + # It captures: window_interval, shard_num, total_shards + pattern_string = ( + r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}-\d{2}-\d{2}), ' + r'(?P\d{4}-\d{2}-\d{2}T\d{2}-\d{2}-\d{2}|Infinity)\)-' + r'(?P\d{5})-of-(?P\d{5})\.txt$') + pattern = re.compile(pattern_string) + file_names = [] + for file_name in glob.glob(self.tempdir + '/ouput_WriteToAvro*'): + match = pattern.match(file_name) + self.assertIsNotNone( + match, f"File name {file_name} did not match expected pattern.") + if match: + file_names.append(file_name) + print("Found files matching expected pattern:", file_names) + # for 5s window size, the input should be processed by 5 windows with + # 2 shards per window + self.assertEqual( + len(file_names), + 10, + "expected %d files, but got: %d" % (num_shards, len(file_names))) + + if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) unittest.main() diff --git a/sdks/python/apache_beam/io/components/adaptive_throttler.py b/sdks/python/apache_beam/io/components/adaptive_throttler.py index f62906360739..3c22891ee8a3 100644 --- a/sdks/python/apache_beam/io/components/adaptive_throttler.py +++ b/sdks/python/apache_beam/io/components/adaptive_throttler.py @@ -21,9 +21,32 @@ # pytype: skip-file +import logging import random +import time from apache_beam.io.components import util +from apache_beam.metrics.metric import Metrics + +_SECONDS_TO_MILLISECONDS = 1_000 + + +class ThrottlingSignaler(object): + """A class that handles signaling throttling of remote requests to the + SDK harness. + """ + def __init__(self, namespace: str = ""): + self.throttling_metric = Metrics.counter( + namespace, "cumulativeThrottlingSeconds") + + def signal_throttled(self, seconds: int): + """Signals to the runner that requests have been throttled for some amount + of time. + + Args: + seconds: int, duration of throttling in seconds. + """ + self.throttling_metric.inc(seconds) class AdaptiveThrottler(object): @@ -94,3 +117,72 @@ def successful_request(self, now): now: int, time in ms since the epoch """ self._successful_requests.add(now, 1) + + +class ReactiveThrottler(AdaptiveThrottler): + """ A wrapper around the AdaptiveThrottler that also handles logging and + signaling throttling to the SDK harness using the provided namespace. + + For usage, instantiate one instance of a ReactiveThrottler class for a + PTransform. When making remote calls to a service, preface that call with + the throttle() method to potentially pre-emptively throttle the request. + This will throttle future calls based on the failure rate of preceding calls, + with higher failure rates leading to longer periods of throttling to allow + system recovery. capture the timestamp of the attempted request, then execute + the request code. On a success, call successful_request(timestamp) to report + the success to the throttler. This flow looks like the following: + + def remote_call(): + throttler.throttle() + + try: + timestamp = time.time() + result = make_request() + throttler.successful_request(timestamp) + return result + except Exception as e: + # do any error handling you want to do + raise + """ + def __init__( + self, + window_ms: int, + bucket_ms: int, + overload_ratio: float, + namespace: str = '', + throttle_delay_secs: int = 5): + """Initializes the ReactiveThrottler. + + Args: + window_ms: int, length of history to consider, in ms, to set + throttling. + bucket_ms: int, granularity of time buckets that we store data in, in + ms. + overload_ratio: float, the target ratio between requests sent and + successful requests. This is "K" in the formula in + https://landing.google.com/sre/book/chapters/handling-overload.html. + namespace: str, the namespace to use for logging and signaling + throttling is occurring + throttle_delay_secs: int, the amount of time in seconds to wait + after preemptively throttled requests + """ + self.throttling_signaler = ThrottlingSignaler(namespace=namespace) + self.logger = logging.getLogger(namespace) + self.throttle_delay_secs = throttle_delay_secs + super().__init__( + window_ms=window_ms, bucket_ms=bucket_ms, overload_ratio=overload_ratio) + + def throttle(self): + """ Stops request code from advancing while the underlying + AdaptiveThrottler is signaling to preemptively throttle the request. + Automatically handles logging the throttling and signaling to the SDK + harness that the request is being throttled. This should be called in any + context where a call to a remote service is being contacted prior to the + call being performed. + """ + while self.throttle_request(time.time() * _SECONDS_TO_MILLISECONDS): + self.logger.info( + "Delaying request for %d seconds due to previous failures", + self.throttle_delay_secs) + time.sleep(self.throttle_delay_secs) + self.throttling_signaler.signal_throttled(self.throttle_delay_secs) diff --git a/sdks/python/apache_beam/io/gcp/bigquery.py b/sdks/python/apache_beam/io/gcp/bigquery.py index 4780f948be23..aa0ebc12ef18 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery.py +++ b/sdks/python/apache_beam/io/gcp/bigquery.py @@ -850,7 +850,8 @@ def _setup_temporary_dataset(self, bq): return location = bq.get_query_location( self._get_project(), self.query.get(), self.use_legacy_sql) - bq.create_temporary_dataset(self._get_project(), location) + bq.create_temporary_dataset( + self._get_project(), location, kms_key=self.kms_key) @check_accessible(['query']) def _execute_query(self, bq): @@ -1062,7 +1063,10 @@ def _setup_temporary_dataset(self, bq): self._get_parent_project(), self.query.get(), self.use_legacy_sql) _LOGGER.warning("### Labels: %s", str(self.bigquery_dataset_labels)) bq.create_temporary_dataset( - self._get_parent_project(), location, self.bigquery_dataset_labels) + self._get_parent_project(), + location, + self.bigquery_dataset_labels, + kms_key=self.kms_key) @check_accessible(['query']) def _execute_query(self, bq): diff --git a/sdks/python/apache_beam/io/gcp/bigquery_file_loads_test.py b/sdks/python/apache_beam/io/gcp/bigquery_file_loads_test.py index f4448578fb6c..c318b1988536 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery_file_loads_test.py +++ b/sdks/python/apache_beam/io/gcp/bigquery_file_loads_test.py @@ -63,6 +63,11 @@ except ImportError: raise unittest.SkipTest('GCP dependencies are not installed') +try: + import dill +except ImportError: + dill = None + _LOGGER = logging.getLogger(__name__) _DESTINATION_ELEMENT_PAIRS = [ @@ -406,6 +411,13 @@ def test_partition_files_dofn_size_split(self): label='CheckSinglePartition') +def maybe_skip(compat_version): + if compat_version and not dill: + raise unittest.SkipTest( + 'Dill dependency not installed which is required for compat_version' + ' <= 2.67.0') + + class TestBigQueryFileLoads(_TestCaseWithTempDirCleanUp): def test_trigger_load_jobs_with_empty_files(self): destination = "project:dataset.table" @@ -485,7 +497,9 @@ def test_records_traverse_transform_with_mocks(self): param(compat_version=None), param(compat_version="2.64.0"), ]) + @pytest.mark.uses_dill def test_reshuffle_before_load(self, compat_version): + maybe_skip(compat_version) destination = 'project1:dataset1.table1' job_reference = bigquery_api.JobReference() @@ -884,7 +898,9 @@ def dynamic_destination_resolver(element, *side_inputs): # For now we don't care about the return value. mock_insert_copy_job.return_value = None - with TestPipeline('DirectRunner') as p: + # Pin to FnApiRunner for now to make mocks act appropriately. + # TODO(https://github.com/apache/beam/issues/34549) + with TestPipeline('FnApiRunner') as p: _ = ( p | beam.Create([ @@ -992,6 +1008,7 @@ def dynamic_destination_resolver(element, *side_inputs): ]) def test_triggering_frequency( self, is_streaming, with_auto_sharding, compat_version): + maybe_skip(compat_version) destination = 'project1:dataset1.table1' job_reference = bigquery_api.JobReference() diff --git a/sdks/python/apache_beam/io/gcp/bigquery_read_internal.py b/sdks/python/apache_beam/io/gcp/bigquery_read_internal.py index e23571f257df..6432f3b4eeac 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery_read_internal.py +++ b/sdks/python/apache_beam/io/gcp/bigquery_read_internal.py @@ -239,6 +239,17 @@ def _get_temp_dataset_id(self): else: raise ValueError("temp_dataset has to be either str or DatasetReference") + def _get_temp_dataset_project(self): + """Returns the project ID for temporary dataset operations. + + If temp_dataset is a DatasetReference, returns its projectId. + Otherwise, returns the pipeline project for billing. + """ + if isinstance(self.temp_dataset, DatasetReference): + return self.temp_dataset.projectId + else: + return self._get_project() + def start_bundle(self): self.bq = bigquery_tools.BigQueryWrapper( temp_dataset_id=self._get_temp_dataset_id(), @@ -278,7 +289,9 @@ def process(self, def finish_bundle(self): if self.bq.created_temp_dataset: - self.bq.clean_up_temporary_dataset(self._get_project()) + # Use the same project that was used to create the temp dataset + temp_dataset_project = self._get_temp_dataset_project() + self.bq.clean_up_temporary_dataset(temp_dataset_project) def _get_bq_metadata(self): if not self.bq_io_metadata: @@ -303,7 +316,11 @@ def _setup_temporary_dataset( element: 'ReadFromBigQueryRequest'): location = bq.get_query_location( self._get_project(), element.query, not element.use_standard_sql) - bq.create_temporary_dataset(self._get_project(), location) + # Use the project from temp_dataset if it's a DatasetReference, + # otherwise use the pipeline project + temp_dataset_project = self._get_temp_dataset_project() + bq.create_temporary_dataset( + temp_dataset_project, location, kms_key=self.kms_key) def _execute_query( self, diff --git a/sdks/python/apache_beam/io/gcp/bigquery_read_internal_test.py b/sdks/python/apache_beam/io/gcp/bigquery_read_internal_test.py new file mode 100644 index 000000000000..46673b4ec2d2 --- /dev/null +++ b/sdks/python/apache_beam/io/gcp/bigquery_read_internal_test.py @@ -0,0 +1,170 @@ +# +# 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. +# + +"""Unit tests for BigQuery read internal module.""" + +import unittest +from unittest import mock + +from apache_beam.io.gcp import bigquery_read_internal +from apache_beam.options.pipeline_options import GoogleCloudOptions +from apache_beam.options.pipeline_options import PipelineOptions +from apache_beam.options.value_provider import StaticValueProvider + +try: + from apache_beam.io.gcp.internal.clients.bigquery import DatasetReference +except ImportError: + DatasetReference = None + + +class BigQueryReadSplitTest(unittest.TestCase): + """Tests for _BigQueryReadSplit DoFn.""" + def setUp(self): + if DatasetReference is None: + self.skipTest('BigQuery dependencies are not installed') + self.options = PipelineOptions() + self.gcp_options = self.options.view_as(GoogleCloudOptions) + self.gcp_options.project = 'test-project' + + def test_get_temp_dataset_project_with_string_temp_dataset(self): + """Test _get_temp_dataset_project with string temp_dataset.""" + split = bigquery_read_internal._BigQueryReadSplit( + options=self.options, temp_dataset='temp_dataset_id') + + # Should return the pipeline project when temp_dataset is a string + self.assertEqual(split._get_temp_dataset_project(), 'test-project') + + def test_get_temp_dataset_project_with_dataset_reference(self): + """Test _get_temp_dataset_project with DatasetReference temp_dataset.""" + dataset_ref = DatasetReference( + projectId='custom-project', datasetId='temp_dataset_id') + split = bigquery_read_internal._BigQueryReadSplit( + options=self.options, temp_dataset=dataset_ref) + + # Should return the project from DatasetReference + self.assertEqual(split._get_temp_dataset_project(), 'custom-project') + + def test_get_temp_dataset_project_with_none_temp_dataset(self): + """Test _get_temp_dataset_project with None temp_dataset.""" + split = bigquery_read_internal._BigQueryReadSplit( + options=self.options, temp_dataset=None) + + # Should return the pipeline project when temp_dataset is None + self.assertEqual(split._get_temp_dataset_project(), 'test-project') + + def test_get_temp_dataset_project_with_value_provider_project(self): + """Test _get_temp_dataset_project with ValueProvider project.""" + self.gcp_options.project = StaticValueProvider(str, 'vp-project') + dataset_ref = DatasetReference( + projectId='custom-project', datasetId='temp_dataset_id') + split = bigquery_read_internal._BigQueryReadSplit( + options=self.options, temp_dataset=dataset_ref) + + # Should still return the project from DatasetReference + self.assertEqual(split._get_temp_dataset_project(), 'custom-project') + + @mock.patch('apache_beam.io.gcp.bigquery_tools.BigQueryWrapper') + def test_setup_temporary_dataset_uses_correct_project(self, mock_bq_wrapper): + """Test that _setup_temporary_dataset uses the correct project.""" + dataset_ref = DatasetReference( + projectId='custom-project', datasetId='temp_dataset_id') + split = bigquery_read_internal._BigQueryReadSplit( + options=self.options, temp_dataset=dataset_ref) + + # Mock the BigQueryWrapper instance + mock_bq = mock.Mock() + mock_bq.get_query_location.return_value = 'US' + + # Mock ReadFromBigQueryRequest + mock_element = mock.Mock() + mock_element.query = 'SELECT * FROM table' + mock_element.use_standard_sql = True + + # Call _setup_temporary_dataset + split._setup_temporary_dataset(mock_bq, mock_element) + + # Verify that create_temporary_dataset was called with the custom project + mock_bq.create_temporary_dataset.assert_called_once_with( + 'custom-project', 'US', kms_key=None) + # Verify that get_query_location was called with the pipeline project + mock_bq.get_query_location.assert_called_once_with( + 'test-project', 'SELECT * FROM table', False) + + @mock.patch('apache_beam.io.gcp.bigquery_tools.BigQueryWrapper') + def test_finish_bundle_uses_correct_project(self, mock_bq_wrapper): + """Test that finish_bundle uses the correct project for cleanup.""" + dataset_ref = DatasetReference( + projectId='custom-project', datasetId='temp_dataset_id') + split = bigquery_read_internal._BigQueryReadSplit( + options=self.options, temp_dataset=dataset_ref) + + # Mock the BigQueryWrapper instance + mock_bq = mock.Mock() + mock_bq.created_temp_dataset = True + split.bq = mock_bq + + # Call finish_bundle + split.finish_bundle() + + # Verify that clean_up_temporary_dataset was called with the custom project + mock_bq.clean_up_temporary_dataset.assert_called_once_with('custom-project') + + @mock.patch('apache_beam.io.gcp.bigquery_tools.BigQueryWrapper') + def test_setup_temporary_dataset_with_string_temp_dataset( + self, mock_bq_wrapper): + """Test _setup_temporary_dataset with string temp_dataset uses pipeline + project.""" + split = bigquery_read_internal._BigQueryReadSplit( + options=self.options, temp_dataset='temp_dataset_id') + + # Mock the BigQueryWrapper instance + mock_bq = mock.Mock() + mock_bq.get_query_location.return_value = 'US' + + # Mock ReadFromBigQueryRequest + mock_element = mock.Mock() + mock_element.query = 'SELECT * FROM table' + mock_element.use_standard_sql = True + + # Call _setup_temporary_dataset + split._setup_temporary_dataset(mock_bq, mock_element) + + # Verify that create_temporary_dataset was called with the pipeline project + mock_bq.create_temporary_dataset.assert_called_once_with( + 'test-project', 'US', kms_key=None) + + @mock.patch('apache_beam.io.gcp.bigquery_tools.BigQueryWrapper') + def test_finish_bundle_with_string_temp_dataset(self, mock_bq_wrapper): + """Test finish_bundle with string temp_dataset uses pipeline project.""" + split = bigquery_read_internal._BigQueryReadSplit( + options=self.options, temp_dataset='temp_dataset_id') + + # Mock the BigQueryWrapper instance + mock_bq = mock.Mock() + mock_bq.created_temp_dataset = True + split.bq = mock_bq + + # Call finish_bundle + split.finish_bundle() + + # Verify that clean_up_temporary_dataset was called with the pipeline + # project + mock_bq.clean_up_temporary_dataset.assert_called_once_with('test-project') + + +if __name__ == '__main__': + unittest.main() diff --git a/sdks/python/apache_beam/io/gcp/bigquery_test.py b/sdks/python/apache_beam/io/gcp/bigquery_test.py index b038f38bd5a1..dcb85d60f87f 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery_test.py +++ b/sdks/python/apache_beam/io/gcp/bigquery_test.py @@ -1567,7 +1567,9 @@ def test_insert_rows_json_intermittent_retriable_exception( exception_type(error_message), exception_type(error_message), [] ] - with beam.Pipeline() as p: + # This relies on DirectRunner-specific mocking behavior which can be + # inconsistent on Prism + with beam.Pipeline('FnApiRunner') as p: _ = ( p | beam.Create([{ diff --git a/sdks/python/apache_beam/io/gcp/bigquery_tools.py b/sdks/python/apache_beam/io/gcp/bigquery_tools.py index 081571bfef99..889d3f1e96e3 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery_tools.py +++ b/sdks/python/apache_beam/io/gcp/bigquery_tools.py @@ -333,6 +333,10 @@ def _build_filter_from_labels(labels): return filter_str +def _build_dataset_encryption_config(kms_key): + return bigquery.EncryptionConfiguration(kmsKeyName=kms_key) + + class BigQueryWrapper(object): """BigQuery client wrapper with utilities for querying. @@ -412,6 +416,17 @@ def _get_temp_table(self, project_id): dataset=self.temp_dataset_id, project=project_id) + def _get_temp_table_project(self, fallback_project_id): + """Returns the project ID for temporary table operations. + + If temp_table_ref exists, returns its projectId. + Otherwise, returns the fallback_project_id. + """ + if self.temp_table_ref: + return self.temp_table_ref.projectId + else: + return fallback_project_id + def _get_temp_dataset(self): if self.temp_table_ref: return self.temp_table_ref.datasetId @@ -639,7 +654,8 @@ def _start_query_job( query=query, useLegacySql=use_legacy_sql, allowLargeResults=not dry_run, - destinationTable=self._get_temp_table(project_id) + destinationTable=self._get_temp_table( + self._get_temp_table_project(project_id)) if not dry_run else None, flattenResults=flatten_results, priority=priority, @@ -823,7 +839,7 @@ def _create_table( num_retries=MAX_RETRIES, retry_filter=retry.retry_on_server_errors_and_timeout_filter) def get_or_create_dataset( - self, project_id, dataset_id, location=None, labels=None): + self, project_id, dataset_id, location=None, labels=None, kms_key=None): # Check if dataset already exists otherwise create it try: dataset = self.client.datasets.Get( @@ -846,6 +862,9 @@ def get_or_create_dataset( dataset.location = location if labels is not None: dataset.labels = _build_dataset_labels(labels) + if kms_key is not None: + dataset.defaultEncryptionConfiguration = ( + _build_dataset_encryption_config(kms_key)) request = bigquery.BigqueryDatasetsInsertRequest( projectId=project_id, dataset=dataset) response = self.client.datasets.Insert(request) @@ -917,9 +936,14 @@ def is_user_configured_dataset(self): @retry.with_exponential_backoff( num_retries=MAX_RETRIES, retry_filter=retry.retry_on_server_errors_and_timeout_filter) - def create_temporary_dataset(self, project_id, location, labels=None): + def create_temporary_dataset( + self, project_id, location, labels=None, kms_key=None): self.get_or_create_dataset( - project_id, self.temp_dataset_id, location=location, labels=labels) + project_id, + self.temp_dataset_id, + location=location, + labels=labels, + kms_key=kms_key) if (project_id is not None and not self.is_user_configured_dataset() and not self.created_temp_dataset): diff --git a/sdks/python/apache_beam/io/gcp/bigquery_tools_test.py b/sdks/python/apache_beam/io/gcp/bigquery_tools_test.py index 522c8667f183..1101317439a9 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery_tools_test.py +++ b/sdks/python/apache_beam/io/gcp/bigquery_tools_test.py @@ -301,6 +301,34 @@ def test_get_or_create_dataset_created(self): new_dataset = wrapper.get_or_create_dataset('project-id', 'dataset_id') self.assertEqual(new_dataset.datasetReference.datasetId, 'dataset_id') + def test_create_temporary_dataset_with_kms_key(self): + kms_key = ( + 'projects/my-project/locations/global/keyRings/my-kr/' + 'cryptoKeys/my-key') + client = mock.Mock() + client.datasets.Get.side_effect = HttpError( + response={'status': '404'}, url='', content='') + + client.datasets.Insert.return_value = bigquery.Dataset( + datasetReference=bigquery.DatasetReference( + projectId='project-id', datasetId='temp_dataset')) + wrapper = beam.io.gcp.bigquery_tools.BigQueryWrapper(client) + + try: + wrapper.create_temporary_dataset( + 'project-id', 'location', kms_key=kms_key) + except Exception: + pass + + args, _ = client.datasets.Insert.call_args + insert_request = args[0] # BigqueryDatasetsInsertRequest + inserted_dataset = insert_request.dataset # Actual Dataset object + + # Assertions + self.assertIsNotNone(inserted_dataset.defaultEncryptionConfiguration) + self.assertEqual( + inserted_dataset.defaultEncryptionConfiguration.kmsKeyName, kms_key) + def test_get_or_create_dataset_fetched(self): client = mock.Mock() client.datasets.Get.return_value = bigquery.Dataset( @@ -578,6 +606,27 @@ def test_start_query_job_priority_configuration(self): client.jobs.Insert.call_args[0][0].job.configuration.query.priority, 'INTERACTIVE') + def test_get_temp_table_project_with_temp_table_ref(self): + """Test _get_temp_table_project returns project from temp_table_ref.""" + client = mock.Mock() + temp_table_ref = bigquery.TableReference( + projectId='temp-project', + datasetId='temp_dataset', + tableId='temp_table') + wrapper = beam.io.gcp.bigquery_tools.BigQueryWrapper( + client, temp_table_ref=temp_table_ref) + + result = wrapper._get_temp_table_project('fallback-project') + self.assertEqual(result, 'temp-project') + + def test_get_temp_table_project_without_temp_table_ref(self): + """Test _get_temp_table_project returns fallback when no temp_table_ref.""" + client = mock.Mock() + wrapper = beam.io.gcp.bigquery_tools.BigQueryWrapper(client) + + result = wrapper._get_temp_table_project('fallback-project') + self.assertEqual(result, 'fallback-project') + @unittest.skipIf(HttpError is None, 'GCP dependencies are not installed') class TestRowAsDictJsonCoder(unittest.TestCase): diff --git a/sdks/python/apache_beam/io/gcp/bigtableio.py b/sdks/python/apache_beam/io/gcp/bigtableio.py index b32433df547a..ff140082a1ef 100644 --- a/sdks/python/apache_beam/io/gcp/bigtableio.py +++ b/sdks/python/apache_beam/io/gcp/bigtableio.py @@ -357,7 +357,8 @@ def expand(self, input): rearrange_based_on_discovery=True, table_id=self._table_id, instance_id=self._instance_id, - project_id=self._project_id) + project_id=self._project_id, + flatten=False) return ( input.pipeline diff --git a/sdks/python/apache_beam/io/gcp/experimental/spannerio.py b/sdks/python/apache_beam/io/gcp/experimental/spannerio.py index 7b615e223cfc..cac66bd2ef54 100644 --- a/sdks/python/apache_beam/io/gcp/experimental/spannerio.py +++ b/sdks/python/apache_beam/io/gcp/experimental/spannerio.py @@ -17,7 +17,7 @@ """Google Cloud Spanner IO -Experimental; no backwards-compatibility guarantees. +Deprecated; use apache_beam.io.gcp.spanner module instead. This is an experimental module for reading and writing data from Google Cloud Spanner. Visit: https://cloud.google.com/spanner for more details. @@ -190,6 +190,7 @@ from apache_beam.transforms.display import DisplayDataItem from apache_beam.typehints import with_input_types from apache_beam.typehints import with_output_types +from apache_beam.utils.annotations import deprecated # Protect against environments where spanner library is not available. # pylint: disable=wrong-import-order, wrong-import-position, ungrouped-imports @@ -356,8 +357,8 @@ def _table_metric(self, table_id, status): labels = { **self.base_labels, monitoring_infos.RESOURCE_LABEL: resource, - monitoring_infos.SPANNER_TABLE_ID: table_id } + if table_id: labels[monitoring_infos.SPANNER_TABLE_ID] = table_id service_call_metric = ServiceCallMetric( request_count_urn=monitoring_infos.API_REQUEST_COUNT_URN, base_labels=labels) @@ -612,8 +613,8 @@ def _table_metric(self, table_id): labels = { **self.base_labels, monitoring_infos.RESOURCE_LABEL: resource, - monitoring_infos.SPANNER_TABLE_ID: table_id } + if table_id: labels[monitoring_infos.SPANNER_TABLE_ID] = table_id service_call_metric = ServiceCallMetric( request_count_urn=monitoring_infos.API_REQUEST_COUNT_URN, base_labels=labels) @@ -675,6 +676,7 @@ def teardown(self): self._snapshot.close() +@deprecated(since='2.68', current='apache_beam.io.gcp.spanner.ReadFromSpanner') class ReadFromSpanner(PTransform): """ A PTransform to perform reads from cloud spanner. @@ -825,6 +827,8 @@ def display_data(self): return res +@deprecated( + since='2.68', current='apache_beam.io.gcp.spanner.WriteToSpannerSchema') class WriteToSpanner(PTransform): def __init__( self, @@ -1224,8 +1228,8 @@ def _register_table_metric(self, table_id): labels = { **self.base_labels, monitoring_infos.RESOURCE_LABEL: resource, - monitoring_infos.SPANNER_TABLE_ID: table_id } + if table_id: labels[monitoring_infos.SPANNER_TABLE_ID] = table_id service_call_metric = ServiceCallMetric( request_count_urn=monitoring_infos.API_REQUEST_COUNT_URN, base_labels=labels) diff --git a/sdks/python/apache_beam/io/gcp/pubsub.py b/sdks/python/apache_beam/io/gcp/pubsub.py index 9e006dbeda93..281827db034b 100644 --- a/sdks/python/apache_beam/io/gcp/pubsub.py +++ b/sdks/python/apache_beam/io/gcp/pubsub.py @@ -17,8 +17,9 @@ """Google Cloud PubSub sources and sinks. -Cloud Pub/Sub sources and sinks are currently supported only in streaming -pipelines, during remote execution. +Cloud Pub/Sub sources are currently supported only in streaming pipelines, +during remote execution. Cloud Pub/Sub sinks (WriteToPubSub) support both +streaming and batch pipelines. This API is currently under development and is subject to change. @@ -42,7 +43,6 @@ from apache_beam import coders from apache_beam.io import iobase from apache_beam.io.iobase import Read -from apache_beam.io.iobase import Write from apache_beam.metrics.metric import Lineage from apache_beam.transforms import DoFn from apache_beam.transforms import Flatten @@ -376,7 +376,12 @@ def report_lineage_once(self): class WriteToPubSub(PTransform): - """A ``PTransform`` for writing messages to Cloud Pub/Sub.""" + """A ``PTransform`` for writing messages to Cloud Pub/Sub. + + This transform supports both streaming and batch pipelines. In streaming mode, + messages are written continuously as they arrive. In batch mode, all messages + are written when the pipeline completes. + """ # Implementation note: This ``PTransform`` is overridden by Directrunner. @@ -435,7 +440,7 @@ def expand(self, pcoll): self.bytes_to_proto_str, self.project, self.topic_name)).with_input_types(Union[bytes, str]) pcoll.element_type = bytes - return pcoll | Write(self._sink) + return pcoll | ParDo(_PubSubWriteDoFn(self)) def to_runner_api_parameter(self, context): # Required as this is identified by type in PTransformOverrides. @@ -541,11 +546,75 @@ def is_bounded(self): return False -# TODO(BEAM-27443): Remove in favor of a proper WriteToPubSub transform. +class _PubSubWriteDoFn(DoFn): + """DoFn for writing messages to Cloud Pub/Sub. + + This DoFn handles both streaming and batch modes by buffering messages + and publishing them in batches to optimize performance. + """ + BUFFER_SIZE_ELEMENTS = 100 + FLUSH_TIMEOUT_SECS = 5 * 60 # 5 minutes + + def __init__(self, transform): + self.project = transform.project + self.short_topic_name = transform.topic_name + self.id_label = transform.id_label + self.timestamp_attribute = transform.timestamp_attribute + self.with_attributes = transform.with_attributes + + # TODO(https://github.com/apache/beam/issues/18939): Add support for + # id_label and timestamp_attribute. + if transform.id_label: + raise NotImplementedError('id_label is not supported for PubSub writes') + if transform.timestamp_attribute: + raise NotImplementedError( + 'timestamp_attribute is not supported for PubSub writes') + + def setup(self): + from google.cloud import pubsub + self._pub_client = pubsub.PublisherClient() + self._topic = self._pub_client.topic_path( + self.project, self.short_topic_name) + + def start_bundle(self): + self._buffer = [] + + def process(self, elem): + self._buffer.append(elem) + if len(self._buffer) >= self.BUFFER_SIZE_ELEMENTS: + self._flush() + + def finish_bundle(self): + self._flush() + + def _flush(self): + if not self._buffer: + return + + import time + + # The elements in buffer are already serialized bytes from the previous + # transforms + futures = [ + self._pub_client.publish(self._topic, elem) for elem in self._buffer + ] + + timer_start = time.time() + for future in futures: + remaining = self.FLUSH_TIMEOUT_SECS - (time.time() - timer_start) + if remaining <= 0: + raise TimeoutError( + f"PubSub publish timeout exceeded {self.FLUSH_TIMEOUT_SECS} seconds" + ) + future.result(remaining) + self._buffer = [] + + class _PubSubSink(object): """Sink for a Cloud Pub/Sub topic. - This ``NativeSource`` is overridden by a native Pubsub implementation. + This sink works for both streaming and batch pipelines by using a DoFn + that buffers and batches messages for efficient publishing. """ def __init__( self, diff --git a/sdks/python/apache_beam/io/gcp/pubsub_integration_test.py b/sdks/python/apache_beam/io/gcp/pubsub_integration_test.py index 28c30df1d559..c88f4af2016d 100644 --- a/sdks/python/apache_beam/io/gcp/pubsub_integration_test.py +++ b/sdks/python/apache_beam/io/gcp/pubsub_integration_test.py @@ -30,6 +30,7 @@ from apache_beam.io.gcp import pubsub_it_pipeline from apache_beam.io.gcp.pubsub import PubsubMessage +from apache_beam.io.gcp.pubsub import WriteToPubSub from apache_beam.io.gcp.tests.pubsub_matcher import PubSubMessageMatcher from apache_beam.runners.runner import PipelineState from apache_beam.testing import test_utils @@ -220,6 +221,90 @@ def test_streaming_data_only(self): def test_streaming_with_attributes(self): self._test_streaming(with_attributes=True) + def _test_batch_write(self, with_attributes): + """Tests batch mode WriteToPubSub functionality. + + Args: + with_attributes: False - Writes message data only. + True - Writes message data and attributes. + """ + from apache_beam.options.pipeline_options import PipelineOptions + from apache_beam.options.pipeline_options import StandardOptions + from apache_beam.transforms import Create + + # Create test messages for batch mode + test_messages = [ + PubsubMessage(b'batch_data001', {'batch_attr': 'value1'}), + PubsubMessage(b'batch_data002', {'batch_attr': 'value2'}), + PubsubMessage(b'batch_data003', {'batch_attr': 'value3'}) + ] + + pipeline_options = PipelineOptions() + # Explicitly set streaming to False for batch mode + pipeline_options.view_as(StandardOptions).streaming = False + + with TestPipeline(options=pipeline_options) as p: + if with_attributes: + messages = p | 'CreateMessages' >> Create(test_messages) + _ = messages | 'WriteToPubSub' >> WriteToPubSub( + self.output_topic.name, with_attributes=True) + else: + # For data-only mode, extract just the data + message_data = [msg.data for msg in test_messages] + messages = p | 'CreateData' >> Create(message_data) + _ = messages | 'WriteToPubSub' >> WriteToPubSub( + self.output_topic.name, with_attributes=False) + + # Verify messages were published by reading from the subscription + time.sleep(10) # Allow time for messages to be published and received + + # Pull messages from the output subscription to verify they were written + response = self.sub_client.pull( + request={ + "subscription": self.output_sub.name, + "max_messages": 10, + }) + + received_messages = [] + for received_message in response.received_messages: + if with_attributes: + # Parse attributes + attrs = dict(received_message.message.attributes) + received_messages.append( + PubsubMessage(received_message.message.data, attrs)) + else: + received_messages.append(received_message.message.data) + + # Acknowledge the message + self.sub_client.acknowledge( + request={ + "subscription": self.output_sub.name, + "ack_ids": [received_message.ack_id], + }) + + # Verify we received the expected number of messages + self.assertEqual(len(received_messages), len(test_messages)) + + if with_attributes: + # Verify message content and attributes + received_data = [msg.data for msg in received_messages] + expected_data = [msg.data for msg in test_messages] + self.assertEqual(sorted(received_data), sorted(expected_data)) + else: + # Verify message data only + expected_data = [msg.data for msg in test_messages] + self.assertEqual(sorted(received_messages), sorted(expected_data)) + + @pytest.mark.it_postcommit + def test_batch_write_data_only(self): + """Test WriteToPubSub in batch mode with data only.""" + self._test_batch_write(with_attributes=False) + + @pytest.mark.it_postcommit + def test_batch_write_with_attributes(self): + """Test WriteToPubSub in batch mode with attributes.""" + self._test_batch_write(with_attributes=True) + if __name__ == '__main__': logging.getLogger().setLevel(logging.DEBUG) diff --git a/sdks/python/apache_beam/io/gcp/pubsub_test.py b/sdks/python/apache_beam/io/gcp/pubsub_test.py index e3fb07a17625..5650e920e635 100644 --- a/sdks/python/apache_beam/io/gcp/pubsub_test.py +++ b/sdks/python/apache_beam/io/gcp/pubsub_test.py @@ -867,12 +867,14 @@ def test_write_messages_success(self, mock_pubsub): | Create(payloads) | WriteToPubSub( 'projects/fakeprj/topics/a_topic', with_attributes=False)) - mock_pubsub.return_value.publish.assert_has_calls( - [mock.call(mock.ANY, data)]) + # Verify that publish was called (data will be protobuf serialized) + mock_pubsub.return_value.publish.assert_called() + # Check that the call was made with the topic and some data + call_args = mock_pubsub.return_value.publish.call_args + self.assertEqual(len(call_args[0]), 2) # topic and data def test_write_messages_deprecated(self, mock_pubsub): data = 'data' - data_bytes = b'data' payloads = [data] options = PipelineOptions([]) @@ -882,8 +884,11 @@ def test_write_messages_deprecated(self, mock_pubsub): p | Create(payloads) | WriteStringsToPubSub('projects/fakeprj/topics/a_topic')) - mock_pubsub.return_value.publish.assert_has_calls( - [mock.call(mock.ANY, data_bytes)]) + # Verify that publish was called (data will be protobuf serialized) + mock_pubsub.return_value.publish.assert_called() + # Check that the call was made with the topic and some data + call_args = mock_pubsub.return_value.publish.call_args + self.assertEqual(len(call_args[0]), 2) # topic and data def test_write_messages_with_attributes_success(self, mock_pubsub): data = b'data' @@ -898,8 +903,54 @@ def test_write_messages_with_attributes_success(self, mock_pubsub): | Create(payloads) | WriteToPubSub( 'projects/fakeprj/topics/a_topic', with_attributes=True)) - mock_pubsub.return_value.publish.assert_has_calls( - [mock.call(mock.ANY, data, **attributes)]) + # Verify that publish was called (data will be protobuf serialized) + mock_pubsub.return_value.publish.assert_called() + # Check that the call was made with the topic and some data + call_args = mock_pubsub.return_value.publish.call_args + self.assertEqual(len(call_args[0]), 2) # topic and data + + def test_write_messages_batch_mode_success(self, mock_pubsub): + """Test WriteToPubSub works in batch mode (non-streaming).""" + data = 'data' + payloads = [data] + + options = PipelineOptions([]) + # Explicitly set streaming to False for batch mode + options.view_as(StandardOptions).streaming = False + with TestPipeline(options=options) as p: + _ = ( + p + | Create(payloads) + | WriteToPubSub( + 'projects/fakeprj/topics/a_topic', with_attributes=False)) + + # Verify that publish was called (data will be protobuf serialized) + mock_pubsub.return_value.publish.assert_called() + # Check that the call was made with the topic and some data + call_args = mock_pubsub.return_value.publish.call_args + self.assertEqual(len(call_args[0]), 2) # topic and data + + def test_write_messages_with_attributes_batch_mode_success(self, mock_pubsub): + """Test WriteToPubSub with attributes works in batch mode.""" + data = b'data' + attributes = {'key': 'value'} + payloads = [PubsubMessage(data, attributes)] + + options = PipelineOptions([]) + # Explicitly set streaming to False for batch mode + options.view_as(StandardOptions).streaming = False + with TestPipeline(options=options) as p: + _ = ( + p + | Create(payloads) + | WriteToPubSub( + 'projects/fakeprj/topics/a_topic', with_attributes=True)) + + # Verify that publish was called (data will be protobuf serialized) + mock_pubsub.return_value.publish.assert_called() + # Check that the call was made with the topic and some data + call_args = mock_pubsub.return_value.publish.call_args + self.assertEqual(len(call_args[0]), 2) # topic and data def test_write_messages_with_attributes_error(self, mock_pubsub): data = 'data' diff --git a/sdks/python/apache_beam/io/gcp/spanner.py b/sdks/python/apache_beam/io/gcp/spanner.py index 9089d746fe1c..03ad91069b99 100644 --- a/sdks/python/apache_beam/io/gcp/spanner.py +++ b/sdks/python/apache_beam/io/gcp/spanner.py @@ -145,6 +145,20 @@ class ReadFromSpannerSchema(NamedTuple): time_unit: Optional[str] +class ReadChangeStreamFromSpannerSchema(NamedTuple): + instance_id: str + database_id: str + project_id: str + changeStreamName: str + inclusiveStartAt: str + inclusiveEndAt: Optional[str] + metadataDatabase: str + metadataInstance: str + metadataTable: Optional[str] + rpcPriority: Optional[str] + watermarkRefreshRate: Optional[str] + + class ReadFromSpanner(ExternalTransform): """ A PTransform which reads from the specified Spanner instance's database. @@ -659,5 +673,94 @@ def __init__( ) +class ReadChangeStreamFromSpanner(ExternalTransform): + """ + A PTransform to read Change Streams from Google Cloud Spanner. + + The output of this transform is a PCollection of JSON strings, + where each string represents a com.google.cloud.spanner.DataChangeRecord. + + Example: + + with beam.Pipeline(options=pipeline_options) as p: + p | + "ReadFromSpannerChangeStream" >> beam_spanner.ReadChangeStreamFromSpanner( + project_id="spanner-project-id", + instance_id="spanner-instance-id", + database_id="spanner-database-id", + changeStreamName="spanner-change-stream", + inclusiveStartAt="2025-05-20T10:00:00Z", + metadataDatabase="spanner-metadata-database", + metadataInstance="spanner-metadata-instance") + + Experimental; no backwards compatibility guarantees. + """ + + URN = 'beam:transform:org.apache.beam:spanner_change_stream_reader:v1' + + def __init__( + self, + project_id, + instance_id, + database_id, + changeStreamName, + metadataDatabase, + metadataInstance, + inclusiveStartAt, + inclusiveEndAt=None, + metadataTable=None, + rpcPriority=None, + watermarkRefreshRate=None, + expansion_service=None, + ): + """ + Reads Change Streams from Google Cloud Spanner. + + :param project_id: (Required) Specifies the Cloud Spanner project. + :param instance_id: (Required) Specifies the Cloud Spanner + instance. + :param database_id: (Required) Specifies the Cloud Spanner + database. + :param changeStreamName: (Required) The name of the Spanner + change stream to read. + :param metadataDatabase: (Required) The database where the + change stream metadata is stored. + :param metadataInstance: (Required) The instance where the + change stream metadata database resides. + :param inclusiveStartAt: (Required) An inclusive start timestamp + for reading the change stream. + :param inclusiveEndAt: (Optional) An inclusive end timestamp for + reading the change stream. If not specified, the stream will be + read indefinitely. + :param metadataTable: (Optional) The name of the metadata table used + by the change stream connector. If not specified, a default table + name will be used. + :param rpcPriority: (Optional) The RPC priority for Spanner operations. + Can be 'HIGH', 'MEDIUM', or 'LOW'. + :param watermarkRefreshRate: (Optional) The duration at which the + watermark is refreshed. + """ + + super().__init__( + self.URN, + NamedTupleBasedPayloadBuilder( + ReadChangeStreamFromSpannerSchema( + instance_id=instance_id, + database_id=database_id, + project_id=project_id, + changeStreamName=changeStreamName, + inclusiveStartAt=inclusiveStartAt, + inclusiveEndAt=inclusiveEndAt, + metadataDatabase=metadataDatabase, + metadataInstance=metadataInstance, + metadataTable=metadataTable, + rpcPriority=rpcPriority, + watermarkRefreshRate=watermarkRefreshRate, + ), + ), + expansion_service=expansion_service or default_io_expansion_service(), + ) + + def _get_enum_name(enum): return None if enum is None else enum.name diff --git a/sdks/python/apache_beam/io/iobase_it_test.py b/sdks/python/apache_beam/io/iobase_it_test.py new file mode 100644 index 000000000000..acb44f4085bc --- /dev/null +++ b/sdks/python/apache_beam/io/iobase_it_test.py @@ -0,0 +1,72 @@ +# +# 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. +# + +# pytype: skip-file + +import logging +import unittest +import uuid + +import apache_beam as beam +from apache_beam.io.textio import WriteToText +from apache_beam.testing.test_pipeline import TestPipeline +from apache_beam.transforms.window import FixedWindows + +# End-to-End tests for iobase +# Usage: +# cd sdks/python +# pip install build && python -m build --sdist +# DataflowRunner: +# python -m pytest -o log_cli=True -o log_level=Info \ +# apache_beam/io/iobase_it_test.py::IOBaseITTest \ +# --test-pipeline-options="--runner=TestDataflowRunner \ +# --project=apache-beam-testing --region=us-central1 \ +# --temp_location=gs://apache-beam-testing-temp/temp \ +# --sdk_location=dist/apache_beam-2.65.0.dev0.tar.gz" + + +class IOBaseITTest(unittest.TestCase): + def setUp(self): + self.test_pipeline = TestPipeline(is_integration_test=True) + self.runner_name = type(self.test_pipeline.runner).__name__ + + def test_unbounded_pcoll_without_global_window(self): + # https://github.com/apache/beam/issues/25598 + + args = self.test_pipeline.get_full_options_as_args(streaming=True) + + topic = 'projects/pubsub-public-data/topics/taxirides-realtime' + unique_id = str(uuid.uuid4()) + output_file = f'gs://apache-beam-testing-integration-testing/iobase/test-{unique_id}' # pylint: disable=line-too-long + + p = beam.Pipeline(argv=args) + # Read from Pub/Sub with fixed windowing + lines = ( + p + | "ReadFromPubSub" >> beam.io.ReadFromPubSub(topic=topic) + | "WindowInto" >> beam.WindowInto(FixedWindows(10))) + + # Write to text file + _ = lines | 'WriteToText' >> WriteToText(output_file) + + result = p.run() + result.wait_until_finish(duration=60 * 1000) + + +if __name__ == "__main__": + logging.getLogger().setLevel(logging.INFO) + unittest.main() diff --git a/sdks/python/apache_beam/io/jdbc.py b/sdks/python/apache_beam/io/jdbc.py index 79e6b3ce315e..df5d7f21a343 100644 --- a/sdks/python/apache_beam/io/jdbc.py +++ b/sdks/python/apache_beam/io/jdbc.py @@ -87,7 +87,6 @@ # pytype: skip-file import contextlib -import datetime import typing import numpy as np @@ -96,10 +95,11 @@ from apache_beam.transforms.external import BeamJarExpansionService from apache_beam.transforms.external import ExternalTransform from apache_beam.transforms.external import NamedTupleBasedPayloadBuilder +from apache_beam.typehints.schemas import JdbcDateType # pylint: disable=unused-import +from apache_beam.typehints.schemas import JdbcTimeType # pylint: disable=unused-import from apache_beam.typehints.schemas import LogicalType from apache_beam.typehints.schemas import MillisInstant from apache_beam.typehints.schemas import typing_to_runner_api -from apache_beam.utils.timestamp import Timestamp __all__ = [ 'WriteToJdbc', @@ -399,91 +399,3 @@ def __init__( ), expansion_service or default_io_expansion_service(classpath), ) - - -@LogicalType.register_logical_type -class JdbcDateType(LogicalType[datetime.date, MillisInstant, str]): - """ - For internal use only; no backwards-compatibility guarantees. - - Support of Legacy JdbcIO DATE logical type. Deemed to change when Java JDBCIO - has been migrated to Beam portable logical types. - """ - def __init__(self, argument=""): - pass - - @classmethod - def representation_type(cls) -> type: - return MillisInstant - - @classmethod - def urn(cls): - return "beam:logical_type:javasdk_date:v1" - - @classmethod - def language_type(cls): - return datetime.date - - def to_representation_type(self, value: datetime.date) -> Timestamp: - return Timestamp.from_utc_datetime( - datetime.datetime.combine( - value, datetime.datetime.min.time(), tzinfo=datetime.timezone.utc)) - - def to_language_type(self, value: Timestamp) -> datetime.date: - return value.to_utc_datetime().date() - - @classmethod - def argument_type(cls): - return str - - def argument(self): - return "" - - @classmethod - def _from_typing(cls, typ): - return cls() - - -@LogicalType.register_logical_type -class JdbcTimeType(LogicalType[datetime.time, MillisInstant, str]): - """ - For internal use only; no backwards-compatibility guarantees. - - Support of Legacy JdbcIO TIME logical type. . Deemed to change when Java - JDBCIO has been migrated to Beam portable logical types. - """ - def __init__(self, argument=""): - pass - - @classmethod - def representation_type(cls) -> type: - return MillisInstant - - @classmethod - def urn(cls): - return "beam:logical_type:javasdk_time:v1" - - @classmethod - def language_type(cls): - return datetime.time - - def to_representation_type(self, value: datetime.date) -> Timestamp: - return Timestamp.from_utc_datetime( - datetime.datetime.combine( - datetime.datetime.utcfromtimestamp(0), - value, - tzinfo=datetime.timezone.utc)) - - def to_language_type(self, value: Timestamp) -> datetime.date: - return value.to_utc_datetime().time() - - @classmethod - def argument_type(cls): - return str - - def argument(self): - return "" - - @classmethod - def _from_typing(cls, typ): - return cls() diff --git a/sdks/python/apache_beam/io/parquetio.py b/sdks/python/apache_beam/io/parquetio.py index 48c51428c17d..82ae9a50ace4 100644 --- a/sdks/python/apache_beam/io/parquetio.py +++ b/sdks/python/apache_beam/io/parquetio.py @@ -48,6 +48,7 @@ from apache_beam.transforms import PTransform from apache_beam.transforms import window from apache_beam.typehints import schemas +from apache_beam.utils.windowed_value import WindowedValue try: import pyarrow as pa @@ -105,8 +106,10 @@ def __init__( self._buffer_size = record_batch_size self._record_batches = [] self._record_batches_byte_size = 0 + self._window = None - def process(self, row): + def process(self, row, w=DoFn.WindowParam, pane=DoFn.PaneInfoParam): + self._window = w if len(self._buffer[0]) >= self._buffer_size: self._flush_buffer() @@ -116,14 +119,29 @@ def process(self, row): # reorder the data in columnar format. for i, n in enumerate(self._schema.names): - self._buffer[i].append(row[n]) + # Handle missing nullable fields by using None as default value + field = self._schema.field(i) + if field.nullable and n not in row: + self._buffer[i].append(None) + else: + self._buffer[i].append(row[n]) def finish_bundle(self): if len(self._buffer[0]) > 0: self._flush_buffer() if self._record_batches_byte_size > 0: table = self._create_table() - yield window.GlobalWindows.windowed_value_at_end_of_window(table) + if self._window is None or isinstance(self._window, window.GlobalWindow): + # bounded input + yield window.GlobalWindows.windowed_value_at_end_of_window(table) + else: + # unbounded input + yield WindowedValue( + table, + timestamp=self._window. + end, #or it could be max of timestamp of the rows processed + windows=[self._window] # TODO(pabloem) HOW DO WE GET THE PANE + ) def display_data(self): res = super().display_data() @@ -476,7 +494,9 @@ def __init__( file_name_suffix='', num_shards=0, shard_name_template=None, - mime_type='application/x-parquet'): + mime_type='application/x-parquet', + triggering_frequency=None, + ): """Initialize a WriteToParquet transform. Writes parquet files from a :class:`~apache_beam.pvalue.PCollection` of @@ -540,14 +560,26 @@ def __init__( the performance of a pipeline. Setting this value is not recommended unless you require a specific number of output files. shard_name_template: A template string containing placeholders for - the shard number and shard count. When constructing a filename for a - particular shard number, the upper-case letters 'S' and 'N' are - replaced with the 0-padded shard number and shard count respectively. - This argument can be '' in which case it behaves as if num_shards was - set to 1 and only one file will be generated. The default pattern used - is '-SSSSS-of-NNNNN' if None is passed as the shard_name_template. + the shard number and shard count. Currently only ``''``, + ``'-SSSSS-of-NNNNN'``, ``'-W-SSSSS-of-NNNNN'`` and + ``'-V-SSSSS-of-NNNNN'`` are patterns accepted by the service. + When constructing a filename for a particular shard number, the + upper-case letters ``S`` and ``N`` are replaced with the ``0``-padded + shard number and shard count respectively. This argument can be ``''`` + in which case it behaves as if num_shards was set to 1 and only one file + will be generated. The default pattern used is ``'-SSSSS-of-NNNNN'`` for + bounded PCollections and for ``'-W-SSSSS-of-NNNNN'`` unbounded + PCollections. + W is used for windowed shard naming and is replaced with + ``[window.start, window.end)`` + V is used for windowed shard naming and is replaced with + ``[window.start.to_utc_datetime().strftime("%Y-%m-%dT%H-%M-%S"), + window.end.to_utc_datetime().strftime("%Y-%m-%dT%H-%M-%S")`` mime_type: The MIME type to use for the produced files, if the filesystem supports specifying MIME types. + triggering_frequency: (int) Every triggering_frequency duration, a window + will be triggered and all bundles in the window will be written. + If set it overrides user windowing. Mandatory for GlobalWindow. Returns: A WriteToParquet transform usable for writing. @@ -567,10 +599,20 @@ def __init__( file_name_suffix, num_shards, shard_name_template, - mime_type + mime_type, + triggering_frequency ) def expand(self, pcoll): + if (not pcoll.is_bounded and self._sink.shard_name_template + == filebasedsink.DEFAULT_SHARD_NAME_TEMPLATE): + self._sink.shard_name_template = ( + filebasedsink.DEFAULT_WINDOW_SHARD_NAME_TEMPLATE) + self._sink.shard_name_format = self._sink._template_to_format( + self._sink.shard_name_template) + self._sink.shard_name_glob_format = self._sink._template_to_glob_format( + self._sink.shard_name_template) + if self._schema is None: try: beam_schema = schemas.schema_from_element_type(pcoll.element_type) @@ -583,7 +625,11 @@ def expand(self, pcoll): else: convert_fn = _RowDictionariesToArrowTable( self._schema, self._row_group_buffer_size, self._record_batch_size) - return pcoll | ParDo(convert_fn) | Write(self._sink) + if pcoll.is_bounded: + return pcoll | ParDo(convert_fn) | Write(self._sink) + else: + self._sink.convert_fn = convert_fn + return pcoll | Write(self._sink) def display_data(self): return { @@ -610,7 +656,7 @@ def __init__( num_shards=0, shard_name_template=None, mime_type='application/x-parquet', - ): + triggering_frequency=None): """Initialize a WriteToParquetBatched transform. Writes parquet files from a :class:`~apache_beam.pvalue.PCollection` of @@ -668,11 +714,21 @@ def __init__( the shard number and shard count. When constructing a filename for a particular shard number, the upper-case letters 'S' and 'N' are replaced with the 0-padded shard number and shard count respectively. + W is used for windowed shard naming and is replaced with + ``[window.start, window.end)`` + V is used for windowed shard naming and is replaced with + ``[window.start.to_utc_datetime().isoformat(), + window.end.to_utc_datetime().isoformat()`` This argument can be '' in which case it behaves as if num_shards was - set to 1 and only one file will be generated. The default pattern used - is '-SSSSS-of-NNNNN' if None is passed as the shard_name_template. + set to 1 and only one file will be generated. + The default pattern used is '-SSSSS-of-NNNNN' if None is passed as the + shard_name_template and the PCollection is bounded. + The default pattern used is '-W-SSSSS-of-NNNNN' if None is passed as the + shard_name_template and the PCollection is unbounded. mime_type: The MIME type to use for the produced files, if the filesystem supports specifying MIME types. + triggering_frequency: (int) Every triggering_frequency duration, a window + will be triggered and all bundles in the window will be written. Returns: A WriteToParquetBatched transform usable for writing. @@ -688,10 +744,19 @@ def __init__( file_name_suffix, num_shards, shard_name_template, - mime_type + mime_type, + triggering_frequency ) def expand(self, pcoll): + if (not pcoll.is_bounded and self._sink.shard_name_template + == filebasedsink.DEFAULT_SHARD_NAME_TEMPLATE): + self._sink.shard_name_template = ( + filebasedsink.DEFAULT_WINDOW_SHARD_NAME_TEMPLATE) + self._sink.shard_name_format = self._sink._template_to_format( + self._sink.shard_name_template) + self._sink.shard_name_glob_format = self._sink._template_to_glob_format( + self._sink.shard_name_template) return pcoll | Write(self._sink) def display_data(self): @@ -707,7 +772,8 @@ def _create_parquet_sink( file_name_suffix, num_shards, shard_name_template, - mime_type): + mime_type, + triggering_frequency=60): return \ _ParquetSink( file_path_prefix, @@ -718,7 +784,8 @@ def _create_parquet_sink( file_name_suffix, num_shards, shard_name_template, - mime_type + mime_type, + triggering_frequency ) @@ -734,7 +801,8 @@ def __init__( file_name_suffix, num_shards, shard_name_template, - mime_type): + mime_type, + triggering_frequency): super().__init__( file_path_prefix, file_name_suffix=file_name_suffix, @@ -744,7 +812,8 @@ def __init__( mime_type=mime_type, # Compression happens at the block level using the supplied codec, and # not at the file level. - compression_type=CompressionTypes.UNCOMPRESSED) + compression_type=CompressionTypes.UNCOMPRESSED, + triggering_frequency=triggering_frequency) self._schema = schema self._codec = codec if ARROW_MAJOR_VERSION == 1 and self._codec.lower() == "lz4": diff --git a/sdks/python/apache_beam/io/parquetio_it_test.py b/sdks/python/apache_beam/io/parquetio_it_test.py index 052b54f3ebfb..b06e7268fec4 100644 --- a/sdks/python/apache_beam/io/parquetio_it_test.py +++ b/sdks/python/apache_beam/io/parquetio_it_test.py @@ -19,10 +19,14 @@ import logging import string import unittest +import uuid from collections import Counter +from datetime import datetime import pytest +import pytz +import apache_beam as beam from apache_beam import Create from apache_beam import DoFn from apache_beam import FlatMap @@ -37,6 +41,7 @@ from apache_beam.testing.util import BeamAssertException from apache_beam.transforms import CombineGlobally from apache_beam.transforms.combiners import Count +from apache_beam.transforms.periodicsequence import PeriodicImpulse try: import pyarrow as pa @@ -142,6 +147,42 @@ def get_int(self): return i +@unittest.skipIf(pa is None, "PyArrow is not installed.") +class WriteStreamingIT(unittest.TestCase): + def setUp(self): + self.test_pipeline = TestPipeline(is_integration_test=True) + self.runner_name = type(self.test_pipeline.runner).__name__ + super().setUp() + + def test_write_streaming_2_shards_default_shard_name_template( + self, num_shards=2): + + args = self.test_pipeline.get_full_options_as_args(streaming=True) + + unique_id = str(uuid.uuid4()) + output_file = f'gs://apache-beam-testing-integration-testing/iobase/test-{unique_id}' # pylint: disable=line-too-long + p = beam.Pipeline(argv=args) + pyschema = pa.schema([('age', pa.int64())]) + + _ = ( + p + | "generate impulse" >> PeriodicImpulse( + start_timestamp=datetime(2021, 3, 1, 0, 0, 1, 0, + tzinfo=pytz.UTC).timestamp(), + stop_timestamp=datetime(2021, 3, 1, 0, 0, 20, 0, + tzinfo=pytz.UTC).timestamp(), + fire_interval=1) + | "generate data" >> beam.Map(lambda t: {'age': t * 10}) + | 'WriteToParquet' >> beam.io.WriteToParquet( + file_path_prefix=output_file, + file_name_suffix=".parquet", + num_shards=num_shards, + triggering_frequency=60, + schema=pyschema)) + result = p.run() + result.wait_until_finish(duration=600 * 1000) + + if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) unittest.main() diff --git a/sdks/python/apache_beam/io/parquetio_test.py b/sdks/python/apache_beam/io/parquetio_test.py index c602f4cc801b..78d1db4cc7c2 100644 --- a/sdks/python/apache_beam/io/parquetio_test.py +++ b/sdks/python/apache_beam/io/parquetio_test.py @@ -16,17 +16,21 @@ # # pytype: skip-file +import glob import json import logging import os +import re import shutil import tempfile import unittest +from datetime import datetime from tempfile import TemporaryDirectory import hamcrest as hc import pandas import pytest +import pytz from parameterized import param from parameterized import parameterized @@ -45,20 +49,21 @@ from apache_beam.io.parquetio import _create_parquet_sink from apache_beam.io.parquetio import _create_parquet_source from apache_beam.testing.test_pipeline import TestPipeline +from apache_beam.testing.test_stream import TestStream from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to from apache_beam.transforms.display import DisplayData from apache_beam.transforms.display_test import DisplayDataItemMatcher +from apache_beam.transforms.util import LogElements try: import pyarrow as pa import pyarrow.parquet as pq + ARROW_MAJOR_VERSION, _, _ = map(int, pa.__version__.split('.')) except ImportError: pa = None - pl = None pq = None - -ARROW_MAJOR_VERSION, _, _ = map(int, pa.__version__.split('.')) + ARROW_MAJOR_VERSION = 0 @unittest.skipIf(pa is None, "PyArrow is not installed.") @@ -416,6 +421,76 @@ def test_schema_read_write(self): | Map(stable_repr)) assert_that(readback, equal_to([stable_repr(r) for r in rows])) + def test_write_with_nullable_fields_missing_data(self): + """Test WriteToParquet with nullable fields where some fields are missing. + + This test addresses the bug reported in: + https://github.com/apache/beam/issues/35791 + where WriteToParquet fails with a KeyError if any nullable + field is missing in the data. + """ + # Define PyArrow schema with all fields nullable + schema = pa.schema([ + pa.field("id", pa.int64(), nullable=True), + pa.field("name", pa.string(), nullable=True), + pa.field("age", pa.int64(), nullable=True), + pa.field("email", pa.string(), nullable=True), + ]) + + # Sample data with missing nullable fields + data = [ + { + 'id': 1, 'name': 'Alice', 'age': 30 + }, # missing 'email' + { + 'id': 2, 'name': 'Bob', 'age': 25, 'email': 'bob@example.com' + }, # all fields present + { + 'id': 3, 'name': 'Charlie', 'age': None, 'email': None + }, # explicit None values + { + 'id': 4, 'name': 'David' + }, # missing 'age' and 'email' + ] + + with TemporaryDirectory() as tmp_dirname: + path = os.path.join(tmp_dirname, 'nullable_test') + + # Write data with missing nullable fields - this should not raise KeyError + with TestPipeline() as p: + _ = ( + p + | Create(data) + | WriteToParquet( + path, schema, num_shards=1, shard_name_template='')) + + # Read back and verify the data + with TestPipeline() as p: + readback = ( + p + | ReadFromParquet(path + '*') + | Map(json.dumps, sort_keys=True)) + + # Expected data should have None for missing nullable fields + expected_data = [ + { + 'id': 1, 'name': 'Alice', 'age': 30, 'email': None + }, + { + 'id': 2, 'name': 'Bob', 'age': 25, 'email': 'bob@example.com' + }, + { + 'id': 3, 'name': 'Charlie', 'age': None, 'email': None + }, + { + 'id': 4, 'name': 'David', 'age': None, 'email': None + }, + ] + + assert_that( + readback, + equal_to([json.dumps(r, sort_keys=True) for r in expected_data])) + def test_batched_read(self): with TemporaryDirectory() as tmp_dirname: path = os.path.join(tmp_dirname + "tmp_filename") @@ -655,6 +730,290 @@ def test_read_all_from_parquet_with_filename(self): equal_to(result)) +class GenerateEvent(beam.PTransform): + @staticmethod + def sample_data(): + return GenerateEvent() + + def expand(self, input): + elemlist = [{'age': 10}, {'age': 20}, {'age': 30}] + elem = elemlist + return ( + input + | TestStream().add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 1, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 2, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 3, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 4, 0, + tzinfo=pytz.UTC).timestamp()). + advance_watermark_to( + datetime(2021, 3, 1, 0, 0, 5, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 5, 0, + tzinfo=pytz.UTC).timestamp()). + add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 6, + 0, tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 7, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 8, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 9, 0, + tzinfo=pytz.UTC).timestamp()). + advance_watermark_to( + datetime(2021, 3, 1, 0, 0, 10, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 10, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 11, 0, + tzinfo=pytz.UTC).timestamp()). + add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 12, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 13, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 14, 0, + tzinfo=pytz.UTC).timestamp()). + advance_watermark_to( + datetime(2021, 3, 1, 0, 0, 15, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 15, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 16, 0, + tzinfo=pytz.UTC).timestamp()). + add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 17, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 18, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 19, 0, + tzinfo=pytz.UTC).timestamp()). + advance_watermark_to( + datetime(2021, 3, 1, 0, 0, 20, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 20, 0, + tzinfo=pytz.UTC).timestamp()).advance_watermark_to( + datetime( + 2021, 3, 1, 0, 0, 25, 0, tzinfo=pytz.UTC). + timestamp()).advance_watermark_to_infinity()) + + +class WriteStreamingTest(unittest.TestCase): + def setUp(self): + super().setUp() + self.tempdir = tempfile.mkdtemp() + + def tearDown(self): + if os.path.exists(self.tempdir): + shutil.rmtree(self.tempdir) + + def test_write_streaming_2_shards_default_shard_name_template( + self, num_shards=2): + with TestPipeline() as p: + output = (p | GenerateEvent.sample_data()) + #ParquetIO + pyschema = pa.schema([('age', pa.int64())]) + output2 = output | 'WriteToParquet' >> beam.io.WriteToParquet( + file_path_prefix=self.tempdir + "/ouput_WriteToParquet", + file_name_suffix=".parquet", + num_shards=num_shards, + triggering_frequency=60, + schema=pyschema) + _ = output2 | 'LogElements after WriteToParquet' >> LogElements( + prefix='after WriteToParquet ', with_window=True, level=logging.INFO) + + # Regex to match the expected windowed file pattern + # Example: + # ouput_WriteToParquet-[1614556800.0, 1614556805.0)-00000-of-00002.parquet + # It captures: window_interval, shard_num, total_shards + pattern_string = ( + r'.*-\[(?P[\d\.]+), ' + r'(?P[\d\.]+|Infinity)\)-' + r'(?P\d{5})-of-(?P\d{5})\.parquet$') + pattern = re.compile(pattern_string) + file_names = [] + for file_name in glob.glob(self.tempdir + '/ouput_WriteToParquet*'): + match = pattern.match(file_name) + self.assertIsNotNone( + match, f"File name {file_name} did not match expected pattern.") + if match: + file_names.append(file_name) + print("Found files matching expected pattern:", file_names) + self.assertEqual( + len(file_names), + num_shards, + "expected %d files, but got: %d" % (num_shards, len(file_names))) + + def test_write_streaming_2_shards_custom_shard_name_template( + self, num_shards=2, shard_name_template='-V-SSSSS-of-NNNNN'): + with TestPipeline() as p: + output = (p | GenerateEvent.sample_data()) + #ParquetIO + pyschema = pa.schema([('age', pa.int64())]) + output2 = output | 'WriteToParquet' >> beam.io.WriteToParquet( + file_path_prefix=self.tempdir + "/ouput_WriteToParquet", + file_name_suffix=".parquet", + shard_name_template=shard_name_template, + num_shards=num_shards, + triggering_frequency=60, + schema=pyschema) + _ = output2 | 'LogElements after WriteToParquet' >> LogElements( + prefix='after WriteToParquet ', with_window=True, level=logging.INFO) + + # Regex to match the expected windowed file pattern + # Example: + # ouput_WriteToParquet-[2021-03-01T00-00-00, 2021-03-01T00-01-00)- + # 00000-of-00002.parquet + # It captures: window_interval, shard_num, total_shards + pattern_string = ( + r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}-\d{2}-\d{2}), ' + r'(?P\d{4}-\d{2}-\d{2}T\d{2}-\d{2}-\d{2}|Infinity)\)-' + r'(?P\d{5})-of-(?P\d{5})\.parquet$') + pattern = re.compile(pattern_string) + file_names = [] + for file_name in glob.glob(self.tempdir + '/ouput_WriteToParquet*'): + match = pattern.match(file_name) + self.assertIsNotNone( + match, f"File name {file_name} did not match expected pattern.") + if match: + file_names.append(file_name) + print("Found files matching expected pattern:", file_names) + self.assertEqual( + len(file_names), + num_shards, + "expected %d files, but got: %d" % (num_shards, len(file_names))) + + def test_write_streaming_2_shards_custom_shard_name_template_5s_window( + self, + num_shards=2, + shard_name_template='-V-SSSSS-of-NNNNN', + triggering_frequency=5): + with TestPipeline() as p: + output = (p | GenerateEvent.sample_data()) + #ParquetIO + pyschema = pa.schema([('age', pa.int64())]) + output2 = output | 'WriteToParquet' >> beam.io.WriteToParquet( + file_path_prefix=self.tempdir + "/ouput_WriteToParquet", + file_name_suffix=".parquet", + shard_name_template=shard_name_template, + num_shards=num_shards, + triggering_frequency=triggering_frequency, + schema=pyschema) + _ = output2 | 'LogElements after WriteToParquet' >> LogElements( + prefix='after WriteToParquet ', with_window=True, level=logging.INFO) + + # Regex to match the expected windowed file pattern + # Example: + # ouput_WriteToParquet-[2021-03-01T00-00-00, 2021-03-01T00-01-00)- + # 00000-of-00002.parquet + # It captures: window_interval, shard_num, total_shards + pattern_string = ( + r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}-\d{2}-\d{2}), ' + r'(?P\d{4}-\d{2}-\d{2}T\d{2}-\d{2}-\d{2}|Infinity)\)-' + r'(?P\d{5})-of-(?P\d{5})\.parquet$') + pattern = re.compile(pattern_string) + file_names = [] + for file_name in glob.glob(self.tempdir + '/ouput_WriteToParquet*'): + match = pattern.match(file_name) + self.assertIsNotNone( + match, f"File name {file_name} did not match expected pattern.") + if match: + file_names.append(file_name) + print("Found files matching expected pattern:", file_names) + # for 5s window size, the input should be processed by 5 windows with + # 2 shards per window + self.assertEqual( + len(file_names), + 10, + "expected %d files, but got: %d" % (num_shards, len(file_names))) + + def test_write_streaming_undef_shards_default_shard_name_template_windowed_pcoll( # pylint: disable=line-too-long + self): + with TestPipeline() as p: + output = ( + p | GenerateEvent.sample_data() + | 'User windowing' >> beam.transforms.core.WindowInto( + beam.transforms.window.FixedWindows(10), + trigger=beam.transforms.trigger.AfterWatermark(), + accumulation_mode=beam.transforms.trigger.AccumulationMode. + DISCARDING, + allowed_lateness=beam.utils.timestamp.Duration(seconds=0))) + #ParquetIO + pyschema = pa.schema([('age', pa.int64())]) + output2 = output | 'WriteToParquet' >> beam.io.WriteToParquet( + file_path_prefix=self.tempdir + "/ouput_WriteToParquet", + file_name_suffix=".parquet", + num_shards=0, + schema=pyschema) + _ = output2 | 'LogElements after WriteToParquet' >> LogElements( + prefix='after WriteToParquet ', with_window=True, level=logging.INFO) + + # Regex to match the expected windowed file pattern + # Example: + # ouput_WriteToParquet-[1614556800.0, 1614556805.0)-00000-of-00002.parquet + # It captures: window_interval, shard_num, total_shards + pattern_string = ( + r'.*-\[(?P[\d\.]+), ' + r'(?P[\d\.]+|Infinity)\)-' + r'(?P\d{5})-of-(?P\d{5})\.parquet$') + pattern = re.compile(pattern_string) + file_names = [] + for file_name in glob.glob(self.tempdir + '/ouput_WriteToParquet*'): + match = pattern.match(file_name) + self.assertIsNotNone( + match, f"File name {file_name} did not match expected pattern.") + if match: + file_names.append(file_name) + print("Found files matching expected pattern:", file_names) + self.assertGreaterEqual( + len(file_names), + 1 * 3, #25s of data covered by 3 10s windows + "expected %d files, but got: %d" % (1 * 3, len(file_names))) + + if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) unittest.main() diff --git a/sdks/python/apache_beam/io/requestresponse_it_test.py b/sdks/python/apache_beam/io/requestresponse_it_test.py index 712ccc7881d6..8ac7cdb6f5fd 100644 --- a/sdks/python/apache_beam/io/requestresponse_it_test.py +++ b/sdks/python/apache_beam/io/requestresponse_it_test.py @@ -17,6 +17,7 @@ import base64 import logging import sys +import time import typing import unittest from dataclasses import dataclass @@ -206,7 +207,7 @@ def __exit__(self, exc_type, exc_val, exc_tb): @pytest.mark.uses_testcontainer class TestRedisCache(unittest.TestCase): def setUp(self) -> None: - self.retries = 3 + self.retries = 5 self._start_container() def test_rrio_cache_all_miss(self): @@ -303,6 +304,8 @@ def _start_container(self): if i == self.retries - 1: _LOGGER.error('Unable to start redis container for RRIO tests.') raise e + # Add a small delay between retries to avoid rapid successive failures + time.sleep(2) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/io/textio.py b/sdks/python/apache_beam/io/textio.py index ad7cbe6ea765..ba28fc608a0c 100644 --- a/sdks/python/apache_beam/io/textio.py +++ b/sdks/python/apache_beam/io/textio.py @@ -479,12 +479,12 @@ def __init__( shard number and shard count respectively. This argument can be ``''`` in which case it behaves as if num_shards was set to 1 and only one file will be generated. The default pattern used is ``'-SSSSS-of-NNNNN'`` for - bounded PCollections and for ``'-W-SSSSS-of-NNNNN'`` unbounded + bounded PCollections and for ``'-W-SSSSS-of-NNNNN'`` unbounded PCollections. - W is used for windowed shard naming and is replaced with + W is used for windowed shard naming and is replaced with ``[window.start, window.end)`` - V is used for windowed shard naming and is replaced with - ``[window.start.to_utc_datetime().strftime("%Y-%m-%dT%H-%M-%S"), + V is used for windowed shard naming and is replaced with + ``[window.start.to_utc_datetime().strftime("%Y-%m-%dT%H-%M-%S"), window.end.to_utc_datetime().strftime("%Y-%m-%dT%H-%M-%S")`` coder: Coder used to encode each line. compression_type: Used to handle compressed output files. Typical value @@ -505,7 +505,7 @@ def __init__( to exceed this value. This also tracks the uncompressed, not compressed, size of the shard. skip_if_empty: Don't write any shards if the PCollection is empty. - triggering_frequency: (int) Every triggering_frequency duration, a window + triggering_frequency: (int) Every triggering_frequency duration, a window will be triggered and all bundles in the window will be written. If set it overrides user windowing. Mandatory for GlobalWindow. @@ -877,12 +877,12 @@ def __init__( shard number and shard count respectively. This argument can be ``''`` in which case it behaves as if num_shards was set to 1 and only one file will be generated. The default pattern used is ``'-SSSSS-of-NNNNN'`` for - bounded PCollections and for ``'-W-SSSSS-of-NNNNN'`` unbounded + bounded PCollections and for ``'-W-SSSSS-of-NNNNN'`` unbounded PCollections. - W is used for windowed shard naming and is replaced with + W is used for windowed shard naming and is replaced with ``[window.start, window.end)`` - V is used for windowed shard naming and is replaced with - ``[window.start.to_utc_datetime().strftime("%Y-%m-%dT%H-%M-%S"), + V is used for windowed shard naming and is replaced with + ``[window.start.to_utc_datetime().strftime("%Y-%m-%dT%H-%M-%S"), window.end.to_utc_datetime().strftime("%Y-%m-%dT%H-%M-%S")`` coder (~apache_beam.coders.coders.Coder): Coder used to encode each line. compression_type (str): Used to handle compressed output files. @@ -908,8 +908,8 @@ def __init__( skip_if_empty: Don't write any shards if the PCollection is empty. In case of an empty PCollection, this will still delete existing files having same file path and not create new ones. - triggering_frequency: (int) Every triggering_frequency duration, a window - will be triggered and all bundles in the window will be written. + triggering_frequency: (int) Every triggering_frequency duration, a window + will be triggered and all bundles in the window will be written. """ self._sink = _TextSink( @@ -973,7 +973,12 @@ def append(dest): @append_pandas_args( pandas.read_csv, exclude=['filepath_or_buffer', 'iterator']) - def ReadFromCsv(path: str, *, splittable: bool = True, **kwargs): + def ReadFromCsv( + path: str, + *, + splittable: bool = True, + filename_column: Optional[str] = None, + **kwargs): """A PTransform for reading comma-separated values (csv) files into a PCollection. @@ -985,11 +990,17 @@ def ReadFromCsv(path: str, *, splittable: bool = True, **kwargs): This should be set to False if single records span multiple lines (e.g. a quoted field has a newline inside of it). Setting this to false may disable liquid sharding. + filename_column (str): If not None, the name of the column to add + to each record, containing the filename of the source file. **kwargs: Extra arguments passed to `pandas.read_csv` (see below). """ from apache_beam.dataframe.io import ReadViaPandas return 'ReadFromCsv' >> ReadViaPandas( - 'csv', path, splittable=splittable, **kwargs) + 'csv', + path, + splittable=splittable, + filename_column=filename_column, + **kwargs) @append_pandas_args( pandas.DataFrame.to_csv, exclude=['path_or_buf', 'index', 'index_label']) diff --git a/sdks/python/apache_beam/io/textio_test.py b/sdks/python/apache_beam/io/textio_test.py index 192ef3c6220f..4f804fa44c44 100644 --- a/sdks/python/apache_beam/io/textio_test.py +++ b/sdks/python/apache_beam/io/textio_test.py @@ -1765,6 +1765,31 @@ def test_csv_read_write(self): assert_that(pcoll, equal_to(records)) + def test_csv_read_with_filename(self): + records = [beam.Row(a='str', b=ix) for ix in range(3)] + with tempfile.TemporaryDirectory() as dest: + file_path = os.path.join(dest, 'out.csv') + with TestPipeline() as p: + # pylint: disable=expression-not-assigned + p | beam.Create(records) | beam.io.WriteToCsv(file_path) + with TestPipeline() as p: + pcoll = ( + p + | beam.io.ReadFromCsv( + file_path + '*', filename_column='source_filename') + | beam.Map(lambda t: beam.Row(**dict(zip(type(t)._fields, t))))) + + # Get the sharded file name + files = glob.glob(file_path + '*') + self.assertEqual(len(files), 1) + sharded_file_path = files[0] + + expected = [ + beam.Row(a=r.a, b=r.b, source_filename=sharded_file_path) + for r in records + ] + assert_that(pcoll, equal_to(expected)) + def test_non_utf8_csv_read_write(self): content = b"\xe0,\xe1,\xe2\n0,1,2\n1,2,3\n" diff --git a/sdks/python/apache_beam/io/tfrecordio.py b/sdks/python/apache_beam/io/tfrecordio.py index b911c64a1348..e27ea5070b06 100644 --- a/sdks/python/apache_beam/io/tfrecordio.py +++ b/sdks/python/apache_beam/io/tfrecordio.py @@ -290,7 +290,8 @@ def __init__( file_name_suffix, num_shards, shard_name_template, - compression_type): + compression_type, + triggering_frequency=60): """Initialize a TFRecordSink. See WriteToTFRecord for details.""" super().__init__( @@ -300,7 +301,8 @@ def __init__( num_shards=num_shards, shard_name_template=shard_name_template, mime_type='application/octet-stream', - compression_type=compression_type) + compression_type=compression_type, + triggering_frequency=triggering_frequency) def write_encoded_record(self, file_handle, value): _TFRecordUtil.write_record(file_handle, value) @@ -315,7 +317,8 @@ def __init__( file_name_suffix='', num_shards=0, shard_name_template=None, - compression_type=CompressionTypes.AUTO): + compression_type=CompressionTypes.AUTO, + triggering_frequency=None): """Initialize WriteToTFRecord transform. Args: @@ -326,16 +329,29 @@ def __init__( file_name_suffix: Suffix for the files written. num_shards: The number of files (shards) used for output. If not set, the default value will be used. + In streaming if not set, the service will write a file per bundle. shard_name_template: A template string containing placeholders for - the shard number and shard count. When constructing a filename for a - particular shard number, the upper-case letters 'S' and 'N' are - replaced with the 0-padded shard number and shard count respectively. - This argument can be '' in which case it behaves as if num_shards was - set to 1 and only one file will be generated. The default pattern used - is '-SSSSS-of-NNNNN' if None is passed as the shard_name_template. + the shard number and shard count. Currently only ``''``, + ``'-SSSSS-of-NNNNN'``, ``'-W-SSSSS-of-NNNNN'`` and + ``'-V-SSSSS-of-NNNNN'`` are patterns accepted by the service. + When constructing a filename for a particular shard number, the + upper-case letters ``S`` and ``N`` are replaced with the ``0``-padded + shard number and shard count respectively. This argument can be ``''`` + in which case it behaves as if num_shards was set to 1 and only one file + will be generated. The default pattern used is ``'-SSSSS-of-NNNNN'`` for + bounded PCollections and for ``'-W-SSSSS-of-NNNNN'`` unbounded + PCollections. + W is used for windowed shard naming and is replaced with + ``[window.start, window.end)`` + V is used for windowed shard naming and is replaced with + ``[window.start.to_utc_datetime().strftime("%Y-%m-%dT%H-%M-%S"), + window.end.to_utc_datetime().strftime("%Y-%m-%dT%H-%M-%S")`` compression_type: Used to handle compressed output files. Typical value is CompressionTypes.AUTO, in which case the file_path's extension will be used to detect the compression. + triggering_frequency: (int) Every triggering_frequency duration, a window + will be triggered and all bundles in the window will be written. + If set it overrides user windowing. Mandatory for GlobalWindow. Returns: A WriteToTFRecord transform object. @@ -347,7 +363,17 @@ def __init__( file_name_suffix, num_shards, shard_name_template, - compression_type) + compression_type, + triggering_frequency) def expand(self, pcoll): + if (not pcoll.is_bounded and self._sink.shard_name_template + == filebasedsink.DEFAULT_SHARD_NAME_TEMPLATE): + self._sink.shard_name_template = ( + filebasedsink.DEFAULT_WINDOW_SHARD_NAME_TEMPLATE) + self._sink.shard_name_format = self._sink._template_to_format( + self._sink.shard_name_template) + self._sink.shard_name_glob_format = self._sink._template_to_glob_format( + self._sink.shard_name_template) + return pcoll | Write(self._sink) diff --git a/sdks/python/apache_beam/io/tfrecordio_test.py b/sdks/python/apache_beam/io/tfrecordio_test.py index a867c0212ad3..6522ade36d80 100644 --- a/sdks/python/apache_beam/io/tfrecordio_test.py +++ b/sdks/python/apache_beam/io/tfrecordio_test.py @@ -21,15 +21,20 @@ import glob import gzip import io +import json import logging import os import pickle import random import re +import shutil +import tempfile import unittest import zlib +from datetime import datetime import crcmod +import pytz import apache_beam as beam from apache_beam import Create @@ -41,9 +46,11 @@ from apache_beam.io.tfrecordio import _TFRecordSink from apache_beam.io.tfrecordio import _TFRecordUtil from apache_beam.testing.test_pipeline import TestPipeline +from apache_beam.testing.test_stream import TestStream from apache_beam.testing.test_utils import TempDir from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to +from apache_beam.transforms.util import LogElements try: import tensorflow.compat.v1 as tf # pylint: disable=import-error @@ -558,6 +565,258 @@ def test_end2end_read_write_read(self): assert_that(actual_data, equal_to(expected_data)) +class GenerateEvent(beam.PTransform): + @staticmethod + def sample_data(): + return GenerateEvent() + + def expand(self, input): + elemlist = [{'age': 10}, {'age': 20}, {'age': 30}] + elem = elemlist + return ( + input + | TestStream().add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 1, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 2, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 3, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 4, 0, + tzinfo=pytz.UTC).timestamp()). + advance_watermark_to( + datetime(2021, 3, 1, 0, 0, 5, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 5, 0, + tzinfo=pytz.UTC).timestamp()). + add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 6, + 0, tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 7, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 8, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 9, 0, + tzinfo=pytz.UTC).timestamp()). + advance_watermark_to( + datetime(2021, 3, 1, 0, 0, 10, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 10, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 11, 0, + tzinfo=pytz.UTC).timestamp()). + add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 12, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 13, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 14, 0, + tzinfo=pytz.UTC).timestamp()). + advance_watermark_to( + datetime(2021, 3, 1, 0, 0, 15, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 15, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 16, 0, + tzinfo=pytz.UTC).timestamp()). + add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 17, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 18, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 19, 0, + tzinfo=pytz.UTC).timestamp()). + advance_watermark_to( + datetime(2021, 3, 1, 0, 0, 20, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 20, 0, + tzinfo=pytz.UTC).timestamp()).advance_watermark_to( + datetime( + 2021, 3, 1, 0, 0, 25, 0, tzinfo=pytz.UTC). + timestamp()).advance_watermark_to_infinity()) + + +class WriteStreamingTest(unittest.TestCase): + def setUp(self): + super().setUp() + self.tempdir = tempfile.mkdtemp() + + def tearDown(self): + if os.path.exists(self.tempdir): + shutil.rmtree(self.tempdir) + + def test_write_streaming_2_shards_default_shard_name_template( + self, num_shards=2): + with TestPipeline() as p: + output = ( + p + | GenerateEvent.sample_data() + | 'User windowing' >> beam.transforms.core.WindowInto( + beam.transforms.window.FixedWindows(60), + trigger=beam.transforms.trigger.AfterWatermark(), + accumulation_mode=beam.transforms.trigger.AccumulationMode. + DISCARDING, + allowed_lateness=beam.utils.timestamp.Duration(seconds=0)) + | "encode" >> beam.Map(lambda s: json.dumps(s).encode('utf-8'))) + #TFrecordIO + output2 = output | 'WriteToTFRecord' >> beam.io.WriteToTFRecord( + file_path_prefix=self.tempdir + "/ouput_WriteToTFRecord", + file_name_suffix=".tfrecord", + num_shards=num_shards, + ) + _ = output2 | 'LogElements after WriteToTFRecord' >> LogElements( + prefix='after WriteToTFRecord ', with_window=True, level=logging.INFO) + + # Regex to match the expected windowed file pattern + # Example: + # ouput_WriteToTFRecord-[1614556800.0, 1614556805.0)-00000-of-00002.tfrecord + # It captures: window_interval, shard_num, total_shards + pattern_string = ( + r'.*-\[(?P[\d\.]+), ' + r'(?P[\d\.]+|Infinity)\)-' + r'(?P\d{5})-of-(?P\d{5})\.tfrecord$') + pattern = re.compile(pattern_string) + file_names = [] + for file_name in glob.glob(self.tempdir + '/ouput_WriteToTFRecord*'): + match = pattern.match(file_name) + self.assertIsNotNone( + match, f"File name {file_name} did not match expected pattern.") + if match: + file_names.append(file_name) + print("Found files matching expected pattern:", file_names) + self.assertEqual( + len(file_names), + num_shards, + "expected %d files, but got: %d" % (num_shards, len(file_names))) + + def test_write_streaming_2_shards_custom_shard_name_template( + self, num_shards=2, shard_name_template='-V-SSSSS-of-NNNNN'): + with TestPipeline() as p: + output = ( + p + | GenerateEvent.sample_data() + | "encode" >> beam.Map(lambda s: json.dumps(s).encode('utf-8'))) + #TFrecordIO + output2 = output | 'WriteToTFRecord' >> beam.io.WriteToTFRecord( + file_path_prefix=self.tempdir + "/ouput_WriteToTFRecord", + file_name_suffix=".tfrecord", + shard_name_template=shard_name_template, + num_shards=num_shards, + triggering_frequency=60, + ) + _ = output2 | 'LogElements after WriteToTFRecord' >> LogElements( + prefix='after WriteToTFRecord ', with_window=True, level=logging.INFO) + + # Regex to match the expected windowed file pattern + # Example: + # ouput_WriteToTFRecord-[2021-03-01T00-00-00, 2021-03-01T00-01-00)- + # 00000-of-00002.tfrecord + # It captures: window_interval, shard_num, total_shards + pattern_string = ( + r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}-\d{2}-\d{2}), ' + r'(?P\d{4}-\d{2}-\d{2}T\d{2}-\d{2}-\d{2}|Infinity)\)-' + r'(?P\d{5})-of-(?P\d{5})\.tfrecord$') + pattern = re.compile(pattern_string) + file_names = [] + for file_name in glob.glob(self.tempdir + '/ouput_WriteToTFRecord*'): + match = pattern.match(file_name) + self.assertIsNotNone( + match, f"File name {file_name} did not match expected pattern.") + if match: + file_names.append(file_name) + print("Found files matching expected pattern:", file_names) + self.assertEqual( + len(file_names), + num_shards, + "expected %d files, but got: %d" % (num_shards, len(file_names))) + + def test_write_streaming_2_shards_custom_shard_name_template_5s_window( + self, + num_shards=2, + shard_name_template='-V-SSSSS-of-NNNNN', + triggering_frequency=5): + with TestPipeline() as p: + output = ( + p + | GenerateEvent.sample_data() + | "encode" >> beam.Map(lambda s: json.dumps(s).encode('utf-8'))) + #TFrecordIO + output2 = output | 'WriteToTFRecord' >> beam.io.WriteToTFRecord( + file_path_prefix=self.tempdir + "/ouput_WriteToTFRecord", + file_name_suffix=".tfrecord", + shard_name_template=shard_name_template, + num_shards=num_shards, + triggering_frequency=triggering_frequency, + ) + _ = output2 | 'LogElements after WriteToTFRecord' >> LogElements( + prefix='after WriteToTFRecord ', with_window=True, level=logging.INFO) + + # Regex to match the expected windowed file pattern + # Example: + # ouput_WriteToTFRecord-[2021-03-01T00-00-00, 2021-03-01T00-01-00)- + # 00000-of-00002.tfrecord + # It captures: window_interval, shard_num, total_shards + pattern_string = ( + r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}-\d{2}-\d{2}), ' + r'(?P\d{4}-\d{2}-\d{2}T\d{2}-\d{2}-\d{2}|Infinity)\)-' + r'(?P\d{5})-of-(?P\d{5})\.tfrecord$') + pattern = re.compile(pattern_string) + file_names = [] + for file_name in glob.glob(self.tempdir + '/ouput_WriteToTFRecord*'): + match = pattern.match(file_name) + self.assertIsNotNone( + match, f"File name {file_name} did not match expected pattern.") + if match: + file_names.append(file_name) + print("Found files matching expected pattern:", file_names) + # for 5s window size, the input should be processed by 5 windows with + # 2 shards per window + self.assertEqual( + len(file_names), + 10, + "expected %d files, but got: %d" % (num_shards, len(file_names))) + + if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) unittest.main() diff --git a/sdks/python/apache_beam/metrics/monitoring_infos.py b/sdks/python/apache_beam/metrics/monitoring_infos.py index 6dc4b7ef9c57..46f856676d34 100644 --- a/sdks/python/apache_beam/metrics/monitoring_infos.py +++ b/sdks/python/apache_beam/metrics/monitoring_infos.py @@ -367,8 +367,8 @@ def create_monitoring_info( urn=urn, type=type_urn, labels=labels or {}, payload=payload) except TypeError as e: raise RuntimeError( - f'Failed to create MonitoringInfo for urn {urn} type {type} labels ' + - '{labels} and payload {payload}') from e + f'Failed to create MonitoringInfo for urn {urn} type {type_urn} ' + f'labels {labels} and payload {payload}') from e def is_counter(monitoring_info_proto): diff --git a/sdks/python/apache_beam/ml/anomaly/specifiable_test.py b/sdks/python/apache_beam/ml/anomaly/specifiable_test.py index ccd8efd286cb..a222cf57973e 100644 --- a/sdks/python/apache_beam/ml/anomaly/specifiable_test.py +++ b/sdks/python/apache_beam/ml/anomaly/specifiable_test.py @@ -22,6 +22,7 @@ import unittest from typing import Optional +import pytest from parameterized import parameterized from apache_beam.internal.cloudpickle import cloudpickle @@ -323,7 +324,10 @@ def __init__(self, arg): self.my_arg = arg * 10 type(self).counter += 1 - def test_on_pickle(self): + @pytest.mark.uses_dill + def test_on_dill_pickle(self): + pytest.importorskip("dill") + FooForPickle = TestInitCallCount.FooForPickle import dill @@ -339,6 +343,9 @@ def test_on_pickle(self): self.assertEqual(FooForPickle.counter, 1) self.assertEqual(new_foo_2.__dict__, foo.__dict__) + def test_on_pickle(self): + FooForPickle = TestInitCallCount.FooForPickle + # Note that pickle does not support classes/functions nested in a function. import pickle FooForPickle.counter = 0 diff --git a/sdks/python/apache_beam/ml/anomaly/transforms.py b/sdks/python/apache_beam/ml/anomaly/transforms.py index ef5501b33786..ce9601074754 100644 --- a/sdks/python/apache_beam/ml/anomaly/transforms.py +++ b/sdks/python/apache_beam/ml/anomaly/transforms.py @@ -569,13 +569,13 @@ class AnomalyDetection(beam.PTransform[beam.PCollection[Union[InputT, Examples:: - # Run a single anomaly detector - p | AnomalyDetection(ZScore(features=["x1"])) + # Run a single anomaly detector + p | AnomalyDetection(ZScore(features=["x1"])) - # Run an ensemble anomaly detector - sub_detectors = [ZScore(features=["x1"]), IQR(features=["x2"])] - p | AnomalyDetection( - EnsembleAnomalyDetector(sub_detectors, aggregation_strategy=AnyVote())) + # Run an ensemble anomaly detector + sub_detectors = [ZScore(features=["x1"]), IQR(features=["x2"])] + p | AnomalyDetection(EnsembleAnomalyDetector( + sub_detectors, aggregation_strategy=AnyVote())) Args: detector: The `AnomalyDetector` or `EnsembleAnomalyDetector` to use. diff --git a/sdks/python/apache_beam/ml/inference/base.py b/sdks/python/apache_beam/ml/inference/base.py index 4881fb74ef7b..2e1c4963f11d 100644 --- a/sdks/python/apache_beam/ml/inference/base.py +++ b/sdks/python/apache_beam/ml/inference/base.py @@ -55,8 +55,7 @@ from typing import Union import apache_beam as beam -from apache_beam.io.components.adaptive_throttler import AdaptiveThrottler -from apache_beam.metrics.metric import Metrics +from apache_beam.io.components.adaptive_throttler import ReactiveThrottler from apache_beam.utils import multi_process_shared from apache_beam.utils import retry from apache_beam.utils import shared @@ -354,14 +353,16 @@ def __init__( window_ms: int = 1 * _MILLISECOND_TO_SECOND, bucket_ms: int = 1 * _MILLISECOND_TO_SECOND, overload_ratio: float = 2): - """Initializes metrics tracking + an AdaptiveThrottler class for enabling - client-side throttling for remote calls to an inference service. + """Initializes a ReactiveThrottler class for enabling + client-side throttling for remote calls to an inference service. Also wraps + provided calls to the service with retry logic. + See https://s.apache.org/beam-client-side-throttling for more details on the configuration of the throttling and retry mechanics. Args: - namespace: the metrics and logging namespace + namespace: the metrics and logging namespace num_retries: the maximum number of times to retry a request on retriable errors before failing throttle_delay_secs: the amount of time to throttle when the client-side @@ -372,19 +373,18 @@ def __init__( window_ms: length of history to consider, in ms, to set throttling. bucket_ms: granularity of time buckets that we store data in, in ms. overload_ratio: the target ratio between requests sent and successful - requests. This is "K" in the formula in + requests. This is "K" in the formula in https://landing.google.com/sre/book/chapters/handling-overload.html. """ - # Configure AdaptiveThrottler and throttling metrics for client-side - # throttling behavior. - self.throttled_secs = Metrics.counter( - namespace, "cumulativeThrottlingSeconds") - self.throttler = AdaptiveThrottler( - window_ms=window_ms, bucket_ms=bucket_ms, overload_ratio=overload_ratio) + # Configure ReactiveThrottler for client-side throttling behavior. + self.throttler = ReactiveThrottler( + window_ms=window_ms, + bucket_ms=bucket_ms, + overload_ratio=overload_ratio, + namespace=namespace, + throttle_delay_secs=throttle_delay_secs) self.logger = logging.getLogger(namespace) - self.num_retries = num_retries - self.throttle_delay_secs = throttle_delay_secs self.retry_filter = retry_filter def __init_subclass__(cls): @@ -434,12 +434,7 @@ def run_inference( Returns: An Iterable of Predictions. """ - while self.throttler.throttle_request(time.time() * _MILLISECOND_TO_SECOND): - self.logger.info( - "Delaying request for %d seconds due to previous failures", - self.throttle_delay_secs) - time.sleep(self.throttle_delay_secs) - self.throttled_secs.inc(self.throttle_delay_secs) + self.throttler.throttle() try: req_time = time.time() @@ -1642,7 +1637,7 @@ def next_model_index(self, num_models): class _ModelStatus(): """A class holding any metadata about a model required by RunInference. - + Currently, this only includes whether or not the model is valid. Uses the model tag to map models to metadata. """ @@ -1656,7 +1651,7 @@ def __init__(self, share_model_across_processes: bool): def try_mark_current_model_invalid(self, min_model_life_seconds): """Mark the current model invalid. - + Since we don't have sufficient information to say which model is being marked invalid, but there may be multiple active models, we will mark all models currently in use as inactive so that they all get reloaded. To @@ -1678,7 +1673,7 @@ def try_mark_current_model_invalid(self, min_model_life_seconds): def get_valid_tag(self, tag: str) -> str: """Takes in a proposed valid tag and returns a valid one. - + Will always return a valid tag. If the passed in tag is valid, this function will simply return it, otherwise it will deterministically generate a new tag to use instead. The new tag will be the original tag @@ -1747,7 +1742,7 @@ def load_model_status( class _SharedModelWrapper(): """A router class to map incoming calls to the correct model. - + This allows us to round robin calls to models sitting in different processes so that we can more efficiently use resources (e.g. GPUs). """ diff --git a/sdks/python/apache_beam/ml/inference/base_test.py b/sdks/python/apache_beam/ml/inference/base_test.py index 2c1b77dca5bb..64fd73682e13 100644 --- a/sdks/python/apache_beam/ml/inference/base_test.py +++ b/sdks/python/apache_beam/ml/inference/base_test.py @@ -1037,7 +1037,7 @@ def test_timing_metrics(self): def test_forwards_batch_args(self): examples = list(range(100)) - with TestPipeline() as pipeline: + with TestPipeline('FnApiRunner') as pipeline: pcoll = pipeline | 'start' >> beam.Create(examples) actual = pcoll | base.RunInference(FakeModelHandlerNeedsBigBatch()) assert_that(actual, equal_to(examples), label='assert:inferences') diff --git a/sdks/python/apache_beam/ml/inference/test_resources/vllm.dockerfile b/sdks/python/apache_beam/ml/inference/test_resources/vllm.dockerfile index f27abbfd0051..5727437809c4 100644 --- a/sdks/python/apache_beam/ml/inference/test_resources/vllm.dockerfile +++ b/sdks/python/apache_beam/ml/inference/test_resources/vllm.dockerfile @@ -15,33 +15,54 @@ # limitations under the License. # Used for any vLLM integration test +# Dockerfile — Beam dev harness + install dev SDK from LOCAL source package FROM nvidia/cuda:12.4.1-devel-ubuntu22.04 -RUN apt update -RUN apt install software-properties-common -y -RUN add-apt-repository ppa:deadsnakes/ppa -RUN apt update +# 1) Non-interactive + timezone +ENV DEBIAN_FRONTEND=noninteractive \ + TZ=Etc/UTC -ARG DEBIAN_FRONTEND=noninteractive +RUN apt-get update && \ + apt-get install -y --no-install-recommends \ + curl \ + tzdata \ + software-properties-common \ + python3.10-full \ + python3.10-distutils \ + build-essential \ + python3.10-dev \ + cython3 && \ + ln -fs /usr/share/zoneinfo/$TZ /etc/localtime && \ + dpkg-reconfigure --frontend noninteractive tzdata && \ + rm -rf /var/lib/apt/lists/* -RUN apt install python3.12 -y -RUN apt install python3.12-venv -y -RUN apt install python3.12-dev -y -RUN rm /usr/bin/python3 -RUN ln -s python3.12 /usr/bin/python3 -RUN python3 --version -RUN apt-get install -y curl -RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.12 && pip install --upgrade pip +# 2) Symlink python3 to 3.10 +RUN ln -sf /usr/bin/python3.10 /usr/bin/python3 && \ + ln -sf /usr/bin/python3.10 /usr/bin/python -RUN pip install --no-cache-dir -vvv apache-beam[gcp]==2.58.1 -RUN pip install openai vllm +# 3) Install pip, setuptools & wheel +RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3 && \ + python3 -m pip install --upgrade pip setuptools wheel -RUN apt install libcairo2-dev pkg-config python3-dev -y -RUN pip install pycairo +# 4) Copy the Beam SDK harness (for Dataflow workers) +COPY --from=gcr.io/apache-beam-testing/beam-sdk/beam_python3.10_sdk:2.68.0.dev \ + /opt/apache/beam /opt/apache/beam -# Copy the Apache Beam worker dependencies from the Beam Python 3.12 SDK image. -COPY --from=apache/beam_python3.12_sdk:2.58.1 /opt/apache/beam /opt/apache/beam +# 5) Make sure the harness is discovered first +ENV PYTHONPATH=/opt/apache/beam:$PYTHONPATH -# Set the entrypoint to Apache Beam SDK worker launcher. -ENTRYPOINT [ "/opt/apache/beam/boot" ] +# 6) Install the Beam dev SDK from the local source package. +# This .tar.gz file will be created by GitHub Actions workflow +# and copied into the build context. +COPY ./sdks/python/build/apache-beam.tar.gz /tmp/beam.tar.gz +RUN python3 -m pip install --no-cache-dir "/tmp/beam.tar.gz[gcp]" + +# 7) Install vLLM, and other dependencies +RUN python3 -m pip install --no-cache-dir \ + openai>=1.52.2 \ + vllm>=0.6.3 \ + triton>=3.1.0 + +# 8) Use the Beam boot script as entrypoint +ENTRYPOINT ["/opt/apache/beam/boot"] \ No newline at end of file diff --git a/sdks/python/apache_beam/ml/inference/test_resources/vllm.dockerfile.old b/sdks/python/apache_beam/ml/inference/test_resources/vllm.dockerfile.old new file mode 100644 index 000000000000..b9c99e49e02f --- /dev/null +++ b/sdks/python/apache_beam/ml/inference/test_resources/vllm.dockerfile.old @@ -0,0 +1,47 @@ +# 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. + +# Used for any vLLM integration test + +FROM nvidia/cuda:12.4.1-devel-ubuntu22.04 + +RUN apt update +RUN apt install software-properties-common -y +RUN add-apt-repository ppa:deadsnakes/ppa +RUN apt update + +ARG DEBIAN_FRONTEND=noninteractive + +RUN apt install python3.12 -y +RUN apt install python3.12-venv -y +RUN apt install python3.12-dev -y +RUN rm /usr/bin/python3 +RUN ln -s python3.12 /usr/bin/python3 +RUN python3 --version +RUN apt-get install -y curl +RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.12 && pip install --upgrade pip + +RUN pip install --no-cache-dir -vvv apache-beam[gcp]==2.58.1 +RUN pip install openai vllm + +RUN apt install libcairo2-dev pkg-config python3-dev -y +RUN pip install pycairo + +# Copy the Apache Beam worker dependencies from the Beam Python 3.12 SDK image. +COPY --from=apache/beam_python3.12_sdk:2.58.1 /opt/apache/beam /opt/apache/beam + +# Set the entrypoint to Apache Beam SDK worker launcher. +ENTRYPOINT [ "/opt/apache/beam/boot" ] \ No newline at end of file diff --git a/sdks/python/apache_beam/ml/inference/vllm_tests_requirements.txt b/sdks/python/apache_beam/ml/inference/vllm_tests_requirements.txt new file mode 100644 index 000000000000..939f0526d808 --- /dev/null +++ b/sdks/python/apache_beam/ml/inference/vllm_tests_requirements.txt @@ -0,0 +1,22 @@ +# 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. +# + +torch>=1.7.1 +torchvision>=0.8.2 +pillow>=8.0.0 +transformers>=4.18.0 +google-cloud-monitoring>=2.27.0 +openai>=1.52.2 \ No newline at end of file diff --git a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py index ebc05722841c..81ceb6b69e71 100644 --- a/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py +++ b/sdks/python/apache_beam/ml/rag/enrichment/milvus_search_it_test.py @@ -34,18 +34,6 @@ import pytest import yaml -from pymilvus import CollectionSchema -from pymilvus import DataType -from pymilvus import FieldSchema -from pymilvus import Function -from pymilvus import FunctionType -from pymilvus import MilvusClient -from pymilvus import RRFRanker -from pymilvus.milvus_client import IndexParams -from testcontainers.core.config import MAX_TRIES as TC_MAX_TRIES -from testcontainers.core.config import testcontainers_config -from testcontainers.core.generic import DbContainer -from testcontainers.milvus import MilvusContainer import apache_beam as beam from apache_beam.ml.rag.types import Chunk @@ -54,7 +42,21 @@ from apache_beam.testing.test_pipeline import TestPipeline from apache_beam.testing.util import assert_that +# pylint: disable=ungrouped-imports try: + from pymilvus import ( + CollectionSchema, + DataType, + FieldSchema, + Function, + FunctionType, + MilvusClient, + RRFRanker) + from pymilvus.milvus_client import IndexParams + from testcontainers.core.config import MAX_TRIES as TC_MAX_TRIES + from testcontainers.core.config import testcontainers_config + from testcontainers.core.generic import DbContainer + from testcontainers.milvus import MilvusContainer from apache_beam.transforms.enrichment import Enrichment from apache_beam.ml.rag.enrichment.milvus_search import ( MilvusSearchEnrichmentHandler, @@ -295,7 +297,7 @@ def __init__( class MilvusEnrichmentTestHelper: @staticmethod def start_db_container( - image="milvusdb/milvus:v2.5.10", + image="milvusdb/milvus:v2.3.9", max_vec_fields=5, vector_client_max_retries=3, tc_max_retries=TC_MAX_TRIES) -> Optional[MilvusDBContainerInfo]: @@ -467,7 +469,7 @@ def create_user_yaml(service_port: int, max_vector_field_num=5): os.remove(path) -@pytest.mark.uses_testcontainer +@pytest.mark.require_docker_in_docker @unittest.skipUnless( platform.system() == "Linux", "Test runs only on Linux due to lack of support, as yet, for nested " @@ -483,22 +485,16 @@ class TestMilvusSearchEnrichment(unittest.TestCase): @classmethod def setUpClass(cls): - try: - cls._db = MilvusEnrichmentTestHelper.start_db_container( - cls._version, vector_client_max_retries=1, tc_max_retries=1) - cls._connection_params = MilvusConnectionParameters( - uri=cls._db.uri, - user=cls._db.user, - password=cls._db.password, - db_id=cls._db.id, - token=cls._db.token) - cls._collection_load_params = MilvusCollectionLoadParameters() - cls._collection_name = MilvusEnrichmentTestHelper.initialize_db_with_data( - cls._connection_params) - except Exception as e: - pytest.skip( - f"Skipping all tests in {cls.__name__} due to DB startup failure: {e}" - ) + cls._db = MilvusEnrichmentTestHelper.start_db_container(cls._version) + cls._connection_params = MilvusConnectionParameters( + uri=cls._db.uri, + user=cls._db.user, + password=cls._db.password, + db_id=cls._db.id, + token=cls._db.token) + cls._collection_load_params = MilvusCollectionLoadParameters() + cls._collection_name = MilvusEnrichmentTestHelper.initialize_db_with_data( + cls._connection_params) @classmethod def tearDownClass(cls): @@ -578,7 +574,7 @@ def test_empty_input_chunks(self): expected_chunks = [] - with TestPipeline(is_integration_test=True) as p: + with TestPipeline() as p: result = (p | beam.Create(test_chunks) | Enrichment(handler)) assert_that( result, @@ -706,7 +702,7 @@ def test_filtered_search_with_cosine_similarity_and_batching(self): embedding=Embedding(dense_embedding=[0.3, 0.4, 0.5])) ] - with TestPipeline(is_integration_test=True) as p: + with TestPipeline() as p: result = (p | beam.Create(test_chunks) | Enrichment(handler)) assert_that( result, @@ -811,7 +807,7 @@ def test_filtered_search_with_bm25_full_text_and_batching(self): embedding=Embedding()) ] - with TestPipeline(is_integration_test=True) as p: + with TestPipeline() as p: result = (p | beam.Create(test_chunks) | Enrichment(handler)) assert_that( result, @@ -952,7 +948,7 @@ def test_vector_search_with_euclidean_distance(self): embedding=Embedding(dense_embedding=[0.3, 0.4, 0.5])) ] - with TestPipeline(is_integration_test=True) as p: + with TestPipeline() as p: result = (p | beam.Create(test_chunks) | Enrichment(handler)) assert_that( result, @@ -1092,7 +1088,7 @@ def test_vector_search_with_inner_product_similarity(self): embedding=Embedding(dense_embedding=[0.3, 0.4, 0.5])) ] - with TestPipeline(is_integration_test=True) as p: + with TestPipeline() as p: result = (p | beam.Create(test_chunks) | Enrichment(handler)) assert_that( result, @@ -1157,7 +1153,7 @@ def test_keyword_search_with_inner_product_sparse_embedding(self): sparse_embedding=([1, 2, 3, 4], [0.05, 0.41, 0.05, 0.41]))) ] - with TestPipeline(is_integration_test=True) as p: + with TestPipeline() as p: result = (p | beam.Create(test_chunks) | Enrichment(handler)) assert_that( result, @@ -1230,7 +1226,7 @@ def test_hybrid_search(self): embedding=Embedding(dense_embedding=[0.1, 0.2, 0.3])) ] - with TestPipeline(is_integration_test=True) as p: + with TestPipeline() as p: result = (p | beam.Create(test_chunks) | Enrichment(handler)) assert_that( result, diff --git a/sdks/python/apache_beam/ml/rag/ingestion/bigquery_it_test.py b/sdks/python/apache_beam/ml/rag/ingestion/bigquery_it_test.py index 7df662ab0554..b21da0443467 100644 --- a/sdks/python/apache_beam/ml/rag/ingestion/bigquery_it_test.py +++ b/sdks/python/apache_beam/ml/rag/ingestion/bigquery_it_test.py @@ -117,7 +117,7 @@ def test_default_schema_missing_embedding(self): Chunk(id="1", content=Content(text="foo"), metadata={"a": "b"}), Chunk(id="2", content=Content(text="bar"), metadata={"c": "d"}) ] - with self.assertRaises(ValueError): + with self.assertRaisesRegex(Exception, "must contain dense embedding"): with beam.Pipeline() as p: _ = (p | beam.Create(chunks) | config.create_write_transform()) diff --git a/sdks/python/apache_beam/ml/transforms/base.py b/sdks/python/apache_beam/ml/transforms/base.py index 3b95ed719e5d..4031777ce152 100644 --- a/sdks/python/apache_beam/ml/transforms/base.py +++ b/sdks/python/apache_beam/ml/transforms/base.py @@ -810,3 +810,42 @@ def get_metrics_namespace(self) -> str: return ( self._underlying.get_metrics_namespace() or 'BeamML_ImageEmbeddingHandler') + + +class _MultiModalEmbeddingHandler(_EmbeddingHandler): + """ + A ModelHandler intended to be work on + list[dict[str, TypedDict(Image, Video, str)]] inputs. + + The inputs to the model handler are expected to be a list of dicts. + + For example, if the original mode is used with RunInference to take a + PCollection[E] to a PCollection[P], this ModelHandler would take a + PCollection[dict[str, E]] to a PCollection[dict[str, P]]. + + _MultiModalEmbeddingHandler will accept an EmbeddingsManager instance, which + contains the details of the model to be loaded and the inference_fn to be + used. The purpose of _MultiMOdalEmbeddingHandler is to generate embeddings + for image, video, and text inputs using the EmbeddingsManager instance. + + If the input is not an Image representation column, a RuntimeError will be + raised. + + This is an internal class and offers no backwards compatibility guarantees. + + Args: + embeddings_manager: An EmbeddingsManager instance. + """ + def _validate_column_data(self, batch): + # Don't want to require framework-specific imports + # here, so just catch columns of primatives for now. + if isinstance(batch[0], (int, str, float, bool)): + raise TypeError( + 'Embeddings can only be generated on ' + ' dict[str, dataclass] types. ' + f'Got dict[str, {type(batch[0])}] instead.') + + def get_metrics_namespace(self) -> str: + return ( + self._underlying.get_metrics_namespace() or + 'BeamML_MultiModalEmbeddingHandler') diff --git a/sdks/python/apache_beam/ml/transforms/base_test.py b/sdks/python/apache_beam/ml/transforms/base_test.py index 309c085f08f8..190381cc2f34 100644 --- a/sdks/python/apache_beam/ml/transforms/base_test.py +++ b/sdks/python/apache_beam/ml/transforms/base_test.py @@ -23,6 +23,7 @@ import time import unittest from collections.abc import Sequence +from dataclasses import dataclass from typing import Any from typing import Optional @@ -629,6 +630,122 @@ def test_handler_with_dict_inputs(self): ) +@dataclass +class FakeMultiModalInput: + image: Optional[PIL_Image] = None + video: Optional[Any] = None + text: Optional[str] = None + + +class FakeMultiModalModel: + def __call__(self, + example: list[FakeMultiModalInput]) -> list[FakeMultiModalInput]: + for i in range(len(example)): + if not isinstance(example[i], FakeMultiModalInput): + raise TypeError('Input must be a MultiModalInput') + return example + + +class FakeMultiModalModelHandler(ModelHandler): + def run_inference( + self, + batch: Sequence[FakeMultiModalInput], + model: Any, + inference_args: Optional[dict[str, Any]] = None): + return model(batch) + + def load_model(self): + return FakeMultiModalModel() + + +class FakeMultiModalEmbeddingsManager(base.EmbeddingsManager): + def __init__(self, columns, **kwargs): + super().__init__(columns=columns, **kwargs) + + def get_model_handler(self) -> ModelHandler: + FakeModelHandler.__repr__ = lambda x: 'FakeMultiModalEmbeddingsManager' # type: ignore[method-assign] + return FakeMultiModalModelHandler() + + def get_ptransform_for_processing(self, **kwargs) -> beam.PTransform: + return (RunInference(model_handler=base._MultiModalEmbeddingHandler(self))) + + def __repr__(self): + return 'FakeMultiModalEmbeddingsManager' + + +class TestMultiModalEmbeddingHandler(unittest.TestCase): + def setUp(self) -> None: + self.embedding_config = FakeMultiModalEmbeddingsManager(columns=['x']) + self.artifact_location = tempfile.mkdtemp() + + def tearDown(self) -> None: + shutil.rmtree(self.artifact_location) + + @unittest.skipIf(PIL is None, 'PIL module is not installed.') + def test_handler_with_non_dict_datatype(self): + image_handler = base._MultiModalEmbeddingHandler( + embeddings_manager=self.embedding_config) + data = [ + ('x', 'hi there'), + ('x', 'not an image'), + ('x', 'image_path.jpg'), + ] + with self.assertRaises(TypeError): + image_handler.run_inference(data, None, None) + + @unittest.skipIf(PIL is None, 'PIL module is not installed.') + def test_handler_with_incorrect_datatype(self): + image_handler = base._MultiModalEmbeddingHandler( + embeddings_manager=self.embedding_config) + data = [ + { + 'x': 'hi there' + }, + { + 'x': 'not an image' + }, + { + 'x': 'image_path.jpg' + }, + ] + with self.assertRaises(TypeError): + image_handler.run_inference(data, None, None) + + @unittest.skipIf(PIL is None, 'PIL module is not installed.') + def test_handler_with_dict_inputs(self): + input_one = FakeMultiModalInput( + image=PIL.Image.new(mode='RGB', size=(1, 1)), text="test image one") + input_two = FakeMultiModalInput( + image=PIL.Image.new(mode='RGB', size=(1, 1)), text="test image two") + input_three = FakeMultiModalInput( + image=PIL.Image.new(mode='RGB', size=(1, 1)), + video=bytes.fromhex('2Ef0 F1f2 '), + text="test image three with video") + data = [ + { + 'x': input_one + }, + { + 'x': input_two + }, + { + 'x': input_three + }, + ] + expected_data = [{key: value for key, value in d.items()} for d in data] + with beam.Pipeline() as p: + result = ( + p + | beam.Create(data) + | base.MLTransform( + write_artifact_location=self.artifact_location).with_transform( + self.embedding_config)) + assert_that( + result, + equal_to(expected_data), + ) + + class TestUtilFunctions(unittest.TestCase): def test_dict_input_fn_normal(self): input_list = [{'a': 1, 'b': 2}, {'a': 3, 'b': 4}] diff --git a/sdks/python/apache_beam/ml/transforms/embeddings/vertex_ai.py b/sdks/python/apache_beam/ml/transforms/embeddings/vertex_ai.py index a645ce32e2a0..c7c46d246b93 100644 --- a/sdks/python/apache_beam/ml/transforms/embeddings/vertex_ai.py +++ b/sdks/python/apache_beam/ml/transforms/embeddings/vertex_ai.py @@ -19,10 +19,14 @@ # Follow https://cloud.google.com/vertex-ai/docs/python-sdk/use-vertex-ai-python-sdk # pylint: disable=line-too-long # to install Vertex AI Python SDK. +import functools import logging +from collections.abc import Callable from collections.abc import Sequence +from dataclasses import dataclass from typing import Any from typing import Optional +from typing import cast from google.api_core.exceptions import ServerError from google.api_core.exceptions import TooManyRequests @@ -33,15 +37,28 @@ from apache_beam.ml.inference.base import ModelHandler from apache_beam.ml.inference.base import RemoteModelHandler from apache_beam.ml.inference.base import RunInference +from apache_beam.ml.rag.types import Chunk +from apache_beam.ml.rag.types import Embedding from apache_beam.ml.transforms.base import EmbeddingsManager +from apache_beam.ml.transforms.base import EmbeddingTypeAdapter from apache_beam.ml.transforms.base import _ImageEmbeddingHandler +from apache_beam.ml.transforms.base import _MultiModalEmbeddingHandler from apache_beam.ml.transforms.base import _TextEmbeddingHandler from vertexai.language_models import TextEmbeddingInput from vertexai.language_models import TextEmbeddingModel from vertexai.vision_models import Image from vertexai.vision_models import MultiModalEmbeddingModel - -__all__ = ["VertexAITextEmbeddings", "VertexAIImageEmbeddings"] +from vertexai.vision_models import MultiModalEmbeddingResponse +from vertexai.vision_models import Video +from vertexai.vision_models import VideoEmbedding +from vertexai.vision_models import VideoSegmentConfig + +__all__ = [ + "VertexAITextEmbeddings", + "VertexAIImageEmbeddings", + "VertexAIMultiModalEmbeddings", + "VertexAIMultiModalInput", +] DEFAULT_TASK_TYPE = "RETRIEVAL_DOCUMENT" # TODO: https://github.com/apache/beam/issues/29356 @@ -54,7 +71,6 @@ "CLUSTERING" ] _BATCH_SIZE = 5 # Vertex AI limits requests to 5 at a time. -_MSEC_TO_SEC = 1000 LOGGER = logging.getLogger("VertexAIEmbeddings") @@ -281,3 +297,222 @@ def get_ptransform_for_processing(self, **kwargs) -> beam.PTransform: return RunInference( model_handler=_ImageEmbeddingHandler(self), inference_args=self.inference_args) + + +@dataclass +class VertexImage: + image_content: Image + embedding: Optional[list[float]] = None + + +@dataclass +class VertexVideo: + video_content: Video + config: VideoSegmentConfig + embeddings: Optional[list[VideoEmbedding]] = None + + +@dataclass +class VertexAIMultiModalInput: + image: Optional[VertexImage] = None + video: Optional[VertexVideo] = None + contextual_text: Optional[Chunk] = None + + +class _VertexAIMultiModalEmbeddingHandler(RemoteModelHandler): + def __init__( + self, + model_name: str, + dimension: Optional[int] = None, + project: Optional[str] = None, + location: Optional[str] = None, + credentials: Optional[Credentials] = None, + **kwargs): + vertexai.init(project=project, location=location, credentials=credentials) + self.model_name = model_name + self.dimension = dimension + + super().__init__( + namespace='VertexAIMultiModelEmbeddingHandler', + retry_filter=_retry_on_appropriate_gcp_error, + **kwargs) + + def request( + self, + batch: Sequence[VertexAIMultiModalInput], + model: MultiModalEmbeddingModel, + inference_args: Optional[dict[str, Any]] = None): + embeddings = [] + # Max request size for multi-modal embedding models is 1 + for input in batch: + image_content: Optional[Image] = None + video_content: Optional[Video] = None + text_content: Optional[str] = None + video_config: Optional[VideoSegmentConfig] = None + + if input.image: + image_content = input.image.image_content + if input.video: + video_content = input.video.video_content + video_config = input.video.config + if input.contextual_text: + text_content = input.contextual_text.content.text + + prediction = model.get_embeddings( + image=image_content, + video=video_content, + contextual_text=text_content, + dimension=self.dimension, + video_segment_config=video_config) + embeddings.append(prediction) + return embeddings + + def create_client(self) -> MultiModalEmbeddingModel: + model = MultiModalEmbeddingModel.from_pretrained(self.model_name) + return model + + def __repr__(self): + # ModelHandler is internal to the user and is not exposed. + # Hence we need to override the __repr__ method to expose + # the name of the class. + return 'VertexAIMultiModalEmbeddings' + + +def _multimodal_dict_input_fn( + image_column: Optional[str], + video_column: Optional[str], + text_column: Optional[str], + batch: Sequence[dict[str, Any]]) -> list[VertexAIMultiModalInput]: + multimodal_inputs: list[VertexAIMultiModalInput] = [] + for item in batch: + img: Optional[VertexImage] = None + vid: Optional[VertexVideo] = None + text: Optional[Chunk] = None + if image_column: + img = item[image_column] + if video_column: + vid = item[video_column] + if text_column: + text = item[text_column] + multimodal_inputs.append( + VertexAIMultiModalInput(image=img, video=vid, contextual_text=text)) + return multimodal_inputs + + +def _multimodal_dict_output_fn( + image_column: Optional[str], + video_column: Optional[str], + text_column: Optional[str], + batch: Sequence[dict[str, Any]], + embeddings: Sequence[MultiModalEmbeddingResponse]) -> list[dict[str, Any]]: + results = [] + for batch_idx, item in enumerate(batch): + mm_embedding = embeddings[batch_idx] + if image_column: + item[image_column].embedding = mm_embedding.image_embedding + if video_column: + item[video_column].embeddings = mm_embedding.video_embeddings + if text_column: + item[text_column].embedding = Embedding( + dense_embedding=mm_embedding.text_embedding) + results.append(item) + return results + + +def _create_multimodal_dict_adapter( + image_column: Optional[str], + video_column: Optional[str], + text_column: Optional[str] +) -> EmbeddingTypeAdapter[dict[str, Any], dict[str, Any]]: + return EmbeddingTypeAdapter[dict[str, Any], dict[str, Any]]( + input_fn=cast( + Callable[[Sequence[dict[str, Any]]], list[str]], + functools.partial( + _multimodal_dict_input_fn, + image_column, + video_column, + text_column)), + output_fn=cast( + Callable[[Sequence[dict[str, Any]], Sequence[Any]], + list[dict[str, Any]]], + functools.partial( + _multimodal_dict_output_fn, + image_column, + video_column, + text_column))) + + +class VertexAIMultiModalEmbeddings(EmbeddingsManager): + def __init__( + self, + model_name: str, + image_column: Optional[str] = None, + video_column: Optional[str] = None, + text_column: Optional[str] = None, + dimension: Optional[int] = None, + project: Optional[str] = None, + location: Optional[str] = None, + credentials: Optional[Credentials] = None, + **kwargs): + """ + Embedding Config for Vertex AI Multi-Modal Embedding models following + https://cloud.google.com/vertex-ai/docs/generative-ai/embeddings/get-multimodal-embeddings # pylint: disable=line-too-long + Multi-Modal Embeddings are generated for a batch of image, video, and + string groupings using the Vertex AI API. Embeddings are returned in a list + for each image in the batch as MultiModalEmbeddingResponses. This + transform makes remote calls to the Vertex AI service and may incur costs + for use. + + Args: + model_name: The name of the Vertex AI Multi-Modal Embedding model. + image_column: The column containing image data to be embedded. This data + is expected to be formatted as VertexImage objects, containing a Vertex + Image object. + video_column: The column containing video data to be embedded. This data + is expected to be formatted as VertexVideo objects, containing a Vertex + Video object an a VideoSegmentConfig object. + text_column: The column containing text data to be embedded. This data is + expected to be formatted as Chunk objects, containing the string to be + embedded in the Chunk's content field. + dimension: The length of the embedding vector to generate. Must be one of + 128, 256, 512, or 1408. If not set, Vertex AI's default value is 1408. + If submitting video content, dimension *musst* be 1408. + project: The default GCP project for API calls. + location: The default location for API calls. + credentials: Custom credentials for API calls. + Defaults to environment credentials. + """ + self.model_name = model_name + self.project = project + self.location = location + self.credentials = credentials + self.kwargs = kwargs + if dimension is not None and dimension not in (128, 256, 512, 1408): + raise ValueError( + "dimension argument must be one of 128, 256, 512, or 1408") + self.dimension = dimension + if not image_column and not video_column and not text_column: + raise ValueError("at least one input column must be specified") + if video_column is not None and dimension != 1408: + raise ValueError( + "Vertex AI does not support custom dimensions for video input, want dimension = 1408, got ", + dimension) + self.type_adapter = _create_multimodal_dict_adapter( + image_column=image_column, + video_column=video_column, + text_column=text_column) + super().__init__(type_adapter=self.type_adapter, **kwargs) + + def get_model_handler(self) -> ModelHandler: + return _VertexAIMultiModalEmbeddingHandler( + model_name=self.model_name, + dimension=self.dimension, + project=self.project, + location=self.location, + credentials=self.credentials, + **self.kwargs) + + def get_ptransform_for_processing(self, **kwargs) -> beam.PTransform: + return RunInference( + model_handler=_MultiModalEmbeddingHandler(self), + inference_args=self.inference_args) diff --git a/sdks/python/apache_beam/ml/transforms/embeddings/vertex_ai_test.py b/sdks/python/apache_beam/ml/transforms/embeddings/vertex_ai_test.py index 1a47f81b665b..ba43ea325089 100644 --- a/sdks/python/apache_beam/ml/transforms/embeddings/vertex_ai_test.py +++ b/sdks/python/apache_beam/ml/transforms/embeddings/vertex_ai_test.py @@ -26,10 +26,18 @@ from apache_beam.ml.transforms.base import MLTransform try: + from apache_beam.ml.rag.types import Chunk + from apache_beam.ml.rag.types import Content + from apache_beam.ml.transforms.embeddings.vertex_ai import VertexAIMultiModalEmbeddings from apache_beam.ml.transforms.embeddings.vertex_ai import VertexAITextEmbeddings from apache_beam.ml.transforms.embeddings.vertex_ai import VertexAIImageEmbeddings + from apache_beam.ml.transforms.embeddings.vertex_ai import VertexImage + from apache_beam.ml.transforms.embeddings.vertex_ai import VertexVideo from vertexai.vision_models import Image + from vertexai.vision_models import Video + from vertexai.vision_models import VideoSegmentConfig except ImportError: + VertexAIMultiModalEmbeddings = None # type: ignore VertexAITextEmbeddings = None # type: ignore VertexAIImageEmbeddings = None # type: ignore @@ -286,5 +294,104 @@ def test_improper_dimension(self): dimension=127) +image_feature_column: str = "img_feature" +text_feature_column: str = "txt_feature" +video_feature_column: str = "vid_feature" + + +def _make_text_chunk(input: str) -> Chunk: + return Chunk(content=Content(text=input)) + + +@unittest.skipIf( + VertexAIMultiModalEmbeddings is None, + 'Vertex AI Python SDK is not installed.') +class VertexAIMultiModalEmbeddingsTest(unittest.TestCase): + def setUp(self) -> None: + self.artifact_location = tempfile.mkdtemp( + prefix='_vertex_ai_multi_modal_test') + self.gcs_artifact_location = os.path.join( + 'gs://temp-storage-for-perf-tests/vertex_ai_multi_modal', + uuid.uuid4().hex) + self.model_name = "multimodalembedding" + self.image_path = "gs://apache-beam-ml/testing/inputs/vertex_images/sunflowers/1008566138_6927679c8a.jpg" # pylint: disable=line-too-long + self.video_path = "gs://cloud-samples-data/vertex-ai-vision/highway_vehicles.mp4" # pylint: disable=line-too-long + self.video_segment_config = VideoSegmentConfig(end_offset_sec=1) + + def tearDown(self) -> None: + shutil.rmtree(self.artifact_location) + + def test_vertex_ai_multimodal_embedding_img_and_text(self): + embedding_config = VertexAIMultiModalEmbeddings( + model_name=self.model_name, + image_column=image_feature_column, + text_column=text_feature_column, + dimension=128, + project="apache-beam-testing", + location="us-central1") + with beam.Pipeline() as pipeline: + transformed_pcoll = ( + pipeline | "CreateData" >> beam.Create([{ + image_feature_column: VertexImage( + image_content=Image(gcs_uri=self.image_path)), + text_feature_column: _make_text_chunk("an image of sunflowers"), + }]) + | "MLTransform" >> MLTransform( + write_artifact_location=self.artifact_location).with_transform( + embedding_config)) + + def assert_element(element): + assert len(element[image_feature_column].embedding) == 128 + assert len( + element[text_feature_column].embedding.dense_embedding) == 128 + + _ = (transformed_pcoll | beam.Map(assert_element)) + + def test_vertex_ai_multimodal_embedding_video(self): + embedding_config = VertexAIMultiModalEmbeddings( + model_name=self.model_name, + video_column=video_feature_column, + dimension=1408, + project="apache-beam-testing", + location="us-central1") + with beam.Pipeline() as pipeline: + transformed_pcoll = ( + pipeline | "CreateData" >> beam.Create([{ + video_feature_column: VertexVideo( + video_content=Video(gcs_uri=self.video_path), + config=self.video_segment_config) + }]) + | "MLTransform" >> MLTransform( + write_artifact_location=self.artifact_location).with_transform( + embedding_config)) + + def assert_element(element): + # Videos are returned in VideoEmbedding objects, must unroll + # for each segment. + for segment in element[video_feature_column].embeddings: + assert len(segment.embedding) == 1408 + + _ = (transformed_pcoll | beam.Map(assert_element)) + + def test_improper_dimension(self): + with self.assertRaises(ValueError): + _ = VertexAIMultiModalEmbeddings( + model_name=self.model_name, + image_column="fake_img_column", + dimension=127) + + def test_missing_columns(self): + with self.assertRaises(ValueError): + _ = VertexAIMultiModalEmbeddings( + model_name=self.model_name, dimension=128) + + def test_improper_video_dimension(self): + with self.assertRaises(ValueError): + _ = VertexAIMultiModalEmbeddings( + model_name=self.model_name, + video_column=video_feature_column, + dimension=128) + + if __name__ == '__main__': unittest.main() diff --git a/sdks/python/apache_beam/options/pipeline_options.py b/sdks/python/apache_beam/options/pipeline_options.py index c3fbdf7c79c8..6595d683911b 100644 --- a/sdks/python/apache_beam/options/pipeline_options.py +++ b/sdks/python/apache_beam/options/pipeline_options.py @@ -20,6 +20,7 @@ # pytype: skip-file import argparse +import difflib import json import logging import os @@ -449,11 +450,30 @@ def from_dictionary(cls, options): return cls(flags) + @staticmethod + def _warn_on_unknown_options(unknown_args, parser): + if not unknown_args: + return + + all_known_options = [ + opt for action in parser._actions for opt in action.option_strings + ] + + for arg in unknown_args: + msg = f"Unparseable argument: {arg}" + if arg.startswith('--'): + arg_name = arg.split('=', 1)[0] + suggestions = difflib.get_close_matches(arg_name, all_known_options) + if suggestions: + msg += f". Did you mean '{suggestions[0]}'?'" + _LOGGER.warning(msg) + def get_all_options( self, drop_default=False, add_extra_args_fn: Optional[Callable[[_BeamArgumentParser], None]] = None, - retain_unknown_options=False) -> Dict[str, Any]: + retain_unknown_options=False, + display_warnings=False) -> Dict[str, Any]: """Returns a dictionary of all defined arguments. Returns a dictionary of all defined arguments (arguments that are defined in @@ -485,12 +505,11 @@ def get_all_options( add_extra_args_fn(parser) known_args, unknown_args = parser.parse_known_args(self._flags) - if retain_unknown_options: - if unknown_args: - _LOGGER.warning( - 'Unknown pipeline options received: %s. Ignore if flags are ' - 'used for internal purposes.' % (','.join(unknown_args))) + if display_warnings: + self._warn_on_unknown_options(unknown_args, parser) + + if retain_unknown_options: seen = set() def add_new_arg(arg, **kwargs): @@ -1021,9 +1040,10 @@ def _add_argparse_args(cls, parser): 'updating-a-pipeline') parser.add_argument( '--enable_streaming_engine', - default=False, + default=True, action='store_true', - help='Enable Windmill Service for this Dataflow job. ') + help='Deprecated. All Python streaming pipelines on Dataflow' + 'use Streaming Engine.') parser.add_argument( '--dataflow_kms_key', default=None, @@ -1455,6 +1475,15 @@ def _add_argparse_args(cls, parser): 'responsible for executing the user code and communicating with ' 'the runner. Depending on the runner, there may be more than one ' 'SDK Harness process running on the same worker node.')) + parser.add_argument( + '--element_processing_timeout_minutes', + type=int, + default=None, + help=( + 'The time limit (in minutes) for any PTransform to finish ' + 'processing a single element. If exceeded, the SDK worker ' + 'process self-terminates and processing may be restarted ' + 'by a runner.')) def validate(self, validator): errors = [] @@ -1609,7 +1638,7 @@ def _add_argparse_args(cls, parser): help=( 'Chooses which pickle library to use. Options are dill, ' 'cloudpickle or default.'), - choices=['cloudpickle', 'default', 'dill']) + choices=['cloudpickle', 'default', 'dill', 'dill_unsafe']) parser.add_argument( '--save_main_session', default=False, @@ -1691,6 +1720,7 @@ def _add_argparse_args(cls, parser): def validate(self, validator): errors = [] errors.extend(validator.validate_container_prebuilding_options(self)) + errors.extend(validator.validate_pickle_library(self)) return errors @@ -1949,6 +1979,13 @@ def _add_argparse_args(cls, parser): help=( 'Controls the log level in Prism. Values can be "debug", "info", ' '"warn", and "error". Default log level is "info".')) + parser.add_argument( + '--prism_log_kind', + default="console", + choices=["dev", "json", "text", "console"], + help=( + 'Controls the log format in Prism. Values can be "dev", "json", ' + '"text", and "console". Default log format is "console".')) class TestOptions(PipelineOptions): diff --git a/sdks/python/apache_beam/options/pipeline_options_test.py b/sdks/python/apache_beam/options/pipeline_options_test.py index 06270d4cd310..cd6cce204b78 100644 --- a/sdks/python/apache_beam/options/pipeline_options_test.py +++ b/sdks/python/apache_beam/options/pipeline_options_test.py @@ -405,10 +405,18 @@ def test_experiments(self): self.assertEqual(options.get_all_options()['experiments'], None) def test_worker_options(self): - options = PipelineOptions(['--machine_type', 'abc', '--disk_type', 'def']) + options = PipelineOptions([ + '--machine_type', + 'abc', + '--disk_type', + 'def', + '--element_processing_timeout_minutes', + '10', + ]) worker_options = options.view_as(WorkerOptions) self.assertEqual(worker_options.machine_type, 'abc') self.assertEqual(worker_options.disk_type, 'def') + self.assertEqual(worker_options.element_processing_timeout_minutes, 10) options = PipelineOptions( ['--worker_machine_type', 'abc', '--worker_disk_type', 'def']) diff --git a/sdks/python/apache_beam/options/pipeline_options_validator.py b/sdks/python/apache_beam/options/pipeline_options_validator.py index ebe9c8f223ce..0217363bc9b8 100644 --- a/sdks/python/apache_beam/options/pipeline_options_validator.py +++ b/sdks/python/apache_beam/options/pipeline_options_validator.py @@ -119,6 +119,15 @@ class PipelineOptionsValidator(object): ERR_REPEATABLE_OPTIONS_NOT_SET_AS_LIST = ( '(%s) is a string. Programmatically set PipelineOptions like (%s) ' 'options need to be specified as a list.') + ERR_DILL_NOT_INSTALLED = ( + 'Option pickle_library=dill requires dill==0.3.1.1. Install apache-beam ' + 'with the dill extra e.g. apache-beam[gcp, dill]. Dill package was not ' + 'found') + ERR_UNSAFE_DILL_VERSION = ( + 'Dill version 0.3.1.1 is required when using pickle_library=dill. Other ' + 'versions of dill are untested with Apache Beam. To install the supported' + ' dill version instal apache-beam[dill] extra. To use an unsupported ' + 'dill version, use pickle_library=dill_unsafe. %s') # GCS path specific patterns. GCS_URI = '(?P[^:]+)://(?P[^/]+)(/(?P.*))?' @@ -196,6 +205,25 @@ def validate_gcs_path(self, view, arg_name): return self._validate_error(self.ERR_INVALID_GCS_OBJECT, arg, arg_name) return [] + def validate_pickle_library(self, view): + """Validates the pickle_library option.""" + if view.pickle_library == 'default' or view.pickle_library == 'cloudpickle': + return [] + + if view.pickle_library == 'dill_unsafe': + return [] + + if view.pickle_library == 'dill': + try: + import dill + if dill.__version__ != "0.3.1.1": + return self._validate_error( + self.ERR_UNSAFE_DILL_VERSION, + f"Dill version found {dill.__version__}") + except ImportError: + return self._validate_error(self.ERR_DILL_NOT_INSTALLED) + return [] + def validate_cloud_options(self, view): """Validates job_name and project arguments.""" errors = [] diff --git a/sdks/python/apache_beam/options/pipeline_options_validator_test.py b/sdks/python/apache_beam/options/pipeline_options_validator_test.py index 56f305a01b74..8206d45dcf03 100644 --- a/sdks/python/apache_beam/options/pipeline_options_validator_test.py +++ b/sdks/python/apache_beam/options/pipeline_options_validator_test.py @@ -22,6 +22,7 @@ import logging import unittest +import pytest from hamcrest import assert_that from hamcrest import contains_string from hamcrest import only_contains @@ -244,6 +245,48 @@ def test_is_service_runner(self, runner, options, expected): validator = PipelineOptionsValidator(PipelineOptions(options), runner) self.assertEqual(validator.is_service_runner(), expected) + def test_pickle_library_dill_not_installed_returns_error(self): + runner = MockRunners.OtherRunner() + # Remove default region for this test. + options = PipelineOptions(['--pickle_library=dill']) + validator = PipelineOptionsValidator(options, runner) + errors = validator.validate() + self.assertEqual(len(errors), 1, errors) + self.assertIn("Option pickle_library=dill requires dill", errors[0]) + + @pytest.mark.uses_dill + def test_pickle_library_dill_installed_returns_no_error(self): + pytest.importorskip("dill") + runner = MockRunners.OtherRunner() + # Remove default region for this test. + options = PipelineOptions(['--pickle_library=dill']) + validator = PipelineOptionsValidator(options, runner) + errors = validator.validate() + self.assertEqual(len(errors), 0, errors) + + @pytest.mark.uses_dill + def test_pickle_library_dill_installed_returns_wrong_version(self): + pytest.importorskip("dill") + with unittest.mock.patch('dill.__version__', '0.3.6'): + runner = MockRunners.OtherRunner() + # Remove default region for this test. + options = PipelineOptions(['--pickle_library=dill']) + validator = PipelineOptionsValidator(options, runner) + errors = validator.validate() + self.assertEqual(len(errors), 1, errors) + self.assertIn("Dill version 0.3.1.1 is required when using ", errors[0]) + + @pytest.mark.uses_dill + def test_pickle_library_dill_unsafe_no_error(self): + pytest.importorskip("dill") + with unittest.mock.patch('dill.__version__', '0.3.6'): + runner = MockRunners.OtherRunner() + # Remove default region for this test. + options = PipelineOptions(['--pickle_library=dill_unsafe']) + validator = PipelineOptionsValidator(options, runner) + errors = validator.validate() + self.assertEqual(len(errors), 0, errors) + def test_dataflow_job_file_and_template_location_mutually_exclusive(self): runner = MockRunners.OtherRunner() options = PipelineOptions( diff --git a/sdks/python/apache_beam/pipeline.py b/sdks/python/apache_beam/pipeline.py index 269b4acdc21d..884ca124b0f6 100644 --- a/sdks/python/apache_beam/pipeline.py +++ b/sdks/python/apache_beam/pipeline.py @@ -76,6 +76,7 @@ from google.protobuf import message from apache_beam import pvalue +from apache_beam.coders import typecoders from apache_beam.internal import pickler from apache_beam.io.filesystems import FileSystems from apache_beam.options.pipeline_options import CrossLanguageOptions @@ -83,6 +84,7 @@ from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.options.pipeline_options import SetupOptions from apache_beam.options.pipeline_options import StandardOptions +from apache_beam.options.pipeline_options import StreamingOptions from apache_beam.options.pipeline_options import TypeOptions from apache_beam.options.pipeline_options_validator import PipelineOptionsValidator from apache_beam.portability import common_urns @@ -115,11 +117,11 @@ class Pipeline(HasDisplayData): - """A pipeline object that manages a DAG of - :class:`~apache_beam.transforms.ptransform.PTransform` s + """A pipeline object that manages a DAG of + :class:`~apache_beam.transforms.ptransform.PTransform` s and their :class:`~apache_beam.pvalue.PValue` s. - Conceptually the :class:`~apache_beam.transforms.ptransform.PTransform` s are + Conceptually the :class:`~apache_beam.transforms.ptransform.PTransform` s are the DAG's nodes and the :class:`~apache_beam.pvalue.PValue` s are the edges. All the transforms applied to the pipeline must have distinct full labels. @@ -229,6 +231,9 @@ def __init__( raise ValueError( 'Pipeline has validations errors: \n' + '\n'.join(errors)) + typecoders.registry.update_compatibility_version = self._options.view_as( + StreamingOptions).update_compatibility_version + # set default experiments for portable runners # (needs to occur prior to pipeline construction) if runner.is_fnapi_compatible(): @@ -575,6 +580,10 @@ def run(self, test_runner_api='AUTO'): # type: (Union[bool, str]) -> PipelineResult """Runs the pipeline. Returns whatever our runner returns after running.""" + # All pipeline options are finalized at this point. + # Call get_all_options to print warnings on invalid options. + self.options.get_all_options( + retain_unknown_options=True, display_warnings=True) for error_handler in self._error_handlers: error_handler.verify_closed() @@ -722,6 +731,10 @@ def apply( return self.apply( transform.transform, pvalueish, label or transform.label) + if not label and isinstance(transform, ptransform._PTransformFnPTransform): + # This must be set before label is inspected. + transform.set_options(self._options) + if not isinstance(transform, ptransform.PTransform): raise TypeError("Expected a PTransform object, got %s" % transform) diff --git a/sdks/python/apache_beam/pipeline_test.py b/sdks/python/apache_beam/pipeline_test.py index dc0d9a7cc58f..6e439aff5848 100644 --- a/sdks/python/apache_beam/pipeline_test.py +++ b/sdks/python/apache_beam/pipeline_test.py @@ -177,7 +177,9 @@ def expand(self, pcoll): _ = pipeline | ParentTransform() | beam.Map(lambda x: x + 1) @mock.patch('logging.info') + @pytest.mark.uses_dill def test_runner_overrides_default_pickler(self, mock_info): + pytest.importorskip("dill") with mock.patch.object(PipelineRunner, 'default_pickle_library_override') as mock_fn: mock_fn.return_value = 'dill' diff --git a/sdks/python/apache_beam/pvalue.py b/sdks/python/apache_beam/pvalue.py index cee3b8f2bca2..3865af184b61 100644 --- a/sdks/python/apache_beam/pvalue.py +++ b/sdks/python/apache_beam/pvalue.py @@ -33,6 +33,7 @@ from typing import Dict from typing import Generic from typing import Iterator +from typing import NamedTuple from typing import Optional from typing import Sequence from typing import TypeVar @@ -675,11 +676,15 @@ def __hash__(self): return hash(self.__dict__.items()) def __eq__(self, other): + if type(self) == type(other): + other_dict = other.__dict__ + elif type(other) == type(NamedTuple): + other_dict = other._asdict() + else: + return False return ( - type(self) == type(other) and - len(self.__dict__) == len(other.__dict__) and all( - s == o - for s, o in zip(self.__dict__.items(), other.__dict__.items()))) + len(self.__dict__) == len(other_dict) and + all(s == o for s, o in zip(self.__dict__.items(), other_dict.items()))) def __reduce__(self): return _make_Row, tuple(self.__dict__.items()) diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py index 19302923b1fb..9e339e289fff 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py @@ -378,6 +378,14 @@ def run_pipeline(self, pipeline, options, pipeline_proto=None): # contain any added PTransforms. pipeline.replace_all(DataflowRunner._PTRANSFORM_OVERRIDES) + # Apply DataflowRunner-specific overrides (e.g., streaming PubSub + # optimizations) + from apache_beam.runners.dataflow.ptransform_overrides import ( + get_dataflow_transform_overrides) + dataflow_overrides = get_dataflow_transform_overrides(options) + if dataflow_overrides: + pipeline.replace_all(dataflow_overrides) + if options.view_as(DebugOptions).lookup_experiment('use_legacy_bq_sink'): warnings.warn( "Native sinks no longer implemented; " @@ -633,23 +641,8 @@ def _check_and_add_missing_streaming_options(options): # Runner v2 only supports using streaming engine (aka windmill service) if options.view_as(StandardOptions).streaming: debug_options = options.view_as(DebugOptions) - google_cloud_options = options.view_as(GoogleCloudOptions) - if (not google_cloud_options.enable_streaming_engine and - (debug_options.lookup_experiment("enable_windmill_service") or - debug_options.lookup_experiment("enable_streaming_engine"))): - raise ValueError( - """Streaming engine both disabled and enabled: - --enable_streaming_engine flag is not set, but - enable_windmill_service and/or enable_streaming_engine experiments - are present. It is recommended you only set the - --enable_streaming_engine flag.""") - - # Ensure that if we detected a streaming pipeline that streaming specific - # options and experiments. - options.view_as(StandardOptions).streaming = True - google_cloud_options.enable_streaming_engine = True - debug_options.add_experiment("enable_streaming_engine") - debug_options.add_experiment("enable_windmill_service") + debug_options.add_experiment('enable_streaming_engine') + debug_options.add_experiment('enable_windmill_service') def _is_runner_v2_disabled(options): diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py index bb9132bdb96e..178a75ec41d9 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py @@ -421,10 +421,9 @@ def test_min_cpu_platform_flag_is_propagated_to_experiments(self): 'min_cpu_platform=Intel Haswell', remote_runner.job.options.view_as(DebugOptions).experiments) - def test_streaming_engine_flag_adds_windmill_experiments(self): + def test_streaming_adds_windmill_experiments(self): remote_runner = DataflowRunner() self.default_properties.append('--streaming') - self.default_properties.append('--enable_streaming_engine') self.default_properties.append('--experiment=some_other_experiment') with Pipeline(remote_runner, PipelineOptions(self.default_properties)) as p: diff --git a/sdks/python/apache_beam/runners/dataflow/internal/names.py b/sdks/python/apache_beam/runners/dataflow/internal/names.py index dfe8e1e1beef..cf9bf6208dc5 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/names.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/names.py @@ -34,6 +34,6 @@ # Unreleased sdks use container image tag specified below. # Update this tag whenever there is a change that # requires changes to SDK harness container or SDK harness launcher. -BEAM_DEV_SDK_CONTAINER_TAG = 'beam-master-20250728' +BEAM_DEV_SDK_CONTAINER_TAG = 'beam-master-20250827' DATAFLOW_CONTAINER_IMAGE_REPOSITORY = 'gcr.io/cloud-dataflow/v1beta3' diff --git a/sdks/python/apache_beam/runners/dataflow/ptransform_overrides.py b/sdks/python/apache_beam/runners/dataflow/ptransform_overrides.py index 8004762f5eec..4e75f202c098 100644 --- a/sdks/python/apache_beam/runners/dataflow/ptransform_overrides.py +++ b/sdks/python/apache_beam/runners/dataflow/ptransform_overrides.py @@ -19,9 +19,70 @@ # pytype: skip-file +from apache_beam.options.pipeline_options import StandardOptions from apache_beam.pipeline import PTransformOverride +class StreamingPubSubWriteDoFnOverride(PTransformOverride): + """Override ParDo(_PubSubWriteDoFn) for streaming mode in DataflowRunner. + + This override specifically targets the final ParDo step in WriteToPubSub + and replaces it with Write(sink) for streaming optimization. + """ + def matches(self, applied_ptransform): + from apache_beam.transforms import ParDo + from apache_beam.io.gcp.pubsub import _PubSubWriteDoFn + + if not isinstance(applied_ptransform.transform, ParDo): + return False + + # Check if this ParDo uses _PubSubWriteDoFn + dofn = applied_ptransform.transform.dofn + return isinstance(dofn, _PubSubWriteDoFn) + + def get_replacement_transform_for_applied_ptransform( + self, applied_ptransform): + from apache_beam.io.iobase import Write + + # Get the WriteToPubSub transform from the DoFn constructor parameter + dofn = applied_ptransform.transform.dofn + + # The DoFn was initialized with the WriteToPubSub transform + # We need to reconstruct the sink from the DoFn's stored properties + if hasattr(dofn, 'project') and hasattr(dofn, 'short_topic_name'): + from apache_beam.io.gcp.pubsub import _PubSubSink + + # Create a sink with the same properties as the original + topic = f"projects/{dofn.project}/topics/{dofn.short_topic_name}" + sink = _PubSubSink( + topic=topic, + id_label=getattr(dofn, 'id_label', None), + timestamp_attribute=getattr(dofn, 'timestamp_attribute', None)) + return Write(sink) + else: + # Fallback: return the original transform if we can't reconstruct it + return applied_ptransform.transform + + +def get_dataflow_transform_overrides(pipeline_options): + """Returns DataflowRunner-specific transform overrides. + + Args: + pipeline_options: Pipeline options to determine which overrides to apply. + + Returns: + List of PTransformOverride objects for DataflowRunner. + """ + overrides = [] + + # Only add streaming-specific overrides when in streaming mode + if pipeline_options.view_as(StandardOptions).streaming: + # Add PubSub ParDo streaming override that targets only the final step + overrides.append(StreamingPubSubWriteDoFnOverride()) + + return overrides + + class NativeReadPTransformOverride(PTransformOverride): """A ``PTransformOverride`` for ``Read`` using native sources. @@ -54,7 +115,7 @@ def expand(self, pbegin): return pvalue.PCollection.from_(pbegin) # Use the source's coder type hint as this replacement's output. Otherwise, - # the typing information is not properly forwarded to the DataflowRunner and - # will choose the incorrect coder for this transform. + # the typing information is not properly forwarded to the DataflowRunner + # and will choose the incorrect coder for this transform. return Read(ptransform.source).with_output_types( ptransform.source.coder.to_type_hint()) diff --git a/sdks/python/apache_beam/runners/direct/direct_runner.py b/sdks/python/apache_beam/runners/direct/direct_runner.py index a629c12a058d..68add6ea3c1a 100644 --- a/sdks/python/apache_beam/runners/direct/direct_runner.py +++ b/sdks/python/apache_beam/runners/direct/direct_runner.py @@ -25,7 +25,6 @@ import itertools import logging -import time import typing from google.protobuf import wrappers_pb2 @@ -137,6 +136,14 @@ def accept(self, pipeline, is_interactive): self.supported_by_prism_runner = False else: pipeline.visit(self) + # Avoid circular import + from apache_beam.pipeline import ExternalTransformFinder + if ExternalTransformFinder.contains_external_transforms(pipeline): + # TODO(https://github.com/apache/beam/issues/33623): Prism currently + # seems to not be able to consistently bring up external transforms. + # It does sometimes, but at volume suites start to fail. We will try + # to enable this in a future release. + self.supported_by_prism_runner = False return self.supported_by_prism_runner def visit_transform(self, applied_ptransform): @@ -145,12 +152,6 @@ def visit_transform(self, applied_ptransform): # being used. if isinstance(transform, TestStream): self.supported_by_prism_runner = False - if isinstance(transform, beam.ExternalTransform): - # TODO(https://github.com/apache/beam/issues/33623): Prism currently - # seems to not be able to consistently bring up external transforms. - # It does sometimes, but at volume suites start to fail. We will try - # to enable this in a future release. - self.supported_by_prism_runner = False if isinstance(transform, beam.ParDo): dofn = transform.dofn # TODO(https://github.com/apache/beam/issues/33623): Prism currently @@ -184,6 +185,13 @@ def visit_transform(self, applied_ptransform): for state in state_specs: if isinstance(state, userstate.CombiningValueStateSpec): self.supported_by_prism_runner = False + if isinstance( + dofn, + beam.transforms.combiners._PartialGroupByKeyCombiningValues): + if len(transform.side_inputs) > 0: + # Prism doesn't support side input combiners (this is within spec) + self.supported_by_prism_runner = False + # TODO(https://github.com/apache/beam/issues/33623): Prism seems to # not handle session windows correctly. Examples are: # util_test.py::ReshuffleTest::test_reshuffle_window_fn_preserved @@ -195,21 +203,9 @@ def visit_transform(self, applied_ptransform): # Use BundleBasedDirectRunner if other runners are missing needed features. runner = BundleBasedDirectRunner() - # Check whether all transforms used in the pipeline are supported by the - # FnApiRunner, and the pipeline was not meant to be run as streaming. - if _FnApiRunnerSupportVisitor().accept(pipeline): - from apache_beam.portability.api import beam_provision_api_pb2 - from apache_beam.runners.portability.fn_api_runner import fn_runner - from apache_beam.runners.portability.portable_runner import JobServiceHandle - all_options = options.get_all_options() - encoded_options = JobServiceHandle.encode_pipeline_options(all_options) - provision_info = fn_runner.ExtendedProvisionInfo( - beam_provision_api_pb2.ProvisionInfo( - pipeline_options=encoded_options)) - runner = fn_runner.FnApiRunner(provision_info=provision_info) # Check whether all transforms used in the pipeline are supported by the # PrismRunner - elif _PrismRunnerSupportVisitor().accept(pipeline, self._is_interactive): + if _PrismRunnerSupportVisitor().accept(pipeline, self._is_interactive): _LOGGER.info('Running pipeline with PrismRunner.') from apache_beam.runners.portability import prism_runner runner = prism_runner.PrismRunner() @@ -233,6 +229,19 @@ def visit_transform(self, applied_ptransform): _LOGGER.info('Falling back to DirectRunner') runner = BundleBasedDirectRunner() + # Check whether all transforms used in the pipeline are supported by the + # FnApiRunner, and the pipeline was not meant to be run as streaming. + if _FnApiRunnerSupportVisitor().accept(pipeline): + from apache_beam.portability.api import beam_provision_api_pb2 + from apache_beam.runners.portability.fn_api_runner import fn_runner + from apache_beam.runners.portability.portable_runner import JobServiceHandle + all_options = options.get_all_options() + encoded_options = JobServiceHandle.encode_pipeline_options(all_options) + provision_info = fn_runner.ExtendedProvisionInfo( + beam_provision_api_pb2.ProvisionInfo( + pipeline_options=encoded_options)) + runner = fn_runner.FnApiRunner(provision_info=provision_info) + return runner.run_pipeline(pipeline, options) @@ -511,59 +520,6 @@ def expand(self, pvalue): return PCollection(self.pipeline, is_bounded=self._source.is_bounded()) -class _DirectWriteToPubSubFn(DoFn): - BUFFER_SIZE_ELEMENTS = 100 - FLUSH_TIMEOUT_SECS = BUFFER_SIZE_ELEMENTS * 0.5 - - def __init__(self, transform): - self.project = transform.project - self.short_topic_name = transform.topic_name - self.id_label = transform.id_label - self.timestamp_attribute = transform.timestamp_attribute - self.with_attributes = transform.with_attributes - - # TODO(https://github.com/apache/beam/issues/18939): Add support for - # id_label and timestamp_attribute. - if transform.id_label: - raise NotImplementedError( - 'DirectRunner: id_label is not supported for ' - 'PubSub writes') - if transform.timestamp_attribute: - raise NotImplementedError( - 'DirectRunner: timestamp_attribute is not ' - 'supported for PubSub writes') - - def start_bundle(self): - self._buffer = [] - - def process(self, elem): - self._buffer.append(elem) - if len(self._buffer) >= self.BUFFER_SIZE_ELEMENTS: - self._flush() - - def finish_bundle(self): - self._flush() - - def _flush(self): - from google.cloud import pubsub - pub_client = pubsub.PublisherClient() - topic = pub_client.topic_path(self.project, self.short_topic_name) - - if self.with_attributes: - futures = [ - pub_client.publish(topic, elem.data, **elem.attributes) - for elem in self._buffer - ] - else: - futures = [pub_client.publish(topic, elem) for elem in self._buffer] - - timer_start = time.time() - for future in futures: - remaining = self.FLUSH_TIMEOUT_SECS - (time.time() - timer_start) - future.result(remaining) - self._buffer = [] - - def _get_pubsub_transform_overrides(pipeline_options): from apache_beam.io.gcp import pubsub as beam_pubsub from apache_beam.pipeline import PTransformOverride @@ -581,19 +537,9 @@ def get_replacement_transform_for_applied_ptransform( '(use the --streaming flag).') return _DirectReadFromPubSub(applied_ptransform.transform._source) - class WriteToPubSubOverride(PTransformOverride): - def matches(self, applied_ptransform): - return isinstance(applied_ptransform.transform, beam_pubsub.WriteToPubSub) - - def get_replacement_transform_for_applied_ptransform( - self, applied_ptransform): - if not pipeline_options.view_as(StandardOptions).streaming: - raise Exception( - 'PubSub I/O is only available in streaming mode ' - '(use the --streaming flag).') - return beam.ParDo(_DirectWriteToPubSubFn(applied_ptransform.transform)) - - return [ReadFromPubSubOverride(), WriteToPubSubOverride()] + # WriteToPubSub no longer needs an override - it works by default for both + # batch and streaming + return [ReadFromPubSubOverride()] class BundleBasedDirectRunner(PipelineRunner): diff --git a/sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/apache_beam_jupyterlab_sidepanel/yaml_parse_utils.py b/sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/apache_beam_jupyterlab_sidepanel/yaml_parse_utils.py new file mode 100644 index 000000000000..aebca7b85d65 --- /dev/null +++ b/sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/apache_beam_jupyterlab_sidepanel/yaml_parse_utils.py @@ -0,0 +1,176 @@ +# Licensed 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. + +import dataclasses +import json +from dataclasses import dataclass +from typing import Any +from typing import Dict +from typing import List +from typing import TypedDict + +import yaml + +import apache_beam as beam +from apache_beam.yaml.main import build_pipeline_components_from_yaml + +# ======================== Type Definitions ======================== + + +@dataclass +class NodeData: + id: str + label: str + type: str = "" + + def __post_init__(self): + # Ensure ID is not empty + if not self.id: + raise ValueError("Node ID cannot be empty") + + +@dataclass +class EdgeData: + source: str + target: str + label: str = "" + + def __post_init__(self): + if not self.source or not self.target: + raise ValueError("Edge source and target cannot be empty") + + +class FlowGraph(TypedDict): + nodes: List[Dict[str, Any]] + edges: List[Dict[str, Any]] + + +# ======================== Main Function ======================== + + +def parse_beam_yaml(yaml_str: str, isDryRunMode: bool = False) -> str: + """ + Parse Beam YAML and convert to flow graph data structure + + Args: + yaml_str: Input YAML string + + Returns: + Standardized response format: + - Success: {'status': 'success', 'data': {...}, 'error': None} + - Failure: {'status': 'error', 'data': None, 'error': 'message'} + """ + # Phase 1: YAML Parsing + try: + parsed_yaml = yaml.safe_load(yaml_str) + if not parsed_yaml or 'pipeline' not in parsed_yaml: + return build_error_response( + "Invalid YAML structure: missing 'pipeline' section") + except yaml.YAMLError as e: + return build_error_response(f"YAML parsing error: {str(e)}") + + # Phase 2: Pipeline Validation + try: + options, constructor = build_pipeline_components_from_yaml( + yaml_str, + [], + validate_schema='per_transform' + ) + if isDryRunMode: + with beam.Pipeline(options=options) as p: + constructor(p) + except Exception as e: + return build_error_response(f"Pipeline validation failed: {str(e)}") + + # Phase 3: Graph Construction + try: + pipeline = parsed_yaml['pipeline'] + transforms = pipeline.get('transforms', []) + + nodes: List[NodeData] = [] + edges: List[EdgeData] = [] + + nodes.append(NodeData(id='0', label='Input', type='input')) + nodes.append(NodeData(id='1', label='Output', type='output')) + + # Process transform nodes + for idx, transform in enumerate(transforms): + if not isinstance(transform, dict): + continue + + payload = {k: v for k, v in transform.items() if k not in {"type"}} + + node_id = f"t{idx}" + node_data = NodeData( + id=node_id, + label=transform.get('type', 'unnamed'), + type='default', + **payload) + nodes.append(node_data) + + # Create connections between nodes + if idx > 0: + edges.append( + EdgeData(source=f"t{idx-1}", target=node_id, label='chain')) + + if transforms: + edges.append(EdgeData(source='0', target='t0', label='start')) + edges.append(EdgeData(source=node_id, target='1', label='stop')) + + def to_dict(node): + if hasattr(node, '__dataclass_fields__'): + return dataclasses.asdict(node) + return node + + nodes_serializable = [to_dict(n) for n in nodes] + + return build_success_response( + nodes=nodes_serializable, edges=[dataclasses.asdict(e) for e in edges]) + + except Exception as e: + return build_error_response(f"Graph construction failed: {str(e)}") + + +# ======================== Utility Functions ======================== + + +def build_success_response( + nodes: List[Dict[str, Any]], edges: List[Dict[str, Any]]) -> str: + """Build success response""" + return json.dumps({'data': {'nodes': nodes, 'edges': edges}, 'error': None}) + + +def build_error_response(error_msg: str) -> str: + """Build error response""" + return json.dumps({'data': None, 'error': error_msg}) + + +if __name__ == "__main__": + # Example usage + example_yaml = """ +pipeline: + transforms: + - type: ReadFromCsv + name: A + config: + path: /path/to/input*.csv + - type: WriteToJson + name: B + config: + path: /path/to/output.json + input: ReadFromCsv + - type: Join + input: [A, B] + """ + + response = parse_beam_yaml(example_yaml, isDryRunMode=False) + print(response) diff --git a/sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/package.json b/sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/package.json index 8b51461f6cd4..eef3fcaa80f4 100644 --- a/sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/package.json +++ b/sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/package.json @@ -47,27 +47,37 @@ "@jupyterlab/launcher": "^4.3.6", "@jupyterlab/mainmenu": "^4.3.6", "@lumino/widgets": "^2.2.1", + "@monaco-editor/react": "^4.7.0", "@rmwc/base": "^14.0.0", "@rmwc/button": "^8.0.6", + "@rmwc/card": "^14.3.5", "@rmwc/data-table": "^8.0.6", "@rmwc/dialog": "^8.0.6", "@rmwc/drawer": "^8.0.6", "@rmwc/fab": "^8.0.6", + "@rmwc/grid": "^14.3.5", "@rmwc/list": "^8.0.6", "@rmwc/ripple": "^14.0.0", "@rmwc/textfield": "^8.0.6", "@rmwc/tooltip": "^8.0.6", "@rmwc/top-app-bar": "^8.0.6", + "@rmwc/touch-target": "^14.3.5", + "@xyflow/react": "^12.8.2", + "dagre": "^0.8.5", + "lodash": "^4.17.21", "material-design-icons": "^3.0.1", "react": "^18.2.0", - "react-dom": "^18.2.0" + "react-dom": "^18.2.0", + "react-split": "^2.0.14" }, "devDependencies": { "@jupyterlab/builder": "^4.3.6", "@testing-library/dom": "^9.3.0", "@testing-library/jest-dom": "^6.1.4", "@testing-library/react": "^14.0.0", + "@types/dagre": "^0.7.53", "@types/jest": "^29.5.14", + "@types/lodash": "^4.17.20", "@types/react": "^18.2.0", "@types/react-dom": "^18.2.0", "@typescript-eslint/eslint-plugin": "^7.3.1", @@ -97,5 +107,6 @@ "test": "jest", "resolutions": { "@types/react": "^18.2.0" - } -} + }, + "packageManager": "yarn@1.22.22+sha512.a6b2f7906b721bba3d67d4aff083df04dad64c399707841b7acf00f6b133b7ac24255f2652fa22ae3534329dc6180534e98d17432037ff6fd140556e2bb3137e" +} \ No newline at end of file diff --git a/sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/src/SidePanel.ts b/sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/src/SidePanel.ts index fb86b0a53fdf..d8f19c278843 100644 --- a/sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/src/SidePanel.ts +++ b/sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/src/SidePanel.ts @@ -58,14 +58,17 @@ export class SidePanel extends BoxPanel { const sessionModelItr = manager.sessions.running(); const firstModel = sessionModelItr.next(); let onlyOneUniqueKernelExists = true; - if (firstModel === undefined) { - // There is zero unique running kernel. + + if (firstModel.done) { + // No Running kernel onlyOneUniqueKernelExists = false; } else { + // firstModel.value is the first session let sessionModel = sessionModelItr.next(); - while (sessionModel !== undefined) { + + while (!sessionModel.done) { + // Check if there is more than one unique kernel if (sessionModel.value.kernel.id !== firstModel.value.kernel.id) { - // There is more than one unique running kernel. onlyOneUniqueKernelExists = false; break; } diff --git a/sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/src/index.ts b/sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/src/index.ts index 3f2b02d11b53..92a1ea3cdbbe 100644 --- a/sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/src/index.ts +++ b/sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/src/index.ts @@ -28,12 +28,15 @@ import { SidePanel } from './SidePanel'; import { InteractiveInspectorWidget } from './inspector/InteractiveInspectorWidget'; +import { YamlWidget } from './yaml/YamlWidget'; namespace CommandIDs { export const open_inspector = 'apache-beam-jupyterlab-sidepanel:open_inspector'; export const open_clusters_panel = 'apache-beam-jupyterlab-sidepanel:open_clusters_panel'; + export const open_yaml_editor = + 'apache-beam-jupyterlab-sidepanel:open_yaml_editor'; } /** @@ -67,6 +70,7 @@ function activate( const category = 'Interactive Beam'; const inspectorCommandLabel = 'Open Inspector'; const clustersCommandLabel = 'Manage Clusters'; + const yamlCommandLabel = 'Edit YAML Pipeline'; const { commands, shell, serviceManager } = app; async function createInspectorPanel(): Promise { @@ -105,6 +109,24 @@ function activate( return panel; } + async function createYamlPanel(): Promise { + const sessionContext = new SessionContext({ + sessionManager: serviceManager.sessions, + specsManager: serviceManager.kernelspecs, + name: 'Interactive Beam YAML Session' + }); + const yamlEditor = new YamlWidget(sessionContext); + const panel = new SidePanel( + serviceManager, + rendermime, + sessionContext, + 'Interactive Beam YAML Editor', + yamlEditor + ); + activatePanel(panel); + return panel; + } + function activatePanel(panel: SidePanel): void { shell.add(panel, 'main'); shell.activateById(panel.id); @@ -122,6 +144,12 @@ function activate( execute: createClustersPanel }); + // The open_yaml_editor command is also used by the below entry points. + commands.addCommand(CommandIDs.open_yaml_editor, { + label: yamlCommandLabel, + execute: createYamlPanel + }); + // Entry point in launcher. if (launcher) { launcher.add({ @@ -132,6 +160,10 @@ function activate( command: CommandIDs.open_clusters_panel, category: category }); + launcher.add({ + command: CommandIDs.open_yaml_editor, + category: category + }); } // Entry point in top menu. @@ -140,10 +172,11 @@ function activate( mainMenu.addMenu(menu); menu.addItem({ command: CommandIDs.open_inspector }); menu.addItem({ command: CommandIDs.open_clusters_panel }); + menu.addItem({ command: CommandIDs.open_yaml_editor }); // Entry point in commands palette. palette.addItem({ command: CommandIDs.open_inspector, category }); palette.addItem({ command: CommandIDs.open_clusters_panel, category }); + palette.addItem({ command: CommandIDs.open_yaml_editor, category }); } - export default extension; diff --git a/sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/src/yaml/CustomStyle.tsx b/sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/src/yaml/CustomStyle.tsx new file mode 100644 index 000000000000..87d93de0b60a --- /dev/null +++ b/sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/src/yaml/CustomStyle.tsx @@ -0,0 +1,179 @@ +// Licensed 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. + +import React, { memo } from 'react'; +import { Handle, Position } from '@xyflow/react'; +import { EdgeProps, BaseEdge, getSmoothStepPath } from '@xyflow/react'; +import { INodeData } from './DataType'; +import { transformEmojiMap } from './EmojiMap'; + +export function DefaultNode({ data }: { data: INodeData }) { + const emoji = data.label + ? transformEmojiMap[data.label] || '📦' + : data.emoji || '📦'; + const typeClass = data.type ? `custom-node-${data.type}` : ''; + + return ( +
    +
    +
    {emoji}
    +
    {data.label}
    +
    + + + +
    + ); +} + +// ===== Input Node ===== +export function InputNode({ data }: { data: INodeData }) { + return ( +
    +
    +
    {data.emoji || '🟢'}
    +
    {data.label}
    +
    + + +
    + ); +} + +// ===== Output Node ===== +export function OutputNode({ data }: { data: INodeData }) { + return ( +
    +
    +
    {data.emoji || '🔴'}
    +
    {data.label}
    +
    + + +
    + ); +} + +export default memo(DefaultNode); + +export function AnimatedSVGEdge({ + id, + sourceX, + sourceY, + targetX, + targetY, + sourcePosition, + targetPosition +}: EdgeProps) { + const [initialEdgePath] = getSmoothStepPath({ + sourceX, + sourceY, + targetX, + targetY, + sourcePosition, + targetPosition + }); + + let edgePath = initialEdgePath; + + // If the edge is almost vertical or horizontal, use a straight line + const dx = Math.abs(targetX - sourceX); + const dy = Math.abs(targetY - sourceY); + if (dx < 1) { + edgePath = `M${sourceX},${sourceY} L${sourceX + 1},${targetY}`; + } else if (dy < 1) { + edgePath = `M${sourceX},${sourceY} L${targetX},${sourceY + 1}`; + } + + const dotCount = 4; + const dotDur = 3.5; + + const dots = Array.from({ length: dotCount }, (_, i) => ( + + + + + )); + + return ( + <> + {/* Gradient Base Edge */} + + + {/* Dots */} + {dots} + + {/* Flow shader line */} + + + + + {/* Gradient Color */} + + + + + + + + + + + + + + ); +} diff --git a/sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/src/yaml/DataType.ts b/sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/src/yaml/DataType.ts new file mode 100644 index 000000000000..0ea535d5fc6a --- /dev/null +++ b/sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/src/yaml/DataType.ts @@ -0,0 +1,37 @@ +// Licensed 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. + +export const nodeWidth = 320; +export const nodeHeight = 100; + +export interface INodeData { + id: string; + label: string; + type?: string; + [key: string]: any; +} + +export interface IEdgeData { + source: string; + target: string; + label?: string; +} + +export interface IFlowGraph { + nodes: INodeData[]; + edges: IEdgeData[]; +} + +export interface IApiResponse { + data: IFlowGraph | null; + error: string | null; +} diff --git a/sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/src/yaml/EditablePanel.tsx b/sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/src/yaml/EditablePanel.tsx new file mode 100644 index 000000000000..d2b19d4371f4 --- /dev/null +++ b/sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/src/yaml/EditablePanel.tsx @@ -0,0 +1,408 @@ +// Licensed 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. + +import React from 'react'; +import { Node } from '@xyflow/react'; +import '../../style/yaml/YamlEditor.css'; +import { transformEmojiMap } from './EmojiMap'; + +type EditableKeyValuePanelProps = { + node: Node; + onChange: (newData: Record) => void; + depth?: number; +}; + +type EditableKeyValuePanelState = { + localData: Record; + collapsedKeys: Set; +}; + +/** + * An editable key-value panel component for displaying + * and modifying node properties. + * + * Features: + * - Nested object support with collapsible sections + * - Real-time key-value editing with validation + * - Dynamic field addition and deletion + * - Support for multi-line text values + * - Object conversion for nested structures + * - Reference documentation integration + * - Visual hierarchy with depth-based indentation + * - Interactive UI with hover effects and transitions + * + * State Management: + * - localData: Local copy of the node data being edited + * - collapsedKeys: Set of keys that are currently collapsed + * + * Props: + * @param {Node} node - The node is data to be edited + * @param {(data: Record) => void} onChange - + * Callback for data changes + * @param {number} [depth=0] - Current nesting depth for recursive rendering + * + * Methods: + * - toggleCollapse: Toggles collapse state of nested objects + * - handleKeyChange: Updates keys with validation + * - handleValueChange: Updates values in the local data + * - handleDelete: Removes key-value pairs + * - handleAddPair: Adds new key-value pairs + * - convertToObject: Converts primitive values to objects + * - renderValueEditor: Renders appropriate input based on value type + * + * UI Features: + * - Collapsible nested object sections + * - Multi-line text support for complex values + * - Add/Delete buttons for field management + * - Reference documentation links + * - Visual feedback for user interactions + * - Responsive design with proper spacing + */ +export class EditableKeyValuePanel extends React.Component< + EditableKeyValuePanelProps, + EditableKeyValuePanelState +> { + static defaultProps = { + depth: 0 + }; + + constructor(props: EditableKeyValuePanelProps) { + super(props); + this.state = { + localData: { ...(props.node ? props.node.data : {}) }, + collapsedKeys: new Set() + }; + } + + componentDidUpdate(prevProps: EditableKeyValuePanelProps) { + if (prevProps.node !== this.props.node && this.props.node) { + this.setState({ localData: { ...(this.props.node.data ?? {}) } }); + } + } + + toggleCollapse = (key: string) => { + this.setState(({ collapsedKeys }) => { + const newSet = new Set(collapsedKeys); + newSet.has(key) ? newSet.delete(key) : newSet.add(key); + return { collapsedKeys: newSet }; + }); + }; + + handleKeyChange = (oldKey: string, newKey: string) => { + newKey = newKey.trim(); + if (newKey === oldKey || newKey === '') { + return alert('Invalid Key!'); + } + if (newKey in this.state.localData) { + return alert('Duplicated Key!'); + } + + const newData: Record = {}; + for (const [k, v] of Object.entries(this.state.localData)) { + newData[k === oldKey ? newKey : k] = v; + } + + this.setState({ localData: newData }, () => this.props.onChange(newData)); + }; + + handleValueChange = (key: string, newValue: any) => { + const newData = { ...this.state.localData, [key]: newValue }; + this.setState({ localData: newData }, () => this.props.onChange(newData)); + }; + + handleDelete = (key: string) => { + const { [key]: _, ...rest } = this.state.localData; + void _; + this.setState({ localData: rest }, () => this.props.onChange(rest)); + }; + + handleAddPair = () => { + let i = 1; + const baseKey = 'newKey'; + while (`${baseKey}${i}` in this.state.localData) { + i++; + } + const newKey = `${baseKey}${i}`; + const newData = { ...this.state.localData, [newKey]: '' }; + this.setState({ localData: newData }, () => this.props.onChange(newData)); + }; + + convertToObject = (key: string) => { + if ( + typeof this.state.localData[key] === 'object' && + this.state.localData[key] !== null + ) { + return; + } + const newData = { ...this.state.localData, [key]: {} }; + this.setState({ localData: newData }, () => this.props.onChange(newData)); + this.setState(({ collapsedKeys }) => { + const newSet = new Set(collapsedKeys); + newSet.delete(key); + return { collapsedKeys: newSet }; + }); + }; + + renderValueEditor = (key: string, value: any) => { + const isMultiline = + key === 'callable' || (typeof value === 'string' && value.includes('\n')); + + return isMultiline ? ( +